From 53edce7df1534e4a75f2a81f7a676648840964bd Mon Sep 17 00:00:00 2001 From: zhangfengcdt Date: Tue, 27 Aug 2024 09:55:36 -0700 Subject: [PATCH 1/6] [SEDONA-648] Implement Distributed K Nearest Neighbor Join --- .../sedona/core/enums/DistanceMetric.java | 40 + .../apache/sedona/core/enums/GridType.java | 8 +- .../joinJudgement/KnnJoinIndexJudgement.java | 187 +++++ .../knnJudgement/EuclideanItemDistance.java | 39 + .../knnJudgement/HaversineItemDistance.java | 40 + .../core/knnJudgement/SpheroidDistance.java | 40 + .../core/spatialOperator/JoinQuery.java | 122 +++ .../BroadcastedSpatialPartitioner.java | 74 ++ .../QuadTreePartitioner.java | 4 +- .../QuadTreeRTPartitioner.java | 114 +++ .../QuadTreeRTPartitioning.java | 521 +++++++++++++ .../QuadtreePartitioning.java | 2 +- .../quadtree/ExtendedQuadTree.java | 244 ++++++ .../sedona/core/spatialRDD/SpatialRDD.java | 33 + .../apache/sedona/core/utils/SedonaConf.java | 46 ++ .../join/BroadcastObjectSideKNNJoinExec.scala | 156 ++++ .../join/BroadcastQuerySideKNNJoinExec.scala | 176 +++++ .../strategy/join/KNNJoinExec.scala | 217 ++++++ .../strategy/join/TraitKNNJoinQueryExec.scala | 313 ++++++++ .../org/apache/sedona/sql/KnnJoinSuite.scala | 733 ++++++++++++++++++ 20 files changed, 3105 insertions(+), 4 deletions(-) create mode 100644 spark/common/src/main/java/org/apache/sedona/core/enums/DistanceMetric.java create mode 100644 spark/common/src/main/java/org/apache/sedona/core/joinJudgement/KnnJoinIndexJudgement.java create mode 100644 spark/common/src/main/java/org/apache/sedona/core/knnJudgement/EuclideanItemDistance.java create mode 100644 spark/common/src/main/java/org/apache/sedona/core/knnJudgement/HaversineItemDistance.java create mode 100644 spark/common/src/main/java/org/apache/sedona/core/knnJudgement/SpheroidDistance.java create mode 100644 spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/BroadcastedSpatialPartitioner.java create mode 100644 spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioner.java create mode 100644 spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioning.java create mode 100644 spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/quadtree/ExtendedQuadTree.java create mode 100644 spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastObjectSideKNNJoinExec.scala create mode 100644 spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastQuerySideKNNJoinExec.scala create mode 100644 spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/KNNJoinExec.scala create mode 100644 spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitKNNJoinQueryExec.scala create mode 100644 spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala diff --git a/spark/common/src/main/java/org/apache/sedona/core/enums/DistanceMetric.java b/spark/common/src/main/java/org/apache/sedona/core/enums/DistanceMetric.java new file mode 100644 index 0000000000..1a5f0496a7 --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/enums/DistanceMetric.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.enums; + +/** + * The DistanceMetric enum represents the different distance metrics that can be used in the + * application. + */ +public enum DistanceMetric { + /** The Euclidean distance metric, also known as straight line distance. */ + EUCLIDEAN, + + /** + * The Haversine distance metric, which measures the shortest distance between two points on the + * surface of a sphere. + */ + HAVERSINE, + + /** + * The Spheroid distance metric, which measures the shortest distance between two points on the + * surface of a spheroid. + */ + SPHEROID +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/enums/GridType.java b/spark/common/src/main/java/org/apache/sedona/core/enums/GridType.java index 3f1b2e7bd3..c25a607ee4 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/enums/GridType.java +++ b/spark/common/src/main/java/org/apache/sedona/core/enums/GridType.java @@ -32,7 +32,13 @@ public enum GridType implements Serializable { QUADTREE, /** K-D-B-tree partitioning (k-dimensional B-tree) */ - KDBTREE; + KDBTREE, + + /** Z-ORDER based partitioning (morton space-filling curve) for KNN joins */ + ZORDER, + + /** Modified Quad-tree partitioning for KNN joins */ + QUADTREE_RTREE; /** * Gets the grid type. diff --git a/spark/common/src/main/java/org/apache/sedona/core/joinJudgement/KnnJoinIndexJudgement.java b/spark/common/src/main/java/org/apache/sedona/core/joinJudgement/KnnJoinIndexJudgement.java new file mode 100644 index 0000000000..1c7fe7a0ae --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/joinJudgement/KnnJoinIndexJudgement.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.joinJudgement; + +import java.io.Serializable; +import java.util.*; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.sedona.core.enums.DistanceMetric; +import org.apache.sedona.core.knnJudgement.EuclideanItemDistance; +import org.apache.sedona.core.knnJudgement.HaversineItemDistance; +import org.apache.sedona.core.knnJudgement.SpheroidDistance; +import org.apache.spark.api.java.function.FlatMapFunction2; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.util.LongAccumulator; +import org.locationtech.jts.geom.Envelope; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.index.SpatialIndex; +import org.locationtech.jts.index.strtree.GeometryItemDistance; +import org.locationtech.jts.index.strtree.ItemDistance; +import org.locationtech.jts.index.strtree.STRtree; + +/** + * This class is responsible for performing a K-nearest neighbors (KNN) join operation using a + * spatial index. It extends the JudgementBase class and implements the FlatMapFunction2 interface. + * + * @param extends Geometry - the type of geometries in the left set + * @param extends Geometry - the type of geometries in the right set + */ +public class KnnJoinIndexJudgement + extends JudgementBase + implements FlatMapFunction2, Iterator, Pair>, Serializable { + private final int k; + private final DistanceMetric distanceMetric; + private final boolean includeTies; + private final Broadcast broadcastedTreeIndex; + + /** + * Constructor for the KnnJoinIndexJudgement class. + * + * @param k the number of nearest neighbors to find + * @param distanceMetric the distance metric to use + * @param buildCount accumulator for the number of geometries processed from the build side + * @param streamCount accumulator for the number of geometries processed from the stream side + * @param resultCount accumulator for the number of join results + * @param candidateCount accumulator for the number of candidate matches + * @param broadcastedTreeIndex the broadcasted spatial index + */ + public KnnJoinIndexJudgement( + int k, + DistanceMetric distanceMetric, + boolean includeTies, + Broadcast broadcastedTreeIndex, + LongAccumulator buildCount, + LongAccumulator streamCount, + LongAccumulator resultCount, + LongAccumulator candidateCount) { + super(null, buildCount, streamCount, resultCount, candidateCount); + this.k = k; + this.distanceMetric = distanceMetric; + this.includeTies = includeTies; + this.broadcastedTreeIndex = broadcastedTreeIndex; + } + + /** + * This method performs the KNN join operation. It iterates over the geometries in the stream side + * and uses the spatial index to find the k nearest neighbors for each geometry. The method + * returns an iterator over the join results. + * + * @param streamShapes iterator over the geometries in the stream side + * @param treeIndexes iterator over the spatial indexes + * @return an iterator over the join results + * @throws Exception if the spatial index is not of type STRtree + */ + @Override + public Iterator> call(Iterator streamShapes, Iterator treeIndexes) + throws Exception { + if (!treeIndexes.hasNext() || !streamShapes.hasNext()) { + buildCount.add(0); + streamCount.add(0); + resultCount.add(0); + candidateCount.add(0); + return Collections.emptyIterator(); + } + + STRtree strTree; + if (broadcastedTreeIndex != null) { + // get the broadcasted spatial index if available + // this is to support the broadcast join + strTree = broadcastedTreeIndex.getValue(); + } else { + // get the spatial index from the iterator + SpatialIndex treeIndex = treeIndexes.next(); + if (!(treeIndex instanceof STRtree)) { + throw new Exception( + "[KnnJoinIndexJudgement][Call] Only STRtree index supports KNN search."); + } + strTree = (STRtree) treeIndex; + } + + List> result = new ArrayList<>(); + ItemDistance itemDistance; + + while (streamShapes.hasNext()) { + T streamShape = streamShapes.next(); + streamCount.add(1); + + Object[] localK; + switch (distanceMetric) { + case EUCLIDEAN: + itemDistance = new EuclideanItemDistance(); + break; + case HAVERSINE: + itemDistance = new HaversineItemDistance(); + break; + case SPHEROID: + itemDistance = new SpheroidDistance(); + break; + default: + itemDistance = new GeometryItemDistance(); + break; + } + + localK = + strTree.nearestNeighbour(streamShape.getEnvelopeInternal(), streamShape, itemDistance, k); + if (includeTies) { + localK = getUpdatedLocalKWithTies(streamShape, localK, strTree); + } + + for (Object obj : localK) { + T candidate = (T) obj; + Pair pair = Pair.of((U) streamShape, candidate); + result.add(pair); + resultCount.add(1); + } + } + + return result.iterator(); + } + + private Object[] getUpdatedLocalKWithTies(T streamShape, Object[] localK, STRtree strTree) { + Envelope searchEnvelope = streamShape.getEnvelopeInternal(); + // get the maximum distance from the k nearest neighbors + double maxDistance = 0.0; + LinkedHashSet uniqueCandidates = new LinkedHashSet<>(); + for (Object obj : localK) { + T candidate = (T) obj; + uniqueCandidates.add(candidate); + double distance = streamShape.distance(candidate); + if (distance > maxDistance) { + maxDistance = distance; + } + } + searchEnvelope.expandBy(maxDistance); + List candidates = strTree.query(searchEnvelope); + if (!candidates.isEmpty()) { + // update localK with all candidates that are within the maxDistance + List tiedResults = new ArrayList<>(); + // add all localK + Collections.addAll(tiedResults, localK); + + for (T candidate : candidates) { + double distance = streamShape.distance(candidate); + if (distance == maxDistance && !uniqueCandidates.contains(candidate)) { + tiedResults.add(candidate); + } + } + localK = tiedResults.toArray(); + } + return localK; + } +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/EuclideanItemDistance.java b/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/EuclideanItemDistance.java new file mode 100644 index 0000000000..a27bf543b1 --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/EuclideanItemDistance.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.knnJudgement; + +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.index.strtree.ItemBoundable; +import org.locationtech.jts.index.strtree.ItemDistance; + +public class EuclideanItemDistance implements ItemDistance { + + public EuclideanItemDistance() {} + + @Override + public double distance(ItemBoundable item1, ItemBoundable item2) { + if (item1 == item2) { + return Double.MAX_VALUE; + } else { + Geometry g1 = (Geometry) item1.getItem(); + Geometry g2 = (Geometry) item2.getItem(); + return g1.distance(g2); + } + } +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/HaversineItemDistance.java b/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/HaversineItemDistance.java new file mode 100644 index 0000000000..9ad1bfbee4 --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/HaversineItemDistance.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.knnJudgement; + +import org.apache.sedona.common.sphere.Haversine; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.index.strtree.ItemBoundable; +import org.locationtech.jts.index.strtree.ItemDistance; + +public class HaversineItemDistance implements ItemDistance { + + public HaversineItemDistance() {} + + @Override + public double distance(ItemBoundable item1, ItemBoundable item2) { + if (item1 == item2) { + return Double.MAX_VALUE; + } else { + Geometry g1 = (Geometry) item1.getItem(); + Geometry g2 = (Geometry) item2.getItem(); + return Haversine.distance(g1, g2); + } + } +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/SpheroidDistance.java b/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/SpheroidDistance.java new file mode 100644 index 0000000000..df22d3565e --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/knnJudgement/SpheroidDistance.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.knnJudgement; + +import org.apache.sedona.common.sphere.Spheroid; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.index.strtree.ItemBoundable; +import org.locationtech.jts.index.strtree.ItemDistance; + +public class SpheroidDistance implements ItemDistance { + + public SpheroidDistance() {} + + @Override + public double distance(ItemBoundable item1, ItemBoundable item2) { + if (item1 == item2) { + return Double.MAX_VALUE; + } else { + Geometry g1 = (Geometry) item1.getItem(); + Geometry g2 = (Geometry) item2.getItem(); + return Spheroid.distance(g1, g2); + } + } +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java b/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java index 7af522b0be..60b698886c 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java @@ -27,6 +27,7 @@ import org.apache.log4j.Logger; import org.apache.sedona.common.geometryObjects.Circle; import org.apache.sedona.common.utils.GeomUtils; +import org.apache.sedona.core.enums.DistanceMetric; import org.apache.sedona.core.enums.IndexType; import org.apache.sedona.core.enums.JoinBuildSide; import org.apache.sedona.core.joinJudgement.*; @@ -40,8 +41,10 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.broadcast.Broadcast; import org.apache.spark.util.LongAccumulator; import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.index.strtree.STRtree; import scala.Tuple2; public class JoinQuery { @@ -697,21 +700,140 @@ public static JavaPairRDD spatial (PairFunction, U, T>) pair -> new Tuple2<>(pair.getKey(), pair.getValue())); } + private static void verifyPartitioningNumberMatch( + SpatialRDD spatialRDD, SpatialRDD queryRDD) throws Exception { + Objects.requireNonNull( + spatialRDD.spatialPartitionedRDD, + "[JoinQuery] spatialRDD SpatialPartitionedRDD is null. Please do spatial partitioning."); + Objects.requireNonNull( + queryRDD.spatialPartitionedRDD, + "[JoinQuery] queryRDD SpatialPartitionedRDD is null. Please use the spatialRDD's grids to do spatial partitioning."); + + final SpatialPartitioner spatialPartitioner = spatialRDD.getPartitioner(); + final SpatialPartitioner queryPartitioner = queryRDD.getPartitioner(); + + final int spatialNumPart = spatialRDD.spatialPartitionedRDD.getNumPartitions(); + final int queryNumPart = queryRDD.spatialPartitionedRDD.getNumPartitions(); + if (spatialNumPart != queryNumPart) { + throw new IllegalArgumentException( + "[JoinQuery] numbers of partitions in queryRDD and spatialRDD don't match: " + + queryNumPart + + " vs. " + + spatialNumPart + + ". Please make sure they both use the same partitioning otherwise wrong results will appear."); + } + } + + /** + * @param queryRDD {@code queryRDD} is the set of geometries which serve as query geometries + * (center points) + * @param objectRDD {@code objectRDD} is the set of geometries (neighbors) to be queried + * @param joinParams {@code joinParams} is the parameters for the join + * @param includeTies {@code includeTies} is a boolean indicating whether to include ties + * @return RDD of pairs where each pair contains a geometry and a set of matching geometries + * @param Type of the geometries in queryRDD set + * @param Type of the geometries in objectRDD set + * @throws Exception + */ + public static JavaPairRDD knnJoin( + SpatialRDD queryRDD, + SpatialRDD objectRDD, + JoinParams joinParams, + boolean includeTies, + boolean broadcastJoin) + throws Exception { + verifyCRSMatch(queryRDD, objectRDD); + if (!broadcastJoin) verifyPartitioningNumberMatch(queryRDD, objectRDD); + + SparkContext sparkContext = queryRDD.rawSpatialRDD.context(); + LongAccumulator buildCount = Metrics.createMetric(sparkContext, "buildCount"); + LongAccumulator streamCount = Metrics.createMetric(sparkContext, "streamCount"); + LongAccumulator resultCount = Metrics.createMetric(sparkContext, "resultCount"); + LongAccumulator candidateCount = Metrics.createMetric(sparkContext, "candidateCount"); + + final Broadcast broadcastedTreeIndex; + if (broadcastJoin) { + // adjust auto broadcast threshold to avoid building index on large RDDs + STRtree strTree = objectRDD.coalesceAndBuildRawIndex(IndexType.RTREE); + broadcastedTreeIndex = JavaSparkContext.fromSparkContext(sparkContext).broadcast(strTree); + } else { + broadcastedTreeIndex = null; + } + + // The reason for using objectRDD as the right side is that the partitions are built on the + // right side. + final JavaRDD> joinResult; + if (objectRDD.indexedRDD != null) { + final KnnJoinIndexJudgement judgement = + new KnnJoinIndexJudgement( + joinParams.k, + joinParams.distanceMetric, + includeTies, + broadcastedTreeIndex, + buildCount, + streamCount, + resultCount, + candidateCount); + joinResult = queryRDD.spatialPartitionedRDD.zipPartitions(objectRDD.indexedRDD, judgement); + } else if (broadcastedTreeIndex != null) { + final KnnJoinIndexJudgement judgement = + new KnnJoinIndexJudgement( + joinParams.k, + joinParams.distanceMetric, + includeTies, + broadcastedTreeIndex, + buildCount, + streamCount, + resultCount, + candidateCount); + int numPartitionsObjects = objectRDD.rawSpatialRDD.getNumPartitions(); + joinResult = + queryRDD + .rawSpatialRDD + .repartition(numPartitionsObjects) + .zipPartitions(objectRDD.rawSpatialRDD, judgement); + } else { + throw new IllegalArgumentException("No index found on the input RDDs."); + } + + return joinResult.mapToPair( + (PairFunction, U, T>) pair -> new Tuple2<>(pair.getKey(), pair.getValue())); + } + public static final class JoinParams { public final boolean useIndex; public final SpatialPredicate spatialPredicate; public final IndexType indexType; public final JoinBuildSide joinBuildSide; + // KNN specific parameters + public final int k; + public final DistanceMetric distanceMetric; + public final Double searchRadius; + public JoinParams( boolean useIndex, SpatialPredicate spatialPredicate, IndexType polygonIndexType, JoinBuildSide joinBuildSide) { + this(useIndex, spatialPredicate, polygonIndexType, joinBuildSide, -1, null, null); + } + + public JoinParams( + boolean useIndex, + SpatialPredicate spatialPredicate, + IndexType polygonIndexType, + JoinBuildSide joinBuildSide, + int k, + DistanceMetric distanceMetric, + Double searchRadius) { this.useIndex = useIndex; this.spatialPredicate = spatialPredicate; this.indexType = polygonIndexType; this.joinBuildSide = joinBuildSide; + this.k = k; + this.distanceMetric = distanceMetric; + this.searchRadius = searchRadius; } public JoinParams(boolean useIndex, SpatialPredicate spatialPredicate) { diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/BroadcastedSpatialPartitioner.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/BroadcastedSpatialPartitioner.java new file mode 100644 index 0000000000..e197baae03 --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/BroadcastedSpatialPartitioner.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.spatialPartitioning; + +import java.util.Iterator; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.sedona.core.joinJudgement.DedupParams; +import org.apache.spark.broadcast.Broadcast; +import org.locationtech.jts.geom.Envelope; +import org.locationtech.jts.geom.Geometry; +import scala.Tuple2; + +/** + * The SpatialPartitioner may contain a large number of grids, which may make the serialized tasks + * to be larger than 1MB and trigger a warning: "WARN DAGScheduler: Broadcasting large task binary + * with size XXXX KB". This class is a wrapper around a SpatialPartitioner that is broadcasted to + * reduce the size of serialized tasks. + */ +public class BroadcastedSpatialPartitioner extends SpatialPartitioner { + private final Broadcast bPartitioner; + private transient SpatialPartitioner partitioner; + + public BroadcastedSpatialPartitioner(Broadcast partitioner) { + super(partitioner.value().gridType, null); + this.bPartitioner = partitioner; + this.partitioner = null; + } + + private SpatialPartitioner getPartitioner() { + if (partitioner == null) { + partitioner = bPartitioner.value(); + } + return partitioner; + } + + @Override + public Iterator> placeObject(T spatialObject) + throws Exception { + return getPartitioner().placeObject(spatialObject); + } + + @Nullable + @Override + public DedupParams getDedupParams() { + return getPartitioner().getDedupParams(); + } + + @Override + public List getGrids() { + return getPartitioner().getGrids(); + } + + @Override + public int numPartitions() { + return getPartitioner().numPartitions(); + } +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreePartitioner.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreePartitioner.java index 91ee5e512c..44f3646999 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreePartitioner.java +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreePartitioner.java @@ -27,9 +27,9 @@ import scala.Tuple2; public class QuadTreePartitioner extends SpatialPartitioner { - private final StandardQuadTree quadTree; + private final StandardQuadTree quadTree; - public QuadTreePartitioner(StandardQuadTree quadTree) { + public QuadTreePartitioner(StandardQuadTree quadTree) { super(GridType.QUADTREE, quadTree.fetchLeafZones()); this.quadTree = quadTree; diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioner.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioner.java new file mode 100644 index 0000000000..64ad97c038 --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioner.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.spatialPartitioning; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.sedona.core.joinJudgement.DedupParams; +import org.apache.sedona.core.spatialPartitioning.quadtree.ExtendedQuadTree; +import org.locationtech.jts.geom.Envelope; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.index.strtree.STRtree; +import scala.Tuple2; + +/** + * This class implements spatial partitioner based on the principles outlined in: + * + *

It uses the Quad-tree partitioning strategy to create balanced partitions This partitioning is + * essential for efficiently executing kNN joins. The process involves the following steps: + * + *

1. **Quad-tree Partitioning**: - Data is partitioned into partitions using the Quad-tree data + * structure. - This ensures balanced partitions and spatial locality preservation. + * + *

2. **Global R-Tree Index Construction**: - A set of random samples S' from the dataset S is + * taken. - An R-tree T is built over S' in the master node (driver program). + * + *

3. **Distance Bound Calculation**: - For each partition Ri, the distance ui from the furthest + * point in Ri to the centroid cri is calculated. - k-nearest neighbors of each centroid cri are + * found using the R-tree T. - A distance bound γi is derived for each Ri, defined as γi = 2ui + + * |cri, sk|, where sk is the k-th nearest neighbor of cri. + * + *

4. **Partitioning Neighbors**: - For each partition Ri, a subset Si ⊂ S is identified such + * that for any r ∈ Ri, knn(r, S) = knn(r, Si) using a circle range query centered at cri with + * radius γi. - This guarantees that the k-nearest neighbors of any point in Ri can be found within + * the subset Si. + * + *

5. **Parallel Local kNN Joins**: - Each combined partition (Ri, Si) is processed in parallel. + * - An R-tree is built over Si, and a local kNN join is performed for each record in Ri. - The + * results from all partitions are combined to produce the final kNN join results. + * + *

Reference: Xie, Dong, et al. "Simba: Efficient in-memory spatial analytics." In Proceedings of + * the 2016 ACM SIGMOD International Conference on Management of Data (SIGMOD '16), 2016, DOI: + * 10.1145/2882903.2915237. + */ +public class QuadTreeRTPartitioner extends QuadTreePartitioner { + + // ExtendedQuadTree is used to generate the expanded boundaries. + ExtendedQuadTree extendedQuadTree; + + public QuadTreeRTPartitioner(ExtendedQuadTree extendedQuadTree) { + super(extendedQuadTree.getQuadTree()); + this.extendedQuadTree = extendedQuadTree; + } + + public QuadTreeRTPartitioner nonOverlappedPartitioner() { + ExtendedQuadTree nonOverlappedTree = new ExtendedQuadTree<>(extendedQuadTree, true); + return new QuadTreeRTPartitioner(nonOverlappedTree); + } + + public Envelope getBoundary() { + return extendedQuadTree.getBoundary(); + } + + /** + * Depending on overlappedPartitioner, return the expanded boundaries or the original boundaries. + * + * @param spatialObject the spatial object + * @return Iterator> + * @throws Exception + */ + @Override + public Iterator> placeObject(Geometry spatialObject) throws Exception { + return extendedQuadTree.placeObject(spatialObject); + } + + @Override + public DedupParams getDedupParams() { + return super.getDedupParams(); + } + + @Override + public List getGrids() { + return super.getGrids(); + } + + @Override + public int numPartitions() { + return super.numPartitions(); + } + + public Map> getOverlappedGrids() { + return extendedQuadTree.getExpandedBoundaries(); + } + + public STRtree getSTRForOverlappedGrids() { + return extendedQuadTree.getSpatialExpandedBoundaryIndex(); + } +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioning.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioning.java new file mode 100644 index 0000000000..37cfea573c --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioning.java @@ -0,0 +1,521 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.spatialPartitioning; + +import static org.apache.sedona.core.formatMapper.shapefileParser.ShapefileRDD.geometryFactory; + +import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import org.apache.log4j.Logger; +import org.apache.sedona.core.knnJudgement.EuclideanItemDistance; +import org.apache.sedona.core.spatialPartitioning.quadtree.QuadRectangle; +import org.apache.sedona.core.utils.SedonaConf; +import org.locationtech.jts.geom.*; +import org.locationtech.jts.index.strtree.STRtree; + +/** + * The class is used to build an R-tree over a random sample of another dataset and uses distance + * bounds to ensure efficient local kNN joins. + * + *

By calculating distance bounds and using circle range queries, it ensures that the subsets Si, + * containing all necessary points for accurate kNN results. The final union of local join results + * provides the complete kNN join result for the datasets R and S. + * + *

It generates List> expandedParitionedBoundaries based on the quad tree. + */ +public class QuadTreeRTPartitioning extends QuadtreePartitioning { + static final Logger log = Logger.getLogger(QuadTreeRTPartitioning.class); + + private SedonaConf sedonaConf; + + private double skewnessCutoffRatio = 1.0; + private double skewnessMinimumMBRCount = 100; + private int skewnessMaximumMBRDivides = 100; + private boolean enableParallelPartitioning = true; + + // A query-only R-tree created using the Sort-Tile-Recursive (STR) algorithm. + private STRtree strTree; + // The expanded partitioned boundaries based on the quad tree + private HashMap> mbrs; + // The spatial index for partitioned MBRs + private STRtree mbrSpatialIndex; + + public QuadTreeRTPartitioning(List samples, Envelope boundary, int partitions) + throws Exception { + super(samples, boundary, partitions); + } + + public QuadTreeRTPartitioning( + List samples, Envelope boundary, int partitions, int minTreeLevel) + throws Exception { + super(samples, boundary, partitions, minTreeLevel); + } + + public HashMap> getMbrs() { + return mbrs; + } + + public STRtree getMbrSpatialIndex() { + return mbrSpatialIndex; + } + + /** + * This function is used to build the STR tree from the quad-tree built from the samples. It is + * used to expand the partitioned boundaries. + * + * @param samples the samples + * @param k the number of neighbor samples + * @return + */ + public STRtree buildSTRTree(List samples, int k) { + // The partitioned MBRs + mbrs = new HashMap<>(); + + // A query-only R-tree created using the Sort-Tile-Recursive (STR) algorithm. + strTree = new STRtree(); + + // Get all MBRs (partitions) from the quad-tree + // The zones might include the one with null partition ids + List partitionMBRs = + partitionTree.getAllZones().stream() + .filter(quadRect -> quadRect.partitionId != null) + .collect(Collectors.toList()); + + for (QuadRectangle quadRect : partitionMBRs) { + Envelope mbr = quadRect.getEnvelope(); + strTree.insert(mbr, mbr); + } + + // Insert samples into an STR tree for k-nearest neighbor search + STRtree sampleTree = new STRtree(); + for (Envelope sample : samples) { + // convert sample to a point + Point point = + geometryFactory.createPoint( + new Coordinate(sample.centre().getX(), sample.centre().getY())); + sampleTree.insert(sample, point); + } + + double minimalGridWidth = getMinimalEnvelopeWidth(partitionMBRs); + + if (isEnableParallelPartitioning()) { + processPartitions( + partitionMBRs, + mbrs, + k, + sampleTree, + geometryFactory, + minimalGridWidth, + skewnessMaximumMBRDivides, + true); + } else { + processPartitions( + partitionMBRs, + mbrs, + k, + sampleTree, + geometryFactory, + minimalGridWidth, + skewnessMaximumMBRDivides, + false); + } + + // Construct a spatial index for the MBRs + this.mbrSpatialIndex = new STRtree(); + for (Integer id : mbrs.keySet()) { + for (Envelope envelope : mbrs.get(id)) { + mbrSpatialIndex.insert(envelope, id); + } + } + + // Return the STR tree + return strTree; + } + + public void processPartitions( + List partitionMBRs, + Map> mbrs, + int k, + STRtree sampleTree, + GeometryFactory geometryFactory, + double minimalGridWidth, + int skewnessMaximumMBRDivides, + boolean parallel) { + + if (parallel) { + processPartitionsInParallel( + partitionMBRs, + mbrs, + k, + sampleTree, + geometryFactory, + minimalGridWidth, + skewnessMaximumMBRDivides); + } else { + processPartitionsSequentially( + partitionMBRs, + mbrs, + k, + sampleTree, + geometryFactory, + minimalGridWidth, + skewnessMaximumMBRDivides); + } + } + + private void processPartitionsInParallel( + List partitionMBRs, + Map> mbrs, + int k, + STRtree sampleTree, + GeometryFactory geometryFactory, + double minimalGridWidth, + int skewnessMaximumMBRDivides) { + + ExecutorService executor = + Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); + + List> futures = new ArrayList<>(); + + for (QuadRectangle quadRect : partitionMBRs) { + futures.add( + executor.submit( + () -> { + processPartition( + partitionMBRs, + quadRect, + mbrs, + k, + sampleTree, + geometryFactory, + minimalGridWidth, + skewnessMaximumMBRDivides); + return null; + })); + } + + // Wait for all tasks to complete + for (Future future : futures) { + try { + future.get(); + } catch (Exception e) { + e.printStackTrace(); + } + } + + executor.shutdown(); + } + + private void processPartitionsSequentially( + List partitionMBRs, + Map> mbrs, + int k, + STRtree sampleTree, + GeometryFactory geometryFactory, + double minimalGridWidth, + int skewnessMaximumMBRDivides) { + for (QuadRectangle quadRect : partitionMBRs) { + processPartition( + partitionMBRs, + quadRect, + mbrs, + k, + sampleTree, + geometryFactory, + minimalGridWidth, + skewnessMaximumMBRDivides); + } + } + + private void processPartition( + List partitionMBRs, + QuadRectangle quadRect, + Map> mbrs, + int k, + STRtree sampleTree, + GeometryFactory geometryFactory, + double minimalGridWidth, + int skewnessMaximumMBRDivides) { + + Envelope partitionMBR = quadRect.getEnvelope(); + + // Calculate the centroid of each MBR in the STR tree + double centroidX = (partitionMBR.getMinX() + partitionMBR.getMaxX()) / 2.0; + double centroidY = (partitionMBR.getMinY() + partitionMBR.getMaxY()) / 2.0; + Coordinate centroidCoord = new Coordinate(centroidX, centroidY); + Point centroid = geometryFactory.createPoint(centroidCoord); + + // Compute the maximum distance ui from the centroid to any point inside the partition + double ui = getUi(centroid, partitionMBR); + + // Calculate the maximum distance from the centroid to the k-nearest neighbors in the samples + double maxDistance = getMaxDistanceFromSamples(k, sampleTree, centroid); + List intersectingMBRs = + getMBRIntersectEnvelopes(ui, maxDistance, centroidX, centroidY); + + // Calculate the MBRs (Minimum Bounding Rectangles) that intersect with the circle. + if (isSkewed(intersectingMBRs, partitionMBRs)) { + int divide = (int) Math.ceil(quadRect.width / minimalGridWidth); + if (skewnessMaximumMBRDivides > 0 && divide > skewnessMaximumMBRDivides) { + log.debug( + "Found skewed partition, and the number of divides is too large: " + + divide + + " for partition: " + + quadRect.partitionId + + " with width: " + + quadRect.width + + " and minimalGridWidth: " + + minimalGridWidth + + ". Using the maximum number of divides: " + + skewnessMaximumMBRDivides); + divide = skewnessMaximumMBRDivides; + } + intersectingMBRs = getEnvelopesForSubDividedGrids(k, partitionMBR, sampleTree, divide); + } + + synchronized (mbrs) { + mbrs.put(quadRect.partitionId, intersectingMBRs); + } + } + + /** + * This function is used to calculate the minimal envelope width of the partitioned MBRs. + * + * @param partitionMBRs + * @return + */ + public double getMinimalEnvelopeWidth(List partitionMBRs) { + double minEnvelopeWidth = Double.MAX_VALUE; + + for (QuadRectangle quadRect : partitionMBRs) { + Envelope partitionMBR = quadRect.getEnvelope(); + + // Calculate the width and height of the envelope + double width = partitionMBR.getMaxX() - partitionMBR.getMinX(); + + // Update the minimal envelope length if the current one is smaller + if (width < minEnvelopeWidth) { + minEnvelopeWidth = width; + } + } + + return minEnvelopeWidth; + } + + /** + * This function is used to check if the partitioned MBRs are from a skewed partitioning strategy. + * It simply checks preset ratios and minimum MBR count, but it can be extended to include more + * sophisticated skewness detection algorithms. + * + * @param intersectingMBRs + * @param partitionMBRs + * @return + */ + private boolean isSkewed(List intersectingMBRs, List partitionMBRs) { + try { + tryLoadConfig(); + return intersectingMBRs.size() > partitionMBRs.size() * skewnessCutoffRatio + && partitionMBRs.size() > skewnessMinimumMBRCount; + } catch (Exception e) { + return false; + } + } + + private boolean isEnableParallelPartitioning() { + try { + tryLoadConfig(); + return enableParallelPartitioning; + } catch (Exception e) { + return true; + } + } + + /** This function is used to load the Sedona configuration. */ + private void tryLoadConfig() { + if (sedonaConf == null) { + sedonaConf = SedonaConf.fromActiveSession(); + skewnessCutoffRatio = sedonaConf.getSkewnessCutoffRatioInKNNJoins(); + skewnessMinimumMBRCount = sedonaConf.getSkewnessMinimumMBRCountInKNNJoins(); + skewnessMaximumMBRDivides = sedonaConf.getSkewnessMaximumMBRDividesInKNNJoins(); + enableParallelPartitioning = sedonaConf.isEnableParallelPartitioningInKNNJoins(); + } + } + + /** + * This function is used to calculate the maximum distance from the centroid to the k-nearest + * neighbors in the samples. It is used to expand the partitioned boundaries. + * + * @param centroid + * @param partitionMBR + * @return + */ + private static double getUi(Point centroid, Envelope partitionMBR) { + double ui = + Math.max( + centroid.distance( + geometryFactory.createPoint( + new Coordinate(partitionMBR.getMinX(), partitionMBR.getMinY()))), + Math.max( + centroid.distance( + geometryFactory.createPoint( + new Coordinate(partitionMBR.getMinX(), partitionMBR.getMaxY()))), + Math.max( + centroid.distance( + geometryFactory.createPoint( + new Coordinate(partitionMBR.getMaxX(), partitionMBR.getMinY()))), + centroid.distance( + geometryFactory.createPoint( + new Coordinate(partitionMBR.getMaxX(), partitionMBR.getMaxY())))))); + return ui; + } + + /** + * This function is used to get the MBRs that intersect with the circle constructed around the + * centroid. It is used to expand the partitioned boundaries. If the number of intersecting MBRs + * is too large, we optimize by considering all vertices of the MBRs to construct the circle. + * + * @param k + * @param partitionMBR + * @param sampleTree + * @param divide + * @return + */ + private List getEnvelopesForSubDividedGrids( + int k, Envelope partitionMBR, STRtree sampleTree, int divide) { + Set optimizedIntersectingMBRs = new HashSet<>(); + double minX = partitionMBR.getMinX(); + double minY = partitionMBR.getMinY(); + double maxX = partitionMBR.getMaxX(); + double maxY = partitionMBR.getMaxY(); + double xStep = (maxX - minX) / divide; + double yStep = (maxY - minY) / divide; + + // Process points on the edges of the grid + for (int i = 0; i <= divide; i++) { + double x = minX + i * xStep; + + // Top edge (minY) + Point pointTop = geometryFactory.createPoint(new Coordinate(x, minY)); + double maxKNNDistanceTop = getMaxDistanceFromSamples(k, sampleTree, pointTop); + optimizedIntersectingMBRs.addAll( + getMBRIntersectEnvelopes(0.0, maxKNNDistanceTop, pointTop.getX(), pointTop.getY())); + + // Bottom edge (maxY) + Point pointBottom = geometryFactory.createPoint(new Coordinate(x, maxY)); + double maxKNNDistanceBottom = getMaxDistanceFromSamples(k, sampleTree, pointBottom); + optimizedIntersectingMBRs.addAll( + getMBRIntersectEnvelopes( + 0.0, maxKNNDistanceBottom, pointBottom.getX(), pointBottom.getY())); + } + + for (int j = 1; j < divide; j++) { + double y = minY + j * yStep; + + // Left edge (minX) + Point pointLeft = geometryFactory.createPoint(new Coordinate(minX, y)); + double maxKNNDistanceLeft = getMaxDistanceFromSamples(k, sampleTree, pointLeft); + optimizedIntersectingMBRs.addAll( + getMBRIntersectEnvelopes(0.0, maxKNNDistanceLeft, pointLeft.getX(), pointLeft.getY())); + + // Right edge (maxX) + Point pointRight = geometryFactory.createPoint(new Coordinate(maxX, y)); + double maxKNNDistanceRight = getMaxDistanceFromSamples(k, sampleTree, pointRight); + optimizedIntersectingMBRs.addAll( + getMBRIntersectEnvelopes(0.0, maxKNNDistanceRight, pointRight.getX(), pointRight.getY())); + } + + return new ArrayList<>(optimizedIntersectingMBRs); + } + + /** + * This function is used to calculate the maximum distance from the centroid to the k-nearest + * neighbors in the samples. It is used to expand the partitioned boundaries. + * + * @param k + * @param sampleTree + * @param centroid + * @return + */ + private static double getMaxDistanceFromSamples(int k, STRtree sampleTree, Point centroid) { + // 3 - Find the k-nearest neighbors in the samples of the centroid in the STR tree + Object[] kNearestNeighbors = + sampleTree.nearestNeighbour( + centroid.getEnvelopeInternal(), centroid, new EuclideanItemDistance(), k); + + // 4 - Calculate the distance to the farthest neighbor + double maxDistance = 0; + for (Object neighbor : kNearestNeighbors) { + if (neighbor instanceof Geometry) { + Envelope neighborEnvelope = ((Geometry) neighbor).getEnvelopeInternal(); + Coordinate neighborCoord = + new Coordinate(neighborEnvelope.centre().getX(), neighborEnvelope.centre().getY()); + Point neighborPoint = geometryFactory.createPoint(neighborCoord); + double distance = centroid.distance(neighborPoint); + if (distance > maxDistance) { + maxDistance = distance; + } + } + } + return maxDistance; + } + + /** + * This function is used to get the MBRs that intersect with the circle constructed around the + * centroid. It is used to expand the partitioned boundaries. If the number of intersecting MBRs + * is too large, we optimize by considering all vertices of the MBRs to construct the circle. This + * approach eliminates the need to add an additional margin (ui) to the maxDistance. + * + * @param ui + * @param maxDistance + * @param centroidX + * @param centroidY + * @return + */ + private List getMBRIntersectEnvelopes( + double ui, double maxDistance, double centroidX, double centroidY) { + // 5 - Construct the circle with radius ui and center centroid + // Calculate the radius of the circle + double gamma_i = 2 * ui + maxDistance; + // Since we're working with rectangles, this would be an envelope that fully contains the + // circle + Envelope circleEnvelope = + new Envelope( + centroidX - gamma_i, centroidX + gamma_i, + centroidY - gamma_i, centroidY + gamma_i); + + Coordinate center = new Coordinate(centroidX, centroidY); + Geometry circle = geometryFactory.createPoint(center).buffer(gamma_i); + + // 6 - Compute all the MBRs that intersect with the circle and add them to a hash map + List candidateEnvelopes = strTree.query(circleEnvelope); + + // Filter the candidate envelopes to find those that intersect with the circle + List intersectingMBRs = new ArrayList<>(); + for (Envelope candidateEnvelope : candidateEnvelopes) { + Geometry envelopeGeometry = geometryFactory.toGeometry(candidateEnvelope); + if (circle.intersects(envelopeGeometry)) { + intersectingMBRs.add(candidateEnvelope); + } + } + return intersectingMBRs; + } +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadtreePartitioning.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadtreePartitioning.java index cc699116de..8824ff2478 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadtreePartitioning.java +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadtreePartitioning.java @@ -27,7 +27,7 @@ public class QuadtreePartitioning implements Serializable { /** The Quad-Tree. */ - private final StandardQuadTree partitionTree; + protected final StandardQuadTree partitionTree; /** * Instantiates a new Quad-Tree partitioning. diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/quadtree/ExtendedQuadTree.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/quadtree/ExtendedQuadTree.java new file mode 100644 index 0000000000..73169363ee --- /dev/null +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/quadtree/ExtendedQuadTree.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core.spatialPartitioning.quadtree; + +import java.io.Serializable; +import java.util.*; +import org.apache.sedona.common.utils.HalfOpenRectangle; +import org.apache.sedona.core.spatialPartitioning.PartitioningUtils; +import org.apache.sedona.core.spatialPartitioning.QuadTreeRTPartitioning; +import org.locationtech.jts.geom.Envelope; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.geom.Point; +import org.locationtech.jts.index.strtree.STRtree; +import scala.Tuple2; + +/** + * The ExtendedQuadTree class uses a modified quad-tree approach for partitioning spatial data, as + * described in "Simba: Efficient In-Memory Spatial Analytics". + * + *

In this approach, a global R-tree index is constructed by taking a set of random samples from + * the dataset and building the R-tree on the master node. For each partition, the distance from the + * furthest point in the partition to its centroid is calculated. Using the R-tree, the k-nearest + * neighbors of each centroid are found, and a distance bound is derived for each partition. This + * bound ensures that the k-nearest neighbors of any point in a partition can be found within a + * subset of the data identified by a circle range query centered at the centroid with the derived + * radius. This method guarantees efficient and accurate k-nearest neighbor joins by leveraging both + * local and global spatial indexing. + */ +public class ExtendedQuadTree extends PartitioningUtils implements Serializable { + // hold the samples temporarily + private final transient List samples = new ArrayList<>(); + + private final Envelope boundary; + private final int numPartitions; + + // original partitioning quad tree + // for non-overlapped partitioning, the original quad tree is used + private StandardQuadTree partitionTree; + + // The expanded boundaries are generated by the quad tree + r-tree + // for overlapped partitioning, the expanded boundaries are used + private HashMap> expandedBoundaries; + + // The spatial index for partitioned MBRs + // for overlapped partitioning, the spatial index is used + private STRtree spatialExpandedBoundaryIndex; + + private boolean useNonOverlapped = false; + + public HashMap> getExpandedBoundaries() { + return expandedBoundaries; + } + + public STRtree getSpatialExpandedBoundaryIndex() { + return spatialExpandedBoundaryIndex; + } + + /** + * Constructor to initialize the partitions list. + * + * @param boundary + * @param numPartitions + */ + public ExtendedQuadTree(Envelope boundary, int numPartitions) { + this.boundary = boundary; + this.numPartitions = numPartitions; + } + + /** + * Constructor to initialize the partitions list with non-overlapped boundaries. + * + * @param extendedQuadTree + * @param useNonOverlapped + */ + public ExtendedQuadTree(ExtendedQuadTree extendedQuadTree, boolean useNonOverlapped) { + this.boundary = extendedQuadTree.boundary; + this.numPartitions = extendedQuadTree.numPartitions; + this.expandedBoundaries = extendedQuadTree.expandedBoundaries; + this.spatialExpandedBoundaryIndex = extendedQuadTree.spatialExpandedBoundaryIndex; + this.partitionTree = extendedQuadTree.partitionTree; + this.useNonOverlapped = useNonOverlapped; + } + + /** + * Returns the boundary of the partition zones. + * + * @return + */ + public Envelope getBoundary() { + return boundary; + } + + /** + * Returns the number of partitions. + * + * @return + */ + public int getPartitionNum() { + return numPartitions; + } + + /** + * Insert a new sample into the sample list. + * + * @param sample Envelope object to be inserted. + */ + public void insert(Envelope sample) { + samples.add(sample); + } + + /** + * Check the geometry against the partition zones to find the IDs of overlapping ranges. Note that + * the geometry can be in multiple ranges because ranges can overlap. + * + * @param geometry Geometry object to be placed. + * @return Iterator of Tuple2 containing partition ID and the corresponding geometry. + */ + @Override + public Iterator> placeObject(Geometry geometry) { + if (useNonOverlapped) { + Objects.requireNonNull(geometry, "spatialObject"); + + // KNN join uses geometry's centroid to calculate the distance + final Envelope envelope = geometry.getCentroid().getEnvelopeInternal(); + + final List matchedPartitions = + this.partitionTree.findZones(new QuadRectangle(envelope)); + + final Point point = geometry.getCentroid(); + + final Set> result = new HashSet<>(); + for (QuadRectangle rectangle : matchedPartitions) { + // For points, make sure to return only one partition + if (point != null && !(new HalfOpenRectangle(rectangle.getEnvelope())).contains(point)) { + continue; + } + + result.add(new Tuple2(rectangle.partitionId, geometry)); + } + + // knn join uses the centroid of the geometry + return result.iterator(); + } else { + // use the expanded boundaries + List> result = new ArrayList<>(); + Envelope objectEnvelope = geometry.getEnvelopeInternal(); + + // Query the spatial index for intersecting envelopes + List intersectingIds = spatialExpandedBoundaryIndex.query(objectEnvelope); + + for (Integer partitionId : intersectingIds) { + result.add(new Tuple2<>(partitionId, geometry)); + } + + return result.iterator(); + } + } + + /** + * Check the geometry against the partition zones to find the IDs of overlapping ranges. Only + * returns the IDs of the overlapping partitions. Note that the geometry can be in multiple ranges + * because ranges can overlap. + * + * @param geometry Geometry object to be checked. + * @return Set of integers representing the IDs of the overlapping partitions. + */ + @Override + public Set getKeys(Geometry geometry) { + if (!useNonOverlapped) { + // knn join uses the centroid of the geometry + return partitionTree.getKeys(geometry.getCentroid()); + } else { + // use the expanded boundaries + Set keys = new HashSet<>(); + Envelope objectEnvelope = geometry.getEnvelopeInternal(); + + // Query the spatial index for intersecting envelopes + List intersectingIds = spatialExpandedBoundaryIndex.query(objectEnvelope); + + keys.addAll(intersectingIds); + + return keys; + } + } + + @Override + public List fetchLeafZones() { + return partitionTree.fetchLeafZones(); + } + + /** + * Builds the quad-tree partitioning structure and calculates the expanded boundaries for + * efficient spatial partitioning. + * + *

This method performs the following steps: 1. Forces the quad-tree to grow to a minimum level + * to ensure a sufficient number of partitions, which might slightly differ from the specified + * number. 2. Constructs the quad-tree partitioning using the given samples and boundary, + * initializing it to the calculated minimum level. 3. Creates the expanded boundaries by building + * an STR (Sort-Tile-Recursive) tree using the provided number of neighbor samples and sampling + * probability. 4. Clears the samples to avoid broadcasting them to all nodes involved in + * partitioning. + * + * @param neighborSampleNumber the number of neighbor samples to consider for building the STR + * tree. + */ + public void build(int neighborSampleNumber) throws Exception { + // Force the quad-tree to grow up to a certain level + // So the actual num of partitions might be slightly different + int minLevel = (int) Math.max(Math.log(numPartitions) / Math.log(4), 0); + QuadTreeRTPartitioning quadTreeRTPartitioning = + new QuadTreeRTPartitioning(samples, boundary, numPartitions, minLevel); + partitionTree = quadTreeRTPartitioning.getPartitionTree(); + + // Create the expanded boundaries + quadTreeRTPartitioning.buildSTRTree(samples, neighborSampleNumber); + expandedBoundaries = quadTreeRTPartitioning.getMbrs(); + spatialExpandedBoundaryIndex = quadTreeRTPartitioning.getMbrSpatialIndex(); + + // Make sure not to broadcast all the samples used to build the Quad + // tree to all nodes which are doing partitioning + samples.clear(); + partitionTree.dropElements(); + } + + public StandardQuadTree getQuadTree() { + return partitionTree; + } +} diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java b/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java index bd386c353d..70c5a12022 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java @@ -32,6 +32,7 @@ import org.apache.sedona.core.enums.GridType; import org.apache.sedona.core.enums.IndexType; import org.apache.sedona.core.spatialPartitioning.*; +import org.apache.sedona.core.spatialPartitioning.quadtree.ExtendedQuadTree; import org.apache.sedona.core.spatialPartitioning.quadtree.StandardQuadTree; import org.apache.sedona.core.spatialRddTool.IndexBuilder; import org.apache.sedona.core.spatialRddTool.StatCalculator; @@ -50,6 +51,7 @@ import org.locationtech.jts.geom.LinearRing; import org.locationtech.jts.geom.Polygon; import org.locationtech.jts.index.SpatialIndex; +import org.locationtech.jts.index.strtree.STRtree; import org.locationtech.jts.io.WKBWriter; import org.locationtech.jts.io.WKTWriter; import org.wololo.geojson.Feature; @@ -94,6 +96,9 @@ public class SpatialRDD implements Serializable { /** The sample number. */ private int sampleNumber = -1; + /** The neighbor sample number. */ + private int neighborSampleNumber = -1; + public int getSampleNumber() { return sampleNumber; } @@ -107,6 +112,15 @@ public void setSampleNumber(int sampleNumber) { this.sampleNumber = sampleNumber; } + /** + * Sets the neighbor sample number. + * + * @param neighborSampleNumber the new neighbor sample number + */ + public void setNeighborSampleNumber(int neighborSampleNumber) { + this.neighborSampleNumber = neighborSampleNumber; + } + /** * CRS transform. * @@ -232,6 +246,14 @@ public Envelope call(T geometry) throws Exception { partitioner = new KDBTreePartitioner(tree); break; } + case QUADTREE_RTREE: + { + ExtendedQuadTree tree = new ExtendedQuadTree<>(paddedBoundary, numPartitions); + ExtendedQuadTree extendedQuadTree = (ExtendedQuadTree) tree; + extendedQuadTree.build(neighborSampleNumber); + partitioner = new QuadTreeRTPartitioner(extendedQuadTree); + } + default: throw new Exception( "[AbstractSpatialRDD][spatialPartitioning] Unsupported spatial partitioning method. " @@ -354,6 +376,17 @@ public void buildIndex(final IndexType indexType, boolean buildIndexOnSpatialPar } } + /** + * Builds the index on coalesced raw spatial RDD. + * + * @param indexType the index type + * @throws Exception the exception + */ + public STRtree coalesceAndBuildRawIndex(final IndexType indexType) { + return (STRtree) + this.rawSpatialRDD.coalesce(1).mapPartitions(new IndexBuilder(indexType)).take(1).get(0); + } + /** * Boundary. * diff --git a/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java b/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java index 0bc5bd174f..86ec0eb50a 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java +++ b/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java @@ -56,6 +56,13 @@ public class SedonaConf implements Serializable { private SpatialJoinOptimizationMode spatialJoinOptimizationMode; + // Parameters for knn joins + private boolean includeTieBreakersInKNNJoins = false; + private double skewnessCutoffRatioInKNNJoins = 1.0; + private int skewnessMinimumMBRCountInKNNJoins = 100; + private int skewnessMaximumMBRDividesInKNNJoins = 100; + private boolean enableParallelPartitioningInKNNJoins = true; + public static SedonaConf fromActiveSession() { return new SedonaConf(SparkSession.active().conf()); } @@ -88,6 +95,25 @@ public SedonaConf(RuntimeConfig runtimeConfig) { this.spatialJoinOptimizationMode = SpatialJoinOptimizationMode.getSpatialJoinOptimizationMode( runtimeConfig.get("sedona.join.optimizationmode", "nonequi")); + + // Parameters for knn joins + this.includeTieBreakersInKNNJoins = + Boolean.parseBoolean( + runtimeConfig.get("spark.sedona.join.knn.includeTieBreakers", "false")); + + this.skewnessCutoffRatioInKNNJoins = + Double.parseDouble(runtimeConfig.get("spark.sedona.join.knn.skewnessCutoffRatio", "1.0")); + + this.skewnessMinimumMBRCountInKNNJoins = + Integer.parseInt(runtimeConfig.get("spark.sedona.join.knn.skewnessMinimumMBRCount", "100")); + + this.skewnessMaximumMBRDividesInKNNJoins = + Integer.parseInt( + runtimeConfig.get("spark.sedona.join.knn.skewnessMaximumMBRDivides", "100")); + + this.enableParallelPartitioningInKNNJoins = + Boolean.parseBoolean( + runtimeConfig.get("spark.sedona.join.knn.enableParallelPartitioning", "true")); } public boolean getUseIndex() { @@ -126,6 +152,26 @@ public long getAutoBroadcastJoinThreshold() { return autoBroadcastJoinThreshold; } + public boolean isIncludeTieBreakersInKNNJoins() { + return includeTieBreakersInKNNJoins; + } + + public double getSkewnessCutoffRatioInKNNJoins() { + return skewnessCutoffRatioInKNNJoins; + } + + public int getSkewnessMinimumMBRCountInKNNJoins() { + return skewnessMinimumMBRCountInKNNJoins; + } + + public int getSkewnessMaximumMBRDividesInKNNJoins() { + return skewnessMaximumMBRDividesInKNNJoins; + } + + public boolean isEnableParallelPartitioningInKNNJoins() { + return enableParallelPartitioningInKNNJoins; + } + public String toString() { try { String sb = ""; diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastObjectSideKNNJoinExec.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastObjectSideKNNJoinExec.scala new file mode 100644 index 0000000000..1b21c79e7c --- /dev/null +++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastObjectSideKNNJoinExec.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.sql.sedona_sql.strategy.join + +import org.apache.sedona.core.enums.{DistanceMetric, IndexType} +import org.apache.sedona.core.spatialOperator.JoinQuery.JoinParams +import org.apache.sedona.core.spatialOperator.SpatialPredicate +import org.apache.sedona.core.spatialRDD.SpatialRDD +import org.apache.sedona.core.utils.SedonaConf +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeRow} +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.sedona_sql.execution.SedonaBinaryExecNode +import org.locationtech.jts.geom.Geometry + +case class BroadcastObjectSideKNNJoinExec( + left: SparkPlan, + right: SparkPlan, + leftShape: Expression, + rightShape: Expression, + joinSide: JoinSide, + joinType: JoinType, + k: Expression, + useApproximate: Boolean, + spatialPredicate: SpatialPredicate, + isGeography: Boolean, + condition: Expression, + extraCondition: Option[Expression] = None) + extends SedonaBinaryExecNode + with TraitKNNJoinQueryExec + with Logging { + + /** + * Convert the both RDDs to SpatialRDDs + * @param leftRdd + * the left RDD + * @param leftShapeExpr + * the shape expression + * @param rightRdd + * the right RDD + * @param rightShapeExpr + * the shape expression + * @return + */ + override def toSpatialRddPair( + leftRdd: RDD[UnsafeRow], + leftShapeExpr: Expression, + rightRdd: RDD[UnsafeRow], + rightShapeExpr: Expression): (SpatialRDD[Geometry], SpatialRDD[Geometry]) = { + (leftToSpatialRDD(leftRdd, leftShapeExpr), rightToSpatialRDD(rightRdd, rightShapeExpr)) + } + + /** + * Convert the left RDD (queries) to SpatialRDD + * @param rdd + * the left RDD + * @param shapeExpression + * the shape expression + * @return + */ + def leftToSpatialRDD(rdd: RDD[UnsafeRow], shapeExpression: Expression): SpatialRDD[Geometry] = { + toSpatialRDD(rdd, shapeExpression) + } + + /** + * Convert the right RDD (queries) to SpatialRDD + * @param rdd + * the right RDD + * @param shapeExpression + * the shape expression + * @param projection + * the projection + * @return + */ + def rightToSpatialRDD( + rdd: RDD[UnsafeRow], + shapeExpression: Expression): SpatialRDD[Geometry] = { + toSpatialRDD(rdd, shapeExpression) + } + + /** + * Broadcast the dominant shapes (objects) to all the partitions + * + * This type of the join does not need to do spatial partition. + * + * For left side (queries) broadcast: the join needs to be reduced after the join. For right + * side (objects) broadcast: the join does not need to be reduced after the join. + * + * @param objectsShapes + * the dominant shapes (objects) + * @param queryShapes + * the follower shapes (queries) + * @param numPartitions + * the number of partitions + * @param sedonaConf + * the Sedona configuration + */ + override def doSpatialPartitioning( + objectsShapes: SpatialRDD[Geometry], + queryShapes: SpatialRDD[Geometry], + numPartitions: Integer, + sedonaConf: SedonaConf): Unit = { + require(numPartitions > 0, "The number of partitions must be greater than 0.") + val kValue: Int = this.k.eval().asInstanceOf[Int] + require(kValue > 0, "The number of neighbors must be greater than 0.") + objectsShapes.setNeighborSampleNumber(kValue) + broadcastJoin = true + } + + /** + * Get the KNN join parameters This is required to determine the join strategy to support + * different KNN join strategies. This function needs to be updated when new join strategies are + * supported. + * + * @return + * the KNN join parameters + */ + override def getKNNJoinParams: JoinParams = { + // Please update this function when new join strategies are added + // Number of neighbors to find + val kValue: Int = this.k.eval().asInstanceOf[Int] + // Metric to use in the join to calculate the distance, only Euclidean and Spheroid are supported + val distanceMetric = if (isGeography) DistanceMetric.SPHEROID else DistanceMetric.EUCLIDEAN + val joinParams = + new JoinParams(true, null, IndexType.RTREE, null, kValue, distanceMetric, null) + joinParams + } + + /** + * Copy the plan with new children + * @param newLeft + * @param newRight + * @return + */ + protected def withNewChildrenInternal(newLeft: SparkPlan, newRight: SparkPlan): SparkPlan = { + copy(left = newLeft, right = newRight) + } +} diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastQuerySideKNNJoinExec.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastQuerySideKNNJoinExec.scala new file mode 100644 index 0000000000..5a3c59ef98 --- /dev/null +++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastQuerySideKNNJoinExec.scala @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.sql.sedona_sql.strategy.join + +import org.apache.sedona.core.enums.{DistanceMetric, GridType, IndexType} +import org.apache.sedona.core.spatialOperator.JoinQuery.JoinParams +import org.apache.sedona.core.spatialOperator.SpatialPredicate +import org.apache.sedona.core.spatialPartitioning.QuadTreeRTPartitioner +import org.apache.sedona.core.spatialRDD.SpatialRDD +import org.apache.sedona.core.utils.SedonaConf +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeRow} +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.sedona_sql.execution.SedonaBinaryExecNode +import org.locationtech.jts.geom.Geometry + +case class BroadcastQuerySideKNNJoinExec( + left: SparkPlan, + right: SparkPlan, + leftShape: Expression, + rightShape: Expression, + joinSide: JoinSide, + joinType: JoinType, + k: Expression, + useApproximate: Boolean, + spatialPredicate: SpatialPredicate, + isGeography: Boolean, + condition: Expression, + extraCondition: Option[Expression] = None) + extends SedonaBinaryExecNode + with TraitKNNJoinQueryExec + with Logging { + + /** + * Convert the both RDDs to SpatialRDDs + * @param leftRdd + * the left RDD + * @param leftShapeExpr + * the shape expression + * @param rightRdd + * the right RDD + * @param rightShapeExpr + * the shape expression + * @return + */ + override def toSpatialRddPair( + leftRdd: RDD[UnsafeRow], + leftShapeExpr: Expression, + rightRdd: RDD[UnsafeRow], + rightShapeExpr: Expression): (SpatialRDD[Geometry], SpatialRDD[Geometry]) = { + (leftToSpatialRDD(leftRdd, leftShapeExpr), rightToSpatialRDD(rightRdd, rightShapeExpr)) + } + + /** + * Convert the left RDD (queries) to SpatialRDD + * @param rdd + * the left RDD + * @param shapeExpression + * the shape expression + * @param projection + * the projection + * @return + */ + def leftToSpatialRDD( + rdd: RDD[UnsafeRow], + shapeExpression: Expression, + projection: Option[Seq[Expression]] = None): SpatialRDD[Geometry] = { + toSpatialRDD(rdd, shapeExpression) + } + + /** + * Convert the right RDD (queries) to SpatialRDD + * @param rdd + * the right RDD + * @param shapeExpression + * the shape expression + * @param projection + * the projection + * @return + */ + def rightToSpatialRDD( + rdd: RDD[UnsafeRow], + shapeExpression: Expression, + projection: Option[Seq[Expression]] = None): SpatialRDD[Geometry] = { + toSpatialRDD(rdd, shapeExpression) + } + + /** + * Broadcast the dominant shapes (objects) to all the partitions + * + * This type of the join does not need to do spatial partition. + * + * For left side (queries) broadcast: the join needs to be reduced after the join. For right + * side (objects) broadcast: the join does not need to be reduced after the join. + * + * @param objectsShapes + * the dominant shapes (objects) + * @param queryShapes + * the follower shapes (queries) + * @param numPartitions + * the number of partitions + * @param sedonaConf + * the Sedona configuration + */ + override def doSpatialPartitioning( + objectsShapes: SpatialRDD[Geometry], + queryShapes: SpatialRDD[Geometry], + numPartitions: Integer, + sedonaConf: SedonaConf): Unit = { + require(numPartitions > 0, "The number of partitions must be greater than 0.") + val kValue: Int = this.k.eval().asInstanceOf[Int] + require(kValue > 0, "The number of neighbors must be greater than 0.") + objectsShapes.setNeighborSampleNumber(kValue) + + val joinPartitions: Integer = numPartitions; + broadcastJoin = false + + // expand the boundary for partition to include both RDDs + objectsShapes.analyze() + queryShapes.analyze() + objectsShapes.boundaryEnvelope.expandToInclude(queryShapes.boundaryEnvelope) + + objectsShapes.spatialPartitioning(GridType.QUADTREE_RTREE, joinPartitions) + queryShapes.spatialPartitioning( + objectsShapes.getPartitioner.asInstanceOf[QuadTreeRTPartitioner].nonOverlappedPartitioner()) + + objectsShapes.buildIndex(IndexType.RTREE, true) + } + + /** + * Get the KNN join parameters This is required to determine the join strategy to support + * different KNN join strategies. This function needs to be updated when new join strategies are + * supported. + * + * @return + * the KNN join parameters + */ + override def getKNNJoinParams: JoinParams = { + // Please update this function when new join strategies are added + // Number of neighbors to find + val kValue: Int = this.k.eval().asInstanceOf[Int] + // Metric to use in the join to calculate the distance, only Euclidean and Spheroid are supported + val distanceMetric = if (isGeography) DistanceMetric.SPHEROID else DistanceMetric.EUCLIDEAN + val joinParams = + new JoinParams(true, null, IndexType.RTREE, null, kValue, distanceMetric, null) + joinParams + } + + /** + * Copy the plan with new children + * @param newLeft + * @param newRight + * @return + */ + protected def withNewChildrenInternal(newLeft: SparkPlan, newRight: SparkPlan): SparkPlan = { + copy(left = newLeft, right = newRight) + } +} diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/KNNJoinExec.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/KNNJoinExec.scala new file mode 100644 index 0000000000..2b9bbfb50b --- /dev/null +++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/KNNJoinExec.scala @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.sql.sedona_sql.strategy.join + +import org.apache.sedona.core.enums.{DistanceMetric, GridType, IndexType} +import org.apache.sedona.core.spatialOperator.JoinQuery.JoinParams +import org.apache.sedona.core.spatialOperator.SpatialPredicate +import org.apache.sedona.core.spatialPartitioning.QuadTreeRTPartitioner +import org.apache.sedona.core.spatialRDD.SpatialRDD +import org.apache.sedona.core.utils.SedonaConf +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeRow} +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.sedona_sql.execution.SedonaBinaryExecNode +import org.locationtech.jts.geom.Geometry + +/** + * KNN / AKNN joins requires target geometries (objects) to be in the same partition as the query + * geometries. To create an overlap and guarantee matching geometries end up in the same + * partition, the target geometry is expanded during partitioning. + * + * E.g., SELECT * FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, $numNeighbors, + * true) SELECT * FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, $numNeighbors, + * true) + * + * @param left + * left side of the join + * @param right + * right side of the join + * @param leftShape + * shape expression for the left side + * @param rightShape + * shape expression for the right side + * @param k + * \- number of neighbors to find + * @param useApproximate + * whether to use approximate distance for the join + * @param spatialPredicate + * spatial predicate as join condition + * @param condition + * full join condition + * @param extraCondition + * extra join condition other than spatialPredicate + */ +case class KNNJoinExec( + left: SparkPlan, + right: SparkPlan, + leftShape: Expression, + rightShape: Expression, + joinType: JoinType, + k: Expression, + spatialPredicate: SpatialPredicate, + isGeography: Boolean, + condition: Expression, + extraCondition: Option[Expression] = None) + extends SedonaBinaryExecNode + with TraitKNNJoinQueryExec + with Logging { + + /** + * Convert the both RDDs to SpatialRDDs + * @param leftRdd + * the left RDD + * @param leftShapeExpr + * the shape expression + * @param rightRdd + * the right RDD + * @param rightShapeExpr + * the shape expression + * @return + */ + override def toSpatialRddPair( + leftRdd: RDD[UnsafeRow], + leftShapeExpr: Expression, + rightRdd: RDD[UnsafeRow], + rightShapeExpr: Expression): (SpatialRDD[Geometry], SpatialRDD[Geometry]) = { + (leftToSpatialRDD(leftRdd, leftShapeExpr), rightToSpatialRDD(rightRdd, rightShapeExpr)) + } + + /** + * Convert the left RDD (queries) to SpatialRDD + * @param rdd + * the left RDD + * @param shapeExpression + * the shape expression + * @param projection + * the projection + * @return + */ + def leftToSpatialRDD( + rdd: RDD[UnsafeRow], + shapeExpression: Expression, + projection: Option[Seq[Expression]] = None): SpatialRDD[Geometry] = { + toSpatialRDD(rdd, shapeExpression) + } + + /** + * Convert the right RDD (queries) to SpatialRDD + * @param rdd + * the right RDD + * @param shapeExpression + * the shape expression + * @param projection + * the projection + * @return + */ + def rightToSpatialRDD( + rdd: RDD[UnsafeRow], + shapeExpression: Expression, + projection: Option[Seq[Expression]] = None): SpatialRDD[Geometry] = { + toSpatialRDD(rdd, shapeExpression) + } + + /** + * Copy the plan with new children + * @param newLeft + * @param newRight + * @return + */ + protected def withNewChildrenInternal(newLeft: SparkPlan, newRight: SparkPlan): SparkPlan = { + copy(left = newLeft, right = newRight) + } + + /** + * Execute the spatial partitioning for KNN join This is required to ensure that the target + * geometries (objects) are in the same partition as the query geometries. + * + * Different KNN algorithms require different partitioning strategies. E.g., approximate KNN + * join requires a different partitioning strategy than exact KNN join. + * + * @param objectsShapes + * the dominant shapes (objects) + * @param queryShapes + * the follower shapes (queries) + * @param numPartitions + * the number of partitions + * @param sedonaConf + * the Sedona configuration + */ + override def doSpatialPartitioning( + objectsShapes: SpatialRDD[Geometry], + queryShapes: SpatialRDD[Geometry], + numPartitions: Integer, + sedonaConf: SedonaConf): Unit = { + require(numPartitions > 0, "The number of partitions must be greater than 0.") + val kValue: Int = this.k.eval().asInstanceOf[Int] + require(kValue > 0, "The number of neighbors must be greater than 0.") + objectsShapes.setNeighborSampleNumber(kValue) + + exactSpatialPartitioning(objectsShapes, queryShapes, numPartitions) + } + + /** + * Exact spatial partitioning for KNN join + * @param dominantShapes + * the dominant (objects) shapes + * @param followerShapes + * the follower (queries) shapes + */ + private def exactSpatialPartitioning( + dominantShapes: SpatialRDD[Geometry], + followerShapes: SpatialRDD[Geometry], + numPartitions: Integer): Unit = { + // analyze the both RDDs to get the statistics (e.g., boundary) + dominantShapes.analyze() + followerShapes.analyze() + + // expand the boundary for partition to include both RDDs + dominantShapes.boundaryEnvelope.expandToInclude(followerShapes.boundaryEnvelope) + + // use modified quadtree partitioning, as it is an exact algorithm + dominantShapes.spatialPartitioning(GridType.QUADTREE_RTREE, numPartitions) + followerShapes.spatialPartitioning( + dominantShapes.getPartitioner + .asInstanceOf[QuadTreeRTPartitioner] + .nonOverlappedPartitioner()) + + dominantShapes.buildIndex(IndexType.RTREE, true) + } + + /** + * Get the KNN join parameters This is required to determine the join strategy to support + * different KNN join strategies. This function needs to be updated when new join strategies are + * supported. + * + * @return + * the KNN join parameters + */ + override def getKNNJoinParams: JoinParams = { + // Please update this function when new join strategies are added + // Number of neighbors to find + val kValue: Int = this.k.eval().asInstanceOf[Int] + // Metric to use in the join to calculate the distance, only Euclidean and Spheroid are supported + val distanceMetric = if (isGeography) DistanceMetric.SPHEROID else DistanceMetric.EUCLIDEAN + val joinParams = + new JoinParams(true, null, IndexType.RTREE, null, kValue, distanceMetric, null) + joinParams + } +} diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitKNNJoinQueryExec.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitKNNJoinQueryExec.scala new file mode 100644 index 0000000000..5eba2a94e1 --- /dev/null +++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitKNNJoinQueryExec.scala @@ -0,0 +1,313 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.sql.sedona_sql.strategy.join + +import org.apache.commons.lang3.Range +import org.apache.sedona.core.spatialOperator.JoinQuery +import org.apache.sedona.core.spatialOperator.JoinQuery.JoinParams +import org.apache.sedona.core.spatialPartitioning.{QuadTreeRTPartitioner, SpatialPartitioner} +import org.apache.sedona.core.utils.SedonaConf +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner +import org.apache.spark.sql.catalyst.expressions.{BindReferences, Expression, Predicate, UnsafeRow} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.{SQLExecution, SparkPlan} +import org.locationtech.jts.geom.{Envelope, Geometry} + +import java.io.PrintWriter +import java.nio.file.Paths +import java.util + +/** + * TraitKNNJoinQueryExec is a trait that extends the TraitJoinQueryExec trait and provides the + * necessary functionality to execute a KNN join operation. + * + * It is used by the KNNJoinExec class to execute a KNN join operation. The KNN join operation is + * a k-nearest neighbors join that finds the k-nearest neighbors of each object in the right + * dataset for each query in the left dataset. + */ +trait TraitKNNJoinQueryExec extends TraitJoinQueryExec { + self: SparkPlan => + + protected var broadcastJoin: Boolean = false + protected var querySide: JoinSide = null + + private lazy val sedonaConf = SedonaConf.fromActiveSession + override lazy val metrics: Map[String, SQLMetric] = Map.empty + + override protected def doExecute(): RDD[InternalRow] = { + // Execute the join + executeKNNJoin(sedonaConf) + } + + /** + * Executes a KNN (k-nearest neighbors) join operation using the Sedona spatial library. + * + * This method binds the left and right shape references to their respective outputs, executes + * the left and right datasets as RDDs, converts them to spatial RDDs, and performs spatial + * partitioning based on Sedona configuration. + * + * The number of partitions is determined either by a predefined fallback value or optimized + * based on the sizes of the object and query shapes. If the partitioning fails, it uses the + * fallback value. + * + * It saves the spatial partitioner to a file if specified, gets the KNN join parameters, + * performs the KNN join, and finally converts the matched RDD to RowRDD. + * + * @param sedonaConf + * The Sedona configuration settings. + * @return + * RDD[InternalRow] The result of the KNN join as an RDD of InternalRows. + */ + private def executeKNNJoin(sedonaConf: SedonaConf): RDD[InternalRow] = { + val (querySparkPlan: SparkPlan, objectSparkPlan: SparkPlan, swapped: Boolean) = + getQueryAndObjectPlans(leftShape) + + val boundQueryShape = BindReferences.bindReference(leftShape, querySparkPlan.output) + val boundObjectShape = BindReferences.bindReference(rightShape, objectSparkPlan.output) + + val queryResultsRaw = querySparkPlan.execute().asInstanceOf[RDD[UnsafeRow]] + val objectResultsRaw = objectSparkPlan.execute().asInstanceOf[RDD[UnsafeRow]] + + val sedonaConf = SedonaConf.fromActiveSession + + val (queryShapes, objectShapes) = + toSpatialRddPair(queryResultsRaw, boundQueryShape, objectResultsRaw, boundObjectShape) + + objectShapes.analyze() + log.info( + "[SedonaSQL] Number of partitions on the objectShapes (right): " + objectResultsRaw.partitions.size) + + val joinParams: JoinParams = getKNNJoinParams + + // calculate the optimized or predefined number of partitions + // and do spatial partitioning + var numPartitions = -1 + try { + if (sedonaConf.getFallbackPartitionNum != -1) { + numPartitions = sedonaConf.getFallbackPartitionNum + } else { + // object shapes are the dominant side + numPartitions = knnJoinPartitionNumOptimizer( + objectShapes.rawSpatialRDD.partitions.size(), + queryShapes.rawSpatialRDD.partitions.size(), + objectShapes.approximateTotalCount, + joinParams.k) + } + // object shapes are the dominant side + doSpatialPartitioning(objectShapes, queryShapes, numPartitions, sedonaConf) + } catch { + case e: IllegalArgumentException => { + log.error(e.getMessage) + // Partition number are not qualified + // Use fallback num partitions specified in SedonaConf + numPartitions = sedonaConf.getFallbackPartitionNum + doSpatialPartitioning(queryShapes, objectShapes, numPartitions, sedonaConf) + } + } + + val matchesRDD: RDD[(Geometry, Geometry)] = + (queryShapes.spatialPartitionedRDD, objectShapes.spatialPartitionedRDD) match { + case (null, null) => + if (broadcastJoin) { + JoinQuery + .knnJoin( + queryShapes, + objectShapes, + joinParams, + sedonaConf.isIncludeTieBreakersInKNNJoins, + broadcastJoin) + .rdd + } else { + sparkContext.parallelize(Seq[(Geometry, Geometry)]()) + } + case _ => + JoinQuery + .knnJoin( + queryShapes, + objectShapes, + joinParams, + sedonaConf.isIncludeTieBreakersInKNNJoins, + broadcastJoin) + .rdd + } + + // Convert the matchesRDD to RowRDD + joinedRddToRowRdd(matchesRDD, swapped) + } + + def knnJoinPartitionNumOptimizer( + objectSidePartNum: Int, + querySidePartNum: Int, + objectSideCount: Long, + numNeighbor: Int): Int = { + log.info("[SedonaSQL] object side count: " + objectSideCount) + var numPartition = -1 + val candidatePartitionNum = (objectSideCount / (numNeighbor * 2)).intValue() + if (objectSidePartNum * 2 > objectSideCount) { + log.warn( + s"[SedonaSQL] KNN join object side partition number $objectSidePartNum is larger than 1/2 of the object side count $objectSideCount") + log.warn( + s"[SedonaSQL] Try to use object (follower) side partition number $querySidePartNum") + if (querySidePartNum * 2 > objectSideCount) { + log.warn( + s"[SedonaSQL] KNN join object (follower) side partition number is also larger than 1/2 of the object side count $objectSideCount") + log.warn( + s"[SedonaSQL] Try to use 1/2 of the object side count $candidatePartitionNum as the partition number of both sides") + if (candidatePartitionNum == 0) { + log.warn( + s"[SedonaSQL] 1/2 of $candidatePartitionNum is equal to 0. Use 1 as the partition number of both sides instead.") + numPartition = 1 + } else numPartition = candidatePartitionNum + } else numPartition = querySidePartNum + } else numPartition = objectSidePartNum + numPartition + } + + /** + * Gets the query and object plans based on the left shape. + * + * This method checks if the left shape is part of the left or right plan and returns the query + * and object plans accordingly. + * + * @param leftShape + * The left shape expression. + * @return + * (SparkPlan, SparkPlan) The query and object plans. + */ + private def getQueryAndObjectPlans(leftShape: Expression) = { + val isLeftQuerySide = + left.toString().toLowerCase().contains(leftShape.toString().toLowerCase()) + if (isLeftQuerySide) { + querySide = LeftSide + (left, right, false) + } else { + querySide = RightSide + (right, left, true) + } + } + + /** + * Converts the joined RDD of geometries to an RDD of InternalRows. + * + * This method maps over the partitions of the joined RDD, creating an UnsafeRow joiner that + * combines the left and right rows based on the given schemas. + * + * Each geometry's user data is expected to be an UnsafeRow, and the joiner is used to produce + * joined rows from the left and right geometry pairs. + * + * @param joinedRdd + * The RDD containing pairs of joined geometries. + * @return + * RDD[InternalRow] The resulting RDD of joined InternalRows. + */ + protected def joinedRddToRowRdd( + joinedRdd: RDD[(Geometry, Geometry)], + swapped: Boolean): RDD[InternalRow] = { + joinedRdd.mapPartitions { iter => + val joinRow = { + val joiner = GenerateUnsafeRowJoiner.create(left.schema, right.schema) + (l: UnsafeRow, r: UnsafeRow) => joiner.join(l, r) + } + + val joined = iter.map { case (l, r) => + val leftRow = l.getUserData.asInstanceOf[UnsafeRow] + val rightRow = r.getUserData.asInstanceOf[UnsafeRow] + if (swapped) + joinRow(rightRow, leftRow) + else + joinRow(leftRow, rightRow) + } + + // Apply the extra join conditions if it exists (e.g., S.ID < Q.ID) + extraCondition match { + case Some(condition) => + val boundCondition = Predicate.create(condition, output) + joined.filter(row => boundCondition.eval(row)) + case None => joined + } + } + } + + private def saveKNNPartitionerToFile( + partitioner: SpatialPartitioner, + savePath: String): Unit = { + partitioner match { + case null => + log.warn("[SedonaSQL] Spatial partitioner is null. Skip saving to file.") + + case qt: QuadTreeRTPartitioner => + val filePath = createFilePath(savePath, "quadtree-rt") + log.info(s"[SedonaSQL] Saving QuadTreeRT partitioner to file: $filePath") + writeGridsToFile(filePath, qt.getOverlappedGrids) + + case _ => + log.info("[SedonaSQL] Spatial partitioner type is not supported for saving to file.") + } + } + + private def createFilePath(savePath: String, partitionerType: String): String = { + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + Paths.get(savePath).toFile.mkdirs() + Paths + .get(savePath, s"partitioner-$partitionerType-$executionId-${System.currentTimeMillis()}") + .toString + } + + private def writeGridsToFile( + filePath: String, + grids: java.util.Map[Integer, java.util.List[Envelope]]): Unit = { + val writer = new PrintWriter(filePath) + try { + grids.forEach { case (key, envelopes) => + envelopes.forEach { envelope => + writer.write( + s"$key,${envelope.getMinX},${envelope.getMinY},${envelope.getMaxX},${envelope.getMaxY}\n") + } + } + } finally { + writer.close() + } + } + + private def writeGridsToFile( + filePath: String, + ranges: util.List[Range[java.lang.Long]]): Unit = { + val writer = new PrintWriter(filePath) + try { + var rangeId = 1 + ranges.forEach { range => + writer.write(s"$rangeId,${range.getMinimum},${range.getMaximum}\n") + rangeId += 1 + } + } finally { + writer.close() + } + } + + // The following methods are abstract and must be implemented by the concrete class + // that extends this trait. + // Override these methods to provide the necessary functionality for the KNN join. + def getKNNJoinParams: JoinParams +} + +object TraitKNNJoinQueryExec { + val counter = new java.util.concurrent.atomic.AtomicLong(0) +} diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala b/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala new file mode 100644 index 0000000000..eadf02d203 --- /dev/null +++ b/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala @@ -0,0 +1,733 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.sql + +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT +import org.apache.spark.sql.sedona_sql.expressions.st_constructors.ST_GeomFromText +import org.apache.spark.sql.sedona_sql.strategy.join.KNNJoinExec +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession} +import org.apache.spark.sql.functions.expr +import org.scalatest.matchers.must.Matchers.{be, include} +import org.scalatest.matchers.should.Matchers.convertToAnyShouldWrapper +import org.scalatest.prop.TableDrivenPropertyChecks +import org.apache.spark.sql.functions._ + +import java.nio.file.Files +import java.util.Random + +/** + * Test suite for KNN spatial join SQLs + * + * This test suite validates the KNN spatial join SQLs for Sedona-SQL module. The main purpose of + * this test suite is to validate the KNN spatial join SQLs but not the validation of the join + * results. + * + * The join results are validated in the Sedona core module with the unit tests. + */ +class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { + + val testDataDelimiter = "\t" + val knnPointsLocationQueries: String = resourceFolder + "knn/queries.csv" + val knnPointsLocationObjects: String = resourceFolder + "knn/objects.csv" + val knnPointsLocationSkewedObjects: String = resourceFolder + "knn/queries-large-skewed.csv" + val knnPointsLocationMultipleSkewedObjects: String = + resourceFolder + "knn/queries-large-skewed-multiple.csv" + val numPartitions = 4 + val numSkewPartitions = 100 + + override def beforeAll(): Unit = { + super.beforeAll() + prepareTempViewsForTestData() + prepareTempViewsForDifferentPartitionsTestData() + prepareTempViewsForSkewedTestData() + } + + describe("KNN spatial join SQLs should be parsed correctly") { + it("KNN Join with approximate algorithms based on euclidean distance") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "") + } + + it( + "KNN Join with approximate algorithms based on euclidean distance using different join clause ordering") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM OBJECTS JOIN QUERIES ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "") + } + + it( + "KNN Join with approximate algorithms based on euclidean distance using join-where clause and apply ST_Distance") { + val df = sparkSession.sql( + s"SELECT QUERIES.GEOM, OBJECTS.GEOM, ST_Distance(QUERIES.GEOM, OBJECTS.GEOM) FROM QUERIES, OBJECTS WHERE ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "") + } + + it( + "KNN Join with approximate algorithms based on euclidean distance using join-where clause and select gem") { + val df = sparkSession.sql( + s"SELECT QUERIES.GEOM, OBJECTS.GEOM FROM QUERIES, OBJECTS WHERE ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "") + } + + it( + "KNN Join with approximate algorithms based on euclidean distance using join-where clause and select all") { + val df = sparkSession.sql( + s"SELECT * FROM QUERIES, OBJECTS WHERE ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "") + } + + it("KNN Join with exact algorithms based on euclidean distance") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 3, true)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = false, + expressionSize = 5, + isGeography = true, + mustInclude = "") + } + + it("KNN Join based on single point on left side should not be supported") { + val exception = intercept[UnsupportedOperationException] { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(ST_MakePoint(100, 100, 1), OBJECTS.GEOM, 3, false)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "") + } + exception.getMessage should include("ST_AKNN filter is not yet supported in the join query") + } + + it("KNN Join based on single point on right side should not be supported") { + val exception = intercept[UnsupportedOperationException] { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(OBJECTS.GEOM, ST_MakePoint(100, 100, 1), 3, false)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "") + } + exception.getMessage should include("ST_AKNN filter is not yet supported in the join query") + } + + it("KNN Join based with complex join conditions using integer columns") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) AND QUERIES.ID <= 88") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "as int) <= 88))") + } + + it("KNN Join based with complex join conditions using text columns") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) AND QUERIES.SHAPE = 'point'") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "= point))") + } + + it("KNN Join based with complex join conditions using text columns and using where clause") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) WHERE QUERIES.SHAPE = 'point'") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "= point))") + } + + it("KNN Join should work with dataframe containing 0 partitions") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, EMPTYTABLE.ID FROM QUERIES JOIN EMPTYTABLE ON ST_AKNN(QUERIES.GEOM, EMPTYTABLE.GEOM, 3, false)") + validateQueryPlan( + df, + numNeighbors = 3, + useApproximate = true, + expressionSize = 5, + isGeography = false, + mustInclude = "") + } + } + + describe("AKNN spatial join SQLs should be executed correctly") { + it("AKNN Join with approximate algorithms based on EUCLIDEAN distance") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it( + "AKNN Join with approximate algorithms based on EUCLIDEAN distance with different join ordering") { + val df = sparkSession.sql( + s"SELECT OBJECTS.ID, QUERIES.ID FROM OBJECTS JOIN QUERIES ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,2][3,1][3,3][5,2][6,1][9,3][11,2][13,1][13,3][15,2][16,1][19,3]") + } + + it("AKNN Join with approximate algorithms based on SPHEROID distance") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 4, true)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it("AKNN Join with approximate algorithms with additional join conditions on id") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false) AND QUERIES.ID > 1") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(8) // 2 queries (filtered out 1) and 4 neighbors each + resultAll.mkString should be("[2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + } + + describe("KNN spatial join SQLs should be executed correctly") { + it("KNN Join with exact algorithms based on EUCLIDEAN distance") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it("KNN Join with exact algorithms based on SPHEROID distance") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, true)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it("KNN Join should support broadcast join hints - left") { + val df = sparkSession.sql( + s"SELECT /*+ BROADCAST(QUERIES) */ QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, true)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it("KNN Join should support broadcast join hints - right") { + val df = sparkSession.sql( + s"SELECT /*+ BROADCAST(OBJECTS) */ QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, true)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it("KNN Join should support broadcast join hints with different partitions - left") { + val df = sparkSession.sql( + s"SELECT /*+ BROADCAST(QUERIES_PAR2) */ QUERIES_PAR2.ID, OBJECTS_PAR4.ID FROM QUERIES_PAR2 JOIN OBJECTS_PAR4 ON ST_KNN(QUERIES_PAR2.GEOM, OBJECTS_PAR4.GEOM, 4, true)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it("KNN Join should support broadcast join hints with different partitions - right") { + val df = sparkSession.sql( + s"SELECT /*+ BROADCAST(OBJECTS_PAR4) */ QUERIES_PAR2.ID, OBJECTS_PAR4.ID FROM QUERIES_PAR2 JOIN OBJECTS_PAR4 ON ST_KNN(QUERIES_PAR2.GEOM, OBJECTS_PAR4.GEOM, 4, true)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it( + "KNN Join with approximate algorithms should work on tiny datasets with lots of partitions") { + val df = sparkSession + .range(0, 4) + .toDF("id") + .withColumn("geom", expr("ST_Point(id, id)")) + .repartition(10) + df.createOrReplaceTempView("df10parts") + val dfResult = sparkSession.sql( + s"SELECT A.ID, B.ID FROM DF10PARTS A JOIN DF10PARTS B ON ST_AKNN(A.GEOM, B.GEOM, 4, false)") + val resultAll = dfResult.collect().sortBy(row => (row.getLong(0), row.getLong(1))) + resultAll.mkString should be( + "[0,0][0,1][0,2][0,3][1,0][1,1][1,2][1,3][2,0][2,1][2,2][2,3][3,0][3,1][3,2][3,3]") + } + + it("KNN Join with exact algorithms with additional join conditions on id") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false) AND QUERIES.ID > 1") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(8) // 2 queries (filtered out 1) and 4 neighbors each + resultAll.mkString should be("[2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + } + + describe("KNN spatial join SQLs should be executed correctly with complex join conditions") { + it( + "KNN Join with approximate algorithms based on EUCLIDEAN distance with no additional join conditions") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") + } + + it( + "KNN Join with exact algorithms based on EUCLIDEAN distance with additional inequality (<) join conditions") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false) AND QUERIES.ID < OBJECTS.ID") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(10) + resultAll.mkString should be("[1,3][1,6][1,13][1,16][2,5][2,11][2,15][3,9][3,13][3,19]") + } + + it( + "KNN Join with exact algorithms based on EUCLIDEAN distance with additional inequality (!=) join conditions") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false) AND QUERIES.ID != OBJECTS.ID") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(11) + resultAll.mkString should be( + "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,9][3,13][3,19]") + } + + it( + "KNN Join with exact algorithms based on EUCLIDEAN distance with additional equality (=) join conditions") { + withOptimizationMode("all") { + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false) AND QUERIES.ID = OBJECTS.ID") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(1) + resultAll.mkString should be("[3,3]") + } + } + + it("KNN Join should respect export partitioner info - single cluster of points") { + var tempDir = + Files + .createTempDirectory("spatial_partitioner_export") + .toString // Create temporary directory + withSpatialPartitionerExport(tempDir) { + val df = sparkSession.sql( + s"SELECT QUERIES_SKEWED.ID, OBJECTS_SKEWED.ID FROM QUERIES_SKEWED JOIN OBJECTS_SKEWED ON ST_KNN(QUERIES_SKEWED.GEOM, OBJECTS_SKEWED.GEOM, 4, false)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + assert(resultAll.length > 0) + } + } + + it("KNN Join should respect export partitioner info - multiple clusters of points") { + var tempDir = + Files + .createTempDirectory("spatial_partitioner_export") + .toString // Create temporary directory + withSpatialPartitionerExport(tempDir) { + val df = sparkSession.sql( + s"SELECT QUERIES_SKEWED_MULTIPLE.ID, OBJECTS_SKEWED_MULTIPLE.ID FROM QUERIES_SKEWED_MULTIPLE JOIN OBJECTS_SKEWED_MULTIPLE ON ST_KNN(QUERIES_SKEWED_MULTIPLE.GEOM, OBJECTS_SKEWED_MULTIPLE.GEOM, 4, false)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + assert(resultAll.length > 0) + } + } + + it("AKNN Join should correctly match the join side and swap them if necessary") { + val df1 = + sparkSession.range(0, 100).toDF("id1").withColumn("geometry", expr("ST_Point(id1, id1)")) + val df2 = + sparkSession + .range(0, 100) + .toDF("id2") + .withColumn("geometry", expr("ST_Point(id2 + 0.1, id2 + 0.1)")) + .withColumn("id2", expr("concat('str', id2)")) + df1.createOrReplaceTempView("df1") + df2.createOrReplaceTempView("df2") + + var dfResult = sparkSession.sql( + "SELECT df1.id1, df2.id2 FROM df1 JOIN df2 ON ST_KNN(df1.geometry, df2.geometry, 1)") + println("ST_KNN(df1.geometry, df2.geometry, 1)") + var resultAll = dfResult.orderBy("id1").take(1) + resultAll.mkString should be("[0,str0]") + + dfResult = sparkSession.sql( + "SELECT df1.id1, df2.id2 FROM df1 JOIN df2 ON ST_KNN(df2.geometry, df1.geometry, 1)") + resultAll = dfResult.orderBy("id1").take(1) + resultAll.mkString should be("[0,str0]") + } + + it("KNN Join with exact algorithms based on EUCLIDEAN distance using random data") { + val queryNumRows = 200 + val objectNumRows = 10000 + val seed = 12345L + val coordRange = (-50.0, 50.0) + val (df1, df2) = generateRandomPointsDataFrames( + sparkSession, + queryNumRows, + objectNumRows, + seed, + coordRange) + + df1.repartition(numPartitions).createOrReplaceTempView("QUERIES") + df2.repartition(numPartitions).createOrReplaceTempView("OBJECTS") + + val df = sparkSession.sql( + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false)") + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(queryNumRows * 4) + } + + it("KNN Join with exact algorithms based on EUCLIDEAN distance using manually generated data") { + val points1 = Seq( + (0, "POINT(2.0 2.0)", "0"), + (1, "POINT(2.0 3.0)", "1"), + (2, "POINT(3.0 3.0)", "2"), + (3, "POINT(3.0 2.0)", "3"), + (4, "POINT(3.0 1.0)", "4"), + (5, "POINT(2.0 1.0)", "5"), + (6, "POINT(1.0 1.0)", "6"), + (7, "POINT(1.0 2.0)", "7"), + (8, "POINT(1.0 3.0)", "8"), + (9, "POINT(0.0 2.0)", "9"), + (10, "POINT(4.0 2.0)", "10")) + + val points2 = Seq( + (0, "POINT(2.0 2.0)", "0"), + (1, "POINT(2.0 3.0)", "1"), + (2, "POINT(3.0 3.0)", "2"), + (3, "POINT(3.0 2.0)", "3"), + (4, "POINT(3.0 1.0)", "4"), + (5, "POINT(2.0 1.0)", "5"), + (6, "POINT(1.0 1.0)", "6"), + (7, "POINT(1.0 2.0)", "7"), + (8, "POINT(1.0 3.0)", "8"), + (9, "POINT(0.0 2.0)", "9"), + (10, "POINT(4.0 2.0)", "10")) + + val (df1, df2) = createPointsDataFrames(sparkSession, points1, points2) + + df1.createOrReplaceTempView("QUERIES") + df2.createOrReplaceTempView("OBJECTS") + + val df = sparkSession.sql( + s"SELECT QUERIES.ID as qid, OBJECTS.ID as oid FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 5, false)") + + val df_group = + df.groupBy("qid").agg(collect_list("oid").as("collected_points")).orderBy("qid") + + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + resultAll.length should be(points1.size * 5) + } + + it("KNN Join with exact algorithms with ties exported") { + val points1 = Seq((0, "POINT(1.0 1.0)", "0")) + + val points2 = Seq( + (0, "POINT(2.0 2.0)", "0"), + (1, "POINT(2.0 3.0)", "1"), + (2, "POINT(3.0 3.0)", "2"), + (3, "POINT(3.0 2.0)", "3"), + (4, "POINT(3.0 1.0)", "4"), + (5, "POINT(2.0 1.0)", "5"), + (6, "POINT(1.0 1.0)", "6"), + (7, "POINT(1.0 2.0)", "7"), + (8, "POINT(1.0 3.0)", "8"), + (9, "POINT(0.0 2.0)", "9"), + (10, "POINT(4.0 2.0)", "10")) + + val (df1, df2) = createPointsDataFrames(sparkSession, points1, points2) + + df1.createOrReplaceTempView("QUERIES") + df2.createOrReplaceTempView("OBJECTS") + + withExportTies(export = true) { + val df = sparkSession.sql( + s"SELECT QUERIES.ID as qid, OBJECTS.ID as oid FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 2, false)") + + val df_group = + df.groupBy("qid").agg(collect_list("oid").as("collected_points")).orderBy("qid") + + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + print(resultAll.mkString) + resultAll.length should be(3) + resultAll.mkString should be("[0,5][0,6][0,7]") + } + + withExportTies(export = false) { + val df = sparkSession.sql( + s"SELECT QUERIES.ID as qid, OBJECTS.ID as oid FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 2, false)") + + val df_group = + df.groupBy("qid").agg(collect_list("oid").as("collected_points")).orderBy("qid") + + val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) + print(resultAll.mkString) + resultAll.length should be(2) + resultAll.mkString should be("[0,6][0,7]") + } + } + } + + private def withOptimizationMode(mode: String)(body: => Unit): Unit = { + withConf(Map("sedona.join.optimizationmode" -> mode))(body) + } + + private def withExportTies(export: Boolean)(body: => Unit): Unit = { + if (export) { + withConf(Map("spark.sedona.join.knn.includeTieBreakers" -> "true"))(body) + } else { + withConf(Map("spark.sedona.join.knn.includeTieBreakers" -> "false"))(body) + } + } + + private def withSpatialPartitionerExport(path: String)(body: => Unit): Unit = { + withConf(Map("spark.sedona.join.debug.spatialPartitionerSavePath" -> path))(body) + } + + def validateQueryPlan( + df: DataFrame, + numNeighbors: Int, + useApproximate: Boolean, + expressionSize: Int, + isGeography: Boolean, + mustInclude: String): Unit = { + print(df.queryExecution.executedPlan.toString) + df.queryExecution.executedPlan.toString should include("KNNJoin") + Option(mustInclude).filter(_.nonEmpty).foreach { text => + df.queryExecution.executedPlan.toString should include(text) + } + df.queryExecution.executedPlan.collect { case p: KNNJoinExec => + p.k should be(Literal(numNeighbors)) + p.isGeography should be(isGeography) + p.expressions.size should be(expressionSize) + } + } + + private def prepareTempViewsForTestData(): (DataFrame, DataFrame) = { + val df1 = sparkSession.read + .format("csv") + .option("header", "false") + .option("delimiter", testDataDelimiter) + .load(knnPointsLocationQueries) + .withColumn("id", col("_c0").cast(IntegerType)) + .withColumn("geom", ST_GeomFromText(new Column("_c1"))) + .withColumn("shape", col("_c1")) + .select("id", "geom", "shape") + val df2 = sparkSession.read + .format("csv") + .option("header", "false") + .option("delimiter", testDataDelimiter) + .load(knnPointsLocationObjects) + .withColumn("id", col("_c0").cast(IntegerType)) + .withColumn("geom", ST_GeomFromText(new Column("_c1"))) + .withColumn("shape", col("_c1")) + .select("id", "geom", "shape") + df1.createOrReplaceTempView("df1") + df2.createOrReplaceTempView("df2") + sparkSession.table("df1").repartition(numPartitions).createOrReplaceTempView("queries") + sparkSession.table("df2").repartition(numPartitions).createOrReplaceTempView("objects") + + val emptyRdd = sparkSession.sparkContext.emptyRDD[Row] + val emptyDf = sparkSession.createDataFrame( + emptyRdd, + StructType(Seq(StructField("id", IntegerType), StructField("geom", GeometryUDT)))) + emptyDf.createOrReplaceTempView("EMPTYTABLE") + + df1.createOrReplaceTempView("df1") + df2.createOrReplaceTempView("df2") + (df1, df2) + } + + private def prepareTempViewsForDifferentPartitionsTestData(): (DataFrame, DataFrame) = { + val df1 = sparkSession.read + .format("csv") + .option("header", "false") + .option("delimiter", testDataDelimiter) + .load(knnPointsLocationQueries) + .withColumn("id", col("_c0").cast(IntegerType)) + .withColumn("geom", ST_GeomFromText(new Column("_c1"))) + .withColumn("shape", col("_c1")) + .select("id", "geom", "shape") + val df2 = sparkSession.read + .format("csv") + .option("header", "false") + .option("delimiter", testDataDelimiter) + .load(knnPointsLocationObjects) + .withColumn("id", col("_c0").cast(IntegerType)) + .withColumn("geom", ST_GeomFromText(new Column("_c1"))) + .withColumn("shape", col("_c1")) + .select("id", "geom", "shape") + df1.repartition(2).createOrReplaceTempView("queries_par2") + df2.repartition(4).createOrReplaceTempView("objects_par4") + (df1, df2) + } + + private def prepareTempViewsForSkewedTestData() = { + val df1 = sparkSession.read + .format("csv") + .option("header", "false") + .option("delimiter", testDataDelimiter) + .load(knnPointsLocationSkewedObjects) + .withColumn("id", col("_c0").cast(IntegerType)) + .withColumn("geom", ST_GeomFromText(new Column("_c1"))) + .withColumn("shape", col("_c1")) + .select("id", "geom", "shape") + df1.repartition(numSkewPartitions).createOrReplaceTempView("queries_skewed") + df1.repartition(numSkewPartitions).createOrReplaceTempView("objects_skewed") + + val df2 = sparkSession.read + .format("csv") + .option("header", "false") + .option("delimiter", testDataDelimiter) + .load(knnPointsLocationMultipleSkewedObjects) + .withColumn("id", col("_c0").cast(IntegerType)) + .withColumn("geom", ST_GeomFromText(new Column("_c1"))) + .withColumn("shape", col("_c1")) + .select("id", "geom", "shape") + df2.repartition(numSkewPartitions).createOrReplaceTempView("queries_skewed_multiple") + df2.repartition(numSkewPartitions).createOrReplaceTempView("objects_skewed_multiple") + } + + def generateRandomPointsDataFrames( + sparkSession: SparkSession, + numRows1: Int, + numRows2: Int, + seed: Long, + coordRange: (Double, Double)): (DataFrame, DataFrame) = { + val schema = StructType( + Seq( + StructField("id", IntegerType, nullable = false), + StructField("geom", StringType, nullable = false), + StructField("shape", StringType, nullable = false))) + + def generateRandomData(numRows: Int, seed: Long, coordRange: (Double, Double)): Seq[Row] = { + val random = new Random(seed) + val (minCoord, maxCoord) = coordRange + (1 to numRows).map { id => + val x = + minCoord + random + .nextDouble() * (maxCoord - minCoord) // Random x coordinate in the range + val y = + minCoord + random + .nextDouble() * (maxCoord - minCoord) // Random y coordinate in the range + val geom = s"POINT($x $y)" + val shape = s"bank$id" + Row(id, geom, shape) + } + } + + val data1 = generateRandomData(numRows1, seed, coordRange) + val data2 = generateRandomData( + numRows2, + seed + 1, + coordRange + ) // Use a different seed for the second DataFrame + + val rdd1 = sparkSession.sparkContext.parallelize(data1) + val rdd2 = sparkSession.sparkContext.parallelize(data2) + + val df1 = sparkSession + .createDataFrame(rdd1, schema) + .withColumn("geom", expr("ST_GeomFromText(geom)")) + + val df2 = sparkSession + .createDataFrame(rdd2, schema) + .withColumn("geom", expr("ST_GeomFromText(geom)")) + + (df1, df2) + } + + def createPointsDataFrames( + sparkSession: SparkSession, + points1: Seq[(Int, String, String)], + points2: Seq[(Int, String, String)]): (DataFrame, DataFrame) = { + val schema = StructType( + Seq( + StructField("id", IntegerType, nullable = false), + StructField("geom", StringType, nullable = false), + StructField("shape", StringType, nullable = false))) + + def createDataFrame(points: Seq[(Int, String, String)]): DataFrame = { + val rows = points.map { case (id, geom, shape) => Row(id, geom, shape) } + val rdd = sparkSession.sparkContext.parallelize(rows) + val df = sparkSession + .createDataFrame(rdd, schema) + .withColumn("geom", expr("ST_GeomFromText(geom)")) + df + } + + val df1 = createDataFrame(points1) + val df2 = createDataFrame(points2) + + (df1, df2) + } +} From 71c35242f5b946277254fa208d7724d614ca8bf2 Mon Sep 17 00:00:00 2001 From: zhangfengcdt Date: Tue, 27 Aug 2024 10:02:12 -0700 Subject: [PATCH 2/6] add test data --- .../resources/knn/all-test-cases-aknn.csv | 21 + .../test/resources/knn/all-test-cases-knn.csv | 21 + .../common/src/test/resources/knn/objects.csv | 20 + .../knn/queries-large-skewed-multiple.csv | 199999 +++++++++++++++ .../resources/knn/queries-large-skewed.csv | 199981 ++++++++++++++ .../common/src/test/resources/knn/queries.csv | 3 + .../src/test/resources/knn/test1-objects.csv | 40 + .../src/test/resources/knn/test1-queries.csv | 1 + .../resources/knn/test1-results-p1-k3.csv | 1 + .../resources/knn/test1-results-p4-k3.csv | 1 + .../src/test/resources/knn/test10-objects.csv | 100 + .../src/test/resources/knn/test10-queries.csv | 20 + .../resources/knn/test10-results-p1-k4.csv | 20 + .../knn/test10-results-p4-k4-approximate.csv | 20 + .../resources/knn/test10-results-p4-k4.csv | 20 + .../src/test/resources/knn/test2-objects.csv | 40 + .../src/test/resources/knn/test2-queries.csv | 2 + .../resources/knn/test2-results-p1-k3.csv | 2 + .../resources/knn/test2-results-p4-k3.csv | 2 + .../src/test/resources/knn/test3-objects.csv | 40 + .../src/test/resources/knn/test3-queries.csv | 1 + .../resources/knn/test3-results-p1-k4.csv | 1 + .../resources/knn/test3-results-p4-k4.csv | 1 + .../src/test/resources/knn/test4-objects.csv | 120 + .../src/test/resources/knn/test4-queries.csv | 5 + .../resources/knn/test4-results-p1-k4.csv | 5 + .../resources/knn/test4-results-p4-k4.csv | 5 + .../src/test/resources/knn/test5-objects.csv | 94 + .../src/test/resources/knn/test5-queries.csv | 5 + .../resources/knn/test5-results-p1-k4.csv | 5 + .../resources/knn/test5-results-p4-k4.csv | 5 + .../src/test/resources/knn/test6-objects.csv | 99 + .../src/test/resources/knn/test6-queries.csv | 3 + .../resources/knn/test6-results-p1-k4.csv | 3 + .../resources/knn/test6-results-p4-k4.csv | 3 + .../src/test/resources/knn/test7-objects.csv | 40 + .../src/test/resources/knn/test7-queries.csv | 2 + .../resources/knn/test7-results-p1-k4.csv | 2 + .../resources/knn/test7-results-p4-k4.csv | 2 + .../src/test/resources/knn/test8-objects.csv | 151 + .../src/test/resources/knn/test8-queries.csv | 5 + .../resources/knn/test8-results-p1-k4.csv | 5 + .../resources/knn/test8-results-p4-k4.csv | 5 + .../src/test/resources/knn/test9-objects.csv | 100 + .../src/test/resources/knn/test9-queries.csv | 3 + .../resources/knn/test9-results-p1-k4.csv | 3 + .../resources/knn/test9-results-p4-k4.csv | 3 + .../org/apache/sedona/sql/KnnJoinSuite.scala | 133 +- 48 files changed, 401038 insertions(+), 125 deletions(-) create mode 100644 spark/common/src/test/resources/knn/all-test-cases-aknn.csv create mode 100644 spark/common/src/test/resources/knn/all-test-cases-knn.csv create mode 100644 spark/common/src/test/resources/knn/objects.csv create mode 100644 spark/common/src/test/resources/knn/queries-large-skewed-multiple.csv create mode 100644 spark/common/src/test/resources/knn/queries-large-skewed.csv create mode 100644 spark/common/src/test/resources/knn/queries.csv create mode 100644 spark/common/src/test/resources/knn/test1-objects.csv create mode 100644 spark/common/src/test/resources/knn/test1-queries.csv create mode 100644 spark/common/src/test/resources/knn/test1-results-p1-k3.csv create mode 100644 spark/common/src/test/resources/knn/test1-results-p4-k3.csv create mode 100644 spark/common/src/test/resources/knn/test10-objects.csv create mode 100644 spark/common/src/test/resources/knn/test10-queries.csv create mode 100644 spark/common/src/test/resources/knn/test10-results-p1-k4.csv create mode 100644 spark/common/src/test/resources/knn/test10-results-p4-k4-approximate.csv create mode 100644 spark/common/src/test/resources/knn/test10-results-p4-k4.csv create mode 100644 spark/common/src/test/resources/knn/test2-objects.csv create mode 100644 spark/common/src/test/resources/knn/test2-queries.csv create mode 100644 spark/common/src/test/resources/knn/test2-results-p1-k3.csv create mode 100644 spark/common/src/test/resources/knn/test2-results-p4-k3.csv create mode 100644 spark/common/src/test/resources/knn/test3-objects.csv create mode 100644 spark/common/src/test/resources/knn/test3-queries.csv create mode 100644 spark/common/src/test/resources/knn/test3-results-p1-k4.csv create mode 100644 spark/common/src/test/resources/knn/test3-results-p4-k4.csv create mode 100644 spark/common/src/test/resources/knn/test4-objects.csv create mode 100644 spark/common/src/test/resources/knn/test4-queries.csv create mode 100644 spark/common/src/test/resources/knn/test4-results-p1-k4.csv create mode 100644 spark/common/src/test/resources/knn/test4-results-p4-k4.csv create mode 100644 spark/common/src/test/resources/knn/test5-objects.csv create mode 100644 spark/common/src/test/resources/knn/test5-queries.csv create mode 100644 spark/common/src/test/resources/knn/test5-results-p1-k4.csv create mode 100644 spark/common/src/test/resources/knn/test5-results-p4-k4.csv create mode 100644 spark/common/src/test/resources/knn/test6-objects.csv create mode 100644 spark/common/src/test/resources/knn/test6-queries.csv create mode 100644 spark/common/src/test/resources/knn/test6-results-p1-k4.csv create mode 100644 spark/common/src/test/resources/knn/test6-results-p4-k4.csv create mode 100644 spark/common/src/test/resources/knn/test7-objects.csv create mode 100644 spark/common/src/test/resources/knn/test7-queries.csv create mode 100644 spark/common/src/test/resources/knn/test7-results-p1-k4.csv create mode 100644 spark/common/src/test/resources/knn/test7-results-p4-k4.csv create mode 100644 spark/common/src/test/resources/knn/test8-objects.csv create mode 100644 spark/common/src/test/resources/knn/test8-queries.csv create mode 100644 spark/common/src/test/resources/knn/test8-results-p1-k4.csv create mode 100644 spark/common/src/test/resources/knn/test8-results-p4-k4.csv create mode 100644 spark/common/src/test/resources/knn/test9-objects.csv create mode 100644 spark/common/src/test/resources/knn/test9-queries.csv create mode 100644 spark/common/src/test/resources/knn/test9-results-p1-k4.csv create mode 100644 spark/common/src/test/resources/knn/test9-results-p4-k4.csv diff --git a/spark/common/src/test/resources/knn/all-test-cases-aknn.csv b/spark/common/src/test/resources/knn/all-test-cases-aknn.csv new file mode 100644 index 0000000000..35937873c2 --- /dev/null +++ b/spark/common/src/test/resources/knn/all-test-cases-aknn.csv @@ -0,0 +1,21 @@ +id,desc,p,k,objectLocation,queryLocation,resultLocation +1,simple,1,3,test1-objects.csv,test1-queries.csv,test1-results-p1-k3.csv +1,simple,4,3,test1-objects.csv,test1-queries.csv,test1-results-p4-k3.csv +2,points on a line,1,3,test2-objects.csv,test2-queries.csv,test2-results-p1-k3.csv +2,points on a line,4,3,test2-objects.csv,test2-queries.csv,test2-results-p4-k3.csv +3,points forming a grid,1,4,test3-objects.csv,test3-queries.csv,test3-results-p1-k4.csv +3,points forming a grid,4,4,test3-objects.csv,test3-queries.csv,test3-results-p4-k4.csv +4,clustered points,1,4,test4-objects.csv,test4-queries.csv,test4-results-p1-k4.csv +4,clustered points,4,4,test4-objects.csv,test4-queries.csv,test4-results-p4-k4.csv +5,sparse points,1,4,test5-objects.csv,test5-queries.csv,test5-results-p1-k4.csv +5,sparse points,4,4,test5-objects.csv,test5-queries.csv,test5-results-p4-k4.csv +6,edge case - query at boundary,1,4,test6-objects.csv,test6-queries.csv,test6-results-p1-k4.csv +6,edge case - query at boundary,4,4,test6-objects.csv,test6-queries.csv,test6-results-p4-k4.csv +7,points forming a circle,1,4,test7-objects.csv,test7-queries.csv,test7-results-p1-k4.csv +7,points forming a circle,4,4,test7-objects.csv,test7-queries.csv,test7-results-p4-k4.csv +8,large dataset,1,4,test8-objects.csv,test8-queries.csv,test8-results-p1-k4.csv +8,large dataset,4,4,test8-objects.csv,test8-queries.csv,test8-results-p4-k4.csv +9,points with same coordinates,1,4,test9-objects.csv,test9-queries.csv,test9-results-p1-k4.csv +9,points with same coordinates,4,4,test9-objects.csv,test9-queries.csv,test9-results-p4-k4.csv +10,random points,1,4,test10-objects.csv,test10-queries.csv,test10-results-p1-k4.csv +10,random points,4,4,test10-objects.csv,test10-queries.csv,test10-results-p4-k4-approximate.csv diff --git a/spark/common/src/test/resources/knn/all-test-cases-knn.csv b/spark/common/src/test/resources/knn/all-test-cases-knn.csv new file mode 100644 index 0000000000..e05042214c --- /dev/null +++ b/spark/common/src/test/resources/knn/all-test-cases-knn.csv @@ -0,0 +1,21 @@ +id,desc,p,k,objectLocation,queryLocation,resultLocation +1,simple,1,3,test1-objects.csv,test1-queries.csv,test1-results-p1-k3.csv +1,simple,4,3,test1-objects.csv,test1-queries.csv,test1-results-p4-k3.csv +2,points on a line,1,3,test2-objects.csv,test2-queries.csv,test2-results-p1-k3.csv +2,points on a line,4,3,test2-objects.csv,test2-queries.csv,test2-results-p4-k3.csv +3,points forming a grid,1,4,test3-objects.csv,test3-queries.csv,test3-results-p1-k4.csv +3,points forming a grid,4,4,test3-objects.csv,test3-queries.csv,test3-results-p4-k4.csv +4,clustered points,1,4,test4-objects.csv,test4-queries.csv,test4-results-p1-k4.csv +4,clustered points,4,4,test4-objects.csv,test4-queries.csv,test4-results-p4-k4.csv +5,sparse points,1,4,test5-objects.csv,test5-queries.csv,test5-results-p1-k4.csv +5,sparse points,4,4,test5-objects.csv,test5-queries.csv,test5-results-p4-k4.csv +6,edge case - query at boundary,1,4,test6-objects.csv,test6-queries.csv,test6-results-p1-k4.csv +6,edge case - query at boundary,4,4,test6-objects.csv,test6-queries.csv,test6-results-p4-k4.csv +7,points forming a circle,1,4,test7-objects.csv,test7-queries.csv,test7-results-p1-k4.csv +7,points forming a circle,4,4,test7-objects.csv,test7-queries.csv,test7-results-p4-k4.csv +8,large dataset,1,4,test8-objects.csv,test8-queries.csv,test8-results-p1-k4.csv +8,large dataset,4,4,test8-objects.csv,test8-queries.csv,test8-results-p4-k4.csv +9,points with same coordinates,1,4,test9-objects.csv,test9-queries.csv,test9-results-p1-k4.csv +9,points with same coordinates,4,4,test9-objects.csv,test9-queries.csv,test9-results-p4-k4.csv +10,random points,1,4,test10-objects.csv,test10-queries.csv,test10-results-p1-k4.csv +10,random points,4,4,test10-objects.csv,test10-queries.csv,test10-results-p4-k4.csv diff --git a/spark/common/src/test/resources/knn/objects.csv b/spark/common/src/test/resources/knn/objects.csv new file mode 100644 index 0000000000..97c9ca577a --- /dev/null +++ b/spark/common/src/test/resources/knn/objects.csv @@ -0,0 +1,20 @@ +1 POINT(11 5) bank1 +2 POINT(12 1) bank2 +3 POINT(-1 -1) bank3 +4 POINT(-3 5) bank4 +5 POINT(9 8) bank5 +6 POINT(4 3) bank6 +7 POINT(-4 -5) bank7 +8 POINT(4 -2) bank8 +9 POINT(-3 1) bank9 +10 POINT(-7 3) bank10 +11 POINT(11 5) bank11 +12 POINT(12 1) bank12 +13 POINT(-1 -1) bank13 +14 POINT(-3 5) bank14 +15 POINT(9 8) bank15 +16 POINT(4 3) bank16 +17 POINT(-4 -5) bank17 +18 POINT(4 -2) bank18 +19 POINT(-3 1) bank19 +20 POINT(-7 3) bank20 diff --git a/spark/common/src/test/resources/knn/queries-large-skewed-multiple.csv b/spark/common/src/test/resources/knn/queries-large-skewed-multiple.csv new file mode 100644 index 0000000000..acbba02d68 --- /dev/null +++ b/spark/common/src/test/resources/knn/queries-large-skewed-multiple.csv @@ -0,0 +1,199999 @@ +1 POINT(0.022595324182536472 0.053784697254229547) bank1 +2 POINT(-0.05349910096236561 -0.03665250104240002) bank2 +3 POINT(0.061563385319064134 0.0683987129125235) bank3 +4 POINT(0.06638542975762729 -0.0926056184184667) bank4 +5 POINT(-0.0787555410198873 0.05639210016950988) bank5 +6 POINT(0.028922920580296718 -0.032049961958149714) bank6 +7 POINT(0.05856947170837545 -0.06769343977715465) bank7 +8 POINT(0.04406809060655842 -0.0382654266597547) bank8 +9 POINT(-0.09723979219599038 -0.056839240798397334) bank9 +10 POINT(0.06884882069486756 0.06624423897308179) bank10 +11 POINT(0.09194398284437447 0.09470166202597169) bank11 +12 POINT(0.009404870760571907 0.008263260425400307) bank12 +13 POINT(-0.05549185492264189 0.0766894811732276) bank13 +14 POINT(0.07023807711246743 -0.08286816765254619) bank14 +15 POINT(-0.041590794713538284 0.09103195153043328) bank15 +16 POINT(0.042775342872557076 0.008766257206098443) bank16 +17 POINT(-0.022018480177839764 -0.05079080729966226) bank17 +18 POINT(0.05140961794211582 -0.07429031908483807) bank18 +19 POINT(0.08619695664323754 0.003707234200437476) bank19 +20 POINT(33.15415349032327 -118.18292160207473) bank20 +21 POINT(34.20361277792822 -119.18285918043017) bank21 +22 POINT(34.93657953219207 -119.18177166895245) bank22 +23 POINT(34.12827183686043 -119.03638771616646) bank23 +24 POINT(34.20408205676455 -117.81550428221131) bank24 +25 POINT(33.15620468210017 -117.8016616597926) bank25 +26 POINT(33.45862733438772 -118.26464759112702) bank26 +27 POINT(34.35267372976836 -119.22956023629166) bank27 +28 POINT(34.91272586017282 -117.58456082700881) bank28 +29 POINT(34.96429867055911 -118.55238316874909) bank29 +30 POINT(33.88024601059685 -117.72161529297358) bank30 +31 POINT(33.16552521756212 -117.37686803281122) bank31 +32 POINT(35.00606851342597 -118.14507315596758) bank32 +33 POINT(34.16191493602971 -118.86796861612235) bank33 +34 POINT(33.869428832142965 -118.792647069425) bank34 +35 POINT(34.72389332542959 -119.08953284631677) bank35 +36 POINT(34.852463837125086 -118.81144896312102) bank36 +37 POINT(34.52062756759615 -117.67210473730981) bank37 +38 POINT(33.663561191458506 -118.08799954595666) bank38 +39 POINT(34.47550452290412 -117.79623871930674) bank39 +40 POINT(33.20070640702583 -118.53344774140461) bank40 +41 POINT(34.22421109499436 -117.67838285065913) bank41 +42 POINT(34.99598587591504 -118.13368381774805) bank42 +43 POINT(34.176079019481065 -118.33362858704048) bank43 +44 POINT(34.38845566223015 -119.00893955258097) bank44 +45 POINT(33.32509579341178 -119.18595821348265) bank45 +46 POINT(35.00694981629277 -119.04812043282841) bank46 +47 POINT(33.90205013963068 -119.08293743402099) bank47 +48 POINT(34.70279038707337 -118.75628186002204) bank48 +49 POINT(33.1787687217863 -117.71176553900459) bank49 +50 POINT(33.85725153326549 -118.68441704514534) bank50 +51 POINT(33.5185099078088 -118.266828481417) bank51 +52 POINT(33.96670133742274 -119.04116916203249) bank52 +53 POINT(33.28554727338884 -117.51106435647549) bank53 +54 POINT(33.184346314622566 -117.87602454915825) bank54 +55 POINT(34.03663787555573 -118.11003499449579) bank55 +56 POINT(33.10532752076911 -118.77626666463226) bank56 +57 POINT(34.636721763171245 -117.88481032115894) bank57 +58 POINT(33.39329660531796 -118.70669512492428) bank58 +59 POINT(33.401748645320225 -117.44088410555634) bank59 +60 POINT(33.948297556602185 -119.17075335821926) bank60 +61 POINT(33.93480255949714 -118.33690821853996) bank61 +62 POINT(34.17014432423948 -119.15446691289375) bank62 +63 POINT(33.89739528195142 -118.01622213485662) bank63 +64 POINT(34.925781700531466 -118.00582589815544) bank64 +65 POINT(33.224763110939364 -118.4194398250453) bank65 +66 POINT(34.18980829906648 -117.42817805275773) bank66 +67 POINT(33.315382801828164 -117.4851320687425) bank67 +68 POINT(35.00680937474447 -117.66981583328501) bank68 +69 POINT(34.48408649102248 -117.59127942406016) bank69 +70 POINT(33.78743444059214 -117.65629238382895) bank70 +71 POINT(34.738075393196766 -117.93052882667277) bank71 +72 POINT(34.30806622390125 -117.83521912441228) bank72 +73 POINT(34.001486288814355 -117.76150216893781) bank73 +74 POINT(34.6436952201252 -119.1790163262245) bank74 +75 POINT(33.41937465066604 -118.14496604078546) bank75 +76 POINT(34.025807798772576 -119.20710369287191) bank76 +77 POINT(33.202418605771975 -117.62239029567073) bank77 +78 POINT(33.980854186397494 -117.39576417848517) bank78 +79 POINT(35.019577458137995 -117.67894522260193) bank79 +80 POINT(33.53911341522712 -118.59083671996032) bank80 +81 POINT(33.05529780167851 -118.33395611889456) bank81 +82 POINT(33.78725637210507 -119.03295075682054) bank82 +83 POINT(33.48720559372325 -117.40061645621753) bank83 +84 POINT(34.92431208518064 -117.25452103786128) bank84 +85 POINT(34.1372493819118 -118.21504236694943) bank85 +86 POINT(34.38652710713016 -118.92167315115846) bank86 +87 POINT(33.46157945637274 -118.80991195347035) bank87 +88 POINT(34.136715086962106 -118.20218113420607) bank88 +89 POINT(34.22669269458557 -117.6629192224824) bank89 +90 POINT(34.44588445336441 -118.8233849030288) bank90 +91 POINT(33.73885467883515 -118.55056821762275) bank91 +92 POINT(33.0894967639084 -119.0966840119326) bank92 +93 POINT(34.961178873628164 -118.36117083007282) bank93 +94 POINT(34.34857243806023 -118.43336180044759) bank94 +95 POINT(33.22649146881279 -118.57799353711907) bank95 +96 POINT(33.68886748762393 -118.80482939568527) bank96 +97 POINT(34.500012154206026 -118.84411523996859) bank97 +98 POINT(34.8593218354909 -117.39035705124218) bank98 +99 POINT(33.7787084149626 -118.33366765695497) bank99 +100 POINT(34.20500387121888 -117.71271332647402) bank100 +101 POINT(34.672356614308164 -118.9365740617085) bank101 +102 POINT(35.02559347764917 -118.90499968128464) bank102 +103 POINT(34.03662524350947 -117.30674860251665) bank103 +104 POINT(33.252184754377396 -118.15209027796433) bank104 +105 POINT(34.4367401339587 -118.32715696490101) bank105 +106 POINT(33.544597742525795 -118.50652508464928) bank106 +107 POINT(33.907736806909774 -118.05803406495244) bank107 +108 POINT(34.78912806669935 -117.63087193299896) bank108 +109 POINT(34.043340316953476 -117.98346276323046) bank109 +110 POINT(34.78474520259556 -117.61927564337735) bank110 +111 POINT(33.13778047141849 -118.32852713917165) bank111 +112 POINT(33.171508447254 -118.66864961799398) bank112 +113 POINT(34.76991418763181 -117.39125107736933) bank113 +114 POINT(34.040098842455215 -118.46836423921025) bank114 +115 POINT(33.236796513404435 -117.86218446012458) bank115 +116 POINT(34.359032176632 -118.10206381520035) bank116 +117 POINT(34.47803384696083 -117.41269829391076) bank117 +118 POINT(33.214174613925756 -117.52315600542762) bank118 +119 POINT(34.982094259027875 -117.54269219072467) bank119 +120 POINT(33.997020408747595 -118.70157217810606) bank120 +121 POINT(34.21131437095322 -119.06612118426442) bank121 +122 POINT(34.28354122684011 -118.1855810741737) bank122 +123 POINT(33.97779743762396 -117.37878372104238) bank123 +124 POINT(33.73076804502931 -117.24860574089226) bank124 +125 POINT(34.36030858923623 -119.22786627496238) bank125 +126 POINT(35.011875189185865 -118.30820025602257) bank126 +127 POINT(33.632294313137145 -119.14997230561436) bank127 +128 POINT(34.7284285555698 -118.54102015640592) bank128 +129 POINT(33.919278212168294 -117.62983904531845) bank129 +130 POINT(34.922675969604896 -118.72260357499285) bank130 +131 POINT(34.53663868456079 -119.12052034217271) bank131 +132 POINT(34.44339541959649 -118.24780691008196) bank132 +133 POINT(33.438495227817846 -117.95571717944831) bank133 +134 POINT(34.81790441198313 -118.42383219178618) bank134 +135 POINT(33.66175481256906 -117.86442404408385) bank135 +136 POINT(33.19295605998576 -118.65057395382534) bank136 +137 POINT(34.06713804984364 -118.69763976055663) bank137 +138 POINT(35.02774612941771 -118.60389600729214) bank138 +139 POINT(33.25942530770159 -118.05614730539726) bank139 +140 POINT(34.98358737505196 -117.93141865362009) bank140 +141 POINT(34.67389421782685 -117.88049397569705) bank141 +142 POINT(34.44080538006807 -118.98539377257549) bank142 +143 POINT(34.94298267607151 -117.79980247174497) bank143 +144 POINT(33.3762071132415 -117.50138111919196) bank144 +145 POINT(33.47438652216806 -118.18183797761199) bank145 +146 POINT(34.36627151119673 -117.845945023688) bank146 +147 POINT(34.391851656896414 -118.2782710180671) bank147 +148 POINT(34.5622796500476 -119.23863706476551) bank148 +149 POINT(34.638044012814376 -118.68761983513352) bank149 +150 POINT(33.7692582742649 -117.26864966796627) bank150 +151 POINT(34.8088749415848 -118.35428787435197) bank151 +152 POINT(33.656243746653935 -117.85328703749035) bank152 +153 POINT(34.62056978669208 -118.49941813185602) bank153 +154 POINT(33.111844549910295 -118.33432507597222) bank154 +155 POINT(34.40946588400816 -117.85852238623184) bank155 +156 POINT(34.25351972796648 -117.31399433158569) bank156 +157 POINT(33.55264998917078 -119.23377903887283) bank157 +158 POINT(33.88878906640636 -117.44414341952357) bank158 +159 POINT(33.72534715820996 -118.12989726967751) bank159 +160 POINT(33.89096974175274 -118.95574202852148) bank160 +161 POINT(33.61634911293731 -118.320713870091) bank161 +162 POINT(33.984848917673524 -117.63636215226524) bank162 +163 POINT(34.66546555601081 -118.47981776243033) bank163 +164 POINT(34.738014843788775 -117.46325099677597) bank164 +165 POINT(34.008181972354976 -117.33570288679971) bank165 +166 POINT(33.242910125735435 -117.7568576212269) bank166 +167 POINT(34.79522985881952 -117.41800173259588) bank167 +168 POINT(34.31728546933782 -117.68868348830738) bank168 +169 POINT(34.6046063312866 -119.1915714202905) bank169 +170 POINT(34.74865623516557 -118.82469516026958) bank170 +171 POINT(34.093344880396906 -118.37471089588676) bank171 +172 POINT(34.880937083789064 -117.57701575628066) bank172 +173 POINT(34.74648394298717 -117.57720851101426) bank173 +174 POINT(34.591063932472146 -118.35954731822237) bank174 +175 POINT(34.470134697864104 -117.493787537422) bank175 +176 POINT(33.17990072712832 -118.85920034376699) bank176 +177 POINT(34.36628886784393 -117.70522729744404) bank177 +178 POINT(34.16239699449453 -118.72033579596902) bank178 +179 POINT(33.269480018860165 -117.74020309327442) bank179 +180 POINT(34.98297548739858 -119.1605005250944) bank180 +181 POINT(34.90005024959558 -118.90070098472526) bank181 +182 POINT(33.66053002476366 -119.008444554753) bank182 +183 POINT(33.4877944227984 -117.34382514593682) bank183 +184 POINT(34.39388686848407 -117.74167219328184) bank184 +185 POINT(33.63727824528998 -117.50389484973162) bank185 +186 POINT(34.302611692051485 -117.55909904183723) bank186 +187 POINT(33.54248766180134 -118.91063052511602) bank187 +188 POINT(33.78354043115822 -117.24736450425259) bank188 +189 POINT(34.96577652195821 -117.57765133299208) bank189 +190 POINT(33.29274922213153 -117.30698068311497) bank190 +191 POINT(34.76255780104802 -118.8677416265135) bank191 +192 POINT(34.8928024696961 -117.85939159928523) bank192 +193 POINT(33.06098609319912 -118.03342674059624) bank193 +194 POINT(34.09819266765751 -117.27378596967786) bank194 +195 POINT(34.455581039462295 -118.32972270255766) bank195 +196 POINT(33.05406268689032 -118.45005210109218) bank196 +197 POINT(33.52988318232049 -118.18249211342057) bank197 +198 POINT(33.13894429003219 -117.47658790376316) bank198 +199 POINT(33.91535723375989 -117.8645416403033) bank199 +200 POINT(34.50998407692436 -118.09673378563171) bank200 +201 POINT(34.36517621759714 -118.78323623957367) bank201 +202 POINT(33.29992455366239 -117.24955253516177) bank202 +203 POINT(33.28994530059063 -117.4811526754007) bank203 +204 POINT(34.44425012108187 -117.57663612676545) bank204 +205 POINT(33.682106201604896 -119.23676037373001) bank205 +206 POINT(33.19712265949253 -118.71530127030094) bank206 +207 POINT(34.09354126526033 -117.46184393557121) bank207 +208 POINT(33.37477691581308 -117.84630839324633) bank208 +209 POINT(34.23866791899242 -117.74720653986317) bank209 +210 POINT(34.13295968536232 -118.99404719885793) bank210 +211 POINT(33.96244844632511 -117.52220464252986) bank211 +212 POINT(34.278657073302625 -118.13746816617605) bank212 +213 POINT(33.49562097191976 -118.61091459000079) bank213 +214 POINT(33.788791218939444 -118.57063090955191) bank214 +215 POINT(33.50125179657698 -118.35778735898552) bank215 +216 POINT(33.08264994769807 -117.29619116752508) bank216 +217 POINT(34.930405110047644 -118.44727952980911) bank217 +218 POINT(34.26072692112566 -117.72242670666981) bank218 +219 POINT(33.270932329985094 -118.92209895298787) bank219 +220 POINT(33.21953747866623 -117.29682089820768) bank220 +221 POINT(34.132428163117936 -117.30812571761136) bank221 +222 POINT(33.38592676975883 -117.86176327786956) bank222 +223 POINT(34.38814648203785 -118.5719994358311) bank223 +224 POINT(33.575857362326204 -118.55377362996893) bank224 +225 POINT(34.11192940524259 -119.17016064790454) bank225 +226 POINT(33.68759580660514 -117.90838812148856) bank226 +227 POINT(34.111401683083116 -118.50730779730262) bank227 +228 POINT(34.63372911904594 -117.95291923599036) bank228 +229 POINT(33.30399966970109 -117.726489868802) bank229 +230 POINT(34.55154023990845 -117.53520736384009) bank230 +231 POINT(33.90006964932046 -118.82635539434864) bank231 +232 POINT(34.796064454834024 -117.50734787360749) bank232 +233 POINT(34.63446533874362 -117.26088044637164) bank233 +234 POINT(35.01594444580128 -118.51485752576714) bank234 +235 POINT(33.287862962249186 -118.66047442447707) bank235 +236 POINT(33.53961492178557 -117.92019580327805) bank236 +237 POINT(33.476484961087145 -118.3565503664301) bank237 +238 POINT(34.995127586481054 -118.32592307000616) bank238 +239 POINT(33.247169563955616 -118.3172666220638) bank239 +240 POINT(33.840653154554886 -118.32640539325045) bank240 +241 POINT(34.58016377042487 -118.923305577536) bank241 +242 POINT(34.69362927636099 -117.99119203486634) bank242 +243 POINT(34.82612581283927 -118.10732658539027) bank243 +244 POINT(33.8176631967805 -117.58150235371689) bank244 +245 POINT(33.327125747541004 -118.17096931139893) bank245 +246 POINT(34.62225717984857 -118.1252911388561) bank246 +247 POINT(34.0256908641922 -118.34675697689501) bank247 +248 POINT(33.43011971789608 -118.90058600205128) bank248 +249 POINT(33.461134157365336 -119.03149184618422) bank249 +250 POINT(34.652355477526754 -117.34629918475434) bank250 +251 POINT(34.681698313745585 -117.81271645404479) bank251 +252 POINT(33.19641592795998 -117.72944483050786) bank252 +253 POINT(33.330547483063285 -118.27600510890721) bank253 +254 POINT(34.416461596333264 -117.92725504160423) bank254 +255 POINT(34.85944120689156 -118.44801208658853) bank255 +256 POINT(34.170763329628016 -118.23947911157654) bank256 +257 POINT(34.52893643276741 -118.78437128148559) bank257 +258 POINT(33.19656079414832 -118.01227465050843) bank258 +259 POINT(34.37550667064411 -119.04673688068819) bank259 +260 POINT(33.636790015863596 -118.22039256338302) bank260 +261 POINT(34.50734100088704 -118.90345656160862) bank261 +262 POINT(34.52015611953791 -117.78484278598705) bank262 +263 POINT(33.76317596558025 -118.97291937481171) bank263 +264 POINT(33.453678675460985 -117.3643240236595) bank264 +265 POINT(33.3726732188533 -118.1657779783731) bank265 +266 POINT(33.61887386101971 -119.1881213327299) bank266 +267 POINT(33.66814240966477 -118.72846715780577) bank267 +268 POINT(34.03246152848972 -118.97298658244983) bank268 +269 POINT(34.16615808498391 -119.07814005234285) bank269 +270 POINT(34.60728496321054 -118.43719062066002) bank270 +271 POINT(34.48611892137451 -118.29554367630595) bank271 +272 POINT(34.95128968806111 -117.62110819958528) bank272 +273 POINT(34.4228602102608 -117.69851298236863) bank273 +274 POINT(34.57813168005466 -117.35261287848067) bank274 +275 POINT(34.97611506017748 -118.42534674811478) bank275 +276 POINT(33.429762326713146 -118.8873842759431) bank276 +277 POINT(33.50326792448273 -117.37754636117278) bank277 +278 POINT(33.47429234703917 -117.40533355621251) bank278 +279 POINT(34.98185897487108 -117.50075626391414) bank279 +280 POINT(34.5580842455626 -118.94064394164037) bank280 +281 POINT(34.5359206151274 -117.90255659469398) bank281 +282 POINT(34.87076289859525 -119.16718086894817) bank282 +283 POINT(34.86158475307407 -119.01590047519701) bank283 +284 POINT(34.963784087286626 -117.54774049679133) bank284 +285 POINT(33.863704233808946 -118.19345579410762) bank285 +286 POINT(33.9895592036957 -117.29305116113669) bank286 +287 POINT(34.72829371409889 -118.53978453437303) bank287 +288 POINT(34.17981477870425 -118.0896526304299) bank288 +289 POINT(34.5915050624389 -117.50783910991326) bank289 +290 POINT(34.80455502504053 -119.0831720000212) bank290 +291 POINT(33.63923812048975 -119.19452359433326) bank291 +292 POINT(33.35368335739577 -118.4939791747066) bank292 +293 POINT(33.80206465933374 -118.33377333627715) bank293 +294 POINT(34.553572373130244 -118.48882996814233) bank294 +295 POINT(34.249863418559016 -118.38645916416293) bank295 +296 POINT(34.843145305263 -117.34813962142807) bank296 +297 POINT(34.24074765848224 -119.0378971077724) bank297 +298 POINT(34.424660696111616 -117.64655094707582) bank298 +299 POINT(33.15919568843533 -118.68045368993414) bank299 +300 POINT(34.681658612068944 -118.01025020763804) bank300 +301 POINT(33.40940192672302 -117.79141134249416) bank301 +302 POINT(33.2661398533095 -118.89350380650842) bank302 +303 POINT(33.561456802966276 -117.83830617373961) bank303 +304 POINT(35.02863934509255 -117.75470759230788) bank304 +305 POINT(34.55461109837096 -117.74121836486114) bank305 +306 POINT(34.007691604245814 -118.94949420434314) bank306 +307 POINT(33.36103370625186 -118.29170763152086) bank307 +308 POINT(34.788821855307106 -117.54957762872942) bank308 +309 POINT(34.64262766045631 -118.18720461771106) bank309 +310 POINT(33.21853923589589 -117.44281549363328) bank310 +311 POINT(34.524879571978765 -117.84532538521124) bank311 +312 POINT(33.63802465199928 -117.62360282089057) bank312 +313 POINT(34.55428556276305 -117.81612611362821) bank313 +314 POINT(34.56225438901596 -118.41551772051376) bank314 +315 POINT(33.63127322700763 -118.3585768341992) bank315 +316 POINT(33.594992853148135 -117.80955677209694) bank316 +317 POINT(34.10266238293245 -118.60631542986727) bank317 +318 POINT(33.90862480836019 -117.53378049374172) bank318 +319 POINT(34.408706766682194 -117.709963360966) bank319 +320 POINT(34.10812730860734 -117.5920982714048) bank320 +321 POINT(33.88194306969572 -118.77266353521325) bank321 +322 POINT(33.076353930248075 -118.64811107280767) bank322 +323 POINT(33.74319665375589 -118.11452057631371) bank323 +324 POINT(33.31107692693581 -119.18159495821305) bank324 +325 POINT(35.04834760875421 -118.6529236689006) bank325 +326 POINT(34.689983737185315 -118.20789498236184) bank326 +327 POINT(34.39272163810169 -118.70936348436939) bank327 +328 POINT(34.24904377488066 -117.79909492795382) bank328 +329 POINT(34.48745771152259 -117.82750821874967) bank329 +330 POINT(34.531312842594915 -118.57984700992635) bank330 +331 POINT(33.11334144606805 -117.70127588207525) bank331 +332 POINT(33.64798100650644 -117.78381272082696) bank332 +333 POINT(33.31164711591338 -117.49206316245271) bank333 +334 POINT(34.11010972587047 -117.71745420515875) bank334 +335 POINT(34.243451846845524 -118.42541768080484) bank335 +336 POINT(33.09969231967897 -117.74627729511329) bank336 +337 POINT(33.249462451984165 -117.34637062037388) bank337 +338 POINT(34.55526299749562 -117.5943816219547) bank338 +339 POINT(34.49111999033587 -118.46298188033497) bank339 +340 POINT(33.6735136606115 -118.16760289144413) bank340 +341 POINT(34.303830116039414 -119.03976208709096) bank341 +342 POINT(34.46011723140074 -118.79613021475296) bank342 +343 POINT(33.27829920562642 -118.00165224072252) bank343 +344 POINT(34.72598537787181 -118.38412216900902) bank344 +345 POINT(34.51814429644989 -118.21662253964317) bank345 +346 POINT(34.99428639130382 -117.62215028708242) bank346 +347 POINT(33.220055199035734 -117.83659816505833) bank347 +348 POINT(34.21758475363223 -118.68033352453261) bank348 +349 POINT(33.88545771745952 -119.13524717501532) bank349 +350 POINT(33.224899242630464 -118.15262216927668) bank350 +351 POINT(34.58026784676319 -117.63141720287108) bank351 +352 POINT(34.33998691585144 -118.21516931995514) bank352 +353 POINT(33.73950300280251 -118.33179088315227) bank353 +354 POINT(34.79233114829202 -117.36323527727812) bank354 +355 POINT(34.67939111025548 -119.02143665453472) bank355 +356 POINT(34.29563309337886 -119.19363041362116) bank356 +357 POINT(33.632486083689095 -118.9195536880736) bank357 +358 POINT(34.1004742338614 -118.21712632981874) bank358 +359 POINT(35.037813584556744 -117.61471614613473) bank359 +360 POINT(34.24350594752116 -118.33711920517484) bank360 +361 POINT(33.15330713228883 -117.68599962510238) bank361 +362 POINT(34.44078671150964 -118.1609650642067) bank362 +363 POINT(33.91536636096218 -119.16156599973071) bank363 +364 POINT(34.310872743084666 -118.08587166094227) bank364 +365 POINT(33.53154876619463 -118.31535898399731) bank365 +366 POINT(33.375323599709226 -118.36303300051325) bank366 +367 POINT(34.77657833844026 -118.01804092373226) bank367 +368 POINT(33.89170217520549 -117.62229061833536) bank368 +369 POINT(33.405658163692536 -118.28301731620029) bank369 +370 POINT(34.94584436307634 -117.77415830636453) bank370 +371 POINT(34.631112235141025 -118.44957923016202) bank371 +372 POINT(34.245157728724585 -119.18680211625855) bank372 +373 POINT(33.96910702506154 -118.16469661397906) bank373 +374 POINT(33.31386269939317 -118.14043391779312) bank374 +375 POINT(34.861697797296 -117.34389364040914) bank375 +376 POINT(33.6930937741454 -119.08105145291073) bank376 +377 POINT(33.495945416719636 -119.2004886820056) bank377 +378 POINT(33.59211147169814 -118.86067676807508) bank378 +379 POINT(34.95613453677754 -118.10961053916708) bank379 +380 POINT(34.14418370687646 -117.3424609392066) bank380 +381 POINT(34.03599105748808 -117.89822392483319) bank381 +382 POINT(34.814117746822824 -118.17282715735661) bank382 +383 POINT(33.478278218688054 -117.93221144230293) bank383 +384 POINT(34.13571488216549 -117.6632408371744) bank384 +385 POINT(34.39905933573414 -117.29407642461697) bank385 +386 POINT(34.909362207401884 -118.67881195167013) bank386 +387 POINT(34.85330144207463 -118.23452372070912) bank387 +388 POINT(33.6468207945134 -118.87086024833258) bank388 +389 POINT(34.466346219268374 -118.09224697760118) bank389 +390 POINT(33.57615210822894 -117.85317478882304) bank390 +391 POINT(33.81509494802865 -118.42239316107151) bank391 +392 POINT(34.56246444774575 -118.836623408027) bank392 +393 POINT(34.168887608679405 -118.12567254364801) bank393 +394 POINT(34.67764526716441 -117.31685022767658) bank394 +395 POINT(34.71479285031468 -118.48367140394636) bank395 +396 POINT(33.1759100011836 -117.93620882718271) bank396 +397 POINT(34.49671728104431 -118.79054468136438) bank397 +398 POINT(34.05855410370167 -118.45065763303761) bank398 +399 POINT(33.84601922948737 -118.4511666331301) bank399 +400 POINT(34.47671480214238 -118.26932074730466) bank400 +401 POINT(33.971334908728274 -119.1045523351213) bank401 +402 POINT(33.17012240647759 -118.26154306912828) bank402 +403 POINT(34.283471756859726 -119.08220503928928) bank403 +404 POINT(33.984198088749935 -118.19068676584997) bank404 +405 POINT(33.344896033504874 -117.56492519303015) bank405 +406 POINT(33.1697553417139 -118.51077978216844) bank406 +407 POINT(33.679049129927876 -119.0535359725092) bank407 +408 POINT(34.434426625806296 -118.25284595342173) bank408 +409 POINT(33.30664771669004 -118.90324763007888) bank409 +410 POINT(33.471718275758434 -117.50125269820887) bank410 +411 POINT(34.80156597796834 -118.86299531840821) bank411 +412 POINT(34.84863242693976 -117.26421102072715) bank412 +413 POINT(34.68961119427297 -118.6695457515505) bank413 +414 POINT(34.29619882925322 -118.85329841537056) bank414 +415 POINT(34.249298453497794 -118.56145791574532) bank415 +416 POINT(34.30355031431669 -117.95195824721324) bank416 +417 POINT(33.96599801354255 -117.3000733242743) bank417 +418 POINT(34.80635021218827 -118.57104893616507) bank418 +419 POINT(33.583177874713854 -118.18338797604311) bank419 +420 POINT(34.81045959930106 -118.89569574697944) bank420 +421 POINT(34.03560474056415 -118.39559373232426) bank421 +422 POINT(33.75982345630615 -118.85483427116297) bank422 +423 POINT(34.38990593305627 -117.48665276175329) bank423 +424 POINT(33.49325690737003 -117.37121600992285) bank424 +425 POINT(34.821934671803255 -117.29998316102363) bank425 +426 POINT(33.25863434083839 -117.49983548048762) bank426 +427 POINT(34.931125190906734 -118.6152657282706) bank427 +428 POINT(33.99979126187901 -118.57414466603245) bank428 +429 POINT(33.21028717706237 -118.85917622163988) bank429 +430 POINT(34.693500986553104 -118.82064183718792) bank430 +431 POINT(34.021116330469496 -117.99844451858932) bank431 +432 POINT(33.16414734612345 -118.19137868109601) bank432 +433 POINT(33.221737047215875 -118.28026281845703) bank433 +434 POINT(34.727400326509326 -119.15864144526022) bank434 +435 POINT(34.00709042206952 -118.23601507577669) bank435 +436 POINT(33.60472254888775 -118.44506860060162) bank436 +437 POINT(34.76272837755233 -117.37408243817758) bank437 +438 POINT(34.320241645102094 -119.209847950424) bank438 +439 POINT(34.90933907385696 -118.76364376655508) bank439 +440 POINT(33.42236680218947 -119.04124430945524) bank440 +441 POINT(33.810432282881884 -118.95992731120216) bank441 +442 POINT(33.960925826443 -119.02923241025442) bank442 +443 POINT(34.68218557898035 -119.03365164904254) bank443 +444 POINT(33.38879887804321 -118.58252407544008) bank444 +445 POINT(33.81657152328775 -118.14334033659009) bank445 +446 POINT(33.82465761483059 -117.28918332345438) bank446 +447 POINT(34.40549434303294 -117.9504304893021) bank447 +448 POINT(33.44551253249891 -118.86360863724691) bank448 +449 POINT(34.55422541276901 -117.96631222364131) bank449 +450 POINT(33.2907707517695 -117.98651189351817) bank450 +451 POINT(34.01841362114887 -118.40575332847752) bank451 +452 POINT(34.471304811806704 -117.95407297434981) bank452 +453 POINT(34.10353380908646 -118.02599017833319) bank453 +454 POINT(34.514177233410486 -118.71535386668697) bank454 +455 POINT(33.80788605525418 -119.05458000083802) bank455 +456 POINT(33.19364074978302 -118.96971726640288) bank456 +457 POINT(35.01658278634422 -118.89607434010111) bank457 +458 POINT(33.16608001492548 -118.3691775186419) bank458 +459 POINT(33.14845988494116 -118.78313122765083) bank459 +460 POINT(34.859756423239155 -118.05493156792735) bank460 +461 POINT(34.33907603750837 -117.36359157936913) bank461 +462 POINT(33.46102576701512 -118.06764759230731) bank462 +463 POINT(34.11644441959319 -117.9981499735255) bank463 +464 POINT(34.90359397320109 -118.21970099874771) bank464 +465 POINT(33.721511924929345 -117.47823880066711) bank465 +466 POINT(33.26749476714027 -118.73781286927701) bank466 +467 POINT(33.27081449452346 -117.40600363071972) bank467 +468 POINT(34.04097107510017 -117.26360610607243) bank468 +469 POINT(34.5127437570966 -117.30976246088794) bank469 +470 POINT(33.63150988863197 -117.35740786511565) bank470 +471 POINT(33.26801934464865 -117.52471460239647) bank471 +472 POINT(33.602494062859265 -118.06144036312124) bank472 +473 POINT(34.07431923909811 -119.15860155047544) bank473 +474 POINT(34.91536643496623 -117.350756627377) bank474 +475 POINT(34.536906233903785 -117.56404726149094) bank475 +476 POINT(33.59294245777176 -118.72125918039676) bank476 +477 POINT(34.99911092241725 -117.87804926036718) bank477 +478 POINT(35.01860133863871 -118.20035602162739) bank478 +479 POINT(33.967254192771456 -117.76312464247741) bank479 +480 POINT(33.13359888708505 -118.1836872181922) bank480 +481 POINT(33.92169505602856 -117.97769430418766) bank481 +482 POINT(33.11096446194098 -117.41170988439505) bank482 +483 POINT(33.36682782714324 -118.90438172294587) bank483 +484 POINT(33.882191365918096 -118.73868812628612) bank484 +485 POINT(34.59811807526736 -117.7178799193971) bank485 +486 POINT(34.441465373973806 -117.6029693496956) bank486 +487 POINT(34.363895024721735 -118.74310572718015) bank487 +488 POINT(34.67245581968341 -118.62665357771897) bank488 +489 POINT(34.25886978007367 -118.35972094709916) bank489 +490 POINT(33.654976593406715 -118.64400351319745) bank490 +491 POINT(33.17818616903553 -117.79289199579931) bank491 +492 POINT(34.44425324388894 -118.69638245123893) bank492 +493 POINT(33.46907511027505 -118.19411077976419) bank493 +494 POINT(34.5772907267781 -117.64252541793891) bank494 +495 POINT(33.118887452980665 -118.70008290171417) bank495 +496 POINT(34.425836262211554 -117.46042855516026) bank496 +497 POINT(33.79269593376895 -117.70401692772214) bank497 +498 POINT(33.785792162552625 -118.01438924657205) bank498 +499 POINT(34.56421192835745 -118.88534832100939) bank499 +500 POINT(33.116437117560764 -119.20205610297256) bank500 +501 POINT(34.82572796133863 -117.27700415223865) bank501 +502 POINT(33.07525420705973 -118.22500829157093) bank502 +503 POINT(33.23337519632383 -118.28488618845928) bank503 +504 POINT(33.61837739357081 -117.72776203706735) bank504 +505 POINT(33.36352032159903 -118.17241028143297) bank505 +506 POINT(33.41114777115969 -117.27536421305705) bank506 +507 POINT(34.47484056669721 -117.55540617885781) bank507 +508 POINT(34.11265586180328 -117.33749723894564) bank508 +509 POINT(33.58963688173185 -117.45365913330399) bank509 +510 POINT(34.94588125772054 -117.44658682850032) bank510 +511 POINT(33.21827305108004 -117.62867063032301) bank511 +512 POINT(33.85734144817475 -117.91591581284185) bank512 +513 POINT(34.84747660583309 -117.45901315336785) bank513 +514 POINT(33.74173545029387 -118.93185162726408) bank514 +515 POINT(34.15208041087693 -117.6662369318598) bank515 +516 POINT(34.007190250508266 -119.0571056525903) bank516 +517 POINT(33.17197638934148 -118.87298609935208) bank517 +518 POINT(34.074127296256705 -118.33086602148595) bank518 +519 POINT(33.62153725367843 -118.44444961207112) bank519 +520 POINT(34.1934201120416 -118.5986177608812) bank520 +521 POINT(33.42261710867332 -117.96001364929765) bank521 +522 POINT(33.30827886653537 -118.30776478937626) bank522 +523 POINT(33.39095598606075 -118.87943002257107) bank523 +524 POINT(34.38948183988677 -117.69955194633754) bank524 +525 POINT(34.22854917027446 -118.71188911810471) bank525 +526 POINT(33.572785320752075 -117.36915667040073) bank526 +527 POINT(33.385339309742186 -117.67080286148544) bank527 +528 POINT(34.616605033936466 -118.39085767320157) bank528 +529 POINT(34.38122130720842 -118.36263132702274) bank529 +530 POINT(34.56543116081089 -117.51575661168616) bank530 +531 POINT(35.025779646534 -117.36857258762613) bank531 +532 POINT(34.870173114132754 -118.02271327166291) bank532 +533 POINT(33.62007165146805 -118.6179998513109) bank533 +534 POINT(33.38025590566169 -119.12577433846472) bank534 +535 POINT(33.336942945386376 -119.04870279936381) bank535 +536 POINT(34.8239601202837 -117.8795578103927) bank536 +537 POINT(34.23802497303286 -118.57134775001862) bank537 +538 POINT(34.45086163191671 -117.70731404642693) bank538 +539 POINT(34.58013824088237 -118.74913674806898) bank539 +540 POINT(33.83299445773505 -118.02281609398926) bank540 +541 POINT(33.91647201464856 -118.87406448083358) bank541 +542 POINT(33.61430489684048 -118.24594852793159) bank542 +543 POINT(33.913289808863 -117.99713263900401) bank543 +544 POINT(34.710467118523866 -118.52660473992745) bank544 +545 POINT(34.71925591873619 -118.7970559483232) bank545 +546 POINT(34.09096843184419 -118.07412126297605) bank546 +547 POINT(33.78653550318693 -117.58551765798154) bank547 +548 POINT(33.06305724329765 -118.85022291326565) bank548 +549 POINT(34.76408556701492 -118.52007051340162) bank549 +550 POINT(33.52544121482634 -118.7995742948883) bank550 +551 POINT(34.285261717431915 -117.64790605231832) bank551 +552 POINT(33.521841082009374 -118.55875436589368) bank552 +553 POINT(34.70419859021943 -117.71379119849861) bank553 +554 POINT(34.266049148912536 -118.56908585910146) bank554 +555 POINT(34.89046864274968 -118.90788879434525) bank555 +556 POINT(34.728962888407146 -118.29132074569915) bank556 +557 POINT(33.315584611099545 -117.92573279000419) bank557 +558 POINT(33.94506641667053 -119.14826399713475) bank558 +559 POINT(34.93975306275532 -118.36159601131884) bank559 +560 POINT(33.39754145550607 -118.406642639403) bank560 +561 POINT(34.22347074326997 -118.35528830912529) bank561 +562 POINT(33.82439676180165 -118.15507382694348) bank562 +563 POINT(34.45523740069854 -117.9492517589456) bank563 +564 POINT(33.84037724311888 -118.5143568822855) bank564 +565 POINT(34.86553595972518 -118.13225051732556) bank565 +566 POINT(33.90817269104225 -117.44644535756026) bank566 +567 POINT(33.661950068333255 -117.3611517476499) bank567 +568 POINT(33.102128784554694 -118.99962969612612) bank568 +569 POINT(34.45927060592851 -119.11019665075756) bank569 +570 POINT(34.46687001787456 -118.3611610316599) bank570 +571 POINT(33.153593025037516 -118.28866280801768) bank571 +572 POINT(34.08865379625335 -118.92322897828235) bank572 +573 POINT(33.31058319696662 -119.09201911678862) bank573 +574 POINT(33.909393964305295 -118.55744546575451) bank574 +575 POINT(34.11424341820633 -118.44376301455044) bank575 +576 POINT(33.843071495340425 -118.51785426459377) bank576 +577 POINT(33.1934069099587 -117.348113796799) bank577 +578 POINT(34.04094191280043 -118.15818474800724) bank578 +579 POINT(34.14335490956015 -118.5543450596975) bank579 +580 POINT(33.94470531304965 -117.32346147543157) bank580 +581 POINT(34.79040205382375 -118.88479499207516) bank581 +582 POINT(33.58637668538071 -118.38483581789365) bank582 +583 POINT(33.34283315993734 -118.60394211052486) bank583 +584 POINT(33.06157400447867 -117.80111282937045) bank584 +585 POINT(34.67815651816166 -117.3948429768365) bank585 +586 POINT(33.4633461913463 -118.35946010031032) bank586 +587 POINT(34.84837613865958 -118.04239500214275) bank587 +588 POINT(33.2456186473309 -117.90814002949998) bank588 +589 POINT(33.25997574276287 -117.53081305616143) bank589 +590 POINT(33.53648266455941 -117.57587831315442) bank590 +591 POINT(34.38734444888823 -117.86467278722672) bank591 +592 POINT(35.03208685095457 -117.37314959115848) bank592 +593 POINT(33.56008543628681 -117.89839995207913) bank593 +594 POINT(33.8927248634251 -118.55368546037337) bank594 +595 POINT(33.365665502796425 -117.64395015999484) bank595 +596 POINT(33.46451114425627 -117.27634554411854) bank596 +597 POINT(33.95506420218658 -118.01005505284355) bank597 +598 POINT(34.28970082831526 -118.42464481844259) bank598 +599 POINT(33.36287066219828 -118.85547391686967) bank599 +600 POINT(33.716503906580904 -119.00866589176528) bank600 +601 POINT(33.21115355649663 -118.660807854468) bank601 +602 POINT(34.166442753462256 -118.93040426316887) bank602 +603 POINT(34.31900321497417 -118.917924165727) bank603 +604 POINT(33.8778216541196 -118.76835149577936) bank604 +605 POINT(34.55038538124724 -117.84518175296077) bank605 +606 POINT(34.130577262826286 -118.09280366586499) bank606 +607 POINT(34.918119451050146 -118.26644382753706) bank607 +608 POINT(34.13145952326055 -118.07621147564947) bank608 +609 POINT(33.15583040191204 -117.82717549472912) bank609 +610 POINT(34.25890941065584 -119.1722266959477) bank610 +611 POINT(34.7447451172069 -118.3060677354553) bank611 +612 POINT(33.19027966612566 -118.15349960077624) bank612 +613 POINT(33.187464618799275 -117.31072911540679) bank613 +614 POINT(34.118262140053716 -119.13183408510015) bank614 +615 POINT(33.935552422428024 -118.86453909165246) bank615 +616 POINT(34.56075942450924 -118.10234909094572) bank616 +617 POINT(34.385041013724695 -118.40756613217194) bank617 +618 POINT(34.473759571363225 -117.48555713294068) bank618 +619 POINT(33.16325594853225 -117.79670896890522) bank619 +620 POINT(33.36564921412534 -117.71085520977324) bank620 +621 POINT(34.79618100321805 -117.69041022562864) bank621 +622 POINT(33.494277964701254 -119.21676626322598) bank622 +623 POINT(33.66764144853548 -117.5498230948642) bank623 +624 POINT(34.75511391953094 -118.20813563748338) bank624 +625 POINT(34.339139147914786 -119.02636790689809) bank625 +626 POINT(33.4186112898946 -118.68293237949396) bank626 +627 POINT(34.970311111006666 -118.10143356867808) bank627 +628 POINT(34.42427823012768 -117.73797618965557) bank628 +629 POINT(33.139246431829754 -118.95479628406868) bank629 +630 POINT(34.280382948849066 -117.93220759506808) bank630 +631 POINT(34.916449491042556 -118.12271683363797) bank631 +632 POINT(34.803176272823514 -117.88477625065339) bank632 +633 POINT(35.05134907334709 -118.48299958421407) bank633 +634 POINT(33.618837438358256 -117.79124594960068) bank634 +635 POINT(34.362164922804325 -117.64654139594751) bank635 +636 POINT(34.503273981393654 -117.60417832378165) bank636 +637 POINT(34.784108197287324 -117.27457960002198) bank637 +638 POINT(34.60140930746701 -117.62098709358354) bank638 +639 POINT(33.14418603236837 -117.89144651404673) bank639 +640 POINT(33.43071739951443 -118.19651967085112) bank640 +641 POINT(34.97386229789053 -118.38004515418085) bank641 +642 POINT(34.016478732387306 -117.88182755523766) bank642 +643 POINT(33.980294119962 -117.83304876277732) bank643 +644 POINT(34.13520159755149 -118.69028429305796) bank644 +645 POINT(33.98164082878372 -117.33515356854079) bank645 +646 POINT(33.36572974547484 -118.76046278880877) bank646 +647 POINT(33.089626524161815 -119.09504146584855) bank647 +648 POINT(33.24012033063096 -118.24449184645526) bank648 +649 POINT(34.10257494080216 -118.23379357076232) bank649 +650 POINT(34.42301459759602 -118.1645862148817) bank650 +651 POINT(34.221989225555085 -117.5331020309462) bank651 +652 POINT(33.985794162181556 -117.38576026084313) bank652 +653 POINT(34.20211308518378 -117.5842650357608) bank653 +654 POINT(33.58665164638121 -118.15150688936542) bank654 +655 POINT(33.39674940828527 -117.5983124254304) bank655 +656 POINT(34.4966444886125 -118.72638974481234) bank656 +657 POINT(33.52617501632551 -118.66914919408676) bank657 +658 POINT(35.02394930247174 -117.74352274358827) bank658 +659 POINT(33.66932908402606 -118.5575389352164) bank659 +660 POINT(33.48835640050977 -118.28068829976594) bank660 +661 POINT(33.39052891942702 -118.50865218528936) bank661 +662 POINT(33.65898820669062 -117.28805189917001) bank662 +663 POINT(34.534220647475316 -117.55501697146012) bank663 +664 POINT(33.611820617670574 -117.85697973360733) bank664 +665 POINT(34.954611186961145 -117.66505782898011) bank665 +666 POINT(33.87964804121816 -118.25403793368933) bank666 +667 POINT(33.293415988272876 -117.6958227557202) bank667 +668 POINT(33.654852766320786 -118.93081466484152) bank668 +669 POINT(35.03963771686106 -119.04884029738429) bank669 +670 POINT(33.95740180343943 -119.0619540403445) bank670 +671 POINT(34.95983164347932 -117.72768655683242) bank671 +672 POINT(33.2782002143566 -118.51085427419052) bank672 +673 POINT(34.99139500127213 -118.10344865105996) bank673 +674 POINT(34.268280320829646 -118.0453621827262) bank674 +675 POINT(34.446319946473096 -119.06005727949018) bank675 +676 POINT(34.618589601389075 -119.15312309455636) bank676 +677 POINT(34.03149071329432 -118.86388811395248) bank677 +678 POINT(33.58033537314076 -118.74136493122379) bank678 +679 POINT(34.24988412439185 -117.31037740912251) bank679 +680 POINT(34.518635733836106 -118.90109921233712) bank680 +681 POINT(34.95265049045138 -119.12236262096306) bank681 +682 POINT(34.33044897801199 -118.75698538886101) bank682 +683 POINT(34.83482881492055 -118.02059306572136) bank683 +684 POINT(34.84582380009666 -117.56502567936812) bank684 +685 POINT(34.50234733017034 -118.7021908740035) bank685 +686 POINT(35.01942202652485 -118.483923215574) bank686 +687 POINT(33.090932211889076 -117.8992913465924) bank687 +688 POINT(34.79380546153393 -118.27617729588954) bank688 +689 POINT(33.54528005422287 -118.36307725480489) bank689 +690 POINT(33.57493413429616 -117.57918768133844) bank690 +691 POINT(34.3434967436695 -118.96579655100598) bank691 +692 POINT(34.4661970605081 -118.91494232228659) bank692 +693 POINT(35.02015236813339 -118.24385423408476) bank693 +694 POINT(34.46239985689351 -117.70901343529843) bank694 +695 POINT(34.285622197713444 -118.65146484264812) bank695 +696 POINT(34.313856531140495 -117.66395015060507) bank696 +697 POINT(33.07039212906621 -118.53596289632547) bank697 +698 POINT(33.280738646965695 -117.54261912156544) bank698 +699 POINT(33.07946941622639 -117.64738891156647) bank699 +700 POINT(33.57331196002572 -117.63525048111148) bank700 +701 POINT(33.14902074018128 -118.71190051228342) bank701 +702 POINT(33.269332415798615 -118.62714325288047) bank702 +703 POINT(33.065480460709864 -118.68880971247832) bank703 +704 POINT(33.12804593873548 -117.58269107585458) bank704 +705 POINT(34.77928184920011 -118.05856317393363) bank705 +706 POINT(33.67738678943069 -118.65975889222464) bank706 +707 POINT(33.215034755179374 -118.59875184123725) bank707 +708 POINT(33.81753073092455 -118.75927056586843) bank708 +709 POINT(34.767818194245386 -118.87014909061527) bank709 +710 POINT(34.555770907104055 -118.94478343196954) bank710 +711 POINT(33.48523049148223 -118.07260735378253) bank711 +712 POINT(34.376762058683596 -118.25319635115036) bank712 +713 POINT(33.69641354824992 -118.20734299327447) bank713 +714 POINT(33.155090480239174 -119.08670126922539) bank714 +715 POINT(34.021830415345036 -117.95987989526006) bank715 +716 POINT(34.772520277649654 -117.6138444024205) bank716 +717 POINT(35.050024490014216 -117.8704451526816) bank717 +718 POINT(34.559487548859465 -117.2904311206284) bank718 +719 POINT(33.423951814568646 -118.78907849292467) bank719 +720 POINT(33.941945879753135 -118.28426762599724) bank720 +721 POINT(34.70043075719717 -118.82246002446766) bank721 +722 POINT(33.80286748544269 -117.88232889785182) bank722 +723 POINT(34.78847622280058 -118.26599734689765) bank723 +724 POINT(33.124370838596846 -118.04625226173039) bank724 +725 POINT(34.24562914920192 -119.00387548353004) bank725 +726 POINT(34.92589667897927 -117.46370108530438) bank726 +727 POINT(33.178203051624145 -118.86486867749257) bank727 +728 POINT(33.32366095545222 -117.64845440599755) bank728 +729 POINT(34.61131268818235 -117.36743736721715) bank729 +730 POINT(33.36326359620327 -118.76624585929814) bank730 +731 POINT(34.491778106758325 -118.57361986896277) bank731 +732 POINT(35.04965429158906 -118.1205326122157) bank732 +733 POINT(33.77818606202472 -118.43364862544323) bank733 +734 POINT(33.13606714311955 -118.71558306323497) bank734 +735 POINT(33.29920684919554 -118.91337272274654) bank735 +736 POINT(33.83964938434112 -119.00079888579066) bank736 +737 POINT(33.83043586136898 -119.17935621007692) bank737 +738 POINT(34.81100989711957 -117.58017140794121) bank738 +739 POINT(34.0010504870688 -119.15526340057033) bank739 +740 POINT(33.428276296634564 -118.16772754627156) bank740 +741 POINT(33.40658960461491 -118.54146830052714) bank741 +742 POINT(33.14266204931114 -118.16081516245859) bank742 +743 POINT(34.02335035355787 -117.6463327597836) bank743 +744 POINT(33.35286228101573 -118.62364020521986) bank744 +745 POINT(33.668684440619714 -118.05571819970633) bank745 +746 POINT(34.15460163919201 -119.16082812779732) bank746 +747 POINT(33.91094314324166 -118.21253404768707) bank747 +748 POINT(34.37521959724532 -118.81292241314884) bank748 +749 POINT(33.08646968402747 -118.88100923770997) bank749 +750 POINT(34.12661920110986 -117.64243987749302) bank750 +751 POINT(34.63088645589007 -117.59057727839014) bank751 +752 POINT(34.559494931208 -117.896497237346) bank752 +753 POINT(33.65493692311324 -118.70549876957472) bank753 +754 POINT(33.54695470643632 -117.87623698964703) bank754 +755 POINT(33.07158982116635 -117.3335246284741) bank755 +756 POINT(33.495555800076055 -119.13403510702544) bank756 +757 POINT(33.135472479321784 -118.61809699077928) bank757 +758 POINT(33.31600842256542 -117.54162289579999) bank758 +759 POINT(33.09282644871178 -117.6529757613639) bank759 +760 POINT(33.33925935815919 -117.76940245462045) bank760 +761 POINT(34.12476344308687 -117.53033788985131) bank761 +762 POINT(33.813055654300946 -118.4988908373302) bank762 +763 POINT(34.413787434262744 -118.99847774953605) bank763 +764 POINT(33.15948692204391 -119.15251864257363) bank764 +765 POINT(33.76699209880414 -118.82020629414936) bank765 +766 POINT(34.52852184729195 -118.19446420928945) bank766 +767 POINT(33.62123533621143 -117.36836443252761) bank767 +768 POINT(34.30115082129616 -118.37744282549951) bank768 +769 POINT(34.2481445255748 -118.31822133189341) bank769 +770 POINT(34.2978091465141 -118.99817872779752) bank770 +771 POINT(34.1290787060839 -118.92731822952408) bank771 +772 POINT(34.379065140864704 -117.9310805179732) bank772 +773 POINT(34.16044069685866 -117.62558040845359) bank773 +774 POINT(34.67875259827059 -117.96179627590482) bank774 +775 POINT(35.037404592967164 -118.72611764781728) bank775 +776 POINT(34.63116107635758 -118.79168163620328) bank776 +777 POINT(33.7359192266891 -117.54980172207257) bank777 +778 POINT(34.06628856237921 -117.9465320604628) bank778 +779 POINT(34.16361558122623 -118.1040314638852) bank779 +780 POINT(33.56723365355275 -118.5714371403987) bank780 +781 POINT(34.71900914629538 -117.74534160478161) bank781 +782 POINT(34.91436813083338 -118.65056595077765) bank782 +783 POINT(34.85398695617622 -118.53339862934412) bank783 +784 POINT(33.94388224822646 -117.9118924031545) bank784 +785 POINT(33.243192936295515 -118.76854586125516) bank785 +786 POINT(34.12816546646601 -118.8949664128035) bank786 +787 POINT(34.37173057697739 -117.66418606011153) bank787 +788 POINT(34.16679135539639 -119.21332769170873) bank788 +789 POINT(33.345685565250434 -118.8494030497592) bank789 +790 POINT(34.4879958159786 -117.43417295222167) bank790 +791 POINT(34.55115516444607 -117.36718665741745) bank791 +792 POINT(33.212687500380206 -119.17669878796578) bank792 +793 POINT(33.893835701757126 -117.26357232673062) bank793 +794 POINT(33.21005036351411 -118.01707631207405) bank794 +795 POINT(34.72057269611199 -117.92769980895184) bank795 +796 POINT(33.87769691428678 -118.0075239941327) bank796 +797 POINT(33.09584508388686 -118.7781592414172) bank797 +798 POINT(33.47393621105459 -118.86004434342831) bank798 +799 POINT(34.527801356452585 -118.254978401191) bank799 +800 POINT(34.001925699231606 -119.0208632451371) bank800 +801 POINT(33.95430699644003 -118.28462431910343) bank801 +802 POINT(33.84912564898491 -117.67743583356467) bank802 +803 POINT(34.966553827503404 -117.99919613237743) bank803 +804 POINT(34.70783379820906 -118.12509554817164) bank804 +805 POINT(34.97555537458317 -117.8831163879699) bank805 +806 POINT(33.3839105770626 -118.95316034054542) bank806 +807 POINT(34.043317212539606 -118.30212270855343) bank807 +808 POINT(33.16331427518189 -117.70587382633107) bank808 +809 POINT(33.64034464869485 -118.15153165155789) bank809 +810 POINT(34.39957806946149 -117.29446549214433) bank810 +811 POINT(33.108372407607234 -117.61386835675613) bank811 +812 POINT(33.48097237255948 -118.36268986440132) bank812 +813 POINT(34.96981076728102 -117.96948809588537) bank813 +814 POINT(33.522387719159916 -117.5286848661903) bank814 +815 POINT(33.41890822555791 -117.83877178970012) bank815 +816 POINT(33.742797621011746 -117.35945182857209) bank816 +817 POINT(35.03338146307587 -117.3929793228522) bank817 +818 POINT(33.160501286452714 -118.62241780593956) bank818 +819 POINT(33.6225506419734 -118.3155682071325) bank819 +820 POINT(34.58118262301749 -118.09641448828714) bank820 +821 POINT(34.174431757107385 -118.78375090818352) bank821 +822 POINT(33.15342482004438 -118.66172253713069) bank822 +823 POINT(34.45676856208124 -118.09598901409645) bank823 +824 POINT(34.443763324563 -118.40442944343987) bank824 +825 POINT(34.07934296747253 -118.1194392724674) bank825 +826 POINT(33.550022832596575 -118.4006093817429) bank826 +827 POINT(34.517014553021724 -118.57437178243939) bank827 +828 POINT(35.00240288503065 -117.6795488240829) bank828 +829 POINT(34.93602750106357 -117.91695280532258) bank829 +830 POINT(34.45695301292958 -119.04008579421539) bank830 +831 POINT(33.69916849085977 -117.99504448070289) bank831 +832 POINT(34.4255495047297 -119.0474059802587) bank832 +833 POINT(33.52924732716279 -118.40375921513878) bank833 +834 POINT(33.41308753980211 -118.60674160922227) bank834 +835 POINT(34.12878528642926 -117.80967224027287) bank835 +836 POINT(33.35091988397577 -118.52516078978364) bank836 +837 POINT(34.634428351089625 -118.11913699033396) bank837 +838 POINT(33.678377026918156 -118.73862996497168) bank838 +839 POINT(34.386040129023485 -117.89061145271288) bank839 +840 POINT(34.00905756696776 -118.55581660283116) bank840 +841 POINT(34.8491592248231 -117.25904389651569) bank841 +842 POINT(33.161544382645396 -119.20521315198079) bank842 +843 POINT(34.83436640739089 -118.92759495488811) bank843 +844 POINT(34.94793269720987 -118.4067097061267) bank844 +845 POINT(33.07786858501501 -117.74879831044247) bank845 +846 POINT(33.649917573765336 -118.01747359387747) bank846 +847 POINT(33.29315016179447 -118.91608966488933) bank847 +848 POINT(33.86276506659628 -117.81463636158244) bank848 +849 POINT(33.78098538057525 -118.98035863678543) bank849 +850 POINT(34.87127056845587 -117.25501139450199) bank850 +851 POINT(33.23852036889585 -118.18926525428056) bank851 +852 POINT(34.622097028128664 -118.73503531389773) bank852 +853 POINT(33.851675512603315 -117.48468554589039) bank853 +854 POINT(34.31184229928954 -118.48908001505025) bank854 +855 POINT(34.180006253230054 -117.33232657250707) bank855 +856 POINT(33.89782560487957 -117.9429626738331) bank856 +857 POINT(34.14311170040683 -118.76275787023533) bank857 +858 POINT(34.29711453256287 -117.82188745018973) bank858 +859 POINT(33.77496103852896 -118.73255671921616) bank859 +860 POINT(33.67458432575382 -117.30421952097689) bank860 +861 POINT(33.275236017382724 -118.37030655756045) bank861 +862 POINT(34.36366241366411 -118.13929500848803) bank862 +863 POINT(33.72577211537474 -118.79281313467067) bank863 +864 POINT(34.13283879352633 -117.50674380248434) bank864 +865 POINT(35.014535422358 -119.06098700291938) bank865 +866 POINT(33.6062532527715 -119.0239907820806) bank866 +867 POINT(34.55382741108959 -117.62006106267376) bank867 +868 POINT(34.25496472755369 -117.60083643240748) bank868 +869 POINT(33.7005697124188 -117.46096710565342) bank869 +870 POINT(33.4217642342074 -118.59505859420834) bank870 +871 POINT(33.957745345815724 -117.71752305546619) bank871 +872 POINT(34.51053607191949 -117.79212105370001) bank872 +873 POINT(34.93845489188213 -118.63471083379639) bank873 +874 POINT(33.568212673942035 -118.25972760524776) bank874 +875 POINT(34.11197611923729 -119.01963119514014) bank875 +876 POINT(33.665311350435374 -117.99775421325961) bank876 +877 POINT(33.853899279250705 -117.37154542544341) bank877 +878 POINT(34.776580443832515 -118.11047058242356) bank878 +879 POINT(34.59750901489441 -118.22397510242386) bank879 +880 POINT(34.47062439993424 -118.28146528097497) bank880 +881 POINT(34.45479898555056 -118.40561030853326) bank881 +882 POINT(33.746491493240086 -119.08306168406038) bank882 +883 POINT(34.81135656520422 -118.94668002671047) bank883 +884 POINT(34.948753839083295 -119.05020562124737) bank884 +885 POINT(34.25131442176546 -118.78555557026937) bank885 +886 POINT(33.309893202927306 -117.41561202969349) bank886 +887 POINT(34.76162265960077 -118.80703741549688) bank887 +888 POINT(34.020300559333975 -119.0381328631568) bank888 +889 POINT(34.39344489634792 -118.54632466818381) bank889 +890 POINT(33.96177529888536 -117.83718170153105) bank890 +891 POINT(33.573187045754686 -118.1436916054076) bank891 +892 POINT(34.87665629909993 -118.41839356634235) bank892 +893 POINT(33.891303224063364 -118.89433696629001) bank893 +894 POINT(34.73869473717661 -119.06360951351523) bank894 +895 POINT(34.50167818018621 -118.75391794180015) bank895 +896 POINT(34.79338867254489 -118.88969554323668) bank896 +897 POINT(34.66012098303864 -118.45074148380247) bank897 +898 POINT(33.81462733326577 -117.63133056344337) bank898 +899 POINT(34.685079580255454 -118.92668346649073) bank899 +900 POINT(33.47149472684973 -117.7794996576451) bank900 +901 POINT(33.73154627900467 -118.6327310919945) bank901 +902 POINT(34.661347330552886 -118.35479479319574) bank902 +903 POINT(34.61853936170321 -117.3907935859131) bank903 +904 POINT(34.2371932739536 -118.67363305729867) bank904 +905 POINT(34.92931103037862 -119.06058604756889) bank905 +906 POINT(34.3023851307356 -117.44118725300865) bank906 +907 POINT(34.706669724149386 -117.84004684994325) bank907 +908 POINT(33.765608863106465 -117.63323445687237) bank908 +909 POINT(34.79600403904087 -118.83465257523423) bank909 +910 POINT(34.24694737949007 -118.06570324152197) bank910 +911 POINT(34.30490571985664 -119.15555520853236) bank911 +912 POINT(33.98544297837867 -117.80383746856732) bank912 +913 POINT(34.92642498417515 -117.71807132461178) bank913 +914 POINT(34.835434916957965 -117.4600424873831) bank914 +915 POINT(33.560142014516785 -117.69102798178874) bank915 +916 POINT(33.44820325575331 -118.63608693936781) bank916 +917 POINT(33.25720650009715 -118.21692570503667) bank917 +918 POINT(33.991817749824754 -119.21492413456555) bank918 +919 POINT(33.34450839910491 -119.12466953909934) bank919 +920 POINT(34.84982552824483 -119.0555964296174) bank920 +921 POINT(34.697190434858655 -117.6123588623234) bank921 +922 POINT(34.84280759614215 -118.10628252281556) bank922 +923 POINT(34.3662779984256 -117.38671873292719) bank923 +924 POINT(33.41776342420577 -118.41023693671424) bank924 +925 POINT(33.106059055697 -118.20559063643204) bank925 +926 POINT(34.00908669347262 -117.37192150609923) bank926 +927 POINT(35.01366332918468 -118.42885594613546) bank927 +928 POINT(33.14056444759341 -118.04998635755965) bank928 +929 POINT(33.71444078024636 -117.5971133555162) bank929 +930 POINT(34.47961189014102 -118.64133816203685) bank930 +931 POINT(33.44803158479276 -118.1683458883012) bank931 +932 POINT(34.229799195890855 -117.46663104150436) bank932 +933 POINT(33.7116562313908 -117.5590176097232) bank933 +934 POINT(33.39817157438949 -117.94598159912657) bank934 +935 POINT(33.451689032583275 -117.83702936723823) bank935 +936 POINT(33.75693925965329 -119.21256946976395) bank936 +937 POINT(33.71536023951626 -118.9475529197198) bank937 +938 POINT(33.08025290686823 -117.36339664688319) bank938 +939 POINT(33.08716134737412 -118.01680162616542) bank939 +940 POINT(34.44176883603759 -118.83021734768649) bank940 +941 POINT(34.58327010771927 -118.46285898835367) bank941 +942 POINT(34.417715246136275 -117.36023504795988) bank942 +943 POINT(34.528405657740436 -118.23687005252005) bank943 +944 POINT(34.45040596425524 -117.87921354717672) bank944 +945 POINT(35.04451092589295 -118.81928035590357) bank945 +946 POINT(33.626913997751174 -117.38657600341203) bank946 +947 POINT(34.755383850218706 -117.4125001354238) bank947 +948 POINT(34.99641736435479 -118.84140569884062) bank948 +949 POINT(33.127709532196455 -118.78068819095418) bank949 +950 POINT(33.753930667322365 -118.15957379176717) bank950 +951 POINT(34.52217235569644 -118.9299390510236) bank951 +952 POINT(33.62522679550214 -117.45973368327452) bank952 +953 POINT(33.33025821074808 -117.77958372899299) bank953 +954 POINT(33.44012953818839 -119.1409394530993) bank954 +955 POINT(33.3829209143822 -119.02052470466231) bank955 +956 POINT(34.77648263951965 -117.79284167821386) bank956 +957 POINT(34.96307601903927 -118.81827556974994) bank957 +958 POINT(33.3602856486694 -117.37172946841393) bank958 +959 POINT(33.478310190944576 -118.2729916458879) bank959 +960 POINT(35.044566898259106 -119.0387409261439) bank960 +961 POINT(33.788590031308566 -118.66134170332326) bank961 +962 POINT(33.64455173253874 -117.50602728160938) bank962 +963 POINT(33.97503765555328 -117.59953272935282) bank963 +964 POINT(33.31623168728345 -117.93812756485822) bank964 +965 POINT(34.50548429735004 -119.03277035809315) bank965 +966 POINT(34.09512004090875 -117.72218293329882) bank966 +967 POINT(34.5424107302029 -117.33256344286987) bank967 +968 POINT(34.48178421743126 -117.56844024283276) bank968 +969 POINT(33.14852635686552 -119.16838553567607) bank969 +970 POINT(34.92529662556447 -118.96929081315126) bank970 +971 POINT(34.43674856789948 -118.82392877476896) bank971 +972 POINT(34.24738500325697 -118.25590181336615) bank972 +973 POINT(34.63985678951593 -117.47433203801843) bank973 +974 POINT(33.26265463846346 -117.7861743607901) bank974 +975 POINT(34.14413646004483 -118.14792204345893) bank975 +976 POINT(34.77325825050321 -118.37077300420688) bank976 +977 POINT(34.63703641703811 -118.83969055837122) bank977 +978 POINT(33.33337159209499 -118.80402016516693) bank978 +979 POINT(34.593608567512604 -118.6726741774579) bank979 +980 POINT(34.77137348451454 -119.05905346441348) bank980 +981 POINT(34.144550672361056 -117.92311958418692) bank981 +982 POINT(34.51196331369068 -119.20642996476947) bank982 +983 POINT(34.44341780068569 -117.7842439845975) bank983 +984 POINT(33.84658068907019 -117.84846020858367) bank984 +985 POINT(33.91474123156051 -117.25736546184027) bank985 +986 POINT(33.65990122989361 -118.3379421133645) bank986 +987 POINT(35.02481920114969 -117.5361009257024) bank987 +988 POINT(33.588564930976446 -117.47949607303927) bank988 +989 POINT(34.58056001270022 -117.47691812897477) bank989 +990 POINT(33.69134929217081 -118.37134301490637) bank990 +991 POINT(33.173870442550154 -118.47843989909187) bank991 +992 POINT(34.89322405730936 -118.38897997484918) bank992 +993 POINT(34.268605453292864 -118.12819548868895) bank993 +994 POINT(34.221260809572776 -118.02778705627729) bank994 +995 POINT(33.38172510429797 -118.60649022608305) bank995 +996 POINT(33.2617476432977 -118.55666122406262) bank996 +997 POINT(34.9399606059743 -117.56606586899908) bank997 +998 POINT(33.214581153676505 -118.04388408060615) bank998 +999 POINT(34.251027960987095 -118.20870307657063) bank999 +1000 POINT(33.080460575314234 -117.98280210956125) bank1000 +1001 POINT(33.19858816602141 -117.68019359467685) bank1001 +1002 POINT(34.29506714504654 -119.03667601883447) bank1002 +1003 POINT(33.5651399447266 -119.1743026149356) bank1003 +1004 POINT(33.219804171302954 -119.1534957146169) bank1004 +1005 POINT(33.80211037261004 -118.7723735489548) bank1005 +1006 POINT(33.69174342984591 -117.48005908951511) bank1006 +1007 POINT(33.25308932877313 -118.73754990662202) bank1007 +1008 POINT(33.19090948268952 -118.64170468117757) bank1008 +1009 POINT(34.332896617228535 -118.74669542723657) bank1009 +1010 POINT(34.58588953290265 -118.70735976838337) bank1010 +1011 POINT(34.83732259865625 -118.7617801462331) bank1011 +1012 POINT(34.65022952646477 -117.38083030122512) bank1012 +1013 POINT(33.8046030346783 -118.48354378637248) bank1013 +1014 POINT(33.59897954272374 -117.56518086555076) bank1014 +1015 POINT(33.32681409490613 -117.46138780784936) bank1015 +1016 POINT(34.79013780671877 -117.7354114151785) bank1016 +1017 POINT(34.64826552582123 -118.58701120797912) bank1017 +1018 POINT(33.80853323386713 -117.96793339781712) bank1018 +1019 POINT(34.55475713779777 -119.17809775620621) bank1019 +1020 POINT(33.73126837129927 -118.85278436118035) bank1020 +1021 POINT(34.66861187955598 -119.06691311579031) bank1021 +1022 POINT(34.066317394097766 -118.7932701921249) bank1022 +1023 POINT(33.23769082354868 -118.70437451560878) bank1023 +1024 POINT(33.43774426356647 -118.65541531537586) bank1024 +1025 POINT(34.0339887075195 -117.89677443193874) bank1025 +1026 POINT(34.65388765866375 -118.55350558834039) bank1026 +1027 POINT(33.70445612475165 -118.94773447249469) bank1027 +1028 POINT(33.51184898219276 -117.7353688089285) bank1028 +1029 POINT(33.17702227615611 -117.61299439430375) bank1029 +1030 POINT(33.109220580818885 -118.91237554093212) bank1030 +1031 POINT(35.012627605868936 -118.67246527741287) bank1031 +1032 POINT(33.52409586146459 -117.9585082332942) bank1032 +1033 POINT(34.056717027878314 -118.30262653622547) bank1033 +1034 POINT(34.393131268295946 -117.35206420863958) bank1034 +1035 POINT(34.01555440538989 -119.18137674382463) bank1035 +1036 POINT(33.126468752021516 -119.02230088090818) bank1036 +1037 POINT(34.731243793274864 -117.2499980341986) bank1037 +1038 POINT(34.66912060658397 -119.13646972519986) bank1038 +1039 POINT(34.03502870572017 -118.12512439004706) bank1039 +1040 POINT(34.899777240795025 -117.87080042833327) bank1040 +1041 POINT(33.70394374547319 -118.68200442815095) bank1041 +1042 POINT(33.53127689742396 -118.68113686481658) bank1042 +1043 POINT(34.13023490241386 -118.75282652183037) bank1043 +1044 POINT(33.150538225015055 -118.38868638240731) bank1044 +1045 POINT(34.81791915497068 -119.02184584603704) bank1045 +1046 POINT(34.88170252187996 -118.58609292614342) bank1046 +1047 POINT(33.27240208408388 -118.0232664483856) bank1047 +1048 POINT(33.76999978757511 -119.09464052739197) bank1048 +1049 POINT(34.85021316996655 -117.74113128272833) bank1049 +1050 POINT(33.091947003300845 -117.94027949401165) bank1050 +1051 POINT(33.23039594884086 -117.93485088328946) bank1051 +1052 POINT(34.67395119542083 -117.82266870773134) bank1052 +1053 POINT(34.10164509737522 -118.710915150817) bank1053 +1054 POINT(34.27865977478159 -117.42749222743491) bank1054 +1055 POINT(33.610763016800874 -119.0737467815335) bank1055 +1056 POINT(33.775331278015585 -118.45126271015519) bank1056 +1057 POINT(34.92771109017849 -117.88586839892407) bank1057 +1058 POINT(34.540644637980236 -117.8838329722606) bank1058 +1059 POINT(33.37559389230785 -117.36618413378514) bank1059 +1060 POINT(34.20346041862597 -119.20196207954753) bank1060 +1061 POINT(33.4530495267963 -119.05035539743155) bank1061 +1062 POINT(34.01647557027404 -117.31221432010692) bank1062 +1063 POINT(34.01251331181885 -117.3262974873873) bank1063 +1064 POINT(34.56923535663861 -117.30583265669274) bank1064 +1065 POINT(33.640378804180536 -119.2431232248685) bank1065 +1066 POINT(35.03252691300332 -117.46375136473404) bank1066 +1067 POINT(34.617435061609434 -118.01806843525168) bank1067 +1068 POINT(33.46990533731443 -118.98582983544314) bank1068 +1069 POINT(33.09826337442978 -117.43455429937023) bank1069 +1070 POINT(33.27439631775678 -118.05764630909786) bank1070 +1071 POINT(33.46874628200809 -118.09863694438712) bank1071 +1072 POINT(33.6473689008163 -118.92736315491473) bank1072 +1073 POINT(34.59475228187378 -118.46410784185254) bank1073 +1074 POINT(34.93129932839751 -118.10865581348912) bank1074 +1075 POINT(33.617840508983846 -117.76241088594323) bank1075 +1076 POINT(34.77498257170271 -118.86382187188053) bank1076 +1077 POINT(34.95089209047783 -117.33473640120748) bank1077 +1078 POINT(33.67343109873578 -118.05952203432324) bank1078 +1079 POINT(34.59280914846088 -118.9412502599773) bank1079 +1080 POINT(34.16883552533178 -117.41801555444675) bank1080 +1081 POINT(33.353567185512844 -118.1151649718521) bank1081 +1082 POINT(34.35932686770413 -118.48550814174105) bank1082 +1083 POINT(34.06196690264829 -118.9187750758131) bank1083 +1084 POINT(33.498480523307556 -117.28049611124007) bank1084 +1085 POINT(33.26138663529523 -118.49162220560225) bank1085 +1086 POINT(33.852788950398136 -118.02884811094275) bank1086 +1087 POINT(33.070505285134026 -119.23257157363086) bank1087 +1088 POINT(33.73849155701083 -118.49942495019936) bank1088 +1089 POINT(33.280102852699216 -119.20582243603317) bank1089 +1090 POINT(33.557692358995396 -119.16183906010232) bank1090 +1091 POINT(33.9463342126617 -117.90768723119346) bank1091 +1092 POINT(33.49175311499857 -118.76454769825645) bank1092 +1093 POINT(33.15409901269362 -118.70258800383648) bank1093 +1094 POINT(34.175917887337334 -118.1043329992395) bank1094 +1095 POINT(34.23564182642548 -118.64000082795593) bank1095 +1096 POINT(34.30377295924855 -118.67817835505919) bank1096 +1097 POINT(34.23512456055296 -118.22118803534151) bank1097 +1098 POINT(33.646510108553734 -118.37305292494904) bank1098 +1099 POINT(33.25945882012597 -117.35529746645543) bank1099 +1100 POINT(34.69619375444023 -118.06251860036868) bank1100 +1101 POINT(34.57952910910915 -117.56346471520582) bank1101 +1102 POINT(33.87112142668756 -118.86301996883971) bank1102 +1103 POINT(34.06288412887263 -117.3854009595705) bank1103 +1104 POINT(34.98056838532805 -117.26374242318079) bank1104 +1105 POINT(33.53334458028302 -117.88388388965042) bank1105 +1106 POINT(33.43271564240942 -118.44674672870278) bank1106 +1107 POINT(34.072990075297 -118.10724415758929) bank1107 +1108 POINT(34.791649690076646 -117.54927862722086) bank1108 +1109 POINT(34.0322541506217 -118.20883081372516) bank1109 +1110 POINT(34.042229811996684 -117.8163012991099) bank1110 +1111 POINT(35.00096786693711 -119.24272788462791) bank1111 +1112 POINT(34.418273376852575 -118.7813498708287) bank1112 +1113 POINT(35.034346221205574 -118.45933115445995) bank1113 +1114 POINT(33.779578815130236 -117.59787572270558) bank1114 +1115 POINT(34.0597177969441 -117.52033451587221) bank1115 +1116 POINT(33.33680754468397 -118.67290690728115) bank1116 +1117 POINT(34.60053585282116 -117.26462224206692) bank1117 +1118 POINT(33.56238076088358 -117.57613540494403) bank1118 +1119 POINT(34.21395596435595 -118.96391075952815) bank1119 +1120 POINT(34.66104613684674 -118.2714592096358) bank1120 +1121 POINT(33.203844211028226 -117.37707294089786) bank1121 +1122 POINT(33.121318491042736 -117.39356737604119) bank1122 +1123 POINT(33.63218791175843 -118.51673155960701) bank1123 +1124 POINT(33.600829386817324 -117.56780648246858) bank1124 +1125 POINT(33.812335408682216 -118.39253135931236) bank1125 +1126 POINT(33.95767963788379 -118.57499185052507) bank1126 +1127 POINT(34.43197034728773 -118.04315373492904) bank1127 +1128 POINT(35.00816896443667 -118.67291156651285) bank1128 +1129 POINT(33.15299851940214 -119.14617520333405) bank1129 +1130 POINT(34.73139283310702 -119.18832256007981) bank1130 +1131 POINT(33.9139323637327 -118.93855688905371) bank1131 +1132 POINT(34.188278518057146 -119.10516729480601) bank1132 +1133 POINT(34.16129656920954 -117.98851209019203) bank1133 +1134 POINT(34.91421424730026 -119.0675971792001) bank1134 +1135 POINT(34.51499523418188 -118.46292254461693) bank1135 +1136 POINT(34.34325555491982 -118.16903822418283) bank1136 +1137 POINT(34.05652915638318 -117.4159631971208) bank1137 +1138 POINT(34.98329278022558 -117.27814994026669) bank1138 +1139 POINT(33.704422091293445 -118.78084776718214) bank1139 +1140 POINT(34.522566625727606 -118.91304549482676) bank1140 +1141 POINT(34.29341662437022 -117.80947954267032) bank1141 +1142 POINT(33.42902343619933 -117.88271749121238) bank1142 +1143 POINT(34.747515371943386 -118.16224224866365) bank1143 +1144 POINT(33.91167724619716 -117.53108528957878) bank1144 +1145 POINT(33.873959298904325 -118.68491159256041) bank1145 +1146 POINT(34.41613125282488 -118.81043345068515) bank1146 +1147 POINT(33.63631002491422 -118.19666066300015) bank1147 +1148 POINT(34.18439575749257 -117.70706084266455) bank1148 +1149 POINT(34.69747539560614 -117.8794348274265) bank1149 +1150 POINT(33.21242777568631 -117.28457139733591) bank1150 +1151 POINT(34.36569699696049 -117.79829225781747) bank1151 +1152 POINT(34.13516623007138 -118.02815084543799) bank1152 +1153 POINT(34.3695349047132 -117.24786952826786) bank1153 +1154 POINT(34.62930109815795 -118.55657967226705) bank1154 +1155 POINT(34.48065432709873 -117.27062238079193) bank1155 +1156 POINT(33.63183673478467 -118.32859348392486) bank1156 +1157 POINT(33.9008410628572 -117.48564495349059) bank1157 +1158 POINT(34.8277065905159 -119.04861998789981) bank1158 +1159 POINT(33.3270318058626 -118.11469240693917) bank1159 +1160 POINT(33.08982164438459 -118.36705394616166) bank1160 +1161 POINT(34.660956129711835 -119.21306616399784) bank1161 +1162 POINT(34.13513280071671 -117.8071606962736) bank1162 +1163 POINT(34.164505536689894 -118.2066791608277) bank1163 +1164 POINT(33.90112154844624 -119.08200088605507) bank1164 +1165 POINT(33.456281047024135 -117.96155288605361) bank1165 +1166 POINT(34.89309788888494 -117.61720961757398) bank1166 +1167 POINT(34.147486016390374 -119.17040389891095) bank1167 +1168 POINT(34.515429504649305 -118.75311453504223) bank1168 +1169 POINT(34.27684875081822 -118.34734400345194) bank1169 +1170 POINT(33.649021338541246 -118.13815742115472) bank1170 +1171 POINT(34.74951591155997 -118.75625377478617) bank1171 +1172 POINT(33.575585022753856 -118.89110233721993) bank1172 +1173 POINT(33.8442848655626 -118.30468460377637) bank1173 +1174 POINT(34.968897130052426 -118.09796925435863) bank1174 +1175 POINT(34.09602530943927 -118.10796337954753) bank1175 +1176 POINT(34.6888200286145 -117.33408382447718) bank1176 +1177 POINT(33.84946531201368 -118.90293727944575) bank1177 +1178 POINT(33.342803342298005 -117.49437900775372) bank1178 +1179 POINT(33.73986784133322 -118.27125765630906) bank1179 +1180 POINT(33.16083280458317 -119.0861823348606) bank1180 +1181 POINT(34.40078210260036 -118.118315578391) bank1181 +1182 POINT(34.40942254881758 -119.07777505582425) bank1182 +1183 POINT(34.06543553422192 -118.26607400597936) bank1183 +1184 POINT(33.475521191411936 -118.95342170297732) bank1184 +1185 POINT(34.78972486021208 -117.53176626305277) bank1185 +1186 POINT(34.84865535990528 -119.04429605993361) bank1186 +1187 POINT(33.68150818826348 -119.0211060008838) bank1187 +1188 POINT(34.69678900562824 -118.09123367546279) bank1188 +1189 POINT(33.9779799378588 -117.37411855200872) bank1189 +1190 POINT(33.39246089874466 -118.44563989360516) bank1190 +1191 POINT(34.871014700894186 -117.88968476647652) bank1191 +1192 POINT(33.15925240835739 -118.24473553764933) bank1192 +1193 POINT(33.38476886556142 -117.43551542499739) bank1193 +1194 POINT(34.83736174676477 -117.63448391650739) bank1194 +1195 POINT(33.96866955761396 -117.30395967946234) bank1195 +1196 POINT(33.54107562568636 -118.11984619859935) bank1196 +1197 POINT(34.539652651220976 -118.23020012638986) bank1197 +1198 POINT(34.75922894622672 -118.10732901104116) bank1198 +1199 POINT(33.369686448737504 -119.01746984108554) bank1199 +1200 POINT(34.31652415485645 -118.43484805750012) bank1200 +1201 POINT(34.13784611664408 -118.60697146544871) bank1201 +1202 POINT(33.337823083699064 -118.5371641637078) bank1202 +1203 POINT(34.33434636518365 -117.92755568047829) bank1203 +1204 POINT(33.66883825645224 -118.68797215272059) bank1204 +1205 POINT(34.442101012807505 -117.95981879986635) bank1205 +1206 POINT(33.81027245065769 -117.71431084602862) bank1206 +1207 POINT(34.831474803364586 -118.85693252541722) bank1207 +1208 POINT(33.15374386295323 -118.47143045885593) bank1208 +1209 POINT(34.29497804183747 -118.65105211433993) bank1209 +1210 POINT(34.09283206671262 -118.11913239673153) bank1210 +1211 POINT(33.839123084517546 -117.32193329203164) bank1211 +1212 POINT(33.74504485188711 -117.99754806689711) bank1212 +1213 POINT(33.68769194377303 -118.7032416889818) bank1213 +1214 POINT(34.62895541721552 -118.9636474550442) bank1214 +1215 POINT(34.50589537255503 -117.35966073400232) bank1215 +1216 POINT(34.482643927197635 -117.72485225312836) bank1216 +1217 POINT(34.43826907763714 -118.10206677021858) bank1217 +1218 POINT(33.44903396118265 -117.60595222708318) bank1218 +1219 POINT(33.67845688914074 -118.77955688550193) bank1219 +1220 POINT(34.491366808252224 -117.85173654474859) bank1220 +1221 POINT(34.48703755333115 -119.15085359419052) bank1221 +1222 POINT(34.57854252100568 -117.6979940853201) bank1222 +1223 POINT(34.60549265187673 -119.19026424902168) bank1223 +1224 POINT(34.43892861463732 -118.19718063218524) bank1224 +1225 POINT(33.94564663932276 -117.54740670193931) bank1225 +1226 POINT(34.97407761565364 -118.1353938548284) bank1226 +1227 POINT(34.24634268576076 -117.90740118173697) bank1227 +1228 POINT(34.176155908263496 -117.63626405023729) bank1228 +1229 POINT(34.44787465671835 -118.45691784617486) bank1229 +1230 POINT(34.96207221914906 -118.70081819553573) bank1230 +1231 POINT(34.496402773006466 -118.63194617087132) bank1231 +1232 POINT(33.21509082840739 -117.2520176474181) bank1232 +1233 POINT(33.63437197675052 -117.27408628429592) bank1233 +1234 POINT(34.88485494221082 -117.49537467709534) bank1234 +1235 POINT(34.338547791900886 -118.887744509621) bank1235 +1236 POINT(33.15611029585996 -118.56560309844104) bank1236 +1237 POINT(33.36688694420617 -118.37661529392409) bank1237 +1238 POINT(33.49169351099499 -118.47517360727618) bank1238 +1239 POINT(34.841033964150846 -119.1747229704817) bank1239 +1240 POINT(34.60375692381127 -119.17803538299022) bank1240 +1241 POINT(34.54758795982847 -119.07405709495053) bank1241 +1242 POINT(33.32061335544685 -118.11149921945837) bank1242 +1243 POINT(34.28824014592055 -118.14935645446292) bank1243 +1244 POINT(34.31111439550191 -117.95888358433352) bank1244 +1245 POINT(33.21258643434397 -118.12147878479387) bank1245 +1246 POINT(34.263882679118026 -118.69687680611372) bank1246 +1247 POINT(34.311829181150195 -118.41767487669489) bank1247 +1248 POINT(33.75043040040244 -118.06559807331654) bank1248 +1249 POINT(33.84236115743297 -118.80773667018367) bank1249 +1250 POINT(34.20583794121059 -118.24575954567884) bank1250 +1251 POINT(34.74457781175839 -117.98791587652906) bank1251 +1252 POINT(33.40509701233874 -118.22259113221982) bank1252 +1253 POINT(33.966042733084095 -118.59347366017091) bank1253 +1254 POINT(34.258402256226766 -118.42437862854887) bank1254 +1255 POINT(35.045469569491345 -118.89139450288623) bank1255 +1256 POINT(33.884697582732095 -117.86845275023008) bank1256 +1257 POINT(34.718250188002024 -118.58695974194815) bank1257 +1258 POINT(33.6996256027398 -118.50418830395836) bank1258 +1259 POINT(33.63144784467964 -118.23341261187936) bank1259 +1260 POINT(34.947035867931255 -119.03461499535189) bank1260 +1261 POINT(34.89029029284799 -117.63825314676322) bank1261 +1262 POINT(34.80579857147277 -118.38068441818668) bank1262 +1263 POINT(34.786573758260175 -117.8206923914262) bank1263 +1264 POINT(34.24258924184228 -117.35480078412388) bank1264 +1265 POINT(35.00428009148765 -117.87129093921531) bank1265 +1266 POINT(34.9694160003929 -118.45703759919988) bank1266 +1267 POINT(33.59616034103162 -118.42374270654642) bank1267 +1268 POINT(34.25018423538893 -118.9816960392865) bank1268 +1269 POINT(33.6761369282608 -118.83466269070739) bank1269 +1270 POINT(34.53388632080739 -118.31498148939015) bank1270 +1271 POINT(33.06675127044904 -118.5461875223701) bank1271 +1272 POINT(33.84529015119778 -118.87201201900974) bank1272 +1273 POINT(35.018950560207955 -117.50043215797487) bank1273 +1274 POINT(33.855647769115244 -117.32918671927874) bank1274 +1275 POINT(33.99135873713021 -118.38125470503574) bank1275 +1276 POINT(33.29854791828596 -118.56875130435456) bank1276 +1277 POINT(33.79377959504907 -118.30216629573381) bank1277 +1278 POINT(33.6413885139295 -117.65206846188516) bank1278 +1279 POINT(34.48624072139427 -118.81275767196401) bank1279 +1280 POINT(33.39327346237189 -117.41171919944782) bank1280 +1281 POINT(33.40779480266727 -118.6997441652956) bank1281 +1282 POINT(33.525801256022085 -118.6709076653358) bank1282 +1283 POINT(34.530270180901596 -118.19977927086619) bank1283 +1284 POINT(34.03558621777427 -118.71053486096967) bank1284 +1285 POINT(33.17189066992206 -118.47553095724308) bank1285 +1286 POINT(34.197167624059965 -119.12863675816249) bank1286 +1287 POINT(34.58643374824445 -117.74524952274479) bank1287 +1288 POINT(33.521229467910665 -118.48136245374128) bank1288 +1289 POINT(34.756209947687786 -118.62095856139688) bank1289 +1290 POINT(34.39412947748962 -117.78710227265508) bank1290 +1291 POINT(34.86379654810849 -118.31619859774833) bank1291 +1292 POINT(34.068764590471574 -119.13234913054326) bank1292 +1293 POINT(34.902144063622664 -117.74863178768418) bank1293 +1294 POINT(34.564946470655634 -117.9355785022276) bank1294 +1295 POINT(34.21788774486314 -117.86302661289278) bank1295 +1296 POINT(33.064949800103136 -119.20122119452625) bank1296 +1297 POINT(33.140582352050544 -117.26447416412836) bank1297 +1298 POINT(33.586066570624524 -118.48447786586628) bank1298 +1299 POINT(33.150083708072046 -118.8573041470656) bank1299 +1300 POINT(33.17944405508713 -118.82151589681357) bank1300 +1301 POINT(34.717762790273106 -117.54173122581136) bank1301 +1302 POINT(33.20535119473409 -117.38488503823707) bank1302 +1303 POINT(33.318909536804064 -118.08899235823908) bank1303 +1304 POINT(33.21491422166922 -119.02283427018583) bank1304 +1305 POINT(33.25783669107537 -117.61656643125497) bank1305 +1306 POINT(34.91236671035675 -118.5728584949862) bank1306 +1307 POINT(33.7121800024178 -117.7478375746607) bank1307 +1308 POINT(33.25206111058127 -118.44804223621904) bank1308 +1309 POINT(33.50068402430514 -119.00083134560187) bank1309 +1310 POINT(33.3452519776309 -119.15652038758776) bank1310 +1311 POINT(33.84890092861119 -118.34513815582658) bank1311 +1312 POINT(33.467541217839894 -117.83580290468046) bank1312 +1313 POINT(33.30202225203821 -117.64957278324458) bank1313 +1314 POINT(34.786213387047205 -118.65097174457495) bank1314 +1315 POINT(33.76790026081442 -119.18863008086434) bank1315 +1316 POINT(34.663920224415655 -117.91313887961753) bank1316 +1317 POINT(34.6306374379098 -117.97830011818826) bank1317 +1318 POINT(35.03105996579251 -118.59598649782994) bank1318 +1319 POINT(34.89126577768902 -118.89260727894755) bank1319 +1320 POINT(34.14841457508528 -119.17758189671137) bank1320 +1321 POINT(33.45527276548442 -117.6509590355107) bank1321 +1322 POINT(34.31483255474911 -117.36417717844614) bank1322 +1323 POINT(34.83310647396638 -117.79620679788285) bank1323 +1324 POINT(33.43745711830092 -117.48559099611802) bank1324 +1325 POINT(34.93323821545114 -117.44109103131768) bank1325 +1326 POINT(33.22676037735031 -117.27229162487728) bank1326 +1327 POINT(33.86054272402475 -119.15490881296834) bank1327 +1328 POINT(34.58982136474072 -119.20914073543088) bank1328 +1329 POINT(33.579683513602475 -117.84225848530838) bank1329 +1330 POINT(33.44351844643933 -118.39055986052365) bank1330 +1331 POINT(34.19208151872794 -117.56170880400869) bank1331 +1332 POINT(35.04094738378465 -117.65895619899823) bank1332 +1333 POINT(34.2692215959285 -118.73742595245864) bank1333 +1334 POINT(33.61974565065729 -117.46231672364722) bank1334 +1335 POINT(35.007307995369 -117.39909227837465) bank1335 +1336 POINT(33.587172427042965 -117.36347919310266) bank1336 +1337 POINT(34.576853297668414 -118.61268941928219) bank1337 +1338 POINT(34.99384409406861 -118.38487461455824) bank1338 +1339 POINT(34.88612964431252 -118.90956955973566) bank1339 +1340 POINT(33.28836573526296 -118.34895257253986) bank1340 +1341 POINT(33.074371325002346 -119.07336174146853) bank1341 +1342 POINT(33.77829547572783 -117.57403938378584) bank1342 +1343 POINT(33.38063238354106 -118.80791589272371) bank1343 +1344 POINT(33.689975482356544 -118.35009949362501) bank1344 +1345 POINT(33.81325299121324 -118.7459609971737) bank1345 +1346 POINT(33.06311241170852 -117.86082042139502) bank1346 +1347 POINT(33.422177528860715 -118.54754235834116) bank1347 +1348 POINT(33.97955384355081 -118.52933408893007) bank1348 +1349 POINT(34.023658507228184 -118.05938873642815) bank1349 +1350 POINT(34.8847418476289 -118.3081468539974) bank1350 +1351 POINT(34.71247094061967 -118.84697788629164) bank1351 +1352 POINT(34.254260700604654 -117.9752198819087) bank1352 +1353 POINT(34.928220912764566 -117.35186214416312) bank1353 +1354 POINT(33.10766422655679 -118.5411252641701) bank1354 +1355 POINT(33.829587445580415 -117.57302661376201) bank1355 +1356 POINT(34.68066051753095 -118.92295352319437) bank1356 +1357 POINT(34.08850952159365 -119.22632131278652) bank1357 +1358 POINT(33.7839040729645 -118.00279249009833) bank1358 +1359 POINT(33.410927985917056 -117.26547136709861) bank1359 +1360 POINT(35.040800963158105 -118.8023197997134) bank1360 +1361 POINT(33.42935042011356 -117.40140906132736) bank1361 +1362 POINT(33.593091109985124 -118.29392879676621) bank1362 +1363 POINT(34.89628880142173 -118.85566213668831) bank1363 +1364 POINT(34.05968314618574 -118.672704247325) bank1364 +1365 POINT(34.115099914269315 -117.37927453375048) bank1365 +1366 POINT(33.36297026514841 -117.79991862437812) bank1366 +1367 POINT(33.98175930809728 -119.15365728618836) bank1367 +1368 POINT(34.801331037796665 -118.7761647770706) bank1368 +1369 POINT(33.5917895662056 -117.99571575012172) bank1369 +1370 POINT(34.93651925119601 -118.58075396602237) bank1370 +1371 POINT(33.961471957611444 -118.22812642095266) bank1371 +1372 POINT(34.3370929596654 -117.281049078087) bank1372 +1373 POINT(33.34395774325537 -117.90261309627037) bank1373 +1374 POINT(34.20002871691985 -117.63436621894488) bank1374 +1375 POINT(33.14733614970254 -117.57795464151862) bank1375 +1376 POINT(34.29094471695124 -118.78288174196439) bank1376 +1377 POINT(35.015867900966924 -117.44135795748232) bank1377 +1378 POINT(33.684686936272676 -118.70522145598784) bank1378 +1379 POINT(34.958330727600554 -118.5482457556981) bank1379 +1380 POINT(34.56587412582072 -118.32864083770104) bank1380 +1381 POINT(33.06055422888897 -117.81664632781964) bank1381 +1382 POINT(34.11096210630566 -117.81837329761196) bank1382 +1383 POINT(35.05044321201545 -119.06446861484368) bank1383 +1384 POINT(34.769466552368065 -118.32982785607598) bank1384 +1385 POINT(33.930181497161456 -117.91046738342466) bank1385 +1386 POINT(33.884893505443024 -117.36644577658002) bank1386 +1387 POINT(34.01280616995176 -117.63783947651798) bank1387 +1388 POINT(33.94470307088613 -119.12744130278179) bank1388 +1389 POINT(33.88001777333027 -119.12831663311503) bank1389 +1390 POINT(33.441066632312584 -118.67019288651954) bank1390 +1391 POINT(34.93059244106053 -119.20861536559208) bank1391 +1392 POINT(34.89925410741713 -117.98406531003965) bank1392 +1393 POINT(34.63473715025458 -118.57628205279808) bank1393 +1394 POINT(33.45129086066098 -118.89168988531635) bank1394 +1395 POINT(34.80468627142118 -118.95083428721087) bank1395 +1396 POINT(34.71093585124225 -118.63437848250607) bank1396 +1397 POINT(34.73494538616713 -118.85902794247089) bank1397 +1398 POINT(34.038820702517086 -117.95158566605635) bank1398 +1399 POINT(34.04473698451043 -117.42941220464229) bank1399 +1400 POINT(34.16909938042385 -118.46232792411733) bank1400 +1401 POINT(34.383365643687874 -118.73222214636196) bank1401 +1402 POINT(33.13565998645268 -118.0381989798405) bank1402 +1403 POINT(35.02790379987167 -118.52914197246812) bank1403 +1404 POINT(33.94455954189492 -117.80057096587255) bank1404 +1405 POINT(33.30367450060958 -117.28544396818054) bank1405 +1406 POINT(33.16688632167657 -118.4412137977921) bank1406 +1407 POINT(33.902628212293756 -117.35063236493662) bank1407 +1408 POINT(34.39823441345691 -119.20872102819611) bank1408 +1409 POINT(33.13069596912033 -118.13450385693812) bank1409 +1410 POINT(34.32422964850726 -117.9669390685595) bank1410 +1411 POINT(33.79654389389935 -119.02779848968673) bank1411 +1412 POINT(34.199980623496536 -117.253423071016) bank1412 +1413 POINT(34.80381386974976 -118.71656715670272) bank1413 +1414 POINT(34.93766275363445 -117.64210163861203) bank1414 +1415 POINT(34.27895127412807 -118.76067504484206) bank1415 +1416 POINT(33.23224173943605 -117.45887211697485) bank1416 +1417 POINT(33.59877992440118 -117.89359110627426) bank1417 +1418 POINT(33.297186791238836 -117.7828423260289) bank1418 +1419 POINT(33.22639628791064 -118.89646749285735) bank1419 +1420 POINT(34.74365013935216 -117.62963803450823) bank1420 +1421 POINT(33.15158571076896 -118.6011868514608) bank1421 +1422 POINT(33.52186714435518 -118.08611436489875) bank1422 +1423 POINT(34.014813358816305 -118.35251910589729) bank1423 +1424 POINT(33.38969088745257 -118.36472548126551) bank1424 +1425 POINT(34.015288992475284 -118.29083098083188) bank1425 +1426 POINT(33.6355835977433 -118.26340949928567) bank1426 +1427 POINT(33.47986365465962 -118.76580632348704) bank1427 +1428 POINT(33.182170858667476 -118.61279525880127) bank1428 +1429 POINT(33.295495562728036 -117.96477269061816) bank1429 +1430 POINT(34.98738702748719 -118.7600703331105) bank1430 +1431 POINT(34.06723200213463 -118.38428452607059) bank1431 +1432 POINT(34.28889932545799 -118.13613122951969) bank1432 +1433 POINT(33.21876963403409 -118.29958421924138) bank1433 +1434 POINT(33.12058460703988 -118.2503547488295) bank1434 +1435 POINT(33.241280816729706 -118.43250170723523) bank1435 +1436 POINT(33.592892171663905 -117.65725275906337) bank1436 +1437 POINT(33.870338191752 -118.66073354848288) bank1437 +1438 POINT(33.804215191102806 -119.21879039505663) bank1438 +1439 POINT(33.804105107790065 -117.76940295980751) bank1439 +1440 POINT(33.256274735350864 -117.84075812531633) bank1440 +1441 POINT(33.22995552401629 -118.447991136724) bank1441 +1442 POINT(34.94292250262189 -118.29215319752176) bank1442 +1443 POINT(33.18765042709955 -117.36025904337) bank1443 +1444 POINT(33.37240295805231 -117.88849446726925) bank1444 +1445 POINT(34.64328151558479 -117.58627278779296) bank1445 +1446 POINT(34.1833065798339 -119.14609489920745) bank1446 +1447 POINT(33.50740674292709 -118.37761756612532) bank1447 +1448 POINT(34.99638289055276 -117.97518969866043) bank1448 +1449 POINT(34.633616311160786 -117.95930389535643) bank1449 +1450 POINT(34.997628403654645 -118.20823084764902) bank1450 +1451 POINT(33.5008390181312 -118.28679647324385) bank1451 +1452 POINT(33.094337715975975 -117.62075383458782) bank1452 +1453 POINT(33.200985075385276 -118.70779495403826) bank1453 +1454 POINT(34.7595064426863 -119.11321965867307) bank1454 +1455 POINT(33.77963695840954 -119.01686582340938) bank1455 +1456 POINT(33.65095442730462 -117.71328504399116) bank1456 +1457 POINT(33.58269057752697 -118.18627799446794) bank1457 +1458 POINT(34.11426919575457 -118.43495586698245) bank1458 +1459 POINT(33.141019418866904 -118.30774192860508) bank1459 +1460 POINT(34.21737691979888 -118.9903362509137) bank1460 +1461 POINT(34.74788950530285 -118.84175288697193) bank1461 +1462 POINT(34.10762921918535 -118.80957493654232) bank1462 +1463 POINT(34.68432934079704 -118.80874166861723) bank1463 +1464 POINT(33.511044443543085 -118.39486153843815) bank1464 +1465 POINT(33.714099170194146 -118.2162042398122) bank1465 +1466 POINT(34.711972842077294 -119.23591217636495) bank1466 +1467 POINT(33.63555528008085 -119.23231404689702) bank1467 +1468 POINT(33.624090487082825 -118.89596898507807) bank1468 +1469 POINT(33.70973006663064 -118.9100233169821) bank1469 +1470 POINT(34.377489894397776 -118.10310600414552) bank1470 +1471 POINT(34.95337036111057 -118.88920470744361) bank1471 +1472 POINT(33.93361152635272 -119.16221048959831) bank1472 +1473 POINT(34.118199611226956 -117.25238092918329) bank1473 +1474 POINT(34.66950670924128 -117.91764759412501) bank1474 +1475 POINT(34.78161399359575 -118.037092409452) bank1475 +1476 POINT(33.95847843367277 -118.35177453232657) bank1476 +1477 POINT(34.65808120318619 -119.03420461866727) bank1477 +1478 POINT(34.91507415514722 -119.1459172681306) bank1478 +1479 POINT(33.639485058564 -118.75514933473919) bank1479 +1480 POINT(33.44773930413318 -117.6854964203048) bank1480 +1481 POINT(34.124962692278835 -117.99293067316809) bank1481 +1482 POINT(33.23820929889462 -117.34950162687781) bank1482 +1483 POINT(34.26612235903517 -118.6323109092213) bank1483 +1484 POINT(33.818927061379654 -117.95861456602884) bank1484 +1485 POINT(34.2330838897966 -117.61917216217093) bank1485 +1486 POINT(34.35323148601906 -117.67921116023638) bank1486 +1487 POINT(34.34285746488472 -117.74905810975987) bank1487 +1488 POINT(33.45913243670402 -117.4320420407641) bank1488 +1489 POINT(33.4004929756863 -117.72279041284628) bank1489 +1490 POINT(33.10109525544543 -118.61247300265639) bank1490 +1491 POINT(34.71253546230216 -118.66249608362185) bank1491 +1492 POINT(34.53650941966116 -118.07134318954373) bank1492 +1493 POINT(33.3096175089003 -118.33105172690453) bank1493 +1494 POINT(34.69623820094766 -117.29029252440365) bank1494 +1495 POINT(33.52919671342546 -118.85808777787655) bank1495 +1496 POINT(33.483208971779426 -117.33303481488193) bank1496 +1497 POINT(34.477891207252405 -119.16282523371724) bank1497 +1498 POINT(33.401077519185186 -117.34193391017014) bank1498 +1499 POINT(33.26445235055296 -118.64518595090112) bank1499 +1500 POINT(34.39660923145653 -118.42102954510844) bank1500 +1501 POINT(34.50875832775485 -118.70122844329079) bank1501 +1502 POINT(34.38097557841805 -117.29615374858847) bank1502 +1503 POINT(34.68738884897162 -117.59103015728851) bank1503 +1504 POINT(33.30174127641952 -118.4200781822412) bank1504 +1505 POINT(33.992564501961056 -119.0871251815986) bank1505 +1506 POINT(33.08695602592731 -117.27158920824549) bank1506 +1507 POINT(33.986357137257365 -117.8738020021868) bank1507 +1508 POINT(34.808747602417675 -118.78088042510457) bank1508 +1509 POINT(33.10198415915245 -118.9764830154175) bank1509 +1510 POINT(34.25725979152867 -118.22360670821544) bank1510 +1511 POINT(34.28278342169139 -118.85868980098832) bank1511 +1512 POINT(33.983554716115776 -117.39952796216052) bank1512 +1513 POINT(33.65762942073582 -118.8061193257628) bank1513 +1514 POINT(34.99033695476252 -117.78213124631819) bank1514 +1515 POINT(33.62221729931378 -118.87844816344169) bank1515 +1516 POINT(34.60098518516165 -118.75683355333018) bank1516 +1517 POINT(34.149170948741684 -118.09104107002683) bank1517 +1518 POINT(33.9971786399623 -118.35372199923118) bank1518 +1519 POINT(34.79808934146739 -118.03656427265979) bank1519 +1520 POINT(33.525732777822824 -117.8478131521472) bank1520 +1521 POINT(33.134537417176745 -119.02664705644423) bank1521 +1522 POINT(34.86130894037235 -118.83631934763916) bank1522 +1523 POINT(34.11350035612836 -118.4685548963292) bank1523 +1524 POINT(34.075074946407334 -117.44029425684181) bank1524 +1525 POINT(34.07629347751723 -118.1164294126438) bank1525 +1526 POINT(33.58837482092659 -117.71435701174889) bank1526 +1527 POINT(34.485020375785474 -119.1955571025903) bank1527 +1528 POINT(33.94840970462043 -117.63566441171864) bank1528 +1529 POINT(33.87226166950725 -118.44307097412668) bank1529 +1530 POINT(34.416750786436104 -117.56929391921823) bank1530 +1531 POINT(33.46514741576958 -118.76875378948942) bank1531 +1532 POINT(33.2288562805134 -117.2776031163015) bank1532 +1533 POINT(33.05243968399211 -117.45298375013483) bank1533 +1534 POINT(34.18796437083722 -118.90581253758934) bank1534 +1535 POINT(34.531944371051104 -117.66752173830996) bank1535 +1536 POINT(34.487138874653674 -118.47626008905488) bank1536 +1537 POINT(33.275274083410416 -118.58784091020246) bank1537 +1538 POINT(34.39382625222123 -118.0382602728624) bank1538 +1539 POINT(34.0251449172309 -119.22458133962233) bank1539 +1540 POINT(34.245432283898616 -118.03216534447229) bank1540 +1541 POINT(33.616325064729665 -117.56516694032634) bank1541 +1542 POINT(34.650420844148535 -118.81556929512263) bank1542 +1543 POINT(33.80307488192349 -117.77057526831058) bank1543 +1544 POINT(34.636127866849506 -118.66893700203204) bank1544 +1545 POINT(34.153563317760145 -119.19346016503515) bank1545 +1546 POINT(34.11148369995411 -118.59406637897114) bank1546 +1547 POINT(33.904784716544434 -118.10821697211271) bank1547 +1548 POINT(33.4316341926192 -117.27849694161291) bank1548 +1549 POINT(33.7029377635107 -119.08233314040977) bank1549 +1550 POINT(34.070683442814286 -117.69134616981944) bank1550 +1551 POINT(33.77147572982781 -118.0618005281331) bank1551 +1552 POINT(33.39430392011397 -118.53317858328288) bank1552 +1553 POINT(33.232872457121466 -118.40470307403659) bank1553 +1554 POINT(33.45714696081748 -119.20652688140135) bank1554 +1555 POINT(33.270311569967326 -117.65767372008567) bank1555 +1556 POINT(34.91899174956492 -117.92681622597377) bank1556 +1557 POINT(34.19859992333578 -119.15000386234566) bank1557 +1558 POINT(34.74157971214336 -117.95831811017595) bank1558 +1559 POINT(33.219922203589164 -118.60265936756507) bank1559 +1560 POINT(33.274834283605436 -117.91876111786164) bank1560 +1561 POINT(34.74994170611488 -117.73660847627784) bank1561 +1562 POINT(34.87309839095782 -118.9931937658901) bank1562 +1563 POINT(35.02420741606909 -117.28217854688816) bank1563 +1564 POINT(33.14478325104399 -117.50768308558465) bank1564 +1565 POINT(33.73720400055097 -117.82355683619667) bank1565 +1566 POINT(34.9960113449715 -118.7560921528968) bank1566 +1567 POINT(33.72355165607404 -119.21228246303589) bank1567 +1568 POINT(33.75486838968381 -118.64454371072561) bank1568 +1569 POINT(33.65129517883523 -118.54304325147669) bank1569 +1570 POINT(33.54199203503124 -118.72593902896689) bank1570 +1571 POINT(33.08590471568052 -118.97103682479533) bank1571 +1572 POINT(33.13416782276486 -118.38876504832159) bank1572 +1573 POINT(34.055567292749785 -118.46481365760748) bank1573 +1574 POINT(33.25358971603678 -118.4269037848691) bank1574 +1575 POINT(34.66049606629314 -118.59157706868447) bank1575 +1576 POINT(34.349335067771094 -119.16621320070678) bank1576 +1577 POINT(33.70532742769839 -117.85307888199104) bank1577 +1578 POINT(33.588337622834544 -118.73624604651205) bank1578 +1579 POINT(34.75355325270906 -117.8445974693611) bank1579 +1580 POINT(34.71349516787466 -117.49829859573836) bank1580 +1581 POINT(34.88713681522596 -119.16339123750537) bank1581 +1582 POINT(34.19909190619127 -118.92439270962834) bank1582 +1583 POINT(33.277350895776976 -117.5085800529757) bank1583 +1584 POINT(35.035522049912686 -117.29976734461344) bank1584 +1585 POINT(33.222924860196684 -117.38168643290102) bank1585 +1586 POINT(34.53262203542813 -118.37258971657874) bank1586 +1587 POINT(34.99806790273487 -118.71170793188105) bank1587 +1588 POINT(34.025620483892425 -119.0543902577485) bank1588 +1589 POINT(34.29552497478009 -117.43275772714448) bank1589 +1590 POINT(33.80613652153845 -117.84804207187807) bank1590 +1591 POINT(34.88138901268408 -118.7573182603718) bank1591 +1592 POINT(34.58168297982487 -118.3676941571583) bank1592 +1593 POINT(33.92338746366457 -117.78662797565917) bank1593 +1594 POINT(33.53974199058474 -117.77568236448226) bank1594 +1595 POINT(33.266380782369325 -118.18258465349375) bank1595 +1596 POINT(34.4368649431498 -118.3542457198713) bank1596 +1597 POINT(33.07875533033172 -117.5912827557332) bank1597 +1598 POINT(33.873558494038264 -117.98867123710953) bank1598 +1599 POINT(33.77706097663066 -117.51094574109969) bank1599 +1600 POINT(33.43724194889404 -119.22231200044436) bank1600 +1601 POINT(33.667135272774644 -118.87563940979683) bank1601 +1602 POINT(33.17469494687268 -117.53046672566518) bank1602 +1603 POINT(34.6953627473509 -117.98273232183243) bank1603 +1604 POINT(34.80201765076572 -118.90249588564869) bank1604 +1605 POINT(33.503073606064454 -117.80334697958855) bank1605 +1606 POINT(33.32952401055099 -118.37156512594748) bank1606 +1607 POINT(34.49915262212041 -118.55430725916837) bank1607 +1608 POINT(33.37073848931905 -119.16022684394804) bank1608 +1609 POINT(34.19783357539191 -118.93179735884112) bank1609 +1610 POINT(34.509868392792974 -117.8831013230915) bank1610 +1611 POINT(33.575526633677946 -118.85602341783296) bank1611 +1612 POINT(34.38160194372488 -117.33886148937081) bank1612 +1613 POINT(33.37205571442393 -118.04768984752523) bank1613 +1614 POINT(33.92600216889355 -117.80942815953999) bank1614 +1615 POINT(34.70171423170309 -118.39739894663967) bank1615 +1616 POINT(33.070452395015195 -118.60265885835055) bank1616 +1617 POINT(34.85737053795762 -118.46057053117497) bank1617 +1618 POINT(34.739290134450435 -119.07658562710876) bank1618 +1619 POINT(33.64249272439952 -117.71349910621255) bank1619 +1620 POINT(33.05265110635614 -117.63931547596758) bank1620 +1621 POINT(34.43993500463794 -118.79760656876884) bank1621 +1622 POINT(34.59934007974253 -118.05965044907448) bank1622 +1623 POINT(33.318119734801066 -118.24814346048935) bank1623 +1624 POINT(34.34163146104627 -118.3621709786337) bank1624 +1625 POINT(34.98216049392212 -118.44513489238584) bank1625 +1626 POINT(34.140499592797 -118.17833066022776) bank1626 +1627 POINT(34.94461710033874 -117.78585347533573) bank1627 +1628 POINT(34.245814476846014 -118.73260635723011) bank1628 +1629 POINT(33.63379377637133 -118.9287382788331) bank1629 +1630 POINT(33.40845410374475 -118.49664263491175) bank1630 +1631 POINT(33.39898046881734 -118.08330556766776) bank1631 +1632 POINT(34.83858563137138 -118.30938040141196) bank1632 +1633 POINT(34.701363635033864 -117.60809570224393) bank1633 +1634 POINT(34.10065906090901 -117.32236416053667) bank1634 +1635 POINT(33.91141366105014 -117.84090930346879) bank1635 +1636 POINT(34.37289634837343 -117.6010342614429) bank1636 +1637 POINT(34.05209457791333 -119.09327029438998) bank1637 +1638 POINT(34.82992408067255 -118.26462862987562) bank1638 +1639 POINT(34.298235063649784 -118.51837869777809) bank1639 +1640 POINT(33.83648293400602 -118.61896910844158) bank1640 +1641 POINT(34.61613099002607 -118.8588351542257) bank1641 +1642 POINT(34.38732520905684 -118.0811527643751) bank1642 +1643 POINT(34.9691760952586 -118.7623574426226) bank1643 +1644 POINT(34.712327848723966 -119.08889289427213) bank1644 +1645 POINT(34.45546282209384 -117.8881157934621) bank1645 +1646 POINT(34.70768229420087 -118.96351413891601) bank1646 +1647 POINT(33.492189824616574 -118.3126885193515) bank1647 +1648 POINT(33.982825154455846 -117.4665276755014) bank1648 +1649 POINT(33.16760841505404 -117.54327811013994) bank1649 +1650 POINT(33.21259350959242 -118.57666551111978) bank1650 +1651 POINT(34.410389522527566 -117.73267374042763) bank1651 +1652 POINT(34.633950656868954 -119.06033348762358) bank1652 +1653 POINT(33.23075318070822 -117.58306449339315) bank1653 +1654 POINT(33.238828875660985 -117.6852313077928) bank1654 +1655 POINT(33.96852801184143 -118.58416053367631) bank1655 +1656 POINT(34.804083776514474 -117.4515698057841) bank1656 +1657 POINT(33.355287440040364 -118.8689066642285) bank1657 +1658 POINT(34.62279334914036 -118.11386331147239) bank1658 +1659 POINT(33.5982562855208 -118.0427531497775) bank1659 +1660 POINT(33.51269118449047 -118.11574937631174) bank1660 +1661 POINT(33.10186244608853 -118.37498798107165) bank1661 +1662 POINT(34.64090726035936 -118.17969709824254) bank1662 +1663 POINT(34.74669339969186 -118.70980854530235) bank1663 +1664 POINT(34.7519578912725 -118.70888920885061) bank1664 +1665 POINT(33.20327370403437 -117.84541124938286) bank1665 +1666 POINT(33.55364595962099 -118.70498543825515) bank1666 +1667 POINT(34.765213487310625 -118.74554345588153) bank1667 +1668 POINT(34.63327582018304 -119.07680002823963) bank1668 +1669 POINT(33.986353508962765 -118.86859339986928) bank1669 +1670 POINT(34.83508739084858 -117.94939518560453) bank1670 +1671 POINT(33.55066969596912 -118.4201858434117) bank1671 +1672 POINT(34.16160383584904 -117.4966655705141) bank1672 +1673 POINT(34.90012306826103 -118.74051454361896) bank1673 +1674 POINT(34.842850829646 -117.31158344495296) bank1674 +1675 POINT(33.309731704908025 -118.93507229569411) bank1675 +1676 POINT(33.6461722111398 -117.3994956581269) bank1676 +1677 POINT(33.37488876444565 -118.48623787313909) bank1677 +1678 POINT(34.008371826658504 -118.37843207862575) bank1678 +1679 POINT(34.17098095821046 -117.64287071588221) bank1679 +1680 POINT(34.51677702300557 -118.22691892286109) bank1680 +1681 POINT(33.68917869879083 -117.24843095700571) bank1681 +1682 POINT(33.14880334701068 -117.35265521858024) bank1682 +1683 POINT(34.72385904150097 -119.04790444558861) bank1683 +1684 POINT(34.99679555190287 -118.68989896151338) bank1684 +1685 POINT(33.05701562991354 -117.57652456660378) bank1685 +1686 POINT(33.835155777800324 -117.36169258143262) bank1686 +1687 POINT(33.20442967788648 -118.58569528259378) bank1687 +1688 POINT(34.45162842712787 -119.13401840337353) bank1688 +1689 POINT(34.64686958048537 -118.19748785976041) bank1689 +1690 POINT(34.183578547553175 -118.85161958988688) bank1690 +1691 POINT(34.99570333924852 -118.98879046902212) bank1691 +1692 POINT(34.63202649855101 -118.1119293544042) bank1692 +1693 POINT(34.26174608009999 -119.17269134442412) bank1693 +1694 POINT(34.19486228949416 -118.44533567124628) bank1694 +1695 POINT(33.16755976425242 -117.886300640104) bank1695 +1696 POINT(34.8391344603157 -118.11018026219743) bank1696 +1697 POINT(33.97308387915825 -118.2148266122304) bank1697 +1698 POINT(33.0567260753262 -118.82270790065253) bank1698 +1699 POINT(33.70111715667594 -118.53274281164819) bank1699 +1700 POINT(34.69249727772326 -118.24950645991055) bank1700 +1701 POINT(34.93957591629951 -118.88413229883024) bank1701 +1702 POINT(33.12306681455716 -117.33813347973752) bank1702 +1703 POINT(33.74326907755574 -118.57406862135127) bank1703 +1704 POINT(33.43488813189041 -118.40946239888083) bank1704 +1705 POINT(33.81469349619873 -118.92809568594393) bank1705 +1706 POINT(34.48260228663297 -118.19443807858964) bank1706 +1707 POINT(33.30842835320075 -117.29718502468194) bank1707 +1708 POINT(34.73142328699249 -118.61702907924037) bank1708 +1709 POINT(33.65755190332904 -118.15137617917478) bank1709 +1710 POINT(33.95267921691917 -119.19425706198669) bank1710 +1711 POINT(34.63333170743051 -118.34918696927316) bank1711 +1712 POINT(33.4031663308054 -117.37784278549582) bank1712 +1713 POINT(33.61686068161275 -118.86250694747885) bank1713 +1714 POINT(33.65003596783181 -118.28815452022306) bank1714 +1715 POINT(34.55078242287162 -118.42869438616243) bank1715 +1716 POINT(34.549809707190725 -117.66559840178908) bank1716 +1717 POINT(33.74212777224491 -117.29653933689362) bank1717 +1718 POINT(34.120088532979025 -117.37538611592448) bank1718 +1719 POINT(34.358372896389454 -117.403985697498) bank1719 +1720 POINT(34.00173422356177 -118.37778649336569) bank1720 +1721 POINT(34.16148832789217 -118.46685477292439) bank1721 +1722 POINT(33.52059180654324 -117.91188510340685) bank1722 +1723 POINT(34.28426813026358 -117.31158937975401) bank1723 +1724 POINT(33.273418316530616 -118.63735537596641) bank1724 +1725 POINT(34.96107042629483 -117.33440928826587) bank1725 +1726 POINT(34.60655093807114 -117.93888455907232) bank1726 +1727 POINT(34.991957238162776 -118.02984824099046) bank1727 +1728 POINT(33.31598718217732 -117.95267299963321) bank1728 +1729 POINT(33.659075672110276 -117.52395645556098) bank1729 +1730 POINT(33.218977521956965 -119.06297699415896) bank1730 +1731 POINT(34.788449369354815 -117.87065076691927) bank1731 +1732 POINT(33.452886084446874 -117.27063983975151) bank1732 +1733 POINT(35.00925950474524 -118.48050017991716) bank1733 +1734 POINT(34.40537269998207 -117.828082912769) bank1734 +1735 POINT(34.7615581238842 -118.23797691692259) bank1735 +1736 POINT(33.14616552946095 -119.09039233888609) bank1736 +1737 POINT(33.785239977505306 -118.33130875731399) bank1737 +1738 POINT(33.613166105125345 -117.56708342562645) bank1738 +1739 POINT(34.54319699634398 -118.46953044891741) bank1739 +1740 POINT(34.70467707876822 -118.30481782269784) bank1740 +1741 POINT(33.266348971723 -119.03374245304202) bank1741 +1742 POINT(34.89055058939029 -118.4534668840348) bank1742 +1743 POINT(34.45026711309191 -118.44173743704309) bank1743 +1744 POINT(33.31047637047778 -118.80027787029236) bank1744 +1745 POINT(33.16354434669362 -118.89261494592012) bank1745 +1746 POINT(33.6090303362333 -118.6109010467299) bank1746 +1747 POINT(34.471599781091335 -118.92492807511667) bank1747 +1748 POINT(34.00396880145937 -118.79519677055215) bank1748 +1749 POINT(34.34245276389245 -117.32852061909462) bank1749 +1750 POINT(34.13597962725671 -117.9063988324151) bank1750 +1751 POINT(33.87353268010037 -118.46331769027078) bank1751 +1752 POINT(33.17773565841287 -118.03614995353345) bank1752 +1753 POINT(33.36038629451735 -117.58767569041031) bank1753 +1754 POINT(34.576608914254415 -118.1092479960669) bank1754 +1755 POINT(33.27391684644904 -117.90716356173068) bank1755 +1756 POINT(34.52710953403132 -118.98315734899391) bank1756 +1757 POINT(34.83074008495532 -117.90548717932514) bank1757 +1758 POINT(34.994636553118156 -118.82747241183928) bank1758 +1759 POINT(34.25886185214365 -117.8031028437776) bank1759 +1760 POINT(34.84152593516142 -117.93179309344073) bank1760 +1761 POINT(34.12978676531555 -118.93389265248632) bank1761 +1762 POINT(34.91778175696683 -118.72974674371643) bank1762 +1763 POINT(33.599225508952095 -118.82234386783848) bank1763 +1764 POINT(34.98587721638941 -118.43998871424431) bank1764 +1765 POINT(34.44425716313501 -119.20128218677283) bank1765 +1766 POINT(33.89343654687114 -118.17504194621878) bank1766 +1767 POINT(33.815313620515305 -118.1218051870844) bank1767 +1768 POINT(33.48985871156483 -119.09127482910614) bank1768 +1769 POINT(33.7931207485759 -117.85251457629904) bank1769 +1770 POINT(34.031031962850726 -119.0938046198075) bank1770 +1771 POINT(34.50376960913629 -118.42129985109648) bank1771 +1772 POINT(34.300060539949406 -118.24689285826453) bank1772 +1773 POINT(34.84210652261412 -118.7983959983597) bank1773 +1774 POINT(34.093945060339045 -118.07595081932442) bank1774 +1775 POINT(34.48268747397634 -118.48710346131845) bank1775 +1776 POINT(34.479728820867905 -119.18816987419274) bank1776 +1777 POINT(33.995810185627086 -117.86725958552286) bank1777 +1778 POINT(34.905154641829476 -118.83453690084248) bank1778 +1779 POINT(34.244741765737146 -117.65317291561286) bank1779 +1780 POINT(33.837473694540634 -118.39090864536527) bank1780 +1781 POINT(34.3967698417054 -117.989552693787) bank1781 +1782 POINT(33.833000915160184 -118.60668621896203) bank1782 +1783 POINT(34.52522531749981 -118.75075615424556) bank1783 +1784 POINT(35.01168364795788 -117.35835077904201) bank1784 +1785 POINT(34.908177403079364 -117.92298052317977) bank1785 +1786 POINT(34.51039467845595 -117.45395315206945) bank1786 +1787 POINT(33.51406826219614 -117.41215019842342) bank1787 +1788 POINT(34.77109453945792 -119.00426897175662) bank1788 +1789 POINT(34.96168682716047 -117.53787820061811) bank1789 +1790 POINT(34.590070215111226 -118.14853987675973) bank1790 +1791 POINT(33.65758257499257 -118.40235113121459) bank1791 +1792 POINT(33.50142135673573 -117.9873923333854) bank1792 +1793 POINT(35.04398155168027 -117.60374556979161) bank1793 +1794 POINT(33.2643002824586 -117.87765362931543) bank1794 +1795 POINT(34.85462267103936 -119.16534870945502) bank1795 +1796 POINT(34.125579345858704 -118.89959719843844) bank1796 +1797 POINT(34.9758707112484 -117.8077256531964) bank1797 +1798 POINT(34.54908009403531 -117.52309039659929) bank1798 +1799 POINT(34.800377191362976 -118.39644086411289) bank1799 +1800 POINT(33.20407533699237 -118.75375111470285) bank1800 +1801 POINT(33.854862131294624 -119.14522469964888) bank1801 +1802 POINT(33.842010261397036 -117.46394463572342) bank1802 +1803 POINT(33.067301242406614 -118.54061826501206) bank1803 +1804 POINT(33.53552552270885 -118.27028804263017) bank1804 +1805 POINT(33.13043239407028 -117.6146973256147) bank1805 +1806 POINT(33.813977011587525 -117.71067258871702) bank1806 +1807 POINT(33.602879647765505 -117.42594776295873) bank1807 +1808 POINT(34.56807178603202 -117.71395422049942) bank1808 +1809 POINT(35.05083781995295 -118.19073625987276) bank1809 +1810 POINT(34.72647735913316 -118.77473578072147) bank1810 +1811 POINT(34.71014429389502 -119.14059724708565) bank1811 +1812 POINT(34.51633104068324 -119.18428882056267) bank1812 +1813 POINT(34.738569648396606 -118.02523658975223) bank1813 +1814 POINT(34.898644125245454 -118.26584621650208) bank1814 +1815 POINT(34.60970463543437 -118.0073979466496) bank1815 +1816 POINT(34.152942085829004 -117.42206986159461) bank1816 +1817 POINT(34.8499269605512 -117.57016283597227) bank1817 +1818 POINT(34.9773017077713 -117.56682620116224) bank1818 +1819 POINT(34.918310858684144 -118.11365478672434) bank1819 +1820 POINT(33.41558682367906 -118.90394000670551) bank1820 +1821 POINT(34.72436474517556 -118.45372539279263) bank1821 +1822 POINT(34.99794534920488 -118.96626240859139) bank1822 +1823 POINT(34.95540591272167 -118.21278620100075) bank1823 +1824 POINT(34.35689161912474 -117.59567597594967) bank1824 +1825 POINT(34.804403085714256 -117.62497985400216) bank1825 +1826 POINT(33.92902288323762 -118.44118374803458) bank1826 +1827 POINT(33.7112888714548 -118.8793954186216) bank1827 +1828 POINT(33.46186793562928 -117.48947611008973) bank1828 +1829 POINT(33.65371955275908 -117.65848178363939) bank1829 +1830 POINT(34.89125168740941 -118.68102286277205) bank1830 +1831 POINT(33.10909936685552 -117.43754178715153) bank1831 +1832 POINT(34.25004261198848 -118.79618161998317) bank1832 +1833 POINT(33.28031888949243 -118.43780254049314) bank1833 +1834 POINT(34.54653014246363 -118.59515939604624) bank1834 +1835 POINT(33.76135330046169 -118.13860240406198) bank1835 +1836 POINT(33.77784400076117 -118.46122753753309) bank1836 +1837 POINT(33.13989338456863 -118.31956181853512) bank1837 +1838 POINT(33.46146674008659 -119.22212991465808) bank1838 +1839 POINT(34.83581594295633 -118.56879803631074) bank1839 +1840 POINT(33.51183627656092 -118.16368952406249) bank1840 +1841 POINT(34.28770939613529 -118.58193466298593) bank1841 +1842 POINT(34.877225875311105 -117.81450140018671) bank1842 +1843 POINT(33.5304147111941 -118.91166651251244) bank1843 +1844 POINT(33.216106956202275 -118.54074444298047) bank1844 +1845 POINT(33.87703624243431 -118.87103196424233) bank1845 +1846 POINT(34.49509565492217 -119.2230234447445) bank1846 +1847 POINT(33.405878498727574 -117.36135261750034) bank1847 +1848 POINT(33.68405479321184 -117.46682732204262) bank1848 +1849 POINT(34.166832014128886 -117.90986409914487) bank1849 +1850 POINT(34.7207746410909 -117.28538943309783) bank1850 +1851 POINT(34.30119615963185 -119.20812803340112) bank1851 +1852 POINT(33.81077654813108 -117.3719531796873) bank1852 +1853 POINT(34.65387052145859 -118.67859825038789) bank1853 +1854 POINT(33.48709662008159 -118.5250783018381) bank1854 +1855 POINT(33.12968068864636 -118.98801099135385) bank1855 +1856 POINT(33.25784693510389 -118.82399175519534) bank1856 +1857 POINT(34.45068518412781 -117.43376560166391) bank1857 +1858 POINT(33.83216906414727 -117.6365306818599) bank1858 +1859 POINT(34.626963809890825 -118.07985001677459) bank1859 +1860 POINT(33.35413101652307 -118.23471477133279) bank1860 +1861 POINT(34.91095029624411 -119.07746367405042) bank1861 +1862 POINT(33.171453998025335 -117.49246850798848) bank1862 +1863 POINT(33.92395460862027 -119.17017815261717) bank1863 +1864 POINT(33.70139606971294 -118.6053436954114) bank1864 +1865 POINT(33.74780440408783 -119.00179113363328) bank1865 +1866 POINT(33.842278341618766 -118.12414689186039) bank1866 +1867 POINT(33.71584462715866 -119.20229060729044) bank1867 +1868 POINT(33.60040881676799 -117.73521416879778) bank1868 +1869 POINT(33.09050939083673 -118.93237980310394) bank1869 +1870 POINT(33.51969656236512 -118.32850936329326) bank1870 +1871 POINT(34.64414726445742 -117.7676763038742) bank1871 +1872 POINT(34.692466357625726 -118.31391399942262) bank1872 +1873 POINT(33.21778366086702 -118.579681780246) bank1873 +1874 POINT(33.96312972951275 -119.10691761935269) bank1874 +1875 POINT(34.76412618880259 -117.69502442964917) bank1875 +1876 POINT(33.89685270407234 -117.29002444964442) bank1876 +1877 POINT(33.46021198590533 -117.6625688174026) bank1877 +1878 POINT(33.61033589678455 -117.77122460701106) bank1878 +1879 POINT(33.789487495779134 -117.27842132388919) bank1879 +1880 POINT(34.23948807091945 -118.27906555939872) bank1880 +1881 POINT(33.34646749882334 -118.4369656299168) bank1881 +1882 POINT(34.17936376040842 -117.51638125390885) bank1882 +1883 POINT(34.05553201370366 -118.66325596414994) bank1883 +1884 POINT(33.066440794379304 -118.55298141619816) bank1884 +1885 POINT(33.765060220469024 -119.15982156557854) bank1885 +1886 POINT(33.47265660562753 -117.55516206629034) bank1886 +1887 POINT(33.10409824472023 -117.4331527439907) bank1887 +1888 POINT(34.93666121672862 -119.12452953517464) bank1888 +1889 POINT(34.448866347936814 -118.01398435979141) bank1889 +1890 POINT(33.128709850273914 -119.23901298818859) bank1890 +1891 POINT(33.88159586501961 -117.28067047046535) bank1891 +1892 POINT(34.40906573926696 -119.09103542821417) bank1892 +1893 POINT(34.71860774257192 -118.71825297525474) bank1893 +1894 POINT(33.3022405622217 -117.96304059355255) bank1894 +1895 POINT(33.56886339227605 -119.02222493178868) bank1895 +1896 POINT(34.20914111459145 -118.65731614033388) bank1896 +1897 POINT(33.638313593248775 -117.88101003024934) bank1897 +1898 POINT(33.508882415714226 -118.9983996483398) bank1898 +1899 POINT(34.25144247548813 -118.87585652415693) bank1899 +1900 POINT(33.918725913195125 -118.26727594384423) bank1900 +1901 POINT(33.66486709579967 -117.30458757882954) bank1901 +1902 POINT(33.883082092671195 -117.6586627884021) bank1902 +1903 POINT(34.39640522926838 -118.91243716377492) bank1903 +1904 POINT(33.36221793906626 -119.03156085251968) bank1904 +1905 POINT(34.752429929877444 -117.70938806390431) bank1905 +1906 POINT(34.551536227187036 -119.0794386918237) bank1906 +1907 POINT(33.91422438068594 -118.63273343279381) bank1907 +1908 POINT(34.87356997290337 -117.68442518608711) bank1908 +1909 POINT(34.66278967022313 -118.13934834070946) bank1909 +1910 POINT(34.30725324926033 -118.5013453582926) bank1910 +1911 POINT(33.28525864061083 -117.79933230472969) bank1911 +1912 POINT(33.70882242913439 -117.40535133551874) bank1912 +1913 POINT(33.62193408541908 -118.07013305570729) bank1913 +1914 POINT(33.90354630482595 -117.85310433921234) bank1914 +1915 POINT(34.924604474250046 -118.15460467452877) bank1915 +1916 POINT(34.990905116911435 -117.5382361289936) bank1916 +1917 POINT(33.93557070173138 -117.30646756622711) bank1917 +1918 POINT(34.91003124436768 -118.64827472701919) bank1918 +1919 POINT(33.88311141583082 -118.58397499959523) bank1919 +1920 POINT(35.03462679641826 -118.63584496255203) bank1920 +1921 POINT(34.47841929615527 -118.90268537687193) bank1921 +1922 POINT(34.07809115524701 -118.75479582636632) bank1922 +1923 POINT(34.137391875148964 -118.68831201265618) bank1923 +1924 POINT(34.80205111872227 -118.03980891481108) bank1924 +1925 POINT(33.359168831350225 -117.6104829391105) bank1925 +1926 POINT(34.31244340580329 -118.40443537573391) bank1926 +1927 POINT(34.39695631696291 -119.08347751887112) bank1927 +1928 POINT(34.78301465313036 -117.42278758614414) bank1928 +1929 POINT(34.341852070257204 -118.64389754092839) bank1929 +1930 POINT(34.53279016105151 -118.71274028608497) bank1930 +1931 POINT(34.3855147016398 -117.5503759945511) bank1931 +1932 POINT(33.40851426498804 -117.28446701861985) bank1932 +1933 POINT(34.041831939785524 -117.82390872469799) bank1933 +1934 POINT(33.192129170195145 -118.94620101548048) bank1934 +1935 POINT(34.45388983607422 -118.08700034285754) bank1935 +1936 POINT(33.33616455526605 -117.27135225873616) bank1936 +1937 POINT(34.82784000324239 -118.56764405330112) bank1937 +1938 POINT(35.03602546636687 -118.06728288946941) bank1938 +1939 POINT(33.308949481088085 -117.64269845644348) bank1939 +1940 POINT(34.68161821190769 -118.57490875453064) bank1940 +1941 POINT(33.813389082666895 -118.64111254473403) bank1941 +1942 POINT(33.57695510794035 -117.5846052612842) bank1942 +1943 POINT(33.55863540136858 -117.45943634400088) bank1943 +1944 POINT(33.50490354011203 -117.7266184709646) bank1944 +1945 POINT(34.658124672511676 -118.09970275130944) bank1945 +1946 POINT(34.491231848515454 -118.82524245048596) bank1946 +1947 POINT(34.90784143990867 -117.27402486096621) bank1947 +1948 POINT(33.7971217301571 -118.10652128685042) bank1948 +1949 POINT(34.63714855168407 -117.54856900333631) bank1949 +1950 POINT(34.778035017341146 -118.27137791806885) bank1950 +1951 POINT(33.69096312413985 -118.41551819094327) bank1951 +1952 POINT(34.51171716790079 -118.21306222247841) bank1952 +1953 POINT(33.116901183738506 -117.75406416777008) bank1953 +1954 POINT(33.982984354385245 -118.34501619525841) bank1954 +1955 POINT(34.70088117322313 -119.00354879185723) bank1955 +1956 POINT(34.63616224039366 -118.16834174156712) bank1956 +1957 POINT(34.06890354451234 -118.23597500679425) bank1957 +1958 POINT(33.403012319183105 -117.72562672565161) bank1958 +1959 POINT(34.171606947727916 -118.9201727611254) bank1959 +1960 POINT(34.06369498735519 -118.27293638980416) bank1960 +1961 POINT(33.51428632986917 -118.45937617614636) bank1961 +1962 POINT(34.85933169682214 -118.95930693546947) bank1962 +1963 POINT(34.060168983815 -118.85713319079692) bank1963 +1964 POINT(33.597846515742916 -117.48171567801982) bank1964 +1965 POINT(33.84532116016244 -118.78626268127579) bank1965 +1966 POINT(33.18324789581849 -118.89051335127127) bank1966 +1967 POINT(34.18795283820521 -117.79520858151864) bank1967 +1968 POINT(33.87860548842953 -118.8954843564044) bank1968 +1969 POINT(33.36649282033624 -118.65389534567184) bank1969 +1970 POINT(34.724740144086496 -117.5834052104378) bank1970 +1971 POINT(33.74355080116406 -118.45042488691631) bank1971 +1972 POINT(33.95559584887111 -118.56861402991032) bank1972 +1973 POINT(33.61734338458542 -118.29256432182059) bank1973 +1974 POINT(33.716423448463296 -117.33287881602286) bank1974 +1975 POINT(33.165087653391154 -117.84377775039003) bank1975 +1976 POINT(33.513931380882205 -118.06368963457253) bank1976 +1977 POINT(33.62611162409634 -118.27653546717833) bank1977 +1978 POINT(33.460162705842336 -117.4635273905409) bank1978 +1979 POINT(34.575541289581295 -119.09907070493117) bank1979 +1980 POINT(34.006578477613374 -117.7762728823973) bank1980 +1981 POINT(33.19455403907388 -118.59265296019713) bank1981 +1982 POINT(34.75725149950798 -117.30915700843077) bank1982 +1983 POINT(34.9706349401463 -117.42954574889286) bank1983 +1984 POINT(34.47020377312414 -117.47891070204601) bank1984 +1985 POINT(33.35844916991643 -118.69245908841187) bank1985 +1986 POINT(33.690454761786775 -117.99565847092997) bank1986 +1987 POINT(33.08092664612881 -118.08978667604637) bank1987 +1988 POINT(34.76612197887079 -117.35404060157312) bank1988 +1989 POINT(34.77639656836016 -118.15866660122208) bank1989 +1990 POINT(33.4630957352378 -117.5079271799992) bank1990 +1991 POINT(33.93771380994176 -118.54545888775594) bank1991 +1992 POINT(34.611196668200996 -117.54674761742706) bank1992 +1993 POINT(34.852423723886645 -117.80840455132602) bank1993 +1994 POINT(33.1345322518028 -117.6601043489415) bank1994 +1995 POINT(34.97254465569966 -118.77934545860232) bank1995 +1996 POINT(34.51292231112764 -118.43979386752042) bank1996 +1997 POINT(33.865018354882714 -119.13049197663132) bank1997 +1998 POINT(34.2018729069324 -117.42937191772691) bank1998 +1999 POINT(33.164033047661555 -117.8565939689503) bank1999 +2000 POINT(33.598631036460255 -118.0163385637613) bank2000 +2001 POINT(33.20682450425093 -117.990951804005) bank2001 +2002 POINT(34.19777458392575 -117.2660888016605) bank2002 +2003 POINT(33.89269232986979 -118.62897862116002) bank2003 +2004 POINT(34.13840576916446 -118.04813489662196) bank2004 +2005 POINT(34.399898825981616 -118.2687756775149) bank2005 +2006 POINT(34.13845105285805 -119.16209866603991) bank2006 +2007 POINT(33.55049471770521 -117.59366245094117) bank2007 +2008 POINT(34.698141301983895 -117.43237428996969) bank2008 +2009 POINT(34.79619357656142 -117.66839984617143) bank2009 +2010 POINT(34.456820461248704 -117.4491810697894) bank2010 +2011 POINT(33.43853134014589 -117.76361465907003) bank2011 +2012 POINT(33.58054915062458 -118.85759008176609) bank2012 +2013 POINT(34.860680870962284 -118.13695146475874) bank2013 +2014 POINT(34.621014607051094 -118.94975983862864) bank2014 +2015 POINT(33.097461910587306 -119.1969736275003) bank2015 +2016 POINT(33.84656057426933 -117.83797061975167) bank2016 +2017 POINT(34.43209054510609 -117.32992558027136) bank2017 +2018 POINT(34.8252944661622 -118.26715098313908) bank2018 +2019 POINT(34.96913124982454 -117.45391027788675) bank2019 +2020 POINT(34.343637022224286 -118.98390850595378) bank2020 +2021 POINT(34.80002858132801 -118.60872940588827) bank2021 +2022 POINT(33.28767028169467 -119.16719119414479) bank2022 +2023 POINT(34.252500617956436 -119.08683472634723) bank2023 +2024 POINT(34.139945805353506 -118.93196618068953) bank2024 +2025 POINT(34.620130055581924 -118.655898278569) bank2025 +2026 POINT(33.88616624878631 -117.50973334932357) bank2026 +2027 POINT(34.21674296361099 -118.07475589038805) bank2027 +2028 POINT(34.26049950871913 -118.26416391307752) bank2028 +2029 POINT(33.54691194857936 -118.5502576073974) bank2029 +2030 POINT(34.75721713122115 -118.31788967768715) bank2030 +2031 POINT(34.055323169586586 -117.69482132542939) bank2031 +2032 POINT(34.0288129762355 -117.85748688574621) bank2032 +2033 POINT(33.09954647956784 -118.90901079622986) bank2033 +2034 POINT(34.30446234993091 -117.7345647405522) bank2034 +2035 POINT(34.91608047664957 -118.63603450730193) bank2035 +2036 POINT(33.919367410491546 -117.74993597110277) bank2036 +2037 POINT(33.070178086068864 -117.6227284242207) bank2037 +2038 POINT(33.81056846314248 -118.87904550641481) bank2038 +2039 POINT(34.32889125591257 -118.3368784300576) bank2039 +2040 POINT(34.95869247335595 -118.78645458139553) bank2040 +2041 POINT(34.82964697606006 -119.0489494754743) bank2041 +2042 POINT(33.75787072919111 -117.97264821534613) bank2042 +2043 POINT(33.31611725482107 -119.20558359050567) bank2043 +2044 POINT(34.78617832094988 -119.09583448693228) bank2044 +2045 POINT(33.60863634731411 -119.07386642697936) bank2045 +2046 POINT(33.75040812326719 -118.8998873778137) bank2046 +2047 POINT(34.834955062382384 -118.45126119949263) bank2047 +2048 POINT(33.44151371375406 -118.52833768003978) bank2048 +2049 POINT(33.066399990605845 -118.58882991355296) bank2049 +2050 POINT(34.151647818089856 -118.1111135271808) bank2050 +2051 POINT(33.74984345787692 -118.63319489656455) bank2051 +2052 POINT(33.51080286766544 -117.99431600575721) bank2052 +2053 POINT(34.63471046596734 -118.88382397908173) bank2053 +2054 POINT(34.84336204232532 -117.44101199125421) bank2054 +2055 POINT(34.72325903426518 -117.9490738874774) bank2055 +2056 POINT(33.61184859877947 -118.2698656590135) bank2056 +2057 POINT(33.15020588363171 -118.64247636301178) bank2057 +2058 POINT(33.612019572733104 -119.13074698687136) bank2058 +2059 POINT(34.249922788537965 -119.22653512785968) bank2059 +2060 POINT(35.04304636201149 -117.71811122876842) bank2060 +2061 POINT(33.715897403331226 -118.7180723476476) bank2061 +2062 POINT(33.128847965748676 -118.67117805472338) bank2062 +2063 POINT(34.1836997853633 -118.98914602536313) bank2063 +2064 POINT(34.48007236043854 -118.45004971276927) bank2064 +2065 POINT(34.70896882964722 -118.22611998719583) bank2065 +2066 POINT(34.54524445074934 -118.40270551321308) bank2066 +2067 POINT(34.1721664857171 -118.52081132237343) bank2067 +2068 POINT(33.22110107095062 -118.6621995027624) bank2068 +2069 POINT(33.457066535088934 -119.18841686024183) bank2069 +2070 POINT(33.98108968785111 -118.21324656446984) bank2070 +2071 POINT(34.91715102937579 -118.92833626132526) bank2071 +2072 POINT(33.72921448631876 -117.80108313255592) bank2072 +2073 POINT(34.28263293220176 -117.52795883749029) bank2073 +2074 POINT(33.690562976828424 -118.91233173949506) bank2074 +2075 POINT(33.34398880908853 -119.04856996651375) bank2075 +2076 POINT(33.72192056438791 -117.92531758024788) bank2076 +2077 POINT(34.499908171211054 -118.83323070097278) bank2077 +2078 POINT(34.841263063404156 -119.14678993612172) bank2078 +2079 POINT(33.39040591580663 -119.15182983688483) bank2079 +2080 POINT(33.92125174412154 -118.5466810081037) bank2080 +2081 POINT(33.63215045080774 -118.8962598873778) bank2081 +2082 POINT(33.623481400481204 -118.49943909668441) bank2082 +2083 POINT(34.36048236350735 -117.96442910166866) bank2083 +2084 POINT(33.81578718951492 -119.18754416881026) bank2084 +2085 POINT(33.62479159598523 -118.82627735419324) bank2085 +2086 POINT(34.28620585774845 -118.59639095191096) bank2086 +2087 POINT(33.778948722380484 -118.36100653591942) bank2087 +2088 POINT(34.05569379825888 -118.37135131633343) bank2088 +2089 POINT(33.67854462172532 -118.48994183412921) bank2089 +2090 POINT(34.62265361172355 -119.12571202320768) bank2090 +2091 POINT(34.619670808978 -117.9275748231514) bank2091 +2092 POINT(34.66136666452305 -118.69761644262279) bank2092 +2093 POINT(33.56276747254111 -119.18988061816012) bank2093 +2094 POINT(34.27832943405768 -118.50760904119707) bank2094 +2095 POINT(34.08966173556889 -118.84920370426794) bank2095 +2096 POINT(34.863780145879836 -117.61314741443215) bank2096 +2097 POINT(35.04750564326296 -119.21083894662318) bank2097 +2098 POINT(34.32518757940885 -118.25779948923861) bank2098 +2099 POINT(34.38454177225611 -117.52071558154742) bank2099 +2100 POINT(34.46237670199809 -118.69495360161754) bank2100 +2101 POINT(34.10275855789437 -118.62648689801554) bank2101 +2102 POINT(34.428298969405716 -117.27540932697858) bank2102 +2103 POINT(33.69519715669095 -118.17855930466885) bank2103 +2104 POINT(33.62229160318805 -117.58056545325489) bank2104 +2105 POINT(33.25387382427743 -118.67090515955248) bank2105 +2106 POINT(34.746880429767415 -118.85544931830297) bank2106 +2107 POINT(33.52330654861983 -117.89853158420252) bank2107 +2108 POINT(34.33330384030838 -117.45963665311764) bank2108 +2109 POINT(34.69190256947986 -118.40540546087408) bank2109 +2110 POINT(34.28556411691785 -118.57883046098078) bank2110 +2111 POINT(33.37190684235255 -118.4256783441724) bank2111 +2112 POINT(33.86582050296365 -117.40752483976473) bank2112 +2113 POINT(33.65921384285644 -118.06188746673689) bank2113 +2114 POINT(34.42457969952516 -117.50365723670004) bank2114 +2115 POINT(34.955121534310464 -119.04327631745531) bank2115 +2116 POINT(33.76016472241576 -119.19283614435844) bank2116 +2117 POINT(34.74073085371782 -117.99382450166338) bank2117 +2118 POINT(34.788019443609066 -118.05123992247992) bank2118 +2119 POINT(33.340396047578 -117.8161988701862) bank2119 +2120 POINT(33.528291517051116 -117.60209401714755) bank2120 +2121 POINT(34.274248762641406 -117.44644828266736) bank2121 +2122 POINT(33.89657534250015 -117.24670543605423) bank2122 +2123 POINT(33.17039633803081 -118.46967490433136) bank2123 +2124 POINT(33.218209219894646 -118.90844518736262) bank2124 +2125 POINT(35.04308664254644 -119.20580638417704) bank2125 +2126 POINT(35.01741556365165 -118.19210914615158) bank2126 +2127 POINT(34.55877621545773 -117.27842404362701) bank2127 +2128 POINT(34.42689224090448 -118.45707837215767) bank2128 +2129 POINT(34.29470946072178 -119.03033024599515) bank2129 +2130 POINT(33.989048285000386 -117.76784318934119) bank2130 +2131 POINT(33.540540125071736 -117.54362534345073) bank2131 +2132 POINT(33.76229837346972 -119.13191486713352) bank2132 +2133 POINT(34.42310997206266 -117.8783718652882) bank2133 +2134 POINT(34.306652265329284 -118.43939692767648) bank2134 +2135 POINT(33.72342042310393 -117.793852225708) bank2135 +2136 POINT(34.23534814176323 -117.25090543086974) bank2136 +2137 POINT(34.92436198745616 -118.87827309253126) bank2137 +2138 POINT(33.76933323886593 -118.96591262130335) bank2138 +2139 POINT(33.254401078916906 -118.05924745399318) bank2139 +2140 POINT(33.32516386597029 -118.70939311585019) bank2140 +2141 POINT(33.4080731662728 -118.47507078586747) bank2141 +2142 POINT(33.58126906933207 -118.16506909398045) bank2142 +2143 POINT(34.22145114799247 -117.52233313894762) bank2143 +2144 POINT(34.143350588378986 -119.01458645112659) bank2144 +2145 POINT(34.91863399225447 -118.20558139080636) bank2145 +2146 POINT(33.80453509604213 -118.50759794537345) bank2146 +2147 POINT(33.68940506097687 -117.98748840497807) bank2147 +2148 POINT(33.0930162024889 -119.22112038947286) bank2148 +2149 POINT(33.87346284977577 -118.83105247126714) bank2149 +2150 POINT(33.376111960340786 -117.99922135233842) bank2150 +2151 POINT(34.20970000500702 -118.8595924413014) bank2151 +2152 POINT(33.08700553737404 -117.5417697206881) bank2152 +2153 POINT(34.61789502531059 -118.12133482223969) bank2153 +2154 POINT(34.968278463083195 -117.54228563898732) bank2154 +2155 POINT(34.82214089800574 -118.1531797024528) bank2155 +2156 POINT(33.773219860020085 -118.104480927738) bank2156 +2157 POINT(34.92890112029691 -117.6574524867396) bank2157 +2158 POINT(33.069851097662124 -118.61459489585663) bank2158 +2159 POINT(33.863750330992275 -117.27834072364838) bank2159 +2160 POINT(35.02463115593332 -117.74650687127195) bank2160 +2161 POINT(33.332144942314926 -118.47812250362672) bank2161 +2162 POINT(34.223010701389875 -117.86429254493412) bank2162 +2163 POINT(33.72496569956768 -119.19403134451701) bank2163 +2164 POINT(34.76566356976082 -119.10623232912943) bank2164 +2165 POINT(33.32650195079345 -119.0013644425082) bank2165 +2166 POINT(34.97580383137171 -117.60609442149705) bank2166 +2167 POINT(33.07627723760074 -117.30784265249328) bank2167 +2168 POINT(33.26188423379234 -117.24621268525893) bank2168 +2169 POINT(35.01985088086699 -117.50305671571395) bank2169 +2170 POINT(33.626492739929006 -117.71573317020649) bank2170 +2171 POINT(34.73041730742166 -118.2816915983124) bank2171 +2172 POINT(34.4833247461812 -117.52117998420742) bank2172 +2173 POINT(33.14415633133684 -118.88739814454168) bank2173 +2174 POINT(33.985559433494025 -117.64723827633024) bank2174 +2175 POINT(33.41422283081502 -119.09553149122074) bank2175 +2176 POINT(33.73950431723785 -117.24558092762064) bank2176 +2177 POINT(34.32073860939335 -117.82892440075427) bank2177 +2178 POINT(33.5703719259665 -117.76674547895222) bank2178 +2179 POINT(33.291162453616835 -118.83534305294528) bank2179 +2180 POINT(33.74889160037991 -117.76048930681803) bank2180 +2181 POINT(33.28055824739424 -117.42083796764915) bank2181 +2182 POINT(33.78522178513488 -118.02176313846704) bank2182 +2183 POINT(34.565076753492804 -118.13653305703507) bank2183 +2184 POINT(33.60083326131678 -118.92554805304118) bank2184 +2185 POINT(33.385945350784574 -117.84955596601613) bank2185 +2186 POINT(33.967731069661255 -119.21038359039926) bank2186 +2187 POINT(33.96733302541339 -118.55699859727906) bank2187 +2188 POINT(34.29898776373397 -119.19083779810371) bank2188 +2189 POINT(34.28643843366336 -117.66717081628676) bank2189 +2190 POINT(34.664385625116026 -118.50697000631719) bank2190 +2191 POINT(34.193524735203276 -117.93382870306893) bank2191 +2192 POINT(33.67835368903245 -117.72962078966651) bank2192 +2193 POINT(34.10532423065995 -117.28492916396128) bank2193 +2194 POINT(33.64492351636536 -117.32744274922935) bank2194 +2195 POINT(33.51996760176124 -118.51356601304025) bank2195 +2196 POINT(33.88953845425247 -118.85216993890563) bank2196 +2197 POINT(33.288793713855746 -118.32653878834635) bank2197 +2198 POINT(34.522185546875946 -118.37942539358467) bank2198 +2199 POINT(34.3261456543334 -118.26787661294725) bank2199 +2200 POINT(34.431818856598056 -118.62796994093698) bank2200 +2201 POINT(33.68200694941188 -118.87023915628403) bank2201 +2202 POINT(33.64046899731772 -118.53441395491569) bank2202 +2203 POINT(34.541045651162705 -118.77630770575446) bank2203 +2204 POINT(34.450101486112956 -118.2420205790316) bank2204 +2205 POINT(33.77189195834079 -118.27751021212995) bank2205 +2206 POINT(34.4738860214339 -118.15650389448514) bank2206 +2207 POINT(34.35815183458861 -117.37261545211365) bank2207 +2208 POINT(33.71646254212087 -118.19965731512772) bank2208 +2209 POINT(34.268746281736945 -118.44133100718237) bank2209 +2210 POINT(33.5761896033166 -117.58784543396374) bank2210 +2211 POINT(34.576092698918764 -117.51462483767854) bank2211 +2212 POINT(34.50181573174983 -118.97556059711164) bank2212 +2213 POINT(33.90545877800765 -117.38901819524183) bank2213 +2214 POINT(34.02117405081655 -118.49690809493362) bank2214 +2215 POINT(34.46068888376326 -118.39476593051593) bank2215 +2216 POINT(34.718440792752624 -119.15219602306492) bank2216 +2217 POINT(33.91322019340703 -118.95956469107101) bank2217 +2218 POINT(33.486781141734426 -118.92813917632647) bank2218 +2219 POINT(33.914185078165495 -118.98249911129923) bank2219 +2220 POINT(35.020339958600964 -119.13715172149071) bank2220 +2221 POINT(34.09905897222835 -118.84499551062594) bank2221 +2222 POINT(33.1789653571441 -117.32561734317484) bank2222 +2223 POINT(33.069080122276034 -118.96895588842091) bank2223 +2224 POINT(34.91611577724817 -117.48111850371852) bank2224 +2225 POINT(34.033162039177036 -118.83991326097568) bank2225 +2226 POINT(34.264809361964744 -118.22916703411335) bank2226 +2227 POINT(33.2328745945727 -118.98976185317699) bank2227 +2228 POINT(34.01363757269977 -119.00369265783422) bank2228 +2229 POINT(34.205699077981684 -118.64212607946439) bank2229 +2230 POINT(33.77665875190049 -117.28196740065782) bank2230 +2231 POINT(34.76004063302434 -118.05056640583315) bank2231 +2232 POINT(34.04058747452165 -118.5877152492571) bank2232 +2233 POINT(34.60045673507274 -119.16994335215648) bank2233 +2234 POINT(33.49990277401558 -117.36292808090856) bank2234 +2235 POINT(34.8622344271156 -117.39216657075697) bank2235 +2236 POINT(33.20170377397976 -118.56377021659179) bank2236 +2237 POINT(33.63246361620545 -117.46803166896227) bank2237 +2238 POINT(33.704045789358545 -117.34833735418754) bank2238 +2239 POINT(34.53107607799404 -118.8347867939479) bank2239 +2240 POINT(34.89450464489465 -117.29492466762534) bank2240 +2241 POINT(34.24701295239292 -118.10631527465033) bank2241 +2242 POINT(33.934641399344386 -117.3863132390837) bank2242 +2243 POINT(33.79273490938465 -118.69419958002321) bank2243 +2244 POINT(34.280524033166536 -118.64922090069379) bank2244 +2245 POINT(33.864273123836576 -118.1632211299569) bank2245 +2246 POINT(33.77460267106511 -117.45584190912784) bank2246 +2247 POINT(34.64793301705663 -118.34010600937498) bank2247 +2248 POINT(34.979071286424194 -119.12828915005) bank2248 +2249 POINT(33.51172483028457 -118.83528356844043) bank2249 +2250 POINT(34.490602339828996 -117.66473405969428) bank2250 +2251 POINT(34.312800982078194 -118.99191031407847) bank2251 +2252 POINT(33.2426266824613 -117.3234895995317) bank2252 +2253 POINT(33.2489131153011 -117.59364288039693) bank2253 +2254 POINT(33.81342086850343 -117.39169488489068) bank2254 +2255 POINT(33.9816529646264 -118.78511199474858) bank2255 +2256 POINT(34.072056762796706 -119.24344481660458) bank2256 +2257 POINT(34.11828906236301 -117.52114781815621) bank2257 +2258 POINT(33.15684076190149 -117.57152904491566) bank2258 +2259 POINT(33.0860796155395 -118.61558134692062) bank2259 +2260 POINT(33.96450346390498 -118.9951491987305) bank2260 +2261 POINT(33.77485285579876 -117.82620962526288) bank2261 +2262 POINT(33.88238965999009 -118.4614497325142) bank2262 +2263 POINT(34.79109751365953 -117.504119036956) bank2263 +2264 POINT(33.30968537341859 -119.06193653373626) bank2264 +2265 POINT(34.35063158235433 -117.50292849403515) bank2265 +2266 POINT(34.8668767840226 -119.03525793635126) bank2266 +2267 POINT(33.243943059220875 -118.44546788520618) bank2267 +2268 POINT(34.832118851297395 -118.98861719689317) bank2268 +2269 POINT(33.11561756840517 -118.15710481368401) bank2269 +2270 POINT(35.047712942085155 -117.31305589132221) bank2270 +2271 POINT(33.84923555681885 -118.32850063279284) bank2271 +2272 POINT(34.36694466160627 -118.7284102210094) bank2272 +2273 POINT(33.336102891733695 -119.04641511459756) bank2273 +2274 POINT(33.59920061894209 -117.37617399995399) bank2274 +2275 POINT(34.20094550023791 -117.39568229855445) bank2275 +2276 POINT(33.88998272958185 -118.39508822330745) bank2276 +2277 POINT(34.424114728376374 -118.64839667382417) bank2277 +2278 POINT(33.9753209557584 -118.99618683968907) bank2278 +2279 POINT(33.179484170269305 -117.52342096566478) bank2279 +2280 POINT(34.922225670497774 -118.60959490411423) bank2280 +2281 POINT(33.15052566903106 -117.47282691167909) bank2281 +2282 POINT(35.04477267805951 -118.62964499333661) bank2282 +2283 POINT(34.547092430716035 -117.36256302830957) bank2283 +2284 POINT(33.564522947383054 -118.24039921177663) bank2284 +2285 POINT(34.602410998236444 -118.83802427708396) bank2285 +2286 POINT(33.92670716593854 -118.23717587176822) bank2286 +2287 POINT(34.14705743668608 -118.32633051373996) bank2287 +2288 POINT(34.00909548423593 -117.53006889391058) bank2288 +2289 POINT(34.03528419813641 -118.04788448216998) bank2289 +2290 POINT(33.09717067784464 -118.30399731349071) bank2290 +2291 POINT(34.124736397767336 -118.51719260948025) bank2291 +2292 POINT(33.500427330559475 -118.77985113298236) bank2292 +2293 POINT(33.124403622421035 -119.14394920717471) bank2293 +2294 POINT(34.77408150166903 -118.67818570400756) bank2294 +2295 POINT(33.35198718938731 -118.07013649245708) bank2295 +2296 POINT(34.41515662875076 -117.92786720913492) bank2296 +2297 POINT(33.45768254253849 -117.6554620219096) bank2297 +2298 POINT(34.0401394848945 -118.20743746823312) bank2298 +2299 POINT(33.51259331394949 -118.40658297302267) bank2299 +2300 POINT(34.85544582731764 -117.85853900562557) bank2300 +2301 POINT(34.2523076871522 -117.74625798154412) bank2301 +2302 POINT(35.01197013036856 -118.78537035676963) bank2302 +2303 POINT(33.20713660025124 -118.6206721929847) bank2303 +2304 POINT(33.34579744331912 -117.98194167212317) bank2304 +2305 POINT(33.33017144460908 -117.92065490066754) bank2305 +2306 POINT(33.07777296036428 -118.36698847092454) bank2306 +2307 POINT(34.297528414556496 -117.90541124317983) bank2307 +2308 POINT(33.35686715667225 -118.63691633150756) bank2308 +2309 POINT(34.7601187554281 -118.41927775522637) bank2309 +2310 POINT(34.190298862646124 -118.6881495865255) bank2310 +2311 POINT(34.970079948303365 -117.53158018471214) bank2311 +2312 POINT(34.58204581115968 -119.19201918183072) bank2312 +2313 POINT(33.85041977517475 -117.35229657215899) bank2313 +2314 POINT(34.440513359921084 -118.79664555088502) bank2314 +2315 POINT(33.82263242290869 -118.35350852310751) bank2315 +2316 POINT(33.965939298064164 -118.24524767834055) bank2316 +2317 POINT(34.21882220072381 -117.54291921083289) bank2317 +2318 POINT(34.37162664911244 -119.18154148530765) bank2318 +2319 POINT(34.30605965580864 -117.77057646240604) bank2319 +2320 POINT(34.8869405334705 -119.1620575816733) bank2320 +2321 POINT(34.2533071213307 -118.04595271941345) bank2321 +2322 POINT(34.61568653344563 -118.27069629588577) bank2322 +2323 POINT(34.42192495581503 -119.18428250742897) bank2323 +2324 POINT(34.758797758294534 -119.02476670099968) bank2324 +2325 POINT(33.194024308660325 -119.1642212693214) bank2325 +2326 POINT(33.1361097830798 -118.27906723635607) bank2326 +2327 POINT(34.89931202933811 -118.69740085866609) bank2327 +2328 POINT(33.59595810181856 -118.70412231662854) bank2328 +2329 POINT(33.093642479794056 -118.32870733505425) bank2329 +2330 POINT(33.24596588840987 -118.7340195262995) bank2330 +2331 POINT(33.340712991450246 -118.48798144662314) bank2331 +2332 POINT(33.45588461724663 -118.1600017391417) bank2332 +2333 POINT(35.00428264099574 -118.84052860312136) bank2333 +2334 POINT(34.72855926148272 -117.4716750010745) bank2334 +2335 POINT(33.883832536085336 -117.42885314049828) bank2335 +2336 POINT(33.3108431469517 -118.03503788329921) bank2336 +2337 POINT(34.80658583004572 -118.0874793383872) bank2337 +2338 POINT(33.75754763680231 -117.54093380110697) bank2338 +2339 POINT(33.85409485934865 -118.29852955692773) bank2339 +2340 POINT(34.109296270301364 -117.42022074132275) bank2340 +2341 POINT(33.30216730715025 -117.50935416943129) bank2341 +2342 POINT(34.81939866334809 -118.91199075651755) bank2342 +2343 POINT(34.578263592001136 -118.09659575879297) bank2343 +2344 POINT(33.76362348540546 -118.13866318540497) bank2344 +2345 POINT(34.40969164174537 -119.2151312416428) bank2345 +2346 POINT(34.07241439029023 -117.61203341877318) bank2346 +2347 POINT(35.00625633128522 -117.41074169212146) bank2347 +2348 POINT(34.7745294615772 -118.59320167365733) bank2348 +2349 POINT(34.986096624095026 -117.91146248334428) bank2349 +2350 POINT(33.328128377894366 -117.84018560394883) bank2350 +2351 POINT(34.42804654829181 -119.1338572649278) bank2351 +2352 POINT(34.98292343524233 -118.73971013533927) bank2352 +2353 POINT(34.08244570149413 -119.13060584091464) bank2353 +2354 POINT(33.17460253288144 -117.32139213436214) bank2354 +2355 POINT(33.18367217623226 -118.38514831503886) bank2355 +2356 POINT(34.8008746067666 -119.10994562709118) bank2356 +2357 POINT(34.20998915744438 -118.90307366587766) bank2357 +2358 POINT(34.498651833465686 -117.69541381646602) bank2358 +2359 POINT(34.34641445758968 -117.83695331684329) bank2359 +2360 POINT(34.35545109159272 -117.6003533580095) bank2360 +2361 POINT(33.549907289756796 -117.61145041322716) bank2361 +2362 POINT(34.02526729789706 -118.3797949498833) bank2362 +2363 POINT(35.021522779196395 -118.6214038362262) bank2363 +2364 POINT(33.96748671416603 -118.22030949201755) bank2364 +2365 POINT(34.24617933530179 -117.31437974959671) bank2365 +2366 POINT(33.12854989717668 -117.4151923672026) bank2366 +2367 POINT(34.63984980381957 -117.98664289088578) bank2367 +2368 POINT(33.96160981886265 -118.47557646594514) bank2368 +2369 POINT(33.81889911082905 -118.09104711469098) bank2369 +2370 POINT(33.35827874811731 -118.25290757454196) bank2370 +2371 POINT(34.50217471008397 -117.92633297112627) bank2371 +2372 POINT(34.02352480049705 -118.415310173544) bank2372 +2373 POINT(34.92183611979213 -117.46303397658822) bank2373 +2374 POINT(33.78825357924231 -119.10169028558983) bank2374 +2375 POINT(33.6243261322712 -117.86974079327084) bank2375 +2376 POINT(34.9264046125685 -117.29060780420183) bank2376 +2377 POINT(33.103558465592016 -118.55364164043428) bank2377 +2378 POINT(34.670663378343704 -118.3678101811913) bank2378 +2379 POINT(33.540002240411866 -119.09056164318348) bank2379 +2380 POINT(34.786719705298154 -117.48178857813437) bank2380 +2381 POINT(34.54196057621155 -118.20459556368972) bank2381 +2382 POINT(34.501342630785466 -117.46360903652935) bank2382 +2383 POINT(34.629684128901445 -117.388951870845) bank2383 +2384 POINT(33.203821402039004 -118.56070390609958) bank2384 +2385 POINT(34.57883228852699 -117.81519623319397) bank2385 +2386 POINT(34.18126386887265 -118.10385305443025) bank2386 +2387 POINT(33.41450887497876 -118.98387041442281) bank2387 +2388 POINT(33.07075773770354 -118.52778019453336) bank2388 +2389 POINT(34.79046716393902 -118.8790473434645) bank2389 +2390 POINT(34.64102959885984 -119.16831395192604) bank2390 +2391 POINT(34.20461372447015 -117.31042073814197) bank2391 +2392 POINT(34.38939845756865 -117.66466476261058) bank2392 +2393 POINT(33.671709042723094 -117.4189014444284) bank2393 +2394 POINT(34.913216776970344 -118.73821213062826) bank2394 +2395 POINT(34.176298730196734 -117.34683860980168) bank2395 +2396 POINT(34.847349605357685 -118.33738928462034) bank2396 +2397 POINT(33.45517317724369 -117.42609404656534) bank2397 +2398 POINT(34.514777034565284 -117.27725205229046) bank2398 +2399 POINT(34.289987702676186 -119.23205511152888) bank2399 +2400 POINT(33.60300363580209 -117.73692926847751) bank2400 +2401 POINT(33.547599575840756 -119.21008561244479) bank2401 +2402 POINT(34.22234525643417 -117.40217368020755) bank2402 +2403 POINT(34.62272894432225 -117.72779607926708) bank2403 +2404 POINT(33.899244440534794 -118.42806466485125) bank2404 +2405 POINT(33.42512805232621 -118.57335452905377) bank2405 +2406 POINT(35.013863237168955 -118.31487940680805) bank2406 +2407 POINT(33.794047474880834 -117.35187043726467) bank2407 +2408 POINT(33.33798609728394 -118.77797466825504) bank2408 +2409 POINT(34.92629832025852 -118.84560820010861) bank2409 +2410 POINT(34.67688720263241 -118.0594377490696) bank2410 +2411 POINT(34.34083896963092 -117.5258086445821) bank2411 +2412 POINT(33.220381444318 -118.22938037344036) bank2412 +2413 POINT(33.24089254526323 -118.71616138089702) bank2413 +2414 POINT(33.50729861252415 -118.3087410411233) bank2414 +2415 POINT(34.04942847204606 -118.38248066104107) bank2415 +2416 POINT(34.42907193661322 -117.7679053481381) bank2416 +2417 POINT(35.027467670245066 -119.1411579004731) bank2417 +2418 POINT(34.98159238702632 -117.61273759231666) bank2418 +2419 POINT(33.59002380521686 -117.48994573575672) bank2419 +2420 POINT(33.94700837995896 -118.04506840283366) bank2420 +2421 POINT(34.715556732764064 -117.25830819907375) bank2421 +2422 POINT(34.7318695669148 -117.939094850301) bank2422 +2423 POINT(33.09462201154833 -118.36715449020934) bank2423 +2424 POINT(33.10130247426529 -118.78336463116331) bank2424 +2425 POINT(33.08433206942354 -117.87171605651305) bank2425 +2426 POINT(34.612728835696245 -117.29511147866864) bank2426 +2427 POINT(34.35166972314308 -118.24922279506913) bank2427 +2428 POINT(33.1212894661137 -118.60528218208165) bank2428 +2429 POINT(33.996695570246686 -117.56113607266981) bank2429 +2430 POINT(34.27991014916217 -117.77123168654691) bank2430 +2431 POINT(34.668622632525604 -118.88444993340623) bank2431 +2432 POINT(34.46811382913937 -118.45586386265312) bank2432 +2433 POINT(34.84940480183364 -118.49294714655647) bank2433 +2434 POINT(34.853815470069485 -118.15516888352546) bank2434 +2435 POINT(33.26877268715005 -118.11187253538232) bank2435 +2436 POINT(35.023579015800564 -117.90642278206191) bank2436 +2437 POINT(33.95916988383108 -117.40466337748077) bank2437 +2438 POINT(33.4018222060084 -118.41985188818458) bank2438 +2439 POINT(34.16513535433983 -117.67299523452492) bank2439 +2440 POINT(34.50964104468489 -117.61109338892503) bank2440 +2441 POINT(34.2523256510575 -119.1955824584817) bank2441 +2442 POINT(34.206875106794875 -117.40165927025073) bank2442 +2443 POINT(33.952414397921984 -117.51276485177424) bank2443 +2444 POINT(33.288849057460055 -118.88723831573479) bank2444 +2445 POINT(34.07679881786516 -118.01127222465136) bank2445 +2446 POINT(34.5527961709736 -118.74774061219566) bank2446 +2447 POINT(33.947066201001945 -117.51614919919491) bank2447 +2448 POINT(34.23778092838335 -118.5635201504038) bank2448 +2449 POINT(33.488952345675145 -117.4579012161085) bank2449 +2450 POINT(34.41288318389423 -118.8964864099349) bank2450 +2451 POINT(34.85152705541821 -117.4225560892966) bank2451 +2452 POINT(33.79391509743464 -117.63608673266961) bank2452 +2453 POINT(33.94085978695319 -119.06665565121243) bank2453 +2454 POINT(33.94824205736604 -117.54295204898129) bank2454 +2455 POINT(34.60339273898001 -117.98226653854707) bank2455 +2456 POINT(33.59053053592876 -118.84988202179426) bank2456 +2457 POINT(34.06083782118116 -117.489665717952) bank2457 +2458 POINT(33.64957513995686 -118.09277663606827) bank2458 +2459 POINT(33.43096403591462 -118.94687569065023) bank2459 +2460 POINT(34.12096437734883 -117.79896556911623) bank2460 +2461 POINT(34.11176621883266 -118.56892554400856) bank2461 +2462 POINT(34.83000651260523 -119.12078313074142) bank2462 +2463 POINT(34.0150045543722 -119.06325759474902) bank2463 +2464 POINT(34.64841318107962 -118.01901123070476) bank2464 +2465 POINT(34.40981570818336 -118.69027777568967) bank2465 +2466 POINT(34.29591736972104 -118.04100889027667) bank2466 +2467 POINT(34.243504711886494 -119.13680783171776) bank2467 +2468 POINT(33.573833845288526 -119.20943967921508) bank2468 +2469 POINT(34.01535375559341 -118.7889754199009) bank2469 +2470 POINT(34.06365517838468 -118.48949830712361) bank2470 +2471 POINT(33.72596888808032 -117.83537768362632) bank2471 +2472 POINT(34.81079530550589 -117.99104281950943) bank2472 +2473 POINT(34.212692106566415 -117.31641030624411) bank2473 +2474 POINT(33.12105726942312 -119.0970382419985) bank2474 +2475 POINT(33.83481808608549 -118.27259966286975) bank2475 +2476 POINT(34.73589896757514 -118.43263634182864) bank2476 +2477 POINT(33.3515630005512 -118.02394896440786) bank2477 +2478 POINT(33.64109863515189 -117.85773167471282) bank2478 +2479 POINT(33.20952929488842 -117.95780765901094) bank2479 +2480 POINT(33.61950269745877 -117.57713596196947) bank2480 +2481 POINT(34.36526448619332 -118.24031572796328) bank2481 +2482 POINT(34.565616697810384 -117.34122771151029) bank2482 +2483 POINT(34.91203781915383 -118.85126306058714) bank2483 +2484 POINT(34.376381368389744 -118.67753388236284) bank2484 +2485 POINT(34.97794819521157 -118.35227147404487) bank2485 +2486 POINT(33.46112294838741 -118.70177798326449) bank2486 +2487 POINT(33.94915416424693 -119.04356739285441) bank2487 +2488 POINT(33.67178377743512 -118.17134220098762) bank2488 +2489 POINT(33.623603924568734 -117.63694197911265) bank2489 +2490 POINT(34.548339825204586 -119.07603759708617) bank2490 +2491 POINT(33.1178923258726 -118.88308325567742) bank2491 +2492 POINT(33.138782115870185 -119.04387892750522) bank2492 +2493 POINT(35.0041895404872 -117.35230408374929) bank2493 +2494 POINT(34.924118110358684 -118.90615465319313) bank2494 +2495 POINT(34.51399671063363 -117.30691213699646) bank2495 +2496 POINT(34.24725353507066 -118.62376567570668) bank2496 +2497 POINT(34.59757172835396 -117.6469931575217) bank2497 +2498 POINT(34.54399804086547 -118.8784029150311) bank2498 +2499 POINT(34.04192980321437 -118.16407624174187) bank2499 +2500 POINT(34.42216619536653 -119.06321844403182) bank2500 +2501 POINT(34.17621700803352 -118.80803014101912) bank2501 +2502 POINT(33.70257539053619 -119.19068788664234) bank2502 +2503 POINT(33.48232368847726 -118.64319745854777) bank2503 +2504 POINT(33.11892687348764 -118.79886560854486) bank2504 +2505 POINT(33.08459700971434 -118.45083369461064) bank2505 +2506 POINT(34.44633115370329 -119.17765561942828) bank2506 +2507 POINT(33.82306035933011 -118.92594039584743) bank2507 +2508 POINT(33.903334906269194 -118.3722042493615) bank2508 +2509 POINT(33.296359242982675 -118.63679324617362) bank2509 +2510 POINT(34.26432716000793 -118.56491795758474) bank2510 +2511 POINT(33.69577238447006 -117.4635592096633) bank2511 +2512 POINT(34.71672402498305 -118.57995454188182) bank2512 +2513 POINT(33.98190802740218 -118.37874405117262) bank2513 +2514 POINT(33.84210543687544 -118.68052029723265) bank2514 +2515 POINT(33.84681972027439 -117.85310349838474) bank2515 +2516 POINT(34.982039388619654 -117.45399138086145) bank2516 +2517 POINT(34.14517936090168 -117.34690477965981) bank2517 +2518 POINT(34.56456332324958 -117.77843791849413) bank2518 +2519 POINT(33.59677449362656 -118.38073104123765) bank2519 +2520 POINT(34.79138861560379 -118.57847244343711) bank2520 +2521 POINT(34.20424458271371 -118.85528176067562) bank2521 +2522 POINT(33.216549441820725 -117.44627244505129) bank2522 +2523 POINT(34.57817237096733 -117.61582871939282) bank2523 +2524 POINT(34.033569890899955 -117.8498640217882) bank2524 +2525 POINT(34.67620974796312 -118.40570453442072) bank2525 +2526 POINT(33.559575349914674 -117.77221426753721) bank2526 +2527 POINT(34.82618453387795 -118.99654015262483) bank2527 +2528 POINT(33.24436652347862 -118.80271815131488) bank2528 +2529 POINT(34.38450405477933 -118.2906606680801) bank2529 +2530 POINT(33.84637889922714 -117.39678635643244) bank2530 +2531 POINT(33.640846726971944 -117.33707283774454) bank2531 +2532 POINT(34.635038534011116 -118.57005099802798) bank2532 +2533 POINT(33.65333087163274 -117.67343171933248) bank2533 +2534 POINT(34.90743939276131 -118.10984642475003) bank2534 +2535 POINT(34.00331275714314 -117.9979028830165) bank2535 +2536 POINT(34.274386792590896 -117.27266888001456) bank2536 +2537 POINT(34.146806692288045 -118.13737075185537) bank2537 +2538 POINT(33.182908170597386 -118.79918130593785) bank2538 +2539 POINT(34.38328256069243 -118.49470597247594) bank2539 +2540 POINT(34.99780914662655 -118.30620578134972) bank2540 +2541 POINT(34.0485306419077 -117.27317002321105) bank2541 +2542 POINT(34.57496734682263 -119.19033944600221) bank2542 +2543 POINT(34.40108931211105 -117.48078208676843) bank2543 +2544 POINT(34.72834013061841 -119.10635453390245) bank2544 +2545 POINT(33.22447268517836 -118.18506989249803) bank2545 +2546 POINT(34.95975456927213 -118.18344923975286) bank2546 +2547 POINT(33.8916756392888 -118.84137838673833) bank2547 +2548 POINT(34.87528320627503 -118.67442022623491) bank2548 +2549 POINT(34.68113314361455 -118.57437664680937) bank2549 +2550 POINT(33.195228978860044 -118.12580940904378) bank2550 +2551 POINT(33.91835596865129 -117.98253184651503) bank2551 +2552 POINT(34.0812676744108 -118.82473946701701) bank2552 +2553 POINT(33.9996829791259 -118.78016371617991) bank2553 +2554 POINT(34.23521833383196 -119.07865610138128) bank2554 +2555 POINT(33.493255062811286 -117.96200420717928) bank2555 +2556 POINT(34.792512233490456 -118.80465251578084) bank2556 +2557 POINT(34.278002213144255 -117.87747055669419) bank2557 +2558 POINT(33.28223367953843 -118.08125548297896) bank2558 +2559 POINT(34.17010793686285 -117.79765639550045) bank2559 +2560 POINT(33.700064911710356 -117.44434020893353) bank2560 +2561 POINT(33.94494502749057 -118.5383774739669) bank2561 +2562 POINT(33.51141488144891 -117.78529059966522) bank2562 +2563 POINT(34.62791053646016 -119.20700978592227) bank2563 +2564 POINT(33.330519632018856 -117.78523511119141) bank2564 +2565 POINT(33.69881441718169 -118.41118856100076) bank2565 +2566 POINT(33.51278924632073 -119.21508711553996) bank2566 +2567 POINT(33.23865138362098 -118.16723693268925) bank2567 +2568 POINT(34.6321826712211 -118.83245601919273) bank2568 +2569 POINT(34.456985728119065 -118.90693298419102) bank2569 +2570 POINT(34.030335099266885 -118.06215020533267) bank2570 +2571 POINT(34.37141204224446 -118.21462447089297) bank2571 +2572 POINT(34.86490689708476 -118.1225661281252) bank2572 +2573 POINT(34.52188503850004 -118.70347511264936) bank2573 +2574 POINT(33.4530043471632 -117.97325921786032) bank2574 +2575 POINT(34.39502980189813 -117.34826253189891) bank2575 +2576 POINT(34.061966637569476 -117.37899786939037) bank2576 +2577 POINT(34.92903513811188 -118.75299149276097) bank2577 +2578 POINT(34.93770588495397 -118.37411848879897) bank2578 +2579 POINT(34.27367124594598 -118.14795382437201) bank2579 +2580 POINT(33.989525339702915 -118.82048807415205) bank2580 +2581 POINT(33.33787875264259 -117.70369262361096) bank2581 +2582 POINT(33.92134815391271 -117.30912381290698) bank2582 +2583 POINT(33.5476905958191 -117.6510411766995) bank2583 +2584 POINT(34.70276950717685 -117.83342009115992) bank2584 +2585 POINT(34.87407834003611 -118.36795222025626) bank2585 +2586 POINT(33.59356430246244 -119.20221299605237) bank2586 +2587 POINT(34.8589021686587 -117.63132763863632) bank2587 +2588 POINT(34.697224418070746 -117.55207776131232) bank2588 +2589 POINT(34.44469759932004 -118.28626686297885) bank2589 +2590 POINT(33.39505971965523 -117.66066027498223) bank2590 +2591 POINT(33.075604962167674 -117.94374424385978) bank2591 +2592 POINT(33.66402624539053 -117.36270268668734) bank2592 +2593 POINT(33.71061137807822 -117.57889688211593) bank2593 +2594 POINT(33.51616718274218 -118.13624609515546) bank2594 +2595 POINT(33.649020769929415 -117.81793274212075) bank2595 +2596 POINT(34.42482380721462 -118.3349089506151) bank2596 +2597 POINT(34.11203643368414 -118.74086948667288) bank2597 +2598 POINT(34.49744228045994 -117.30312706924163) bank2598 +2599 POINT(33.25492280881629 -119.05974241909453) bank2599 +2600 POINT(33.20647968712764 -118.40351940156478) bank2600 +2601 POINT(33.81318558016396 -118.27374810754091) bank2601 +2602 POINT(34.964457223129365 -118.51782519188865) bank2602 +2603 POINT(34.23470172017407 -117.73943799270354) bank2603 +2604 POINT(34.53030732099493 -118.28944027866186) bank2604 +2605 POINT(33.66591020860418 -117.54553088907997) bank2605 +2606 POINT(33.47351305094374 -118.8466443471034) bank2606 +2607 POINT(33.87267955702668 -118.23203335907074) bank2607 +2608 POINT(34.754694369204 -117.85226501183693) bank2608 +2609 POINT(34.498144389716 -118.57792798512438) bank2609 +2610 POINT(34.4261053557773 -118.15847230070918) bank2610 +2611 POINT(34.58090333927138 -118.82784830544065) bank2611 +2612 POINT(34.8722764782728 -117.68828741021835) bank2612 +2613 POINT(33.47656954088854 -118.71544645036158) bank2613 +2614 POINT(33.202099700395486 -117.6462148917534) bank2614 +2615 POINT(34.43610119078519 -119.10966114457925) bank2615 +2616 POINT(33.59305906366949 -119.238237908249) bank2616 +2617 POINT(33.68975616698571 -119.03152026271742) bank2617 +2618 POINT(33.62577093699259 -117.52693159116065) bank2618 +2619 POINT(33.48590457127984 -118.89439376751352) bank2619 +2620 POINT(34.59256283182963 -117.97082839293985) bank2620 +2621 POINT(33.49934829292477 -118.84328709839076) bank2621 +2622 POINT(33.48637757125634 -118.09297437680604) bank2622 +2623 POINT(34.79965182942537 -118.97947881724548) bank2623 +2624 POINT(34.319052408414876 -119.2343675380737) bank2624 +2625 POINT(33.600438617718716 -117.4761281473446) bank2625 +2626 POINT(34.88091629897091 -118.01496580121324) bank2626 +2627 POINT(33.878641564920954 -117.25469440103012) bank2627 +2628 POINT(34.07936761135346 -117.6032328980729) bank2628 +2629 POINT(33.75556856534727 -117.68360803011906) bank2629 +2630 POINT(34.03086714179537 -117.79076593636535) bank2630 +2631 POINT(33.762208865903595 -118.17541967596817) bank2631 +2632 POINT(34.58803386675582 -118.13513008227791) bank2632 +2633 POINT(33.942157773591504 -117.62301240892295) bank2633 +2634 POINT(33.86128941559661 -119.095191577333) bank2634 +2635 POINT(34.19055727347072 -118.81274265182996) bank2635 +2636 POINT(33.95639618379843 -118.94213666458245) bank2636 +2637 POINT(34.2526399693376 -118.40810782470939) bank2637 +2638 POINT(34.45923218158888 -118.956950687076) bank2638 +2639 POINT(33.99602321266353 -118.49292639666048) bank2639 +2640 POINT(34.985810324215215 -118.80408080849435) bank2640 +2641 POINT(34.53631625984062 -117.94650240176634) bank2641 +2642 POINT(34.649534840031926 -117.44731970734915) bank2642 +2643 POINT(34.81157992888024 -119.23723459103593) bank2643 +2644 POINT(34.58202348982579 -118.44698615067009) bank2644 +2645 POINT(33.96420099957038 -118.16779387924909) bank2645 +2646 POINT(34.78460608087956 -117.58674041246697) bank2646 +2647 POINT(33.74186897281778 -118.49945176926192) bank2647 +2648 POINT(33.835851255345126 -117.40384060851862) bank2648 +2649 POINT(33.16211667388662 -118.41756550517462) bank2649 +2650 POINT(34.93183563424352 -118.74019101959998) bank2650 +2651 POINT(34.9012639163592 -118.08206322412956) bank2651 +2652 POINT(34.400449893236 -117.69137527249806) bank2652 +2653 POINT(33.19319361675788 -117.31783358014098) bank2653 +2654 POINT(34.40243309225051 -118.51673730323817) bank2654 +2655 POINT(33.626333309201705 -119.19965932768618) bank2655 +2656 POINT(34.285959078728325 -118.7751937737246) bank2656 +2657 POINT(33.276714944954364 -118.03880242758599) bank2657 +2658 POINT(33.7240373468186 -117.98701291946969) bank2658 +2659 POINT(34.71614493985022 -118.08439123288987) bank2659 +2660 POINT(34.70120637855661 -117.72028634736371) bank2660 +2661 POINT(33.47252597113943 -117.4813594213983) bank2661 +2662 POINT(34.32165044561389 -117.75346201528168) bank2662 +2663 POINT(34.39939905872512 -117.97046825051424) bank2663 +2664 POINT(34.67959217637121 -118.16670843835858) bank2664 +2665 POINT(33.47904027329723 -117.82209828274756) bank2665 +2666 POINT(34.25659201933453 -117.83652211907057) bank2666 +2667 POINT(33.41497478044439 -119.11258204880987) bank2667 +2668 POINT(33.62046866592335 -117.2466671765259) bank2668 +2669 POINT(34.698253324047926 -118.20240136894732) bank2669 +2670 POINT(33.733328569727554 -117.68465099164449) bank2670 +2671 POINT(33.679549545971554 -118.62922530663002) bank2671 +2672 POINT(33.17476147396053 -117.37728412839989) bank2672 +2673 POINT(34.00041779747449 -118.14679805667252) bank2673 +2674 POINT(33.089427623481804 -118.55391833833652) bank2674 +2675 POINT(33.48870168290301 -117.54273495527414) bank2675 +2676 POINT(33.14240607930031 -118.26813492838068) bank2676 +2677 POINT(34.2687555438438 -118.46862089665946) bank2677 +2678 POINT(33.83587249517269 -119.17544606670499) bank2678 +2679 POINT(33.93536172576513 -118.83093458443425) bank2679 +2680 POINT(33.833104159586085 -118.72787954532352) bank2680 +2681 POINT(34.01440336417781 -118.43430396209025) bank2681 +2682 POINT(33.7751204388483 -118.58331275684819) bank2682 +2683 POINT(34.42627995647448 -119.1514876217052) bank2683 +2684 POINT(33.945880098861785 -119.17993025523386) bank2684 +2685 POINT(33.1759484897172 -117.64464558377293) bank2685 +2686 POINT(34.543618955271846 -117.62397300960569) bank2686 +2687 POINT(33.20313211418126 -118.28341430300088) bank2687 +2688 POINT(33.26403177530555 -118.3542482041407) bank2688 +2689 POINT(34.64409000980116 -118.67633039232427) bank2689 +2690 POINT(33.43231699730051 -117.82762388649712) bank2690 +2691 POINT(34.772069057063646 -118.06213182980059) bank2691 +2692 POINT(33.777646003442605 -118.39664624766127) bank2692 +2693 POINT(34.34403639584884 -118.56293244378318) bank2693 +2694 POINT(34.37967319496005 -119.21972382609042) bank2694 +2695 POINT(33.190882819333176 -118.02368050710446) bank2695 +2696 POINT(33.47550255257741 -117.96645109207715) bank2696 +2697 POINT(35.042517526474555 -118.69475070795961) bank2697 +2698 POINT(33.95690929641003 -117.76804491159224) bank2698 +2699 POINT(33.2862551999168 -119.19654403806452) bank2699 +2700 POINT(33.77644302737117 -117.62656188286255) bank2700 +2701 POINT(34.00127827632975 -117.28233939788974) bank2701 +2702 POINT(33.96715974846493 -119.19530067479617) bank2702 +2703 POINT(34.81804267052271 -118.58018467581806) bank2703 +2704 POINT(34.04931199112185 -119.08981801687757) bank2704 +2705 POINT(33.280180916507135 -118.28424636015103) bank2705 +2706 POINT(34.79504040364198 -117.62856821873962) bank2706 +2707 POINT(34.136058113080715 -118.91019512623107) bank2707 +2708 POINT(33.348201038622165 -117.65394597593527) bank2708 +2709 POINT(33.33485889959194 -117.41042271009619) bank2709 +2710 POINT(33.22713124745045 -117.76555629294089) bank2710 +2711 POINT(33.59479325673305 -117.59207923460224) bank2711 +2712 POINT(33.61575801297737 -118.65753188402414) bank2712 +2713 POINT(33.897856601993205 -117.54829417659546) bank2713 +2714 POINT(34.887724203499225 -119.12296728942441) bank2714 +2715 POINT(34.77113574670504 -118.78135669365875) bank2715 +2716 POINT(33.91247758074045 -119.18785039247174) bank2716 +2717 POINT(34.755158459902 -118.2473356838208) bank2717 +2718 POINT(34.415411889345876 -117.84557515444159) bank2718 +2719 POINT(34.31275302934646 -119.0701641097531) bank2719 +2720 POINT(33.07571537324903 -118.39814476095341) bank2720 +2721 POINT(33.96054150239965 -118.44115513076488) bank2721 +2722 POINT(33.713310116586285 -117.87611871111733) bank2722 +2723 POINT(33.12848605431582 -117.73442895078445) bank2723 +2724 POINT(34.62197686606269 -118.69552818865142) bank2724 +2725 POINT(33.68194251877804 -117.86026412481154) bank2725 +2726 POINT(33.36208428760439 -117.84591021358527) bank2726 +2727 POINT(33.88006832789386 -117.82304191754295) bank2727 +2728 POINT(34.44564385599336 -117.83356290537625) bank2728 +2729 POINT(33.86952882938154 -118.4503036127348) bank2729 +2730 POINT(34.40381207077043 -119.03656240206472) bank2730 +2731 POINT(34.24635304738426 -117.9580812947937) bank2731 +2732 POINT(34.622090829802346 -118.93761296810867) bank2732 +2733 POINT(33.6199067277662 -117.9041208442221) bank2733 +2734 POINT(33.10971147366698 -117.93793445256526) bank2734 +2735 POINT(34.96585812254798 -119.01506200563678) bank2735 +2736 POINT(33.263252673735636 -117.81320601634498) bank2736 +2737 POINT(34.68019386118341 -118.07533790346832) bank2737 +2738 POINT(34.74805945409075 -118.0238646490673) bank2738 +2739 POINT(33.32005780977968 -119.05468429093614) bank2739 +2740 POINT(33.60278796819129 -117.27926126075957) bank2740 +2741 POINT(33.69381751539841 -117.35830735771019) bank2741 +2742 POINT(34.55909517085285 -119.03459885758693) bank2742 +2743 POINT(34.46567937316387 -117.67267935675622) bank2743 +2744 POINT(34.37466011283466 -118.26047092334646) bank2744 +2745 POINT(34.232620026269 -118.13337134102038) bank2745 +2746 POINT(33.184769987592524 -118.5649561375097) bank2746 +2747 POINT(34.487778018654126 -117.99687294292877) bank2747 +2748 POINT(33.287931820328346 -118.02008664409541) bank2748 +2749 POINT(34.19658816255277 -118.81975942726142) bank2749 +2750 POINT(33.933274032916486 -117.55009752240888) bank2750 +2751 POINT(33.34712533419923 -118.44950059091171) bank2751 +2752 POINT(34.02785996606005 -118.03627539315262) bank2752 +2753 POINT(33.50274571307089 -118.66301348028006) bank2753 +2754 POINT(33.84635152405176 -118.42599781896915) bank2754 +2755 POINT(33.439023767078965 -117.60849125415491) bank2755 +2756 POINT(34.924513002564446 -118.8613309196298) bank2756 +2757 POINT(33.339225592327246 -118.87901170680685) bank2757 +2758 POINT(34.007096734980244 -118.04376668338101) bank2758 +2759 POINT(34.17208417175708 -118.53486808960086) bank2759 +2760 POINT(33.5124418072869 -118.98878120635707) bank2760 +2761 POINT(33.27772040286912 -117.73594232889744) bank2761 +2762 POINT(33.16013895904277 -118.86714521158623) bank2762 +2763 POINT(33.88723058319561 -117.75634860635712) bank2763 +2764 POINT(34.603615011334696 -117.49038602127492) bank2764 +2765 POINT(34.08529966974087 -118.9403649322192) bank2765 +2766 POINT(34.51627373512852 -118.63632045507256) bank2766 +2767 POINT(34.397274603243325 -118.16780303635771) bank2767 +2768 POINT(33.378141438143004 -118.03910068714468) bank2768 +2769 POINT(34.203153011465176 -117.94019070319362) bank2769 +2770 POINT(34.19202837407081 -119.12018987639809) bank2770 +2771 POINT(33.68552100462739 -118.52551367260706) bank2771 +2772 POINT(33.08833900107212 -118.38483975990673) bank2772 +2773 POINT(34.70072787617367 -118.10952499088846) bank2773 +2774 POINT(33.186621454843966 -117.55201798849966) bank2774 +2775 POINT(34.83303201659026 -118.84550596826317) bank2775 +2776 POINT(33.348618913600966 -117.42401980599976) bank2776 +2777 POINT(33.339020205448584 -117.7212340920901) bank2777 +2778 POINT(34.51491988464477 -118.57929330933993) bank2778 +2779 POINT(33.57554207323573 -117.35240419148973) bank2779 +2780 POINT(34.82936246386109 -118.60750742789735) bank2780 +2781 POINT(34.42051670121481 -118.31477260618318) bank2781 +2782 POINT(33.60647141057361 -118.14764845367796) bank2782 +2783 POINT(34.777820739725776 -117.33371396467591) bank2783 +2784 POINT(34.014216834921335 -118.45069404837382) bank2784 +2785 POINT(33.803520710835905 -118.0038257514729) bank2785 +2786 POINT(34.20603914353491 -118.45668904014687) bank2786 +2787 POINT(33.8857175564698 -117.86767371509876) bank2787 +2788 POINT(34.356285241285875 -118.36176830353934) bank2788 +2789 POINT(33.38779400146645 -117.28240552868839) bank2789 +2790 POINT(34.888562211989445 -117.41790736183376) bank2790 +2791 POINT(33.335618546457184 -118.68590201247723) bank2791 +2792 POINT(34.11563347746324 -118.80818743637221) bank2792 +2793 POINT(34.82068496400368 -118.23313429036584) bank2793 +2794 POINT(34.24951491227158 -118.29651102065937) bank2794 +2795 POINT(34.49235255436296 -118.88921396725831) bank2795 +2796 POINT(34.18056234546747 -118.43675281939062) bank2796 +2797 POINT(34.113826533536454 -118.8328740704976) bank2797 +2798 POINT(34.24014269685305 -118.40057152081194) bank2798 +2799 POINT(33.47206952900779 -118.44310696365484) bank2799 +2800 POINT(34.39577197161216 -117.67108383692671) bank2800 +2801 POINT(33.5340629585864 -117.24822320941635) bank2801 +2802 POINT(34.09191397399667 -117.50007108799574) bank2802 +2803 POINT(34.32657173608304 -118.30807821895397) bank2803 +2804 POINT(34.11216966203424 -118.43673996652333) bank2804 +2805 POINT(34.73401547924947 -117.44406733789641) bank2805 +2806 POINT(34.308108441339286 -117.6847874518036) bank2806 +2807 POINT(33.80477879817795 -119.18444405683923) bank2807 +2808 POINT(33.5220172055566 -117.9810019622891) bank2808 +2809 POINT(33.80665900124505 -118.06116165907844) bank2809 +2810 POINT(33.866632955216794 -118.55176622677206) bank2810 +2811 POINT(33.92488825984322 -118.45011314687359) bank2811 +2812 POINT(33.15185212502648 -118.92180321775852) bank2812 +2813 POINT(33.9892548804293 -119.14720011594171) bank2813 +2814 POINT(34.39757235339665 -118.47430899807385) bank2814 +2815 POINT(34.3574525592973 -117.40176188046739) bank2815 +2816 POINT(34.33002721468903 -118.02514508366052) bank2816 +2817 POINT(34.39401992751409 -118.02432956894869) bank2817 +2818 POINT(35.014116584107356 -118.30143107300006) bank2818 +2819 POINT(34.69623671402934 -118.86221644696433) bank2819 +2820 POINT(33.24501028555901 -117.34674411256395) bank2820 +2821 POINT(34.88982242267632 -118.71800581946981) bank2821 +2822 POINT(33.08769591859255 -118.50762349598571) bank2822 +2823 POINT(33.84239300887056 -117.36763652400428) bank2823 +2824 POINT(34.407668033164384 -119.16495405008884) bank2824 +2825 POINT(34.87260998699463 -118.51547404415479) bank2825 +2826 POINT(33.74892530966447 -117.29718760838547) bank2826 +2827 POINT(34.75400758038954 -118.45366243989889) bank2827 +2828 POINT(33.059849989252776 -118.65765286482043) bank2828 +2829 POINT(34.14462651694579 -117.80051083057519) bank2829 +2830 POINT(34.43453241428938 -118.93859795756163) bank2830 +2831 POINT(33.52772298960697 -117.75385863522733) bank2831 +2832 POINT(34.74658215252965 -118.72237826809426) bank2832 +2833 POINT(34.849203052649294 -117.96319022530994) bank2833 +2834 POINT(33.346135778408275 -118.46626865545514) bank2834 +2835 POINT(33.27821657070521 -118.16869339712991) bank2835 +2836 POINT(33.566171147316965 -118.94507546946365) bank2836 +2837 POINT(34.470703631378306 -118.78346043447004) bank2837 +2838 POINT(33.67545359202303 -117.75683968932577) bank2838 +2839 POINT(33.80282367207465 -119.00839725187468) bank2839 +2840 POINT(33.143484430052624 -119.1310097301998) bank2840 +2841 POINT(34.51664736910359 -118.86569795806957) bank2841 +2842 POINT(33.1728157164584 -117.48362366468223) bank2842 +2843 POINT(33.63356340040154 -118.43127049579348) bank2843 +2844 POINT(33.06176513193946 -118.97895949089424) bank2844 +2845 POINT(34.72918974460758 -117.42600664805394) bank2845 +2846 POINT(33.226229844278095 -117.52828641771237) bank2846 +2847 POINT(33.5020490563089 -118.13880217888365) bank2847 +2848 POINT(34.06441847796973 -118.7371200583643) bank2848 +2849 POINT(34.25467616989085 -118.65739291172238) bank2849 +2850 POINT(34.81520872493969 -117.84060280840185) bank2850 +2851 POINT(34.67717666155588 -117.34448823762676) bank2851 +2852 POINT(33.21261679036982 -118.15925386799374) bank2852 +2853 POINT(33.94546913354818 -118.010237036506) bank2853 +2854 POINT(33.0641398352387 -118.23600282353244) bank2854 +2855 POINT(34.65853861566854 -118.24599091926798) bank2855 +2856 POINT(33.400882965736116 -117.56464087675909) bank2856 +2857 POINT(33.14876066212346 -118.63640447361642) bank2857 +2858 POINT(34.203766535878216 -117.53509195983483) bank2858 +2859 POINT(34.08911672534817 -117.32635543377869) bank2859 +2860 POINT(33.62231613162575 -117.28509034561553) bank2860 +2861 POINT(33.562964367128146 -118.37921792417379) bank2861 +2862 POINT(33.4924890892409 -117.29562330925107) bank2862 +2863 POINT(33.399857675377156 -117.25576520627543) bank2863 +2864 POINT(33.2027991996254 -118.21472579594699) bank2864 +2865 POINT(34.90303421267917 -117.79613328277902) bank2865 +2866 POINT(33.07538726453153 -118.54189028044834) bank2866 +2867 POINT(33.515422414434816 -118.59591930828427) bank2867 +2868 POINT(34.334737712971005 -117.48285481658097) bank2868 +2869 POINT(33.06359191690901 -118.93818215563921) bank2869 +2870 POINT(34.5356047888321 -117.55197104163801) bank2870 +2871 POINT(34.007152288593105 -117.89552539526525) bank2871 +2872 POINT(34.182835445804685 -117.67000760336815) bank2872 +2873 POINT(33.37794796318978 -117.56303210962473) bank2873 +2874 POINT(34.04272846723994 -119.08824395324258) bank2874 +2875 POINT(33.094790315673535 -118.5867388914726) bank2875 +2876 POINT(34.48608801053907 -118.96256192560031) bank2876 +2877 POINT(33.22805873107847 -118.79082545894873) bank2877 +2878 POINT(33.6979608739762 -118.60801738082941) bank2878 +2879 POINT(34.816549841470554 -118.26605379608694) bank2879 +2880 POINT(34.60762796424482 -117.99872367329795) bank2880 +2881 POINT(33.36998793200119 -118.32005310391058) bank2881 +2882 POINT(34.009938884663924 -118.95749181262057) bank2882 +2883 POINT(34.24289159281169 -118.9693326986769) bank2883 +2884 POINT(33.23176946454856 -118.03608352368752) bank2884 +2885 POINT(33.8698888260704 -117.4025623521183) bank2885 +2886 POINT(33.62515297081099 -118.58373621124998) bank2886 +2887 POINT(34.34664197906492 -117.52479426661037) bank2887 +2888 POINT(33.68419818879238 -117.26338232322354) bank2888 +2889 POINT(33.49031005291304 -118.65976696961965) bank2889 +2890 POINT(34.84935301102715 -118.64859128338503) bank2890 +2891 POINT(33.45198930718628 -118.39992371634645) bank2891 +2892 POINT(34.03207833893494 -119.22691945234904) bank2892 +2893 POINT(33.54344700374541 -119.04682395826528) bank2893 +2894 POINT(34.70993686449336 -118.65096804978695) bank2894 +2895 POINT(34.061100487321724 -118.87834396864312) bank2895 +2896 POINT(33.42288538333351 -117.32255849757733) bank2896 +2897 POINT(34.94523818945006 -117.95771283499931) bank2897 +2898 POINT(34.60408007815168 -118.8957127577988) bank2898 +2899 POINT(33.79301571790441 -118.62431092879699) bank2899 +2900 POINT(34.79563050918954 -118.50216366737277) bank2900 +2901 POINT(33.6087656785174 -118.8532329038806) bank2901 +2902 POINT(34.575836324775864 -117.51535285909632) bank2902 +2903 POINT(33.112125080014756 -117.47732347331717) bank2903 +2904 POINT(34.87272279117049 -117.88273686980543) bank2904 +2905 POINT(34.95361516760323 -117.40838474474097) bank2905 +2906 POINT(34.93968460957353 -118.14370764923099) bank2906 +2907 POINT(33.06997645632017 -118.34319740724227) bank2907 +2908 POINT(34.355906701068584 -117.50754420752578) bank2908 +2909 POINT(34.57876867323693 -117.5120837297946) bank2909 +2910 POINT(34.11032121809769 -118.65099058045382) bank2910 +2911 POINT(34.733685103816555 -117.53272227704052) bank2911 +2912 POINT(33.63976675795583 -118.34778125918365) bank2912 +2913 POINT(33.45281408759204 -117.87255937342462) bank2913 +2914 POINT(34.06849524512025 -117.99112874790067) bank2914 +2915 POINT(34.22077058952324 -117.24613410369624) bank2915 +2916 POINT(34.58458432364595 -117.55397170176461) bank2916 +2917 POINT(33.70294333109498 -118.38615988182437) bank2917 +2918 POINT(33.873667500471015 -118.79230393107798) bank2918 +2919 POINT(34.806694522613306 -118.14712654525786) bank2919 +2920 POINT(33.32355992999176 -119.16009899807774) bank2920 +2921 POINT(35.013344330103905 -118.49028309751694) bank2921 +2922 POINT(33.29706629766923 -118.31154437841236) bank2922 +2923 POINT(34.719339459155336 -117.8179660404083) bank2923 +2924 POINT(33.10788644770145 -118.54107034432302) bank2924 +2925 POINT(34.242648739921904 -118.68097722112549) bank2925 +2926 POINT(33.3065639428825 -118.7908472606007) bank2926 +2927 POINT(33.99150963851952 -119.17258931082868) bank2927 +2928 POINT(33.71206119343263 -118.8484459647866) bank2928 +2929 POINT(34.60590525268927 -119.15506875732865) bank2929 +2930 POINT(33.252308474163264 -117.93173554536453) bank2930 +2931 POINT(34.552753000518884 -118.06618235743635) bank2931 +2932 POINT(34.64829967739795 -118.43629086676847) bank2932 +2933 POINT(33.31407431889727 -118.07922883481955) bank2933 +2934 POINT(35.005234464658116 -117.53407927336293) bank2934 +2935 POINT(34.03397351121846 -117.50604835796223) bank2935 +2936 POINT(33.98390726117184 -118.40331629411611) bank2936 +2937 POINT(34.574208978246965 -118.27454276700868) bank2937 +2938 POINT(34.37830924237402 -118.50451204624652) bank2938 +2939 POINT(35.03594380883482 -117.49730233611471) bank2939 +2940 POINT(33.64733325905472 -118.47106789775022) bank2940 +2941 POINT(35.02600269351587 -117.265261041135) bank2941 +2942 POINT(33.71431356391903 -118.05078707689562) bank2942 +2943 POINT(34.788091638866455 -118.18709644253607) bank2943 +2944 POINT(35.03247976116215 -118.35616765249915) bank2944 +2945 POINT(34.59943786656677 -119.14709162824057) bank2945 +2946 POINT(33.860571218427204 -118.8156997562711) bank2946 +2947 POINT(34.60586079453415 -117.43780865629361) bank2947 +2948 POINT(35.028939021517964 -118.13312997787583) bank2948 +2949 POINT(33.75932638750944 -118.73400373333213) bank2949 +2950 POINT(33.38123478912842 -118.00476200491208) bank2950 +2951 POINT(34.71052738976311 -117.74906960412413) bank2951 +2952 POINT(34.72484960301352 -118.10300824350895) bank2952 +2953 POINT(33.29267321282958 -117.2807603394106) bank2953 +2954 POINT(34.130874024169046 -118.66941565489729) bank2954 +2955 POINT(34.24728535837691 -117.58698504033376) bank2955 +2956 POINT(33.86235746352692 -118.29865334468522) bank2956 +2957 POINT(34.04659101277421 -118.85248797875929) bank2957 +2958 POINT(33.82968326286822 -117.52702275666293) bank2958 +2959 POINT(33.985738371911694 -118.4269054398571) bank2959 +2960 POINT(33.27989667135445 -117.89153588501304) bank2960 +2961 POINT(33.21727747798905 -117.59400045294451) bank2961 +2962 POINT(34.33793070359538 -119.04068783136327) bank2962 +2963 POINT(33.09212433628672 -117.41554861626436) bank2963 +2964 POINT(33.8029617438767 -118.18607779907485) bank2964 +2965 POINT(33.47426805596842 -119.10343264302652) bank2965 +2966 POINT(34.99086686923945 -118.23112588160522) bank2966 +2967 POINT(34.76831370724577 -118.60694292897482) bank2967 +2968 POINT(34.62431824735303 -118.0016430938689) bank2968 +2969 POINT(33.32179120768294 -118.94223859149933) bank2969 +2970 POINT(33.92357188085141 -117.5843984822323) bank2970 +2971 POINT(34.3530867915835 -118.69058144403355) bank2971 +2972 POINT(33.261911037775064 -118.47093245497813) bank2972 +2973 POINT(34.50924808114933 -118.54732606950027) bank2973 +2974 POINT(33.88176773650338 -117.72710266435489) bank2974 +2975 POINT(34.71443019631178 -119.11895417555843) bank2975 +2976 POINT(34.13607465683395 -117.28585630441574) bank2976 +2977 POINT(33.58521245414247 -117.78325064722361) bank2977 +2978 POINT(34.58949146746016 -118.96520961919313) bank2978 +2979 POINT(34.642220547896734 -117.82129407789263) bank2979 +2980 POINT(33.51246104934532 -118.53367067764599) bank2980 +2981 POINT(34.9133850197669 -119.20652070320355) bank2981 +2982 POINT(33.3338773099536 -119.0128433227054) bank2982 +2983 POINT(34.98076114970801 -117.9123589645833) bank2983 +2984 POINT(33.52932003793217 -117.4536757808682) bank2984 +2985 POINT(34.945066391637546 -117.87035982549718) bank2985 +2986 POINT(34.56587133407973 -118.19784169601549) bank2986 +2987 POINT(34.065021594172485 -117.50421603225675) bank2987 +2988 POINT(34.673457433878134 -118.45970059727082) bank2988 +2989 POINT(33.5116797802195 -118.5235359680606) bank2989 +2990 POINT(34.483705604963845 -117.88502072864993) bank2990 +2991 POINT(33.66886244776302 -117.98704383907447) bank2991 +2992 POINT(33.1837720332417 -118.7013927024547) bank2992 +2993 POINT(33.63610377955299 -117.70768383565267) bank2993 +2994 POINT(34.60897310517572 -117.92268390117387) bank2994 +2995 POINT(34.79245989054739 -118.26201824688346) bank2995 +2996 POINT(34.073735687156805 -117.62044961535327) bank2996 +2997 POINT(33.61885197944637 -117.71144569937755) bank2997 +2998 POINT(34.12234922254955 -117.27936472200544) bank2998 +2999 POINT(33.39104965860537 -118.71826348028738) bank2999 +3000 POINT(35.0355051936169 -117.388976291217) bank3000 +3001 POINT(33.356683568754505 -119.22638212099177) bank3001 +3002 POINT(33.2327369633105 -117.46031436059991) bank3002 +3003 POINT(34.56436245802414 -117.45837431042297) bank3003 +3004 POINT(33.90178032521869 -118.03469676811439) bank3004 +3005 POINT(35.04244042858904 -118.73382379516455) bank3005 +3006 POINT(34.22229961921785 -119.2059122883445) bank3006 +3007 POINT(33.92492109874238 -118.22978318410208) bank3007 +3008 POINT(34.43849812999194 -117.67707235145218) bank3008 +3009 POINT(34.30993380683559 -118.36577877342268) bank3009 +3010 POINT(33.15479310555361 -117.24708154189379) bank3010 +3011 POINT(33.942606953611076 -117.58233248369217) bank3011 +3012 POINT(33.537041240040125 -117.88899036154407) bank3012 +3013 POINT(33.59909906345699 -118.76529506810411) bank3013 +3014 POINT(33.278198739470184 -117.65039463906471) bank3014 +3015 POINT(34.92469496691633 -117.68013077591873) bank3015 +3016 POINT(33.83588409114692 -119.22398820853635) bank3016 +3017 POINT(34.22552247158535 -118.92104481870408) bank3017 +3018 POINT(33.656265387539015 -117.31059670568605) bank3018 +3019 POINT(33.07277552018477 -118.29593203221567) bank3019 +3020 POINT(34.12762710029365 -118.09078263433832) bank3020 +3021 POINT(34.61474254776455 -118.87440796057486) bank3021 +3022 POINT(33.86589382011216 -117.80556798091175) bank3022 +3023 POINT(33.57652145547004 -117.6271697556085) bank3023 +3024 POINT(33.97637922465103 -117.7944828974459) bank3024 +3025 POINT(33.59305455214977 -117.59732761345819) bank3025 +3026 POINT(34.865404316187835 -118.57343174273052) bank3026 +3027 POINT(33.313246040342975 -118.69636729387923) bank3027 +3028 POINT(34.535652215308716 -118.95872769383763) bank3028 +3029 POINT(33.05769064508765 -118.66743779502652) bank3029 +3030 POINT(33.31899999393984 -118.40453665560511) bank3030 +3031 POINT(33.585922214144844 -118.49120578779997) bank3031 +3032 POINT(34.05225736731421 -118.96530253609535) bank3032 +3033 POINT(34.7220210420801 -118.09646478448005) bank3033 +3034 POINT(34.27906755667954 -117.42981359521002) bank3034 +3035 POINT(34.94216678690473 -118.47175706223952) bank3035 +3036 POINT(33.80580624579947 -117.43692166650175) bank3036 +3037 POINT(33.37425570334809 -117.98897472709882) bank3037 +3038 POINT(34.52759568902153 -117.74486017567007) bank3038 +3039 POINT(33.116440246635975 -118.08361201098276) bank3039 +3040 POINT(33.80769547843542 -117.75227151915992) bank3040 +3041 POINT(34.31373671950541 -118.77431035716694) bank3041 +3042 POINT(34.40989906932176 -117.26795066136476) bank3042 +3043 POINT(33.90598369372104 -119.01717901384501) bank3043 +3044 POINT(34.32837217969424 -118.21248919587674) bank3044 +3045 POINT(33.09226399091856 -118.03561364045599) bank3045 +3046 POINT(33.75236291604775 -117.25692667512024) bank3046 +3047 POINT(33.11091142610427 -118.9780429948266) bank3047 +3048 POINT(33.0734157487273 -117.73216000107308) bank3048 +3049 POINT(34.4506277277645 -117.89448808098119) bank3049 +3050 POINT(34.56028585628738 -117.57640505301572) bank3050 +3051 POINT(33.56677673924241 -117.56871176874625) bank3051 +3052 POINT(35.04073276363042 -118.14365072573887) bank3052 +3053 POINT(33.802358033932 -117.69112342616631) bank3053 +3054 POINT(34.33162773415867 -117.31178695688425) bank3054 +3055 POINT(33.195047169173826 -119.01601652507522) bank3055 +3056 POINT(33.49661105714118 -117.68026948526835) bank3056 +3057 POINT(33.57618635546741 -118.25748843677971) bank3057 +3058 POINT(34.01222186791245 -117.75400969414517) bank3058 +3059 POINT(33.643917290726876 -118.95789132629604) bank3059 +3060 POINT(34.588417032943596 -117.77374373034876) bank3060 +3061 POINT(33.58911856370336 -117.91183377213709) bank3061 +3062 POINT(34.56350878319265 -118.08147914612516) bank3062 +3063 POINT(33.550769539639994 -118.3582317633861) bank3063 +3064 POINT(34.905622563039366 -117.4962311995991) bank3064 +3065 POINT(34.10031568582406 -117.30470342436672) bank3065 +3066 POINT(34.90699552527764 -119.13378217459815) bank3066 +3067 POINT(34.85976878826055 -118.85308351314097) bank3067 +3068 POINT(35.01397181968906 -118.72257704991536) bank3068 +3069 POINT(34.85458233619576 -117.68065484398106) bank3069 +3070 POINT(33.40635142395139 -118.75143704915124) bank3070 +3071 POINT(33.88143681845081 -118.47348833302287) bank3071 +3072 POINT(33.597029274721386 -117.3650697852215) bank3072 +3073 POINT(34.61357946388637 -118.97042012306376) bank3073 +3074 POINT(34.09672701760067 -117.49418857895596) bank3074 +3075 POINT(33.55758037853084 -117.61110921614524) bank3075 +3076 POINT(34.79859935737579 -118.7611767659665) bank3076 +3077 POINT(34.82092657660676 -118.80776109140328) bank3077 +3078 POINT(33.45847011487957 -119.12542598503538) bank3078 +3079 POINT(33.62979549083623 -118.83907411559781) bank3079 +3080 POINT(33.74281009260583 -118.50718206185789) bank3080 +3081 POINT(34.147138411148305 -119.16304998864493) bank3081 +3082 POINT(34.49158843412938 -117.71885979868483) bank3082 +3083 POINT(33.952260517824215 -119.08885888325501) bank3083 +3084 POINT(35.047423241088175 -118.3343986048198) bank3084 +3085 POINT(34.915309325545124 -118.5291387199707) bank3085 +3086 POINT(33.68038426891319 -119.0698662561068) bank3086 +3087 POINT(33.548053085508826 -117.63637082630704) bank3087 +3088 POINT(34.10541102015653 -118.87134151028975) bank3088 +3089 POINT(34.654336696572344 -119.09744316669297) bank3089 +3090 POINT(34.14179469009596 -118.35732727381068) bank3090 +3091 POINT(33.231667978998146 -117.61713847324775) bank3091 +3092 POINT(33.890759864807755 -119.02301430464026) bank3092 +3093 POINT(34.40743765435806 -118.60936093852753) bank3093 +3094 POINT(34.45252968234817 -118.4178892847657) bank3094 +3095 POINT(33.075457904702425 -118.17895039650075) bank3095 +3096 POINT(33.096152277219815 -118.51717588435125) bank3096 +3097 POINT(33.484489278063315 -117.76748438881584) bank3097 +3098 POINT(34.709837272882446 -118.44028214636167) bank3098 +3099 POINT(33.25029053073942 -119.11828544455906) bank3099 +3100 POINT(33.64059031646194 -117.78434655898393) bank3100 +3101 POINT(33.27777015339056 -118.53510912742455) bank3101 +3102 POINT(34.470536263248576 -117.45101820785929) bank3102 +3103 POINT(34.263422621982066 -117.86420042183063) bank3103 +3104 POINT(34.00615333292391 -118.83719260525676) bank3104 +3105 POINT(34.45528392262521 -118.4962562062474) bank3105 +3106 POINT(34.409649695771485 -117.38398898028542) bank3106 +3107 POINT(34.334819288246166 -118.53684610286201) bank3107 +3108 POINT(33.50451502220547 -118.38806647955062) bank3108 +3109 POINT(33.7959376879652 -117.81789500635813) bank3109 +3110 POINT(33.2174195873442 -117.85399245557508) bank3110 +3111 POINT(34.46103074996717 -119.22229854582962) bank3111 +3112 POINT(34.73967741529374 -118.75226625923739) bank3112 +3113 POINT(33.46005885597783 -117.94191539736421) bank3113 +3114 POINT(33.65604110824227 -119.22005714410622) bank3114 +3115 POINT(33.08749126473863 -119.22152543479169) bank3115 +3116 POINT(33.994784567634184 -118.05355263750198) bank3116 +3117 POINT(33.780951157000295 -117.66542666925915) bank3117 +3118 POINT(34.284731839571236 -117.83333700137672) bank3118 +3119 POINT(33.952930736779116 -117.9952345127854) bank3119 +3120 POINT(34.43525691062403 -117.3871503274582) bank3120 +3121 POINT(34.09637740336625 -119.00453087122418) bank3121 +3122 POINT(34.608642545142864 -118.6676196471086) bank3122 +3123 POINT(34.00708545058585 -118.12814545513228) bank3123 +3124 POINT(34.804842867389134 -117.5765664678582) bank3124 +3125 POINT(34.465474392116555 -118.17376927083095) bank3125 +3126 POINT(34.938923216986375 -118.74850918536943) bank3126 +3127 POINT(34.43184020022344 -118.03617880008231) bank3127 +3128 POINT(33.474082176209535 -117.82765768492801) bank3128 +3129 POINT(34.89338103263341 -118.94423100399423) bank3129 +3130 POINT(34.353646439485075 -117.36068832366551) bank3130 +3131 POINT(34.75294233073549 -117.2703045004259) bank3131 +3132 POINT(34.46992866408985 -118.09632849733286) bank3132 +3133 POINT(34.28126262742501 -118.71478594463458) bank3133 +3134 POINT(33.537966076537174 -119.12767606782116) bank3134 +3135 POINT(34.48126690014454 -117.60855999005192) bank3135 +3136 POINT(33.32216058804651 -118.2791244328813) bank3136 +3137 POINT(33.26142526853035 -118.29299080855245) bank3137 +3138 POINT(33.70034187778137 -118.57296115382104) bank3138 +3139 POINT(33.27276666397189 -117.4041998180932) bank3139 +3140 POINT(34.52440003735488 -119.05044451701927) bank3140 +3141 POINT(33.76813992703712 -118.93566611977347) bank3141 +3142 POINT(34.19206724307843 -117.4246631390968) bank3142 +3143 POINT(33.85050167046399 -117.78214858133688) bank3143 +3144 POINT(34.95136804399126 -118.50558453551818) bank3144 +3145 POINT(34.0146432430593 -117.73619169675233) bank3145 +3146 POINT(33.79279045301186 -118.70213907946446) bank3146 +3147 POINT(34.30899764529214 -118.77752328343189) bank3147 +3148 POINT(34.34629769267679 -118.68925850012417) bank3148 +3149 POINT(33.95450933484743 -118.30096451161978) bank3149 +3150 POINT(34.790827739002246 -118.37799067100018) bank3150 +3151 POINT(34.63621039056689 -117.42752159333327) bank3151 +3152 POINT(33.103215982411825 -117.5575756086415) bank3152 +3153 POINT(34.740351968347845 -118.56604620786852) bank3153 +3154 POINT(33.38708769068636 -118.62412637410124) bank3154 +3155 POINT(34.33817906875199 -118.41351055895018) bank3155 +3156 POINT(33.87985948529986 -117.55087954430654) bank3156 +3157 POINT(33.065595131658924 -117.46281903837861) bank3157 +3158 POINT(33.76811178564652 -117.55281848212772) bank3158 +3159 POINT(33.20142026668893 -118.12002603566006) bank3159 +3160 POINT(34.943637861351974 -117.30444990669763) bank3160 +3161 POINT(34.106962106716274 -118.7486066206074) bank3161 +3162 POINT(33.3306846748128 -117.26304298635964) bank3162 +3163 POINT(34.44473223213452 -117.33914133884144) bank3163 +3164 POINT(33.94395704834047 -118.69580806015884) bank3164 +3165 POINT(33.20190993005322 -118.26975826843825) bank3165 +3166 POINT(34.76143238955507 -118.16675063086099) bank3166 +3167 POINT(33.52173120655926 -117.32784590991744) bank3167 +3168 POINT(34.177013829566725 -117.59869891390893) bank3168 +3169 POINT(33.278169333880925 -117.35062190995495) bank3169 +3170 POINT(33.55557908608631 -117.59765915218216) bank3170 +3171 POINT(34.208423323138774 -117.36218631530325) bank3171 +3172 POINT(33.06881332742298 -118.47286332468721) bank3172 +3173 POINT(33.753177768395794 -118.5955084293249) bank3173 +3174 POINT(34.264683871093844 -117.8067767010673) bank3174 +3175 POINT(34.41757692037368 -118.39598867546034) bank3175 +3176 POINT(33.25895944028665 -118.99303232928976) bank3176 +3177 POINT(34.313407648515515 -117.96603460493199) bank3177 +3178 POINT(34.93202742867092 -118.31891884855584) bank3178 +3179 POINT(34.785197282861695 -118.40627947693051) bank3179 +3180 POINT(33.49653799559344 -119.06132564551302) bank3180 +3181 POINT(33.87325921023403 -118.4307520980628) bank3181 +3182 POINT(33.088211829982974 -117.43473884675585) bank3182 +3183 POINT(33.20941485369449 -117.83816603702392) bank3183 +3184 POINT(34.35384974581076 -117.91377888918403) bank3184 +3185 POINT(34.16681527051214 -117.70054163269346) bank3185 +3186 POINT(34.65685600525799 -117.47547579386179) bank3186 +3187 POINT(34.959341372546135 -117.40689653522503) bank3187 +3188 POINT(33.68337454694885 -118.98899387849862) bank3188 +3189 POINT(33.776494450182085 -119.00885966230682) bank3189 +3190 POINT(34.74590214128637 -117.87924712230765) bank3190 +3191 POINT(34.297396740040334 -118.66117550757694) bank3191 +3192 POINT(34.80411184303092 -118.09413303504978) bank3192 +3193 POINT(33.653827208153444 -118.18687296613817) bank3193 +3194 POINT(34.19308691530738 -117.97459412731381) bank3194 +3195 POINT(33.92648826639394 -118.54449309639497) bank3195 +3196 POINT(34.91641932364495 -118.38554679243146) bank3196 +3197 POINT(33.47233777701936 -118.07688106961005) bank3197 +3198 POINT(33.98511625553423 -118.2401638074127) bank3198 +3199 POINT(33.85419994363566 -118.16666748910471) bank3199 +3200 POINT(34.848741652391595 -118.42109940305424) bank3200 +3201 POINT(33.70954134876407 -119.22532983743216) bank3201 +3202 POINT(34.08888954945122 -118.88113795214063) bank3202 +3203 POINT(33.378209497773454 -117.60046152517889) bank3203 +3204 POINT(33.82127512412648 -118.86832457976362) bank3204 +3205 POINT(33.73771859779007 -117.66076341753703) bank3205 +3206 POINT(34.93143088994344 -118.37382215907006) bank3206 +3207 POINT(34.00102926536576 -119.20447353844838) bank3207 +3208 POINT(33.54151895132548 -117.40300292302044) bank3208 +3209 POINT(34.35402593416767 -118.28409800433727) bank3209 +3210 POINT(33.667887075150176 -119.12234465900221) bank3210 +3211 POINT(33.33555184902984 -117.72866922012733) bank3211 +3212 POINT(34.856421830266875 -118.41560038092811) bank3212 +3213 POINT(33.74172609420102 -118.03885788354714) bank3213 +3214 POINT(34.622720996221226 -117.75580245581537) bank3214 +3215 POINT(33.09958325000149 -117.7532141488724) bank3215 +3216 POINT(33.917434369285374 -119.16677026428295) bank3216 +3217 POINT(34.47128541574333 -118.86403865480749) bank3217 +3218 POINT(33.322993391001475 -117.8530965633253) bank3218 +3219 POINT(34.85927251483068 -117.3896565521504) bank3219 +3220 POINT(35.02512553931889 -117.4552816131937) bank3220 +3221 POINT(34.529628789669374 -118.23218751404008) bank3221 +3222 POINT(34.96023989271829 -118.91011351332189) bank3222 +3223 POINT(33.359151136559234 -118.68561076070566) bank3223 +3224 POINT(33.258416852289216 -118.62495050468641) bank3224 +3225 POINT(33.76982015745682 -117.67724179024886) bank3225 +3226 POINT(33.15161888734558 -118.12610221377435) bank3226 +3227 POINT(33.28406522135121 -117.76209210687679) bank3227 +3228 POINT(34.2452413168072 -117.56883678298105) bank3228 +3229 POINT(34.36374731480046 -117.24610815498615) bank3229 +3230 POINT(34.00240639571745 -117.98058938001354) bank3230 +3231 POINT(33.96813339007395 -118.28483763972183) bank3231 +3232 POINT(33.73614342930493 -117.28170438214092) bank3232 +3233 POINT(33.92216287003335 -118.03970666116277) bank3233 +3234 POINT(33.47815338213349 -117.89622969404688) bank3234 +3235 POINT(33.92988056307804 -118.93078473174212) bank3235 +3236 POINT(33.800801047670035 -118.94190853569933) bank3236 +3237 POINT(33.38842139999035 -117.97208190108316) bank3237 +3238 POINT(33.81076256014403 -118.17405333140772) bank3238 +3239 POINT(34.46245485289862 -117.32040713433393) bank3239 +3240 POINT(34.908399957202356 -118.1701165712272) bank3240 +3241 POINT(34.9113250287012 -117.78191409597362) bank3241 +3242 POINT(35.01348325942964 -119.07981588340806) bank3242 +3243 POINT(33.0957678033657 -118.13388297843049) bank3243 +3244 POINT(33.44666678467652 -117.86798455317437) bank3244 +3245 POINT(34.128186573782706 -118.2172958035952) bank3245 +3246 POINT(33.33224030355256 -118.0100025474724) bank3246 +3247 POINT(35.01431511462132 -117.28300816070741) bank3247 +3248 POINT(34.190536796060286 -118.97292200265628) bank3248 +3249 POINT(34.51125478633962 -118.47255340677498) bank3249 +3250 POINT(33.92962812571171 -117.47508999952908) bank3250 +3251 POINT(33.3578702155539 -118.42020340867477) bank3251 +3252 POINT(33.38281097483013 -118.43167455913998) bank3252 +3253 POINT(33.94174985156372 -118.16401828658907) bank3253 +3254 POINT(34.30634104320463 -118.30291667270079) bank3254 +3255 POINT(34.85399516982719 -118.3208824815994) bank3255 +3256 POINT(34.8079535746736 -117.72590428544083) bank3256 +3257 POINT(34.63506902759003 -118.96802798839525) bank3257 +3258 POINT(33.49693844203251 -119.04267893885662) bank3258 +3259 POINT(34.69387883226247 -118.57166450135747) bank3259 +3260 POINT(33.40882186653934 -117.33284987593849) bank3260 +3261 POINT(34.330084169702715 -118.34861319786759) bank3261 +3262 POINT(34.183651125011316 -118.23424268981111) bank3262 +3263 POINT(33.093272126701955 -118.06710775893772) bank3263 +3264 POINT(33.56651554906922 -119.23605447469404) bank3264 +3265 POINT(34.12937703990455 -117.54679313068152) bank3265 +3266 POINT(33.913764813685475 -118.46630005309883) bank3266 +3267 POINT(34.66100815272416 -118.59770389242678) bank3267 +3268 POINT(34.72940042115115 -118.57958249477849) bank3268 +3269 POINT(33.61717116088599 -118.55508468536517) bank3269 +3270 POINT(33.47292009448228 -118.48958883451735) bank3270 +3271 POINT(33.44591915223962 -118.67403779708548) bank3271 +3272 POINT(34.53434545608064 -118.93258167985891) bank3272 +3273 POINT(34.05935676045056 -118.45317644477223) bank3273 +3274 POINT(33.78227598728651 -117.67952285904894) bank3274 +3275 POINT(34.446447723563764 -118.82426569916603) bank3275 +3276 POINT(34.61966361316487 -117.38467844775815) bank3276 +3277 POINT(34.63775664345335 -117.9271880545343) bank3277 +3278 POINT(34.01300793300272 -118.96705487410556) bank3278 +3279 POINT(33.953250398419534 -117.64786825273289) bank3279 +3280 POINT(33.858113948921385 -118.8270311293743) bank3280 +3281 POINT(33.143615473788195 -117.4301015902819) bank3281 +3282 POINT(34.978165012283206 -119.06596431330087) bank3282 +3283 POINT(34.58510569191924 -117.68986226834151) bank3283 +3284 POINT(33.06205919333625 -118.48855120689201) bank3284 +3285 POINT(34.7130734629508 -117.29220656354518) bank3285 +3286 POINT(34.0562822163809 -118.88452950177667) bank3286 +3287 POINT(33.293717412440614 -117.71103163181898) bank3287 +3288 POINT(34.9463461298707 -117.85938442191082) bank3288 +3289 POINT(34.33669032006979 -117.38157071934482) bank3289 +3290 POINT(34.118566758455515 -117.37264562932137) bank3290 +3291 POINT(33.652068192133044 -118.43890860186907) bank3291 +3292 POINT(34.22484140610311 -118.5011871788624) bank3292 +3293 POINT(33.9274748918479 -117.38437844164075) bank3293 +3294 POINT(34.55659671121446 -119.02061788440945) bank3294 +3295 POINT(33.741596634211085 -118.08632121392262) bank3295 +3296 POINT(34.69532426063632 -117.56354900933215) bank3296 +3297 POINT(33.706468509363695 -117.52682404581789) bank3297 +3298 POINT(34.835552955984724 -118.41332913874913) bank3298 +3299 POINT(34.462711983310854 -118.19590761291586) bank3299 +3300 POINT(34.86089246777023 -117.8303182014272) bank3300 +3301 POINT(34.22324981181532 -118.93429831016203) bank3301 +3302 POINT(33.5573618032727 -118.97414530318856) bank3302 +3303 POINT(34.18153561797251 -117.51122521264934) bank3303 +3304 POINT(33.64598085531361 -118.16148393239453) bank3304 +3305 POINT(34.44615234626565 -117.37836069944807) bank3305 +3306 POINT(34.79556259347125 -118.08356325105413) bank3306 +3307 POINT(34.32952720812658 -118.8884496676183) bank3307 +3308 POINT(34.46912130366335 -118.46938824883834) bank3308 +3309 POINT(34.25619646216774 -118.00948252664915) bank3309 +3310 POINT(33.51712404076972 -117.66318761484621) bank3310 +3311 POINT(35.035070727902074 -118.33366748349191) bank3311 +3312 POINT(34.870322094110186 -117.44227505476516) bank3312 +3313 POINT(34.887021665964 -118.39015148865963) bank3313 +3314 POINT(34.65138608494801 -117.48781769108562) bank3314 +3315 POINT(34.25994795910183 -118.30170546540668) bank3315 +3316 POINT(34.230597481441166 -118.49909674975379) bank3316 +3317 POINT(33.823248102412244 -118.41550134614711) bank3317 +3318 POINT(34.80652216661897 -117.55802411923352) bank3318 +3319 POINT(33.27051769875104 -118.1341987627615) bank3319 +3320 POINT(34.10526245346346 -119.16845931562278) bank3320 +3321 POINT(34.37175193807636 -117.83951379714819) bank3321 +3322 POINT(34.86063131513604 -118.51895468251335) bank3322 +3323 POINT(34.22938180373963 -118.64545272117672) bank3323 +3324 POINT(33.5748533869463 -117.7336041754901) bank3324 +3325 POINT(34.89615610122556 -118.35805018193832) bank3325 +3326 POINT(34.400744070323505 -117.69830573293675) bank3326 +3327 POINT(34.08168786397406 -118.37307113918098) bank3327 +3328 POINT(33.19976735155653 -117.69999670298479) bank3328 +3329 POINT(34.174845123950796 -118.28109932687208) bank3329 +3330 POINT(33.47588256787545 -118.4042003598573) bank3330 +3331 POINT(34.99443966163755 -117.25761323990072) bank3331 +3332 POINT(33.53212796965696 -119.19184726399413) bank3332 +3333 POINT(33.100793426644685 -118.79685520406494) bank3333 +3334 POINT(33.28996818330169 -118.97077381361352) bank3334 +3335 POINT(33.759407195559355 -118.79472035646636) bank3335 +3336 POINT(34.75733873103011 -117.97383551943261) bank3336 +3337 POINT(33.81036813281394 -118.29473914371657) bank3337 +3338 POINT(33.41439744655171 -119.04820140505399) bank3338 +3339 POINT(33.9760024149902 -117.90730376556496) bank3339 +3340 POINT(34.79596181481297 -119.06012205098155) bank3340 +3341 POINT(33.76328381442963 -118.78259957566742) bank3341 +3342 POINT(33.33219619759555 -118.0694050186011) bank3342 +3343 POINT(34.68655139003984 -117.76981924052286) bank3343 +3344 POINT(34.43857495594475 -118.13900090672931) bank3344 +3345 POINT(34.92766698081906 -117.69972728350646) bank3345 +3346 POINT(34.56425332330894 -118.26805416799492) bank3346 +3347 POINT(35.007702825231846 -117.9027463974402) bank3347 +3348 POINT(34.55900616751878 -118.63965434796172) bank3348 +3349 POINT(34.614070188419504 -118.12157194507655) bank3349 +3350 POINT(33.11155667852905 -119.1318952745797) bank3350 +3351 POINT(34.57274860467501 -118.00075163905704) bank3351 +3352 POINT(34.662086044685424 -118.96345549559618) bank3352 +3353 POINT(33.57434773114559 -118.95105848234296) bank3353 +3354 POINT(33.620347076216476 -118.8443298902044) bank3354 +3355 POINT(35.04611827722527 -117.49898755409549) bank3355 +3356 POINT(35.0233138949613 -118.82982814744983) bank3356 +3357 POINT(33.947422221562704 -118.6952802270945) bank3357 +3358 POINT(34.35134910236694 -117.73564111196269) bank3358 +3359 POINT(34.2421002657284 -117.61480996514483) bank3359 +3360 POINT(33.63065763075486 -118.33313008617667) bank3360 +3361 POINT(34.007610540879746 -117.70077779981014) bank3361 +3362 POINT(34.60006087245814 -117.6439153212534) bank3362 +3363 POINT(33.3422237626302 -118.99082245441491) bank3363 +3364 POINT(34.4877473938902 -117.69489421543987) bank3364 +3365 POINT(33.85265098509698 -118.62346286567029) bank3365 +3366 POINT(34.72320488149847 -118.39281869198093) bank3366 +3367 POINT(34.60150699353557 -118.39560683872634) bank3367 +3368 POINT(34.17816122972081 -118.92964643521037) bank3368 +3369 POINT(33.406530597285034 -118.60959575937073) bank3369 +3370 POINT(34.76827713980953 -117.7159386527388) bank3370 +3371 POINT(33.794546110636574 -118.46288318008932) bank3371 +3372 POINT(34.06469850322034 -118.25874432024516) bank3372 +3373 POINT(33.11669771997739 -117.62053863585729) bank3373 +3374 POINT(33.08322639413412 -117.83330058392153) bank3374 +3375 POINT(34.03124048928546 -118.04752315090168) bank3375 +3376 POINT(33.86865273432584 -118.35774422714653) bank3376 +3377 POINT(33.882274334225514 -118.78389217715969) bank3377 +3378 POINT(33.3940095581672 -117.45755313883029) bank3378 +3379 POINT(33.95839564127337 -117.8400461142843) bank3379 +3380 POINT(34.23013781671808 -119.16690245502497) bank3380 +3381 POINT(34.36039005749268 -117.69021310677624) bank3381 +3382 POINT(34.3954594637594 -117.67984100929709) bank3382 +3383 POINT(35.028507195742016 -118.54618882251505) bank3383 +3384 POINT(33.833156263390016 -117.96228650835218) bank3384 +3385 POINT(34.554623298971705 -117.71988867387948) bank3385 +3386 POINT(34.07336190533754 -117.49239997875317) bank3386 +3387 POINT(34.04713464017507 -117.25785677552723) bank3387 +3388 POINT(33.42985645798566 -118.30067760984979) bank3388 +3389 POINT(33.81394155713513 -119.14812187858246) bank3389 +3390 POINT(34.35248146819138 -117.6627066479359) bank3390 +3391 POINT(34.75432879444638 -117.53781763657692) bank3391 +3392 POINT(34.08360169623999 -117.7560266248365) bank3392 +3393 POINT(33.950572983545115 -119.15462386311643) bank3393 +3394 POINT(34.715254227692924 -118.57607428667282) bank3394 +3395 POINT(34.25864546673174 -118.20994525294712) bank3395 +3396 POINT(33.713034034837385 -117.37674833224477) bank3396 +3397 POINT(34.99250947343063 -118.45809316966951) bank3397 +3398 POINT(33.06340446590638 -117.9690545913431) bank3398 +3399 POINT(33.86039142321967 -118.83236813698181) bank3399 +3400 POINT(34.54667409409984 -118.47127158571475) bank3400 +3401 POINT(34.84334902473142 -118.83012191902289) bank3401 +3402 POINT(33.332933315136025 -118.99663251913272) bank3402 +3403 POINT(33.14867369092512 -118.46552510300945) bank3403 +3404 POINT(35.02862223981686 -117.8146119277046) bank3404 +3405 POINT(34.90036311716572 -117.9662555142192) bank3405 +3406 POINT(33.48820155241155 -117.81366291982884) bank3406 +3407 POINT(33.817882686769266 -117.73925602966) bank3407 +3408 POINT(34.92265077630981 -117.56110286752592) bank3408 +3409 POINT(34.00846669486258 -118.4352712342152) bank3409 +3410 POINT(33.52599564289067 -119.0451271801625) bank3410 +3411 POINT(33.627444200841836 -117.9226338583904) bank3411 +3412 POINT(33.89741492473473 -118.99689557806559) bank3412 +3413 POINT(33.064701074488966 -117.44324902679202) bank3413 +3414 POINT(34.886368931371145 -117.7662164731078) bank3414 +3415 POINT(33.484272490218174 -117.30103916415823) bank3415 +3416 POINT(34.68393567286602 -117.63982440883652) bank3416 +3417 POINT(33.66968785355038 -117.37391809179779) bank3417 +3418 POINT(33.80215350565477 -119.00723989786852) bank3418 +3419 POINT(34.7671267024947 -119.19176202967891) bank3419 +3420 POINT(33.891849801833594 -118.97029654511196) bank3420 +3421 POINT(34.1522219663314 -117.3677988070611) bank3421 +3422 POINT(34.01786913088496 -117.36594956980814) bank3422 +3423 POINT(34.53249622674842 -118.00786762975295) bank3423 +3424 POINT(34.16341248572492 -119.17377074590847) bank3424 +3425 POINT(34.19972764996932 -118.09233067451427) bank3425 +3426 POINT(33.365161450818704 -118.93481542642023) bank3426 +3427 POINT(33.61883270461514 -119.1367742197127) bank3427 +3428 POINT(34.305426162410626 -117.43203553356982) bank3428 +3429 POINT(34.36621121288499 -118.92010743761736) bank3429 +3430 POINT(34.078795941106634 -119.11836678876587) bank3430 +3431 POINT(33.764184295857774 -118.3388507296959) bank3431 +3432 POINT(34.27811081031184 -118.02051553947993) bank3432 +3433 POINT(33.3556845458632 -118.22672049758248) bank3433 +3434 POINT(34.65691442098565 -118.57648622921684) bank3434 +3435 POINT(33.15976356754927 -118.22014707280569) bank3435 +3436 POINT(34.529867066688695 -117.95201624962905) bank3436 +3437 POINT(33.78729241400249 -118.78147058283439) bank3437 +3438 POINT(34.0820494032965 -117.24990100900273) bank3438 +3439 POINT(33.97166926317331 -118.51488700760571) bank3439 +3440 POINT(34.99270808275169 -117.92525946330333) bank3440 +3441 POINT(34.13237584536457 -118.27666530427487) bank3441 +3442 POINT(35.04410500277425 -117.91990092333299) bank3442 +3443 POINT(34.89718863749928 -118.20819010454814) bank3443 +3444 POINT(34.53920213662565 -119.18412781125903) bank3444 +3445 POINT(33.24036173653148 -118.90521718589031) bank3445 +3446 POINT(34.276614171733705 -118.2315278492919) bank3446 +3447 POINT(34.125056442512765 -118.55961798668322) bank3447 +3448 POINT(33.30795934210214 -117.87141504707184) bank3448 +3449 POINT(34.12198401188899 -118.55130789175458) bank3449 +3450 POINT(34.33084210600557 -118.37574820529498) bank3450 +3451 POINT(33.877782579528194 -118.8307975396082) bank3451 +3452 POINT(34.07832563341717 -117.71475529354704) bank3452 +3453 POINT(34.90063442081372 -118.17562876204667) bank3453 +3454 POINT(34.52460516022736 -117.59601478110557) bank3454 +3455 POINT(33.24255675511654 -118.22961885577077) bank3455 +3456 POINT(34.45751126257549 -117.32233731449708) bank3456 +3457 POINT(34.728905777507535 -117.70121943431293) bank3457 +3458 POINT(34.12168474884069 -118.24020946847091) bank3458 +3459 POINT(34.51123228509636 -118.43745938679463) bank3459 +3460 POINT(34.30677552156194 -119.00646591146294) bank3460 +3461 POINT(34.46266595854246 -117.84740604532274) bank3461 +3462 POINT(34.533983698774364 -117.33730787831026) bank3462 +3463 POINT(33.783437628446904 -117.36226133596324) bank3463 +3464 POINT(34.34090692642105 -118.54356886760263) bank3464 +3465 POINT(33.0601271469098 -117.4488002087785) bank3465 +3466 POINT(34.76352517384092 -118.26145773746158) bank3466 +3467 POINT(33.202697009039426 -118.11490665685096) bank3467 +3468 POINT(34.04546437960641 -118.85675238681553) bank3468 +3469 POINT(33.288762726196566 -118.13886098637083) bank3469 +3470 POINT(34.08626120735638 -118.3878934388043) bank3470 +3471 POINT(34.1933545803823 -117.58977102810418) bank3471 +3472 POINT(33.18765686999301 -117.76016776816945) bank3472 +3473 POINT(34.988006081271855 -118.80905431164477) bank3473 +3474 POINT(33.5664302239893 -118.31065123039328) bank3474 +3475 POINT(33.75114174192675 -117.63993706955843) bank3475 +3476 POINT(34.22566089309207 -118.70303316460934) bank3476 +3477 POINT(33.919891103963266 -119.15795396098154) bank3477 +3478 POINT(34.290896627781144 -117.9486277242508) bank3478 +3479 POINT(34.07877899433428 -117.99411752303004) bank3479 +3480 POINT(33.70901359544553 -117.29489289300795) bank3480 +3481 POINT(34.62306106891467 -118.27883702550203) bank3481 +3482 POINT(34.615587592118 -118.28412402287968) bank3482 +3483 POINT(33.77008827096619 -117.44982503523154) bank3483 +3484 POINT(33.338002829271154 -118.67304896086586) bank3484 +3485 POINT(33.77495948631437 -118.95900854466834) bank3485 +3486 POINT(33.616075223657624 -117.35661052783931) bank3486 +3487 POINT(34.61401246031541 -117.99841317564994) bank3487 +3488 POINT(33.82439030214456 -118.58483390166764) bank3488 +3489 POINT(33.24472303262895 -119.10180002353614) bank3489 +3490 POINT(34.15301069504693 -117.37196122491446) bank3490 +3491 POINT(33.259279173738406 -119.13551086811313) bank3491 +3492 POINT(33.8414763841137 -117.38762653252444) bank3492 +3493 POINT(33.12957596440398 -119.19778481369283) bank3493 +3494 POINT(33.361589183398024 -118.21640522578723) bank3494 +3495 POINT(34.089721861047565 -118.29264013188136) bank3495 +3496 POINT(33.1310354903061 -119.074542751171) bank3496 +3497 POINT(34.08797380754035 -117.58865839806084) bank3497 +3498 POINT(34.609558309195016 -119.2108309726293) bank3498 +3499 POINT(34.72959772229215 -118.56136311203414) bank3499 +3500 POINT(34.072359519116326 -119.20709781139814) bank3500 +3501 POINT(33.57092109833591 -117.65924465426858) bank3501 +3502 POINT(33.55779229689336 -118.56287497667127) bank3502 +3503 POINT(33.85421427875343 -117.607111079674) bank3503 +3504 POINT(33.93632636305955 -117.91987569356829) bank3504 +3505 POINT(34.40237001683593 -118.31077772359296) bank3505 +3506 POINT(33.158552299730616 -119.2014928378556) bank3506 +3507 POINT(33.395070072967805 -118.74359574946389) bank3507 +3508 POINT(33.778621773814564 -117.8284675728669) bank3508 +3509 POINT(34.947502990755574 -118.73755740358422) bank3509 +3510 POINT(34.97820844972148 -117.67995202172057) bank3510 +3511 POINT(33.8485748284768 -118.10337681518946) bank3511 +3512 POINT(35.02899088543708 -117.9931859082823) bank3512 +3513 POINT(33.274390773603 -117.27479044994259) bank3513 +3514 POINT(33.394650201880594 -117.82404584386943) bank3514 +3515 POINT(34.60180261401029 -119.23236696119677) bank3515 +3516 POINT(34.27430118557873 -117.48249049447595) bank3516 +3517 POINT(34.08540079130061 -118.83097510298276) bank3517 +3518 POINT(33.55145777363196 -118.41423512585301) bank3518 +3519 POINT(33.6014624185005 -118.96741786586372) bank3519 +3520 POINT(34.60779090404612 -118.64361339730542) bank3520 +3521 POINT(34.59843655664691 -117.35645931240857) bank3521 +3522 POINT(34.20243566222831 -117.4671609608282) bank3522 +3523 POINT(33.45995196444723 -118.3565326894969) bank3523 +3524 POINT(33.61664053495058 -118.4230210421944) bank3524 +3525 POINT(33.80522911952391 -118.29546687542044) bank3525 +3526 POINT(34.48930138178635 -118.09961242180997) bank3526 +3527 POINT(34.94109839524722 -119.16949647229654) bank3527 +3528 POINT(33.26006731680197 -117.73040542787457) bank3528 +3529 POINT(33.196146035111624 -118.22386426367436) bank3529 +3530 POINT(33.93033323549321 -118.1125533132746) bank3530 +3531 POINT(33.124867282139 -118.01668248424532) bank3531 +3532 POINT(33.373404878810476 -117.37679741262293) bank3532 +3533 POINT(34.41792355718557 -117.7922319699736) bank3533 +3534 POINT(33.84929773221118 -119.21537265947616) bank3534 +3535 POINT(34.12494162076814 -117.51257713341212) bank3535 +3536 POINT(34.21976233293568 -118.58990405367719) bank3536 +3537 POINT(34.46409425313147 -119.21107949322179) bank3537 +3538 POINT(33.962407493937164 -117.9621352396718) bank3538 +3539 POINT(34.758324399370956 -118.40457470869626) bank3539 +3540 POINT(33.34717237333432 -117.37795671650439) bank3540 +3541 POINT(34.95851228399306 -118.97037313322774) bank3541 +3542 POINT(33.40510693424959 -118.21478891300725) bank3542 +3543 POINT(33.149381204909865 -117.3586228916044) bank3543 +3544 POINT(34.12360057364411 -117.88641970793518) bank3544 +3545 POINT(34.261503382403674 -117.3667222633663) bank3545 +3546 POINT(34.522237094367306 -118.00854622412984) bank3546 +3547 POINT(34.559916129228796 -118.41253904096668) bank3547 +3548 POINT(33.889542144655536 -118.63549419517857) bank3548 +3549 POINT(34.85395035582586 -118.36183560106207) bank3549 +3550 POINT(33.70813279094842 -117.51106615170431) bank3550 +3551 POINT(34.35705469241271 -117.32056842892776) bank3551 +3552 POINT(34.49704020748263 -118.24328184513912) bank3552 +3553 POINT(33.596442958037 -118.61844962436706) bank3553 +3554 POINT(33.08926707815503 -118.34459281474984) bank3554 +3555 POINT(34.57073643776699 -119.08652987398237) bank3555 +3556 POINT(33.596004081037904 -118.61930987923743) bank3556 +3557 POINT(33.83840650002786 -117.9821099986167) bank3557 +3558 POINT(33.109839582521886 -118.43111459022894) bank3558 +3559 POINT(33.85782963427247 -117.25080949402104) bank3559 +3560 POINT(34.50563573384329 -119.03751897561615) bank3560 +3561 POINT(33.29457416843308 -117.67174009085116) bank3561 +3562 POINT(34.59167857690327 -119.20834230170006) bank3562 +3563 POINT(33.1390168809516 -117.88816690778594) bank3563 +3564 POINT(33.064554096202684 -118.07945566193892) bank3564 +3565 POINT(33.1913801224098 -118.09674481693744) bank3565 +3566 POINT(33.11913734146857 -118.67791779426429) bank3566 +3567 POINT(34.348707698765395 -118.93852261089323) bank3567 +3568 POINT(34.62534544572514 -117.38863725548924) bank3568 +3569 POINT(34.8697120749403 -117.95302121061825) bank3569 +3570 POINT(33.947975300294054 -117.95185480528137) bank3570 +3571 POINT(34.874595894774345 -117.68782968492947) bank3571 +3572 POINT(33.61198202875378 -117.34981141705454) bank3572 +3573 POINT(33.344196719192865 -118.93746786282709) bank3573 +3574 POINT(33.96920358054959 -117.99795532761132) bank3574 +3575 POINT(33.08003466973958 -117.57517893363078) bank3575 +3576 POINT(33.894589719414014 -119.1128168332142) bank3576 +3577 POINT(34.452572415669344 -118.9029467482912) bank3577 +3578 POINT(34.00083490138735 -117.39322449936033) bank3578 +3579 POINT(33.64308718382721 -118.98608082781493) bank3579 +3580 POINT(34.079765321168495 -118.71702541190413) bank3580 +3581 POINT(34.425472815858335 -119.23349586109528) bank3581 +3582 POINT(33.59475775516015 -118.07658283240976) bank3582 +3583 POINT(34.81581494917869 -117.92676060783994) bank3583 +3584 POINT(33.966543401432425 -118.85319457658885) bank3584 +3585 POINT(33.63476342550418 -118.81130630634169) bank3585 +3586 POINT(33.93862524864642 -119.10539807172815) bank3586 +3587 POINT(33.335003335865764 -119.07966934859758) bank3587 +3588 POINT(34.2062626892638 -117.46504750721539) bank3588 +3589 POINT(33.53845692510437 -119.23087750881177) bank3589 +3590 POINT(33.926903428174896 -117.55938102871886) bank3590 +3591 POINT(33.86656860281312 -118.4844610828292) bank3591 +3592 POINT(34.39528513179212 -119.22371337745697) bank3592 +3593 POINT(34.89143163760369 -117.80433819770614) bank3593 +3594 POINT(34.73027374344013 -118.02229530645246) bank3594 +3595 POINT(33.151082251305944 -117.42738225477088) bank3595 +3596 POINT(34.2366158217052 -118.55418775953453) bank3596 +3597 POINT(33.81802323491022 -119.19692353882327) bank3597 +3598 POINT(33.596535132492 -118.38987234881417) bank3598 +3599 POINT(34.016503687271495 -117.94383451956126) bank3599 +3600 POINT(34.39896346561351 -117.86119448364865) bank3600 +3601 POINT(34.537348498411426 -118.64573526349776) bank3601 +3602 POINT(33.181214224694834 -117.3066539118907) bank3602 +3603 POINT(34.96778620081712 -117.51295441239117) bank3603 +3604 POINT(34.49257640627192 -118.28972667447323) bank3604 +3605 POINT(34.30996843279166 -118.97103780317958) bank3605 +3606 POINT(34.26878182037998 -117.79899511667686) bank3606 +3607 POINT(33.38029196358049 -117.33689089599171) bank3607 +3608 POINT(34.678989806200065 -118.32864683359118) bank3608 +3609 POINT(34.02080475220019 -119.14307083827342) bank3609 +3610 POINT(33.07152781876691 -117.30352675234748) bank3610 +3611 POINT(34.91859205943694 -118.67854756015375) bank3611 +3612 POINT(34.48900808491875 -118.32114873198759) bank3612 +3613 POINT(34.338658998984116 -118.68229385809897) bank3613 +3614 POINT(34.73943012442385 -117.86891280834992) bank3614 +3615 POINT(34.10542367404691 -118.37035059691429) bank3615 +3616 POINT(34.4828861850528 -118.64955583085623) bank3616 +3617 POINT(33.1680098932847 -118.7075602914676) bank3617 +3618 POINT(33.89887035293271 -117.64695570531966) bank3618 +3619 POINT(34.171536785007255 -119.03332442532702) bank3619 +3620 POINT(34.314829663212336 -117.89493867505314) bank3620 +3621 POINT(33.46935459526974 -117.93611555591842) bank3621 +3622 POINT(33.92298518166178 -118.25685987316506) bank3622 +3623 POINT(33.8999212627132 -118.01007596274901) bank3623 +3624 POINT(33.10924551320464 -119.08655458293227) bank3624 +3625 POINT(33.63776951794076 -118.79441774768169) bank3625 +3626 POINT(34.17422118733135 -117.99065491816889) bank3626 +3627 POINT(33.35709502836907 -117.36259251873275) bank3627 +3628 POINT(33.762597623276214 -117.87298401251371) bank3628 +3629 POINT(33.8832043180278 -119.14624044813684) bank3629 +3630 POINT(34.223397518076716 -117.26085791342305) bank3630 +3631 POINT(34.318727826227544 -118.61940826707409) bank3631 +3632 POINT(34.681939577045924 -118.67923363606764) bank3632 +3633 POINT(33.62119102967109 -118.06664767661735) bank3633 +3634 POINT(33.19647838276525 -117.57426701751805) bank3634 +3635 POINT(34.82641880354297 -118.39991720856284) bank3635 +3636 POINT(34.695238707495186 -119.19439037432878) bank3636 +3637 POINT(34.996764775096345 -117.43841752319769) bank3637 +3638 POINT(34.07730024553291 -118.32959590499266) bank3638 +3639 POINT(34.29849044622797 -118.8590607648961) bank3639 +3640 POINT(33.27892628630022 -118.91369207663523) bank3640 +3641 POINT(34.506712870111876 -117.52561132198483) bank3641 +3642 POINT(34.077897577811605 -118.91238289021969) bank3642 +3643 POINT(34.97483593359002 -117.2511306231068) bank3643 +3644 POINT(34.27570938008297 -118.55476159579216) bank3644 +3645 POINT(34.37716499602842 -119.23848246278223) bank3645 +3646 POINT(34.25573326065788 -117.39716578200394) bank3646 +3647 POINT(34.99615246523454 -117.27700804843222) bank3647 +3648 POINT(34.15197429802163 -119.11280790918154) bank3648 +3649 POINT(33.43290521565219 -119.14515622070446) bank3649 +3650 POINT(33.520798075524674 -119.09593141921636) bank3650 +3651 POINT(33.60941139234357 -118.07078898641922) bank3651 +3652 POINT(34.57012296665153 -118.25798673832229) bank3652 +3653 POINT(35.02077580248742 -118.61495920028595) bank3653 +3654 POINT(33.30531415185559 -118.1027130503141) bank3654 +3655 POINT(33.67292039131063 -118.95362284542196) bank3655 +3656 POINT(34.67980720088695 -117.98504018139113) bank3656 +3657 POINT(33.4094757041283 -117.61720136962924) bank3657 +3658 POINT(34.13264230110842 -117.71703513737324) bank3658 +3659 POINT(34.52298096337124 -117.57130213927219) bank3659 +3660 POINT(34.875201623283765 -117.43320539596189) bank3660 +3661 POINT(33.437777852091465 -119.07023290876492) bank3661 +3662 POINT(33.67328756683685 -118.65285948710797) bank3662 +3663 POINT(33.63288136255645 -117.46374420079323) bank3663 +3664 POINT(34.65175219781953 -118.71406016127715) bank3664 +3665 POINT(33.146980414141616 -117.64729921979087) bank3665 +3666 POINT(33.5309352922221 -117.29247296820908) bank3666 +3667 POINT(34.90373909622356 -118.53682100869096) bank3667 +3668 POINT(33.484894158154944 -118.35377235068385) bank3668 +3669 POINT(34.391461496676214 -118.49772392395725) bank3669 +3670 POINT(33.12449879587545 -117.59906213776048) bank3670 +3671 POINT(33.06967105033824 -117.42951413992816) bank3671 +3672 POINT(34.04134761648766 -117.46108867174488) bank3672 +3673 POINT(34.902675162686926 -118.35129730332396) bank3673 +3674 POINT(34.852582617268105 -117.26324710488703) bank3674 +3675 POINT(34.10093587115444 -117.34054134757208) bank3675 +3676 POINT(34.112942055239095 -117.80000006267375) bank3676 +3677 POINT(33.084338207542096 -117.58276753080601) bank3677 +3678 POINT(33.88267952636878 -117.40727535564784) bank3678 +3679 POINT(35.026847468161336 -118.73186037903072) bank3679 +3680 POINT(34.46746138194551 -117.24699282075649) bank3680 +3681 POINT(33.43765472944516 -117.7683103157058) bank3681 +3682 POINT(33.29816856332582 -118.87803475823311) bank3682 +3683 POINT(34.83466364806179 -118.25274846649951) bank3683 +3684 POINT(33.711534849411166 -117.44043910514507) bank3684 +3685 POINT(34.056241568431496 -118.4835735635906) bank3685 +3686 POINT(33.94737935739494 -117.56510765814707) bank3686 +3687 POINT(33.890511503409876 -118.17126423783778) bank3687 +3688 POINT(34.7479228798655 -118.69460077637261) bank3688 +3689 POINT(34.83758263977851 -119.11682579663523) bank3689 +3690 POINT(33.68975383569068 -117.44475218001976) bank3690 +3691 POINT(34.13073105836799 -118.75140157562994) bank3691 +3692 POINT(34.90165693425773 -119.04966829694708) bank3692 +3693 POINT(33.712181610781826 -119.0238257771856) bank3693 +3694 POINT(34.5524678635392 -119.10884650043194) bank3694 +3695 POINT(34.777362531629855 -117.84814184476309) bank3695 +3696 POINT(33.605511024112104 -117.7669197906428) bank3696 +3697 POINT(34.23677450604298 -118.19072612963244) bank3697 +3698 POINT(34.40230837581391 -117.6534959506107) bank3698 +3699 POINT(33.86737013790593 -118.11270056625595) bank3699 +3700 POINT(34.53532861578629 -117.9207335362632) bank3700 +3701 POINT(34.33907977978177 -118.96310506296598) bank3701 +3702 POINT(33.474219956124294 -117.69248210506056) bank3702 +3703 POINT(34.39298501045836 -118.55451399605457) bank3703 +3704 POINT(33.54295264213843 -118.5029905650225) bank3704 +3705 POINT(34.060378673770664 -117.93544823825377) bank3705 +3706 POINT(34.025500251240516 -117.26294618095491) bank3706 +3707 POINT(33.91121420214027 -118.01228928757315) bank3707 +3708 POINT(34.350017544643435 -119.23275256402098) bank3708 +3709 POINT(34.7789420837243 -117.26905367380964) bank3709 +3710 POINT(34.51878878232687 -119.09366459180238) bank3710 +3711 POINT(33.062137928839086 -117.49659687890076) bank3711 +3712 POINT(34.573964443402865 -119.0764911066399) bank3712 +3713 POINT(33.06063127378713 -117.67949181345547) bank3713 +3714 POINT(33.904242715970234 -118.77692697114672) bank3714 +3715 POINT(33.72143070939341 -118.91228292200745) bank3715 +3716 POINT(34.602375710260034 -118.96202442513375) bank3716 +3717 POINT(34.784340793887715 -118.53852030556027) bank3717 +3718 POINT(33.89302884054267 -118.18696128226942) bank3718 +3719 POINT(33.24455666151615 -118.48532361629664) bank3719 +3720 POINT(33.531970987958424 -117.93335300132529) bank3720 +3721 POINT(33.93644759638291 -119.1938260712011) bank3721 +3722 POINT(34.20496266045323 -118.56086133794196) bank3722 +3723 POINT(34.20959197653188 -118.76252825400633) bank3723 +3724 POINT(34.98231831381043 -118.45078772984147) bank3724 +3725 POINT(33.70734949843443 -117.44295623620903) bank3725 +3726 POINT(34.9343599808393 -117.86723081421675) bank3726 +3727 POINT(34.869362675130105 -117.77854249861555) bank3727 +3728 POINT(34.98315785156613 -118.23299194202853) bank3728 +3729 POINT(34.23684979715534 -117.9594847477641) bank3729 +3730 POINT(33.71231705739511 -117.85166818851305) bank3730 +3731 POINT(34.47455414779165 -119.08261293025295) bank3731 +3732 POINT(33.7728821375136 -118.52756693376585) bank3732 +3733 POINT(33.15431657053802 -117.29971303651824) bank3733 +3734 POINT(34.340389803078835 -117.64757894959014) bank3734 +3735 POINT(33.833006033188816 -117.36355520924798) bank3735 +3736 POINT(34.95685971412412 -117.25592865209362) bank3736 +3737 POINT(34.508596041281514 -118.02746924452859) bank3737 +3738 POINT(34.502907233680766 -117.83634275670374) bank3738 +3739 POINT(34.42809515976076 -118.57693957709948) bank3739 +3740 POINT(34.8594408232694 -118.06056265054653) bank3740 +3741 POINT(34.247754744287676 -118.63606328724539) bank3741 +3742 POINT(34.16403723532348 -117.28025560552328) bank3742 +3743 POINT(34.03775959520512 -118.20751692457051) bank3743 +3744 POINT(33.681721848648415 -117.33635167534153) bank3744 +3745 POINT(34.001627990063206 -119.23232740137952) bank3745 +3746 POINT(34.36485269458606 -118.78182329417834) bank3746 +3747 POINT(33.43328448856546 -117.36083976381013) bank3747 +3748 POINT(33.27020899940052 -118.45804653752127) bank3748 +3749 POINT(33.53923880655947 -117.92958177556663) bank3749 +3750 POINT(34.39641912849744 -119.17863646173588) bank3750 +3751 POINT(34.07519507043883 -117.47220501930087) bank3751 +3752 POINT(34.51359788727052 -117.70900206694681) bank3752 +3753 POINT(33.25333952116357 -118.11842154266971) bank3753 +3754 POINT(34.39709756179234 -117.48944988364879) bank3754 +3755 POINT(34.031863672627985 -119.22854155041153) bank3755 +3756 POINT(34.008447827629944 -118.17682648139186) bank3756 +3757 POINT(33.16809507506433 -118.28894361219051) bank3757 +3758 POINT(34.483449775443475 -117.72889248176226) bank3758 +3759 POINT(33.68531777643948 -117.79458729992473) bank3759 +3760 POINT(34.610602146019026 -117.50377774828114) bank3760 +3761 POINT(33.566220737401956 -117.52036155950829) bank3761 +3762 POINT(33.58815276566263 -117.2731218217081) bank3762 +3763 POINT(34.40875467177321 -117.75078036452582) bank3763 +3764 POINT(34.587612864427975 -117.44381670267872) bank3764 +3765 POINT(33.80438776970846 -119.04034279320604) bank3765 +3766 POINT(34.42638923173209 -118.55530503357838) bank3766 +3767 POINT(35.03582636242563 -119.13785664472222) bank3767 +3768 POINT(34.83322158786607 -117.28896586094694) bank3768 +3769 POINT(35.0354404343723 -118.27564528477288) bank3769 +3770 POINT(33.77111724464144 -118.77684606975455) bank3770 +3771 POINT(33.324722389654376 -117.75753590712365) bank3771 +3772 POINT(33.83875427543776 -118.61768734365876) bank3772 +3773 POINT(33.85397975203964 -117.85797952165245) bank3773 +3774 POINT(34.04696742353486 -119.0188657352595) bank3774 +3775 POINT(34.473584978495325 -117.51682582414524) bank3775 +3776 POINT(34.77201797265879 -117.30438932628549) bank3776 +3777 POINT(33.43608052401459 -118.46310897755036) bank3777 +3778 POINT(33.16361107709869 -118.67507209290137) bank3778 +3779 POINT(33.22119856596554 -119.04393556348656) bank3779 +3780 POINT(33.254971482480435 -118.91961317471133) bank3780 +3781 POINT(33.9662728171221 -117.66895592412841) bank3781 +3782 POINT(33.65698174296976 -118.95402445769547) bank3782 +3783 POINT(33.82605900989444 -117.4830168578348) bank3783 +3784 POINT(34.71133197263734 -117.2871245877605) bank3784 +3785 POINT(33.15282700428167 -118.06303574150002) bank3785 +3786 POINT(33.90577555545386 -118.7404093795463) bank3786 +3787 POINT(34.434711180325266 -117.9697283684) bank3787 +3788 POINT(34.71380927513028 -118.98562423454743) bank3788 +3789 POINT(33.97189036398597 -118.46331522336256) bank3789 +3790 POINT(34.47219858569787 -119.1902110154178) bank3790 +3791 POINT(33.225824541936504 -119.15268378245409) bank3791 +3792 POINT(33.32286634023794 -119.02767427154049) bank3792 +3793 POINT(34.53466578089769 -118.43789456194332) bank3793 +3794 POINT(34.30045665701752 -118.06304158703037) bank3794 +3795 POINT(33.43500093966346 -118.3190026526365) bank3795 +3796 POINT(34.30966599985679 -118.09780703402974) bank3796 +3797 POINT(34.87649995976509 -118.97206187862253) bank3797 +3798 POINT(34.870491741826704 -118.10846507107706) bank3798 +3799 POINT(34.64841804399664 -118.75659603005298) bank3799 +3800 POINT(34.32075153758897 -118.59853004337873) bank3800 +3801 POINT(33.31547706599359 -118.4461227621986) bank3801 +3802 POINT(34.50556169826256 -117.37377176756924) bank3802 +3803 POINT(34.02235008431553 -118.96626266282439) bank3803 +3804 POINT(34.503793114263274 -117.80478013136286) bank3804 +3805 POINT(34.36070173374777 -117.44309328578025) bank3805 +3806 POINT(34.709722425831295 -119.16824126366032) bank3806 +3807 POINT(33.14174028475077 -117.68041233848435) bank3807 +3808 POINT(34.27835031167746 -118.03779147088953) bank3808 +3809 POINT(34.012834935646715 -118.37747229572041) bank3809 +3810 POINT(34.26121302018498 -117.48247455747584) bank3810 +3811 POINT(33.490275547918074 -117.79177284780235) bank3811 +3812 POINT(34.01253906577165 -118.53489402977497) bank3812 +3813 POINT(34.04438400160669 -119.21707939616371) bank3813 +3814 POINT(33.266876118441495 -118.52998373089164) bank3814 +3815 POINT(34.53847514993382 -118.70472385879457) bank3815 +3816 POINT(33.45212990894819 -118.95706395425063) bank3816 +3817 POINT(33.42657724337133 -118.6773302929796) bank3817 +3818 POINT(34.19370171724731 -117.92620436325684) bank3818 +3819 POINT(33.59374888699796 -118.05472316988178) bank3819 +3820 POINT(34.797280891748954 -117.63893821315366) bank3820 +3821 POINT(33.22896504904912 -117.60499853333037) bank3821 +3822 POINT(33.179222731506115 -118.79081994759059) bank3822 +3823 POINT(34.8253852991417 -118.59252138698955) bank3823 +3824 POINT(34.08345360052124 -118.04838871590431) bank3824 +3825 POINT(34.936152477368694 -118.17208565877613) bank3825 +3826 POINT(33.80961743858305 -117.65478005758185) bank3826 +3827 POINT(33.5285603068637 -118.48715282847097) bank3827 +3828 POINT(34.18956190758227 -117.85270780362714) bank3828 +3829 POINT(33.821901381700044 -117.49098921240541) bank3829 +3830 POINT(33.592951490948266 -118.13726824580736) bank3830 +3831 POINT(34.07978436994796 -118.37029037255027) bank3831 +3832 POINT(34.856019428320224 -119.04001377558161) bank3832 +3833 POINT(33.3849525076715 -117.54892159585093) bank3833 +3834 POINT(34.724245079519996 -118.98707650033208) bank3834 +3835 POINT(33.653066470158414 -118.5666169492814) bank3835 +3836 POINT(34.11497990238567 -119.02951794442316) bank3836 +3837 POINT(34.314766261218956 -117.317201757703) bank3837 +3838 POINT(33.36453166846742 -117.93043541931046) bank3838 +3839 POINT(33.5892484642889 -118.77138939817799) bank3839 +3840 POINT(33.14046326505478 -119.16389536903034) bank3840 +3841 POINT(34.19456842745145 -118.80606552818878) bank3841 +3842 POINT(34.27714390887191 -117.80892986160185) bank3842 +3843 POINT(34.88385191564774 -118.25258099427957) bank3843 +3844 POINT(33.276962015262605 -117.47031027030154) bank3844 +3845 POINT(33.22408732775627 -118.72129789068634) bank3845 +3846 POINT(34.92881984740447 -117.73053367751572) bank3846 +3847 POINT(35.00127737564922 -117.9330151686979) bank3847 +3848 POINT(33.22472106340033 -118.91986885971279) bank3848 +3849 POINT(34.0797391778936 -118.68445385299523) bank3849 +3850 POINT(34.02574545125145 -117.28913426992322) bank3850 +3851 POINT(33.43158703335208 -117.35412356100558) bank3851 +3852 POINT(34.69349296779007 -117.46421995197892) bank3852 +3853 POINT(34.37207033121218 -118.9809588874977) bank3853 +3854 POINT(33.17803819466904 -118.83663362473831) bank3854 +3855 POINT(34.43806232580934 -118.13646625227304) bank3855 +3856 POINT(33.95019832197702 -119.14068748276836) bank3856 +3857 POINT(34.55799167272843 -118.71825776134634) bank3857 +3858 POINT(34.66625427299143 -117.74421753855046) bank3858 +3859 POINT(33.43640599191548 -117.62400950845029) bank3859 +3860 POINT(34.11544946957419 -119.056100555716) bank3860 +3861 POINT(33.73867884986857 -117.9624813215594) bank3861 +3862 POINT(34.91299453259097 -118.30504766701041) bank3862 +3863 POINT(33.79732987267665 -118.81265376991799) bank3863 +3864 POINT(34.5572829337801 -117.85534979595901) bank3864 +3865 POINT(34.44404142168089 -118.72148274783915) bank3865 +3866 POINT(33.50629863573749 -117.7217787282445) bank3866 +3867 POINT(33.67934869311423 -118.83365895534074) bank3867 +3868 POINT(34.436362976786036 -118.99873286255989) bank3868 +3869 POINT(33.470391555802046 -118.71287858188077) bank3869 +3870 POINT(33.10898115113606 -117.3486347452691) bank3870 +3871 POINT(33.64400319095328 -118.72181846147619) bank3871 +3872 POINT(33.29682993575389 -117.34423394237115) bank3872 +3873 POINT(33.19804466656066 -117.64115227468685) bank3873 +3874 POINT(34.197210300126585 -117.83633655197039) bank3874 +3875 POINT(33.93092061443256 -117.96079141100194) bank3875 +3876 POINT(33.70278663542886 -118.14374033054888) bank3876 +3877 POINT(34.51868966349714 -117.28117829061692) bank3877 +3878 POINT(33.569283406389815 -118.00131012896145) bank3878 +3879 POINT(33.073967924480925 -118.69102103748685) bank3879 +3880 POINT(34.69952717823827 -118.92736842405958) bank3880 +3881 POINT(33.57503123653349 -118.12991645204607) bank3881 +3882 POINT(33.25669326761502 -117.6826567877656) bank3882 +3883 POINT(33.57053404732342 -118.62513027629396) bank3883 +3884 POINT(33.812406161501464 -119.14974921341815) bank3884 +3885 POINT(34.447363333330685 -117.30022200325521) bank3885 +3886 POINT(34.15246266758368 -118.26751470766438) bank3886 +3887 POINT(33.45235033134517 -119.0451573102126) bank3887 +3888 POINT(34.120591253882154 -117.72160091136756) bank3888 +3889 POINT(34.48698509172312 -118.98195847299583) bank3889 +3890 POINT(34.145044474572686 -118.91762778379525) bank3890 +3891 POINT(34.520053891117385 -118.25790074166082) bank3891 +3892 POINT(34.63469591314607 -119.16135753913137) bank3892 +3893 POINT(34.10400720610987 -118.2267796584498) bank3893 +3894 POINT(34.18229127865984 -117.75123839273543) bank3894 +3895 POINT(34.17353498290883 -117.49700774116964) bank3895 +3896 POINT(35.0140147422814 -117.41294402292444) bank3896 +3897 POINT(33.51212450694693 -117.48670664127965) bank3897 +3898 POINT(33.88479601678877 -118.8037059092057) bank3898 +3899 POINT(34.759990641345894 -117.91149796851158) bank3899 +3900 POINT(34.83040153682662 -119.02040373959834) bank3900 +3901 POINT(33.59666110530487 -118.92006975408955) bank3901 +3902 POINT(34.45292565883486 -118.1994718959399) bank3902 +3903 POINT(33.97583202994313 -119.19118167269967) bank3903 +3904 POINT(34.87665292573247 -118.6745957314678) bank3904 +3905 POINT(33.29707272110558 -118.75151549911354) bank3905 +3906 POINT(33.77720060859913 -118.23732555347965) bank3906 +3907 POINT(33.170599027241586 -118.70776206156953) bank3907 +3908 POINT(33.8410499441096 -118.16877337637231) bank3908 +3909 POINT(34.99995207486187 -118.21000054873507) bank3909 +3910 POINT(33.09800430167102 -118.3226089728789) bank3910 +3911 POINT(34.62260547093941 -117.43219396516884) bank3911 +3912 POINT(35.01253763728886 -118.90971022607773) bank3912 +3913 POINT(34.46168664561971 -118.25010253276692) bank3913 +3914 POINT(34.22425287824209 -119.21913449632949) bank3914 +3915 POINT(34.42077385205659 -118.11738719918915) bank3915 +3916 POINT(34.47393678474208 -118.00747390140968) bank3916 +3917 POINT(34.67055647225438 -117.6467916112148) bank3917 +3918 POINT(33.352438477583654 -118.41302803851947) bank3918 +3919 POINT(34.95479697293322 -118.33434671574608) bank3919 +3920 POINT(34.556137888445136 -118.69998341449461) bank3920 +3921 POINT(33.42693843856768 -119.1640855391915) bank3921 +3922 POINT(34.37957826104154 -117.25498968460505) bank3922 +3923 POINT(34.490113369947586 -118.32536487687443) bank3923 +3924 POINT(34.41763410135357 -118.53069600742414) bank3924 +3925 POINT(34.22039544779314 -118.16810788068497) bank3925 +3926 POINT(33.05973643865596 -119.12375148227511) bank3926 +3927 POINT(33.21327718520748 -118.9713549957796) bank3927 +3928 POINT(34.5498232054408 -118.14828016234178) bank3928 +3929 POINT(34.39818071606488 -117.25196571915507) bank3929 +3930 POINT(33.901189807781954 -118.20772092210713) bank3930 +3931 POINT(33.429160253460566 -118.67378594267784) bank3931 +3932 POINT(34.11296515274691 -118.85845545123433) bank3932 +3933 POINT(34.028029454930895 -118.00018086786298) bank3933 +3934 POINT(34.00434235728837 -118.69618587428535) bank3934 +3935 POINT(33.093362904996305 -117.74894424753408) bank3935 +3936 POINT(34.30322321320143 -119.02089327434943) bank3936 +3937 POINT(33.0640285136905 -118.86340125712584) bank3937 +3938 POINT(34.17808750366712 -117.7687176322519) bank3938 +3939 POINT(34.35310839721932 -117.60246669755068) bank3939 +3940 POINT(34.73620795029994 -117.88216589335698) bank3940 +3941 POINT(34.18187084758987 -118.52774835518862) bank3941 +3942 POINT(33.074342906714335 -118.73218159189737) bank3942 +3943 POINT(33.918879855886914 -117.43451228840641) bank3943 +3944 POINT(33.85416116090912 -117.60167533403543) bank3944 +3945 POINT(33.56679589524899 -117.52223630777675) bank3945 +3946 POINT(34.98725602645211 -117.84459455003608) bank3946 +3947 POINT(34.57113257687657 -117.99360623382398) bank3947 +3948 POINT(34.18776074680975 -117.6454918810889) bank3948 +3949 POINT(33.815610187408986 -117.51805632010797) bank3949 +3950 POINT(33.585475041616874 -117.40579404955469) bank3950 +3951 POINT(33.93819973113041 -118.17976204375918) bank3951 +3952 POINT(33.698434572905946 -118.69064262140695) bank3952 +3953 POINT(34.589328288786554 -119.1911059531149) bank3953 +3954 POINT(33.34766032845489 -117.40984960226838) bank3954 +3955 POINT(34.262655453772425 -119.02972717307432) bank3955 +3956 POINT(34.84062334124111 -118.62714006216508) bank3956 +3957 POINT(34.079673006094616 -119.23589468465444) bank3957 +3958 POINT(33.5886391200085 -117.46682780568194) bank3958 +3959 POINT(33.7114854234669 -118.07894111097704) bank3959 +3960 POINT(33.286502244728155 -117.52673813231688) bank3960 +3961 POINT(33.234044707910755 -117.67951252642293) bank3961 +3962 POINT(33.89587626148312 -118.80773473241746) bank3962 +3963 POINT(33.77821689536492 -118.5026883606972) bank3963 +3964 POINT(33.90051173650338 -117.87251595834765) bank3964 +3965 POINT(34.294452177971934 -118.7227107482892) bank3965 +3966 POINT(33.56723794196427 -118.26943380305617) bank3966 +3967 POINT(34.539509317837954 -117.26792145143337) bank3967 +3968 POINT(34.616280260412765 -118.23239532856748) bank3968 +3969 POINT(34.21654724539739 -118.06440571868687) bank3969 +3970 POINT(34.57737235876515 -119.1165404501846) bank3970 +3971 POINT(34.20363009178802 -119.20482409509113) bank3971 +3972 POINT(34.69098035805327 -118.63820748188155) bank3972 +3973 POINT(34.04300550939795 -117.54681723632936) bank3973 +3974 POINT(33.61798687479631 -119.06695853859327) bank3974 +3975 POINT(33.69411834817558 -117.72848393406099) bank3975 +3976 POINT(34.17374329984829 -119.03822903343024) bank3976 +3977 POINT(34.69091966458398 -118.75609808801852) bank3977 +3978 POINT(34.180582532881566 -118.78926319619417) bank3978 +3979 POINT(33.79641242685248 -119.11416508964794) bank3979 +3980 POINT(34.60589942287293 -118.89955954346897) bank3980 +3981 POINT(33.78178277874823 -117.5018400246251) bank3981 +3982 POINT(33.37604573961892 -119.05171871800903) bank3982 +3983 POINT(34.260146957896715 -118.3661035430007) bank3983 +3984 POINT(33.36561445167936 -118.59099271789943) bank3984 +3985 POINT(33.44273566723632 -117.48633702749935) bank3985 +3986 POINT(34.507428659109564 -117.72189946840504) bank3986 +3987 POINT(34.248250980813495 -119.22469451033813) bank3987 +3988 POINT(34.89017287889586 -118.62550113186016) bank3988 +3989 POINT(34.60470028419226 -118.68220868941803) bank3989 +3990 POINT(33.43054003034899 -118.10425761111597) bank3990 +3991 POINT(33.6979634847511 -117.64992751081671) bank3991 +3992 POINT(34.45204513439471 -118.94515687923449) bank3992 +3993 POINT(34.392757354143576 -117.87609622315549) bank3993 +3994 POINT(33.350693432754774 -117.3404206104244) bank3994 +3995 POINT(33.18368086024961 -118.10335018900024) bank3995 +3996 POINT(34.888099811259174 -118.76874393590063) bank3996 +3997 POINT(33.749295925635394 -118.37533040298328) bank3997 +3998 POINT(34.6355337236228 -118.65587299959388) bank3998 +3999 POINT(34.444649027392245 -118.13462455271188) bank3999 +4000 POINT(33.374017827864805 -118.73291227182483) bank4000 +4001 POINT(34.57726890733434 -117.9616681435912) bank4001 +4002 POINT(33.77068862580439 -118.52023750468183) bank4002 +4003 POINT(33.22165934167272 -117.71829916383516) bank4003 +4004 POINT(33.89883293046463 -118.0322760151327) bank4004 +4005 POINT(33.67150940524668 -118.9991650839056) bank4005 +4006 POINT(34.57346238459637 -117.9477549242447) bank4006 +4007 POINT(34.781945880630026 -117.29395023165306) bank4007 +4008 POINT(34.797079515056204 -118.8425212901257) bank4008 +4009 POINT(33.26949476926895 -117.43825734650152) bank4009 +4010 POINT(33.13084049659484 -119.01026999346504) bank4010 +4011 POINT(34.04510330173664 -117.30113567415735) bank4011 +4012 POINT(33.829415737919895 -118.37451183845705) bank4012 +4013 POINT(34.19191342457683 -118.9580988927799) bank4013 +4014 POINT(34.148757737681876 -118.30264221417276) bank4014 +4015 POINT(34.3795806640795 -117.89855121325186) bank4015 +4016 POINT(33.90180300920725 -117.49196559619531) bank4016 +4017 POINT(34.56831110237459 -118.86441195397973) bank4017 +4018 POINT(34.032133181109224 -117.99949701329365) bank4018 +4019 POINT(34.23609727278149 -118.84329016183403) bank4019 +4020 POINT(34.803835531786774 -119.05318596815313) bank4020 +4021 POINT(34.42380170907588 -117.8836545953463) bank4021 +4022 POINT(33.36028524636402 -118.40154737753207) bank4022 +4023 POINT(34.71891888626049 -118.63329768990211) bank4023 +4024 POINT(33.187318807883365 -117.80497814127072) bank4024 +4025 POINT(34.73549716722172 -118.20223506578708) bank4025 +4026 POINT(33.61101914274753 -117.75193006443968) bank4026 +4027 POINT(34.52750150411851 -118.89048454625967) bank4027 +4028 POINT(33.15813756467137 -119.16140513690253) bank4028 +4029 POINT(33.272566914317885 -118.66078243722941) bank4029 +4030 POINT(34.6032368531232 -118.34627874267322) bank4030 +4031 POINT(33.452827481683755 -118.26118805697169) bank4031 +4032 POINT(34.74791853159953 -118.57690049923248) bank4032 +4033 POINT(34.87693829370059 -119.17344042429953) bank4033 +4034 POINT(34.57940303420456 -117.52114125711009) bank4034 +4035 POINT(33.70447046942259 -118.11751726152326) bank4035 +4036 POINT(34.35205832216104 -117.62829216645461) bank4036 +4037 POINT(33.204146570875025 -118.53801841756668) bank4037 +4038 POINT(33.97340071907054 -117.72526501721859) bank4038 +4039 POINT(34.49616496641598 -117.6599456276851) bank4039 +4040 POINT(34.97613406307042 -119.1972410191737) bank4040 +4041 POINT(34.878197919735605 -118.14997504397391) bank4041 +4042 POINT(33.35836361071946 -118.0446505902664) bank4042 +4043 POINT(33.95404610156085 -118.58372107892707) bank4043 +4044 POINT(33.19850142147437 -118.96817558441062) bank4044 +4045 POINT(33.176223951650066 -117.88515070037644) bank4045 +4046 POINT(33.877931313664455 -118.8837465107297) bank4046 +4047 POINT(34.76880147422051 -117.80127039960047) bank4047 +4048 POINT(33.50612277868918 -118.24023664218711) bank4048 +4049 POINT(33.883815367573334 -118.64791981972151) bank4049 +4050 POINT(33.30428698634334 -118.98890579752792) bank4050 +4051 POINT(33.852093743861545 -118.72670572385852) bank4051 +4052 POINT(34.04343394304364 -117.58112972973582) bank4052 +4053 POINT(33.87463723490974 -117.53806738348038) bank4053 +4054 POINT(35.02162533818082 -119.06675084688622) bank4054 +4055 POINT(33.375599459367976 -118.23729472314554) bank4055 +4056 POINT(33.452700020300135 -117.24375736043665) bank4056 +4057 POINT(33.854909980573424 -117.74205752017048) bank4057 +4058 POINT(34.67179678793766 -117.91013793790822) bank4058 +4059 POINT(34.18943165164352 -118.40709310106723) bank4059 +4060 POINT(33.55179878254516 -117.77377873565739) bank4060 +4061 POINT(34.043486674966964 -118.03210521327743) bank4061 +4062 POINT(33.97926189724631 -117.37228585880116) bank4062 +4063 POINT(34.102108571794375 -119.19329963822348) bank4063 +4064 POINT(33.296740374119395 -118.68701882487265) bank4064 +4065 POINT(34.938699031867266 -118.80165316721022) bank4065 +4066 POINT(33.82163531634142 -119.21061125890832) bank4066 +4067 POINT(33.15021601734466 -117.78298240214149) bank4067 +4068 POINT(34.0159666127959 -118.66272410437077) bank4068 +4069 POINT(34.52092770549283 -118.51105848614917) bank4069 +4070 POINT(34.833344061231145 -118.48284177309337) bank4070 +4071 POINT(34.71270785145576 -118.44838330461957) bank4071 +4072 POINT(34.72462876918542 -117.54383074843658) bank4072 +4073 POINT(34.19638871068295 -117.93040943576004) bank4073 +4074 POINT(34.755867465020415 -118.02290119509132) bank4074 +4075 POINT(33.14517208708365 -118.09879818666782) bank4075 +4076 POINT(34.34687645190906 -118.05135483110692) bank4076 +4077 POINT(34.352490699383225 -117.3669268091023) bank4077 +4078 POINT(34.35686433588144 -117.62361919759175) bank4078 +4079 POINT(34.79123648052357 -118.5245488273632) bank4079 +4080 POINT(33.35772592908195 -119.09966602653753) bank4080 +4081 POINT(33.215421764310875 -118.79546131998337) bank4081 +4082 POINT(33.99915867485267 -118.76612606593936) bank4082 +4083 POINT(34.96734561400688 -117.75841972214398) bank4083 +4084 POINT(34.117437398995186 -118.3425992295276) bank4084 +4085 POINT(34.79414494383742 -117.3581979568497) bank4085 +4086 POINT(33.64212140070104 -118.48911260655133) bank4086 +4087 POINT(33.62702999430421 -117.89174325274203) bank4087 +4088 POINT(33.25916160566185 -118.74001253017373) bank4088 +4089 POINT(34.72801452877125 -118.65075374624126) bank4089 +4090 POINT(33.62666751632533 -118.81355364471985) bank4090 +4091 POINT(35.03181375618739 -118.1689385109071) bank4091 +4092 POINT(34.759129897281284 -118.24490231215579) bank4092 +4093 POINT(34.111493155044485 -118.73141221114535) bank4093 +4094 POINT(34.011848604172236 -117.50760960833453) bank4094 +4095 POINT(34.561661686609945 -118.72892603453727) bank4095 +4096 POINT(33.76901241715196 -117.57532339688318) bank4096 +4097 POINT(33.88446934541905 -117.5581785738101) bank4097 +4098 POINT(34.32875763575439 -117.55367506667241) bank4098 +4099 POINT(33.26110158165656 -118.1003106034064) bank4099 +4100 POINT(33.0619284640454 -119.00795766362361) bank4100 +4101 POINT(34.362490391168436 -118.91303585085639) bank4101 +4102 POINT(33.53291412204595 -119.19231620152364) bank4102 +4103 POINT(33.23354136723081 -118.05199352853809) bank4103 +4104 POINT(33.22371611852036 -118.86521467774001) bank4104 +4105 POINT(34.04523187071517 -119.04553510832893) bank4105 +4106 POINT(33.3078562175767 -119.22425316272583) bank4106 +4107 POINT(34.25097398542901 -117.90897502829257) bank4107 +4108 POINT(34.619922267936175 -118.7335840682138) bank4108 +4109 POINT(34.38698957251827 -117.2825066794305) bank4109 +4110 POINT(33.5279248242404 -118.57929945558547) bank4110 +4111 POINT(33.6989345348483 -119.16657579296546) bank4111 +4112 POINT(34.12139849928711 -117.34493861371664) bank4112 +4113 POINT(33.85746074103648 -119.1863046415772) bank4113 +4114 POINT(34.891135691883875 -118.40224744487409) bank4114 +4115 POINT(34.59547720250179 -118.23622383688145) bank4115 +4116 POINT(33.96820103114897 -117.69961449229837) bank4116 +4117 POINT(34.93268924084404 -118.50386078396879) bank4117 +4118 POINT(33.543845789654185 -118.12037215159562) bank4118 +4119 POINT(34.1405695970468 -118.63030915342293) bank4119 +4120 POINT(35.02854393096036 -118.39510089329441) bank4120 +4121 POINT(33.84890464690671 -118.30329498958447) bank4121 +4122 POINT(34.471397627510726 -118.9761847508686) bank4122 +4123 POINT(34.51497094823914 -119.05480676079966) bank4123 +4124 POINT(33.29754802877106 -117.78952267150514) bank4124 +4125 POINT(33.25644136557425 -117.57490463406441) bank4125 +4126 POINT(34.05061560629469 -117.51905420458297) bank4126 +4127 POINT(33.248467010299215 -117.38131490696713) bank4127 +4128 POINT(34.33320298983906 -119.07735301611895) bank4128 +4129 POINT(33.47165250917999 -117.67563609848392) bank4129 +4130 POINT(33.71057330490786 -117.7238033723205) bank4130 +4131 POINT(33.533365340459234 -119.12045876067288) bank4131 +4132 POINT(34.607763310624414 -118.66877594093201) bank4132 +4133 POINT(34.917991265383925 -117.58199273945065) bank4133 +4134 POINT(33.724425205458616 -118.40632143825987) bank4134 +4135 POINT(33.207816685110515 -117.82076783701821) bank4135 +4136 POINT(34.587268561700284 -119.03877548490671) bank4136 +4137 POINT(33.18753867766629 -118.90950916465238) bank4137 +4138 POINT(34.40771363474343 -117.91070415828645) bank4138 +4139 POINT(33.42552043406408 -118.52818033531271) bank4139 +4140 POINT(33.50950232230491 -117.54125909915393) bank4140 +4141 POINT(33.97594016591854 -118.07686806273084) bank4141 +4142 POINT(33.81679114562337 -118.88046990320223) bank4142 +4143 POINT(34.95253132992027 -117.32306697137012) bank4143 +4144 POINT(33.776197671480965 -117.92317827861076) bank4144 +4145 POINT(34.421388832445885 -118.17113863528162) bank4145 +4146 POINT(33.2706750339742 -119.22400992217177) bank4146 +4147 POINT(34.09641343081477 -117.93520010841075) bank4147 +4148 POINT(34.932399119202294 -117.6777783096065) bank4148 +4149 POINT(33.926260562542915 -118.91163343904255) bank4149 +4150 POINT(33.260788848980475 -117.86596665619972) bank4150 +4151 POINT(34.83420707314655 -118.85780630982099) bank4151 +4152 POINT(34.123887385484934 -119.19871031220046) bank4152 +4153 POINT(35.03981646937656 -119.09104714199094) bank4153 +4154 POINT(34.20885576194801 -117.46371933414449) bank4154 +4155 POINT(34.9278111115887 -118.8898296178883) bank4155 +4156 POINT(34.49598297320239 -118.03886378338645) bank4156 +4157 POINT(33.09194007006863 -118.64124109150126) bank4157 +4158 POINT(34.119203467270545 -117.6976676426056) bank4158 +4159 POINT(34.063974365293575 -117.36984519845059) bank4159 +4160 POINT(33.22375417789715 -118.55902537396466) bank4160 +4161 POINT(33.17874704972635 -118.42820350228719) bank4161 +4162 POINT(33.93341218432512 -118.09147716443665) bank4162 +4163 POINT(34.80107175420611 -117.2789712635713) bank4163 +4164 POINT(34.01975396257627 -118.87056804616282) bank4164 +4165 POINT(34.887301043378784 -119.13718567762884) bank4165 +4166 POINT(34.057705858474264 -117.2839568795174) bank4166 +4167 POINT(34.94486573155027 -118.78826596829165) bank4167 +4168 POINT(34.15651705467809 -119.2341395189083) bank4168 +4169 POINT(34.450166701061654 -118.55265186559747) bank4169 +4170 POINT(33.245024206323386 -117.90018134539383) bank4170 +4171 POINT(34.40788291351308 -117.36125355950858) bank4171 +4172 POINT(33.456497876582006 -118.38038650191561) bank4172 +4173 POINT(33.66691149761766 -118.15470131369148) bank4173 +4174 POINT(33.14974847104729 -118.51194331125168) bank4174 +4175 POINT(33.33116480561459 -117.33437663238743) bank4175 +4176 POINT(33.72508409254716 -117.38725300665732) bank4176 +4177 POINT(34.33020451750062 -118.75558830629814) bank4177 +4178 POINT(33.41914578902888 -117.73299522964973) bank4178 +4179 POINT(34.14914135633575 -119.22593093936376) bank4179 +4180 POINT(34.09568428910383 -118.37373297914633) bank4180 +4181 POINT(33.33450546270293 -118.84980939852124) bank4181 +4182 POINT(34.93280252479684 -118.3646644649356) bank4182 +4183 POINT(34.05501836728231 -118.24636345912032) bank4183 +4184 POINT(34.79933295122903 -118.12971350613194) bank4184 +4185 POINT(34.643618329907696 -119.07095694102968) bank4185 +4186 POINT(33.5596046569133 -117.29660291404156) bank4186 +4187 POINT(34.533925974703436 -118.8674508542628) bank4187 +4188 POINT(34.36883411322682 -118.18767528918967) bank4188 +4189 POINT(34.49461610452513 -117.4063276203363) bank4189 +4190 POINT(34.185316198179535 -117.73746174633109) bank4190 +4191 POINT(34.065399432835754 -119.18020370446796) bank4191 +4192 POINT(33.276094757583934 -117.24866287876053) bank4192 +4193 POINT(33.703391664530635 -118.00326958905113) bank4193 +4194 POINT(34.863894234695856 -117.79685746346436) bank4194 +4195 POINT(35.0413821201078 -118.77482034431917) bank4195 +4196 POINT(34.266332669438476 -118.55730008950762) bank4196 +4197 POINT(33.74694481525469 -118.7752445049206) bank4197 +4198 POINT(33.22247721588444 -119.22741162341607) bank4198 +4199 POINT(34.599662467881316 -117.78165906567995) bank4199 +4200 POINT(34.88140722335996 -117.87716156871316) bank4200 +4201 POINT(33.9654054282033 -117.60558182018653) bank4201 +4202 POINT(34.48130100348939 -118.62995732848138) bank4202 +4203 POINT(34.56733352073337 -118.95375409667483) bank4203 +4204 POINT(34.26462468815527 -119.02876520367461) bank4204 +4205 POINT(34.0501988771474 -117.5627525424898) bank4205 +4206 POINT(34.91075306212909 -117.73696196031726) bank4206 +4207 POINT(35.008535691389056 -119.17221441278797) bank4207 +4208 POINT(34.08102048604255 -118.16858379272355) bank4208 +4209 POINT(34.90094540202176 -118.3214762013484) bank4209 +4210 POINT(33.99283580617783 -118.93006361129791) bank4210 +4211 POINT(34.2551977450065 -117.44007540622454) bank4211 +4212 POINT(34.53907894173367 -118.35867609537758) bank4212 +4213 POINT(34.221197866615434 -117.46408926473123) bank4213 +4214 POINT(33.08401548207973 -118.03892023362414) bank4214 +4215 POINT(34.97411133106759 -117.9356534588849) bank4215 +4216 POINT(33.849857129501565 -119.22717337176313) bank4216 +4217 POINT(34.94617822315161 -118.52187319144112) bank4217 +4218 POINT(33.15394896953576 -117.95522664240119) bank4218 +4219 POINT(34.88108513548695 -117.72968483298851) bank4219 +4220 POINT(33.72948183649819 -118.06638432308517) bank4220 +4221 POINT(34.06412748336184 -117.34062546205398) bank4221 +4222 POINT(34.950190167966504 -118.23772922101186) bank4222 +4223 POINT(35.04628368646394 -117.36243903091156) bank4223 +4224 POINT(34.29833677159194 -118.11943077687008) bank4224 +4225 POINT(33.405818728666404 -118.95061190763306) bank4225 +4226 POINT(33.42498170560168 -118.67961070480546) bank4226 +4227 POINT(34.38844290633383 -117.73287453564762) bank4227 +4228 POINT(34.85328523646879 -118.07275490749672) bank4228 +4229 POINT(33.41561400220491 -117.48875723297735) bank4229 +4230 POINT(33.155225672797066 -118.36165856893378) bank4230 +4231 POINT(34.432016339737125 -119.13430252294967) bank4231 +4232 POINT(34.32445758330859 -117.95325117169374) bank4232 +4233 POINT(35.01225459796806 -117.27287758224386) bank4233 +4234 POINT(34.38831315064704 -118.18862675953865) bank4234 +4235 POINT(33.18166977580274 -117.70287522794752) bank4235 +4236 POINT(34.479863815659016 -118.02132273980287) bank4236 +4237 POINT(34.0223121542551 -118.65767332879682) bank4237 +4238 POINT(34.58437173212283 -117.30853732220164) bank4238 +4239 POINT(34.792940912154336 -117.99251326490518) bank4239 +4240 POINT(34.79650860977548 -117.47651324588062) bank4240 +4241 POINT(33.9860076465832 -117.74405797594083) bank4241 +4242 POINT(33.60442728917947 -117.92349623379093) bank4242 +4243 POINT(33.23110289501788 -118.03599071329437) bank4243 +4244 POINT(33.19061785136006 -119.17879912641185) bank4244 +4245 POINT(34.36998122581897 -117.72996488418059) bank4245 +4246 POINT(33.527760731023676 -118.13538456895579) bank4246 +4247 POINT(34.45191248716763 -118.49184387685077) bank4247 +4248 POINT(33.07424370985341 -117.99785095176553) bank4248 +4249 POINT(33.41287778177614 -118.08682675184617) bank4249 +4250 POINT(34.68270880571687 -118.69383064053352) bank4250 +4251 POINT(34.484607097849285 -119.09070961546898) bank4251 +4252 POINT(34.2809048430904 -118.20483343532509) bank4252 +4253 POINT(34.69701426506657 -117.29214772492493) bank4253 +4254 POINT(33.48940830877188 -117.37553912178157) bank4254 +4255 POINT(33.67860480437571 -118.79468255485475) bank4255 +4256 POINT(33.98193757813066 -117.5669921919294) bank4256 +4257 POINT(34.36591846499982 -118.77126469400507) bank4257 +4258 POINT(34.6457260687307 -118.22355640541019) bank4258 +4259 POINT(33.94606259289702 -118.64415247836251) bank4259 +4260 POINT(33.07896351744655 -118.30101234785405) bank4260 +4261 POINT(34.90598370290077 -119.17357968757996) bank4261 +4262 POINT(33.46507267376158 -118.9949004793525) bank4262 +4263 POINT(34.11685427722554 -118.9198791726312) bank4263 +4264 POINT(33.65302399457815 -119.001045258657) bank4264 +4265 POINT(34.16012044220264 -119.05774846730992) bank4265 +4266 POINT(34.66889695747879 -117.90272740265549) bank4266 +4267 POINT(34.71459450384176 -118.50423192499852) bank4267 +4268 POINT(33.45355448690639 -118.18004435472521) bank4268 +4269 POINT(33.28971471262335 -118.50991651612861) bank4269 +4270 POINT(34.6697351483793 -117.77649640016185) bank4270 +4271 POINT(33.47434921567107 -118.67874280845517) bank4271 +4272 POINT(34.332646586739536 -117.94179754344528) bank4272 +4273 POINT(33.1092483427569 -118.18397812424976) bank4273 +4274 POINT(33.45348420771243 -118.44816829494374) bank4274 +4275 POINT(34.49075350672189 -118.78034282949625) bank4275 +4276 POINT(33.920328581098616 -118.34675235718817) bank4276 +4277 POINT(34.6741437520603 -118.06756739945001) bank4277 +4278 POINT(34.738799973830645 -117.54289421304173) bank4278 +4279 POINT(33.60141879122868 -117.44780734284957) bank4279 +4280 POINT(33.892997105850384 -118.37363170120219) bank4280 +4281 POINT(33.158047653330975 -119.16863198224007) bank4281 +4282 POINT(34.507531605486996 -119.15648293541761) bank4282 +4283 POINT(34.668330558165 -118.23996933975114) bank4283 +4284 POINT(34.82478319169435 -118.83338925433492) bank4284 +4285 POINT(33.08618679545631 -118.29584934010572) bank4285 +4286 POINT(33.95799990497882 -117.55416469144343) bank4286 +4287 POINT(34.19961297817088 -117.55134929283945) bank4287 +4288 POINT(33.507609257873774 -118.9171628337926) bank4288 +4289 POINT(33.39401762385593 -117.26251311028375) bank4289 +4290 POINT(34.8172495513438 -118.50900368916473) bank4290 +4291 POINT(34.561375666974776 -117.72789055032935) bank4291 +4292 POINT(33.76200285586831 -117.36801387104502) bank4292 +4293 POINT(33.80095115835582 -118.22677025037767) bank4293 +4294 POINT(34.23497070250256 -118.4525596110091) bank4294 +4295 POINT(34.23936099533658 -119.20151541330542) bank4295 +4296 POINT(33.18893417354165 -118.7827649089551) bank4296 +4297 POINT(34.73404964960445 -117.9238486606431) bank4297 +4298 POINT(34.67952281907387 -118.18488984769837) bank4298 +4299 POINT(33.79352758729348 -117.35191659228985) bank4299 +4300 POINT(34.140858538131084 -118.30293292309604) bank4300 +4301 POINT(33.65438878142456 -118.89639365374) bank4301 +4302 POINT(34.262690987907554 -118.93906193700708) bank4302 +4303 POINT(33.67649821168501 -117.37907969066892) bank4303 +4304 POINT(33.615903529249216 -119.08615472424462) bank4304 +4305 POINT(33.619483852538934 -117.91175331344793) bank4305 +4306 POINT(33.512287119659256 -119.01674145359486) bank4306 +4307 POINT(33.65865158920772 -118.47146448161605) bank4307 +4308 POINT(34.81562567058525 -118.11164917931005) bank4308 +4309 POINT(33.92836184309887 -118.76577989467528) bank4309 +4310 POINT(34.08514172996774 -119.00251686497437) bank4310 +4311 POINT(33.251991472632795 -117.4801322636892) bank4311 +4312 POINT(34.09874907779612 -118.13790297874708) bank4312 +4313 POINT(34.57669962831246 -117.50524646861611) bank4313 +4314 POINT(34.47270821153866 -117.53963773121235) bank4314 +4315 POINT(34.60511947129351 -118.88041317627203) bank4315 +4316 POINT(33.57047687579624 -118.88271112912867) bank4316 +4317 POINT(34.267048061913464 -119.22007324348249) bank4317 +4318 POINT(34.00882128266171 -117.57279323218958) bank4318 +4319 POINT(33.97968881443391 -117.8503566884835) bank4319 +4320 POINT(33.476496523819634 -118.44208622735823) bank4320 +4321 POINT(33.65644317008756 -117.9901593771575) bank4321 +4322 POINT(33.13904752029059 -118.50254144666629) bank4322 +4323 POINT(34.83637901613162 -117.66898360587247) bank4323 +4324 POINT(33.06726136543548 -118.36842662249293) bank4324 +4325 POINT(34.822823935720415 -117.56468331507477) bank4325 +4326 POINT(34.070296748517976 -118.5972720434337) bank4326 +4327 POINT(33.92804820680449 -117.98390346513995) bank4327 +4328 POINT(33.5655188235978 -118.4388865084952) bank4328 +4329 POINT(34.499668022811825 -119.0746784133686) bank4329 +4330 POINT(33.51912207766759 -117.33732747573053) bank4330 +4331 POINT(34.941270511258004 -118.24264392157251) bank4331 +4332 POINT(33.53027794200022 -117.82301307373406) bank4332 +4333 POINT(34.28468354557385 -119.16205294346476) bank4333 +4334 POINT(34.14787043112584 -117.53599468902279) bank4334 +4335 POINT(34.41859010151484 -117.2745565172268) bank4335 +4336 POINT(33.40506944672978 -117.79462736544095) bank4336 +4337 POINT(34.228393357074374 -118.91258125747595) bank4337 +4338 POINT(33.37709545148403 -118.7092044035754) bank4338 +4339 POINT(34.54190160366919 -117.49972974729634) bank4339 +4340 POINT(34.407457715654886 -118.17995693734431) bank4340 +4341 POINT(34.5917323547189 -117.27628847267759) bank4341 +4342 POINT(33.30765007203549 -117.7305461371244) bank4342 +4343 POINT(35.048607776300706 -119.02878240373782) bank4343 +4344 POINT(33.44890655257174 -118.2623638757871) bank4344 +4345 POINT(33.199300789446625 -119.16929300676927) bank4345 +4346 POINT(33.10342931575908 -117.64856153627323) bank4346 +4347 POINT(34.925463789710726 -117.7054704341158) bank4347 +4348 POINT(34.173906093798834 -117.30228916743077) bank4348 +4349 POINT(34.096809401276104 -119.1059223039512) bank4349 +4350 POINT(34.69774132580924 -118.68887917609804) bank4350 +4351 POINT(33.44236626975492 -117.94962059252687) bank4351 +4352 POINT(33.10839308185935 -119.22944865181047) bank4352 +4353 POINT(33.14744762934673 -118.7888592323681) bank4353 +4354 POINT(35.01823073681848 -118.16358891505864) bank4354 +4355 POINT(34.91326355034188 -118.92077405400204) bank4355 +4356 POINT(34.26928572472783 -119.15391435073111) bank4356 +4357 POINT(33.18877792764238 -118.81353058406756) bank4357 +4358 POINT(33.34451622938154 -119.21251579842377) bank4358 +4359 POINT(33.92059394738182 -117.71646040109026) bank4359 +4360 POINT(33.923864276273996 -117.95506900393715) bank4360 +4361 POINT(33.59895135459412 -118.6117849535655) bank4361 +4362 POINT(34.9088707315061 -117.72491783705206) bank4362 +4363 POINT(34.35349330770705 -119.0911544434776) bank4363 +4364 POINT(34.00717913336106 -117.81774566510384) bank4364 +4365 POINT(34.661066427425126 -117.70132715582368) bank4365 +4366 POINT(33.108372494593965 -118.73803326207177) bank4366 +4367 POINT(33.0735846119828 -117.89255249009793) bank4367 +4368 POINT(34.26151057189053 -117.59136731779537) bank4368 +4369 POINT(34.54018058837651 -118.91954980919975) bank4369 +4370 POINT(33.198049888948326 -118.29713862800216) bank4370 +4371 POINT(34.695290351013895 -118.81828071898342) bank4371 +4372 POINT(34.90359504568759 -118.02264696296017) bank4372 +4373 POINT(34.15938174073011 -117.82421629292766) bank4373 +4374 POINT(33.11529966212978 -118.56122854939741) bank4374 +4375 POINT(34.60533061370551 -119.1022406667872) bank4375 +4376 POINT(33.55245123166388 -117.76338650925278) bank4376 +4377 POINT(34.82752005160637 -118.31850103345072) bank4377 +4378 POINT(33.474833183727846 -118.44577132734037) bank4378 +4379 POINT(33.93855061189706 -117.9434874098543) bank4379 +4380 POINT(33.664576251282114 -118.03155500396173) bank4380 +4381 POINT(33.56573320176526 -119.19251802697484) bank4381 +4382 POINT(33.606217755221955 -117.90732079271977) bank4382 +4383 POINT(34.55356138467072 -119.11586681043599) bank4383 +4384 POINT(33.98344790537925 -118.8993239756941) bank4384 +4385 POINT(33.15620061101674 -117.72404534306219) bank4385 +4386 POINT(33.42594353361034 -118.98363905549968) bank4386 +4387 POINT(33.86744215118972 -118.54201964530819) bank4387 +4388 POINT(34.287976260401464 -119.19357334347154) bank4388 +4389 POINT(33.207802276426925 -118.97266414107222) bank4389 +4390 POINT(34.21548426371155 -118.59389334068725) bank4390 +4391 POINT(34.01390714417392 -118.13037714574506) bank4391 +4392 POINT(34.01374233926389 -117.45904939988397) bank4392 +4393 POINT(34.60302557955657 -117.63807157010066) bank4393 +4394 POINT(33.49501413618625 -118.27210438044305) bank4394 +4395 POINT(33.40913058195908 -118.87637267394341) bank4395 +4396 POINT(34.51013442643111 -118.40301287570723) bank4396 +4397 POINT(33.68149835575813 -117.3158380115632) bank4397 +4398 POINT(33.533022379424715 -118.08171789680793) bank4398 +4399 POINT(33.8606878868204 -118.47844146178953) bank4399 +4400 POINT(34.56252451953088 -117.7303565009374) bank4400 +4401 POINT(34.97703041601587 -118.35527926707285) bank4401 +4402 POINT(33.196025998676966 -118.99159368801767) bank4402 +4403 POINT(33.215738338796584 -117.68282902429002) bank4403 +4404 POINT(34.956818894667116 -117.3291640038629) bank4404 +4405 POINT(34.58947421154477 -117.4560101089076) bank4405 +4406 POINT(33.81644780830577 -117.635546438967) bank4406 +4407 POINT(33.26172566236614 -119.23090185697038) bank4407 +4408 POINT(34.838491600359816 -117.86447769799956) bank4408 +4409 POINT(34.62886581247387 -118.77979907961982) bank4409 +4410 POINT(33.51384941476168 -118.99902145999391) bank4410 +4411 POINT(33.719260732979784 -118.72309831588018) bank4411 +4412 POINT(33.398439820054 -119.05964159470754) bank4412 +4413 POINT(34.535994996098125 -118.53924543219973) bank4413 +4414 POINT(34.6897846784942 -117.90510911352051) bank4414 +4415 POINT(33.46084554388122 -117.75656911490728) bank4415 +4416 POINT(34.94121052593974 -117.61640756959636) bank4416 +4417 POINT(33.840905166158144 -118.35693151067345) bank4417 +4418 POINT(33.96244529886887 -118.36341225033806) bank4418 +4419 POINT(33.30315640745835 -119.02759182324637) bank4419 +4420 POINT(34.225893217484966 -117.32953729846047) bank4420 +4421 POINT(34.149709535723616 -119.1817440342778) bank4421 +4422 POINT(33.122328296395864 -118.57622620314517) bank4422 +4423 POINT(34.20417158576208 -117.3485678196778) bank4423 +4424 POINT(34.45810851007522 -118.51118089628697) bank4424 +4425 POINT(34.97699382115646 -118.92065732081394) bank4425 +4426 POINT(34.292178862664436 -117.76525787733841) bank4426 +4427 POINT(33.11427129011288 -118.81868343699337) bank4427 +4428 POINT(33.58675211201418 -119.06638426305824) bank4428 +4429 POINT(33.741878622208034 -118.08050366065396) bank4429 +4430 POINT(34.09811618742386 -117.36705945245481) bank4430 +4431 POINT(33.61190909657318 -118.39571168616315) bank4431 +4432 POINT(33.73635219889884 -118.4204969422909) bank4432 +4433 POINT(34.47139897656725 -119.14568961141245) bank4433 +4434 POINT(34.00191902288639 -118.31351630207779) bank4434 +4435 POINT(33.10464494625558 -118.17255306487186) bank4435 +4436 POINT(33.550293964929175 -118.88905116349684) bank4436 +4437 POINT(35.02046277647608 -117.38092523996094) bank4437 +4438 POINT(34.12730755195009 -118.56982089700243) bank4438 +4439 POINT(33.19549598999395 -118.40804822284781) bank4439 +4440 POINT(33.496959121879804 -118.29778555195982) bank4440 +4441 POINT(33.17838017653848 -118.27686223391237) bank4441 +4442 POINT(34.46162967062347 -117.61975217074912) bank4442 +4443 POINT(33.97913916053913 -117.9223694351842) bank4443 +4444 POINT(33.78515319836172 -117.58587544581918) bank4444 +4445 POINT(33.386048415280676 -119.06372466009168) bank4445 +4446 POINT(33.39373743963935 -118.97424669615354) bank4446 +4447 POINT(33.67465225500976 -117.71608504889173) bank4447 +4448 POINT(34.478016760470084 -117.52647908638696) bank4448 +4449 POINT(34.68737886699301 -118.65910916517744) bank4449 +4450 POINT(34.03673481401125 -118.23213736141405) bank4450 +4451 POINT(34.33596705332536 -119.16903753054072) bank4451 +4452 POINT(34.64576067840852 -118.51931359937633) bank4452 +4453 POINT(33.729399780371295 -117.70218800575786) bank4453 +4454 POINT(33.286523671651416 -118.77549746407342) bank4454 +4455 POINT(34.442571489812785 -117.63311774684243) bank4455 +4456 POINT(34.965803794548734 -118.90620237408649) bank4456 +4457 POINT(34.20435215056901 -118.6967825442411) bank4457 +4458 POINT(34.265672008077544 -118.34851891047613) bank4458 +4459 POINT(33.92186992815716 -117.48432329271012) bank4459 +4460 POINT(33.304719492374296 -117.75451638826381) bank4460 +4461 POINT(34.122783426258685 -118.85168940274404) bank4461 +4462 POINT(33.86305079299874 -118.55876916054689) bank4462 +4463 POINT(34.120946575890095 -117.70236991372207) bank4463 +4464 POINT(35.02252015435862 -118.11058776857432) bank4464 +4465 POINT(34.63788484842731 -117.6102869432845) bank4465 +4466 POINT(33.432401793528626 -118.80943372483327) bank4466 +4467 POINT(34.81586280960746 -118.05673400626567) bank4467 +4468 POINT(33.21312426737176 -118.78872924238416) bank4468 +4469 POINT(34.921068810315504 -117.29906397621156) bank4469 +4470 POINT(34.49152884316906 -117.68167316325544) bank4470 +4471 POINT(34.834140748513875 -119.19286415233245) bank4471 +4472 POINT(34.13047712263936 -118.53335296396061) bank4472 +4473 POINT(33.39172170671809 -118.54791969641356) bank4473 +4474 POINT(34.31772115337527 -117.34529988106911) bank4474 +4475 POINT(34.004471282469815 -117.79668302565115) bank4475 +4476 POINT(33.56463262783832 -117.89633469866509) bank4476 +4477 POINT(34.27553329736063 -118.09981287133166) bank4477 +4478 POINT(33.614278009408096 -119.14898025607692) bank4478 +4479 POINT(34.78088660764669 -117.59372520976774) bank4479 +4480 POINT(33.978750578746926 -118.98861823889732) bank4480 +4481 POINT(34.397503926203235 -118.77546662263241) bank4481 +4482 POINT(33.84420882744174 -117.46259617825088) bank4482 +4483 POINT(33.05977056949771 -117.40866820703148) bank4483 +4484 POINT(34.571991589379905 -118.5011590850964) bank4484 +4485 POINT(34.79190215296473 -118.41685972214684) bank4485 +4486 POINT(34.39542865709361 -117.95854468030214) bank4486 +4487 POINT(33.70944933713868 -118.38550774326971) bank4487 +4488 POINT(33.62789615490781 -118.88975323941615) bank4488 +4489 POINT(33.698763922221794 -118.39767015518285) bank4489 +4490 POINT(34.16135532783347 -118.01121628816152) bank4490 +4491 POINT(33.43347123042621 -117.9310153667689) bank4491 +4492 POINT(34.97011398453228 -117.50456621974315) bank4492 +4493 POINT(33.4502250121337 -118.75379719850342) bank4493 +4494 POINT(34.547082601007745 -118.07712510386757) bank4494 +4495 POINT(33.6838537083082 -118.34211112814143) bank4495 +4496 POINT(34.83101318531273 -118.87865333955651) bank4496 +4497 POINT(34.65614783020172 -118.2560518939856) bank4497 +4498 POINT(34.416816197124625 -118.8042933395794) bank4498 +4499 POINT(33.15110147547225 -118.49630009608856) bank4499 +4500 POINT(33.576747403631224 -118.77281231639644) bank4500 +4501 POINT(34.000716212967 -117.39668181437327) bank4501 +4502 POINT(33.70950324513289 -117.7704707975431) bank4502 +4503 POINT(33.61864425305891 -117.55716676980961) bank4503 +4504 POINT(33.80056950690131 -117.87667059578811) bank4504 +4505 POINT(33.20634095210004 -118.03456779406146) bank4505 +4506 POINT(34.56560187643299 -118.64184236599088) bank4506 +4507 POINT(34.384969234432006 -117.95567640864624) bank4507 +4508 POINT(34.75433250202978 -119.03034505767637) bank4508 +4509 POINT(33.91520034794463 -119.12885102536131) bank4509 +4510 POINT(33.531486424685426 -118.95944118637853) bank4510 +4511 POINT(34.69069603605498 -118.66372550138784) bank4511 +4512 POINT(33.56005270463249 -117.55453388727497) bank4512 +4513 POINT(34.7604337581529 -118.68760795825308) bank4513 +4514 POINT(33.14697245622276 -117.70847564945069) bank4514 +4515 POINT(34.1172730822656 -117.49802050116656) bank4515 +4516 POINT(33.48032345448513 -118.32993345895153) bank4516 +4517 POINT(34.51779192331787 -119.12801777995786) bank4517 +4518 POINT(34.9991959998357 -117.61507069878888) bank4518 +4519 POINT(34.14432505599671 -118.79930787048528) bank4519 +4520 POINT(33.751184933401476 -118.62421512880108) bank4520 +4521 POINT(34.244712042801545 -117.9253095574936) bank4521 +4522 POINT(34.72672523134869 -117.31940512287618) bank4522 +4523 POINT(34.99024053076555 -117.40075332750777) bank4523 +4524 POINT(34.57742517241932 -117.59404293178409) bank4524 +4525 POINT(34.35748880681014 -118.41696501453315) bank4525 +4526 POINT(34.88281534757473 -118.82855819504509) bank4526 +4527 POINT(33.29500818589716 -117.35748595244286) bank4527 +4528 POINT(34.01017573787433 -118.11118868738093) bank4528 +4529 POINT(34.51206757955912 -119.02419745143982) bank4529 +4530 POINT(33.0804058900943 -118.77531639030826) bank4530 +4531 POINT(34.008237355945106 -117.32043284775372) bank4531 +4532 POINT(33.32486908433006 -118.59158108723862) bank4532 +4533 POINT(34.728175878156584 -117.7434069816147) bank4533 +4534 POINT(33.930605054085746 -118.60031805381654) bank4534 +4535 POINT(34.18075861946687 -118.07859039535838) bank4535 +4536 POINT(33.17029039994347 -118.38458720351225) bank4536 +4537 POINT(33.08482279295694 -118.65503169182212) bank4537 +4538 POINT(34.88860466722692 -118.67151031438681) bank4538 +4539 POINT(34.62891138656558 -118.53231593857011) bank4539 +4540 POINT(34.95866985245209 -118.3034924266735) bank4540 +4541 POINT(35.00465818057353 -117.49334238771984) bank4541 +4542 POINT(33.54125015016596 -117.81843009830281) bank4542 +4543 POINT(34.89937944548033 -118.53549385941189) bank4543 +4544 POINT(34.022541711117036 -118.60841529744836) bank4544 +4545 POINT(34.15163055798247 -117.58220712010429) bank4545 +4546 POINT(34.32489460853761 -118.67550153362603) bank4546 +4547 POINT(34.139521511029436 -118.13105985347534) bank4547 +4548 POINT(33.31942550102278 -119.11282763195108) bank4548 +4549 POINT(34.82860275262895 -117.97353926000743) bank4549 +4550 POINT(34.88696628962213 -118.88147658638496) bank4550 +4551 POINT(33.990352839962114 -118.11556046156741) bank4551 +4552 POINT(34.70236469000545 -118.47276920716533) bank4552 +4553 POINT(33.378493494793716 -118.24186265037369) bank4553 +4554 POINT(33.145033615491386 -118.1126570208689) bank4554 +4555 POINT(33.76746650276973 -118.21707181313054) bank4555 +4556 POINT(33.37731327568002 -117.85045741355822) bank4556 +4557 POINT(33.19341771690982 -118.37703207840443) bank4557 +4558 POINT(33.50571284287067 -117.41446854207081) bank4558 +4559 POINT(34.503706441173264 -117.46317240624312) bank4559 +4560 POINT(33.116724769025794 -117.25579104349062) bank4560 +4561 POINT(34.18518785076614 -118.11276270319081) bank4561 +4562 POINT(34.87102936235068 -117.92418708277957) bank4562 +4563 POINT(33.1669785000783 -119.1272474219184) bank4563 +4564 POINT(34.29266443794525 -118.89396043035323) bank4564 +4565 POINT(33.59131186615741 -118.16314682988428) bank4565 +4566 POINT(34.74370211393049 -118.21340235191644) bank4566 +4567 POINT(33.46167699553038 -118.31927958646811) bank4567 +4568 POINT(33.898948358857176 -117.97898850988366) bank4568 +4569 POINT(33.10407470114633 -119.218325887404) bank4569 +4570 POINT(35.02363238864905 -118.46691996964152) bank4570 +4571 POINT(34.15454738692852 -117.71521230675675) bank4571 +4572 POINT(34.50416091174676 -117.87536014732449) bank4572 +4573 POINT(33.43663757720303 -118.41606108860822) bank4573 +4574 POINT(33.11957053420347 -117.68918449458246) bank4574 +4575 POINT(34.007531202752276 -117.45538107838536) bank4575 +4576 POINT(34.460585584283024 -117.49846216135626) bank4576 +4577 POINT(34.660164268382395 -118.65813117285464) bank4577 +4578 POINT(34.11257092108742 -117.6552304282325) bank4578 +4579 POINT(33.819655700565384 -118.20585962408035) bank4579 +4580 POINT(33.770049005126026 -119.00505476632155) bank4580 +4581 POINT(33.31908695257042 -117.25122190060277) bank4581 +4582 POINT(34.521187058409694 -117.40456764893892) bank4582 +4583 POINT(33.68100275790234 -117.37446506968423) bank4583 +4584 POINT(34.099366304288125 -118.57229795413268) bank4584 +4585 POINT(34.641633560507294 -119.2240818850355) bank4585 +4586 POINT(33.80425351057505 -118.78350033692702) bank4586 +4587 POINT(34.93154040993932 -117.8227571670762) bank4587 +4588 POINT(33.75813179889221 -117.88191057323027) bank4588 +4589 POINT(33.683152837113475 -119.19731301198782) bank4589 +4590 POINT(34.99513769678398 -118.18474623803417) bank4590 +4591 POINT(34.61966062337615 -118.84719326205223) bank4591 +4592 POINT(33.57717935589682 -119.02967517055694) bank4592 +4593 POINT(33.20526124878292 -117.76439505301943) bank4593 +4594 POINT(34.743463850376926 -119.0280531794435) bank4594 +4595 POINT(33.31642346645343 -118.66359128527854) bank4595 +4596 POINT(33.21633406003338 -119.09364820924966) bank4596 +4597 POINT(33.60914373727887 -118.30741558764137) bank4597 +4598 POINT(34.76433532076912 -117.58402826141487) bank4598 +4599 POINT(33.175143180842355 -118.67769853588132) bank4599 +4600 POINT(33.36004300097953 -118.53063880042247) bank4600 +4601 POINT(35.04850649510713 -118.28463396029431) bank4601 +4602 POINT(33.60949260891815 -119.21705305807933) bank4602 +4603 POINT(34.09382153952525 -119.00753041791883) bank4603 +4604 POINT(34.27049496479775 -118.04863748124862) bank4604 +4605 POINT(34.33421858374811 -118.72551096373354) bank4605 +4606 POINT(34.87672155105441 -118.48419767539917) bank4606 +4607 POINT(33.53926573036336 -118.3797084941346) bank4607 +4608 POINT(34.26486128346783 -117.6310266690176) bank4608 +4609 POINT(34.427690018989914 -118.62813690372832) bank4609 +4610 POINT(33.40566990667025 -119.18609989068835) bank4610 +4611 POINT(34.68734994943489 -117.64980007231085) bank4611 +4612 POINT(33.39447680609008 -118.94798518942086) bank4612 +4613 POINT(33.97809263952052 -118.67189698563945) bank4613 +4614 POINT(33.49722274373912 -117.46469266148718) bank4614 +4615 POINT(34.02380555454804 -117.99171865823523) bank4615 +4616 POINT(34.888911398709105 -119.2372731814865) bank4616 +4617 POINT(34.397026624280166 -118.25165668737755) bank4617 +4618 POINT(33.375047170930976 -119.14012516165144) bank4618 +4619 POINT(33.137131389776975 -117.84787028649532) bank4619 +4620 POINT(34.284685471329226 -118.31829986702763) bank4620 +4621 POINT(34.08489847392652 -117.98150912869617) bank4621 +4622 POINT(34.45523079984876 -118.65324764015563) bank4622 +4623 POINT(34.66385831446463 -118.10946160073568) bank4623 +4624 POINT(33.46042566624928 -119.2040969205261) bank4624 +4625 POINT(34.92393782270944 -118.83271138380694) bank4625 +4626 POINT(33.38475879602009 -119.01777718223373) bank4626 +4627 POINT(33.45796220287911 -118.33891098353052) bank4627 +4628 POINT(35.051562308964414 -117.30709097670604) bank4628 +4629 POINT(33.88960473331778 -118.41915738319362) bank4629 +4630 POINT(33.22397783673732 -117.2915079526992) bank4630 +4631 POINT(34.97909610353356 -117.32794676979027) bank4631 +4632 POINT(33.58759488479156 -118.4675672717463) bank4632 +4633 POINT(34.37431105629984 -119.052455743402) bank4633 +4634 POINT(33.983436831234755 -118.57567186264525) bank4634 +4635 POINT(34.52009660932138 -119.01901336881623) bank4635 +4636 POINT(34.92558801933642 -117.81439135729352) bank4636 +4637 POINT(33.19753326442221 -117.80380727741185) bank4637 +4638 POINT(34.889769292429946 -118.26931404914009) bank4638 +4639 POINT(33.36364699031789 -118.06194994565537) bank4639 +4640 POINT(33.74469928846793 -117.36025149279865) bank4640 +4641 POINT(33.70194397837148 -117.75817073843655) bank4641 +4642 POINT(35.04695023057707 -118.48616968731245) bank4642 +4643 POINT(33.53535674387098 -117.36054393755431) bank4643 +4644 POINT(33.78771908491275 -118.20737328052424) bank4644 +4645 POINT(34.60234156088353 -117.28180702795372) bank4645 +4646 POINT(33.2643700244694 -119.02961216776441) bank4646 +4647 POINT(34.89696299936956 -118.57600679687586) bank4647 +4648 POINT(33.19414499985766 -117.26861161631497) bank4648 +4649 POINT(33.965781474357875 -119.03738295559698) bank4649 +4650 POINT(33.29116624730557 -119.03425607376771) bank4650 +4651 POINT(34.17679945621899 -119.04185825595351) bank4651 +4652 POINT(34.28422862496283 -118.19678975475186) bank4652 +4653 POINT(33.657339288882326 -117.6788709583808) bank4653 +4654 POINT(33.95869298858401 -118.87602166887947) bank4654 +4655 POINT(33.41425651893532 -117.98831149516707) bank4655 +4656 POINT(33.132364717715156 -118.426588922275) bank4656 +4657 POINT(34.85962122422591 -118.89905580936043) bank4657 +4658 POINT(34.77922764968094 -117.36429089231868) bank4658 +4659 POINT(34.2182582592739 -118.69383048215037) bank4659 +4660 POINT(33.820280667225525 -118.8677673187277) bank4660 +4661 POINT(34.093033391747525 -117.25920143459436) bank4661 +4662 POINT(34.57784316206621 -118.7738397737762) bank4662 +4663 POINT(33.61393175281614 -118.85070373458298) bank4663 +4664 POINT(33.97938247304786 -119.18761107818915) bank4664 +4665 POINT(34.7399793213399 -118.87572311491395) bank4665 +4666 POINT(34.29410163713991 -117.9949706433132) bank4666 +4667 POINT(34.25618206420618 -118.17884081509244) bank4667 +4668 POINT(34.49455345173492 -118.80109293523161) bank4668 +4669 POINT(34.96890795510612 -117.91632932749397) bank4669 +4670 POINT(33.13142571367176 -117.78704493922979) bank4670 +4671 POINT(33.4457483084195 -118.52346785658034) bank4671 +4672 POINT(33.53256393637899 -117.45958641180755) bank4672 +4673 POINT(34.99028705438852 -117.75582680147842) bank4673 +4674 POINT(34.174087642123936 -118.34829464950795) bank4674 +4675 POINT(33.64354031481619 -119.1550876674505) bank4675 +4676 POINT(34.086589997091444 -118.89547092072353) bank4676 +4677 POINT(34.37583195820539 -118.48991490258716) bank4677 +4678 POINT(33.09701243046774 -118.0251523505286) bank4678 +4679 POINT(34.358117875168546 -119.24297590765008) bank4679 +4680 POINT(34.20887543917729 -118.36944789152878) bank4680 +4681 POINT(34.43549752930687 -118.68016650721782) bank4681 +4682 POINT(33.687609927809625 -119.05969791924167) bank4682 +4683 POINT(34.51493132896759 -118.34669221645429) bank4683 +4684 POINT(33.235812044783465 -117.83144779625296) bank4684 +4685 POINT(34.00259882798916 -117.6737558921948) bank4685 +4686 POINT(33.574360363816645 -117.33759333576327) bank4686 +4687 POINT(34.92739727299151 -117.85082602644205) bank4687 +4688 POINT(33.084473704038224 -117.73478563173848) bank4688 +4689 POINT(33.14723650233941 -119.0659916829371) bank4689 +4690 POINT(33.09947609474897 -117.60061858229683) bank4690 +4691 POINT(33.17467465678955 -118.53252499419607) bank4691 +4692 POINT(34.296220085505226 -117.31806879447336) bank4692 +4693 POINT(33.92625519101786 -118.31231327614711) bank4693 +4694 POINT(34.89772063501124 -118.47675115577593) bank4694 +4695 POINT(33.45425569166417 -118.71321433730884) bank4695 +4696 POINT(34.45482204677762 -118.69926044513815) bank4696 +4697 POINT(34.15985327504745 -118.69202345179107) bank4697 +4698 POINT(34.388099409525665 -117.47063829410276) bank4698 +4699 POINT(34.722284592648045 -117.47296306459357) bank4699 +4700 POINT(33.48335752128989 -117.97610465705304) bank4700 +4701 POINT(33.59190495831667 -119.12210627982151) bank4701 +4702 POINT(33.875687439408644 -117.4813576643024) bank4702 +4703 POINT(33.56478115949348 -117.78057259838705) bank4703 +4704 POINT(33.492812484492894 -119.17524251750552) bank4704 +4705 POINT(34.91997062464901 -118.80234618583147) bank4705 +4706 POINT(33.981260988040056 -119.03103291034014) bank4706 +4707 POINT(33.905738533881575 -118.04699374031482) bank4707 +4708 POINT(33.280766582233326 -117.99285277642298) bank4708 +4709 POINT(34.372114261096705 -118.36294909794654) bank4709 +4710 POINT(33.25631183145045 -117.4860332838921) bank4710 +4711 POINT(34.25111086205684 -118.25301550756444) bank4711 +4712 POINT(33.24014961825915 -118.33077583747226) bank4712 +4713 POINT(33.1144867539151 -118.33234535692448) bank4713 +4714 POINT(34.139769286611816 -118.30743729775432) bank4714 +4715 POINT(33.16086827009103 -118.87289001568861) bank4715 +4716 POINT(33.82449644184723 -118.12702836700845) bank4716 +4717 POINT(33.68870736683951 -118.30017229188351) bank4717 +4718 POINT(34.94517218701997 -117.56037094600222) bank4718 +4719 POINT(34.33829465238773 -117.82839446533539) bank4719 +4720 POINT(34.15170196364366 -117.77331716774245) bank4720 +4721 POINT(34.907812901272074 -118.97347079402735) bank4721 +4722 POINT(33.9708590854936 -118.13723708514185) bank4722 +4723 POINT(33.14634201122611 -119.17120669922694) bank4723 +4724 POINT(34.956103177856725 -118.32868038670503) bank4724 +4725 POINT(34.457300525562744 -118.91150983062002) bank4725 +4726 POINT(34.249996174067014 -117.9223802209471) bank4726 +4727 POINT(34.922558494214805 -118.65048371199691) bank4727 +4728 POINT(34.7976946378866 -118.16225996101122) bank4728 +4729 POINT(33.70356534315572 -119.07479869579915) bank4729 +4730 POINT(34.01296551035786 -117.38170807677169) bank4730 +4731 POINT(33.157203680687566 -118.44382175523961) bank4731 +4732 POINT(34.704606336631024 -118.01172097975319) bank4732 +4733 POINT(33.23109178913634 -119.02084418741315) bank4733 +4734 POINT(33.14153770608033 -117.58817455230273) bank4734 +4735 POINT(33.262342049978706 -117.82055215628272) bank4735 +4736 POINT(33.11528538184757 -118.22395668020175) bank4736 +4737 POINT(33.641536076059595 -118.22856498399746) bank4737 +4738 POINT(33.24525788023404 -118.4574875084948) bank4738 +4739 POINT(33.98116740662604 -117.55147131845933) bank4739 +4740 POINT(34.94443136703278 -117.32687756152372) bank4740 +4741 POINT(33.88350763243413 -118.65769380884763) bank4741 +4742 POINT(34.04348038920549 -117.6918553916575) bank4742 +4743 POINT(33.08943009763456 -118.29250296629766) bank4743 +4744 POINT(33.2647457156485 -119.05378190687493) bank4744 +4745 POINT(33.9193994424952 -117.4851173128877) bank4745 +4746 POINT(33.256136102920586 -118.66673543924611) bank4746 +4747 POINT(34.75612052755057 -117.55261236217588) bank4747 +4748 POINT(33.775212735926964 -117.4543161798681) bank4748 +4749 POINT(34.33074750487157 -118.02074029314534) bank4749 +4750 POINT(34.3144579210082 -118.46849387335709) bank4750 +4751 POINT(33.641730409332574 -117.3792662946612) bank4751 +4752 POINT(33.95627771259269 -117.92713722745513) bank4752 +4753 POINT(34.19359657086677 -118.04378702150801) bank4753 +4754 POINT(33.22985986722965 -117.79213883327132) bank4754 +4755 POINT(33.71433852662845 -117.39527418367705) bank4755 +4756 POINT(34.90131714720958 -119.17589058662197) bank4756 +4757 POINT(35.042253958360604 -117.6411814976951) bank4757 +4758 POINT(34.633523432714505 -118.37572440999894) bank4758 +4759 POINT(33.139931194393064 -117.79856696714182) bank4759 +4760 POINT(34.09986822800173 -118.43077284887721) bank4760 +4761 POINT(33.161843254311115 -119.19004978867997) bank4761 +4762 POINT(33.77133146610277 -118.40981618442571) bank4762 +4763 POINT(34.00082293733506 -117.35167222597555) bank4763 +4764 POINT(33.781711713210505 -117.27031730512627) bank4764 +4765 POINT(34.71078575712597 -118.48461560621404) bank4765 +4766 POINT(34.8029857409267 -117.53629611449003) bank4766 +4767 POINT(33.85936100685748 -117.38497567348317) bank4767 +4768 POINT(33.29667980184815 -118.15336417519026) bank4768 +4769 POINT(34.45807983871107 -118.22362240104034) bank4769 +4770 POINT(34.55877413474422 -118.1639643262708) bank4770 +4771 POINT(34.459921626978435 -117.73255926590056) bank4771 +4772 POINT(34.6129086982489 -117.65589002236567) bank4772 +4773 POINT(33.204088306481424 -118.78590288885412) bank4773 +4774 POINT(33.53083449836858 -118.83291518863494) bank4774 +4775 POINT(34.64031554865024 -119.19894743879458) bank4775 +4776 POINT(33.70420144749431 -117.73840431627916) bank4776 +4777 POINT(34.23196127599308 -118.57110338801147) bank4777 +4778 POINT(34.82322256454315 -117.98444387657491) bank4778 +4779 POINT(33.80419725685745 -118.15627095035272) bank4779 +4780 POINT(33.66987359107216 -117.97323660899731) bank4780 +4781 POINT(34.13375680551697 -119.13406540737078) bank4781 +4782 POINT(33.61030797839392 -118.33020250683255) bank4782 +4783 POINT(33.147256949483015 -117.41223957774534) bank4783 +4784 POINT(33.313307973839926 -118.91698297808051) bank4784 +4785 POINT(34.670338232306385 -119.10142932958614) bank4785 +4786 POINT(33.76787907715586 -119.00515601274643) bank4786 +4787 POINT(33.60973711300686 -118.9273417187501) bank4787 +4788 POINT(34.372345895034314 -118.77802199670947) bank4788 +4789 POINT(33.402512425550405 -117.84383870012611) bank4789 +4790 POINT(33.09238102172835 -118.91945069191569) bank4790 +4791 POINT(34.50481820915988 -117.60961331358855) bank4791 +4792 POINT(33.556994742838036 -119.00834659340546) bank4792 +4793 POINT(33.816282321350634 -117.27381826773446) bank4793 +4794 POINT(33.06827326920141 -118.76445785146352) bank4794 +4795 POINT(33.278483934694194 -118.65265913060759) bank4795 +4796 POINT(34.01088948983728 -117.53396345362846) bank4796 +4797 POINT(33.46568334148092 -119.08459059895577) bank4797 +4798 POINT(33.90009204774297 -118.44412289487107) bank4798 +4799 POINT(33.53072619086429 -119.14295688621695) bank4799 +4800 POINT(33.15757364808025 -118.87653739707294) bank4800 +4801 POINT(33.23537878660964 -118.17733855023172) bank4801 +4802 POINT(33.53123106802172 -119.15000556259047) bank4802 +4803 POINT(34.924228009308635 -118.80862617440532) bank4803 +4804 POINT(34.77814547144921 -117.83836263625568) bank4804 +4805 POINT(34.73095292954501 -118.18237978813863) bank4805 +4806 POINT(33.19038933005189 -117.6516081798648) bank4806 +4807 POINT(33.650641209483474 -117.68153335006559) bank4807 +4808 POINT(33.784838331807826 -117.24750841762874) bank4808 +4809 POINT(35.03868087947419 -118.70773899247685) bank4809 +4810 POINT(34.98300412815706 -118.77192500404318) bank4810 +4811 POINT(33.514486265415385 -118.78127600427055) bank4811 +4812 POINT(34.38280957047231 -119.20902927127274) bank4812 +4813 POINT(33.35156783824803 -118.39858284842188) bank4813 +4814 POINT(34.76421868258103 -118.62017940812798) bank4814 +4815 POINT(33.09827294976535 -117.6338642325778) bank4815 +4816 POINT(33.09848509486516 -119.07711565674305) bank4816 +4817 POINT(33.33324589511474 -119.1602672330535) bank4817 +4818 POINT(33.530151168432084 -118.34637183453995) bank4818 +4819 POINT(33.51928381641477 -119.20430263544976) bank4819 +4820 POINT(33.68926806935467 -117.81688693677978) bank4820 +4821 POINT(34.302868767877186 -117.4040828372948) bank4821 +4822 POINT(33.57398327754936 -118.44068297985814) bank4822 +4823 POINT(33.46324434619464 -117.95795183472731) bank4823 +4824 POINT(33.260611269304945 -117.52343524600624) bank4824 +4825 POINT(33.20639391387816 -118.06416102416587) bank4825 +4826 POINT(33.94054966068409 -117.2648813787611) bank4826 +4827 POINT(33.05253053566662 -117.57891030911384) bank4827 +4828 POINT(34.131943715532856 -118.67678455753729) bank4828 +4829 POINT(34.216664475107294 -118.03581439806632) bank4829 +4830 POINT(33.124529246528475 -119.24287871509964) bank4830 +4831 POINT(33.4241020304374 -117.47175062431955) bank4831 +4832 POINT(33.89705443657373 -118.84848149666911) bank4832 +4833 POINT(33.469127014636236 -118.43288223867003) bank4833 +4834 POINT(34.79664020328429 -118.46005773853453) bank4834 +4835 POINT(33.19585045825898 -117.99221834048963) bank4835 +4836 POINT(34.80309195454689 -118.1144944879566) bank4836 +4837 POINT(34.1446408862056 -119.1355347896129) bank4837 +4838 POINT(34.46566666143953 -118.02392403143203) bank4838 +4839 POINT(33.59126857868542 -117.70472752290623) bank4839 +4840 POINT(33.81850136390903 -118.15998681134495) bank4840 +4841 POINT(33.86236466762103 -117.98829714358456) bank4841 +4842 POINT(34.24058746910028 -117.76719841337325) bank4842 +4843 POINT(33.59422770447815 -117.89699509649077) bank4843 +4844 POINT(34.92007079411374 -117.97181299230611) bank4844 +4845 POINT(33.49537919166546 -118.10911904492473) bank4845 +4846 POINT(33.82376932725439 -118.57447885507983) bank4846 +4847 POINT(34.16765972871027 -119.2154914665869) bank4847 +4848 POINT(34.36044578058571 -118.72064326092601) bank4848 +4849 POINT(33.545945624013235 -117.31243138787217) bank4849 +4850 POINT(34.64385271897296 -118.10437658879296) bank4850 +4851 POINT(34.68317458741064 -118.35521042560337) bank4851 +4852 POINT(34.98750360315059 -118.12927208459368) bank4852 +4853 POINT(34.726837565921485 -118.13758561903349) bank4853 +4854 POINT(34.72414097357424 -119.12361257752342) bank4854 +4855 POINT(33.52209027337157 -117.35292485632513) bank4855 +4856 POINT(33.11373517800286 -118.71394048363263) bank4856 +4857 POINT(33.74979363017836 -118.16926051290811) bank4857 +4858 POINT(33.62969190227378 -118.21961155676975) bank4858 +4859 POINT(34.278941079023724 -118.4981083010264) bank4859 +4860 POINT(33.7552305409655 -118.72074459827894) bank4860 +4861 POINT(34.65876446825854 -118.36453186727448) bank4861 +4862 POINT(33.32253109560963 -118.48603815529854) bank4862 +4863 POINT(34.33726736057444 -117.44418306327445) bank4863 +4864 POINT(33.91110420404727 -118.17303089491094) bank4864 +4865 POINT(34.086174172188 -117.26015498660792) bank4865 +4866 POINT(34.68769584921555 -119.16936302653515) bank4866 +4867 POINT(33.77571829592383 -118.14353588531351) bank4867 +4868 POINT(34.84359622838204 -118.28979717636182) bank4868 +4869 POINT(33.204267282617714 -118.61251896272093) bank4869 +4870 POINT(34.85215799068089 -117.63388321749741) bank4870 +4871 POINT(33.060741498906836 -119.05643737457396) bank4871 +4872 POINT(33.59297650174232 -117.92941383241849) bank4872 +4873 POINT(33.36855497905257 -117.31194074709164) bank4873 +4874 POINT(33.77278940503472 -117.74313760841261) bank4874 +4875 POINT(34.01061707015034 -118.88511016315624) bank4875 +4876 POINT(34.14605738646463 -117.4309206206947) bank4876 +4877 POINT(33.45111015705291 -117.99276640739298) bank4877 +4878 POINT(34.62232592242355 -117.83186053730657) bank4878 +4879 POINT(33.429730193365685 -118.25964214080724) bank4879 +4880 POINT(34.27180911929807 -119.17006242327975) bank4880 +4881 POINT(34.404966626256126 -118.38551595679934) bank4881 +4882 POINT(33.660888962213626 -118.84073016678218) bank4882 +4883 POINT(33.18367806677452 -118.92091753808936) bank4883 +4884 POINT(33.31446974648725 -118.69743998394942) bank4884 +4885 POINT(33.51245052246426 -117.83893208623466) bank4885 +4886 POINT(34.45559873213831 -117.73646152560428) bank4886 +4887 POINT(34.55157809736279 -118.12502622009227) bank4887 +4888 POINT(34.568743339199806 -117.53721820012127) bank4888 +4889 POINT(34.83822489620484 -117.95799588677792) bank4889 +4890 POINT(34.156417214063694 -118.03253290880762) bank4890 +4891 POINT(33.45112636388037 -118.04804406262991) bank4891 +4892 POINT(33.215653873480896 -119.23003613873314) bank4892 +4893 POINT(34.51945711734156 -117.64424518527271) bank4893 +4894 POINT(33.616204269552355 -117.27316238796554) bank4894 +4895 POINT(33.38289003039271 -119.06113899787786) bank4895 +4896 POINT(34.37231321793871 -118.44430874183142) bank4896 +4897 POINT(34.670537420572245 -118.41152600776904) bank4897 +4898 POINT(33.933796935441165 -117.75149795058823) bank4898 +4899 POINT(33.625334442074426 -118.11176193808444) bank4899 +4900 POINT(34.85729634623228 -117.57874274883667) bank4900 +4901 POINT(34.35627116893417 -117.42095025327295) bank4901 +4902 POINT(33.30349413275938 -118.89814334965186) bank4902 +4903 POINT(34.03014110241909 -118.0737730023498) bank4903 +4904 POINT(33.18847829744156 -118.23313250955646) bank4904 +4905 POINT(35.02010983952161 -117.84244171756472) bank4905 +4906 POINT(34.91380753723656 -118.19623573160584) bank4906 +4907 POINT(34.57059823490004 -117.8857211371474) bank4907 +4908 POINT(33.79324188715229 -117.2838430243756) bank4908 +4909 POINT(34.820199678542245 -118.20658426889946) bank4909 +4910 POINT(33.651788567750295 -117.55783750760001) bank4910 +4911 POINT(34.241858508790294 -117.40916355667291) bank4911 +4912 POINT(33.4866312212825 -118.58929049718267) bank4912 +4913 POINT(33.36003109564563 -118.73553780589758) bank4913 +4914 POINT(34.7511850708127 -119.12342908363509) bank4914 +4915 POINT(33.50583635449727 -118.50566364732312) bank4915 +4916 POINT(33.661164403166325 -117.70681683645856) bank4916 +4917 POINT(33.87765238450997 -118.02230464382477) bank4917 +4918 POINT(33.86170749160776 -117.95780424430963) bank4918 +4919 POINT(34.36552471173084 -117.54634123122551) bank4919 +4920 POINT(34.40480681979828 -119.07214043190696) bank4920 +4921 POINT(33.10383610476008 -118.60386985976659) bank4921 +4922 POINT(33.51449057716271 -117.62250716292202) bank4922 +4923 POINT(34.883425517782214 -118.95477230484875) bank4923 +4924 POINT(34.48431652538172 -118.24368007634023) bank4924 +4925 POINT(33.18715947989648 -118.52592090304663) bank4925 +4926 POINT(33.245221134141936 -118.09603356462716) bank4926 +4927 POINT(34.64188386174606 -117.34788729447482) bank4927 +4928 POINT(34.342216373651056 -118.90065231886803) bank4928 +4929 POINT(34.36748764535278 -117.99103724902655) bank4929 +4930 POINT(33.45606358388375 -118.56922195420408) bank4930 +4931 POINT(34.1815476931865 -118.46666904979587) bank4931 +4932 POINT(33.76758912399271 -119.22455351197958) bank4932 +4933 POINT(34.327393342214826 -117.5540125647751) bank4933 +4934 POINT(33.54909757621341 -118.60928885794952) bank4934 +4935 POINT(33.41420035702366 -117.89670530591367) bank4935 +4936 POINT(34.199817059291505 -117.82053093934367) bank4936 +4937 POINT(33.42310543974919 -118.75573246748279) bank4937 +4938 POINT(33.973399561047806 -119.14358248162122) bank4938 +4939 POINT(35.002988022203056 -118.97759666068929) bank4939 +4940 POINT(34.9082121607358 -117.82876855242182) bank4940 +4941 POINT(34.27179403714983 -117.44681221489019) bank4941 +4942 POINT(33.956083852602106 -118.8579602810354) bank4942 +4943 POINT(34.498861963082625 -119.12161698781799) bank4943 +4944 POINT(33.68915154114114 -118.96542261896452) bank4944 +4945 POINT(34.18248258193185 -119.06979995637889) bank4945 +4946 POINT(34.90199305110769 -118.70220778294737) bank4946 +4947 POINT(33.24553715279991 -117.43900320625637) bank4947 +4948 POINT(34.747587802123995 -117.36643170040388) bank4948 +4949 POINT(33.768927668803784 -117.8645907131489) bank4949 +4950 POINT(34.58826941067056 -117.60152557026053) bank4950 +4951 POINT(34.466567948431866 -117.59423881473663) bank4951 +4952 POINT(34.99217691276631 -119.08149004890335) bank4952 +4953 POINT(33.213586599371745 -117.68690179051964) bank4953 +4954 POINT(33.1829092031242 -117.6086410964535) bank4954 +4955 POINT(34.25501985864281 -117.68385194004067) bank4955 +4956 POINT(33.41325872371581 -119.05147162267903) bank4956 +4957 POINT(33.45765636328442 -117.65976455824112) bank4957 +4958 POINT(34.454931458479464 -119.11291313757897) bank4958 +4959 POINT(33.83022920448835 -117.7161908840806) bank4959 +4960 POINT(34.379572796864466 -117.82336938612362) bank4960 +4961 POINT(33.079283923492675 -117.41231461555586) bank4961 +4962 POINT(33.997850433885596 -118.06390277960178) bank4962 +4963 POINT(33.13364649787504 -117.9477779749367) bank4963 +4964 POINT(33.63558873452352 -119.02210427449906) bank4964 +4965 POINT(33.562706547201415 -117.40822849217466) bank4965 +4966 POINT(33.679166165206276 -117.7053937256582) bank4966 +4967 POINT(33.53105631501746 -117.24667540990613) bank4967 +4968 POINT(34.89513069348153 -117.94161189756153) bank4968 +4969 POINT(33.33500940485624 -117.97662479359998) bank4969 +4970 POINT(34.15893654028886 -119.21842578774222) bank4970 +4971 POINT(34.9159071542811 -119.12169710387953) bank4971 +4972 POINT(34.26171040879505 -117.45173501839639) bank4972 +4973 POINT(33.96167183461837 -119.08996794257793) bank4973 +4974 POINT(34.14860516103527 -118.4314609198687) bank4974 +4975 POINT(33.88445736477419 -118.2191295111455) bank4975 +4976 POINT(34.189715038601996 -117.77363007859324) bank4976 +4977 POINT(33.543045952053944 -118.54066681140095) bank4977 +4978 POINT(34.822399599870664 -118.90359915620569) bank4978 +4979 POINT(33.23567036467745 -118.87696108547217) bank4979 +4980 POINT(33.505159313997076 -117.35020022509121) bank4980 +4981 POINT(33.30127812605843 -118.74706992542238) bank4981 +4982 POINT(34.11039323319956 -118.60008049475239) bank4982 +4983 POINT(33.63202301626883 -119.1584109767826) bank4983 +4984 POINT(34.76862527795398 -119.15526235328467) bank4984 +4985 POINT(33.940142019559694 -118.83102623956097) bank4985 +4986 POINT(34.06072826845643 -117.95241945390342) bank4986 +4987 POINT(33.5041257628372 -117.35845036104527) bank4987 +4988 POINT(33.12507472573283 -118.38046125599388) bank4988 +4989 POINT(34.53144651290221 -118.31054491638743) bank4989 +4990 POINT(34.18555095263473 -117.57128778078362) bank4990 +4991 POINT(33.89804137291047 -117.9582357962685) bank4991 +4992 POINT(33.88501829259122 -118.53601872939925) bank4992 +4993 POINT(34.99060880793586 -117.35596166639696) bank4993 +4994 POINT(34.42173101172502 -117.53292689826743) bank4994 +4995 POINT(33.97360169373458 -117.9782881561796) bank4995 +4996 POINT(35.031471886457894 -117.462060803622) bank4996 +4997 POINT(34.70166587197595 -118.12628530522917) bank4997 +4998 POINT(33.87070735256458 -119.12265561155755) bank4998 +4999 POINT(33.369929983650565 -119.05467160425819) bank4999 +5000 POINT(33.256934857051455 -118.10744297914393) bank5000 +5001 POINT(34.71858845699199 -117.55914927364243) bank5001 +5002 POINT(34.383666215248375 -117.85237522380294) bank5002 +5003 POINT(33.71492604139354 -118.51823303041266) bank5003 +5004 POINT(34.518518313280495 -118.31079208277121) bank5004 +5005 POINT(33.71536446663645 -118.39368578002447) bank5005 +5006 POINT(35.04728102138992 -118.47024538733513) bank5006 +5007 POINT(33.494229271178824 -118.11321636259989) bank5007 +5008 POINT(33.465323871263024 -118.70250943908312) bank5008 +5009 POINT(33.62078722830089 -118.95505793771387) bank5009 +5010 POINT(34.054380969275314 -118.94885413479332) bank5010 +5011 POINT(34.33658543514141 -118.56161168656999) bank5011 +5012 POINT(35.03437120413175 -118.97992372903262) bank5012 +5013 POINT(34.07866830891491 -118.49521015108778) bank5013 +5014 POINT(34.93138405016008 -117.73571921324648) bank5014 +5015 POINT(33.70102090881358 -118.21961685086634) bank5015 +5016 POINT(34.592660879750504 -118.03596575086581) bank5016 +5017 POINT(34.6639242439555 -119.173514345702) bank5017 +5018 POINT(34.41451724980858 -118.3843344494608) bank5018 +5019 POINT(33.49973832669278 -118.0837706770862) bank5019 +5020 POINT(34.879102047254925 -118.34602958766337) bank5020 +5021 POINT(33.35677248993769 -119.11751224967935) bank5021 +5022 POINT(33.97577276712957 -117.8739124596051) bank5022 +5023 POINT(34.39820362766922 -117.68772238797642) bank5023 +5024 POINT(34.083385963511965 -119.08360957704683) bank5024 +5025 POINT(35.0181048603879 -118.03131140224376) bank5025 +5026 POINT(33.79264481147878 -118.45219922217419) bank5026 +5027 POINT(34.89220105729759 -117.93942605816906) bank5027 +5028 POINT(34.013207729012606 -117.66981707312914) bank5028 +5029 POINT(34.9535514370965 -117.9732968136089) bank5029 +5030 POINT(34.062668835931056 -117.37386756402998) bank5030 +5031 POINT(34.315737997888185 -117.35829856256234) bank5031 +5032 POINT(33.2843241569482 -117.34444118398152) bank5032 +5033 POINT(34.729292698678826 -118.6145368239666) bank5033 +5034 POINT(33.79834208362378 -117.5353692842726) bank5034 +5035 POINT(34.754066908035384 -117.31883773658903) bank5035 +5036 POINT(34.868163540955855 -118.38540868627585) bank5036 +5037 POINT(33.86374832867961 -119.06582881143548) bank5037 +5038 POINT(33.2278536441027 -118.38032132919346) bank5038 +5039 POINT(33.84098382136804 -117.49349107319186) bank5039 +5040 POINT(34.692932618376254 -118.22974854922865) bank5040 +5041 POINT(34.237009012707574 -118.72095972544551) bank5041 +5042 POINT(34.64646000444225 -118.42464599808898) bank5042 +5043 POINT(34.5287396208675 -117.32306880291682) bank5043 +5044 POINT(34.09975918072038 -117.87696119094757) bank5044 +5045 POINT(34.43884507207875 -119.12893018370266) bank5045 +5046 POINT(34.574471388522085 -118.8337787905538) bank5046 +5047 POINT(34.62703936727996 -118.1028255641957) bank5047 +5048 POINT(34.180770131655315 -117.56201386424976) bank5048 +5049 POINT(33.86076648622578 -119.0764955102326) bank5049 +5050 POINT(35.01848051866103 -119.04833980369253) bank5050 +5051 POINT(33.51679321145785 -119.18916509427548) bank5051 +5052 POINT(33.39227714594299 -119.02645281759429) bank5052 +5053 POINT(33.11225846509676 -118.23613736746617) bank5053 +5054 POINT(34.28589060085541 -118.84018375910506) bank5054 +5055 POINT(34.091204338932066 -117.93647301761513) bank5055 +5056 POINT(33.756774235249864 -118.74482100988006) bank5056 +5057 POINT(34.61214654490481 -119.10188157880789) bank5057 +5058 POINT(34.352876286095736 -117.97317498186302) bank5058 +5059 POINT(33.938205798111376 -117.91227241243618) bank5059 +5060 POINT(33.61496524732581 -117.38231917863381) bank5060 +5061 POINT(34.15462854419143 -117.99241742739807) bank5061 +5062 POINT(33.95056984662543 -118.51834694832378) bank5062 +5063 POINT(35.03486209516103 -117.65523351310651) bank5063 +5064 POINT(34.35799634941903 -117.70539050182767) bank5064 +5065 POINT(34.973409552919605 -119.12963514935883) bank5065 +5066 POINT(33.14036392061046 -117.66472602500362) bank5066 +5067 POINT(33.27998556513277 -118.67265720120241) bank5067 +5068 POINT(34.043068924059554 -117.99631342617593) bank5068 +5069 POINT(34.29698397542163 -119.06940439300779) bank5069 +5070 POINT(33.5978109831846 -118.72086328657909) bank5070 +5071 POINT(34.98578565318463 -117.76059873631763) bank5071 +5072 POINT(33.18836774562588 -117.39272797613965) bank5072 +5073 POINT(34.6240444174559 -118.31119948823951) bank5073 +5074 POINT(33.8401755408633 -117.4204922559845) bank5074 +5075 POINT(34.54207410625494 -119.24274752848018) bank5075 +5076 POINT(33.49561591504246 -119.11612309334687) bank5076 +5077 POINT(34.77197877063634 -118.3266593836798) bank5077 +5078 POINT(34.921989200678574 -117.73577532543524) bank5078 +5079 POINT(33.92163571541993 -117.81985558666423) bank5079 +5080 POINT(33.27080023780285 -117.4848824618443) bank5080 +5081 POINT(34.595593126305175 -117.27450092393894) bank5081 +5082 POINT(34.675023168653745 -118.14618180247007) bank5082 +5083 POINT(33.42285645926564 -118.3112233864084) bank5083 +5084 POINT(33.66169665984276 -117.72517033042003) bank5084 +5085 POINT(34.122672004949706 -118.31237503839998) bank5085 +5086 POINT(34.62504210582446 -117.82544328537793) bank5086 +5087 POINT(33.484107535005485 -119.22770633605609) bank5087 +5088 POINT(33.07647754741991 -117.48915011258113) bank5088 +5089 POINT(33.706807564355884 -118.30459754072933) bank5089 +5090 POINT(33.61624544190639 -118.65774444069433) bank5090 +5091 POINT(34.05914180152047 -118.75884034445268) bank5091 +5092 POINT(34.071033345126914 -118.42911907457443) bank5092 +5093 POINT(34.746422827730946 -117.63241205677053) bank5093 +5094 POINT(34.527326989197306 -117.3211907811305) bank5094 +5095 POINT(34.82913818074041 -119.09498878692725) bank5095 +5096 POINT(34.35479128191773 -118.147287323515) bank5096 +5097 POINT(33.09976705781464 -119.22566544158545) bank5097 +5098 POINT(34.948881793623755 -119.09321656896678) bank5098 +5099 POINT(33.1513718231561 -118.03385111127487) bank5099 +5100 POINT(34.43207507474508 -118.45882662898832) bank5100 +5101 POINT(33.251773185405284 -118.48573901595452) bank5101 +5102 POINT(33.683549401029175 -119.05216029025613) bank5102 +5103 POINT(34.29680306197064 -118.21855695108846) bank5103 +5104 POINT(35.00272464107433 -118.0152259636931) bank5104 +5105 POINT(33.653036007736986 -118.67339105072016) bank5105 +5106 POINT(34.943788870102445 -117.57564325925648) bank5106 +5107 POINT(34.80196829490653 -117.83248989758478) bank5107 +5108 POINT(33.20752795487536 -118.89740354864712) bank5108 +5109 POINT(34.218069285406564 -117.6369203239996) bank5109 +5110 POINT(33.70415571219014 -117.64406373288415) bank5110 +5111 POINT(33.84678630982321 -118.45757553858171) bank5111 +5112 POINT(34.90301534973977 -117.63197198527945) bank5112 +5113 POINT(33.556333402199854 -117.6852955364259) bank5113 +5114 POINT(33.521366397008514 -118.24421462905944) bank5114 +5115 POINT(33.14667194365467 -118.01962293773505) bank5115 +5116 POINT(33.96166291018572 -118.75219385694622) bank5116 +5117 POINT(34.92185696480939 -118.05967092206812) bank5117 +5118 POINT(34.514938817001436 -117.26627307187294) bank5118 +5119 POINT(35.026977220613986 -117.94261124182495) bank5119 +5120 POINT(33.49302197096501 -119.22641448002018) bank5120 +5121 POINT(34.13147397533218 -118.09144199159704) bank5121 +5122 POINT(34.66652989798239 -117.32734090114727) bank5122 +5123 POINT(34.628012157826454 -119.06236193533871) bank5123 +5124 POINT(34.63452568023547 -118.78850924330358) bank5124 +5125 POINT(34.36707186546469 -118.73276270935128) bank5125 +5126 POINT(33.34727736731786 -117.44053882247842) bank5126 +5127 POINT(33.76625388602165 -118.4149616693048) bank5127 +5128 POINT(33.82770806060073 -119.0798031669141) bank5128 +5129 POINT(34.018375089066296 -119.19009280672698) bank5129 +5130 POINT(34.02043328485257 -117.70823907382848) bank5130 +5131 POINT(35.035416915872304 -117.63124546710277) bank5131 +5132 POINT(33.200157169568854 -119.18400551149087) bank5132 +5133 POINT(34.23115400924143 -117.35068478690748) bank5133 +5134 POINT(34.498907320135466 -117.65708994436615) bank5134 +5135 POINT(33.63347357806241 -117.839546963269) bank5135 +5136 POINT(33.87401286298348 -118.76201007070985) bank5136 +5137 POINT(33.30923905742357 -117.75947234664345) bank5137 +5138 POINT(33.80536452089211 -117.55481340550543) bank5138 +5139 POINT(33.120919004836345 -117.99953591850054) bank5139 +5140 POINT(33.83855597230629 -117.49307601844924) bank5140 +5141 POINT(34.28377852492283 -118.67038861371238) bank5141 +5142 POINT(33.221602978056154 -119.0616135103502) bank5142 +5143 POINT(35.001168174942926 -117.94071944254365) bank5143 +5144 POINT(34.24379207840273 -118.35003790737137) bank5144 +5145 POINT(34.74650942860535 -118.19847408259167) bank5145 +5146 POINT(34.916395026167685 -118.68880467323812) bank5146 +5147 POINT(34.58539387726181 -117.25062161468276) bank5147 +5148 POINT(34.092001531008236 -117.84797521428719) bank5148 +5149 POINT(33.10595124021847 -119.07396733809024) bank5149 +5150 POINT(34.66906350080051 -119.1218585666131) bank5150 +5151 POINT(33.31231297931882 -118.2171737759707) bank5151 +5152 POINT(33.33334873895677 -118.63072214139598) bank5152 +5153 POINT(34.04084634860407 -117.95353102374416) bank5153 +5154 POINT(34.221137216978114 -118.21703482739537) bank5154 +5155 POINT(35.02306862468117 -118.82813372786923) bank5155 +5156 POINT(33.696559383919215 -119.08751464018665) bank5156 +5157 POINT(33.144292262432316 -119.08907865437683) bank5157 +5158 POINT(33.571180398693784 -119.17817407896509) bank5158 +5159 POINT(33.70867194544276 -118.7250677811547) bank5159 +5160 POINT(33.50666615224584 -118.18736330258776) bank5160 +5161 POINT(34.676363129268545 -118.15840485820992) bank5161 +5162 POINT(33.75418368907316 -119.00399269420434) bank5162 +5163 POINT(34.39820190265014 -118.22667009989091) bank5163 +5164 POINT(34.144976649175526 -117.66797804215933) bank5164 +5165 POINT(33.557056215642056 -119.0962641339616) bank5165 +5166 POINT(34.633542418011785 -119.10212200660767) bank5166 +5167 POINT(33.913308305553336 -119.18652113697686) bank5167 +5168 POINT(33.595571744181406 -118.17892974918821) bank5168 +5169 POINT(34.61601715388111 -118.4834111356368) bank5169 +5170 POINT(33.49147320603581 -118.5270831121352) bank5170 +5171 POINT(34.798873947002 -117.68653478592293) bank5171 +5172 POINT(34.32409876403084 -118.87175614969728) bank5172 +5173 POINT(34.841296802057414 -119.18582443934231) bank5173 +5174 POINT(34.627073324830796 -118.70794769341822) bank5174 +5175 POINT(33.801484029085245 -117.28597972200559) bank5175 +5176 POINT(35.04531339230733 -117.76801921476611) bank5176 +5177 POINT(34.430103593932905 -118.87097873111422) bank5177 +5178 POINT(34.25719173558302 -117.60598451973786) bank5178 +5179 POINT(34.696937488728885 -117.72087004702254) bank5179 +5180 POINT(34.446611831905706 -117.27353833378777) bank5180 +5181 POINT(34.42215669906391 -117.64089721548777) bank5181 +5182 POINT(33.15807120325806 -117.29875571848449) bank5182 +5183 POINT(34.50025003539125 -118.1923175470321) bank5183 +5184 POINT(34.140397013434075 -118.08604648939037) bank5184 +5185 POINT(33.94242119831859 -117.3716967431951) bank5185 +5186 POINT(33.49306556185562 -117.7542160057726) bank5186 +5187 POINT(33.12828442812984 -118.66217690240008) bank5187 +5188 POINT(33.176436234410275 -119.16941743921852) bank5188 +5189 POINT(34.6024905382547 -118.08881339785918) bank5189 +5190 POINT(34.03319471905078 -118.93998292487301) bank5190 +5191 POINT(33.47488111343438 -118.64945722140297) bank5191 +5192 POINT(34.62042558558368 -119.02235216111401) bank5192 +5193 POINT(34.6839643221612 -118.00584526359435) bank5193 +5194 POINT(35.01439761793638 -118.55081753707384) bank5194 +5195 POINT(33.97118907502625 -118.37827238836071) bank5195 +5196 POINT(33.63574723950511 -117.998518934777) bank5196 +5197 POINT(33.765067516054245 -118.5902557899596) bank5197 +5198 POINT(33.50182402982172 -117.76228361960078) bank5198 +5199 POINT(33.58018782780621 -118.83706521771343) bank5199 +5200 POINT(33.55553293442214 -118.95573886602952) bank5200 +5201 POINT(33.29366789370745 -117.95373550063705) bank5201 +5202 POINT(34.5704637317599 -117.7639162138151) bank5202 +5203 POINT(33.19393420018031 -118.50391716344991) bank5203 +5204 POINT(34.51928121780202 -118.59560545842609) bank5204 +5205 POINT(34.179853445821124 -117.28994442581325) bank5205 +5206 POINT(34.20552524355195 -118.74815873707931) bank5206 +5207 POINT(34.92586408174123 -117.85119273953849) bank5207 +5208 POINT(33.83675888496086 -119.02483678254796) bank5208 +5209 POINT(33.18467859469778 -118.11672884636266) bank5209 +5210 POINT(34.58802105336818 -117.75901673326672) bank5210 +5211 POINT(33.731174895681455 -118.38106604912501) bank5211 +5212 POINT(33.65462077922893 -118.63723920009667) bank5212 +5213 POINT(34.7305803318606 -119.20349183022272) bank5213 +5214 POINT(33.95748062462131 -118.69870747452049) bank5214 +5215 POINT(34.810485081124746 -117.45410130938178) bank5215 +5216 POINT(34.05757653758219 -118.13166277510976) bank5216 +5217 POINT(34.72553049298461 -117.33514273505848) bank5217 +5218 POINT(34.95794561879123 -119.17630670978082) bank5218 +5219 POINT(34.406000930501854 -118.7883121742812) bank5219 +5220 POINT(34.93769821977141 -119.14663393194) bank5220 +5221 POINT(33.48605242126087 -118.06785374039178) bank5221 +5222 POINT(34.30005056020112 -117.30027881263331) bank5222 +5223 POINT(33.18736443385497 -117.95035911200989) bank5223 +5224 POINT(33.680937243942616 -117.4777946562624) bank5224 +5225 POINT(33.09089263862038 -117.67039491462103) bank5225 +5226 POINT(34.098641434679294 -118.39975731194335) bank5226 +5227 POINT(34.91373374987844 -117.89403769337297) bank5227 +5228 POINT(34.60750304941209 -118.19223783236215) bank5228 +5229 POINT(33.28080880026091 -117.30618277799027) bank5229 +5230 POINT(34.7630571952054 -118.44199263370382) bank5230 +5231 POINT(35.03048857892917 -117.6979199441993) bank5231 +5232 POINT(33.30817697012235 -117.79623177189903) bank5232 +5233 POINT(34.06117750556595 -117.46007832542548) bank5233 +5234 POINT(34.20050305583919 -118.35592240097188) bank5234 +5235 POINT(34.676890028951966 -118.7731996225171) bank5235 +5236 POINT(33.217661600680536 -118.73947040109218) bank5236 +5237 POINT(33.505580391340736 -117.52194234423385) bank5237 +5238 POINT(33.31947412519561 -118.4521287561713) bank5238 +5239 POINT(33.98456687661649 -118.76014680473104) bank5239 +5240 POINT(34.03955661642222 -118.25178511263324) bank5240 +5241 POINT(34.56278053785473 -118.57887503135345) bank5241 +5242 POINT(33.43855268402445 -119.02785908841471) bank5242 +5243 POINT(34.705735571366674 -118.83383665760769) bank5243 +5244 POINT(34.5343219381528 -117.52772086826785) bank5244 +5245 POINT(34.24763459242765 -117.53089630972075) bank5245 +5246 POINT(33.46846266720996 -117.98533956205311) bank5246 +5247 POINT(34.76420785973855 -117.82282518057868) bank5247 +5248 POINT(34.439706915283914 -118.7653727993939) bank5248 +5249 POINT(34.792357934733346 -118.63584859412452) bank5249 +5250 POINT(34.37291941648218 -118.30659852765791) bank5250 +5251 POINT(33.09583798588517 -118.31995499707183) bank5251 +5252 POINT(33.130190978952434 -118.0692200668114) bank5252 +5253 POINT(34.210747373861324 -117.594560974899) bank5253 +5254 POINT(34.00528342621469 -117.92775881366897) bank5254 +5255 POINT(34.41380267776633 -117.30906877482988) bank5255 +5256 POINT(34.87354587846117 -118.40824516539935) bank5256 +5257 POINT(34.071039392697564 -117.4399703160733) bank5257 +5258 POINT(33.977408714409975 -118.12568351128039) bank5258 +5259 POINT(33.10187877507031 -119.03612496468146) bank5259 +5260 POINT(34.46610980189393 -117.89589077709356) bank5260 +5261 POINT(34.7436306954072 -119.01220302477161) bank5261 +5262 POINT(33.75883737758367 -118.2401839472433) bank5262 +5263 POINT(34.97586953184482 -119.18226638907286) bank5263 +5264 POINT(34.26014567709123 -117.57045112871323) bank5264 +5265 POINT(34.340505809999755 -117.57407292627896) bank5265 +5266 POINT(33.8201347934841 -117.77459867528918) bank5266 +5267 POINT(33.836447713514346 -118.24591994015893) bank5267 +5268 POINT(34.221411276661556 -118.0802837280428) bank5268 +5269 POINT(33.317415682086086 -118.30881941517096) bank5269 +5270 POINT(34.907828091410764 -117.41793822961598) bank5270 +5271 POINT(33.687054565724566 -117.91538132920029) bank5271 +5272 POINT(34.299164177522364 -117.56780005300527) bank5272 +5273 POINT(34.73839865324766 -118.88445678664655) bank5273 +5274 POINT(33.38569018439398 -118.83268890643151) bank5274 +5275 POINT(35.0031079185631 -118.29683800051244) bank5275 +5276 POINT(35.018160031360615 -118.51112777091451) bank5276 +5277 POINT(34.19740802161361 -118.22163520500347) bank5277 +5278 POINT(34.179598119802435 -118.91481478707347) bank5278 +5279 POINT(34.19697935877741 -118.62400660695896) bank5279 +5280 POINT(34.46281345751472 -118.74420152446498) bank5280 +5281 POINT(33.998199463225916 -118.78500540249574) bank5281 +5282 POINT(34.12554051322062 -119.19171490533317) bank5282 +5283 POINT(34.031953895532354 -117.45531138098029) bank5283 +5284 POINT(34.26264722379921 -118.04621619489895) bank5284 +5285 POINT(34.99202501574575 -118.07273992047976) bank5285 +5286 POINT(34.84012410448985 -118.85312893440107) bank5286 +5287 POINT(34.46750125843254 -118.5406408361354) bank5287 +5288 POINT(33.79239035000107 -118.13837125802273) bank5288 +5289 POINT(34.55418969537116 -118.52652593150472) bank5289 +5290 POINT(34.27845811872855 -118.03546996976414) bank5290 +5291 POINT(34.18011428313645 -117.7038957950199) bank5291 +5292 POINT(33.83818986985629 -118.67563568521585) bank5292 +5293 POINT(33.10284433986426 -117.40939826271936) bank5293 +5294 POINT(34.161652598747864 -118.73931354500246) bank5294 +5295 POINT(34.70940438628752 -118.72054999994566) bank5295 +5296 POINT(33.268611939612924 -118.92402731747681) bank5296 +5297 POINT(33.81027468425904 -118.58787005436807) bank5297 +5298 POINT(33.39329485211662 -118.1578278868974) bank5298 +5299 POINT(34.68598028921881 -118.50993727292366) bank5299 +5300 POINT(34.86649417310083 -118.01487307906753) bank5300 +5301 POINT(33.17932792713916 -118.1312742089126) bank5301 +5302 POINT(34.05310915370463 -117.35211201574901) bank5302 +5303 POINT(33.18504915612432 -118.84355851646939) bank5303 +5304 POINT(33.61418866354913 -118.76662624095543) bank5304 +5305 POINT(34.86805698678883 -118.38106442414423) bank5305 +5306 POINT(34.26009800888612 -118.78812984237727) bank5306 +5307 POINT(33.70897623808377 -117.8186998094506) bank5307 +5308 POINT(34.20222051131432 -118.40422952871) bank5308 +5309 POINT(33.44821899994309 -117.35133090786529) bank5309 +5310 POINT(34.20543919110522 -118.48989575689747) bank5310 +5311 POINT(33.31785801924175 -118.53951817822808) bank5311 +5312 POINT(33.43505132232484 -118.5943587196385) bank5312 +5313 POINT(34.70670973275524 -119.11393503185839) bank5313 +5314 POINT(33.11732533256324 -118.8848915781294) bank5314 +5315 POINT(34.03287389982582 -118.15789562107696) bank5315 +5316 POINT(34.974295122147915 -117.31566490389818) bank5316 +5317 POINT(33.617046672835315 -117.86367253325993) bank5317 +5318 POINT(33.13774554396037 -118.0707265557973) bank5318 +5319 POINT(34.274613651463156 -117.58755128054217) bank5319 +5320 POINT(33.90010209971701 -118.41182624650048) bank5320 +5321 POINT(34.54232519222882 -118.04881388696282) bank5321 +5322 POINT(34.07567294162727 -118.14062525508749) bank5322 +5323 POINT(33.88711710200798 -118.26821786085979) bank5323 +5324 POINT(33.128978702167366 -118.39343046907484) bank5324 +5325 POINT(33.63445826799935 -117.98908633839704) bank5325 +5326 POINT(33.93857198106422 -117.28264009555133) bank5326 +5327 POINT(33.38222778577828 -118.93968939199046) bank5327 +5328 POINT(34.405437316875606 -117.98768565301157) bank5328 +5329 POINT(34.66173162296936 -118.6289141456518) bank5329 +5330 POINT(33.85064458268877 -118.51372132203369) bank5330 +5331 POINT(34.79798059333275 -118.56151325531685) bank5331 +5332 POINT(33.06607775079401 -119.14923460772127) bank5332 +5333 POINT(33.731857482170106 -117.28324544822165) bank5333 +5334 POINT(34.0780636771293 -118.29608083087238) bank5334 +5335 POINT(33.08873136211292 -117.85657834484638) bank5335 +5336 POINT(34.93981877701763 -118.92308365750242) bank5336 +5337 POINT(33.39866497712592 -118.76873877678314) bank5337 +5338 POINT(34.93693513206808 -117.73469433709263) bank5338 +5339 POINT(33.39295094161006 -119.18671622327923) bank5339 +5340 POINT(33.147197720296056 -119.13635273487985) bank5340 +5341 POINT(34.16350071945609 -117.46072053302173) bank5341 +5342 POINT(34.66714864114554 -118.78882183874427) bank5342 +5343 POINT(34.53613012309589 -118.9283806910737) bank5343 +5344 POINT(34.532244543725334 -117.99591497526477) bank5344 +5345 POINT(33.92628138000051 -118.4561332505466) bank5345 +5346 POINT(33.43355229726724 -117.43990424028614) bank5346 +5347 POINT(33.43179192917593 -118.27139492286483) bank5347 +5348 POINT(34.165839887767326 -118.36206161462857) bank5348 +5349 POINT(33.062050854662104 -118.37569041670628) bank5349 +5350 POINT(34.59986672807824 -118.26230053605765) bank5350 +5351 POINT(34.00000022070241 -118.68805131093796) bank5351 +5352 POINT(34.230728415525135 -119.12704851174635) bank5352 +5353 POINT(34.94722660240617 -118.66528915583343) bank5353 +5354 POINT(33.82362261747157 -117.78883899291039) bank5354 +5355 POINT(34.74709417542274 -117.70202443214936) bank5355 +5356 POINT(33.12626593833842 -117.54224670637795) bank5356 +5357 POINT(33.55577934078978 -117.57044925333166) bank5357 +5358 POINT(34.62988958269648 -117.80480149781873) bank5358 +5359 POINT(34.8713041931519 -118.57365949305104) bank5359 +5360 POINT(34.3184327538876 -118.94870376606593) bank5360 +5361 POINT(34.68190031643544 -117.60194274879694) bank5361 +5362 POINT(33.72335405468376 -118.51615705132788) bank5362 +5363 POINT(33.535809192503244 -118.06470827726832) bank5363 +5364 POINT(33.802798808641704 -118.61793340863224) bank5364 +5365 POINT(34.334336196106605 -118.49578371586809) bank5365 +5366 POINT(34.51273515854851 -119.21229027678227) bank5366 +5367 POINT(33.3475344159054 -118.34993249159864) bank5367 +5368 POINT(33.590993672949416 -118.17689014722343) bank5368 +5369 POINT(34.939969437138814 -117.6734283830813) bank5369 +5370 POINT(33.88837440163437 -118.06123578525657) bank5370 +5371 POINT(33.146239272222346 -118.94779824606557) bank5371 +5372 POINT(33.66760010744572 -117.25306365338703) bank5372 +5373 POINT(33.54184059192722 -117.5727322767281) bank5373 +5374 POINT(34.66569102668536 -118.35475213197185) bank5374 +5375 POINT(34.209196231382606 -118.56617806152336) bank5375 +5376 POINT(34.488477273252755 -118.78907507621987) bank5376 +5377 POINT(33.682677492076394 -117.46119722434392) bank5377 +5378 POINT(34.58087071967584 -117.62582735985899) bank5378 +5379 POINT(33.47401515331541 -118.19114199520637) bank5379 +5380 POINT(33.306676151172724 -118.68856402420239) bank5380 +5381 POINT(33.12539248229035 -118.05669341786162) bank5381 +5382 POINT(34.21930193329059 -117.24873224920826) bank5382 +5383 POINT(34.35003686804005 -117.84234721108609) bank5383 +5384 POINT(33.07933980433367 -118.07144953812018) bank5384 +5385 POINT(33.915082019528874 -117.62999121762604) bank5385 +5386 POINT(34.58316361086843 -118.82240626559977) bank5386 +5387 POINT(33.40946064169365 -118.66784455482563) bank5387 +5388 POINT(34.6597811217286 -119.14646995789367) bank5388 +5389 POINT(34.39738642856626 -118.24057951933857) bank5389 +5390 POINT(34.67079841594324 -118.46319290602871) bank5390 +5391 POINT(34.53032536066474 -118.43657445207243) bank5391 +5392 POINT(34.71986893339757 -118.36857087599958) bank5392 +5393 POINT(34.78664542318454 -117.66561276821422) bank5393 +5394 POINT(34.094967321517714 -118.83853436414428) bank5394 +5395 POINT(33.53408396706885 -117.49108050905504) bank5395 +5396 POINT(34.46235917415546 -117.75318636518433) bank5396 +5397 POINT(33.904133675735714 -118.11801946286016) bank5397 +5398 POINT(34.62715517837223 -119.23751266176613) bank5398 +5399 POINT(33.46844282557858 -117.5582794598803) bank5399 +5400 POINT(34.886673216362034 -117.37600584128295) bank5400 +5401 POINT(33.88132959569676 -119.24041533301705) bank5401 +5402 POINT(33.40937551576996 -117.76462819930097) bank5402 +5403 POINT(33.31519978050509 -117.98864829702099) bank5403 +5404 POINT(34.977372447373966 -117.92490018193057) bank5404 +5405 POINT(34.13825094947444 -117.58058909449738) bank5405 +5406 POINT(33.75646069984911 -117.3804923872565) bank5406 +5407 POINT(34.649279888644045 -118.65423214918471) bank5407 +5408 POINT(34.58279505761095 -119.1224507855378) bank5408 +5409 POINT(34.28679715460307 -119.17026005848975) bank5409 +5410 POINT(34.54053412743307 -119.17161794021614) bank5410 +5411 POINT(34.85391379134655 -119.1117097111527) bank5411 +5412 POINT(34.45621956608296 -118.94214153724643) bank5412 +5413 POINT(33.8671739230822 -118.33625521823366) bank5413 +5414 POINT(33.453385828108246 -118.81007832492506) bank5414 +5415 POINT(34.01341847227085 -118.92835994953049) bank5415 +5416 POINT(34.24184049534992 -118.42943929420701) bank5416 +5417 POINT(34.663903718269964 -118.68535794587821) bank5417 +5418 POINT(34.00200325320955 -119.01418732060917) bank5418 +5419 POINT(34.12907515747533 -117.82247240808078) bank5419 +5420 POINT(33.315454446559336 -117.68344231611958) bank5420 +5421 POINT(35.02772088888597 -119.06153341067711) bank5421 +5422 POINT(33.36378992588817 -119.1926511754545) bank5422 +5423 POINT(34.30077633805836 -118.86138143435053) bank5423 +5424 POINT(34.96498942586898 -118.06284854799686) bank5424 +5425 POINT(33.730585095415314 -117.58773071542186) bank5425 +5426 POINT(33.300767413330725 -117.92471955246896) bank5426 +5427 POINT(33.413940796027134 -117.62537692716928) bank5427 +5428 POINT(33.2309130664647 -117.87083732787252) bank5428 +5429 POINT(33.550837025565976 -119.20011138626512) bank5429 +5430 POINT(34.38131830110626 -118.04172461556911) bank5430 +5431 POINT(33.121780918788616 -118.61125231765894) bank5431 +5432 POINT(33.70260613017543 -118.21882997237671) bank5432 +5433 POINT(33.501748823004604 -119.00412700023203) bank5433 +5434 POINT(33.81539214116306 -119.20513173746329) bank5434 +5435 POINT(34.55256174950587 -118.74499911607504) bank5435 +5436 POINT(33.29050471837633 -118.61107889457344) bank5436 +5437 POINT(34.94719517020943 -118.09443683701632) bank5437 +5438 POINT(35.05168515609983 -118.46937194717778) bank5438 +5439 POINT(33.33094004165727 -118.64329623034773) bank5439 +5440 POINT(33.314439763450395 -118.22137428856473) bank5440 +5441 POINT(34.23092111215102 -118.06638624971085) bank5441 +5442 POINT(33.081828493592894 -118.37009262645935) bank5442 +5443 POINT(33.177297149857026 -119.01220961292105) bank5443 +5444 POINT(34.94937999570418 -117.24595788134668) bank5444 +5445 POINT(34.8269282272856 -118.56800084370356) bank5445 +5446 POINT(34.6565116262945 -118.48618898544638) bank5446 +5447 POINT(33.52212082826986 -119.01188372833853) bank5447 +5448 POINT(34.214671949481875 -117.27076106395974) bank5448 +5449 POINT(34.97887286728667 -118.67738874503416) bank5449 +5450 POINT(34.576418576614216 -117.40822251853302) bank5450 +5451 POINT(33.237592943994706 -119.06395175424197) bank5451 +5452 POINT(34.814675848072746 -118.02008808825647) bank5452 +5453 POINT(34.830539220201395 -118.42794619908823) bank5453 +5454 POINT(33.87826263647786 -118.82190041642234) bank5454 +5455 POINT(33.7862415013247 -118.65904872427001) bank5455 +5456 POINT(33.549333032231374 -118.81779316721399) bank5456 +5457 POINT(33.94129450775951 -118.81897094928473) bank5457 +5458 POINT(33.65059302856803 -118.19609446245363) bank5458 +5459 POINT(34.928008947982725 -117.92211225207096) bank5459 +5460 POINT(33.232785412464196 -117.90676857633272) bank5460 +5461 POINT(35.02367298761594 -118.9862641103724) bank5461 +5462 POINT(34.11699194585374 -118.94218465669448) bank5462 +5463 POINT(34.71962114352072 -118.03972774867152) bank5463 +5464 POINT(33.138829159019444 -117.89796549536896) bank5464 +5465 POINT(34.77632291468998 -118.28470537087071) bank5465 +5466 POINT(34.501783555523815 -119.11451657196704) bank5466 +5467 POINT(34.782186568252094 -118.32408547878451) bank5467 +5468 POINT(34.01007807879776 -117.26137267581176) bank5468 +5469 POINT(33.571384856812436 -118.9652623759241) bank5469 +5470 POINT(34.08226449877884 -119.19208388688635) bank5470 +5471 POINT(34.11929768402772 -117.47853243394896) bank5471 +5472 POINT(33.60350435858553 -117.84006401189805) bank5472 +5473 POINT(33.166404713973456 -117.96540893876474) bank5473 +5474 POINT(33.95769658757996 -118.39043249658967) bank5474 +5475 POINT(34.66243978955088 -118.07946843761822) bank5475 +5476 POINT(34.05804625513254 -117.55719079144612) bank5476 +5477 POINT(33.86184040090113 -117.3642300572096) bank5477 +5478 POINT(34.04642456295385 -118.6531302224374) bank5478 +5479 POINT(33.23031097780114 -117.9718077256704) bank5479 +5480 POINT(33.91419772011655 -117.51628572116738) bank5480 +5481 POINT(33.640211722786944 -119.00654983585258) bank5481 +5482 POINT(33.79469279338596 -117.26243861664553) bank5482 +5483 POINT(33.24997114369601 -117.93292812228609) bank5483 +5484 POINT(34.62444486746007 -117.76156441391424) bank5484 +5485 POINT(34.05743164351342 -119.013921217617) bank5485 +5486 POINT(33.82344748631651 -117.76280141179821) bank5486 +5487 POINT(33.250750873959305 -117.44387839930735) bank5487 +5488 POINT(34.67062959085669 -117.55265434481211) bank5488 +5489 POINT(34.557218838960765 -117.85821798334173) bank5489 +5490 POINT(33.509829413156794 -118.89557465901683) bank5490 +5491 POINT(33.7450839521204 -118.83750762056596) bank5491 +5492 POINT(34.3235617213634 -118.95431853716676) bank5492 +5493 POINT(33.91688984884358 -118.01881585075931) bank5493 +5494 POINT(34.49301686677114 -119.02368366590014) bank5494 +5495 POINT(33.27484319783335 -117.38773519344961) bank5495 +5496 POINT(34.84956183027638 -117.723691069326) bank5496 +5497 POINT(34.36216373390647 -118.2356135850094) bank5497 +5498 POINT(34.593042069846405 -117.48087216693584) bank5498 +5499 POINT(33.37785923478354 -117.53725133968645) bank5499 +5500 POINT(33.22940751969726 -119.20352168664695) bank5500 +5501 POINT(34.36566270864714 -118.85651254518656) bank5501 +5502 POINT(34.41279953899239 -118.77520315670284) bank5502 +5503 POINT(33.993490125778024 -117.7700440549322) bank5503 +5504 POINT(34.59137774651473 -118.59212865918603) bank5504 +5505 POINT(33.6614976714746 -118.33254166410991) bank5505 +5506 POINT(33.83778891392483 -118.8993863156169) bank5506 +5507 POINT(34.09498821200416 -118.14560008013096) bank5507 +5508 POINT(33.28195683044434 -117.53743504007763) bank5508 +5509 POINT(33.323659356058734 -117.66917295035839) bank5509 +5510 POINT(33.327637405746025 -117.35508133460425) bank5510 +5511 POINT(34.58655225509133 -117.97793048264874) bank5511 +5512 POINT(34.5027888027105 -117.36148071037407) bank5512 +5513 POINT(33.223183240538866 -118.56300012832943) bank5513 +5514 POINT(33.370796205204556 -118.33673390256409) bank5514 +5515 POINT(33.74020307546118 -117.93158040210905) bank5515 +5516 POINT(35.00607633307269 -117.64345792301563) bank5516 +5517 POINT(33.242019710548355 -118.70545487009774) bank5517 +5518 POINT(33.88012487407396 -117.61834364666616) bank5518 +5519 POINT(34.813656740561576 -119.05669970345382) bank5519 +5520 POINT(34.46365213487713 -117.98789954352286) bank5520 +5521 POINT(33.8493509746786 -119.17864362762757) bank5521 +5522 POINT(33.30548153496914 -119.237013479927) bank5522 +5523 POINT(33.47431154653319 -117.79738163997176) bank5523 +5524 POINT(33.11851725264104 -117.77080661804419) bank5524 +5525 POINT(34.06882824536764 -118.787805341078) bank5525 +5526 POINT(34.70812233407259 -118.62351816480471) bank5526 +5527 POINT(34.19843830716769 -118.0502142385138) bank5527 +5528 POINT(33.61375912568222 -119.03913916821836) bank5528 +5529 POINT(34.68579162012746 -117.79179658291943) bank5529 +5530 POINT(34.25043733166773 -118.63869821580411) bank5530 +5531 POINT(34.92760109007265 -117.50509515692434) bank5531 +5532 POINT(33.35756891733344 -117.86326479953367) bank5532 +5533 POINT(33.13020269020711 -117.35258591598762) bank5533 +5534 POINT(34.8791962694118 -118.9912761116198) bank5534 +5535 POINT(33.82707626208893 -117.44278775622021) bank5535 +5536 POINT(34.880537963018924 -118.08632138940503) bank5536 +5537 POINT(33.661538652109826 -117.71946436220092) bank5537 +5538 POINT(34.11170575374463 -119.22870352922011) bank5538 +5539 POINT(34.513988848085525 -117.89838633021745) bank5539 +5540 POINT(34.34463724739615 -117.6498693646768) bank5540 +5541 POINT(34.838495854871056 -118.34607648476398) bank5541 +5542 POINT(34.210179058337076 -117.92008950014551) bank5542 +5543 POINT(34.12095456234284 -119.1174790583129) bank5543 +5544 POINT(34.94335801431494 -117.45780668286014) bank5544 +5545 POINT(33.709722661961784 -117.38246952698653) bank5545 +5546 POINT(35.010280291556704 -117.99280550278884) bank5546 +5547 POINT(33.90933478665903 -117.56053990976737) bank5547 +5548 POINT(34.942055703382024 -119.21465803581816) bank5548 +5549 POINT(35.00806366773143 -118.4951388180528) bank5549 +5550 POINT(34.55929851237474 -117.35117428124676) bank5550 +5551 POINT(34.38478195067881 -117.68267778940509) bank5551 +5552 POINT(34.17727682661758 -117.83738813761605) bank5552 +5553 POINT(34.78749861183117 -117.67843372581996) bank5553 +5554 POINT(33.08626566368721 -118.7230940824063) bank5554 +5555 POINT(33.47909642932759 -118.17921793729082) bank5555 +5556 POINT(33.0913015874122 -118.09620484495836) bank5556 +5557 POINT(34.20025833084396 -117.29491307645766) bank5557 +5558 POINT(34.34687725532997 -117.82346248765428) bank5558 +5559 POINT(33.906230307376966 -117.53352778322316) bank5559 +5560 POINT(33.603155548732374 -119.14297386980653) bank5560 +5561 POINT(33.48555622572 -118.11322375456685) bank5561 +5562 POINT(34.807126862025676 -118.29617704491437) bank5562 +5563 POINT(33.271428552933216 -117.9220846933013) bank5563 +5564 POINT(34.59551116305444 -119.17489053988068) bank5564 +5565 POINT(33.96958318652887 -118.48271827635863) bank5565 +5566 POINT(33.53869111910913 -117.4722310442304) bank5566 +5567 POINT(34.00886170404818 -117.33372735568933) bank5567 +5568 POINT(34.877898400689844 -118.97016276608353) bank5568 +5569 POINT(34.40104234804585 -117.26859047400767) bank5569 +5570 POINT(34.418051642110996 -117.90585962685972) bank5570 +5571 POINT(34.499591166437874 -118.4115387597915) bank5571 +5572 POINT(35.030593198774696 -119.14579704688893) bank5572 +5573 POINT(34.1412482178233 -117.95491035100828) bank5573 +5574 POINT(34.7365603182559 -119.06719411719823) bank5574 +5575 POINT(34.60779001945127 -117.38790445821327) bank5575 +5576 POINT(34.33224861273659 -118.103097544443) bank5576 +5577 POINT(33.28047840283409 -119.20282826994395) bank5577 +5578 POINT(34.410465031473144 -118.07407421526476) bank5578 +5579 POINT(34.23817364373026 -118.27383982536158) bank5579 +5580 POINT(33.5286978336164 -119.2010536958706) bank5580 +5581 POINT(34.955973156540416 -117.60041665590548) bank5581 +5582 POINT(33.779044245907016 -118.11595715168681) bank5582 +5583 POINT(33.73864965690184 -119.19332517139541) bank5583 +5584 POINT(33.86680990017182 -118.04072050305177) bank5584 +5585 POINT(33.18513702360667 -119.24252920595357) bank5585 +5586 POINT(33.89242932493709 -118.4489889144262) bank5586 +5587 POINT(34.46574262419514 -117.50223722053641) bank5587 +5588 POINT(34.69813781816943 -118.20454525841501) bank5588 +5589 POINT(34.69233286537172 -118.60701048742808) bank5589 +5590 POINT(33.6516493912511 -117.3314951462403) bank5590 +5591 POINT(34.27434919247549 -118.41665852937261) bank5591 +5592 POINT(34.03198617779851 -118.31244242514775) bank5592 +5593 POINT(34.410008441642745 -117.64803896332126) bank5593 +5594 POINT(34.99515460985101 -119.14714178662668) bank5594 +5595 POINT(33.46125853830863 -118.56915489878149) bank5595 +5596 POINT(33.69519106713854 -119.06145490922239) bank5596 +5597 POINT(34.08857072472124 -118.1343179628124) bank5597 +5598 POINT(34.45506372223968 -117.54446420256066) bank5598 +5599 POINT(33.992769735027586 -118.51571830254306) bank5599 +5600 POINT(34.996942975286295 -118.42305922231394) bank5600 +5601 POINT(34.84749161629566 -119.15255052692574) bank5601 +5602 POINT(33.99219674073625 -118.51925738972965) bank5602 +5603 POINT(33.21018215651752 -117.64074236260294) bank5603 +5604 POINT(33.76719214660215 -119.21701185776749) bank5604 +5605 POINT(33.86023908121756 -118.98724709370401) bank5605 +5606 POINT(33.96251836550477 -119.11027203771535) bank5606 +5607 POINT(33.71422624024788 -118.0653875952463) bank5607 +5608 POINT(33.991407463027834 -119.20678331765154) bank5608 +5609 POINT(33.459645318147594 -118.11710404173601) bank5609 +5610 POINT(33.89781618010176 -118.50256764892735) bank5610 +5611 POINT(33.73529517173472 -118.98767090583124) bank5611 +5612 POINT(34.38401779535763 -118.87818957542758) bank5612 +5613 POINT(34.05278082679655 -119.1755589500433) bank5613 +5614 POINT(34.60662329652632 -117.51452523085803) bank5614 +5615 POINT(34.02123818857569 -117.56503581895325) bank5615 +5616 POINT(34.26248020845773 -118.07392841518988) bank5616 +5617 POINT(34.76865091258808 -117.61373116186762) bank5617 +5618 POINT(34.21716469139314 -118.6276677914736) bank5618 +5619 POINT(34.17989776461189 -117.3341548013862) bank5619 +5620 POINT(33.40312344333069 -117.69056265426457) bank5620 +5621 POINT(33.872287807518944 -118.20401898335442) bank5621 +5622 POINT(34.53438026337527 -118.1267610668875) bank5622 +5623 POINT(33.94447618087661 -118.81737366831257) bank5623 +5624 POINT(33.180086324124574 -117.67657489891575) bank5624 +5625 POINT(33.26882395775265 -117.27001489116488) bank5625 +5626 POINT(34.99960581605407 -117.59340520694909) bank5626 +5627 POINT(33.83167249871298 -117.49685370232369) bank5627 +5628 POINT(33.190180405774804 -118.72183039198069) bank5628 +5629 POINT(34.564975587153874 -117.35787860793175) bank5629 +5630 POINT(33.79620769539115 -118.66836395830599) bank5630 +5631 POINT(34.203933368979165 -118.56633997888049) bank5631 +5632 POINT(34.92786653938988 -118.95336233324102) bank5632 +5633 POINT(34.75814810418008 -118.84794803357232) bank5633 +5634 POINT(33.57925551455103 -117.98882986755339) bank5634 +5635 POINT(35.02599635776981 -117.66884431125447) bank5635 +5636 POINT(34.068252714770296 -118.01325689945583) bank5636 +5637 POINT(33.992562407952704 -119.20996497051955) bank5637 +5638 POINT(33.27773149928876 -118.99320518264096) bank5638 +5639 POINT(34.018180307786224 -117.37294982769056) bank5639 +5640 POINT(34.83165645949862 -119.00734539221261) bank5640 +5641 POINT(33.60290635606806 -117.67491175690161) bank5641 +5642 POINT(33.656149710120495 -119.13387124004137) bank5642 +5643 POINT(33.812453555694255 -117.45535500232191) bank5643 +5644 POINT(34.15256840651239 -118.79766673839218) bank5644 +5645 POINT(33.835608871095594 -118.82009369208288) bank5645 +5646 POINT(34.223453691028006 -118.1878286769703) bank5646 +5647 POINT(33.98888022685872 -117.74526025717141) bank5647 +5648 POINT(33.14597529454343 -117.83100378423231) bank5648 +5649 POINT(34.15498831642982 -118.86342011092265) bank5649 +5650 POINT(33.946910267256484 -119.06995154269839) bank5650 +5651 POINT(34.05502825422418 -118.1387772786343) bank5651 +5652 POINT(33.66214747614929 -117.30176929770334) bank5652 +5653 POINT(34.39408087957542 -117.54076697671616) bank5653 +5654 POINT(34.84558586107529 -118.00212782722684) bank5654 +5655 POINT(33.44963043152113 -119.16409250622308) bank5655 +5656 POINT(33.16291753813951 -118.30457038474353) bank5656 +5657 POINT(33.9937354157681 -117.31882940097498) bank5657 +5658 POINT(33.63221382119909 -117.55825816907526) bank5658 +5659 POINT(34.143465332188924 -118.53370070860487) bank5659 +5660 POINT(33.81144190959906 -117.71611936560076) bank5660 +5661 POINT(34.02835333735886 -118.4624333617076) bank5661 +5662 POINT(33.41332762413589 -117.58777587289694) bank5662 +5663 POINT(33.84850427474338 -118.1559924628007) bank5663 +5664 POINT(34.15292600971074 -118.5151016804642) bank5664 +5665 POINT(34.556327775969535 -118.98207958101189) bank5665 +5666 POINT(33.3728808088146 -117.75665124278935) bank5666 +5667 POINT(34.2078491207641 -117.47043687253066) bank5667 +5668 POINT(33.5942903563748 -117.41074105401374) bank5668 +5669 POINT(33.869982524529995 -117.41700094765137) bank5669 +5670 POINT(34.75184330510909 -117.410996905786) bank5670 +5671 POINT(33.839739911376114 -118.17483429789715) bank5671 +5672 POINT(34.205177996743345 -117.95326627843434) bank5672 +5673 POINT(33.600733560744 -118.77617974007573) bank5673 +5674 POINT(33.52590413430719 -119.05251123342828) bank5674 +5675 POINT(33.312568598302946 -118.74719469749817) bank5675 +5676 POINT(33.568556601031794 -118.03478371271166) bank5676 +5677 POINT(33.23737248402483 -118.28061411611769) bank5677 +5678 POINT(34.09012690867677 -118.06924624306903) bank5678 +5679 POINT(34.99594135933666 -117.66808842639496) bank5679 +5680 POINT(33.94825162609682 -118.12705269893938) bank5680 +5681 POINT(34.329150671527096 -117.7330553262438) bank5681 +5682 POINT(33.48419842810124 -118.8142138081551) bank5682 +5683 POINT(34.80475523332793 -117.68696940858004) bank5683 +5684 POINT(34.40020045326761 -118.1935773448915) bank5684 +5685 POINT(33.82551904723051 -117.62787687859965) bank5685 +5686 POINT(34.078746970318434 -118.2379482791796) bank5686 +5687 POINT(34.751574652406795 -117.80046486596157) bank5687 +5688 POINT(34.40038537120186 -117.34661827049642) bank5688 +5689 POINT(34.9150518532917 -119.24286558279842) bank5689 +5690 POINT(34.45120466871401 -118.71572496260487) bank5690 +5691 POINT(34.46409464772489 -118.58372294743572) bank5691 +5692 POINT(34.2362957573391 -118.40222121391747) bank5692 +5693 POINT(34.552770385749945 -118.43276908326442) bank5693 +5694 POINT(33.44583083564983 -118.21933380710233) bank5694 +5695 POINT(33.52568913255029 -118.87544685019667) bank5695 +5696 POINT(33.441843071008165 -119.2094764083255) bank5696 +5697 POINT(34.357391784204886 -117.29502642946693) bank5697 +5698 POINT(33.89150096746631 -117.26677565005258) bank5698 +5699 POINT(34.70404912128458 -119.17203314052041) bank5699 +5700 POINT(33.440760754558504 -118.9239958467588) bank5700 +5701 POINT(33.14792227288675 -118.99251985879552) bank5701 +5702 POINT(33.533847264795 -118.14789501068145) bank5702 +5703 POINT(33.43133277956425 -117.38354917255181) bank5703 +5704 POINT(33.81939210068117 -118.2060741545441) bank5704 +5705 POINT(35.01958268159515 -117.51981363127003) bank5705 +5706 POINT(34.394655723166046 -117.73748489768278) bank5706 +5707 POINT(33.62658019696866 -118.96255450257331) bank5707 +5708 POINT(34.93950497804491 -117.96991358246832) bank5708 +5709 POINT(33.32420080556298 -117.88274833213703) bank5709 +5710 POINT(33.93589520421268 -118.56991206096146) bank5710 +5711 POINT(33.33711699000178 -118.5619708390565) bank5711 +5712 POINT(34.73525810809634 -117.87555544477657) bank5712 +5713 POINT(33.83135805665772 -118.6163922150206) bank5713 +5714 POINT(33.06314122901787 -117.92034587538787) bank5714 +5715 POINT(33.18420970576709 -118.94294585990181) bank5715 +5716 POINT(34.74973815103841 -117.6488584725617) bank5716 +5717 POINT(33.4226215749902 -118.06699832667539) bank5717 +5718 POINT(33.538580650781796 -119.04669796920483) bank5718 +5719 POINT(34.15019745084893 -118.52346389802725) bank5719 +5720 POINT(33.09282634174605 -118.64107445333681) bank5720 +5721 POINT(34.38836530083846 -117.81433585508881) bank5721 +5722 POINT(34.14431416558229 -118.24186935968349) bank5722 +5723 POINT(33.3175091120545 -118.84762757526273) bank5723 +5724 POINT(33.347653631522434 -118.1301328322303) bank5724 +5725 POINT(33.593790839451536 -117.894792482863) bank5725 +5726 POINT(33.0845476040936 -119.23345608943175) bank5726 +5727 POINT(33.61705166150155 -117.57872754932039) bank5727 +5728 POINT(33.65492388859136 -118.67238914979866) bank5728 +5729 POINT(34.23952063743266 -119.22388284652645) bank5729 +5730 POINT(34.75450067001216 -118.23764570715664) bank5730 +5731 POINT(34.21488104441617 -119.07232997470017) bank5731 +5732 POINT(34.76005464974119 -118.83333990843965) bank5732 +5733 POINT(34.072789329610174 -117.97661275833046) bank5733 +5734 POINT(33.11841470314419 -119.22491066817729) bank5734 +5735 POINT(34.451643079378435 -118.30198993395666) bank5735 +5736 POINT(33.460669494957 -119.16902110008996) bank5736 +5737 POINT(34.72187775541918 -119.18354892570515) bank5737 +5738 POINT(35.01236166501637 -117.80635026438826) bank5738 +5739 POINT(33.935882257941834 -118.4548594355387) bank5739 +5740 POINT(33.38185254780474 -118.93934732912629) bank5740 +5741 POINT(34.789183247345456 -118.72330090180972) bank5741 +5742 POINT(33.79021482218532 -118.35986975632042) bank5742 +5743 POINT(33.32391916491441 -117.74412803147621) bank5743 +5744 POINT(34.831994516090845 -117.62828915367685) bank5744 +5745 POINT(34.31279190417001 -119.01340809724586) bank5745 +5746 POINT(33.60249086274393 -117.80972538652705) bank5746 +5747 POINT(33.31366901186925 -118.11498315892656) bank5747 +5748 POINT(33.845789738903804 -118.0087808186501) bank5748 +5749 POINT(34.1102299008401 -117.84751986319887) bank5749 +5750 POINT(34.13466262419501 -117.65732736630785) bank5750 +5751 POINT(33.562923145300886 -117.37416859331547) bank5751 +5752 POINT(34.81153810778598 -117.6170056193873) bank5752 +5753 POINT(33.83162802106971 -117.25268627803749) bank5753 +5754 POINT(33.84750679740767 -118.4037025353038) bank5754 +5755 POINT(33.68971139346262 -117.79839628526629) bank5755 +5756 POINT(34.32597734546501 -118.9928861246723) bank5756 +5757 POINT(33.98719275637278 -119.16388203115112) bank5757 +5758 POINT(34.65401225003979 -117.29536582695991) bank5758 +5759 POINT(34.45621825184757 -118.77433492914182) bank5759 +5760 POINT(33.98184895203176 -117.63428656687478) bank5760 +5761 POINT(34.74882273687078 -118.51887226629846) bank5761 +5762 POINT(33.82505648948334 -117.98426113708122) bank5762 +5763 POINT(34.5642698828124 -119.20802060061783) bank5763 +5764 POINT(35.04279382491157 -118.71615836430907) bank5764 +5765 POINT(33.10108959672623 -118.9548730416955) bank5765 +5766 POINT(34.346857616137186 -118.5689517683995) bank5766 +5767 POINT(34.47746366004338 -118.18923629545135) bank5767 +5768 POINT(34.394908359166074 -118.31066665176654) bank5768 +5769 POINT(33.07147882689056 -118.09822864644823) bank5769 +5770 POINT(33.91127784306672 -118.61734058258932) bank5770 +5771 POINT(34.43494390180673 -117.57951520196625) bank5771 +5772 POINT(34.84913532880022 -119.16296803703838) bank5772 +5773 POINT(33.459875172442416 -117.62475266898893) bank5773 +5774 POINT(35.043433960386906 -118.1106544342596) bank5774 +5775 POINT(33.2274284579961 -117.34788855835876) bank5775 +5776 POINT(33.17021827947364 -117.97958736678993) bank5776 +5777 POINT(33.3086691050355 -118.46714668884498) bank5777 +5778 POINT(33.84976744713495 -117.26728726370037) bank5778 +5779 POINT(34.44391218436474 -118.2507303898601) bank5779 +5780 POINT(34.056608313166265 -118.58635695218643) bank5780 +5781 POINT(34.003358150872536 -117.43882558610348) bank5781 +5782 POINT(33.88571495099203 -117.67591157868868) bank5782 +5783 POINT(33.319507399960884 -118.68422583109333) bank5783 +5784 POINT(33.33622339647301 -118.2897402810645) bank5784 +5785 POINT(34.98153186619078 -118.01355794443832) bank5785 +5786 POINT(33.215925205220536 -118.32228261760666) bank5786 +5787 POINT(35.02068154272578 -117.96461602705998) bank5787 +5788 POINT(34.20105939469334 -119.06548986428861) bank5788 +5789 POINT(33.59940279227406 -117.47015067552462) bank5789 +5790 POINT(33.49224343014891 -118.97640470936564) bank5790 +5791 POINT(35.01013189956921 -118.63053151567955) bank5791 +5792 POINT(33.949304798764274 -117.47561886733415) bank5792 +5793 POINT(34.685760681576824 -117.37704684506008) bank5793 +5794 POINT(33.949920452660244 -119.1495266406285) bank5794 +5795 POINT(34.003905505575844 -118.48250843216245) bank5795 +5796 POINT(33.17925376575657 -117.33301954731085) bank5796 +5797 POINT(33.32594327429887 -119.0185967541414) bank5797 +5798 POINT(34.18064797704467 -119.22743915207394) bank5798 +5799 POINT(33.1646587134876 -118.18044871832765) bank5799 +5800 POINT(33.873767309168535 -118.35149774803558) bank5800 +5801 POINT(33.352794099146465 -119.12064298719758) bank5801 +5802 POINT(33.41135874963713 -118.81654477682045) bank5802 +5803 POINT(34.198164517041455 -117.5369763038243) bank5803 +5804 POINT(33.60516903628251 -118.54061070944158) bank5804 +5805 POINT(33.615932346576145 -118.93957978040683) bank5805 +5806 POINT(34.22135706336919 -118.52663914733284) bank5806 +5807 POINT(33.66850266093415 -117.70498103064114) bank5807 +5808 POINT(34.39245101342105 -118.42096365764128) bank5808 +5809 POINT(33.99949959488128 -118.5982594572913) bank5809 +5810 POINT(34.17271762683523 -117.94959328896246) bank5810 +5811 POINT(34.207231953418926 -117.31310013509105) bank5811 +5812 POINT(34.77784581926762 -117.56893440024612) bank5812 +5813 POINT(33.983028920084145 -117.52337264939484) bank5813 +5814 POINT(33.79896287253312 -118.73766887448572) bank5814 +5815 POINT(34.064596354800344 -118.23333663388408) bank5815 +5816 POINT(34.83722626349115 -119.22287289736276) bank5816 +5817 POINT(33.8637231495042 -117.37989363453244) bank5817 +5818 POINT(33.936082147467914 -118.9398357735659) bank5818 +5819 POINT(33.48253978031933 -117.30250142176212) bank5819 +5820 POINT(34.80713624693819 -118.53713831863175) bank5820 +5821 POINT(34.768307722461984 -119.17271772502461) bank5821 +5822 POINT(33.88853706023131 -118.80308429129474) bank5822 +5823 POINT(34.51155295772632 -118.30712713296784) bank5823 +5824 POINT(34.04510699945829 -117.93015089535535) bank5824 +5825 POINT(33.554541314400325 -118.71853993894499) bank5825 +5826 POINT(33.69051261544246 -117.67223274026554) bank5826 +5827 POINT(35.003427260822754 -119.13681426568277) bank5827 +5828 POINT(33.716229991999555 -119.13084793340795) bank5828 +5829 POINT(33.13338011666881 -119.17931523503968) bank5829 +5830 POINT(33.333444117029636 -118.9387536274994) bank5830 +5831 POINT(34.080295001700506 -118.02895174582714) bank5831 +5832 POINT(33.091860478566026 -117.7160904193186) bank5832 +5833 POINT(34.302535323781235 -118.4179773570208) bank5833 +5834 POINT(34.04153744778851 -119.16170057220917) bank5834 +5835 POINT(34.381960893082855 -117.31500466687706) bank5835 +5836 POINT(33.68423875284353 -117.25814605775781) bank5836 +5837 POINT(33.94699477033361 -118.09279758587022) bank5837 +5838 POINT(33.23465588001611 -117.60027253388209) bank5838 +5839 POINT(34.3649313551037 -118.23647174691192) bank5839 +5840 POINT(34.28648701462787 -117.42218156196893) bank5840 +5841 POINT(34.27741089725541 -117.5404729133519) bank5841 +5842 POINT(33.284105141031645 -119.21396106709358) bank5842 +5843 POINT(34.197940479039524 -118.11531378924317) bank5843 +5844 POINT(35.035076695051345 -118.48984805240075) bank5844 +5845 POINT(34.2227801233758 -119.04127452067937) bank5845 +5846 POINT(34.80477118188626 -119.21577281013593) bank5846 +5847 POINT(34.30984848021029 -117.91228283396066) bank5847 +5848 POINT(33.2264431174685 -118.38180119062787) bank5848 +5849 POINT(34.230562141441254 -119.08640216462739) bank5849 +5850 POINT(34.896948449587434 -117.94457031322338) bank5850 +5851 POINT(34.678633746148144 -117.58728287916202) bank5851 +5852 POINT(33.58917438867598 -118.57955921218796) bank5852 +5853 POINT(33.17824301040482 -117.81383349685036) bank5853 +5854 POINT(34.5235952950508 -119.23640099114866) bank5854 +5855 POINT(35.00343398605817 -117.55033495210701) bank5855 +5856 POINT(34.24984459238233 -119.08312102288008) bank5856 +5857 POINT(33.57250884636873 -118.02244622749828) bank5857 +5858 POINT(34.36293349971339 -118.40787410068025) bank5858 +5859 POINT(33.96065839860209 -118.98441434353231) bank5859 +5860 POINT(34.08865664156197 -118.87362871878308) bank5860 +5861 POINT(33.935610545721396 -118.8289509294556) bank5861 +5862 POINT(34.457308065702655 -119.12250066982014) bank5862 +5863 POINT(33.820724232158085 -118.50431553593194) bank5863 +5864 POINT(35.03907319131115 -118.6595654270914) bank5864 +5865 POINT(33.72677943110341 -119.1507028398077) bank5865 +5866 POINT(34.175478202821445 -118.00012287721653) bank5866 +5867 POINT(33.20296657887736 -118.64522372180147) bank5867 +5868 POINT(34.4719966449369 -117.79122402205526) bank5868 +5869 POINT(34.09788806221204 -119.21059148417282) bank5869 +5870 POINT(34.830720639337706 -119.087763154978) bank5870 +5871 POINT(34.28344418596455 -117.49277430483937) bank5871 +5872 POINT(33.32364081398383 -118.88256605441609) bank5872 +5873 POINT(34.97058350671501 -118.54690218979303) bank5873 +5874 POINT(33.63194849110943 -117.95383379708488) bank5874 +5875 POINT(33.912339454471464 -118.92763988308883) bank5875 +5876 POINT(34.657094033974126 -118.63522360559554) bank5876 +5877 POINT(34.168982560529585 -119.07653476110616) bank5877 +5878 POINT(33.48286061368225 -118.37617708131823) bank5878 +5879 POINT(33.14468188144798 -118.89725970569049) bank5879 +5880 POINT(34.26763478851084 -118.3744728637607) bank5880 +5881 POINT(33.690132537582485 -117.54824056972637) bank5881 +5882 POINT(33.12962177566689 -118.66279978206282) bank5882 +5883 POINT(33.182509295787995 -118.25303416915123) bank5883 +5884 POINT(34.991177669240116 -119.12324620025034) bank5884 +5885 POINT(34.5687192630149 -117.69052415128476) bank5885 +5886 POINT(33.52969605283539 -117.68587989708236) bank5886 +5887 POINT(34.47209295092181 -119.11440928219663) bank5887 +5888 POINT(34.52183211051146 -118.47271412565428) bank5888 +5889 POINT(34.15662045831916 -117.66625434930334) bank5889 +5890 POINT(33.10926042297851 -118.67193613771349) bank5890 +5891 POINT(33.76141677368343 -118.35867179077326) bank5891 +5892 POINT(34.65638952276041 -118.40647959031894) bank5892 +5893 POINT(34.63806238363884 -117.84958943925854) bank5893 +5894 POINT(33.49243695990096 -117.62750123106349) bank5894 +5895 POINT(34.619381362745266 -119.06792327731362) bank5895 +5896 POINT(33.08411365193622 -119.13126752333746) bank5896 +5897 POINT(34.70342079848477 -117.42384792947999) bank5897 +5898 POINT(34.75050805711113 -118.0813167559235) bank5898 +5899 POINT(34.32196020952777 -117.34538187550119) bank5899 +5900 POINT(33.35740008501974 -118.53104609593021) bank5900 +5901 POINT(33.9678166267261 -118.62554984070545) bank5901 +5902 POINT(33.71516562491022 -118.54766519777013) bank5902 +5903 POINT(33.46326513684424 -118.88215836294499) bank5903 +5904 POINT(34.951698165766864 -119.03253189774044) bank5904 +5905 POINT(33.23409086903997 -117.51760919159675) bank5905 +5906 POINT(34.66672242822804 -118.03121946966553) bank5906 +5907 POINT(34.90324809481186 -117.34421389870191) bank5907 +5908 POINT(34.328430746514535 -118.66471675535506) bank5908 +5909 POINT(34.959748849329166 -119.20109467031101) bank5909 +5910 POINT(34.73054611531798 -118.38615728939021) bank5910 +5911 POINT(33.315125353162884 -118.33997219188514) bank5911 +5912 POINT(33.16229759203534 -118.93582071261596) bank5912 +5913 POINT(34.72733699323652 -118.58899990950144) bank5913 +5914 POINT(33.28177981379336 -117.95386512945485) bank5914 +5915 POINT(34.989428712005996 -117.4572267710659) bank5915 +5916 POINT(34.60694128802609 -118.2070313888006) bank5916 +5917 POINT(34.05987960319675 -118.01896042096844) bank5917 +5918 POINT(34.40176226381219 -117.67042435756827) bank5918 +5919 POINT(33.752169676690094 -118.43108763331617) bank5919 +5920 POINT(33.93678605449332 -117.54761022445673) bank5920 +5921 POINT(33.84773552680514 -118.1539623749493) bank5921 +5922 POINT(34.18298452255284 -118.87895553820084) bank5922 +5923 POINT(33.391897739705044 -118.82048309340927) bank5923 +5924 POINT(33.22854557101228 -117.85533843697115) bank5924 +5925 POINT(34.016507889076735 -118.58764679637989) bank5925 +5926 POINT(34.54763116785776 -117.45795334556064) bank5926 +5927 POINT(33.147507102854966 -117.9158108363413) bank5927 +5928 POINT(33.8701751278167 -119.24306846209672) bank5928 +5929 POINT(33.8777097326378 -118.9352244969023) bank5929 +5930 POINT(33.51270150340964 -117.64097410469375) bank5930 +5931 POINT(33.75957958366478 -118.1924179811312) bank5931 +5932 POINT(33.65553489401383 -117.66693743237398) bank5932 +5933 POINT(33.930565179386015 -118.01403124361818) bank5933 +5934 POINT(33.60538160196339 -118.51664033078113) bank5934 +5935 POINT(34.043635509202254 -117.65827878662056) bank5935 +5936 POINT(33.874907267969974 -117.92044871611039) bank5936 +5937 POINT(34.418846653652935 -118.21671380259401) bank5937 +5938 POINT(34.082413781725876 -118.4268476742567) bank5938 +5939 POINT(33.75176579295525 -119.02452287759321) bank5939 +5940 POINT(33.51947801860801 -117.76693473497976) bank5940 +5941 POINT(33.86760728072529 -118.20600539368495) bank5941 +5942 POINT(34.643819966971364 -119.0500006559094) bank5942 +5943 POINT(34.56717066415349 -118.55533210208358) bank5943 +5944 POINT(33.78005505824819 -117.25816754226972) bank5944 +5945 POINT(34.97210804379187 -118.68420799813629) bank5945 +5946 POINT(34.882701376549065 -117.46674762526678) bank5946 +5947 POINT(34.90737464028912 -117.6566063633681) bank5947 +5948 POINT(34.28727023440695 -119.17904723522392) bank5948 +5949 POINT(34.17667882592761 -117.66589270286296) bank5949 +5950 POINT(34.57301924441632 -119.07973908802887) bank5950 +5951 POINT(34.95684766906881 -118.56864390341622) bank5951 +5952 POINT(34.39037867394518 -118.27064600333753) bank5952 +5953 POINT(33.65546970208153 -118.18704280237827) bank5953 +5954 POINT(33.856804707826036 -118.89771057174501) bank5954 +5955 POINT(34.55334746938805 -117.91152095460399) bank5955 +5956 POINT(33.334025922895634 -117.70085449378561) bank5956 +5957 POINT(34.03561263876961 -118.25233794569154) bank5957 +5958 POINT(34.79087742835824 -118.04869198994362) bank5958 +5959 POINT(34.81214964495219 -118.80594668701778) bank5959 +5960 POINT(34.98033623670881 -117.74093278336744) bank5960 +5961 POINT(34.01193396086383 -117.24557578171994) bank5961 +5962 POINT(34.15455722750124 -118.55279682210035) bank5962 +5963 POINT(33.231964140420644 -118.40574937621076) bank5963 +5964 POINT(34.5124856385993 -118.38913527965472) bank5964 +5965 POINT(34.89063532476441 -117.53099676739163) bank5965 +5966 POINT(35.02263845574506 -117.88034721054198) bank5966 +5967 POINT(33.16736734342218 -117.35172931541048) bank5967 +5968 POINT(34.77715337986095 -118.57846544222309) bank5968 +5969 POINT(34.33282837468327 -118.02281221645579) bank5969 +5970 POINT(33.369298218873936 -118.52280415929317) bank5970 +5971 POINT(34.5296580047559 -118.80286507890676) bank5971 +5972 POINT(34.61202525446187 -119.06533508305917) bank5972 +5973 POINT(33.84248918366709 -117.91422926974292) bank5973 +5974 POINT(34.90345117881144 -119.0321949791374) bank5974 +5975 POINT(34.99109979053254 -118.30379437563364) bank5975 +5976 POINT(34.44350868987449 -118.96375278701946) bank5976 +5977 POINT(34.08274587887261 -118.35039085459043) bank5977 +5978 POINT(34.12342732985964 -117.4282198083106) bank5978 +5979 POINT(34.131100235262664 -118.45764605008961) bank5979 +5980 POINT(34.292358745732095 -118.81632237912368) bank5980 +5981 POINT(33.622019585753854 -118.67705014408892) bank5981 +5982 POINT(33.76583076967749 -119.17180110242396) bank5982 +5983 POINT(33.78370233200569 -117.3099313565421) bank5983 +5984 POINT(34.12303858764681 -117.80255337423336) bank5984 +5985 POINT(33.19390215688395 -117.59131176954034) bank5985 +5986 POINT(33.2104812718925 -118.26411402346925) bank5986 +5987 POINT(34.25945340475644 -119.13156826352639) bank5987 +5988 POINT(34.04277934763336 -118.2122789928973) bank5988 +5989 POINT(34.923736886939636 -118.16414533780663) bank5989 +5990 POINT(34.71220550398102 -118.18131153993669) bank5990 +5991 POINT(33.755839767299996 -119.05559710045047) bank5991 +5992 POINT(34.91103677800429 -117.85025616404441) bank5992 +5993 POINT(33.79585310136667 -118.93416723649247) bank5993 +5994 POINT(34.47155376314757 -118.48883252144947) bank5994 +5995 POINT(34.314419131074764 -118.59698282605795) bank5995 +5996 POINT(33.3936729895394 -119.23786533883796) bank5996 +5997 POINT(34.248643633576805 -117.58928796704774) bank5997 +5998 POINT(33.68379245108862 -118.21858392012987) bank5998 +5999 POINT(34.44729621492132 -117.94298040720942) bank5999 +6000 POINT(34.68200122827581 -118.35962650257223) bank6000 +6001 POINT(33.47335905388593 -118.5877646844462) bank6001 +6002 POINT(34.54511197017824 -118.28171072282261) bank6002 +6003 POINT(33.6373298935572 -118.96614040072116) bank6003 +6004 POINT(34.809085196112 -118.57773340694212) bank6004 +6005 POINT(34.08411711089374 -118.09233907719057) bank6005 +6006 POINT(33.80504605317157 -118.12116998815652) bank6006 +6007 POINT(34.79784373789172 -118.9161679209971) bank6007 +6008 POINT(34.957643552891746 -118.0867684437042) bank6008 +6009 POINT(34.650739757903146 -118.88628743580061) bank6009 +6010 POINT(33.76764919598751 -118.2441988598351) bank6010 +6011 POINT(34.30264132174138 -117.25751033467674) bank6011 +6012 POINT(34.135649422348095 -119.23743303839217) bank6012 +6013 POINT(33.154680760605245 -117.79946975289518) bank6013 +6014 POINT(33.55909371532838 -117.41666468571981) bank6014 +6015 POINT(34.721853849590424 -118.7588133204238) bank6015 +6016 POINT(34.320823270119675 -117.44363753531727) bank6016 +6017 POINT(33.42635857204234 -117.42320766870829) bank6017 +6018 POINT(34.770959530402344 -117.37290874262902) bank6018 +6019 POINT(33.82524118486768 -118.29451574788055) bank6019 +6020 POINT(33.36711540486563 -118.93204043307185) bank6020 +6021 POINT(33.458344885658676 -118.27630516162627) bank6021 +6022 POINT(33.242316867262026 -117.61933874014608) bank6022 +6023 POINT(34.85671571754011 -118.24563300505591) bank6023 +6024 POINT(35.003061897669966 -117.49714302789451) bank6024 +6025 POINT(33.185429816028346 -117.40118032864122) bank6025 +6026 POINT(34.10096916050843 -118.96309356994782) bank6026 +6027 POINT(33.31059014501594 -117.88613964360687) bank6027 +6028 POINT(34.23178533684749 -118.91574121087746) bank6028 +6029 POINT(34.83575250567953 -117.45696446891212) bank6029 +6030 POINT(34.90192407965757 -117.69039462781507) bank6030 +6031 POINT(33.48141658384623 -119.20831884635237) bank6031 +6032 POINT(34.630796135147925 -119.0576105619403) bank6032 +6033 POINT(34.2608138005335 -117.44980368851881) bank6033 +6034 POINT(33.830562373744215 -117.55068955903947) bank6034 +6035 POINT(34.227811402745445 -117.7045510400668) bank6035 +6036 POINT(34.02015522606639 -117.7526363110769) bank6036 +6037 POINT(33.60067356909473 -117.5532354275814) bank6037 +6038 POINT(33.198244817326 -118.08087838336976) bank6038 +6039 POINT(33.965786165287334 -117.44303062819395) bank6039 +6040 POINT(33.41359704043671 -118.74774745447507) bank6040 +6041 POINT(33.68537750750836 -117.30430276968835) bank6041 +6042 POINT(33.39837356018279 -117.40960752066506) bank6042 +6043 POINT(34.8696623375569 -117.31387740207033) bank6043 +6044 POINT(34.876232486709114 -119.11188513929807) bank6044 +6045 POINT(33.86653006942754 -117.53179866888156) bank6045 +6046 POINT(33.53454691805082 -117.38405796825344) bank6046 +6047 POINT(34.42184878513003 -118.1881157118469) bank6047 +6048 POINT(33.45977307462958 -118.17627413822271) bank6048 +6049 POINT(34.24964417008904 -118.60372919523854) bank6049 +6050 POINT(33.481090721367465 -118.94766411421959) bank6050 +6051 POINT(33.81215673093348 -117.75445366039499) bank6051 +6052 POINT(33.616595163441 -117.72948788412835) bank6052 +6053 POINT(34.62839048614309 -117.88222164482593) bank6053 +6054 POINT(33.430573763138234 -118.18440074755858) bank6054 +6055 POINT(34.73333726955452 -118.31438448978436) bank6055 +6056 POINT(33.0609078675042 -118.923129120975) bank6056 +6057 POINT(33.59159461728365 -117.89173268772794) bank6057 +6058 POINT(34.19778047210631 -118.92875808552864) bank6058 +6059 POINT(34.41081106145627 -118.53516706714132) bank6059 +6060 POINT(33.583121224188886 -118.41767045984435) bank6060 +6061 POINT(34.338514183914334 -118.35202520689575) bank6061 +6062 POINT(34.89932638048112 -118.36207603122456) bank6062 +6063 POINT(33.45649112667403 -117.4053895439022) bank6063 +6064 POINT(33.992821613558014 -119.02660446374882) bank6064 +6065 POINT(34.85574007666472 -119.01437817037693) bank6065 +6066 POINT(34.145620337246136 -118.32331070455994) bank6066 +6067 POINT(34.349697667302195 -117.60751796108741) bank6067 +6068 POINT(34.998220568336166 -119.21265412884189) bank6068 +6069 POINT(34.556938682855815 -118.55844731051428) bank6069 +6070 POINT(34.46946508725088 -118.67066156242547) bank6070 +6071 POINT(34.25910198910512 -119.22021860799575) bank6071 +6072 POINT(34.70781798544458 -117.8520556289724) bank6072 +6073 POINT(34.04933253973569 -117.90673945816803) bank6073 +6074 POINT(33.604097664255704 -118.81165763813755) bank6074 +6075 POINT(34.35943109452746 -117.34064944468754) bank6075 +6076 POINT(34.1521514909766 -117.73487359255164) bank6076 +6077 POINT(34.41025768665964 -118.49385813538055) bank6077 +6078 POINT(33.188582703451566 -117.30954673134642) bank6078 +6079 POINT(35.011074092874885 -119.15638868342866) bank6079 +6080 POINT(33.77654572622926 -119.15595649491502) bank6080 +6081 POINT(34.02362796017322 -118.0096983743093) bank6081 +6082 POINT(34.950667761211946 -117.38986190058374) bank6082 +6083 POINT(33.58994895116919 -119.21546314343453) bank6083 +6084 POINT(34.89432988133468 -118.7580476753961) bank6084 +6085 POINT(33.704518803138534 -118.44203849544634) bank6085 +6086 POINT(34.206121369986995 -117.56663851247949) bank6086 +6087 POINT(34.67682348476873 -118.99058217394541) bank6087 +6088 POINT(33.30501028490046 -117.56364784117363) bank6088 +6089 POINT(34.43426118510338 -117.4215227594979) bank6089 +6090 POINT(34.92736459989796 -119.12232462650422) bank6090 +6091 POINT(34.39553663689642 -118.71318187075114) bank6091 +6092 POINT(33.87746184389911 -118.44977959548174) bank6092 +6093 POINT(33.114149696842226 -118.61326458614052) bank6093 +6094 POINT(33.708978459651625 -117.43996995569032) bank6094 +6095 POINT(34.14903786306955 -117.61717885268956) bank6095 +6096 POINT(34.271096493803476 -117.40130988563335) bank6096 +6097 POINT(34.538351149225896 -117.5087045144213) bank6097 +6098 POINT(34.79758813411645 -117.28073572233797) bank6098 +6099 POINT(33.54839494103914 -118.72697120908268) bank6099 +6100 POINT(34.51457292532432 -119.21066418074793) bank6100 +6101 POINT(33.929030504042395 -117.76052316228858) bank6101 +6102 POINT(34.3604119857686 -117.33246852252338) bank6102 +6103 POINT(33.46210242298126 -119.02697306416226) bank6103 +6104 POINT(34.06015853500016 -118.29101367854348) bank6104 +6105 POINT(34.34498312395047 -119.04292869202143) bank6105 +6106 POINT(33.8108217166888 -118.53914476789114) bank6106 +6107 POINT(35.00987234985302 -117.43859458728257) bank6107 +6108 POINT(34.9333961481682 -118.85883197034264) bank6108 +6109 POINT(34.16024093966456 -119.0598984028488) bank6109 +6110 POINT(34.6007360153163 -118.35746212054782) bank6110 +6111 POINT(33.8551411411976 -119.08727144037128) bank6111 +6112 POINT(34.480284294932744 -119.0544672032653) bank6112 +6113 POINT(33.63451378893862 -118.22210307486779) bank6113 +6114 POINT(33.70424047788648 -118.59495985719015) bank6114 +6115 POINT(34.706199138458906 -118.46229175987233) bank6115 +6116 POINT(34.60671058147919 -117.49444776625025) bank6116 +6117 POINT(34.814885452713774 -117.4079765114608) bank6117 +6118 POINT(34.49299557480635 -118.29800763191392) bank6118 +6119 POINT(33.32848552208908 -118.10297159060293) bank6119 +6120 POINT(33.38281834492613 -119.00146219450586) bank6120 +6121 POINT(33.22380490212631 -118.40386511393989) bank6121 +6122 POINT(34.304550127041146 -119.00077805361575) bank6122 +6123 POINT(34.1404545057963 -119.16083721036694) bank6123 +6124 POINT(33.369835665839645 -117.81638872567888) bank6124 +6125 POINT(33.955678039985784 -118.38500659489571) bank6125 +6126 POINT(33.94099888050723 -118.95256749628811) bank6126 +6127 POINT(33.52941233637787 -118.86418914724817) bank6127 +6128 POINT(33.97227939080552 -117.55251193611251) bank6128 +6129 POINT(34.606619936261566 -118.39018070285478) bank6129 +6130 POINT(34.66105427776336 -118.96019981031003) bank6130 +6131 POINT(33.50999477120752 -117.70650014736248) bank6131 +6132 POINT(34.827962529235364 -119.04018496176316) bank6132 +6133 POINT(33.69999364872679 -117.72573693752476) bank6133 +6134 POINT(33.25730693197314 -118.83332696860518) bank6134 +6135 POINT(33.54385066018836 -118.3770743255788) bank6135 +6136 POINT(34.70455255833391 -118.85925978113029) bank6136 +6137 POINT(34.11174003753133 -118.02802410829105) bank6137 +6138 POINT(34.959358486175944 -118.330756026969) bank6138 +6139 POINT(33.08571006972995 -118.52278762813287) bank6139 +6140 POINT(33.833925874208205 -117.93247273617159) bank6140 +6141 POINT(34.143444276474085 -117.82946912961883) bank6141 +6142 POINT(34.05483625796876 -118.82172103297408) bank6142 +6143 POINT(34.00259053663659 -119.16127585464329) bank6143 +6144 POINT(33.60056815370064 -118.2228858635116) bank6144 +6145 POINT(33.894442907964354 -117.80684061635226) bank6145 +6146 POINT(33.73065728483975 -118.46477458290683) bank6146 +6147 POINT(33.38711588423188 -119.10786562803453) bank6147 +6148 POINT(34.45127699820031 -117.85953616269529) bank6148 +6149 POINT(34.5770162444819 -118.36319133013151) bank6149 +6150 POINT(34.667132736958294 -118.36914887021116) bank6150 +6151 POINT(34.85660819826381 -119.18360094617088) bank6151 +6152 POINT(33.77076031858117 -117.40440720958085) bank6152 +6153 POINT(34.36131361310914 -117.30279230452274) bank6153 +6154 POINT(33.7216211191088 -118.78392806952736) bank6154 +6155 POINT(33.77929514547437 -118.27603129554684) bank6155 +6156 POINT(34.56824813545228 -119.10915215768681) bank6156 +6157 POINT(34.43527577127663 -118.85052409273064) bank6157 +6158 POINT(33.67338867199991 -118.4145129769941) bank6158 +6159 POINT(34.487385534383094 -118.1395432637605) bank6159 +6160 POINT(34.496159469577755 -117.54846200185362) bank6160 +6161 POINT(34.39211228846119 -118.91736957511594) bank6161 +6162 POINT(33.85019452329738 -118.56341499137378) bank6162 +6163 POINT(33.700937548587035 -117.81193249684166) bank6163 +6164 POINT(34.57807310589287 -118.57559052285247) bank6164 +6165 POINT(34.245009715634794 -118.85210779620645) bank6165 +6166 POINT(33.173526329651104 -118.03306399149992) bank6166 +6167 POINT(33.74459224146535 -119.12570143448309) bank6167 +6168 POINT(34.1479765163751 -118.66105222918644) bank6168 +6169 POINT(34.84416703347471 -117.60786091455758) bank6169 +6170 POINT(33.607121503172195 -118.33100323805974) bank6170 +6171 POINT(34.01589134423822 -117.72794529169447) bank6171 +6172 POINT(34.5048287717334 -117.57462438525367) bank6172 +6173 POINT(33.937682279518405 -119.23668500590146) bank6173 +6174 POINT(34.83951206337706 -117.59743851804066) bank6174 +6175 POINT(33.95084812273161 -119.15879462285501) bank6175 +6176 POINT(34.014286978741474 -118.95757746928935) bank6176 +6177 POINT(33.91376110960013 -117.26590037250217) bank6177 +6178 POINT(33.26748392793679 -118.75443398364048) bank6178 +6179 POINT(33.663139128310995 -119.00915223484157) bank6179 +6180 POINT(33.710896574874134 -118.7667237238142) bank6180 +6181 POINT(33.37598029621178 -119.16644788587612) bank6181 +6182 POINT(34.687509856653655 -118.84530067672561) bank6182 +6183 POINT(33.605058986673164 -117.48618345149262) bank6183 +6184 POINT(33.50471927232196 -118.96948065827382) bank6184 +6185 POINT(33.844636908036755 -118.80354701039309) bank6185 +6186 POINT(34.96781627285402 -118.92964962031252) bank6186 +6187 POINT(33.08842650724166 -117.73978120513414) bank6187 +6188 POINT(34.59040615843273 -117.68810484116324) bank6188 +6189 POINT(33.854504252162286 -119.00751194930614) bank6189 +6190 POINT(34.46712692359929 -118.27409859257023) bank6190 +6191 POINT(34.20241256278772 -118.38412908764498) bank6191 +6192 POINT(33.9393617920414 -118.60274722140969) bank6192 +6193 POINT(33.56471119951373 -118.23570418143554) bank6193 +6194 POINT(33.17412697708577 -117.69747894312303) bank6194 +6195 POINT(33.90405701527059 -117.84075728873843) bank6195 +6196 POINT(34.60742924039415 -119.0180166965916) bank6196 +6197 POINT(34.28490360757486 -118.88478452162852) bank6197 +6198 POINT(34.66178863171173 -119.09977496058752) bank6198 +6199 POINT(34.1579152681145 -117.68976526111439) bank6199 +6200 POINT(34.32033066564292 -117.52302436936574) bank6200 +6201 POINT(34.46960074158076 -118.72544473308669) bank6201 +6202 POINT(34.358226161948416 -118.48665857102715) bank6202 +6203 POINT(34.042581283052805 -117.68026522987581) bank6203 +6204 POINT(33.31141500251552 -118.43856503737162) bank6204 +6205 POINT(33.3625131952312 -117.92379326017351) bank6205 +6206 POINT(34.85198766634651 -119.22019438914906) bank6206 +6207 POINT(33.48478433130458 -118.9341837410047) bank6207 +6208 POINT(34.2391318859128 -118.72463823215794) bank6208 +6209 POINT(34.0815873316696 -117.66037865511517) bank6209 +6210 POINT(33.8551248573199 -117.3424070980858) bank6210 +6211 POINT(34.77232687070104 -117.46105768164648) bank6211 +6212 POINT(34.35351417984826 -118.88834817679567) bank6212 +6213 POINT(34.38880771381173 -117.89087764730314) bank6213 +6214 POINT(33.67038497972527 -117.37963500096355) bank6214 +6215 POINT(33.81865562837549 -118.75363355240333) bank6215 +6216 POINT(33.72017334917887 -118.90179137428409) bank6216 +6217 POINT(33.77535595844713 -117.84782933256467) bank6217 +6218 POINT(33.82280724820221 -117.70729325111502) bank6218 +6219 POINT(33.89901996621792 -119.12165396281095) bank6219 +6220 POINT(33.36670928076596 -118.7475723696927) bank6220 +6221 POINT(34.70175790177073 -118.78087941412325) bank6221 +6222 POINT(33.690496857733166 -117.54826105505376) bank6222 +6223 POINT(34.45441202293231 -117.34855731999104) bank6223 +6224 POINT(34.17130736401964 -118.85679965569098) bank6224 +6225 POINT(33.465362297885626 -118.89714560607075) bank6225 +6226 POINT(34.11897021283453 -118.08426094148632) bank6226 +6227 POINT(33.07171758880171 -118.28729031149322) bank6227 +6228 POINT(34.488226397945795 -118.09272638595822) bank6228 +6229 POINT(34.66703859887971 -118.47372352559707) bank6229 +6230 POINT(33.65976126770573 -118.24545501180319) bank6230 +6231 POINT(33.94846242856996 -117.78778292480582) bank6231 +6232 POINT(34.483391310889694 -118.31362997583551) bank6232 +6233 POINT(34.26072001550792 -117.35295342469941) bank6233 +6234 POINT(33.95364632284805 -117.35002244479675) bank6234 +6235 POINT(35.021037941551995 -118.91682770497543) bank6235 +6236 POINT(34.0810397600981 -118.43796722431364) bank6236 +6237 POINT(34.20562429004967 -119.16202422648428) bank6237 +6238 POINT(34.04202322350726 -118.69733895946675) bank6238 +6239 POINT(33.304197973574034 -118.1122207703732) bank6239 +6240 POINT(34.92227373555753 -118.372284459579) bank6240 +6241 POINT(34.641615959446945 -117.41137980851056) bank6241 +6242 POINT(33.18212762981544 -118.81217971387129) bank6242 +6243 POINT(33.19147528444447 -118.65072533050969) bank6243 +6244 POINT(34.31881162625211 -118.52747426312796) bank6244 +6245 POINT(33.80943102507873 -118.77637102791152) bank6245 +6246 POINT(33.15587879434634 -119.15182909167342) bank6246 +6247 POINT(33.441228996403744 -119.00843280643836) bank6247 +6248 POINT(34.94363984176962 -118.70237379675667) bank6248 +6249 POINT(33.26557509099568 -118.29751279853366) bank6249 +6250 POINT(33.98318140331559 -117.62408458449417) bank6250 +6251 POINT(34.435445862763146 -118.03583168498078) bank6251 +6252 POINT(34.45001289302055 -117.61520808463226) bank6252 +6253 POINT(33.82738633027534 -117.87169226013224) bank6253 +6254 POINT(34.30686452851031 -118.71260334225897) bank6254 +6255 POINT(33.78941140257941 -118.83205463805095) bank6255 +6256 POINT(34.63348225110629 -117.87308706693248) bank6256 +6257 POINT(34.75869668803813 -118.64616413899171) bank6257 +6258 POINT(33.642506467299285 -117.55360933158012) bank6258 +6259 POINT(34.85637403680376 -117.5362965741504) bank6259 +6260 POINT(33.965139070229505 -117.29969185797621) bank6260 +6261 POINT(34.06205165751572 -118.5256119301971) bank6261 +6262 POINT(34.12339608750628 -118.29309938748096) bank6262 +6263 POINT(33.61355759911376 -118.78468851292405) bank6263 +6264 POINT(33.99187577552822 -118.54741058459612) bank6264 +6265 POINT(34.622887030564335 -118.41233193826434) bank6265 +6266 POINT(34.30920790963597 -117.44883921707105) bank6266 +6267 POINT(34.07773781415108 -117.60417980014982) bank6267 +6268 POINT(33.50619200544617 -118.97911265680206) bank6268 +6269 POINT(33.897184003575994 -118.68506783412627) bank6269 +6270 POINT(34.13563938153365 -117.2913764957097) bank6270 +6271 POINT(34.21439886205341 -118.05344670687327) bank6271 +6272 POINT(33.18071488227691 -117.8965286786978) bank6272 +6273 POINT(34.63155797989834 -118.52823555205347) bank6273 +6274 POINT(34.63515335697924 -117.83358040020202) bank6274 +6275 POINT(34.98115177180611 -117.33499522013655) bank6275 +6276 POINT(33.64457708596333 -117.57713209516032) bank6276 +6277 POINT(33.59716429091466 -119.05727627239774) bank6277 +6278 POINT(33.37189684477053 -119.05516471219688) bank6278 +6279 POINT(34.35621665251519 -119.1411426003279) bank6279 +6280 POINT(33.85793226168656 -119.19767558085009) bank6280 +6281 POINT(34.05925362455037 -118.8990955109634) bank6281 +6282 POINT(34.48427762910185 -117.29579491349153) bank6282 +6283 POINT(33.83017829935681 -118.81878223333209) bank6283 +6284 POINT(34.39635983523816 -117.92217957808522) bank6284 +6285 POINT(34.7334905448902 -117.69635078158498) bank6285 +6286 POINT(33.809106148524116 -117.24596198660109) bank6286 +6287 POINT(33.268300724044174 -118.84754343267316) bank6287 +6288 POINT(33.06793225401141 -117.69183881241503) bank6288 +6289 POINT(35.009335140743445 -118.87407756431772) bank6289 +6290 POINT(33.5984218928912 -117.68105314571811) bank6290 +6291 POINT(33.33278534462575 -117.74440295619534) bank6291 +6292 POINT(34.23424240668468 -118.78196987448796) bank6292 +6293 POINT(34.9244677233584 -119.14578712134909) bank6293 +6294 POINT(34.68956859767915 -118.07551239562832) bank6294 +6295 POINT(34.74843325236611 -118.6970253529449) bank6295 +6296 POINT(34.47702972495705 -119.10278274859859) bank6296 +6297 POINT(33.080152508782206 -118.14368045400835) bank6297 +6298 POINT(33.909227611098224 -117.6935604791249) bank6298 +6299 POINT(33.74428162732619 -117.39351190410268) bank6299 +6300 POINT(34.516641767030144 -117.3979412947614) bank6300 +6301 POINT(34.156274796718925 -118.70936577832856) bank6301 +6302 POINT(34.04163331838474 -118.85418713914984) bank6302 +6303 POINT(33.31393690269537 -118.39351269366396) bank6303 +6304 POINT(33.194411188584844 -117.27646941540554) bank6304 +6305 POINT(34.39759554990861 -117.51500430492865) bank6305 +6306 POINT(33.40224844347111 -118.1899720244565) bank6306 +6307 POINT(34.274633257217275 -117.61942777464611) bank6307 +6308 POINT(33.761852225804034 -117.47727927354588) bank6308 +6309 POINT(33.8567868087982 -117.93744637361027) bank6309 +6310 POINT(34.769356662311985 -118.66300683376082) bank6310 +6311 POINT(34.894078769634575 -118.66251792282499) bank6311 +6312 POINT(34.39051373254697 -117.76247641573875) bank6312 +6313 POINT(33.89101673020121 -118.91596299961645) bank6313 +6314 POINT(33.32540823616686 -117.89724669777512) bank6314 +6315 POINT(33.05946841283223 -119.19009973993924) bank6315 +6316 POINT(34.72357232864423 -118.46250650751223) bank6316 +6317 POINT(34.662280029436594 -117.80748088397668) bank6317 +6318 POINT(34.477127279013914 -117.34541032630389) bank6318 +6319 POINT(34.93079446971334 -118.90585769766719) bank6319 +6320 POINT(33.51016923808443 -117.44495864059361) bank6320 +6321 POINT(34.35730605187541 -118.46697620631417) bank6321 +6322 POINT(34.864106638646945 -117.57617466404284) bank6322 +6323 POINT(34.32591499054957 -118.4398061930465) bank6323 +6324 POINT(33.34884313306385 -119.03042042018903) bank6324 +6325 POINT(33.674485348046325 -117.57127299959049) bank6325 +6326 POINT(35.024866976330905 -117.24989742758339) bank6326 +6327 POINT(33.238978506791895 -119.17939304420028) bank6327 +6328 POINT(33.248674777685856 -119.12568164323835) bank6328 +6329 POINT(33.50248149199442 -118.60880672007411) bank6329 +6330 POINT(34.442300954494584 -118.76798957191609) bank6330 +6331 POINT(33.786611243804906 -119.18625470927898) bank6331 +6332 POINT(34.14331710331431 -117.44370429591224) bank6332 +6333 POINT(34.65717770914246 -119.09235167385337) bank6333 +6334 POINT(34.67163805152506 -117.25541380225405) bank6334 +6335 POINT(33.74891655837379 -118.15330901397675) bank6335 +6336 POINT(34.051225395547654 -117.3714299836885) bank6336 +6337 POINT(34.566258855024394 -118.7819681599507) bank6337 +6338 POINT(34.48843350972957 -118.59272534480435) bank6338 +6339 POINT(34.82457382057857 -118.38973793901664) bank6339 +6340 POINT(33.985624079267915 -118.36868323177018) bank6340 +6341 POINT(33.72904508289838 -119.08582098515168) bank6341 +6342 POINT(33.1540912834719 -118.83783680797455) bank6342 +6343 POINT(33.677360585140235 -118.17036188722085) bank6343 +6344 POINT(35.03596357677845 -117.66435587811601) bank6344 +6345 POINT(34.373839053375406 -117.25278452649073) bank6345 +6346 POINT(33.667157348876415 -117.76236016686512) bank6346 +6347 POINT(34.70594975315268 -117.70597699867736) bank6347 +6348 POINT(34.95153860702787 -118.86132987334247) bank6348 +6349 POINT(33.54330903926987 -118.22833013317741) bank6349 +6350 POINT(33.51755837524052 -118.50121354446244) bank6350 +6351 POINT(34.9125953443858 -117.78937194259935) bank6351 +6352 POINT(34.947548089368 -117.69172983377264) bank6352 +6353 POINT(33.0798300490971 -117.26994235893378) bank6353 +6354 POINT(34.613161740507216 -117.78428816193066) bank6354 +6355 POINT(34.03926875819965 -118.59074214527925) bank6355 +6356 POINT(34.85493135750406 -118.94735400483736) bank6356 +6357 POINT(33.98165057600897 -117.91066306021271) bank6357 +6358 POINT(33.37011521494072 -118.00006635733443) bank6358 +6359 POINT(33.98602966557496 -118.02440338772153) bank6359 +6360 POINT(34.651903541111764 -117.67363190403515) bank6360 +6361 POINT(33.3324831008631 -119.15424648459877) bank6361 +6362 POINT(33.35274788118072 -117.50374084481216) bank6362 +6363 POINT(33.38328113423195 -119.0386785649315) bank6363 +6364 POINT(33.94653564463658 -118.87366976594913) bank6364 +6365 POINT(33.41390942274975 -119.13584877791318) bank6365 +6366 POINT(33.76938613085785 -119.05088419869553) bank6366 +6367 POINT(33.75065984707601 -118.33462845710974) bank6367 +6368 POINT(33.999958845187535 -119.15113254576113) bank6368 +6369 POINT(34.20657989055167 -118.87114536017273) bank6369 +6370 POINT(33.94253889763663 -118.87206016249448) bank6370 +6371 POINT(34.723113153296005 -118.34467582041351) bank6371 +6372 POINT(34.91389229850271 -118.4103340910286) bank6372 +6373 POINT(33.39160256422136 -117.6433065070134) bank6373 +6374 POINT(33.77334987472112 -119.1992258659655) bank6374 +6375 POINT(34.20102829126643 -117.77078791548902) bank6375 +6376 POINT(33.43002861222887 -118.32253247519678) bank6376 +6377 POINT(34.44085073222813 -118.9907750206002) bank6377 +6378 POINT(33.741556052753396 -119.15483592217545) bank6378 +6379 POINT(33.93844130059039 -118.98996913033902) bank6379 +6380 POINT(33.93564922641675 -118.92548568982642) bank6380 +6381 POINT(34.93067468604721 -118.37640440736924) bank6381 +6382 POINT(34.465935504996736 -119.02883281674785) bank6382 +6383 POINT(34.667403750269884 -117.35612156540498) bank6383 +6384 POINT(33.903626142731575 -118.0402484484909) bank6384 +6385 POINT(34.474688355044286 -117.35236902444325) bank6385 +6386 POINT(34.17075416924777 -118.23389133425071) bank6386 +6387 POINT(33.47883649841007 -118.94259451935223) bank6387 +6388 POINT(34.631340321916454 -117.95698725854372) bank6388 +6389 POINT(33.09283377710466 -117.99044170838555) bank6389 +6390 POINT(33.56197056021986 -117.39196118656407) bank6390 +6391 POINT(34.243098404979904 -118.77181067734027) bank6391 +6392 POINT(33.72462342073047 -118.42630687248527) bank6392 +6393 POINT(33.95726575987993 -118.24727693938627) bank6393 +6394 POINT(33.08019416745553 -117.24606163033721) bank6394 +6395 POINT(33.93337856015999 -117.47236824090551) bank6395 +6396 POINT(33.44636258699409 -117.94013454023258) bank6396 +6397 POINT(33.1354097158158 -118.01984476501141) bank6397 +6398 POINT(33.43751798637469 -117.90560671851456) bank6398 +6399 POINT(34.52211178200169 -119.22354192165574) bank6399 +6400 POINT(34.971283248282646 -118.31493905062867) bank6400 +6401 POINT(34.74244346684596 -118.89185308364826) bank6401 +6402 POINT(33.73287224747325 -117.52009127535408) bank6402 +6403 POINT(35.00261435623022 -118.0858804074041) bank6403 +6404 POINT(33.575984715927106 -118.15102705380382) bank6404 +6405 POINT(33.84021271646774 -118.28113479625677) bank6405 +6406 POINT(33.7338290113482 -118.79737876190356) bank6406 +6407 POINT(33.940440204324794 -118.73113889173474) bank6407 +6408 POINT(33.57440486661836 -117.93140637699362) bank6408 +6409 POINT(34.822565072699376 -118.9187335568458) bank6409 +6410 POINT(34.426330170366505 -118.56652575207201) bank6410 +6411 POINT(34.58720890894991 -117.24912831162597) bank6411 +6412 POINT(33.87053373343044 -117.6941473081421) bank6412 +6413 POINT(34.120744485508084 -118.27392196428103) bank6413 +6414 POINT(34.44674880378061 -118.46760702182895) bank6414 +6415 POINT(33.58192926348201 -118.7330757631294) bank6415 +6416 POINT(33.80807361095848 -118.33072354103098) bank6416 +6417 POINT(34.71059878601959 -118.22482695324607) bank6417 +6418 POINT(33.48275658666823 -118.11188944720077) bank6418 +6419 POINT(33.96647475129804 -118.4456222215645) bank6419 +6420 POINT(33.127706044393676 -118.98458698661676) bank6420 +6421 POINT(34.917552279461596 -118.31747133879564) bank6421 +6422 POINT(33.89182898036569 -117.32669396700445) bank6422 +6423 POINT(33.086108147668206 -118.05332083736243) bank6423 +6424 POINT(33.505320787917206 -118.00342926645801) bank6424 +6425 POINT(34.222814338694185 -118.95007107591378) bank6425 +6426 POINT(33.60410016474925 -117.78597043269187) bank6426 +6427 POINT(34.890201653389205 -119.14181043984262) bank6427 +6428 POINT(34.17766904916105 -119.10547000144287) bank6428 +6429 POINT(33.11925869984545 -118.42118069426493) bank6429 +6430 POINT(33.395651560875194 -117.80824413940765) bank6430 +6431 POINT(34.41681763642747 -118.602440346107) bank6431 +6432 POINT(34.05500210383356 -118.78836427563812) bank6432 +6433 POINT(34.34886025311948 -118.28555021563311) bank6433 +6434 POINT(34.87842460063519 -118.5793512965826) bank6434 +6435 POINT(34.9276753602621 -117.8378735399535) bank6435 +6436 POINT(33.297844829364855 -117.89978700939797) bank6436 +6437 POINT(35.03885273974825 -118.87981139556199) bank6437 +6438 POINT(33.84554275604195 -118.52436117044255) bank6438 +6439 POINT(33.803185596670424 -117.68182280785541) bank6439 +6440 POINT(34.71733263053387 -118.71330885252836) bank6440 +6441 POINT(34.35749397026727 -118.90845787572408) bank6441 +6442 POINT(33.40612214311401 -117.95446408966714) bank6442 +6443 POINT(34.91311583754705 -117.85604771594367) bank6443 +6444 POINT(33.346646557841915 -119.05194082200467) bank6444 +6445 POINT(34.4835230920072 -118.78555789365697) bank6445 +6446 POINT(34.91678199293041 -117.90496412356876) bank6446 +6447 POINT(33.20782599027989 -118.28470620799465) bank6447 +6448 POINT(34.22225898075661 -117.56119220256521) bank6448 +6449 POINT(33.76076005275277 -117.34818157197314) bank6449 +6450 POINT(33.95485495098574 -117.57471141674591) bank6450 +6451 POINT(34.17389051808154 -117.34581026546851) bank6451 +6452 POINT(35.04141554982791 -118.89558142808275) bank6452 +6453 POINT(33.595527772864784 -118.35531414140944) bank6453 +6454 POINT(33.79349698893567 -117.98234793363561) bank6454 +6455 POINT(35.03213413093952 -117.51203313654491) bank6455 +6456 POINT(34.9837462611144 -118.34259493122026) bank6456 +6457 POINT(33.50973763498543 -118.99634022435082) bank6457 +6458 POINT(33.26472377314899 -118.00657669660515) bank6458 +6459 POINT(34.53360234525206 -117.4463141099567) bank6459 +6460 POINT(34.70711578171789 -118.06956292714234) bank6460 +6461 POINT(33.09073137254119 -117.39558518150916) bank6461 +6462 POINT(34.09325281913582 -118.00133834260582) bank6462 +6463 POINT(33.31440915216516 -117.54567054056794) bank6463 +6464 POINT(34.463432129404424 -118.27605361056068) bank6464 +6465 POINT(34.26982953923725 -119.19535976664284) bank6465 +6466 POINT(34.44275061228643 -118.97174436166236) bank6466 +6467 POINT(33.440063719026995 -118.30905821285268) bank6467 +6468 POINT(34.82337800255038 -117.29434183971992) bank6468 +6469 POINT(34.01174098567871 -118.80023168821617) bank6469 +6470 POINT(33.218951655794605 -118.68301614701635) bank6470 +6471 POINT(34.95050496350761 -118.32305253613396) bank6471 +6472 POINT(33.44950516010141 -117.43085641269357) bank6472 +6473 POINT(34.386204876440914 -118.5666045322325) bank6473 +6474 POINT(33.64678770631175 -117.79883652142681) bank6474 +6475 POINT(33.98063347311258 -118.09733665939021) bank6475 +6476 POINT(34.94488599295373 -117.47145121058178) bank6476 +6477 POINT(33.10585773684887 -118.21773707991039) bank6477 +6478 POINT(33.20587599954483 -117.51871424523212) bank6478 +6479 POINT(33.74867925314343 -117.76402624664907) bank6479 +6480 POINT(34.98836642522607 -119.16675016287797) bank6480 +6481 POINT(34.587344662939195 -117.59992766371427) bank6481 +6482 POINT(34.702176742154954 -118.47168733817641) bank6482 +6483 POINT(33.114708194979684 -118.69158586217465) bank6483 +6484 POINT(33.96342369924256 -118.78147787756734) bank6484 +6485 POINT(35.01360062120464 -117.88342905921684) bank6485 +6486 POINT(33.644142480806885 -118.97252964058474) bank6486 +6487 POINT(34.041229374551 -118.58893503457573) bank6487 +6488 POINT(34.56983406119983 -117.41830190605455) bank6488 +6489 POINT(33.51889669258054 -118.60432653784791) bank6489 +6490 POINT(34.551610886484994 -119.0808377033563) bank6490 +6491 POINT(33.76848354036522 -117.71590228712154) bank6491 +6492 POINT(33.64232166905024 -117.7643343217599) bank6492 +6493 POINT(34.53412415735611 -118.91537835515665) bank6493 +6494 POINT(33.53977437052017 -118.65165512558505) bank6494 +6495 POINT(34.863109345888674 -118.33051495087311) bank6495 +6496 POINT(34.01407457313859 -117.4301744520887) bank6496 +6497 POINT(33.624286972469434 -118.65768094757914) bank6497 +6498 POINT(34.263651663934 -119.1031522171997) bank6498 +6499 POINT(34.580252704993846 -118.18340859241732) bank6499 +6500 POINT(33.49162084577915 -118.81596006181562) bank6500 +6501 POINT(34.906505575722406 -118.45711106337023) bank6501 +6502 POINT(34.818057884224935 -118.85820207395717) bank6502 +6503 POINT(34.40977714345284 -118.36502251115824) bank6503 +6504 POINT(33.50186906548351 -117.85052193917656) bank6504 +6505 POINT(34.34508235610874 -117.54342008723381) bank6505 +6506 POINT(33.34217680863752 -118.48382457895731) bank6506 +6507 POINT(33.60052942870606 -117.79790674055631) bank6507 +6508 POINT(34.10895813000803 -119.08057162105236) bank6508 +6509 POINT(33.50487415239002 -117.96469023677987) bank6509 +6510 POINT(34.722095214863515 -119.10908511068793) bank6510 +6511 POINT(33.58202087468564 -117.40115615543866) bank6511 +6512 POINT(34.06640349206735 -118.97112711837948) bank6512 +6513 POINT(34.45911944858101 -117.67406084160808) bank6513 +6514 POINT(34.84223537625407 -117.43087798539734) bank6514 +6515 POINT(33.57651905324579 -118.95470039231213) bank6515 +6516 POINT(34.178933910551585 -117.47586293045832) bank6516 +6517 POINT(33.41989695776762 -119.21283279687874) bank6517 +6518 POINT(33.65722983310273 -118.82661988835206) bank6518 +6519 POINT(34.931926629877644 -118.67411932167366) bank6519 +6520 POINT(34.28597348091113 -117.79349639444877) bank6520 +6521 POINT(33.644252161214446 -118.7741409339157) bank6521 +6522 POINT(34.8101124457802 -118.2445713983263) bank6522 +6523 POINT(33.944302508564256 -118.00268149667188) bank6523 +6524 POINT(33.699385283548395 -117.38369880371003) bank6524 +6525 POINT(33.79689072528295 -117.93893072384226) bank6525 +6526 POINT(33.92832551918871 -118.46686811996325) bank6526 +6527 POINT(33.23114446431806 -118.57272767755312) bank6527 +6528 POINT(33.10539950170285 -118.87077899930127) bank6528 +6529 POINT(33.83074318491728 -118.00610301215212) bank6529 +6530 POINT(34.11800826371855 -118.1120903143936) bank6530 +6531 POINT(34.6854667852953 -117.57731939056374) bank6531 +6532 POINT(34.39661178018293 -117.62858894083274) bank6532 +6533 POINT(34.116978546190566 -118.20312872837927) bank6533 +6534 POINT(34.84242177061861 -117.72016411124211) bank6534 +6535 POINT(34.78758253350437 -118.83237276689489) bank6535 +6536 POINT(33.44444119095862 -117.51062794599895) bank6536 +6537 POINT(33.13078481907059 -118.37389609713622) bank6537 +6538 POINT(33.27034333997206 -118.05731174536007) bank6538 +6539 POINT(33.94775373265356 -118.92693514323602) bank6539 +6540 POINT(35.03162785484396 -118.41459572116446) bank6540 +6541 POINT(33.14087944904129 -118.38181179113005) bank6541 +6542 POINT(34.122488561731934 -118.2095996935208) bank6542 +6543 POINT(33.605926256292776 -118.90771126902233) bank6543 +6544 POINT(33.18329938342689 -117.63229820847619) bank6544 +6545 POINT(33.71376651607936 -118.12179759564066) bank6545 +6546 POINT(34.884865718187676 -117.69386945916457) bank6546 +6547 POINT(34.111957936782325 -118.51207023835758) bank6547 +6548 POINT(33.16204574926096 -118.09708445698223) bank6548 +6549 POINT(34.405169521293075 -117.8725220701915) bank6549 +6550 POINT(34.83015487612114 -119.20521486013325) bank6550 +6551 POINT(33.22433146802626 -117.88606134717122) bank6551 +6552 POINT(34.935570474539446 -118.62461598654872) bank6552 +6553 POINT(34.82748384487047 -118.76342390360988) bank6553 +6554 POINT(33.9966901596683 -117.57290319764842) bank6554 +6555 POINT(34.861493780425576 -117.89036868346363) bank6555 +6556 POINT(33.60516373196538 -117.9823365296048) bank6556 +6557 POINT(33.21724363527234 -117.33756282426802) bank6557 +6558 POINT(34.50545897881742 -119.15696611506058) bank6558 +6559 POINT(33.57907716652445 -118.94266365906864) bank6559 +6560 POINT(34.409360841903066 -118.01235382163878) bank6560 +6561 POINT(34.184373292291085 -117.3782764867241) bank6561 +6562 POINT(34.39986721049366 -118.71935150039126) bank6562 +6563 POINT(34.47705083351227 -117.7350669392013) bank6563 +6564 POINT(33.38791638389115 -117.43737718946828) bank6564 +6565 POINT(33.72660149613482 -118.2206084271598) bank6565 +6566 POINT(33.532830812987626 -117.24604674253422) bank6566 +6567 POINT(33.90926636296177 -117.98634822327321) bank6567 +6568 POINT(33.71634731602367 -117.42397842772078) bank6568 +6569 POINT(33.912701016850484 -117.78547922542721) bank6569 +6570 POINT(34.088125763735555 -117.634265352391) bank6570 +6571 POINT(33.935012052273564 -118.57484631604068) bank6571 +6572 POINT(34.016195456663084 -118.47021472380659) bank6572 +6573 POINT(33.71293696112577 -117.5480549381755) bank6573 +6574 POINT(33.255718781977514 -118.1153188005875) bank6574 +6575 POINT(34.1234569523533 -117.36291709579724) bank6575 +6576 POINT(34.30598106881296 -117.65838943736787) bank6576 +6577 POINT(34.69278300644001 -117.77076236603881) bank6577 +6578 POINT(33.379360952103255 -119.04471473355166) bank6578 +6579 POINT(34.35980510243076 -117.43183817124456) bank6579 +6580 POINT(33.61421979796615 -117.4736207787935) bank6580 +6581 POINT(33.939988334411794 -119.18815475576008) bank6581 +6582 POINT(35.03515266202574 -117.37794194010297) bank6582 +6583 POINT(33.42568243893538 -117.88976573671174) bank6583 +6584 POINT(33.19852865452432 -118.9931792599393) bank6584 +6585 POINT(33.66366829393805 -117.86215949812838) bank6585 +6586 POINT(33.60147183168922 -117.69651633387346) bank6586 +6587 POINT(33.926536347039765 -118.29781138592678) bank6587 +6588 POINT(33.8384771219193 -118.89111722075485) bank6588 +6589 POINT(35.002758331510464 -117.34564636374365) bank6589 +6590 POINT(33.53118433701231 -117.72692676285527) bank6590 +6591 POINT(33.57535837160895 -118.31736292061392) bank6591 +6592 POINT(33.52623704681504 -118.49274843505275) bank6592 +6593 POINT(33.12985888750142 -119.09003041427651) bank6593 +6594 POINT(34.61470761653627 -118.5369942996086) bank6594 +6595 POINT(33.97467727714546 -117.40679431036037) bank6595 +6596 POINT(33.3859982255419 -118.18353243555589) bank6596 +6597 POINT(34.05924636191049 -119.05516470150724) bank6597 +6598 POINT(34.92920102499024 -117.79032627302652) bank6598 +6599 POINT(34.497582930564775 -117.62510520035744) bank6599 +6600 POINT(33.6335361921929 -118.31424108294968) bank6600 +6601 POINT(34.80156131477429 -117.99693575648041) bank6601 +6602 POINT(33.90792613970507 -117.71349467549535) bank6602 +6603 POINT(34.83446309950919 -118.91039967988384) bank6603 +6604 POINT(34.470527033227285 -118.66681438333005) bank6604 +6605 POINT(34.11637666049132 -118.97660902139073) bank6605 +6606 POINT(34.01279847394435 -117.63205953274876) bank6606 +6607 POINT(35.01255249808497 -118.80804107546899) bank6607 +6608 POINT(33.71091669841449 -117.7393485128324) bank6608 +6609 POINT(33.89869598411865 -118.82224608045436) bank6609 +6610 POINT(34.191793934996895 -118.51452201097142) bank6610 +6611 POINT(33.580658559825444 -117.52759090633705) bank6611 +6612 POINT(33.755165537330605 -118.39282118672705) bank6612 +6613 POINT(33.71991739297869 -118.27852553450951) bank6613 +6614 POINT(34.53500891922928 -118.00621840539775) bank6614 +6615 POINT(34.018237202783055 -118.03420964094587) bank6615 +6616 POINT(33.33862079290368 -118.01047874525088) bank6616 +6617 POINT(34.85772786039421 -118.87554297595356) bank6617 +6618 POINT(34.41382656214623 -117.79541824548166) bank6618 +6619 POINT(34.75347276037538 -117.61410048541003) bank6619 +6620 POINT(34.83126647881508 -118.72334223741824) bank6620 +6621 POINT(33.696178718554016 -118.32279347659887) bank6621 +6622 POINT(33.4200724531429 -118.91056997546586) bank6622 +6623 POINT(34.77334135503426 -118.9989959003049) bank6623 +6624 POINT(34.16041803206748 -118.14846900367584) bank6624 +6625 POINT(34.93985358360324 -117.88479314484572) bank6625 +6626 POINT(34.43658948178033 -118.36499811223875) bank6626 +6627 POINT(33.44371819731127 -117.79217412693056) bank6627 +6628 POINT(35.00713677625619 -118.58594317098046) bank6628 +6629 POINT(34.00690316263992 -118.8196919272065) bank6629 +6630 POINT(34.690748375802414 -117.48916365329686) bank6630 +6631 POINT(34.96047544914714 -118.38915032185488) bank6631 +6632 POINT(33.485752533440184 -118.2354521677069) bank6632 +6633 POINT(33.88683060107396 -117.76827529720983) bank6633 +6634 POINT(34.14798218786446 -117.8403734574513) bank6634 +6635 POINT(33.46218769253058 -118.16362099490796) bank6635 +6636 POINT(33.510287222890184 -118.1576672700392) bank6636 +6637 POINT(34.52775293814871 -119.13068026287469) bank6637 +6638 POINT(33.094017481495634 -118.96803986471637) bank6638 +6639 POINT(34.55521247575403 -118.86253161032694) bank6639 +6640 POINT(33.28524716005453 -118.0731006483665) bank6640 +6641 POINT(34.085511362337265 -117.5352214502488) bank6641 +6642 POINT(34.98256251126549 -118.17851738459694) bank6642 +6643 POINT(34.70159143720712 -119.23501150981431) bank6643 +6644 POINT(34.74743928915418 -118.69160159018578) bank6644 +6645 POINT(33.88298003537714 -118.80812512862124) bank6645 +6646 POINT(33.75542459191693 -118.24842129916043) bank6646 +6647 POINT(33.06473457601613 -119.00235154614349) bank6647 +6648 POINT(33.64327603756851 -118.1914259231982) bank6648 +6649 POINT(33.811647359338835 -117.5036003847673) bank6649 +6650 POINT(33.268765455861065 -118.36544869221973) bank6650 +6651 POINT(34.21802222360718 -118.79283044408544) bank6651 +6652 POINT(34.199102722077235 -118.46140850428326) bank6652 +6653 POINT(33.70995506138058 -117.96031963180347) bank6653 +6654 POINT(34.33881863690748 -117.62079366261233) bank6654 +6655 POINT(34.725261577222085 -117.74199860219342) bank6655 +6656 POINT(33.772215204052806 -117.82965520665033) bank6656 +6657 POINT(34.10238187652187 -118.72196750492833) bank6657 +6658 POINT(35.02573396792956 -118.97503283483019) bank6658 +6659 POINT(33.59142623487869 -118.688498280739) bank6659 +6660 POINT(34.18288281703123 -117.6204512656232) bank6660 +6661 POINT(34.93593824980849 -118.72769118057892) bank6661 +6662 POINT(33.914422178746726 -117.73184361835874) bank6662 +6663 POINT(34.68822698012319 -118.6137457100354) bank6663 +6664 POINT(34.82096306271749 -117.308020014368) bank6664 +6665 POINT(33.2812862730243 -118.87901477937025) bank6665 +6666 POINT(33.227712287320585 -117.93275571103118) bank6666 +6667 POINT(34.49413315062638 -117.91795506589952) bank6667 +6668 POINT(34.05097159080784 -118.63371392239064) bank6668 +6669 POINT(33.18768946754798 -118.03924912792337) bank6669 +6670 POINT(33.714962110582924 -117.72926122896446) bank6670 +6671 POINT(33.07421109884107 -118.93819313415638) bank6671 +6672 POINT(34.89466203295073 -117.83609831136873) bank6672 +6673 POINT(33.748245705518976 -117.67689955815904) bank6673 +6674 POINT(34.792112843272186 -117.34066533562877) bank6674 +6675 POINT(34.35392455412957 -119.05696899957323) bank6675 +6676 POINT(34.980414569906046 -118.99102293126502) bank6676 +6677 POINT(34.448447153798604 -117.81693377569954) bank6677 +6678 POINT(33.89152386680197 -118.16122930511176) bank6678 +6679 POINT(33.95480606561555 -118.7211843434401) bank6679 +6680 POINT(34.459885995925546 -117.57062339811623) bank6680 +6681 POINT(34.51011147850811 -117.25141938839141) bank6681 +6682 POINT(34.135096092598964 -117.47515780402433) bank6682 +6683 POINT(34.74426088452196 -117.92743787982316) bank6683 +6684 POINT(33.83953627957016 -117.30159487138425) bank6684 +6685 POINT(34.593143267987 -118.44612070862648) bank6685 +6686 POINT(33.76174220896926 -117.67211323093296) bank6686 +6687 POINT(33.95274682745458 -119.09120014801478) bank6687 +6688 POINT(34.76815465993024 -118.21190840467409) bank6688 +6689 POINT(34.68506085137503 -117.76799821628256) bank6689 +6690 POINT(34.113647695853196 -118.63366276465491) bank6690 +6691 POINT(33.130020863576185 -118.81775502586073) bank6691 +6692 POINT(34.91298338371539 -119.21523392370062) bank6692 +6693 POINT(33.48992469234637 -118.42578209220396) bank6693 +6694 POINT(34.54075205804286 -118.17342255192054) bank6694 +6695 POINT(34.714444015922126 -119.01275397509251) bank6695 +6696 POINT(33.518766043922945 -117.89219271397296) bank6696 +6697 POINT(33.059068036863295 -117.27087879359814) bank6697 +6698 POINT(34.60565713015492 -119.05698406712463) bank6698 +6699 POINT(34.500081639829716 -118.80699295496018) bank6699 +6700 POINT(33.841109504752765 -117.37357030151183) bank6700 +6701 POINT(33.06540996546019 -117.56387333616148) bank6701 +6702 POINT(34.521594869318385 -118.15173538081734) bank6702 +6703 POINT(34.036465034501816 -117.29516418448647) bank6703 +6704 POINT(34.54876618341198 -117.90108715383597) bank6704 +6705 POINT(33.888009039998536 -118.46283167892342) bank6705 +6706 POINT(34.68322546021186 -117.63223873440472) bank6706 +6707 POINT(33.85852130233182 -117.46888657785111) bank6707 +6708 POINT(34.447628194781714 -117.28817389398367) bank6708 +6709 POINT(34.89388778310595 -118.57888983099537) bank6709 +6710 POINT(33.317942229590464 -118.43646289569706) bank6710 +6711 POINT(34.708180014731695 -118.83883050045914) bank6711 +6712 POINT(34.38087609855916 -118.92406596823392) bank6712 +6713 POINT(33.701961607158296 -117.86412929381093) bank6713 +6714 POINT(33.33632916449313 -117.98001178812012) bank6714 +6715 POINT(34.587561851113456 -118.05484050161851) bank6715 +6716 POINT(33.94751112544595 -118.24306618305742) bank6716 +6717 POINT(33.81316665769754 -117.93369459017266) bank6717 +6718 POINT(33.79573777355639 -118.4501298761615) bank6718 +6719 POINT(34.2102665941421 -118.83019282988077) bank6719 +6720 POINT(33.867623271503874 -118.1258452897859) bank6720 +6721 POINT(34.018473584141155 -119.11842447633353) bank6721 +6722 POINT(33.851082132546814 -119.01237811861571) bank6722 +6723 POINT(34.56719270141402 -117.4051778452106) bank6723 +6724 POINT(33.13697049430311 -117.91113766784751) bank6724 +6725 POINT(34.07506549490793 -117.88619271392623) bank6725 +6726 POINT(33.96648660483417 -117.4547247851652) bank6726 +6727 POINT(34.66096259667354 -117.64239087547844) bank6727 +6728 POINT(33.47314838746861 -118.2224156367314) bank6728 +6729 POINT(33.72712460179795 -117.39988629083857) bank6729 +6730 POINT(34.40465357047137 -118.02620992425138) bank6730 +6731 POINT(33.48170804096266 -119.16022104949995) bank6731 +6732 POINT(34.691206316768394 -117.96784645216792) bank6732 +6733 POINT(35.00712629800683 -118.10869483141936) bank6733 +6734 POINT(33.99410288119765 -117.63773951971424) bank6734 +6735 POINT(33.08249126240096 -118.68937038372391) bank6735 +6736 POINT(33.312782713253114 -117.65741958996963) bank6736 +6737 POINT(35.026183384209 -117.99598282414675) bank6737 +6738 POINT(33.385521739787876 -118.65490421291642) bank6738 +6739 POINT(34.876350505692024 -117.9773081852701) bank6739 +6740 POINT(34.57159686088877 -119.1459255785046) bank6740 +6741 POINT(33.894858950350915 -117.63226900894476) bank6741 +6742 POINT(35.00589753373443 -119.17718498561199) bank6742 +6743 POINT(34.81100106533707 -117.5888136166052) bank6743 +6744 POINT(34.255162273376015 -117.66338096643545) bank6744 +6745 POINT(34.69381603645615 -118.49493395913085) bank6745 +6746 POINT(33.225431936202625 -118.50494453394022) bank6746 +6747 POINT(34.865680763130676 -117.47070306615794) bank6747 +6748 POINT(34.09998137827972 -118.70230126258777) bank6748 +6749 POINT(34.41151035053548 -118.9156964425756) bank6749 +6750 POINT(33.51535743839239 -117.6959726716942) bank6750 +6751 POINT(34.48109163847295 -118.39699591146412) bank6751 +6752 POINT(34.24941577421795 -119.09053239307666) bank6752 +6753 POINT(34.82939116323878 -118.58079179432306) bank6753 +6754 POINT(34.412514536837776 -117.35338413672974) bank6754 +6755 POINT(34.558459619129025 -118.90997540778241) bank6755 +6756 POINT(35.02566621758856 -118.86916923560284) bank6756 +6757 POINT(34.80548081700855 -117.63134397570458) bank6757 +6758 POINT(34.043696321454334 -118.23073588327794) bank6758 +6759 POINT(34.218157257072434 -118.38485657815349) bank6759 +6760 POINT(35.02414350435493 -118.99618971524369) bank6760 +6761 POINT(34.42319346296052 -118.43785373658679) bank6761 +6762 POINT(34.841244650342745 -118.43912822827978) bank6762 +6763 POINT(33.9957955217379 -118.62162802348925) bank6763 +6764 POINT(33.519044727439706 -117.91831954712075) bank6764 +6765 POINT(33.9661676221292 -117.67736809786662) bank6765 +6766 POINT(33.69739801353368 -117.27414439613909) bank6766 +6767 POINT(33.848219390906216 -118.31020697481148) bank6767 +6768 POINT(33.266451640077804 -117.41155988249213) bank6768 +6769 POINT(33.44040449490714 -119.06220932048844) bank6769 +6770 POINT(34.300429823785485 -118.22057794382235) bank6770 +6771 POINT(34.87969776625636 -117.77047589918789) bank6771 +6772 POINT(34.92750076646284 -118.82253487473913) bank6772 +6773 POINT(33.18316186742436 -118.63919937651518) bank6773 +6774 POINT(33.65987674476453 -118.17273591653966) bank6774 +6775 POINT(33.578044279564274 -119.0962604197052) bank6775 +6776 POINT(33.07908762423444 -117.43042632013305) bank6776 +6777 POINT(34.28816652027144 -118.5112142406561) bank6777 +6778 POINT(33.53994560822523 -117.30301255025151) bank6778 +6779 POINT(34.03132064952595 -117.72179627461833) bank6779 +6780 POINT(34.474585866705986 -117.45271397264898) bank6780 +6781 POINT(34.44946657963032 -117.28483619819912) bank6781 +6782 POINT(34.13474145627729 -118.911018128267) bank6782 +6783 POINT(34.432161556074554 -117.35971266552754) bank6783 +6784 POINT(34.26665187754505 -118.62389858259614) bank6784 +6785 POINT(33.80997789852291 -118.78380794490498) bank6785 +6786 POINT(34.93303204729558 -119.14956138380887) bank6786 +6787 POINT(34.75638991873607 -117.52664066012112) bank6787 +6788 POINT(33.569803499487165 -117.26767385929391) bank6788 +6789 POINT(33.20088851164799 -117.54091731124245) bank6789 +6790 POINT(34.18498522915109 -117.66032180862412) bank6790 +6791 POINT(34.34187147941887 -118.54001950760127) bank6791 +6792 POINT(34.148114564511985 -118.03605486322034) bank6792 +6793 POINT(33.93644168130035 -119.10859953295353) bank6793 +6794 POINT(34.09876279159246 -118.08430418104146) bank6794 +6795 POINT(33.0995973109905 -118.80104871572007) bank6795 +6796 POINT(34.554530199796645 -117.79536700088912) bank6796 +6797 POINT(33.292583278467426 -118.77121907948079) bank6797 +6798 POINT(33.522932373785316 -117.9453884806836) bank6798 +6799 POINT(34.17803689799996 -117.66469449136869) bank6799 +6800 POINT(34.52840160379869 -118.37828379615966) bank6800 +6801 POINT(33.32416469500083 -117.70914571720861) bank6801 +6802 POINT(34.85164895578403 -117.9319392374124) bank6802 +6803 POINT(34.80295256797168 -117.72624372051395) bank6803 +6804 POINT(33.9947293739065 -117.28569096157605) bank6804 +6805 POINT(33.12352893225721 -117.99160183001163) bank6805 +6806 POINT(34.765591307280566 -118.46899875375026) bank6806 +6807 POINT(33.951195688735034 -117.5883134824588) bank6807 +6808 POINT(33.933741693353355 -117.87184450898506) bank6808 +6809 POINT(33.128474429285056 -117.91991691406515) bank6809 +6810 POINT(34.90371429066892 -118.52633539177845) bank6810 +6811 POINT(33.643071256201175 -118.36739799961487) bank6811 +6812 POINT(33.621589901780816 -118.95085652775221) bank6812 +6813 POINT(35.01059853106936 -118.2633878280586) bank6813 +6814 POINT(34.67508104969253 -117.79292628900043) bank6814 +6815 POINT(33.34677348996956 -117.6051416682954) bank6815 +6816 POINT(34.83249076426262 -118.46071242912927) bank6816 +6817 POINT(33.39702586161456 -118.82156812497236) bank6817 +6818 POINT(34.30043971700273 -117.74365408346546) bank6818 +6819 POINT(34.212637860560605 -117.80769220728608) bank6819 +6820 POINT(33.11274590019777 -118.8508916910891) bank6820 +6821 POINT(34.85190433376868 -117.74085719274294) bank6821 +6822 POINT(34.84671587331186 -118.4060211216649) bank6822 +6823 POINT(33.60593351688913 -119.2288133514985) bank6823 +6824 POINT(33.78713860444367 -118.95742369646896) bank6824 +6825 POINT(34.213058972256206 -117.96169844109157) bank6825 +6826 POINT(34.7253050243131 -118.67169353950148) bank6826 +6827 POINT(33.09940461121925 -117.30028354254117) bank6827 +6828 POINT(33.785038411113504 -118.27368638565461) bank6828 +6829 POINT(34.36441349290849 -117.3241899782926) bank6829 +6830 POINT(34.99550539435546 -118.43928149783687) bank6830 +6831 POINT(33.82175352893698 -118.91353920852484) bank6831 +6832 POINT(34.74545429237089 -118.43429916645168) bank6832 +6833 POINT(34.463884661436296 -117.57481951315671) bank6833 +6834 POINT(34.68067345418348 -118.09672887955051) bank6834 +6835 POINT(34.24616243886509 -118.63860724789906) bank6835 +6836 POINT(33.84289606542638 -117.92182318135401) bank6836 +6837 POINT(34.458094909933436 -118.00113834473301) bank6837 +6838 POINT(34.89705368279834 -118.28509547601205) bank6838 +6839 POINT(33.86970138590543 -117.38445028133515) bank6839 +6840 POINT(34.22057363740664 -118.62565616675899) bank6840 +6841 POINT(33.981358681008736 -117.51943303425891) bank6841 +6842 POINT(34.15670836414903 -118.49378856197056) bank6842 +6843 POINT(33.69669687249644 -118.37449299625516) bank6843 +6844 POINT(33.41311469011432 -118.63413531356956) bank6844 +6845 POINT(33.078465452796834 -117.50618791979024) bank6845 +6846 POINT(33.67107489808959 -118.81294573127573) bank6846 +6847 POINT(33.93766654409342 -118.3183236046117) bank6847 +6848 POINT(34.17513296232898 -118.72673706025938) bank6848 +6849 POINT(33.074802822311646 -117.50248170040386) bank6849 +6850 POINT(35.02503236133261 -118.51327224499846) bank6850 +6851 POINT(34.671618174807364 -118.70847750961671) bank6851 +6852 POINT(33.895284290548034 -119.13655893699247) bank6852 +6853 POINT(34.67000397658077 -119.0839054533493) bank6853 +6854 POINT(33.95251547713827 -118.72206104689675) bank6854 +6855 POINT(33.51698330592633 -118.04279572601118) bank6855 +6856 POINT(34.19345470595482 -119.08625013658346) bank6856 +6857 POINT(34.97380330182361 -118.17631618260612) bank6857 +6858 POINT(34.69444572859244 -118.72569087674846) bank6858 +6859 POINT(34.32878075339475 -117.70459921202583) bank6859 +6860 POINT(34.46500819139056 -117.2860382793888) bank6860 +6861 POINT(34.72132542840728 -119.21225870286224) bank6861 +6862 POINT(34.33367616400346 -117.47981630456972) bank6862 +6863 POINT(33.14011381521173 -118.06457626931362) bank6863 +6864 POINT(34.26809456633812 -117.3790141465769) bank6864 +6865 POINT(34.421905201036544 -117.76579881762933) bank6865 +6866 POINT(34.23454911230145 -119.20851104460158) bank6866 +6867 POINT(34.758748492222644 -117.89157384319479) bank6867 +6868 POINT(34.92115337763501 -118.3779409098431) bank6868 +6869 POINT(33.77302958940625 -118.71441203666554) bank6869 +6870 POINT(33.13310350632119 -118.55275680521817) bank6870 +6871 POINT(34.65276075885162 -118.10275555720281) bank6871 +6872 POINT(34.179486375669896 -118.66551425556332) bank6872 +6873 POINT(34.01763237812281 -118.61261873069255) bank6873 +6874 POINT(33.38154887042576 -117.92226381328123) bank6874 +6875 POINT(33.71826905984866 -117.431877222138) bank6875 +6876 POINT(34.22427960866896 -119.1782597848311) bank6876 +6877 POINT(33.708301821604486 -117.53223820599436) bank6877 +6878 POINT(33.49645165785976 -117.65524310120705) bank6878 +6879 POINT(33.09777400000133 -118.67232990281367) bank6879 +6880 POINT(34.862419540595475 -118.1272846402848) bank6880 +6881 POINT(34.41519119343458 -117.64687806494014) bank6881 +6882 POINT(34.18452859077196 -117.69582822800383) bank6882 +6883 POINT(33.05378507801007 -118.30068213072201) bank6883 +6884 POINT(33.691174245719175 -118.6615609855427) bank6884 +6885 POINT(34.27167992090875 -118.45909842692089) bank6885 +6886 POINT(34.67017106881131 -118.0938737440776) bank6886 +6887 POINT(34.164237337056015 -118.7238129591857) bank6887 +6888 POINT(34.604057052083604 -117.69850061988271) bank6888 +6889 POINT(33.31923863924468 -119.21796589371779) bank6889 +6890 POINT(34.145576454631325 -117.7155302810072) bank6890 +6891 POINT(34.206500831066585 -118.87620839382468) bank6891 +6892 POINT(34.02950811144834 -118.9056312060174) bank6892 +6893 POINT(34.87427206738838 -118.83739602859902) bank6893 +6894 POINT(33.38999124755146 -117.96027795559131) bank6894 +6895 POINT(33.08599334603386 -119.04653539487846) bank6895 +6896 POINT(33.296564268559614 -117.78118425598475) bank6896 +6897 POINT(34.50748143551956 -117.90039708096285) bank6897 +6898 POINT(33.23026940631 -118.30674744313102) bank6898 +6899 POINT(34.05660923853285 -119.16018544554431) bank6899 +6900 POINT(34.794190666487985 -118.60684556999361) bank6900 +6901 POINT(34.620469201968554 -118.92297515835163) bank6901 +6902 POINT(33.71591088498146 -118.6373074613594) bank6902 +6903 POINT(33.53088207078567 -118.08836380884537) bank6903 +6904 POINT(34.93716811068553 -118.40657557409227) bank6904 +6905 POINT(33.29408991868132 -118.51573978750321) bank6905 +6906 POINT(33.764706169193154 -118.80419700566677) bank6906 +6907 POINT(34.70414369631502 -118.02527270459784) bank6907 +6908 POINT(34.14003421163416 -118.45587705085309) bank6908 +6909 POINT(33.35281075851678 -119.01109452989947) bank6909 +6910 POINT(33.29013998046613 -117.69341421932417) bank6910 +6911 POINT(33.35235743149974 -119.2361472310408) bank6911 +6912 POINT(35.040631459450246 -117.80294317109407) bank6912 +6913 POINT(33.585563446402766 -118.5888907264388) bank6913 +6914 POINT(33.93274933803216 -118.07904152139909) bank6914 +6915 POINT(34.30865981630005 -118.36844058508869) bank6915 +6916 POINT(35.00779869627074 -118.43934994614852) bank6916 +6917 POINT(34.33186494695899 -117.99149441509917) bank6917 +6918 POINT(34.27134889269519 -119.19751232151988) bank6918 +6919 POINT(34.040489944700006 -118.75354331647345) bank6919 +6920 POINT(34.03743780924024 -117.90592002058098) bank6920 +6921 POINT(33.7278542866583 -118.6922765306979) bank6921 +6922 POINT(34.37054617910636 -117.55913406589588) bank6922 +6923 POINT(33.2475071695015 -117.33507432290294) bank6923 +6924 POINT(33.46197864938163 -118.28524596291412) bank6924 +6925 POINT(34.606816888906124 -118.91891845915339) bank6925 +6926 POINT(33.36328149123023 -117.34895947401964) bank6926 +6927 POINT(33.91896708270207 -117.98986910976228) bank6927 +6928 POINT(33.87912722062571 -118.76644585667508) bank6928 +6929 POINT(34.068990957806875 -118.51506898470599) bank6929 +6930 POINT(34.50906354477485 -119.02272610381985) bank6930 +6931 POINT(33.771661510573956 -117.31879342846199) bank6931 +6932 POINT(33.06332835791688 -118.28126877045891) bank6932 +6933 POINT(33.845869571734106 -118.35597755038378) bank6933 +6934 POINT(33.710977305330886 -117.84140417936308) bank6934 +6935 POINT(34.57246229470039 -118.52562060822791) bank6935 +6936 POINT(33.54011360723002 -117.52859842327051) bank6936 +6937 POINT(34.85485083206162 -118.10239869583273) bank6937 +6938 POINT(34.691438962897166 -118.28704458228746) bank6938 +6939 POINT(35.01965303653823 -117.72429703291678) bank6939 +6940 POINT(33.91811501385691 -117.32588988280102) bank6940 +6941 POINT(34.19312850517583 -118.61402280430512) bank6941 +6942 POINT(34.84525749691787 -118.40385844069843) bank6942 +6943 POINT(33.32595223029966 -118.81796529222957) bank6943 +6944 POINT(34.59472073476324 -117.43361698947508) bank6944 +6945 POINT(34.91223037763304 -117.71717588959012) bank6945 +6946 POINT(34.11939984721355 -118.76937608765995) bank6946 +6947 POINT(34.53401605776027 -118.91572692859006) bank6947 +6948 POINT(34.64294079184812 -117.44506319834159) bank6948 +6949 POINT(34.75131392409728 -117.54028586271889) bank6949 +6950 POINT(33.27221971830777 -117.74200730705908) bank6950 +6951 POINT(33.45856216614361 -117.57621532605623) bank6951 +6952 POINT(33.26271402447012 -119.01114586672094) bank6952 +6953 POINT(33.89560134122261 -117.89403549287951) bank6953 +6954 POINT(34.99734053983547 -118.89612086686765) bank6954 +6955 POINT(33.44031584375329 -118.6496335958971) bank6955 +6956 POINT(34.124349847870285 -118.08405824778785) bank6956 +6957 POINT(35.03722927395522 -118.98972545384484) bank6957 +6958 POINT(33.876302795421786 -117.79929727206306) bank6958 +6959 POINT(34.38932719098552 -117.88155923304946) bank6959 +6960 POINT(34.512945021955034 -118.49894052018017) bank6960 +6961 POINT(33.492120222464614 -119.20915575443385) bank6961 +6962 POINT(33.769316295785664 -117.55059522102746) bank6962 +6963 POINT(34.8405416944536 -118.6437523113069) bank6963 +6964 POINT(33.61234041433031 -118.27464526080361) bank6964 +6965 POINT(33.379579316389815 -118.04359099481182) bank6965 +6966 POINT(33.58467971372489 -118.65598513029832) bank6966 +6967 POINT(33.44560403956727 -118.75831603008659) bank6967 +6968 POINT(33.4195495213699 -117.40491903447952) bank6968 +6969 POINT(33.62031523446719 -118.64151204557554) bank6969 +6970 POINT(33.66688436697123 -119.17203923131014) bank6970 +6971 POINT(33.921748129220276 -118.42422180829402) bank6971 +6972 POINT(33.93037328424917 -118.53430210663711) bank6972 +6973 POINT(34.9133445881353 -118.78512677883509) bank6973 +6974 POINT(34.60891905095529 -118.30066584330258) bank6974 +6975 POINT(33.74744711655584 -117.37896665596084) bank6975 +6976 POINT(33.68561602667941 -118.40967946085735) bank6976 +6977 POINT(33.29281408621575 -119.15264902047495) bank6977 +6978 POINT(33.29695575456792 -117.87334302983759) bank6978 +6979 POINT(34.237008937356634 -118.11665394162723) bank6979 +6980 POINT(34.90628648112372 -117.49351500654423) bank6980 +6981 POINT(34.03457767265515 -118.6738404806738) bank6981 +6982 POINT(33.42712225298795 -117.41515240855546) bank6982 +6983 POINT(34.9596455331361 -117.32605559933236) bank6983 +6984 POINT(34.147956204829924 -117.36346646925202) bank6984 +6985 POINT(33.61856703389491 -119.00257776365738) bank6985 +6986 POINT(33.51656414016358 -118.93745697588385) bank6986 +6987 POINT(34.43208429251078 -117.5426030341664) bank6987 +6988 POINT(33.40470989983889 -118.29643411521002) bank6988 +6989 POINT(34.677713740816245 -117.3339234045187) bank6989 +6990 POINT(34.61906940131644 -118.72103742307304) bank6990 +6991 POINT(33.836658191617104 -118.22837989917913) bank6991 +6992 POINT(33.78825929178013 -118.227066204431) bank6992 +6993 POINT(33.34724849912268 -118.92486765445076) bank6993 +6994 POINT(34.715646250569094 -118.4844431812165) bank6994 +6995 POINT(33.8384549610564 -118.16403661228183) bank6995 +6996 POINT(34.22876834358328 -117.26002221001723) bank6996 +6997 POINT(34.30994920241189 -118.73881306605277) bank6997 +6998 POINT(34.748611737095814 -117.49298516412611) bank6998 +6999 POINT(33.478266110261366 -119.18067905365398) bank6999 +7000 POINT(35.01965817660634 -117.91975360361927) bank7000 +7001 POINT(34.366848643001944 -117.4348864168473) bank7001 +7002 POINT(33.15632860912453 -117.31083123256442) bank7002 +7003 POINT(34.650906623027936 -118.0887827239718) bank7003 +7004 POINT(34.35194789732445 -118.07995907471633) bank7004 +7005 POINT(33.33778423843212 -118.6787867935794) bank7005 +7006 POINT(33.314864765051006 -117.94742883092673) bank7006 +7007 POINT(34.76590964877308 -118.93818116336453) bank7007 +7008 POINT(34.90127940602171 -117.42923272885767) bank7008 +7009 POINT(34.59101127124223 -118.12811154743413) bank7009 +7010 POINT(34.226301604217994 -119.21692781444762) bank7010 +7011 POINT(34.859682148984334 -118.55615950755431) bank7011 +7012 POINT(34.46097290694403 -118.83753449165037) bank7012 +7013 POINT(34.504327962121245 -118.25912992935957) bank7013 +7014 POINT(34.44677790679003 -118.5342378541541) bank7014 +7015 POINT(33.34150638682151 -118.79855878608556) bank7015 +7016 POINT(33.65313351967155 -119.23870028099597) bank7016 +7017 POINT(35.03213617913129 -117.56552449071319) bank7017 +7018 POINT(34.05402692922581 -118.29350149972721) bank7018 +7019 POINT(34.23384452018236 -117.34662610527737) bank7019 +7020 POINT(33.176728086324424 -118.20038215222613) bank7020 +7021 POINT(33.56705477165415 -118.9064664318255) bank7021 +7022 POINT(34.75096842861794 -118.40838958705207) bank7022 +7023 POINT(34.93730425487146 -118.63910532293538) bank7023 +7024 POINT(33.70514216312933 -118.3599089981319) bank7024 +7025 POINT(33.40917216163831 -117.84525778677447) bank7025 +7026 POINT(34.40064372893993 -118.82056034597764) bank7026 +7027 POINT(34.26417107872187 -117.71802654240601) bank7027 +7028 POINT(34.18206472308894 -117.54959345802018) bank7028 +7029 POINT(33.95621634366719 -117.96417814027387) bank7029 +7030 POINT(35.01897981650618 -117.59388277952421) bank7030 +7031 POINT(33.879726015930004 -117.68601718652369) bank7031 +7032 POINT(33.56341789797811 -117.89637785804771) bank7032 +7033 POINT(34.94108654290316 -118.17214076785213) bank7033 +7034 POINT(33.955812128374745 -118.97540970405659) bank7034 +7035 POINT(33.873097809359834 -118.6008908104439) bank7035 +7036 POINT(33.69582851275837 -118.65422052241007) bank7036 +7037 POINT(34.831397251692835 -117.75573538541641) bank7037 +7038 POINT(33.17936981869576 -117.70693350622227) bank7038 +7039 POINT(33.22744097630782 -117.5465833819003) bank7039 +7040 POINT(33.903501230467675 -117.6867182958359) bank7040 +7041 POINT(34.138687579647915 -117.68627416156151) bank7041 +7042 POINT(34.38411672249258 -119.02949673615197) bank7042 +7043 POINT(33.24756200084322 -118.50154189167846) bank7043 +7044 POINT(34.3415279599621 -118.31334552472819) bank7044 +7045 POINT(33.59162235504091 -118.1426628656336) bank7045 +7046 POINT(33.35397114276422 -119.22434351724549) bank7046 +7047 POINT(34.05826913006838 -118.6957342010465) bank7047 +7048 POINT(34.51322823134426 -117.40395596007234) bank7048 +7049 POINT(34.37662189413573 -119.22726238281993) bank7049 +7050 POINT(34.88532350853395 -117.45353376988363) bank7050 +7051 POINT(33.35502732858881 -118.03843994409517) bank7051 +7052 POINT(34.35983190434341 -118.35725900236682) bank7052 +7053 POINT(33.9437312579252 -119.01199898517346) bank7053 +7054 POINT(34.379819044053065 -118.54406122873311) bank7054 +7055 POINT(35.0329204742365 -118.72960139908898) bank7055 +7056 POINT(34.315206608209046 -118.65690781971634) bank7056 +7057 POINT(34.01548093361236 -119.10724413063186) bank7057 +7058 POINT(34.22862625176023 -119.03630844309939) bank7058 +7059 POINT(34.07850076259014 -117.42078012059737) bank7059 +7060 POINT(34.75141042848247 -117.86834724061262) bank7060 +7061 POINT(34.20630572473248 -118.65477494895977) bank7061 +7062 POINT(34.256579759905776 -117.97464432686627) bank7062 +7063 POINT(34.346391657524045 -118.52677301766246) bank7063 +7064 POINT(34.26422426034314 -118.66135921207214) bank7064 +7065 POINT(34.343192589271624 -118.91429600124633) bank7065 +7066 POINT(34.25117466370282 -118.91593292933159) bank7066 +7067 POINT(34.387744453134 -119.01417630800974) bank7067 +7068 POINT(34.208182771513 -117.50216046336747) bank7068 +7069 POINT(33.69650668145581 -117.33084395465292) bank7069 +7070 POINT(34.581782626967275 -117.82083401786248) bank7070 +7071 POINT(33.71609627968439 -117.40642839844388) bank7071 +7072 POINT(33.84692460777427 -117.61083155965392) bank7072 +7073 POINT(33.29277657498397 -117.34447254272469) bank7073 +7074 POINT(34.41867468038011 -118.5094564732405) bank7074 +7075 POINT(34.60997329589678 -118.0386916688376) bank7075 +7076 POINT(33.6277487365295 -118.51100382750063) bank7076 +7077 POINT(33.098051323637165 -118.80860162834179) bank7077 +7078 POINT(34.911188553760056 -118.99982161339462) bank7078 +7079 POINT(34.37979917053339 -117.93284524999416) bank7079 +7080 POINT(34.39816795201348 -119.09773488012905) bank7080 +7081 POINT(34.092930091388624 -119.08663563304775) bank7081 +7082 POINT(33.38985931516013 -117.8310526874577) bank7082 +7083 POINT(34.00825590524045 -118.67050209601837) bank7083 +7084 POINT(34.1556766360994 -117.82182710260359) bank7084 +7085 POINT(33.92532292901076 -117.38053930782874) bank7085 +7086 POINT(34.86176261796136 -117.56549247295233) bank7086 +7087 POINT(33.80206588249314 -117.96897227284977) bank7087 +7088 POINT(33.087067302121646 -117.92049123438335) bank7088 +7089 POINT(33.402961228205776 -118.18389528729182) bank7089 +7090 POINT(33.99914929496351 -117.24597422441877) bank7090 +7091 POINT(34.31836738121068 -117.59738502590768) bank7091 +7092 POINT(34.071801220880424 -118.48622058444357) bank7092 +7093 POINT(34.28275644662429 -117.47015483140366) bank7093 +7094 POINT(34.12451446903162 -117.55400160334673) bank7094 +7095 POINT(34.28856153009907 -117.4465808425211) bank7095 +7096 POINT(34.93986463731791 -118.74419825930701) bank7096 +7097 POINT(34.03524091918541 -118.40426551831528) bank7097 +7098 POINT(33.8955140376751 -117.81287872228364) bank7098 +7099 POINT(34.384112583865225 -118.83939901748361) bank7099 +7100 POINT(33.77167232018719 -118.25573189057722) bank7100 +7101 POINT(33.310074873517344 -118.76235241558508) bank7101 +7102 POINT(34.589103270521775 -117.70051920399446) bank7102 +7103 POINT(34.165780293612706 -117.95272904032335) bank7103 +7104 POINT(34.24445115529645 -118.23617237918263) bank7104 +7105 POINT(34.16753177906459 -118.37067401043704) bank7105 +7106 POINT(34.96368584908537 -118.85586886714792) bank7106 +7107 POINT(34.49807396686321 -117.30245304549739) bank7107 +7108 POINT(33.16666846765101 -118.46297260986915) bank7108 +7109 POINT(34.6687973317542 -118.28102252358755) bank7109 +7110 POINT(34.36918275336205 -117.270830553762) bank7110 +7111 POINT(34.22731668861211 -119.16478193172517) bank7111 +7112 POINT(34.04216491392081 -118.9104024000187) bank7112 +7113 POINT(34.54212026173525 -117.41812528087411) bank7113 +7114 POINT(34.827375639574015 -118.63150336152148) bank7114 +7115 POINT(33.768623276116266 -119.21735172648842) bank7115 +7116 POINT(34.88534461359041 -117.82902127830408) bank7116 +7117 POINT(34.64480467290571 -119.08943331489856) bank7117 +7118 POINT(34.18548367607277 -118.16643826003525) bank7118 +7119 POINT(34.96287843201311 -118.99128988983267) bank7119 +7120 POINT(33.873585620641926 -118.49379390274808) bank7120 +7121 POINT(34.362814588936104 -117.78208457546869) bank7121 +7122 POINT(33.99302459164937 -118.33068801603265) bank7122 +7123 POINT(33.339654653118814 -118.19872419849762) bank7123 +7124 POINT(33.71109496038138 -117.34114280293443) bank7124 +7125 POINT(34.54879235957613 -117.57522448160786) bank7125 +7126 POINT(33.34922997116359 -119.24282048168942) bank7126 +7127 POINT(33.3430846742306 -117.49622763236404) bank7127 +7128 POINT(34.24647214019391 -118.89878640141367) bank7128 +7129 POINT(34.84001306376246 -117.64205998958293) bank7129 +7130 POINT(34.140787353365525 -117.44586071292812) bank7130 +7131 POINT(33.79182693340985 -118.19317404964642) bank7131 +7132 POINT(34.87918418116609 -117.63705348601539) bank7132 +7133 POINT(34.32265685998196 -119.03731778256993) bank7133 +7134 POINT(34.12778237356866 -118.37784867288663) bank7134 +7135 POINT(33.56937615086488 -118.06869302107155) bank7135 +7136 POINT(33.71951623354521 -118.45426426102951) bank7136 +7137 POINT(33.46761008372307 -118.50723840759225) bank7137 +7138 POINT(33.95755371429897 -118.92358091819109) bank7138 +7139 POINT(33.36483678954001 -118.54548638725561) bank7139 +7140 POINT(33.594884027640894 -117.41585948300431) bank7140 +7141 POINT(34.08324171363553 -118.8213417332103) bank7141 +7142 POINT(34.15804140143213 -117.58210925265335) bank7142 +7143 POINT(33.151962151980214 -118.79550416690111) bank7143 +7144 POINT(34.80384446984431 -118.13863861019409) bank7144 +7145 POINT(34.28821627658758 -117.9604557191439) bank7145 +7146 POINT(33.09923609699837 -117.87398108374796) bank7146 +7147 POINT(33.148073424670955 -118.44533325864977) bank7147 +7148 POINT(33.91834321131905 -117.26286492177817) bank7148 +7149 POINT(34.34162657190423 -119.18910540446602) bank7149 +7150 POINT(34.03581071504754 -118.8564949724828) bank7150 +7151 POINT(34.9619572044027 -117.82728942074583) bank7151 +7152 POINT(34.77783952860704 -117.47537890705837) bank7152 +7153 POINT(35.05175429885144 -118.03683184744224) bank7153 +7154 POINT(33.860079973547904 -117.62009140930222) bank7154 +7155 POINT(33.508761707886826 -117.96550237676287) bank7155 +7156 POINT(33.75776150910781 -119.05108134768071) bank7156 +7157 POINT(34.67612791273334 -119.0388726932087) bank7157 +7158 POINT(34.300074820900626 -117.27884778156094) bank7158 +7159 POINT(33.89200853978138 -118.12579465216503) bank7159 +7160 POINT(34.33506849326366 -119.23585536811675) bank7160 +7161 POINT(34.11747856882902 -117.46102977391571) bank7161 +7162 POINT(33.20757079175783 -117.35840381895107) bank7162 +7163 POINT(33.46696992771682 -118.52208283173438) bank7163 +7164 POINT(34.66519987369652 -118.78289720538729) bank7164 +7165 POINT(34.4997391872419 -117.29518975262987) bank7165 +7166 POINT(34.50163401681025 -117.59888353624774) bank7166 +7167 POINT(34.07935452442881 -118.67793795035946) bank7167 +7168 POINT(34.27307094878021 -117.77323483367938) bank7168 +7169 POINT(34.72626633428004 -119.2152250496142) bank7169 +7170 POINT(34.470494952016374 -118.78418294167162) bank7170 +7171 POINT(34.33501525162809 -117.95138686607024) bank7171 +7172 POINT(34.00323126003878 -117.37739030027373) bank7172 +7173 POINT(34.2146573099665 -117.82994534312323) bank7173 +7174 POINT(33.97789266706435 -117.64776030214533) bank7174 +7175 POINT(33.949440161238115 -117.84085798583732) bank7175 +7176 POINT(33.820193389504645 -118.63910561820569) bank7176 +7177 POINT(33.9025012516198 -118.93623152064777) bank7177 +7178 POINT(33.180835605473845 -118.32389866013155) bank7178 +7179 POINT(33.727179333349056 -117.68137144022009) bank7179 +7180 POINT(34.70410590579186 -118.63553285745637) bank7180 +7181 POINT(34.61674505025419 -118.12994172906238) bank7181 +7182 POINT(33.9299174880535 -118.4334136740688) bank7182 +7183 POINT(33.303140280250766 -117.52141778115353) bank7183 +7184 POINT(33.31020276214418 -118.6387112089661) bank7184 +7185 POINT(33.957145854122366 -119.11756249440592) bank7185 +7186 POINT(33.961046426788926 -118.6874850320011) bank7186 +7187 POINT(33.88112517406097 -118.30117279844758) bank7187 +7188 POINT(34.22902162298748 -119.1460780319833) bank7188 +7189 POINT(35.02949470096851 -119.0960939814088) bank7189 +7190 POINT(34.75168433867603 -118.36687905921275) bank7190 +7191 POINT(34.73078301179577 -117.98225339212023) bank7191 +7192 POINT(34.09155052160054 -117.46609426811891) bank7192 +7193 POINT(34.36658958918529 -117.96090171077182) bank7193 +7194 POINT(33.23412322975684 -117.47930849881041) bank7194 +7195 POINT(34.32301991394384 -117.83817627129444) bank7195 +7196 POINT(33.11572980875078 -117.5719586947404) bank7196 +7197 POINT(33.9001365889498 -118.24589959800724) bank7197 +7198 POINT(33.92905256579653 -118.66038865956587) bank7198 +7199 POINT(34.562168734547036 -117.24921619988395) bank7199 +7200 POINT(33.11716147356982 -118.70087255660822) bank7200 +7201 POINT(33.627094003329 -118.44912401637298) bank7201 +7202 POINT(33.96433399464023 -118.17726909982947) bank7202 +7203 POINT(33.38351308138097 -118.29527781775833) bank7203 +7204 POINT(34.97784399067189 -118.6609282561952) bank7204 +7205 POINT(34.11950956877739 -119.101370537202) bank7205 +7206 POINT(34.96262069386813 -118.67618235135559) bank7206 +7207 POINT(33.83486965504014 -117.45861688056809) bank7207 +7208 POINT(33.35533169641988 -117.87204237150856) bank7208 +7209 POINT(34.24392083242666 -118.19925034778015) bank7209 +7210 POINT(33.507001603919775 -118.53569722052582) bank7210 +7211 POINT(33.97046284463694 -118.05064861035224) bank7211 +7212 POINT(33.22609930284767 -118.58814543383652) bank7212 +7213 POINT(35.01241704284064 -118.22402329028638) bank7213 +7214 POINT(34.05226883407764 -117.58983948788538) bank7214 +7215 POINT(33.867615382921365 -117.66663548015416) bank7215 +7216 POINT(34.4661213223241 -117.89573802490668) bank7216 +7217 POINT(33.54180403173702 -118.81789341320219) bank7217 +7218 POINT(33.12526532849151 -118.57098107543368) bank7218 +7219 POINT(33.72934107888281 -119.1207148887065) bank7219 +7220 POINT(34.09707872566803 -118.03468639872817) bank7220 +7221 POINT(34.161172478218866 -117.7293894042234) bank7221 +7222 POINT(33.64284742458652 -118.84678969793947) bank7222 +7223 POINT(34.637725305459064 -117.88427690939046) bank7223 +7224 POINT(34.46272622622286 -119.14648269323429) bank7224 +7225 POINT(33.36274876627899 -118.15168348967951) bank7225 +7226 POINT(34.49190320937941 -117.85709321579776) bank7226 +7227 POINT(33.77087195491263 -117.92077068701505) bank7227 +7228 POINT(34.41861334675801 -118.98713541583045) bank7228 +7229 POINT(33.302123247131426 -118.47332626203746) bank7229 +7230 POINT(34.002666127001426 -117.57590077273913) bank7230 +7231 POINT(33.69870881699099 -118.89177007784723) bank7231 +7232 POINT(34.69001427034048 -118.36804053562149) bank7232 +7233 POINT(33.43324792250033 -118.15304241633083) bank7233 +7234 POINT(35.04205038775933 -117.49415545194053) bank7234 +7235 POINT(34.35580529065834 -119.20264513237865) bank7235 +7236 POINT(34.73717490108095 -118.99884303359426) bank7236 +7237 POINT(33.73357772011212 -118.69668497845706) bank7237 +7238 POINT(34.30603332324695 -118.1021579535775) bank7238 +7239 POINT(33.54596901933874 -119.16827654427107) bank7239 +7240 POINT(34.719301497030024 -118.48897809621145) bank7240 +7241 POINT(34.53089910306284 -117.40639568989182) bank7241 +7242 POINT(33.44438370704231 -117.92715473060446) bank7242 +7243 POINT(34.48017402751229 -119.18865299680805) bank7243 +7244 POINT(33.35789560935637 -118.9731870771053) bank7244 +7245 POINT(33.492469414230506 -118.49478037446737) bank7245 +7246 POINT(34.339737890532994 -117.5121287389034) bank7246 +7247 POINT(34.642169348569574 -117.7683470350778) bank7247 +7248 POINT(34.90357269667035 -118.47995606730599) bank7248 +7249 POINT(33.66037998127143 -118.03225556701601) bank7249 +7250 POINT(34.74692147930769 -118.64848961841831) bank7250 +7251 POINT(34.79075843861614 -119.14795115921547) bank7251 +7252 POINT(33.883438623304095 -117.89486991459769) bank7252 +7253 POINT(33.188240187477405 -117.42444268455557) bank7253 +7254 POINT(34.996660772338785 -118.29594491995948) bank7254 +7255 POINT(34.92896663177994 -117.28954659771371) bank7255 +7256 POINT(33.555978380249236 -117.40517929499362) bank7256 +7257 POINT(34.62595372019428 -117.99532560741058) bank7257 +7258 POINT(33.08615631718043 -117.91346059148614) bank7258 +7259 POINT(34.7276782433833 -118.54491473491028) bank7259 +7260 POINT(34.70057941003328 -117.42318073958292) bank7260 +7261 POINT(34.233477432278136 -117.94379547415392) bank7261 +7262 POINT(34.31156252163285 -117.96056937951568) bank7262 +7263 POINT(33.60353611624766 -119.11935669547306) bank7263 +7264 POINT(33.211605670688826 -117.6956931072745) bank7264 +7265 POINT(34.52533768696508 -118.88733110986988) bank7265 +7266 POINT(34.77381160018102 -118.32681842009859) bank7266 +7267 POINT(34.111065375538516 -118.07634336399659) bank7267 +7268 POINT(33.198006184051835 -118.46147822645923) bank7268 +7269 POINT(34.19496521009945 -117.95614244543363) bank7269 +7270 POINT(34.46648398813246 -117.53753160702065) bank7270 +7271 POINT(34.55055351240146 -118.05584370103698) bank7271 +7272 POINT(33.677853935860675 -117.6426644425811) bank7272 +7273 POINT(35.00110347768288 -117.39970417591809) bank7273 +7274 POINT(34.39201480657624 -118.1673065214282) bank7274 +7275 POINT(33.25334543893409 -118.188807707735) bank7275 +7276 POINT(33.721213718294784 -117.78216224032715) bank7276 +7277 POINT(33.628331653483976 -118.6395363187784) bank7277 +7278 POINT(34.317091348863734 -119.17934476742553) bank7278 +7279 POINT(33.70498244952873 -117.45538151027156) bank7279 +7280 POINT(34.99967452012691 -118.29797134963871) bank7280 +7281 POINT(33.31458680056122 -117.63992146623916) bank7281 +7282 POINT(34.2962474242822 -119.04692392289058) bank7282 +7283 POINT(33.515433885207784 -117.69727365970101) bank7283 +7284 POINT(33.51481943039783 -118.85479076423087) bank7284 +7285 POINT(34.236464967619156 -118.9116748315396) bank7285 +7286 POINT(33.29702172645084 -118.10795499884094) bank7286 +7287 POINT(33.32414182945026 -118.2649501653954) bank7287 +7288 POINT(33.95262030309735 -117.72926776486361) bank7288 +7289 POINT(33.41279861297316 -118.42579786906471) bank7289 +7290 POINT(34.203004747193745 -119.16897595746907) bank7290 +7291 POINT(33.2267725921321 -118.59201628930796) bank7291 +7292 POINT(34.21743382371313 -117.83167697500559) bank7292 +7293 POINT(33.709122000833744 -119.18965189061885) bank7293 +7294 POINT(34.77922728794144 -118.06788732320757) bank7294 +7295 POINT(33.35628543085816 -117.5817410540548) bank7295 +7296 POINT(33.183286886906885 -118.27296158500026) bank7296 +7297 POINT(34.56706171994463 -118.19791546221578) bank7297 +7298 POINT(34.11245135160401 -118.86365571197493) bank7298 +7299 POINT(33.35046085906189 -118.69244306694713) bank7299 +7300 POINT(34.6670146853475 -118.11149789607919) bank7300 +7301 POINT(33.25921098398457 -118.00854586389389) bank7301 +7302 POINT(33.15723410314775 -118.29812577994852) bank7302 +7303 POINT(34.4054939585635 -118.68930033894465) bank7303 +7304 POINT(34.77585789522221 -118.17375785380717) bank7304 +7305 POINT(34.186180986522466 -118.70549202966095) bank7305 +7306 POINT(33.1098171272783 -119.21852062759136) bank7306 +7307 POINT(34.26645459683623 -118.19748495118668) bank7307 +7308 POINT(33.10622977997539 -119.12745306575142) bank7308 +7309 POINT(34.58773635083531 -117.8803240006444) bank7309 +7310 POINT(34.16234921264927 -118.85750109602583) bank7310 +7311 POINT(34.588730629084836 -118.18879940489704) bank7311 +7312 POINT(34.31300534312122 -119.04698978587022) bank7312 +7313 POINT(34.96726677339157 -117.63275967314068) bank7313 +7314 POINT(33.87300344119049 -118.61992307575471) bank7314 +7315 POINT(34.73449995913285 -118.33942350326882) bank7315 +7316 POINT(34.027043108926044 -117.68699560895296) bank7316 +7317 POINT(33.41081658258983 -117.5025505864343) bank7317 +7318 POINT(35.028983372562976 -118.83940023534004) bank7318 +7319 POINT(34.329602774509794 -117.95952611783915) bank7319 +7320 POINT(33.942891564613404 -117.51840387290852) bank7320 +7321 POINT(34.44463263994656 -118.5093636775694) bank7321 +7322 POINT(33.08933497712148 -118.3421155408889) bank7322 +7323 POINT(33.134374288126196 -118.75107414991466) bank7323 +7324 POINT(34.68070886831053 -117.73822840188504) bank7324 +7325 POINT(33.05314686666191 -117.27359766850165) bank7325 +7326 POINT(33.36007784848349 -118.08445780097588) bank7326 +7327 POINT(34.53214590269797 -117.93367043557046) bank7327 +7328 POINT(35.04640421186381 -117.28273652099925) bank7328 +7329 POINT(34.889978959325695 -118.66825174384812) bank7329 +7330 POINT(34.89267120880163 -119.19622739628272) bank7330 +7331 POINT(34.22890957210271 -118.81488830058412) bank7331 +7332 POINT(33.15006739790707 -117.51728762708348) bank7332 +7333 POINT(33.15907696047345 -118.46550250172103) bank7333 +7334 POINT(34.1488094843099 -117.86210688069838) bank7334 +7335 POINT(34.61961833199359 -118.80991101369551) bank7335 +7336 POINT(34.712341343161214 -118.20665431451543) bank7336 +7337 POINT(34.15054535755692 -118.62130670549753) bank7337 +7338 POINT(33.1752564055996 -117.54780486192365) bank7338 +7339 POINT(33.86978126708005 -118.11384036517578) bank7339 +7340 POINT(34.29349047922185 -117.98102062211605) bank7340 +7341 POINT(33.56577564056859 -117.26627046951879) bank7341 +7342 POINT(33.80669501492084 -118.0966050872459) bank7342 +7343 POINT(33.210923428414745 -118.71253422469724) bank7343 +7344 POINT(34.64204187433618 -118.2895809403239) bank7344 +7345 POINT(33.092954742666606 -118.26094490829593) bank7345 +7346 POINT(34.296125649281784 -118.88015318295385) bank7346 +7347 POINT(33.20964720500872 -118.96042537662095) bank7347 +7348 POINT(34.68333768058915 -118.82939814178667) bank7348 +7349 POINT(33.92046048073351 -117.71601740936724) bank7349 +7350 POINT(34.843410278098325 -118.49299996430042) bank7350 +7351 POINT(34.483272580755994 -119.16399906083835) bank7351 +7352 POINT(33.851590360880515 -117.81757899965942) bank7352 +7353 POINT(33.37582430273688 -119.14578745011327) bank7353 +7354 POINT(34.47869266774801 -119.20603029660371) bank7354 +7355 POINT(34.168421349925445 -119.08820851323304) bank7355 +7356 POINT(33.98198764152544 -119.11559346381914) bank7356 +7357 POINT(33.89475804033161 -119.0361997386821) bank7357 +7358 POINT(34.00138710920838 -118.84887600263882) bank7358 +7359 POINT(34.99346399269746 -118.52352079360503) bank7359 +7360 POINT(33.19901678736425 -118.7110807746755) bank7360 +7361 POINT(34.63908095620212 -118.1719850103608) bank7361 +7362 POINT(33.25844005957318 -118.68650204599794) bank7362 +7363 POINT(34.97739811704244 -117.65301716241146) bank7363 +7364 POINT(34.38791967152733 -117.29097311733285) bank7364 +7365 POINT(34.743934891440304 -117.94967843628913) bank7365 +7366 POINT(33.89307104750195 -118.816192244747) bank7366 +7367 POINT(33.234893244911014 -117.61063555654098) bank7367 +7368 POINT(34.03244141964756 -117.98661720264981) bank7368 +7369 POINT(34.82525243667673 -118.26890466951099) bank7369 +7370 POINT(34.561416905368446 -118.30896066793566) bank7370 +7371 POINT(33.9191538061339 -117.3425464965476) bank7371 +7372 POINT(33.25413046482841 -117.37430894532027) bank7372 +7373 POINT(33.14270692720731 -117.73734351163532) bank7373 +7374 POINT(33.685056080181866 -118.28966076898375) bank7374 +7375 POINT(33.912305327298206 -117.4880852687008) bank7375 +7376 POINT(33.862318840745395 -117.58427364579109) bank7376 +7377 POINT(34.62315093964774 -117.791898734634) bank7377 +7378 POINT(33.81837289751595 -118.8196280588093) bank7378 +7379 POINT(33.98515875495571 -118.41100691069673) bank7379 +7380 POINT(33.72764707718194 -119.2113302011328) bank7380 +7381 POINT(33.30561837465729 -117.44153873055086) bank7381 +7382 POINT(34.505056772005936 -117.77713673871354) bank7382 +7383 POINT(34.486359654481966 -118.37609080764737) bank7383 +7384 POINT(33.137350329903406 -119.0798458923003) bank7384 +7385 POINT(34.65430427198156 -117.57740276959947) bank7385 +7386 POINT(34.178963124848295 -118.53767990620725) bank7386 +7387 POINT(33.533992025461366 -117.8689907790865) bank7387 +7388 POINT(33.937730625143985 -118.28797344739894) bank7388 +7389 POINT(34.35870751492425 -119.23112848203192) bank7389 +7390 POINT(34.85905099549568 -117.73824863797107) bank7390 +7391 POINT(34.70639336452858 -118.23590942294474) bank7391 +7392 POINT(33.30323451547756 -117.9699435040487) bank7392 +7393 POINT(34.866397551347454 -117.95524530302187) bank7393 +7394 POINT(34.747451050518265 -118.02530578491317) bank7394 +7395 POINT(33.98341646630856 -118.66989528130902) bank7395 +7396 POINT(33.41421577890993 -117.96296608819615) bank7396 +7397 POINT(33.55446221408016 -118.64673417878367) bank7397 +7398 POINT(34.05187849776951 -118.24449669365713) bank7398 +7399 POINT(33.26259616029288 -117.37492255468425) bank7399 +7400 POINT(34.57458171302547 -119.15658576624263) bank7400 +7401 POINT(34.26829886219224 -118.73816720653825) bank7401 +7402 POINT(33.712501991854445 -117.75645497078693) bank7402 +7403 POINT(34.46056222605374 -118.1906349390561) bank7403 +7404 POINT(34.15219504035897 -117.71560442029082) bank7404 +7405 POINT(33.78382545415525 -119.17439756680746) bank7405 +7406 POINT(33.56583643240263 -118.76383061100026) bank7406 +7407 POINT(34.28411551330828 -118.08511258335997) bank7407 +7408 POINT(33.9080894700982 -118.25822568345843) bank7408 +7409 POINT(34.88180259256213 -118.49949521180183) bank7409 +7410 POINT(34.28745299598012 -118.73530654352294) bank7410 +7411 POINT(34.0353745392841 -117.69433224435467) bank7411 +7412 POINT(34.578209872309465 -119.23105153013985) bank7412 +7413 POINT(33.09416586678435 -117.35080175225815) bank7413 +7414 POINT(34.101318559291634 -117.56784872200922) bank7414 +7415 POINT(34.800404587166376 -118.58516216425723) bank7415 +7416 POINT(34.65301623426382 -117.75743315221646) bank7416 +7417 POINT(34.51765383929619 -117.49695154655063) bank7417 +7418 POINT(33.09875117618454 -118.03665936484566) bank7418 +7419 POINT(33.5005571487696 -117.3472557672454) bank7419 +7420 POINT(35.00911031141312 -117.67323481321577) bank7420 +7421 POINT(33.45246491137021 -118.87315345370547) bank7421 +7422 POINT(33.5120579107185 -117.4897258466985) bank7422 +7423 POINT(34.84583169231139 -118.04936099665696) bank7423 +7424 POINT(33.92335911288418 -118.48814513245911) bank7424 +7425 POINT(34.50955734649484 -118.37922629261543) bank7425 +7426 POINT(34.01347358962697 -118.34223628964683) bank7426 +7427 POINT(33.828016840759176 -118.34741837137298) bank7427 +7428 POINT(33.848875967949226 -117.84822903920015) bank7428 +7429 POINT(34.2997350840138 -119.12530823989518) bank7429 +7430 POINT(34.90068715551767 -117.38739822111079) bank7430 +7431 POINT(34.79927862773752 -118.20915769492582) bank7431 +7432 POINT(33.19817206259767 -117.61385881876903) bank7432 +7433 POINT(34.16020946435805 -118.19496659938332) bank7433 +7434 POINT(33.78373098009131 -117.58807019313124) bank7434 +7435 POINT(34.09247342948545 -118.72472733884852) bank7435 +7436 POINT(33.9346579050582 -117.43184634262093) bank7436 +7437 POINT(34.36862290943519 -118.30273470922525) bank7437 +7438 POINT(34.4417403582295 -118.63636396365361) bank7438 +7439 POINT(33.429066742276376 -117.27214235032206) bank7439 +7440 POINT(33.10849677490478 -117.51731977085609) bank7440 +7441 POINT(33.6348507873501 -119.0540019757188) bank7441 +7442 POINT(33.252321238452396 -118.65134009125605) bank7442 +7443 POINT(33.221722456906015 -118.24938826935232) bank7443 +7444 POINT(33.61370446879683 -118.56756673462127) bank7444 +7445 POINT(34.362863350005526 -118.54990377608948) bank7445 +7446 POINT(33.710148298409635 -117.99014650959639) bank7446 +7447 POINT(34.67973221891511 -117.90702751312793) bank7447 +7448 POINT(33.677264988477326 -117.65526136181171) bank7448 +7449 POINT(33.62497422449821 -119.05154519463433) bank7449 +7450 POINT(34.13118230996997 -117.81809335681899) bank7450 +7451 POINT(33.72966846896824 -117.6662831981317) bank7451 +7452 POINT(33.43582985783903 -118.57026711508058) bank7452 +7453 POINT(33.204032471478584 -118.71466282998853) bank7453 +7454 POINT(33.29656374263317 -119.06252302334357) bank7454 +7455 POINT(34.19574325236689 -119.19177746051689) bank7455 +7456 POINT(33.69962412687891 -118.72130145960449) bank7456 +7457 POINT(34.103273085866505 -118.05559342263246) bank7457 +7458 POINT(34.78037519893592 -117.69423107631266) bank7458 +7459 POINT(34.22639241502292 -118.36929538214306) bank7459 +7460 POINT(34.571038165913045 -119.2275560586661) bank7460 +7461 POINT(34.333766367001815 -118.4919749226986) bank7461 +7462 POINT(34.16076629420733 -117.37753751017915) bank7462 +7463 POINT(34.12873543196791 -117.3935553425565) bank7463 +7464 POINT(34.7581030461519 -117.78464901177624) bank7464 +7465 POINT(34.3542536221109 -117.48654422496743) bank7465 +7466 POINT(33.24147967002924 -117.70362251784725) bank7466 +7467 POINT(35.030300133972354 -117.54925279142505) bank7467 +7468 POINT(34.36445589358934 -118.04320754664664) bank7468 +7469 POINT(33.65432127366831 -117.84604853542285) bank7469 +7470 POINT(34.28187171469466 -117.36694799695945) bank7470 +7471 POINT(34.38914025853615 -117.359253551255) bank7471 +7472 POINT(34.672918180650626 -117.33131225761808) bank7472 +7473 POINT(34.32795055681087 -117.35769481361156) bank7473 +7474 POINT(34.314434248993386 -119.1271191041447) bank7474 +7475 POINT(34.447260348601034 -117.34165123478193) bank7475 +7476 POINT(34.66619904622718 -117.59473361707795) bank7476 +7477 POINT(33.26738363264874 -117.88514725401546) bank7477 +7478 POINT(33.53556108477309 -117.27173389212008) bank7478 +7479 POINT(34.42582210141054 -118.67020056825461) bank7479 +7480 POINT(34.12737142224125 -118.76034663547816) bank7480 +7481 POINT(34.24426387000652 -117.53175283588) bank7481 +7482 POINT(35.002665806996454 -118.54277705598085) bank7482 +7483 POINT(33.36127916119646 -118.0691468465421) bank7483 +7484 POINT(34.617675239433055 -117.88184682554012) bank7484 +7485 POINT(33.78224126852213 -117.48618788789281) bank7485 +7486 POINT(34.6046919987159 -117.8948306863547) bank7486 +7487 POINT(34.424149815645215 -118.63977102094965) bank7487 +7488 POINT(35.016992698338434 -117.3437779764833) bank7488 +7489 POINT(33.45691238098253 -117.87451185302635) bank7489 +7490 POINT(35.03093958778144 -118.14553161166803) bank7490 +7491 POINT(34.89766026105371 -118.07496270466805) bank7491 +7492 POINT(34.19782337497715 -118.93270614165017) bank7492 +7493 POINT(34.73011642725744 -117.93596417047623) bank7493 +7494 POINT(34.44956245392678 -118.04564727150591) bank7494 +7495 POINT(33.288319163093036 -117.88014567564045) bank7495 +7496 POINT(34.21345708460649 -118.27114435624279) bank7496 +7497 POINT(34.67159814286078 -118.84516473421404) bank7497 +7498 POINT(34.93267820467558 -117.97530497412808) bank7498 +7499 POINT(33.81155215465423 -118.92608075677005) bank7499 +7500 POINT(33.64886241782087 -118.03317238267867) bank7500 +7501 POINT(33.3464655543475 -118.45699547199249) bank7501 +7502 POINT(33.3147893762158 -117.38892290720936) bank7502 +7503 POINT(34.72524443433534 -118.3216241688091) bank7503 +7504 POINT(33.39745883272432 -118.22961005890373) bank7504 +7505 POINT(33.902414970278265 -117.76750908341678) bank7505 +7506 POINT(33.53115806304383 -117.79611538738284) bank7506 +7507 POINT(34.30141550865138 -117.76248014838752) bank7507 +7508 POINT(33.218241983647616 -118.33110844309613) bank7508 +7509 POINT(33.19631261913501 -118.94799566719631) bank7509 +7510 POINT(34.2862202565992 -119.08631891706489) bank7510 +7511 POINT(33.64299647432262 -117.72601790589205) bank7511 +7512 POINT(34.753433547357105 -118.0374938560258) bank7512 +7513 POINT(34.29282792604611 -117.87563393995406) bank7513 +7514 POINT(34.83611429942976 -118.17054992043667) bank7514 +7515 POINT(34.55095149129242 -117.54595003968248) bank7515 +7516 POINT(33.890614375129374 -118.6757404030214) bank7516 +7517 POINT(33.27660407770619 -118.22860567218508) bank7517 +7518 POINT(33.665734173461495 -118.39815459542372) bank7518 +7519 POINT(33.896339148997285 -118.45654893265207) bank7519 +7520 POINT(34.328060001717496 -117.43614601258572) bank7520 +7521 POINT(34.67517634621405 -119.19341804975039) bank7521 +7522 POINT(34.06773405699422 -118.09465944778069) bank7522 +7523 POINT(33.26740920219723 -118.75575017999766) bank7523 +7524 POINT(33.77014258373652 -117.67102620638983) bank7524 +7525 POINT(33.97177629976316 -118.8909095109323) bank7525 +7526 POINT(33.33130224396986 -119.23961491612683) bank7526 +7527 POINT(33.05741301091551 -117.46274598343439) bank7527 +7528 POINT(33.71509162723752 -117.77142810329516) bank7528 +7529 POINT(34.70812739215865 -118.75421537450255) bank7529 +7530 POINT(34.969439604643554 -119.02338135173083) bank7530 +7531 POINT(34.06005199422815 -118.09049603488677) bank7531 +7532 POINT(33.39601388537424 -118.83159385084866) bank7532 +7533 POINT(34.18983750108334 -118.61366766028412) bank7533 +7534 POINT(34.615264286616906 -118.39203251474132) bank7534 +7535 POINT(34.57189345389903 -118.91237588479748) bank7535 +7536 POINT(33.136899041017486 -119.12179603843737) bank7536 +7537 POINT(33.970738130057015 -118.40294273585339) bank7537 +7538 POINT(33.17187023925016 -117.37716450480991) bank7538 +7539 POINT(34.184054732622045 -117.94680663043846) bank7539 +7540 POINT(33.82938791934775 -118.54261378883757) bank7540 +7541 POINT(33.664355988113705 -118.04453079712611) bank7541 +7542 POINT(33.315693409389006 -117.46053452182561) bank7542 +7543 POINT(34.44019324080765 -118.98962152443231) bank7543 +7544 POINT(33.305651318909 -118.30897236882616) bank7544 +7545 POINT(33.54213310611235 -119.13422891464288) bank7545 +7546 POINT(34.393317100948565 -118.08695757185419) bank7546 +7547 POINT(34.930958128802565 -117.51863534794329) bank7547 +7548 POINT(34.40853017784024 -117.97803269749382) bank7548 +7549 POINT(34.48486945498009 -118.97199904655218) bank7549 +7550 POINT(34.54120615545988 -118.45435333910196) bank7550 +7551 POINT(33.15250679603093 -118.95866957314911) bank7551 +7552 POINT(34.69749833092465 -118.59877556880836) bank7552 +7553 POINT(34.78472050922029 -119.16113653071041) bank7553 +7554 POINT(34.399268635457005 -119.16999381704305) bank7554 +7555 POINT(33.618106620827746 -118.29027382845601) bank7555 +7556 POINT(34.873917190231865 -118.5175022124534) bank7556 +7557 POINT(33.880527814386966 -118.87384397520022) bank7557 +7558 POINT(33.62216362816682 -118.6116735679117) bank7558 +7559 POINT(33.75787345749778 -117.59016059242369) bank7559 +7560 POINT(34.052205005517784 -118.33942729235119) bank7560 +7561 POINT(33.61616643934927 -117.37476800955723) bank7561 +7562 POINT(34.92507912938646 -118.4424789795491) bank7562 +7563 POINT(34.15135674877969 -118.7125666726521) bank7563 +7564 POINT(34.138635660883715 -118.34238869537282) bank7564 +7565 POINT(34.65523100625204 -119.01836224376076) bank7565 +7566 POINT(34.513860487289136 -119.20987615497873) bank7566 +7567 POINT(34.90862108333913 -118.78152689675353) bank7567 +7568 POINT(34.9248824005623 -119.13317350463497) bank7568 +7569 POINT(33.54210659259636 -117.7696598010622) bank7569 +7570 POINT(33.84360448574524 -117.7536104980037) bank7570 +7571 POINT(33.87115949637635 -118.81171118163644) bank7571 +7572 POINT(34.22808057599796 -117.97596993928929) bank7572 +7573 POINT(33.63777785243008 -118.13190840104537) bank7573 +7574 POINT(33.38399294570515 -117.35086193776108) bank7574 +7575 POINT(34.371700605513205 -117.64087646373639) bank7575 +7576 POINT(34.39960149815754 -117.69843297195762) bank7576 +7577 POINT(34.371719521080834 -119.07335496324083) bank7577 +7578 POINT(34.00485975622349 -118.84509513005015) bank7578 +7579 POINT(34.83253477897553 -117.71633037292744) bank7579 +7580 POINT(34.493758342054434 -119.10087588990899) bank7580 +7581 POINT(34.18911216802709 -118.76253176005427) bank7581 +7582 POINT(34.07057950371623 -117.71449469390153) bank7582 +7583 POINT(34.124306459798326 -118.84744934740844) bank7583 +7584 POINT(33.500730801952706 -118.53420153836794) bank7584 +7585 POINT(33.58476691773221 -119.16751230405607) bank7585 +7586 POINT(33.606054278877494 -118.17246856752783) bank7586 +7587 POINT(34.76769965035238 -118.89890332951774) bank7587 +7588 POINT(33.642011002652055 -118.86783531776535) bank7588 +7589 POINT(33.79845197835536 -118.92783175006394) bank7589 +7590 POINT(34.39142975710371 -117.34723000289159) bank7590 +7591 POINT(33.55741148368245 -119.0080836567777) bank7591 +7592 POINT(34.83362926978947 -119.24199634841682) bank7592 +7593 POINT(33.07319866018271 -118.50064358425128) bank7593 +7594 POINT(34.54199213207927 -117.26811150415332) bank7594 +7595 POINT(35.009577406230726 -118.56565323606702) bank7595 +7596 POINT(33.14608583417839 -118.12179114742563) bank7596 +7597 POINT(33.75416678788006 -117.31703618180698) bank7597 +7598 POINT(34.81534499537553 -117.53446610790554) bank7598 +7599 POINT(33.59420660213452 -117.93480995947037) bank7599 +7600 POINT(34.11481695734812 -117.26418882248099) bank7600 +7601 POINT(33.58560298380639 -118.3196726343787) bank7601 +7602 POINT(34.81376796922784 -118.82947655668556) bank7602 +7603 POINT(33.389533256643944 -118.35522668384729) bank7603 +7604 POINT(34.31062566029442 -117.70136250139748) bank7604 +7605 POINT(33.47166497902397 -117.68383082382114) bank7605 +7606 POINT(33.239028107678315 -117.60512150135509) bank7606 +7607 POINT(34.418059722487044 -118.71540546673249) bank7607 +7608 POINT(33.59115528680617 -118.5958644508405) bank7608 +7609 POINT(33.95230264287254 -118.60180118745129) bank7609 +7610 POINT(34.11878940773507 -118.86051065544767) bank7610 +7611 POINT(33.08943403640522 -117.68236429344809) bank7611 +7612 POINT(33.230629548922174 -118.29235353051546) bank7612 +7613 POINT(33.24769122387574 -119.22014785003073) bank7613 +7614 POINT(34.840994110172794 -119.03168358444185) bank7614 +7615 POINT(33.19348929160231 -117.29116156014214) bank7615 +7616 POINT(33.661760573010646 -118.65227801465676) bank7616 +7617 POINT(33.99526588751867 -119.00759780253787) bank7617 +7618 POINT(33.11553184469938 -117.68595059583397) bank7618 +7619 POINT(34.115608287626515 -119.16332491969709) bank7619 +7620 POINT(34.10934283841525 -118.97725319353496) bank7620 +7621 POINT(34.628547758734804 -119.23274828039874) bank7621 +7622 POINT(33.17652796478101 -117.55926131816481) bank7622 +7623 POINT(34.93567270586364 -118.03261055530395) bank7623 +7624 POINT(34.458789361702244 -117.6762797798545) bank7624 +7625 POINT(33.86402944582173 -117.5019006171561) bank7625 +7626 POINT(34.81556982382101 -117.73040941208967) bank7626 +7627 POINT(33.52529550371587 -118.91269770590281) bank7627 +7628 POINT(33.84362229870131 -117.73589189247197) bank7628 +7629 POINT(35.046354353364514 -118.56227564353743) bank7629 +7630 POINT(33.96485646361648 -119.14770884208149) bank7630 +7631 POINT(33.70819186042142 -117.25367329728066) bank7631 +7632 POINT(33.72908381382096 -118.89451500500256) bank7632 +7633 POINT(33.44005217960992 -118.0589919518427) bank7633 +7634 POINT(34.649419584643866 -117.76937995776974) bank7634 +7635 POINT(33.805105401212515 -118.77159894269991) bank7635 +7636 POINT(34.76449104903962 -118.88104030444968) bank7636 +7637 POINT(34.713287217178475 -117.99612593659332) bank7637 +7638 POINT(33.440441828308025 -117.42250618493753) bank7638 +7639 POINT(34.12999772892218 -117.7983967899073) bank7639 +7640 POINT(34.8039558999526 -118.19817054428833) bank7640 +7641 POINT(33.08305990628097 -118.0533775558576) bank7641 +7642 POINT(34.36375752402957 -117.4299209625525) bank7642 +7643 POINT(34.99616991684885 -119.14214278212764) bank7643 +7644 POINT(34.6979694906058 -119.13144475320503) bank7644 +7645 POINT(34.273706168573085 -118.5848266704138) bank7645 +7646 POINT(33.372443631665185 -118.16453452815868) bank7646 +7647 POINT(33.729688737443084 -118.95330537843613) bank7647 +7648 POINT(33.78281679495325 -118.2549208745454) bank7648 +7649 POINT(34.894449114538446 -118.62108847324002) bank7649 +7650 POINT(34.60735956124178 -119.05426942691545) bank7650 +7651 POINT(33.29303658891366 -117.6480039726487) bank7651 +7652 POINT(34.43128080324138 -119.11865864297313) bank7652 +7653 POINT(34.33793137103464 -119.23834658686658) bank7653 +7654 POINT(33.70107659749667 -118.29072092270165) bank7654 +7655 POINT(33.467713587415574 -119.03722293034889) bank7655 +7656 POINT(33.93641881664742 -117.48286566166384) bank7656 +7657 POINT(34.23669182581317 -117.75826331767502) bank7657 +7658 POINT(33.752619527973295 -118.7617440302093) bank7658 +7659 POINT(33.27970485043765 -117.57493384222856) bank7659 +7660 POINT(34.810037411761165 -117.24695923467934) bank7660 +7661 POINT(34.97365360149194 -118.20954333676927) bank7661 +7662 POINT(34.207972459718796 -117.91027603102046) bank7662 +7663 POINT(34.241510517450855 -119.08374339678949) bank7663 +7664 POINT(34.29179051437563 -118.60993760956028) bank7664 +7665 POINT(33.33534096472069 -117.41864320985724) bank7665 +7666 POINT(34.28043695483332 -118.58655620958044) bank7666 +7667 POINT(33.93218810377581 -118.88948015495532) bank7667 +7668 POINT(34.62493654175526 -118.95530875250789) bank7668 +7669 POINT(33.59607732167261 -117.46102375416844) bank7669 +7670 POINT(34.30136316716397 -118.67957069511002) bank7670 +7671 POINT(33.198916543520696 -118.06876330041115) bank7671 +7672 POINT(34.49159112439748 -118.81264807232851) bank7672 +7673 POINT(33.86858864680397 -118.08922148814156) bank7673 +7674 POINT(33.05691948828849 -117.35839024805489) bank7674 +7675 POINT(33.209459402879816 -119.0742264201906) bank7675 +7676 POINT(33.72592401100909 -118.80994659620863) bank7676 +7677 POINT(33.206525343791874 -118.70372704286953) bank7677 +7678 POINT(34.65349487379843 -117.29393773813095) bank7678 +7679 POINT(33.5303122411746 -119.04047627913491) bank7679 +7680 POINT(34.77349710430822 -119.19703697933885) bank7680 +7681 POINT(34.16705661408797 -117.36562408398724) bank7681 +7682 POINT(33.73886936406873 -118.78762103015602) bank7682 +7683 POINT(34.32018035532603 -118.62654907396518) bank7683 +7684 POINT(33.244817445011314 -117.62864198141381) bank7684 +7685 POINT(33.34939514549351 -119.23972716223133) bank7685 +7686 POINT(34.153941429437396 -118.49220193471825) bank7686 +7687 POINT(33.871329740815575 -117.76987081463979) bank7687 +7688 POINT(33.427586528795764 -118.91531097420074) bank7688 +7689 POINT(34.45884621550907 -118.83581071260251) bank7689 +7690 POINT(33.82747768210309 -118.85574700836223) bank7690 +7691 POINT(34.572457439324474 -118.66766207297563) bank7691 +7692 POINT(34.67857011576321 -118.58282091034025) bank7692 +7693 POINT(33.37555539910101 -118.6079115032036) bank7693 +7694 POINT(33.327144031561865 -118.31797132605332) bank7694 +7695 POINT(33.513511746567126 -118.04412041759932) bank7695 +7696 POINT(34.42339683985139 -117.86469043244811) bank7696 +7697 POINT(34.842741186063364 -118.88298753051187) bank7697 +7698 POINT(34.613778625270484 -118.86924337031368) bank7698 +7699 POINT(34.70762319976913 -118.70956878959811) bank7699 +7700 POINT(34.566650880713055 -118.13520880180222) bank7700 +7701 POINT(34.40486489986439 -119.20767808773753) bank7701 +7702 POINT(34.94934819409545 -117.4646425950997) bank7702 +7703 POINT(34.14926802875606 -117.81584752337392) bank7703 +7704 POINT(34.4725289848622 -118.88692242711555) bank7704 +7705 POINT(33.139632711567714 -118.8576140392239) bank7705 +7706 POINT(34.33495003341236 -118.71804122021658) bank7706 +7707 POINT(34.422167926821814 -117.91735428362568) bank7707 +7708 POINT(33.1554224322654 -118.38401221406713) bank7708 +7709 POINT(34.36277076220643 -117.25486901135794) bank7709 +7710 POINT(34.6248548716373 -119.02307768075923) bank7710 +7711 POINT(33.74357658837377 -118.89802486070863) bank7711 +7712 POINT(34.92775278204325 -117.30697559721969) bank7712 +7713 POINT(34.597220440589794 -118.23300697803022) bank7713 +7714 POINT(34.35720965879731 -118.27372553496578) bank7714 +7715 POINT(33.263853937132666 -118.10547090740774) bank7715 +7716 POINT(33.99234732277625 -117.80404118945434) bank7716 +7717 POINT(34.38081027936466 -119.23616678975392) bank7717 +7718 POINT(33.396161034155604 -117.68003564326136) bank7718 +7719 POINT(34.72456269093445 -117.3143516092945) bank7719 +7720 POINT(33.18218426623425 -118.06016007980418) bank7720 +7721 POINT(35.009689751629566 -117.97047864119399) bank7721 +7722 POINT(34.14070777021572 -117.3323362847934) bank7722 +7723 POINT(33.70183713757527 -118.66735336566762) bank7723 +7724 POINT(33.98322414944316 -117.37466917007687) bank7724 +7725 POINT(33.38502957097293 -118.3525116321177) bank7725 +7726 POINT(33.920446644909354 -118.01770478066072) bank7726 +7727 POINT(34.00037943909569 -118.34140984699167) bank7727 +7728 POINT(33.328688646495166 -117.86849153739797) bank7728 +7729 POINT(33.33734027470489 -118.1851153334782) bank7729 +7730 POINT(33.602092653481364 -117.33485899745864) bank7730 +7731 POINT(33.494818088060704 -118.0062962246917) bank7731 +7732 POINT(33.32509371570899 -117.32316192318724) bank7732 +7733 POINT(33.72523896478351 -118.80980911794879) bank7733 +7734 POINT(34.19308458362005 -119.16966843602656) bank7734 +7735 POINT(34.52174187374531 -117.72505920920007) bank7735 +7736 POINT(34.30247350663943 -117.27304841573674) bank7736 +7737 POINT(34.77264964295728 -118.68104121469868) bank7737 +7738 POINT(34.64648933464141 -118.50618344566911) bank7738 +7739 POINT(33.97491132312095 -118.08823516286708) bank7739 +7740 POINT(34.690297826145134 -117.73509200069547) bank7740 +7741 POINT(34.46246122830355 -118.68633217075) bank7741 +7742 POINT(34.05221690651564 -118.93943813399315) bank7742 +7743 POINT(34.583248540415326 -117.89969537185634) bank7743 +7744 POINT(34.98064245359789 -118.6514358359091) bank7744 +7745 POINT(35.015771084390536 -118.26262786342447) bank7745 +7746 POINT(34.91121889578667 -118.57356417105133) bank7746 +7747 POINT(33.8976926887718 -117.70210042889083) bank7747 +7748 POINT(33.429939769280864 -118.31842987781823) bank7748 +7749 POINT(33.383231296698696 -119.18494704148883) bank7749 +7750 POINT(34.081093381696746 -117.9108877140783) bank7750 +7751 POINT(34.76749735453537 -117.72454320267133) bank7751 +7752 POINT(34.091657080007096 -118.1219238530458) bank7752 +7753 POINT(34.97444803723734 -117.89175936239897) bank7753 +7754 POINT(33.88355280529279 -118.2579250676209) bank7754 +7755 POINT(33.446455251856094 -118.76042695822535) bank7755 +7756 POINT(33.160855759908706 -117.87034043155552) bank7756 +7757 POINT(33.21674543550244 -118.23373745501748) bank7757 +7758 POINT(34.76916727921281 -117.82831771512411) bank7758 +7759 POINT(33.87009289074199 -117.60286603846522) bank7759 +7760 POINT(34.8068537873491 -117.98504276904535) bank7760 +7761 POINT(34.691303303779605 -119.17908354634183) bank7761 +7762 POINT(34.41847775305168 -117.70604746310367) bank7762 +7763 POINT(34.019546530396255 -117.46679845996722) bank7763 +7764 POINT(34.20683251837726 -118.37668205922435) bank7764 +7765 POINT(33.377271352425225 -117.72891765507394) bank7765 +7766 POINT(33.48019311655142 -118.55929380588803) bank7766 +7767 POINT(33.38671732519088 -118.6165762295947) bank7767 +7768 POINT(34.7744502092793 -119.13557086344565) bank7768 +7769 POINT(33.46208548432163 -117.77693139878492) bank7769 +7770 POINT(33.329783827323226 -118.62546974847201) bank7770 +7771 POINT(33.53041566023073 -118.15645329354787) bank7771 +7772 POINT(34.250094963982015 -119.02285999137538) bank7772 +7773 POINT(33.231030815648865 -119.0301497490902) bank7773 +7774 POINT(33.83432122183366 -118.4334813624709) bank7774 +7775 POINT(34.79434402024665 -119.03750917088284) bank7775 +7776 POINT(34.74527406127477 -117.80615716436341) bank7776 +7777 POINT(33.36319679195136 -117.93085067776595) bank7777 +7778 POINT(33.391240462831284 -117.49512547145405) bank7778 +7779 POINT(33.53545843520005 -117.35118114480258) bank7779 +7780 POINT(33.54290605154714 -118.27596716890262) bank7780 +7781 POINT(33.391120965382065 -118.46053872652311) bank7781 +7782 POINT(33.400836111609024 -117.90239099815517) bank7782 +7783 POINT(34.09960252464454 -118.48792361988916) bank7783 +7784 POINT(34.62254524193942 -118.00914949006396) bank7784 +7785 POINT(33.98169710803479 -118.59354138466924) bank7785 +7786 POINT(33.317281026932775 -117.99546753767348) bank7786 +7787 POINT(34.57050888147232 -117.51871720485494) bank7787 +7788 POINT(34.49585901966958 -117.78581597518496) bank7788 +7789 POINT(34.12513587750028 -117.90586710358157) bank7789 +7790 POINT(34.24876723174665 -117.49851756711726) bank7790 +7791 POINT(34.077178069562294 -117.52542421610019) bank7791 +7792 POINT(34.459133594676636 -117.85377682102816) bank7792 +7793 POINT(33.3587325173097 -118.88510360773438) bank7793 +7794 POINT(33.206274371264236 -118.94502296676322) bank7794 +7795 POINT(34.66341830620456 -118.59001829676797) bank7795 +7796 POINT(33.242394414976246 -118.23839538559749) bank7796 +7797 POINT(33.902964315162635 -118.82810768746917) bank7797 +7798 POINT(33.90105300614331 -118.68009460095324) bank7798 +7799 POINT(34.7222115509189 -117.67758105086855) bank7799 +7800 POINT(33.7263710119108 -118.71739012082452) bank7800 +7801 POINT(34.79030154515632 -117.48439859072225) bank7801 +7802 POINT(34.544813961669576 -118.57390608530471) bank7802 +7803 POINT(33.77684386055216 -118.21182135771537) bank7803 +7804 POINT(34.88751579494326 -118.58857554208207) bank7804 +7805 POINT(34.91571994980355 -117.27264299669116) bank7805 +7806 POINT(33.75924966825144 -118.74633314570745) bank7806 +7807 POINT(34.18979775500465 -118.72125232360663) bank7807 +7808 POINT(34.24216453243536 -118.01302884202984) bank7808 +7809 POINT(33.74838096386795 -119.0458273882797) bank7809 +7810 POINT(34.32943772785217 -117.69560453587737) bank7810 +7811 POINT(33.93426311675131 -118.5253871127731) bank7811 +7812 POINT(34.4524985055952 -118.45689436141475) bank7812 +7813 POINT(33.2311958938032 -117.99074887726734) bank7813 +7814 POINT(34.4432864954862 -117.7745196502911) bank7814 +7815 POINT(33.83154009084991 -117.45542929431491) bank7815 +7816 POINT(33.783138149461635 -118.6452738446977) bank7816 +7817 POINT(34.6451258718453 -118.38510478816691) bank7817 +7818 POINT(34.94417079353822 -117.77430037927867) bank7818 +7819 POINT(34.060908096585564 -117.43544698308095) bank7819 +7820 POINT(33.667759428810434 -118.92084209747995) bank7820 +7821 POINT(34.10416509858445 -117.456397053742) bank7821 +7822 POINT(34.0889067742495 -117.86341931215826) bank7822 +7823 POINT(34.92003720794141 -117.67234223282162) bank7823 +7824 POINT(33.10526833500444 -118.00055031759048) bank7824 +7825 POINT(33.42452889077311 -118.62488935909757) bank7825 +7826 POINT(35.03810030462147 -118.20497632118912) bank7826 +7827 POINT(34.527270049199565 -118.94126398632794) bank7827 +7828 POINT(33.409047315040205 -117.30443424350074) bank7828 +7829 POINT(33.364446646543776 -118.71236889104811) bank7829 +7830 POINT(34.40034660741847 -118.82764835350649) bank7830 +7831 POINT(34.87795826312736 -118.65522303918881) bank7831 +7832 POINT(34.637680811062296 -118.23764965323811) bank7832 +7833 POINT(33.68846666062704 -118.02821601667155) bank7833 +7834 POINT(34.52034004977747 -119.1428921735107) bank7834 +7835 POINT(34.099514978489594 -117.9010405731409) bank7835 +7836 POINT(33.87242773398345 -117.64581418464553) bank7836 +7837 POINT(34.536133316727216 -118.03369739537223) bank7837 +7838 POINT(34.16427798677393 -117.64153641719544) bank7838 +7839 POINT(33.475065526640016 -118.23802443120884) bank7839 +7840 POINT(33.93130489148496 -118.05078958993934) bank7840 +7841 POINT(34.85805643321386 -117.42919201828401) bank7841 +7842 POINT(34.707644212873596 -118.38405081525312) bank7842 +7843 POINT(34.02880938436755 -118.74624273595096) bank7843 +7844 POINT(34.255235668219726 -117.49178964671259) bank7844 +7845 POINT(34.14209022081048 -119.10478695495483) bank7845 +7846 POINT(34.619390286851946 -118.39750119155167) bank7846 +7847 POINT(33.951188537274106 -118.14555332399627) bank7847 +7848 POINT(34.285178922586525 -117.64504271647954) bank7848 +7849 POINT(33.35261680418847 -118.54274083213697) bank7849 +7850 POINT(33.09510805801019 -117.42482062913994) bank7850 +7851 POINT(34.36642093299198 -119.16430922188111) bank7851 +7852 POINT(34.91587294133005 -119.23245496795448) bank7852 +7853 POINT(34.79340673505807 -118.47341081878263) bank7853 +7854 POINT(33.76699925522039 -118.14673750986495) bank7854 +7855 POINT(34.109193377575906 -118.53039753288141) bank7855 +7856 POINT(33.395515292470726 -117.32554254265781) bank7856 +7857 POINT(34.23548197709322 -118.59695782618473) bank7857 +7858 POINT(33.46799658786091 -118.26477479055264) bank7858 +7859 POINT(34.10824734478389 -117.52576499328592) bank7859 +7860 POINT(35.02794533886951 -118.00344853694354) bank7860 +7861 POINT(34.43743033598105 -118.42646396393692) bank7861 +7862 POINT(34.85487848199479 -118.64235556852627) bank7862 +7863 POINT(34.64703050937313 -118.28644153246924) bank7863 +7864 POINT(33.87607697604252 -117.60941426616911) bank7864 +7865 POINT(34.70641679357461 -118.88322991204443) bank7865 +7866 POINT(34.257845808865014 -118.41363251813425) bank7866 +7867 POINT(33.468566403217466 -118.0282954167774) bank7867 +7868 POINT(34.91885327066407 -117.4997521506971) bank7868 +7869 POINT(34.16770459782037 -118.52962508244705) bank7869 +7870 POINT(34.75768762428316 -118.16005040548524) bank7870 +7871 POINT(34.13692359030523 -118.47072844568393) bank7871 +7872 POINT(34.23976595291438 -118.80884245378357) bank7872 +7873 POINT(34.71802595442494 -119.17447358873503) bank7873 +7874 POINT(33.81412020810934 -118.73289297983666) bank7874 +7875 POINT(34.26523398387676 -118.8692551352446) bank7875 +7876 POINT(34.91989301768333 -119.06468926500612) bank7876 +7877 POINT(34.28758812162818 -117.65773677433573) bank7877 +7878 POINT(34.53054879045239 -118.94103759332708) bank7878 +7879 POINT(34.13746397480561 -118.65575878806295) bank7879 +7880 POINT(34.902275657506046 -118.08477899908513) bank7880 +7881 POINT(34.18176780836045 -117.42611692028564) bank7881 +7882 POINT(33.061682160927404 -117.2887693062704) bank7882 +7883 POINT(34.39043637587006 -118.0311239660687) bank7883 +7884 POINT(34.093646878994825 -118.92548871904356) bank7884 +7885 POINT(33.76532787492863 -118.96544320728415) bank7885 +7886 POINT(34.578717275807826 -118.5749019639508) bank7886 +7887 POINT(34.25485190705601 -118.6455051396127) bank7887 +7888 POINT(34.17808696461039 -118.59241137741397) bank7888 +7889 POINT(33.53676797758472 -117.54444181855227) bank7889 +7890 POINT(34.83083263584444 -117.7932526339558) bank7890 +7891 POINT(34.255792524372865 -118.80035979220617) bank7891 +7892 POINT(33.215466780246345 -118.49903662121325) bank7892 +7893 POINT(34.654538356965375 -118.77385262189529) bank7893 +7894 POINT(34.3202350000048 -117.89200521595419) bank7894 +7895 POINT(34.52932121674841 -118.47875396028753) bank7895 +7896 POINT(33.54891423414642 -118.1438378384456) bank7896 +7897 POINT(34.78011380029868 -118.32024272871452) bank7897 +7898 POINT(33.374868603367844 -119.21057790876854) bank7898 +7899 POINT(33.19915562804287 -118.11550654518919) bank7899 +7900 POINT(34.54194952089099 -118.92814485534704) bank7900 +7901 POINT(33.20393344421542 -118.88574199953817) bank7901 +7902 POINT(34.10557106629028 -117.41867705948056) bank7902 +7903 POINT(34.96663590071159 -118.97175413162087) bank7903 +7904 POINT(34.52143309872255 -118.79033483409003) bank7904 +7905 POINT(35.01381319760163 -117.28878092320093) bank7905 +7906 POINT(34.12586306233045 -118.43292672863348) bank7906 +7907 POINT(33.89506309306031 -119.11133271181761) bank7907 +7908 POINT(34.99991713233812 -118.02116662043866) bank7908 +7909 POINT(33.95728238698906 -119.04015842546056) bank7909 +7910 POINT(34.980272198067865 -118.59074285848422) bank7910 +7911 POINT(33.46821908647775 -117.33100681541465) bank7911 +7912 POINT(33.96276267191572 -117.49243787550245) bank7912 +7913 POINT(34.31287947226812 -119.08182787646837) bank7913 +7914 POINT(34.80846183706339 -118.16631515461913) bank7914 +7915 POINT(34.63419615793392 -118.63473565732548) bank7915 +7916 POINT(33.960223785334634 -118.3017143119661) bank7916 +7917 POINT(33.5007721983635 -117.80288530426706) bank7917 +7918 POINT(33.736728799160595 -118.69934359891732) bank7918 +7919 POINT(33.715185197446466 -118.34142405504275) bank7919 +7920 POINT(33.55151286628603 -118.338066210245) bank7920 +7921 POINT(33.37684494423808 -117.85597938167666) bank7921 +7922 POINT(34.29211802218081 -119.0596507323205) bank7922 +7923 POINT(33.17386195695412 -117.89767013615119) bank7923 +7924 POINT(33.831189135568664 -117.81474595187746) bank7924 +7925 POINT(34.80647974833205 -118.81972531592709) bank7925 +7926 POINT(34.60749616262648 -117.80866420780733) bank7926 +7927 POINT(33.22406637647418 -118.5747499821194) bank7927 +7928 POINT(33.16161544969817 -118.16020986844269) bank7928 +7929 POINT(33.29822812326563 -118.6952189944884) bank7929 +7930 POINT(33.068195825160984 -119.10745748453839) bank7930 +7931 POINT(34.46487826502467 -117.84036993688252) bank7931 +7932 POINT(33.90777697277745 -119.10620924178453) bank7932 +7933 POINT(33.247948468463385 -118.57581921163376) bank7933 +7934 POINT(33.80631330721522 -117.92912768886478) bank7934 +7935 POINT(33.40156123516934 -118.07843121791821) bank7935 +7936 POINT(34.469413977865585 -117.95398803287895) bank7936 +7937 POINT(34.36170384316806 -117.91943428250505) bank7937 +7938 POINT(34.96954904838129 -118.43061505776585) bank7938 +7939 POINT(33.75053940490294 -119.15151142333318) bank7939 +7940 POINT(33.43852928528618 -117.6966390569334) bank7940 +7941 POINT(34.06171956539901 -118.73452607733503) bank7941 +7942 POINT(33.60336911490868 -118.19402658712876) bank7942 +7943 POINT(34.37580938920408 -118.21317151765717) bank7943 +7944 POINT(34.821718224548405 -118.66612330427229) bank7944 +7945 POINT(34.819151115979786 -117.45044328289117) bank7945 +7946 POINT(34.46286408215707 -117.76926401932826) bank7946 +7947 POINT(34.71604896726932 -118.78231358037053) bank7947 +7948 POINT(34.78559364483449 -117.51729551924274) bank7948 +7949 POINT(34.16446589986666 -117.90075261927149) bank7949 +7950 POINT(34.92323806735689 -117.39935283247233) bank7950 +7951 POINT(33.060319940417145 -118.93359242607889) bank7951 +7952 POINT(34.296045808468726 -118.51453956608775) bank7952 +7953 POINT(34.15809456823848 -117.78443588354078) bank7953 +7954 POINT(35.0034075035946 -118.99202463463646) bank7954 +7955 POINT(33.95806310608017 -118.95524453069596) bank7955 +7956 POINT(33.39150745071141 -117.78555675944372) bank7956 +7957 POINT(34.342205851064094 -118.7728428336214) bank7957 +7958 POINT(34.63264503379205 -117.86697057459979) bank7958 +7959 POINT(33.14154841806348 -117.89550709018818) bank7959 +7960 POINT(34.853632739143656 -118.40330231181464) bank7960 +7961 POINT(33.549159014501804 -118.88663691242994) bank7961 +7962 POINT(33.593711202498724 -117.83370180874792) bank7962 +7963 POINT(33.44288366366558 -117.56066996711009) bank7963 +7964 POINT(33.29516364262185 -117.56899876943604) bank7964 +7965 POINT(34.112269234319086 -117.47639517812586) bank7965 +7966 POINT(33.24435262117894 -117.50592538650932) bank7966 +7967 POINT(35.04182521748887 -117.674192065698) bank7967 +7968 POINT(34.45599568711947 -118.10668053324171) bank7968 +7969 POINT(34.77662322048768 -117.98276119117168) bank7969 +7970 POINT(33.91363989687625 -118.77372742812365) bank7970 +7971 POINT(34.4390511426946 -119.12849791737031) bank7971 +7972 POINT(34.6436166947999 -118.50247489353588) bank7972 +7973 POINT(33.72463616641612 -117.41360243724547) bank7973 +7974 POINT(33.42681277408675 -117.92478719817449) bank7974 +7975 POINT(34.60911144318187 -118.07791998147896) bank7975 +7976 POINT(34.77056587280914 -118.6147039002718) bank7976 +7977 POINT(33.938368034191306 -117.60294433513268) bank7977 +7978 POINT(33.89048709910947 -117.6065255100035) bank7978 +7979 POINT(34.4267914646879 -118.9858579234381) bank7979 +7980 POINT(34.27013366928192 -117.86255837911703) bank7980 +7981 POINT(34.50152637517452 -118.71759358102875) bank7981 +7982 POINT(34.37632664844004 -119.09152118995466) bank7982 +7983 POINT(33.56743634813919 -117.78767520277906) bank7983 +7984 POINT(34.408047936883726 -118.03921309829778) bank7984 +7985 POINT(33.54638208669934 -118.22131444440924) bank7985 +7986 POINT(33.68853180166988 -118.37353174416465) bank7986 +7987 POINT(34.79315297063759 -117.39445503282363) bank7987 +7988 POINT(34.46797675946956 -117.89779716007243) bank7988 +7989 POINT(34.19573992008338 -117.78993815300491) bank7989 +7990 POINT(34.01715722794781 -117.38830537381538) bank7990 +7991 POINT(33.53658008341812 -117.47506572705818) bank7991 +7992 POINT(33.4619863350556 -119.0560576567691) bank7992 +7993 POINT(34.32687229588008 -117.60223078922678) bank7993 +7994 POINT(33.19995337511042 -117.29095773410756) bank7994 +7995 POINT(33.32440507722316 -118.29632658255525) bank7995 +7996 POINT(33.47018684539344 -118.9653281098979) bank7996 +7997 POINT(34.49628095265864 -119.11998175361096) bank7997 +7998 POINT(34.0457122965966 -118.20723263807092) bank7998 +7999 POINT(34.856173933139296 -119.0403557614024) bank7999 +8000 POINT(34.96852793642073 -117.84361180616708) bank8000 +8001 POINT(34.65770371405419 -117.33538133669727) bank8001 +8002 POINT(33.371519990649745 -118.18556149325232) bank8002 +8003 POINT(33.94544436578917 -118.91784274951527) bank8003 +8004 POINT(34.853481313801645 -118.40820326955588) bank8004 +8005 POINT(33.43255974590589 -118.88389976453824) bank8005 +8006 POINT(33.60781822648995 -117.28818983721345) bank8006 +8007 POINT(33.476625506771256 -118.34164480108134) bank8007 +8008 POINT(35.02288137114212 -118.31988148057823) bank8008 +8009 POINT(33.260251721330995 -119.1402013282517) bank8009 +8010 POINT(34.53934655443949 -118.30942550548846) bank8010 +8011 POINT(33.90617626113591 -119.0040665675365) bank8011 +8012 POINT(34.9289129158954 -118.60914179526962) bank8012 +8013 POINT(33.41607242560369 -118.81325548880658) bank8013 +8014 POINT(35.0459264151412 -117.45920426038117) bank8014 +8015 POINT(33.36077006650702 -118.16980013279763) bank8015 +8016 POINT(33.279016600522375 -118.48282063500756) bank8016 +8017 POINT(34.05862207939711 -117.82174698688249) bank8017 +8018 POINT(34.67556901939653 -117.7057768563192) bank8018 +8019 POINT(33.60770564463882 -119.12896937498195) bank8019 +8020 POINT(33.607413490656945 -117.31386890120939) bank8020 +8021 POINT(34.32888846563164 -118.96475398773713) bank8021 +8022 POINT(34.98006336862534 -117.77934979438227) bank8022 +8023 POINT(34.374377096734236 -117.72229945642462) bank8023 +8024 POINT(34.49917182209388 -118.78810098760839) bank8024 +8025 POINT(34.78403985481214 -118.58284443450883) bank8025 +8026 POINT(33.879361353980215 -117.65560698444993) bank8026 +8027 POINT(33.587019486146524 -118.86306421556628) bank8027 +8028 POINT(33.43130888540992 -118.90673679034632) bank8028 +8029 POINT(34.11839588345978 -117.48098010306758) bank8029 +8030 POINT(34.552386355018776 -118.82684085730332) bank8030 +8031 POINT(33.532524743075946 -117.84271905678546) bank8031 +8032 POINT(33.120326902664296 -117.67165322166638) bank8032 +8033 POINT(34.38519537452576 -118.14190524709238) bank8033 +8034 POINT(34.65384991845022 -118.74096179629781) bank8034 +8035 POINT(34.732044643827244 -117.77446728587327) bank8035 +8036 POINT(34.0863285716684 -118.64115116092712) bank8036 +8037 POINT(34.355413826748794 -117.47774053556482) bank8037 +8038 POINT(33.15528970557065 -119.18917426856973) bank8038 +8039 POINT(34.909963426272625 -117.59195021458466) bank8039 +8040 POINT(34.079145440881284 -118.95360262384692) bank8040 +8041 POINT(34.16721858998852 -117.94036203708707) bank8041 +8042 POINT(33.92733810912203 -118.6488141139735) bank8042 +8043 POINT(33.83705733085951 -117.29026102261678) bank8043 +8044 POINT(33.902707280821694 -118.44530549030868) bank8044 +8045 POINT(34.460050489873495 -119.16811624362076) bank8045 +8046 POINT(33.22092343816795 -117.47371599685209) bank8046 +8047 POINT(34.959383196626526 -119.21245285367407) bank8047 +8048 POINT(35.00328692411828 -117.2752313767223) bank8048 +8049 POINT(34.30245338410216 -117.25840230070642) bank8049 +8050 POINT(34.24684724173202 -118.54671839693444) bank8050 +8051 POINT(33.76031888860734 -117.7622629771529) bank8051 +8052 POINT(34.917987017848354 -117.72696638553784) bank8052 +8053 POINT(33.75472739835325 -118.77201088299061) bank8053 +8054 POINT(34.68460257668142 -118.4393861362918) bank8054 +8055 POINT(33.74982549124636 -118.95799661923482) bank8055 +8056 POINT(33.61431896336028 -118.587314414217) bank8056 +8057 POINT(33.37199600779282 -117.95534518460164) bank8057 +8058 POINT(34.32755935224178 -118.26417639662765) bank8058 +8059 POINT(33.18341823925211 -118.62181427524948) bank8059 +8060 POINT(34.92978813581822 -117.7448719537503) bank8060 +8061 POINT(34.716951591181285 -118.46923725001906) bank8061 +8062 POINT(33.53899565206307 -118.0289845765626) bank8062 +8063 POINT(34.28793534095413 -118.10288124720064) bank8063 +8064 POINT(33.29696325565674 -117.62049408609644) bank8064 +8065 POINT(34.57482852818051 -118.1912125564898) bank8065 +8066 POINT(34.28856544810979 -117.70239296796923) bank8066 +8067 POINT(34.42200602399563 -119.20505671886413) bank8067 +8068 POINT(34.763667629105 -118.96931063125601) bank8068 +8069 POINT(33.91420056897947 -118.09069253060144) bank8069 +8070 POINT(34.900534227356935 -119.08405729161458) bank8070 +8071 POINT(34.57028521342077 -118.80772901842305) bank8071 +8072 POINT(33.88210714499782 -119.06990949306652) bank8072 +8073 POINT(34.82983507774524 -118.64429857967092) bank8073 +8074 POINT(33.28938362165241 -118.7097172198215) bank8074 +8075 POINT(33.271348028828896 -118.43644637727736) bank8075 +8076 POINT(34.635988960514425 -118.46065581614096) bank8076 +8077 POINT(34.47259584309982 -117.3073049492493) bank8077 +8078 POINT(34.224933311553926 -119.0685939330611) bank8078 +8079 POINT(34.75864827604746 -117.8033390305179) bank8079 +8080 POINT(33.800454388444884 -117.96032746253543) bank8080 +8081 POINT(33.137787519889095 -117.82870062940064) bank8081 +8082 POINT(34.16797062650899 -118.02380603222299) bank8082 +8083 POINT(33.29005170073633 -117.8022252052682) bank8083 +8084 POINT(33.496025605996344 -117.93574250224503) bank8084 +8085 POINT(33.430192171008045 -118.12144073746076) bank8085 +8086 POINT(33.51283715656547 -118.055750896506) bank8086 +8087 POINT(33.62472752287926 -117.41800595144136) bank8087 +8088 POINT(33.37367585417646 -118.29030457787485) bank8088 +8089 POINT(34.287408688653755 -117.4334704553965) bank8089 +8090 POINT(34.834709060313706 -117.35150794776597) bank8090 +8091 POINT(34.94339265907462 -117.42736970143375) bank8091 +8092 POINT(33.83218300089324 -118.20280977201953) bank8092 +8093 POINT(33.731865872369475 -118.56062580231456) bank8093 +8094 POINT(33.44172143150765 -118.4184665818752) bank8094 +8095 POINT(33.21160722277119 -117.34800916363534) bank8095 +8096 POINT(34.5771789909973 -118.93382554956663) bank8096 +8097 POINT(33.743327851562505 -118.61907275057649) bank8097 +8098 POINT(33.63822384335284 -118.29602594894725) bank8098 +8099 POINT(34.53213687491869 -118.58241052500556) bank8099 +8100 POINT(34.196857962501326 -118.72177980050124) bank8100 +8101 POINT(34.18579427996431 -118.76489288298112) bank8101 +8102 POINT(33.289732630763716 -118.1360322133568) bank8102 +8103 POINT(34.11417668119808 -117.47870805749892) bank8103 +8104 POINT(34.64416806640193 -117.47410521210652) bank8104 +8105 POINT(33.23838673455981 -118.71858946322556) bank8105 +8106 POINT(33.77675477433252 -118.30344567595372) bank8106 +8107 POINT(34.237412426878244 -118.95640677913391) bank8107 +8108 POINT(33.6099300900568 -119.03699059768351) bank8108 +8109 POINT(34.93230268375134 -118.71167683250276) bank8109 +8110 POINT(34.08185159639882 -117.65229160219039) bank8110 +8111 POINT(33.058131184130325 -118.77780526741354) bank8111 +8112 POINT(34.5312093450949 -118.16271995624128) bank8112 +8113 POINT(33.954051568661285 -119.04926851626436) bank8113 +8114 POINT(33.073501398700216 -117.93539133318586) bank8114 +8115 POINT(33.28621855349769 -118.74627829440132) bank8115 +8116 POINT(33.592033064409996 -117.57268625445491) bank8116 +8117 POINT(33.19843541341485 -118.15159554811258) bank8117 +8118 POINT(33.30598991153716 -118.16286122019223) bank8118 +8119 POINT(33.67786585418702 -117.68717685304037) bank8119 +8120 POINT(34.883593716110894 -118.38592483469786) bank8120 +8121 POINT(34.40007122964821 -118.58592062374714) bank8121 +8122 POINT(33.202494991162126 -118.69681008191569) bank8122 +8123 POINT(34.20558768546205 -118.77176793028224) bank8123 +8124 POINT(34.4074445538206 -119.14671255052465) bank8124 +8125 POINT(33.734836329101675 -119.00360250368954) bank8125 +8126 POINT(34.166131613067144 -117.86098184594611) bank8126 +8127 POINT(34.01284198191206 -118.60942726340146) bank8127 +8128 POINT(34.138843077620336 -117.96375472311696) bank8128 +8129 POINT(34.87388878212226 -117.76547574119081) bank8129 +8130 POINT(33.72848473397459 -117.90440847898626) bank8130 +8131 POINT(33.1450372846798 -117.45685178878222) bank8131 +8132 POINT(33.05940260817956 -119.0700633887142) bank8132 +8133 POINT(34.53545695824227 -118.29402217228821) bank8133 +8134 POINT(33.953117658551676 -117.47344932587748) bank8134 +8135 POINT(33.186578659487864 -117.33826558148866) bank8135 +8136 POINT(33.52713916363138 -117.45777283698042) bank8136 +8137 POINT(33.19960669696083 -118.26482240711712) bank8137 +8138 POINT(34.28031239218164 -119.24241495962423) bank8138 +8139 POINT(34.74809336819843 -118.94356168053227) bank8139 +8140 POINT(33.61101151285543 -118.06777576746849) bank8140 +8141 POINT(34.59112192385427 -118.8171439593881) bank8141 +8142 POINT(34.26176040656951 -118.98538763636742) bank8142 +8143 POINT(34.85127424953502 -117.33320025288097) bank8143 +8144 POINT(33.931996265283 -118.9867008603044) bank8144 +8145 POINT(34.980664224918506 -117.27278776778121) bank8145 +8146 POINT(34.44886792189746 -118.50871802076018) bank8146 +8147 POINT(34.45393400229663 -118.4476672010148) bank8147 +8148 POINT(34.062825860537814 -118.83689387353606) bank8148 +8149 POINT(33.984371765438965 -118.70511654261105) bank8149 +8150 POINT(34.83110951979499 -118.76480750706753) bank8150 +8151 POINT(34.59841317317514 -117.57785945440443) bank8151 +8152 POINT(34.71418159560939 -118.82532388844609) bank8152 +8153 POINT(34.43083061073244 -119.242191509319) bank8153 +8154 POINT(34.398045215756795 -118.95405341072006) bank8154 +8155 POINT(34.70599045209503 -118.4495615572051) bank8155 +8156 POINT(33.7048894793068 -118.09572863160416) bank8156 +8157 POINT(33.36427339167583 -118.33409089014138) bank8157 +8158 POINT(33.61409957998896 -117.97208866475229) bank8158 +8159 POINT(33.63448141150943 -117.32874199322892) bank8159 +8160 POINT(33.916306039991674 -118.47489505837956) bank8160 +8161 POINT(33.52031623497729 -119.16406202935703) bank8161 +8162 POINT(34.316484204452856 -117.81043188109723) bank8162 +8163 POINT(33.960734081594225 -119.22799340708475) bank8163 +8164 POINT(34.12923677636731 -118.67606314186632) bank8164 +8165 POINT(34.98948677089465 -117.7987186709087) bank8165 +8166 POINT(34.532503475250145 -117.31739995420106) bank8166 +8167 POINT(34.576527490486434 -118.63480970272) bank8167 +8168 POINT(33.44500063770727 -117.34863142789906) bank8168 +8169 POINT(34.20034910362644 -117.94961461592655) bank8169 +8170 POINT(33.18079858421826 -117.41146192732815) bank8170 +8171 POINT(33.61538660551235 -118.44780028259935) bank8171 +8172 POINT(33.308698178027086 -117.62831328589047) bank8172 +8173 POINT(34.468186422414774 -118.64809008733162) bank8173 +8174 POINT(33.310244524746516 -118.61598616397066) bank8174 +8175 POINT(34.75662131676281 -117.6717216688705) bank8175 +8176 POINT(34.4518974589133 -117.8763746212517) bank8176 +8177 POINT(34.43852049755774 -117.32545195699034) bank8177 +8178 POINT(34.57902805634564 -118.45594126338274) bank8178 +8179 POINT(34.208042602866236 -118.93060408707072) bank8179 +8180 POINT(34.16696009947666 -118.32641410391007) bank8180 +8181 POINT(34.97106991755905 -117.72212116548525) bank8181 +8182 POINT(33.99079005999626 -117.60177414867985) bank8182 +8183 POINT(33.242178567068976 -118.74968105509525) bank8183 +8184 POINT(34.30496980316245 -117.77392302671844) bank8184 +8185 POINT(34.28804172887612 -117.37547792610289) bank8185 +8186 POINT(33.94659480477412 -117.47676642661284) bank8186 +8187 POINT(33.77729243162354 -117.46264478916981) bank8187 +8188 POINT(33.85758911106832 -119.08487568691257) bank8188 +8189 POINT(34.63959676689065 -119.21485768153256) bank8189 +8190 POINT(33.87825542324916 -119.17668948415364) bank8190 +8191 POINT(34.72374456206442 -117.37459981507318) bank8191 +8192 POINT(34.04883629492803 -117.49205007574852) bank8192 +8193 POINT(33.54255483444261 -119.09957349355865) bank8193 +8194 POINT(33.304905627095515 -118.49291261982913) bank8194 +8195 POINT(34.24677321981413 -117.91187454514797) bank8195 +8196 POINT(34.27838666866085 -117.44285164089064) bank8196 +8197 POINT(34.95988448878786 -118.06540596463353) bank8197 +8198 POINT(33.253975119668 -118.9349668815258) bank8198 +8199 POINT(33.66985854165672 -118.18803540494905) bank8199 +8200 POINT(33.49950563013195 -118.3185351461719) bank8200 +8201 POINT(34.74744604919818 -117.86155193130782) bank8201 +8202 POINT(34.01204791598608 -117.83805172949617) bank8202 +8203 POINT(34.16217285641246 -118.6404569369195) bank8203 +8204 POINT(34.599947947402626 -118.64983652415495) bank8204 +8205 POINT(33.226966767915485 -118.34610733584306) bank8205 +8206 POINT(34.774657136801636 -119.18575874674575) bank8206 +8207 POINT(34.58229633684544 -118.46513240086045) bank8207 +8208 POINT(33.709301456668925 -117.61883359077636) bank8208 +8209 POINT(33.9385972885446 -117.96141056825128) bank8209 +8210 POINT(34.918223887850225 -118.79690746339766) bank8210 +8211 POINT(34.04113721372858 -117.47777890764618) bank8211 +8212 POINT(34.052568067861074 -117.7253052694515) bank8212 +8213 POINT(34.18386599931881 -117.8460140891123) bank8213 +8214 POINT(33.645122177324374 -117.72194022510743) bank8214 +8215 POINT(33.89570696072104 -117.44150925062348) bank8215 +8216 POINT(33.19512227206461 -117.41637389156769) bank8216 +8217 POINT(33.745691101480936 -118.2233300144598) bank8217 +8218 POINT(34.932693433851775 -118.67350650048196) bank8218 +8219 POINT(33.37196426959591 -118.97459409152317) bank8219 +8220 POINT(33.76754575074373 -117.80535552183069) bank8220 +8221 POINT(33.56596270523681 -119.19525604524247) bank8221 +8222 POINT(33.730214034489634 -117.57215388559538) bank8222 +8223 POINT(33.77855203187677 -117.40863043366426) bank8223 +8224 POINT(34.604853873871896 -117.85997685615011) bank8224 +8225 POINT(34.79563214312926 -118.94313918781434) bank8225 +8226 POINT(34.64191172676443 -117.82446170428857) bank8226 +8227 POINT(34.76628785580522 -117.65617860371123) bank8227 +8228 POINT(34.483919306852265 -117.66317063451574) bank8228 +8229 POINT(33.3877910210481 -118.60137094540995) bank8229 +8230 POINT(35.04721047869505 -117.40847702302096) bank8230 +8231 POINT(34.935263823557825 -119.13861424040137) bank8231 +8232 POINT(35.00567659452626 -119.11166812881822) bank8232 +8233 POINT(34.208849850566345 -117.88797196318333) bank8233 +8234 POINT(35.015503037262825 -118.9027879787837) bank8234 +8235 POINT(33.060256116867635 -118.75250746415925) bank8235 +8236 POINT(34.92236977591258 -118.32907276523935) bank8236 +8237 POINT(34.55016315444636 -118.13243967429821) bank8237 +8238 POINT(33.55253937069138 -117.90202087232227) bank8238 +8239 POINT(34.31022491862884 -117.75073117059907) bank8239 +8240 POINT(33.825515327795024 -117.82664379964628) bank8240 +8241 POINT(34.421982628424544 -117.62285437892048) bank8241 +8242 POINT(33.35273919267447 -118.46236350488385) bank8242 +8243 POINT(33.1777146457903 -117.28616821421971) bank8243 +8244 POINT(35.02419660424168 -118.01304410393408) bank8244 +8245 POINT(33.16752816020346 -118.83281957214248) bank8245 +8246 POINT(34.5798243446622 -117.959971537538) bank8246 +8247 POINT(33.126445963369754 -117.6608008842928) bank8247 +8248 POINT(33.36209723888842 -117.27408774715992) bank8248 +8249 POINT(34.098767306530796 -117.86074745513454) bank8249 +8250 POINT(33.417362872869944 -118.62551504138303) bank8250 +8251 POINT(33.35574279223577 -118.06802219085702) bank8251 +8252 POINT(34.63999784965361 -117.66972696741009) bank8252 +8253 POINT(34.52956798032605 -118.16971987110068) bank8253 +8254 POINT(34.99554541020387 -118.30535887457522) bank8254 +8255 POINT(34.955270437234255 -119.22719366655737) bank8255 +8256 POINT(33.96606860889941 -117.89969471263855) bank8256 +8257 POINT(33.173461403675326 -118.91019545997011) bank8257 +8258 POINT(33.74780549615916 -117.6970181344462) bank8258 +8259 POINT(34.94512713321 -118.29459497844307) bank8259 +8260 POINT(34.94984919480514 -117.75761687692011) bank8260 +8261 POINT(33.25630984628187 -118.25476170085336) bank8261 +8262 POINT(34.08084590045426 -117.25129111828721) bank8262 +8263 POINT(34.315318314007406 -118.3386337539375) bank8263 +8264 POINT(34.51641469442073 -118.56654249150796) bank8264 +8265 POINT(34.77476748892005 -118.2577840377005) bank8265 +8266 POINT(33.455260206709625 -118.49247106976779) bank8266 +8267 POINT(34.858127140088364 -118.55902252854784) bank8267 +8268 POINT(33.696141255784504 -118.92296935544613) bank8268 +8269 POINT(34.22887220534097 -118.76720603194572) bank8269 +8270 POINT(34.78457038596712 -117.7925106672149) bank8270 +8271 POINT(34.96698019068931 -118.20203203445031) bank8271 +8272 POINT(33.43459573692929 -118.35574844613393) bank8272 +8273 POINT(34.82120828725663 -117.41307306642659) bank8273 +8274 POINT(33.567469845144736 -119.10888508550185) bank8274 +8275 POINT(34.173854320751595 -118.6302770745145) bank8275 +8276 POINT(34.84589074313448 -118.65719584928982) bank8276 +8277 POINT(33.76732228454828 -118.84700354798045) bank8277 +8278 POINT(34.38564995338544 -118.28995309778284) bank8278 +8279 POINT(34.638985309865035 -117.70985133263198) bank8279 +8280 POINT(34.335177157486186 -118.5157436693656) bank8280 +8281 POINT(34.832426658150474 -117.62619708676326) bank8281 +8282 POINT(34.4230115243361 -118.22686781984247) bank8282 +8283 POINT(34.6213663984674 -119.22934594498143) bank8283 +8284 POINT(34.9368590076798 -117.56558325534904) bank8284 +8285 POINT(33.512538817064204 -119.16102378825045) bank8285 +8286 POINT(34.79298013247242 -118.1919972657304) bank8286 +8287 POINT(34.82200924304919 -118.97667571556542) bank8287 +8288 POINT(33.99301972445982 -118.20266127285032) bank8288 +8289 POINT(33.88109359416261 -117.44991049135726) bank8289 +8290 POINT(33.13721456091234 -117.5271450924919) bank8290 +8291 POINT(34.696444852922454 -117.57241174138201) bank8291 +8292 POINT(33.15269439867648 -117.37497818651208) bank8292 +8293 POINT(33.95289053059163 -117.40852522598794) bank8293 +8294 POINT(34.571870359605384 -118.44319477547457) bank8294 +8295 POINT(33.558872798402824 -117.94764754932176) bank8295 +8296 POINT(34.442957235482346 -118.27752939848821) bank8296 +8297 POINT(34.343301117956564 -118.25226330298293) bank8297 +8298 POINT(33.77856097049496 -119.03667317703675) bank8298 +8299 POINT(33.35703576338167 -118.42829275889) bank8299 +8300 POINT(34.186586767337566 -118.40768880979725) bank8300 +8301 POINT(33.90179666850227 -117.67245997227984) bank8301 +8302 POINT(33.256812336873125 -118.35958522802947) bank8302 +8303 POINT(33.73523667119318 -117.43519914139594) bank8303 +8304 POINT(33.841328778784195 -118.38362752671156) bank8304 +8305 POINT(34.023397808726536 -119.07819009982167) bank8305 +8306 POINT(33.39493241932723 -118.34330031611243) bank8306 +8307 POINT(33.45853593532754 -117.49785356880993) bank8307 +8308 POINT(34.48710138908259 -119.06071542560387) bank8308 +8309 POINT(34.75337069295759 -117.87511918376146) bank8309 +8310 POINT(33.737211231124576 -119.05319203133249) bank8310 +8311 POINT(33.147978711709754 -117.25703887261497) bank8311 +8312 POINT(33.211281551321605 -118.1791444464333) bank8312 +8313 POINT(33.74075321655796 -118.92632996558987) bank8313 +8314 POINT(34.689030514611986 -117.27866034883989) bank8314 +8315 POINT(33.43104399359689 -117.9981047896446) bank8315 +8316 POINT(33.082026360648044 -118.4077158573707) bank8316 +8317 POINT(34.544672534666205 -118.26697172033704) bank8317 +8318 POINT(34.906251275460654 -118.31845552452405) bank8318 +8319 POINT(33.65895454003584 -117.8987707921834) bank8319 +8320 POINT(34.089853216938124 -118.21853986913132) bank8320 +8321 POINT(34.672995140141346 -117.89716860511307) bank8321 +8322 POINT(33.10098017116019 -117.42934539683382) bank8322 +8323 POINT(34.84917696076326 -118.44227984035913) bank8323 +8324 POINT(34.16208910657285 -118.88121548578461) bank8324 +8325 POINT(34.03860186003064 -117.37235964996597) bank8325 +8326 POINT(33.2319542294297 -118.21208653360759) bank8326 +8327 POINT(33.56524988791839 -119.03272243488193) bank8327 +8328 POINT(34.542510740688975 -117.87371329958802) bank8328 +8329 POINT(33.07231840885407 -118.90799423485375) bank8329 +8330 POINT(33.53463119783268 -117.81215038573175) bank8330 +8331 POINT(33.20698664339596 -119.0371740146899) bank8331 +8332 POINT(34.92438081907433 -118.48056476671253) bank8332 +8333 POINT(34.115463695782864 -118.19329684436086) bank8333 +8334 POINT(33.84771571977156 -118.29624434397331) bank8334 +8335 POINT(33.55062699551837 -118.66331928102022) bank8335 +8336 POINT(34.306280925397395 -118.50971778640806) bank8336 +8337 POINT(33.72573925072143 -117.45715324932797) bank8337 +8338 POINT(34.19149633764701 -117.57587025460039) bank8338 +8339 POINT(33.622502684370886 -118.98927649769489) bank8339 +8340 POINT(34.11331737517204 -118.01713257176297) bank8340 +8341 POINT(34.697413885348524 -118.34507294136415) bank8341 +8342 POINT(33.63090642286751 -118.52297429336325) bank8342 +8343 POINT(33.300920717213145 -118.57157510001905) bank8343 +8344 POINT(34.67435863298858 -118.31642665901732) bank8344 +8345 POINT(33.963541659666646 -117.25993822370116) bank8345 +8346 POINT(33.75397030694844 -118.96709634406037) bank8346 +8347 POINT(33.45380800064063 -117.62830456415769) bank8347 +8348 POINT(34.12963481897635 -118.73615200851064) bank8348 +8349 POINT(33.52390284301161 -119.1604552732062) bank8349 +8350 POINT(34.93172716371099 -118.44532581564401) bank8350 +8351 POINT(33.50710620511436 -117.95259992011879) bank8351 +8352 POINT(33.866705811996376 -117.28714107633755) bank8352 +8353 POINT(34.19064891749773 -118.76946895568699) bank8353 +8354 POINT(34.83578599849116 -117.33970035173344) bank8354 +8355 POINT(34.51238429253432 -117.88743074428022) bank8355 +8356 POINT(33.33994729319623 -117.30688913178771) bank8356 +8357 POINT(33.412221742855515 -117.47454162528584) bank8357 +8358 POINT(34.88411024064029 -117.65517551321963) bank8358 +8359 POINT(33.619730070276056 -118.45095831272363) bank8359 +8360 POINT(33.803363643841486 -118.55443254902521) bank8360 +8361 POINT(33.175639004619356 -118.33815352883715) bank8361 +8362 POINT(34.452711332413486 -119.18239326567539) bank8362 +8363 POINT(33.706985836595365 -118.58280634050897) bank8363 +8364 POINT(34.41947316607748 -118.78071262727325) bank8364 +8365 POINT(34.39482269764281 -117.53584212613) bank8365 +8366 POINT(33.06067533632851 -117.7578123432872) bank8366 +8367 POINT(33.883390064294545 -118.46340251883684) bank8367 +8368 POINT(34.95071001231829 -119.07863220566652) bank8368 +8369 POINT(33.1447600690169 -118.03484692914694) bank8369 +8370 POINT(33.22923130128439 -117.68815074278002) bank8370 +8371 POINT(34.51388558618766 -117.56581238098659) bank8371 +8372 POINT(34.22243672707014 -117.82137946553986) bank8372 +8373 POINT(35.02719926266244 -118.87510793505399) bank8373 +8374 POINT(33.4101226665795 -117.48305528029805) bank8374 +8375 POINT(34.24422969817019 -117.2643562055929) bank8375 +8376 POINT(34.94341432665845 -117.97664064202213) bank8376 +8377 POINT(34.76556773014834 -118.54536544850691) bank8377 +8378 POINT(34.717643660931806 -118.82006704202637) bank8378 +8379 POINT(34.03720733377961 -117.53564629520555) bank8379 +8380 POINT(33.71210575727384 -117.47459555466806) bank8380 +8381 POINT(34.837670465818434 -119.02839151246175) bank8381 +8382 POINT(34.171448172578636 -119.14305336379311) bank8382 +8383 POINT(33.85563283373303 -118.60926373567015) bank8383 +8384 POINT(34.365178246190446 -119.22290190641222) bank8384 +8385 POINT(33.90977710887637 -117.44915271047174) bank8385 +8386 POINT(33.41607023834735 -119.01227836937151) bank8386 +8387 POINT(33.347107993696305 -119.08181564952916) bank8387 +8388 POINT(33.10851806583053 -118.47135926718656) bank8388 +8389 POINT(34.38677083358021 -118.70992322484639) bank8389 +8390 POINT(34.423929271723175 -119.04048970554508) bank8390 +8391 POINT(34.25958190157689 -117.48615122800389) bank8391 +8392 POINT(34.462208913907986 -118.16300581525529) bank8392 +8393 POINT(34.01852286763317 -118.90760961172667) bank8393 +8394 POINT(33.89219416418568 -118.33297944076632) bank8394 +8395 POINT(33.60916086886719 -118.6398512703674) bank8395 +8396 POINT(33.37838452882603 -117.639103579304) bank8396 +8397 POINT(34.25966232397741 -118.0261240954066) bank8397 +8398 POINT(34.633931993662195 -117.69499654488237) bank8398 +8399 POINT(33.34085804884449 -117.54523703388632) bank8399 +8400 POINT(35.01674066328484 -118.8321203164258) bank8400 +8401 POINT(34.76912191240424 -118.09223569871324) bank8401 +8402 POINT(34.523981764029394 -119.15949966599699) bank8402 +8403 POINT(34.14673721352124 -118.54363924729005) bank8403 +8404 POINT(35.03354435972528 -119.05855859673967) bank8404 +8405 POINT(33.9973092849999 -119.05504042640213) bank8405 +8406 POINT(33.582166584663646 -117.93266561420944) bank8406 +8407 POINT(33.43354227632326 -118.776131198002) bank8407 +8408 POINT(33.284591546348246 -119.20568895105592) bank8408 +8409 POINT(33.91894058318101 -118.47663729482122) bank8409 +8410 POINT(34.89761634113382 -118.4648436275279) bank8410 +8411 POINT(33.88818857578991 -118.84132079693012) bank8411 +8412 POINT(34.82992266872623 -118.6163898362668) bank8412 +8413 POINT(34.59180488334982 -118.68838242254303) bank8413 +8414 POINT(33.620038817981325 -118.29201049536844) bank8414 +8415 POINT(33.430102917844195 -119.14351707807359) bank8415 +8416 POINT(34.35172305944385 -117.96899771545594) bank8416 +8417 POINT(34.37787881257275 -119.06589087616135) bank8417 +8418 POINT(33.95511704665571 -119.17543346650135) bank8418 +8419 POINT(34.973970732730905 -118.37012615945696) bank8419 +8420 POINT(33.70283567345146 -118.69203049811668) bank8420 +8421 POINT(34.05208018348829 -117.85386134989729) bank8421 +8422 POINT(33.428659522041926 -117.2476170182468) bank8422 +8423 POINT(33.06885848585498 -118.28815245974059) bank8423 +8424 POINT(33.962851396973946 -118.10959152659528) bank8424 +8425 POINT(34.39015861461175 -119.23147440464633) bank8425 +8426 POINT(33.9371269877221 -118.30748076974226) bank8426 +8427 POINT(34.85466338112622 -119.11185569355712) bank8427 +8428 POINT(34.52320296918974 -118.26585995871088) bank8428 +8429 POINT(33.904822998939984 -117.45708013551987) bank8429 +8430 POINT(34.151023866583735 -117.73602060500235) bank8430 +8431 POINT(34.844736993079266 -118.66317365798938) bank8431 +8432 POINT(34.97468209455807 -117.82078477045076) bank8432 +8433 POINT(33.093994541951474 -117.71262658507533) bank8433 +8434 POINT(33.507487911560105 -118.5173737736925) bank8434 +8435 POINT(33.390284135219595 -118.34192607104993) bank8435 +8436 POINT(34.92226665929495 -119.0006316968719) bank8436 +8437 POINT(34.56876790596805 -117.53403414143409) bank8437 +8438 POINT(33.09538579169492 -119.22291756902112) bank8438 +8439 POINT(33.08612915702698 -118.89457219939843) bank8439 +8440 POINT(34.38424644702654 -118.40262455821994) bank8440 +8441 POINT(34.34345963385762 -118.54226167312224) bank8441 +8442 POINT(34.15784452910045 -117.92502548365583) bank8442 +8443 POINT(34.516541535371005 -119.22635651467581) bank8443 +8444 POINT(34.20815019661801 -118.54457125631221) bank8444 +8445 POINT(33.790058802994025 -118.99887356350258) bank8445 +8446 POINT(33.29371184068619 -118.56789136562345) bank8446 +8447 POINT(34.14470192452345 -118.80769304696281) bank8447 +8448 POINT(33.17491050114551 -118.2553761451571) bank8448 +8449 POINT(34.780651579135025 -118.90418662949172) bank8449 +8450 POINT(33.56006994113945 -118.36028184648947) bank8450 +8451 POINT(34.541463742063016 -118.498243371462) bank8451 +8452 POINT(33.101911559020735 -118.45061968880147) bank8452 +8453 POINT(33.87622697879385 -117.30090417863896) bank8453 +8454 POINT(34.78513107900016 -118.16799834582244) bank8454 +8455 POINT(33.345746185298104 -118.55219880863916) bank8455 +8456 POINT(33.995075052574435 -118.83266024496815) bank8456 +8457 POINT(34.510471665487856 -118.38381806088675) bank8457 +8458 POINT(34.44461640780907 -117.80955763259566) bank8458 +8459 POINT(34.275699318745296 -119.06661219977241) bank8459 +8460 POINT(33.86880373031251 -118.25436917638757) bank8460 +8461 POINT(33.56062037196323 -117.68983848860981) bank8461 +8462 POINT(34.02960214669066 -117.74438092667307) bank8462 +8463 POINT(33.74782754564437 -118.42629982997542) bank8463 +8464 POINT(33.54720209666015 -119.1262473758284) bank8464 +8465 POINT(33.99036078550032 -117.8520612819786) bank8465 +8466 POINT(33.08003636475252 -118.73611495839647) bank8466 +8467 POINT(34.81653272696837 -119.14448828654295) bank8467 +8468 POINT(33.90108472183951 -117.48257281767394) bank8468 +8469 POINT(34.73341159977988 -118.25622086207075) bank8469 +8470 POINT(35.01322734069764 -118.64430019725899) bank8470 +8471 POINT(34.1517953065433 -118.13488623943663) bank8471 +8472 POINT(34.82729367038231 -117.37503477020992) bank8472 +8473 POINT(34.59485618671131 -118.4479244190402) bank8473 +8474 POINT(34.999772820577924 -118.95814901508945) bank8474 +8475 POINT(34.438353884100714 -118.48963993470144) bank8475 +8476 POINT(33.385229120182174 -117.54362154267913) bank8476 +8477 POINT(34.08271859320481 -118.92241573007686) bank8477 +8478 POINT(34.016635133091235 -118.41948499432182) bank8478 +8479 POINT(33.85719208606807 -118.54004264479477) bank8479 +8480 POINT(34.103816990976995 -118.20105496132999) bank8480 +8481 POINT(34.9309782783745 -118.94551639614086) bank8481 +8482 POINT(33.92459948561685 -119.22314342818657) bank8482 +8483 POINT(34.573413815294764 -117.72161495175834) bank8483 +8484 POINT(33.24760813215112 -118.08177372598722) bank8484 +8485 POINT(34.033411974471505 -117.59051204880863) bank8485 +8486 POINT(33.955955319792736 -118.1891703370621) bank8486 +8487 POINT(34.718882751910286 -118.7991907138411) bank8487 +8488 POINT(33.88378187374995 -117.98786347289698) bank8488 +8489 POINT(34.90290428016854 -118.69400392106715) bank8489 +8490 POINT(33.81275130989886 -119.0732544422757) bank8490 +8491 POINT(33.127088325154375 -117.84027266836826) bank8491 +8492 POINT(33.09229952100864 -118.30738232293007) bank8492 +8493 POINT(35.03756727117483 -117.28514092218647) bank8493 +8494 POINT(33.961916414507535 -117.33311606131035) bank8494 +8495 POINT(34.310043179119205 -117.87089821782806) bank8495 +8496 POINT(33.21268703572102 -118.01696160639487) bank8496 +8497 POINT(34.16792794276079 -118.92379559066828) bank8497 +8498 POINT(34.98773387936932 -117.26316863161001) bank8498 +8499 POINT(35.00392150109578 -117.4954742166673) bank8499 +8500 POINT(33.64588050495943 -118.45599772859612) bank8500 +8501 POINT(34.991807662022396 -117.74369651960498) bank8501 +8502 POINT(33.47401090274676 -118.82933054959808) bank8502 +8503 POINT(34.02576368111089 -117.73405821055269) bank8503 +8504 POINT(33.61920373744797 -117.99228682914301) bank8504 +8505 POINT(33.05385101979082 -118.79183649998326) bank8505 +8506 POINT(34.170366492835406 -117.35559496322084) bank8506 +8507 POINT(33.56521758800534 -117.7168293680774) bank8507 +8508 POINT(34.70372302875869 -117.94990638222026) bank8508 +8509 POINT(33.859633990251204 -117.48499752612197) bank8509 +8510 POINT(33.14847499936086 -118.00230864044981) bank8510 +8511 POINT(33.99422374180607 -118.37902666458618) bank8511 +8512 POINT(33.69030615977117 -119.12339640282956) bank8512 +8513 POINT(33.62305772378951 -118.66125064373105) bank8513 +8514 POINT(33.37776534594763 -119.18844493951583) bank8514 +8515 POINT(34.96658698531978 -119.13005682398581) bank8515 +8516 POINT(33.51036431701378 -117.6747076308473) bank8516 +8517 POINT(34.03782646426037 -117.82175671058278) bank8517 +8518 POINT(34.97969618062298 -118.7101520608976) bank8518 +8519 POINT(33.24345073063053 -117.27063522365205) bank8519 +8520 POINT(34.72834171631124 -117.9307656660824) bank8520 +8521 POINT(33.376519865373595 -117.52103495920973) bank8521 +8522 POINT(33.08167543137356 -118.09940069953053) bank8522 +8523 POINT(33.08441650604281 -119.08227067073646) bank8523 +8524 POINT(34.59186857026269 -118.86387978889508) bank8524 +8525 POINT(34.588203923716975 -118.89719099102679) bank8525 +8526 POINT(34.881195391796346 -118.7199356408712) bank8526 +8527 POINT(34.46824076333453 -117.53331826132643) bank8527 +8528 POINT(34.684796239470096 -118.86014451891775) bank8528 +8529 POINT(33.175819996670654 -117.64423349260547) bank8529 +8530 POINT(34.23813298424745 -117.41200386074993) bank8530 +8531 POINT(34.84208145270904 -117.9077579284007) bank8531 +8532 POINT(34.23350931797676 -117.33099689481801) bank8532 +8533 POINT(34.04142996649965 -118.71201226166006) bank8533 +8534 POINT(34.95826604719727 -118.5415651007014) bank8534 +8535 POINT(33.263289655672594 -118.4690184129871) bank8535 +8536 POINT(34.425392305468804 -118.33538279842523) bank8536 +8537 POINT(33.61098645252526 -119.053109474267) bank8537 +8538 POINT(33.180808016959375 -118.06249489514711) bank8538 +8539 POINT(34.0466530209874 -118.9334311048382) bank8539 +8540 POINT(33.77039789381716 -118.55794665578351) bank8540 +8541 POINT(33.442946839367266 -118.8508527476157) bank8541 +8542 POINT(34.10337167771137 -117.58700873936674) bank8542 +8543 POINT(33.12712289189007 -117.28495395327656) bank8543 +8544 POINT(34.51831597414269 -117.8619069285088) bank8544 +8545 POINT(33.63869340111161 -117.73563510774596) bank8545 +8546 POINT(34.632602218168664 -117.55924877424867) bank8546 +8547 POINT(34.23009097320952 -118.90232472933197) bank8547 +8548 POINT(33.731579268916185 -118.81871752216225) bank8548 +8549 POINT(34.47146059240407 -117.3403532534879) bank8549 +8550 POINT(34.19058683832959 -119.01327854118222) bank8550 +8551 POINT(33.3127003528949 -119.1526231646701) bank8551 +8552 POINT(34.2996379975813 -118.94846300333981) bank8552 +8553 POINT(34.972287526559384 -118.20244945999966) bank8553 +8554 POINT(34.14483763076913 -118.46613095928727) bank8554 +8555 POINT(34.13802478848976 -118.36656684342546) bank8555 +8556 POINT(33.45329016791463 -118.1585237107094) bank8556 +8557 POINT(33.88292449941414 -118.72167846015675) bank8557 +8558 POINT(34.527106775444366 -119.19268763689023) bank8558 +8559 POINT(33.20594502223343 -119.11878504777226) bank8559 +8560 POINT(33.14243062386016 -119.18674596065406) bank8560 +8561 POINT(34.1009684582692 -118.2341939378199) bank8561 +8562 POINT(33.21102336206173 -117.69340255041901) bank8562 +8563 POINT(33.73138588801023 -118.66211822249355) bank8563 +8564 POINT(34.68789944636363 -118.10460125461742) bank8564 +8565 POINT(33.621690291886175 -117.31674440137888) bank8565 +8566 POINT(33.16667416119068 -118.02461030376817) bank8566 +8567 POINT(34.48168097006932 -118.47653496995264) bank8567 +8568 POINT(33.38656526320134 -118.15560278917437) bank8568 +8569 POINT(34.296124249670875 -119.05374066187937) bank8569 +8570 POINT(33.50159457331591 -117.7840011508115) bank8570 +8571 POINT(33.74263202406051 -118.65585374427704) bank8571 +8572 POINT(35.04895356865128 -119.18562265341065) bank8572 +8573 POINT(34.47000218255387 -118.32418153546521) bank8573 +8574 POINT(33.371388758028736 -118.58995648955143) bank8574 +8575 POINT(34.1829039283118 -118.01810931580833) bank8575 +8576 POINT(34.93283667286727 -118.2813047849887) bank8576 +8577 POINT(34.51675964297722 -118.81344216466282) bank8577 +8578 POINT(34.778534187936266 -118.98631916878746) bank8578 +8579 POINT(33.73332038431156 -118.36134221433359) bank8579 +8580 POINT(34.82084769552253 -118.59741891899348) bank8580 +8581 POINT(33.368990747835795 -118.5101388788944) bank8581 +8582 POINT(34.324428411865405 -119.0598428193551) bank8582 +8583 POINT(34.8350602337389 -118.85986371078141) bank8583 +8584 POINT(33.27893621393267 -118.35432468761246) bank8584 +8585 POINT(33.24907065114263 -117.98101919905169) bank8585 +8586 POINT(33.573882256046 -117.81730696083473) bank8586 +8587 POINT(34.38167523561597 -118.44737374898556) bank8587 +8588 POINT(34.57887619952277 -118.84790764705265) bank8588 +8589 POINT(34.32158733391255 -117.58727922813434) bank8589 +8590 POINT(33.844825781708145 -119.14938236129603) bank8590 +8591 POINT(33.868143527891604 -118.40651180236536) bank8591 +8592 POINT(33.25883674277643 -117.36310491238899) bank8592 +8593 POINT(34.87719751558605 -117.76563418113318) bank8593 +8594 POINT(33.973752034522235 -118.4961754706165) bank8594 +8595 POINT(34.937960521657345 -117.77326294303643) bank8595 +8596 POINT(35.03830130862561 -118.98744843407387) bank8596 +8597 POINT(34.08536436791052 -118.9419472959572) bank8597 +8598 POINT(34.7411601627262 -118.9580212783323) bank8598 +8599 POINT(34.9945553292327 -117.3378130095731) bank8599 +8600 POINT(34.47752789596004 -117.58719604786467) bank8600 +8601 POINT(35.00025862853816 -117.6047426975391) bank8601 +8602 POINT(33.20435805110192 -117.6690362489228) bank8602 +8603 POINT(34.18568168829345 -119.18082643939373) bank8603 +8604 POINT(33.27360266127045 -119.05201857634022) bank8604 +8605 POINT(33.83715423204839 -119.12787050252906) bank8605 +8606 POINT(34.51790672451319 -119.07391261457532) bank8606 +8607 POINT(34.82337501249792 -118.24031472420143) bank8607 +8608 POINT(33.9069577092232 -117.87408089989518) bank8608 +8609 POINT(33.40115412602651 -118.7733141381743) bank8609 +8610 POINT(33.56773389370899 -118.59836231539589) bank8610 +8611 POINT(33.56545782389015 -117.68238264783143) bank8611 +8612 POINT(33.970211438271484 -118.20196755713712) bank8612 +8613 POINT(33.247548458497576 -117.47858898254664) bank8613 +8614 POINT(34.95776095104059 -119.20303949840377) bank8614 +8615 POINT(33.4883050477922 -117.72968336910803) bank8615 +8616 POINT(33.7055175210092 -117.50037346052753) bank8616 +8617 POINT(33.118670204862326 -118.55591826041002) bank8617 +8618 POINT(33.25659070039638 -118.9496861305964) bank8618 +8619 POINT(34.819266936646386 -117.37862097300464) bank8619 +8620 POINT(33.24358579845993 -118.63046692892247) bank8620 +8621 POINT(34.61311444460963 -117.77306410656367) bank8621 +8622 POINT(33.116119924017056 -117.760253298931) bank8622 +8623 POINT(33.958478143036885 -117.40726867809846) bank8623 +8624 POINT(33.59516145654066 -118.83546623755764) bank8624 +8625 POINT(33.667432720170545 -118.51719738940481) bank8625 +8626 POINT(34.25311001048508 -118.04534022480287) bank8626 +8627 POINT(33.63147784248923 -119.21359467299342) bank8627 +8628 POINT(33.96725362339833 -117.82089519006738) bank8628 +8629 POINT(33.19717579239445 -118.52509340884913) bank8629 +8630 POINT(34.39316390663982 -117.61517535828942) bank8630 +8631 POINT(33.81160549095153 -117.92113350273979) bank8631 +8632 POINT(34.188979212164895 -119.14233950200021) bank8632 +8633 POINT(33.70397229360704 -117.829163036037) bank8633 +8634 POINT(34.28429512133089 -118.96418097728507) bank8634 +8635 POINT(34.035180835523704 -117.77220277941963) bank8635 +8636 POINT(34.025496877588814 -118.76263745173856) bank8636 +8637 POINT(33.67154513574963 -117.28264633248166) bank8637 +8638 POINT(34.02881814745037 -118.085949685946) bank8638 +8639 POINT(33.779553983663675 -118.43520526722786) bank8639 +8640 POINT(34.02030501343839 -119.00367287314522) bank8640 +8641 POINT(34.249994909700064 -119.09121695346236) bank8641 +8642 POINT(33.16254729883434 -118.30359795198089) bank8642 +8643 POINT(33.86824588356819 -118.72420897237308) bank8643 +8644 POINT(33.53582386421527 -118.45108257846043) bank8644 +8645 POINT(33.39626015972719 -118.29284004524438) bank8645 +8646 POINT(34.0681646304545 -119.21753074580761) bank8646 +8647 POINT(34.852212537218186 -118.5742977991529) bank8647 +8648 POINT(34.964771967801866 -119.13693084261043) bank8648 +8649 POINT(33.912702811605044 -117.91422443201628) bank8649 +8650 POINT(34.74126600578621 -117.7701112831131) bank8650 +8651 POINT(33.24026341492229 -117.48309620565564) bank8651 +8652 POINT(34.668509412311764 -118.15447251265236) bank8652 +8653 POINT(33.15749141877426 -118.38695813303964) bank8653 +8654 POINT(34.34939547043104 -117.73377946413513) bank8654 +8655 POINT(33.26980695632964 -118.7849712694291) bank8655 +8656 POINT(33.76935260812141 -119.01294149143051) bank8656 +8657 POINT(33.807074652461 -117.51696874875425) bank8657 +8658 POINT(33.923280160019374 -118.0894342270371) bank8658 +8659 POINT(34.535142757480976 -118.4035146325271) bank8659 +8660 POINT(33.9009287413529 -119.07903730148526) bank8660 +8661 POINT(33.41835098143969 -117.86638734513443) bank8661 +8662 POINT(34.41206204121315 -118.41741487226396) bank8662 +8663 POINT(33.9844668135994 -118.87208860780717) bank8663 +8664 POINT(33.12278257840973 -117.48912794321105) bank8664 +8665 POINT(33.146913942813235 -119.10270554003111) bank8665 +8666 POINT(33.68655430963536 -118.17003138305412) bank8666 +8667 POINT(34.86355948545906 -118.92325717760491) bank8667 +8668 POINT(33.13625865562573 -119.1241205249608) bank8668 +8669 POINT(34.29954925757139 -118.05446754940837) bank8669 +8670 POINT(35.04365397518813 -118.92166070333973) bank8670 +8671 POINT(34.308517751641936 -118.4069523750829) bank8671 +8672 POINT(34.75216139860529 -119.10759835321669) bank8672 +8673 POINT(33.47475418109739 -117.74761501957889) bank8673 +8674 POINT(34.930422703431006 -117.46973312056265) bank8674 +8675 POINT(34.185764454328776 -117.73649680271794) bank8675 +8676 POINT(34.96237074004035 -117.3928399604888) bank8676 +8677 POINT(33.74338073817331 -118.54906622392008) bank8677 +8678 POINT(34.25849951981616 -117.62665233363404) bank8678 +8679 POINT(33.47580680598896 -117.71040545849323) bank8679 +8680 POINT(33.95029152162867 -117.46487829673897) bank8680 +8681 POINT(34.27798338969752 -118.80932023683064) bank8681 +8682 POINT(33.89630554776535 -117.98803851972238) bank8682 +8683 POINT(33.21379852464187 -117.97929927751102) bank8683 +8684 POINT(34.42371338649968 -117.67831533630583) bank8684 +8685 POINT(33.27332612966065 -119.1680864912231) bank8685 +8686 POINT(33.56369656586126 -118.47252140908674) bank8686 +8687 POINT(33.34069953286581 -118.93736720381911) bank8687 +8688 POINT(34.871740547545805 -118.9190859233734) bank8688 +8689 POINT(33.9425302192457 -119.08370919132324) bank8689 +8690 POINT(33.708654435712596 -118.62804171106477) bank8690 +8691 POINT(34.27043065903032 -117.37186165580967) bank8691 +8692 POINT(34.00130595364522 -118.42239765116332) bank8692 +8693 POINT(34.78578663808506 -117.50463603141098) bank8693 +8694 POINT(33.25283733153407 -117.5913026163002) bank8694 +8695 POINT(34.098655221491676 -117.7305848067808) bank8695 +8696 POINT(33.66420102118767 -118.09194313485365) bank8696 +8697 POINT(33.08244224028354 -118.62669938533938) bank8697 +8698 POINT(34.549331280947854 -117.49463500455875) bank8698 +8699 POINT(33.650016470112234 -119.18982889859154) bank8699 +8700 POINT(33.790458154051855 -119.16484057132811) bank8700 +8701 POINT(33.29526275754562 -117.94299128623005) bank8701 +8702 POINT(33.915190762791504 -118.93011232863813) bank8702 +8703 POINT(34.63464693387841 -118.32175257268146) bank8703 +8704 POINT(34.466382478025224 -119.150872839682) bank8704 +8705 POINT(34.846267833502885 -117.82425799735861) bank8705 +8706 POINT(33.95076517801594 -118.92872919002849) bank8706 +8707 POINT(33.501314084539324 -118.18875890191383) bank8707 +8708 POINT(34.188759101879455 -118.85383342206558) bank8708 +8709 POINT(33.39269347209626 -117.67369820682102) bank8709 +8710 POINT(33.75585759133156 -118.5195426804991) bank8710 +8711 POINT(34.581488369409385 -117.71800385178045) bank8711 +8712 POINT(33.936056897973124 -118.35078122115007) bank8712 +8713 POINT(33.343871000300695 -117.50000294054779) bank8713 +8714 POINT(34.303622699230274 -117.27709935543423) bank8714 +8715 POINT(34.9291808656116 -118.35666768083138) bank8715 +8716 POINT(33.845703332100776 -118.97999626978032) bank8716 +8717 POINT(33.38427064304749 -119.14651616363419) bank8717 +8718 POINT(33.10332084338387 -117.31765947866272) bank8718 +8719 POINT(34.820432778848314 -118.26529195538261) bank8719 +8720 POINT(33.22883339079446 -118.38933808778629) bank8720 +8721 POINT(33.96915624822482 -117.35493667076632) bank8721 +8722 POINT(33.19222918886415 -117.41071194413672) bank8722 +8723 POINT(33.77587312578418 -117.66767523746647) bank8723 +8724 POINT(33.1616077882176 -118.90934829583819) bank8724 +8725 POINT(34.39705829780879 -118.67967802860929) bank8725 +8726 POINT(34.4125136912827 -118.5850567212593) bank8726 +8727 POINT(33.123478010937774 -118.48042588993813) bank8727 +8728 POINT(33.28648503393791 -119.03887637340236) bank8728 +8729 POINT(34.42703566530787 -117.35975302749982) bank8729 +8730 POINT(34.394291893800364 -117.7499295671632) bank8730 +8731 POINT(34.95657457908807 -117.67104168953817) bank8731 +8732 POINT(34.140900991674876 -117.39662641431191) bank8732 +8733 POINT(33.33007085329963 -117.4045952270288) bank8733 +8734 POINT(33.419786615441716 -118.80364920197934) bank8734 +8735 POINT(34.30884823772705 -118.3370752760793) bank8735 +8736 POINT(33.480779881674366 -118.70875252932123) bank8736 +8737 POINT(34.895729253580235 -119.16971720195671) bank8737 +8738 POINT(33.384300127632535 -117.8239042683444) bank8738 +8739 POINT(33.59653497320982 -117.37525136459669) bank8739 +8740 POINT(34.45855447720808 -117.40814886506854) bank8740 +8741 POINT(34.90250000762513 -118.49248079231727) bank8741 +8742 POINT(33.8311288881061 -119.03566495291365) bank8742 +8743 POINT(33.25916214520393 -118.36040118489458) bank8743 +8744 POINT(33.29828796649733 -117.36095208622483) bank8744 +8745 POINT(34.771141874880925 -118.92425077952964) bank8745 +8746 POINT(34.90362029036936 -117.35249729958603) bank8746 +8747 POINT(34.430763441999616 -118.32984532991253) bank8747 +8748 POINT(35.03245710752292 -118.90406413860853) bank8748 +8749 POINT(33.43995594343737 -119.0874048923668) bank8749 +8750 POINT(34.11937354529762 -118.27867753341762) bank8750 +8751 POINT(34.23800180840351 -118.7034085704737) bank8751 +8752 POINT(34.36544414275996 -118.13030488365754) bank8752 +8753 POINT(33.70048301590181 -118.45283787337256) bank8753 +8754 POINT(33.93109690464997 -117.34634129029185) bank8754 +8755 POINT(34.06528904937704 -117.84971068937365) bank8755 +8756 POINT(34.65077761300952 -119.09894866971695) bank8756 +8757 POINT(34.53297433804759 -117.57686878573026) bank8757 +8758 POINT(33.415733785288374 -118.9419973581326) bank8758 +8759 POINT(35.017194306243404 -117.95610990729062) bank8759 +8760 POINT(35.01373883944562 -118.22896866361896) bank8760 +8761 POINT(33.972072498609144 -118.14945266483817) bank8761 +8762 POINT(33.965598305632774 -118.87958082694996) bank8762 +8763 POINT(34.06941286268596 -117.58906951162497) bank8763 +8764 POINT(34.125768821659044 -118.17217793642013) bank8764 +8765 POINT(34.149392557903866 -117.49273095670206) bank8765 +8766 POINT(33.910867358012744 -118.37921988455983) bank8766 +8767 POINT(33.38759281724337 -117.50334158478321) bank8767 +8768 POINT(33.41768040878518 -117.34654370099162) bank8768 +8769 POINT(34.821283424174446 -119.22218234655173) bank8769 +8770 POINT(34.69009166548881 -118.16422559242143) bank8770 +8771 POINT(33.54967537989019 -119.06113249759102) bank8771 +8772 POINT(34.05237462521228 -118.12666597341641) bank8772 +8773 POINT(34.840787647421735 -117.28783014682716) bank8773 +8774 POINT(34.62399873204895 -118.86732581700365) bank8774 +8775 POINT(34.129627614589985 -118.02817195735403) bank8775 +8776 POINT(33.322541975372616 -117.37779997545573) bank8776 +8777 POINT(34.90797647690386 -119.14014276082115) bank8777 +8778 POINT(34.25351682611746 -119.19408096497223) bank8778 +8779 POINT(33.1069829722751 -117.53197052642803) bank8779 +8780 POINT(34.16335622915384 -118.42226024995757) bank8780 +8781 POINT(34.82511462943417 -118.82554695377816) bank8781 +8782 POINT(34.10874384131639 -117.39883599077832) bank8782 +8783 POINT(33.54400608252751 -117.73170699762888) bank8783 +8784 POINT(34.36786526164374 -117.256641671725) bank8784 +8785 POINT(33.10752058785769 -117.26978959434406) bank8785 +8786 POINT(34.861513323388685 -119.12112595008462) bank8786 +8787 POINT(33.64615619620665 -117.25615348864608) bank8787 +8788 POINT(34.814598998310046 -119.09371802372908) bank8788 +8789 POINT(34.31293421406922 -117.43717926619722) bank8789 +8790 POINT(33.84053747389447 -119.04341534157193) bank8790 +8791 POINT(34.57927421827459 -118.51398702372414) bank8791 +8792 POINT(33.24943694809568 -117.6306418763047) bank8792 +8793 POINT(34.19065924581571 -118.75852101621824) bank8793 +8794 POINT(33.30867497272254 -118.80625852289138) bank8794 +8795 POINT(34.12175183787036 -118.10804563176114) bank8795 +8796 POINT(33.50188812254117 -119.0298169236843) bank8796 +8797 POINT(34.61036993603795 -117.50647050700033) bank8797 +8798 POINT(34.00193314302863 -118.29706917666053) bank8798 +8799 POINT(33.68830450373052 -117.93213518563856) bank8799 +8800 POINT(33.06923426229178 -118.65059924527579) bank8800 +8801 POINT(34.40605327002073 -118.43404125899444) bank8801 +8802 POINT(33.66464673910318 -118.18915742998071) bank8802 +8803 POINT(33.98920888376224 -117.95244287833388) bank8803 +8804 POINT(34.3965157233992 -119.07529100297776) bank8804 +8805 POINT(34.69136572719092 -118.66520950325317) bank8805 +8806 POINT(33.19509596846574 -118.99062956114815) bank8806 +8807 POINT(34.46018543161917 -118.48044544959352) bank8807 +8808 POINT(34.90846753610391 -117.70293812805565) bank8808 +8809 POINT(34.87972935703491 -118.75721287509099) bank8809 +8810 POINT(33.069362509381655 -118.82250633178217) bank8810 +8811 POINT(33.26002766732869 -117.75535497794814) bank8811 +8812 POINT(34.17819412897564 -118.03479215695153) bank8812 +8813 POINT(34.49740842758589 -117.31612962936084) bank8813 +8814 POINT(34.289938498297474 -119.17016537238734) bank8814 +8815 POINT(34.412901744285335 -117.96213270297491) bank8815 +8816 POINT(33.23189486976391 -119.11509334434905) bank8816 +8817 POINT(34.88461466318512 -117.82930405219844) bank8817 +8818 POINT(33.37912077280707 -118.77987488615346) bank8818 +8819 POINT(34.205241583504176 -118.64905780301756) bank8819 +8820 POINT(34.200394486576485 -119.09271783717031) bank8820 +8821 POINT(34.8353277886062 -117.91934395223367) bank8821 +8822 POINT(34.23037394521678 -117.7361251643142) bank8822 +8823 POINT(33.47653410504096 -117.98078780983026) bank8823 +8824 POINT(34.8693399120476 -117.95075322727178) bank8824 +8825 POINT(33.22807474708108 -117.83676852499339) bank8825 +8826 POINT(33.9024972132921 -117.7778817740519) bank8826 +8827 POINT(34.517739885294944 -117.73702423481434) bank8827 +8828 POINT(34.342550057519624 -118.06992759920595) bank8828 +8829 POINT(34.36864971974364 -118.35983985926082) bank8829 +8830 POINT(33.05516936681941 -118.68439069322801) bank8830 +8831 POINT(33.58699956971936 -117.41753083611825) bank8831 +8832 POINT(33.22652570971368 -117.43195018664711) bank8832 +8833 POINT(33.87541803954432 -118.16547904121364) bank8833 +8834 POINT(34.426035946979276 -119.11030972855635) bank8834 +8835 POINT(33.41534116196534 -118.94645596889836) bank8835 +8836 POINT(34.90284670233024 -118.5985223425407) bank8836 +8837 POINT(33.27716798733773 -118.7006128131124) bank8837 +8838 POINT(33.28682543894482 -117.44961300844827) bank8838 +8839 POINT(34.85607336680663 -119.05709765421925) bank8839 +8840 POINT(34.20739854122826 -118.4855114249763) bank8840 +8841 POINT(33.22754344285013 -118.44971073729565) bank8841 +8842 POINT(34.68987268670356 -118.16912408660697) bank8842 +8843 POINT(34.1518321917968 -118.29303108591189) bank8843 +8844 POINT(33.80858510639137 -119.18833546657102) bank8844 +8845 POINT(34.650933431424676 -118.47367846056424) bank8845 +8846 POINT(34.3940447004261 -118.22030471686658) bank8846 +8847 POINT(33.965676899138785 -119.09698138562058) bank8847 +8848 POINT(35.01847065035443 -117.85601846538869) bank8848 +8849 POINT(33.67322539568869 -118.52475536913667) bank8849 +8850 POINT(33.495875986137115 -118.30621080792467) bank8850 +8851 POINT(33.848958971806894 -118.88388497413652) bank8851 +8852 POINT(33.20320379173342 -119.11314679750265) bank8852 +8853 POINT(34.939001674296925 -118.26923856961007) bank8853 +8854 POINT(33.947058739191874 -118.0844550389797) bank8854 +8855 POINT(33.19856635241062 -117.78290844383646) bank8855 +8856 POINT(33.523765472829155 -118.38948251888617) bank8856 +8857 POINT(33.39492491905076 -118.32368953799804) bank8857 +8858 POINT(33.067538160492965 -118.78720284591991) bank8858 +8859 POINT(33.158482779068486 -118.13283243865787) bank8859 +8860 POINT(33.46937494161709 -118.13043353901261) bank8860 +8861 POINT(33.827751167373094 -118.16563196884509) bank8861 +8862 POINT(34.861130016745406 -117.54692874079299) bank8862 +8863 POINT(33.47803221473764 -118.51858562637777) bank8863 +8864 POINT(34.385825066060534 -117.48725994159707) bank8864 +8865 POINT(33.34543232778164 -118.50864967911383) bank8865 +8866 POINT(34.73432616041553 -118.87146392594484) bank8866 +8867 POINT(34.01203391942522 -118.86177849231662) bank8867 +8868 POINT(34.89909176111379 -119.22534017376822) bank8868 +8869 POINT(33.09626554290341 -118.31054166296242) bank8869 +8870 POINT(33.57476845082408 -117.77292885052871) bank8870 +8871 POINT(33.3624147857954 -119.08299800240226) bank8871 +8872 POINT(33.828214506129385 -117.8593409820993) bank8872 +8873 POINT(34.21767537575789 -117.92164905995661) bank8873 +8874 POINT(33.792979557254746 -117.38226671781644) bank8874 +8875 POINT(35.00708918534814 -118.61634115407915) bank8875 +8876 POINT(34.718487740039286 -118.28068144353801) bank8876 +8877 POINT(34.69366121588819 -119.16354361289683) bank8877 +8878 POINT(34.81861772083254 -118.99115388277765) bank8878 +8879 POINT(33.097395149916274 -118.55353528350774) bank8879 +8880 POINT(33.220120773730045 -117.8354972234131) bank8880 +8881 POINT(34.85374355896429 -118.59326491882302) bank8881 +8882 POINT(34.058204978111824 -119.23721744703354) bank8882 +8883 POINT(33.16048352029402 -119.19032762625795) bank8883 +8884 POINT(34.51128799072471 -118.58742806931645) bank8884 +8885 POINT(33.76684310125354 -118.49527066710397) bank8885 +8886 POINT(34.73608446267841 -117.51185006129948) bank8886 +8887 POINT(34.45360429734449 -119.12027237819842) bank8887 +8888 POINT(33.908985911581205 -117.51840232154326) bank8888 +8889 POINT(34.81962662325544 -118.52091525016176) bank8889 +8890 POINT(34.25863362828705 -118.3486768424954) bank8890 +8891 POINT(34.86541461938571 -118.27509125102058) bank8891 +8892 POINT(34.54053469192842 -117.46963195309884) bank8892 +8893 POINT(33.903329484088474 -119.02662385207768) bank8893 +8894 POINT(33.61779868024778 -117.75114717454605) bank8894 +8895 POINT(33.19629534141891 -118.62257990854704) bank8895 +8896 POINT(33.068847258789155 -118.8894040420605) bank8896 +8897 POINT(33.634306576895554 -118.83086249050162) bank8897 +8898 POINT(34.25073641371538 -118.31874541575934) bank8898 +8899 POINT(34.28835457966414 -118.35236648074358) bank8899 +8900 POINT(33.14916968691044 -118.9078235128066) bank8900 +8901 POINT(34.773238922887124 -118.960206793585) bank8901 +8902 POINT(33.392072153070565 -117.45513273726812) bank8902 +8903 POINT(34.82026593067349 -117.88533083443492) bank8903 +8904 POINT(33.787123958711966 -118.8516641710869) bank8904 +8905 POINT(34.97506991047637 -118.22150735205045) bank8905 +8906 POINT(33.54813782889697 -119.14803925294288) bank8906 +8907 POINT(34.823453165466915 -117.99577079472674) bank8907 +8908 POINT(34.63671151810566 -117.80421843804501) bank8908 +8909 POINT(34.98882762847799 -118.5622424231356) bank8909 +8910 POINT(34.60207266951153 -118.73881021823652) bank8910 +8911 POINT(33.993092226150125 -117.87705557572572) bank8911 +8912 POINT(33.81842696490757 -117.85531625127456) bank8912 +8913 POINT(34.47146768470097 -117.57731424379196) bank8913 +8914 POINT(34.13058333070621 -117.46346654466967) bank8914 +8915 POINT(34.74140505558137 -117.54824483404144) bank8915 +8916 POINT(34.92657589513094 -118.25816619221749) bank8916 +8917 POINT(33.96356138069109 -118.09072940035522) bank8917 +8918 POINT(33.26966921634475 -119.116328982669) bank8918 +8919 POINT(34.22066277903474 -117.36289758563164) bank8919 +8920 POINT(33.40986341169127 -118.74256937129559) bank8920 +8921 POINT(34.5508970958607 -117.78690542140568) bank8921 +8922 POINT(34.003456995429005 -119.19042809679716) bank8922 +8923 POINT(33.6136085644688 -118.7536818040799) bank8923 +8924 POINT(33.54514211808841 -117.70643354338155) bank8924 +8925 POINT(33.510929988141704 -118.73616461792965) bank8925 +8926 POINT(33.08288866468244 -117.3900356158806) bank8926 +8927 POINT(34.3038708100507 -117.66941185871607) bank8927 +8928 POINT(34.223677924341146 -119.22434879551636) bank8928 +8929 POINT(34.07777051231447 -118.09589421004496) bank8929 +8930 POINT(33.98003136841743 -118.51857453033519) bank8930 +8931 POINT(34.58113970361911 -118.45838089685124) bank8931 +8932 POINT(34.570266210528644 -117.4281375259191) bank8932 +8933 POINT(34.05392189334931 -118.10740253160186) bank8933 +8934 POINT(33.12816699328019 -118.27029398602187) bank8934 +8935 POINT(34.99039199040513 -117.73065759986106) bank8935 +8936 POINT(34.05096580747836 -118.00138677850002) bank8936 +8937 POINT(35.01389628957296 -118.67260409752181) bank8937 +8938 POINT(34.295552876277924 -118.12581671698184) bank8938 +8939 POINT(34.81173104456605 -118.03919489110196) bank8939 +8940 POINT(34.67255112261623 -118.30070439060492) bank8940 +8941 POINT(33.90092116787019 -118.07690448011343) bank8941 +8942 POINT(34.58428667662582 -117.77733882285588) bank8942 +8943 POINT(35.039566859058034 -118.61981393819973) bank8943 +8944 POINT(34.80375059814388 -117.3550531592302) bank8944 +8945 POINT(34.01137991270718 -118.2291893623198) bank8945 +8946 POINT(34.10175712226871 -118.57484116147099) bank8946 +8947 POINT(34.13961664769467 -118.74079835613095) bank8947 +8948 POINT(33.15556838356072 -117.53802828680601) bank8948 +8949 POINT(34.34501716931582 -118.74575301341856) bank8949 +8950 POINT(34.99661292502531 -118.68947804284693) bank8950 +8951 POINT(34.63048383825128 -118.62395579143423) bank8951 +8952 POINT(33.606889390665955 -118.34199990785008) bank8952 +8953 POINT(34.95378272079127 -118.45960925320941) bank8953 +8954 POINT(34.62171013459151 -117.39704994458481) bank8954 +8955 POINT(34.134098550675844 -117.71811500377454) bank8955 +8956 POINT(33.57898260416909 -118.65620243573193) bank8956 +8957 POINT(34.9606113646112 -117.55304744106768) bank8957 +8958 POINT(33.62584712827902 -118.67253193351995) bank8958 +8959 POINT(33.138494265000745 -118.64528462282613) bank8959 +8960 POINT(33.360504562456 -117.57501296574084) bank8960 +8961 POINT(34.02040772234832 -117.26889039335371) bank8961 +8962 POINT(34.45559412816654 -119.11253450509223) bank8962 +8963 POINT(33.26302998482643 -118.35895397281553) bank8963 +8964 POINT(34.52027501458282 -119.01187374108078) bank8964 +8965 POINT(35.0364658234799 -117.39122043469756) bank8965 +8966 POINT(33.07236332868249 -117.73948528116696) bank8966 +8967 POINT(33.11776777318122 -117.96043952154699) bank8967 +8968 POINT(33.545099792494106 -118.10479400878717) bank8968 +8969 POINT(34.46161653416357 -119.02396779713872) bank8969 +8970 POINT(34.69428190378955 -117.49009593741407) bank8970 +8971 POINT(33.191694952322656 -118.88575796086005) bank8971 +8972 POINT(34.688794606074296 -117.90944747543266) bank8972 +8973 POINT(34.60253114561853 -118.86943721658666) bank8973 +8974 POINT(34.27795155413789 -117.8770842451293) bank8974 +8975 POINT(33.774365121266605 -118.67278462528354) bank8975 +8976 POINT(33.64750977025577 -117.34877990495711) bank8976 +8977 POINT(34.79489542089845 -119.13078591347279) bank8977 +8978 POINT(34.66749473984051 -117.47460583166549) bank8978 +8979 POINT(34.59782554478146 -117.40412517055057) bank8979 +8980 POINT(33.652273605353024 -118.39935290600707) bank8980 +8981 POINT(34.705538882500136 -118.88073405480752) bank8981 +8982 POINT(33.17614371112958 -118.73713387894088) bank8982 +8983 POINT(34.460798978917715 -118.20444267124138) bank8983 +8984 POINT(34.21595264008894 -117.2878522676682) bank8984 +8985 POINT(33.49631614741785 -119.03963607559092) bank8985 +8986 POINT(33.42731470158357 -117.97978520714292) bank8986 +8987 POINT(34.98423673282967 -118.00710786374158) bank8987 +8988 POINT(34.579910647175915 -119.06107270519557) bank8988 +8989 POINT(33.142026984543094 -117.61082839817834) bank8989 +8990 POINT(35.003418648346766 -119.23378179416629) bank8990 +8991 POINT(33.06759479450927 -118.73570445955522) bank8991 +8992 POINT(34.595461055081785 -118.60658823045605) bank8992 +8993 POINT(33.9757061558791 -117.26336301961132) bank8993 +8994 POINT(33.28232378265941 -119.10327301451706) bank8994 +8995 POINT(34.14436749769133 -117.3153103523162) bank8995 +8996 POINT(33.98360671949454 -118.26324359213126) bank8996 +8997 POINT(34.79568289243066 -117.61411214382126) bank8997 +8998 POINT(34.72290690725731 -118.7786866095012) bank8998 +8999 POINT(33.84737795007262 -119.2311942998814) bank8999 +9000 POINT(33.69914550020396 -117.5275465046125) bank9000 +9001 POINT(33.591081216947906 -117.65842187961954) bank9001 +9002 POINT(33.76575535392884 -118.82409740569607) bank9002 +9003 POINT(33.48315277539897 -117.59092336785862) bank9003 +9004 POINT(34.773679450439296 -118.95355947197862) bank9004 +9005 POINT(33.884808113146065 -119.12836174149668) bank9005 +9006 POINT(35.000717784988645 -118.57560017296969) bank9006 +9007 POINT(33.639521145245055 -119.19701514594803) bank9007 +9008 POINT(33.772223034974544 -117.3262144271878) bank9008 +9009 POINT(33.46706945005085 -117.78141396425531) bank9009 +9010 POINT(34.40459096984629 -118.18740971784793) bank9010 +9011 POINT(33.482796262123394 -119.21246924035711) bank9011 +9012 POINT(34.36821895535433 -119.13866397075533) bank9012 +9013 POINT(34.804203259282325 -117.65310417230197) bank9013 +9014 POINT(34.680924799496886 -118.88591662665948) bank9014 +9015 POINT(33.82730984948709 -118.76047545785117) bank9015 +9016 POINT(33.14389164841911 -117.68743370533612) bank9016 +9017 POINT(33.13367831559438 -118.5363587111319) bank9017 +9018 POINT(34.43380441587896 -117.93254463756453) bank9018 +9019 POINT(34.20920497396913 -118.5902783246125) bank9019 +9020 POINT(34.54775081459961 -118.4168395548489) bank9020 +9021 POINT(34.302185432721245 -118.32855451038405) bank9021 +9022 POINT(34.21794695644478 -118.47999886091601) bank9022 +9023 POINT(33.890376132876874 -118.57501400129655) bank9023 +9024 POINT(33.594029780465 -118.89731125175763) bank9024 +9025 POINT(34.40288253651192 -117.75269099665496) bank9025 +9026 POINT(33.82685412180476 -117.79414558991641) bank9026 +9027 POINT(33.246005693286165 -117.96228212877024) bank9027 +9028 POINT(33.357004986351875 -118.9125706215831) bank9028 +9029 POINT(34.848810899268685 -117.62531956943543) bank9029 +9030 POINT(34.773599075306215 -118.1761181297041) bank9030 +9031 POINT(34.74905552523076 -117.78685778539949) bank9031 +9032 POINT(34.16161143814751 -118.57259107171436) bank9032 +9033 POINT(34.91794087091641 -118.13786915427568) bank9033 +9034 POINT(34.543845758891166 -117.56856533889963) bank9034 +9035 POINT(34.499281328842095 -117.38817336120903) bank9035 +9036 POINT(33.62522237106296 -117.80709056876054) bank9036 +9037 POINT(34.14587258932299 -118.00230116324305) bank9037 +9038 POINT(33.87326880364291 -117.71111763522818) bank9038 +9039 POINT(34.666264567073085 -117.70241274746081) bank9039 +9040 POINT(34.516408252685025 -118.41695973217719) bank9040 +9041 POINT(33.32502506925714 -117.81392711198086) bank9041 +9042 POINT(33.644628010705595 -117.68431734086036) bank9042 +9043 POINT(34.61388994812129 -117.2811098868) bank9043 +9044 POINT(34.88912757479049 -117.88843404664254) bank9044 +9045 POINT(33.263411090430125 -117.65283507283603) bank9045 +9046 POINT(33.34524772702363 -117.92845170148647) bank9046 +9047 POINT(33.952615856818014 -118.9851937725707) bank9047 +9048 POINT(33.29126775580048 -118.00438511774239) bank9048 +9049 POINT(33.813007876369944 -117.86581063419749) bank9049 +9050 POINT(33.24530326666234 -117.81157187893497) bank9050 +9051 POINT(33.286620359432874 -118.6926344257311) bank9051 +9052 POINT(33.632648860125514 -117.85362523491432) bank9052 +9053 POINT(33.27561715561655 -117.39728401902622) bank9053 +9054 POINT(33.37496704493416 -117.43747436202426) bank9054 +9055 POINT(33.54493827225108 -119.1652653504693) bank9055 +9056 POINT(33.401188695565594 -118.74288717286684) bank9056 +9057 POINT(33.455287114079674 -117.50485924328603) bank9057 +9058 POINT(34.630264220143154 -117.84128110686724) bank9058 +9059 POINT(34.836440180760356 -119.12847892188351) bank9059 +9060 POINT(34.28539420542245 -118.66020479485476) bank9060 +9061 POINT(33.69181962530341 -117.37790543841604) bank9061 +9062 POINT(34.779891762033444 -118.35125262110407) bank9062 +9063 POINT(34.691978476416914 -118.1239484727043) bank9063 +9064 POINT(33.977958711418765 -118.94600164187008) bank9064 +9065 POINT(34.64204575287043 -117.93939940118186) bank9065 +9066 POINT(34.15120405850953 -117.75864017157673) bank9066 +9067 POINT(34.46947909825001 -118.37149139077215) bank9067 +9068 POINT(33.67188092968195 -118.60426207713763) bank9068 +9069 POINT(33.81238799111707 -119.19717260342097) bank9069 +9070 POINT(34.82124943383739 -117.36383502008275) bank9070 +9071 POINT(33.058433562795315 -117.58915652345732) bank9071 +9072 POINT(34.680139394904614 -117.59386444897476) bank9072 +9073 POINT(33.30004510817815 -118.3383702592239) bank9073 +9074 POINT(33.839591129455854 -118.89971668637216) bank9074 +9075 POINT(34.3096595450612 -118.8904389821999) bank9075 +9076 POINT(33.913351855071475 -118.59208724703106) bank9076 +9077 POINT(34.978294920866794 -118.96018573024229) bank9077 +9078 POINT(34.85495923839109 -118.3650212258894) bank9078 +9079 POINT(33.737233846307795 -117.90210810558926) bank9079 +9080 POINT(33.74081865576789 -118.05537744468137) bank9080 +9081 POINT(34.29824003398379 -118.07432268420337) bank9081 +9082 POINT(34.035448853268875 -118.35344248199205) bank9082 +9083 POINT(34.84098231655473 -118.62698877614932) bank9083 +9084 POINT(33.11063392898959 -118.33438301140332) bank9084 +9085 POINT(34.64336526528801 -117.28531707343674) bank9085 +9086 POINT(34.98210493222815 -117.69901898026633) bank9086 +9087 POINT(33.14808191565957 -118.92271035239342) bank9087 +9088 POINT(34.28714446696084 -117.66877702001172) bank9088 +9089 POINT(34.603391076739044 -117.70847261700678) bank9089 +9090 POINT(33.85008752681324 -118.16110494758877) bank9090 +9091 POINT(34.01662687789904 -118.39858864023755) bank9091 +9092 POINT(34.15364203181622 -117.9565471220469) bank9092 +9093 POINT(34.09499007951764 -118.43526628429723) bank9093 +9094 POINT(35.018473641886594 -117.40171369829567) bank9094 +9095 POINT(33.48294866761383 -117.8008622446044) bank9095 +9096 POINT(34.400405317781114 -118.14236457008873) bank9096 +9097 POINT(34.90984302786814 -118.06177861267317) bank9097 +9098 POINT(33.58946855172716 -118.31487645924298) bank9098 +9099 POINT(34.61907999357709 -118.53325781095748) bank9099 +9100 POINT(35.01856554885796 -117.91031759480903) bank9100 +9101 POINT(35.030714382850995 -117.32129854164508) bank9101 +9102 POINT(34.76895209595433 -118.1383279162738) bank9102 +9103 POINT(34.04885233136458 -118.67511502821934) bank9103 +9104 POINT(33.96439659810905 -117.56314814619806) bank9104 +9105 POINT(33.96781564340277 -119.14173653551761) bank9105 +9106 POINT(34.18074800475231 -118.21781104077215) bank9106 +9107 POINT(33.56993314982236 -119.14865657138843) bank9107 +9108 POINT(34.478554497675674 -118.98199568601724) bank9108 +9109 POINT(33.734425107282306 -118.03884720256738) bank9109 +9110 POINT(33.21219681960757 -118.80578919060972) bank9110 +9111 POINT(34.65270357475317 -117.7864046485402) bank9111 +9112 POINT(33.142462957893926 -117.66930295940354) bank9112 +9113 POINT(33.32197372663313 -117.80333019684687) bank9113 +9114 POINT(34.27290735202687 -117.55874506995497) bank9114 +9115 POINT(34.4499605414806 -117.65361120999007) bank9115 +9116 POINT(34.288838565725236 -118.61986062137613) bank9116 +9117 POINT(34.58290117945741 -118.30061966583384) bank9117 +9118 POINT(34.859452290923755 -118.55493191206102) bank9118 +9119 POINT(33.72250731798819 -118.85101537140208) bank9119 +9120 POINT(33.44585254333187 -118.46804221039204) bank9120 +9121 POINT(34.993406992704635 -118.65264501056558) bank9121 +9122 POINT(33.44425920673004 -118.46177193132542) bank9122 +9123 POINT(34.712577028224885 -118.35301585536723) bank9123 +9124 POINT(34.260477500773874 -118.739657104039) bank9124 +9125 POINT(33.76172262917603 -117.37967090105428) bank9125 +9126 POINT(33.104231292848624 -118.62922550387012) bank9126 +9127 POINT(33.96612774084827 -119.11494815122562) bank9127 +9128 POINT(34.41761559855074 -118.85954731571422) bank9128 +9129 POINT(34.83889199677253 -117.57291424687855) bank9129 +9130 POINT(34.93047644327091 -117.51298878782144) bank9130 +9131 POINT(34.3158280617567 -117.59084588039872) bank9131 +9132 POINT(34.059841218755324 -117.65516770587736) bank9132 +9133 POINT(34.21996939574912 -117.91226471554256) bank9133 +9134 POINT(33.57505305065025 -117.3772082901617) bank9134 +9135 POINT(34.359720006326086 -118.69186537525415) bank9135 +9136 POINT(34.96001770436412 -119.05282497461792) bank9136 +9137 POINT(33.82512097768647 -119.24054126755453) bank9137 +9138 POINT(34.72563012998149 -118.3635305947783) bank9138 +9139 POINT(33.81132223745451 -118.4164572019816) bank9139 +9140 POINT(34.67632682623011 -118.33024488345283) bank9140 +9141 POINT(33.24986226719164 -117.46623759180383) bank9141 +9142 POINT(33.279754238290884 -117.56513090913818) bank9142 +9143 POINT(33.07378589662098 -117.8805251739451) bank9143 +9144 POINT(33.28027092191433 -117.9629058697377) bank9144 +9145 POINT(34.02753397400381 -118.05349546742507) bank9145 +9146 POINT(34.92395468733439 -117.7402128786887) bank9146 +9147 POINT(35.017785722354496 -117.70440765735479) bank9147 +9148 POINT(34.496188423006004 -117.97220798413528) bank9148 +9149 POINT(34.66275349623601 -117.47903669838891) bank9149 +9150 POINT(34.81078843948552 -118.950737894727) bank9150 +9151 POINT(33.978575042642795 -118.33809406326009) bank9151 +9152 POINT(34.715122729282974 -117.29560772172606) bank9152 +9153 POINT(33.685615476364426 -117.7930334246844) bank9153 +9154 POINT(34.14461449920352 -117.91955138342277) bank9154 +9155 POINT(33.52018769012212 -117.82104503429382) bank9155 +9156 POINT(33.98667604244936 -117.91400372835783) bank9156 +9157 POINT(34.4593719502293 -118.57367319390346) bank9157 +9158 POINT(34.94788715843928 -117.44378886463323) bank9158 +9159 POINT(33.859754082247534 -117.85915541540555) bank9159 +9160 POINT(34.07646704743133 -118.618968550049) bank9160 +9161 POINT(34.417743071083066 -117.86794523229493) bank9161 +9162 POINT(35.04203764039801 -117.74813741465465) bank9162 +9163 POINT(33.87520077325767 -118.4458378225696) bank9163 +9164 POINT(34.89760002187146 -118.99682459098565) bank9164 +9165 POINT(34.16005295707222 -118.87403790332506) bank9165 +9166 POINT(33.88786610808452 -117.53584964076848) bank9166 +9167 POINT(34.73671273379582 -117.70787905441635) bank9167 +9168 POINT(33.4182418105476 -118.33007044944668) bank9168 +9169 POINT(33.84849012301563 -118.4607171241834) bank9169 +9170 POINT(33.877196468208744 -118.83480586437366) bank9170 +9171 POINT(33.75879692669997 -117.93850288786093) bank9171 +9172 POINT(34.71929346189936 -117.82712806553482) bank9172 +9173 POINT(34.98690643303748 -117.5089324640705) bank9173 +9174 POINT(33.09796396035635 -117.47212808679514) bank9174 +9175 POINT(34.82488662750588 -117.82344913464857) bank9175 +9176 POINT(33.300916930378804 -118.02296705957225) bank9176 +9177 POINT(33.621106388604815 -117.32972102270371) bank9177 +9178 POINT(33.906597957298835 -118.09104466651985) bank9178 +9179 POINT(33.29113997152283 -117.61904988363578) bank9179 +9180 POINT(34.85920953109568 -117.92582829249092) bank9180 +9181 POINT(35.050958663428 -119.02328457322146) bank9181 +9182 POINT(33.92750846742772 -119.18823220977427) bank9182 +9183 POINT(34.365852303832085 -119.09405339377297) bank9183 +9184 POINT(33.519832666861795 -118.60771963737007) bank9184 +9185 POINT(34.0958930393088 -119.23264230474942) bank9185 +9186 POINT(33.50465806277708 -117.90840511047094) bank9186 +9187 POINT(34.53816953354697 -118.10760111448738) bank9187 +9188 POINT(33.106090743535724 -118.13494315980425) bank9188 +9189 POINT(33.95090497834569 -117.40841905777472) bank9189 +9190 POINT(34.67770932211609 -117.66272279071947) bank9190 +9191 POINT(33.45756220243123 -118.70028465051209) bank9191 +9192 POINT(34.417131430944536 -119.22710317226982) bank9192 +9193 POINT(33.15453087903092 -118.27591115482377) bank9193 +9194 POINT(34.7903637780718 -119.1383456404659) bank9194 +9195 POINT(34.546877412281 -117.55840865890653) bank9195 +9196 POINT(33.78411318610944 -118.43455018099931) bank9196 +9197 POINT(33.4578712910096 -118.52184605158877) bank9197 +9198 POINT(33.60323077140961 -118.38827052728897) bank9198 +9199 POINT(34.26638784439376 -117.71273028059846) bank9199 +9200 POINT(33.964256043344534 -118.40303079584588) bank9200 +9201 POINT(34.25236031309898 -118.90723606301398) bank9201 +9202 POINT(33.66548602731685 -119.14390669231092) bank9202 +9203 POINT(33.502814042295796 -118.16448735471714) bank9203 +9204 POINT(34.504480283730906 -117.27372647680839) bank9204 +9205 POINT(34.698264332435734 -117.47994427327696) bank9205 +9206 POINT(33.91203772078422 -117.43835522604509) bank9206 +9207 POINT(34.0654305361509 -117.2619473674267) bank9207 +9208 POINT(33.15439249025795 -118.24351929020723) bank9208 +9209 POINT(33.43723778131159 -118.92081035523307) bank9209 +9210 POINT(33.76217057469441 -119.1112502973941) bank9210 +9211 POINT(33.67965035858506 -118.79452651355045) bank9211 +9212 POINT(33.969718315122336 -117.6524654404713) bank9212 +9213 POINT(34.18893419867989 -118.02103634796575) bank9213 +9214 POINT(34.94802602589046 -118.35730175111779) bank9214 +9215 POINT(33.11550014920829 -119.07450024039173) bank9215 +9216 POINT(35.04095570003588 -117.59569461532591) bank9216 +9217 POINT(34.46503638214779 -118.01865554818798) bank9217 +9218 POINT(33.34887266918521 -117.85302118363373) bank9218 +9219 POINT(33.90709292392461 -117.31996221665257) bank9219 +9220 POINT(34.99342713452643 -118.49101354796906) bank9220 +9221 POINT(34.10767386447518 -118.14585181068729) bank9221 +9222 POINT(33.80587830033437 -118.78277305372086) bank9222 +9223 POINT(33.6836758646493 -118.62297305178085) bank9223 +9224 POINT(34.33166415234798 -119.0849524042942) bank9224 +9225 POINT(34.01233069945502 -118.8677238037171) bank9225 +9226 POINT(34.3064993161709 -117.80799491251324) bank9226 +9227 POINT(33.15820532900695 -118.25375695144467) bank9227 +9228 POINT(33.44838869540094 -118.75503899630534) bank9228 +9229 POINT(34.08989363613716 -117.90341457865318) bank9229 +9230 POINT(33.63103727577793 -117.98966399156646) bank9230 +9231 POINT(34.87870549206271 -118.60679002377208) bank9231 +9232 POINT(34.40412740848253 -117.29148848020967) bank9232 +9233 POINT(34.12767285319824 -118.45879388962018) bank9233 +9234 POINT(33.31129954090242 -117.404139718956) bank9234 +9235 POINT(33.4781889506028 -118.0702571113267) bank9235 +9236 POINT(33.58804170612411 -117.87455582949474) bank9236 +9237 POINT(34.59929525495169 -118.33494380877347) bank9237 +9238 POINT(34.2210654939041 -118.76125464227387) bank9238 +9239 POINT(34.4803701972461 -118.13636263697444) bank9239 +9240 POINT(33.2150885824595 -118.33171580114086) bank9240 +9241 POINT(33.22950703490834 -117.83404858596899) bank9241 +9242 POINT(33.15240013454879 -117.88580308172529) bank9242 +9243 POINT(33.980086009940514 -118.08044816568066) bank9243 +9244 POINT(34.65256883522332 -118.4409750906619) bank9244 +9245 POINT(34.81879105004155 -119.2420509525283) bank9245 +9246 POINT(34.872699660660736 -118.13340632716543) bank9246 +9247 POINT(34.865790803854324 -119.21862354511087) bank9247 +9248 POINT(34.99177455200157 -117.31565572258978) bank9248 +9249 POINT(33.33656754597751 -117.8105929655827) bank9249 +9250 POINT(33.3284364914317 -117.51697012101809) bank9250 +9251 POINT(34.09334263412198 -117.53634455438502) bank9251 +9252 POINT(34.64125874129551 -117.52646910176665) bank9252 +9253 POINT(33.1086315302344 -118.53556955422674) bank9253 +9254 POINT(34.26250548028331 -118.12574711795452) bank9254 +9255 POINT(33.67722299170593 -117.97672572786495) bank9255 +9256 POINT(33.270781589579364 -118.41942617129676) bank9256 +9257 POINT(34.676913349329304 -118.37795424049921) bank9257 +9258 POINT(33.11443757074876 -117.58810330511618) bank9258 +9259 POINT(33.41714288707907 -119.06975399693349) bank9259 +9260 POINT(34.59478241004984 -117.82002356326267) bank9260 +9261 POINT(33.72305307209447 -117.31431858772494) bank9261 +9262 POINT(34.480930055240215 -118.07306757456186) bank9262 +9263 POINT(34.5488289160698 -117.62918507661912) bank9263 +9264 POINT(33.337415229733715 -119.11147653326077) bank9264 +9265 POINT(33.20622050592272 -118.70096154319961) bank9265 +9266 POINT(34.23726116654721 -117.55620489053078) bank9266 +9267 POINT(34.09792266022408 -117.93588191756919) bank9267 +9268 POINT(33.524705457270294 -118.26066856579129) bank9268 +9269 POINT(33.642111486675866 -117.49283744232878) bank9269 +9270 POINT(33.38384262349668 -118.79631364736038) bank9270 +9271 POINT(34.18619208054665 -118.96577990436703) bank9271 +9272 POINT(33.21827557222965 -118.18283129860939) bank9272 +9273 POINT(34.96370475086029 -118.52403066565621) bank9273 +9274 POINT(34.97039200086634 -118.27643235528399) bank9274 +9275 POINT(34.86729887978793 -118.10156120747354) bank9275 +9276 POINT(34.79389391773509 -117.26993753877476) bank9276 +9277 POINT(33.07334381615893 -118.30052894511859) bank9277 +9278 POINT(33.24410221527398 -119.14263546600559) bank9278 +9279 POINT(34.25656205765789 -118.33647764147042) bank9279 +9280 POINT(33.220375782918936 -117.52615089776508) bank9280 +9281 POINT(33.52793046706718 -117.34413388465926) bank9281 +9282 POINT(34.901655329261374 -118.33337677338157) bank9282 +9283 POINT(34.88993038664335 -118.8628229063882) bank9283 +9284 POINT(33.45868760305009 -118.26830352530513) bank9284 +9285 POINT(34.51114802907875 -117.90291707181802) bank9285 +9286 POINT(33.52971878901144 -118.67083171491095) bank9286 +9287 POINT(33.88051633480183 -117.48592700350484) bank9287 +9288 POINT(34.19253274208346 -117.64522392939075) bank9288 +9289 POINT(34.18806187261468 -119.0175791359286) bank9289 +9290 POINT(33.2745877812149 -118.98176134033912) bank9290 +9291 POINT(33.16026232998756 -118.06102601185607) bank9291 +9292 POINT(34.780068330395665 -118.00642988083445) bank9292 +9293 POINT(34.19371664583268 -118.84587642271855) bank9293 +9294 POINT(33.714751133472184 -118.34660180396351) bank9294 +9295 POINT(33.20166890803384 -119.01262242932377) bank9295 +9296 POINT(33.35499847078452 -117.35187986675483) bank9296 +9297 POINT(33.66909702197295 -118.18131448593428) bank9297 +9298 POINT(33.2255823961098 -119.15494270970075) bank9298 +9299 POINT(34.692178099612555 -118.42237325229041) bank9299 +9300 POINT(33.65307800426444 -117.32729530550401) bank9300 +9301 POINT(33.36831377918774 -118.61389280484833) bank9301 +9302 POINT(33.489505776970375 -117.69774215278203) bank9302 +9303 POINT(34.30057506846968 -119.13307473011517) bank9303 +9304 POINT(34.62708314895694 -118.0375741688916) bank9304 +9305 POINT(34.57933502395587 -117.68377154046539) bank9305 +9306 POINT(33.064841198927056 -117.58341256882187) bank9306 +9307 POINT(34.64851320196201 -119.21158490234782) bank9307 +9308 POINT(34.08041502048211 -118.26650641703047) bank9308 +9309 POINT(33.621905717676924 -118.51666526097293) bank9309 +9310 POINT(34.933125173063054 -118.69003261651332) bank9310 +9311 POINT(33.665718000274616 -118.43606033682856) bank9311 +9312 POINT(34.30448568794605 -117.37451967257208) bank9312 +9313 POINT(33.36888814870527 -118.07706479575484) bank9313 +9314 POINT(34.701349428480356 -117.87536431585025) bank9314 +9315 POINT(33.9489688774268 -119.04227725129682) bank9315 +9316 POINT(33.2580748533481 -119.21700128705602) bank9316 +9317 POINT(35.023032548663984 -118.66699623458769) bank9317 +9318 POINT(33.22374344589466 -119.02872554043923) bank9318 +9319 POINT(34.653726385026246 -118.1654626447579) bank9319 +9320 POINT(34.73334736107044 -117.37667115098986) bank9320 +9321 POINT(35.02056362212472 -118.38706440416836) bank9321 +9322 POINT(34.03838789747246 -117.68622026206788) bank9322 +9323 POINT(34.61723417058682 -117.36355880864083) bank9323 +9324 POINT(34.52392558291211 -117.52398597420287) bank9324 +9325 POINT(34.733271136873746 -117.64693413777813) bank9325 +9326 POINT(33.58165163960917 -117.44785534786942) bank9326 +9327 POINT(33.948074426640346 -118.32944285461734) bank9327 +9328 POINT(34.95140787261751 -118.30459588980148) bank9328 +9329 POINT(33.66024301645173 -117.33148857890299) bank9329 +9330 POINT(33.44601680307011 -117.79123179769422) bank9330 +9331 POINT(34.645257081173355 -117.54954192368581) bank9331 +9332 POINT(34.994056354920986 -118.24194674250796) bank9332 +9333 POINT(34.731097060497895 -118.12395276208181) bank9333 +9334 POINT(34.56608068629525 -118.5667000641078) bank9334 +9335 POINT(33.28276107867039 -118.76402711151374) bank9335 +9336 POINT(34.757296543380185 -117.39802602579088) bank9336 +9337 POINT(34.53566274567386 -118.73240095290431) bank9337 +9338 POINT(33.77263568243955 -117.91619290826699) bank9338 +9339 POINT(33.324931386731556 -118.22459460631731) bank9339 +9340 POINT(34.17358109044072 -119.17035538742721) bank9340 +9341 POINT(34.01673688145377 -117.86929325813851) bank9341 +9342 POINT(34.766378055290886 -118.68155312669872) bank9342 +9343 POINT(34.44779989970244 -118.85081633588784) bank9343 +9344 POINT(34.42940902774347 -117.3440170818257) bank9344 +9345 POINT(33.45382585141725 -118.9816115198047) bank9345 +9346 POINT(33.901010895168355 -117.90329325389679) bank9346 +9347 POINT(34.62509467256578 -117.34811681657825) bank9347 +9348 POINT(34.77329431611256 -118.33254341356435) bank9348 +9349 POINT(34.923719125440286 -118.6676318253098) bank9349 +9350 POINT(34.98280109892952 -117.69667110017804) bank9350 +9351 POINT(34.54757039664786 -118.61737672270716) bank9351 +9352 POINT(34.19307857230731 -118.14096267534978) bank9352 +9353 POINT(33.97563714684903 -117.50025319589435) bank9353 +9354 POINT(33.53600574710702 -118.65068601522614) bank9354 +9355 POINT(34.17626756518109 -118.88388724031766) bank9355 +9356 POINT(33.71368890489566 -117.7605633028345) bank9356 +9357 POINT(35.051124092472605 -117.26686947716458) bank9357 +9358 POINT(34.321061002827854 -119.16678409179205) bank9358 +9359 POINT(34.38904697106094 -118.67882012801151) bank9359 +9360 POINT(35.00218552026213 -117.3070171870313) bank9360 +9361 POINT(33.727326807807074 -119.02949097993799) bank9361 +9362 POINT(33.6788457332421 -117.64121244718433) bank9362 +9363 POINT(34.06151644351918 -117.5701827395547) bank9363 +9364 POINT(33.82392678836228 -119.0435972978925) bank9364 +9365 POINT(33.352385631642704 -118.88082636996232) bank9365 +9366 POINT(34.20983053787383 -117.56107375984367) bank9366 +9367 POINT(34.72495929748126 -117.83451529218331) bank9367 +9368 POINT(34.900470774404106 -119.21056404738279) bank9368 +9369 POINT(33.16897919741988 -117.66784684576609) bank9369 +9370 POINT(34.08172368596372 -118.85816343162637) bank9370 +9371 POINT(34.90328409317832 -119.2243016094589) bank9371 +9372 POINT(33.54611966382894 -118.73091533280794) bank9372 +9373 POINT(34.26124494197042 -118.35407524652193) bank9373 +9374 POINT(33.26579356935596 -118.35716162272692) bank9374 +9375 POINT(33.416941979149506 -119.16355043932539) bank9375 +9376 POINT(33.26659622856492 -117.46609370857426) bank9376 +9377 POINT(34.313176815040194 -117.71527135771424) bank9377 +9378 POINT(33.20532775089781 -118.32469595307072) bank9378 +9379 POINT(33.5818490893926 -117.29697778863513) bank9379 +9380 POINT(33.85199605906753 -118.41381310490095) bank9380 +9381 POINT(34.28061632356257 -117.71360481420442) bank9381 +9382 POINT(34.65866978611447 -118.7380318226335) bank9382 +9383 POINT(34.65394108240751 -118.76914931176036) bank9383 +9384 POINT(33.90094020821244 -117.6347242153717) bank9384 +9385 POINT(34.302463825353605 -117.76667706102046) bank9385 +9386 POINT(34.610706899239936 -118.18014194837224) bank9386 +9387 POINT(34.16455955271574 -118.69710173292458) bank9387 +9388 POINT(33.20531908860993 -117.39467171836506) bank9388 +9389 POINT(34.60503384933415 -118.78571387190574) bank9389 +9390 POINT(34.78645647548895 -117.73533879770842) bank9390 +9391 POINT(33.400896799006105 -118.80550089004316) bank9391 +9392 POINT(34.81846366339428 -117.78144178442517) bank9392 +9393 POINT(33.11883058410305 -118.40248760455458) bank9393 +9394 POINT(33.751659700585336 -118.54293788479917) bank9394 +9395 POINT(33.42321201856709 -118.98670680577942) bank9395 +9396 POINT(33.870647709224315 -118.4692689235411) bank9396 +9397 POINT(34.53759473019086 -117.41725548100739) bank9397 +9398 POINT(34.87832431921191 -117.66968757357242) bank9398 +9399 POINT(33.60863978164676 -118.74126907578514) bank9399 +9400 POINT(34.44030826353042 -119.10318928687434) bank9400 +9401 POINT(34.56977647236209 -119.17178191181424) bank9401 +9402 POINT(33.197965398887625 -117.47336157136553) bank9402 +9403 POINT(33.55892146809833 -118.00067934780571) bank9403 +9404 POINT(34.90794495993449 -117.93466121994561) bank9404 +9405 POINT(34.41265663919825 -118.5094128974237) bank9405 +9406 POINT(33.763541353103385 -118.91466989453716) bank9406 +9407 POINT(34.60470967651702 -118.9015817690983) bank9407 +9408 POINT(34.334008516884516 -117.33080717631996) bank9408 +9409 POINT(33.81916743622385 -118.90531889423113) bank9409 +9410 POINT(33.44563093153766 -118.18593934562072) bank9410 +9411 POINT(33.59805205607263 -117.27637224836374) bank9411 +9412 POINT(33.326346519759255 -117.383855147742) bank9412 +9413 POINT(33.463474004612955 -117.87808219047783) bank9413 +9414 POINT(33.6269964737413 -117.24921240545116) bank9414 +9415 POINT(33.307824002431374 -118.3728847311851) bank9415 +9416 POINT(35.00095851045331 -118.92451528326623) bank9416 +9417 POINT(34.844919251511364 -118.86269700568107) bank9417 +9418 POINT(34.827270296524105 -117.34549906206213) bank9418 +9419 POINT(33.711634347154934 -119.20243019386498) bank9419 +9420 POINT(33.91768908628661 -119.2227695741745) bank9420 +9421 POINT(33.29957247655376 -118.00406271345504) bank9421 +9422 POINT(34.40702258471414 -118.50714188656117) bank9422 +9423 POINT(34.88612873386651 -118.13945177976925) bank9423 +9424 POINT(34.98557450184708 -118.3555700633877) bank9424 +9425 POINT(34.0174237013419 -117.6629204431821) bank9425 +9426 POINT(34.07711464070332 -117.8685097291419) bank9426 +9427 POINT(33.61269124226998 -118.98788899618633) bank9427 +9428 POINT(33.72358913381243 -118.6746275537849) bank9428 +9429 POINT(34.16821956653524 -117.7382303580371) bank9429 +9430 POINT(33.71352129343846 -117.25411324270516) bank9430 +9431 POINT(34.43942404179024 -117.36674404324253) bank9431 +9432 POINT(34.035138186962804 -118.15734216448269) bank9432 +9433 POINT(33.069645631541334 -117.85532861895182) bank9433 +9434 POINT(34.5676980963039 -118.37684340104344) bank9434 +9435 POINT(34.15625721394619 -118.62477760030778) bank9435 +9436 POINT(34.7722075104398 -117.57175868930484) bank9436 +9437 POINT(34.34800087120565 -119.02971053898935) bank9437 +9438 POINT(35.020927628490156 -117.41827670891075) bank9438 +9439 POINT(33.29869070397325 -117.51684461440301) bank9439 +9440 POINT(33.28525923407851 -118.24283025126863) bank9440 +9441 POINT(33.1001348947255 -118.90410606427646) bank9441 +9442 POINT(34.03155260633003 -119.22429411833686) bank9442 +9443 POINT(34.512708368930525 -117.80354527626102) bank9443 +9444 POINT(33.58619016704445 -117.33869269226578) bank9444 +9445 POINT(34.30316000731896 -118.55599714004302) bank9445 +9446 POINT(33.68428234701052 -119.12145828184208) bank9446 +9447 POINT(34.77145407346605 -119.21452087853956) bank9447 +9448 POINT(33.20954136913192 -118.24504082965974) bank9448 +9449 POINT(33.92438599020108 -117.32973728234556) bank9449 +9450 POINT(35.01915306070608 -118.51654919617096) bank9450 +9451 POINT(35.01488598731254 -118.99043831935494) bank9451 +9452 POINT(33.21445330535805 -118.09800104467168) bank9452 +9453 POINT(33.83259156613991 -118.17598384480533) bank9453 +9454 POINT(33.78898829505202 -118.60097924593258) bank9454 +9455 POINT(34.98901912496175 -117.41817767649862) bank9455 +9456 POINT(33.31321355212288 -118.05733233589429) bank9456 +9457 POINT(34.015029238879784 -118.58578721572042) bank9457 +9458 POINT(34.37036190537486 -117.57029126006158) bank9458 +9459 POINT(33.6879866593932 -117.73758665540512) bank9459 +9460 POINT(34.37167317817762 -118.0872757774062) bank9460 +9461 POINT(33.5931863138639 -117.31258320931636) bank9461 +9462 POINT(34.23662798050667 -119.14228121311476) bank9462 +9463 POINT(33.787611629325866 -118.1212525656058) bank9463 +9464 POINT(33.98930881147352 -118.6831670489269) bank9464 +9465 POINT(33.76065245608238 -117.38927643542355) bank9465 +9466 POINT(33.51097502483338 -118.18679687674073) bank9466 +9467 POINT(33.95875815996471 -118.87473548263222) bank9467 +9468 POINT(34.3512585840939 -118.6432555589235) bank9468 +9469 POINT(33.708186688024874 -117.97946405789374) bank9469 +9470 POINT(34.8496700404719 -117.59079560202275) bank9470 +9471 POINT(34.55729393486879 -118.52927646687982) bank9471 +9472 POINT(33.69392199232216 -118.51395627313313) bank9472 +9473 POINT(34.06989892052401 -118.46180357581211) bank9473 +9474 POINT(33.501636851133085 -118.82992731319948) bank9474 +9475 POINT(33.18729171777941 -118.89225242733576) bank9475 +9476 POINT(34.30652421402342 -118.33401128709194) bank9476 +9477 POINT(33.22050186816342 -118.00729283199544) bank9477 +9478 POINT(34.68050051928623 -117.90115937697433) bank9478 +9479 POINT(33.710826136958666 -118.32097664080393) bank9479 +9480 POINT(34.71299631556104 -118.62398829373211) bank9480 +9481 POINT(34.8911377988276 -117.54529124933586) bank9481 +9482 POINT(33.71832706125504 -117.44073467556609) bank9482 +9483 POINT(34.021036025896215 -118.10165749237392) bank9483 +9484 POINT(33.6357067284117 -118.70900329880295) bank9484 +9485 POINT(33.66377557932813 -118.93976149862856) bank9485 +9486 POINT(33.25268844590524 -119.01242205427681) bank9486 +9487 POINT(33.81787216984028 -118.29172281465627) bank9487 +9488 POINT(33.5869787668145 -117.77841035633855) bank9488 +9489 POINT(33.70464431220405 -118.81344869102907) bank9489 +9490 POINT(33.529281057741066 -117.74473418977982) bank9490 +9491 POINT(34.17506977739198 -117.67835422126612) bank9491 +9492 POINT(33.25460739000581 -119.09242077242261) bank9492 +9493 POINT(34.417871691047615 -119.22281281601566) bank9493 +9494 POINT(33.2735975621128 -118.22756370893099) bank9494 +9495 POINT(33.974570041405606 -118.75059709959741) bank9495 +9496 POINT(33.140215344729135 -117.54833601336881) bank9496 +9497 POINT(33.31889561708534 -118.83255045088413) bank9497 +9498 POINT(33.23660099759505 -117.31407413037367) bank9498 +9499 POINT(33.5165486255196 -119.05631611647367) bank9499 +9500 POINT(34.128504660372464 -117.84914717364957) bank9500 +9501 POINT(34.179522679792946 -117.54837288006787) bank9501 +9502 POINT(34.53112596748729 -117.51997842467279) bank9502 +9503 POINT(34.25979735649571 -118.10631438326715) bank9503 +9504 POINT(33.80700402468004 -118.2362113097896) bank9504 +9505 POINT(34.37769722959547 -117.40852052987731) bank9505 +9506 POINT(34.27882471184138 -119.13640996409515) bank9506 +9507 POINT(33.0966485047203 -117.93812552151415) bank9507 +9508 POINT(34.60159708471465 -118.43332562698757) bank9508 +9509 POINT(35.02466433966965 -118.037553669692) bank9509 +9510 POINT(33.31669990294085 -118.8554558759806) bank9510 +9511 POINT(33.69954306388412 -117.99833952237609) bank9511 +9512 POINT(34.73869894646834 -117.97455133938091) bank9512 +9513 POINT(34.009238117015094 -118.35954084371369) bank9513 +9514 POINT(33.70636940344264 -117.78865740871743) bank9514 +9515 POINT(33.89473663949832 -117.9498004542775) bank9515 +9516 POINT(34.639876983918455 -119.08662157888779) bank9516 +9517 POINT(34.3369876950985 -118.5569424265462) bank9517 +9518 POINT(33.7346124477862 -117.62810436907796) bank9518 +9519 POINT(33.55867044893299 -118.91619751802922) bank9519 +9520 POINT(33.60792068600268 -117.97491320735327) bank9520 +9521 POINT(33.85131158473275 -118.38028238480261) bank9521 +9522 POINT(34.75469832868193 -117.34969477500549) bank9522 +9523 POINT(33.756315795854896 -118.74054435886666) bank9523 +9524 POINT(33.219260728338675 -118.96843829011884) bank9524 +9525 POINT(34.279486527237 -119.16953729117155) bank9525 +9526 POINT(33.22832731328322 -118.35880783950121) bank9526 +9527 POINT(34.67077652716235 -119.09168262649887) bank9527 +9528 POINT(34.856878713868554 -118.72996013147718) bank9528 +9529 POINT(33.613116332314924 -118.9291592757487) bank9529 +9530 POINT(34.87393480274087 -118.74696821380965) bank9530 +9531 POINT(33.40805534598586 -118.76356727130263) bank9531 +9532 POINT(34.581546748994306 -117.4227310579766) bank9532 +9533 POINT(34.07008897550873 -118.66010827280138) bank9533 +9534 POINT(33.344915675911935 -117.28629948559684) bank9534 +9535 POINT(34.56484474648538 -117.9581843262352) bank9535 +9536 POINT(34.30129506566006 -118.6289554647415) bank9536 +9537 POINT(34.54681080197927 -119.1541960361246) bank9537 +9538 POINT(34.457428049424585 -118.47835349530084) bank9538 +9539 POINT(33.34410163933881 -119.0668958366217) bank9539 +9540 POINT(34.71783983645115 -118.97716786538318) bank9540 +9541 POINT(33.749022020247985 -119.18333565406387) bank9541 +9542 POINT(33.671791821982175 -117.36801229119517) bank9542 +9543 POINT(33.59095999840813 -117.40337686362867) bank9543 +9544 POINT(33.98512922636511 -117.51312261198034) bank9544 +9545 POINT(33.056604103569185 -118.33102957427145) bank9545 +9546 POINT(33.60540983146354 -118.31165909393115) bank9546 +9547 POINT(33.41588668583206 -117.43072598191793) bank9547 +9548 POINT(34.446647919872355 -118.18561986787849) bank9548 +9549 POINT(34.362600571112495 -118.19689367636742) bank9549 +9550 POINT(34.12661811926973 -119.11850501127283) bank9550 +9551 POINT(34.88261024483833 -119.0008705085543) bank9551 +9552 POINT(34.05076472371433 -117.62019200947321) bank9552 +9553 POINT(33.078077822219996 -118.84026708806884) bank9553 +9554 POINT(34.76667848997845 -117.36316374110108) bank9554 +9555 POINT(33.83349919301047 -117.790874153814) bank9555 +9556 POINT(33.70652906182958 -117.99758144455616) bank9556 +9557 POINT(33.93994421565522 -118.24974711798674) bank9557 +9558 POINT(33.63172589971846 -117.8594371846621) bank9558 +9559 POINT(33.89226876368406 -117.9617468676756) bank9559 +9560 POINT(33.39362405423665 -119.07420855667286) bank9560 +9561 POINT(33.73392344522721 -117.347693714936) bank9561 +9562 POINT(33.99421701217709 -117.7923619038067) bank9562 +9563 POINT(34.701225278439175 -118.0325012533716) bank9563 +9564 POINT(33.39956819425726 -118.30150379325116) bank9564 +9565 POINT(33.355971502583316 -117.96913021690743) bank9565 +9566 POINT(34.884180055170184 -118.32736741283354) bank9566 +9567 POINT(34.799001465411216 -117.69899531896658) bank9567 +9568 POINT(33.61484876792725 -117.43561676128067) bank9568 +9569 POINT(33.741644075154035 -118.88363078502773) bank9569 +9570 POINT(33.763411982082644 -117.34239054142597) bank9570 +9571 POINT(33.075432669228874 -119.06875281993989) bank9571 +9572 POINT(34.78343283768185 -119.13075840729016) bank9572 +9573 POINT(33.21037265532442 -117.41168079164687) bank9573 +9574 POINT(33.73210197817567 -118.52711073346612) bank9574 +9575 POINT(33.06034502524091 -118.54594383484428) bank9575 +9576 POINT(34.703616619001345 -118.84537624772027) bank9576 +9577 POINT(33.30953004739587 -118.27548597427015) bank9577 +9578 POINT(33.12970637374816 -117.72905344903238) bank9578 +9579 POINT(33.67881275362579 -118.73034275241628) bank9579 +9580 POINT(34.117048873567796 -119.00287616135743) bank9580 +9581 POINT(33.12289180766221 -118.38722774329133) bank9581 +9582 POINT(33.87093787096965 -118.85145025455846) bank9582 +9583 POINT(33.68906960958989 -119.21353396365109) bank9583 +9584 POINT(34.53772501163326 -119.02162020759259) bank9584 +9585 POINT(33.773786764361404 -118.76809922635653) bank9585 +9586 POINT(34.24050930331466 -117.24535697717423) bank9586 +9587 POINT(34.699377374038335 -118.27455784891735) bank9587 +9588 POINT(33.98496320777044 -117.57749376993175) bank9588 +9589 POINT(33.13752196772848 -118.41380446571755) bank9589 +9590 POINT(33.068629284060435 -117.49497651548327) bank9590 +9591 POINT(33.91043376684052 -118.5879573412084) bank9591 +9592 POINT(34.764614623177984 -118.26861975405131) bank9592 +9593 POINT(33.633977971066166 -118.9809524533109) bank9593 +9594 POINT(34.75966831849066 -118.63437587979048) bank9594 +9595 POINT(33.66544994930542 -119.14379004394556) bank9595 +9596 POINT(34.38130776492027 -117.84876271626422) bank9596 +9597 POINT(33.37291498960238 -118.08274477755326) bank9597 +9598 POINT(35.03100361719943 -119.09683146579418) bank9598 +9599 POINT(34.60448834204157 -118.51959151614358) bank9599 +9600 POINT(34.407373173767844 -117.47702957286651) bank9600 +9601 POINT(34.92847337925384 -117.956680421132) bank9601 +9602 POINT(34.667375940603264 -118.72283625525564) bank9602 +9603 POINT(34.20618277557575 -117.65410837098973) bank9603 +9604 POINT(34.33652279026558 -118.68466277088152) bank9604 +9605 POINT(34.34016776490529 -118.6940815231627) bank9605 +9606 POINT(34.73204700640471 -118.06115291672138) bank9606 +9607 POINT(33.77774915655287 -117.59174106928056) bank9607 +9608 POINT(33.88974406042578 -117.30757769505027) bank9608 +9609 POINT(34.55938767828738 -118.0529286093988) bank9609 +9610 POINT(34.9938652528471 -118.3808314859274) bank9610 +9611 POINT(33.71117471730256 -118.54701379505397) bank9611 +9612 POINT(33.06686968440799 -117.64028457721552) bank9612 +9613 POINT(34.606702300893396 -117.98564437066564) bank9613 +9614 POINT(34.27078386327059 -117.6777471030248) bank9614 +9615 POINT(33.978758314389104 -118.8009073931705) bank9615 +9616 POINT(33.72593163397022 -118.72401880126064) bank9616 +9617 POINT(34.25483581261786 -117.59462693557234) bank9617 +9618 POINT(34.71224704457454 -118.32008021896358) bank9618 +9619 POINT(34.63254898827275 -117.52883592440242) bank9619 +9620 POINT(33.3697673207298 -118.39389475036805) bank9620 +9621 POINT(34.96033063630858 -119.23461001438905) bank9621 +9622 POINT(34.897212064653544 -118.16660361860175) bank9622 +9623 POINT(33.43351968664423 -118.0932500829074) bank9623 +9624 POINT(34.64475551340371 -118.99612776599594) bank9624 +9625 POINT(34.13725092626231 -117.31719630408026) bank9625 +9626 POINT(34.03156789503415 -118.02806073338115) bank9626 +9627 POINT(33.42199587003108 -117.30299748891673) bank9627 +9628 POINT(34.55016200865813 -117.62829775758458) bank9628 +9629 POINT(33.4129991303475 -118.8488372218608) bank9629 +9630 POINT(33.459188432486215 -117.27187997480448) bank9630 +9631 POINT(34.59735584844849 -117.8823169055491) bank9631 +9632 POINT(34.879239545306476 -117.50648115597878) bank9632 +9633 POINT(34.3751978823365 -118.72920949634478) bank9633 +9634 POINT(34.15763847260992 -117.78478597057484) bank9634 +9635 POINT(33.98386737937406 -118.67051998530481) bank9635 +9636 POINT(33.91325516415462 -118.25929491270563) bank9636 +9637 POINT(34.62733907008395 -118.35560326049274) bank9637 +9638 POINT(34.77838782695036 -117.64980012897811) bank9638 +9639 POINT(33.72981015746288 -118.27885580565942) bank9639 +9640 POINT(34.85252117528323 -117.24526395129863) bank9640 +9641 POINT(33.991439508053546 -118.30872633123677) bank9641 +9642 POINT(33.41060990538447 -118.37385713116623) bank9642 +9643 POINT(33.85096036097811 -117.3615309524343) bank9643 +9644 POINT(33.693288575331515 -119.1381794110326) bank9644 +9645 POINT(34.39069797578705 -117.69671510607566) bank9645 +9646 POINT(34.964346974489004 -118.67561166136743) bank9646 +9647 POINT(34.17798376832022 -118.13134288177768) bank9647 +9648 POINT(33.25207878320601 -118.89641130190108) bank9648 +9649 POINT(33.171253498110936 -118.1680772112254) bank9649 +9650 POINT(34.05229573172392 -118.25771730690809) bank9650 +9651 POINT(33.92099002909718 -119.18916780383758) bank9651 +9652 POINT(34.76358762949355 -118.40621603194485) bank9652 +9653 POINT(35.03177787674325 -118.9803150251862) bank9653 +9654 POINT(34.15574108022219 -118.39400165257886) bank9654 +9655 POINT(34.33666583840895 -117.79298847020257) bank9655 +9656 POINT(34.551052773806724 -117.35497026394516) bank9656 +9657 POINT(33.415232765454256 -119.23706825638233) bank9657 +9658 POINT(34.04423954905878 -118.5375344627027) bank9658 +9659 POINT(34.71633889671223 -117.48973195673952) bank9659 +9660 POINT(33.709929141583245 -117.8410491519406) bank9660 +9661 POINT(34.53586500171147 -117.68198957445523) bank9661 +9662 POINT(34.296057515958005 -118.46327924941238) bank9662 +9663 POINT(33.16961950948095 -117.5816279598558) bank9663 +9664 POINT(34.2281593686709 -117.92670678357466) bank9664 +9665 POINT(33.896829340918934 -117.37718794023843) bank9665 +9666 POINT(34.00662531388209 -119.20618690070317) bank9666 +9667 POINT(34.05282664561759 -118.90297712618495) bank9667 +9668 POINT(33.68246556298711 -117.72847787788375) bank9668 +9669 POINT(34.10974136465009 -118.22449107756688) bank9669 +9670 POINT(34.62391080703549 -117.41445880759213) bank9670 +9671 POINT(34.87832705219186 -118.6936240872172) bank9671 +9672 POINT(34.340003906215664 -117.44909694185425) bank9672 +9673 POINT(34.39811107374388 -119.18581858309544) bank9673 +9674 POINT(33.44564014218356 -118.80975886096003) bank9674 +9675 POINT(34.95135869030609 -118.27390572084809) bank9675 +9676 POINT(34.97461032987164 -118.89648847751081) bank9676 +9677 POINT(34.80175649598399 -117.32366663400023) bank9677 +9678 POINT(35.008019579963424 -119.04085432038643) bank9678 +9679 POINT(33.232847454392385 -117.47441128776964) bank9679 +9680 POINT(35.040127751510596 -117.44560426534008) bank9680 +9681 POINT(33.93079002417285 -118.35723004756116) bank9681 +9682 POINT(33.39403238752003 -117.49616108436628) bank9682 +9683 POINT(34.62621145304076 -118.49263683491014) bank9683 +9684 POINT(34.09399156967596 -118.95963675649271) bank9684 +9685 POINT(34.334107871681574 -117.41289608764433) bank9685 +9686 POINT(34.67464357084893 -117.56097549596706) bank9686 +9687 POINT(34.35533216302353 -118.823354796226) bank9687 +9688 POINT(34.97380010270158 -117.40737478131929) bank9688 +9689 POINT(33.625044608040376 -118.9118868147393) bank9689 +9690 POINT(33.15334974792469 -118.4703745899541) bank9690 +9691 POINT(33.74748187358046 -117.86736640702175) bank9691 +9692 POINT(34.2464917316466 -118.76289116374097) bank9692 +9693 POINT(33.38912810917615 -118.28924094349762) bank9693 +9694 POINT(34.907222173902845 -118.95883158451043) bank9694 +9695 POINT(34.82567777166678 -117.36642162904533) bank9695 +9696 POINT(33.4603427157879 -117.93845127790914) bank9696 +9697 POINT(33.849493825081055 -117.8443549153151) bank9697 +9698 POINT(33.234527142388316 -117.53713556686877) bank9698 +9699 POINT(34.01938736473426 -117.43975711398872) bank9699 +9700 POINT(34.84433742643061 -118.97026742560003) bank9700 +9701 POINT(33.44486374957245 -119.09905882626215) bank9701 +9702 POINT(34.8426610643427 -118.5698020309893) bank9702 +9703 POINT(33.75954895792412 -118.15215921941581) bank9703 +9704 POINT(34.414725436327785 -118.71907148180019) bank9704 +9705 POINT(33.17346295001098 -118.26240877308744) bank9705 +9706 POINT(34.051318136771435 -117.3848179691237) bank9706 +9707 POINT(33.56052043416566 -118.29482080465931) bank9707 +9708 POINT(34.79095748406567 -118.96838111385786) bank9708 +9709 POINT(34.93925505380345 -118.39608151434422) bank9709 +9710 POINT(33.05549374914025 -119.17728099395016) bank9710 +9711 POINT(33.364645979570874 -118.6162910674466) bank9711 +9712 POINT(35.03630510137342 -118.59257291918101) bank9712 +9713 POINT(34.345819636143105 -118.47182936144596) bank9713 +9714 POINT(33.38351266217681 -118.7974929601628) bank9714 +9715 POINT(34.46624191882946 -117.75221988633024) bank9715 +9716 POINT(34.823332019343894 -118.60862714947135) bank9716 +9717 POINT(33.58464984771126 -118.718988168836) bank9717 +9718 POINT(33.58631672855724 -118.98822370605264) bank9718 +9719 POINT(34.67385671683032 -118.93343329835645) bank9719 +9720 POINT(34.14164783886362 -117.55950261116331) bank9720 +9721 POINT(33.508792170108315 -117.31935470515826) bank9721 +9722 POINT(34.77886588562274 -117.59232211758959) bank9722 +9723 POINT(34.548995060224925 -118.29651453870052) bank9723 +9724 POINT(34.074195948260254 -118.56945354437919) bank9724 +9725 POINT(34.52345535068137 -118.3557137233254) bank9725 +9726 POINT(34.42452847800004 -118.95347713758366) bank9726 +9727 POINT(34.28692930714021 -117.60551208975727) bank9727 +9728 POINT(34.11869657489538 -117.35783048578539) bank9728 +9729 POINT(33.96742338962854 -117.51146104218523) bank9729 +9730 POINT(33.21211656033135 -118.36093224020257) bank9730 +9731 POINT(34.17270865781583 -118.2211116644684) bank9731 +9732 POINT(33.68758415789921 -117.7706510723286) bank9732 +9733 POINT(33.51842752392481 -119.0607629497005) bank9733 +9734 POINT(34.70907336542768 -117.74134825202627) bank9734 +9735 POINT(33.900591444577586 -118.74993158255172) bank9735 +9736 POINT(33.319676021856544 -118.05505063360219) bank9736 +9737 POINT(33.87036716283196 -118.38070234757329) bank9737 +9738 POINT(34.37289674354055 -117.49443574638482) bank9738 +9739 POINT(33.62348166660508 -119.22510201714698) bank9739 +9740 POINT(34.91221648263957 -118.32067181668586) bank9740 +9741 POINT(33.23069174345123 -118.9839259264723) bank9741 +9742 POINT(34.1030311531558 -118.32167996386471) bank9742 +9743 POINT(33.518313032129925 -117.72895390373155) bank9743 +9744 POINT(33.5772422059045 -118.6777680012454) bank9744 +9745 POINT(33.566122582969925 -118.29881028215624) bank9745 +9746 POINT(34.531751150278694 -118.36557387530559) bank9746 +9747 POINT(34.155159802020194 -119.0495267623652) bank9747 +9748 POINT(33.69775636220759 -118.44316941145671) bank9748 +9749 POINT(34.484917447221456 -117.69413534045873) bank9749 +9750 POINT(33.43320413499965 -118.70941262715706) bank9750 +9751 POINT(33.70976424751573 -119.17647184087893) bank9751 +9752 POINT(34.22756643748103 -117.83288888983873) bank9752 +9753 POINT(33.28559464915717 -118.9228277336576) bank9753 +9754 POINT(34.98731757505075 -118.11386815098825) bank9754 +9755 POINT(34.98935604498223 -118.30859533702385) bank9755 +9756 POINT(34.25274264618329 -117.29965594477505) bank9756 +9757 POINT(33.712660925955454 -118.8596698454944) bank9757 +9758 POINT(33.92082445149081 -118.78801252673642) bank9758 +9759 POINT(33.99341530865873 -118.03910180325654) bank9759 +9760 POINT(34.88146598318852 -118.55675944393484) bank9760 +9761 POINT(34.550180004606645 -117.82090935117789) bank9761 +9762 POINT(33.352178000914 -118.86426827402592) bank9762 +9763 POINT(34.19123476846843 -118.88845573224123) bank9763 +9764 POINT(34.73724714729027 -117.70209436407951) bank9764 +9765 POINT(33.799743730311874 -117.46355155777981) bank9765 +9766 POINT(33.71839129752536 -118.12946064116544) bank9766 +9767 POINT(33.6399725559027 -117.51168601137346) bank9767 +9768 POINT(33.44177997347408 -117.3551406826917) bank9768 +9769 POINT(33.3405561726816 -117.6138372764688) bank9769 +9770 POINT(33.849238759239256 -118.27232991105674) bank9770 +9771 POINT(33.953522107176475 -117.79130738735337) bank9771 +9772 POINT(34.43475141197029 -117.59513975817703) bank9772 +9773 POINT(34.20525970659941 -117.49201286514305) bank9773 +9774 POINT(33.56331335607578 -118.04014609539756) bank9774 +9775 POINT(33.936302918515096 -118.95970793084484) bank9775 +9776 POINT(35.003236567251115 -118.67502304952158) bank9776 +9777 POINT(34.8083243549226 -117.70895219192967) bank9777 +9778 POINT(34.70865706444786 -119.10800952339022) bank9778 +9779 POINT(34.23322527645823 -118.71817231486015) bank9779 +9780 POINT(33.149709594610805 -119.17696689621391) bank9780 +9781 POINT(33.75544152238071 -117.62357076946023) bank9781 +9782 POINT(34.55684920682363 -119.01386854138889) bank9782 +9783 POINT(34.2634245032057 -117.78042390038453) bank9783 +9784 POINT(33.065695827095865 -118.92734704132047) bank9784 +9785 POINT(34.012350543104375 -117.7229307274062) bank9785 +9786 POINT(33.346482019457284 -117.25596873538218) bank9786 +9787 POINT(33.28783328043963 -117.48489914835889) bank9787 +9788 POINT(34.561910351983286 -118.29224014001021) bank9788 +9789 POINT(34.71020969988185 -118.7735327102054) bank9789 +9790 POINT(34.0455685869507 -117.71417888749887) bank9790 +9791 POINT(34.17661531679456 -117.64233701805618) bank9791 +9792 POINT(35.048134443864576 -117.84978554209884) bank9792 +9793 POINT(34.40588604672232 -118.33220245064824) bank9793 +9794 POINT(34.30571483576042 -117.91688223520266) bank9794 +9795 POINT(34.63172749049253 -117.38580130415448) bank9795 +9796 POINT(33.91845228600779 -117.41585234656336) bank9796 +9797 POINT(33.84204945690383 -118.33299812733908) bank9797 +9798 POINT(34.67997530986403 -118.44454956863329) bank9798 +9799 POINT(34.75540452738594 -118.20369067026952) bank9799 +9800 POINT(33.684770564899736 -118.89326080411902) bank9800 +9801 POINT(34.548679478680214 -118.27523363762577) bank9801 +9802 POINT(33.135062966705455 -118.24760630392981) bank9802 +9803 POINT(33.88021346589269 -118.16565775149573) bank9803 +9804 POINT(35.01147124430844 -118.0671470370748) bank9804 +9805 POINT(34.229306546560814 -118.50765372168111) bank9805 +9806 POINT(33.14825020000962 -118.27500636142197) bank9806 +9807 POINT(34.63002579057994 -118.61271187365169) bank9807 +9808 POINT(34.76137706528658 -118.82781524723869) bank9808 +9809 POINT(34.9097304041529 -118.19022670686142) bank9809 +9810 POINT(33.82875577635491 -117.86802406929341) bank9810 +9811 POINT(33.657166747091765 -117.6215197153029) bank9811 +9812 POINT(34.73545182302989 -118.88972597146899) bank9812 +9813 POINT(33.99412370939179 -117.55628073992149) bank9813 +9814 POINT(34.23958082330021 -117.39241131239356) bank9814 +9815 POINT(33.14198811577485 -118.74651826908135) bank9815 +9816 POINT(33.18312236350514 -118.71706358887685) bank9816 +9817 POINT(34.91182110098715 -119.23312536239574) bank9817 +9818 POINT(33.98913475955317 -118.26409959421495) bank9818 +9819 POINT(34.77526824868811 -118.75885255578456) bank9819 +9820 POINT(34.89776160511358 -118.40114453819628) bank9820 +9821 POINT(34.39964810984094 -117.81461426933396) bank9821 +9822 POINT(34.93227233228312 -118.97585508400843) bank9822 +9823 POINT(33.34944081635711 -117.7432332613035) bank9823 +9824 POINT(34.210810131049904 -119.18206423143371) bank9824 +9825 POINT(34.508516908423125 -119.08605450007543) bank9825 +9826 POINT(34.2648441215136 -117.60986217857126) bank9826 +9827 POINT(34.32658824253671 -117.7282216155064) bank9827 +9828 POINT(33.31109869555044 -117.34299906986497) bank9828 +9829 POINT(34.7974542763565 -118.42296492339757) bank9829 +9830 POINT(33.65250015704015 -119.11998873585561) bank9830 +9831 POINT(34.2848835994969 -118.32418908592719) bank9831 +9832 POINT(33.39778459049156 -117.70677549411701) bank9832 +9833 POINT(34.6222504317532 -118.32235311542216) bank9833 +9834 POINT(33.43935522968257 -117.3698167761204) bank9834 +9835 POINT(34.770308903621924 -118.59348194829785) bank9835 +9836 POINT(33.19938773230155 -117.26355605806833) bank9836 +9837 POINT(34.11761193473011 -117.47039832101423) bank9837 +9838 POINT(34.704157532301274 -117.27006864267867) bank9838 +9839 POINT(34.95498574902156 -117.4595015640914) bank9839 +9840 POINT(33.37123257680584 -118.98015340159729) bank9840 +9841 POINT(33.94016011433275 -117.8530125831374) bank9841 +9842 POINT(34.590217445054975 -118.79045460238031) bank9842 +9843 POINT(34.86909186671748 -118.75534164313544) bank9843 +9844 POINT(34.39382518765996 -117.3210909766564) bank9844 +9845 POINT(33.428545669115564 -118.02140082187027) bank9845 +9846 POINT(34.02448746389807 -118.45275774490288) bank9846 +9847 POINT(34.09306903868589 -118.74620795460874) bank9847 +9848 POINT(33.533409440885215 -118.39179197073064) bank9848 +9849 POINT(34.19773961981623 -117.72407802439109) bank9849 +9850 POINT(33.378958682829065 -117.74121766280814) bank9850 +9851 POINT(35.04589547834945 -118.79735709383247) bank9851 +9852 POINT(33.709805255651226 -118.0734436229235) bank9852 +9853 POINT(33.841397374265384 -118.62651851998622) bank9853 +9854 POINT(33.46483353266988 -118.84062983308517) bank9854 +9855 POINT(33.77493895454682 -117.57093265145399) bank9855 +9856 POINT(34.58563964464783 -117.92213629164607) bank9856 +9857 POINT(33.45123384751274 -119.19502472338449) bank9857 +9858 POINT(33.490934451339605 -117.42525377428231) bank9858 +9859 POINT(34.731497892659974 -118.25458637318293) bank9859 +9860 POINT(34.928583723189085 -118.51234856620601) bank9860 +9861 POINT(33.315141290594845 -118.85750007197198) bank9861 +9862 POINT(35.05036774411073 -117.76810644753803) bank9862 +9863 POINT(33.44233772839188 -118.76376947291163) bank9863 +9864 POINT(34.9762764465653 -117.4195088466828) bank9864 +9865 POINT(34.54063303357532 -119.03448589582692) bank9865 +9866 POINT(34.49805203392958 -118.66741863564435) bank9866 +9867 POINT(33.2229543530297 -118.31708490766526) bank9867 +9868 POINT(33.21035056645804 -117.9011680871445) bank9868 +9869 POINT(33.37409700286802 -117.90843923161898) bank9869 +9870 POINT(34.54474158264659 -118.0828849852125) bank9870 +9871 POINT(34.44159668187771 -119.21707382247361) bank9871 +9872 POINT(34.07283550171809 -118.28608124372079) bank9872 +9873 POINT(34.72992234494867 -118.14612267980722) bank9873 +9874 POINT(33.83608717041734 -118.34586721786334) bank9874 +9875 POINT(34.0301313475931 -117.28448279786855) bank9875 +9876 POINT(35.016837197361404 -118.67888174972552) bank9876 +9877 POINT(33.48163967252384 -118.29416427355174) bank9877 +9878 POINT(33.20084928613001 -118.86818725280587) bank9878 +9879 POINT(34.55774689470608 -118.90385847298519) bank9879 +9880 POINT(34.594855540774084 -117.86240184546656) bank9880 +9881 POINT(33.73831568074069 -118.39459770690583) bank9881 +9882 POINT(33.72517220353989 -118.1058545535626) bank9882 +9883 POINT(34.25762050087276 -117.36759469114371) bank9883 +9884 POINT(34.46663719175554 -117.73111398677561) bank9884 +9885 POINT(33.068840794069445 -119.16607809275561) bank9885 +9886 POINT(34.04111136548872 -118.14262992770132) bank9886 +9887 POINT(34.41578912848736 -118.56193494027536) bank9887 +9888 POINT(33.8630585562735 -119.08072744191871) bank9888 +9889 POINT(34.9104957331958 -118.24514116491977) bank9889 +9890 POINT(34.90609400321145 -118.16185572548753) bank9890 +9891 POINT(34.29843794208775 -117.5678379370927) bank9891 +9892 POINT(33.27324648369531 -118.8866372303636) bank9892 +9893 POINT(34.1157119488006 -118.00540123296274) bank9893 +9894 POINT(33.81335946553541 -117.81517764533099) bank9894 +9895 POINT(33.36780528978931 -119.12298554636921) bank9895 +9896 POINT(33.070366401092095 -117.85527959346147) bank9896 +9897 POINT(34.85989037660091 -118.21861760678192) bank9897 +9898 POINT(34.78478341816429 -117.26896287280725) bank9898 +9899 POINT(34.0407025344654 -118.11115351626498) bank9899 +9900 POINT(34.066389010273454 -117.82508278138197) bank9900 +9901 POINT(34.22081363159333 -118.54726339469508) bank9901 +9902 POINT(34.41762720605962 -118.01618352614382) bank9902 +9903 POINT(34.742883868771806 -118.570139923538) bank9903 +9904 POINT(33.732596052986224 -118.02688370418625) bank9904 +9905 POINT(33.26930630345436 -117.40978186238011) bank9905 +9906 POINT(34.79230011513179 -118.05833835450203) bank9906 +9907 POINT(34.55949352167968 -117.5468804406223) bank9907 +9908 POINT(34.777272476628454 -117.48191404915902) bank9908 +9909 POINT(34.92980342185485 -119.12581776577235) bank9909 +9910 POINT(34.585632559469005 -118.2352034501717) bank9910 +9911 POINT(33.61577111363317 -118.34323171026918) bank9911 +9912 POINT(34.41483689622062 -117.38241209570448) bank9912 +9913 POINT(33.75021232465238 -118.15072422728932) bank9913 +9914 POINT(33.408630758513766 -118.9499232839582) bank9914 +9915 POINT(33.93133074829818 -117.36582865502533) bank9915 +9916 POINT(33.37047835106531 -118.52369091042843) bank9916 +9917 POINT(34.992568278005514 -117.92991841254029) bank9917 +9918 POINT(34.62678152849956 -118.0735016900212) bank9918 +9919 POINT(34.43541582230018 -118.49596834922292) bank9919 +9920 POINT(34.04670802696812 -117.72669287318877) bank9920 +9921 POINT(33.5854551608759 -117.51459359671551) bank9921 +9922 POINT(33.76991177051141 -117.87413850296441) bank9922 +9923 POINT(33.59200264828525 -118.85650814258865) bank9923 +9924 POINT(34.41203547757433 -118.8287922611124) bank9924 +9925 POINT(33.936075791786806 -117.8141610227175) bank9925 +9926 POINT(33.109741837060284 -119.11479729366519) bank9926 +9927 POINT(33.221295414611845 -117.78334431862774) bank9927 +9928 POINT(34.986958291269474 -117.53999885398272) bank9928 +9929 POINT(33.409739522983436 -117.71426732576154) bank9929 +9930 POINT(33.765367992894085 -117.59138234338923) bank9930 +9931 POINT(33.66077036375853 -118.87125675448553) bank9931 +9932 POINT(33.592318037356335 -118.52893916008937) bank9932 +9933 POINT(33.38938243846594 -118.57618155202006) bank9933 +9934 POINT(34.02147876006302 -119.08870658185896) bank9934 +9935 POINT(34.73305204503821 -118.76542864198167) bank9935 +9936 POINT(34.662356460036065 -118.31857370791909) bank9936 +9937 POINT(34.07234594975444 -118.33465652584404) bank9937 +9938 POINT(34.91423028807852 -117.36728683171968) bank9938 +9939 POINT(33.566713283466136 -117.48993158595903) bank9939 +9940 POINT(34.61817889973814 -117.98654174406907) bank9940 +9941 POINT(33.71849727665264 -118.19823188677003) bank9941 +9942 POINT(34.16849692528191 -118.98518714659096) bank9942 +9943 POINT(34.62915088183803 -118.23397920802083) bank9943 +9944 POINT(33.48714586010774 -118.25570759076639) bank9944 +9945 POINT(34.0067736969921 -118.66074524176366) bank9945 +9946 POINT(34.27561178228272 -118.3267467161743) bank9946 +9947 POINT(34.606267104621686 -117.4006156402375) bank9947 +9948 POINT(34.57749936156475 -119.03447018125546) bank9948 +9949 POINT(33.93353347242117 -118.62996924674135) bank9949 +9950 POINT(33.29886017903517 -118.27717968078457) bank9950 +9951 POINT(34.330926191253326 -117.44951111381098) bank9951 +9952 POINT(34.76335946257084 -119.1233841747141) bank9952 +9953 POINT(34.98745020652099 -118.65672230856445) bank9953 +9954 POINT(34.83655732973827 -118.00413773501519) bank9954 +9955 POINT(33.747773258338334 -117.61026679694405) bank9955 +9956 POINT(33.79856177156159 -118.54851056595923) bank9956 +9957 POINT(34.83415450269044 -119.11111961468814) bank9957 +9958 POINT(34.65033949288087 -118.33539772442268) bank9958 +9959 POINT(33.74347155627151 -117.82367378305567) bank9959 +9960 POINT(34.20008244683353 -117.34043498065226) bank9960 +9961 POINT(33.39437281411841 -118.59090488364149) bank9961 +9962 POINT(33.274540051965886 -119.15153548935105) bank9962 +9963 POINT(34.01981724542418 -119.23667261944364) bank9963 +9964 POINT(33.44314771228365 -118.7475391513889) bank9964 +9965 POINT(34.53055000169869 -118.71799549299867) bank9965 +9966 POINT(33.47854233692869 -119.16036623929651) bank9966 +9967 POINT(34.57336468909619 -118.29964098343059) bank9967 +9968 POINT(34.98296465962125 -117.88929503367346) bank9968 +9969 POINT(34.31962256292241 -118.37051851896678) bank9969 +9970 POINT(33.44452538069103 -119.16344827286115) bank9970 +9971 POINT(33.46179780476845 -118.43808484126555) bank9971 +9972 POINT(35.02471350404543 -118.2477778640218) bank9972 +9973 POINT(33.773844994771856 -117.84218908701476) bank9973 +9974 POINT(33.32130577436502 -118.1220082621) bank9974 +9975 POINT(34.017602055399955 -119.22426145159791) bank9975 +9976 POINT(34.32954877332494 -117.3110287339001) bank9976 +9977 POINT(34.673152027009394 -118.3955863177114) bank9977 +9978 POINT(33.87015297571218 -117.78630179481017) bank9978 +9979 POINT(34.729370291109426 -117.98776923196684) bank9979 +9980 POINT(33.156934075995146 -118.87356334343076) bank9980 +9981 POINT(34.439755835737465 -117.62723628228997) bank9981 +9982 POINT(34.574345666624616 -118.18017970103948) bank9982 +9983 POINT(33.92434277547046 -117.35107137900232) bank9983 +9984 POINT(33.320261359541064 -118.39120583268944) bank9984 +9985 POINT(33.19097156269247 -118.47528839946104) bank9985 +9986 POINT(33.29380786966338 -118.42458398763476) bank9986 +9987 POINT(34.534949106995924 -118.20822599449087) bank9987 +9988 POINT(33.225652986100954 -119.16139851130617) bank9988 +9989 POINT(33.636921745373584 -117.28326633413448) bank9989 +9990 POINT(33.61706042437872 -117.73403347569801) bank9990 +9991 POINT(34.47708461329366 -118.34746005806426) bank9991 +9992 POINT(33.678385459015075 -119.17861890926483) bank9992 +9993 POINT(35.00939137973683 -117.51843016840857) bank9993 +9994 POINT(34.755186341077916 -119.24280352563795) bank9994 +9995 POINT(34.5662039185285 -117.96333226945676) bank9995 +9996 POINT(33.45350814176539 -118.00952993412629) bank9996 +9997 POINT(33.07620770540016 -117.48775312996469) bank9997 +9998 POINT(34.963257834801034 -117.30111306961923) bank9998 +9999 POINT(33.37997374743659 -117.45401429398456) bank9999 +10000 POINT(34.66921763517918 -118.66512247548435) bank10000 +10001 POINT(33.279906183516374 -118.92948544771576) bank10001 +10002 POINT(35.02574763349247 -117.98351356207155) bank10002 +10003 POINT(34.666497784788156 -119.08177653619822) bank10003 +10004 POINT(33.974944299739164 -118.0034978739387) bank10004 +10005 POINT(33.285979678456094 -117.70429532299893) bank10005 +10006 POINT(34.09193307013384 -117.25462117714366) bank10006 +10007 POINT(34.302029813076466 -118.61428464230538) bank10007 +10008 POINT(35.035387131620965 -118.93042442122709) bank10008 +10009 POINT(34.97951073752712 -118.0216316276789) bank10009 +10010 POINT(33.642289825189906 -118.50238503189524) bank10010 +10011 POINT(33.64850408078041 -118.64349453499796) bank10011 +10012 POINT(34.58941659827475 -119.044507166325) bank10012 +10013 POINT(34.743064321601246 -118.25956369484459) bank10013 +10014 POINT(33.14915041524906 -117.40652747635284) bank10014 +10015 POINT(33.20821789064497 -119.13941707761167) bank10015 +10016 POINT(33.728301023456666 -117.84923173888262) bank10016 +10017 POINT(34.145862737833 -118.56357092906919) bank10017 +10018 POINT(34.67727822765015 -118.15836075658824) bank10018 +10019 POINT(34.4051581226298 -118.00944005526578) bank10019 +10020 POINT(34.34553592266703 -117.80387504460167) bank10020 +10021 POINT(34.573341951392 -117.44230681763585) bank10021 +10022 POINT(34.7728010976118 -119.09806624245084) bank10022 +10023 POINT(33.292128540318316 -118.14584304796578) bank10023 +10024 POINT(33.14116694055709 -117.30076346713763) bank10024 +10025 POINT(33.63057532070344 -118.77989115060596) bank10025 +10026 POINT(34.04442580997893 -118.32416175417698) bank10026 +10027 POINT(34.718332259263896 -117.32677105340326) bank10027 +10028 POINT(34.22490977934571 -117.2555560945563) bank10028 +10029 POINT(34.01447437458465 -118.07955763682146) bank10029 +10030 POINT(33.778427977527095 -118.63853344132899) bank10030 +10031 POINT(33.49748617460814 -117.98997847795904) bank10031 +10032 POINT(34.6102440826491 -117.52776178049723) bank10032 +10033 POINT(34.837125378554596 -119.17880139857087) bank10033 +10034 POINT(34.937317873214084 -117.4921155221206) bank10034 +10035 POINT(33.785650344486605 -118.64715696963115) bank10035 +10036 POINT(34.1352206633587 -118.40790512405631) bank10036 +10037 POINT(34.550706835236284 -117.38337514324391) bank10037 +10038 POINT(33.67036329599689 -118.48170054109187) bank10038 +10039 POINT(33.425402093801544 -117.5462048130568) bank10039 +10040 POINT(33.508833647265824 -117.29697427039655) bank10040 +10041 POINT(33.86713392298512 -118.29473296507263) bank10041 +10042 POINT(34.008673403908034 -118.56628214212328) bank10042 +10043 POINT(34.54314268047607 -117.64176663276119) bank10043 +10044 POINT(33.45174269382653 -117.43592174480098) bank10044 +10045 POINT(33.16363378326113 -118.56944114078733) bank10045 +10046 POINT(34.83951999668113 -117.54576648346489) bank10046 +10047 POINT(34.45375709686666 -117.78430171670843) bank10047 +10048 POINT(34.77510053156348 -119.09646210633736) bank10048 +10049 POINT(34.185886626750595 -119.04033697140979) bank10049 +10050 POINT(33.88604580151556 -119.20298194279414) bank10050 +10051 POINT(33.563667461531125 -117.94713647103383) bank10051 +10052 POINT(34.84995730111487 -118.80279953969416) bank10052 +10053 POINT(33.62815840260604 -118.30263979699545) bank10053 +10054 POINT(33.745605824021105 -118.13016865852968) bank10054 +10055 POINT(33.35842424466268 -117.88422814779346) bank10055 +10056 POINT(33.06651988356881 -118.35893055872458) bank10056 +10057 POINT(34.94099559509477 -118.52270742705392) bank10057 +10058 POINT(34.12959716781447 -119.09178851243452) bank10058 +10059 POINT(34.95500222669898 -117.60389309337127) bank10059 +10060 POINT(34.00618218112846 -117.32389414099362) bank10060 +10061 POINT(34.896093668421294 -119.18791314459611) bank10061 +10062 POINT(33.15103076900422 -118.02051144959493) bank10062 +10063 POINT(34.03109757090102 -118.41581887009276) bank10063 +10064 POINT(34.79583234788388 -117.25567879294405) bank10064 +10065 POINT(33.16192618277677 -118.86274691548222) bank10065 +10066 POINT(33.28483119175079 -117.9150413751256) bank10066 +10067 POINT(34.90663408832984 -117.87210055969209) bank10067 +10068 POINT(34.742261645301646 -117.77965639901842) bank10068 +10069 POINT(33.90924831734465 -117.51466825256286) bank10069 +10070 POINT(33.493889626506316 -117.73781248015666) bank10070 +10071 POINT(33.10750434996338 -119.11912226717138) bank10071 +10072 POINT(34.01600694676756 -117.29827705204428) bank10072 +10073 POINT(34.979466178403186 -117.27456337544587) bank10073 +10074 POINT(33.69305412680575 -118.68609096971691) bank10074 +10075 POINT(33.76449179762343 -118.13726368374988) bank10075 +10076 POINT(34.8924296647715 -118.77725856855359) bank10076 +10077 POINT(34.1688545740474 -118.42065648728634) bank10077 +10078 POINT(35.03112066717429 -118.05373330492014) bank10078 +10079 POINT(34.02964626064085 -117.56865158800706) bank10079 +10080 POINT(34.77475780907134 -117.74356308592095) bank10080 +10081 POINT(33.91637121452654 -118.07691892644378) bank10081 +10082 POINT(33.9227723464695 -117.40485518483462) bank10082 +10083 POINT(33.244198084338706 -118.51828259534084) bank10083 +10084 POINT(33.88329725975413 -118.14623380227815) bank10084 +10085 POINT(34.45872737969872 -118.04190171687713) bank10085 +10086 POINT(34.40946259886331 -117.88695150403792) bank10086 +10087 POINT(34.4603497839815 -119.13455461242778) bank10087 +10088 POINT(34.06019151659552 -118.97552268145732) bank10088 +10089 POINT(35.04190665907678 -117.49570554547455) bank10089 +10090 POINT(33.10808525115619 -118.93974903092594) bank10090 +10091 POINT(34.55896223399781 -117.84874119606243) bank10091 +10092 POINT(33.54147396399513 -117.59019230802951) bank10092 +10093 POINT(33.658643081774756 -118.41556963751734) bank10093 +10094 POINT(33.92452618200963 -118.44500686748306) bank10094 +10095 POINT(33.15043920016676 -118.45379644765303) bank10095 +10096 POINT(34.015611228172254 -119.10183841660364) bank10096 +10097 POINT(35.05120993200782 -117.51600091007448) bank10097 +10098 POINT(34.72682465694948 -118.74266107115996) bank10098 +10099 POINT(34.73901891535622 -117.92016224802803) bank10099 +10100 POINT(34.129431567901605 -118.73513415184912) bank10100 +10101 POINT(33.66456170450851 -118.01725973434947) bank10101 +10102 POINT(35.012282736852804 -117.67747804490617) bank10102 +10103 POINT(34.32718272613074 -117.84303314180868) bank10103 +10104 POINT(33.06309583213061 -119.14671458065494) bank10104 +10105 POINT(34.49232379804497 -117.44683328038016) bank10105 +10106 POINT(33.27434688943487 -117.51116776506619) bank10106 +10107 POINT(33.27099316225093 -117.61577191934366) bank10107 +10108 POINT(34.768971680591875 -117.41140095253608) bank10108 +10109 POINT(34.52143720053547 -118.66329649215331) bank10109 +10110 POINT(33.77518500954713 -118.01474245765269) bank10110 +10111 POINT(34.83479605662373 -118.67313057311854) bank10111 +10112 POINT(34.3705566104832 -117.79526828257065) bank10112 +10113 POINT(33.394942855374374 -119.0292824358075) bank10113 +10114 POINT(33.2193808535361 -118.46706633926061) bank10114 +10115 POINT(33.411313664210866 -118.81642228767117) bank10115 +10116 POINT(33.37899430845746 -117.92032610469136) bank10116 +10117 POINT(34.92339093316061 -118.96934841301105) bank10117 +10118 POINT(33.741775553697906 -117.87515088552969) bank10118 +10119 POINT(34.98310506476053 -117.47710945196935) bank10119 +10120 POINT(34.137218029316074 -118.47443735145379) bank10120 +10121 POINT(34.977930844969876 -118.22132879085763) bank10121 +10122 POINT(33.2545976207132 -118.78335415538615) bank10122 +10123 POINT(33.51432482426642 -118.20569448340585) bank10123 +10124 POINT(34.775437234398424 -118.21768474180935) bank10124 +10125 POINT(34.2351039324314 -118.80445327601373) bank10125 +10126 POINT(33.318478596212536 -118.64881179998558) bank10126 +10127 POINT(34.10157663574241 -117.83591864518777) bank10127 +10128 POINT(34.933807279978275 -118.57527389121292) bank10128 +10129 POINT(34.44889522859033 -118.35630523285624) bank10129 +10130 POINT(34.781297537511 -117.92829989551409) bank10130 +10131 POINT(33.21116381555237 -117.41323072583141) bank10131 +10132 POINT(34.99706579586043 -117.77639481466832) bank10132 +10133 POINT(33.485634185822555 -117.82296768907351) bank10133 +10134 POINT(35.04103908752386 -118.12302852541502) bank10134 +10135 POINT(33.53269011604233 -118.78962862679602) bank10135 +10136 POINT(34.07974760957684 -119.22142903596163) bank10136 +10137 POINT(33.95153285612237 -118.88714187276052) bank10137 +10138 POINT(34.97837816668199 -118.55238000591632) bank10138 +10139 POINT(34.64489706345233 -117.70653351161401) bank10139 +10140 POINT(34.03071418195546 -118.13271341448656) bank10140 +10141 POINT(34.2018205830827 -117.58751872536787) bank10141 +10142 POINT(34.5913783981369 -117.80204742356554) bank10142 +10143 POINT(33.2043152121441 -118.7746507078685) bank10143 +10144 POINT(33.803495813395244 -118.09593926396218) bank10144 +10145 POINT(33.82493265302295 -119.12927097624606) bank10145 +10146 POINT(34.09431568443502 -117.63513839887057) bank10146 +10147 POINT(34.3785537789759 -118.79175199498378) bank10147 +10148 POINT(33.894523900882795 -118.48115060375711) bank10148 +10149 POINT(34.33602203390122 -117.77083819669966) bank10149 +10150 POINT(33.840887397483144 -119.19484329334263) bank10150 +10151 POINT(34.463057102442846 -117.38406337602122) bank10151 +10152 POINT(33.80769694053792 -117.43279354963379) bank10152 +10153 POINT(33.80527096816038 -119.0390001516295) bank10153 +10154 POINT(34.45685492263627 -118.62884740872022) bank10154 +10155 POINT(33.254361368819346 -118.16792299669729) bank10155 +10156 POINT(33.29725462187311 -117.3521019424099) bank10156 +10157 POINT(33.60879593316809 -118.96443205893694) bank10157 +10158 POINT(33.63253009683941 -118.96255807632765) bank10158 +10159 POINT(33.09974372243936 -117.29829836068485) bank10159 +10160 POINT(34.333945415998016 -118.6897737882176) bank10160 +10161 POINT(34.90307530982244 -119.2093558096031) bank10161 +10162 POINT(34.026082123429994 -118.8885482068717) bank10162 +10163 POINT(33.75507966485809 -118.26462047395644) bank10163 +10164 POINT(33.602700834084985 -118.72512862059429) bank10164 +10165 POINT(33.571649364742456 -118.06590607645714) bank10165 +10166 POINT(33.10051054525163 -119.17370936298038) bank10166 +10167 POINT(34.66794610734646 -118.7339601973768) bank10167 +10168 POINT(33.659285037556344 -117.94660973892324) bank10168 +10169 POINT(34.99206580488169 -117.49903152644596) bank10169 +10170 POINT(34.720074095994654 -118.73450872782587) bank10170 +10171 POINT(34.45571173320897 -118.39710043809127) bank10171 +10172 POINT(33.64715264944699 -117.50888102552048) bank10172 +10173 POINT(33.752653573911566 -118.431119469503) bank10173 +10174 POINT(33.67225177310759 -118.27593285536183) bank10174 +10175 POINT(33.294795098708555 -118.07634571438864) bank10175 +10176 POINT(33.28698195876892 -117.49155260599761) bank10176 +10177 POINT(33.287490266594354 -117.45731146117362) bank10177 +10178 POINT(34.01180117163647 -118.90854560011343) bank10178 +10179 POINT(34.389561218709424 -118.45871617538906) bank10179 +10180 POINT(34.636555795578076 -117.89170442571334) bank10180 +10181 POINT(33.42556740612105 -118.53564180761899) bank10181 +10182 POINT(33.86974389116139 -118.60625695856724) bank10182 +10183 POINT(34.21344715745816 -117.88116989819244) bank10183 +10184 POINT(33.79287037561683 -119.01982279831655) bank10184 +10185 POINT(33.94120596290945 -119.19532757387893) bank10185 +10186 POINT(34.26278714849238 -119.03108607818828) bank10186 +10187 POINT(33.12998641867408 -119.23996596158602) bank10187 +10188 POINT(34.268039484291265 -117.6607853677487) bank10188 +10189 POINT(33.421816959926865 -117.78931330341513) bank10189 +10190 POINT(34.39509592676895 -118.12933136099757) bank10190 +10191 POINT(33.340790933053405 -117.63416738285862) bank10191 +10192 POINT(34.02019952773786 -118.36342629603186) bank10192 +10193 POINT(34.15189989285233 -118.99667509890735) bank10193 +10194 POINT(34.16716929110311 -117.56464319826642) bank10194 +10195 POINT(34.229121183929216 -118.77138897484656) bank10195 +10196 POINT(34.024419075135704 -118.00171518056649) bank10196 +10197 POINT(33.93801161854019 -118.06180880957051) bank10197 +10198 POINT(33.25428976573557 -118.47934236668299) bank10198 +10199 POINT(33.308430397705145 -119.19493450822708) bank10199 +10200 POINT(34.959316761125834 -118.12295000746701) bank10200 +10201 POINT(34.11990132184696 -119.19108267477966) bank10201 +10202 POINT(33.17551632533678 -119.01794040349851) bank10202 +10203 POINT(33.926821691359216 -118.01833333023133) bank10203 +10204 POINT(34.65404456582919 -119.16455591714588) bank10204 +10205 POINT(33.54332267015039 -117.53977935137637) bank10205 +10206 POINT(33.65220549037828 -118.3279174112098) bank10206 +10207 POINT(33.960197318882514 -119.23527233221868) bank10207 +10208 POINT(33.30543411245752 -119.19036756546072) bank10208 +10209 POINT(34.56539928620324 -117.37790485957728) bank10209 +10210 POINT(33.33322897219526 -118.18085317050163) bank10210 +10211 POINT(34.50272827690153 -117.94327290290751) bank10211 +10212 POINT(34.614864290823654 -119.06263816533477) bank10212 +10213 POINT(34.899200272008436 -117.48373843526869) bank10213 +10214 POINT(33.78280452517105 -117.60532161686987) bank10214 +10215 POINT(33.07589141532623 -119.0962514154157) bank10215 +10216 POINT(34.25316740788054 -118.17040051285485) bank10216 +10217 POINT(33.49363702429421 -118.61874117226698) bank10217 +10218 POINT(34.945351613881435 -118.45638768601171) bank10218 +10219 POINT(34.918643976236126 -118.07782478671885) bank10219 +10220 POINT(33.200804606752605 -117.74611031783866) bank10220 +10221 POINT(34.84780279540832 -117.87666930650452) bank10221 +10222 POINT(33.97498751445459 -117.9861316766282) bank10222 +10223 POINT(34.64758181998539 -118.53004075012817) bank10223 +10224 POINT(34.54431918143249 -119.02889586939169) bank10224 +10225 POINT(35.007412170984516 -117.56901424346451) bank10225 +10226 POINT(34.41716600834813 -118.59164026552267) bank10226 +10227 POINT(33.09299508212885 -118.42138896979249) bank10227 +10228 POINT(34.29624232168269 -117.4508164144607) bank10228 +10229 POINT(34.848398265374655 -119.10652889749113) bank10229 +10230 POINT(33.986365349015394 -119.1346419993251) bank10230 +10231 POINT(33.951134818896435 -117.40105884228609) bank10231 +10232 POINT(33.45616098706767 -118.69440727763478) bank10232 +10233 POINT(33.364784590249 -117.59942960163642) bank10233 +10234 POINT(34.56626807245165 -117.71363001202198) bank10234 +10235 POINT(34.49731216344445 -118.9367406902133) bank10235 +10236 POINT(34.42307286227908 -118.88049365903598) bank10236 +10237 POINT(33.39156182864646 -118.97193174919305) bank10237 +10238 POINT(34.10087857383006 -118.41787336494804) bank10238 +10239 POINT(33.071662989069516 -117.95553756203907) bank10239 +10240 POINT(33.21700773377869 -117.47009923081455) bank10240 +10241 POINT(34.506314417244845 -117.3281699155067) bank10241 +10242 POINT(34.9536750201936 -118.71503103896228) bank10242 +10243 POINT(34.159197917900805 -119.09773942578808) bank10243 +10244 POINT(33.82664336837341 -117.34010805727961) bank10244 +10245 POINT(34.40793710208078 -117.31108742135757) bank10245 +10246 POINT(33.65636798051789 -118.06891303934744) bank10246 +10247 POINT(34.681462378162514 -117.42143474172556) bank10247 +10248 POINT(34.639529693157584 -118.76193717773062) bank10248 +10249 POINT(33.61421717262702 -118.36952378207498) bank10249 +10250 POINT(34.877180550895524 -117.64880530150357) bank10250 +10251 POINT(33.52544864746982 -117.3657321732109) bank10251 +10252 POINT(34.2691413528105 -117.88367114140257) bank10252 +10253 POINT(33.0897234385275 -118.03314593977908) bank10253 +10254 POINT(33.24954361865618 -118.51535047872171) bank10254 +10255 POINT(33.363563209060764 -117.49629902071457) bank10255 +10256 POINT(33.17126713759044 -117.3971147696818) bank10256 +10257 POINT(33.748718056856106 -117.42160364580648) bank10257 +10258 POINT(34.50169064179197 -118.95658695501798) bank10258 +10259 POINT(34.01304775980795 -117.53295951703079) bank10259 +10260 POINT(34.62103403619978 -118.8184337141484) bank10260 +10261 POINT(34.50593203481931 -118.58607711962814) bank10261 +10262 POINT(33.42304338054336 -118.43467990177078) bank10262 +10263 POINT(34.057245893678804 -118.5424164956028) bank10263 +10264 POINT(34.34165919585149 -119.16771653057154) bank10264 +10265 POINT(33.446348008555134 -118.86768692398577) bank10265 +10266 POINT(33.05226175663349 -118.56784114940707) bank10266 +10267 POINT(33.29889048056419 -118.02310007115037) bank10267 +10268 POINT(33.41427915329915 -119.16091682456192) bank10268 +10269 POINT(34.343644047576255 -117.9825612179657) bank10269 +10270 POINT(34.46758807741821 -117.87625080855042) bank10270 +10271 POINT(34.37435607717984 -119.09083854675802) bank10271 +10272 POINT(34.542178739836835 -117.48367429693575) bank10272 +10273 POINT(34.262163182135765 -117.36036506704947) bank10273 +10274 POINT(33.867556967195725 -118.32167436521425) bank10274 +10275 POINT(34.563003439637306 -117.51857496038308) bank10275 +10276 POINT(33.998643562107816 -118.19586463418239) bank10276 +10277 POINT(34.31888698503403 -119.10008164447272) bank10277 +10278 POINT(34.1251626316157 -117.55093259834464) bank10278 +10279 POINT(34.616746296403555 -118.45777015980909) bank10279 +10280 POINT(34.611336576950414 -118.89088464985265) bank10280 +10281 POINT(34.92701828949582 -118.05195384307144) bank10281 +10282 POINT(33.140025425428135 -118.64918365948645) bank10282 +10283 POINT(34.759590061361436 -118.7240285844346) bank10283 +10284 POINT(33.15987019372848 -118.2207848010975) bank10284 +10285 POINT(34.61875996683027 -117.38254727416106) bank10285 +10286 POINT(33.99565909578512 -118.07163053542124) bank10286 +10287 POINT(34.044657501927105 -117.50014698574414) bank10287 +10288 POINT(33.56573227374084 -118.63197351451248) bank10288 +10289 POINT(34.722889052048835 -118.62089157463879) bank10289 +10290 POINT(34.94929602609483 -117.92464019410214) bank10290 +10291 POINT(34.609645651781165 -117.4838748092452) bank10291 +10292 POINT(34.18805114519273 -117.64423696199937) bank10292 +10293 POINT(33.22786213627785 -118.06512365520501) bank10293 +10294 POINT(34.02330271628248 -118.75839896121848) bank10294 +10295 POINT(34.21867293062928 -119.17088651214819) bank10295 +10296 POINT(33.59928245631295 -117.88635783259978) bank10296 +10297 POINT(34.905315661592 -119.20566397153746) bank10297 +10298 POINT(34.99907032809747 -117.42015570524197) bank10298 +10299 POINT(34.2587124100172 -118.50263219416776) bank10299 +10300 POINT(34.24821361681654 -118.35457228929566) bank10300 +10301 POINT(34.70788216844613 -118.91946231579806) bank10301 +10302 POINT(33.30460969511543 -118.47469834223696) bank10302 +10303 POINT(33.21961546411229 -118.0759856074027) bank10303 +10304 POINT(34.39431141793887 -118.64735575341497) bank10304 +10305 POINT(33.61451470694513 -117.2581011655275) bank10305 +10306 POINT(34.651541100719214 -117.74632445799347) bank10306 +10307 POINT(33.675639015317564 -117.63874936741172) bank10307 +10308 POINT(33.86827624859331 -118.47673208362707) bank10308 +10309 POINT(34.63471099195048 -118.22426832909794) bank10309 +10310 POINT(33.257661032894866 -118.11960963970235) bank10310 +10311 POINT(34.520283000295656 -118.97576237699296) bank10311 +10312 POINT(33.18596635001735 -118.35373971167115) bank10312 +10313 POINT(34.022634348500205 -117.542674050047) bank10313 +10314 POINT(33.064664769024034 -118.69323140048165) bank10314 +10315 POINT(34.4668266619991 -117.6747105935692) bank10315 +10316 POINT(34.66869314847089 -119.00679725712257) bank10316 +10317 POINT(34.250962034937146 -118.50321525941241) bank10317 +10318 POINT(33.48695874302 -117.28029151161233) bank10318 +10319 POINT(34.806477546797844 -117.57066224192319) bank10319 +10320 POINT(33.752366587031865 -117.69681748925704) bank10320 +10321 POINT(33.740998502537124 -118.85972164070962) bank10321 +10322 POINT(33.07743256088141 -118.57105875331814) bank10322 +10323 POINT(34.50107619637948 -118.24624913455995) bank10323 +10324 POINT(33.65996507075793 -118.00139383390439) bank10324 +10325 POINT(33.67318813303921 -117.38496386281288) bank10325 +10326 POINT(33.55986941867484 -118.91934412683945) bank10326 +10327 POINT(33.68347318010126 -119.20879091012934) bank10327 +10328 POINT(33.95652390263616 -117.85768598233145) bank10328 +10329 POINT(35.04148774797667 -118.80043031629205) bank10329 +10330 POINT(34.39530078185798 -117.94670730931925) bank10330 +10331 POINT(33.21642199378078 -117.71100797121126) bank10331 +10332 POINT(34.871942361713295 -119.09469221183808) bank10332 +10333 POINT(34.6990118188214 -118.96426474280973) bank10333 +10334 POINT(33.892630478815036 -118.68655300552315) bank10334 +10335 POINT(34.04708278500177 -117.94491498937518) bank10335 +10336 POINT(33.3484192927034 -118.08742858505958) bank10336 +10337 POINT(34.315745337532874 -118.55921429132194) bank10337 +10338 POINT(33.532009307061855 -117.97560956901383) bank10338 +10339 POINT(34.18345289171715 -118.26848724951992) bank10339 +10340 POINT(33.571743380831066 -117.61743230781138) bank10340 +10341 POINT(34.20918785501342 -117.53703286607998) bank10341 +10342 POINT(34.069533343336545 -118.48398377575509) bank10342 +10343 POINT(33.52403135114942 -118.17105364211254) bank10343 +10344 POINT(35.03232061112631 -118.45653404897949) bank10344 +10345 POINT(33.49147419001463 -118.48153646437103) bank10345 +10346 POINT(33.94083690858194 -118.30656959979635) bank10346 +10347 POINT(33.11885701389423 -117.29954134144667) bank10347 +10348 POINT(34.47732040843662 -118.58754891590553) bank10348 +10349 POINT(35.00577796778231 -118.65006880996584) bank10349 +10350 POINT(34.418452911309174 -118.23517467420017) bank10350 +10351 POINT(33.9384208746463 -117.31474576365093) bank10351 +10352 POINT(33.36044957758719 -117.37773295679582) bank10352 +10353 POINT(33.59580347183079 -117.50188798672717) bank10353 +10354 POINT(35.01853586139556 -117.41467812474754) bank10354 +10355 POINT(33.36136565927637 -118.54721633146103) bank10355 +10356 POINT(33.304816833673186 -117.99776642312425) bank10356 +10357 POINT(33.24634807192121 -118.88295728008045) bank10357 +10358 POINT(33.6534329984592 -119.12209002271008) bank10358 +10359 POINT(34.28454435057414 -118.10876398598454) bank10359 +10360 POINT(34.832346182353334 -118.62765155401989) bank10360 +10361 POINT(34.769523649074635 -118.67760903489469) bank10361 +10362 POINT(34.157792174261836 -118.42431721415036) bank10362 +10363 POINT(33.37241819140889 -118.87942174528972) bank10363 +10364 POINT(33.06124078095539 -118.42075892779188) bank10364 +10365 POINT(34.80992273009314 -117.77294202001048) bank10365 +10366 POINT(34.070882983657896 -118.54331798707352) bank10366 +10367 POINT(34.39630772386169 -118.28152367240962) bank10367 +10368 POINT(34.2034221833693 -117.73932500862337) bank10368 +10369 POINT(34.16766597287147 -117.79361221012853) bank10369 +10370 POINT(34.822731856833 -118.03570264881179) bank10370 +10371 POINT(33.39469312086401 -118.2517693148389) bank10371 +10372 POINT(33.728660704570586 -117.82208830111966) bank10372 +10373 POINT(33.52537937069442 -117.94129915910405) bank10373 +10374 POINT(33.75204088301202 -118.6260268155405) bank10374 +10375 POINT(34.6790084464864 -118.75865069136995) bank10375 +10376 POINT(34.533685491468034 -118.67835841846163) bank10376 +10377 POINT(34.452743216488834 -117.6830984356125) bank10377 +10378 POINT(34.44353437831255 -117.97998897549996) bank10378 +10379 POINT(34.808988709141786 -117.78019629609088) bank10379 +10380 POINT(33.68956563888541 -118.59276169867483) bank10380 +10381 POINT(33.82193258459625 -118.27873916719943) bank10381 +10382 POINT(34.123989172326084 -117.42094304982628) bank10382 +10383 POINT(33.279398653042435 -119.23985383987748) bank10383 +10384 POINT(33.73603140110709 -117.58085454152804) bank10384 +10385 POINT(34.10424714003473 -118.41577320908851) bank10385 +10386 POINT(33.465310496507776 -118.89399977806471) bank10386 +10387 POINT(34.62939309386314 -117.85835894030346) bank10387 +10388 POINT(34.995661028708085 -117.26463660318326) bank10388 +10389 POINT(33.32171783637408 -119.05057398035969) bank10389 +10390 POINT(33.79385850514137 -117.76298743185184) bank10390 +10391 POINT(34.891772320796925 -118.91303253059708) bank10391 +10392 POINT(34.257395097380375 -117.29005299972191) bank10392 +10393 POINT(34.14416370428353 -119.18675297451016) bank10393 +10394 POINT(33.42128319390363 -118.8509618365238) bank10394 +10395 POINT(33.28893426001308 -117.74625635773533) bank10395 +10396 POINT(34.027746340924004 -118.16187451795122) bank10396 +10397 POINT(34.861131046488374 -118.56677116293965) bank10397 +10398 POINT(33.979643596383376 -117.49264529717149) bank10398 +10399 POINT(34.04227044496531 -117.82226519081725) bank10399 +10400 POINT(34.44363947009643 -119.10070446353913) bank10400 +10401 POINT(35.03204300917094 -118.43071443142486) bank10401 +10402 POINT(34.134210657168396 -119.14751025845918) bank10402 +10403 POINT(33.40984048593405 -118.6559597030123) bank10403 +10404 POINT(33.2795513154948 -117.66833450197223) bank10404 +10405 POINT(34.59807975911331 -119.22166042412316) bank10405 +10406 POINT(34.49869580068682 -118.64198155021263) bank10406 +10407 POINT(34.099272625461715 -118.41008704811948) bank10407 +10408 POINT(33.650494416089884 -117.40241886203734) bank10408 +10409 POINT(34.437317763486035 -117.5533194510084) bank10409 +10410 POINT(33.44494084998684 -118.53308646182832) bank10410 +10411 POINT(34.78547972537509 -118.2624597970547) bank10411 +10412 POINT(33.8788521430394 -117.35225574319325) bank10412 +10413 POINT(34.21045714841069 -118.75114714241514) bank10413 +10414 POINT(35.033527562461586 -117.7085892905243) bank10414 +10415 POINT(34.668081946093245 -119.02692048580822) bank10415 +10416 POINT(33.43127257512082 -118.2290649439119) bank10416 +10417 POINT(35.014740972471365 -118.48993306073507) bank10417 +10418 POINT(33.25607256632559 -118.15830205728835) bank10418 +10419 POINT(35.03187349900214 -117.91775207007655) bank10419 +10420 POINT(33.23134909528463 -118.81947075611974) bank10420 +10421 POINT(33.12266689747075 -117.88736924233314) bank10421 +10422 POINT(34.9809610465665 -117.5366861324037) bank10422 +10423 POINT(34.28776999046064 -117.49761888042298) bank10423 +10424 POINT(33.72121695449103 -118.29759732033378) bank10424 +10425 POINT(34.19199045282571 -118.74661927321047) bank10425 +10426 POINT(34.405735671008294 -117.73210245651802) bank10426 +10427 POINT(34.684657968745896 -118.03199644226518) bank10427 +10428 POINT(33.44592521889811 -117.58811423857387) bank10428 +10429 POINT(34.57070429575893 -117.47916744723527) bank10429 +10430 POINT(33.99584869604835 -117.90826609110965) bank10430 +10431 POINT(33.316942831780175 -118.05438488765145) bank10431 +10432 POINT(34.44566867234848 -118.53550815499877) bank10432 +10433 POINT(33.16612025616622 -117.79652099411726) bank10433 +10434 POINT(34.54865355507879 -117.92247604085586) bank10434 +10435 POINT(33.141940806236924 -117.60451223809886) bank10435 +10436 POINT(33.91085052122297 -118.9418739783232) bank10436 +10437 POINT(34.5541927616006 -119.2238731810371) bank10437 +10438 POINT(33.767472294995 -117.44075174352339) bank10438 +10439 POINT(34.17619525748174 -119.15200419907322) bank10439 +10440 POINT(33.05854937401204 -118.90536884189513) bank10440 +10441 POINT(33.533590142975584 -118.13335455930181) bank10441 +10442 POINT(34.20249904430194 -117.45400817574846) bank10442 +10443 POINT(33.103850500107775 -117.43578676598709) bank10443 +10444 POINT(33.18400862349435 -118.9123672069864) bank10444 +10445 POINT(34.36804611368388 -118.06857114601662) bank10445 +10446 POINT(34.05134330283593 -118.3047737480666) bank10446 +10447 POINT(33.17777323184986 -118.04258299405956) bank10447 +10448 POINT(34.26234773653708 -117.53258242221969) bank10448 +10449 POINT(35.03654330176968 -117.32113194337097) bank10449 +10450 POINT(33.40167419911361 -119.13286234217273) bank10450 +10451 POINT(33.14426639168733 -119.0534355042665) bank10451 +10452 POINT(35.0288132529168 -117.31282094863843) bank10452 +10453 POINT(34.0401502096913 -117.59093676536936) bank10453 +10454 POINT(34.41657253842597 -118.61677147127483) bank10454 +10455 POINT(33.07913221444408 -118.70254330169982) bank10455 +10456 POINT(34.1541058161977 -117.4109973792784) bank10456 +10457 POINT(33.87672888162556 -117.46516440854121) bank10457 +10458 POINT(33.401718306637974 -118.52254382746628) bank10458 +10459 POINT(34.53465191629769 -118.5635909161479) bank10459 +10460 POINT(33.31297712149868 -119.2132750359049) bank10460 +10461 POINT(34.85680458287366 -117.27749949502977) bank10461 +10462 POINT(33.395176169758976 -117.91089105563597) bank10462 +10463 POINT(34.66364304763282 -118.46303689193776) bank10463 +10464 POINT(34.65924693706618 -117.45196282267472) bank10464 +10465 POINT(34.831980569994364 -118.39826373656982) bank10465 +10466 POINT(33.75523260749407 -118.77587965402486) bank10466 +10467 POINT(34.37512794189862 -119.15613763093276) bank10467 +10468 POINT(34.30269953286589 -118.89754372796575) bank10468 +10469 POINT(34.809150474019326 -117.3640133161842) bank10469 +10470 POINT(34.17151128386458 -118.07654508121625) bank10470 +10471 POINT(33.83064166146717 -117.88333165276651) bank10471 +10472 POINT(34.83075070133646 -118.53153903570092) bank10472 +10473 POINT(33.210859622163184 -117.91792376982605) bank10473 +10474 POINT(33.77044026824328 -118.95717575997412) bank10474 +10475 POINT(33.98142060409258 -117.27182418679423) bank10475 +10476 POINT(33.70720446569297 -117.90139165267219) bank10476 +10477 POINT(33.08117889066087 -119.01045778285277) bank10477 +10478 POINT(34.677377853248395 -118.66927506492378) bank10478 +10479 POINT(33.30676089956084 -118.21826110363892) bank10479 +10480 POINT(33.9421698173565 -117.64248050775822) bank10480 +10481 POINT(33.74946857741061 -118.41439334182232) bank10481 +10482 POINT(34.627267904355065 -117.46905561094897) bank10482 +10483 POINT(34.29678074977162 -117.88935700544859) bank10483 +10484 POINT(34.78838336432296 -117.34707241426847) bank10484 +10485 POINT(34.48060929293402 -117.593406550982) bank10485 +10486 POINT(33.498070519312414 -118.39351342714006) bank10486 +10487 POINT(34.81663213422422 -117.99495538612976) bank10487 +10488 POINT(34.78559408222916 -118.78842909516594) bank10488 +10489 POINT(34.6490976593859 -117.38044921416804) bank10489 +10490 POINT(33.25934647712368 -118.16045623938543) bank10490 +10491 POINT(33.32948811966404 -118.89276233313196) bank10491 +10492 POINT(33.90778915584 -117.88748081641063) bank10492 +10493 POINT(33.147530942981525 -118.82542779541063) bank10493 +10494 POINT(33.49266795704633 -118.2279904326633) bank10494 +10495 POINT(34.898547990071584 -117.355977375618) bank10495 +10496 POINT(33.217251500288974 -118.2501717284745) bank10496 +10497 POINT(34.88056084774318 -118.47507761772187) bank10497 +10498 POINT(33.554901001207845 -117.55692238773855) bank10498 +10499 POINT(33.87535861689905 -117.59983823676797) bank10499 +10500 POINT(33.181910888362495 -118.20461940789693) bank10500 +10501 POINT(34.187037166339785 -117.99236421745383) bank10501 +10502 POINT(34.803507522408694 -117.6636368509559) bank10502 +10503 POINT(33.65555888248831 -118.94070178169149) bank10503 +10504 POINT(33.13097960307914 -117.24704211320771) bank10504 +10505 POINT(33.08194670976126 -118.34270147639683) bank10505 +10506 POINT(34.31301379752954 -118.30616638972542) bank10506 +10507 POINT(34.489884038783714 -117.89870565407121) bank10507 +10508 POINT(34.20976303474105 -118.12669887757569) bank10508 +10509 POINT(34.7354322627316 -118.97197274919783) bank10509 +10510 POINT(33.30866810874143 -119.09634159763584) bank10510 +10511 POINT(33.21641310070956 -119.12578328226756) bank10511 +10512 POINT(33.509865485313405 -118.60376993409756) bank10512 +10513 POINT(33.520430371191445 -117.28664565015734) bank10513 +10514 POINT(34.801937545142785 -117.54943750763967) bank10514 +10515 POINT(33.504656525517035 -118.648480897436) bank10515 +10516 POINT(33.17079611075575 -118.29375705390012) bank10516 +10517 POINT(33.23699309431853 -117.52610752839074) bank10517 +10518 POINT(34.36855459888014 -118.63670534595765) bank10518 +10519 POINT(33.84879192536317 -119.11668443887916) bank10519 +10520 POINT(33.95725004688792 -117.56548253680805) bank10520 +10521 POINT(33.40867655165415 -117.50591771330056) bank10521 +10522 POINT(33.105428682939674 -118.11503789355183) bank10522 +10523 POINT(34.36607737979804 -118.48863924374147) bank10523 +10524 POINT(33.46450631549496 -118.7079638326907) bank10524 +10525 POINT(33.14057159233233 -118.39272003588056) bank10525 +10526 POINT(34.93978371162467 -117.53123400865712) bank10526 +10527 POINT(34.85381283912895 -118.75858957718832) bank10527 +10528 POINT(33.407081830621 -118.35737628885212) bank10528 +10529 POINT(33.48893839139389 -119.09861095806401) bank10529 +10530 POINT(34.759177142761075 -119.0169059788159) bank10530 +10531 POINT(34.115793374415766 -118.133721755949) bank10531 +10532 POINT(34.31901647176948 -119.09220155628526) bank10532 +10533 POINT(35.03182233249731 -117.92324023755847) bank10533 +10534 POINT(33.86794329349534 -118.25502709305438) bank10534 +10535 POINT(33.085688418130246 -118.30571422245951) bank10535 +10536 POINT(33.90734450422044 -119.02691719833072) bank10536 +10537 POINT(34.56815494888767 -117.96441313700815) bank10537 +10538 POINT(33.10926151240068 -118.40787210467894) bank10538 +10539 POINT(35.05087757719178 -117.37349719513472) bank10539 +10540 POINT(33.265599970081745 -118.53782182267406) bank10540 +10541 POINT(34.14958113421689 -117.54151273467032) bank10541 +10542 POINT(33.610233444991024 -118.38645677170074) bank10542 +10543 POINT(33.85133717244081 -118.21126207979327) bank10543 +10544 POINT(34.5181701722142 -117.96760681355441) bank10544 +10545 POINT(34.368154552487646 -117.55942642565223) bank10545 +10546 POINT(33.15476393058878 -118.15249781487181) bank10546 +10547 POINT(34.9296793027556 -118.55132138717607) bank10547 +10548 POINT(33.996605740952305 -118.31925532970514) bank10548 +10549 POINT(33.83281857018112 -118.30383134716743) bank10549 +10550 POINT(33.778147818834896 -118.80161200110562) bank10550 +10551 POINT(33.71130700220389 -118.96873639492796) bank10551 +10552 POINT(33.63052161238616 -119.15840469538449) bank10552 +10553 POINT(33.65217308919431 -118.95635570977953) bank10553 +10554 POINT(33.96514693886404 -118.50685553258057) bank10554 +10555 POINT(33.06799331098679 -117.87689205592797) bank10555 +10556 POINT(34.82082323512128 -117.66859888692116) bank10556 +10557 POINT(33.206320563732426 -118.26476138898711) bank10557 +10558 POINT(34.160941673883414 -117.85116648525043) bank10558 +10559 POINT(34.619718937118435 -118.97546484755254) bank10559 +10560 POINT(34.82474034119574 -117.7451737086774) bank10560 +10561 POINT(33.31671368800511 -118.46319669407603) bank10561 +10562 POINT(33.708263851391486 -118.28640126633361) bank10562 +10563 POINT(33.83683317469046 -117.39329600507259) bank10563 +10564 POINT(33.22541699835772 -117.44424500003872) bank10564 +10565 POINT(35.03024240707599 -118.3000586349649) bank10565 +10566 POINT(33.91214092926928 -118.52169690531001) bank10566 +10567 POINT(35.01308713019413 -117.99812661483563) bank10567 +10568 POINT(33.24735508738084 -118.30666635021412) bank10568 +10569 POINT(34.59979147845457 -118.58113399135745) bank10569 +10570 POINT(33.83106220244698 -118.16400295133157) bank10570 +10571 POINT(33.242150237234114 -117.60389575036037) bank10571 +10572 POINT(34.06232866324783 -118.67169525666677) bank10572 +10573 POINT(34.912295667371396 -118.63390208961691) bank10573 +10574 POINT(33.86324532283792 -119.06506684515985) bank10574 +10575 POINT(34.54758788172121 -118.96041697015055) bank10575 +10576 POINT(34.19581580501466 -118.08323691043677) bank10576 +10577 POINT(33.709146401215044 -117.3175766120852) bank10577 +10578 POINT(34.099036944566905 -118.22294856559603) bank10578 +10579 POINT(34.20492589999191 -117.33585739579928) bank10579 +10580 POINT(33.883734298481066 -117.98688781784374) bank10580 +10581 POINT(34.65217423529513 -118.74899042800921) bank10581 +10582 POINT(34.788118186215605 -117.464765597651) bank10582 +10583 POINT(33.90116421442807 -118.96907109144895) bank10583 +10584 POINT(33.14828817701026 -117.68966636276507) bank10584 +10585 POINT(33.92032237095622 -117.25696930725074) bank10585 +10586 POINT(33.95186996572238 -118.83032602557819) bank10586 +10587 POINT(34.2543393334053 -117.70062728289639) bank10587 +10588 POINT(33.76862628029815 -117.57445070411842) bank10588 +10589 POINT(33.99580559705516 -118.04335430202828) bank10589 +10590 POINT(34.78531399193261 -117.77577766052127) bank10590 +10591 POINT(34.23074538812499 -117.72772662857415) bank10591 +10592 POINT(33.88311928552034 -118.5038252206075) bank10592 +10593 POINT(33.77065899415531 -118.55065422642203) bank10593 +10594 POINT(34.81535428276993 -118.45706410845324) bank10594 +10595 POINT(34.92800725145428 -118.91572122660916) bank10595 +10596 POINT(34.710880699470835 -118.06309155228323) bank10596 +10597 POINT(33.342230749803626 -118.32213031274273) bank10597 +10598 POINT(34.057570849266206 -118.17578684282456) bank10598 +10599 POINT(33.56477563929648 -118.05674486328454) bank10599 +10600 POINT(34.65651809428116 -117.6919758171392) bank10600 +10601 POINT(34.2600413580242 -118.0307828349019) bank10601 +10602 POINT(34.07482325309197 -118.74456823890672) bank10602 +10603 POINT(33.98366025346479 -118.55610950801685) bank10603 +10604 POINT(33.593040850440595 -118.54892380657826) bank10604 +10605 POINT(33.73671567016777 -117.36350137869141) bank10605 +10606 POINT(34.85482056879259 -117.7798554339242) bank10606 +10607 POINT(33.1711212551555 -119.23239693580045) bank10607 +10608 POINT(33.51947659894151 -117.96840641848664) bank10608 +10609 POINT(33.88492037076629 -118.6446503800152) bank10609 +10610 POINT(34.52538790199454 -118.52943713095976) bank10610 +10611 POINT(35.032632898975606 -117.9509051661703) bank10611 +10612 POINT(34.267139369328305 -117.9554082566248) bank10612 +10613 POINT(34.21675759381778 -119.01230116629085) bank10613 +10614 POINT(34.10587183653692 -118.35972626322544) bank10614 +10615 POINT(33.96268778756883 -117.55641847079377) bank10615 +10616 POINT(34.994255237765714 -119.10288650923295) bank10616 +10617 POINT(34.54375775246463 -119.21490952392202) bank10617 +10618 POINT(33.50344115125199 -117.692508410055) bank10618 +10619 POINT(34.09400915582661 -119.14673842784705) bank10619 +10620 POINT(33.33171895439552 -118.68327504604432) bank10620 +10621 POINT(34.981588988240105 -118.04296476685276) bank10621 +10622 POINT(34.25140302926909 -118.00450690484608) bank10622 +10623 POINT(33.86069600944336 -118.34350236035256) bank10623 +10624 POINT(33.71730658739587 -117.31523885111051) bank10624 +10625 POINT(34.225265306917514 -117.39438419393824) bank10625 +10626 POINT(33.88019707123135 -117.59400818247022) bank10626 +10627 POINT(34.905859465634244 -117.76146137088716) bank10627 +10628 POINT(34.57297327315516 -117.4665877739382) bank10628 +10629 POINT(35.04595230462039 -117.44661870278948) bank10629 +10630 POINT(34.922786810247175 -118.60132739763978) bank10630 +10631 POINT(34.439325858506045 -119.01142315241394) bank10631 +10632 POINT(34.428527146503875 -119.13228414590797) bank10632 +10633 POINT(34.538906045379676 -118.31825964239275) bank10633 +10634 POINT(34.30211045184013 -119.18310030068324) bank10634 +10635 POINT(33.90546605487943 -117.57871172759894) bank10635 +10636 POINT(34.67638375285162 -119.18747477806629) bank10636 +10637 POINT(34.26913810074242 -118.27038805631418) bank10637 +10638 POINT(34.56632051318068 -117.76417375332495) bank10638 +10639 POINT(33.88242448873379 -118.28790408754192) bank10639 +10640 POINT(34.38738167131366 -117.38184505866329) bank10640 +10641 POINT(34.04958432012986 -118.78294088241184) bank10641 +10642 POINT(34.59425387624857 -118.25188448106027) bank10642 +10643 POINT(33.839898813966876 -118.84261356936662) bank10643 +10644 POINT(33.50677339527506 -118.85718664768777) bank10644 +10645 POINT(34.637434333206485 -118.28442711778509) bank10645 +10646 POINT(33.78438501961156 -117.9287397943594) bank10646 +10647 POINT(34.68142469809419 -117.99999181310157) bank10647 +10648 POINT(34.99373681258898 -119.19014497565979) bank10648 +10649 POINT(33.75341442885926 -117.95981441853185) bank10649 +10650 POINT(34.023312545135994 -119.24342206481438) bank10650 +10651 POINT(34.12912332347817 -118.30809830762708) bank10651 +10652 POINT(33.640710071333714 -117.90190479375185) bank10652 +10653 POINT(34.241967519910354 -117.38859810701736) bank10653 +10654 POINT(34.35286710204881 -117.35816251501927) bank10654 +10655 POINT(34.46792627555047 -117.87846044107042) bank10655 +10656 POINT(33.77022802072111 -117.93361829069903) bank10656 +10657 POINT(33.159857270209216 -117.92677133394527) bank10657 +10658 POINT(34.17327081227148 -118.54571572553279) bank10658 +10659 POINT(34.83552604590451 -118.1337918382828) bank10659 +10660 POINT(33.99201268653868 -117.58252583167891) bank10660 +10661 POINT(33.90175732373081 -117.42384695227476) bank10661 +10662 POINT(34.15904313679062 -118.45203027451345) bank10662 +10663 POINT(34.99064868018194 -119.00444549549891) bank10663 +10664 POINT(34.847753086131156 -118.0945423672664) bank10664 +10665 POINT(33.60284785514272 -117.34013798723109) bank10665 +10666 POINT(35.023759833194696 -118.300756872701) bank10666 +10667 POINT(34.8694693565407 -117.75683976456563) bank10667 +10668 POINT(34.08563708164398 -118.37787263642174) bank10668 +10669 POINT(33.75357067263008 -118.69059884493434) bank10669 +10670 POINT(33.17090355422047 -119.0796992535553) bank10670 +10671 POINT(33.43558127051773 -118.98082228288776) bank10671 +10672 POINT(35.01419670871665 -117.70757636622201) bank10672 +10673 POINT(34.25646536326447 -117.38258197453493) bank10673 +10674 POINT(33.55120499073355 -117.39839009532471) bank10674 +10675 POINT(34.26555699937772 -117.33558664702343) bank10675 +10676 POINT(34.63879013250424 -119.11928645168531) bank10676 +10677 POINT(34.42465848072655 -118.09593038172979) bank10677 +10678 POINT(35.009452862415806 -117.52642892198674) bank10678 +10679 POINT(34.042129826001556 -118.11075955234213) bank10679 +10680 POINT(34.491118450939204 -117.77990336730478) bank10680 +10681 POINT(34.82787204849053 -117.57810843593978) bank10681 +10682 POINT(33.34383520907828 -117.9670783465507) bank10682 +10683 POINT(33.539565858456584 -118.45697052622549) bank10683 +10684 POINT(33.78473473813211 -118.88423310339314) bank10684 +10685 POINT(33.877653453452865 -118.09131233868555) bank10685 +10686 POINT(33.14785090763052 -118.19482330387113) bank10686 +10687 POINT(34.40938378875369 -117.93989092544139) bank10687 +10688 POINT(33.35275203534525 -117.35185856017036) bank10688 +10689 POINT(33.618254405756126 -117.93612498451476) bank10689 +10690 POINT(34.7603312945789 -119.10577997967691) bank10690 +10691 POINT(33.37591526190361 -117.3057621812948) bank10691 +10692 POINT(34.01392835265857 -118.33558722281776) bank10692 +10693 POINT(33.53233216569467 -118.5356631107582) bank10693 +10694 POINT(34.94715579876114 -118.09648758016064) bank10694 +10695 POINT(33.19706321772371 -117.61142299480359) bank10695 +10696 POINT(34.213304391418994 -117.400835221501) bank10696 +10697 POINT(34.35061877205933 -118.27114565567098) bank10697 +10698 POINT(34.68793292971677 -119.14435177272853) bank10698 +10699 POINT(34.0730977971279 -118.1842964004642) bank10699 +10700 POINT(34.93400353735375 -117.58978935035108) bank10700 +10701 POINT(33.96833149488308 -119.21818946763001) bank10701 +10702 POINT(34.76932794509168 -118.82357102797033) bank10702 +10703 POINT(33.4809643962062 -117.73477030181817) bank10703 +10704 POINT(34.602460153400635 -117.38214042222754) bank10704 +10705 POINT(33.61719535422668 -119.23297269330752) bank10705 +10706 POINT(33.54652011008084 -117.74665030457847) bank10706 +10707 POINT(34.83104034760138 -117.76209663413286) bank10707 +10708 POINT(34.83008681971158 -117.61065195587427) bank10708 +10709 POINT(33.15636001920371 -117.58460707757737) bank10709 +10710 POINT(33.66734923999755 -117.81263162133564) bank10710 +10711 POINT(33.127220049446706 -117.67209444197606) bank10711 +10712 POINT(34.31101992791344 -119.16175096522839) bank10712 +10713 POINT(33.734661406595905 -117.71706280495802) bank10713 +10714 POINT(33.44632429053683 -117.98429043121372) bank10714 +10715 POINT(33.96819088172843 -118.92484149738368) bank10715 +10716 POINT(34.890205603238684 -118.21751773211865) bank10716 +10717 POINT(34.26342844699255 -117.24512554425029) bank10717 +10718 POINT(34.09610956336804 -119.0562027098727) bank10718 +10719 POINT(34.54166908385907 -118.63972610151522) bank10719 +10720 POINT(33.789783747659804 -118.4382679908763) bank10720 +10721 POINT(34.256526253517926 -117.7674959426322) bank10721 +10722 POINT(33.742954576590044 -118.34313086714897) bank10722 +10723 POINT(33.79633871491048 -117.27288159694501) bank10723 +10724 POINT(35.001347184993996 -117.99989548987641) bank10724 +10725 POINT(34.162410757978776 -118.01031071417306) bank10725 +10726 POINT(33.255055263756525 -118.95392639839693) bank10726 +10727 POINT(34.256852585789765 -117.90317811578416) bank10727 +10728 POINT(33.35014036022528 -117.4764394351945) bank10728 +10729 POINT(34.92102230723535 -117.24894662452621) bank10729 +10730 POINT(34.35012980739592 -117.70438851647978) bank10730 +10731 POINT(34.2995735550118 -117.99940815514114) bank10731 +10732 POINT(33.72582303379963 -118.34748589949615) bank10732 +10733 POINT(34.915185432457925 -117.57811026985468) bank10733 +10734 POINT(33.872780570091706 -118.06032843884908) bank10734 +10735 POINT(34.62730657291084 -118.50650746746977) bank10735 +10736 POINT(34.33999180352916 -118.0257335801632) bank10736 +10737 POINT(34.56435620233242 -118.09631840121087) bank10737 +10738 POINT(33.61072202102083 -117.9604814695152) bank10738 +10739 POINT(34.759909158207726 -117.2680349292791) bank10739 +10740 POINT(33.715741507579764 -118.57910173522039) bank10740 +10741 POINT(33.35287753184449 -117.65286748342115) bank10741 +10742 POINT(33.759074417369284 -118.89483887295302) bank10742 +10743 POINT(34.321090944966954 -118.27639736345795) bank10743 +10744 POINT(34.762227906573486 -118.63433858797472) bank10744 +10745 POINT(33.79737767006027 -117.71846939312248) bank10745 +10746 POINT(33.39365182704321 -118.38001455133934) bank10746 +10747 POINT(34.46423512949888 -117.25593615756128) bank10747 +10748 POINT(33.85950968354086 -117.36375796832817) bank10748 +10749 POINT(34.20413114792728 -118.67999482311953) bank10749 +10750 POINT(33.76582227701536 -118.79632535936103) bank10750 +10751 POINT(33.16938285109324 -119.03886540551514) bank10751 +10752 POINT(34.28239412997093 -117.41755270557209) bank10752 +10753 POINT(34.26602993953733 -117.6349107543215) bank10753 +10754 POINT(33.05563448984975 -118.34069238438873) bank10754 +10755 POINT(34.37293852026758 -119.05775085248452) bank10755 +10756 POINT(33.613237443908844 -117.26756942008572) bank10756 +10757 POINT(34.023151659816534 -117.38434796871546) bank10757 +10758 POINT(33.06236020172427 -119.13423392779586) bank10758 +10759 POINT(34.45533735425293 -118.72475059060504) bank10759 +10760 POINT(33.710787709171534 -118.10312595577928) bank10760 +10761 POINT(34.473937545475486 -118.22857102284394) bank10761 +10762 POINT(34.41560275098657 -118.38614642361944) bank10762 +10763 POINT(34.079383374711625 -117.49364189249877) bank10763 +10764 POINT(34.630726845718435 -117.854371369467) bank10764 +10765 POINT(33.86016676985532 -118.01977681381608) bank10765 +10766 POINT(34.05225095388996 -118.17690034790678) bank10766 +10767 POINT(33.54713051133049 -117.67231851838913) bank10767 +10768 POINT(34.824871835839076 -118.72702311437187) bank10768 +10769 POINT(34.39867882419515 -119.06474122949892) bank10769 +10770 POINT(34.603660056446664 -118.75546948294988) bank10770 +10771 POINT(34.77637935838351 -118.9942013968797) bank10771 +10772 POINT(33.22614889739236 -119.20864435722125) bank10772 +10773 POINT(33.77794709368504 -117.78472098292907) bank10773 +10774 POINT(34.8241753311331 -117.27808428931633) bank10774 +10775 POINT(33.78576453695603 -118.49082919722845) bank10775 +10776 POINT(34.26391428628352 -117.71833321688221) bank10776 +10777 POINT(33.60546070998268 -118.90747516621899) bank10777 +10778 POINT(33.11518888787175 -117.60857805237397) bank10778 +10779 POINT(33.538438002957804 -117.84432803932613) bank10779 +10780 POINT(34.68673958047157 -118.61599616661762) bank10780 +10781 POINT(33.456044110023285 -117.50848433741824) bank10781 +10782 POINT(35.00526443992641 -118.86067481976674) bank10782 +10783 POINT(34.14931809644639 -117.42320563190805) bank10783 +10784 POINT(35.0190580856327 -118.18017281927587) bank10784 +10785 POINT(33.235311653857686 -117.79915220607204) bank10785 +10786 POINT(33.54432425454538 -118.95650382511498) bank10786 +10787 POINT(33.85558735493035 -117.74593174024665) bank10787 +10788 POINT(33.369721998110016 -118.46256080498011) bank10788 +10789 POINT(34.64247647138694 -118.64349769434415) bank10789 +10790 POINT(33.76345471831102 -119.09903525685615) bank10790 +10791 POINT(33.175034891692334 -117.3077749227077) bank10791 +10792 POINT(33.5125524256057 -117.81100223378445) bank10792 +10793 POINT(34.95623278441031 -117.55017710614077) bank10793 +10794 POINT(33.594977056900206 -119.11909584019588) bank10794 +10795 POINT(34.56301279677964 -117.90741178647619) bank10795 +10796 POINT(34.4186689943816 -119.09131593072841) bank10796 +10797 POINT(34.504633124658504 -118.21205902926603) bank10797 +10798 POINT(33.44402224921806 -117.69276794075269) bank10798 +10799 POINT(33.9102627261208 -117.57840097627319) bank10799 +10800 POINT(34.74068683764053 -118.93882756908849) bank10800 +10801 POINT(33.38372303975021 -118.9531250925768) bank10801 +10802 POINT(34.12532483300094 -118.39501506318355) bank10802 +10803 POINT(35.02553374677704 -117.86256106359042) bank10803 +10804 POINT(34.60862806709619 -118.85738773999434) bank10804 +10805 POINT(34.82080601922878 -117.84864651717672) bank10805 +10806 POINT(34.86370077836253 -118.20120286025869) bank10806 +10807 POINT(33.95464532782672 -118.51266423433933) bank10807 +10808 POINT(34.9304727607981 -117.91159600573077) bank10808 +10809 POINT(34.38331323190709 -117.43526144959024) bank10809 +10810 POINT(33.291514326413406 -117.7339325143504) bank10810 +10811 POINT(34.96453895582578 -118.48494569191266) bank10811 +10812 POINT(33.64384730980212 -119.10794495500241) bank10812 +10813 POINT(34.959430437920496 -118.43289424159907) bank10813 +10814 POINT(33.3360859196481 -118.87004352324567) bank10814 +10815 POINT(34.93368398896126 -118.69298052719842) bank10815 +10816 POINT(34.3066553012462 -118.26261477071088) bank10816 +10817 POINT(33.05298113182746 -118.58104598830329) bank10817 +10818 POINT(33.124197444415174 -117.73304516852893) bank10818 +10819 POINT(33.96750455642173 -117.46291484417965) bank10819 +10820 POINT(34.08189255477364 -117.93753889223493) bank10820 +10821 POINT(34.76585731506266 -117.25836689183646) bank10821 +10822 POINT(33.38679888419803 -118.04848351046044) bank10822 +10823 POINT(34.87765075839961 -118.86527936206596) bank10823 +10824 POINT(34.05717992031973 -118.31794258955487) bank10824 +10825 POINT(34.08197495054712 -119.07636051391789) bank10825 +10826 POINT(33.88998830337039 -118.25569381449964) bank10826 +10827 POINT(33.288166168525784 -117.59745697794553) bank10827 +10828 POINT(33.71849325576861 -118.12047634986345) bank10828 +10829 POINT(33.189926731693866 -117.51850231814984) bank10829 +10830 POINT(34.89598761497723 -117.35092454968724) bank10830 +10831 POINT(33.435476616129556 -118.99342935243635) bank10831 +10832 POINT(33.65094099361898 -117.40784765723875) bank10832 +10833 POINT(34.76118507400954 -118.61582547403874) bank10833 +10834 POINT(33.06653694739605 -118.82259749351422) bank10834 +10835 POINT(33.23825319384592 -117.71424083001737) bank10835 +10836 POINT(33.65550398563874 -118.48468866006449) bank10836 +10837 POINT(34.14391763111678 -118.41423656436496) bank10837 +10838 POINT(34.788451619352884 -117.98626286595872) bank10838 +10839 POINT(33.53273332786205 -118.31333322996427) bank10839 +10840 POINT(34.40950731354055 -118.69509360856564) bank10840 +10841 POINT(34.308422907777604 -119.09005489957809) bank10841 +10842 POINT(33.59875703509936 -117.96123724518235) bank10842 +10843 POINT(33.48509244632586 -117.88120433052912) bank10843 +10844 POINT(34.88404509676241 -118.78213409327068) bank10844 +10845 POINT(33.72151073116196 -118.58930019534421) bank10845 +10846 POINT(34.788343447686756 -118.21692535332416) bank10846 +10847 POINT(34.4838996132389 -118.81652863985694) bank10847 +10848 POINT(34.52689418980957 -119.05073363431555) bank10848 +10849 POINT(34.0299260001446 -118.89491190084526) bank10849 +10850 POINT(34.5554223555452 -117.9975928774722) bank10850 +10851 POINT(33.624257483335214 -119.20593204326471) bank10851 +10852 POINT(33.34049687821913 -118.39618924140778) bank10852 +10853 POINT(34.67072610796865 -118.68591396788082) bank10853 +10854 POINT(34.74241551443786 -117.35989008568846) bank10854 +10855 POINT(33.504421639247056 -118.83984925967994) bank10855 +10856 POINT(33.56435048167018 -117.41439131818348) bank10856 +10857 POINT(34.48580640416046 -118.12904932533671) bank10857 +10858 POINT(34.95472126415386 -119.11905163255295) bank10858 +10859 POINT(34.26825350805603 -117.58467275121215) bank10859 +10860 POINT(33.49507806210992 -117.67749185652178) bank10860 +10861 POINT(34.80840891957834 -118.86692003023626) bank10861 +10862 POINT(34.29383639274672 -119.06795916147351) bank10862 +10863 POINT(33.69518155102319 -118.60944901726276) bank10863 +10864 POINT(33.72487403362142 -117.91297328730921) bank10864 +10865 POINT(33.997837620325804 -117.42129587324746) bank10865 +10866 POINT(33.94284474476134 -118.23682041329592) bank10866 +10867 POINT(33.21069576987712 -118.8392713052729) bank10867 +10868 POINT(33.2602449572148 -117.9579040252897) bank10868 +10869 POINT(34.84578303210048 -118.04739648185762) bank10869 +10870 POINT(34.23372435494172 -118.43150027048533) bank10870 +10871 POINT(33.63837173653717 -117.6842296552687) bank10871 +10872 POINT(34.976133535361335 -119.06278869492095) bank10872 +10873 POINT(34.76715707351656 -117.72583934599274) bank10873 +10874 POINT(33.29808779394801 -117.74386753394512) bank10874 +10875 POINT(33.70430074812991 -117.56809897100207) bank10875 +10876 POINT(33.31423557299237 -117.32342287166206) bank10876 +10877 POINT(34.08513261354582 -117.61894910678079) bank10877 +10878 POINT(33.07320494959318 -118.00811762068844) bank10878 +10879 POINT(33.868907248518056 -118.14999345518706) bank10879 +10880 POINT(34.84041596664263 -118.58228634907056) bank10880 +10881 POINT(34.36673683449534 -118.43078062471784) bank10881 +10882 POINT(34.92955742911131 -117.28117150013361) bank10882 +10883 POINT(34.685133082056986 -117.6282809252925) bank10883 +10884 POINT(33.90049212344718 -118.94529334451141) bank10884 +10885 POINT(34.575403863387244 -118.30098153585048) bank10885 +10886 POINT(33.539890471428585 -117.73740433465835) bank10886 +10887 POINT(34.5095961433374 -118.91426408584054) bank10887 +10888 POINT(33.7402385852856 -118.06013081208356) bank10888 +10889 POINT(34.22162376718773 -117.5923154317449) bank10889 +10890 POINT(33.37478400409382 -117.31635047484703) bank10890 +10891 POINT(34.82293863056634 -117.82087885446163) bank10891 +10892 POINT(33.138576148987376 -117.68992643955261) bank10892 +10893 POINT(34.15858287588745 -117.77070842532092) bank10893 +10894 POINT(33.581896265230604 -118.04352264249935) bank10894 +10895 POINT(34.53776738860722 -117.80802471513732) bank10895 +10896 POINT(33.803413777819905 -117.52042887055414) bank10896 +10897 POINT(34.69190088389017 -118.55566207820138) bank10897 +10898 POINT(33.44282679005449 -118.69212506124356) bank10898 +10899 POINT(34.376476331648036 -117.81470238454578) bank10899 +10900 POINT(34.36017627998893 -118.1609352460548) bank10900 +10901 POINT(33.5856405947195 -119.18531433126648) bank10901 +10902 POINT(33.53128439916516 -118.38505994317711) bank10902 +10903 POINT(33.41341267600087 -117.38846784672737) bank10903 +10904 POINT(34.01684701354776 -117.96501655076261) bank10904 +10905 POINT(33.65759872736533 -118.66779299628404) bank10905 +10906 POINT(34.15106042541737 -117.40128215763806) bank10906 +10907 POINT(33.77566266535073 -119.04407320049454) bank10907 +10908 POINT(34.763027503205 -117.3572512807594) bank10908 +10909 POINT(33.295002569539385 -118.30855195511155) bank10909 +10910 POINT(33.687096193995636 -117.78138761631514) bank10910 +10911 POINT(34.668077470993545 -118.96599075916482) bank10911 +10912 POINT(33.97031943762991 -118.19646735890282) bank10912 +10913 POINT(34.68928377689488 -118.6301751266391) bank10913 +10914 POINT(34.136208371837704 -117.77628012385294) bank10914 +10915 POINT(33.203482696581695 -119.14344132585387) bank10915 +10916 POINT(33.14908268661835 -119.02906192435921) bank10916 +10917 POINT(33.15119219819183 -118.21214409468153) bank10917 +10918 POINT(33.74230691054745 -118.8991221291898) bank10918 +10919 POINT(34.380701643646574 -118.31301997830852) bank10919 +10920 POINT(34.18227346599908 -117.58011704941896) bank10920 +10921 POINT(34.043973885296964 -118.97941774091693) bank10921 +10922 POINT(33.49211555625572 -118.16091309136644) bank10922 +10923 POINT(33.80628292877234 -118.44790466603635) bank10923 +10924 POINT(33.632334974474 -118.84611213503412) bank10924 +10925 POINT(35.03788455600973 -117.44872228203107) bank10925 +10926 POINT(34.99077742459773 -118.59057277803339) bank10926 +10927 POINT(34.1784282025105 -117.92251389894173) bank10927 +10928 POINT(33.61717642366401 -117.4117766940801) bank10928 +10929 POINT(35.010262739822366 -118.4467907022541) bank10929 +10930 POINT(34.011685877979325 -118.17586257940849) bank10930 +10931 POINT(34.82450750548818 -118.47727563198023) bank10931 +10932 POINT(33.442765550807664 -117.67310404197501) bank10932 +10933 POINT(34.647030052400964 -118.83329913805338) bank10933 +10934 POINT(33.1391846440327 -118.73529737920731) bank10934 +10935 POINT(34.40094010403543 -117.4117344197455) bank10935 +10936 POINT(33.81064716130684 -117.70971525053885) bank10936 +10937 POINT(33.57582159251784 -117.77382590828505) bank10937 +10938 POINT(33.36875613923702 -118.44608532988761) bank10938 +10939 POINT(34.51065633586253 -118.44856988535794) bank10939 +10940 POINT(33.57553573067408 -117.4200661723846) bank10940 +10941 POINT(34.92510932064862 -118.5788287351382) bank10941 +10942 POINT(33.9768586702441 -117.94614934796236) bank10942 +10943 POINT(34.9752896361495 -117.83920584629342) bank10943 +10944 POINT(33.48017555882811 -119.13006967743402) bank10944 +10945 POINT(34.309611295834706 -118.4944611796855) bank10945 +10946 POINT(33.70859436785888 -117.52118197183104) bank10946 +10947 POINT(33.9622093676848 -118.70205765340323) bank10947 +10948 POINT(33.301757357048906 -119.22342140244342) bank10948 +10949 POINT(34.25544028536221 -117.5017637990061) bank10949 +10950 POINT(34.63359695269381 -118.68419961236555) bank10950 +10951 POINT(33.87541012062502 -118.47559544590143) bank10951 +10952 POINT(34.77248849940229 -117.6069387222878) bank10952 +10953 POINT(34.40731617495292 -117.37304253741281) bank10953 +10954 POINT(33.232307515146104 -118.53247664183199) bank10954 +10955 POINT(33.55868816112109 -118.2936705247608) bank10955 +10956 POINT(33.22210825250231 -118.11889663218302) bank10956 +10957 POINT(34.28946941547684 -117.90136934418464) bank10957 +10958 POINT(34.40123052381377 -117.74680222364908) bank10958 +10959 POINT(34.062187026619014 -118.94791498444926) bank10959 +10960 POINT(34.185218163555476 -119.16192592381563) bank10960 +10961 POINT(34.707015368219565 -117.48289654350785) bank10961 +10962 POINT(33.19330087639988 -117.42139005895822) bank10962 +10963 POINT(33.745162301501814 -117.32361526962121) bank10963 +10964 POINT(34.583641528897246 -119.14410102583268) bank10964 +10965 POINT(34.104111816439755 -117.28626353454243) bank10965 +10966 POINT(33.398924943183125 -118.5901700194665) bank10966 +10967 POINT(34.77686070407621 -118.93684290798798) bank10967 +10968 POINT(33.7928538887318 -119.13334482377189) bank10968 +10969 POINT(33.16619746602808 -118.8971919614776) bank10969 +10970 POINT(33.49965796761752 -118.69605909860168) bank10970 +10971 POINT(34.37192786639094 -117.33680917665387) bank10971 +10972 POINT(33.116597427733744 -117.93049313954148) bank10972 +10973 POINT(34.855290379052924 -119.08440054487014) bank10973 +10974 POINT(34.52304966525817 -119.01499797049884) bank10974 +10975 POINT(34.202241577906555 -117.86560655377964) bank10975 +10976 POINT(34.01445261027176 -117.90019435906817) bank10976 +10977 POINT(34.32425630281898 -117.79001741676333) bank10977 +10978 POINT(33.60005000468778 -118.86858718676216) bank10978 +10979 POINT(33.62341028442177 -118.13653894317447) bank10979 +10980 POINT(33.57818856469934 -117.91877795569597) bank10980 +10981 POINT(33.13686672825137 -118.6585155501737) bank10981 +10982 POINT(34.776824774788906 -118.08604617321991) bank10982 +10983 POINT(34.18906742193705 -118.46006419142917) bank10983 +10984 POINT(34.383397682420394 -119.02666906520565) bank10984 +10985 POINT(33.793260070117604 -118.62645122805232) bank10985 +10986 POINT(33.91964112329929 -119.21935953743157) bank10986 +10987 POINT(34.045491381401895 -118.65625646512191) bank10987 +10988 POINT(34.619390805021986 -118.00778250741625) bank10988 +10989 POINT(34.87749638474584 -118.84052814966228) bank10989 +10990 POINT(34.259858151451 -117.44952893141141) bank10990 +10991 POINT(34.814225451306385 -118.03290971659717) bank10991 +10992 POINT(34.11223308538234 -118.04749540071884) bank10992 +10993 POINT(33.55428800699062 -118.80120433798194) bank10993 +10994 POINT(33.930630055771225 -119.18569196900307) bank10994 +10995 POINT(34.2292321733053 -118.07081631964179) bank10995 +10996 POINT(34.79247691392405 -117.61400800626535) bank10996 +10997 POINT(34.72684046726542 -118.03869916877358) bank10997 +10998 POINT(34.36906117928632 -117.43957451429027) bank10998 +10999 POINT(33.537098340762654 -119.12971748092059) bank10999 +11000 POINT(33.56654436953296 -118.37739718560876) bank11000 +11001 POINT(34.5100222196907 -118.91190731308926) bank11001 +11002 POINT(34.68889229014582 -118.46342816160906) bank11002 +11003 POINT(34.01653902419344 -118.81862026563532) bank11003 +11004 POINT(33.60136438737348 -119.1205054729032) bank11004 +11005 POINT(33.3901527946212 -117.32761261103484) bank11005 +11006 POINT(34.14570387346231 -119.06385536865078) bank11006 +11007 POINT(34.447562233475885 -118.27325479522591) bank11007 +11008 POINT(34.601801128197984 -119.02142333849099) bank11008 +11009 POINT(33.90777464268591 -119.07838740176356) bank11009 +11010 POINT(34.206725626010176 -119.17381529283013) bank11010 +11011 POINT(34.065946467148905 -118.45204516227685) bank11011 +11012 POINT(33.40884687797962 -117.46232446301138) bank11012 +11013 POINT(33.84559286805793 -118.24508805056706) bank11013 +11014 POINT(34.46873386768734 -118.96971532160568) bank11014 +11015 POINT(34.509365833209046 -118.25085512567861) bank11015 +11016 POINT(34.887739860277435 -117.82857946019159) bank11016 +11017 POINT(34.52828525442208 -117.895814406896) bank11017 +11018 POINT(34.98831797904996 -117.83585341962178) bank11018 +11019 POINT(34.01337459758529 -118.04559196133064) bank11019 +11020 POINT(33.36636412170511 -119.02312751813713) bank11020 +11021 POINT(33.15384782539086 -117.24891130744184) bank11021 +11022 POINT(34.07703180953551 -117.5890057477407) bank11022 +11023 POINT(33.707021625523865 -117.41328085467347) bank11023 +11024 POINT(34.0900198507035 -119.05789973622858) bank11024 +11025 POINT(34.934141999584 -117.40463733064031) bank11025 +11026 POINT(33.81861319900949 -117.49720225232353) bank11026 +11027 POINT(34.11919604256177 -117.56472339563477) bank11027 +11028 POINT(33.260138310047786 -118.75224328395082) bank11028 +11029 POINT(33.474914700228325 -117.53742642019839) bank11029 +11030 POINT(34.109948068078104 -118.2458386471798) bank11030 +11031 POINT(34.17314413259645 -118.42375716266156) bank11031 +11032 POINT(34.25872113161514 -118.14364509858095) bank11032 +11033 POINT(34.24493103099922 -117.909890507449) bank11033 +11034 POINT(34.73394638283606 -117.704554672227) bank11034 +11035 POINT(33.08263231520326 -117.94956433179682) bank11035 +11036 POINT(34.6078764000664 -118.33521871163155) bank11036 +11037 POINT(34.87295810526007 -117.26881714897642) bank11037 +11038 POINT(33.82168202189159 -118.05652102171983) bank11038 +11039 POINT(34.11358670155926 -118.1608756854541) bank11039 +11040 POINT(34.21932064691373 -117.55003491384346) bank11040 +11041 POINT(34.00555612770675 -117.82025679163073) bank11041 +11042 POINT(34.00192249659423 -118.5474006842693) bank11042 +11043 POINT(33.12421213339548 -118.56590459540767) bank11043 +11044 POINT(34.02241517779581 -118.77416063563297) bank11044 +11045 POINT(33.13060567524242 -117.67386603025092) bank11045 +11046 POINT(33.71906333179188 -118.16028207465291) bank11046 +11047 POINT(34.294279595072844 -118.73166784694256) bank11047 +11048 POINT(34.50949616691345 -118.32191929438086) bank11048 +11049 POINT(34.77860276972585 -117.59458997870506) bank11049 +11050 POINT(34.92112531846627 -117.68500889138086) bank11050 +11051 POINT(34.74828909160323 -119.2196195789403) bank11051 +11052 POINT(33.8833642016508 -118.42078642165824) bank11052 +11053 POINT(34.41445686993469 -118.96637119180741) bank11053 +11054 POINT(33.90667404588799 -117.62410957601705) bank11054 +11055 POINT(34.79514441426185 -119.1811722973881) bank11055 +11056 POINT(33.413970310040426 -117.57732581183258) bank11056 +11057 POINT(34.05816960635196 -117.32867816638185) bank11057 +11058 POINT(33.169301454903916 -118.12106528809346) bank11058 +11059 POINT(34.23300119404409 -118.86303698512832) bank11059 +11060 POINT(33.7246363242802 -118.28168763470434) bank11060 +11061 POINT(34.89036446657929 -118.5104740539029) bank11061 +11062 POINT(34.88309978685983 -119.03521029802857) bank11062 +11063 POINT(34.61183513049497 -117.67977615104691) bank11063 +11064 POINT(33.90664470482661 -117.74972964989244) bank11064 +11065 POINT(34.44464899236535 -118.91110629971146) bank11065 +11066 POINT(33.137420280209604 -117.58359553467913) bank11066 +11067 POINT(34.030596049206 -118.82735420938063) bank11067 +11068 POINT(34.13015046886933 -118.15332225393028) bank11068 +11069 POINT(34.24752523849233 -118.63205247105729) bank11069 +11070 POINT(33.85909300214673 -118.76177018581417) bank11070 +11071 POINT(33.84518370573679 -117.36101252304631) bank11071 +11072 POINT(33.124371237330344 -118.407718039493) bank11072 +11073 POINT(34.25080094269848 -118.42889184979737) bank11073 +11074 POINT(34.163701567064514 -117.63511014418718) bank11074 +11075 POINT(33.603662349416204 -117.28805261155446) bank11075 +11076 POINT(33.59713692923174 -118.54821966295505) bank11076 +11077 POINT(34.084622991548926 -118.78899305801295) bank11077 +11078 POINT(33.92747695047235 -117.83455231474807) bank11078 +11079 POINT(34.991128857000625 -117.53673275802164) bank11079 +11080 POINT(33.37423151423228 -117.5515188344909) bank11080 +11081 POINT(33.93523611137051 -119.02949327960845) bank11081 +11082 POINT(34.878872273540296 -119.01442052954731) bank11082 +11083 POINT(33.8542416243936 -117.43005511058949) bank11083 +11084 POINT(33.36784094870683 -118.87040951035638) bank11084 +11085 POINT(33.052433321532334 -118.30187613442347) bank11085 +11086 POINT(33.881677440325745 -117.42053586503586) bank11086 +11087 POINT(33.498670638818616 -117.32873517324639) bank11087 +11088 POINT(34.04777044880177 -117.912565269988) bank11088 +11089 POINT(33.869758054796925 -118.57691262043267) bank11089 +11090 POINT(33.146911186110266 -118.09037468314678) bank11090 +11091 POINT(34.83729981505153 -118.02380789793963) bank11091 +11092 POINT(34.63520679051912 -118.18631822770303) bank11092 +11093 POINT(34.47451769648029 -119.02500075038657) bank11093 +11094 POINT(34.7152377669952 -119.147042383434) bank11094 +11095 POINT(34.86382596218937 -117.96498916728775) bank11095 +11096 POINT(33.442039821920936 -117.74604939213212) bank11096 +11097 POINT(33.406830424184044 -117.79483903130229) bank11097 +11098 POINT(33.471316383979456 -118.01011161668082) bank11098 +11099 POINT(33.40147854326486 -118.80433451338523) bank11099 +11100 POINT(34.015084482168156 -117.6681148332715) bank11100 +11101 POINT(34.88413759041928 -117.72472818188265) bank11101 +11102 POINT(35.04516941360775 -118.47548445081596) bank11102 +11103 POINT(33.5022771905953 -118.15548229838998) bank11103 +11104 POINT(33.645026188680866 -117.58299254329428) bank11104 +11105 POINT(33.15143673585015 -118.2849157269325) bank11105 +11106 POINT(34.616186638055716 -118.80570631379472) bank11106 +11107 POINT(33.52542594929356 -118.85164435415808) bank11107 +11108 POINT(33.83488679799761 -117.78557055100826) bank11108 +11109 POINT(34.13976847377947 -118.13350135920828) bank11109 +11110 POINT(34.408876224996284 -118.66375892386395) bank11110 +11111 POINT(33.863064174437305 -117.98512013952524) bank11111 +11112 POINT(33.49400845116273 -118.18362488313336) bank11112 +11113 POINT(33.205361102230455 -117.86506944375385) bank11113 +11114 POINT(34.16541335827393 -119.09559056171841) bank11114 +11115 POINT(34.70894644504878 -117.41641565216045) bank11115 +11116 POINT(34.63986132754603 -117.9565388838816) bank11116 +11117 POINT(34.15713118877373 -118.28655386296815) bank11117 +11118 POINT(33.570471705706495 -119.15505184094907) bank11118 +11119 POINT(33.632060147386646 -117.72068981002862) bank11119 +11120 POINT(34.26628873789961 -117.97667845091564) bank11120 +11121 POINT(34.614572164094184 -118.79446537254316) bank11121 +11122 POINT(34.109959440118786 -118.40034383735498) bank11122 +11123 POINT(34.84199374333636 -117.40401239694914) bank11123 +11124 POINT(33.8238882792396 -117.47944196361884) bank11124 +11125 POINT(33.81503759330745 -118.21695423197392) bank11125 +11126 POINT(33.33726605701694 -118.2077701888465) bank11126 +11127 POINT(34.42745416145113 -118.76721436683181) bank11127 +11128 POINT(33.639701189591186 -118.73225743673272) bank11128 +11129 POINT(33.78655051590061 -118.11871623512747) bank11129 +11130 POINT(33.595622597417396 -118.3258909543807) bank11130 +11131 POINT(35.00710758387079 -119.22619368530415) bank11131 +11132 POINT(34.25810499107123 -117.53844950387762) bank11132 +11133 POINT(34.94667064135538 -118.30474261704128) bank11133 +11134 POINT(34.254177430059976 -118.37288731052124) bank11134 +11135 POINT(34.85781550794415 -118.46225858832729) bank11135 +11136 POINT(34.74663775361965 -119.17102793077969) bank11136 +11137 POINT(34.748853011453186 -117.97697756308892) bank11137 +11138 POINT(33.52423586598183 -118.85952936991262) bank11138 +11139 POINT(33.92360699701719 -118.0159507785142) bank11139 +11140 POINT(33.5587986766458 -117.9980790465631) bank11140 +11141 POINT(34.287524892451 -117.84064300124341) bank11141 +11142 POINT(33.229127297618554 -117.9148603905073) bank11142 +11143 POINT(33.36299817100898 -118.58224363027429) bank11143 +11144 POINT(33.17254387792481 -118.40407841714368) bank11144 +11145 POINT(34.31798830102429 -117.99183302393321) bank11145 +11146 POINT(34.313201059546074 -118.98413484259217) bank11146 +11147 POINT(34.307458858610474 -118.75653100785412) bank11147 +11148 POINT(34.169509308666726 -118.07724706226067) bank11148 +11149 POINT(33.594867504491795 -118.15096495059015) bank11149 +11150 POINT(33.111342782731064 -118.78404582245282) bank11150 +11151 POINT(34.756261926369135 -117.76271659888823) bank11151 +11152 POINT(33.39023380140018 -118.74874273937137) bank11152 +11153 POINT(34.91443844422675 -118.17054106043634) bank11153 +11154 POINT(33.30182674128112 -117.31951603292224) bank11154 +11155 POINT(34.62922335516064 -119.11112592567437) bank11155 +11156 POINT(34.49636166799962 -117.97513145777289) bank11156 +11157 POINT(34.85631531117212 -117.92058392676807) bank11157 +11158 POINT(33.52278130887752 -118.35848315921793) bank11158 +11159 POINT(34.44556025421054 -118.55772880480755) bank11159 +11160 POINT(33.463105593731136 -117.36398877212633) bank11160 +11161 POINT(34.25459665661189 -119.21256608011248) bank11161 +11162 POINT(35.01068442216026 -117.38168382673123) bank11162 +11163 POINT(33.8004301306082 -117.8281401409648) bank11163 +11164 POINT(33.54542577275481 -117.41783099958778) bank11164 +11165 POINT(33.374669079062066 -117.89563164002017) bank11165 +11166 POINT(34.4814038204482 -118.71874115712042) bank11166 +11167 POINT(33.14971234425776 -118.61439238378206) bank11167 +11168 POINT(33.754104008619926 -118.44985717727812) bank11168 +11169 POINT(34.91866042868204 -117.41395110587648) bank11169 +11170 POINT(33.13401319989172 -117.58044172756487) bank11170 +11171 POINT(33.67991684462401 -117.8395986653468) bank11171 +11172 POINT(34.52378921439225 -118.85033226119234) bank11172 +11173 POINT(33.42103092088795 -119.05822686737369) bank11173 +11174 POINT(34.86999365639253 -118.98043762619986) bank11174 +11175 POINT(34.518568835359574 -118.84893654993522) bank11175 +11176 POINT(34.56527399786844 -118.52189657851434) bank11176 +11177 POINT(34.44495674530517 -117.85727381317433) bank11177 +11178 POINT(33.61283446413225 -117.63908816868583) bank11178 +11179 POINT(33.42529208433917 -118.17530792758416) bank11179 +11180 POINT(34.174200257237636 -119.07502097635408) bank11180 +11181 POINT(33.61215510825378 -118.7014257548051) bank11181 +11182 POINT(33.06051725020234 -118.98819952262404) bank11182 +11183 POINT(34.644430985194546 -118.11723062618073) bank11183 +11184 POINT(34.050349930651706 -117.67455844215165) bank11184 +11185 POINT(34.78868505041674 -119.0096608135687) bank11185 +11186 POINT(34.80814819877585 -117.33944827509825) bank11186 +11187 POINT(34.68483143127489 -117.60369513843399) bank11187 +11188 POINT(34.8701077445513 -119.18263174625964) bank11188 +11189 POINT(34.30244487605663 -118.75454411490693) bank11189 +11190 POINT(34.726697720836626 -117.91651312898762) bank11190 +11191 POINT(34.92513401679276 -118.34259680906835) bank11191 +11192 POINT(33.77095024991455 -118.86278608408517) bank11192 +11193 POINT(33.72671099033484 -118.99108152817725) bank11193 +11194 POINT(34.985668728773575 -117.59123036522469) bank11194 +11195 POINT(33.996762746518264 -118.7854423707983) bank11195 +11196 POINT(34.18485214316897 -118.02503689549962) bank11196 +11197 POINT(34.39486548269425 -119.04978027243395) bank11197 +11198 POINT(33.34589968902589 -118.15442326377665) bank11198 +11199 POINT(33.08094237333927 -117.5564572928566) bank11199 +11200 POINT(33.60316509370089 -117.6215996521257) bank11200 +11201 POINT(34.660772724995816 -117.41409679338595) bank11201 +11202 POINT(35.04901284180126 -117.51519256299456) bank11202 +11203 POINT(33.14058916225862 -118.70747014545391) bank11203 +11204 POINT(34.65334261150328 -118.58170057609395) bank11204 +11205 POINT(33.43634471393368 -118.82737545382871) bank11205 +11206 POINT(33.356580621373084 -118.23612870660205) bank11206 +11207 POINT(34.24306422368055 -119.20735531899035) bank11207 +11208 POINT(33.29924863785497 -119.05259731162849) bank11208 +11209 POINT(33.8579116984739 -118.75595163752578) bank11209 +11210 POINT(33.070911714073105 -118.16979960967352) bank11210 +11211 POINT(34.49081536848503 -118.9745213716244) bank11211 +11212 POINT(33.667641541704 -118.32662270891328) bank11212 +11213 POINT(33.43620984508668 -118.63419673141577) bank11213 +11214 POINT(33.52273669275451 -117.46548711123467) bank11214 +11215 POINT(34.61504035691354 -118.3800108547071) bank11215 +11216 POINT(34.32500346955292 -117.80682121434998) bank11216 +11217 POINT(34.38055935532795 -118.97359740480906) bank11217 +11218 POINT(34.444137624657486 -118.6304241015001) bank11218 +11219 POINT(34.76878626657059 -117.75953131800767) bank11219 +11220 POINT(34.75385600933096 -118.47589644561432) bank11220 +11221 POINT(33.2181876338992 -117.81561066028446) bank11221 +11222 POINT(34.87440869657015 -118.39704827468235) bank11222 +11223 POINT(33.15522625999672 -118.52527785981985) bank11223 +11224 POINT(34.08180389933492 -118.64317394782438) bank11224 +11225 POINT(34.457102638247385 -118.82513577748121) bank11225 +11226 POINT(34.802675407521114 -117.48108929826364) bank11226 +11227 POINT(34.43855250836749 -118.60275286329862) bank11227 +11228 POINT(34.52017581902933 -119.08088734833889) bank11228 +11229 POINT(34.83107544257509 -117.84285970768212) bank11229 +11230 POINT(34.93652185181037 -118.39327954380575) bank11230 +11231 POINT(33.15833787224417 -118.77086944660522) bank11231 +11232 POINT(34.262050940163796 -118.07106420709455) bank11232 +11233 POINT(33.42092203345387 -119.17084816801477) bank11233 +11234 POINT(34.028320132968574 -118.21221667220529) bank11234 +11235 POINT(34.678506014884825 -117.42837688366436) bank11235 +11236 POINT(35.04556785835371 -118.47368399024379) bank11236 +11237 POINT(33.421750036389476 -118.1900492771911) bank11237 +11238 POINT(33.29173988432112 -117.88532373964163) bank11238 +11239 POINT(34.289527604035726 -118.24926567050292) bank11239 +11240 POINT(34.60537113542519 -117.63016039988777) bank11240 +11241 POINT(34.5892138228598 -118.358975967214) bank11241 +11242 POINT(34.79546007838161 -117.61792281041149) bank11242 +11243 POINT(33.09177737646203 -117.68317911129971) bank11243 +11244 POINT(33.59399706945649 -118.44371054441424) bank11244 +11245 POINT(34.86943162914576 -118.6588863265419) bank11245 +11246 POINT(33.61960782136758 -118.66774576015526) bank11246 +11247 POINT(35.00787579528765 -118.90350852977801) bank11247 +11248 POINT(34.74322630582539 -117.99636535292062) bank11248 +11249 POINT(33.15282976826677 -118.47766006386374) bank11249 +11250 POINT(33.77549713627298 -117.9590668433371) bank11250 +11251 POINT(34.88539354779861 -117.90462750080727) bank11251 +11252 POINT(34.85350170925406 -118.7837658392312) bank11252 +11253 POINT(34.92899498908982 -119.21001078425442) bank11253 +11254 POINT(34.46579062538852 -117.24840673254312) bank11254 +11255 POINT(34.03445204653907 -118.9616732282455) bank11255 +11256 POINT(34.755893968568145 -119.00813176609164) bank11256 +11257 POINT(33.41526298460627 -118.86707903485303) bank11257 +11258 POINT(34.18403596145601 -117.47171830961366) bank11258 +11259 POINT(33.96235340177761 -118.34687928336729) bank11259 +11260 POINT(33.6344200853624 -118.36176350173668) bank11260 +11261 POINT(33.07604934998179 -118.84933308804011) bank11261 +11262 POINT(34.38216874970686 -117.60681357142697) bank11262 +11263 POINT(33.63037316341542 -119.23819462994834) bank11263 +11264 POINT(33.11716421495851 -118.5828178937098) bank11264 +11265 POINT(34.303767883780694 -117.59044389351642) bank11265 +11266 POINT(34.82858757979513 -117.29294199649557) bank11266 +11267 POINT(33.2535505591971 -118.37576619882714) bank11267 +11268 POINT(33.64261462865282 -118.64451105733303) bank11268 +11269 POINT(34.92117402640349 -118.06144687792622) bank11269 +11270 POINT(34.91302386853045 -118.00782112465642) bank11270 +11271 POINT(33.07355423045514 -117.68581327610109) bank11271 +11272 POINT(33.724278423700184 -117.37051346212199) bank11272 +11273 POINT(33.79372868980428 -117.8772292757485) bank11273 +11274 POINT(34.56296318196166 -119.0197625365793) bank11274 +11275 POINT(33.15261038963381 -117.80625992576635) bank11275 +11276 POINT(33.62987076040532 -117.87738422508579) bank11276 +11277 POINT(33.83079912205631 -117.63246625403413) bank11277 +11278 POINT(34.108061454792214 -119.07987891900312) bank11278 +11279 POINT(33.87518829046895 -118.46181247201586) bank11279 +11280 POINT(33.32431984864545 -118.48052921751669) bank11280 +11281 POINT(33.558567196599675 -118.87051851921747) bank11281 +11282 POINT(35.00370261249024 -118.8303654940599) bank11282 +11283 POINT(33.90007723712533 -117.46410726796944) bank11283 +11284 POINT(34.29468422570675 -117.44252198495077) bank11284 +11285 POINT(33.47697658632094 -119.15282574345603) bank11285 +11286 POINT(34.722865758196654 -117.60068403241286) bank11286 +11287 POINT(33.14183194120615 -118.68293890571219) bank11287 +11288 POINT(34.61952194965795 -117.67435596584869) bank11288 +11289 POINT(34.02894909330916 -117.84419261904512) bank11289 +11290 POINT(34.79730441287654 -117.34386447227551) bank11290 +11291 POINT(34.68012638032384 -118.71429113620538) bank11291 +11292 POINT(33.23299673400406 -118.96941012275148) bank11292 +11293 POINT(34.03098757817477 -117.6068108111627) bank11293 +11294 POINT(33.29947960592115 -117.39631546978259) bank11294 +11295 POINT(34.106752856924906 -118.7169519635355) bank11295 +11296 POINT(33.45018859268265 -119.00011525220079) bank11296 +11297 POINT(34.82617615168236 -118.24568412009474) bank11297 +11298 POINT(34.30926621199174 -118.05456800886522) bank11298 +11299 POINT(34.284524244206075 -117.73000960293038) bank11299 +11300 POINT(34.46394828995062 -117.79293705923753) bank11300 +11301 POINT(34.42045489089805 -117.56184551701078) bank11301 +11302 POINT(33.99554611700256 -117.47938707602299) bank11302 +11303 POINT(34.037479534136956 -118.53068615964195) bank11303 +11304 POINT(34.27598662912066 -119.01240783901582) bank11304 +11305 POINT(33.72022203520655 -117.7782917360518) bank11305 +11306 POINT(33.27459534844026 -117.56337367124274) bank11306 +11307 POINT(34.95215276069919 -117.31319179099887) bank11307 +11308 POINT(34.17705483653274 -117.38406132203896) bank11308 +11309 POINT(34.93555498843615 -118.70920847430389) bank11309 +11310 POINT(34.244715555898786 -117.4803828834744) bank11310 +11311 POINT(33.205746893355176 -118.25821945438241) bank11311 +11312 POINT(33.85119643132026 -117.31909812231933) bank11312 +11313 POINT(34.37796206758012 -118.43771971002245) bank11313 +11314 POINT(33.53147523970138 -117.34308387446528) bank11314 +11315 POINT(34.82520768956533 -119.13921547983418) bank11315 +11316 POINT(33.07777032243295 -118.54153602663288) bank11316 +11317 POINT(34.522004056609894 -117.72569164281715) bank11317 +11318 POINT(33.64836668390986 -118.10406594937236) bank11318 +11319 POINT(33.72755196696675 -117.45952750024178) bank11319 +11320 POINT(34.414893755200744 -118.1963862519194) bank11320 +11321 POINT(34.87106918188911 -119.22480198891368) bank11321 +11322 POINT(33.76310506392268 -117.68596217834916) bank11322 +11323 POINT(34.15343509135566 -117.37999284143382) bank11323 +11324 POINT(34.427934318931726 -117.64442089790285) bank11324 +11325 POINT(34.20987454828103 -118.30061788517543) bank11325 +11326 POINT(34.271045065723165 -117.36959216641455) bank11326 +11327 POINT(34.539456256767984 -119.03740135073083) bank11327 +11328 POINT(33.062078721663816 -119.0290695478828) bank11328 +11329 POINT(34.903451228966105 -118.489078271344) bank11329 +11330 POINT(33.330209568596686 -117.53800248207726) bank11330 +11331 POINT(34.90965759203708 -117.31305271832257) bank11331 +11332 POINT(33.72338636617081 -118.95646694541576) bank11332 +11333 POINT(33.185876212190685 -119.20369624732962) bank11333 +11334 POINT(33.72650814062159 -118.1169238296716) bank11334 +11335 POINT(34.06448179032102 -117.25062301077287) bank11335 +11336 POINT(33.133835671522455 -117.3798684535958) bank11336 +11337 POINT(34.605732470576704 -118.05298416604707) bank11337 +11338 POINT(33.54882200139282 -117.98033841383463) bank11338 +11339 POINT(34.281847002748655 -119.03319636115722) bank11339 +11340 POINT(34.12409071158451 -117.87291890084738) bank11340 +11341 POINT(33.09337677249699 -117.34062531976495) bank11341 +11342 POINT(33.409564666410326 -118.27877199882093) bank11342 +11343 POINT(33.90127948397193 -118.23725639239525) bank11343 +11344 POINT(35.03445561677522 -118.32431353764605) bank11344 +11345 POINT(33.552191656065844 -118.1153391248883) bank11345 +11346 POINT(33.4609041812196 -118.06786724645045) bank11346 +11347 POINT(34.87977028971827 -117.95227297810348) bank11347 +11348 POINT(34.934653358437075 -117.28155950386878) bank11348 +11349 POINT(33.430047680549464 -118.861616653375) bank11349 +11350 POINT(33.42957694998742 -117.60805243465559) bank11350 +11351 POINT(34.8499268077753 -117.74617737039034) bank11351 +11352 POINT(34.27850539738102 -118.82443971861964) bank11352 +11353 POINT(34.08079175209504 -117.79643049233812) bank11353 +11354 POINT(34.25260621479392 -118.89766686689649) bank11354 +11355 POINT(34.62781369470175 -117.4592035366556) bank11355 +11356 POINT(34.33679501689668 -118.33674787333725) bank11356 +11357 POINT(34.70223905892297 -117.41430037558426) bank11357 +11358 POINT(34.493245757101846 -117.28407327511528) bank11358 +11359 POINT(33.06306874209408 -118.11469998649167) bank11359 +11360 POINT(34.089828699377996 -118.17278835918667) bank11360 +11361 POINT(33.94444764436583 -117.58496571617918) bank11361 +11362 POINT(33.662918313009165 -118.32890081086094) bank11362 +11363 POINT(34.3227880815471 -118.95605827239584) bank11363 +11364 POINT(33.76999574640399 -118.75455073812098) bank11364 +11365 POINT(33.25320948106911 -117.50346956067511) bank11365 +11366 POINT(33.24932612803489 -118.79703369581078) bank11366 +11367 POINT(33.15180479770076 -118.69859062395356) bank11367 +11368 POINT(33.2614094335584 -117.47301838050905) bank11368 +11369 POINT(34.560022546037814 -117.50847952403825) bank11369 +11370 POINT(33.76518306432243 -117.84816321220269) bank11370 +11371 POINT(33.47943013968882 -117.39343209600615) bank11371 +11372 POINT(33.778332964398594 -118.07649630872841) bank11372 +11373 POINT(34.5107007309668 -117.85357973774617) bank11373 +11374 POINT(34.0951701180099 -117.76730358299352) bank11374 +11375 POINT(34.67885597142634 -117.599460945448) bank11375 +11376 POINT(34.03117418109711 -118.85853261836222) bank11376 +11377 POINT(33.41348917422537 -119.02249520321845) bank11377 +11378 POINT(33.244073996098344 -118.70039998553091) bank11378 +11379 POINT(34.93936276694753 -118.21526007961371) bank11379 +11380 POINT(33.21172395972433 -117.92034605445625) bank11380 +11381 POINT(33.571882219050984 -117.31227024256278) bank11381 +11382 POINT(34.44543406629923 -118.18151046912922) bank11382 +11383 POINT(34.18724784340252 -117.27676997733177) bank11383 +11384 POINT(33.79257845915299 -118.31446872604272) bank11384 +11385 POINT(33.33691661683889 -119.19348098354881) bank11385 +11386 POINT(34.944422890178366 -117.83169443827772) bank11386 +11387 POINT(34.159458221680666 -118.96628401468286) bank11387 +11388 POINT(33.231876467812945 -118.03125883542818) bank11388 +11389 POINT(33.500435580564144 -118.73012392107114) bank11389 +11390 POINT(34.18309140568358 -117.56156438209513) bank11390 +11391 POINT(34.08363336297619 -118.15285270233173) bank11391 +11392 POINT(33.144013337920235 -118.6741066390785) bank11392 +11393 POINT(34.85981150130632 -117.47967288581692) bank11393 +11394 POINT(34.33382059289524 -118.27444999593662) bank11394 +11395 POINT(33.84730662410054 -119.15408373473025) bank11395 +11396 POINT(34.45528763044796 -118.598228507259) bank11396 +11397 POINT(33.48177288846766 -117.51758846462461) bank11397 +11398 POINT(34.818460474566265 -117.72003359896473) bank11398 +11399 POINT(34.76001132348834 -117.38060570480509) bank11399 +11400 POINT(33.601931733771345 -117.37122410035332) bank11400 +11401 POINT(34.390980358915975 -118.1689214587935) bank11401 +11402 POINT(34.202392281264096 -117.42359602745483) bank11402 +11403 POINT(33.69330301950452 -118.14552508498535) bank11403 +11404 POINT(33.417036532320225 -118.13869653389878) bank11404 +11405 POINT(33.953695396424784 -119.11869061236584) bank11405 +11406 POINT(34.48537436724362 -118.74692162932348) bank11406 +11407 POINT(34.20332799766581 -117.98002288256535) bank11407 +11408 POINT(33.96037911106453 -119.01547210035513) bank11408 +11409 POINT(33.93589391607476 -117.50543115626117) bank11409 +11410 POINT(34.29450875940197 -118.93570547392389) bank11410 +11411 POINT(33.79832334389645 -118.03731771688808) bank11411 +11412 POINT(34.381875958616696 -118.90569968971472) bank11412 +11413 POINT(34.93839714384026 -118.0813681917074) bank11413 +11414 POINT(35.024145956728475 -117.76727843773521) bank11414 +11415 POINT(34.60734288045313 -118.97191944214067) bank11415 +11416 POINT(33.91574698453785 -117.57532786281502) bank11416 +11417 POINT(34.13612000906762 -119.11611926886346) bank11417 +11418 POINT(33.84017030763162 -117.98015956487397) bank11418 +11419 POINT(34.0988650444191 -118.19026847285558) bank11419 +11420 POINT(33.41344166651806 -119.21845251650169) bank11420 +11421 POINT(33.42263760791428 -117.98822647976698) bank11421 +11422 POINT(34.50240345240098 -118.83341583087814) bank11422 +11423 POINT(33.58805722138952 -117.74376826746753) bank11423 +11424 POINT(33.15883302729129 -118.19459951404237) bank11424 +11425 POINT(33.78134957690625 -118.51143907516119) bank11425 +11426 POINT(33.514463853609136 -118.62074702382307) bank11426 +11427 POINT(34.31774564385866 -118.09612226207955) bank11427 +11428 POINT(33.491054679994136 -117.41534809806949) bank11428 +11429 POINT(34.024284496831605 -117.78902894517009) bank11429 +11430 POINT(33.33911603333379 -118.99022007317356) bank11430 +11431 POINT(34.5429751556424 -118.12445212165508) bank11431 +11432 POINT(33.42702055102763 -118.06705867393444) bank11432 +11433 POINT(34.25820877314179 -117.25229694936742) bank11433 +11434 POINT(34.42778652456602 -119.17882247545788) bank11434 +11435 POINT(33.46196510859394 -118.30487159719792) bank11435 +11436 POINT(34.43292866845486 -118.66622797395561) bank11436 +11437 POINT(34.90614791630968 -118.80690384956176) bank11437 +11438 POINT(33.55365044611554 -118.05304323503242) bank11438 +11439 POINT(34.61593884555031 -117.90576216089994) bank11439 +11440 POINT(34.192626319222406 -117.61748918506024) bank11440 +11441 POINT(33.77911227014383 -117.8675504332527) bank11441 +11442 POINT(33.980772983661815 -118.57006330066172) bank11442 +11443 POINT(33.10965737815639 -118.44491601903681) bank11443 +11444 POINT(33.79438029475697 -118.406243537664) bank11444 +11445 POINT(33.70487856839225 -118.72798477445548) bank11445 +11446 POINT(33.75160930687139 -117.28570050320292) bank11446 +11447 POINT(34.16710423043108 -118.01642578732859) bank11447 +11448 POINT(34.580418420876285 -118.92990669087659) bank11448 +11449 POINT(33.37360111800554 -117.58103889913501) bank11449 +11450 POINT(34.76242032156392 -118.8109196245401) bank11450 +11451 POINT(33.40415636995188 -118.60065538449645) bank11451 +11452 POINT(33.32417467084808 -118.03735444850315) bank11452 +11453 POINT(33.77238005268106 -117.91687877421363) bank11453 +11454 POINT(33.33028218440063 -117.67818243638631) bank11454 +11455 POINT(33.372793045603494 -119.0233561229395) bank11455 +11456 POINT(34.94301693031383 -118.83629312575209) bank11456 +11457 POINT(33.54609083533994 -118.24927944103686) bank11457 +11458 POINT(33.513190851574855 -117.38204144297256) bank11458 +11459 POINT(34.26342242926928 -117.54836708172427) bank11459 +11460 POINT(34.46075149707834 -118.11949001079515) bank11460 +11461 POINT(34.81820960325014 -118.13597012564523) bank11461 +11462 POINT(34.76854248339282 -118.95481435765565) bank11462 +11463 POINT(33.75382174074819 -118.24001633907821) bank11463 +11464 POINT(33.754079137761195 -117.50681878791741) bank11464 +11465 POINT(34.91453100686181 -118.45439744116501) bank11465 +11466 POINT(33.60487040142768 -117.42568053135) bank11466 +11467 POINT(33.062623672982006 -118.56397162179206) bank11467 +11468 POINT(33.792360756703154 -118.61148766747249) bank11468 +11469 POINT(33.91784392372499 -118.79915395177845) bank11469 +11470 POINT(34.33681464099846 -118.9525349750201) bank11470 +11471 POINT(34.044426295367884 -119.19786380471363) bank11471 +11472 POINT(34.10488287817508 -117.28221006722852) bank11472 +11473 POINT(33.53129483510185 -119.17265205355899) bank11473 +11474 POINT(33.15872817985422 -119.15956607807706) bank11474 +11475 POINT(33.50329929455897 -118.94831213255553) bank11475 +11476 POINT(33.44411341124644 -118.44600999364309) bank11476 +11477 POINT(33.939387247319814 -118.21299637151097) bank11477 +11478 POINT(34.69495889816902 -117.57078432549523) bank11478 +11479 POINT(33.12722463094156 -118.44360451904747) bank11479 +11480 POINT(33.19684971537867 -118.03735588921532) bank11480 +11481 POINT(34.9318329961861 -118.2213822416296) bank11481 +11482 POINT(33.9004200861758 -118.10264394447397) bank11482 +11483 POINT(33.803151518485876 -118.53406444627527) bank11483 +11484 POINT(33.144047323187586 -118.80750189542755) bank11484 +11485 POINT(35.03165785925105 -118.92407525112279) bank11485 +11486 POINT(33.40089884263114 -118.13102054316529) bank11486 +11487 POINT(34.728564737435605 -117.32166296646349) bank11487 +11488 POINT(33.14167428675181 -117.6384482714444) bank11488 +11489 POINT(33.61834718315779 -118.85642760938636) bank11489 +11490 POINT(34.148467566205426 -117.83577142587988) bank11490 +11491 POINT(33.57681820426777 -118.4802215830057) bank11491 +11492 POINT(34.35886833085801 -118.99797198524725) bank11492 +11493 POINT(34.66622632015919 -118.79950486044558) bank11493 +11494 POINT(33.772492953080466 -118.21890264848425) bank11494 +11495 POINT(34.17059649950743 -117.84884282972178) bank11495 +11496 POINT(34.21519891295248 -118.05784124778803) bank11496 +11497 POINT(34.67249901580036 -118.73195487357545) bank11497 +11498 POINT(34.43990488466529 -118.7399429784893) bank11498 +11499 POINT(33.60721265974729 -119.08224158652327) bank11499 +11500 POINT(34.710000965391515 -118.70296335996402) bank11500 +11501 POINT(33.76230430402208 -118.14987568084119) bank11501 +11502 POINT(33.11883912539093 -117.79221735292028) bank11502 +11503 POINT(34.89260824851942 -117.88537652565371) bank11503 +11504 POINT(34.91723651745365 -117.80363284479132) bank11504 +11505 POINT(34.12411688264879 -117.98680025478326) bank11505 +11506 POINT(33.572453613188365 -117.8383858206042) bank11506 +11507 POINT(34.42804419156851 -118.01859460697437) bank11507 +11508 POINT(34.072922598629376 -118.27003291194369) bank11508 +11509 POINT(33.28949258454989 -117.8690483120036) bank11509 +11510 POINT(34.19868074544965 -117.63498470410347) bank11510 +11511 POINT(35.001237198676485 -118.70489101754235) bank11511 +11512 POINT(33.69126257957858 -118.10969001362157) bank11512 +11513 POINT(34.80765059560483 -117.6887512533107) bank11513 +11514 POINT(34.4280673517062 -118.10988988438207) bank11514 +11515 POINT(33.606071121267945 -118.67880040916174) bank11515 +11516 POINT(33.36010300562516 -118.96000731019181) bank11516 +11517 POINT(33.48242941874652 -118.1478066337622) bank11517 +11518 POINT(33.09156680280914 -117.3071279147202) bank11518 +11519 POINT(34.93214498876727 -118.23510831583866) bank11519 +11520 POINT(34.41806979222816 -118.20646040192699) bank11520 +11521 POINT(34.95711078783211 -118.99757828501131) bank11521 +11522 POINT(34.935473631927074 -118.15121756390641) bank11522 +11523 POINT(34.73245548976825 -117.99629092286192) bank11523 +11524 POINT(34.21108323338842 -117.89064622737185) bank11524 +11525 POINT(34.69551538944091 -117.45901024752365) bank11525 +11526 POINT(34.300877280262625 -117.57084379470948) bank11526 +11527 POINT(33.79452990923526 -117.86528321775926) bank11527 +11528 POINT(33.95055344396653 -118.67820297659878) bank11528 +11529 POINT(33.45141802022377 -117.7121753127217) bank11529 +11530 POINT(34.21454009017052 -118.30767012148661) bank11530 +11531 POINT(34.31597790660649 -117.86681783538087) bank11531 +11532 POINT(34.61640801323819 -117.94390065366711) bank11532 +11533 POINT(33.533241037084366 -118.6780984931944) bank11533 +11534 POINT(34.119246530089065 -118.2475804897759) bank11534 +11535 POINT(33.49464194317086 -118.82992934071054) bank11535 +11536 POINT(34.41247904586168 -118.56967063409812) bank11536 +11537 POINT(34.436832150674135 -117.99897233954208) bank11537 +11538 POINT(34.233578360418065 -119.14085092144853) bank11538 +11539 POINT(33.217469532890306 -117.55982739797237) bank11539 +11540 POINT(33.852422506889965 -119.08531003069035) bank11540 +11541 POINT(34.884595700698156 -118.3207909645946) bank11541 +11542 POINT(33.36842835963915 -117.80285695618895) bank11542 +11543 POINT(34.558642415439095 -117.7569272336227) bank11543 +11544 POINT(33.98356230408058 -118.44293823623046) bank11544 +11545 POINT(33.73673901223451 -117.74192717460005) bank11545 +11546 POINT(33.27995178050928 -117.9426490033818) bank11546 +11547 POINT(34.59148065235652 -118.23555114406213) bank11547 +11548 POINT(35.0256320178304 -118.07666827472929) bank11548 +11549 POINT(33.40583283819482 -117.73698772565449) bank11549 +11550 POINT(33.17835472770181 -118.4073990430903) bank11550 +11551 POINT(33.91195047945875 -118.92964831414352) bank11551 +11552 POINT(33.69339151520176 -119.13127130185161) bank11552 +11553 POINT(34.88670283016683 -117.62746726771769) bank11553 +11554 POINT(33.50708062560934 -118.08197218329857) bank11554 +11555 POINT(34.92828910751625 -118.47517345004647) bank11555 +11556 POINT(33.574972818267774 -117.89033863590475) bank11556 +11557 POINT(34.16276437091915 -118.65627840760287) bank11557 +11558 POINT(34.76978602885891 -117.46541789432632) bank11558 +11559 POINT(34.275019427684896 -119.05282739802814) bank11559 +11560 POINT(33.46355133336579 -118.96614795816984) bank11560 +11561 POINT(33.96493736118345 -118.09377527628563) bank11561 +11562 POINT(33.38765808079753 -118.66093701668021) bank11562 +11563 POINT(33.675187270482574 -118.58021515029814) bank11563 +11564 POINT(34.59867608177869 -118.89032847402248) bank11564 +11565 POINT(34.0777469869678 -118.31945578432594) bank11565 +11566 POINT(33.22259042979916 -117.37284640715717) bank11566 +11567 POINT(33.197493064494864 -118.51801396903365) bank11567 +11568 POINT(34.92896693336143 -117.61180654497217) bank11568 +11569 POINT(33.17642869297673 -117.6647790486504) bank11569 +11570 POINT(33.853895950883015 -118.63980544204638) bank11570 +11571 POINT(34.35125016566785 -118.6194097172951) bank11571 +11572 POINT(33.948805712036936 -118.53363274120846) bank11572 +11573 POINT(33.90138833994758 -117.83478027053407) bank11573 +11574 POINT(34.277789372067055 -118.89044244949878) bank11574 +11575 POINT(33.556936303385996 -117.57178835728953) bank11575 +11576 POINT(33.91460523192974 -119.0912603079782) bank11576 +11577 POINT(34.391137820331785 -118.5365230320681) bank11577 +11578 POINT(34.22466339921642 -117.89077265918354) bank11578 +11579 POINT(34.301297682009334 -117.64558681551671) bank11579 +11580 POINT(33.377936943717614 -118.60582414638793) bank11580 +11581 POINT(34.25139426146946 -119.0061329028107) bank11581 +11582 POINT(33.99214203820637 -117.25131526383754) bank11582 +11583 POINT(34.5034142126172 -117.9399012670278) bank11583 +11584 POINT(34.729780316686266 -117.96462491140976) bank11584 +11585 POINT(33.3915810257805 -118.61608203059525) bank11585 +11586 POINT(33.64365446800526 -117.74360614294183) bank11586 +11587 POINT(33.70611597695106 -117.26197797592452) bank11587 +11588 POINT(34.20961863207423 -119.12025441887816) bank11588 +11589 POINT(34.28609040788829 -118.08839485120788) bank11589 +11590 POINT(33.50755104318599 -118.85736580726031) bank11590 +11591 POINT(34.045587041142724 -117.63036340591331) bank11591 +11592 POINT(34.02390306775209 -117.86148581032464) bank11592 +11593 POINT(33.496851079341155 -118.45861916183618) bank11593 +11594 POINT(34.438215716432545 -118.67372409504786) bank11594 +11595 POINT(33.363777675842265 -118.39549241441212) bank11595 +11596 POINT(33.25368455534527 -117.75649076961864) bank11596 +11597 POINT(33.07381844269665 -118.1584987882066) bank11597 +11598 POINT(34.938665294264894 -117.3166009923463) bank11598 +11599 POINT(34.62092001944067 -118.79740673068545) bank11599 +11600 POINT(33.079503532129685 -117.64431909639426) bank11600 +11601 POINT(34.27006866463709 -117.30880335306576) bank11601 +11602 POINT(34.46192935301806 -118.79968455628914) bank11602 +11603 POINT(34.93109014801861 -117.71341342095253) bank11603 +11604 POINT(34.99934792333627 -117.53824349090179) bank11604 +11605 POINT(34.650907434244985 -118.98848927512897) bank11605 +11606 POINT(34.72759898705758 -117.5841308141116) bank11606 +11607 POINT(35.044212667504034 -117.51003731871567) bank11607 +11608 POINT(33.29821137360112 -118.94863879243978) bank11608 +11609 POINT(33.71202734038652 -118.30800122105484) bank11609 +11610 POINT(33.371377538867165 -118.65873258834256) bank11610 +11611 POINT(33.23215350839849 -117.41919037532703) bank11611 +11612 POINT(34.64285599249775 -117.6902209157012) bank11612 +11613 POINT(33.39967166594886 -118.28122752093454) bank11613 +11614 POINT(33.84533762454851 -119.1473828940506) bank11614 +11615 POINT(33.09997752498482 -117.35332665573323) bank11615 +11616 POINT(33.28177562809848 -119.01842235005402) bank11616 +11617 POINT(33.897100703325926 -118.54571318506584) bank11617 +11618 POINT(33.96965927646141 -117.30005093732609) bank11618 +11619 POINT(34.54049336852816 -118.44614619864541) bank11619 +11620 POINT(33.82348063730515 -117.84194872635932) bank11620 +11621 POINT(34.15540544072133 -118.41564570389829) bank11621 +11622 POINT(34.12250406270041 -117.35675452577031) bank11622 +11623 POINT(34.182634211890864 -118.88836411292944) bank11623 +11624 POINT(33.391685211654114 -118.87815263818047) bank11624 +11625 POINT(34.65558159756988 -118.57226885865322) bank11625 +11626 POINT(34.13826349205472 -117.74602760797478) bank11626 +11627 POINT(34.37702783257151 -118.15338947542507) bank11627 +11628 POINT(33.924191516965486 -118.89852061996977) bank11628 +11629 POINT(34.453486508682495 -118.69861515373564) bank11629 +11630 POINT(33.54934372909174 -117.66939702022128) bank11630 +11631 POINT(33.74831329279925 -118.31818273521309) bank11631 +11632 POINT(33.16698982478592 -118.38481822794418) bank11632 +11633 POINT(33.809566433402665 -119.18661633915661) bank11633 +11634 POINT(34.69327310680624 -118.11187062274628) bank11634 +11635 POINT(33.140596533077805 -117.43633353006072) bank11635 +11636 POINT(33.38692741276936 -117.58461447707859) bank11636 +11637 POINT(34.83076851995433 -117.42526168704397) bank11637 +11638 POINT(33.09219697505473 -119.14158487913299) bank11638 +11639 POINT(34.47497111412981 -118.0735709332762) bank11639 +11640 POINT(33.34350701517695 -118.17464085512626) bank11640 +11641 POINT(33.7759124243245 -117.96240455796055) bank11641 +11642 POINT(33.83811545558475 -117.51457606548482) bank11642 +11643 POINT(33.69569899900533 -118.46882429097549) bank11643 +11644 POINT(33.10378508031731 -118.85079129256262) bank11644 +11645 POINT(33.44244115582855 -118.15111712041892) bank11645 +11646 POINT(33.325397483666535 -117.99248721287984) bank11646 +11647 POINT(33.13476162153129 -118.60865045758587) bank11647 +11648 POINT(33.96927156875786 -117.27381864080246) bank11648 +11649 POINT(34.26156083425412 -119.03035495030693) bank11649 +11650 POINT(33.87782014262398 -118.94208522774534) bank11650 +11651 POINT(33.65257009027894 -118.36091926757638) bank11651 +11652 POINT(33.63496649490806 -119.1096660532017) bank11652 +11653 POINT(34.964022686669395 -118.3600575350612) bank11653 +11654 POINT(33.21483268530626 -118.85299891601859) bank11654 +11655 POINT(33.54822974995003 -118.8744961184994) bank11655 +11656 POINT(34.629053479256825 -117.68171461829684) bank11656 +11657 POINT(33.580107041265194 -119.12367706047266) bank11657 +11658 POINT(33.34506642244519 -119.17958086063197) bank11658 +11659 POINT(34.805536064127224 -118.86755886826222) bank11659 +11660 POINT(34.29069019955091 -119.0451734175578) bank11660 +11661 POINT(34.3129279415668 -118.18256270749043) bank11661 +11662 POINT(33.51460479381406 -117.40401434049572) bank11662 +11663 POINT(33.3684222366014 -117.4710616478518) bank11663 +11664 POINT(33.35406327978272 -117.31050292289254) bank11664 +11665 POINT(34.61239247963363 -118.73907967395185) bank11665 +11666 POINT(34.97545866570007 -118.50649480551907) bank11666 +11667 POINT(34.80258599629921 -118.75924287036669) bank11667 +11668 POINT(33.78326747189135 -118.22604081909954) bank11668 +11669 POINT(34.8443465186105 -118.99639451584278) bank11669 +11670 POINT(34.095952507547196 -117.24562461702682) bank11670 +11671 POINT(34.980402791969375 -118.61463620573247) bank11671 +11672 POINT(34.421395644375075 -118.38955885882487) bank11672 +11673 POINT(33.89521232452414 -117.89434188515366) bank11673 +11674 POINT(33.66366007127221 -117.75090725290964) bank11674 +11675 POINT(33.19870074341413 -117.6454333663363) bank11675 +11676 POINT(33.25743577142729 -117.30811292311887) bank11676 +11677 POINT(33.89352139843488 -118.80205487287786) bank11677 +11678 POINT(33.917841774107025 -118.90394485190936) bank11678 +11679 POINT(33.81827991440849 -118.00484025599644) bank11679 +11680 POINT(33.525234105843644 -118.54117166976069) bank11680 +11681 POINT(35.011443833901495 -118.08781710401121) bank11681 +11682 POINT(34.764373602146975 -117.36259474431698) bank11682 +11683 POINT(33.1353618326912 -118.80615816091716) bank11683 +11684 POINT(33.54053561589875 -118.19895049058778) bank11684 +11685 POINT(33.748472948896676 -117.29092524315926) bank11685 +11686 POINT(34.354979118854374 -118.15708882286584) bank11686 +11687 POINT(34.96915907726927 -118.8068427439877) bank11687 +11688 POINT(34.86790150688973 -117.37859587194713) bank11688 +11689 POINT(33.66695511654469 -118.48712768400252) bank11689 +11690 POINT(33.28851448396637 -118.23933965811162) bank11690 +11691 POINT(33.35648060291291 -117.88590176878748) bank11691 +11692 POINT(33.42203117327048 -118.19107114058932) bank11692 +11693 POINT(33.80710760797928 -118.71458891638) bank11693 +11694 POINT(33.802645271267856 -117.77497554376549) bank11694 +11695 POINT(33.32107540549625 -119.06564624755197) bank11695 +11696 POINT(33.44487887523625 -118.28187880815454) bank11696 +11697 POINT(33.56329130842299 -118.0025895358558) bank11697 +11698 POINT(34.97820531561829 -118.65397192759282) bank11698 +11699 POINT(33.250085160383186 -118.09319211572202) bank11699 +11700 POINT(33.895513094773804 -118.82539642294121) bank11700 +11701 POINT(33.71134526092698 -119.20737977899917) bank11701 +11702 POINT(33.900018834021644 -117.34483205130248) bank11702 +11703 POINT(34.277449628173635 -118.54605845650057) bank11703 +11704 POINT(33.688696197944275 -118.52992582703331) bank11704 +11705 POINT(33.60217511630176 -118.1696448338099) bank11705 +11706 POINT(33.990082813871304 -117.60860085273096) bank11706 +11707 POINT(34.22699097361275 -117.72247090013221) bank11707 +11708 POINT(34.49718155440595 -117.7324483559014) bank11708 +11709 POINT(33.51970104002055 -117.64454081245118) bank11709 +11710 POINT(33.70696460573257 -119.21820811289467) bank11710 +11711 POINT(34.18253850493442 -117.43820976561896) bank11711 +11712 POINT(33.53952945356884 -117.72416043096061) bank11712 +11713 POINT(34.67375333003848 -118.76872319203432) bank11713 +11714 POINT(33.28986788012654 -118.87234631734002) bank11714 +11715 POINT(33.46824520394975 -118.60307374534933) bank11715 +11716 POINT(33.35308990205816 -117.96412255227926) bank11716 +11717 POINT(33.682298115639675 -119.20998422020546) bank11717 +11718 POINT(34.43807017106066 -117.83980084673783) bank11718 +11719 POINT(34.86601385740315 -117.90896388270184) bank11719 +11720 POINT(34.16319995206822 -117.81037626755068) bank11720 +11721 POINT(33.41580657473112 -118.9855451561602) bank11721 +11722 POINT(34.66006402155675 -119.19706113691885) bank11722 +11723 POINT(34.26762198013384 -118.72558787243288) bank11723 +11724 POINT(33.424995725274215 -119.17115762340106) bank11724 +11725 POINT(33.10084935796911 -117.98400347771101) bank11725 +11726 POINT(33.25595831882024 -118.12498273161816) bank11726 +11727 POINT(34.83813577755687 -118.92012518741193) bank11727 +11728 POINT(33.0944827509995 -118.40630112532119) bank11728 +11729 POINT(35.03112504144508 -118.90530125984208) bank11729 +11730 POINT(33.85664683063547 -117.87054500992035) bank11730 +11731 POINT(33.64003742858177 -117.81047607418833) bank11731 +11732 POINT(33.85572157292404 -117.96731154432882) bank11732 +11733 POINT(34.44496438458564 -118.54579283232992) bank11733 +11734 POINT(34.66354404473418 -118.1173728991833) bank11734 +11735 POINT(34.31643612081876 -118.94665702259631) bank11735 +11736 POINT(34.49628657268698 -117.87713094699599) bank11736 +11737 POINT(33.386843656813994 -118.92694702245355) bank11737 +11738 POINT(34.74989571075328 -117.44693666079786) bank11738 +11739 POINT(33.95158600580648 -117.96624213971367) bank11739 +11740 POINT(34.77684534686793 -118.31132514618976) bank11740 +11741 POINT(34.22062286002882 -118.90931518941572) bank11741 +11742 POINT(33.95487201528756 -118.11889980669018) bank11742 +11743 POINT(33.7971305385127 -118.7286879273209) bank11743 +11744 POINT(33.167661424477195 -118.36570986537512) bank11744 +11745 POINT(33.183063072607105 -117.28346115042937) bank11745 +11746 POINT(34.485321010863736 -118.5378133265637) bank11746 +11747 POINT(33.35447338184306 -118.25025495366953) bank11747 +11748 POINT(34.18688907262058 -118.67120966552604) bank11748 +11749 POINT(34.02260604156395 -118.2413573981375) bank11749 +11750 POINT(34.23475480978237 -117.28308762717003) bank11750 +11751 POINT(33.707757914929275 -118.61392169461813) bank11751 +11752 POINT(34.85340282363469 -118.3249162646106) bank11752 +11753 POINT(34.35777086794975 -117.59655358356036) bank11753 +11754 POINT(34.40176389199386 -118.8792756587861) bank11754 +11755 POINT(33.44271638520287 -118.25986536873577) bank11755 +11756 POINT(34.55124013855609 -119.04544799221077) bank11756 +11757 POINT(33.57470908847793 -117.64079701112573) bank11757 +11758 POINT(34.8578033120015 -118.15478532675638) bank11758 +11759 POINT(34.303217849197246 -118.14385511198837) bank11759 +11760 POINT(33.26516068831101 -117.88489777040779) bank11760 +11761 POINT(33.464760696655965 -118.13173294353919) bank11761 +11762 POINT(34.23020364800112 -118.62775326250119) bank11762 +11763 POINT(34.34379718283134 -117.2480237429653) bank11763 +11764 POINT(34.61891177533793 -117.491335598502) bank11764 +11765 POINT(34.418061542614474 -119.04621811103858) bank11765 +11766 POINT(33.16447309981591 -117.88161328645174) bank11766 +11767 POINT(35.050439528977215 -118.45461005698913) bank11767 +11768 POINT(34.19845424767237 -119.13997614182044) bank11768 +11769 POINT(35.04415571636272 -118.44405362208713) bank11769 +11770 POINT(33.80993452213221 -117.64089265269865) bank11770 +11771 POINT(33.25333499926388 -117.60572879915335) bank11771 +11772 POINT(34.32178285711198 -117.52853664092939) bank11772 +11773 POINT(35.029710301729715 -118.6411102627203) bank11773 +11774 POINT(34.513951666370104 -118.71257173163505) bank11774 +11775 POINT(33.138390655714915 -117.73933093475044) bank11775 +11776 POINT(33.97518860043557 -118.28829881424622) bank11776 +11777 POINT(34.75271217514671 -118.76739539292291) bank11777 +11778 POINT(34.10497711481855 -117.43914971934718) bank11778 +11779 POINT(35.010210536321736 -118.73837212921744) bank11779 +11780 POINT(34.77646618767582 -118.26209218849156) bank11780 +11781 POINT(34.91914080874915 -117.48402877419112) bank11781 +11782 POINT(33.61967828168073 -118.4190867563253) bank11782 +11783 POINT(34.78605566072645 -118.83401450619408) bank11783 +11784 POINT(34.71379467938904 -119.00606377917565) bank11784 +11785 POINT(34.31109778896936 -117.6498315507819) bank11785 +11786 POINT(33.165431405138094 -117.84017077736388) bank11786 +11787 POINT(33.247951541539955 -118.8136204504841) bank11787 +11788 POINT(33.26341523110912 -117.97948234171221) bank11788 +11789 POINT(34.68345820072956 -118.70378239203404) bank11789 +11790 POINT(34.74443077555563 -118.78690532432135) bank11790 +11791 POINT(34.96646455313306 -117.98999126449371) bank11791 +11792 POINT(34.178653670225756 -117.55489866161638) bank11792 +11793 POINT(33.759103926833355 -117.72504400894637) bank11793 +11794 POINT(33.946333719964294 -118.44389071874038) bank11794 +11795 POINT(33.62459003170286 -117.8163093364498) bank11795 +11796 POINT(34.4028803605912 -118.24357623731325) bank11796 +11797 POINT(34.50426342315997 -118.34322933048544) bank11797 +11798 POINT(34.66572648159475 -117.36477589774319) bank11798 +11799 POINT(33.36958924595648 -118.74791398299313) bank11799 +11800 POINT(34.99899118434558 -118.8251086781038) bank11800 +11801 POINT(34.37175032514916 -119.04701100328704) bank11801 +11802 POINT(34.35978463767201 -118.72480573219097) bank11802 +11803 POINT(34.93037402866641 -118.64500915286992) bank11803 +11804 POINT(33.65526676130909 -117.78525436149043) bank11804 +11805 POINT(33.71375762735038 -118.35592462119054) bank11805 +11806 POINT(33.47089672825257 -118.17832226219467) bank11806 +11807 POINT(33.85815697099298 -117.8518628402383) bank11807 +11808 POINT(33.7559982072512 -118.28387022493861) bank11808 +11809 POINT(34.98822729584562 -117.35345769431328) bank11809 +11810 POINT(34.519806935833294 -118.78569811214622) bank11810 +11811 POINT(34.53922387549615 -118.23858999498361) bank11811 +11812 POINT(34.26602100233512 -119.02421233695603) bank11812 +11813 POINT(34.88888670818316 -117.87560050557863) bank11813 +11814 POINT(33.438657070965526 -117.4571709921162) bank11814 +11815 POINT(34.31737933416859 -119.17061080861339) bank11815 +11816 POINT(34.844040455967615 -118.49143682531289) bank11816 +11817 POINT(34.52542520975219 -118.31793294136133) bank11817 +11818 POINT(34.83591580198928 -118.0393056897458) bank11818 +11819 POINT(33.12533560062547 -117.27374060221135) bank11819 +11820 POINT(33.379382467047485 -118.06351146875997) bank11820 +11821 POINT(34.814247951725406 -118.91396758788301) bank11821 +11822 POINT(34.287641456865245 -118.78541416145625) bank11822 +11823 POINT(34.72607531560237 -117.28307288804679) bank11823 +11824 POINT(34.22436305230203 -118.93749602403355) bank11824 +11825 POINT(34.175467450426865 -118.60266095607975) bank11825 +11826 POINT(33.76078372397075 -118.76992354113824) bank11826 +11827 POINT(33.822575357713426 -117.2450642415174) bank11827 +11828 POINT(34.973531407315335 -117.35798968904898) bank11828 +11829 POINT(34.20967532269965 -118.33047934159906) bank11829 +11830 POINT(34.938776892061036 -117.25613618685243) bank11830 +11831 POINT(33.15280705573157 -118.05740426306676) bank11831 +11832 POINT(33.30465746483269 -118.74787472533897) bank11832 +11833 POINT(33.53530824780509 -118.4164447015432) bank11833 +11834 POINT(33.88951413966439 -119.02153802129175) bank11834 +11835 POINT(34.657235538466004 -117.96759992960625) bank11835 +11836 POINT(34.94193255919506 -117.28113359637963) bank11836 +11837 POINT(33.74348714105558 -119.00062164817153) bank11837 +11838 POINT(33.98274250889287 -118.12843517275662) bank11838 +11839 POINT(33.18823198881635 -117.71761020732232) bank11839 +11840 POINT(35.03041683450547 -118.27309516962026) bank11840 +11841 POINT(33.655057711918715 -117.81589511036329) bank11841 +11842 POINT(34.557439460830004 -119.18690522484751) bank11842 +11843 POINT(33.604470205198915 -117.94983634352818) bank11843 +11844 POINT(34.89390368158602 -118.48105927165508) bank11844 +11845 POINT(34.951326269001136 -118.67989606417312) bank11845 +11846 POINT(34.31758098521409 -117.5558338644782) bank11846 +11847 POINT(33.52808553906921 -118.09421740819168) bank11847 +11848 POINT(33.80632391169027 -118.75360345016718) bank11848 +11849 POINT(33.23228548515811 -117.45653373684556) bank11849 +11850 POINT(33.91270513564314 -117.45937956861471) bank11850 +11851 POINT(34.087269696004306 -118.16569105946706) bank11851 +11852 POINT(34.12293404148519 -117.82560749038689) bank11852 +11853 POINT(33.48846079761984 -118.96037441554985) bank11853 +11854 POINT(33.931553123247 -118.32180780876199) bank11854 +11855 POINT(34.27936872586416 -117.36623970628675) bank11855 +11856 POINT(33.451309154644754 -117.99504577470181) bank11856 +11857 POINT(34.43321311234682 -118.4262765620591) bank11857 +11858 POINT(34.72685637898818 -117.31380847429564) bank11858 +11859 POINT(34.28090700371516 -117.35191459679938) bank11859 +11860 POINT(34.56521845882704 -117.41306593107029) bank11860 +11861 POINT(34.61768051680404 -117.64785620031041) bank11861 +11862 POINT(33.2826115494522 -117.26189201979001) bank11862 +11863 POINT(33.5331298268179 -117.55085590477039) bank11863 +11864 POINT(34.879320967124464 -117.99836722132147) bank11864 +11865 POINT(34.51219364634827 -118.5548553014068) bank11865 +11866 POINT(34.03080614611698 -119.1533485520265) bank11866 +11867 POINT(33.4362122732229 -117.84098706118735) bank11867 +11868 POINT(34.554640380412 -118.1031444715463) bank11868 +11869 POINT(34.30969329500389 -117.59828576238301) bank11869 +11870 POINT(35.04530528175806 -118.67462196370907) bank11870 +11871 POINT(33.602743394504266 -117.24705522838748) bank11871 +11872 POINT(33.315239393129225 -117.50725309877151) bank11872 +11873 POINT(33.942457873302324 -118.5468198650278) bank11873 +11874 POINT(33.88037066337211 -118.67547655257597) bank11874 +11875 POINT(34.977247195954234 -118.00360467699223) bank11875 +11876 POINT(33.7430977564941 -119.22222613920161) bank11876 +11877 POINT(34.99853207094347 -117.84246164339575) bank11877 +11878 POINT(34.282978207653734 -117.58917801776433) bank11878 +11879 POINT(33.08697583777995 -118.28142607512028) bank11879 +11880 POINT(34.19977464354165 -119.10423849981002) bank11880 +11881 POINT(34.38318736102732 -117.25587639577232) bank11881 +11882 POINT(33.38538381034 -118.63031065857032) bank11882 +11883 POINT(34.788673334845875 -119.23061437443762) bank11883 +11884 POINT(34.65111870492184 -118.20296220469011) bank11884 +11885 POINT(34.14806689504739 -119.03617969939752) bank11885 +11886 POINT(33.99338174345678 -118.82171926329676) bank11886 +11887 POINT(33.191405657988994 -117.79029398052711) bank11887 +11888 POINT(34.898221495565764 -118.84513109374497) bank11888 +11889 POINT(33.80453408821424 -118.83525385546655) bank11889 +11890 POINT(34.29988917864117 -117.68109394572853) bank11890 +11891 POINT(34.38990790477663 -119.09348703932507) bank11891 +11892 POINT(34.65130221159123 -117.45992454647903) bank11892 +11893 POINT(33.956814326909324 -118.51874253962812) bank11893 +11894 POINT(34.11952670756078 -118.92805469995882) bank11894 +11895 POINT(34.106889092782716 -117.52578383222865) bank11895 +11896 POINT(34.982327876841396 -118.45052588759489) bank11896 +11897 POINT(33.67893676067203 -117.66392648364753) bank11897 +11898 POINT(34.59807336940853 -118.59310143930497) bank11898 +11899 POINT(33.25431912415384 -117.60775491926002) bank11899 +11900 POINT(33.1069456733495 -118.22128113112133) bank11900 +11901 POINT(34.61089090599758 -117.8565514411297) bank11901 +11902 POINT(34.494100377530934 -118.09452755937693) bank11902 +11903 POINT(34.928841028176606 -117.65532318635266) bank11903 +11904 POINT(35.007530482116245 -118.28911949814344) bank11904 +11905 POINT(33.602424656790916 -118.42498369684373) bank11905 +11906 POINT(34.007427165451496 -118.66539485143933) bank11906 +11907 POINT(34.489678931433424 -118.02657954125208) bank11907 +11908 POINT(33.545449654488166 -117.35509243557183) bank11908 +11909 POINT(34.87884003719953 -117.9297066464607) bank11909 +11910 POINT(33.66756297581116 -118.89735912827484) bank11910 +11911 POINT(33.70226354817828 -118.3573981713528) bank11911 +11912 POINT(34.06007323682143 -118.43097418132048) bank11912 +11913 POINT(33.92331276904883 -118.29428185011392) bank11913 +11914 POINT(33.567967282289636 -118.13861072763545) bank11914 +11915 POINT(34.25453780577805 -118.80677737117459) bank11915 +11916 POINT(33.921941617920375 -118.95162652274921) bank11916 +11917 POINT(33.697807149871345 -118.02596685301876) bank11917 +11918 POINT(33.48477507937577 -118.43427696967294) bank11918 +11919 POINT(33.396457782565186 -119.0699466148501) bank11919 +11920 POINT(33.94171362196565 -117.7899216592911) bank11920 +11921 POINT(34.06389111513319 -119.16255519622754) bank11921 +11922 POINT(33.10440613670292 -118.82374887465456) bank11922 +11923 POINT(33.63715857637775 -117.28905161248827) bank11923 +11924 POINT(33.577828097305755 -118.69432736028145) bank11924 +11925 POINT(33.89051217269803 -118.94700426983394) bank11925 +11926 POINT(34.81408028336396 -119.16566519627656) bank11926 +11927 POINT(33.915816274193844 -118.21951616186561) bank11927 +11928 POINT(34.03816796784649 -118.59324345343961) bank11928 +11929 POINT(33.82782328057839 -118.45125290645684) bank11929 +11930 POINT(34.752855964209466 -117.46275127793274) bank11930 +11931 POINT(33.68725672484081 -119.10468608869859) bank11931 +11932 POINT(33.50895005196878 -117.83339738230244) bank11932 +11933 POINT(35.012682704939984 -118.15966922870355) bank11933 +11934 POINT(33.7784758061587 -117.72685919350621) bank11934 +11935 POINT(33.34767391233436 -118.06854011414049) bank11935 +11936 POINT(34.86747619996486 -118.74271363590012) bank11936 +11937 POINT(34.36745750065306 -118.60779427448932) bank11937 +11938 POINT(33.061974520222485 -118.85069557907758) bank11938 +11939 POINT(33.20968163341003 -117.25616163404133) bank11939 +11940 POINT(33.839332711981854 -118.97688759197455) bank11940 +11941 POINT(34.64691177128744 -118.87045098656466) bank11941 +11942 POINT(34.937721541368155 -118.97767565679452) bank11942 +11943 POINT(33.531195923997295 -118.41535529739517) bank11943 +11944 POINT(34.92032196778647 -118.82542817389587) bank11944 +11945 POINT(34.14617553435776 -117.73315292194955) bank11945 +11946 POINT(34.27737334965261 -117.57765454441946) bank11946 +11947 POINT(33.0983941160795 -118.76529021144306) bank11947 +11948 POINT(35.00143666031156 -118.84061229746568) bank11948 +11949 POINT(34.92940051955804 -119.22963758836833) bank11949 +11950 POINT(34.411266593079624 -117.47948656068723) bank11950 +11951 POINT(33.32968954200703 -118.17345350970623) bank11951 +11952 POINT(33.60064656683886 -118.37490741875813) bank11952 +11953 POINT(34.9333412338963 -117.28647576604932) bank11953 +11954 POINT(33.48522054721451 -118.34172417975235) bank11954 +11955 POINT(33.68850461958117 -118.48094000544367) bank11955 +11956 POINT(34.191751571195574 -118.32640673809027) bank11956 +11957 POINT(34.17452829794496 -117.76417099846465) bank11957 +11958 POINT(33.512071745882174 -117.52942820375813) bank11958 +11959 POINT(33.6520751530607 -118.82140427702005) bank11959 +11960 POINT(34.516775436185846 -117.8979315015344) bank11960 +11961 POINT(34.272685538022905 -118.6930611907548) bank11961 +11962 POINT(34.32554644857723 -118.83414152264692) bank11962 +11963 POINT(33.67670676118152 -117.7293131762474) bank11963 +11964 POINT(33.806522743134195 -117.77788558703116) bank11964 +11965 POINT(33.95832393694502 -118.23432852776857) bank11965 +11966 POINT(33.17377934530912 -118.66401840757975) bank11966 +11967 POINT(33.6492545146549 -118.99166576527237) bank11967 +11968 POINT(33.20279815094052 -117.96417286094238) bank11968 +11969 POINT(34.5958069387249 -117.5919476819274) bank11969 +11970 POINT(33.098261414529965 -118.09440540300953) bank11970 +11971 POINT(34.15038171912275 -119.23338019339269) bank11971 +11972 POINT(33.80459912030242 -117.3786961006914) bank11972 +11973 POINT(34.86020076720511 -117.55638300655811) bank11973 +11974 POINT(34.06076317227178 -118.04866666682796) bank11974 +11975 POINT(33.717280373828174 -118.35637394268575) bank11975 +11976 POINT(33.98484170984706 -118.0973484784551) bank11976 +11977 POINT(33.84671668570384 -118.2656759842168) bank11977 +11978 POINT(34.56745067970632 -119.1256318450324) bank11978 +11979 POINT(33.951993362049656 -117.44322682132722) bank11979 +11980 POINT(33.74749731230909 -118.44233832526263) bank11980 +11981 POINT(34.84368034645837 -117.96110390557875) bank11981 +11982 POINT(34.631045496709135 -118.5988672148964) bank11982 +11983 POINT(33.68276429867591 -117.8891700445055) bank11983 +11984 POINT(34.081196811884055 -118.95147833768596) bank11984 +11985 POINT(33.52530744640651 -118.66268956718842) bank11985 +11986 POINT(33.645782102641185 -119.02680238963316) bank11986 +11987 POINT(34.650449345228495 -117.34346899615531) bank11987 +11988 POINT(33.76237493199551 -119.01960930777955) bank11988 +11989 POINT(34.44443495594779 -119.20625893354638) bank11989 +11990 POINT(34.77352116757863 -118.51260895292575) bank11990 +11991 POINT(34.779942020898154 -117.8373280910425) bank11991 +11992 POINT(34.429579806324114 -118.27505258049563) bank11992 +11993 POINT(34.8815398299611 -117.42349976303876) bank11993 +11994 POINT(34.777480281859795 -118.87593283128545) bank11994 +11995 POINT(33.74557419283438 -118.26361348793662) bank11995 +11996 POINT(33.72872940925354 -117.72520546366903) bank11996 +11997 POINT(34.72673378516099 -118.08025968785141) bank11997 +11998 POINT(34.028084976056945 -117.25754083149305) bank11998 +11999 POINT(34.53304326258854 -118.99425564172641) bank11999 +12000 POINT(33.82341258296349 -118.33395729408944) bank12000 +12001 POINT(34.64064547871075 -117.6961755428313) bank12001 +12002 POINT(34.60322544146776 -119.07110620934913) bank12002 +12003 POINT(33.331151404820226 -118.7419881769766) bank12003 +12004 POINT(35.04875516511093 -118.0809431339895) bank12004 +12005 POINT(34.82810114708359 -118.86372126502945) bank12005 +12006 POINT(34.801154960723224 -117.38015550790956) bank12006 +12007 POINT(34.405212940917025 -118.90556433720893) bank12007 +12008 POINT(33.05226780824182 -117.67185939771785) bank12008 +12009 POINT(34.81055622287452 -118.70797009722845) bank12009 +12010 POINT(34.407417155729064 -117.3191921418554) bank12010 +12011 POINT(33.8199930501453 -118.6318392303436) bank12011 +12012 POINT(34.002857988920354 -118.67454805276844) bank12012 +12013 POINT(34.89291881406169 -117.78070106618135) bank12013 +12014 POINT(34.787231330223705 -118.08825601884242) bank12014 +12015 POINT(33.8375773052508 -117.77405488699355) bank12015 +12016 POINT(34.2091942280988 -118.43917093312234) bank12016 +12017 POINT(33.672727160226145 -118.2557977220019) bank12017 +12018 POINT(33.43874644869092 -118.6689487929715) bank12018 +12019 POINT(33.81768319835441 -118.56941570053068) bank12019 +12020 POINT(34.77989552849021 -118.04669095445134) bank12020 +12021 POINT(34.2367426905661 -117.94201938657953) bank12021 +12022 POINT(33.79972213506587 -118.75108184149067) bank12022 +12023 POINT(34.47675857807062 -118.51934708171731) bank12023 +12024 POINT(33.27670034559501 -119.04206868235362) bank12024 +12025 POINT(34.98012305615414 -118.57532993444318) bank12025 +12026 POINT(33.8452320010949 -117.58029834615236) bank12026 +12027 POINT(34.44347108194073 -117.79207385131335) bank12027 +12028 POINT(34.22594785360873 -119.01084957026197) bank12028 +12029 POINT(33.7538676044393 -118.81346390633708) bank12029 +12030 POINT(34.3401010297014 -118.223323183873) bank12030 +12031 POINT(34.98456720538439 -119.00732473067364) bank12031 +12032 POINT(33.43125056141069 -117.60323901633696) bank12032 +12033 POINT(33.49330455445727 -117.93340005973103) bank12033 +12034 POINT(34.79712417318868 -117.4881707649408) bank12034 +12035 POINT(34.304092530416625 -118.71749925090633) bank12035 +12036 POINT(33.28445286061771 -118.57356516931397) bank12036 +12037 POINT(34.09870249941389 -117.56090681844648) bank12037 +12038 POINT(34.70327476113673 -118.66072632753198) bank12038 +12039 POINT(33.12333225333034 -117.51558419120634) bank12039 +12040 POINT(34.02422773724244 -118.81949250500234) bank12040 +12041 POINT(34.0602655172523 -117.33262591606454) bank12041 +12042 POINT(34.5103583344306 -118.22184454568159) bank12042 +12043 POINT(33.95611604938321 -117.76131212145397) bank12043 +12044 POINT(35.006091229932906 -118.22267348374581) bank12044 +12045 POINT(34.17489923430624 -118.91006002383413) bank12045 +12046 POINT(34.86217078450648 -117.43650952943023) bank12046 +12047 POINT(33.520607913393576 -117.94598294252346) bank12047 +12048 POINT(33.107691342403776 -118.60008802456406) bank12048 +12049 POINT(33.310722037299094 -117.62286614821286) bank12049 +12050 POINT(35.035710292625836 -117.3515092828602) bank12050 +12051 POINT(34.55663565632309 -119.1909564491011) bank12051 +12052 POINT(34.693750537545895 -117.8255858370682) bank12052 +12053 POINT(34.925986857308885 -118.05948454721229) bank12053 +12054 POINT(33.342533752885565 -117.77409150785473) bank12054 +12055 POINT(33.15326353760634 -118.70496165052084) bank12055 +12056 POINT(33.87474511204137 -117.90187127527876) bank12056 +12057 POINT(33.2203289334469 -118.78080000155785) bank12057 +12058 POINT(33.38005554244761 -117.6707824658264) bank12058 +12059 POINT(34.980362312116355 -117.4757573127473) bank12059 +12060 POINT(34.24136090828757 -118.6602076136938) bank12060 +12061 POINT(33.93435558631694 -117.59726327525118) bank12061 +12062 POINT(33.3704660673623 -117.86991935643222) bank12062 +12063 POINT(33.53224703862076 -118.84235005091438) bank12063 +12064 POINT(33.822850575292854 -118.24020276215289) bank12064 +12065 POINT(33.83265168266494 -117.4846695805303) bank12065 +12066 POINT(33.75548897973407 -117.85322312910908) bank12066 +12067 POINT(33.75208142209325 -117.64694013938232) bank12067 +12068 POINT(34.356483918237636 -117.26690421463667) bank12068 +12069 POINT(34.29400110407694 -118.13655113485596) bank12069 +12070 POINT(34.818055828338736 -118.69169789926826) bank12070 +12071 POINT(34.29265583910478 -118.24153421852726) bank12071 +12072 POINT(34.479134255168766 -118.0015384292159) bank12072 +12073 POINT(34.67619342081888 -119.18758904610317) bank12073 +12074 POINT(33.59195845627077 -118.88026220612024) bank12074 +12075 POINT(33.12317837828509 -118.59243543892488) bank12075 +12076 POINT(33.288700730334305 -119.01037832501115) bank12076 +12077 POINT(34.88270458864629 -118.57827799943087) bank12077 +12078 POINT(34.46284883854433 -117.84990482097948) bank12078 +12079 POINT(34.573857926702 -119.09055713885715) bank12079 +12080 POINT(33.7368692856485 -119.09385808847367) bank12080 +12081 POINT(33.08899719580207 -117.85116281685126) bank12081 +12082 POINT(34.25109740453486 -118.6789451533716) bank12082 +12083 POINT(33.125378534308524 -118.34649436535499) bank12083 +12084 POINT(34.73748300597378 -117.39977063673635) bank12084 +12085 POINT(33.60968131561841 -119.05745314211812) bank12085 +12086 POINT(34.05215765310497 -117.37601413180202) bank12086 +12087 POINT(33.91074300125668 -117.93212230586484) bank12087 +12088 POINT(33.1559925294716 -118.42367569378717) bank12088 +12089 POINT(34.36627761559287 -117.70222189143432) bank12089 +12090 POINT(33.42122346561625 -117.33728966080622) bank12090 +12091 POINT(34.28576586650738 -117.41352723964651) bank12091 +12092 POINT(34.793508291242674 -119.0888076007061) bank12092 +12093 POINT(34.48102762044926 -118.63447388686455) bank12093 +12094 POINT(33.7358055751926 -119.23353004164431) bank12094 +12095 POINT(33.97969928943511 -117.7335508322862) bank12095 +12096 POINT(33.34715372360378 -117.31585794835571) bank12096 +12097 POINT(33.85355293654577 -118.82985041734554) bank12097 +12098 POINT(33.80130483806162 -118.85239367490749) bank12098 +12099 POINT(33.31937002597197 -118.72467463939772) bank12099 +12100 POINT(33.352164724387045 -118.68466242712601) bank12100 +12101 POINT(34.7917894803027 -118.05732448636174) bank12101 +12102 POINT(34.55346295664275 -119.22497385455053) bank12102 +12103 POINT(34.74784313694263 -118.06059768285917) bank12103 +12104 POINT(34.31600680601103 -117.48794426924678) bank12104 +12105 POINT(33.858302337067514 -118.15069592197895) bank12105 +12106 POINT(34.05965760678558 -118.35038222617644) bank12106 +12107 POINT(33.19554338617538 -118.24658775022544) bank12107 +12108 POINT(34.392702389087844 -117.28798008261973) bank12108 +12109 POINT(33.265406017041734 -118.18720689103104) bank12109 +12110 POINT(33.6070036517482 -118.86831353537005) bank12110 +12111 POINT(34.93681264478424 -117.4591091586903) bank12111 +12112 POINT(34.581472852874015 -118.57552895951095) bank12112 +12113 POINT(34.38839566895413 -118.73276029820937) bank12113 +12114 POINT(33.879596690502844 -118.80615965472693) bank12114 +12115 POINT(34.15877198946874 -118.66070087882937) bank12115 +12116 POINT(33.628082654064634 -117.25520551060443) bank12116 +12117 POINT(34.76853990516184 -117.42749407104115) bank12117 +12118 POINT(34.60160579957755 -117.96402720198292) bank12118 +12119 POINT(33.536661578519755 -118.42379979605691) bank12119 +12120 POINT(33.46058436081943 -117.73074825107426) bank12120 +12121 POINT(33.18737791846462 -118.26090275979301) bank12121 +12122 POINT(34.14903048549008 -117.56883601225672) bank12122 +12123 POINT(33.364779576544635 -117.48374027896311) bank12123 +12124 POINT(33.587354132666555 -119.10539576206892) bank12124 +12125 POINT(33.16367562369688 -118.55971017234562) bank12125 +12126 POINT(34.27326646992241 -117.8859815841577) bank12126 +12127 POINT(33.66856598706013 -117.59922652050199) bank12127 +12128 POINT(34.2189584812718 -118.08662178126802) bank12128 +12129 POINT(33.69270380587816 -118.11473821611166) bank12129 +12130 POINT(34.810930955750926 -117.8589727642223) bank12130 +12131 POINT(33.17796803846719 -117.92933955253665) bank12131 +12132 POINT(33.7571223818121 -117.77789147334464) bank12132 +12133 POINT(34.288793273520334 -118.3522663971484) bank12133 +12134 POINT(34.16287460987529 -117.78072841004396) bank12134 +12135 POINT(34.79687833338028 -118.68164650480398) bank12135 +12136 POINT(34.950883435388704 -118.35896943256516) bank12136 +12137 POINT(33.83566037975735 -117.86072160502218) bank12137 +12138 POINT(33.6396918215952 -117.64852835158254) bank12138 +12139 POINT(33.93665015288174 -117.66246332414795) bank12139 +12140 POINT(34.26932141455757 -118.90084883631707) bank12140 +12141 POINT(33.873131484037046 -118.78205367392712) bank12141 +12142 POINT(34.43729073687786 -117.68030805822585) bank12142 +12143 POINT(34.99299211261381 -118.45064858437365) bank12143 +12144 POINT(33.357285507460276 -118.6089128806359) bank12144 +12145 POINT(34.453042771191875 -118.01277743882066) bank12145 +12146 POINT(34.8195729157149 -118.76243365004285) bank12146 +12147 POINT(34.31797419110001 -117.58857705553146) bank12147 +12148 POINT(33.63988799937316 -118.32316973352025) bank12148 +12149 POINT(34.13369588333609 -117.45568710415886) bank12149 +12150 POINT(34.47908616637452 -118.68553539251585) bank12150 +12151 POINT(33.343286142001034 -118.52038393131251) bank12151 +12152 POINT(33.531077909574044 -118.52986874180952) bank12152 +12153 POINT(34.23947535271111 -117.83584263543749) bank12153 +12154 POINT(34.475388969587655 -119.07276077404853) bank12154 +12155 POINT(33.63809097702434 -118.90886189293113) bank12155 +12156 POINT(33.911051757475605 -118.4158591576711) bank12156 +12157 POINT(33.08658398304008 -117.726539304109) bank12157 +12158 POINT(33.722652511165656 -117.39204806676175) bank12158 +12159 POINT(34.95850464486416 -118.34335742417107) bank12159 +12160 POINT(33.31323668819661 -117.57723453184217) bank12160 +12161 POINT(34.97801101362044 -119.20456689034437) bank12161 +12162 POINT(33.675729382630436 -118.21452022457706) bank12162 +12163 POINT(34.956731692676385 -119.12609533425574) bank12163 +12164 POINT(33.80853773871769 -118.0503935677205) bank12164 +12165 POINT(34.31977020035967 -118.04181458260368) bank12165 +12166 POINT(34.30280225300748 -117.78221148592428) bank12166 +12167 POINT(34.46734300079153 -117.80418554652782) bank12167 +12168 POINT(33.84332236420412 -117.98099329595146) bank12168 +12169 POINT(34.074729946645064 -119.14490315249682) bank12169 +12170 POINT(34.66416789300482 -118.02810015351676) bank12170 +12171 POINT(34.463582246406766 -118.15796402468735) bank12171 +12172 POINT(33.94568596599331 -117.79938962516783) bank12172 +12173 POINT(34.27183854911599 -118.99589629482948) bank12173 +12174 POINT(34.810294362349325 -117.93224326314417) bank12174 +12175 POINT(33.45827612394057 -118.12061423945971) bank12175 +12176 POINT(34.71549201425237 -118.76412530456923) bank12176 +12177 POINT(34.81276223758892 -119.20082859445108) bank12177 +12178 POINT(34.337927723635495 -118.08092166767763) bank12178 +12179 POINT(34.24084073513477 -118.9627042950785) bank12179 +12180 POINT(33.145338382357444 -118.94728886176524) bank12180 +12181 POINT(33.35279599765693 -118.36468894604452) bank12181 +12182 POINT(34.23087455112933 -119.16289939010132) bank12182 +12183 POINT(33.07732985332415 -117.70369569847826) bank12183 +12184 POINT(34.43444928066468 -117.60274711275375) bank12184 +12185 POINT(34.16307633175366 -117.88128811385116) bank12185 +12186 POINT(33.978133128295674 -119.14296770783004) bank12186 +12187 POINT(33.12728819465086 -118.97952320939149) bank12187 +12188 POINT(35.00089221171973 -118.23399707110475) bank12188 +12189 POINT(34.024498717852914 -119.2008800041872) bank12189 +12190 POINT(33.8532750626952 -117.41734944658609) bank12190 +12191 POINT(33.264211584286144 -118.13909157077548) bank12191 +12192 POINT(33.29623230884775 -119.12632477052622) bank12192 +12193 POINT(33.87141901327217 -118.16521768444721) bank12193 +12194 POINT(33.21533878581801 -117.42240154584178) bank12194 +12195 POINT(33.7062365948177 -117.66294382258752) bank12195 +12196 POINT(34.01715437471792 -118.87666538627712) bank12196 +12197 POINT(34.9262394203602 -118.72874568813995) bank12197 +12198 POINT(34.4152943880096 -118.28344028637369) bank12198 +12199 POINT(34.54117791354281 -118.08655898988785) bank12199 +12200 POINT(34.717796734288434 -117.71208222313761) bank12200 +12201 POINT(34.688627290382506 -117.91339495782097) bank12201 +12202 POINT(34.36310408896929 -117.96456442020302) bank12202 +12203 POINT(34.65940009574423 -117.74323846122684) bank12203 +12204 POINT(34.14700841373528 -118.79937828951664) bank12204 +12205 POINT(34.41091277646075 -118.78327995753924) bank12205 +12206 POINT(34.53435277251077 -118.53024535247181) bank12206 +12207 POINT(33.47019679884695 -117.59955515542518) bank12207 +12208 POINT(33.31362941120404 -118.28364059103664) bank12208 +12209 POINT(34.932358820246584 -118.51715658262195) bank12209 +12210 POINT(33.907260465576734 -119.23491974776047) bank12210 +12211 POINT(34.090180743028746 -118.4155561427463) bank12211 +12212 POINT(34.34281931518983 -118.11188645969577) bank12212 +12213 POINT(33.39426039431403 -118.58479776547841) bank12213 +12214 POINT(33.07396195691 -118.84904396405626) bank12214 +12215 POINT(34.41174498048403 -118.17023728251954) bank12215 +12216 POINT(34.361696321239 -118.7553256664283) bank12216 +12217 POINT(34.75940128781597 -118.60911302615705) bank12217 +12218 POINT(33.514352866899635 -117.69412026659111) bank12218 +12219 POINT(34.945599471588956 -117.53909552223323) bank12219 +12220 POINT(33.094338659147624 -117.95034988963513) bank12220 +12221 POINT(33.39955739060503 -118.86630501449224) bank12221 +12222 POINT(33.36240588131977 -118.45590317620747) bank12222 +12223 POINT(34.825894585890126 -118.46994395708217) bank12223 +12224 POINT(33.43791083844443 -119.11845768455088) bank12224 +12225 POINT(33.73828706414052 -119.06417372122186) bank12225 +12226 POINT(33.4152788918593 -117.76127282063943) bank12226 +12227 POINT(33.5881573306462 -118.52900743362325) bank12227 +12228 POINT(34.77084487466421 -117.38308468639168) bank12228 +12229 POINT(33.14450417993508 -117.4025111629696) bank12229 +12230 POINT(34.25619941611816 -117.91419576009861) bank12230 +12231 POINT(33.40855859262589 -118.24496083703697) bank12231 +12232 POINT(33.93103405208271 -117.33006713567656) bank12232 +12233 POINT(33.77341720304372 -118.00349696007295) bank12233 +12234 POINT(33.84675326717664 -118.00196055170257) bank12234 +12235 POINT(33.19805647217693 -117.67587300239154) bank12235 +12236 POINT(33.82578923589185 -118.71911990259132) bank12236 +12237 POINT(34.49890295674672 -117.43672982948459) bank12237 +12238 POINT(34.10868415948967 -118.07159274549416) bank12238 +12239 POINT(33.939014954791446 -118.36537876016602) bank12239 +12240 POINT(34.67155286726371 -119.24270214362316) bank12240 +12241 POINT(34.35907645326998 -118.3848916579969) bank12241 +12242 POINT(33.89686827169783 -117.75238825801263) bank12242 +12243 POINT(34.02365823104398 -117.88475291916183) bank12243 +12244 POINT(34.917262604708355 -118.61169113957774) bank12244 +12245 POINT(33.28736701654751 -118.51918102248499) bank12245 +12246 POINT(33.56498870725796 -118.12991847391582) bank12246 +12247 POINT(33.190591734563796 -117.40615292511029) bank12247 +12248 POINT(34.97405694392372 -118.5121641987878) bank12248 +12249 POINT(33.078468520332095 -117.9305480409078) bank12249 +12250 POINT(33.95065097265953 -117.80800530464253) bank12250 +12251 POINT(33.38330670691405 -119.13707072662575) bank12251 +12252 POINT(33.45981789602083 -117.68808189299101) bank12252 +12253 POINT(33.8691166076804 -117.34600824374809) bank12253 +12254 POINT(34.73804212231767 -118.52961414457687) bank12254 +12255 POINT(34.06382334837923 -117.66931202410065) bank12255 +12256 POINT(33.89401673353403 -117.30945931882678) bank12256 +12257 POINT(35.01953763043321 -118.77225498097896) bank12257 +12258 POINT(34.90817830409763 -118.32250849669974) bank12258 +12259 POINT(33.16996222497708 -118.9395063775965) bank12259 +12260 POINT(34.032863942877185 -118.69503740440886) bank12260 +12261 POINT(34.26640118003166 -119.0945382360811) bank12261 +12262 POINT(34.165005617905685 -118.0477822461814) bank12262 +12263 POINT(34.12740757664251 -118.14241271937857) bank12263 +12264 POINT(33.450685046816524 -118.44788947202662) bank12264 +12265 POINT(34.11970936319692 -118.47013177890008) bank12265 +12266 POINT(33.1581566020677 -117.81315112867263) bank12266 +12267 POINT(34.91700523950768 -118.39913022289562) bank12267 +12268 POINT(33.58220328723562 -118.95825607625954) bank12268 +12269 POINT(33.38718201613054 -119.10305731238044) bank12269 +12270 POINT(33.06699291437833 -118.83928622882797) bank12270 +12271 POINT(33.41155463480118 -118.84622537605456) bank12271 +12272 POINT(35.008176714255754 -118.71398858653725) bank12272 +12273 POINT(34.5615702784852 -119.23045715228574) bank12273 +12274 POINT(34.03094601560345 -118.87597234323022) bank12274 +12275 POINT(34.090581600221775 -118.32536838373214) bank12275 +12276 POINT(33.18720125007638 -117.9586258482928) bank12276 +12277 POINT(33.893415559444385 -117.66193111724782) bank12277 +12278 POINT(34.570999528461705 -118.34627623680974) bank12278 +12279 POINT(34.73366785836066 -118.29997534408832) bank12279 +12280 POINT(34.66657149758431 -117.5791683506208) bank12280 +12281 POINT(33.385230923453456 -118.22775368667075) bank12281 +12282 POINT(34.696931049388944 -117.48941789102) bank12282 +12283 POINT(33.43387293313573 -118.2323681754508) bank12283 +12284 POINT(33.49496245299724 -117.82790490826397) bank12284 +12285 POINT(34.913281056601754 -119.15751833204894) bank12285 +12286 POINT(33.3841901785719 -118.10401079317629) bank12286 +12287 POINT(33.54649156262879 -118.49425975059607) bank12287 +12288 POINT(34.16162718855703 -118.34023533465154) bank12288 +12289 POINT(33.71866735592933 -117.81561527038885) bank12289 +12290 POINT(33.594783347420574 -118.34782733978692) bank12290 +12291 POINT(34.57892047979799 -117.6586310046455) bank12291 +12292 POINT(33.16809249848642 -118.96694045019184) bank12292 +12293 POINT(34.00526855500199 -117.4962583462359) bank12293 +12294 POINT(33.47786621147112 -118.42724996361119) bank12294 +12295 POINT(33.432607965936775 -117.5020681084965) bank12295 +12296 POINT(34.57933447538632 -117.82944820795322) bank12296 +12297 POINT(34.406055240553634 -118.31462418490773) bank12297 +12298 POINT(34.67293473352687 -118.37005737009555) bank12298 +12299 POINT(33.500109780172316 -118.7840250527701) bank12299 +12300 POINT(34.10351279115008 -119.1130362401866) bank12300 +12301 POINT(33.99770817754782 -117.44112851263148) bank12301 +12302 POINT(34.92571179090987 -119.18393386548861) bank12302 +12303 POINT(35.006835559315284 -118.52165893991273) bank12303 +12304 POINT(33.84423217116294 -118.90492170506396) bank12304 +12305 POINT(34.660194579213446 -117.37891610227405) bank12305 +12306 POINT(33.36209309069425 -117.79279280269778) bank12306 +12307 POINT(34.40054813149731 -119.23267383384487) bank12307 +12308 POINT(34.665520442666995 -118.60216254777299) bank12308 +12309 POINT(34.063927309946514 -118.57439960809857) bank12309 +12310 POINT(34.81932130878043 -118.04760142479773) bank12310 +12311 POINT(34.34443635212872 -118.60059306549856) bank12311 +12312 POINT(33.933084294987026 -117.90698949608333) bank12312 +12313 POINT(33.76935162080091 -117.38879495528455) bank12313 +12314 POINT(33.70485883929887 -117.86824836606571) bank12314 +12315 POINT(34.29867303412681 -118.02061903372581) bank12315 +12316 POINT(33.21799018908842 -118.62798864741742) bank12316 +12317 POINT(34.71909525081341 -117.33591479595161) bank12317 +12318 POINT(35.013446334313066 -117.33925978681833) bank12318 +12319 POINT(33.19697053612108 -119.11911669976205) bank12319 +12320 POINT(34.850487363351455 -118.82414854555968) bank12320 +12321 POINT(33.426461359528886 -117.36068052096422) bank12321 +12322 POINT(34.919969946456916 -118.95407381634516) bank12322 +12323 POINT(33.10229542430273 -118.81511027703716) bank12323 +12324 POINT(33.672342282903465 -118.43996548527394) bank12324 +12325 POINT(33.603185364609985 -118.45117318344904) bank12325 +12326 POINT(33.578514981802186 -118.81167138169369) bank12326 +12327 POINT(33.836521025748574 -117.50004158272368) bank12327 +12328 POINT(34.92080965104651 -118.77475346210194) bank12328 +12329 POINT(33.857045598520884 -117.31870885514343) bank12329 +12330 POINT(33.225929685225125 -118.54076647885478) bank12330 +12331 POINT(33.44920508656493 -118.07687105360776) bank12331 +12332 POINT(33.421568189965406 -117.94508111316433) bank12332 +12333 POINT(33.071497367310414 -118.61165855124784) bank12333 +12334 POINT(34.04637385346906 -117.34746260051973) bank12334 +12335 POINT(34.53533556609175 -117.30631743468234) bank12335 +12336 POINT(34.793844381064616 -118.85245353378889) bank12336 +12337 POINT(34.204935957386475 -118.78998974973752) bank12337 +12338 POINT(33.23514997669351 -117.84881438397747) bank12338 +12339 POINT(33.58164273489548 -117.9653349040634) bank12339 +12340 POINT(33.566795796429595 -117.42238452840103) bank12340 +12341 POINT(34.54187768374331 -118.45802212358124) bank12341 +12342 POINT(33.22299129300867 -118.94413988629545) bank12342 +12343 POINT(34.99176620344388 -117.28349259336207) bank12343 +12344 POINT(34.66192136294078 -118.30413578980667) bank12344 +12345 POINT(34.31062135415892 -118.91830405898176) bank12345 +12346 POINT(33.15042439928706 -118.0835984911898) bank12346 +12347 POINT(33.8387745677648 -118.70955380281393) bank12347 +12348 POINT(33.727206335887004 -117.80154141655726) bank12348 +12349 POINT(33.60748594001307 -119.05327191589508) bank12349 +12350 POINT(34.358960525447365 -119.19910351095147) bank12350 +12351 POINT(34.865499950953655 -118.7626536985073) bank12351 +12352 POINT(33.96223172990749 -118.33060938622678) bank12352 +12353 POINT(34.55870456862296 -117.74772674120358) bank12353 +12354 POINT(34.0218558317435 -117.53057099366112) bank12354 +12355 POINT(33.8298358059385 -118.23129756404221) bank12355 +12356 POINT(33.06070084668704 -117.52216877255727) bank12356 +12357 POINT(34.836917177658805 -119.11005912860018) bank12357 +12358 POINT(34.805634001678385 -118.64299173253528) bank12358 +12359 POINT(34.784906436486914 -118.62173606406996) bank12359 +12360 POINT(34.572922828576885 -117.98642490990187) bank12360 +12361 POINT(33.22705947757636 -117.59525400136778) bank12361 +12362 POINT(34.418207368642236 -117.89265829013654) bank12362 +12363 POINT(33.260968537227114 -117.62007204278758) bank12363 +12364 POINT(34.69464792545968 -118.69262354957924) bank12364 +12365 POINT(33.87842049162877 -118.76571930942735) bank12365 +12366 POINT(34.58000914066746 -118.60001204275008) bank12366 +12367 POINT(34.55622865604599 -118.42977938433464) bank12367 +12368 POINT(34.16432923888318 -118.5450871861746) bank12368 +12369 POINT(33.68518532759009 -118.97701457296898) bank12369 +12370 POINT(34.74290315021924 -117.98018166677221) bank12370 +12371 POINT(34.890472890648525 -118.7423017483123) bank12371 +12372 POINT(34.578713495645026 -117.7511271831849) bank12372 +12373 POINT(33.15941017346626 -118.74582018782381) bank12373 +12374 POINT(34.090717530026055 -117.88065748789158) bank12374 +12375 POINT(34.704811613307186 -118.72337747724421) bank12375 +12376 POINT(33.676202400780134 -118.48562248411557) bank12376 +12377 POINT(34.15415305634876 -118.86545670476761) bank12377 +12378 POINT(34.02361505953369 -118.86219280042154) bank12378 +12379 POINT(34.64361743284942 -118.28800838702534) bank12379 +12380 POINT(35.006006907552695 -117.5242415792348) bank12380 +12381 POINT(33.21478362767405 -119.19191575520799) bank12381 +12382 POINT(33.94772831195991 -118.1631547204021) bank12382 +12383 POINT(33.95656209480542 -119.1291811294082) bank12383 +12384 POINT(33.88023597373232 -119.05429880062057) bank12384 +12385 POINT(33.70559221013033 -118.14444626870029) bank12385 +12386 POINT(33.21709511072903 -119.09839755844001) bank12386 +12387 POINT(34.15495105634627 -118.4353540766217) bank12387 +12388 POINT(34.828860558411456 -117.70018598165494) bank12388 +12389 POINT(34.76686748688006 -118.45677623008254) bank12389 +12390 POINT(33.65755494325154 -118.55127640905692) bank12390 +12391 POINT(34.55334232155384 -117.98362124690748) bank12391 +12392 POINT(33.186516299765195 -118.88190404892715) bank12392 +12393 POINT(35.00208038045782 -119.05599987566629) bank12393 +12394 POINT(33.71560029260267 -117.64178398792566) bank12394 +12395 POINT(34.32634606313138 -118.99338140805531) bank12395 +12396 POINT(33.12075134258266 -118.55309169161796) bank12396 +12397 POINT(34.39317979491049 -118.18281896711734) bank12397 +12398 POINT(34.99831524711321 -117.28843828008999) bank12398 +12399 POINT(34.99616436447586 -117.30350535227595) bank12399 +12400 POINT(33.83703097423054 -117.47861859564246) bank12400 +12401 POINT(34.9208403225388 -118.32624062155354) bank12401 +12402 POINT(34.21383081854479 -118.97973725228371) bank12402 +12403 POINT(34.188716589927765 -118.78140117265555) bank12403 +12404 POINT(34.694288032995736 -118.847512005776) bank12404 +12405 POINT(33.27516755147055 -118.1927749966064) bank12405 +12406 POINT(34.233137952521595 -118.09922785614344) bank12406 +12407 POINT(33.10564285742755 -118.72394014496949) bank12407 +12408 POINT(33.38694804441803 -118.08617643609252) bank12408 +12409 POINT(34.314103672980025 -118.06250075385046) bank12409 +12410 POINT(34.83311259441265 -118.13676457054262) bank12410 +12411 POINT(34.03421460471761 -117.8065913008547) bank12411 +12412 POINT(34.23492006682302 -117.35272187517019) bank12412 +12413 POINT(33.8782398329856 -118.42590309493964) bank12413 +12414 POINT(34.260455598691124 -118.51313729939173) bank12414 +12415 POINT(34.34780425432605 -117.47594884297696) bank12415 +12416 POINT(34.64415392869989 -119.06535252598084) bank12416 +12417 POINT(34.655855665243465 -117.30892600948631) bank12417 +12418 POINT(33.77290320816192 -117.48645922303974) bank12418 +12419 POINT(33.30686207609375 -118.1791779835076) bank12419 +12420 POINT(34.278055792672085 -118.81370706939944) bank12420 +12421 POINT(34.982297044711245 -117.8492760658103) bank12421 +12422 POINT(34.90745405272378 -118.52381152754683) bank12422 +12423 POINT(34.03691607199251 -118.57093522764198) bank12423 +12424 POINT(34.72327853095169 -117.50612592606039) bank12424 +12425 POINT(33.3232052759311 -117.35062127040527) bank12425 +12426 POINT(34.544587279369836 -118.3631212920215) bank12426 +12427 POINT(33.76216242347447 -117.74469779332475) bank12427 +12428 POINT(33.76722002284013 -117.99444845808237) bank12428 +12429 POINT(34.21253144610624 -117.4471376243661) bank12429 +12430 POINT(33.85609948973032 -118.85166016760823) bank12430 +12431 POINT(33.79418630261251 -118.71018092423124) bank12431 +12432 POINT(34.50407012961434 -117.97767040313687) bank12432 +12433 POINT(34.14233773056252 -118.21043849432394) bank12433 +12434 POINT(33.641250019337235 -118.89742699726916) bank12434 +12435 POINT(34.59387514648526 -118.15806007481396) bank12435 +12436 POINT(33.11061324734214 -117.35482697327087) bank12436 +12437 POINT(33.90675795254213 -117.5235958212309) bank12437 +12438 POINT(33.41082542387783 -119.04668922732851) bank12438 +12439 POINT(34.074285784991105 -119.23687081447541) bank12439 +12440 POINT(33.990508382151575 -119.23271625966078) bank12440 +12441 POINT(34.52364331777911 -118.43477825109144) bank12441 +12442 POINT(34.51980366530506 -117.35092931060248) bank12442 +12443 POINT(34.08202862157798 -118.94131096282713) bank12443 +12444 POINT(34.726909400820404 -118.61487203246593) bank12444 +12445 POINT(33.630275051945866 -117.46867117977365) bank12445 +12446 POINT(33.989058825694585 -117.69989198874993) bank12446 +12447 POINT(35.02374489504885 -117.5010191369454) bank12447 +12448 POINT(34.342638458163336 -118.51765541863105) bank12448 +12449 POINT(33.349085551953344 -117.25222896880138) bank12449 +12450 POINT(34.86439640904498 -118.50233509759606) bank12450 +12451 POINT(34.93829857140581 -117.82376499358084) bank12451 +12452 POINT(33.357554104478666 -119.24099673198987) bank12452 +12453 POINT(35.03171895747412 -117.71109878035452) bank12453 +12454 POINT(34.85559182428729 -118.02106975539262) bank12454 +12455 POINT(33.60796720755522 -117.34109456731319) bank12455 +12456 POINT(33.671372594139655 -118.92862906372423) bank12456 +12457 POINT(33.59078952472107 -118.63172591391985) bank12457 +12458 POINT(33.752025502741304 -117.59207241877647) bank12458 +12459 POINT(34.45820273421052 -119.07344781315489) bank12459 +12460 POINT(34.387817638185794 -117.57172570333933) bank12460 +12461 POINT(34.1280266797363 -118.74841416893206) bank12461 +12462 POINT(34.64575491363144 -118.61916285041474) bank12462 +12463 POINT(33.18609046690155 -118.8671135340157) bank12463 +12464 POINT(33.72758141616917 -118.94192233939164) bank12464 +12465 POINT(34.67051416243265 -117.5025187912013) bank12465 +12466 POINT(33.57181994479818 -119.02071466934798) bank12466 +12467 POINT(34.672415070192166 -117.31923120166124) bank12467 +12468 POINT(34.89074495086925 -117.56771078615316) bank12468 +12469 POINT(33.31664279602235 -118.57997199618488) bank12469 +12470 POINT(34.824317923856704 -118.54663866176156) bank12470 +12471 POINT(34.46636223677756 -118.36173996336683) bank12471 +12472 POINT(33.47577370332943 -117.72959327789364) bank12472 +12473 POINT(34.56147389159107 -117.54977975732696) bank12473 +12474 POINT(34.84932414676217 -117.6711053234646) bank12474 +12475 POINT(34.9880114947479 -117.88682098132772) bank12475 +12476 POINT(34.55065091263706 -117.91522672546259) bank12476 +12477 POINT(33.424216235137095 -118.5941275001988) bank12477 +12478 POINT(33.99496896125148 -117.56378691776894) bank12478 +12479 POINT(33.76754572169945 -119.10051022378636) bank12479 +12480 POINT(33.15681445760288 -118.01437293285653) bank12480 +12481 POINT(34.16460273279638 -119.13966973558212) bank12481 +12482 POINT(34.7400431164129 -117.93147911334657) bank12482 +12483 POINT(34.60312135088499 -117.40875331008726) bank12483 +12484 POINT(34.94411203641305 -117.97223523640496) bank12484 +12485 POINT(34.62346332766413 -118.77673653417638) bank12485 +12486 POINT(34.0472371515757 -119.01212294217606) bank12486 +12487 POINT(33.85533455069483 -117.53713868615627) bank12487 +12488 POINT(33.19640735850633 -117.50248101722379) bank12488 +12489 POINT(33.824455000861754 -117.59730425421695) bank12489 +12490 POINT(33.83262685204269 -118.94729095724506) bank12490 +12491 POINT(34.95253314567035 -117.58512121348156) bank12491 +12492 POINT(33.385981051830655 -119.05499394768519) bank12492 +12493 POINT(34.10281263756471 -117.98683546549968) bank12493 +12494 POINT(33.3130224629226 -118.20162705349317) bank12494 +12495 POINT(33.13693056868603 -118.80404780445373) bank12495 +12496 POINT(33.22373215526716 -118.5870081031826) bank12496 +12497 POINT(34.44010332376546 -118.80717776132258) bank12497 +12498 POINT(33.08656934544423 -118.56183466144432) bank12498 +12499 POINT(34.15607309359699 -117.79956076501276) bank12499 +12500 POINT(34.05965644754307 -118.95246125154871) bank12500 +12501 POINT(34.86066777436065 -117.3809784325865) bank12501 +12502 POINT(34.558378050793245 -118.13268662400563) bank12502 +12503 POINT(34.55051852822332 -118.98465913367296) bank12503 +12504 POINT(34.93950243042738 -119.11784363022606) bank12504 +12505 POINT(34.467710055160204 -118.54119387571026) bank12505 +12506 POINT(34.07605573543921 -118.78495606810455) bank12506 +12507 POINT(33.42529429292353 -117.99659853001673) bank12507 +12508 POINT(33.33868598963338 -117.60082224098588) bank12508 +12509 POINT(34.914224569960425 -118.53513874362343) bank12509 +12510 POINT(33.07858943044872 -117.74127953813627) bank12510 +12511 POINT(34.7221791383681 -117.99778045200512) bank12511 +12512 POINT(34.1744398915476 -118.17933705102718) bank12512 +12513 POINT(34.891372746940114 -117.86042157252177) bank12513 +12514 POINT(34.685800251849585 -118.06635476600277) bank12514 +12515 POINT(34.52452684484422 -117.4839759675904) bank12515 +12516 POINT(33.119719206947565 -118.9829192385752) bank12516 +12517 POINT(33.39176364639429 -117.41826973987794) bank12517 +12518 POINT(33.124825422820464 -117.33708640214208) bank12518 +12519 POINT(34.576985332672365 -117.3881972841848) bank12519 +12520 POINT(34.85999493784269 -119.24241277092653) bank12520 +12521 POINT(34.83253044008978 -119.16550108368034) bank12521 +12522 POINT(33.60892838695919 -117.51834402514105) bank12522 +12523 POINT(34.888632927103096 -117.71206233830878) bank12523 +12524 POINT(33.738902943230485 -119.03956605391205) bank12524 +12525 POINT(34.33699181464171 -117.85981869233427) bank12525 +12526 POINT(33.88641081139549 -117.56611941592254) bank12526 +12527 POINT(33.58401658313197 -117.76734412429268) bank12527 +12528 POINT(33.452829607914445 -119.20189743865977) bank12528 +12529 POINT(33.79324363026714 -118.85168215896361) bank12529 +12530 POINT(34.97990722170061 -118.48752696802326) bank12530 +12531 POINT(34.774723107058506 -118.46253662193456) bank12531 +12532 POINT(34.0792163864918 -118.0532856452471) bank12532 +12533 POINT(33.26692061616106 -118.89430381876967) bank12533 +12534 POINT(33.83130593292348 -117.58105272794909) bank12534 +12535 POINT(34.85633684850384 -117.82321248587725) bank12535 +12536 POINT(34.086894302024085 -118.6388930217337) bank12536 +12537 POINT(34.0762691422452 -118.07115567202092) bank12537 +12538 POINT(34.61310853431245 -118.38604668348093) bank12538 +12539 POINT(33.28977218079517 -117.86675625839591) bank12539 +12540 POINT(34.68517538097263 -119.04505565336575) bank12540 +12541 POINT(34.65417009805615 -119.05762131299818) bank12541 +12542 POINT(33.79207872256697 -118.66515391494013) bank12542 +12543 POINT(34.06801956431721 -118.78723028109337) bank12543 +12544 POINT(34.65967705484803 -119.04300786267886) bank12544 +12545 POINT(33.72915607911573 -119.1251722861924) bank12545 +12546 POINT(34.402398061626194 -117.50789179725106) bank12546 +12547 POINT(33.23580050564944 -118.94477218659115) bank12547 +12548 POINT(33.068928721579596 -118.93582848840893) bank12548 +12549 POINT(34.22560449697594 -118.18559105661248) bank12549 +12550 POINT(34.02169320957533 -117.82369820534353) bank12550 +12551 POINT(34.632178512950084 -117.7591908787291) bank12551 +12552 POINT(34.29213635691372 -118.36550672368253) bank12552 +12553 POINT(34.262969499996075 -117.7876375541258) bank12553 +12554 POINT(34.48342440189955 -118.38690037243344) bank12554 +12555 POINT(34.120696390211855 -117.2842427993435) bank12555 +12556 POINT(33.35897312719154 -119.08877950248976) bank12556 +12557 POINT(34.88360093739967 -119.02160813669913) bank12557 +12558 POINT(33.713881859169526 -117.61165027957465) bank12558 +12559 POINT(33.78362717074672 -118.85017819255842) bank12559 +12560 POINT(34.66087838600147 -117.62855896259519) bank12560 +12561 POINT(33.36158820370321 -118.75622299722794) bank12561 +12562 POINT(33.311054559151465 -118.552944920481) bank12562 +12563 POINT(34.160758872365186 -118.19373580681464) bank12563 +12564 POINT(33.21100689235954 -117.9665692087997) bank12564 +12565 POINT(33.905133268892634 -117.6052538732783) bank12565 +12566 POINT(33.70034174649498 -118.03739767514347) bank12566 +12567 POINT(34.691054328464986 -118.41084340162946) bank12567 +12568 POINT(33.53496068269767 -117.79054769782275) bank12568 +12569 POINT(33.65005373421602 -119.00940267722855) bank12569 +12570 POINT(34.94757517071799 -118.43317119059192) bank12570 +12571 POINT(33.0713752269917 -118.05620584876786) bank12571 +12572 POINT(34.57897863934566 -118.04252482889987) bank12572 +12573 POINT(34.479440270312345 -117.69746565150754) bank12573 +12574 POINT(34.453979015709315 -117.64225200049793) bank12574 +12575 POINT(34.23856627101234 -118.0601914706798) bank12575 +12576 POINT(34.46188162138339 -117.49707314241991) bank12576 +12577 POINT(34.86534933013793 -117.25236714343919) bank12577 +12578 POINT(34.23352344875481 -118.40131722076205) bank12578 +12579 POINT(34.96484943489501 -118.97123427513056) bank12579 +12580 POINT(33.59629384997366 -119.06078133000813) bank12580 +12581 POINT(33.72171717590256 -118.7850684166359) bank12581 +12582 POINT(34.39806101526833 -118.87001217097358) bank12582 +12583 POINT(33.25821111282881 -118.76637634934555) bank12583 +12584 POINT(33.82474275014055 -117.69841915718507) bank12584 +12585 POINT(33.57469050672703 -119.0929343455438) bank12585 +12586 POINT(34.92338017477164 -117.72294672975279) bank12586 +12587 POINT(33.65223570334453 -118.31452703503517) bank12587 +12588 POINT(33.209913282537926 -118.69436314780303) bank12588 +12589 POINT(34.65687475563552 -118.91610131813795) bank12589 +12590 POINT(34.14104745747035 -119.05779810486526) bank12590 +12591 POINT(35.00568899188007 -117.36780255976808) bank12591 +12592 POINT(34.93118201056572 -118.74733774163086) bank12592 +12593 POINT(34.037794893478136 -117.63004381367624) bank12593 +12594 POINT(33.28270547539223 -117.99322449119175) bank12594 +12595 POINT(33.24018964462235 -118.07455182163979) bank12595 +12596 POINT(33.36824331376586 -118.51203105112188) bank12596 +12597 POINT(33.59811298029942 -117.95972535529044) bank12597 +12598 POINT(33.07531596494493 -118.22332613279949) bank12598 +12599 POINT(34.35585579971719 -117.92770743309491) bank12599 +12600 POINT(34.2980369166137 -117.87988911786) bank12600 +12601 POINT(34.29540287906388 -118.66722587269466) bank12601 +12602 POINT(33.074792463196424 -118.25129340432326) bank12602 +12603 POINT(34.81001191700446 -117.74038795488696) bank12603 +12604 POINT(33.12984773943491 -117.49050035039035) bank12604 +12605 POINT(34.755845846919684 -119.22332107026278) bank12605 +12606 POINT(33.50006910330107 -117.81956864478234) bank12606 +12607 POINT(34.898637018243754 -117.88449154599373) bank12607 +12608 POINT(33.76935878711075 -118.57914448200793) bank12608 +12609 POINT(33.07094999776217 -117.61326997786013) bank12609 +12610 POINT(33.06263811397606 -118.31209926416733) bank12610 +12611 POINT(34.157337409944674 -118.71689256911273) bank12611 +12612 POINT(33.50972480027265 -118.47005540383422) bank12612 +12613 POINT(33.32419050705405 -118.46806663777897) bank12613 +12614 POINT(33.55205080242005 -117.79744665440498) bank12614 +12615 POINT(34.816500801806 -118.15552769268714) bank12615 +12616 POINT(34.82053948368113 -119.14672332568793) bank12616 +12617 POINT(33.26047047785322 -119.22793867190234) bank12617 +12618 POINT(34.93232428151484 -118.60535081488283) bank12618 +12619 POINT(33.940596290802006 -118.40299803445969) bank12619 +12620 POINT(34.6750809782574 -119.05478576450614) bank12620 +12621 POINT(34.8709753106192 -117.37908564762611) bank12621 +12622 POINT(33.52183865728989 -117.78433897187344) bank12622 +12623 POINT(34.3819990688168 -117.7726562212719) bank12623 +12624 POINT(35.007737553551046 -118.74682420020926) bank12624 +12625 POINT(34.829374608662555 -117.38627677699175) bank12625 +12626 POINT(34.10274907277559 -117.29632493662389) bank12626 +12627 POINT(34.188282806625814 -118.74756533352057) bank12627 +12628 POINT(33.79932229020619 -117.6146867946579) bank12628 +12629 POINT(34.93139413542865 -118.3963187220506) bank12629 +12630 POINT(33.741115790087484 -118.5708442521525) bank12630 +12631 POINT(33.23186636341581 -118.33345089412373) bank12631 +12632 POINT(34.003812674246035 -117.74837467161842) bank12632 +12633 POINT(33.79822529822246 -117.95004861374916) bank12633 +12634 POINT(33.36507932690456 -119.07643075288877) bank12634 +12635 POINT(33.467273435503955 -118.00944265979717) bank12635 +12636 POINT(34.458039845920155 -118.93599192413784) bank12636 +12637 POINT(34.33526617800019 -117.46123386091287) bank12637 +12638 POINT(34.3171462305513 -117.6716485176365) bank12638 +12639 POINT(34.22356650764615 -118.03883736839553) bank12639 +12640 POINT(34.81593765699059 -119.23866174286701) bank12640 +12641 POINT(34.17899697729481 -118.26138615292894) bank12641 +12642 POINT(34.59861957584032 -117.7521086564314) bank12642 +12643 POINT(34.21088098877968 -117.74146964292277) bank12643 +12644 POINT(33.805909313210044 -118.45927450677452) bank12644 +12645 POINT(33.247827295971426 -117.74679669462908) bank12645 +12646 POINT(34.18941827591539 -117.85060139797133) bank12646 +12647 POINT(33.30603188234932 -118.33879512936679) bank12647 +12648 POINT(33.10226418900481 -117.66303437385075) bank12648 +12649 POINT(33.302509863461765 -118.97134558978225) bank12649 +12650 POINT(33.519757495768005 -119.12220519731537) bank12650 +12651 POINT(34.563407087673355 -117.57063107812992) bank12651 +12652 POINT(34.59458365424704 -118.41948154395331) bank12652 +12653 POINT(33.23501116410608 -119.11985523691467) bank12653 +12654 POINT(33.11547345701014 -119.22075730685035) bank12654 +12655 POINT(33.84251027510925 -118.92761367731232) bank12655 +12656 POINT(33.634878610326886 -117.56835953385364) bank12656 +12657 POINT(34.99721778638814 -118.53166819748577) bank12657 +12658 POINT(34.413298169707204 -118.31929441398357) bank12658 +12659 POINT(34.03682759362061 -118.73953222883586) bank12659 +12660 POINT(34.501170253714136 -118.1714449093442) bank12660 +12661 POINT(34.86825896459909 -117.66916878319378) bank12661 +12662 POINT(35.00586330940405 -119.12986520135797) bank12662 +12663 POINT(34.15568333950691 -118.48343309163587) bank12663 +12664 POINT(33.912763674314306 -117.97862041472042) bank12664 +12665 POINT(34.231383523505585 -118.82378779831559) bank12665 +12666 POINT(33.968637462543555 -119.09430922733839) bank12666 +12667 POINT(33.99267931764314 -118.2874437582099) bank12667 +12668 POINT(33.48655673063807 -119.17557002578465) bank12668 +12669 POINT(33.96969842999176 -117.69730274398) bank12669 +12670 POINT(33.37569386946924 -117.45510276617759) bank12670 +12671 POINT(34.36675368325831 -118.23594489170264) bank12671 +12672 POINT(34.995841694656185 -117.48489646275432) bank12672 +12673 POINT(33.28967193580098 -117.2574802534902) bank12673 +12674 POINT(34.3700637483356 -118.03814506435424) bank12674 +12675 POINT(34.911868275551996 -118.89931541798093) bank12675 +12676 POINT(33.46428720434935 -117.44384539998359) bank12676 +12677 POINT(34.184698820748956 -117.99100695369738) bank12677 +12678 POINT(33.911724678565555 -118.27878106506535) bank12678 +12679 POINT(34.69678905148897 -117.71846455488206) bank12679 +12680 POINT(34.691436486444815 -117.88739071960269) bank12680 +12681 POINT(33.82542070096634 -118.90869553021535) bank12681 +12682 POINT(33.20662465080822 -119.11826262340303) bank12682 +12683 POINT(34.37827760271769 -118.93345260437584) bank12683 +12684 POINT(34.310499845872826 -117.2618309282524) bank12684 +12685 POINT(34.43237083238338 -118.62465521999759) bank12685 +12686 POINT(34.960594034865665 -118.28698909339315) bank12686 +12687 POINT(33.58281008368514 -118.45812200994317) bank12687 +12688 POINT(33.45995164714314 -117.8628811895872) bank12688 +12689 POINT(33.26286378434163 -118.82919191371212) bank12689 +12690 POINT(33.401887028737264 -118.2439482465905) bank12690 +12691 POINT(35.02650161537339 -117.71887123048256) bank12691 +12692 POINT(34.60604065566787 -117.9271910739506) bank12692 +12693 POINT(33.601181206536495 -118.84723168400406) bank12693 +12694 POINT(33.361095397679826 -118.84991674939572) bank12694 +12695 POINT(33.53144266008168 -117.54688631069118) bank12695 +12696 POINT(34.387997373680165 -118.04973916074263) bank12696 +12697 POINT(34.68794394814811 -117.3795926712634) bank12697 +12698 POINT(34.645845333283226 -119.03018191726807) bank12698 +12699 POINT(34.35470726543641 -119.08014624511387) bank12699 +12700 POINT(34.51978973407477 -117.41826604575016) bank12700 +12701 POINT(34.73353956063601 -117.38765119142764) bank12701 +12702 POINT(34.06185327949947 -117.43259769940548) bank12702 +12703 POINT(35.0138816353252 -118.53500055124151) bank12703 +12704 POINT(34.867799911662956 -117.68575117561792) bank12704 +12705 POINT(33.9299845160984 -117.2987559029738) bank12705 +12706 POINT(34.99397411822836 -118.2437790471635) bank12706 +12707 POINT(34.97338338604109 -118.6825454854321) bank12707 +12708 POINT(33.72348920052776 -118.43016157975305) bank12708 +12709 POINT(33.465343391776955 -117.9310121614758) bank12709 +12710 POINT(34.5021178989153 -118.4981926946478) bank12710 +12711 POINT(33.1350292019654 -117.36351939517976) bank12711 +12712 POINT(34.31328168610537 -118.53378758384329) bank12712 +12713 POINT(34.28021041455126 -117.70035392824268) bank12713 +12714 POINT(34.22303386663943 -117.41443745737229) bank12714 +12715 POINT(33.59909336118228 -117.51945620748585) bank12715 +12716 POINT(34.685419508498505 -117.35024619482729) bank12716 +12717 POINT(34.05342555621797 -117.76520673324923) bank12717 +12718 POINT(34.52745204281551 -117.85198302953286) bank12718 +12719 POINT(34.37494541724588 -119.02652622205377) bank12719 +12720 POINT(33.07385984358825 -117.99721545298101) bank12720 +12721 POINT(33.22897590269987 -118.46512276848033) bank12721 +12722 POINT(33.260273878974395 -118.58858330754171) bank12722 +12723 POINT(33.49636367961667 -119.03642880954553) bank12723 +12724 POINT(34.90437774657563 -117.37899889250649) bank12724 +12725 POINT(33.539531295910635 -118.17164646240195) bank12725 +12726 POINT(33.67219671694469 -117.78398408700885) bank12726 +12727 POINT(33.061767834333814 -118.63930637025382) bank12727 +12728 POINT(34.89271128716814 -117.88972862413115) bank12728 +12729 POINT(33.07007542503077 -119.24117227584222) bank12729 +12730 POINT(33.97704075723982 -118.13156028148241) bank12730 +12731 POINT(33.93919833475588 -119.14423115767391) bank12731 +12732 POINT(34.08444972908974 -118.67163365161197) bank12732 +12733 POINT(33.74589086647655 -118.81623973600752) bank12733 +12734 POINT(34.04653911881201 -118.99353022476875) bank12734 +12735 POINT(33.10251870836723 -118.76226389317364) bank12735 +12736 POINT(34.630642195795275 -118.17992544042731) bank12736 +12737 POINT(34.47916356265597 -118.75737623783654) bank12737 +12738 POINT(34.0224031762137 -117.98996235569432) bank12738 +12739 POINT(34.363184401294376 -118.90669673017955) bank12739 +12740 POINT(33.89520218890441 -118.04281633421294) bank12740 +12741 POINT(34.04437855089918 -119.02514992882045) bank12741 +12742 POINT(33.19049799605295 -119.03677963750857) bank12742 +12743 POINT(33.26173815923989 -118.09188412834156) bank12743 +12744 POINT(34.65580815965518 -119.02363249050794) bank12744 +12745 POINT(35.04523704925774 -118.97630411575423) bank12745 +12746 POINT(34.30557049139286 -118.7535753308064) bank12746 +12747 POINT(33.12552371320182 -118.29804677928824) bank12747 +12748 POINT(33.4424882151289 -117.38237184089544) bank12748 +12749 POINT(35.02675439695511 -118.66370158615406) bank12749 +12750 POINT(33.23218867803083 -118.1252098071881) bank12750 +12751 POINT(34.770865772407554 -117.39397480076319) bank12751 +12752 POINT(34.965402822780774 -119.11410073677955) bank12752 +12753 POINT(33.130485081370935 -119.00215253561501) bank12753 +12754 POINT(33.69649691404938 -117.38125230493809) bank12754 +12755 POINT(33.0667897911316 -117.78558134479191) bank12755 +12756 POINT(34.42273619751267 -117.8877212481356) bank12756 +12757 POINT(33.84776817373067 -118.93396379023868) bank12757 +12758 POINT(34.7736386369364 -117.46330212405789) bank12758 +12759 POINT(35.043007644242564 -118.42774494990263) bank12759 +12760 POINT(33.611377637917876 -118.97102409457727) bank12760 +12761 POINT(33.51377444111713 -118.18014086019483) bank12761 +12762 POINT(34.058214810845485 -118.93611850034753) bank12762 +12763 POINT(33.100402693175546 -118.95890651884386) bank12763 +12764 POINT(34.07562005475318 -119.19691039451057) bank12764 +12765 POINT(33.0592564357807 -119.03232545944648) bank12765 +12766 POINT(34.98409393681169 -118.11465038854273) bank12766 +12767 POINT(33.93185468636945 -118.12111376900774) bank12767 +12768 POINT(33.96292325680574 -118.07243772830435) bank12768 +12769 POINT(34.554600730048165 -118.04764914224295) bank12769 +12770 POINT(33.61713501380492 -118.82227986970885) bank12770 +12771 POINT(34.16561141577694 -118.32903034158818) bank12771 +12772 POINT(33.69371708318335 -118.74752508915174) bank12772 +12773 POINT(34.77320865555939 -117.94568912702465) bank12773 +12774 POINT(33.81924792471296 -118.45036751373289) bank12774 +12775 POINT(34.372453282584765 -118.1075804471938) bank12775 +12776 POINT(33.83915255048092 -118.71702864228335) bank12776 +12777 POINT(34.24469333093383 -118.59746411296494) bank12777 +12778 POINT(33.79132683370145 -118.50556367290642) bank12778 +12779 POINT(33.937174640680944 -117.99855420678783) bank12779 +12780 POINT(33.148371305040015 -117.5464862476669) bank12780 +12781 POINT(35.05138185927497 -118.95553097348228) bank12781 +12782 POINT(34.31899369286098 -118.52923567894167) bank12782 +12783 POINT(33.1722529384486 -117.59517335164134) bank12783 +12784 POINT(33.96286730073209 -118.97647087652888) bank12784 +12785 POINT(34.8343601725362 -118.5035224563168) bank12785 +12786 POINT(33.42381023610066 -118.85656152682392) bank12786 +12787 POINT(33.682068126603596 -118.67050512004418) bank12787 +12788 POINT(33.80736992624453 -117.65495962244722) bank12788 +12789 POINT(34.58764524117794 -117.74985578403133) bank12789 +12790 POINT(34.21728477883239 -117.35452518560245) bank12790 +12791 POINT(33.13830781361013 -119.16744422102312) bank12791 +12792 POINT(33.34492009554067 -117.52817834670675) bank12792 +12793 POINT(33.55045956743229 -117.54733213414208) bank12793 +12794 POINT(35.050723899189904 -119.1992404765803) bank12794 +12795 POINT(33.94896849376567 -119.05194300329455) bank12795 +12796 POINT(33.865699865315875 -117.29652170418512) bank12796 +12797 POINT(33.23514342373004 -119.23305545004843) bank12797 +12798 POINT(33.923547648148315 -117.24633427158261) bank12798 +12799 POINT(33.65560073591996 -117.29746285308954) bank12799 +12800 POINT(33.783258598650704 -117.34563885401734) bank12800 +12801 POINT(34.28248314379615 -118.8645596501481) bank12801 +12802 POINT(33.12015198047705 -117.40173853576847) bank12802 +12803 POINT(34.401801021870355 -117.29188986672482) bank12803 +12804 POINT(34.5361646623162 -118.42848882678422) bank12804 +12805 POINT(33.08882128288971 -118.7154701506623) bank12805 +12806 POINT(33.38327212509631 -119.1105791303587) bank12806 +12807 POINT(33.36194785538615 -118.02024304957544) bank12807 +12808 POINT(33.09239629333621 -118.52265565148352) bank12808 +12809 POINT(33.813826203707805 -119.03881027208254) bank12809 +12810 POINT(33.38027281062095 -118.92444496758212) bank12810 +12811 POINT(33.447382551815814 -117.78302875069026) bank12811 +12812 POINT(34.6169477651119 -117.42670915307878) bank12812 +12813 POINT(34.558656702882644 -119.19178602051998) bank12813 +12814 POINT(33.860606686164424 -117.96005959890718) bank12814 +12815 POINT(33.31747322109516 -117.61644772250082) bank12815 +12816 POINT(33.11041569891858 -118.57192899076776) bank12816 +12817 POINT(34.762747220977715 -118.37444488708515) bank12817 +12818 POINT(34.076165883906484 -119.05896569872716) bank12818 +12819 POINT(33.36081020149163 -118.3400252121176) bank12819 +12820 POINT(33.24636889569469 -118.67891077241266) bank12820 +12821 POINT(33.27978944845675 -118.66781358642255) bank12821 +12822 POINT(34.22367009930859 -118.0105000703355) bank12822 +12823 POINT(34.412991466448055 -118.3372366839602) bank12823 +12824 POINT(33.24003147857529 -118.78561747874228) bank12824 +12825 POINT(33.825376336546746 -118.72097145531801) bank12825 +12826 POINT(34.62814518140027 -118.27500033392359) bank12826 +12827 POINT(33.34596390593437 -118.79606490891756) bank12827 +12828 POINT(33.12029917766522 -117.78075402069453) bank12828 +12829 POINT(34.68578815001715 -118.57440941455691) bank12829 +12830 POINT(33.16237530256855 -118.96794980017899) bank12830 +12831 POINT(34.41654383589783 -118.08378815921604) bank12831 +12832 POINT(34.989061255232514 -118.12074404951024) bank12832 +12833 POINT(34.933623456635985 -117.67267070861271) bank12833 +12834 POINT(33.13177886546459 -119.00237303336111) bank12834 +12835 POINT(34.68515478368764 -118.54724837694185) bank12835 +12836 POINT(34.26628474376412 -118.93559191911451) bank12836 +12837 POINT(33.892777240358136 -118.60160709867259) bank12837 +12838 POINT(34.49270513027872 -117.48318566193548) bank12838 +12839 POINT(33.553943566269545 -118.53925544976707) bank12839 +12840 POINT(33.26495230811933 -119.16638751976325) bank12840 +12841 POINT(33.5073129664931 -117.26014689402518) bank12841 +12842 POINT(34.82012112740377 -117.57915296265497) bank12842 +12843 POINT(34.3336404845294 -118.34017740861867) bank12843 +12844 POINT(33.23589806776452 -118.40496158366147) bank12844 +12845 POINT(33.24982356949544 -117.89559595058833) bank12845 +12846 POINT(33.24145535602817 -117.7672112298824) bank12846 +12847 POINT(33.61751245042777 -117.29504226873155) bank12847 +12848 POINT(33.572158821348296 -119.08184337320296) bank12848 +12849 POINT(34.09461579987545 -119.09656201128988) bank12849 +12850 POINT(33.59743083261748 -119.10288683678077) bank12850 +12851 POINT(34.186655420413025 -119.05840265083793) bank12851 +12852 POINT(33.230842990844266 -118.90605441805016) bank12852 +12853 POINT(33.34971510566862 -118.73898750107051) bank12853 +12854 POINT(34.872387949661444 -118.14944385410774) bank12854 +12855 POINT(33.363718253474396 -119.17611748682539) bank12855 +12856 POINT(34.876968064167116 -119.09299222646226) bank12856 +12857 POINT(33.16237256045461 -117.42084659899784) bank12857 +12858 POINT(33.93888961501606 -117.49471235057302) bank12858 +12859 POINT(33.098557350187136 -117.99391712481955) bank12859 +12860 POINT(34.65016174512695 -118.64277145732873) bank12860 +12861 POINT(34.84903083436492 -118.54558004351824) bank12861 +12862 POINT(34.041986426284836 -118.37568923301896) bank12862 +12863 POINT(33.90129752991843 -117.78109436525156) bank12863 +12864 POINT(35.0518420139935 -118.25933090536795) bank12864 +12865 POINT(34.693008850218355 -118.26709185050197) bank12865 +12866 POINT(33.565103773732616 -117.4902273923319) bank12866 +12867 POINT(33.13782547632764 -117.30807049886896) bank12867 +12868 POINT(33.95429263078886 -117.88924094564531) bank12868 +12869 POINT(33.37935575572072 -117.40949089559223) bank12869 +12870 POINT(33.93122353413445 -117.58774521756187) bank12870 +12871 POINT(34.25071592134152 -117.93203139324122) bank12871 +12872 POINT(34.29705760786302 -118.9353213906017) bank12872 +12873 POINT(34.44849184892229 -118.23880777526135) bank12873 +12874 POINT(34.459658536007474 -117.67088586810742) bank12874 +12875 POINT(34.101943589862124 -117.79681890043783) bank12875 +12876 POINT(33.967630514034525 -118.82870839253799) bank12876 +12877 POINT(33.40381250674848 -118.86675930188096) bank12877 +12878 POINT(34.95823925870151 -117.32649558400945) bank12878 +12879 POINT(33.07591318699205 -118.90955702196278) bank12879 +12880 POINT(33.90225806505346 -118.38292860378836) bank12880 +12881 POINT(34.61125666431065 -118.64339824289696) bank12881 +12882 POINT(33.77488955233036 -118.84132281027262) bank12882 +12883 POINT(35.03132175078817 -117.99125433381288) bank12883 +12884 POINT(33.89934009390373 -118.37359008016595) bank12884 +12885 POINT(33.670658755111 -119.16695704662669) bank12885 +12886 POINT(35.039973269185126 -118.60054911763567) bank12886 +12887 POINT(33.560425034501115 -118.26695625867433) bank12887 +12888 POINT(34.08069842118335 -118.47133933961109) bank12888 +12889 POINT(34.867751665594746 -118.4469624372967) bank12889 +12890 POINT(33.113493295040804 -117.98782101498458) bank12890 +12891 POINT(33.41546090985477 -118.4512723697) bank12891 +12892 POINT(34.92098370453769 -118.26723986030574) bank12892 +12893 POINT(34.28142963347447 -118.77733756773605) bank12893 +12894 POINT(33.84985184165022 -117.82620199038736) bank12894 +12895 POINT(33.76725897831887 -117.27884260459302) bank12895 +12896 POINT(33.079807125929165 -117.27732630054925) bank12896 +12897 POINT(34.20880569533564 -118.03889218200506) bank12897 +12898 POINT(35.04278345516215 -118.10735117826285) bank12898 +12899 POINT(34.01263858064651 -119.19650144070307) bank12899 +12900 POINT(34.807482242863436 -118.70308408890489) bank12900 +12901 POINT(34.49584014290879 -117.52793119695498) bank12901 +12902 POINT(34.26681187048984 -118.13681025591326) bank12902 +12903 POINT(34.761491785795116 -118.78924519508485) bank12903 +12904 POINT(33.5740477148958 -117.41666429721266) bank12904 +12905 POINT(35.00359549804215 -118.26882099488273) bank12905 +12906 POINT(33.721906149634464 -117.82501166861316) bank12906 +12907 POINT(33.686578045993805 -117.40582823143919) bank12907 +12908 POINT(33.14434983511404 -118.3420404840953) bank12908 +12909 POINT(34.652362422201286 -117.38806528613098) bank12909 +12910 POINT(34.37833518229295 -117.6712586424915) bank12910 +12911 POINT(34.492314933705224 -117.6022343511896) bank12911 +12912 POINT(33.45009891169143 -118.66478482641726) bank12912 +12913 POINT(34.11746591873606 -117.76051532922617) bank12913 +12914 POINT(34.23201473181299 -118.73240450511712) bank12914 +12915 POINT(33.62809274393854 -118.6315719870818) bank12915 +12916 POINT(34.028517270147546 -117.49513784613015) bank12916 +12917 POINT(33.83839222160924 -118.14380242947695) bank12917 +12918 POINT(33.499516503081594 -117.69938394859417) bank12918 +12919 POINT(33.33873586917023 -118.7427622394235) bank12919 +12920 POINT(33.18409252795928 -117.51911715154517) bank12920 +12921 POINT(34.721377017786295 -118.95238767398169) bank12921 +12922 POINT(33.567002897032026 -118.91228091282105) bank12922 +12923 POINT(34.58882002014553 -117.97051630212079) bank12923 +12924 POINT(33.081189863670176 -118.00578229157203) bank12924 +12925 POINT(33.869789474376226 -118.03995017543895) bank12925 +12926 POINT(33.42771669819629 -118.3579090946321) bank12926 +12927 POINT(33.489207285833366 -118.02900048620728) bank12927 +12928 POINT(34.839390026330584 -118.99928375693577) bank12928 +12929 POINT(34.80190305425978 -118.96306129534284) bank12929 +12930 POINT(33.2487129916554 -118.50224068821042) bank12930 +12931 POINT(34.310108806910186 -117.38295227601746) bank12931 +12932 POINT(33.543230686021325 -118.97824507543453) bank12932 +12933 POINT(34.58028421944246 -117.88888080532426) bank12933 +12934 POINT(34.959746834986014 -118.72212844463937) bank12934 +12935 POINT(33.49367262367966 -118.884384508301) bank12935 +12936 POINT(33.404193898873615 -117.2762934459848) bank12936 +12937 POINT(34.40681655412913 -118.85508678211076) bank12937 +12938 POINT(34.327916020737916 -118.9918775872621) bank12938 +12939 POINT(34.66567192266404 -118.54847686297731) bank12939 +12940 POINT(34.306116268459434 -118.77725207668094) bank12940 +12941 POINT(33.78046005628346 -118.56385502708457) bank12941 +12942 POINT(35.00263217526798 -117.99953826061314) bank12942 +12943 POINT(34.80392044442766 -118.91526220406443) bank12943 +12944 POINT(33.61776472855775 -117.52253353731018) bank12944 +12945 POINT(33.26776416774241 -118.43708301502568) bank12945 +12946 POINT(33.700787804898695 -117.34426343235495) bank12946 +12947 POINT(34.47820138771386 -117.38077264647039) bank12947 +12948 POINT(34.94517587038338 -118.88099132347419) bank12948 +12949 POINT(34.49122709659814 -117.77925841387001) bank12949 +12950 POINT(33.30590524009748 -118.75976466415398) bank12950 +12951 POINT(33.67458804346837 -117.4611253449782) bank12951 +12952 POINT(34.69671121738455 -117.80647687810907) bank12952 +12953 POINT(33.59227503505557 -117.26647025527758) bank12953 +12954 POINT(34.95263004379085 -119.23813731434964) bank12954 +12955 POINT(33.363414807284364 -117.47139774985718) bank12955 +12956 POINT(34.646979281325684 -118.1123542490505) bank12956 +12957 POINT(34.45226649605937 -118.03889634905873) bank12957 +12958 POINT(34.79563766862509 -118.12864178141183) bank12958 +12959 POINT(34.37830698039249 -118.15605198877037) bank12959 +12960 POINT(33.98528881929698 -118.85355588534279) bank12960 +12961 POINT(34.736013683189455 -119.18644322516964) bank12961 +12962 POINT(34.26123379635733 -118.0075106606798) bank12962 +12963 POINT(34.41666527350429 -118.45137332352795) bank12963 +12964 POINT(33.43675447987383 -117.91119559036538) bank12964 +12965 POINT(33.78087197454702 -117.30936009560354) bank12965 +12966 POINT(34.791029298840144 -118.65214671833067) bank12966 +12967 POINT(33.41212050015977 -118.982940367001) bank12967 +12968 POINT(34.054418170231244 -118.95630544108585) bank12968 +12969 POINT(34.806086003438296 -117.61788374587815) bank12969 +12970 POINT(33.868869367116964 -118.98798585372867) bank12970 +12971 POINT(33.54410827507884 -117.63084416006853) bank12971 +12972 POINT(33.40374745221197 -117.59836446603929) bank12972 +12973 POINT(34.40457756228474 -117.61823044304215) bank12973 +12974 POINT(34.57517555117529 -117.41256333509995) bank12974 +12975 POINT(33.0899419193866 -118.0411604793523) bank12975 +12976 POINT(33.73435749256631 -118.06328691672762) bank12976 +12977 POINT(33.12973658053212 -118.52191986455954) bank12977 +12978 POINT(34.25993250771764 -119.13298855297708) bank12978 +12979 POINT(33.8436748902887 -118.25987165365693) bank12979 +12980 POINT(34.14729457842172 -118.16228543377203) bank12980 +12981 POINT(33.40045438925451 -118.74792399857358) bank12981 +12982 POINT(34.31679808996194 -118.66597442972343) bank12982 +12983 POINT(33.34962969429051 -118.13822727492833) bank12983 +12984 POINT(34.34173686170356 -118.46956021219289) bank12984 +12985 POINT(34.89244080267723 -118.85416041258517) bank12985 +12986 POINT(33.39837675248634 -118.77886589948437) bank12986 +12987 POINT(34.53589428156136 -117.50817935809954) bank12987 +12988 POINT(33.861675194189516 -117.73523760482513) bank12988 +12989 POINT(33.20116591012619 -118.98333272013404) bank12989 +12990 POINT(35.04519640205856 -117.52352498107432) bank12990 +12991 POINT(34.75640395941686 -119.09262744050392) bank12991 +12992 POINT(33.346020762338284 -117.89604017185643) bank12992 +12993 POINT(33.80516701952682 -119.039158665918) bank12993 +12994 POINT(35.03607604559085 -117.73142261723346) bank12994 +12995 POINT(34.64917150121033 -119.20747690793878) bank12995 +12996 POINT(34.963326130902324 -118.2886070634914) bank12996 +12997 POINT(34.01629366822905 -118.94660896640508) bank12997 +12998 POINT(34.9668288162646 -118.42614415201346) bank12998 +12999 POINT(33.8564091047032 -118.60191037270579) bank12999 +13000 POINT(34.31595896070944 -117.66568481809436) bank13000 +13001 POINT(34.238757935893005 -118.28423924893305) bank13001 +13002 POINT(33.62985199536606 -117.45224077099577) bank13002 +13003 POINT(34.84802876559633 -117.9278488490584) bank13003 +13004 POINT(33.17157051195935 -119.11972978821177) bank13004 +13005 POINT(34.52920438499249 -119.04049735885319) bank13005 +13006 POINT(34.33803049203826 -119.16423287099431) bank13006 +13007 POINT(34.52837869491368 -117.39816350033544) bank13007 +13008 POINT(33.72658046251039 -118.61522676871932) bank13008 +13009 POINT(34.70301608091401 -118.58612012116228) bank13009 +13010 POINT(33.059087460771515 -118.8221401899093) bank13010 +13011 POINT(33.65638908864778 -118.6974843199497) bank13011 +13012 POINT(34.86644092785795 -118.15272474187674) bank13012 +13013 POINT(33.81963583097953 -118.49509913156626) bank13013 +13014 POINT(33.18703120912647 -118.14822759589416) bank13014 +13015 POINT(33.79825857092934 -117.50888793473462) bank13015 +13016 POINT(33.55278764830003 -118.42730999227759) bank13016 +13017 POINT(33.86621048165685 -118.1193685289559) bank13017 +13018 POINT(34.859363188862154 -117.92052554260735) bank13018 +13019 POINT(34.459336886519175 -117.50811309273567) bank13019 +13020 POINT(34.60518015137387 -117.94656470251039) bank13020 +13021 POINT(33.83011855079485 -117.36875051403229) bank13021 +13022 POINT(34.86750810920447 -118.84501082133393) bank13022 +13023 POINT(33.6920612629617 -117.61577729592261) bank13023 +13024 POINT(33.558658768964264 -118.13769769629359) bank13024 +13025 POINT(33.97785106484371 -118.81670108249318) bank13025 +13026 POINT(33.82035770335928 -118.60663227472719) bank13026 +13027 POINT(33.99241175846791 -117.93314082448846) bank13027 +13028 POINT(34.90359927617973 -118.28607805636543) bank13028 +13029 POINT(34.95844530735213 -117.84660492136723) bank13029 +13030 POINT(33.723805666724864 -118.2483450146035) bank13030 +13031 POINT(33.27434698637167 -119.23902881447229) bank13031 +13032 POINT(34.840461718177195 -117.7568414541725) bank13032 +13033 POINT(33.137785768441766 -117.57348920924238) bank13033 +13034 POINT(33.794964484220586 -117.80860044561877) bank13034 +13035 POINT(34.324048408282366 -117.72340886408182) bank13035 +13036 POINT(33.80848170509749 -118.00417237893471) bank13036 +13037 POINT(34.16592927088773 -117.89189870092602) bank13037 +13038 POINT(33.548654297564056 -119.06933088224835) bank13038 +13039 POINT(33.693851760785314 -119.22689829474325) bank13039 +13040 POINT(34.02698180893523 -117.45225560975902) bank13040 +13041 POINT(34.59810291343007 -118.78334026262291) bank13041 +13042 POINT(33.84104654322333 -117.5463393996971) bank13042 +13043 POINT(34.9540123928468 -117.46844293021228) bank13043 +13044 POINT(34.41581755798339 -118.42570871051488) bank13044 +13045 POINT(34.51684926681936 -117.90670672006772) bank13045 +13046 POINT(33.81902533014973 -118.19231004070282) bank13046 +13047 POINT(34.10308138383414 -117.54160689511687) bank13047 +13048 POINT(33.452999766287704 -118.85500305805421) bank13048 +13049 POINT(35.032038158175986 -118.64056332741012) bank13049 +13050 POINT(34.84273980749262 -118.7338221678407) bank13050 +13051 POINT(33.571127171794586 -117.78841949552368) bank13051 +13052 POINT(34.29069192779884 -117.60468874546174) bank13052 +13053 POINT(33.29589916637714 -117.70912470931546) bank13053 +13054 POINT(34.52144421153105 -118.47100429247791) bank13054 +13055 POINT(34.7910521041263 -117.8493022822384) bank13055 +13056 POINT(33.15989228124464 -117.46422487776547) bank13056 +13057 POINT(33.326885984286434 -119.0844454098581) bank13057 +13058 POINT(33.56433196816222 -118.94482789761283) bank13058 +13059 POINT(34.27276789775094 -118.22743681102475) bank13059 +13060 POINT(34.33111274674103 -119.04837087828662) bank13060 +13061 POINT(33.08087918573246 -118.74042360452285) bank13061 +13062 POINT(33.86347932409935 -117.92665898338525) bank13062 +13063 POINT(33.56283668804686 -117.83910913130867) bank13063 +13064 POINT(33.97055451281996 -117.84669511070572) bank13064 +13065 POINT(33.43740150782919 -119.20400696209971) bank13065 +13066 POINT(34.669985691380155 -117.4561147641091) bank13066 +13067 POINT(34.50049498593913 -117.40655248124465) bank13067 +13068 POINT(34.602020738246715 -117.89613290780076) bank13068 +13069 POINT(34.540034172076595 -117.2998706607676) bank13069 +13070 POINT(33.345735780276094 -117.4244547158104) bank13070 +13071 POINT(33.06780919344989 -118.74539878586336) bank13071 +13072 POINT(33.5035015896599 -118.83947921007216) bank13072 +13073 POINT(33.189081735274115 -117.89220594440775) bank13073 +13074 POINT(33.73716824095653 -118.58727528738694) bank13074 +13075 POINT(33.53548800761701 -117.90473077783196) bank13075 +13076 POINT(33.20788654225721 -118.16140040653593) bank13076 +13077 POINT(33.940122475245154 -118.61968403641845) bank13077 +13078 POINT(33.282738245144465 -118.54064589398456) bank13078 +13079 POINT(34.45430927998214 -118.57206387283074) bank13079 +13080 POINT(34.12116428425406 -119.12881162140468) bank13080 +13081 POINT(33.77345372759067 -117.32627076106083) bank13081 +13082 POINT(35.02601561217684 -118.05929517054001) bank13082 +13083 POINT(33.75819273524205 -118.21253525078306) bank13083 +13084 POINT(34.09560954148475 -118.29067075233472) bank13084 +13085 POINT(34.29175120110798 -117.59441950522334) bank13085 +13086 POINT(33.433388466691774 -118.18095693160056) bank13086 +13087 POINT(33.5838086365427 -118.39143195014196) bank13087 +13088 POINT(33.72886252323585 -119.13612249446635) bank13088 +13089 POINT(34.33729946986942 -118.61035134777441) bank13089 +13090 POINT(34.19474126774684 -118.68588139097301) bank13090 +13091 POINT(33.525128911010256 -118.1238681689832) bank13091 +13092 POINT(34.12274830083507 -118.63617763407228) bank13092 +13093 POINT(34.000964990173394 -118.40402783105979) bank13093 +13094 POINT(33.58628289305201 -118.68207169565154) bank13094 +13095 POINT(33.88377907605094 -117.98000538808913) bank13095 +13096 POINT(35.01929384172821 -119.0988764370518) bank13096 +13097 POINT(34.557647151539506 -117.71484124827516) bank13097 +13098 POINT(33.327876741734144 -118.38324051610651) bank13098 +13099 POINT(34.58149285806078 -118.46013698173955) bank13099 +13100 POINT(34.790281699628856 -117.63797973805293) bank13100 +13101 POINT(33.36852913492915 -117.54987249705735) bank13101 +13102 POINT(33.38808075369147 -118.22855355222023) bank13102 +13103 POINT(33.87349762932159 -118.05279091387142) bank13103 +13104 POINT(34.490995104768146 -117.78193234659983) bank13104 +13105 POINT(33.53982293460609 -119.06778888473704) bank13105 +13106 POINT(33.11448673212455 -117.26112450431395) bank13106 +13107 POINT(34.732674743335096 -119.21783078753357) bank13107 +13108 POINT(33.615307517546775 -118.0278345090198) bank13108 +13109 POINT(34.15267647791958 -119.03810437026983) bank13109 +13110 POINT(34.784505706079464 -117.94711955856222) bank13110 +13111 POINT(34.8961996774202 -118.45679718645378) bank13111 +13112 POINT(34.094432878431924 -119.00538084549201) bank13112 +13113 POINT(35.02797374426255 -117.59284735354363) bank13113 +13114 POINT(33.67645444746958 -117.38724548907412) bank13114 +13115 POINT(33.423217384258095 -117.50609752922344) bank13115 +13116 POINT(34.89930355568449 -117.77551050972802) bank13116 +13117 POINT(34.98257769294213 -118.83415476219494) bank13117 +13118 POINT(33.33085850787935 -117.51271735466663) bank13118 +13119 POINT(33.90717641554351 -117.93821766547153) bank13119 +13120 POINT(33.54666726822382 -118.9316374096425) bank13120 +13121 POINT(34.23586790214762 -117.69156305844345) bank13121 +13122 POINT(34.168941220089714 -118.94238219637204) bank13122 +13123 POINT(33.09782454870524 -117.34421980632845) bank13123 +13124 POINT(34.307329100231506 -118.28024619265402) bank13124 +13125 POINT(33.376178690812324 -117.73687172415097) bank13125 +13126 POINT(33.33863546749361 -118.244805803791) bank13126 +13127 POINT(33.52169277390693 -117.45811933304383) bank13127 +13128 POINT(34.793167697973416 -118.00154555603194) bank13128 +13129 POINT(34.33481194857056 -118.40257958256596) bank13129 +13130 POINT(34.32178802078912 -119.04424906632659) bank13130 +13131 POINT(34.77940233815012 -119.18143572373764) bank13131 +13132 POINT(33.60205471335366 -117.27949165031164) bank13132 +13133 POINT(33.51049437187191 -118.09686649205696) bank13133 +13134 POINT(34.522773552761294 -118.15035711801181) bank13134 +13135 POINT(33.11274214520573 -118.38848727763649) bank13135 +13136 POINT(34.23937544650802 -119.13013744143562) bank13136 +13137 POINT(33.98743597465059 -118.91040956099881) bank13137 +13138 POINT(33.529887817280326 -118.69272890467843) bank13138 +13139 POINT(33.26653712288093 -117.47654111826999) bank13139 +13140 POINT(33.47296938921269 -117.76839803318308) bank13140 +13141 POINT(34.11627651309295 -117.28809898426509) bank13141 +13142 POINT(34.530771577709814 -118.72708222653635) bank13142 +13143 POINT(34.839275458606906 -118.15695002026771) bank13143 +13144 POINT(34.82740457158694 -118.7910479962517) bank13144 +13145 POINT(33.58123050318409 -117.7217965314388) bank13145 +13146 POINT(34.777420842375136 -119.06772503023707) bank13146 +13147 POINT(33.54631829390471 -117.38344625237467) bank13147 +13148 POINT(33.185100588730315 -118.87840233194329) bank13148 +13149 POINT(33.29669514702084 -118.36709641246023) bank13149 +13150 POINT(34.170354717460185 -118.54628012003236) bank13150 +13151 POINT(34.86451912703133 -118.62937225165254) bank13151 +13152 POINT(33.39797703820888 -118.6825202862909) bank13152 +13153 POINT(35.05163342863015 -117.82947448410542) bank13153 +13154 POINT(34.82575744248348 -118.70890209640451) bank13154 +13155 POINT(33.10519952750672 -117.26290206861692) bank13155 +13156 POINT(33.865219351426376 -118.22880005227906) bank13156 +13157 POINT(34.35430808012546 -117.92601387190646) bank13157 +13158 POINT(34.76823849884197 -117.57552606230179) bank13158 +13159 POINT(33.940270087104054 -118.04850543222483) bank13159 +13160 POINT(34.37069184371839 -118.10296340173346) bank13160 +13161 POINT(34.581960822015276 -117.92662977306593) bank13161 +13162 POINT(33.60207473107638 -118.14665024125792) bank13162 +13163 POINT(33.558066045508134 -119.06308457555502) bank13163 +13164 POINT(34.21614504703481 -118.54720312426896) bank13164 +13165 POINT(34.15281385959723 -117.2525402130883) bank13165 +13166 POINT(33.70101188886607 -117.59572009705174) bank13166 +13167 POINT(34.199950891456005 -117.46767318962772) bank13167 +13168 POINT(33.501416120538 -119.19558036153134) bank13168 +13169 POINT(34.41736931564737 -117.75969214446701) bank13169 +13170 POINT(33.26957799116193 -117.31081244561855) bank13170 +13171 POINT(33.23128714550644 -119.03218794610243) bank13171 +13172 POINT(34.29142766482525 -118.04120234854659) bank13172 +13173 POINT(34.568354567042036 -119.1453184481784) bank13173 +13174 POINT(34.867617247014174 -117.30486723855407) bank13174 +13175 POINT(34.83803772454422 -118.13925525306473) bank13175 +13176 POINT(33.72470174624072 -117.96575597167032) bank13176 +13177 POINT(33.41391822143109 -117.67645814540215) bank13177 +13178 POINT(34.17826757253611 -118.2258545043845) bank13178 +13179 POINT(34.82694660659714 -117.33585597090885) bank13179 +13180 POINT(33.81649356716457 -118.5310592929596) bank13180 +13181 POINT(33.189369012621334 -118.04317886968357) bank13181 +13182 POINT(33.62409607002362 -118.53024945073919) bank13182 +13183 POINT(34.746377460598026 -118.41420591647186) bank13183 +13184 POINT(33.65519275825373 -119.16470459563968) bank13184 +13185 POINT(33.77796149318044 -118.886532497777) bank13185 +13186 POINT(34.88558252377583 -118.71340472400344) bank13186 +13187 POINT(34.064494133173554 -117.42019825195435) bank13187 +13188 POINT(33.20828781910214 -118.18642614698712) bank13188 +13189 POINT(34.700117772841836 -117.9365214015114) bank13189 +13190 POINT(34.30710255500019 -118.86067726391465) bank13190 +13191 POINT(33.171685276280094 -117.80037663254964) bank13191 +13192 POINT(33.64618017651798 -117.48715984172043) bank13192 +13193 POINT(34.50183502963263 -117.90308251781624) bank13193 +13194 POINT(34.698351847280904 -119.20592984539299) bank13194 +13195 POINT(34.64219230028037 -119.00656740723933) bank13195 +13196 POINT(34.06046834227494 -117.99470856838549) bank13196 +13197 POINT(33.0790823025841 -118.05311364393887) bank13197 +13198 POINT(33.39646713118377 -118.06011761314782) bank13198 +13199 POINT(34.64654171493014 -118.96334271499012) bank13199 +13200 POINT(34.05514634943781 -117.81147766143705) bank13200 +13201 POINT(33.42525695095206 -118.20217885192517) bank13201 +13202 POINT(33.81231579682109 -118.89637790571412) bank13202 +13203 POINT(33.772446018757755 -118.8263287424004) bank13203 +13204 POINT(34.12569529555479 -118.21665317881006) bank13204 +13205 POINT(34.3687077301931 -118.43696472616313) bank13205 +13206 POINT(34.00885554208929 -118.26921537062648) bank13206 +13207 POINT(34.72476302674656 -117.55589932275903) bank13207 +13208 POINT(34.56960482350077 -119.06378156898386) bank13208 +13209 POINT(34.799848822676026 -117.9788077207208) bank13209 +13210 POINT(33.552122776504305 -119.06775884699917) bank13210 +13211 POINT(34.718782324920035 -118.29638366107397) bank13211 +13212 POINT(33.41884320863096 -118.28297697419842) bank13212 +13213 POINT(33.91219330007082 -118.31560443906723) bank13213 +13214 POINT(34.25928523096212 -118.7948862046569) bank13214 +13215 POINT(33.54495332469035 -119.04758585683665) bank13215 +13216 POINT(33.31229053817188 -118.0941338438926) bank13216 +13217 POINT(34.49554427226048 -118.20295642692452) bank13217 +13218 POINT(34.09191981611054 -117.28718670089404) bank13218 +13219 POINT(34.51322893185346 -118.92531114164352) bank13219 +13220 POINT(34.209501285263855 -118.25336807777283) bank13220 +13221 POINT(33.57929127859757 -118.19977089466175) bank13221 +13222 POINT(33.97080929829733 -117.98689425877808) bank13222 +13223 POINT(34.65652958662345 -119.05316068917381) bank13223 +13224 POINT(33.19565158703059 -118.63018370046342) bank13224 +13225 POINT(33.86107629020828 -118.81984887918016) bank13225 +13226 POINT(34.88912499018477 -119.0894373337303) bank13226 +13227 POINT(34.68544711331343 -118.96724383375991) bank13227 +13228 POINT(34.202421066707316 -119.05398413653117) bank13228 +13229 POINT(34.15410871941639 -118.17164462389465) bank13229 +13230 POINT(33.2356438925012 -118.8655663803414) bank13230 +13231 POINT(34.29885462951899 -117.54280925112741) bank13231 +13232 POINT(33.06998243997675 -117.65869211188685) bank13232 +13233 POINT(33.388049327091736 -118.9122041709867) bank13233 +13234 POINT(34.634901700086594 -117.99973879585667) bank13234 +13235 POINT(35.01234687258785 -118.57051296623789) bank13235 +13236 POINT(33.19545123308655 -118.47611669094611) bank13236 +13237 POINT(33.47346481536623 -119.23140745919952) bank13237 +13238 POINT(33.94928111979756 -118.80380215810624) bank13238 +13239 POINT(34.06642656403106 -118.67236230995292) bank13239 +13240 POINT(34.37932442432777 -117.93672746824384) bank13240 +13241 POINT(34.94953942574225 -118.05263668799998) bank13241 +13242 POINT(34.93045488108777 -118.16140505854965) bank13242 +13243 POINT(33.615331850708095 -118.05944062756062) bank13243 +13244 POINT(34.302657280862746 -118.1315864904885) bank13244 +13245 POINT(34.39077014155937 -117.51666424941787) bank13245 +13246 POINT(34.585312483146204 -117.30864622108113) bank13246 +13247 POINT(33.29767706788739 -117.61428936077108) bank13247 +13248 POINT(33.89817582251469 -118.96207538589452) bank13248 +13249 POINT(33.13753493545507 -119.07179073843022) bank13249 +13250 POINT(33.97904312599586 -118.68727116100709) bank13250 +13251 POINT(33.49175370954811 -118.46208895071494) bank13251 +13252 POINT(34.89144750230504 -117.33288038461026) bank13252 +13253 POINT(33.248768463267325 -117.4690237772449) bank13253 +13254 POINT(33.168265486435416 -118.3996435763902) bank13254 +13255 POINT(34.342871808250756 -117.38220068431805) bank13255 +13256 POINT(34.09039120029712 -117.47058947634963) bank13256 +13257 POINT(34.94165995729971 -118.7973810485955) bank13257 +13258 POINT(33.133643134218254 -119.15982888187315) bank13258 +13259 POINT(35.011203629305086 -118.5601767261967) bank13259 +13260 POINT(34.805982584722386 -117.8986683606817) bank13260 +13261 POINT(33.37459757507644 -118.62811676849299) bank13261 +13262 POINT(34.99627167322679 -117.37537094116975) bank13262 +13263 POINT(33.10424265808926 -117.98248547798207) bank13263 +13264 POINT(34.625353413777745 -118.24923218077846) bank13264 +13265 POINT(34.10795091193359 -118.02326047762976) bank13265 +13266 POINT(34.270434054136885 -118.15135305405124) bank13266 +13267 POINT(33.11469319483124 -117.6561744144928) bank13267 +13268 POINT(33.95453413450896 -119.12047118868325) bank13268 +13269 POINT(33.954114437665474 -118.43570328189834) bank13269 +13270 POINT(33.98008064006918 -118.89874727749104) bank13270 +13271 POINT(34.958067847420395 -117.8487798273653) bank13271 +13272 POINT(34.75278856835688 -119.21105343814769) bank13272 +13273 POINT(33.247398095228135 -117.44311862777703) bank13273 +13274 POINT(33.19883776772296 -118.83596439738233) bank13274 +13275 POINT(34.51460075567929 -117.75405320372448) bank13275 +13276 POINT(34.549907094459314 -119.13535744241901) bank13276 +13277 POINT(34.639604738470084 -118.84649259412035) bank13277 +13278 POINT(33.771050206576874 -117.82360832231575) bank13278 +13279 POINT(33.18183433683359 -118.78384111552585) bank13279 +13280 POINT(33.904264095492884 -117.78951863536976) bank13280 +13281 POINT(34.38341470817038 -117.81335793159683) bank13281 +13282 POINT(33.53861583984698 -117.4085749918081) bank13282 +13283 POINT(34.902541288067596 -118.09922752990387) bank13283 +13284 POINT(34.71563933660487 -118.82409367225175) bank13284 +13285 POINT(34.7452047989952 -117.48280833771004) bank13285 +13286 POINT(33.965366255044295 -119.19306251174396) bank13286 +13287 POINT(33.68829239625634 -117.93507635273448) bank13287 +13288 POINT(34.53863869654144 -117.39072737209915) bank13288 +13289 POINT(33.47126808659014 -118.92015380749062) bank13289 +13290 POINT(33.787923018307914 -119.02442834522655) bank13290 +13291 POINT(33.54447513513702 -119.14533112563402) bank13291 +13292 POINT(33.35243248398666 -118.96646460335748) bank13292 +13293 POINT(34.414497763268834 -118.66268218054084) bank13293 +13294 POINT(34.9025210760858 -118.55641322320213) bank13294 +13295 POINT(33.14057247494235 -118.1516765654726) bank13295 +13296 POINT(34.504554358483 -118.39896961603134) bank13296 +13297 POINT(34.47312623094678 -118.73748486952603) bank13297 +13298 POINT(33.99390069055891 -119.06285416375884) bank13298 +13299 POINT(34.69287499576496 -118.09587464515747) bank13299 +13300 POINT(34.97921182632359 -118.04727004712275) bank13300 +13301 POINT(33.64080395696567 -117.70417281236732) bank13301 +13302 POINT(34.45323974060193 -117.98880486616551) bank13302 +13303 POINT(34.0616192627579 -119.11022399403276) bank13303 +13304 POINT(34.258693123986504 -117.32744291000492) bank13304 +13305 POINT(34.8287587843352 -118.33599411529607) bank13305 +13306 POINT(33.77631086570559 -117.7382067061006) bank13306 +13307 POINT(33.471567586533155 -118.1993443724625) bank13307 +13308 POINT(35.009097002355624 -117.39832286013646) bank13308 +13309 POINT(33.22361202414225 -118.5565674607465) bank13309 +13310 POINT(34.27131692095282 -117.90138165396708) bank13310 +13311 POINT(33.24305949979344 -119.03055606230032) bank13311 +13312 POINT(33.370606729979 -119.18285507532066) bank13312 +13313 POINT(34.753638149949964 -118.69559435486701) bank13313 +13314 POINT(34.69873725720488 -118.3635662417539) bank13314 +13315 POINT(34.275740414284684 -117.71503149973842) bank13315 +13316 POINT(33.14454642146604 -119.07601174293787) bank13316 +13317 POINT(35.04006553197249 -117.9485462660809) bank13317 +13318 POINT(33.937523356250445 -119.06780769139506) bank13318 +13319 POINT(33.56348130078486 -118.63847544749952) bank13319 +13320 POINT(33.13377395238703 -117.75692263605306) bank13320 +13321 POINT(33.683914241491 -118.12730832554595) bank13321 +13322 POINT(34.832452800742935 -117.76527643673037) bank13322 +13323 POINT(33.22911041179015 -119.03763790871663) bank13323 +13324 POINT(33.14563350959694 -118.71631070153336) bank13324 +13325 POINT(34.998512703167776 -117.2688965983129) bank13325 +13326 POINT(33.60051622646168 -118.42373385193972) bank13326 +13327 POINT(33.786377501527326 -118.08470125163497) bank13327 +13328 POINT(34.436546445962435 -117.58161365449418) bank13328 +13329 POINT(34.35609862886425 -118.56116918377349) bank13329 +13330 POINT(34.59214281360285 -118.99201754761211) bank13330 +13331 POINT(33.8164902418068 -117.31250793625452) bank13331 +13332 POINT(34.75320212668489 -117.88381769643736) bank13332 +13333 POINT(34.89283841704488 -117.71073608370102) bank13333 +13334 POINT(34.16407307595265 -117.90164710713152) bank13334 +13335 POINT(34.84780187230493 -119.15965295755686) bank13335 +13336 POINT(33.10671042689014 -118.83168413577314) bank13336 +13337 POINT(34.30841811513301 -119.19845096483046) bank13337 +13338 POINT(33.51261682904943 -117.53140859602095) bank13338 +13339 POINT(33.91359293306242 -117.71888290120852) bank13339 +13340 POINT(34.24338084266328 -117.93128386070208) bank13340 +13341 POINT(33.41956198915106 -117.48814630319772) bank13341 +13342 POINT(33.420078636608544 -118.9037273616825) bank13342 +13343 POINT(33.12021408249439 -118.70244274280748) bank13343 +13344 POINT(34.27408721671669 -118.28374615889247) bank13344 +13345 POINT(34.78125002838034 -118.94866243514161) bank13345 +13346 POINT(33.43721551991294 -118.4468650056316) bank13346 +13347 POINT(33.165728782285306 -117.81997722444508) bank13347 +13348 POINT(34.6893521117008 -117.74113250698677) bank13348 +13349 POINT(34.87873605705903 -118.19953004054257) bank13349 +13350 POINT(33.76687667191064 -118.6365315484604) bank13350 +13351 POINT(34.96308303763156 -119.15694789559831) bank13351 +13352 POINT(34.935064520696805 -117.53748753096815) bank13352 +13353 POINT(34.07959365319091 -119.10639525594226) bank13353 +13354 POINT(34.96572811892463 -118.890654739497) bank13354 +13355 POINT(34.11270670266522 -117.51020388102634) bank13355 +13356 POINT(34.97621129850492 -117.71453622500255) bank13356 +13357 POINT(34.87721956778847 -118.98146878287127) bank13357 +13358 POINT(34.924812558700474 -118.77712792315178) bank13358 +13359 POINT(33.23067085205592 -118.34872917156004) bank13359 +13360 POINT(33.452561293987515 -118.01650664834494) bank13360 +13361 POINT(33.67373778360388 -118.60793101866291) bank13361 +13362 POINT(35.02349429886316 -118.3906946132784) bank13362 +13363 POINT(33.89531551269277 -118.18794315696005) bank13363 +13364 POINT(34.74099562512604 -117.26882799425671) bank13364 +13365 POINT(33.88685294698178 -117.4101752217823) bank13365 +13366 POINT(33.609144555675876 -119.22422733772866) bank13366 +13367 POINT(33.53661338676172 -118.15400478659915) bank13367 +13368 POINT(33.0535403536786 -118.50168953568799) bank13368 +13369 POINT(33.69820136933331 -119.07858621156782) bank13369 +13370 POINT(34.69806929715006 -119.18113589112765) bank13370 +13371 POINT(33.75847723171974 -117.75677457448302) bank13371 +13372 POINT(34.205978467274235 -118.16632228284597) bank13372 +13373 POINT(34.13666022050423 -118.03772345253645) bank13373 +13374 POINT(33.410152809913384 -118.66718092032332) bank13374 +13375 POINT(33.526805308994916 -118.32299682146078) bank13375 +13376 POINT(34.39361991763047 -119.22425978010003) bank13376 +13377 POINT(34.19501524368891 -118.56377349190059) bank13377 +13378 POINT(33.359836132966464 -118.68225752820004) bank13378 +13379 POINT(33.06172367007964 -118.93544016253787) bank13379 +13380 POINT(34.30061482131432 -117.9699820699647) bank13380 +13381 POINT(34.227801820801254 -118.73943944924174) bank13381 +13382 POINT(34.11757128737738 -118.96458136334229) bank13382 +13383 POINT(33.681936522097615 -117.25240714968209) bank13383 +13384 POINT(34.98003319489323 -119.13886858455334) bank13384 +13385 POINT(34.793472548268724 -118.75808697223758) bank13385 +13386 POINT(33.20294843942384 -118.49083815393409) bank13386 +13387 POINT(34.88517379583523 -119.19254583099122) bank13387 +13388 POINT(33.13428491504318 -117.85846908657682) bank13388 +13389 POINT(33.17902989932127 -119.1288711756949) bank13389 +13390 POINT(33.79330014925609 -118.37509547492611) bank13390 +13391 POINT(33.983631120979666 -118.85740990965989) bank13391 +13392 POINT(34.41623471415438 -117.82812061442341) bank13392 +13393 POINT(34.0976366655076 -117.6772132710065) bank13393 +13394 POINT(33.24963647585957 -117.62812648936743) bank13394 +13395 POINT(33.9042431919149 -119.04540938315556) bank13395 +13396 POINT(34.70283066305404 -118.2675800496405) bank13396 +13397 POINT(34.54671420067034 -117.76109526532936) bank13397 +13398 POINT(33.37948754423407 -118.92869908305371) bank13398 +13399 POINT(34.55384567221645 -118.59480321963171) bank13399 +13400 POINT(33.411812590308074 -118.27966641715584) bank13400 +13401 POINT(34.744286023196956 -118.31350881660254) bank13401 +13402 POINT(34.59391845369138 -118.48927531045744) bank13402 +13403 POINT(33.45601222413988 -118.89852990987211) bank13403 +13404 POINT(33.382180636349126 -118.59616069967261) bank13404 +13405 POINT(33.74389848913761 -118.93209347319437) bank13405 +13406 POINT(33.71617538718636 -118.48506476584033) bank13406 +13407 POINT(35.031954039914275 -117.71482327197917) bank13407 +13408 POINT(34.25574652029889 -119.05840348713659) bank13408 +13409 POINT(33.74692212538993 -117.33140118144843) bank13409 +13410 POINT(33.426180098236244 -118.1200116776975) bank13410 +13411 POINT(34.30065584172017 -117.79318078849478) bank13411 +13412 POINT(33.77538703928568 -117.27081851050983) bank13412 +13413 POINT(34.940206450285494 -117.67208273245383) bank13413 +13414 POINT(33.58887087680114 -117.56814838251259) bank13414 +13415 POINT(34.63960290021455 -117.97285357401053) bank13415 +13416 POINT(34.7790033766683 -119.05075879687546) bank13416 +13417 POINT(34.1964748764392 -118.90740723988839) bank13417 +13418 POINT(33.1778810122897 -118.78685158272397) bank13418 +13419 POINT(34.900565000745125 -118.93332790141135) bank13419 +13420 POINT(33.943431924773364 -118.00767291318668) bank13420 +13421 POINT(33.26290531615705 -118.80805654082775) bank13421 +13422 POINT(34.83430748542328 -118.95622333923725) bank13422 +13423 POINT(33.83166467009094 -118.48593977958056) bank13423 +13424 POINT(34.50007960540892 -117.50088279947019) bank13424 +13425 POINT(34.69430148052669 -119.1694449465817) bank13425 +13426 POINT(34.64640628295154 -118.88048584645131) bank13426 +13427 POINT(34.374314682386135 -118.01510728443007) bank13427 +13428 POINT(33.54065879595079 -117.67799443101349) bank13428 +13429 POINT(33.82717077152199 -117.80745398652839) bank13429 +13430 POINT(33.07908271419181 -118.85531873624005) bank13430 +13431 POINT(33.189785904353705 -117.94952446548417) bank13431 +13432 POINT(34.84382388137538 -117.81011932803573) bank13432 +13433 POINT(34.424812722703315 -118.10781365650297) bank13433 +13434 POINT(33.290118897811006 -117.24844069927143) bank13434 +13435 POINT(34.14985592683531 -118.74611335415857) bank13435 +13436 POINT(34.769346526158984 -117.64751811550116) bank13436 +13437 POINT(34.332340516144896 -117.74529563127234) bank13437 +13438 POINT(33.204300431430475 -118.1765368481824) bank13438 +13439 POINT(33.940316419351475 -118.1561643403356) bank13439 +13440 POINT(33.722595174717924 -117.79998627887049) bank13440 +13441 POINT(33.44995949629301 -118.11528323662368) bank13441 +13442 POINT(34.8953183913739 -119.22006702287067) bank13442 +13443 POINT(34.611573017495495 -117.81348152594802) bank13443 +13444 POINT(33.282203070437944 -119.08520568766156) bank13444 +13445 POINT(34.14417604496439 -118.7097361038696) bank13445 +13446 POINT(33.582519745075295 -117.57426820997287) bank13446 +13447 POINT(33.92703872276907 -117.47763902800847) bank13447 +13448 POINT(34.31954558590162 -119.0078263529334) bank13448 +13449 POINT(33.16343886223552 -117.61260688319365) bank13449 +13450 POINT(33.54279794997669 -118.27033005545343) bank13450 +13451 POINT(34.52591286175407 -118.79975016992178) bank13451 +13452 POINT(34.87932950533756 -118.35137435297992) bank13452 +13453 POINT(34.53980872990179 -118.82979699921552) bank13453 +13454 POINT(34.86671045294706 -117.40311635911347) bank13454 +13455 POINT(34.34182038345619 -117.52360098995186) bank13455 +13456 POINT(34.70944015771724 -118.02984463517512) bank13456 +13457 POINT(34.228460392428204 -118.10899231496214) bank13457 +13458 POINT(34.27543278922705 -117.98596871337753) bank13458 +13459 POINT(33.51927614237864 -117.34231889249635) bank13459 +13460 POINT(33.272658508062925 -119.12512599398285) bank13460 +13461 POINT(33.79995283203829 -119.00161343070526) bank13461 +13462 POINT(33.52320975483572 -118.98957975091669) bank13462 +13463 POINT(33.207593192442616 -117.92904429659876) bank13463 +13464 POINT(34.32192132391135 -118.78976756874565) bank13464 +13465 POINT(33.272193397296824 -119.12339455463109) bank13465 +13466 POINT(34.40573036351521 -118.38435600200454) bank13466 +13467 POINT(34.638828235697815 -118.95376613956182) bank13467 +13468 POINT(34.65459228608023 -118.79089223272712) bank13468 +13469 POINT(34.87037994507872 -117.71920877421523) bank13469 +13470 POINT(33.37477625319321 -118.72645576323872) bank13470 +13471 POINT(34.43062329778866 -118.4233070929407) bank13471 +13472 POINT(34.108229269699 -118.36288089120139) bank13472 +13473 POINT(34.02506457348811 -117.86687146039992) bank13473 +13474 POINT(34.85228576697816 -118.00042414409602) bank13474 +13475 POINT(34.70083611325932 -117.99501151591083) bank13475 +13476 POINT(34.84682802887529 -118.66234455238516) bank13476 +13477 POINT(34.693560825514254 -118.1713067150606) bank13477 +13478 POINT(33.16261769563022 -118.12783317654792) bank13478 +13479 POINT(33.87679863085159 -118.37932431917608) bank13479 +13480 POINT(34.56852188625038 -118.4010271772999) bank13480 +13481 POINT(33.308743595160685 -118.05082716564011) bank13481 +13482 POINT(33.53200880414153 -117.40569975400993) bank13482 +13483 POINT(33.13700336371812 -118.32249855728027) bank13483 +13484 POINT(34.277960977098736 -118.95687028255104) bank13484 +13485 POINT(33.468651944790786 -119.07519670217356) bank13485 +13486 POINT(34.573866411330265 -117.68795736223443) bank13486 +13487 POINT(34.35111017018784 -117.90515691100437) bank13487 +13488 POINT(33.54509753267997 -118.8465006526194) bank13488 +13489 POINT(33.90603799068078 -118.63159650059902) bank13489 +13490 POINT(34.79871532382087 -117.86319365272593) bank13490 +13491 POINT(34.83602798215926 -119.081872793878) bank13491 +13492 POINT(34.22969026311495 -118.60342711932442) bank13492 +13493 POINT(34.96681960156551 -118.94293324643085) bank13493 +13494 POINT(34.0604166453037 -117.44118257030395) bank13494 +13495 POINT(34.18100184242602 -118.26256586884483) bank13495 +13496 POINT(33.42532448432896 -118.09061141160161) bank13496 +13497 POINT(34.07409188217901 -117.25110912753954) bank13497 +13498 POINT(33.62945971262205 -119.10333219719853) bank13498 +13499 POINT(33.61522286493789 -118.91891309874663) bank13499 +13500 POINT(34.51643523616559 -118.78478753951488) bank13500 +13501 POINT(34.33733067748526 -117.77797707728772) bank13501 +13502 POINT(34.222811035895916 -118.059434185534) bank13502 +13503 POINT(33.755440387400476 -117.44737242608561) bank13503 +13504 POINT(33.80111400515478 -117.68866515773986) bank13504 +13505 POINT(33.623334438370364 -119.06751433800916) bank13505 +13506 POINT(33.59280261048838 -118.49001375184724) bank13506 +13507 POINT(34.25271962893178 -119.12734500911455) bank13507 +13508 POINT(34.075629127508186 -119.16053533210207) bank13508 +13509 POINT(34.238600159659036 -118.9577023029879) bank13509 +13510 POINT(33.23189105423817 -118.30281870593187) bank13510 +13511 POINT(33.3075307552124 -118.09347650409582) bank13511 +13512 POINT(33.64107891188846 -117.47503269901155) bank13512 +13513 POINT(33.97346970799472 -118.58546306166163) bank13513 +13514 POINT(33.079847831882034 -117.73678696016457) bank13514 +13515 POINT(33.31618020979014 -117.9538888585555) bank13515 +13516 POINT(33.48364064515945 -118.9426189255546) bank13516 +13517 POINT(34.240414148910695 -118.54986219457717) bank13517 +13518 POINT(34.60838389435582 -118.48135853031677) bank13518 +13519 POINT(34.30218112962274 -118.63403092762837) bank13519 +13520 POINT(34.601530449945976 -118.05408977210595) bank13520 +13521 POINT(34.70689491280419 -117.34025661849353) bank13521 +13522 POINT(34.71519363765874 -118.73432757839268) bank13522 +13523 POINT(34.494598825460756 -117.99107549024512) bank13523 +13524 POINT(33.259315519920534 -118.39549436566566) bank13524 +13525 POINT(34.3711148439167 -118.52602969702166) bank13525 +13526 POINT(33.58583459911291 -117.36721096112133) bank13526 +13527 POINT(34.018444732755825 -118.0579652210054) bank13527 +13528 POINT(33.984752942136026 -117.59045879433317) bank13528 +13529 POINT(34.97545769321747 -118.21580647997476) bank13529 +13530 POINT(34.725736535788904 -117.80975773844536) bank13530 +13531 POINT(33.85885924584584 -118.56040368256706) bank13531 +13532 POINT(34.294699000166496 -117.33317450057184) bank13532 +13533 POINT(34.81655472946361 -118.39521140915508) bank13533 +13534 POINT(33.090219925935465 -119.22776139094779) bank13534 +13535 POINT(33.39175453594979 -118.76665599988999) bank13535 +13536 POINT(33.873218959357686 -117.92395792421289) bank13536 +13537 POINT(34.29110148185712 -118.84560119971837) bank13537 +13538 POINT(33.80083236057811 -118.28533920468936) bank13538 +13539 POINT(33.176727590953504 -119.00319719854224) bank13539 +13540 POINT(33.17942753698255 -117.45851171221013) bank13540 +13541 POINT(33.1135063856139 -118.75011341825895) bank13541 +13542 POINT(33.91698019915733 -118.01182670912898) bank13542 +13543 POINT(34.26912229857221 -118.91429034638094) bank13543 +13544 POINT(34.76724014416939 -118.08999016950264) bank13544 +13545 POINT(34.83339709020434 -119.19097567793028) bank13545 +13546 POINT(34.353617380576594 -118.05027633024253) bank13546 +13547 POINT(34.36752441348546 -117.80256718121873) bank13547 +13548 POINT(34.98616460701382 -117.93148321875066) bank13548 +13549 POINT(34.90136483860337 -119.19273079076422) bank13549 +13550 POINT(34.62892131741795 -118.88615555218456) bank13550 +13551 POINT(34.26626771949686 -118.07336841971913) bank13551 +13552 POINT(33.756350902343755 -117.31928804938471) bank13552 +13553 POINT(34.608083163263075 -117.38786273215416) bank13553 +13554 POINT(34.62274688005097 -118.05770067200646) bank13554 +13555 POINT(34.41138670220007 -117.37874362267551) bank13555 +13556 POINT(33.84939070362573 -118.65108851622475) bank13556 +13557 POINT(33.10276615737822 -117.54680648630855) bank13557 +13558 POINT(34.72690834116206 -117.9862079775615) bank13558 +13559 POINT(33.34994017349766 -118.84183091970607) bank13559 +13560 POINT(33.477640700581475 -118.07147141573247) bank13560 +13561 POINT(34.688318659192575 -118.23777074553902) bank13561 +13562 POINT(35.015882201038735 -119.15771522801597) bank13562 +13563 POINT(33.098058251425584 -117.69760713770596) bank13563 +13564 POINT(33.26119973912139 -118.22742642308351) bank13564 +13565 POINT(34.70550584913437 -119.15887607325199) bank13565 +13566 POINT(34.6128315710548 -117.80970617539514) bank13566 +13567 POINT(34.72249895975999 -117.62916902893889) bank13567 +13568 POINT(33.930003788068554 -117.49697634849727) bank13568 +13569 POINT(34.18021528441385 -118.1248252371677) bank13569 +13570 POINT(34.31731539192435 -117.9145881614836) bank13570 +13571 POINT(33.18390974732728 -119.08771046258484) bank13571 +13572 POINT(33.50788920267635 -119.131512310478) bank13572 +13573 POINT(34.287502632724404 -117.82105913362386) bank13573 +13574 POINT(34.879107207044704 -118.01449876151686) bank13574 +13575 POINT(34.29453323897498 -118.33692078898844) bank13575 +13576 POINT(34.23371178013639 -119.17068645384947) bank13576 +13577 POINT(34.686919160899095 -118.09719386652476) bank13577 +13578 POINT(34.472139089076954 -119.02256665221236) bank13578 +13579 POINT(34.71722452492107 -117.67147981231658) bank13579 +13580 POINT(33.542092318587386 -118.92206290987693) bank13580 +13581 POINT(34.4626981723759 -119.06710531730543) bank13581 +13582 POINT(33.89697564549287 -118.01197175652035) bank13582 +13583 POINT(33.680634365572914 -119.09154433890879) bank13583 +13584 POINT(34.705309883436485 -117.67520684332655) bank13584 +13585 POINT(33.69715554304939 -118.03573277743574) bank13585 +13586 POINT(33.8393348416411 -118.93587105518341) bank13586 +13587 POINT(34.39769801211471 -118.11023115674264) bank13587 +13588 POINT(34.324323059246076 -118.41505157537416) bank13588 +13589 POINT(34.04867115621161 -118.40768792787554) bank13589 +13590 POINT(34.8023003074555 -118.0605205686777) bank13590 +13591 POINT(34.11190126636402 -119.02631059848943) bank13591 +13592 POINT(33.957264147962235 -119.2170909158172) bank13592 +13593 POINT(34.645834508834554 -118.82655465438069) bank13593 +13594 POINT(35.03671498906587 -118.79097123811133) bank13594 +13595 POINT(33.900609966497775 -118.04702121001678) bank13595 +13596 POINT(35.03230937372601 -118.39034214226861) bank13596 +13597 POINT(34.987730613056804 -118.7518648516552) bank13597 +13598 POINT(33.6558323491538 -118.6266337082849) bank13598 +13599 POINT(34.875314144574205 -118.40927316368575) bank13599 +13600 POINT(33.60170682427585 -118.3868477379755) bank13600 +13601 POINT(34.07560739439106 -117.81768257802892) bank13601 +13602 POINT(34.306816095886326 -118.49255798515341) bank13602 +13603 POINT(33.15914535069149 -117.86905712760932) bank13603 +13604 POINT(34.464428110272856 -117.35501685794853) bank13604 +13605 POINT(33.249776318013105 -117.29874436485612) bank13605 +13606 POINT(34.15971462178657 -117.54429171025498) bank13606 +13607 POINT(33.50456397434999 -118.56737936163324) bank13607 +13608 POINT(33.225115511623905 -118.8747416091578) bank13608 +13609 POINT(34.30904536305859 -118.92541999844389) bank13609 +13610 POINT(33.20409927916918 -117.75659551954469) bank13610 +13611 POINT(34.37883004947893 -118.07656145393581) bank13611 +13612 POINT(34.061548418504294 -117.78486052701845) bank13612 +13613 POINT(33.404712017080044 -117.47873071015734) bank13613 +13614 POINT(34.36898369897275 -118.78731921693142) bank13614 +13615 POINT(33.144556817452 -118.37607992558108) bank13615 +13616 POINT(34.80361050367728 -117.8955049151025) bank13616 +13617 POINT(34.14112551650257 -117.97600093364942) bank13617 +13618 POINT(34.96571268751046 -118.37703073327837) bank13618 +13619 POINT(34.6710272068426 -119.11084704400527) bank13619 +13620 POINT(34.02735355107471 -118.01472335240409) bank13620 +13621 POINT(33.21804857182385 -117.67606901782432) bank13621 +13622 POINT(34.20102868332781 -118.4558403037598) bank13622 +13623 POINT(33.391329474351615 -118.4850186542322) bank13623 +13624 POINT(33.65975732910203 -118.30235320684803) bank13624 +13625 POINT(33.22281527862294 -117.7618520301864) bank13625 +13626 POINT(34.31061983926415 -117.64843330796924) bank13626 +13627 POINT(33.16404863326816 -119.1077290379003) bank13627 +13628 POINT(34.83840660455268 -117.48993897282679) bank13628 +13629 POINT(34.03125846455745 -118.1484689186325) bank13629 +13630 POINT(34.85912422001175 -118.42454805964519) bank13630 +13631 POINT(34.607749290862344 -117.95982184119089) bank13631 +13632 POINT(34.95972257203519 -117.77916235766185) bank13632 +13633 POINT(33.57531759385961 -118.99066182913792) bank13633 +13634 POINT(34.37994405738129 -117.4561697009771) bank13634 +13635 POINT(33.968329839774746 -118.97855767079234) bank13635 +13636 POINT(34.45706801048762 -117.92764743906118) bank13636 +13637 POINT(33.3838810542898 -118.86593649389538) bank13637 +13638 POINT(34.21291418792786 -119.08613365403158) bank13638 +13639 POINT(35.037489436173715 -117.85133488853054) bank13639 +13640 POINT(34.17903482017337 -118.89148307298781) bank13640 +13641 POINT(34.80736451418864 -118.83772929946865) bank13641 +13642 POINT(33.87527597871477 -118.70481419921181) bank13642 +13643 POINT(34.676216084167635 -117.84340439456228) bank13643 +13644 POINT(33.68362133669878 -118.97153344494718) bank13644 +13645 POINT(34.58733742944413 -118.57390973599199) bank13645 +13646 POINT(33.89905141898295 -118.66589998346636) bank13646 +13647 POINT(33.1100847928958 -118.66542109259288) bank13647 +13648 POINT(33.76647109544462 -118.20578917218178) bank13648 +13649 POINT(34.76333664303939 -118.25996482220931) bank13649 +13650 POINT(34.749926985132674 -118.06991402529066) bank13650 +13651 POINT(34.36040004691077 -118.06758590204139) bank13651 +13652 POINT(33.85247054416495 -117.9113304205099) bank13652 +13653 POINT(34.033201907648156 -118.84155246753086) bank13653 +13654 POINT(33.86256306332504 -118.68090091323445) bank13654 +13655 POINT(34.69040009187863 -118.93070522372984) bank13655 +13656 POINT(34.31524611215171 -118.11686904289981) bank13656 +13657 POINT(33.5650554081601 -117.28418416941471) bank13657 +13658 POINT(33.72176642271711 -117.80623796742267) bank13658 +13659 POINT(34.05219442538718 -118.17795384430676) bank13659 +13660 POINT(33.7594666237607 -118.81446946452384) bank13660 +13661 POINT(34.32503823748926 -118.5932354705787) bank13661 +13662 POINT(33.57833571697502 -118.376498617701) bank13662 +13663 POINT(34.92323987150133 -118.20107636047248) bank13663 +13664 POINT(34.777287222237945 -117.781610973919) bank13664 +13665 POINT(34.755947767223454 -118.25075663973453) bank13665 +13666 POINT(34.514604886474565 -119.16995724890046) bank13666 +13667 POINT(33.245021817546494 -118.83800579569794) bank13667 +13668 POINT(33.074890490498646 -118.72223944828264) bank13668 +13669 POINT(34.98079664450002 -118.58289852883968) bank13669 +13670 POINT(34.63522987745535 -119.04014920204314) bank13670 +13671 POINT(33.121570993282866 -119.14416001553528) bank13671 +13672 POINT(34.026990610652724 -119.03138185762799) bank13672 +13673 POINT(33.45635360101155 -119.05550349285521) bank13673 +13674 POINT(34.83409163314106 -118.69516527053533) bank13674 +13675 POINT(34.823153030616595 -117.89825833967451) bank13675 +13676 POINT(33.50199713031347 -118.77339266384178) bank13676 +13677 POINT(34.78487118302463 -117.5905147095118) bank13677 +13678 POINT(33.70711195532717 -119.235738310047) bank13678 +13679 POINT(34.65047120139215 -119.12288587026842) bank13679 +13680 POINT(33.8937594934581 -118.00871891716864) bank13680 +13681 POINT(34.12173247220227 -118.66737831428927) bank13681 +13682 POINT(34.79610988408038 -118.32860957445794) bank13682 +13683 POINT(33.419481476457236 -117.34687905828216) bank13683 +13684 POINT(33.72091675102628 -117.33595876506337) bank13684 +13685 POINT(33.845944380975226 -118.04252429484225) bank13685 +13686 POINT(34.19296297310565 -117.49091994280292) bank13686 +13687 POINT(33.459666410114146 -117.87923653757848) bank13687 +13688 POINT(34.3025194454687 -117.98226378660443) bank13688 +13689 POINT(34.81655117782855 -117.56399893859341) bank13689 +13690 POINT(34.202101132431864 -118.17300922966237) bank13690 +13691 POINT(33.55504598553296 -118.7460228639127) bank13691 +13692 POINT(34.58081778991911 -119.15249089702188) bank13692 +13693 POINT(34.50128117560862 -117.3381987675632) bank13693 +13694 POINT(34.156473225797676 -118.28227270052304) bank13694 +13695 POINT(34.126960036993154 -118.72770032148401) bank13695 +13696 POINT(33.50568859126486 -118.12740694235444) bank13696 +13697 POINT(34.84633773217489 -117.78248978854523) bank13697 +13698 POINT(34.10536542287856 -117.35925675997137) bank13698 +13699 POINT(34.47932261616864 -117.35693265342897) bank13699 +13700 POINT(33.377651179470334 -117.74982200139806) bank13700 +13701 POINT(33.63844509109025 -118.50821551622143) bank13701 +13702 POINT(34.266420353611196 -118.15416516586718) bank13702 +13703 POINT(34.776470041773976 -117.47328452408983) bank13703 +13704 POINT(34.218446563933064 -118.02007740599717) bank13704 +13705 POINT(33.99511844091141 -118.55306959467542) bank13705 +13706 POINT(34.10912489550217 -117.60606231122814) bank13706 +13707 POINT(33.175939166993444 -119.18454734680122) bank13707 +13708 POINT(33.18382418615287 -118.61511528749631) bank13708 +13709 POINT(33.94587045928295 -117.81514814626995) bank13709 +13710 POINT(34.87687117347345 -118.05484941637756) bank13710 +13711 POINT(33.967328601612934 -117.49889204825061) bank13711 +13712 POINT(34.70320724028251 -118.17652850341604) bank13712 +13713 POINT(33.12806962718961 -117.87530336501204) bank13713 +13714 POINT(34.763877560221395 -118.65710210546982) bank13714 +13715 POINT(33.22105149117708 -117.3245949360754) bank13715 +13716 POINT(34.09741658679849 -118.9139479228863) bank13716 +13717 POINT(34.53502477264315 -119.23449014774991) bank13717 +13718 POINT(34.858417348185014 -118.04262875120287) bank13718 +13719 POINT(33.39185235856889 -118.3764046344999) bank13719 +13720 POINT(33.79678001757303 -117.28911592348405) bank13720 +13721 POINT(33.87834770432617 -119.15340980016695) bank13721 +13722 POINT(33.575968266065196 -119.08599304136416) bank13722 +13723 POINT(33.47489392084354 -117.32616549042636) bank13723 +13724 POINT(33.96190783534008 -117.53511875397838) bank13724 +13725 POINT(33.35213444520493 -118.49196572489649) bank13725 +13726 POINT(33.06229801837108 -119.1917259704447) bank13726 +13727 POINT(33.424687222403115 -119.09850427473366) bank13727 +13728 POINT(33.683622304439005 -119.16560000603363) bank13728 +13729 POINT(33.74730757659148 -118.1143973171935) bank13729 +13730 POINT(33.06089017200284 -118.05234031812913) bank13730 +13731 POINT(33.323267118414286 -118.51499724342779) bank13731 +13732 POINT(34.81388579147958 -117.98053306282121) bank13732 +13733 POINT(35.03497215107447 -118.96955938917218) bank13733 +13734 POINT(34.98304999476988 -118.4946963943906) bank13734 +13735 POINT(34.54530904757756 -118.31041935100185) bank13735 +13736 POINT(34.35100780136864 -117.30836772266778) bank13736 +13737 POINT(34.59882529541804 -117.43262946567313) bank13737 +13738 POINT(34.303804115970316 -119.21039887508289) bank13738 +13739 POINT(34.59588037582491 -118.27114545117493) bank13739 +13740 POINT(34.37607283334628 -118.29895163693398) bank13740 +13741 POINT(34.743688457422785 -118.70826257952876) bank13741 +13742 POINT(34.552657293567734 -117.61178226146444) bank13742 +13743 POINT(34.33416542435618 -117.87052408876225) bank13743 +13744 POINT(34.27583720244541 -119.10201617259291) bank13744 +13745 POINT(34.73362549141839 -118.48267464040568) bank13745 +13746 POINT(35.05124673683568 -118.47932057112587) bank13746 +13747 POINT(34.26429970713633 -118.9494958722387) bank13747 +13748 POINT(33.08460546154043 -118.39966627905962) bank13748 +13749 POINT(34.65266856049695 -118.77062133618203) bank13749 +13750 POINT(34.75126768130612 -118.98985805194323) bank13750 +13751 POINT(34.35863386852059 -118.18832897417576) bank13751 +13752 POINT(33.64805827074456 -117.42985544428136) bank13752 +13753 POINT(34.154088796716344 -118.76439873271772) bank13753 +13754 POINT(34.023731735401164 -117.50957406825667) bank13754 +13755 POINT(34.55013239204838 -119.22575784464313) bank13755 +13756 POINT(34.81670812751112 -118.21135274602936) bank13756 +13757 POINT(33.91474870082157 -118.24439303552292) bank13757 +13758 POINT(34.411943374663664 -118.68314576420111) bank13758 +13759 POINT(33.14163671083499 -118.96227575127017) bank13759 +13760 POINT(34.22801897907758 -117.87675813783484) bank13760 +13761 POINT(34.26347632779153 -118.44165623502693) bank13761 +13762 POINT(34.381090949851654 -119.18760728690134) bank13762 +13763 POINT(33.49549353532014 -118.1454325303434) bank13763 +13764 POINT(34.678504470044786 -117.34036677464532) bank13764 +13765 POINT(33.846020159605736 -119.07035575462787) bank13765 +13766 POINT(33.28050733570635 -118.2805841436777) bank13766 +13767 POINT(33.22266647484014 -117.7921149397717) bank13767 +13768 POINT(33.17904505388654 -117.76901665647499) bank13768 +13769 POINT(33.87306644872629 -119.12587410375882) bank13769 +13770 POINT(33.93511447109291 -117.85256147103135) bank13770 +13771 POINT(33.72082543891243 -117.65849215182975) bank13771 +13772 POINT(33.06155998898295 -118.14542838132049) bank13772 +13773 POINT(33.229609552018545 -117.34613095997791) bank13773 +13774 POINT(34.558171610710055 -117.4080462089502) bank13774 +13775 POINT(33.43176376188441 -118.14717820269766) bank13775 +13776 POINT(33.787192507520025 -118.67247270011414) bank13776 +13777 POINT(34.07752881698396 -117.6774473020339) bank13777 +13778 POINT(34.49548497653063 -117.42455231824725) bank13778 +13779 POINT(34.4050226463788 -119.0664615397196) bank13779 +13780 POINT(33.81441476104449 -117.71917676224633) bank13780 +13781 POINT(33.165997215632856 -117.4352593448551) bank13781 +13782 POINT(34.48346928766319 -117.80560256149963) bank13782 +13783 POINT(34.16117676919102 -119.04014076071383) bank13783 +13784 POINT(34.59158905433914 -119.23587962163889) bank13784 +13785 POINT(33.76921427389531 -118.92776243224567) bank13785 +13786 POINT(33.147944468207406 -119.21159146967746) bank13786 +13787 POINT(33.28361508781578 -117.37976172284525) bank13787 +13788 POINT(33.97864604874548 -118.87475846996249) bank13788 +13789 POINT(34.91212109546452 -117.40625007768071) bank13789 +13790 POINT(34.099398376062986 -119.00092205231672) bank13790 +13791 POINT(33.25490003344811 -118.77966298478175) bank13791 +13792 POINT(33.08982177391371 -119.01317446146804) bank13792 +13793 POINT(34.733274566895204 -118.45591273370736) bank13793 +13794 POINT(33.9516097032999 -118.55742391583223) bank13794 +13795 POINT(34.92866129572431 -117.2566915916357) bank13795 +13796 POINT(34.393224113139595 -117.46040586216111) bank13796 +13797 POINT(34.03817949538605 -117.82975097567682) bank13797 +13798 POINT(34.02995538763538 -117.32162218916105) bank13798 +13799 POINT(33.841508288970225 -118.54240018756477) bank13799 +13800 POINT(34.32986556626994 -117.43747659113802) bank13800 +13801 POINT(33.43688365893247 -118.13278302872402) bank13801 +13802 POINT(34.20079412671177 -118.52522000335398) bank13802 +13803 POINT(33.652325241435946 -118.04388163424633) bank13803 +13804 POINT(33.81854556915263 -118.53196616608413) bank13804 +13805 POINT(34.892477925747215 -117.54962628869738) bank13805 +13806 POINT(34.896740863851576 -117.30546589836916) bank13806 +13807 POINT(34.61238989850331 -119.05382406767204) bank13807 +13808 POINT(33.08444985004107 -117.96419737802488) bank13808 +13809 POINT(34.610425766061034 -118.08494878614805) bank13809 +13810 POINT(33.59195645719441 -118.65206665183007) bank13810 +13811 POINT(33.20878473315872 -117.68125906650987) bank13811 +13812 POINT(34.752245451722786 -118.61682037643529) bank13812 +13813 POINT(33.288536032708876 -118.30121517490727) bank13813 +13814 POINT(33.149592196004896 -118.26181356032683) bank13814 +13815 POINT(34.70390815043002 -117.35733219913952) bank13815 +13816 POINT(34.80933012812669 -118.3718741784864) bank13816 +13817 POINT(33.89030658136269 -117.805487629176) bank13817 +13818 POINT(34.30472587760243 -117.42940661229972) bank13818 +13819 POINT(33.38251405694157 -118.51559618599562) bank13819 +13820 POINT(33.37650723693081 -118.03583196385529) bank13820 +13821 POINT(34.412385678468816 -117.9016224987523) bank13821 +13822 POINT(33.48627454033411 -117.71977160199022) bank13822 +13823 POINT(33.86509180277122 -118.70090176182944) bank13823 +13824 POINT(34.026779274475864 -117.25346413166058) bank13824 +13825 POINT(33.924191225774365 -118.17236289588895) bank13825 +13826 POINT(33.60231272872085 -118.79148171220798) bank13826 +13827 POINT(35.01404419647669 -117.34532829533212) bank13827 +13828 POINT(33.643044930828594 -118.46386371154452) bank13828 +13829 POINT(34.30195033066895 -119.19450486663901) bank13829 +13830 POINT(34.99993484586878 -119.2344199733625) bank13830 +13831 POINT(33.194784992112766 -118.01402685648928) bank13831 +13832 POINT(34.459561884423465 -119.0547219893722) bank13832 +13833 POINT(33.9723067556777 -118.06117681455353) bank13833 +13834 POINT(33.895316733523394 -118.3049309066446) bank13834 +13835 POINT(34.0652595809981 -119.03574578859329) bank13835 +13836 POINT(33.05858255770176 -118.49769562626066) bank13836 +13837 POINT(33.37950980872721 -117.64083519034612) bank13837 +13838 POINT(34.57222671580143 -118.83271589269108) bank13838 +13839 POINT(34.84103984897981 -117.86817421421598) bank13839 +13840 POINT(34.203924084405806 -118.01246549858921) bank13840 +13841 POINT(34.710140707067524 -118.21778774294307) bank13841 +13842 POINT(34.66368116160804 -118.45196257193216) bank13842 +13843 POINT(34.796019597659374 -118.37506962548164) bank13843 +13844 POINT(33.92804021861849 -118.00861549898472) bank13844 +13845 POINT(34.25892072608006 -118.46050609603992) bank13845 +13846 POINT(34.09430435112354 -117.29467667719807) bank13846 +13847 POINT(34.2813045645714 -118.85916652318942) bank13847 +13848 POINT(33.597018859337844 -118.23356159733315) bank13848 +13849 POINT(34.459683702621994 -118.43713056253914) bank13849 +13850 POINT(33.917942345360366 -119.24199574402223) bank13850 +13851 POINT(33.23846896028071 -118.529470492281) bank13851 +13852 POINT(34.7122739102483 -118.973542170002) bank13852 +13853 POINT(33.338862981015744 -117.46468656242651) bank13853 +13854 POINT(34.34579259995279 -118.37226197345616) bank13854 +13855 POINT(34.84978006714743 -119.23365047920672) bank13855 +13856 POINT(34.01385305682141 -118.33031484459904) bank13856 +13857 POINT(33.61613253145832 -117.37465039227015) bank13857 +13858 POINT(33.779547053916986 -117.37004133353224) bank13858 +13859 POINT(34.8056854188086 -118.26848291262816) bank13859 +13860 POINT(34.844562894582324 -119.1268179475581) bank13860 +13861 POINT(34.83836335009911 -118.43702038310651) bank13861 +13862 POINT(34.17405551768393 -119.23485103314707) bank13862 +13863 POINT(33.333063603674034 -118.33132995650075) bank13863 +13864 POINT(33.805541330875286 -118.16833749556277) bank13864 +13865 POINT(33.478998683169195 -117.60350194826765) bank13865 +13866 POINT(33.1227807288724 -118.77399687467124) bank13866 +13867 POINT(34.0976632839341 -118.55120303490165) bank13867 +13868 POINT(33.32006973113417 -117.75688181649686) bank13868 +13869 POINT(34.68058594109104 -119.08467392537015) bank13869 +13870 POINT(34.913901379822526 -117.28124104199796) bank13870 +13871 POINT(33.66244738350389 -118.33329007132615) bank13871 +13872 POINT(34.63218415868016 -117.84884765955215) bank13872 +13873 POINT(34.610949350821386 -118.7766744285325) bank13873 +13874 POINT(33.3532606208421 -119.18017616026575) bank13874 +13875 POINT(34.953578754255005 -118.00372101677945) bank13875 +13876 POINT(33.59630108390561 -118.65192160281592) bank13876 +13877 POINT(34.269705202997855 -118.15898005246501) bank13877 +13878 POINT(34.37672795231279 -117.81622154504281) bank13878 +13879 POINT(33.237398915136446 -118.01773504778612) bank13879 +13880 POINT(34.66560913785538 -117.59924914438506) bank13880 +13881 POINT(34.98999290250641 -117.4119168112827) bank13881 +13882 POINT(34.54504954769023 -117.42477623301374) bank13882 +13883 POINT(34.11918769786165 -117.54906212597855) bank13883 +13884 POINT(33.8844969063175 -118.07789802741866) bank13884 +13885 POINT(33.38081874918 -118.95537966646212) bank13885 +13886 POINT(34.01006546724494 -118.63406905134711) bank13886 +13887 POINT(33.44937080654601 -118.28868365696374) bank13887 +13888 POINT(34.90734861909897 -118.26613736284288) bank13888 +13889 POINT(34.30948560505925 -117.60648844687017) bank13889 +13890 POINT(34.81330106260611 -118.61938840611529) bank13890 +13891 POINT(33.26384329992655 -117.97809098232831) bank13891 +13892 POINT(33.4775350265311 -118.74469190202123) bank13892 +13893 POINT(34.960690872013785 -118.68214851148579) bank13893 +13894 POINT(34.46833162735704 -117.54938275764114) bank13894 +13895 POINT(33.44148922734062 -118.57904252527511) bank13895 +13896 POINT(34.115494180941354 -118.23056737847139) bank13896 +13897 POINT(34.212987370171604 -118.47423766929248) bank13897 +13898 POINT(34.95829249453957 -119.14045063661382) bank13898 +13899 POINT(34.24162898235534 -117.2828826829852) bank13899 +13900 POINT(33.32669612691704 -119.06737599575928) bank13900 +13901 POINT(34.585291518458384 -118.69775296223968) bank13901 +13902 POINT(33.82308802954254 -117.82742526181801) bank13902 +13903 POINT(33.302502348547485 -118.80652170506558) bank13903 +13904 POINT(33.3387337424684 -118.67053035104499) bank13904 +13905 POINT(34.864033251087406 -118.31290662357976) bank13905 +13906 POINT(33.64843831626202 -118.81635084930627) bank13906 +13907 POINT(34.28182658616005 -118.93900321764588) bank13907 +13908 POINT(34.84949108489799 -117.25243742416326) bank13908 +13909 POINT(34.652017108656906 -118.49939250473938) bank13909 +13910 POINT(33.084671332935514 -117.76402374896212) bank13910 +13911 POINT(34.57691334885032 -117.66538703699776) bank13911 +13912 POINT(34.64350729332578 -117.96822014779154) bank13912 +13913 POINT(33.78523499057897 -118.87530722185596) bank13913 +13914 POINT(33.34910974379426 -117.93123876433734) bank13914 +13915 POINT(34.580377941424004 -117.29720119141673) bank13915 +13916 POINT(33.95210774106673 -118.52408071376333) bank13916 +13917 POINT(33.27311704155599 -117.59849873801474) bank13917 +13918 POINT(34.789258626838105 -117.97600831774608) bank13918 +13919 POINT(34.83443029493076 -118.65293962887613) bank13919 +13920 POINT(33.48984474676129 -118.47379037454812) bank13920 +13921 POINT(33.061389178948865 -118.50452338303884) bank13921 +13922 POINT(33.94672451724565 -117.50682501538918) bank13922 +13923 POINT(34.089090129821116 -118.21540818218155) bank13923 +13924 POINT(33.06668458677895 -117.27448988631356) bank13924 +13925 POINT(33.22126339380207 -117.54112054449618) bank13925 +13926 POINT(33.557432654131055 -118.49221137070538) bank13926 +13927 POINT(33.94806665792688 -117.63334125686457) bank13927 +13928 POINT(33.567054484330725 -118.90394139898797) bank13928 +13929 POINT(33.54313844564178 -118.71953126651323) bank13929 +13930 POINT(34.07357241803322 -118.24015903758814) bank13930 +13931 POINT(34.658358779224145 -118.14259545193846) bank13931 +13932 POINT(33.07213402738158 -119.20681720434023) bank13932 +13933 POINT(34.82094261450014 -119.03724837987609) bank13933 +13934 POINT(33.274114752032695 -118.15758191559627) bank13934 +13935 POINT(33.97239242086591 -119.06359971368664) bank13935 +13936 POINT(33.776531108087866 -118.74443375009974) bank13936 +13937 POINT(34.38030240490201 -117.85427784590279) bank13937 +13938 POINT(33.10772908118544 -118.45842211967721) bank13938 +13939 POINT(33.38722799133073 -118.90943849575305) bank13939 +13940 POINT(34.37907009859255 -119.17004838432887) bank13940 +13941 POINT(33.65599890824871 -118.83608125622416) bank13941 +13942 POINT(33.76006572317831 -118.7832223226156) bank13942 +13943 POINT(34.990980640249745 -119.07451870519458) bank13943 +13944 POINT(34.85412056359343 -117.78712720463615) bank13944 +13945 POINT(33.45597562457895 -117.41450600493577) bank13945 +13946 POINT(34.415867541414244 -117.2837380690234) bank13946 +13947 POINT(33.662254318355124 -119.13181303388478) bank13947 +13948 POINT(33.4490433048196 -118.71740319088985) bank13948 +13949 POINT(34.17976017444695 -118.47267375112375) bank13949 +13950 POINT(33.9685778464576 -117.87261242064122) bank13950 +13951 POINT(33.5887317880899 -118.68229523833564) bank13951 +13952 POINT(33.166442455562624 -117.26759772344887) bank13952 +13953 POINT(34.177667627357714 -117.60446142127053) bank13953 +13954 POINT(33.33541031898074 -118.82981378174983) bank13954 +13955 POINT(33.82214446046686 -117.49295064920673) bank13955 +13956 POINT(34.22908901815781 -119.1191562596455) bank13956 +13957 POINT(33.18389648176134 -118.1637511190892) bank13957 +13958 POINT(33.93350501216269 -118.67770284315161) bank13958 +13959 POINT(34.89125268372001 -117.5685242859186) bank13959 +13960 POINT(33.5226586078889 -119.03680281289401) bank13960 +13961 POINT(34.92492787944946 -117.48006039114568) bank13961 +13962 POINT(34.382989077846645 -117.49195349613531) bank13962 +13963 POINT(34.515501235577375 -117.77974808109053) bank13963 +13964 POINT(34.87013893122006 -117.3311703100143) bank13964 +13965 POINT(34.344417207622236 -117.38962187649405) bank13965 +13966 POINT(33.98244116644576 -117.59555760179676) bank13966 +13967 POINT(33.88399761342796 -118.36566980869311) bank13967 +13968 POINT(33.86578024933287 -117.35336178408612) bank13968 +13969 POINT(34.152090843411216 -117.6471103508416) bank13969 +13970 POINT(34.87057527203535 -118.10378583400204) bank13970 +13971 POINT(34.400743323045354 -118.7644652992174) bank13971 +13972 POINT(33.87578330423967 -117.42564835654368) bank13972 +13973 POINT(33.66244370269787 -118.67959697640336) bank13973 +13974 POINT(34.62944181920064 -117.34059124527114) bank13974 +13975 POINT(34.025814316912715 -118.98540296767207) bank13975 +13976 POINT(33.96686549309297 -118.47939904453499) bank13976 +13977 POINT(33.65297491746412 -118.36799154775348) bank13977 +13978 POINT(34.19044417703262 -117.6804922027185) bank13978 +13979 POINT(34.061149154512535 -117.32396644164177) bank13979 +13980 POINT(33.88012421397827 -117.66173915842171) bank13980 +13981 POINT(33.10483489708622 -118.81292446693551) bank13981 +13982 POINT(34.74336818468303 -118.11262236933068) bank13982 +13983 POINT(33.100618384033055 -119.09206056013122) bank13983 +13984 POINT(34.69878569001721 -117.96444289556166) bank13984 +13985 POINT(34.44988517963524 -118.03922763763663) bank13985 +13986 POINT(33.38391572350786 -118.99121989855738) bank13986 +13987 POINT(34.2204227828171 -119.14401360084925) bank13987 +13988 POINT(33.73397953092865 -117.43754997544255) bank13988 +13989 POINT(34.9304818486684 -118.95427081319968) bank13989 +13990 POINT(34.88105639194239 -117.4327814669314) bank13990 +13991 POINT(33.43149714946605 -118.258468215938) bank13991 +13992 POINT(33.78963008966409 -118.94890485485605) bank13992 +13993 POINT(34.4650431915549 -117.79340055349799) bank13993 +13994 POINT(33.247461565279785 -118.08116105385841) bank13994 +13995 POINT(34.15575750354267 -117.40975978704081) bank13995 +13996 POINT(33.247739393091685 -118.89373302505986) bank13996 +13997 POINT(33.281718242957574 -118.51385596222553) bank13997 +13998 POINT(33.912209572122485 -118.22910518410208) bank13998 +13999 POINT(34.35670267454701 -118.4141932222046) bank13999 +14000 POINT(34.59441041678059 -117.37281217893477) bank14000 +14001 POINT(33.91544867157788 -118.06355034558723) bank14001 +14002 POINT(33.38761430642249 -118.76927811823718) bank14002 +14003 POINT(34.29324330504432 -118.31594057266304) bank14003 +14004 POINT(34.913695155193835 -118.43423143265619) bank14004 +14005 POINT(34.848189440327104 -117.62229412923298) bank14005 +14006 POINT(34.832314862849636 -118.79776246276816) bank14006 +14007 POINT(34.187068555636465 -117.77177223169619) bank14007 +14008 POINT(34.53524889360461 -118.42953943678098) bank14008 +14009 POINT(34.0943350022778 -117.3313296182652) bank14009 +14010 POINT(34.53098302336526 -118.00161768765729) bank14010 +14011 POINT(33.25107619720294 -118.34315751427951) bank14011 +14012 POINT(33.88148753664848 -117.47780955895745) bank14012 +14013 POINT(33.59038326772719 -118.94259707031316) bank14013 +14014 POINT(33.960066089033475 -117.58070878300765) bank14014 +14015 POINT(33.27128545653281 -117.95512884991463) bank14015 +14016 POINT(34.3344449970058 -118.91715187271922) bank14016 +14017 POINT(33.75929528719482 -118.29741261356739) bank14017 +14018 POINT(33.50393705964387 -117.26716709675041) bank14018 +14019 POINT(33.323713033760995 -118.57452265182779) bank14019 +14020 POINT(34.507745225068064 -118.78709837741869) bank14020 +14021 POINT(33.80310002620029 -118.81412305202024) bank14021 +14022 POINT(34.815397538390854 -118.45421484744003) bank14022 +14023 POINT(33.695597196761284 -117.49396874951266) bank14023 +14024 POINT(33.81966321688231 -118.7045156804633) bank14024 +14025 POINT(34.760058036982265 -118.13879804527511) bank14025 +14026 POINT(33.22415682322067 -117.97275596310729) bank14026 +14027 POINT(33.62706535794643 -117.2469978587621) bank14027 +14028 POINT(33.21168665813773 -118.21694990987172) bank14028 +14029 POINT(34.89015595269333 -118.00552491089071) bank14029 +14030 POINT(33.8160651408193 -117.993780954814) bank14030 +14031 POINT(33.69912401163693 -117.984614789758) bank14031 +14032 POINT(33.496993184871116 -118.46380397627883) bank14032 +14033 POINT(33.72354079128125 -119.19124629451936) bank14033 +14034 POINT(34.05541671810663 -118.7010540576986) bank14034 +14035 POINT(33.78481856233827 -117.6228150533403) bank14035 +14036 POINT(34.77846503982493 -118.94651262297931) bank14036 +14037 POINT(33.65471062744718 -117.30220542087784) bank14037 +14038 POINT(33.61695743346062 -118.09625252367374) bank14038 +14039 POINT(33.1578621954767 -117.85886520525725) bank14039 +14040 POINT(33.990738862464475 -117.42858316439883) bank14040 +14041 POINT(34.45321800487742 -117.92339931576788) bank14041 +14042 POINT(34.98291134691495 -118.36521161085297) bank14042 +14043 POINT(33.88664649136641 -118.46510217968468) bank14043 +14044 POINT(33.329657110081016 -117.7329882531101) bank14044 +14045 POINT(33.162727936340666 -117.28155328106362) bank14045 +14046 POINT(33.12982939982621 -117.7454480274056) bank14046 +14047 POINT(34.53260300991119 -118.34791221273031) bank14047 +14048 POINT(33.67490344753566 -117.58044615644084) bank14048 +14049 POINT(34.259709545775664 -118.54350175327436) bank14049 +14050 POINT(34.5683169277873 -118.43960262309513) bank14050 +14051 POINT(34.12910208895711 -117.55909541811518) bank14051 +14052 POINT(34.43360529855137 -117.72825292076814) bank14052 +14053 POINT(33.20612531719614 -118.42273481634028) bank14053 +14054 POINT(33.644165911440574 -118.77424622273843) bank14054 +14055 POINT(34.09862171351556 -118.11838988330773) bank14055 +14056 POINT(33.082894769843016 -117.42608578569168) bank14056 +14057 POINT(34.46512951210094 -117.68544388779962) bank14057 +14058 POINT(34.8650551629805 -118.42175757650949) bank14058 +14059 POINT(33.62819635029084 -118.22129747177362) bank14059 +14060 POINT(33.169050715183 -118.18700070779029) bank14060 +14061 POINT(33.335301145789494 -118.71710928772977) bank14061 +14062 POINT(34.27547896109023 -118.99494807013302) bank14062 +14063 POINT(33.76653923368847 -118.36870446821986) bank14063 +14064 POINT(33.61443086671321 -117.68027351856328) bank14064 +14065 POINT(33.53518560019297 -117.29803347743005) bank14065 +14066 POINT(33.53256141143099 -118.8899667949977) bank14066 +14067 POINT(34.34991819342097 -118.86518627160277) bank14067 +14068 POINT(34.895124107427804 -118.65173000504912) bank14068 +14069 POINT(34.04411269680329 -117.24645181560807) bank14069 +14070 POINT(33.7754596944694 -119.19606816474723) bank14070 +14071 POINT(33.308992197847104 -118.55451913590687) bank14071 +14072 POINT(34.846641838407805 -118.74222237953096) bank14072 +14073 POINT(34.17447511859231 -118.66736138801386) bank14073 +14074 POINT(33.81338687582808 -119.19978273235752) bank14074 +14075 POINT(34.470121980668694 -119.18952811694147) bank14075 +14076 POINT(33.312396441042424 -117.90101034020337) bank14076 +14077 POINT(34.98183605104363 -117.88057264137318) bank14077 +14078 POINT(33.086987621511824 -117.53810414192789) bank14078 +14079 POINT(33.717177782682754 -117.78628659430159) bank14079 +14080 POINT(34.141724194543286 -117.50885532675596) bank14080 +14081 POINT(33.45472450190557 -118.99683135682072) bank14081 +14082 POINT(34.54325708522114 -118.29916686171934) bank14082 +14083 POINT(35.007828558324 -117.85908205574981) bank14083 +14084 POINT(34.678646494565974 -117.84030792479463) bank14084 +14085 POINT(33.19855393167231 -118.42002055311515) bank14085 +14086 POINT(34.180963076674 -118.46965760692132) bank14086 +14087 POINT(34.0341936051612 -119.00245890778794) bank14087 +14088 POINT(33.19281340592569 -117.45922359326434) bank14088 +14089 POINT(34.136014183277936 -118.9925177700867) bank14089 +14090 POINT(34.34945648946422 -117.28089259235368) bank14090 +14091 POINT(33.932968263441765 -118.81275037564862) bank14091 +14092 POINT(34.829887672580966 -117.97603028457006) bank14092 +14093 POINT(33.9520726965744 -119.18415775418812) bank14093 +14094 POINT(33.38561682285811 -117.58647601727358) bank14094 +14095 POINT(33.41734951169282 -117.72012217488724) bank14095 +14096 POINT(33.66306433973354 -118.47618993613632) bank14096 +14097 POINT(34.77240409589978 -118.2319560466669) bank14097 +14098 POINT(33.12059102104341 -117.89129696639012) bank14098 +14099 POINT(33.56804310732018 -118.23794969962609) bank14099 +14100 POINT(33.97615055428363 -117.34060334166027) bank14100 +14101 POINT(34.857254642169636 -119.13459540219607) bank14101 +14102 POINT(34.64493362172428 -119.0106545426626) bank14102 +14103 POINT(34.37574190268926 -117.41471051541117) bank14103 +14104 POINT(33.188335807085444 -117.78985167077434) bank14104 +14105 POINT(34.79658917577343 -117.59771566510568) bank14105 +14106 POINT(33.114963478526825 -118.76622903739731) bank14106 +14107 POINT(33.86103266247951 -118.29516845821658) bank14107 +14108 POINT(34.22285852723141 -118.3933806061685) bank14108 +14109 POINT(33.72757456771291 -117.95251409889164) bank14109 +14110 POINT(34.65638433957908 -119.08923305218235) bank14110 +14111 POINT(33.58305431815831 -117.67780408917942) bank14111 +14112 POINT(33.86283151625718 -117.84088445690725) bank14112 +14113 POINT(34.178866838568055 -117.35685309157051) bank14113 +14114 POINT(34.81315628368064 -118.8516598351283) bank14114 +14115 POINT(34.639107719606585 -117.78699448874377) bank14115 +14116 POINT(34.54907847724583 -118.96365162916246) bank14116 +14117 POINT(34.77067405265593 -118.91701197004608) bank14117 +14118 POINT(34.834458046606336 -117.85497018426466) bank14118 +14119 POINT(34.27098093045136 -118.31614851118937) bank14119 +14120 POINT(33.903070244542874 -119.06088280131974) bank14120 +14121 POINT(33.72001270151001 -118.96025800559168) bank14121 +14122 POINT(34.6850631730104 -119.01751253806191) bank14122 +14123 POINT(33.67610647367187 -119.13425805569511) bank14123 +14124 POINT(34.61630145642886 -117.62193097000052) bank14124 +14125 POINT(34.345909627937125 -118.78738295855443) bank14125 +14126 POINT(34.07356217364072 -118.83240092351248) bank14126 +14127 POINT(33.09879920005845 -118.76042326897382) bank14127 +14128 POINT(34.55012608372767 -117.49687207407793) bank14128 +14129 POINT(34.18260794451433 -119.19716468206383) bank14129 +14130 POINT(33.30658449832751 -118.65023997281318) bank14130 +14131 POINT(33.571756029588805 -117.41840950251503) bank14131 +14132 POINT(34.94121002381796 -118.18258294455639) bank14132 +14133 POINT(33.16655051993752 -117.28116915793143) bank14133 +14134 POINT(34.497492966353 -117.26344033329462) bank14134 +14135 POINT(33.26746244883209 -118.39131230448154) bank14135 +14136 POINT(34.658803649177 -118.32944493626223) bank14136 +14137 POINT(34.25022339485834 -117.718612782591) bank14137 +14138 POINT(33.43568070011424 -117.43131050973628) bank14138 +14139 POINT(34.896720875787 -119.23096287548407) bank14139 +14140 POINT(33.6704874843476 -117.956545820047) bank14140 +14141 POINT(34.82796286154147 -118.51177350938478) bank14141 +14142 POINT(33.625148786049415 -117.60055732512458) bank14142 +14143 POINT(33.115205502104985 -118.14573869947672) bank14143 +14144 POINT(34.013029918493416 -118.3840065742783) bank14144 +14145 POINT(33.39846484094004 -119.1337452268026) bank14145 +14146 POINT(33.88614780290094 -117.64447390948284) bank14146 +14147 POINT(34.51147211376088 -117.89358216719917) bank14147 +14148 POINT(33.84473845313495 -118.84507529611035) bank14148 +14149 POINT(33.913365316204235 -118.06002776835095) bank14149 +14150 POINT(34.04347845795348 -117.88216457526416) bank14150 +14151 POINT(35.00902154516794 -119.23101068749068) bank14151 +14152 POINT(34.418635034674764 -118.46361396732665) bank14152 +14153 POINT(34.180284075002355 -117.29145485268442) bank14153 +14154 POINT(33.62858328738886 -118.90570623444339) bank14154 +14155 POINT(34.48848847982666 -119.04239614861356) bank14155 +14156 POINT(34.169795919647896 -119.1615618239923) bank14156 +14157 POINT(33.44073210793439 -117.6858398231939) bank14157 +14158 POINT(34.62407402624524 -117.58746011174732) bank14158 +14159 POINT(34.78429537822747 -117.34268798983213) bank14159 +14160 POINT(34.321616081185795 -118.1331710525223) bank14160 +14161 POINT(33.95228789060225 -118.56793982396445) bank14161 +14162 POINT(34.60541345463422 -118.2970096665764) bank14162 +14163 POINT(34.16186550261405 -118.54309563119972) bank14163 +14164 POINT(34.050257444987786 -117.84630126630334) bank14164 +14165 POINT(34.54540738765665 -118.32460011115492) bank14165 +14166 POINT(33.12127184725194 -119.18772210833293) bank14166 +14167 POINT(33.63400892881037 -118.59120985209944) bank14167 +14168 POINT(34.0210549675402 -118.12535680623641) bank14168 +14169 POINT(33.916712281711014 -118.29082773846868) bank14169 +14170 POINT(33.964359570640696 -118.78091660862903) bank14170 +14171 POINT(34.80622348562302 -118.88172295780531) bank14171 +14172 POINT(34.544065990334765 -118.61107269697737) bank14172 +14173 POINT(34.26487794729492 -118.14036089456367) bank14173 +14174 POINT(33.34461721885146 -118.88937710663946) bank14174 +14175 POINT(34.92137950959076 -118.71775643526841) bank14175 +14176 POINT(34.09648709001812 -117.55717349244782) bank14176 +14177 POINT(33.805444171330116 -117.85514614795692) bank14177 +14178 POINT(33.586133427152745 -118.48955379962581) bank14178 +14179 POINT(34.95520480765718 -117.781443666679) bank14179 +14180 POINT(34.74239469824204 -118.7358955846661) bank14180 +14181 POINT(33.43879047390584 -117.65097503780873) bank14181 +14182 POINT(33.49624690034935 -117.80273987386039) bank14182 +14183 POINT(34.81885509252702 -118.56693745457557) bank14183 +14184 POINT(33.06715085641487 -118.5565802287567) bank14184 +14185 POINT(33.58205633285412 -117.971908094091) bank14185 +14186 POINT(34.0881703248289 -117.29292672866795) bank14186 +14187 POINT(34.72763760869572 -117.77019989511797) bank14187 +14188 POINT(34.78873144567032 -118.72464939553669) bank14188 +14189 POINT(34.903690334907736 -118.01779375920279) bank14189 +14190 POINT(33.69453765031214 -117.3052178077159) bank14190 +14191 POINT(33.13399480957281 -119.06151108022004) bank14191 +14192 POINT(34.44014474745811 -118.13049585475589) bank14192 +14193 POINT(34.57986822316609 -118.08747323673633) bank14193 +14194 POINT(34.25688565998279 -118.77899685130878) bank14194 +14195 POINT(33.65456421965497 -117.6681918785013) bank14195 +14196 POINT(33.53072675850118 -119.21742005935253) bank14196 +14197 POINT(33.843420641996836 -118.30862393047772) bank14197 +14198 POINT(33.22531772778092 -118.09797738126896) bank14198 +14199 POINT(33.67928267168511 -117.39339952499596) bank14199 +14200 POINT(34.800776748160864 -118.93929347163241) bank14200 +14201 POINT(34.56746039777326 -119.15520048282491) bank14201 +14202 POINT(34.75329493714314 -118.33029090811569) bank14202 +14203 POINT(33.42342843077968 -117.8658277831467) bank14203 +14204 POINT(34.11046637226365 -118.13537298768526) bank14204 +14205 POINT(34.85628577277106 -119.13482651354587) bank14205 +14206 POINT(34.51893015163246 -118.81124878696022) bank14206 +14207 POINT(33.64057360716523 -118.86490843212813) bank14207 +14208 POINT(34.34016240039097 -117.55245727229335) bank14208 +14209 POINT(33.177834139597664 -118.22034530609326) bank14209 +14210 POINT(33.40203175700731 -117.31930276948533) bank14210 +14211 POINT(33.96396516500741 -118.0409457689631) bank14211 +14212 POINT(34.26392128220001 -117.43782698240561) bank14212 +14213 POINT(33.44148904172214 -118.43362521511654) bank14213 +14214 POINT(34.05462498895719 -118.48537397460737) bank14214 +14215 POINT(33.40485014552884 -118.41835594812909) bank14215 +14216 POINT(34.624446593285604 -119.18313193115887) bank14216 +14217 POINT(34.07233712998926 -117.60574796630618) bank14217 +14218 POINT(34.94271749925391 -118.15934310368273) bank14218 +14219 POINT(33.56347588588852 -117.27336654263846) bank14219 +14220 POINT(33.32624448157359 -118.20339282991155) bank14220 +14221 POINT(34.74865761249495 -117.61204105897414) bank14221 +14222 POINT(35.0074596291929 -118.14169657903327) bank14222 +14223 POINT(33.448048858161464 -118.51944414236651) bank14223 +14224 POINT(33.72177074832657 -117.60736619706012) bank14224 +14225 POINT(34.696109236953646 -118.29517965551551) bank14225 +14226 POINT(33.81156868747941 -118.93934971086146) bank14226 +14227 POINT(33.21910050558165 -118.76977068027512) bank14227 +14228 POINT(34.86825490582346 -118.57821176823731) bank14228 +14229 POINT(33.83848017213452 -118.17274628445821) bank14229 +14230 POINT(33.129013001742756 -118.21763986431891) bank14230 +14231 POINT(33.44006911062956 -117.94800053630318) bank14231 +14232 POINT(34.18596334510665 -119.04445074649848) bank14232 +14233 POINT(33.778032654966445 -117.53009793026091) bank14233 +14234 POINT(34.70457544599926 -117.57519145745373) bank14234 +14235 POINT(33.64757617303244 -117.27124906710543) bank14235 +14236 POINT(33.66552419972785 -118.68038158650515) bank14236 +14237 POINT(33.820715182778045 -118.49983929255399) bank14237 +14238 POINT(34.06858157682956 -118.12397385038643) bank14238 +14239 POINT(33.96509646834374 -118.77958885609189) bank14239 +14240 POINT(34.31771379942437 -117.4307104882108) bank14240 +14241 POINT(34.69513989300285 -119.0988354763727) bank14241 +14242 POINT(33.20284155866408 -118.6546195004318) bank14242 +14243 POINT(34.53097850878145 -117.66374198848708) bank14243 +14244 POINT(34.484144304704145 -117.25635751307293) bank14244 +14245 POINT(34.40203066006938 -118.7927686971363) bank14245 +14246 POINT(33.7197464645353 -118.43027786557309) bank14246 +14247 POINT(34.58966310096306 -118.78396514894949) bank14247 +14248 POINT(33.91255595077933 -118.35486259547977) bank14248 +14249 POINT(34.90689173418742 -117.2892397960937) bank14249 +14250 POINT(34.10467875693537 -118.54150592329597) bank14250 +14251 POINT(34.22063476039557 -119.02893819715797) bank14251 +14252 POINT(34.08809251414605 -118.21436323148819) bank14252 +14253 POINT(34.39147420857894 -117.80289046553652) bank14253 +14254 POINT(33.59708238471029 -118.28174171986655) bank14254 +14255 POINT(33.38720317703836 -118.56054291598416) bank14255 +14256 POINT(33.18055585039136 -117.54800997720297) bank14256 +14257 POINT(34.17312336788183 -117.67883701355264) bank14257 +14258 POINT(33.09037655188628 -118.60097914334085) bank14258 +14259 POINT(33.284595776381984 -117.37615343971619) bank14259 +14260 POINT(34.64663152621168 -118.50052214852599) bank14260 +14261 POINT(34.97750905567856 -118.46992387722118) bank14261 +14262 POINT(35.03702248893903 -117.917891476981) bank14262 +14263 POINT(33.812413693206814 -119.05587962978161) bank14263 +14264 POINT(33.811014843415585 -118.10164084444887) bank14264 +14265 POINT(34.03103400021186 -119.00740473328985) bank14265 +14266 POINT(33.938142625457665 -118.74211706609026) bank14266 +14267 POINT(34.95502792948887 -117.92695635784814) bank14267 +14268 POINT(34.35266089373303 -119.14282765599866) bank14268 +14269 POINT(35.02219477902129 -118.74807358260317) bank14269 +14270 POINT(34.03407830423257 -118.58212682822854) bank14270 +14271 POINT(34.52468498456453 -117.41914494301116) bank14271 +14272 POINT(33.397312536307496 -117.30386807450205) bank14272 +14273 POINT(33.77457002344116 -118.0645259656508) bank14273 +14274 POINT(34.78252754662373 -119.21419587617568) bank14274 +14275 POINT(34.84130103046065 -118.00034535975433) bank14275 +14276 POINT(34.31323735113552 -119.05247177697102) bank14276 +14277 POINT(33.3926303597833 -117.46814172010866) bank14277 +14278 POINT(33.82348783010719 -118.2925931967176) bank14278 +14279 POINT(34.63544394743285 -117.90279715326672) bank14279 +14280 POINT(34.94270344635257 -117.38353641692221) bank14280 +14281 POINT(33.399659881391365 -118.25546380068508) bank14281 +14282 POINT(33.232475567463005 -118.84198013850146) bank14282 +14283 POINT(34.00074963383637 -117.31538478614617) bank14283 +14284 POINT(34.46880767135841 -118.43372113861425) bank14284 +14285 POINT(34.51480089321266 -118.50836122249662) bank14285 +14286 POINT(34.08293251302346 -118.33555990582158) bank14286 +14287 POINT(33.112728181203465 -118.28831424866853) bank14287 +14288 POINT(34.642056398536035 -118.95982100573461) bank14288 +14289 POINT(33.38905342247297 -118.54257694816722) bank14289 +14290 POINT(33.14674744880701 -117.8206975523101) bank14290 +14291 POINT(34.2963037847956 -119.21818968518315) bank14291 +14292 POINT(34.927125785881586 -118.16130006802268) bank14292 +14293 POINT(34.14061926820419 -117.31081463606958) bank14293 +14294 POINT(34.1875628826809 -118.14198975290725) bank14294 +14295 POINT(34.324746265312825 -118.2480227973094) bank14295 +14296 POINT(33.81640297022385 -119.12142406928314) bank14296 +14297 POINT(33.99447467535198 -117.55919917030361) bank14297 +14298 POINT(34.748465676070815 -117.65274414121895) bank14298 +14299 POINT(34.036273245644736 -117.99087691437465) bank14299 +14300 POINT(33.34873751920157 -119.14143745784429) bank14300 +14301 POINT(33.248682401363375 -118.16179568904714) bank14301 +14302 POINT(34.32096544149785 -117.63384931149206) bank14302 +14303 POINT(34.11837675864541 -117.56599868891567) bank14303 +14304 POINT(34.42588959585235 -119.10703201582847) bank14304 +14305 POINT(33.91738663511734 -118.78321007066872) bank14305 +14306 POINT(33.053606735824076 -119.1776355104585) bank14306 +14307 POINT(34.812164220934115 -119.16981319157551) bank14307 +14308 POINT(33.66939048315676 -118.37468995682002) bank14308 +14309 POINT(33.463220188892116 -118.54060031851704) bank14309 +14310 POINT(33.53464344266831 -118.0528294760089) bank14310 +14311 POINT(34.63702792377052 -117.41587737064576) bank14311 +14312 POINT(33.19444112622255 -119.00040560961588) bank14312 +14313 POINT(33.272895839115236 -118.96066308290246) bank14313 +14314 POINT(33.41308552260166 -118.09735335647673) bank14314 +14315 POINT(33.75053685209607 -118.18348412018017) bank14315 +14316 POINT(34.619644130533324 -117.75272879425904) bank14316 +14317 POINT(33.4662378443221 -118.98091110210179) bank14317 +14318 POINT(33.21426572744764 -118.20636894206092) bank14318 +14319 POINT(34.880979503498246 -119.05851055230438) bank14319 +14320 POINT(33.9842634242146 -118.6939207191669) bank14320 +14321 POINT(34.33366252889088 -117.92349380617365) bank14321 +14322 POINT(34.46470132947662 -118.5914963354216) bank14322 +14323 POINT(33.12701451859086 -118.47869056924472) bank14323 +14324 POINT(34.84056984756286 -117.81608579336906) bank14324 +14325 POINT(33.44948918849641 -118.42041463039712) bank14325 +14326 POINT(33.82210041392028 -117.63621805853552) bank14326 +14327 POINT(33.978376104480226 -118.98492106940952) bank14327 +14328 POINT(34.90446187155077 -119.1759968655334) bank14328 +14329 POINT(33.49127013081967 -119.04532160826685) bank14329 +14330 POINT(33.167701115110404 -118.57729932297518) bank14330 +14331 POINT(33.17114074618024 -119.00187834713577) bank14331 +14332 POINT(34.87325579361787 -118.72368815144107) bank14332 +14333 POINT(34.81000977757351 -119.23360991561722) bank14333 +14334 POINT(33.90120233004565 -118.67510044939397) bank14334 +14335 POINT(33.829149396491744 -117.91688946778063) bank14335 +14336 POINT(34.55408002864784 -119.21899755749259) bank14336 +14337 POINT(33.48617942169502 -117.47062237135303) bank14337 +14338 POINT(33.643761430335864 -118.42751733124102) bank14338 +14339 POINT(34.544114377748485 -118.81041149492876) bank14339 +14340 POINT(34.489249754278504 -118.33528439699283) bank14340 +14341 POINT(33.98563654292544 -118.55522297174127) bank14341 +14342 POINT(33.35324453426017 -118.54863082648484) bank14342 +14343 POINT(34.67276351567535 -117.6771610329117) bank14343 +14344 POINT(33.77349559575638 -118.74228209329573) bank14344 +14345 POINT(34.33139904032541 -117.7454653360198) bank14345 +14346 POINT(34.489480770431015 -117.99916290046602) bank14346 +14347 POINT(34.263042067655256 -118.79086860898019) bank14347 +14348 POINT(33.346412610869706 -118.68563952641996) bank14348 +14349 POINT(34.93359789422552 -118.88780187564241) bank14349 +14350 POINT(34.00617756369801 -117.64691904260476) bank14350 +14351 POINT(33.339202601166484 -118.65707827216009) bank14351 +14352 POINT(34.456492244837506 -117.69545656520765) bank14352 +14353 POINT(34.23514439282964 -118.54586758589373) bank14353 +14354 POINT(33.37005992894339 -117.580687901153) bank14354 +14355 POINT(33.97262530857085 -118.02568472015761) bank14355 +14356 POINT(33.280872595471855 -118.41336233772036) bank14356 +14357 POINT(34.16847165371299 -117.61346617972713) bank14357 +14358 POINT(34.78911922241758 -118.96204223090314) bank14358 +14359 POINT(33.14246550411928 -118.38897625959821) bank14359 +14360 POINT(34.29499526299908 -119.07353124357944) bank14360 +14361 POINT(34.4634538428228 -117.80410165955013) bank14361 +14362 POINT(33.50760998839104 -117.58202757602015) bank14362 +14363 POINT(33.151884721934564 -118.4717729840865) bank14363 +14364 POINT(33.86435567112429 -118.07565537540589) bank14364 +14365 POINT(33.69432401402999 -117.91549625957423) bank14365 +14366 POINT(33.824483460530146 -118.18550175668621) bank14366 +14367 POINT(33.98726770097995 -118.73515128620703) bank14367 +14368 POINT(34.66309274434629 -119.0328886863944) bank14368 +14369 POINT(33.33043898320203 -117.83269816969107) bank14369 +14370 POINT(33.9546701460863 -119.09806943317683) bank14370 +14371 POINT(34.59769973027123 -117.79073008263313) bank14371 +14372 POINT(34.865620267943804 -118.95991436169294) bank14372 +14373 POINT(33.69053509057658 -117.40697650887073) bank14373 +14374 POINT(33.05309912837421 -118.04442417987866) bank14374 +14375 POINT(34.00468318223351 -118.64857556223615) bank14375 +14376 POINT(34.31225729293847 -118.64289651885666) bank14376 +14377 POINT(33.9725446567655 -119.09374237830598) bank14377 +14378 POINT(34.38551164684298 -117.85466380031987) bank14378 +14379 POINT(34.90297979876473 -118.09158923356063) bank14379 +14380 POINT(33.44959089621875 -119.14212843258008) bank14380 +14381 POINT(34.183447062182815 -117.46488410210857) bank14381 +14382 POINT(34.39313868486885 -119.10664242124157) bank14382 +14383 POINT(34.66425499126865 -118.33918395017311) bank14383 +14384 POINT(33.738050406911896 -118.51431619345387) bank14384 +14385 POINT(34.20916194140896 -119.1716694538123) bank14385 +14386 POINT(33.61875293266712 -117.89170338179224) bank14386 +14387 POINT(34.788097487992665 -117.703779558607) bank14387 +14388 POINT(33.38356275401628 -118.20201022397842) bank14388 +14389 POINT(34.09939063224912 -117.59291930924431) bank14389 +14390 POINT(33.153587739278656 -118.73180533680159) bank14390 +14391 POINT(34.10166453000716 -118.30218167275854) bank14391 +14392 POINT(33.63688711308815 -119.10570269572463) bank14392 +14393 POINT(33.49190825904691 -118.46900055926838) bank14393 +14394 POINT(34.48075354417867 -118.4538960069938) bank14394 +14395 POINT(34.05876511886479 -118.96540204006952) bank14395 +14396 POINT(33.26426190429508 -118.58584857474493) bank14396 +14397 POINT(34.52481160999936 -117.64890609745952) bank14397 +14398 POINT(33.8067723166995 -117.29341357338511) bank14398 +14399 POINT(34.01969772386354 -118.97857794359575) bank14399 +14400 POINT(34.43924670172702 -117.7380905161696) bank14400 +14401 POINT(33.39776485234203 -118.16471933750842) bank14401 +14402 POINT(33.653662430035666 -118.8000475411952) bank14402 +14403 POINT(34.57244083859122 -118.47654330872604) bank14403 +14404 POINT(34.293820099810915 -117.78759878542763) bank14404 +14405 POINT(34.34755999312602 -117.39463040419957) bank14405 +14406 POINT(33.85167843236876 -118.38008318137953) bank14406 +14407 POINT(34.19102233513096 -118.19506490446821) bank14407 +14408 POINT(34.29517018286122 -117.89232167068677) bank14408 +14409 POINT(33.395643067062444 -117.40479391681895) bank14409 +14410 POINT(33.45947414227397 -118.22292376679341) bank14410 +14411 POINT(34.46108806446156 -118.7446447945823) bank14411 +14412 POINT(35.01757259937276 -117.78345015196597) bank14412 +14413 POINT(34.3913044991283 -118.87919454326129) bank14413 +14414 POINT(34.16113449379666 -118.86991600454147) bank14414 +14415 POINT(34.75475544322002 -118.51520156044086) bank14415 +14416 POINT(33.7865705084428 -118.3053505953201) bank14416 +14417 POINT(34.769393664186076 -118.9668893280688) bank14417 +14418 POINT(33.217495581242225 -119.10720076212137) bank14418 +14419 POINT(34.574340433543476 -117.47856545429158) bank14419 +14420 POINT(34.1035208064215 -117.708065566821) bank14420 +14421 POINT(34.378700281654126 -117.40651836649275) bank14421 +14422 POINT(33.32991279306409 -118.20479274129308) bank14422 +14423 POINT(34.21770553121245 -118.07854044472171) bank14423 +14424 POINT(33.68214570239989 -117.97682805163845) bank14424 +14425 POINT(33.68884952188181 -117.45449140137137) bank14425 +14426 POINT(33.70871178533249 -118.68576565419814) bank14426 +14427 POINT(33.96540881971956 -118.59756303474266) bank14427 +14428 POINT(34.03455982717471 -118.72431425349826) bank14428 +14429 POINT(34.3422469847466 -117.45036163314637) bank14429 +14430 POINT(33.300307454160524 -119.06500590246407) bank14430 +14431 POINT(33.169704064069954 -119.07429516469578) bank14431 +14432 POINT(33.8542810529538 -117.4682559169817) bank14432 +14433 POINT(34.471590698192585 -118.76911712262188) bank14433 +14434 POINT(34.53310909338999 -118.51730183391739) bank14434 +14435 POINT(33.43159074201895 -118.04362418310798) bank14435 +14436 POINT(34.2751864993393 -118.07691592606697) bank14436 +14437 POINT(34.73844172870215 -118.79962325456384) bank14437 +14438 POINT(33.15079953429663 -118.67300635330697) bank14438 +14439 POINT(34.001417676232414 -118.13209235963652) bank14439 +14440 POINT(34.2448789579869 -117.57254611840426) bank14440 +14441 POINT(33.95115752258598 -117.94138432820061) bank14441 +14442 POINT(33.389121086672816 -117.5153905330661) bank14442 +14443 POINT(34.87823728043096 -117.72695008952364) bank14443 +14444 POINT(34.64644105847457 -117.50960641913223) bank14444 +14445 POINT(34.12665375901102 -119.21735840163534) bank14445 +14446 POINT(33.602542815344094 -117.87640393783609) bank14446 +14447 POINT(33.65482838276544 -119.22845484189564) bank14447 +14448 POINT(34.431440770420274 -117.84167343315214) bank14448 +14449 POINT(33.0585438205385 -118.61567674655926) bank14449 +14450 POINT(34.162744603672216 -117.8850092166319) bank14450 +14451 POINT(33.54155193314545 -117.40498808886224) bank14451 +14452 POINT(34.429010067201936 -118.32968593846495) bank14452 +14453 POINT(34.94279838822196 -118.77290350259122) bank14453 +14454 POINT(34.09437716311117 -118.43526719826731) bank14454 +14455 POINT(34.81938161192575 -118.31436468271056) bank14455 +14456 POINT(34.6820588903484 -117.25470282921513) bank14456 +14457 POINT(33.13934617375683 -118.93733217317858) bank14457 +14458 POINT(34.99418747573529 -117.27809034286103) bank14458 +14459 POINT(34.008046579567925 -118.50374014890967) bank14459 +14460 POINT(33.56139502067248 -118.37638109868513) bank14460 +14461 POINT(34.04102502659437 -117.29027290068723) bank14461 +14462 POINT(34.81768917280967 -118.9799122011267) bank14462 +14463 POINT(34.14807147978871 -117.92305882782667) bank14463 +14464 POINT(33.47349713804868 -119.19599927338017) bank14464 +14465 POINT(33.09975235684029 -118.69521367180619) bank14465 +14466 POINT(34.00367630431673 -118.30758681858617) bank14466 +14467 POINT(34.64678791225659 -117.59451992026978) bank14467 +14468 POINT(34.483824314821014 -117.41950929600289) bank14468 +14469 POINT(34.1723279610959 -118.46459558021854) bank14469 +14470 POINT(34.62093602369058 -117.38102232936235) bank14470 +14471 POINT(34.53508126257457 -118.14565186028939) bank14471 +14472 POINT(33.89179714300549 -118.04258348479031) bank14472 +14473 POINT(34.259635264753015 -118.0667427658644) bank14473 +14474 POINT(33.9147341874822 -118.09915184561899) bank14474 +14475 POINT(34.96425883761772 -118.68066384001878) bank14475 +14476 POINT(34.565148285206725 -118.87306759695862) bank14476 +14477 POINT(34.86775077151711 -117.42998433895659) bank14477 +14478 POINT(33.808190457372376 -118.97641332891892) bank14478 +14479 POINT(33.73640482797889 -117.2458631646396) bank14479 +14480 POINT(33.31086771537334 -117.39425932196937) bank14480 +14481 POINT(34.868304629418155 -119.03249626198624) bank14481 +14482 POINT(33.29763030067523 -117.61730315835341) bank14482 +14483 POINT(34.9608128280646 -118.63714277049671) bank14483 +14484 POINT(34.407267224513085 -119.07185408870149) bank14484 +14485 POINT(33.82898107613792 -117.9667958412472) bank14485 +14486 POINT(33.98682563378918 -118.75492539739747) bank14486 +14487 POINT(34.828606800016345 -117.36428092952907) bank14487 +14488 POINT(33.714911878850046 -119.13546446519838) bank14488 +14489 POINT(34.97036449456159 -117.46136118572358) bank14489 +14490 POINT(34.357174268585524 -119.09540304047812) bank14490 +14491 POINT(34.04504724783335 -118.23923905412653) bank14491 +14492 POINT(33.38124801929342 -118.6860291447841) bank14492 +14493 POINT(34.36428878018002 -119.02146133075217) bank14493 +14494 POINT(34.17761134078488 -118.27972609982865) bank14494 +14495 POINT(34.24166784040295 -118.07328687706635) bank14495 +14496 POINT(34.16053216705123 -118.54156969426985) bank14496 +14497 POINT(33.61864365590731 -118.83908672953616) bank14497 +14498 POINT(33.768282285753266 -117.34643198928867) bank14498 +14499 POINT(34.491821235578584 -118.51282761750016) bank14499 +14500 POINT(35.03864951672095 -117.60053384467372) bank14500 +14501 POINT(34.11268594010307 -119.04708796572908) bank14501 +14502 POINT(34.21760565397086 -117.35986274303595) bank14502 +14503 POINT(33.51273631361587 -119.13130255011744) bank14503 +14504 POINT(34.50892601673579 -117.93441333033033) bank14504 +14505 POINT(33.50842954596349 -118.50556044819969) bank14505 +14506 POINT(33.42445304419978 -117.56221197728208) bank14506 +14507 POINT(34.79810777970589 -118.62076937629797) bank14507 +14508 POINT(34.27322999012545 -117.30775613631558) bank14508 +14509 POINT(33.407256752669745 -117.53444011215652) bank14509 +14510 POINT(33.38080469839927 -118.66692865147115) bank14510 +14511 POINT(34.99350012231253 -118.54247865280087) bank14511 +14512 POINT(33.6162694033851 -118.48666887150628) bank14512 +14513 POINT(34.9432219733883 -117.72590353711927) bank14513 +14514 POINT(34.201048221756665 -119.06346517710897) bank14514 +14515 POINT(34.88341609344708 -117.26857623798686) bank14515 +14516 POINT(34.87471225503149 -117.52636581882336) bank14516 +14517 POINT(33.63250801283053 -117.48467844129621) bank14517 +14518 POINT(33.15930276097545 -118.52738997141643) bank14518 +14519 POINT(34.36920434248346 -118.59901189765041) bank14519 +14520 POINT(34.001847168617616 -119.02808881200384) bank14520 +14521 POINT(34.094998584351565 -118.67711806319662) bank14521 +14522 POINT(33.379515820026526 -117.63515235946232) bank14522 +14523 POINT(35.01338898539334 -118.99053561337641) bank14523 +14524 POINT(33.76249271332013 -118.06281662116486) bank14524 +14525 POINT(33.088614597941564 -118.07135957613657) bank14525 +14526 POINT(34.90132982119685 -119.04582377140761) bank14526 +14527 POINT(34.70654609909063 -118.96297997411838) bank14527 +14528 POINT(33.33001136183673 -118.81221729807517) bank14528 +14529 POINT(35.03718916256032 -117.96082553190782) bank14529 +14530 POINT(34.361912065231984 -118.10137761883045) bank14530 +14531 POINT(34.22395629172808 -118.75143436020124) bank14531 +14532 POINT(34.848587984495744 -118.55094562302682) bank14532 +14533 POINT(33.75994378854836 -117.70028598737852) bank14533 +14534 POINT(33.261873318806636 -117.32347715130416) bank14534 +14535 POINT(34.55796832975482 -117.76912267665789) bank14535 +14536 POINT(34.11504838872323 -118.57744564767034) bank14536 +14537 POINT(33.39722532069972 -117.73886683307366) bank14537 +14538 POINT(33.94444530855952 -117.34350120781633) bank14538 +14539 POINT(33.692665271126366 -117.63808238640239) bank14539 +14540 POINT(34.8871303842046 -118.30110491504978) bank14540 +14541 POINT(35.01351095378407 -117.49411825878403) bank14541 +14542 POINT(33.89304447581642 -118.38032616077636) bank14542 +14543 POINT(33.973616545091815 -118.76080721768095) bank14543 +14544 POINT(33.124695476932125 -118.64460765757151) bank14544 +14545 POINT(34.04119399775471 -118.84609920889014) bank14545 +14546 POINT(33.54415898964287 -118.75461098348448) bank14546 +14547 POINT(34.08403198147447 -118.55322857823721) bank14547 +14548 POINT(34.73775750275749 -119.17778863709303) bank14548 +14549 POINT(34.50315709006551 -117.6083338179583) bank14549 +14550 POINT(34.743801694906054 -118.76335992935674) bank14550 +14551 POINT(33.33142729305373 -117.85322509349866) bank14551 +14552 POINT(33.13606029730417 -118.87754069655324) bank14552 +14553 POINT(33.761249988837754 -117.33098111886703) bank14553 +14554 POINT(34.328777172619695 -119.03083375835686) bank14554 +14555 POINT(33.813526862414584 -117.57173497776861) bank14555 +14556 POINT(34.27116304403306 -118.32422388320951) bank14556 +14557 POINT(34.27212819290951 -117.49323400394456) bank14557 +14558 POINT(34.89037539003925 -118.1634750828861) bank14558 +14559 POINT(34.59129824026809 -118.3098327982693) bank14559 +14560 POINT(33.993313563988934 -117.54184405860819) bank14560 +14561 POINT(34.463685890744856 -117.31489527318556) bank14561 +14562 POINT(33.71695309244159 -117.83152472569473) bank14562 +14563 POINT(34.528721454580825 -118.14878061087987) bank14563 +14564 POINT(34.632304080552466 -117.33430264663168) bank14564 +14565 POINT(33.988572293552565 -118.483366056584) bank14565 +14566 POINT(34.663200766815265 -118.08227860246707) bank14566 +14567 POINT(34.09999955686954 -118.9109344572828) bank14567 +14568 POINT(34.53200755012469 -117.46149569095846) bank14568 +14569 POINT(34.65673303064136 -119.13902772241016) bank14569 +14570 POINT(34.96656274571128 -118.28916931342427) bank14570 +14571 POINT(33.68604043334495 -118.56358756881653) bank14571 +14572 POINT(33.908550827872915 -117.38699082244051) bank14572 +14573 POINT(34.5222760951073 -118.15754879821641) bank14573 +14574 POINT(33.96760649106946 -118.0611508488047) bank14574 +14575 POINT(33.19529118165881 -118.42959426377664) bank14575 +14576 POINT(33.66715012801381 -117.47778217678784) bank14576 +14577 POINT(34.813649680121046 -117.95876339625082) bank14577 +14578 POINT(33.615994919996 -117.4077871250436) bank14578 +14579 POINT(34.99414634812688 -117.74254222639216) bank14579 +14580 POINT(34.176658152358726 -119.11318576120101) bank14580 +14581 POINT(33.85828497364839 -118.60088439606378) bank14581 +14582 POINT(34.85305052804243 -117.41374963773522) bank14582 +14583 POINT(33.99514398171764 -118.75381342416594) bank14583 +14584 POINT(34.01660981545252 -118.07016364641184) bank14584 +14585 POINT(33.78677123015358 -117.45525951540763) bank14585 +14586 POINT(33.386852736452205 -118.965355137958) bank14586 +14587 POINT(34.044249484186146 -118.27524043647479) bank14587 +14588 POINT(34.7029594150438 -118.31762972141479) bank14588 +14589 POINT(33.465126549298915 -118.21281912023545) bank14589 +14590 POINT(33.157105746067266 -118.74119152881927) bank14590 +14591 POINT(33.210680674692014 -119.01181068488707) bank14591 +14592 POINT(34.81813723230147 -117.48264453349648) bank14592 +14593 POINT(33.82223073959492 -118.16152699220422) bank14593 +14594 POINT(34.96714323086932 -118.57811182865818) bank14594 +14595 POINT(34.335617131989714 -118.6404582229793) bank14595 +14596 POINT(34.67571432639237 -118.71375781512705) bank14596 +14597 POINT(33.33338214470808 -117.64055465994359) bank14597 +14598 POINT(33.76210427282632 -117.26260050220253) bank14598 +14599 POINT(33.39365123069757 -118.214306042612) bank14599 +14600 POINT(33.714130182705894 -118.5612389529245) bank14600 +14601 POINT(33.96386506183433 -119.08375780967985) bank14601 +14602 POINT(34.41664020626616 -117.86558720315959) bank14602 +14603 POINT(34.94199436217797 -118.49524266425185) bank14603 +14604 POINT(34.3373062910661 -117.849003424006) bank14604 +14605 POINT(33.33856596451761 -117.60667711691532) bank14605 +14606 POINT(33.42001056609559 -118.68056545610904) bank14606 +14607 POINT(33.65865542687167 -118.21484149268089) bank14607 +14608 POINT(33.950432326866746 -117.44472836611403) bank14608 +14609 POINT(33.17905670599669 -117.4575186749157) bank14609 +14610 POINT(34.24632424271862 -118.36470348549395) bank14610 +14611 POINT(33.853597150970415 -117.4366526425531) bank14611 +14612 POINT(34.700221139381306 -117.93813381043365) bank14612 +14613 POINT(33.82267301196768 -117.73845640653617) bank14613 +14614 POINT(34.404797544553915 -118.02468694353703) bank14614 +14615 POINT(34.810594506379914 -118.28552747992359) bank14615 +14616 POINT(34.59432495479379 -118.3171148570735) bank14616 +14617 POINT(34.384213750004825 -117.91703367068129) bank14617 +14618 POINT(34.71210601259549 -119.21299166304318) bank14618 +14619 POINT(35.00102625822631 -118.90379688671979) bank14619 +14620 POINT(34.8877894512193 -118.53849188933903) bank14620 +14621 POINT(33.80629341730616 -118.18774552110217) bank14621 +14622 POINT(33.61086805285215 -118.76847193962622) bank14622 +14623 POINT(33.498203409386264 -117.74122353763964) bank14623 +14624 POINT(33.35133900648859 -118.12456730617674) bank14624 +14625 POINT(33.88400740405735 -119.17221346600691) bank14625 +14626 POINT(33.70560382586884 -117.56856387961565) bank14626 +14627 POINT(33.4925643037143 -117.90521488737869) bank14627 +14628 POINT(33.40244156276668 -118.54068430482523) bank14628 +14629 POINT(34.46520478918476 -118.93834567734002) bank14629 +14630 POINT(34.38325070306803 -118.4996780930964) bank14630 +14631 POINT(34.372423160224784 -118.29733775981799) bank14631 +14632 POINT(34.88775552453278 -117.81148267092185) bank14632 +14633 POINT(34.227820123764886 -118.60574536776097) bank14633 +14634 POINT(33.30907306958013 -117.29832427616138) bank14634 +14635 POINT(33.84937780865116 -118.74406776227342) bank14635 +14636 POINT(34.696805374080654 -118.62984435641292) bank14636 +14637 POINT(33.622974188463154 -119.0580063295098) bank14637 +14638 POINT(35.04527386677842 -119.21161513445792) bank14638 +14639 POINT(33.275033058997884 -118.9411105306229) bank14639 +14640 POINT(34.514844127031765 -118.6152460335232) bank14640 +14641 POINT(34.8130737653251 -117.66889028949863) bank14641 +14642 POINT(34.383645631960334 -117.3129900794812) bank14642 +14643 POINT(33.38443912858761 -118.41547424820169) bank14643 +14644 POINT(33.150131703099404 -119.0978390513948) bank14644 +14645 POINT(33.29858270313714 -117.2975908357202) bank14645 +14646 POINT(33.88860406366176 -117.71767090893769) bank14646 +14647 POINT(34.671346490414685 -118.83022657983335) bank14647 +14648 POINT(34.97705229707906 -118.99865764007792) bank14648 +14649 POINT(33.74068002832183 -119.06245463681495) bank14649 +14650 POINT(34.93011686494044 -118.39256365700419) bank14650 +14651 POINT(33.29427588002817 -118.13744605595203) bank14651 +14652 POINT(33.272696448103225 -117.99396635614667) bank14652 +14653 POINT(33.57740074691684 -118.16629960037929) bank14653 +14654 POINT(34.61226822772325 -118.13484289520781) bank14654 +14655 POINT(34.47995014480417 -117.77481498706315) bank14655 +14656 POINT(34.084426427235115 -118.016307166063) bank14656 +14657 POINT(33.11282985173296 -117.52506919459863) bank14657 +14658 POINT(34.489281278961386 -118.13042621626991) bank14658 +14659 POINT(33.161043741178545 -118.47627758784475) bank14659 +14660 POINT(34.746258763080945 -117.46416000761032) bank14660 +14661 POINT(33.167315139430606 -118.16318126547228) bank14661 +14662 POINT(33.06442313314619 -118.72546741596999) bank14662 +14663 POINT(33.21592690376344 -118.32819078273656) bank14663 +14664 POINT(34.14583686102378 -118.32397600459545) bank14664 +14665 POINT(33.962250349490226 -118.24923620061277) bank14665 +14666 POINT(34.40664242510593 -118.46493279099833) bank14666 +14667 POINT(34.09597000231773 -117.61567240836034) bank14667 +14668 POINT(33.7947405858478 -119.22612868784108) bank14668 +14669 POINT(34.77254659909745 -119.22464076339317) bank14669 +14670 POINT(33.56730607086973 -117.35727945922264) bank14670 +14671 POINT(33.64707948679671 -117.55280778963429) bank14671 +14672 POINT(34.46421909253412 -119.06313515216529) bank14672 +14673 POINT(35.03777687635928 -118.7057123076121) bank14673 +14674 POINT(34.906411690354716 -117.4827881490666) bank14674 +14675 POINT(34.76731519609394 -117.26149282747372) bank14675 +14676 POINT(34.71367583362387 -118.37261265988859) bank14676 +14677 POINT(33.650459395664186 -117.57260218925956) bank14677 +14678 POINT(34.35316868785673 -117.95730917514338) bank14678 +14679 POINT(34.631544312377926 -119.16099486579735) bank14679 +14680 POINT(34.79587134331662 -118.92955320861113) bank14680 +14681 POINT(33.258511607576864 -118.76920876140885) bank14681 +14682 POINT(35.014596187787475 -119.14380784387879) bank14682 +14683 POINT(35.03427603815959 -118.3762918629233) bank14683 +14684 POINT(34.735228105195276 -118.86752487713574) bank14684 +14685 POINT(33.25162731466468 -118.12582268772627) bank14685 +14686 POINT(33.08308006590955 -118.86477898545866) bank14686 +14687 POINT(33.14224696702928 -118.92977532627532) bank14687 +14688 POINT(34.3381837486041 -117.75374527676111) bank14688 +14689 POINT(33.67222516283078 -118.00343884477262) bank14689 +14690 POINT(33.85583647532323 -119.1665589275605) bank14690 +14691 POINT(34.26158451263502 -118.26325859364209) bank14691 +14692 POINT(35.032302502862734 -119.0806010022325) bank14692 +14693 POINT(33.97856933909315 -117.74342483687532) bank14693 +14694 POINT(33.304099047604204 -118.96705127966644) bank14694 +14695 POINT(33.454086727749186 -117.76633931491783) bank14695 +14696 POINT(33.633098677022346 -117.86533230418918) bank14696 +14697 POINT(34.58839039953283 -118.92181613976507) bank14697 +14698 POINT(34.691245452966754 -119.02553622355734) bank14698 +14699 POINT(34.22142051790191 -119.22829789458005) bank14699 +14700 POINT(34.94485355173152 -118.70318826603919) bank14700 +14701 POINT(33.8021862473388 -117.7365039644751) bank14701 +14702 POINT(34.313420811889095 -118.2445034745096) bank14702 +14703 POINT(34.32801973874331 -117.98530053499033) bank14703 +14704 POINT(33.14125962259602 -118.42728534159414) bank14704 +14705 POINT(33.1864201207422 -117.78758369008109) bank14705 +14706 POINT(34.14441065503072 -119.05324453342939) bank14706 +14707 POINT(33.50403107993957 -118.32405969647331) bank14707 +14708 POINT(33.38431163380267 -118.2657788672798) bank14708 +14709 POINT(34.27736324188407 -118.45145508319983) bank14709 +14710 POINT(34.83275305040116 -117.91460544186228) bank14710 +14711 POINT(34.83030292869836 -118.1900816369305) bank14711 +14712 POINT(33.45675611196864 -117.42697939062427) bank14712 +14713 POINT(33.707849372817364 -117.86620018837225) bank14713 +14714 POINT(33.25231397585104 -118.95405871296451) bank14714 +14715 POINT(33.183524150619185 -119.16921409670171) bank14715 +14716 POINT(33.17039065451231 -118.23113311409844) bank14716 +14717 POINT(34.56215795703694 -118.95253985011497) bank14717 +14718 POINT(33.613486404369425 -118.56199512687321) bank14718 +14719 POINT(33.66022131823961 -117.49866900303482) bank14719 +14720 POINT(34.94741075751443 -117.88051441755542) bank14720 +14721 POINT(33.74325727887052 -117.62400146830788) bank14721 +14722 POINT(34.39974457127711 -117.73151670712213) bank14722 +14723 POINT(33.669425343907456 -117.71369989920129) bank14723 +14724 POINT(33.73770607142575 -117.57457226443951) bank14724 +14725 POINT(34.63136361020885 -117.68708487420763) bank14725 +14726 POINT(34.00613162087595 -118.63679451665672) bank14726 +14727 POINT(33.750042976825675 -118.74733419412891) bank14727 +14728 POINT(34.00794206214577 -118.873294223348) bank14728 +14729 POINT(34.0760838368436 -119.08989476522702) bank14729 +14730 POINT(33.28873917133811 -119.04853870296688) bank14730 +14731 POINT(34.79095676950876 -118.3554997387404) bank14731 +14732 POINT(34.70486530065578 -118.17238709400424) bank14732 +14733 POINT(33.56296664051753 -117.55594458519703) bank14733 +14734 POINT(35.033854237373205 -118.70360165259768) bank14734 +14735 POINT(33.67611960746293 -118.5270491012935) bank14735 +14736 POINT(33.909665707973566 -117.93506253943224) bank14736 +14737 POINT(33.796239573985034 -118.81399508604152) bank14737 +14738 POINT(33.53436285208306 -119.09932314004635) bank14738 +14739 POINT(33.96885761301321 -117.46740334509762) bank14739 +14740 POINT(33.13992928549446 -119.03756524209805) bank14740 +14741 POINT(33.76066833274891 -119.00781351558223) bank14741 +14742 POINT(34.48104325102103 -119.0927932510072) bank14742 +14743 POINT(34.4423493227745 -117.89098048548794) bank14743 +14744 POINT(34.97464093023769 -118.38414940857642) bank14744 +14745 POINT(33.95471569335957 -117.47165116265391) bank14745 +14746 POINT(34.6395749487578 -118.74467800155695) bank14746 +14747 POINT(33.08067901745194 -118.89546311728276) bank14747 +14748 POINT(34.60560939336574 -118.67661164254788) bank14748 +14749 POINT(33.74789752145174 -118.63282453919115) bank14749 +14750 POINT(33.455901859321784 -118.74404117586248) bank14750 +14751 POINT(33.650469753892324 -118.02399761168334) bank14751 +14752 POINT(34.96005673560775 -117.40626126551324) bank14752 +14753 POINT(34.034471892671085 -119.16513214810561) bank14753 +14754 POINT(34.575617996231756 -119.1091984348255) bank14754 +14755 POINT(34.84251998125099 -118.70296165646991) bank14755 +14756 POINT(33.88475326230288 -117.72714072951311) bank14756 +14757 POINT(33.503717285234266 -117.46190757797221) bank14757 +14758 POINT(34.995646554002775 -118.0244720024173) bank14758 +14759 POINT(33.89564816917425 -118.43759386075718) bank14759 +14760 POINT(33.39743477981783 -117.2901839443642) bank14760 +14761 POINT(34.04802471400165 -118.54958875792539) bank14761 +14762 POINT(33.20155864260185 -118.72506541916009) bank14762 +14763 POINT(33.6881450397067 -118.79269020813285) bank14763 +14764 POINT(35.02310619220368 -118.62155117213922) bank14764 +14765 POINT(34.96798531254481 -118.21691291584204) bank14765 +14766 POINT(34.07629805942497 -118.34314035136698) bank14766 +14767 POINT(34.17019909799578 -117.61716468514541) bank14767 +14768 POINT(34.55649901206397 -118.51716795821591) bank14768 +14769 POINT(33.98214774657714 -118.9040453074533) bank14769 +14770 POINT(34.89151516096646 -117.47937948223836) bank14770 +14771 POINT(33.452872485689426 -117.66805131824326) bank14771 +14772 POINT(34.196536404688096 -118.6226860955032) bank14772 +14773 POINT(33.2979506888212 -119.21565589019072) bank14773 +14774 POINT(33.86074792896328 -117.25070201198828) bank14774 +14775 POINT(34.807092390142394 -117.31679224418826) bank14775 +14776 POINT(34.94246698726821 -118.58098654995355) bank14776 +14777 POINT(35.0266872430984 -118.78560626064643) bank14777 +14778 POINT(34.90551923196176 -118.21653174013858) bank14778 +14779 POINT(33.97110977269182 -117.36494512292572) bank14779 +14780 POINT(33.313596812803496 -117.81697899844248) bank14780 +14781 POINT(34.56667451230269 -117.43093591611918) bank14781 +14782 POINT(34.8169751587113 -118.19622742120993) bank14782 +14783 POINT(35.04086618267776 -118.46771336085328) bank14783 +14784 POINT(33.2486800402437 -117.76607809665448) bank14784 +14785 POINT(33.72862601348607 -119.19942495629073) bank14785 +14786 POINT(33.32136531086599 -117.70322731018213) bank14786 +14787 POINT(33.4995261097158 -117.64354093647475) bank14787 +14788 POINT(33.89056671569926 -117.99991654019156) bank14788 +14789 POINT(33.16987308643568 -118.17224333485031) bank14789 +14790 POINT(34.06967524858198 -117.67341315783612) bank14790 +14791 POINT(34.790514590791396 -118.56324040958502) bank14791 +14792 POINT(33.925555363738944 -118.3025192094919) bank14792 +14793 POINT(33.516910373824295 -118.49126231156838) bank14793 +14794 POINT(34.18631405800547 -118.88860046937937) bank14794 +14795 POINT(34.47846738654868 -118.51814193294747) bank14795 +14796 POINT(33.054108553054256 -118.5905961714748) bank14796 +14797 POINT(34.16544238763394 -117.38523226457009) bank14797 +14798 POINT(33.07357209458144 -117.96003957468463) bank14798 +14799 POINT(33.40777739763657 -118.31049992332272) bank14799 +14800 POINT(34.992392430917064 -118.56769246690287) bank14800 +14801 POINT(33.87896261242412 -118.58794898336475) bank14801 +14802 POINT(33.49725204849757 -118.00819766573254) bank14802 +14803 POINT(34.364490442511716 -118.37989979868216) bank14803 +14804 POINT(34.273851520404826 -118.03320574552679) bank14804 +14805 POINT(33.96673824992353 -119.16472771181009) bank14805 +14806 POINT(33.81489224155753 -119.13687910036435) bank14806 +14807 POINT(34.31918400433284 -117.7242392455853) bank14807 +14808 POINT(34.98287276956152 -118.63702441643794) bank14808 +14809 POINT(35.005490501852194 -119.16873148500798) bank14809 +14810 POINT(33.97873840856136 -117.6446695536764) bank14810 +14811 POINT(34.610284444290166 -117.66052211749607) bank14811 +14812 POINT(34.81081083291069 -118.30768980971446) bank14812 +14813 POINT(33.10354835733193 -119.06779535916495) bank14813 +14814 POINT(33.4464419782629 -118.95203737533927) bank14814 +14815 POINT(34.44398660657138 -118.5376087061499) bank14815 +14816 POINT(34.788320311797094 -118.3681262997715) bank14816 +14817 POINT(34.67538383291132 -117.63187316019577) bank14817 +14818 POINT(33.68282151728764 -118.7936682454615) bank14818 +14819 POINT(34.885249070894474 -118.44619872500185) bank14819 +14820 POINT(34.804249752661114 -117.65246866124859) bank14820 +14821 POINT(34.50158423095651 -117.94556690127614) bank14821 +14822 POINT(33.72981099348772 -117.48722858763853) bank14822 +14823 POINT(33.64296320153952 -119.01151109167844) bank14823 +14824 POINT(34.084282277530086 -119.07162550998852) bank14824 +14825 POINT(34.421637569260625 -119.13257044102399) bank14825 +14826 POINT(33.8678587818737 -117.61009547043314) bank14826 +14827 POINT(34.751908459598994 -118.54898236688767) bank14827 +14828 POINT(33.08694702354983 -117.82170528687739) bank14828 +14829 POINT(33.34433365246459 -118.07602713015052) bank14829 +14830 POINT(33.4018701154426 -118.43526308282178) bank14830 +14831 POINT(34.93354375037055 -117.37283979258585) bank14831 +14832 POINT(33.366221430492715 -118.48640913265794) bank14832 +14833 POINT(33.140903241653994 -117.45242887124742) bank14833 +14834 POINT(34.10735186248266 -118.84350579793266) bank14834 +14835 POINT(33.2063004470907 -117.33547399781365) bank14835 +14836 POINT(34.2587955222643 -118.59354027109791) bank14836 +14837 POINT(33.09470103041425 -118.50174344648427) bank14837 +14838 POINT(33.535372505202574 -118.93095318953827) bank14838 +14839 POINT(33.9334250106629 -119.01100448303738) bank14839 +14840 POINT(35.050118613828815 -118.50826660060255) bank14840 +14841 POINT(33.18056126768332 -118.73526559811353) bank14841 +14842 POINT(34.243971220276336 -117.92867792786879) bank14842 +14843 POINT(35.01936968362212 -117.7414928295463) bank14843 +14844 POINT(34.65345705237576 -117.30131386477693) bank14844 +14845 POINT(34.046659952369716 -117.29951835216552) bank14845 +14846 POINT(33.76487073593489 -118.22218772015373) bank14846 +14847 POINT(34.679601409857646 -117.74555844709863) bank14847 +14848 POINT(33.4219535621073 -118.96180308736697) bank14848 +14849 POINT(34.037144754582144 -117.49590452143383) bank14849 +14850 POINT(33.375882744326574 -118.24706157545123) bank14850 +14851 POINT(34.3296911681652 -117.29320686688885) bank14851 +14852 POINT(33.80082843007785 -117.65619862772674) bank14852 +14853 POINT(34.022949379805034 -117.32204184562761) bank14853 +14854 POINT(34.05178223173187 -117.72088444879645) bank14854 +14855 POINT(34.75465243313559 -117.56461106343897) bank14855 +14856 POINT(33.207088197759035 -118.58827981826214) bank14856 +14857 POINT(34.88690363794952 -117.24802072920389) bank14857 +14858 POINT(33.721431218032684 -118.5253263310078) bank14858 +14859 POINT(33.82643905619134 -118.86954258685978) bank14859 +14860 POINT(34.69490645584859 -117.58188063301637) bank14860 +14861 POINT(33.333548169783846 -117.30308371751609) bank14861 +14862 POINT(34.54558834289626 -118.25195072600069) bank14862 +14863 POINT(34.44243115570978 -118.65980726188988) bank14863 +14864 POINT(33.256707113164715 -119.07331862839216) bank14864 +14865 POINT(34.90805267966047 -119.19676422236068) bank14865 +14866 POINT(34.40274426325017 -117.71375577044425) bank14866 +14867 POINT(33.5259607624 -119.11527857995628) bank14867 +14868 POINT(33.27272864395412 -119.16787242940343) bank14868 +14869 POINT(34.118118905175656 -117.30575034081845) bank14869 +14870 POINT(34.84611340056591 -117.78270229516377) bank14870 +14871 POINT(34.82181352458899 -117.50360260001194) bank14871 +14872 POINT(33.91490409591972 -118.82287270302818) bank14872 +14873 POINT(34.61671524979045 -117.6078012645727) bank14873 +14874 POINT(33.23034364918488 -118.17322264765845) bank14874 +14875 POINT(34.47899212783439 -119.14865859605774) bank14875 +14876 POINT(34.5987455443266 -119.08065543112657) bank14876 +14877 POINT(34.386511412718725 -118.3426964433444) bank14877 +14878 POINT(34.09962307836374 -118.16654009508) bank14878 +14879 POINT(34.467132498744256 -118.24372931584017) bank14879 +14880 POINT(33.94451908314426 -117.816895970764) bank14880 +14881 POINT(33.454087719120025 -118.65168725822593) bank14881 +14882 POINT(34.45225159421132 -118.96202431294334) bank14882 +14883 POINT(33.51886601943457 -117.41963790174363) bank14883 +14884 POINT(34.545102265980404 -118.67396200529473) bank14884 +14885 POINT(34.42076316004735 -117.25183750729902) bank14885 +14886 POINT(34.746261279175364 -118.60596718700036) bank14886 +14887 POINT(33.16059424003942 -117.67797374744605) bank14887 +14888 POINT(33.78856869681696 -117.95858852956376) bank14888 +14889 POINT(33.79270014454634 -118.57711237276287) bank14889 +14890 POINT(33.623216932088766 -117.31424011634991) bank14890 +14891 POINT(34.76153065362775 -117.5095369982985) bank14891 +14892 POINT(33.9766377598722 -117.31571903610926) bank14892 +14893 POINT(33.93571040770829 -117.98558813466086) bank14893 +14894 POINT(33.15341277979849 -117.47488352008668) bank14894 +14895 POINT(35.02256335040544 -119.12735198731514) bank14895 +14896 POINT(33.578131874719126 -119.11031895904281) bank14896 +14897 POINT(33.493581081340444 -117.94542863891505) bank14897 +14898 POINT(34.83870901206327 -118.32946242591848) bank14898 +14899 POINT(33.66581037123886 -118.62522174392332) bank14899 +14900 POINT(34.351558517522925 -117.47088340026522) bank14900 +14901 POINT(33.97431980411292 -117.33067450078586) bank14901 +14902 POINT(34.11525280971336 -118.34541013919126) bank14902 +14903 POINT(33.11891761527578 -118.01827592000245) bank14903 +14904 POINT(34.55494529760228 -118.01076550588759) bank14904 +14905 POINT(34.44845882173068 -118.57822818292519) bank14905 +14906 POINT(33.96445957531901 -119.19717850174786) bank14906 +14907 POINT(33.6203063413702 -118.42063156666472) bank14907 +14908 POINT(33.33551740209294 -117.65073435937795) bank14908 +14909 POINT(33.92297191133548 -118.17595130153502) bank14909 +14910 POINT(33.68244444146493 -118.55282287081555) bank14910 +14911 POINT(34.60766593016906 -118.76095658175444) bank14911 +14912 POINT(33.89844450966628 -118.49484972954596) bank14912 +14913 POINT(34.852509924424375 -119.16536379169189) bank14913 +14914 POINT(33.654820275598446 -118.80297453468631) bank14914 +14915 POINT(34.8886370687248 -117.47166263556274) bank14915 +14916 POINT(33.49141250585849 -117.68582870947071) bank14916 +14917 POINT(33.46454333798222 -118.03154568465327) bank14917 +14918 POINT(34.9852548699703 -118.07545941236425) bank14918 +14919 POINT(34.482940690360564 -119.04777014211582) bank14919 +14920 POINT(33.47526406790208 -118.27818131528679) bank14920 +14921 POINT(34.78557673856847 -119.11913540583059) bank14921 +14922 POINT(33.84623636690914 -118.25853633197278) bank14922 +14923 POINT(33.35945286061641 -117.53856332221507) bank14923 +14924 POINT(34.24904310284263 -117.8276926039255) bank14924 +14925 POINT(33.26032585239975 -119.0543637737685) bank14925 +14926 POINT(33.855011765123066 -118.29673914409692) bank14926 +14927 POINT(33.37460317127321 -117.54807752880582) bank14927 +14928 POINT(33.459523927409705 -117.65456995794692) bank14928 +14929 POINT(34.869570115387646 -118.66960579745799) bank14929 +14930 POINT(33.31474500446038 -117.94584139586809) bank14930 +14931 POINT(33.42450005974669 -119.17939709110784) bank14931 +14932 POINT(34.710513272008484 -117.5241398223775) bank14932 +14933 POINT(34.64426482661267 -117.57036206966521) bank14933 +14934 POINT(34.69874197130055 -117.44374940073031) bank14934 +14935 POINT(33.070175016264656 -117.92372837976689) bank14935 +14936 POINT(33.49192068176151 -118.51286931620378) bank14936 +14937 POINT(33.237213010100795 -117.35797592461053) bank14937 +14938 POINT(33.10432271856352 -117.51832718151228) bank14938 +14939 POINT(34.42897113963659 -117.38952639311617) bank14939 +14940 POINT(34.123592610595026 -119.08205843939885) bank14940 +14941 POINT(35.03653115656317 -119.11567972914236) bank14941 +14942 POINT(33.541063840861874 -117.26104897260797) bank14942 +14943 POINT(34.87735256966025 -117.8056105022758) bank14943 +14944 POINT(33.69025029746238 -118.07175704936422) bank14944 +14945 POINT(35.0082428302817 -117.43001784493543) bank14945 +14946 POINT(34.08418513352323 -118.98885493856751) bank14946 +14947 POINT(34.39313164779135 -117.67477931694557) bank14947 +14948 POINT(33.551379975965176 -117.3123749117079) bank14948 +14949 POINT(34.63497288930478 -118.65860624114917) bank14949 +14950 POINT(34.41302595526491 -117.94202045504042) bank14950 +14951 POINT(34.28373170943663 -117.31626916202715) bank14951 +14952 POINT(33.38145432007463 -117.30286524163108) bank14952 +14953 POINT(33.37296948240643 -118.86514883382435) bank14953 +14954 POINT(34.579650526607985 -119.15144438856117) bank14954 +14955 POINT(33.52634086498069 -118.398595932861) bank14955 +14956 POINT(34.53911009666355 -118.38899929006185) bank14956 +14957 POINT(34.67891754203743 -118.80613589487693) bank14957 +14958 POINT(34.1018352465118 -118.92618055436971) bank14958 +14959 POINT(34.87857335558286 -117.91725952488574) bank14959 +14960 POINT(34.28441631006041 -118.89430890778739) bank14960 +14961 POINT(34.55169155261161 -118.84076258499535) bank14961 +14962 POINT(33.50920548586827 -119.21307940238593) bank14962 +14963 POINT(34.07111838303608 -118.19629778259363) bank14963 +14964 POINT(33.06979705426746 -118.47023901459215) bank14964 +14965 POINT(33.58626469407816 -117.3490800430236) bank14965 +14966 POINT(33.822983503312415 -119.2075619766451) bank14966 +14967 POINT(33.57102184348295 -119.10476965533796) bank14967 +14968 POINT(34.7802370326474 -118.16839604692525) bank14968 +14969 POINT(33.45106690275345 -118.70457018144057) bank14969 +14970 POINT(34.60248664304258 -117.73854298791221) bank14970 +14971 POINT(33.26314420548717 -118.27830805296787) bank14971 +14972 POINT(34.44530082560643 -118.59458918004172) bank14972 +14973 POINT(34.624743027651014 -117.44738165122853) bank14973 +14974 POINT(33.52887070115567 -118.26237912780992) bank14974 +14975 POINT(34.98130943027203 -118.79622937182602) bank14975 +14976 POINT(34.71191255020575 -119.20538426844901) bank14976 +14977 POINT(34.10847106252665 -118.5165859265263) bank14977 +14978 POINT(34.957292792412794 -118.78247594028515) bank14978 +14979 POINT(33.7175259991824 -118.86862617720962) bank14979 +14980 POINT(33.92357227562869 -118.8424501481469) bank14980 +14981 POINT(33.85268018042304 -118.60166573342866) bank14981 +14982 POINT(33.83905705606877 -119.19579860710222) bank14982 +14983 POINT(33.26059978360528 -119.01037641463833) bank14983 +14984 POINT(34.65609818826449 -118.9853813778784) bank14984 +14985 POINT(33.65164249507279 -118.61455218551748) bank14985 +14986 POINT(34.76292227962003 -118.4714651684209) bank14986 +14987 POINT(34.920571311260545 -118.34029057842928) bank14987 +14988 POINT(33.97213518251723 -117.35969929228582) bank14988 +14989 POINT(33.80998655922991 -118.70941834553454) bank14989 +14990 POINT(33.11321644267137 -118.7342626739003) bank14990 +14991 POINT(34.523106904526465 -118.58671799664808) bank14991 +14992 POINT(34.67476659400181 -119.05325061270656) bank14992 +14993 POINT(33.803857875511746 -117.36238681242948) bank14993 +14994 POINT(34.032248229382084 -119.05885098860095) bank14994 +14995 POINT(34.03842445938882 -117.96964598210953) bank14995 +14996 POINT(33.835388047910605 -117.50387218187586) bank14996 +14997 POINT(34.62505982374934 -117.60951986904323) bank14997 +14998 POINT(34.695953357437304 -118.8959260178541) bank14998 +14999 POINT(33.662835790488884 -118.58473315918896) bank14999 +15000 POINT(34.59782320120186 -117.94341922729338) bank15000 +15001 POINT(33.45027063886569 -117.52738794612273) bank15001 +15002 POINT(34.445630765489426 -118.7583200314601) bank15002 +15003 POINT(34.32862215961672 -117.69700748117621) bank15003 +15004 POINT(33.826444891693264 -117.96786629529282) bank15004 +15005 POINT(33.698919599243084 -117.72513506682803) bank15005 +15006 POINT(34.613527889660396 -118.23743066170418) bank15006 +15007 POINT(34.098948950721024 -118.52223750943115) bank15007 +15008 POINT(34.575164546875826 -117.8921605765642) bank15008 +15009 POINT(33.76599658495594 -117.9301915661602) bank15009 +15010 POINT(34.37287690125578 -118.60494420754704) bank15010 +15011 POINT(34.89967009031555 -118.61680220801985) bank15011 +15012 POINT(33.558515496265436 -118.0440588550128) bank15012 +15013 POINT(33.54571224233058 -118.29007358802365) bank15013 +15014 POINT(33.92142124682691 -118.51591465524785) bank15014 +15015 POINT(33.784160785447554 -117.63178073721939) bank15015 +15016 POINT(33.13570911482485 -118.35325317597588) bank15016 +15017 POINT(33.28857676767892 -118.3170058425538) bank15017 +15018 POINT(33.52680512982928 -118.15623152800681) bank15018 +15019 POINT(33.585037968040595 -118.33888876617442) bank15019 +15020 POINT(33.42593702798883 -118.28694543859767) bank15020 +15021 POINT(33.956483042076535 -117.85334611297345) bank15021 +15022 POINT(33.92202051015943 -117.7641488896606) bank15022 +15023 POINT(33.43957344375001 -118.02877606245936) bank15023 +15024 POINT(34.3581779917438 -118.03950342736451) bank15024 +15025 POINT(33.117175979739955 -118.731738058807) bank15025 +15026 POINT(35.04368668648981 -117.81339282990042) bank15026 +15027 POINT(33.85940025241739 -118.84667104708551) bank15027 +15028 POINT(33.20024079846494 -119.01536268849452) bank15028 +15029 POINT(34.27474124318395 -117.3088248450378) bank15029 +15030 POINT(34.94001195294644 -117.29484113018005) bank15030 +15031 POINT(33.8706634173003 -118.60531114842888) bank15031 +15032 POINT(33.22755312365801 -117.80698936910342) bank15032 +15033 POINT(33.11310309807064 -117.31239962558067) bank15033 +15034 POINT(33.48528497093243 -118.04594264701835) bank15034 +15035 POINT(33.75386418822198 -118.43045186327566) bank15035 +15036 POINT(35.00793793936013 -118.54321392389755) bank15036 +15037 POINT(34.72265381249872 -118.67428612275322) bank15037 +15038 POINT(33.5995642257372 -117.71864342335533) bank15038 +15039 POINT(34.210377194014924 -117.31702277560107) bank15039 +15040 POINT(33.71189117287116 -117.58269347364514) bank15040 +15041 POINT(34.23435358122819 -118.05909217207693) bank15041 +15042 POINT(33.89230202304496 -117.6067524327297) bank15042 +15043 POINT(34.933637064394 -118.348512625395) bank15043 +15044 POINT(33.6692252351544 -118.06121642603263) bank15044 +15045 POINT(34.724087347237784 -118.59171933286011) bank15045 +15046 POINT(35.004093564687054 -117.25638287266747) bank15046 +15047 POINT(33.41698427679224 -119.19307828939837) bank15047 +15048 POINT(34.670410403756044 -118.74042135192623) bank15048 +15049 POINT(34.735729124426946 -118.3353072895399) bank15049 +15050 POINT(34.829454977012624 -117.70991878469988) bank15050 +15051 POINT(34.309067433608675 -119.07050175556272) bank15051 +15052 POINT(33.16113267368369 -117.61806623535242) bank15052 +15053 POINT(33.63689533684899 -118.76744861776973) bank15053 +15054 POINT(33.813461234241984 -119.14197896203208) bank15054 +15055 POINT(33.49360937791032 -119.22448989975445) bank15055 +15056 POINT(33.85501298108581 -119.05485515505052) bank15056 +15057 POINT(34.49100610883941 -117.8759226126269) bank15057 +15058 POINT(33.73249646404149 -118.59678412135389) bank15058 +15059 POINT(34.22902443580857 -119.1012627635943) bank15059 +15060 POINT(34.331206824881406 -118.82607547286734) bank15060 +15061 POINT(33.485976913809864 -119.14704789083025) bank15061 +15062 POINT(33.67934327421603 -119.02990215895765) bank15062 +15063 POINT(34.191542810079234 -117.37094477062638) bank15063 +15064 POINT(33.355943997508504 -118.02439892505633) bank15064 +15065 POINT(33.813843682622334 -117.89175967967319) bank15065 +15066 POINT(34.87947339189908 -117.80898244536296) bank15066 +15067 POINT(33.56094933449541 -117.66541479833292) bank15067 +15068 POINT(33.21504923016472 -118.39786866695862) bank15068 +15069 POINT(33.70479438586619 -118.68811979142386) bank15069 +15070 POINT(33.23570074977361 -118.00953686012089) bank15070 +15071 POINT(33.83179775509158 -119.2005352409093) bank15071 +15072 POINT(34.38179379882401 -117.3815828919378) bank15072 +15073 POINT(34.53802973818124 -118.97402692647316) bank15073 +15074 POINT(33.65346143647294 -118.35634450383398) bank15074 +15075 POINT(34.34658524897112 -117.26289831146255) bank15075 +15076 POINT(33.845542975206605 -118.58500584633164) bank15076 +15077 POINT(33.899171738147025 -118.4158955979747) bank15077 +15078 POINT(34.864900982035095 -117.3891163926886) bank15078 +15079 POINT(34.183409319653514 -117.48264913093655) bank15079 +15080 POINT(33.57372747986046 -118.695879380827) bank15080 +15081 POINT(34.87565475538099 -118.88006656458049) bank15081 +15082 POINT(34.09057178446877 -118.24952624919203) bank15082 +15083 POINT(34.747082298573595 -117.88662951219328) bank15083 +15084 POINT(33.3055448217731 -117.78557379074476) bank15084 +15085 POINT(33.067139794169925 -118.95325609346123) bank15085 +15086 POINT(34.1348058209109 -118.49343336696664) bank15086 +15087 POINT(34.85074512913316 -118.55979823156542) bank15087 +15088 POINT(34.52669186664346 -119.15756110209333) bank15088 +15089 POINT(33.45476064355167 -118.95051051721252) bank15089 +15090 POINT(35.00693933228584 -117.91272747519128) bank15090 +15091 POINT(34.93087382250632 -118.45755331149107) bank15091 +15092 POINT(34.60434517219994 -117.88078430566937) bank15092 +15093 POINT(33.680900314759775 -118.03291494842036) bank15093 +15094 POINT(33.21510319904913 -117.88351823263537) bank15094 +15095 POINT(34.094817414802684 -117.69139252239499) bank15095 +15096 POINT(34.79763758652471 -118.64491911646972) bank15096 +15097 POINT(33.42410201025237 -118.93497833277783) bank15097 +15098 POINT(34.027015705071854 -118.90864977540078) bank15098 +15099 POINT(34.11304800566947 -119.14880085005137) bank15099 +15100 POINT(34.99110826327987 -117.85635878855912) bank15100 +15101 POINT(34.87241625651623 -118.75066509812223) bank15101 +15102 POINT(34.180448650557025 -118.76863382171335) bank15102 +15103 POINT(34.081578066897045 -117.49849086177782) bank15103 +15104 POINT(33.867257662920444 -118.3264505332688) bank15104 +15105 POINT(34.74140534030764 -117.95916386054908) bank15105 +15106 POINT(34.96086837025952 -118.16390897958907) bank15106 +15107 POINT(34.47421250857174 -117.90815396560832) bank15107 +15108 POINT(34.695113097060634 -119.05462998961677) bank15108 +15109 POINT(33.343561616788286 -118.50539255486042) bank15109 +15110 POINT(33.34506701304761 -118.54704638699592) bank15110 +15111 POINT(35.00150432518127 -118.24450265786318) bank15111 +15112 POINT(33.16584775111363 -118.78021816119059) bank15112 +15113 POINT(33.58410412764128 -118.70140974646358) bank15113 +15114 POINT(34.09166372305645 -117.79660181282598) bank15114 +15115 POINT(34.519198269345324 -118.60227026469522) bank15115 +15116 POINT(34.647240508601534 -117.56836285189104) bank15116 +15117 POINT(35.01061001219088 -117.26340039827846) bank15117 +15118 POINT(34.204949632954175 -117.44295191043648) bank15118 +15119 POINT(33.90040157109345 -118.13922136211629) bank15119 +15120 POINT(33.58759066831274 -117.60264418027324) bank15120 +15121 POINT(33.200889663128464 -117.76646087682482) bank15121 +15122 POINT(34.50031946605696 -118.77113569791615) bank15122 +15123 POINT(33.33793012011954 -118.25566733029306) bank15123 +15124 POINT(33.793499344526516 -117.30145259567968) bank15124 +15125 POINT(33.23540287785254 -118.8842007792615) bank15125 +15126 POINT(33.33547873275866 -118.75470203687296) bank15126 +15127 POINT(34.6091190166189 -117.8307276417471) bank15127 +15128 POINT(34.546046594742826 -118.18403022231956) bank15128 +15129 POINT(34.20083796609977 -118.28009435380007) bank15129 +15130 POINT(34.81420103624187 -117.76172215381257) bank15130 +15131 POINT(34.10902051842226 -119.00150399039107) bank15131 +15132 POINT(33.35732304608027 -117.28711985604369) bank15132 +15133 POINT(34.59565677485719 -118.0774238022056) bank15133 +15134 POINT(34.10489825388507 -118.39289875456379) bank15134 +15135 POINT(33.73830821626348 -119.03479687211336) bank15135 +15136 POINT(34.79750824359938 -118.60283664362373) bank15136 +15137 POINT(34.11155618968793 -117.47646034966304) bank15137 +15138 POINT(33.35962120993567 -118.60400175474545) bank15138 +15139 POINT(34.02928983259833 -118.69824840986367) bank15139 +15140 POINT(33.373353480743866 -118.07839123228852) bank15140 +15141 POINT(33.912860075719536 -117.93273764814654) bank15141 +15142 POINT(33.489860850861106 -119.18159913058619) bank15142 +15143 POINT(34.707067150975256 -118.53166382689682) bank15143 +15144 POINT(33.985546653439 -118.33673959642184) bank15144 +15145 POINT(35.04045440006999 -118.95682734666535) bank15145 +15146 POINT(34.81391220056232 -118.13449434014903) bank15146 +15147 POINT(33.597239578454136 -118.12087384345404) bank15147 +15148 POINT(33.477086090312035 -118.80675466916982) bank15148 +15149 POINT(34.96562561273396 -118.04343037038412) bank15149 +15150 POINT(34.408406981197714 -117.3683180040307) bank15150 +15151 POINT(33.49988438211898 -119.18477215313342) bank15151 +15152 POINT(33.75173239593276 -117.69394974866869) bank15152 +15153 POINT(33.40291286410188 -117.62516836078527) bank15153 +15154 POINT(33.650614561776 -118.91799414326447) bank15154 +15155 POINT(33.5512091635966 -118.74729128693087) bank15155 +15156 POINT(34.98984245531527 -118.82055757150583) bank15156 +15157 POINT(33.83374334689183 -117.56945994099738) bank15157 +15158 POINT(34.64698522862512 -118.53894272976503) bank15158 +15159 POINT(34.56824918099105 -117.72718322002507) bank15159 +15160 POINT(33.659424854214244 -118.90840535578963) bank15160 +15161 POINT(33.28677771447033 -117.85559715073106) bank15161 +15162 POINT(33.38453022291476 -117.25282205854965) bank15162 +15163 POINT(33.88730853066209 -117.39504817788044) bank15163 +15164 POINT(33.18963934053172 -117.6673839777724) bank15164 +15165 POINT(33.85650997060822 -117.84461630444878) bank15165 +15166 POINT(34.40609562502196 -118.16384987404602) bank15166 +15167 POINT(34.598270646044504 -117.31407391607826) bank15167 +15168 POINT(34.672580721221856 -118.46908181603658) bank15168 +15169 POINT(33.26184655362567 -118.39717890358189) bank15169 +15170 POINT(33.77708965075761 -118.07071756631082) bank15170 +15171 POINT(33.32441306444122 -118.85814179076465) bank15171 +15172 POINT(34.96153557724887 -118.42099807838281) bank15172 +15173 POINT(33.143993320891184 -117.75068255572762) bank15173 +15174 POINT(33.30180464673918 -118.00234690118033) bank15174 +15175 POINT(34.338807759332894 -118.28518452704783) bank15175 +15176 POINT(33.798146359473726 -119.14429697081026) bank15176 +15177 POINT(34.0623627904094 -117.3162859284358) bank15177 +15178 POINT(33.79121364600342 -119.03217963270114) bank15178 +15179 POINT(34.218810382226444 -118.555139581521) bank15179 +15180 POINT(33.68213836099328 -117.85600859883213) bank15180 +15181 POINT(34.81223546347585 -118.24391956327295) bank15181 +15182 POINT(33.881027644435626 -119.1884831868993) bank15182 +15183 POINT(33.857444327012885 -118.9395493398532) bank15183 +15184 POINT(34.5531074936168 -117.9286445094848) bank15184 +15185 POINT(33.14646596703422 -119.02830507416739) bank15185 +15186 POINT(33.29298319601014 -117.55552280916127) bank15186 +15187 POINT(34.0373514498551 -117.5882476163293) bank15187 +15188 POINT(34.60685711081885 -118.25308019558747) bank15188 +15189 POINT(34.554165793218125 -119.0695453514122) bank15189 +15190 POINT(34.72071536298204 -117.26661749752847) bank15190 +15191 POINT(33.176320334277904 -117.54546728621574) bank15191 +15192 POINT(34.26698647590611 -117.67326747672854) bank15192 +15193 POINT(33.46670939116222 -117.66315102686731) bank15193 +15194 POINT(33.28361493921685 -117.5685060197219) bank15194 +15195 POINT(34.7352004191577 -118.89010347437664) bank15195 +15196 POINT(33.072221718248 -117.85247397737581) bank15196 +15197 POINT(34.124708701073466 -118.77711751476247) bank15197 +15198 POINT(34.62354643188631 -118.88149003523789) bank15198 +15199 POINT(34.65661052489724 -118.84994282460453) bank15199 +15200 POINT(34.53199639368898 -117.35429291742979) bank15200 +15201 POINT(33.72604933187117 -117.7671556737508) bank15201 +15202 POINT(33.48801938561177 -117.71314612417272) bank15202 +15203 POINT(34.543068534303615 -118.44317163017386) bank15203 +15204 POINT(33.059673196008916 -118.00640096953148) bank15204 +15205 POINT(33.201465140734186 -118.72890961536544) bank15205 +15206 POINT(33.49671517295374 -118.57925215738567) bank15206 +15207 POINT(33.510885891946636 -117.56724732857228) bank15207 +15208 POINT(33.11933003280779 -117.89710469144639) bank15208 +15209 POINT(34.046968936292394 -119.23152277870844) bank15209 +15210 POINT(33.069844820125184 -117.8874252363489) bank15210 +15211 POINT(34.1690444076168 -118.80893858041439) bank15211 +15212 POINT(33.137919572241266 -118.77938496861246) bank15212 +15213 POINT(34.3920239419795 -119.0935039263602) bank15213 +15214 POINT(34.07201341476472 -118.68669629211733) bank15214 +15215 POINT(33.36288096093663 -118.61902923724145) bank15215 +15216 POINT(33.05562092069918 -117.30595297080204) bank15216 +15217 POINT(33.58406025919674 -118.90240008926689) bank15217 +15218 POINT(33.15150417759539 -117.79283973337628) bank15218 +15219 POINT(33.97001775586034 -119.21960700218565) bank15219 +15220 POINT(33.12788969854403 -118.99753183788918) bank15220 +15221 POINT(33.331639908222385 -118.28477265154889) bank15221 +15222 POINT(34.36381223216837 -118.69190755555633) bank15222 +15223 POINT(33.827940854362936 -117.52714892752584) bank15223 +15224 POINT(33.81968292387178 -117.46436540252051) bank15224 +15225 POINT(33.85743238667058 -117.89858831484958) bank15225 +15226 POINT(33.78478686518423 -118.46786201534871) bank15226 +15227 POINT(33.48978463338077 -118.61129673104296) bank15227 +15228 POINT(33.54045637697785 -117.68661049633654) bank15228 +15229 POINT(33.53100460447013 -117.52241539981438) bank15229 +15230 POINT(34.10392909737416 -118.05787509520735) bank15230 +15231 POINT(33.68317519411219 -118.00204078051105) bank15231 +15232 POINT(34.541142020079256 -118.8866128067073) bank15232 +15233 POINT(34.09361631854425 -118.65857846918883) bank15233 +15234 POINT(34.015239077207596 -117.38094694195344) bank15234 +15235 POINT(33.21047580424879 -119.00757723004595) bank15235 +15236 POINT(33.30166029328053 -119.11424020108804) bank15236 +15237 POINT(33.19130257958182 -118.22075310633969) bank15237 +15238 POINT(34.91474972336596 -118.79958972345604) bank15238 +15239 POINT(34.28003311621286 -117.61216024286252) bank15239 +15240 POINT(33.39175733325164 -119.1535244555782) bank15240 +15241 POINT(34.4059821774251 -119.12322493918924) bank15241 +15242 POINT(34.60980350745332 -117.41533925015366) bank15242 +15243 POINT(33.77669269799818 -117.53506479971271) bank15243 +15244 POINT(33.62831788596797 -118.6008478846941) bank15244 +15245 POINT(34.056735156879284 -117.79390589003839) bank15245 +15246 POINT(34.346856265710755 -117.66176199542862) bank15246 +15247 POINT(33.44245478458133 -117.37952731326492) bank15247 +15248 POINT(34.885208591973935 -118.68004528340511) bank15248 +15249 POINT(34.23837896107253 -118.92999941460978) bank15249 +15250 POINT(33.18338335519396 -117.68960132070168) bank15250 +15251 POINT(34.19185105746809 -118.35242277525677) bank15251 +15252 POINT(34.03820308179347 -118.2676805050406) bank15252 +15253 POINT(34.29155794746706 -118.71452165192323) bank15253 +15254 POINT(33.90772410144814 -117.26915132084432) bank15254 +15255 POINT(34.58502413291295 -117.62323144100291) bank15255 +15256 POINT(35.0103389056702 -118.67362509663954) bank15256 +15257 POINT(34.75318881923847 -117.41732804966476) bank15257 +15258 POINT(33.96261956778261 -119.2427458883158) bank15258 +15259 POINT(34.837158300713575 -118.70948996491195) bank15259 +15260 POINT(34.63730202160751 -119.0172662187719) bank15260 +15261 POINT(33.3675759983612 -117.49918007294019) bank15261 +15262 POINT(33.80877839567327 -117.32925396764911) bank15262 +15263 POINT(33.73296455631127 -117.53934041628968) bank15263 +15264 POINT(33.58996966015569 -117.92436610015315) bank15264 +15265 POINT(33.886480275870575 -117.56698872071314) bank15265 +15266 POINT(33.9204194949006 -119.16777767942638) bank15266 +15267 POINT(33.19682185415624 -118.88603133493203) bank15267 +15268 POINT(33.88952163000272 -117.57847220600102) bank15268 +15269 POINT(34.65816305507028 -119.05291737016837) bank15269 +15270 POINT(33.9051308900216 -118.06038732025831) bank15270 +15271 POINT(33.508420731375196 -117.61313231983203) bank15271 +15272 POINT(33.675264648448426 -118.49657495050639) bank15272 +15273 POINT(34.92415537666733 -117.27101423952705) bank15273 +15274 POINT(34.84507181523655 -118.55201018058227) bank15274 +15275 POINT(34.60629617756628 -119.0972162515583) bank15275 +15276 POINT(34.14245268165348 -118.40661693046343) bank15276 +15277 POINT(34.86286893138029 -118.18178915680447) bank15277 +15278 POINT(34.169408080617 -118.84277507890779) bank15278 +15279 POINT(34.88053077646348 -118.97446577150805) bank15279 +15280 POINT(33.771662583380866 -118.24944492790105) bank15280 +15281 POINT(34.26244195059533 -118.17204917655603) bank15281 +15282 POINT(34.61535611824501 -118.8272508638984) bank15282 +15283 POINT(34.4185266533243 -118.2470571594608) bank15283 +15284 POINT(34.22911382392475 -117.38617606960543) bank15284 +15285 POINT(34.38687627820478 -118.46482184396685) bank15285 +15286 POINT(34.80211009233401 -118.98010693949942) bank15286 +15287 POINT(33.890392565756535 -118.89550690382721) bank15287 +15288 POINT(34.55883199436289 -117.81102969536947) bank15288 +15289 POINT(33.371112630076894 -119.04725347988986) bank15289 +15290 POINT(33.447461903020276 -118.88442626099243) bank15290 +15291 POINT(33.16500038849714 -117.3202746445129) bank15291 +15292 POINT(34.40125198428854 -118.45126236547823) bank15292 +15293 POINT(33.852910924694285 -119.07998104352635) bank15293 +15294 POINT(34.50460583605497 -117.32242184578362) bank15294 +15295 POINT(34.602045971242354 -118.44767334680282) bank15295 +15296 POINT(33.7588877204756 -118.72461013596069) bank15296 +15297 POINT(33.11940936766766 -117.9724825016802) bank15297 +15298 POINT(34.35475945406243 -117.34729456922305) bank15298 +15299 POINT(33.225560227269604 -117.60231794951186) bank15299 +15300 POINT(35.017356804449186 -118.5513038807008) bank15300 +15301 POINT(33.38512996784815 -118.80853782581613) bank15301 +15302 POINT(34.44950310814982 -117.38503297240693) bank15302 +15303 POINT(33.558169984686266 -118.26606586984049) bank15303 +15304 POINT(34.62628091300607 -118.75667317764885) bank15304 +15305 POINT(34.259470420460104 -117.74910572748828) bank15305 +15306 POINT(34.526984224095166 -119.23046935614195) bank15306 +15307 POINT(33.42511766654234 -118.99743883894908) bank15307 +15308 POINT(33.11316879964769 -117.69537495713737) bank15308 +15309 POINT(33.29448756872207 -118.60724391401685) bank15309 +15310 POINT(33.43512377009444 -117.54332962975317) bank15310 +15311 POINT(33.27173348369395 -119.14303987915771) bank15311 +15312 POINT(34.006427367634885 -117.37836957667565) bank15312 +15313 POINT(34.1439897356432 -118.47391826945243) bank15313 +15314 POINT(33.725319414493335 -118.28137096418035) bank15314 +15315 POINT(33.87138452811078 -118.13530150278659) bank15315 +15316 POINT(33.957196840171214 -117.75185023761667) bank15316 +15317 POINT(33.43552144891767 -119.03037173913675) bank15317 +15318 POINT(33.200804852623456 -117.5258802606941) bank15318 +15319 POINT(33.779719978083584 -118.69507145299154) bank15319 +15320 POINT(34.67452457818293 -117.4651466179929) bank15320 +15321 POINT(34.40816713780645 -117.77965175781095) bank15321 +15322 POINT(34.224282444976545 -119.1261081439081) bank15322 +15323 POINT(34.17609198852939 -118.78967472460566) bank15323 +15324 POINT(34.00784201097629 -117.65339777778254) bank15324 +15325 POINT(34.5043915697418 -118.35930975384866) bank15325 +15326 POINT(34.545794415352496 -119.15626534993686) bank15326 +15327 POINT(34.56465874750402 -118.8999550547451) bank15327 +15328 POINT(33.18979566705239 -117.94935875311042) bank15328 +15329 POINT(33.158691290286825 -117.29965955583422) bank15329 +15330 POINT(34.82163398685147 -117.36498424088195) bank15330 +15331 POINT(34.78063586517665 -119.2301728528857) bank15331 +15332 POINT(33.74196086930715 -118.20071588593775) bank15332 +15333 POINT(34.62726903580605 -118.12632820018783) bank15333 +15334 POINT(33.34947499528223 -118.94699454612763) bank15334 +15335 POINT(33.78157152163314 -118.77163855857164) bank15335 +15336 POINT(33.784542925682544 -117.40230521943896) bank15336 +15337 POINT(34.4340670142921 -118.60278718763053) bank15337 +15338 POINT(33.209243522602456 -117.35302531814825) bank15338 +15339 POINT(34.77566427722194 -118.86816927072759) bank15339 +15340 POINT(33.66129267696966 -118.12146522965352) bank15340 +15341 POINT(33.094442343949225 -119.04637111395245) bank15341 +15342 POINT(34.02310421757305 -118.0969089648585) bank15342 +15343 POINT(34.39849617671413 -118.06209063785413) bank15343 +15344 POINT(33.54773470896971 -117.27630725486819) bank15344 +15345 POINT(34.38288225821393 -118.19623031564225) bank15345 +15346 POINT(33.481173008333045 -117.3273886470536) bank15346 +15347 POINT(33.25974066431292 -118.70745502949771) bank15347 +15348 POINT(34.627502873807266 -117.62949999724009) bank15348 +15349 POINT(34.60625448642397 -118.18099672364156) bank15349 +15350 POINT(33.624730369605324 -119.21495976665479) bank15350 +15351 POINT(34.458180695186904 -118.43845501916653) bank15351 +15352 POINT(34.15561182510332 -118.8891016087111) bank15352 +15353 POINT(34.130747200803235 -117.45212111240537) bank15353 +15354 POINT(33.54766833153073 -117.99075913702461) bank15354 +15355 POINT(34.64570948105178 -117.36105890960937) bank15355 +15356 POINT(33.10943199517283 -118.89933222015813) bank15356 +15357 POINT(34.44515515695589 -117.48534688633276) bank15357 +15358 POINT(34.998832898046146 -117.28026456406779) bank15358 +15359 POINT(33.076884683984474 -117.3936947458225) bank15359 +15360 POINT(34.82598077210613 -118.00106964078333) bank15360 +15361 POINT(34.71289772261759 -118.23283316115966) bank15361 +15362 POINT(34.7729580508789 -117.97188170099426) bank15362 +15363 POINT(34.935659074763784 -118.44562930597216) bank15363 +15364 POINT(33.520748011409744 -117.35166108109705) bank15364 +15365 POINT(34.11316012071189 -117.38883768658523) bank15365 +15366 POINT(34.232559306624644 -118.44929216748442) bank15366 +15367 POINT(33.28895561392713 -118.11056442471669) bank15367 +15368 POINT(34.183525752278165 -119.19869198132169) bank15368 +15369 POINT(33.51592934060802 -117.28332270569841) bank15369 +15370 POINT(34.423227548784894 -119.22014856821912) bank15370 +15371 POINT(33.833448599315766 -117.65250280158284) bank15371 +15372 POINT(33.319785471252686 -119.03799042284183) bank15372 +15373 POINT(33.21349869012919 -118.69204999431318) bank15373 +15374 POINT(33.62028221105178 -118.77340086809778) bank15374 +15375 POINT(33.08586150028369 -119.22393137413522) bank15375 +15376 POINT(34.90072665770722 -117.61819497939139) bank15376 +15377 POINT(33.46538112176118 -117.72292374091228) bank15377 +15378 POINT(33.36578950270804 -118.28814744496576) bank15378 +15379 POINT(33.68069458054123 -118.00219260166946) bank15379 +15380 POINT(33.31247251836994 -118.73167859292722) bank15380 +15381 POINT(33.161980622853605 -118.08460420530281) bank15381 +15382 POINT(33.71849879276316 -119.09944676893072) bank15382 +15383 POINT(33.242783438667345 -118.26558767646536) bank15383 +15384 POINT(34.39063702878234 -118.42478280496647) bank15384 +15385 POINT(33.19382427607137 -118.67813821637901) bank15385 +15386 POINT(33.46186446477012 -118.68927660414997) bank15386 +15387 POINT(33.30197532166639 -118.84009747910551) bank15387 +15388 POINT(33.820351254153834 -118.66919527745162) bank15388 +15389 POINT(33.591627871545214 -117.55165256773356) bank15389 +15390 POINT(34.22214528875342 -117.97690085846196) bank15390 +15391 POINT(33.36071081973353 -118.06415858369877) bank15391 +15392 POINT(33.20520737264945 -117.96913985295693) bank15392 +15393 POINT(33.383245322910724 -117.92436493565383) bank15393 +15394 POINT(34.65764361145398 -118.13986161404287) bank15394 +15395 POINT(34.25801128789964 -117.27181928736658) bank15395 +15396 POINT(34.27101529439142 -118.94500794663338) bank15396 +15397 POINT(34.2537155204924 -118.02142639864775) bank15397 +15398 POINT(34.48779883485031 -118.44016629545453) bank15398 +15399 POINT(33.111331279276584 -118.06422255294594) bank15399 +15400 POINT(34.63953932438246 -118.60366294394449) bank15400 +15401 POINT(33.85173362018181 -117.95348097543716) bank15401 +15402 POINT(34.90199678239383 -118.94817879521847) bank15402 +15403 POINT(34.08370333918715 -118.60431031174184) bank15403 +15404 POINT(34.29545178971291 -117.66507737788419) bank15404 +15405 POINT(34.73383563791587 -119.09045380857977) bank15405 +15406 POINT(33.11676114514859 -118.90384439868602) bank15406 +15407 POINT(33.74863280901753 -117.93454543333941) bank15407 +15408 POINT(33.87008447167806 -118.96373469115744) bank15408 +15409 POINT(33.96116532899436 -118.65657695312858) bank15409 +15410 POINT(33.761061758247806 -119.18694103992019) bank15410 +15411 POINT(33.50055167657022 -117.59974621160163) bank15411 +15412 POINT(34.42498684090246 -118.8263543517886) bank15412 +15413 POINT(33.84322383925095 -118.57490895899979) bank15413 +15414 POINT(34.69292228563094 -117.53975334806587) bank15414 +15415 POINT(35.02024288196862 -119.02319449048308) bank15415 +15416 POINT(34.871484520740566 -117.62797379547705) bank15416 +15417 POINT(34.10991879407976 -117.48814768540569) bank15417 +15418 POINT(34.81113125228147 -119.04362568848192) bank15418 +15419 POINT(34.5740660531339 -118.00912510532372) bank15419 +15420 POINT(33.54383340534461 -119.23897484163383) bank15420 +15421 POINT(33.44854912777562 -118.1694578926428) bank15421 +15422 POINT(34.096212179955636 -117.6837633524862) bank15422 +15423 POINT(33.37606390166659 -118.732531550011) bank15423 +15424 POINT(34.75599345581782 -119.11853984910726) bank15424 +15425 POINT(33.3903765216413 -117.39506081218731) bank15425 +15426 POINT(33.281424767087415 -119.10314194644373) bank15426 +15427 POINT(34.93023971243717 -118.38553635725899) bank15427 +15428 POINT(33.462000791238104 -117.59352659043432) bank15428 +15429 POINT(33.85044801719705 -117.69316250477857) bank15429 +15430 POINT(35.00048928788688 -117.46693623233288) bank15430 +15431 POINT(34.94344371588717 -117.59388466024718) bank15431 +15432 POINT(34.293732029758374 -118.02826281349274) bank15432 +15433 POINT(34.02666216659475 -118.16794800777467) bank15433 +15434 POINT(34.68664112867223 -117.36809748729634) bank15434 +15435 POINT(33.164311351904715 -117.31188720306321) bank15435 +15436 POINT(34.19879693321163 -117.41775237226345) bank15436 +15437 POINT(33.896096060839355 -119.10351257234277) bank15437 +15438 POINT(34.41710648473342 -117.82094882154023) bank15438 +15439 POINT(34.602290499959516 -118.49160217732495) bank15439 +15440 POINT(33.831536915444424 -118.02768691118318) bank15440 +15441 POINT(34.15308144419 -117.74588176650717) bank15441 +15442 POINT(33.07850725778272 -117.76891159070964) bank15442 +15443 POINT(34.157487762608504 -117.60679786405372) bank15443 +15444 POINT(33.209593495750944 -118.75486493791065) bank15444 +15445 POINT(33.2129959846819 -117.88320173852102) bank15445 +15446 POINT(33.20037432040536 -118.93066058894755) bank15446 +15447 POINT(34.15596880541432 -117.57043867589414) bank15447 +15448 POINT(33.82232157745022 -117.2802965151471) bank15448 +15449 POINT(33.47605627590988 -118.39299677358827) bank15449 +15450 POINT(33.959909947783835 -118.229093660523) bank15450 +15451 POINT(34.92406514254912 -117.28381264908211) bank15451 +15452 POINT(33.900671309238696 -117.30012266460255) bank15452 +15453 POINT(34.82377351866534 -117.91921113757576) bank15453 +15454 POINT(34.560761218008594 -117.5870629892162) bank15454 +15455 POINT(34.350849095984806 -118.84914966515036) bank15455 +15456 POINT(33.879482442320594 -117.84442468285793) bank15456 +15457 POINT(33.07704979918261 -117.42869039655459) bank15457 +15458 POINT(34.50003169908548 -118.39539147185732) bank15458 +15459 POINT(33.59403587959032 -117.32029668639792) bank15459 +15460 POINT(33.35351239201886 -118.98299810787714) bank15460 +15461 POINT(34.74327209097953 -118.81595192068919) bank15461 +15462 POINT(34.53573272617184 -117.71650459015022) bank15462 +15463 POINT(35.032677404672576 -118.98121164532964) bank15463 +15464 POINT(33.9335256211662 -118.04384166084239) bank15464 +15465 POINT(33.779979155335944 -119.02022566537075) bank15465 +15466 POINT(33.9754120080853 -117.6454054522116) bank15466 +15467 POINT(33.74923698757271 -117.98736758431444) bank15467 +15468 POINT(34.24076570572832 -117.9803194846871) bank15468 +15469 POINT(34.65215992471995 -117.36509281693537) bank15469 +15470 POINT(33.635106814801965 -117.98390706953342) bank15470 +15471 POINT(33.374726261013656 -117.93007202142236) bank15471 +15472 POINT(34.80779215859778 -118.60000247883238) bank15472 +15473 POINT(34.49930288239586 -119.04210365844241) bank15473 +15474 POINT(34.91932894576063 -117.33643323132769) bank15474 +15475 POINT(34.300711328106715 -117.83872577563052) bank15475 +15476 POINT(33.40334022219226 -118.65822896926518) bank15476 +15477 POINT(33.70725087924544 -119.16554549824932) bank15477 +15478 POINT(34.76975098880897 -117.72291769317533) bank15478 +15479 POINT(33.67960072327071 -119.17327751384795) bank15479 +15480 POINT(34.01151297618591 -118.92197839452162) bank15480 +15481 POINT(34.750874473019394 -118.49792889378462) bank15481 +15482 POINT(34.34851749811509 -117.4412334543012) bank15482 +15483 POINT(34.53841417818991 -117.80760855443418) bank15483 +15484 POINT(33.89822925687632 -117.9034396668129) bank15484 +15485 POINT(33.53736777556795 -117.99961170339397) bank15485 +15486 POINT(33.289230566535444 -117.81112899054334) bank15486 +15487 POINT(33.78808164914781 -117.37572377767994) bank15487 +15488 POINT(33.485300956193825 -117.40426899734364) bank15488 +15489 POINT(34.21589905067445 -119.21904063409914) bank15489 +15490 POINT(33.86507056271439 -119.24237107343775) bank15490 +15491 POINT(34.20988205652518 -118.20663045722243) bank15491 +15492 POINT(33.27070493359647 -117.92944157379776) bank15492 +15493 POINT(33.974559879731125 -119.15498194204577) bank15493 +15494 POINT(34.55434488571795 -117.46528170931127) bank15494 +15495 POINT(34.39087676985981 -118.20847142170649) bank15495 +15496 POINT(33.6611256520074 -118.77189420170821) bank15496 +15497 POINT(33.24812421351733 -117.55535873078551) bank15497 +15498 POINT(34.63101107243554 -118.22658393861008) bank15498 +15499 POINT(34.367982376928715 -118.92847163879459) bank15499 +15500 POINT(33.24340734526992 -118.042596779907) bank15500 +15501 POINT(33.85395861047559 -117.86004777328411) bank15501 +15502 POINT(34.06028365299638 -118.62968154757515) bank15502 +15503 POINT(33.56797868684289 -118.81394584921794) bank15503 +15504 POINT(34.980012362660986 -118.68954243384155) bank15504 +15505 POINT(33.571532909879494 -119.01484347945377) bank15505 +15506 POINT(33.64186223821359 -118.64436333906443) bank15506 +15507 POINT(33.99108671220296 -117.88672871574208) bank15507 +15508 POINT(33.4923339229374 -119.11973742140773) bank15508 +15509 POINT(34.59260226371388 -117.38302582949412) bank15509 +15510 POINT(33.59642732786576 -117.77073623975629) bank15510 +15511 POINT(34.114783963147424 -119.0650310210376) bank15511 +15512 POINT(34.88411158277803 -118.40734513403021) bank15512 +15513 POINT(34.676017493645595 -118.40504686567402) bank15513 +15514 POINT(33.925787705529395 -118.29297925080076) bank15514 +15515 POINT(33.480642398126776 -119.00238305734959) bank15515 +15516 POINT(34.20527291690717 -118.54679538808523) bank15516 +15517 POINT(34.31153336427787 -118.00252760800448) bank15517 +15518 POINT(33.29728852611926 -117.75936928556408) bank15518 +15519 POINT(33.7040708235831 -117.66361871936473) bank15519 +15520 POINT(34.05738056796869 -118.8125549836471) bank15520 +15521 POINT(34.637633061629884 -117.93659037118645) bank15521 +15522 POINT(33.79532428077737 -117.6953379067665) bank15522 +15523 POINT(33.545336003945806 -117.43189791183093) bank15523 +15524 POINT(34.13270489941858 -118.92762780231011) bank15524 +15525 POINT(34.66334207593538 -117.70231313849854) bank15525 +15526 POINT(34.783204717557496 -117.24448695787117) bank15526 +15527 POINT(34.761553423828495 -118.41500132721062) bank15527 +15528 POINT(34.84371640412515 -118.1843844822578) bank15528 +15529 POINT(34.66784989255378 -117.73939993660137) bank15529 +15530 POINT(33.32613682578902 -119.10998818875687) bank15530 +15531 POINT(34.32456926750542 -119.23372774717139) bank15531 +15532 POINT(34.65356949482936 -119.05643557694908) bank15532 +15533 POINT(34.82580909245937 -118.6569114405822) bank15533 +15534 POINT(34.216831604279186 -117.80788006179186) bank15534 +15535 POINT(34.22015874808661 -118.56622876611347) bank15535 +15536 POINT(33.96582710444349 -117.39486754983275) bank15536 +15537 POINT(34.27122393939963 -118.53397295064374) bank15537 +15538 POINT(33.43966943763652 -119.09870571102847) bank15538 +15539 POINT(33.74461104655906 -117.38022730022021) bank15539 +15540 POINT(33.212003173221824 -118.7699332694876) bank15540 +15541 POINT(34.569175197429345 -118.07795372652977) bank15541 +15542 POINT(34.98644662396714 -117.93391978036094) bank15542 +15543 POINT(34.92449071831928 -118.49345416402362) bank15543 +15544 POINT(33.46401271965486 -117.28361613159211) bank15544 +15545 POINT(33.84455161243423 -118.70446183697757) bank15545 +15546 POINT(34.960008620282466 -117.54592252292906) bank15546 +15547 POINT(34.33369869285754 -119.03366337331391) bank15547 +15548 POINT(33.33500823661605 -119.01324183841864) bank15548 +15549 POINT(33.6571868230235 -119.23436137586839) bank15549 +15550 POINT(33.604644226439 -118.261892228057) bank15550 +15551 POINT(33.166746225693856 -117.71671814419989) bank15551 +15552 POINT(33.986400841072864 -117.74546389771562) bank15552 +15553 POINT(33.82760044009839 -118.2015367034344) bank15553 +15554 POINT(34.30611838966437 -118.67891661952997) bank15554 +15555 POINT(33.918086145753236 -119.13695570207575) bank15555 +15556 POINT(33.996957404659966 -117.89921079735825) bank15556 +15557 POINT(33.8545763706967 -118.46938835863503) bank15557 +15558 POINT(34.33158566506684 -119.16987665072274) bank15558 +15559 POINT(34.24323526586179 -119.21994239212148) bank15559 +15560 POINT(33.63308262503774 -118.95859597478308) bank15560 +15561 POINT(34.899861527800155 -118.09351238973329) bank15561 +15562 POINT(34.205008120193774 -117.3441473174608) bank15562 +15563 POINT(33.93346524513526 -117.48528057812149) bank15563 +15564 POINT(34.53831488883386 -118.93664804090305) bank15564 +15565 POINT(33.987949270243426 -117.27299146278014) bank15565 +15566 POINT(34.95365556689143 -118.46511464334007) bank15566 +15567 POINT(34.85351517926242 -118.49778427200272) bank15567 +15568 POINT(34.68910148791839 -117.43740601460689) bank15568 +15569 POINT(33.8419417574182 -117.89707719320288) bank15569 +15570 POINT(34.65087501133439 -119.03742759114202) bank15570 +15571 POINT(33.077893958686886 -118.20155088984014) bank15571 +15572 POINT(33.46248997024266 -119.18180548499146) bank15572 +15573 POINT(34.19863258239223 -118.38848068442863) bank15573 +15574 POINT(34.896459917758506 -119.24256421518686) bank15574 +15575 POINT(33.155432308219346 -117.77680285533675) bank15575 +15576 POINT(34.131259015004794 -118.27229249934099) bank15576 +15577 POINT(34.350024492678024 -118.8082936395972) bank15577 +15578 POINT(34.41233363910015 -118.3540999514815) bank15578 +15579 POINT(34.5427361336009 -117.30734708738458) bank15579 +15580 POINT(34.33060358443539 -118.19774929735952) bank15580 +15581 POINT(34.896887721442106 -118.37682533279711) bank15581 +15582 POINT(34.50452400010009 -119.07929859376723) bank15582 +15583 POINT(34.49989282873546 -118.01842708182193) bank15583 +15584 POINT(33.342429520861174 -119.23149729321584) bank15584 +15585 POINT(33.270780274411244 -118.4805419881932) bank15585 +15586 POINT(34.781249804858646 -117.4845583749072) bank15586 +15587 POINT(34.50849551052934 -118.57825166301666) bank15587 +15588 POINT(33.682023833644365 -119.12573625732246) bank15588 +15589 POINT(34.84690312380383 -118.35967243520471) bank15589 +15590 POINT(34.41886499162625 -117.62850402928026) bank15590 +15591 POINT(34.26384982530961 -118.55535955787347) bank15591 +15592 POINT(33.86336255063966 -117.88650612520787) bank15592 +15593 POINT(34.37284069599188 -117.93096467422887) bank15593 +15594 POINT(34.9260866863805 -119.12887020834955) bank15594 +15595 POINT(33.314524910579955 -118.02827856266688) bank15595 +15596 POINT(34.426833603259155 -118.42294416025811) bank15596 +15597 POINT(33.620305084132056 -118.00320483010516) bank15597 +15598 POINT(34.993604363175514 -117.55481940343851) bank15598 +15599 POINT(34.20319225966716 -118.73092743713252) bank15599 +15600 POINT(33.31896139311663 -118.49535381717338) bank15600 +15601 POINT(34.05948739184209 -118.45673052418933) bank15601 +15602 POINT(34.45115977849693 -118.19925055277211) bank15602 +15603 POINT(34.538683335764794 -118.59128479790148) bank15603 +15604 POINT(34.83130230233172 -118.4096428659279) bank15604 +15605 POINT(33.26162345579268 -117.46552133632925) bank15605 +15606 POINT(34.15674907712537 -119.17839412536145) bank15606 +15607 POINT(34.78111778649751 -118.49668080277044) bank15607 +15608 POINT(33.96533808228678 -118.65594769233037) bank15608 +15609 POINT(33.35645038969387 -118.18940371618514) bank15609 +15610 POINT(33.785590160026295 -118.9886423518751) bank15610 +15611 POINT(33.81010676408348 -118.55957827718073) bank15611 +15612 POINT(33.599974443418695 -118.90300941603354) bank15612 +15613 POINT(35.03275334445239 -117.55753650768655) bank15613 +15614 POINT(33.13851445213681 -118.54574799422704) bank15614 +15615 POINT(33.17680041781709 -118.95725396762707) bank15615 +15616 POINT(33.953105367204934 -118.39720346082319) bank15616 +15617 POINT(33.44448420527224 -117.56245723700006) bank15617 +15618 POINT(33.410140808599216 -117.5903627070906) bank15618 +15619 POINT(34.873820823894754 -118.1500782941104) bank15619 +15620 POINT(33.700658419507555 -119.06603036040144) bank15620 +15621 POINT(34.007799658805645 -118.877300512918) bank15621 +15622 POINT(33.319157065086884 -118.03105185524774) bank15622 +15623 POINT(34.06346321897929 -117.43617944661752) bank15623 +15624 POINT(33.84029125458194 -119.03637773424613) bank15624 +15625 POINT(34.99041770046046 -119.02985216909542) bank15625 +15626 POINT(34.070974999833105 -118.82089037224956) bank15626 +15627 POINT(33.654395629943195 -117.55824006925458) bank15627 +15628 POINT(33.87177275226061 -117.96823487700178) bank15628 +15629 POINT(33.40701958260647 -118.47361895024875) bank15629 +15630 POINT(34.21463694019373 -118.43337521865386) bank15630 +15631 POINT(33.62028700529928 -118.58815059437865) bank15631 +15632 POINT(33.35472864635604 -117.56279224400411) bank15632 +15633 POINT(34.828947627505045 -117.64553500536525) bank15633 +15634 POINT(34.13786847787275 -117.65650786262617) bank15634 +15635 POINT(33.72768403008862 -118.43822232482886) bank15635 +15636 POINT(33.46521867619319 -117.91234128577679) bank15636 +15637 POINT(34.75376440957702 -117.93306256436485) bank15637 +15638 POINT(34.716359205102584 -117.65603209607177) bank15638 +15639 POINT(34.71395407581009 -117.77592675546073) bank15639 +15640 POINT(33.65930543803826 -117.41515011382008) bank15640 +15641 POINT(34.5609331675568 -117.62203875996643) bank15641 +15642 POINT(35.01014795939865 -118.47268196814606) bank15642 +15643 POINT(34.93840530434015 -117.78843237918196) bank15643 +15644 POINT(33.65799055968725 -118.69805580637697) bank15644 +15645 POINT(34.756297886981095 -117.41840622384147) bank15645 +15646 POINT(34.62110655228462 -118.4178652144429) bank15646 +15647 POINT(33.617121556560434 -118.54235903742335) bank15647 +15648 POINT(33.69266149800093 -117.42849506824969) bank15648 +15649 POINT(34.2239224643631 -117.62608274432087) bank15649 +15650 POINT(34.8018547478676 -118.39602590168158) bank15650 +15651 POINT(34.7168333986198 -117.45404647569121) bank15651 +15652 POINT(33.598378974053915 -118.69685567362227) bank15652 +15653 POINT(33.69665917803958 -118.11628968964669) bank15653 +15654 POINT(33.20815772384939 -118.696913272219) bank15654 +15655 POINT(33.86345603058055 -117.68691171606373) bank15655 +15656 POINT(34.89740845322085 -118.3556095876842) bank15656 +15657 POINT(34.445172800426334 -118.58938918058479) bank15657 +15658 POINT(33.75407392083065 -118.9426758446209) bank15658 +15659 POINT(34.866554614272964 -119.21560564758786) bank15659 +15660 POINT(34.463342930869 -119.09005609397934) bank15660 +15661 POINT(33.61337547474017 -119.01596340939072) bank15661 +15662 POINT(34.10439734710444 -118.02562891197607) bank15662 +15663 POINT(33.77439260693469 -118.91947279906141) bank15663 +15664 POINT(33.82859678672345 -118.14287955600592) bank15664 +15665 POINT(34.475231776880534 -117.5340097879685) bank15665 +15666 POINT(33.09924600981794 -118.80441176056138) bank15666 +15667 POINT(34.53296149084197 -118.95137767902798) bank15667 +15668 POINT(34.41322115989072 -118.08665127770188) bank15668 +15669 POINT(34.46449531157118 -118.94253648611253) bank15669 +15670 POINT(33.09884345255785 -117.53172894938822) bank15670 +15671 POINT(34.97633010144463 -118.82601339770738) bank15671 +15672 POINT(33.66641055336614 -118.23870302535052) bank15672 +15673 POINT(34.73200538927759 -118.51659870960066) bank15673 +15674 POINT(33.36396484271827 -119.11746842009902) bank15674 +15675 POINT(33.12512141445949 -118.63512436456168) bank15675 +15676 POINT(34.91841351439846 -117.42367700082102) bank15676 +15677 POINT(34.63325361077152 -118.77106936407338) bank15677 +15678 POINT(33.821296094112 -117.4916976909259) bank15678 +15679 POINT(34.87879239819199 -118.66299484283851) bank15679 +15680 POINT(34.116338898472094 -117.66891679080206) bank15680 +15681 POINT(33.484377863705696 -117.87142915057191) bank15681 +15682 POINT(34.53484526734036 -117.72989026832703) bank15682 +15683 POINT(34.59137565259415 -117.79977026156985) bank15683 +15684 POINT(33.51216456586422 -118.35205460458972) bank15684 +15685 POINT(33.53322636864143 -118.34623593177358) bank15685 +15686 POINT(34.29090520646916 -117.94703708148121) bank15686 +15687 POINT(33.93251756167732 -117.26093401593025) bank15687 +15688 POINT(33.49826308456238 -118.59182406575843) bank15688 +15689 POINT(33.51025518797613 -118.03069785822238) bank15689 +15690 POINT(33.96783380885337 -117.757336223555) bank15690 +15691 POINT(34.14923708590768 -118.75982843044467) bank15691 +15692 POINT(34.11980229298403 -118.32024422432362) bank15692 +15693 POINT(33.98841143070981 -118.46111254862555) bank15693 +15694 POINT(34.39354254627625 -118.96231651923304) bank15694 +15695 POINT(33.26255918853647 -118.51232117714423) bank15695 +15696 POINT(34.096127263935095 -118.67618860253572) bank15696 +15697 POINT(34.74358934982154 -119.17431684398801) bank15697 +15698 POINT(34.76568244814123 -118.2095141758384) bank15698 +15699 POINT(33.543433463119555 -119.0740050107217) bank15699 +15700 POINT(33.42832876056335 -118.26479884892402) bank15700 +15701 POINT(33.82894078448227 -118.81017537097223) bank15701 +15702 POINT(34.84102115631294 -118.7012353936071) bank15702 +15703 POINT(34.547268164619354 -118.4232458296786) bank15703 +15704 POINT(33.89407886890466 -118.38987712171011) bank15704 +15705 POINT(33.38624438798916 -118.95591274351062) bank15705 +15706 POINT(33.589747649064584 -118.64092716986853) bank15706 +15707 POINT(33.81589127600753 -117.49681665760914) bank15707 +15708 POINT(35.01848460291426 -118.08873442971934) bank15708 +15709 POINT(33.078789093694155 -118.1281211798514) bank15709 +15710 POINT(33.57856772413221 -117.26552140555053) bank15710 +15711 POINT(33.310780476150285 -117.54759385764153) bank15711 +15712 POINT(34.772402387524124 -117.510518200923) bank15712 +15713 POINT(33.13153489316695 -119.11701515010591) bank15713 +15714 POINT(33.68254038127796 -118.20137780555262) bank15714 +15715 POINT(33.96424923713744 -118.7472297317654) bank15715 +15716 POINT(33.24872520919399 -117.94889424479089) bank15716 +15717 POINT(34.78227303519809 -117.26411212871079) bank15717 +15718 POINT(33.70378769972157 -118.00330075376971) bank15718 +15719 POINT(34.83747181096445 -117.75845177895388) bank15719 +15720 POINT(33.701263909755646 -118.6754477641069) bank15720 +15721 POINT(35.04806841345434 -118.54298212256101) bank15721 +15722 POINT(33.49749812515386 -119.11990266478455) bank15722 +15723 POINT(34.72185153963387 -118.16440586490253) bank15723 +15724 POINT(34.47299853378009 -117.59926108727306) bank15724 +15725 POINT(33.70513829622391 -119.19981865289458) bank15725 +15726 POINT(33.42130456686795 -117.7733715460432) bank15726 +15727 POINT(33.22759172381064 -118.27244532161895) bank15727 +15728 POINT(34.46634160709579 -117.70819909388068) bank15728 +15729 POINT(34.00567573861437 -118.82079652502256) bank15729 +15730 POINT(33.20965950290341 -117.50334916008082) bank15730 +15731 POINT(33.21209548368962 -118.103415622515) bank15731 +15732 POINT(34.89964928127571 -118.70328329541258) bank15732 +15733 POINT(34.47151104677561 -118.70418058514507) bank15733 +15734 POINT(33.591002020272896 -117.6524776041975) bank15734 +15735 POINT(33.29647830929415 -119.23358101329076) bank15735 +15736 POINT(33.93025743501486 -118.12651635369267) bank15736 +15737 POINT(34.61946143237648 -117.8265372052523) bank15737 +15738 POINT(33.636229662325206 -118.85798014786067) bank15738 +15739 POINT(33.384544729529686 -118.15738151786775) bank15739 +15740 POINT(33.84884654783078 -117.74282592186684) bank15740 +15741 POINT(34.84497475827068 -117.53575767030033) bank15741 +15742 POINT(34.57465654603563 -118.2497060794103) bank15742 +15743 POINT(34.34579433220399 -118.03491924609958) bank15743 +15744 POINT(33.83433149812885 -118.28261375442446) bank15744 +15745 POINT(34.09149760077961 -117.34261847856384) bank15745 +15746 POINT(34.10456892354964 -117.77342036345308) bank15746 +15747 POINT(33.93863590984979 -118.32838201665597) bank15747 +15748 POINT(33.596369033124 -117.96426486323624) bank15748 +15749 POINT(33.82730386484528 -117.98678219564627) bank15749 +15750 POINT(34.165055689186744 -117.75018401340598) bank15750 +15751 POINT(33.59765352022508 -118.73883239444841) bank15751 +15752 POINT(34.812961961624886 -117.93908121618882) bank15752 +15753 POINT(33.89009481756102 -118.74357900800113) bank15753 +15754 POINT(34.50875954320259 -117.53564857517485) bank15754 +15755 POINT(34.96169396312096 -118.35566972696098) bank15755 +15756 POINT(33.05930733822185 -117.72194047691143) bank15756 +15757 POINT(34.215561236586446 -118.19697994650728) bank15757 +15758 POINT(33.68020655084046 -119.04198514669608) bank15758 +15759 POINT(34.92060462395668 -118.07309934565068) bank15759 +15760 POINT(34.268504975801314 -117.537017465678) bank15760 +15761 POINT(34.02191575354658 -117.50562853326048) bank15761 +15762 POINT(34.43630342508532 -117.91858633685548) bank15762 +15763 POINT(34.056372642938875 -117.76892934693377) bank15763 +15764 POINT(34.664481144236476 -117.32565378158209) bank15764 +15765 POINT(34.44682184081141 -117.90108711791008) bank15765 +15766 POINT(33.93780754032175 -118.74151345707841) bank15766 +15767 POINT(33.64221653472344 -118.87487407700351) bank15767 +15768 POINT(34.75311703324314 -117.5001083132961) bank15768 +15769 POINT(33.20728690182005 -119.15000955961962) bank15769 +15770 POINT(34.63149527002435 -118.99057335420346) bank15770 +15771 POINT(34.14223520689817 -117.80009748886717) bank15771 +15772 POINT(33.19566665041519 -117.29690492678999) bank15772 +15773 POINT(33.60028234677719 -117.6358254288072) bank15773 +15774 POINT(33.778756583433804 -118.88152359020823) bank15774 +15775 POINT(33.25877885530298 -117.92217576795134) bank15775 +15776 POINT(33.21284742929166 -118.41365391169667) bank15776 +15777 POINT(33.671591728213095 -118.35616892857927) bank15777 +15778 POINT(34.724952555517426 -117.63941142931596) bank15778 +15779 POINT(33.80386044178377 -118.67838050565767) bank15779 +15780 POINT(34.81471110751531 -118.57326863817494) bank15780 +15781 POINT(34.50469858436205 -118.9565087813171) bank15781 +15782 POINT(34.38122114627433 -118.02446447477821) bank15782 +15783 POINT(35.00284896382422 -118.16070551735363) bank15783 +15784 POINT(34.28666596144181 -118.26507175723322) bank15784 +15785 POINT(34.660959137013364 -117.50219073006704) bank15785 +15786 POINT(33.61152653949219 -119.12689479202703) bank15786 +15787 POINT(33.32711744085009 -119.20272542982016) bank15787 +15788 POINT(33.40835532220354 -119.11876709512705) bank15788 +15789 POINT(33.24896728460561 -117.54411620726225) bank15789 +15790 POINT(33.82860887158092 -118.08515799338628) bank15790 +15791 POINT(33.31986306776981 -118.65704269800169) bank15791 +15792 POINT(34.134400851953615 -118.41643175711482) bank15792 +15793 POINT(34.258231282541644 -117.81597988642632) bank15793 +15794 POINT(34.42348896224425 -118.46429456004505) bank15794 +15795 POINT(33.22320847106406 -118.91655733467195) bank15795 +15796 POINT(33.43003211452609 -117.48209239148015) bank15796 +15797 POINT(33.64164252647113 -117.54231702399194) bank15797 +15798 POINT(33.28691731625924 -118.97698809433707) bank15798 +15799 POINT(34.77631708417932 -119.17844167765796) bank15799 +15800 POINT(34.062387768754675 -118.11146664598203) bank15800 +15801 POINT(34.87104330507938 -118.90174835197278) bank15801 +15802 POINT(33.300445558011184 -117.5690775616442) bank15802 +15803 POINT(33.968906465356554 -118.83171829832713) bank15803 +15804 POINT(34.14583656453516 -118.3942305402355) bank15804 +15805 POINT(33.40506748548507 -118.69407649715049) bank15805 +15806 POINT(33.13772659349192 -117.68275539558377) bank15806 +15807 POINT(33.734162728127664 -118.03000750987506) bank15807 +15808 POINT(34.209997769875905 -117.57712991889777) bank15808 +15809 POINT(33.86812324830982 -119.09635139788499) bank15809 +15810 POINT(33.46874222357451 -119.22534587381858) bank15810 +15811 POINT(34.24156758324906 -117.71399913162851) bank15811 +15812 POINT(33.31207916178466 -117.43382303885245) bank15812 +15813 POINT(33.46544583218006 -117.56888363366605) bank15813 +15814 POINT(33.26728444188435 -118.40504247122875) bank15814 +15815 POINT(34.47925021165048 -118.38532432278097) bank15815 +15816 POINT(33.65558103838818 -118.0553583892542) bank15816 +15817 POINT(33.41782803738457 -118.37433991439633) bank15817 +15818 POINT(33.24678935221447 -117.93758282803893) bank15818 +15819 POINT(33.12998845754855 -119.0976022565985) bank15819 +15820 POINT(34.572299626075385 -118.04960328611287) bank15820 +15821 POINT(34.408155563845824 -117.39363408459174) bank15821 +15822 POINT(33.05535900616992 -117.58351862279882) bank15822 +15823 POINT(34.84975168543693 -118.95393249460294) bank15823 +15824 POINT(33.720771319891014 -118.33571395834686) bank15824 +15825 POINT(33.6316394598975 -119.19352821506865) bank15825 +15826 POINT(33.149073717567575 -117.86080776533763) bank15826 +15827 POINT(34.49072511701132 -118.14928756808908) bank15827 +15828 POINT(33.988111990091795 -117.9294802230828) bank15828 +15829 POINT(34.99662914771107 -119.12333224188629) bank15829 +15830 POINT(34.78424863143436 -117.39712909814946) bank15830 +15831 POINT(34.42174939236587 -117.5737564915087) bank15831 +15832 POINT(34.12705252129839 -117.52424028409979) bank15832 +15833 POINT(35.00165589329957 -118.8271206417181) bank15833 +15834 POINT(33.265694692573334 -119.02066439886191) bank15834 +15835 POINT(34.38051298436749 -117.35979494109743) bank15835 +15836 POINT(34.11877804642309 -119.22618562565884) bank15836 +15837 POINT(34.08300213867778 -118.15145106460454) bank15837 +15838 POINT(33.268641763846446 -117.8908532062298) bank15838 +15839 POINT(33.83955467404075 -118.53720791393239) bank15839 +15840 POINT(33.131216483582946 -118.88689312146612) bank15840 +15841 POINT(33.3906290454575 -119.20094081370434) bank15841 +15842 POINT(34.794144400412364 -118.252546441943) bank15842 +15843 POINT(33.55085000704465 -117.66464754831733) bank15843 +15844 POINT(33.763295409725735 -117.24999606774513) bank15844 +15845 POINT(33.40494574678954 -117.89122840158034) bank15845 +15846 POINT(34.93007981748134 -117.46042741576072) bank15846 +15847 POINT(33.198194218604684 -117.4894511606869) bank15847 +15848 POINT(33.44688234365312 -118.28143071808496) bank15848 +15849 POINT(34.30331190621467 -118.12054059500686) bank15849 +15850 POINT(33.19292935441879 -118.05865873244781) bank15850 +15851 POINT(34.0919515041829 -117.8222180333445) bank15851 +15852 POINT(33.4124220442513 -117.63817301469606) bank15852 +15853 POINT(34.163975935505725 -117.9837223549859) bank15853 +15854 POINT(33.073155208903614 -117.99411568200574) bank15854 +15855 POINT(34.998675180693695 -118.5213816868467) bank15855 +15856 POINT(34.28890313484616 -118.50385820297933) bank15856 +15857 POINT(33.86915713744912 -117.43502893601021) bank15857 +15858 POINT(34.21593629412573 -117.30095363488087) bank15858 +15859 POINT(33.40302356158169 -118.38921588904313) bank15859 +15860 POINT(34.05226040461222 -117.4282068162766) bank15860 +15861 POINT(34.69803702704585 -118.27707814277976) bank15861 +15862 POINT(34.51036542777547 -117.55014400762072) bank15862 +15863 POINT(34.70686852457912 -118.45108002148442) bank15863 +15864 POINT(34.55010961459168 -119.01454547809945) bank15864 +15865 POINT(33.35092513929408 -119.24305661874106) bank15865 +15866 POINT(34.37664282920995 -117.96907662716922) bank15866 +15867 POINT(33.65069651763007 -118.12446347373749) bank15867 +15868 POINT(34.72326333848824 -117.86129610578361) bank15868 +15869 POINT(34.53792242515344 -118.24190167127102) bank15869 +15870 POINT(33.31630399374264 -118.83451838584618) bank15870 +15871 POINT(35.02985977945313 -117.42091893791797) bank15871 +15872 POINT(34.693722529257485 -118.86046687754911) bank15872 +15873 POINT(34.86996919127322 -118.09076420805222) bank15873 +15874 POINT(33.6936343794903 -117.86484385842364) bank15874 +15875 POINT(33.3401010043476 -118.40943661326592) bank15875 +15876 POINT(34.82944681443378 -118.36921173819043) bank15876 +15877 POINT(33.19123657424063 -119.09166994841001) bank15877 +15878 POINT(34.43205877206359 -117.41645405366977) bank15878 +15879 POINT(33.494532533011366 -118.88596694373734) bank15879 +15880 POINT(34.65611028885969 -118.25081104381601) bank15880 +15881 POINT(34.37555238058526 -119.16995215263381) bank15881 +15882 POINT(33.599837356451104 -117.85311663231656) bank15882 +15883 POINT(33.20469833622012 -118.21138571870326) bank15883 +15884 POINT(33.948035459993804 -118.67110096499907) bank15884 +15885 POINT(33.18644479028468 -118.88642335727071) bank15885 +15886 POINT(33.812000747073476 -117.42875013867067) bank15886 +15887 POINT(33.849532484925845 -119.11435288679903) bank15887 +15888 POINT(33.36141296718536 -118.50885199492984) bank15888 +15889 POINT(34.58580668848329 -118.39908877495715) bank15889 +15890 POINT(34.409709144746465 -117.2752575840209) bank15890 +15891 POINT(33.37629803572901 -118.99983156356751) bank15891 +15892 POINT(34.13971508636783 -117.96304641669913) bank15892 +15893 POINT(34.58105126042411 -118.62085160507436) bank15893 +15894 POINT(33.47948845488875 -117.39250201175471) bank15894 +15895 POINT(34.360874520180055 -119.1206159187515) bank15895 +15896 POINT(33.75866617647498 -118.33851144823797) bank15896 +15897 POINT(34.61767088026444 -117.67826775267628) bank15897 +15898 POINT(33.37626820889843 -117.56360361881362) bank15898 +15899 POINT(34.669589302910815 -118.6281196400853) bank15899 +15900 POINT(33.415319348442466 -118.25782120398155) bank15900 +15901 POINT(34.55000160968642 -118.78454665529887) bank15901 +15902 POINT(33.093619320748715 -118.00496555793863) bank15902 +15903 POINT(34.991752739416455 -117.40073145248583) bank15903 +15904 POINT(34.319683980047955 -118.89602359863157) bank15904 +15905 POINT(34.93167907806872 -118.58208633155557) bank15905 +15906 POINT(34.43845220406338 -118.6841622824445) bank15906 +15907 POINT(33.58129146281636 -117.67745810469101) bank15907 +15908 POINT(34.86426853015442 -117.91655872282251) bank15908 +15909 POINT(34.863962335439474 -117.8594726423472) bank15909 +15910 POINT(33.78601462264505 -118.12365593060075) bank15910 +15911 POINT(34.22979054458148 -118.88578294300886) bank15911 +15912 POINT(34.201844281213425 -118.25341966957745) bank15912 +15913 POINT(33.16003997995619 -117.30846246251494) bank15913 +15914 POINT(34.17928690747817 -118.84388228804946) bank15914 +15915 POINT(34.1501304359504 -118.41292074372782) bank15915 +15916 POINT(33.067950969343144 -118.84248791882123) bank15916 +15917 POINT(33.67298250037481 -118.12563874806632) bank15917 +15918 POINT(34.598342456962946 -117.5352229025106) bank15918 +15919 POINT(33.41163624454431 -118.78835604943544) bank15919 +15920 POINT(34.242418035551374 -117.46656137041634) bank15920 +15921 POINT(33.83086138745705 -119.16143461259487) bank15921 +15922 POINT(34.41528561235269 -117.97522813975593) bank15922 +15923 POINT(34.00666153734053 -117.58092892910636) bank15923 +15924 POINT(33.55259252931038 -118.08555024586391) bank15924 +15925 POINT(33.99222753418466 -118.36629896189976) bank15925 +15926 POINT(35.032229670715616 -118.78127284049086) bank15926 +15927 POINT(34.467762396945986 -117.42052766705415) bank15927 +15928 POINT(33.50376124572379 -117.91022463717232) bank15928 +15929 POINT(34.20767339436338 -119.21357812083357) bank15929 +15930 POINT(34.53717680497927 -118.83503893680509) bank15930 +15931 POINT(33.62643573753851 -117.7454054411847) bank15931 +15932 POINT(34.13383148142126 -118.51206500216601) bank15932 +15933 POINT(35.01019830706061 -118.10185806651513) bank15933 +15934 POINT(34.896717613687464 -117.8722583558954) bank15934 +15935 POINT(34.104066398158075 -117.76385286607966) bank15935 +15936 POINT(33.141586559539356 -117.71701123092016) bank15936 +15937 POINT(34.357573311502954 -117.40932343938252) bank15937 +15938 POINT(33.98703933713201 -118.08189273278163) bank15938 +15939 POINT(34.43612528392193 -118.06845910866706) bank15939 +15940 POINT(34.47197907925651 -118.00549010330194) bank15940 +15941 POINT(33.719980261010356 -118.13940982282723) bank15941 +15942 POINT(33.28198339258147 -118.18071737528295) bank15942 +15943 POINT(33.36451597476309 -118.98022954135791) bank15943 +15944 POINT(34.8890389089691 -117.43291699175047) bank15944 +15945 POINT(34.22925056954498 -118.55872843808397) bank15945 +15946 POINT(33.89086086297138 -117.59269117075758) bank15946 +15947 POINT(34.578769470483486 -118.58689577150612) bank15947 +15948 POINT(34.50496346129952 -118.22045930248225) bank15948 +15949 POINT(34.673320865905154 -117.26612339471045) bank15949 +15950 POINT(33.980722629496526 -117.64793269727501) bank15950 +15951 POINT(34.07381088630206 -119.21611446125385) bank15951 +15952 POINT(34.42406053678159 -117.43381908293739) bank15952 +15953 POINT(34.198236136323956 -119.17320676686343) bank15953 +15954 POINT(34.06034991407099 -118.08869855796395) bank15954 +15955 POINT(33.91150483896698 -118.25002352505359) bank15955 +15956 POINT(34.300087363326135 -117.68273998310632) bank15956 +15957 POINT(35.022381620603575 -117.51117329855293) bank15957 +15958 POINT(34.004360636055715 -119.04247201340885) bank15958 +15959 POINT(34.70110891427571 -117.38192366522787) bank15959 +15960 POINT(34.36797044243353 -118.94273917107117) bank15960 +15961 POINT(33.295266745479566 -118.88143994408763) bank15961 +15962 POINT(34.89845692867142 -117.44170210825902) bank15962 +15963 POINT(33.57410496681966 -119.00161995006604) bank15963 +15964 POINT(34.21768428321273 -119.14206771423343) bank15964 +15965 POINT(34.24565480171601 -118.21611216924966) bank15965 +15966 POINT(33.711077519662545 -118.68476745167251) bank15966 +15967 POINT(34.02745474808819 -119.01699130321445) bank15967 +15968 POINT(34.49198930827446 -117.55865022622402) bank15968 +15969 POINT(34.71746955905865 -118.71266454484544) bank15969 +15970 POINT(33.948381815592384 -117.94330544373794) bank15970 +15971 POINT(34.80203096069316 -118.12680227360038) bank15971 +15972 POINT(34.21687564668488 -119.02554528822085) bank15972 +15973 POINT(34.901697664501356 -118.19057974047037) bank15973 +15974 POINT(34.85648870546082 -118.22780309993654) bank15974 +15975 POINT(33.23951003670354 -117.9224918308849) bank15975 +15976 POINT(33.95439503282669 -118.82883151111604) bank15976 +15977 POINT(33.48924499286896 -118.49861265977) bank15977 +15978 POINT(33.36795351254279 -117.52984844192206) bank15978 +15979 POINT(33.17425382978471 -117.28711981354893) bank15979 +15980 POINT(33.922309190046505 -119.10928338656383) bank15980 +15981 POINT(33.62890979751521 -119.01531383740172) bank15981 +15982 POINT(33.3401894011724 -118.5929837773745) bank15982 +15983 POINT(33.98299802064823 -117.73777764373601) bank15983 +15984 POINT(33.41134373323375 -118.14506496562473) bank15984 +15985 POINT(33.78922806556191 -119.02597575597467) bank15985 +15986 POINT(33.33740968531419 -117.33458408659489) bank15986 +15987 POINT(35.0317543691345 -118.6590585742624) bank15987 +15988 POINT(33.88788836169244 -117.67024502715633) bank15988 +15989 POINT(34.13202135588114 -117.65612540299483) bank15989 +15990 POINT(35.02962890794591 -118.78351724293728) bank15990 +15991 POINT(33.133460954678334 -119.00629837475569) bank15991 +15992 POINT(33.85653166287501 -118.955989859021) bank15992 +15993 POINT(34.50679368586439 -118.84426048658275) bank15993 +15994 POINT(33.101503603126886 -118.10114499038497) bank15994 +15995 POINT(34.31575036260124 -119.02410487464573) bank15995 +15996 POINT(34.07784229651977 -117.69142443844808) bank15996 +15997 POINT(34.62493271041679 -117.85266561314953) bank15997 +15998 POINT(34.36296106450272 -118.37543049811471) bank15998 +15999 POINT(33.986738775135265 -119.18610392288333) bank15999 +16000 POINT(33.86526526030561 -117.85731172841753) bank16000 +16001 POINT(34.97532457064604 -118.27403871320713) bank16001 +16002 POINT(33.43839664169926 -118.82929558765657) bank16002 +16003 POINT(33.489678581528565 -119.11684976382833) bank16003 +16004 POINT(33.26931458252669 -117.41148405268558) bank16004 +16005 POINT(34.98871189793292 -117.41626346499878) bank16005 +16006 POINT(33.07851611328871 -117.51285834217936) bank16006 +16007 POINT(33.186091034915265 -118.96564288664315) bank16007 +16008 POINT(35.02272954821438 -119.20513982417717) bank16008 +16009 POINT(34.559660060155466 -117.47161275271456) bank16009 +16010 POINT(33.05300811800359 -117.45819448433934) bank16010 +16011 POINT(33.14791250588341 -117.46952027468691) bank16011 +16012 POINT(33.0961115534473 -117.82633033918056) bank16012 +16013 POINT(33.25977967100836 -119.17040699112097) bank16013 +16014 POINT(33.79603023753941 -117.9085948361432) bank16014 +16015 POINT(33.914349076396796 -119.19024584943679) bank16015 +16016 POINT(33.20581666576684 -117.72679476643978) bank16016 +16017 POINT(34.8733652088424 -118.84819887562466) bank16017 +16018 POINT(33.989228038006665 -118.6764053537016) bank16018 +16019 POINT(34.33480107878032 -118.23344740542092) bank16019 +16020 POINT(34.4139601637188 -117.81421327039396) bank16020 +16021 POINT(33.75601088301798 -117.73604686697736) bank16021 +16022 POINT(33.07828742315692 -118.39276626162577) bank16022 +16023 POINT(33.580676869589446 -117.9163615814893) bank16023 +16024 POINT(33.88317357032346 -117.93243948144737) bank16024 +16025 POINT(33.7292535325253 -117.76040271594881) bank16025 +16026 POINT(33.694746388177755 -117.58155842193848) bank16026 +16027 POINT(33.66617286699718 -117.64995459059897) bank16027 +16028 POINT(33.05479982090399 -118.09746026115299) bank16028 +16029 POINT(33.79153379965347 -117.86166441507737) bank16029 +16030 POINT(33.09513605083171 -119.06127393038209) bank16030 +16031 POINT(34.291749725047794 -117.79064937059088) bank16031 +16032 POINT(34.66262015790458 -119.10963125121812) bank16032 +16033 POINT(33.75246304466926 -119.17598626839789) bank16033 +16034 POINT(33.568627419236435 -118.35561676379638) bank16034 +16035 POINT(33.46710594601446 -117.78159577384814) bank16035 +16036 POINT(33.418763441663664 -118.25417170727872) bank16036 +16037 POINT(33.61892364424537 -117.2744914672837) bank16037 +16038 POINT(34.19406618135468 -117.57433569051632) bank16038 +16039 POINT(34.7810699755786 -117.92989831978477) bank16039 +16040 POINT(33.269793714840105 -117.72382901712291) bank16040 +16041 POINT(34.318794876351674 -117.71852732073144) bank16041 +16042 POINT(33.84184520231269 -118.21146321463411) bank16042 +16043 POINT(34.01942883906028 -118.51793611713323) bank16043 +16044 POINT(34.91031301607946 -117.58513764759317) bank16044 +16045 POINT(34.85347434269031 -117.27320359889961) bank16045 +16046 POINT(33.80791162889033 -118.10400448413252) bank16046 +16047 POINT(33.17021529460184 -118.70697960759742) bank16047 +16048 POINT(33.527791785582295 -118.25842022452873) bank16048 +16049 POINT(34.23235525863487 -119.07267937854826) bank16049 +16050 POINT(33.45150579358945 -117.89841804355983) bank16050 +16051 POINT(34.74789691223882 -117.42301145185918) bank16051 +16052 POINT(33.47857036897722 -117.41509828785576) bank16052 +16053 POINT(33.35143305171959 -117.56238248275817) bank16053 +16054 POINT(33.41259972380237 -118.94449475521995) bank16054 +16055 POINT(33.73329512028888 -117.82534104752193) bank16055 +16056 POINT(33.127229272224675 -118.2612623789617) bank16056 +16057 POINT(33.87241683112715 -118.04005509528274) bank16057 +16058 POINT(34.606693102170816 -118.53464392351115) bank16058 +16059 POINT(33.60167461608005 -118.42940030758865) bank16059 +16060 POINT(33.827950051519736 -119.23208974704819) bank16060 +16061 POINT(34.24419793383044 -117.81096451251375) bank16061 +16062 POINT(34.06504469253855 -118.83947781411194) bank16062 +16063 POINT(33.88260248294721 -117.62492958684221) bank16063 +16064 POINT(34.18838901285373 -118.84556548871204) bank16064 +16065 POINT(34.89392628280304 -119.04935790419125) bank16065 +16066 POINT(34.664234714595565 -119.16778074731661) bank16066 +16067 POINT(33.147630351885084 -117.95847068507784) bank16067 +16068 POINT(33.73242660106936 -117.40609003275337) bank16068 +16069 POINT(34.26991414124697 -118.21031643420702) bank16069 +16070 POINT(34.85906039871082 -119.10082725028407) bank16070 +16071 POINT(34.9807778113209 -117.37217292255468) bank16071 +16072 POINT(33.89313278879692 -118.2028102609992) bank16072 +16073 POINT(33.919671146529495 -117.82345122303524) bank16073 +16074 POINT(34.170740455314686 -118.38848662105711) bank16074 +16075 POINT(34.80869941537042 -117.25645775792825) bank16075 +16076 POINT(34.15415151422527 -118.74732747650258) bank16076 +16077 POINT(33.99875047868908 -117.67998925074835) bank16077 +16078 POINT(34.31550744226079 -117.39752788408974) bank16078 +16079 POINT(33.607715607470865 -117.97921138767414) bank16079 +16080 POINT(34.81789869878159 -118.61270762667617) bank16080 +16081 POINT(33.7082075419405 -119.11703772987562) bank16081 +16082 POINT(33.44933425700213 -119.12279760466026) bank16082 +16083 POINT(34.25497802244817 -117.25493092560653) bank16083 +16084 POINT(33.492578728119305 -119.03290033379834) bank16084 +16085 POINT(34.43601763479894 -118.29605267035433) bank16085 +16086 POINT(34.969588233764135 -117.8043297254309) bank16086 +16087 POINT(33.22086274405821 -118.72832676966553) bank16087 +16088 POINT(33.475654129351646 -118.61097080255813) bank16088 +16089 POINT(33.70427761251961 -117.7612031324196) bank16089 +16090 POINT(33.565686743208644 -117.49780026469178) bank16090 +16091 POINT(33.26388723171885 -117.73672665707592) bank16091 +16092 POINT(34.65238265002869 -118.88676916374483) bank16092 +16093 POINT(33.283896778059834 -118.70680754570283) bank16093 +16094 POINT(33.53770134630404 -118.37355112155315) bank16094 +16095 POINT(33.46154422214455 -117.98619094066511) bank16095 +16096 POINT(34.15872708381906 -117.49967676610514) bank16096 +16097 POINT(34.88307889086993 -117.26746273726016) bank16097 +16098 POINT(33.74298767724091 -118.66153482903555) bank16098 +16099 POINT(34.09636544660971 -118.01968748619964) bank16099 +16100 POINT(34.46888842539237 -118.92687877677822) bank16100 +16101 POINT(33.110150292560135 -118.32289905415887) bank16101 +16102 POINT(33.83321477044963 -118.74001476665822) bank16102 +16103 POINT(34.27846109358467 -118.44834388891526) bank16103 +16104 POINT(34.99812414547652 -118.58847130481088) bank16104 +16105 POINT(34.399656243437526 -118.5485760918789) bank16105 +16106 POINT(34.711545604554665 -118.88848418806923) bank16106 +16107 POINT(33.71492480958951 -118.53648572277794) bank16107 +16108 POINT(35.00523807672886 -119.00529381924406) bank16108 +16109 POINT(34.41229063569679 -118.73854970683442) bank16109 +16110 POINT(34.16523714413498 -118.85520012564038) bank16110 +16111 POINT(34.55256558532486 -117.37949059373061) bank16111 +16112 POINT(33.72168677941465 -118.36345707973899) bank16112 +16113 POINT(33.5000385993402 -119.16688238092047) bank16113 +16114 POINT(33.328089553813115 -117.63236088864308) bank16114 +16115 POINT(34.92806302734234 -119.03322076217852) bank16115 +16116 POINT(33.3735860340731 -117.31940617947832) bank16116 +16117 POINT(34.69566540369638 -118.40383827290832) bank16117 +16118 POINT(33.096059989646115 -117.74203759593756) bank16118 +16119 POINT(34.84781464017527 -118.29902250362024) bank16119 +16120 POINT(34.996588797912786 -117.74310618148938) bank16120 +16121 POINT(33.47981212840388 -117.98518179088171) bank16121 +16122 POINT(34.063068821197895 -117.48906149277363) bank16122 +16123 POINT(34.38328636650093 -118.10595298871718) bank16123 +16124 POINT(34.185558235694224 -118.27397765580545) bank16124 +16125 POINT(33.05679810614741 -117.72812913910107) bank16125 +16126 POINT(34.277046013999446 -118.96847430560939) bank16126 +16127 POINT(34.14873524514148 -118.53125384224532) bank16127 +16128 POINT(33.78579668269513 -117.86793212790691) bank16128 +16129 POINT(33.293069339747774 -118.36446900556865) bank16129 +16130 POINT(33.434592415030856 -117.84309330141949) bank16130 +16131 POINT(34.31291799909791 -118.07290035616205) bank16131 +16132 POINT(35.04886264602475 -119.01103954096061) bank16132 +16133 POINT(34.548364572250485 -117.6254201411801) bank16133 +16134 POINT(34.817758665200344 -119.04130978051431) bank16134 +16135 POINT(34.55330871841116 -118.80660748774922) bank16135 +16136 POINT(33.847573496754876 -117.79952086920541) bank16136 +16137 POINT(34.147161343699295 -118.93047292994495) bank16137 +16138 POINT(33.91155800050325 -118.29602811166015) bank16138 +16139 POINT(33.93704890750548 -118.32084499787094) bank16139 +16140 POINT(34.22282383556905 -119.11816091513012) bank16140 +16141 POINT(34.01285304311861 -117.91552342573978) bank16141 +16142 POINT(33.25319168641912 -117.66850996574753) bank16142 +16143 POINT(33.731242452241695 -119.0277539150515) bank16143 +16144 POINT(33.19074881050641 -117.6549495842631) bank16144 +16145 POINT(33.33136527997885 -117.82317953886053) bank16145 +16146 POINT(34.36621947615905 -118.81372320281658) bank16146 +16147 POINT(34.17379800000809 -117.32617325213835) bank16147 +16148 POINT(34.04883056242968 -117.92783357059996) bank16148 +16149 POINT(33.25178295803776 -118.7689509216697) bank16149 +16150 POINT(33.18114667502191 -117.49393414920587) bank16150 +16151 POINT(34.75542511569383 -117.45253413969313) bank16151 +16152 POINT(33.334327223821056 -117.88765296852446) bank16152 +16153 POINT(33.7072838391576 -117.43580155654789) bank16153 +16154 POINT(33.44350778836505 -117.85961378094972) bank16154 +16155 POINT(34.7635209826317 -117.58952578043161) bank16155 +16156 POINT(33.112724297102666 -118.17863917310636) bank16156 +16157 POINT(34.43404618334003 -117.71933378343091) bank16157 +16158 POINT(33.272924516347004 -117.86682156270976) bank16158 +16159 POINT(34.20239772765943 -117.57271845676648) bank16159 +16160 POINT(33.47296873963619 -118.31444968114222) bank16160 +16161 POINT(34.18081640383833 -118.5337192514294) bank16161 +16162 POINT(34.54321600971552 -118.6474998193154) bank16162 +16163 POINT(33.05976893695916 -119.10588648277377) bank16163 +16164 POINT(33.829038070660566 -118.58950662585305) bank16164 +16165 POINT(33.142693431415175 -117.63324919700578) bank16165 +16166 POINT(34.54832473314079 -117.92000996936845) bank16166 +16167 POINT(33.59567958829576 -118.98216276374521) bank16167 +16168 POINT(33.22710694445702 -118.65849893960163) bank16168 +16169 POINT(34.10493120008956 -119.03905561728162) bank16169 +16170 POINT(33.09853277795004 -118.15376316270628) bank16170 +16171 POINT(34.17782423295974 -117.53665910276823) bank16171 +16172 POINT(34.34663251818713 -118.72267997671577) bank16172 +16173 POINT(34.14415244180594 -117.8531079634292) bank16173 +16174 POINT(33.05401463345301 -118.27541307388859) bank16174 +16175 POINT(33.309592847289714 -119.19585663047245) bank16175 +16176 POINT(34.47114071447852 -117.8176457394) bank16176 +16177 POINT(34.59944394598508 -118.81609432335448) bank16177 +16178 POINT(33.803656226475596 -117.5143961907497) bank16178 +16179 POINT(34.67363317814284 -118.55071914090223) bank16179 +16180 POINT(33.673408843451924 -118.36612550289888) bank16180 +16181 POINT(34.99537112544613 -117.35849872671845) bank16181 +16182 POINT(34.542743196464855 -117.41816193556164) bank16182 +16183 POINT(34.82335095066751 -117.41977883052299) bank16183 +16184 POINT(34.63132831646125 -117.5136709379468) bank16184 +16185 POINT(33.1353017930155 -117.68326228259856) bank16185 +16186 POINT(33.06708300411797 -117.57649875167964) bank16186 +16187 POINT(33.11681344988929 -117.81784459252664) bank16187 +16188 POINT(33.819584914410164 -118.44032750002646) bank16188 +16189 POINT(33.06137529607427 -118.67364617282875) bank16189 +16190 POINT(34.28931475060675 -117.49066439948135) bank16190 +16191 POINT(34.44200602470693 -118.3141853896312) bank16191 +16192 POINT(34.24914455044042 -117.8350343330112) bank16192 +16193 POINT(33.098962312139946 -117.38672698639114) bank16193 +16194 POINT(34.76217938448802 -118.96475908192546) bank16194 +16195 POINT(33.11773691635199 -117.87893035103335) bank16195 +16196 POINT(34.73625914263914 -118.70833114072695) bank16196 +16197 POINT(33.168214439285286 -118.19888595160698) bank16197 +16198 POINT(33.50283713799338 -119.02395092069052) bank16198 +16199 POINT(33.197243381936666 -119.05726692811761) bank16199 +16200 POINT(33.91613411378274 -118.67992091225817) bank16200 +16201 POINT(34.19579101525991 -117.49255692891035) bank16201 +16202 POINT(33.72137178459233 -118.04496285032812) bank16202 +16203 POINT(33.2386638436629 -117.53053078734743) bank16203 +16204 POINT(34.912762081670316 -117.30717142651746) bank16204 +16205 POINT(33.1498308171173 -117.2570097175252) bank16205 +16206 POINT(33.286336062536 -117.86458918215367) bank16206 +16207 POINT(33.10563545326439 -117.57581387159104) bank16207 +16208 POINT(35.00040875548929 -117.38439749597285) bank16208 +16209 POINT(33.21598496746058 -117.29580318842103) bank16209 +16210 POINT(34.93716966607869 -118.54571307085448) bank16210 +16211 POINT(34.89390101516493 -119.14930599336613) bank16211 +16212 POINT(33.74346870612444 -118.97060896545094) bank16212 +16213 POINT(34.494172329348814 -119.14663281331485) bank16213 +16214 POINT(33.96807514669185 -118.35016344777863) bank16214 +16215 POINT(34.90166163350701 -119.11065109137459) bank16215 +16216 POINT(34.643602449860204 -119.16932732514992) bank16216 +16217 POINT(34.57901452595107 -117.33101931719663) bank16217 +16218 POINT(33.96326409157219 -118.33421461297416) bank16218 +16219 POINT(35.04328980961597 -118.6399357428271) bank16219 +16220 POINT(34.69246398937494 -117.72024037626743) bank16220 +16221 POINT(33.3072335331318 -117.94255182982941) bank16221 +16222 POINT(34.727739048949864 -118.57739162484233) bank16222 +16223 POINT(33.2545545379648 -117.54378104182868) bank16223 +16224 POINT(33.323894978691534 -118.57651728686199) bank16224 +16225 POINT(34.35469946786284 -117.51483850165475) bank16225 +16226 POINT(34.717122025683985 -117.86252702715119) bank16226 +16227 POINT(33.82434101562164 -117.96846463416341) bank16227 +16228 POINT(33.24537745301495 -118.90250361808414) bank16228 +16229 POINT(33.67423488629826 -117.83772578361076) bank16229 +16230 POINT(34.21074055715608 -118.80038058374623) bank16230 +16231 POINT(33.372029448288224 -119.20770803494581) bank16231 +16232 POINT(33.67855324162113 -117.63046039684757) bank16232 +16233 POINT(34.486738316159204 -118.51443233516716) bank16233 +16234 POINT(33.26100917339491 -118.91891394932208) bank16234 +16235 POINT(34.58035190184693 -117.42282021996816) bank16235 +16236 POINT(33.90316961599903 -118.11377506330747) bank16236 +16237 POINT(33.59592892352997 -117.31959666963986) bank16237 +16238 POINT(33.2394022492946 -117.28405688950994) bank16238 +16239 POINT(35.05117016145327 -117.28709090831306) bank16239 +16240 POINT(34.9353150644448 -118.04155424507809) bank16240 +16241 POINT(33.56218542080182 -117.27013843615234) bank16241 +16242 POINT(33.2215117710223 -118.50249803119557) bank16242 +16243 POINT(33.243720228934684 -118.00015630774784) bank16243 +16244 POINT(34.91330299857398 -117.61673806130275) bank16244 +16245 POINT(33.869706558374325 -118.21452543264989) bank16245 +16246 POINT(34.560210226858636 -118.66228618812218) bank16246 +16247 POINT(33.63689736503411 -118.81631926824454) bank16247 +16248 POINT(34.84105364068064 -119.18122462248103) bank16248 +16249 POINT(34.14492752509717 -117.91648017439967) bank16249 +16250 POINT(34.81792034950934 -117.39339931283546) bank16250 +16251 POINT(33.12823754839226 -117.51277863325738) bank16251 +16252 POINT(33.6598424815995 -117.46429859996822) bank16252 +16253 POINT(34.09268123058108 -119.14668345483987) bank16253 +16254 POINT(34.92238339753881 -117.74541020653979) bank16254 +16255 POINT(33.38217972498594 -117.70379513373315) bank16255 +16256 POINT(34.456069034515416 -119.1697802431678) bank16256 +16257 POINT(33.79655734233071 -118.04295188268775) bank16257 +16258 POINT(34.04708265462324 -117.85310099228249) bank16258 +16259 POINT(33.50150734934458 -117.89370701272429) bank16259 +16260 POINT(33.5660118240019 -118.11838912287747) bank16260 +16261 POINT(34.48205703643822 -118.86133864397662) bank16261 +16262 POINT(33.24016535001351 -118.76505760113227) bank16262 +16263 POINT(34.473985395959396 -117.82778444709389) bank16263 +16264 POINT(33.36975537818952 -119.02473012100073) bank16264 +16265 POINT(34.70557698020123 -118.49292205951801) bank16265 +16266 POINT(34.0080574758771 -117.69170197866977) bank16266 +16267 POINT(34.61599762616609 -118.91646888907653) bank16267 +16268 POINT(34.699028629271616 -118.8699036484886) bank16268 +16269 POINT(34.99768004356172 -117.53703813281481) bank16269 +16270 POINT(33.935057074159545 -118.70553375053123) bank16270 +16271 POINT(33.586584085088866 -117.25367031725095) bank16271 +16272 POINT(33.76502980257532 -118.05181111938195) bank16272 +16273 POINT(33.56182813566716 -118.67076731582266) bank16273 +16274 POINT(33.717560583458756 -118.7844617477406) bank16274 +16275 POINT(34.650155990109255 -119.17070750374494) bank16275 +16276 POINT(33.219589032597604 -117.74089311601998) bank16276 +16277 POINT(33.97704260837264 -118.51725528314026) bank16277 +16278 POINT(34.7102721447296 -117.43808179938374) bank16278 +16279 POINT(33.40612138521159 -119.01267523494137) bank16279 +16280 POINT(35.04285562294243 -117.81630947832761) bank16280 +16281 POINT(33.98932559214002 -118.05830027948207) bank16281 +16282 POINT(33.67577482895408 -118.13287779266359) bank16282 +16283 POINT(33.29227366464266 -117.79551614589182) bank16283 +16284 POINT(34.601930083597054 -119.23287167894209) bank16284 +16285 POINT(35.01428115147985 -118.30125220423697) bank16285 +16286 POINT(33.44924874199211 -119.01963841065144) bank16286 +16287 POINT(33.65657969854521 -118.75066683201185) bank16287 +16288 POINT(34.962216126355514 -118.99582956088813) bank16288 +16289 POINT(34.51654994269936 -118.94267913498027) bank16289 +16290 POINT(34.71465134428724 -119.06192226641419) bank16290 +16291 POINT(33.97029253230187 -118.07867669800538) bank16291 +16292 POINT(34.96933539010105 -117.67025195469903) bank16292 +16293 POINT(33.59451917174436 -118.22697478050662) bank16293 +16294 POINT(34.34205931309739 -118.25429749847483) bank16294 +16295 POINT(34.7451863926503 -119.21925055033405) bank16295 +16296 POINT(34.85448288838518 -119.16247494007347) bank16296 +16297 POINT(34.87405074744783 -119.17058863773791) bank16297 +16298 POINT(35.00839804771721 -118.65229628946742) bank16298 +16299 POINT(34.13921370020503 -117.43124762713688) bank16299 +16300 POINT(33.36938000770425 -118.38041186320555) bank16300 +16301 POINT(33.12346377647127 -118.88674414887514) bank16301 +16302 POINT(34.89567815955262 -118.60105013428304) bank16302 +16303 POINT(33.95980948165399 -118.91045778339792) bank16303 +16304 POINT(34.92869578753237 -118.33572274867137) bank16304 +16305 POINT(34.4860474916791 -117.57542330680474) bank16305 +16306 POINT(34.04325904706047 -119.09806682757517) bank16306 +16307 POINT(34.85328471938468 -118.08293009978334) bank16307 +16308 POINT(33.35686966195782 -118.1664005982335) bank16308 +16309 POINT(33.42168280170983 -117.33750356475802) bank16309 +16310 POINT(33.59979004390392 -118.02465120497062) bank16310 +16311 POINT(33.54271889440205 -118.67345304033077) bank16311 +16312 POINT(34.27561366877294 -118.2596085486251) bank16312 +16313 POINT(34.4597733841141 -117.49531161987525) bank16313 +16314 POINT(33.967427042061644 -118.3059833645782) bank16314 +16315 POINT(34.00606154940537 -118.67178873149251) bank16315 +16316 POINT(33.215090580544576 -117.68067126967047) bank16316 +16317 POINT(33.92293742685098 -118.57214493691666) bank16317 +16318 POINT(33.34640070372886 -117.43489905048642) bank16318 +16319 POINT(33.70574575786394 -118.92696149770686) bank16319 +16320 POINT(33.20570846224883 -118.32477099473007) bank16320 +16321 POINT(34.511152492244 -118.09712778302003) bank16321 +16322 POINT(34.89917109895862 -117.71755402217885) bank16322 +16323 POINT(33.709305504058264 -118.07623475968171) bank16323 +16324 POINT(33.458021292516875 -118.71892565498129) bank16324 +16325 POINT(34.15953883930678 -119.20656748717205) bank16325 +16326 POINT(33.16629077292105 -118.1700534978032) bank16326 +16327 POINT(34.5753216025419 -117.50594737930402) bank16327 +16328 POINT(33.425651519236766 -118.83181422832573) bank16328 +16329 POINT(34.92135332633795 -117.31041341713927) bank16329 +16330 POINT(34.90517888172474 -119.22136554113152) bank16330 +16331 POINT(34.01608934805003 -118.86845182437943) bank16331 +16332 POINT(33.57915001574941 -118.48466135184354) bank16332 +16333 POINT(34.02107234286092 -118.81226554728292) bank16333 +16334 POINT(33.4030993548228 -117.2948529865898) bank16334 +16335 POINT(34.48169015548819 -118.13657522895974) bank16335 +16336 POINT(33.80902138161032 -118.04217589901583) bank16336 +16337 POINT(34.990074000740655 -118.32659991821015) bank16337 +16338 POINT(33.97736203900873 -117.88586753133353) bank16338 +16339 POINT(34.25354441071035 -118.0596124375107) bank16339 +16340 POINT(33.282632086797676 -117.39201103527652) bank16340 +16341 POINT(33.886739566296036 -118.87698582641478) bank16341 +16342 POINT(34.084107897282635 -118.71318441242673) bank16342 +16343 POINT(34.194101308677524 -117.40891262359835) bank16343 +16344 POINT(33.82990184829555 -118.13185965051333) bank16344 +16345 POINT(34.544394513627736 -117.49412807162885) bank16345 +16346 POINT(34.89724091931687 -118.39023235423477) bank16346 +16347 POINT(34.63532299375631 -118.85529798967386) bank16347 +16348 POINT(34.25476096148735 -118.2392924405556) bank16348 +16349 POINT(33.99001766818198 -118.84053395245681) bank16349 +16350 POINT(33.351050094978625 -118.83874347316545) bank16350 +16351 POINT(34.752812906908666 -118.27373910261704) bank16351 +16352 POINT(33.17703785625031 -117.61194963695955) bank16352 +16353 POINT(33.407577117237686 -117.70772138947696) bank16353 +16354 POINT(33.17219589868755 -118.09268279658706) bank16354 +16355 POINT(34.09621596169133 -117.89378403758533) bank16355 +16356 POINT(33.159363476287 -118.40824337624946) bank16356 +16357 POINT(34.46041642680076 -118.13780952752943) bank16357 +16358 POINT(34.881908263542975 -119.1071537824568) bank16358 +16359 POINT(34.753470909408804 -119.09240595279083) bank16359 +16360 POINT(33.86199830935546 -117.86152351688197) bank16360 +16361 POINT(33.45726560425193 -118.99871955682775) bank16361 +16362 POINT(34.63004402790897 -118.52530321886042) bank16362 +16363 POINT(33.34174735428259 -117.44600561329254) bank16363 +16364 POINT(33.32628291135748 -118.02295219031475) bank16364 +16365 POINT(34.39324506963006 -118.15906930547165) bank16365 +16366 POINT(34.85764988264028 -118.62395622751016) bank16366 +16367 POINT(34.512131627489566 -118.53050458150395) bank16367 +16368 POINT(34.606989794282015 -118.41254142308428) bank16368 +16369 POINT(33.77924203661634 -117.75695013543327) bank16369 +16370 POINT(34.539148862248055 -117.39851618719851) bank16370 +16371 POINT(34.61325039511669 -117.30400060518677) bank16371 +16372 POINT(33.573573579071066 -118.54415417585392) bank16372 +16373 POINT(34.045947216273824 -118.90800831019867) bank16373 +16374 POINT(35.03404999579438 -117.62537736249843) bank16374 +16375 POINT(33.2247629378691 -118.28368426583754) bank16375 +16376 POINT(34.752100185892544 -118.22604765098495) bank16376 +16377 POINT(34.7064870198933 -117.30326872618308) bank16377 +16378 POINT(33.89320457319913 -119.21213804481187) bank16378 +16379 POINT(33.550115957838536 -118.94954386867964) bank16379 +16380 POINT(35.02026994382315 -118.86923486387495) bank16380 +16381 POINT(34.57364441537022 -117.77361382706908) bank16381 +16382 POINT(33.42307989598963 -118.04390658960313) bank16382 +16383 POINT(33.73971583805781 -118.98705440541814) bank16383 +16384 POINT(34.334805251904996 -118.67518947152388) bank16384 +16385 POINT(33.71714874455067 -118.36055799350046) bank16385 +16386 POINT(33.69129406600361 -117.4738257946303) bank16386 +16387 POINT(33.441847162697464 -118.52151769673858) bank16387 +16388 POINT(34.88293060732884 -118.41198148024084) bank16388 +16389 POINT(33.522710227043596 -119.01892703658083) bank16389 +16390 POINT(33.74445523696093 -118.83215895778933) bank16390 +16391 POINT(33.44236964138941 -117.74504412781191) bank16391 +16392 POINT(33.89931477174924 -118.625075687903) bank16392 +16393 POINT(33.26569044348961 -117.71823729588169) bank16393 +16394 POINT(33.508619580087135 -118.15539771734161) bank16394 +16395 POINT(34.68335091237333 -118.96963684124935) bank16395 +16396 POINT(33.830346043609545 -118.25707215422884) bank16396 +16397 POINT(34.7478449638602 -118.82058119930267) bank16397 +16398 POINT(34.11172911382515 -118.40532421154526) bank16398 +16399 POINT(34.53596300626256 -118.26758432507762) bank16399 +16400 POINT(33.360696068393906 -118.57749841350973) bank16400 +16401 POINT(34.36795091886175 -117.64386793209752) bank16401 +16402 POINT(33.6775144104351 -117.39010179604637) bank16402 +16403 POINT(33.65066128615542 -118.58733064114777) bank16403 +16404 POINT(33.55505470101465 -117.681827698501) bank16404 +16405 POINT(33.708051119174115 -118.58984623215717) bank16405 +16406 POINT(33.83587668271323 -118.40269845288985) bank16406 +16407 POINT(34.78101521740358 -117.4005547284914) bank16407 +16408 POINT(34.08040184129952 -118.82878304237776) bank16408 +16409 POINT(33.43652789784776 -118.18634876617145) bank16409 +16410 POINT(34.16479426284556 -118.2832342751909) bank16410 +16411 POINT(33.513883782261814 -118.11652628886871) bank16411 +16412 POINT(34.46801324531591 -117.82228356881065) bank16412 +16413 POINT(33.05447804940238 -118.85409654367828) bank16413 +16414 POINT(33.59973244005287 -117.2996461974294) bank16414 +16415 POINT(33.18070144175698 -117.95004403976445) bank16415 +16416 POINT(33.200870201794864 -118.2379354848961) bank16416 +16417 POINT(33.45900853645404 -118.78233674043369) bank16417 +16418 POINT(34.17681445997148 -117.28094635885417) bank16418 +16419 POINT(34.76560836917339 -118.06211618491957) bank16419 +16420 POINT(33.590425271714516 -117.55383015122052) bank16420 +16421 POINT(33.99379941465243 -118.56891860477666) bank16421 +16422 POINT(34.49741286877058 -118.05582919035827) bank16422 +16423 POINT(33.05622875943946 -118.95763505923608) bank16423 +16424 POINT(33.149711727215404 -117.82347054555783) bank16424 +16425 POINT(34.286427169439705 -118.77286330924574) bank16425 +16426 POINT(34.56624082518586 -119.10051412669017) bank16426 +16427 POINT(34.82742581249127 -118.08377081457621) bank16427 +16428 POINT(33.75213007176524 -118.01664710841669) bank16428 +16429 POINT(33.87472848156852 -118.06546409118519) bank16429 +16430 POINT(33.88206947021061 -118.26221868003252) bank16430 +16431 POINT(33.52930424758598 -118.02197836543576) bank16431 +16432 POINT(34.509783708621455 -117.55630866247712) bank16432 +16433 POINT(34.792405136231366 -117.77509472454419) bank16433 +16434 POINT(33.78324393049216 -117.80932975163122) bank16434 +16435 POINT(33.964521270549824 -119.01667042739798) bank16435 +16436 POINT(34.479391395057135 -118.40240765270582) bank16436 +16437 POINT(34.03162228125723 -118.93743792412747) bank16437 +16438 POINT(33.171408077396705 -117.37865420073112) bank16438 +16439 POINT(35.0469420175219 -117.99387776102144) bank16439 +16440 POINT(33.16019638188616 -117.89165658546005) bank16440 +16441 POINT(34.77452993838013 -117.79751164539148) bank16441 +16442 POINT(34.426804470759606 -118.19055733423642) bank16442 +16443 POINT(33.493900466820776 -118.35091150002775) bank16443 +16444 POINT(34.653091934601676 -119.0537463645753) bank16444 +16445 POINT(35.01686591726842 -119.05703989602387) bank16445 +16446 POINT(33.16826098144691 -118.24095671528032) bank16446 +16447 POINT(33.16449129116824 -119.13630195800182) bank16447 +16448 POINT(34.13450288538593 -117.77864321412002) bank16448 +16449 POINT(34.298751140968754 -117.50465120487479) bank16449 +16450 POINT(34.37931106986673 -117.88155824665446) bank16450 +16451 POINT(33.6009300302066 -118.19085599332851) bank16451 +16452 POINT(33.54366952078373 -118.54655235199422) bank16452 +16453 POINT(34.405380691830594 -117.59591211531794) bank16453 +16454 POINT(34.337144157547904 -119.01557879354085) bank16454 +16455 POINT(33.50983999871508 -117.46185019656188) bank16455 +16456 POINT(33.375320114158306 -119.07216454356124) bank16456 +16457 POINT(34.11706982787421 -118.20869856061913) bank16457 +16458 POINT(33.911044802609396 -119.20011487901058) bank16458 +16459 POINT(33.44851560717528 -118.0107466197191) bank16459 +16460 POINT(33.705100022905086 -118.1941630331603) bank16460 +16461 POINT(34.59740088311606 -118.96932662946772) bank16461 +16462 POINT(33.90670160906474 -118.22856264037539) bank16462 +16463 POINT(33.85202958770919 -117.29009659851863) bank16463 +16464 POINT(34.49058839125201 -118.6401794690501) bank16464 +16465 POINT(34.54510605541277 -118.61930562830997) bank16465 +16466 POINT(35.01674758602753 -117.65233997409966) bank16466 +16467 POINT(34.88239493748419 -117.82149066292273) bank16467 +16468 POINT(33.652794005842665 -117.64669610799642) bank16468 +16469 POINT(33.06762588188229 -118.0957733639609) bank16469 +16470 POINT(33.58408385630102 -117.67777077640467) bank16470 +16471 POINT(33.63456109291137 -118.9755660758207) bank16471 +16472 POINT(34.225343132693084 -117.85926990453824) bank16472 +16473 POINT(33.294417275310735 -118.31233265359015) bank16473 +16474 POINT(34.040408158645626 -117.70181976513793) bank16474 +16475 POINT(33.857214355637325 -119.13225920809188) bank16475 +16476 POINT(34.07596421564084 -117.75275313029441) bank16476 +16477 POINT(34.20447753066592 -118.22078787171478) bank16477 +16478 POINT(33.24742796470739 -118.71371337640429) bank16478 +16479 POINT(34.281712481660904 -117.32994304071111) bank16479 +16480 POINT(34.10006829316031 -117.41367140157199) bank16480 +16481 POINT(33.29066772919975 -118.6008424455672) bank16481 +16482 POINT(34.995942948532246 -118.73048099156429) bank16482 +16483 POINT(33.28201686640137 -118.16341705820965) bank16483 +16484 POINT(33.26849853808592 -118.79143449518006) bank16484 +16485 POINT(34.42822992650658 -118.48314748613386) bank16485 +16486 POINT(34.47036698176993 -117.33378897745025) bank16486 +16487 POINT(34.916296770897404 -117.66763815828521) bank16487 +16488 POINT(34.59423225465972 -117.3997531844005) bank16488 +16489 POINT(33.866207462330316 -118.66647498094585) bank16489 +16490 POINT(33.113042302283624 -117.72240738425432) bank16490 +16491 POINT(33.312606904634166 -118.11424893866881) bank16491 +16492 POINT(33.8964352586968 -118.90946933890532) bank16492 +16493 POINT(33.418659184075146 -117.76464444421424) bank16493 +16494 POINT(34.723246991878575 -118.87664057236259) bank16494 +16495 POINT(34.42854503971084 -118.03893652148412) bank16495 +16496 POINT(34.332485416201344 -118.76962970248552) bank16496 +16497 POINT(34.31077925655848 -118.66465414284974) bank16497 +16498 POINT(33.952256185127105 -117.56617245098033) bank16498 +16499 POINT(34.427869186458615 -118.68257355003064) bank16499 +16500 POINT(34.30767156210036 -118.685180114319) bank16500 +16501 POINT(33.31691727470864 -117.68874317536526) bank16501 +16502 POINT(34.805663691203094 -117.57674346167546) bank16502 +16503 POINT(35.05180988934783 -118.33432041709051) bank16503 +16504 POINT(33.59184682620346 -119.09342234499816) bank16504 +16505 POINT(34.16857552294433 -118.64379419004607) bank16505 +16506 POINT(33.52256044337467 -118.29658617806345) bank16506 +16507 POINT(33.187716770695275 -119.03756933751002) bank16507 +16508 POINT(33.77020083643417 -117.44649056416235) bank16508 +16509 POINT(33.91387277735126 -119.04283976904719) bank16509 +16510 POINT(34.43869166173305 -118.10152551412679) bank16510 +16511 POINT(34.66032067538356 -117.25087205716099) bank16511 +16512 POINT(34.22087513067525 -117.65734300443998) bank16512 +16513 POINT(33.44668831726054 -118.3743730558661) bank16513 +16514 POINT(33.77835919877608 -117.62711911841848) bank16514 +16515 POINT(33.23271583751953 -117.49810194746) bank16515 +16516 POINT(34.305591656958924 -117.94560819901693) bank16516 +16517 POINT(34.987494281416325 -117.87146382645645) bank16517 +16518 POINT(34.66959486055564 -118.62424625874662) bank16518 +16519 POINT(34.39312109416149 -119.04226972915485) bank16519 +16520 POINT(33.47822824856464 -119.20659698031707) bank16520 +16521 POINT(34.11420600791709 -117.98156928591717) bank16521 +16522 POINT(33.72100255389714 -118.4668653196111) bank16522 +16523 POINT(33.444708982281995 -117.42129219815166) bank16523 +16524 POINT(34.15419058344306 -118.4975112152613) bank16524 +16525 POINT(34.14847134224722 -119.22670309152302) bank16525 +16526 POINT(34.197073248612384 -117.61808027389452) bank16526 +16527 POINT(34.94277177571176 -117.27563114343894) bank16527 +16528 POINT(34.054574835536535 -117.6841792387229) bank16528 +16529 POINT(33.28531967075041 -117.73627984791189) bank16529 +16530 POINT(34.58947544582376 -118.08643047944487) bank16530 +16531 POINT(34.97946919616757 -118.54069903254721) bank16531 +16532 POINT(33.381098404666126 -118.38901888232114) bank16532 +16533 POINT(33.17594262820005 -118.77426122645407) bank16533 +16534 POINT(34.03312540444266 -118.00408785370088) bank16534 +16535 POINT(33.7555708529774 -117.965453332323) bank16535 +16536 POINT(33.16176649020565 -117.3928938840582) bank16536 +16537 POINT(33.555008777542625 -117.57927434792164) bank16537 +16538 POINT(33.44859963773327 -118.68815997173829) bank16538 +16539 POINT(34.1987647554104 -118.838376491847) bank16539 +16540 POINT(34.65057560931544 -118.97630195074765) bank16540 +16541 POINT(34.443859854174676 -119.1453092513755) bank16541 +16542 POINT(34.49179576437585 -117.28988359740902) bank16542 +16543 POINT(33.438525720076846 -117.47186153470066) bank16543 +16544 POINT(34.5558312081789 -119.09956571994701) bank16544 +16545 POINT(33.12585603302341 -118.54266106699451) bank16545 +16546 POINT(33.97536889582399 -117.99155253765049) bank16546 +16547 POINT(33.81472959614113 -117.34955671607324) bank16547 +16548 POINT(33.45513410150094 -117.50558665006413) bank16548 +16549 POINT(33.27288319526331 -117.41305308346571) bank16549 +16550 POINT(33.217599469539195 -117.60450612122726) bank16550 +16551 POINT(34.37301441341993 -118.05046424965444) bank16551 +16552 POINT(34.63005506297911 -117.70550005681888) bank16552 +16553 POINT(33.731555863791655 -118.89337789618257) bank16553 +16554 POINT(34.03967387633432 -118.7828511027298) bank16554 +16555 POINT(33.92006766633614 -117.77447955086912) bank16555 +16556 POINT(34.49551031055529 -117.92637720237484) bank16556 +16557 POINT(34.074901598138744 -118.37633866355439) bank16557 +16558 POINT(33.98091196361007 -118.81497967259055) bank16558 +16559 POINT(34.341021155959574 -117.27578526480858) bank16559 +16560 POINT(34.478680265763636 -118.06270051223964) bank16560 +16561 POINT(33.228733194107875 -118.68959177481773) bank16561 +16562 POINT(34.91236294044924 -119.0769457478273) bank16562 +16563 POINT(34.66903916141862 -118.77525013584767) bank16563 +16564 POINT(34.9648204140763 -117.43648148203341) bank16564 +16565 POINT(33.29705776894272 -118.97973415462936) bank16565 +16566 POINT(34.0676634276467 -118.38230189006177) bank16566 +16567 POINT(33.7660274499226 -117.89252007361438) bank16567 +16568 POINT(34.20660868314419 -118.45737197851172) bank16568 +16569 POINT(33.84014959735678 -117.31926635843178) bank16569 +16570 POINT(33.77089941878525 -118.67376499400196) bank16570 +16571 POINT(33.68896880774413 -117.66730857070301) bank16571 +16572 POINT(33.48971206217243 -118.41794547458491) bank16572 +16573 POINT(34.151960006197726 -119.05843819525427) bank16573 +16574 POINT(34.05839346126321 -117.51571147944021) bank16574 +16575 POINT(33.256960299191626 -118.28788519665946) bank16575 +16576 POINT(33.65683498058312 -118.36550684749882) bank16576 +16577 POINT(33.59916835311715 -118.5983801366512) bank16577 +16578 POINT(33.641185148682325 -118.06938696688755) bank16578 +16579 POINT(33.43415294278075 -119.01069894491144) bank16579 +16580 POINT(33.352795864737224 -117.48350666672026) bank16580 +16581 POINT(34.10634075184501 -117.84911775647869) bank16581 +16582 POINT(34.359519299424775 -119.11306034852484) bank16582 +16583 POINT(34.19259845027173 -118.74918354343842) bank16583 +16584 POINT(33.85440906249049 -119.1595619860198) bank16584 +16585 POINT(34.907840484256525 -118.0588498403163) bank16585 +16586 POINT(33.48204578222244 -117.71258327920168) bank16586 +16587 POINT(34.00965548694206 -118.85849116642342) bank16587 +16588 POINT(33.22583853415931 -117.40152593657228) bank16588 +16589 POINT(34.67363188933399 -118.28509715043997) bank16589 +16590 POINT(33.623738612268106 -117.71429537610395) bank16590 +16591 POINT(33.82784710495986 -118.38230319682067) bank16591 +16592 POINT(33.89985487761007 -117.94641044686375) bank16592 +16593 POINT(34.52611457408692 -118.28288463555579) bank16593 +16594 POINT(33.16598548750429 -117.70499384655989) bank16594 +16595 POINT(33.810736882474046 -117.35509097908471) bank16595 +16596 POINT(33.66188981059068 -117.64835339789155) bank16596 +16597 POINT(34.429632740076485 -117.98953115814862) bank16597 +16598 POINT(34.47684253448934 -118.76535725350242) bank16598 +16599 POINT(35.023608106607625 -118.75240317172836) bank16599 +16600 POINT(34.094021446738466 -117.8417112335258) bank16600 +16601 POINT(33.82230913780909 -118.58223378411742) bank16601 +16602 POINT(33.85961924175399 -119.08675887148878) bank16602 +16603 POINT(33.8614303649916 -118.79407439194503) bank16603 +16604 POINT(34.29647194527503 -119.07492284653966) bank16604 +16605 POINT(33.204357778487285 -118.85261251041794) bank16605 +16606 POINT(34.92594170895353 -119.11805856173521) bank16606 +16607 POINT(33.10144335624834 -118.6099821960005) bank16607 +16608 POINT(33.480342882849406 -118.85828131991181) bank16608 +16609 POINT(34.38206551495169 -118.4537545148285) bank16609 +16610 POINT(34.11906331116659 -117.59527782169415) bank16610 +16611 POINT(34.392358799395765 -117.80230971682951) bank16611 +16612 POINT(33.10330392097704 -118.57085484723898) bank16612 +16613 POINT(34.69325104840372 -117.83018344207991) bank16613 +16614 POINT(33.38466210988792 -118.97617797884482) bank16614 +16615 POINT(34.26942330551703 -119.0222412208898) bank16615 +16616 POINT(33.290439163630964 -117.67129956985949) bank16616 +16617 POINT(34.43933695110541 -118.69833168374188) bank16617 +16618 POINT(34.47866706459537 -118.92561047063307) bank16618 +16619 POINT(33.153904353167086 -117.59256464379288) bank16619 +16620 POINT(33.156417038146124 -117.33535212008573) bank16620 +16621 POINT(34.24314153982331 -119.07798981497557) bank16621 +16622 POINT(33.34465461069744 -117.88571482480648) bank16622 +16623 POINT(33.306531376500985 -118.79598202893007) bank16623 +16624 POINT(34.52189429805272 -118.24016239157179) bank16624 +16625 POINT(33.8465831964553 -118.33709041750944) bank16625 +16626 POINT(35.0199703692118 -118.57344070106501) bank16626 +16627 POINT(33.137431402553425 -117.84815137837597) bank16627 +16628 POINT(34.88753645376335 -118.81985968471948) bank16628 +16629 POINT(34.442432665282816 -118.17617988913193) bank16629 +16630 POINT(35.03560239093203 -118.32221304872071) bank16630 +16631 POINT(33.1889916327332 -117.39408207296538) bank16631 +16632 POINT(33.05712322472269 -117.77342582316074) bank16632 +16633 POINT(33.43262640602656 -118.16935657998229) bank16633 +16634 POINT(34.60660605817224 -118.69853708718097) bank16634 +16635 POINT(34.95110543639184 -117.29783686127983) bank16635 +16636 POINT(34.17949045048949 -117.97624335048492) bank16636 +16637 POINT(33.3855020438032 -118.9634766613531) bank16637 +16638 POINT(34.233539838456934 -117.48361498608992) bank16638 +16639 POINT(33.70473580620843 -119.22448670688121) bank16639 +16640 POINT(33.179160837489036 -118.56591254985976) bank16640 +16641 POINT(33.92031565332282 -118.23786211515205) bank16641 +16642 POINT(33.70928657964194 -118.12624439327) bank16642 +16643 POINT(33.16909971157886 -118.21921805143326) bank16643 +16644 POINT(33.68591542594055 -117.29681231374575) bank16644 +16645 POINT(34.10003791079631 -118.80509340112363) bank16645 +16646 POINT(33.93326807125793 -118.9325692850309) bank16646 +16647 POINT(33.51561771369895 -118.32042690613028) bank16647 +16648 POINT(34.77910551024721 -119.15814959283776) bank16648 +16649 POINT(33.2746208949057 -117.90904821242776) bank16649 +16650 POINT(34.16422938350699 -118.99181869828703) bank16650 +16651 POINT(33.300874881224956 -119.03533811147358) bank16651 +16652 POINT(34.81022925039968 -118.38416196535562) bank16652 +16653 POINT(34.915788327273305 -118.04949437642219) bank16653 +16654 POINT(33.07752818006374 -118.0852286882921) bank16654 +16655 POINT(33.83397738443784 -118.36446763156835) bank16655 +16656 POINT(34.95387598831216 -118.3769525713388) bank16656 +16657 POINT(34.618268278603786 -117.69776467608628) bank16657 +16658 POINT(34.0323616629291 -119.10233702346088) bank16658 +16659 POINT(34.746852054078325 -117.87469893456336) bank16659 +16660 POINT(34.82174996506933 -117.389399236807) bank16660 +16661 POINT(34.91037465366103 -117.99606936031196) bank16661 +16662 POINT(34.8659911250038 -118.50660343146822) bank16662 +16663 POINT(33.87980408241886 -117.30173307937626) bank16663 +16664 POINT(34.261429887599824 -118.29136446085568) bank16664 +16665 POINT(33.58383093839451 -118.36771374389058) bank16665 +16666 POINT(33.953978297882976 -117.49935278876836) bank16666 +16667 POINT(33.95132769659291 -118.15274091335395) bank16667 +16668 POINT(33.72964664660302 -118.79908172448734) bank16668 +16669 POINT(33.2286449279252 -118.66750666331522) bank16669 +16670 POINT(34.67093968795365 -117.50542224761018) bank16670 +16671 POINT(33.2526175256375 -118.61947744663757) bank16671 +16672 POINT(33.961191313967994 -118.43040955255796) bank16672 +16673 POINT(33.704365601155544 -118.69700299599982) bank16673 +16674 POINT(34.54398921369753 -118.87495347059748) bank16674 +16675 POINT(34.48075241050564 -118.17997899359227) bank16675 +16676 POINT(34.60564647984066 -118.3408011887926) bank16676 +16677 POINT(34.15523613193976 -118.64422999153612) bank16677 +16678 POINT(34.72187790634071 -119.01870530113356) bank16678 +16679 POINT(34.42288118995558 -117.34042589791873) bank16679 +16680 POINT(33.26763496800374 -117.49373179667914) bank16680 +16681 POINT(33.51048133450087 -119.21074488102659) bank16681 +16682 POINT(34.74306241484341 -117.57501467858826) bank16682 +16683 POINT(33.077718236662065 -119.01820289665959) bank16683 +16684 POINT(34.61321811600616 -117.56818300334457) bank16684 +16685 POINT(33.504972385325985 -117.86267003897646) bank16685 +16686 POINT(34.74225425677106 -119.07402233543979) bank16686 +16687 POINT(33.46538700940388 -117.78498446461968) bank16687 +16688 POINT(34.662566149956476 -118.15596658177778) bank16688 +16689 POINT(35.02224876243085 -119.06457731483967) bank16689 +16690 POINT(33.65372357908655 -119.10310626205178) bank16690 +16691 POINT(33.93900119425971 -117.75677418775193) bank16691 +16692 POINT(34.15303719935389 -117.71988729142065) bank16692 +16693 POINT(34.51717831087926 -118.11988303619951) bank16693 +16694 POINT(34.22440081652213 -119.0597368454684) bank16694 +16695 POINT(33.256771530293484 -118.34605297194395) bank16695 +16696 POINT(34.19933452997985 -118.31012356013602) bank16696 +16697 POINT(34.70001248712409 -117.96498061484934) bank16697 +16698 POINT(33.331496946947944 -117.74702408077096) bank16698 +16699 POINT(34.76015859367109 -119.03719796783297) bank16699 +16700 POINT(33.360304465362034 -118.35560500122483) bank16700 +16701 POINT(34.55846037359673 -119.2095307420315) bank16701 +16702 POINT(33.74742093834786 -119.05172471984547) bank16702 +16703 POINT(33.16953485837817 -119.0626031603165) bank16703 +16704 POINT(33.755880539242625 -117.89998384791672) bank16704 +16705 POINT(33.9769088497265 -118.45966173912655) bank16705 +16706 POINT(34.4258370085302 -118.79238429457708) bank16706 +16707 POINT(33.15387257646177 -118.93095571152375) bank16707 +16708 POINT(33.563303242997705 -117.86609336782338) bank16708 +16709 POINT(34.11550921112507 -117.41029263348881) bank16709 +16710 POINT(34.53990750818608 -118.29837989054707) bank16710 +16711 POINT(33.4749057836595 -117.6305227100439) bank16711 +16712 POINT(34.36754313071058 -118.47125951786573) bank16712 +16713 POINT(33.08919041415099 -119.0001998719712) bank16713 +16714 POINT(33.95693491236146 -117.94034454860054) bank16714 +16715 POINT(34.64091093465954 -118.96069621704446) bank16715 +16716 POINT(33.34513811382649 -119.13791158562458) bank16716 +16717 POINT(34.2623818663042 -117.86474934820957) bank16717 +16718 POINT(34.85625065577403 -118.76798275865468) bank16718 +16719 POINT(33.36687120066332 -118.61079624943162) bank16719 +16720 POINT(33.852413787535035 -117.76303216476882) bank16720 +16721 POINT(33.95623683143886 -117.36244030825127) bank16721 +16722 POINT(33.39863918912745 -119.08661960612012) bank16722 +16723 POINT(33.15106764733619 -118.8393182449811) bank16723 +16724 POINT(34.38587488831972 -117.40365950815655) bank16724 +16725 POINT(33.207189412627464 -118.58990804968678) bank16725 +16726 POINT(33.987156470668076 -119.16681309495034) bank16726 +16727 POINT(34.08046347037144 -119.09531682192603) bank16727 +16728 POINT(34.78772451684326 -117.9070088158329) bank16728 +16729 POINT(34.237587216926975 -117.47217300062937) bank16729 +16730 POINT(33.058484274951724 -118.46106297228165) bank16730 +16731 POINT(34.89267032280673 -117.82102077877639) bank16731 +16732 POINT(34.789074730484614 -118.29058272310026) bank16732 +16733 POINT(34.280883180513044 -117.84278326728665) bank16733 +16734 POINT(33.26766253219651 -117.36368386709486) bank16734 +16735 POINT(33.246073414868604 -118.72867143212176) bank16735 +16736 POINT(34.65586441776976 -118.62181641790934) bank16736 +16737 POINT(33.212716588591775 -118.05333419219455) bank16737 +16738 POINT(34.195314745684556 -117.3091348166404) bank16738 +16739 POINT(34.381075469284454 -118.17542034958647) bank16739 +16740 POINT(34.36382071261327 -119.01452875316616) bank16740 +16741 POINT(34.270683219633305 -118.55558536188012) bank16741 +16742 POINT(34.906097695968946 -118.32375547573191) bank16742 +16743 POINT(33.42542335572933 -118.81751323842455) bank16743 +16744 POINT(33.61432116936332 -117.74372087977812) bank16744 +16745 POINT(33.451114828211175 -118.85205324647498) bank16745 +16746 POINT(33.77681142076324 -117.8477310952371) bank16746 +16747 POINT(34.26452913186683 -117.81887161736563) bank16747 +16748 POINT(34.339189396504935 -118.52936458816463) bank16748 +16749 POINT(33.30758118235658 -117.50249457893214) bank16749 +16750 POINT(34.03557902632519 -118.94770805077036) bank16750 +16751 POINT(33.667633113524715 -118.99845583925571) bank16751 +16752 POINT(33.88641881506641 -117.92773920989161) bank16752 +16753 POINT(33.961681487248555 -118.72335003413271) bank16753 +16754 POINT(34.09164637271816 -117.72847360820866) bank16754 +16755 POINT(34.81330646874286 -117.69299175814572) bank16755 +16756 POINT(33.16681395506688 -118.09347148958594) bank16756 +16757 POINT(34.07891499934192 -117.61012719311813) bank16757 +16758 POINT(34.77744864720453 -117.50694049065235) bank16758 +16759 POINT(33.574654008093944 -117.92263620961808) bank16759 +16760 POINT(33.71276192329691 -117.83930310476634) bank16760 +16761 POINT(33.85994030643128 -117.58351719407601) bank16761 +16762 POINT(34.23273613169126 -117.85174473850536) bank16762 +16763 POINT(34.524710476971656 -118.6113583052755) bank16763 +16764 POINT(34.188629638223105 -118.5532242865312) bank16764 +16765 POINT(33.2456413043277 -118.74532793550502) bank16765 +16766 POINT(34.42150026820523 -119.01749598170635) bank16766 +16767 POINT(33.313526586448674 -117.86822195807531) bank16767 +16768 POINT(33.23414653818313 -118.01094239037856) bank16768 +16769 POINT(34.91543187945256 -117.65573484223769) bank16769 +16770 POINT(34.6535243385884 -118.49747391653067) bank16770 +16771 POINT(33.59350615042081 -118.42088648195868) bank16771 +16772 POINT(34.547776800074864 -119.03366638121183) bank16772 +16773 POINT(34.43102245300352 -118.10617764870562) bank16773 +16774 POINT(33.8552494057221 -117.94494456326164) bank16774 +16775 POINT(34.11959024655876 -117.69876163600918) bank16775 +16776 POINT(33.759893917158834 -118.85361526157187) bank16776 +16777 POINT(33.98522750176334 -118.98855240079112) bank16777 +16778 POINT(34.921690506358274 -119.07017543588678) bank16778 +16779 POINT(34.86164330375574 -118.66283717287935) bank16779 +16780 POINT(33.072297475129595 -117.9006784958128) bank16780 +16781 POINT(33.62568457615965 -118.61194833559956) bank16781 +16782 POINT(34.567715232949475 -118.74196458088313) bank16782 +16783 POINT(34.52572809928404 -118.24959749082977) bank16783 +16784 POINT(34.037836424152395 -117.57312078286192) bank16784 +16785 POINT(33.96443288398259 -117.74406456872364) bank16785 +16786 POINT(33.42509992477406 -118.73639650935512) bank16786 +16787 POINT(34.488463310954174 -119.15488988250098) bank16787 +16788 POINT(33.165550332934316 -117.41824065851934) bank16788 +16789 POINT(33.80696868829466 -117.66821967878472) bank16789 +16790 POINT(33.804711690776394 -118.6898058633184) bank16790 +16791 POINT(34.10870195236011 -118.2126806555345) bank16791 +16792 POINT(34.56327338316404 -117.77608017520792) bank16792 +16793 POINT(33.99466754511466 -117.82583713424073) bank16793 +16794 POINT(34.62812481362383 -117.67328886825435) bank16794 +16795 POINT(34.90220003247837 -118.3809771530094) bank16795 +16796 POINT(34.73685665614975 -118.21568729498175) bank16796 +16797 POINT(33.47499631728399 -118.30209177560869) bank16797 +16798 POINT(33.398490687294796 -118.59058195280552) bank16798 +16799 POINT(33.113132300528555 -118.3578556057899) bank16799 +16800 POINT(34.93605811081798 -118.86598275936636) bank16800 +16801 POINT(33.058348631495825 -117.94713698362077) bank16801 +16802 POINT(34.25707813600716 -117.41496269883189) bank16802 +16803 POINT(33.64190671272195 -118.9920660536416) bank16803 +16804 POINT(33.62415251032328 -117.98014980440574) bank16804 +16805 POINT(33.69171935618466 -118.4598901477711) bank16805 +16806 POINT(34.61363116139613 -118.94076396587562) bank16806 +16807 POINT(33.59040752715748 -118.85922421400664) bank16807 +16808 POINT(34.05273230711988 -119.15910656019052) bank16808 +16809 POINT(34.60497674396927 -117.35183856048255) bank16809 +16810 POINT(34.02905881566743 -117.90533948898145) bank16810 +16811 POINT(33.75499084904064 -117.37246900173915) bank16811 +16812 POINT(34.149677892022446 -117.24633550199299) bank16812 +16813 POINT(33.62770247900187 -117.91807687333623) bank16813 +16814 POINT(33.61186078744795 -117.50899118754124) bank16814 +16815 POINT(34.0355233722978 -117.29158793322414) bank16815 +16816 POINT(34.62139362596913 -117.53153708100956) bank16816 +16817 POINT(34.94135224619262 -118.93126852113893) bank16817 +16818 POINT(34.270445503698895 -117.51811192591374) bank16818 +16819 POINT(34.909894447578125 -118.90894877344695) bank16819 +16820 POINT(33.07509676597367 -118.91686319014362) bank16820 +16821 POINT(33.924551216483415 -117.94586274778226) bank16821 +16822 POINT(34.892517540296026 -117.93464956000341) bank16822 +16823 POINT(33.843354955293734 -119.0393909694103) bank16823 +16824 POINT(33.33703925928043 -118.36568455692301) bank16824 +16825 POINT(33.127509722471146 -117.88575570547974) bank16825 +16826 POINT(34.79385810688777 -118.16810362502255) bank16826 +16827 POINT(34.46208310986123 -118.51316931543705) bank16827 +16828 POINT(33.726188768545995 -117.69492343141097) bank16828 +16829 POINT(33.39886947019199 -118.82296052066779) bank16829 +16830 POINT(33.50188967909389 -119.01139643680436) bank16830 +16831 POINT(33.3724442603541 -118.02837640510639) bank16831 +16832 POINT(33.99310654600393 -117.5520955252122) bank16832 +16833 POINT(34.39334876669389 -117.60655078764258) bank16833 +16834 POINT(34.74997076097388 -119.2259992436042) bank16834 +16835 POINT(33.759607018638434 -117.58598824393721) bank16835 +16836 POINT(34.06736336258922 -119.08945403999606) bank16836 +16837 POINT(34.933305730788 -118.21210358840027) bank16837 +16838 POINT(34.96131581935451 -118.56656251400464) bank16838 +16839 POINT(34.980749712386675 -118.60176831348764) bank16839 +16840 POINT(34.19282841569458 -118.43822782880116) bank16840 +16841 POINT(34.173303971396166 -117.30852105499933) bank16841 +16842 POINT(34.768833779304686 -119.09319338705963) bank16842 +16843 POINT(33.77754868979047 -117.26104835263534) bank16843 +16844 POINT(33.42162805926109 -117.27925629414845) bank16844 +16845 POINT(35.03351493288539 -118.7729537542161) bank16845 +16846 POINT(33.28218898101227 -117.5100910250821) bank16846 +16847 POINT(34.39162704944229 -119.00581515060725) bank16847 +16848 POINT(34.91763567423992 -118.24049026809573) bank16848 +16849 POINT(33.17283658129533 -117.83645383351215) bank16849 +16850 POINT(34.94496848533358 -117.88054583045954) bank16850 +16851 POINT(33.67433806325229 -118.34527693528527) bank16851 +16852 POINT(33.81323079718099 -117.60112265866321) bank16852 +16853 POINT(33.78318980336357 -119.04246245172915) bank16853 +16854 POINT(34.76848478750523 -117.63893859513271) bank16854 +16855 POINT(33.55893072424457 -119.10727825754324) bank16855 +16856 POINT(34.006881678041886 -118.64368031378619) bank16856 +16857 POINT(33.46067596754507 -119.24130955777846) bank16857 +16858 POINT(33.55764416478788 -117.66154867336482) bank16858 +16859 POINT(34.71305887883616 -117.61043150957548) bank16859 +16860 POINT(33.893798359087185 -117.7963874961303) bank16860 +16861 POINT(33.674301448563064 -118.6245426258029) bank16861 +16862 POINT(34.103954668558096 -118.59397204048328) bank16862 +16863 POINT(34.20556372581489 -117.5674613396439) bank16863 +16864 POINT(34.36536582742607 -119.18846875983213) bank16864 +16865 POINT(33.399848146449955 -117.2557781679948) bank16865 +16866 POINT(34.00454166379894 -117.56233275896857) bank16866 +16867 POINT(33.58191920151619 -117.44615149183461) bank16867 +16868 POINT(34.18839092865448 -118.42851731553257) bank16868 +16869 POINT(33.635788185592816 -119.03602963945808) bank16869 +16870 POINT(33.397753696011186 -118.41359849027232) bank16870 +16871 POINT(33.05819432423414 -117.5886827160782) bank16871 +16872 POINT(33.41686066870752 -118.23506287248975) bank16872 +16873 POINT(34.4180322452487 -117.9746462596613) bank16873 +16874 POINT(33.48796502744756 -117.75590513119091) bank16874 +16875 POINT(34.4834621772856 -119.18670502760409) bank16875 +16876 POINT(33.634347964663846 -118.1248445971264) bank16876 +16877 POINT(34.41434579927981 -118.30950027012538) bank16877 +16878 POINT(34.03320505438694 -119.0806745685587) bank16878 +16879 POINT(34.227023267177145 -118.31323541581293) bank16879 +16880 POINT(34.81847243423811 -118.52239305718774) bank16880 +16881 POINT(35.01925194663269 -118.13143500797737) bank16881 +16882 POINT(34.2131484191509 -117.93267659136339) bank16882 +16883 POINT(33.49576060027799 -118.06227621967598) bank16883 +16884 POINT(34.11128042050457 -117.26349657124784) bank16884 +16885 POINT(34.3042361417854 -117.2759128771799) bank16885 +16886 POINT(34.62654727046607 -117.38287839312879) bank16886 +16887 POINT(34.63679497431008 -117.5279170913641) bank16887 +16888 POINT(34.57608610592034 -117.32880129258542) bank16888 +16889 POINT(34.4285639573216 -118.23840767039911) bank16889 +16890 POINT(33.45097852777141 -118.83487112557515) bank16890 +16891 POINT(34.71540579872191 -117.84483355624762) bank16891 +16892 POINT(34.08843716734529 -117.78468269481841) bank16892 +16893 POINT(33.977938496831506 -118.09580007779242) bank16893 +16894 POINT(33.765303726764515 -117.98320443294239) bank16894 +16895 POINT(33.755217925577206 -118.61554350480552) bank16895 +16896 POINT(33.27034104125896 -118.98407106762525) bank16896 +16897 POINT(34.68346820331583 -117.56164853777501) bank16897 +16898 POINT(34.84184804899069 -119.13424340091687) bank16898 +16899 POINT(34.11511832244178 -117.51239536821491) bank16899 +16900 POINT(34.4608955631168 -117.45676920983753) bank16900 +16901 POINT(34.62685412923695 -117.42601321471656) bank16901 +16902 POINT(34.04259383615849 -118.25665192904417) bank16902 +16903 POINT(34.349146992656124 -118.12196241012371) bank16903 +16904 POINT(33.26334260223254 -118.11768667603194) bank16904 +16905 POINT(33.67227073533822 -117.74381683252155) bank16905 +16906 POINT(34.12719326291795 -118.71012426647657) bank16906 +16907 POINT(33.113212336527056 -118.58150602900132) bank16907 +16908 POINT(33.85351879777003 -117.88946046162651) bank16908 +16909 POINT(34.92918896159422 -117.96071334804148) bank16909 +16910 POINT(33.08058980094125 -118.8362388464026) bank16910 +16911 POINT(34.18529571073866 -118.10889116617348) bank16911 +16912 POINT(33.611009716191916 -118.59081877140152) bank16912 +16913 POINT(34.355411014472644 -117.591715786198) bank16913 +16914 POINT(34.165589661899155 -119.18932334648935) bank16914 +16915 POINT(33.23815564987605 -119.0872495471004) bank16915 +16916 POINT(34.574922602041276 -117.45127102690869) bank16916 +16917 POINT(34.854053836485946 -117.92388403620035) bank16917 +16918 POINT(33.394046684664694 -117.75702595132539) bank16918 +16919 POINT(34.10652791973318 -118.94263614070063) bank16919 +16920 POINT(34.5447892193867 -118.49077098426979) bank16920 +16921 POINT(33.33569184975953 -118.2912408627005) bank16921 +16922 POINT(33.74329193186993 -118.96653716211182) bank16922 +16923 POINT(34.15311425163246 -117.31345118595604) bank16923 +16924 POINT(33.72211308207369 -117.3451981726322) bank16924 +16925 POINT(33.222216761000524 -118.25893979910016) bank16925 +16926 POINT(34.118131046348545 -117.28158542264943) bank16926 +16927 POINT(34.19554655709402 -118.61457247238923) bank16927 +16928 POINT(33.12061498700034 -117.37707622390596) bank16928 +16929 POINT(33.315126914019224 -117.4510712274783) bank16929 +16930 POINT(34.577955018647 -118.4013480042832) bank16930 +16931 POINT(33.26336050553658 -119.16353923151168) bank16931 +16932 POINT(34.45489452980118 -118.34083788319333) bank16932 +16933 POINT(34.97161561474381 -117.55129555127438) bank16933 +16934 POINT(34.24473375340918 -118.21795293884819) bank16934 +16935 POINT(33.25025492596243 -119.14638651782673) bank16935 +16936 POINT(34.04032827918586 -117.34234296902322) bank16936 +16937 POINT(33.81913404006028 -119.04381006909213) bank16937 +16938 POINT(33.290294043234404 -119.2144772237294) bank16938 +16939 POINT(34.57840876205735 -117.6249882984421) bank16939 +16940 POINT(33.662386956035846 -119.07730560157128) bank16940 +16941 POINT(34.484631619692244 -118.66438990831035) bank16941 +16942 POINT(34.997056586377965 -118.35278694661879) bank16942 +16943 POINT(33.09558337144602 -117.5531327657588) bank16943 +16944 POINT(33.90517951891681 -118.7371314405763) bank16944 +16945 POINT(34.185720999790284 -117.58356758889549) bank16945 +16946 POINT(34.15476365136815 -119.12098295541973) bank16946 +16947 POINT(33.86171591889926 -118.09491131274186) bank16947 +16948 POINT(34.7083494373291 -118.18574670620339) bank16948 +16949 POINT(33.61886854495292 -118.39388632045485) bank16949 +16950 POINT(33.40940887653279 -118.81193315682337) bank16950 +16951 POINT(33.08392497103747 -117.8310136312876) bank16951 +16952 POINT(34.85051562812794 -118.10494710984685) bank16952 +16953 POINT(33.52449121893797 -117.6491775462667) bank16953 +16954 POINT(33.53573595515808 -118.01598880153082) bank16954 +16955 POINT(33.69080546531111 -118.71895239393507) bank16955 +16956 POINT(33.102652559011005 -118.27420740113153) bank16956 +16957 POINT(34.25606679895972 -119.16695251263536) bank16957 +16958 POINT(33.89707282203145 -117.67000047933819) bank16958 +16959 POINT(34.19989731873995 -118.658234454485) bank16959 +16960 POINT(33.06744408236371 -117.83058815831356) bank16960 +16961 POINT(34.796800940415444 -119.05314761911706) bank16961 +16962 POINT(34.10259308199286 -117.73255288489995) bank16962 +16963 POINT(34.56689521949268 -118.32400665663945) bank16963 +16964 POINT(33.92451388219047 -118.45525079616462) bank16964 +16965 POINT(34.128618007876376 -119.03482849218203) bank16965 +16966 POINT(34.78491334362284 -118.39575017345099) bank16966 +16967 POINT(33.07401562387667 -118.69167269454165) bank16967 +16968 POINT(33.511767098815476 -117.98130978163249) bank16968 +16969 POINT(34.52298599552166 -117.3947289534114) bank16969 +16970 POINT(33.8853196363245 -117.43503418806661) bank16970 +16971 POINT(34.8047528878822 -118.22411818279964) bank16971 +16972 POINT(34.76565643448034 -118.96725425095204) bank16972 +16973 POINT(34.35989311872694 -117.27003614757501) bank16973 +16974 POINT(34.36149071477625 -117.54166247411753) bank16974 +16975 POINT(33.69411536572725 -119.145683564076) bank16975 +16976 POINT(34.95347222592973 -118.86280010928573) bank16976 +16977 POINT(34.83984909433862 -117.54694277330822) bank16977 +16978 POINT(34.74993871271191 -117.42531091862647) bank16978 +16979 POINT(33.46642683548482 -118.14099710716158) bank16979 +16980 POINT(33.80251842816807 -118.67748791432774) bank16980 +16981 POINT(33.32735785980728 -117.75458523124499) bank16981 +16982 POINT(34.6889260213497 -118.18790015042298) bank16982 +16983 POINT(33.280259347180625 -118.47421050349028) bank16983 +16984 POINT(34.67691216412596 -118.12783714148449) bank16984 +16985 POINT(33.786678444780875 -117.9564942870163) bank16985 +16986 POINT(33.13422128035666 -119.15267245181792) bank16986 +16987 POINT(33.40596480413843 -118.79883289228144) bank16987 +16988 POINT(34.61364105005213 -117.49301004693861) bank16988 +16989 POINT(33.81524793312149 -117.34553258877591) bank16989 +16990 POINT(33.39555147898592 -117.69434471986094) bank16990 +16991 POINT(33.917178110776696 -118.62109211421291) bank16991 +16992 POINT(35.01937114273365 -117.33179988656273) bank16992 +16993 POINT(33.160746815285094 -119.02742657632673) bank16993 +16994 POINT(33.76767251150002 -118.75206995557045) bank16994 +16995 POINT(34.144872199155834 -119.11957555500565) bank16995 +16996 POINT(34.130828912896064 -118.13480091551094) bank16996 +16997 POINT(33.09521098792288 -118.79388790472765) bank16997 +16998 POINT(34.79909895630303 -118.80121560133759) bank16998 +16999 POINT(33.350379635080614 -118.26477921213517) bank16999 +17000 POINT(33.24011022556547 -118.54228167792458) bank17000 +17001 POINT(33.52647909414189 -118.80713369751193) bank17001 +17002 POINT(33.78495455583125 -118.85705085339688) bank17002 +17003 POINT(33.44073280211901 -119.2052509201822) bank17003 +17004 POINT(33.832428340647844 -118.86101608044201) bank17004 +17005 POINT(34.20882631111897 -117.44280821059883) bank17005 +17006 POINT(34.548364746612435 -118.29813554311441) bank17006 +17007 POINT(34.93750592741834 -117.61136039305538) bank17007 +17008 POINT(33.102097047767295 -118.18593265380358) bank17008 +17009 POINT(34.33149151546223 -118.65786845508087) bank17009 +17010 POINT(34.85616840759415 -119.06678032137081) bank17010 +17011 POINT(34.12145298701174 -117.83137345254468) bank17011 +17012 POINT(33.434171198491654 -117.96970692277726) bank17012 +17013 POINT(34.092630198755174 -117.32888089688744) bank17013 +17014 POINT(34.66180520059266 -118.22612533095324) bank17014 +17015 POINT(33.66562642670914 -118.04631628640301) bank17015 +17016 POINT(34.957019069133494 -117.68843274414466) bank17016 +17017 POINT(33.72370846232405 -117.60085326610154) bank17017 +17018 POINT(34.598724228984864 -117.9437290897254) bank17018 +17019 POINT(33.34492739694984 -118.14131998307651) bank17019 +17020 POINT(34.57164035996836 -119.05999406641025) bank17020 +17021 POINT(34.47301356934848 -118.05881838905768) bank17021 +17022 POINT(33.72384939752199 -119.09082210620417) bank17022 +17023 POINT(34.97947707101348 -118.25908042159465) bank17023 +17024 POINT(34.56148770752677 -119.10297944788694) bank17024 +17025 POINT(34.59820239193246 -118.51997329945323) bank17025 +17026 POINT(34.01269875337273 -117.5685454833757) bank17026 +17027 POINT(33.140999485086425 -118.43568534060205) bank17027 +17028 POINT(33.28505008854883 -117.36071921440828) bank17028 +17029 POINT(33.403161843327304 -119.22256966362308) bank17029 +17030 POINT(33.90785826746732 -119.11211463872183) bank17030 +17031 POINT(33.78276029529658 -118.95154238486154) bank17031 +17032 POINT(34.269794565945425 -118.81777206253301) bank17032 +17033 POINT(33.99115925672352 -118.76561610538062) bank17033 +17034 POINT(34.88256983256536 -118.33050679268814) bank17034 +17035 POINT(34.67094381614458 -118.2892461708931) bank17035 +17036 POINT(34.51865879569102 -117.45879921213641) bank17036 +17037 POINT(33.50747040292018 -118.73856481370527) bank17037 +17038 POINT(34.42099137688866 -117.49510658951272) bank17038 +17039 POINT(33.900056186141235 -117.73351999644213) bank17039 +17040 POINT(33.41428672901616 -117.24657620691693) bank17040 +17041 POINT(33.594086548930676 -118.23902179854494) bank17041 +17042 POINT(34.90051372886977 -117.32810128874634) bank17042 +17043 POINT(34.873178833593606 -117.93369905076614) bank17043 +17044 POINT(33.66766448735144 -117.44501072868653) bank17044 +17045 POINT(34.92600577037729 -117.4633202836119) bank17045 +17046 POINT(34.80337007657457 -119.01655318158106) bank17046 +17047 POINT(34.07942790381464 -119.18729660240749) bank17047 +17048 POINT(33.700718277099156 -118.9591959953527) bank17048 +17049 POINT(34.711642061624566 -118.08231039934215) bank17049 +17050 POINT(34.27255341496727 -119.21956617554366) bank17050 +17051 POINT(33.321797929504974 -118.09641514948291) bank17051 +17052 POINT(34.7525083788475 -118.35140789440034) bank17052 +17053 POINT(33.99313690907141 -118.56215862633009) bank17053 +17054 POINT(34.405057271888424 -118.43053758756515) bank17054 +17055 POINT(33.3651904655279 -119.00098326041139) bank17055 +17056 POINT(33.47677251154194 -117.63033477920129) bank17056 +17057 POINT(33.41122826820379 -119.18458091145236) bank17057 +17058 POINT(33.87249674603872 -117.67240304209217) bank17058 +17059 POINT(33.96529179545539 -117.4788200769312) bank17059 +17060 POINT(35.017572189614924 -117.90325757708956) bank17060 +17061 POINT(34.66212218137034 -117.81168085676185) bank17061 +17062 POINT(33.7524482494492 -118.2854189722979) bank17062 +17063 POINT(33.43285765403275 -119.23427108359617) bank17063 +17064 POINT(33.2527680401521 -119.09163874244831) bank17064 +17065 POINT(33.725676410654565 -118.1831224886801) bank17065 +17066 POINT(33.42742712686783 -119.20371701753616) bank17066 +17067 POINT(34.602390550861735 -117.78044867235143) bank17067 +17068 POINT(33.8515968714559 -117.92624866437386) bank17068 +17069 POINT(34.03646984989702 -119.04900224600829) bank17069 +17070 POINT(34.61920737461127 -118.62557407462035) bank17070 +17071 POINT(33.424650536128645 -117.43019312452772) bank17071 +17072 POINT(34.764668947556146 -117.88580128430011) bank17072 +17073 POINT(33.07189859923431 -118.18219885052201) bank17073 +17074 POINT(33.88759836945134 -117.71125191070736) bank17074 +17075 POINT(34.21890436792962 -117.4103983252832) bank17075 +17076 POINT(34.68129541237268 -118.47977543831743) bank17076 +17077 POINT(33.65728458015928 -117.9797363538274) bank17077 +17078 POINT(33.09364168984107 -117.32233420252996) bank17078 +17079 POINT(33.77696690136597 -118.173987163835) bank17079 +17080 POINT(34.994025600035975 -118.89211228303272) bank17080 +17081 POINT(33.80082925030349 -118.97089508331472) bank17081 +17082 POINT(33.554763097407964 -118.32603257690498) bank17082 +17083 POINT(33.39860883045883 -117.34227060156384) bank17083 +17084 POINT(33.411451001637154 -118.10180932701768) bank17084 +17085 POINT(33.79393915165658 -118.08087737638346) bank17085 +17086 POINT(33.303679989256835 -118.65779896283375) bank17086 +17087 POINT(34.89142505666213 -118.3406073047976) bank17087 +17088 POINT(33.680028250609084 -118.54116673670237) bank17088 +17089 POINT(33.670002090418265 -117.4264488819243) bank17089 +17090 POINT(34.45457237936933 -118.65713228451615) bank17090 +17091 POINT(33.9243807259197 -118.32478942720513) bank17091 +17092 POINT(34.51198450149241 -118.6606990363053) bank17092 +17093 POINT(33.15509688125193 -119.19868788408883) bank17093 +17094 POINT(34.365454835997525 -118.44282329721896) bank17094 +17095 POINT(34.46014217685465 -119.07321905363959) bank17095 +17096 POINT(33.149555820135696 -117.42430440033246) bank17096 +17097 POINT(33.98507522048289 -118.02564255493361) bank17097 +17098 POINT(35.02249406185786 -117.5959688899118) bank17098 +17099 POINT(33.89573689493298 -118.76270652886444) bank17099 +17100 POINT(33.35043787887381 -118.4012558648009) bank17100 +17101 POINT(34.6495537826804 -117.56093678585577) bank17101 +17102 POINT(34.74487394704113 -117.33919249608533) bank17102 +17103 POINT(33.63818563453173 -118.47024717516743) bank17103 +17104 POINT(34.33756948513335 -117.548089641085) bank17104 +17105 POINT(34.173401566013055 -117.74615955495254) bank17105 +17106 POINT(33.98849048856653 -118.49232779392442) bank17106 +17107 POINT(34.2423953223344 -118.39809958893423) bank17107 +17108 POINT(34.618688958000995 -117.83682539113022) bank17108 +17109 POINT(34.96959764193063 -117.80729486417185) bank17109 +17110 POINT(34.62639322712885 -118.48955581579997) bank17110 +17111 POINT(34.85738374879615 -118.68101426347656) bank17111 +17112 POINT(34.07496562150438 -118.2309398170211) bank17112 +17113 POINT(33.999394372637695 -118.83301078069336) bank17113 +17114 POINT(34.898710713954614 -117.5163455775027) bank17114 +17115 POINT(33.54755485270967 -117.64765614890237) bank17115 +17116 POINT(34.50811114468541 -117.73219612314102) bank17116 +17117 POINT(33.931619690652454 -118.1881578878537) bank17117 +17118 POINT(34.9143840705673 -117.47999923382017) bank17118 +17119 POINT(34.67468917725357 -117.31024818017332) bank17119 +17120 POINT(34.41104968121332 -117.85535867165461) bank17120 +17121 POINT(33.67404983476135 -119.2078971799307) bank17121 +17122 POINT(33.972613635339926 -117.97321339256061) bank17122 +17123 POINT(34.73561026348591 -118.51341944997849) bank17123 +17124 POINT(34.90294131598145 -117.687659192558) bank17124 +17125 POINT(34.00444612732581 -118.3569469224443) bank17125 +17126 POINT(34.86076360131806 -118.74189600890165) bank17126 +17127 POINT(34.73879724148324 -119.02375467045003) bank17127 +17128 POINT(34.292837325687195 -119.05150867354484) bank17128 +17129 POINT(34.39884017416128 -119.2263390541194) bank17129 +17130 POINT(34.00863782023471 -118.77801601389633) bank17130 +17131 POINT(33.74015586191375 -118.64284880126037) bank17131 +17132 POINT(33.764423769049024 -119.17638540854814) bank17132 +17133 POINT(34.48147319518791 -119.12000196493064) bank17133 +17134 POINT(33.64148943873693 -118.92775197219267) bank17134 +17135 POINT(34.16058695933862 -117.70921325432111) bank17135 +17136 POINT(34.85521126957915 -118.814117261964) bank17136 +17137 POINT(33.097483877984544 -117.34654445644999) bank17137 +17138 POINT(33.316152248934316 -118.2532186928302) bank17138 +17139 POINT(34.70030750261991 -119.04963037976694) bank17139 +17140 POINT(34.057136044585114 -118.58202608434694) bank17140 +17141 POINT(33.75270757827164 -118.55150517145178) bank17141 +17142 POINT(34.911346313249595 -117.37076418799283) bank17142 +17143 POINT(34.531889339233366 -118.83060329856094) bank17143 +17144 POINT(35.01554744241144 -117.37107442807702) bank17144 +17145 POINT(34.23703737013233 -118.27316043623651) bank17145 +17146 POINT(33.76501612718704 -117.8435562981948) bank17146 +17147 POINT(34.410203657302894 -117.26692494171127) bank17147 +17148 POINT(34.53066724830657 -117.80512728776085) bank17148 +17149 POINT(33.9530893359797 -118.426929017407) bank17149 +17150 POINT(34.92143862624308 -117.34697675152393) bank17150 +17151 POINT(35.029253776908945 -118.40700221962145) bank17151 +17152 POINT(33.207470461650466 -118.25804063610695) bank17152 +17153 POINT(33.289597211671555 -117.86978748079493) bank17153 +17154 POINT(33.240997565860134 -118.02955392186219) bank17154 +17155 POINT(34.615260870611465 -118.2739160849715) bank17155 +17156 POINT(34.783277650557274 -117.77688990762891) bank17156 +17157 POINT(34.45833738374455 -118.19625526260786) bank17157 +17158 POINT(34.14924968518828 -117.52188275993961) bank17158 +17159 POINT(34.346221158048415 -119.01881417468451) bank17159 +17160 POINT(33.83677299144701 -118.19821403063123) bank17160 +17161 POINT(34.18234828083734 -119.01491312901621) bank17161 +17162 POINT(34.46810542229938 -117.66112533622318) bank17162 +17163 POINT(34.28563200288538 -117.51888918908173) bank17163 +17164 POINT(34.78505173089011 -117.50381430430713) bank17164 +17165 POINT(34.34860926117752 -118.64014248110777) bank17165 +17166 POINT(34.44946253744527 -118.80669860817368) bank17166 +17167 POINT(34.13947408171886 -117.26638988801747) bank17167 +17168 POINT(34.38894964546362 -117.74836578341649) bank17168 +17169 POINT(33.69307151110157 -117.93833127446116) bank17169 +17170 POINT(33.55995403291455 -117.53677429730192) bank17170 +17171 POINT(34.47486668793541 -118.23493341166814) bank17171 +17172 POINT(33.67948144107891 -118.53024019704225) bank17172 +17173 POINT(34.05205686084413 -117.96393385491223) bank17173 +17174 POINT(33.5776109638249 -117.88540996945653) bank17174 +17175 POINT(34.7147519371174 -118.58239889883188) bank17175 +17176 POINT(33.5879450828971 -118.87856100278624) bank17176 +17177 POINT(34.793954171497575 -117.67470664878522) bank17177 +17178 POINT(34.1868991290031 -118.37049700977936) bank17178 +17179 POINT(33.613255847941446 -118.06087835341374) bank17179 +17180 POINT(34.240560631785115 -119.04590508055749) bank17180 +17181 POINT(34.11141592977191 -118.0253847555704) bank17181 +17182 POINT(33.113020616839705 -117.28790849271874) bank17182 +17183 POINT(33.06557339140533 -117.74921971200773) bank17183 +17184 POINT(34.603469107787376 -118.5496678300258) bank17184 +17185 POINT(34.59943951105286 -118.1060512241244) bank17185 +17186 POINT(34.22734553965338 -117.87998538839653) bank17186 +17187 POINT(33.99506550571273 -118.0060646757448) bank17187 +17188 POINT(34.492367249650016 -118.6345533667794) bank17188 +17189 POINT(35.02466827781702 -117.85414118452988) bank17189 +17190 POINT(33.16251689905969 -118.16179435427598) bank17190 +17191 POINT(34.4661362142529 -119.02287039605487) bank17191 +17192 POINT(34.49830497947449 -117.90687558350191) bank17192 +17193 POINT(33.77784835309386 -117.95101899783502) bank17193 +17194 POINT(33.09661634627021 -118.24301240335461) bank17194 +17195 POINT(34.0175538308094 -117.44329368204616) bank17195 +17196 POINT(34.67004710097022 -119.04796924821113) bank17196 +17197 POINT(34.959368391909976 -117.40612889220138) bank17197 +17198 POINT(34.83928367394153 -118.03219200863651) bank17198 +17199 POINT(33.61598412987711 -119.21645238555112) bank17199 +17200 POINT(34.87059508565349 -119.08315586160397) bank17200 +17201 POINT(34.150968469776835 -117.9760049183202) bank17201 +17202 POINT(33.16252701805152 -118.84204083041948) bank17202 +17203 POINT(34.73762176522783 -118.19473658543866) bank17203 +17204 POINT(34.463137052626855 -118.00743014887152) bank17204 +17205 POINT(34.08915517052174 -118.2093826197198) bank17205 +17206 POINT(33.8798030342136 -117.7969514609864) bank17206 +17207 POINT(34.71752508856694 -117.26918504715945) bank17207 +17208 POINT(33.727600195233514 -118.04024018371649) bank17208 +17209 POINT(33.96322128562541 -117.4508507438291) bank17209 +17210 POINT(34.64342651818428 -119.08606006475307) bank17210 +17211 POINT(33.62769211078562 -117.76999780993086) bank17211 +17212 POINT(33.942395425422 -117.67464026020815) bank17212 +17213 POINT(33.83611583872223 -118.23233866602833) bank17213 +17214 POINT(33.77423995713001 -117.27953303964884) bank17214 +17215 POINT(33.206625278659835 -118.9103765210453) bank17215 +17216 POINT(33.14655505564055 -119.13425633344778) bank17216 +17217 POINT(34.69830074130527 -118.00175012104577) bank17217 +17218 POINT(34.14815095615014 -118.94035782450382) bank17218 +17219 POINT(34.63757572336901 -118.02593220651198) bank17219 +17220 POINT(33.64616395660317 -118.53823525815389) bank17220 +17221 POINT(33.490860687129484 -118.19315101019669) bank17221 +17222 POINT(34.55623892802941 -117.9295133499276) bank17222 +17223 POINT(33.58930634962745 -118.05965024001263) bank17223 +17224 POINT(33.47523915776726 -117.45163998854402) bank17224 +17225 POINT(33.50464179562049 -117.55889884831183) bank17225 +17226 POINT(34.48782635906226 -117.57134944769243) bank17226 +17227 POINT(34.901214055557524 -118.09104823442095) bank17227 +17228 POINT(34.10542874067227 -118.9304208649317) bank17228 +17229 POINT(35.02372212581449 -118.92519229071038) bank17229 +17230 POINT(34.07607911538386 -119.11620506565279) bank17230 +17231 POINT(34.192306209864185 -117.78842850107996) bank17231 +17232 POINT(33.74075902760426 -117.79195049737773) bank17232 +17233 POINT(33.688358598310494 -119.08007852236292) bank17233 +17234 POINT(34.806334108878154 -118.91265458335904) bank17234 +17235 POINT(33.681384980908895 -117.33788982667058) bank17235 +17236 POINT(33.43321003599378 -118.76966034025139) bank17236 +17237 POINT(33.80480813267606 -117.68535792230765) bank17237 +17238 POINT(34.2869845643679 -117.28927671202553) bank17238 +17239 POINT(34.42858476829902 -118.48140734610847) bank17239 +17240 POINT(34.34456983642884 -118.17297269964003) bank17240 +17241 POINT(34.42486999292014 -118.31016771453605) bank17241 +17242 POINT(33.85849664828161 -118.8273851101046) bank17242 +17243 POINT(34.09592932915576 -117.9652293634715) bank17243 +17244 POINT(34.19923615619504 -118.15141429030434) bank17244 +17245 POINT(33.637307852956084 -117.93235282726069) bank17245 +17246 POINT(34.500345815590435 -118.99431963524752) bank17246 +17247 POINT(34.63686520741859 -118.98052453750695) bank17247 +17248 POINT(34.39434728975984 -118.18739624302675) bank17248 +17249 POINT(35.049024457077074 -117.75153713608677) bank17249 +17250 POINT(34.24980198477284 -118.92191549190653) bank17250 +17251 POINT(33.94760725366431 -118.09230433617235) bank17251 +17252 POINT(34.12384728839405 -117.89636107881573) bank17252 +17253 POINT(33.75200370171854 -118.05102188079512) bank17253 +17254 POINT(34.8122115849261 -119.179961887652) bank17254 +17255 POINT(34.132860654721206 -118.82901981247507) bank17255 +17256 POINT(34.471331323484144 -118.35313323568684) bank17256 +17257 POINT(35.017988259026176 -119.18609188464357) bank17257 +17258 POINT(34.9095048769507 -119.02488402602916) bank17258 +17259 POINT(35.023878077554585 -118.56564590350675) bank17259 +17260 POINT(33.239633595743996 -118.2387428088628) bank17260 +17261 POINT(34.994439812496225 -118.5259104207689) bank17261 +17262 POINT(34.23692804583008 -119.17309807219775) bank17262 +17263 POINT(34.01674225010783 -118.76270964511666) bank17263 +17264 POINT(34.74223066982937 -118.15171019483321) bank17264 +17265 POINT(34.90538772232626 -118.72690702889747) bank17265 +17266 POINT(34.256576407035 -117.70801449054692) bank17266 +17267 POINT(34.7626793372497 -118.02350084935965) bank17267 +17268 POINT(34.241543177331614 -118.31974216046233) bank17268 +17269 POINT(34.03080202632544 -118.3649776062721) bank17269 +17270 POINT(33.636320687973 -117.25918368181128) bank17270 +17271 POINT(34.09285254602895 -119.1294079384541) bank17271 +17272 POINT(35.02876036461249 -118.88533948960824) bank17272 +17273 POINT(34.27898005923438 -117.5756546164191) bank17273 +17274 POINT(34.63344189330179 -117.39938982296667) bank17274 +17275 POINT(34.303638893715735 -118.02033845305844) bank17275 +17276 POINT(34.33988111867517 -119.16441032078293) bank17276 +17277 POINT(34.413955828532934 -118.01013694041671) bank17277 +17278 POINT(34.5027217424912 -119.1440101765984) bank17278 +17279 POINT(33.90325127795583 -118.5872945637433) bank17279 +17280 POINT(34.543763638279 -118.04864880968167) bank17280 +17281 POINT(34.44920010146772 -117.72263843714218) bank17281 +17282 POINT(33.17700687986047 -117.49540135080099) bank17282 +17283 POINT(34.89337058795669 -118.00896634367686) bank17283 +17284 POINT(33.20714873589196 -119.01820544070208) bank17284 +17285 POINT(34.01960459818628 -117.38293542186281) bank17285 +17286 POINT(33.42414448133409 -117.2906383440516) bank17286 +17287 POINT(33.25897779379604 -118.29996578510558) bank17287 +17288 POINT(34.309871834497585 -118.3049258403635) bank17288 +17289 POINT(34.1223437859298 -118.97516346324248) bank17289 +17290 POINT(33.22865794433387 -119.12357870142871) bank17290 +17291 POINT(33.76265375552838 -118.68700046441427) bank17291 +17292 POINT(34.72864092309953 -118.37110623892947) bank17292 +17293 POINT(34.986502201014936 -118.40890672092083) bank17293 +17294 POINT(33.13183620069332 -119.17037234977181) bank17294 +17295 POINT(34.975783465886536 -117.63663798948835) bank17295 +17296 POINT(33.968611410621314 -119.14780779994868) bank17296 +17297 POINT(33.53936267294031 -119.0962271430974) bank17297 +17298 POINT(33.628117600573724 -118.15765103071423) bank17298 +17299 POINT(34.750959190372235 -117.42345038876732) bank17299 +17300 POINT(34.612192736927604 -118.54973071336394) bank17300 +17301 POINT(33.926909269446234 -118.54343192372438) bank17301 +17302 POINT(34.63715720306882 -117.26397320972573) bank17302 +17303 POINT(34.95738812804995 -118.51181822024508) bank17303 +17304 POINT(33.82421190727735 -117.5483192400609) bank17304 +17305 POINT(34.09799099452404 -119.0357623332539) bank17305 +17306 POINT(33.87457450220591 -119.10523017925658) bank17306 +17307 POINT(34.95514829110187 -119.06558933019984) bank17307 +17308 POINT(33.68679426764445 -118.44735907432428) bank17308 +17309 POINT(34.95980424023697 -117.75249430344242) bank17309 +17310 POINT(33.24713636621486 -119.15609917942037) bank17310 +17311 POINT(34.5609662550721 -118.92974042941088) bank17311 +17312 POINT(33.5593369311308 -117.65310273273926) bank17312 +17313 POINT(34.39952027106719 -117.58402937182682) bank17313 +17314 POINT(34.27812719162477 -117.69649727524245) bank17314 +17315 POINT(33.263720604712475 -119.061494281401) bank17315 +17316 POINT(34.0417932453131 -118.32425521570185) bank17316 +17317 POINT(33.14715299723657 -117.90998855947458) bank17317 +17318 POINT(34.780755921401116 -118.02315389006613) bank17318 +17319 POINT(34.67958691840063 -119.11310052574935) bank17319 +17320 POINT(34.006678421103516 -119.21871981372867) bank17320 +17321 POINT(33.779485533109785 -118.59809155381133) bank17321 +17322 POINT(34.712294206221564 -117.8121818493498) bank17322 +17323 POINT(34.774645483029644 -118.56563155583228) bank17323 +17324 POINT(34.81668038277505 -117.43530541298608) bank17324 +17325 POINT(33.93314680626909 -118.21165899791886) bank17325 +17326 POINT(35.021914480957754 -118.66837676738955) bank17326 +17327 POINT(33.405754788662584 -117.64059766716301) bank17327 +17328 POINT(34.45997442652226 -119.17548365885213) bank17328 +17329 POINT(33.302739860844454 -118.22986595827876) bank17329 +17330 POINT(33.448825034050856 -117.90232651668616) bank17330 +17331 POINT(33.49026972619755 -117.43528702170951) bank17331 +17332 POINT(34.38652396611747 -118.39528157901348) bank17332 +17333 POINT(33.27578613977274 -118.03146305453035) bank17333 +17334 POINT(34.578505575495846 -119.06198067133711) bank17334 +17335 POINT(34.24743253342184 -119.03317661697557) bank17335 +17336 POINT(35.02580242278674 -118.55481369587065) bank17336 +17337 POINT(33.27477356545169 -118.34136217409014) bank17337 +17338 POINT(34.48026833839463 -118.3200184843546) bank17338 +17339 POINT(34.27017271099148 -117.80160391013303) bank17339 +17340 POINT(33.85187007225871 -118.98970818523942) bank17340 +17341 POINT(33.90238673937673 -117.74169492599927) bank17341 +17342 POINT(34.50528288393805 -117.69264503284522) bank17342 +17343 POINT(34.06706762213825 -118.90703544524193) bank17343 +17344 POINT(33.37431772957824 -117.41238805380623) bank17344 +17345 POINT(33.98417061540711 -118.26230519781006) bank17345 +17346 POINT(34.09507327471808 -118.43701107954162) bank17346 +17347 POINT(33.78160693280438 -118.43550580372663) bank17347 +17348 POINT(33.40889956002807 -118.67899880392898) bank17348 +17349 POINT(34.821624948209184 -118.11227744543217) bank17349 +17350 POINT(34.442675356621365 -118.47876090375185) bank17350 +17351 POINT(33.643075867665694 -118.86545816287841) bank17351 +17352 POINT(34.68317351885421 -118.14032584497973) bank17352 +17353 POINT(33.1905396254911 -118.29403516281366) bank17353 +17354 POINT(34.18739345673889 -118.78465803453368) bank17354 +17355 POINT(34.452807453252014 -119.0365972363043) bank17355 +17356 POINT(33.75060437486403 -117.91667777070626) bank17356 +17357 POINT(34.22948166964055 -118.03112049361808) bank17357 +17358 POINT(34.28123526735062 -117.3142281443235) bank17358 +17359 POINT(34.30370864448187 -118.64444101854068) bank17359 +17360 POINT(33.45670493156724 -119.19847745119056) bank17360 +17361 POINT(34.84047208266345 -118.363964614009) bank17361 +17362 POINT(34.12385827155538 -118.74182790969186) bank17362 +17363 POINT(33.79334556435416 -118.11506354552193) bank17363 +17364 POINT(33.27428514727614 -118.70429239177699) bank17364 +17365 POINT(33.24933279724133 -117.44456877622855) bank17365 +17366 POINT(33.652252899134204 -119.1150488012308) bank17366 +17367 POINT(33.78297505343994 -117.50518948902604) bank17367 +17368 POINT(33.283782640651346 -117.60998248902365) bank17368 +17369 POINT(34.400050282418924 -117.88997753393218) bank17369 +17370 POINT(34.27325391051355 -117.53938654378992) bank17370 +17371 POINT(34.97134192701451 -117.95680800499936) bank17371 +17372 POINT(34.74030897782627 -117.70626908887971) bank17372 +17373 POINT(33.06258088958624 -118.96178833140314) bank17373 +17374 POINT(34.683732201046546 -119.03753251975192) bank17374 +17375 POINT(33.82950465037102 -119.089943415544) bank17375 +17376 POINT(33.80156211974052 -118.64449401859147) bank17376 +17377 POINT(33.137970532548735 -117.74943013010132) bank17377 +17378 POINT(35.04661745559032 -117.87015306604336) bank17378 +17379 POINT(35.044459465547106 -118.89512811158423) bank17379 +17380 POINT(33.123387869166024 -118.52493677104387) bank17380 +17381 POINT(33.825391024530255 -118.4546335411612) bank17381 +17382 POINT(34.919877011429975 -118.43585978659594) bank17382 +17383 POINT(34.84531029127993 -118.95978749130579) bank17383 +17384 POINT(34.224337050388705 -118.0391847936119) bank17384 +17385 POINT(33.77321700800342 -119.01284846480526) bank17385 +17386 POINT(33.86616730715803 -118.10548570500264) bank17386 +17387 POINT(34.69924121923619 -118.51779234447903) bank17387 +17388 POINT(34.88933561884128 -118.52044806199375) bank17388 +17389 POINT(35.04347117729297 -118.16925111547161) bank17389 +17390 POINT(33.51976698547974 -118.2583341492749) bank17390 +17391 POINT(33.05970103049914 -117.45168382216357) bank17391 +17392 POINT(33.338609552007085 -118.43073772597648) bank17392 +17393 POINT(33.47293143918087 -118.19118041815202) bank17393 +17394 POINT(34.843053898276565 -117.49487092136735) bank17394 +17395 POINT(33.248009474770754 -118.18121683736885) bank17395 +17396 POINT(33.10179525084176 -117.89756465956823) bank17396 +17397 POINT(34.8402329545863 -117.34587476975616) bank17397 +17398 POINT(34.44291849701852 -118.13403566407656) bank17398 +17399 POINT(33.89457982292207 -118.8837679583999) bank17399 +17400 POINT(33.5314683456726 -117.82530070223554) bank17400 +17401 POINT(34.14357818206938 -117.98680923590499) bank17401 +17402 POINT(33.07945659788005 -117.87706817369292) bank17402 +17403 POINT(33.28811893113052 -117.38754420248875) bank17403 +17404 POINT(33.59703241887444 -118.95242231665944) bank17404 +17405 POINT(34.98122087196596 -117.6209080359232) bank17405 +17406 POINT(34.44929082209573 -117.54648277547483) bank17406 +17407 POINT(33.10224416273182 -117.58141647677003) bank17407 +17408 POINT(33.83575165400316 -118.64649365520205) bank17408 +17409 POINT(34.345297691919875 -118.53557590114968) bank17409 +17410 POINT(34.026308423964906 -117.83846490741048) bank17410 +17411 POINT(33.74602301810406 -117.95478612809104) bank17411 +17412 POINT(34.595533406179456 -118.54676979321547) bank17412 +17413 POINT(33.43423095716172 -118.10561782891968) bank17413 +17414 POINT(33.89599937230093 -118.77249067792366) bank17414 +17415 POINT(33.47236691191648 -118.4873398012694) bank17415 +17416 POINT(33.70741910936632 -117.31719163830657) bank17416 +17417 POINT(33.10920898986239 -118.7044227601194) bank17417 +17418 POINT(33.249043972544925 -118.8521788869889) bank17418 +17419 POINT(34.530055838371645 -117.46765109860105) bank17419 +17420 POINT(33.89949737740235 -118.22767555899485) bank17420 +17421 POINT(33.80122716025827 -118.82596807436533) bank17421 +17422 POINT(33.05974661101373 -118.75724797360438) bank17422 +17423 POINT(34.20111161176673 -118.50166424142489) bank17423 +17424 POINT(34.04800432611921 -117.67480787255303) bank17424 +17425 POINT(33.8958056424637 -117.66997083467713) bank17425 +17426 POINT(33.93155288693269 -118.28346031315644) bank17426 +17427 POINT(33.56550950152086 -118.34596905111955) bank17427 +17428 POINT(33.71939562780066 -117.61789600993863) bank17428 +17429 POINT(33.571573604420145 -118.06384342664616) bank17429 +17430 POINT(34.806526821278844 -117.77879546383734) bank17430 +17431 POINT(34.173539243513055 -117.74349013934201) bank17431 +17432 POINT(34.660789402502175 -118.02430678607878) bank17432 +17433 POINT(34.939644225075305 -117.94668860716432) bank17433 +17434 POINT(34.71544264027657 -118.52022462234248) bank17434 +17435 POINT(33.56523741877194 -117.56262439495728) bank17435 +17436 POINT(33.847763229888976 -117.85940639176964) bank17436 +17437 POINT(34.34967274592142 -117.78867836195965) bank17437 +17438 POINT(33.994082919200295 -117.83423876447694) bank17438 +17439 POINT(34.7859917547902 -117.75393710943351) bank17439 +17440 POINT(34.38252814722575 -119.2345554218543) bank17440 +17441 POINT(33.62317245732915 -118.0508268142508) bank17441 +17442 POINT(33.376499242579754 -118.97652426828802) bank17442 +17443 POINT(35.04920125886602 -117.64440525557023) bank17443 +17444 POINT(34.23804050200671 -118.24083872743097) bank17444 +17445 POINT(34.49401887211305 -117.81046475513104) bank17445 +17446 POINT(35.02428319031325 -118.93473074669758) bank17446 +17447 POINT(34.66762017863127 -118.41860902352434) bank17447 +17448 POINT(34.243104840874125 -118.67440597537869) bank17448 +17449 POINT(34.79571425959477 -117.36588248491029) bank17449 +17450 POINT(33.62924151318929 -119.2261130341382) bank17450 +17451 POINT(33.619945066556106 -117.49696363763532) bank17451 +17452 POINT(33.31087719158127 -118.34655268764872) bank17452 +17453 POINT(34.57271166079748 -119.20056576305335) bank17453 +17454 POINT(33.66168490036534 -118.435429548404) bank17454 +17455 POINT(33.28772300482841 -118.14191818673298) bank17455 +17456 POINT(33.71523452071004 -119.01759696410673) bank17456 +17457 POINT(33.177761628520265 -117.62848296494889) bank17457 +17458 POINT(34.89470750541267 -119.05946966901853) bank17458 +17459 POINT(34.732551710499514 -118.4630306223955) bank17459 +17460 POINT(34.5427147535124 -119.1713962798361) bank17460 +17461 POINT(34.097455131837826 -118.47209015433111) bank17461 +17462 POINT(33.949798483350676 -117.75882133884187) bank17462 +17463 POINT(33.65628451104717 -118.68748612808669) bank17463 +17464 POINT(33.24264013323342 -119.22635553907078) bank17464 +17465 POINT(33.08522585569641 -117.32767637031344) bank17465 +17466 POINT(34.15432143330556 -118.38697723749165) bank17466 +17467 POINT(33.20770150651185 -117.42959943855863) bank17467 +17468 POINT(34.73812917895069 -117.30297156673275) bank17468 +17469 POINT(35.037666472808255 -118.98992850277334) bank17469 +17470 POINT(34.88872916700455 -117.99470271286025) bank17470 +17471 POINT(33.80259424126901 -117.80395876702) bank17471 +17472 POINT(33.765480642854335 -119.05862119649132) bank17472 +17473 POINT(34.4095781965487 -117.37838528990525) bank17473 +17474 POINT(34.069971089963666 -117.42056926610348) bank17474 +17475 POINT(34.33326547303598 -118.3347040600993) bank17475 +17476 POINT(33.07848954282958 -117.72155684852802) bank17476 +17477 POINT(34.6402333210652 -118.24055783887593) bank17477 +17478 POINT(34.72680898434016 -118.01736821974492) bank17478 +17479 POINT(33.81059365049225 -118.10474377916861) bank17479 +17480 POINT(34.61093163049555 -117.79216460632952) bank17480 +17481 POINT(33.787122705828764 -117.85407157732358) bank17481 +17482 POINT(34.10841549951243 -117.279284276438) bank17482 +17483 POINT(34.09861264309822 -117.62408520400461) bank17483 +17484 POINT(33.94920110485965 -118.29994021418128) bank17484 +17485 POINT(33.18617563808808 -117.36732572724357) bank17485 +17486 POINT(33.23831907852125 -118.89067783948872) bank17486 +17487 POINT(33.28569348301081 -118.4211099326764) bank17487 +17488 POINT(33.206940009245784 -117.88210621285343) bank17488 +17489 POINT(33.42515916914525 -117.64874900504962) bank17489 +17490 POINT(33.42342460815402 -118.40201796105436) bank17490 +17491 POINT(33.46907539674373 -118.51572849002962) bank17491 +17492 POINT(33.24554587864899 -117.4481269666544) bank17492 +17493 POINT(34.02637974630899 -117.29481792841354) bank17493 +17494 POINT(33.56502454672723 -119.10539751378123) bank17494 +17495 POINT(33.72173954449295 -117.94475890067444) bank17495 +17496 POINT(33.707657202266546 -118.9977059405577) bank17496 +17497 POINT(34.645180495862775 -118.64841786531386) bank17497 +17498 POINT(33.87932450394934 -117.81168220650812) bank17498 +17499 POINT(34.27126322420407 -117.74918872763087) bank17499 +17500 POINT(35.027892645656294 -117.56295901494276) bank17500 +17501 POINT(33.41448200336616 -118.30633541475031) bank17501 +17502 POINT(33.65861220280915 -117.24802470054055) bank17502 +17503 POINT(34.758410077768374 -118.20403660344518) bank17503 +17504 POINT(34.7464017828998 -118.79705059522738) bank17504 +17505 POINT(33.57361080377475 -118.62779807184229) bank17505 +17506 POINT(34.61272075849099 -118.3489185909319) bank17506 +17507 POINT(33.21644284760677 -117.84460325021784) bank17507 +17508 POINT(33.81698225347849 -118.8150326226128) bank17508 +17509 POINT(33.84242040980362 -117.9396886329019) bank17509 +17510 POINT(33.6539283106451 -117.63878736806144) bank17510 +17511 POINT(34.11196229796393 -118.41749855612287) bank17511 +17512 POINT(34.504754646024416 -118.16013111588347) bank17512 +17513 POINT(34.50183218367348 -117.74965759148384) bank17513 +17514 POINT(34.569753156402534 -118.67469942820385) bank17514 +17515 POINT(33.10029547937659 -118.95364552251687) bank17515 +17516 POINT(34.25159517536693 -118.38544943647727) bank17516 +17517 POINT(33.88979365121888 -117.47102477051556) bank17517 +17518 POINT(34.165631749083644 -118.55375839420081) bank17518 +17519 POINT(34.800409669880686 -118.57782893101943) bank17519 +17520 POINT(34.93739582933007 -118.43934736494816) bank17520 +17521 POINT(34.67153805911166 -117.5950713032985) bank17521 +17522 POINT(33.89462999428595 -118.65990128453097) bank17522 +17523 POINT(33.2037363730957 -118.3598806907425) bank17523 +17524 POINT(34.298324879060786 -117.89321957341973) bank17524 +17525 POINT(34.696448732479865 -117.99581977990788) bank17525 +17526 POINT(34.34504034151 -118.12692957263141) bank17526 +17527 POINT(33.10189386697603 -118.61585435991225) bank17527 +17528 POINT(34.57401660056841 -119.22492810088276) bank17528 +17529 POINT(33.981457185974584 -118.49560024659456) bank17529 +17530 POINT(33.52090584179868 -118.00865126933812) bank17530 +17531 POINT(33.873481300453214 -119.09115945771661) bank17531 +17532 POINT(33.551693518636014 -118.4534927866868) bank17532 +17533 POINT(33.19746112089517 -117.99904662701543) bank17533 +17534 POINT(33.57351608812957 -118.60721828282894) bank17534 +17535 POINT(33.31931286323618 -119.08932540388037) bank17535 +17536 POINT(34.955748879559366 -117.29127881387451) bank17536 +17537 POINT(33.39981017477084 -117.40165622266963) bank17537 +17538 POINT(34.514054512216084 -118.08454470268696) bank17538 +17539 POINT(34.22579081304656 -117.54679770196908) bank17539 +17540 POINT(33.257143665864625 -118.57336262326615) bank17540 +17541 POINT(34.38446191213098 -118.83532924886613) bank17541 +17542 POINT(34.05405211305939 -118.78464381856958) bank17542 +17543 POINT(34.5287891743196 -117.70550009487066) bank17543 +17544 POINT(34.66716873186914 -118.66742096654782) bank17544 +17545 POINT(33.76665904704785 -118.65324115315057) bank17545 +17546 POINT(34.83819480728559 -118.86527028089638) bank17546 +17547 POINT(34.8246485192087 -118.64679175586332) bank17547 +17548 POINT(34.3138552807349 -117.75207584440957) bank17548 +17549 POINT(33.2378383714109 -118.76500716741671) bank17549 +17550 POINT(34.80820416161974 -118.87784006260927) bank17550 +17551 POINT(34.01824696737084 -118.38148473646837) bank17551 +17552 POINT(33.08571462308417 -119.10186974549727) bank17552 +17553 POINT(33.260598199651085 -118.63163395489788) bank17553 +17554 POINT(33.510590832535236 -118.4913714278574) bank17554 +17555 POINT(33.65297152973325 -117.8525082569999) bank17555 +17556 POINT(33.761627829273216 -118.32047553147865) bank17556 +17557 POINT(34.69765249994939 -117.49185608117227) bank17557 +17558 POINT(33.45218658080915 -118.84633277772166) bank17558 +17559 POINT(33.323084010562894 -117.55964416334331) bank17559 +17560 POINT(34.127228585726066 -117.34022106832745) bank17560 +17561 POINT(33.66575779449973 -117.35452291775685) bank17561 +17562 POINT(34.817774987594696 -118.66994105988978) bank17562 +17563 POINT(33.126819054153955 -118.82549239208099) bank17563 +17564 POINT(34.35277770078022 -118.95983859524803) bank17564 +17565 POINT(34.64942656464507 -118.52889216500712) bank17565 +17566 POINT(33.798583674127066 -118.38266565642942) bank17566 +17567 POINT(33.85078018010183 -117.71515282119253) bank17567 +17568 POINT(34.92598067491807 -118.22974522516485) bank17568 +17569 POINT(34.14604343905928 -117.77727282522963) bank17569 +17570 POINT(33.55925663695321 -117.7467008954527) bank17570 +17571 POINT(34.34051392768091 -117.38368602101221) bank17571 +17572 POINT(33.84869721254198 -117.45722838776614) bank17572 +17573 POINT(33.79240445776193 -118.79415022097663) bank17573 +17574 POINT(34.39725547812094 -119.15735476329132) bank17574 +17575 POINT(33.11582326330984 -117.91596125899186) bank17575 +17576 POINT(33.195158960260144 -118.13401889783682) bank17576 +17577 POINT(35.00378130863821 -117.4024943288881) bank17577 +17578 POINT(33.396319809935676 -117.27450779577403) bank17578 +17579 POINT(33.35595599667414 -118.41302794794646) bank17579 +17580 POINT(34.83552164956734 -118.59355860316012) bank17580 +17581 POINT(34.147377450933504 -117.75163321856239) bank17581 +17582 POINT(33.803439405896704 -117.33787690303669) bank17582 +17583 POINT(33.25831180753791 -117.83002403563177) bank17583 +17584 POINT(33.23372512332516 -117.72300133342202) bank17584 +17585 POINT(34.93399162096334 -117.92609919264991) bank17585 +17586 POINT(33.91346506018621 -117.48110373397247) bank17586 +17587 POINT(33.22110291265249 -117.37434660380666) bank17587 +17588 POINT(33.57303230387578 -118.50781185927738) bank17588 +17589 POINT(34.12884794213427 -119.1744389614648) bank17589 +17590 POINT(33.209924526952356 -117.99998677240218) bank17590 +17591 POINT(34.60325402503995 -118.22954150778607) bank17591 +17592 POINT(33.353052782613496 -118.35502238692041) bank17592 +17593 POINT(34.13679684538069 -118.6221161021493) bank17593 +17594 POINT(33.94137754199195 -117.69432050457334) bank17594 +17595 POINT(33.90619636499237 -118.15060413369305) bank17595 +17596 POINT(35.028676250727315 -118.78018000968123) bank17596 +17597 POINT(34.96770269153664 -117.75954782668475) bank17597 +17598 POINT(34.01659792830603 -118.71332165321246) bank17598 +17599 POINT(33.405852550699265 -117.48275068186723) bank17599 +17600 POINT(34.025378877576074 -118.81442275828702) bank17600 +17601 POINT(33.546202231099365 -117.57650499698609) bank17601 +17602 POINT(33.35950655345764 -119.08702408723097) bank17602 +17603 POINT(34.27318298502556 -117.51103826531863) bank17603 +17604 POINT(33.42447267374443 -118.40883435819447) bank17604 +17605 POINT(34.1049787259396 -118.09891404924105) bank17605 +17606 POINT(34.36604184830491 -118.55744093098123) bank17606 +17607 POINT(33.74447972804478 -117.65832204586172) bank17607 +17608 POINT(34.39443243584768 -118.7790659574337) bank17608 +17609 POINT(34.66631100022938 -118.86452709298699) bank17609 +17610 POINT(34.91516676915557 -119.07998549405512) bank17610 +17611 POINT(34.995733312819944 -118.3033071678659) bank17611 +17612 POINT(34.46548990975399 -119.15504805375829) bank17612 +17613 POINT(33.100982368975934 -118.24911247301662) bank17613 +17614 POINT(33.58205608618207 -117.7451894458409) bank17614 +17615 POINT(33.672340794207145 -118.13513044710567) bank17615 +17616 POINT(34.28718114267068 -118.26863451294614) bank17616 +17617 POINT(34.93191822069648 -117.62064553590857) bank17617 +17618 POINT(33.56189745153429 -118.28028459618314) bank17618 +17619 POINT(34.99355779276454 -117.30199841583487) bank17619 +17620 POINT(33.26165478895746 -118.02277512830126) bank17620 +17621 POINT(34.66349697599276 -119.23458082567309) bank17621 +17622 POINT(33.071522263583816 -118.95255740374792) bank17622 +17623 POINT(33.5094607095723 -118.05508759933629) bank17623 +17624 POINT(33.583260258913995 -118.28662758578115) bank17624 +17625 POINT(35.034027215172664 -117.50435661231933) bank17625 +17626 POINT(34.10531876077278 -118.99554731982428) bank17626 +17627 POINT(34.10730535460828 -118.94267539317723) bank17627 +17628 POINT(34.461086319257305 -117.99321748330388) bank17628 +17629 POINT(34.18208268122069 -118.33967017744934) bank17629 +17630 POINT(33.798278861391495 -117.45336486065277) bank17630 +17631 POINT(33.402523903835075 -118.63662759362356) bank17631 +17632 POINT(33.96962412003843 -118.70779328432408) bank17632 +17633 POINT(34.57358221236958 -118.8137833505869) bank17633 +17634 POINT(33.15138727990385 -118.1990463575856) bank17634 +17635 POINT(33.95170668626565 -119.06631964128665) bank17635 +17636 POINT(34.38553732448866 -119.13983517505622) bank17636 +17637 POINT(34.100346612396216 -118.60974102686673) bank17637 +17638 POINT(34.55848060045724 -118.09685128659292) bank17638 +17639 POINT(33.63797503677018 -117.8916214178731) bank17639 +17640 POINT(34.55930368189375 -117.86149441745687) bank17640 +17641 POINT(33.448561217454355 -118.4190927970654) bank17641 +17642 POINT(33.227538913730115 -117.69733851947842) bank17642 +17643 POINT(34.87818188568614 -118.59078862731933) bank17643 +17644 POINT(33.41488677142089 -118.0587005055335) bank17644 +17645 POINT(34.52052190206906 -118.14081079342834) bank17645 +17646 POINT(34.26117157467304 -118.75416939116786) bank17646 +17647 POINT(34.66741164178655 -118.15878172204891) bank17647 +17648 POINT(34.72974454757708 -118.02674613923344) bank17648 +17649 POINT(34.27234983067558 -118.79014105089577) bank17649 +17650 POINT(34.347129217981774 -117.81769243406404) bank17650 +17651 POINT(33.16899856856448 -119.1809479427306) bank17651 +17652 POINT(33.76698252975495 -117.79277965046107) bank17652 +17653 POINT(33.6191944843123 -119.06696625723934) bank17653 +17654 POINT(34.74819320125815 -117.2892854424778) bank17654 +17655 POINT(34.90198961689541 -117.69252500308558) bank17655 +17656 POINT(33.80362671038097 -118.52806723827742) bank17656 +17657 POINT(33.49180668376532 -118.08045759442226) bank17657 +17658 POINT(34.26300784660295 -118.03011416425082) bank17658 +17659 POINT(34.689587267200544 -119.22566136241774) bank17659 +17660 POINT(34.36030664939258 -118.68024412798819) bank17660 +17661 POINT(34.65883694891289 -117.95257253395) bank17661 +17662 POINT(33.459773007046095 -117.68062846173342) bank17662 +17663 POINT(34.684116349979156 -118.73059382527285) bank17663 +17664 POINT(34.14293838007391 -117.73439919405187) bank17664 +17665 POINT(34.6315014217808 -119.22890200334405) bank17665 +17666 POINT(34.73740305433525 -118.95034596893564) bank17666 +17667 POINT(34.40333849690015 -118.64097455278029) bank17667 +17668 POINT(34.96809639979765 -118.80222920286988) bank17668 +17669 POINT(34.47203028807498 -119.09925660943239) bank17669 +17670 POINT(33.955294494997226 -117.6462346424881) bank17670 +17671 POINT(33.69379169299122 -118.3748221173384) bank17671 +17672 POINT(33.640468008019006 -117.68642189210433) bank17672 +17673 POINT(34.148357752500836 -117.95524469096935) bank17673 +17674 POINT(33.5971692269521 -117.41829564790847) bank17674 +17675 POINT(34.40496944209674 -117.3642940326539) bank17675 +17676 POINT(34.29207884146928 -117.48039922709107) bank17676 +17677 POINT(33.308856895716275 -118.5091109277133) bank17677 +17678 POINT(33.758335474603534 -118.91196457333797) bank17678 +17679 POINT(33.06129926004601 -118.75796469017001) bank17679 +17680 POINT(34.38108886950227 -117.86667683702478) bank17680 +17681 POINT(33.16982734532588 -119.00958504510461) bank17681 +17682 POINT(34.514929725285725 -117.99681034904208) bank17682 +17683 POINT(34.51802108784265 -117.27735580844981) bank17683 +17684 POINT(34.52472857858928 -118.5292577540642) bank17684 +17685 POINT(33.30566203708949 -119.15548008222096) bank17685 +17686 POINT(34.11621462160258 -117.61219707456954) bank17686 +17687 POINT(35.02495465285829 -118.90069323148396) bank17687 +17688 POINT(34.17264721482223 -118.97656625908444) bank17688 +17689 POINT(34.125513305310015 -117.44116358836469) bank17689 +17690 POINT(34.691233097836125 -118.19660608373009) bank17690 +17691 POINT(33.173805008533414 -118.38508338067147) bank17691 +17692 POINT(33.89618753788438 -118.04999878879315) bank17692 +17693 POINT(34.725541373042354 -117.7046637040536) bank17693 +17694 POINT(34.40765067827407 -118.37969979016638) bank17694 +17695 POINT(33.83706478679496 -118.56466266100445) bank17695 +17696 POINT(33.17794901529936 -118.27583434182954) bank17696 +17697 POINT(33.22690134406363 -117.79493001924166) bank17697 +17698 POINT(33.23898887083696 -117.6861589759481) bank17698 +17699 POINT(33.71880118216386 -117.62642669302217) bank17699 +17700 POINT(34.62161028985223 -118.32865397919096) bank17700 +17701 POINT(33.80145831167994 -117.78534676082198) bank17701 +17702 POINT(33.53955050395287 -117.89876870123611) bank17702 +17703 POINT(33.445287194942686 -119.14427156447302) bank17703 +17704 POINT(34.02143282813529 -119.0356325847087) bank17704 +17705 POINT(33.561476445992625 -118.34544940399307) bank17705 +17706 POINT(33.876190879387536 -117.54782271164005) bank17706 +17707 POINT(33.58973677184609 -118.92038615547807) bank17707 +17708 POINT(34.62445174874223 -117.94481975507134) bank17708 +17709 POINT(33.666986121120395 -118.51383195832562) bank17709 +17710 POINT(33.94005098928121 -117.44325795895294) bank17710 +17711 POINT(33.843064676309865 -117.48104323274016) bank17711 +17712 POINT(34.12481185353572 -117.42113422297602) bank17712 +17713 POINT(33.74462164504243 -117.88942682271245) bank17713 +17714 POINT(34.33354349230891 -118.68875788824276) bank17714 +17715 POINT(33.37164045766133 -118.92826932926077) bank17715 +17716 POINT(34.172389857510105 -119.19291062946915) bank17716 +17717 POINT(34.878477682857984 -118.13358836537071) bank17717 +17718 POINT(33.07218004130491 -118.90716249739029) bank17718 +17719 POINT(33.122360893816506 -118.87979857580437) bank17719 +17720 POINT(34.58118096134184 -117.66607102263642) bank17720 +17721 POINT(34.291756046444625 -118.44261356849036) bank17721 +17722 POINT(34.41181799888965 -118.86472892540043) bank17722 +17723 POINT(33.854871519309 -118.05451161956309) bank17723 +17724 POINT(33.29382516276095 -117.35848690277444) bank17724 +17725 POINT(33.733121196761566 -117.91135378603148) bank17725 +17726 POINT(34.2149349053521 -117.48582846632745) bank17726 +17727 POINT(33.94981990083041 -117.48435582439969) bank17727 +17728 POINT(33.70682189079966 -119.03625395809111) bank17728 +17729 POINT(34.56844631318681 -118.09113225992333) bank17729 +17730 POINT(33.641919804295256 -118.53104651192898) bank17730 +17731 POINT(33.764934740076434 -118.77250726272624) bank17731 +17732 POINT(34.83957161250898 -118.0308652098575) bank17732 +17733 POINT(34.060337991744994 -119.21264356137694) bank17733 +17734 POINT(34.400416824165596 -118.72312508708383) bank17734 +17735 POINT(34.75604018112308 -117.75089661459414) bank17735 +17736 POINT(33.321837052706876 -118.43981298486618) bank17736 +17737 POINT(34.82653867116965 -118.92268844331966) bank17737 +17738 POINT(34.923125504411715 -119.04043894530632) bank17738 +17739 POINT(33.20012044996683 -118.31485820956136) bank17739 +17740 POINT(33.69629463057253 -117.79466259189928) bank17740 +17741 POINT(33.66221714859375 -118.38449431228335) bank17741 +17742 POINT(34.76517448557287 -118.11382549670073) bank17742 +17743 POINT(33.396066672154255 -118.7998133285475) bank17743 +17744 POINT(34.157622912891675 -117.82790864023977) bank17744 +17745 POINT(34.57437658979183 -117.75283178665163) bank17745 +17746 POINT(33.88824051450989 -117.2455726638327) bank17746 +17747 POINT(33.796913935507696 -117.5002993851944) bank17747 +17748 POINT(33.79907811725106 -118.13092836805649) bank17748 +17749 POINT(33.72911676765998 -118.18784709560124) bank17749 +17750 POINT(34.53807190475541 -119.02020504397454) bank17750 +17751 POINT(33.899008407445436 -118.23537391430442) bank17751 +17752 POINT(33.33183808821042 -118.87769210772629) bank17752 +17753 POINT(33.29209946370942 -118.2048919822878) bank17753 +17754 POINT(34.99102519059978 -118.28398998257558) bank17754 +17755 POINT(33.05956236615256 -118.41935184051593) bank17755 +17756 POINT(34.09054667592607 -117.56833149848102) bank17756 +17757 POINT(35.04514738921944 -117.45232010036379) bank17757 +17758 POINT(34.666719079372434 -117.48347699565011) bank17758 +17759 POINT(33.7718632451953 -118.55359634766917) bank17759 +17760 POINT(34.0803965219994 -119.19465938385588) bank17760 +17761 POINT(34.128695516528914 -118.12448112517198) bank17761 +17762 POINT(33.843844758195196 -118.9704757454096) bank17762 +17763 POINT(34.92778106377621 -117.89079502381709) bank17763 +17764 POINT(34.250893632124104 -117.47105134751759) bank17764 +17765 POINT(33.48453053259745 -117.7966405383624) bank17765 +17766 POINT(34.66289794739099 -117.80147606003256) bank17766 +17767 POINT(33.530972047446106 -118.86761179171451) bank17767 +17768 POINT(34.01030781042109 -117.7046640318164) bank17768 +17769 POINT(33.646876570183544 -118.05410526550257) bank17769 +17770 POINT(34.839576648365686 -117.9481564184942) bank17770 +17771 POINT(34.523965851898105 -118.84840905736726) bank17771 +17772 POINT(33.649586188745346 -118.15734892439392) bank17772 +17773 POINT(34.2790523499755 -118.04484884052378) bank17773 +17774 POINT(33.52110734075593 -118.08369195647676) bank17774 +17775 POINT(34.41337323521306 -117.75962604595378) bank17775 +17776 POINT(33.589629931747396 -118.12088572704636) bank17776 +17777 POINT(34.94167663904525 -118.31664616928116) bank17777 +17778 POINT(33.439976917049485 -118.63428067252407) bank17778 +17779 POINT(34.52995584481255 -117.70869543371788) bank17779 +17780 POINT(34.85270864491289 -118.52374936816204) bank17780 +17781 POINT(34.25228914265709 -117.4059291251396) bank17781 +17782 POINT(34.91185438740916 -117.3087293277228) bank17782 +17783 POINT(33.31825429211437 -117.30667759837772) bank17783 +17784 POINT(33.42205544638572 -119.16587251795009) bank17784 +17785 POINT(34.77196297641812 -117.94109131094015) bank17785 +17786 POINT(34.85918141438226 -118.21787230479518) bank17786 +17787 POINT(34.59567117841005 -118.10374825292703) bank17787 +17788 POINT(33.84300287117934 -117.5043377034702) bank17788 +17789 POINT(34.739674731569934 -118.30676856934534) bank17789 +17790 POINT(35.01941768506134 -118.11471381410114) bank17790 +17791 POINT(33.48636954096855 -118.7824944724284) bank17791 +17792 POINT(34.027837779146246 -118.76760717276665) bank17792 +17793 POINT(33.7320052651931 -118.93032543768001) bank17793 +17794 POINT(34.404113235890186 -118.70173464013251) bank17794 +17795 POINT(34.72435429499423 -117.8525538285125) bank17795 +17796 POINT(33.632292253740644 -118.60802849771507) bank17796 +17797 POINT(33.577440113783524 -117.84714716982441) bank17797 +17798 POINT(33.998875599030946 -118.73901154400089) bank17798 +17799 POINT(34.19106582209159 -117.76458660723797) bank17799 +17800 POINT(33.11543338260015 -119.01750269634077) bank17800 +17801 POINT(33.84316789384058 -117.32349246594906) bank17801 +17802 POINT(34.96647936026892 -118.46135396060731) bank17802 +17803 POINT(33.61461422812491 -117.76559731836804) bank17803 +17804 POINT(34.93955038403494 -117.46480683434247) bank17804 +17805 POINT(33.89374038423879 -117.92703693889236) bank17805 +17806 POINT(34.171785632455 -118.39285802164379) bank17806 +17807 POINT(33.71853306053804 -118.36282527891107) bank17807 +17808 POINT(33.6523128968867 -117.2813223794229) bank17808 +17809 POINT(33.87142555519636 -118.0270340443492) bank17809 +17810 POINT(35.047161124226825 -117.47126403934304) bank17810 +17811 POINT(34.229476855444226 -118.65745791858991) bank17811 +17812 POINT(33.93063468962341 -119.15807898124467) bank17812 +17813 POINT(33.77055708580093 -117.32006152017652) bank17813 +17814 POINT(34.76348117994568 -117.63217900940326) bank17814 +17815 POINT(34.141087788389925 -117.40457006135898) bank17815 +17816 POINT(33.33488182044854 -117.44719480402588) bank17816 +17817 POINT(33.18414718848432 -117.33357017160522) bank17817 +17818 POINT(34.09341516864547 -117.58822998699495) bank17818 +17819 POINT(34.53848080671722 -117.7904466803508) bank17819 +17820 POINT(33.714249401719684 -118.66830725012454) bank17820 +17821 POINT(34.43119053784174 -118.71634330262506) bank17821 +17822 POINT(33.67123376853988 -118.5232019331526) bank17822 +17823 POINT(34.67191436065143 -118.70380280587318) bank17823 +17824 POINT(34.19537877239473 -118.62284469197103) bank17824 +17825 POINT(34.3150810913478 -117.95413868945083) bank17825 +17826 POINT(33.37785987623777 -118.9102360994167) bank17826 +17827 POINT(33.16966384254568 -117.6994066999629) bank17827 +17828 POINT(33.13855065107039 -117.91097168268004) bank17828 +17829 POINT(34.24521445345808 -117.91068266769521) bank17829 +17830 POINT(33.36319157343806 -117.44018194453535) bank17830 +17831 POINT(35.03262035918007 -118.783444105633) bank17831 +17832 POINT(35.02503206512973 -118.11076819675587) bank17832 +17833 POINT(33.89097623351596 -118.43056861321487) bank17833 +17834 POINT(34.25366345703383 -117.53950031021476) bank17834 +17835 POINT(34.1283885682007 -118.35321033185375) bank17835 +17836 POINT(34.730700233275 -118.9863754576595) bank17836 +17837 POINT(33.81895517671801 -117.33706729606438) bank17837 +17838 POINT(34.81029226155673 -117.74805452087328) bank17838 +17839 POINT(33.25166856755711 -117.8437876086157) bank17839 +17840 POINT(34.767567745369185 -117.6949814134454) bank17840 +17841 POINT(34.587939814429056 -119.05222472146566) bank17841 +17842 POINT(34.01644427051108 -118.9668663775737) bank17842 +17843 POINT(34.864069561583726 -118.35631759284328) bank17843 +17844 POINT(33.67469304060843 -117.26355270456322) bank17844 +17845 POINT(34.86653467567951 -118.65810124466344) bank17845 +17846 POINT(33.99915295340317 -117.46283902442754) bank17846 +17847 POINT(33.084925838171074 -118.43142367982853) bank17847 +17848 POINT(33.15792988063264 -117.45800467488792) bank17848 +17849 POINT(34.01055444942791 -118.26018256027058) bank17849 +17850 POINT(34.729045929663734 -118.31698968770743) bank17850 +17851 POINT(34.86309082779252 -118.34225975933967) bank17851 +17852 POINT(34.99961771318785 -118.18550814939708) bank17852 +17853 POINT(34.601581697951794 -119.09456696240204) bank17853 +17854 POINT(34.853730611713544 -118.477367863965) bank17854 +17855 POINT(34.31240441570808 -118.22812931065711) bank17855 +17856 POINT(34.747653078604515 -117.41324958006895) bank17856 +17857 POINT(33.653451580033256 -118.1768899186056) bank17857 +17858 POINT(34.593693975785385 -118.23650298236537) bank17858 +17859 POINT(33.084531846434615 -117.45732567350584) bank17859 +17860 POINT(33.08767449368221 -117.67480279291222) bank17860 +17861 POINT(34.16402139823018 -118.52097159774989) bank17861 +17862 POINT(34.8150181315992 -119.02550069037211) bank17862 +17863 POINT(34.15406472673744 -117.64444036991999) bank17863 +17864 POINT(34.46727117224082 -119.10925991985368) bank17864 +17865 POINT(33.270032074992585 -119.1061416456936) bank17865 +17866 POINT(33.190698824655136 -117.42582212456858) bank17866 +17867 POINT(33.90864368753823 -117.39978197968657) bank17867 +17868 POINT(34.06544152247695 -118.33259660148485) bank17868 +17869 POINT(34.42092410222957 -117.24624376709181) bank17869 +17870 POINT(33.51270054564745 -118.81134942258724) bank17870 +17871 POINT(34.81792880811856 -118.00619117074912) bank17871 +17872 POINT(33.62584817761593 -117.78893363830501) bank17872 +17873 POINT(33.3796592902087 -117.44974118857269) bank17873 +17874 POINT(34.523878170300364 -117.47535545858746) bank17874 +17875 POINT(33.41536946386116 -118.17656347726597) bank17875 +17876 POINT(33.594260496290225 -118.85327179161078) bank17876 +17877 POINT(33.54929275784542 -118.3969098157742) bank17877 +17878 POINT(33.55960151678661 -117.70645022240254) bank17878 +17879 POINT(34.75716358221559 -118.61660509763384) bank17879 +17880 POINT(33.43221825914571 -119.0964756852252) bank17880 +17881 POINT(34.85059270802516 -117.69510124661998) bank17881 +17882 POINT(35.027528107619624 -118.5385299912971) bank17882 +17883 POINT(34.483041393611934 -118.87729600443953) bank17883 +17884 POINT(33.074749879970504 -119.1212731459453) bank17884 +17885 POINT(34.44002352985063 -118.92720099248267) bank17885 +17886 POINT(33.09667857345404 -118.80408053372955) bank17886 +17887 POINT(33.14526372577897 -117.58394493698299) bank17887 +17888 POINT(33.225739915641626 -117.53948581097426) bank17888 +17889 POINT(33.994887483824535 -117.94915478819684) bank17889 +17890 POINT(33.186914710701544 -117.53227062975834) bank17890 +17891 POINT(34.970009403649094 -119.23027747954495) bank17891 +17892 POINT(33.43683671241024 -119.24301039268866) bank17892 +17893 POINT(33.78228540045741 -119.22105499179577) bank17893 +17894 POINT(34.30850799155263 -118.21118401969909) bank17894 +17895 POINT(33.71229171313236 -118.98303864517986) bank17895 +17896 POINT(33.39169911799419 -118.87711003857186) bank17896 +17897 POINT(34.30522276576971 -119.07115762230504) bank17897 +17898 POINT(34.024229263253815 -119.10913464575454) bank17898 +17899 POINT(33.96374468336464 -117.28697364058935) bank17899 +17900 POINT(33.17204386023495 -119.1400118978207) bank17900 +17901 POINT(34.33836387046436 -117.55572118612021) bank17901 +17902 POINT(34.0901488615129 -119.00515281256993) bank17902 +17903 POINT(33.33950601975874 -118.28673264889328) bank17903 +17904 POINT(34.92546119471885 -117.99908721241493) bank17904 +17905 POINT(34.53665019288551 -117.91658453853813) bank17905 +17906 POINT(34.63476380626708 -119.16609757445418) bank17906 +17907 POINT(33.32308239886095 -119.22277966712478) bank17907 +17908 POINT(33.7350970124353 -118.13202377033629) bank17908 +17909 POINT(34.13980490778832 -118.42559371394792) bank17909 +17910 POINT(33.95896086986165 -118.93397385312643) bank17910 +17911 POINT(33.67276711134147 -117.96240205879714) bank17911 +17912 POINT(33.14985217958715 -117.4097129390196) bank17912 +17913 POINT(34.71241456144113 -118.55118301947543) bank17913 +17914 POINT(34.17849948692704 -118.26871681525567) bank17914 +17915 POINT(34.255886439418575 -118.4566209341756) bank17915 +17916 POINT(34.678691713622776 -118.0537730483775) bank17916 +17917 POINT(34.22532390582099 -117.61750681859928) bank17917 +17918 POINT(34.22084908759377 -117.58320492530532) bank17918 +17919 POINT(34.01626977644919 -117.56657325857503) bank17919 +17920 POINT(33.94882106237769 -118.63438195854688) bank17920 +17921 POINT(34.883723808304886 -118.31535941366418) bank17921 +17922 POINT(34.148995328071834 -117.41435799295603) bank17922 +17923 POINT(34.865167610758064 -117.74422435772985) bank17923 +17924 POINT(34.94932039508939 -119.01282670228323) bank17924 +17925 POINT(34.58160278368868 -119.21668391101329) bank17925 +17926 POINT(33.785787561032734 -118.39647878095556) bank17926 +17927 POINT(34.99571176127113 -118.24562746994422) bank17927 +17928 POINT(34.2283832184825 -118.9494667177915) bank17928 +17929 POINT(34.043776190405126 -118.93203090933338) bank17929 +17930 POINT(33.85720390423745 -118.26685036791085) bank17930 +17931 POINT(34.59270508242141 -117.81084550572297) bank17931 +17932 POINT(33.621210622471004 -117.36458721388632) bank17932 +17933 POINT(33.84403869533685 -117.70508265416427) bank17933 +17934 POINT(34.985138692698165 -117.78739416567866) bank17934 +17935 POINT(34.465759676587034 -118.68679913969974) bank17935 +17936 POINT(34.939518998512256 -117.30417154466649) bank17936 +17937 POINT(34.44863414522101 -118.54642205273333) bank17937 +17938 POINT(34.92392177815468 -117.55067763480565) bank17938 +17939 POINT(34.51493581507515 -118.87086185527686) bank17939 +17940 POINT(34.434454949177955 -118.53361276255768) bank17940 +17941 POINT(33.79363456222887 -117.86022220054727) bank17941 +17942 POINT(33.723987056477135 -117.73021165066685) bank17942 +17943 POINT(33.423372039993666 -117.38884531978644) bank17943 +17944 POINT(34.21034444891297 -118.32335160065612) bank17944 +17945 POINT(33.597140343968775 -117.60243248937199) bank17945 +17946 POINT(33.4666538187033 -118.58850000516453) bank17946 +17947 POINT(33.818096938278195 -117.50381959585586) bank17947 +17948 POINT(33.93683884703856 -118.11577208645878) bank17948 +17949 POINT(33.76514304231992 -118.7067327271189) bank17949 +17950 POINT(34.81379738355664 -117.68757661476427) bank17950 +17951 POINT(33.45836698963103 -117.75961475329113) bank17951 +17952 POINT(34.578150979299735 -118.81853660768604) bank17952 +17953 POINT(34.26760489104486 -118.5969671449676) bank17953 +17954 POINT(33.55384798735123 -118.2278493879236) bank17954 +17955 POINT(33.5504350625062 -118.50473255064261) bank17955 +17956 POINT(33.3926650699482 -118.5179200757211) bank17956 +17957 POINT(34.22662424817535 -118.32779351698754) bank17957 +17958 POINT(33.42361895178728 -117.46244676338674) bank17958 +17959 POINT(33.93121540857174 -118.36462751266828) bank17959 +17960 POINT(34.61170230931466 -118.32324565893602) bank17960 +17961 POINT(34.00629951459971 -118.33618866531864) bank17961 +17962 POINT(34.09263240930284 -117.62407875134066) bank17962 +17963 POINT(33.76739160933024 -117.3855685155833) bank17963 +17964 POINT(33.3455435668335 -118.44061633537308) bank17964 +17965 POINT(33.06825619296425 -118.67337105906206) bank17965 +17966 POINT(33.875043783676084 -119.1885718275279) bank17966 +17967 POINT(34.44351426979483 -117.63280927886169) bank17967 +17968 POINT(34.90225750956275 -117.47455169223421) bank17968 +17969 POINT(33.84576409049182 -119.04209863097739) bank17969 +17970 POINT(34.24183950172579 -119.01956716368434) bank17970 +17971 POINT(33.06839059516331 -117.71476946515315) bank17971 +17972 POINT(34.41333021545063 -118.08210125278505) bank17972 +17973 POINT(33.80410486159215 -118.46348118845128) bank17973 +17974 POINT(34.115285154081555 -117.99166935607865) bank17974 +17975 POINT(33.460963882759145 -118.58442235861003) bank17975 +17976 POINT(33.46781172316118 -118.38308392450439) bank17976 +17977 POINT(33.87534609759393 -118.76930412673015) bank17977 +17978 POINT(34.46277881850872 -117.88143969530441) bank17978 +17979 POINT(33.38739246711774 -117.6818664462693) bank17979 +17980 POINT(34.82173120115337 -118.82224382818325) bank17980 +17981 POINT(33.529197041075015 -119.15595742494331) bank17981 +17982 POINT(35.040805359338094 -117.37772838433996) bank17982 +17983 POINT(34.109287811523814 -119.12679516593535) bank17983 +17984 POINT(33.4998308519677 -118.58031269690052) bank17984 +17985 POINT(33.64964624094314 -118.58492607392515) bank17985 +17986 POINT(34.857456924663545 -118.83324471258314) bank17986 +17987 POINT(34.73726305443031 -118.09032779205788) bank17987 +17988 POINT(34.92953866269356 -117.79208631780436) bank17988 +17989 POINT(34.270741763939064 -117.96235094069262) bank17989 +17990 POINT(33.82700661276984 -117.24856662029703) bank17990 +17991 POINT(34.55532108576054 -118.29270951691223) bank17991 +17992 POINT(33.10089530852458 -117.55061823697812) bank17992 +17993 POINT(34.635735662061954 -118.99855889726493) bank17993 +17994 POINT(33.90512454433306 -119.04866702579426) bank17994 +17995 POINT(33.17217491037098 -117.89166470459739) bank17995 +17996 POINT(33.3607535736466 -117.89018263994927) bank17996 +17997 POINT(34.860702828963646 -118.8321758181362) bank17997 +17998 POINT(34.29165096274418 -118.0304466878687) bank17998 +17999 POINT(33.235287846297126 -119.21644401205616) bank17999 +18000 POINT(34.58464104734114 -117.42686678106655) bank18000 +18001 POINT(34.52770601925255 -118.62141182133381) bank18001 +18002 POINT(33.50373127454948 -118.89303430404439) bank18002 +18003 POINT(33.23589406546802 -118.21991543103898) bank18003 +18004 POINT(33.27353462589813 -119.0924868688546) bank18004 +18005 POINT(34.09406717769329 -118.24888673725016) bank18005 +18006 POINT(33.182140287833434 -117.62149080565692) bank18006 +18007 POINT(33.61382090902332 -117.80139024040783) bank18007 +18008 POINT(34.652022456025435 -119.18261793411587) bank18008 +18009 POINT(33.189103485212655 -118.58805977150301) bank18009 +18010 POINT(33.87488794955147 -118.66296133662264) bank18010 +18011 POINT(33.46310153027961 -118.57963794335522) bank18011 +18012 POINT(33.07498728874061 -117.31973588223045) bank18012 +18013 POINT(34.72181069907673 -117.95699929166267) bank18013 +18014 POINT(33.366646175181366 -117.49465534778503) bank18014 +18015 POINT(33.569758784838875 -118.49400918403755) bank18015 +18016 POINT(33.40771805402243 -118.4902940062557) bank18016 +18017 POINT(34.795414476977164 -117.38351985634955) bank18017 +18018 POINT(33.74000271499903 -118.1377827240094) bank18018 +18019 POINT(34.79566580338359 -118.92449545793511) bank18019 +18020 POINT(34.46727574738881 -118.68485246503224) bank18020 +18021 POINT(34.563361525146895 -118.71570224366518) bank18021 +18022 POINT(34.491616917038776 -118.44400866377318) bank18022 +18023 POINT(33.06564301987804 -117.73310262691264) bank18023 +18024 POINT(34.02416067504065 -117.87727210000406) bank18024 +18025 POINT(33.62362152718598 -118.12126657058424) bank18025 +18026 POINT(34.04383732599034 -118.08658144786787) bank18026 +18027 POINT(34.26199458500294 -117.75068574482013) bank18027 +18028 POINT(33.38261156117385 -118.70874956709557) bank18028 +18029 POINT(33.156993053525945 -118.3609717107896) bank18029 +18030 POINT(33.99524079182867 -118.85772683145859) bank18030 +18031 POINT(33.932789182736435 -117.67895043166696) bank18031 +18032 POINT(34.051866430879244 -117.29128532855067) bank18032 +18033 POINT(34.531733919090236 -119.11025937370604) bank18033 +18034 POINT(33.957991903283585 -118.73168181961195) bank18034 +18035 POINT(34.57228593895934 -118.17811380058531) bank18035 +18036 POINT(34.446145014388996 -118.17611609950697) bank18036 +18037 POINT(33.20366372021437 -117.89499610674326) bank18037 +18038 POINT(34.54801342526714 -117.24653858489104) bank18038 +18039 POINT(34.38876773188434 -117.56706672557584) bank18039 +18040 POINT(33.80001201559104 -117.74221241119169) bank18040 +18041 POINT(34.942363973170615 -117.89555341124479) bank18041 +18042 POINT(34.64157209592399 -118.56335318024122) bank18042 +18043 POINT(33.57326066637109 -117.24840522207772) bank18043 +18044 POINT(33.73741752139085 -118.60577535589069) bank18044 +18045 POINT(34.142537515502106 -117.24928627288523) bank18045 +18046 POINT(34.640900936740046 -118.63822694256517) bank18046 +18047 POINT(33.21657888007554 -117.4488508249871) bank18047 +18048 POINT(33.53045542643125 -118.63323884406927) bank18048 +18049 POINT(33.27022335068883 -117.3716616527773) bank18049 +18050 POINT(33.74972668696206 -118.08553433648609) bank18050 +18051 POINT(35.00305434240019 -118.04690056535692) bank18051 +18052 POINT(33.72164474995318 -117.95534792114107) bank18052 +18053 POINT(33.28286447614189 -118.96719839639606) bank18053 +18054 POINT(34.859691919513594 -118.0484993145221) bank18054 +18055 POINT(33.45859927774396 -118.44497666205287) bank18055 +18056 POINT(33.253875275853524 -117.32990418475511) bank18056 +18057 POINT(33.598584290909145 -117.71495967226494) bank18057 +18058 POINT(33.65714602653532 -117.98001769470677) bank18058 +18059 POINT(34.87578242271061 -117.46113499116953) bank18059 +18060 POINT(33.939784467765115 -117.37272513493733) bank18060 +18061 POINT(34.02363126812153 -117.84981084013745) bank18061 +18062 POINT(34.224959284005834 -117.74611719242463) bank18062 +18063 POINT(33.72459266772036 -117.68502132304381) bank18063 +18064 POINT(35.02994051412345 -118.10128325369759) bank18064 +18065 POINT(33.62639695916843 -118.65986091759771) bank18065 +18066 POINT(33.51245630257342 -117.71136684307798) bank18066 +18067 POINT(34.834251206559564 -118.42814883582162) bank18067 +18068 POINT(34.35803790646641 -117.3785827443272) bank18068 +18069 POINT(33.44567172292986 -118.43302548964715) bank18069 +18070 POINT(34.42757497913204 -117.87334226540145) bank18070 +18071 POINT(33.07528549769873 -118.29913453111946) bank18071 +18072 POINT(34.906531524813275 -118.41891042101878) bank18072 +18073 POINT(33.486787271995595 -118.3187956165913) bank18073 +18074 POINT(35.04824258855274 -117.29035960431473) bank18074 +18075 POINT(34.32072688904051 -119.06978322180595) bank18075 +18076 POINT(34.145878645716365 -118.69734010753294) bank18076 +18077 POINT(34.86889214167922 -118.88469028587753) bank18077 +18078 POINT(34.25832867807556 -118.73400470819902) bank18078 +18079 POINT(34.323862300596346 -118.68908212488702) bank18079 +18080 POINT(33.62328749073406 -118.00700313890131) bank18080 +18081 POINT(33.91149030728265 -118.39066010372112) bank18081 +18082 POINT(33.645500930181065 -117.85951049166096) bank18082 +18083 POINT(34.22898167961527 -117.26284043201872) bank18083 +18084 POINT(34.65344685366315 -117.72816854072879) bank18084 +18085 POINT(33.2574229930557 -118.83749172053756) bank18085 +18086 POINT(34.60430332697657 -118.84290173905315) bank18086 +18087 POINT(33.18112891028012 -118.37190459108773) bank18087 +18088 POINT(33.7991541098724 -117.73077038334665) bank18088 +18089 POINT(34.05130744246924 -118.87005895632286) bank18089 +18090 POINT(33.56834936571912 -118.45595707739731) bank18090 +18091 POINT(33.96732516765164 -118.11386805901783) bank18091 +18092 POINT(33.632086067103074 -118.24617458795552) bank18092 +18093 POINT(34.25475948520393 -118.76354123351872) bank18093 +18094 POINT(35.01404297702898 -118.83896950511077) bank18094 +18095 POINT(33.876537665083625 -118.51234687349061) bank18095 +18096 POINT(33.7368643145465 -118.57329643023698) bank18096 +18097 POINT(34.04462771926998 -118.97406552358802) bank18097 +18098 POINT(33.673260861371276 -118.39456162784307) bank18098 +18099 POINT(33.99390513139352 -118.47680541959541) bank18099 +18100 POINT(34.72131213299539 -117.94825208693646) bank18100 +18101 POINT(33.21199192618071 -117.35529315311786) bank18101 +18102 POINT(34.19268368709774 -117.63595498109324) bank18102 +18103 POINT(33.48362526482045 -118.42727224990895) bank18103 +18104 POINT(34.63709114134662 -118.74044834151388) bank18104 +18105 POINT(34.29652610182117 -117.54104725547325) bank18105 +18106 POINT(34.06431538744716 -118.6501123047355) bank18106 +18107 POINT(34.84139615850677 -118.09143945252288) bank18107 +18108 POINT(33.524547916503046 -117.50559649548609) bank18108 +18109 POINT(33.14265011499127 -117.49327538832121) bank18109 +18110 POINT(33.99111166102547 -117.57007627712812) bank18110 +18111 POINT(33.67972517217332 -118.95757602292903) bank18111 +18112 POINT(34.76144958962533 -118.52939163250448) bank18112 +18113 POINT(34.19420522321244 -117.30182041618214) bank18113 +18114 POINT(34.428506243003675 -119.11050423849895) bank18114 +18115 POINT(33.959836404897175 -118.07945405776587) bank18115 +18116 POINT(33.97151574285031 -117.552825250605) bank18116 +18117 POINT(33.85489257311167 -117.92634028849977) bank18117 +18118 POINT(34.08054686989852 -117.69694222787685) bank18118 +18119 POINT(34.7719594532872 -117.33546599581942) bank18119 +18120 POINT(33.675052590581075 -118.30003662230483) bank18120 +18121 POINT(34.20474922589102 -118.2633723565295) bank18121 +18122 POINT(34.64662863606917 -117.98094375314832) bank18122 +18123 POINT(33.23787313926339 -117.91186407079411) bank18123 +18124 POINT(34.0885364106479 -118.59449194590415) bank18124 +18125 POINT(33.45288349016155 -117.63243076117512) bank18125 +18126 POINT(34.62999625446287 -119.12897346450562) bank18126 +18127 POINT(34.96477988513619 -117.55627461658273) bank18127 +18128 POINT(34.29019167391887 -117.98613146039521) bank18128 +18129 POINT(33.2857065810464 -118.29826141239279) bank18129 +18130 POINT(34.957553768943775 -117.44848237756797) bank18130 +18131 POINT(33.730792542551384 -118.22237580097074) bank18131 +18132 POINT(34.718740444084574 -118.7261099200581) bank18132 +18133 POINT(33.20292436468823 -118.4815685003884) bank18133 +18134 POINT(33.546191112273206 -117.97379314096536) bank18134 +18135 POINT(34.277177736994396 -118.9750827867291) bank18135 +18136 POINT(34.54954992363477 -118.27042352903315) bank18136 +18137 POINT(35.01581639050556 -119.07041421079063) bank18137 +18138 POINT(34.441253337910545 -117.89299277531023) bank18138 +18139 POINT(33.14244159211365 -117.54142366563845) bank18139 +18140 POINT(34.567230413668085 -118.80713068829792) bank18140 +18141 POINT(33.89446029100727 -118.84889981738603) bank18141 +18142 POINT(33.656285270890685 -118.05711856689742) bank18142 +18143 POINT(34.089397727205956 -118.23990051702077) bank18143 +18144 POINT(33.45261236964933 -117.72583153371673) bank18144 +18145 POINT(34.50804355048972 -118.06553562768946) bank18145 +18146 POINT(34.70635795068927 -118.89793493274846) bank18146 +18147 POINT(33.13104059329876 -118.73339355137684) bank18147 +18148 POINT(33.27010645336943 -118.24066565600414) bank18148 +18149 POINT(33.9724102809463 -118.43340853496898) bank18149 +18150 POINT(34.616478254831165 -118.71696232204098) bank18150 +18151 POINT(33.67036136388144 -118.62471341904126) bank18151 +18152 POINT(34.514424648596815 -119.13662548842869) bank18152 +18153 POINT(33.71483896817519 -117.60840043189805) bank18153 +18154 POINT(33.30211113690697 -117.72569483099164) bank18154 +18155 POINT(34.84970680008389 -117.24477164041477) bank18155 +18156 POINT(34.10224842512406 -118.24967848208034) bank18156 +18157 POINT(34.43955716055261 -119.08013289821247) bank18157 +18158 POINT(34.8739362939738 -117.4759254743053) bank18158 +18159 POINT(33.98789802675635 -117.32267241155034) bank18159 +18160 POINT(34.58292755750469 -118.58200135608939) bank18160 +18161 POINT(34.6020638542044 -118.7296750387322) bank18161 +18162 POINT(33.06524701588965 -119.17047636900742) bank18162 +18163 POINT(33.594171590186235 -117.27430598953104) bank18163 +18164 POINT(33.5162440539196 -117.77037222862928) bank18164 +18165 POINT(34.68718168225188 -117.41851782436193) bank18165 +18166 POINT(33.43749570517635 -118.10962100368046) bank18166 +18167 POINT(34.49160313010423 -119.07587778324219) bank18167 +18168 POINT(33.40813965957622 -117.49505081211406) bank18168 +18169 POINT(34.90131118993437 -117.41689779652718) bank18169 +18170 POINT(33.93769857560706 -118.9857524236966) bank18170 +18171 POINT(33.66080473595216 -117.33576537894118) bank18171 +18172 POINT(35.02070573629089 -119.20899021855638) bank18172 +18173 POINT(34.34913879261433 -117.292621167424) bank18173 +18174 POINT(34.12325723148925 -118.10593564475198) bank18174 +18175 POINT(34.369264705425664 -119.21024952143975) bank18175 +18176 POINT(34.73414698027612 -118.64379084746818) bank18176 +18177 POINT(33.959831678354924 -118.58668874266306) bank18177 +18178 POINT(33.73706374403831 -118.2581668704228) bank18178 +18179 POINT(33.809570328183 -117.28719896251) bank18179 +18180 POINT(33.429886704287405 -118.8071492762815) bank18180 +18181 POINT(34.293270036229636 -118.69006610047794) bank18181 +18182 POINT(34.48563757394449 -117.57345271140511) bank18182 +18183 POINT(35.019504560535495 -119.09710976923557) bank18183 +18184 POINT(34.488139598407486 -117.91985227096674) bank18184 +18185 POINT(34.69022776174116 -117.71677317801645) bank18185 +18186 POINT(33.60758829104391 -117.6537450177115) bank18186 +18187 POINT(33.636020380485185 -117.46634811740496) bank18187 +18188 POINT(33.69316179173636 -118.90984249998313) bank18188 +18189 POINT(33.1569546583824 -118.0791785645242) bank18189 +18190 POINT(34.67504397849313 -118.25011824205752) bank18190 +18191 POINT(33.63788547766598 -117.29691827266072) bank18191 +18192 POINT(34.752397066468816 -118.53548269889328) bank18192 +18193 POINT(33.22070010151436 -118.5432196150166) bank18193 +18194 POINT(34.55402421135429 -117.35320507408505) bank18194 +18195 POINT(33.12287692367855 -118.57261923256848) bank18195 +18196 POINT(35.026577954534105 -118.6770813469703) bank18196 +18197 POINT(33.252155368632465 -117.73469347889109) bank18197 +18198 POINT(33.226085248834195 -117.57086741487862) bank18198 +18199 POINT(33.76346040546731 -119.08914938463835) bank18199 +18200 POINT(33.96357422024054 -119.16850190345822) bank18200 +18201 POINT(33.17441764580012 -118.13168450006401) bank18201 +18202 POINT(33.28656180745413 -117.97279198551128) bank18202 +18203 POINT(34.90256468153391 -118.61455719081411) bank18203 +18204 POINT(33.262493162354325 -117.26030307266053) bank18204 +18205 POINT(34.78677448712859 -118.00520514531021) bank18205 +18206 POINT(33.15617206747421 -117.41533827270099) bank18206 +18207 POINT(35.0358929726612 -119.17808149864163) bank18207 +18208 POINT(33.12784936538164 -117.28670319765457) bank18208 +18209 POINT(34.977202991985315 -117.85648771279136) bank18209 +18210 POINT(33.707819172956356 -118.06464131025226) bank18210 +18211 POINT(34.04940423137001 -117.62166638616154) bank18211 +18212 POINT(34.219377517467926 -118.52053037445224) bank18212 +18213 POINT(34.65082723273918 -118.83233326695557) bank18213 +18214 POINT(33.403052743755765 -118.06000272765505) bank18214 +18215 POINT(34.55267220462986 -118.83839669548735) bank18215 +18216 POINT(34.72768732384132 -119.09803531039317) bank18216 +18217 POINT(34.3269412086141 -117.41993229077289) bank18217 +18218 POINT(33.432209287505074 -117.58735510327453) bank18218 +18219 POINT(34.709461624517495 -117.8628210467193) bank18219 +18220 POINT(34.27731502082025 -117.91693928980847) bank18220 +18221 POINT(34.1059771899224 -118.56330123468534) bank18221 +18222 POINT(34.960188276596064 -118.18998098085665) bank18222 +18223 POINT(34.36969152674181 -117.64379016321142) bank18223 +18224 POINT(33.6892970885 -118.20688359131351) bank18224 +18225 POINT(34.98540549421463 -117.80481037898105) bank18225 +18226 POINT(34.70948078769854 -118.94258213222697) bank18226 +18227 POINT(33.08449391738433 -117.3055821858837) bank18227 +18228 POINT(34.06927482035551 -117.36768819135618) bank18228 +18229 POINT(33.96467402115401 -117.97104584224678) bank18229 +18230 POINT(33.98364904046256 -118.30036145382857) bank18230 +18231 POINT(33.08094191975095 -118.72171349750631) bank18231 +18232 POINT(34.82355389327964 -117.81687922241933) bank18232 +18233 POINT(34.16869969450768 -118.98383983078256) bank18233 +18234 POINT(33.90927313255552 -118.78747405293292) bank18234 +18235 POINT(33.974696326917595 -118.96942340742417) bank18235 +18236 POINT(34.26010024107517 -117.384378029183) bank18236 +18237 POINT(33.07696363031856 -117.45055680549964) bank18237 +18238 POINT(34.38600701381604 -118.53818731894604) bank18238 +18239 POINT(33.134323695856 -118.2965002020726) bank18239 +18240 POINT(33.83936558082979 -118.79229269685928) bank18240 +18241 POINT(34.63004891942024 -117.53142418439371) bank18241 +18242 POINT(34.00011873066917 -118.38710759111599) bank18242 +18243 POINT(33.52446177736524 -118.12652153698052) bank18243 +18244 POINT(34.7113368262093 -117.25162372940942) bank18244 +18245 POINT(33.30042956634783 -117.70709851045964) bank18245 +18246 POINT(33.34598444063833 -117.30186803862321) bank18246 +18247 POINT(34.53935214787653 -117.9057009714969) bank18247 +18248 POINT(34.195643611821176 -118.97679705577814) bank18248 +18249 POINT(34.65566805486259 -118.70922553380208) bank18249 +18250 POINT(34.5424275214806 -117.99930079029188) bank18250 +18251 POINT(34.33205461840224 -117.27388714415653) bank18251 +18252 POINT(33.31580321751206 -119.24098839770808) bank18252 +18253 POINT(33.46640441987062 -118.99896591848872) bank18253 +18254 POINT(34.79299968354791 -118.15039262205516) bank18254 +18255 POINT(33.39639220735649 -119.19647526084012) bank18255 +18256 POINT(33.213094737803424 -118.77636186951145) bank18256 +18257 POINT(33.85950214329847 -119.1418855497881) bank18257 +18258 POINT(34.707799754838405 -118.62120722047854) bank18258 +18259 POINT(33.82666459721688 -118.58039357559014) bank18259 +18260 POINT(33.33211319054593 -117.83214985124071) bank18260 +18261 POINT(34.134539606470746 -117.37293751234934) bank18261 +18262 POINT(33.569133056740874 -118.42986668350206) bank18262 +18263 POINT(34.11211091632192 -117.5902009963248) bank18263 +18264 POINT(33.34611850013749 -117.44681531179805) bank18264 +18265 POINT(33.713530045994794 -117.32030907710173) bank18265 +18266 POINT(33.20669879463351 -117.84882811611925) bank18266 +18267 POINT(34.875570120426545 -118.87123012466442) bank18267 +18268 POINT(34.47928292360001 -118.17589461953878) bank18268 +18269 POINT(34.46980821313231 -119.13892770081478) bank18269 +18270 POINT(33.32156425030423 -118.33543164777788) bank18270 +18271 POINT(33.53208397048264 -117.45896948790154) bank18271 +18272 POINT(34.994596401103884 -118.69688339091635) bank18272 +18273 POINT(33.68868261171572 -118.70950060198403) bank18273 +18274 POINT(34.28773478174323 -118.59618217049993) bank18274 +18275 POINT(33.27327702773831 -118.84278816996917) bank18275 +18276 POINT(34.44053077255264 -119.18123439169734) bank18276 +18277 POINT(33.111877276685206 -117.97587521467575) bank18277 +18278 POINT(33.874149962653895 -118.96975045847553) bank18278 +18279 POINT(33.580804948103534 -117.51837198217979) bank18279 +18280 POINT(33.402997147378 -118.06643351661194) bank18280 +18281 POINT(33.53200313172049 -119.1711782351852) bank18281 +18282 POINT(34.150577855101666 -118.77769752580187) bank18282 +18283 POINT(34.51696283795902 -118.91565194977531) bank18283 +18284 POINT(34.23799368657394 -117.69565311808523) bank18284 +18285 POINT(33.536196968299905 -117.81065667268146) bank18285 +18286 POINT(33.632261159213506 -118.67825078638616) bank18286 +18287 POINT(34.22909298487602 -118.46064538357018) bank18287 +18288 POINT(34.913915383121115 -118.60532958953864) bank18288 +18289 POINT(34.07153948304782 -118.04988526683475) bank18289 +18290 POINT(34.03023297097928 -117.98531212697615) bank18290 +18291 POINT(33.88822770435718 -118.53066796874032) bank18291 +18292 POINT(33.17305459306814 -118.41406248056968) bank18292 +18293 POINT(34.69555278510092 -118.73566137392487) bank18293 +18294 POINT(33.41749523166761 -119.12467405202005) bank18294 +18295 POINT(34.817670896574676 -118.69469873472546) bank18295 +18296 POINT(34.57279234929987 -117.64525123515503) bank18296 +18297 POINT(34.616253794571094 -118.89083685536913) bank18297 +18298 POINT(33.27301120313612 -119.22888821244393) bank18298 +18299 POINT(35.049579977133874 -118.76257955758896) bank18299 +18300 POINT(34.27947934771095 -118.13367415598742) bank18300 +18301 POINT(33.91073361714852 -117.98008017576002) bank18301 +18302 POINT(34.45745286137644 -118.09629389059165) bank18302 +18303 POINT(34.11146584943681 -118.46504907055025) bank18303 +18304 POINT(33.764322288636336 -118.28503708989571) bank18304 +18305 POINT(34.69284369783799 -119.22860902133999) bank18305 +18306 POINT(34.53074212876997 -118.98399793114596) bank18306 +18307 POINT(34.57805582987712 -118.21394541812951) bank18307 +18308 POINT(33.42711154423694 -118.84870608382317) bank18308 +18309 POINT(34.854871387525584 -117.41986566261075) bank18309 +18310 POINT(34.84105587509528 -117.34870824993236) bank18310 +18311 POINT(33.5429612581322 -117.47370462674647) bank18311 +18312 POINT(33.96374393331453 -119.19443503617441) bank18312 +18313 POINT(33.466486522297714 -118.72910189592574) bank18313 +18314 POINT(33.381217045160284 -117.95004021821092) bank18314 +18315 POINT(34.97543675567056 -117.78869049908927) bank18315 +18316 POINT(34.19980212310039 -118.17540152935581) bank18316 +18317 POINT(33.521937556420085 -119.23697653736845) bank18317 +18318 POINT(33.77829351804556 -118.10230237004666) bank18318 +18319 POINT(33.36194564249491 -118.34957773769239) bank18319 +18320 POINT(34.00756238321691 -118.40278562530712) bank18320 +18321 POINT(34.54538034700129 -119.18878916611604) bank18321 +18322 POINT(33.22844014979932 -118.29051162200578) bank18322 +18323 POINT(34.84591796273671 -119.13378463136547) bank18323 +18324 POINT(33.9474805399112 -117.77645635098179) bank18324 +18325 POINT(34.475697618250145 -118.86709663354965) bank18325 +18326 POINT(35.029703637550256 -119.1519439798043) bank18326 +18327 POINT(33.631897625416784 -118.15903923521985) bank18327 +18328 POINT(33.45996477946069 -118.07083101483491) bank18328 +18329 POINT(34.453309552820784 -119.20797759297393) bank18329 +18330 POINT(33.48290549875323 -117.55983848271167) bank18330 +18331 POINT(33.46683108379081 -118.38432074474862) bank18331 +18332 POINT(34.83940121308051 -118.99171235694548) bank18332 +18333 POINT(33.06597875120245 -118.72946029006572) bank18333 +18334 POINT(33.45315461282563 -118.73978680489124) bank18334 +18335 POINT(34.49854826728143 -117.2652624571017) bank18335 +18336 POINT(34.98578228992126 -117.57194534271598) bank18336 +18337 POINT(33.11474094926472 -117.81717902060876) bank18337 +18338 POINT(34.64012035081681 -117.5024818891414) bank18338 +18339 POINT(33.526389310322784 -117.5327319919917) bank18339 +18340 POINT(34.49344499262038 -117.31267281831732) bank18340 +18341 POINT(34.253497765834915 -117.54327607254453) bank18341 +18342 POINT(34.56968945810809 -118.5797861618616) bank18342 +18343 POINT(34.81406837261873 -117.30970943905811) bank18343 +18344 POINT(33.54834652255072 -117.63236577018259) bank18344 +18345 POINT(33.267159349632074 -117.42851892678284) bank18345 +18346 POINT(34.298002818592245 -118.82282383723926) bank18346 +18347 POINT(33.27968889116198 -117.65939370637129) bank18347 +18348 POINT(33.22297557536609 -117.63199132373403) bank18348 +18349 POINT(33.66126605132703 -118.10044672966306) bank18349 +18350 POINT(34.635436120491065 -118.70083573135679) bank18350 +18351 POINT(34.8434243107987 -117.61669556242273) bank18351 +18352 POINT(34.67348981962873 -118.9060174796304) bank18352 +18353 POINT(34.29636239740382 -118.9119435436651) bank18353 +18354 POINT(33.18854689182677 -117.74152627427335) bank18354 +18355 POINT(34.82894503605851 -118.14898026815314) bank18355 +18356 POINT(33.51322008420142 -118.99047872369653) bank18356 +18357 POINT(33.99203374010336 -118.52771922855491) bank18357 +18358 POINT(33.19488142642944 -119.13619510037807) bank18358 +18359 POINT(34.164969393275314 -117.84906010848448) bank18359 +18360 POINT(33.62395184630362 -118.56949859136705) bank18360 +18361 POINT(34.97846020923948 -118.2494584952366) bank18361 +18362 POINT(34.77183458531585 -117.83672175918414) bank18362 +18363 POINT(34.29506457552298 -118.38086339999509) bank18363 +18364 POINT(33.78350734980762 -118.96559490923897) bank18364 +18365 POINT(33.446752210522774 -117.36473770806185) bank18365 +18366 POINT(34.33242393660767 -118.02486042073788) bank18366 +18367 POINT(33.34025625865261 -118.35775509204855) bank18367 +18368 POINT(33.19262252810526 -118.01177502001968) bank18368 +18369 POINT(34.96064643843451 -117.27549502677738) bank18369 +18370 POINT(34.512537663819366 -118.57827485214767) bank18370 +18371 POINT(33.436020444244285 -118.80599111675157) bank18371 +18372 POINT(34.18147210951893 -118.85134273903878) bank18372 +18373 POINT(33.91970310188623 -117.86690804274066) bank18373 +18374 POINT(34.435846570205086 -118.19013593706711) bank18374 +18375 POINT(33.27149682637665 -118.74460796709579) bank18375 +18376 POINT(34.97477444684089 -117.36265970045874) bank18376 +18377 POINT(34.69214944474785 -118.8168754152521) bank18377 +18378 POINT(35.00133589207788 -119.13297494425397) bank18378 +18379 POINT(33.54127979990131 -118.8849970858698) bank18379 +18380 POINT(33.06083034223559 -117.26726207971068) bank18380 +18381 POINT(33.52086943031335 -117.79899970652436) bank18381 +18382 POINT(34.00991286542779 -117.84534051550898) bank18382 +18383 POINT(34.965376959050886 -118.84499897048953) bank18383 +18384 POINT(34.65324080672155 -117.84382024604363) bank18384 +18385 POINT(34.41027244469706 -118.76352855619992) bank18385 +18386 POINT(33.76464863428147 -119.1484444473391) bank18386 +18387 POINT(34.999326825589065 -117.86923530218267) bank18387 +18388 POINT(33.106965056201545 -117.8903529015554) bank18388 +18389 POINT(33.22243659947083 -118.20786425783436) bank18389 +18390 POINT(33.84705206019953 -119.05088436229448) bank18390 +18391 POINT(34.779460869846226 -117.38576549883929) bank18391 +18392 POINT(34.26836104563023 -118.61960478760302) bank18392 +18393 POINT(33.286825706756105 -118.78426688679465) bank18393 +18394 POINT(33.891696094637155 -118.95896953270108) bank18394 +18395 POINT(33.069198760285644 -117.24801703004012) bank18395 +18396 POINT(34.43791044134192 -118.5185016219628) bank18396 +18397 POINT(34.850882421930635 -117.60947494871186) bank18397 +18398 POINT(33.58510632481843 -118.93966919740883) bank18398 +18399 POINT(34.644304206972336 -117.97274660892933) bank18399 +18400 POINT(34.37091003913587 -117.88300483275457) bank18400 +18401 POINT(34.116618682375226 -117.7127740071365) bank18401 +18402 POINT(34.790467304081446 -117.746948834592) bank18402 +18403 POINT(34.262464061338406 -117.6857351134166) bank18403 +18404 POINT(33.50919320115483 -117.36945357285444) bank18404 +18405 POINT(34.00132086717914 -118.30841791997017) bank18405 +18406 POINT(34.072261772404964 -118.10984806060577) bank18406 +18407 POINT(33.986894372140284 -118.19704610235529) bank18407 +18408 POINT(33.93599970224718 -118.18665967164674) bank18408 +18409 POINT(33.18205017333832 -118.92340860190835) bank18409 +18410 POINT(33.23966586648651 -118.02175233293978) bank18410 +18411 POINT(33.23332898496117 -117.71288090602981) bank18411 +18412 POINT(33.215815221340655 -118.7744904399848) bank18412 +18413 POINT(33.29271798981219 -118.1275152728925) bank18413 +18414 POINT(33.40061363750722 -118.99230883046293) bank18414 +18415 POINT(34.00993555440651 -118.15045680905992) bank18415 +18416 POINT(33.62682925973619 -117.56220965303693) bank18416 +18417 POINT(33.26477015956996 -118.08482250519818) bank18417 +18418 POINT(33.67750596679646 -118.40086596723438) bank18418 +18419 POINT(34.755290954798014 -118.73767236885433) bank18419 +18420 POINT(34.23369745616175 -118.55450233506669) bank18420 +18421 POINT(34.877513193316716 -117.67608305244893) bank18421 +18422 POINT(33.15369331153231 -117.53833115981236) bank18422 +18423 POINT(33.394367141848576 -117.74848861985531) bank18423 +18424 POINT(33.116246051932954 -118.3517447062581) bank18424 +18425 POINT(34.21386283722692 -118.38405569848832) bank18425 +18426 POINT(33.74318488394437 -117.30122178818155) bank18426 +18427 POINT(33.82227548069607 -118.04122957238401) bank18427 +18428 POINT(34.05169582150422 -117.28236110995597) bank18428 +18429 POINT(34.278269658702364 -118.88675130436903) bank18429 +18430 POINT(34.508835580738925 -119.18723459330951) bank18430 +18431 POINT(33.92700358902339 -118.87079451872744) bank18431 +18432 POINT(33.765763840247715 -118.05967352195078) bank18432 +18433 POINT(33.628009165253594 -118.81043428973067) bank18433 +18434 POINT(33.116301055637415 -118.88137270591338) bank18434 +18435 POINT(34.53206216237168 -117.4601810818104) bank18435 +18436 POINT(34.10954077173871 -118.65421983272212) bank18436 +18437 POINT(34.78438107470798 -118.04035513388325) bank18437 +18438 POINT(33.66102027986437 -118.60371455252506) bank18438 +18439 POINT(33.94457005010458 -118.07275869439229) bank18439 +18440 POINT(34.50736227968006 -119.2003917316528) bank18440 +18441 POINT(33.40323029035323 -117.3605386666331) bank18441 +18442 POINT(33.7983551467178 -117.92683468709437) bank18442 +18443 POINT(34.64440461422056 -117.62304277922847) bank18443 +18444 POINT(34.525682919285046 -118.94303213035023) bank18444 +18445 POINT(34.155485224745924 -118.4219411848186) bank18445 +18446 POINT(34.50284121003289 -118.58430430176917) bank18446 +18447 POINT(34.06536674121216 -119.17766228158126) bank18447 +18448 POINT(34.6916898178945 -118.66079426146197) bank18448 +18449 POINT(34.97845923985911 -118.00488331216059) bank18449 +18450 POINT(33.224313767936074 -118.44401473830962) bank18450 +18451 POINT(33.63469073630244 -118.34180704806846) bank18451 +18452 POINT(34.209785420927474 -118.55106102942787) bank18452 +18453 POINT(33.83290745918545 -118.34104359842735) bank18453 +18454 POINT(34.692132726714455 -118.65249965487996) bank18454 +18455 POINT(34.24484458322674 -117.70048033801932) bank18455 +18456 POINT(33.373910757676576 -119.20761174589495) bank18456 +18457 POINT(34.581545974205135 -117.71578387513021) bank18457 +18458 POINT(33.58659444214217 -118.29025193424297) bank18458 +18459 POINT(34.70980554483957 -117.40430156641372) bank18459 +18460 POINT(33.060915906885015 -117.6874492291741) bank18460 +18461 POINT(34.78320740438531 -118.37823795072678) bank18461 +18462 POINT(34.1038204064321 -117.2528963424443) bank18462 +18463 POINT(34.40873376167334 -118.50944648488867) bank18463 +18464 POINT(33.172582325540446 -118.84903422740685) bank18464 +18465 POINT(33.551806794381456 -117.29915232111176) bank18465 +18466 POINT(33.2146455962383 -119.16103108066267) bank18466 +18467 POINT(34.819594131650675 -117.72258859326442) bank18467 +18468 POINT(34.86774249268073 -118.68828904759319) bank18468 +18469 POINT(33.644898532061916 -118.82058061432136) bank18469 +18470 POINT(33.79377831540146 -118.39340043394526) bank18470 +18471 POINT(34.054637599277655 -117.56773089813429) bank18471 +18472 POINT(33.6017162471759 -117.7411718855346) bank18472 +18473 POINT(34.444807660734995 -118.32112169443671) bank18473 +18474 POINT(35.01876162681121 -118.92071551200334) bank18474 +18475 POINT(34.47822396097373 -117.50398198169654) bank18475 +18476 POINT(34.14648436961949 -119.19089372567474) bank18476 +18477 POINT(33.22177337610252 -117.32131022664146) bank18477 +18478 POINT(34.37579080896954 -118.64079065100907) bank18478 +18479 POINT(34.24820163001021 -118.17376284806713) bank18479 +18480 POINT(33.96994641839654 -119.18997662884006) bank18480 +18481 POINT(34.002624301269954 -118.44360474752774) bank18481 +18482 POINT(34.04323551966487 -118.31663807969947) bank18482 +18483 POINT(34.232175058448824 -118.82921735357668) bank18483 +18484 POINT(34.494217382866566 -118.333975674624) bank18484 +18485 POINT(34.319183076216625 -118.73841755900973) bank18485 +18486 POINT(33.367935860228506 -118.20425435692604) bank18486 +18487 POINT(34.795712220835014 -118.42980510469614) bank18487 +18488 POINT(33.0949396769636 -118.82703687246247) bank18488 +18489 POINT(34.67361298523059 -119.13118388139414) bank18489 +18490 POINT(33.16104053154894 -117.65089638919736) bank18490 +18491 POINT(34.068457170098796 -117.43616505421144) bank18491 +18492 POINT(34.05724143614348 -117.9524306178344) bank18492 +18493 POINT(34.86344216011452 -119.06966643316771) bank18493 +18494 POINT(34.878152698671535 -119.10812878464198) bank18494 +18495 POINT(33.71991843705218 -117.54988775136052) bank18495 +18496 POINT(34.519938062258795 -118.67543297612775) bank18496 +18497 POINT(34.64653839189205 -117.53537161612401) bank18497 +18498 POINT(33.07761709316022 -117.60567998130938) bank18498 +18499 POINT(33.850563379207976 -119.17188416579893) bank18499 +18500 POINT(34.42981068967886 -117.60369198933188) bank18500 +18501 POINT(34.57622955907664 -118.4727934834647) bank18501 +18502 POINT(33.3226455551045 -117.80277980990462) bank18502 +18503 POINT(33.212413627636636 -118.88428190580144) bank18503 +18504 POINT(34.401195744172796 -118.8254694648846) bank18504 +18505 POINT(34.84745231203093 -117.83289166829053) bank18505 +18506 POINT(33.940214517097594 -117.80005410682652) bank18506 +18507 POINT(34.31849381527077 -117.74287868003503) bank18507 +18508 POINT(33.21776353852605 -117.69476858160695) bank18508 +18509 POINT(33.930492381280054 -117.4538959109641) bank18509 +18510 POINT(33.08658568747067 -118.02875591758406) bank18510 +18511 POINT(33.947958995712646 -118.18991697869477) bank18511 +18512 POINT(34.04317131526152 -118.35223505526461) bank18512 +18513 POINT(34.99840348328067 -119.03458938983232) bank18513 +18514 POINT(33.21542090085985 -117.8339582502776) bank18514 +18515 POINT(34.158899091659315 -117.57747219272937) bank18515 +18516 POINT(33.6648972918261 -117.41442163812495) bank18516 +18517 POINT(34.056587311578554 -118.97182124804604) bank18517 +18518 POINT(33.16904791848934 -118.33847460356553) bank18518 +18519 POINT(33.29090347510222 -118.31625702296792) bank18519 +18520 POINT(34.11992359979332 -119.0826715806226) bank18520 +18521 POINT(33.558147859414554 -117.65299670909782) bank18521 +18522 POINT(34.19988462152497 -118.8383663777201) bank18522 +18523 POINT(34.597574226487964 -118.26079359373806) bank18523 +18524 POINT(34.1877796033974 -119.01814328263544) bank18524 +18525 POINT(34.79503750782524 -119.04476584058344) bank18525 +18526 POINT(33.133461350363085 -117.98895802708127) bank18526 +18527 POINT(33.47207991862039 -118.15778905746824) bank18527 +18528 POINT(33.08677807591465 -117.65152553983481) bank18528 +18529 POINT(33.06439403185802 -117.47277958295236) bank18529 +18530 POINT(33.70420734771443 -117.74675073968133) bank18530 +18531 POINT(34.14093597872016 -118.86641394977454) bank18531 +18532 POINT(34.25206115905538 -118.00678758951977) bank18532 +18533 POINT(33.74710398249306 -118.82644559371624) bank18533 +18534 POINT(33.34851031935899 -117.4583262470385) bank18534 +18535 POINT(33.53205544038161 -117.90889882268785) bank18535 +18536 POINT(34.318576394226874 -118.57825262399021) bank18536 +18537 POINT(33.15749458394202 -118.32949022097185) bank18537 +18538 POINT(33.63959800908403 -117.86292928969259) bank18538 +18539 POINT(33.29208627007144 -117.30058652939121) bank18539 +18540 POINT(35.03438225682315 -118.15141328869244) bank18540 +18541 POINT(33.62625852935733 -118.20644686025852) bank18541 +18542 POINT(33.78070145592617 -118.29702674491708) bank18542 +18543 POINT(34.63788627535004 -117.47739837608393) bank18543 +18544 POINT(33.3318516074973 -117.34126597709933) bank18544 +18545 POINT(34.838866524270195 -117.49850367124533) bank18545 +18546 POINT(34.83918998820864 -118.36604221152274) bank18546 +18547 POINT(33.35508881594079 -118.34422766498436) bank18547 +18548 POINT(33.64181144078744 -117.8680587904882) bank18548 +18549 POINT(34.3634639263936 -117.92534321172917) bank18549 +18550 POINT(34.14879362598361 -118.67444997157102) bank18550 +18551 POINT(34.38184562336254 -118.54670436982207) bank18551 +18552 POINT(34.81156465639551 -118.6635587946579) bank18552 +18553 POINT(33.65621902142112 -117.96533075087314) bank18553 +18554 POINT(33.65025458174277 -119.12587774177138) bank18554 +18555 POINT(34.79390165321958 -119.02864372635217) bank18555 +18556 POINT(33.77962258024128 -119.0830538475692) bank18556 +18557 POINT(34.27374066241396 -117.37049498788872) bank18557 +18558 POINT(34.122490683464086 -118.36964336462786) bank18558 +18559 POINT(33.497506700526586 -118.25517730307834) bank18559 +18560 POINT(34.048612575345246 -117.88735631348523) bank18560 +18561 POINT(34.85691664721488 -118.06089387078386) bank18561 +18562 POINT(33.97624667407827 -118.74433048130393) bank18562 +18563 POINT(33.74859892595838 -118.04737746081483) bank18563 +18564 POINT(34.41555084178381 -119.1268339408017) bank18564 +18565 POINT(33.24693763876024 -118.32633118230441) bank18565 +18566 POINT(33.99181282824669 -118.77544149585407) bank18566 +18567 POINT(34.617515342840456 -118.20242418062973) bank18567 +18568 POINT(33.07990490418382 -118.17253503583844) bank18568 +18569 POINT(33.7115552442072 -118.06126364100203) bank18569 +18570 POINT(34.874681642914844 -117.28079483119764) bank18570 +18571 POINT(33.496619049165396 -117.9870649867559) bank18571 +18572 POINT(33.13221426791828 -118.43685209683805) bank18572 +18573 POINT(34.98047326449675 -118.9507473730901) bank18573 +18574 POINT(33.33739927781534 -118.02891030214865) bank18574 +18575 POINT(33.34768023884416 -118.71310540630277) bank18575 +18576 POINT(34.182042943686966 -119.12935043431938) bank18576 +18577 POINT(34.882402649608586 -119.18007074089516) bank18577 +18578 POINT(33.869755777702395 -117.46834311861078) bank18578 +18579 POINT(34.50410188982743 -117.35533728413955) bank18579 +18580 POINT(34.051911197133265 -117.89828945720426) bank18580 +18581 POINT(34.06269389056214 -118.7139537695851) bank18581 +18582 POINT(34.70284576370057 -117.53078370267636) bank18582 +18583 POINT(35.022380822829106 -118.10177844003893) bank18583 +18584 POINT(34.230403680161196 -118.9038756527054) bank18584 +18585 POINT(34.09646809746045 -118.11211060889808) bank18585 +18586 POINT(33.50764708232178 -118.9516005244867) bank18586 +18587 POINT(33.58291868054992 -117.68350507188164) bank18587 +18588 POINT(33.57888383728205 -117.78224091312013) bank18588 +18589 POINT(33.99137073272309 -118.24433256662833) bank18589 +18590 POINT(33.33513233771892 -117.2794605847747) bank18590 +18591 POINT(33.81344589726826 -117.96490521332528) bank18591 +18592 POINT(34.81947898077409 -117.66419220087275) bank18592 +18593 POINT(33.467546484838635 -118.57967206600534) bank18593 +18594 POINT(33.30857108272029 -117.74771389603498) bank18594 +18595 POINT(33.65967112886072 -118.75255396979387) bank18595 +18596 POINT(34.80920495570962 -117.57228858361614) bank18596 +18597 POINT(33.17985441935503 -117.79085670990902) bank18597 +18598 POINT(33.10648133545005 -117.6343626647709) bank18598 +18599 POINT(34.92973916584526 -118.22896603489764) bank18599 +18600 POINT(34.66118045969259 -118.69503251255891) bank18600 +18601 POINT(33.30787638885383 -118.26054682727278) bank18601 +18602 POINT(33.77652324684935 -118.49030510298446) bank18602 +18603 POINT(33.95131859578368 -118.98420149546634) bank18603 +18604 POINT(34.89779725570213 -118.88300166561781) bank18604 +18605 POINT(33.34223682439476 -119.17120622226575) bank18605 +18606 POINT(34.8531875602844 -119.03965726414829) bank18606 +18607 POINT(34.749394498473954 -118.02833026979476) bank18607 +18608 POINT(34.91383012970771 -117.3642737919013) bank18608 +18609 POINT(34.764837222621 -117.4339565214812) bank18609 +18610 POINT(34.721573824851156 -117.93647912425669) bank18610 +18611 POINT(34.54223109114064 -118.05373141426377) bank18611 +18612 POINT(34.1389690106088 -119.09023363367704) bank18612 +18613 POINT(33.35588174071257 -118.20931293444302) bank18613 +18614 POINT(34.5766160964261 -118.57606586374382) bank18614 +18615 POINT(33.33021244790173 -118.23011315831333) bank18615 +18616 POINT(33.22485201498354 -118.57762034940885) bank18616 +18617 POINT(34.24098751479018 -118.16010349748883) bank18617 +18618 POINT(34.7477827714436 -119.10970716555556) bank18618 +18619 POINT(34.23468646637521 -117.98921801774772) bank18619 +18620 POINT(34.99827181083687 -117.46728664981315) bank18620 +18621 POINT(33.54796743252512 -117.33048107440462) bank18621 +18622 POINT(33.46798228551845 -117.24848267878363) bank18622 +18623 POINT(34.88316347342653 -118.87046156533206) bank18623 +18624 POINT(34.66673360621312 -119.23708824162) bank18624 +18625 POINT(34.63971807589872 -117.38329261222792) bank18625 +18626 POINT(34.29027740365343 -118.76581937209038) bank18626 +18627 POINT(34.93914537618757 -118.87471047346946) bank18627 +18628 POINT(33.88835427240594 -119.06557237196898) bank18628 +18629 POINT(34.93229571789314 -117.5352936462888) bank18629 +18630 POINT(33.419392452521166 -118.55005041945495) bank18630 +18631 POINT(34.5782690226592 -119.2213579571847) bank18631 +18632 POINT(33.05826261318665 -118.11061211741475) bank18632 +18633 POINT(33.59510472878782 -117.71947775985922) bank18633 +18634 POINT(33.26114881790137 -118.26292416271258) bank18634 +18635 POINT(34.97034795315427 -119.12032197529103) bank18635 +18636 POINT(33.541711437450665 -117.82709682828356) bank18636 +18637 POINT(34.60336055053156 -117.48755608349882) bank18637 +18638 POINT(33.518955778419496 -118.8116463327166) bank18638 +18639 POINT(34.220476373786184 -118.1735460706577) bank18639 +18640 POINT(33.62556297123972 -118.04650517654183) bank18640 +18641 POINT(34.95473544227562 -118.91926535531061) bank18641 +18642 POINT(33.52140132946341 -118.09936641157066) bank18642 +18643 POINT(34.072497716897196 -118.55467465495668) bank18643 +18644 POINT(34.33128659531245 -118.62486506784339) bank18644 +18645 POINT(34.048869170177376 -118.46867752615915) bank18645 +18646 POINT(33.257769172350876 -117.6328954695728) bank18646 +18647 POINT(33.641450273693295 -117.51474952486004) bank18647 +18648 POINT(33.33461167965388 -118.3000910541632) bank18648 +18649 POINT(34.00452988125709 -118.12218937153104) bank18649 +18650 POINT(33.57671007299409 -117.75310247076057) bank18650 +18651 POINT(33.352278247068774 -117.83111505537421) bank18651 +18652 POINT(33.80544546286247 -119.22997238727297) bank18652 +18653 POINT(34.07373745665671 -119.22493789148629) bank18653 +18654 POINT(33.77840269141856 -118.20019291227598) bank18654 +18655 POINT(33.77787934458896 -118.29321061774004) bank18655 +18656 POINT(34.412784513877014 -118.51257300906552) bank18656 +18657 POINT(33.32199007109072 -118.44468556257462) bank18657 +18658 POINT(34.3419703599751 -119.20569679793682) bank18658 +18659 POINT(33.655166806629985 -118.15070694322478) bank18659 +18660 POINT(33.1480505133624 -119.11915857230633) bank18660 +18661 POINT(33.66155789298638 -118.35876666625052) bank18661 +18662 POINT(33.22457972074975 -119.04001954784697) bank18662 +18663 POINT(33.53820458875288 -118.70249757413833) bank18663 +18664 POINT(34.696283581870496 -117.98551504805118) bank18664 +18665 POINT(33.159891629449724 -117.89210912074383) bank18665 +18666 POINT(33.816843652430464 -119.09440948789855) bank18666 +18667 POINT(34.220416924443086 -118.17751887813559) bank18667 +18668 POINT(33.29089940196929 -118.02666434164539) bank18668 +18669 POINT(34.18572221175526 -118.82350481135241) bank18669 +18670 POINT(34.459002678215484 -117.63264726437765) bank18670 +18671 POINT(33.062992935752646 -118.49413001237156) bank18671 +18672 POINT(34.33433913577028 -117.69980666432755) bank18672 +18673 POINT(34.4150929531045 -118.62063503897647) bank18673 +18674 POINT(33.888218268365755 -118.18676400719617) bank18674 +18675 POINT(34.76749835168564 -118.40493429351321) bank18675 +18676 POINT(33.91627215872672 -118.50686093802634) bank18676 +18677 POINT(33.78824654777552 -117.76062822354773) bank18677 +18678 POINT(33.57301309420175 -119.10490916342356) bank18678 +18679 POINT(34.3557663374806 -118.94720863043419) bank18679 +18680 POINT(34.92169857382813 -118.1780447820761) bank18680 +18681 POINT(33.649982834676564 -118.21952559759717) bank18681 +18682 POINT(34.47149031534694 -118.41820544461676) bank18682 +18683 POINT(34.22927543241906 -117.68218737327332) bank18683 +18684 POINT(34.003125034304574 -118.86435427406904) bank18684 +18685 POINT(34.3073043056325 -117.60243934691962) bank18685 +18686 POINT(34.03521863112541 -118.97139958975745) bank18686 +18687 POINT(34.81003763324831 -119.19295522705768) bank18687 +18688 POINT(34.97503912404803 -118.06338479372893) bank18688 +18689 POINT(34.3223542264805 -118.34390404429529) bank18689 +18690 POINT(34.8467086805493 -119.21904188238474) bank18690 +18691 POINT(34.33544911448662 -118.31324133671951) bank18691 +18692 POINT(34.13617141019038 -117.97931949965866) bank18692 +18693 POINT(33.72443670154494 -117.29120259015549) bank18693 +18694 POINT(34.14247732103094 -117.54484834391751) bank18694 +18695 POINT(33.81164041112664 -118.77702470403254) bank18695 +18696 POINT(33.483099488418084 -118.30981414222494) bank18696 +18697 POINT(33.75354838935613 -118.73353023051247) bank18697 +18698 POINT(33.94032453329009 -118.99026168274915) bank18698 +18699 POINT(34.03056535883533 -118.78916464106186) bank18699 +18700 POINT(34.250606622247346 -117.36626956451332) bank18700 +18701 POINT(34.85649588675689 -118.32095301460903) bank18701 +18702 POINT(33.061277524261975 -118.07055109393053) bank18702 +18703 POINT(34.33475796183949 -117.45861169340262) bank18703 +18704 POINT(34.731566958528504 -118.58587579038965) bank18704 +18705 POINT(34.61809169520181 -118.35087105923508) bank18705 +18706 POINT(34.99514271077881 -118.70260536675308) bank18706 +18707 POINT(33.454589515719476 -118.57247423680943) bank18707 +18708 POINT(34.93621629122862 -117.91574435133596) bank18708 +18709 POINT(34.51305762735973 -118.76040848311459) bank18709 +18710 POINT(34.45472271686136 -118.42618073658078) bank18710 +18711 POINT(34.54655861672851 -117.84811521614178) bank18711 +18712 POINT(34.26323859477931 -118.87543163684245) bank18712 +18713 POINT(34.49525429738013 -118.14134601296087) bank18713 +18714 POINT(34.01461028678701 -118.66511145576823) bank18714 +18715 POINT(34.432221147866606 -117.34858757467002) bank18715 +18716 POINT(33.73745590577581 -118.74974456622961) bank18716 +18717 POINT(34.56339546160674 -118.88473110427503) bank18717 +18718 POINT(33.5504702066493 -118.50939608579618) bank18718 +18719 POINT(34.35002721829215 -117.6224762636804) bank18719 +18720 POINT(33.75042994549541 -118.21113664811075) bank18720 +18721 POINT(33.35266376041239 -117.4530620839683) bank18721 +18722 POINT(34.99082118201752 -118.43888924745403) bank18722 +18723 POINT(34.58673204355238 -117.46616450664884) bank18723 +18724 POINT(34.66045384330942 -117.66951650568487) bank18724 +18725 POINT(34.85109524080928 -117.51274836387465) bank18725 +18726 POINT(34.725588986705304 -118.195727284651) bank18726 +18727 POINT(34.34132698181906 -117.72306919711363) bank18727 +18728 POINT(33.17911847682976 -117.81336183076805) bank18728 +18729 POINT(33.915576104973766 -118.11816507587346) bank18729 +18730 POINT(33.671520774217555 -118.53758519239423) bank18730 +18731 POINT(33.599360698923505 -117.55921815425289) bank18731 +18732 POINT(33.80559113050723 -118.50080573824604) bank18732 +18733 POINT(34.64623851975475 -119.2201891450823) bank18733 +18734 POINT(34.39878810668864 -117.73711662872523) bank18734 +18735 POINT(33.46092697766759 -117.57868282185244) bank18735 +18736 POINT(34.2106989196311 -118.5589725627529) bank18736 +18737 POINT(33.12434053701631 -119.19326462498017) bank18737 +18738 POINT(34.17837592753461 -118.31533995405754) bank18738 +18739 POINT(34.382717054641915 -118.31515987881797) bank18739 +18740 POINT(34.29211749907225 -117.4179063695775) bank18740 +18741 POINT(33.412973261781765 -117.86181977804588) bank18741 +18742 POINT(33.15156423262982 -118.93124930347089) bank18742 +18743 POINT(34.75419095605167 -118.16358797706432) bank18743 +18744 POINT(34.95337350044603 -117.67330549695075) bank18744 +18745 POINT(33.103335978048946 -117.64423949759346) bank18745 +18746 POINT(34.7819427872096 -117.83111447474212) bank18746 +18747 POINT(34.70373325014152 -117.59405614548325) bank18747 +18748 POINT(33.65193507932007 -118.0076231959552) bank18748 +18749 POINT(35.051664539031115 -118.01118380175845) bank18749 +18750 POINT(33.40585970817293 -117.6949113139194) bank18750 +18751 POINT(34.41936059186254 -118.79409631868816) bank18751 +18752 POINT(34.44421757567676 -119.21072521193378) bank18752 +18753 POINT(33.13749523471385 -118.71647349508585) bank18753 +18754 POINT(34.93685598971267 -117.57270966856522) bank18754 +18755 POINT(34.856850810501385 -118.30461792358506) bank18755 +18756 POINT(33.341673696607536 -118.1949154544395) bank18756 +18757 POINT(34.921693000659054 -118.84597640984829) bank18757 +18758 POINT(34.03257351020094 -117.79938194531704) bank18758 +18759 POINT(34.90880493554171 -118.65394175305816) bank18759 +18760 POINT(34.67879171534754 -118.83831346780688) bank18760 +18761 POINT(34.496568877618365 -118.20925922548416) bank18761 +18762 POINT(33.45188070357625 -119.16804734179826) bank18762 +18763 POINT(33.38867967628804 -119.22569973788326) bank18763 +18764 POINT(34.172695072218914 -118.01886126548459) bank18764 +18765 POINT(34.24666846917145 -118.96116626319603) bank18765 +18766 POINT(33.36759439557405 -117.43274505766912) bank18766 +18767 POINT(34.34437387561373 -118.84763991290906) bank18767 +18768 POINT(33.98065127614994 -117.61033768739577) bank18768 +18769 POINT(34.90588229910226 -118.34066742049579) bank18769 +18770 POINT(35.00643909629619 -117.6765994418954) bank18770 +18771 POINT(33.51039395528759 -117.84829799944593) bank18771 +18772 POINT(34.23736327079489 -117.70401183861996) bank18772 +18773 POINT(33.156297802166144 -117.85135950550213) bank18773 +18774 POINT(34.40873605386953 -117.62259465256368) bank18774 +18775 POINT(34.79108452139438 -118.57210085686921) bank18775 +18776 POINT(34.16791008119766 -118.07392690698902) bank18776 +18777 POINT(34.97556317375796 -118.00915645406734) bank18777 +18778 POINT(35.02383698342591 -117.77109370382556) bank18778 +18779 POINT(34.58321449589152 -118.97403156344149) bank18779 +18780 POINT(34.05376865915542 -119.02121446942384) bank18780 +18781 POINT(33.59581330577729 -118.63034452477834) bank18781 +18782 POINT(35.03518319744993 -117.89766430477707) bank18782 +18783 POINT(34.63479622956805 -118.6933606624358) bank18783 +18784 POINT(33.16711352798063 -118.75604838386924) bank18784 +18785 POINT(34.786593285236236 -118.34716893620998) bank18785 +18786 POINT(33.4627774595092 -117.53661725485219) bank18786 +18787 POINT(34.66369437287786 -118.04832180445187) bank18787 +18788 POINT(33.37432923607418 -118.24814333975173) bank18788 +18789 POINT(34.821399777045286 -118.17098137675218) bank18789 +18790 POINT(33.863987773918026 -118.27203790624954) bank18790 +18791 POINT(35.049985016984806 -118.52369981498123) bank18791 +18792 POINT(34.22344991127807 -117.57526652108287) bank18792 +18793 POINT(34.729063982397356 -118.8411015867544) bank18793 +18794 POINT(34.099535190198694 -119.15256938698377) bank18794 +18795 POINT(33.71342991226431 -117.2853917329806) bank18795 +18796 POINT(34.40277872981734 -118.93120989205579) bank18796 +18797 POINT(34.3206627906514 -119.23324065184326) bank18797 +18798 POINT(33.68809716025534 -117.54063997833849) bank18798 +18799 POINT(34.72820007580176 -117.26341753534457) bank18799 +18800 POINT(33.69986618601989 -117.54172668488229) bank18800 +18801 POINT(34.02707460876667 -119.13232989714288) bank18801 +18802 POINT(33.97111800944481 -118.8404208295485) bank18802 +18803 POINT(33.58129900042949 -117.97325523440009) bank18803 +18804 POINT(34.04445569536791 -117.8599970407584) bank18804 +18805 POINT(34.11463284036027 -118.54142814572391) bank18805 +18806 POINT(34.45223653638777 -117.38632840010483) bank18806 +18807 POINT(34.43959217011372 -118.17074395211132) bank18807 +18808 POINT(34.86274703164435 -117.51224717206014) bank18808 +18809 POINT(34.168084079175095 -117.80079971953595) bank18809 +18810 POINT(34.799250969641164 -117.48393343404925) bank18810 +18811 POINT(34.726120211145286 -117.76293753296561) bank18811 +18812 POINT(33.774942359587705 -119.11537893152516) bank18812 +18813 POINT(33.953617437141894 -118.76732883566838) bank18813 +18814 POINT(33.74566712292279 -119.21744889185145) bank18814 +18815 POINT(34.60629815931468 -119.2090228555321) bank18815 +18816 POINT(34.141391367475535 -118.5179150366664) bank18816 +18817 POINT(33.167296406162855 -119.18965891628082) bank18817 +18818 POINT(33.129280488438496 -118.4532500496077) bank18818 +18819 POINT(33.44771602620515 -118.06101972021946) bank18819 +18820 POINT(33.75599075287637 -118.06447274101689) bank18820 +18821 POINT(34.65613023063051 -118.26073233054582) bank18821 +18822 POINT(33.916954153248106 -118.37809107750314) bank18822 +18823 POINT(34.39578350679994 -118.06520241176513) bank18823 +18824 POINT(34.62517434742616 -117.80443951320729) bank18824 +18825 POINT(33.22706288440314 -117.60015770493919) bank18825 +18826 POINT(34.426958441591395 -117.41561743694567) bank18826 +18827 POINT(34.73021253821192 -119.18385595099426) bank18827 +18828 POINT(34.36009875439731 -118.99143912745255) bank18828 +18829 POINT(34.77088839852982 -118.55830249977525) bank18829 +18830 POINT(34.81365987468494 -117.7616472394673) bank18830 +18831 POINT(33.776076265085194 -118.27935038470386) bank18831 +18832 POINT(33.67930572161287 -117.25147388721255) bank18832 +18833 POINT(33.11152409939088 -117.31377650464174) bank18833 +18834 POINT(33.06089173695717 -117.80103290369946) bank18834 +18835 POINT(34.96560359729046 -118.26780372278749) bank18835 +18836 POINT(33.104295869895644 -118.25778613380896) bank18836 +18837 POINT(34.651691919491604 -117.96307842027774) bank18837 +18838 POINT(35.01641891307881 -117.54537216636524) bank18838 +18839 POINT(34.57125870366202 -118.42409726403706) bank18839 +18840 POINT(34.21276805744989 -118.56360614552747) bank18840 +18841 POINT(34.60484867696675 -117.84651852775703) bank18841 +18842 POINT(34.54292569634739 -119.08557544227337) bank18842 +18843 POINT(34.98696594474704 -118.8154104786905) bank18843 +18844 POINT(33.34207723456032 -118.3766368652143) bank18844 +18845 POINT(33.13978002649135 -118.65955454244272) bank18845 +18846 POINT(33.177739841427645 -118.0329044864242) bank18846 +18847 POINT(34.905965941389766 -118.8543497120252) bank18847 +18848 POINT(33.94077365222535 -118.73810311817245) bank18848 +18849 POINT(33.82086478754916 -118.76923885939141) bank18849 +18850 POINT(34.12740634275747 -117.34910294135082) bank18850 +18851 POINT(33.47156052266169 -117.40928752428673) bank18851 +18852 POINT(34.97685728448576 -118.50227669677358) bank18852 +18853 POINT(34.2365838328548 -118.00772658211352) bank18853 +18854 POINT(34.63531318399201 -118.06412658444526) bank18854 +18855 POINT(34.55706742345353 -117.47074631818079) bank18855 +18856 POINT(34.532642392571894 -118.93084347250041) bank18856 +18857 POINT(34.09882875462473 -117.78028574328341) bank18857 +18858 POINT(33.13616995808352 -118.88716024172132) bank18858 +18859 POINT(34.03998569799148 -117.68555253612882) bank18859 +18860 POINT(33.45605273495486 -118.43006948249744) bank18860 +18861 POINT(34.26158861753932 -118.91252967164155) bank18861 +18862 POINT(33.61210221535378 -117.8440463913577) bank18862 +18863 POINT(33.50061510892937 -117.76333326592169) bank18863 +18864 POINT(34.04152152264706 -117.60779492286433) bank18864 +18865 POINT(33.92959967207178 -118.12273625107444) bank18865 +18866 POINT(35.02022581086036 -117.74178975639705) bank18866 +18867 POINT(33.3427298682974 -118.77183568887693) bank18867 +18868 POINT(34.177026156913605 -118.3002671522099) bank18868 +18869 POINT(34.38836350214306 -118.8341011328711) bank18869 +18870 POINT(34.5323796965662 -118.74861829544129) bank18870 +18871 POINT(33.54080685236145 -118.43709753054038) bank18871 +18872 POINT(33.175679095583696 -117.62761614487435) bank18872 +18873 POINT(34.398737991280235 -118.30713829269982) bank18873 +18874 POINT(33.236500081315526 -117.9411963867521) bank18874 +18875 POINT(34.18319884852072 -117.6809911030582) bank18875 +18876 POINT(33.41197616744236 -117.80776731553615) bank18876 +18877 POINT(34.627585309283795 -118.78586630621552) bank18877 +18878 POINT(34.27644835048764 -118.88508362256715) bank18878 +18879 POINT(34.75600603022604 -117.96537705796214) bank18879 +18880 POINT(33.583626705555425 -118.07846836571392) bank18880 +18881 POINT(33.42442378007705 -118.03297313200676) bank18881 +18882 POINT(34.643830272640685 -118.98253375107188) bank18882 +18883 POINT(33.73013631344881 -119.22649215539764) bank18883 +18884 POINT(34.278240210722196 -118.77050986856753) bank18884 +18885 POINT(34.87626166135691 -118.66559618439786) bank18885 +18886 POINT(33.73673878298604 -117.92133072159356) bank18886 +18887 POINT(35.03095495810204 -117.6380164446012) bank18887 +18888 POINT(34.86134265582287 -119.17817453084405) bank18888 +18889 POINT(33.30993330678633 -118.18410783200957) bank18889 +18890 POINT(34.83471301379578 -118.40805085128346) bank18890 +18891 POINT(33.70684322542031 -117.49462687648696) bank18891 +18892 POINT(34.285606672681865 -117.56340231990907) bank18892 +18893 POINT(33.35982650841695 -117.8573642027053) bank18893 +18894 POINT(33.175124883915544 -118.81000912013101) bank18894 +18895 POINT(33.24267173486412 -117.60791240439444) bank18895 +18896 POINT(34.63591402314903 -117.7945606090676) bank18896 +18897 POINT(34.63642867040705 -117.41230790489865) bank18897 +18898 POINT(33.27958268059593 -118.68533160719012) bank18898 +18899 POINT(34.973257576953046 -117.52797911340215) bank18899 +18900 POINT(34.2183420075991 -118.1367323816497) bank18900 +18901 POINT(34.01189503003115 -118.38504074603097) bank18901 +18902 POINT(33.81011106115959 -118.1256341475024) bank18902 +18903 POINT(34.478260761049064 -118.79113227300726) bank18903 +18904 POINT(34.26080522355156 -119.07296937504107) bank18904 +18905 POINT(33.48263596485496 -117.48200713453426) bank18905 +18906 POINT(34.59711145832769 -118.83937477953452) bank18906 +18907 POINT(33.698392185227824 -119.16551361801132) bank18907 +18908 POINT(34.133589499646106 -119.22000712977082) bank18908 +18909 POINT(33.62636540442342 -118.09620917080912) bank18909 +18910 POINT(34.135230188019065 -117.86865776630651) bank18910 +18911 POINT(33.73987789627625 -117.39426087367481) bank18911 +18912 POINT(33.74813857718475 -117.2923869726922) bank18912 +18913 POINT(33.968143959636635 -117.65336060015586) bank18913 +18914 POINT(33.20039143127965 -117.98779198679952) bank18914 +18915 POINT(33.917303872434864 -117.62689891548989) bank18915 +18916 POINT(34.87218791497005 -118.195611810814) bank18916 +18917 POINT(33.14332025164067 -118.35292232683823) bank18917 +18918 POINT(34.6714439078783 -119.14306454417145) bank18918 +18919 POINT(34.83507131674089 -118.04298549008867) bank18919 +18920 POINT(33.30161089818713 -118.97615207230807) bank18920 +18921 POINT(34.347775380747535 -117.78888877112782) bank18921 +18922 POINT(33.23121987855867 -118.22357202553899) bank18922 +18923 POINT(33.57272196619537 -118.43667921841818) bank18923 +18924 POINT(33.11379218868725 -117.55728136114101) bank18924 +18925 POINT(33.123575201055566 -117.6300007981025) bank18925 +18926 POINT(34.187106798156435 -118.52972273911969) bank18926 +18927 POINT(34.53450855963522 -118.94009496666999) bank18927 +18928 POINT(33.83987370308029 -117.50877574330906) bank18928 +18929 POINT(33.82717391288353 -117.30286939862123) bank18929 +18930 POINT(34.355568237923976 -118.46918568266635) bank18930 +18931 POINT(33.67568007668352 -118.96345816648821) bank18931 +18932 POINT(33.114290510218794 -119.21480439883712) bank18932 +18933 POINT(34.26724316877088 -118.17125408155368) bank18933 +18934 POINT(33.53807996028322 -118.44249930664483) bank18934 +18935 POINT(34.816375591187 -118.82037382301773) bank18935 +18936 POINT(33.43689328777566 -117.87197426677089) bank18936 +18937 POINT(34.09643972671157 -117.5156426723362) bank18937 +18938 POINT(33.623811215478554 -118.82213851065504) bank18938 +18939 POINT(33.32371964627585 -117.43641613923374) bank18939 +18940 POINT(34.13873865132551 -117.89677639719764) bank18940 +18941 POINT(34.74149601897645 -117.34183080458547) bank18941 +18942 POINT(33.50589603191124 -117.27829201519039) bank18942 +18943 POINT(34.11583491664839 -117.53657817884785) bank18943 +18944 POINT(34.24824957372432 -117.6515145177045) bank18944 +18945 POINT(34.52109849315396 -117.97996582105705) bank18945 +18946 POINT(34.306276833242045 -117.25492565727536) bank18946 +18947 POINT(34.25122296977582 -118.03299978835908) bank18947 +18948 POINT(33.781630927499094 -118.66471211106882) bank18948 +18949 POINT(33.13197975984588 -119.09725405245982) bank18949 +18950 POINT(34.51721603347048 -118.23684928585473) bank18950 +18951 POINT(33.36639063640445 -117.449236695861) bank18951 +18952 POINT(34.45241459755677 -118.26848347984564) bank18952 +18953 POINT(34.10398381883712 -117.50928598159054) bank18953 +18954 POINT(34.31785013658752 -117.99712566957756) bank18954 +18955 POINT(34.16558470841096 -117.8485122885868) bank18955 +18956 POINT(33.21720372706599 -117.73595607076777) bank18956 +18957 POINT(34.11944211279337 -119.07310245744466) bank18957 +18958 POINT(33.88000711712847 -117.82556781586295) bank18958 +18959 POINT(34.782764181140294 -117.45736014739731) bank18959 +18960 POINT(33.067678228819 -117.46200405683632) bank18960 +18961 POINT(33.29769963005934 -119.1960745542099) bank18961 +18962 POINT(34.344562301231456 -119.22557263323627) bank18962 +18963 POINT(34.49567193976361 -118.35851734231649) bank18963 +18964 POINT(33.78044481595646 -118.9838054567322) bank18964 +18965 POINT(33.58305668887664 -117.82033895476707) bank18965 +18966 POINT(34.691678682940825 -118.35337159684615) bank18966 +18967 POINT(34.72847919416112 -118.82055173588267) bank18967 +18968 POINT(33.13850714892286 -118.20916425166749) bank18968 +18969 POINT(34.83254938522092 -118.20390548330789) bank18969 +18970 POINT(34.54747750145373 -118.68137537705684) bank18970 +18971 POINT(34.65539255820829 -118.7525222365588) bank18971 +18972 POINT(33.40396527294146 -117.79500294958333) bank18972 +18973 POINT(34.51018928506528 -118.00402454183364) bank18973 +18974 POINT(34.66456991839249 -117.42539612508344) bank18974 +18975 POINT(34.16485505758147 -117.71823147715415) bank18975 +18976 POINT(33.79669879314451 -119.13104981722779) bank18976 +18977 POINT(34.37103350837674 -118.41407438200841) bank18977 +18978 POINT(34.739148075468904 -119.06475469058678) bank18978 +18979 POINT(33.82577211579863 -118.22748039358052) bank18979 +18980 POINT(34.89633855922418 -117.83955847772431) bank18980 +18981 POINT(34.017675025715356 -118.58259875120851) bank18981 +18982 POINT(33.239959489813536 -119.11783918963263) bank18982 +18983 POINT(34.2550867274756 -117.35438433917128) bank18983 +18984 POINT(33.13881776673854 -117.32456987877501) bank18984 +18985 POINT(33.070673012990824 -118.65921666251994) bank18985 +18986 POINT(34.48119845799247 -118.97373242233873) bank18986 +18987 POINT(33.865024817329 -117.70845587731414) bank18987 +18988 POINT(34.86886930433687 -118.36165714924775) bank18988 +18989 POINT(33.91322002420129 -118.82494061008505) bank18989 +18990 POINT(33.99834067142187 -118.81866300399972) bank18990 +18991 POINT(34.514335154034164 -119.11710562665122) bank18991 +18992 POINT(35.044680304160636 -118.60458093959551) bank18992 +18993 POINT(33.67722786237834 -118.84537403472187) bank18993 +18994 POINT(33.64506651799367 -118.05126610454779) bank18994 +18995 POINT(33.115562511119116 -118.26317612188524) bank18995 +18996 POINT(34.420717152180806 -117.63318134688417) bank18996 +18997 POINT(34.904236093870075 -118.95154265033969) bank18997 +18998 POINT(34.86171382416412 -118.55961752596448) bank18998 +18999 POINT(33.42605261933199 -118.695839247903) bank18999 +19000 POINT(34.256457547877055 -118.91592603077491) bank19000 +19001 POINT(33.42225176865441 -119.02900028353115) bank19001 +19002 POINT(33.801387366392895 -118.77204969262706) bank19002 +19003 POINT(33.706459135191025 -117.52209640925963) bank19003 +19004 POINT(34.74760395738966 -119.1208468010615) bank19004 +19005 POINT(34.628308091639404 -118.38492277195299) bank19005 +19006 POINT(33.45549515177988 -119.00827144819269) bank19006 +19007 POINT(33.589608629090385 -117.52058335478092) bank19007 +19008 POINT(34.20018420097296 -118.86524276462208) bank19008 +19009 POINT(34.36941235076037 -118.37799853844541) bank19009 +19010 POINT(33.274366289457355 -118.84772697376148) bank19010 +19011 POINT(34.00919132950246 -119.19538538446281) bank19011 +19012 POINT(33.56306993206233 -118.02381632659112) bank19012 +19013 POINT(33.26842386938787 -117.39164458050024) bank19013 +19014 POINT(34.81721126427407 -117.63205575058413) bank19014 +19015 POINT(35.037460969206506 -118.77232044506891) bank19015 +19016 POINT(33.23884089555324 -117.91303457435698) bank19016 +19017 POINT(34.50525052943797 -118.7445782760764) bank19017 +19018 POINT(34.80777290699322 -117.78617630723693) bank19018 +19019 POINT(33.382657130772614 -117.51233894571347) bank19019 +19020 POINT(34.89717104711481 -119.06161158727286) bank19020 +19021 POINT(33.09620735547821 -118.48668240167527) bank19021 +19022 POINT(33.31662228023752 -118.46185424087352) bank19022 +19023 POINT(33.92533715665275 -118.78727057174237) bank19023 +19024 POINT(34.3842844987198 -118.59516109152976) bank19024 +19025 POINT(34.50100167279194 -117.73441763934669) bank19025 +19026 POINT(33.070741066857046 -119.22412995433663) bank19026 +19027 POINT(34.85250598864136 -118.51692713839667) bank19027 +19028 POINT(34.983763405704295 -118.93969702167718) bank19028 +19029 POINT(33.1847073258871 -118.31329669866817) bank19029 +19030 POINT(33.6504308102014 -118.50836899900148) bank19030 +19031 POINT(34.20664765466067 -118.34762903934914) bank19031 +19032 POINT(34.98964349844535 -119.01361462789555) bank19032 +19033 POINT(34.54665729720741 -118.52794551248455) bank19033 +19034 POINT(33.56489651527563 -117.95403877747297) bank19034 +19035 POINT(34.847008087362234 -117.7488973948626) bank19035 +19036 POINT(34.473623600305906 -118.001234674068) bank19036 +19037 POINT(33.97812684282413 -118.54584230334244) bank19037 +19038 POINT(33.09708713037101 -118.93280941755029) bank19038 +19039 POINT(34.545630192520356 -118.66953607305933) bank19039 +19040 POINT(34.90645424175242 -118.60245949229991) bank19040 +19041 POINT(35.03766861231775 -117.26889508428351) bank19041 +19042 POINT(34.853050784843454 -117.96774075935758) bank19042 +19043 POINT(33.05962344347232 -118.4545465530653) bank19043 +19044 POINT(33.83262664288094 -117.56926687645834) bank19044 +19045 POINT(33.84372525037228 -118.47980784779772) bank19045 +19046 POINT(34.47669208949869 -118.34190680525215) bank19046 +19047 POINT(33.38819443478601 -119.04269015595925) bank19047 +19048 POINT(33.59463266229574 -118.54640211507032) bank19048 +19049 POINT(33.079825884075944 -117.699916820001) bank19049 +19050 POINT(33.23151949674365 -117.61786299221664) bank19050 +19051 POINT(34.950621185085865 -118.42611396888948) bank19051 +19052 POINT(33.10910141960266 -117.32600153229694) bank19052 +19053 POINT(34.32300803384507 -117.29746374104583) bank19053 +19054 POINT(34.07590354761428 -118.42849749893593) bank19054 +19055 POINT(33.643176608750686 -117.78655757391066) bank19055 +19056 POINT(33.58482217423978 -118.38785845096417) bank19056 +19057 POINT(33.75734295781894 -118.42769627163496) bank19057 +19058 POINT(34.06396586355527 -118.26793415930219) bank19058 +19059 POINT(33.219308843415554 -118.45347302885769) bank19059 +19060 POINT(33.42600192589132 -118.51079057495697) bank19060 +19061 POINT(34.76949761766137 -118.38979319447489) bank19061 +19062 POINT(33.27093457560619 -118.22161047100309) bank19062 +19063 POINT(33.97808972943039 -118.41874410854476) bank19063 +19064 POINT(33.67742814292762 -118.65611149469603) bank19064 +19065 POINT(34.12265382294515 -117.38655362223753) bank19065 +19066 POINT(34.161432509001024 -117.81011395286171) bank19066 +19067 POINT(34.78655154993855 -117.60724752438263) bank19067 +19068 POINT(33.71805195608072 -118.34793569348211) bank19068 +19069 POINT(34.91823369019499 -117.87757063610441) bank19069 +19070 POINT(33.87316771349023 -118.79910464110118) bank19070 +19071 POINT(33.70034535556038 -118.20363849858494) bank19071 +19072 POINT(33.41459213216722 -118.6550833099878) bank19072 +19073 POINT(33.40826590901567 -119.18296026404727) bank19073 +19074 POINT(35.026372597965874 -118.35663167419482) bank19074 +19075 POINT(34.100985520427805 -117.65979430605982) bank19075 +19076 POINT(33.99474423233326 -118.25228254250308) bank19076 +19077 POINT(33.50958819439303 -118.32801948947959) bank19077 +19078 POINT(33.955211585694116 -117.79090268965396) bank19078 +19079 POINT(33.12336223535107 -117.42366346550634) bank19079 +19080 POINT(33.08304844221286 -117.91301794014615) bank19080 +19081 POINT(33.29975462259189 -118.1201000412694) bank19081 +19082 POINT(34.417245596686186 -118.77645271911975) bank19082 +19083 POINT(33.25006811965123 -117.37130811631137) bank19083 +19084 POINT(34.888215817171506 -118.42156036818263) bank19084 +19085 POINT(34.9405063929155 -119.18660909687569) bank19085 +19086 POINT(34.90411171126557 -118.45434350310595) bank19086 +19087 POINT(34.41061397939679 -117.91422152239123) bank19087 +19088 POINT(34.876196298468116 -118.98940598010618) bank19088 +19089 POINT(33.13108355291712 -117.56862272277444) bank19089 +19090 POINT(33.403572827062575 -118.83265149388217) bank19090 +19091 POINT(34.70841235012429 -117.76257881219938) bank19091 +19092 POINT(34.15252261937386 -118.76426424938607) bank19092 +19093 POINT(34.68965694191753 -118.82357656570065) bank19093 +19094 POINT(34.97929711866901 -118.06888346444) bank19094 +19095 POINT(34.13957140817836 -119.04991280516526) bank19095 +19096 POINT(34.41308040434073 -118.92385628842645) bank19096 +19097 POINT(34.676032420903056 -117.8002181037847) bank19097 +19098 POINT(34.37130421588469 -118.66824414319534) bank19098 +19099 POINT(33.86105661091459 -118.82130042173486) bank19099 +19100 POINT(33.7964780203164 -119.05346350653821) bank19100 +19101 POINT(34.758849876885456 -118.41506029360526) bank19101 +19102 POINT(33.632525538127155 -119.12444648908615) bank19102 +19103 POINT(34.62356177396537 -119.03186649108979) bank19103 +19104 POINT(34.99341209590579 -119.10707168558244) bank19104 +19105 POINT(33.052686209690535 -117.30113110025049) bank19105 +19106 POINT(34.10985066271533 -118.09306975474426) bank19106 +19107 POINT(33.80634820400892 -118.46507465905371) bank19107 +19108 POINT(34.62538452160422 -118.6962073217622) bank19108 +19109 POINT(33.44823249874556 -117.88805527626218) bank19109 +19110 POINT(33.847691193991544 -118.89586218101981) bank19110 +19111 POINT(34.757475115362325 -117.40142826274096) bank19111 +19112 POINT(33.948519121876394 -117.48325578910953) bank19112 +19113 POINT(33.44107998555445 -117.56660638087791) bank19113 +19114 POINT(34.824344815246924 -118.91206715392757) bank19114 +19115 POINT(33.7022597732481 -118.05542489515176) bank19115 +19116 POINT(34.66940169871909 -118.64791296938948) bank19116 +19117 POINT(33.60143634902236 -117.68190442032278) bank19117 +19118 POINT(34.4542942616913 -117.79732905048407) bank19118 +19119 POINT(33.20317191344986 -118.99487645352393) bank19119 +19120 POINT(33.492990858550705 -118.49684732383668) bank19120 +19121 POINT(33.35022076303832 -117.56705804892769) bank19121 +19122 POINT(33.24350199239401 -118.05677540580423) bank19122 +19123 POINT(34.23790043429211 -117.48830878040395) bank19123 +19124 POINT(33.270914963516 -118.41445034806318) bank19124 +19125 POINT(35.04645499364659 -117.72271323254337) bank19125 +19126 POINT(34.59376322506218 -119.04044523603764) bank19126 +19127 POINT(33.80012790894031 -118.18989563190425) bank19127 +19128 POINT(34.74956140071512 -118.00740079289024) bank19128 +19129 POINT(33.49811617015681 -118.96210928797075) bank19129 +19130 POINT(34.893033425062654 -117.52977078526763) bank19130 +19131 POINT(34.308856412160644 -117.50800711002007) bank19131 +19132 POINT(34.85592061197013 -117.59840743674005) bank19132 +19133 POINT(33.24950110209353 -117.96041226375316) bank19133 +19134 POINT(34.413569253471934 -117.29302808352703) bank19134 +19135 POINT(33.28517692456626 -118.3880244074788) bank19135 +19136 POINT(33.82570383088008 -118.327154252755) bank19136 +19137 POINT(34.82634864519288 -117.98064149478152) bank19137 +19138 POINT(33.32433259159807 -118.80374621296036) bank19138 +19139 POINT(34.86334414542197 -118.01362856184473) bank19139 +19140 POINT(34.26336621073138 -118.01193246304895) bank19140 +19141 POINT(34.163552362933636 -117.6050167274451) bank19141 +19142 POINT(33.102963220976676 -118.55747665028508) bank19142 +19143 POINT(33.713532403481935 -117.38167153559263) bank19143 +19144 POINT(33.25873664089507 -117.5017092902848) bank19144 +19145 POINT(34.33373114051315 -119.02264066770367) bank19145 +19146 POINT(33.43587352260103 -117.44062581934864) bank19146 +19147 POINT(34.601545148825565 -119.11641843660131) bank19147 +19148 POINT(34.61539441509876 -119.02296938700906) bank19148 +19149 POINT(34.03565519244503 -117.48834407752834) bank19149 +19150 POINT(34.883284047869495 -118.63864641182236) bank19150 +19151 POINT(34.68965512639321 -117.74091597785423) bank19151 +19152 POINT(33.531509385066116 -118.8761364039215) bank19152 +19153 POINT(33.5596728657805 -118.96267557783014) bank19153 +19154 POINT(33.45424297891161 -118.25782270721372) bank19154 +19155 POINT(34.05073597314116 -118.05657745617468) bank19155 +19156 POINT(33.77635062694328 -117.35308801141164) bank19156 +19157 POINT(34.12017059185201 -117.71312738698411) bank19157 +19158 POINT(34.41688642660581 -118.20620580227704) bank19158 +19159 POINT(35.00554700219669 -118.10197136966336) bank19159 +19160 POINT(34.41245438089781 -118.50481959480445) bank19160 +19161 POINT(33.82950298834851 -117.45554256946133) bank19161 +19162 POINT(34.782315368851165 -117.63033000930965) bank19162 +19163 POINT(33.08387802961275 -118.42273391283268) bank19163 +19164 POINT(33.61066227830994 -118.35354331668938) bank19164 +19165 POINT(34.745562056866405 -118.37953968813754) bank19165 +19166 POINT(34.091909071016275 -119.06172675569373) bank19166 +19167 POINT(33.526015528832076 -117.25143887529357) bank19167 +19168 POINT(33.76720219070002 -118.08003389323419) bank19168 +19169 POINT(34.26390694198565 -117.7211251244729) bank19169 +19170 POINT(33.82697111723301 -118.61366740671804) bank19170 +19171 POINT(34.632146592234214 -117.89070864167665) bank19171 +19172 POINT(34.229334978084296 -118.67552990458199) bank19172 +19173 POINT(34.795834961999645 -118.51172592162425) bank19173 +19174 POINT(34.31801349734507 -119.00512127160181) bank19174 +19175 POINT(34.94814295433063 -118.645930982669) bank19175 +19176 POINT(33.79548526067085 -118.58173576833893) bank19176 +19177 POINT(33.85808793582584 -117.91139865220613) bank19177 +19178 POINT(33.44406805783876 -117.76912407897822) bank19178 +19179 POINT(33.12939595475888 -119.11903173665503) bank19179 +19180 POINT(34.14002580632423 -119.2265760898334) bank19180 +19181 POINT(34.38978960674072 -117.70161827968677) bank19181 +19182 POINT(34.298596503329016 -117.63594145395223) bank19182 +19183 POINT(33.551317241436216 -119.10068833634823) bank19183 +19184 POINT(33.31057400492723 -118.59380030087594) bank19184 +19185 POINT(33.23755921604621 -118.15566629302893) bank19185 +19186 POINT(33.47777704591798 -118.46628044557313) bank19186 +19187 POINT(33.30528898734971 -118.23985728176866) bank19187 +19188 POINT(34.64831431810014 -118.22737823206558) bank19188 +19189 POINT(33.93780054527881 -118.20470069516844) bank19189 +19190 POINT(33.74546376723877 -117.32386698457535) bank19190 +19191 POINT(34.06030902468274 -119.21781966458198) bank19191 +19192 POINT(33.28626059666648 -118.69709875574922) bank19192 +19193 POINT(34.480368301010785 -117.98703110163261) bank19193 +19194 POINT(33.96585937031273 -118.79719033391216) bank19194 +19195 POINT(34.98598111655886 -117.55261422443877) bank19195 +19196 POINT(34.308065730318276 -118.72626309246077) bank19196 +19197 POINT(33.852607402814435 -117.29234608843285) bank19197 +19198 POINT(33.202417338625985 -117.78899688227645) bank19198 +19199 POINT(34.944859018997136 -117.97217627535666) bank19199 +19200 POINT(33.589392041416524 -118.4699140360873) bank19200 +19201 POINT(34.49195633717391 -118.63786301680699) bank19201 +19202 POINT(33.39837067832773 -118.45443565415933) bank19202 +19203 POINT(34.079042416224496 -118.39226530921918) bank19203 +19204 POINT(33.0822677934311 -117.8747175390362) bank19204 +19205 POINT(35.03114140470188 -119.17079367838879) bank19205 +19206 POINT(33.47109092268988 -117.89196268115461) bank19206 +19207 POINT(35.019598507428675 -117.73768483661492) bank19207 +19208 POINT(34.70347208188914 -117.51392123061743) bank19208 +19209 POINT(33.112970625309764 -117.87681111399569) bank19209 +19210 POINT(34.32200505544196 -119.14213734837658) bank19210 +19211 POINT(34.81833280391092 -117.3272294769572) bank19211 +19212 POINT(33.53380853512913 -119.03724601441597) bank19212 +19213 POINT(33.20582648762977 -117.82550258681975) bank19213 +19214 POINT(34.17103940493138 -117.84539977912513) bank19214 +19215 POINT(35.03280017644188 -117.81504046097801) bank19215 +19216 POINT(34.46233772599985 -117.72387096606055) bank19216 +19217 POINT(34.53662042344078 -118.82042340544399) bank19217 +19218 POINT(35.049194859788116 -118.18675280707599) bank19218 +19219 POINT(34.56470764798641 -117.29190534644204) bank19219 +19220 POINT(34.80543103902858 -117.37869175348388) bank19220 +19221 POINT(33.96379082014576 -118.78497156457506) bank19221 +19222 POINT(33.45868649314174 -117.41060892686113) bank19222 +19223 POINT(33.397863224055584 -117.94122835362542) bank19223 +19224 POINT(34.29924403470056 -118.34893511207108) bank19224 +19225 POINT(35.05138572530722 -118.4311721356563) bank19225 +19226 POINT(33.05846888447598 -118.52558546789743) bank19226 +19227 POINT(34.92749831040801 -117.3451667210011) bank19227 +19228 POINT(33.131973269839094 -118.46780231817785) bank19228 +19229 POINT(34.0173363336209 -118.34073516557014) bank19229 +19230 POINT(34.11153132046994 -117.45997658153507) bank19230 +19231 POINT(34.10119415377085 -117.95112121151804) bank19231 +19232 POINT(33.950035419290145 -119.07243934781214) bank19232 +19233 POINT(34.46131346502708 -119.19814151608183) bank19233 +19234 POINT(33.90636797649787 -117.79789513473939) bank19234 +19235 POINT(33.658500955157166 -117.6039228905594) bank19235 +19236 POINT(34.25501371854728 -117.38937142003006) bank19236 +19237 POINT(33.64219291943433 -118.2605553422767) bank19237 +19238 POINT(33.994742754817295 -118.47611132861331) bank19238 +19239 POINT(34.50836196504707 -118.40423758529043) bank19239 +19240 POINT(34.86489844469566 -118.06714927805648) bank19240 +19241 POINT(33.270209188117185 -118.14515104323245) bank19241 +19242 POINT(34.3236709694117 -117.67477620838162) bank19242 +19243 POINT(34.993650026222866 -117.90756899652708) bank19243 +19244 POINT(33.62380516108835 -118.05611726022549) bank19244 +19245 POINT(33.540548093761004 -118.23536647129913) bank19245 +19246 POINT(35.03852272586818 -117.48573710449814) bank19246 +19247 POINT(34.140163405425795 -117.48978984129799) bank19247 +19248 POINT(34.080662607143466 -117.33409452970514) bank19248 +19249 POINT(34.8089090282134 -117.46123105132752) bank19249 +19250 POINT(34.82470957051242 -117.68492044434277) bank19250 +19251 POINT(34.156918271524916 -117.67895903356339) bank19251 +19252 POINT(34.84399975189654 -117.9745460642646) bank19252 +19253 POINT(33.90026096186952 -118.90619412757349) bank19253 +19254 POINT(34.45369690408793 -118.42525876545692) bank19254 +19255 POINT(34.207943542116055 -118.0604773393583) bank19255 +19256 POINT(34.003852819540924 -118.29912235444283) bank19256 +19257 POINT(33.90197987932365 -117.45071358932648) bank19257 +19258 POINT(34.181734482191175 -117.75161732833709) bank19258 +19259 POINT(33.533038975687916 -117.92892895879986) bank19259 +19260 POINT(33.80742928814626 -118.53346907664799) bank19260 +19261 POINT(34.331512570066 -118.00909818958625) bank19261 +19262 POINT(34.31860458155478 -119.08444312208178) bank19262 +19263 POINT(33.31568274919465 -118.29585212325867) bank19263 +19264 POINT(33.07517561178876 -117.93073440566857) bank19264 +19265 POINT(34.016442197560856 -118.45218332332868) bank19265 +19266 POINT(35.047616434589 -118.16918558383351) bank19266 +19267 POINT(34.228464262521804 -118.05916880599095) bank19267 +19268 POINT(33.97725839951375 -117.84586810801004) bank19268 +19269 POINT(34.52018473728946 -117.43253741466447) bank19269 +19270 POINT(34.43040526954846 -119.1767080583774) bank19270 +19271 POINT(33.56957411060499 -119.22128081051932) bank19271 +19272 POINT(34.96890820608765 -118.5551988744943) bank19272 +19273 POINT(33.19330981713815 -119.17613641890011) bank19273 +19274 POINT(34.27170310454044 -117.5347474981723) bank19274 +19275 POINT(33.762316388526635 -118.80561172215145) bank19275 +19276 POINT(35.046101667103635 -118.37969023710637) bank19276 +19277 POINT(34.8670642324131 -119.08693394631062) bank19277 +19278 POINT(34.469362634036735 -118.84752187559702) bank19278 +19279 POINT(33.99284062416056 -118.30017262197026) bank19279 +19280 POINT(33.617241473061945 -118.9474152591138) bank19280 +19281 POINT(34.68299235303853 -117.55985846196438) bank19281 +19282 POINT(33.917890578508334 -118.5175413090028) bank19282 +19283 POINT(34.82526565577567 -118.99648360882998) bank19283 +19284 POINT(34.56479328926274 -118.59662461452207) bank19284 +19285 POINT(33.073336747984804 -119.21140556256182) bank19285 +19286 POINT(34.09610626900191 -118.29903504437057) bank19286 +19287 POINT(33.39968786195912 -118.27218965412379) bank19287 +19288 POINT(33.48278648358586 -117.73341876667104) bank19288 +19289 POINT(33.332616599393525 -118.20563911863641) bank19289 +19290 POINT(34.54669086190199 -117.27000224345647) bank19290 +19291 POINT(34.75090095210959 -117.29129497758876) bank19291 +19292 POINT(33.157903031187175 -117.45424837311859) bank19292 +19293 POINT(34.61208743447355 -117.86283801587139) bank19293 +19294 POINT(34.172332130951084 -118.26947904397741) bank19294 +19295 POINT(33.82853933883266 -118.41715221456298) bank19295 +19296 POINT(34.66732872037402 -118.13107038194406) bank19296 +19297 POINT(33.43796892337108 -118.61478089009219) bank19297 +19298 POINT(34.271235656362435 -118.58346013051319) bank19298 +19299 POINT(34.05152794751713 -118.00746324126352) bank19299 +19300 POINT(34.42274143035364 -118.2085984974782) bank19300 +19301 POINT(34.54299500760151 -117.49601977507115) bank19301 +19302 POINT(33.567970381322226 -118.70324804424614) bank19302 +19303 POINT(34.722110580823134 -118.90356822424806) bank19303 +19304 POINT(33.100633524828446 -119.18279011657683) bank19304 +19305 POINT(33.31807964027682 -118.06756624354773) bank19305 +19306 POINT(34.175378289234565 -117.46429931805892) bank19306 +19307 POINT(33.39074015835953 -118.86657478152482) bank19307 +19308 POINT(33.4700588028305 -117.7533270719098) bank19308 +19309 POINT(34.114257136921 -117.39895269235052) bank19309 +19310 POINT(34.31258017742452 -118.25648281981418) bank19310 +19311 POINT(34.06173361750142 -118.86948984036029) bank19311 +19312 POINT(34.81670194567084 -117.6169894068575) bank19312 +19313 POINT(33.593321221695284 -119.18783205109746) bank19313 +19314 POINT(34.3803327546152 -117.42323189576108) bank19314 +19315 POINT(33.87448645453801 -118.27642445692709) bank19315 +19316 POINT(34.15238327832339 -118.46125078705312) bank19316 +19317 POINT(33.329210951768054 -117.52050990224795) bank19317 +19318 POINT(34.7432149429178 -117.62339605153286) bank19318 +19319 POINT(34.04029565400641 -119.06911216180409) bank19319 +19320 POINT(33.547044384114244 -118.71663735315614) bank19320 +19321 POINT(34.83406905796399 -117.9556490216226) bank19321 +19322 POINT(33.87827337619909 -119.03948468115271) bank19322 +19323 POINT(33.69314836394894 -117.96512155534313) bank19323 +19324 POINT(33.85366073912859 -117.84756129007974) bank19324 +19325 POINT(33.96814725956211 -118.61572664800076) bank19325 +19326 POINT(34.78667227640289 -118.09048657231736) bank19326 +19327 POINT(34.80691446924487 -118.61915345913339) bank19327 +19328 POINT(33.95577015978232 -117.64609985213505) bank19328 +19329 POINT(34.6014037016699 -118.79548921735622) bank19329 +19330 POINT(33.30803446294717 -118.70436266428246) bank19330 +19331 POINT(34.977061646109796 -118.05015638737308) bank19331 +19332 POINT(33.33255102117212 -118.24746481404766) bank19332 +19333 POINT(34.1696015813121 -119.08670789833981) bank19333 +19334 POINT(33.73250680104438 -117.90343481493039) bank19334 +19335 POINT(34.371664098091756 -118.2824332328873) bank19335 +19336 POINT(34.5567151996466 -118.72153262453946) bank19336 +19337 POINT(34.16474857407223 -117.57247986892328) bank19337 +19338 POINT(33.30555578242817 -117.69834894954266) bank19338 +19339 POINT(33.94635955168127 -117.29436473611918) bank19339 +19340 POINT(33.79532801963757 -118.6501509029339) bank19340 +19341 POINT(33.53194590844658 -117.63763352630572) bank19341 +19342 POINT(34.20062807211957 -118.33972731872498) bank19342 +19343 POINT(33.68232837769277 -117.79975659260984) bank19343 +19344 POINT(33.31463582125967 -117.76391255321792) bank19344 +19345 POINT(34.33630479573271 -117.89052833686257) bank19345 +19346 POINT(33.75857147081861 -117.48345902268018) bank19346 +19347 POINT(33.75487293158456 -117.25237910400631) bank19347 +19348 POINT(34.41343468476011 -118.77386861236646) bank19348 +19349 POINT(34.76786743475919 -118.21986865013255) bank19349 +19350 POINT(33.70933099030661 -118.63942561565877) bank19350 +19351 POINT(34.53843953979398 -119.24260487357436) bank19351 +19352 POINT(33.11929538545337 -119.154120237985) bank19352 +19353 POINT(33.74099224203941 -119.05785016220015) bank19353 +19354 POINT(34.182314080472 -117.74712402324757) bank19354 +19355 POINT(33.81837592417756 -117.73449516954967) bank19355 +19356 POINT(33.41439471374374 -117.45123307654843) bank19356 +19357 POINT(33.87909294469848 -117.48183286341201) bank19357 +19358 POINT(34.73897713879749 -119.15235245698614) bank19358 +19359 POINT(34.19937738217257 -117.98152922440367) bank19359 +19360 POINT(34.37209845596286 -119.21163821985017) bank19360 +19361 POINT(33.566349029998094 -119.21528118944332) bank19361 +19362 POINT(33.23896354892886 -117.4924924339651) bank19362 +19363 POINT(33.20043825996704 -118.70937410006766) bank19363 +19364 POINT(34.151870677604144 -118.56841577557854) bank19364 +19365 POINT(33.84430200038909 -117.70758151193812) bank19365 +19366 POINT(34.49733941449566 -118.31819254788591) bank19366 +19367 POINT(34.31773015902147 -117.88147938517224) bank19367 +19368 POINT(34.524841943780714 -117.4378998637367) bank19368 +19369 POINT(34.00508339721018 -117.67908305704724) bank19369 +19370 POINT(33.920854483729485 -119.13810966658869) bank19370 +19371 POINT(34.39810373819709 -117.41209018474011) bank19371 +19372 POINT(33.73493210631581 -118.15969017485588) bank19372 +19373 POINT(34.43600664931372 -118.68799447044019) bank19373 +19374 POINT(34.827556036319514 -118.48719187182441) bank19374 +19375 POINT(33.17959827079537 -117.90255063660871) bank19375 +19376 POINT(33.43416057841407 -118.0738147052655) bank19376 +19377 POINT(33.522000480883925 -117.54404232016867) bank19377 +19378 POINT(34.61731539640239 -117.67512191654936) bank19378 +19379 POINT(34.79073345687475 -117.68326820429255) bank19379 +19380 POINT(34.46945065182913 -118.03758232492082) bank19380 +19381 POINT(33.1060126591495 -118.07885472886055) bank19381 +19382 POINT(34.71775111382248 -118.066097376824) bank19382 +19383 POINT(34.003658403543845 -117.7911276337912) bank19383 +19384 POINT(34.0847894501254 -119.10590651673259) bank19384 +19385 POINT(34.50961285756498 -117.46729442908976) bank19385 +19386 POINT(34.13910445707341 -118.8246395379288) bank19386 +19387 POINT(34.00368912162746 -118.16812470360323) bank19387 +19388 POINT(34.22361922273808 -117.97672873395987) bank19388 +19389 POINT(34.83495968309426 -118.38874416537331) bank19389 +19390 POINT(33.639123367591594 -118.72344680874797) bank19390 +19391 POINT(34.16699008879601 -118.93474416709141) bank19391 +19392 POINT(34.96070581248573 -118.3721529087245) bank19392 +19393 POINT(34.75828338098448 -118.68638908263867) bank19393 +19394 POINT(34.38841123419462 -118.34672363420653) bank19394 +19395 POINT(33.16002984043105 -117.7103382165057) bank19395 +19396 POINT(34.80855564470013 -117.89052493136037) bank19396 +19397 POINT(34.6836094122084 -118.24349704985892) bank19397 +19398 POINT(34.88312093507354 -118.12296393542682) bank19398 +19399 POINT(33.06058323667172 -118.93680002607577) bank19399 +19400 POINT(34.03544256689998 -117.50373311072212) bank19400 +19401 POINT(34.04985218557508 -119.16265063764945) bank19401 +19402 POINT(33.57873646013175 -118.11724326886947) bank19402 +19403 POINT(33.321784856053924 -117.43259043422772) bank19403 +19404 POINT(35.04718738836049 -118.78937212274485) bank19404 +19405 POINT(33.53920534312675 -117.76635710178232) bank19405 +19406 POINT(34.54110151796728 -118.79840474311096) bank19406 +19407 POINT(34.82099894016308 -119.02607578234282) bank19407 +19408 POINT(33.81096454928766 -118.27854513872664) bank19408 +19409 POINT(34.82890674302572 -119.2059497805607) bank19409 +19410 POINT(33.306019564907125 -119.20862519553714) bank19410 +19411 POINT(33.77792052998152 -117.80413264417982) bank19411 +19412 POINT(34.997432984005556 -118.07020108051304) bank19412 +19413 POINT(33.35478270046566 -118.59236157060113) bank19413 +19414 POINT(33.39333158630931 -118.36085280422807) bank19414 +19415 POINT(33.73222424009221 -117.59562582351354) bank19415 +19416 POINT(34.098224527125666 -119.17007765324884) bank19416 +19417 POINT(34.913155007451714 -117.28245826673833) bank19417 +19418 POINT(34.73122921825628 -118.51398121365271) bank19418 +19419 POINT(33.44264425948121 -117.3880928193861) bank19419 +19420 POINT(33.567926305737146 -119.17108824834042) bank19420 +19421 POINT(34.94045956205429 -118.64998631620487) bank19421 +19422 POINT(33.216540914137575 -118.8793409436695) bank19422 +19423 POINT(34.36014660762027 -117.91008655727919) bank19423 +19424 POINT(34.118830888260014 -118.21492103108977) bank19424 +19425 POINT(34.34379259565333 -118.03862746745358) bank19425 +19426 POINT(33.688263723100405 -117.67050865722818) bank19426 +19427 POINT(33.776485859518395 -117.40741667254521) bank19427 +19428 POINT(34.38463986715398 -118.64778389338088) bank19428 +19429 POINT(33.73851191324425 -118.04118602516611) bank19429 +19430 POINT(33.729473519353085 -118.49182886030772) bank19430 +19431 POINT(34.87814950418274 -118.31015363953527) bank19431 +19432 POINT(33.32440326430511 -117.83665865412183) bank19432 +19433 POINT(34.8650080593422 -117.80896085463004) bank19433 +19434 POINT(33.23574413188271 -118.95311316570599) bank19434 +19435 POINT(34.87893292515696 -117.3855587277341) bank19435 +19436 POINT(34.49551776642276 -118.97037087652532) bank19436 +19437 POINT(34.00042597395786 -117.50203755092922) bank19437 +19438 POINT(33.41204319155379 -119.09722178177215) bank19438 +19439 POINT(33.86923761037375 -118.08466814041824) bank19439 +19440 POINT(33.26714449324859 -117.77907538408638) bank19440 +19441 POINT(34.95320397819424 -118.06917606338733) bank19441 +19442 POINT(34.012948108407585 -118.88100842060494) bank19442 +19443 POINT(34.072075705728025 -117.84318662895906) bank19443 +19444 POINT(34.81106687593619 -117.2585208681095) bank19444 +19445 POINT(34.3114435840417 -118.92210396636202) bank19445 +19446 POINT(34.52080395636948 -117.97322982761052) bank19446 +19447 POINT(34.4260193771077 -117.3024879011713) bank19447 +19448 POINT(34.26014006665141 -118.46513179161177) bank19448 +19449 POINT(33.0642451872803 -117.44667513850166) bank19449 +19450 POINT(34.05005500385912 -118.50079652346147) bank19450 +19451 POINT(34.73004700364437 -118.22593758690103) bank19451 +19452 POINT(33.754937250527504 -119.00786305148016) bank19452 +19453 POINT(33.36964815607606 -118.78668040899035) bank19453 +19454 POINT(34.673150949447916 -117.4655915684109) bank19454 +19455 POINT(34.06944671116541 -117.66616055921219) bank19455 +19456 POINT(34.81856354609238 -117.89097182299257) bank19456 +19457 POINT(33.94293851951605 -117.46526991199674) bank19457 +19458 POINT(35.01684408884758 -117.68826837571744) bank19458 +19459 POINT(34.58377270689916 -117.4077105484775) bank19459 +19460 POINT(34.75503348178699 -118.52684617527203) bank19460 +19461 POINT(33.426879813857646 -118.24831194626327) bank19461 +19462 POINT(34.590966834356664 -118.17373180083452) bank19462 +19463 POINT(33.9935253808346 -119.10198843061403) bank19463 +19464 POINT(33.40594785729412 -118.59410668754987) bank19464 +19465 POINT(33.94767766260467 -117.88025967346752) bank19465 +19466 POINT(34.92885501888578 -119.09969952181898) bank19466 +19467 POINT(33.860916657700464 -119.16851872193268) bank19467 +19468 POINT(33.762742726753594 -117.74535163331508) bank19468 +19469 POINT(33.65625766712026 -117.26093477057417) bank19469 +19470 POINT(33.52750552669696 -117.75216765387648) bank19470 +19471 POINT(34.3899715701191 -118.48723524609733) bank19471 +19472 POINT(33.536213670811776 -118.47778680285967) bank19472 +19473 POINT(33.449225713124 -117.68696223830263) bank19473 +19474 POINT(34.140099873745676 -118.2944909885001) bank19474 +19475 POINT(33.7334719880487 -119.11886039545804) bank19475 +19476 POINT(34.000443984037446 -117.6863094638565) bank19476 +19477 POINT(34.5408339883193 -118.4763286869564) bank19477 +19478 POINT(33.73233179415543 -117.89975116254466) bank19478 +19479 POINT(34.33677712547156 -117.52312545649899) bank19479 +19480 POINT(33.419488215778124 -117.53135257240952) bank19480 +19481 POINT(34.35887173216702 -118.32117422511038) bank19481 +19482 POINT(33.900458971679235 -118.76819327639741) bank19482 +19483 POINT(33.57868194064939 -118.2763374315682) bank19483 +19484 POINT(33.751627759870686 -118.5119431547319) bank19484 +19485 POINT(33.07182295192917 -119.21353141923402) bank19485 +19486 POINT(34.34688254026996 -117.9075374341046) bank19486 +19487 POINT(34.60209710633892 -117.9111812054118) bank19487 +19488 POINT(33.37642913322206 -117.83944965890622) bank19488 +19489 POINT(34.88210868527919 -118.27298406825645) bank19489 +19490 POINT(34.2777398450914 -117.75708311738663) bank19490 +19491 POINT(34.90588031501627 -118.60961645527217) bank19491 +19492 POINT(34.5891608845269 -118.89429071089594) bank19492 +19493 POINT(34.39440289916282 -118.89725279159364) bank19493 +19494 POINT(34.29178790588026 -117.98011549437598) bank19494 +19495 POINT(34.35070485451596 -118.73352377932864) bank19495 +19496 POINT(34.83225483692336 -117.50433851037883) bank19496 +19497 POINT(34.435624654886425 -117.57502308147566) bank19497 +19498 POINT(33.9758794655404 -118.28976175700214) bank19498 +19499 POINT(34.97539871840336 -118.12667536773608) bank19499 +19500 POINT(34.742470903644765 -118.17556862083592) bank19500 +19501 POINT(34.71009063515048 -118.68617885047266) bank19501 +19502 POINT(33.14545234240106 -117.44717993453428) bank19502 +19503 POINT(34.456012162242345 -118.28034620037182) bank19503 +19504 POINT(33.08995003499709 -118.3029613772051) bank19504 +19505 POINT(33.7070049775649 -117.38802359590858) bank19505 +19506 POINT(34.20278568410744 -118.58897331607373) bank19506 +19507 POINT(34.355563795177645 -117.33618647148751) bank19507 +19508 POINT(34.825359551327544 -117.35666767531134) bank19508 +19509 POINT(34.81852892390562 -117.33676240219627) bank19509 +19510 POINT(33.829255511173706 -117.65336211670504) bank19510 +19511 POINT(33.112504720878654 -118.17441352341578) bank19511 +19512 POINT(33.64518449385993 -117.60971324530828) bank19512 +19513 POINT(34.07891359221798 -118.07903104637991) bank19513 +19514 POINT(33.9554268187828 -118.4640526884956) bank19514 +19515 POINT(34.457131199126096 -117.97185171251851) bank19515 +19516 POINT(34.80984792369295 -118.41613571231356) bank19516 +19517 POINT(34.12868479549372 -117.92332648898109) bank19517 +19518 POINT(34.66815886946604 -118.37620664989169) bank19518 +19519 POINT(34.74339578370477 -118.43197457759312) bank19519 +19520 POINT(33.37606523523931 -118.12485429134598) bank19520 +19521 POINT(34.75271533910234 -118.63967825860618) bank19521 +19522 POINT(33.49792865105134 -118.6920497463394) bank19522 +19523 POINT(34.11924429463215 -117.99167597504446) bank19523 +19524 POINT(33.79680174499462 -117.53155654435129) bank19524 +19525 POINT(33.38290396510985 -118.61857535196948) bank19525 +19526 POINT(34.77485039444354 -117.68503207634217) bank19526 +19527 POINT(34.637770623696355 -117.85458225658535) bank19527 +19528 POINT(34.93723737643988 -118.15567516874266) bank19528 +19529 POINT(35.02960168376667 -118.1291493948341) bank19529 +19530 POINT(34.841186801181536 -117.50094334532535) bank19530 +19531 POINT(34.50888791385657 -118.91448659561857) bank19531 +19532 POINT(34.2057105544056 -118.8529756370102) bank19532 +19533 POINT(34.353238943844396 -118.55737099663631) bank19533 +19534 POINT(33.84828070096385 -117.9266189788795) bank19534 +19535 POINT(34.48531840498218 -118.78317151940622) bank19535 +19536 POINT(33.27967482234014 -117.42992562590098) bank19536 +19537 POINT(33.3281572966476 -117.77674983063329) bank19537 +19538 POINT(33.889914916761825 -118.39148296834232) bank19538 +19539 POINT(33.758488329054934 -117.65488201672832) bank19539 +19540 POINT(34.511457509655656 -117.85620475012274) bank19540 +19541 POINT(34.6969836720183 -118.36302890419293) bank19541 +19542 POINT(34.02945485437111 -117.6241815044406) bank19542 +19543 POINT(34.50231826587936 -119.12200390679936) bank19543 +19544 POINT(33.58111194757633 -118.66121935606355) bank19544 +19545 POINT(33.704493494160666 -117.55163851966056) bank19545 +19546 POINT(34.99154342248083 -117.35061482361446) bank19546 +19547 POINT(33.93223339114055 -117.5973578143074) bank19547 +19548 POINT(33.88018829403138 -117.80062232649637) bank19548 +19549 POINT(33.277789157649 -119.1614390751963) bank19549 +19550 POINT(33.29993200915847 -118.46083000570363) bank19550 +19551 POINT(33.32303055041501 -118.38663814416991) bank19551 +19552 POINT(34.42348288266622 -117.32071965534053) bank19552 +19553 POINT(33.60577420917451 -118.69731491597356) bank19553 +19554 POINT(34.04411015096924 -118.68031589535005) bank19554 +19555 POINT(34.26396508566449 -118.11670460223984) bank19555 +19556 POINT(34.8528648001366 -118.10256906842703) bank19556 +19557 POINT(34.56796553744072 -117.4725699819682) bank19557 +19558 POINT(33.17471516880377 -118.37298695469991) bank19558 +19559 POINT(33.485099852468146 -119.1570489799847) bank19559 +19560 POINT(34.93227223765712 -118.66805235792292) bank19560 +19561 POINT(33.64443661130074 -117.49313550274711) bank19561 +19562 POINT(33.482241392478684 -118.34036730537959) bank19562 +19563 POINT(33.18149860690437 -117.40358454796022) bank19563 +19564 POINT(33.19754097369938 -118.49833657356672) bank19564 +19565 POINT(33.53285021060525 -117.87118044035913) bank19565 +19566 POINT(33.259913954910395 -119.20469387495713) bank19566 +19567 POINT(33.120306412627954 -117.76304415687004) bank19567 +19568 POINT(33.207781917304004 -119.21020276515569) bank19568 +19569 POINT(33.58764975312523 -117.94591396308529) bank19569 +19570 POINT(34.24625473476493 -118.75345248188276) bank19570 +19571 POINT(34.943565304551235 -118.59271924435077) bank19571 +19572 POINT(34.572635251105346 -117.82323171105301) bank19572 +19573 POINT(34.8625386296267 -117.55664717168848) bank19573 +19574 POINT(33.914768940751095 -118.41415824181622) bank19574 +19575 POINT(33.46770417466412 -117.58980767708317) bank19575 +19576 POINT(34.31055154050712 -118.27947918031083) bank19576 +19577 POINT(34.54772076524208 -118.13654422166638) bank19577 +19578 POINT(33.42827215952585 -118.64283182419017) bank19578 +19579 POINT(34.876528890643165 -118.47575234313504) bank19579 +19580 POINT(34.62069318603017 -117.98793915312226) bank19580 +19581 POINT(34.26640101801895 -119.04428980766274) bank19581 +19582 POINT(33.06737749014223 -117.50158258074322) bank19582 +19583 POINT(33.239883953446665 -119.09078697219483) bank19583 +19584 POINT(34.07168218534866 -119.00176159565683) bank19584 +19585 POINT(34.58006340184783 -117.9248152919207) bank19585 +19586 POINT(33.72804489818246 -118.90728391278819) bank19586 +19587 POINT(33.07325123348155 -117.25253221802124) bank19587 +19588 POINT(34.86810962462976 -118.31370264261136) bank19588 +19589 POINT(34.79600485083769 -118.99676129958031) bank19589 +19590 POINT(34.13016936950178 -118.0626871138854) bank19590 +19591 POINT(33.057094078096476 -118.95156149137799) bank19591 +19592 POINT(34.7105776016247 -118.35482670441682) bank19592 +19593 POINT(34.231944456581964 -117.33814258347552) bank19593 +19594 POINT(34.59699494701786 -118.36688206313522) bank19594 +19595 POINT(34.5782774394872 -117.69843527167441) bank19595 +19596 POINT(34.521685338638214 -118.36150803866775) bank19596 +19597 POINT(33.58638782127853 -118.16438253571275) bank19597 +19598 POINT(33.4212572399258 -117.77382129630556) bank19598 +19599 POINT(34.70500883576725 -118.27348353375446) bank19599 +19600 POINT(34.93574179382475 -118.01195902364137) bank19600 +19601 POINT(33.07612648839211 -118.8477691864178) bank19601 +19602 POINT(34.313208486812904 -118.85577377022048) bank19602 +19603 POINT(33.56113876992792 -118.73672882741411) bank19603 +19604 POINT(35.00484585238645 -117.41232517081218) bank19604 +19605 POINT(34.079071782373234 -119.01978244664397) bank19605 +19606 POINT(34.70445381762132 -117.82265553921754) bank19606 +19607 POINT(33.76726284824105 -117.88114636514214) bank19607 +19608 POINT(33.30402993297289 -118.3165256620032) bank19608 +19609 POINT(35.00038183305683 -118.82220187038968) bank19609 +19610 POINT(33.43277656511464 -117.24405382548599) bank19610 +19611 POINT(34.222967755652796 -119.12559679875119) bank19611 +19612 POINT(34.396791001064386 -117.81413667046178) bank19612 +19613 POINT(33.33054041680953 -118.06463820732446) bank19613 +19614 POINT(34.058292428115394 -117.2705665459714) bank19614 +19615 POINT(33.41723043728703 -117.41941081630341) bank19615 +19616 POINT(34.80275683111731 -119.06548563193256) bank19616 +19617 POINT(33.70335526654929 -117.94001779978062) bank19617 +19618 POINT(33.254281163309315 -118.05965240145665) bank19618 +19619 POINT(34.303472156094216 -118.18613085343225) bank19619 +19620 POINT(33.98763954165704 -118.42628760715743) bank19620 +19621 POINT(33.13449051101485 -117.85386348124678) bank19621 +19622 POINT(33.811917916165655 -117.76023798956783) bank19622 +19623 POINT(34.269724696086044 -118.07836618007771) bank19623 +19624 POINT(34.73506670376033 -117.42722817745371) bank19624 +19625 POINT(34.34144850486407 -118.01945306890163) bank19625 +19626 POINT(34.346027431871754 -118.63485830825852) bank19626 +19627 POINT(34.79104512475987 -117.88108306228624) bank19627 +19628 POINT(33.40970427273563 -117.55249714015915) bank19628 +19629 POINT(33.73720561780496 -118.1738824940332) bank19629 +19630 POINT(34.32642661625036 -118.00391798191704) bank19630 +19631 POINT(34.360733573375214 -119.15180916869448) bank19631 +19632 POINT(34.89204747572018 -118.71858777195645) bank19632 +19633 POINT(34.4898847154264 -118.09469047717988) bank19633 +19634 POINT(34.88107925279966 -117.76876683193707) bank19634 +19635 POINT(34.87847628428535 -117.84771645737202) bank19635 +19636 POINT(34.06686887806587 -118.45306103742034) bank19636 +19637 POINT(33.773426992498386 -117.83002674442604) bank19637 +19638 POINT(33.471852113509044 -117.96350618047072) bank19638 +19639 POINT(34.2620585743537 -117.53756628406651) bank19639 +19640 POINT(34.55546989300299 -118.50956759487244) bank19640 +19641 POINT(33.870966331679014 -119.2026987986959) bank19641 +19642 POINT(34.255522931790615 -117.26616037803491) bank19642 +19643 POINT(33.520268212543606 -117.47879981739506) bank19643 +19644 POINT(33.10478023217039 -117.68279491907363) bank19644 +19645 POINT(33.86506482807758 -118.6414432121299) bank19645 +19646 POINT(34.543679829305404 -117.96191710261283) bank19646 +19647 POINT(33.639705254115654 -118.92384372200051) bank19647 +19648 POINT(33.066008850407414 -117.39183821879709) bank19648 +19649 POINT(34.841691220716086 -117.85217487776336) bank19649 +19650 POINT(33.88065750900028 -118.86075310726753) bank19650 +19651 POINT(34.142867666946245 -117.88252448721252) bank19651 +19652 POINT(33.83744553368665 -117.57176331618795) bank19652 +19653 POINT(33.189344358991775 -117.86393668170571) bank19653 +19654 POINT(33.66013360940416 -118.57839606618667) bank19654 +19655 POINT(33.64572380959566 -118.3416631363233) bank19655 +19656 POINT(33.51292970154884 -119.04898485481833) bank19656 +19657 POINT(33.431986819689286 -118.4983017478338) bank19657 +19658 POINT(34.69396724913623 -118.18418689924468) bank19658 +19659 POINT(33.64916608640328 -118.0166786608548) bank19659 +19660 POINT(33.75131762920306 -118.9120617586514) bank19660 +19661 POINT(33.2393125053585 -117.24371411287272) bank19661 +19662 POINT(34.2029692945293 -118.70828124847216) bank19662 +19663 POINT(33.73864555435173 -117.6460454941829) bank19663 +19664 POINT(33.614444068346614 -118.40616726460651) bank19664 +19665 POINT(33.907342100012485 -118.32313007348849) bank19665 +19666 POINT(34.43788232849406 -118.70226911158379) bank19666 +19667 POINT(33.998078786754775 -118.4289349291173) bank19667 +19668 POINT(34.10385690125755 -119.19300883508649) bank19668 +19669 POINT(34.50838501886566 -117.84745577717364) bank19669 +19670 POINT(34.855570498225994 -117.3285796862916) bank19670 +19671 POINT(34.87300349968107 -117.92076316967207) bank19671 +19672 POINT(33.93642082017464 -118.021285622209) bank19672 +19673 POINT(34.477290116948424 -118.00392067439911) bank19673 +19674 POINT(33.849547715243396 -117.62248376053907) bank19674 +19675 POINT(34.64325946518997 -118.29843758127564) bank19675 +19676 POINT(33.14191631688287 -118.11621262838217) bank19676 +19677 POINT(34.38167044044576 -119.02133365528866) bank19677 +19678 POINT(34.29023470375042 -117.99337469680474) bank19678 +19679 POINT(33.80351538856319 -118.80208155962782) bank19679 +19680 POINT(34.88981162116973 -117.76378517101757) bank19680 +19681 POINT(34.493879560255245 -118.58575285112359) bank19681 +19682 POINT(34.41893515559794 -118.15143458975405) bank19682 +19683 POINT(34.22419203773292 -118.64314321857789) bank19683 +19684 POINT(34.397252570660605 -117.38754788050827) bank19684 +19685 POINT(34.58040528833947 -118.29261826229504) bank19685 +19686 POINT(33.23392747882823 -117.25246451697758) bank19686 +19687 POINT(34.18836660107302 -118.51268424781745) bank19687 +19688 POINT(33.65905821832453 -117.36706424241814) bank19688 +19689 POINT(33.460106583229454 -117.85643806482715) bank19689 +19690 POINT(33.35179210724868 -118.5414516520955) bank19690 +19691 POINT(35.02064812069497 -118.52947759413786) bank19691 +19692 POINT(33.140440636822696 -118.04893558066314) bank19692 +19693 POINT(34.30613779963598 -117.90441715562797) bank19693 +19694 POINT(33.72758475468518 -117.70833824331332) bank19694 +19695 POINT(34.297236099111984 -118.02975381475707) bank19695 +19696 POINT(34.95683847128567 -118.8634839610628) bank19696 +19697 POINT(33.234205887699275 -117.60835103980736) bank19697 +19698 POINT(34.065508504251504 -119.10689191636214) bank19698 +19699 POINT(34.82510843171585 -117.41468317664021) bank19699 +19700 POINT(33.33391333483349 -119.09120091687811) bank19700 +19701 POINT(34.374505950659724 -118.66812857535305) bank19701 +19702 POINT(33.15545040052339 -117.29552877163925) bank19702 +19703 POINT(33.26336183117431 -118.64179776806651) bank19703 +19704 POINT(33.99666248997122 -119.15136248622353) bank19704 +19705 POINT(33.267765557468636 -119.0443851406727) bank19705 +19706 POINT(34.96913641676784 -117.97305262728196) bank19706 +19707 POINT(34.87506918014497 -118.14524787473141) bank19707 +19708 POINT(34.954689022456265 -117.931076395234) bank19708 +19709 POINT(34.19099497197158 -117.61479621618683) bank19709 +19710 POINT(34.75431805408905 -117.83343675754406) bank19710 +19711 POINT(33.2225458868394 -118.083955647462) bank19711 +19712 POINT(33.78794932136324 -117.34444747161626) bank19712 +19713 POINT(34.02932366670502 -117.84462844088316) bank19713 +19714 POINT(34.39232887165985 -118.4795662251682) bank19714 +19715 POINT(34.98476221616543 -118.100282610814) bank19715 +19716 POINT(34.979564711687615 -118.36330629069367) bank19716 +19717 POINT(34.45345864174699 -117.8403998948791) bank19717 +19718 POINT(34.18935890461185 -117.39840763523382) bank19718 +19719 POINT(33.7307140168225 -117.84497121607257) bank19719 +19720 POINT(33.16654732347693 -118.2614241363788) bank19720 +19721 POINT(33.15095174000883 -119.15487973583488) bank19721 +19722 POINT(33.809347417576774 -118.3469981765285) bank19722 +19723 POINT(33.50523489793803 -117.69093926911614) bank19723 +19724 POINT(34.30347277231337 -117.65401741355691) bank19724 +19725 POINT(33.686342090197265 -119.08690116203353) bank19725 +19726 POINT(33.31711761431379 -118.02204919937753) bank19726 +19727 POINT(34.66731786738583 -118.3364242149011) bank19727 +19728 POINT(33.83212284181143 -117.94727205903196) bank19728 +19729 POINT(33.888893542154804 -119.1436698306471) bank19729 +19730 POINT(34.42424345739104 -117.26547479627655) bank19730 +19731 POINT(34.49828274403671 -117.93368039009293) bank19731 +19732 POINT(35.04202615028414 -119.04023065907829) bank19732 +19733 POINT(34.284089894391755 -119.20955982289618) bank19733 +19734 POINT(33.3080882243993 -117.89724232725999) bank19734 +19735 POINT(33.919576220467846 -118.67789879758976) bank19735 +19736 POINT(34.717883680970424 -118.78913137120952) bank19736 +19737 POINT(34.272531678901245 -119.11627602747042) bank19737 +19738 POINT(33.3466887448609 -118.66472213589769) bank19738 +19739 POINT(33.25749891508575 -118.58137657652384) bank19739 +19740 POINT(33.05771257299488 -117.62872672673882) bank19740 +19741 POINT(33.51343202096213 -118.21878245161612) bank19741 +19742 POINT(34.608767664118794 -118.25999928102532) bank19742 +19743 POINT(34.28125889391638 -118.1802325116576) bank19743 +19744 POINT(34.135568155984046 -117.55446592800787) bank19744 +19745 POINT(34.08669378449534 -118.28620421293982) bank19745 +19746 POINT(34.36094448396983 -119.21800481990776) bank19746 +19747 POINT(33.84553199500862 -119.11329076914132) bank19747 +19748 POINT(33.71538296317253 -117.75863225085835) bank19748 +19749 POINT(34.87123793371391 -118.77382481482118) bank19749 +19750 POINT(34.16554173270148 -118.03248039264147) bank19750 +19751 POINT(34.45412118475867 -119.00493975072557) bank19751 +19752 POINT(33.245372727844426 -117.8740731395905) bank19752 +19753 POINT(33.14249538119645 -117.31795478481355) bank19753 +19754 POINT(34.5270174578672 -117.34723130257701) bank19754 +19755 POINT(34.89089595477543 -117.7811599878476) bank19755 +19756 POINT(34.799227649626566 -118.91101209675993) bank19756 +19757 POINT(34.59386828862885 -117.67555970347723) bank19757 +19758 POINT(33.834950725995135 -118.20719864784944) bank19758 +19759 POINT(33.859625683261875 -117.625255524372) bank19759 +19760 POINT(33.35208427273708 -118.33021892641788) bank19760 +19761 POINT(33.69466179011943 -118.25103860942217) bank19761 +19762 POINT(33.50860368392741 -118.4225533971249) bank19762 +19763 POINT(34.18383586134869 -117.56527827121798) bank19763 +19764 POINT(34.780614901159254 -118.63205003103249) bank19764 +19765 POINT(34.23857365977032 -117.53321006841858) bank19765 +19766 POINT(33.32252871415549 -119.0136970912415) bank19766 +19767 POINT(34.65416338807365 -118.66142111443978) bank19767 +19768 POINT(34.722888270972064 -117.57489042273767) bank19768 +19769 POINT(34.14374982373571 -118.59058156416032) bank19769 +19770 POINT(33.874471571489195 -119.10336828427786) bank19770 +19771 POINT(33.51108054220672 -118.64257715711103) bank19771 +19772 POINT(34.39710516038158 -118.83547050928765) bank19772 +19773 POINT(34.527753893544066 -118.28970608748133) bank19773 +19774 POINT(34.06302693038144 -117.28972084363733) bank19774 +19775 POINT(33.1309984461166 -118.65460525274347) bank19775 +19776 POINT(34.1889605495836 -117.73661759555196) bank19776 +19777 POINT(33.173576845199534 -118.4283442011878) bank19777 +19778 POINT(33.27355645955161 -118.09377507090147) bank19778 +19779 POINT(33.38143973388316 -118.85893619495548) bank19779 +19780 POINT(34.64335035378543 -118.666012168308) bank19780 +19781 POINT(34.007259231976455 -118.1267870024846) bank19781 +19782 POINT(33.502386207647014 -117.61739924176834) bank19782 +19783 POINT(35.04532771089125 -117.40137447931322) bank19783 +19784 POINT(33.354600739150406 -118.63910780204253) bank19784 +19785 POINT(34.58878999332496 -118.36981463746572) bank19785 +19786 POINT(33.35779631605304 -117.58537499505334) bank19786 +19787 POINT(33.82299838949001 -117.28050970437117) bank19787 +19788 POINT(33.42245512272048 -118.6608739314293) bank19788 +19789 POINT(33.08431225188624 -117.69358918904148) bank19789 +19790 POINT(34.610862330457174 -117.43704244123909) bank19790 +19791 POINT(33.36004571151646 -117.54038741890871) bank19791 +19792 POINT(34.14384827072907 -118.37056544784896) bank19792 +19793 POINT(33.718603609336455 -117.44892613096465) bank19793 +19794 POINT(33.5965876367099 -117.57189839528841) bank19794 +19795 POINT(34.582757031952326 -117.92561157386149) bank19795 +19796 POINT(33.62855421079551 -118.49409220515908) bank19796 +19797 POINT(34.460414200668566 -118.44682406502908) bank19797 +19798 POINT(34.48164976756536 -118.59636754063477) bank19798 +19799 POINT(34.03097245145684 -118.43517891055173) bank19799 +19800 POINT(33.63103358216142 -118.35131766250468) bank19800 +19801 POINT(33.938574008261305 -117.88012910607681) bank19801 +19802 POINT(34.460695000232064 -118.3626697331597) bank19802 +19803 POINT(34.643261925954505 -119.0769794341696) bank19803 +19804 POINT(33.66053817012774 -117.71760459037606) bank19804 +19805 POINT(34.69774144451149 -117.96156611544097) bank19805 +19806 POINT(34.875290026585155 -119.06148847102126) bank19806 +19807 POINT(33.98180861794847 -117.63448435105735) bank19807 +19808 POINT(34.6012166862807 -118.99657122964481) bank19808 +19809 POINT(34.5648499339061 -119.22223682313025) bank19809 +19810 POINT(33.41066998552478 -118.643902058065) bank19810 +19811 POINT(33.60372413008572 -117.8144642046077) bank19811 +19812 POINT(33.53807814121079 -117.29083216103707) bank19812 +19813 POINT(34.74749024003415 -118.38467517990725) bank19813 +19814 POINT(33.44468777126128 -118.76825363982813) bank19814 +19815 POINT(34.754987238186466 -118.14631781705327) bank19815 +19816 POINT(33.47460199086589 -119.06720235470922) bank19816 +19817 POINT(33.314080131361536 -117.55382673921916) bank19817 +19818 POINT(33.2580164063041 -118.79205974874421) bank19818 +19819 POINT(33.793966583206455 -117.61368699611884) bank19819 +19820 POINT(34.71586911029399 -117.41925219650129) bank19820 +19821 POINT(34.91867720685787 -118.80204688796212) bank19821 +19822 POINT(34.66357399193521 -118.32181866472261) bank19822 +19823 POINT(33.75083462352933 -117.6337412220654) bank19823 +19824 POINT(34.58548120928764 -118.64028480901007) bank19824 +19825 POINT(34.03495199610958 -117.32236328232393) bank19825 +19826 POINT(34.999465126293124 -118.51880020476207) bank19826 +19827 POINT(33.62913064604102 -118.39256985366391) bank19827 +19828 POINT(33.732436889113856 -117.9781361693422) bank19828 +19829 POINT(34.43246873957095 -117.3239510834813) bank19829 +19830 POINT(34.39271752250966 -118.35508539616268) bank19830 +19831 POINT(34.55657903440875 -117.69504937104568) bank19831 +19832 POINT(34.43636583341859 -118.51884577560091) bank19832 +19833 POINT(34.73105344857627 -118.18271229188203) bank19833 +19834 POINT(34.311997623706105 -117.92898757824628) bank19834 +19835 POINT(34.77886277494647 -118.94350936762174) bank19835 +19836 POINT(33.74043622313513 -117.76303110242625) bank19836 +19837 POINT(34.21822410867768 -117.37876144477468) bank19837 +19838 POINT(33.70108991553371 -117.99072880387907) bank19838 +19839 POINT(33.16924304536419 -118.68243107323933) bank19839 +19840 POINT(34.07873796165785 -118.74005082088878) bank19840 +19841 POINT(33.38802286943249 -119.09677354321546) bank19841 +19842 POINT(34.5360449123219 -119.11819811552755) bank19842 +19843 POINT(34.009914638165874 -118.3804966062424) bank19843 +19844 POINT(33.305636669742206 -118.75302990099676) bank19844 +19845 POINT(34.70793955196489 -117.33758001119456) bank19845 +19846 POINT(33.60916983462681 -118.07562201784289) bank19846 +19847 POINT(34.814969022014 -117.65111756488884) bank19847 +19848 POINT(34.60574377631695 -117.67535174142147) bank19848 +19849 POINT(33.96861983160025 -118.9486804203462) bank19849 +19850 POINT(34.098248978802374 -119.04946839334228) bank19850 +19851 POINT(34.912537573539076 -118.7477207729898) bank19851 +19852 POINT(34.90713646932216 -118.68127856330412) bank19852 +19853 POINT(33.10658959277599 -117.28182448257255) bank19853 +19854 POINT(34.968290275154104 -118.95643110107036) bank19854 +19855 POINT(33.40992485917274 -119.230296250025) bank19855 +19856 POINT(33.30673430522573 -118.32526768483298) bank19856 +19857 POINT(34.48586920700468 -118.65681886041175) bank19857 +19858 POINT(33.60548723521769 -117.31323747854249) bank19858 +19859 POINT(34.59406488761504 -117.43335819677675) bank19859 +19860 POINT(34.52112168070947 -118.74341319058405) bank19860 +19861 POINT(33.369942074932865 -119.00666193771978) bank19861 +19862 POINT(34.54948836487081 -118.93517792805575) bank19862 +19863 POINT(33.53023653097529 -117.73265370789495) bank19863 +19864 POINT(33.054376932310724 -117.53635336785332) bank19864 +19865 POINT(34.20331367968592 -118.50520212275991) bank19865 +19866 POINT(33.88848528161496 -118.4693086677494) bank19866 +19867 POINT(33.70837986197547 -119.0134999766759) bank19867 +19868 POINT(34.37179628737474 -117.8536069058699) bank19868 +19869 POINT(34.95810886021434 -118.43187051815899) bank19869 +19870 POINT(34.98545204588409 -117.34423494595275) bank19870 +19871 POINT(34.21764718227933 -118.7655053884348) bank19871 +19872 POINT(34.12810285008568 -118.67589961081623) bank19872 +19873 POINT(34.514959144720564 -118.82210938097556) bank19873 +19874 POINT(34.4472826232733 -118.69602829948045) bank19874 +19875 POINT(33.36192045890239 -117.90262323724828) bank19875 +19876 POINT(34.733908557017386 -117.87386963552491) bank19876 +19877 POINT(34.48782843230225 -119.09216183870203) bank19877 +19878 POINT(34.72096939742671 -118.08292706611752) bank19878 +19879 POINT(33.81272019669094 -117.5738290172583) bank19879 +19880 POINT(34.313152027698585 -118.31338306070712) bank19880 +19881 POINT(33.40747440154637 -117.33960491802249) bank19881 +19882 POINT(34.25684694964929 -118.46349347966037) bank19882 +19883 POINT(35.0434355224115 -117.93459693317877) bank19883 +19884 POINT(33.28811085722011 -117.79229472189483) bank19884 +19885 POINT(34.63025269585785 -119.0309691452123) bank19885 +19886 POINT(33.96618204179527 -118.72267064939781) bank19886 +19887 POINT(34.06725739991559 -117.68806850857185) bank19887 +19888 POINT(33.664784049259175 -117.72852821955371) bank19888 +19889 POINT(33.376723429397686 -117.73297295894946) bank19889 +19890 POINT(33.353224778410684 -118.83819126161437) bank19890 +19891 POINT(33.45523203293288 -117.84093501994447) bank19891 +19892 POINT(34.147796771267565 -117.43846897155997) bank19892 +19893 POINT(34.09371164704619 -118.83471484811756) bank19893 +19894 POINT(34.87798264259395 -117.31295808772083) bank19894 +19895 POINT(33.927017675923 -117.50856667023373) bank19895 +19896 POINT(34.04234470776269 -118.9486353547578) bank19896 +19897 POINT(33.72492877352163 -118.79317351267926) bank19897 +19898 POINT(33.12524140580057 -117.93998701379739) bank19898 +19899 POINT(33.316628873857475 -118.98349189750827) bank19899 +19900 POINT(33.08992055991244 -119.16276853998379) bank19900 +19901 POINT(34.308577384054544 -118.1951000769894) bank19901 +19902 POINT(33.61916490538259 -118.26495320354113) bank19902 +19903 POINT(34.87856072723413 -118.7946577984208) bank19903 +19904 POINT(34.18394125593217 -118.81791795794457) bank19904 +19905 POINT(33.22710612060607 -118.05538307747798) bank19905 +19906 POINT(33.244373462812064 -119.01621382563563) bank19906 +19907 POINT(34.0358531790411 -118.79342941737066) bank19907 +19908 POINT(34.194851860497295 -117.63345168306091) bank19908 +19909 POINT(33.206730440164286 -118.11628576949185) bank19909 +19910 POINT(34.4318590483532 -118.81183993182735) bank19910 +19911 POINT(33.20438445938928 -117.70835219101284) bank19911 +19912 POINT(34.56670572969856 -117.862884725489) bank19912 +19913 POINT(34.83261974897006 -117.9363425738436) bank19913 +19914 POINT(33.93229628919419 -117.33851856823544) bank19914 +19915 POINT(33.60084800508392 -117.93714731424159) bank19915 +19916 POINT(33.19851104255899 -118.87932216909701) bank19916 +19917 POINT(34.09688522828066 -118.18685402753506) bank19917 +19918 POINT(34.561070883391395 -118.24347106633533) bank19918 +19919 POINT(34.761082792003585 -118.20389999160149) bank19919 +19920 POINT(33.36663803404851 -118.98839035661877) bank19920 +19921 POINT(33.201571290942645 -117.58067441716305) bank19921 +19922 POINT(33.08775106142265 -118.27179857963401) bank19922 +19923 POINT(34.434637393356624 -117.84300893132841) bank19923 +19924 POINT(34.19806373068213 -118.66312914467605) bank19924 +19925 POINT(34.27624403323759 -117.28983385240728) bank19925 +19926 POINT(35.02419652223018 -118.06536112029325) bank19926 +19927 POINT(33.63540730693238 -118.51544999237198) bank19927 +19928 POINT(33.40655096018617 -118.37209452040993) bank19928 +19929 POINT(33.47073227252192 -117.71572869671951) bank19929 +19930 POINT(33.30376746671614 -117.49719380749839) bank19930 +19931 POINT(35.04641304940971 -117.86802563774673) bank19931 +19932 POINT(34.780595214705684 -117.35063867675699) bank19932 +19933 POINT(33.5803198219369 -117.91974634192736) bank19933 +19934 POINT(34.31109713038676 -117.73214885128547) bank19934 +19935 POINT(33.894048987346146 -117.74317141352618) bank19935 +19936 POINT(33.82146842720657 -117.75584933548495) bank19936 +19937 POINT(33.707558477879 -118.71158381010082) bank19937 +19938 POINT(35.031675443948224 -117.48708859896392) bank19938 +19939 POINT(34.06646829504873 -118.45599840750887) bank19939 +19940 POINT(34.24702279875989 -117.35008592882559) bank19940 +19941 POINT(34.25713426838328 -118.71316085262639) bank19941 +19942 POINT(33.82518282671258 -117.45272444975973) bank19942 +19943 POINT(34.58020058667494 -119.18526694625665) bank19943 +19944 POINT(34.14641008637073 -117.64673413415699) bank19944 +19945 POINT(34.82421999063706 -117.49712701940122) bank19945 +19946 POINT(33.71942582917383 -117.84085461550852) bank19946 +19947 POINT(34.67340965157692 -118.96624618957613) bank19947 +19948 POINT(33.40069990429291 -117.77188203301041) bank19948 +19949 POINT(33.91101559092882 -118.76028594437773) bank19949 +19950 POINT(34.784424641688176 -117.46018496382892) bank19950 +19951 POINT(34.26881142719343 -118.05864907285347) bank19951 +19952 POINT(34.88599734182206 -117.82300050687235) bank19952 +19953 POINT(33.710213986066776 -118.81857628496032) bank19953 +19954 POINT(33.07408286929295 -118.8522628921583) bank19954 +19955 POINT(34.93102348448573 -118.8762641922361) bank19955 +19956 POINT(33.739786970242996 -117.82619092589417) bank19956 +19957 POINT(33.50975958995216 -118.71494791201677) bank19957 +19958 POINT(33.315072571083604 -117.70268088781509) bank19958 +19959 POINT(33.67018228889031 -118.36470071802195) bank19959 +19960 POINT(34.491903839303845 -119.07096864356812) bank19960 +19961 POINT(33.83634457623486 -117.63256496065894) bank19961 +19962 POINT(34.92069203232426 -117.25484684613849) bank19962 +19963 POINT(33.322595529267446 -119.02936319141749) bank19963 +19964 POINT(33.087573313800654 -117.61825213966685) bank19964 +19965 POINT(33.10935020665406 -117.67923405118378) bank19965 +19966 POINT(34.71808416238712 -117.9500023633226) bank19966 +19967 POINT(33.696529455767745 -119.02556661004832) bank19967 +19968 POINT(33.18527127659178 -118.11652633841564) bank19968 +19969 POINT(34.28765841077492 -119.22944713151523) bank19969 +19970 POINT(33.2579622678813 -118.43774242523557) bank19970 +19971 POINT(34.47633552945687 -117.88905161203961) bank19971 +19972 POINT(33.67085507895606 -117.31904647894304) bank19972 +19973 POINT(34.91145291473879 -118.88418732676483) bank19973 +19974 POINT(34.5603834041706 -118.38581746874837) bank19974 +19975 POINT(34.98325430062964 -119.06053417700349) bank19975 +19976 POINT(34.140887335922635 -118.20856106567565) bank19976 +19977 POINT(33.15407842134319 -118.00218452339253) bank19977 +19978 POINT(34.978619234886466 -118.75762675337248) bank19978 +19979 POINT(34.03907898803016 -118.4482639747235) bank19979 +19980 POINT(34.68962598332159 -118.84102387715532) bank19980 +19981 POINT(34.80660244357871 -117.71824873867423) bank19981 +19982 POINT(33.1697945051384 -118.04292203418564) bank19982 +19983 POINT(34.152034309444296 -119.10286988230469) bank19983 +19984 POINT(34.56392312893452 -117.82100791204397) bank19984 +19985 POINT(33.85995254209589 -117.55490348786448) bank19985 +19986 POINT(34.165948598940204 -117.28028812010055) bank19986 +19987 POINT(34.19478866894972 -118.06349581127773) bank19987 +19988 POINT(33.83425810356411 -119.02778063581232) bank19988 +19989 POINT(34.50438058589171 -118.10582000328509) bank19989 +19990 POINT(33.80476686805144 -119.08109328982938) bank19990 +19991 POINT(33.586308563770174 -117.38741266172475) bank19991 +19992 POINT(34.60333315438832 -118.44439629564646) bank19992 +19993 POINT(34.412960750937216 -118.29410597036947) bank19993 +19994 POINT(34.538630538595605 -117.55269140312676) bank19994 +19995 POINT(34.34596211379429 -118.51315320269426) bank19995 +19996 POINT(33.4780360990639 -117.73695233382706) bank19996 +19997 POINT(34.87031340644958 -117.927399620196) bank19997 +19998 POINT(33.681188211099915 -118.16508411806326) bank19998 +19999 POINT(33.69802724782468 -118.61221518778822) bank19999 +20000 POINT(34.45866681252919 -117.48223668267163) bank20000 +20001 POINT(33.5944648925765 -118.72782577876207) bank20001 +20002 POINT(34.393055687617284 -117.76530473607757) bank20002 +20003 POINT(33.811844629235736 -118.20808359815136) bank20003 +20004 POINT(33.72633913943629 -118.19579669980803) bank20004 +20005 POINT(35.04078612505886 -119.06467678069674) bank20005 +20006 POINT(33.79062184637672 -118.08887395882974) bank20006 +20007 POINT(34.686492124933615 -117.76607876888353) bank20007 +20008 POINT(33.181930658812284 -117.82649119274615) bank20008 +20009 POINT(33.892803973759754 -118.83423592347332) bank20009 +20010 POINT(33.39557160943399 -119.0987857749413) bank20010 +20011 POINT(34.318052413033165 -118.8052015073335) bank20011 +20012 POINT(34.63681951657793 -118.93759145398751) bank20012 +20013 POINT(33.69968880162958 -118.28247178862074) bank20013 +20014 POINT(34.99092764671789 -118.07113417319498) bank20014 +20015 POINT(33.082022955291485 -118.6732702420838) bank20015 +20016 POINT(33.6149033444905 -118.3887308777358) bank20016 +20017 POINT(34.91613168185107 -117.64645170485424) bank20017 +20018 POINT(34.83911218676458 -119.00704571956828) bank20018 +20019 POINT(33.75305546052856 -117.25222385997283) bank20019 +20020 POINT(34.06164988345096 -118.80668673784785) bank20020 +20021 POINT(34.85580945410051 -118.11879500798607) bank20021 +20022 POINT(34.64128633351604 -117.93568022867296) bank20022 +20023 POINT(33.94288104852676 -117.34038769460328) bank20023 +20024 POINT(34.8206155617736 -119.07332515269626) bank20024 +20025 POINT(33.59415529044071 -117.61255746386674) bank20025 +20026 POINT(34.60810178681483 -118.99922727154653) bank20026 +20027 POINT(34.462060851862994 -119.0102854220531) bank20027 +20028 POINT(33.17729722465757 -117.55347906531337) bank20028 +20029 POINT(33.46563379499906 -118.22262022090568) bank20029 +20030 POINT(34.87503475772296 -117.27355246111883) bank20030 +20031 POINT(34.82346526257607 -118.39163579210165) bank20031 +20032 POINT(34.72786112966681 -117.41308580194962) bank20032 +20033 POINT(33.76326974741391 -117.45200593631608) bank20033 +20034 POINT(33.87965776812406 -118.00937951327144) bank20034 +20035 POINT(35.015326282788465 -117.6655072905134) bank20035 +20036 POINT(33.78008784407249 -118.45957445927873) bank20036 +20037 POINT(34.869503879490516 -119.0671789076957) bank20037 +20038 POINT(34.92408320599808 -117.39012390147575) bank20038 +20039 POINT(34.93758360213811 -119.11046272129701) bank20039 +20040 POINT(34.48410192929841 -117.82654474810177) bank20040 +20041 POINT(33.30619526493768 -118.94717377840854) bank20041 +20042 POINT(33.259636672599164 -118.5658820135764) bank20042 +20043 POINT(34.5915643328273 -118.22017746230313) bank20043 +20044 POINT(34.994018645862525 -118.76779315704155) bank20044 +20045 POINT(34.14740492614634 -117.55572886071091) bank20045 +20046 POINT(34.6845919105188 -117.37398356195948) bank20046 +20047 POINT(33.20456398939395 -118.679614042286) bank20047 +20048 POINT(34.33178806106514 -118.90259713562325) bank20048 +20049 POINT(34.67240922036365 -117.5775700168617) bank20049 +20050 POINT(34.45273888656866 -118.85978806572716) bank20050 +20051 POINT(33.3350183590869 -119.17639623329723) bank20051 +20052 POINT(34.56348069992187 -119.0400070603932) bank20052 +20053 POINT(33.62866057021279 -118.99751488199378) bank20053 +20054 POINT(34.20722462238608 -118.94668400841174) bank20054 +20055 POINT(33.46476748313586 -117.44697757652585) bank20055 +20056 POINT(33.30702147579841 -118.64105653548128) bank20056 +20057 POINT(34.65862541301901 -119.04284549395136) bank20057 +20058 POINT(34.667748475078156 -119.03179191674741) bank20058 +20059 POINT(33.43289191307622 -118.66366068016596) bank20059 +20060 POINT(34.49995436426254 -118.90205703097648) bank20060 +20061 POINT(33.26145675242307 -117.76664503863046) bank20061 +20062 POINT(34.72598276709921 -118.08936222628336) bank20062 +20063 POINT(33.820563204379496 -117.4347567972642) bank20063 +20064 POINT(34.85774297002319 -117.76950870615605) bank20064 +20065 POINT(33.9397932671393 -117.97925918292042) bank20065 +20066 POINT(34.90733120988833 -118.13274176275935) bank20066 +20067 POINT(34.21420685365588 -118.32829594694277) bank20067 +20068 POINT(34.73580956514334 -117.3434202988258) bank20068 +20069 POINT(33.34874086630297 -118.93669095604706) bank20069 +20070 POINT(34.73952324387538 -118.01908163165287) bank20070 +20071 POINT(34.23016670677002 -118.329099069572) bank20071 +20072 POINT(33.9445348030017 -117.88151200002136) bank20072 +20073 POINT(34.73152138836056 -118.62656457675213) bank20073 +20074 POINT(34.364042377537345 -118.72530214891738) bank20074 +20075 POINT(34.709719834391876 -119.11687780963508) bank20075 +20076 POINT(34.72122273747365 -119.02395689273493) bank20076 +20077 POINT(33.74104727471828 -118.1279862567792) bank20077 +20078 POINT(34.00006664571987 -118.06306589491777) bank20078 +20079 POINT(34.7081893242825 -119.22070234702291) bank20079 +20080 POINT(33.156979337084444 -118.02477587120573) bank20080 +20081 POINT(35.0299202270601 -118.53457922140915) bank20081 +20082 POINT(34.48502151934449 -118.10161802251248) bank20082 +20083 POINT(34.16588930621924 -119.08473121430391) bank20083 +20084 POINT(34.70245389690766 -117.34334883909379) bank20084 +20085 POINT(33.68349084405212 -118.1974419392109) bank20085 +20086 POINT(34.996098776668994 -118.07926686081548) bank20086 +20087 POINT(34.66378559518029 -117.52052524287339) bank20087 +20088 POINT(33.86132094715634 -118.1550368249458) bank20088 +20089 POINT(34.87517350457365 -118.27697483080664) bank20089 +20090 POINT(33.93877320941926 -118.91173890427075) bank20090 +20091 POINT(34.87042226772115 -118.99427382925154) bank20091 +20092 POINT(34.14479007680539 -117.92495307574279) bank20092 +20093 POINT(34.16381793007348 -118.23870017522843) bank20093 +20094 POINT(33.46054065899631 -117.49994130114138) bank20094 +20095 POINT(33.995639251981885 -117.76559942443237) bank20095 +20096 POINT(34.75724226159734 -118.55325952681272) bank20096 +20097 POINT(34.72576250898802 -118.4971281771727) bank20097 +20098 POINT(33.13553599900224 -119.08677818225804) bank20098 +20099 POINT(34.70986411361184 -118.60472233109688) bank20099 +20100 POINT(34.86105574338644 -117.32346334320913) bank20100 +20101 POINT(33.38952869954447 -117.68002299202607) bank20101 +20102 POINT(35.0266308321621 -117.67173777559506) bank20102 +20103 POINT(34.65764815741614 -117.83774015474427) bank20103 +20104 POINT(33.41619632804743 -119.10939703671325) bank20104 +20105 POINT(33.61975545265042 -117.8896250850423) bank20105 +20106 POINT(33.4439004394952 -118.52178764563433) bank20106 +20107 POINT(34.942142559487735 -118.4712969496147) bank20107 +20108 POINT(33.686281812121834 -118.94172572665359) bank20108 +20109 POINT(34.79128193036112 -118.25016827977244) bank20109 +20110 POINT(34.65684106663305 -117.87713338765506) bank20110 +20111 POINT(33.1944895920314 -117.84519165220995) bank20111 +20112 POINT(34.92141616403664 -117.86324819369189) bank20112 +20113 POINT(33.933477602162974 -117.3339406817555) bank20113 +20114 POINT(33.478282911279656 -118.81189558455219) bank20114 +20115 POINT(34.94810728154108 -117.5573547785211) bank20115 +20116 POINT(33.421087912564495 -118.10037599368991) bank20116 +20117 POINT(33.747904464582064 -117.74661847373152) bank20117 +20118 POINT(34.712396462635546 -118.7477048700263) bank20118 +20119 POINT(33.14351147571831 -118.46259352972002) bank20119 +20120 POINT(34.620853530316815 -119.05952661822654) bank20120 +20121 POINT(34.02295226785824 -118.66767158482885) bank20121 +20122 POINT(34.404343161231125 -117.59815360132758) bank20122 +20123 POINT(33.40398439943329 -117.36587988311939) bank20123 +20124 POINT(34.66400324364474 -118.70031638431983) bank20124 +20125 POINT(34.85232338378302 -117.39752027526234) bank20125 +20126 POINT(34.738706128742756 -118.69246543886008) bank20126 +20127 POINT(34.42146941287977 -117.2907737839724) bank20127 +20128 POINT(33.803965659393626 -119.04274943088713) bank20128 +20129 POINT(33.3322113510393 -118.51483393799015) bank20129 +20130 POINT(34.39476047835733 -118.62968128106903) bank20130 +20131 POINT(34.150784770571796 -118.74933186813568) bank20131 +20132 POINT(34.89564233764439 -118.65649238936096) bank20132 +20133 POINT(34.868573343762975 -117.78339511021446) bank20133 +20134 POINT(34.278670151359165 -117.85823046612381) bank20134 +20135 POINT(34.87539359954831 -117.79531153379477) bank20135 +20136 POINT(34.75118430791997 -117.45107452328432) bank20136 +20137 POINT(33.70959463204848 -117.56285006918625) bank20137 +20138 POINT(33.797985062752076 -117.84824331671844) bank20138 +20139 POINT(34.29755890788407 -117.28279140502733) bank20139 +20140 POINT(34.931561709614854 -118.8866570911336) bank20140 +20141 POINT(34.15917827799731 -118.53761767022317) bank20141 +20142 POINT(34.894482294194844 -117.60401925874581) bank20142 +20143 POINT(35.02488640621204 -118.4307849820853) bank20143 +20144 POINT(33.53779243832136 -118.89755204004595) bank20144 +20145 POINT(34.18037703032423 -119.06290779844743) bank20145 +20146 POINT(34.33132911162306 -118.9182961604611) bank20146 +20147 POINT(33.47317470250094 -117.53141202585921) bank20147 +20148 POINT(34.88530736908103 -118.39291461158656) bank20148 +20149 POINT(34.76559987306138 -119.23492498986184) bank20149 +20150 POINT(33.33669880430123 -117.43054723568923) bank20150 +20151 POINT(33.811739128173954 -117.92247431252763) bank20151 +20152 POINT(33.80393848202127 -118.81775702154683) bank20152 +20153 POINT(34.83747306130029 -119.15345787889905) bank20153 +20154 POINT(33.12401372119612 -117.30848612650782) bank20154 +20155 POINT(34.096363166039424 -117.70323358367646) bank20155 +20156 POINT(34.979684540914825 -118.23745389249048) bank20156 +20157 POINT(33.44760229565037 -117.32410059073003) bank20157 +20158 POINT(34.618623494190985 -118.70835164617019) bank20158 +20159 POINT(33.91972899132127 -117.36723166326355) bank20159 +20160 POINT(34.23610339603618 -118.78367960892966) bank20160 +20161 POINT(33.964978444926494 -119.0980838337087) bank20161 +20162 POINT(34.68689180336685 -117.32152759898045) bank20162 +20163 POINT(33.1016865034145 -118.1455895769838) bank20163 +20164 POINT(34.967274159652995 -117.65062699709638) bank20164 +20165 POINT(33.05664166863653 -117.2801172126368) bank20165 +20166 POINT(33.2693640888454 -117.29950277245305) bank20166 +20167 POINT(33.560940108572424 -118.07822614519768) bank20167 +20168 POINT(33.871637169186904 -117.71275052936683) bank20168 +20169 POINT(34.91775241784812 -118.11722704716196) bank20169 +20170 POINT(34.81620000352973 -117.6776316362161) bank20170 +20171 POINT(33.70031966950018 -118.49583611725672) bank20171 +20172 POINT(34.37530967095697 -117.29161773923367) bank20172 +20173 POINT(33.6415616493838 -118.35899566344075) bank20173 +20174 POINT(34.958060709269795 -117.34872572447237) bank20174 +20175 POINT(33.79598680758761 -118.40577475821797) bank20175 +20176 POINT(33.7939280780405 -118.52953192732942) bank20176 +20177 POINT(34.15527141317282 -118.95642598011275) bank20177 +20178 POINT(33.842182250535146 -118.49092218768948) bank20178 +20179 POINT(34.04837861843117 -118.59935303362109) bank20179 +20180 POINT(34.13933771517748 -117.65505244904662) bank20180 +20181 POINT(34.24195163704738 -118.83552433183041) bank20181 +20182 POINT(34.72257922045068 -118.34831972473654) bank20182 +20183 POINT(33.391541944229154 -117.64467815326948) bank20183 +20184 POINT(33.83132907158135 -118.33765306661513) bank20184 +20185 POINT(33.52926700773803 -117.37271381098438) bank20185 +20186 POINT(33.89484500731127 -117.53495577110111) bank20186 +20187 POINT(34.95028297372395 -117.5329172148665) bank20187 +20188 POINT(34.45084512739664 -118.14004099008807) bank20188 +20189 POINT(34.54381387427707 -118.94060918450083) bank20189 +20190 POINT(34.18197536233277 -117.5224789212803) bank20190 +20191 POINT(34.9885020623754 -118.5651455702896) bank20191 +20192 POINT(35.00821606443137 -118.12365768314916) bank20192 +20193 POINT(33.96853204471501 -118.02871908572914) bank20193 +20194 POINT(33.962682345186856 -118.34995636172164) bank20194 +20195 POINT(33.195124572966236 -119.06772531207542) bank20195 +20196 POINT(33.65502717601506 -117.4451883967021) bank20196 +20197 POINT(33.13071940423541 -117.66311464513414) bank20197 +20198 POINT(34.96603248649373 -117.53963596305634) bank20198 +20199 POINT(34.94306565003548 -118.16296115575311) bank20199 +20200 POINT(34.34355046197666 -118.04064302351382) bank20200 +20201 POINT(33.33650876405578 -117.45539862768776) bank20201 +20202 POINT(33.80993076858698 -118.4165439370176) bank20202 +20203 POINT(34.02527690868417 -117.81217514268914) bank20203 +20204 POINT(34.98933247743225 -118.46036170775196) bank20204 +20205 POINT(34.856984023299695 -117.62908476633595) bank20205 +20206 POINT(34.7251219927923 -119.00782123091194) bank20206 +20207 POINT(33.72813636265901 -118.05003777084923) bank20207 +20208 POINT(33.239493761250756 -118.98025245038687) bank20208 +20209 POINT(34.449632760484285 -117.57648311586263) bank20209 +20210 POINT(34.844623721658635 -117.8547190711629) bank20210 +20211 POINT(33.10108857812508 -117.93092693854524) bank20211 +20212 POINT(34.547531188173465 -117.302889159205) bank20212 +20213 POINT(33.254949327378746 -117.44146144716005) bank20213 +20214 POINT(33.698451004994354 -118.39253809083193) bank20214 +20215 POINT(34.750376552354616 -117.86564629553332) bank20215 +20216 POINT(35.016138687435436 -117.57925287333062) bank20216 +20217 POINT(34.67161545193171 -119.04084246721993) bank20217 +20218 POINT(34.547664590605834 -118.0932302294957) bank20218 +20219 POINT(33.82726727016203 -118.5027737304011) bank20219 +20220 POINT(33.11437084502723 -118.70544666602932) bank20220 +20221 POINT(33.68768541115924 -117.3120739153921) bank20221 +20222 POINT(33.41092829663876 -118.58607875486258) bank20222 +20223 POINT(33.532991072725714 -118.38101925652323) bank20223 +20224 POINT(33.098128726272826 -119.00545052126856) bank20224 +20225 POINT(33.76442540472025 -118.44759665696492) bank20225 +20226 POINT(34.70880385554826 -119.15720834608291) bank20226 +20227 POINT(34.111029154419214 -118.32928212006968) bank20227 +20228 POINT(33.70623507581139 -118.44562296480106) bank20228 +20229 POINT(34.05707023658065 -118.12367646334545) bank20229 +20230 POINT(33.53483520425996 -118.04802719074691) bank20230 +20231 POINT(34.011305722464705 -117.80619504924313) bank20231 +20232 POINT(34.291711819323986 -118.50755048733038) bank20232 +20233 POINT(33.55057571955 -117.75149612492896) bank20233 +20234 POINT(34.97024720019115 -118.6049289880255) bank20234 +20235 POINT(33.710088059911115 -118.96529356259728) bank20235 +20236 POINT(34.23098813057848 -118.41642789773815) bank20236 +20237 POINT(33.96255627928906 -118.87405844736672) bank20237 +20238 POINT(33.30742410018195 -118.165748756655) bank20238 +20239 POINT(33.664679208866936 -118.42445254615106) bank20239 +20240 POINT(33.67175823064229 -117.3805061346) bank20240 +20241 POINT(34.77151591974069 -118.46633175165144) bank20241 +20242 POINT(34.665112112116766 -118.85586937388913) bank20242 +20243 POINT(33.09631078906341 -118.6657204187674) bank20243 +20244 POINT(34.19842495892982 -119.04196338819789) bank20244 +20245 POINT(35.04606917148026 -117.94707322678592) bank20245 +20246 POINT(34.59000507352238 -117.96472247822503) bank20246 +20247 POINT(33.40609651731336 -118.67602634564285) bank20247 +20248 POINT(33.80517052712751 -118.41964434201515) bank20248 +20249 POINT(34.80855777638605 -117.9478742307115) bank20249 +20250 POINT(34.19610962011896 -118.91495837663001) bank20250 +20251 POINT(34.81879493685213 -118.36635703930108) bank20251 +20252 POINT(33.28807938339311 -118.01804075930818) bank20252 +20253 POINT(34.021920993769164 -117.77396060294198) bank20253 +20254 POINT(34.966866191775416 -118.65227854244242) bank20254 +20255 POINT(33.315500838917856 -118.88779617011745) bank20255 +20256 POINT(34.9095917595086 -118.07046563170343) bank20256 +20257 POINT(34.609620091590514 -118.74911586885031) bank20257 +20258 POINT(33.54313935838047 -118.50151854529855) bank20258 +20259 POINT(33.8323603215462 -117.6227222170578) bank20259 +20260 POINT(33.202100779273486 -117.46245328714014) bank20260 +20261 POINT(34.31982885755068 -117.99934084660683) bank20261 +20262 POINT(34.26567449594019 -117.30870819552689) bank20262 +20263 POINT(34.31086790684231 -119.22842987932668) bank20263 +20264 POINT(34.0882253003045 -118.772695378268) bank20264 +20265 POINT(33.178897218107416 -118.76592053190625) bank20265 +20266 POINT(34.3362514245441 -117.86937048595479) bank20266 +20267 POINT(34.92928670092368 -117.59772120628668) bank20267 +20268 POINT(34.377331713128996 -118.40937211090937) bank20268 +20269 POINT(34.95884233517358 -117.92750316359408) bank20269 +20270 POINT(33.073976433932316 -119.07788172359464) bank20270 +20271 POINT(33.786559856025335 -119.02806351969062) bank20271 +20272 POINT(33.82569855457027 -118.42757311238572) bank20272 +20273 POINT(34.70133165331723 -118.37510228310126) bank20273 +20274 POINT(34.966749534557614 -118.72773379130436) bank20274 +20275 POINT(34.33462399567975 -117.71864211277553) bank20275 +20276 POINT(34.044094010461535 -118.39962213295698) bank20276 +20277 POINT(34.209980260785336 -117.34684851232859) bank20277 +20278 POINT(33.6893315137914 -117.76482773698613) bank20278 +20279 POINT(33.56313894542644 -117.67911093127864) bank20279 +20280 POINT(33.37224153190469 -118.857987392694) bank20280 +20281 POINT(34.72437824120047 -118.55027869848917) bank20281 +20282 POINT(34.229173759271866 -118.7458799296624) bank20282 +20283 POINT(34.49440897851542 -118.20332965791589) bank20283 +20284 POINT(34.30155675876911 -118.85872259249649) bank20284 +20285 POINT(33.645382695707106 -118.32150959227482) bank20285 +20286 POINT(34.91842070673265 -118.1883288334203) bank20286 +20287 POINT(34.98363089875794 -118.19436269774029) bank20287 +20288 POINT(33.508253137255394 -117.83919356760781) bank20288 +20289 POINT(33.210206184572975 -118.0283793268526) bank20289 +20290 POINT(34.24438660781939 -117.39872287803142) bank20290 +20291 POINT(33.652267210609246 -119.07589279831811) bank20291 +20292 POINT(34.30002066056653 -118.12589715506635) bank20292 +20293 POINT(35.01730857074967 -118.66122665907096) bank20293 +20294 POINT(34.752312575590786 -118.70299093866105) bank20294 +20295 POINT(33.561611905795004 -118.0544937430789) bank20295 +20296 POINT(33.66926358616176 -118.09290025361284) bank20296 +20297 POINT(33.859756854570314 -117.42416872973851) bank20297 +20298 POINT(34.0741627986991 -118.72316077799684) bank20298 +20299 POINT(33.30836367650221 -118.66938192781872) bank20299 +20300 POINT(33.64691366373181 -118.55530720972192) bank20300 +20301 POINT(34.81146184297966 -117.99297397555065) bank20301 +20302 POINT(34.31673839286204 -117.32313626459683) bank20302 +20303 POINT(34.604590062916785 -117.28624591903134) bank20303 +20304 POINT(33.96986174780149 -117.6687370673761) bank20304 +20305 POINT(34.852453270443924 -117.82326182243087) bank20305 +20306 POINT(34.276563577040925 -117.9983145182103) bank20306 +20307 POINT(34.43195527459211 -118.53269081591935) bank20307 +20308 POINT(34.508880813969085 -118.4658384134774) bank20308 +20309 POINT(34.347918997323674 -118.74949652729885) bank20309 +20310 POINT(33.62687849705025 -118.98013670535181) bank20310 +20311 POINT(33.90166213565268 -118.25176241779954) bank20311 +20312 POINT(33.65124075066286 -117.63477915752662) bank20312 +20313 POINT(34.26882801582789 -118.49311655259478) bank20313 +20314 POINT(34.08635051463626 -118.60525158915445) bank20314 +20315 POINT(34.81310571329032 -117.9298664202589) bank20315 +20316 POINT(34.036951148611436 -117.41090977892631) bank20316 +20317 POINT(33.42964328920139 -119.06534825728353) bank20317 +20318 POINT(34.3475595250086 -117.39446497799365) bank20318 +20319 POINT(34.71377024705709 -118.76621532258625) bank20319 +20320 POINT(34.84673204315593 -117.558311020417) bank20320 +20321 POINT(33.38868251027591 -117.46597798601883) bank20321 +20322 POINT(33.59510830642697 -118.11171894403182) bank20322 +20323 POINT(33.12103462004295 -118.14340814649829) bank20323 +20324 POINT(34.00440087608653 -118.36548548300358) bank20324 +20325 POINT(33.99632768278597 -118.9968615992929) bank20325 +20326 POINT(34.10395312270143 -118.45228901228207) bank20326 +20327 POINT(35.023656481530836 -117.39851521405258) bank20327 +20328 POINT(34.47474223291208 -117.642815685116) bank20328 +20329 POINT(33.19133777831123 -118.84686406052951) bank20329 +20330 POINT(33.18144448045473 -118.76874362542684) bank20330 +20331 POINT(33.25632056517867 -117.68238811503616) bank20331 +20332 POINT(34.93665542663132 -117.99059636293235) bank20332 +20333 POINT(34.33939857283845 -118.88311333676444) bank20333 +20334 POINT(34.23152894338532 -118.12869390351746) bank20334 +20335 POINT(34.94585959221035 -118.74030373860155) bank20335 +20336 POINT(33.18233667693719 -118.68805912838303) bank20336 +20337 POINT(34.95114619191947 -119.0464794527081) bank20337 +20338 POINT(34.56510836729711 -118.59783711696079) bank20338 +20339 POINT(34.22157702009887 -118.3089021374651) bank20339 +20340 POINT(34.83974739888916 -117.30578812560118) bank20340 +20341 POINT(33.55079737169734 -119.14352360310826) bank20341 +20342 POINT(33.699569868941055 -117.65990175523501) bank20342 +20343 POINT(34.66957052725037 -117.78169161134176) bank20343 +20344 POINT(33.571986542187204 -118.58196783342967) bank20344 +20345 POINT(33.677536348135654 -117.87584017226108) bank20345 +20346 POINT(33.795241071412164 -117.76447886333938) bank20346 +20347 POINT(33.1037873370967 -119.10467786505806) bank20347 +20348 POINT(33.92779253630604 -119.23562254254962) bank20348 +20349 POINT(34.62609499770035 -117.34170738823083) bank20349 +20350 POINT(33.412473086994105 -118.48591464215589) bank20350 +20351 POINT(34.918959702267635 -117.6318852097198) bank20351 +20352 POINT(34.306788454645435 -119.07655257323212) bank20352 +20353 POINT(34.2986373915732 -118.2921713067223) bank20353 +20354 POINT(34.28840012133317 -117.9052543898419) bank20354 +20355 POINT(34.68912759832766 -118.49824285335538) bank20355 +20356 POINT(35.014126033071705 -117.9003802971461) bank20356 +20357 POINT(34.5422145203258 -117.73422705397363) bank20357 +20358 POINT(34.287919393742044 -118.90115956459182) bank20358 +20359 POINT(33.49701679340936 -117.28571584494222) bank20359 +20360 POINT(33.67758100261104 -118.37998695083176) bank20360 +20361 POINT(33.53663409314725 -117.24564992877622) bank20361 +20362 POINT(34.37671029957543 -118.2114705559531) bank20362 +20363 POINT(33.997953455034896 -118.14225924815159) bank20363 +20364 POINT(34.727148439635315 -118.90158490897889) bank20364 +20365 POINT(33.925841926085866 -117.62085642424564) bank20365 +20366 POINT(33.67513951724313 -119.1516966609302) bank20366 +20367 POINT(34.13317582365597 -119.10854505163906) bank20367 +20368 POINT(33.6900843460148 -118.99420867682042) bank20368 +20369 POINT(33.18361474872841 -118.66796641995221) bank20369 +20370 POINT(33.48301886354126 -118.69576373580699) bank20370 +20371 POINT(33.15478893500088 -117.30602647162462) bank20371 +20372 POINT(33.454069100771456 -118.64110060161559) bank20372 +20373 POINT(33.60943689461144 -119.07243192335979) bank20373 +20374 POINT(34.748534323352196 -117.65962221149303) bank20374 +20375 POINT(34.5411054287966 -118.5179657090543) bank20375 +20376 POINT(34.20144866391474 -118.21302301177509) bank20376 +20377 POINT(33.1573120705355 -119.1945773117688) bank20377 +20378 POINT(34.758452982160485 -118.75408579346082) bank20378 +20379 POINT(34.582883003135535 -118.73440718751722) bank20379 +20380 POINT(33.59049190626133 -117.93356743387497) bank20380 +20381 POINT(33.604464199757395 -118.37593264279776) bank20381 +20382 POINT(33.428956576880445 -117.81975988656258) bank20382 +20383 POINT(33.30271507206576 -118.267075185698) bank20383 +20384 POINT(33.77767179670465 -117.55065420265984) bank20384 +20385 POINT(33.83465214334146 -118.7038343030702) bank20385 +20386 POINT(34.473259934634385 -118.1923464329378) bank20386 +20387 POINT(34.81283964780264 -118.73065509280384) bank20387 +20388 POINT(34.15727188952971 -117.8625921223711) bank20388 +20389 POINT(34.564594675832886 -119.01480676460459) bank20389 +20390 POINT(34.25840170419977 -117.51085271247017) bank20390 +20391 POINT(33.89587500445935 -117.47675792147992) bank20391 +20392 POINT(34.204447801818745 -117.93905354077157) bank20392 +20393 POINT(33.94272865462703 -118.53627357912865) bank20393 +20394 POINT(33.73360955776326 -117.56843773197033) bank20394 +20395 POINT(33.63171712282019 -118.14618947664444) bank20395 +20396 POINT(33.13647244811 -119.23157159189128) bank20396 +20397 POINT(33.9190926775844 -118.04868896140972) bank20397 +20398 POINT(33.68640716890455 -118.90369883209297) bank20398 +20399 POINT(34.93222986858295 -119.01747696688408) bank20399 +20400 POINT(33.161994989299885 -118.52232951292869) bank20400 +20401 POINT(33.401394328085026 -118.7677437441975) bank20401 +20402 POINT(34.62090964111373 -118.11074179358206) bank20402 +20403 POINT(33.5686363548832 -118.4922852297057) bank20403 +20404 POINT(33.8208969668854 -117.82902280752837) bank20404 +20405 POINT(34.9101014141046 -119.01869864759173) bank20405 +20406 POINT(34.02377508978277 -119.06576040405243) bank20406 +20407 POINT(33.45387827004315 -118.75082531738629) bank20407 +20408 POINT(34.79737844251143 -117.58951659741766) bank20408 +20409 POINT(33.87036742112432 -117.40316488351735) bank20409 +20410 POINT(33.105165922889086 -118.93737036752094) bank20410 +20411 POINT(34.42568898552245 -118.20784071965944) bank20411 +20412 POINT(34.58797258745749 -119.17194510740133) bank20412 +20413 POINT(34.09478150332474 -118.36037311561165) bank20413 +20414 POINT(34.296262452612424 -118.968293753523) bank20414 +20415 POINT(33.56728150964779 -118.17574606970362) bank20415 +20416 POINT(34.395277009412574 -119.10816496896294) bank20416 +20417 POINT(33.23701903386481 -118.45623546681122) bank20417 +20418 POINT(33.71765686603851 -117.97171911161625) bank20418 +20419 POINT(33.51881486210512 -118.44854265242653) bank20419 +20420 POINT(34.8240748507396 -118.23719595913123) bank20420 +20421 POINT(33.10011636637495 -118.33834855759149) bank20421 +20422 POINT(34.79035036255348 -117.77093818840521) bank20422 +20423 POINT(34.839589785013686 -119.24249837701802) bank20423 +20424 POINT(34.56396914974149 -117.51959669757534) bank20424 +20425 POINT(33.201276617428725 -118.11850726222394) bank20425 +20426 POINT(33.98840152788593 -117.67013764600411) bank20426 +20427 POINT(34.001604083891664 -118.03966813912973) bank20427 +20428 POINT(33.76134082073184 -119.222823642391) bank20428 +20429 POINT(33.6427475026173 -118.03552700531345) bank20429 +20430 POINT(33.93177053070483 -117.31615868101515) bank20430 +20431 POINT(34.80175119731144 -118.46427016159673) bank20431 +20432 POINT(33.87315070849917 -118.97418267745412) bank20432 +20433 POINT(34.9596642654756 -117.82345799607846) bank20433 +20434 POINT(33.3526885997035 -117.98326477388919) bank20434 +20435 POINT(33.527283110084035 -118.57888992510468) bank20435 +20436 POINT(34.21856315693729 -119.11278654551337) bank20436 +20437 POINT(34.80403508264441 -118.96547258682182) bank20437 +20438 POINT(34.64551096133215 -119.18341357936629) bank20438 +20439 POINT(34.05899108279965 -118.94657835326117) bank20439 +20440 POINT(33.75416781960618 -118.17806435670461) bank20440 +20441 POINT(34.79271182322628 -117.91581827027434) bank20441 +20442 POINT(33.44072582181729 -117.71631173093711) bank20442 +20443 POINT(34.629790693719144 -117.29436661150986) bank20443 +20444 POINT(33.23251082975386 -119.19378971383573) bank20444 +20445 POINT(33.19874353929831 -118.68733399567377) bank20445 +20446 POINT(34.63250399234446 -119.13401894281245) bank20446 +20447 POINT(34.40745396857204 -118.5277441114739) bank20447 +20448 POINT(33.250618429412214 -117.25165591805988) bank20448 +20449 POINT(33.82709053228044 -118.55702678662968) bank20449 +20450 POINT(33.80830082181212 -117.82128260911023) bank20450 +20451 POINT(33.73637534852559 -117.60801001508031) bank20451 +20452 POINT(33.584046770790714 -117.71807443689673) bank20452 +20453 POINT(34.119108406226566 -118.04302723531357) bank20453 +20454 POINT(34.278249482706954 -118.51489796143628) bank20454 +20455 POINT(34.4030627182377 -117.96604059438616) bank20455 +20456 POINT(34.93195573076272 -117.91450851411949) bank20456 +20457 POINT(34.63553959948098 -117.89885755965696) bank20457 +20458 POINT(34.67712949554054 -118.8927471386815) bank20458 +20459 POINT(33.2855820873593 -117.59733542981414) bank20459 +20460 POINT(34.21189525588128 -119.21168675033223) bank20460 +20461 POINT(34.8210874510696 -117.31774768952008) bank20461 +20462 POINT(34.84686546986163 -117.68145872183612) bank20462 +20463 POINT(33.38767631445456 -118.61513590043018) bank20463 +20464 POINT(34.12993382135031 -119.00216100838846) bank20464 +20465 POINT(34.97554495067999 -118.9063212547064) bank20465 +20466 POINT(33.686798170073274 -117.37936497693059) bank20466 +20467 POINT(33.63239221375754 -117.94098219923407) bank20467 +20468 POINT(34.34951989882797 -117.74558575257959) bank20468 +20469 POINT(34.656656134804656 -117.9991319289593) bank20469 +20470 POINT(34.711391564624066 -118.33488935712384) bank20470 +20471 POINT(33.424006408032014 -119.02363350784236) bank20471 +20472 POINT(33.9031647228648 -117.89812390529136) bank20472 +20473 POINT(33.31500640714536 -118.27525160776078) bank20473 +20474 POINT(33.69423859493644 -117.62310269448801) bank20474 +20475 POINT(34.39115589624566 -117.78887325932074) bank20475 +20476 POINT(34.30420331588253 -118.61022093770984) bank20476 +20477 POINT(33.5677209130747 -117.9799011002898) bank20477 +20478 POINT(34.675052230385596 -118.91984909856447) bank20478 +20479 POINT(34.39361674570636 -117.30282416715262) bank20479 +20480 POINT(34.5667392516166 -117.6797678378295) bank20480 +20481 POINT(34.91131301703547 -117.31407616259362) bank20481 +20482 POINT(34.930635889985105 -118.02779725879309) bank20482 +20483 POINT(34.8181838062266 -118.68453607078322) bank20483 +20484 POINT(34.58723522932532 -118.25220757469478) bank20484 +20485 POINT(33.16404435309032 -117.93890241142188) bank20485 +20486 POINT(34.81998132856529 -118.06806450238375) bank20486 +20487 POINT(34.17526849462768 -117.76221345510638) bank20487 +20488 POINT(34.153210985328265 -118.39543000991692) bank20488 +20489 POINT(34.6259804203804 -119.17173485288808) bank20489 +20490 POINT(33.23750454759414 -117.60042580371059) bank20490 +20491 POINT(34.77056847042711 -118.32385979952299) bank20491 +20492 POINT(34.324537493890304 -118.18664497102858) bank20492 +20493 POINT(33.24446931903178 -118.06003128637347) bank20493 +20494 POINT(33.21088093055448 -118.36750568264445) bank20494 +20495 POINT(33.11276114324918 -117.55141547797558) bank20495 +20496 POINT(34.46599966724189 -118.06064115583875) bank20496 +20497 POINT(33.70598167958021 -119.06155750336043) bank20497 +20498 POINT(33.732956797757986 -119.16316681848187) bank20498 +20499 POINT(33.433597379657414 -117.7749439745819) bank20499 +20500 POINT(34.51334056627858 -117.57734408938582) bank20500 +20501 POINT(34.06970565874374 -117.5981573235975) bank20501 +20502 POINT(33.53922937152443 -117.51443388403996) bank20502 +20503 POINT(34.16650475482469 -119.14725384928688) bank20503 +20504 POINT(35.04676378357374 -117.33585719192764) bank20504 +20505 POINT(33.92597303731834 -118.10487168779495) bank20505 +20506 POINT(34.43916270792717 -118.37827287928353) bank20506 +20507 POINT(34.30314780443704 -118.80501950843524) bank20507 +20508 POINT(34.94337045476321 -117.93637040134413) bank20508 +20509 POINT(35.00700661046322 -118.3373280611025) bank20509 +20510 POINT(34.10527768363036 -118.43431658374426) bank20510 +20511 POINT(34.54098130455869 -117.46766112488896) bank20511 +20512 POINT(33.37076146271017 -118.10214962370456) bank20512 +20513 POINT(34.33213962092026 -118.8125685840778) bank20513 +20514 POINT(34.77726357808895 -117.57900513036591) bank20514 +20515 POINT(33.82083580477491 -117.90290920167412) bank20515 +20516 POINT(34.652857077550664 -117.95517181224389) bank20516 +20517 POINT(34.68253593449844 -118.01431868046362) bank20517 +20518 POINT(34.55505821569831 -117.73317149270015) bank20518 +20519 POINT(33.80145632869377 -119.18594994850822) bank20519 +20520 POINT(33.57908027874449 -118.415468263007) bank20520 +20521 POINT(33.08953127103912 -117.51946220987065) bank20521 +20522 POINT(34.89355780429071 -118.21606957718832) bank20522 +20523 POINT(33.329328114279434 -118.34043922603679) bank20523 +20524 POINT(33.75895076487859 -118.62062534656832) bank20524 +20525 POINT(33.699724568671435 -117.41529415417085) bank20525 +20526 POINT(33.91230731316511 -117.29416990923508) bank20526 +20527 POINT(34.940251636799594 -117.76803857207788) bank20527 +20528 POINT(34.706719800828374 -117.34716010879485) bank20528 +20529 POINT(33.39552357356805 -117.87086012652742) bank20529 +20530 POINT(33.75476912529883 -117.96362206323816) bank20530 +20531 POINT(33.54492336148049 -118.72867873339364) bank20531 +20532 POINT(34.05684931736812 -117.35480363966131) bank20532 +20533 POINT(33.47783874460854 -118.99742214142546) bank20533 +20534 POINT(34.28272975242605 -118.23228552424271) bank20534 +20535 POINT(33.41874755881524 -119.21811395088113) bank20535 +20536 POINT(33.15414079449231 -117.51525670410503) bank20536 +20537 POINT(34.974594543528724 -117.60585752838757) bank20537 +20538 POINT(34.5782896734987 -117.72260079903302) bank20538 +20539 POINT(33.74381487215028 -118.04016006943894) bank20539 +20540 POINT(33.610291933202554 -118.63775364570073) bank20540 +20541 POINT(34.54204709271982 -117.85888110554313) bank20541 +20542 POINT(33.4625360413407 -117.38497743308777) bank20542 +20543 POINT(34.401488483140604 -118.16737719112912) bank20543 +20544 POINT(34.490780264871375 -118.09625733291368) bank20544 +20545 POINT(34.42038103135919 -117.48629270666325) bank20545 +20546 POINT(34.654997237782524 -117.55984134523352) bank20546 +20547 POINT(33.16199971287618 -117.38384617892955) bank20547 +20548 POINT(33.7069633814271 -118.34514822539651) bank20548 +20549 POINT(34.908243160018124 -117.78432003470596) bank20549 +20550 POINT(33.36821924418113 -117.86026069907602) bank20550 +20551 POINT(33.5439355804186 -118.8116975161224) bank20551 +20552 POINT(34.24248795895095 -119.00009296942501) bank20552 +20553 POINT(33.684972499408204 -118.02122487988034) bank20553 +20554 POINT(34.0942882997751 -118.97827627339231) bank20554 +20555 POINT(33.06636003909933 -118.91654647453782) bank20555 +20556 POINT(33.269832605873525 -117.7761660972363) bank20556 +20557 POINT(34.11846659615686 -119.19689868062456) bank20557 +20558 POINT(34.42260630331117 -119.13287853615667) bank20558 +20559 POINT(34.655017338349936 -118.3381413729366) bank20559 +20560 POINT(33.40816254887306 -117.76664956714227) bank20560 +20561 POINT(34.22548302330249 -118.21846710805768) bank20561 +20562 POINT(33.480435232961995 -117.99834399003326) bank20562 +20563 POINT(34.14799927667543 -118.48576113933501) bank20563 +20564 POINT(34.92457732954225 -117.44862190416177) bank20564 +20565 POINT(34.61634994631665 -118.73196129225188) bank20565 +20566 POINT(34.45176260955815 -119.20292187362762) bank20566 +20567 POINT(34.969440409415256 -117.57925734548415) bank20567 +20568 POINT(34.80393171556891 -117.57132211603403) bank20568 +20569 POINT(33.90127589236032 -117.78092963106705) bank20569 +20570 POINT(33.80148208731935 -117.60748297659286) bank20570 +20571 POINT(34.745697990186365 -117.5077205994882) bank20571 +20572 POINT(33.6613687671232 -117.94026572842523) bank20572 +20573 POINT(33.91360942170056 -118.57026087378573) bank20573 +20574 POINT(34.523485224822615 -118.32778134604342) bank20574 +20575 POINT(33.08288541999192 -118.80122027445772) bank20575 +20576 POINT(34.91527439786595 -117.81195484351677) bank20576 +20577 POINT(34.48197358873352 -118.41227360758121) bank20577 +20578 POINT(33.15266999887675 -119.10468352564493) bank20578 +20579 POINT(34.9405531516163 -117.46336424722656) bank20579 +20580 POINT(33.57777137676222 -118.04259715454575) bank20580 +20581 POINT(33.87027915867553 -117.25156897424489) bank20581 +20582 POINT(34.6451266064364 -117.56193760348627) bank20582 +20583 POINT(34.714901029958156 -117.94719169359792) bank20583 +20584 POINT(34.60214125477904 -117.38456713081241) bank20584 +20585 POINT(34.863030953724426 -118.43042639183271) bank20585 +20586 POINT(34.14369547228517 -119.01902279669542) bank20586 +20587 POINT(34.36629370370031 -118.85428519172008) bank20587 +20588 POINT(34.38964863483397 -119.11717037763572) bank20588 +20589 POINT(34.94165968212685 -117.76663338216815) bank20589 +20590 POINT(34.640062400096454 -117.60327593049415) bank20590 +20591 POINT(34.03939177052857 -119.06934655526918) bank20591 +20592 POINT(34.151086326224615 -119.1628181695539) bank20592 +20593 POINT(34.58025504741828 -117.43344607129994) bank20593 +20594 POINT(34.405289795493196 -117.55723931996889) bank20594 +20595 POINT(33.76338132784624 -118.76116053927761) bank20595 +20596 POINT(34.24739363740878 -117.65331898978299) bank20596 +20597 POINT(33.2874880629449 -118.80055007986404) bank20597 +20598 POINT(34.63755987589552 -118.25397694542352) bank20598 +20599 POINT(33.39457069186205 -117.64595771985026) bank20599 +20600 POINT(34.82284038014116 -117.61698338095985) bank20600 +20601 POINT(34.54582096924765 -117.64078471596395) bank20601 +20602 POINT(34.25545868220913 -118.78891906783993) bank20602 +20603 POINT(33.16718477417234 -117.37532620375421) bank20603 +20604 POINT(34.47491705799717 -117.97735660573785) bank20604 +20605 POINT(34.11681719654242 -119.20568850514397) bank20605 +20606 POINT(34.211637191579364 -119.01834145255219) bank20606 +20607 POINT(33.73598343659591 -117.79980274901605) bank20607 +20608 POINT(33.86554454974263 -117.61015423251125) bank20608 +20609 POINT(33.379736870062196 -118.28857137719875) bank20609 +20610 POINT(33.645830298695 -117.75698233884202) bank20610 +20611 POINT(34.29563307230379 -118.52605127629606) bank20611 +20612 POINT(33.21578847979826 -118.58629203354026) bank20612 +20613 POINT(34.574800739013675 -117.95340298114229) bank20613 +20614 POINT(34.02491121893921 -117.85549867905472) bank20614 +20615 POINT(33.69752302471917 -119.03442197559704) bank20615 +20616 POINT(33.14299679842541 -117.95356408167137) bank20616 +20617 POINT(33.428876561451936 -117.86001143238153) bank20617 +20618 POINT(34.09580987183194 -118.20404820261427) bank20618 +20619 POINT(35.00710017823186 -118.64977719502754) bank20619 +20620 POINT(33.34154428450145 -118.41211382203024) bank20620 +20621 POINT(33.36911421777933 -117.2964153197165) bank20621 +20622 POINT(33.48410737773758 -118.11004527551216) bank20622 +20623 POINT(35.04676755349223 -118.85959045671437) bank20623 +20624 POINT(33.31486353234094 -118.98275576156604) bank20624 +20625 POINT(33.52772697311782 -118.44611203025427) bank20625 +20626 POINT(34.97008470478683 -119.08339345469822) bank20626 +20627 POINT(33.83990848745491 -119.13070503227607) bank20627 +20628 POINT(33.70479332557868 -117.71478287216172) bank20628 +20629 POINT(34.25598488421428 -117.43598969170162) bank20629 +20630 POINT(34.1315229555235 -119.0754355767573) bank20630 +20631 POINT(34.24180029262968 -118.50978997400489) bank20631 +20632 POINT(34.760413917865016 -118.85266047662537) bank20632 +20633 POINT(34.9447003639645 -118.03758634532828) bank20633 +20634 POINT(33.232447717457205 -119.04779414794082) bank20634 +20635 POINT(34.0391063587191 -118.7895511801818) bank20635 +20636 POINT(34.24273636974167 -117.38932937859181) bank20636 +20637 POINT(34.9757435081599 -118.0931541195149) bank20637 +20638 POINT(34.121312539403945 -118.00540048543908) bank20638 +20639 POINT(33.823816151184 -118.29997090623581) bank20639 +20640 POINT(33.306623739212704 -119.16598660669442) bank20640 +20641 POINT(34.01403736830083 -119.06670649624976) bank20641 +20642 POINT(34.03079942428725 -119.2013584244916) bank20642 +20643 POINT(34.159216105984164 -118.15027734135474) bank20643 +20644 POINT(34.501241952676885 -119.11117059526083) bank20644 +20645 POINT(33.07838635852708 -117.55384248584753) bank20645 +20646 POINT(34.32505612251775 -119.07553751728229) bank20646 +20647 POINT(34.34186972543535 -117.85909542000294) bank20647 +20648 POINT(34.309655025182344 -119.18562028312753) bank20648 +20649 POINT(33.710437189958014 -118.41712191625543) bank20649 +20650 POINT(34.62526874928323 -117.43341213230684) bank20650 +20651 POINT(33.86698576804034 -119.17091317009168) bank20651 +20652 POINT(33.40631281779682 -118.09232695882491) bank20652 +20653 POINT(33.93866817488664 -118.47976560678362) bank20653 +20654 POINT(34.74593192331388 -117.87376827886138) bank20654 +20655 POINT(33.83229844675249 -119.23350042383298) bank20655 +20656 POINT(34.15116843800879 -117.31380268466758) bank20656 +20657 POINT(33.3044682416916 -117.73910377288647) bank20657 +20658 POINT(35.00115132790198 -118.46736521311746) bank20658 +20659 POINT(33.85165496840353 -117.75267314821576) bank20659 +20660 POINT(34.641879053207006 -118.13980859620884) bank20660 +20661 POINT(33.1321433659952 -118.249193141959) bank20661 +20662 POINT(34.63039457533536 -119.15260245012752) bank20662 +20663 POINT(34.98565176197516 -117.74457902069473) bank20663 +20664 POINT(34.625557682681034 -118.76655281931811) bank20664 +20665 POINT(34.711198926098675 -117.6238992080669) bank20665 +20666 POINT(34.967291102318185 -117.9332938898237) bank20666 +20667 POINT(33.11196517146172 -118.3908672245116) bank20667 +20668 POINT(33.44419140590927 -118.63884646983198) bank20668 +20669 POINT(33.65031142557641 -118.30165456180197) bank20669 +20670 POINT(34.48434994593049 -117.3908775278113) bank20670 +20671 POINT(33.579782273125716 -118.948143294988) bank20671 +20672 POINT(33.53522454010182 -117.89246702953598) bank20672 +20673 POINT(34.05997207766005 -119.21089042289061) bank20673 +20674 POINT(33.123637415850915 -118.73332134897856) bank20674 +20675 POINT(33.49951875994803 -118.21149020847695) bank20675 +20676 POINT(33.63795173256242 -118.8409361439996) bank20676 +20677 POINT(33.6642603064917 -117.46369868506497) bank20677 +20678 POINT(33.76682541712849 -117.39703428731667) bank20678 +20679 POINT(33.29520541500709 -118.6106469035863) bank20679 +20680 POINT(34.17069534545716 -118.05336537767532) bank20680 +20681 POINT(34.43601215553005 -117.94536783043189) bank20681 +20682 POINT(34.4939376019658 -119.13109118168921) bank20682 +20683 POINT(33.37464374633248 -118.33352965530554) bank20683 +20684 POINT(33.85413634556208 -118.41911360621073) bank20684 +20685 POINT(33.592625109079236 -119.22594166961215) bank20685 +20686 POINT(33.112346271708375 -118.88960041430337) bank20686 +20687 POINT(35.0279341903348 -117.8331268277753) bank20687 +20688 POINT(33.91490482460367 -118.30240350656665) bank20688 +20689 POINT(34.71205062297526 -117.56000102455046) bank20689 +20690 POINT(35.02612456945902 -117.33635858523353) bank20690 +20691 POINT(34.21023289593165 -119.19672334238436) bank20691 +20692 POINT(33.58514408803466 -117.26778980365034) bank20692 +20693 POINT(35.034683541358845 -117.84549156886989) bank20693 +20694 POINT(33.84896730427596 -117.47850613992824) bank20694 +20695 POINT(33.58115376315876 -117.48529746474672) bank20695 +20696 POINT(34.3435608035961 -118.19362370835377) bank20696 +20697 POINT(34.58628802561301 -117.63300587262552) bank20697 +20698 POINT(33.59261539206958 -118.51973089596694) bank20698 +20699 POINT(33.305405745839956 -118.58031428488168) bank20699 +20700 POINT(33.0877116319315 -118.61823947685346) bank20700 +20701 POINT(33.735726670070974 -118.13461756099038) bank20701 +20702 POINT(34.230546175196956 -117.50280470456862) bank20702 +20703 POINT(34.72760137117493 -117.56787323435721) bank20703 +20704 POINT(33.90435575532346 -118.36328628958421) bank20704 +20705 POINT(34.78698522498443 -118.4793902033471) bank20705 +20706 POINT(33.08976463615316 -119.15480008362395) bank20706 +20707 POINT(34.032785188972454 -117.31372722757328) bank20707 +20708 POINT(34.863338943212284 -118.63090087101273) bank20708 +20709 POINT(34.21232246859494 -118.17560870001238) bank20709 +20710 POINT(34.18888408237607 -117.4891763874519) bank20710 +20711 POINT(34.81705219928621 -117.57912949095228) bank20711 +20712 POINT(33.56960274238664 -118.71368417419498) bank20712 +20713 POINT(33.89449125394818 -117.6462115205811) bank20713 +20714 POINT(34.83786520530782 -117.57188557031306) bank20714 +20715 POINT(34.978243819112635 -117.36974382856239) bank20715 +20716 POINT(33.09862006719567 -118.50021054625488) bank20716 +20717 POINT(34.438567454055914 -118.15212502412427) bank20717 +20718 POINT(34.42656368431034 -118.54460805381123) bank20718 +20719 POINT(33.20073347463094 -118.41468701947215) bank20719 +20720 POINT(34.98876664397582 -118.39362171111996) bank20720 +20721 POINT(33.93865730480473 -118.97115351781089) bank20721 +20722 POINT(33.611206797175115 -118.11362399478394) bank20722 +20723 POINT(34.407679394524706 -117.30252275256089) bank20723 +20724 POINT(33.78998228731313 -117.79245162280661) bank20724 +20725 POINT(33.27430925262099 -117.85137775221615) bank20725 +20726 POINT(33.49692712492561 -118.79371209823535) bank20726 +20727 POINT(34.15825214047865 -117.53226456376856) bank20727 +20728 POINT(35.03881910558862 -119.20893629732657) bank20728 +20729 POINT(33.16928630593463 -118.59404573071879) bank20729 +20730 POINT(34.46881067415454 -118.58607172991285) bank20730 +20731 POINT(34.79102505726319 -119.23091228246716) bank20731 +20732 POINT(34.28560842137836 -117.98742985646462) bank20732 +20733 POINT(33.440362645627985 -118.11764926837279) bank20733 +20734 POINT(34.349937154632514 -117.51582965387732) bank20734 +20735 POINT(33.78793857251601 -118.41390169493347) bank20735 +20736 POINT(34.9174354895917 -118.65539338327923) bank20736 +20737 POINT(33.75120601802729 -118.5132772284638) bank20737 +20738 POINT(33.10512604631365 -117.98915372531192) bank20738 +20739 POINT(34.22074311975957 -118.17796568870855) bank20739 +20740 POINT(33.133063466972494 -119.10609989113925) bank20740 +20741 POINT(33.86662413709686 -118.81443813888403) bank20741 +20742 POINT(34.123244280065414 -118.7357078395785) bank20742 +20743 POINT(33.54355323278843 -117.47990792900804) bank20743 +20744 POINT(33.089228609746186 -118.79854715816606) bank20744 +20745 POINT(33.86573020862495 -117.91462759504088) bank20745 +20746 POINT(34.940636451710525 -118.96898253377799) bank20746 +20747 POINT(34.40602462261484 -118.45217775743373) bank20747 +20748 POINT(34.94693598185075 -118.18733822003972) bank20748 +20749 POINT(33.9125118690507 -117.47723208281533) bank20749 +20750 POINT(34.97647425708988 -118.99661720942235) bank20750 +20751 POINT(34.872790080719646 -119.06398105803015) bank20751 +20752 POINT(34.2792398445502 -118.00285705899964) bank20752 +20753 POINT(33.24547830085527 -117.64331756408173) bank20753 +20754 POINT(33.6880143740194 -117.66587694616524) bank20754 +20755 POINT(33.16398846342879 -119.16133679084925) bank20755 +20756 POINT(33.70580140597641 -118.31548576550216) bank20756 +20757 POINT(34.587448905445726 -118.518241974882) bank20757 +20758 POINT(34.29482262557076 -117.2946323470268) bank20758 +20759 POINT(34.53287744968767 -119.21724437737966) bank20759 +20760 POINT(33.51934030105983 -118.17279728206036) bank20760 +20761 POINT(34.46019785452586 -118.70462040886375) bank20761 +20762 POINT(34.2408561892879 -117.51859584660109) bank20762 +20763 POINT(33.36763147234513 -119.23973058516872) bank20763 +20764 POINT(33.4374541561955 -119.22343944345242) bank20764 +20765 POINT(35.01762556537235 -117.96410784009134) bank20765 +20766 POINT(33.21067258836687 -117.36345268741212) bank20766 +20767 POINT(34.41084807144631 -118.44941252260074) bank20767 +20768 POINT(33.50270506958033 -117.34455879609872) bank20768 +20769 POINT(34.436664551719595 -117.53754545276614) bank20769 +20770 POINT(34.376660856657644 -119.23173333222596) bank20770 +20771 POINT(34.00569080570017 -117.95460536629707) bank20771 +20772 POINT(35.02587283899979 -118.75754227332676) bank20772 +20773 POINT(33.9875386562118 -117.9104939114287) bank20773 +20774 POINT(34.372959205872895 -118.3104892086311) bank20774 +20775 POINT(33.23447016715649 -119.03917388635246) bank20775 +20776 POINT(33.34808927990753 -117.76045254039143) bank20776 +20777 POINT(34.07185024282786 -118.20976730687698) bank20777 +20778 POINT(35.022403709590435 -117.3001681193538) bank20778 +20779 POINT(33.085167242779455 -119.10838100569991) bank20779 +20780 POINT(33.073794247685754 -118.55540639246952) bank20780 +20781 POINT(33.33219473660576 -118.54291387108768) bank20781 +20782 POINT(35.013789338545365 -119.06691169966281) bank20782 +20783 POINT(34.125985991546436 -118.37657163924369) bank20783 +20784 POINT(34.58670730587643 -118.59080219348266) bank20784 +20785 POINT(33.9634094832793 -119.24043525093647) bank20785 +20786 POINT(33.85920340954369 -119.24326680274112) bank20786 +20787 POINT(33.81499386680827 -118.88929364912899) bank20787 +20788 POINT(33.306622372378634 -118.38018758695087) bank20788 +20789 POINT(34.952978858777485 -118.81974904076749) bank20789 +20790 POINT(33.489935372803544 -118.44683155592847) bank20790 +20791 POINT(34.37815708791586 -117.4022254959259) bank20791 +20792 POINT(34.12105248695369 -117.85542798403297) bank20792 +20793 POINT(33.072418323120814 -117.34424078272008) bank20793 +20794 POINT(33.32412418702998 -119.17493502571891) bank20794 +20795 POINT(34.280441698489305 -119.03833485980569) bank20795 +20796 POINT(34.03843536210724 -119.0985375274524) bank20796 +20797 POINT(35.01652268662194 -118.47520188059642) bank20797 +20798 POINT(34.641551855968736 -118.77422415146522) bank20798 +20799 POINT(33.1716435554688 -119.20674557330008) bank20799 +20800 POINT(34.92473159046551 -119.23909649248432) bank20800 +20801 POINT(33.84350727930541 -118.823837121709) bank20801 +20802 POINT(34.46082272328127 -117.73794140373744) bank20802 +20803 POINT(33.562823716753286 -117.92929347245934) bank20803 +20804 POINT(33.42198391176449 -117.80591979753058) bank20804 +20805 POINT(34.526546361324655 -118.84769214676025) bank20805 +20806 POINT(34.1465358610288 -117.25730827328739) bank20806 +20807 POINT(34.694552476691044 -117.93036590682459) bank20807 +20808 POINT(33.82941872474929 -118.36788364984575) bank20808 +20809 POINT(33.9048683070392 -118.62989997753772) bank20809 +20810 POINT(34.08742063599649 -118.6029786680553) bank20810 +20811 POINT(33.57190800594901 -118.29153008882162) bank20811 +20812 POINT(34.02884848527413 -118.6985753996454) bank20812 +20813 POINT(33.5276281543786 -119.06254659743755) bank20813 +20814 POINT(34.934350411483855 -118.5749839716674) bank20814 +20815 POINT(33.77255451876078 -118.58040722073488) bank20815 +20816 POINT(33.56769780610474 -118.37112340009031) bank20816 +20817 POINT(34.886739880617334 -119.03147073584755) bank20817 +20818 POINT(33.497343094959426 -118.01554552268018) bank20818 +20819 POINT(33.15276331996835 -118.20508008801608) bank20819 +20820 POINT(33.85128511664866 -118.63628327569289) bank20820 +20821 POINT(34.99992958834414 -119.23226842178326) bank20821 +20822 POINT(33.73548941956639 -117.43171125661055) bank20822 +20823 POINT(33.191783415218545 -118.19574168095491) bank20823 +20824 POINT(33.071289489502504 -118.44518103410361) bank20824 +20825 POINT(34.29039497840922 -117.63791671573267) bank20825 +20826 POINT(34.36450196415804 -117.54413330193717) bank20826 +20827 POINT(33.35763476104649 -117.89475711812996) bank20827 +20828 POINT(34.147440638525495 -118.38651360347565) bank20828 +20829 POINT(34.63392661146648 -119.07188696854371) bank20829 +20830 POINT(34.40502886218159 -117.40456517134791) bank20830 +20831 POINT(33.483118769294094 -117.5679456671955) bank20831 +20832 POINT(34.05592696220684 -118.33645431217754) bank20832 +20833 POINT(34.99060154863296 -118.83647384394011) bank20833 +20834 POINT(33.06873330303875 -118.50742454672603) bank20834 +20835 POINT(33.87384773264701 -119.07042394849016) bank20835 +20836 POINT(34.49428423101742 -118.41257196503412) bank20836 +20837 POINT(33.157083982880685 -117.78083238499431) bank20837 +20838 POINT(33.41621045715583 -117.63266009925287) bank20838 +20839 POINT(34.80612527781636 -118.80199335876793) bank20839 +20840 POINT(33.60782942370032 -118.94035592835449) bank20840 +20841 POINT(34.50261668412831 -118.89166332938156) bank20841 +20842 POINT(33.89163461569473 -117.95464762541401) bank20842 +20843 POINT(34.00329665896733 -117.89599976745967) bank20843 +20844 POINT(33.30580738430511 -119.06222812415393) bank20844 +20845 POINT(34.92392384546063 -117.64599080291472) bank20845 +20846 POINT(33.87015054494888 -118.46276985736266) bank20846 +20847 POINT(34.548139562004884 -118.2903546299135) bank20847 +20848 POINT(33.50141930695439 -117.47191806247568) bank20848 +20849 POINT(33.76987071941031 -118.30828001249051) bank20849 +20850 POINT(33.122585457086366 -118.58541647292341) bank20850 +20851 POINT(34.63190987844883 -117.5685729832882) bank20851 +20852 POINT(33.359349472077405 -118.78677028120602) bank20852 +20853 POINT(33.187984443982806 -117.31539671344711) bank20853 +20854 POINT(33.26790691000793 -118.6970220942753) bank20854 +20855 POINT(33.07537736819566 -117.95788901566931) bank20855 +20856 POINT(34.05230164171911 -118.01359152394319) bank20856 +20857 POINT(34.542613247320524 -117.27649328743267) bank20857 +20858 POINT(33.17390828854588 -118.36132523438901) bank20858 +20859 POINT(34.392096879374016 -117.78719481189367) bank20859 +20860 POINT(34.639365283397005 -117.57134995440755) bank20860 +20861 POINT(33.21831473031547 -118.83259694071677) bank20861 +20862 POINT(34.51386789033236 -118.14263201467243) bank20862 +20863 POINT(34.013758081178146 -118.40323381798942) bank20863 +20864 POINT(33.72797546948434 -118.5651792554768) bank20864 +20865 POINT(34.07868701043198 -118.72908761170285) bank20865 +20866 POINT(34.425694280848674 -118.28392539063232) bank20866 +20867 POINT(33.45425766591758 -117.84979684149654) bank20867 +20868 POINT(34.82429526391389 -117.2707626368698) bank20868 +20869 POINT(33.20727025710751 -119.05129676188055) bank20869 +20870 POINT(33.39301209152597 -118.0103473106505) bank20870 +20871 POINT(34.91856350515313 -118.55792490883024) bank20871 +20872 POINT(34.02502554228202 -117.58206105924168) bank20872 +20873 POINT(34.41991385209403 -118.82038127961752) bank20873 +20874 POINT(34.01392481694715 -118.72736913905952) bank20874 +20875 POINT(33.596064408098954 -118.98162769013209) bank20875 +20876 POINT(33.984400556371114 -117.57867202397368) bank20876 +20877 POINT(34.81630750254123 -118.94854212792059) bank20877 +20878 POINT(34.94649032445004 -117.73801446005909) bank20878 +20879 POINT(33.71827074845297 -119.06136614494116) bank20879 +20880 POINT(34.0524565679616 -118.3539627827728) bank20880 +20881 POINT(34.9217307943161 -118.08775353951503) bank20881 +20882 POINT(34.163756599648345 -118.46676440734112) bank20882 +20883 POINT(34.89049900523854 -117.34641740956292) bank20883 +20884 POINT(34.924334041538735 -117.27012969801436) bank20884 +20885 POINT(33.248210661379204 -118.04355505275826) bank20885 +20886 POINT(33.2286226045533 -117.49096090096612) bank20886 +20887 POINT(33.600352174626 -119.06201037790169) bank20887 +20888 POINT(33.57564372144672 -118.3517160988971) bank20888 +20889 POINT(34.087374635426634 -117.26500652638174) bank20889 +20890 POINT(34.11565563058145 -117.96674388366571) bank20890 +20891 POINT(34.76670473381285 -118.5160866646277) bank20891 +20892 POINT(33.16766807740328 -117.82137216675142) bank20892 +20893 POINT(33.89685717769106 -119.0959381153734) bank20893 +20894 POINT(34.569046014209775 -119.1439532409838) bank20894 +20895 POINT(33.62940885594399 -117.76146692758677) bank20895 +20896 POINT(34.184684766711996 -119.23318803643534) bank20896 +20897 POINT(33.58103416128066 -118.25677472870998) bank20897 +20898 POINT(33.47112094538774 -117.61127866527703) bank20898 +20899 POINT(33.73556123857682 -118.09949792786219) bank20899 +20900 POINT(34.95491964147703 -118.53372982721392) bank20900 +20901 POINT(33.87663415088877 -119.14369630447325) bank20901 +20902 POINT(34.33698798653349 -118.31124785749472) bank20902 +20903 POINT(34.161223773418094 -118.09565572289193) bank20903 +20904 POINT(33.32540142872897 -117.83155625789792) bank20904 +20905 POINT(34.95996822828891 -118.09640877907562) bank20905 +20906 POINT(33.92330064633742 -119.19612547746536) bank20906 +20907 POINT(33.79380484061891 -118.81274219264894) bank20907 +20908 POINT(34.001578852474054 -118.45654444627364) bank20908 +20909 POINT(33.43445300862937 -117.87885120409058) bank20909 +20910 POINT(34.90533709654651 -118.11571482513564) bank20910 +20911 POINT(34.73984218101701 -118.65601520077355) bank20911 +20912 POINT(34.68246075584245 -119.20621632835925) bank20912 +20913 POINT(34.58596133665743 -117.63211789485048) bank20913 +20914 POINT(34.485959775231414 -117.9643415019136) bank20914 +20915 POINT(34.652323746563084 -119.15656472650286) bank20915 +20916 POINT(34.94967026651321 -118.63033963336427) bank20916 +20917 POINT(33.916128076721556 -117.89039133220449) bank20917 +20918 POINT(34.98046184292986 -119.11339459255755) bank20918 +20919 POINT(34.15949130065291 -119.16935481332857) bank20919 +20920 POINT(33.32227710353972 -118.11257662726017) bank20920 +20921 POINT(34.593022914345305 -118.75103463666049) bank20921 +20922 POINT(33.77860224315122 -118.41552908603445) bank20922 +20923 POINT(33.77865018067081 -118.38947599451893) bank20923 +20924 POINT(33.33874812291085 -118.27630909809159) bank20924 +20925 POINT(33.28419386128949 -118.72307931115878) bank20925 +20926 POINT(33.85086044079857 -119.03503919314117) bank20926 +20927 POINT(33.76643925986123 -118.23084107178845) bank20927 +20928 POINT(33.122335504578025 -118.69502707662171) bank20928 +20929 POINT(33.07430776633573 -118.19010587541446) bank20929 +20930 POINT(33.737000765755205 -117.61008738507125) bank20930 +20931 POINT(33.72407354203263 -118.5471898299462) bank20931 +20932 POINT(33.33710481190189 -118.11579897949318) bank20932 +20933 POINT(33.28719876343993 -119.05734112984554) bank20933 +20934 POINT(33.96547329613141 -118.63673591223414) bank20934 +20935 POINT(34.10305317235515 -118.5579942431263) bank20935 +20936 POINT(35.03758583626849 -117.24589860625404) bank20936 +20937 POINT(34.408840368654474 -119.07083823276587) bank20937 +20938 POINT(33.29244582225015 -117.98204067155346) bank20938 +20939 POINT(34.365987545200134 -117.80913536407895) bank20939 +20940 POINT(35.00249616277758 -118.19605523109743) bank20940 +20941 POINT(33.66507153690897 -118.39908928572396) bank20941 +20942 POINT(34.522888975325436 -117.76337925277753) bank20942 +20943 POINT(35.04168601624747 -118.48423990139648) bank20943 +20944 POINT(34.2095996363067 -117.29915870445696) bank20944 +20945 POINT(34.12274817274504 -117.61788589559173) bank20945 +20946 POINT(35.00995062180854 -117.9288960237746) bank20946 +20947 POINT(33.28134418719085 -119.20308602229761) bank20947 +20948 POINT(35.04453647333959 -118.74515110348733) bank20948 +20949 POINT(33.86846790562823 -118.61555866702227) bank20949 +20950 POINT(33.56651299558871 -119.10975801168689) bank20950 +20951 POINT(33.20732379873698 -117.37543239123093) bank20951 +20952 POINT(33.90245911108227 -117.9316804609029) bank20952 +20953 POINT(33.3886000667107 -117.93505240011784) bank20953 +20954 POINT(34.39816485959262 -117.41899547770824) bank20954 +20955 POINT(34.0271382684968 -118.29207498025062) bank20955 +20956 POINT(33.05387753700947 -118.32928338256335) bank20956 +20957 POINT(33.40890796092777 -118.04462055742037) bank20957 +20958 POINT(34.81967272165663 -117.77482476836552) bank20958 +20959 POINT(33.360027061934865 -118.62212369068145) bank20959 +20960 POINT(33.428117171702596 -117.61464900046647) bank20960 +20961 POINT(34.55091278265277 -118.08011945982102) bank20961 +20962 POINT(33.96390713628259 -119.04537899647383) bank20962 +20963 POINT(33.59862715539073 -119.07916086959527) bank20963 +20964 POINT(34.317286220812036 -117.56047013155046) bank20964 +20965 POINT(33.82026668425708 -117.50997992274345) bank20965 +20966 POINT(33.37272873984729 -118.19828935800018) bank20966 +20967 POINT(34.299718059300424 -117.46314906494786) bank20967 +20968 POINT(33.884723747743315 -117.88773312430972) bank20968 +20969 POINT(33.18293709496873 -118.97333139867149) bank20969 +20970 POINT(34.12384330435037 -117.70051016948447) bank20970 +20971 POINT(34.051016714393214 -117.61368935449727) bank20971 +20972 POINT(33.9710883248386 -119.15346126475816) bank20972 +20973 POINT(34.187005574497576 -118.64863425921483) bank20973 +20974 POINT(33.96933702640544 -119.07102677648992) bank20974 +20975 POINT(34.30018333501113 -118.01049217078537) bank20975 +20976 POINT(33.852147682026285 -119.09674792630209) bank20976 +20977 POINT(34.775409464109785 -117.69352004350581) bank20977 +20978 POINT(34.347348371880436 -118.431747553805) bank20978 +20979 POINT(34.5216523178252 -118.78713735242638) bank20979 +20980 POINT(33.41219470343594 -118.62120742574614) bank20980 +20981 POINT(33.7354632457363 -117.81461884756408) bank20981 +20982 POINT(33.60733341464956 -118.26964151180442) bank20982 +20983 POINT(34.05852979797417 -118.6864319714103) bank20983 +20984 POINT(34.795778912225536 -117.31449799937012) bank20984 +20985 POINT(34.12111003891344 -119.07815174170972) bank20985 +20986 POINT(34.386537510239954 -117.46172480427843) bank20986 +20987 POINT(33.6861829291683 -118.91794937751884) bank20987 +20988 POINT(33.350137735495856 -118.06304810070385) bank20988 +20989 POINT(33.56389810234897 -118.07133459843382) bank20989 +20990 POINT(34.44578077407591 -117.26914648290723) bank20990 +20991 POINT(34.51569108302152 -118.39594924739728) bank20991 +20992 POINT(34.393438078895564 -117.41317682205843) bank20992 +20993 POINT(34.716466197237445 -117.38774757779659) bank20993 +20994 POINT(34.86310390987324 -119.19484777129212) bank20994 +20995 POINT(33.68925925000171 -117.91874677860815) bank20995 +20996 POINT(33.20404643793706 -117.25434966896408) bank20996 +20997 POINT(34.544747928810615 -117.70874377261515) bank20997 +20998 POINT(34.34388495034353 -117.46365570043352) bank20998 +20999 POINT(33.394393872031394 -118.62677602427101) bank20999 +21000 POINT(34.09854917118558 -118.30080896360276) bank21000 +21001 POINT(33.274577453018175 -118.51018649272403) bank21001 +21002 POINT(33.29953677582337 -118.42092323971536) bank21002 +21003 POINT(34.15782426825688 -117.90235360423604) bank21003 +21004 POINT(34.813139727188094 -117.25985979726472) bank21004 +21005 POINT(35.03160145960722 -118.6502879802989) bank21005 +21006 POINT(33.132714646338364 -118.90496639380672) bank21006 +21007 POINT(33.9650503364738 -117.85707919229046) bank21007 +21008 POINT(34.22761521493854 -119.22653830754238) bank21008 +21009 POINT(33.45986113704505 -118.80488351222402) bank21009 +21010 POINT(34.612307620911025 -119.22884338140798) bank21010 +21011 POINT(33.54688723706699 -118.6110827505276) bank21011 +21012 POINT(34.92279673946666 -118.41024713326182) bank21012 +21013 POINT(33.57153430395099 -117.61735600099666) bank21013 +21014 POINT(34.08425243470248 -118.13038037790531) bank21014 +21015 POINT(33.97686557956508 -119.00570712539012) bank21015 +21016 POINT(33.1592583205686 -117.35302305254743) bank21016 +21017 POINT(33.66820019265221 -118.90273060089162) bank21017 +21018 POINT(34.5501369276154 -118.92534950281892) bank21018 +21019 POINT(34.75015109743764 -118.43669654173704) bank21019 +21020 POINT(33.25956325757884 -118.77261497532142) bank21020 +21021 POINT(33.73488258792908 -118.30841219643355) bank21021 +21022 POINT(33.82362363595966 -118.35822418651733) bank21022 +21023 POINT(34.453831792513185 -118.27927870584894) bank21023 +21024 POINT(33.381869170109155 -118.73557378506806) bank21024 +21025 POINT(34.09755787537787 -117.65244952773728) bank21025 +21026 POINT(34.60886807809036 -117.39747302895881) bank21026 +21027 POINT(33.537248500172474 -118.98574781827608) bank21027 +21028 POINT(33.58452711327896 -118.49845581443161) bank21028 +21029 POINT(33.671494347898864 -117.2639685421915) bank21029 +21030 POINT(34.83332189981148 -118.08897516764412) bank21030 +21031 POINT(34.141538595979064 -119.12094661118483) bank21031 +21032 POINT(33.064590210895474 -118.85114271640634) bank21032 +21033 POINT(34.490447166863355 -118.96667609548038) bank21033 +21034 POINT(34.23289221497493 -117.48666688151135) bank21034 +21035 POINT(33.101438792421746 -117.85883839423056) bank21035 +21036 POINT(33.670019537759245 -117.59705008338332) bank21036 +21037 POINT(33.50382528096064 -117.53067859059001) bank21037 +21038 POINT(34.27597966378242 -117.24504262988363) bank21038 +21039 POINT(34.61857925277908 -119.16092963990263) bank21039 +21040 POINT(34.63598869627299 -117.24835656349937) bank21040 +21041 POINT(33.281610236271284 -119.22546117691269) bank21041 +21042 POINT(34.26850222806907 -118.39791759487353) bank21042 +21043 POINT(34.61344571908664 -117.48193551991984) bank21043 +21044 POINT(33.323839219354056 -118.80891128046078) bank21044 +21045 POINT(34.00354694039992 -117.92358729339371) bank21045 +21046 POINT(34.155911097008826 -119.24059932129272) bank21046 +21047 POINT(34.55919084664852 -117.25119040917663) bank21047 +21048 POINT(34.945998683734075 -118.0860052993938) bank21048 +21049 POINT(34.544252653370506 -118.43356684039819) bank21049 +21050 POINT(34.782692634150756 -117.75346504158858) bank21050 +21051 POINT(33.998104235166025 -117.89105637317829) bank21051 +21052 POINT(33.703431948379716 -118.80561048837086) bank21052 +21053 POINT(34.20485176096294 -118.95097297948986) bank21053 +21054 POINT(34.15505291176939 -117.92825195513366) bank21054 +21055 POINT(33.831444824821915 -118.27705684107613) bank21055 +21056 POINT(33.671553791404165 -118.35481668661478) bank21056 +21057 POINT(34.894564606623746 -118.1309776002674) bank21057 +21058 POINT(34.07992004530909 -118.24045224317308) bank21058 +21059 POINT(33.79495001910051 -118.71070062206114) bank21059 +21060 POINT(33.133649016309306 -118.71940965057944) bank21060 +21061 POINT(34.71642634697768 -117.68401460178185) bank21061 +21062 POINT(34.077989475153245 -117.50116725033094) bank21062 +21063 POINT(34.215294166221476 -117.80215537519267) bank21063 +21064 POINT(33.67318594958318 -117.53335798040347) bank21064 +21065 POINT(34.054990808107675 -117.35284002439234) bank21065 +21066 POINT(33.555836048551626 -117.7407760420761) bank21066 +21067 POINT(34.596599578301436 -117.27192699315766) bank21067 +21068 POINT(34.922159377258666 -119.01400843956485) bank21068 +21069 POINT(33.601529338437956 -118.29049907710132) bank21069 +21070 POINT(34.207968727988685 -118.74779286576094) bank21070 +21071 POINT(33.92306189667828 -117.61366210548103) bank21071 +21072 POINT(33.98441703196928 -117.26194762262404) bank21072 +21073 POINT(34.929147983510504 -117.90454002920576) bank21073 +21074 POINT(35.01286227657133 -118.62145770606888) bank21074 +21075 POINT(34.4939995813197 -118.55970936665327) bank21075 +21076 POINT(33.189520758073975 -117.8194317656314) bank21076 +21077 POINT(33.481616838895334 -117.33815296282562) bank21077 +21078 POINT(34.900656542349836 -118.00759459008802) bank21078 +21079 POINT(33.261974550398214 -119.1451558533403) bank21079 +21080 POINT(33.57307828833778 -118.1040310867803) bank21080 +21081 POINT(33.87984919250471 -117.91924085215584) bank21081 +21082 POINT(33.757796894167264 -118.98863334763891) bank21082 +21083 POINT(33.110128244273334 -117.83058763981889) bank21083 +21084 POINT(34.77910512725089 -118.97753441286679) bank21084 +21085 POINT(33.97669444156349 -119.24041820908998) bank21085 +21086 POINT(34.02757040535907 -118.20160241154981) bank21086 +21087 POINT(34.50263348696623 -117.347522646779) bank21087 +21088 POINT(33.223156033236215 -117.82049521610985) bank21088 +21089 POINT(33.64761460717178 -118.82705043198561) bank21089 +21090 POINT(34.16858716699598 -117.45064212357735) bank21090 +21091 POINT(33.937115986029994 -118.2734019639352) bank21091 +21092 POINT(34.076002187509374 -118.56636849424723) bank21092 +21093 POINT(33.79717857604245 -117.254270195327) bank21093 +21094 POINT(33.19054209625455 -119.16739695148773) bank21094 +21095 POINT(33.90120244990931 -117.45047638980482) bank21095 +21096 POINT(34.63533209430649 -117.93911712752424) bank21096 +21097 POINT(33.160848268764695 -118.59597630309771) bank21097 +21098 POINT(34.665834717994656 -118.39763878200509) bank21098 +21099 POINT(34.246071037418794 -118.19479696074615) bank21099 +21100 POINT(33.12410973861136 -118.91341072874134) bank21100 +21101 POINT(34.92903477214003 -118.54417987467275) bank21101 +21102 POINT(33.83442761850808 -119.14823560447117) bank21102 +21103 POINT(34.716265702535836 -119.21727370108937) bank21103 +21104 POINT(34.17650665245246 -118.82066914441269) bank21104 +21105 POINT(34.645674356102745 -117.52144384953745) bank21105 +21106 POINT(33.43987083003838 -117.24515040904119) bank21106 +21107 POINT(33.90780470524528 -117.85355342256062) bank21107 +21108 POINT(34.57781269315089 -118.35548565555024) bank21108 +21109 POINT(33.362609173689194 -118.63637451259814) bank21109 +21110 POINT(33.333564667223904 -117.89100222225815) bank21110 +21111 POINT(33.1200459474681 -118.2267333998805) bank21111 +21112 POINT(34.68871654889433 -118.09089708728122) bank21112 +21113 POINT(34.13634424061969 -118.43614214171234) bank21113 +21114 POINT(34.85607801549109 -118.54390128137798) bank21114 +21115 POINT(34.04784619823398 -118.8013789566128) bank21115 +21116 POINT(33.8934854694764 -118.68874751784158) bank21116 +21117 POINT(34.883261514005284 -118.05555974762119) bank21117 +21118 POINT(34.926969777707164 -117.68683867898255) bank21118 +21119 POINT(33.970052432102385 -117.66009523158131) bank21119 +21120 POINT(34.30761120048243 -118.23749003327505) bank21120 +21121 POINT(34.26062626363344 -119.10041367713683) bank21121 +21122 POINT(35.040890704062754 -118.77506844277794) bank21122 +21123 POINT(33.71940166132313 -118.31771166157107) bank21123 +21124 POINT(33.29924638036377 -118.6280456616633) bank21124 +21125 POINT(33.53097768033048 -118.17853007785953) bank21125 +21126 POINT(34.176509791120225 -118.1084437437902) bank21126 +21127 POINT(33.7066813271976 -117.5788400001972) bank21127 +21128 POINT(34.06038404523082 -117.44342805877179) bank21128 +21129 POINT(34.17768484507135 -118.82386602854561) bank21129 +21130 POINT(34.829610039237664 -118.4396838493489) bank21130 +21131 POINT(33.95896046200073 -118.1130876800659) bank21131 +21132 POINT(33.56496490675897 -118.87823441791559) bank21132 +21133 POINT(33.625208061525434 -117.56304326584596) bank21133 +21134 POINT(34.13376217947808 -117.8322264878708) bank21134 +21135 POINT(33.40683098563498 -117.58473984955404) bank21135 +21136 POINT(34.663276790596896 -117.30273828500061) bank21136 +21137 POINT(33.54046436509733 -118.55422634311994) bank21137 +21138 POINT(34.16337501851442 -119.05645323921668) bank21138 +21139 POINT(33.13692406551559 -117.54203774472818) bank21139 +21140 POINT(34.99564243350252 -117.29581703957015) bank21140 +21141 POINT(33.80326536416567 -119.15184059013903) bank21141 +21142 POINT(33.341332238907256 -118.34882417599734) bank21142 +21143 POINT(34.84770413610642 -117.600387582035) bank21143 +21144 POINT(33.972720375703084 -118.62677323830772) bank21144 +21145 POINT(33.90980081183496 -118.40084430188104) bank21145 +21146 POINT(33.29552700686041 -118.22921774433316) bank21146 +21147 POINT(34.50347376686578 -119.0317695752656) bank21147 +21148 POINT(33.20902984983512 -117.98735389620202) bank21148 +21149 POINT(33.602524876284036 -117.36405490427154) bank21149 +21150 POINT(33.722203720551576 -118.94482967432023) bank21150 +21151 POINT(33.44599352296944 -118.80663974959161) bank21151 +21152 POINT(34.736319387016955 -118.25700728294925) bank21152 +21153 POINT(33.60584275169574 -118.78940576755204) bank21153 +21154 POINT(34.999123587762476 -117.65937823462245) bank21154 +21155 POINT(34.014990804813706 -118.87085187069165) bank21155 +21156 POINT(33.126764280261156 -118.7297860139631) bank21156 +21157 POINT(33.57961041059903 -118.74639416888675) bank21157 +21158 POINT(34.79454682623913 -117.80557028942121) bank21158 +21159 POINT(34.12952336041449 -117.73267906848618) bank21159 +21160 POINT(33.364462987017674 -118.16276063100653) bank21160 +21161 POINT(34.84151365740716 -118.57559148313241) bank21161 +21162 POINT(33.37616846539959 -117.49992172839602) bank21162 +21163 POINT(33.92997988745186 -117.486521311139) bank21163 +21164 POINT(34.772574280902276 -117.65423740494853) bank21164 +21165 POINT(35.00423611922263 -118.54906382118732) bank21165 +21166 POINT(35.014923639714496 -117.68965199626041) bank21166 +21167 POINT(33.38095068248334 -118.98101994920067) bank21167 +21168 POINT(33.12553994713002 -118.68967788077136) bank21168 +21169 POINT(33.62660756645744 -118.63905857223244) bank21169 +21170 POINT(34.9917587117043 -119.11509090130318) bank21170 +21171 POINT(34.33954611478578 -117.82068656841894) bank21171 +21172 POINT(34.137758121159386 -118.08799736304945) bank21172 +21173 POINT(34.42937382938695 -118.8164675905834) bank21173 +21174 POINT(34.31086540952749 -118.68342057544476) bank21174 +21175 POINT(34.871971540107964 -117.39549760508751) bank21175 +21176 POINT(34.892216233697724 -118.88269079673167) bank21176 +21177 POINT(33.30999190101574 -118.74464548705836) bank21177 +21178 POINT(34.43504579364239 -117.72962760199616) bank21178 +21179 POINT(34.895019127966044 -117.30177304006948) bank21179 +21180 POINT(33.51191417054501 -118.37589869377595) bank21180 +21181 POINT(33.67177772899114 -117.9576466988514) bank21181 +21182 POINT(34.51623864789397 -119.11576127232736) bank21182 +21183 POINT(33.88728694202736 -118.43464213231516) bank21183 +21184 POINT(33.7704728268988 -118.07710313328414) bank21184 +21185 POINT(33.875645593555 -118.42933755448861) bank21185 +21186 POINT(33.117848157586664 -118.30954932567347) bank21186 +21187 POINT(33.260194461278175 -118.19708575064935) bank21187 +21188 POINT(33.18554012281895 -118.51527516417646) bank21188 +21189 POINT(34.06447739343231 -117.79288417430273) bank21189 +21190 POINT(34.98777592652998 -118.85998410472975) bank21190 +21191 POINT(34.508982997907076 -118.81987085833042) bank21191 +21192 POINT(33.857381845930156 -119.08861463085925) bank21192 +21193 POINT(34.10204009863441 -117.79561574525773) bank21193 +21194 POINT(34.01029889388655 -117.88001946259425) bank21194 +21195 POINT(34.054366080479234 -119.2391338016885) bank21195 +21196 POINT(33.44266303007665 -117.53907229548318) bank21196 +21197 POINT(34.52640505052558 -117.63892061452728) bank21197 +21198 POINT(34.960471444312425 -117.39408889702521) bank21198 +21199 POINT(34.21667572887234 -117.70268311714707) bank21199 +21200 POINT(34.935952677873665 -117.33975696748499) bank21200 +21201 POINT(33.82415666543365 -118.62002781723734) bank21201 +21202 POINT(34.60643302127619 -118.60001778038747) bank21202 +21203 POINT(34.111669749056944 -117.97437757947992) bank21203 +21204 POINT(33.62587663922993 -118.12221847379067) bank21204 +21205 POINT(33.429527449219414 -117.59788890532634) bank21205 +21206 POINT(34.202111006939106 -119.17364688796619) bank21206 +21207 POINT(34.03530585932923 -117.55402314345122) bank21207 +21208 POINT(34.68278194336135 -118.1674819691295) bank21208 +21209 POINT(33.98964011349752 -118.51241094946282) bank21209 +21210 POINT(34.810512214937724 -119.13781209651287) bank21210 +21211 POINT(34.35718358004539 -117.78710237115492) bank21211 +21212 POINT(35.04860032819878 -117.45790163600171) bank21212 +21213 POINT(34.81022021643529 -118.93144606560574) bank21213 +21214 POINT(33.40918145665865 -118.71157551506519) bank21214 +21215 POINT(34.05594774933094 -118.18236048256146) bank21215 +21216 POINT(33.14893952852387 -118.81236444715756) bank21216 +21217 POINT(34.684336078296546 -118.98948410241543) bank21217 +21218 POINT(33.32988830059779 -118.13458118317884) bank21218 +21219 POINT(34.24372094519594 -117.33422695957505) bank21219 +21220 POINT(33.67244419673566 -119.19731733035212) bank21220 +21221 POINT(34.204827136276755 -117.59257980050106) bank21221 +21222 POINT(33.246891274857276 -119.17198155394873) bank21222 +21223 POINT(33.346361876380435 -118.24239568022251) bank21223 +21224 POINT(33.25384656439904 -118.88010686277364) bank21224 +21225 POINT(34.47912339281943 -118.68401876207822) bank21225 +21226 POINT(34.07633527236319 -117.8555082872387) bank21226 +21227 POINT(33.88952780549395 -118.60514179810397) bank21227 +21228 POINT(33.68150094331991 -117.35062922863216) bank21228 +21229 POINT(34.819766573008465 -119.19634669187847) bank21229 +21230 POINT(34.1803933406781 -117.84542041731864) bank21230 +21231 POINT(34.49418512282166 -117.75707176364438) bank21231 +21232 POINT(33.11318021527305 -117.66746141525444) bank21232 +21233 POINT(34.55035269058097 -118.65577625917757) bank21233 +21234 POINT(34.20345619159181 -117.75532950946804) bank21234 +21235 POINT(34.478966923079604 -118.45415713237294) bank21235 +21236 POINT(33.1055686143697 -118.46407866877506) bank21236 +21237 POINT(33.85219497582825 -118.22452781208534) bank21237 +21238 POINT(33.43364793303796 -118.25621071994777) bank21238 +21239 POINT(34.39479753769429 -117.44400416682426) bank21239 +21240 POINT(34.66196056087885 -117.6236777735823) bank21240 +21241 POINT(33.70325440183158 -118.44168291861389) bank21241 +21242 POINT(34.86947004025356 -117.6946109225388) bank21242 +21243 POINT(34.645383861794144 -118.84174863212013) bank21243 +21244 POINT(34.67297995776453 -118.45406585982896) bank21244 +21245 POINT(34.30271432353453 -118.67207325028492) bank21245 +21246 POINT(34.983523986160996 -117.58130975494714) bank21246 +21247 POINT(33.93980147472364 -118.15571064261749) bank21247 +21248 POINT(33.990919265635156 -117.54260563888526) bank21248 +21249 POINT(33.428831941933815 -117.92644399997062) bank21249 +21250 POINT(33.74583724031037 -119.01893367438008) bank21250 +21251 POINT(33.681476153659986 -119.21877337957613) bank21251 +21252 POINT(34.77212095105285 -117.87465325896974) bank21252 +21253 POINT(33.67908363043837 -117.9856319600474) bank21253 +21254 POINT(33.946589103784675 -117.8062813462524) bank21254 +21255 POINT(33.71964932295303 -118.0514663130299) bank21255 +21256 POINT(33.235767694368825 -118.28441629985826) bank21256 +21257 POINT(34.59109244108305 -118.6020390999434) bank21257 +21258 POINT(34.52600413939081 -118.6723898421259) bank21258 +21259 POINT(33.55316276042434 -117.86564686673204) bank21259 +21260 POINT(34.36416739263499 -117.76047550368271) bank21260 +21261 POINT(33.861332772584426 -119.00325115248802) bank21261 +21262 POINT(33.98019514074567 -117.9384714737926) bank21262 +21263 POINT(33.80948709214179 -119.20395228266216) bank21263 +21264 POINT(33.22366931847206 -117.41450289528848) bank21264 +21265 POINT(34.54725857009652 -118.81770567943434) bank21265 +21266 POINT(34.12399647223473 -118.84404517289968) bank21266 +21267 POINT(34.82867421207923 -117.32551054939375) bank21267 +21268 POINT(34.06525969332566 -118.91420845386897) bank21268 +21269 POINT(33.371440663958396 -117.34398064718162) bank21269 +21270 POINT(34.62196577486581 -118.9755700724238) bank21270 +21271 POINT(33.882278877279234 -119.09577021442907) bank21271 +21272 POINT(33.397442243809444 -118.1240153062814) bank21272 +21273 POINT(33.062141123947214 -117.61763404874826) bank21273 +21274 POINT(33.0583538404146 -118.86666158241243) bank21274 +21275 POINT(33.386020443242955 -118.7923043151556) bank21275 +21276 POINT(33.7979464514289 -119.12300048517868) bank21276 +21277 POINT(34.39559985103537 -118.57689624362966) bank21277 +21278 POINT(33.29337877403133 -119.03964947421564) bank21278 +21279 POINT(33.11545643203732 -117.90361918363978) bank21279 +21280 POINT(33.99368897067591 -117.81466605437694) bank21280 +21281 POINT(33.45365834845832 -118.01994794154066) bank21281 +21282 POINT(33.30360079746778 -118.05809893265757) bank21282 +21283 POINT(33.983646299690925 -117.70930242452856) bank21283 +21284 POINT(33.43849538077266 -117.68270040783372) bank21284 +21285 POINT(33.29593360225154 -117.31346616080731) bank21285 +21286 POINT(34.93146017649303 -118.2032865793646) bank21286 +21287 POINT(34.008762599398175 -118.77528685929123) bank21287 +21288 POINT(34.111689317647354 -118.88330585605561) bank21288 +21289 POINT(34.220249423620345 -117.80782085704882) bank21289 +21290 POINT(33.054140443369526 -118.06751012579919) bank21290 +21291 POINT(34.65147704623946 -117.99310937698954) bank21291 +21292 POINT(34.93940465439061 -118.15873476620102) bank21292 +21293 POINT(33.1275228972619 -118.61256994829851) bank21293 +21294 POINT(34.36974582527875 -117.3930637204) bank21294 +21295 POINT(34.60373954468224 -117.27975044900124) bank21295 +21296 POINT(33.08916583853265 -118.04132246856854) bank21296 +21297 POINT(34.96844358533527 -118.9007822612543) bank21297 +21298 POINT(33.84720673059783 -117.68852040612151) bank21298 +21299 POINT(33.43086291269282 -118.95058376742554) bank21299 +21300 POINT(33.262401129786376 -117.6383544783426) bank21300 +21301 POINT(34.85071166804341 -117.44953113494968) bank21301 +21302 POINT(34.9176626438392 -119.0990737028474) bank21302 +21303 POINT(34.18689139227636 -117.86956789040778) bank21303 +21304 POINT(34.92750699346034 -118.54398267655151) bank21304 +21305 POINT(33.72559796788426 -117.28490076829725) bank21305 +21306 POINT(33.14731461093096 -118.80362840225115) bank21306 +21307 POINT(33.91751823435352 -118.12831540397869) bank21307 +21308 POINT(33.57672749470886 -118.17988498288425) bank21308 +21309 POINT(34.46121310085748 -118.25881245752161) bank21309 +21310 POINT(33.48545053247189 -118.69614917934085) bank21310 +21311 POINT(33.19534292908161 -118.18871240038737) bank21311 +21312 POINT(34.014977546006655 -118.6707846882831) bank21312 +21313 POINT(33.50802603431259 -118.98415531421696) bank21313 +21314 POINT(34.89008494012661 -117.5717386225106) bank21314 +21315 POINT(34.89502119601425 -118.47120912177736) bank21315 +21316 POINT(33.74286300026378 -118.91099528973248) bank21316 +21317 POINT(34.78965161243832 -119.13497504170702) bank21317 +21318 POINT(35.004145465374705 -118.50497386521344) bank21318 +21319 POINT(34.55481948828058 -117.8731448526231) bank21319 +21320 POINT(33.88222639876639 -119.08656512902144) bank21320 +21321 POINT(34.4991614485467 -118.6194051482346) bank21321 +21322 POINT(33.34084655957994 -118.3256120613574) bank21322 +21323 POINT(33.24640691644 -118.56784038560679) bank21323 +21324 POINT(34.93905485268213 -117.72785047085374) bank21324 +21325 POINT(33.914074814047325 -118.7961020402941) bank21325 +21326 POINT(34.40884513602834 -118.28956950754647) bank21326 +21327 POINT(34.46802181912066 -117.89611364623723) bank21327 +21328 POINT(33.27637589144233 -119.1250891453163) bank21328 +21329 POINT(34.15261368052934 -118.36648803957408) bank21329 +21330 POINT(34.83289338503924 -118.64897414119258) bank21330 +21331 POINT(33.51347515115585 -118.05058386958883) bank21331 +21332 POINT(33.4492279588088 -118.08674538476195) bank21332 +21333 POINT(33.554041624676245 -118.09682474688505) bank21333 +21334 POINT(33.490739551431886 -118.06915632711774) bank21334 +21335 POINT(33.163056081070046 -117.3007639550384) bank21335 +21336 POINT(33.30900701418642 -119.02420066596716) bank21336 +21337 POINT(34.21559867650137 -117.56699472385483) bank21337 +21338 POINT(33.68149415051677 -117.70390576403192) bank21338 +21339 POINT(33.57741868007838 -118.93017892608982) bank21339 +21340 POINT(34.27623210443134 -118.66348965467145) bank21340 +21341 POINT(34.90834838679652 -117.7754408652811) bank21341 +21342 POINT(33.83590238881911 -117.24648427579804) bank21342 +21343 POINT(33.42028005688241 -117.57831028804914) bank21343 +21344 POINT(33.26081762316306 -118.91612372134463) bank21344 +21345 POINT(34.37249571797377 -117.85560847907915) bank21345 +21346 POINT(34.45461802640499 -118.79519782766718) bank21346 +21347 POINT(34.75803937320042 -118.33985057446863) bank21347 +21348 POINT(34.86710271165961 -117.49220681861779) bank21348 +21349 POINT(33.2843748963136 -118.06922041080371) bank21349 +21350 POINT(34.638583798491005 -117.80159032763342) bank21350 +21351 POINT(33.847091742816055 -118.73328608307045) bank21351 +21352 POINT(33.12133834128766 -119.24137801642578) bank21352 +21353 POINT(34.71780467028807 -117.42195193163607) bank21353 +21354 POINT(34.94866913974548 -117.58473142103487) bank21354 +21355 POINT(34.94840446035806 -119.01318675861621) bank21355 +21356 POINT(33.57845247973524 -118.41387681573568) bank21356 +21357 POINT(34.60206723967367 -118.17534176652455) bank21357 +21358 POINT(33.25250259334435 -119.21641966408781) bank21358 +21359 POINT(33.82397227694745 -117.91973439946614) bank21359 +21360 POINT(33.92852783419718 -117.25197925387734) bank21360 +21361 POINT(33.53717864662622 -118.9481343991947) bank21361 +21362 POINT(33.09213316620248 -118.89995864812079) bank21362 +21363 POINT(34.744995276312835 -118.79317884752918) bank21363 +21364 POINT(33.27073604501491 -119.07374754980758) bank21364 +21365 POINT(33.21641644457243 -118.76095274601515) bank21365 +21366 POINT(33.30670520800305 -118.51943458365581) bank21366 +21367 POINT(33.27127235788396 -117.25536997321242) bank21367 +21368 POINT(34.76772791327323 -117.42979916295054) bank21368 +21369 POINT(33.792352673041734 -118.46364336021652) bank21369 +21370 POINT(34.44444899775314 -117.95855318125828) bank21370 +21371 POINT(33.852370223301335 -117.56236762010859) bank21371 +21372 POINT(34.35839563720497 -118.04040071254856) bank21372 +21373 POINT(34.73775866897339 -118.45338189913504) bank21373 +21374 POINT(33.61642875652039 -118.20211126041265) bank21374 +21375 POINT(33.278287347117335 -117.58144786187567) bank21375 +21376 POINT(34.657780758502575 -118.26623579345014) bank21376 +21377 POINT(33.14001730569531 -118.96053851678431) bank21377 +21378 POINT(34.3607131274041 -118.56575629416893) bank21378 +21379 POINT(33.082440798638956 -117.82699490661268) bank21379 +21380 POINT(33.823989710294555 -118.77525050181946) bank21380 +21381 POINT(34.39229425341722 -118.52606380226051) bank21381 +21382 POINT(34.68637334919224 -118.3562211107816) bank21382 +21383 POINT(33.507823794078796 -118.30080393360936) bank21383 +21384 POINT(34.55119902542991 -118.0645668107758) bank21384 +21385 POINT(34.19918088585121 -119.2378521189204) bank21385 +21386 POINT(34.61846694047787 -117.81151401367886) bank21386 +21387 POINT(34.32876188747007 -118.97390159822343) bank21387 +21388 POINT(33.167770450854114 -119.05204989440843) bank21388 +21389 POINT(33.13320092749559 -118.17129729186865) bank21389 +21390 POINT(33.80193008075862 -118.83724132818394) bank21390 +21391 POINT(33.88166183837265 -118.83733936954211) bank21391 +21392 POINT(33.47251630543257 -119.02861596792638) bank21392 +21393 POINT(34.12989480920382 -118.53131850802131) bank21393 +21394 POINT(33.17526909605717 -118.0894027192875) bank21394 +21395 POINT(33.67191924606444 -118.7133659800119) bank21395 +21396 POINT(33.701387748832936 -118.96725551014703) bank21396 +21397 POINT(34.34152214605561 -117.84200621972585) bank21397 +21398 POINT(34.04202898007908 -118.13981936738101) bank21398 +21399 POINT(34.478448793634044 -118.93989425650311) bank21399 +21400 POINT(33.16584332097903 -118.0581033866701) bank21400 +21401 POINT(33.31273908725417 -117.62249212005568) bank21401 +21402 POINT(33.540147282518454 -117.28456153246522) bank21402 +21403 POINT(34.47772405545738 -118.82002522674895) bank21403 +21404 POINT(33.49511364819864 -119.01561434642727) bank21404 +21405 POINT(33.18830898825913 -118.77818366343408) bank21405 +21406 POINT(34.971035922379855 -118.25877059451426) bank21406 +21407 POINT(34.13176402746818 -117.88400851995345) bank21407 +21408 POINT(33.982961150409835 -117.60443136047644) bank21408 +21409 POINT(33.19279922401554 -118.69997609325505) bank21409 +21410 POINT(34.69510547666488 -118.43965488268088) bank21410 +21411 POINT(33.30889008937261 -118.63732039492783) bank21411 +21412 POINT(33.233002201155145 -118.27730673505833) bank21412 +21413 POINT(33.19322629664485 -118.23276125739069) bank21413 +21414 POINT(34.62226043253397 -118.12921589501221) bank21414 +21415 POINT(34.73213661692791 -117.62815322083515) bank21415 +21416 POINT(33.69568460663792 -118.50991019073004) bank21416 +21417 POINT(34.96211891094886 -117.26979071597444) bank21417 +21418 POINT(33.719681280489134 -118.32164237317649) bank21418 +21419 POINT(34.04935801930486 -117.62314545945756) bank21419 +21420 POINT(33.93985331437992 -118.48064980771906) bank21420 +21421 POINT(35.0132167226843 -117.91711132959732) bank21421 +21422 POINT(34.568979939265226 -118.31893630448788) bank21422 +21423 POINT(34.030453556807 -118.13325281420009) bank21423 +21424 POINT(33.361371378224035 -119.08564103477647) bank21424 +21425 POINT(33.201544814891236 -118.24917783335992) bank21425 +21426 POINT(33.08054461367391 -118.6054746984824) bank21426 +21427 POINT(33.16437915853788 -119.08890208897047) bank21427 +21428 POINT(33.582583830771156 -118.99485298206227) bank21428 +21429 POINT(34.05432004215329 -118.05876141722767) bank21429 +21430 POINT(34.29139346081193 -118.79768905450055) bank21430 +21431 POINT(34.72752790835976 -118.96348164013666) bank21431 +21432 POINT(34.96682627989684 -118.88004365856318) bank21432 +21433 POINT(33.297529556438576 -117.62636285895326) bank21433 +21434 POINT(33.47514015484567 -118.90604484828017) bank21434 +21435 POINT(34.34453227927238 -117.5189392187524) bank21435 +21436 POINT(34.83861923953709 -118.78760999300319) bank21436 +21437 POINT(34.378072339905934 -117.5968048553227) bank21437 +21438 POINT(33.64579403033816 -119.00069026368746) bank21438 +21439 POINT(34.57226830466068 -117.98407775986443) bank21439 +21440 POINT(33.431515844124995 -118.40560553503192) bank21440 +21441 POINT(33.641252373187086 -117.2699395848626) bank21441 +21442 POINT(33.479016444447865 -118.55929397921163) bank21442 +21443 POINT(33.685235212826925 -119.23348768048155) bank21443 +21444 POINT(34.16488500574613 -118.68931715422382) bank21444 +21445 POINT(34.14098034079604 -118.55666665986598) bank21445 +21446 POINT(33.846159030987984 -118.189674086733) bank21446 +21447 POINT(34.4565338288146 -118.55225675876233) bank21447 +21448 POINT(33.66799235343683 -118.88816805440801) bank21448 +21449 POINT(34.15936311029464 -118.37337239971139) bank21449 +21450 POINT(34.7068577936501 -117.6396535828171) bank21450 +21451 POINT(34.99889482131522 -119.11289210890943) bank21451 +21452 POINT(34.8328852027426 -119.01334922691954) bank21452 +21453 POINT(33.61595532939383 -118.57530614600816) bank21453 +21454 POINT(33.93904369029681 -117.78649298181622) bank21454 +21455 POINT(34.471793781270016 -118.57755496656237) bank21455 +21456 POINT(33.358534467164645 -118.9314591220514) bank21456 +21457 POINT(34.21978870351345 -119.06106849273155) bank21457 +21458 POINT(34.80689426820918 -118.84831205473628) bank21458 +21459 POINT(33.05723475622352 -117.28597729678684) bank21459 +21460 POINT(33.191287744968484 -118.27594722736504) bank21460 +21461 POINT(34.587482489239726 -117.8880321874771) bank21461 +21462 POINT(34.59933253818883 -117.86003525795174) bank21462 +21463 POINT(35.02263588078832 -118.99123102954954) bank21463 +21464 POINT(34.85374841419961 -118.76111133676437) bank21464 +21465 POINT(34.08263372219805 -118.33846615360518) bank21465 +21466 POINT(33.611153620100204 -118.54291691695373) bank21466 +21467 POINT(34.564281095290745 -118.33789606268208) bank21467 +21468 POINT(34.76337390148059 -117.36422726063519) bank21468 +21469 POINT(33.37819169189836 -118.94795730318927) bank21469 +21470 POINT(34.238422546944754 -117.97288709600988) bank21470 +21471 POINT(33.18032950843782 -119.16505480747617) bank21471 +21472 POINT(34.437577230266044 -117.51383904920665) bank21472 +21473 POINT(33.151557162306815 -117.78630767770193) bank21473 +21474 POINT(34.72994108977829 -117.76862085497481) bank21474 +21475 POINT(34.223448794748876 -117.86799993157233) bank21475 +21476 POINT(33.56811529541007 -117.95248248480988) bank21476 +21477 POINT(33.91853427555929 -117.32801559934406) bank21477 +21478 POINT(33.782181316664094 -117.3071988394298) bank21478 +21479 POINT(33.51493661387932 -119.15672059342816) bank21479 +21480 POINT(33.64950392246367 -117.93353704706766) bank21480 +21481 POINT(34.775900325357036 -117.65475826415117) bank21481 +21482 POINT(34.231443275805645 -118.92518104566933) bank21482 +21483 POINT(33.1654095155952 -118.86594027540289) bank21483 +21484 POINT(34.63921807799824 -118.13987789819802) bank21484 +21485 POINT(33.94816356796063 -118.6543241354865) bank21485 +21486 POINT(34.79853426667528 -118.60810223181021) bank21486 +21487 POINT(34.83233566054357 -118.89371412302393) bank21487 +21488 POINT(33.48790287052426 -119.23511968886204) bank21488 +21489 POINT(33.11670938201027 -119.12856168602431) bank21489 +21490 POINT(33.98721474708429 -119.13423731392031) bank21490 +21491 POINT(33.95589936737187 -118.17145281426457) bank21491 +21492 POINT(33.8391951584634 -118.39098832565891) bank21492 +21493 POINT(33.971503202409345 -118.27257117418871) bank21493 +21494 POINT(33.393750021850494 -118.65707300727159) bank21494 +21495 POINT(33.78692725561013 -117.48944818209524) bank21495 +21496 POINT(33.8678261713011 -118.55513807351971) bank21496 +21497 POINT(34.136272956942996 -117.472903143636) bank21497 +21498 POINT(34.576404214573124 -117.25920344773218) bank21498 +21499 POINT(34.022941579865076 -118.86437893905219) bank21499 +21500 POINT(33.398289364150344 -118.61882048688692) bank21500 +21501 POINT(34.935858920120936 -117.53786294728373) bank21501 +21502 POINT(33.50204815358653 -119.12114578566923) bank21502 +21503 POINT(34.35139383446259 -119.07398728537596) bank21503 +21504 POINT(33.89848696280103 -117.58101963945961) bank21504 +21505 POINT(34.96682574567318 -117.69620092475877) bank21505 +21506 POINT(33.90623630029971 -118.85399055465717) bank21506 +21507 POINT(33.84970749557552 -117.47382635634571) bank21507 +21508 POINT(34.45178387274515 -118.0887019285275) bank21508 +21509 POINT(34.07879845581428 -117.29555205996219) bank21509 +21510 POINT(34.644155391355106 -119.00178577938283) bank21510 +21511 POINT(33.37246947867905 -118.30478720498226) bank21511 +21512 POINT(33.23118920057706 -118.47530093663146) bank21512 +21513 POINT(34.61207934001778 -119.03934316454756) bank21513 +21514 POINT(33.34810877177673 -118.99557102440008) bank21514 +21515 POINT(34.30323433511848 -118.24244372521866) bank21515 +21516 POINT(34.45448923113557 -117.91956303259994) bank21516 +21517 POINT(34.13832025413342 -117.28963388145154) bank21517 +21518 POINT(33.96163358231699 -118.51310805646334) bank21518 +21519 POINT(33.226214013138055 -117.65269432428249) bank21519 +21520 POINT(33.29788328838629 -117.84025360609719) bank21520 +21521 POINT(33.546649477343806 -117.9141998636197) bank21521 +21522 POINT(34.57851372372557 -118.60007330757736) bank21522 +21523 POINT(33.139785497397305 -118.46967220151997) bank21523 +21524 POINT(33.85880098364885 -117.34090140366368) bank21524 +21525 POINT(34.89022923904604 -118.63723542778894) bank21525 +21526 POINT(33.17747202595866 -117.88654027123782) bank21526 +21527 POINT(33.573796903470324 -118.44776639858038) bank21527 +21528 POINT(33.13676032444401 -117.85276752912169) bank21528 +21529 POINT(34.386061237569216 -118.05270257614114) bank21529 +21530 POINT(33.90611836720564 -118.84656219134936) bank21530 +21531 POINT(34.518558513800066 -118.43948495870714) bank21531 +21532 POINT(33.69027876205613 -118.6694016146206) bank21532 +21533 POINT(34.458028644495066 -117.64775854950307) bank21533 +21534 POINT(35.03850059271192 -117.98970903366478) bank21534 +21535 POINT(34.36670146945846 -117.32763224547142) bank21535 +21536 POINT(34.21063575374936 -118.33004538459299) bank21536 +21537 POINT(33.214161535699766 -117.59837889867481) bank21537 +21538 POINT(33.93415147283801 -118.72273868558757) bank21538 +21539 POINT(34.57647560581162 -118.03499685319248) bank21539 +21540 POINT(34.828046317435465 -118.01678882061718) bank21540 +21541 POINT(33.80970181486465 -117.72807159543275) bank21541 +21542 POINT(34.33598811465429 -117.87804402308218) bank21542 +21543 POINT(34.26937019149937 -117.36467959738121) bank21543 +21544 POINT(34.88581953945242 -118.28198182302701) bank21544 +21545 POINT(33.398899741511556 -117.52760631846958) bank21545 +21546 POINT(33.96730647643423 -118.00253238725162) bank21546 +21547 POINT(34.675004329102016 -117.49877692058143) bank21547 +21548 POINT(34.70894874152112 -117.31447158646117) bank21548 +21549 POINT(34.362729356227035 -118.05011263119326) bank21549 +21550 POINT(33.41891902066571 -117.58377474395921) bank21550 +21551 POINT(33.692733671161065 -118.35863548185436) bank21551 +21552 POINT(33.83365776364108 -117.78364282868492) bank21552 +21553 POINT(33.328693452153956 -118.28924270381015) bank21553 +21554 POINT(34.88567265344054 -119.22911900563811) bank21554 +21555 POINT(34.911168572735534 -117.95266979131462) bank21555 +21556 POINT(33.7792952395608 -118.60730200492254) bank21556 +21557 POINT(33.263709343157515 -117.95747376837667) bank21557 +21558 POINT(35.02004049690921 -118.68420158897037) bank21558 +21559 POINT(33.67998296003027 -119.0573352146988) bank21559 +21560 POINT(33.29426508754444 -118.70603088369747) bank21560 +21561 POINT(33.432480614987355 -118.81094222341747) bank21561 +21562 POINT(34.22205733938442 -117.77704714352063) bank21562 +21563 POINT(33.733538009567454 -117.80558824182631) bank21563 +21564 POINT(33.7334101098935 -119.15969958518676) bank21564 +21565 POINT(34.59156716560606 -118.48495414975834) bank21565 +21566 POINT(33.21808318533894 -118.01102852936751) bank21566 +21567 POINT(34.4646759899122 -119.06086781116497) bank21567 +21568 POINT(33.9033995192512 -118.97928478150396) bank21568 +21569 POINT(33.54994407172049 -117.89802815669528) bank21569 +21570 POINT(33.6954105531763 -118.38790990235647) bank21570 +21571 POINT(34.63237023231652 -118.79562405344359) bank21571 +21572 POINT(34.259154291250276 -118.75625474020812) bank21572 +21573 POINT(33.446801480984625 -118.18103279734318) bank21573 +21574 POINT(33.33315041706571 -118.71640579500736) bank21574 +21575 POINT(34.05495795259092 -117.87467240665113) bank21575 +21576 POINT(34.659140420615 -118.45114915095049) bank21576 +21577 POINT(33.564926172001485 -117.68010079564017) bank21577 +21578 POINT(34.91909647289262 -117.38624071201195) bank21578 +21579 POINT(34.06419635648488 -118.11557890460142) bank21579 +21580 POINT(33.76057926181832 -119.17614679489654) bank21580 +21581 POINT(33.64384834840342 -117.64962369592982) bank21581 +21582 POINT(33.985165900317334 -118.91940268452767) bank21582 +21583 POINT(34.35752162377553 -118.97186597929534) bank21583 +21584 POINT(33.09332230179742 -119.17265599012734) bank21584 +21585 POINT(34.1553048685079 -117.76806893796189) bank21585 +21586 POINT(34.03030617887984 -118.78172245075312) bank21586 +21587 POINT(34.36277870233598 -118.87498227263755) bank21587 +21588 POINT(33.53705847756278 -119.10932622365517) bank21588 +21589 POINT(33.762526262841405 -118.81007916864375) bank21589 +21590 POINT(34.672806314420676 -117.33357969850368) bank21590 +21591 POINT(33.76176979826021 -118.19992737710089) bank21591 +21592 POINT(33.235901748671644 -118.08671979273325) bank21592 +21593 POINT(33.73986058520584 -117.46681644760204) bank21593 +21594 POINT(34.79560324975527 -117.344109114213) bank21594 +21595 POINT(33.55572795642883 -118.72772366797888) bank21595 +21596 POINT(34.216818150718034 -117.98278388397725) bank21596 +21597 POINT(33.44892135278398 -117.52824238447765) bank21597 +21598 POINT(34.32352571977415 -117.39406265362405) bank21598 +21599 POINT(34.99852590015502 -119.01482333622825) bank21599 +21600 POINT(33.878014161176125 -118.03952657231847) bank21600 +21601 POINT(34.33660927508856 -118.19094228666921) bank21601 +21602 POINT(34.93670236066767 -118.5540156329564) bank21602 +21603 POINT(34.488563057195606 -117.3582619070568) bank21603 +21604 POINT(34.28300075382654 -117.71033493893478) bank21604 +21605 POINT(33.778570866065394 -119.15041937284334) bank21605 +21606 POINT(34.185261636109566 -117.6636535697222) bank21606 +21607 POINT(33.51671128732571 -118.60471546143536) bank21607 +21608 POINT(33.77929302203509 -117.48676551404226) bank21608 +21609 POINT(34.72006644062234 -118.35594065956003) bank21609 +21610 POINT(34.71224800219812 -118.50783390996041) bank21610 +21611 POINT(34.421534929961524 -117.84081052638831) bank21611 +21612 POINT(33.823271330789 -118.78812024406093) bank21612 +21613 POINT(34.11462148460165 -118.7403292228231) bank21613 +21614 POINT(34.57510186065371 -118.42893775325216) bank21614 +21615 POINT(34.02197284351855 -118.86706510581759) bank21615 +21616 POINT(34.21314920613304 -119.03207149940553) bank21616 +21617 POINT(33.904650192528344 -118.94841815745383) bank21617 +21618 POINT(33.16509565898615 -118.13336799213499) bank21618 +21619 POINT(34.465007275012994 -119.02044960734801) bank21619 +21620 POINT(33.7803795681425 -118.39931563773317) bank21620 +21621 POINT(34.429079456861025 -117.66227315465238) bank21621 +21622 POINT(34.40576840149281 -117.66970259284065) bank21622 +21623 POINT(34.34749100742396 -117.33626013636594) bank21623 +21624 POINT(35.05091048845639 -118.42125428548577) bank21624 +21625 POINT(34.22232169020878 -117.8862059901228) bank21625 +21626 POINT(33.98357676536497 -118.65926846792489) bank21626 +21627 POINT(34.65757804376878 -117.47856721257543) bank21627 +21628 POINT(35.01214711988701 -118.92660982425382) bank21628 +21629 POINT(33.887054114205725 -117.44681831628901) bank21629 +21630 POINT(34.547110059342934 -118.19117810615155) bank21630 +21631 POINT(34.994989243548254 -117.52772025010448) bank21631 +21632 POINT(34.39922666106773 -118.28955694985312) bank21632 +21633 POINT(34.062519521911724 -118.3327165561696) bank21633 +21634 POINT(33.074208253623794 -119.07478043007721) bank21634 +21635 POINT(34.401073175314124 -118.64925281174641) bank21635 +21636 POINT(33.56392799262322 -118.58374502155684) bank21636 +21637 POINT(34.954020754267816 -118.46784920122346) bank21637 +21638 POINT(33.60157453076738 -119.243651967937) bank21638 +21639 POINT(34.432170610428344 -118.76949365078313) bank21639 +21640 POINT(33.292065920524564 -117.77904524996788) bank21640 +21641 POINT(34.760303093399145 -118.45204585623749) bank21641 +21642 POINT(34.275404345364315 -117.30846098924867) bank21642 +21643 POINT(33.68144864539544 -117.45431943507991) bank21643 +21644 POINT(33.47945887603851 -118.03995819993737) bank21644 +21645 POINT(34.4641117438327 -118.00718136521664) bank21645 +21646 POINT(33.50677611395465 -117.99115139517642) bank21646 +21647 POINT(33.70672216566792 -117.76097920618537) bank21647 +21648 POINT(33.77346295147345 -118.12831062770653) bank21648 +21649 POINT(35.04797812671715 -119.04267897812522) bank21649 +21650 POINT(34.861424793751794 -117.4214155977433) bank21650 +21651 POINT(33.875414038816714 -118.87390199507509) bank21651 +21652 POINT(34.56523003325051 -118.15122770585896) bank21652 +21653 POINT(34.96413047207553 -117.79037233996776) bank21653 +21654 POINT(33.85800690814026 -117.49664800197817) bank21654 +21655 POINT(33.15532380258154 -117.46642266708318) bank21655 +21656 POINT(34.387825465529794 -118.27822920278541) bank21656 +21657 POINT(34.28772315460051 -119.07883126460422) bank21657 +21658 POINT(33.54472165492204 -118.86517449217617) bank21658 +21659 POINT(33.39061969996097 -118.7686537521538) bank21659 +21660 POINT(34.871013982195414 -118.4934777642315) bank21660 +21661 POINT(34.55614380266918 -117.7216547841672) bank21661 +21662 POINT(33.53442648822771 -117.55866787587017) bank21662 +21663 POINT(34.576377459482764 -118.44563241900549) bank21663 +21664 POINT(33.86429758647144 -118.8761631962349) bank21664 +21665 POINT(34.67438974569622 -118.32085172997033) bank21665 +21666 POINT(34.00666989563756 -119.07105120438803) bank21666 +21667 POINT(34.41061374621661 -117.73418981845629) bank21667 +21668 POINT(34.959518524291525 -118.74126496103919) bank21668 +21669 POINT(35.04110100961924 -117.47178449239689) bank21669 +21670 POINT(33.75364846285674 -118.3050576683714) bank21670 +21671 POINT(33.91661316298729 -118.48950697692943) bank21671 +21672 POINT(34.28424017276101 -118.92859745225118) bank21672 +21673 POINT(33.656543298572515 -117.49363970180569) bank21673 +21674 POINT(33.53429828049974 -117.77720480697315) bank21674 +21675 POINT(34.40809003388402 -117.50808607741075) bank21675 +21676 POINT(34.647425329226685 -118.42498055993241) bank21676 +21677 POINT(33.51219166665862 -118.93797408038863) bank21677 +21678 POINT(34.100633789018865 -118.26723223439707) bank21678 +21679 POINT(33.4743087478496 -118.66822510738704) bank21679 +21680 POINT(34.358881287286906 -118.96214698743216) bank21680 +21681 POINT(34.833270435164245 -118.50924769248235) bank21681 +21682 POINT(34.19131528730713 -118.71748731695294) bank21682 +21683 POINT(34.14756196179821 -118.61343058114765) bank21683 +21684 POINT(34.12943426150043 -118.42026838455784) bank21684 +21685 POINT(34.5810222034969 -119.2228057985298) bank21685 +21686 POINT(33.48447927095789 -118.28000018030549) bank21686 +21687 POINT(33.504613789528875 -117.26282253296495) bank21687 +21688 POINT(33.31037342155585 -118.83470960473073) bank21688 +21689 POINT(33.69951730268647 -119.07574312793335) bank21689 +21690 POINT(34.31872067783684 -118.43585665415897) bank21690 +21691 POINT(34.11556854036503 -117.88785184122229) bank21691 +21692 POINT(34.56601607337999 -118.52156544490776) bank21692 +21693 POINT(33.154292456190106 -117.51306474553274) bank21693 +21694 POINT(34.79339923676119 -117.77370039230063) bank21694 +21695 POINT(33.26303783135549 -118.83178344517329) bank21695 +21696 POINT(33.57522938201133 -118.48891100629668) bank21696 +21697 POINT(34.3202280517895 -117.85269900858509) bank21697 +21698 POINT(33.1917593637147 -117.61608164716344) bank21698 +21699 POINT(34.86750810476157 -118.3616654194203) bank21699 +21700 POINT(33.59600664408307 -119.10004871393416) bank21700 +21701 POINT(34.110086624845195 -117.3995846999206) bank21701 +21702 POINT(34.753141917491924 -119.21608771987019) bank21702 +21703 POINT(33.508508828187125 -117.58865132349807) bank21703 +21704 POINT(34.07442366868588 -117.79963365400616) bank21704 +21705 POINT(34.96203837606215 -117.48079095977971) bank21705 +21706 POINT(34.19066566322022 -117.62017877896376) bank21706 +21707 POINT(34.20521006003637 -117.92239717333608) bank21707 +21708 POINT(33.91551460786721 -118.64053022670478) bank21708 +21709 POINT(33.4928565639228 -119.03683926817715) bank21709 +21710 POINT(34.69487282843397 -118.17010987372981) bank21710 +21711 POINT(33.7399037781362 -118.88529101676795) bank21711 +21712 POINT(34.129502226745764 -118.4957503576233) bank21712 +21713 POINT(34.50369330714235 -117.63328804655036) bank21713 +21714 POINT(34.06821754303305 -119.10903138432771) bank21714 +21715 POINT(34.28504067033405 -118.03031892521054) bank21715 +21716 POINT(34.45019566324802 -118.34571786562724) bank21716 +21717 POINT(34.567138857166285 -118.187970054241) bank21717 +21718 POINT(33.229846854428374 -117.80271094188147) bank21718 +21719 POINT(34.42793780492032 -118.36416464732233) bank21719 +21720 POINT(34.16540088890933 -117.30244718063594) bank21720 +21721 POINT(33.55852539180914 -118.8026062754443) bank21721 +21722 POINT(34.86341309108016 -117.9020414830238) bank21722 +21723 POINT(34.82020088547544 -118.32583380284572) bank21723 +21724 POINT(33.89447436514471 -118.93656541021255) bank21724 +21725 POINT(33.64824994345877 -117.52154826971775) bank21725 +21726 POINT(33.71044089131332 -118.8394865173576) bank21726 +21727 POINT(33.3474846301608 -118.2546213818283) bank21727 +21728 POINT(33.905037231018575 -118.15885702941664) bank21728 +21729 POINT(34.7668344977489 -118.92008336474464) bank21729 +21730 POINT(34.23968428453541 -118.09765509174629) bank21730 +21731 POINT(34.42848596185734 -118.42037694088592) bank21731 +21732 POINT(33.09575626619276 -117.75087269983966) bank21732 +21733 POINT(33.33618177213622 -117.90133292726125) bank21733 +21734 POINT(34.90031540211293 -118.95420467103231) bank21734 +21735 POINT(33.93767607595308 -118.85669382880677) bank21735 +21736 POINT(34.23699100785623 -118.37380336725147) bank21736 +21737 POINT(34.28572210947707 -117.40021489116674) bank21737 +21738 POINT(34.617105533392326 -117.29394953614336) bank21738 +21739 POINT(34.89837155310232 -117.97165585976974) bank21739 +21740 POINT(34.92634852249195 -118.20034951736287) bank21740 +21741 POINT(34.58080499582286 -118.5215240257987) bank21741 +21742 POINT(34.52912733504852 -118.52152577459974) bank21742 +21743 POINT(33.30136931739195 -117.75314682900273) bank21743 +21744 POINT(33.80233681602291 -118.62230574594929) bank21744 +21745 POINT(34.71332035235444 -118.54468434474516) bank21745 +21746 POINT(34.97520207230337 -118.19071557445231) bank21746 +21747 POINT(34.84102610391598 -118.38835489630463) bank21747 +21748 POINT(33.334914318884785 -117.94747715871509) bank21748 +21749 POINT(34.93219429936346 -117.38440866941828) bank21749 +21750 POINT(34.72574366639095 -118.47000215818016) bank21750 +21751 POINT(33.35197541672576 -117.50935361194949) bank21751 +21752 POINT(33.647284352149576 -117.56981158198369) bank21752 +21753 POINT(34.27808935407947 -117.55740566613267) bank21753 +21754 POINT(34.203991595789894 -117.8238585615945) bank21754 +21755 POINT(34.656004519413756 -118.87859352050089) bank21755 +21756 POINT(34.016512718306785 -118.75255873393412) bank21756 +21757 POINT(33.87402126742049 -117.49802358078206) bank21757 +21758 POINT(34.26563752120325 -117.87078947087839) bank21758 +21759 POINT(34.02982549205466 -117.54534437242664) bank21759 +21760 POINT(34.683419654253015 -119.10291942641777) bank21760 +21761 POINT(34.901456133240195 -118.62855895006676) bank21761 +21762 POINT(33.68649960577368 -117.94042609714597) bank21762 +21763 POINT(33.97243715499234 -118.85465083897546) bank21763 +21764 POINT(34.89223005327415 -119.23781639935572) bank21764 +21765 POINT(33.184178849277004 -118.92903901425044) bank21765 +21766 POINT(33.147319574465634 -118.44656253527323) bank21766 +21767 POINT(34.460124903457356 -118.4755947316837) bank21767 +21768 POINT(33.594897924090105 -117.48637706061653) bank21768 +21769 POINT(33.90595897161678 -118.07702324259537) bank21769 +21770 POINT(34.896032683212816 -117.31643445791143) bank21770 +21771 POINT(33.33555097396278 -118.7283180585122) bank21771 +21772 POINT(34.171109548020766 -117.85093249238533) bank21772 +21773 POINT(34.016850586166704 -118.23114305082639) bank21773 +21774 POINT(34.38365370811229 -118.91831099228467) bank21774 +21775 POINT(34.51658054064892 -118.9860233584395) bank21775 +21776 POINT(34.03867606533972 -118.75131115332205) bank21776 +21777 POINT(33.80054420352196 -117.79435079010341) bank21777 +21778 POINT(33.07830542422376 -118.94310343778223) bank21778 +21779 POINT(34.52633070186237 -117.98637517297345) bank21779 +21780 POINT(33.90265508420546 -118.68202541863691) bank21780 +21781 POINT(34.213623179639576 -118.58722903123126) bank21781 +21782 POINT(34.17540747496316 -118.01931260590291) bank21782 +21783 POINT(33.14874391726805 -118.42955641568913) bank21783 +21784 POINT(34.2515201734455 -117.4930729440745) bank21784 +21785 POINT(33.969071086617966 -117.88924087602395) bank21785 +21786 POINT(34.00351086372747 -118.40688997167823) bank21786 +21787 POINT(33.87168474475128 -117.50641929819113) bank21787 +21788 POINT(34.57839034554097 -119.00777662647323) bank21788 +21789 POINT(33.5440642380863 -118.12380232874109) bank21789 +21790 POINT(33.78150280072717 -119.10976171690922) bank21790 +21791 POINT(33.616648319195015 -117.3623133477756) bank21791 +21792 POINT(33.44061102678652 -117.48655031290826) bank21792 +21793 POINT(34.43971446935145 -118.76940354873683) bank21793 +21794 POINT(34.26056907916116 -117.97516850878658) bank21794 +21795 POINT(34.854360575066416 -118.51654436052365) bank21795 +21796 POINT(33.915485343646466 -118.31536696632244) bank21796 +21797 POINT(33.346267712109324 -118.03053030243191) bank21797 +21798 POINT(33.53855755998665 -118.09784503693332) bank21798 +21799 POINT(34.79177435135718 -118.67448669240592) bank21799 +21800 POINT(34.63306179429218 -118.50240140374876) bank21800 +21801 POINT(34.01125762350851 -117.49620995868548) bank21801 +21802 POINT(33.783673542094995 -117.60958362662336) bank21802 +21803 POINT(34.80793586990754 -117.25878047683253) bank21803 +21804 POINT(34.01422260285851 -118.70581362289604) bank21804 +21805 POINT(33.18440770968592 -118.66221926655952) bank21805 +21806 POINT(34.44363087308942 -117.28790398138078) bank21806 +21807 POINT(33.35010096656274 -119.06058741377207) bank21807 +21808 POINT(33.09480216155955 -118.91662444798348) bank21808 +21809 POINT(34.56883487566488 -118.25117163332013) bank21809 +21810 POINT(34.1693004153829 -117.39173376557379) bank21810 +21811 POINT(34.613599853961894 -117.6657394632201) bank21811 +21812 POINT(34.92270072187815 -118.79794120450983) bank21812 +21813 POINT(34.210825683026606 -119.08533571841923) bank21813 +21814 POINT(34.528395079326444 -118.84757943066586) bank21814 +21815 POINT(33.08810741435684 -118.53825401011574) bank21815 +21816 POINT(34.6640721068603 -119.19180483348015) bank21816 +21817 POINT(33.48958622667657 -118.20911089562061) bank21817 +21818 POINT(34.56677452242431 -118.98443440134456) bank21818 +21819 POINT(33.257566323146634 -117.2848370590072) bank21819 +21820 POINT(34.157758969352095 -118.54987475330907) bank21820 +21821 POINT(33.114108928402885 -118.61925596637617) bank21821 +21822 POINT(34.643580855151406 -119.14929044700122) bank21822 +21823 POINT(34.91591096349961 -117.39727617214947) bank21823 +21824 POINT(33.13886627377699 -117.73302474940749) bank21824 +21825 POINT(33.236366077656264 -117.9486316152292) bank21825 +21826 POINT(35.051925697938415 -118.07069873003528) bank21826 +21827 POINT(34.91196984967863 -118.71087986593426) bank21827 +21828 POINT(34.948912613299804 -119.09562203967928) bank21828 +21829 POINT(33.40773955178232 -119.13048804766484) bank21829 +21830 POINT(34.97185242639226 -117.7548747042135) bank21830 +21831 POINT(33.345398594723356 -118.21097273473018) bank21831 +21832 POINT(34.92241894923199 -117.46827440715658) bank21832 +21833 POINT(33.08072082914843 -118.49887307070561) bank21833 +21834 POINT(33.47446671093092 -117.59868948351708) bank21834 +21835 POINT(34.89703169630382 -117.28223804614301) bank21835 +21836 POINT(34.024863269329636 -117.28444430221505) bank21836 +21837 POINT(33.96491520825557 -118.93245636891758) bank21837 +21838 POINT(33.107350585211115 -117.73612953156282) bank21838 +21839 POINT(33.2921223058788 -117.25258950450721) bank21839 +21840 POINT(34.50202102787083 -118.16444849898124) bank21840 +21841 POINT(33.173012593759935 -117.82340526724474) bank21841 +21842 POINT(34.512241326947624 -119.14279693675958) bank21842 +21843 POINT(34.68544198502022 -117.8271578487176) bank21843 +21844 POINT(33.30960999921247 -118.20083065408703) bank21844 +21845 POINT(33.60171699282588 -117.65602537559417) bank21845 +21846 POINT(33.518048519368435 -117.61068829065891) bank21846 +21847 POINT(33.394285706746636 -118.86710341876122) bank21847 +21848 POINT(34.35446054389685 -118.37149017543085) bank21848 +21849 POINT(33.73905231501112 -117.59255047837061) bank21849 +21850 POINT(33.479091634665814 -118.10340837058546) bank21850 +21851 POINT(33.640922519767464 -117.92457023495051) bank21851 +21852 POINT(33.104098090926705 -117.4776571126214) bank21852 +21853 POINT(34.72479236273719 -119.14935108982569) bank21853 +21854 POINT(33.41088855886424 -118.30715617534428) bank21854 +21855 POINT(34.35153282309372 -117.71105421441015) bank21855 +21856 POINT(33.677751797801626 -119.02035905666763) bank21856 +21857 POINT(34.17638369108231 -117.82716500885765) bank21857 +21858 POINT(34.875764782433414 -117.68346482493997) bank21858 +21859 POINT(33.941053701064355 -118.66670400396546) bank21859 +21860 POINT(34.03870880297867 -118.31674495691266) bank21860 +21861 POINT(33.51894942847878 -117.76392653174106) bank21861 +21862 POINT(33.39454119303072 -117.72687071898437) bank21862 +21863 POINT(34.35377215675121 -118.05179438409304) bank21863 +21864 POINT(34.119880031917916 -117.50059265198614) bank21864 +21865 POINT(34.34203783777037 -117.7529080036183) bank21865 +21866 POINT(34.52360715867126 -118.65726882832406) bank21866 +21867 POINT(34.53530849982714 -117.52990991229275) bank21867 +21868 POINT(33.347966699984475 -117.47050310298238) bank21868 +21869 POINT(34.037935124643965 -118.26553494753665) bank21869 +21870 POINT(34.49943453728827 -118.94648346062925) bank21870 +21871 POINT(34.35310082073977 -118.44755300599313) bank21871 +21872 POINT(33.55633921685495 -118.11548936569672) bank21872 +21873 POINT(34.81303746439934 -118.57529408363919) bank21873 +21874 POINT(34.08810146241514 -119.069832576733) bank21874 +21875 POINT(33.26419700894285 -117.71642093057797) bank21875 +21876 POINT(34.95717100882941 -118.44667454735925) bank21876 +21877 POINT(33.77530392266254 -118.21485352992997) bank21877 +21878 POINT(33.232034905581415 -117.49553072069997) bank21878 +21879 POINT(33.80027313906127 -118.42653142227243) bank21879 +21880 POINT(34.649207526588576 -117.899358701381) bank21880 +21881 POINT(34.65850223232161 -119.13690888515274) bank21881 +21882 POINT(33.70634136610101 -118.17989978913148) bank21882 +21883 POINT(34.95977981367635 -117.59763602373387) bank21883 +21884 POINT(34.44495653742614 -118.02943117363549) bank21884 +21885 POINT(34.68086089596777 -118.30206154677663) bank21885 +21886 POINT(34.00613559761532 -117.77897715452957) bank21886 +21887 POINT(33.702607510592536 -118.61103607449527) bank21887 +21888 POINT(33.42004548444603 -118.67468013030197) bank21888 +21889 POINT(34.83527791218599 -118.11536604165396) bank21889 +21890 POINT(34.22333993408268 -117.7261354954569) bank21890 +21891 POINT(33.156782089486796 -118.6567785756988) bank21891 +21892 POINT(33.6372602120064 -117.87067421133793) bank21892 +21893 POINT(34.026950174481655 -118.91553824223456) bank21893 +21894 POINT(34.10178860634508 -119.0540231371264) bank21894 +21895 POINT(34.49668398807538 -118.46178012369916) bank21895 +21896 POINT(33.648511854474485 -118.5667239613945) bank21896 +21897 POINT(33.33017293483262 -119.11459322301648) bank21897 +21898 POINT(33.72319480426612 -118.83741086577034) bank21898 +21899 POINT(34.28606184991771 -117.47764716961875) bank21899 +21900 POINT(33.51004838357732 -117.89378088880495) bank21900 +21901 POINT(33.089986991645766 -118.06447919998898) bank21901 +21902 POINT(33.95137855161411 -117.64023624223627) bank21902 +21903 POINT(34.06880204147174 -117.96328633954283) bank21903 +21904 POINT(34.002983404763135 -117.86346890643773) bank21904 +21905 POINT(35.003094228780384 -117.60928843342926) bank21905 +21906 POINT(34.13360429887494 -118.67407886088344) bank21906 +21907 POINT(34.05547169813262 -119.22319277478657) bank21907 +21908 POINT(34.99260220510333 -117.59830503827413) bank21908 +21909 POINT(33.26560581095687 -118.21435841016877) bank21909 +21910 POINT(34.50131988813689 -119.1225581572119) bank21910 +21911 POINT(34.08223699865975 -117.304296756597) bank21911 +21912 POINT(34.720640360512164 -118.2596281378557) bank21912 +21913 POINT(33.31201907315273 -119.17813722552997) bank21913 +21914 POINT(33.89626784312379 -118.66656500050273) bank21914 +21915 POINT(33.749075210392036 -118.73993600889256) bank21915 +21916 POINT(34.75008356788556 -118.60218572705153) bank21916 +21917 POINT(34.150424876428545 -117.26861816089783) bank21917 +21918 POINT(34.375881394656375 -117.91725736125169) bank21918 +21919 POINT(34.44238429126859 -118.64282363708757) bank21919 +21920 POINT(33.15205905555472 -119.20364728465617) bank21920 +21921 POINT(34.427287336066584 -118.6722077897761) bank21921 +21922 POINT(34.88780323269784 -117.44827900932997) bank21922 +21923 POINT(34.015695924057724 -117.30605564759134) bank21923 +21924 POINT(33.36278854965298 -117.65471301277552) bank21924 +21925 POINT(33.39549719594806 -118.02069144743683) bank21925 +21926 POINT(33.56127460464716 -118.36685262912623) bank21926 +21927 POINT(33.13655812222858 -119.1484249552327) bank21927 +21928 POINT(33.10373386907685 -118.00192552606063) bank21928 +21929 POINT(34.660516162307836 -118.58047813972672) bank21929 +21930 POINT(34.27347329896733 -117.25378994851346) bank21930 +21931 POINT(34.2679944680397 -119.19129982131543) bank21931 +21932 POINT(33.366131908777646 -118.74711501475564) bank21932 +21933 POINT(33.412296549660965 -118.93990089175577) bank21933 +21934 POINT(34.58637500128591 -118.15246617839978) bank21934 +21935 POINT(33.11417073771584 -117.87929724573758) bank21935 +21936 POINT(33.24684729022242 -118.64788993189919) bank21936 +21937 POINT(33.266295813726046 -118.50441449211648) bank21937 +21938 POINT(33.07917125600394 -118.82476098709782) bank21938 +21939 POINT(34.786945565557204 -117.71841220706258) bank21939 +21940 POINT(34.79862082771416 -118.49259298582375) bank21940 +21941 POINT(33.5835854696756 -117.33656746458222) bank21941 +21942 POINT(33.58926876514992 -118.34910699291775) bank21942 +21943 POINT(33.640967918513184 -117.40980133137276) bank21943 +21944 POINT(34.87578035377302 -118.07330677506015) bank21944 +21945 POINT(33.17856887017412 -117.87432242710086) bank21945 +21946 POINT(34.33385891341132 -117.542220510208) bank21946 +21947 POINT(34.71170695147338 -117.87758374451504) bank21947 +21948 POINT(33.84056909608201 -117.5647807845561) bank21948 +21949 POINT(34.779805901631406 -117.64386487119113) bank21949 +21950 POINT(34.612321269173535 -117.40805682886369) bank21950 +21951 POINT(33.890013138228944 -119.11670795287039) bank21951 +21952 POINT(34.964786879439266 -118.00772641205833) bank21952 +21953 POINT(33.237667488143984 -117.90413818971602) bank21953 +21954 POINT(34.827958037380434 -117.91473757347715) bank21954 +21955 POINT(34.57751915448316 -118.56845860395593) bank21955 +21956 POINT(33.5162421296587 -118.1159107473148) bank21956 +21957 POINT(33.32427883042268 -118.25842559945069) bank21957 +21958 POINT(35.028705026057885 -118.99457408622217) bank21958 +21959 POINT(33.79466211529642 -118.14871806942737) bank21959 +21960 POINT(34.89772966321071 -117.84292131931434) bank21960 +21961 POINT(34.90121869386936 -118.44169391341532) bank21961 +21962 POINT(33.63704324221248 -118.75218044604937) bank21962 +21963 POINT(34.12989480848841 -117.27911907689929) bank21963 +21964 POINT(34.8294249448708 -118.50607239921919) bank21964 +21965 POINT(33.505385149363484 -117.4223647971274) bank21965 +21966 POINT(33.54219977892702 -117.64969035773899) bank21966 +21967 POINT(34.66355706393183 -118.4886289633278) bank21967 +21968 POINT(34.00184901235642 -118.97618841760507) bank21968 +21969 POINT(33.568870768278906 -118.30244831381354) bank21969 +21970 POINT(33.67442621577066 -118.25934320309422) bank21970 +21971 POINT(34.01368331252151 -117.54489353280431) bank21971 +21972 POINT(33.55172632486572 -117.48770231332402) bank21972 +21973 POINT(33.581864907222865 -118.86171784113438) bank21973 +21974 POINT(34.37789335706631 -118.6572367316457) bank21974 +21975 POINT(34.83265524455486 -117.49713918153444) bank21975 +21976 POINT(33.43871532495056 -117.4056764477331) bank21976 +21977 POINT(34.43207515852771 -118.22479748089683) bank21977 +21978 POINT(33.42524234482557 -118.6020785489242) bank21978 +21979 POINT(34.12595989629774 -117.70678859321077) bank21979 +21980 POINT(34.68546844452688 -118.49000580690704) bank21980 +21981 POINT(33.54750114774405 -117.35430928678265) bank21981 +21982 POINT(33.8831544133449 -117.4385403725438) bank21982 +21983 POINT(34.9804505392036 -118.04263496701344) bank21983 +21984 POINT(33.557844734213035 -118.23091438135847) bank21984 +21985 POINT(33.63546062718065 -117.2738498087119) bank21985 +21986 POINT(33.933688071871984 -118.52462904216013) bank21986 +21987 POINT(35.00387127935037 -118.77258113744996) bank21987 +21988 POINT(34.30686354310442 -118.2208158739321) bank21988 +21989 POINT(34.01477285417415 -119.08764302091342) bank21989 +21990 POINT(33.70031308920065 -117.3322941017249) bank21990 +21991 POINT(33.89426737175161 -117.50703987036083) bank21991 +21992 POINT(34.175985510387555 -118.81757254785016) bank21992 +21993 POINT(33.33661909683708 -118.5167129476723) bank21993 +21994 POINT(34.19579491936267 -117.34495890842818) bank21994 +21995 POINT(34.943451846695325 -117.5395172481656) bank21995 +21996 POINT(33.43509369981897 -118.03386584611209) bank21996 +21997 POINT(34.21483747935142 -118.05648557599713) bank21997 +21998 POINT(33.20362592222711 -118.87205702979226) bank21998 +21999 POINT(33.96064732783878 -118.77842396602335) bank21999 +22000 POINT(33.28600507286496 -117.75358873424652) bank22000 +22001 POINT(35.037557209710215 -118.72042948415528) bank22001 +22002 POINT(34.46611809130024 -118.26687569412141) bank22002 +22003 POINT(33.61781450346491 -119.16212708022036) bank22003 +22004 POINT(34.55670658660381 -119.03074949974865) bank22004 +22005 POINT(34.395114069718446 -118.64972440780201) bank22005 +22006 POINT(34.7992984312184 -117.78043292235536) bank22006 +22007 POINT(33.245378560579304 -118.11505269410888) bank22007 +22008 POINT(34.93014584218159 -118.74269263194394) bank22008 +22009 POINT(33.51376077369729 -117.48944944689573) bank22009 +22010 POINT(33.252251466533366 -118.17373186681803) bank22010 +22011 POINT(33.837216245914426 -118.1124414853868) bank22011 +22012 POINT(34.14195927038719 -118.96559583300822) bank22012 +22013 POINT(33.58112504472894 -117.58809291810762) bank22013 +22014 POINT(33.697605576711865 -117.31873894482882) bank22014 +22015 POINT(33.226543686161705 -118.05379770677743) bank22015 +22016 POINT(34.68589977589724 -117.94706923837616) bank22016 +22017 POINT(33.25073753755816 -118.11638173510674) bank22017 +22018 POINT(33.35149279176907 -118.44151021147049) bank22018 +22019 POINT(33.175743407113444 -117.28616850501916) bank22019 +22020 POINT(34.923585051525116 -119.13709915960278) bank22020 +22021 POINT(33.31578787398761 -118.30973258671872) bank22021 +22022 POINT(33.930662534649116 -119.16686970895773) bank22022 +22023 POINT(33.58412365714252 -118.02415482124626) bank22023 +22024 POINT(33.81954542982029 -118.71124887653916) bank22024 +22025 POINT(33.395579390648265 -119.2365316420693) bank22025 +22026 POINT(34.63124287740721 -117.41363560464522) bank22026 +22027 POINT(33.20530193774633 -118.74899312902762) bank22027 +22028 POINT(34.10750535966653 -118.4602206369174) bank22028 +22029 POINT(33.213150175271956 -117.58319642046132) bank22029 +22030 POINT(33.9023329083727 -118.34279003916156) bank22030 +22031 POINT(33.70294013689599 -118.52688582104804) bank22031 +22032 POINT(34.3011689849666 -118.58955768189276) bank22032 +22033 POINT(33.27378461090447 -119.13082792441115) bank22033 +22034 POINT(33.09414318605393 -118.47654842514618) bank22034 +22035 POINT(33.74627072875602 -117.98904705842145) bank22035 +22036 POINT(33.1034440823055 -117.49608889465131) bank22036 +22037 POINT(34.94175357316303 -118.18210359195427) bank22037 +22038 POINT(33.274681741050145 -117.6603131790438) bank22038 +22039 POINT(34.58571789791003 -117.90368586854723) bank22039 +22040 POINT(33.8400205587696 -119.16633268040512) bank22040 +22041 POINT(33.53987084965575 -117.2533111727168) bank22041 +22042 POINT(33.29022823417775 -117.36216605459443) bank22042 +22043 POINT(33.95537644350931 -117.48871146897214) bank22043 +22044 POINT(33.42900968020579 -118.95196138663371) bank22044 +22045 POINT(33.51495718202102 -119.12456146332543) bank22045 +22046 POINT(33.43002651039443 -117.96150197743638) bank22046 +22047 POINT(34.685594338404385 -118.04133583453468) bank22047 +22048 POINT(34.42388604219402 -118.9720630691789) bank22048 +22049 POINT(34.767996872316964 -118.2622206610848) bank22049 +22050 POINT(33.75609607864799 -117.84028028196322) bank22050 +22051 POINT(34.998169312155234 -117.59885166901678) bank22051 +22052 POINT(34.01417495460879 -118.67003547270556) bank22052 +22053 POINT(34.293575231269685 -117.87751847320497) bank22053 +22054 POINT(34.73913349421265 -118.60491265164696) bank22054 +22055 POINT(34.34642445070259 -118.44769855441689) bank22055 +22056 POINT(34.18622701649017 -118.01451676435094) bank22056 +22057 POINT(34.907658966621796 -118.66110837309837) bank22057 +22058 POINT(34.875051325737275 -119.12935825800234) bank22058 +22059 POINT(33.2309509083915 -117.32780466864322) bank22059 +22060 POINT(33.58087206864543 -119.0148200751189) bank22060 +22061 POINT(34.58883529577524 -117.54565575774363) bank22061 +22062 POINT(33.41356757360853 -118.79684278489813) bank22062 +22063 POINT(33.97476381230482 -117.78351904430771) bank22063 +22064 POINT(34.32491666964866 -117.78221368109905) bank22064 +22065 POINT(34.782590015808054 -117.95723869808289) bank22065 +22066 POINT(34.34399131673584 -118.64870723044092) bank22066 +22067 POINT(34.41561611114518 -117.27487477984671) bank22067 +22068 POINT(33.13555652154192 -118.9533931017486) bank22068 +22069 POINT(34.750181962188805 -117.30747003722416) bank22069 +22070 POINT(34.54224564768563 -118.09580662853931) bank22070 +22071 POINT(33.40751900467881 -119.23381148024404) bank22071 +22072 POINT(34.07521699016456 -118.41187486573808) bank22072 +22073 POINT(34.43568876807605 -117.68198740805947) bank22073 +22074 POINT(34.43793712358079 -117.32865519758404) bank22074 +22075 POINT(33.344098868973376 -118.02755738656128) bank22075 +22076 POINT(33.28486330594101 -119.14060539256162) bank22076 +22077 POINT(34.63351617929061 -117.41921779177603) bank22077 +22078 POINT(33.94285993193264 -119.13282771065072) bank22078 +22079 POINT(34.676109500664246 -119.07543186126867) bank22079 +22080 POINT(34.87920001474998 -117.56085102093188) bank22080 +22081 POINT(33.866318069179485 -118.48567631684821) bank22081 +22082 POINT(33.69022949854254 -118.0198600188349) bank22082 +22083 POINT(33.246375432802985 -117.35603026525347) bank22083 +22084 POINT(34.0337330926386 -119.08081978050981) bank22084 +22085 POINT(35.0520084585369 -119.00028694975141) bank22085 +22086 POINT(33.128552295897364 -117.25707085501315) bank22086 +22087 POINT(34.727670716020555 -117.9374520554783) bank22087 +22088 POINT(33.32831764698283 -118.20437750698866) bank22088 +22089 POINT(35.040192658439516 -118.34851686502128) bank22089 +22090 POINT(34.698109097137376 -118.12401070145994) bank22090 +22091 POINT(33.841958275978186 -119.16424330105794) bank22091 +22092 POINT(34.67210263051249 -117.27733553599711) bank22092 +22093 POINT(34.77520880641917 -117.72406427029289) bank22093 +22094 POINT(33.61908353379184 -118.04350426546603) bank22094 +22095 POINT(33.65526664042506 -119.12467416442591) bank22095 +22096 POINT(33.117903086646976 -118.07832651025734) bank22096 +22097 POINT(33.685836342394374 -117.91304371528435) bank22097 +22098 POINT(34.88071723457816 -117.69161880118291) bank22098 +22099 POINT(34.420388486111705 -118.12526928095649) bank22099 +22100 POINT(33.85098084007533 -118.91443392552672) bank22100 +22101 POINT(33.13429682576387 -118.89276230788909) bank22101 +22102 POINT(33.47214091461105 -118.5103051203102) bank22102 +22103 POINT(34.55254402626951 -119.12575701387999) bank22103 +22104 POINT(34.120602740075824 -117.43562828261791) bank22104 +22105 POINT(33.22811811911862 -117.56659685752379) bank22105 +22106 POINT(33.951388331711435 -118.51190789398193) bank22106 +22107 POINT(34.76850116061053 -117.2884450132643) bank22107 +22108 POINT(35.036636579443424 -118.42856876303858) bank22108 +22109 POINT(33.94839727333398 -118.14294093231167) bank22109 +22110 POINT(34.06926964732797 -117.99439905266551) bank22110 +22111 POINT(33.34177078058986 -117.65220098484193) bank22111 +22112 POINT(34.60338512226301 -117.50201744870684) bank22112 +22113 POINT(33.990791895785506 -118.5693330739858) bank22113 +22114 POINT(33.576772552105595 -118.52314410458705) bank22114 +22115 POINT(34.891908172612126 -119.23547220488109) bank22115 +22116 POINT(35.01831671550367 -117.43420688964308) bank22116 +22117 POINT(34.720517664416576 -117.72577922694421) bank22117 +22118 POINT(33.47438897084189 -117.36907617315727) bank22118 +22119 POINT(33.780898123476646 -118.80479090765992) bank22119 +22120 POINT(33.621241727586884 -117.91305780070397) bank22120 +22121 POINT(33.4992872892681 -117.49147687119623) bank22121 +22122 POINT(34.1106244603242 -117.63535837443732) bank22122 +22123 POINT(34.35729147554955 -119.09731554550535) bank22123 +22124 POINT(33.67645153460816 -118.03734025254124) bank22124 +22125 POINT(33.16800426220935 -117.93158838728048) bank22125 +22126 POINT(34.43346599420477 -118.49931815405077) bank22126 +22127 POINT(33.070092137629466 -117.42000327024235) bank22127 +22128 POINT(33.88978088837641 -117.96039762232694) bank22128 +22129 POINT(35.046353110121636 -118.17104903531289) bank22129 +22130 POINT(34.869173086002135 -118.44035667039394) bank22130 +22131 POINT(34.34709699733968 -117.9378640489174) bank22131 +22132 POINT(34.87441460135175 -117.80456484693433) bank22132 +22133 POINT(34.85042586645789 -117.29503041296942) bank22133 +22134 POINT(33.76820301202623 -118.00739913875839) bank22134 +22135 POINT(34.30956888004891 -118.97255728499162) bank22135 +22136 POINT(33.612843075582425 -118.60408458678815) bank22136 +22137 POINT(34.32529986944543 -118.67065036211908) bank22137 +22138 POINT(34.922234820472255 -118.52153419711715) bank22138 +22139 POINT(34.62602237511742 -117.65966959071659) bank22139 +22140 POINT(34.54212935755018 -117.50565804422718) bank22140 +22141 POINT(34.12209290405438 -117.73060985399016) bank22141 +22142 POINT(33.12560072653009 -119.17275473513126) bank22142 +22143 POINT(33.351145136024044 -119.21065896520699) bank22143 +22144 POINT(33.09143166134163 -117.9957022013434) bank22144 +22145 POINT(34.880406037380915 -117.68158745141702) bank22145 +22146 POINT(33.93792815596324 -118.1055003152057) bank22146 +22147 POINT(33.21481588487906 -117.57052029090768) bank22147 +22148 POINT(34.75531813346763 -118.75518260669978) bank22148 +22149 POINT(34.726214958644825 -117.83909631546993) bank22149 +22150 POINT(34.367392127361875 -118.55630935392996) bank22150 +22151 POINT(33.30606463108204 -118.86692525985211) bank22151 +22152 POINT(33.614706795951506 -117.60765268635008) bank22152 +22153 POINT(34.23496594995633 -117.35793136064787) bank22153 +22154 POINT(33.6127435859929 -117.72351048300938) bank22154 +22155 POINT(34.298141169212386 -118.23530094434945) bank22155 +22156 POINT(33.52035677931353 -118.41901710478594) bank22156 +22157 POINT(34.23514654171281 -117.25159280477077) bank22157 +22158 POINT(34.69067734751964 -118.10112919577972) bank22158 +22159 POINT(33.470090352807034 -118.72416719871836) bank22159 +22160 POINT(34.849284985662614 -117.94233347972373) bank22160 +22161 POINT(33.08700077607672 -117.26902351492866) bank22161 +22162 POINT(33.19677557524219 -118.08120317813115) bank22162 +22163 POINT(33.336518595501154 -117.8716488436935) bank22163 +22164 POINT(33.102914110450925 -118.25562374288984) bank22164 +22165 POINT(34.15635764175164 -117.9100211054428) bank22165 +22166 POINT(34.9322598126357 -118.88644123033309) bank22166 +22167 POINT(34.706846564775844 -117.8857468684079) bank22167 +22168 POINT(34.90074556601516 -117.79070601383908) bank22168 +22169 POINT(33.795521406883566 -118.06498939413012) bank22169 +22170 POINT(33.10891642796301 -118.31742593391239) bank22170 +22171 POINT(33.658350444336016 -118.35711126463988) bank22171 +22172 POINT(33.76468812760028 -119.23704410843719) bank22172 +22173 POINT(34.01677489530852 -117.86419818292217) bank22173 +22174 POINT(33.55997186439132 -117.48779941490524) bank22174 +22175 POINT(33.51463119084609 -118.05599099364098) bank22175 +22176 POINT(33.177453867126275 -118.82155746759275) bank22176 +22177 POINT(33.1872347431635 -118.82401380870965) bank22177 +22178 POINT(33.40654308086826 -118.70507988347951) bank22178 +22179 POINT(33.803485725901346 -118.752893761319) bank22179 +22180 POINT(34.556476130304816 -119.0399966509458) bank22180 +22181 POINT(34.882965012701156 -117.68780111400926) bank22181 +22182 POINT(34.267202183992666 -117.30018780925259) bank22182 +22183 POINT(34.4578317966279 -118.16381447087296) bank22183 +22184 POINT(33.22612916941519 -117.67912519625817) bank22184 +22185 POINT(34.0097353941161 -118.87159762757713) bank22185 +22186 POINT(34.52166134541254 -118.92434253533013) bank22186 +22187 POINT(34.17302680501929 -117.33314411394537) bank22187 +22188 POINT(34.56050887837118 -118.81414649860535) bank22188 +22189 POINT(33.60954773626155 -118.54933196320671) bank22189 +22190 POINT(34.227069363671774 -117.5663563685542) bank22190 +22191 POINT(34.24514790417834 -117.92357967715377) bank22191 +22192 POINT(33.36782388331634 -118.79587146966188) bank22192 +22193 POINT(34.29794983970012 -118.19642734312342) bank22193 +22194 POINT(33.235246583095545 -117.99719350568289) bank22194 +22195 POINT(33.34040682389701 -117.70099987921404) bank22195 +22196 POINT(33.91471720192197 -117.33226255836871) bank22196 +22197 POINT(34.48037714781349 -117.69830759672512) bank22197 +22198 POINT(34.51971200838234 -117.30570303667855) bank22198 +22199 POINT(34.37867054948803 -117.86376673961136) bank22199 +22200 POINT(34.6404873311271 -117.65591059288785) bank22200 +22201 POINT(33.51360598078997 -118.30087518443383) bank22201 +22202 POINT(33.47728286873785 -117.58155262350353) bank22202 +22203 POINT(33.96914040945419 -118.68979799206895) bank22203 +22204 POINT(33.562508177599454 -119.13761415910858) bank22204 +22205 POINT(33.72626260192225 -118.71593665503015) bank22205 +22206 POINT(34.862137781383055 -118.01051631638819) bank22206 +22207 POINT(34.36900579146751 -118.63025709058675) bank22207 +22208 POINT(34.9836531650036 -118.34041086022988) bank22208 +22209 POINT(34.222371197415754 -119.11802882208364) bank22209 +22210 POINT(34.901068761566805 -118.32301105858836) bank22210 +22211 POINT(34.44844418812998 -118.2239880734912) bank22211 +22212 POINT(35.037612945119484 -118.78635875088169) bank22212 +22213 POINT(33.2054108457759 -118.38251098692648) bank22213 +22214 POINT(35.00086211921901 -119.22554428348258) bank22214 +22215 POINT(33.56603814855998 -119.1768171714288) bank22215 +22216 POINT(34.35267926570025 -117.62426187433513) bank22216 +22217 POINT(33.51301722035334 -117.55609606829589) bank22217 +22218 POINT(34.652378529870774 -118.89043988296554) bank22218 +22219 POINT(34.8665759146741 -119.09592420833117) bank22219 +22220 POINT(33.10187669044772 -118.0652742043857) bank22220 +22221 POINT(33.537824129004946 -117.43176801251367) bank22221 +22222 POINT(33.07487039203951 -118.64004676533295) bank22222 +22223 POINT(34.128011155293 -118.92245762666842) bank22223 +22224 POINT(34.51375147045066 -118.57959744439017) bank22224 +22225 POINT(33.29905123598557 -117.35255952486693) bank22225 +22226 POINT(33.94626974119896 -117.733090889818) bank22226 +22227 POINT(34.76065794964478 -117.44096256505662) bank22227 +22228 POINT(33.336537335388535 -117.47176772346049) bank22228 +22229 POINT(34.61358517113809 -118.01830003904627) bank22229 +22230 POINT(33.472349037548526 -118.97138783438758) bank22230 +22231 POINT(33.65952999506019 -118.46808769754472) bank22231 +22232 POINT(34.73107583336131 -117.56558567459221) bank22232 +22233 POINT(34.345136797672474 -117.46452017829577) bank22233 +22234 POINT(33.630532591453424 -117.6487096798689) bank22234 +22235 POINT(35.031797231218405 -118.39347053078895) bank22235 +22236 POINT(34.49805469738485 -119.01222240270593) bank22236 +22237 POINT(33.42898829507927 -119.16547960665147) bank22237 +22238 POINT(33.523411709823115 -118.31495267359495) bank22238 +22239 POINT(33.97655624980231 -117.28702251053113) bank22239 +22240 POINT(33.46901381418826 -117.72012827949685) bank22240 +22241 POINT(33.91549864550515 -118.38264993162743) bank22241 +22242 POINT(34.59695702661197 -117.91626438277295) bank22242 +22243 POINT(33.96480741988006 -119.1193998243983) bank22243 +22244 POINT(34.43061172812017 -118.18025132435632) bank22244 +22245 POINT(34.24852273193679 -119.2384044183253) bank22245 +22246 POINT(33.26861039797387 -117.67828024122814) bank22246 +22247 POINT(33.64644166105698 -119.19602676717241) bank22247 +22248 POINT(34.383175137886184 -117.74840191201913) bank22248 +22249 POINT(33.376812627589715 -119.2101146003363) bank22249 +22250 POINT(34.30205351451203 -118.23499062493049) bank22250 +22251 POINT(33.94687641426875 -117.5183667674685) bank22251 +22252 POINT(34.54226933199885 -117.56463599001368) bank22252 +22253 POINT(35.01368474066016 -118.34610704215106) bank22253 +22254 POINT(33.758509292386215 -118.54352421080165) bank22254 +22255 POINT(34.416622755256626 -117.9244473511297) bank22255 +22256 POINT(34.24531780365115 -118.75867219393018) bank22256 +22257 POINT(33.97613753305373 -117.54403340576079) bank22257 +22258 POINT(34.95880727822574 -117.5449841201958) bank22258 +22259 POINT(33.52965022070437 -119.16365827937823) bank22259 +22260 POINT(33.33980755336691 -117.48229583362591) bank22260 +22261 POINT(33.472433396192514 -117.98188206079524) bank22261 +22262 POINT(33.56598507238971 -118.03717044938179) bank22262 +22263 POINT(34.987099821241856 -117.60346128985373) bank22263 +22264 POINT(34.6963376628765 -118.61199563845294) bank22264 +22265 POINT(34.085560514151695 -118.07451518426241) bank22265 +22266 POINT(34.49392492640433 -119.02306532491232) bank22266 +22267 POINT(34.29608977745193 -118.45265857946454) bank22267 +22268 POINT(34.96387709426872 -118.09261280544513) bank22268 +22269 POINT(33.975099361296934 -117.43038667091366) bank22269 +22270 POINT(34.12067567544135 -118.90795183196069) bank22270 +22271 POINT(34.41139334359694 -118.86891996832071) bank22271 +22272 POINT(34.55121355974554 -118.62112453188597) bank22272 +22273 POINT(34.555052084071924 -118.87847719212071) bank22273 +22274 POINT(35.04523953025896 -117.43552250794876) bank22274 +22275 POINT(34.17511621836776 -117.47531189719824) bank22275 +22276 POINT(33.63573880665142 -118.50134687151237) bank22276 +22277 POINT(33.682771984420405 -118.22497065222912) bank22277 +22278 POINT(33.11954401612553 -118.14619862577638) bank22278 +22279 POINT(33.41302554795703 -118.86130981842564) bank22279 +22280 POINT(34.272236925194534 -118.5533364887131) bank22280 +22281 POINT(34.1104517503305 -118.6559364469937) bank22281 +22282 POINT(33.33141607898933 -117.80990372232833) bank22282 +22283 POINT(33.14905851045381 -118.21389890423609) bank22283 +22284 POINT(33.78630304202506 -117.26752192821792) bank22284 +22285 POINT(33.25119168152843 -117.78837928626311) bank22285 +22286 POINT(34.321714018941144 -119.1742095016132) bank22286 +22287 POINT(33.96990847215689 -117.98581743749004) bank22287 +22288 POINT(34.42645434818207 -118.67192832329046) bank22288 +22289 POINT(33.31571136463837 -118.51087490853912) bank22289 +22290 POINT(33.453000105088954 -118.47894852403893) bank22290 +22291 POINT(33.49484954177347 -117.27934212998873) bank22291 +22292 POINT(34.78521478265356 -118.61817715869098) bank22292 +22293 POINT(34.69737960639885 -118.50427067256196) bank22293 +22294 POINT(33.29075983922113 -119.19506696235116) bank22294 +22295 POINT(33.46184664717243 -117.68644012357628) bank22295 +22296 POINT(34.59083172406588 -119.08615745106407) bank22296 +22297 POINT(34.190505068082864 -118.18736392897573) bank22297 +22298 POINT(34.5164873679987 -119.12328975975024) bank22298 +22299 POINT(34.82406010340564 -118.48609117246555) bank22299 +22300 POINT(33.39331088655585 -119.15339804425051) bank22300 +22301 POINT(33.84753403595216 -118.40188073544051) bank22301 +22302 POINT(33.15442368592695 -117.92258250533118) bank22302 +22303 POINT(34.76342763510844 -117.64237073182356) bank22303 +22304 POINT(34.15830638900509 -119.01112449760411) bank22304 +22305 POINT(33.713675756399454 -117.43523155081463) bank22305 +22306 POINT(34.16965874159606 -118.30287593020343) bank22306 +22307 POINT(33.11390917089567 -117.73397213052243) bank22307 +22308 POINT(33.33787508520582 -118.91086989469201) bank22308 +22309 POINT(33.693776758213055 -117.67387858205153) bank22309 +22310 POINT(33.43983999878096 -118.74738512845747) bank22310 +22311 POINT(34.238713701513426 -118.94931891102135) bank22311 +22312 POINT(33.110429173529226 -117.99352126387824) bank22312 +22313 POINT(34.69517529390238 -117.49709721005041) bank22313 +22314 POINT(33.711316246643015 -117.70663690070373) bank22314 +22315 POINT(34.02977329466294 -117.6968150836659) bank22315 +22316 POINT(34.86091832222273 -118.50959507942933) bank22316 +22317 POINT(34.59158761287711 -118.14696423513334) bank22317 +22318 POINT(33.060627733489135 -117.31785741338487) bank22318 +22319 POINT(33.09168290621789 -118.20872087408279) bank22319 +22320 POINT(33.655380783401576 -117.39731425921049) bank22320 +22321 POINT(34.002320397921366 -118.42529577296688) bank22321 +22322 POINT(34.17063013925898 -118.3552693916187) bank22322 +22323 POINT(34.624657118948505 -117.93355074475177) bank22323 +22324 POINT(33.58580295262137 -119.00131534905852) bank22324 +22325 POINT(35.020515164976445 -117.98417429099541) bank22325 +22326 POINT(33.16824966120872 -118.713760905362) bank22326 +22327 POINT(34.448904560842735 -117.24936634934488) bank22327 +22328 POINT(33.837554871781826 -118.22182905633306) bank22328 +22329 POINT(33.26257676017548 -119.01252406492706) bank22329 +22330 POINT(34.574712824986214 -118.60395894973631) bank22330 +22331 POINT(34.363063623097084 -118.55738473320282) bank22331 +22332 POINT(34.77265415864392 -118.68940839772594) bank22332 +22333 POINT(33.91258265518367 -117.92536644643911) bank22333 +22334 POINT(33.67885278516571 -118.84147903758597) bank22334 +22335 POINT(33.12247043213191 -117.40848962219448) bank22335 +22336 POINT(33.391437119931965 -118.41064877883115) bank22336 +22337 POINT(34.22944535619387 -118.48973469226853) bank22337 +22338 POINT(33.32927544615189 -118.41491224388497) bank22338 +22339 POINT(34.287155505546316 -117.75044144175097) bank22339 +22340 POINT(34.439296457702135 -118.03630160144442) bank22340 +22341 POINT(33.92036213208123 -119.07028390117812) bank22341 +22342 POINT(33.89050341485516 -119.10622580925049) bank22342 +22343 POINT(33.550809277268804 -118.23969455260571) bank22343 +22344 POINT(34.97853040456339 -118.74498759283615) bank22344 +22345 POINT(34.79496097667848 -117.27694216470039) bank22345 +22346 POINT(33.747925924899995 -117.4725900837918) bank22346 +22347 POINT(34.30951457947993 -118.16527205139873) bank22347 +22348 POINT(34.0873199673274 -118.15920900578163) bank22348 +22349 POINT(34.622076083179806 -117.73476460509427) bank22349 +22350 POINT(33.321420984155715 -118.08172189105889) bank22350 +22351 POINT(34.49457455070613 -118.57026118776179) bank22351 +22352 POINT(33.73131989465611 -117.97999400804406) bank22352 +22353 POINT(34.78007243828474 -118.16814918609788) bank22353 +22354 POINT(33.9060167622326 -118.69674071509317) bank22354 +22355 POINT(33.190764157709694 -117.79854462345556) bank22355 +22356 POINT(33.75336198795308 -119.16885565810155) bank22356 +22357 POINT(33.110791146961695 -117.96958538136072) bank22357 +22358 POINT(34.796822914676994 -117.46874947809837) bank22358 +22359 POINT(33.49629385114883 -117.56412136993367) bank22359 +22360 POINT(34.581271356751984 -117.58217128866994) bank22360 +22361 POINT(33.65368250567509 -118.93040318628199) bank22361 +22362 POINT(33.37234805330714 -118.76563128460563) bank22362 +22363 POINT(34.47030577000497 -119.13009161454109) bank22363 +22364 POINT(33.2468987559503 -118.11458173616462) bank22364 +22365 POINT(34.59304577827261 -118.60202306986092) bank22365 +22366 POINT(33.35409834435596 -117.40831359168229) bank22366 +22367 POINT(33.82821722169044 -118.36284303602997) bank22367 +22368 POINT(34.3920399303122 -118.54842754685897) bank22368 +22369 POINT(33.483181559270164 -118.01602107923806) bank22369 +22370 POINT(34.01892917308216 -118.80166782183063) bank22370 +22371 POINT(33.509879871798084 -118.75759521155112) bank22371 +22372 POINT(34.63569272216097 -117.70240320283469) bank22372 +22373 POINT(33.83663672343863 -117.66698101553327) bank22373 +22374 POINT(33.10901241560556 -119.050231784336) bank22374 +22375 POINT(34.07049790778499 -117.81913034813773) bank22375 +22376 POINT(33.380256405173405 -118.43860096277241) bank22376 +22377 POINT(33.885096188183184 -118.56201664674988) bank22377 +22378 POINT(33.47853073432663 -117.31341022140934) bank22378 +22379 POINT(34.879740867171115 -117.58945802139088) bank22379 +22380 POINT(34.02895828921869 -117.84719203923694) bank22380 +22381 POINT(33.26622870341471 -117.88938372735912) bank22381 +22382 POINT(34.81285363596937 -118.52881458029314) bank22382 +22383 POINT(34.11559252863472 -118.42614495850083) bank22383 +22384 POINT(33.51360343091876 -118.35825649119316) bank22384 +22385 POINT(33.37610950369254 -119.07476303862653) bank22385 +22386 POINT(33.862292829186394 -118.32153835757578) bank22386 +22387 POINT(34.524514853955814 -117.32247246145884) bank22387 +22388 POINT(34.988405717129076 -118.07353106198151) bank22388 +22389 POINT(33.85926053874774 -118.66144166901337) bank22389 +22390 POINT(33.57139438654457 -117.25217767860562) bank22390 +22391 POINT(34.82705893050628 -119.20782663836295) bank22391 +22392 POINT(33.27554962846135 -119.21686662440037) bank22392 +22393 POINT(34.01867748407898 -117.43711721739668) bank22393 +22394 POINT(34.89753168524365 -117.30954534554172) bank22394 +22395 POINT(33.21410701295554 -119.14915762348694) bank22395 +22396 POINT(33.99623374289656 -118.90223499541229) bank22396 +22397 POINT(34.42649261327601 -118.21319741308564) bank22397 +22398 POINT(33.61577541035719 -118.75464061799187) bank22398 +22399 POINT(34.17716685886223 -118.60472592345945) bank22399 +22400 POINT(33.811736200086784 -117.68313134025487) bank22400 +22401 POINT(34.271278500830824 -118.55501765805964) bank22401 +22402 POINT(33.551602737700776 -118.34385026243828) bank22402 +22403 POINT(34.20233746013644 -118.52410892931658) bank22403 +22404 POINT(34.08312852017066 -118.37539012601077) bank22404 +22405 POINT(34.97433755986642 -117.39826070292634) bank22405 +22406 POINT(34.049414504757 -118.82284397601485) bank22406 +22407 POINT(33.3992796987092 -118.82583556082362) bank22407 +22408 POINT(34.14993293380562 -118.75754103211911) bank22408 +22409 POINT(34.52562455979568 -117.5244443387504) bank22409 +22410 POINT(34.18901893398055 -118.47964932379809) bank22410 +22411 POINT(34.84613678085742 -117.85207446317395) bank22411 +22412 POINT(33.22421434882715 -117.93403803083737) bank22412 +22413 POINT(34.758773996214515 -119.00970990797762) bank22413 +22414 POINT(34.03494764533455 -118.06561577577142) bank22414 +22415 POINT(34.12570879450809 -117.38187272063828) bank22415 +22416 POINT(33.49963076558623 -118.34057143124309) bank22416 +22417 POINT(33.09933132100953 -117.38192560702056) bank22417 +22418 POINT(34.341208353366525 -117.84971407985351) bank22418 +22419 POINT(33.179397092204255 -118.98392330203302) bank22419 +22420 POINT(33.14678544030333 -117.39386688255088) bank22420 +22421 POINT(33.59481947366217 -117.78164330547867) bank22421 +22422 POINT(33.22052538570075 -119.02612450976945) bank22422 +22423 POINT(33.96614107377727 -118.91154682693688) bank22423 +22424 POINT(34.372941708802 -117.92483446569815) bank22424 +22425 POINT(34.17509755464806 -118.21448265330717) bank22425 +22426 POINT(33.957654373173646 -118.3386214490187) bank22426 +22427 POINT(33.64874181024495 -118.79559443515551) bank22427 +22428 POINT(34.9546933898509 -117.45370124564879) bank22428 +22429 POINT(33.127019300351115 -118.37811006314212) bank22429 +22430 POINT(33.911685051829885 -118.38870499863391) bank22430 +22431 POINT(34.49844665068685 -118.37249320039355) bank22431 +22432 POINT(33.53207202057039 -119.09750501852893) bank22432 +22433 POINT(34.11757827873783 -117.29745861450463) bank22433 +22434 POINT(34.33557333842717 -118.73311767310175) bank22434 +22435 POINT(33.51354969386796 -117.41843683806017) bank22435 +22436 POINT(33.867797345649436 -118.8571632403162) bank22436 +22437 POINT(33.631844756583526 -117.30614166161766) bank22437 +22438 POINT(34.38543488485759 -118.44963642455984) bank22438 +22439 POINT(33.689327430402784 -117.30662304314617) bank22439 +22440 POINT(34.088651024239375 -118.83594684834127) bank22440 +22441 POINT(34.259444595792594 -118.41125985112814) bank22441 +22442 POINT(33.8784619387572 -118.27498270538975) bank22442 +22443 POINT(33.26728806573914 -118.7688537696734) bank22443 +22444 POINT(34.5666961181534 -117.99724162855144) bank22444 +22445 POINT(34.71342445066687 -117.86236813063522) bank22445 +22446 POINT(33.13231161612421 -118.10978332046015) bank22446 +22447 POINT(34.65888470302948 -117.67309731975121) bank22447 +22448 POINT(35.0073966689011 -117.75801235269186) bank22448 +22449 POINT(34.91203334867681 -117.59421269954542) bank22449 +22450 POINT(33.200309823142 -117.55041168431252) bank22450 +22451 POINT(33.16153954211535 -117.84595638568824) bank22451 +22452 POINT(33.57381808494753 -119.05080101964735) bank22452 +22453 POINT(33.38144154393689 -119.21495699686082) bank22453 +22454 POINT(33.42271770986431 -119.19815085910491) bank22454 +22455 POINT(33.66693218888483 -118.24946081579095) bank22455 +22456 POINT(34.6048934341792 -118.20106264054867) bank22456 +22457 POINT(34.42620316499807 -118.58329456828922) bank22457 +22458 POINT(34.16206381856689 -118.31491144942285) bank22458 +22459 POINT(34.217248829434126 -117.75627247417144) bank22459 +22460 POINT(33.2500808566824 -117.6638763514083) bank22460 +22461 POINT(33.93914925602687 -118.62499547681503) bank22461 +22462 POINT(33.69225757012154 -118.9161405820641) bank22462 +22463 POINT(34.910980716749215 -118.19707146012844) bank22463 +22464 POINT(33.095969785588196 -117.49805444364195) bank22464 +22465 POINT(34.881963480146474 -118.22083848376663) bank22465 +22466 POINT(34.82513082602212 -117.92409797991878) bank22466 +22467 POINT(33.68665599742258 -118.95737221281223) bank22467 +22468 POINT(33.972038796303465 -118.26393275592598) bank22468 +22469 POINT(34.313548512838324 -117.6559870398116) bank22469 +22470 POINT(33.06116646770875 -117.38144048476829) bank22470 +22471 POINT(33.296793795706506 -117.27723388355867) bank22471 +22472 POINT(33.32005016051023 -118.52142408085483) bank22472 +22473 POINT(33.65155872732609 -117.29819353331834) bank22473 +22474 POINT(33.45377130746848 -118.47737251371824) bank22474 +22475 POINT(33.11050459299964 -118.76250954235883) bank22475 +22476 POINT(34.88228476750565 -119.1499940804966) bank22476 +22477 POINT(34.06589866476484 -117.56485265367618) bank22477 +22478 POINT(34.938628930606995 -117.32811405010364) bank22478 +22479 POINT(34.595759131007775 -117.53529618007688) bank22479 +22480 POINT(34.58279003092239 -119.1585013098234) bank22480 +22481 POINT(34.15201302325171 -118.29355114572613) bank22481 +22482 POINT(34.67045884152418 -117.96621774298634) bank22482 +22483 POINT(33.93044443161882 -119.07531346447486) bank22483 +22484 POINT(34.59126784783337 -117.82492542062897) bank22484 +22485 POINT(33.97663123129723 -119.00692307470014) bank22485 +22486 POINT(33.27687070026934 -117.67788501501822) bank22486 +22487 POINT(33.663411835179375 -117.99576524728484) bank22487 +22488 POINT(33.52735686887493 -119.18459183783642) bank22488 +22489 POINT(33.68418065488375 -117.59175428250131) bank22489 +22490 POINT(34.06891371680617 -118.18150892226993) bank22490 +22491 POINT(33.29944137438651 -118.99279163977918) bank22491 +22492 POINT(34.15603833071894 -117.24894164748991) bank22492 +22493 POINT(34.87591739762321 -118.39988449530208) bank22493 +22494 POINT(34.407923148297435 -118.14157205582772) bank22494 +22495 POINT(33.22213513113678 -117.56042460000488) bank22495 +22496 POINT(34.03107414172682 -118.60112521420785) bank22496 +22497 POINT(34.24921634554856 -118.18049835338614) bank22497 +22498 POINT(34.767621126877174 -118.32630140214887) bank22498 +22499 POINT(33.367184211446606 -118.4241471063391) bank22499 +22500 POINT(34.92534451154188 -118.05143141401304) bank22500 +22501 POINT(34.63672649916544 -117.65665371260758) bank22501 +22502 POINT(34.97217716888553 -117.80268911760497) bank22502 +22503 POINT(34.155713732097475 -117.86973569154263) bank22503 +22504 POINT(33.49845252473783 -118.5579468729901) bank22504 +22505 POINT(34.81344757120739 -117.67545369114521) bank22505 +22506 POINT(33.10530772150938 -119.1575212034758) bank22506 +22507 POINT(34.561769364650274 -118.23336581467268) bank22507 +22508 POINT(33.13306190812855 -117.42280538942138) bank22508 +22509 POINT(33.30074871732742 -117.39197939813222) bank22509 +22510 POINT(33.28989235445014 -118.5496056776874) bank22510 +22511 POINT(33.546815423327914 -119.13525380834999) bank22511 +22512 POINT(33.56317968111156 -119.08005402168895) bank22512 +22513 POINT(34.8537844254723 -118.70441752127165) bank22513 +22514 POINT(34.90936732884157 -118.04984448865837) bank22514 +22515 POINT(33.05894905657164 -117.49466267670397) bank22515 +22516 POINT(34.04042153357254 -119.1169854045075) bank22516 +22517 POINT(33.857219954041824 -118.02525529709303) bank22517 +22518 POINT(35.04094853850859 -118.07814795193177) bank22518 +22519 POINT(33.19417260011623 -119.19268855461387) bank22519 +22520 POINT(33.67616668217192 -117.82207215452765) bank22520 +22521 POINT(34.077228001810305 -119.08827047528061) bank22521 +22522 POINT(33.30378388907093 -117.74535013707924) bank22522 +22523 POINT(33.944954093679456 -118.99953655514419) bank22523 +22524 POINT(33.909056147696546 -117.2970473812826) bank22524 +22525 POINT(33.780738178786166 -117.43352639786919) bank22525 +22526 POINT(34.31492997279977 -118.70025183499818) bank22526 +22527 POINT(33.10601319108043 -118.90330128284234) bank22527 +22528 POINT(34.52284555149314 -118.5341959393949) bank22528 +22529 POINT(33.54002240821581 -118.66419057113427) bank22529 +22530 POINT(33.64863648208747 -118.19894584457116) bank22530 +22531 POINT(33.63251879678871 -118.5500664913281) bank22531 +22532 POINT(33.098355909667596 -118.92142626373251) bank22532 +22533 POINT(33.7366636924503 -117.30035736235398) bank22533 +22534 POINT(34.65424940083641 -118.91429023312831) bank22534 +22535 POINT(33.64673614289 -118.62544527144203) bank22535 +22536 POINT(33.80250821792892 -118.63998129002746) bank22536 +22537 POINT(33.66032594622774 -119.1533365265738) bank22537 +22538 POINT(35.01573613694883 -119.12175704291052) bank22538 +22539 POINT(33.35032960286007 -118.55491891285119) bank22539 +22540 POINT(34.84655936483933 -118.2840138047267) bank22540 +22541 POINT(33.75019611141213 -118.36579916259497) bank22541 +22542 POINT(33.76394733186732 -118.93196154726976) bank22542 +22543 POINT(34.472146897359835 -118.11233250352433) bank22543 +22544 POINT(33.87108309042257 -118.01305710792163) bank22544 +22545 POINT(34.65375809211197 -118.77282680709779) bank22545 +22546 POINT(34.447406444642475 -117.75922330917646) bank22546 +22547 POINT(34.14125509545394 -118.22457446318408) bank22547 +22548 POINT(33.662597732431756 -118.37260939111879) bank22548 +22549 POINT(33.75050513217457 -117.67768608482245) bank22549 +22550 POINT(33.85601822434081 -118.98403313088183) bank22550 +22551 POINT(33.5441053216465 -117.53187239600308) bank22551 +22552 POINT(34.55342928001387 -117.62337261077337) bank22552 +22553 POINT(33.40259986805988 -119.07039913629319) bank22553 +22554 POINT(33.98334789675368 -118.1325477289544) bank22554 +22555 POINT(35.00047646732342 -117.49571672896975) bank22555 +22556 POINT(34.08747179274216 -118.29333412016045) bank22556 +22557 POINT(34.190515666930786 -118.74077255360163) bank22557 +22558 POINT(33.755416375199765 -117.58631778843322) bank22558 +22559 POINT(34.46042593349982 -117.87963345052871) bank22559 +22560 POINT(34.330148441597245 -119.22319115072038) bank22560 +22561 POINT(34.23663237263277 -117.62605723762569) bank22561 +22562 POINT(33.34808123545559 -117.69090534475198) bank22562 +22563 POINT(33.59565600379499 -118.19947189298071) bank22563 +22564 POINT(35.01882513417852 -117.7399756398536) bank22564 +22565 POINT(33.96025690973246 -118.84594239844893) bank22565 +22566 POINT(34.02639384233959 -118.40035421941728) bank22566 +22567 POINT(34.503737226405626 -118.21844001916256) bank22567 +22568 POINT(34.920325428412106 -119.08319559044104) bank22568 +22569 POINT(34.768474266876964 -118.35984467652216) bank22569 +22570 POINT(34.65983816258377 -118.24300415680095) bank22570 +22571 POINT(34.056454992211215 -118.96661790997041) bank22571 +22572 POINT(33.54657590696193 -118.78272791328291) bank22572 +22573 POINT(33.7504243137394 -117.4770096149798) bank22573 +22574 POINT(34.735261967264094 -118.61486617101136) bank22574 +22575 POINT(33.60598361523276 -117.96629926689955) bank22575 +22576 POINT(33.87761978010508 -118.1592787515767) bank22576 +22577 POINT(35.00963584237804 -119.07626902777434) bank22577 +22578 POINT(34.495107892275435 -117.27984641727194) bank22578 +22579 POINT(33.22666780582913 -118.40434730929378) bank22579 +22580 POINT(33.422416344752705 -118.92659203509626) bank22580 +22581 POINT(34.261288170227836 -117.55123034268327) bank22581 +22582 POINT(34.11757787667497 -118.70633236552034) bank22582 +22583 POINT(33.50931426461181 -117.6371247847252) bank22583 +22584 POINT(34.84151963007896 -118.633170592436) bank22584 +22585 POINT(33.254393522207074 -118.71862150031393) bank22585 +22586 POINT(34.204314385651315 -118.83817778512625) bank22586 +22587 POINT(33.83491084560464 -118.28322060259134) bank22587 +22588 POINT(34.82059314077064 -118.97822966535678) bank22588 +22589 POINT(33.074015392220474 -117.79537829216555) bank22589 +22590 POINT(34.14164022941181 -118.04924071783589) bank22590 +22591 POINT(34.67890726982275 -119.0346963387263) bank22591 +22592 POINT(33.676276010849314 -117.67821928104443) bank22592 +22593 POINT(33.37393866116204 -117.90830344170479) bank22593 +22594 POINT(33.397879099695274 -119.01011246366483) bank22594 +22595 POINT(35.036269674729866 -117.46171686323325) bank22595 +22596 POINT(34.24597200459405 -117.71433330719282) bank22596 +22597 POINT(34.32751934545534 -117.7481866297023) bank22597 +22598 POINT(34.09957622771423 -118.54975852812252) bank22598 +22599 POINT(33.6841478711715 -117.55891524167518) bank22599 +22600 POINT(34.13618451102612 -118.19338468302087) bank22600 +22601 POINT(35.016877416343995 -119.05005155984139) bank22601 +22602 POINT(33.27661249612178 -117.8258152792381) bank22602 +22603 POINT(33.173959584154154 -118.46398792542593) bank22603 +22604 POINT(33.98379541051566 -117.30021148235659) bank22604 +22605 POINT(34.8643128607372 -117.8639745157948) bank22605 +22606 POINT(34.760971506728325 -118.59813496165121) bank22606 +22607 POINT(34.99310389141507 -118.1835066444781) bank22607 +22608 POINT(34.64063406712023 -118.0142547321591) bank22608 +22609 POINT(35.02397358356491 -117.37987005445267) bank22609 +22610 POINT(34.14231820593447 -118.3435717764944) bank22610 +22611 POINT(34.00768371142854 -119.23020370893057) bank22611 +22612 POINT(34.12427905271702 -117.47135672435631) bank22612 +22613 POINT(34.680344495785185 -117.58384570337883) bank22613 +22614 POINT(33.49590978351441 -119.04874582627848) bank22614 +22615 POINT(34.67470004085734 -117.5883969883049) bank22615 +22616 POINT(33.51994020881818 -117.33038555467346) bank22616 +22617 POINT(34.49195798592579 -118.02410295861166) bank22617 +22618 POINT(33.4205142820957 -117.95000641236938) bank22618 +22619 POINT(34.0629702763861 -117.59921121714599) bank22619 +22620 POINT(34.17936107566408 -117.76282523354473) bank22620 +22621 POINT(34.94361955683447 -118.67795949834185) bank22621 +22622 POINT(34.03377264577045 -118.57063991290977) bank22622 +22623 POINT(33.9950970248734 -117.36391438992192) bank22623 +22624 POINT(33.97399792882299 -118.17979996534282) bank22624 +22625 POINT(34.56651645595792 -118.72491086171637) bank22625 +22626 POINT(33.097628670201665 -117.6918182294845) bank22626 +22627 POINT(35.03263358411386 -118.38039400947183) bank22627 +22628 POINT(34.842432178626325 -118.20101412201933) bank22628 +22629 POINT(33.96384825420842 -117.32354243793704) bank22629 +22630 POINT(33.86758821941544 -117.32752703656337) bank22630 +22631 POINT(34.34520588507883 -118.62336447691538) bank22631 +22632 POINT(34.916946590520155 -117.34653039964219) bank22632 +22633 POINT(34.25558079532984 -118.52524476675354) bank22633 +22634 POINT(34.808178191178726 -118.54898368965615) bank22634 +22635 POINT(33.11885695709842 -117.35434697637845) bank22635 +22636 POINT(33.32852908457855 -117.8583893230218) bank22636 +22637 POINT(34.33878489933919 -118.92004373795976) bank22637 +22638 POINT(33.189460569668285 -118.91380671625313) bank22638 +22639 POINT(34.60622407805382 -117.32021465513078) bank22639 +22640 POINT(33.65171285003977 -118.2510202210061) bank22640 +22641 POINT(33.719243383407814 -117.52147895583863) bank22641 +22642 POINT(33.336542203984436 -118.1844488666443) bank22642 +22643 POINT(34.00744811078343 -119.03709645838816) bank22643 +22644 POINT(33.26933643453942 -118.23450478513871) bank22644 +22645 POINT(33.214757863573084 -117.5367579176536) bank22645 +22646 POINT(34.91610048901075 -118.35438784276072) bank22646 +22647 POINT(33.981721250068716 -117.55702496145214) bank22647 +22648 POINT(33.3675568125098 -118.16682563375313) bank22648 +22649 POINT(34.53030224840862 -117.75139399179666) bank22649 +22650 POINT(34.97047807366027 -117.55788005541196) bank22650 +22651 POINT(33.41579668545153 -119.05221701123787) bank22651 +22652 POINT(33.98877034623599 -118.37839869049513) bank22652 +22653 POINT(34.04771927412124 -117.9675798154004) bank22653 +22654 POINT(34.39848995812357 -118.89464255633234) bank22654 +22655 POINT(34.08926933551399 -118.1535508073582) bank22655 +22656 POINT(34.48089932212274 -117.4565865640394) bank22656 +22657 POINT(33.64879302617817 -117.86894457503735) bank22657 +22658 POINT(33.2435979868042 -118.01993853667663) bank22658 +22659 POINT(34.62856385015262 -118.24854137806776) bank22659 +22660 POINT(34.28480945034568 -117.94121434803773) bank22660 +22661 POINT(34.55311726622224 -118.99394531705421) bank22661 +22662 POINT(33.71757727497434 -118.93484992833731) bank22662 +22663 POINT(33.827990708252734 -118.4806439241519) bank22663 +22664 POINT(33.38507075644505 -118.75655154072703) bank22664 +22665 POINT(34.24412031077468 -119.1770712770946) bank22665 +22666 POINT(33.85267566012324 -118.78678358775767) bank22666 +22667 POINT(34.71619014968558 -117.34788511283767) bank22667 +22668 POINT(34.03164842753011 -118.7322144174487) bank22668 +22669 POINT(33.605606308484525 -117.37887974168903) bank22669 +22670 POINT(35.01369030941671 -118.6640808789758) bank22670 +22671 POINT(34.79752377690202 -119.03055624610057) bank22671 +22672 POINT(33.76083433855348 -117.48462210166035) bank22672 +22673 POINT(34.77292504828421 -117.37587940301785) bank22673 +22674 POINT(33.44075324695229 -117.41466218185617) bank22674 +22675 POINT(34.46774279682486 -118.8154456493076) bank22675 +22676 POINT(33.94112918159144 -117.5698178541034) bank22676 +22677 POINT(33.575779036007894 -119.04124397156329) bank22677 +22678 POINT(34.38202572342343 -118.03796740380112) bank22678 +22679 POINT(33.96609620922825 -118.94305253210771) bank22679 +22680 POINT(33.618314293519674 -117.80459070125067) bank22680 +22681 POINT(34.92788111705005 -118.79049964066814) bank22681 +22682 POINT(33.13144190851473 -118.74488876314315) bank22682 +22683 POINT(33.743973640794366 -118.12212817738857) bank22683 +22684 POINT(34.262504843776874 -117.27861553437592) bank22684 +22685 POINT(34.92061393327958 -117.66351600781536) bank22685 +22686 POINT(34.0599717594513 -118.0597795154062) bank22686 +22687 POINT(35.032041051562494 -117.28332971237847) bank22687 +22688 POINT(34.46958909380178 -118.19984473941608) bank22688 +22689 POINT(33.773487877643696 -118.2547133745072) bank22689 +22690 POINT(35.00201161946199 -119.13343021678357) bank22690 +22691 POINT(33.934323724416345 -117.5563168369248) bank22691 +22692 POINT(34.63275791492711 -118.44119672323022) bank22692 +22693 POINT(34.59357617453585 -118.66595085499938) bank22693 +22694 POINT(34.53918626921998 -117.3338881566133) bank22694 +22695 POINT(33.69119552371776 -118.96770858274715) bank22695 +22696 POINT(34.09083541817373 -118.09187537828083) bank22696 +22697 POINT(33.85942414824711 -117.93742854837515) bank22697 +22698 POINT(34.86094610574797 -117.29744159804474) bank22698 +22699 POINT(34.46849157346642 -119.07265144509041) bank22699 +22700 POINT(33.4810302525223 -118.55515802444368) bank22700 +22701 POINT(34.28045221333296 -118.45101468248279) bank22701 +22702 POINT(33.064985319970965 -117.73048330954806) bank22702 +22703 POINT(34.531999738288555 -117.7315629737115) bank22703 +22704 POINT(34.96297435157236 -118.66801062618757) bank22704 +22705 POINT(33.63613601500218 -118.84110373760504) bank22705 +22706 POINT(33.93811312662811 -118.36429231861946) bank22706 +22707 POINT(33.53082463737518 -117.4410933676841) bank22707 +22708 POINT(33.23524993084006 -118.7922916745985) bank22708 +22709 POINT(33.790705333371776 -118.48722589239391) bank22709 +22710 POINT(34.65776047496302 -118.60515261249813) bank22710 +22711 POINT(33.87276120640333 -119.14550799562485) bank22711 +22712 POINT(34.17919927904786 -119.11062630634954) bank22712 +22713 POINT(33.6408849622835 -117.77142257848197) bank22713 +22714 POINT(35.041058482239606 -118.70449942731557) bank22714 +22715 POINT(34.523183159364976 -118.20521124255686) bank22715 +22716 POINT(33.73392198511623 -119.09242003124882) bank22716 +22717 POINT(34.328557832740664 -118.82052205351815) bank22717 +22718 POINT(34.190132896592516 -117.83907230702172) bank22718 +22719 POINT(34.55419660823581 -118.76425088893839) bank22719 +22720 POINT(34.656160605066695 -118.48480612517142) bank22720 +22721 POINT(33.925449318775804 -117.49333270401178) bank22721 +22722 POINT(33.551652374551445 -118.16874932461701) bank22722 +22723 POINT(33.38425155077785 -118.10704849696873) bank22723 +22724 POINT(33.34615181802233 -119.08486149461918) bank22724 +22725 POINT(35.002543393198685 -117.4431679253305) bank22725 +22726 POINT(34.91789711666796 -118.12890899014182) bank22726 +22727 POINT(34.37790985429192 -119.12291265033151) bank22727 +22728 POINT(33.95319145725165 -118.5943955809553) bank22728 +22729 POINT(34.89212978504836 -117.95765641644566) bank22729 +22730 POINT(34.94920071275714 -117.39691078688159) bank22730 +22731 POINT(33.636434800557154 -118.23515305876433) bank22731 +22732 POINT(33.83444165049274 -117.27600836651058) bank22732 +22733 POINT(33.09425484141642 -118.02215814156696) bank22733 +22734 POINT(33.622864463752656 -118.65137122970323) bank22734 +22735 POINT(33.28334879668256 -118.84127508684881) bank22735 +22736 POINT(33.91780584390992 -118.52804516132325) bank22736 +22737 POINT(34.527972624712326 -118.83147351816166) bank22737 +22738 POINT(33.278431634946486 -118.21666872323733) bank22738 +22739 POINT(33.177820530772536 -118.50378726355224) bank22739 +22740 POINT(34.31663991496572 -118.98267978776722) bank22740 +22741 POINT(33.12996547769838 -118.27447687290729) bank22741 +22742 POINT(34.56998411790715 -118.64337053805167) bank22742 +22743 POINT(34.47494806380705 -117.44811192125623) bank22743 +22744 POINT(33.77071879328099 -118.74510071666391) bank22744 +22745 POINT(33.97638309797785 -118.10045160683286) bank22745 +22746 POINT(34.6723478611206 -117.8806700870348) bank22746 +22747 POINT(35.039975045464665 -119.07093702383584) bank22747 +22748 POINT(34.89487728945566 -118.06540346499476) bank22748 +22749 POINT(33.63110436198492 -117.76242427838629) bank22749 +22750 POINT(33.601228635203796 -118.50862329338112) bank22750 +22751 POINT(33.250123337461986 -118.25800170723666) bank22751 +22752 POINT(34.26517006097462 -118.6046910972205) bank22752 +22753 POINT(34.14578436843337 -118.9365429390783) bank22753 +22754 POINT(33.50832608974003 -118.49862966252347) bank22754 +22755 POINT(33.83299144629035 -118.65914764313666) bank22755 +22756 POINT(34.221953113345585 -118.72521285759275) bank22756 +22757 POINT(33.452353219406625 -118.87528422681721) bank22757 +22758 POINT(34.97920280022833 -117.64051483095139) bank22758 +22759 POINT(34.28549675037052 -118.75533422183848) bank22759 +22760 POINT(33.234014472799515 -119.22301566491787) bank22760 +22761 POINT(33.88218064965079 -117.58212013108009) bank22761 +22762 POINT(34.04106320240905 -117.38643571112382) bank22762 +22763 POINT(34.48722759679632 -118.063647922305) bank22763 +22764 POINT(33.1631054046318 -117.43662199042828) bank22764 +22765 POINT(33.984706765586175 -117.37146508714615) bank22765 +22766 POINT(34.234737138491546 -117.97815898544573) bank22766 +22767 POINT(33.54394552706011 -117.58634989806912) bank22767 +22768 POINT(34.953376964412904 -117.63059667930307) bank22768 +22769 POINT(33.79542917917394 -118.77459208233002) bank22769 +22770 POINT(34.07238174449324 -118.20153620788882) bank22770 +22771 POINT(33.42870397371541 -117.4634007520949) bank22771 +22772 POINT(34.54459620214121 -117.31264817719554) bank22772 +22773 POINT(34.30561907821215 -118.57690506168241) bank22773 +22774 POINT(33.649517306211855 -119.08609634613894) bank22774 +22775 POINT(33.94423864960925 -118.86088489658853) bank22775 +22776 POINT(34.498010100683096 -119.1310855165066) bank22776 +22777 POINT(33.388003017666605 -119.02773921477234) bank22777 +22778 POINT(34.27402242519956 -118.13462339433276) bank22778 +22779 POINT(33.1000673538643 -119.06651097544415) bank22779 +22780 POINT(33.731770923668 -117.79174018806086) bank22780 +22781 POINT(34.62912928506122 -118.1804215174904) bank22781 +22782 POINT(34.97626687915898 -117.66433525593486) bank22782 +22783 POINT(35.03674855247314 -117.4577296298298) bank22783 +22784 POINT(33.17754557005724 -118.66446599154075) bank22784 +22785 POINT(34.663343246848186 -117.96546215755198) bank22785 +22786 POINT(33.80029627432076 -118.98884308989926) bank22786 +22787 POINT(34.12858592041805 -117.84479721479889) bank22787 +22788 POINT(33.670248881254935 -118.86395977198573) bank22788 +22789 POINT(34.00240363627121 -118.97917349326714) bank22789 +22790 POINT(34.4589902434496 -118.6601221270184) bank22790 +22791 POINT(34.762628773848384 -119.13849447708122) bank22791 +22792 POINT(33.50433067031257 -118.86487281976683) bank22792 +22793 POINT(33.38688711203912 -117.64434115126089) bank22793 +22794 POINT(33.518070526026584 -119.00567951074308) bank22794 +22795 POINT(34.02301129977666 -118.18417590032354) bank22795 +22796 POINT(33.15680936794165 -118.81454191318221) bank22796 +22797 POINT(33.768490114066154 -117.88229484242366) bank22797 +22798 POINT(34.8891464112443 -118.02093117143272) bank22798 +22799 POINT(34.659870960947245 -118.79469829367711) bank22799 +22800 POINT(33.8619201858209 -117.94101836296421) bank22800 +22801 POINT(33.45615036539743 -119.2026520248651) bank22801 +22802 POINT(35.011527208828916 -118.1297128816869) bank22802 +22803 POINT(33.99986009684113 -117.27223902567002) bank22803 +22804 POINT(33.09596315910699 -117.39181086244123) bank22804 +22805 POINT(33.90838795296452 -118.50379988382181) bank22805 +22806 POINT(33.95348634078958 -117.4417791458975) bank22806 +22807 POINT(33.39580557191217 -117.47256744688997) bank22807 +22808 POINT(33.329207804233164 -117.42663524349305) bank22808 +22809 POINT(34.884420149870515 -118.8433840413192) bank22809 +22810 POINT(34.85255908821279 -118.92167521707864) bank22810 +22811 POINT(33.493250928753234 -117.64787589223141) bank22811 +22812 POINT(33.7114130181948 -118.02346227286424) bank22812 +22813 POINT(35.01979891715995 -118.65633458445562) bank22813 +22814 POINT(33.37232760490466 -117.26638490040547) bank22814 +22815 POINT(34.137155323793664 -117.55834435182526) bank22815 +22816 POINT(33.8041159606527 -117.60867672026015) bank22816 +22817 POINT(34.02145435605715 -118.89957205374927) bank22817 +22818 POINT(33.73488069480948 -119.12180172921778) bank22818 +22819 POINT(34.08249990246003 -117.80971440132363) bank22819 +22820 POINT(33.744389956402074 -119.00107645715312) bank22820 +22821 POINT(33.37985662597713 -118.66953945531628) bank22821 +22822 POINT(33.06087279488271 -119.02128065731456) bank22822 +22823 POINT(33.19561811074183 -117.45187881218122) bank22823 +22824 POINT(34.35167666048294 -119.04665667593972) bank22824 +22825 POINT(33.11100074375442 -118.6071615295585) bank22825 +22826 POINT(33.13880939112457 -118.70143159734073) bank22826 +22827 POINT(33.923894077183796 -118.21734993000322) bank22827 +22828 POINT(33.23128039215347 -118.98062451942944) bank22828 +22829 POINT(34.366387781783736 -118.51787215221754) bank22829 +22830 POINT(34.54259849962011 -118.9095445786324) bank22830 +22831 POINT(33.09148603126205 -117.69644287806766) bank22831 +22832 POINT(33.943346805495956 -118.6165005605164) bank22832 +22833 POINT(33.319259372057246 -118.7375275107394) bank22833 +22834 POINT(33.856440990818015 -118.46330947572774) bank22834 +22835 POINT(34.19424803111895 -117.95140536078324) bank22835 +22836 POINT(35.00735656175644 -118.97833108414592) bank22836 +22837 POINT(34.342527912904295 -118.15260360203796) bank22837 +22838 POINT(34.1522401679703 -118.70209014131402) bank22838 +22839 POINT(34.61400846334332 -118.52214635100422) bank22839 +22840 POINT(33.80443008213424 -117.31601219317031) bank22840 +22841 POINT(33.33683549540542 -117.76399310334835) bank22841 +22842 POINT(33.96580578364161 -118.25013575909261) bank22842 +22843 POINT(34.50472619996108 -119.22356327395607) bank22843 +22844 POINT(34.95882560979857 -117.79828205475651) bank22844 +22845 POINT(34.88489022968176 -117.67667065723293) bank22845 +22846 POINT(33.39830690217353 -118.54465138771047) bank22846 +22847 POINT(34.63391363260959 -118.07141486937789) bank22847 +22848 POINT(34.44772917726194 -118.74468230073062) bank22848 +22849 POINT(33.22929776944868 -117.70920501866578) bank22849 +22850 POINT(33.58875776224632 -119.1393853288221) bank22850 +22851 POINT(33.09325180955438 -117.52963754265392) bank22851 +22852 POINT(34.96437526871892 -118.75046925019998) bank22852 +22853 POINT(33.43375982165892 -119.01294194065312) bank22853 +22854 POINT(34.66181672905778 -117.2803767739793) bank22854 +22855 POINT(34.69852545953009 -117.56432635485766) bank22855 +22856 POINT(33.48625095792039 -119.12702877050836) bank22856 +22857 POINT(34.962467336988745 -118.62524770474029) bank22857 +22858 POINT(33.53191414655106 -117.37068236006785) bank22858 +22859 POINT(34.37872655620022 -118.29019765948075) bank22859 +22860 POINT(34.88356653801517 -118.73924069172111) bank22860 +22861 POINT(34.781055140327254 -117.79047062780258) bank22861 +22862 POINT(34.47157489376325 -118.75226817728463) bank22862 +22863 POINT(34.11855201442828 -119.11468420823692) bank22863 +22864 POINT(33.781464702758605 -118.55379219731445) bank22864 +22865 POINT(33.82545327476697 -117.55041800304721) bank22865 +22866 POINT(33.56700243393823 -119.03235214965963) bank22866 +22867 POINT(34.960387250810534 -118.27548433406379) bank22867 +22868 POINT(33.657074801906404 -118.88811122759667) bank22868 +22869 POINT(33.38783879913183 -118.55733938549218) bank22869 +22870 POINT(35.01533355822668 -118.79551671898514) bank22870 +22871 POINT(34.243404433559014 -117.80492082167756) bank22871 +22872 POINT(34.32200262362534 -119.23572470036213) bank22872 +22873 POINT(34.887375596856515 -118.63780420112505) bank22873 +22874 POINT(33.057370475326984 -119.22965790089454) bank22874 +22875 POINT(33.673928647267054 -118.6575938292247) bank22875 +22876 POINT(33.87834682531883 -118.83867755101112) bank22876 +22877 POINT(34.73530829598426 -119.17404595767859) bank22877 +22878 POINT(34.89134089703203 -118.02445637650446) bank22878 +22879 POINT(34.881669217503344 -117.48159078297088) bank22879 +22880 POINT(34.61512717215523 -119.00551168903787) bank22880 +22881 POINT(34.136482328694534 -117.33844140171428) bank22881 +22882 POINT(34.944855762575 -117.2513568873954) bank22882 +22883 POINT(33.07874910873275 -117.3676051160229) bank22883 +22884 POINT(34.56958353341224 -119.17520806313335) bank22884 +22885 POINT(34.3764696593481 -118.74516899878235) bank22885 +22886 POINT(33.74641972251887 -118.57014194386474) bank22886 +22887 POINT(33.47132712610323 -118.58189104926464) bank22887 +22888 POINT(33.1361569061046 -118.43073329963154) bank22888 +22889 POINT(33.08840921065805 -118.14469725480176) bank22889 +22890 POINT(33.545123338620286 -118.80877481651446) bank22890 +22891 POINT(33.35887848223044 -117.52512830674416) bank22891 +22892 POINT(33.58717854504005 -119.19087861507363) bank22892 +22893 POINT(34.8772473986061 -118.11325906803162) bank22893 +22894 POINT(34.84654257789125 -118.61521955048563) bank22894 +22895 POINT(33.27480066160086 -117.8192829488639) bank22895 +22896 POINT(33.225550117066675 -117.53539590457218) bank22896 +22897 POINT(33.125546723665074 -119.11762994724086) bank22897 +22898 POINT(34.79993063161508 -119.0905647760532) bank22898 +22899 POINT(33.38490521070916 -118.30054108610823) bank22899 +22900 POINT(34.95967826230106 -118.22839655828047) bank22900 +22901 POINT(34.21317275103801 -117.86731428873097) bank22901 +22902 POINT(33.53176238066342 -118.4459045080527) bank22902 +22903 POINT(34.84408700396581 -118.77447171007377) bank22903 +22904 POINT(33.79044312339784 -118.2703702510637) bank22904 +22905 POINT(33.36770635456056 -117.49732273412909) bank22905 +22906 POINT(34.816829035265364 -118.02374307338293) bank22906 +22907 POINT(34.3388221738116 -118.73722061389492) bank22907 +22908 POINT(33.07586534300486 -118.60378563468738) bank22908 +22909 POINT(33.85224842412242 -117.95474783364334) bank22909 +22910 POINT(33.62699758360821 -118.38201748772762) bank22910 +22911 POINT(34.83689266290168 -117.8930048356246) bank22911 +22912 POINT(34.415781835483045 -118.88851685991904) bank22912 +22913 POINT(34.67206071037133 -117.84982792252995) bank22913 +22914 POINT(33.75722287478525 -117.49278724908675) bank22914 +22915 POINT(33.386251404887666 -117.95650469300263) bank22915 +22916 POINT(33.74279912294013 -117.97289444490082) bank22916 +22917 POINT(33.458071473901505 -118.1006192540994) bank22917 +22918 POINT(33.96784080855308 -117.92194538070717) bank22918 +22919 POINT(34.9657274264842 -117.50110562442369) bank22919 +22920 POINT(33.09824227401946 -117.6101642739501) bank22920 +22921 POINT(33.42412472658943 -117.83960489051587) bank22921 +22922 POINT(33.98738561873326 -117.86436126673868) bank22922 +22923 POINT(33.42650765379575 -117.73203884036793) bank22923 +22924 POINT(34.14339962037829 -118.22506343919892) bank22924 +22925 POINT(33.659622867275836 -118.4309842618239) bank22925 +22926 POINT(34.56631374184898 -119.0657081521835) bank22926 +22927 POINT(33.09208647002565 -118.60322515991346) bank22927 +22928 POINT(33.34314090471385 -117.6752199178153) bank22928 +22929 POINT(33.57531691684199 -118.9757973418337) bank22929 +22930 POINT(33.223752342992476 -119.1840384129556) bank22930 +22931 POINT(33.660912141735096 -118.61472807270962) bank22931 +22932 POINT(33.359347171328515 -118.48532531308629) bank22932 +22933 POINT(33.8201714772002 -117.2630824838979) bank22933 +22934 POINT(34.2396847607798 -118.56335588670915) bank22934 +22935 POINT(33.59606079284559 -118.55057863631681) bank22935 +22936 POINT(34.69205273532867 -118.42708632770476) bank22936 +22937 POINT(33.90910023925261 -117.30731410138935) bank22937 +22938 POINT(34.34941093586394 -118.4380918613023) bank22938 +22939 POINT(34.214699372244695 -117.5297582691725) bank22939 +22940 POINT(34.81315828476679 -117.76768294059102) bank22940 +22941 POINT(33.985151873399374 -117.8908236884747) bank22941 +22942 POINT(33.06957116936112 -118.4685402763792) bank22942 +22943 POINT(33.50361368415153 -117.87593513783378) bank22943 +22944 POINT(34.30105308942382 -118.64846133432765) bank22944 +22945 POINT(34.65149206255359 -117.62270987643055) bank22945 +22946 POINT(33.45474309333577 -117.45220667396454) bank22946 +22947 POINT(33.53765440469339 -117.71433307280762) bank22947 +22948 POINT(34.18752539978808 -117.38223884951998) bank22948 +22949 POINT(33.72004378408682 -118.64993225259384) bank22949 +22950 POINT(34.29012607267402 -118.69667627148425) bank22950 +22951 POINT(34.43990697575079 -117.34563674530901) bank22951 +22952 POINT(34.28241264933266 -118.3322436854614) bank22952 +22953 POINT(33.67106365856036 -117.98430182601659) bank22953 +22954 POINT(33.53977720786702 -117.43010398467949) bank22954 +22955 POINT(33.957437251970894 -118.46255590036355) bank22955 +22956 POINT(34.91672870392108 -118.99088621990789) bank22956 +22957 POINT(34.87842288789508 -118.43075861371271) bank22957 +22958 POINT(33.31720412945578 -118.32873557544185) bank22958 +22959 POINT(33.88594471786546 -118.08027243939671) bank22959 +22960 POINT(33.7281843884612 -119.13610575798504) bank22960 +22961 POINT(34.39032250026467 -118.27365106886202) bank22961 +22962 POINT(34.87124968571821 -118.86604797847832) bank22962 +22963 POINT(34.81204805795467 -119.21850628125004) bank22963 +22964 POINT(33.57737568315899 -117.69002487052255) bank22964 +22965 POINT(35.03549229852862 -119.06700292653723) bank22965 +22966 POINT(33.84227273833914 -118.97518915172158) bank22966 +22967 POINT(33.856526907622666 -117.55243850643174) bank22967 +22968 POINT(33.7087348733209 -119.11787178244342) bank22968 +22969 POINT(34.177444755367006 -118.47372304573501) bank22969 +22970 POINT(34.28294221151316 -118.98464010709225) bank22970 +22971 POINT(34.502991180758244 -118.21473307331384) bank22971 +22972 POINT(34.28770402842408 -118.06112498130379) bank22972 +22973 POINT(33.68772564048804 -118.49107510640268) bank22973 +22974 POINT(34.83950566232771 -117.25859550084215) bank22974 +22975 POINT(33.27562370154568 -119.1592787239946) bank22975 +22976 POINT(34.749975248411424 -117.43035710655597) bank22976 +22977 POINT(33.240564470265895 -118.58293662785769) bank22977 +22978 POINT(33.996219616810265 -119.04510539190005) bank22978 +22979 POINT(34.28766597522005 -117.54598266812955) bank22979 +22980 POINT(35.04635732824175 -118.1128729578058) bank22980 +22981 POINT(34.971179182964896 -119.09794505077254) bank22981 +22982 POINT(34.28480925069683 -118.37475507195458) bank22982 +22983 POINT(35.04288805540631 -118.73394348405938) bank22983 +22984 POINT(33.48436854259287 -119.00272649133284) bank22984 +22985 POINT(33.28833510874636 -119.15368772230416) bank22985 +22986 POINT(33.93073717540989 -118.94653149096126) bank22986 +22987 POINT(33.30518805216959 -119.20012246277962) bank22987 +22988 POINT(34.83651378435787 -117.84030730983953) bank22988 +22989 POINT(34.18854357503101 -118.40343328907508) bank22989 +22990 POINT(33.3992537687758 -118.99370581288315) bank22990 +22991 POINT(33.806572127849 -118.20666171389459) bank22991 +22992 POINT(34.59985541384942 -118.39638145287908) bank22992 +22993 POINT(33.78510785036924 -119.217693116324) bank22993 +22994 POINT(33.21622071849877 -119.11964100561025) bank22994 +22995 POINT(33.7385639836039 -119.09132599095827) bank22995 +22996 POINT(33.07201721693306 -119.23885676381371) bank22996 +22997 POINT(34.28432134524134 -118.3395824366151) bank22997 +22998 POINT(34.35382818068999 -119.1571420313358) bank22998 +22999 POINT(33.13007374825464 -119.04140918775776) bank22999 +23000 POINT(33.0696321536305 -117.8500890637636) bank23000 +23001 POINT(33.19090033795643 -118.3455358955476) bank23001 +23002 POINT(33.898140310115025 -118.98630456753591) bank23002 +23003 POINT(33.49696261343418 -117.8361873364332) bank23003 +23004 POINT(34.93071214263617 -117.8935963481554) bank23004 +23005 POINT(33.22714757820444 -119.03292230596452) bank23005 +23006 POINT(33.07829906562497 -117.93794411475598) bank23006 +23007 POINT(33.956901490456595 -118.95732478837633) bank23007 +23008 POINT(34.131660195659414 -118.73794204886593) bank23008 +23009 POINT(34.291973220237956 -118.897183398981) bank23009 +23010 POINT(33.75591980882424 -117.32443887963393) bank23010 +23011 POINT(34.32769974594217 -118.86689419685705) bank23011 +23012 POINT(34.23550555215399 -118.96119033844555) bank23012 +23013 POINT(34.22094440944699 -118.98186918435934) bank23013 +23014 POINT(33.258453521701554 -117.95533776224492) bank23014 +23015 POINT(33.65640009283579 -117.66582291249289) bank23015 +23016 POINT(33.578207132894754 -118.93103285393761) bank23016 +23017 POINT(34.98897070158232 -118.51500340946689) bank23017 +23018 POINT(33.87942070679544 -118.37686454963115) bank23018 +23019 POINT(33.10303596131996 -117.37064991406848) bank23019 +23020 POINT(33.21465548971192 -118.97348563973587) bank23020 +23021 POINT(34.42496171008044 -117.25364011804591) bank23021 +23022 POINT(33.815361743984575 -118.4693320151325) bank23022 +23023 POINT(33.473876233086344 -118.98291625997052) bank23023 +23024 POINT(33.64240013219192 -119.19815439626232) bank23024 +23025 POINT(33.62894725123618 -117.65497141990944) bank23025 +23026 POINT(33.272524085896166 -117.56346870376372) bank23026 +23027 POINT(33.412425453763646 -117.57434446451427) bank23027 +23028 POINT(33.78021500799231 -118.0697441452554) bank23028 +23029 POINT(33.105430312361634 -117.36003968468094) bank23029 +23030 POINT(33.51547243735407 -118.29693582610211) bank23030 +23031 POINT(34.01503392386821 -117.29617742562374) bank23031 +23032 POINT(34.30359961987415 -117.70569469041571) bank23032 +23033 POINT(34.12907382677739 -118.11017458836429) bank23033 +23034 POINT(34.878580056803436 -117.56206023662587) bank23034 +23035 POINT(33.97399598768793 -117.81593320411757) bank23035 +23036 POINT(34.104241118831325 -117.61887968132102) bank23036 +23037 POINT(34.082852587809356 -117.88466404355314) bank23037 +23038 POINT(34.74814522113369 -117.290030607526) bank23038 +23039 POINT(34.686592180869184 -118.61815136609829) bank23039 +23040 POINT(34.37128016265094 -117.87583556309366) bank23040 +23041 POINT(33.669360463341576 -117.60774984768825) bank23041 +23042 POINT(34.445803765608396 -117.64691957308223) bank23042 +23043 POINT(33.479176179412924 -119.11649643108741) bank23043 +23044 POINT(33.40990340682566 -118.84359017820127) bank23044 +23045 POINT(34.97026809951283 -119.03365041016944) bank23045 +23046 POINT(34.77401230599396 -117.57453138123786) bank23046 +23047 POINT(34.68138561270019 -118.13655179092589) bank23047 +23048 POINT(34.44737571729837 -119.00944146228011) bank23048 +23049 POINT(34.23931041663542 -117.62572917806412) bank23049 +23050 POINT(33.37190858938762 -119.03738647063174) bank23050 +23051 POINT(34.446553633727866 -119.04336073445658) bank23051 +23052 POINT(33.110852784769115 -117.64806956453478) bank23052 +23053 POINT(33.65627744695972 -118.71688803030746) bank23053 +23054 POINT(33.502425209014554 -118.07033832728548) bank23054 +23055 POINT(34.03161828706048 -118.26901020696185) bank23055 +23056 POINT(34.90643642367663 -117.68201876820953) bank23056 +23057 POINT(33.13216010024529 -118.4238267379295) bank23057 +23058 POINT(34.50367712812263 -119.00832924629063) bank23058 +23059 POINT(33.35519116913061 -117.53341667799575) bank23059 +23060 POINT(33.58871530847682 -117.38116484838274) bank23060 +23061 POINT(33.53855813125176 -118.20756958306322) bank23061 +23062 POINT(33.61486100009765 -118.76204507830678) bank23062 +23063 POINT(33.0750362402996 -118.59112552624391) bank23063 +23064 POINT(33.813225428702005 -117.76734105462228) bank23064 +23065 POINT(34.532212871620395 -117.25246339717684) bank23065 +23066 POINT(33.49231890323823 -117.78566167543454) bank23066 +23067 POINT(33.996733789017846 -118.68848860559356) bank23067 +23068 POINT(33.26528067571367 -118.83209149045781) bank23068 +23069 POINT(34.72537051611772 -117.83403299307288) bank23069 +23070 POINT(34.783929024415556 -117.27376493615031) bank23070 +23071 POINT(33.91643091237229 -118.45751826144013) bank23071 +23072 POINT(33.1829904063739 -118.37808303776751) bank23072 +23073 POINT(34.22147590555006 -117.77579589299972) bank23073 +23074 POINT(35.04903241123057 -119.18762341212042) bank23074 +23075 POINT(33.4290680061919 -117.8419040966374) bank23075 +23076 POINT(33.906369310465955 -118.56510331134369) bank23076 +23077 POINT(33.36182722973696 -119.02451768478218) bank23077 +23078 POINT(34.75778522500958 -118.02749324515676) bank23078 +23079 POINT(34.486297000107236 -118.61954792449251) bank23079 +23080 POINT(34.73557102260237 -117.85121810474108) bank23080 +23081 POINT(33.15316913946799 -117.94672723265103) bank23081 +23082 POINT(33.798730939077615 -117.45471805099425) bank23082 +23083 POINT(34.984538279727346 -117.64053414309274) bank23083 +23084 POINT(34.86257946230378 -118.0829539063202) bank23084 +23085 POINT(34.19450726271799 -119.15260518983406) bank23085 +23086 POINT(33.98078702490828 -118.09992280054287) bank23086 +23087 POINT(34.71648993291782 -118.88620101748516) bank23087 +23088 POINT(34.52069575066584 -119.1005634203568) bank23088 +23089 POINT(33.80791969965262 -117.79569815557097) bank23089 +23090 POINT(34.16147889847477 -118.8558072031551) bank23090 +23091 POINT(34.455309630973034 -118.13073797000436) bank23091 +23092 POINT(33.59233029607127 -118.23200906311908) bank23092 +23093 POINT(34.87356263762365 -118.62298759914569) bank23093 +23094 POINT(34.85271801667917 -117.95000700205465) bank23094 +23095 POINT(33.89998996717148 -118.58374888822267) bank23095 +23096 POINT(34.29429462353716 -118.63980906924151) bank23096 +23097 POINT(34.12542169483843 -117.9102385937859) bank23097 +23098 POINT(34.943354951941956 -119.2181122344911) bank23098 +23099 POINT(33.55907755825927 -119.00613191008956) bank23099 +23100 POINT(33.29770841709617 -117.83882313284033) bank23100 +23101 POINT(34.02577090044164 -117.58781353943806) bank23101 +23102 POINT(33.470192214901026 -118.84252781962223) bank23102 +23103 POINT(34.64017381951378 -118.3272620564588) bank23103 +23104 POINT(33.428096331349145 -119.1291732446848) bank23104 +23105 POINT(35.0085833330597 -117.36881011887783) bank23105 +23106 POINT(34.72357499504062 -117.47895418620142) bank23106 +23107 POINT(34.870761091415325 -117.41935463100616) bank23107 +23108 POINT(33.09719967327702 -118.32418121149523) bank23108 +23109 POINT(35.03475059415152 -117.79053948914098) bank23109 +23110 POINT(34.232793123601404 -117.92185735465581) bank23110 +23111 POINT(33.95271718634991 -118.60288111901023) bank23111 +23112 POINT(34.88178637528647 -117.95485658828649) bank23112 +23113 POINT(34.385666596613355 -118.08228363415891) bank23113 +23114 POINT(34.187364622244544 -118.96865656242235) bank23114 +23115 POINT(34.97443500570439 -118.88884920912272) bank23115 +23116 POINT(33.09722624551939 -118.98543650594682) bank23116 +23117 POINT(34.45489621550722 -119.1470442623498) bank23117 +23118 POINT(35.03880029170779 -119.16257722212103) bank23118 +23119 POINT(34.41962881156537 -117.48093756511393) bank23119 +23120 POINT(33.85093701217254 -119.02185706828189) bank23120 +23121 POINT(33.893302824434116 -118.17971622190142) bank23121 +23122 POINT(34.637894129130736 -117.33277410839429) bank23122 +23123 POINT(34.80003929943527 -118.62082177026967) bank23123 +23124 POINT(34.2241030596719 -117.90773721233643) bank23124 +23125 POINT(33.346794909167045 -118.50282949236549) bank23125 +23126 POINT(33.236599030760935 -118.546162912011) bank23126 +23127 POINT(33.369387385522295 -117.4022430112511) bank23127 +23128 POINT(33.2391964562815 -118.3031093117869) bank23128 +23129 POINT(34.98829520673043 -118.10906979562318) bank23129 +23130 POINT(33.739242177618586 -118.63462925789048) bank23130 +23131 POINT(33.10141853472782 -117.72518864682726) bank23131 +23132 POINT(33.20677805397087 -118.85355631142698) bank23132 +23133 POINT(33.63739564410453 -117.24982696393934) bank23133 +23134 POINT(34.485329418517196 -118.52316086810075) bank23134 +23135 POINT(34.51633744120201 -118.2379321804902) bank23135 +23136 POINT(34.829191027444104 -117.33961958608226) bank23136 +23137 POINT(34.50417857614191 -117.47125369192115) bank23137 +23138 POINT(34.534106322604906 -118.87416343487716) bank23138 +23139 POINT(33.61564932000073 -119.13471239889847) bank23139 +23140 POINT(34.76606119929956 -118.77181148441859) bank23140 +23141 POINT(33.42838680282181 -117.25293494093201) bank23141 +23142 POINT(33.16877344645475 -118.30501406339903) bank23142 +23143 POINT(35.00066941536346 -117.54982261446295) bank23143 +23144 POINT(33.53863665498183 -117.77669447072692) bank23144 +23145 POINT(33.10971965810522 -117.45408708366766) bank23145 +23146 POINT(33.7553123275499 -118.75195169819354) bank23146 +23147 POINT(33.538438943321125 -117.74379380617158) bank23147 +23148 POINT(33.14657286796373 -117.6838591403252) bank23148 +23149 POINT(34.69214208032174 -118.13156859686414) bank23149 +23150 POINT(33.91028440067528 -117.99161219096382) bank23150 +23151 POINT(34.60994263782693 -118.76579928721827) bank23151 +23152 POINT(33.240912617240994 -117.39513808498133) bank23152 +23153 POINT(33.20088866402526 -118.06124920410532) bank23153 +23154 POINT(34.80804845289885 -117.60923340332273) bank23154 +23155 POINT(33.388850223681125 -119.1370215040459) bank23155 +23156 POINT(33.71932790210435 -117.98504075338491) bank23156 +23157 POINT(34.61310088354462 -117.62779179234373) bank23157 +23158 POINT(34.82356677766028 -117.89841200249994) bank23158 +23159 POINT(33.28493464773751 -117.39150699932459) bank23159 +23160 POINT(34.38844854152191 -118.22988368425824) bank23160 +23161 POINT(33.986151601711256 -119.00016076496438) bank23161 +23162 POINT(33.98227393221039 -119.16915299173961) bank23162 +23163 POINT(33.24920092678769 -118.88085678178273) bank23163 +23164 POINT(33.08079133334576 -117.6685783643613) bank23164 +23165 POINT(34.15881345503976 -117.78697748235477) bank23165 +23166 POINT(33.79454444009497 -118.73847357320103) bank23166 +23167 POINT(34.880838717775305 -117.79588822637385) bank23167 +23168 POINT(33.53658527303859 -119.06403088137999) bank23168 +23169 POINT(33.16275429952471 -118.77460033899024) bank23169 +23170 POINT(33.287743281184106 -118.97229833928735) bank23170 +23171 POINT(33.96211986535226 -117.7505402794281) bank23171 +23172 POINT(33.61500318091501 -117.92539121659543) bank23172 +23173 POINT(34.537210820025855 -118.15964278863693) bank23173 +23174 POINT(34.7349167850562 -119.2226672754627) bank23174 +23175 POINT(33.410819563011785 -117.3140200498727) bank23175 +23176 POINT(33.146259061943375 -117.90373191380162) bank23176 +23177 POINT(33.25992733962896 -117.40691659953859) bank23177 +23178 POINT(34.90337553427727 -118.70973610150286) bank23178 +23179 POINT(33.98696040681968 -117.53379082729985) bank23179 +23180 POINT(33.6175678277779 -118.27691898589175) bank23180 +23181 POINT(33.54478567414922 -118.82036619910487) bank23181 +23182 POINT(34.236760680783995 -118.90512196065332) bank23182 +23183 POINT(33.49975411028984 -118.44576145196427) bank23183 +23184 POINT(34.31772059595705 -119.09922975020872) bank23184 +23185 POINT(34.318010451189245 -119.08290585747717) bank23185 +23186 POINT(33.13749686805486 -118.50002098744314) bank23186 +23187 POINT(33.06861334107911 -118.0800126504539) bank23187 +23188 POINT(33.83984976600584 -117.58383139891852) bank23188 +23189 POINT(33.15624113144264 -119.22758624498691) bank23189 +23190 POINT(34.134709959118325 -117.8601268806009) bank23190 +23191 POINT(34.68530707264691 -119.1017759573701) bank23191 +23192 POINT(33.518101321513555 -117.28340314059653) bank23192 +23193 POINT(33.94484702343566 -119.12955159829798) bank23193 +23194 POINT(33.507770698015754 -118.22330591441218) bank23194 +23195 POINT(34.784066197147425 -118.08853733712003) bank23195 +23196 POINT(33.59100874725204 -118.32157849736595) bank23196 +23197 POINT(33.571391690202375 -118.95488392695474) bank23197 +23198 POINT(34.28682290588764 -118.96217487375348) bank23198 +23199 POINT(33.798330154334444 -119.06244321661755) bank23199 +23200 POINT(34.055740489478914 -117.32170876947201) bank23200 +23201 POINT(34.06365810491744 -117.46844264653609) bank23201 +23202 POINT(34.17585262752348 -118.48485838294152) bank23202 +23203 POINT(33.85637561037454 -117.39633796588322) bank23203 +23204 POINT(34.61191497873675 -118.86571207981589) bank23204 +23205 POINT(34.566902048217145 -117.8819335739112) bank23205 +23206 POINT(33.098881026946366 -117.31027570909845) bank23206 +23207 POINT(33.169716508257565 -117.30289380819731) bank23207 +23208 POINT(33.71819892011693 -117.35430871169552) bank23208 +23209 POINT(33.91645781843712 -118.52436458597832) bank23209 +23210 POINT(34.70419472293991 -117.49831447790551) bank23210 +23211 POINT(34.536287299953244 -118.11880363601584) bank23211 +23212 POINT(34.38190497465359 -118.5154762151601) bank23212 +23213 POINT(33.448836326888426 -118.24708574332715) bank23213 +23214 POINT(34.12836723182927 -118.67804641418721) bank23214 +23215 POINT(33.28151192396867 -118.52633424015112) bank23215 +23216 POINT(34.12051791052342 -119.09515185189247) bank23216 +23217 POINT(33.721216801405724 -118.05140478157779) bank23217 +23218 POINT(33.36705481262045 -118.27728076776128) bank23218 +23219 POINT(33.21756087284373 -117.64647050994665) bank23219 +23220 POINT(34.91935939018868 -118.51735652978672) bank23220 +23221 POINT(33.13797990357092 -118.85279467986071) bank23221 +23222 POINT(33.531599560360945 -118.73775843181883) bank23222 +23223 POINT(33.73501283216274 -118.6589923387981) bank23223 +23224 POINT(34.98643569145482 -117.63270078071734) bank23224 +23225 POINT(34.30406073520186 -118.77973790441021) bank23225 +23226 POINT(33.60714281806458 -119.10485511615879) bank23226 +23227 POINT(34.45234366805715 -117.40498023895042) bank23227 +23228 POINT(33.5864620311945 -119.15258617813981) bank23228 +23229 POINT(33.16276863702349 -118.22066292700828) bank23229 +23230 POINT(34.0645177336616 -118.42621076381246) bank23230 +23231 POINT(33.1553487648854 -119.13009142824485) bank23231 +23232 POINT(34.80315750608807 -118.43950573066105) bank23232 +23233 POINT(34.3548938349764 -118.67550479882924) bank23233 +23234 POINT(33.57606485355606 -117.95127302016608) bank23234 +23235 POINT(34.34050095904893 -118.09577095333901) bank23235 +23236 POINT(33.19843053607464 -119.23783747082012) bank23236 +23237 POINT(34.12255288734314 -119.14319562665534) bank23237 +23238 POINT(33.73773776144565 -119.20694413537751) bank23238 +23239 POINT(34.17126081671532 -118.64433990207955) bank23239 +23240 POINT(34.688559575078905 -117.68313835344175) bank23240 +23241 POINT(34.69573608677711 -117.66852642390874) bank23241 +23242 POINT(33.583342571094605 -119.11715570508623) bank23242 +23243 POINT(33.53088246075081 -118.56745055873385) bank23243 +23244 POINT(33.69064512560393 -117.56139244360747) bank23244 +23245 POINT(34.63866055165076 -118.70807612061134) bank23245 +23246 POINT(34.66377131792829 -117.33282376036236) bank23246 +23247 POINT(34.18137035615133 -118.30953387668227) bank23247 +23248 POINT(34.05703509656079 -117.32745745356017) bank23248 +23249 POINT(33.845214893963146 -118.17467916634293) bank23249 +23250 POINT(34.822126780904256 -117.71625627450277) bank23250 +23251 POINT(33.58805504491253 -118.23386975231732) bank23251 +23252 POINT(33.33370087357361 -117.89753577742161) bank23252 +23253 POINT(33.57544078667403 -117.63967266537722) bank23253 +23254 POINT(33.543962566966684 -117.92010934675544) bank23254 +23255 POINT(34.721108617533226 -117.53824745596141) bank23255 +23256 POINT(33.88918198733312 -119.01186609614403) bank23256 +23257 POINT(33.76307323615435 -119.21032361905631) bank23257 +23258 POINT(35.03845584774908 -118.2207978185654) bank23258 +23259 POINT(33.83234779695264 -118.1078061902259) bank23259 +23260 POINT(34.224651763629744 -117.79268567114104) bank23260 +23261 POINT(33.26410484324584 -118.36853359754726) bank23261 +23262 POINT(34.755317241682306 -118.62899996098709) bank23262 +23263 POINT(33.245569861120494 -117.6967567133104) bank23263 +23264 POINT(33.473927740534954 -118.67284301394542) bank23264 +23265 POINT(33.551770786914254 -118.5959018008163) bank23265 +23266 POINT(34.03313253743476 -117.34930144300033) bank23266 +23267 POINT(34.96941864057924 -118.79469669708504) bank23267 +23268 POINT(33.13603386908069 -117.48424573185788) bank23268 +23269 POINT(34.58455174321266 -117.59852989912584) bank23269 +23270 POINT(34.824837717741 -118.28853090155988) bank23270 +23271 POINT(33.54292834528812 -117.41428434626673) bank23271 +23272 POINT(34.710643167337864 -118.16891249628938) bank23272 +23273 POINT(33.12452623621286 -118.67925562571803) bank23273 +23274 POINT(34.06279965107027 -118.96776496705216) bank23274 +23275 POINT(35.015297134764424 -117.5861336068897) bank23275 +23276 POINT(34.478791637693604 -119.24005666110067) bank23276 +23277 POINT(34.08870819058779 -118.11500546101432) bank23277 +23278 POINT(33.45741389118063 -118.93705467349083) bank23278 +23279 POINT(33.56025074626934 -118.71580198890183) bank23279 +23280 POINT(34.30816597571639 -119.20843955007248) bank23280 +23281 POINT(34.265556254171464 -118.41137965756006) bank23281 +23282 POINT(34.87018197241328 -118.36155072875263) bank23282 +23283 POINT(34.57152793840636 -119.22347646388957) bank23283 +23284 POINT(33.25745318776132 -117.61589230528323) bank23284 +23285 POINT(33.860366392240635 -117.60655808132958) bank23285 +23286 POINT(33.61864758328712 -117.46508755261847) bank23286 +23287 POINT(33.96898076302018 -118.87180168282626) bank23287 +23288 POINT(35.03145412606225 -118.90835306380485) bank23288 +23289 POINT(34.57361552875957 -117.9040343647502) bank23289 +23290 POINT(34.92977083412878 -118.56084079433236) bank23290 +23291 POINT(33.43153792102325 -117.42810393964245) bank23291 +23292 POINT(34.38442147610873 -117.32321304914147) bank23292 +23293 POINT(34.98387923278345 -117.53013737089691) bank23293 +23294 POINT(34.94177853217786 -118.09683958491986) bank23294 +23295 POINT(34.37909961220975 -117.56177790264931) bank23295 +23296 POINT(33.17424009727149 -117.27167882789192) bank23296 +23297 POINT(34.13670592651088 -118.77919799259028) bank23297 +23298 POINT(33.40072285822557 -118.62799046477136) bank23298 +23299 POINT(33.68204530360804 -118.97205938168628) bank23299 +23300 POINT(34.768085433031146 -118.13208581462139) bank23300 +23301 POINT(33.25101269157255 -118.42165737886984) bank23301 +23302 POINT(33.39420518515838 -117.4579078824676) bank23302 +23303 POINT(34.366052324393515 -118.25993125297664) bank23303 +23304 POINT(34.67212936599784 -117.34197192606234) bank23304 +23305 POINT(33.488757248555586 -117.68710526430945) bank23305 +23306 POINT(33.82530599901016 -118.16110067056854) bank23306 +23307 POINT(33.83367443280737 -117.87169586384931) bank23307 +23308 POINT(34.04737831934757 -119.10919439495937) bank23308 +23309 POINT(34.202782600307586 -118.0400752072047) bank23309 +23310 POINT(33.955577165750206 -119.16315958161238) bank23310 +23311 POINT(33.14421217159668 -118.35028311375922) bank23311 +23312 POINT(34.5486234250879 -117.69789681213794) bank23312 +23313 POINT(33.46143720555225 -119.12847168294878) bank23313 +23314 POINT(33.130765967441036 -118.43052420563771) bank23314 +23315 POINT(34.602284844013376 -118.98825605362151) bank23315 +23316 POINT(34.265359512702275 -117.65697759051841) bank23316 +23317 POINT(34.74243123715577 -118.3953082376643) bank23317 +23318 POINT(34.541468230543444 -118.89013992854265) bank23318 +23319 POINT(34.27538952498297 -118.23710315598231) bank23319 +23320 POINT(34.71423581368846 -117.61578394168974) bank23320 +23321 POINT(34.04705981874414 -117.38409499747966) bank23321 +23322 POINT(33.3504839744151 -118.27055084804948) bank23322 +23323 POINT(33.226872793096746 -117.623096025993) bank23323 +23324 POINT(34.17088465230589 -118.9294251398872) bank23324 +23325 POINT(34.60858510937554 -118.24902451485225) bank23325 +23326 POINT(33.56566240538751 -118.46608333007867) bank23326 +23327 POINT(35.03629556918731 -118.05434294894248) bank23327 +23328 POINT(34.504039575009905 -117.4520850940008) bank23328 +23329 POINT(34.541159245760205 -117.6129418893562) bank23329 +23330 POINT(33.49296504145196 -117.46222765931556) bank23330 +23331 POINT(34.666782787374345 -117.33562442904888) bank23331 +23332 POINT(34.489540397915604 -119.0218464850926) bank23332 +23333 POINT(34.74073375943016 -117.8680406439792) bank23333 +23334 POINT(33.31736272648626 -117.46148283547278) bank23334 +23335 POINT(34.788866740165616 -119.20919986605024) bank23335 +23336 POINT(34.644891400866754 -118.05456260450667) bank23336 +23337 POINT(33.723241010678734 -118.58942994785586) bank23337 +23338 POINT(34.94695543460296 -118.5263310662832) bank23338 +23339 POINT(34.56186332987063 -119.01727672532186) bank23339 +23340 POINT(33.44015412716664 -117.83469184172577) bank23340 +23341 POINT(33.99777013027482 -117.38915671113229) bank23341 +23342 POINT(34.32577809840661 -117.28315356873325) bank23342 +23343 POINT(34.29626391004717 -119.15791780503493) bank23343 +23344 POINT(34.4730713747985 -117.58510400565234) bank23344 +23345 POINT(34.69670902090972 -119.13760757512527) bank23345 +23346 POINT(33.63594707046658 -117.743909235414) bank23346 +23347 POINT(34.32822779978702 -118.96891191016674) bank23347 +23348 POINT(34.57415370056864 -117.51717964441644) bank23348 +23349 POINT(33.50545635968402 -118.10927344575396) bank23349 +23350 POINT(34.20440868472725 -117.5283991236854) bank23350 +23351 POINT(33.432401714969814 -118.63436929169237) bank23351 +23352 POINT(33.067984760681725 -118.41083694042891) bank23352 +23353 POINT(33.2040380027306 -118.37458462581576) bank23353 +23354 POINT(33.84932683243507 -118.62860037400652) bank23354 +23355 POINT(34.22544411673769 -118.22371722673854) bank23355 +23356 POINT(33.52752425170944 -117.61982113125471) bank23356 +23357 POINT(34.97921859244032 -118.24928440730663) bank23357 +23358 POINT(33.986645455518406 -118.47995930716357) bank23358 +23359 POINT(34.31397129140892 -117.79848847389073) bank23359 +23360 POINT(34.401238327519536 -118.85586685758399) bank23360 +23361 POINT(35.001436363843 -117.64316768629108) bank23361 +23362 POINT(34.85616916056569 -118.07789203676576) bank23362 +23363 POINT(33.41049924096411 -118.86634482347336) bank23363 +23364 POINT(34.658278149995446 -119.20088641606522) bank23364 +23365 POINT(33.7094886267535 -118.96932796224185) bank23365 +23366 POINT(33.196305216902424 -117.38660421382751) bank23366 +23367 POINT(33.09972648607842 -118.87429535880386) bank23367 +23368 POINT(35.00758973323087 -118.53149756040409) bank23368 +23369 POINT(33.25976676864041 -118.62192936060059) bank23369 +23370 POINT(34.79265224620266 -117.66824367873998) bank23370 +23371 POINT(33.23035705376637 -117.84890002986084) bank23371 +23372 POINT(34.20509423975617 -117.86835908706162) bank23372 +23373 POINT(33.844388414987456 -117.43077400753613) bank23373 +23374 POINT(35.025283373324655 -117.60472693055686) bank23374 +23375 POINT(34.52262549472186 -117.82787665610377) bank23375 +23376 POINT(33.4685943283302 -118.10672515092315) bank23376 +23377 POINT(34.53589886688202 -118.6793500410112) bank23377 +23378 POINT(34.87963913301206 -119.2095238641637) bank23378 +23379 POINT(34.67917513757475 -117.50302808264799) bank23379 +23380 POINT(34.67291497736473 -117.78497530864263) bank23380 +23381 POINT(33.30706007350023 -117.52701204959327) bank23381 +23382 POINT(33.21376371763308 -117.5742274375459) bank23382 +23383 POINT(34.544923041162846 -119.12666293031816) bank23383 +23384 POINT(33.05226435044314 -117.75363517532337) bank23384 +23385 POINT(33.69189114695277 -117.98237476956521) bank23385 +23386 POINT(34.409204358205145 -119.0298415728429) bank23386 +23387 POINT(34.794557955808045 -118.02014826981708) bank23387 +23388 POINT(33.73285075962494 -117.25296396508124) bank23388 +23389 POINT(34.14351323320075 -118.16351671343614) bank23389 +23390 POINT(34.45316181564119 -118.49942418772561) bank23390 +23391 POINT(33.12173245678801 -118.93413736365372) bank23391 +23392 POINT(34.02237174022233 -118.34483867768763) bank23392 +23393 POINT(34.942916846973596 -118.609704098638) bank23393 +23394 POINT(33.87172645654394 -118.23850444860531) bank23394 +23395 POINT(33.08324641164099 -119.1498694356756) bank23395 +23396 POINT(34.91316158639944 -118.82584874623636) bank23396 +23397 POINT(33.786751764527075 -117.77024389016185) bank23397 +23398 POINT(34.06963637670188 -118.96176596492505) bank23398 +23399 POINT(33.18561069001867 -118.35673909902025) bank23399 +23400 POINT(34.65949337864346 -118.45072766659906) bank23400 +23401 POINT(33.729829115042484 -117.4649032528905) bank23401 +23402 POINT(33.33658471141309 -117.47601616107777) bank23402 +23403 POINT(34.382080100032695 -118.90641998008705) bank23403 +23404 POINT(33.162441353752556 -118.44710123768294) bank23404 +23405 POINT(33.37577020091588 -117.26420992896388) bank23405 +23406 POINT(34.766996283857765 -118.60840232105647) bank23406 +23407 POINT(33.76794392604133 -117.59378164504488) bank23407 +23408 POINT(33.10228599339004 -117.28224022737513) bank23408 +23409 POINT(34.306742711741656 -118.50726972709131) bank23409 +23410 POINT(33.54960059567651 -117.92464619719098) bank23410 +23411 POINT(34.7903983935546 -118.68031124950798) bank23411 +23412 POINT(34.51387472930566 -119.1551180813032) bank23412 +23413 POINT(33.0573688605353 -118.07544751608629) bank23413 +23414 POINT(34.21835766675917 -118.43764443672015) bank23414 +23415 POINT(34.918516692958804 -117.89632161893392) bank23415 +23416 POINT(34.556072439630235 -117.38340473182902) bank23416 +23417 POINT(34.310381727055066 -118.20534941661268) bank23417 +23418 POINT(33.79769035412402 -118.00812444822742) bank23418 +23419 POINT(33.94798578409475 -117.89886226256537) bank23419 +23420 POINT(34.65901541434007 -118.62996755832847) bank23420 +23421 POINT(34.92464529574923 -118.68247756031882) bank23421 +23422 POINT(33.58553090266398 -117.84357483609404) bank23422 +23423 POINT(33.078866609431415 -117.26109402840683) bank23423 +23424 POINT(33.527460450701284 -118.61267810871378) bank23424 +23425 POINT(33.09088640823377 -117.71773355434303) bank23425 +23426 POINT(33.22348425382497 -118.51991102152736) bank23426 +23427 POINT(33.193434190299286 -117.73696176423533) bank23427 +23428 POINT(34.80568532378438 -118.63006803853708) bank23428 +23429 POINT(34.48276091591236 -118.1185405697825) bank23429 +23430 POINT(34.35519794294821 -117.4892797328899) bank23430 +23431 POINT(34.47071608390946 -118.75100443930747) bank23431 +23432 POINT(34.32454177386912 -117.93625892292226) bank23432 +23433 POINT(33.77606690935837 -117.35918502138071) bank23433 +23434 POINT(33.19228067286241 -118.61994266783526) bank23434 +23435 POINT(34.07756887907792 -118.75176846870298) bank23435 +23436 POINT(33.38037230567174 -117.41851191319017) bank23436 +23437 POINT(33.22399492659063 -117.86968312144278) bank23437 +23438 POINT(34.865384757881 -118.52024499339686) bank23438 +23439 POINT(35.02436137698739 -118.6961173489571) bank23439 +23440 POINT(34.821351727488185 -118.15854907861899) bank23440 +23441 POINT(33.72215468031111 -117.77211670373049) bank23441 +23442 POINT(33.562525054309305 -118.38550780779265) bank23442 +23443 POINT(33.15675019429417 -119.23011992021017) bank23443 +23444 POINT(33.98467932941077 -117.86621939006858) bank23444 +23445 POINT(33.787305177311254 -118.61352378128379) bank23445 +23446 POINT(34.62451648740068 -119.0938497841492) bank23446 +23447 POINT(34.1762989960241 -119.10978007527507) bank23447 +23448 POINT(34.87132356886407 -117.4979167315205) bank23448 +23449 POINT(34.68665526383758 -118.61707530208525) bank23449 +23450 POINT(33.824561598263664 -119.12469164984195) bank23450 +23451 POINT(34.094854814120275 -119.13142279668834) bank23451 +23452 POINT(34.31645032083906 -118.24899029261705) bank23452 +23453 POINT(35.03575464436784 -118.3677045878269) bank23453 +23454 POINT(34.01478468594212 -117.40991329871115) bank23454 +23455 POINT(34.75862148931208 -117.83871153362303) bank23455 +23456 POINT(34.918535984755664 -118.94849654735454) bank23456 +23457 POINT(34.9405852916995 -119.08452618422587) bank23457 +23458 POINT(34.8953464370252 -117.74357585708816) bank23458 +23459 POINT(35.018026718586455 -118.8886824786958) bank23459 +23460 POINT(34.69127089323138 -118.62258620098461) bank23460 +23461 POINT(34.942921407649784 -117.94715696892803) bank23461 +23462 POINT(33.0891608109197 -117.25799120366526) bank23462 +23463 POINT(34.102333961875395 -117.95682195176849) bank23463 +23464 POINT(33.525808331058165 -119.15914358556013) bank23464 +23465 POINT(33.38577705726005 -118.52005711713967) bank23465 +23466 POINT(34.27066513722214 -118.45766794843152) bank23466 +23467 POINT(33.28545662249901 -118.85390352081622) bank23467 +23468 POINT(33.79247943423293 -118.10486092688002) bank23468 +23469 POINT(34.84829955789094 -119.0217783076107) bank23469 +23470 POINT(34.722869584736166 -118.7052570543343) bank23470 +23471 POINT(34.65433050940103 -118.48872078451144) bank23471 +23472 POINT(34.888986320064724 -118.55221949385854) bank23472 +23473 POINT(34.89975333153296 -118.74063709566946) bank23473 +23474 POINT(33.132823724730144 -118.0043257393535) bank23474 +23475 POINT(34.527063466224945 -118.19952558107221) bank23475 +23476 POINT(33.07526183922379 -117.98466252767548) bank23476 +23477 POINT(33.40288804294885 -117.35680046795905) bank23477 +23478 POINT(33.58504985290526 -117.7542051523858) bank23478 +23479 POINT(34.7433960761525 -118.25754617639761) bank23479 +23480 POINT(34.15718891709679 -117.28453247406284) bank23480 +23481 POINT(35.02089211530617 -117.4953895648434) bank23481 +23482 POINT(33.3710749160365 -119.14172174182521) bank23482 +23483 POINT(34.956405220830284 -117.67430296249935) bank23483 +23484 POINT(34.40327537867519 -117.36209940918111) bank23484 +23485 POINT(33.70764712860969 -118.97532438353427) bank23485 +23486 POINT(33.83674677350681 -118.82399247870657) bank23486 +23487 POINT(33.923176803724054 -119.01688539800917) bank23487 +23488 POINT(34.20602679255595 -118.49606561633853) bank23488 +23489 POINT(33.226065101166114 -118.80021281065524) bank23489 +23490 POINT(33.628185286211306 -118.75445771782648) bank23490 +23491 POINT(33.66819116756939 -117.66021501443636) bank23491 +23492 POINT(34.899409213535904 -117.47125540986258) bank23492 +23493 POINT(33.937446295589076 -118.25274586595793) bank23493 +23494 POINT(34.35784905859177 -117.97903036542094) bank23494 +23495 POINT(34.985575295043965 -118.42993708829394) bank23495 +23496 POINT(34.44663542224035 -118.68419388778554) bank23496 +23497 POINT(34.44698538084956 -118.60779470856342) bank23497 +23498 POINT(34.05542274970653 -118.25848539311815) bank23498 +23499 POINT(33.422595581237466 -117.46194211280951) bank23499 +23500 POINT(34.14495583261008 -118.96570309028824) bank23500 +23501 POINT(34.67611961402106 -117.53035583496657) bank23501 +23502 POINT(34.15501443964738 -117.50386742925292) bank23502 +23503 POINT(33.582091822063994 -118.64531502528327) bank23503 +23504 POINT(34.64967386650784 -118.42066407499614) bank23504 +23505 POINT(34.18650878911122 -117.9068346082738) bank23505 +23506 POINT(33.14290401415184 -119.00974367040116) bank23506 +23507 POINT(33.88785673067128 -118.59078325390236) bank23507 +23508 POINT(33.784891201792966 -118.18830838419836) bank23508 +23509 POINT(34.23387563928995 -118.1802939911842) bank23509 +23510 POINT(33.418013167276484 -117.62707981715286) bank23510 +23511 POINT(34.106047735009575 -118.87617881890067) bank23511 +23512 POINT(33.29233691308869 -117.69036356415467) bank23512 +23513 POINT(33.897372027121456 -117.89651141537598) bank23513 +23514 POINT(34.06092325145867 -117.85009977275169) bank23514 +23515 POINT(33.26749024229616 -118.0753430392134) bank23515 +23516 POINT(33.399243063364345 -119.12401507555137) bank23516 +23517 POINT(34.49604823135098 -118.55058902661439) bank23517 +23518 POINT(34.71869955114785 -118.70380267521828) bank23518 +23519 POINT(33.17427988920206 -117.78293074314307) bank23519 +23520 POINT(33.46335036929513 -118.59056511207957) bank23520 +23521 POINT(34.092950025647305 -118.48947729926806) bank23521 +23522 POINT(34.83743973693503 -117.6828334263739) bank23522 +23523 POINT(35.031090803123156 -119.13951581385598) bank23523 +23524 POINT(33.88779804386736 -118.7884066677956) bank23524 +23525 POINT(34.49676160413092 -117.35794058878852) bank23525 +23526 POINT(34.88474255796075 -117.94752083057473) bank23526 +23527 POINT(33.256150515728194 -119.10593146802218) bank23527 +23528 POINT(35.00984450008841 -117.97438447414503) bank23528 +23529 POINT(33.15944906893326 -117.25646459504584) bank23529 +23530 POINT(33.34206905886965 -117.68768725445115) bank23530 +23531 POINT(34.66141983875076 -118.04258729215458) bank23531 +23532 POINT(33.51839604699539 -117.83346517212321) bank23532 +23533 POINT(34.7455500147073 -118.2032463217778) bank23533 +23534 POINT(33.85549641135255 -117.75131109853263) bank23534 +23535 POINT(34.323285882801464 -117.95576413292402) bank23535 +23536 POINT(33.48048267332479 -117.85443114202225) bank23536 +23537 POINT(34.19516874884125 -118.39749136069678) bank23537 +23538 POINT(33.151670084573 -117.55888720546713) bank23538 +23539 POINT(33.93640035795018 -117.7788817948521) bank23539 +23540 POINT(34.22876939069179 -117.78166106648077) bank23540 +23541 POINT(34.64115071501998 -118.15566965770367) bank23541 +23542 POINT(33.371291543601465 -118.49539063315778) bank23542 +23543 POINT(33.95110955058253 -119.19324565973726) bank23543 +23544 POINT(34.448928298211385 -118.71047420932116) bank23544 +23545 POINT(34.03143610063844 -118.0933995245709) bank23545 +23546 POINT(33.33188945540953 -118.48153235748573) bank23546 +23547 POINT(33.623785984331995 -118.14841048884733) bank23547 +23548 POINT(34.20784610628209 -117.6447433404149) bank23548 +23549 POINT(34.88711970899752 -117.77473592825098) bank23549 +23550 POINT(33.11394395730108 -119.22825928981688) bank23550 +23551 POINT(35.03361123784791 -118.3044905829854) bank23551 +23552 POINT(34.889026159623995 -118.17623849258659) bank23552 +23553 POINT(33.799534915441825 -117.56235583918448) bank23553 +23554 POINT(34.75438000201518 -118.7823948141099) bank23554 +23555 POINT(33.220395441142436 -117.91441287740902) bank23555 +23556 POINT(34.23414515995083 -117.60160224818038) bank23556 +23557 POINT(34.351838937061096 -118.73809651749505) bank23557 +23558 POINT(33.82181777933966 -119.07475259926987) bank23558 +23559 POINT(34.022970723467296 -117.68708571821422) bank23559 +23560 POINT(33.94175139768927 -118.84672998087333) bank23560 +23561 POINT(34.16512598791452 -117.98497464955236) bank23561 +23562 POINT(34.80250704034035 -117.82182521592387) bank23562 +23563 POINT(33.11272229673805 -118.45640025918092) bank23563 +23564 POINT(34.06082214501627 -118.13195033653307) bank23564 +23565 POINT(33.55529387189331 -117.55260879698706) bank23565 +23566 POINT(34.4030638851498 -118.50132720637752) bank23566 +23567 POINT(34.42413853068804 -118.16325252058448) bank23567 +23568 POINT(35.04102218765305 -119.1821969506471) bank23568 +23569 POINT(34.68071141024192 -118.68880398698231) bank23569 +23570 POINT(33.097286671157036 -117.60677884606672) bank23570 +23571 POINT(33.18171322042039 -118.21163128606332) bank23571 +23572 POINT(34.12287570576948 -118.57523838052239) bank23572 +23573 POINT(34.650980434537715 -117.48753551299882) bank23573 +23574 POINT(34.47492953901196 -118.3747553439039) bank23574 +23575 POINT(35.04095043314685 -119.1147562092551) bank23575 +23576 POINT(33.6382428159575 -118.90947113916509) bank23576 +23577 POINT(33.19044780591195 -117.91543687541812) bank23577 +23578 POINT(34.90520110479405 -117.39211197378582) bank23578 +23579 POINT(33.54507386212198 -119.08795109699875) bank23579 +23580 POINT(33.46047457661776 -118.47376404271672) bank23580 +23581 POINT(33.29316657728419 -119.07736370661597) bank23581 +23582 POINT(33.41986690398045 -117.5732421355121) bank23582 +23583 POINT(33.07404825725241 -117.37839773691589) bank23583 +23584 POINT(34.69206846978652 -118.84818078435771) bank23584 +23585 POINT(33.48110473064626 -117.75762194030783) bank23585 +23586 POINT(34.63378848177786 -117.80093960976109) bank23586 +23587 POINT(33.404696481100345 -118.69327475898828) bank23587 +23588 POINT(34.523974115434385 -117.82781184998802) bank23588 +23589 POINT(34.4422618859568 -118.55531806005483) bank23589 +23590 POINT(34.377567977016135 -118.24438802999074) bank23590 +23591 POINT(34.44200846249706 -118.62951192091126) bank23591 +23592 POINT(34.25943219105084 -117.69469325579169) bank23592 +23593 POINT(34.87376921092789 -118.3316328744184) bank23593 +23594 POINT(34.556505536250235 -117.42244819856614) bank23594 +23595 POINT(33.76904233151614 -118.35191193130224) bank23595 +23596 POINT(34.29877335075645 -118.19146638791345) bank23596 +23597 POINT(34.553760224304824 -118.54579631873493) bank23597 +23598 POINT(34.40730060238242 -118.3289086895098) bank23598 +23599 POINT(33.48505942759247 -118.92391989169055) bank23599 +23600 POINT(33.599344547887185 -118.83241868721306) bank23600 +23601 POINT(33.14940676022107 -118.79465926803375) bank23601 +23602 POINT(34.715345374139254 -118.94802567791973) bank23602 +23603 POINT(34.944283172365346 -117.26085766333287) bank23603 +23604 POINT(33.75349515993321 -117.99521067982943) bank23604 +23605 POINT(33.428691199626385 -118.18418662879273) bank23605 +23606 POINT(34.870731687867625 -119.04824557150332) bank23606 +23607 POINT(33.14470871170599 -119.19401894632614) bank23607 +23608 POINT(34.64515398962799 -118.93016582316461) bank23608 +23609 POINT(34.30830743173915 -117.71331174650622) bank23609 +23610 POINT(33.28490611084399 -118.3738870701107) bank23610 +23611 POINT(33.51827439221092 -118.11272438355147) bank23611 +23612 POINT(34.710507611363866 -117.4102543194788) bank23612 +23613 POINT(33.24925818475651 -118.85624243945604) bank23613 +23614 POINT(34.981319613413824 -117.49017253074963) bank23614 +23615 POINT(34.203043570852635 -118.93436310893705) bank23615 +23616 POINT(34.27591427955926 -118.39836343919607) bank23616 +23617 POINT(34.474412590166665 -117.88613234165636) bank23617 +23618 POINT(34.63515213829766 -117.62890554254324) bank23618 +23619 POINT(33.95803568421148 -118.92309318939147) bank23619 +23620 POINT(33.15824191574558 -117.57360378792733) bank23620 +23621 POINT(33.87558953159627 -117.25292813337187) bank23621 +23622 POINT(34.07641750666068 -117.99572452430193) bank23622 +23623 POINT(34.62455608936577 -117.70212206884482) bank23623 +23624 POINT(34.41098042298138 -117.7215734907718) bank23624 +23625 POINT(33.321518478375665 -118.50449279860138) bank23625 +23626 POINT(34.37345012592187 -119.14787124542924) bank23626 +23627 POINT(34.8516991948622 -118.86215990766229) bank23627 +23628 POINT(33.595251606681245 -117.55628082993921) bank23628 +23629 POINT(34.71608420032227 -117.75579923159873) bank23629 +23630 POINT(33.42251718526333 -117.41571626576349) bank23630 +23631 POINT(33.99446867643812 -117.98761528766056) bank23631 +23632 POINT(33.19451054038975 -117.43183096466453) bank23632 +23633 POINT(33.60365936691068 -119.10604160668353) bank23633 +23634 POINT(33.107130313090124 -118.23630938137183) bank23634 +23635 POINT(34.75425074250427 -117.38669915687045) bank23635 +23636 POINT(33.574365537863976 -118.87725227169692) bank23636 +23637 POINT(34.11423879461635 -117.77696238589203) bank23637 +23638 POINT(35.017959904042776 -119.2344710851099) bank23638 +23639 POINT(34.26765332666521 -117.48140908837927) bank23639 +23640 POINT(33.27592336856022 -117.3630344309269) bank23640 +23641 POINT(33.63490505312565 -117.29809774866216) bank23641 +23642 POINT(34.28950459905286 -117.76921910302624) bank23642 +23643 POINT(34.53368537893238 -119.07348578492767) bank23643 +23644 POINT(34.47737203550522 -117.67572125139968) bank23644 +23645 POINT(34.205321136991145 -117.51737099545456) bank23645 +23646 POINT(33.78868279346812 -117.56882723217372) bank23646 +23647 POINT(33.236099590169765 -118.28159704566849) bank23647 +23648 POINT(34.54735925404192 -119.16636155306047) bank23648 +23649 POINT(34.247804158843714 -118.00274018743548) bank23649 +23650 POINT(33.44465837167582 -119.14316473582637) bank23650 +23651 POINT(33.063150833313166 -117.49623096065848) bank23651 +23652 POINT(34.43466068925811 -118.45285492706972) bank23652 +23653 POINT(34.84590894884101 -118.39609461612368) bank23653 +23654 POINT(34.999269059172285 -117.35047375629945) bank23654 +23655 POINT(35.016278200144974 -117.46675269016995) bank23655 +23656 POINT(34.335023687798696 -118.46886212068733) bank23656 +23657 POINT(34.55608104285033 -119.08142945302771) bank23657 +23658 POINT(34.80157609348747 -117.84838271416375) bank23658 +23659 POINT(33.21626792179663 -118.82527803741043) bank23659 +23660 POINT(34.70577933089332 -117.89642166310891) bank23660 +23661 POINT(34.86003875215507 -117.31005764122683) bank23661 +23662 POINT(34.09506031575044 -117.7138631538874) bank23662 +23663 POINT(34.41461710798499 -119.08334855775001) bank23663 +23664 POINT(33.53035585676809 -118.08838275884496) bank23664 +23665 POINT(34.27491497427165 -118.18745407924493) bank23665 +23666 POINT(35.02213763675272 -117.50038547074482) bank23666 +23667 POINT(34.57327102444317 -117.40484665710518) bank23667 +23668 POINT(34.02010641699987 -118.35112786725371) bank23668 +23669 POINT(33.45458095951244 -118.54824461588369) bank23669 +23670 POINT(34.57781325914862 -118.43576642687314) bank23670 +23671 POINT(34.23798641385316 -117.35450436531704) bank23671 +23672 POINT(34.53269470215466 -118.77806105726646) bank23672 +23673 POINT(33.311891188534105 -118.84392389771624) bank23673 +23674 POINT(34.802126915992 -117.9429217103842) bank23674 +23675 POINT(34.12817858565271 -119.03617886457138) bank23675 +23676 POINT(34.81772181073439 -118.07212663660098) bank23676 +23677 POINT(33.27421767744067 -117.52261489438317) bank23677 +23678 POINT(33.554510620500686 -117.37950717175715) bank23678 +23679 POINT(35.015496859896444 -119.08221672807133) bank23679 +23680 POINT(34.965114738889945 -117.94086484217141) bank23680 +23681 POINT(33.73179521526208 -118.61068634165947) bank23681 +23682 POINT(33.23454758188521 -118.22063703573832) bank23682 +23683 POINT(34.574602654258236 -117.72770510764923) bank23683 +23684 POINT(33.30692622382989 -117.62093186150534) bank23684 +23685 POINT(35.05212038189727 -118.4071838795419) bank23685 +23686 POINT(35.03892120175191 -118.10477885958755) bank23686 +23687 POINT(33.58906994949042 -118.11888042952476) bank23687 +23688 POINT(33.26391105709593 -119.13750560551892) bank23688 +23689 POINT(34.819783869316474 -117.87485204334023) bank23689 +23690 POINT(34.09407976837222 -118.87606300694695) bank23690 +23691 POINT(35.00352567041505 -118.870539918177) bank23691 +23692 POINT(35.005814434649665 -117.6576137984453) bank23692 +23693 POINT(34.29117194881356 -118.01164776758164) bank23693 +23694 POINT(34.025764431157555 -117.74648777149797) bank23694 +23695 POINT(34.93732877968917 -119.066945075864) bank23695 +23696 POINT(33.65444084592277 -117.32859183482364) bank23696 +23697 POINT(33.59436339375282 -118.70762653233817) bank23697 +23698 POINT(33.89360004190199 -118.36435461808543) bank23698 +23699 POINT(33.85121881136865 -117.85563055957545) bank23699 +23700 POINT(34.29098616409012 -118.97311417553675) bank23700 +23701 POINT(34.56085595307421 -117.59712723860468) bank23701 +23702 POINT(33.15202953406356 -118.51918549465248) bank23702 +23703 POINT(34.910756195140934 -118.64742012419966) bank23703 +23704 POINT(34.41437929978102 -119.00537900918158) bank23704 +23705 POINT(34.311345665600356 -117.69374668121878) bank23705 +23706 POINT(34.609294560661226 -117.28461977982172) bank23706 +23707 POINT(34.43374734662271 -117.2444783853843) bank23707 +23708 POINT(35.04075787651553 -118.41909147855968) bank23708 +23709 POINT(33.49465983353341 -118.29533381659076) bank23709 +23710 POINT(34.33113201824109 -118.60541861801629) bank23710 +23711 POINT(34.13367313316753 -117.89750644984626) bank23711 +23712 POINT(33.87347063283791 -117.34892615070245) bank23712 +23713 POINT(33.97212681585898 -117.49278382123434) bank23713 +23714 POINT(33.28692033075482 -117.90872244731094) bank23714 +23715 POINT(33.90322322790304 -117.57538195353966) bank23715 +23716 POINT(34.16824213419318 -118.26027530196725) bank23716 +23717 POINT(33.38009203262045 -118.60722993052929) bank23717 +23718 POINT(34.71521553742121 -118.3858944989904) bank23718 +23719 POINT(33.696394149414665 -117.50235339965516) bank23719 +23720 POINT(34.82217457716105 -118.58677915762408) bank23720 +23721 POINT(33.877765703722055 -118.8567989042257) bank23721 +23722 POINT(34.752173961301196 -117.81683680617897) bank23722 +23723 POINT(33.528500449659184 -118.04645234976792) bank23723 +23724 POINT(33.12145597485526 -118.64888666493916) bank23724 +23725 POINT(33.91209812218379 -117.73484033065168) bank23725 +23726 POINT(33.05742212691515 -117.88614812917149) bank23726 +23727 POINT(33.26101667897375 -118.6857192006159) bank23727 +23728 POINT(34.2520088427991 -118.15183383588742) bank23728 +23729 POINT(33.5444637469327 -118.48300989863384) bank23729 +23730 POINT(33.76277688421148 -118.57906620368843) bank23730 +23731 POINT(33.67509504660575 -118.60874175537093) bank23731 +23732 POINT(33.703614928185246 -118.1859223202513) bank23732 +23733 POINT(34.38146293772485 -117.52410469765226) bank23733 +23734 POINT(34.75083517432019 -117.70121787810258) bank23734 +23735 POINT(34.15284965327869 -118.73947378219705) bank23735 +23736 POINT(33.101069792613 -119.19218070997094) bank23736 +23737 POINT(33.82620207420446 -118.36382448953196) bank23737 +23738 POINT(34.93571021042627 -117.30181916318317) bank23738 +23739 POINT(33.3046393832654 -119.1138674036061) bank23739 +23740 POINT(34.67274997448548 -118.6444347599607) bank23740 +23741 POINT(33.81383572761114 -119.12413770270643) bank23741 +23742 POINT(33.468376136267345 -117.77603017203272) bank23742 +23743 POINT(33.09979481243509 -119.1249020921412) bank23743 +23744 POINT(34.59398707354783 -119.22607037074584) bank23744 +23745 POINT(34.311716000625026 -118.42630241972229) bank23745 +23746 POINT(33.929585885817204 -118.62626189519713) bank23746 +23747 POINT(33.53845005646582 -119.22149515581305) bank23747 +23748 POINT(33.15053341106373 -117.80974512039863) bank23748 +23749 POINT(34.27803107423824 -119.00260068566872) bank23749 +23750 POINT(33.58539565309242 -118.43486480911774) bank23750 +23751 POINT(33.719558279530744 -118.87027656360198) bank23751 +23752 POINT(33.18149970379806 -118.41596268617074) bank23752 +23753 POINT(34.888910495043746 -117.36524281547838) bank23753 +23754 POINT(34.77760274970282 -118.64612883540616) bank23754 +23755 POINT(34.52523102844456 -119.09927334083608) bank23755 +23756 POINT(33.203093676029916 -118.572113189598) bank23756 +23757 POINT(34.71961264605291 -118.65975041377374) bank23757 +23758 POINT(33.125136903155855 -118.08861333747863) bank23758 +23759 POINT(34.312075090919215 -117.3915452926219) bank23759 +23760 POINT(33.197926296106004 -118.71731017755) bank23760 +23761 POINT(34.548648380184524 -118.91986413796148) bank23761 +23762 POINT(35.04656589168391 -118.70200413749967) bank23762 +23763 POINT(34.30172730015781 -118.15029277293398) bank23763 +23764 POINT(34.23746675719861 -117.81933929492564) bank23764 +23765 POINT(33.224505211628475 -118.5713380086605) bank23765 +23766 POINT(34.40166240206981 -117.52947853568836) bank23766 +23767 POINT(34.409128120668434 -118.90642158118426) bank23767 +23768 POINT(33.985863557383496 -118.68565645135654) bank23768 +23769 POINT(34.26618078130695 -118.05038412480415) bank23769 +23770 POINT(34.11320694251149 -117.99404385255725) bank23770 +23771 POINT(34.443556675780506 -118.29509510604136) bank23771 +23772 POINT(33.99659018508357 -118.56630336260699) bank23772 +23773 POINT(34.57324001616427 -117.592290414961) bank23773 +23774 POINT(33.94759567030579 -119.05082769872948) bank23774 +23775 POINT(33.64738155260974 -119.20533872880604) bank23775 +23776 POINT(33.42346127668006 -117.5029008575582) bank23776 +23777 POINT(33.99999152563146 -117.97108678161466) bank23777 +23778 POINT(34.721872737074534 -119.10833922397845) bank23778 +23779 POINT(34.26292792488577 -117.39963008952071) bank23779 +23780 POINT(34.96400086346135 -117.34423113128776) bank23780 +23781 POINT(33.55092827406779 -118.86510976477847) bank23781 +23782 POINT(33.72861251215046 -118.66188086181552) bank23782 +23783 POINT(33.24426833105287 -117.93947285262831) bank23783 +23784 POINT(33.15854941754884 -118.61102934354925) bank23784 +23785 POINT(34.2120549662985 -118.78541472473536) bank23785 +23786 POINT(33.44800206488882 -117.34977175718929) bank23786 +23787 POINT(33.33662818349277 -118.88068232235885) bank23787 +23788 POINT(33.445152062687555 -117.77278631369843) bank23788 +23789 POINT(34.714279749281495 -117.41413742643158) bank23789 +23790 POINT(33.60431774527126 -118.57108888227519) bank23790 +23791 POINT(33.558264051818064 -118.65816752835173) bank23791 +23792 POINT(33.306677744100135 -117.38481048504315) bank23792 +23793 POINT(34.043048411896685 -119.21909677657871) bank23793 +23794 POINT(33.32469492218295 -118.40583033874657) bank23794 +23795 POINT(33.725435897617196 -118.37680269431998) bank23795 +23796 POINT(33.49622504446005 -118.7236796759887) bank23796 +23797 POINT(33.68366707770922 -118.6287691458769) bank23797 +23798 POINT(33.125672459983015 -117.49437201784593) bank23798 +23799 POINT(34.16730571811478 -119.01123457244546) bank23799 +23800 POINT(34.6622439694429 -119.07975999674213) bank23800 +23801 POINT(33.32744547022253 -117.68792771824278) bank23801 +23802 POINT(34.94081065755175 -119.0096924507159) bank23802 +23803 POINT(34.573120014955684 -118.19571529918481) bank23803 +23804 POINT(33.1477118062584 -117.26263926654867) bank23804 +23805 POINT(34.81043817517969 -118.6860950608453) bank23805 +23806 POINT(33.707733773393656 -118.65454665734397) bank23806 +23807 POINT(34.02226368426785 -117.93251088028481) bank23807 +23808 POINT(33.79263079797678 -119.04422906194485) bank23808 +23809 POINT(34.17080104904047 -118.36972797101635) bank23809 +23810 POINT(33.71115244755966 -118.63308866940413) bank23810 +23811 POINT(34.25975828594825 -118.48737895164291) bank23811 +23812 POINT(33.445394347455135 -118.41872267376176) bank23812 +23813 POINT(34.537212154941244 -117.41770278424013) bank23813 +23814 POINT(34.56210608570835 -117.33497390701748) bank23814 +23815 POINT(34.81009950324008 -119.00114011102585) bank23815 +23816 POINT(33.64510293647527 -118.59957126022141) bank23816 +23817 POINT(33.179142080275646 -118.6748260524714) bank23817 +23818 POINT(33.93654074008157 -118.33660545558119) bank23818 +23819 POINT(33.9259438305406 -118.22105847352596) bank23819 +23820 POINT(33.69624457354776 -118.26801545041593) bank23820 +23821 POINT(33.46707085970711 -117.34882658502532) bank23821 +23822 POINT(34.98071024229181 -118.7029716487403) bank23822 +23823 POINT(33.63712994568676 -117.48901108930734) bank23823 +23824 POINT(33.39990543285493 -117.96399759351219) bank23824 +23825 POINT(34.34512134550557 -119.04451830737813) bank23825 +23826 POINT(33.3117526080932 -117.65895501889189) bank23826 +23827 POINT(33.852509672424105 -118.80180134361662) bank23827 +23828 POINT(33.1489055533495 -118.61770707873269) bank23828 +23829 POINT(34.047878728720434 -118.76561283446566) bank23829 +23830 POINT(33.69245239879589 -118.27044666666522) bank23830 +23831 POINT(34.56007833593755 -118.93985982648141) bank23831 +23832 POINT(33.34000856051621 -117.72518765802563) bank23832 +23833 POINT(35.0517784761144 -118.9045489657849) bank23833 +23834 POINT(34.864526641918715 -119.24271759164068) bank23834 +23835 POINT(34.531292040717915 -119.19721509019767) bank23835 +23836 POINT(33.122520223459276 -118.73810270109503) bank23836 +23837 POINT(33.108549035381856 -119.06445252582705) bank23837 +23838 POINT(34.05518237474672 -118.13356132875681) bank23838 +23839 POINT(34.436318684619096 -118.48300197084657) bank23839 +23840 POINT(34.9872506792836 -119.00055718733647) bank23840 +23841 POINT(33.695566674040066 -119.23914559481405) bank23841 +23842 POINT(33.665492935314816 -117.92304488443516) bank23842 +23843 POINT(33.19661897861988 -118.85926549376083) bank23843 +23844 POINT(33.57001839932834 -118.57261313762035) bank23844 +23845 POINT(34.440017492755466 -117.48416671269877) bank23845 +23846 POINT(33.66386585009988 -117.68754910807375) bank23846 +23847 POINT(33.68962968558626 -118.05798260855136) bank23847 +23848 POINT(35.020948364683655 -117.38388329012575) bank23848 +23849 POINT(33.52096087418584 -118.52709277945945) bank23849 +23850 POINT(33.41587203550869 -117.78082268004984) bank23850 +23851 POINT(33.34520940715092 -118.69711444184323) bank23851 +23852 POINT(34.47329005629333 -117.62965685518397) bank23852 +23853 POINT(34.0289220357338 -119.17679149547166) bank23853 +23854 POINT(34.48129991203319 -118.63555910069034) bank23854 +23855 POINT(33.31173050477206 -117.97418049119631) bank23855 +23856 POINT(33.32576886758092 -119.16213533425973) bank23856 +23857 POINT(33.49507122022525 -117.96892270639165) bank23857 +23858 POINT(33.86341838932578 -118.88192626956281) bank23858 +23859 POINT(33.36167592357943 -117.95538138287206) bank23859 +23860 POINT(34.793546978074886 -118.01983221070834) bank23860 +23861 POINT(33.564013346664126 -119.115392349053) bank23861 +23862 POINT(34.24235121460574 -118.8814834075253) bank23862 +23863 POINT(34.5112797534879 -117.29559084771113) bank23863 +23864 POINT(34.08857672622221 -118.5537044465152) bank23864 +23865 POINT(33.97266695168967 -117.39646042902676) bank23865 +23866 POINT(34.96424778190966 -117.64517938319169) bank23866 +23867 POINT(33.3345609128915 -118.83723820487067) bank23867 +23868 POINT(33.943282357877266 -117.40780367689085) bank23868 +23869 POINT(34.38356204991309 -118.45156744040202) bank23869 +23870 POINT(34.236722347500745 -118.54799439408747) bank23870 +23871 POINT(34.21469611450185 -118.54724811494812) bank23871 +23872 POINT(34.35017640116091 -118.0620495276174) bank23872 +23873 POINT(34.43766238377436 -117.42620774032608) bank23873 +23874 POINT(33.13841373778825 -118.89860151567491) bank23874 +23875 POINT(33.17912108635582 -118.8288703637942) bank23875 +23876 POINT(33.60394902057029 -118.64609746096146) bank23876 +23877 POINT(34.945998727918834 -117.42350171810507) bank23877 +23878 POINT(34.746747104764566 -119.11689380450869) bank23878 +23879 POINT(33.93528547717391 -118.16048424319321) bank23879 +23880 POINT(34.165856955819145 -118.35531808781748) bank23880 +23881 POINT(33.85169318076584 -119.0349736897426) bank23881 +23882 POINT(33.18020959294153 -118.4804430602799) bank23882 +23883 POINT(34.279407472631696 -118.43196125594727) bank23883 +23884 POINT(33.97742721314676 -119.15535221813074) bank23884 +23885 POINT(33.582544166345436 -118.15323997485888) bank23885 +23886 POINT(33.07880658987593 -119.14521962660893) bank23886 +23887 POINT(34.92511361582537 -117.68550561347912) bank23887 +23888 POINT(34.919790879698574 -117.42878692147889) bank23888 +23889 POINT(33.26550102395424 -118.71797572371007) bank23889 +23890 POINT(34.30876875208294 -118.83563651880972) bank23890 +23891 POINT(34.25510383727976 -118.83223235457275) bank23891 +23892 POINT(35.00310355524851 -117.33126024383976) bank23892 +23893 POINT(34.322436771594106 -118.85567013699465) bank23893 +23894 POINT(33.75018402458236 -117.4923247348172) bank23894 +23895 POINT(35.0511498100183 -118.97235461987665) bank23895 +23896 POINT(33.57639292129072 -118.14870593223664) bank23896 +23897 POINT(34.52439883605395 -118.53205943447915) bank23897 +23898 POINT(34.7953639767896 -117.65624059248537) bank23898 +23899 POINT(34.04458900253925 -117.78141607938097) bank23899 +23900 POINT(33.16557134646833 -117.28270278709859) bank23900 +23901 POINT(33.18612560211795 -118.93156100108608) bank23901 +23902 POINT(34.11108887790667 -119.2012437621251) bank23902 +23903 POINT(33.62486502400382 -118.12301017526174) bank23903 +23904 POINT(34.819757386760806 -118.2410891560929) bank23904 +23905 POINT(33.50445900194408 -119.12897911178966) bank23905 +23906 POINT(33.66375219961852 -118.64536278138301) bank23906 +23907 POINT(35.010369783582206 -118.3116204523865) bank23907 +23908 POINT(33.32610926310289 -118.28307200162679) bank23908 +23909 POINT(34.825115025824346 -118.03662986321798) bank23909 +23910 POINT(34.05341429022335 -117.93866927293715) bank23910 +23911 POINT(33.72100619640005 -117.97839322886145) bank23911 +23912 POINT(33.51766050189532 -117.51733561889152) bank23912 +23913 POINT(33.44093093670899 -118.6058777355919) bank23913 +23914 POINT(34.094680352001944 -119.02241373058092) bank23914 +23915 POINT(33.96992023440898 -118.70982509237466) bank23915 +23916 POINT(34.88771103972093 -117.42880459108783) bank23916 +23917 POINT(34.706611907383035 -118.91255225557997) bank23917 +23918 POINT(33.10410765338916 -117.49968186450313) bank23918 +23919 POINT(34.975860923729975 -117.31198052304354) bank23919 +23920 POINT(34.26283151055755 -118.45340673302047) bank23920 +23921 POINT(33.828563313390184 -118.33076915180568) bank23921 +23922 POINT(33.62333108034009 -118.60178100052204) bank23922 +23923 POINT(33.94401309443698 -119.18222683926535) bank23923 +23924 POINT(34.606023196169225 -118.44486538670476) bank23924 +23925 POINT(34.333404904591355 -118.86991381673751) bank23925 +23926 POINT(33.82549828916071 -118.84778794835368) bank23926 +23927 POINT(34.98633046804944 -117.43282767899491) bank23927 +23928 POINT(33.110174331507785 -117.92817088870598) bank23928 +23929 POINT(34.80103841596495 -117.87419306131265) bank23929 +23930 POINT(33.24987673918559 -117.4477618023709) bank23930 +23931 POINT(34.036777460702154 -117.52652105600949) bank23931 +23932 POINT(33.10292864134938 -118.42035903790894) bank23932 +23933 POINT(33.613691286296316 -118.69195194141585) bank23933 +23934 POINT(33.67271393144848 -117.30912998441504) bank23934 +23935 POINT(34.428594590701024 -118.53969049681778) bank23935 +23936 POINT(34.424594800611644 -118.62627110376698) bank23936 +23937 POINT(34.97954238254701 -118.44962021800787) bank23937 +23938 POINT(34.50791487520894 -118.366353544975) bank23938 +23939 POINT(33.164856545741245 -117.86325068753044) bank23939 +23940 POINT(34.51313438888926 -119.02184200320147) bank23940 +23941 POINT(34.421660440418385 -117.94972328386139) bank23941 +23942 POINT(34.034919615449795 -118.20495008187899) bank23942 +23943 POINT(33.225890301187455 -118.57769761443309) bank23943 +23944 POINT(33.418878255940285 -118.50086320842355) bank23944 +23945 POINT(33.33582442383987 -117.99212279831622) bank23945 +23946 POINT(34.20542454692596 -117.30269024730654) bank23946 +23947 POINT(33.48810773407893 -117.45857113080717) bank23947 +23948 POINT(33.766419732097454 -117.95442060627708) bank23948 +23949 POINT(34.50390803746943 -117.72675673627415) bank23949 +23950 POINT(33.99075727768567 -117.95703676906248) bank23950 +23951 POINT(33.22028868720978 -118.62794809496643) bank23951 +23952 POINT(34.18655322780915 -117.29939113269135) bank23952 +23953 POINT(33.19782254942623 -118.79884659122685) bank23953 +23954 POINT(33.24187349982124 -117.30550662129859) bank23954 +23955 POINT(33.5821611382683 -118.6513333646734) bank23955 +23956 POINT(33.70603871749025 -118.26866732726504) bank23956 +23957 POINT(33.50285751061385 -117.3566022795775) bank23957 +23958 POINT(34.31128853826899 -119.2015182623259) bank23958 +23959 POINT(34.08643007514889 -118.21784721555102) bank23959 +23960 POINT(34.59154911960465 -118.0722288001361) bank23960 +23961 POINT(33.98687093087326 -117.59367724513787) bank23961 +23962 POINT(34.38933968390628 -118.73733891502734) bank23962 +23963 POINT(34.65948449433391 -118.41182522563457) bank23963 +23964 POINT(33.78687166836485 -118.1041457816394) bank23964 +23965 POINT(33.43708286867831 -117.39370454720405) bank23965 +23966 POINT(33.99799328856096 -118.67465156376834) bank23966 +23967 POINT(34.2306750087851 -117.34722154296813) bank23967 +23968 POINT(33.827322558684074 -118.81500309354608) bank23968 +23969 POINT(33.92992132259952 -118.95745629004449) bank23969 +23970 POINT(33.13264722300206 -117.76521735688954) bank23970 +23971 POINT(33.24876217270336 -117.79154560803751) bank23971 +23972 POINT(33.61545169929072 -117.27140834440655) bank23972 +23973 POINT(34.34993644155471 -117.49026048950094) bank23973 +23974 POINT(34.376442745651985 -118.97837350388204) bank23974 +23975 POINT(33.5536559428075 -117.52679034470948) bank23975 +23976 POINT(34.883926052954585 -117.72819561115485) bank23976 +23977 POINT(34.13901390401423 -118.81540417364592) bank23977 +23978 POINT(33.776478349190604 -117.59455331697278) bank23978 +23979 POINT(34.0015725826653 -118.04658994063959) bank23979 +23980 POINT(33.48762511740741 -118.38391443515751) bank23980 +23981 POINT(33.052284854474145 -118.69876358666365) bank23981 +23982 POINT(34.279804760738706 -119.12826347897034) bank23982 +23983 POINT(34.047691470319194 -119.15698507592002) bank23983 +23984 POINT(33.308147618564625 -119.14169538108926) bank23984 +23985 POINT(34.74136354699232 -117.65521736600438) bank23985 +23986 POINT(34.97866658207225 -118.76799017035555) bank23986 +23987 POINT(34.80604793224795 -117.72113202942111) bank23987 +23988 POINT(34.0405335056807 -117.49949764822694) bank23988 +23989 POINT(34.676702468648884 -117.9461307992253) bank23989 +23990 POINT(33.837891739295785 -119.1332481685209) bank23990 +23991 POINT(33.16520475084032 -118.17093872571591) bank23991 +23992 POINT(33.44509669085272 -117.63062806600595) bank23992 +23993 POINT(34.34124611217382 -117.30018482814909) bank23993 +23994 POINT(34.396243957244664 -117.60432849217139) bank23994 +23995 POINT(34.92199368071389 -118.0346078150645) bank23995 +23996 POINT(33.18370808052139 -117.70205966832441) bank23996 +23997 POINT(34.04776470942958 -117.72452474432933) bank23997 +23998 POINT(33.801602672062906 -118.92485162319072) bank23998 +23999 POINT(34.16821216979025 -118.78397247326124) bank23999 +24000 POINT(34.876031750363765 -118.09843946212568) bank24000 +24001 POINT(33.588556492509355 -118.8058786450726) bank24001 +24002 POINT(33.07683600683121 -118.68777211442352) bank24002 +24003 POINT(33.314517178511856 -117.38929222951948) bank24003 +24004 POINT(33.917472676327776 -118.71211250092433) bank24004 +24005 POINT(33.458734577071745 -118.11287092937744) bank24005 +24006 POINT(33.074327103040744 -118.00936489838998) bank24006 +24007 POINT(33.8681652720844 -119.01859357402851) bank24007 +24008 POINT(33.18662843139782 -117.69673410707873) bank24008 +24009 POINT(34.68100067198748 -118.46894529744038) bank24009 +24010 POINT(34.572074951457445 -118.59333539090366) bank24010 +24011 POINT(34.754429497407656 -118.75562699145675) bank24011 +24012 POINT(33.43850832737425 -118.89594725504261) bank24012 +24013 POINT(34.30510329800085 -119.14271015508007) bank24013 +24014 POINT(33.556821937794965 -118.06642748160272) bank24014 +24015 POINT(33.30610192377609 -117.8318547663766) bank24015 +24016 POINT(34.28809346633364 -119.03675494733939) bank24016 +24017 POINT(34.87022130113057 -118.62412901784234) bank24017 +24018 POINT(34.14382366030099 -119.08351486846897) bank24018 +24019 POINT(34.72245982751998 -118.91192599244096) bank24019 +24020 POINT(34.2219500084831 -119.12010291424733) bank24020 +24021 POINT(33.51552229849977 -118.27681160111477) bank24021 +24022 POINT(34.86408615507475 -118.85581656344743) bank24022 +24023 POINT(34.22416756623829 -118.59567589974948) bank24023 +24024 POINT(34.10554083872338 -118.11674429998679) bank24024 +24025 POINT(34.691027051152055 -118.16951665343669) bank24025 +24026 POINT(34.750731781787636 -119.18341503601461) bank24026 +24027 POINT(34.005917053407636 -117.30257059619949) bank24027 +24028 POINT(34.566429898764206 -119.05565661514926) bank24028 +24029 POINT(34.698602049857634 -117.57657319415014) bank24029 +24030 POINT(34.45889598630113 -118.33344956121958) bank24030 +24031 POINT(33.32743933395682 -118.9548964986882) bank24031 +24032 POINT(33.10126914125454 -117.9296020493279) bank24032 +24033 POINT(33.11433630369085 -117.5058141142523) bank24033 +24034 POINT(33.28763743587509 -118.24337447223319) bank24034 +24035 POINT(33.30486799469442 -117.28285573119506) bank24035 +24036 POINT(33.3570867099034 -118.5876586247697) bank24036 +24037 POINT(34.235785972847545 -117.34909259269043) bank24037 +24038 POINT(34.006687406568226 -117.74106939820881) bank24038 +24039 POINT(33.51463595973729 -118.84539312770542) bank24039 +24040 POINT(33.64087538340632 -118.01661644786154) bank24040 +24041 POINT(33.770264033909044 -117.61673549725869) bank24041 +24042 POINT(33.09868763002998 -117.30358517390006) bank24042 +24043 POINT(33.432812215342246 -117.96290354187668) bank24043 +24044 POINT(33.92711997897611 -117.53117208477988) bank24044 +24045 POINT(33.24696617561999 -118.27658615696028) bank24045 +24046 POINT(33.54666842951562 -117.39070134521221) bank24046 +24047 POINT(33.495701356517664 -118.19487215604364) bank24047 +24048 POINT(34.86606396984507 -118.03262866816611) bank24048 +24049 POINT(33.651254462730385 -118.42467876613622) bank24049 +24050 POINT(33.55458373629091 -118.42937887364037) bank24050 +24051 POINT(34.05204370249974 -118.19354407506587) bank24051 +24052 POINT(35.05119753386878 -118.77523866772452) bank24052 +24053 POINT(34.99485652806061 -118.36292991375451) bank24053 +24054 POINT(34.35045889763115 -119.00734445180606) bank24054 +24055 POINT(33.74620231962525 -117.94613237381938) bank24055 +24056 POINT(33.218004178584856 -117.86652681631475) bank24056 +24057 POINT(33.280267273855735 -118.75642577371276) bank24057 +24058 POINT(33.6340672162583 -117.50118778446858) bank24058 +24059 POINT(34.79877698740722 -117.81991743377806) bank24059 +24060 POINT(35.002524977142 -117.27035972693514) bank24060 +24061 POINT(33.241093756409924 -118.54543086992459) bank24061 +24062 POINT(33.927927076572786 -119.16407743109569) bank24062 +24063 POINT(34.56271297540934 -117.58356856179523) bank24063 +24064 POINT(34.278675096787936 -117.98330861307134) bank24064 +24065 POINT(33.38661888166829 -117.30734897959942) bank24065 +24066 POINT(34.961594799618574 -118.24658980497323) bank24066 +24067 POINT(34.27840930500878 -118.97263025496437) bank24067 +24068 POINT(33.11848956891421 -117.36023581616722) bank24068 +24069 POINT(34.12702586275131 -118.0372272929853) bank24069 +24070 POINT(34.969062785054355 -118.56391279676482) bank24070 +24071 POINT(34.036943514361084 -117.82331177913102) bank24071 +24072 POINT(33.31797042040997 -118.95389078775139) bank24072 +24073 POINT(33.77326969572348 -118.31233828525478) bank24073 +24074 POINT(33.50872404829561 -117.63142358204283) bank24074 +24075 POINT(34.317645062725234 -118.62801005304217) bank24075 +24076 POINT(35.003016658406224 -117.79061324096662) bank24076 +24077 POINT(33.611875909730806 -118.93301879753122) bank24077 +24078 POINT(34.708969498936234 -118.6606628493181) bank24078 +24079 POINT(34.58932828033725 -117.41087561849125) bank24079 +24080 POINT(34.920922252327365 -118.17231478996926) bank24080 +24081 POINT(34.61366878551137 -117.64267104740894) bank24081 +24082 POINT(35.02016787261127 -118.28906401697445) bank24082 +24083 POINT(34.680897985916346 -117.30875510050272) bank24083 +24084 POINT(34.00714525309632 -117.56322568400847) bank24084 +24085 POINT(34.92153853395487 -117.45787078451895) bank24085 +24086 POINT(33.23814015685038 -118.52360053223867) bank24086 +24087 POINT(33.42650038653679 -117.53686498335964) bank24087 +24088 POINT(33.77825488961269 -117.89388846607402) bank24088 +24089 POINT(33.582899926654456 -117.84698598013505) bank24089 +24090 POINT(34.83502676640247 -119.04189118331173) bank24090 +24091 POINT(33.30119051547534 -117.68019353010634) bank24091 +24092 POINT(34.143015419495235 -118.56460711811815) bank24092 +24093 POINT(35.02019719085957 -117.84238296264415) bank24093 +24094 POINT(34.48063203872746 -118.05080444337878) bank24094 +24095 POINT(33.51509309585366 -117.85516535810297) bank24095 +24096 POINT(34.565853306743506 -118.31632275745032) bank24096 +24097 POINT(34.59092085408389 -119.21076783782536) bank24097 +24098 POINT(33.12792907228306 -117.99298026053157) bank24098 +24099 POINT(34.84695306518861 -118.59815901874077) bank24099 +24100 POINT(33.29904783884119 -117.51710405235166) bank24100 +24101 POINT(33.092986078456676 -118.76516096920284) bank24101 +24102 POINT(35.04154967293839 -118.44997146565161) bank24102 +24103 POINT(34.12760998471104 -118.87865802440099) bank24103 +24104 POINT(33.92595967774975 -119.03367900835816) bank24104 +24105 POINT(33.944122125809734 -117.4928057870623) bank24105 +24106 POINT(33.83930385992918 -118.94617231467926) bank24106 +24107 POINT(33.497121528867766 -118.0645365104506) bank24107 +24108 POINT(33.226844861293934 -117.60209075245264) bank24108 +24109 POINT(33.062549839452345 -118.9182640972273) bank24109 +24110 POINT(34.21424203312177 -118.33184626336784) bank24110 +24111 POINT(33.99373538842054 -118.4982699419492) bank24111 +24112 POINT(34.9115708632509 -118.84725241723555) bank24112 +24113 POINT(34.59846825375279 -118.84071353864505) bank24113 +24114 POINT(33.652159469572815 -117.94953026995354) bank24114 +24115 POINT(34.487577049376206 -118.31463259241922) bank24115 +24116 POINT(33.52428324025278 -117.49363849290744) bank24116 +24117 POINT(34.92109962695305 -119.19223989299802) bank24117 +24118 POINT(33.73331661637545 -118.29517347078733) bank24118 +24119 POINT(34.997444341872864 -118.84542905362953) bank24119 +24120 POINT(33.40747341652941 -117.35463191456014) bank24120 +24121 POINT(34.121898871039306 -118.38589608287846) bank24121 +24122 POINT(33.98592038094187 -117.29516002166712) bank24122 +24123 POINT(33.883495874152494 -117.79226008909232) bank24123 +24124 POINT(33.798921086108194 -117.9120266823813) bank24124 +24125 POINT(33.09093255436999 -117.94590018483274) bank24125 +24126 POINT(33.78028055325158 -118.4019413568977) bank24126 +24127 POINT(34.166917802648555 -117.76212301874239) bank24127 +24128 POINT(34.31183560897921 -117.96379941864882) bank24128 +24129 POINT(33.56471470054083 -118.54303439643424) bank24129 +24130 POINT(33.12087585650804 -119.03624373553866) bank24130 +24131 POINT(33.82758244169616 -118.63472196701319) bank24131 +24132 POINT(34.65183550073486 -118.4568977994508) bank24132 +24133 POINT(34.435614386090755 -118.6157561472314) bank24133 +24134 POINT(34.786447304018374 -117.61104174677291) bank24134 +24135 POINT(33.68420575069775 -117.6094131152133) bank24135 +24136 POINT(33.68380433675234 -117.53827059107283) bank24136 +24137 POINT(34.56764721501677 -118.682149397783) bank24137 +24138 POINT(34.29106634971974 -118.58280023877668) bank24138 +24139 POINT(33.249769874113326 -117.77873988248679) bank24139 +24140 POINT(34.61675721921611 -118.24208053461687) bank24140 +24141 POINT(34.61518411398041 -117.7029681946501) bank24141 +24142 POINT(34.77296500000546 -117.72528744940179) bank24142 +24143 POINT(33.498812850642295 -117.8193380242255) bank24143 +24144 POINT(34.33841739772132 -117.95226783707723) bank24144 +24145 POINT(34.0176197300431 -117.9561008374084) bank24145 +24146 POINT(33.35008193198234 -119.17250497419319) bank24146 +24147 POINT(34.47546480568351 -119.16445901746084) bank24147 +24148 POINT(33.58664117620162 -118.72256804364027) bank24148 +24149 POINT(34.806573555447684 -117.79940006442084) bank24149 +24150 POINT(33.714887058946346 -117.73179738417869) bank24150 +24151 POINT(34.1559710358567 -118.5438783189084) bank24151 +24152 POINT(34.09879089879809 -118.36832704444957) bank24152 +24153 POINT(34.93378509879078 -117.82429511987307) bank24153 +24154 POINT(34.732547167325905 -117.79204933715424) bank24154 +24155 POINT(34.75962985330196 -118.4706606761869) bank24155 +24156 POINT(33.19967866495498 -117.45647325580867) bank24156 +24157 POINT(34.40348563708371 -117.43306665653351) bank24157 +24158 POINT(33.09845963842084 -118.21377249090978) bank24158 +24159 POINT(34.188472783304206 -117.7143285014407) bank24159 +24160 POINT(35.046313002751425 -118.16780804544602) bank24160 +24161 POINT(34.3903273803748 -118.70525572330246) bank24161 +24162 POINT(34.18325779559301 -117.65194273414319) bank24162 +24163 POINT(33.51591753094796 -117.8534417093658) bank24163 +24164 POINT(33.940193110630105 -118.87053269868771) bank24164 +24165 POINT(33.991407611830034 -118.40186064477805) bank24165 +24166 POINT(34.632859126120465 -118.65422639658505) bank24166 +24167 POINT(34.20191988863342 -118.95195979641728) bank24167 +24168 POINT(33.71116760587389 -118.1044059905491) bank24168 +24169 POINT(33.52739531553602 -118.99481902552516) bank24169 +24170 POINT(33.5609014866675 -117.36942383748395) bank24170 +24171 POINT(34.98179656590498 -118.89453918478318) bank24171 +24172 POINT(34.86337624682386 -118.19599013105329) bank24172 +24173 POINT(34.69372821284591 -117.5010686623091) bank24173 +24174 POINT(34.381413649277775 -118.97492695956002) bank24174 +24175 POINT(34.83268404429121 -118.70985003365796) bank24175 +24176 POINT(33.81920276034797 -118.97696998842984) bank24176 +24177 POINT(34.02579825071432 -118.77381558704577) bank24177 +24178 POINT(33.833088456026246 -117.76342029855275) bank24178 +24179 POINT(34.228306366340504 -117.51022573413712) bank24179 +24180 POINT(34.93336801722896 -119.17526305578046) bank24180 +24181 POINT(33.470565679152095 -118.7792249524512) bank24181 +24182 POINT(34.13434226573371 -117.36842988627616) bank24182 +24183 POINT(34.714833085552314 -117.87684549538602) bank24183 +24184 POINT(34.88993680201585 -117.9494569711289) bank24184 +24185 POINT(33.433932440596614 -117.72374385194537) bank24185 +24186 POINT(33.74261322197826 -118.96167403544047) bank24186 +24187 POINT(33.57272570612341 -117.90256290224454) bank24187 +24188 POINT(34.39947649286336 -117.69556164638901) bank24188 +24189 POINT(33.16024627037888 -119.13868864749486) bank24189 +24190 POINT(34.88585286102091 -118.91751268971433) bank24190 +24191 POINT(33.98916917216498 -118.18820915051931) bank24191 +24192 POINT(33.53614597732353 -119.13790920877861) bank24192 +24193 POINT(34.1280077688573 -119.1777434700094) bank24193 +24194 POINT(34.4772551077913 -118.83856247794097) bank24194 +24195 POINT(33.68429491564962 -119.01945647975242) bank24195 +24196 POINT(34.81026439240974 -117.25638358678313) bank24196 +24197 POINT(34.19493836024237 -118.8566884950194) bank24197 +24198 POINT(34.52307172505022 -117.375657947634) bank24198 +24199 POINT(34.932882074002485 -117.3967038279729) bank24199 +24200 POINT(34.416185015413184 -118.7804317923464) bank24200 +24201 POINT(33.365527596724476 -118.79842311742495) bank24201 +24202 POINT(33.34332060906543 -118.19649856000504) bank24202 +24203 POINT(33.826371629797144 -117.72110397947691) bank24203 +24204 POINT(33.73930723747434 -118.56418861744731) bank24204 +24205 POINT(33.30733919720687 -118.22309471793339) bank24205 +24206 POINT(33.53468354078037 -118.55776131420976) bank24206 +24207 POINT(33.55840211060898 -118.61685272118541) bank24207 +24208 POINT(34.99047481540177 -117.25948129941033) bank24208 +24209 POINT(34.425255962230096 -118.5681630759254) bank24209 +24210 POINT(33.48147549775163 -118.14929635684312) bank24210 +24211 POINT(34.962490164717146 -118.82546114838762) bank24211 +24212 POINT(33.39382233291087 -118.08592624411051) bank24212 +24213 POINT(33.0931186729689 -117.73758241848536) bank24213 +24214 POINT(33.477878573049765 -117.45294964160922) bank24214 +24215 POINT(34.834046424680906 -117.43042590452347) bank24215 +24216 POINT(34.72392041859131 -119.0025182209444) bank24216 +24217 POINT(33.724111427455924 -119.11157932908114) bank24217 +24218 POINT(34.6313950416738 -118.97038869621865) bank24218 +24219 POINT(33.637762589237965 -117.621853493418) bank24219 +24220 POINT(33.59923744643209 -118.13392372199701) bank24220 +24221 POINT(34.695146357673316 -118.01220744764011) bank24221 +24222 POINT(34.45524155383258 -119.05189910732814) bank24222 +24223 POINT(34.10535360161817 -118.80063245557776) bank24223 +24224 POINT(33.501598239938104 -118.8400126221757) bank24224 +24225 POINT(33.19459588080514 -117.65019642701843) bank24225 +24226 POINT(33.94842588317012 -119.1882080477671) bank24226 +24227 POINT(34.37753151155662 -118.93062884822275) bank24227 +24228 POINT(34.72668586547706 -119.13193998491596) bank24228 +24229 POINT(34.442718782986546 -117.59433313289094) bank24229 +24230 POINT(34.35702358057512 -117.66722604165018) bank24230 +24231 POINT(33.89195482068294 -117.54743891009804) bank24231 +24232 POINT(34.41313069054782 -118.87254965356227) bank24232 +24233 POINT(33.848128413343886 -119.03673133819588) bank24233 +24234 POINT(34.344578886307815 -119.0706982363934) bank24234 +24235 POINT(34.1830328939725 -118.61646877034245) bank24235 +24236 POINT(33.71961848741821 -117.46803844632207) bank24236 +24237 POINT(34.39773547807702 -117.617915412529) bank24237 +24238 POINT(33.922443334019185 -119.02413237831543) bank24238 +24239 POINT(34.69218334659716 -118.78503527573899) bank24239 +24240 POINT(33.621939137889036 -117.55756825585271) bank24240 +24241 POINT(33.162519940032865 -118.47227560120152) bank24241 +24242 POINT(33.7144844933896 -118.0920107985796) bank24242 +24243 POINT(34.80958489605213 -117.55624650216332) bank24243 +24244 POINT(34.81338767272302 -117.49934168149434) bank24244 +24245 POINT(34.244512488333825 -117.82992558825227) bank24245 +24246 POINT(33.562403821011365 -119.2376525214794) bank24246 +24247 POINT(34.013039576849195 -118.82577335559687) bank24247 +24248 POINT(34.99307416757767 -119.2322226493717) bank24248 +24249 POINT(34.69426990847034 -118.0214968701662) bank24249 +24250 POINT(34.2194735958724 -117.70402393045262) bank24250 +24251 POINT(34.94769921884431 -117.66320010607711) bank24251 +24252 POINT(33.50081305888261 -117.60388273356571) bank24252 +24253 POINT(33.7753738567663 -118.32181127165104) bank24253 +24254 POINT(33.07138168920751 -118.16359029268797) bank24254 +24255 POINT(33.55027620688133 -117.83933971401844) bank24255 +24256 POINT(35.025909567542236 -118.22785964713594) bank24256 +24257 POINT(34.34368111130562 -118.01721460226825) bank24257 +24258 POINT(33.95707778276965 -118.09548247132558) bank24258 +24259 POINT(34.99800917564025 -118.12562786678967) bank24259 +24260 POINT(34.588090105002365 -118.41243793445148) bank24260 +24261 POINT(33.62064591121939 -118.48129063214003) bank24261 +24262 POINT(34.5051379921031 -118.57860674248863) bank24262 +24263 POINT(33.94994484319425 -118.74216810203583) bank24263 +24264 POINT(34.73058637125387 -118.5071948980838) bank24264 +24265 POINT(34.43877432007381 -118.79433580317267) bank24265 +24266 POINT(34.02141087453186 -117.5141784844561) bank24266 +24267 POINT(33.88979337747507 -117.53542675014332) bank24267 +24268 POINT(34.11652641667225 -117.90979799122329) bank24268 +24269 POINT(33.34076322890148 -118.57613680866854) bank24269 +24270 POINT(34.60919360659848 -119.10179462752617) bank24270 +24271 POINT(33.175749304964405 -118.75772722845156) bank24271 +24272 POINT(33.72171704635951 -117.46023235851946) bank24272 +24273 POINT(34.9820249254533 -118.57718460265257) bank24273 +24274 POINT(33.17011746286358 -118.68837447342258) bank24274 +24275 POINT(33.998774231927044 -117.43282609719026) bank24275 +24276 POINT(34.77859550553787 -118.9171772051737) bank24276 +24277 POINT(35.04339837383755 -117.87510066241673) bank24277 +24278 POINT(34.22761490667956 -119.145387774577) bank24278 +24279 POINT(33.992511220431005 -117.85220412948019) bank24279 +24280 POINT(34.41186718749798 -117.52983008969551) bank24280 +24281 POINT(34.32379043712778 -118.10043583941737) bank24281 +24282 POINT(34.90647283144255 -117.75754927558651) bank24282 +24283 POINT(34.21513191901926 -118.43350199255131) bank24283 +24284 POINT(33.65357846092164 -117.48574968125594) bank24284 +24285 POINT(33.77392252211448 -118.51048350085411) bank24285 +24286 POINT(34.90764081366033 -117.304225986333) bank24286 +24287 POINT(34.86799554873598 -117.4998717662533) bank24287 +24288 POINT(34.454032993401725 -118.94652165894018) bank24288 +24289 POINT(33.62802083068448 -118.51850599196143) bank24289 +24290 POINT(33.116469597912605 -119.16931148236979) bank24290 +24291 POINT(33.57222985537866 -118.59293846136272) bank24291 +24292 POINT(33.06646570384272 -117.53500487166099) bank24292 +24293 POINT(34.610372003066956 -117.92949796948771) bank24293 +24294 POINT(34.001777021256686 -117.93713068247169) bank24294 +24295 POINT(33.47151558381996 -117.6059406932809) bank24295 +24296 POINT(34.68533339530774 -117.46446339903181) bank24296 +24297 POINT(33.51291604663656 -118.32776027431765) bank24297 +24298 POINT(34.3990493980809 -118.90202360444056) bank24298 +24299 POINT(34.41502671548164 -117.70367758987335) bank24299 +24300 POINT(34.01910249826865 -118.78511533951833) bank24300 +24301 POINT(33.113178105109256 -118.52446919570215) bank24301 +24302 POINT(33.12050967932646 -117.25549103620374) bank24302 +24303 POINT(33.08262745377545 -118.47253998830443) bank24303 +24304 POINT(34.619922803459666 -117.46474950741666) bank24304 +24305 POINT(34.207727399156155 -118.65532716094556) bank24305 +24306 POINT(34.05468223973858 -117.95381852240227) bank24306 +24307 POINT(34.63196834781731 -118.40711343394273) bank24307 +24308 POINT(33.10972112577554 -119.12611494529472) bank24308 +24309 POINT(33.9046114706217 -117.46814846932809) bank24309 +24310 POINT(34.91799566859528 -118.19974630974787) bank24310 +24311 POINT(33.947528125102856 -117.86746327995829) bank24311 +24312 POINT(34.99922771669373 -118.27849487454094) bank24312 +24313 POINT(34.67102587254392 -117.98841071275518) bank24313 +24314 POINT(34.98843391657188 -118.73148744889967) bank24314 +24315 POINT(34.762340469446364 -117.33268090593546) bank24315 +24316 POINT(33.37241440765491 -117.86440828631548) bank24316 +24317 POINT(34.89658119507743 -117.71482141674224) bank24317 +24318 POINT(34.03982701358113 -118.64088595522541) bank24318 +24319 POINT(34.61911443520628 -117.9961623733713) bank24319 +24320 POINT(33.935489629232045 -119.07748329911338) bank24320 +24321 POINT(34.42411630970155 -118.78865686079516) bank24321 +24322 POINT(33.804260460102455 -118.5583786870076) bank24322 +24323 POINT(33.78929009181478 -117.60360169883296) bank24323 +24324 POINT(33.37384089774201 -118.98812284498577) bank24324 +24325 POINT(33.05738977668655 -117.904025561801) bank24325 +24326 POINT(34.15779108755119 -118.07548533167896) bank24326 +24327 POINT(33.67462262917731 -119.18640057094359) bank24327 +24328 POINT(33.40210551281039 -118.76335200983809) bank24328 +24329 POINT(33.66614281892422 -118.9577641209945) bank24329 +24330 POINT(33.67130940364383 -117.54399021643924) bank24330 +24331 POINT(34.45841433622023 -117.67021199911706) bank24331 +24332 POINT(34.300382541126226 -118.62057161310001) bank24332 +24333 POINT(33.397306346611465 -119.21777746939162) bank24333 +24334 POINT(34.374044122037205 -119.18988351746079) bank24334 +24335 POINT(33.29965705403579 -118.10712794784682) bank24335 +24336 POINT(33.65393575149265 -118.38466542554193) bank24336 +24337 POINT(33.165132644149104 -119.10254921856566) bank24337 +24338 POINT(33.17307600388859 -119.08408304258192) bank24338 +24339 POINT(33.87766200478128 -117.42975822913175) bank24339 +24340 POINT(33.152065245298665 -117.38786733738903) bank24340 +24341 POINT(33.41893534583031 -117.90008278390367) bank24341 +24342 POINT(34.130037077700344 -117.99882507375098) bank24342 +24343 POINT(34.91813779627307 -118.23943412446367) bank24343 +24344 POINT(34.20759939113469 -118.21692372241554) bank24344 +24345 POINT(34.86541129924011 -119.04004981297606) bank24345 +24346 POINT(33.62746927114483 -118.57905824305242) bank24346 +24347 POINT(33.3400726864487 -118.58374340112086) bank24347 +24348 POINT(34.15652451113359 -117.99934226948244) bank24348 +24349 POINT(33.320529189001995 -118.53627028274573) bank24349 +24350 POINT(33.40637295290594 -117.33065082837076) bank24350 +24351 POINT(35.05204842637965 -118.70204192185965) bank24351 +24352 POINT(34.08677571840765 -118.95525758922304) bank24352 +24353 POINT(33.626405185746506 -118.82522216168216) bank24353 +24354 POINT(34.097208545649146 -118.64398790724692) bank24354 +24355 POINT(33.98296334795489 -117.27768552842886) bank24355 +24356 POINT(34.01180232504063 -118.37429972756132) bank24356 +24357 POINT(33.41810766552944 -118.14954916022772) bank24357 +24358 POINT(33.56529341355413 -118.33670349177814) bank24358 +24359 POINT(33.77884145276779 -117.30920450830071) bank24359 +24360 POINT(33.107967398822545 -118.10170653960446) bank24360 +24361 POINT(34.006811047907654 -118.9510699436713) bank24361 +24362 POINT(33.76953767818821 -117.84465505944868) bank24362 +24363 POINT(34.271524681205854 -117.94804749604715) bank24363 +24364 POINT(33.44688499790876 -118.67747186515918) bank24364 +24365 POINT(34.20268428373864 -117.98846275160099) bank24365 +24366 POINT(34.901533227190534 -118.75876378432727) bank24366 +24367 POINT(33.168788314614105 -118.87940188966415) bank24367 +24368 POINT(34.6595095505333 -117.3031223601827) bank24368 +24369 POINT(33.9425787299797 -118.49754151057203) bank24369 +24370 POINT(33.51466449288453 -118.16967986817677) bank24370 +24371 POINT(34.24380050683009 -118.5076297686755) bank24371 +24372 POINT(33.929524323583884 -118.89822291516714) bank24372 +24373 POINT(34.80017360474263 -119.00086280432696) bank24373 +24374 POINT(33.959292721655565 -118.72258896928344) bank24374 +24375 POINT(33.629805675853646 -117.75960669322339) bank24375 +24376 POINT(34.40593003158097 -118.97847004379912) bank24376 +24377 POINT(33.072917643866184 -117.38571214925943) bank24377 +24378 POINT(35.00975749185505 -119.17683674902263) bank24378 +24379 POINT(34.39760447489868 -117.45189326638014) bank24379 +24380 POINT(33.41792461488297 -118.26167057437574) bank24380 +24381 POINT(34.306862514007165 -118.9793261926615) bank24381 +24382 POINT(33.35530447478104 -118.0625080762805) bank24382 +24383 POINT(34.1572447771684 -118.58003216483978) bank24383 +24384 POINT(34.38568301501999 -117.87492306406288) bank24384 +24385 POINT(33.6278129495307 -118.32944266322338) bank24385 +24386 POINT(34.23518945653656 -117.55834944863473) bank24386 +24387 POINT(34.17660243934295 -117.93527102685158) bank24387 +24388 POINT(34.65473078000655 -118.26801484858584) bank24388 +24389 POINT(33.781953215190704 -117.47410692116479) bank24389 +24390 POINT(33.65809828868253 -118.95136363094774) bank24390 +24391 POINT(33.88085152740413 -117.39127046341396) bank24391 +24392 POINT(34.31894140500206 -118.14119350445227) bank24392 +24393 POINT(34.49690941353747 -118.66211137926484) bank24393 +24394 POINT(33.89264698278667 -118.36174065521662) bank24394 +24395 POINT(34.77283363780366 -118.76312953618692) bank24395 +24396 POINT(34.79499838690879 -117.48057243404934) bank24396 +24397 POINT(33.453044341774046 -119.10172916230283) bank24397 +24398 POINT(33.78540559004958 -118.27441120157201) bank24398 +24399 POINT(34.81440109381913 -118.07756722715995) bank24399 +24400 POINT(34.734010393657506 -119.03262746363903) bank24400 +24401 POINT(34.86680316548468 -118.75128991040874) bank24401 +24402 POINT(33.096404786789925 -119.24367445489979) bank24402 +24403 POINT(33.126220578203544 -118.42871261000721) bank24403 +24404 POINT(34.463134713709465 -117.52932474695763) bank24404 +24405 POINT(33.14699311820334 -117.9997208516048) bank24405 +24406 POINT(34.39586239650931 -118.50583531154582) bank24406 +24407 POINT(34.459324049117285 -118.23886290899154) bank24407 +24408 POINT(33.91695702761142 -118.20941673174478) bank24408 +24409 POINT(34.8860110346098 -117.43665929039558) bank24409 +24410 POINT(33.54802045385505 -118.4413559770733) bank24410 +24411 POINT(33.275215123361335 -117.34071149730674) bank24411 +24412 POINT(34.81526035426023 -117.71817837644673) bank24412 +24413 POINT(33.5247623746138 -118.23549505527511) bank24413 +24414 POINT(33.119066799579144 -117.47571681121056) bank24414 +24415 POINT(34.40467179861282 -118.70997714756554) bank24415 +24416 POINT(34.51964093871067 -118.08983605061822) bank24416 +24417 POINT(33.949785165532504 -117.98252757803196) bank24417 +24418 POINT(34.50601150566379 -117.51805773950734) bank24418 +24419 POINT(33.805960247215964 -117.31627446502429) bank24419 +24420 POINT(34.07425905040607 -118.42249551502726) bank24420 +24421 POINT(33.81442959596509 -117.98804462885056) bank24421 +24422 POINT(33.113820091784845 -118.60044795913541) bank24422 +24423 POINT(33.95013015899565 -118.53134623495954) bank24423 +24424 POINT(33.61680707358467 -118.82868146611834) bank24424 +24425 POINT(34.48464986015104 -117.86336128537869) bank24425 +24426 POINT(33.54989124091353 -118.52378994670484) bank24426 +24427 POINT(34.40704785106941 -118.07834559779502) bank24427 +24428 POINT(34.81333243065142 -117.46136919824858) bank24428 +24429 POINT(33.701564472736855 -118.10762035460732) bank24429 +24430 POINT(34.958449204511695 -118.46755718180907) bank24430 +24431 POINT(33.26019394406538 -118.85972149486852) bank24431 +24432 POINT(33.547222093526635 -118.8293552783952) bank24432 +24433 POINT(34.96757266943559 -119.0364892006165) bank24433 +24434 POINT(33.701236642000666 -118.15259561801236) bank24434 +24435 POINT(34.070460171188124 -118.92940199037436) bank24435 +24436 POINT(33.46638611144469 -117.86067080907786) bank24436 +24437 POINT(34.75639768983406 -118.68946600505537) bank24437 +24438 POINT(33.261083527102 -117.46604409364286) bank24438 +24439 POINT(34.20255868124685 -117.51156817211596) bank24439 +24440 POINT(33.537689836094174 -118.28933790198226) bank24440 +24441 POINT(34.69059688556597 -118.20597209251385) bank24441 +24442 POINT(34.27732222835682 -119.05478748128698) bank24442 +24443 POINT(33.4592789114159 -117.62358396232631) bank24443 +24444 POINT(33.82440967713647 -118.7790508686076) bank24444 +24445 POINT(34.026190531814834 -118.03845184607925) bank24445 +24446 POINT(33.215791222307125 -117.83459098528282) bank24446 +24447 POINT(34.67459971897971 -118.892561728669) bank24447 +24448 POINT(34.480197714820406 -118.76122046260146) bank24448 +24449 POINT(33.62683001892337 -117.99662810313036) bank24449 +24450 POINT(34.392752555094894 -117.6105944860475) bank24450 +24451 POINT(34.67680301053983 -117.99399053485362) bank24451 +24452 POINT(34.57962665890689 -118.28361698821841) bank24452 +24453 POINT(33.929907416527556 -118.41725688404314) bank24453 +24454 POINT(34.99089778211719 -118.08000692218891) bank24454 +24455 POINT(34.436583008920095 -117.58851295113153) bank24455 +24456 POINT(34.55411382491734 -119.22054107545603) bank24456 +24457 POINT(33.17475045510854 -117.45395577777762) bank24457 +24458 POINT(33.272596707523 -118.49533665493426) bank24458 +24459 POINT(34.84949876182063 -118.6681901786781) bank24459 +24460 POINT(33.859525176273266 -117.91571593735003) bank24460 +24461 POINT(34.396863281054 -119.20639779788296) bank24461 +24462 POINT(34.04083054422638 -119.10355389107792) bank24462 +24463 POINT(33.980702272942985 -119.0380072662794) bank24463 +24464 POINT(34.26213200918389 -117.9054214520978) bank24464 +24465 POINT(33.99714085584854 -117.6945891609915) bank24465 +24466 POINT(33.78307597431943 -117.78491691387312) bank24466 +24467 POINT(33.72910543499033 -118.38857349771436) bank24467 +24468 POINT(34.592499257822766 -118.70007004384411) bank24468 +24469 POINT(34.00289237133141 -117.9961287580551) bank24469 +24470 POINT(33.40709476957287 -118.05692786152598) bank24470 +24471 POINT(33.93074264526892 -118.03934273582769) bank24471 +24472 POINT(34.36476072492584 -117.72122264569181) bank24472 +24473 POINT(33.57617267624103 -117.88284555796267) bank24473 +24474 POINT(34.44977527030242 -117.24932021154707) bank24474 +24475 POINT(33.787263544855186 -118.444937366781) bank24475 +24476 POINT(34.40243593277214 -117.41415883131006) bank24476 +24477 POINT(33.91723689517614 -118.20430981548492) bank24477 +24478 POINT(33.334803387667684 -118.91385211016897) bank24478 +24479 POINT(34.33119742600175 -117.3235342224578) bank24479 +24480 POINT(33.65764356636774 -117.98453184344508) bank24480 +24481 POINT(33.33210778097565 -118.37095960739363) bank24481 +24482 POINT(34.02165697050669 -119.21009432074507) bank24482 +24483 POINT(33.24848243160475 -117.56611557569458) bank24483 +24484 POINT(34.02934343189027 -117.28361899723045) bank24484 +24485 POINT(33.4586449979505 -117.43803697442273) bank24485 +24486 POINT(33.753382684302814 -117.87628516979376) bank24486 +24487 POINT(33.165177405249615 -118.12491385391714) bank24487 +24488 POINT(33.56335467820826 -118.53588261555635) bank24488 +24489 POINT(35.021150374193816 -118.64021515844664) bank24489 +24490 POINT(33.765644530476095 -118.3568275643366) bank24490 +24491 POINT(34.12127897146071 -119.1251065804091) bank24491 +24492 POINT(34.7180472683137 -118.0491356969193) bank24492 +24493 POINT(33.867452955500376 -118.86333002334041) bank24493 +24494 POINT(33.12675536487594 -117.90870589198911) bank24494 +24495 POINT(33.7911560022488 -118.75044163494381) bank24495 +24496 POINT(33.059212203285114 -117.64006576020559) bank24496 +24497 POINT(33.69340988343291 -118.62935210901433) bank24497 +24498 POINT(34.255586927531404 -118.15550680934165) bank24498 +24499 POINT(34.399896391614064 -118.09559488338141) bank24499 +24500 POINT(33.24865597588782 -118.9983834859301) bank24500 +24501 POINT(33.9355687473513 -119.21582948983527) bank24501 +24502 POINT(34.13890875208195 -117.49269725742084) bank24502 +24503 POINT(34.180557501024786 -119.1695910069476) bank24503 +24504 POINT(35.03612059452941 -118.93874198154165) bank24504 +24505 POINT(33.93838560026851 -119.17116963952569) bank24505 +24506 POINT(33.83721537413269 -118.1332139860279) bank24506 +24507 POINT(33.50125747986463 -118.21746336634745) bank24507 +24508 POINT(33.08513786586191 -118.79772059504687) bank24508 +24509 POINT(33.726208327459645 -117.79806828823513) bank24509 +24510 POINT(33.4013152986588 -118.65860697402647) bank24510 +24511 POINT(34.830204873351605 -118.20882496529521) bank24511 +24512 POINT(34.07726440074329 -118.06211995623487) bank24512 +24513 POINT(34.56471844560856 -117.5731189335851) bank24513 +24514 POINT(33.23075294966077 -118.75744024786746) bank24514 +24515 POINT(33.43383111166928 -118.055501017798) bank24515 +24516 POINT(34.56695910988746 -119.04401978699966) bank24516 +24517 POINT(34.19691326396558 -119.04591088855724) bank24517 +24518 POINT(34.37307028580353 -117.58165041101475) bank24518 +24519 POINT(33.31813620791735 -118.38384077845177) bank24519 +24520 POINT(33.396254864344435 -117.95427355990954) bank24520 +24521 POINT(34.87086358696154 -118.26197680609687) bank24521 +24522 POINT(33.584658905990636 -118.72545246442326) bank24522 +24523 POINT(33.52622317752318 -118.06219824254431) bank24523 +24524 POINT(33.80153213763145 -119.1397191791428) bank24524 +24525 POINT(33.25475361788043 -118.93324579256958) bank24525 +24526 POINT(35.02041111939613 -118.83763748927679) bank24526 +24527 POINT(34.06645230288022 -117.39954296125363) bank24527 +24528 POINT(33.98034275746929 -118.44084382081229) bank24528 +24529 POINT(34.46099630512979 -118.52904719653759) bank24529 +24530 POINT(34.40734679860791 -118.09232221985316) bank24530 +24531 POINT(33.591334590182825 -118.28313563745377) bank24531 +24532 POINT(35.03114630821824 -118.46660306022544) bank24532 +24533 POINT(33.28766589323588 -117.75943968982482) bank24533 +24534 POINT(33.53458382369517 -117.43150745910772) bank24534 +24535 POINT(33.80918768727266 -117.57317090477923) bank24535 +24536 POINT(34.982360238256284 -117.34031697248416) bank24536 +24537 POINT(33.57914999323508 -117.51848728161201) bank24537 +24538 POINT(34.42181280957782 -117.53608812803485) bank24538 +24539 POINT(33.60941160710839 -117.90525936402378) bank24539 +24540 POINT(34.108661849561656 -118.94283715566611) bank24540 +24541 POINT(33.63113851786079 -117.8987353832051) bank24541 +24542 POINT(33.06300134726567 -117.90624091208944) bank24542 +24543 POINT(33.47218195498448 -117.89740766546943) bank24543 +24544 POINT(34.35426926745094 -117.33171664967293) bank24544 +24545 POINT(34.73233438793693 -118.46450051643951) bank24545 +24546 POINT(33.507776217417664 -118.70928989492238) bank24546 +24547 POINT(34.7122197313141 -118.61228951527019) bank24547 +24548 POINT(33.88992287042919 -118.06473965456472) bank24548 +24549 POINT(33.98808348140612 -119.22940250939043) bank24549 +24550 POINT(33.77359838362873 -117.31520185711244) bank24550 +24551 POINT(33.41521751553681 -118.61043890359988) bank24551 +24552 POINT(33.96554075895253 -118.14879009327557) bank24552 +24553 POINT(34.651036918975464 -117.50324931953546) bank24553 +24554 POINT(33.7622126987173 -117.36374636232465) bank24554 +24555 POINT(33.388454787392945 -118.64889483181133) bank24555 +24556 POINT(34.81272840785718 -118.32798859726383) bank24556 +24557 POINT(33.694379246152394 -117.7660324103549) bank24557 +24558 POINT(34.34332233443602 -117.88640360162827) bank24558 +24559 POINT(33.14581356949592 -117.28801839481565) bank24559 +24560 POINT(34.87648486749751 -118.23521330220125) bank24560 +24561 POINT(34.5304023763004 -117.46731994459279) bank24561 +24562 POINT(34.11986704266929 -118.01077808865412) bank24562 +24563 POINT(33.323648687519984 -118.0499929267789) bank24563 +24564 POINT(34.1576601821392 -118.6107143301472) bank24564 +24565 POINT(33.47385716865586 -118.10011962213882) bank24565 +24566 POINT(34.209180654468284 -118.10491689674551) bank24566 +24567 POINT(33.19986654705112 -117.61468528382757) bank24567 +24568 POINT(34.67641309681693 -118.60088233667867) bank24568 +24569 POINT(34.88453482373828 -117.48067032911314) bank24569 +24570 POINT(33.98958453820302 -117.60106748600631) bank24570 +24571 POINT(34.290861079253624 -117.86884984528184) bank24571 +24572 POINT(33.727431812280344 -117.2587449906802) bank24572 +24573 POINT(33.60240533166833 -118.34612219158228) bank24573 +24574 POINT(34.136766134017336 -117.80513983234378) bank24574 +24575 POINT(34.66148020449591 -118.68343794308694) bank24575 +24576 POINT(34.17769089553878 -118.11042011099704) bank24576 +24577 POINT(34.1668737893108 -118.58272007238179) bank24577 +24578 POINT(34.66779530811466 -117.78450292038683) bank24578 +24579 POINT(34.18516289585578 -118.44046989470891) bank24579 +24580 POINT(34.01345183195482 -118.24023265875441) bank24580 +24581 POINT(33.671239024660586 -117.9816010134384) bank24581 +24582 POINT(33.1787795087329 -119.1326048827645) bank24582 +24583 POINT(33.55479463686732 -118.57578236435141) bank24583 +24584 POINT(34.66580418198531 -118.05995854781358) bank24584 +24585 POINT(33.170671772882805 -118.61581184554984) bank24585 +24586 POINT(34.32583538232394 -117.6821025786987) bank24586 +24587 POINT(34.47755462711632 -118.32251362544358) bank24587 +24588 POINT(34.9361045565388 -118.47939013631319) bank24588 +24589 POINT(34.76299943485866 -118.5546889494497) bank24589 +24590 POINT(34.83313367274475 -117.94901036410093) bank24590 +24591 POINT(34.8263342513147 -118.39168162197707) bank24591 +24592 POINT(33.13700753724803 -117.9652122390217) bank24592 +24593 POINT(34.259725416699645 -117.2443052937443) bank24593 +24594 POINT(34.379620731662875 -117.504011840887) bank24594 +24595 POINT(34.78592286313207 -117.36872657722662) bank24595 +24596 POINT(34.23067228886534 -117.94059519444048) bank24596 +24597 POINT(34.36873122404549 -117.45796438058204) bank24597 +24598 POINT(33.64957374738108 -118.68230462095961) bank24598 +24599 POINT(34.93161768029705 -118.94971574855484) bank24599 +24600 POINT(33.253331659338265 -117.50779478612284) bank24600 +24601 POINT(33.72620693123638 -119.07465663818115) bank24601 +24602 POINT(34.495445804862754 -118.78529725198206) bank24602 +24603 POINT(34.47519668931986 -119.08147783721849) bank24603 +24604 POINT(33.634902213690225 -119.15597109899878) bank24604 +24605 POINT(33.64060592191291 -118.99965949245004) bank24605 +24606 POINT(33.10887069647534 -117.27844245596755) bank24606 +24607 POINT(34.11191905782341 -117.60909110157499) bank24607 +24608 POINT(34.903657678920595 -118.10160977998734) bank24608 +24609 POINT(33.88970655239805 -118.00320501455705) bank24609 +24610 POINT(33.700112350852464 -117.74137917627604) bank24610 +24611 POINT(34.23663598621081 -118.66112904865831) bank24611 +24612 POINT(33.530339039190274 -118.68556518153743) bank24612 +24613 POINT(34.93495683082467 -118.2949705679869) bank24613 +24614 POINT(34.811107147193376 -117.76799085705723) bank24614 +24615 POINT(33.662393239271616 -117.34592424943867) bank24615 +24616 POINT(33.40309142359021 -118.7808097077737) bank24616 +24617 POINT(34.09065663472594 -117.5075984148029) bank24617 +24618 POINT(33.6198336656696 -118.16967932255852) bank24618 +24619 POINT(34.41450041925704 -118.01019773030953) bank24619 +24620 POINT(34.83308112205343 -117.4460575272348) bank24620 +24621 POINT(34.08264109392253 -118.19652088742622) bank24621 +24622 POINT(33.35906406989529 -119.18638066157698) bank24622 +24623 POINT(33.29016001321644 -118.10384931251735) bank24623 +24624 POINT(33.72836894505481 -117.55317632110983) bank24624 +24625 POINT(34.53849254167994 -119.02407950658757) bank24625 +24626 POINT(34.928964107969094 -118.92998066606958) bank24626 +24627 POINT(34.71077153732556 -118.18686038939228) bank24627 +24628 POINT(34.42253167552268 -118.74975843212101) bank24628 +24629 POINT(34.11317368932371 -117.8000861651446) bank24629 +24630 POINT(34.203276662914064 -118.83153819796335) bank24630 +24631 POINT(34.48874098034559 -118.49542120110837) bank24631 +24632 POINT(34.64405756000349 -117.34050354213352) bank24632 +24633 POINT(33.87443984069889 -117.89427428644436) bank24633 +24634 POINT(33.40146527537978 -119.23893273071559) bank24634 +24635 POINT(33.83337878243493 -118.87515748769691) bank24635 +24636 POINT(34.50987020306699 -117.93814644166508) bank24636 +24637 POINT(33.604157477648684 -119.04888293241142) bank24637 +24638 POINT(34.54169680480804 -117.80587131710601) bank24638 +24639 POINT(34.80856986425015 -117.67952386454562) bank24639 +24640 POINT(33.44101833676806 -117.91049608703212) bank24640 +24641 POINT(33.37446015784812 -118.5815184214268) bank24641 +24642 POINT(33.149692652951465 -117.83929489950349) bank24642 +24643 POINT(34.30218727646022 -118.62556593268054) bank24643 +24644 POINT(33.95472216223543 -119.21646207774369) bank24644 +24645 POINT(34.804401303098984 -118.65128864962065) bank24645 +24646 POINT(34.69332092647572 -117.27546203072248) bank24646 +24647 POINT(34.325741440612624 -117.28581060322249) bank24647 +24648 POINT(35.00031500476516 -118.79749254660747) bank24648 +24649 POINT(33.45444691170133 -117.71590549891316) bank24649 +24650 POINT(33.42320119048944 -117.34186630098196) bank24650 +24651 POINT(34.56950377412007 -118.97825500664578) bank24651 +24652 POINT(33.61143766975709 -119.00185734761124) bank24652 +24653 POINT(34.36032800878895 -118.30204576344657) bank24653 +24654 POINT(34.143877415906864 -117.25396028331706) bank24654 +24655 POINT(33.55146060041849 -119.02016016965297) bank24655 +24656 POINT(33.95492165467205 -119.0255540088632) bank24656 +24657 POINT(35.02243948940511 -118.2857217081664) bank24657 +24658 POINT(33.47503047054074 -118.49018830002406) bank24658 +24659 POINT(33.328370762756315 -117.68692136419757) bank24659 +24660 POINT(33.99725343617197 -118.07864127765019) bank24660 +24661 POINT(34.12952987036362 -118.35834194465997) bank24661 +24662 POINT(34.6542133981498 -117.80123554884909) bank24662 +24663 POINT(34.79092865115224 -118.90796223763303) bank24663 +24664 POINT(33.3219679753501 -117.54373093244484) bank24664 +24665 POINT(34.575565964813265 -117.30136998001024) bank24665 +24666 POINT(33.96234024874941 -117.79002694243346) bank24666 +24667 POINT(34.38599439330992 -117.85952673698183) bank24667 +24668 POINT(33.688730127886224 -117.75680429141782) bank24668 +24669 POINT(33.547532716896 -118.26783604267571) bank24669 +24670 POINT(33.59659922362272 -117.28261901273547) bank24670 +24671 POINT(33.79120253649225 -117.40414676691783) bank24671 +24672 POINT(33.59038502386155 -117.31408760581854) bank24672 +24673 POINT(33.21602660430442 -117.87844181808774) bank24673 +24674 POINT(33.959734445314716 -118.10865094237168) bank24674 +24675 POINT(33.32936471435558 -117.78376852348217) bank24675 +24676 POINT(33.852264789813376 -118.45975610949328) bank24676 +24677 POINT(33.54813249196539 -118.76506564068367) bank24677 +24678 POINT(34.78492976020082 -117.65831418001504) bank24678 +24679 POINT(34.217852379536936 -117.46962235615491) bank24679 +24680 POINT(33.074436340856025 -117.7905017156031) bank24680 +24681 POINT(35.01484709918542 -119.20721487017154) bank24681 +24682 POINT(34.53460612446651 -118.4952307104535) bank24682 +24683 POINT(33.94225913328602 -118.24882013687748) bank24683 +24684 POINT(33.5784693497078 -118.22628144663662) bank24684 +24685 POINT(34.1942682770125 -117.51205956313032) bank24685 +24686 POINT(34.33664602684667 -117.28196972610864) bank24686 +24687 POINT(34.88070864717648 -118.86586295888627) bank24687 +24688 POINT(34.43813017745579 -117.98100098969786) bank24688 +24689 POINT(33.80848989795902 -119.16302986048576) bank24689 +24690 POINT(33.441309041711335 -117.53569397141773) bank24690 +24691 POINT(33.251329951038954 -118.5435154241857) bank24691 +24692 POINT(35.020873575470254 -119.03424478980091) bank24692 +24693 POINT(34.426104280531106 -119.23842546997994) bank24693 +24694 POINT(33.799299358307046 -118.79274373642065) bank24694 +24695 POINT(34.68125080373015 -118.80864471540326) bank24695 +24696 POINT(34.43879384965858 -118.5121104148174) bank24696 +24697 POINT(34.78453653824219 -118.56620356126182) bank24697 +24698 POINT(33.11616486268539 -118.43431146241159) bank24698 +24699 POINT(34.82405452179708 -118.45930980641536) bank24699 +24700 POINT(34.64115003318966 -118.86936178277914) bank24700 +24701 POINT(34.280787495364144 -117.55866862154087) bank24701 +24702 POINT(33.05452565371753 -117.43351994141409) bank24702 +24703 POINT(33.80871597803823 -118.47306349463094) bank24703 +24704 POINT(34.364816682219534 -118.77222672520686) bank24704 +24705 POINT(34.144481235223914 -119.03247151466968) bank24705 +24706 POINT(34.98081077475454 -118.74667898395803) bank24706 +24707 POINT(34.59718022811083 -119.23117135051801) bank24707 +24708 POINT(33.23209920405188 -118.49780066215092) bank24708 +24709 POINT(34.84234317004622 -119.20315144091849) bank24709 +24710 POINT(33.32730609252913 -117.63351899854919) bank24710 +24711 POINT(34.05591587749822 -117.76353141024455) bank24711 +24712 POINT(33.05759737863935 -118.70635228698288) bank24712 +24713 POINT(34.27980653971059 -117.77893431911444) bank24713 +24714 POINT(33.1719202619028 -117.81178456713862) bank24714 +24715 POINT(33.45711731516087 -118.30320712104405) bank24715 +24716 POINT(33.77127352953982 -119.20942144659692) bank24716 +24717 POINT(34.698827478524784 -117.75987190447604) bank24717 +24718 POINT(33.61931550067716 -118.94911091561914) bank24718 +24719 POINT(34.8315267090117 -118.35815626441264) bank24719 +24720 POINT(34.94072159189315 -117.72939963485358) bank24720 +24721 POINT(33.56809923002902 -117.95767393350762) bank24721 +24722 POINT(34.66414820025568 -118.93568514277237) bank24722 +24723 POINT(34.14256624612339 -117.40264826482023) bank24723 +24724 POINT(34.10997141248536 -118.44741114887121) bank24724 +24725 POINT(33.51688625519245 -118.00470083875715) bank24725 +24726 POINT(33.85323240076848 -118.95924626401523) bank24726 +24727 POINT(33.22008941501413 -118.3543283302586) bank24727 +24728 POINT(33.40869199778031 -119.01310643552866) bank24728 +24729 POINT(34.57943813322194 -118.77917606464943) bank24729 +24730 POINT(34.46182183274513 -118.47659217250686) bank24730 +24731 POINT(34.371604026894445 -117.51705473680693) bank24731 +24732 POINT(34.82735762010793 -118.25111297766506) bank24732 +24733 POINT(34.43412418177318 -117.3368869982029) bank24733 +24734 POINT(35.00938154683111 -118.37834345023073) bank24734 +24735 POINT(33.06785133558976 -118.0951044995454) bank24735 +24736 POINT(33.25608412519535 -118.9085462247833) bank24736 +24737 POINT(34.992678209180795 -118.7531918027027) bank24737 +24738 POINT(34.936853017600065 -117.9589199991944) bank24738 +24739 POINT(33.58171342615257 -117.3077520260332) bank24739 +24740 POINT(34.041281788155835 -118.32469643998796) bank24740 +24741 POINT(33.82568893552106 -118.4575558596223) bank24741 +24742 POINT(34.694324260370784 -117.39716403100046) bank24742 +24743 POINT(34.256289734028144 -118.78506898515509) bank24743 +24744 POINT(33.24682421562076 -118.6263414561449) bank24744 +24745 POINT(33.42468803304089 -118.68148230485646) bank24745 +24746 POINT(34.26910052518633 -119.03898045881688) bank24746 +24747 POINT(34.742702094363224 -119.21659295636823) bank24747 +24748 POINT(33.43739918676053 -117.5229506880448) bank24748 +24749 POINT(34.75755610578618 -117.85272105581599) bank24749 +24750 POINT(33.679452912877856 -119.19058768141181) bank24750 +24751 POINT(34.34281032809266 -118.28021308260736) bank24751 +24752 POINT(34.5721860954833 -118.93137140577329) bank24752 +24753 POINT(34.39008845463222 -118.81301377166433) bank24753 +24754 POINT(33.2496965440422 -117.46643415589902) bank24754 +24755 POINT(33.83852284182233 -118.67884905845142) bank24755 +24756 POINT(34.0888340795635 -119.04420289142273) bank24756 +24757 POINT(35.000976937409185 -117.70028613691427) bank24757 +24758 POINT(34.4725651340111 -119.03983546112765) bank24758 +24759 POINT(33.75853354912104 -118.94733491741106) bank24759 +24760 POINT(34.686272592552875 -118.8005054960303) bank24760 +24761 POINT(34.041159733068426 -118.15957034289889) bank24761 +24762 POINT(35.017749224318955 -118.85615073876468) bank24762 +24763 POINT(34.67820267542447 -117.72783945878861) bank24763 +24764 POINT(34.35085517047685 -117.36753305080985) bank24764 +24765 POINT(33.328716690578915 -117.2564219044296) bank24765 +24766 POINT(34.67321389907715 -118.13966161969903) bank24766 +24767 POINT(33.71103269318166 -118.64920622159582) bank24767 +24768 POINT(34.953252680930845 -119.01405118320322) bank24768 +24769 POINT(34.29061340953628 -118.8294313042122) bank24769 +24770 POINT(34.598000464880684 -118.71086817900765) bank24770 +24771 POINT(33.74328090264082 -118.47410672721664) bank24771 +24772 POINT(34.87331749405961 -118.53885080692126) bank24772 +24773 POINT(34.40552838228112 -119.10607828108924) bank24773 +24774 POINT(33.576095731496814 -119.15139117853923) bank24774 +24775 POINT(33.87074980267453 -118.06054691116302) bank24775 +24776 POINT(34.00330091870709 -118.08338010320983) bank24776 +24777 POINT(33.204685794915186 -118.4002832808055) bank24777 +24778 POINT(33.771984631152954 -118.66531765045077) bank24778 +24779 POINT(33.49384319180875 -118.89949223421223) bank24779 +24780 POINT(34.68836195426085 -118.44987868705533) bank24780 +24781 POINT(34.277414868146636 -119.17188370703866) bank24781 +24782 POINT(33.790996707171935 -118.5804827497137) bank24782 +24783 POINT(33.08613149897431 -117.82616576479654) bank24783 +24784 POINT(33.45625155653977 -118.45253846087512) bank24784 +24785 POINT(33.34793194531666 -119.19298658465455) bank24785 +24786 POINT(34.48099354047894 -117.57312825283293) bank24786 +24787 POINT(34.168973841290835 -119.01608485503348) bank24787 +24788 POINT(34.549631430415396 -117.34117564966671) bank24788 +24789 POINT(34.94605205955094 -118.10435285471743) bank24789 +24790 POINT(33.904643107970344 -117.59086438331747) bank24790 +24791 POINT(33.604182794484565 -118.71548165073523) bank24791 +24792 POINT(34.255294444434995 -118.02208719006148) bank24792 +24793 POINT(33.40146560304196 -117.88364520433397) bank24793 +24794 POINT(34.41173303981839 -118.57748489914135) bank24794 +24795 POINT(33.79304736922026 -117.78750188845214) bank24795 +24796 POINT(33.47244627669355 -117.4124434618571) bank24796 +24797 POINT(33.8205412200252 -118.27935190831957) bank24797 +24798 POINT(33.99156712256757 -117.81069355935418) bank24798 +24799 POINT(33.83617691245153 -118.06355407718907) bank24799 +24800 POINT(33.6742591587483 -118.55132292838042) bank24800 +24801 POINT(34.56936594771649 -117.27518243639695) bank24801 +24802 POINT(33.60763406402807 -117.63937962998641) bank24802 +24803 POINT(33.7254782336474 -118.41477111613085) bank24803 +24804 POINT(33.49832174222018 -118.43846043833108) bank24804 +24805 POINT(34.25457572327308 -118.125758170216) bank24805 +24806 POINT(34.79124501698228 -118.84583029000062) bank24806 +24807 POINT(34.33034413175045 -118.53455247641025) bank24807 +24808 POINT(33.12990836319669 -118.12086961456126) bank24808 +24809 POINT(33.77079093184441 -118.45569325827898) bank24809 +24810 POINT(33.18373221398764 -118.48431447660387) bank24810 +24811 POINT(34.10182488368878 -118.61896456053866) bank24811 +24812 POINT(33.8429994106655 -119.19204067166442) bank24812 +24813 POINT(33.945065195837365 -118.52936620268926) bank24813 +24814 POINT(35.00587029069051 -118.5128378612254) bank24814 +24815 POINT(34.92514257193861 -118.16420829044593) bank24815 +24816 POINT(34.40958494065063 -118.7967185304315) bank24816 +24817 POINT(33.66525266157125 -118.40331480676919) bank24817 +24818 POINT(33.76198437079442 -118.82260193178774) bank24818 +24819 POINT(33.29075210048204 -118.4220033571977) bank24819 +24820 POINT(34.85643393001066 -118.34118195065204) bank24820 +24821 POINT(33.89658454643042 -118.80021615858406) bank24821 +24822 POINT(33.752340140228405 -119.15552809121318) bank24822 +24823 POINT(33.502495693245294 -118.44240278987589) bank24823 +24824 POINT(34.69296053184791 -118.6834304954932) bank24824 +24825 POINT(33.89391959124908 -117.74223326061198) bank24825 +24826 POINT(33.556042210497836 -117.39150626452069) bank24826 +24827 POINT(33.77056492113904 -118.83937019111845) bank24827 +24828 POINT(34.42857388636522 -118.59640745157765) bank24828 +24829 POINT(34.557676638421405 -118.70350753982065) bank24829 +24830 POINT(34.196748152836385 -118.49545112012493) bank24830 +24831 POINT(34.897574621199084 -117.67814165566406) bank24831 +24832 POINT(34.00895416329628 -118.64478099451752) bank24832 +24833 POINT(33.26107713743167 -117.88023708037802) bank24833 +24834 POINT(33.73434443864916 -119.07605917336228) bank24834 +24835 POINT(34.71580835049964 -117.25651529361858) bank24835 +24836 POINT(34.99631177092115 -118.18301288446732) bank24836 +24837 POINT(33.63510272067577 -118.9093128056669) bank24837 +24838 POINT(34.83997432392684 -119.10079831502941) bank24838 +24839 POINT(34.69243865993579 -117.77292758562517) bank24839 +24840 POINT(34.46794556633602 -117.94943943966682) bank24840 +24841 POINT(34.64204831485857 -117.89311574638364) bank24841 +24842 POINT(33.22291665450477 -117.41412410624167) bank24842 +24843 POINT(34.79072640600919 -118.19917616739593) bank24843 +24844 POINT(34.71957418115262 -118.30667372585066) bank24844 +24845 POINT(33.76277204774148 -119.06774194437118) bank24845 +24846 POINT(33.85538951081664 -117.67916256993543) bank24846 +24847 POINT(33.502737808144275 -117.47391315792976) bank24847 +24848 POINT(33.937217877133506 -117.79162262807965) bank24848 +24849 POINT(33.66640913090686 -118.3899812211334) bank24849 +24850 POINT(34.690855214756134 -118.0895284339359) bank24850 +24851 POINT(34.44001830025819 -117.8299862131467) bank24851 +24852 POINT(34.88113747787071 -117.36896631407099) bank24852 +24853 POINT(33.64530888141501 -117.72208765215332) bank24853 +24854 POINT(34.01124560273654 -118.60800987960383) bank24854 +24855 POINT(34.739736658108114 -117.49352000981553) bank24855 +24856 POINT(34.213161401761276 -117.65920547655587) bank24856 +24857 POINT(34.25375096750373 -118.35499309639098) bank24857 +24858 POINT(34.596757776539306 -117.97050903066115) bank24858 +24859 POINT(34.360437598647586 -117.84397883941641) bank24859 +24860 POINT(34.33365178782406 -117.72605107503246) bank24860 +24861 POINT(34.48171936380432 -117.88828773309092) bank24861 +24862 POINT(33.97243209071926 -117.94869830141921) bank24862 +24863 POINT(33.08185622032647 -117.44312567586297) bank24863 +24864 POINT(33.4410394986678 -119.19197545706538) bank24864 +24865 POINT(33.54009292295456 -119.07445312489223) bank24865 +24866 POINT(34.135194553898174 -118.87747847875916) bank24866 +24867 POINT(34.973378136213924 -119.17249665100888) bank24867 +24868 POINT(33.63771931232331 -119.17943536737444) bank24868 +24869 POINT(33.65460216325473 -117.9844940731238) bank24869 +24870 POINT(34.88219769530741 -117.73156419387539) bank24870 +24871 POINT(34.2315413651733 -118.83890029416939) bank24871 +24872 POINT(33.51494285595349 -118.06874384519129) bank24872 +24873 POINT(34.30882186245817 -118.27312896750402) bank24873 +24874 POINT(33.5028923914172 -118.91027865796892) bank24874 +24875 POINT(34.60785145460501 -117.32082167198546) bank24875 +24876 POINT(33.35774412138799 -118.11109635067665) bank24876 +24877 POINT(33.6890828214541 -118.57408281997238) bank24877 +24878 POINT(33.194867971943246 -118.12198407557916) bank24878 +24879 POINT(33.154387849713274 -119.15722500124073) bank24879 +24880 POINT(33.852207601866745 -119.12190339623388) bank24880 +24881 POINT(34.2748715794205 -118.26075130904275) bank24881 +24882 POINT(34.82929745453038 -117.6332918934702) bank24882 +24883 POINT(34.976948805937944 -118.46484277015516) bank24883 +24884 POINT(33.70960254199972 -118.87979974721561) bank24884 +24885 POINT(34.04577306779047 -117.79786185181065) bank24885 +24886 POINT(33.9630679252691 -117.3393056596101) bank24886 +24887 POINT(34.95054121116484 -119.02430106241256) bank24887 +24888 POINT(35.02151182733521 -118.44234303050015) bank24888 +24889 POINT(33.983734151054165 -118.71815660415953) bank24889 +24890 POINT(33.26418375292676 -118.99364379199172) bank24890 +24891 POINT(33.16677274576501 -118.75965333357283) bank24891 +24892 POINT(34.915935780758424 -117.51587708909163) bank24892 +24893 POINT(34.23493045153438 -118.4934687629441) bank24893 +24894 POINT(34.65789971122689 -117.34966261351123) bank24894 +24895 POINT(34.9488428255175 -118.5545958787876) bank24895 +24896 POINT(34.535134279761124 -117.57214132186719) bank24896 +24897 POINT(33.85750100321163 -118.3696778945618) bank24897 +24898 POINT(34.030677898765525 -117.8234059484503) bank24898 +24899 POINT(33.78555268323039 -118.88725079554793) bank24899 +24900 POINT(34.266700334085115 -118.55716717771253) bank24900 +24901 POINT(34.01122395322179 -118.27525236458656) bank24901 +24902 POINT(34.5849592716211 -118.54932792994002) bank24902 +24903 POINT(33.133243750081775 -117.47807293731621) bank24903 +24904 POINT(33.92398481562918 -118.35943014883321) bank24904 +24905 POINT(33.272571721754375 -117.73985967659462) bank24905 +24906 POINT(33.52461628388361 -117.41792803769006) bank24906 +24907 POINT(34.00467560189706 -119.14266058436283) bank24907 +24908 POINT(34.95615977617232 -117.77746893030056) bank24908 +24909 POINT(33.643811307217575 -118.67170638782717) bank24909 +24910 POINT(33.39151149691088 -118.72215527300699) bank24910 +24911 POINT(34.09243385030389 -118.91370988549569) bank24911 +24912 POINT(34.34525546719497 -117.38799248888745) bank24912 +24913 POINT(34.15650148686832 -117.46260129514064) bank24913 +24914 POINT(33.22977068826852 -118.30715651390732) bank24914 +24915 POINT(33.101247675502414 -118.4851682145809) bank24915 +24916 POINT(35.04311929889766 -118.23186711222607) bank24916 +24917 POINT(34.573876239400455 -118.76732891084484) bank24917 +24918 POINT(33.83651613485793 -118.5443517722261) bank24918 +24919 POINT(34.83247388591359 -118.67583580004664) bank24919 +24920 POINT(34.97620475957662 -118.41588759362524) bank24920 +24921 POINT(34.20543887373913 -118.33226452792576) bank24921 +24922 POINT(34.87888268355165 -118.57364081494578) bank24922 +24923 POINT(33.946329893651026 -118.58250052055482) bank24923 +24924 POINT(33.66076510990389 -118.62691287141959) bank24924 +24925 POINT(34.77999217960269 -117.24602005411577) bank24925 +24926 POINT(33.99942206349585 -117.4285164816384) bank24926 +24927 POINT(33.06428732173327 -117.83294712416075) bank24927 +24928 POINT(33.81904181209159 -118.44941217975152) bank24928 +24929 POINT(34.18597135997304 -118.35854873515342) bank24929 +24930 POINT(33.841569519042054 -118.00990112004203) bank24930 +24931 POINT(33.14495478382786 -117.26375275142406) bank24931 +24932 POINT(34.15704183421339 -118.38413668589462) bank24932 +24933 POINT(33.14119600328861 -117.78864085304542) bank24933 +24934 POINT(34.66835028043138 -118.0581333300426) bank24934 +24935 POINT(33.57349363668936 -118.87075869525691) bank24935 +24936 POINT(34.881769009004366 -118.74774700111983) bank24936 +24937 POINT(33.605891342893834 -117.26025524059922) bank24937 +24938 POINT(33.13779540655269 -118.43996940803376) bank24938 +24939 POINT(34.390119307386584 -117.38278045504377) bank24939 +24940 POINT(33.27227261545323 -117.35555611300023) bank24940 +24941 POINT(34.236295237450214 -118.99469524289714) bank24941 +24942 POINT(34.769835117007226 -118.31764026607483) bank24942 +24943 POINT(35.050171761044645 -118.91285539072146) bank24943 +24944 POINT(34.07401483877424 -118.09357704177268) bank24944 +24945 POINT(34.08540449799092 -117.39686233184396) bank24945 +24946 POINT(33.129545318915405 -119.10144913500883) bank24946 +24947 POINT(33.14564727934184 -117.33238915590246) bank24947 +24948 POINT(34.94719510835176 -117.62826145038184) bank24948 +24949 POINT(34.667011123007455 -118.93913672021797) bank24949 +24950 POINT(34.12780361022574 -118.9370857207) bank24950 +24951 POINT(33.96109814573596 -118.81281915984381) bank24951 +24952 POINT(33.222093476650095 -119.17843571763798) bank24952 +24953 POINT(33.320746958225335 -117.47817888758247) bank24953 +24954 POINT(34.46624379422391 -118.66327347902687) bank24954 +24955 POINT(34.806339434619304 -117.59314535043265) bank24955 +24956 POINT(34.022301891807984 -119.14731131651902) bank24956 +24957 POINT(34.22952841085727 -119.11634552832676) bank24957 +24958 POINT(34.67766653578017 -117.71961686098112) bank24958 +24959 POINT(34.84907222105218 -117.92597610087297) bank24959 +24960 POINT(33.69231108568187 -118.5290680063236) bank24960 +24961 POINT(34.442108675629484 -118.99442679018487) bank24961 +24962 POINT(34.920369607724915 -119.22565010348995) bank24962 +24963 POINT(34.82371640085811 -118.1694680726936) bank24963 +24964 POINT(34.949327227465425 -118.62049198318961) bank24964 +24965 POINT(34.50624431469594 -118.83324028300243) bank24965 +24966 POINT(33.31264572326888 -117.49579078631889) bank24966 +24967 POINT(33.23809687668776 -117.97962682115589) bank24967 +24968 POINT(33.97775485607735 -118.42152168501873) bank24968 +24969 POINT(33.75744925923461 -117.42374669595237) bank24969 +24970 POINT(34.832151114290575 -117.30371688458759) bank24970 +24971 POINT(34.33057216250695 -117.96808388535104) bank24971 +24972 POINT(34.64318913601909 -118.65359506671138) bank24972 +24973 POINT(33.86463672821023 -118.31078273690595) bank24973 +24974 POINT(34.31744097540088 -118.0956938404364) bank24974 +24975 POINT(34.96617641481332 -118.6475603876724) bank24975 +24976 POINT(34.72317205103727 -117.67282949519173) bank24976 +24977 POINT(33.258898631597475 -118.57574157483961) bank24977 +24978 POINT(34.776577155482464 -117.99121015863965) bank24978 +24979 POINT(33.837998144400444 -119.21920806022425) bank24979 +24980 POINT(33.95690062498923 -118.31893219606343) bank24980 +24981 POINT(34.84889528268993 -118.89116988433368) bank24981 +24982 POINT(34.484599436850026 -119.1432924732579) bank24982 +24983 POINT(34.191563910179894 -118.9359333693848) bank24983 +24984 POINT(33.450313684017104 -117.5564396665266) bank24984 +24985 POINT(33.92402909921579 -117.70078149637095) bank24985 +24986 POINT(34.08174702635435 -117.62397390083197) bank24986 +24987 POINT(34.978600829269226 -118.96856136694774) bank24987 +24988 POINT(34.966505310522216 -118.75388147901903) bank24988 +24989 POINT(33.63135152387618 -117.6215327594428) bank24989 +24990 POINT(33.86064502535347 -118.58221692572941) bank24990 +24991 POINT(34.71469886202681 -117.63669899502194) bank24991 +24992 POINT(33.80863275313465 -118.59255690471925) bank24992 +24993 POINT(33.66227489630543 -118.17907054401728) bank24993 +24994 POINT(34.50014754992721 -118.32165179985894) bank24994 +24995 POINT(33.31471589954092 -118.13328562083188) bank24995 +24996 POINT(33.76423718683208 -118.88329144161835) bank24996 +24997 POINT(34.07073082680335 -119.13115191341225) bank24997 +24998 POINT(34.17075553206404 -117.2635157712875) bank24998 +24999 POINT(34.535660706979975 -118.51265612338912) bank24999 +25000 POINT(33.546662779010354 -118.29587113890085) bank25000 +25001 POINT(33.60806564693359 -118.54533410353649) bank25001 +25002 POINT(34.209094394788615 -118.1090588923361) bank25002 +25003 POINT(33.39454424500627 -118.24235232385972) bank25003 +25004 POINT(34.172202306291474 -118.93878188170925) bank25004 +25005 POINT(34.81600612758396 -119.05398037144316) bank25005 +25006 POINT(33.26957008572787 -117.32649406424657) bank25006 +25007 POINT(33.64027944663819 -117.49991344012683) bank25007 +25008 POINT(33.21359994855924 -118.19347000204628) bank25008 +25009 POINT(34.57206377828254 -117.76335877914498) bank25009 +25010 POINT(33.36638085293554 -118.70724349223468) bank25010 +25011 POINT(34.113592289951946 -118.34485885881163) bank25011 +25012 POINT(34.65716395273265 -117.35356682780701) bank25012 +25013 POINT(34.406945050592256 -118.03968503722774) bank25013 +25014 POINT(34.57155846836419 -117.37485394470201) bank25014 +25015 POINT(33.10861214254025 -118.0674496434187) bank25015 +25016 POINT(33.65174511997295 -117.47665732304863) bank25016 +25017 POINT(34.21253059639546 -118.72624100913933) bank25017 +25018 POINT(33.43691333252313 -118.34704004863194) bank25018 +25019 POINT(33.287628227724085 -118.35173633954868) bank25019 +25020 POINT(34.87625317569835 -118.53489381437701) bank25020 +25021 POINT(33.98371663262704 -117.61756913982076) bank25021 +25022 POINT(34.28102523330605 -118.23150143761666) bank25022 +25023 POINT(34.27533346731653 -117.89185216651757) bank25023 +25024 POINT(34.289343746549214 -117.6462046294861) bank25024 +25025 POINT(34.73651649636088 -119.2367266744394) bank25025 +25026 POINT(33.801139017452854 -118.70292267048984) bank25026 +25027 POINT(33.46650339168195 -118.97740326823441) bank25027 +25028 POINT(35.00907443692434 -118.31843593787792) bank25028 +25029 POINT(33.600144905374 -118.79367966219543) bank25029 +25030 POINT(34.97292817036133 -118.49825132434057) bank25030 +25031 POINT(33.2583283680268 -117.94718195959219) bank25031 +25032 POINT(33.81866424352339 -118.13261963847) bank25032 +25033 POINT(34.33548251520091 -117.37961513397141) bank25033 +25034 POINT(34.20023124397023 -118.48595041755664) bank25034 +25035 POINT(33.2011671451644 -118.9514628668085) bank25035 +25036 POINT(34.260632789957725 -117.77236170174069) bank25036 +25037 POINT(33.97825121497251 -118.6297336960275) bank25037 +25038 POINT(34.497747716213425 -118.4732870155037) bank25038 +25039 POINT(33.37095449933991 -118.19828220775169) bank25039 +25040 POINT(34.09348220654008 -119.22368826279002) bank25040 +25041 POINT(34.77344339754786 -118.70789265688984) bank25041 +25042 POINT(34.645330118712764 -118.39843566067782) bank25042 +25043 POINT(34.32571927745042 -118.26988336503499) bank25043 +25044 POINT(33.47334944601327 -117.86713880645134) bank25044 +25045 POINT(34.7630969220295 -117.67188115804484) bank25045 +25046 POINT(34.278359338326275 -118.6971978493771) bank25046 +25047 POINT(34.535161275275755 -118.84226819538095) bank25047 +25048 POINT(33.93359462923105 -117.82817511856541) bank25048 +25049 POINT(33.6056555040674 -118.47993132506507) bank25049 +25050 POINT(34.59277772881794 -117.9026898855237) bank25050 +25051 POINT(33.37991075404709 -117.36919176344439) bank25051 +25052 POINT(33.74982459133237 -117.73979160941224) bank25052 +25053 POINT(33.83214339226088 -118.37909617045614) bank25053 +25054 POINT(34.934537422416724 -117.79133428634302) bank25054 +25055 POINT(34.65110750980885 -117.536893289627) bank25055 +25056 POINT(33.561045126546325 -118.93013055207169) bank25056 +25057 POINT(34.68235873012338 -118.5347948250936) bank25057 +25058 POINT(34.06954665841557 -119.16496708238061) bank25058 +25059 POINT(34.6685532797662 -117.71101400553783) bank25059 +25060 POINT(33.83340367236729 -118.02886582670943) bank25060 +25061 POINT(34.43585776186161 -117.32048190043966) bank25061 +25062 POINT(33.717834533185695 -118.19899760457646) bank25062 +25063 POINT(34.95891254947072 -118.3377705535264) bank25063 +25064 POINT(33.48592960351682 -118.93199573525106) bank25064 +25065 POINT(33.211034836355545 -118.78440613913658) bank25065 +25066 POINT(34.90785758815274 -117.99893082406385) bank25066 +25067 POINT(34.384478176556755 -118.23437970457165) bank25067 +25068 POINT(34.86679399953303 -117.71681250723576) bank25068 +25069 POINT(33.919883669247405 -118.85084785371478) bank25069 +25070 POINT(33.523518874900184 -117.63223021897325) bank25070 +25071 POINT(35.034051120517866 -117.28386685078152) bank25071 +25072 POINT(33.33052659405022 -118.17381829825133) bank25072 +25073 POINT(34.61365641898365 -117.92098828459079) bank25073 +25074 POINT(33.705209376133965 -118.75784279993269) bank25074 +25075 POINT(34.70897495270983 -117.41760607565823) bank25075 +25076 POINT(34.46213642871253 -119.032539898979) bank25076 +25077 POINT(33.90731918742657 -118.921300093244) bank25077 +25078 POINT(33.278191239666405 -117.40528450491233) bank25078 +25079 POINT(34.26684609910695 -117.35261371769055) bank25079 +25080 POINT(34.74946371587306 -118.87676736314266) bank25080 +25081 POINT(34.72007180143691 -117.68907909034264) bank25081 +25082 POINT(34.18047698645027 -117.33384267720487) bank25082 +25083 POINT(34.624540569280796 -118.8992936123038) bank25083 +25084 POINT(34.74358509289779 -117.98415547295234) bank25084 +25085 POINT(33.19485406910531 -117.86001395610089) bank25085 +25086 POINT(33.612487220776444 -117.37068905402938) bank25086 +25087 POINT(33.34724884629222 -117.62069773773331) bank25087 +25088 POINT(34.565867417922476 -118.23428980915865) bank25088 +25089 POINT(33.84130151209273 -118.92064116447402) bank25089 +25090 POINT(34.3071975170199 -118.9230856123347) bank25090 +25091 POINT(33.19112986600058 -117.75755252463532) bank25091 +25092 POINT(33.69796884960923 -118.30822203158581) bank25092 +25093 POINT(33.21277764214462 -118.9446492693245) bank25093 +25094 POINT(33.624843133928664 -117.65751897936981) bank25094 +25095 POINT(34.514492783175875 -117.86627754490317) bank25095 +25096 POINT(33.56968179531207 -118.3945666871641) bank25096 +25097 POINT(34.69606707061453 -119.05539881452057) bank25097 +25098 POINT(33.388428817433145 -119.10179096299028) bank25098 +25099 POINT(33.617794933434006 -118.00720784632539) bank25099 +25100 POINT(34.150848745589414 -117.81782278249541) bank25100 +25101 POINT(33.178121462902105 -117.94097746154947) bank25101 +25102 POINT(34.130910121523485 -119.21653507099997) bank25102 +25103 POINT(33.10175756763546 -118.80832824263632) bank25103 +25104 POINT(33.18211565269452 -118.31633831901597) bank25104 +25105 POINT(33.96625780083623 -117.66289789419021) bank25105 +25106 POINT(34.75832963145234 -118.24044770120221) bank25106 +25107 POINT(34.49925146545905 -117.9315009393679) bank25107 +25108 POINT(33.95521656673331 -118.60792621249487) bank25108 +25109 POINT(34.01082691054263 -118.0860470819706) bank25109 +25110 POINT(34.12043288690744 -118.39747442196625) bank25110 +25111 POINT(34.192538383707806 -117.79249328550891) bank25111 +25112 POINT(33.71282763942927 -118.79803781562511) bank25112 +25113 POINT(33.93091634833221 -117.29130312934971) bank25113 +25114 POINT(34.0768514398761 -118.60359635751963) bank25114 +25115 POINT(34.59908815018794 -117.82077283171482) bank25115 +25116 POINT(34.07899243803639 -118.21298058642434) bank25116 +25117 POINT(33.18337835910232 -119.20650295911065) bank25117 +25118 POINT(34.933035460572846 -117.92499749234979) bank25118 +25119 POINT(34.45213573442092 -119.16464417187589) bank25119 +25120 POINT(34.962636675962024 -118.29114421981488) bank25120 +25121 POINT(34.4670630060592 -118.5871057747515) bank25121 +25122 POINT(34.25090934412613 -117.85279634697102) bank25122 +25123 POINT(33.54631556696369 -117.84466573184315) bank25123 +25124 POINT(34.52749638562208 -118.47977303133672) bank25124 +25125 POINT(33.270641083981644 -118.34790453204755) bank25125 +25126 POINT(34.21260255883216 -117.44173644438986) bank25126 +25127 POINT(34.947761763027145 -117.98189795706863) bank25127 +25128 POINT(34.367047113164546 -117.56928221350614) bank25128 +25129 POINT(34.31269668237628 -118.00277741773488) bank25129 +25130 POINT(34.05966468010924 -117.65200989334102) bank25130 +25131 POINT(34.19698114907601 -119.23943347904839) bank25131 +25132 POINT(34.610652226316034 -118.8577764950539) bank25132 +25133 POINT(34.51457637596561 -118.71518452264127) bank25133 +25134 POINT(33.84457099254071 -117.34974740524844) bank25134 +25135 POINT(33.73353768640674 -117.83867311703743) bank25135 +25136 POINT(33.51494309475852 -118.48082110383538) bank25136 +25137 POINT(33.27697585444924 -117.2938700055703) bank25137 +25138 POINT(33.799283517417926 -119.10797890715261) bank25138 +25139 POINT(34.2469061367087 -119.00148280555541) bank25139 +25140 POINT(33.6248439111742 -118.9642685899567) bank25140 +25141 POINT(34.67784635186497 -117.40487376347612) bank25141 +25142 POINT(34.49640715549907 -119.16556665499715) bank25142 +25143 POINT(33.36661001849751 -117.2439211223151) bank25143 +25144 POINT(33.55466521044493 -118.76667471251733) bank25144 +25145 POINT(33.68716507314878 -118.12631197138616) bank25145 +25146 POINT(33.313331602893825 -118.20246802068193) bank25146 +25147 POINT(34.54388591284636 -118.25489367959389) bank25147 +25148 POINT(34.91264831299575 -118.6158881360166) bank25148 +25149 POINT(34.16273378725727 -118.27799734361086) bank25149 +25150 POINT(34.23197645581003 -118.84604771717511) bank25150 +25151 POINT(33.87190149176905 -118.8697736634625) bank25151 +25152 POINT(33.632100160302066 -118.017631529139) bank25152 +25153 POINT(33.2302021328124 -118.9934831128693) bank25153 +25154 POINT(33.453103866501465 -117.41093008169038) bank25154 +25155 POINT(35.03877384266976 -117.39678081072965) bank25155 +25156 POINT(33.45807820885812 -118.01553687368941) bank25156 +25157 POINT(33.43076728639361 -118.29047557213737) bank25157 +25158 POINT(34.21821021393036 -118.35543733355223) bank25158 +25159 POINT(34.124575679037946 -117.64556860235633) bank25159 +25160 POINT(33.74122451097487 -118.68989437203075) bank25160 +25161 POINT(35.00925673970631 -117.39893700303782) bank25161 +25162 POINT(33.19492025437891 -118.54561099257832) bank25162 +25163 POINT(34.344028840850406 -118.01535207486683) bank25163 +25164 POINT(34.41820473044173 -117.94537646553864) bank25164 +25165 POINT(34.44645561835732 -119.08345961682357) bank25165 +25166 POINT(34.92683569310998 -117.89888866622533) bank25166 +25167 POINT(34.0502210955616 -118.06334133019115) bank25167 +25168 POINT(33.24556859802075 -117.92337641694341) bank25168 +25169 POINT(34.279164677661335 -117.66114259594795) bank25169 +25170 POINT(34.31510395177187 -119.15054558956389) bank25170 +25171 POINT(35.008437858832885 -118.76943702833277) bank25171 +25172 POINT(34.85607221184095 -118.54058988280345) bank25172 +25173 POINT(33.47086157001196 -119.1374718156797) bank25173 +25174 POINT(33.95227439339135 -118.11580745846591) bank25174 +25175 POINT(33.775045509738675 -118.03320557542632) bank25175 +25176 POINT(33.373558894927676 -118.56824342894858) bank25176 +25177 POINT(33.51264400976984 -118.42066799125881) bank25177 +25178 POINT(34.584338019601105 -117.49585704635737) bank25178 +25179 POINT(34.48365857208436 -118.41224760323107) bank25179 +25180 POINT(34.06886759794339 -118.30119303055177) bank25180 +25181 POINT(34.6815906192755 -118.42966688028805) bank25181 +25182 POINT(33.29872933566237 -119.04158984842695) bank25182 +25183 POINT(34.98363475394633 -117.52160476399278) bank25183 +25184 POINT(34.003537910354325 -118.02832593468922) bank25184 +25185 POINT(34.315749757326756 -118.4676020392083) bank25185 +25186 POINT(33.27515120153686 -118.30101736388708) bank25186 +25187 POINT(33.46927008421863 -118.79682240812937) bank25187 +25188 POINT(34.58737781693614 -118.12122640487105) bank25188 +25189 POINT(33.79789999730925 -118.44512252281663) bank25189 +25190 POINT(33.77780488774062 -118.70612443863982) bank25190 +25191 POINT(34.50262108326505 -117.49410192120311) bank25191 +25192 POINT(34.79319713919101 -119.02525191329774) bank25192 +25193 POINT(34.947266455237525 -118.37119586359363) bank25193 +25194 POINT(34.42026316433293 -119.11462067187404) bank25194 +25195 POINT(33.446825999874015 -117.71230935712765) bank25195 +25196 POINT(33.613106154435826 -117.55546807222939) bank25196 +25197 POINT(33.133590854667666 -117.74176809470809) bank25197 +25198 POINT(33.9636204330066 -119.17893854854083) bank25198 +25199 POINT(33.24951128319112 -119.02333418797359) bank25199 +25200 POINT(33.74269354492299 -117.98034179995761) bank25200 +25201 POINT(34.851504488528455 -119.02056488941018) bank25201 +25202 POINT(34.26017418724487 -118.9186325524946) bank25202 +25203 POINT(34.42154739834739 -119.12809124187024) bank25203 +25204 POINT(34.54482292004757 -117.61057816003834) bank25204 +25205 POINT(33.459086276769526 -117.99056115337054) bank25205 +25206 POINT(33.645714519911685 -119.14310515316545) bank25206 +25207 POINT(33.28460408153898 -118.44449133027246) bank25207 +25208 POINT(34.3392185222113 -119.22106980663243) bank25208 +25209 POINT(33.935575437476416 -117.5517509908864) bank25209 +25210 POINT(34.36874496139128 -117.50843220891511) bank25210 +25211 POINT(34.81263321634288 -117.83082701958233) bank25211 +25212 POINT(33.67395884327688 -118.47374366101883) bank25212 +25213 POINT(33.13998026279321 -117.72353971956724) bank25213 +25214 POINT(33.204035140111415 -117.28236812681276) bank25214 +25215 POINT(33.604078705436336 -118.02837960656642) bank25215 +25216 POINT(35.01103678626413 -117.61502861689723) bank25216 +25217 POINT(33.3806505133566 -118.1310547855631) bank25217 +25218 POINT(33.08836221654988 -118.27415197634276) bank25218 +25219 POINT(34.93436716898115 -118.80710952123776) bank25219 +25220 POINT(34.140373562554146 -118.58337632759307) bank25220 +25221 POINT(34.61890939987447 -118.04644769802722) bank25221 +25222 POINT(34.56424091057593 -119.14718692175592) bank25222 +25223 POINT(33.65684572936771 -117.53347986944664) bank25223 +25224 POINT(33.39518564718809 -117.26908659086605) bank25224 +25225 POINT(34.01372786504627 -118.76422812170628) bank25225 +25226 POINT(34.92722043683545 -118.04540722930626) bank25226 +25227 POINT(33.91812774936192 -117.94534426883978) bank25227 +25228 POINT(34.048249262826936 -117.42653271837861) bank25228 +25229 POINT(33.3626681879736 -118.65941755892389) bank25229 +25230 POINT(33.50979381589424 -118.28155223230992) bank25230 +25231 POINT(34.481265610835635 -117.85762928554671) bank25231 +25232 POINT(33.30633363142308 -119.00922941380671) bank25232 +25233 POINT(34.64113749947522 -117.59834217029345) bank25233 +25234 POINT(34.031710363814305 -117.35847377177875) bank25234 +25235 POINT(33.95863631720709 -119.1834760816422) bank25235 +25236 POINT(33.24996862612724 -119.14240464641172) bank25236 +25237 POINT(34.422103028920766 -118.26702076344606) bank25237 +25238 POINT(34.40190168279127 -117.64149312355637) bank25238 +25239 POINT(34.01721389618355 -117.48117281702874) bank25239 +25240 POINT(34.64439066851717 -117.91577292752073) bank25240 +25241 POINT(33.37464754951522 -117.58633130427903) bank25241 +25242 POINT(33.16007720800067 -118.39311053984753) bank25242 +25243 POINT(34.774907056602665 -117.4131614541797) bank25243 +25244 POINT(34.163636931368316 -118.6847594909846) bank25244 +25245 POINT(34.37966753239449 -118.76373086486112) bank25245 +25246 POINT(35.04716232074284 -119.05000788331944) bank25246 +25247 POINT(33.1265438558568 -118.76396557124676) bank25247 +25248 POINT(34.26120855091444 -119.01494690629086) bank25248 +25249 POINT(34.57868335601822 -117.70345739873814) bank25249 +25250 POINT(33.06441961726064 -117.36079794944486) bank25250 +25251 POINT(34.816595048083926 -117.54051880647607) bank25251 +25252 POINT(33.288336572885704 -118.7388540208811) bank25252 +25253 POINT(33.66662346671147 -118.12828117423568) bank25253 +25254 POINT(34.40786078186203 -119.1792568084131) bank25254 +25255 POINT(34.76637156221043 -117.77202383283633) bank25255 +25256 POINT(34.152190113464755 -117.4305372874741) bank25256 +25257 POINT(33.37479965890485 -118.21966251218153) bank25257 +25258 POINT(34.09477225036603 -118.35202748214607) bank25258 +25259 POINT(33.48706368655542 -119.11332617621684) bank25259 +25260 POINT(33.68817268070604 -117.75234241927274) bank25260 +25261 POINT(34.83018457689104 -118.14470917772316) bank25261 +25262 POINT(34.219530350024414 -117.47188729804742) bank25262 +25263 POINT(34.145401409754 -118.89832204859366) bank25263 +25264 POINT(34.64371040090121 -118.01934242163082) bank25264 +25265 POINT(34.964181298235616 -118.01019326531116) bank25265 +25266 POINT(35.02315080532294 -117.82230487731537) bank25266 +25267 POINT(33.878113430363236 -117.52313439062729) bank25267 +25268 POINT(33.7962268271812 -118.1732058016312) bank25268 +25269 POINT(33.482831782551294 -118.24812928291226) bank25269 +25270 POINT(33.60700226490042 -117.53391724111071) bank25270 +25271 POINT(33.14558485470666 -118.58691713000518) bank25271 +25272 POINT(33.856521720948685 -118.54071648683463) bank25272 +25273 POINT(33.54058014689299 -118.3093974051377) bank25273 +25274 POINT(35.04526328256046 -117.45826024351453) bank25274 +25275 POINT(34.47871238090628 -118.38215335916473) bank25275 +25276 POINT(33.47821654460827 -117.5152388628113) bank25276 +25277 POINT(34.023250206365766 -117.69428525503226) bank25277 +25278 POINT(34.21334036911645 -117.42491619264484) bank25278 +25279 POINT(33.098666570761935 -118.02277736915644) bank25279 +25280 POINT(34.9035059928912 -118.9874503469576) bank25280 +25281 POINT(33.76075122539232 -118.89252293195885) bank25281 +25282 POINT(33.344527746538546 -118.11040576158268) bank25282 +25283 POINT(33.7879735136136 -117.50839460651618) bank25283 +25284 POINT(34.745884961849306 -117.73669037134839) bank25284 +25285 POINT(34.18775899359619 -118.68340031614942) bank25285 +25286 POINT(34.53237237759388 -117.94847989432509) bank25286 +25287 POINT(34.83648696717411 -118.5642063376064) bank25287 +25288 POINT(33.86869516615522 -118.25693978742949) bank25288 +25289 POINT(33.93433985622464 -117.46545201902059) bank25289 +25290 POINT(33.06396894269538 -117.92046664608216) bank25290 +25291 POINT(34.51591550791341 -117.28494990167029) bank25291 +25292 POINT(34.19533932490592 -117.4512701703808) bank25292 +25293 POINT(33.94112145991379 -119.19616954592284) bank25293 +25294 POINT(33.799662859595045 -119.08239147462916) bank25294 +25295 POINT(33.24666105175722 -118.19636820551844) bank25295 +25296 POINT(33.90744578323872 -117.86630096687719) bank25296 +25297 POINT(35.01724401258452 -118.42225259717924) bank25297 +25298 POINT(34.719272587885875 -118.94376230976191) bank25298 +25299 POINT(33.17296739581556 -119.13757940896869) bank25299 +25300 POINT(33.596170173853956 -118.63177234387454) bank25300 +25301 POINT(34.457723289886026 -117.38428087946475) bank25301 +25302 POINT(34.038636900884065 -117.79592958303708) bank25302 +25303 POINT(34.52894302820021 -118.81790613589824) bank25303 +25304 POINT(33.244572787653894 -118.61936565598543) bank25304 +25305 POINT(34.69105394002053 -117.74814394156945) bank25305 +25306 POINT(34.60683425538101 -117.99513136022635) bank25306 +25307 POINT(34.2751082069383 -119.05261404317255) bank25307 +25308 POINT(33.123651816862484 -117.56294702399703) bank25308 +25309 POINT(33.81489821839264 -119.12208022589259) bank25309 +25310 POINT(33.32937354752261 -119.14702593223971) bank25310 +25311 POINT(33.73156861064943 -119.20583441730413) bank25311 +25312 POINT(34.39371156121897 -118.11795244672801) bank25312 +25313 POINT(35.047892209613934 -117.34183946479487) bank25313 +25314 POINT(33.3271379779845 -117.64268734477187) bank25314 +25315 POINT(34.20722533650726 -118.83306223874145) bank25315 +25316 POINT(34.84864101662575 -118.36463633955935) bank25316 +25317 POINT(34.704316297223194 -118.45967870940024) bank25317 +25318 POINT(33.56349293397808 -118.20699709437635) bank25318 +25319 POINT(33.71163718589339 -118.41782100725386) bank25319 +25320 POINT(34.16372837412596 -118.63166179414209) bank25320 +25321 POINT(34.25143946825589 -117.56471208887841) bank25321 +25322 POINT(34.66151672397942 -117.78154959000483) bank25322 +25323 POINT(34.66571220553979 -118.56831851773998) bank25323 +25324 POINT(33.67071612065254 -117.46420533507673) bank25324 +25325 POINT(34.335024936604086 -118.11634544796715) bank25325 +25326 POINT(34.209259562541355 -117.87592060154775) bank25326 +25327 POINT(33.59869916397417 -117.28388166145946) bank25327 +25328 POINT(33.217270544803945 -118.98738702072328) bank25328 +25329 POINT(34.72962940457097 -117.45499906951329) bank25329 +25330 POINT(35.04688225645357 -117.5810958847832) bank25330 +25331 POINT(34.039458447409956 -119.15561764937314) bank25331 +25332 POINT(34.041396464088294 -118.48793068101662) bank25332 +25333 POINT(33.11410217783511 -117.9793399981761) bank25333 +25334 POINT(33.74143207977775 -117.3760310154411) bank25334 +25335 POINT(34.56667461390052 -117.93589973549774) bank25335 +25336 POINT(33.94132005213948 -117.35903171349786) bank25336 +25337 POINT(33.69921800355286 -118.94296497000325) bank25337 +25338 POINT(34.71609230377305 -118.48702599938103) bank25338 +25339 POINT(33.87085169435199 -119.11094254489281) bank25339 +25340 POINT(33.15766347050698 -118.09603261962955) bank25340 +25341 POINT(35.05111693135317 -117.37411148420975) bank25341 +25342 POINT(33.95201791818182 -118.96079608222753) bank25342 +25343 POINT(33.68840311731515 -118.29638070518321) bank25343 +25344 POINT(34.921378646123294 -118.16224868509377) bank25344 +25345 POINT(33.3816279611444 -119.2260142134071) bank25345 +25346 POINT(34.629185970872925 -117.30932792138987) bank25346 +25347 POINT(33.460460490536725 -117.7401806109026) bank25347 +25348 POINT(34.8824578157691 -118.69047145785225) bank25348 +25349 POINT(33.14797917325224 -117.6732962107163) bank25349 +25350 POINT(33.19529783258293 -118.23989146141606) bank25350 +25351 POINT(34.90878664849885 -118.74189646750025) bank25351 +25352 POINT(33.65652669558537 -118.93262208602671) bank25352 +25353 POINT(33.93171239144477 -117.68637074513863) bank25353 +25354 POINT(33.53756823234601 -117.73780355810717) bank25354 +25355 POINT(33.89331796166979 -117.90515306785231) bank25355 +25356 POINT(33.577989782757676 -118.32217619660076) bank25356 +25357 POINT(33.097960225975754 -117.6778900460208) bank25357 +25358 POINT(33.87342754596417 -117.61369277074273) bank25358 +25359 POINT(33.717506003485944 -117.42800813142678) bank25359 +25360 POINT(33.341285515754834 -119.2105050244998) bank25360 +25361 POINT(33.771735678442234 -118.25406530008881) bank25361 +25362 POINT(34.13383956504262 -118.71668532481029) bank25362 +25363 POINT(34.07043592083216 -117.75345212977595) bank25363 +25364 POINT(34.69499103130073 -118.99402793477944) bank25364 +25365 POINT(34.50891979366392 -118.1282561884859) bank25365 +25366 POINT(34.50680145011808 -117.29795099494949) bank25366 +25367 POINT(34.72290561439789 -117.64564786734712) bank25367 +25368 POINT(33.899924117115944 -119.09458000211782) bank25368 +25369 POINT(33.115586436823506 -117.93167356305369) bank25369 +25370 POINT(33.45068099249724 -118.08581069220523) bank25370 +25371 POINT(33.53699660767501 -118.53002469494052) bank25371 +25372 POINT(34.84042333087993 -118.73112213060212) bank25372 +25373 POINT(34.35134510213376 -117.92944954607408) bank25373 +25374 POINT(33.58905109315973 -117.36574902022257) bank25374 +25375 POINT(33.879022026859396 -119.17030632120246) bank25375 +25376 POINT(34.98249498598379 -118.61013125680597) bank25376 +25377 POINT(33.15510008001231 -117.6544273227583) bank25377 +25378 POINT(34.959588916889004 -118.63844548495305) bank25378 +25379 POINT(34.59872474524769 -118.26133112533532) bank25379 +25380 POINT(33.64427057731376 -117.42043045093416) bank25380 +25381 POINT(34.725626160914196 -117.95506017051565) bank25381 +25382 POINT(33.808941304699935 -118.34320406803535) bank25382 +25383 POINT(33.57188995012161 -118.50825583331957) bank25383 +25384 POINT(34.65132316726863 -118.61445115193723) bank25384 +25385 POINT(33.50328972284088 -118.48527064607411) bank25385 +25386 POINT(34.26141407667261 -118.50137028803414) bank25386 +25387 POINT(33.515776582171746 -118.51304903277374) bank25387 +25388 POINT(33.303272641681275 -119.03107226422792) bank25388 +25389 POINT(33.62322128983801 -118.48254858243979) bank25389 +25390 POINT(34.1245992465572 -118.86921354805337) bank25390 +25391 POINT(33.20890983624822 -118.56059659650782) bank25391 +25392 POINT(34.16727581174912 -118.34080745008282) bank25392 +25393 POINT(34.161683597588 -117.83535098953448) bank25393 +25394 POINT(34.49478832851029 -118.64139733172517) bank25394 +25395 POINT(33.09383909289578 -117.9150002618841) bank25395 +25396 POINT(33.30270553126621 -118.83496256965485) bank25396 +25397 POINT(33.602672515171385 -117.75239138776442) bank25397 +25398 POINT(33.908916292009934 -118.9543779035488) bank25398 +25399 POINT(34.15611762602184 -117.2613403474624) bank25399 +25400 POINT(33.997448107079855 -118.02195718860386) bank25400 +25401 POINT(35.01020528678855 -117.99251056385253) bank25401 +25402 POINT(34.04717568053056 -118.83837433490254) bank25402 +25403 POINT(33.28830123692837 -118.49800758624492) bank25403 +25404 POINT(34.11515787331355 -118.43865848683147) bank25404 +25405 POINT(34.39734143893954 -117.79024896354417) bank25405 +25406 POINT(33.78427935229223 -119.02859975328062) bank25406 +25407 POINT(33.14952508737403 -117.92986353718227) bank25407 +25408 POINT(33.3109141048336 -117.56637792591138) bank25408 +25409 POINT(34.89482826493461 -118.4716574988417) bank25409 +25410 POINT(34.37715176274677 -118.4497947737605) bank25410 +25411 POINT(34.61075931061489 -117.47981241266511) bank25411 +25412 POINT(33.45707733468131 -118.13161024217074) bank25412 +25413 POINT(33.4435621226896 -117.92549480415046) bank25413 +25414 POINT(34.914284550016845 -118.52516710746602) bank25414 +25415 POINT(34.03414948935154 -119.00331812473155) bank25415 +25416 POINT(34.53088082223073 -117.56381100171164) bank25416 +25417 POINT(33.991182632407806 -118.7150450340584) bank25417 +25418 POINT(33.40287203391146 -117.27370516510047) bank25418 +25419 POINT(34.405326207014625 -118.43278282236975) bank25419 +25420 POINT(34.407458894257175 -119.13933153170323) bank25420 +25421 POINT(34.473490895534134 -119.0372645660668) bank25421 +25422 POINT(33.77031778125592 -117.31378233330149) bank25422 +25423 POINT(33.52125349685933 -118.64769236643869) bank25423 +25424 POINT(34.121837724283694 -118.21517361129244) bank25424 +25425 POINT(33.281411082282126 -117.51272600513624) bank25425 +25426 POINT(35.00667727065718 -119.14815136118895) bank25426 +25427 POINT(34.0352563816432 -117.79641302021246) bank25427 +25428 POINT(34.3585765079691 -117.81992330710634) bank25428 +25429 POINT(33.77870631568008 -118.78873910178083) bank25429 +25430 POINT(34.03478117764803 -118.11671383001627) bank25430 +25431 POINT(33.2841157579182 -117.51749712518142) bank25431 +25432 POINT(34.94423581720782 -118.75144495515556) bank25432 +25433 POINT(34.18928683798831 -118.6436865493138) bank25433 +25434 POINT(33.45233151525061 -118.33526705153257) bank25434 +25435 POINT(33.79843286165309 -118.39371434124388) bank25435 +25436 POINT(33.999337186791124 -118.0528372390441) bank25436 +25437 POINT(34.35432774117419 -118.61110611166012) bank25437 +25438 POINT(33.846098249399105 -118.51441365858369) bank25438 +25439 POINT(33.7167938728502 -117.84320567551805) bank25439 +25440 POINT(33.61642017201214 -117.74172607820798) bank25440 +25441 POINT(33.29686251742267 -118.03877081992727) bank25441 +25442 POINT(33.81792187446795 -117.8542639661565) bank25442 +25443 POINT(35.014183432982314 -117.83235547818788) bank25443 +25444 POINT(34.71267017839207 -117.29337142981097) bank25444 +25445 POINT(33.20025707100548 -118.47557574111336) bank25445 +25446 POINT(34.06702327410689 -117.45029407420854) bank25446 +25447 POINT(34.94318199507681 -119.05216875532513) bank25447 +25448 POINT(33.522770010820004 -118.27359446874529) bank25448 +25449 POINT(33.33924906158663 -118.43192685381828) bank25449 +25450 POINT(33.39411749939217 -117.71832946314838) bank25450 +25451 POINT(33.07542728059806 -119.19988204334896) bank25451 +25452 POINT(33.891974742470744 -117.7698331989171) bank25452 +25453 POINT(35.033354323314796 -117.66325394522194) bank25453 +25454 POINT(34.02863760833676 -119.2242450312485) bank25454 +25455 POINT(33.39871743799633 -118.28991575368126) bank25455 +25456 POINT(34.80825832654647 -118.8971537609772) bank25456 +25457 POINT(33.159896156454764 -118.84521726885117) bank25457 +25458 POINT(34.092830974230544 -117.36658947884209) bank25458 +25459 POINT(33.760401632048115 -118.81505604108652) bank25459 +25460 POINT(33.934590873564275 -118.32047106903939) bank25460 +25461 POINT(33.92196545751723 -119.021654759007) bank25461 +25462 POINT(34.67481528583633 -119.23674838294986) bank25462 +25463 POINT(34.837786582584926 -117.73980421079071) bank25463 +25464 POINT(33.525887915820704 -119.18741824212324) bank25464 +25465 POINT(33.111152632776175 -117.44339781760098) bank25465 +25466 POINT(34.27482163652299 -117.61990558994134) bank25466 +25467 POINT(33.30751207086011 -117.73945004179073) bank25467 +25468 POINT(34.085828677986626 -118.63173897364041) bank25468 +25469 POINT(34.85089302238465 -118.56204472238204) bank25469 +25470 POINT(34.05914132423584 -118.5816272137236) bank25470 +25471 POINT(34.124938730760896 -118.99152432243282) bank25471 +25472 POINT(33.291418437581385 -118.98514503946548) bank25472 +25473 POINT(34.56334108662474 -118.15920163886035) bank25473 +25474 POINT(33.61243009010935 -117.80399666837515) bank25474 +25475 POINT(34.926355558817484 -118.93315723734382) bank25475 +25476 POINT(33.27365493850441 -117.70721879188275) bank25476 +25477 POINT(34.65976128455171 -117.35056038784846) bank25477 +25478 POINT(34.8478046320077 -117.83947095234169) bank25478 +25479 POINT(34.83517942552681 -118.36726555380255) bank25479 +25480 POINT(33.067142474286804 -118.29030416374162) bank25480 +25481 POINT(33.355500500795536 -119.02823580444502) bank25481 +25482 POINT(33.27712248237522 -117.5385320019362) bank25482 +25483 POINT(34.595291549495286 -117.97447893483559) bank25483 +25484 POINT(34.77784221832757 -117.76828125698437) bank25484 +25485 POINT(34.27448840612984 -117.84377565835777) bank25485 +25486 POINT(33.78961923867973 -117.81200951921855) bank25486 +25487 POINT(33.4091476867597 -118.59022619606966) bank25487 +25488 POINT(34.985179587277834 -117.54423686547472) bank25488 +25489 POINT(33.892002072317005 -118.94997550179714) bank25489 +25490 POINT(33.995767778515535 -117.8971919464213) bank25490 +25491 POINT(33.91325848700764 -117.53675021804167) bank25491 +25492 POINT(34.259386463022125 -117.50315477180894) bank25492 +25493 POINT(34.98054945041697 -118.55026486956184) bank25493 +25494 POINT(34.87369723225095 -118.48962917433637) bank25494 +25495 POINT(34.85767679141285 -118.12584811256882) bank25495 +25496 POINT(33.319112029637814 -118.82608691118016) bank25496 +25497 POINT(33.322168882172825 -118.78196058353205) bank25497 +25498 POINT(33.091149461812606 -118.66711885341203) bank25498 +25499 POINT(34.04413473500899 -117.8997443307636) bank25499 +25500 POINT(34.81863542983309 -117.89150293240347) bank25500 +25501 POINT(35.019714017820135 -118.30145081421814) bank25501 +25502 POINT(33.17844264587117 -119.08200510216724) bank25502 +25503 POINT(33.51680937985471 -117.7410065965898) bank25503 +25504 POINT(34.13921353916671 -119.20411978622805) bank25504 +25505 POINT(34.8924515570741 -118.1024580044107) bank25505 +25506 POINT(34.885450528603755 -117.42346005718566) bank25506 +25507 POINT(34.3492199210973 -118.77656331695349) bank25507 +25508 POINT(33.174261540250754 -117.50011629954687) bank25508 +25509 POINT(34.80234790260923 -118.1306341399452) bank25509 +25510 POINT(34.31059126260382 -118.21959079505903) bank25510 +25511 POINT(33.297667988497636 -118.66540195208654) bank25511 +25512 POINT(34.28486968291218 -117.36821612326011) bank25512 +25513 POINT(34.854347478652386 -118.31889472513544) bank25513 +25514 POINT(34.794371877878255 -117.9751748596063) bank25514 +25515 POINT(33.21933146029156 -119.20961949362389) bank25515 +25516 POINT(33.41306177314339 -117.48593791600398) bank25516 +25517 POINT(34.33028404572091 -117.50234892250346) bank25517 +25518 POINT(33.30219562198716 -118.5385178499943) bank25518 +25519 POINT(33.6485980599622 -117.38233448648364) bank25519 +25520 POINT(34.08123395381505 -118.42238200581362) bank25520 +25521 POINT(33.88247764952108 -118.89163537123582) bank25521 +25522 POINT(34.882573525169796 -117.81228702737457) bank25522 +25523 POINT(34.072194452645135 -117.35668658964325) bank25523 +25524 POINT(33.15835733112402 -117.91264157970365) bank25524 +25525 POINT(34.37887972611983 -117.26007478404773) bank25525 +25526 POINT(34.70820126656308 -118.28629976101212) bank25526 +25527 POINT(34.53061831587949 -117.41716156109183) bank25527 +25528 POINT(34.00488795824847 -117.7961728391079) bank25528 +25529 POINT(33.56367004857 -117.48510154705474) bank25529 +25530 POINT(33.75859699272115 -117.95013318808599) bank25530 +25531 POINT(33.219840136713614 -118.46098351733751) bank25531 +25532 POINT(33.69946792893905 -119.1626870405116) bank25532 +25533 POINT(34.70292562386405 -118.71418583536642) bank25533 +25534 POINT(34.04986868838731 -118.0008551175363) bank25534 +25535 POINT(34.32613566228736 -117.60366087167718) bank25535 +25536 POINT(33.818691592295735 -119.16570671845747) bank25536 +25537 POINT(33.2922645155104 -118.65409368677498) bank25537 +25538 POINT(33.28165470524509 -118.1161742822884) bank25538 +25539 POINT(34.722997372412124 -118.36082296498651) bank25539 +25540 POINT(33.3973782837235 -118.26882273962005) bank25540 +25541 POINT(33.46304905151696 -117.77020340235323) bank25541 +25542 POINT(34.13081797204678 -118.87897491054646) bank25542 +25543 POINT(34.87765197424858 -117.82725566161629) bank25543 +25544 POINT(34.69387966412326 -117.81227981648723) bank25544 +25545 POINT(33.68218401797732 -119.02580666388934) bank25545 +25546 POINT(33.89669753405735 -118.3220093685747) bank25546 +25547 POINT(34.17082465534549 -117.37804871623678) bank25547 +25548 POINT(33.626292677343784 -118.40980153052907) bank25548 +25549 POINT(33.24313098395332 -117.47923649092768) bank25549 +25550 POINT(33.95254628518171 -118.03860194786499) bank25550 +25551 POINT(33.25103755114496 -118.21674914357475) bank25551 +25552 POINT(34.8372579191198 -117.29066941366382) bank25552 +25553 POINT(34.363177624637466 -119.11550348881727) bank25553 +25554 POINT(33.93364843712753 -117.54284710275755) bank25554 +25555 POINT(34.800401697176724 -117.3383910783912) bank25555 +25556 POINT(34.008445413967436 -118.73352274566275) bank25556 +25557 POINT(33.26323561153387 -117.73984016490807) bank25557 +25558 POINT(34.78172959859221 -118.86327468797768) bank25558 +25559 POINT(33.58931892902266 -119.12473535654588) bank25559 +25560 POINT(33.680175635504185 -118.43053919775643) bank25560 +25561 POINT(33.52494651648932 -119.14410203802149) bank25561 +25562 POINT(33.08638255709758 -119.13860584913955) bank25562 +25563 POINT(34.49325184780142 -118.3192440276764) bank25563 +25564 POINT(34.055005067340886 -118.71086873632869) bank25564 +25565 POINT(33.651149271970766 -117.56733048456658) bank25565 +25566 POINT(33.2937051969698 -117.33220001425921) bank25566 +25567 POINT(33.940242366821764 -117.37762348685703) bank25567 +25568 POINT(33.288135746045064 -117.98130547057065) bank25568 +25569 POINT(34.43620463162259 -117.69258704279984) bank25569 +25570 POINT(34.37155338431265 -117.4128001669194) bank25570 +25571 POINT(34.22418480975401 -118.65280649460401) bank25571 +25572 POINT(33.172361912026126 -118.07740208310398) bank25572 +25573 POINT(33.18577936349296 -118.48789720576667) bank25573 +25574 POINT(33.221074459925624 -117.4406797914995) bank25574 +25575 POINT(33.310454073648245 -118.99792313580532) bank25575 +25576 POINT(34.95296782854325 -118.14273445186059) bank25576 +25577 POINT(35.01715974417723 -118.21412267102008) bank25577 +25578 POINT(33.29313410033337 -118.25819171912023) bank25578 +25579 POINT(34.322550076834084 -117.7808602020331) bank25579 +25580 POINT(34.11000938053345 -118.07112766416572) bank25580 +25581 POINT(34.29556673592167 -119.11176537110168) bank25581 +25582 POINT(34.26091235405722 -118.50225542838002) bank25582 +25583 POINT(34.888758746510916 -117.76673579462162) bank25583 +25584 POINT(34.946494141840574 -117.9286344674752) bank25584 +25585 POINT(33.62184293368848 -117.54829848824765) bank25585 +25586 POINT(34.81815946159161 -117.69309746977251) bank25586 +25587 POINT(33.46204233992586 -118.3826852933851) bank25587 +25588 POINT(34.639227816255776 -117.61264849590725) bank25588 +25589 POINT(34.15802537732647 -117.9238516732662) bank25589 +25590 POINT(34.40679364893242 -118.62114654400499) bank25590 +25591 POINT(33.859681214705574 -118.95654889903689) bank25591 +25592 POINT(34.683139500923936 -117.80405392964884) bank25592 +25593 POINT(33.83127134360616 -119.07089995857237) bank25593 +25594 POINT(33.053808799086305 -118.2117386210615) bank25594 +25595 POINT(34.29112973099369 -119.14477961930818) bank25595 +25596 POINT(33.45644834887739 -119.13157827941755) bank25596 +25597 POINT(34.8983142332966 -119.08405676273419) bank25597 +25598 POINT(34.46501872298914 -117.61940133889549) bank25598 +25599 POINT(34.953125893723026 -118.68508469416442) bank25599 +25600 POINT(33.171739180722895 -118.92813014391497) bank25600 +25601 POINT(33.9414946759623 -117.69089622465404) bank25601 +25602 POINT(33.71649233461685 -117.74755227558481) bank25602 +25603 POINT(34.453747602307985 -118.79236468030356) bank25603 +25604 POINT(34.02623207843393 -117.88779350589508) bank25604 +25605 POINT(33.09565372433447 -117.43351276253549) bank25605 +25606 POINT(34.205140509169176 -117.58580394510233) bank25606 +25607 POINT(33.625536037491464 -117.98381407867717) bank25607 +25608 POINT(33.56879647178246 -118.64709367471592) bank25608 +25609 POINT(34.434852642219724 -117.4543880996099) bank25609 +25610 POINT(33.66716049048141 -117.41859197142082) bank25610 +25611 POINT(34.26299927393241 -117.3103100707643) bank25611 +25612 POINT(33.23641524809913 -119.22952004372381) bank25612 +25613 POINT(34.969458023515536 -119.02207929928217) bank25613 +25614 POINT(34.38144417964005 -117.90673775922401) bank25614 +25615 POINT(34.814680808719714 -117.81567003804751) bank25615 +25616 POINT(34.52912919684023 -119.14934792610927) bank25616 +25617 POINT(34.35206014593195 -118.09918919132366) bank25617 +25618 POINT(33.4030285695826 -117.91258424578044) bank25618 +25619 POINT(33.261396735547635 -118.76409060872199) bank25619 +25620 POINT(33.728447763888724 -119.11285501936585) bank25620 +25621 POINT(34.31373917039667 -118.74219231321315) bank25621 +25622 POINT(33.58806180762315 -118.91018694389606) bank25622 +25623 POINT(34.73688791131112 -117.64254790140753) bank25623 +25624 POINT(34.98080009481867 -117.78673205401283) bank25624 +25625 POINT(33.99812688771961 -118.81948570443284) bank25625 +25626 POINT(34.78223957004121 -118.89909345997438) bank25626 +25627 POINT(33.06567337858562 -119.10022937228388) bank25627 +25628 POINT(33.71577279233704 -118.07569930544558) bank25628 +25629 POINT(33.60344354694555 -119.13823480535316) bank25629 +25630 POINT(33.47792930520415 -118.36585768728692) bank25630 +25631 POINT(33.320127051603784 -118.20848101569226) bank25631 +25632 POINT(33.69634664542948 -117.31849665852727) bank25632 +25633 POINT(33.6868534883049 -117.90752188309655) bank25633 +25634 POINT(33.61557852802685 -118.13387302700663) bank25634 +25635 POINT(34.20265137557215 -117.77947300011586) bank25635 +25636 POINT(34.435465458150524 -117.41064481029892) bank25636 +25637 POINT(34.23505966325603 -118.73862950173344) bank25637 +25638 POINT(33.4606010322802 -118.14164661251955) bank25638 +25639 POINT(34.1475658435999 -118.21875031429678) bank25639 +25640 POINT(33.145763758537484 -118.73187684001422) bank25640 +25641 POINT(34.14843826630747 -118.46280023743266) bank25641 +25642 POINT(33.359663238765286 -118.66985966837237) bank25642 +25643 POINT(33.25166217197274 -119.07241147337594) bank25643 +25644 POINT(33.099562103796195 -118.69916167033196) bank25644 +25645 POINT(33.854354175094024 -117.86218685431388) bank25645 +25646 POINT(34.676311467509166 -119.24260774671426) bank25646 +25647 POINT(34.73551292922546 -118.88692760987949) bank25647 +25648 POINT(33.5513773610185 -117.47948353597957) bank25648 +25649 POINT(34.16258566820113 -118.80481162580551) bank25649 +25650 POINT(34.53185207184892 -118.639855986111) bank25650 +25651 POINT(33.13607511645598 -118.42555642170126) bank25651 +25652 POINT(33.39607452991015 -119.12300604046528) bank25652 +25653 POINT(33.54665062132694 -118.04000594265206) bank25653 +25654 POINT(34.61208014964235 -117.50605583331351) bank25654 +25655 POINT(34.046388843863554 -117.66927635060125) bank25655 +25656 POINT(33.44549289602487 -117.25001683070217) bank25656 +25657 POINT(33.69185002133203 -118.45839762223045) bank25657 +25658 POINT(34.79986988059047 -117.7009269555106) bank25658 +25659 POINT(33.23149764943516 -118.09473747203276) bank25659 +25660 POINT(33.696599339905184 -119.06815594780701) bank25660 +25661 POINT(34.887483272291476 -117.39646711343427) bank25661 +25662 POINT(34.688338169384046 -118.07282472961721) bank25662 +25663 POINT(34.85488583265402 -117.58745678655163) bank25663 +25664 POINT(33.82176336065206 -117.40932408921563) bank25664 +25665 POINT(33.78568667693314 -117.99229995617115) bank25665 +25666 POINT(33.569939513511926 -118.6862344825506) bank25666 +25667 POINT(34.88466322546022 -117.49216929666824) bank25667 +25668 POINT(34.48843514176855 -117.43247250077559) bank25668 +25669 POINT(34.96729642492083 -117.86064396365775) bank25669 +25670 POINT(33.10608382866034 -119.00930629669068) bank25670 +25671 POINT(33.375007072874055 -117.50346963773643) bank25671 +25672 POINT(33.960973185204196 -118.47019682058931) bank25672 +25673 POINT(33.8469134037581 -118.08532289267718) bank25673 +25674 POINT(33.61030545323901 -118.64200537365618) bank25674 +25675 POINT(34.728584254813754 -117.29686749064302) bank25675 +25676 POINT(33.24061428236816 -117.40678284668081) bank25676 +25677 POINT(33.72578154373978 -118.88148323199125) bank25677 +25678 POINT(33.4922032810189 -117.4714429160003) bank25678 +25679 POINT(33.574326091727634 -118.86254623796282) bank25679 +25680 POINT(34.18120760355092 -118.0596893491322) bank25680 +25681 POINT(33.17655554906896 -117.64085502273171) bank25681 +25682 POINT(33.52021433768951 -118.02937716862075) bank25682 +25683 POINT(34.85620844896846 -118.95226383702699) bank25683 +25684 POINT(34.58484367398221 -117.79540948145458) bank25684 +25685 POINT(34.76580842014117 -117.51180756397002) bank25685 +25686 POINT(33.75159634053896 -119.14433649574428) bank25686 +25687 POINT(34.44661463989597 -119.20819043754314) bank25687 +25688 POINT(33.991410873846554 -118.4367505727769) bank25688 +25689 POINT(34.025397956045786 -117.71375248258862) bank25689 +25690 POINT(34.000653847172586 -118.63989167854223) bank25690 +25691 POINT(33.88943050791191 -119.10577416128206) bank25691 +25692 POINT(34.08476061246764 -119.0472122859169) bank25692 +25693 POINT(33.48979720831954 -118.57578097187015) bank25693 +25694 POINT(34.59164420693052 -119.18619582815418) bank25694 +25695 POINT(33.95259792401154 -119.19969498605127) bank25695 +25696 POINT(34.08243074945034 -117.53549027352567) bank25696 +25697 POINT(34.1371128906069 -117.96867345591689) bank25697 +25698 POINT(33.05262275981084 -118.00053879207042) bank25698 +25699 POINT(33.86697171674831 -118.59937342195758) bank25699 +25700 POINT(33.24766094840094 -118.54148621883309) bank25700 +25701 POINT(34.61769870021766 -118.42523087492975) bank25701 +25702 POINT(33.79086213849885 -118.59567340646473) bank25702 +25703 POINT(33.46827789283923 -118.09553969076141) bank25703 +25704 POINT(33.084713151502214 -118.4870889039645) bank25704 +25705 POINT(34.6009159089621 -118.01995703938466) bank25705 +25706 POINT(33.91553435266432 -117.96742047373893) bank25706 +25707 POINT(33.12315257951082 -118.81959651349166) bank25707 +25708 POINT(33.85732480956616 -118.138700681857) bank25708 +25709 POINT(33.65840016487843 -117.33819270106349) bank25709 +25710 POINT(34.085023324812305 -118.63704492606105) bank25710 +25711 POINT(33.524179523442555 -119.0555607038116) bank25711 +25712 POINT(33.90669632542875 -117.86179918207321) bank25712 +25713 POINT(33.99191275097167 -118.91483209932005) bank25713 +25714 POINT(34.78749152484007 -118.1224998279104) bank25714 +25715 POINT(34.52567217744308 -117.49338174855522) bank25715 +25716 POINT(34.37705006695115 -117.93159389477317) bank25716 +25717 POINT(33.524664308900334 -117.33267103940652) bank25717 +25718 POINT(33.42903590266061 -118.2624777105019) bank25718 +25719 POINT(33.73037457667045 -118.93508854792879) bank25719 +25720 POINT(33.287728278102875 -118.50687295982256) bank25720 +25721 POINT(34.4540851814365 -117.43037674515507) bank25721 +25722 POINT(34.00662897167173 -118.34340754710674) bank25722 +25723 POINT(33.243896635222875 -118.7411966545294) bank25723 +25724 POINT(34.07472198933292 -117.89114129934863) bank25724 +25725 POINT(34.126013207488164 -119.23070973465938) bank25725 +25726 POINT(34.53188913540564 -118.6811394352213) bank25726 +25727 POINT(33.59839307833784 -118.00562480685065) bank25727 +25728 POINT(34.910632996209365 -117.26656463394121) bank25728 +25729 POINT(33.348128478896186 -117.8750117665327) bank25729 +25730 POINT(33.62297348348297 -118.82425916031438) bank25730 +25731 POINT(33.63579318357922 -117.47859130260119) bank25731 +25732 POINT(33.28302319912134 -117.40488758937055) bank25732 +25733 POINT(33.277940320315835 -118.7927837144677) bank25733 +25734 POINT(34.482209708031114 -117.31196141817247) bank25734 +25735 POINT(34.794399324915425 -119.14393215553986) bank25735 +25736 POINT(34.71740764895104 -117.2665064265487) bank25736 +25737 POINT(33.699892072678026 -118.79179053078556) bank25737 +25738 POINT(33.5631676674997 -117.61687953054623) bank25738 +25739 POINT(33.640268125683406 -118.10451896367199) bank25739 +25740 POINT(33.400245576896204 -117.37551712278491) bank25740 +25741 POINT(34.81429615397353 -117.76797572240469) bank25741 +25742 POINT(33.89847471871862 -118.28994976833327) bank25742 +25743 POINT(33.797015925487194 -117.40706436882652) bank25743 +25744 POINT(33.931194327977096 -118.36554179017364) bank25744 +25745 POINT(34.03421633266979 -118.70826865687117) bank25745 +25746 POINT(34.83434698175638 -117.59762501862247) bank25746 +25747 POINT(34.86092987201325 -118.92885807876408) bank25747 +25748 POINT(34.20238488337079 -119.0151503879278) bank25748 +25749 POINT(33.58501008876607 -119.03958623709673) bank25749 +25750 POINT(34.98059405357968 -118.48464254169271) bank25750 +25751 POINT(33.462865747390396 -117.94377716200708) bank25751 +25752 POINT(34.04426545190514 -117.70919250145712) bank25752 +25753 POINT(33.82170472614559 -118.89008646937162) bank25753 +25754 POINT(33.89705334618312 -118.21372986037392) bank25754 +25755 POINT(34.501298271490555 -117.69501446308912) bank25755 +25756 POINT(34.013521367666065 -118.89988874777828) bank25756 +25757 POINT(34.63170627461181 -118.54480825649887) bank25757 +25758 POINT(34.64688356980623 -117.36326139223432) bank25758 +25759 POINT(34.92045554985776 -117.59901447840699) bank25759 +25760 POINT(34.79951083130896 -119.22192686996001) bank25760 +25761 POINT(34.07802912822126 -118.89705231486548) bank25761 +25762 POINT(34.72184723164047 -117.85826917990714) bank25762 +25763 POINT(34.86474405380067 -117.74490998207408) bank25763 +25764 POINT(33.76181747616198 -117.76791244710887) bank25764 +25765 POINT(34.551141155642135 -117.3843373804403) bank25765 +25766 POINT(34.57900134682198 -117.34499048176416) bank25766 +25767 POINT(34.75952276501959 -119.0294084376487) bank25767 +25768 POINT(34.194248367095575 -117.8094025810759) bank25768 +25769 POINT(34.31827224240821 -118.08495415178434) bank25769 +25770 POINT(33.1450919713232 -118.71618805877424) bank25770 +25771 POINT(34.54474107838017 -118.34545843535219) bank25771 +25772 POINT(33.82405465977959 -117.66207832526888) bank25772 +25773 POINT(34.06846146366635 -117.46753454170238) bank25773 +25774 POINT(34.846889545424155 -117.8653926844168) bank25774 +25775 POINT(33.778507650507976 -117.3554611719274) bank25775 +25776 POINT(33.44346913279733 -118.84176958091591) bank25776 +25777 POINT(34.943057673590125 -118.05902772422063) bank25777 +25778 POINT(34.65304536472376 -119.17071075632334) bank25778 +25779 POINT(34.92725062331354 -118.89638132996875) bank25779 +25780 POINT(35.03392152922801 -118.32856324878517) bank25780 +25781 POINT(33.704385913788116 -119.08478268416579) bank25781 +25782 POINT(34.92064422410323 -118.43863328348677) bank25782 +25783 POINT(34.53207094759778 -118.22382103532144) bank25783 +25784 POINT(33.674225597261504 -117.57731583117014) bank25784 +25785 POINT(33.25098442360101 -117.84846975150586) bank25785 +25786 POINT(34.23577293788523 -118.21032078963644) bank25786 +25787 POINT(34.58710792694704 -117.47785375193573) bank25787 +25788 POINT(33.63353211040448 -117.39579202108736) bank25788 +25789 POINT(33.114277515284066 -117.98790675077065) bank25789 +25790 POINT(34.74346761361195 -118.52313235166821) bank25790 +25791 POINT(33.49588862044671 -118.54690975341501) bank25791 +25792 POINT(34.27961384746734 -118.56397640415872) bank25792 +25793 POINT(34.338694011365284 -118.41881257342861) bank25793 +25794 POINT(34.19777824090113 -118.27650127631149) bank25794 +25795 POINT(33.16719924408896 -119.15079345675201) bank25795 +25796 POINT(34.30522981397529 -118.91397343016484) bank25796 +25797 POINT(34.50928552138043 -118.23349153598393) bank25797 +25798 POINT(34.635711118666556 -119.13622557315448) bank25798 +25799 POINT(34.65023452813332 -118.77601320951898) bank25799 +25800 POINT(34.17694148823271 -117.45536203082725) bank25800 +25801 POINT(34.07616682535026 -117.62522178105917) bank25801 +25802 POINT(34.12592682739243 -117.43617451126616) bank25802 +25803 POINT(33.83929313407144 -117.93230460768041) bank25803 +25804 POINT(34.363439270976244 -119.06867264713553) bank25804 +25805 POINT(34.4143029413288 -118.86588754333559) bank25805 +25806 POINT(34.27256720426778 -118.24357137792636) bank25806 +25807 POINT(33.157835085066324 -118.78722644572328) bank25807 +25808 POINT(34.464703564679304 -117.92311176581858) bank25808 +25809 POINT(34.20043394504575 -118.86614423333185) bank25809 +25810 POINT(33.29086728218724 -118.24093600524604) bank25810 +25811 POINT(33.28149382411952 -117.4734421101184) bank25811 +25812 POINT(34.083314334121795 -117.96986998281164) bank25812 +25813 POINT(34.270645291108856 -118.60057364167115) bank25813 +25814 POINT(34.93759921898197 -117.79783694144928) bank25814 +25815 POINT(33.71907073419567 -117.82682961683999) bank25815 +25816 POINT(34.022098901389214 -118.73609335778771) bank25816 +25817 POINT(35.040866574403594 -118.31536828695936) bank25817 +25818 POINT(33.54612034738287 -119.20939621788685) bank25818 +25819 POINT(34.95883996591579 -118.92305796226684) bank25819 +25820 POINT(33.60349818424214 -118.16623190175675) bank25820 +25821 POINT(33.28639071811477 -117.64667564029554) bank25821 +25822 POINT(33.63277375149062 -118.38870636475117) bank25822 +25823 POINT(34.84830481027866 -117.46743195502677) bank25823 +25824 POINT(34.172160126287025 -117.35675917744935) bank25824 +25825 POINT(34.831469954860374 -117.82315963727298) bank25825 +25826 POINT(34.76924694329949 -117.8677905178964) bank25826 +25827 POINT(34.060350645142734 -118.47352665543565) bank25827 +25828 POINT(34.75140846160008 -118.60543118787258) bank25828 +25829 POINT(34.5384767512314 -117.88439330199711) bank25829 +25830 POINT(34.28478302048222 -118.46314391866129) bank25830 +25831 POINT(33.113089769807 -119.12520918526658) bank25831 +25832 POINT(34.30158706927982 -118.53630161474354) bank25832 +25833 POINT(34.558056278638574 -117.3491275950351) bank25833 +25834 POINT(34.076417024669624 -119.06208089152113) bank25834 +25835 POINT(33.14720575587113 -119.07199846222987) bank25835 +25836 POINT(34.978414475046435 -117.77345364844645) bank25836 +25837 POINT(34.12815692254783 -118.48269551443217) bank25837 +25838 POINT(35.03754879808471 -117.69593805007526) bank25838 +25839 POINT(33.92061289233158 -117.58523739158208) bank25839 +25840 POINT(33.574769714177265 -118.84874675598805) bank25840 +25841 POINT(34.546707651971175 -119.15189445028761) bank25841 +25842 POINT(34.96787963927656 -118.41115373515734) bank25842 +25843 POINT(33.24969811847854 -119.0953549878522) bank25843 +25844 POINT(33.535859618468145 -118.62406259611394) bank25844 +25845 POINT(34.11859278107471 -119.20332123144813) bank25845 +25846 POINT(33.96178265694739 -117.38659156024475) bank25846 +25847 POINT(33.46355798601209 -118.2912346849919) bank25847 +25848 POINT(33.394964022399435 -119.21576617006535) bank25848 +25849 POINT(34.57439737254557 -119.03194489343073) bank25849 +25850 POINT(34.647668309863256 -118.96905768979448) bank25850 +25851 POINT(33.64346710618459 -118.201923764378) bank25851 +25852 POINT(34.00132611712251 -118.10278453416377) bank25852 +25853 POINT(34.03641173280465 -117.87922218884344) bank25853 +25854 POINT(33.93801565239366 -119.20589180319817) bank25854 +25855 POINT(33.487180305799775 -118.32347942823043) bank25855 +25856 POINT(34.96793774421677 -118.0909972023946) bank25856 +25857 POINT(34.17750881805911 -117.82618759248943) bank25857 +25858 POINT(33.09975410536534 -118.20239976804021) bank25858 +25859 POINT(34.06076806520171 -118.92493668243416) bank25859 +25860 POINT(33.754254633931275 -119.02814553935927) bank25860 +25861 POINT(34.39869588235621 -118.15607767088223) bank25861 +25862 POINT(34.2612233866182 -118.29615369503263) bank25862 +25863 POINT(33.568477938826824 -118.32551732297003) bank25863 +25864 POINT(33.45898138371722 -117.3400487275962) bank25864 +25865 POINT(34.53947205580609 -119.14790413399467) bank25865 +25866 POINT(34.422218729724456 -119.19198324425712) bank25866 +25867 POINT(33.849347321662606 -117.88155952982952) bank25867 +25868 POINT(34.90709966920631 -117.54481548059648) bank25868 +25869 POINT(33.21876284105821 -117.67206080907715) bank25869 +25870 POINT(33.894817223685486 -117.4184265828488) bank25870 +25871 POINT(33.35112988639328 -117.91177101341896) bank25871 +25872 POINT(33.40251032518111 -118.70742367218477) bank25872 +25873 POINT(33.607287506259404 -117.96577703472063) bank25873 +25874 POINT(33.45288202554125 -117.34546630661261) bank25874 +25875 POINT(34.810559851417075 -117.27453232169714) bank25875 +25876 POINT(33.17046420297825 -117.35221592757307) bank25876 +25877 POINT(34.45192117678448 -118.67752065481079) bank25877 +25878 POINT(33.80818083111524 -118.77201523116963) bank25878 +25879 POINT(34.58901592393852 -117.98948425179601) bank25879 +25880 POINT(33.34784757193045 -118.17553869029906) bank25880 +25881 POINT(34.54269910221295 -119.07522283234215) bank25881 +25882 POINT(33.23290346810972 -118.70820058416555) bank25882 +25883 POINT(33.611461743707245 -117.84935607314641) bank25883 +25884 POINT(34.085410652128594 -117.9142384167489) bank25884 +25885 POINT(33.95465303686476 -118.56068489275489) bank25885 +25886 POINT(34.21310645716795 -118.61270628117204) bank25886 +25887 POINT(34.061676961885915 -118.72767478168792) bank25887 +25888 POINT(33.589453447089234 -118.25811817404029) bank25888 +25889 POINT(33.875676466675536 -118.50494941989298) bank25889 +25890 POINT(35.03124426856988 -117.9362741270892) bank25890 +25891 POINT(33.134313016746454 -118.69417193738728) bank25891 +25892 POINT(34.81046696684723 -118.53605159643325) bank25892 +25893 POINT(34.867372063379506 -118.53763783630784) bank25893 +25894 POINT(33.473284544538664 -117.39945561156853) bank25894 +25895 POINT(33.94589930237332 -117.44080892529627) bank25895 +25896 POINT(34.95246937148608 -117.26285136922233) bank25896 +25897 POINT(34.23880273925702 -119.179880574138) bank25897 +25898 POINT(33.09640334332539 -118.35229950483145) bank25898 +25899 POINT(33.893718563848815 -118.26901294311921) bank25899 +25900 POINT(34.46839354893478 -117.37249395650645) bank25900 +25901 POINT(34.74205805273151 -119.18046428476477) bank25901 +25902 POINT(33.45928062515892 -118.31168443331197) bank25902 +25903 POINT(34.04912192600388 -117.78280779907682) bank25903 +25904 POINT(34.20523727844772 -118.86832146883093) bank25904 +25905 POINT(33.584721794679645 -118.92541853827986) bank25905 +25906 POINT(34.43798974238476 -118.83146317901286) bank25906 +25907 POINT(33.92035378701879 -117.34461853374802) bank25907 +25908 POINT(35.045472861984805 -119.23626576171233) bank25908 +25909 POINT(33.31641605947919 -119.05521168542502) bank25909 +25910 POINT(33.38207136520737 -118.37003310012302) bank25910 +25911 POINT(33.28716544818447 -119.06869626751819) bank25911 +25912 POINT(33.58258169100095 -117.61988600578877) bank25912 +25913 POINT(33.42747876209317 -118.80531173379595) bank25913 +25914 POINT(34.834050665412505 -117.90230810051663) bank25914 +25915 POINT(33.399535486698504 -118.43497622319634) bank25915 +25916 POINT(33.62342178339627 -117.86681155766516) bank25916 +25917 POINT(34.53746068517438 -117.6099149739944) bank25917 +25918 POINT(34.09539843416985 -118.27835247217232) bank25918 +25919 POINT(33.69223329078964 -118.77420472182258) bank25919 +25920 POINT(34.69098646886192 -118.44761475617541) bank25920 +25921 POINT(33.65157094069767 -118.59326794346214) bank25921 +25922 POINT(34.61022926077514 -117.55306371479458) bank25922 +25923 POINT(34.03822077507708 -117.81700836020453) bank25923 +25924 POINT(34.22166906450051 -117.9065417911073) bank25924 +25925 POINT(33.21519069924813 -118.12194583135444) bank25925 +25926 POINT(33.436846245529054 -118.81055396983783) bank25926 +25927 POINT(34.12206801926508 -118.38058852485044) bank25927 +25928 POINT(33.13771840817905 -117.64525737893268) bank25928 +25929 POINT(34.56253205290772 -117.4809310672452) bank25929 +25930 POINT(34.932188060187684 -118.80647842599612) bank25930 +25931 POINT(33.54527944683074 -118.41591487648247) bank25931 +25932 POINT(33.737820287848585 -118.40178212138424) bank25932 +25933 POINT(33.91463767585108 -117.72461056095182) bank25933 +25934 POINT(33.44615069184992 -118.65332802177477) bank25934 +25935 POINT(33.138511497035516 -118.15248002677981) bank25935 +25936 POINT(34.105718583112385 -118.74078859385014) bank25936 +25937 POINT(33.25706597260376 -118.86448678184578) bank25937 +25938 POINT(34.00163622272158 -117.47361010100796) bank25938 +25939 POINT(34.84611698392836 -118.31319709005628) bank25939 +25940 POINT(33.64908213283701 -118.29392469810662) bank25940 +25941 POINT(33.96482459725934 -117.79541402323682) bank25941 +25942 POINT(33.40441370835916 -118.76767965997897) bank25942 +25943 POINT(33.23322848985443 -118.42409221686484) bank25943 +25944 POINT(33.17168506350136 -117.2877334960193) bank25944 +25945 POINT(33.49525621684657 -117.29206065109013) bank25945 +25946 POINT(33.506573748539054 -117.92431693168723) bank25946 +25947 POINT(33.36506042362892 -118.79105395770115) bank25947 +25948 POINT(34.9419025843623 -118.89177850802851) bank25948 +25949 POINT(34.412680912983745 -118.43723351683494) bank25949 +25950 POINT(33.88503220946545 -117.73670252786066) bank25950 +25951 POINT(33.663069879120414 -118.68463649329932) bank25951 +25952 POINT(33.79950441950807 -117.31035992412666) bank25952 +25953 POINT(33.87567943184031 -118.20902377138844) bank25953 +25954 POINT(34.280405246734986 -118.75530146755992) bank25954 +25955 POINT(34.8651756683217 -118.93741371563014) bank25955 +25956 POINT(34.939071414092204 -118.1059399605591) bank25956 +25957 POINT(33.086766480782266 -118.28139192516579) bank25957 +25958 POINT(34.802667495999344 -118.50457106364807) bank25958 +25959 POINT(33.08214682139264 -117.57698036935061) bank25959 +25960 POINT(34.50533425400534 -118.16083394619457) bank25960 +25961 POINT(33.53938796453991 -118.38528938497309) bank25961 +25962 POINT(34.419395633084896 -117.65654575419785) bank25962 +25963 POINT(34.29551088249286 -118.24185114215302) bank25963 +25964 POINT(34.581534295984184 -118.83049425571204) bank25964 +25965 POINT(33.561881301915236 -117.62395550010963) bank25965 +25966 POINT(33.47614745757248 -117.47340632992417) bank25966 +25967 POINT(33.42274378132118 -119.18225406572265) bank25967 +25968 POINT(33.693234451465614 -117.91193418019179) bank25968 +25969 POINT(33.598920942762874 -118.48701703145011) bank25969 +25970 POINT(34.87147357002129 -118.66292330014188) bank25970 +25971 POINT(33.05695177539014 -117.37739449309643) bank25971 +25972 POINT(34.901567444956086 -117.43313895099641) bank25972 +25973 POINT(35.013908328918184 -118.06570216782349) bank25973 +25974 POINT(33.9487303746576 -118.17321364331201) bank25974 +25975 POINT(33.984195864545754 -118.85377696148605) bank25975 +25976 POINT(34.243565010795606 -117.89395693224124) bank25976 +25977 POINT(34.62848707599557 -117.43528106435123) bank25977 +25978 POINT(34.70369239399556 -117.53833511784391) bank25978 +25979 POINT(34.737041372833346 -118.83843341368625) bank25979 +25980 POINT(34.865656870526934 -118.87026188460469) bank25980 +25981 POINT(34.791670345482096 -118.02267862336666) bank25981 +25982 POINT(33.56489567164964 -118.71697858054647) bank25982 +25983 POINT(34.5405283094011 -117.5160651202243) bank25983 +25984 POINT(33.17514534251422 -117.59180902002156) bank25984 +25985 POINT(34.702639953697734 -118.53404811764291) bank25985 +25986 POINT(34.32119735498937 -118.31289123026309) bank25986 +25987 POINT(33.284583380004904 -117.73568622275766) bank25987 +25988 POINT(34.105302825897766 -117.55521447588455) bank25988 +25989 POINT(33.3072208862954 -117.78602357253271) bank25989 +25990 POINT(33.846430910930955 -117.74882648203858) bank25990 +25991 POINT(33.58315896853576 -117.40100812427073) bank25991 +25992 POINT(33.13952808471227 -117.36612108185686) bank25992 +25993 POINT(33.53168490246879 -117.30142816613544) bank25993 +25994 POINT(33.59155389825389 -118.85582511609934) bank25994 +25995 POINT(33.16116675442488 -117.40613536721052) bank25995 +25996 POINT(35.03120120068262 -117.63598908587824) bank25996 +25997 POINT(33.28369325343091 -118.11971962321414) bank25997 +25998 POINT(34.71607085688871 -118.50920098084538) bank25998 +25999 POINT(33.34234297102145 -117.87014002709935) bank25999 +26000 POINT(34.98988583752738 -119.16672366400992) bank26000 +26001 POINT(34.49605174924571 -118.4569233819984) bank26001 +26002 POINT(34.63437820976872 -117.80895279199896) bank26002 +26003 POINT(33.99701262455617 -117.51343196272396) bank26003 +26004 POINT(33.88763869932434 -118.62488811798993) bank26004 +26005 POINT(34.54857790802132 -117.39224194148287) bank26005 +26006 POINT(34.942004067087204 -118.28632736971576) bank26006 +26007 POINT(34.60775398930793 -117.32864340378872) bank26007 +26008 POINT(33.38866143968086 -117.65416595032238) bank26008 +26009 POINT(34.61064713771166 -117.47223293496273) bank26009 +26010 POINT(33.380450994001016 -118.67621150914228) bank26010 +26011 POINT(33.6896113611778 -118.54387076381401) bank26011 +26012 POINT(34.23981523929652 -117.46788739501477) bank26012 +26013 POINT(34.522829658193814 -117.77789973406132) bank26013 +26014 POINT(34.47597702838015 -119.06085741830904) bank26014 +26015 POINT(33.61276595205487 -118.99848627449495) bank26015 +26016 POINT(33.280761154178116 -118.51333218391687) bank26016 +26017 POINT(34.93994037348811 -118.28111007306349) bank26017 +26018 POINT(34.70535323951022 -118.24243415548348) bank26018 +26019 POINT(34.90695455171236 -118.33559747570294) bank26019 +26020 POINT(33.08057703682255 -119.13538176278125) bank26020 +26021 POINT(33.619349870251945 -118.96675749548677) bank26021 +26022 POINT(33.060691147423384 -118.34700403808141) bank26022 +26023 POINT(35.0406312962347 -117.25108253779904) bank26023 +26024 POINT(33.256692126528534 -117.50137504880337) bank26024 +26025 POINT(33.91158428039105 -117.65816001356922) bank26025 +26026 POINT(34.94601216406192 -118.34041746911005) bank26026 +26027 POINT(34.31837109996284 -118.36450623612997) bank26027 +26028 POINT(34.28599466887849 -119.13217190349732) bank26028 +26029 POINT(34.83503927814869 -117.82896113481381) bank26029 +26030 POINT(33.2066637046902 -119.01558113588821) bank26030 +26031 POINT(35.01186052107048 -117.85586850808856) bank26031 +26032 POINT(34.91605991555764 -118.98435486117107) bank26032 +26033 POINT(34.98925733773271 -118.46154740921385) bank26033 +26034 POINT(34.35353820195372 -117.39225443635895) bank26034 +26035 POINT(33.214556059769244 -119.06857033485817) bank26035 +26036 POINT(34.06306392720863 -118.6896676931991) bank26036 +26037 POINT(34.97797251261269 -118.5437762364108) bank26037 +26038 POINT(34.3973959711061 -118.45384954613228) bank26038 +26039 POINT(34.17721329437979 -118.78694116652794) bank26039 +26040 POINT(34.69706829158796 -118.05403933843706) bank26040 +26041 POINT(34.392482349427446 -118.31674223125317) bank26041 +26042 POINT(33.136550032519985 -118.10025936897637) bank26042 +26043 POINT(34.95534239744264 -118.84289780986968) bank26043 +26044 POINT(34.141016285319715 -118.49592986869573) bank26044 +26045 POINT(34.03303468153067 -119.23028587893) bank26045 +26046 POINT(34.33966155793082 -118.57677355636058) bank26046 +26047 POINT(34.773957526292676 -118.8043382021899) bank26047 +26048 POINT(33.41266339925908 -119.1782571247194) bank26048 +26049 POINT(34.96859742640443 -117.63305417299236) bank26049 +26050 POINT(33.730545534196004 -117.41493153941762) bank26050 +26051 POINT(33.47235776315571 -117.6977582036897) bank26051 +26052 POINT(34.45014126531884 -118.5872091408484) bank26052 +26053 POINT(33.30255906180096 -117.90300474144644) bank26053 +26054 POINT(34.84405521866876 -117.4382412800755) bank26054 +26055 POINT(33.49117947155262 -117.4313431824) bank26055 +26056 POINT(34.336390511418244 -118.70278812069536) bank26056 +26057 POINT(33.29777928008651 -117.28001718020614) bank26057 +26058 POINT(34.24039207357478 -117.40646613468913) bank26058 +26059 POINT(33.09136349069081 -118.583553106264) bank26059 +26060 POINT(34.86342947027145 -117.79304258337497) bank26060 +26061 POINT(33.656532749448466 -119.01499808194022) bank26061 +26062 POINT(34.16539842456185 -118.17058418265492) bank26062 +26063 POINT(34.81043215498764 -118.67917346974242) bank26063 +26064 POINT(34.540416511969916 -118.31902606764682) bank26064 +26065 POINT(33.26027361667308 -118.0598285552969) bank26065 +26066 POINT(33.577415566823 -117.29259080905372) bank26066 +26067 POINT(34.87358944518582 -118.23700925533046) bank26067 +26068 POINT(34.85635880380551 -118.23911120934909) bank26068 +26069 POINT(34.45745247423234 -118.65824948458982) bank26069 +26070 POINT(34.55392503409744 -118.2252245991338) bank26070 +26071 POINT(34.73933530669152 -118.92077023485766) bank26071 +26072 POINT(34.92204886951237 -119.02361154339889) bank26072 +26073 POINT(34.15649331142706 -117.3190826606297) bank26073 +26074 POINT(33.606666931367904 -118.7164269462276) bank26074 +26075 POINT(33.61218988272738 -118.81330857780992) bank26075 +26076 POINT(33.449714464616825 -118.37582753171618) bank26076 +26077 POINT(34.558331958998274 -118.12888000613204) bank26077 +26078 POINT(34.870625925895276 -117.63116126506954) bank26078 +26079 POINT(33.76310258852417 -117.66467579826568) bank26079 +26080 POINT(33.484116668582104 -118.22296595593858) bank26080 +26081 POINT(34.06068153027783 -117.57264413050176) bank26081 +26082 POINT(33.34324669585562 -117.93568138816765) bank26082 +26083 POINT(34.58672234746782 -118.7872988044894) bank26083 +26084 POINT(33.41344569134653 -118.37421484682339) bank26084 +26085 POINT(34.227249529477064 -118.71834095566707) bank26085 +26086 POINT(34.53788718254391 -118.0588409142784) bank26086 +26087 POINT(34.751871611058995 -118.95014154999399) bank26087 +26088 POINT(33.88127386550893 -119.07812315993965) bank26088 +26089 POINT(34.234835902688985 -118.67081028902668) bank26089 +26090 POINT(34.64046007440751 -119.17752071318746) bank26090 +26091 POINT(34.32578725777282 -118.29419434634602) bank26091 +26092 POINT(33.830734141742724 -118.13942828903205) bank26092 +26093 POINT(33.08217179669519 -119.1168557017204) bank26093 +26094 POINT(34.315029055246086 -118.26947806716846) bank26094 +26095 POINT(33.45244112303927 -118.25876333860435) bank26095 +26096 POINT(33.12902967423577 -118.33371570235767) bank26096 +26097 POINT(35.02746059158263 -119.09115393562966) bank26097 +26098 POINT(34.14040400113585 -117.24508914625567) bank26098 +26099 POINT(33.22683845176186 -118.53868535592406) bank26099 +26100 POINT(34.67375980183529 -117.5223769188407) bank26100 +26101 POINT(34.01128585099113 -117.65722625953705) bank26101 +26102 POINT(33.186508645296406 -117.97945305850065) bank26102 +26103 POINT(34.667934998582815 -119.0425987544301) bank26103 +26104 POINT(33.546110770534725 -117.94243320569711) bank26104 +26105 POINT(34.102850534076026 -117.47828903239211) bank26105 +26106 POINT(34.1465001430602 -118.57054716422826) bank26106 +26107 POINT(33.828755828578885 -117.94614679479712) bank26107 +26108 POINT(34.035878384340926 -117.74011872139677) bank26108 +26109 POINT(34.546056051264586 -117.83685646692054) bank26109 +26110 POINT(34.0424614888103 -118.27852482742819) bank26110 +26111 POINT(34.77392899336209 -117.86619521269151) bank26111 +26112 POINT(33.636125423043985 -117.41012550674067) bank26112 +26113 POINT(33.708623685765644 -117.59640909587964) bank26113 +26114 POINT(34.04049373073537 -119.24280729772507) bank26114 +26115 POINT(34.28623293297383 -117.6296976768667) bank26115 +26116 POINT(34.298020232199775 -117.51547476825364) bank26116 +26117 POINT(34.98476261301728 -118.32676085710125) bank26117 +26118 POINT(33.61095076949879 -117.5908642794011) bank26118 +26119 POINT(33.17329709398235 -117.56895860589997) bank26119 +26120 POINT(33.76387663678931 -118.16067258565556) bank26120 +26121 POINT(33.975688279225324 -118.13166242502575) bank26121 +26122 POINT(33.44211654568143 -117.62445137048618) bank26122 +26123 POINT(35.045021003564 -117.38037404511151) bank26123 +26124 POINT(34.89367512761454 -118.72067625585176) bank26124 +26125 POINT(33.62818601060134 -118.82051137873889) bank26125 +26126 POINT(33.4681282264776 -117.68194619151025) bank26126 +26127 POINT(34.988586174980256 -118.66008799156577) bank26127 +26128 POINT(34.62203517830115 -117.73244693874986) bank26128 +26129 POINT(33.829795930299895 -119.1857130371827) bank26129 +26130 POINT(34.98812175908107 -119.05410502157935) bank26130 +26131 POINT(35.016521176435205 -118.47038850653942) bank26131 +26132 POINT(33.77341702781697 -118.62626520597925) bank26132 +26133 POINT(34.41968085697706 -118.60984881311215) bank26133 +26134 POINT(34.70738883467131 -117.4313033669165) bank26134 +26135 POINT(34.20663487979052 -118.45012849190326) bank26135 +26136 POINT(33.12961940592309 -118.19446258632276) bank26136 +26137 POINT(33.85887076102385 -118.25042868553335) bank26137 +26138 POINT(33.93441077943887 -118.96968930643709) bank26138 +26139 POINT(34.11964914704686 -118.43493939388968) bank26139 +26140 POINT(34.94449625683711 -118.0516487906246) bank26140 +26141 POINT(34.6789894770547 -117.61783834560242) bank26141 +26142 POINT(34.665304249744665 -119.23034138015001) bank26142 +26143 POINT(33.8726800634178 -118.75481556105244) bank26143 +26144 POINT(34.21841654562365 -118.64156571248519) bank26144 +26145 POINT(33.55866403638683 -117.78082389098319) bank26145 +26146 POINT(33.85096352411285 -118.48210125392917) bank26146 +26147 POINT(34.07628598225307 -117.91765631769236) bank26147 +26148 POINT(33.16585768408873 -118.01950990579019) bank26148 +26149 POINT(33.95539127659649 -118.30291271817843) bank26149 +26150 POINT(33.12050204410757 -117.25567281041985) bank26150 +26151 POINT(33.697088905781335 -118.72646718752773) bank26151 +26152 POINT(34.36114030044495 -118.72353423861048) bank26152 +26153 POINT(33.759081146102126 -117.3222394521508) bank26153 +26154 POINT(33.39956994609695 -118.38417159436652) bank26154 +26155 POINT(34.66994569186157 -117.36061607881736) bank26155 +26156 POINT(35.03293121330799 -119.0071286055881) bank26156 +26157 POINT(33.54208223277715 -118.85708063403882) bank26157 +26158 POINT(34.56181016198251 -118.47117930377296) bank26158 +26159 POINT(34.82708550039929 -118.76108452220087) bank26159 +26160 POINT(34.65093174876372 -118.16153880147593) bank26160 +26161 POINT(34.20497381849328 -118.11317388723104) bank26161 +26162 POINT(33.58664599808424 -118.3490527061712) bank26162 +26163 POINT(34.26243708705805 -117.53407860026894) bank26163 +26164 POINT(34.09135130106881 -118.41517562568991) bank26164 +26165 POINT(33.40398681593631 -118.9313517372647) bank26165 +26166 POINT(33.41342426808831 -117.41412153196632) bank26166 +26167 POINT(34.97499883097343 -117.92046807790388) bank26167 +26168 POINT(34.87373825562283 -118.38359495400442) bank26168 +26169 POINT(34.69975338711177 -117.89705466276324) bank26169 +26170 POINT(33.44333003016414 -118.09762273261248) bank26170 +26171 POINT(33.56560659429383 -117.40700411706312) bank26171 +26172 POINT(33.16205792082443 -117.69711280492174) bank26172 +26173 POINT(33.72987724058628 -118.96188001898024) bank26173 +26174 POINT(34.25105013686942 -118.26638950788612) bank26174 +26175 POINT(34.70683643145446 -118.64171396016594) bank26175 +26176 POINT(33.49856725099636 -118.588462434618) bank26176 +26177 POINT(34.1818498698262 -118.59546142437515) bank26177 +26178 POINT(34.38916707385355 -118.60983342910431) bank26178 +26179 POINT(34.28877032651476 -119.03021399515768) bank26179 +26180 POINT(34.80920773415293 -117.4667701148127) bank26180 +26181 POINT(34.32720138663585 -118.79540588340583) bank26181 +26182 POINT(33.222267293504146 -118.63936146208412) bank26182 +26183 POINT(33.95733579690636 -117.39428418987947) bank26183 +26184 POINT(33.761531821234705 -119.12040956669526) bank26184 +26185 POINT(34.75089600921002 -118.89973884955556) bank26185 +26186 POINT(34.70093233892884 -118.20687260042246) bank26186 +26187 POINT(34.971251354260026 -117.96856944028623) bank26187 +26188 POINT(34.141801403044326 -119.0541123129358) bank26188 +26189 POINT(33.73185542577421 -118.08082414748675) bank26189 +26190 POINT(34.971115390039955 -117.33922114473472) bank26190 +26191 POINT(33.657129767467744 -118.26761919123918) bank26191 +26192 POINT(34.07653155262101 -118.43101156010594) bank26192 +26193 POINT(34.2231312556827 -118.05147266538766) bank26193 +26194 POINT(34.5420213043137 -117.41752468478101) bank26194 +26195 POINT(34.00282865102537 -117.42062730333643) bank26195 +26196 POINT(34.686686645121846 -117.76703488681927) bank26196 +26197 POINT(34.353567834095436 -118.42673075202579) bank26197 +26198 POINT(33.88827655218441 -117.91888682185852) bank26198 +26199 POINT(34.63080719783319 -118.4912379427037) bank26199 +26200 POINT(34.53589636973892 -118.86661455012306) bank26200 +26201 POINT(34.71895248372957 -119.05219372439554) bank26201 +26202 POINT(34.967690755113935 -119.02903156351142) bank26202 +26203 POINT(33.0788424302015 -119.04587590296488) bank26203 +26204 POINT(33.932388676758244 -117.37839543094594) bank26204 +26205 POINT(34.826195638049185 -119.17790947557609) bank26205 +26206 POINT(33.19052012731212 -117.4989686704861) bank26206 +26207 POINT(33.14844264051313 -119.24124229367565) bank26207 +26208 POINT(33.88233938515935 -117.39690424177405) bank26208 +26209 POINT(34.818054198878066 -117.96101938388274) bank26209 +26210 POINT(34.404356650933686 -117.79909299513537) bank26210 +26211 POINT(33.94838016315055 -118.27316343152033) bank26211 +26212 POINT(34.34488098535324 -117.73338515700728) bank26212 +26213 POINT(34.24325721379707 -118.6747699516403) bank26213 +26214 POINT(33.54834442366836 -118.21126218141157) bank26214 +26215 POINT(33.283126625127935 -118.29912971782056) bank26215 +26216 POINT(33.81921799608484 -117.64023724169832) bank26216 +26217 POINT(33.545674637392594 -118.47144373815529) bank26217 +26218 POINT(34.873951544864056 -118.63280828408223) bank26218 +26219 POINT(34.39104414100669 -118.39897014711204) bank26219 +26220 POINT(34.96640874492742 -117.84924558507961) bank26220 +26221 POINT(33.94751710752482 -117.4504682415786) bank26221 +26222 POINT(33.474101908123785 -119.09420837027784) bank26222 +26223 POINT(34.37278960943073 -117.85513574461619) bank26223 +26224 POINT(33.55580434529988 -118.40319480835333) bank26224 +26225 POINT(34.95768044722029 -117.94210238853213) bank26225 +26226 POINT(34.532132767861704 -117.36285499834099) bank26226 +26227 POINT(34.46266017648005 -119.01575555103292) bank26227 +26228 POINT(34.968615327289974 -117.62612307185525) bank26228 +26229 POINT(33.15929103489943 -117.2744411210741) bank26229 +26230 POINT(34.38154778009362 -118.65272017821776) bank26230 +26231 POINT(34.636038634443366 -118.3767852437603) bank26231 +26232 POINT(34.89669340625926 -117.71645125530316) bank26232 +26233 POINT(33.58625178419802 -117.46593758328522) bank26233 +26234 POINT(34.392143714209915 -117.45057662492637) bank26234 +26235 POINT(34.923692953336314 -119.07428673303511) bank26235 +26236 POINT(34.125352566418556 -117.6163376292462) bank26236 +26237 POINT(33.64234883852401 -118.3936242468431) bank26237 +26238 POINT(33.06910304747622 -119.00394987263424) bank26238 +26239 POINT(33.110629130814885 -119.19922366317147) bank26239 +26240 POINT(33.67958812929829 -118.4559172902815) bank26240 +26241 POINT(35.01645526285334 -118.14575377278139) bank26241 +26242 POINT(34.270769099575865 -118.81541316370294) bank26242 +26243 POINT(34.34210371063084 -118.03516406604753) bank26243 +26244 POINT(33.98384440085019 -118.09887411305728) bank26244 +26245 POINT(33.11252532633574 -117.26479991491388) bank26245 +26246 POINT(33.378733463074475 -118.38310678704231) bank26246 +26247 POINT(35.03143661306723 -117.73697818789579) bank26247 +26248 POINT(33.88345075040699 -119.20166311640882) bank26248 +26249 POINT(33.62714492286644 -118.00948904166978) bank26249 +26250 POINT(33.062760962267724 -119.05313561690775) bank26250 +26251 POINT(34.90350593842788 -118.8322779239361) bank26251 +26252 POINT(33.77358411889513 -118.57151507977645) bank26252 +26253 POINT(34.42814046797396 -118.47118711587407) bank26253 +26254 POINT(34.83442355668477 -119.08949810412626) bank26254 +26255 POINT(34.57004872054127 -118.59202639592998) bank26255 +26256 POINT(34.754931415124354 -119.01336390838833) bank26256 +26257 POINT(34.22695110614292 -119.0651465268433) bank26257 +26258 POINT(34.507597009093935 -118.60468634335149) bank26258 +26259 POINT(33.21511555187336 -118.07066144660037) bank26259 +26260 POINT(34.04295001759123 -119.23851159826889) bank26260 +26261 POINT(33.26132697186556 -117.27308983100761) bank26261 +26262 POINT(35.01664838993187 -118.81827077231215) bank26262 +26263 POINT(34.23056082554811 -117.95747414898814) bank26263 +26264 POINT(34.54869268115839 -119.10928952665931) bank26264 +26265 POINT(33.72948854526651 -117.47583295337378) bank26265 +26266 POINT(34.47729628939905 -117.8650573634226) bank26266 +26267 POINT(33.512148896600806 -118.56894080324099) bank26267 +26268 POINT(33.12085278473236 -118.8709780856264) bank26268 +26269 POINT(34.25268801664499 -117.77435291212034) bank26269 +26270 POINT(33.56088507158341 -117.64944640804644) bank26270 +26271 POINT(34.49773412571448 -118.77616289152833) bank26271 +26272 POINT(33.45014903486899 -118.14183344981554) bank26272 +26273 POINT(34.46181189913157 -117.95248122288542) bank26273 +26274 POINT(33.157130262812046 -117.53196404602558) bank26274 +26275 POINT(33.61893758413224 -117.96989376505127) bank26275 +26276 POINT(33.20825029520398 -117.53176003128927) bank26276 +26277 POINT(34.153659931276145 -117.55584543469601) bank26277 +26278 POINT(33.29618674640592 -117.87487604946864) bank26278 +26279 POINT(34.31937176482025 -118.49491009051003) bank26279 +26280 POINT(33.897752162539874 -118.25428096693891) bank26280 +26281 POINT(33.37680743084904 -117.86236505680992) bank26281 +26282 POINT(34.1339089575652 -118.5515134215682) bank26282 +26283 POINT(34.030943368375326 -117.47697908446582) bank26283 +26284 POINT(33.07868536598176 -117.57839611991751) bank26284 +26285 POINT(34.09437304824712 -118.60493894001515) bank26285 +26286 POINT(33.41101068680796 -119.06261554557358) bank26286 +26287 POINT(33.25200213476517 -118.2347795710268) bank26287 +26288 POINT(33.9208168822479 -117.57180074943994) bank26288 +26289 POINT(33.79522583181731 -118.3446032155224) bank26289 +26290 POINT(33.91844982110576 -117.25126089502136) bank26290 +26291 POINT(33.189959460420866 -118.00082088944718) bank26291 +26292 POINT(33.95798267775431 -118.5190089230928) bank26292 +26293 POINT(33.854255722112235 -118.60958160722754) bank26293 +26294 POINT(34.76351185428162 -117.51239843831019) bank26294 +26295 POINT(33.852957475543704 -118.64833548157904) bank26295 +26296 POINT(33.99456510658716 -118.58470239872493) bank26296 +26297 POINT(34.237023307269844 -117.4257620243869) bank26297 +26298 POINT(33.3534037621323 -117.71460239533141) bank26298 +26299 POINT(34.19964514507602 -117.78124741011428) bank26299 +26300 POINT(33.57712413017337 -118.46265000523294) bank26300 +26301 POINT(34.64870201529536 -118.16162974764308) bank26301 +26302 POINT(34.3560639875738 -117.71931353517246) bank26302 +26303 POINT(34.25992659415748 -119.02939765561825) bank26303 +26304 POINT(34.36864687194669 -118.28121395172481) bank26304 +26305 POINT(34.03983208579202 -118.39282083274513) bank26305 +26306 POINT(33.5545227944148 -117.54288711184218) bank26306 +26307 POINT(34.57143824775066 -118.72629128251768) bank26307 +26308 POINT(34.839835899980805 -118.40255054574823) bank26308 +26309 POINT(33.895749781824755 -117.99811923936389) bank26309 +26310 POINT(33.64612331505579 -117.55167729738753) bank26310 +26311 POINT(34.42360375502266 -117.50946409969116) bank26311 +26312 POINT(34.92291716856379 -117.95537741810263) bank26312 +26313 POINT(34.126986605443285 -118.61321274933546) bank26313 +26314 POINT(33.75379029846848 -118.82278038174442) bank26314 +26315 POINT(33.92968061725152 -117.84567728968686) bank26315 +26316 POINT(34.405591872257865 -119.05084869235424) bank26316 +26317 POINT(33.064650485676545 -118.06044244224606) bank26317 +26318 POINT(34.16243202060673 -117.49027075370341) bank26318 +26319 POINT(33.81597056951871 -118.2384559430824) bank26319 +26320 POINT(34.546042143397266 -118.07453262445904) bank26320 +26321 POINT(33.71556888393112 -118.70548898327178) bank26321 +26322 POINT(34.22014164588774 -119.07232593365224) bank26322 +26323 POINT(34.88977408321484 -117.56478712651818) bank26323 +26324 POINT(33.15845227964399 -117.96401945532043) bank26324 +26325 POINT(33.96604307806353 -119.22699478512318) bank26325 +26326 POINT(34.781729742513065 -117.79455882791108) bank26326 +26327 POINT(34.35010509459745 -117.27091804014765) bank26327 +26328 POINT(34.698932167863354 -119.06770167285359) bank26328 +26329 POINT(33.71044605346605 -118.44256438088527) bank26329 +26330 POINT(34.09824635558631 -118.96108116230813) bank26330 +26331 POINT(34.07994294230351 -118.4373799944185) bank26331 +26332 POINT(33.12505981178101 -118.31108238056596) bank26332 +26333 POINT(34.379304784906196 -119.24331293596249) bank26333 +26334 POINT(34.99519614040413 -118.01340395684971) bank26334 +26335 POINT(34.2140000511238 -119.16762645175663) bank26335 +26336 POINT(33.10801539466818 -118.88937669136644) bank26336 +26337 POINT(34.18565646882018 -117.49797546409359) bank26337 +26338 POINT(34.69828712603694 -117.62548628257048) bank26338 +26339 POINT(34.94385613448881 -118.50010158921786) bank26339 +26340 POINT(34.77708039639439 -118.68826448462558) bank26340 +26341 POINT(34.04689887596817 -118.07247013308569) bank26341 +26342 POINT(34.909999894889665 -118.94317977897757) bank26342 +26343 POINT(33.76606789488509 -118.40099384121285) bank26343 +26344 POINT(33.37901934376694 -117.54402583502434) bank26344 +26345 POINT(35.01424702052744 -118.87533968372567) bank26345 +26346 POINT(33.318042363284825 -117.2709674738531) bank26346 +26347 POINT(34.2714690992956 -119.21735249063755) bank26347 +26348 POINT(33.28635490546761 -118.54295522036048) bank26348 +26349 POINT(33.72325940696333 -119.10426779282933) bank26349 +26350 POINT(33.23747743729305 -117.59943473201592) bank26350 +26351 POINT(33.12311688167807 -118.02712208156372) bank26351 +26352 POINT(33.739032353068254 -119.20951965371472) bank26352 +26353 POINT(33.92954603701146 -117.7984007128685) bank26353 +26354 POINT(33.66092133343575 -118.27614411691232) bank26354 +26355 POINT(35.00801968078539 -119.11628205244999) bank26355 +26356 POINT(34.48674806320406 -117.93022354992615) bank26356 +26357 POINT(33.798135754960434 -117.52476498152342) bank26357 +26358 POINT(33.83874319859105 -118.3348771437221) bank26358 +26359 POINT(34.659409933879225 -118.40483617856226) bank26359 +26360 POINT(33.40101230020278 -117.25904241259076) bank26360 +26361 POINT(34.30300598659718 -117.36782785072587) bank26361 +26362 POINT(33.86878907841342 -117.41334697422012) bank26362 +26363 POINT(33.32871041918472 -118.71318441801488) bank26363 +26364 POINT(34.45840923882134 -118.32222632392754) bank26364 +26365 POINT(34.28207006476547 -117.73510682432756) bank26365 +26366 POINT(33.559997155183005 -118.40795442554708) bank26366 +26367 POINT(33.47298339498033 -117.30547057044308) bank26367 +26368 POINT(34.99773768740343 -118.40211067541846) bank26368 +26369 POINT(33.72169239377485 -118.58045381878351) bank26369 +26370 POINT(34.34970098465869 -118.67702433138075) bank26370 +26371 POINT(33.91971328286644 -119.19935448380832) bank26371 +26372 POINT(33.72113445027355 -118.29022274670362) bank26372 +26373 POINT(33.59843555944355 -118.28780864095621) bank26373 +26374 POINT(34.76301160118932 -119.14911515530517) bank26374 +26375 POINT(34.05810934233638 -118.70998999772267) bank26375 +26376 POINT(34.11408160000445 -118.81922611129377) bank26376 +26377 POINT(34.36316828053876 -118.44120503182768) bank26377 +26378 POINT(33.69608769520411 -119.13871330726346) bank26378 +26379 POINT(34.11323492596031 -118.88984093233691) bank26379 +26380 POINT(34.5105359234542 -118.03944200435284) bank26380 +26381 POINT(33.81064659075817 -117.71264529101545) bank26381 +26382 POINT(33.3461779665924 -117.76992924006571) bank26382 +26383 POINT(33.77485955948695 -117.71432742272421) bank26383 +26384 POINT(33.9838711817226 -118.79133148881263) bank26384 +26385 POINT(34.75649373298409 -117.56091499865356) bank26385 +26386 POINT(33.49984620327096 -117.73671561861427) bank26386 +26387 POINT(34.76901057379373 -117.82531071246366) bank26387 +26388 POINT(33.1117329259087 -118.21289150165777) bank26388 +26389 POINT(33.65930656104041 -119.15904782072475) bank26389 +26390 POINT(34.24974561836247 -119.0879649415442) bank26390 +26391 POINT(33.16966758965983 -117.45093669257096) bank26391 +26392 POINT(34.93920271742867 -117.90457892330676) bank26392 +26393 POINT(33.889756890676004 -117.3220260425334) bank26393 +26394 POINT(34.98122221057196 -117.7446283145906) bank26394 +26395 POINT(34.19844904206984 -118.70171175776895) bank26395 +26396 POINT(34.114750521354885 -118.10340704238915) bank26396 +26397 POINT(33.85887578439712 -117.40458031937631) bank26397 +26398 POINT(33.27854340250632 -119.24074027988875) bank26398 +26399 POINT(34.87786101664311 -117.52887428809728) bank26399 +26400 POINT(34.802325639311924 -118.87952005609584) bank26400 +26401 POINT(33.16030753257046 -117.81406147268558) bank26401 +26402 POINT(34.43526812307126 -118.61786495159812) bank26402 +26403 POINT(33.88411842185707 -117.58827518728462) bank26403 +26404 POINT(33.770375620149046 -117.24488278952407) bank26404 +26405 POINT(34.20304050022447 -117.6353359291381) bank26405 +26406 POINT(34.59937352335099 -117.97630841691468) bank26406 +26407 POINT(34.86497110625948 -117.92325975875755) bank26407 +26408 POINT(34.58110852914224 -117.5312418221224) bank26408 +26409 POINT(33.13282979977504 -118.42539470256536) bank26409 +26410 POINT(33.756906774628106 -117.6010774428256) bank26410 +26411 POINT(33.44135669820586 -117.75752679952802) bank26411 +26412 POINT(33.739938682596325 -118.38635203048236) bank26412 +26413 POINT(34.9016137632494 -117.64275364015299) bank26413 +26414 POINT(34.540007262453365 -118.45536794611799) bank26414 +26415 POINT(34.95157329091552 -118.6178245341142) bank26415 +26416 POINT(34.613744723343395 -117.67734001676439) bank26416 +26417 POINT(33.546842038755635 -119.15482897831106) bank26417 +26418 POINT(33.447762068477914 -118.93777241152848) bank26418 +26419 POINT(34.433346224628316 -119.09274134606099) bank26419 +26420 POINT(34.34036030853255 -119.1226390020919) bank26420 +26421 POINT(34.9381798633221 -117.28182003605485) bank26421 +26422 POINT(34.60902456123828 -117.39430771631046) bank26422 +26423 POINT(34.58469197692155 -118.40157665465186) bank26423 +26424 POINT(34.929697195978065 -118.88719569565251) bank26424 +26425 POINT(33.57505810643837 -118.85800054081035) bank26425 +26426 POINT(33.218625016812446 -118.0233252490592) bank26426 +26427 POINT(33.67770837929245 -118.41510208787471) bank26427 +26428 POINT(34.64074814149906 -118.4867009536864) bank26428 +26429 POINT(34.56521230985649 -118.24035180144605) bank26429 +26430 POINT(34.80139077324966 -118.66144199934408) bank26430 +26431 POINT(33.63080371950252 -118.3087030116688) bank26431 +26432 POINT(34.21610033434718 -117.94606277185703) bank26432 +26433 POINT(34.31418865306743 -118.36407799987285) bank26433 +26434 POINT(33.77850494901387 -118.46064408793977) bank26434 +26435 POINT(33.81957996929642 -119.04955579384949) bank26435 +26436 POINT(34.86319437888353 -118.56959977433729) bank26436 +26437 POINT(34.118846174203576 -118.46169749164838) bank26437 +26438 POINT(35.02772184863552 -118.64148970110205) bank26438 +26439 POINT(33.58872467710432 -118.80632794390628) bank26439 +26440 POINT(34.13100696150278 -117.38784088470088) bank26440 +26441 POINT(33.888508506663136 -118.19242786801983) bank26441 +26442 POINT(33.22134258366315 -118.25735422415354) bank26442 +26443 POINT(34.42895492946399 -118.38185223297924) bank26443 +26444 POINT(33.18534066238116 -118.16842444145941) bank26444 +26445 POINT(34.583367140956874 -117.3224037284177) bank26445 +26446 POINT(34.71601098189492 -118.83448918930007) bank26446 +26447 POINT(34.38361136798479 -118.19252507715197) bank26447 +26448 POINT(34.82013857870186 -119.03675166740447) bank26448 +26449 POINT(34.03306289750042 -117.68834680502866) bank26449 +26450 POINT(34.062735371676354 -119.0098380950878) bank26450 +26451 POINT(33.72683015892619 -118.59033646125872) bank26451 +26452 POINT(33.56578311932548 -118.20517424850125) bank26452 +26453 POINT(33.39602614468395 -117.38610502384935) bank26453 +26454 POINT(33.29646837701024 -118.02357096906776) bank26454 +26455 POINT(34.60664891118543 -117.86427203768403) bank26455 +26456 POINT(34.50546193049321 -117.70370533158005) bank26456 +26457 POINT(34.05159412646305 -119.03167544590538) bank26457 +26458 POINT(33.30249360411375 -118.1692721820273) bank26458 +26459 POINT(33.19110325589513 -119.11750152967878) bank26459 +26460 POINT(34.43276734541636 -117.60247231501214) bank26460 +26461 POINT(34.77758046151548 -117.97795487097825) bank26461 +26462 POINT(34.97564975806096 -118.42650113789114) bank26462 +26463 POINT(34.392507662800625 -117.69071817161016) bank26463 +26464 POINT(33.7876223698221 -117.33112786702401) bank26464 +26465 POINT(33.52780774053866 -119.14963033618618) bank26465 +26466 POINT(34.91660209075034 -117.96491034345797) bank26466 +26467 POINT(34.53306519074808 -118.45072726697119) bank26467 +26468 POINT(33.415426907383974 -117.45005208389307) bank26468 +26469 POINT(34.248824080041274 -119.05718168729143) bank26469 +26470 POINT(33.79276869616162 -119.04737695709308) bank26470 +26471 POINT(34.060287987007406 -117.62113806982862) bank26471 +26472 POINT(34.21201244375964 -117.81775632601756) bank26472 +26473 POINT(35.034410472560644 -117.33871052125741) bank26473 +26474 POINT(34.31313940225723 -117.28349719330006) bank26474 +26475 POINT(33.079173259361184 -118.7284863178898) bank26475 +26476 POINT(34.7714636063982 -119.00712796757131) bank26476 +26477 POINT(34.83018584502398 -118.9875889355119) bank26477 +26478 POINT(34.843225131195055 -118.51267426718486) bank26478 +26479 POINT(34.90917648768422 -119.24143693131576) bank26479 +26480 POINT(34.756306091837004 -118.89780855809492) bank26480 +26481 POINT(33.260304834487364 -118.47487100374173) bank26481 +26482 POINT(33.94587964480738 -118.03847455514364) bank26482 +26483 POINT(33.22958782767705 -118.54702827735434) bank26483 +26484 POINT(33.52026728003513 -118.47057728613363) bank26484 +26485 POINT(34.01554663083216 -119.08072017345023) bank26485 +26486 POINT(33.93146884045925 -117.99574694075086) bank26486 +26487 POINT(34.76593429019227 -117.29514721233477) bank26487 +26488 POINT(34.41616818490836 -118.36059490352902) bank26488 +26489 POINT(34.72787893416107 -118.79124081792153) bank26489 +26490 POINT(33.47654105259107 -118.27141617386242) bank26490 +26491 POINT(34.22670651010401 -117.55914717158369) bank26491 +26492 POINT(34.70211195845788 -117.83785771567295) bank26492 +26493 POINT(33.33179977247916 -117.99294847960958) bank26493 +26494 POINT(34.946525430194846 -117.74373326731066) bank26494 +26495 POINT(34.691342973426764 -118.85392608171826) bank26495 +26496 POINT(33.549677878399 -117.98703065487787) bank26496 +26497 POINT(34.77690150554794 -119.12173733210967) bank26497 +26498 POINT(34.825155900368465 -117.78031768187684) bank26498 +26499 POINT(33.6846025485257 -117.89588955452504) bank26499 +26500 POINT(34.24055021089638 -117.87033252978257) bank26500 +26501 POINT(34.56897852928947 -117.35777581212793) bank26501 +26502 POINT(34.45391704098523 -118.90023301952577) bank26502 +26503 POINT(34.92792047938635 -118.58494538667625) bank26503 +26504 POINT(34.839212346647685 -118.84731580811152) bank26504 +26505 POINT(34.743094312074476 -119.01897210903365) bank26505 +26506 POINT(33.26693621080628 -118.35417298232126) bank26506 +26507 POINT(33.33587867698691 -117.58829519346736) bank26507 +26508 POINT(33.64955746571905 -118.72513886764177) bank26508 +26509 POINT(35.03830643854705 -117.82848509032773) bank26509 +26510 POINT(34.868143700432014 -118.53374363209883) bank26510 +26511 POINT(34.358053604312 -118.24555679540354) bank26511 +26512 POINT(34.00838911194552 -118.84164770559066) bank26512 +26513 POINT(33.85993478368592 -117.93072101872853) bank26513 +26514 POINT(34.593454797451294 -118.24808708989204) bank26514 +26515 POINT(34.119755951731435 -118.99096099957076) bank26515 +26516 POINT(34.07274386830019 -117.69131104608691) bank26516 +26517 POINT(33.06533551689353 -119.01675652906606) bank26517 +26518 POINT(34.42746648662071 -118.69537886958068) bank26518 +26519 POINT(33.33741175967611 -117.45668905132123) bank26519 +26520 POINT(34.15928018356501 -118.47897753680618) bank26520 +26521 POINT(33.40235539510709 -117.52534403479456) bank26521 +26522 POINT(33.30368335547198 -117.81541768511947) bank26522 +26523 POINT(34.825338294676136 -117.77362719822007) bank26523 +26524 POINT(34.94714634194207 -118.14802036122234) bank26524 +26525 POINT(34.14865904134268 -117.96037243073474) bank26525 +26526 POINT(34.48907801215317 -119.21592363186942) bank26526 +26527 POINT(34.10705372869187 -117.85729261485601) bank26527 +26528 POINT(34.701939760735875 -118.41181994009622) bank26528 +26529 POINT(33.63227008566424 -118.73716734497579) bank26529 +26530 POINT(33.59423515650234 -118.98713449516463) bank26530 +26531 POINT(34.5304352010226 -119.21770766424713) bank26531 +26532 POINT(33.49475669276388 -117.96484379516602) bank26532 +26533 POINT(33.548964781028516 -117.69593953902566) bank26533 +26534 POINT(33.08640584157804 -117.81839792076354) bank26534 +26535 POINT(33.430911428283906 -118.34088637598254) bank26535 +26536 POINT(34.713943213565535 -119.1585119153376) bank26536 +26537 POINT(33.858874678581515 -118.78207000354489) bank26537 +26538 POINT(33.522416139236455 -118.55185927704295) bank26538 +26539 POINT(33.46718914347114 -118.71840935344079) bank26539 +26540 POINT(33.92760214253296 -117.2600106903727) bank26540 +26541 POINT(33.592354013657896 -118.26082975579399) bank26541 +26542 POINT(34.68343928983132 -117.43210351547079) bank26542 +26543 POINT(34.272183305592634 -117.64152123663244) bank26543 +26544 POINT(33.4940829238471 -118.23300510803942) bank26544 +26545 POINT(33.082164326809426 -118.11460383663824) bank26545 +26546 POINT(34.72349902168419 -117.49094411662864) bank26546 +26547 POINT(34.5455203866233 -118.40977627604106) bank26547 +26548 POINT(33.40779688990082 -118.5644251496435) bank26548 +26549 POINT(34.716080116182766 -118.6542029865395) bank26549 +26550 POINT(33.75205741308705 -118.75132656253068) bank26550 +26551 POINT(33.21882984079733 -118.65030756721417) bank26551 +26552 POINT(34.50676271279237 -119.19825331803452) bank26552 +26553 POINT(33.14700098242771 -118.75860398441891) bank26553 +26554 POINT(34.819246600768636 -118.06286918119335) bank26554 +26555 POINT(35.01323888976983 -118.75828254341155) bank26555 +26556 POINT(33.41867425676481 -119.0682280876949) bank26556 +26557 POINT(34.476953068193176 -118.54835939933974) bank26557 +26558 POINT(34.41635361101005 -118.14640500040397) bank26558 +26559 POINT(33.37007648617603 -118.18310523492968) bank26559 +26560 POINT(33.347259714671864 -117.96560773263217) bank26560 +26561 POINT(33.776615850098224 -117.60564568033335) bank26561 +26562 POINT(33.90746534647662 -118.71003741964662) bank26562 +26563 POINT(34.82084680818092 -119.19697630354455) bank26563 +26564 POINT(34.7247268519309 -118.79490453162597) bank26564 +26565 POINT(34.94540876732814 -117.89804089378148) bank26565 +26566 POINT(33.836661593576345 -117.60744739088845) bank26566 +26567 POINT(33.85282186597747 -118.06207851714997) bank26567 +26568 POINT(33.59277974181351 -117.71760777632213) bank26568 +26569 POINT(34.19654509220778 -117.52229649342846) bank26569 +26570 POINT(35.0231653778902 -117.96651286266408) bank26570 +26571 POINT(34.14012788686328 -119.08490277782965) bank26571 +26572 POINT(34.98381894796215 -117.89922919597312) bank26572 +26573 POINT(33.446714158249755 -118.46613959464278) bank26573 +26574 POINT(33.27642901435567 -118.70168124888234) bank26574 +26575 POINT(34.68191363443101 -117.83339023565605) bank26575 +26576 POINT(34.52267702999304 -118.86036647024557) bank26576 +26577 POINT(34.034772739188206 -119.06060474509917) bank26577 +26578 POINT(33.79359777189038 -118.68303062761571) bank26578 +26579 POINT(33.758698490040665 -118.6516701239387) bank26579 +26580 POINT(33.112671374318055 -118.6082278908003) bank26580 +26581 POINT(34.595760571998326 -118.36472326218544) bank26581 +26582 POINT(33.67628654218705 -117.51425386103483) bank26582 +26583 POINT(34.88001860428489 -118.63450780504374) bank26583 +26584 POINT(33.61990055303252 -118.55725605225446) bank26584 +26585 POINT(33.39440129521259 -119.22052728718178) bank26585 +26586 POINT(34.159356394818886 -117.46220869088255) bank26586 +26587 POINT(33.50212071964024 -117.4837065703822) bank26587 +26588 POINT(34.342753473782274 -118.76680491399345) bank26588 +26589 POINT(34.776009179223514 -117.78391750862673) bank26589 +26590 POINT(33.24821801412029 -117.44759044035858) bank26590 +26591 POINT(34.51702448523128 -118.8534035344046) bank26591 +26592 POINT(34.93665007369419 -118.54897775515039) bank26592 +26593 POINT(33.88827065082795 -118.27851731508) bank26593 +26594 POINT(34.270337834571436 -117.26038265889186) bank26594 +26595 POINT(33.455223590515054 -119.0930369269494) bank26595 +26596 POINT(33.540874140499 -117.36191839830758) bank26596 +26597 POINT(34.25544151931906 -119.17190492231038) bank26597 +26598 POINT(34.99295575611426 -117.59573948007917) bank26598 +26599 POINT(34.64643643093962 -118.08386446140624) bank26599 +26600 POINT(33.61651425175923 -118.59774904050525) bank26600 +26601 POINT(34.34733734425969 -119.07497937508002) bank26601 +26602 POINT(34.25749033761375 -118.43913183192879) bank26602 +26603 POINT(34.48251706210233 -117.35979538473308) bank26603 +26604 POINT(33.50500368236116 -119.04758705546053) bank26604 +26605 POINT(33.84179310237539 -119.09730216475376) bank26605 +26606 POINT(33.35685700779495 -119.0277685810393) bank26606 +26607 POINT(33.97687486495153 -118.62594203622551) bank26607 +26608 POINT(33.50525895785524 -117.69080866017872) bank26608 +26609 POINT(34.65972781156375 -118.13340913244468) bank26609 +26610 POINT(34.39225277119617 -118.06812454333843) bank26610 +26611 POINT(33.72373191512126 -117.50827084393195) bank26611 +26612 POINT(33.46324602749636 -118.50039859331591) bank26612 +26613 POINT(34.571290601963184 -117.66815049479405) bank26613 +26614 POINT(33.95896574116768 -117.6935158826369) bank26614 +26615 POINT(33.39863153846237 -118.20090232146009) bank26615 +26616 POINT(34.18666560057964 -117.62290422350392) bank26616 +26617 POINT(33.39555961217042 -117.6742949750306) bank26617 +26618 POINT(34.84223424430249 -117.58298269017344) bank26618 +26619 POINT(34.42440152931054 -119.02750094945127) bank26619 +26620 POINT(33.98120593648261 -118.57111470164172) bank26620 +26621 POINT(33.62611158556126 -118.04660568221215) bank26621 +26622 POINT(34.615290358992304 -118.35550502837295) bank26622 +26623 POINT(33.11209567428573 -118.03451040465333) bank26623 +26624 POINT(34.2906968661767 -118.69874955373872) bank26624 +26625 POINT(34.36991667941733 -117.9625043938763) bank26625 +26626 POINT(34.29392779179092 -118.4665694241997) bank26626 +26627 POINT(34.7379114961426 -117.9141579024671) bank26627 +26628 POINT(33.695966912638895 -117.85540067259302) bank26628 +26629 POINT(34.26934318772766 -117.37779779799335) bank26629 +26630 POINT(33.74047923889737 -118.801173916353) bank26630 +26631 POINT(33.49462357924049 -119.20992354255654) bank26631 +26632 POINT(33.89392540889957 -118.52636738494452) bank26632 +26633 POINT(34.6586634073492 -118.48661455921727) bank26633 +26634 POINT(33.05318630117826 -118.73818097634978) bank26634 +26635 POINT(33.329803330342905 -118.27478528830059) bank26635 +26636 POINT(33.84240599375341 -119.16185352319451) bank26636 +26637 POINT(34.061547608715365 -119.16677305312203) bank26637 +26638 POINT(33.8225355602435 -118.93437855404439) bank26638 +26639 POINT(33.14838854505735 -117.5262375845343) bank26639 +26640 POINT(34.07766889111287 -118.87971609599792) bank26640 +26641 POINT(33.602137727653286 -117.30556424179575) bank26641 +26642 POINT(33.19072466272766 -117.48153625199308) bank26642 +26643 POINT(33.776462203222565 -117.88746666628) bank26643 +26644 POINT(33.420941338663475 -118.25330169933578) bank26644 +26645 POINT(33.267352153731906 -119.18566386426258) bank26645 +26646 POINT(33.67881682984162 -118.46994929394843) bank26646 +26647 POINT(34.64926849152387 -117.54460139448553) bank26647 +26648 POINT(34.95651533604828 -118.05752835188026) bank26648 +26649 POINT(34.93130133170158 -118.89560112034629) bank26649 +26650 POINT(34.77602774910211 -117.99712955839169) bank26650 +26651 POINT(34.69693963781244 -118.261525657454) bank26651 +26652 POINT(34.18562758922848 -117.36434731424745) bank26652 +26653 POINT(33.090957760950126 -117.59138442323247) bank26653 +26654 POINT(34.536271917718054 -119.1269462340712) bank26654 +26655 POINT(33.32992119664381 -118.07671663180945) bank26655 +26656 POINT(33.66515205413331 -119.02789267707492) bank26656 +26657 POINT(33.83436720802031 -119.09815052727794) bank26657 +26658 POINT(34.600324622473266 -118.75107433761028) bank26658 +26659 POINT(33.26970159142546 -117.61900197921246) bank26659 +26660 POINT(34.42165513276801 -117.79531980848736) bank26660 +26661 POINT(34.510787795066385 -117.75705553046734) bank26661 +26662 POINT(34.23632047443387 -117.77163326407873) bank26662 +26663 POINT(33.72605906126139 -119.01242584403751) bank26663 +26664 POINT(34.553928621885674 -117.59393625946524) bank26664 +26665 POINT(34.2606513612494 -118.51780802698099) bank26665 +26666 POINT(33.69616195481947 -117.80340464304577) bank26666 +26667 POINT(33.7738762607078 -118.37511709611991) bank26667 +26668 POINT(33.55368392561369 -118.68128978009882) bank26668 +26669 POINT(33.50092200915114 -118.21228120420417) bank26669 +26670 POINT(34.27581472481434 -119.01011104019834) bank26670 +26671 POINT(34.91122798751915 -118.64849120916584) bank26671 +26672 POINT(33.32257889347175 -117.67411885155089) bank26672 +26673 POINT(34.410717740653496 -117.77001978845651) bank26673 +26674 POINT(33.778707874179844 -117.75990814135213) bank26674 +26675 POINT(34.44131883298246 -117.55203254249894) bank26675 +26676 POINT(33.487697640687635 -119.23477579875497) bank26676 +26677 POINT(33.68752682742512 -118.06698471242774) bank26677 +26678 POINT(34.01941929294512 -118.44094014743301) bank26678 +26679 POINT(33.975906473519274 -119.15453682113609) bank26679 +26680 POINT(34.102216524411006 -117.82611890990434) bank26680 +26681 POINT(33.93770255337351 -118.89836091994167) bank26681 +26682 POINT(33.92347438307094 -118.17857380094073) bank26682 +26683 POINT(33.629929426755474 -117.93092138719587) bank26683 +26684 POINT(34.654651522909234 -117.37140867751914) bank26684 +26685 POINT(33.33654234842791 -118.24650624378356) bank26685 +26686 POINT(33.69689070278296 -118.15152199836025) bank26686 +26687 POINT(33.5198661187677 -118.06192673573827) bank26687 +26688 POINT(33.89347897152499 -118.30067659854687) bank26688 +26689 POINT(33.311682540385256 -117.91958905266047) bank26689 +26690 POINT(33.19765001754586 -119.14399874039418) bank26690 +26691 POINT(34.71618059135644 -118.79972306605785) bank26691 +26692 POINT(34.61845794557302 -119.03416720322684) bank26692 +26693 POINT(34.56646536795883 -118.8487314846211) bank26693 +26694 POINT(33.7882705138377 -118.82909330011687) bank26694 +26695 POINT(34.303235739061996 -118.5213099829462) bank26695 +26696 POINT(34.39239267670223 -119.0899499465884) bank26696 +26697 POINT(33.441957853283185 -118.71805225094302) bank26697 +26698 POINT(33.33689230131093 -118.28918412363355) bank26698 +26699 POINT(33.240459164083866 -117.80694950710753) bank26699 +26700 POINT(34.0011493301224 -118.57918175791636) bank26700 +26701 POINT(34.85150893236242 -118.27289844194955) bank26701 +26702 POINT(33.725017386864025 -118.2767843876865) bank26702 +26703 POINT(33.68987273572058 -117.58356159583903) bank26703 +26704 POINT(33.877398456365924 -118.11925596790276) bank26704 +26705 POINT(33.81440544281762 -118.77107415767222) bank26705 +26706 POINT(34.92714508496151 -118.08874915293046) bank26706 +26707 POINT(34.15681611019433 -119.16343168175906) bank26707 +26708 POINT(34.182198922262714 -118.4169624785008) bank26708 +26709 POINT(33.73969032695406 -118.9312470726404) bank26709 +26710 POINT(34.714891851216365 -118.1869031708142) bank26710 +26711 POINT(33.13614005435745 -117.7740507966686) bank26711 +26712 POINT(33.83573825891044 -117.97362032838572) bank26712 +26713 POINT(33.22044669870167 -117.49209661563314) bank26713 +26714 POINT(33.23096575641103 -118.95877347384237) bank26714 +26715 POINT(34.08236322985724 -118.73310003702001) bank26715 +26716 POINT(33.31609527577999 -118.2531396169429) bank26716 +26717 POINT(34.30844551159217 -118.99239529997651) bank26717 +26718 POINT(34.78792963369305 -117.36570164846222) bank26718 +26719 POINT(33.55262711364206 -117.56740219658222) bank26719 +26720 POINT(33.907090431971426 -118.65756615151467) bank26720 +26721 POINT(33.85652702220797 -119.05045603138555) bank26721 +26722 POINT(34.0156094478743 -118.95943978370691) bank26722 +26723 POINT(33.13533193878851 -118.0592695298763) bank26723 +26724 POINT(34.30516406080413 -118.09892954283565) bank26724 +26725 POINT(33.06291500149317 -117.6327227373172) bank26725 +26726 POINT(34.52460243864892 -117.8005030182653) bank26726 +26727 POINT(33.45430959227474 -118.97032013314639) bank26727 +26728 POINT(34.50488796163953 -119.07519510542137) bank26728 +26729 POINT(33.534692443847106 -118.0337679721497) bank26729 +26730 POINT(34.85547188865402 -118.5333659411892) bank26730 +26731 POINT(33.19753374546288 -117.5523841837978) bank26731 +26732 POINT(34.59668766425484 -119.17265258737235) bank26732 +26733 POINT(34.536269334145274 -119.02909587485975) bank26733 +26734 POINT(34.86842506201566 -119.04355515081366) bank26734 +26735 POINT(33.222854239966516 -118.81308573573676) bank26735 +26736 POINT(33.77829340055121 -118.20338620526539) bank26736 +26737 POINT(34.38290710596214 -118.29531308030981) bank26737 +26738 POINT(33.09043772609308 -118.30022928224875) bank26738 +26739 POINT(33.09737741909314 -117.79371750763436) bank26739 +26740 POINT(34.07179053689106 -118.0515091580015) bank26740 +26741 POINT(33.769079566816586 -118.44186634727757) bank26741 +26742 POINT(34.5988572825141 -118.6486729881377) bank26742 +26743 POINT(35.041870449016365 -118.0269150723443) bank26743 +26744 POINT(33.2608056840812 -117.60462166116262) bank26744 +26745 POINT(34.17198000428364 -118.74507320411844) bank26745 +26746 POINT(34.79265370051427 -117.3392077535465) bank26746 +26747 POINT(34.92113248179114 -118.83172372996349) bank26747 +26748 POINT(33.61331760998636 -118.92374141096751) bank26748 +26749 POINT(33.49346304438099 -117.98623371638413) bank26749 +26750 POINT(33.68916694565351 -118.15285860359444) bank26750 +26751 POINT(35.05028999489352 -117.52492731520186) bank26751 +26752 POINT(33.259091052952336 -118.98184507972017) bank26752 +26753 POINT(34.229408647261835 -117.58619638308203) bank26753 +26754 POINT(33.41765300988057 -118.31828696546056) bank26754 +26755 POINT(33.76449358233657 -117.66749143094356) bank26755 +26756 POINT(34.90141180631468 -117.84397405162365) bank26756 +26757 POINT(33.1973563149097 -117.98667567870925) bank26757 +26758 POINT(34.057089111176985 -118.48272473857593) bank26758 +26759 POINT(34.50585860677871 -118.79005417241339) bank26759 +26760 POINT(33.571169964761765 -118.43755789416782) bank26760 +26761 POINT(35.005125453094294 -119.1786935737053) bank26761 +26762 POINT(33.31354557817773 -118.61267855095471) bank26762 +26763 POINT(33.65392328482984 -118.3591776756122) bank26763 +26764 POINT(33.082456273281835 -119.20609768390757) bank26764 +26765 POINT(34.34211401869717 -117.78840367469627) bank26765 +26766 POINT(34.274640784697944 -118.03561196950534) bank26766 +26767 POINT(33.115553023863086 -118.16628866464474) bank26767 +26768 POINT(34.01984277566724 -118.5352921433242) bank26768 +26769 POINT(34.779963114611505 -118.0382816501808) bank26769 +26770 POINT(34.26880619350452 -118.5002484780424) bank26770 +26771 POINT(33.10427158867683 -117.80304473680914) bank26771 +26772 POINT(33.14896164017633 -117.79568128659268) bank26772 +26773 POINT(33.61031419158249 -117.2946823758449) bank26773 +26774 POINT(34.23900999162127 -118.41135903351777) bank26774 +26775 POINT(34.374606577316015 -118.50416977571885) bank26775 +26776 POINT(34.93541321937453 -118.89870920796317) bank26776 +26777 POINT(33.61812220050075 -117.97599901249754) bank26777 +26778 POINT(33.234705509582085 -117.69582865393221) bank26778 +26779 POINT(34.17020870113834 -118.12240360806659) bank26779 +26780 POINT(34.62291958254684 -118.150400696304) bank26780 +26781 POINT(34.95717278404444 -118.97160198323098) bank26781 +26782 POINT(33.209221295775684 -117.66038666839093) bank26782 +26783 POINT(34.29632897931122 -118.08149578002286) bank26783 +26784 POINT(34.472275506509675 -118.01001382118338) bank26784 +26785 POINT(34.42675993989175 -118.3003546840372) bank26785 +26786 POINT(33.270714860323466 -118.98299111171164) bank26786 +26787 POINT(33.81367565856462 -117.63831687602836) bank26787 +26788 POINT(33.83407965589313 -118.38915209677377) bank26788 +26789 POINT(34.101081556889454 -117.3233355851006) bank26789 +26790 POINT(34.68050498384786 -118.33137736962182) bank26790 +26791 POINT(33.1103500282979 -118.22073616161993) bank26791 +26792 POINT(33.646204182866846 -118.60320143804739) bank26792 +26793 POINT(33.704017959945745 -117.99750163279307) bank26793 +26794 POINT(33.554812523021056 -117.47464348715563) bank26794 +26795 POINT(33.62404255303997 -117.5620523775915) bank26795 +26796 POINT(33.281839272127094 -119.06312297752233) bank26796 +26797 POINT(33.928137526637386 -117.81150077979571) bank26797 +26798 POINT(34.74170163623735 -117.90990547304175) bank26798 +26799 POINT(34.88282712062627 -118.47527908946282) bank26799 +26800 POINT(33.50124784179901 -118.62955381826802) bank26800 +26801 POINT(34.12848057937814 -117.56008918277021) bank26801 +26802 POINT(34.17545680500187 -118.25326994306799) bank26802 +26803 POINT(33.75235934050928 -117.9605972164403) bank26803 +26804 POINT(33.80914162177747 -118.07559734249877) bank26804 +26805 POINT(34.74886775643535 -117.71772516051179) bank26805 +26806 POINT(34.1809427772634 -118.36581383587527) bank26806 +26807 POINT(33.172328185094166 -118.35311977510062) bank26807 +26808 POINT(34.00799405538457 -118.1424740069152) bank26808 +26809 POINT(33.288177891581256 -118.23972518906184) bank26809 +26810 POINT(33.48633657790618 -118.46230703430142) bank26810 +26811 POINT(33.78106286713249 -119.16435563800833) bank26811 +26812 POINT(34.44088855353495 -117.93854732687295) bank26812 +26813 POINT(33.095111781907384 -119.18156056716943) bank26813 +26814 POINT(34.542653632880686 -117.606450246637) bank26814 +26815 POINT(34.402934137208106 -117.71497696594749) bank26815 +26816 POINT(33.56225605257092 -119.05651934888085) bank26816 +26817 POINT(33.96100364120121 -118.38401002885698) bank26817 +26818 POINT(34.27358408208933 -117.36925308475308) bank26818 +26819 POINT(33.501320278460234 -118.28806406136208) bank26819 +26820 POINT(34.06940379687847 -119.18854394464556) bank26820 +26821 POINT(34.73686839148851 -117.30660806344751) bank26821 +26822 POINT(33.53732665386725 -117.47896535235806) bank26822 +26823 POINT(33.77766089751755 -118.39733552272072) bank26823 +26824 POINT(33.5212081571222 -118.84169353296056) bank26824 +26825 POINT(34.010283226425756 -119.21715459944733) bank26825 +26826 POINT(34.13318243724014 -117.59095836004474) bank26826 +26827 POINT(34.72729226756117 -118.90720211534571) bank26827 +26828 POINT(33.42778990942278 -118.25621926543393) bank26828 +26829 POINT(34.16555696283178 -118.89004941416705) bank26829 +26830 POINT(34.53463118490547 -118.53951817996918) bank26830 +26831 POINT(34.18075590991761 -118.4240670069717) bank26831 +26832 POINT(34.49377241995474 -118.3651586299721) bank26832 +26833 POINT(33.30500916006592 -117.43695888562746) bank26833 +26834 POINT(33.21546316168656 -118.34384433304066) bank26834 +26835 POINT(33.106786371108704 -118.38383863154574) bank26835 +26836 POINT(33.612004000452586 -117.56835783277852) bank26836 +26837 POINT(33.51485669166798 -119.04407274124293) bank26837 +26838 POINT(34.31399581548485 -118.22752631870904) bank26838 +26839 POINT(34.35452790864512 -117.37464958509763) bank26839 +26840 POINT(33.599627641014635 -119.17722881739499) bank26840 +26841 POINT(34.17064341714421 -118.10883096967784) bank26841 +26842 POINT(34.9783389190312 -117.56476772237802) bank26842 +26843 POINT(34.91896610570643 -118.00765780225225) bank26843 +26844 POINT(34.205693513847514 -118.6806059038348) bank26844 +26845 POINT(33.665343280477906 -117.54621515991622) bank26845 +26846 POINT(33.48861960455546 -118.56541647298732) bank26846 +26847 POINT(33.82553810900517 -119.20946117798499) bank26847 +26848 POINT(34.03912399068535 -118.49864723830781) bank26848 +26849 POINT(33.72928346319587 -117.58872145925194) bank26849 +26850 POINT(34.64363844850098 -118.7806271926376) bank26850 +26851 POINT(33.364520008447094 -118.88622035974616) bank26851 +26852 POINT(33.54980749734269 -119.0747483417241) bank26852 +26853 POINT(34.970135531727564 -118.69465717895137) bank26853 +26854 POINT(34.73833104791343 -119.08566696020333) bank26854 +26855 POINT(34.59347798800342 -119.01032418641397) bank26855 +26856 POINT(33.19922916347442 -119.15947106871403) bank26856 +26857 POINT(34.84335480571798 -117.759738122086) bank26857 +26858 POINT(34.5047102976733 -119.08758554219975) bank26858 +26859 POINT(34.929996583161476 -118.79973478610039) bank26859 +26860 POINT(33.290413998650514 -119.10575701053138) bank26860 +26861 POINT(35.051382140155084 -118.72129842367971) bank26861 +26862 POINT(33.573477497111185 -118.51915732746208) bank26862 +26863 POINT(34.198423203501854 -118.0967039094767) bank26863 +26864 POINT(34.0457245026293 -117.8142047387455) bank26864 +26865 POINT(33.779432568547335 -117.4133718214368) bank26865 +26866 POINT(33.691852763654595 -117.59137911770286) bank26866 +26867 POINT(33.425426551822206 -117.96428977465965) bank26867 +26868 POINT(35.020193712632256 -118.80968733153303) bank26868 +26869 POINT(34.48750502317149 -119.20677473213848) bank26869 +26870 POINT(33.699890257402345 -118.28617561570572) bank26870 +26871 POINT(34.12534537294244 -118.61330201420415) bank26871 +26872 POINT(33.40311566950698 -117.56394939564615) bank26872 +26873 POINT(33.4578527283044 -118.25202772504039) bank26873 +26874 POINT(33.42714622946767 -117.56512191052218) bank26874 +26875 POINT(33.41904143871422 -117.35521914457469) bank26875 +26876 POINT(34.56194692811382 -117.29353996758957) bank26876 +26877 POINT(33.68732091299257 -117.35781913285662) bank26877 +26878 POINT(33.655067300523235 -117.89511687973457) bank26878 +26879 POINT(33.4737578086539 -118.41964410610521) bank26879 +26880 POINT(33.151242772879215 -118.73175631550187) bank26880 +26881 POINT(33.55865542059165 -117.65970346626023) bank26881 +26882 POINT(33.338423525804636 -119.14469596223783) bank26882 +26883 POINT(34.82240765597426 -117.71550876615997) bank26883 +26884 POINT(33.09553091484013 -118.48023934563497) bank26884 +26885 POINT(34.87512786344626 -119.14100685266271) bank26885 +26886 POINT(34.6220246872335 -119.10233945717442) bank26886 +26887 POINT(33.95331726519915 -117.9034641454588) bank26887 +26888 POINT(34.14774588828231 -118.14343067582661) bank26888 +26889 POINT(33.26947935990689 -118.06978385298726) bank26889 +26890 POINT(33.3066914499089 -119.04170802154391) bank26890 +26891 POINT(33.856378971146924 -117.7301118890213) bank26891 +26892 POINT(33.61022232255378 -118.8608787206924) bank26892 +26893 POINT(33.47631355655166 -118.12714484204113) bank26893 +26894 POINT(33.09263164092847 -118.49902080559986) bank26894 +26895 POINT(33.699432283715865 -117.54902367618116) bank26895 +26896 POINT(34.89600491351163 -117.67812987860017) bank26896 +26897 POINT(34.46534073905995 -118.26559992777078) bank26897 +26898 POINT(33.54785577933316 -119.23405146647397) bank26898 +26899 POINT(34.58492800802617 -117.99624055283394) bank26899 +26900 POINT(33.629913802731586 -117.67785704179434) bank26900 +26901 POINT(33.264337597232355 -117.24748994307109) bank26901 +26902 POINT(34.022681192939395 -117.33096344606749) bank26902 +26903 POINT(34.99282824091494 -118.39183802861791) bank26903 +26904 POINT(34.00229990754002 -118.96329768165792) bank26904 +26905 POINT(34.39568287192872 -119.19404192546735) bank26905 +26906 POINT(33.27933574260206 -118.3123852339948) bank26906 +26907 POINT(33.34016286126904 -117.90096324260841) bank26907 +26908 POINT(33.84822952078932 -118.96419359038195) bank26908 +26909 POINT(33.460599787845574 -118.41379882182805) bank26909 +26910 POINT(33.851857525843656 -118.7668610170145) bank26910 +26911 POINT(33.715869849637095 -119.05302909400092) bank26911 +26912 POINT(33.24997682657371 -118.52552980064739) bank26912 +26913 POINT(34.84774146486974 -117.65820737798151) bank26913 +26914 POINT(34.57033139039704 -117.9581028194394) bank26914 +26915 POINT(33.964989940516716 -118.83832614191354) bank26915 +26916 POINT(33.48066541087815 -117.30575266808268) bank26916 +26917 POINT(34.28675567867961 -118.58170610780333) bank26917 +26918 POINT(34.00875584162459 -118.85501405872287) bank26918 +26919 POINT(33.66268789984155 -119.09224160243554) bank26919 +26920 POINT(34.6993386766148 -117.94500427610885) bank26920 +26921 POINT(33.34707036704083 -118.59665593532445) bank26921 +26922 POINT(34.82477935603674 -118.31467916309482) bank26922 +26923 POINT(33.4762081561539 -118.78810818851998) bank26923 +26924 POINT(33.7997896456339 -118.43724707826861) bank26924 +26925 POINT(34.14829977150559 -117.33008952969385) bank26925 +26926 POINT(34.20878302810594 -117.78973705617119) bank26926 +26927 POINT(34.7239168604629 -118.92997833388442) bank26927 +26928 POINT(33.96002317524511 -117.52535344283136) bank26928 +26929 POINT(33.959495328047765 -118.0228284097695) bank26929 +26930 POINT(34.94758288617522 -118.20435930815955) bank26930 +26931 POINT(33.54583173124059 -118.54246457813076) bank26931 +26932 POINT(34.06191758923708 -117.45557415104467) bank26932 +26933 POINT(34.90936976676329 -118.8084708215812) bank26933 +26934 POINT(34.85648274916317 -118.53006196543492) bank26934 +26935 POINT(34.20533181038071 -117.30163679720724) bank26935 +26936 POINT(34.427015068334924 -118.53635476364522) bank26936 +26937 POINT(34.691652604356825 -117.58160592117125) bank26937 +26938 POINT(33.46550618538349 -117.52160703514474) bank26938 +26939 POINT(34.64311092356142 -117.3266686384088) bank26939 +26940 POINT(34.139485349763895 -117.63268549001342) bank26940 +26941 POINT(33.992970516203215 -117.47653147902447) bank26941 +26942 POINT(33.44911789147274 -119.13470467451567) bank26942 +26943 POINT(33.704462977897116 -117.31753263011805) bank26943 +26944 POINT(34.66471487785302 -118.60511767065708) bank26944 +26945 POINT(35.0404240250641 -117.48607970982152) bank26945 +26946 POINT(34.427752808789016 -119.21629710103944) bank26946 +26947 POINT(34.13245390494529 -118.43332496600873) bank26947 +26948 POINT(33.071978519183105 -118.86812106962446) bank26948 +26949 POINT(33.85459913468899 -118.36487106708884) bank26949 +26950 POINT(34.55216936897603 -117.88633680197644) bank26950 +26951 POINT(33.091990623631865 -119.01455049028942) bank26951 +26952 POINT(33.24456194551249 -117.72744484164802) bank26952 +26953 POINT(33.87099509092103 -117.9964407837997) bank26953 +26954 POINT(33.2782835850279 -118.17974933526851) bank26954 +26955 POINT(33.063852615497424 -118.95215110470838) bank26955 +26956 POINT(33.60745129394394 -117.44832245323443) bank26956 +26957 POINT(34.789131501987534 -117.51263269239891) bank26957 +26958 POINT(33.909031812476655 -118.67537549295781) bank26958 +26959 POINT(33.05588629087137 -118.22611329426374) bank26959 +26960 POINT(33.99363310496723 -117.86634895629031) bank26960 +26961 POINT(34.18162133675808 -117.29568152207037) bank26961 +26962 POINT(33.9540194256756 -118.61638755852023) bank26962 +26963 POINT(34.74509855104534 -117.35565101870525) bank26963 +26964 POINT(34.46080596061164 -118.0760844946868) bank26964 +26965 POINT(34.34988532889049 -119.01111654856811) bank26965 +26966 POINT(34.02485217606406 -119.03886487850288) bank26966 +26967 POINT(34.40062924576792 -118.24645066289622) bank26967 +26968 POINT(33.47990543785077 -119.10330395494833) bank26968 +26969 POINT(34.41207749200986 -117.96358317061761) bank26969 +26970 POINT(34.58963444704128 -118.80988656794025) bank26970 +26971 POINT(33.841333290760005 -118.97123180209628) bank26971 +26972 POINT(34.24027290208141 -117.90304158308413) bank26972 +26973 POINT(33.05504181795106 -118.5773179449155) bank26973 +26974 POINT(34.089235592852745 -118.76894335449138) bank26974 +26975 POINT(33.49556296336059 -118.84153664543373) bank26975 +26976 POINT(33.38510304347925 -117.37917408305671) bank26976 +26977 POINT(33.49465626503253 -118.913490881605) bank26977 +26978 POINT(34.37344614352445 -118.38243084243659) bank26978 +26979 POINT(33.87632634755454 -117.81753223028437) bank26979 +26980 POINT(34.69461923973001 -117.63540952841437) bank26980 +26981 POINT(33.81599779140597 -117.7313865193293) bank26981 +26982 POINT(33.326697960106145 -118.55352647866678) bank26982 +26983 POINT(34.03109840934897 -118.97621433381858) bank26983 +26984 POINT(34.57795764766553 -117.85249657838892) bank26984 +26985 POINT(34.90649894412609 -118.66607784284155) bank26985 +26986 POINT(33.33990047351178 -117.94102180637239) bank26986 +26987 POINT(33.324559020070886 -119.22819903643575) bank26987 +26988 POINT(34.92657280755792 -119.06650833823403) bank26988 +26989 POINT(33.797351788406324 -117.36878356307243) bank26989 +26990 POINT(33.89909503141937 -118.12540449404217) bank26990 +26991 POINT(33.71254225000079 -118.47542163911812) bank26991 +26992 POINT(34.81348681104588 -118.53475121215192) bank26992 +26993 POINT(34.894343090674454 -117.48865712576415) bank26993 +26994 POINT(34.1164849538343 -117.82516193853508) bank26994 +26995 POINT(34.89588524262918 -118.26565897542834) bank26995 +26996 POINT(33.24022275632599 -118.0968299471871) bank26996 +26997 POINT(34.7017001704085 -118.10403803243969) bank26997 +26998 POINT(34.78390395737169 -117.3898965885771) bank26998 +26999 POINT(34.13925101716188 -118.31535788258877) bank26999 +27000 POINT(33.402495070732165 -118.6879603062041) bank27000 +27001 POINT(34.94783301186462 -117.30358890442614) bank27001 +27002 POINT(33.883033794224815 -118.05958809498034) bank27002 +27003 POINT(34.60058499669072 -118.02771085894877) bank27003 +27004 POINT(34.77350816506744 -117.4959584783345) bank27004 +27005 POINT(34.4607016368237 -118.87284651871741) bank27005 +27006 POINT(33.91168772290853 -117.82976415619207) bank27006 +27007 POINT(33.92266558852436 -117.52379931585946) bank27007 +27008 POINT(33.96180649533597 -117.60878824152137) bank27008 +27009 POINT(33.094148901147165 -117.60371873909216) bank27009 +27010 POINT(34.95552669453099 -118.45746727272022) bank27010 +27011 POINT(33.57136385545398 -118.56415149229774) bank27011 +27012 POINT(33.26510951672045 -118.82079093313862) bank27012 +27013 POINT(33.83684314898375 -117.76072020975757) bank27013 +27014 POINT(34.94699484802095 -117.91065433223885) bank27014 +27015 POINT(34.885362390757514 -118.91742221901616) bank27015 +27016 POINT(34.024200547345664 -118.91765636098114) bank27016 +27017 POINT(33.2097265426382 -117.61217124272882) bank27017 +27018 POINT(33.198850360122485 -117.83905245025227) bank27018 +27019 POINT(33.63931150777858 -117.70524693545482) bank27019 +27020 POINT(34.42096977816185 -118.10832477902564) bank27020 +27021 POINT(33.42474011909339 -118.20048172139772) bank27021 +27022 POINT(34.73501418250018 -118.5419765804468) bank27022 +27023 POINT(33.10190510158114 -119.21887135885379) bank27023 +27024 POINT(34.598252093322166 -118.41837750937273) bank27024 +27025 POINT(33.95752136132187 -118.15682408128033) bank27025 +27026 POINT(34.79716201528081 -117.40054427714482) bank27026 +27027 POINT(34.61278223609875 -118.06744911281002) bank27027 +27028 POINT(33.85675647918375 -117.24780422106944) bank27028 +27029 POINT(34.57352488468149 -117.34456768973307) bank27029 +27030 POINT(33.460403401912515 -117.42403302388448) bank27030 +27031 POINT(33.412590189692295 -117.82806991464871) bank27031 +27032 POINT(33.81015592935256 -118.6649662666681) bank27032 +27033 POINT(33.291951947163554 -118.62891371479033) bank27033 +27034 POINT(33.788428922607174 -118.90539532275773) bank27034 +27035 POINT(34.665746070399464 -117.80116550929168) bank27035 +27036 POINT(34.11853579604017 -117.95059308488949) bank27036 +27037 POINT(34.88781896206562 -117.91733475731422) bank27037 +27038 POINT(33.731161606017935 -118.25444815925601) bank27038 +27039 POINT(33.613032368431384 -118.84922351886006) bank27039 +27040 POINT(34.4391271113182 -117.40686056584663) bank27040 +27041 POINT(34.69543151807111 -118.16577058044204) bank27041 +27042 POINT(34.012693496567174 -117.85090892143079) bank27042 +27043 POINT(34.938485113849865 -117.3523880678679) bank27043 +27044 POINT(34.729141186969024 -118.56458501859923) bank27044 +27045 POINT(34.79375943995129 -118.87469380740256) bank27045 +27046 POINT(34.28053607861218 -118.38180019294606) bank27046 +27047 POINT(33.33467440497977 -118.22152664152348) bank27047 +27048 POINT(33.31912037777128 -117.83640555933592) bank27048 +27049 POINT(33.56637050764156 -118.04164056253973) bank27049 +27050 POINT(33.15832299933523 -117.5068972113355) bank27050 +27051 POINT(34.53813836462457 -118.0903142979502) bank27051 +27052 POINT(34.83495224980557 -118.03914606284324) bank27052 +27053 POINT(34.30295105770363 -118.21955990155593) bank27053 +27054 POINT(34.00494056732914 -118.87672396275735) bank27054 +27055 POINT(33.44359670436466 -117.83129334489783) bank27055 +27056 POINT(33.82960785204015 -118.12260592888525) bank27056 +27057 POINT(34.231612822199686 -117.44883346668331) bank27057 +27058 POINT(33.71067840391532 -119.05614031309824) bank27058 +27059 POINT(33.632816997903625 -117.38427246858046) bank27059 +27060 POINT(35.02698520252754 -117.29554150705033) bank27060 +27061 POINT(33.10924987508576 -118.52111750230533) bank27061 +27062 POINT(33.408121276512894 -118.75566788800339) bank27062 +27063 POINT(33.49585125496222 -118.69288013006219) bank27063 +27064 POINT(33.676619385983784 -119.00049180314386) bank27064 +27065 POINT(33.24038050323436 -118.14107966917666) bank27065 +27066 POINT(33.130207750922196 -117.61381767738978) bank27066 +27067 POINT(33.206073249419596 -118.01091098719456) bank27067 +27068 POINT(34.69872528629678 -118.57190752403251) bank27068 +27069 POINT(33.93716263288391 -117.53369041245026) bank27069 +27070 POINT(33.918622414487025 -118.95999668475275) bank27070 +27071 POINT(33.726659262211186 -118.16156960177199) bank27071 +27072 POINT(33.49304661674515 -117.25155229548531) bank27072 +27073 POINT(34.301751228795816 -118.80791615271521) bank27073 +27074 POINT(34.30612196768494 -118.78369276945197) bank27074 +27075 POINT(34.37261788772849 -117.78098737123088) bank27075 +27076 POINT(33.96043471985571 -118.5461786973779) bank27076 +27077 POINT(33.51042104867501 -117.87094811031896) bank27077 +27078 POINT(33.20348872375908 -118.88236189029885) bank27078 +27079 POINT(33.35157473021797 -118.26665055089774) bank27079 +27080 POINT(33.69404050587311 -117.78179148742667) bank27080 +27081 POINT(33.77705616211456 -118.28433426706788) bank27081 +27082 POINT(33.43586677936507 -117.51850511041302) bank27082 +27083 POINT(33.77059601943486 -117.98065073673423) bank27083 +27084 POINT(34.65198907349235 -118.1366688438352) bank27084 +27085 POINT(34.50842535154069 -118.7848773506974) bank27085 +27086 POINT(34.47957375030297 -118.84715477810497) bank27086 +27087 POINT(34.68993442473583 -119.17068147030086) bank27087 +27088 POINT(33.54196054496153 -117.58666589863027) bank27088 +27089 POINT(34.11547026788438 -119.0647978451109) bank27089 +27090 POINT(33.1307692820387 -118.68166425330068) bank27090 +27091 POINT(33.957334338745376 -117.6404357119689) bank27091 +27092 POINT(34.93617958989059 -117.53760509914449) bank27092 +27093 POINT(33.67871238300833 -118.03135511907544) bank27093 +27094 POINT(33.384951429158384 -118.82149507380059) bank27094 +27095 POINT(34.4594016581229 -118.65268247896836) bank27095 +27096 POINT(33.967301923046044 -119.09969776644702) bank27096 +27097 POINT(33.091731134198184 -118.32821216902236) bank27097 +27098 POINT(33.115535726933636 -117.88982835631357) bank27098 +27099 POINT(33.743804750176956 -117.26455644723808) bank27099 +27100 POINT(34.30207332810606 -117.26317164551504) bank27100 +27101 POINT(33.547096169962565 -119.17134739886347) bank27101 +27102 POINT(34.50827461146677 -118.82520007308287) bank27102 +27103 POINT(33.209447632419966 -118.6204597311473) bank27103 +27104 POINT(34.764339838109564 -117.78119829353733) bank27104 +27105 POINT(33.93401759283333 -118.75320344696185) bank27105 +27106 POINT(34.573575383148516 -118.37934404524384) bank27106 +27107 POINT(34.942899333295316 -118.46119427923247) bank27107 +27108 POINT(34.65793567653075 -118.59266332710772) bank27108 +27109 POINT(34.79244531652396 -118.72861570940883) bank27109 +27110 POINT(33.06963270214986 -118.35902230060175) bank27110 +27111 POINT(34.82336582629205 -117.30532452499925) bank27111 +27112 POINT(34.378439330519534 -117.54505788585566) bank27112 +27113 POINT(34.42638984823919 -117.41363772580424) bank27113 +27114 POINT(33.19339431915833 -118.18493215174972) bank27114 +27115 POINT(34.45250602753779 -118.30215529567259) bank27115 +27116 POINT(33.35479028572333 -119.1891107280982) bank27116 +27117 POINT(34.34312733626287 -117.63276311137352) bank27117 +27118 POINT(33.43474223693923 -119.02411895647033) bank27118 +27119 POINT(33.494769067380005 -117.88089898023306) bank27119 +27120 POINT(33.1146711957653 -118.44100169226986) bank27120 +27121 POINT(33.74306674733943 -117.7342865971588) bank27121 +27122 POINT(34.7768902228158 -117.51787899051308) bank27122 +27123 POINT(34.99515868274278 -117.86295186117754) bank27123 +27124 POINT(33.136354959544875 -117.52365268258825) bank27124 +27125 POINT(33.3174812693902 -118.20635485913627) bank27125 +27126 POINT(33.22139673353267 -118.19407348035638) bank27126 +27127 POINT(34.66896347034506 -118.25866911751581) bank27127 +27128 POINT(34.90812169729146 -119.06786274105137) bank27128 +27129 POINT(34.923052672502564 -117.29028011463441) bank27129 +27130 POINT(33.249927320328666 -118.50984593591095) bank27130 +27131 POINT(34.777538295412356 -118.11145412482675) bank27131 +27132 POINT(33.52852646965084 -118.77966516622658) bank27132 +27133 POINT(33.85569191296551 -118.50721664713065) bank27133 +27134 POINT(34.96077222333386 -118.89483666655498) bank27134 +27135 POINT(34.22676167273017 -117.82411554508941) bank27135 +27136 POINT(34.169459096363205 -117.6377415156394) bank27136 +27137 POINT(33.26673843473833 -118.3519740217853) bank27137 +27138 POINT(34.32741068073154 -118.67245606303203) bank27138 +27139 POINT(34.74762936550503 -118.22366073000664) bank27139 +27140 POINT(34.35437624812959 -118.18771635037626) bank27140 +27141 POINT(34.28783182639629 -118.74407592267985) bank27141 +27142 POINT(35.044299934224114 -118.8907166002978) bank27142 +27143 POINT(33.109050023837675 -119.14109826769794) bank27143 +27144 POINT(33.44097833875642 -117.96244255346629) bank27144 +27145 POINT(33.91809745112718 -118.87392255919438) bank27145 +27146 POINT(33.81364971110556 -117.66209215474767) bank27146 +27147 POINT(33.54834395440167 -118.33938467463274) bank27147 +27148 POINT(33.18460577738584 -117.84406766743368) bank27148 +27149 POINT(33.27509345037186 -117.24954211323374) bank27149 +27150 POINT(34.95094241294395 -118.79306648680529) bank27150 +27151 POINT(33.83097804839172 -118.4523370763084) bank27151 +27152 POINT(34.00988093417236 -117.81177227568568) bank27152 +27153 POINT(34.18769831144795 -119.15462558008853) bank27153 +27154 POINT(34.65644323279909 -118.2578139492624) bank27154 +27155 POINT(33.1862430839575 -117.25854946323561) bank27155 +27156 POINT(34.77081691462282 -117.54719430815801) bank27156 +27157 POINT(34.80803670221805 -117.53039241061083) bank27157 +27158 POINT(33.85220355094144 -117.75667585479991) bank27158 +27159 POINT(33.3268550003984 -117.95992693174017) bank27159 +27160 POINT(34.28245312060595 -118.13253724941502) bank27160 +27161 POINT(33.843145146139115 -117.48767044556789) bank27161 +27162 POINT(33.435506838408614 -117.83214328947987) bank27162 +27163 POINT(34.32535907974626 -117.96383277675028) bank27163 +27164 POINT(34.21228056025745 -117.2651225882822) bank27164 +27165 POINT(33.6004270898437 -117.64582609164539) bank27165 +27166 POINT(33.524166203605155 -117.83458410797991) bank27166 +27167 POINT(33.95937755827922 -118.25688233952953) bank27167 +27168 POINT(34.48493572603648 -118.98834630889945) bank27168 +27169 POINT(34.636638601523345 -118.30377468908554) bank27169 +27170 POINT(33.810855013118754 -118.94940483167125) bank27170 +27171 POINT(34.76471679435495 -118.35078795202445) bank27171 +27172 POINT(33.68076209321388 -117.4567857655666) bank27172 +27173 POINT(33.696258871091736 -118.52642604180042) bank27173 +27174 POINT(33.07779675653563 -117.56975664644438) bank27174 +27175 POINT(33.950649621472564 -117.80241377015588) bank27175 +27176 POINT(34.72367200351037 -118.73988001535358) bank27176 +27177 POINT(33.83794579348109 -118.69234022176876) bank27177 +27178 POINT(34.69660378774645 -118.66640856871449) bank27178 +27179 POINT(33.65432408831825 -118.7018244779897) bank27179 +27180 POINT(34.30245388824033 -117.48448481261937) bank27180 +27181 POINT(33.22492492694315 -118.73054676694363) bank27181 +27182 POINT(34.035416898039294 -118.57468592313485) bank27182 +27183 POINT(33.95894092048849 -117.45141331291131) bank27183 +27184 POINT(33.09998923787793 -118.31756039607713) bank27184 +27185 POINT(33.117343315560305 -117.80231573980147) bank27185 +27186 POINT(33.30107500121121 -118.0662181932653) bank27186 +27187 POINT(34.66286563747607 -118.94565216969866) bank27187 +27188 POINT(33.28236293789061 -117.98323199504436) bank27188 +27189 POINT(34.4207870037909 -118.64061945238355) bank27189 +27190 POINT(34.80173593557541 -118.69680869144631) bank27190 +27191 POINT(34.237348031875094 -118.5422688690415) bank27191 +27192 POINT(34.2428242840343 -118.20104709556425) bank27192 +27193 POINT(33.359587201531994 -117.94017028279231) bank27193 +27194 POINT(34.501334862639105 -118.73947794767973) bank27194 +27195 POINT(33.66953254728649 -119.2321481717279) bank27195 +27196 POINT(34.11604148631928 -117.35505934994772) bank27196 +27197 POINT(33.062278008805016 -118.01418854281746) bank27197 +27198 POINT(34.861197101855744 -117.70258062146492) bank27198 +27199 POINT(33.48055235601498 -117.96842905194679) bank27199 +27200 POINT(33.96837631763591 -118.18791503352213) bank27200 +27201 POINT(34.72425445482173 -118.3661476282113) bank27201 +27202 POINT(33.33610453021087 -118.65827267421766) bank27202 +27203 POINT(35.00893941766296 -119.1911787849039) bank27203 +27204 POINT(33.65625912393304 -118.8246865163495) bank27204 +27205 POINT(34.45284618335283 -117.53411873549545) bank27205 +27206 POINT(34.379628415011965 -119.1978643423436) bank27206 +27207 POINT(34.346795591533684 -119.1958950626753) bank27207 +27208 POINT(34.49802417632952 -118.97590379160398) bank27208 +27209 POINT(34.60806443271288 -117.72340516546323) bank27209 +27210 POINT(33.37539414258809 -118.79181699117952) bank27210 +27211 POINT(33.6428173186694 -117.27125037261973) bank27211 +27212 POINT(33.16295381740287 -117.72748147342188) bank27212 +27213 POINT(33.31878935861629 -117.93145487152472) bank27213 +27214 POINT(34.6699227180805 -119.20211294886204) bank27214 +27215 POINT(34.22735057215936 -118.49405320840435) bank27215 +27216 POINT(34.28437314298638 -117.52366455584715) bank27216 +27217 POINT(33.28588321687697 -118.6770677659557) bank27217 +27218 POINT(33.60356496501926 -118.50267025242098) bank27218 +27219 POINT(33.98685419170149 -119.10947589285601) bank27219 +27220 POINT(34.512937959102395 -117.95303046541252) bank27220 +27221 POINT(33.730619975552635 -117.70610272919598) bank27221 +27222 POINT(33.60034845256722 -118.98334106099371) bank27222 +27223 POINT(34.670144137021374 -117.7080496854948) bank27223 +27224 POINT(34.22461583476985 -118.77627930798619) bank27224 +27225 POINT(33.859841844868704 -118.6201933420702) bank27225 +27226 POINT(33.53136971547776 -118.61373036348424) bank27226 +27227 POINT(33.19655072574802 -117.88923277881656) bank27227 +27228 POINT(33.9757576419139 -118.94886311336741) bank27228 +27229 POINT(34.55282195583573 -117.51737685047316) bank27229 +27230 POINT(34.93139310945685 -118.58790802889426) bank27230 +27231 POINT(33.66533349074657 -118.4965785302889) bank27231 +27232 POINT(33.8855440898684 -117.37787858471309) bank27232 +27233 POINT(33.613807277458235 -117.35388263883883) bank27233 +27234 POINT(33.06830622723387 -118.87370562149096) bank27234 +27235 POINT(33.88031551726923 -117.96077931043978) bank27235 +27236 POINT(34.54329177955341 -117.51753531596754) bank27236 +27237 POINT(33.56918300807847 -117.39565344534529) bank27237 +27238 POINT(34.20359208120907 -119.23130653523346) bank27238 +27239 POINT(33.88720998320841 -118.55008795872433) bank27239 +27240 POINT(34.16173785478878 -119.16414371510443) bank27240 +27241 POINT(34.0855538112353 -118.48471874608984) bank27241 +27242 POINT(34.56128593008893 -118.05850036532996) bank27242 +27243 POINT(33.83735136280259 -118.10507331827064) bank27243 +27244 POINT(34.799838342955816 -118.73224869513139) bank27244 +27245 POINT(34.41646131613725 -118.74548558959522) bank27245 +27246 POINT(33.628682350165846 -117.69878990676102) bank27246 +27247 POINT(33.961806820205766 -118.45795425893466) bank27247 +27248 POINT(34.347731098255366 -118.93831789493433) bank27248 +27249 POINT(34.54735944887912 -119.14537574503905) bank27249 +27250 POINT(33.05255158269217 -118.25074953615974) bank27250 +27251 POINT(33.29751048670963 -117.27854307303055) bank27251 +27252 POINT(33.30431016758409 -118.48861955954469) bank27252 +27253 POINT(34.33182609049906 -118.88816859863793) bank27253 +27254 POINT(34.13898474793099 -118.31536064806237) bank27254 +27255 POINT(33.43385649590023 -119.01934443861833) bank27255 +27256 POINT(35.02300869793653 -118.1025122153849) bank27256 +27257 POINT(34.60139049121096 -119.16342667975705) bank27257 +27258 POINT(33.159528863871124 -119.11475498563027) bank27258 +27259 POINT(33.32444290923301 -118.43448106983563) bank27259 +27260 POINT(33.18268297273808 -119.15490231888391) bank27260 +27261 POINT(33.93786609246653 -118.43209915992496) bank27261 +27262 POINT(33.11064473873705 -118.76099694201034) bank27262 +27263 POINT(33.32921719969299 -118.60848941462653) bank27263 +27264 POINT(34.157089765212966 -118.90169056982182) bank27264 +27265 POINT(34.33268121627538 -118.78604961642645) bank27265 +27266 POINT(33.38809689063989 -119.20493925642201) bank27266 +27267 POINT(34.20378079207392 -118.91954298141607) bank27267 +27268 POINT(35.00969063622096 -117.62172700519285) bank27268 +27269 POINT(33.19129693309583 -118.08005377211875) bank27269 +27270 POINT(34.39412602438089 -117.92017666139648) bank27270 +27271 POINT(34.2787831442199 -118.39677697200547) bank27271 +27272 POINT(33.8283748498063 -117.81268538371559) bank27272 +27273 POINT(33.38112303260763 -119.1358975678975) bank27273 +27274 POINT(34.35509210753763 -118.41432276508655) bank27274 +27275 POINT(34.88429799625846 -118.90130833883622) bank27275 +27276 POINT(34.99439838590423 -117.44546529287673) bank27276 +27277 POINT(34.580098460044496 -117.54278774846865) bank27277 +27278 POINT(34.48478233952682 -118.57679158393654) bank27278 +27279 POINT(34.728220249693784 -118.07855836672437) bank27279 +27280 POINT(33.806696393102534 -118.88084077312242) bank27280 +27281 POINT(34.97287122482655 -118.67762417472817) bank27281 +27282 POINT(33.65679558323991 -117.74350478473775) bank27282 +27283 POINT(33.40434165640143 -118.92109384426604) bank27283 +27284 POINT(34.08043510026067 -117.2977219495556) bank27284 +27285 POINT(34.831682055708136 -118.31521705368509) bank27285 +27286 POINT(34.56919821925917 -117.25166044478648) bank27286 +27287 POINT(33.11421161093427 -118.39680066616408) bank27287 +27288 POINT(34.54369971999284 -118.673016114258) bank27288 +27289 POINT(34.845069693047115 -117.37685137472161) bank27289 +27290 POINT(33.17832842471425 -118.30973296419758) bank27290 +27291 POINT(34.371496840391565 -117.34155700762904) bank27291 +27292 POINT(33.89232531313894 -119.17711405705225) bank27292 +27293 POINT(34.942653309285404 -118.06023897931512) bank27293 +27294 POINT(34.278580550311005 -117.90814812564443) bank27294 +27295 POINT(34.707275198372756 -117.71044573488896) bank27295 +27296 POINT(35.03572166743117 -117.88664851074036) bank27296 +27297 POINT(34.31918416994661 -118.53864075534958) bank27297 +27298 POINT(33.782728537815174 -118.39858048888931) bank27298 +27299 POINT(34.21671659889356 -117.3861685921897) bank27299 +27300 POINT(33.16687851437856 -117.58982787229529) bank27300 +27301 POINT(34.72305279042537 -118.25119247627055) bank27301 +27302 POINT(34.61502213487528 -118.70282309016739) bank27302 +27303 POINT(34.496831571729565 -118.83205386630479) bank27303 +27304 POINT(33.93127050557898 -117.72664951239771) bank27304 +27305 POINT(34.75402410792746 -118.06317683963111) bank27305 +27306 POINT(34.52740827184608 -118.17756394477134) bank27306 +27307 POINT(34.02873823339562 -118.67144324721596) bank27307 +27308 POINT(34.172905212102705 -118.54305633765398) bank27308 +27309 POINT(33.705124836267174 -118.68457739155382) bank27309 +27310 POINT(33.953919673431024 -118.46327651876715) bank27310 +27311 POINT(33.448087190668126 -118.01295997253686) bank27311 +27312 POINT(34.37931747764241 -118.06861663588926) bank27312 +27313 POINT(34.88003503016098 -117.680719324515) bank27313 +27314 POINT(34.38415112252914 -118.33224664458281) bank27314 +27315 POINT(33.80378714372601 -117.56602939673934) bank27315 +27316 POINT(34.219164717916314 -118.98792557126967) bank27316 +27317 POINT(34.508933447859754 -117.93124229635546) bank27317 +27318 POINT(33.84870934495968 -118.08692221490892) bank27318 +27319 POINT(33.80726908706469 -117.54687547251169) bank27319 +27320 POINT(33.63457675696136 -118.29232083806863) bank27320 +27321 POINT(33.32142951149292 -117.7901227644091) bank27321 +27322 POINT(33.06363432222028 -118.24768112181052) bank27322 +27323 POINT(33.4127253629374 -118.9505598833272) bank27323 +27324 POINT(33.85294846518654 -117.56342009307895) bank27324 +27325 POINT(33.57152079944966 -119.15306051460388) bank27325 +27326 POINT(33.735130593510334 -118.84311099611702) bank27326 +27327 POINT(34.66219843353673 -118.96096826644717) bank27327 +27328 POINT(34.981146801654575 -118.63460697684438) bank27328 +27329 POINT(34.469534755423474 -118.80964438967318) bank27329 +27330 POINT(35.00509157011139 -119.05093281467988) bank27330 +27331 POINT(33.40305881785735 -118.8893361444713) bank27331 +27332 POINT(33.75230811290831 -118.46092020097824) bank27332 +27333 POINT(34.30928662811415 -118.17087755482316) bank27333 +27334 POINT(34.83622657965746 -118.43753999659467) bank27334 +27335 POINT(34.31132036257438 -117.81957311631271) bank27335 +27336 POINT(34.60129885168209 -117.64114130871363) bank27336 +27337 POINT(33.86233277431632 -118.76302801202166) bank27337 +27338 POINT(33.98445113649339 -118.24608544249743) bank27338 +27339 POINT(33.052696719125464 -117.70074114637744) bank27339 +27340 POINT(34.74292898346591 -119.06017778123653) bank27340 +27341 POINT(33.89226893845048 -117.72806548664421) bank27341 +27342 POINT(34.30647159474013 -118.06387821290636) bank27342 +27343 POINT(34.30765175806118 -118.0664232177) bank27343 +27344 POINT(33.801333835192004 -117.34100405802756) bank27344 +27345 POINT(33.871860812449064 -117.32709634748095) bank27345 +27346 POINT(33.56275757187891 -119.01938544096721) bank27346 +27347 POINT(34.53802335195489 -118.34214229634327) bank27347 +27348 POINT(33.50240301245589 -117.314212255539) bank27348 +27349 POINT(33.62442235997254 -118.77212254364287) bank27349 +27350 POINT(33.838217264777704 -118.72997547177536) bank27350 +27351 POINT(34.96367024624533 -118.4460151310584) bank27351 +27352 POINT(34.527483907852805 -117.91726416754521) bank27352 +27353 POINT(33.80598105904846 -118.8510691228958) bank27353 +27354 POINT(34.159229491766204 -118.79112556504391) bank27354 +27355 POINT(33.74890782727259 -117.2672451494751) bank27355 +27356 POINT(34.94431419262804 -119.048095219923) bank27356 +27357 POINT(33.915116414836504 -117.82151780990104) bank27357 +27358 POINT(33.30556622486314 -118.30153428033782) bank27358 +27359 POINT(34.312467999629774 -118.59018218633096) bank27359 +27360 POINT(33.150701638059864 -118.18030665701308) bank27360 +27361 POINT(33.69346930058501 -118.52664058158233) bank27361 +27362 POINT(34.97681640915226 -118.85497329276073) bank27362 +27363 POINT(33.11221927734009 -117.84355618688636) bank27363 +27364 POINT(34.90925569779146 -117.95812261369913) bank27364 +27365 POINT(34.12844337144407 -117.61495312331049) bank27365 +27366 POINT(34.956209793215066 -119.04837340336994) bank27366 +27367 POINT(33.79991054497614 -117.56542893320929) bank27367 +27368 POINT(34.38276590519086 -118.98538755284618) bank27368 +27369 POINT(34.00849599183737 -117.48231039508158) bank27369 +27370 POINT(34.37263760913244 -117.93990024353887) bank27370 +27371 POINT(34.16222969061578 -118.71862146785635) bank27371 +27372 POINT(33.53296100968931 -117.80391401380626) bank27372 +27373 POINT(34.7240566519139 -118.37283471405314) bank27373 +27374 POINT(33.360332103261364 -118.91953493682841) bank27374 +27375 POINT(34.49713094711358 -117.91119242196183) bank27375 +27376 POINT(35.023370403187315 -118.51575397200182) bank27376 +27377 POINT(33.63984403541551 -117.9141163842516) bank27377 +27378 POINT(34.58415641623282 -117.71998734455997) bank27378 +27379 POINT(33.47813539390367 -117.52711422916333) bank27379 +27380 POINT(33.32344031683945 -118.37606200737446) bank27380 +27381 POINT(34.09531265021862 -118.31356631616134) bank27381 +27382 POINT(34.80805259880874 -118.68978787954912) bank27382 +27383 POINT(33.485523898300016 -118.67547205069562) bank27383 +27384 POINT(34.758761654324125 -119.17726092521771) bank27384 +27385 POINT(34.620304016538974 -117.39732001838355) bank27385 +27386 POINT(34.85071116326129 -118.2787290253374) bank27386 +27387 POINT(34.33567219725663 -118.13721074514672) bank27387 +27388 POINT(34.355755794601585 -118.4515263156478) bank27388 +27389 POINT(34.09180501668568 -118.69285006343947) bank27389 +27390 POINT(34.05761730468472 -118.05146654492044) bank27390 +27391 POINT(34.96785234880012 -117.5365644395453) bank27391 +27392 POINT(33.189138048422784 -117.38453055164737) bank27392 +27393 POINT(33.6433392826739 -118.78207122978148) bank27393 +27394 POINT(34.634797520943636 -119.02179402214385) bank27394 +27395 POINT(33.669565828058055 -118.31543904655058) bank27395 +27396 POINT(34.289412389995185 -118.12227857758812) bank27396 +27397 POINT(33.328926950058325 -117.88757543042959) bank27397 +27398 POINT(33.64244182502162 -118.69386503464717) bank27398 +27399 POINT(34.20726168789962 -118.31812985292567) bank27399 +27400 POINT(33.68056344250044 -118.92241858714435) bank27400 +27401 POINT(34.15962226924167 -119.09436394021631) bank27401 +27402 POINT(33.47785962181476 -119.2064699285327) bank27402 +27403 POINT(33.10691446772552 -117.9310380257243) bank27403 +27404 POINT(33.38096247272615 -117.96896940621839) bank27404 +27405 POINT(33.80379014030338 -118.45520855530161) bank27405 +27406 POINT(33.45959527338323 -118.77619551526595) bank27406 +27407 POINT(34.78391102806528 -117.94692377711704) bank27407 +27408 POINT(34.53712172420039 -117.89085202594609) bank27408 +27409 POINT(34.68368038280829 -118.35348943467959) bank27409 +27410 POINT(33.74172339311928 -117.5396823880863) bank27410 +27411 POINT(34.061581305186834 -118.46459253441887) bank27411 +27412 POINT(33.06400321488291 -117.75795688136607) bank27412 +27413 POINT(33.67440866351073 -117.78885696107517) bank27413 +27414 POINT(34.544135885547085 -117.55573556825911) bank27414 +27415 POINT(34.120591813048875 -119.1044412883649) bank27415 +27416 POINT(33.33151922993713 -118.20025230255239) bank27416 +27417 POINT(34.55214827053094 -119.04977679465786) bank27417 +27418 POINT(34.809010849535746 -118.4077268949035) bank27418 +27419 POINT(34.49063595917708 -118.37305528632015) bank27419 +27420 POINT(34.422095184965855 -119.14933523691398) bank27420 +27421 POINT(34.83372594788836 -118.2084050538812) bank27421 +27422 POINT(34.32508300259632 -119.04291855956572) bank27422 +27423 POINT(33.47035174687268 -118.96779599327016) bank27423 +27424 POINT(33.72328796731665 -119.19890153917022) bank27424 +27425 POINT(34.350400908829336 -117.61074931865316) bank27425 +27426 POINT(33.91831089324641 -117.90281265733356) bank27426 +27427 POINT(34.751325075157126 -119.18345218249483) bank27427 +27428 POINT(33.9931990140675 -118.12553704808951) bank27428 +27429 POINT(34.087201351219875 -117.91472452711473) bank27429 +27430 POINT(33.80500516375938 -117.25921847523767) bank27430 +27431 POINT(34.5806534955999 -117.44104960844123) bank27431 +27432 POINT(34.77346458370608 -118.77827562063885) bank27432 +27433 POINT(34.79894536427464 -118.27545450623002) bank27433 +27434 POINT(34.6716082907911 -117.83899832922924) bank27434 +27435 POINT(33.20208519708914 -118.87232094340206) bank27435 +27436 POINT(34.3375414210322 -118.74670407662055) bank27436 +27437 POINT(33.084515629459325 -118.67022457300914) bank27437 +27438 POINT(34.66187927113121 -118.25617062443602) bank27438 +27439 POINT(34.732601737772164 -119.0089292093494) bank27439 +27440 POINT(34.37796335972523 -117.98518039550638) bank27440 +27441 POINT(34.59535561442597 -119.0815282007542) bank27441 +27442 POINT(33.75038013266993 -118.07464427671596) bank27442 +27443 POINT(34.36323000633379 -118.26020522687082) bank27443 +27444 POINT(33.72651161552458 -119.11058980733229) bank27444 +27445 POINT(33.710746454436745 -117.72649210792338) bank27445 +27446 POINT(33.771552839828715 -117.9340995123351) bank27446 +27447 POINT(33.38740128206394 -118.09625775842369) bank27447 +27448 POINT(34.62193088421749 -118.71991048890246) bank27448 +27449 POINT(34.291801599049315 -118.43017375004557) bank27449 +27450 POINT(34.271958854641085 -118.2856199462551) bank27450 +27451 POINT(33.63470707292045 -117.93472367693174) bank27451 +27452 POINT(33.96925491708563 -118.33003236766493) bank27452 +27453 POINT(33.93837555577646 -119.10768381050438) bank27453 +27454 POINT(33.12926779479766 -119.05198022852109) bank27454 +27455 POINT(34.95263883543862 -117.27776091515757) bank27455 +27456 POINT(34.38648050093966 -119.10009382623838) bank27456 +27457 POINT(33.85309784269666 -117.7695511033777) bank27457 +27458 POINT(35.0312526860783 -118.6529140682698) bank27458 +27459 POINT(34.334737736942486 -117.40862644501456) bank27459 +27460 POINT(33.88912242718747 -118.3839360523844) bank27460 +27461 POINT(33.789628929778324 -118.00545506407023) bank27461 +27462 POINT(34.17932483304502 -118.48410510481732) bank27462 +27463 POINT(34.974838582576986 -117.7985451581507) bank27463 +27464 POINT(34.081079982434126 -118.28115557887085) bank27464 +27465 POINT(33.2138362097911 -118.05067151627865) bank27465 +27466 POINT(34.31041983032252 -117.9220599780366) bank27466 +27467 POINT(33.97108277799071 -117.9787975509626) bank27467 +27468 POINT(35.00826393325338 -118.15304525515552) bank27468 +27469 POINT(35.03550902684261 -118.61066355374771) bank27469 +27470 POINT(33.07683465198197 -118.87246947290633) bank27470 +27471 POINT(35.04362998015029 -119.0126771313131) bank27471 +27472 POINT(34.106274281440314 -117.8875054776429) bank27472 +27473 POINT(33.09005151511738 -117.83005335558799) bank27473 +27474 POINT(33.65681030548976 -118.04858247753691) bank27474 +27475 POINT(33.463482125671845 -119.1701369746283) bank27475 +27476 POINT(33.700343466641876 -118.82532760213934) bank27476 +27477 POINT(33.83234656790698 -117.91447416262594) bank27477 +27478 POINT(33.42544130791495 -118.61887616602414) bank27478 +27479 POINT(33.46654409376098 -118.53889481435431) bank27479 +27480 POINT(33.299961129443815 -118.8116472774578) bank27480 +27481 POINT(33.69792083180945 -118.18953550590491) bank27481 +27482 POINT(33.09362533280034 -117.58940645719434) bank27482 +27483 POINT(34.57406811076368 -118.56351415191278) bank27483 +27484 POINT(33.362532001740824 -117.97405226504931) bank27484 +27485 POINT(34.4365714357577 -118.30410461009879) bank27485 +27486 POINT(35.05068382289598 -118.54094840097387) bank27486 +27487 POINT(33.063563310259475 -117.4324579486902) bank27487 +27488 POINT(34.73295047967807 -119.05819810035395) bank27488 +27489 POINT(34.93665906334191 -118.56543859306387) bank27489 +27490 POINT(35.00847800237349 -118.43202686274759) bank27490 +27491 POINT(33.14341632082114 -117.6317476081061) bank27491 +27492 POINT(34.27400056623499 -119.2435650103993) bank27492 +27493 POINT(34.691261888321186 -118.52442965870225) bank27493 +27494 POINT(33.152938779321204 -118.6158756820185) bank27494 +27495 POINT(35.02712393128438 -119.20101176951141) bank27495 +27496 POINT(34.52140294979755 -118.09331708622959) bank27496 +27497 POINT(33.973075471760055 -118.09992822019765) bank27497 +27498 POINT(34.49691250813272 -118.35466899405773) bank27498 +27499 POINT(33.252816036227564 -118.54317365996842) bank27499 +27500 POINT(34.75688223990765 -117.71079031071125) bank27500 +27501 POINT(33.7848188287896 -118.67618827912611) bank27501 +27502 POINT(34.45866764403922 -117.28635833141031) bank27502 +27503 POINT(34.05206293355349 -118.61780989772635) bank27503 +27504 POINT(34.47552978036265 -117.26155148659228) bank27504 +27505 POINT(34.87228901129454 -117.3262180736824) bank27505 +27506 POINT(34.3280043992103 -117.93911350914414) bank27506 +27507 POINT(33.99541689097319 -118.34135578620361) bank27507 +27508 POINT(33.32995087826371 -118.79418941909317) bank27508 +27509 POINT(33.26226636770586 -117.56418183406072) bank27509 +27510 POINT(34.745991493527306 -117.67418473036574) bank27510 +27511 POINT(33.639413771773334 -118.14151523200188) bank27511 +27512 POINT(34.17697044036096 -117.7904251433055) bank27512 +27513 POINT(35.03254712194814 -118.8250484457416) bank27513 +27514 POINT(34.427569112994966 -117.8391947233476) bank27514 +27515 POINT(34.79446634216033 -117.74279599107078) bank27515 +27516 POINT(33.96127276100585 -118.64565039953915) bank27516 +27517 POINT(33.11583795737955 -119.13290132152513) bank27517 +27518 POINT(33.14976572609253 -118.01169188336952) bank27518 +27519 POINT(34.655515375749374 -118.85139935591825) bank27519 +27520 POINT(33.579464044018984 -118.55862035066258) bank27520 +27521 POINT(33.79266944789841 -117.39288758392046) bank27521 +27522 POINT(33.7273707783942 -118.92137545990106) bank27522 +27523 POINT(34.28341771480398 -119.18810260102548) bank27523 +27524 POINT(34.8773715286626 -119.0774662421917) bank27524 +27525 POINT(34.542993678460746 -117.96038689317464) bank27525 +27526 POINT(34.51906452117633 -119.1309646035774) bank27526 +27527 POINT(35.050979780779564 -118.27831188121928) bank27527 +27528 POINT(34.92172782384959 -119.12122853311666) bank27528 +27529 POINT(33.2239208286751 -119.08141100281435) bank27529 +27530 POINT(34.683376479058694 -118.04920720917403) bank27530 +27531 POINT(33.9519465573961 -118.68968787418169) bank27531 +27532 POINT(34.15234102410122 -117.2539042684782) bank27532 +27533 POINT(34.18313766665556 -118.80878842312966) bank27533 +27534 POINT(34.31281324619569 -118.5705023820188) bank27534 +27535 POINT(34.33779368374595 -117.95125933814377) bank27535 +27536 POINT(34.619789504585505 -117.5264682583231) bank27536 +27537 POINT(33.35154307452268 -117.42407381536452) bank27537 +27538 POINT(33.17243278246523 -117.47568858195967) bank27538 +27539 POINT(34.40213269360971 -119.1832150658777) bank27539 +27540 POINT(34.938038546008755 -117.48291276600582) bank27540 +27541 POINT(35.01154510383035 -118.65273725248444) bank27541 +27542 POINT(33.30740287142291 -118.0624041565907) bank27542 +27543 POINT(33.16224051284201 -118.90226481660747) bank27543 +27544 POINT(33.10498320058788 -117.31855555154759) bank27544 +27545 POINT(34.49237230595932 -118.60190364644164) bank27545 +27546 POINT(34.42137173054979 -117.35448078003994) bank27546 +27547 POINT(33.52464585442557 -118.72697026806625) bank27547 +27548 POINT(33.63000490686555 -118.88628838758187) bank27548 +27549 POINT(34.03846685014251 -118.18200368405302) bank27549 +27550 POINT(34.95870855094874 -118.86098526613453) bank27550 +27551 POINT(34.31365537993624 -118.07805307068568) bank27551 +27552 POINT(34.510207899319276 -117.7067276306503) bank27552 +27553 POINT(34.3270748871472 -117.85110874533709) bank27553 +27554 POINT(34.14138151180237 -117.77289781951286) bank27554 +27555 POINT(34.0836573091801 -118.94562541600254) bank27555 +27556 POINT(33.28443813168564 -117.40734217878388) bank27556 +27557 POINT(34.03443542385077 -118.26359549282134) bank27557 +27558 POINT(33.330779289930476 -117.61001009607891) bank27558 +27559 POINT(34.73403954585363 -118.29442179168373) bank27559 +27560 POINT(34.3729414683417 -118.18802684617864) bank27560 +27561 POINT(33.087102585605216 -118.55830931459887) bank27561 +27562 POINT(34.74793198015951 -117.37731787435936) bank27562 +27563 POINT(34.03850466272652 -117.38743742719662) bank27563 +27564 POINT(34.185277880939964 -118.94973248004344) bank27564 +27565 POINT(34.63753570780056 -117.55457897132422) bank27565 +27566 POINT(33.90917061694727 -118.3667935865125) bank27566 +27567 POINT(34.95206121931806 -118.64875824622546) bank27567 +27568 POINT(34.80152003656528 -118.24508579340282) bank27568 +27569 POINT(34.14895545121974 -118.43161254172674) bank27569 +27570 POINT(34.73684137096945 -118.93141455191366) bank27570 +27571 POINT(33.63527062468125 -118.82158058033482) bank27571 +27572 POINT(34.94683110053367 -117.25626518671152) bank27572 +27573 POINT(33.623036092700744 -118.93618623978668) bank27573 +27574 POINT(33.270773663816136 -117.71799048599658) bank27574 +27575 POINT(34.048867611829884 -117.52825692907052) bank27575 +27576 POINT(34.26180970244482 -117.32153077986902) bank27576 +27577 POINT(34.418543071031415 -117.44767348291425) bank27577 +27578 POINT(34.74826254292582 -117.9706924750089) bank27578 +27579 POINT(33.89729105931366 -118.95953605938496) bank27579 +27580 POINT(33.59970271351632 -118.69696170610474) bank27580 +27581 POINT(34.140651018318934 -118.63175670650557) bank27581 +27582 POINT(33.90315022066724 -118.8657382839789) bank27582 +27583 POINT(34.73765681355068 -117.70699560682824) bank27583 +27584 POINT(34.37219996644425 -117.7405035650157) bank27584 +27585 POINT(33.376234357064405 -118.75219028688558) bank27585 +27586 POINT(33.12962130356687 -119.23853187763187) bank27586 +27587 POINT(34.112902557783706 -117.46787324803674) bank27587 +27588 POINT(33.48180604377534 -117.90589256854692) bank27588 +27589 POINT(33.482683260370855 -117.96211351589476) bank27589 +27590 POINT(33.47053886041879 -119.18243069679951) bank27590 +27591 POINT(34.540559916553256 -117.48349972727138) bank27591 +27592 POINT(34.38913042036967 -118.08986402300198) bank27592 +27593 POINT(33.278285275784974 -118.48154215983493) bank27593 +27594 POINT(33.052523295482835 -117.50713839839261) bank27594 +27595 POINT(34.95423189819495 -118.82841859811408) bank27595 +27596 POINT(34.41897611712752 -117.95501750184074) bank27596 +27597 POINT(34.19915144333731 -118.02946081975041) bank27597 +27598 POINT(34.73858073504691 -119.08070246920029) bank27598 +27599 POINT(34.91033368263583 -118.86083608998504) bank27599 +27600 POINT(33.81695695366741 -117.94165839039484) bank27600 +27601 POINT(34.3044025639605 -117.98834501434659) bank27601 +27602 POINT(34.196797935631494 -118.37773697526471) bank27602 +27603 POINT(34.14068403076153 -118.78496307780685) bank27603 +27604 POINT(33.92395105307506 -118.10778154547253) bank27604 +27605 POINT(34.75992637512648 -117.92227634278383) bank27605 +27606 POINT(33.483471936354405 -117.39214669526453) bank27606 +27607 POINT(34.201700605052785 -118.5136862218822) bank27607 +27608 POINT(34.314394329141884 -118.8637994286856) bank27608 +27609 POINT(34.77026156592949 -117.68716903836507) bank27609 +27610 POINT(34.984323051206786 -117.77689030982533) bank27610 +27611 POINT(33.855822748840296 -117.34474567795469) bank27611 +27612 POINT(34.61940498778717 -119.01665850667231) bank27612 +27613 POINT(34.736011256917735 -119.19056656569343) bank27613 +27614 POINT(33.754578394393256 -118.97759989701464) bank27614 +27615 POINT(33.64658337096116 -118.0088374573639) bank27615 +27616 POINT(33.14420813452115 -117.54464178934603) bank27616 +27617 POINT(34.71884307030782 -117.28258544713084) bank27617 +27618 POINT(34.29292603599338 -118.04730754492245) bank27618 +27619 POINT(33.45796947351938 -119.00874596883072) bank27619 +27620 POINT(33.792798071088704 -117.78412110931433) bank27620 +27621 POINT(34.19424389834234 -118.77257017386975) bank27621 +27622 POINT(33.19302834951749 -118.655850944984) bank27622 +27623 POINT(34.74489119201304 -118.76645099750412) bank27623 +27624 POINT(33.06262401801644 -118.60097324606615) bank27624 +27625 POINT(33.39357628043546 -117.27883682752874) bank27625 +27626 POINT(34.04033303576356 -118.50730522422076) bank27626 +27627 POINT(33.13463252694353 -117.3110820260854) bank27627 +27628 POINT(33.36337096678433 -117.6136564762177) bank27628 +27629 POINT(34.17379639567479 -118.15696051790506) bank27629 +27630 POINT(33.66688440677089 -117.59705032735677) bank27630 +27631 POINT(33.36966926078983 -118.54839204989855) bank27631 +27632 POINT(34.85176364451582 -118.87432794833812) bank27632 +27633 POINT(35.000172108547076 -117.30291505166694) bank27633 +27634 POINT(34.44316605180966 -118.23843836041186) bank27634 +27635 POINT(34.879305322288914 -119.2009201364731) bank27635 +27636 POINT(34.73043399689678 -118.69372785344468) bank27636 +27637 POINT(33.38820980554122 -118.18381768404686) bank27637 +27638 POINT(33.69570132522644 -117.82632501848661) bank27638 +27639 POINT(34.95411550137053 -117.52536799497392) bank27639 +27640 POINT(33.21814646892802 -118.39698400020511) bank27640 +27641 POINT(34.05802004629392 -117.92184755498376) bank27641 +27642 POINT(34.72563073439304 -118.64350146751282) bank27642 +27643 POINT(33.28695251665333 -118.04168212033986) bank27643 +27644 POINT(33.307328409580656 -118.10427861675858) bank27644 +27645 POINT(35.02251813388325 -117.94322261055912) bank27645 +27646 POINT(33.743769870760005 -118.04523114810041) bank27646 +27647 POINT(34.294674395387744 -117.78535761061741) bank27647 +27648 POINT(33.955534475138336 -117.38623047908831) bank27648 +27649 POINT(33.4502254119992 -118.68056896818202) bank27649 +27650 POINT(34.68416599667167 -118.56326431279537) bank27650 +27651 POINT(34.01728318834359 -118.06925297180315) bank27651 +27652 POINT(34.103089574537776 -117.5260102665564) bank27652 +27653 POINT(35.02998566189834 -118.0757611398953) bank27653 +27654 POINT(34.57768872296027 -118.93585952874741) bank27654 +27655 POINT(34.71186847159495 -117.80167006522234) bank27655 +27656 POINT(35.03446364233684 -118.1690001745788) bank27656 +27657 POINT(34.84904569888494 -118.46699824144311) bank27657 +27658 POINT(34.80836424238999 -119.14295418755012) bank27658 +27659 POINT(34.17787408983264 -118.21617875597475) bank27659 +27660 POINT(33.92765213849896 -119.14939414672392) bank27660 +27661 POINT(34.78471865587684 -117.74659265196992) bank27661 +27662 POINT(34.146473664595085 -117.86948993635329) bank27662 +27663 POINT(34.897142663732126 -117.60336059665633) bank27663 +27664 POINT(33.625041863397385 -117.24972073207373) bank27664 +27665 POINT(33.760863331175685 -118.2776077747834) bank27665 +27666 POINT(33.08389549737784 -117.38830845479102) bank27666 +27667 POINT(33.91284375226777 -118.14897585989354) bank27667 +27668 POINT(33.151821160917244 -117.85733665450925) bank27668 +27669 POINT(33.590043621902424 -118.98206563229) bank27669 +27670 POINT(34.05186799026845 -118.99723370345538) bank27670 +27671 POINT(34.49132126714671 -117.3775812184565) bank27671 +27672 POINT(34.36869071047248 -118.50386394635537) bank27672 +27673 POINT(33.14145788902213 -117.52495150801346) bank27673 +27674 POINT(33.598823418573495 -117.30286191973099) bank27674 +27675 POINT(34.607921401888134 -118.69772362465996) bank27675 +27676 POINT(34.019691163663275 -119.01453906369082) bank27676 +27677 POINT(34.44795698349151 -118.03371594128768) bank27677 +27678 POINT(34.57407398685967 -117.36684935602939) bank27678 +27679 POINT(34.56434277339845 -119.07185435764896) bank27679 +27680 POINT(34.074844776575546 -118.55114414553091) bank27680 +27681 POINT(34.44869010922305 -118.52830674498941) bank27681 +27682 POINT(33.64797635276073 -118.17589882767942) bank27682 +27683 POINT(34.178701316424956 -118.62928945314867) bank27683 +27684 POINT(34.93542126689912 -118.65732581984562) bank27684 +27685 POINT(33.365689587121935 -117.67410794666822) bank27685 +27686 POINT(33.99509819390399 -119.18012638754838) bank27686 +27687 POINT(34.073839687158255 -118.78368932910425) bank27687 +27688 POINT(34.78334763739918 -118.2189705741163) bank27688 +27689 POINT(33.6837759392654 -117.45914217157993) bank27689 +27690 POINT(34.16422047670266 -117.73063526099469) bank27690 +27691 POINT(33.600131405466755 -118.71562590195589) bank27691 +27692 POINT(33.50997030820969 -118.34340494765189) bank27692 +27693 POINT(33.600343971033745 -118.17838426386007) bank27693 +27694 POINT(33.74789035840851 -118.35707449042111) bank27694 +27695 POINT(34.2787512105948 -118.0296975152199) bank27695 +27696 POINT(33.47547316704481 -119.03144979794742) bank27696 +27697 POINT(33.77243323194397 -118.08002912764037) bank27697 +27698 POINT(34.652114344058816 -118.2155114114741) bank27698 +27699 POINT(34.14508768420039 -118.95031860025998) bank27699 +27700 POINT(33.754637143747274 -118.67092122733494) bank27700 +27701 POINT(33.257208582208705 -117.35130476577834) bank27701 +27702 POINT(34.2225285611001 -119.15608364455493) bank27702 +27703 POINT(34.339686601873986 -117.47563917798138) bank27703 +27704 POINT(33.9587317554923 -117.92491919236394) bank27704 +27705 POINT(34.99894812642761 -118.34838072970821) bank27705 +27706 POINT(33.22686962837655 -117.49249104540746) bank27706 +27707 POINT(34.68877664099758 -118.5173686978428) bank27707 +27708 POINT(34.15678534302708 -117.30810931646732) bank27708 +27709 POINT(34.866913953940255 -117.32663744576553) bank27709 +27710 POINT(33.45806183043406 -117.46338653586176) bank27710 +27711 POINT(34.31034045134195 -117.47323545660576) bank27711 +27712 POINT(34.6368582582285 -117.92340678884716) bank27712 +27713 POINT(33.994429060503556 -119.05287263772748) bank27713 +27714 POINT(33.583896102077006 -118.91973397706441) bank27714 +27715 POINT(34.36257855126029 -118.45963031950282) bank27715 +27716 POINT(35.05037694118098 -117.7310128614821) bank27716 +27717 POINT(33.45645709973793 -119.17122119298556) bank27717 +27718 POINT(33.219045185713014 -118.88378932447786) bank27718 +27719 POINT(33.43544402111149 -117.8146388120144) bank27719 +27720 POINT(33.864672940571175 -117.67469182524422) bank27720 +27721 POINT(33.637185310050356 -118.36823540969898) bank27721 +27722 POINT(33.96144164126774 -118.69337518417767) bank27722 +27723 POINT(34.05210837556723 -118.48357293509144) bank27723 +27724 POINT(33.973194505750826 -118.7559943618184) bank27724 +27725 POINT(34.394743812258646 -117.28539204006205) bank27725 +27726 POINT(33.6994590756312 -117.42664883027601) bank27726 +27727 POINT(33.63220990723382 -117.42350828923672) bank27727 +27728 POINT(34.4409105901656 -117.25736011726502) bank27728 +27729 POINT(34.09194039215406 -118.40990167869585) bank27729 +27730 POINT(33.57927852056361 -118.69665649901903) bank27730 +27731 POINT(34.102150582550784 -119.11128161812994) bank27731 +27732 POINT(34.32827486482613 -117.84005355501988) bank27732 +27733 POINT(33.06124088280491 -118.95264273176117) bank27733 +27734 POINT(34.03067089506476 -118.94888556569894) bank27734 +27735 POINT(34.88492461521622 -119.06294049037355) bank27735 +27736 POINT(33.68157474145833 -118.98272293008841) bank27736 +27737 POINT(34.22736881433721 -118.27530347304995) bank27737 +27738 POINT(34.04376278728884 -118.51255011770274) bank27738 +27739 POINT(33.54165741971789 -117.774509611085) bank27739 +27740 POINT(33.84305064025142 -117.84338015158394) bank27740 +27741 POINT(33.31842936244262 -119.1157292344476) bank27741 +27742 POINT(33.66954620565865 -118.0967455513038) bank27742 +27743 POINT(34.36483158650288 -118.2532984968834) bank27743 +27744 POINT(33.43957573352983 -118.91233997160246) bank27744 +27745 POINT(34.39516864377813 -118.45491607319744) bank27745 +27746 POINT(34.39082964795891 -117.37759765224423) bank27746 +27747 POINT(34.433118158850164 -119.12119330166904) bank27747 +27748 POINT(33.11487359429145 -118.0484512280422) bank27748 +27749 POINT(34.32325152646217 -119.1779344253013) bank27749 +27750 POINT(33.84678292814095 -117.86776300460632) bank27750 +27751 POINT(33.86629126538653 -117.63382783363869) bank27751 +27752 POINT(34.605737122899114 -117.2765296295348) bank27752 +27753 POINT(33.67590026568833 -118.30950833493475) bank27753 +27754 POINT(34.029078425389216 -118.11909600605931) bank27754 +27755 POINT(33.342057550886864 -118.25214631586151) bank27755 +27756 POINT(33.98895210613642 -117.67136068269733) bank27756 +27757 POINT(33.15361579922912 -119.19676409991283) bank27757 +27758 POINT(33.13499609962724 -118.84637036834974) bank27758 +27759 POINT(33.265476253817376 -118.92915090145712) bank27759 +27760 POINT(34.49864493239838 -117.67682330850634) bank27760 +27761 POINT(34.34950349430239 -119.06762400665052) bank27761 +27762 POINT(33.717663184046465 -117.97127920832524) bank27762 +27763 POINT(34.195395802042825 -117.40437438819909) bank27763 +27764 POINT(33.27673435685001 -117.25876317960898) bank27764 +27765 POINT(34.44277914873142 -118.77122291755752) bank27765 +27766 POINT(33.950956906827464 -119.15776860751667) bank27766 +27767 POINT(34.10629873615655 -117.92321071314214) bank27767 +27768 POINT(34.96247840164197 -118.38045418417053) bank27768 +27769 POINT(33.395406260545954 -117.29330968084872) bank27769 +27770 POINT(34.659824166252065 -118.01659368132414) bank27770 +27771 POINT(33.71324151712296 -117.92891353926623) bank27771 +27772 POINT(33.37528308741068 -118.56413785662797) bank27772 +27773 POINT(34.64775680479571 -119.17181039273734) bank27773 +27774 POINT(34.668474900153036 -117.65693905350453) bank27774 +27775 POINT(34.96431952818075 -117.70623455811153) bank27775 +27776 POINT(33.70469800100985 -119.19398931252232) bank27776 +27777 POINT(34.317481149235896 -117.88615269260623) bank27777 +27778 POINT(34.097999663994365 -117.69891086002984) bank27778 +27779 POINT(34.36089310043852 -118.46843536072798) bank27779 +27780 POINT(33.386898963477485 -119.22206764076026) bank27780 +27781 POINT(33.92213556813481 -118.4500163657066) bank27781 +27782 POINT(33.530499514867095 -118.37068817706036) bank27782 +27783 POINT(34.55059388796829 -118.42932328842127) bank27783 +27784 POINT(35.03256600477559 -118.60478073870809) bank27784 +27785 POINT(33.965858470400434 -117.53233023012481) bank27785 +27786 POINT(33.736206415746004 -118.0074307326194) bank27786 +27787 POINT(33.18703817010518 -118.59536679862099) bank27787 +27788 POINT(33.66445412447378 -118.50302656518332) bank27788 +27789 POINT(34.109539972277 -117.49447302622006) bank27789 +27790 POINT(34.86847780884262 -117.66041801252439) bank27790 +27791 POINT(34.07237221590168 -117.89154590311902) bank27791 +27792 POINT(33.96961648852071 -117.86180301827284) bank27792 +27793 POINT(33.3668705934083 -117.89413631392858) bank27793 +27794 POINT(33.78176188332844 -119.06098621674843) bank27794 +27795 POINT(33.38382997927862 -119.19184184205155) bank27795 +27796 POINT(34.3541118362498 -117.85491446493552) bank27796 +27797 POINT(33.52442622085547 -118.24320602883152) bank27797 +27798 POINT(33.72297463755083 -118.01276298073242) bank27798 +27799 POINT(34.01965945706323 -118.06464788219083) bank27799 +27800 POINT(34.837859133792406 -118.38579688393622) bank27800 +27801 POINT(33.24404581491119 -117.63719583191322) bank27801 +27802 POINT(33.44464442122395 -117.77197612116943) bank27802 +27803 POINT(34.11825389125657 -118.73157197353525) bank27803 +27804 POINT(34.59251395132727 -117.47676176912663) bank27804 +27805 POINT(33.64147733221526 -118.13175909223183) bank27805 +27806 POINT(33.323477839619365 -118.76102460448672) bank27806 +27807 POINT(34.95957134672017 -118.07542655523201) bank27807 +27808 POINT(34.06545980854482 -119.00903274808235) bank27808 +27809 POINT(34.38754036091592 -118.64408859507198) bank27809 +27810 POINT(34.02635865933613 -118.39995017759595) bank27810 +27811 POINT(34.97098788580805 -118.28942842977088) bank27811 +27812 POINT(34.51058567831275 -118.90643954049042) bank27812 +27813 POINT(33.728280911902324 -118.93893644309115) bank27813 +27814 POINT(34.952503305678874 -117.56735621801073) bank27814 +27815 POINT(34.375885099757916 -118.673246476428) bank27815 +27816 POINT(35.04751278585449 -118.95985306151613) bank27816 +27817 POINT(33.54540200417552 -117.98116531602518) bank27817 +27818 POINT(33.75978330553752 -118.05436106771782) bank27818 +27819 POINT(34.67844939416653 -118.42948169925604) bank27819 +27820 POINT(33.187454988629696 -117.43813915975471) bank27820 +27821 POINT(33.40247199070959 -118.60507564934096) bank27821 +27822 POINT(33.45780618658605 -117.26802506260668) bank27822 +27823 POINT(33.49834297571822 -117.42524384627248) bank27823 +27824 POINT(33.05593425914469 -118.05028728199089) bank27824 +27825 POINT(33.38320095129855 -117.55288672452349) bank27825 +27826 POINT(33.293723878461215 -118.16887189419579) bank27826 +27827 POINT(34.18687694091986 -118.66610793306778) bank27827 +27828 POINT(33.90555796318666 -118.5558628479282) bank27828 +27829 POINT(34.82034164673884 -117.61936875964554) bank27829 +27830 POINT(34.89307207860866 -117.83780088277825) bank27830 +27831 POINT(34.30925345959469 -117.81018225078545) bank27831 +27832 POINT(33.732954433150454 -118.10982934078035) bank27832 +27833 POINT(33.82922499691968 -117.5323711197862) bank27833 +27834 POINT(34.8136181727332 -118.59463001097133) bank27834 +27835 POINT(34.22607289537463 -117.80204426834003) bank27835 +27836 POINT(33.24771855766453 -118.61287409998872) bank27836 +27837 POINT(33.62358571848972 -118.94999940281032) bank27837 +27838 POINT(35.022013278885794 -119.17726960343593) bank27838 +27839 POINT(33.13782027083855 -117.82036216814659) bank27839 +27840 POINT(33.194780611652206 -117.27314733361274) bank27840 +27841 POINT(34.53237870672328 -118.46324534940524) bank27841 +27842 POINT(33.77604004564408 -117.65486810994622) bank27842 +27843 POINT(33.525499772799535 -117.31701362799099) bank27843 +27844 POINT(34.2801435468847 -118.04775480728655) bank27844 +27845 POINT(34.60337034460562 -118.32763143279766) bank27845 +27846 POINT(34.90357909838605 -118.79538026318404) bank27846 +27847 POINT(34.59103396486524 -118.10108883973537) bank27847 +27848 POINT(33.8114846886316 -117.97027568395757) bank27848 +27849 POINT(33.90868548620675 -118.40518419831247) bank27849 +27850 POINT(33.88938002318994 -117.73008646733014) bank27850 +27851 POINT(35.049811427855616 -117.33037210080435) bank27851 +27852 POINT(33.63158514052881 -117.78311283222213) bank27852 +27853 POINT(34.42065491589985 -119.12989891995716) bank27853 +27854 POINT(33.2603591936024 -119.0149816905471) bank27854 +27855 POINT(33.329300672035785 -117.2991746101697) bank27855 +27856 POINT(34.30145172586844 -118.58707538297422) bank27856 +27857 POINT(34.57331287476493 -117.4417245228908) bank27857 +27858 POINT(33.90272171019853 -118.37184858922004) bank27858 +27859 POINT(34.87882101924375 -117.52639788348807) bank27859 +27860 POINT(33.395208171695096 -118.43122418980363) bank27860 +27861 POINT(33.538600856326134 -118.09210155352943) bank27861 +27862 POINT(34.50162847861452 -118.10259937662109) bank27862 +27863 POINT(34.84104979432357 -117.52543538166043) bank27863 +27864 POINT(34.541636205318376 -119.08159158746332) bank27864 +27865 POINT(35.03191634627761 -117.63976970066717) bank27865 +27866 POINT(33.5641273540909 -117.99229180273299) bank27866 +27867 POINT(33.42855350456817 -118.50320227663684) bank27867 +27868 POINT(33.963282981043776 -119.15329375884225) bank27868 +27869 POINT(34.36842360598303 -118.29723008465548) bank27869 +27870 POINT(33.184896123747315 -118.90409735157664) bank27870 +27871 POINT(33.941653296804404 -117.24759265283926) bank27871 +27872 POINT(34.20924533523061 -119.12724403930699) bank27872 +27873 POINT(34.53971051225128 -118.38785432720061) bank27873 +27874 POINT(34.60405737022643 -118.42324298896477) bank27874 +27875 POINT(34.788575655508836 -119.23856277763676) bank27875 +27876 POINT(34.844485743168626 -117.8926421738517) bank27876 +27877 POINT(33.81680937424183 -117.82024882833237) bank27877 +27878 POINT(33.84301118029282 -117.77540494267723) bank27878 +27879 POINT(33.16512098482103 -118.36049661966616) bank27879 +27880 POINT(34.150985615982364 -117.27540299247289) bank27880 +27881 POINT(33.27449508064424 -118.55798081085605) bank27881 +27882 POINT(34.14536343547046 -117.59430839423064) bank27882 +27883 POINT(33.948870297317114 -118.65225284304879) bank27883 +27884 POINT(34.14107693177421 -118.25413483347998) bank27884 +27885 POINT(34.82796485954966 -117.62201271423321) bank27885 +27886 POINT(33.1520219618127 -118.87422588061264) bank27886 +27887 POINT(34.76268588928229 -118.18527813009011) bank27887 +27888 POINT(33.05700596631097 -117.4862688016995) bank27888 +27889 POINT(33.388260348927744 -118.90620663732841) bank27889 +27890 POINT(33.35240979183189 -118.55252409141117) bank27890 +27891 POINT(33.33263758891331 -117.72054736314671) bank27891 +27892 POINT(34.3135007756067 -117.3806601183652) bank27892 +27893 POINT(33.527962793533064 -118.90351253376393) bank27893 +27894 POINT(34.683795647904354 -118.71729105781971) bank27894 +27895 POINT(34.883436506060924 -117.96558639160376) bank27895 +27896 POINT(34.01995051621052 -118.98165855086118) bank27896 +27897 POINT(33.36569385685333 -117.71047318971384) bank27897 +27898 POINT(34.47246299356944 -117.92130820195705) bank27898 +27899 POINT(33.61007007420357 -117.57297206241985) bank27899 +27900 POINT(33.06287279699961 -117.91742520478606) bank27900 +27901 POINT(33.601470762859094 -118.8693756677724) bank27901 +27902 POINT(34.86888842489817 -118.89735700189644) bank27902 +27903 POINT(33.219514194230115 -117.40452092810538) bank27903 +27904 POINT(34.0714277237076 -118.76314952972251) bank27904 +27905 POINT(34.180836464301414 -118.30069595846966) bank27905 +27906 POINT(34.18911338819108 -118.23076872755394) bank27906 +27907 POINT(34.658718695564694 -118.1183164536554) bank27907 +27908 POINT(33.586046842559256 -118.5064021518142) bank27908 +27909 POINT(34.44590596730638 -117.30250433181594) bank27909 +27910 POINT(34.551887104417 -118.7678640091461) bank27910 +27911 POINT(34.44523204293883 -118.03117034182493) bank27911 +27912 POINT(33.63711423611644 -117.45296330316886) bank27912 +27913 POINT(34.00455724590927 -117.82122318570525) bank27913 +27914 POINT(33.54040431301534 -118.78459564803683) bank27914 +27915 POINT(33.86822435541344 -118.61470436575141) bank27915 +27916 POINT(34.89920868456707 -117.9877381529737) bank27916 +27917 POINT(33.766227194607815 -118.39773658379964) bank27917 +27918 POINT(34.93677145028692 -118.12527974762374) bank27918 +27919 POINT(34.4441784231097 -118.53702506843078) bank27919 +27920 POINT(33.56924027668904 -118.24600982882961) bank27920 +27921 POINT(34.584912655930054 -117.581215094591) bank27921 +27922 POINT(34.35327791304261 -118.3354632435368) bank27922 +27923 POINT(33.50888702953712 -119.19348540070042) bank27923 +27924 POINT(33.4782340962088 -119.23789612767847) bank27924 +27925 POINT(33.27116381550525 -119.08097483726154) bank27925 +27926 POINT(33.4311661937771 -119.09404411921854) bank27926 +27927 POINT(33.7205443594212 -117.58359584442464) bank27927 +27928 POINT(34.18648624159001 -118.47138309012918) bank27928 +27929 POINT(34.58752684580835 -117.70974486042019) bank27929 +27930 POINT(34.70744186111871 -118.61053102248492) bank27930 +27931 POINT(34.31950517095094 -118.35581748300018) bank27931 +27932 POINT(33.16550110310909 -118.58352480545763) bank27932 +27933 POINT(34.05033747121334 -118.83550734422604) bank27933 +27934 POINT(33.51817355104636 -118.97306262981265) bank27934 +27935 POINT(33.33639216186479 -118.26269600831044) bank27935 +27936 POINT(34.1818114691047 -118.12426931455603) bank27936 +27937 POINT(34.878898565522796 -117.24886882785714) bank27937 +27938 POINT(33.4340068697074 -117.3283298782838) bank27938 +27939 POINT(34.08179795841872 -117.45739982217806) bank27939 +27940 POINT(34.65916070432365 -117.72269457502338) bank27940 +27941 POINT(34.10941948977361 -118.41011340449126) bank27941 +27942 POINT(34.933790074983264 -117.2704173711466) bank27942 +27943 POINT(33.3931048451815 -117.46340565920542) bank27943 +27944 POINT(34.229994818444354 -119.12999610198943) bank27944 +27945 POINT(33.1632388308743 -117.45989252876574) bank27945 +27946 POINT(34.502393033133266 -118.18068005907028) bank27946 +27947 POINT(33.737467063286736 -117.77199267185408) bank27947 +27948 POINT(35.04856217992713 -118.7998588795481) bank27948 +27949 POINT(33.29066601656713 -117.35548182531703) bank27949 +27950 POINT(34.79199893378092 -117.48463159894825) bank27950 +27951 POINT(33.077365779383285 -117.55089763811135) bank27951 +27952 POINT(34.25531468896619 -117.56062426086342) bank27952 +27953 POINT(34.35257369820415 -118.31769159621007) bank27953 +27954 POINT(33.84643430227688 -118.88237705777347) bank27954 +27955 POINT(33.32599980715858 -118.7041551710397) bank27955 +27956 POINT(34.915480760729146 -118.57813408874067) bank27956 +27957 POINT(34.09908768876544 -118.31730224764252) bank27957 +27958 POINT(33.53514327058242 -117.6207864632541) bank27958 +27959 POINT(34.79455444076933 -119.23513621207147) bank27959 +27960 POINT(33.24320900376327 -117.6597351066231) bank27960 +27961 POINT(33.102168706303104 -118.01611903607251) bank27961 +27962 POINT(34.57756402118763 -118.67843357852243) bank27962 +27963 POINT(33.290014567496506 -117.4781377819391) bank27963 +27964 POINT(33.28561043831475 -117.27172539035365) bank27964 +27965 POINT(34.052602440776525 -117.67527964942288) bank27965 +27966 POINT(34.73638996830219 -117.74638541476727) bank27966 +27967 POINT(34.172733266215246 -117.75869964179864) bank27967 +27968 POINT(33.60373850806386 -117.65236271105651) bank27968 +27969 POINT(34.50294819152422 -117.25087695544083) bank27969 +27970 POINT(33.136721973161386 -118.17127141110417) bank27970 +27971 POINT(34.779376249629 -119.05719581021282) bank27971 +27972 POINT(34.202872325464206 -119.08105326586634) bank27972 +27973 POINT(33.964275391534755 -118.42290138344083) bank27973 +27974 POINT(33.66414306961344 -119.05252315134044) bank27974 +27975 POINT(34.90743815189018 -117.84453990181602) bank27975 +27976 POINT(34.77229127408015 -119.00111029716213) bank27976 +27977 POINT(33.59864328863328 -118.65640420527095) bank27977 +27978 POINT(33.423270465478296 -118.99029710873717) bank27978 +27979 POINT(34.712284122099334 -118.21977443281685) bank27979 +27980 POINT(34.643161530230465 -117.7300218483768) bank27980 +27981 POINT(34.748018803351336 -117.78476982786786) bank27981 +27982 POINT(34.92593609827788 -117.99832315619817) bank27982 +27983 POINT(33.86005532681238 -118.32052549503854) bank27983 +27984 POINT(33.6318424708626 -117.92108110811228) bank27984 +27985 POINT(34.76247483971941 -118.22631574776477) bank27985 +27986 POINT(34.59756362055156 -117.9135812209044) bank27986 +27987 POINT(33.741071698802166 -117.73610187669259) bank27987 +27988 POINT(33.2887461400335 -117.86770031158646) bank27988 +27989 POINT(34.54333757332805 -118.5874103140241) bank27989 +27990 POINT(34.243378340817124 -119.05301735093742) bank27990 +27991 POINT(34.0468013934989 -117.6205773504282) bank27991 +27992 POINT(34.686385152912244 -117.34371905160734) bank27992 +27993 POINT(34.05524193854078 -119.18612104863902) bank27993 +27994 POINT(33.20559592454754 -117.76234506186208) bank27994 +27995 POINT(34.57608221498015 -117.73851046709432) bank27995 +27996 POINT(33.241009059066684 -118.07117700583886) bank27996 +27997 POINT(33.0884119544508 -119.18495877050825) bank27997 +27998 POINT(34.99074904406112 -118.24390789400132) bank27998 +27999 POINT(34.679786973887005 -118.8773991571949) bank27999 +28000 POINT(34.90883845715213 -117.27418784693405) bank28000 +28001 POINT(34.28263065475021 -118.04324829565958) bank28001 +28002 POINT(33.14815321920467 -117.65831896883627) bank28002 +28003 POINT(34.244344765345105 -117.37505606310637) bank28003 +28004 POINT(34.094023688217554 -119.0879974859876) bank28004 +28005 POINT(33.913665482727474 -117.91535937050233) bank28005 +28006 POINT(33.393251563884114 -117.57714166703147) bank28006 +28007 POINT(34.29368398729967 -118.93875326489844) bank28007 +28008 POINT(34.210858989842095 -117.48803584510725) bank28008 +28009 POINT(33.31372353791941 -117.37069894092824) bank28009 +28010 POINT(34.33780242371707 -118.0865090123617) bank28010 +28011 POINT(33.83807516950492 -118.43587672516209) bank28011 +28012 POINT(34.39834773005853 -117.81292123261794) bank28012 +28013 POINT(34.25805429017666 -117.41262014849472) bank28013 +28014 POINT(33.16481671269145 -118.93646378113475) bank28014 +28015 POINT(34.146850411198486 -117.47295158325396) bank28015 +28016 POINT(33.789857829202624 -117.5108309303465) bank28016 +28017 POINT(33.324102806981585 -117.3839773693786) bank28017 +28018 POINT(33.34038051096706 -117.81926528259451) bank28018 +28019 POINT(34.37527926658711 -118.02825514683697) bank28019 +28020 POINT(33.119958144094305 -117.56841181749985) bank28020 +28021 POINT(33.92626625945417 -117.93411444736083) bank28021 +28022 POINT(33.642212304785176 -118.98896581340834) bank28022 +28023 POINT(34.50838860990356 -119.14944499996102) bank28023 +28024 POINT(34.65270314051308 -118.082601438516) bank28024 +28025 POINT(34.456929504200865 -118.52499963704108) bank28025 +28026 POINT(34.20570288988649 -117.68705380858388) bank28026 +28027 POINT(34.30729548716268 -118.16240044931924) bank28027 +28028 POINT(34.8506141096878 -118.37042230980886) bank28028 +28029 POINT(34.96074431839427 -117.28754487660315) bank28029 +28030 POINT(33.99084609857989 -117.82558304457407) bank28030 +28031 POINT(33.80918324376945 -118.9846208602423) bank28031 +28032 POINT(34.3846711925087 -119.19848043276137) bank28032 +28033 POINT(33.122970368600015 -118.98049601502346) bank28033 +28034 POINT(33.1191047252217 -117.55586220964751) bank28034 +28035 POINT(34.20630276714606 -117.574976167211) bank28035 +28036 POINT(34.34280173117719 -118.74818050036586) bank28036 +28037 POINT(33.48086659749118 -118.44480072221484) bank28037 +28038 POINT(34.414354775888974 -117.43019100717629) bank28038 +28039 POINT(34.9945069233264 -117.30759483440843) bank28039 +28040 POINT(33.96972535542993 -118.12944552501796) bank28040 +28041 POINT(34.68801656276604 -118.64751948452461) bank28041 +28042 POINT(34.44053658296768 -118.43279225799697) bank28042 +28043 POINT(34.534841311261914 -118.961417169556) bank28043 +28044 POINT(34.61592667457177 -117.52172020266994) bank28044 +28045 POINT(33.417516969461204 -118.9494001786844) bank28045 +28046 POINT(33.44853947154556 -118.15779573702741) bank28046 +28047 POINT(33.34779288244281 -117.71835695192735) bank28047 +28048 POINT(34.36498702263775 -117.68032313934647) bank28048 +28049 POINT(33.5726417111282 -118.79957761385948) bank28049 +28050 POINT(34.574511578655084 -118.53661358859249) bank28050 +28051 POINT(34.83525700106976 -118.90234816744861) bank28051 +28052 POINT(34.0220664544409 -118.07103156027028) bank28052 +28053 POINT(33.84048683189754 -117.34996252487466) bank28053 +28054 POINT(34.58243885226159 -117.65310270144319) bank28054 +28055 POINT(34.15897603329032 -118.65887970312302) bank28055 +28056 POINT(33.252652168810265 -119.1705637105515) bank28056 +28057 POINT(34.43975136081555 -117.40645391035245) bank28057 +28058 POINT(33.07762321274085 -118.99420593393951) bank28058 +28059 POINT(33.534743502258344 -118.15521951882637) bank28059 +28060 POINT(34.71949533882671 -117.70548953000159) bank28060 +28061 POINT(34.46870901879834 -118.19483378256854) bank28061 +28062 POINT(33.84939467768982 -117.6165212450188) bank28062 +28063 POINT(33.94274949614934 -118.90486935074404) bank28063 +28064 POINT(34.50848195745126 -117.44593632078521) bank28064 +28065 POINT(34.49645336934737 -117.3436535726695) bank28065 +28066 POINT(34.652514773265445 -117.59421951321306) bank28066 +28067 POINT(33.27325789215707 -118.52299130405765) bank28067 +28068 POINT(34.418080747823 -118.69747292603925) bank28068 +28069 POINT(34.58543563049357 -119.16578662490149) bank28069 +28070 POINT(34.3770480131588 -118.97309587354987) bank28070 +28071 POINT(33.52086281206 -118.41056742869483) bank28071 +28072 POINT(34.11933834842369 -118.08236237536424) bank28072 +28073 POINT(33.11371522613117 -118.40515143235149) bank28073 +28074 POINT(33.67650708658497 -119.1337662787911) bank28074 +28075 POINT(34.79683950192894 -119.20190946945135) bank28075 +28076 POINT(33.266355450976995 -117.76029272502014) bank28076 +28077 POINT(34.50998292540185 -118.55536301484041) bank28077 +28078 POINT(34.17990386519477 -117.99891069489217) bank28078 +28079 POINT(34.59547526768608 -118.21994013428443) bank28079 +28080 POINT(33.88984990480916 -118.80074912265424) bank28080 +28081 POINT(34.73175734850512 -118.4984456832658) bank28081 +28082 POINT(34.14959116991449 -119.11575519324852) bank28082 +28083 POINT(33.89997129216827 -118.90676612542688) bank28083 +28084 POINT(34.96910003625056 -118.86359606479787) bank28084 +28085 POINT(33.812919002053576 -119.17163111695874) bank28085 +28086 POINT(34.34287322391296 -118.15210784069751) bank28086 +28087 POINT(33.70512491052442 -117.28888736086677) bank28087 +28088 POINT(33.38578187923716 -117.66636737491785) bank28088 +28089 POINT(34.92408339291258 -118.13593349992631) bank28089 +28090 POINT(34.666714484107445 -119.10395730722692) bank28090 +28091 POINT(34.49900958902127 -118.59429699313944) bank28091 +28092 POINT(33.21239095964577 -118.43803506476235) bank28092 +28093 POINT(33.661776910336734 -117.81097532636916) bank28093 +28094 POINT(35.026888015172844 -117.28056987684194) bank28094 +28095 POINT(33.89329684931232 -117.9964870193897) bank28095 +28096 POINT(35.00437288920838 -117.89791850313097) bank28096 +28097 POINT(33.72842656261898 -119.02603136765876) bank28097 +28098 POINT(33.468008931997076 -117.74662833448225) bank28098 +28099 POINT(34.34295802438519 -117.68978631148256) bank28099 +28100 POINT(34.48253646528395 -117.93114144425945) bank28100 +28101 POINT(33.122648949441015 -117.81008619237795) bank28101 +28102 POINT(34.39877955314319 -118.39336141302383) bank28102 +28103 POINT(33.586024038610084 -118.72045150247834) bank28103 +28104 POINT(34.65648497189377 -117.27331141337612) bank28104 +28105 POINT(34.04996918465021 -117.92466467173857) bank28105 +28106 POINT(34.82169667081203 -118.45118963303706) bank28106 +28107 POINT(33.48834617586252 -117.84961093821506) bank28107 +28108 POINT(34.45408531659635 -118.87114777921305) bank28108 +28109 POINT(33.54985527302922 -117.29326231087869) bank28109 +28110 POINT(33.43967657725333 -118.54986162433232) bank28110 +28111 POINT(34.03451103877875 -118.79472291629126) bank28111 +28112 POINT(34.91264110845072 -118.81896721858831) bank28112 +28113 POINT(33.66858781767412 -118.75102374081553) bank28113 +28114 POINT(33.24206528593673 -117.37947941861077) bank28114 +28115 POINT(34.62068703905846 -117.90343370450186) bank28115 +28116 POINT(33.58580747461623 -117.27404984534122) bank28116 +28117 POINT(33.342210613752776 -118.5466379760829) bank28117 +28118 POINT(34.32073941056987 -118.50085763337623) bank28118 +28119 POINT(33.82949189017672 -118.34859299064534) bank28119 +28120 POINT(34.608670165435015 -118.39045085593841) bank28120 +28121 POINT(33.61526411996128 -118.47156474662707) bank28121 +28122 POINT(34.66924987301104 -118.45419790193668) bank28122 +28123 POINT(33.98045860425245 -119.07710560030199) bank28123 +28124 POINT(33.177374166309015 -118.15157300914701) bank28124 +28125 POINT(33.12101173781161 -119.19052349586096) bank28125 +28126 POINT(34.56212816931712 -119.0238164156449) bank28126 +28127 POINT(34.05133811698726 -117.3168929338954) bank28127 +28128 POINT(34.503173180705666 -118.99312192672214) bank28128 +28129 POINT(34.76546809676064 -119.00335088762085) bank28129 +28130 POINT(34.82239211797298 -117.3578284167141) bank28130 +28131 POINT(34.6299652636748 -117.49048329254693) bank28131 +28132 POINT(35.037096320677335 -119.04722305271389) bank28132 +28133 POINT(34.48315929561219 -118.61851764560055) bank28133 +28134 POINT(35.00358377230068 -119.07333898145025) bank28134 +28135 POINT(34.86464351478304 -117.83291162272087) bank28135 +28136 POINT(34.59859917362448 -117.46302563339552) bank28136 +28137 POINT(33.593329121514564 -118.69974135534088) bank28137 +28138 POINT(34.157174818373996 -117.60784388107929) bank28138 +28139 POINT(33.93609431348377 -118.74753080205656) bank28139 +28140 POINT(34.94204354611036 -118.6158995780016) bank28140 +28141 POINT(34.49201470269949 -117.61894974516308) bank28141 +28142 POINT(33.704044063958 -119.06306613867665) bank28142 +28143 POINT(34.16730207311093 -118.7330577532116) bank28143 +28144 POINT(34.55245646724305 -117.86220459942713) bank28144 +28145 POINT(34.01391373219248 -118.87586076766516) bank28145 +28146 POINT(34.15816523061758 -117.28914093839742) bank28146 +28147 POINT(34.6566852699342 -118.64914504472141) bank28147 +28148 POINT(33.0840849187948 -118.31612258942454) bank28148 +28149 POINT(34.041187960898064 -118.16198132353891) bank28149 +28150 POINT(33.97101743728325 -119.03134407156702) bank28150 +28151 POINT(34.16009719168179 -117.7267148722515) bank28151 +28152 POINT(34.56901121272847 -117.73014718340424) bank28152 +28153 POINT(33.4587809116053 -118.42520281183863) bank28153 +28154 POINT(34.731141059773165 -118.29308966712667) bank28154 +28155 POINT(34.524708753330145 -118.41941323558946) bank28155 +28156 POINT(33.85891461902233 -118.92205375567406) bank28156 +28157 POINT(33.22275950496506 -119.1155576836199) bank28157 +28158 POINT(34.319936789534495 -118.59375047202452) bank28158 +28159 POINT(34.43022777141135 -118.83136834428826) bank28159 +28160 POINT(34.16881659294223 -118.6610901474894) bank28160 +28161 POINT(34.55251746434525 -117.50084733142637) bank28161 +28162 POINT(34.247221071233774 -118.31477885987056) bank28162 +28163 POINT(33.05311475257808 -119.11956483891221) bank28163 +28164 POINT(33.786480970298946 -117.45053941936321) bank28164 +28165 POINT(33.637707493619104 -118.3790782325466) bank28165 +28166 POINT(34.5822692100835 -118.22343771679505) bank28166 +28167 POINT(33.802848838439644 -118.3791545183658) bank28167 +28168 POINT(35.00427678863546 -117.82705246043257) bank28168 +28169 POINT(33.1144608098462 -119.1659846992634) bank28169 +28170 POINT(33.96296507780174 -119.03782808138966) bank28170 +28171 POINT(33.061499417218755 -118.52169471785471) bank28171 +28172 POINT(33.938116867329725 -118.43408551119391) bank28172 +28173 POINT(34.7501131946545 -118.11390722675142) bank28173 +28174 POINT(35.01863453833685 -117.95035859445385) bank28174 +28175 POINT(33.785294867686446 -117.86396475929985) bank28175 +28176 POINT(33.31506039299181 -118.066487457862) bank28176 +28177 POINT(34.50005339497245 -117.80724969435822) bank28177 +28178 POINT(34.53167281423159 -118.62912380673102) bank28178 +28179 POINT(35.03961469881191 -117.35711577899696) bank28179 +28180 POINT(34.85206972031548 -118.03684809113179) bank28180 +28181 POINT(34.57377459087234 -117.59193739133012) bank28181 +28182 POINT(34.10190247156843 -117.47657285880774) bank28182 +28183 POINT(34.2701532264663 -118.18965265085) bank28183 +28184 POINT(33.469244990987704 -118.80182058536427) bank28184 +28185 POINT(34.6181852598263 -118.70530792409853) bank28185 +28186 POINT(34.33905853123859 -117.55318693101765) bank28186 +28187 POINT(34.34065955102829 -118.10091430638434) bank28187 +28188 POINT(34.86974721198687 -117.88797320052466) bank28188 +28189 POINT(33.338465657359954 -117.71396445593912) bank28189 +28190 POINT(33.60223226900042 -117.89905750788931) bank28190 +28191 POINT(34.83141984452075 -118.83368969993634) bank28191 +28192 POINT(33.704450285127976 -117.54854685279851) bank28192 +28193 POINT(33.37126160685215 -117.33052248819754) bank28193 +28194 POINT(34.58544725105784 -118.7643590269323) bank28194 +28195 POINT(33.59991008273531 -117.81487893894877) bank28195 +28196 POINT(34.355889463215746 -118.77694588647869) bank28196 +28197 POINT(34.84762011929909 -117.74104413303624) bank28197 +28198 POINT(35.00345502524294 -118.5470232740019) bank28198 +28199 POINT(34.58374615239866 -118.51208922984289) bank28199 +28200 POINT(33.15083649394141 -118.46413402646922) bank28200 +28201 POINT(34.69334328425879 -117.92331296936055) bank28201 +28202 POINT(33.87185303222779 -118.22754918116972) bank28202 +28203 POINT(33.340065713787354 -118.6326354763815) bank28203 +28204 POINT(33.47659577514831 -118.05835245681958) bank28204 +28205 POINT(33.22317861419884 -117.53399699471154) bank28205 +28206 POINT(33.6272440591712 -118.65877779662154) bank28206 +28207 POINT(33.59001581509274 -118.81563305674442) bank28207 +28208 POINT(34.01776178380082 -118.22884432278501) bank28208 +28209 POINT(34.421801968460365 -117.53790360511994) bank28209 +28210 POINT(34.68502328545899 -117.55559865118352) bank28210 +28211 POINT(34.3327785540361 -117.31546991842974) bank28211 +28212 POINT(34.82210022139149 -118.19911751174646) bank28212 +28213 POINT(33.25159135817992 -119.09720355443189) bank28213 +28214 POINT(34.6652831417251 -118.07685282943555) bank28214 +28215 POINT(34.70824370095939 -118.91666245792223) bank28215 +28216 POINT(33.6845801059061 -117.47526215313242) bank28216 +28217 POINT(33.561165550063215 -117.90131738879596) bank28217 +28218 POINT(33.62026593557523 -117.33114605154782) bank28218 +28219 POINT(33.98386189159229 -119.11787187532346) bank28219 +28220 POINT(35.003545713744884 -119.03777432839297) bank28220 +28221 POINT(35.01888555463651 -119.23753555307157) bank28221 +28222 POINT(33.892662002469514 -118.0332617032329) bank28222 +28223 POINT(33.61965204661786 -117.36314617815599) bank28223 +28224 POINT(33.10125009275956 -118.09355842101478) bank28224 +28225 POINT(33.25549685208503 -118.96041124527387) bank28225 +28226 POINT(34.523780437994915 -118.49076776808907) bank28226 +28227 POINT(33.73389027085673 -118.06203294755166) bank28227 +28228 POINT(34.49667643736736 -117.75815086765823) bank28228 +28229 POINT(33.148607819820626 -118.4880354526128) bank28229 +28230 POINT(33.15548272483197 -118.65722522356884) bank28230 +28231 POINT(34.766599448926584 -117.53434202644539) bank28231 +28232 POINT(33.69942393164322 -118.10905538955049) bank28232 +28233 POINT(33.97262810105364 -119.00657923971625) bank28233 +28234 POINT(33.23636849519103 -118.61679845058066) bank28234 +28235 POINT(33.84762883664047 -118.9983690407768) bank28235 +28236 POINT(33.48312258655876 -117.54640013480201) bank28236 +28237 POINT(34.99947439507265 -119.16417111466286) bank28237 +28238 POINT(34.04569477189544 -118.65632528885419) bank28238 +28239 POINT(34.28239760292556 -117.33014726364222) bank28239 +28240 POINT(33.88128421465745 -118.3648169716122) bank28240 +28241 POINT(34.4071869548879 -118.99332946940525) bank28241 +28242 POINT(34.84122240357897 -117.72794157783852) bank28242 +28243 POINT(33.70576023545088 -119.12814104389028) bank28243 +28244 POINT(34.276337053039086 -119.03069554423446) bank28244 +28245 POINT(33.181998448474786 -118.44099165028145) bank28245 +28246 POINT(33.31204699662124 -117.63286165867773) bank28246 +28247 POINT(33.54468738625203 -117.35057562657606) bank28247 +28248 POINT(34.24130936381603 -119.01928700657065) bank28248 +28249 POINT(33.81493789123726 -118.59324580839359) bank28249 +28250 POINT(33.3941175069304 -119.04184795650224) bank28250 +28251 POINT(33.2936395536318 -118.79209961063242) bank28251 +28252 POINT(33.50891129902824 -117.60756965302816) bank28252 +28253 POINT(34.76291244243146 -119.00822746083634) bank28253 +28254 POINT(34.10929985987823 -117.625659431821) bank28254 +28255 POINT(33.833156152936255 -117.34811593792345) bank28255 +28256 POINT(33.49664730523312 -117.81683458296746) bank28256 +28257 POINT(34.175169169986575 -117.77935899196834) bank28257 +28258 POINT(34.62097943622376 -119.05231284975298) bank28258 +28259 POINT(34.38815639312256 -118.66644098892459) bank28259 +28260 POINT(34.62517551262887 -119.24269658538076) bank28260 +28261 POINT(34.24483057972868 -117.97739360354458) bank28261 +28262 POINT(33.81501156515678 -118.71327873385853) bank28262 +28263 POINT(34.95403094193445 -118.99857095578031) bank28263 +28264 POINT(34.15395673495783 -119.14215694162479) bank28264 +28265 POINT(33.433760519299845 -117.49196789217308) bank28265 +28266 POINT(33.619249530568176 -118.85648137467) bank28266 +28267 POINT(34.32953010820558 -117.84962738072376) bank28267 +28268 POINT(34.27220039562848 -117.25491537307225) bank28268 +28269 POINT(34.00329969178372 -118.7835285937788) bank28269 +28270 POINT(34.64252152633399 -117.46085463891487) bank28270 +28271 POINT(33.35136433004244 -118.79863946692372) bank28271 +28272 POINT(33.227920475935086 -118.2974469279728) bank28272 +28273 POINT(34.15916179533909 -117.51891951144376) bank28273 +28274 POINT(34.27860616156643 -118.58022624487548) bank28274 +28275 POINT(33.49177085071874 -118.7289283443504) bank28275 +28276 POINT(34.82928233586959 -117.29609794378334) bank28276 +28277 POINT(34.62089241594585 -118.4463496922272) bank28277 +28278 POINT(34.40786306777057 -118.9155958062606) bank28278 +28279 POINT(34.11091141076995 -117.41611873108945) bank28279 +28280 POINT(33.15800176864844 -117.95167293399328) bank28280 +28281 POINT(34.68147902803125 -118.75657087364121) bank28281 +28282 POINT(33.94162885220272 -118.08586940052393) bank28282 +28283 POINT(34.77372385854751 -117.48723588821822) bank28283 +28284 POINT(33.53246911544279 -118.30270420306825) bank28284 +28285 POINT(33.92574287419567 -118.76551268727061) bank28285 +28286 POINT(34.16011814579441 -118.98589368771756) bank28286 +28287 POINT(33.563326403426366 -118.47091926469123) bank28287 +28288 POINT(33.78331870626169 -117.99874806104269) bank28288 +28289 POINT(34.084124181630884 -119.11943314912567) bank28289 +28290 POINT(33.28760973372924 -118.08049573554952) bank28290 +28291 POINT(33.91878006556756 -117.66467485966339) bank28291 +28292 POINT(33.4718241722675 -118.6633358271365) bank28292 +28293 POINT(33.68640053507415 -117.60237260443316) bank28293 +28294 POINT(34.571103552778816 -117.29834905194375) bank28294 +28295 POINT(33.482504156032775 -118.73113679737143) bank28295 +28296 POINT(33.803633147642905 -118.81603334132679) bank28296 +28297 POINT(34.0866056967371 -118.61886106959838) bank28297 +28298 POINT(34.85140689370492 -118.71802084287353) bank28298 +28299 POINT(34.58088194873251 -118.7843128771598) bank28299 +28300 POINT(33.10246937340548 -118.11331315088403) bank28300 +28301 POINT(34.79885178722329 -117.65383910253937) bank28301 +28302 POINT(33.5132703050903 -117.73893287796739) bank28302 +28303 POINT(34.1581652098892 -118.32334268031944) bank28303 +28304 POINT(34.50579903210682 -118.26127233120897) bank28304 +28305 POINT(33.28503002528064 -118.32316995732359) bank28305 +28306 POINT(34.4759063124506 -118.680461610468) bank28306 +28307 POINT(33.491649905195615 -117.7200239640977) bank28307 +28308 POINT(33.80227151525747 -117.37161969532826) bank28308 +28309 POINT(33.305275234090104 -118.4737270523992) bank28309 +28310 POINT(34.610609508774566 -118.95378070929358) bank28310 +28311 POINT(33.3423321139482 -118.22298173397193) bank28311 +28312 POINT(33.47355933169772 -118.63476192150156) bank28312 +28313 POINT(34.992777987399194 -118.32412617854861) bank28313 +28314 POINT(34.13673392154572 -117.31634147846636) bank28314 +28315 POINT(33.46754827969476 -119.00822036876221) bank28315 +28316 POINT(34.025933200448904 -118.4825177792723) bank28316 +28317 POINT(33.59009720613262 -117.60261602522576) bank28317 +28318 POINT(34.26735705896176 -119.079367341151) bank28318 +28319 POINT(33.49077567734582 -117.99348292876641) bank28319 +28320 POINT(33.39025426950754 -118.85294436782004) bank28320 +28321 POINT(33.270354732716655 -117.91128360884645) bank28321 +28322 POINT(34.261914101055844 -117.66694153021385) bank28322 +28323 POINT(34.67879202454668 -117.58651516285515) bank28323 +28324 POINT(34.61270180929607 -118.13057167254831) bank28324 +28325 POINT(33.45552438012836 -118.78608975257173) bank28325 +28326 POINT(33.67391674392238 -118.31418024117052) bank28326 +28327 POINT(34.02717143808458 -118.9690249563217) bank28327 +28328 POINT(33.920559468669126 -118.73075635807314) bank28328 +28329 POINT(34.66762350690579 -118.71237953413475) bank28329 +28330 POINT(34.86927289610179 -118.77761329003651) bank28330 +28331 POINT(33.85114709261864 -119.07074178791633) bank28331 +28332 POINT(33.81885806792192 -117.44411317376897) bank28332 +28333 POINT(33.70100885085298 -118.21177772109594) bank28333 +28334 POINT(33.181757327257145 -117.95986785817514) bank28334 +28335 POINT(33.05832724678242 -118.47097116347553) bank28335 +28336 POINT(33.30678913844283 -119.03790118580774) bank28336 +28337 POINT(33.37567427769082 -119.09501282140491) bank28337 +28338 POINT(34.84489434485478 -117.32168107136336) bank28338 +28339 POINT(34.07504897089658 -118.58767682445784) bank28339 +28340 POINT(34.391124115126104 -118.59166925307646) bank28340 +28341 POINT(33.31545002058344 -119.05670181106704) bank28341 +28342 POINT(33.97516375240564 -117.7410346356724) bank28342 +28343 POINT(33.39985409572176 -117.63507412209213) bank28343 +28344 POINT(33.74254524481862 -119.16276519187119) bank28344 +28345 POINT(34.842269593866064 -117.45374239880093) bank28345 +28346 POINT(33.05277730590988 -118.80009904557619) bank28346 +28347 POINT(34.28121858805387 -118.64527935263803) bank28347 +28348 POINT(34.32959358682053 -118.74042028621776) bank28348 +28349 POINT(33.562074670853 -118.15653083951347) bank28349 +28350 POINT(35.02451418636544 -118.66287351107917) bank28350 +28351 POINT(33.756030516927275 -117.61063583574412) bank28351 +28352 POINT(34.375537540964366 -117.77151388919125) bank28352 +28353 POINT(34.99317736710824 -119.20960712165393) bank28353 +28354 POINT(34.95087829013003 -118.69202488388736) bank28354 +28355 POINT(34.45544804362764 -118.1780450222794) bank28355 +28356 POINT(34.55257345294479 -117.78159911229639) bank28356 +28357 POINT(34.631292737648394 -117.40055175590076) bank28357 +28358 POINT(34.39035445026676 -117.65516288963873) bank28358 +28359 POINT(33.97363087630433 -117.56411564270824) bank28359 +28360 POINT(33.37297771563696 -118.18344774087103) bank28360 +28361 POINT(33.39581379689126 -118.58516680424954) bank28361 +28362 POINT(33.22247787497254 -119.12628652122595) bank28362 +28363 POINT(34.16492819816335 -117.24596070793399) bank28363 +28364 POINT(34.218698624329434 -117.58640060569645) bank28364 +28365 POINT(33.55198539535412 -118.36443658273225) bank28365 +28366 POINT(33.99448094057424 -117.80789235069582) bank28366 +28367 POINT(34.59291937558537 -118.29764158043221) bank28367 +28368 POINT(33.421162934272374 -117.96310966291443) bank28368 +28369 POINT(33.29774416110592 -117.35015705884656) bank28369 +28370 POINT(33.95305235683114 -118.270739993801) bank28370 +28371 POINT(34.51509373420542 -118.77878705688816) bank28371 +28372 POINT(34.48805036427668 -117.37471663867805) bank28372 +28373 POINT(33.06166259426074 -118.37706961637191) bank28373 +28374 POINT(33.893520164794666 -118.7219610341132) bank28374 +28375 POINT(34.968428447729124 -117.40558877689024) bank28375 +28376 POINT(34.19627944597871 -117.81531029325419) bank28376 +28377 POINT(33.94135253962358 -118.7402595836226) bank28377 +28378 POINT(34.769949607802374 -118.03752705234795) bank28378 +28379 POINT(34.614345061856234 -118.78837696183017) bank28379 +28380 POINT(33.69064584309492 -117.93621642173457) bank28380 +28381 POINT(34.25152465694641 -117.43659820207868) bank28381 +28382 POINT(34.405043274326744 -118.13988538774652) bank28382 +28383 POINT(34.884834911401114 -118.52837777242299) bank28383 +28384 POINT(33.71500613898023 -118.25057673019597) bank28384 +28385 POINT(33.79851648123324 -117.87217997624582) bank28385 +28386 POINT(34.44486957877361 -117.46137557430337) bank28386 +28387 POINT(33.744992477596846 -118.78787943220013) bank28387 +28388 POINT(34.23922279430144 -117.78991724695581) bank28388 +28389 POINT(34.59662481393218 -117.49750394020646) bank28389 +28390 POINT(33.922658285267126 -117.92619547428232) bank28390 +28391 POINT(34.015914061281464 -118.40014423903465) bank28391 +28392 POINT(33.190863734775775 -117.2880520870256) bank28392 +28393 POINT(33.13630040872863 -118.6162467804499) bank28393 +28394 POINT(33.344630167431 -118.57972354021888) bank28394 +28395 POINT(33.443763678515424 -117.37472427176863) bank28395 +28396 POINT(34.2958622443412 -118.67624804159651) bank28396 +28397 POINT(34.445386922571494 -119.07321994376652) bank28397 +28398 POINT(33.40651974902323 -117.61078245439138) bank28398 +28399 POINT(34.066499187141495 -117.63019870742384) bank28399 +28400 POINT(33.305643675487325 -119.20683747268559) bank28400 +28401 POINT(33.56702986598073 -118.14343804033882) bank28401 +28402 POINT(34.43937836828834 -117.8932039986642) bank28402 +28403 POINT(34.1324911240435 -117.65735016471692) bank28403 +28404 POINT(35.039525169977175 -118.63592958112821) bank28404 +28405 POINT(33.562758445383594 -118.07426804915553) bank28405 +28406 POINT(33.6643580898437 -118.58677372624666) bank28406 +28407 POINT(33.55768858720926 -118.2028978490128) bank28407 +28408 POINT(33.67934601483447 -117.47548936500192) bank28408 +28409 POINT(33.483437168280616 -117.98790587616712) bank28409 +28410 POINT(33.54045875007914 -117.98525194690126) bank28410 +28411 POINT(33.32886527712854 -119.13485809959077) bank28411 +28412 POINT(34.73363996222462 -119.09299662459011) bank28412 +28413 POINT(34.22031915808371 -118.72453255911596) bank28413 +28414 POINT(34.51279888431825 -117.2617566438617) bank28414 +28415 POINT(33.88829821984564 -118.8328393675138) bank28415 +28416 POINT(34.896599428220085 -118.69879120989025) bank28416 +28417 POINT(33.316491173151405 -118.56558323708444) bank28417 +28418 POINT(34.27664586958804 -118.3973793248329) bank28418 +28419 POINT(34.86008833836246 -118.82160814589477) bank28419 +28420 POINT(33.699397633791364 -117.31150020453812) bank28420 +28421 POINT(34.1501054157505 -117.99102902483368) bank28421 +28422 POINT(34.94172311256658 -117.79523811879496) bank28422 +28423 POINT(33.92540696828028 -117.25559131503631) bank28423 +28424 POINT(33.32874675845742 -117.49233832632586) bank28424 +28425 POINT(33.64741052005204 -118.95434912731359) bank28425 +28426 POINT(33.93865995935898 -117.45309400706222) bank28426 +28427 POINT(33.56323182933203 -117.45035404328416) bank28427 +28428 POINT(33.97220717097451 -118.02783120461753) bank28428 +28429 POINT(34.96903367776546 -118.62886016292808) bank28429 +28430 POINT(33.99056838862056 -118.39024539862642) bank28430 +28431 POINT(34.497707495932325 -117.60646691775948) bank28431 +28432 POINT(34.64028665337948 -119.08777883888027) bank28432 +28433 POINT(34.44338414999722 -117.72340708579708) bank28433 +28434 POINT(33.23983026154862 -119.21435041759489) bank28434 +28435 POINT(34.68185381006617 -117.73848551784866) bank28435 +28436 POINT(34.81212133445679 -117.9161836804096) bank28436 +28437 POINT(34.99018717524813 -118.7598905576719) bank28437 +28438 POINT(34.20463239406622 -117.7981445586704) bank28438 +28439 POINT(33.94239187776719 -118.1855547266286) bank28439 +28440 POINT(35.00119078207826 -118.87717488959413) bank28440 +28441 POINT(34.16701032112446 -118.46241172104548) bank28441 +28442 POINT(34.265235669010025 -119.22648789706506) bank28442 +28443 POINT(33.15994346297466 -118.94220354208562) bank28443 +28444 POINT(34.362457234946895 -118.32195317872846) bank28444 +28445 POINT(34.39591682684891 -118.42213137899527) bank28445 +28446 POINT(33.729901345076456 -117.64952917698305) bank28446 +28447 POINT(33.69596256489064 -118.3229065959302) bank28447 +28448 POINT(34.749084882985755 -117.93402942314478) bank28448 +28449 POINT(34.66608229664525 -118.01418499572142) bank28449 +28450 POINT(33.893622199140374 -118.75964964890731) bank28450 +28451 POINT(33.50460436415048 -117.31861691620665) bank28451 +28452 POINT(34.06602784436357 -119.02670819158088) bank28452 +28453 POINT(34.30043304511632 -117.57880655750778) bank28453 +28454 POINT(34.34380603990389 -118.27479452495525) bank28454 +28455 POINT(33.514606567690805 -119.08993009225387) bank28455 +28456 POINT(34.37033034483601 -119.1988475202986) bank28456 +28457 POINT(33.7075744048026 -117.33316867423582) bank28457 +28458 POINT(34.67079186968578 -118.25785742076769) bank28458 +28459 POINT(33.808010750752395 -118.72550932915125) bank28459 +28460 POINT(34.89167561915761 -118.59181946925946) bank28460 +28461 POINT(34.34655208653279 -117.72059016363022) bank28461 +28462 POINT(33.82286608899008 -118.15307624030567) bank28462 +28463 POINT(33.553750669571066 -117.68478005803344) bank28463 +28464 POINT(33.080675192441184 -118.88131351038355) bank28464 +28465 POINT(33.35061319497271 -117.56905616517712) bank28465 +28466 POINT(34.45154446232856 -117.49545592186529) bank28466 +28467 POINT(33.079916267218515 -118.92518732492248) bank28467 +28468 POINT(33.99618352479739 -118.7540464812605) bank28468 +28469 POINT(34.23759671158917 -118.2319839899245) bank28469 +28470 POINT(33.1274511498817 -117.27069442939113) bank28470 +28471 POINT(34.275567806613644 -117.7816662274328) bank28471 +28472 POINT(34.88616975809221 -118.90080477106268) bank28472 +28473 POINT(33.443339366576886 -117.96086077530083) bank28473 +28474 POINT(34.56033685467743 -118.19731750717544) bank28474 +28475 POINT(34.98906049182662 -118.1414992945267) bank28475 +28476 POINT(33.35426118038259 -119.06317675949884) bank28476 +28477 POINT(34.94334673952276 -117.5000011198228) bank28477 +28478 POINT(33.09746281783641 -118.8482240836563) bank28478 +28479 POINT(33.8717458804624 -118.25727704038731) bank28479 +28480 POINT(33.068528363669394 -117.71158526576595) bank28480 +28481 POINT(34.56301496960959 -117.35925823628384) bank28481 +28482 POINT(33.2115276090172 -117.54913602320684) bank28482 +28483 POINT(34.85468186013697 -119.21380551686465) bank28483 +28484 POINT(33.06105600783477 -119.05011174521715) bank28484 +28485 POINT(34.78663227021956 -117.31207006462527) bank28485 +28486 POINT(34.353868834992255 -118.67506547575495) bank28486 +28487 POINT(33.93446229139654 -117.70894742324042) bank28487 +28488 POINT(33.4055688882722 -118.46389822012543) bank28488 +28489 POINT(33.82981015407991 -117.9263261712482) bank28489 +28490 POINT(33.848687111172374 -117.7719793570638) bank28490 +28491 POINT(34.456937550077114 -117.36593425620448) bank28491 +28492 POINT(34.53311720067181 -117.48133424828097) bank28492 +28493 POINT(33.93164522094749 -117.49305743997091) bank28493 +28494 POINT(33.750403144301536 -117.73700022499567) bank28494 +28495 POINT(33.77636493361653 -118.95245121695677) bank28495 +28496 POINT(33.73105137935912 -118.14029676778546) bank28496 +28497 POINT(33.60677938613812 -117.91780688376251) bank28497 +28498 POINT(34.765066762844235 -118.99762109360948) bank28498 +28499 POINT(34.43012277021191 -119.0995248275787) bank28499 +28500 POINT(33.52900959961497 -117.8721017457382) bank28500 +28501 POINT(35.00327849543775 -118.71554135264383) bank28501 +28502 POINT(33.42663415570423 -118.1998191398211) bank28502 +28503 POINT(34.22342699384779 -117.91457026511127) bank28503 +28504 POINT(34.343889971793395 -117.79739515648619) bank28504 +28505 POINT(33.817767874883316 -117.26608232356143) bank28505 +28506 POINT(33.908126983710694 -118.86230535142406) bank28506 +28507 POINT(34.43156265738474 -117.56046515969268) bank28507 +28508 POINT(33.81002362128234 -118.55351899000196) bank28508 +28509 POINT(33.59822800454217 -118.15063252768331) bank28509 +28510 POINT(34.36961629395039 -118.96278771811949) bank28510 +28511 POINT(33.3060858131063 -119.21978666898761) bank28511 +28512 POINT(34.022352255226416 -117.55954347561244) bank28512 +28513 POINT(34.441128314855234 -117.47879421475783) bank28513 +28514 POINT(33.568648766431345 -118.45001325096366) bank28514 +28515 POINT(33.197097637708424 -118.51804951236575) bank28515 +28516 POINT(33.298222730541994 -117.5372146712502) bank28516 +28517 POINT(34.01746340433705 -118.08366645482488) bank28517 +28518 POINT(34.23560237689793 -119.03762604546196) bank28518 +28519 POINT(34.722895682867666 -117.37724535187867) bank28519 +28520 POINT(33.1146945012326 -117.75476433061559) bank28520 +28521 POINT(33.391442970254474 -117.78779319918398) bank28521 +28522 POINT(33.85171879207696 -118.96912003870283) bank28522 +28523 POINT(34.51567087124101 -118.12642878584397) bank28523 +28524 POINT(33.357538996434826 -118.1866147698473) bank28524 +28525 POINT(33.7084293416988 -118.26401742080337) bank28525 +28526 POINT(34.889107453535495 -118.47767710445217) bank28526 +28527 POINT(34.71062173053162 -119.13551883970432) bank28527 +28528 POINT(33.513416528580976 -117.36608748522596) bank28528 +28529 POINT(33.172281363458765 -117.31174435903054) bank28529 +28530 POINT(33.934140571159865 -118.01717391425925) bank28530 +28531 POINT(33.741257900313954 -119.0854987747371) bank28531 +28532 POINT(34.796010505336874 -118.7649144455018) bank28532 +28533 POINT(34.71350938602351 -118.56466755026653) bank28533 +28534 POINT(33.4367399986782 -119.01684503532385) bank28534 +28535 POINT(33.411127751125704 -118.94713464562301) bank28535 +28536 POINT(33.91168094083692 -118.38624490132734) bank28536 +28537 POINT(33.78975196066519 -118.84833394361446) bank28537 +28538 POINT(34.78962292190737 -118.4693792122924) bank28538 +28539 POINT(33.60801987860845 -117.25785821967528) bank28539 +28540 POINT(33.35304031147956 -117.84131579282077) bank28540 +28541 POINT(33.277245756983426 -117.94752179253395) bank28541 +28542 POINT(33.354906781793744 -117.56921268471126) bank28542 +28543 POINT(33.1802983794754 -117.77413043512693) bank28543 +28544 POINT(33.358012086289115 -118.34598213427564) bank28544 +28545 POINT(33.342224274173795 -117.46186117840107) bank28545 +28546 POINT(33.88352165216122 -118.94675629216763) bank28546 +28547 POINT(34.792868939730596 -118.73952432881111) bank28547 +28548 POINT(34.95688885711826 -118.00665752437973) bank28548 +28549 POINT(34.82968259011361 -118.18071189474608) bank28549 +28550 POINT(34.62193581462218 -118.82974335740737) bank28550 +28551 POINT(33.130856563005686 -117.30506438717146) bank28551 +28552 POINT(33.311121386179856 -118.743279832193) bank28552 +28553 POINT(34.19120183324607 -118.71285168946171) bank28553 +28554 POINT(33.98988518747928 -117.49362384405312) bank28554 +28555 POINT(34.08741050464681 -117.34827865994386) bank28555 +28556 POINT(34.10848220354612 -118.48710694256869) bank28556 +28557 POINT(33.90435596821052 -117.28518868219162) bank28557 +28558 POINT(33.78958669796302 -119.08180692119282) bank28558 +28559 POINT(33.35920075742468 -117.97461311668818) bank28559 +28560 POINT(34.22092042322864 -117.90765032639118) bank28560 +28561 POINT(33.442676056027 -117.85834034881468) bank28561 +28562 POINT(33.830754415217555 -118.12389045303594) bank28562 +28563 POINT(33.85337079270868 -117.89776061911792) bank28563 +28564 POINT(34.64058063630769 -118.0947673511632) bank28564 +28565 POINT(33.13296698730861 -118.1591052061827) bank28565 +28566 POINT(33.49778096086493 -118.241508736551) bank28566 +28567 POINT(33.19607010976023 -119.21514273211977) bank28567 +28568 POINT(35.02077540677066 -117.67966566466058) bank28568 +28569 POINT(34.54453031692428 -118.90819786224247) bank28569 +28570 POINT(33.669576534958004 -117.53366766016751) bank28570 +28571 POINT(34.39668858344274 -118.28834055810643) bank28571 +28572 POINT(33.835837580510386 -117.31118952491879) bank28572 +28573 POINT(34.613997580067 -118.01969986056203) bank28573 +28574 POINT(33.86440875171367 -117.79535753083053) bank28574 +28575 POINT(34.25737013112948 -118.49232817610559) bank28575 +28576 POINT(33.21048556639178 -118.00072303072247) bank28576 +28577 POINT(34.92154764009152 -118.71727897785888) bank28577 +28578 POINT(33.08847763881002 -119.22968908537506) bank28578 +28579 POINT(35.0198001940504 -118.2078795874923) bank28579 +28580 POINT(33.110184823177114 -118.95392711355014) bank28580 +28581 POINT(35.01780012882466 -118.73062131305076) bank28581 +28582 POINT(34.001026038405385 -117.83638812821667) bank28582 +28583 POINT(33.81008556236296 -118.75021503641582) bank28583 +28584 POINT(34.4315280648024 -118.23861121149756) bank28584 +28585 POINT(33.77042074541332 -117.95229003855847) bank28585 +28586 POINT(33.62662799348834 -119.01268487985949) bank28586 +28587 POINT(33.18182732557195 -117.9104647127813) bank28587 +28588 POINT(34.65680045496666 -119.10870968822817) bank28588 +28589 POINT(33.124929210388785 -118.41911031761451) bank28589 +28590 POINT(34.20209727616344 -117.45860704397174) bank28590 +28591 POINT(34.169410758950924 -118.33703725693975) bank28591 +28592 POINT(33.2172679539934 -118.21382082385242) bank28592 +28593 POINT(33.86635137030272 -118.93034708625335) bank28593 +28594 POINT(34.62856662113263 -119.10758472842413) bank28594 +28595 POINT(33.44013137551498 -117.6323138782484) bank28595 +28596 POINT(34.91410384561838 -119.08812671407114) bank28596 +28597 POINT(34.02669709222195 -118.27443228784797) bank28597 +28598 POINT(34.34561027534959 -117.30496066408857) bank28598 +28599 POINT(34.95176369829039 -118.14866462177945) bank28599 +28600 POINT(33.22176796819667 -117.64637784351004) bank28600 +28601 POINT(34.132741906964185 -117.45242732955435) bank28601 +28602 POINT(34.196350659657526 -117.46943705128875) bank28602 +28603 POINT(34.58170141914819 -118.9770576573458) bank28603 +28604 POINT(34.904109405387125 -117.45360151387173) bank28604 +28605 POINT(34.14349146453117 -117.88722757290459) bank28605 +28606 POINT(33.48116599595068 -118.86052191951495) bank28606 +28607 POINT(33.63077745955633 -118.63199609637083) bank28607 +28608 POINT(33.45874183558387 -118.93702243829478) bank28608 +28609 POINT(34.447711801403216 -119.08569419263968) bank28609 +28610 POINT(33.774471749513346 -117.30086175419154) bank28610 +28611 POINT(33.86698661191356 -117.65874231256066) bank28611 +28612 POINT(34.252221239468 -117.76127037749669) bank28612 +28613 POINT(33.387263939348756 -117.9610341413145) bank28613 +28614 POINT(33.91932533851053 -117.35599211415428) bank28614 +28615 POINT(34.83937328266886 -118.8015658574628) bank28615 +28616 POINT(33.29810550638254 -118.79288105524515) bank28616 +28617 POINT(33.973461619810706 -117.84120969801623) bank28617 +28618 POINT(34.15003504551474 -117.864651154431) bank28618 +28619 POINT(34.52728919775142 -118.81086681496095) bank28619 +28620 POINT(34.51958172952005 -118.31250271602565) bank28620 +28621 POINT(33.857217679294195 -117.67366843842653) bank28621 +28622 POINT(34.37863776512743 -117.61122761593043) bank28622 +28623 POINT(34.70507364284561 -118.80801022551577) bank28623 +28624 POINT(34.5732535283401 -118.01560908853632) bank28624 +28625 POINT(33.629714494096916 -118.83189863833657) bank28625 +28626 POINT(34.06616977041406 -118.3534555360764) bank28626 +28627 POINT(34.00798830253371 -117.44638851023154) bank28627 +28628 POINT(33.33545859884184 -118.83537832700824) bank28628 +28629 POINT(34.00648740450477 -118.03276699330274) bank28629 +28630 POINT(34.09084129015857 -118.78392859169502) bank28630 +28631 POINT(34.21343763010051 -118.6200869405237) bank28631 +28632 POINT(34.83291866366216 -118.37155593948181) bank28632 +28633 POINT(34.05330853393642 -118.80845176485285) bank28633 +28634 POINT(35.041118678478654 -117.8779859140055) bank28634 +28635 POINT(34.425844146034095 -118.13388342008717) bank28635 +28636 POINT(34.97527811426301 -119.1064033272357) bank28636 +28637 POINT(34.327867245239595 -118.2041348419266) bank28637 +28638 POINT(33.346916552593825 -117.71024139448927) bank28638 +28639 POINT(34.58192885429802 -117.7620990177186) bank28639 +28640 POINT(34.4490639523342 -118.36906256689882) bank28640 +28641 POINT(33.10607803620557 -117.46915005336132) bank28641 +28642 POINT(34.84076362894213 -117.35626071645832) bank28642 +28643 POINT(33.36194240428657 -118.90669386716155) bank28643 +28644 POINT(33.78206722326447 -117.87780442611871) bank28644 +28645 POINT(33.95836243211033 -117.62405835833667) bank28645 +28646 POINT(33.73875164272087 -118.89640138940374) bank28646 +28647 POINT(33.05874313898729 -117.35946879761795) bank28647 +28648 POINT(34.967927773570935 -117.66817560312496) bank28648 +28649 POINT(33.1963816688219 -119.16872038748387) bank28649 +28650 POINT(33.3988001016467 -118.40343173277097) bank28650 +28651 POINT(34.39320573804226 -117.51449866083941) bank28651 +28652 POINT(34.05775824676147 -117.59657550193558) bank28652 +28653 POINT(34.362880927558685 -117.26900724872908) bank28653 +28654 POINT(34.54916401078792 -118.73076408511798) bank28654 +28655 POINT(33.07006945784964 -118.22051488653977) bank28655 +28656 POINT(33.375539087329706 -117.55125707448776) bank28656 +28657 POINT(34.763453371328005 -117.44290754478939) bank28657 +28658 POINT(33.28465459735211 -117.84952772674781) bank28658 +28659 POINT(33.96792294351189 -117.6137743730392) bank28659 +28660 POINT(34.808186150892766 -118.95591365512755) bank28660 +28661 POINT(34.11718597053188 -119.10601776799129) bank28661 +28662 POINT(34.82100116584313 -118.24990535639252) bank28662 +28663 POINT(34.90111942647459 -118.09100981726341) bank28663 +28664 POINT(34.68339112100639 -118.38198745454561) bank28664 +28665 POINT(33.538674517865225 -118.79698667624494) bank28665 +28666 POINT(34.83966662613472 -118.38101938772411) bank28666 +28667 POINT(33.186792429956064 -118.0255178913814) bank28667 +28668 POINT(33.628481482837074 -117.61987071827822) bank28668 +28669 POINT(34.2724965746022 -117.8923983703714) bank28669 +28670 POINT(33.78975073405242 -118.2400218515123) bank28670 +28671 POINT(34.6853870853972 -118.76247676623109) bank28671 +28672 POINT(33.44644852174311 -118.98226434377463) bank28672 +28673 POINT(33.95741865673329 -118.06931411372474) bank28673 +28674 POINT(34.97492736865726 -117.98706162381261) bank28674 +28675 POINT(33.368231999723015 -118.64825424421149) bank28675 +28676 POINT(33.748047105564176 -117.54348093866864) bank28676 +28677 POINT(33.640593260258875 -117.96572390886281) bank28677 +28678 POINT(34.12525434072505 -118.237681836493) bank28678 +28679 POINT(33.820296150329526 -118.26544969811195) bank28679 +28680 POINT(33.53172883477345 -118.39911172992366) bank28680 +28681 POINT(33.494762826712396 -117.8172776824661) bank28681 +28682 POINT(33.78172518719602 -118.08532391164509) bank28682 +28683 POINT(35.00275475854251 -118.86969603997437) bank28683 +28684 POINT(33.86204803781464 -119.06971920227724) bank28684 +28685 POINT(34.568966538071216 -119.10350556506634) bank28685 +28686 POINT(33.38730559924439 -117.94237574407126) bank28686 +28687 POINT(34.72017022816548 -117.46288417299972) bank28687 +28688 POINT(33.491322327187824 -117.41382202229708) bank28688 +28689 POINT(33.91321855684655 -118.12783203911559) bank28689 +28690 POINT(34.40276210676699 -118.37715353292676) bank28690 +28691 POINT(33.129739831036076 -117.85495219396162) bank28691 +28692 POINT(34.800578391221045 -117.93052635258456) bank28692 +28693 POINT(34.81499587169318 -118.63111647533489) bank28693 +28694 POINT(33.466763983517744 -117.63809505118435) bank28694 +28695 POINT(33.06918042837393 -118.45138731599013) bank28695 +28696 POINT(34.782894475672876 -119.00767934479673) bank28696 +28697 POINT(34.93418175823842 -117.9462319738331) bank28697 +28698 POINT(33.834148006458946 -118.13123261244414) bank28698 +28699 POINT(34.73180143547162 -118.2233077185599) bank28699 +28700 POINT(34.43394913222533 -118.6604859800244) bank28700 +28701 POINT(33.21018511420167 -117.34402405875585) bank28701 +28702 POINT(34.68941723927142 -118.408028682315) bank28702 +28703 POINT(34.493519251871156 -117.42679313539315) bank28703 +28704 POINT(34.43151410183894 -118.04513877519751) bank28704 +28705 POINT(34.83458438761784 -118.95074062115624) bank28705 +28706 POINT(34.83025494357468 -118.50037931485997) bank28706 +28707 POINT(33.43678601145278 -117.24833121228815) bank28707 +28708 POINT(34.15050045726502 -117.25988335716016) bank28708 +28709 POINT(34.01067593766744 -117.32884079378103) bank28709 +28710 POINT(33.19575486625516 -118.10162220959351) bank28710 +28711 POINT(34.55095731429408 -118.8660626180778) bank28711 +28712 POINT(34.550344074687736 -117.59037781952088) bank28712 +28713 POINT(33.2956053076082 -117.83038942180153) bank28713 +28714 POINT(34.687586740938464 -119.15468275878675) bank28714 +28715 POINT(33.356960936383594 -118.86863705747471) bank28715 +28716 POINT(34.25869737146155 -117.78510456081665) bank28716 +28717 POINT(33.20066605176828 -117.56113435093727) bank28717 +28718 POINT(34.13221697845778 -118.47841434060443) bank28718 +28719 POINT(34.214215048554294 -117.87878079382774) bank28719 +28720 POINT(33.49470665655157 -118.58546949021056) bank28720 +28721 POINT(34.945261402515676 -118.55313739071048) bank28721 +28722 POINT(33.70729349489554 -118.7745411571932) bank28722 +28723 POINT(34.923295209005985 -117.28962088858835) bank28723 +28724 POINT(34.58415733192966 -117.95581353935032) bank28724 +28725 POINT(34.49117640439331 -117.95001508879905) bank28725 +28726 POINT(34.6012199588113 -118.5496489445973) bank28726 +28727 POINT(34.03394590174054 -118.95096873592065) bank28727 +28728 POINT(33.90074825208382 -118.54642092837659) bank28728 +28729 POINT(33.546231312770765 -118.87573764942849) bank28729 +28730 POINT(35.02789443040298 -117.74081642481887) bank28730 +28731 POINT(33.232558197121136 -117.994239922183) bank28731 +28732 POINT(34.89812238873935 -118.95209592501207) bank28732 +28733 POINT(33.10451136328043 -117.70595717642945) bank28733 +28734 POINT(34.3513216148061 -117.86790012490931) bank28734 +28735 POINT(34.04575159579073 -118.61727514280706) bank28735 +28736 POINT(33.67738448450671 -119.18825473715417) bank28736 +28737 POINT(34.25928496514124 -118.60978500223138) bank28737 +28738 POINT(34.33906960942798 -117.39202066931992) bank28738 +28739 POINT(34.40156779058893 -118.0245636544043) bank28739 +28740 POINT(33.48741062563325 -117.92131747940255) bank28740 +28741 POINT(34.794970588804354 -117.8310204682572) bank28741 +28742 POINT(33.60162984147152 -119.0715109063896) bank28742 +28743 POINT(34.086417670060875 -118.09636966369281) bank28743 +28744 POINT(33.829040865549594 -118.30760918326237) bank28744 +28745 POINT(33.341335764267484 -118.17363561977398) bank28745 +28746 POINT(33.4648239415207 -118.65588844189955) bank28746 +28747 POINT(33.06874088851463 -117.91144281837317) bank28747 +28748 POINT(33.77664315408781 -118.81622306144851) bank28748 +28749 POINT(33.38979342520149 -118.96611377439723) bank28749 +28750 POINT(34.23093800710177 -118.04994661851282) bank28750 +28751 POINT(33.67878734325964 -118.46813591650337) bank28751 +28752 POINT(34.80763408611566 -118.55470431194755) bank28752 +28753 POINT(34.01385955098419 -119.17649574964214) bank28753 +28754 POINT(34.2423589833674 -118.53597894988367) bank28754 +28755 POINT(33.49407227713902 -117.30729651003986) bank28755 +28756 POINT(33.26663385570559 -117.63720168398112) bank28756 +28757 POINT(34.838606567424236 -117.45952517747479) bank28757 +28758 POINT(33.57693868287392 -117.37058617064835) bank28758 +28759 POINT(34.054698373797414 -118.56353164304116) bank28759 +28760 POINT(34.99983021418109 -118.66133275542238) bank28760 +28761 POINT(34.689105232319854 -118.57147028148944) bank28761 +28762 POINT(33.26952315064073 -118.67750358133407) bank28762 +28763 POINT(33.21992878915964 -119.08156340129008) bank28763 +28764 POINT(34.95314781221968 -118.1215882684861) bank28764 +28765 POINT(33.54625962545592 -118.49999288084959) bank28765 +28766 POINT(34.25028716808448 -118.84948817113651) bank28766 +28767 POINT(33.78755748253347 -117.91334543399833) bank28767 +28768 POINT(34.17523137307908 -118.3350746761045) bank28768 +28769 POINT(33.75978948237839 -119.10816472257792) bank28769 +28770 POINT(33.71265692547826 -118.86972375596842) bank28770 +28771 POINT(33.14371178626486 -119.06280227177471) bank28771 +28772 POINT(34.43751006603656 -117.55252964402464) bank28772 +28773 POINT(33.8287747229368 -118.02348640157136) bank28773 +28774 POINT(34.240719780362454 -118.140296289886) bank28774 +28775 POINT(33.32417784309341 -117.58155836048365) bank28775 +28776 POINT(33.46389880806288 -117.87946631003489) bank28776 +28777 POINT(33.693092534227326 -119.01028249203902) bank28777 +28778 POINT(34.07429160254638 -118.91445287690587) bank28778 +28779 POINT(34.484981153624865 -117.44436900131352) bank28779 +28780 POINT(34.22448008744297 -117.95301280862762) bank28780 +28781 POINT(34.18273401543335 -118.63120530975124) bank28781 +28782 POINT(33.135408698714045 -118.65718053555129) bank28782 +28783 POINT(33.3038991503977 -117.98838777864441) bank28783 +28784 POINT(33.44899238410495 -118.40072763712949) bank28784 +28785 POINT(33.978741023345094 -118.73601509665176) bank28785 +28786 POINT(33.69062017389799 -118.45545568660143) bank28786 +28787 POINT(34.14430038745302 -118.372805675935) bank28787 +28788 POINT(33.48448847317507 -118.64005519920255) bank28788 +28789 POINT(33.43717099516753 -117.82846366888535) bank28789 +28790 POINT(33.58554142561251 -118.68118421068313) bank28790 +28791 POINT(34.41729505013199 -118.96699896186199) bank28791 +28792 POINT(33.58384808521303 -118.92516004219164) bank28792 +28793 POINT(33.235466061166825 -118.94218351554613) bank28793 +28794 POINT(33.92697854604801 -118.7593915109254) bank28794 +28795 POINT(34.54675752341389 -117.78076487431895) bank28795 +28796 POINT(34.61828484936465 -119.21960351831636) bank28796 +28797 POINT(34.582292977996076 -117.47656012190447) bank28797 +28798 POINT(33.2851107411618 -117.49443686753114) bank28798 +28799 POINT(34.40288917374449 -118.27138992216163) bank28799 +28800 POINT(34.948697693396426 -118.42524699048536) bank28800 +28801 POINT(33.77539817829308 -118.65629849208008) bank28801 +28802 POINT(33.05969390916828 -118.7030266952371) bank28802 +28803 POINT(33.8066973450191 -117.65044515017838) bank28803 +28804 POINT(33.21948120725593 -118.76536749761787) bank28804 +28805 POINT(33.63658354277905 -118.31715971692063) bank28805 +28806 POINT(33.46655485242079 -118.5991471847145) bank28806 +28807 POINT(33.45386653164292 -118.86000065501634) bank28807 +28808 POINT(34.48575491268829 -118.75352812218613) bank28808 +28809 POINT(34.17979730261555 -118.44117858351828) bank28809 +28810 POINT(33.67588315525012 -117.95893320506926) bank28810 +28811 POINT(34.19070274338857 -118.09507194212983) bank28811 +28812 POINT(35.01184752921551 -117.7906043370135) bank28812 +28813 POINT(34.25563166110922 -118.41382534108175) bank28813 +28814 POINT(34.86765578076913 -118.08984278121372) bank28814 +28815 POINT(33.2664269656199 -118.4098387537637) bank28815 +28816 POINT(34.3863770378676 -118.33964725859506) bank28816 +28817 POINT(33.07742832115372 -118.82113145723605) bank28817 +28818 POINT(34.51468214581142 -118.03724669997611) bank28818 +28819 POINT(33.156712952245705 -118.12757724863344) bank28819 +28820 POINT(33.14117483558875 -118.45412343088053) bank28820 +28821 POINT(33.46888569852426 -119.23537195867203) bank28821 +28822 POINT(34.78220547205399 -118.60990881270996) bank28822 +28823 POINT(33.63469750163981 -118.96078041356817) bank28823 +28824 POINT(33.0976015181153 -118.95597577209442) bank28824 +28825 POINT(34.751381328865115 -118.37434711806154) bank28825 +28826 POINT(33.97477856437906 -118.91819199580826) bank28826 +28827 POINT(34.0099845791177 -118.32708611667132) bank28827 +28828 POINT(33.27893004195584 -117.61022717740276) bank28828 +28829 POINT(33.951650157658705 -118.15166092517924) bank28829 +28830 POINT(33.684357671879205 -118.7198974511326) bank28830 +28831 POINT(34.830616304413105 -118.03669634938848) bank28831 +28832 POINT(33.36127174999722 -118.78184530306187) bank28832 +28833 POINT(33.15289510066172 -119.21217911802819) bank28833 +28834 POINT(33.23772816952362 -118.90147096730769) bank28834 +28835 POINT(34.4580112333673 -118.8756620452657) bank28835 +28836 POINT(34.08502898400994 -119.23805465545881) bank28836 +28837 POINT(34.22770272414081 -118.39510117544876) bank28837 +28838 POINT(34.87994298746071 -117.42554935004141) bank28838 +28839 POINT(34.3668463180576 -118.39877792881315) bank28839 +28840 POINT(34.67455234112289 -119.17412922280184) bank28840 +28841 POINT(33.54986984396851 -119.11789495175255) bank28841 +28842 POINT(34.97979489424659 -117.96906236531268) bank28842 +28843 POINT(33.90738907356518 -118.18430203806426) bank28843 +28844 POINT(35.033326207582434 -118.19585972537058) bank28844 +28845 POINT(34.961982259611815 -118.18799560269115) bank28845 +28846 POINT(33.629665141700116 -118.96620388089188) bank28846 +28847 POINT(34.31438819244315 -119.14994467028002) bank28847 +28848 POINT(33.46666235934931 -118.55218020414802) bank28848 +28849 POINT(34.92373830033192 -118.00488923595462) bank28849 +28850 POINT(33.2754804064505 -118.19611202956719) bank28850 +28851 POINT(34.39015955959095 -118.19873071543066) bank28851 +28852 POINT(33.22824763091026 -117.66185601843982) bank28852 +28853 POINT(34.70956294732785 -117.52106327286958) bank28853 +28854 POINT(33.572375539147075 -119.06395030491038) bank28854 +28855 POINT(33.63028557268731 -119.23850847114053) bank28855 +28856 POINT(33.664553561612216 -118.4271506627806) bank28856 +28857 POINT(33.94999790694039 -117.68631737778185) bank28857 +28858 POINT(34.85118038415758 -118.97803719998528) bank28858 +28859 POINT(33.32900349272584 -118.16581806098856) bank28859 +28860 POINT(33.554533309665985 -117.50857709921726) bank28860 +28861 POINT(34.919595349657946 -118.42636973659842) bank28861 +28862 POINT(33.80941437974776 -117.4217703818881) bank28862 +28863 POINT(34.9908756146494 -117.49527961834406) bank28863 +28864 POINT(33.667896453143356 -118.69985685478564) bank28864 +28865 POINT(34.17958712430156 -117.37894511164808) bank28865 +28866 POINT(33.20336291253505 -118.02693526541377) bank28866 +28867 POINT(34.49684911080798 -117.88674425269686) bank28867 +28868 POINT(33.67884383120493 -119.12303750657415) bank28868 +28869 POINT(34.70943501914495 -119.16202940650287) bank28869 +28870 POINT(33.50167585741379 -118.02359807951288) bank28870 +28871 POINT(33.915103908558926 -119.21110602452137) bank28871 +28872 POINT(33.16537421023318 -118.95441167912274) bank28872 +28873 POINT(34.01909713558889 -117.90625576741769) bank28873 +28874 POINT(34.817353775782806 -117.68101054540634) bank28874 +28875 POINT(33.118719659303004 -117.52121734031435) bank28875 +28876 POINT(33.08654075812187 -118.17233069203611) bank28876 +28877 POINT(34.69796819051412 -119.18040482935064) bank28877 +28878 POINT(34.887335865007614 -118.94351188111105) bank28878 +28879 POINT(33.43053200497717 -117.94793949169531) bank28879 +28880 POINT(33.930955718651816 -117.80453075593127) bank28880 +28881 POINT(33.845508124324375 -118.54744586013685) bank28881 +28882 POINT(33.09559199711295 -117.70103170606555) bank28882 +28883 POINT(34.20790204614623 -118.15459797090004) bank28883 +28884 POINT(34.994809731491415 -118.82080878163032) bank28884 +28885 POINT(34.589617227597145 -117.3043667320166) bank28885 +28886 POINT(34.43383119496048 -118.22759586070471) bank28886 +28887 POINT(33.89679536667721 -118.67434307356861) bank28887 +28888 POINT(34.86715414423016 -118.85470632089904) bank28888 +28889 POINT(33.655632654735925 -119.14014018108305) bank28889 +28890 POINT(34.1939738557617 -117.80663817207225) bank28890 +28891 POINT(34.25742669850114 -118.32389756281171) bank28891 +28892 POINT(33.59058632725429 -117.37822260829144) bank28892 +28893 POINT(34.971804288043295 -117.45279565606634) bank28893 +28894 POINT(34.383172095279235 -118.25073466274748) bank28894 +28895 POINT(34.64356006428976 -118.68761691715005) bank28895 +28896 POINT(33.23696794561608 -118.1408891586931) bank28896 +28897 POINT(34.03077823566724 -117.62573268466839) bank28897 +28898 POINT(33.97373709967284 -119.0108789368238) bank28898 +28899 POINT(33.4852933074842 -118.8279398223453) bank28899 +28900 POINT(33.54902307396458 -119.06730571938927) bank28900 +28901 POINT(34.07522031007781 -118.00793760912613) bank28901 +28902 POINT(33.16345546245441 -119.1087019693293) bank28902 +28903 POINT(34.36587009669637 -118.28780672379867) bank28903 +28904 POINT(33.69475035201782 -117.69705741787195) bank28904 +28905 POINT(34.37002947045615 -117.67554086975996) bank28905 +28906 POINT(33.322262752595215 -117.95055397838547) bank28906 +28907 POINT(34.53313476268285 -117.27061845445832) bank28907 +28908 POINT(34.64673887516861 -117.76549083077995) bank28908 +28909 POINT(33.72320952527522 -117.79244453598463) bank28909 +28910 POINT(33.23038991206373 -118.57071782715796) bank28910 +28911 POINT(33.11038463565943 -117.86643459900769) bank28911 +28912 POINT(33.10172649608694 -118.12351544820415) bank28912 +28913 POINT(34.75068798859215 -118.82657269767653) bank28913 +28914 POINT(33.2561595906068 -119.1621423896233) bank28914 +28915 POINT(33.92846445559572 -117.5850413035045) bank28915 +28916 POINT(33.10798569437591 -117.73710918858897) bank28916 +28917 POINT(34.560194399139675 -118.50231516895208) bank28917 +28918 POINT(34.15717946877256 -117.72070268780926) bank28918 +28919 POINT(34.131771571447786 -118.05484177659206) bank28919 +28920 POINT(33.702445131297786 -118.80752505870487) bank28920 +28921 POINT(34.768418517205674 -117.85135372782862) bank28921 +28922 POINT(33.263415139860854 -118.96886526220452) bank28922 +28923 POINT(33.56664142912822 -117.72471099835344) bank28923 +28924 POINT(33.4666533983214 -117.87210355772979) bank28924 +28925 POINT(33.505577717733786 -117.50689655356591) bank28925 +28926 POINT(34.30607117743563 -119.08250887521447) bank28926 +28927 POINT(34.82925389087372 -118.52953246336233) bank28927 +28928 POINT(33.76888978780997 -118.42732742294854) bank28928 +28929 POINT(34.34789840566624 -117.27590464725974) bank28929 +28930 POINT(34.808458312495304 -117.49791398803774) bank28930 +28931 POINT(34.36724337039996 -118.44224152261606) bank28931 +28932 POINT(34.88700896359133 -119.24137213553725) bank28932 +28933 POINT(34.53035323257825 -117.59587750099125) bank28933 +28934 POINT(33.15974269315003 -117.8622683695105) bank28934 +28935 POINT(34.671841377484405 -118.50542257831086) bank28935 +28936 POINT(34.415449633718964 -118.04221401157375) bank28936 +28937 POINT(33.80364414155813 -118.70363711314253) bank28937 +28938 POINT(33.645815861555704 -118.66133356969077) bank28938 +28939 POINT(34.15736733448895 -117.69184052738034) bank28939 +28940 POINT(34.14842065916455 -119.03946194324573) bank28940 +28941 POINT(34.888582778630145 -117.48793856519339) bank28941 +28942 POINT(33.57877246734612 -119.1935709206413) bank28942 +28943 POINT(34.53987153362976 -118.72815893111428) bank28943 +28944 POINT(33.776982693150856 -118.09122202389747) bank28944 +28945 POINT(34.636529269258006 -118.55856487564994) bank28945 +28946 POINT(34.045628783179936 -118.4087482124942) bank28946 +28947 POINT(33.296338890497246 -117.88557344451155) bank28947 +28948 POINT(33.916932398804335 -117.48871669162564) bank28948 +28949 POINT(33.938151734779375 -117.6415670115104) bank28949 +28950 POINT(33.86038696571581 -117.36020690816721) bank28950 +28951 POINT(33.807804257082864 -118.83679088312822) bank28951 +28952 POINT(34.32269118059516 -118.82205529058088) bank28952 +28953 POINT(34.76500976553937 -118.48282350588568) bank28953 +28954 POINT(34.485166839643355 -118.63603045315864) bank28954 +28955 POINT(34.73506327451711 -118.21557952576401) bank28955 +28956 POINT(33.62972301575025 -118.6787732952775) bank28956 +28957 POINT(33.97164596036886 -118.37406256675538) bank28957 +28958 POINT(34.18418775002539 -118.64483494379785) bank28958 +28959 POINT(33.15505049780116 -118.26859335885747) bank28959 +28960 POINT(34.2865729874036 -119.10876992216579) bank28960 +28961 POINT(33.50873704883742 -118.99293648058783) bank28961 +28962 POINT(34.54199142167724 -118.66697977759505) bank28962 +28963 POINT(33.48615920994568 -118.60307104093785) bank28963 +28964 POINT(33.16003570268355 -117.57860751856514) bank28964 +28965 POINT(34.14603040376799 -119.18617960692275) bank28965 +28966 POINT(34.7989038817354 -117.69255779698811) bank28966 +28967 POINT(33.738879163261096 -118.88725980016142) bank28967 +28968 POINT(34.33564211395892 -118.83600825081541) bank28968 +28969 POINT(33.86876237110152 -118.78826045150701) bank28969 +28970 POINT(34.96727298998858 -118.10341177068258) bank28970 +28971 POINT(34.20526967004591 -118.89476905997131) bank28971 +28972 POINT(33.45702018563083 -118.92467466772125) bank28972 +28973 POINT(34.93934588642945 -118.19908943100246) bank28973 +28974 POINT(33.124528193379085 -118.40877845359182) bank28974 +28975 POINT(34.930416681083294 -118.86062673513429) bank28975 +28976 POINT(33.59083468989391 -118.59300051046824) bank28976 +28977 POINT(33.909268487183404 -117.93367225359665) bank28977 +28978 POINT(35.04114830917194 -119.05356751026987) bank28978 +28979 POINT(33.28523979375657 -118.30279064379222) bank28979 +28980 POINT(34.922882886491735 -117.78165003317183) bank28980 +28981 POINT(33.918549611666826 -117.87408678021899) bank28981 +28982 POINT(34.575814096417055 -117.4477380211986) bank28982 +28983 POINT(34.583203203335195 -118.30231293588733) bank28983 +28984 POINT(33.34239941167266 -118.03940469908183) bank28984 +28985 POINT(33.811340871708815 -117.44402522081442) bank28985 +28986 POINT(33.44074270177886 -118.43571248406076) bank28986 +28987 POINT(34.670063287270736 -118.67496633339961) bank28987 +28988 POINT(34.14240553670139 -119.05201422907336) bank28988 +28989 POINT(34.80986484727146 -118.1948626367979) bank28989 +28990 POINT(33.911798440903546 -118.47931483593206) bank28990 +28991 POINT(33.79970559100615 -118.41775540889188) bank28991 +28992 POINT(33.94765791122836 -117.25303078024545) bank28992 +28993 POINT(34.50605624791507 -118.80648213785176) bank28993 +28994 POINT(35.00232971858889 -118.41509510084772) bank28994 +28995 POINT(34.49487160803776 -119.18085321406545) bank28995 +28996 POINT(34.82944363053908 -118.34514814061026) bank28996 +28997 POINT(34.28986645907016 -119.08451494739666) bank28997 +28998 POINT(33.13568623106005 -118.60781598503515) bank28998 +28999 POINT(34.257597172250925 -118.59872821997214) bank28999 +29000 POINT(34.74988145511561 -118.42114887210572) bank29000 +29001 POINT(34.192725724458164 -118.93016666421698) bank29001 +29002 POINT(34.870652965548615 -117.92073926245973) bank29002 +29003 POINT(33.64486610678836 -118.29681913979131) bank29003 +29004 POINT(33.379325597813796 -117.57765780697328) bank29004 +29005 POINT(33.8635896513476 -118.83965357982494) bank29005 +29006 POINT(34.46874016451905 -117.71482779450692) bank29006 +29007 POINT(34.85635924922127 -119.00487255908284) bank29007 +29008 POINT(34.53903825618213 -118.66531038872999) bank29008 +29009 POINT(35.00044310685127 -118.79222771414773) bank29009 +29010 POINT(33.36998038888012 -118.40437345324347) bank29010 +29011 POINT(34.52852109035601 -118.96046104032821) bank29011 +29012 POINT(33.80860588200898 -117.78279009889702) bank29012 +29013 POINT(34.831808146377256 -118.50456612320365) bank29013 +29014 POINT(34.43734415919583 -118.43386399410008) bank29014 +29015 POINT(33.99045783246556 -119.18728496344343) bank29015 +29016 POINT(33.341424679536296 -118.02147801440736) bank29016 +29017 POINT(33.87876953351642 -118.09978782111928) bank29017 +29018 POINT(33.23437770976165 -117.39431258815817) bank29018 +29019 POINT(34.92888442604486 -117.30520764217455) bank29019 +29020 POINT(33.83043728787444 -119.14249729208548) bank29020 +29021 POINT(33.999238155801706 -119.22081193291734) bank29021 +29022 POINT(34.401741342925554 -117.53753678457868) bank29022 +29023 POINT(33.521462510457475 -118.67783149934407) bank29023 +29024 POINT(34.994504541834665 -118.5759088297779) bank29024 +29025 POINT(34.39542980534166 -118.26026365763052) bank29025 +29026 POINT(33.88328915175824 -119.13534321866966) bank29026 +29027 POINT(33.77815825353178 -118.65005838100123) bank29027 +29028 POINT(33.63950073684274 -117.86052431317898) bank29028 +29029 POINT(33.22233063044424 -117.36432915706906) bank29029 +29030 POINT(33.35140021745611 -117.98167488719479) bank29030 +29031 POINT(34.67713705632353 -119.2044923245303) bank29031 +29032 POINT(34.076130544425475 -119.01444638664266) bank29032 +29033 POINT(33.614896756596146 -118.47226188642166) bank29033 +29034 POINT(34.8945527053738 -117.64716937430892) bank29034 +29035 POINT(34.064975752538466 -117.71076653593356) bank29035 +29036 POINT(34.867024816430934 -118.20053028273394) bank29036 +29037 POINT(34.14009672977886 -118.24674151870481) bank29037 +29038 POINT(33.14679513784068 -118.88477760569809) bank29038 +29039 POINT(33.96077556695902 -119.11084197877975) bank29039 +29040 POINT(34.034552798001265 -118.18783716034046) bank29040 +29041 POINT(34.40348758112878 -117.79933535546981) bank29041 +29042 POINT(35.03068082850764 -117.36785923953636) bank29042 +29043 POINT(33.46301315872331 -118.38688458183836) bank29043 +29044 POINT(34.64987614097625 -118.00153343348819) bank29044 +29045 POINT(34.67135328219901 -118.7302007873785) bank29045 +29046 POINT(33.704149792852036 -119.04416986363826) bank29046 +29047 POINT(33.54040637081144 -118.53526672000713) bank29047 +29048 POINT(33.48561312562641 -119.16660875592892) bank29048 +29049 POINT(34.21066800333572 -118.89389037303435) bank29049 +29050 POINT(34.78225469435503 -118.18521572142565) bank29050 +29051 POINT(34.47130058492205 -119.14961541203513) bank29051 +29052 POINT(33.38417895723012 -118.36334582305463) bank29052 +29053 POINT(34.60644147202493 -119.01156268033917) bank29053 +29054 POINT(34.96907840311809 -118.27877331369463) bank29054 +29055 POINT(33.85373133790111 -118.53536875219045) bank29055 +29056 POINT(33.0859011195613 -118.3667092762229) bank29056 +29057 POINT(33.8118676082249 -118.62080516657171) bank29057 +29058 POINT(33.49541522919936 -117.33584754710448) bank29058 +29059 POINT(33.46822446702404 -117.77484526109667) bank29059 +29060 POINT(33.85677306993823 -117.34158211639675) bank29060 +29061 POINT(34.127463051435385 -118.66448610633005) bank29061 +29062 POINT(35.012089355467594 -118.48455025878829) bank29062 +29063 POINT(33.1714161624735 -117.39555541751824) bank29063 +29064 POINT(34.62454572714789 -118.66184984962155) bank29064 +29065 POINT(33.34417115932021 -117.77572342389789) bank29065 +29066 POINT(33.902888409398514 -118.63317658604068) bank29066 +29067 POINT(33.85667263408572 -118.24343935917065) bank29067 +29068 POINT(34.49383687642502 -118.00137019198709) bank29068 +29069 POINT(33.83566742065605 -117.82311950190271) bank29069 +29070 POINT(34.17484154702985 -118.31413908456325) bank29070 +29071 POINT(33.466473569757625 -117.42621490866456) bank29071 +29072 POINT(34.16368659770625 -118.64851696459326) bank29072 +29073 POINT(33.750059921171456 -118.02605101651895) bank29073 +29074 POINT(34.38297529563323 -119.13122404573778) bank29074 +29075 POINT(35.03273107969091 -118.4820930275542) bank29075 +29076 POINT(33.45065911568444 -119.08584292577254) bank29076 +29077 POINT(34.51177087418573 -118.34276971393447) bank29077 +29078 POINT(34.07170623837205 -117.73019727272182) bank29078 +29079 POINT(34.79716933566089 -118.35555305913834) bank29079 +29080 POINT(34.642531482452505 -119.0685893300138) bank29080 +29081 POINT(34.820805586056295 -117.77891100148872) bank29081 +29082 POINT(33.242093757360564 -117.59043030793337) bank29082 +29083 POINT(33.443404557600275 -117.84547031878533) bank29083 +29084 POINT(34.34992003710143 -119.17093670908491) bank29084 +29085 POINT(34.65495323731803 -118.54928989309013) bank29085 +29086 POINT(34.478057334747774 -118.32104440542143) bank29086 +29087 POINT(34.69076656756968 -117.87369354917267) bank29087 +29088 POINT(34.65130836357782 -117.72782854164062) bank29088 +29089 POINT(33.11621722558443 -118.69798118483679) bank29089 +29090 POINT(34.14327391935956 -118.70184524808255) bank29090 +29091 POINT(34.693752559090946 -118.47575110074658) bank29091 +29092 POINT(34.1818065794989 -118.79260734297335) bank29092 +29093 POINT(33.533092380702875 -118.40585217136326) bank29093 +29094 POINT(34.557944503691324 -118.35105586368505) bank29094 +29095 POINT(33.405266213108334 -118.51962756983235) bank29095 +29096 POINT(33.23865657171843 -118.1519170642709) bank29096 +29097 POINT(33.80850903442146 -118.9797847323117) bank29097 +29098 POINT(34.614138863759536 -118.69675250796512) bank29098 +29099 POINT(34.54240212523479 -119.21404837489115) bank29099 +29100 POINT(33.88200341731775 -117.81237807240126) bank29100 +29101 POINT(35.025470882694314 -118.34256643191922) bank29101 +29102 POINT(33.77045484399234 -117.6510259595778) bank29102 +29103 POINT(33.704501429501775 -118.52651387113892) bank29103 +29104 POINT(34.46672119749165 -118.02173850914284) bank29104 +29105 POINT(33.44560546301341 -118.70524891383737) bank29105 +29106 POINT(34.631030612595794 -118.20497770691239) bank29106 +29107 POINT(34.86833569162902 -118.26434794192522) bank29107 +29108 POINT(34.36989430721179 -118.490843111356) bank29108 +29109 POINT(34.86742639729806 -118.05194533810965) bank29109 +29110 POINT(34.36013456057709 -118.43377176623015) bank29110 +29111 POINT(34.33630135245015 -117.4602661347871) bank29111 +29112 POINT(34.7837226572461 -117.4491124289047) bank29112 +29113 POINT(33.44212476905493 -118.94292658831863) bank29113 +29114 POINT(34.81479614954202 -119.03765788248782) bank29114 +29115 POINT(34.339868187103534 -118.15873604943667) bank29115 +29116 POINT(33.36062766199419 -119.14734337857874) bank29116 +29117 POINT(34.46132470621809 -118.20282648295051) bank29117 +29118 POINT(33.28846176773157 -117.87462724528484) bank29118 +29119 POINT(34.62241987717862 -118.2547322328225) bank29119 +29120 POINT(34.216596357662404 -119.13262936157088) bank29120 +29121 POINT(34.46325788661386 -117.79479199320923) bank29121 +29122 POINT(34.698177013575076 -117.99197337805104) bank29122 +29123 POINT(33.35752995971038 -118.68213333070624) bank29123 +29124 POINT(33.107265095676034 -119.0990258107919) bank29124 +29125 POINT(33.79842247480736 -117.85456436466112) bank29125 +29126 POINT(34.377096273256214 -117.65909705272904) bank29126 +29127 POINT(34.8399704196816 -117.40085224770799) bank29127 +29128 POINT(33.80967405447703 -117.30453464162777) bank29128 +29129 POINT(34.28357043282584 -118.98906441437713) bank29129 +29130 POINT(33.10233000693632 -118.18640158579795) bank29130 +29131 POINT(34.48908699283287 -117.82876565325616) bank29131 +29132 POINT(33.408735864898986 -118.70072573616686) bank29132 +29133 POINT(33.249649964424755 -118.4410983398968) bank29133 +29134 POINT(34.76659551329611 -118.01929943377263) bank29134 +29135 POINT(34.425650154775155 -118.59036402072174) bank29135 +29136 POINT(34.04814517940389 -118.82234575027559) bank29136 +29137 POINT(34.29817070055782 -117.71670272528603) bank29137 +29138 POINT(34.80356090605546 -118.58027258499871) bank29138 +29139 POINT(35.00330073987765 -117.91738145641794) bank29139 +29140 POINT(34.05633350127961 -119.23834602385892) bank29140 +29141 POINT(33.97229151078968 -119.03246970690523) bank29141 +29142 POINT(33.784420182024796 -118.69803466586013) bank29142 +29143 POINT(34.1803031346771 -118.58178774942905) bank29143 +29144 POINT(35.045580040667986 -118.6726416444218) bank29144 +29145 POINT(34.03816799993945 -118.69393793086246) bank29145 +29146 POINT(33.48932487162542 -118.10271067587671) bank29146 +29147 POINT(34.24511394840677 -119.19054895546132) bank29147 +29148 POINT(33.07583713538194 -117.95514099970893) bank29148 +29149 POINT(33.06596321833783 -118.77014915380718) bank29149 +29150 POINT(34.35070181487932 -119.02723070630388) bank29150 +29151 POINT(34.675343513090766 -117.58309906484054) bank29151 +29152 POINT(33.54984572619004 -118.60818399365431) bank29152 +29153 POINT(33.99255164411016 -118.33981033784839) bank29153 +29154 POINT(34.124505982156464 -118.00831533550642) bank29154 +29155 POINT(34.40120622374739 -118.70183693768718) bank29155 +29156 POINT(34.927844957649235 -119.1279417023307) bank29156 +29157 POINT(34.43268674595931 -119.12335279152744) bank29157 +29158 POINT(34.75767448061158 -117.63371037748686) bank29158 +29159 POINT(34.56368830252582 -117.3253738862763) bank29159 +29160 POINT(33.72094992403455 -117.55836405624247) bank29160 +29161 POINT(34.213190199736886 -118.33582249024477) bank29161 +29162 POINT(33.79236408582711 -117.77836819897266) bank29162 +29163 POINT(34.87059272607134 -118.56856071109715) bank29163 +29164 POINT(33.909801796258364 -117.35369879297825) bank29164 +29165 POINT(34.31670419710372 -118.42937324287946) bank29165 +29166 POINT(34.93219942616571 -117.368096009036) bank29166 +29167 POINT(34.39096164842348 -117.75529904023423) bank29167 +29168 POINT(34.555593284789126 -118.26357923637241) bank29168 +29169 POINT(33.27901626030846 -118.92245170624271) bank29169 +29170 POINT(33.54579648289681 -118.60878701645812) bank29170 +29171 POINT(34.27363413225351 -118.11263585678932) bank29171 +29172 POINT(33.89407384385994 -117.74438494015628) bank29172 +29173 POINT(33.50878372900391 -118.604699583778) bank29173 +29174 POINT(33.39073513651697 -117.52505352767305) bank29174 +29175 POINT(34.73070085283601 -118.09298923497775) bank29175 +29176 POINT(34.69024711425466 -117.32317286056424) bank29176 +29177 POINT(34.30890876274654 -117.5825970426819) bank29177 +29178 POINT(33.60787125740391 -119.014427331251) bank29178 +29179 POINT(33.3150905264592 -119.20464039654141) bank29179 +29180 POINT(34.39118718392313 -118.10349925505402) bank29180 +29181 POINT(34.874458221293864 -118.88351875383708) bank29181 +29182 POINT(33.31755409947946 -118.383324206482) bank29182 +29183 POINT(34.087199305071714 -118.03976080272187) bank29183 +29184 POINT(34.18796287000176 -118.57316267498605) bank29184 +29185 POINT(34.78245653383057 -117.57048899260135) bank29185 +29186 POINT(34.891106160741614 -119.10778778827756) bank29186 +29187 POINT(34.63964011864537 -118.7741675128919) bank29187 +29188 POINT(33.731548798286795 -117.49120597777188) bank29188 +29189 POINT(34.049637428319365 -118.55296358761346) bank29189 +29190 POINT(34.27659464111603 -118.17021689713852) bank29190 +29191 POINT(34.00297447008849 -117.49573928027637) bank29191 +29192 POINT(34.42760680865031 -118.36093794886334) bank29192 +29193 POINT(33.337673907098726 -118.78238360405929) bank29193 +29194 POINT(34.00985957315442 -118.97483057263206) bank29194 +29195 POINT(34.92002190523853 -117.60881685234067) bank29195 +29196 POINT(34.21845987804432 -117.60639832478321) bank29196 +29197 POINT(34.10333783812134 -118.86658515293618) bank29197 +29198 POINT(34.33783726392433 -118.28884899428358) bank29198 +29199 POINT(33.85289945783001 -117.85743454404059) bank29199 +29200 POINT(33.832745503869155 -118.33845239207983) bank29200 +29201 POINT(33.85715920045183 -117.4945574834239) bank29201 +29202 POINT(34.83117417592488 -117.99332806997188) bank29202 +29203 POINT(33.80111987547844 -119.02768599907746) bank29203 +29204 POINT(33.34934349796783 -118.77808169135692) bank29204 +29205 POINT(34.842952332693514 -119.09996893002842) bank29205 +29206 POINT(33.17452654119553 -117.97252958215829) bank29206 +29207 POINT(34.34259273851315 -119.07966652557816) bank29207 +29208 POINT(34.23724873347975 -119.0268394953887) bank29208 +29209 POINT(33.61165920048377 -117.48159723090767) bank29209 +29210 POINT(33.399856103462575 -118.40084852073274) bank29210 +29211 POINT(34.037574980062985 -117.55935455945112) bank29211 +29212 POINT(34.87017273533771 -118.665909350664) bank29212 +29213 POINT(33.24935391866449 -118.46887064773736) bank29213 +29214 POINT(34.483822599675094 -117.96636668211485) bank29214 +29215 POINT(34.57894479115515 -117.90180513135537) bank29215 +29216 POINT(34.675948490770494 -118.07638468292352) bank29216 +29217 POINT(34.09773818869686 -117.6161344613099) bank29217 +29218 POINT(34.89561559082619 -118.5596624732483) bank29218 +29219 POINT(34.19470059667772 -118.19479016972691) bank29219 +29220 POINT(33.746294848937325 -117.96329586209815) bank29220 +29221 POINT(34.430776355448046 -117.6420661570498) bank29221 +29222 POINT(33.964883367542704 -118.8745898656931) bank29222 +29223 POINT(33.75705051281425 -117.25256195408531) bank29223 +29224 POINT(34.84732853627634 -118.46842907351348) bank29224 +29225 POINT(33.83524820077639 -117.45172964855257) bank29225 +29226 POINT(33.22488834440027 -119.1845165436789) bank29226 +29227 POINT(34.52530258427204 -118.42698523536788) bank29227 +29228 POINT(34.07766103798841 -119.21782105412599) bank29228 +29229 POINT(33.51907426324008 -118.90458969625323) bank29229 +29230 POINT(33.973511522300015 -117.84352668372745) bank29230 +29231 POINT(34.67136884521617 -119.18561625770914) bank29231 +29232 POINT(34.10493423219586 -119.01664119878176) bank29232 +29233 POINT(34.11694405282199 -118.63173830922932) bank29233 +29234 POINT(33.89697855191574 -118.40259080700766) bank29234 +29235 POINT(33.748311032193 -117.32696219219267) bank29235 +29236 POINT(33.944042472605865 -117.44677517810264) bank29236 +29237 POINT(34.76382259097263 -118.92721902214163) bank29237 +29238 POINT(34.62994918789078 -118.35377565252337) bank29238 +29239 POINT(33.922864154969005 -118.22597819867012) bank29239 +29240 POINT(34.1460460900812 -119.01439698648994) bank29240 +29241 POINT(34.37702975416991 -117.56455851428235) bank29241 +29242 POINT(34.30524988180808 -118.84085517334286) bank29242 +29243 POINT(34.83963102901505 -117.78817063985507) bank29243 +29244 POINT(33.48145875147851 -119.0761341210385) bank29244 +29245 POINT(34.39695589374448 -117.26502599689698) bank29245 +29246 POINT(34.523273821470625 -118.61229617142823) bank29246 +29247 POINT(34.47079306439716 -119.03231096870007) bank29247 +29248 POINT(33.894616368100394 -118.14214574898288) bank29248 +29249 POINT(33.34053763662441 -117.72471998184353) bank29249 +29250 POINT(34.87324294773211 -117.45847762746197) bank29250 +29251 POINT(34.738623768145786 -117.96330734539974) bank29251 +29252 POINT(34.404712223882164 -118.13306542705925) bank29252 +29253 POINT(34.65543318182679 -118.25079707869872) bank29253 +29254 POINT(34.42957078978962 -118.73019524984133) bank29254 +29255 POINT(34.447310011752435 -118.08143168820664) bank29255 +29256 POINT(34.295729311803605 -119.02217200647421) bank29256 +29257 POINT(34.85470086810658 -119.02993755435071) bank29257 +29258 POINT(34.58433923877349 -119.01651914836594) bank29258 +29259 POINT(34.706005822157195 -117.75680738752995) bank29259 +29260 POINT(34.96131780521564 -117.70281851532029) bank29260 +29261 POINT(33.39058816626266 -118.99298476332395) bank29261 +29262 POINT(34.431032541459395 -117.27392087916247) bank29262 +29263 POINT(34.78305029529276 -119.15125492586094) bank29263 +29264 POINT(34.24580773043753 -118.12710046275762) bank29264 +29265 POINT(34.115776010297 -117.59449315078608) bank29265 +29266 POINT(33.618678708182316 -118.54278939804949) bank29266 +29267 POINT(33.71706995017418 -118.43851970248282) bank29267 +29268 POINT(33.10474682464469 -118.66275684265108) bank29268 +29269 POINT(33.923905905233596 -118.56631083645722) bank29269 +29270 POINT(33.90942563806886 -118.37903262205799) bank29270 +29271 POINT(33.91923062839954 -118.84819314633107) bank29271 +29272 POINT(33.508041925291096 -117.95819878473718) bank29272 +29273 POINT(33.65773997168861 -118.41468827688821) bank29273 +29274 POINT(34.866858089752036 -117.51131195186328) bank29274 +29275 POINT(34.31851845320499 -117.40295007004256) bank29275 +29276 POINT(33.17079758246804 -118.58689804245179) bank29276 +29277 POINT(33.46569039098127 -117.77943741656827) bank29277 +29278 POINT(35.015026171004635 -117.3694246628262) bank29278 +29279 POINT(34.08612940888399 -117.55392673095922) bank29279 +29280 POINT(34.11848462224268 -117.79116833114448) bank29280 +29281 POINT(33.99725157944984 -118.28912535677745) bank29281 +29282 POINT(34.54196212019292 -118.5449791156326) bank29282 +29283 POINT(33.259438886463045 -118.04296093867248) bank29283 +29284 POINT(34.97209372115692 -117.82966612610169) bank29284 +29285 POINT(34.467454938603346 -117.43019339014533) bank29285 +29286 POINT(34.26309742422485 -117.4609736026997) bank29286 +29287 POINT(33.775130591912564 -118.83420058414329) bank29287 +29288 POINT(35.02257586479563 -118.42930870176224) bank29288 +29289 POINT(34.62193388000553 -118.34076590237909) bank29289 +29290 POINT(34.67435511982907 -119.01801605326969) bank29290 +29291 POINT(34.43994789553705 -117.36116486556277) bank29291 +29292 POINT(34.787855098930216 -118.50038265702776) bank29292 +29293 POINT(34.56044150038705 -118.14078564614351) bank29293 +29294 POINT(34.44098155991557 -118.51559371244454) bank29294 +29295 POINT(34.47358902213846 -117.59214212695898) bank29295 +29296 POINT(33.13078573032791 -117.77052752807641) bank29296 +29297 POINT(34.595805323182624 -117.77585540287379) bank29297 +29298 POINT(34.893557800569006 -117.51231426817372) bank29298 +29299 POINT(34.76596413786306 -117.9192311798177) bank29299 +29300 POINT(33.17952197523014 -117.93815898258143) bank29300 +29301 POINT(34.282273033139 -118.57766368014705) bank29301 +29302 POINT(34.803427964433844 -117.58278924402158) bank29302 +29303 POINT(33.16209203923529 -119.16133739246226) bank29303 +29304 POINT(33.438937424625216 -117.42013910008903) bank29304 +29305 POINT(34.67348604689326 -117.88007847154199) bank29305 +29306 POINT(34.961433734392585 -118.4202018134759) bank29306 +29307 POINT(33.6171379877686 -118.86392715812859) bank29307 +29308 POINT(33.70823335183219 -118.7081207402937) bank29308 +29309 POINT(34.35521229854926 -118.61979810187476) bank29309 +29310 POINT(33.76138570095392 -117.71135296451601) bank29310 +29311 POINT(34.87812770692142 -118.8749552314177) bank29311 +29312 POINT(34.566846900709535 -119.22031646601005) bank29312 +29313 POINT(33.919097318076595 -119.07870717185052) bank29313 +29314 POINT(34.900465207554404 -117.96003589701259) bank29314 +29315 POINT(33.603305026667826 -117.67339094535777) bank29315 +29316 POINT(34.41621654455707 -117.3945848079456) bank29316 +29317 POINT(34.658549660128585 -118.35623738684149) bank29317 +29318 POINT(33.901341028291185 -118.2013752094616) bank29318 +29319 POINT(33.12139743575858 -117.42374036473089) bank29319 +29320 POINT(33.38340626959787 -117.83469452506334) bank29320 +29321 POINT(34.48884025649924 -119.04219079885245) bank29321 +29322 POINT(33.13885469808672 -117.40782649007657) bank29322 +29323 POINT(34.82159587517127 -117.78722862791875) bank29323 +29324 POINT(33.90940346856355 -118.80265967197815) bank29324 +29325 POINT(34.517275692625056 -117.9716128234293) bank29325 +29326 POINT(33.057607738511834 -117.97854575755885) bank29326 +29327 POINT(33.971396453199766 -118.65818965088131) bank29327 +29328 POINT(33.33804723922253 -118.04598253033606) bank29328 +29329 POINT(33.65103071888228 -118.27204514865005) bank29329 +29330 POINT(33.84983121067958 -117.54288147085663) bank29330 +29331 POINT(33.11631796056198 -117.37354257689572) bank29331 +29332 POINT(34.24240753125424 -119.04385881684124) bank29332 +29333 POINT(33.73492852734337 -117.57716553268757) bank29333 +29334 POINT(34.00985011309801 -119.06465345784784) bank29334 +29335 POINT(33.099598041079645 -117.63734060949923) bank29335 +29336 POINT(34.060284232233414 -117.63329256797498) bank29336 +29337 POINT(33.923763458340304 -118.91523230353158) bank29337 +29338 POINT(33.15392075790792 -118.90346050900853) bank29338 +29339 POINT(33.20723321252419 -118.00908393063902) bank29339 +29340 POINT(33.207538930123285 -118.1149581890335) bank29340 +29341 POINT(33.91318595223692 -117.32395902045288) bank29341 +29342 POINT(34.10793090964318 -117.52076687124924) bank29342 +29343 POINT(34.70812125687605 -118.52049181575691) bank29343 +29344 POINT(34.53294337080171 -118.16041998890832) bank29344 +29345 POINT(33.074443452020745 -117.44650767018875) bank29345 +29346 POINT(34.87125144228478 -118.50043539328033) bank29346 +29347 POINT(34.675961235171435 -118.21593555511961) bank29347 +29348 POINT(33.83120738404013 -118.1107253180709) bank29348 +29349 POINT(34.28055942811028 -118.40897399356786) bank29349 +29350 POINT(34.44359543404906 -117.9578137295273) bank29350 +29351 POINT(33.38261302049329 -117.42586440576916) bank29351 +29352 POINT(33.160160678641084 -118.69302431102685) bank29352 +29353 POINT(33.65023507802063 -118.63273139974288) bank29353 +29354 POINT(34.8089235150529 -118.5363556345446) bank29354 +29355 POINT(33.53766506906475 -117.60659906939014) bank29355 +29356 POINT(33.37256584617395 -117.94059676735709) bank29356 +29357 POINT(33.064466767548815 -118.07704342759354) bank29357 +29358 POINT(33.75151108499713 -117.30241284083367) bank29358 +29359 POINT(33.487654841122534 -118.33252051639235) bank29359 +29360 POINT(33.87971700994415 -118.0073021956698) bank29360 +29361 POINT(33.16837701170073 -117.36152354830065) bank29361 +29362 POINT(34.32891470309881 -119.19913288592997) bank29362 +29363 POINT(33.12992739216932 -117.40269139886934) bank29363 +29364 POINT(34.72559910865938 -119.2000902258383) bank29364 +29365 POINT(34.03813002734947 -117.42095282197596) bank29365 +29366 POINT(34.26113061747949 -118.35732771987554) bank29366 +29367 POINT(33.63098214278308 -119.0077671966638) bank29367 +29368 POINT(33.25879044408359 -117.48563412868998) bank29368 +29369 POINT(33.27083628007379 -117.58725251907269) bank29369 +29370 POINT(34.32739313948741 -117.64512583004638) bank29370 +29371 POINT(33.48149160248773 -118.77704580473352) bank29371 +29372 POINT(34.285739344457326 -117.9857287403134) bank29372 +29373 POINT(33.68153183881213 -118.61629108411819) bank29373 +29374 POINT(33.92324297500165 -118.22954800711487) bank29374 +29375 POINT(34.49640672901067 -118.93825417994597) bank29375 +29376 POINT(33.314462930914544 -117.59428382983303) bank29376 +29377 POINT(33.516311888186785 -118.64642417207598) bank29377 +29378 POINT(33.735341312716294 -118.00842477435913) bank29378 +29379 POINT(33.832036932818916 -118.90431521712773) bank29379 +29380 POINT(33.96069555401251 -119.11236651353941) bank29380 +29381 POINT(34.781333474653024 -117.7982802015903) bank29381 +29382 POINT(33.759426059299024 -117.91240038618682) bank29382 +29383 POINT(33.84042298478751 -119.16107202100517) bank29383 +29384 POINT(34.146413523562316 -119.21520820390957) bank29384 +29385 POINT(34.40185415592236 -118.9726232625379) bank29385 +29386 POINT(33.972497037010406 -117.70619602114974) bank29386 +29387 POINT(35.034038581607426 -119.16595988256981) bank29387 +29388 POINT(34.92380019286934 -117.94582563784405) bank29388 +29389 POINT(34.1354151971325 -117.88815881848197) bank29389 +29390 POINT(33.57725032769998 -117.42458711440634) bank29390 +29391 POINT(33.92677813563918 -118.82662455348942) bank29391 +29392 POINT(33.1973339342684 -118.19835998752781) bank29392 +29393 POINT(33.110214193913826 -118.93791484333795) bank29393 +29394 POINT(34.85464479516417 -118.6499168486006) bank29394 +29395 POINT(33.377289494842586 -118.1069988996355) bank29395 +29396 POINT(33.994092402425494 -118.15651403102092) bank29396 +29397 POINT(34.40760186410191 -118.3235723964276) bank29397 +29398 POINT(34.07894488358352 -117.36547633779465) bank29398 +29399 POINT(33.75693802078846 -118.10207326775917) bank29399 +29400 POINT(34.73048951485715 -118.0156732839778) bank29400 +29401 POINT(34.807464604358046 -117.64020644901416) bank29401 +29402 POINT(34.62818187659636 -118.99621282275226) bank29402 +29403 POINT(33.68410673389219 -118.74646906098944) bank29403 +29404 POINT(34.437042411889294 -117.31033321099255) bank29404 +29405 POINT(33.60919678705585 -118.93124554766737) bank29405 +29406 POINT(33.3406857405877 -118.27360969952491) bank29406 +29407 POINT(35.00162787927972 -118.55717539312634) bank29407 +29408 POINT(33.513799420963046 -117.96101891008274) bank29408 +29409 POINT(33.440588611877374 -119.04334029723758) bank29409 +29410 POINT(33.637153920787384 -118.77526226337376) bank29410 +29411 POINT(34.02112793189859 -118.3199146465114) bank29411 +29412 POINT(33.20058599388802 -117.49523311466183) bank29412 +29413 POINT(34.65056371165712 -117.55471648775243) bank29413 +29414 POINT(33.624782699127906 -117.60085062746178) bank29414 +29415 POINT(33.90350614830903 -117.73694066258227) bank29415 +29416 POINT(33.98476550067913 -117.2465649406567) bank29416 +29417 POINT(34.379372191187514 -118.71613141718069) bank29417 +29418 POINT(33.62252500154826 -117.71124089344575) bank29418 +29419 POINT(33.27851366064742 -118.7350749357635) bank29419 +29420 POINT(34.809248514509164 -118.14996908523189) bank29420 +29421 POINT(33.20895857339146 -117.87960108905298) bank29421 +29422 POINT(34.58809038727051 -118.96930898680623) bank29422 +29423 POINT(34.03017992030527 -118.37318717539944) bank29423 +29424 POINT(34.10651910411536 -118.30607008363855) bank29424 +29425 POINT(34.59584154430083 -118.23551156601265) bank29425 +29426 POINT(33.54372478169932 -119.07684566485896) bank29426 +29427 POINT(33.40977929299021 -117.8570090749938) bank29427 +29428 POINT(33.25793812217114 -119.19101550930044) bank29428 +29429 POINT(34.82403414936765 -117.92241457552207) bank29429 +29430 POINT(33.3018448234983 -118.4796304510638) bank29430 +29431 POINT(33.15434549823722 -118.45423873591847) bank29431 +29432 POINT(33.95665529534657 -118.55507023877092) bank29432 +29433 POINT(34.69724470991587 -119.00425221406802) bank29433 +29434 POINT(33.12536645732305 -118.33099896788906) bank29434 +29435 POINT(35.04290431686736 -119.08348983094193) bank29435 +29436 POINT(33.204792822596424 -117.28157009214729) bank29436 +29437 POINT(34.88978433212769 -117.49317642844272) bank29437 +29438 POINT(34.93722294863862 -117.88824997209288) bank29438 +29439 POINT(33.780347731905806 -117.98347082996634) bank29439 +29440 POINT(33.206690570456566 -117.76616172611625) bank29440 +29441 POINT(33.767049556127276 -118.6670916157476) bank29441 +29442 POINT(33.85578927286601 -119.06172863683494) bank29442 +29443 POINT(33.41491855693358 -119.23862239319965) bank29443 +29444 POINT(34.64840590823016 -118.96602540432495) bank29444 +29445 POINT(34.54822185426189 -118.65493131119086) bank29445 +29446 POINT(33.77778380870883 -117.90757809494215) bank29446 +29447 POINT(34.860554087555315 -117.878453322127) bank29447 +29448 POINT(33.342059565061255 -119.00826228123121) bank29448 +29449 POINT(34.44612371917057 -119.088884501974) bank29449 +29450 POINT(33.8349597654611 -117.80433416012042) bank29450 +29451 POINT(33.61727144293243 -118.44006646803675) bank29451 +29452 POINT(34.06647715931147 -118.79065874942889) bank29452 +29453 POINT(33.09609913520093 -117.55366345843532) bank29453 +29454 POINT(33.31861756627004 -119.00777827232135) bank29454 +29455 POINT(33.73078477493313 -118.29529393339791) bank29455 +29456 POINT(34.58789176301264 -119.05064306037143) bank29456 +29457 POINT(34.916007853894136 -117.39032051108539) bank29457 +29458 POINT(33.37576944050363 -118.43344822341226) bank29458 +29459 POINT(33.32889975436644 -117.30471010118961) bank29459 +29460 POINT(33.889035640754365 -117.43566739513943) bank29460 +29461 POINT(33.71097241430628 -119.18312859431143) bank29461 +29462 POINT(34.152550085796456 -117.80047376855451) bank29462 +29463 POINT(33.92481916435967 -117.65106890340725) bank29463 +29464 POINT(34.33301719651563 -118.88383247462485) bank29464 +29465 POINT(34.04131662936536 -118.11432607318719) bank29465 +29466 POINT(33.2661622175841 -118.54194101007458) bank29466 +29467 POINT(34.99063784591096 -118.73456279112033) bank29467 +29468 POINT(34.874874947464576 -118.41480142328432) bank29468 +29469 POINT(34.920194667512504 -117.77451523961884) bank29469 +29470 POINT(33.45088061145045 -119.10024803551127) bank29470 +29471 POINT(33.514862522788206 -118.66849042376722) bank29471 +29472 POINT(34.96929516985882 -118.63885735816571) bank29472 +29473 POINT(34.61471386367315 -118.5003765869552) bank29473 +29474 POINT(33.78053520279361 -117.35603375211743) bank29474 +29475 POINT(34.86821875962093 -119.11129361448707) bank29475 +29476 POINT(33.26008925720873 -118.98613938099633) bank29476 +29477 POINT(34.346415793530724 -117.60854470225026) bank29477 +29478 POINT(34.568133028232054 -117.82306455985302) bank29478 +29479 POINT(34.5534407688272 -117.32942560096464) bank29479 +29480 POINT(34.981985220785745 -118.8364428265953) bank29480 +29481 POINT(33.780745727607446 -118.79409710008528) bank29481 +29482 POINT(34.983891708303744 -117.85305969942677) bank29482 +29483 POINT(34.508104389381295 -118.50156475300628) bank29483 +29484 POINT(33.90695162739098 -118.96708589271981) bank29484 +29485 POINT(34.494493375115205 -117.4702299711617) bank29485 +29486 POINT(34.89845348662019 -118.53194675942333) bank29486 +29487 POINT(33.73687504412645 -117.82337839188506) bank29487 +29488 POINT(34.23842660910967 -118.07309153541472) bank29488 +29489 POINT(33.738322695980706 -118.92628323859648) bank29489 +29490 POINT(33.6200798705218 -117.7460346059245) bank29490 +29491 POINT(33.66716577593969 -118.7080827146969) bank29491 +29492 POINT(33.43140693359782 -117.34324338871991) bank29492 +29493 POINT(34.18561688879165 -117.79030859711105) bank29493 +29494 POINT(33.12493662107402 -117.3035796098508) bank29494 +29495 POINT(33.10002196862793 -119.07009857391802) bank29495 +29496 POINT(34.778886158709426 -117.65009246716883) bank29496 +29497 POINT(33.075932493775376 -118.95355957633426) bank29497 +29498 POINT(34.758799181336876 -119.22526521551971) bank29498 +29499 POINT(34.119436372975706 -117.25199148334907) bank29499 +29500 POINT(34.12099532591607 -117.92540963840096) bank29500 +29501 POINT(33.0613384320417 -117.4228540524352) bank29501 +29502 POINT(33.1690337298685 -119.17478291632759) bank29502 +29503 POINT(34.316706481078626 -119.22102377459375) bank29503 +29504 POINT(33.5813453710142 -118.82326165297567) bank29504 +29505 POINT(34.73144040828813 -118.6186083631452) bank29505 +29506 POINT(34.19843056281057 -118.88128212283928) bank29506 +29507 POINT(33.12313937418822 -117.42223488751995) bank29507 +29508 POINT(33.95243240180284 -118.99380193352135) bank29508 +29509 POINT(34.82404872561251 -118.90199303348147) bank29509 +29510 POINT(33.74035501920695 -118.98525704339147) bank29510 +29511 POINT(34.08324732243482 -118.24544286130151) bank29511 +29512 POINT(34.690487924355665 -117.54032709219545) bank29512 +29513 POINT(35.02657754187121 -118.205287893195) bank29513 +29514 POINT(33.44509938913004 -118.85399658207511) bank29514 +29515 POINT(34.55546117202292 -119.14498720972492) bank29515 +29516 POINT(34.75778171159328 -118.61189098179918) bank29516 +29517 POINT(34.725609934855676 -118.07463227457166) bank29517 +29518 POINT(34.6785272084331 -119.18334973315038) bank29518 +29519 POINT(33.299476567272386 -118.16801430501323) bank29519 +29520 POINT(33.908146102009916 -118.71090269116853) bank29520 +29521 POINT(33.80167746721901 -119.14878870755955) bank29521 +29522 POINT(34.398371221808745 -118.30156058725339) bank29522 +29523 POINT(33.66648202107271 -117.8652461906294) bank29523 +29524 POINT(34.32307394216486 -117.37127360976993) bank29524 +29525 POINT(33.16213185058665 -118.39351508260413) bank29525 +29526 POINT(33.06027345226627 -118.20258555311987) bank29526 +29527 POINT(33.10178973668801 -118.85711698835128) bank29527 +29528 POINT(33.20260278668051 -118.94083351489357) bank29528 +29529 POINT(33.91562696824807 -117.64240033760665) bank29529 +29530 POINT(34.59194841827714 -117.7040951282099) bank29530 +29531 POINT(33.87436551021232 -118.44494252481071) bank29531 +29532 POINT(33.82527542572904 -118.59452061062461) bank29532 +29533 POINT(34.52496436032669 -117.40347945144599) bank29533 +29534 POINT(33.80977438392802 -118.90660136834848) bank29534 +29535 POINT(33.965858407173286 -117.46431286235675) bank29535 +29536 POINT(34.40064260233711 -117.26740028606505) bank29536 +29537 POINT(33.546131880810115 -118.98800963749522) bank29537 +29538 POINT(34.76125762616832 -118.00461778842269) bank29538 +29539 POINT(34.96017746313798 -118.06932627894437) bank29539 +29540 POINT(33.626715436854816 -117.63765473273199) bank29540 +29541 POINT(34.58668913788764 -118.13005593948263) bank29541 +29542 POINT(33.088606365724985 -117.47295298749712) bank29542 +29543 POINT(33.80323762316016 -117.64125656761499) bank29543 +29544 POINT(34.09845561087425 -118.5999958450826) bank29544 +29545 POINT(35.001167147655465 -118.17521405204725) bank29545 +29546 POINT(34.16235832291936 -117.57421081392083) bank29546 +29547 POINT(34.69713190703541 -117.99652151509544) bank29547 +29548 POINT(34.07096628074223 -118.83269486539078) bank29548 +29549 POINT(33.60386139450417 -117.94959544206746) bank29549 +29550 POINT(34.20650711981106 -118.06599373082372) bank29550 +29551 POINT(35.03862807093463 -118.77120193096818) bank29551 +29552 POINT(34.63380449897903 -118.16521638180286) bank29552 +29553 POINT(34.865209437029264 -117.68152474226797) bank29553 +29554 POINT(33.339959850948375 -117.94852759411619) bank29554 +29555 POINT(34.571161513217604 -118.35285570486903) bank29555 +29556 POINT(34.83188878119328 -117.9895085921233) bank29556 +29557 POINT(34.96025955388932 -118.2265277259218) bank29557 +29558 POINT(33.37215099105957 -118.73982178055783) bank29558 +29559 POINT(33.6468023266081 -118.93557316552416) bank29559 +29560 POINT(34.01553354173721 -119.0617439256401) bank29560 +29561 POINT(34.70417995971579 -117.71801100956122) bank29561 +29562 POINT(34.635077345681744 -118.55802043448038) bank29562 +29563 POINT(33.83781451015814 -118.10692592067055) bank29563 +29564 POINT(33.74124374042136 -118.40220876768102) bank29564 +29565 POINT(33.15249519162928 -118.34604752011268) bank29565 +29566 POINT(34.42719530965509 -119.0947952140737) bank29566 +29567 POINT(33.33752863102804 -119.07207660044128) bank29567 +29568 POINT(33.23918744524685 -118.38379045784619) bank29568 +29569 POINT(34.02990339365744 -117.81935131556634) bank29569 +29570 POINT(34.52340524804639 -119.19735118251936) bank29570 +29571 POINT(33.593778162030475 -118.85188307774627) bank29571 +29572 POINT(34.57422161604389 -117.84048457892249) bank29572 +29573 POINT(33.717031060491365 -118.20682315805145) bank29573 +29574 POINT(34.96813879780939 -119.20080521298681) bank29574 +29575 POINT(34.99738787308193 -117.76139934151436) bank29575 +29576 POINT(34.403298845163704 -118.64639587858609) bank29576 +29577 POINT(33.896781400897325 -118.18834783175605) bank29577 +29578 POINT(34.933443819483486 -118.89907070334066) bank29578 +29579 POINT(33.46971015031084 -119.09948477080418) bank29579 +29580 POINT(33.20577036305489 -118.96563081849465) bank29580 +29581 POINT(33.18428783867999 -118.69370680463962) bank29581 +29582 POINT(33.81675018959138 -119.2156090652872) bank29582 +29583 POINT(33.9953348572879 -117.87529814423804) bank29583 +29584 POINT(33.754332811939825 -117.52138050955665) bank29584 +29585 POINT(34.577090608703166 -117.69034914989372) bank29585 +29586 POINT(33.97762523699912 -117.61975164226769) bank29586 +29587 POINT(33.20295782283572 -118.35854443114752) bank29587 +29588 POINT(34.377694370314885 -118.77554053076712) bank29588 +29589 POINT(33.855400076064896 -118.86407780571034) bank29589 +29590 POINT(34.94036372826021 -118.54479217280208) bank29590 +29591 POINT(33.69396045150533 -118.93053903333852) bank29591 +29592 POINT(34.052161307462754 -117.62048054390382) bank29592 +29593 POINT(34.47867898518502 -118.97250679584613) bank29593 +29594 POINT(35.001351467762845 -118.86923273851484) bank29594 +29595 POINT(34.44652884573496 -118.62925075933657) bank29595 +29596 POINT(34.14898229475095 -117.37507622366448) bank29596 +29597 POINT(34.43882729314826 -117.63028078009735) bank29597 +29598 POINT(33.79251961253326 -117.62598819949086) bank29598 +29599 POINT(34.47847722358994 -117.35227302954625) bank29599 +29600 POINT(33.24858434013878 -118.19256453930254) bank29600 +29601 POINT(33.15316465580476 -117.8447516097202) bank29601 +29602 POINT(34.31465706041138 -117.62286991717674) bank29602 +29603 POINT(33.96825190805627 -117.88853380873562) bank29603 +29604 POINT(34.87049719417339 -118.34705020577795) bank29604 +29605 POINT(33.41016633717853 -117.83585372300328) bank29605 +29606 POINT(34.360141291528095 -118.44381142735543) bank29606 +29607 POINT(34.399349930196834 -118.12538988127922) bank29607 +29608 POINT(34.759260318898136 -118.2980218750319) bank29608 +29609 POINT(34.54333305285038 -117.9923074809634) bank29609 +29610 POINT(33.545864281987384 -117.4600630025149) bank29610 +29611 POINT(34.24480174427918 -117.68056363916887) bank29611 +29612 POINT(34.24455527307113 -117.82534322261874) bank29612 +29613 POINT(33.25617950424904 -118.26440731041258) bank29613 +29614 POINT(34.59442987159394 -118.19175961198451) bank29614 +29615 POINT(33.593313233182556 -117.86293428277705) bank29615 +29616 POINT(33.49075449663465 -118.86198061381154) bank29616 +29617 POINT(34.54839156982796 -118.54742155782603) bank29617 +29618 POINT(34.36931898787163 -118.09351109343928) bank29618 +29619 POINT(33.05498026022093 -118.42206102037711) bank29619 +29620 POINT(34.17787128681162 -117.6129348481616) bank29620 +29621 POINT(34.86142854868623 -119.16933086624083) bank29621 +29622 POINT(33.359934136703956 -119.02387666856634) bank29622 +29623 POINT(34.17081372464482 -118.92536933317676) bank29623 +29624 POINT(34.773996952232366 -118.22283039944065) bank29624 +29625 POINT(34.62880053740742 -117.93495901464354) bank29625 +29626 POINT(34.071739635854364 -119.11847436760877) bank29626 +29627 POINT(33.443030587785465 -118.91818409425383) bank29627 +29628 POINT(33.54282963498985 -117.52407619242751) bank29628 +29629 POINT(34.33854458073577 -118.71482176230106) bank29629 +29630 POINT(33.78491815436396 -118.36303396242617) bank29630 +29631 POINT(33.898168604100164 -118.09730582981656) bank29631 +29632 POINT(34.3570830244653 -118.7565807958276) bank29632 +29633 POINT(33.175840406999775 -118.83277219490462) bank29633 +29634 POINT(33.156481169088714 -117.62887374688373) bank29634 +29635 POINT(34.60415204688442 -118.44358235805561) bank29635 +29636 POINT(33.08023965576875 -117.87068935770527) bank29636 +29637 POINT(33.374012592396724 -117.73446250828583) bank29637 +29638 POINT(33.26108408207672 -118.53757758051943) bank29638 +29639 POINT(34.309277844216524 -117.77681017109713) bank29639 +29640 POINT(35.014033169754356 -117.57298909731497) bank29640 +29641 POINT(33.954590310929 -118.69320765604155) bank29641 +29642 POINT(35.04492038567178 -118.1320544032916) bank29642 +29643 POINT(34.80134237838877 -119.18743661538419) bank29643 +29644 POINT(33.52013154036779 -118.7808763623343) bank29644 +29645 POINT(34.98836614171057 -118.84983621617269) bank29645 +29646 POINT(33.94731492873592 -118.28071278103215) bank29646 +29647 POINT(34.93363318201281 -117.2569005600712) bank29647 +29648 POINT(33.075581675560116 -118.19262474491003) bank29648 +29649 POINT(34.87682732999665 -117.76928551979675) bank29649 +29650 POINT(33.78179551947063 -118.78589926551868) bank29650 +29651 POINT(33.144022204878475 -118.9794473945361) bank29651 +29652 POINT(33.397602306403876 -117.7131835946526) bank29652 +29653 POINT(34.8008548452712 -117.80125594299096) bank29653 +29654 POINT(34.622107241477195 -118.47791275517173) bank29654 +29655 POINT(33.36345836537921 -118.1846138693943) bank29655 +29656 POINT(33.25511438675012 -118.75858582203426) bank29656 +29657 POINT(34.46526549397925 -118.28834331639275) bank29657 +29658 POINT(34.15263422524069 -118.91584294780712) bank29658 +29659 POINT(33.92793913182322 -118.70800984255042) bank29659 +29660 POINT(33.93203370011879 -118.58031658391961) bank29660 +29661 POINT(34.275598301046024 -117.63810207452757) bank29661 +29662 POINT(34.98896871020182 -118.63853128462117) bank29662 +29663 POINT(34.29260224554355 -118.3448648209339) bank29663 +29664 POINT(34.768231700240996 -118.91528862519209) bank29664 +29665 POINT(33.64291568720241 -119.02575432152636) bank29665 +29666 POINT(34.48250030793511 -118.41024494290842) bank29666 +29667 POINT(34.68066055367079 -117.79254087361387) bank29667 +29668 POINT(33.947321916403084 -117.35784350250556) bank29668 +29669 POINT(34.09359269890077 -117.84892252116934) bank29669 +29670 POINT(33.75455301851403 -118.75866033029672) bank29670 +29671 POINT(33.985165341512484 -118.64860993730926) bank29671 +29672 POINT(33.263499040186616 -118.31860394632702) bank29672 +29673 POINT(34.525645709731236 -118.02439314837241) bank29673 +29674 POINT(33.431236729134866 -119.16915209136036) bank29674 +29675 POINT(34.926246829997446 -117.45360711653221) bank29675 +29676 POINT(33.310379622032414 -119.22246166742691) bank29676 +29677 POINT(34.91951551233101 -118.40780687488406) bank29677 +29678 POINT(33.43285711124094 -118.4274074125082) bank29678 +29679 POINT(34.865660003521036 -118.49469090570184) bank29679 +29680 POINT(33.098785354956824 -117.41403174288686) bank29680 +29681 POINT(34.77433658059024 -118.33270862030263) bank29681 +29682 POINT(34.556394397817215 -118.5319334158757) bank29682 +29683 POINT(34.696235540844036 -118.96435011896399) bank29683 +29684 POINT(34.210399739395015 -118.87998067234508) bank29684 +29685 POINT(33.641790768906006 -117.61900377553683) bank29685 +29686 POINT(33.60254740826098 -119.12290967373242) bank29686 +29687 POINT(33.23919461484573 -117.599119211439) bank29687 +29688 POINT(34.84771349755443 -117.53720653725553) bank29688 +29689 POINT(34.87054276259182 -118.27244809953113) bank29689 +29690 POINT(34.956889281286315 -118.35035675349519) bank29690 +29691 POINT(33.60223489947051 -117.40245525191929) bank29691 +29692 POINT(34.666374771422994 -118.90351551323556) bank29692 +29693 POINT(34.90852173530101 -118.00972652073897) bank29693 +29694 POINT(33.840751957839004 -118.44214580307604) bank29694 +29695 POINT(34.615324712507395 -117.32120226379989) bank29695 +29696 POINT(34.851955932993114 -117.69340331970481) bank29696 +29697 POINT(33.14952293532733 -117.7678451211303) bank29697 +29698 POINT(33.89277255183977 -117.7010467419046) bank29698 +29699 POINT(34.39149292281613 -117.31072089106435) bank29699 +29700 POINT(34.09211452310054 -117.24571362127686) bank29700 +29701 POINT(34.779873447511804 -117.6798675775347) bank29701 +29702 POINT(33.1000612758471 -117.36928980188223) bank29702 +29703 POINT(34.41643303662995 -119.22470083667) bank29703 +29704 POINT(33.778615038737705 -118.94470547231079) bank29704 +29705 POINT(34.389415736938815 -119.16421354746943) bank29705 +29706 POINT(34.86593497909394 -117.50766480700457) bank29706 +29707 POINT(33.172560674092956 -118.04338321509526) bank29707 +29708 POINT(34.07437440499291 -118.28474348187508) bank29708 +29709 POINT(33.195212258050354 -117.88780029016786) bank29709 +29710 POINT(34.163419918350534 -118.36588172651658) bank29710 +29711 POINT(34.003787191930805 -118.9985189728824) bank29711 +29712 POINT(34.302032618460295 -117.26635331027812) bank29712 +29713 POINT(34.84510228340051 -118.06539159268122) bank29713 +29714 POINT(33.88390674464957 -118.97127725692232) bank29714 +29715 POINT(33.58471023550031 -117.86540340359447) bank29715 +29716 POINT(33.16471970446663 -118.05254879556205) bank29716 +29717 POINT(34.437740928903246 -117.36027941926842) bank29717 +29718 POINT(34.52703050045075 -118.8374984235938) bank29718 +29719 POINT(33.86371308973653 -118.48493123613538) bank29719 +29720 POINT(33.52588108075194 -118.78776809557417) bank29720 +29721 POINT(33.44865213422052 -118.45104410734098) bank29721 +29722 POINT(33.53331255670128 -118.0888042950397) bank29722 +29723 POINT(33.280203775654456 -118.90237143175463) bank29723 +29724 POINT(34.72438841976867 -118.49123298507554) bank29724 +29725 POINT(33.309404352040545 -118.70358790273579) bank29725 +29726 POINT(33.30258866475144 -118.05486298996786) bank29726 +29727 POINT(34.57285394587442 -117.33489365807183) bank29727 +29728 POINT(33.2965620821851 -118.84527600094039) bank29728 +29729 POINT(33.785766961028756 -117.30317218384442) bank29729 +29730 POINT(34.871906975392854 -118.10540595330222) bank29730 +29731 POINT(33.257944766762066 -118.58881546068142) bank29731 +29732 POINT(33.54253617131312 -118.01178889505582) bank29732 +29733 POINT(34.69896546554409 -117.50993735303527) bank29733 +29734 POINT(35.01476127258579 -117.88731200749402) bank29734 +29735 POINT(35.0383501649766 -117.41690079318525) bank29735 +29736 POINT(34.80946504500611 -118.44725763938068) bank29736 +29737 POINT(34.175946499997565 -119.2435550582611) bank29737 +29738 POINT(34.25786797321137 -117.71724857082046) bank29738 +29739 POINT(33.621738623600805 -118.84812317072206) bank29739 +29740 POINT(34.60404986515697 -118.45801936216814) bank29740 +29741 POINT(33.50799455754219 -118.08971320239917) bank29741 +29742 POINT(33.26631923373389 -118.22301172755598) bank29742 +29743 POINT(34.8451751628349 -119.00955500718364) bank29743 +29744 POINT(34.202516849175 -117.44395905273096) bank29744 +29745 POINT(34.555726152754914 -117.88455025072913) bank29745 +29746 POINT(34.24364739815631 -117.88029662543393) bank29746 +29747 POINT(33.29375101202477 -118.24411775481059) bank29747 +29748 POINT(34.218159165809496 -118.15463745601893) bank29748 +29749 POINT(33.1046694825998 -117.25092036695939) bank29749 +29750 POINT(34.9011006499623 -118.53907780363289) bank29750 +29751 POINT(33.55155967628479 -117.71725508349657) bank29751 +29752 POINT(34.003930742657126 -119.00972874767379) bank29752 +29753 POINT(34.36568445932834 -117.82514097492678) bank29753 +29754 POINT(33.42457643299592 -118.0010753515261) bank29754 +29755 POINT(33.72460392974666 -118.1570229636571) bank29755 +29756 POINT(33.76775801658333 -118.49418393692999) bank29756 +29757 POINT(34.08629207665096 -118.33978151644695) bank29757 +29758 POINT(34.67982142528101 -118.50761862287358) bank29758 +29759 POINT(33.735569419708604 -117.38628547453597) bank29759 +29760 POINT(34.99372523711279 -118.28400827715888) bank29760 +29761 POINT(33.93085992908538 -118.73248835843566) bank29761 +29762 POINT(33.45127257050326 -117.69003736026329) bank29762 +29763 POINT(33.63365530688719 -117.52186195217993) bank29763 +29764 POINT(33.09120133950045 -119.03463610114602) bank29764 +29765 POINT(34.10851082470548 -117.64104096925797) bank29765 +29766 POINT(34.5780367526872 -118.48669132714396) bank29766 +29767 POINT(34.244561556703246 -117.85395181188686) bank29767 +29768 POINT(34.00796606611522 -118.44461644298633) bank29768 +29769 POINT(34.305237846525884 -118.95278469138707) bank29769 +29770 POINT(35.012247408975284 -118.86868961228193) bank29770 +29771 POINT(33.72943184288341 -118.54906862302902) bank29771 +29772 POINT(33.572701077006485 -118.34434456950974) bank29772 +29773 POINT(34.706054029860745 -119.23432719366998) bank29773 +29774 POINT(34.946707546493094 -117.68654219623512) bank29774 +29775 POINT(34.16852226408329 -117.80959669160963) bank29775 +29776 POINT(34.34111318547708 -117.85009180903789) bank29776 +29777 POINT(33.40948749805162 -117.42316641929482) bank29777 +29778 POINT(34.976707893480516 -117.84857097463347) bank29778 +29779 POINT(33.17084937623857 -117.30348182430708) bank29779 +29780 POINT(33.16312165100999 -118.2479408054407) bank29780 +29781 POINT(34.47891335423746 -118.81159501152047) bank29781 +29782 POINT(33.742551911055116 -118.54229755609144) bank29782 +29783 POINT(33.952649742842766 -118.23359561680665) bank29783 +29784 POINT(33.375274542026496 -117.44737089592297) bank29784 +29785 POINT(34.7230766647793 -118.64946924510599) bank29785 +29786 POINT(34.06960758432906 -119.22630108365904) bank29786 +29787 POINT(34.02843144326607 -119.01941633662379) bank29787 +29788 POINT(34.01407980756686 -117.53191771458488) bank29788 +29789 POINT(34.26171630518256 -119.20726140939605) bank29789 +29790 POINT(35.03428961283486 -119.1270402042046) bank29790 +29791 POINT(33.71400815554007 -118.59291833475004) bank29791 +29792 POINT(33.99503328403604 -119.21016936959542) bank29792 +29793 POINT(33.17061976276902 -117.90211354360588) bank29793 +29794 POINT(33.544657082340535 -117.50680778636183) bank29794 +29795 POINT(33.99760603987614 -117.59265946995707) bank29795 +29796 POINT(33.77504357405874 -118.14285974116007) bank29796 +29797 POINT(34.59107462753729 -118.96629238249027) bank29797 +29798 POINT(33.710079211386414 -118.98381528513879) bank29798 +29799 POINT(33.53873819573433 -118.98492413323433) bank29799 +29800 POINT(34.022853191803534 -118.35860628501241) bank29800 +29801 POINT(33.84474689173309 -117.25998353576053) bank29801 +29802 POINT(33.53153983573699 -118.97695325952927) bank29802 +29803 POINT(34.10816286686873 -117.75018810179844) bank29803 +29804 POINT(33.149379163084824 -117.92947447282106) bank29804 +29805 POINT(33.68016420829916 -117.8643202903471) bank29805 +29806 POINT(33.51777887437009 -118.40802737141203) bank29806 +29807 POINT(34.954831113113116 -117.81561030190679) bank29807 +29808 POINT(33.69169022109308 -117.52133857940247) bank29808 +29809 POINT(33.21340233695904 -117.70474117011736) bank29809 +29810 POINT(33.34286852651713 -118.73642580725965) bank29810 +29811 POINT(34.28226785033357 -118.74867494839297) bank29811 +29812 POINT(34.819252997127315 -119.19530301046991) bank29812 +29813 POINT(33.9929310819876 -117.78928720300507) bank29813 +29814 POINT(35.00520598795376 -117.25838826556223) bank29814 +29815 POINT(33.52033500198797 -118.29581498889588) bank29815 +29816 POINT(33.74716081271242 -118.24043517298685) bank29816 +29817 POINT(34.0575092989499 -117.72975723346073) bank29817 +29818 POINT(33.192643994727845 -117.89578244565764) bank29818 +29819 POINT(33.94491274733225 -119.04481885903199) bank29819 +29820 POINT(33.49741743348866 -118.67072227171806) bank29820 +29821 POINT(33.631988248494594 -118.25698616920009) bank29821 +29822 POINT(33.128447366264396 -118.43571122270974) bank29822 +29823 POINT(34.758819003644895 -118.79492879553165) bank29823 +29824 POINT(34.728588203341346 -118.61744035292972) bank29824 +29825 POINT(34.073366509634184 -117.87629260925819) bank29825 +29826 POINT(34.94567249149257 -117.95858247268332) bank29826 +29827 POINT(34.23059031638788 -118.40906216874043) bank29827 +29828 POINT(34.87470368267914 -117.84787419558927) bank29828 +29829 POINT(34.853190000789816 -118.38121199536756) bank29829 +29830 POINT(33.70983032019279 -117.32981178320397) bank29830 +29831 POINT(34.083394696839115 -117.56278285806543) bank29831 +29832 POINT(34.28474979818414 -119.06899867167414) bank29832 +29833 POINT(34.96886857153997 -117.97314091186324) bank29833 +29834 POINT(33.234593637020616 -118.33767561173714) bank29834 +29835 POINT(34.96753162471305 -117.56043389644783) bank29835 +29836 POINT(34.113202797315054 -118.66346402081095) bank29836 +29837 POINT(33.90477823023326 -118.06752407925286) bank29837 +29838 POINT(33.93943241040577 -117.3584824356883) bank29838 +29839 POINT(34.59977688395799 -118.99684363099323) bank29839 +29840 POINT(34.68043605051844 -117.7793764080434) bank29840 +29841 POINT(33.49961479203203 -119.20706964421403) bank29841 +29842 POINT(34.85650712636561 -117.8625261832853) bank29842 +29843 POINT(35.044151082134974 -118.40196833950269) bank29843 +29844 POINT(34.09019753917303 -117.34430039855799) bank29844 +29845 POINT(33.355401077189576 -118.13258096101696) bank29845 +29846 POINT(33.79605671606628 -118.41437306692254) bank29846 +29847 POINT(34.89181553844901 -117.69615343020031) bank29847 +29848 POINT(34.091097948232566 -117.96650255543565) bank29848 +29849 POINT(33.622969003473294 -117.72313735183778) bank29849 +29850 POINT(34.81266542073647 -119.00629832475373) bank29850 +29851 POINT(34.66811327968052 -117.2487523639161) bank29851 +29852 POINT(34.10032314744087 -117.519936543982) bank29852 +29853 POINT(33.28159692423995 -118.49395561847025) bank29853 +29854 POINT(33.27263069352098 -117.97028525838917) bank29854 +29855 POINT(33.26744292843207 -119.21564264960739) bank29855 +29856 POINT(33.33973392785895 -117.4152890499942) bank29856 +29857 POINT(35.04532103119984 -117.39075661568751) bank29857 +29858 POINT(33.314734341443234 -118.97828993952108) bank29858 +29859 POINT(33.990281472583376 -118.52574095825952) bank29859 +29860 POINT(35.02284598890274 -118.0062433353361) bank29860 +29861 POINT(33.971835413700454 -117.44520165324738) bank29861 +29862 POINT(34.792851567801975 -118.14145984956268) bank29862 +29863 POINT(33.43849683282746 -119.21263597818324) bank29863 +29864 POINT(34.301202558537085 -118.51557510104038) bank29864 +29865 POINT(34.655702999466335 -118.84761752813279) bank29865 +29866 POINT(34.811802545490735 -118.62836752327553) bank29866 +29867 POINT(34.24259371512206 -118.16178041847354) bank29867 +29868 POINT(33.27462359269296 -117.99502852751137) bank29868 +29869 POINT(35.04209347591155 -117.72816560134086) bank29869 +29870 POINT(33.206798483055465 -118.68086545229662) bank29870 +29871 POINT(33.73894195531756 -118.36659740637893) bank29871 +29872 POINT(33.21532836797412 -117.28453884123763) bank29872 +29873 POINT(34.25702415001557 -117.96652357055866) bank29873 +29874 POINT(34.75910399492412 -117.33817399385632) bank29874 +29875 POINT(34.764722245064156 -119.1330162297748) bank29875 +29876 POINT(33.87815706952996 -119.06988106725363) bank29876 +29877 POINT(33.775520822972155 -118.12792491158045) bank29877 +29878 POINT(34.382587806966825 -119.20246547197955) bank29878 +29879 POINT(35.029086390245865 -117.55054653339192) bank29879 +29880 POINT(34.432616149826934 -118.23989740398044) bank29880 +29881 POINT(34.751726873434706 -118.66523027042035) bank29881 +29882 POINT(33.7949177550184 -118.44376575958417) bank29882 +29883 POINT(33.647863838737656 -117.75890880959408) bank29883 +29884 POINT(34.48279576491117 -117.485844328475) bank29884 +29885 POINT(34.644167884660206 -118.37263572387798) bank29885 +29886 POINT(34.76144645571399 -117.40414531468215) bank29886 +29887 POINT(34.575669463710476 -117.381658941063) bank29887 +29888 POINT(33.93852401825061 -117.79169879268336) bank29888 +29889 POINT(33.28663365189372 -118.79681394782911) bank29889 +29890 POINT(33.34585618076796 -119.05605533603908) bank29890 +29891 POINT(33.6674569146013 -118.3959605519055) bank29891 +29892 POINT(34.81109432692751 -118.46759356583426) bank29892 +29893 POINT(33.15051637856224 -118.78274400006761) bank29893 +29894 POINT(34.863857559671246 -118.51037584449304) bank29894 +29895 POINT(33.74661381041926 -119.18852701806512) bank29895 +29896 POINT(34.161746499341334 -117.46824434391557) bank29896 +29897 POINT(34.50001036561587 -119.23042985059563) bank29897 +29898 POINT(33.8395406074234 -117.61689745080967) bank29898 +29899 POINT(33.3562916875218 -118.98555515933575) bank29899 +29900 POINT(33.70208141917983 -118.92496223524844) bank29900 +29901 POINT(33.05580183975853 -118.26222642855228) bank29901 +29902 POINT(33.1447576551689 -119.12582292867828) bank29902 +29903 POINT(34.38537703607293 -118.5403222425221) bank29903 +29904 POINT(33.723953849914245 -117.56693748062727) bank29904 +29905 POINT(34.71094265996369 -117.48663212079967) bank29905 +29906 POINT(34.162703551886324 -118.44875822085311) bank29906 +29907 POINT(33.23790604220956 -117.3986711635792) bank29907 +29908 POINT(34.07030280968542 -117.5667377108591) bank29908 +29909 POINT(34.19642265802663 -118.33005601828494) bank29909 +29910 POINT(33.59498187592118 -117.6641098276016) bank29910 +29911 POINT(34.02040192868022 -118.04444771143947) bank29911 +29912 POINT(34.72553902052523 -118.43922227776292) bank29912 +29913 POINT(33.71344703083144 -117.56504518677134) bank29913 +29914 POINT(33.67028052554117 -117.73254422950747) bank29914 +29915 POINT(33.936027732310826 -119.04040938727444) bank29915 +29916 POINT(33.37675547000386 -118.44510768968836) bank29916 +29917 POINT(34.90732379541599 -117.84854195181141) bank29917 +29918 POINT(34.90193639859831 -118.11951544882253) bank29918 +29919 POINT(33.679186431187574 -118.01523497520348) bank29919 +29920 POINT(34.66863482493585 -117.39983655681183) bank29920 +29921 POINT(34.01651534311121 -117.75375879610121) bank29921 +29922 POINT(34.51221000709626 -118.40010731489873) bank29922 +29923 POINT(34.065763542802856 -117.69313983100476) bank29923 +29924 POINT(33.704404816816535 -117.3690866666264) bank29924 +29925 POINT(34.312867862353 -118.79477692200885) bank29925 +29926 POINT(33.28374393857007 -118.44230439718366) bank29926 +29927 POINT(34.136683525792215 -119.21057978341709) bank29927 +29928 POINT(33.36579253723122 -118.62682361856054) bank29928 +29929 POINT(34.4419272291487 -118.42596124678192) bank29929 +29930 POINT(33.738910045576624 -117.29103480178563) bank29930 +29931 POINT(33.47387801189936 -117.55917364749219) bank29931 +29932 POINT(35.03286646357713 -117.27975993769422) bank29932 +29933 POINT(33.501068785379715 -119.0402379381054) bank29933 +29934 POINT(35.0305446731986 -118.72970324494804) bank29934 +29935 POINT(33.873242126671755 -117.65642892390846) bank29935 +29936 POINT(34.00518770774998 -118.81065192765756) bank29936 +29937 POINT(35.0308025408214 -118.11092586391032) bank29937 +29938 POINT(33.492462392267285 -117.49049690915996) bank29938 +29939 POINT(34.43606633661317 -117.40129341499564) bank29939 +29940 POINT(34.46006762809237 -118.32470923014706) bank29940 +29941 POINT(33.81517741116015 -117.3151680309059) bank29941 +29942 POINT(33.19373703521314 -117.28638879844414) bank29942 +29943 POINT(34.338301409793395 -117.7834073561816) bank29943 +29944 POINT(34.27286244571069 -118.4116354692795) bank29944 +29945 POINT(33.8185234096997 -118.29416577529133) bank29945 +29946 POINT(34.75756901410877 -118.26982733439185) bank29946 +29947 POINT(34.55958429113036 -119.24104198678518) bank29947 +29948 POINT(34.75015997845404 -119.03914248991279) bank29948 +29949 POINT(33.64120798502173 -117.291060620034) bank29949 +29950 POINT(33.110663389358955 -117.26130804285911) bank29950 +29951 POINT(33.38858086751641 -118.08416369337661) bank29951 +29952 POINT(33.45287668984922 -117.48756542626279) bank29952 +29953 POINT(34.371327799249485 -119.12449442736016) bank29953 +29954 POINT(33.254731696035556 -119.18423335840306) bank29954 +29955 POINT(34.096055687530686 -118.6918434381065) bank29955 +29956 POINT(34.50835140247343 -119.1772181197215) bank29956 +29957 POINT(33.685930504376124 -118.69249474720856) bank29957 +29958 POINT(33.87238083098702 -117.75225672325233) bank29958 +29959 POINT(34.82518369593672 -117.83721054490793) bank29959 +29960 POINT(34.208080676618586 -118.38112862154368) bank29960 +29961 POINT(33.19037077080438 -118.51373027082454) bank29961 +29962 POINT(34.51192147248682 -118.07297770662596) bank29962 +29963 POINT(34.22752557708982 -117.66371400254131) bank29963 +29964 POINT(33.71104425064418 -117.37758501668736) bank29964 +29965 POINT(33.6019286653375 -118.21199027067956) bank29965 +29966 POINT(34.573909268687004 -119.06999521256779) bank29966 +29967 POINT(33.962556223530626 -117.3227570063979) bank29967 +29968 POINT(33.814969768484126 -118.4041612308493) bank29968 +29969 POINT(34.1861198640922 -118.18462217016969) bank29969 +29970 POINT(33.265889366351146 -118.25131672732843) bank29970 +29971 POINT(33.73538379973975 -119.15298020804671) bank29971 +29972 POINT(34.4996070749294 -118.85888834232597) bank29972 +29973 POINT(34.64231513780506 -118.37716782786741) bank29973 +29974 POINT(33.996706895500836 -117.71365978611757) bank29974 +29975 POINT(33.64155918998246 -119.15846763885938) bank29975 +29976 POINT(34.17796001106806 -118.42122507608498) bank29976 +29977 POINT(34.1976894028875 -118.00521055569992) bank29977 +29978 POINT(34.98435744883936 -119.1847810400714) bank29978 +29979 POINT(34.88897089298549 -117.37462172363158) bank29979 +29980 POINT(34.27570874799789 -117.82544052122878) bank29980 +29981 POINT(33.50361545945928 -117.29784868205348) bank29981 +29982 POINT(34.864489320308984 -119.20394389802438) bank29982 +29983 POINT(34.321524477745335 -119.24168457873124) bank29983 +29984 POINT(34.38085879076412 -117.90052392808151) bank29984 +29985 POINT(34.43061131350573 -117.53922287715835) bank29985 +29986 POINT(33.659868858245495 -118.6342742373963) bank29986 +29987 POINT(33.888916988351404 -118.38117743706329) bank29987 +29988 POINT(33.357223433909496 -118.25917877142888) bank29988 +29989 POINT(33.96467052684765 -117.80223261775458) bank29989 +29990 POINT(34.180177397345176 -118.37213898492773) bank29990 +29991 POINT(33.879218675845465 -117.33176510307041) bank29991 +29992 POINT(34.25090384950759 -118.37025432702323) bank29992 +29993 POINT(35.01608546464964 -118.23539458472115) bank29993 +29994 POINT(33.66298384800734 -119.199013440445) bank29994 +29995 POINT(34.93254890018328 -117.45942883452112) bank29995 +29996 POINT(33.873936365736185 -118.11456661119949) bank29996 +29997 POINT(33.37090358152457 -117.9209630495241) bank29997 +29998 POINT(34.325712771982865 -118.72712939767655) bank29998 +29999 POINT(34.642678307081894 -117.25250358116575) bank29999 +30000 POINT(33.41306137577282 -118.01672213537918) bank30000 +30001 POINT(34.63855582204302 -117.90486104223802) bank30001 +30002 POINT(34.10673916018265 -118.00802304655565) bank30002 +30003 POINT(34.19246702059486 -117.55844571971681) bank30003 +30004 POINT(33.91598195792674 -117.80810908479769) bank30004 +30005 POINT(33.195734294611434 -118.4840786181446) bank30005 +30006 POINT(33.700115839061056 -117.44351323352964) bank30006 +30007 POINT(34.0554190712543 -117.76608009065161) bank30007 +30008 POINT(34.42700892976162 -118.01509361181475) bank30008 +30009 POINT(33.43001841915197 -117.71619115736367) bank30009 +30010 POINT(34.15437378130159 -117.9233113719242) bank30010 +30011 POINT(34.99546986766354 -118.81479933213546) bank30011 +30012 POINT(34.64675916387228 -118.4117171983324) bank30012 +30013 POINT(34.9044761292393 -119.21242362917404) bank30013 +30014 POINT(34.36313991247658 -119.2289875304747) bank30014 +30015 POINT(34.311652596089054 -118.71631965390857) bank30015 +30016 POINT(33.93478525480431 -117.87575164124502) bank30016 +30017 POINT(33.48637011055472 -117.5649452184867) bank30017 +30018 POINT(33.40171057464132 -118.70562528466961) bank30018 +30019 POINT(34.185416436853814 -118.05746579089035) bank30019 +30020 POINT(33.0646965650839 -118.15685382223309) bank30020 +30021 POINT(34.73299348449107 -118.49207359744591) bank30021 +30022 POINT(33.676461378048025 -118.05344571393626) bank30022 +30023 POINT(34.2907078987649 -117.34273865469432) bank30023 +30024 POINT(34.36774214742247 -117.34320685464957) bank30024 +30025 POINT(34.8339594056798 -117.60028440315443) bank30025 +30026 POINT(33.469195440492754 -117.5968914644099) bank30026 +30027 POINT(34.89285913346382 -117.50353607100035) bank30027 +30028 POINT(34.72218027691912 -118.61021383220029) bank30028 +30029 POINT(33.32579300949073 -118.26465517277029) bank30029 +30030 POINT(34.01375803899601 -117.5907523228712) bank30030 +30031 POINT(34.13310995692843 -117.28285299374565) bank30031 +30032 POINT(34.897817031917405 -118.61229935641207) bank30032 +30033 POINT(33.37881489860719 -117.36840874234323) bank30033 +30034 POINT(33.33423296452691 -118.31148185357279) bank30034 +30035 POINT(33.56861573153799 -118.72257484581144) bank30035 +30036 POINT(33.56672548773834 -117.32837289653003) bank30036 +30037 POINT(33.8805297097534 -118.37770147397033) bank30037 +30038 POINT(34.801855519029175 -118.20205411723191) bank30038 +30039 POINT(34.9609466224311 -117.34469562254914) bank30039 +30040 POINT(33.08355223035166 -117.685956877384) bank30040 +30041 POINT(34.28126912253805 -117.8351083435436) bank30041 +30042 POINT(34.34453853583662 -118.00153845802753) bank30042 +30043 POINT(33.40879449650173 -118.20585343948238) bank30043 +30044 POINT(33.6782386528439 -117.68091883019241) bank30044 +30045 POINT(34.87559243118338 -117.47499462643582) bank30045 +30046 POINT(34.750051247522066 -117.67815013509795) bank30046 +30047 POINT(33.68821337325255 -117.54014986396274) bank30047 +30048 POINT(33.394091151315145 -118.9028837706112) bank30048 +30049 POINT(34.10197564589561 -117.93573486911248) bank30049 +30050 POINT(34.53930950531756 -117.43345220895732) bank30050 +30051 POINT(34.484311996507444 -117.78990648026596) bank30051 +30052 POINT(33.62005917756644 -118.00243464363373) bank30052 +30053 POINT(33.66026828824096 -118.59721835027369) bank30053 +30054 POINT(33.795053378279455 -117.75694844450632) bank30054 +30055 POINT(33.11584630884966 -118.21353375954747) bank30055 +30056 POINT(34.631927431548554 -117.81900737223891) bank30056 +30057 POINT(34.209889939537234 -117.78324645069704) bank30057 +30058 POINT(33.97535166859185 -118.84276165337984) bank30058 +30059 POINT(34.99826705769689 -117.42020554236194) bank30059 +30060 POINT(34.37576048142552 -119.18013860827443) bank30060 +30061 POINT(34.3955896926824 -118.73897586154601) bank30061 +30062 POINT(34.45431104960177 -118.14233682356452) bank30062 +30063 POINT(33.523691323172464 -117.69576422833288) bank30063 +30064 POINT(34.16804438677509 -117.77922381905066) bank30064 +30065 POINT(33.357818779353636 -118.88601861402894) bank30065 +30066 POINT(33.21277441547276 -119.16104054403355) bank30066 +30067 POINT(33.05562395093192 -118.0897620208127) bank30067 +30068 POINT(34.70728659765735 -118.83302663549904) bank30068 +30069 POINT(33.37784116603899 -117.83500521085674) bank30069 +30070 POINT(34.96301154869456 -118.69203707154212) bank30070 +30071 POINT(34.8862352150872 -118.3102400425453) bank30071 +30072 POINT(33.88794630579624 -119.18178146251064) bank30072 +30073 POINT(34.23724663290237 -117.87849364308045) bank30073 +30074 POINT(33.962163418908496 -117.66070207815933) bank30074 +30075 POINT(34.95316525040268 -118.84418869183385) bank30075 +30076 POINT(34.819712685866754 -118.94616329640562) bank30076 +30077 POINT(33.494229266008055 -118.12158190605844) bank30077 +30078 POINT(34.89030195741627 -118.8650052875581) bank30078 +30079 POINT(33.858733484451186 -117.96652504337118) bank30079 +30080 POINT(34.121523212176534 -118.66153924811003) bank30080 +30081 POINT(33.807622945054284 -119.16332296337762) bank30081 +30082 POINT(34.120764527874655 -117.9717741961753) bank30082 +30083 POINT(34.34936277046607 -118.15813876860454) bank30083 +30084 POINT(33.94623220706051 -118.3930732626708) bank30084 +30085 POINT(34.09339201100878 -118.13962031106445) bank30085 +30086 POINT(33.494384115537855 -118.25618480644462) bank30086 +30087 POINT(33.61018333244644 -118.90518060402131) bank30087 +30088 POINT(34.61842174070654 -117.97931596655066) bank30088 +30089 POINT(34.114136511143904 -117.9925056556475) bank30089 +30090 POINT(33.61126661468034 -119.10258061027605) bank30090 +30091 POINT(34.13006797976153 -117.47846124044055) bank30091 +30092 POINT(34.397157711384125 -117.8377429117708) bank30092 +30093 POINT(34.275091187977466 -118.01143290039798) bank30093 +30094 POINT(33.719560827869024 -118.32173320539775) bank30094 +30095 POINT(33.396571665217365 -118.6628949299872) bank30095 +30096 POINT(33.0738721942195 -117.58781057493282) bank30096 +30097 POINT(33.09239768443153 -117.40818452401277) bank30097 +30098 POINT(33.40297992181505 -118.89075742271446) bank30098 +30099 POINT(34.99575359728327 -117.95629103141184) bank30099 +30100 POINT(34.16480211078667 -117.53791334342479) bank30100 +30101 POINT(33.59763269984498 -118.4251091646204) bank30101 +30102 POINT(34.58116454662838 -117.88631692381581) bank30102 +30103 POINT(33.74580458807551 -118.43424867052006) bank30103 +30104 POINT(33.48351038896815 -118.24269178525289) bank30104 +30105 POINT(33.699100336354086 -118.074616945917) bank30105 +30106 POINT(34.98744153323502 -117.31650456386744) bank30106 +30107 POINT(33.46789049049806 -119.13490680104401) bank30107 +30108 POINT(35.012304186385094 -118.7377788975171) bank30108 +30109 POINT(33.87581136540462 -119.04225641726951) bank30109 +30110 POINT(33.19838661427381 -117.47661629205801) bank30110 +30111 POINT(34.355935566466876 -117.70899535203293) bank30111 +30112 POINT(34.74152728209492 -118.46547952849224) bank30112 +30113 POINT(34.96526428032932 -118.0247728725202) bank30113 +30114 POINT(33.88573491000157 -119.12550112639428) bank30114 +30115 POINT(34.76719088022519 -118.27061588456193) bank30115 +30116 POINT(33.17763292220262 -117.65675404384379) bank30116 +30117 POINT(33.06816690237081 -117.80637221777091) bank30117 +30118 POINT(34.0607900539121 -117.44861279835712) bank30118 +30119 POINT(33.94095227336569 -118.52597200588006) bank30119 +30120 POINT(34.998239079645195 -118.88159095391013) bank30120 +30121 POINT(33.87061147536465 -119.24318911305497) bank30121 +30122 POINT(33.71804306439024 -118.34156304271804) bank30122 +30123 POINT(34.76612920458016 -117.74537420942693) bank30123 +30124 POINT(34.72850185031419 -119.20300107787592) bank30124 +30125 POINT(33.99115376050081 -117.6577055542561) bank30125 +30126 POINT(34.07030204026098 -118.41741427903978) bank30126 +30127 POINT(34.62643043191043 -119.09479437246) bank30127 +30128 POINT(34.40542905020228 -117.80099426946936) bank30128 +30129 POINT(33.2685639964126 -117.58853386595538) bank30129 +30130 POINT(34.70328002624775 -118.53874694765359) bank30130 +30131 POINT(33.7321458159664 -117.99625258357088) bank30131 +30132 POINT(33.843143455158184 -117.29885749936496) bank30132 +30133 POINT(34.46835682150829 -118.15672961227509) bank30133 +30134 POINT(33.89728501402012 -117.7547102889626) bank30134 +30135 POINT(34.60614380113594 -117.35433250952623) bank30135 +30136 POINT(34.26091756589127 -118.85157970543085) bank30136 +30137 POINT(33.159455947214006 -118.16569085129717) bank30137 +30138 POINT(33.51411468655815 -118.02472977285599) bank30138 +30139 POINT(33.49728400041478 -118.95596623576678) bank30139 +30140 POINT(33.270397585769686 -118.50408740745722) bank30140 +30141 POINT(34.87880324515506 -118.96794210005152) bank30141 +30142 POINT(35.00305650408224 -117.33447710964069) bank30142 +30143 POINT(34.498043147644516 -118.74328708768124) bank30143 +30144 POINT(34.6125101344262 -117.51864519374018) bank30144 +30145 POINT(34.04959971126258 -118.96427480863086) bank30145 +30146 POINT(33.54335074108628 -119.1569555382178) bank30146 +30147 POINT(33.18156033492345 -118.54522654224705) bank30147 +30148 POINT(33.3361867001455 -117.24709414564778) bank30148 +30149 POINT(33.98748033647986 -117.56365489524683) bank30149 +30150 POINT(34.55369085559695 -118.5895245401664) bank30150 +30151 POINT(35.004138363859134 -118.73156033956657) bank30151 +30152 POINT(34.655564821585266 -118.3129187921109) bank30152 +30153 POINT(34.44932102415392 -118.20975646915653) bank30153 +30154 POINT(34.1655404569253 -118.34243770418004) bank30154 +30155 POINT(33.204144875356455 -118.33021931989096) bank30155 +30156 POINT(34.38359136018772 -119.17441033708768) bank30156 +30157 POINT(34.52303790089958 -118.39839500287303) bank30157 +30158 POINT(33.48496458004472 -118.80550706007251) bank30158 +30159 POINT(34.935443566692136 -118.28114643140903) bank30159 +30160 POINT(34.39190511205704 -117.79645195523729) bank30160 +30161 POINT(34.538629379670255 -118.06086146466708) bank30161 +30162 POINT(34.56295548206289 -118.96484109230573) bank30162 +30163 POINT(34.47675789486424 -117.64809215569804) bank30163 +30164 POINT(33.238266299966924 -117.84618426798113) bank30164 +30165 POINT(33.77255824564507 -117.33678091130315) bank30165 +30166 POINT(33.93535341228534 -117.52815260632997) bank30166 +30167 POINT(34.95642232992501 -118.95001789173457) bank30167 +30168 POINT(34.51951663662979 -117.80993749237386) bank30168 +30169 POINT(33.66989109510141 -119.18322056744411) bank30169 +30170 POINT(33.28652982180082 -117.86774692123518) bank30170 +30171 POINT(33.141960666272546 -117.5689801228002) bank30171 +30172 POINT(33.28608056586513 -117.49297378744751) bank30172 +30173 POINT(33.157354222591316 -117.42495666290885) bank30173 +30174 POINT(34.13354237059984 -118.44919653403464) bank30174 +30175 POINT(33.591235960427575 -119.00322417760532) bank30175 +30176 POINT(34.40536370361482 -117.4207610904296) bank30176 +30177 POINT(34.23772204450459 -119.0413611710593) bank30177 +30178 POINT(33.762810399841804 -117.75508729109633) bank30178 +30179 POINT(34.492865590704895 -117.63950481451928) bank30179 +30180 POINT(34.629050621555024 -117.53611348576707) bank30180 +30181 POINT(33.09800940512634 -118.06334777326894) bank30181 +30182 POINT(33.65864830752521 -117.64372600949768) bank30182 +30183 POINT(34.868551552718856 -117.53727461680786) bank30183 +30184 POINT(33.66319512367396 -118.91343964069894) bank30184 +30185 POINT(34.48192686549551 -118.72200777243525) bank30185 +30186 POINT(34.58127622591637 -118.69442901905548) bank30186 +30187 POINT(33.9869789876093 -118.91990541549723) bank30187 +30188 POINT(33.39677342497655 -118.50325729042608) bank30188 +30189 POINT(34.76179201928664 -118.61445942823882) bank30189 +30190 POINT(34.29963881583908 -117.90964724178532) bank30190 +30191 POINT(34.022933548052734 -117.83795414424542) bank30191 +30192 POINT(33.17248339086126 -118.55665892697164) bank30192 +30193 POINT(34.556897477746276 -119.23848336760747) bank30193 +30194 POINT(33.92955635884881 -118.13218979841554) bank30194 +30195 POINT(33.52366594689278 -118.23939513998975) bank30195 +30196 POINT(34.00763820616654 -118.52231876114497) bank30196 +30197 POINT(34.21133860764571 -118.46134935605673) bank30197 +30198 POINT(33.38674105934948 -118.76343252421569) bank30198 +30199 POINT(34.478628906415246 -117.94423085025629) bank30199 +30200 POINT(34.03529076272812 -118.17464829527255) bank30200 +30201 POINT(33.66869243905657 -118.78724366063818) bank30201 +30202 POINT(33.16977034795894 -118.11629203644486) bank30202 +30203 POINT(34.20041195718044 -118.15547005436144) bank30203 +30204 POINT(34.805467963044855 -118.28404301683291) bank30204 +30205 POINT(33.07210620854465 -118.98581427335716) bank30205 +30206 POINT(33.1284176289235 -117.81921960468574) bank30206 +30207 POINT(33.74153566942465 -118.13842647168487) bank30207 +30208 POINT(34.73830285258491 -117.74813168798187) bank30208 +30209 POINT(34.955584999152336 -117.45977447119259) bank30209 +30210 POINT(34.628619929003236 -118.08495376776148) bank30210 +30211 POINT(33.91939362010751 -117.40570471560787) bank30211 +30212 POINT(33.59265730320513 -118.42818014783616) bank30212 +30213 POINT(33.80773261276342 -117.39179185331494) bank30213 +30214 POINT(34.798074543217474 -118.02695989391871) bank30214 +30215 POINT(34.7315123923995 -117.45488062097274) bank30215 +30216 POINT(33.065126080771854 -118.2069168230569) bank30216 +30217 POINT(34.72331317545281 -117.66427317936521) bank30217 +30218 POINT(33.62969428282749 -119.18487049524714) bank30218 +30219 POINT(34.48658706776888 -117.44906047757628) bank30219 +30220 POINT(34.96529267440878 -119.17229802064503) bank30220 +30221 POINT(34.38385876980796 -117.69893810001045) bank30221 +30222 POINT(33.21374529002711 -118.4679242941343) bank30222 +30223 POINT(33.798343405906365 -117.28177273772296) bank30223 +30224 POINT(34.58218721183781 -119.00959880278802) bank30224 +30225 POINT(33.858619929415106 -118.94876152672033) bank30225 +30226 POINT(35.00572749464932 -118.92077231470341) bank30226 +30227 POINT(34.00255956705945 -117.27456180693441) bank30227 +30228 POINT(34.167152934100265 -117.62894756826356) bank30228 +30229 POINT(34.273544002674235 -117.4795365245912) bank30229 +30230 POINT(34.92889723496102 -118.08721287537102) bank30230 +30231 POINT(34.34613151877675 -117.38873672892277) bank30231 +30232 POINT(33.25067507289621 -117.37839087434841) bank30232 +30233 POINT(33.86483834595492 -118.1342367194856) bank30233 +30234 POINT(33.20843224162096 -117.37046977847434) bank30234 +30235 POINT(33.60567660136427 -118.36805533764255) bank30235 +30236 POINT(33.2117220337702 -117.70062818145513) bank30236 +30237 POINT(33.3178572406935 -118.00043308660142) bank30237 +30238 POINT(34.173115547648294 -118.04330546674593) bank30238 +30239 POINT(34.41573348533642 -118.6275228237363) bank30239 +30240 POINT(34.39906246166617 -117.45345425243612) bank30240 +30241 POINT(34.162850435954354 -117.84610692774223) bank30241 +30242 POINT(34.26466035052183 -119.12442266772408) bank30242 +30243 POINT(34.98469747591153 -117.50450125814811) bank30243 +30244 POINT(34.145096748788305 -117.76494099936144) bank30244 +30245 POINT(35.00829560353826 -119.1228970080003) bank30245 +30246 POINT(34.18641407916182 -118.65163801893536) bank30246 +30247 POINT(34.98556247847897 -118.29193221784928) bank30247 +30248 POINT(33.88562813067737 -118.72907816815015) bank30248 +30249 POINT(34.3350349520502 -118.47614137114729) bank30249 +30250 POINT(33.282593461234356 -119.08477522342206) bank30250 +30251 POINT(35.03386180004389 -117.80539332896122) bank30251 +30252 POINT(33.13497481858964 -118.1507224533379) bank30252 +30253 POINT(34.4535993977411 -118.59765773219846) bank30253 +30254 POINT(33.52252883103384 -117.98478762751157) bank30254 +30255 POINT(34.598901131471 -119.07239222473426) bank30255 +30256 POINT(33.24286221598288 -117.89968870901944) bank30256 +30257 POINT(33.926524446439025 -117.52653645935645) bank30257 +30258 POINT(33.70666690340419 -117.4382484107674) bank30258 +30259 POINT(34.320259375043044 -118.24867701010794) bank30259 +30260 POINT(33.274637362545384 -118.04465018550727) bank30260 +30261 POINT(35.03230811626855 -117.76500307599811) bank30261 +30262 POINT(34.734740926061285 -118.87863350870583) bank30262 +30263 POINT(34.34945847019677 -118.65328741401028) bank30263 +30264 POINT(35.050227972477565 -118.64888434884962) bank30264 +30265 POINT(33.83950105363773 -118.85192088049845) bank30265 +30266 POINT(33.37541132409319 -119.0859018968938) bank30266 +30267 POINT(34.863292012496274 -118.08447660181226) bank30267 +30268 POINT(34.789602349262715 -118.4230848630037) bank30268 +30269 POINT(34.63384050969342 -118.85679565394649) bank30269 +30270 POINT(34.99391859876832 -118.9652031117002) bank30270 +30271 POINT(33.974906605571384 -119.13578476053159) bank30271 +30272 POINT(33.23370502294791 -118.07467768404646) bank30272 +30273 POINT(33.78316687314287 -118.79417244060517) bank30273 +30274 POINT(33.944961274554366 -118.07203591587106) bank30274 +30275 POINT(33.69214772707994 -119.07548498624857) bank30275 +30276 POINT(34.269956658108285 -118.99194685995224) bank30276 +30277 POINT(33.4924249015826 -119.0385799536727) bank30277 +30278 POINT(33.452804677412544 -117.28236058618707) bank30278 +30279 POINT(35.02145338144705 -117.36691236984605) bank30279 +30280 POINT(34.30666606811681 -118.01898853973783) bank30280 +30281 POINT(33.191544296526224 -118.78909570953954) bank30281 +30282 POINT(33.18587945158464 -118.39942636587683) bank30282 +30283 POINT(34.533943548561325 -118.30412647413557) bank30283 +30284 POINT(34.3141483722844 -118.76351641961217) bank30284 +30285 POINT(34.960626795347046 -117.82842988108933) bank30285 +30286 POINT(34.3381063500193 -119.17774658651459) bank30286 +30287 POINT(34.74076683242238 -117.56336822750882) bank30287 +30288 POINT(34.33480096720731 -119.21447249669349) bank30288 +30289 POINT(33.58794391275936 -117.44397810986567) bank30289 +30290 POINT(34.671184524611384 -118.66898415834694) bank30290 +30291 POINT(34.838430164901155 -118.00375445382444) bank30291 +30292 POINT(33.46441183987095 -118.61549754679844) bank30292 +30293 POINT(34.034951829995926 -118.2098203399654) bank30293 +30294 POINT(33.82283457133299 -118.0068390482902) bank30294 +30295 POINT(33.876636230568536 -118.86879143071778) bank30295 +30296 POINT(33.923859083248985 -118.01878351306209) bank30296 +30297 POINT(33.55571288321668 -117.92066589302293) bank30297 +30298 POINT(33.1268544763293 -117.78532795930593) bank30298 +30299 POINT(34.93963466256889 -117.93053426692121) bank30299 +30300 POINT(34.490068466699185 -117.5800694022886) bank30300 +30301 POINT(33.8156979322056 -117.66114573260722) bank30301 +30302 POINT(34.46431691274947 -118.53346056161111) bank30302 +30303 POINT(33.441685023977065 -118.26099083026969) bank30303 +30304 POINT(33.76988453148113 -118.69167987509354) bank30304 +30305 POINT(33.360062642029014 -118.7049589567836) bank30305 +30306 POINT(34.97198624872057 -118.58518230478249) bank30306 +30307 POINT(33.55468086528644 -117.73332019909209) bank30307 +30308 POINT(33.9177856777275 -117.98393850845284) bank30308 +30309 POINT(34.79738062505645 -117.3740285436226) bank30309 +30310 POINT(34.37341005858649 -117.68721156977101) bank30310 +30311 POINT(33.72874089793982 -118.55382712967888) bank30311 +30312 POINT(34.767598082594255 -117.80736561032228) bank30312 +30313 POINT(34.80845740860356 -118.54741989696613) bank30313 +30314 POINT(33.30210666538991 -118.53361122733999) bank30314 +30315 POINT(33.24702195807663 -118.03184871904499) bank30315 +30316 POINT(33.40402549903256 -118.44241134196471) bank30316 +30317 POINT(34.38889427388248 -118.84891501460169) bank30317 +30318 POINT(33.7012153409282 -117.3992574832104) bank30318 +30319 POINT(33.42900294837346 -118.48659406227398) bank30319 +30320 POINT(33.616732011718966 -119.09769237450465) bank30320 +30321 POINT(33.86533133146513 -117.48003611783582) bank30321 +30322 POINT(33.492207126422315 -118.66425107537152) bank30322 +30323 POINT(33.9685467780419 -117.50628019721657) bank30323 +30324 POINT(33.68245358003974 -118.19564926909499) bank30324 +30325 POINT(33.80502815774576 -119.0954115102012) bank30325 +30326 POINT(34.266237955506185 -119.03474898517217) bank30326 +30327 POINT(33.647076457300386 -118.62237380074089) bank30327 +30328 POINT(34.12448856702834 -117.56358485273208) bank30328 +30329 POINT(33.485427384725746 -118.57101659936839) bank30329 +30330 POINT(34.875729920936934 -117.32336525063202) bank30330 +30331 POINT(33.06468773878685 -118.81115750457816) bank30331 +30332 POINT(33.482141815067266 -118.44390192260175) bank30332 +30333 POINT(34.88775345812415 -117.325685197782) bank30333 +30334 POINT(34.1187531438329 -117.6542733841172) bank30334 +30335 POINT(33.956948597517155 -117.31526322610715) bank30335 +30336 POINT(33.79249765698293 -117.61627762262609) bank30336 +30337 POINT(34.24404075347039 -117.54080169181157) bank30337 +30338 POINT(34.079704792936475 -117.58137644815456) bank30338 +30339 POINT(34.32387870788214 -118.66161814584598) bank30339 +30340 POINT(34.42401966562977 -118.91378988383299) bank30340 +30341 POINT(34.630108281492554 -117.37313002505819) bank30341 +30342 POINT(33.64352344524292 -117.73876115553671) bank30342 +30343 POINT(33.26372455315799 -118.5283251336963) bank30343 +30344 POINT(34.922810490678835 -118.43476976989949) bank30344 +30345 POINT(33.17937138996551 -117.90689600340365) bank30345 +30346 POINT(33.28626823331218 -118.34933937913743) bank30346 +30347 POINT(34.48481514874034 -117.77942291981547) bank30347 +30348 POINT(34.32409661952678 -117.65117594590885) bank30348 +30349 POINT(34.91138388176664 -119.08415926791116) bank30349 +30350 POINT(33.525785454308 -118.73714817104165) bank30350 +30351 POINT(34.90823828239995 -118.71323309085776) bank30351 +30352 POINT(34.851921822657985 -118.76981043644402) bank30352 +30353 POINT(33.506607519278376 -119.1764150415185) bank30353 +30354 POINT(33.59832615857884 -118.49510187848517) bank30354 +30355 POINT(34.06593589653926 -119.19643241714569) bank30355 +30356 POINT(33.14208837960032 -118.37948518284672) bank30356 +30357 POINT(33.45149521952442 -117.24465188383257) bank30357 +30358 POINT(34.39811709684766 -119.0052876839874) bank30358 +30359 POINT(33.29589124929499 -117.26083598388355) bank30359 +30360 POINT(33.70492578971582 -118.78260642806083) bank30360 +30361 POINT(33.726845747315544 -117.66091030553166) bank30361 +30362 POINT(34.100328547863604 -118.77021909038572) bank30362 +30363 POINT(33.66734725666326 -117.37598432839214) bank30363 +30364 POINT(34.582197149724784 -117.88490191237484) bank30364 +30365 POINT(33.65824515888803 -119.08543914830939) bank30365 +30366 POINT(34.53358457594479 -117.5224726779417) bank30366 +30367 POINT(34.986485444401154 -117.86446057670206) bank30367 +30368 POINT(33.99691643747501 -118.47836526519211) bank30368 +30369 POINT(33.876661618254495 -119.05469052134993) bank30369 +30370 POINT(34.29710050146549 -118.4553106041876) bank30370 +30371 POINT(33.12047393562168 -119.13401943200539) bank30371 +30372 POINT(34.72583392811299 -118.79304182943571) bank30372 +30373 POINT(34.56490870674193 -118.33373142454394) bank30373 +30374 POINT(34.199964720676235 -118.27546987908033) bank30374 +30375 POINT(34.31388156964877 -118.56902232890742) bank30375 +30376 POINT(34.30500748286322 -117.44889599046498) bank30376 +30377 POINT(33.431628825883834 -117.81060623274202) bank30377 +30378 POINT(33.59980749022574 -119.11905734901315) bank30378 +30379 POINT(33.10885074158958 -118.34127631947047) bank30379 +30380 POINT(34.22424758820451 -118.6481856408971) bank30380 +30381 POINT(33.48986397547669 -119.08408320952574) bank30381 +30382 POINT(34.75467510178011 -118.32488031602509) bank30382 +30383 POINT(34.896081395503735 -117.2468200537093) bank30383 +30384 POINT(34.11557961536853 -118.39372450331284) bank30384 +30385 POINT(34.840938015242806 -117.77639109994168) bank30385 +30386 POINT(33.580622721002634 -117.63582490897558) bank30386 +30387 POINT(34.01400181060229 -118.25237578642071) bank30387 +30388 POINT(34.288070068005794 -118.60060304766425) bank30388 +30389 POINT(34.077659698193365 -118.94006553038466) bank30389 +30390 POINT(34.98385912338233 -118.82492204449285) bank30390 +30391 POINT(33.464484866693354 -118.32834291725085) bank30391 +30392 POINT(34.68458247466116 -119.12931434946982) bank30392 +30393 POINT(33.117653848296236 -117.43430652085414) bank30393 +30394 POINT(33.36664202817812 -117.97584807321842) bank30394 +30395 POINT(33.33001284127896 -118.98081929164037) bank30395 +30396 POINT(34.29370138028699 -118.06994124595647) bank30396 +30397 POINT(33.35466719904009 -118.67923991775208) bank30397 +30398 POINT(35.01920897094609 -118.10857790703167) bank30398 +30399 POINT(34.05789075149624 -118.13717294580988) bank30399 +30400 POINT(34.84410087535161 -118.44318101313067) bank30400 +30401 POINT(34.48529094447761 -119.10459488992215) bank30401 +30402 POINT(33.06834784892094 -118.07599509915374) bank30402 +30403 POINT(34.78263423824662 -118.15148222652883) bank30403 +30404 POINT(33.84785012213419 -118.53906813411602) bank30404 +30405 POINT(34.73016845412513 -117.97020307315574) bank30405 +30406 POINT(34.4375406171459 -118.59875332278116) bank30406 +30407 POINT(33.29383555855458 -118.41179313943083) bank30407 +30408 POINT(34.20066509379551 -117.40348690939229) bank30408 +30409 POINT(33.53972630757868 -118.51281864893546) bank30409 +30410 POINT(33.68346298162477 -119.06864910340585) bank30410 +30411 POINT(33.7325123583883 -118.0101704404485) bank30411 +30412 POINT(34.6607173714693 -118.38221527274672) bank30412 +30413 POINT(33.60882662887376 -118.64475707757653) bank30413 +30414 POINT(33.51002574935214 -118.54988055502426) bank30414 +30415 POINT(33.56349168391364 -119.02590772831815) bank30415 +30416 POINT(33.07540817637466 -118.8571709782182) bank30416 +30417 POINT(34.9999453692645 -118.5741092314426) bank30417 +30418 POINT(33.750527150550745 -118.62239894786245) bank30418 +30419 POINT(33.638574104284224 -118.6474002963713) bank30419 +30420 POINT(33.50912797561837 -117.49348733308643) bank30420 +30421 POINT(34.8476411637164 -118.12136607863208) bank30421 +30422 POINT(33.6194865207689 -117.62881161312188) bank30422 +30423 POINT(33.25386053766141 -118.16954743875483) bank30423 +30424 POINT(33.389697264341244 -118.98579633634427) bank30424 +30425 POINT(34.60366108450013 -118.72245682514678) bank30425 +30426 POINT(33.39818172669846 -117.74701128042653) bank30426 +30427 POINT(34.62870289654731 -118.22004531627509) bank30427 +30428 POINT(34.99286126448692 -117.58262606846057) bank30428 +30429 POINT(34.564571244136175 -118.18142469374297) bank30429 +30430 POINT(33.31609036545582 -117.98618618062049) bank30430 +30431 POINT(34.97120870738233 -118.63772479428307) bank30431 +30432 POINT(34.19448142526109 -117.67250726237617) bank30432 +30433 POINT(34.16329410536392 -118.86386791610356) bank30433 +30434 POINT(34.78230542230814 -118.24701406892352) bank30434 +30435 POINT(34.648466877792515 -118.91970617092983) bank30435 +30436 POINT(34.24854763354923 -117.40745528771862) bank30436 +30437 POINT(33.48641312761088 -117.5387630345937) bank30437 +30438 POINT(34.30859801686642 -118.78425223554576) bank30438 +30439 POINT(33.7357556183311 -117.41755394091452) bank30439 +30440 POINT(34.08043526288119 -118.0680638116462) bank30440 +30441 POINT(33.39231106669784 -118.00693038231766) bank30441 +30442 POINT(34.455952147416724 -118.39761088052894) bank30442 +30443 POINT(33.19938589900369 -117.52899192877429) bank30443 +30444 POINT(33.17456374850324 -117.74505208291987) bank30444 +30445 POINT(34.11578708018336 -117.50051191079623) bank30445 +30446 POINT(33.949105130080795 -118.99369386750902) bank30446 +30447 POINT(34.9100011892869 -118.91908506131597) bank30447 +30448 POINT(33.45115669471162 -117.43659973793874) bank30448 +30449 POINT(33.42551915438703 -117.92045899277547) bank30449 +30450 POINT(34.65623234015359 -117.43034520103397) bank30450 +30451 POINT(34.5723506164634 -117.25255119498172) bank30451 +30452 POINT(33.56521114526487 -118.09835696699591) bank30452 +30453 POINT(34.13966385446352 -117.81735701936118) bank30453 +30454 POINT(34.638566038830945 -118.60346178321947) bank30454 +30455 POINT(33.94444515530972 -118.60471112572839) bank30455 +30456 POINT(33.52944320858948 -119.06148667972167) bank30456 +30457 POINT(34.85652940669229 -117.8062565347399) bank30457 +30458 POINT(33.75817254431199 -118.48581097978078) bank30458 +30459 POINT(33.302680300767165 -118.57209893195036) bank30459 +30460 POINT(34.939090665834435 -118.49058573203007) bank30460 +30461 POINT(33.05501398353649 -119.2385032083932) bank30461 +30462 POINT(34.47344450294439 -118.86702612196409) bank30462 +30463 POINT(33.8732342538062 -117.6108713309508) bank30463 +30464 POINT(34.52871380654741 -118.57776259724562) bank30464 +30465 POINT(33.20868972276269 -117.33082347806483) bank30465 +30466 POINT(33.734949049590476 -118.87540730509954) bank30466 +30467 POINT(34.28606607849867 -117.85550693505377) bank30467 +30468 POINT(34.833616877307 -117.25432491542115) bank30468 +30469 POINT(33.5428752318359 -117.54951755709065) bank30469 +30470 POINT(33.81761014597206 -119.11879013919872) bank30470 +30471 POINT(33.89754538446171 -119.07934849195729) bank30471 +30472 POINT(33.3376415047904 -117.6281535373998) bank30472 +30473 POINT(34.29668912045678 -117.56653969581332) bank30473 +30474 POINT(34.50211567461751 -118.45753032913369) bank30474 +30475 POINT(34.44478318181701 -118.15802108679185) bank30475 +30476 POINT(33.19639981261438 -118.1432920788156) bank30476 +30477 POINT(34.32168214335998 -118.93755165430285) bank30477 +30478 POINT(34.65988395510596 -118.42684149460653) bank30478 +30479 POINT(34.26336689867117 -117.4964047099122) bank30479 +30480 POINT(34.93390800895322 -118.61019600104201) bank30480 +30481 POINT(34.20852994379036 -117.82705435188173) bank30481 +30482 POINT(34.29999597254715 -117.8008581517321) bank30482 +30483 POINT(34.478142106371955 -117.25477782853011) bank30483 +30484 POINT(33.1945742727568 -118.80432208830871) bank30484 +30485 POINT(34.22320681709147 -117.73304386227903) bank30485 +30486 POINT(34.036146943727566 -117.61551314745218) bank30486 +30487 POINT(33.90971840817124 -117.35062196461328) bank30487 +30488 POINT(34.119439893431675 -117.89949158414647) bank30488 +30489 POINT(34.60870181890124 -117.50105833607863) bank30489 +30490 POINT(34.19786079343345 -117.36435039852516) bank30490 +30491 POINT(33.95387748296913 -118.17891444693625) bank30491 +30492 POINT(33.79707381501741 -118.04957875013184) bank30492 +30493 POINT(33.32318812342009 -118.62096517508876) bank30493 +30494 POINT(34.67029542591422 -118.93278438343685) bank30494 +30495 POINT(34.80651051564055 -118.18537459803161) bank30495 +30496 POINT(34.85103976481373 -117.77041791848696) bank30496 +30497 POINT(34.737156317364196 -118.79281659810181) bank30497 +30498 POINT(34.41258175758774 -117.96124469373592) bank30498 +30499 POINT(33.60447643891643 -119.03541165753552) bank30499 +30500 POINT(33.11863166022703 -117.50077002332047) bank30500 +30501 POINT(34.22205266736113 -118.25093676672375) bank30501 +30502 POINT(33.913170300874704 -118.94878477929964) bank30502 +30503 POINT(33.378178095327314 -118.49233501957444) bank30503 +30504 POINT(33.62846970695329 -117.55899837637375) bank30504 +30505 POINT(33.681201354984296 -118.42867311548397) bank30505 +30506 POINT(33.58004768482758 -117.71994089358655) bank30506 +30507 POINT(34.18071594324024 -119.12896843252743) bank30507 +30508 POINT(34.93121796625873 -118.30063999113769) bank30508 +30509 POINT(33.81772134929654 -118.24066797891965) bank30509 +30510 POINT(34.58475403366512 -117.34977896393744) bank30510 +30511 POINT(33.332730295011416 -118.15912597518114) bank30511 +30512 POINT(34.261858708575666 -118.79246504304683) bank30512 +30513 POINT(33.4003448404306 -118.1478691771139) bank30513 +30514 POINT(34.43581984539868 -119.11448048725617) bank30514 +30515 POINT(33.738012520751106 -117.40216894767691) bank30515 +30516 POINT(34.07407492613231 -117.58114554176058) bank30516 +30517 POINT(34.82290820306439 -118.97074118929541) bank30517 +30518 POINT(33.606405487516625 -119.15422887662453) bank30518 +30519 POINT(34.71357095613111 -118.21058654787657) bank30519 +30520 POINT(33.53595700982053 -117.55796808961121) bank30520 +30521 POINT(33.05403217543097 -117.32244352065996) bank30521 +30522 POINT(35.00183056633416 -117.84774248750014) bank30522 +30523 POINT(34.26963397056583 -118.24568856092588) bank30523 +30524 POINT(33.720022839840055 -118.10690906388956) bank30524 +30525 POINT(34.771517077186914 -118.34245534443211) bank30525 +30526 POINT(34.93942434965649 -118.36327844708144) bank30526 +30527 POINT(33.23702062464111 -117.47789884843812) bank30527 +30528 POINT(33.57348859018352 -119.14200704650368) bank30528 +30529 POINT(34.383354047044946 -118.21457068049706) bank30529 +30530 POINT(33.677769204315155 -118.51664406904493) bank30530 +30531 POINT(33.681535756364866 -118.53201401124335) bank30531 +30532 POINT(34.12395633728556 -117.54197045977548) bank30532 +30533 POINT(34.21357791114235 -117.37888308308591) bank30533 +30534 POINT(34.2306476872962 -117.47035678439518) bank30534 +30535 POINT(34.180334096398816 -119.17353353278334) bank30535 +30536 POINT(33.11521386848741 -117.43460830997837) bank30536 +30537 POINT(34.95035975086962 -118.93316631511789) bank30537 +30538 POINT(33.15670752441512 -118.33115808393508) bank30538 +30539 POINT(33.25195085891977 -118.75135797080453) bank30539 +30540 POINT(34.12239518380319 -118.12708310560843) bank30540 +30541 POINT(34.022168938722444 -118.0133397475765) bank30541 +30542 POINT(33.293565298406286 -118.48486615580735) bank30542 +30543 POINT(33.255112609751606 -118.68782228044834) bank30543 +30544 POINT(34.927486819256416 -118.42136988567907) bank30544 +30545 POINT(34.66208814632017 -118.13404861307099) bank30545 +30546 POINT(33.44543882875012 -118.79485023126122) bank30546 +30547 POINT(33.37526761587944 -118.71988421228181) bank30547 +30548 POINT(33.27279612022678 -117.64685323755556) bank30548 +30549 POINT(34.26474167670969 -117.5557916731055) bank30549 +30550 POINT(33.06974911128938 -118.34973848794522) bank30550 +30551 POINT(34.22639194268735 -117.66781449236184) bank30551 +30552 POINT(35.044142737293434 -117.39460784907347) bank30552 +30553 POINT(34.91467146937116 -118.30946879930333) bank30553 +30554 POINT(34.94539409968235 -118.26074032087915) bank30554 +30555 POINT(33.105891204866225 -117.94868168494216) bank30555 +30556 POINT(34.976071650012344 -118.7694504279444) bank30556 +30557 POINT(34.83178288423076 -118.36284366777761) bank30557 +30558 POINT(33.18601728910172 -118.49039934173642) bank30558 +30559 POINT(34.59942428951258 -118.79428402072838) bank30559 +30560 POINT(33.359000427609885 -118.6983897492237) bank30560 +30561 POINT(33.83737051204258 -118.28058996488294) bank30561 +30562 POINT(34.38813388339324 -118.61836777058915) bank30562 +30563 POINT(34.7364857489313 -118.6765200812793) bank30563 +30564 POINT(33.68052229438948 -117.65101481079867) bank30564 +30565 POINT(34.96769919766067 -117.34829650533456) bank30565 +30566 POINT(33.31016252526521 -118.528859174331) bank30566 +30567 POINT(33.999887814355304 -117.71774822508651) bank30567 +30568 POINT(33.07494879599012 -119.06862338010377) bank30568 +30569 POINT(33.1408756166301 -119.17474923218991) bank30569 +30570 POINT(33.230141074840446 -118.60357783995686) bank30570 +30571 POINT(33.22841337003032 -119.10396313940117) bank30571 +30572 POINT(34.29217934608755 -117.9221559328769) bank30572 +30573 POINT(34.26282494168754 -118.79356833891549) bank30573 +30574 POINT(34.324499726398095 -118.31791002933093) bank30574 +30575 POINT(34.060092398993035 -118.35353434333028) bank30575 +30576 POINT(33.125800498711136 -118.69785867159035) bank30576 +30577 POINT(33.67446426247056 -118.47156024900606) bank30577 +30578 POINT(33.455378514421575 -118.35159571789997) bank30578 +30579 POINT(34.34890668294973 -117.49970438396659) bank30579 +30580 POINT(34.95089407802383 -119.06010650707901) bank30580 +30581 POINT(33.26378488784664 -118.40662302427178) bank30581 +30582 POINT(34.18334656698712 -117.97490714432271) bank30582 +30583 POINT(33.253115773962456 -117.8671293635885) bank30583 +30584 POINT(33.25008947535894 -118.15973175640794) bank30584 +30585 POINT(33.91472220319507 -118.47449635264141) bank30585 +30586 POINT(34.22686517115924 -118.3368184994562) bank30586 +30587 POINT(34.721556355090534 -117.47202166347357) bank30587 +30588 POINT(34.91764145150502 -118.55841238240605) bank30588 +30589 POINT(34.16427153417478 -118.97584857233316) bank30589 +30590 POINT(33.93361699475593 -117.52432467591926) bank30590 +30591 POINT(33.419635909495085 -117.30912909641002) bank30591 +30592 POINT(34.5253202789141 -117.63816137192043) bank30592 +30593 POINT(33.41831910509297 -117.99749035645254) bank30593 +30594 POINT(33.41532749135835 -118.59271166566859) bank30594 +30595 POINT(34.393821597765246 -118.59065361293757) bank30595 +30596 POINT(33.334808580372965 -118.31155030108715) bank30596 +30597 POINT(34.399899802167134 -117.4256683023896) bank30597 +30598 POINT(33.80259053609818 -117.30470749496898) bank30598 +30599 POINT(34.99470198143347 -119.06513631070632) bank30599 +30600 POINT(34.55904590318472 -118.29345994458522) bank30600 +30601 POINT(34.65736971116828 -119.05371726666783) bank30601 +30602 POINT(34.16754291477152 -119.09403807787852) bank30602 +30603 POINT(34.45529268853416 -117.61806571713609) bank30603 +30604 POINT(33.28360042797356 -118.09832332174362) bank30604 +30605 POINT(33.32071827550839 -117.911463840739) bank30605 +30606 POINT(33.68033801267394 -118.31233574101665) bank30606 +30607 POINT(33.15479272709438 -119.15859220046349) bank30607 +30608 POINT(34.455778081548004 -117.50640541438598) bank30608 +30609 POINT(33.082925022259104 -118.47975894294535) bank30609 +30610 POINT(33.665659326015145 -118.7119212401198) bank30610 +30611 POINT(33.69735664712005 -117.3529404256021) bank30611 +30612 POINT(35.02738979524717 -117.74593227258515) bank30612 +30613 POINT(34.49872776744652 -117.44450892407005) bank30613 +30614 POINT(33.31787672505503 -118.93971440701702) bank30614 +30615 POINT(33.116920737925824 -117.37465412828269) bank30615 +30616 POINT(33.39707631065069 -118.70866609089096) bank30616 +30617 POINT(33.64184266141697 -118.30059997935219) bank30617 +30618 POINT(34.12617489271716 -118.02523585549945) bank30618 +30619 POINT(33.4053904213026 -118.77517359352068) bank30619 +30620 POINT(34.69840953689903 -117.81535639398531) bank30620 +30621 POINT(33.17621693804634 -118.59253805206036) bank30621 +30622 POINT(33.74553584899085 -118.1643433011093) bank30622 +30623 POINT(34.80920794502326 -118.92166734383513) bank30623 +30624 POINT(33.445483678665674 -118.83401049361512) bank30624 +30625 POINT(34.029419775813274 -118.98577229087375) bank30625 +30626 POINT(34.80135695758682 -119.16968266581814) bank30626 +30627 POINT(33.30675305179191 -117.28981649494045) bank30627 +30628 POINT(33.984747124701606 -117.5149103087169) bank30628 +30629 POINT(34.05276666546636 -118.30341268697835) bank30629 +30630 POINT(33.52232614858145 -118.13974781479163) bank30630 +30631 POINT(33.51683738939482 -118.33479201717972) bank30631 +30632 POINT(34.357282021905355 -119.20675690407886) bank30632 +30633 POINT(34.6477858893932 -118.41585807635524) bank30633 +30634 POINT(34.393725971443 -117.32170170198047) bank30634 +30635 POINT(33.5449129016746 -117.51378125995929) bank30635 +30636 POINT(34.40956450746103 -119.01787857577858) bank30636 +30637 POINT(33.95583389014661 -118.08365326450715) bank30637 +30638 POINT(33.61127811360252 -117.53194096253648) bank30638 +30639 POINT(33.56580427697861 -118.37056748295129) bank30639 +30640 POINT(34.708429891715376 -117.57796695341669) bank30640 +30641 POINT(33.990240983854996 -118.05695110676994) bank30641 +30642 POINT(33.99252162961088 -119.08298745504725) bank30642 +30643 POINT(34.82005831411935 -118.90530637248015) bank30643 +30644 POINT(33.92350951813468 -118.68939971297384) bank30644 +30645 POINT(33.37657075543584 -118.57209095654105) bank30645 +30646 POINT(34.31105299521742 -117.9371997078796) bank30646 +30647 POINT(34.143257959027 -118.10021282806981) bank30647 +30648 POINT(33.792734203542615 -118.35886260100521) bank30648 +30649 POINT(34.42726687321114 -117.9662966950058) bank30649 +30650 POINT(33.73631764127295 -118.16173786119549) bank30650 +30651 POINT(33.604856283920775 -118.58898731355032) bank30651 +30652 POINT(33.32269085334913 -117.2859613812428) bank30652 +30653 POINT(34.56518793550296 -118.5684978194834) bank30653 +30654 POINT(34.31290994782309 -118.0131618810228) bank30654 +30655 POINT(33.797043524299575 -118.83938012563407) bank30655 +30656 POINT(33.86146986150334 -117.93872352639318) bank30656 +30657 POINT(34.38646514796521 -118.99355639640105) bank30657 +30658 POINT(33.297958597607405 -118.39091484015779) bank30658 +30659 POINT(34.5168459426927 -118.52513915282204) bank30659 +30660 POINT(33.150551461232425 -118.52545573866462) bank30660 +30661 POINT(33.99312016282079 -118.26381188178861) bank30661 +30662 POINT(33.06173452876746 -119.10819424998523) bank30662 +30663 POINT(34.21129519452119 -117.53578645099327) bank30663 +30664 POINT(34.33635616124538 -118.25997243593812) bank30664 +30665 POINT(34.07395747082009 -117.77308114703114) bank30665 +30666 POINT(33.24110236384105 -117.68513854972431) bank30666 +30667 POINT(34.7181683410321 -117.89001187923175) bank30667 +30668 POINT(33.22491938813377 -118.10953801507942) bank30668 +30669 POINT(33.09296894483778 -119.06816851815303) bank30669 +30670 POINT(33.532376340962905 -118.46208440526662) bank30670 +30671 POINT(33.6654805584226 -117.5370820957411) bank30671 +30672 POINT(33.6179420749213 -117.44072119151717) bank30672 +30673 POINT(34.75932210150466 -118.2643098417505) bank30673 +30674 POINT(34.13143938896249 -118.16567112099305) bank30674 +30675 POINT(33.23097000118324 -118.29858618503347) bank30675 +30676 POINT(34.343208844092175 -117.47441710771307) bank30676 +30677 POINT(34.3335752090635 -118.61000826404812) bank30677 +30678 POINT(33.20372455240949 -117.55848922772743) bank30678 +30679 POINT(33.89653153585145 -117.99210176080359) bank30679 +30680 POINT(33.904882534683935 -118.84502155662746) bank30680 +30681 POINT(35.00046954744837 -117.51840421541361) bank30681 +30682 POINT(33.47183368694624 -118.83420975587315) bank30682 +30683 POINT(34.52632912553999 -117.49384126483154) bank30683 +30684 POINT(35.03192710503621 -118.64814241703803) bank30684 +30685 POINT(34.59573997154172 -118.74799595616308) bank30685 +30686 POINT(34.75975002917115 -117.8955880830549) bank30686 +30687 POINT(35.002349254548896 -119.22510723726599) bank30687 +30688 POINT(34.75162445499802 -118.72713063164787) bank30688 +30689 POINT(33.333504830962205 -118.12922835263491) bank30689 +30690 POINT(35.03702610035134 -118.84238399655815) bank30690 +30691 POINT(34.25627853966972 -117.68236374743805) bank30691 +30692 POINT(33.42982722879623 -119.14908636553642) bank30692 +30693 POINT(33.21725710991093 -117.64732203726199) bank30693 +30694 POINT(33.82201473190469 -118.35898927724274) bank30694 +30695 POINT(34.30248514187691 -119.1718642948505) bank30695 +30696 POINT(33.6334712352073 -117.58089952846638) bank30696 +30697 POINT(33.37597343539847 -118.2055598526641) bank30697 +30698 POINT(33.74215542425962 -119.00464169334647) bank30698 +30699 POINT(34.128151896399395 -118.29694025511589) bank30699 +30700 POINT(33.864645012811714 -117.77726266711537) bank30700 +30701 POINT(34.58998334613291 -118.29082558615276) bank30701 +30702 POINT(34.90841143601195 -118.50336626356598) bank30702 +30703 POINT(34.950188295693756 -118.77389939818671) bank30703 +30704 POINT(34.14774050682833 -118.46475235073862) bank30704 +30705 POINT(34.14987931911249 -117.93052059338524) bank30705 +30706 POINT(33.24546991444896 -118.66533364843534) bank30706 +30707 POINT(33.15725330307673 -118.76754928719197) bank30707 +30708 POINT(34.52235583771694 -117.7501841766944) bank30708 +30709 POINT(34.114981556557986 -118.11119092322433) bank30709 +30710 POINT(33.16574163332431 -118.15084535706434) bank30710 +30711 POINT(34.857924586219795 -117.5510231658003) bank30711 +30712 POINT(33.41387157061138 -118.11842384243548) bank30712 +30713 POINT(34.192766703672824 -118.59487663310286) bank30713 +30714 POINT(33.666637743459525 -118.33319585404226) bank30714 +30715 POINT(33.8988235374162 -117.7393980512785) bank30715 +30716 POINT(35.034277195327455 -117.61015079114874) bank30716 +30717 POINT(33.48094378260238 -117.99910902001453) bank30717 +30718 POINT(33.85960912511726 -117.8174068100419) bank30718 +30719 POINT(34.66445266960236 -118.22312800701637) bank30719 +30720 POINT(34.67383589074019 -117.4379537538435) bank30720 +30721 POINT(33.82238725171332 -117.76718454432729) bank30721 +30722 POINT(33.59534002875103 -118.37958704352911) bank30722 +30723 POINT(34.97740688712567 -118.18752112993593) bank30723 +30724 POINT(34.9764834298147 -118.20600527312664) bank30724 +30725 POINT(34.16792931739957 -119.24205272849632) bank30725 +30726 POINT(33.846624588244204 -117.69785157727914) bank30726 +30727 POINT(34.078867324983506 -117.37783044613745) bank30727 +30728 POINT(33.972083951296035 -118.833355997575) bank30728 +30729 POINT(34.073811188928595 -118.57679001882713) bank30729 +30730 POINT(33.749793692610645 -118.93758004316082) bank30730 +30731 POINT(35.01003337421556 -117.31695944589282) bank30731 +30732 POINT(33.6381800397977 -117.73502658075967) bank30732 +30733 POINT(33.222465771104176 -117.68138091721944) bank30733 +30734 POINT(34.35672241350993 -117.54482633360301) bank30734 +30735 POINT(33.502630407651594 -118.47128044043855) bank30735 +30736 POINT(34.81333673883031 -118.81007356644892) bank30736 +30737 POINT(33.67018943020781 -117.61236874485755) bank30737 +30738 POINT(33.82499601486428 -117.37959321431981) bank30738 +30739 POINT(33.21454665484667 -117.61859188959698) bank30739 +30740 POINT(34.014928642415356 -117.77671708466589) bank30740 +30741 POINT(33.599192770152484 -117.60694303308267) bank30741 +30742 POINT(33.13708699757744 -117.95460571518106) bank30742 +30743 POINT(33.77748045394813 -118.19143956282252) bank30743 +30744 POINT(33.43595875039222 -119.11902904569001) bank30744 +30745 POINT(34.95958609200562 -118.66171095280325) bank30745 +30746 POINT(33.621195809388574 -119.15692251196154) bank30746 +30747 POINT(33.8795709323733 -117.84775015502053) bank30747 +30748 POINT(33.673396909709716 -117.45185912365983) bank30748 +30749 POINT(33.24692732951273 -117.65278374669136) bank30749 +30750 POINT(33.982932508496155 -118.48952018674865) bank30750 +30751 POINT(34.65345360264448 -118.3341982645722) bank30751 +30752 POINT(34.611072902955726 -117.5459450576974) bank30752 +30753 POINT(33.256722458766966 -118.80166140864418) bank30753 +30754 POINT(34.34617885986041 -118.91970166056764) bank30754 +30755 POINT(33.71441473968187 -118.21236673735203) bank30755 +30756 POINT(34.16598249875034 -118.19970865168241) bank30756 +30757 POINT(34.234153021812986 -117.85468866387639) bank30757 +30758 POINT(34.33558394006111 -117.79459214140272) bank30758 +30759 POINT(33.412343727958074 -119.06040484283905) bank30759 +30760 POINT(33.87801607413973 -118.9139408433641) bank30760 +30761 POINT(33.248163375651274 -118.15753245095792) bank30761 +30762 POINT(34.53701579871032 -118.79887558202807) bank30762 +30763 POINT(34.609420844422786 -118.52189959331727) bank30763 +30764 POINT(33.859127552787044 -119.05729291226308) bank30764 +30765 POINT(33.41569432143995 -117.37629569504927) bank30765 +30766 POINT(35.0109831974385 -117.25435813511999) bank30766 +30767 POINT(33.178538763916585 -118.47663241821353) bank30767 +30768 POINT(33.83689463946598 -118.48701367503517) bank30768 +30769 POINT(33.30574838415536 -117.95000482060294) bank30769 +30770 POINT(33.160716883226414 -118.47122502286805) bank30770 +30771 POINT(33.298110599045806 -119.22645298252797) bank30771 +30772 POINT(33.27344705813964 -117.7566891868295) bank30772 +30773 POINT(33.238984291741275 -118.1934006635184) bank30773 +30774 POINT(34.99055295951781 -119.1861375525194) bank30774 +30775 POINT(33.805802149326205 -118.30435453460613) bank30775 +30776 POINT(34.645207508635444 -118.97072217991537) bank30776 +30777 POINT(33.27413213036484 -118.50539270526183) bank30777 +30778 POINT(34.58659306533315 -117.78201703568183) bank30778 +30779 POINT(33.950525854125466 -118.85193377557395) bank30779 +30780 POINT(34.35831472146481 -118.5083686980297) bank30780 +30781 POINT(34.42335016583761 -118.5181898961672) bank30781 +30782 POINT(34.95491538049631 -117.80262719367228) bank30782 +30783 POINT(33.07155681086089 -119.01899260644072) bank30783 +30784 POINT(34.633958589268836 -117.92460599479996) bank30784 +30785 POINT(33.9993532818186 -118.03303019542419) bank30785 +30786 POINT(33.6362874119988 -117.58808667279705) bank30786 +30787 POINT(34.88405320401181 -117.60327798460547) bank30787 +30788 POINT(34.48627773881246 -118.77263268902674) bank30788 +30789 POINT(34.52229988820761 -119.07387480337107) bank30789 +30790 POINT(33.592514392161554 -118.24264922377733) bank30790 +30791 POINT(34.5053610556414 -118.3482529479693) bank30791 +30792 POINT(33.24686048083666 -117.52631544013641) bank30792 +30793 POINT(33.44806194534154 -118.78727014802708) bank30793 +30794 POINT(33.087065640259674 -117.34284544577098) bank30794 +30795 POINT(34.73860132740185 -117.68850148670272) bank30795 +30796 POINT(34.5262276097452 -117.65486428304679) bank30796 +30797 POINT(34.34076998449612 -118.13443702695523) bank30797 +30798 POINT(34.08879911683349 -118.15625353852465) bank30798 +30799 POINT(33.85157513475569 -118.4960040410287) bank30799 +30800 POINT(33.53259495693778 -117.91395830872325) bank30800 +30801 POINT(34.26677599337604 -118.60525422146996) bank30801 +30802 POINT(34.892897108310784 -118.12732485877984) bank30802 +30803 POINT(33.58020138425796 -118.45379370949136) bank30803 +30804 POINT(33.38532921236445 -118.40359826180514) bank30804 +30805 POINT(33.97191818059067 -119.14201110343444) bank30805 +30806 POINT(33.21510549198037 -117.84566091377907) bank30806 +30807 POINT(33.12327514215432 -117.56819592450644) bank30807 +30808 POINT(34.68629937364677 -117.35087953431098) bank30808 +30809 POINT(34.06548896883113 -118.27031187518264) bank30809 +30810 POINT(34.508999231450666 -119.127664092594) bank30810 +30811 POINT(33.210440690180626 -118.08449756447668) bank30811 +30812 POINT(33.73322989693564 -118.90692714894557) bank30812 +30813 POINT(33.88204319399061 -118.01628143804419) bank30813 +30814 POINT(33.79382906190204 -118.5990601104172) bank30814 +30815 POINT(34.67254149266153 -118.40393554494278) bank30815 +30816 POINT(34.09761468462051 -118.9084288208304) bank30816 +30817 POINT(34.397419282064114 -119.046434726994) bank30817 +30818 POINT(34.120560989493995 -118.53428260248441) bank30818 +30819 POINT(34.14974899170251 -117.57286956348277) bank30819 +30820 POINT(33.18406274207507 -118.83882938285203) bank30820 +30821 POINT(34.16342882686421 -118.05389193347071) bank30821 +30822 POINT(34.23682687240268 -118.51268768182092) bank30822 +30823 POINT(34.8137982191206 -118.84269670968453) bank30823 +30824 POINT(33.738107891771065 -118.2924192944332) bank30824 +30825 POINT(34.98451800956818 -117.33832180365353) bank30825 +30826 POINT(34.49231966574487 -118.37132947474822) bank30826 +30827 POINT(34.988283060462884 -118.33495462791119) bank30827 +30828 POINT(34.04510467668871 -119.07679860959192) bank30828 +30829 POINT(34.76706631845245 -117.8037317635108) bank30829 +30830 POINT(34.50673219199221 -118.32853409701424) bank30830 +30831 POINT(34.415711392740256 -119.09133230228848) bank30831 +30832 POINT(34.49762225488921 -117.43969643684727) bank30832 +30833 POINT(33.12297285118836 -119.08943635349881) bank30833 +30834 POINT(34.7515120344694 -118.77360570852782) bank30834 +30835 POINT(33.38058976495946 -118.62654915560776) bank30835 +30836 POINT(34.85149322424254 -117.32762886641125) bank30836 +30837 POINT(34.98612740062811 -117.72127431442028) bank30837 +30838 POINT(34.69131753684729 -118.697637783758) bank30838 +30839 POINT(33.2724304864668 -118.02208169883552) bank30839 +30840 POINT(34.65088183646284 -117.55297720676157) bank30840 +30841 POINT(34.08857353760064 -117.74540062746553) bank30841 +30842 POINT(33.548191668970695 -119.16378143851304) bank30842 +30843 POINT(34.210098054355 -118.3031295562837) bank30843 +30844 POINT(33.967390383967285 -117.53123429906788) bank30844 +30845 POINT(33.11217679362564 -118.82779130471458) bank30845 +30846 POINT(34.93255612213137 -117.25191445486855) bank30846 +30847 POINT(34.20621583668392 -118.99194252488724) bank30847 +30848 POINT(34.847096568067215 -118.69071024039665) bank30848 +30849 POINT(34.844565422927516 -118.68286607652117) bank30849 +30850 POINT(33.32541658908477 -118.18465712832784) bank30850 +30851 POINT(33.49109667575891 -119.18267328749025) bank30851 +30852 POINT(34.353974264023385 -118.50803033652728) bank30852 +30853 POINT(33.13811930889322 -118.86730460358406) bank30853 +30854 POINT(33.51276281615267 -118.7820217666847) bank30854 +30855 POINT(35.02656039030249 -117.81108463273281) bank30855 +30856 POINT(34.80875522776487 -118.33897552361042) bank30856 +30857 POINT(34.92054681574072 -117.35789505819567) bank30857 +30858 POINT(33.876770137736386 -118.64005405232845) bank30858 +30859 POINT(35.004218025777426 -118.94343655636868) bank30859 +30860 POINT(33.85009302530226 -118.69315594154946) bank30860 +30861 POINT(33.18426497255402 -118.81824177675787) bank30861 +30862 POINT(34.42197466509434 -118.44986161958528) bank30862 +30863 POINT(33.99892283662803 -118.05841482492251) bank30863 +30864 POINT(33.4254119257826 -118.30728002240521) bank30864 +30865 POINT(33.081213403348706 -119.09165377018266) bank30865 +30866 POINT(34.73756063793948 -118.76123741400428) bank30866 +30867 POINT(33.38354221558582 -118.94034087795022) bank30867 +30868 POINT(33.85946008503353 -118.09242194492272) bank30868 +30869 POINT(33.09193994337114 -118.64841902032764) bank30869 +30870 POINT(34.12677910970606 -118.74040921945952) bank30870 +30871 POINT(33.211140110980594 -117.38312746358629) bank30871 +30872 POINT(34.12891271703085 -118.16428560302387) bank30872 +30873 POINT(33.76367858612707 -118.76707066569865) bank30873 +30874 POINT(34.74256112569705 -119.20913141268471) bank30874 +30875 POINT(34.95900151510882 -119.19295905715846) bank30875 +30876 POINT(34.30024456597881 -119.16756861091821) bank30876 +30877 POINT(33.77775672918525 -117.8202855409172) bank30877 +30878 POINT(34.11233289232426 -118.87112539784684) bank30878 +30879 POINT(33.98871270355144 -117.29969212661841) bank30879 +30880 POINT(33.66297358489118 -117.92667971479729) bank30880 +30881 POINT(33.22748544502823 -118.34631578357578) bank30881 +30882 POINT(34.29662457782263 -117.48030313538575) bank30882 +30883 POINT(33.42866609437749 -118.4775696750501) bank30883 +30884 POINT(34.02905020963685 -118.77695467184863) bank30884 +30885 POINT(33.964425527449855 -118.63029265673552) bank30885 +30886 POINT(34.839944868110656 -118.47088848499375) bank30886 +30887 POINT(34.85112763059349 -118.45145744786949) bank30887 +30888 POINT(33.2198607054354 -117.67332674064426) bank30888 +30889 POINT(34.91587860182828 -117.33346232387349) bank30889 +30890 POINT(33.59439681794279 -119.00870564457635) bank30890 +30891 POINT(34.44053508068098 -118.66516970154471) bank30891 +30892 POINT(34.54976670472806 -117.92029353770656) bank30892 +30893 POINT(34.487543810513436 -117.90310605817838) bank30893 +30894 POINT(34.29485624720864 -117.63502083115353) bank30894 +30895 POINT(34.849844920862125 -118.65889104197696) bank30895 +30896 POINT(34.233240870716976 -118.45754075192202) bank30896 +30897 POINT(34.26083648552469 -117.4924313107432) bank30897 +30898 POINT(34.071530242484215 -118.16657470873736) bank30898 +30899 POINT(34.28110244641203 -118.07464284598768) bank30899 +30900 POINT(33.875280451984594 -117.53496330206099) bank30900 +30901 POINT(33.342684796985964 -118.1469257091103) bank30901 +30902 POINT(34.08797621126841 -117.34776873409402) bank30902 +30903 POINT(34.12813529816202 -119.21254819891162) bank30903 +30904 POINT(34.15656227631788 -117.2441727066171) bank30904 +30905 POINT(34.15581081925403 -118.41297816243707) bank30905 +30906 POINT(33.30266889028523 -118.00637488258494) bank30906 +30907 POINT(33.66759831001935 -118.47998984288796) bank30907 +30908 POINT(34.47495268457994 -118.6472403991808) bank30908 +30909 POINT(34.765057224121776 -118.36462017512417) bank30909 +30910 POINT(33.610457020321526 -118.26792123420074) bank30910 +30911 POINT(34.00025014620361 -117.32919953246494) bank30911 +30912 POINT(33.67173129515511 -118.03957456342454) bank30912 +30913 POINT(35.04031030101746 -118.32642896683609) bank30913 +30914 POINT(33.446955492638686 -118.65555174005722) bank30914 +30915 POINT(33.59897500954182 -118.62770566605494) bank30915 +30916 POINT(33.58028007330033 -117.90789103438503) bank30916 +30917 POINT(34.632962873127404 -119.00896549671538) bank30917 +30918 POINT(35.008163042240696 -118.38592209594856) bank30918 +30919 POINT(34.6529205164162 -117.71106266232121) bank30919 +30920 POINT(33.950568361963924 -118.83805028989855) bank30920 +30921 POINT(34.02657266083474 -117.94681360108572) bank30921 +30922 POINT(34.3779512510536 -118.58684603751388) bank30922 +30923 POINT(33.564464514529455 -118.93356262806813) bank30923 +30924 POINT(34.09689498822051 -118.60512706572958) bank30924 +30925 POINT(34.51031706780116 -117.45952210180869) bank30925 +30926 POINT(33.7199906644526 -118.52248847737093) bank30926 +30927 POINT(34.32720100305008 -118.15975553968308) bank30927 +30928 POINT(34.16897802480891 -118.41037411951233) bank30928 +30929 POINT(34.61965133921061 -118.39887265252243) bank30929 +30930 POINT(33.91468059930102 -118.30939460251383) bank30930 +30931 POINT(33.14440186226857 -118.35062306102704) bank30931 +30932 POINT(33.6536480768578 -117.81384315986266) bank30932 +30933 POINT(34.00149393329729 -117.27141546896918) bank30933 +30934 POINT(33.36439801610878 -117.68119933305866) bank30934 +30935 POINT(33.6557084698525 -119.11327098824165) bank30935 +30936 POINT(33.625867998858304 -119.23179595662329) bank30936 +30937 POINT(33.917204145359285 -117.74604353319629) bank30937 +30938 POINT(34.661770820949116 -118.37602140098188) bank30938 +30939 POINT(33.81294205822151 -119.01012299483835) bank30939 +30940 POINT(34.094859387331795 -118.18643601223809) bank30940 +30941 POINT(34.0306564933161 -118.22441514056794) bank30941 +30942 POINT(33.42889372648833 -117.95514181653766) bank30942 +30943 POINT(34.23547933422046 -117.58012952659428) bank30943 +30944 POINT(33.15630350242074 -119.0239057802446) bank30944 +30945 POINT(34.650881488870986 -118.01052220660027) bank30945 +30946 POINT(35.00171747650261 -118.56060653077365) bank30946 +30947 POINT(34.115040011110835 -118.7613123353029) bank30947 +30948 POINT(33.731428923510755 -118.62878957474476) bank30948 +30949 POINT(34.04596758225552 -118.48646351702452) bank30949 +30950 POINT(33.75719734072298 -118.80983961602344) bank30950 +30951 POINT(34.64955742553966 -118.7656366656464) bank30951 +30952 POINT(34.14200744993809 -117.57319503284366) bank30952 +30953 POINT(34.725270993688895 -118.19776241436446) bank30953 +30954 POINT(34.4893814292426 -118.04997049292113) bank30954 +30955 POINT(33.329822905867516 -118.90053097545456) bank30955 +30956 POINT(33.31009653515821 -117.58058428388803) bank30956 +30957 POINT(34.41475331804723 -118.47052430717879) bank30957 +30958 POINT(34.31585997163324 -117.61917043836695) bank30958 +30959 POINT(34.09573784225974 -118.76747807890615) bank30959 +30960 POINT(34.221817772600815 -117.3844005101176) bank30960 +30961 POINT(34.24243313639472 -117.3191355563864) bank30961 +30962 POINT(34.90756258115321 -118.8154975716367) bank30962 +30963 POINT(34.650588595310225 -117.59305872704911) bank30963 +30964 POINT(34.065510377608305 -119.00211697192115) bank30964 +30965 POINT(33.3612656750214 -117.5916786238396) bank30965 +30966 POINT(33.31658642000046 -118.07854422551702) bank30966 +30967 POINT(34.959227533355794 -119.15071394400687) bank30967 +30968 POINT(34.63462578028544 -118.09838916873525) bank30968 +30969 POINT(34.246109065037984 -117.26568559662142) bank30969 +30970 POINT(33.9898862253549 -117.7315301125921) bank30970 +30971 POINT(34.680211011308224 -117.75498257785569) bank30971 +30972 POINT(34.451232203935824 -117.56012936056162) bank30972 +30973 POINT(34.33510789085453 -117.62945747036703) bank30973 +30974 POINT(34.28781968227157 -117.49707350262345) bank30974 +30975 POINT(33.80065458444241 -118.87311958657185) bank30975 +30976 POINT(34.21438985419987 -118.10796622589402) bank30976 +30977 POINT(33.59272187128269 -117.95074844770775) bank30977 +30978 POINT(34.41491950694566 -119.13218395402902) bank30978 +30979 POINT(34.85025879388401 -117.38775780435786) bank30979 +30980 POINT(34.27704504972008 -117.54046406612312) bank30980 +30981 POINT(34.14600020083345 -118.47594401085509) bank30981 +30982 POINT(33.75528883789409 -117.38305103698511) bank30982 +30983 POINT(33.42509294717768 -117.41849504440357) bank30983 +30984 POINT(33.32847749437185 -117.35541027029122) bank30984 +30985 POINT(33.542006978524626 -118.39543347270683) bank30985 +30986 POINT(33.83431318253036 -118.53868756910873) bank30986 +30987 POINT(33.203817950497225 -117.72147855380162) bank30987 +30988 POINT(34.36163078873951 -118.81389377794737) bank30988 +30989 POINT(34.8281690406432 -118.49164602067) bank30989 +30990 POINT(34.6621918086549 -118.88038758118141) bank30990 +30991 POINT(34.9823121426281 -118.45929183561142) bank30991 +30992 POINT(34.67449218744455 -118.92853375365988) bank30992 +30993 POINT(33.3000363430408 -117.5231167454579) bank30993 +30994 POINT(33.401892079689084 -117.8733791654615) bank30994 +30995 POINT(33.66392328726521 -119.00587610850565) bank30995 +30996 POINT(34.21519943645503 -119.12618012776012) bank30996 +30997 POINT(34.94913136773943 -118.7187408984468) bank30997 +30998 POINT(33.79563689642799 -119.22935652526832) bank30998 +30999 POINT(34.591209148606026 -118.88384390200966) bank30999 +31000 POINT(33.144966948195204 -117.31916449284616) bank31000 +31001 POINT(34.447815736789245 -118.4339354538455) bank31001 +31002 POINT(33.91416147125069 -117.9005964380353) bank31002 +31003 POINT(33.469357886152665 -118.10504118368644) bank31003 +31004 POINT(33.99020526638446 -117.50823818014477) bank31004 +31005 POINT(34.435927958068234 -117.50341007136453) bank31005 +31006 POINT(34.079921674301026 -118.54203395668323) bank31006 +31007 POINT(33.7795720509337 -119.20801307710694) bank31007 +31008 POINT(33.65502045157585 -118.26557887933183) bank31008 +31009 POINT(34.26292518215407 -118.17839028702267) bank31009 +31010 POINT(33.97826572650722 -118.13949949753341) bank31010 +31011 POINT(34.76123798299477 -117.41487409792101) bank31011 +31012 POINT(35.00550632433026 -118.50234169212376) bank31012 +31013 POINT(34.50460939704236 -118.37240802500615) bank31013 +31014 POINT(33.1397653928998 -118.17043084136111) bank31014 +31015 POINT(33.31799873179331 -119.21757623550498) bank31015 +31016 POINT(34.048528547619085 -118.7191276851771) bank31016 +31017 POINT(33.53937095364791 -118.52703722664336) bank31017 +31018 POINT(33.52759708147192 -117.30297709940547) bank31018 +31019 POINT(33.7768617674303 -117.58285035765887) bank31019 +31020 POINT(33.4667187213564 -118.16350771605921) bank31020 +31021 POINT(33.70296063288031 -117.31050337458137) bank31021 +31022 POINT(33.84544998465995 -117.8072129856279) bank31022 +31023 POINT(34.771626567158876 -118.07156502771566) bank31023 +31024 POINT(33.928080428136134 -118.41093924177322) bank31024 +31025 POINT(34.93995408678221 -117.65544234198887) bank31025 +31026 POINT(33.53513042564132 -118.84448116548708) bank31026 +31027 POINT(33.471435687139085 -118.84462228394759) bank31027 +31028 POINT(33.558598773878174 -117.41267286972757) bank31028 +31029 POINT(34.46080413341156 -118.75871007226291) bank31029 +31030 POINT(34.07525394051758 -118.89420218641489) bank31030 +31031 POINT(34.24501731598404 -118.11732393848938) bank31031 +31032 POINT(34.9092464127275 -118.47609099488949) bank31032 +31033 POINT(34.38747260016413 -118.54030530390857) bank31033 +31034 POINT(33.18769380037612 -117.67219554954329) bank31034 +31035 POINT(33.56144214134966 -118.95022819648155) bank31035 +31036 POINT(34.80872815284696 -117.29434155524457) bank31036 +31037 POINT(34.436777643798095 -117.91991108958317) bank31037 +31038 POINT(34.642048051510635 -118.31220905052007) bank31038 +31039 POINT(34.1484699813701 -117.26456240846697) bank31039 +31040 POINT(33.56833468756277 -119.13188980332157) bank31040 +31041 POINT(34.25708602897751 -117.47891500834501) bank31041 +31042 POINT(33.66500067434732 -119.00487664184557) bank31042 +31043 POINT(34.63085651868716 -118.54739458766016) bank31043 +31044 POINT(34.58003780622568 -118.37410651803724) bank31044 +31045 POINT(34.016406512399286 -117.93531837696699) bank31045 +31046 POINT(34.141887641610566 -117.63638582922404) bank31046 +31047 POINT(34.461027974901846 -117.45951055657947) bank31047 +31048 POINT(34.37133012986565 -118.87484628388546) bank31048 +31049 POINT(34.10082278780216 -117.55540128734859) bank31049 +31050 POINT(34.059826446385806 -119.05909109002837) bank31050 +31051 POINT(34.229081210135796 -118.96133818368673) bank31051 +31052 POINT(33.81844581373347 -117.28065245529591) bank31052 +31053 POINT(33.394111909667 -118.76790649471249) bank31053 +31054 POINT(33.65838592575879 -118.48379208322739) bank31054 +31055 POINT(34.40495859538487 -118.04778015920269) bank31055 +31056 POINT(34.77169597526694 -117.45532485817799) bank31056 +31057 POINT(33.20367366354372 -118.79650346261181) bank31057 +31058 POINT(34.26716977496927 -117.66111702993304) bank31058 +31059 POINT(34.23255486029323 -117.80930445653028) bank31059 +31060 POINT(33.8596212318123 -118.67232908737915) bank31060 +31061 POINT(35.01306386112722 -118.34213810059627) bank31061 +31062 POINT(33.814674620915184 -118.02734352460654) bank31062 +31063 POINT(34.479733141900354 -118.49074664916762) bank31063 +31064 POINT(33.07958168427691 -118.24366183323355) bank31064 +31065 POINT(33.93581590780909 -118.66944342874231) bank31065 +31066 POINT(33.21463023222318 -118.59688530691139) bank31066 +31067 POINT(33.301120239356365 -118.67420494590976) bank31067 +31068 POINT(34.72680119934038 -119.19872031272006) bank31068 +31069 POINT(34.52761917769524 -117.84830659881561) bank31069 +31070 POINT(33.12751090085418 -117.31107564907957) bank31070 +31071 POINT(34.7604402972394 -118.01136200725854) bank31071 +31072 POINT(34.5579670356922 -119.01966126450029) bank31072 +31073 POINT(33.838561539757094 -118.99775906805934) bank31073 +31074 POINT(34.77040408570403 -117.90518858736611) bank31074 +31075 POINT(33.31683087884004 -118.16614701732544) bank31075 +31076 POINT(33.477998697547505 -117.8648383056144) bank31076 +31077 POINT(34.8283885596329 -118.44750282025653) bank31077 +31078 POINT(33.23032677240424 -117.35107039255263) bank31078 +31079 POINT(33.901725630427265 -118.16190031143854) bank31079 +31080 POINT(33.58618620122793 -117.8931700199644) bank31080 +31081 POINT(34.73416763310227 -117.34761751027087) bank31081 +31082 POINT(34.25495456885771 -117.75266523853374) bank31082 +31083 POINT(34.9817048529638 -119.1930332655382) bank31083 +31084 POINT(33.141240870980866 -117.78492106752772) bank31084 +31085 POINT(34.525865896577 -117.41483167022537) bank31085 +31086 POINT(34.23771407896046 -117.77422456998666) bank31086 +31087 POINT(33.340164891073805 -117.37511659502516) bank31087 +31088 POINT(33.228741564463256 -117.44122699256987) bank31088 +31089 POINT(33.054986361375875 -118.37106090205423) bank31089 +31090 POINT(34.33765368455148 -117.24820332767855) bank31090 +31091 POINT(33.89551728728343 -118.88157654981656) bank31091 +31092 POINT(34.74400628820698 -117.28718293070149) bank31092 +31093 POINT(33.324378787935665 -119.06874801609582) bank31093 +31094 POINT(33.456811253567025 -117.54410881246326) bank31094 +31095 POINT(33.36631849200206 -118.74587281960208) bank31095 +31096 POINT(34.698193138067815 -118.356168129743) bank31096 +31097 POINT(33.536075736604566 -117.4616372312915) bank31097 +31098 POINT(34.06254969113815 -117.52281953108195) bank31098 +31099 POINT(33.4536798105423 -118.31578543595408) bank31099 +31100 POINT(34.12978485233912 -117.90753897839393) bank31100 +31101 POINT(34.84279170400673 -119.03920850646195) bank31101 +31102 POINT(34.291392143680355 -118.70638389655089) bank31102 +31103 POINT(33.36151897055685 -117.50378433806647) bank31103 +31104 POINT(33.616891148488804 -118.9900086981786) bank31104 +31105 POINT(33.09012208446566 -119.15464537620623) bank31105 +31106 POINT(34.87682116551943 -118.95448514275084) bank31106 +31107 POINT(33.209611222352905 -118.82543202612608) bank31107 +31108 POINT(33.21807508707292 -117.45741623773293) bank31108 +31109 POINT(33.45199050430151 -117.63886480795078) bank31109 +31110 POINT(34.82450269121153 -117.74658306889556) bank31110 +31111 POINT(34.43871491194512 -117.85203201956749) bank31111 +31112 POINT(33.47435627193184 -118.94619336066572) bank31112 +31113 POINT(34.70547588412574 -117.50039594494243) bank31113 +31114 POINT(34.05966097124506 -118.95732173872753) bank31114 +31115 POINT(33.825512532799166 -117.81412606322873) bank31115 +31116 POINT(33.15565393575974 -117.74107375844437) bank31116 +31117 POINT(33.77209873398206 -117.83989514384054) bank31117 +31118 POINT(33.950860474789415 -117.55306899260609) bank31118 +31119 POINT(33.34873478882716 -118.9369348898083) bank31119 +31120 POINT(33.500750494979584 -117.39616083350195) bank31120 +31121 POINT(33.786234187694184 -118.96311417268208) bank31121 +31122 POINT(33.76286366778863 -117.89426520945784) bank31122 +31123 POINT(34.23414544742394 -117.43196047793974) bank31123 +31124 POINT(34.78187251282978 -118.44948992569454) bank31124 +31125 POINT(34.38082653265579 -117.34473426116585) bank31125 +31126 POINT(33.65492141598127 -117.97538736988844) bank31126 +31127 POINT(34.04102285362868 -119.01242058373174) bank31127 +31128 POINT(33.622331959258496 -118.35841719302628) bank31128 +31129 POINT(33.356885031755795 -118.55595769422804) bank31129 +31130 POINT(33.08772887619585 -117.54868802583616) bank31130 +31131 POINT(34.73617661845394 -119.21907443008219) bank31131 +31132 POINT(34.01576300188426 -118.7882137408719) bank31132 +31133 POINT(33.75641242747006 -118.10287166074522) bank31133 +31134 POINT(34.652291632099804 -117.31600507943224) bank31134 +31135 POINT(33.40958612094913 -117.45070864558625) bank31135 +31136 POINT(33.97451116608726 -118.89788694197739) bank31136 +31137 POINT(34.378739262554475 -118.85867621804597) bank31137 +31138 POINT(33.377861326730894 -118.767950029875) bank31138 +31139 POINT(34.97244184060031 -118.67524696923576) bank31139 +31140 POINT(34.69661483962362 -119.2004373108991) bank31140 +31141 POINT(34.86922246291425 -118.93691492889921) bank31141 +31142 POINT(33.77388520607413 -117.4063017827911) bank31142 +31143 POINT(34.586593142953674 -118.36665552242371) bank31143 +31144 POINT(33.61670377164711 -118.46932061608884) bank31144 +31145 POINT(34.67312501180112 -118.04203450439377) bank31145 +31146 POINT(33.16256092629549 -119.10178610102278) bank31146 +31147 POINT(33.45317747443298 -117.7470958597158) bank31147 +31148 POINT(34.81300752494104 -118.83768953753287) bank31148 +31149 POINT(34.23557790234011 -118.94619978911363) bank31149 +31150 POINT(34.11458756129552 -118.97831708727462) bank31150 +31151 POINT(34.662545976661676 -117.79722684330939) bank31151 +31152 POINT(33.21189663665643 -117.92673948072454) bank31152 +31153 POINT(34.44369211928054 -118.41684824618945) bank31153 +31154 POINT(34.57058898321347 -118.4917467151364) bank31154 +31155 POINT(33.96872494112507 -118.97266161844487) bank31155 +31156 POINT(34.399534171571126 -117.52997824994598) bank31156 +31157 POINT(34.73598982913681 -119.22886430310479) bank31157 +31158 POINT(34.25493949502916 -118.38119909170915) bank31158 +31159 POINT(33.68339766827547 -118.17236081487222) bank31159 +31160 POINT(33.523805176679254 -119.19277817101931) bank31160 +31161 POINT(33.48588399205032 -117.35917240973397) bank31161 +31162 POINT(33.17289493375579 -118.44671492442916) bank31162 +31163 POINT(34.745211519730944 -118.37959542215646) bank31163 +31164 POINT(34.829901306543846 -118.13837296442075) bank31164 +31165 POINT(34.19942858692159 -117.3008115893098) bank31165 +31166 POINT(33.532525989163915 -118.89125533092249) bank31166 +31167 POINT(34.78750876676031 -118.4115783063555) bank31167 +31168 POINT(33.05305218388243 -118.86829729762876) bank31168 +31169 POINT(34.897962885077156 -117.25598722414279) bank31169 +31170 POINT(33.33641125033927 -118.54723988888014) bank31170 +31171 POINT(33.76654638522536 -118.43797339962336) bank31171 +31172 POINT(33.628887683115636 -117.37544804973581) bank31172 +31173 POINT(33.738187866403464 -118.01240366377773) bank31173 +31174 POINT(33.83962798227212 -117.64362426016731) bank31174 +31175 POINT(33.143840777665005 -119.02053835742312) bank31175 +31176 POINT(33.22284768310238 -118.54830505762119) bank31176 +31177 POINT(35.00125551135435 -118.10966806457002) bank31177 +31178 POINT(34.306755922090744 -117.5230065528581) bank31178 +31179 POINT(33.41404606666587 -117.25223164056845) bank31179 +31180 POINT(33.22116200436229 -118.53701699564871) bank31180 +31181 POINT(33.29830820266328 -117.76984828183112) bank31181 +31182 POINT(34.1061391969134 -119.18664073478229) bank31182 +31183 POINT(34.651690781674304 -119.07116431076103) bank31183 +31184 POINT(34.832563787252056 -117.478274644027) bank31184 +31185 POINT(33.886198022142786 -118.5288194958206) bank31185 +31186 POINT(34.68069558868327 -117.91425938858927) bank31186 +31187 POINT(33.760957415439144 -119.1573247227778) bank31187 +31188 POINT(34.13376471461761 -118.82190356473897) bank31188 +31189 POINT(33.319075786420555 -117.64530684163873) bank31189 +31190 POINT(33.547885559867055 -117.83249760816518) bank31190 +31191 POINT(33.85773050224502 -117.30868352959469) bank31191 +31192 POINT(33.10454896607671 -118.96201584944703) bank31192 +31193 POINT(33.29470345588901 -118.41340405851956) bank31193 +31194 POINT(34.361384439579304 -117.43340607289839) bank31194 +31195 POINT(34.852023805317714 -117.80097317564416) bank31195 +31196 POINT(33.22698946970818 -118.58799839405877) bank31196 +31197 POINT(34.291796656331066 -117.4958695667857) bank31197 +31198 POINT(33.08914623290754 -119.01429997372824) bank31198 +31199 POINT(33.67236219887505 -118.52576148662135) bank31199 +31200 POINT(33.06819421540594 -117.26720619746453) bank31200 +31201 POINT(34.4789833867339 -118.05289022644025) bank31201 +31202 POINT(33.333039445924285 -118.43977881103625) bank31202 +31203 POINT(33.227309644356545 -118.70703989732354) bank31203 +31204 POINT(35.02556563420322 -119.00807001636794) bank31204 +31205 POINT(34.846069609342855 -118.71747350164254) bank31205 +31206 POINT(34.36787115740179 -118.89903442892171) bank31206 +31207 POINT(33.78058713438741 -117.55233989877283) bank31207 +31208 POINT(34.19893930001423 -119.05035836825711) bank31208 +31209 POINT(34.348634010381836 -119.17076008660321) bank31209 +31210 POINT(34.46887304142274 -117.70273186564356) bank31210 +31211 POINT(33.640377239455496 -117.35658294793471) bank31211 +31212 POINT(34.59197202329025 -117.380297170249) bank31212 +31213 POINT(34.70958710446338 -118.18128187647383) bank31213 +31214 POINT(33.557557899225124 -117.750379731553) bank31214 +31215 POINT(34.7486529725316 -118.85193636057451) bank31215 +31216 POINT(33.05389797214831 -118.9015338626174) bank31216 +31217 POINT(33.34412308297525 -119.15991328978853) bank31217 +31218 POINT(33.23846449846631 -117.68329491994396) bank31218 +31219 POINT(34.25688300977541 -118.55552983973102) bank31219 +31220 POINT(34.31231414388164 -118.74325208718598) bank31220 +31221 POINT(34.693749535061905 -118.59749631957679) bank31221 +31222 POINT(34.93664235034492 -118.68247470632525) bank31222 +31223 POINT(34.857947099923436 -118.78073245636303) bank31223 +31224 POINT(34.58729906946616 -117.98946267982473) bank31224 +31225 POINT(33.07447945414118 -119.18096716214663) bank31225 +31226 POINT(34.56072315771978 -117.89880562554683) bank31226 +31227 POINT(34.483709281196525 -117.9383262382068) bank31227 +31228 POINT(33.211158222615126 -118.839490819574) bank31228 +31229 POINT(33.79764532863503 -118.11300200326751) bank31229 +31230 POINT(34.3898531800988 -117.33673771478591) bank31230 +31231 POINT(33.4191451733771 -118.3437467752358) bank31231 +31232 POINT(33.34846970190225 -118.21498515111436) bank31232 +31233 POINT(34.46794523941789 -119.1947768139635) bank31233 +31234 POINT(33.56220892235459 -118.63159467228026) bank31234 +31235 POINT(34.0766136302128 -118.75584160879444) bank31235 +31236 POINT(34.06714480734874 -117.60698566164211) bank31236 +31237 POINT(33.60276331331434 -117.34669139125681) bank31237 +31238 POINT(34.16804255097697 -118.61385279863492) bank31238 +31239 POINT(34.935737076769385 -118.17506150134137) bank31239 +31240 POINT(34.037413218987595 -118.5783570037683) bank31240 +31241 POINT(33.34979453857813 -118.12805923056598) bank31241 +31242 POINT(33.83778927663339 -118.48034263033858) bank31242 +31243 POINT(34.75100860934044 -118.72411833910549) bank31243 +31244 POINT(34.71906509620242 -117.86212112651282) bank31244 +31245 POINT(34.38843479120413 -117.5151569377395) bank31245 +31246 POINT(34.13848674407668 -117.32017242524682) bank31246 +31247 POINT(34.16118078171052 -118.99734639192829) bank31247 +31248 POINT(33.77794121285793 -117.37149519919227) bank31248 +31249 POINT(34.46591818137497 -119.08022424394633) bank31249 +31250 POINT(34.92767839370649 -117.69441586853416) bank31250 +31251 POINT(33.737795601052866 -118.63018289560318) bank31251 +31252 POINT(34.811647121916586 -118.25238709787538) bank31252 +31253 POINT(33.88366519180274 -119.06265607040615) bank31253 +31254 POINT(33.434894368266086 -118.1543037652861) bank31254 +31255 POINT(33.28457886892462 -117.40950145917063) bank31255 +31256 POINT(34.55371392190478 -117.26616684694726) bank31256 +31257 POINT(34.10787735316226 -118.8756628872209) bank31257 +31258 POINT(34.573450603525785 -117.38938500706118) bank31258 +31259 POINT(33.41387788919686 -117.93438769529622) bank31259 +31260 POINT(34.52176211683955 -119.13239151908527) bank31260 +31261 POINT(34.05678323682826 -117.2681614693382) bank31261 +31262 POINT(33.223497551970084 -118.37253662964056) bank31262 +31263 POINT(33.74344577991504 -119.04696823146935) bank31263 +31264 POINT(34.70933254863167 -119.12862637154646) bank31264 +31265 POINT(34.41744996067188 -118.80870814336146) bank31265 +31266 POINT(34.84752491820755 -117.25063761669685) bank31266 +31267 POINT(33.75463470608194 -117.56583879750721) bank31267 +31268 POINT(33.63128526126835 -118.16663995134192) bank31268 +31269 POINT(33.17396159735049 -118.68833369559893) bank31269 +31270 POINT(33.50311971654826 -117.72694227553256) bank31270 +31271 POINT(33.76944940549185 -118.87682898606958) bank31271 +31272 POINT(33.753112503536435 -118.29941108234016) bank31272 +31273 POINT(33.42090771173333 -118.80129495319898) bank31273 +31274 POINT(33.38526187583658 -117.97196726264443) bank31274 +31275 POINT(33.48628390800757 -118.00394287839505) bank31275 +31276 POINT(33.66319381570668 -118.83695418604812) bank31276 +31277 POINT(33.80914931664685 -118.38221596243372) bank31277 +31278 POINT(33.613978372730074 -117.55353322295477) bank31278 +31279 POINT(33.60931652333438 -117.5301908133763) bank31279 +31280 POINT(33.81535363556575 -118.88470240252123) bank31280 +31281 POINT(34.05248966412103 -117.50269182449189) bank31281 +31282 POINT(34.5630473748419 -118.28351066012956) bank31282 +31283 POINT(34.93512277249394 -118.40069518016568) bank31283 +31284 POINT(34.266156305279004 -119.13353277352836) bank31284 +31285 POINT(33.24423351195936 -118.38649618416692) bank31285 +31286 POINT(34.113683408476795 -117.73835116507611) bank31286 +31287 POINT(33.06285055620054 -118.17259563480985) bank31287 +31288 POINT(34.99384159226769 -118.78701598294133) bank31288 +31289 POINT(33.40185106742149 -118.69463026871196) bank31289 +31290 POINT(33.177906625537005 -117.88819152216638) bank31290 +31291 POINT(34.528213818353734 -117.34433527114001) bank31291 +31292 POINT(34.01100075316623 -117.68868651824141) bank31292 +31293 POINT(33.73525752817452 -118.21135099710709) bank31293 +31294 POINT(34.29759422272682 -118.54670606599566) bank31294 +31295 POINT(33.835695618528355 -117.33497827062845) bank31295 +31296 POINT(33.90909212955416 -118.85614592563842) bank31296 +31297 POINT(34.96648992428347 -117.49025991459943) bank31297 +31298 POINT(34.337923292316084 -118.89962543109073) bank31298 +31299 POINT(33.948530530039044 -117.63148598645415) bank31299 +31300 POINT(33.12616743633316 -117.86393146303783) bank31300 +31301 POINT(33.056152219622426 -117.968225874517) bank31301 +31302 POINT(34.52456937234292 -118.89780559931502) bank31302 +31303 POINT(33.59494062832465 -117.36255351095315) bank31303 +31304 POINT(33.20907509348108 -119.06017122247994) bank31304 +31305 POINT(34.94915247603071 -118.5928991326589) bank31305 +31306 POINT(33.17546946472415 -117.98115459912691) bank31306 +31307 POINT(34.888285631385486 -117.34158958685602) bank31307 +31308 POINT(34.73636077132313 -118.2922708118865) bank31308 +31309 POINT(33.45800164816445 -118.03362308295014) bank31309 +31310 POINT(34.467254248736126 -118.0285879364621) bank31310 +31311 POINT(34.34511904715525 -118.19339265291836) bank31311 +31312 POINT(34.671540957108306 -117.49384719585468) bank31312 +31313 POINT(34.092559731085544 -117.42133976457056) bank31313 +31314 POINT(33.50076719073467 -117.50022932783592) bank31314 +31315 POINT(34.09761970684295 -117.86858526619697) bank31315 +31316 POINT(33.92741638761973 -118.51712177705737) bank31316 +31317 POINT(33.06970042823211 -118.02520549915242) bank31317 +31318 POINT(33.99976320024893 -118.70790184987646) bank31318 +31319 POINT(33.577505366026614 -117.37720846416693) bank31319 +31320 POINT(33.39643079159814 -117.95652522627985) bank31320 +31321 POINT(34.40952170435092 -118.944040333182) bank31321 +31322 POINT(33.47038011817308 -118.57775292529833) bank31322 +31323 POINT(34.892413542162885 -118.6817535657875) bank31323 +31324 POINT(34.83007244354037 -119.21698720168861) bank31324 +31325 POINT(34.71484443296215 -118.65632750021517) bank31325 +31326 POINT(33.9364915226169 -119.1074430047554) bank31326 +31327 POINT(34.56556670807753 -118.70286941561801) bank31327 +31328 POINT(34.3383128801403 -117.87757550176651) bank31328 +31329 POINT(35.046333645292485 -117.81275625618461) bank31329 +31330 POINT(34.18809556051406 -117.52959819348365) bank31330 +31331 POINT(33.81021521899835 -119.15791091955946) bank31331 +31332 POINT(34.11483607018476 -117.80708120970719) bank31332 +31333 POINT(33.14508229740752 -117.35226238127905) bank31333 +31334 POINT(35.01166419989978 -118.22648005103173) bank31334 +31335 POINT(33.47586503895285 -118.69275190366662) bank31335 +31336 POINT(34.91331235073513 -118.65056807672256) bank31336 +31337 POINT(33.067737189270034 -119.11896910508212) bank31337 +31338 POINT(34.7259710351753 -118.69387555235666) bank31338 +31339 POINT(33.159151035523266 -119.0575481656632) bank31339 +31340 POINT(35.00498092500114 -118.52603986391821) bank31340 +31341 POINT(34.61640042732254 -118.05502480928402) bank31341 +31342 POINT(33.56722971150593 -118.28327343494153) bank31342 +31343 POINT(34.81645898853866 -117.51016310894504) bank31343 +31344 POINT(33.75418734227164 -117.3713790040265) bank31344 +31345 POINT(33.319361588292374 -118.78962602926941) bank31345 +31346 POINT(34.07291159586062 -118.69513181560187) bank31346 +31347 POINT(33.428765751456986 -118.81408743034518) bank31347 +31348 POINT(33.15600721547602 -117.3972382990202) bank31348 +31349 POINT(33.93969889085155 -118.876931796259) bank31349 +31350 POINT(33.70690514374544 -118.78534363444923) bank31350 +31351 POINT(34.32304991565921 -118.04492344968872) bank31351 +31352 POINT(33.21034592123031 -118.49113220838413) bank31352 +31353 POINT(34.49132911409471 -119.13931607759991) bank31353 +31354 POINT(33.973260403156985 -117.63872235842791) bank31354 +31355 POINT(34.11433795858802 -118.62369327857681) bank31355 +31356 POINT(34.36561228553783 -119.05475976662166) bank31356 +31357 POINT(34.18357341214573 -118.71083694368758) bank31357 +31358 POINT(34.410823498690526 -118.95339191281897) bank31358 +31359 POINT(33.344424167019454 -117.97500378779856) bank31359 +31360 POINT(33.272481548872705 -118.9001255599656) bank31360 +31361 POINT(34.680769605154325 -117.77378034351274) bank31361 +31362 POINT(34.84218429966036 -118.59546717340301) bank31362 +31363 POINT(33.54163648978439 -118.45482637911117) bank31363 +31364 POINT(34.85574910311379 -119.15769665693662) bank31364 +31365 POINT(34.1474674682566 -118.94780838315391) bank31365 +31366 POINT(34.353417694726396 -118.89431970489159) bank31366 +31367 POINT(33.92692587486584 -119.06823365150859) bank31367 +31368 POINT(33.78954016604648 -118.41961616220883) bank31368 +31369 POINT(33.225084722206056 -117.82873608118378) bank31369 +31370 POINT(33.91446436187026 -119.09510299244947) bank31370 +31371 POINT(33.89436541721998 -118.27336785026161) bank31371 +31372 POINT(34.613008783381964 -118.4208590018126) bank31372 +31373 POINT(33.42600006587769 -117.35810378630444) bank31373 +31374 POINT(35.020799359527295 -117.25652096115503) bank31374 +31375 POINT(33.16582232234488 -118.84905848995234) bank31375 +31376 POINT(34.9266182721912 -117.81504000587039) bank31376 +31377 POINT(33.47377149135053 -118.58481200212233) bank31377 +31378 POINT(34.14982275563626 -118.98405519124391) bank31378 +31379 POINT(34.66314476444765 -119.05874591262491) bank31379 +31380 POINT(34.933284085591964 -117.4711194259112) bank31380 +31381 POINT(34.53695402097504 -117.85693146838882) bank31381 +31382 POINT(34.09681991200733 -118.13281728677498) bank31382 +31383 POINT(33.071341034218655 -118.03975139370239) bank31383 +31384 POINT(34.473428852288656 -118.86280694245069) bank31384 +31385 POINT(33.425171506028676 -118.55906651836271) bank31385 +31386 POINT(34.230517769220334 -117.32194720076156) bank31386 +31387 POINT(34.007299557591146 -118.24500653662791) bank31387 +31388 POINT(33.88567956519196 -118.91505619726176) bank31388 +31389 POINT(34.037972434413014 -119.22198136088552) bank31389 +31390 POINT(34.26883661845174 -118.92689220082558) bank31390 +31391 POINT(33.95227178721457 -118.35376984007017) bank31391 +31392 POINT(33.66569678714044 -119.01620749619354) bank31392 +31393 POINT(35.01123694644819 -117.78187748987604) bank31393 +31394 POINT(34.78856890038415 -118.47461006837311) bank31394 +31395 POINT(34.77179951486872 -118.2832226558851) bank31395 +31396 POINT(34.83189716094097 -117.47320478477205) bank31396 +31397 POINT(34.32788799973804 -118.34824773330185) bank31397 +31398 POINT(33.45323454671821 -118.55920235184946) bank31398 +31399 POINT(33.32166084357123 -117.95086838067283) bank31399 +31400 POINT(33.09988308739856 -119.07123993828513) bank31400 +31401 POINT(33.27793017203217 -117.90783012726514) bank31401 +31402 POINT(33.8145830040285 -118.33425706606687) bank31402 +31403 POINT(34.8226239743391 -117.33678333367844) bank31403 +31404 POINT(33.30728873038775 -117.69255429497225) bank31404 +31405 POINT(33.439588404432435 -118.41914919124386) bank31405 +31406 POINT(34.29717861520564 -117.89481956404866) bank31406 +31407 POINT(34.00936410034012 -119.1381597532966) bank31407 +31408 POINT(33.21109595106656 -117.32777314199755) bank31408 +31409 POINT(33.37887474262907 -118.74619667998041) bank31409 +31410 POINT(33.23053532917808 -117.74200405703124) bank31410 +31411 POINT(33.46173648230168 -118.4310797721374) bank31411 +31412 POINT(33.59071835754909 -118.70540205504199) bank31412 +31413 POINT(33.0797900811789 -118.03787942245486) bank31413 +31414 POINT(33.11364106716847 -117.63207402673548) bank31414 +31415 POINT(34.63109753884571 -117.25470929864757) bank31415 +31416 POINT(34.15562744163768 -117.76481744531388) bank31416 +31417 POINT(34.956973965013745 -117.27411727658324) bank31417 +31418 POINT(34.08677003615535 -119.14902726682823) bank31418 +31419 POINT(33.19076582929346 -118.59044696161241) bank31419 +31420 POINT(34.98190734730525 -118.55678319340457) bank31420 +31421 POINT(33.486495572039146 -117.73466927575045) bank31421 +31422 POINT(33.28752457669025 -118.85672734705108) bank31422 +31423 POINT(34.47213144009132 -119.11483186607552) bank31423 +31424 POINT(34.893979635692816 -119.15478388778494) bank31424 +31425 POINT(33.81858014220471 -118.44701714466802) bank31425 +31426 POINT(34.84125384665147 -117.4716923128094) bank31426 +31427 POINT(33.20103130740581 -117.36284309577215) bank31427 +31428 POINT(33.876829653065066 -117.9309440606326) bank31428 +31429 POINT(33.076394100101396 -119.20300124160606) bank31429 +31430 POINT(34.011287672808805 -117.57349036592606) bank31430 +31431 POINT(35.02505216591498 -118.46216531834861) bank31431 +31432 POINT(33.36281560427802 -118.6979366843547) bank31432 +31433 POINT(33.25898234629414 -118.039250815494) bank31433 +31434 POINT(33.990030244429974 -117.26058114585051) bank31434 +31435 POINT(34.617576423464435 -118.93510286900376) bank31435 +31436 POINT(33.894742243026826 -118.73476005032707) bank31436 +31437 POINT(34.306797137635684 -117.82120889439213) bank31437 +31438 POINT(34.60445032688411 -117.53887599002512) bank31438 +31439 POINT(33.06385445263415 -118.02681142469824) bank31439 +31440 POINT(33.69967512326923 -118.36259356420123) bank31440 +31441 POINT(33.72075706839331 -117.94300241639388) bank31441 +31442 POINT(33.739029936015065 -118.0823843542507) bank31442 +31443 POINT(33.965467995838196 -119.14203105760348) bank31443 +31444 POINT(34.38023259818913 -118.10397122916856) bank31444 +31445 POINT(34.941136052056905 -118.3927024001073) bank31445 +31446 POINT(34.31370937437298 -117.58906611018114) bank31446 +31447 POINT(33.52770160972787 -118.34815239710231) bank31447 +31448 POINT(35.049870796006985 -118.46627364800752) bank31448 +31449 POINT(33.80043621547205 -117.4589436588722) bank31449 +31450 POINT(34.37713191949886 -118.58510745286219) bank31450 +31451 POINT(35.030434537804005 -119.11621003368897) bank31451 +31452 POINT(34.13671511139721 -118.85706915755954) bank31452 +31453 POINT(34.82475581367182 -117.33910659876835) bank31453 +31454 POINT(34.64604437509445 -118.79172053676373) bank31454 +31455 POINT(34.35893156595961 -118.04949556638056) bank31455 +31456 POINT(34.122438705010566 -117.67985272726406) bank31456 +31457 POINT(34.61484657875921 -117.65429714661059) bank31457 +31458 POINT(33.824500883597 -117.99699049756218) bank31458 +31459 POINT(33.84575328181128 -118.6684494991459) bank31459 +31460 POINT(33.64296366016682 -117.46668315807118) bank31460 +31461 POINT(34.981190439529136 -117.72056096920944) bank31461 +31462 POINT(33.1754346042597 -117.29133677686248) bank31462 +31463 POINT(35.050344262881424 -119.04781269948644) bank31463 +31464 POINT(34.792108467653414 -119.19534487435658) bank31464 +31465 POINT(33.118748761008575 -118.8218841636291) bank31465 +31466 POINT(33.29294379232672 -118.13406498923798) bank31466 +31467 POINT(34.73567253084977 -118.48039815241246) bank31467 +31468 POINT(33.79663091387087 -119.06408666511669) bank31468 +31469 POINT(34.00683259640247 -118.65585353835452) bank31469 +31470 POINT(34.9035219762803 -117.98105619887953) bank31470 +31471 POINT(33.40791193300004 -119.2043457380283) bank31471 +31472 POINT(34.73487108170095 -118.20796967533799) bank31472 +31473 POINT(34.864881467665505 -119.2141733565249) bank31473 +31474 POINT(34.17451207164008 -117.97344350552486) bank31474 +31475 POINT(33.9410703863855 -118.52790385588463) bank31475 +31476 POINT(34.79546117938438 -118.77841786144585) bank31476 +31477 POINT(33.648634418110824 -118.08923618319149) bank31477 +31478 POINT(34.49422244001924 -119.00458786539998) bank31478 +31479 POINT(33.55325891937751 -118.0504787161614) bank31479 +31480 POINT(34.76730630369681 -119.09948488612606) bank31480 +31481 POINT(34.629066358442316 -117.26348808915657) bank31481 +31482 POINT(34.499139341204554 -118.06084531373914) bank31482 +31483 POINT(33.532330903948264 -118.23281320648111) bank31483 +31484 POINT(33.689677646075914 -118.35924398940229) bank31484 +31485 POINT(33.748370703434645 -117.72846394575426) bank31485 +31486 POINT(34.8781185754481 -118.41293945181184) bank31486 +31487 POINT(33.83128785976683 -118.1796007922703) bank31487 +31488 POINT(33.340728336150015 -118.27141434616348) bank31488 +31489 POINT(33.48130580663782 -117.840198161524) bank31489 +31490 POINT(34.17472489089068 -117.41071274150352) bank31490 +31491 POINT(34.30178322985943 -118.29202830428515) bank31491 +31492 POINT(33.7317388553873 -118.66407200787269) bank31492 +31493 POINT(34.33049466473938 -117.52330932792431) bank31493 +31494 POINT(33.84544944731284 -118.25119857911278) bank31494 +31495 POINT(35.0166165233451 -118.98867939520579) bank31495 +31496 POINT(33.536064182328815 -117.42066888693843) bank31496 +31497 POINT(34.96833539733148 -118.86914212128268) bank31497 +31498 POINT(33.90773155749896 -118.91874280662527) bank31498 +31499 POINT(34.74930981288442 -118.54898164138204) bank31499 +31500 POINT(33.29389988364343 -118.61348794621503) bank31500 +31501 POINT(34.19896009557322 -118.48578473361603) bank31501 +31502 POINT(34.55465472699115 -118.41082279079404) bank31502 +31503 POINT(33.176379653918396 -117.51628349186298) bank31503 +31504 POINT(34.73884815309881 -117.3891573531952) bank31504 +31505 POINT(33.2622273156759 -117.71789027939435) bank31505 +31506 POINT(34.8460360265391 -118.04787511003377) bank31506 +31507 POINT(33.244687848568816 -117.72429094131546) bank31507 +31508 POINT(33.794871087004445 -117.8188647020994) bank31508 +31509 POINT(34.403358001532176 -117.97877227181341) bank31509 +31510 POINT(34.75051689386324 -119.18375256899061) bank31510 +31511 POINT(33.33037374826272 -118.3309953419717) bank31511 +31512 POINT(34.71445561745006 -118.00352252434577) bank31512 +31513 POINT(34.93680783349694 -118.34314054074858) bank31513 +31514 POINT(34.278552892344656 -117.2629213402513) bank31514 +31515 POINT(34.91046362907484 -118.40816460323916) bank31515 +31516 POINT(34.44780843313904 -118.53098998287614) bank31516 +31517 POINT(34.79960858785108 -118.3572620440697) bank31517 +31518 POINT(33.543870565996876 -118.20189375546506) bank31518 +31519 POINT(33.247998663603504 -118.66309478516429) bank31519 +31520 POINT(34.15865554959187 -118.53883020325556) bank31520 +31521 POINT(34.51421139464549 -118.78784895642612) bank31521 +31522 POINT(33.45931147211495 -117.95037165839102) bank31522 +31523 POINT(34.581718486696104 -119.06211567386973) bank31523 +31524 POINT(35.048123556614144 -118.7644835115024) bank31524 +31525 POINT(34.58611467920925 -117.73833797611427) bank31525 +31526 POINT(33.950345751361475 -118.68215974480253) bank31526 +31527 POINT(34.74172955775243 -118.34332601633892) bank31527 +31528 POINT(34.44710664624594 -119.09533815869192) bank31528 +31529 POINT(34.21044242766758 -118.29294505759285) bank31529 +31530 POINT(34.46107668304285 -118.46416494733667) bank31530 +31531 POINT(34.33677622523711 -117.47229687403804) bank31531 +31532 POINT(34.28232415180944 -117.5711332921765) bank31532 +31533 POINT(34.53087661568455 -118.06412737603443) bank31533 +31534 POINT(33.376241927698445 -118.57452380162047) bank31534 +31535 POINT(34.17040253015329 -118.61747067151617) bank31535 +31536 POINT(33.877529112990665 -117.28668838891747) bank31536 +31537 POINT(33.452327690994366 -118.16989602316848) bank31537 +31538 POINT(33.73408645818596 -118.75348066591165) bank31538 +31539 POINT(34.84230948931811 -117.60507087326745) bank31539 +31540 POINT(33.1874405603066 -119.17552129798814) bank31540 +31541 POINT(33.87952081571288 -118.41685060617814) bank31541 +31542 POINT(34.73202462122903 -118.72958314421238) bank31542 +31543 POINT(33.436384121454886 -118.2064681197012) bank31543 +31544 POINT(34.559486199139286 -118.52900880933753) bank31544 +31545 POINT(33.34448238255576 -118.80056437946594) bank31545 +31546 POINT(34.49101855361711 -117.7070940126526) bank31546 +31547 POINT(34.43530829040702 -118.63320444450206) bank31547 +31548 POINT(34.76665912589105 -119.01364199225493) bank31548 +31549 POINT(33.47006533969887 -118.02214348637906) bank31549 +31550 POINT(34.03358455574976 -117.36177211775353) bank31550 +31551 POINT(34.83112918655234 -119.13935669455603) bank31551 +31552 POINT(34.115366446793395 -117.53098695643108) bank31552 +31553 POINT(34.11999498676545 -117.83261933159704) bank31553 +31554 POINT(34.73733033589533 -119.06638715833346) bank31554 +31555 POINT(33.12957947986875 -117.40098635937593) bank31555 +31556 POINT(34.922502947993145 -118.28034269786903) bank31556 +31557 POINT(33.48751860834289 -117.7769134068347) bank31557 +31558 POINT(34.2682414768511 -118.91769496326881) bank31558 +31559 POINT(34.18065384554592 -117.34625549509792) bank31559 +31560 POINT(33.552702963143346 -117.38609837298976) bank31560 +31561 POINT(34.92028044841231 -118.46577740851323) bank31561 +31562 POINT(33.99440248418185 -118.91710038974855) bank31562 +31563 POINT(34.134356004277016 -118.79710514076676) bank31563 +31564 POINT(33.06579657637425 -117.52846712135761) bank31564 +31565 POINT(34.55498906026858 -119.0659954285415) bank31565 +31566 POINT(33.69738976403273 -118.93559949512945) bank31566 +31567 POINT(34.471891294208795 -118.31136300542903) bank31567 +31568 POINT(34.83885593730773 -117.83234973895034) bank31568 +31569 POINT(34.900881680346316 -117.32528046486586) bank31569 +31570 POINT(34.641737102775224 -119.13530252588697) bank31570 +31571 POINT(34.77012501808851 -117.29978314267845) bank31571 +31572 POINT(33.623321379723365 -117.94788247198936) bank31572 +31573 POINT(33.854423459711356 -118.71980565682999) bank31573 +31574 POINT(34.66420268692748 -118.26342412237352) bank31574 +31575 POINT(33.238333694950825 -118.59043116860491) bank31575 +31576 POINT(33.86340577331018 -117.39519314556266) bank31576 +31577 POINT(34.70452175246924 -118.93352806483298) bank31577 +31578 POINT(34.5012470510475 -118.64267301487926) bank31578 +31579 POINT(34.500784078130934 -118.56065918503502) bank31579 +31580 POINT(34.79648909375292 -117.90455878380769) bank31580 +31581 POINT(34.88072827382282 -117.78479046880119) bank31581 +31582 POINT(33.872986818181566 -118.92738217104952) bank31582 +31583 POINT(34.603495965767365 -118.80354995705405) bank31583 +31584 POINT(34.857301615819395 -117.39816750631186) bank31584 +31585 POINT(34.275101645797406 -118.17022217140197) bank31585 +31586 POINT(33.61658917313902 -119.1738342082154) bank31586 +31587 POINT(34.81912105998674 -117.95010660651619) bank31587 +31588 POINT(33.7369304801663 -118.37351509811027) bank31588 +31589 POINT(33.22980318338559 -118.96378984329822) bank31589 +31590 POINT(34.18001288635054 -118.9770905974746) bank31590 +31591 POINT(33.92412091775276 -119.18437211479493) bank31591 +31592 POINT(34.209106070835375 -119.21315547228156) bank31592 +31593 POINT(34.07184313695663 -118.96567186681193) bank31593 +31594 POINT(34.08845501332714 -118.82782002786799) bank31594 +31595 POINT(33.47277911044127 -118.56359202701775) bank31595 +31596 POINT(34.50524703403994 -118.99159959004031) bank31596 +31597 POINT(33.264129032301724 -118.68745180153556) bank31597 +31598 POINT(33.060587995972924 -118.59032970367677) bank31598 +31599 POINT(34.99764153559702 -118.63795020221974) bank31599 +31600 POINT(34.28374680329782 -117.51455799946591) bank31600 +31601 POINT(34.88485863101664 -117.53400399919754) bank31601 +31602 POINT(33.32908854650424 -118.9549229285426) bank31602 +31603 POINT(33.799660738153506 -117.76142736285956) bank31603 +31604 POINT(33.787208538401444 -117.79563140905435) bank31604 +31605 POINT(33.08289226290808 -118.69505789025926) bank31605 +31606 POINT(33.23315578602223 -117.39999245779107) bank31606 +31607 POINT(33.18103932159358 -118.33284026557051) bank31607 +31608 POINT(34.684316971396626 -118.60443511894327) bank31608 +31609 POINT(35.01652682454707 -117.87480993488018) bank31609 +31610 POINT(34.14409575206314 -118.58988679615994) bank31610 +31611 POINT(33.28730770771853 -118.82016479160585) bank31611 +31612 POINT(33.535976267698814 -118.24874381393371) bank31612 +31613 POINT(34.58947305219076 -119.08093006020307) bank31613 +31614 POINT(34.13366552169648 -119.14614940447325) bank31614 +31615 POINT(33.09382051078576 -117.90773451551902) bank31615 +31616 POINT(33.2705263881099 -118.1191150949102) bank31616 +31617 POINT(34.75191891449386 -117.75656842425774) bank31617 +31618 POINT(33.73803280308576 -118.60660396802498) bank31618 +31619 POINT(33.63057188979392 -117.71975028427525) bank31619 +31620 POINT(34.01042690741488 -117.40805668825232) bank31620 +31621 POINT(34.90016405882118 -118.0942510270245) bank31621 +31622 POINT(33.880930788952824 -117.3611428202604) bank31622 +31623 POINT(33.802118683456435 -117.51287872449747) bank31623 +31624 POINT(35.03125502666824 -119.2353299062667) bank31624 +31625 POINT(34.299146539368564 -118.43321895573952) bank31625 +31626 POINT(34.67513870448724 -118.255939939464) bank31626 +31627 POINT(33.18970169361786 -118.11102859935194) bank31627 +31628 POINT(33.37648382800272 -118.04757585782083) bank31628 +31629 POINT(33.535198862696404 -118.58291788924024) bank31629 +31630 POINT(34.051018980089154 -118.18001522646615) bank31630 +31631 POINT(34.480167802787086 -117.95224628580331) bank31631 +31632 POINT(34.44309046915792 -118.76624154932188) bank31632 +31633 POINT(34.26511171334685 -118.28307941678776) bank31633 +31634 POINT(33.80737517821026 -117.97797862581274) bank31634 +31635 POINT(34.55549698771187 -117.77027129895204) bank31635 +31636 POINT(34.472262246370654 -117.67387023007844) bank31636 +31637 POINT(34.38044751066531 -117.88527446454161) bank31637 +31638 POINT(33.673800673524894 -117.26783849627535) bank31638 +31639 POINT(33.89297149882227 -118.15807179141385) bank31639 +31640 POINT(33.534091319610724 -117.43657909732659) bank31640 +31641 POINT(33.35013190355129 -117.64633184517118) bank31641 +31642 POINT(34.63537166552607 -118.91043210976466) bank31642 +31643 POINT(34.83388130262798 -117.54147386369127) bank31643 +31644 POINT(34.62739975834305 -119.1364536752881) bank31644 +31645 POINT(33.732949224270605 -117.92887891183888) bank31645 +31646 POINT(34.22991038657924 -119.18284026860808) bank31646 +31647 POINT(33.475595142308315 -117.95269567688354) bank31647 +31648 POINT(33.73583881282569 -118.50928215545875) bank31648 +31649 POINT(33.27259160194601 -118.57092648640935) bank31649 +31650 POINT(33.62416998073358 -119.04232187092676) bank31650 +31651 POINT(33.653530010668895 -118.4952662962615) bank31651 +31652 POINT(34.61393458846162 -118.69934692058018) bank31652 +31653 POINT(34.35382697627159 -117.94538494964225) bank31653 +31654 POINT(34.092199786416025 -117.4487304142282) bank31654 +31655 POINT(33.6498334600579 -118.22086478096027) bank31655 +31656 POINT(34.51907056425998 -118.3289849488121) bank31656 +31657 POINT(34.487269761646466 -117.54864243255793) bank31657 +31658 POINT(33.50895067196829 -117.4144312053384) bank31658 +31659 POINT(33.61965171226529 -118.30652359634166) bank31659 +31660 POINT(33.52002146542181 -117.4134150836892) bank31660 +31661 POINT(33.13975373069172 -118.35297484664818) bank31661 +31662 POINT(33.629716024631826 -117.76756829099065) bank31662 +31663 POINT(33.620396743432636 -118.07737160005706) bank31663 +31664 POINT(34.07155028129883 -118.92936825751504) bank31664 +31665 POINT(34.707701614951205 -118.83363727654296) bank31665 +31666 POINT(33.79211163341359 -118.29301921733858) bank31666 +31667 POINT(34.28801922919041 -117.9320259263611) bank31667 +31668 POINT(34.48749083796237 -118.54409437699745) bank31668 +31669 POINT(34.480937882258864 -118.07469093338771) bank31669 +31670 POINT(33.836576050733655 -118.16483311349212) bank31670 +31671 POINT(33.3768529990999 -117.32111656032356) bank31671 +31672 POINT(33.50940092537955 -118.28058062277036) bank31672 +31673 POINT(33.22350386941788 -118.18577649441042) bank31673 +31674 POINT(33.263452265813726 -118.91327735721929) bank31674 +31675 POINT(35.032287210594454 -118.67326760571294) bank31675 +31676 POINT(34.104827522671336 -117.49015258788899) bank31676 +31677 POINT(34.497080728872156 -117.57644845492757) bank31677 +31678 POINT(33.87616689757858 -118.0525695331317) bank31678 +31679 POINT(33.47079426331338 -118.8998012293293) bank31679 +31680 POINT(34.529076147650706 -117.66643981360887) bank31680 +31681 POINT(34.838412620952454 -118.00371737528896) bank31681 +31682 POINT(33.57376918210757 -118.74345620765243) bank31682 +31683 POINT(33.627418256088355 -117.46115707643642) bank31683 +31684 POINT(34.0096981152005 -118.96495578323432) bank31684 +31685 POINT(33.513510117048156 -118.23435016741487) bank31685 +31686 POINT(33.54234631637532 -118.93383065096573) bank31686 +31687 POINT(33.91205803954088 -118.28815431887668) bank31687 +31688 POINT(34.01024426007692 -119.17637384925558) bank31688 +31689 POINT(34.47980441641709 -118.5068862827039) bank31689 +31690 POINT(33.96996169237752 -117.28370498974864) bank31690 +31691 POINT(34.21714485244738 -118.41921577570338) bank31691 +31692 POINT(34.45770880220242 -117.78511592460116) bank31692 +31693 POINT(34.82755892789867 -118.10327778489487) bank31693 +31694 POINT(34.96971702547659 -118.50124254953292) bank31694 +31695 POINT(34.47137204899255 -118.78478560789375) bank31695 +31696 POINT(34.473898995177564 -117.91206566251174) bank31696 +31697 POINT(33.53070315376857 -117.79779106517594) bank31697 +31698 POINT(34.778945641574055 -118.32280003558542) bank31698 +31699 POINT(34.98205219294414 -118.25130809088658) bank31699 +31700 POINT(33.14633519122748 -118.71671668404663) bank31700 +31701 POINT(33.890521994804125 -118.26535900646475) bank31701 +31702 POINT(33.68022854983483 -117.24899828175619) bank31702 +31703 POINT(34.11634565592217 -117.86616960475452) bank31703 +31704 POINT(34.5757290512024 -117.70104196746125) bank31704 +31705 POINT(34.36418655624191 -119.08849450433684) bank31705 +31706 POINT(34.41566097433912 -117.64054895401095) bank31706 +31707 POINT(35.00111873048103 -118.85834225483845) bank31707 +31708 POINT(34.079896831262495 -118.45929207861934) bank31708 +31709 POINT(33.22401719559311 -119.12226271630006) bank31709 +31710 POINT(34.4627944471603 -118.6277855159166) bank31710 +31711 POINT(34.075703454582495 -119.14569904385777) bank31711 +31712 POINT(34.17098399543885 -118.70176248075342) bank31712 +31713 POINT(33.117289978649396 -117.8006692263181) bank31713 +31714 POINT(33.943795770184444 -118.53841104614362) bank31714 +31715 POINT(34.81248909897974 -118.03127206465915) bank31715 +31716 POINT(34.2730519289883 -118.99709697690527) bank31716 +31717 POINT(34.50762833856186 -117.38765245342795) bank31717 +31718 POINT(33.579429174586735 -118.20956470289138) bank31718 +31719 POINT(33.50339242681456 -117.28196640835465) bank31719 +31720 POINT(34.88176285017222 -118.50567663830984) bank31720 +31721 POINT(33.13015923080437 -118.06874130047657) bank31721 +31722 POINT(34.56466445348542 -117.57465103212496) bank31722 +31723 POINT(34.86678703857486 -118.04316503084323) bank31723 +31724 POINT(34.25533893428287 -119.05137693876156) bank31724 +31725 POINT(34.62643521566258 -118.82814522677802) bank31725 +31726 POINT(35.04221036864003 -118.07353266048844) bank31726 +31727 POINT(34.96164753085534 -117.53912736449317) bank31727 +31728 POINT(34.51625696752154 -118.54199757544221) bank31728 +31729 POINT(33.15235323622838 -117.64930101777215) bank31729 +31730 POINT(33.75973852358908 -117.9505689224754) bank31730 +31731 POINT(34.49151254317855 -117.26492911055092) bank31731 +31732 POINT(34.116282039009434 -118.81468734855088) bank31732 +31733 POINT(34.447688897781525 -117.25147313251922) bank31733 +31734 POINT(33.679768414346945 -118.1364754435193) bank31734 +31735 POINT(33.7310162969985 -118.45379806541311) bank31735 +31736 POINT(33.75151712451904 -117.65465577154924) bank31736 +31737 POINT(33.51469732696736 -118.11112106474988) bank31737 +31738 POINT(33.37719761759109 -118.17632455966434) bank31738 +31739 POINT(34.232896813205656 -118.1360827417693) bank31739 +31740 POINT(33.27178834755352 -119.04249735330356) bank31740 +31741 POINT(34.418703135495754 -119.1640665512147) bank31741 +31742 POINT(34.277627499564375 -118.59458391553376) bank31742 +31743 POINT(34.820836983514404 -118.03657255210199) bank31743 +31744 POINT(34.91772915600127 -119.18503291543541) bank31744 +31745 POINT(33.93935967928351 -117.68948356017162) bank31745 +31746 POINT(34.223038437164256 -117.591364116141) bank31746 +31747 POINT(33.556841633004574 -118.49546791705662) bank31747 +31748 POINT(34.335604846150694 -117.93021895682789) bank31748 +31749 POINT(33.139583904132486 -118.17632876708993) bank31749 +31750 POINT(34.53804271825025 -119.04827144595107) bank31750 +31751 POINT(33.173461980798635 -117.97405748889776) bank31751 +31752 POINT(33.32275318821057 -119.09675881182226) bank31752 +31753 POINT(34.17156322186536 -118.71837675669998) bank31753 +31754 POINT(33.13097178967757 -119.17891506480372) bank31754 +31755 POINT(33.45366439119767 -117.44567377564658) bank31755 +31756 POINT(33.79933844532891 -118.35812640751895) bank31756 +31757 POINT(34.51966268107838 -118.60080577070903) bank31757 +31758 POINT(33.861835964763586 -118.47854735125624) bank31758 +31759 POINT(33.41534943425812 -117.76557193823855) bank31759 +31760 POINT(34.26748107807323 -117.28463984631203) bank31760 +31761 POINT(33.35401787157563 -118.87033352408457) bank31761 +31762 POINT(33.88427004356429 -118.20187353859302) bank31762 +31763 POINT(33.353854431604134 -117.79582466527181) bank31763 +31764 POINT(33.45362978834815 -117.96460578021623) bank31764 +31765 POINT(33.12900787394252 -119.18690617549463) bank31765 +31766 POINT(34.08322857835515 -117.47491432243312) bank31766 +31767 POINT(34.11283164570079 -117.50438075649457) bank31767 +31768 POINT(34.89260256209829 -117.34696939275315) bank31768 +31769 POINT(33.70583635990296 -117.61911033226939) bank31769 +31770 POINT(33.73244889007364 -118.54677261108299) bank31770 +31771 POINT(34.851188955179396 -118.49396685510096) bank31771 +31772 POINT(34.61634593957358 -117.96902034143079) bank31772 +31773 POINT(33.91473912015758 -119.13001125496193) bank31773 +31774 POINT(33.710624023231695 -117.8018861945481) bank31774 +31775 POINT(33.839765916907176 -118.62931635650162) bank31775 +31776 POINT(34.733329596153325 -118.71229569764827) bank31776 +31777 POINT(33.911860161070486 -118.44046882337034) bank31777 +31778 POINT(33.72748352160186 -118.28890314308165) bank31778 +31779 POINT(33.4045042947241 -118.9329468214232) bank31779 +31780 POINT(34.14920446854965 -117.71512773175728) bank31780 +31781 POINT(34.158373462840814 -118.68407874160766) bank31781 +31782 POINT(34.552521117225126 -118.61137117235624) bank31782 +31783 POINT(34.84238254112166 -118.60488984453362) bank31783 +31784 POINT(34.09541861242237 -118.05062451488517) bank31784 +31785 POINT(33.49570922619796 -118.75398693359757) bank31785 +31786 POINT(34.3412829149742 -118.61094889105517) bank31786 +31787 POINT(34.30543558156579 -118.1813222242115) bank31787 +31788 POINT(33.12610746993591 -117.84385061235017) bank31788 +31789 POINT(34.357820718565996 -117.55997866137895) bank31789 +31790 POINT(33.69151368213635 -117.35393297823512) bank31790 +31791 POINT(33.850274127511426 -117.83939134633991) bank31791 +31792 POINT(33.471510849352235 -118.57282113724963) bank31792 +31793 POINT(34.868256071147925 -117.47595315406214) bank31793 +31794 POINT(34.12572448395528 -118.72228118729632) bank31794 +31795 POINT(33.64761693710904 -119.1531495653773) bank31795 +31796 POINT(34.10872610131598 -117.63612898349953) bank31796 +31797 POINT(34.555519400250866 -118.92148234504907) bank31797 +31798 POINT(33.90910934227401 -117.90622677208097) bank31798 +31799 POINT(34.171810613890436 -118.90668326329066) bank31799 +31800 POINT(34.77882595531524 -117.39042004967756) bank31800 +31801 POINT(33.451863573500155 -119.11775865733324) bank31801 +31802 POINT(33.38756184503396 -118.40829510212257) bank31802 +31803 POINT(34.880189134053616 -118.07015370006532) bank31803 +31804 POINT(34.244593940609775 -118.40802028299805) bank31804 +31805 POINT(33.72306824445018 -118.04152181670797) bank31805 +31806 POINT(34.119014639538456 -117.69969693462033) bank31806 +31807 POINT(34.802405843942495 -118.48480275842324) bank31807 +31808 POINT(33.583870072229765 -118.39362643436111) bank31808 +31809 POINT(33.12968596792997 -117.52247328081953) bank31809 +31810 POINT(33.8115170481092 -117.36860707739105) bank31810 +31811 POINT(33.593353484821364 -119.03064917205445) bank31811 +31812 POINT(33.24446016217751 -117.8674886164358) bank31812 +31813 POINT(34.77861331844149 -119.15570793140606) bank31813 +31814 POINT(34.870266161447454 -119.0991860626304) bank31814 +31815 POINT(34.172183542721726 -117.53112954282545) bank31815 +31816 POINT(33.49458914925564 -118.63743762326855) bank31816 +31817 POINT(34.90418557345942 -117.59924559040172) bank31817 +31818 POINT(34.934925334219486 -118.70345288325247) bank31818 +31819 POINT(33.781112342592074 -117.71462347349413) bank31819 +31820 POINT(34.32639518437812 -117.99432551696404) bank31820 +31821 POINT(34.90147477313118 -117.86827096393313) bank31821 +31822 POINT(34.77967843917292 -117.37488230531973) bank31822 +31823 POINT(34.91721539872534 -118.15104105674519) bank31823 +31824 POINT(34.13214491754429 -118.81317586679678) bank31824 +31825 POINT(34.31621891532978 -118.70024513067794) bank31825 +31826 POINT(33.60599327497716 -117.94199449316665) bank31826 +31827 POINT(33.736576475091134 -118.76652477562833) bank31827 +31828 POINT(33.69278368037928 -119.04527098467582) bank31828 +31829 POINT(34.70934008818517 -117.60988808097531) bank31829 +31830 POINT(33.42219811663118 -118.93085683880335) bank31830 +31831 POINT(35.04924593046291 -117.6949081661921) bank31831 +31832 POINT(34.00385654032812 -118.99145288247054) bank31832 +31833 POINT(33.17566606767299 -117.72701191207192) bank31833 +31834 POINT(34.324242367716714 -118.72275518276406) bank31834 +31835 POINT(33.882214607181 -117.84600421424186) bank31835 +31836 POINT(35.03995532025692 -117.61313869672088) bank31836 +31837 POINT(34.69137778972999 -117.30346298687367) bank31837 +31838 POINT(34.64976790632678 -117.56516501854547) bank31838 +31839 POINT(33.360233043336486 -118.78943041079351) bank31839 +31840 POINT(33.567940067173275 -118.86269757735417) bank31840 +31841 POINT(34.84742385041886 -118.55694607109275) bank31841 +31842 POINT(34.99376330706154 -117.92618595735073) bank31842 +31843 POINT(33.67136309178387 -117.72401235205942) bank31843 +31844 POINT(34.79620078284677 -117.87258672354008) bank31844 +31845 POINT(35.00846855939016 -118.03521040367221) bank31845 +31846 POINT(34.03845317794588 -118.27669355389838) bank31846 +31847 POINT(33.145399836421795 -117.2458629704958) bank31847 +31848 POINT(34.51523767442121 -118.08368094944662) bank31848 +31849 POINT(33.800591251712554 -117.51905933614636) bank31849 +31850 POINT(34.571768101060115 -118.25115177011901) bank31850 +31851 POINT(34.72891150899537 -117.35500340214192) bank31851 +31852 POINT(34.53636243824647 -118.70979274862158) bank31852 +31853 POINT(34.99316887466283 -117.87564916805417) bank31853 +31854 POINT(34.73745752708639 -119.08719539959351) bank31854 +31855 POINT(33.334812021373665 -117.57342301825202) bank31855 +31856 POINT(34.86446598456362 -119.01980868956845) bank31856 +31857 POINT(34.49415025876577 -117.5677149105634) bank31857 +31858 POINT(34.97814807140411 -117.44320117345576) bank31858 +31859 POINT(34.089826653758465 -117.69225883692292) bank31859 +31860 POINT(33.55975943943051 -119.11279016968034) bank31860 +31861 POINT(34.556092742422884 -117.96540829898343) bank31861 +31862 POINT(34.577755467888096 -117.50294818595083) bank31862 +31863 POINT(33.55014391891171 -117.27428984815242) bank31863 +31864 POINT(34.28863459504812 -118.5868425304448) bank31864 +31865 POINT(34.31559407264361 -117.95751797517346) bank31865 +31866 POINT(35.01502537146363 -117.44781052432359) bank31866 +31867 POINT(34.434408526855336 -118.777532216016) bank31867 +31868 POINT(33.334551030398615 -119.08170155021679) bank31868 +31869 POINT(34.03679624119089 -117.63303300515474) bank31869 +31870 POINT(34.77469417136979 -118.01324322843165) bank31870 +31871 POINT(33.28408793751388 -118.75771082696536) bank31871 +31872 POINT(34.33464298904263 -119.04334710694802) bank31872 +31873 POINT(33.70233283602995 -117.72315076460161) bank31873 +31874 POINT(34.87414517354063 -119.10686060366962) bank31874 +31875 POINT(33.425894228594544 -118.16911914783529) bank31875 +31876 POINT(34.06382624940812 -118.76193268735602) bank31876 +31877 POINT(34.43531573889909 -118.39543708980965) bank31877 +31878 POINT(33.14564515713372 -117.45586692359883) bank31878 +31879 POINT(34.56253076195343 -117.4767726352) bank31879 +31880 POINT(34.96480530628887 -118.41055183607953) bank31880 +31881 POINT(34.10300144279996 -118.31686561740624) bank31881 +31882 POINT(33.725490176224724 -118.0494247057121) bank31882 +31883 POINT(33.49172868148641 -117.6814626557314) bank31883 +31884 POINT(33.434660802332914 -118.31906541717443) bank31884 +31885 POINT(33.154361697793014 -117.52931625251085) bank31885 +31886 POINT(33.977887164289456 -119.2009940622521) bank31886 +31887 POINT(34.56470103345452 -117.63419588507658) bank31887 +31888 POINT(35.01118398790443 -118.18211418920009) bank31888 +31889 POINT(33.64555299173591 -118.33373519891826) bank31889 +31890 POINT(34.61280581997018 -118.9020556263843) bank31890 +31891 POINT(34.70904813166116 -117.32445300808357) bank31891 +31892 POINT(33.15595161636529 -118.08778924928254) bank31892 +31893 POINT(34.364473360453864 -119.13249475775821) bank31893 +31894 POINT(33.75287389511096 -119.03226434483129) bank31894 +31895 POINT(33.3757725967598 -117.31837015091658) bank31895 +31896 POINT(34.6416041184274 -118.41187150428077) bank31896 +31897 POINT(33.10706060273364 -117.55269440343004) bank31897 +31898 POINT(33.986192540293786 -117.88764004990789) bank31898 +31899 POINT(33.70403004267643 -118.53300430493375) bank31899 +31900 POINT(34.539231493783454 -118.58053765546708) bank31900 +31901 POINT(33.91410303526552 -119.18215218669269) bank31901 +31902 POINT(34.17560147389251 -117.50074753434994) bank31902 +31903 POINT(34.33262580255555 -117.95679752200641) bank31903 +31904 POINT(34.634104778496344 -118.7711912776999) bank31904 +31905 POINT(34.4198119972184 -118.50098611281669) bank31905 +31906 POINT(33.54924404144896 -118.90740984716896) bank31906 +31907 POINT(33.97053825683706 -118.71026339386569) bank31907 +31908 POINT(34.20882529505569 -117.25170457082018) bank31908 +31909 POINT(35.048783823834995 -118.13597175434877) bank31909 +31910 POINT(33.808776463248535 -118.50411000132787) bank31910 +31911 POINT(34.59952585033921 -118.4994220503874) bank31911 +31912 POINT(34.9774682272354 -117.89912437834464) bank31912 +31913 POINT(33.65872358191764 -117.68171731142928) bank31913 +31914 POINT(33.11260340372276 -118.34603977359146) bank31914 +31915 POINT(33.78159667731898 -118.98896935258135) bank31915 +31916 POINT(33.22425171489288 -118.80135278964548) bank31916 +31917 POINT(34.21892512756654 -117.47843436568144) bank31917 +31918 POINT(33.472846279040596 -118.97344511984387) bank31918 +31919 POINT(33.451921647103376 -117.40657445963876) bank31919 +31920 POINT(34.43810843839961 -118.19030432578356) bank31920 +31921 POINT(33.560718451622975 -117.79947960619653) bank31921 +31922 POINT(34.428774759821735 -117.3831812738225) bank31922 +31923 POINT(34.50372738542976 -118.31008142898911) bank31923 +31924 POINT(33.58228584703179 -118.06100979831913) bank31924 +31925 POINT(33.41939663345939 -117.46146515824894) bank31925 +31926 POINT(33.08932138392327 -119.09027720047541) bank31926 +31927 POINT(33.419117135552334 -118.46998510434223) bank31927 +31928 POINT(34.68498201289899 -118.89721233164819) bank31928 +31929 POINT(34.797778673205094 -118.50406251532198) bank31929 +31930 POINT(34.056372774444945 -119.01024173749464) bank31930 +31931 POINT(33.91550890571182 -118.11057014217234) bank31931 +31932 POINT(33.90676397046053 -118.29105718166645) bank31932 +31933 POINT(34.60159328404686 -118.45035659512286) bank31933 +31934 POINT(33.976038093890836 -118.74382301333338) bank31934 +31935 POINT(33.74125879729202 -118.00606786657924) bank31935 +31936 POINT(34.11104423052406 -118.38377217296122) bank31936 +31937 POINT(33.713126210326024 -118.24859534609294) bank31937 +31938 POINT(33.922228289580744 -117.75194902597056) bank31938 +31939 POINT(34.79036691208839 -118.28540571515202) bank31939 +31940 POINT(33.78800175469007 -117.51353067415299) bank31940 +31941 POINT(33.554503740254034 -118.45818543439485) bank31941 +31942 POINT(33.77984283223394 -118.63954460086003) bank31942 +31943 POINT(34.86989441303682 -118.09644267812467) bank31943 +31944 POINT(34.23017215043325 -118.03114478052821) bank31944 +31945 POINT(34.07384439257908 -119.13534357253202) bank31945 +31946 POINT(33.76419556672164 -118.43121046715429) bank31946 +31947 POINT(33.44637884951465 -117.3624974296347) bank31947 +31948 POINT(33.53338964131562 -118.07584517441961) bank31948 +31949 POINT(33.13145804719535 -118.14693877168526) bank31949 +31950 POINT(34.29882012114419 -118.27050237642018) bank31950 +31951 POINT(33.154257446457805 -118.46583499652392) bank31951 +31952 POINT(34.74740086218526 -117.45209067907135) bank31952 +31953 POINT(33.98177290843195 -118.922760976593) bank31953 +31954 POINT(34.21808341621576 -117.83862566995964) bank31954 +31955 POINT(33.22243987551176 -117.65046215757215) bank31955 +31956 POINT(33.52825862496967 -117.38500527645886) bank31956 +31957 POINT(33.95034654201681 -118.68216802881926) bank31957 +31958 POINT(34.46113948133642 -118.53098218981711) bank31958 +31959 POINT(34.346618010829914 -118.97219970875733) bank31959 +31960 POINT(33.30034100336186 -118.93242507969332) bank31960 +31961 POINT(34.044957887381244 -117.67528477179879) bank31961 +31962 POINT(33.34415525188528 -117.72918168474378) bank31962 +31963 POINT(34.36585896049574 -118.34677438087601) bank31963 +31964 POINT(33.09977773557547 -118.03487295306252) bank31964 +31965 POINT(34.5850480770475 -117.30685179111829) bank31965 +31966 POINT(34.509179886629354 -117.80539269166832) bank31966 +31967 POINT(34.022618042344874 -117.89614855807932) bank31967 +31968 POINT(34.82536929734976 -118.47940343190277) bank31968 +31969 POINT(34.85299716306923 -117.49673053303164) bank31969 +31970 POINT(34.108087858718115 -117.26622178987255) bank31970 +31971 POINT(33.52912979180681 -118.86012795166093) bank31971 +31972 POINT(33.58864292100739 -118.95886718646506) bank31972 +31973 POINT(34.9428820109021 -117.67326166148004) bank31973 +31974 POINT(34.51343338984633 -117.92104893032163) bank31974 +31975 POINT(34.322365588034316 -117.26425383262477) bank31975 +31976 POINT(34.965786819947496 -117.97865471169094) bank31976 +31977 POINT(34.680031004043116 -118.86292094810145) bank31977 +31978 POINT(33.13690308038511 -118.85643053005174) bank31978 +31979 POINT(33.567506110276426 -118.4252542951537) bank31979 +31980 POINT(33.32615373664243 -118.0680530127084) bank31980 +31981 POINT(34.70508731872999 -117.62904875279395) bank31981 +31982 POINT(34.3698328945336 -118.39422148153078) bank31982 +31983 POINT(33.546157272007854 -118.88137380664806) bank31983 +31984 POINT(34.91871665074108 -118.52742694982682) bank31984 +31985 POINT(34.87314860064324 -118.36347727299722) bank31985 +31986 POINT(33.48032932393573 -117.26840743520921) bank31986 +31987 POINT(34.53434641362287 -118.99057686759022) bank31987 +31988 POINT(33.54753841610797 -118.91143891410984) bank31988 +31989 POINT(33.60484107401698 -118.07559242981037) bank31989 +31990 POINT(34.12252930455217 -117.60637400774316) bank31990 +31991 POINT(34.29643718247566 -117.781522304665) bank31991 +31992 POINT(34.67084779904294 -119.02305469924823) bank31992 +31993 POINT(33.844214309082666 -118.3819895434188) bank31993 +31994 POINT(34.54354340056922 -117.36663713441575) bank31994 +31995 POINT(33.282982985851014 -119.12120865285144) bank31995 +31996 POINT(33.30883940959695 -117.77228415391619) bank31996 +31997 POINT(35.01153395383442 -118.45027531420159) bank31997 +31998 POINT(33.4140641577303 -118.00523690856801) bank31998 +31999 POINT(33.76712889984986 -119.17442526961673) bank31999 +32000 POINT(34.0744659705134 -118.86906155205013) bank32000 +32001 POINT(34.982796154889684 -119.06037307923752) bank32001 +32002 POINT(33.46218708586041 -117.60767800708763) bank32002 +32003 POINT(34.3288187431496 -118.07777380723378) bank32003 +32004 POINT(33.14539726933795 -117.34803259653557) bank32004 +32005 POINT(33.32013122355629 -117.29100846959821) bank32005 +32006 POINT(33.31770764058808 -117.8354716417546) bank32006 +32007 POINT(34.573559053490975 -118.76232665272487) bank32007 +32008 POINT(34.850761388423926 -117.3354965459701) bank32008 +32009 POINT(34.584785724092065 -119.07084282048754) bank32009 +32010 POINT(34.6473424299348 -117.50299002856327) bank32010 +32011 POINT(34.81854907331956 -117.48772383264233) bank32011 +32012 POINT(34.56117238119789 -118.95174190904669) bank32012 +32013 POINT(33.85836138804867 -117.35093625840209) bank32013 +32014 POINT(33.468141961993496 -118.30687514478849) bank32014 +32015 POINT(34.92634784347027 -117.98692793288575) bank32015 +32016 POINT(33.24511975847342 -117.43458968182311) bank32016 +32017 POINT(34.78491465529251 -117.74537157655351) bank32017 +32018 POINT(33.84271059879156 -118.07960905395404) bank32018 +32019 POINT(34.371028048216225 -118.90019273559449) bank32019 +32020 POINT(35.00298808977715 -117.3535962496158) bank32020 +32021 POINT(33.90462189137664 -118.19882904878038) bank32021 +32022 POINT(33.8103045492727 -118.50948704627737) bank32022 +32023 POINT(34.65480850230479 -117.7244093845739) bank32023 +32024 POINT(33.55994171914431 -118.70746452435002) bank32024 +32025 POINT(34.41037080846729 -118.53740939805773) bank32025 +32026 POINT(33.82668598405622 -118.32239564038564) bank32026 +32027 POINT(34.95405514289916 -117.85156098667437) bank32027 +32028 POINT(34.43058136011758 -117.35834054532974) bank32028 +32029 POINT(33.744428967675766 -118.56568267967691) bank32029 +32030 POINT(33.22029473838339 -118.38455430241325) bank32030 +32031 POINT(34.192052316062856 -118.22981293464966) bank32031 +32032 POINT(34.089018494198534 -118.36728971762864) bank32032 +32033 POINT(33.86329674929759 -118.27050676082742) bank32033 +32034 POINT(33.22308508130516 -117.29256474573074) bank32034 +32035 POINT(34.06218587855684 -118.13679409749575) bank32035 +32036 POINT(34.151036584357946 -117.82524218647916) bank32036 +32037 POINT(34.32561601751337 -118.36897675847308) bank32037 +32038 POINT(33.881273029810096 -118.42549429663627) bank32038 +32039 POINT(34.331888978175414 -117.59151395039196) bank32039 +32040 POINT(34.12521862594196 -118.34687165065307) bank32040 +32041 POINT(34.11008158950012 -117.29338507273317) bank32041 +32042 POINT(34.52305727347269 -117.45462709431101) bank32042 +32043 POINT(33.902679009933344 -117.7879622394433) bank32043 +32044 POINT(33.373987943814285 -118.09969137504208) bank32044 +32045 POINT(34.37470621263253 -118.72338702498439) bank32045 +32046 POINT(34.79883997149174 -118.37425538612011) bank32046 +32047 POINT(33.35492486023288 -117.74478663223299) bank32047 +32048 POINT(34.867184555275024 -117.90837648572892) bank32048 +32049 POINT(34.156874675714 -117.81416215810137) bank32049 +32050 POINT(33.96050131678999 -118.70603556163303) bank32050 +32051 POINT(34.86089183359417 -118.91576304118865) bank32051 +32052 POINT(34.037255881872305 -117.63419097646515) bank32052 +32053 POINT(34.89435622401815 -117.33207034073354) bank32053 +32054 POINT(34.000171290652105 -118.39559879511947) bank32054 +32055 POINT(34.66055275472542 -118.03886082057033) bank32055 +32056 POINT(34.44203373639036 -118.10193101810763) bank32056 +32057 POINT(33.42902449604363 -118.11643365290783) bank32057 +32058 POINT(33.4902048688166 -117.59097767856197) bank32058 +32059 POINT(34.784575788355234 -118.20631925648954) bank32059 +32060 POINT(34.20958844090578 -117.60122018799652) bank32060 +32061 POINT(33.56813496052093 -118.89867315720878) bank32061 +32062 POINT(34.30372369586889 -118.03044596895461) bank32062 +32063 POINT(33.83251099984617 -118.52348922137152) bank32063 +32064 POINT(34.60323687184014 -118.32363360178846) bank32064 +32065 POINT(34.310792046776726 -117.83901222542681) bank32065 +32066 POINT(33.9277008910118 -119.07245134426435) bank32066 +32067 POINT(33.476903240231486 -117.38686743901563) bank32067 +32068 POINT(34.75415435504992 -117.80840757568896) bank32068 +32069 POINT(33.74406686851342 -118.70426491484078) bank32069 +32070 POINT(34.102166308554345 -118.94959080547223) bank32070 +32071 POINT(33.94937242989586 -118.72654898985446) bank32071 +32072 POINT(33.410064778196706 -117.89264083688096) bank32072 +32073 POINT(33.826126356789246 -118.64544717229111) bank32073 +32074 POINT(34.673518239157715 -118.5088890984408) bank32074 +32075 POINT(34.64066090630926 -118.18197168545949) bank32075 +32076 POINT(34.19718790550374 -117.85830650566294) bank32076 +32077 POINT(33.12990535588038 -117.81442149932305) bank32077 +32078 POINT(34.459379387203654 -118.78235640840155) bank32078 +32079 POINT(34.24750229347509 -117.29711197175712) bank32079 +32080 POINT(34.37779595613141 -119.03902722893743) bank32080 +32081 POINT(34.63715148454577 -118.30112458144373) bank32081 +32082 POINT(33.65576298587967 -118.2864351312125) bank32082 +32083 POINT(34.51766144813783 -117.59128188118297) bank32083 +32084 POINT(33.16686380977017 -117.58251907383617) bank32084 +32085 POINT(33.25568834132078 -118.80069448079284) bank32085 +32086 POINT(33.40653079289085 -117.41217595650068) bank32086 +32087 POINT(33.43465278928214 -118.93326635055044) bank32087 +32088 POINT(34.37479042620568 -117.46205966916645) bank32088 +32089 POINT(33.90573099041444 -119.17022897326325) bank32089 +32090 POINT(33.48188059468698 -117.75355413451803) bank32090 +32091 POINT(33.520484069889285 -118.20063694240305) bank32091 +32092 POINT(34.75767428497364 -118.76121647278013) bank32092 +32093 POINT(33.426365512861395 -119.0343165822223) bank32093 +32094 POINT(34.41884864162169 -118.42856514503282) bank32094 +32095 POINT(33.64036469117366 -118.15074450200247) bank32095 +32096 POINT(34.2630349014805 -118.43117591634007) bank32096 +32097 POINT(34.887022126682346 -118.9187230467361) bank32097 +32098 POINT(33.68561285848897 -118.64068258728265) bank32098 +32099 POINT(33.886712858689414 -118.68598769153274) bank32099 +32100 POINT(33.267343171265004 -117.74713827343184) bank32100 +32101 POINT(34.50129560888671 -118.4252817601805) bank32101 +32102 POINT(33.411283397980135 -117.4197936102229) bank32102 +32103 POINT(33.29443902254573 -118.57274755683112) bank32103 +32104 POINT(33.649127358166076 -118.49897299345326) bank32104 +32105 POINT(33.99151339162874 -119.23437937530656) bank32105 +32106 POINT(34.075112967025156 -117.78290840962879) bank32106 +32107 POINT(34.45336044906936 -117.32595221128135) bank32107 +32108 POINT(33.92717834841247 -117.2583295020122) bank32108 +32109 POINT(34.7904121738966 -117.84983593710778) bank32109 +32110 POINT(34.41386635210105 -118.9540172673499) bank32110 +32111 POINT(33.66093841578261 -118.39353913828316) bank32111 +32112 POINT(33.332736271758385 -118.40875187354672) bank32112 +32113 POINT(34.45561909967908 -117.89346436293411) bank32113 +32114 POINT(34.63367291105014 -118.68769664967951) bank32114 +32115 POINT(34.509662162248176 -117.64482287833133) bank32115 +32116 POINT(33.645428090841435 -118.65310586463887) bank32116 +32117 POINT(33.58105660413382 -117.27780314401821) bank32117 +32118 POINT(34.37706885555662 -119.05320349018695) bank32118 +32119 POINT(33.585955616507654 -118.83776756301812) bank32119 +32120 POINT(34.06773652592627 -117.92316647756807) bank32120 +32121 POINT(33.966375352720966 -118.17484230002884) bank32121 +32122 POINT(34.026852786201886 -117.6860698327706) bank32122 +32123 POINT(34.90921342812186 -118.22727103719807) bank32123 +32124 POINT(33.05264441689067 -117.38499954216812) bank32124 +32125 POINT(34.961557892535936 -118.92800424365929) bank32125 +32126 POINT(34.18574049721783 -118.40186540409707) bank32126 +32127 POINT(33.59856427212371 -117.49265525221165) bank32127 +32128 POINT(34.650439710756366 -118.93788917063209) bank32128 +32129 POINT(35.032942920414094 -118.33983717121652) bank32129 +32130 POINT(34.61371095265968 -117.52667787247874) bank32130 +32131 POINT(35.003572686089676 -118.82586596941688) bank32131 +32132 POINT(34.42218977307615 -117.98461175524285) bank32132 +32133 POINT(33.47864575583816 -118.06188524269514) bank32133 +32134 POINT(35.04251566226653 -118.76669906209133) bank32134 +32135 POINT(33.23847876467194 -118.35970626846213) bank32135 +32136 POINT(34.71004064587075 -118.14627824631236) bank32136 +32137 POINT(34.267148282763664 -117.92016065524655) bank32137 +32138 POINT(34.45323064235932 -117.24371802226113) bank32138 +32139 POINT(33.335889245027396 -118.33839910423322) bank32139 +32140 POINT(34.8848166340345 -119.02023962279776) bank32140 +32141 POINT(33.13813973950414 -118.43329308274619) bank32141 +32142 POINT(34.45010100646597 -117.47622588654922) bank32142 +32143 POINT(34.8880416120267 -117.94783225073837) bank32143 +32144 POINT(34.24113256939042 -118.30176176666879) bank32144 +32145 POINT(34.67089663144667 -118.09913306858195) bank32145 +32146 POINT(33.33486739858369 -118.92957820470055) bank32146 +32147 POINT(34.67261049000003 -118.33223957475943) bank32147 +32148 POINT(33.24119918635144 -118.79274213452042) bank32148 +32149 POINT(34.890766657950536 -117.41623895232519) bank32149 +32150 POINT(35.0370420817284 -117.65583275331277) bank32150 +32151 POINT(34.795637521095884 -117.56481894861162) bank32151 +32152 POINT(33.63820145321743 -117.84432682613063) bank32152 +32153 POINT(33.16375278240199 -118.34570218055323) bank32153 +32154 POINT(33.31679740226025 -118.91409800028892) bank32154 +32155 POINT(33.974901497697374 -117.71379064730112) bank32155 +32156 POINT(34.220997352467236 -118.56045757017996) bank32156 +32157 POINT(33.054597630113804 -118.92515781262213) bank32157 +32158 POINT(33.69459731476046 -119.04900556720754) bank32158 +32159 POINT(33.67948970465912 -117.51397590008534) bank32159 +32160 POINT(33.935783479703815 -118.30588882446226) bank32160 +32161 POINT(34.046067834618555 -118.40395171119557) bank32161 +32162 POINT(34.07803564413584 -118.8281562096731) bank32162 +32163 POINT(34.98429633312715 -117.54915539245168) bank32163 +32164 POINT(34.99461527565132 -117.89968797558166) bank32164 +32165 POINT(34.254454296326 -118.05375555840105) bank32165 +32166 POINT(33.58012581912655 -117.30712499785348) bank32166 +32167 POINT(34.915548363821344 -117.61591576537732) bank32167 +32168 POINT(34.6666323516138 -118.86113892355897) bank32168 +32169 POINT(33.13767810803247 -118.36103418322939) bank32169 +32170 POINT(34.0898770768764 -117.24660401180348) bank32170 +32171 POINT(33.861177750559506 -118.16967672253529) bank32171 +32172 POINT(33.91522604517398 -119.02929165354082) bank32172 +32173 POINT(34.96709390313677 -118.06459740711426) bank32173 +32174 POINT(35.04184673489885 -117.64112510375175) bank32174 +32175 POINT(33.37724095901107 -118.76182531713462) bank32175 +32176 POINT(34.68694326840871 -118.61402706222229) bank32176 +32177 POINT(33.98112320411253 -117.61030544147165) bank32177 +32178 POINT(34.584403674037105 -117.39746707569975) bank32178 +32179 POINT(34.69653333315516 -117.31082779637671) bank32179 +32180 POINT(33.32378762603677 -117.3090838075242) bank32180 +32181 POINT(33.36254867938342 -118.99584698015626) bank32181 +32182 POINT(33.60775650442372 -118.48135686294731) bank32182 +32183 POINT(34.11891585655341 -118.22148244985415) bank32183 +32184 POINT(33.71730080201784 -117.74462120220213) bank32184 +32185 POINT(34.04869316883015 -118.81376812556) bank32185 +32186 POINT(33.62253897324058 -118.15793201767433) bank32186 +32187 POINT(33.69593650267488 -118.12650006928094) bank32187 +32188 POINT(33.13905885314399 -118.35725638286007) bank32188 +32189 POINT(34.29476383759501 -119.08333447393309) bank32189 +32190 POINT(34.06718917304229 -118.11566686623371) bank32190 +32191 POINT(35.029341485708485 -119.18375975668795) bank32191 +32192 POINT(34.24034153434485 -117.41095116028015) bank32192 +32193 POINT(34.99339803197116 -117.87227849396255) bank32193 +32194 POINT(33.73716497558618 -118.65076308289531) bank32194 +32195 POINT(34.07740727578906 -118.97499842673771) bank32195 +32196 POINT(34.312058780189645 -117.73672593976524) bank32196 +32197 POINT(34.50036758994119 -117.79173884105117) bank32197 +32198 POINT(33.26390732704891 -117.64793692317556) bank32198 +32199 POINT(35.033274376119465 -118.10332862526552) bank32199 +32200 POINT(34.02042619900078 -117.76969532927232) bank32200 +32201 POINT(34.216295603113316 -118.81129709779435) bank32201 +32202 POINT(34.85964546002541 -118.38212930552099) bank32202 +32203 POINT(33.70600292230172 -117.73313842910365) bank32203 +32204 POINT(34.77770793729258 -119.06342300899554) bank32204 +32205 POINT(33.2532285464273 -118.03435508595669) bank32205 +32206 POINT(33.10656139496019 -118.28104491501733) bank32206 +32207 POINT(33.09826147895466 -117.38288437759613) bank32207 +32208 POINT(33.59665148786724 -118.69118558638233) bank32208 +32209 POINT(34.12953159216458 -117.96097829652952) bank32209 +32210 POINT(34.466266356195554 -117.59503627546447) bank32210 +32211 POINT(34.144726066972126 -117.89122664113548) bank32211 +32212 POINT(34.12844461101074 -117.49626941493148) bank32212 +32213 POINT(33.407276253172256 -118.95968162173844) bank32213 +32214 POINT(34.762347607741994 -118.1534182331239) bank32214 +32215 POINT(34.32305411843532 -118.05243431056049) bank32215 +32216 POINT(34.11445659828593 -117.85956931789278) bank32216 +32217 POINT(33.26369573094482 -117.5782582605279) bank32217 +32218 POINT(34.48967634129267 -117.79630920387756) bank32218 +32219 POINT(34.71097275444987 -118.59492288012885) bank32219 +32220 POINT(34.306809467924445 -117.42759245483452) bank32220 +32221 POINT(35.04255860204581 -118.51320464065228) bank32221 +32222 POINT(34.78299523379176 -117.93203077461455) bank32222 +32223 POINT(33.15336059866037 -118.22120008250486) bank32223 +32224 POINT(33.78707951204292 -118.76002578116226) bank32224 +32225 POINT(34.605406620519155 -117.41690273741978) bank32225 +32226 POINT(33.144261173774225 -118.96599278874798) bank32226 +32227 POINT(34.190562715842404 -117.51419741363976) bank32227 +32228 POINT(34.76742035215469 -117.96562149381538) bank32228 +32229 POINT(33.511959389153546 -118.07548757774508) bank32229 +32230 POINT(34.69965426490373 -118.43280154050274) bank32230 +32231 POINT(33.536232583404285 -119.2097592907354) bank32231 +32232 POINT(34.51628794580396 -118.13287569483302) bank32232 +32233 POINT(34.720525871475985 -117.78715315936746) bank32233 +32234 POINT(34.71274217644014 -117.89689330373105) bank32234 +32235 POINT(33.86504679091845 -118.15641298487233) bank32235 +32236 POINT(33.125048359725795 -118.63987323013279) bank32236 +32237 POINT(35.0188139757476 -118.03005455532953) bank32237 +32238 POINT(33.40507040547885 -118.4409668125914) bank32238 +32239 POINT(33.431402179673704 -118.4871696020303) bank32239 +32240 POINT(34.78912015257577 -119.14722444846518) bank32240 +32241 POINT(33.1299373419886 -119.11454011900747) bank32241 +32242 POINT(34.30220900721578 -117.51335776690622) bank32242 +32243 POINT(33.379442457009546 -117.44951654524624) bank32243 +32244 POINT(34.63497661642859 -118.68643214599709) bank32244 +32245 POINT(33.598529159433994 -119.23243452535718) bank32245 +32246 POINT(33.10745381486941 -118.97628937573468) bank32246 +32247 POINT(33.081633405604585 -118.80613672798584) bank32247 +32248 POINT(34.61632433091055 -117.67883115629172) bank32248 +32249 POINT(33.802964160607004 -117.47022849365293) bank32249 +32250 POINT(33.63838392771139 -118.38788004832533) bank32250 +32251 POINT(33.51266792268222 -118.81531307520713) bank32251 +32252 POINT(33.65345832985031 -117.63464278888704) bank32252 +32253 POINT(33.7742175157467 -118.9176621413057) bank32253 +32254 POINT(33.6534409800654 -119.18192115470445) bank32254 +32255 POINT(33.45252728385955 -118.57638306688175) bank32255 +32256 POINT(34.47368637983835 -118.7557571821211) bank32256 +32257 POINT(34.536790996131 -118.41453043802994) bank32257 +32258 POINT(33.7269458720341 -118.21459541931293) bank32258 +32259 POINT(33.848075608285534 -118.3978404942111) bank32259 +32260 POINT(33.90375144427946 -119.0615263153264) bank32260 +32261 POINT(34.763051633691674 -118.92925722748134) bank32261 +32262 POINT(33.30056949622876 -119.02226416736593) bank32262 +32263 POINT(34.1030494960346 -118.33558922093117) bank32263 +32264 POINT(33.625809616089505 -118.8401564772212) bank32264 +32265 POINT(33.1936337604164 -117.49310284854454) bank32265 +32266 POINT(33.963761908786935 -117.49934830587233) bank32266 +32267 POINT(33.44638929682502 -117.91952273784057) bank32267 +32268 POINT(34.11165223644454 -117.85201612559106) bank32268 +32269 POINT(33.18856476691558 -117.39509963820689) bank32269 +32270 POINT(34.43374156162535 -117.46746522068707) bank32270 +32271 POINT(33.10412873004706 -119.17023653664644) bank32271 +32272 POINT(33.975198699640195 -118.13719032676433) bank32272 +32273 POINT(34.971839544732156 -118.8439615645813) bank32273 +32274 POINT(34.74964673979126 -118.48244212124969) bank32274 +32275 POINT(33.40322115070343 -117.40616029386656) bank32275 +32276 POINT(34.02715724687195 -118.56301848729085) bank32276 +32277 POINT(34.223415345548474 -119.22975503237086) bank32277 +32278 POINT(33.766798054409506 -117.34425519255909) bank32278 +32279 POINT(34.177899809345604 -117.67242767234274) bank32279 +32280 POINT(33.382630104284154 -118.69761436045157) bank32280 +32281 POINT(34.527198916693465 -117.9733908866395) bank32281 +32282 POINT(34.289219592139595 -118.20787861751506) bank32282 +32283 POINT(33.87415312289453 -118.28544830118864) bank32283 +32284 POINT(33.844288634999785 -118.71271254156181) bank32284 +32285 POINT(33.35041434552314 -117.27253510910427) bank32285 +32286 POINT(34.32265064518033 -117.47767818101688) bank32286 +32287 POINT(33.38950836957631 -118.61078052843415) bank32287 +32288 POINT(34.97932863971844 -117.79684950188413) bank32288 +32289 POINT(34.27849491222095 -117.44309452844396) bank32289 +32290 POINT(33.89853694027547 -117.98467759091358) bank32290 +32291 POINT(33.497361350850035 -117.7450013406828) bank32291 +32292 POINT(33.147517375014615 -117.66147237009554) bank32292 +32293 POINT(33.324716234727866 -119.2329827069999) bank32293 +32294 POINT(34.803760549755715 -117.74514661838934) bank32294 +32295 POINT(34.422459493580156 -118.6068150875274) bank32295 +32296 POINT(34.243055567690966 -117.26931148791174) bank32296 +32297 POINT(34.31726319075068 -118.29594765770165) bank32297 +32298 POINT(34.83940926751151 -118.42354169345788) bank32298 +32299 POINT(34.28774705523454 -117.61267351789226) bank32299 +32300 POINT(34.87298770279584 -117.68061532222917) bank32300 +32301 POINT(34.900262469882136 -117.87297247309145) bank32301 +32302 POINT(34.31068866915189 -118.88406659230374) bank32302 +32303 POINT(33.54836729161345 -117.77318544958369) bank32303 +32304 POINT(33.65467277360996 -118.6329610656551) bank32304 +32305 POINT(34.446609312655795 -119.05107520098615) bank32305 +32306 POINT(34.49586308896973 -117.95545387452795) bank32306 +32307 POINT(34.12675513931262 -118.00337653904779) bank32307 +32308 POINT(33.45475020684605 -117.68065361443405) bank32308 +32309 POINT(35.03098456876106 -117.27848487585666) bank32309 +32310 POINT(33.93519033860153 -118.43677442202784) bank32310 +32311 POINT(34.86096543914611 -118.51040749042723) bank32311 +32312 POINT(33.91938022950021 -117.83666028654721) bank32312 +32313 POINT(34.83990938634158 -119.23975291086681) bank32313 +32314 POINT(33.415519577180085 -117.71743762314081) bank32314 +32315 POINT(33.521978739695946 -119.15840214964207) bank32315 +32316 POINT(34.03058510458631 -117.6132344158089) bank32316 +32317 POINT(34.727202349853606 -117.42989587976702) bank32317 +32318 POINT(34.67831613225636 -117.37082810605823) bank32318 +32319 POINT(33.993326522144606 -119.000680240532) bank32319 +32320 POINT(34.20059617485996 -119.19552807816301) bank32320 +32321 POINT(34.85747174173248 -117.98515442964329) bank32321 +32322 POINT(33.27324442333517 -119.08830358045421) bank32322 +32323 POINT(33.70675365023894 -119.18375323357363) bank32323 +32324 POINT(33.98178716433024 -117.82708954031033) bank32324 +32325 POINT(34.81149267491836 -118.52246604607613) bank32325 +32326 POINT(34.12205952919135 -117.32397449098723) bank32326 +32327 POINT(34.02930501420434 -118.69824191817584) bank32327 +32328 POINT(33.56893238050608 -117.5616403554429) bank32328 +32329 POINT(33.38333274591474 -119.05678325244496) bank32329 +32330 POINT(34.330857133443736 -118.87506101540686) bank32330 +32331 POINT(34.386356272326736 -118.35109819550304) bank32331 +32332 POINT(34.572124921137444 -118.59000446618862) bank32332 +32333 POINT(33.99549638676122 -117.86167467887843) bank32333 +32334 POINT(33.19872819668846 -118.38092274449755) bank32334 +32335 POINT(34.27022198663977 -117.27276774502216) bank32335 +32336 POINT(34.5777916327292 -117.40065744889641) bank32336 +32337 POINT(33.62729185720507 -117.5405405328418) bank32337 +32338 POINT(33.10905776151741 -118.23997433344172) bank32338 +32339 POINT(33.790592494503876 -118.81364259104087) bank32339 +32340 POINT(34.8392579642355 -118.0768756969731) bank32340 +32341 POINT(34.418435213508765 -117.57027554194141) bank32341 +32342 POINT(33.149670882028055 -117.82077525483315) bank32342 +32343 POINT(33.60612715711228 -118.89266010034801) bank32343 +32344 POINT(34.09318461091711 -118.90454963840047) bank32344 +32345 POINT(35.00994790574687 -119.02479249409942) bank32345 +32346 POINT(33.83770515799177 -117.49399073363067) bank32346 +32347 POINT(33.31588943932348 -118.56001472413593) bank32347 +32348 POINT(34.437001065979864 -118.41167584952599) bank32348 +32349 POINT(33.35503251714943 -119.16316928186684) bank32349 +32350 POINT(33.74623345614317 -118.46954507120844) bank32350 +32351 POINT(33.411397400494124 -117.97334984550406) bank32351 +32352 POINT(33.56118435670641 -118.39214159134445) bank32352 +32353 POINT(33.91336923386365 -118.77086676891088) bank32353 +32354 POINT(34.93094423189407 -117.89789038219442) bank32354 +32355 POINT(34.281809943329044 -118.00901925414637) bank32355 +32356 POINT(34.67607711185653 -118.24585400441764) bank32356 +32357 POINT(33.27863638166321 -117.76942416158735) bank32357 +32358 POINT(33.28358076030098 -117.67504244443519) bank32358 +32359 POINT(34.79332094191506 -118.57729972393031) bank32359 +32360 POINT(33.895940202882954 -119.23450402878937) bank32360 +32361 POINT(34.591618744392285 -118.86218278799672) bank32361 +32362 POINT(34.70503403647573 -119.22300059798327) bank32362 +32363 POINT(34.18169775858601 -118.81469132970368) bank32363 +32364 POINT(33.49504341889987 -118.77978061851042) bank32364 +32365 POINT(33.71754121772128 -117.73048096351076) bank32365 +32366 POINT(34.9651397896208 -117.69316123980917) bank32366 +32367 POINT(33.52643638619445 -117.69619765473078) bank32367 +32368 POINT(34.391692117493825 -117.95748677514409) bank32368 +32369 POINT(33.12503530411835 -118.79975268065547) bank32369 +32370 POINT(34.895684242019335 -118.49304466678602) bank32370 +32371 POINT(33.72782176022331 -117.25718192038813) bank32371 +32372 POINT(34.527145419601496 -119.18908593103387) bank32372 +32373 POINT(33.4133111592619 -118.08257845012601) bank32373 +32374 POINT(34.513614782481724 -118.51677690322771) bank32374 +32375 POINT(33.28051288054293 -117.68330060189423) bank32375 +32376 POINT(34.852595381470834 -117.82112144865938) bank32376 +32377 POINT(33.865313973069185 -118.90991144776108) bank32377 +32378 POINT(34.99040774772573 -119.15222426483002) bank32378 +32379 POINT(33.938044527589895 -118.11822473789147) bank32379 +32380 POINT(34.96938624242657 -118.92311094113207) bank32380 +32381 POINT(33.28451563790986 -118.6585624721812) bank32381 +32382 POINT(34.199498398246334 -118.56398840133576) bank32382 +32383 POINT(33.109888174609566 -118.35107384571015) bank32383 +32384 POINT(33.404470404984615 -118.96575053159422) bank32384 +32385 POINT(34.62665776284592 -118.44179412025932) bank32385 +32386 POINT(33.41876904762202 -117.42686308254508) bank32386 +32387 POINT(34.1911508093718 -117.46868321510412) bank32387 +32388 POINT(34.537517999544676 -119.09728163695152) bank32388 +32389 POINT(34.3418725985334 -118.44176041362962) bank32389 +32390 POINT(34.10170350168176 -117.6829442243369) bank32390 +32391 POINT(34.238552980797344 -118.17896085393326) bank32391 +32392 POINT(33.734965542323046 -118.8521457509976) bank32392 +32393 POINT(33.51067343891642 -118.52975186468738) bank32393 +32394 POINT(34.17347682216527 -118.361553557301) bank32394 +32395 POINT(33.700303596503005 -118.88922043800139) bank32395 +32396 POINT(33.740693719198354 -117.61735676942455) bank32396 +32397 POINT(33.08370803579525 -118.20280196884518) bank32397 +32398 POINT(33.11652765788312 -118.51901598991898) bank32398 +32399 POINT(34.54798777336244 -117.54272566272788) bank32399 +32400 POINT(33.3148877288664 -118.36345057463771) bank32400 +32401 POINT(34.52167324575339 -117.89188542684217) bank32401 +32402 POINT(33.8167194272996 -117.98622793375631) bank32402 +32403 POINT(34.86885012827002 -118.78991949450474) bank32403 +32404 POINT(33.94571120550523 -118.77553756798915) bank32404 +32405 POINT(34.03158436429057 -117.75734367837086) bank32405 +32406 POINT(34.694026954929754 -118.40228730102683) bank32406 +32407 POINT(34.116094193673376 -117.57789632969543) bank32407 +32408 POINT(33.426468222082796 -118.70333120607964) bank32408 +32409 POINT(34.85466721133165 -118.75330707631478) bank32409 +32410 POINT(33.83255109367496 -117.30491727042326) bank32410 +32411 POINT(34.05125086240469 -117.67546534621788) bank32411 +32412 POINT(34.94751903524525 -117.45232567928237) bank32412 +32413 POINT(33.83622394225133 -118.363499174019) bank32413 +32414 POINT(34.42336530565323 -118.01303535610899) bank32414 +32415 POINT(33.22624893834386 -117.84333359640024) bank32415 +32416 POINT(34.74975831521912 -117.28992557073745) bank32416 +32417 POINT(33.42425396591684 -118.83214976259764) bank32417 +32418 POINT(34.704682096145135 -117.48093380778974) bank32418 +32419 POINT(34.383246765031664 -119.0309281794902) bank32419 +32420 POINT(34.0568398406042 -117.90079291867589) bank32420 +32421 POINT(34.819143892964036 -118.51890568284284) bank32421 +32422 POINT(33.26716935215695 -118.95401994984331) bank32422 +32423 POINT(33.635974161206505 -118.07219695955773) bank32423 +32424 POINT(34.23658075499393 -118.98535322549277) bank32424 +32425 POINT(33.60478160689555 -118.08924927001006) bank32425 +32426 POINT(33.60471506242415 -118.78393622491089) bank32426 +32427 POINT(33.07760614931245 -118.03755795089886) bank32427 +32428 POINT(34.44653080919118 -118.82227500204016) bank32428 +32429 POINT(33.28257662973969 -117.91451134202238) bank32429 +32430 POINT(34.40414456370403 -118.87955451486694) bank32430 +32431 POINT(33.85615475478426 -117.89931392840582) bank32431 +32432 POINT(33.810642351428854 -118.42320784090204) bank32432 +32433 POINT(33.24029777604213 -117.49484456991921) bank32433 +32434 POINT(34.43217558308286 -118.14333972077006) bank32434 +32435 POINT(34.216766405060525 -118.10123952266683) bank32435 +32436 POINT(33.231678307102 -117.86308018413956) bank32436 +32437 POINT(34.65910521674091 -117.89371685517962) bank32437 +32438 POINT(34.05658491643131 -118.75314717338952) bank32438 +32439 POINT(33.61344643771324 -119.21530957942805) bank32439 +32440 POINT(34.345212566311375 -118.36500345109997) bank32440 +32441 POINT(33.598400776694945 -117.35964234766497) bank32441 +32442 POINT(34.15540956507228 -119.12306752947175) bank32442 +32443 POINT(34.89628505438031 -118.48699150389524) bank32443 +32444 POINT(33.13281650264504 -118.29449174687) bank32444 +32445 POINT(33.79883277447395 -118.85539455829688) bank32445 +32446 POINT(33.17970520803357 -117.8844947399147) bank32446 +32447 POINT(34.39315931583794 -118.53602388957115) bank32447 +32448 POINT(34.543847508223905 -118.58944686832086) bank32448 +32449 POINT(34.60337628375584 -118.83384844622219) bank32449 +32450 POINT(34.06495831480972 -118.08064320215698) bank32450 +32451 POINT(34.354937393100435 -118.57386189096022) bank32451 +32452 POINT(34.07756961771924 -119.22056067593377) bank32452 +32453 POINT(33.85272516163158 -117.48197804845766) bank32453 +32454 POINT(33.57962674810412 -119.15115540913831) bank32454 +32455 POINT(34.448696727152296 -118.12571211734826) bank32455 +32456 POINT(33.90318576672337 -117.64836583704016) bank32456 +32457 POINT(33.80696669685737 -118.69276608965009) bank32457 +32458 POINT(33.89570635693577 -118.92765244676328) bank32458 +32459 POINT(34.42478546057601 -118.49663278404051) bank32459 +32460 POINT(34.25554146679194 -117.87337747860217) bank32460 +32461 POINT(34.42053447834643 -118.14979180459164) bank32461 +32462 POINT(33.7115280790863 -117.27918464318404) bank32462 +32463 POINT(33.5618019881191 -118.26743293986274) bank32463 +32464 POINT(34.59424701188597 -117.81950485854354) bank32464 +32465 POINT(33.598100040294725 -117.83383033252444) bank32465 +32466 POINT(33.134705531324336 -118.5470871400844) bank32466 +32467 POINT(34.49583982986973 -118.02873658054733) bank32467 +32468 POINT(34.32907523189519 -118.67098194740998) bank32468 +32469 POINT(33.54826409463573 -118.20530452969493) bank32469 +32470 POINT(34.83748982336748 -118.90421480067485) bank32470 +32471 POINT(33.37040905762126 -118.31035898760638) bank32471 +32472 POINT(34.15751825990361 -118.51892575846234) bank32472 +32473 POINT(34.48084568819832 -117.65426998951472) bank32473 +32474 POINT(34.736308150661856 -119.16539478193603) bank32474 +32475 POINT(34.634607141663665 -117.26461004276085) bank32475 +32476 POINT(34.7061861613287 -117.65091184124141) bank32476 +32477 POINT(34.687373915576615 -118.78053323845313) bank32477 +32478 POINT(34.054456036904234 -118.41248008670975) bank32478 +32479 POINT(34.74962205305902 -117.53142365773262) bank32479 +32480 POINT(34.73583055030789 -118.94347757628266) bank32480 +32481 POINT(33.52754605143815 -118.86970765991182) bank32481 +32482 POINT(34.44831910749489 -117.71255144512982) bank32482 +32483 POINT(34.756236558179744 -117.93810584551446) bank32483 +32484 POINT(34.87960588814466 -118.6936334618467) bank32484 +32485 POINT(34.3090241935974 -118.34027172772589) bank32485 +32486 POINT(33.70614197448491 -117.58946017079056) bank32486 +32487 POINT(34.07762054921236 -117.31039701212707) bank32487 +32488 POINT(33.41559047136528 -118.94508614698543) bank32488 +32489 POINT(34.77322054273231 -119.04898285309908) bank32489 +32490 POINT(33.65443373153728 -118.55793968202984) bank32490 +32491 POINT(34.3757845983793 -118.27961850326183) bank32491 +32492 POINT(33.276301560487 -117.89086052237612) bank32492 +32493 POINT(34.171003157271244 -118.54656435483768) bank32493 +32494 POINT(34.05346472589465 -118.20060362235236) bank32494 +32495 POINT(33.529831427139705 -118.82401310727994) bank32495 +32496 POINT(33.626583917233376 -117.72535260625892) bank32496 +32497 POINT(33.3036967688682 -119.11076231236873) bank32497 +32498 POINT(34.96426247656186 -117.54800118071252) bank32498 +32499 POINT(33.86318531873357 -118.62920835211492) bank32499 +32500 POINT(33.67407377710211 -117.51238027086272) bank32500 +32501 POINT(33.68369429416153 -117.79766419839791) bank32501 +32502 POINT(34.32937313944569 -117.82820196364263) bank32502 +32503 POINT(33.25800081064534 -118.74841923120098) bank32503 +32504 POINT(33.93879857648398 -118.60302582479221) bank32504 +32505 POINT(33.940964527697716 -117.88964047040278) bank32505 +32506 POINT(34.595019919232215 -117.26377992374675) bank32506 +32507 POINT(34.75497093566805 -119.200677838169) bank32507 +32508 POINT(33.59734483442501 -117.79069487166527) bank32508 +32509 POINT(33.53680295386797 -118.3386395424527) bank32509 +32510 POINT(34.493194132347064 -118.31339116753803) bank32510 +32511 POINT(34.16297122596129 -117.41645542499815) bank32511 +32512 POINT(34.319184783975835 -118.1670188962429) bank32512 +32513 POINT(34.3737227981259 -118.99392738682047) bank32513 +32514 POINT(34.11450995402766 -117.70657201023383) bank32514 +32515 POINT(34.76449743490767 -118.28924389947082) bank32515 +32516 POINT(33.93267690793312 -119.23623593175958) bank32516 +32517 POINT(33.78459962120413 -117.58258372870947) bank32517 +32518 POINT(34.179353393879246 -117.92164645123218) bank32518 +32519 POINT(34.43511712672684 -117.91395622867562) bank32519 +32520 POINT(34.130321686508445 -119.17324803338883) bank32520 +32521 POINT(34.62602496018903 -118.81197506724133) bank32521 +32522 POINT(33.26465864368558 -118.66523959311316) bank32522 +32523 POINT(33.51864095071648 -118.79972841096193) bank32523 +32524 POINT(34.653890072785 -117.95420858754822) bank32524 +32525 POINT(34.351794717942134 -118.89068141392443) bank32525 +32526 POINT(33.259059831217556 -118.28003301750499) bank32526 +32527 POINT(34.95854479181208 -118.06217189490803) bank32527 +32528 POINT(33.9033671711917 -118.39541365736562) bank32528 +32529 POINT(33.72112566895908 -118.86918207885114) bank32529 +32530 POINT(34.84957397285675 -117.37166228674269) bank32530 +32531 POINT(34.58379674923741 -117.29206045744881) bank32531 +32532 POINT(33.165806190402705 -118.32324392458574) bank32532 +32533 POINT(34.477235426621164 -117.90278866095778) bank32533 +32534 POINT(33.59728779551308 -117.3401957256098) bank32534 +32535 POINT(34.26824651910517 -117.55499086344373) bank32535 +32536 POINT(33.11729051885516 -118.83211629594877) bank32536 +32537 POINT(34.67225198701811 -119.2078336220042) bank32537 +32538 POINT(33.9755058230088 -118.81317109652242) bank32538 +32539 POINT(34.65055878913704 -118.01188111353922) bank32539 +32540 POINT(34.92790074413449 -117.87626321518921) bank32540 +32541 POINT(33.22093503737871 -117.94909588657427) bank32541 +32542 POINT(34.87730516561264 -117.58566058242143) bank32542 +32543 POINT(34.0560862951558 -118.75858037472746) bank32543 +32544 POINT(33.37232231567877 -118.49327038052603) bank32544 +32545 POINT(34.834015448531154 -118.6633852694239) bank32545 +32546 POINT(34.686050358649496 -117.34272196747011) bank32546 +32547 POINT(34.07281865395743 -118.79971708154112) bank32547 +32548 POINT(33.78063006320256 -119.14706111554909) bank32548 +32549 POINT(34.012264141503806 -118.49228670585305) bank32549 +32550 POINT(33.150761023665176 -118.02810399521825) bank32550 +32551 POINT(34.97712774028543 -117.74982719615151) bank32551 +32552 POINT(33.50946473380928 -118.49099520673559) bank32552 +32553 POINT(33.19200271884811 -118.88236100006624) bank32553 +32554 POINT(33.760628179011505 -117.5053936267421) bank32554 +32555 POINT(33.31372873312032 -118.59909834415858) bank32555 +32556 POINT(33.56128912544696 -119.20304248822445) bank32556 +32557 POINT(34.41090066530711 -118.89618711675524) bank32557 +32558 POINT(33.78378553055123 -117.52916725946054) bank32558 +32559 POINT(33.70094188965507 -118.69393106750194) bank32559 +32560 POINT(33.45287722664556 -118.15602130821144) bank32560 +32561 POINT(35.04353609344573 -118.58074496867502) bank32561 +32562 POINT(33.079595865892934 -117.2689123522608) bank32562 +32563 POINT(34.07600110255268 -118.93180539029787) bank32563 +32564 POINT(33.31964366657651 -119.15023515058132) bank32564 +32565 POINT(33.54610258962881 -119.09527274339109) bank32565 +32566 POINT(33.95924337830898 -118.2948018217613) bank32566 +32567 POINT(33.318841978176316 -117.2588746824484) bank32567 +32568 POINT(33.15382140288448 -117.79642822541497) bank32568 +32569 POINT(33.58242258149325 -118.22731559845971) bank32569 +32570 POINT(33.17953364088997 -117.59290268372048) bank32570 +32571 POINT(34.820301467068795 -117.2533942626003) bank32571 +32572 POINT(34.53529547765061 -118.06573931566626) bank32572 +32573 POINT(34.37088425355985 -117.58880177312547) bank32573 +32574 POINT(34.62677613395704 -118.46837387154207) bank32574 +32575 POINT(34.02444111343431 -117.61143888379861) bank32575 +32576 POINT(33.38142712692918 -118.9509813489388) bank32576 +32577 POINT(33.833129338384175 -118.74814048760018) bank32577 +32578 POINT(33.630280951542304 -118.37627715110597) bank32578 +32579 POINT(34.276799281057 -117.76215093914868) bank32579 +32580 POINT(34.4850791298634 -117.50806884386252) bank32580 +32581 POINT(34.422287768905335 -117.46590647101745) bank32581 +32582 POINT(33.37113842244412 -118.69928429388388) bank32582 +32583 POINT(33.92289178662823 -118.10818393522285) bank32583 +32584 POINT(33.46666065564287 -118.99546258900607) bank32584 +32585 POINT(34.4102015209877 -118.41587114906505) bank32585 +32586 POINT(34.602180157140204 -118.77230458600471) bank32586 +32587 POINT(34.16775734617982 -118.03904870769894) bank32587 +32588 POINT(34.22787849889087 -117.71456009284194) bank32588 +32589 POINT(33.60105247049541 -118.9945693431469) bank32589 +32590 POINT(34.328992307060396 -117.55427362577643) bank32590 +32591 POINT(33.44454229854531 -118.91743176868206) bank32591 +32592 POINT(33.47794166945489 -119.0684326733466) bank32592 +32593 POINT(34.60626983220724 -119.14941068440419) bank32593 +32594 POINT(34.4054534061859 -117.55475259311947) bank32594 +32595 POINT(34.76229564394733 -117.79276409362289) bank32595 +32596 POINT(35.02100311304641 -119.07199241135518) bank32596 +32597 POINT(34.07568191511556 -117.25081343825752) bank32597 +32598 POINT(33.087212092683366 -118.33426039177697) bank32598 +32599 POINT(33.32786147229016 -118.35759077379053) bank32599 +32600 POINT(34.501133331828306 -118.98834125901907) bank32600 +32601 POINT(33.198945348824836 -118.60275003019802) bank32601 +32602 POINT(34.42578489717993 -118.42377422707263) bank32602 +32603 POINT(34.23115748729322 -117.93205861813861) bank32603 +32604 POINT(33.98040621168593 -117.52220057041981) bank32604 +32605 POINT(34.336920869349285 -118.35974124787157) bank32605 +32606 POINT(33.57014466228851 -119.01097499629178) bank32606 +32607 POINT(33.81320769459188 -118.17462887764471) bank32607 +32608 POINT(35.02716296977759 -119.16574321293623) bank32608 +32609 POINT(33.53936618803116 -118.69974917210622) bank32609 +32610 POINT(33.30926898916973 -118.26404451226803) bank32610 +32611 POINT(33.537392919003864 -117.75284595431386) bank32611 +32612 POINT(34.59386994474487 -118.46096693964608) bank32612 +32613 POINT(34.538443642936336 -118.22854359934813) bank32613 +32614 POINT(33.730456805919744 -117.53999618223703) bank32614 +32615 POINT(33.13832714612549 -118.07587301590759) bank32615 +32616 POINT(34.975339981180156 -118.28642642934342) bank32616 +32617 POINT(33.09109309183511 -119.12560755354204) bank32617 +32618 POINT(33.46718813020725 -118.53295988381056) bank32618 +32619 POINT(34.93150243929645 -118.43274765663776) bank32619 +32620 POINT(34.97660765636428 -117.32996258489328) bank32620 +32621 POINT(33.701310901088355 -117.5798279780643) bank32621 +32622 POINT(33.432880438951926 -117.95518887917895) bank32622 +32623 POINT(33.426557734070144 -118.70603604857286) bank32623 +32624 POINT(33.92840442811888 -117.30172498165153) bank32624 +32625 POINT(34.97113544221269 -118.76413213260932) bank32625 +32626 POINT(33.103988158561535 -117.74403265872941) bank32626 +32627 POINT(34.157414380228616 -117.57243638947278) bank32627 +32628 POINT(34.681457956892196 -118.9993326208809) bank32628 +32629 POINT(34.44513956783218 -118.04213905959521) bank32629 +32630 POINT(34.38319174341922 -118.64275070130937) bank32630 +32631 POINT(33.85087926959023 -118.0474421050929) bank32631 +32632 POINT(33.09633538041382 -117.70109825946624) bank32632 +32633 POINT(33.43286842004955 -119.01622622332046) bank32633 +32634 POINT(33.30385576595118 -118.41362277686741) bank32634 +32635 POINT(35.03239265929814 -119.12022681558447) bank32635 +32636 POINT(33.84360421346893 -117.54243356415614) bank32636 +32637 POINT(33.10021396098759 -118.57487538218646) bank32637 +32638 POINT(34.52805174177682 -118.92171587848361) bank32638 +32639 POINT(33.31835401883343 -117.38570976321414) bank32639 +32640 POINT(34.274433545388156 -118.19012177361826) bank32640 +32641 POINT(33.65400764720815 -118.92410344765088) bank32641 +32642 POINT(33.94340780964941 -117.29554562874901) bank32642 +32643 POINT(33.21684253675877 -118.46071502933226) bank32643 +32644 POINT(33.29327434686852 -119.23793017809767) bank32644 +32645 POINT(33.091283831372486 -119.02097614358843) bank32645 +32646 POINT(33.343585013090866 -118.34697407139984) bank32646 +32647 POINT(34.828988917428994 -118.99062477596507) bank32647 +32648 POINT(33.78111420575152 -118.90481769898382) bank32648 +32649 POINT(34.90111538033314 -118.88540963308928) bank32649 +32650 POINT(34.720302883292476 -119.07926316812896) bank32650 +32651 POINT(34.29881873867585 -117.70096534939316) bank32651 +32652 POINT(33.43101049121839 -118.15932769437009) bank32652 +32653 POINT(33.26167180222275 -117.67914482611585) bank32653 +32654 POINT(33.57760600735272 -118.26110215431589) bank32654 +32655 POINT(34.19812378419264 -119.0951784807874) bank32655 +32656 POINT(34.64281977962794 -117.30256661538353) bank32656 +32657 POINT(33.569065336850755 -117.7165947726113) bank32657 +32658 POINT(34.5385014989745 -117.37372907339282) bank32658 +32659 POINT(33.067794101550604 -117.74197514659214) bank32659 +32660 POINT(33.55418105774239 -118.7883668110989) bank32660 +32661 POINT(33.09942842550398 -118.71925632517244) bank32661 +32662 POINT(33.187927745544656 -118.03117347171485) bank32662 +32663 POINT(33.62181614121732 -118.68592591259593) bank32663 +32664 POINT(33.854925009032996 -119.13816838511653) bank32664 +32665 POINT(34.238390237596214 -118.57254912217326) bank32665 +32666 POINT(33.35892417481049 -118.5191917149196) bank32666 +32667 POINT(33.33113887246864 -118.29233426618029) bank32667 +32668 POINT(35.02831429413896 -117.68425572595852) bank32668 +32669 POINT(34.266431797721076 -118.16494809642631) bank32669 +32670 POINT(34.41489558059496 -119.18097572132952) bank32670 +32671 POINT(34.25233462798802 -118.01070534833389) bank32671 +32672 POINT(33.440930583957204 -118.55277234127995) bank32672 +32673 POINT(33.298215250558805 -118.57957116846448) bank32673 +32674 POINT(33.32955663610952 -117.86965898689226) bank32674 +32675 POINT(33.67396005590316 -117.24844248814479) bank32675 +32676 POINT(34.51093081570622 -118.94973348209686) bank32676 +32677 POINT(34.718233062313104 -118.11661857039643) bank32677 +32678 POINT(33.67387255658162 -117.42017899939017) bank32678 +32679 POINT(34.757556790174256 -117.76649351123493) bank32679 +32680 POINT(33.89603400101793 -117.67462164623643) bank32680 +32681 POINT(34.167093907918016 -118.59120535183382) bank32681 +32682 POINT(33.44317727434252 -119.10315247182727) bank32682 +32683 POINT(33.110949059772146 -119.03834269086133) bank32683 +32684 POINT(34.55434885157733 -118.75447544146262) bank32684 +32685 POINT(33.64145894772232 -118.14300877236111) bank32685 +32686 POINT(34.156535342706626 -117.89107104770234) bank32686 +32687 POINT(33.33789310060106 -117.88435896917017) bank32687 +32688 POINT(34.950793146722056 -118.96190314600314) bank32688 +32689 POINT(35.019335054458644 -118.51814272068233) bank32689 +32690 POINT(33.60085642536569 -118.57145475930874) bank32690 +32691 POINT(33.50166373008113 -118.163227568965) bank32691 +32692 POINT(33.35828093636439 -117.97982629855318) bank32692 +32693 POINT(33.30511081609477 -117.2684831791309) bank32693 +32694 POINT(34.32653786835343 -119.1587403791196) bank32694 +32695 POINT(33.84992617962402 -118.10318937780633) bank32695 +32696 POINT(34.125804954347835 -117.87599178234584) bank32696 +32697 POINT(33.92681374311489 -119.0809454644184) bank32697 +32698 POINT(34.107925655763495 -118.64129452399001) bank32698 +32699 POINT(33.21590062134155 -117.43434260384994) bank32699 +32700 POINT(33.62695482572059 -117.80372149616433) bank32700 +32701 POINT(34.98573217943336 -118.12345487744044) bank32701 +32702 POINT(33.246818528284045 -118.28885476220799) bank32702 +32703 POINT(33.76053911687123 -119.05907883207853) bank32703 +32704 POINT(33.681654377036516 -117.68573501269361) bank32704 +32705 POINT(34.81365581520136 -119.13331438835064) bank32705 +32706 POINT(35.01246696952076 -118.85974663390475) bank32706 +32707 POINT(33.201357618665156 -117.81887014147401) bank32707 +32708 POINT(33.67885887886795 -118.96256978446849) bank32708 +32709 POINT(33.15825324345108 -118.10428454197906) bank32709 +32710 POINT(33.916702439071535 -118.27591164794907) bank32710 +32711 POINT(33.35339148647827 -118.15120721523024) bank32711 +32712 POINT(34.62842410122209 -118.2472005975177) bank32712 +32713 POINT(34.75380317201507 -117.92691969773126) bank32713 +32714 POINT(34.52226416169995 -117.8291396972023) bank32714 +32715 POINT(34.50862090214551 -117.41031017440864) bank32715 +32716 POINT(33.15500427073898 -117.66541432725876) bank32716 +32717 POINT(35.018476654373714 -118.29857187899398) bank32717 +32718 POINT(34.40114836667538 -118.05492541573959) bank32718 +32719 POINT(33.68025992592368 -118.71556888820052) bank32719 +32720 POINT(33.16585620836431 -119.03263477230563) bank32720 +32721 POINT(34.07830098253143 -117.8556695491418) bank32721 +32722 POINT(33.12876823391125 -118.62330520520877) bank32722 +32723 POINT(33.65553279193265 -118.14974407239359) bank32723 +32724 POINT(34.01492853714253 -118.35312922650633) bank32724 +32725 POINT(33.882443089222384 -118.5184995611473) bank32725 +32726 POINT(34.618070210746524 -119.10474352624183) bank32726 +32727 POINT(33.67305072175068 -117.81519191805691) bank32727 +32728 POINT(33.615168353524005 -117.54879077543568) bank32728 +32729 POINT(34.88290761944067 -118.79076189857325) bank32729 +32730 POINT(33.31322204956712 -118.73668011469285) bank32730 +32731 POINT(34.1948045683039 -118.20578501453468) bank32731 +32732 POINT(33.67791735326196 -118.97042265231723) bank32732 +32733 POINT(34.454649350597684 -118.86609705230623) bank32733 +32734 POINT(34.643509038171075 -117.63981416238808) bank32734 +32735 POINT(34.708281378283736 -119.09143811990488) bank32735 +32736 POINT(35.03476934735158 -118.48517703514213) bank32736 +32737 POINT(33.245053018194774 -118.19945927586542) bank32737 +32738 POINT(33.97071453104512 -118.2493386668344) bank32738 +32739 POINT(34.000884705895274 -118.23513054287476) bank32739 +32740 POINT(33.87573935393886 -117.35592730870815) bank32740 +32741 POINT(33.96784073834513 -118.8504943863484) bank32741 +32742 POINT(33.824157177854374 -119.07199984954615) bank32742 +32743 POINT(33.698940687419565 -118.19618322474196) bank32743 +32744 POINT(33.988947869690584 -117.9615993688305) bank32744 +32745 POINT(34.80909500455252 -119.18458757319169) bank32745 +32746 POINT(33.11962822662368 -118.27177381278129) bank32746 +32747 POINT(34.281377540383346 -117.93004069768625) bank32747 +32748 POINT(33.38446008790279 -118.67963568627265) bank32748 +32749 POINT(34.03359885722478 -117.40694730998455) bank32749 +32750 POINT(34.52829103392051 -117.75083159336305) bank32750 +32751 POINT(33.229762622915096 -118.81385868937942) bank32751 +32752 POINT(33.37745697087593 -118.10899309187714) bank32752 +32753 POINT(33.1316464535461 -118.12090473629239) bank32753 +32754 POINT(34.46585111360464 -118.24579145647486) bank32754 +32755 POINT(34.40851065744737 -117.68354625709371) bank32755 +32756 POINT(35.046375527139055 -117.49098175411415) bank32756 +32757 POINT(33.221623860313244 -117.53109488265497) bank32757 +32758 POINT(34.26587412076115 -118.76280872798885) bank32758 +32759 POINT(34.541908781798625 -119.12345852744673) bank32759 +32760 POINT(33.603021703218 -117.30893140067653) bank32760 +32761 POINT(33.22782303109135 -118.50635464846741) bank32761 +32762 POINT(34.07550904819849 -118.36047825044994) bank32762 +32763 POINT(33.42210020903801 -118.02217349753785) bank32763 +32764 POINT(34.38884676684662 -118.04634935587674) bank32764 +32765 POINT(34.76080550657124 -117.5691820945492) bank32765 +32766 POINT(34.96712544469996 -117.28811616231664) bank32766 +32767 POINT(33.97305752430356 -117.35322963601166) bank32767 +32768 POINT(33.34319251362095 -118.7593234832859) bank32768 +32769 POINT(34.499296973716824 -117.2950016338193) bank32769 +32770 POINT(33.20050641345848 -119.02287208970026) bank32770 +32771 POINT(33.966062596837425 -117.39333894259687) bank32771 +32772 POINT(33.34473017589789 -118.69751010304864) bank32772 +32773 POINT(33.58142798518863 -118.37291051990755) bank32773 +32774 POINT(34.87406521108383 -117.8788157304972) bank32774 +32775 POINT(33.89027801443242 -117.3678805709578) bank32775 +32776 POINT(34.71796462697575 -118.48831456987486) bank32776 +32777 POINT(34.53314725022851 -118.95469056389481) bank32777 +32778 POINT(34.936488580366124 -117.30697737337138) bank32778 +32779 POINT(34.357109986490066 -118.4955982737877) bank32779 +32780 POINT(34.786130624002446 -117.32459473422561) bank32780 +32781 POINT(34.501690698821484 -118.7462211858739) bank32781 +32782 POINT(34.668924573897236 -117.59119891143656) bank32782 +32783 POINT(33.947420026611105 -117.65077373055449) bank32783 +32784 POINT(34.776069197813825 -118.16523504936589) bank32784 +32785 POINT(33.745129077781606 -118.1610619959415) bank32785 +32786 POINT(34.31690186997736 -117.28901530744241) bank32786 +32787 POINT(33.11269510881333 -118.45936693085712) bank32787 +32788 POINT(34.747688603675435 -117.50068129812934) bank32788 +32789 POINT(33.133062475447126 -118.38772842862383) bank32789 +32790 POINT(33.05278590225315 -117.86160159131754) bank32790 +32791 POINT(33.71375082764258 -117.49137860225626) bank32791 +32792 POINT(33.09534952409906 -117.26626979169718) bank32792 +32793 POINT(34.917647931974656 -118.9700914828154) bank32793 +32794 POINT(33.643960931725296 -118.82938317954957) bank32794 +32795 POINT(33.36580722352099 -117.88700633733788) bank32795 +32796 POINT(33.905999135092166 -119.11023437347649) bank32796 +32797 POINT(34.20002363832178 -118.29863151406839) bank32797 +32798 POINT(33.54111365841757 -117.32201541514978) bank32798 +32799 POINT(33.63418645457423 -117.72815142810194) bank32799 +32800 POINT(33.8327221922266 -118.94007009205288) bank32800 +32801 POINT(34.41813328214739 -117.57246897744602) bank32801 +32802 POINT(34.50644563575805 -118.68607016720992) bank32802 +32803 POINT(33.515199665165326 -119.24111043875932) bank32803 +32804 POINT(33.82459316252732 -117.75031876899163) bank32804 +32805 POINT(33.230679133829945 -117.42679762760905) bank32805 +32806 POINT(34.116334160481934 -118.34779001347007) bank32806 +32807 POINT(34.30485402699984 -117.5414140491248) bank32807 +32808 POINT(33.98800598459555 -117.86434166590035) bank32808 +32809 POINT(33.16562424112371 -117.50097848703663) bank32809 +32810 POINT(35.03925169361697 -118.63026428490346) bank32810 +32811 POINT(33.28007507092631 -118.75688108323259) bank32811 +32812 POINT(34.51852394957622 -118.41552774334279) bank32812 +32813 POINT(33.17576860810889 -117.8602186514021) bank32813 +32814 POINT(34.76516530433668 -119.01989494495093) bank32814 +32815 POINT(34.426021544794 -117.59387606435907) bank32815 +32816 POINT(34.765787653445656 -118.08248703971984) bank32816 +32817 POINT(33.26999891989629 -118.80003067493571) bank32817 +32818 POINT(33.461916583016034 -117.83468146738517) bank32818 +32819 POINT(34.340242090006356 -118.68547540470995) bank32819 +32820 POINT(34.063708060344055 -117.7896830705845) bank32820 +32821 POINT(33.22671956241624 -117.29875696771832) bank32821 +32822 POINT(34.29202627815376 -118.79684285999078) bank32822 +32823 POINT(33.65833178407773 -118.16968697738174) bank32823 +32824 POINT(34.63039091711156 -118.61203696650041) bank32824 +32825 POINT(33.39660074827224 -119.05475151715567) bank32825 +32826 POINT(34.409037571068154 -119.15849583874063) bank32826 +32827 POINT(34.964590978348184 -119.12110669584455) bank32827 +32828 POINT(33.18386291154103 -118.99005644546656) bank32828 +32829 POINT(34.86665490609503 -119.06351554762608) bank32829 +32830 POINT(33.97324183419935 -117.57758096895128) bank32830 +32831 POINT(33.693465171163744 -118.75704969730579) bank32831 +32832 POINT(33.40512886550305 -119.19838611254754) bank32832 +32833 POINT(34.34528527796835 -119.17494534448004) bank32833 +32834 POINT(34.45024561547789 -118.3411362889075) bank32834 +32835 POINT(33.27914501329817 -117.3582769091655) bank32835 +32836 POINT(33.38303548811861 -117.45856845898491) bank32836 +32837 POINT(34.67826150851095 -118.28210044402468) bank32837 +32838 POINT(33.31711335715853 -117.89241875081805) bank32838 +32839 POINT(34.61113676225923 -118.00576486575822) bank32839 +32840 POINT(34.06168667350019 -118.14927241554099) bank32840 +32841 POINT(35.02051108207365 -119.05675516824701) bank32841 +32842 POINT(33.66467440274127 -117.97176800910117) bank32842 +32843 POINT(34.88762877241998 -117.46320038314495) bank32843 +32844 POINT(33.1187890701835 -118.97433344925064) bank32844 +32845 POINT(33.801764853107876 -118.58495934872816) bank32845 +32846 POINT(33.61758693583971 -118.87008691725383) bank32846 +32847 POINT(33.30579170776174 -118.20926863451542) bank32847 +32848 POINT(33.189159955072995 -117.84064362317561) bank32848 +32849 POINT(34.104908814742885 -118.98668516982872) bank32849 +32850 POINT(33.28340609106483 -118.26870449233239) bank32850 +32851 POINT(34.2952473627681 -118.20752484572088) bank32851 +32852 POINT(33.743565533121775 -118.99188830000709) bank32852 +32853 POINT(33.84269079682049 -118.46694376486691) bank32853 +32854 POINT(34.886685305344685 -118.14948447260694) bank32854 +32855 POINT(34.34944626313925 -118.86296022563707) bank32855 +32856 POINT(33.88324122897869 -118.96251575002049) bank32856 +32857 POINT(34.37876515250493 -119.0513244002715) bank32857 +32858 POINT(34.27575711978254 -117.85563063942604) bank32858 +32859 POINT(34.29206084491504 -117.42703168009176) bank32859 +32860 POINT(34.7081991812618 -117.6555764339656) bank32860 +32861 POINT(34.19695093398816 -118.43258789786387) bank32861 +32862 POINT(33.09522061468734 -118.12809852370142) bank32862 +32863 POINT(34.664656753139326 -117.88975179300171) bank32863 +32864 POINT(33.616890531869494 -118.32611056344997) bank32864 +32865 POINT(33.243012752543386 -118.56901868965296) bank32865 +32866 POINT(33.12760444410009 -118.25934864922189) bank32866 +32867 POINT(33.72032755416804 -118.77651878838633) bank32867 +32868 POINT(34.41737417851431 -118.1642488740015) bank32868 +32869 POINT(34.28611992739402 -117.52196426557217) bank32869 +32870 POINT(33.72007285133115 -117.58814595584553) bank32870 +32871 POINT(33.371083118746746 -117.59628486287635) bank32871 +32872 POINT(33.43069641585338 -119.04862426204667) bank32872 +32873 POINT(33.85878614349525 -117.71240957041455) bank32873 +32874 POINT(33.26423341963798 -118.61680034325433) bank32874 +32875 POINT(34.163259654400306 -118.62699102003255) bank32875 +32876 POINT(34.73904820535447 -117.90556345435101) bank32876 +32877 POINT(35.00824061217451 -118.98129493908917) bank32877 +32878 POINT(34.26088763105199 -117.36409122212834) bank32878 +32879 POINT(34.86409394687225 -117.34635381374488) bank32879 +32880 POINT(34.54270245818631 -117.68422645732967) bank32880 +32881 POINT(33.42868683561874 -117.41026879854051) bank32881 +32882 POINT(33.65800825344632 -119.01081229667399) bank32882 +32883 POINT(33.93309073274367 -119.00293492991376) bank32883 +32884 POINT(34.85254408038639 -119.04952352782517) bank32884 +32885 POINT(34.48596402887145 -117.91223414282034) bank32885 +32886 POINT(34.38631533353359 -117.78693197339695) bank32886 +32887 POINT(33.7630535102312 -118.03327702163476) bank32887 +32888 POINT(33.115863044260976 -117.27989253298395) bank32888 +32889 POINT(34.468279569413134 -119.14082349164794) bank32889 +32890 POINT(33.12225673453774 -117.6923918088677) bank32890 +32891 POINT(33.971382948568746 -117.64846250479013) bank32891 +32892 POINT(34.48521760939147 -119.12523314679211) bank32892 +32893 POINT(34.59254498638778 -118.13876989443668) bank32893 +32894 POINT(33.78043462936612 -118.2368385743996) bank32894 +32895 POINT(33.70933021495095 -117.4036773800171) bank32895 +32896 POINT(33.63809975132948 -118.96524509297501) bank32896 +32897 POINT(34.987952949461906 -118.11312045259953) bank32897 +32898 POINT(33.56603736201799 -118.74162913741101) bank32898 +32899 POINT(33.37401813313581 -119.22013143681684) bank32899 +32900 POINT(33.92292220462082 -117.43078548803567) bank32900 +32901 POINT(33.60209386533672 -118.52634309999713) bank32901 +32902 POINT(34.149655682307234 -117.85993884487716) bank32902 +32903 POINT(35.01217039540552 -117.84870480319557) bank32903 +32904 POINT(33.40855912791697 -118.96298033850469) bank32904 +32905 POINT(34.84072478661061 -117.2953403466923) bank32905 +32906 POINT(33.910141782507395 -118.82219828649208) bank32906 +32907 POINT(33.86471876936207 -118.84430935630046) bank32907 +32908 POINT(34.48718509214056 -118.92902991029439) bank32908 +32909 POINT(33.361164165478904 -118.97842522362279) bank32909 +32910 POINT(33.29217762113824 -117.90168043332147) bank32910 +32911 POINT(35.037847605960934 -118.10014815589177) bank32911 +32912 POINT(34.82685705067635 -118.19958496267756) bank32912 +32913 POINT(34.45220471014566 -117.57975233385534) bank32913 +32914 POINT(33.29390733318436 -118.12767578072798) bank32914 +32915 POINT(33.28302543241963 -117.68374381148206) bank32915 +32916 POINT(33.15081711670996 -118.53887434818691) bank32916 +32917 POINT(34.07026006665556 -117.70665465894929) bank32917 +32918 POINT(33.98268903104652 -117.71810694835044) bank32918 +32919 POINT(34.2822675323475 -117.41251880154553) bank32919 +32920 POINT(33.247104945475805 -119.21701838726098) bank32920 +32921 POINT(33.9002752702814 -118.99817584366927) bank32921 +32922 POINT(34.93461668977948 -118.49584154434184) bank32922 +32923 POINT(33.47656558890273 -118.54059539804166) bank32923 +32924 POINT(34.53650861949005 -118.43490606044074) bank32924 +32925 POINT(34.304239426167584 -117.84603153898342) bank32925 +32926 POINT(34.87159039016844 -118.81204985159697) bank32926 +32927 POINT(33.57753175231763 -119.11157326132442) bank32927 +32928 POINT(34.991262385684124 -118.81672602385048) bank32928 +32929 POINT(33.67327784762699 -117.39501978926276) bank32929 +32930 POINT(34.335496845009985 -119.17656331609042) bank32930 +32931 POINT(34.92023026422439 -117.99046254961526) bank32931 +32932 POINT(34.41517295367416 -117.57031195286247) bank32932 +32933 POINT(33.738419431607355 -118.59408013172188) bank32933 +32934 POINT(33.491011926056736 -118.56654446291867) bank32934 +32935 POINT(34.50097419764232 -118.81218679386483) bank32935 +32936 POINT(34.46691872705104 -118.815217563179) bank32936 +32937 POINT(33.48172447228916 -117.45401074626591) bank32937 +32938 POINT(34.36737458907321 -118.75190986778235) bank32938 +32939 POINT(33.54275497309333 -117.76568181116308) bank32939 +32940 POINT(33.746725424790824 -117.81336113890929) bank32940 +32941 POINT(33.35017652574404 -118.33152221183303) bank32941 +32942 POINT(34.385318784865404 -118.28785306863297) bank32942 +32943 POINT(33.41596797151055 -118.85865304794274) bank32943 +32944 POINT(33.28935143490405 -118.0125018909276) bank32944 +32945 POINT(34.80865738878916 -117.28964036739497) bank32945 +32946 POINT(34.294625416293556 -118.55738156738607) bank32946 +32947 POINT(34.58997129531953 -118.70240383035748) bank32947 +32948 POINT(33.70773697331205 -119.23210310238171) bank32948 +32949 POINT(33.05382179664903 -118.70700462569917) bank32949 +32950 POINT(33.42217208022091 -117.87993504487915) bank32950 +32951 POINT(33.852733273265265 -119.0214793889415) bank32951 +32952 POINT(34.366192487801975 -117.53586941410158) bank32952 +32953 POINT(34.501457167732674 -117.74987817472073) bank32953 +32954 POINT(33.94925102320848 -117.82373824016554) bank32954 +32955 POINT(34.720880872232314 -118.74529601209322) bank32955 +32956 POINT(34.60601707707571 -118.4411715726865) bank32956 +32957 POINT(33.94188969012468 -117.41085068919568) bank32957 +32958 POINT(34.80627730504621 -117.8277099970547) bank32958 +32959 POINT(33.73872730161026 -118.02825502102814) bank32959 +32960 POINT(33.37671058652767 -118.70448848005893) bank32960 +32961 POINT(33.14325871321187 -118.66290742929243) bank32961 +32962 POINT(34.52326251383712 -118.06249700035488) bank32962 +32963 POINT(33.14070266738249 -118.04811558264248) bank32963 +32964 POINT(33.814675490495326 -118.91169338301783) bank32964 +32965 POINT(33.052682304197084 -117.56563827684064) bank32965 +32966 POINT(33.83163356295326 -118.46504017683806) bank32966 +32967 POINT(34.647451333799445 -117.50155232950785) bank32967 +32968 POINT(33.62737994968496 -118.29160796922856) bank32968 +32969 POINT(34.1953727176527 -118.88781051871246) bank32969 +32970 POINT(33.79489185453219 -118.65630841905511) bank32970 +32971 POINT(33.38811111911996 -117.60865447204573) bank32971 +32972 POINT(34.78575141470358 -118.25181423950434) bank32972 +32973 POINT(34.038936027655154 -118.98400265701005) bank32973 +32974 POINT(34.01847091514117 -118.61369267160121) bank32974 +32975 POINT(33.35755438617208 -117.678090085129) bank32975 +32976 POINT(33.602565584031076 -117.45406575367392) bank32976 +32977 POINT(33.72857760433934 -118.81457289911904) bank32977 +32978 POINT(34.69858821235632 -117.76347615478039) bank32978 +32979 POINT(34.45865663670323 -118.80429013158333) bank32979 +32980 POINT(34.768972213591645 -117.86465751263587) bank32980 +32981 POINT(33.64417034205958 -118.0730256486784) bank32981 +32982 POINT(33.82927083683457 -118.49769355106676) bank32982 +32983 POINT(33.12673959989425 -117.90204697541458) bank32983 +32984 POINT(35.050778432584245 -119.09502782982561) bank32984 +32985 POINT(34.657062585235096 -117.7830404752084) bank32985 +32986 POINT(34.73057183583768 -118.28643679137792) bank32986 +32987 POINT(33.933008854709335 -118.3146428821419) bank32987 +32988 POINT(33.420214671196355 -119.17285540912123) bank32988 +32989 POINT(33.746470832106986 -117.66081866256349) bank32989 +32990 POINT(34.959773359134786 -118.35990473087557) bank32990 +32991 POINT(34.851053499987096 -118.9184124992815) bank32991 +32992 POINT(34.78374106523198 -118.54546151299107) bank32992 +32993 POINT(33.40800118098979 -118.46012160821854) bank32993 +32994 POINT(33.2499231794227 -118.16227556027209) bank32994 +32995 POINT(33.94083975932594 -118.58857871339058) bank32995 +32996 POINT(33.35105784340477 -118.13738078634678) bank32996 +32997 POINT(34.766823691388126 -117.85308138626847) bank32997 +32998 POINT(33.48174941816362 -117.5111432649987) bank32998 +32999 POINT(33.66738583101952 -117.29450119102619) bank32999 +33000 POINT(33.79564107017836 -117.76990536308519) bank33000 +33001 POINT(35.007473517937676 -118.95957246984804) bank33001 +33002 POINT(34.674928466348874 -117.48993861119705) bank33002 +33003 POINT(34.752685746849494 -117.73138274058947) bank33003 +33004 POINT(33.378138874440005 -118.1119968679122) bank33004 +33005 POINT(33.21323802644734 -117.38703395325665) bank33005 +33006 POINT(33.607554656765785 -118.08855730503345) bank33006 +33007 POINT(34.26931778681507 -118.41753160586623) bank33007 +33008 POINT(33.39512277586459 -118.57256917222271) bank33008 +33009 POINT(34.864352063698654 -119.23484735232353) bank33009 +33010 POINT(34.03134251866425 -118.26829978878646) bank33010 +33011 POINT(34.79207710850967 -118.22574290463017) bank33011 +33012 POINT(34.71601455608455 -118.33679702748067) bank33012 +33013 POINT(34.755879653786835 -117.80044879914925) bank33013 +33014 POINT(33.068868698569524 -118.1424524705654) bank33014 +33015 POINT(33.66419742580661 -117.72957692053238) bank33015 +33016 POINT(33.61883983237683 -118.89837969971133) bank33016 +33017 POINT(34.390057046027565 -118.82126332622305) bank33017 +33018 POINT(34.94273315621361 -118.34100903628143) bank33018 +33019 POINT(33.268585857896305 -117.93686281938822) bank33019 +33020 POINT(34.55266355452548 -117.79992443281523) bank33020 +33021 POINT(34.88444546725004 -118.99616646153422) bank33021 +33022 POINT(33.2261666932563 -117.59256141356916) bank33022 +33023 POINT(33.752811446975855 -117.77721496939495) bank33023 +33024 POINT(34.68926553087723 -117.56706613758155) bank33024 +33025 POINT(33.43517558979079 -118.73413117644168) bank33025 +33026 POINT(34.857446964732716 -118.010621602224) bank33026 +33027 POINT(34.71730355397282 -119.11881049397935) bank33027 +33028 POINT(34.126202247686074 -118.9002382182954) bank33028 +33029 POINT(34.49376929510872 -118.41720868253172) bank33029 +33030 POINT(34.4625584575761 -117.93062776616253) bank33030 +33031 POINT(34.47892987313288 -118.85223423683378) bank33031 +33032 POINT(33.82609680249712 -118.42738718858114) bank33032 +33033 POINT(33.732143573137705 -118.75859805027868) bank33033 +33034 POINT(33.484025259946996 -118.56016287140501) bank33034 +33035 POINT(34.90122371599385 -118.07453458298639) bank33035 +33036 POINT(33.694459377994 -118.32238850599279) bank33036 +33037 POINT(34.01397211406328 -117.31000433765576) bank33037 +33038 POINT(34.938229518972676 -118.11733390509038) bank33038 +33039 POINT(33.074452375818964 -118.92293768203513) bank33039 +33040 POINT(33.31048193999448 -117.4360336181319) bank33040 +33041 POINT(34.42630037534494 -118.75528810312082) bank33041 +33042 POINT(34.18331742225103 -119.02332892725191) bank33042 +33043 POINT(33.288532304174446 -118.00844070931765) bank33043 +33044 POINT(34.92347664952668 -118.04433297911547) bank33044 +33045 POINT(33.44453026350767 -117.96011068918439) bank33045 +33046 POINT(34.341883089294626 -118.45602351135277) bank33046 +33047 POINT(33.06592223026144 -118.61901249593777) bank33047 +33048 POINT(34.92789984867017 -117.86171525313385) bank33048 +33049 POINT(33.28594806414901 -117.75279745491846) bank33049 +33050 POINT(34.45289281932543 -119.12383278666327) bank33050 +33051 POINT(33.21098309984687 -118.9676011925669) bank33051 +33052 POINT(34.44548528352591 -118.85365005050195) bank33052 +33053 POINT(34.04887018213026 -118.07455292615546) bank33053 +33054 POINT(34.86898855991147 -118.6947614943758) bank33054 +33055 POINT(33.37341069030285 -118.99371448291745) bank33055 +33056 POINT(34.64836791263516 -117.61993923316874) bank33056 +33057 POINT(33.777306149189556 -118.70522739094349) bank33057 +33058 POINT(34.57454489084647 -117.74090879282399) bank33058 +33059 POINT(33.95257176654105 -118.66271620789232) bank33059 +33060 POINT(34.82639777257873 -118.22416325408693) bank33060 +33061 POINT(34.307699001813894 -119.06418003691955) bank33061 +33062 POINT(33.87243664478274 -118.70099256244114) bank33062 +33063 POINT(34.43997382576276 -117.28655060033509) bank33063 +33064 POINT(34.27781178241638 -118.22535206022673) bank33064 +33065 POINT(33.46940704165162 -118.40933435075617) bank33065 +33066 POINT(34.69929978916909 -119.23855737978273) bank33066 +33067 POINT(35.00689994315776 -117.90329302441442) bank33067 +33068 POINT(33.74391725068361 -118.49837555044579) bank33068 +33069 POINT(34.584980453449056 -118.92444153570034) bank33069 +33070 POINT(33.3601196589668 -118.84440960488419) bank33070 +33071 POINT(34.39396052655706 -118.55800971664009) bank33071 +33072 POINT(33.1670179724701 -117.29453782516471) bank33072 +33073 POINT(34.910480724102044 -118.3705546576471) bank33073 +33074 POINT(33.12974382310005 -117.47889611259895) bank33074 +33075 POINT(33.632243369051274 -117.45536837690099) bank33075 +33076 POINT(35.00470895921523 -118.58513070853304) bank33076 +33077 POINT(33.42253609959921 -117.71413152887655) bank33077 +33078 POINT(34.35459790019552 -118.07784359051195) bank33078 +33079 POINT(34.977601711169996 -117.7203442146541) bank33079 +33080 POINT(33.820738104631346 -118.82999594090046) bank33080 +33081 POINT(33.386449787908596 -117.3266103341843) bank33081 +33082 POINT(33.63083503879994 -117.35564223374304) bank33082 +33083 POINT(33.18324930785359 -119.12421809878978) bank33083 +33084 POINT(33.38542934386389 -117.65715991139949) bank33084 +33085 POINT(34.844954553224255 -117.53668462250799) bank33085 +33086 POINT(33.2683167169197 -117.94145562399251) bank33086 +33087 POINT(34.20197444970333 -118.58759074034147) bank33087 +33088 POINT(34.76224388584738 -118.45623451514942) bank33088 +33089 POINT(34.52477211987507 -118.93686310134822) bank33089 +33090 POINT(33.52071095797118 -118.79654068731605) bank33090 +33091 POINT(33.42307609605216 -117.6273492998825) bank33091 +33092 POINT(33.24009018260021 -119.07501270370484) bank33092 +33093 POINT(33.40769539220758 -117.42374401749265) bank33093 +33094 POINT(33.58459622303901 -119.22963522829338) bank33094 +33095 POINT(33.98279881732984 -118.68010902613076) bank33095 +33096 POINT(34.70095179024046 -118.4752079242856) bank33096 +33097 POINT(33.804187123612174 -117.45573595332675) bank33097 +33098 POINT(34.34020993300832 -117.70760842553614) bank33098 +33099 POINT(34.98044306612304 -117.99529124101679) bank33099 +33100 POINT(34.3134419444298 -117.72047450347968) bank33100 +33101 POINT(34.902274734443445 -119.20147549781954) bank33101 +33102 POINT(33.89497616050401 -117.85987857746592) bank33102 +33103 POINT(34.181939077437576 -118.45852068677122) bank33103 +33104 POINT(33.382518788661 -118.04210423678128) bank33104 +33105 POINT(34.66894768193706 -119.19982667436456) bank33105 +33106 POINT(34.47294762595784 -117.51714418224692) bank33106 +33107 POINT(33.5105241312296 -117.74173062403048) bank33107 +33108 POINT(34.81681371852146 -117.76019651741412) bank33108 +33109 POINT(34.52572336993639 -118.54932244121646) bank33109 +33110 POINT(34.821730661353975 -119.20930525537783) bank33110 +33111 POINT(34.11598183228693 -117.84809110594723) bank33111 +33112 POINT(33.12617035163019 -117.4609575704918) bank33112 +33113 POINT(34.8723504010374 -117.7509704963185) bank33113 +33114 POINT(34.5289708566501 -117.64435602609696) bank33114 +33115 POINT(34.90868346720634 -117.44911507513358) bank33115 +33116 POINT(34.39566096410659 -119.07394961400031) bank33116 +33117 POINT(33.63385528043832 -119.05340076011396) bank33117 +33118 POINT(34.73106893827354 -117.78015681018208) bank33118 +33119 POINT(33.1745963344731 -117.2928336229462) bank33119 +33120 POINT(33.79008357207996 -118.12338489477439) bank33120 +33121 POINT(34.86387959108022 -119.1268835236333) bank33121 +33122 POINT(33.86917387528405 -118.22442574540543) bank33122 +33123 POINT(33.37085695939203 -117.40936335644314) bank33123 +33124 POINT(34.09567983164627 -118.35259758163872) bank33124 +33125 POINT(33.41551143428649 -119.19494751272076) bank33125 +33126 POINT(33.71005444629277 -118.8970026298936) bank33126 +33127 POINT(34.30750969386908 -118.09601892202166) bank33127 +33128 POINT(34.32487662572171 -118.12467707482685) bank33128 +33129 POINT(34.40999854781947 -117.8450822798741) bank33129 +33130 POINT(33.66203411485789 -117.7184843402034) bank33130 +33131 POINT(34.96196447231326 -118.09832459253091) bank33131 +33132 POINT(33.95020727353055 -119.21607561351534) bank33132 +33133 POINT(34.7674357588773 -117.58744721557994) bank33133 +33134 POINT(34.919372993893106 -119.17031736269259) bank33134 +33135 POINT(34.4542392980947 -118.49682698111344) bank33135 +33136 POINT(34.51539919349291 -117.67645341927192) bank33136 +33137 POINT(34.42768327428171 -118.51876752339065) bank33137 +33138 POINT(34.682779945197865 -118.2119257781129) bank33138 +33139 POINT(34.503726760432194 -117.6054842537916) bank33139 +33140 POINT(34.81070819423411 -119.1300070024666) bank33140 +33141 POINT(33.36028160421862 -117.73338436662894) bank33141 +33142 POINT(34.15846653278285 -118.06792656125498) bank33142 +33143 POINT(33.430582439480155 -118.04324275314644) bank33143 +33144 POINT(34.429304091477285 -117.82079676824065) bank33144 +33145 POINT(33.75673347740449 -118.90283170730542) bank33145 +33146 POINT(34.864473761450036 -117.77146369730924) bank33146 +33147 POINT(33.95019121318847 -117.70777968090341) bank33147 +33148 POINT(35.002199593330545 -117.95994053769853) bank33148 +33149 POINT(33.7071874245458 -118.85093882712302) bank33149 +33150 POINT(33.87858321192738 -118.41591626256783) bank33150 +33151 POINT(34.83665882229398 -118.02799232600645) bank33151 +33152 POINT(33.53511636302382 -117.71227741964681) bank33152 +33153 POINT(33.219909782080016 -117.86843509370686) bank33153 +33154 POINT(33.111290156830286 -117.48454314613558) bank33154 +33155 POINT(33.861247593548065 -117.662115437103) bank33155 +33156 POINT(34.995005213244674 -117.30101967731028) bank33156 +33157 POINT(34.38484449966884 -118.64187897768862) bank33157 +33158 POINT(33.24314396540615 -118.36470900401899) bank33158 +33159 POINT(33.40341451319441 -117.91534976057557) bank33159 +33160 POINT(34.9410211067227 -117.92465835683399) bank33160 +33161 POINT(33.35470357174115 -119.04021446379585) bank33161 +33162 POINT(33.5037448016994 -117.87427453600431) bank33162 +33163 POINT(34.17856384159257 -117.28907959288411) bank33163 +33164 POINT(33.74564480691449 -118.47288999035378) bank33164 +33165 POINT(34.6484266940476 -119.24136252722876) bank33165 +33166 POINT(33.20560852071994 -117.35348418963872) bank33166 +33167 POINT(33.58327333607944 -118.15914536545046) bank33167 +33168 POINT(33.897406678193555 -117.28246778233714) bank33168 +33169 POINT(34.41313156451698 -118.88689767543725) bank33169 +33170 POINT(34.440869720974845 -119.17863073863754) bank33170 +33171 POINT(33.344018585772886 -118.34731585130004) bank33171 +33172 POINT(35.00241179916806 -117.51309363753602) bank33172 +33173 POINT(34.43792458301141 -117.96367569812708) bank33173 +33174 POINT(34.892675309063826 -118.9026010767974) bank33174 +33175 POINT(33.48292600349207 -118.47897875516291) bank33175 +33176 POINT(33.12205593071832 -117.75744323174773) bank33176 +33177 POINT(33.57247222443875 -118.4560254951269) bank33177 +33178 POINT(33.92406556424523 -117.24372310285153) bank33178 +33179 POINT(34.0639335191732 -117.72023235442569) bank33179 +33180 POINT(34.168203567415475 -118.4609927893082) bank33180 +33181 POINT(34.887841987863666 -117.7552136071615) bank33181 +33182 POINT(35.002462057486845 -117.49351216321706) bank33182 +33183 POINT(33.15336211685516 -118.58068929208203) bank33183 +33184 POINT(34.729869511150355 -119.00208027051765) bank33184 +33185 POINT(34.608147933936856 -118.27602731905966) bank33185 +33186 POINT(34.03745385200112 -117.84215334509202) bank33186 +33187 POINT(33.111554114017046 -119.06211263553558) bank33187 +33188 POINT(34.53215213691862 -119.0053580199794) bank33188 +33189 POINT(33.38357929845892 -117.67517193957966) bank33189 +33190 POINT(34.46403050749757 -118.8511003541682) bank33190 +33191 POINT(34.78204985419079 -118.36061514490119) bank33191 +33192 POINT(34.50766367410566 -117.34380788811151) bank33192 +33193 POINT(34.23330480234379 -117.84656465969732) bank33193 +33194 POINT(34.861747074330665 -118.75255290784969) bank33194 +33195 POINT(33.70063935504701 -118.43227421313365) bank33195 +33196 POINT(33.772685913216726 -117.25917586579416) bank33196 +33197 POINT(33.384846015275386 -119.21817891877416) bank33197 +33198 POINT(33.76748704474466 -119.15564538699002) bank33198 +33199 POINT(34.96529092930737 -118.21842349323258) bank33199 +33200 POINT(34.42554414976872 -117.46561269334939) bank33200 +33201 POINT(34.370644145582844 -118.09055978266882) bank33201 +33202 POINT(34.04195224018253 -117.70680157349848) bank33202 +33203 POINT(34.90447012934787 -117.40477749175396) bank33203 +33204 POINT(34.14397510716012 -118.90228363410766) bank33204 +33205 POINT(33.86603136179169 -119.23803955096145) bank33205 +33206 POINT(34.053932985269924 -118.94297337313363) bank33206 +33207 POINT(34.293554507573205 -117.73389613859462) bank33207 +33208 POINT(34.76028409032111 -118.56856111219003) bank33208 +33209 POINT(34.36249164581091 -117.75063411333315) bank33209 +33210 POINT(34.44375436594058 -118.89195409143397) bank33210 +33211 POINT(33.355580894352 -118.63957166282445) bank33211 +33212 POINT(34.561080868238854 -117.72522130371888) bank33212 +33213 POINT(34.86146029523035 -119.03359199705281) bank33213 +33214 POINT(33.63870767962696 -118.34786834653617) bank33214 +33215 POINT(34.70242684692681 -118.56678767898835) bank33215 +33216 POINT(34.396063371380485 -118.36174505406095) bank33216 +33217 POINT(33.15781234012712 -118.83053213509946) bank33217 +33218 POINT(34.500124522698336 -118.90180867114886) bank33218 +33219 POINT(33.566335085182274 -117.57665777922931) bank33219 +33220 POINT(35.01723562936296 -117.62175419469598) bank33220 +33221 POINT(33.57367799691926 -117.29404205060746) bank33221 +33222 POINT(34.00432806493311 -118.23117145715916) bank33222 +33223 POINT(33.68167003936172 -117.79562456427377) bank33223 +33224 POINT(33.23818744724924 -119.18871523899334) bank33224 +33225 POINT(34.50899770188665 -117.31398225234382) bank33225 +33226 POINT(33.703253252021824 -118.99987303687755) bank33226 +33227 POINT(33.540286631594554 -117.68360256794406) bank33227 +33228 POINT(34.04510802737395 -119.10445538218346) bank33228 +33229 POINT(34.937784215871744 -119.10324316502191) bank33229 +33230 POINT(33.171578913118154 -119.18886727371817) bank33230 +33231 POINT(33.478757071487045 -118.88246405700814) bank33231 +33232 POINT(34.3512013128052 -117.34709574828862) bank33232 +33233 POINT(33.86558046170139 -118.98622489377412) bank33233 +33234 POINT(33.476569310475774 -118.70663445655462) bank33234 +33235 POINT(34.91212144889604 -117.56988621288608) bank33235 +33236 POINT(34.957662750767085 -117.25774208874881) bank33236 +33237 POINT(34.436262244382206 -118.951344548656) bank33237 +33238 POINT(34.693378050237044 -117.9074636982767) bank33238 +33239 POINT(34.29674511927052 -119.08920207244921) bank33239 +33240 POINT(34.3198161982921 -118.34217949944978) bank33240 +33241 POINT(33.790585166521986 -118.0683667743814) bank33241 +33242 POINT(33.339649173784515 -118.9894497353458) bank33242 +33243 POINT(34.16578608878437 -117.53775127939616) bank33243 +33244 POINT(33.822480118268494 -118.19908336581123) bank33244 +33245 POINT(33.80704206587144 -118.68868442603441) bank33245 +33246 POINT(33.34647094527136 -119.14915806498021) bank33246 +33247 POINT(33.101518160055846 -118.9222013914324) bank33247 +33248 POINT(34.28249543602478 -118.06748631481221) bank33248 +33249 POINT(33.60084288350014 -117.86259797398084) bank33249 +33250 POINT(34.27240658097965 -117.73511232134221) bank33250 +33251 POINT(35.002336856210675 -119.19533205576482) bank33251 +33252 POINT(33.12788726437083 -118.35274361521348) bank33252 +33253 POINT(34.35702666936658 -118.65597393512382) bank33253 +33254 POINT(34.53332650225249 -118.72266227082036) bank33254 +33255 POINT(34.254008482094854 -118.17870475430898) bank33255 +33256 POINT(34.5360800749015 -118.07706316940452) bank33256 +33257 POINT(33.74136327552183 -118.97783955135078) bank33257 +33258 POINT(33.79686051311198 -119.0132207910459) bank33258 +33259 POINT(33.398332153068004 -117.74110932111758) bank33259 +33260 POINT(34.922192686961424 -118.7746453098343) bank33260 +33261 POINT(34.279856115667755 -119.17808733204893) bank33261 +33262 POINT(34.38283590197808 -119.07653202278327) bank33262 +33263 POINT(34.254872925849874 -118.42843755112767) bank33263 +33264 POINT(33.13277498446762 -118.46096859453462) bank33264 +33265 POINT(33.82727348933498 -118.64782551321602) bank33265 +33266 POINT(34.86596533137061 -118.70186801685291) bank33266 +33267 POINT(34.72659599696105 -117.38523347360248) bank33267 +33268 POINT(33.652034931937166 -118.46764387999075) bank33268 +33269 POINT(33.26539850641655 -117.85260556813155) bank33269 +33270 POINT(33.979953344984864 -117.73490303329197) bank33270 +33271 POINT(33.4140101749505 -118.67408966272282) bank33271 +33272 POINT(33.30448748798871 -119.21231387377695) bank33272 +33273 POINT(33.68952421645877 -119.20200679039863) bank33273 +33274 POINT(33.635767909267464 -118.22259650483839) bank33274 +33275 POINT(33.10650435583743 -118.99514898046706) bank33275 +33276 POINT(34.648416699291495 -117.68870428326643) bank33276 +33277 POINT(33.57905045521121 -118.0174917787047) bank33277 +33278 POINT(33.324527654757304 -118.89123531090979) bank33278 +33279 POINT(33.49843363982141 -118.93978520072444) bank33279 +33280 POINT(33.07008172867525 -118.26877915025722) bank33280 +33281 POINT(34.29352797085274 -117.93612950256605) bank33281 +33282 POINT(34.619958885798845 -118.03541677235731) bank33282 +33283 POINT(33.648024200563796 -117.98698123704312) bank33283 +33284 POINT(34.53508310595194 -118.42665264706658) bank33284 +33285 POINT(34.77670045854075 -117.81756920894118) bank33285 +33286 POINT(33.921733821699284 -118.4198741570939) bank33286 +33287 POINT(33.39019220161003 -118.71804023310058) bank33287 +33288 POINT(33.63475916790864 -118.83505035393367) bank33288 +33289 POINT(33.15682385253256 -118.77011475531357) bank33289 +33290 POINT(34.75171825556216 -118.78771007800752) bank33290 +33291 POINT(33.410321373284354 -117.6666222942506) bank33291 +33292 POINT(33.22388802955889 -118.22682874953686) bank33292 +33293 POINT(34.49347186252052 -117.4400148902917) bank33293 +33294 POINT(34.08156165664693 -117.263232240332) bank33294 +33295 POINT(34.75613000861214 -117.41016059271772) bank33295 +33296 POINT(33.218655887595304 -117.92056456333047) bank33296 +33297 POINT(33.74607456441191 -118.23597462746626) bank33297 +33298 POINT(34.939223468051445 -118.81621946619838) bank33298 +33299 POINT(33.75677290870911 -117.68129921409387) bank33299 +33300 POINT(33.715700392583585 -118.18265323281236) bank33300 +33301 POINT(33.770337938555436 -118.95154097830387) bank33301 +33302 POINT(34.27447042850346 -119.23451826146449) bank33302 +33303 POINT(34.85125218893201 -117.83766833516776) bank33303 +33304 POINT(34.62918741681694 -117.57632348319656) bank33304 +33305 POINT(33.988683258702935 -118.70793690531094) bank33305 +33306 POINT(34.72838498756476 -118.45296096403922) bank33306 +33307 POINT(34.15398440714302 -117.7357083442706) bank33307 +33308 POINT(34.18911536256901 -118.80799557417416) bank33308 +33309 POINT(34.79525130060648 -118.87451445602001) bank33309 +33310 POINT(34.93070827604034 -118.21139334710281) bank33310 +33311 POINT(34.02024056491696 -118.41822905201201) bank33311 +33312 POINT(33.38284594105475 -118.78557526524625) bank33312 +33313 POINT(33.168239798752175 -117.58579062790868) bank33313 +33314 POINT(33.569904674203 -118.2420085192103) bank33314 +33315 POINT(34.409964950378985 -118.34077655525152) bank33315 +33316 POINT(34.734479813509836 -117.8399136770179) bank33316 +33317 POINT(33.3714587124283 -118.02422829001553) bank33317 +33318 POINT(34.00425610964069 -118.40924161820139) bank33318 +33319 POINT(34.93721403863589 -117.5526856789146) bank33319 +33320 POINT(33.33705120521686 -117.50461442481479) bank33320 +33321 POINT(34.92851366489018 -118.26031214362992) bank33321 +33322 POINT(34.98852976798554 -118.65285346074772) bank33322 +33323 POINT(34.66829233843545 -118.72359036480096) bank33323 +33324 POINT(34.79788584182272 -117.89994628676457) bank33324 +33325 POINT(34.251782173931204 -118.4670759445648) bank33325 +33326 POINT(33.99987326528803 -118.49927660714332) bank33326 +33327 POINT(34.88547608940578 -118.34997432378542) bank33327 +33328 POINT(34.251585189163606 -118.68872487438776) bank33328 +33329 POINT(34.86256230548927 -118.77419573717518) bank33329 +33330 POINT(33.28000991515856 -117.74714953499817) bank33330 +33331 POINT(33.51923905719807 -117.7799517559668) bank33331 +33332 POINT(33.742514909123635 -118.62330142813714) bank33332 +33333 POINT(34.58160267222128 -117.47949760887828) bank33333 +33334 POINT(33.6279521883767 -117.76373150414459) bank33334 +33335 POINT(34.034404919882235 -117.67243435189336) bank33335 +33336 POINT(34.63931730353877 -118.46534262017448) bank33336 +33337 POINT(33.43239593114683 -118.95898221786925) bank33337 +33338 POINT(34.91335922548216 -118.89341537965474) bank33338 +33339 POINT(34.53637305405255 -117.35216085789926) bank33339 +33340 POINT(34.90788059542942 -118.77263811196593) bank33340 +33341 POINT(34.35212124389322 -118.02104149540301) bank33341 +33342 POINT(34.69982132455692 -117.82346966367164) bank33342 +33343 POINT(33.63774282900149 -118.41879844449556) bank33343 +33344 POINT(34.8575131880465 -117.48308653905482) bank33344 +33345 POINT(34.38461930499574 -117.38803433495941) bank33345 +33346 POINT(34.508834056134795 -118.39166733522289) bank33346 +33347 POINT(34.59492709069498 -117.68226256011499) bank33347 +33348 POINT(34.14865892429525 -118.91047489612697) bank33348 +33349 POINT(33.166340686933175 -119.0229037698312) bank33349 +33350 POINT(29.363488974106716 -94.9004867547148) bank33350 +33351 POINT(29.147762832841494 -95.34022628073106) bank33351 +33352 POINT(30.219239221711927 -95.21020338144727) bank33352 +33353 POINT(28.909533090284818 -94.4398791103626) bank33353 +33354 POINT(29.59300713379736 -94.89904869928075) bank33354 +33355 POINT(28.790409591659873 -94.92899238076905) bank33355 +33356 POINT(29.246328276263817 -96.24146956307249) bank33356 +33357 POINT(30.492140221915463 -95.60794892260014) bank33357 +33358 POINT(30.361741684927857 -95.29748156926438) bank33358 +33359 POINT(30.37339409125099 -96.06537597191388) bank33359 +33360 POINT(29.905676434905093 -95.57409010361899) bank33360 +33361 POINT(29.24658313898338 -95.79708939686287) bank33361 +33362 POINT(30.207544411759574 -95.43639901443959) bank33362 +33363 POINT(29.69256975973194 -96.05180371931749) bank33363 +33364 POINT(28.89032872335288 -95.30602696937021) bank33364 +33365 POINT(30.653615547828903 -95.32130550805729) bank33365 +33366 POINT(30.049775484425595 -96.06730344592202) bank33366 +33367 POINT(29.912036981941533 -94.65399251584539) bank33367 +33368 POINT(29.51407922468008 -94.4584817748043) bank33368 +33369 POINT(29.742587421117904 -95.94423406661141) bank33369 +33370 POINT(29.85059729600114 -95.22336362214284) bank33370 +33371 POINT(29.332239870595913 -96.11796492905707) bank33371 +33372 POINT(29.306871006692383 -94.65216580799446) bank33372 +33373 POINT(30.722476642080732 -96.17249648922896) bank33373 +33374 POINT(30.68197785156746 -95.14033901551065) bank33374 +33375 POINT(30.648452855410582 -95.31870912588364) bank33375 +33376 POINT(30.094274835069076 -96.05420208522067) bank33376 +33377 POINT(29.706043302814614 -94.51659018975215) bank33377 +33378 POINT(30.39578703142748 -95.85700001313813) bank33378 +33379 POINT(29.637692288859476 -94.39862593790671) bank33379 +33380 POINT(29.220434452023795 -95.49665435224222) bank33380 +33381 POINT(29.48775032497083 -96.03035088268604) bank33381 +33382 POINT(29.81616674643198 -96.20547127836353) bank33382 +33383 POINT(30.221225195481225 -94.71628217757711) bank33383 +33384 POINT(29.047323306371556 -94.94646186266868) bank33384 +33385 POINT(29.334396579020314 -95.63547062330004) bank33385 +33386 POINT(29.47253978761841 -95.67702853810155) bank33386 +33387 POINT(30.456591264981085 -95.83879534522534) bank33387 +33388 POINT(28.789407902650996 -96.06290022164133) bank33388 +33389 POINT(30.64419278984328 -96.27789535179333) bank33389 +33390 POINT(30.250524924816766 -96.34945582239935) bank33390 +33391 POINT(30.195052970687556 -95.25782762391101) bank33391 +33392 POINT(29.247987903893588 -95.55303958678306) bank33392 +33393 POINT(30.01986049771306 -95.38769717278282) bank33393 +33394 POINT(28.970149007381927 -96.18358514769453) bank33394 +33395 POINT(28.99009904955329 -96.35566317174106) bank33395 +33396 POINT(29.01383397287482 -95.9904570395842) bank33396 +33397 POINT(29.203144862503017 -95.29749140766113) bank33397 +33398 POINT(29.05490944244311 -95.95949992526421) bank33398 +33399 POINT(30.011740440517215 -94.87852113689162) bank33399 +33400 POINT(29.52771422131312 -95.86648797993395) bank33400 +33401 POINT(29.71567252787992 -94.8328599569593) bank33401 +33402 POINT(29.686700880176428 -95.66201669645098) bank33402 +33403 POINT(29.247639321723497 -95.90768444652863) bank33403 +33404 POINT(30.677854846974824 -95.6772312504433) bank33404 +33405 POINT(29.56002263232036 -95.95617502839868) bank33405 +33406 POINT(30.614837567887868 -95.60351830949253) bank33406 +33407 POINT(29.316656627600054 -95.36452629818905) bank33407 +33408 POINT(30.756647951185993 -95.61922454241711) bank33408 +33409 POINT(30.724272628448656 -95.8956988305514) bank33409 +33410 POINT(30.35099130806407 -95.81258405395684) bank33410 +33411 POINT(30.39190794774354 -94.97360873763482) bank33411 +33412 POINT(29.337896162839563 -94.53328826862358) bank33412 +33413 POINT(29.982494938143674 -95.6845642910837) bank33413 +33414 POINT(30.50636873149505 -95.7922035100241) bank33414 +33415 POINT(28.989894628890305 -95.00899195978103) bank33415 +33416 POINT(30.68371044766805 -95.71329477903092) bank33416 +33417 POINT(29.965743008650485 -95.92925294971663) bank33417 +33418 POINT(29.045558093724072 -95.48396979679049) bank33418 +33419 POINT(30.36025907052612 -95.52211458266498) bank33419 +33420 POINT(29.35512651588471 -94.81103735366953) bank33420 +33421 POINT(30.221842389463312 -95.48045137899696) bank33421 +33422 POINT(29.679389888724074 -95.78987680365191) bank33422 +33423 POINT(29.776103380725438 -95.51118034588472) bank33423 +33424 POINT(30.131821134512364 -95.96042204791934) bank33424 +33425 POINT(29.71611829509207 -94.39200057350975) bank33425 +33426 POINT(30.439085634715983 -95.75258502458426) bank33426 +33427 POINT(29.376406057655732 -95.15488058507299) bank33427 +33428 POINT(29.43979320796065 -95.27468141636832) bank33428 +33429 POINT(28.834092696754414 -95.46462994177337) bank33429 +33430 POINT(30.055833873569465 -95.08028302661461) bank33430 +33431 POINT(28.922187720478142 -95.04792484496213) bank33431 +33432 POINT(30.56074386951982 -94.48016195488756) bank33432 +33433 POINT(29.1235887067065 -94.8295516523039) bank33433 +33434 POINT(29.866729930105244 -94.96755024407392) bank33434 +33435 POINT(30.68298291346744 -95.04132842929955) bank33435 +33436 POINT(30.242372115014952 -95.2991545821423) bank33436 +33437 POINT(30.090536235164834 -95.56088164024055) bank33437 +33438 POINT(30.09986909816706 -96.17776052529726) bank33438 +33439 POINT(30.51046869120923 -96.24155600910085) bank33439 +33440 POINT(30.6763536418368 -94.71117417797306) bank33440 +33441 POINT(30.54748951124437 -95.55930479451435) bank33441 +33442 POINT(29.311338602160657 -94.64435993985089) bank33442 +33443 POINT(28.994699955141186 -95.4936381237937) bank33443 +33444 POINT(29.949918464227046 -94.37265525453951) bank33444 +33445 POINT(30.37713248036363 -96.21301264529245) bank33445 +33446 POINT(30.137326856238595 -94.4746995374815) bank33446 +33447 POINT(29.807019379119115 -95.83984967015283) bank33447 +33448 POINT(30.482101496242866 -94.43030470341817) bank33448 +33449 POINT(28.809705939349826 -94.85617156650086) bank33449 +33450 POINT(29.016462162666322 -95.90767602729994) bank33450 +33451 POINT(30.504909148614935 -94.4174049346141) bank33451 +33452 POINT(29.268584523345215 -96.0576578111444) bank33452 +33453 POINT(28.895832492908525 -94.52397651288881) bank33453 +33454 POINT(29.970599970692845 -94.9845016885694) bank33454 +33455 POINT(30.66730438382879 -95.14964046323115) bank33455 +33456 POINT(30.2206253672569 -95.58065870742196) bank33456 +33457 POINT(29.344817611324842 -95.01849304656845) bank33457 +33458 POINT(29.715040371449202 -95.15797152143607) bank33458 +33459 POINT(30.676970251541317 -95.13723110545311) bank33459 +33460 POINT(29.36741603608545 -95.15485154040157) bank33460 +33461 POINT(28.81935655715712 -95.76698249597176) bank33461 +33462 POINT(29.870411801631768 -95.47726928017451) bank33462 +33463 POINT(29.14592464831871 -96.22280238825284) bank33463 +33464 POINT(29.42362808417939 -96.34865995283225) bank33464 +33465 POINT(29.213727147375792 -94.49062966489579) bank33465 +33466 POINT(29.23244875826197 -96.17469586212982) bank33466 +33467 POINT(29.841309237122662 -95.66565292390045) bank33467 +33468 POINT(29.803695457321083 -96.21698192412966) bank33468 +33469 POINT(29.8370031223272 -95.35289352981624) bank33469 +33470 POINT(29.02278541877866 -94.58843989478527) bank33470 +33471 POINT(29.692857121823018 -96.1223531061739) bank33471 +33472 POINT(29.43266494724078 -95.02987189964568) bank33472 +33473 POINT(28.98429961407339 -96.16409529899433) bank33473 +33474 POINT(30.6806775250797 -94.63866978762776) bank33474 +33475 POINT(28.788163858778915 -95.99152360025671) bank33475 +33476 POINT(29.836476880687425 -95.37564979009616) bank33476 +33477 POINT(30.168954536307677 -95.6116442550726) bank33477 +33478 POINT(30.631326855091363 -94.78438311448888) bank33478 +33479 POINT(29.97850048667342 -96.35353107649233) bank33479 +33480 POINT(30.556404604064163 -95.69627084344089) bank33480 +33481 POINT(29.183123527086583 -95.05192873136096) bank33481 +33482 POINT(29.63015760480793 -96.26709934326273) bank33482 +33483 POINT(30.371657605657067 -94.7452428014918) bank33483 +33484 POINT(29.841472024624387 -94.49662485247079) bank33484 +33485 POINT(29.378438086544467 -96.29460729236287) bank33485 +33486 POINT(29.308031976739038 -95.51077182682069) bank33486 +33487 POINT(29.393224964638975 -96.19251769243715) bank33487 +33488 POINT(28.833672441987485 -95.10181206623353) bank33488 +33489 POINT(29.365163999591385 -95.61131278441705) bank33489 +33490 POINT(30.17094230720447 -95.75613409918522) bank33490 +33491 POINT(29.696202773837115 -96.24025467550472) bank33491 +33492 POINT(29.258509611234718 -95.51216418375968) bank33492 +33493 POINT(30.235819657156874 -96.19006182679968) bank33493 +33494 POINT(29.845158032237666 -94.73306389196982) bank33494 +33495 POINT(30.695399405340105 -94.46132557026097) bank33495 +33496 POINT(30.313358184609577 -94.81588938209471) bank33496 +33497 POINT(30.56294414917128 -94.70953158516839) bank33497 +33498 POINT(29.26134461076803 -94.71927217345731) bank33498 +33499 POINT(30.405134778014997 -95.22416083075039) bank33499 +33500 POINT(29.161736878460143 -94.72636048720808) bank33500 +33501 POINT(29.606272926642585 -95.53041510899101) bank33501 +33502 POINT(29.57628157027895 -95.9885514737268) bank33502 +33503 POINT(30.60272474227533 -95.66465339571135) bank33503 +33504 POINT(29.821030475695682 -96.11774673872642) bank33504 +33505 POINT(30.569949147722454 -96.34019745652024) bank33505 +33506 POINT(29.916856942691155 -95.36111589504094) bank33506 +33507 POINT(30.55319503179678 -95.27537083307324) bank33507 +33508 POINT(30.63123125008071 -95.8499992461692) bank33508 +33509 POINT(30.04472838390485 -94.46563885587125) bank33509 +33510 POINT(30.137503362829143 -96.21340312798125) bank33510 +33511 POINT(28.930035096212627 -95.39539340378909) bank33511 +33512 POINT(28.790234974654034 -96.19704338772058) bank33512 +33513 POINT(29.37218672967378 -94.53211780077356) bank33513 +33514 POINT(30.664185082167872 -94.79381929705463) bank33514 +33515 POINT(30.379473893417714 -95.9084404112945) bank33515 +33516 POINT(28.773937936689872 -95.9617046937102) bank33516 +33517 POINT(28.96170318113455 -95.52457053404778) bank33517 +33518 POINT(30.721023185274788 -94.45394265673995) bank33518 +33519 POINT(30.478524600107917 -95.60479189518762) bank33519 +33520 POINT(30.494282836069807 -96.3189867925182) bank33520 +33521 POINT(29.257923012455933 -96.06499879927192) bank33521 +33522 POINT(30.580572824279436 -94.86547566399649) bank33522 +33523 POINT(28.938710150801697 -94.3818472890414) bank33523 +33524 POINT(30.189694189605653 -95.0631089781909) bank33524 +33525 POINT(29.187856441746266 -95.23649185960538) bank33525 +33526 POINT(30.686942000794023 -95.46387720192669) bank33526 +33527 POINT(29.633607330100986 -94.87649575635393) bank33527 +33528 POINT(29.462168619407297 -96.10266552246553) bank33528 +33529 POINT(30.39438168672666 -95.52404201745064) bank33529 +33530 POINT(29.124920936331385 -94.70695839176629) bank33530 +33531 POINT(30.73937487152332 -96.12858084887912) bank33531 +33532 POINT(29.556758508187702 -95.21466997552662) bank33532 +33533 POINT(29.981753845701512 -94.56426466949982) bank33533 +33534 POINT(30.4297267685661 -94.79602882354484) bank33534 +33535 POINT(30.64440700278972 -96.30444851096598) bank33535 +33536 POINT(29.02151092393128 -96.35320067371035) bank33536 +33537 POINT(29.04290477484884 -94.58280468572406) bank33537 +33538 POINT(29.3404046939259 -96.16838261991708) bank33538 +33539 POINT(29.258896827549957 -96.34402051081737) bank33539 +33540 POINT(30.19364232837913 -95.16238494511514) bank33540 +33541 POINT(29.67407399067151 -95.67510827740763) bank33541 +33542 POINT(29.226518839931448 -94.49513651613356) bank33542 +33543 POINT(29.79207163377948 -95.13166693384223) bank33543 +33544 POINT(28.9647200255912 -95.60673042091382) bank33544 +33545 POINT(30.40270609571121 -95.99258154439019) bank33545 +33546 POINT(29.114012064288602 -95.65091131892278) bank33546 +33547 POINT(30.307540938347984 -96.16033290363414) bank33547 +33548 POINT(30.314562826597324 -95.71423856251599) bank33548 +33549 POINT(29.545964138646593 -94.51451239290445) bank33549 +33550 POINT(30.10662472490553 -95.47545572645083) bank33550 +33551 POINT(30.62042327411097 -95.01427201775543) bank33551 +33552 POINT(29.799328898812018 -96.14940423488483) bank33552 +33553 POINT(29.47291245974239 -95.9797518357379) bank33553 +33554 POINT(29.383611243320246 -95.99398123758337) bank33554 +33555 POINT(30.612010663253162 -95.01119893807144) bank33555 +33556 POINT(30.018297116572036 -95.92133997647) bank33556 +33557 POINT(29.40333587652987 -95.64834462697868) bank33557 +33558 POINT(30.132905634374303 -94.49343425211974) bank33558 +33559 POINT(29.36254134016689 -95.71568495319256) bank33559 +33560 POINT(30.206823046972392 -95.66526073776333) bank33560 +33561 POINT(29.51840638726773 -95.18255911515253) bank33561 +33562 POINT(29.812102883760556 -94.76236198622382) bank33562 +33563 POINT(30.080879944137685 -95.13968813096592) bank33563 +33564 POINT(30.570951427472654 -95.23386995089348) bank33564 +33565 POINT(30.335660455919786 -94.6223499599924) bank33565 +33566 POINT(29.81855705532057 -95.64178129879018) bank33566 +33567 POINT(29.654782937736893 -95.44648764623598) bank33567 +33568 POINT(28.769183546345857 -94.64126570573464) bank33568 +33569 POINT(29.52906021606202 -94.95114517365008) bank33569 +33570 POINT(30.010269975698787 -94.78127235348505) bank33570 +33571 POINT(29.49181177150883 -96.11905015299084) bank33571 +33572 POINT(29.856341901260052 -95.23126183020216) bank33572 +33573 POINT(30.16744091629725 -95.71652884648749) bank33573 +33574 POINT(29.008766916898878 -95.92564318177895) bank33574 +33575 POINT(29.02738830253595 -96.01352492886161) bank33575 +33576 POINT(29.771224632994997 -94.7882460784363) bank33576 +33577 POINT(30.53074012275201 -94.95013602004445) bank33577 +33578 POINT(28.81666082809384 -95.70769202212445) bank33578 +33579 POINT(29.408434802786978 -96.31445579518352) bank33579 +33580 POINT(30.524779561879857 -95.36672086777385) bank33580 +33581 POINT(29.994692157314105 -96.18197449470786) bank33581 +33582 POINT(30.57013657814568 -94.86866803765326) bank33582 +33583 POINT(30.74766898964052 -95.59226126287759) bank33583 +33584 POINT(29.43837057075599 -95.20413425007038) bank33584 +33585 POINT(30.456046428089092 -95.37213110400616) bank33585 +33586 POINT(29.912641845464417 -95.39224782278814) bank33586 +33587 POINT(29.726179425880904 -94.72309116949043) bank33587 +33588 POINT(30.612699774159868 -95.36661541353351) bank33588 +33589 POINT(29.203753381721395 -94.86411351867613) bank33589 +33590 POINT(28.90927804676295 -95.30424769350378) bank33590 +33591 POINT(30.20316345460129 -96.02313976241875) bank33591 +33592 POINT(30.096968698656077 -95.23336217474706) bank33592 +33593 POINT(29.95831269122236 -96.25775906830376) bank33593 +33594 POINT(29.15581347915104 -95.00319928726839) bank33594 +33595 POINT(29.126053191009696 -94.82103709015635) bank33595 +33596 POINT(30.49437463900286 -95.8355112647128) bank33596 +33597 POINT(29.629017060552705 -95.7911475670287) bank33597 +33598 POINT(30.217329791181804 -94.90946098687971) bank33598 +33599 POINT(28.888752902821928 -95.73409491044094) bank33599 +33600 POINT(30.099303356693735 -94.53495227706226) bank33600 +33601 POINT(29.688247378142457 -94.37630097821952) bank33601 +33602 POINT(29.61037273242127 -94.90630715333175) bank33602 +33603 POINT(29.650632088058725 -96.22609611810562) bank33603 +33604 POINT(29.858270217319763 -94.95871483879178) bank33604 +33605 POINT(29.477910782033423 -95.8330394507172) bank33605 +33606 POINT(29.974941713608544 -95.7279868887688) bank33606 +33607 POINT(29.040228276145964 -94.58911452064159) bank33607 +33608 POINT(29.606281395801737 -94.68884653055825) bank33608 +33609 POINT(30.75914813841739 -95.85717109324253) bank33609 +33610 POINT(30.56768910971476 -96.07712313878811) bank33610 +33611 POINT(30.397561577254642 -95.32715515721527) bank33611 +33612 POINT(29.547901678816245 -96.09523423308002) bank33612 +33613 POINT(29.251859211942442 -95.35909622857126) bank33613 +33614 POINT(30.32178707617217 -95.92683391871412) bank33614 +33615 POINT(29.522875163640457 -95.86107985526857) bank33615 +33616 POINT(29.880107404060674 -95.96473235044749) bank33616 +33617 POINT(28.929942348219214 -95.55151499205962) bank33617 +33618 POINT(28.790360210030265 -95.63010945828137) bank33618 +33619 POINT(30.094889584226355 -95.02720083840079) bank33619 +33620 POINT(30.60078666691337 -96.02275477353133) bank33620 +33621 POINT(29.22430255577355 -95.45122279244457) bank33621 +33622 POINT(29.218221778377647 -94.72544643142749) bank33622 +33623 POINT(30.734926423553155 -95.09752204791586) bank33623 +33624 POINT(30.473728182426004 -95.03631997966725) bank33624 +33625 POINT(29.785195377930357 -95.23144505969685) bank33625 +33626 POINT(29.223379922397687 -94.45526680530544) bank33626 +33627 POINT(29.140218915831433 -94.8489542433651) bank33627 +33628 POINT(30.074033628202002 -95.61832930141014) bank33628 +33629 POINT(29.299024705737448 -94.65716037919205) bank33629 +33630 POINT(30.631399213164638 -96.2007009494509) bank33630 +33631 POINT(29.848872684373383 -94.81036087131037) bank33631 +33632 POINT(30.741221037738033 -94.63062342714373) bank33632 +33633 POINT(30.588533757166914 -94.55811772786164) bank33633 +33634 POINT(29.562324107546438 -95.91524749970327) bank33634 +33635 POINT(28.96535615445069 -94.91868084730756) bank33635 +33636 POINT(29.89402110543766 -95.31161004755043) bank33636 +33637 POINT(29.873716218610305 -96.18079230801455) bank33637 +33638 POINT(30.05282425647633 -94.6307491501528) bank33638 +33639 POINT(29.743729652664825 -94.66065844986639) bank33639 +33640 POINT(30.324928852294445 -95.19946436464441) bank33640 +33641 POINT(30.698423269239562 -95.58122780631467) bank33641 +33642 POINT(28.806338550437932 -95.17798324162891) bank33642 +33643 POINT(29.009264614222854 -94.59963087269144) bank33643 +33644 POINT(30.5780104031241 -95.05336094987592) bank33644 +33645 POINT(29.779027177985316 -94.50678035021483) bank33645 +33646 POINT(30.33805833614704 -95.45593434496789) bank33646 +33647 POINT(28.930038909093156 -95.19281205008294) bank33647 +33648 POINT(28.794456577326628 -95.90756447592626) bank33648 +33649 POINT(29.82400391435098 -96.0305489775527) bank33649 +33650 POINT(29.28789521383509 -95.06733341118462) bank33650 +33651 POINT(30.288208644508632 -96.00327660151862) bank33651 +33652 POINT(29.290838945646733 -94.99244465088529) bank33652 +33653 POINT(28.9031120392873 -95.93088711027151) bank33653 +33654 POINT(28.778279815883245 -95.41135598836168) bank33654 +33655 POINT(29.447195745216863 -96.3687793940767) bank33655 +33656 POINT(29.447431589141132 -94.94607442292038) bank33656 +33657 POINT(29.74031464405508 -95.42361650204181) bank33657 +33658 POINT(29.377010114563703 -95.98838691960371) bank33658 +33659 POINT(30.632068842845943 -95.6663391495921) bank33659 +33660 POINT(29.48758350232875 -96.07548647735078) bank33660 +33661 POINT(30.65073997430201 -94.7348555820613) bank33661 +33662 POINT(29.95026876400125 -95.69345172876405) bank33662 +33663 POINT(29.281056753286432 -95.91834725777156) bank33663 +33664 POINT(28.870000251908472 -94.86771351379288) bank33664 +33665 POINT(30.509976219720656 -96.1393441270411) bank33665 +33666 POINT(30.02184319019726 -95.13284782235915) bank33666 +33667 POINT(29.60840183313808 -96.02143159609943) bank33667 +33668 POINT(30.21478219671682 -95.5169905555767) bank33668 +33669 POINT(28.966991065898473 -94.70734010532233) bank33669 +33670 POINT(29.638154366079995 -94.94296746191218) bank33670 +33671 POINT(29.896815094100806 -95.36103357009485) bank33671 +33672 POINT(30.582822077304883 -95.85938177710821) bank33672 +33673 POINT(28.89414750956181 -95.90536006014645) bank33673 +33674 POINT(30.325555630976414 -95.31343642982536) bank33674 +33675 POINT(29.592183756408502 -96.30590709442208) bank33675 +33676 POINT(29.940924350661824 -94.51923436060767) bank33676 +33677 POINT(30.183451866936053 -95.14598606421595) bank33677 +33678 POINT(30.065010841511643 -95.68930599666623) bank33678 +33679 POINT(29.779474976937415 -96.30906618017887) bank33679 +33680 POINT(30.63763820270078 -96.1531496557445) bank33680 +33681 POINT(29.011297543962964 -95.80509884420542) bank33681 +33682 POINT(28.869117835141687 -95.07937695978282) bank33682 +33683 POINT(30.487875161060746 -95.87965191387063) bank33683 +33684 POINT(30.08617574860586 -95.04740220818593) bank33684 +33685 POINT(29.16544778468688 -96.28422320902229) bank33685 +33686 POINT(30.063781042187145 -95.14231889665622) bank33686 +33687 POINT(30.168505748928464 -95.11970584975504) bank33687 +33688 POINT(29.792244302903434 -95.85833245091968) bank33688 +33689 POINT(29.067797638318858 -95.77150044094277) bank33689 +33690 POINT(30.169795164466738 -95.92649472109684) bank33690 +33691 POINT(29.21425406227695 -94.99174434631844) bank33691 +33692 POINT(28.990842325160806 -94.78840663309394) bank33692 +33693 POINT(30.189498801028215 -95.18353635648602) bank33693 +33694 POINT(28.889327758071367 -95.82117901485663) bank33694 +33695 POINT(30.315865815687857 -95.55683625927287) bank33695 +33696 POINT(29.990218250557213 -94.55258707249207) bank33696 +33697 POINT(28.800447763860973 -94.78496639403885) bank33697 +33698 POINT(28.962839795340855 -94.87813118541138) bank33698 +33699 POINT(30.13125313844146 -94.40645561454035) bank33699 +33700 POINT(29.77728690896468 -95.77095424825538) bank33700 +33701 POINT(30.34328323983634 -94.94989564524589) bank33701 +33702 POINT(29.473874153586703 -95.93198222141237) bank33702 +33703 POINT(30.602535932154826 -96.20014536829208) bank33703 +33704 POINT(30.26459547142548 -95.79693905934184) bank33704 +33705 POINT(29.320571090974802 -95.11775610764553) bank33705 +33706 POINT(29.332987285699225 -96.3471562065297) bank33706 +33707 POINT(29.412006672864937 -96.16084463141286) bank33707 +33708 POINT(29.991832692580736 -95.00487154567656) bank33708 +33709 POINT(28.911589492426963 -94.55726219008486) bank33709 +33710 POINT(30.56000193880922 -96.19486029789655) bank33710 +33711 POINT(30.532652137168643 -94.47447599840586) bank33711 +33712 POINT(29.448271398700893 -95.68000921373644) bank33712 +33713 POINT(29.217533590208422 -95.48365275449949) bank33713 +33714 POINT(29.009034490508093 -95.92022725167176) bank33714 +33715 POINT(29.40722676516963 -95.64444355262593) bank33715 +33716 POINT(29.218218582816913 -96.30252598067129) bank33716 +33717 POINT(29.032404142288414 -95.94984198049472) bank33717 +33718 POINT(29.766544401869805 -96.17381123360344) bank33718 +33719 POINT(29.845446525256953 -95.50941805622065) bank33719 +33720 POINT(29.330860627396767 -96.13390049681038) bank33720 +33721 POINT(29.298088790466398 -95.2612720841452) bank33721 +33722 POINT(29.510633376392892 -94.92947076756887) bank33722 +33723 POINT(30.730049512751215 -95.37143827475103) bank33723 +33724 POINT(29.14818799226485 -94.52846456132035) bank33724 +33725 POINT(30.152797904009198 -95.85802792141814) bank33725 +33726 POINT(28.87325630347685 -96.36908534748241) bank33726 +33727 POINT(28.944312003679343 -96.18906116874315) bank33727 +33728 POINT(29.644654943592553 -95.54226559151279) bank33728 +33729 POINT(29.056688034050573 -95.19146895225708) bank33729 +33730 POINT(30.078724019236933 -94.93010452581458) bank33730 +33731 POINT(30.650867861713696 -96.00826338875832) bank33731 +33732 POINT(30.009642196834776 -94.53184298557989) bank33732 +33733 POINT(30.524851671220308 -95.04042082211944) bank33733 +33734 POINT(29.76393813141048 -95.41337738219654) bank33734 +33735 POINT(29.99733118935393 -95.43738831816896) bank33735 +33736 POINT(30.440565495197802 -95.5345565624946) bank33736 +33737 POINT(29.289944000289278 -95.98438867919064) bank33737 +33738 POINT(30.400720110000428 -96.14036870829362) bank33738 +33739 POINT(30.06477058055436 -95.30778733069096) bank33739 +33740 POINT(29.71908498000106 -95.6330497698037) bank33740 +33741 POINT(29.42727263962027 -95.18422147731155) bank33741 +33742 POINT(30.608899581215844 -95.57431375364314) bank33742 +33743 POINT(28.7954410387258 -94.84471856694303) bank33743 +33744 POINT(30.427935056457134 -95.90484138081511) bank33744 +33745 POINT(28.920073850419833 -95.67724966712666) bank33745 +33746 POINT(30.45864655332877 -94.69744210709509) bank33746 +33747 POINT(30.392728857811896 -95.868053918125) bank33747 +33748 POINT(29.50585992042996 -95.45524319448126) bank33748 +33749 POINT(29.336642937253558 -94.5582598949627) bank33749 +33750 POINT(29.772779643308827 -95.9780443523806) bank33750 +33751 POINT(30.45730027217451 -95.87432450955822) bank33751 +33752 POINT(30.724954386180098 -95.9668926990038) bank33752 +33753 POINT(29.978827598269742 -96.04237415478933) bank33753 +33754 POINT(30.182627076311764 -94.86574662826631) bank33754 +33755 POINT(30.344790010296514 -95.4904967774644) bank33755 +33756 POINT(29.370342407946822 -95.37374422053671) bank33756 +33757 POINT(29.78439703345501 -95.86986853523847) bank33757 +33758 POINT(29.86769736623193 -94.37019604734738) bank33758 +33759 POINT(29.673960467066085 -94.87284286039844) bank33759 +33760 POINT(30.30466973634492 -95.03638275480311) bank33760 +33761 POINT(30.035886193018385 -96.24145060293469) bank33761 +33762 POINT(29.76725498595672 -95.37443560627067) bank33762 +33763 POINT(30.729108265532812 -95.82349117590647) bank33763 +33764 POINT(30.185440246272005 -95.3178882442223) bank33764 +33765 POINT(29.022288063565693 -94.78423677530805) bank33765 +33766 POINT(29.663447826881647 -94.54237504374308) bank33766 +33767 POINT(29.128780303027952 -95.93638626815122) bank33767 +33768 POINT(29.881136821697982 -95.02834359048734) bank33768 +33769 POINT(29.374979989888434 -94.98462275275125) bank33769 +33770 POINT(29.52363278320143 -95.1852006249289) bank33770 +33771 POINT(30.188275155572562 -95.54417830187896) bank33771 +33772 POINT(29.878785655042716 -95.10307135031749) bank33772 +33773 POINT(29.499759382625122 -94.72772426151198) bank33773 +33774 POINT(28.863330674096837 -94.97881474789897) bank33774 +33775 POINT(29.285190154995085 -95.96168827945328) bank33775 +33776 POINT(30.46466463487031 -94.47563822823997) bank33776 +33777 POINT(29.599071097028816 -94.37566434856328) bank33777 +33778 POINT(29.566405767291197 -96.1757496858031) bank33778 +33779 POINT(29.76827002260741 -95.59637101523816) bank33779 +33780 POINT(28.801633162608134 -96.28852936725922) bank33780 +33781 POINT(30.186236718924878 -96.20164920443023) bank33781 +33782 POINT(30.109638882907365 -95.73608169256624) bank33782 +33783 POINT(28.829714038339155 -95.67724502776734) bank33783 +33784 POINT(28.833738192851495 -96.17525545894703) bank33784 +33785 POINT(29.094845596376846 -94.83937363641246) bank33785 +33786 POINT(30.411376132248787 -94.64803686935662) bank33786 +33787 POINT(29.276002423628658 -95.31387164603348) bank33787 +33788 POINT(29.18235143577441 -95.79709701383396) bank33788 +33789 POINT(29.30362492318802 -95.94653030571833) bank33789 +33790 POINT(30.33631188556885 -96.15522992845746) bank33790 +33791 POINT(28.9301497632334 -94.58222775298844) bank33791 +33792 POINT(30.31700514655877 -94.92275818239958) bank33792 +33793 POINT(30.62973350323773 -94.65265947246185) bank33793 +33794 POINT(30.175708182553567 -94.89098877353327) bank33794 +33795 POINT(29.475545531695655 -94.78758834004067) bank33795 +33796 POINT(30.75383911785777 -94.53985011369333) bank33796 +33797 POINT(29.25813649646473 -95.2114003219534) bank33797 +33798 POINT(29.2280905907957 -95.19798983401782) bank33798 +33799 POINT(30.039834475476766 -95.41068712494307) bank33799 +33800 POINT(29.611289090165332 -95.77976371585467) bank33800 +33801 POINT(29.212850293348133 -95.10000108777831) bank33801 +33802 POINT(30.312077536119794 -94.51860574517787) bank33802 +33803 POINT(30.480050480177354 -95.91562412044856) bank33803 +33804 POINT(28.778808044038264 -95.10618238515428) bank33804 +33805 POINT(29.111961480347304 -96.00559867440055) bank33805 +33806 POINT(29.27776669175421 -95.0463151809653) bank33806 +33807 POINT(29.522549917886113 -96.2880772056118) bank33807 +33808 POINT(29.664370285731742 -95.33333346489238) bank33808 +33809 POINT(29.521778050061158 -95.82043362820941) bank33809 +33810 POINT(30.385428890487724 -95.69774317450694) bank33810 +33811 POINT(28.918638683726243 -96.27652936185446) bank33811 +33812 POINT(30.373057791215828 -95.81358434292062) bank33812 +33813 POINT(30.49935049878518 -95.43548534404701) bank33813 +33814 POINT(29.656221971833126 -96.06047205282748) bank33814 +33815 POINT(30.205016678016914 -94.37862609241549) bank33815 +33816 POINT(29.45164240247582 -95.07084609805833) bank33816 +33817 POINT(29.225316441552753 -96.09036792159031) bank33817 +33818 POINT(29.067205660919253 -94.47605985279753) bank33818 +33819 POINT(30.248541287657613 -94.92200090419334) bank33819 +33820 POINT(30.187538473720668 -96.34667718423678) bank33820 +33821 POINT(29.22963558361596 -95.29814521504942) bank33821 +33822 POINT(28.87637139144538 -94.40374339343009) bank33822 +33823 POINT(30.73889764394398 -95.0898491778775) bank33823 +33824 POINT(29.36738746168552 -94.95219070434644) bank33824 +33825 POINT(30.71614146404539 -95.00471508560655) bank33825 +33826 POINT(30.186848567614394 -95.39988311520365) bank33826 +33827 POINT(29.893963779569695 -95.92593163747055) bank33827 +33828 POINT(29.545587262954772 -94.67473716757256) bank33828 +33829 POINT(29.236244469425667 -95.44427852564984) bank33829 +33830 POINT(30.109068486292795 -95.51420382843133) bank33830 +33831 POINT(29.85248600085459 -95.85584477736725) bank33831 +33832 POINT(30.177835114775952 -95.15893285281001) bank33832 +33833 POINT(30.69055165414767 -96.19258176514006) bank33833 +33834 POINT(29.756015863959046 -95.2071965025911) bank33834 +33835 POINT(29.22093389057074 -95.39476702816593) bank33835 +33836 POINT(30.287339967931437 -95.81849429118577) bank33836 +33837 POINT(29.75872472354725 -96.00546941351969) bank33837 +33838 POINT(29.37683412689758 -95.00068647244576) bank33838 +33839 POINT(29.975560959045925 -94.70864243635968) bank33839 +33840 POINT(29.78584585383942 -95.6137917325187) bank33840 +33841 POINT(29.71845464926258 -96.02492146124796) bank33841 +33842 POINT(29.979394041022847 -94.38628991665603) bank33842 +33843 POINT(30.459925762775583 -94.5455168265187) bank33843 +33844 POINT(29.99006541641615 -95.3242140781426) bank33844 +33845 POINT(29.034119634994596 -95.68452583021633) bank33845 +33846 POINT(29.19829190590821 -95.59825792231324) bank33846 +33847 POINT(29.560957929709836 -95.99587335141672) bank33847 +33848 POINT(30.625836284591163 -95.85926410340916) bank33848 +33849 POINT(30.3112568641841 -96.12516706120307) bank33849 +33850 POINT(30.057174306260794 -94.8892613937753) bank33850 +33851 POINT(28.793177841732216 -94.48851206473249) bank33851 +33852 POINT(29.134335799184615 -95.14567528005134) bank33852 +33853 POINT(30.620824669982294 -94.56166582543372) bank33853 +33854 POINT(29.737729268917555 -95.9269619567373) bank33854 +33855 POINT(28.879173040089846 -94.74709237817473) bank33855 +33856 POINT(29.346228409643647 -95.86715388021952) bank33856 +33857 POINT(29.178585460790053 -95.03028090938226) bank33857 +33858 POINT(29.490087673300227 -96.0821910552311) bank33858 +33859 POINT(30.650641198298803 -95.4932760256861) bank33859 +33860 POINT(29.138364123915256 -94.95636538844043) bank33860 +33861 POINT(29.00043277510888 -95.8638902388922) bank33861 +33862 POINT(30.74466601730775 -94.94191306192333) bank33862 +33863 POINT(30.421279360608178 -95.48505025498653) bank33863 +33864 POINT(30.379275589459812 -94.77226175729002) bank33864 +33865 POINT(30.222034849235925 -94.61261979045223) bank33865 +33866 POINT(28.838512373315417 -95.66267799706375) bank33866 +33867 POINT(29.80949133635165 -94.96767163096546) bank33867 +33868 POINT(29.077018824576104 -95.61733962680661) bank33868 +33869 POINT(30.656086230940563 -95.49090121209873) bank33869 +33870 POINT(29.261033014967634 -95.98255967836701) bank33870 +33871 POINT(29.95350637177919 -94.82318732443251) bank33871 +33872 POINT(30.037784721433827 -95.77175322716812) bank33872 +33873 POINT(30.073685754437722 -95.65141327274485) bank33873 +33874 POINT(29.8988220951701 -95.63135021418213) bank33874 +33875 POINT(29.32555676228581 -96.05763896282839) bank33875 +33876 POINT(30.143041336682153 -94.92200934432998) bank33876 +33877 POINT(29.12451200068 -95.14144176562849) bank33877 +33878 POINT(30.15247088099712 -94.56744225480769) bank33878 +33879 POINT(30.714628719273634 -95.4175145758815) bank33879 +33880 POINT(29.298313591048363 -95.81495079720938) bank33880 +33881 POINT(30.69136016719944 -95.71230536484903) bank33881 +33882 POINT(28.913201952228093 -95.15097263451922) bank33882 +33883 POINT(29.852535187850947 -95.16146322263944) bank33883 +33884 POINT(29.35019786381874 -95.28143742000839) bank33884 +33885 POINT(30.250681353638 -95.631270088628) bank33885 +33886 POINT(29.061051651531855 -94.48581249241425) bank33886 +33887 POINT(30.029552449957574 -94.80507953029428) bank33887 +33888 POINT(30.72218016244074 -95.9074901923162) bank33888 +33889 POINT(29.320082525064993 -95.04151018633794) bank33889 +33890 POINT(30.646951590672792 -95.1136710084982) bank33890 +33891 POINT(30.070175280577036 -95.9753312843362) bank33891 +33892 POINT(28.963512499840707 -95.80790198523118) bank33892 +33893 POINT(29.953032336177344 -94.59456614943883) bank33893 +33894 POINT(30.479283841856137 -95.56947296920353) bank33894 +33895 POINT(29.651998501014834 -95.82068158100282) bank33895 +33896 POINT(30.532392760369493 -95.08852350620198) bank33896 +33897 POINT(30.08025422410327 -94.44923183350332) bank33897 +33898 POINT(30.1157362222323 -96.20242816184354) bank33898 +33899 POINT(29.13961656040712 -94.43670594313005) bank33899 +33900 POINT(28.846509057980533 -94.50768427076193) bank33900 +33901 POINT(28.956278757224794 -96.34116441274442) bank33901 +33902 POINT(30.63062175213573 -95.78674913651625) bank33902 +33903 POINT(28.89696568041575 -95.99028356909257) bank33903 +33904 POINT(29.379530546485068 -94.87329446725734) bank33904 +33905 POINT(30.518323657879805 -95.7741398603812) bank33905 +33906 POINT(30.26891729864675 -95.90309373425066) bank33906 +33907 POINT(30.65954390498881 -95.63193605605116) bank33907 +33908 POINT(30.19003424296787 -95.19498905518395) bank33908 +33909 POINT(29.439027024711155 -95.48165979622505) bank33909 +33910 POINT(30.334010874098734 -96.34887372659337) bank33910 +33911 POINT(29.411787903953797 -95.60458624394224) bank33911 +33912 POINT(29.735551697119686 -94.51126557114871) bank33912 +33913 POINT(29.168872757567666 -95.78405303718479) bank33913 +33914 POINT(29.60102616928803 -95.37184531059671) bank33914 +33915 POINT(30.147491514204948 -95.68634644895549) bank33915 +33916 POINT(30.232094343604576 -95.73584631527557) bank33916 +33917 POINT(30.542244483419125 -94.39223620155501) bank33917 +33918 POINT(29.21883409542509 -96.28344610518698) bank33918 +33919 POINT(29.108097967742125 -96.07578978824482) bank33919 +33920 POINT(29.06265546514429 -95.58098067776193) bank33920 +33921 POINT(29.168916248647438 -94.91483798321019) bank33921 +33922 POINT(29.5382728947527 -95.64487877572634) bank33922 +33923 POINT(30.125683814589017 -95.92230633789418) bank33923 +33924 POINT(29.100490299085685 -95.22851081700792) bank33924 +33925 POINT(28.90538872049919 -94.72379759735722) bank33925 +33926 POINT(30.33528926634424 -95.44920833765072) bank33926 +33927 POINT(29.1074678483354 -95.26664041418088) bank33927 +33928 POINT(28.803576401785165 -95.77856921386335) bank33928 +33929 POINT(30.04148686149964 -95.368360211653) bank33929 +33930 POINT(30.048915458851667 -95.5105689772345) bank33930 +33931 POINT(30.298280107720405 -95.93346534785371) bank33931 +33932 POINT(29.128687657865743 -95.17122363780165) bank33932 +33933 POINT(29.609204515875213 -94.64518912139563) bank33933 +33934 POINT(30.33765887424877 -96.36649723553381) bank33934 +33935 POINT(29.812545044651 -95.39443875065017) bank33935 +33936 POINT(30.461441543226602 -95.6736159256183) bank33936 +33937 POINT(30.590489470137793 -94.88900100831108) bank33937 +33938 POINT(29.536258745624387 -96.0522203492018) bank33938 +33939 POINT(30.126513006746737 -94.37514498035634) bank33939 +33940 POINT(30.15733502819096 -94.55769041421277) bank33940 +33941 POINT(28.931492257670143 -94.7431563112688) bank33941 +33942 POINT(29.47405343412561 -96.2021091063991) bank33942 +33943 POINT(30.517041653448935 -96.01842250813753) bank33943 +33944 POINT(30.7331793000238 -96.04409207555685) bank33944 +33945 POINT(28.7797804563751 -95.29250498529416) bank33945 +33946 POINT(30.720423724873058 -96.01665654615962) bank33946 +33947 POINT(29.831696226558115 -96.27412658667048) bank33947 +33948 POINT(30.012065464989107 -94.62006944077339) bank33948 +33949 POINT(29.86602985549202 -95.02515602692503) bank33949 +33950 POINT(30.73901741400587 -94.54270965799614) bank33950 +33951 POINT(30.329762496258343 -96.32916460906647) bank33951 +33952 POINT(29.524524219251155 -94.61055506458756) bank33952 +33953 POINT(29.249017791636305 -95.4505359448668) bank33953 +33954 POINT(29.841297110325502 -96.01687523141504) bank33954 +33955 POINT(29.328274164787377 -95.92168631719805) bank33955 +33956 POINT(30.682615349931964 -95.81550278886897) bank33956 +33957 POINT(28.807641175988607 -94.489403552005) bank33957 +33958 POINT(29.87027046077639 -94.44944611531355) bank33958 +33959 POINT(29.810130292886136 -95.59868951578062) bank33959 +33960 POINT(29.54148350271353 -95.5741543812489) bank33960 +33961 POINT(30.324327852562536 -95.33686643542005) bank33961 +33962 POINT(30.022426786714863 -95.79892807946302) bank33962 +33963 POINT(30.216510057480154 -95.11818457926086) bank33963 +33964 POINT(30.06427422781676 -96.0091795601202) bank33964 +33965 POINT(29.22083215279174 -95.51162991344545) bank33965 +33966 POINT(30.121968251565658 -96.03178127402099) bank33966 +33967 POINT(30.07292871100234 -96.02124958173728) bank33967 +33968 POINT(29.80833225066084 -94.61902208411801) bank33968 +33969 POINT(28.9821450846468 -95.23208144009365) bank33969 +33970 POINT(30.746516502404372 -96.28445349101544) bank33970 +33971 POINT(29.01171136674434 -96.19729424793955) bank33971 +33972 POINT(29.498407660076293 -95.02338532733036) bank33972 +33973 POINT(28.799244102275637 -95.14567397327497) bank33973 +33974 POINT(29.109894688232536 -95.01179128271968) bank33974 +33975 POINT(29.32427789523644 -94.90691710072078) bank33975 +33976 POINT(29.489626340599596 -94.74412461496672) bank33976 +33977 POINT(30.246182831813904 -95.81942112889357) bank33977 +33978 POINT(29.843939163294262 -94.43861536505011) bank33978 +33979 POINT(30.169163849869303 -96.03742984105811) bank33979 +33980 POINT(29.77242863912024 -94.88869392925686) bank33980 +33981 POINT(29.13002200352968 -95.95825672544186) bank33981 +33982 POINT(28.82640287448978 -94.7308303809167) bank33982 +33983 POINT(28.834964084201044 -95.27255957077428) bank33983 +33984 POINT(30.391892162241835 -95.62776649818552) bank33984 +33985 POINT(30.091149702705266 -95.20144171905578) bank33985 +33986 POINT(28.774598220701417 -95.33990450560015) bank33986 +33987 POINT(29.007018801458877 -95.04045284171737) bank33987 +33988 POINT(30.446411802730346 -95.4677392027244) bank33988 +33989 POINT(29.497003296352794 -94.77239910801967) bank33989 +33990 POINT(30.243508174179397 -95.4128771480285) bank33990 +33991 POINT(29.058515298046675 -94.93667241910892) bank33991 +33992 POINT(30.406408680887875 -94.66013776876153) bank33992 +33993 POINT(30.383516225943062 -95.37136970320306) bank33993 +33994 POINT(29.72473157921184 -94.86378430414345) bank33994 +33995 POINT(30.632878891976294 -94.83378328723977) bank33995 +33996 POINT(28.93258660400381 -95.4937293555143) bank33996 +33997 POINT(29.168493774222153 -94.54233754721415) bank33997 +33998 POINT(30.1980716798769 -96.1109711609451) bank33998 +33999 POINT(30.390451200377488 -94.86236258888538) bank33999 +34000 POINT(29.11075200825848 -95.04406340149285) bank34000 +34001 POINT(28.822843475271533 -94.71987138546054) bank34001 +34002 POINT(29.8106641550479 -96.24656657038375) bank34002 +34003 POINT(29.21346586074208 -95.7992441358683) bank34003 +34004 POINT(30.741803286828734 -94.40079586680146) bank34004 +34005 POINT(30.71082586651126 -95.81974708583661) bank34005 +34006 POINT(29.197091850390258 -94.7511909878449) bank34006 +34007 POINT(29.035193072263816 -94.88607870855672) bank34007 +34008 POINT(29.706363999791726 -94.75520362742827) bank34008 +34009 POINT(29.650090510048457 -96.22824762833878) bank34009 +34010 POINT(28.820786302671088 -95.26390691530143) bank34010 +34011 POINT(30.471330386507304 -96.3470935371245) bank34011 +34012 POINT(29.618542183352833 -96.06675038021638) bank34012 +34013 POINT(30.112236812799914 -94.96665768840013) bank34013 +34014 POINT(28.820488401016565 -95.94427544039262) bank34014 +34015 POINT(30.01638746635405 -94.48138973144373) bank34015 +34016 POINT(29.496202878127153 -95.42817968953814) bank34016 +34017 POINT(29.375184383211252 -94.83128913548879) bank34017 +34018 POINT(30.309140080149334 -96.29701680452682) bank34018 +34019 POINT(29.06997334077948 -95.25746627512035) bank34019 +34020 POINT(30.359272658381062 -96.12548103385049) bank34020 +34021 POINT(30.4279551520698 -96.22615385880995) bank34021 +34022 POINT(30.594939773368896 -94.63547123119987) bank34022 +34023 POINT(29.528473401946137 -96.18229312717904) bank34023 +34024 POINT(29.928668232403847 -95.10434231174939) bank34024 +34025 POINT(29.073559090239925 -96.0459750809748) bank34025 +34026 POINT(30.261852086970453 -95.78886499161807) bank34026 +34027 POINT(30.215391078928103 -94.67999099796569) bank34027 +34028 POINT(28.76732466227724 -95.75254270489152) bank34028 +34029 POINT(30.239620671399017 -95.01693655732565) bank34029 +34030 POINT(29.059471915422872 -95.41523322401913) bank34030 +34031 POINT(30.733047914553616 -94.98717354364547) bank34031 +34032 POINT(30.742002907651663 -94.92800203685596) bank34032 +34033 POINT(29.396567207578713 -95.05148376354211) bank34033 +34034 POINT(30.757237412156154 -95.62536130287153) bank34034 +34035 POINT(28.885912127838324 -96.22967481431584) bank34035 +34036 POINT(30.738524939947524 -94.76261252395217) bank34036 +34037 POINT(29.233148841671994 -96.23507046981638) bank34037 +34038 POINT(30.490053463261987 -94.67691336141343) bank34038 +34039 POINT(29.772712071479248 -95.48158581989723) bank34039 +34040 POINT(29.51440713407582 -94.56191793526872) bank34040 +34041 POINT(29.099169444116274 -96.31126744644277) bank34041 +34042 POINT(29.629250882910394 -94.51253673040475) bank34042 +34043 POINT(30.484506151246563 -95.26178794475078) bank34043 +34044 POINT(29.01124893474473 -95.42386024555711) bank34044 +34045 POINT(29.309390145889736 -96.18568772754233) bank34045 +34046 POINT(30.039001860759793 -94.8751748674643) bank34046 +34047 POINT(29.574791194529666 -94.43436848511107) bank34047 +34048 POINT(28.862030608471823 -96.01188718003988) bank34048 +34049 POINT(29.51802859648953 -96.22895291648754) bank34049 +34050 POINT(29.845313170381424 -94.86977001559583) bank34050 +34051 POINT(29.73110765653717 -95.57216887833837) bank34051 +34052 POINT(28.8623829782408 -94.50575437941039) bank34052 +34053 POINT(30.39518241656062 -94.4035771858935) bank34053 +34054 POINT(29.631130457199223 -94.96913035736027) bank34054 +34055 POINT(30.54416003753921 -94.84497565876777) bank34055 +34056 POINT(30.6028427383718 -95.53084636085788) bank34056 +34057 POINT(30.256023209077053 -94.50145823986486) bank34057 +34058 POINT(29.561308392747453 -95.08594430764477) bank34058 +34059 POINT(29.639220112372527 -95.9727013435354) bank34059 +34060 POINT(29.64190879818999 -95.33979573012483) bank34060 +34061 POINT(28.949988120874046 -94.57334690167053) bank34061 +34062 POINT(29.152758235645084 -94.7679394981093) bank34062 +34063 POINT(30.727667472615583 -95.06217883936569) bank34063 +34064 POINT(28.849561777443203 -95.52479920254835) bank34064 +34065 POINT(30.557708903202425 -94.7120213806831) bank34065 +34066 POINT(28.927733178513563 -94.73314309219715) bank34066 +34067 POINT(29.564977456515614 -94.74468663705035) bank34067 +34068 POINT(29.216366629401758 -96.15610608969838) bank34068 +34069 POINT(30.63096000321538 -94.90894299043426) bank34069 +34070 POINT(29.027074481009713 -95.73317729243517) bank34070 +34071 POINT(29.78910994166078 -95.63447991988093) bank34071 +34072 POINT(30.68218289691694 -95.93594258496343) bank34072 +34073 POINT(28.82822075054305 -94.80722359643369) bank34073 +34074 POINT(29.94111198168109 -95.97330450294038) bank34074 +34075 POINT(30.67709254066447 -95.69404129518665) bank34075 +34076 POINT(29.143312085231898 -95.47223960251758) bank34076 +34077 POINT(29.392088271732373 -95.85832902361301) bank34077 +34078 POINT(29.385127729517603 -95.78876855130895) bank34078 +34079 POINT(29.62978522978611 -95.43014385145831) bank34079 +34080 POINT(29.663911294323235 -95.19799111845262) bank34080 +34081 POINT(28.969524684053102 -94.73452941956377) bank34081 +34082 POINT(29.799967193278945 -95.55345204430718) bank34082 +34083 POINT(29.541215967684728 -96.22876335948374) bank34083 +34084 POINT(30.42742309507846 -95.60800728130117) bank34084 +34085 POINT(29.212876523286283 -95.93803221299076) bank34085 +34086 POINT(30.4502324088301 -94.758106576954) bank34086 +34087 POINT(29.605549149222742 -95.53210888979545) bank34087 +34088 POINT(30.35152028261782 -95.99981017391671) bank34088 +34089 POINT(30.167223396860404 -95.89320412029642) bank34089 +34090 POINT(29.958881730949667 -95.06131917104005) bank34090 +34091 POINT(30.147526751462653 -95.58827295583079) bank34091 +34092 POINT(29.555083246224054 -94.96147056078387) bank34092 +34093 POINT(29.673452515366076 -94.90095678148091) bank34093 +34094 POINT(29.270211307940176 -94.52593722412362) bank34094 +34095 POINT(30.05835766334316 -94.46642673539921) bank34095 +34096 POINT(30.033126331723157 -95.89726413655058) bank34096 +34097 POINT(30.018579063013483 -95.11441622484892) bank34097 +34098 POINT(30.55730451487305 -94.76764826284715) bank34098 +34099 POINT(29.078586537519357 -94.98167998255559) bank34099 +34100 POINT(29.931063088137364 -95.67276444866482) bank34100 +34101 POINT(29.158990662092968 -95.13654062562658) bank34101 +34102 POINT(29.824160005139245 -95.81641673042424) bank34102 +34103 POINT(29.380830986992407 -96.3169756394218) bank34103 +34104 POINT(29.400759640923066 -96.00897021922809) bank34104 +34105 POINT(30.028708585161382 -95.10287685886485) bank34105 +34106 POINT(28.773900297185303 -94.54234829009681) bank34106 +34107 POINT(29.338979472314595 -94.4410864016265) bank34107 +34108 POINT(30.215842664105633 -94.41419343154341) bank34108 +34109 POINT(30.14598611850971 -95.88534217245416) bank34109 +34110 POINT(29.385192512837232 -96.33944999215068) bank34110 +34111 POINT(29.754262786941926 -94.99139531844992) bank34111 +34112 POINT(29.804441599332424 -94.94536159707378) bank34112 +34113 POINT(30.13768250173418 -96.3598273499795) bank34113 +34114 POINT(28.963538398816493 -96.03656010509872) bank34114 +34115 POINT(29.219953721921307 -95.8207049966543) bank34115 +34116 POINT(28.962114118160905 -94.8902750488464) bank34116 +34117 POINT(28.9088513093737 -95.0911886408288) bank34117 +34118 POINT(28.795840318318042 -95.14945502780947) bank34118 +34119 POINT(30.152085263423185 -95.04069443409418) bank34119 +34120 POINT(29.64280866211989 -95.37053687480199) bank34120 +34121 POINT(30.264846380956378 -95.35012366841526) bank34121 +34122 POINT(29.956097983077083 -95.30191109274679) bank34122 +34123 POINT(29.099465076288936 -95.50162531379797) bank34123 +34124 POINT(29.947495076165726 -94.55534866092117) bank34124 +34125 POINT(30.102503692045264 -95.41726026476583) bank34125 +34126 POINT(29.94472386921519 -95.96051226798386) bank34126 +34127 POINT(29.098266220255436 -94.7197228927058) bank34127 +34128 POINT(29.999487459049586 -94.95609215386115) bank34128 +34129 POINT(29.323658259953415 -94.67493304758088) bank34129 +34130 POINT(30.46042686637522 -95.52860571942648) bank34130 +34131 POINT(29.87888405650225 -94.44064279418278) bank34131 +34132 POINT(29.839994082392522 -95.81913461131352) bank34132 +34133 POINT(28.762948173985965 -94.90163436001336) bank34133 +34134 POINT(28.82375434811676 -95.77343292094982) bank34134 +34135 POINT(29.932185886229693 -95.0135222336736) bank34135 +34136 POINT(29.782607083307273 -95.03007315159775) bank34136 +34137 POINT(29.317145375049716 -96.13024199401504) bank34137 +34138 POINT(29.56053722988794 -95.85759353095571) bank34138 +34139 POINT(29.643822780796484 -95.14309196958648) bank34139 +34140 POINT(29.63231752354313 -95.19899560739296) bank34140 +34141 POINT(29.27057023862029 -94.37451225751967) bank34141 +34142 POINT(30.459236680517066 -94.37512474257231) bank34142 +34143 POINT(29.902853968886642 -95.83121043953507) bank34143 +34144 POINT(29.282764345848957 -94.83591152513716) bank34144 +34145 POINT(30.06397185394615 -96.10778873859164) bank34145 +34146 POINT(29.49799316335355 -95.43189301231575) bank34146 +34147 POINT(30.757553807727934 -94.7849619215798) bank34147 +34148 POINT(29.672214798052124 -95.93163457662313) bank34148 +34149 POINT(30.69058810172607 -95.75862179558777) bank34149 +34150 POINT(30.642909489654024 -95.44925192686907) bank34150 +34151 POINT(28.823072828529263 -95.85338602685457) bank34151 +34152 POINT(30.24011801195801 -94.85677565222389) bank34152 +34153 POINT(29.432616921448982 -94.69792320419383) bank34153 +34154 POINT(30.195498973575248 -95.1001324504417) bank34154 +34155 POINT(30.338413941076304 -95.23654899484691) bank34155 +34156 POINT(29.65330851699413 -95.46241359944084) bank34156 +34157 POINT(30.584202113469814 -95.59243559814792) bank34157 +34158 POINT(30.474346203292654 -95.0327623390668) bank34158 +34159 POINT(29.40026462192907 -95.53728149342311) bank34159 +34160 POINT(30.313049970901456 -94.50763197200959) bank34160 +34161 POINT(30.522888709057884 -94.46336140356937) bank34161 +34162 POINT(29.767913137116647 -94.52197490509708) bank34162 +34163 POINT(29.059976529051585 -94.54167042213756) bank34163 +34164 POINT(29.522450224993783 -95.69872724338911) bank34164 +34165 POINT(29.66979176941974 -95.85504461464839) bank34165 +34166 POINT(29.329998742358395 -94.56409182697446) bank34166 +34167 POINT(29.333112787375647 -95.09206847734933) bank34167 +34168 POINT(29.073600110932283 -95.50795135832841) bank34168 +34169 POINT(29.811814421706515 -96.3472167512455) bank34169 +34170 POINT(30.252324395104814 -94.38921578527847) bank34170 +34171 POINT(29.226198606754927 -95.00820348930195) bank34171 +34172 POINT(28.838352410281477 -95.76029752037712) bank34172 +34173 POINT(29.450028915078043 -94.8943137531206) bank34173 +34174 POINT(28.821103442623354 -94.81844729009072) bank34174 +34175 POINT(29.860664708024938 -94.68451504905711) bank34175 +34176 POINT(30.55015182543866 -95.29587515721383) bank34176 +34177 POINT(30.49683146154104 -94.67607532032095) bank34177 +34178 POINT(30.31301006623383 -95.57846611518742) bank34178 +34179 POINT(29.72500923187283 -96.23442788535142) bank34179 +34180 POINT(29.243751556289755 -94.63197033123372) bank34180 +34181 POINT(29.29135240020013 -96.17000939361857) bank34181 +34182 POINT(30.601487093284607 -95.21973356257492) bank34182 +34183 POINT(29.88103805605729 -95.70499090812832) bank34183 +34184 POINT(30.641841265860638 -94.76036676233979) bank34184 +34185 POINT(29.404190064951194 -94.83271774850508) bank34185 +34186 POINT(30.545699602314766 -95.18092261235395) bank34186 +34187 POINT(30.012764026086725 -95.32417546656701) bank34187 +34188 POINT(29.153916924851295 -95.17690675281025) bank34188 +34189 POINT(29.144797922204376 -95.34748554722117) bank34189 +34190 POINT(29.202072274523545 -94.95432287438571) bank34190 +34191 POINT(29.610863461817203 -95.25577166293375) bank34191 +34192 POINT(28.963195390809027 -94.5922320534186) bank34192 +34193 POINT(29.718851217190682 -94.72396588077929) bank34193 +34194 POINT(30.296196959423003 -95.20012348530992) bank34194 +34195 POINT(29.286579080531336 -94.880090829899) bank34195 +34196 POINT(30.306763681411265 -96.14726864600651) bank34196 +34197 POINT(29.424476878998533 -96.29718240441876) bank34197 +34198 POINT(29.97524870013301 -95.6787837720659) bank34198 +34199 POINT(29.13201324920303 -96.23086341339103) bank34199 +34200 POINT(30.746609801673895 -94.74862772154685) bank34200 +34201 POINT(29.797120906965485 -94.77610441289853) bank34201 +34202 POINT(30.15767123919695 -95.52707712621296) bank34202 +34203 POINT(29.826631262781174 -95.88524159201162) bank34203 +34204 POINT(30.117144496701304 -95.723910782856) bank34204 +34205 POINT(29.100357827181405 -94.75832493923082) bank34205 +34206 POINT(29.45796572393751 -95.26753834202415) bank34206 +34207 POINT(29.698335184321685 -95.19129634637045) bank34207 +34208 POINT(29.544013930863514 -95.86107290998875) bank34208 +34209 POINT(30.387206487418833 -94.92032639510442) bank34209 +34210 POINT(28.837513987101865 -95.27389058740475) bank34210 +34211 POINT(30.026391457259923 -94.93883717714603) bank34211 +34212 POINT(29.029878415745955 -95.3874425847987) bank34212 +34213 POINT(30.756223099720287 -94.77912258303196) bank34213 +34214 POINT(29.02725105527364 -95.44977292411015) bank34214 +34215 POINT(29.27189634861397 -95.61220035304952) bank34215 +34216 POINT(30.755164918682077 -95.57154420534727) bank34216 +34217 POINT(28.994442227361255 -94.80964311915939) bank34217 +34218 POINT(28.811460175440395 -95.94651350510804) bank34218 +34219 POINT(28.82539441697535 -94.60966555985242) bank34219 +34220 POINT(28.987177468918688 -94.6684740788711) bank34220 +34221 POINT(29.180224027169125 -95.94209329286956) bank34221 +34222 POINT(29.182627788463904 -95.77068058815988) bank34222 +34223 POINT(28.938198266184095 -95.08077047294614) bank34223 +34224 POINT(29.207051007836565 -96.08988148077289) bank34224 +34225 POINT(30.213097490162895 -94.62552071086002) bank34225 +34226 POINT(30.75542819500836 -94.58186661623753) bank34226 +34227 POINT(29.015474520040026 -96.13000199069084) bank34227 +34228 POINT(29.182854949670364 -95.34346285463631) bank34228 +34229 POINT(29.263215395171926 -96.06071254262002) bank34229 +34230 POINT(29.679066506365093 -96.11814592714965) bank34230 +34231 POINT(29.173437691450268 -95.5412774906951) bank34231 +34232 POINT(29.88925507121722 -95.23824801530415) bank34232 +34233 POINT(29.179370955915537 -95.15534736729585) bank34233 +34234 POINT(30.567907025820293 -95.80086995733294) bank34234 +34235 POINT(30.082751044204525 -94.73985971049015) bank34235 +34236 POINT(29.884369045803336 -95.66256969041642) bank34236 +34237 POINT(29.611070123886808 -94.50056602431019) bank34237 +34238 POINT(29.20595411691784 -96.26069074953405) bank34238 +34239 POINT(30.742018511570006 -94.6225676113767) bank34239 +34240 POINT(29.371349076800932 -94.95772240391818) bank34240 +34241 POINT(29.376744107284356 -95.45804054600035) bank34241 +34242 POINT(29.975226992163456 -94.59718016409694) bank34242 +34243 POINT(29.73342049177809 -94.67107175475319) bank34243 +34244 POINT(30.464664467264193 -96.1619864306011) bank34244 +34245 POINT(30.04977881681337 -95.2830910619841) bank34245 +34246 POINT(29.60926127034669 -94.84092577952838) bank34246 +34247 POINT(28.85401301212577 -96.03607194700379) bank34247 +34248 POINT(29.03221064451486 -95.11276393200288) bank34248 +34249 POINT(29.18607488688931 -96.19465673231159) bank34249 +34250 POINT(30.503823036918057 -94.65183843097736) bank34250 +34251 POINT(28.835432033822215 -95.7360255982107) bank34251 +34252 POINT(30.65354427501867 -95.35833797526297) bank34252 +34253 POINT(29.245226800833375 -95.43145027911312) bank34253 +34254 POINT(30.14683891004365 -95.55532486417366) bank34254 +34255 POINT(29.678723219694884 -95.05829345951165) bank34255 +34256 POINT(30.57920929044801 -94.37647174703865) bank34256 +34257 POINT(30.410603410313414 -94.86678719636774) bank34257 +34258 POINT(28.902337066596818 -95.15576603308152) bank34258 +34259 POINT(30.283254071771346 -95.52138733851446) bank34259 +34260 POINT(29.600343135068563 -95.01971277676397) bank34260 +34261 POINT(30.492826782580572 -95.78304465977108) bank34261 +34262 POINT(29.096415473383264 -94.87331679698393) bank34262 +34263 POINT(29.662780866016234 -95.69194005218226) bank34263 +34264 POINT(29.286154711513998 -94.58144329595972) bank34264 +34265 POINT(29.063615346819084 -95.71649717020087) bank34265 +34266 POINT(29.086704390948537 -96.00291956432667) bank34266 +34267 POINT(28.784465097735314 -94.56874393494282) bank34267 +34268 POINT(29.60114407124965 -96.06791002267177) bank34268 +34269 POINT(29.949233279377438 -94.50587303885442) bank34269 +34270 POINT(29.316780643096074 -95.27030797488501) bank34270 +34271 POINT(29.658710647698854 -95.7318433149575) bank34271 +34272 POINT(29.3202539398418 -95.95173506582574) bank34272 +34273 POINT(29.63984615955335 -96.06305749527701) bank34273 +34274 POINT(30.325632493426294 -96.11352915502398) bank34274 +34275 POINT(29.74892987924416 -94.70440312313033) bank34275 +34276 POINT(29.085484370375514 -95.24884390548347) bank34276 +34277 POINT(29.396627001324614 -95.00807298916318) bank34277 +34278 POINT(30.40653659428304 -95.63701176082459) bank34278 +34279 POINT(29.26780926781259 -95.7630680725996) bank34279 +34280 POINT(30.75405056901536 -94.6092396798273) bank34280 +34281 POINT(29.33537832386248 -94.57925541877235) bank34281 +34282 POINT(30.185624907947236 -95.5547387188008) bank34282 +34283 POINT(29.571129562106876 -94.5567052527397) bank34283 +34284 POINT(28.771148341810136 -94.5901591046672) bank34284 +34285 POINT(30.299104534972646 -94.68003779408369) bank34285 +34286 POINT(30.559421821374563 -95.59962480498231) bank34286 +34287 POINT(29.541494397898095 -96.19067103524482) bank34287 +34288 POINT(30.165480588113745 -96.06832978476099) bank34288 +34289 POINT(29.412727062337883 -95.18746264203584) bank34289 +34290 POINT(30.24035853898878 -94.45670857037743) bank34290 +34291 POINT(30.24072886290603 -95.25617076281841) bank34291 +34292 POINT(28.779444267021795 -95.98327359509494) bank34292 +34293 POINT(30.19529541064037 -95.35667899992131) bank34293 +34294 POINT(30.051698650707515 -95.56934048087273) bank34294 +34295 POINT(30.02939723812756 -96.28642118351416) bank34295 +34296 POINT(29.61338737724834 -95.32920142332439) bank34296 +34297 POINT(29.285580406413324 -96.36815301886627) bank34297 +34298 POINT(30.38249534595472 -94.91307077043112) bank34298 +34299 POINT(29.033064716743798 -96.25404625963421) bank34299 +34300 POINT(29.859524256815956 -94.67171885060228) bank34300 +34301 POINT(29.47474046312407 -95.92175321196602) bank34301 +34302 POINT(30.60908608248186 -95.98094189252984) bank34302 +34303 POINT(28.959486324793062 -94.75358838682163) bank34303 +34304 POINT(29.348845698416135 -95.34309391752271) bank34304 +34305 POINT(30.22822220138162 -95.35659231377886) bank34305 +34306 POINT(30.21825890580989 -96.10274618621023) bank34306 +34307 POINT(30.759231667887107 -95.57512963862972) bank34307 +34308 POINT(28.780064524379753 -94.94951468732573) bank34308 +34309 POINT(29.984827129434084 -96.1574984598645) bank34309 +34310 POINT(30.364993202454528 -95.44302466896832) bank34310 +34311 POINT(29.64806423284104 -95.09629634394825) bank34311 +34312 POINT(28.813099940914086 -96.0303038980287) bank34312 +34313 POINT(28.975932921323647 -95.03917691862733) bank34313 +34314 POINT(28.763105408680037 -96.21923601417669) bank34314 +34315 POINT(29.239706224101077 -96.03592187519028) bank34315 +34316 POINT(28.821725278428563 -96.01520587678972) bank34316 +34317 POINT(29.246783990312387 -95.55815212036951) bank34317 +34318 POINT(30.46521050196312 -95.41316711594794) bank34318 +34319 POINT(29.20243960764391 -96.25680497218187) bank34319 +34320 POINT(29.330392918713667 -95.20344428898282) bank34320 +34321 POINT(29.119073233607928 -96.23753463354838) bank34321 +34322 POINT(29.876668404492428 -95.38238998023186) bank34322 +34323 POINT(28.898596225993103 -95.26530431204517) bank34323 +34324 POINT(30.521595033589037 -94.39496517092297) bank34324 +34325 POINT(30.116022893645752 -95.13923063482521) bank34325 +34326 POINT(30.758759660706367 -94.5984078085643) bank34326 +34327 POINT(29.090659785745583 -96.12572788717176) bank34327 +34328 POINT(29.53259140738104 -94.9705136403331) bank34328 +34329 POINT(29.466910669373615 -95.63362413138297) bank34329 +34330 POINT(29.27826161123682 -95.04347714641203) bank34330 +34331 POINT(29.711654975479618 -95.3537802258378) bank34331 +34332 POINT(29.8459883172436 -94.38589737309732) bank34332 +34333 POINT(30.002278832625056 -94.40340018925897) bank34333 +34334 POINT(29.49886190379204 -95.92781221144257) bank34334 +34335 POINT(29.158717874713112 -96.30794095357622) bank34335 +34336 POINT(29.59999315336163 -95.84741539644389) bank34336 +34337 POINT(29.29764190640266 -94.99112875423911) bank34337 +34338 POINT(30.446866589045754 -95.86233053182144) bank34338 +34339 POINT(29.47762705034434 -95.05158011981058) bank34339 +34340 POINT(29.50388090211899 -95.45305088531843) bank34340 +34341 POINT(29.525614683286424 -95.19871828900544) bank34341 +34342 POINT(28.892241303685296 -95.21251830212034) bank34342 +34343 POINT(30.379386153911433 -94.75101153703653) bank34343 +34344 POINT(29.078686275833682 -95.7197905531266) bank34344 +34345 POINT(30.19862678514986 -94.90722725683521) bank34345 +34346 POINT(30.740222168794663 -96.35015595621061) bank34346 +34347 POINT(30.036824179815703 -95.84135529624056) bank34347 +34348 POINT(29.056670462500144 -96.2831226740453) bank34348 +34349 POINT(30.088464763637138 -95.21531319928513) bank34349 +34350 POINT(28.840757094157567 -95.71161728741299) bank34350 +34351 POINT(28.830526904245755 -94.9494397466937) bank34351 +34352 POINT(30.10940745894572 -95.36758438934046) bank34352 +34353 POINT(29.122997524340896 -95.01005685392501) bank34353 +34354 POINT(30.30655944605196 -95.3595381420777) bank34354 +34355 POINT(30.352515204071963 -94.95794836212865) bank34355 +34356 POINT(29.914455557532712 -95.71764229692248) bank34356 +34357 POINT(30.04647467909619 -95.79992285677) bank34357 +34358 POINT(29.671289846486093 -94.79256765562731) bank34358 +34359 POINT(29.199302153783997 -95.50619341970078) bank34359 +34360 POINT(30.480743045938837 -95.4354144741406) bank34360 +34361 POINT(29.121379276922042 -95.09317054553642) bank34361 +34362 POINT(28.958300923025813 -96.00788085771102) bank34362 +34363 POINT(29.432989374120595 -95.27478347000235) bank34363 +34364 POINT(28.92200114387956 -96.19754056370601) bank34364 +34365 POINT(30.10588847409309 -95.58128702796704) bank34365 +34366 POINT(29.514639804686826 -95.38186734378607) bank34366 +34367 POINT(30.484832867684545 -94.37214052429911) bank34367 +34368 POINT(30.361770576809757 -95.73606033264967) bank34368 +34369 POINT(29.737915758759094 -96.03489757844082) bank34369 +34370 POINT(28.909074857318103 -95.52918718629674) bank34370 +34371 POINT(29.336885395822776 -95.05417577203882) bank34371 +34372 POINT(30.329793452674718 -94.93295799646666) bank34372 +34373 POINT(29.365228920043833 -95.77515326329485) bank34373 +34374 POINT(30.708667494449344 -95.95464433094558) bank34374 +34375 POINT(30.243039955530367 -96.04539339556914) bank34375 +34376 POINT(28.764330714640284 -95.98820230540115) bank34376 +34377 POINT(30.058472680790132 -96.12194653186276) bank34377 +34378 POINT(30.223874419895463 -95.23745543536897) bank34378 +34379 POINT(30.74915394984404 -95.49790234617322) bank34379 +34380 POINT(29.375473693029917 -94.53888929428193) bank34380 +34381 POINT(29.85745481412813 -94.72636176288847) bank34381 +34382 POINT(30.075810176084136 -96.23505895108933) bank34382 +34383 POINT(29.382027070250466 -95.00655719685288) bank34383 +34384 POINT(29.263913915583206 -95.34850143350187) bank34384 +34385 POINT(29.7946290643205 -94.96796732253983) bank34385 +34386 POINT(29.06092111540552 -94.80675576717996) bank34386 +34387 POINT(30.655118454895486 -95.45497557990163) bank34387 +34388 POINT(29.354179457854315 -96.11826693820655) bank34388 +34389 POINT(28.82124937051505 -95.55806765328121) bank34389 +34390 POINT(29.874403794451485 -96.09757154875058) bank34390 +34391 POINT(28.986789165819726 -95.44459883998033) bank34391 +34392 POINT(29.403794418076423 -95.24860346209641) bank34392 +34393 POINT(29.7146728321105 -95.15872353412391) bank34393 +34394 POINT(30.532361063871818 -95.67993871205196) bank34394 +34395 POINT(28.7814000646842 -95.4961962086335) bank34395 +34396 POINT(29.55406793374584 -96.12732672277069) bank34396 +34397 POINT(28.993075585663643 -95.31605492598393) bank34397 +34398 POINT(29.18140052517129 -95.5554601722186) bank34398 +34399 POINT(29.436664496135503 -95.43134539084005) bank34399 +34400 POINT(29.19720519670869 -95.72706854007755) bank34400 +34401 POINT(29.197828884397534 -96.30864553766907) bank34401 +34402 POINT(30.376557357646607 -95.67281685733792) bank34402 +34403 POINT(29.553635316420827 -96.0963174853577) bank34403 +34404 POINT(29.199108978459904 -95.8172198540857) bank34404 +34405 POINT(29.364664205365987 -95.95003696214975) bank34405 +34406 POINT(30.59713826298443 -95.06454266451004) bank34406 +34407 POINT(28.850457005355853 -94.97858230468168) bank34407 +34408 POINT(30.334277700970873 -96.18008249371753) bank34408 +34409 POINT(28.884070543427804 -94.65203787031085) bank34409 +34410 POINT(29.105603480494015 -96.33065949101608) bank34410 +34411 POINT(30.546875939851184 -95.85269048409147) bank34411 +34412 POINT(29.378270922420928 -96.03547464227921) bank34412 +34413 POINT(30.695373597245563 -95.22486078875558) bank34413 +34414 POINT(29.26333057072408 -96.32227378926298) bank34414 +34415 POINT(30.008976623624136 -95.83855743853755) bank34415 +34416 POINT(28.875890582252552 -95.36228780180257) bank34416 +34417 POINT(30.320152159516976 -96.02898005437179) bank34417 +34418 POINT(28.890925547970898 -94.45407534787147) bank34418 +34419 POINT(30.27912305620651 -95.0605433108055) bank34419 +34420 POINT(29.94653114919318 -94.61803451992776) bank34420 +34421 POINT(28.9189279045622 -95.09124635720597) bank34421 +34422 POINT(29.85374788918327 -95.09961351493688) bank34422 +34423 POINT(29.73370711267044 -96.2504321848299) bank34423 +34424 POINT(29.833771944927893 -95.99250258455679) bank34424 +34425 POINT(30.324610612638278 -96.18845494840069) bank34425 +34426 POINT(30.046259048665746 -95.54326422397264) bank34426 +34427 POINT(29.195971441531395 -95.61031407157255) bank34427 +34428 POINT(29.35520846756757 -94.47394689941612) bank34428 +34429 POINT(30.521701233627454 -94.76008252230275) bank34429 +34430 POINT(30.555253111749874 -96.20152839590584) bank34430 +34431 POINT(30.281768839203444 -95.27279208981317) bank34431 +34432 POINT(28.95295069650884 -95.81777823901308) bank34432 +34433 POINT(28.90694449317437 -95.92384117661) bank34433 +34434 POINT(30.75901227179793 -95.34235137514304) bank34434 +34435 POINT(29.31981961640194 -96.22757506656602) bank34435 +34436 POINT(30.20567883038349 -94.72187130740224) bank34436 +34437 POINT(29.2521479033821 -95.71937751170337) bank34437 +34438 POINT(29.211439595659336 -95.05979321166643) bank34438 +34439 POINT(29.16161815492693 -95.58371686905822) bank34439 +34440 POINT(28.856691606228882 -94.84186209330485) bank34440 +34441 POINT(30.10671230508374 -96.25053806975168) bank34441 +34442 POINT(29.76648930862981 -94.40357496343631) bank34442 +34443 POINT(30.370792175897126 -95.69275043142518) bank34443 +34444 POINT(29.177127456951233 -95.25242624355695) bank34444 +34445 POINT(29.41075839461717 -95.01923167915363) bank34445 +34446 POINT(29.984679445787346 -95.9554469636045) bank34446 +34447 POINT(29.375875153792624 -94.48716348623515) bank34447 +34448 POINT(30.228355793600166 -94.65882710353685) bank34448 +34449 POINT(30.48967089840544 -95.75035741133019) bank34449 +34450 POINT(30.569832004260082 -95.10641496209416) bank34450 +34451 POINT(30.495856962003476 -96.02517441854027) bank34451 +34452 POINT(29.353544762211296 -96.06067852248805) bank34452 +34453 POINT(29.978296737115304 -95.03282989938957) bank34453 +34454 POINT(29.226094879730578 -95.2681206972608) bank34454 +34455 POINT(30.1639318503749 -95.37488542975095) bank34455 +34456 POINT(30.39714953712933 -95.74234796676161) bank34456 +34457 POINT(29.223631998639917 -95.68496969913174) bank34457 +34458 POINT(29.90493785490654 -95.86201017886738) bank34458 +34459 POINT(30.503564030854967 -95.28998550566773) bank34459 +34460 POINT(30.276809381947558 -95.25528760857928) bank34460 +34461 POINT(29.623279914423872 -94.42629252705308) bank34461 +34462 POINT(29.25080989797621 -95.39658411355238) bank34462 +34463 POINT(29.815248094583115 -96.32250119182254) bank34463 +34464 POINT(28.76110189443062 -95.41127618692171) bank34464 +34465 POINT(30.073566229991386 -95.88551823442921) bank34465 +34466 POINT(30.270021203685275 -96.13638611866229) bank34466 +34467 POINT(30.5915241033101 -95.2400267538909) bank34467 +34468 POINT(29.475859232639063 -94.44673060508849) bank34468 +34469 POINT(30.095305107325242 -95.91748895009594) bank34469 +34470 POINT(30.202624499776928 -95.17155538373335) bank34470 +34471 POINT(30.172416280954067 -95.54359456765633) bank34471 +34472 POINT(28.940452006594743 -95.8722356252627) bank34472 +34473 POINT(29.200370837702504 -95.53032993618173) bank34473 +34474 POINT(28.820217243378693 -94.88511968664616) bank34474 +34475 POINT(28.88829644393692 -95.22450436419402) bank34475 +34476 POINT(30.284823412164336 -96.04772495186457) bank34476 +34477 POINT(29.300959477279157 -96.1670879432507) bank34477 +34478 POINT(29.938856326250896 -94.51151252869512) bank34478 +34479 POINT(29.860440685669353 -95.48352970242858) bank34479 +34480 POINT(29.837616993551688 -96.015598587856) bank34480 +34481 POINT(30.168726420165267 -96.21651695742695) bank34481 +34482 POINT(30.074493406667806 -94.9929807334097) bank34482 +34483 POINT(30.17721038731048 -94.70637178528189) bank34483 +34484 POINT(29.370496199080506 -94.48230868984838) bank34484 +34485 POINT(30.541823076315957 -95.06124750622745) bank34485 +34486 POINT(29.769418428853637 -95.48546696575589) bank34486 +34487 POINT(29.971843977047364 -95.09118359499976) bank34487 +34488 POINT(29.12068342973181 -94.46966205115568) bank34488 +34489 POINT(30.438734417370814 -95.82639660157731) bank34489 +34490 POINT(30.414051359060338 -94.93472100205369) bank34490 +34491 POINT(30.735520106517605 -96.04427408036621) bank34491 +34492 POINT(28.83071416727462 -95.83278971760807) bank34492 +34493 POINT(29.257460450619874 -95.90284123673902) bank34493 +34494 POINT(29.44530219503784 -95.93665969697255) bank34494 +34495 POINT(30.317780580827097 -94.90955455083872) bank34495 +34496 POINT(30.068746945562147 -95.99349240963949) bank34496 +34497 POINT(29.783141754654235 -95.91092817945551) bank34497 +34498 POINT(29.51124136542128 -95.79238070500875) bank34498 +34499 POINT(29.05748769250453 -95.27117855371078) bank34499 +34500 POINT(29.353097753569244 -95.64309379485407) bank34500 +34501 POINT(30.68324506802628 -95.57541629207913) bank34501 +34502 POINT(30.59704024305869 -94.70587662615982) bank34502 +34503 POINT(29.796010651261817 -94.8214845375264) bank34503 +34504 POINT(29.178389173470915 -96.0548977938022) bank34504 +34505 POINT(29.7572878255785 -95.17811172567443) bank34505 +34506 POINT(29.518992911858188 -95.81355839389502) bank34506 +34507 POINT(29.861639304282107 -95.94366161655523) bank34507 +34508 POINT(29.843754725733774 -95.9619486548883) bank34508 +34509 POINT(30.03786246089579 -95.5594912325761) bank34509 +34510 POINT(29.456897335848517 -95.87984509572208) bank34510 +34511 POINT(29.22999872870293 -95.66336004105729) bank34511 +34512 POINT(28.878369610969123 -94.38283270664046) bank34512 +34513 POINT(29.521183454919907 -94.4626371414825) bank34513 +34514 POINT(30.041282199460294 -96.13418954609443) bank34514 +34515 POINT(28.807455038418432 -95.0217129801723) bank34515 +34516 POINT(28.978749496767023 -95.4795002002227) bank34516 +34517 POINT(29.537762687412545 -95.92327498479001) bank34517 +34518 POINT(30.398063041747054 -95.12453809341106) bank34518 +34519 POINT(30.19790397303145 -96.29167132555443) bank34519 +34520 POINT(29.746802487016495 -95.3672026264143) bank34520 +34521 POINT(30.418944504805996 -96.28484356981461) bank34521 +34522 POINT(30.576294591106898 -96.03971633739104) bank34522 +34523 POINT(30.024882644068327 -95.74079062238643) bank34523 +34524 POINT(29.243309726234727 -95.49225572598773) bank34524 +34525 POINT(29.273604898477295 -94.84540692494026) bank34525 +34526 POINT(29.797816889468574 -94.89737851136354) bank34526 +34527 POINT(30.126797284290003 -94.66074251846777) bank34527 +34528 POINT(30.388486566972205 -94.37312261797418) bank34528 +34529 POINT(30.23723695094751 -95.58178648272064) bank34529 +34530 POINT(29.345349296783972 -96.27552366044203) bank34530 +34531 POINT(30.480274979082836 -95.93373368698154) bank34531 +34532 POINT(30.463980099739025 -96.22825664906088) bank34532 +34533 POINT(30.232732077005792 -95.92846311557497) bank34533 +34534 POINT(28.794544393128337 -96.21531469825061) bank34534 +34535 POINT(29.507840986426753 -94.7363254217337) bank34535 +34536 POINT(30.381093212359772 -95.12442239198516) bank34536 +34537 POINT(30.075179999146425 -94.98191788478877) bank34537 +34538 POINT(28.81875861157836 -95.00644038748727) bank34538 +34539 POINT(29.2568118460262 -96.12762195210529) bank34539 +34540 POINT(29.444960172185574 -94.82247766286116) bank34540 +34541 POINT(30.294929855197942 -95.43879165048021) bank34541 +34542 POINT(29.575241415152618 -95.94174557572926) bank34542 +34543 POINT(30.177777713099406 -94.81050580373665) bank34543 +34544 POINT(29.520397439394813 -94.95542364937047) bank34544 +34545 POINT(29.479476134936206 -95.95858199758744) bank34545 +34546 POINT(30.104034498276178 -95.76228416591815) bank34546 +34547 POINT(29.187236632826938 -94.42193762200615) bank34547 +34548 POINT(29.245711773474937 -94.88263804259036) bank34548 +34549 POINT(29.60856714478124 -94.45494109413806) bank34549 +34550 POINT(29.075781097305754 -96.31225438441464) bank34550 +34551 POINT(28.85272581236665 -95.90318285390494) bank34551 +34552 POINT(28.864337492387794 -95.45975454696682) bank34552 +34553 POINT(29.489598609550626 -96.05900119138308) bank34553 +34554 POINT(29.065179293647276 -95.52471608018482) bank34554 +34555 POINT(30.348463479869597 -95.54793146363606) bank34555 +34556 POINT(30.472714861580396 -94.92274797031313) bank34556 +34557 POINT(29.147317286541753 -94.75300285613291) bank34557 +34558 POINT(29.355429380205006 -94.58696558534858) bank34558 +34559 POINT(30.134785759925958 -94.42245745630538) bank34559 +34560 POINT(29.81386095469872 -94.81377167702894) bank34560 +34561 POINT(29.54903846867328 -95.45988771943667) bank34561 +34562 POINT(30.70186690062372 -96.16676195923607) bank34562 +34563 POINT(30.229241495320263 -95.88545788005868) bank34563 +34564 POINT(29.016247411486287 -96.12706407855983) bank34564 +34565 POINT(28.957172099094286 -96.24722403216231) bank34565 +34566 POINT(28.8658287176389 -95.23499164190618) bank34566 +34567 POINT(30.518344261868688 -95.68430489385715) bank34567 +34568 POINT(29.305611058790543 -95.61933510502624) bank34568 +34569 POINT(30.52640677515496 -95.29955802391987) bank34569 +34570 POINT(29.072825065119176 -95.2670326722124) bank34570 +34571 POINT(28.89508733921763 -95.05125086464737) bank34571 +34572 POINT(28.831418943584488 -95.5274810402476) bank34572 +34573 POINT(30.34397942495908 -94.7307671429339) bank34573 +34574 POINT(29.14329962557346 -94.87054814938422) bank34574 +34575 POINT(29.460875824351874 -94.58793907837568) bank34575 +34576 POINT(29.162413005400637 -95.69146716062913) bank34576 +34577 POINT(30.240771544389013 -95.52943157865478) bank34577 +34578 POINT(28.788469622118566 -96.33346618775603) bank34578 +34579 POINT(29.481347073635515 -95.94867660207726) bank34579 +34580 POINT(28.923839479213537 -95.67718548769149) bank34580 +34581 POINT(29.857679679050197 -95.13081536352873) bank34581 +34582 POINT(30.177407328636217 -95.80506553933569) bank34582 +34583 POINT(29.24984035317585 -94.50389094520841) bank34583 +34584 POINT(29.49161411938085 -96.15138087531156) bank34584 +34585 POINT(29.46117805313012 -95.17768661542605) bank34585 +34586 POINT(30.616106728157625 -95.30116263785673) bank34586 +34587 POINT(28.783928393913765 -94.75215755796769) bank34587 +34588 POINT(29.014893546979312 -95.73432876366364) bank34588 +34589 POINT(29.697113178537986 -94.70766366552718) bank34589 +34590 POINT(29.561527198557318 -94.48417662536272) bank34590 +34591 POINT(29.423675360346756 -95.75177703230094) bank34591 +34592 POINT(29.726653021376226 -94.85906374678088) bank34592 +34593 POINT(30.06736344113092 -95.98459458930317) bank34593 +34594 POINT(29.39100004519052 -95.42917156958458) bank34594 +34595 POINT(28.96818165295525 -94.92391567213937) bank34595 +34596 POINT(29.758110968765447 -95.86024078409248) bank34596 +34597 POINT(29.45134143844915 -94.7581891705597) bank34597 +34598 POINT(29.38416877197374 -95.52459623962612) bank34598 +34599 POINT(29.255546371569867 -95.7334391188584) bank34599 +34600 POINT(29.69229133330716 -94.47406178585372) bank34600 +34601 POINT(29.581574513701767 -95.06907405237916) bank34601 +34602 POINT(30.539830678176177 -95.46022753343486) bank34602 +34603 POINT(29.01848338937586 -94.66909828258444) bank34603 +34604 POINT(29.05567666150534 -96.1016755668264) bank34604 +34605 POINT(29.77577463498031 -94.6495323905474) bank34605 +34606 POINT(28.978755754157547 -96.31633200559027) bank34606 +34607 POINT(30.296430022340644 -94.65656622795132) bank34607 +34608 POINT(30.072847821322142 -96.28765420041609) bank34608 +34609 POINT(30.418167032993534 -95.24386563779545) bank34609 +34610 POINT(28.990147501628357 -94.65266477734934) bank34610 +34611 POINT(29.159273844732798 -94.55412160217978) bank34611 +34612 POINT(29.130657252740516 -95.36358425943016) bank34612 +34613 POINT(29.636197622679543 -95.80120443058654) bank34613 +34614 POINT(29.07342527972346 -95.15553956724627) bank34614 +34615 POINT(29.091111822893907 -95.88478485520744) bank34615 +34616 POINT(28.811726716437562 -94.58681784931552) bank34616 +34617 POINT(29.29978806475695 -96.02498249208973) bank34617 +34618 POINT(30.04587617555104 -96.09945414109252) bank34618 +34619 POINT(30.401170348335572 -94.40788022434435) bank34619 +34620 POINT(29.466114578003904 -95.82175425687636) bank34620 +34621 POINT(30.086545628101053 -94.70907673085314) bank34621 +34622 POINT(28.99040546107484 -96.02177041518732) bank34622 +34623 POINT(29.600843822185006 -94.69428200167322) bank34623 +34624 POINT(29.964723188274444 -95.8805727631264) bank34624 +34625 POINT(29.586357446629826 -95.848230112686) bank34625 +34626 POINT(30.350093719906667 -95.41528056696899) bank34626 +34627 POINT(30.2830311490454 -95.19524489139091) bank34627 +34628 POINT(29.993380410721834 -94.82607207284715) bank34628 +34629 POINT(30.325711188457493 -95.22807596639767) bank34629 +34630 POINT(30.236568076157365 -94.59738973149298) bank34630 +34631 POINT(29.91928098898467 -95.30963960597731) bank34631 +34632 POINT(30.59066802891556 -95.28107979309691) bank34632 +34633 POINT(30.586051748193857 -94.96856626857506) bank34633 +34634 POINT(30.66011476946043 -94.58827766334174) bank34634 +34635 POINT(29.967003895121394 -95.11460970365641) bank34635 +34636 POINT(29.664757226981113 -95.41124385482645) bank34636 +34637 POINT(30.575634578430293 -95.40989374466848) bank34637 +34638 POINT(29.881221320738604 -94.85190241637183) bank34638 +34639 POINT(29.717699447846158 -94.90263566786334) bank34639 +34640 POINT(29.955077973730788 -94.82135460334503) bank34640 +34641 POINT(28.770913631709423 -96.12499298956968) bank34641 +34642 POINT(29.634846099238754 -94.8699354385178) bank34642 +34643 POINT(30.227514790948163 -95.13845218930433) bank34643 +34644 POINT(29.81471034266645 -95.41666513768212) bank34644 +34645 POINT(29.32648270586058 -95.37032042292672) bank34645 +34646 POINT(30.393485390049374 -95.7098630673212) bank34646 +34647 POINT(30.229810850238533 -94.95455237930003) bank34647 +34648 POINT(30.077645886026577 -95.8380417514894) bank34648 +34649 POINT(30.021263341788597 -96.27026541881504) bank34649 +34650 POINT(29.998176268831507 -95.3950440978676) bank34650 +34651 POINT(30.12607298791758 -95.9620436941033) bank34651 +34652 POINT(29.004164533345673 -94.80347109511021) bank34652 +34653 POINT(29.460236259285054 -95.58777238909146) bank34653 +34654 POINT(29.621297505138354 -95.0696863526365) bank34654 +34655 POINT(29.951377939964825 -95.05173175720391) bank34655 +34656 POINT(30.06414623787502 -94.41053797406548) bank34656 +34657 POINT(30.563556673035638 -95.3013235997099) bank34657 +34658 POINT(30.480840480096564 -95.63469657278168) bank34658 +34659 POINT(30.362731157020793 -95.3435041401451) bank34659 +34660 POINT(29.472697826332702 -95.50344959870876) bank34660 +34661 POINT(29.780949277536877 -94.89740767312975) bank34661 +34662 POINT(30.389538272009478 -95.24161927444207) bank34662 +34663 POINT(28.76743345446802 -95.7514238575014) bank34663 +34664 POINT(30.237356708240327 -95.33395481315719) bank34664 +34665 POINT(30.190717858176193 -96.19904257238835) bank34665 +34666 POINT(29.187740449628663 -94.90321484836838) bank34666 +34667 POINT(29.507873810486366 -95.33524685040523) bank34667 +34668 POINT(30.754692175920145 -96.1632456369185) bank34668 +34669 POINT(29.08229552493846 -94.81390137338329) bank34669 +34670 POINT(30.26575325527436 -94.73825284263573) bank34670 +34671 POINT(28.95976909405993 -96.14574065282427) bank34671 +34672 POINT(29.30274090384412 -96.26145880083185) bank34672 +34673 POINT(29.520388549035513 -94.98911195622595) bank34673 +34674 POINT(28.982163647393268 -95.88789613663566) bank34674 +34675 POINT(30.60427972839107 -94.95673273121731) bank34675 +34676 POINT(29.244662733313163 -94.88478049798157) bank34676 +34677 POINT(29.31863587487005 -96.09414406068491) bank34677 +34678 POINT(28.968820254188557 -94.60267494830194) bank34678 +34679 POINT(30.465500126118364 -94.62582218285625) bank34679 +34680 POINT(28.764781396777067 -94.66054834025364) bank34680 +34681 POINT(30.745514546934295 -95.04665101737586) bank34681 +34682 POINT(29.89667039071081 -94.55059974605736) bank34682 +34683 POINT(28.906772122001424 -95.571125364003) bank34683 +34684 POINT(30.220939369149576 -94.5451128550828) bank34684 +34685 POINT(30.11114440378769 -96.00930174839056) bank34685 +34686 POINT(30.569918509733533 -94.55753954667028) bank34686 +34687 POINT(30.365969369703738 -95.83025056442635) bank34687 +34688 POINT(29.279064776242087 -95.15512955808441) bank34688 +34689 POINT(30.755719981612256 -94.45639959162392) bank34689 +34690 POINT(29.846445047876628 -95.90657424383387) bank34690 +34691 POINT(30.45502918154039 -95.98814771535045) bank34691 +34692 POINT(29.809807756420764 -95.25824163280639) bank34692 +34693 POINT(30.66277156292307 -96.03071043305852) bank34693 +34694 POINT(30.320595830156442 -95.18319632741415) bank34694 +34695 POINT(28.95907774548908 -94.66908141812283) bank34695 +34696 POINT(30.583938467343263 -95.4686350874087) bank34696 +34697 POINT(29.392095219553788 -95.37826178545629) bank34697 +34698 POINT(30.379079715055656 -94.8478218199596) bank34698 +34699 POINT(30.614945659692832 -95.99840777623443) bank34699 +34700 POINT(29.949490525115376 -94.50592424204457) bank34700 +34701 POINT(29.066070850415546 -94.52079467270833) bank34701 +34702 POINT(29.90424261749051 -95.06589646688158) bank34702 +34703 POINT(28.877769733941204 -96.34893544453294) bank34703 +34704 POINT(29.045194348879576 -94.40946055454907) bank34704 +34705 POINT(29.68830358430536 -94.53905098297199) bank34705 +34706 POINT(30.321759026957302 -95.69205208191602) bank34706 +34707 POINT(30.167423615360644 -95.7567741052824) bank34707 +34708 POINT(30.077879871325482 -94.70539586117336) bank34708 +34709 POINT(28.820070791942022 -94.62352298144) bank34709 +34710 POINT(30.003275530754944 -95.04567928470672) bank34710 +34711 POINT(29.169083756767815 -95.85826742778244) bank34711 +34712 POINT(30.44819130938877 -95.45244006128111) bank34712 +34713 POINT(29.847560580983824 -94.5076990410909) bank34713 +34714 POINT(30.152435444596556 -94.58487492047286) bank34714 +34715 POINT(29.54600845682609 -94.375450902864) bank34715 +34716 POINT(28.96805381650641 -95.8806690303061) bank34716 +34717 POINT(29.555941859366822 -94.61047487869197) bank34717 +34718 POINT(30.475020177472683 -95.31359663407136) bank34718 +34719 POINT(30.52770036255281 -94.80795583540848) bank34719 +34720 POINT(29.744781129230905 -96.30842832578136) bank34720 +34721 POINT(30.55012917128153 -95.9139423286199) bank34721 +34722 POINT(29.59250382758962 -95.97889702655333) bank34722 +34723 POINT(29.132550101285144 -94.59293863632911) bank34723 +34724 POINT(30.049813485327064 -94.78458146612928) bank34724 +34725 POINT(28.924097216744073 -95.39315121813785) bank34725 +34726 POINT(29.332384214625804 -95.19024079210332) bank34726 +34727 POINT(29.560700904658173 -94.90500081420367) bank34727 +34728 POINT(29.91553511124664 -95.19493344017691) bank34728 +34729 POINT(29.16246502951941 -94.96595632125543) bank34729 +34730 POINT(29.299484096492968 -95.61978214948829) bank34730 +34731 POINT(30.421829237711137 -96.30367232705308) bank34731 +34732 POINT(30.166769869175067 -95.86856858188658) bank34732 +34733 POINT(28.872375054426442 -95.05549003652507) bank34733 +34734 POINT(29.457927639404446 -95.44037720695226) bank34734 +34735 POINT(29.765178341071067 -96.04884548839334) bank34735 +34736 POINT(29.69142779999961 -95.60211523378571) bank34736 +34737 POINT(29.30583853389442 -95.05050615703438) bank34737 +34738 POINT(29.755917344633136 -96.03947526870022) bank34738 +34739 POINT(29.350249934166875 -95.72517520593384) bank34739 +34740 POINT(30.00059090250241 -95.47624710916891) bank34740 +34741 POINT(30.17024377484378 -95.55742856123074) bank34741 +34742 POINT(30.489366583172703 -95.2105404135448) bank34742 +34743 POINT(29.369646856049155 -94.6541417227041) bank34743 +34744 POINT(30.18011998418083 -95.53970237951894) bank34744 +34745 POINT(29.41513007157344 -95.95565478627024) bank34745 +34746 POINT(29.14732852904818 -95.7486451853163) bank34746 +34747 POINT(30.689766725931467 -94.50643601961339) bank34747 +34748 POINT(30.02707614768758 -95.67893103317432) bank34748 +34749 POINT(30.05672839612329 -96.3545561872107) bank34749 +34750 POINT(29.142096159250947 -95.6568348762014) bank34750 +34751 POINT(29.620154493657676 -95.35786604347796) bank34751 +34752 POINT(30.577765570536492 -94.58142145004949) bank34752 +34753 POINT(30.262091103292793 -94.41840345079198) bank34753 +34754 POINT(30.132139171461624 -94.89881060176397) bank34754 +34755 POINT(30.04002846144731 -95.51481709363347) bank34755 +34756 POINT(28.805327144712756 -95.72541373879461) bank34756 +34757 POINT(30.568706468755465 -95.82031350801144) bank34757 +34758 POINT(29.73674582464959 -94.79890134608836) bank34758 +34759 POINT(29.011472813597628 -95.34655106176649) bank34759 +34760 POINT(28.78276964658288 -94.39422661459933) bank34760 +34761 POINT(29.404994414007135 -94.80864920806698) bank34761 +34762 POINT(29.57943329912612 -94.70952258427388) bank34762 +34763 POINT(28.943769645523304 -95.77753405733988) bank34763 +34764 POINT(30.58434180171885 -95.59856906176766) bank34764 +34765 POINT(29.56818252971211 -94.81439897917559) bank34765 +34766 POINT(29.197256750025645 -94.95003183412075) bank34766 +34767 POINT(28.947350035701525 -95.25492908541575) bank34767 +34768 POINT(29.1540658353306 -94.49822639829944) bank34768 +34769 POINT(29.355033513661915 -94.44966562923484) bank34769 +34770 POINT(29.86032327693953 -94.38945440507301) bank34770 +34771 POINT(29.111371470531047 -95.4505580792382) bank34771 +34772 POINT(29.933500251614852 -96.12147111319705) bank34772 +34773 POINT(29.095092527153476 -95.83327059390032) bank34773 +34774 POINT(30.214026207783014 -96.00784882871761) bank34774 +34775 POINT(29.511553869359087 -95.36179684750662) bank34775 +34776 POINT(29.191587035079333 -95.71283052185514) bank34776 +34777 POINT(30.48554309835202 -95.67798174691211) bank34777 +34778 POINT(29.697946160042683 -95.96643910478777) bank34778 +34779 POINT(30.371792748579786 -95.98155365291717) bank34779 +34780 POINT(30.648669936608737 -94.85585112161509) bank34780 +34781 POINT(28.936083828702305 -95.71031009856989) bank34781 +34782 POINT(30.370649963445693 -95.59078151031362) bank34782 +34783 POINT(30.67812390409398 -95.25756938473062) bank34783 +34784 POINT(29.640731595836975 -95.60597396033064) bank34784 +34785 POINT(30.04478221547492 -95.93051672576706) bank34785 +34786 POINT(29.603183831031462 -95.7774415009495) bank34786 +34787 POINT(29.646784818530183 -95.12435755375236) bank34787 +34788 POINT(29.089181010945026 -94.4775343786243) bank34788 +34789 POINT(29.375972662677214 -96.24165769308576) bank34789 +34790 POINT(29.47239781787763 -96.05396996466611) bank34790 +34791 POINT(29.849407780389964 -94.9913230059025) bank34791 +34792 POINT(29.748201485805282 -95.24889266774966) bank34792 +34793 POINT(30.542759040954863 -95.51785429133544) bank34793 +34794 POINT(29.646374677640644 -95.86812538859759) bank34794 +34795 POINT(29.524484301459253 -95.72928830550916) bank34795 +34796 POINT(30.091740621758177 -94.46404313655263) bank34796 +34797 POINT(29.397425750777554 -96.02747724370127) bank34797 +34798 POINT(29.825513769450563 -95.6701166391193) bank34798 +34799 POINT(29.073662549478275 -94.84340462444469) bank34799 +34800 POINT(29.547902799843296 -95.76621220690703) bank34800 +34801 POINT(29.37068788472359 -96.2558269930312) bank34801 +34802 POINT(29.485215648096517 -96.35766294502064) bank34802 +34803 POINT(29.181207576452877 -95.95486922448278) bank34803 +34804 POINT(30.629576422183696 -95.67644628793632) bank34804 +34805 POINT(29.418745005133875 -94.53756725794703) bank34805 +34806 POINT(29.63014861826113 -95.87416134325777) bank34806 +34807 POINT(29.537874616329486 -95.78993144775221) bank34807 +34808 POINT(29.98448302100443 -94.54023458700833) bank34808 +34809 POINT(29.88355140609004 -95.57537767663544) bank34809 +34810 POINT(30.604313986604872 -94.45643298195203) bank34810 +34811 POINT(28.98704880785694 -94.98884790000477) bank34811 +34812 POINT(29.219733248591776 -96.06501950920405) bank34812 +34813 POINT(30.620423309773244 -95.42890776380196) bank34813 +34814 POINT(29.06650815083948 -95.20930974845864) bank34814 +34815 POINT(29.101305330601235 -95.85452610175757) bank34815 +34816 POINT(29.050309741705227 -94.77842879957265) bank34816 +34817 POINT(28.83422753539001 -94.92969959467845) bank34817 +34818 POINT(30.605287519418205 -96.19657429196377) bank34818 +34819 POINT(30.54124752035269 -96.09441040584942) bank34819 +34820 POINT(30.241344428602243 -94.5242193766876) bank34820 +34821 POINT(29.168174759957445 -95.38020802193556) bank34821 +34822 POINT(30.277813079129878 -95.5171714267988) bank34822 +34823 POINT(30.257895448212835 -95.90619109795963) bank34823 +34824 POINT(29.076867715637235 -94.63311038153128) bank34824 +34825 POINT(29.790794714233282 -95.82577579258546) bank34825 +34826 POINT(30.57352465748583 -94.42705448499271) bank34826 +34827 POINT(30.54024064752231 -95.22608141768251) bank34827 +34828 POINT(28.811497578712128 -96.06828860181662) bank34828 +34829 POINT(29.671853448888786 -94.91863906528923) bank34829 +34830 POINT(30.25711988407775 -96.16106561342156) bank34830 +34831 POINT(29.04123506611693 -94.79237809524001) bank34831 +34832 POINT(29.41557466268699 -95.55043935168646) bank34832 +34833 POINT(28.80752293232128 -95.99076766413496) bank34833 +34834 POINT(28.945283994321887 -95.52062230794061) bank34834 +34835 POINT(28.871281210503415 -94.6798459196552) bank34835 +34836 POINT(29.86224472776509 -94.79155722526914) bank34836 +34837 POINT(29.49622727638801 -95.04010769624631) bank34837 +34838 POINT(30.654175496782734 -96.23012793717703) bank34838 +34839 POINT(29.12504024634892 -96.28993213452617) bank34839 +34840 POINT(29.75253348267986 -94.51553429889742) bank34840 +34841 POINT(29.56281770343266 -95.60339169614049) bank34841 +34842 POINT(28.86643278946991 -96.00906072430998) bank34842 +34843 POINT(30.411848517340687 -95.19750004526207) bank34843 +34844 POINT(28.869641042548004 -94.72825440846582) bank34844 +34845 POINT(29.32187654521213 -95.38937226739435) bank34845 +34846 POINT(29.98562410828972 -95.1259169464691) bank34846 +34847 POINT(30.65298969456715 -95.71567217412687) bank34847 +34848 POINT(30.06463762210217 -95.7258685082536) bank34848 +34849 POINT(28.978402962846868 -96.06771483942693) bank34849 +34850 POINT(30.548002702689985 -94.63838431057238) bank34850 +34851 POINT(29.384065614769796 -95.1734479488035) bank34851 +34852 POINT(29.024651438388943 -94.80316335567109) bank34852 +34853 POINT(29.44674101689105 -96.22364115846645) bank34853 +34854 POINT(29.1969811611565 -94.45851698670575) bank34854 +34855 POINT(28.89869783936266 -94.73880336456239) bank34855 +34856 POINT(30.253998963574492 -95.3482306118601) bank34856 +34857 POINT(29.619948259446858 -96.1367038814449) bank34857 +34858 POINT(30.387072611712984 -95.01297332304598) bank34858 +34859 POINT(29.43321624616582 -95.50531495007787) bank34859 +34860 POINT(29.006236321430027 -94.4284975082567) bank34860 +34861 POINT(29.93275327468159 -96.32222001417145) bank34861 +34862 POINT(29.708102438494098 -94.61881467515684) bank34862 +34863 POINT(29.84913560583587 -94.80032881662927) bank34863 +34864 POINT(30.589356823101543 -95.21882028851658) bank34864 +34865 POINT(30.736321755250067 -95.89897163801815) bank34865 +34866 POINT(29.98999707346229 -94.39704940003926) bank34866 +34867 POINT(29.983022695144125 -95.69966385824196) bank34867 +34868 POINT(29.34400317073118 -95.86496862765495) bank34868 +34869 POINT(30.369607152669904 -95.21817571421934) bank34869 +34870 POINT(30.03903262544163 -95.04664525344222) bank34870 +34871 POINT(29.371557700148667 -94.81782356279597) bank34871 +34872 POINT(29.654190390596796 -95.40267771485006) bank34872 +34873 POINT(29.378370796502452 -95.46224634920387) bank34873 +34874 POINT(29.407882931997428 -96.0819970264489) bank34874 +34875 POINT(30.604857311775547 -95.16143967394267) bank34875 +34876 POINT(28.779030704539185 -95.86240476281387) bank34876 +34877 POINT(29.373131358184786 -96.12402069669007) bank34877 +34878 POINT(29.393844913234386 -95.77981954686904) bank34878 +34879 POINT(29.76613429440386 -96.2733353143507) bank34879 +34880 POINT(29.130359407625114 -94.658458281044) bank34880 +34881 POINT(30.23845500224277 -95.06630478155793) bank34881 +34882 POINT(30.091521786594768 -96.13486840031699) bank34882 +34883 POINT(29.47664797777912 -95.19425289594967) bank34883 +34884 POINT(28.86035375836654 -95.38113437423603) bank34884 +34885 POINT(30.447090898087072 -95.64883214571323) bank34885 +34886 POINT(29.144383719972478 -95.35403443875524) bank34886 +34887 POINT(30.06278568760702 -94.72086943958735) bank34887 +34888 POINT(30.083220809021928 -96.22076072215627) bank34888 +34889 POINT(30.08950635669554 -96.01917444827436) bank34889 +34890 POINT(30.259476946260932 -94.60391418998843) bank34890 +34891 POINT(30.65217521731975 -94.3995892244715) bank34891 +34892 POINT(29.567268107535416 -94.46590903916061) bank34892 +34893 POINT(30.733406108424813 -95.53307180478953) bank34893 +34894 POINT(29.054351558419505 -95.49785554635075) bank34894 +34895 POINT(29.388280148483283 -95.54385508016415) bank34895 +34896 POINT(29.07691967457906 -95.010115898792) bank34896 +34897 POINT(30.73973989873866 -96.10536406513134) bank34897 +34898 POINT(29.74473114684568 -95.13562906614307) bank34898 +34899 POINT(29.485820802746893 -96.08943919325509) bank34899 +34900 POINT(30.749501281218556 -95.502755773061) bank34900 +34901 POINT(28.916649723194276 -95.91644776449236) bank34901 +34902 POINT(29.35400260294239 -94.58464451806222) bank34902 +34903 POINT(29.3222489958604 -94.63466616857481) bank34903 +34904 POINT(30.56406747930741 -95.90506747828252) bank34904 +34905 POINT(30.22699019230937 -95.62296946810709) bank34905 +34906 POINT(30.321504309252102 -95.10986258971914) bank34906 +34907 POINT(29.20367231797954 -95.28857585630399) bank34907 +34908 POINT(29.800002700624695 -94.83554345852255) bank34908 +34909 POINT(30.2882914213013 -95.21770838842218) bank34909 +34910 POINT(29.05815540202169 -94.46384444276106) bank34910 +34911 POINT(28.83320725149568 -94.94908084372047) bank34911 +34912 POINT(30.477066396951873 -96.1663787771756) bank34912 +34913 POINT(28.949273784620825 -94.93255733854238) bank34913 +34914 POINT(30.667802488249595 -95.97831896735038) bank34914 +34915 POINT(29.513758802966564 -96.05072550438844) bank34915 +34916 POINT(29.07609138546481 -94.85170782051848) bank34916 +34917 POINT(29.662600574697226 -94.44146521468505) bank34917 +34918 POINT(30.723129734758285 -96.00984427452221) bank34918 +34919 POINT(29.379944848384536 -95.35914789236634) bank34919 +34920 POINT(29.94968673286688 -95.26156826474922) bank34920 +34921 POINT(30.475937632881227 -96.07919098042031) bank34921 +34922 POINT(29.947898339006912 -96.1997847757932) bank34922 +34923 POINT(29.587569561493957 -95.25469885216128) bank34923 +34924 POINT(30.660160477842176 -95.78598102434997) bank34924 +34925 POINT(29.00509222157097 -95.72608573609422) bank34925 +34926 POINT(29.564374481389812 -96.24016919508067) bank34926 +34927 POINT(29.478800132503643 -96.07646867207563) bank34927 +34928 POINT(29.126897484851757 -95.25347285054826) bank34928 +34929 POINT(28.778010910980626 -96.23777764378376) bank34929 +34930 POINT(29.31987265667124 -94.45240485325856) bank34930 +34931 POINT(29.36501754336057 -95.91435420520628) bank34931 +34932 POINT(30.157158414747055 -94.90010239657732) bank34932 +34933 POINT(29.58398736804341 -95.69043809671139) bank34933 +34934 POINT(29.50466612218803 -95.0649701878961) bank34934 +34935 POINT(30.213656471684573 -96.14193347855729) bank34935 +34936 POINT(28.76176772469136 -95.7241262660344) bank34936 +34937 POINT(30.004397551180126 -95.71847825542135) bank34937 +34938 POINT(30.336554432749352 -96.20654681201553) bank34938 +34939 POINT(29.73876879447628 -95.21131764926317) bank34939 +34940 POINT(30.744906336594617 -95.32126932131536) bank34940 +34941 POINT(29.60106527934993 -95.47746346227908) bank34941 +34942 POINT(30.00895324578203 -95.75791386662455) bank34942 +34943 POINT(28.923921664870072 -94.45701773521155) bank34943 +34944 POINT(30.09196986110701 -95.98567422810285) bank34944 +34945 POINT(29.077545598711797 -96.10472584726445) bank34945 +34946 POINT(29.565329336489324 -94.81867236995532) bank34946 +34947 POINT(29.534781266750706 -95.74828088628826) bank34947 +34948 POINT(29.49593867231059 -95.12426067375958) bank34948 +34949 POINT(29.498752462749025 -96.24232553041307) bank34949 +34950 POINT(30.029567074134924 -95.96340872627859) bank34950 +34951 POINT(29.621963452028655 -96.17588331098274) bank34951 +34952 POINT(29.928731318598324 -94.93225925094889) bank34952 +34953 POINT(30.093067785351998 -95.68834768362775) bank34953 +34954 POINT(30.654564016180654 -95.48479675215323) bank34954 +34955 POINT(29.575985815688284 -95.37774897314033) bank34955 +34956 POINT(30.134294751528948 -95.77339121504339) bank34956 +34957 POINT(30.198277540434418 -95.59045320132401) bank34957 +34958 POINT(29.604163263666535 -95.51256024047436) bank34958 +34959 POINT(29.35810362465342 -94.57804311938591) bank34959 +34960 POINT(29.78903152460143 -95.05958079710916) bank34960 +34961 POINT(30.138252876903277 -94.6912230238622) bank34961 +34962 POINT(29.800434497565753 -96.08263452989632) bank34962 +34963 POINT(29.47752200894119 -94.81838468175717) bank34963 +34964 POINT(30.592159161374894 -95.94962254588985) bank34964 +34965 POINT(29.988687514633973 -96.21615635135728) bank34965 +34966 POINT(30.36286281871034 -95.33120162912027) bank34966 +34967 POINT(29.433383516496658 -94.58315267836181) bank34967 +34968 POINT(28.884645418161742 -95.64469187068627) bank34968 +34969 POINT(29.53038040843542 -94.94131623266908) bank34969 +34970 POINT(30.022237824621342 -95.02166829581599) bank34970 +34971 POINT(30.10496057078837 -94.79148710112995) bank34971 +34972 POINT(29.26442683777585 -95.44014284913563) bank34972 +34973 POINT(29.7734158430732 -95.99376541938385) bank34973 +34974 POINT(30.573673765995704 -94.64936523595361) bank34974 +34975 POINT(29.826107274787816 -95.11931615178945) bank34975 +34976 POINT(29.026310097323446 -95.19995263935495) bank34976 +34977 POINT(29.863052024524613 -95.99638282316656) bank34977 +34978 POINT(29.275144419703107 -95.95676941164392) bank34978 +34979 POINT(30.19753121239747 -94.97505323378562) bank34979 +34980 POINT(30.02601993094427 -94.75808911607268) bank34980 +34981 POINT(30.147118883856464 -95.28097879787337) bank34981 +34982 POINT(29.277837855361042 -94.83069033508292) bank34982 +34983 POINT(29.511581455530273 -96.32839215331626) bank34983 +34984 POINT(29.814369841273685 -94.37132390113047) bank34984 +34985 POINT(29.386349727795363 -95.10187347325936) bank34985 +34986 POINT(30.20759224770053 -95.61114392415845) bank34986 +34987 POINT(29.57160210725626 -96.01119886463654) bank34987 +34988 POINT(30.39249746566073 -95.68769878743797) bank34988 +34989 POINT(28.764598147136912 -96.28294579323479) bank34989 +34990 POINT(30.316177511572857 -94.87525758917236) bank34990 +34991 POINT(29.035165088729837 -94.80324474180298) bank34991 +34992 POINT(29.96175257323633 -95.42770669901269) bank34992 +34993 POINT(30.233542535981062 -96.29288431217822) bank34993 +34994 POINT(29.662002891726715 -95.06605464325268) bank34994 +34995 POINT(28.84232196894738 -96.16981080461888) bank34995 +34996 POINT(29.49684927625971 -95.7615051226187) bank34996 +34997 POINT(29.034031396884103 -96.04093405330792) bank34997 +34998 POINT(29.693423491015096 -95.880822906682) bank34998 +34999 POINT(30.523976786875746 -95.60866595153018) bank34999 +35000 POINT(29.06124789169859 -96.29962138081707) bank35000 +35001 POINT(29.767463579225904 -95.19685438199593) bank35001 +35002 POINT(30.20823677839771 -94.96590610329439) bank35002 +35003 POINT(29.508201619293093 -94.93088293599024) bank35003 +35004 POINT(29.89442511060961 -94.72339789861542) bank35004 +35005 POINT(30.572000896534238 -95.43609415330873) bank35005 +35006 POINT(30.573228229256724 -95.24939363978062) bank35006 +35007 POINT(28.862930485969883 -96.21692172371648) bank35007 +35008 POINT(29.715696030519016 -95.56847981372411) bank35008 +35009 POINT(29.867060614481435 -95.54810523094757) bank35009 +35010 POINT(28.83365907263992 -94.60355755239517) bank35010 +35011 POINT(29.88121267770201 -95.64352499311819) bank35011 +35012 POINT(29.734482164715022 -95.37169605440666) bank35012 +35013 POINT(29.596513418948796 -94.74058631802436) bank35013 +35014 POINT(30.723304329357035 -95.25553631504752) bank35014 +35015 POINT(30.257810371085377 -96.17899111179217) bank35015 +35016 POINT(30.753333734637398 -96.33189489593578) bank35016 +35017 POINT(28.924090901078962 -96.05494900348084) bank35017 +35018 POINT(29.32730352708143 -94.42110227235025) bank35018 +35019 POINT(28.932550080298913 -95.31119845839193) bank35019 +35020 POINT(30.70747346608039 -95.84997900505162) bank35020 +35021 POINT(29.052313371015853 -94.48028035354524) bank35021 +35022 POINT(29.836943356128486 -94.8796722174499) bank35022 +35023 POINT(30.539515904372166 -94.5694522377018) bank35023 +35024 POINT(30.542835311232828 -94.87273644553028) bank35024 +35025 POINT(30.364503086848387 -96.3163852032381) bank35025 +35026 POINT(28.898114672090884 -95.65928495704443) bank35026 +35027 POINT(30.498497719588475 -94.69758858741403) bank35027 +35028 POINT(30.045732817154494 -95.830828627166) bank35028 +35029 POINT(29.42397609459071 -94.40873095164788) bank35029 +35030 POINT(29.61786201765051 -96.18704023207401) bank35030 +35031 POINT(29.937791375341487 -95.1843298829164) bank35031 +35032 POINT(29.72792529525081 -94.78329198816844) bank35032 +35033 POINT(29.617816127274725 -95.79904671964564) bank35033 +35034 POINT(29.171208684757246 -94.6760819053078) bank35034 +35035 POINT(30.134226714879638 -94.85599049884856) bank35035 +35036 POINT(29.806319297048997 -95.41660770648156) bank35036 +35037 POINT(29.368886363789898 -94.60203365449284) bank35037 +35038 POINT(29.016272642631073 -94.57236910804137) bank35038 +35039 POINT(29.892921542775586 -96.05362306596653) bank35039 +35040 POINT(29.429925049424956 -96.06558019228278) bank35040 +35041 POINT(30.324260870964828 -94.76042019100633) bank35041 +35042 POINT(29.524071899104875 -95.95656420097671) bank35042 +35043 POINT(29.567518202509252 -95.22197171625258) bank35043 +35044 POINT(30.208404380712224 -94.77654971181775) bank35044 +35045 POINT(29.45527015662642 -95.66572364001973) bank35045 +35046 POINT(30.330073085974696 -94.80601476706136) bank35046 +35047 POINT(29.240665837558325 -95.66320726405081) bank35047 +35048 POINT(30.66532207629277 -94.98219779650823) bank35048 +35049 POINT(30.16876328386726 -94.4262155499886) bank35049 +35050 POINT(30.420397219239824 -95.5564187393138) bank35050 +35051 POINT(30.075323559857203 -95.86880876011566) bank35051 +35052 POINT(30.355591271539243 -94.46462730150625) bank35052 +35053 POINT(30.702046470522152 -96.2255171130018) bank35053 +35054 POINT(28.925122267691464 -94.43882682373321) bank35054 +35055 POINT(29.063034245513805 -95.2797408674633) bank35055 +35056 POINT(29.70084234824586 -95.55568049022642) bank35056 +35057 POINT(30.048605800858294 -94.37155917624021) bank35057 +35058 POINT(30.635474457907254 -95.8532153035598) bank35058 +35059 POINT(29.55719286863226 -95.63015419365112) bank35059 +35060 POINT(30.392628048420917 -94.78294564032088) bank35060 +35061 POINT(28.995219853299112 -95.56755453133457) bank35061 +35062 POINT(30.747789033501864 -96.21518859944271) bank35062 +35063 POINT(29.845488469886412 -94.61152939967997) bank35063 +35064 POINT(30.19923273188765 -95.05971785004164) bank35064 +35065 POINT(29.15268174833998 -95.73049372168337) bank35065 +35066 POINT(30.265452469740463 -96.32785794290825) bank35066 +35067 POINT(29.573404985332004 -95.36420863466687) bank35067 +35068 POINT(30.549880670257156 -94.7192613612286) bank35068 +35069 POINT(29.224152168171845 -95.73734553889682) bank35069 +35070 POINT(30.580557964103644 -95.65813073490953) bank35070 +35071 POINT(29.49725542174884 -96.10637381064731) bank35071 +35072 POINT(30.22648258242992 -96.09743789464667) bank35072 +35073 POINT(30.02971430332581 -95.19038832429749) bank35073 +35074 POINT(28.959879328698474 -94.72171222172905) bank35074 +35075 POINT(30.312769097206242 -95.44274864813241) bank35075 +35076 POINT(29.1915870374766 -96.13178730477861) bank35076 +35077 POINT(29.07091634858859 -95.94337774804944) bank35077 +35078 POINT(29.7041692222383 -95.12373012479748) bank35078 +35079 POINT(29.590472748994728 -95.03272110231212) bank35079 +35080 POINT(30.190270508889874 -94.93485351667971) bank35080 +35081 POINT(30.66204928821664 -95.03825781060628) bank35081 +35082 POINT(29.775957658360095 -94.74207032973244) bank35082 +35083 POINT(29.294078670177527 -94.56606968346293) bank35083 +35084 POINT(30.358165647400433 -95.99926530093386) bank35084 +35085 POINT(29.063917809205996 -94.41461197527755) bank35085 +35086 POINT(29.148195004277387 -95.89866955426552) bank35086 +35087 POINT(30.477031560815654 -94.95208832528527) bank35087 +35088 POINT(30.27247059291209 -94.38135552707332) bank35088 +35089 POINT(30.655172879585315 -95.68229446766277) bank35089 +35090 POINT(30.55579824097533 -96.2390528937796) bank35090 +35091 POINT(28.930709736131686 -94.37936120982604) bank35091 +35092 POINT(30.01504958847927 -94.38717866677446) bank35092 +35093 POINT(30.21442059696032 -95.91461980233302) bank35093 +35094 POINT(30.38950969341824 -95.6508555925381) bank35094 +35095 POINT(29.20362051357563 -96.3035365150758) bank35095 +35096 POINT(29.60127945664921 -94.97671180137627) bank35096 +35097 POINT(29.960022819045555 -95.92520394668405) bank35097 +35098 POINT(29.42819075529431 -96.28788929183894) bank35098 +35099 POINT(29.763742015929697 -95.27089893401532) bank35099 +35100 POINT(28.92157297593807 -95.27502445177497) bank35100 +35101 POINT(29.43770068016003 -95.18865051559602) bank35101 +35102 POINT(28.92122294821315 -94.85192151054241) bank35102 +35103 POINT(29.911341287033814 -95.16438068142882) bank35103 +35104 POINT(29.793438348353888 -94.37852263062716) bank35104 +35105 POINT(28.80149136018815 -94.60291581540665) bank35105 +35106 POINT(29.288198359718372 -94.99376100822063) bank35106 +35107 POINT(30.715270097804034 -94.48532854310353) bank35107 +35108 POINT(30.172229281322448 -94.83656400472005) bank35108 +35109 POINT(29.125724236662673 -96.13530178183795) bank35109 +35110 POINT(29.055460056536088 -94.80618393259188) bank35110 +35111 POINT(28.89260068118743 -94.69731863717904) bank35111 +35112 POINT(29.225847862968475 -96.1624264534051) bank35112 +35113 POINT(30.348214405282434 -96.26928810613715) bank35113 +35114 POINT(30.024201620536882 -96.13541606899653) bank35114 +35115 POINT(29.64987039567441 -94.47281588345311) bank35115 +35116 POINT(30.70208559832153 -94.42882108892452) bank35116 +35117 POINT(29.376610285628914 -94.69717389745271) bank35117 +35118 POINT(29.892290807466406 -94.825973714203) bank35118 +35119 POINT(30.332269216536144 -95.80296557644832) bank35119 +35120 POINT(30.196373177577374 -95.88687739942716) bank35120 +35121 POINT(29.603780935633917 -95.54757273767137) bank35121 +35122 POINT(28.896378164733143 -95.96300045737507) bank35122 +35123 POINT(29.272432628554014 -95.13348865104435) bank35123 +35124 POINT(29.664822606385002 -94.48343939107299) bank35124 +35125 POINT(29.15094265491677 -95.27448903396167) bank35125 +35126 POINT(29.902560629869498 -95.66686883269126) bank35126 +35127 POINT(29.3352940361045 -95.11293834923559) bank35127 +35128 POINT(30.186750650388106 -94.54434629621646) bank35128 +35129 POINT(30.29700918338738 -94.5324999216793) bank35129 +35130 POINT(29.89667334681494 -95.2950234797691) bank35130 +35131 POINT(29.145300029874736 -94.44326137380527) bank35131 +35132 POINT(29.21537799261279 -95.98408967967141) bank35132 +35133 POINT(29.59281435739895 -95.82284286419879) bank35133 +35134 POINT(28.8819015778173 -95.02350317793125) bank35134 +35135 POINT(29.316064813116714 -95.75391360097522) bank35135 +35136 POINT(30.18728553006985 -95.69314018207248) bank35136 +35137 POINT(30.550118405648213 -95.13639072622695) bank35137 +35138 POINT(29.084644247950173 -96.34283113911985) bank35138 +35139 POINT(29.667199845150183 -94.65218480236008) bank35139 +35140 POINT(30.718483150283998 -94.52697137571532) bank35140 +35141 POINT(29.304835359511284 -94.95213661496238) bank35141 +35142 POINT(29.524632899977867 -95.65312272927753) bank35142 +35143 POINT(30.300100622426793 -96.03569455432648) bank35143 +35144 POINT(29.88354460450128 -95.80714011915498) bank35144 +35145 POINT(29.903067388476437 -95.67458436665979) bank35145 +35146 POINT(29.35060179363868 -94.86256227546822) bank35146 +35147 POINT(29.40068841057265 -94.97839331005486) bank35147 +35148 POINT(29.929883971953345 -94.41895966993495) bank35148 +35149 POINT(29.640989570825376 -95.56220544591135) bank35149 +35150 POINT(29.652689026055405 -95.08461424393822) bank35150 +35151 POINT(30.600187971851767 -95.53376707117128) bank35151 +35152 POINT(29.85773774482243 -94.99819487003228) bank35152 +35153 POINT(30.231213811776644 -95.93662691469743) bank35153 +35154 POINT(29.268459820290314 -95.79905896802339) bank35154 +35155 POINT(29.39799345086596 -94.82060535909567) bank35155 +35156 POINT(30.59314100683729 -95.92157553903319) bank35156 +35157 POINT(29.25199414140681 -95.17684052663685) bank35157 +35158 POINT(28.83689778935679 -95.00917686315546) bank35158 +35159 POINT(28.865814842680244 -95.34791801087181) bank35159 +35160 POINT(29.17812294832365 -95.98662326969725) bank35160 +35161 POINT(30.368902034295363 -95.90323892431137) bank35161 +35162 POINT(29.95890708311698 -94.82375859476451) bank35162 +35163 POINT(29.294468105742432 -96.28111298682785) bank35163 +35164 POINT(30.33753890627566 -95.14210949458386) bank35164 +35165 POINT(29.1274555648298 -94.48663484124916) bank35165 +35166 POINT(29.289121987663748 -95.8297750626563) bank35166 +35167 POINT(29.42918714983621 -95.38866549099474) bank35167 +35168 POINT(29.4494458909367 -95.62581120207633) bank35168 +35169 POINT(30.292476546741767 -96.30330170126257) bank35169 +35170 POINT(30.114186646178222 -96.05976474392415) bank35170 +35171 POINT(29.658035288292474 -95.42766259697265) bank35171 +35172 POINT(29.021511007983243 -94.71783502545748) bank35172 +35173 POINT(30.034580848915923 -95.59387567983681) bank35173 +35174 POINT(29.088844115267598 -95.27891669885726) bank35174 +35175 POINT(30.477802077550784 -96.27348110704892) bank35175 +35176 POINT(29.61255133476651 -95.97713759957341) bank35176 +35177 POINT(29.22153335039266 -96.32707216390851) bank35177 +35178 POINT(30.076685767468142 -94.80687829785603) bank35178 +35179 POINT(30.236090774139637 -95.39753235972908) bank35179 +35180 POINT(30.540197793161276 -95.30155837898063) bank35180 +35181 POINT(29.657787240316367 -95.26816357843619) bank35181 +35182 POINT(30.697696337428052 -94.47205400681656) bank35182 +35183 POINT(28.923151372022215 -94.50244361392568) bank35183 +35184 POINT(30.558944675457234 -96.00184330739829) bank35184 +35185 POINT(29.26651402295048 -95.89025069316259) bank35185 +35186 POINT(28.844191200354548 -94.90108061966833) bank35186 +35187 POINT(30.362367836461797 -94.4048834558673) bank35187 +35188 POINT(29.21461478584332 -95.36592526530397) bank35188 +35189 POINT(30.33533786049183 -95.85539322072677) bank35189 +35190 POINT(29.279334548108345 -94.76901680457688) bank35190 +35191 POINT(30.26327053830221 -94.73274039108401) bank35191 +35192 POINT(30.196295437122256 -96.31742767855296) bank35192 +35193 POINT(29.136565456977095 -94.95378802466466) bank35193 +35194 POINT(29.52245136655165 -94.3796059148312) bank35194 +35195 POINT(29.71779357422378 -96.15075484442713) bank35195 +35196 POINT(30.6155502445825 -95.34014513322062) bank35196 +35197 POINT(28.842140614789923 -95.43704751092696) bank35197 +35198 POINT(28.983417936551835 -94.56300283539333) bank35198 +35199 POINT(30.04522265278796 -95.36634529396513) bank35199 +35200 POINT(30.280458866757886 -95.64877840515621) bank35200 +35201 POINT(29.935626629950523 -95.22769771880411) bank35201 +35202 POINT(30.229872034132292 -95.58396844769031) bank35202 +35203 POINT(30.723153079209126 -95.9718204966296) bank35203 +35204 POINT(30.59609620052477 -95.56936278749073) bank35204 +35205 POINT(29.43958077875234 -95.97910167954421) bank35205 +35206 POINT(30.28806309457961 -96.11994089675918) bank35206 +35207 POINT(29.8103522238312 -94.45401756509258) bank35207 +35208 POINT(28.981896898639437 -95.74938659909618) bank35208 +35209 POINT(30.129784022667884 -95.22980881257517) bank35209 +35210 POINT(28.997895678045662 -96.32130869202534) bank35210 +35211 POINT(30.307400021039513 -95.09398576281652) bank35211 +35212 POINT(29.65162827491105 -94.86641636054142) bank35212 +35213 POINT(30.423716720557962 -95.80144617587139) bank35213 +35214 POINT(29.230676990985753 -95.12453531014357) bank35214 +35215 POINT(29.113592740344874 -95.98282280445287) bank35215 +35216 POINT(29.52979228783605 -95.92079007134738) bank35216 +35217 POINT(30.12586871526995 -94.7994384523659) bank35217 +35218 POINT(29.757971626513655 -94.90449627157824) bank35218 +35219 POINT(30.158866565784237 -94.56370875669842) bank35219 +35220 POINT(29.543864654417824 -94.86986564980911) bank35220 +35221 POINT(30.394109795867127 -95.0718911216482) bank35221 +35222 POINT(29.6470939576225 -95.91926650992465) bank35222 +35223 POINT(28.992602868715238 -95.26569415043775) bank35223 +35224 POINT(29.79327140572793 -95.72576521791399) bank35224 +35225 POINT(29.95430490091673 -94.41052981361747) bank35225 +35226 POINT(30.120409913682288 -95.09384868363622) bank35226 +35227 POINT(30.09052285824873 -95.06732049789663) bank35227 +35228 POINT(29.479466797160768 -94.64229616943133) bank35228 +35229 POINT(29.845050606980035 -95.85400754591608) bank35229 +35230 POINT(30.53280525854809 -95.8324941303101) bank35230 +35231 POINT(30.55814739271368 -94.83156290772804) bank35231 +35232 POINT(29.749994645119827 -95.66973389359308) bank35232 +35233 POINT(30.402362372156215 -95.66383746061577) bank35233 +35234 POINT(29.251282693607177 -95.5209076652082) bank35234 +35235 POINT(30.129086851280768 -95.34488775917373) bank35235 +35236 POINT(29.257682597391707 -95.40400758450697) bank35236 +35237 POINT(29.57210242840373 -95.71984000845424) bank35237 +35238 POINT(30.69588366248069 -95.53676411713776) bank35238 +35239 POINT(29.908709304434026 -96.36665902990543) bank35239 +35240 POINT(30.21325556195769 -95.55817317612488) bank35240 +35241 POINT(29.761532169784054 -95.74104664795641) bank35241 +35242 POINT(30.402159693296834 -95.96798614004247) bank35242 +35243 POINT(30.549729268033296 -94.53030720173237) bank35243 +35244 POINT(30.543483452914558 -95.83779551489707) bank35244 +35245 POINT(30.580891685367593 -95.98578576663465) bank35245 +35246 POINT(29.665903695399425 -95.41111732807809) bank35246 +35247 POINT(29.3244602007714 -95.5073825142221) bank35247 +35248 POINT(29.56219742388272 -96.33936365525801) bank35248 +35249 POINT(29.265317547702608 -94.99873246779492) bank35249 +35250 POINT(28.943597894038426 -94.72105182898808) bank35250 +35251 POINT(29.998806095073295 -95.0524633929843) bank35251 +35252 POINT(29.429107949708683 -96.27805934853374) bank35252 +35253 POINT(29.564063627102836 -95.55114593543773) bank35253 +35254 POINT(29.789310753658153 -95.08893428003977) bank35254 +35255 POINT(30.144208762895133 -95.35763300437985) bank35255 +35256 POINT(29.012959193173828 -94.64344712658648) bank35256 +35257 POINT(30.502563214386637 -96.0446065609764) bank35257 +35258 POINT(29.190040674919242 -95.39811427956201) bank35258 +35259 POINT(29.831598894254974 -94.60325079024307) bank35259 +35260 POINT(29.692375744675786 -94.82706748786957) bank35260 +35261 POINT(30.11626141018777 -94.50102753021334) bank35261 +35262 POINT(28.994980581366377 -96.2359556533734) bank35262 +35263 POINT(30.39589549518088 -95.4773909312878) bank35263 +35264 POINT(29.85452265373562 -95.71974515703752) bank35264 +35265 POINT(30.339935007897342 -95.97250445976866) bank35265 +35266 POINT(30.257818570656084 -94.97116787103937) bank35266 +35267 POINT(28.87144946445756 -96.00922184466634) bank35267 +35268 POINT(29.98328184402921 -94.536921796655) bank35268 +35269 POINT(29.331704480100544 -94.75610633346571) bank35269 +35270 POINT(30.049365250874004 -95.79253843515112) bank35270 +35271 POINT(28.97555956571902 -94.67628709093576) bank35271 +35272 POINT(30.739882713073815 -96.02564064896845) bank35272 +35273 POINT(28.97484309542262 -96.20614843445878) bank35273 +35274 POINT(29.118954437681445 -96.24020130472222) bank35274 +35275 POINT(28.811203648317335 -96.35025197752744) bank35275 +35276 POINT(29.06185543674152 -95.12436652437002) bank35276 +35277 POINT(30.061274601309346 -94.60076147878685) bank35277 +35278 POINT(29.374863006674914 -96.25470507264333) bank35278 +35279 POINT(29.000765731997753 -95.33530557346106) bank35279 +35280 POINT(30.71848065275876 -96.31757291346626) bank35280 +35281 POINT(29.461591715981093 -95.5964216813967) bank35281 +35282 POINT(29.020950029505432 -95.212475762455) bank35282 +35283 POINT(28.827198184834707 -95.13019098512831) bank35283 +35284 POINT(30.19759583042776 -95.07778803005006) bank35284 +35285 POINT(30.080109290522383 -95.70086782315003) bank35285 +35286 POINT(30.050027593910478 -95.13403500235852) bank35286 +35287 POINT(29.166072703129952 -95.7712378912243) bank35287 +35288 POINT(29.88910463051161 -94.87971084686939) bank35288 +35289 POINT(30.55822470464791 -94.99431429125158) bank35289 +35290 POINT(28.93668645359542 -95.65508352952403) bank35290 +35291 POINT(29.58413238039694 -95.86910235261628) bank35291 +35292 POINT(28.936121608895174 -96.02751313452929) bank35292 +35293 POINT(29.809544868278834 -96.02166224978947) bank35293 +35294 POINT(29.750190976897812 -95.02019771983689) bank35294 +35295 POINT(29.753935998983646 -95.6979360815641) bank35295 +35296 POINT(30.4714551742942 -95.03565569588271) bank35296 +35297 POINT(29.392354399547973 -95.08745818533775) bank35297 +35298 POINT(29.26508759683534 -96.07722637555639) bank35298 +35299 POINT(29.42880748431965 -95.53529643964637) bank35299 +35300 POINT(29.321321145482898 -94.65952156883459) bank35300 +35301 POINT(28.928019671428018 -95.60650778846386) bank35301 +35302 POINT(29.30240969191792 -95.78861504015326) bank35302 +35303 POINT(29.623732009827304 -95.00293407102242) bank35303 +35304 POINT(30.225984230117028 -95.00951253881296) bank35304 +35305 POINT(30.629403192227098 -95.4957178699248) bank35305 +35306 POINT(30.534966672594425 -94.41473465152451) bank35306 +35307 POINT(30.15281704517059 -94.80567564518377) bank35307 +35308 POINT(30.321389660732912 -94.97498730535393) bank35308 +35309 POINT(29.49836812306313 -95.60729594580263) bank35309 +35310 POINT(29.734204594817374 -95.7590035766959) bank35310 +35311 POINT(30.45896204476761 -94.80051028305706) bank35311 +35312 POINT(29.07039545321843 -95.3652284699138) bank35312 +35313 POINT(29.88913070943739 -94.81692931366933) bank35313 +35314 POINT(29.788060804059235 -94.51155779713503) bank35314 +35315 POINT(29.685305368853598 -95.46742721439931) bank35315 +35316 POINT(30.587954062439 -96.27673783907301) bank35316 +35317 POINT(29.437632931012235 -95.95879322563262) bank35317 +35318 POINT(29.11526299712891 -95.55181226965101) bank35318 +35319 POINT(29.047573598394727 -95.88510257750522) bank35319 +35320 POINT(30.43185992126312 -95.57955494498766) bank35320 +35321 POINT(30.751316213373606 -95.85708546402549) bank35321 +35322 POINT(30.76010503444106 -96.1981162795656) bank35322 +35323 POINT(30.535837588790717 -95.36083287846566) bank35323 +35324 POINT(30.054718170044865 -94.43101898592941) bank35324 +35325 POINT(30.103011667865808 -94.41109885365823) bank35325 +35326 POINT(30.04467733007796 -96.16434842549397) bank35326 +35327 POINT(29.16665289851188 -95.85656239224076) bank35327 +35328 POINT(29.76217954344005 -95.08625634435022) bank35328 +35329 POINT(29.594678563862914 -95.44334809527372) bank35329 +35330 POINT(29.150760127392406 -94.73516694640179) bank35330 +35331 POINT(28.88817495034665 -95.7757347659965) bank35331 +35332 POINT(29.8009439237259 -95.2613788809672) bank35332 +35333 POINT(29.15487729646772 -95.90835534308636) bank35333 +35334 POINT(29.61723547400428 -95.98001126953302) bank35334 +35335 POINT(30.272007821089083 -95.8205584614902) bank35335 +35336 POINT(29.955652381525585 -95.94104822939383) bank35336 +35337 POINT(29.110860096941117 -95.87074882604549) bank35337 +35338 POINT(30.608744700307067 -96.22878136273499) bank35338 +35339 POINT(29.107496852510614 -94.85162879401356) bank35339 +35340 POINT(29.042190985861662 -95.47235323453411) bank35340 +35341 POINT(30.65305356892165 -95.06475145266313) bank35341 +35342 POINT(29.955678555978444 -95.66980772747112) bank35342 +35343 POINT(29.57439194544825 -94.40927424085082) bank35343 +35344 POINT(28.88583063209948 -95.93988845364487) bank35344 +35345 POINT(30.102843814175383 -95.90792225245133) bank35345 +35346 POINT(30.443211198789005 -94.46983062466713) bank35346 +35347 POINT(30.570358909996614 -94.74062857959503) bank35347 +35348 POINT(29.022548665382832 -96.17059893876313) bank35348 +35349 POINT(29.865571369614656 -94.37185645175269) bank35349 +35350 POINT(29.574251771634078 -95.71930921427344) bank35350 +35351 POINT(28.94232182793542 -95.08790878798753) bank35351 +35352 POINT(30.451506405129155 -95.26229037586464) bank35352 +35353 POINT(30.340051229545693 -94.67235258730548) bank35353 +35354 POINT(28.783413467052043 -95.83886826838562) bank35354 +35355 POINT(29.48059054791996 -95.42441642937477) bank35355 +35356 POINT(29.296894462256294 -95.26735678545867) bank35356 +35357 POINT(29.226720438491167 -94.74920749472501) bank35357 +35358 POINT(28.81212664814839 -95.83531871622455) bank35358 +35359 POINT(30.52049773160719 -94.73713511189442) bank35359 +35360 POINT(29.27667945609486 -94.85915019239252) bank35360 +35361 POINT(28.914053813102818 -94.61767805767015) bank35361 +35362 POINT(29.316703984267253 -94.64807919762366) bank35362 +35363 POINT(29.62640656355228 -95.94798429712435) bank35363 +35364 POINT(29.76693107939813 -95.79006335098133) bank35364 +35365 POINT(29.57850560782833 -94.87162644125648) bank35365 +35366 POINT(29.266776409842855 -95.73777998080168) bank35366 +35367 POINT(29.30005736109039 -95.99881357616543) bank35367 +35368 POINT(29.644259092837196 -95.75125475013489) bank35368 +35369 POINT(29.109293945656145 -95.0283727519161) bank35369 +35370 POINT(29.89341041455979 -95.81728476982771) bank35370 +35371 POINT(29.642587219231473 -94.66038281577343) bank35371 +35372 POINT(30.339924748838268 -95.86551124313175) bank35372 +35373 POINT(30.39923798538089 -94.4521061004748) bank35373 +35374 POINT(29.60140233885889 -94.90061366429843) bank35374 +35375 POINT(29.46663296701339 -95.64550084338791) bank35375 +35376 POINT(29.55491445476851 -95.77928431181421) bank35376 +35377 POINT(29.59381430433843 -95.01503844169797) bank35377 +35378 POINT(28.960754621558763 -95.42288019090772) bank35378 +35379 POINT(29.02968712535669 -96.1276971025436) bank35379 +35380 POINT(28.925542911784312 -95.22733097882097) bank35380 +35381 POINT(30.107285883585583 -95.24713145545589) bank35381 +35382 POINT(30.528393935832906 -94.82203571012548) bank35382 +35383 POINT(29.624401378747343 -95.17491496266538) bank35383 +35384 POINT(30.391632193457013 -94.66394982702343) bank35384 +35385 POINT(29.900832095490827 -95.17480739408408) bank35385 +35386 POINT(30.046522535340227 -95.18173694458453) bank35386 +35387 POINT(30.626527224146617 -95.48278091220384) bank35387 +35388 POINT(29.620116143006054 -95.69422426356616) bank35388 +35389 POINT(29.384715297541053 -95.65442191754552) bank35389 +35390 POINT(30.605211600603813 -95.95501767092719) bank35390 +35391 POINT(29.57292027550685 -95.40619213168836) bank35391 +35392 POINT(30.575032223457118 -94.8391688114769) bank35392 +35393 POINT(30.382713040208156 -95.85658818285496) bank35393 +35394 POINT(30.27798828860403 -95.24332130446388) bank35394 +35395 POINT(28.961339508637813 -96.1292962837364) bank35395 +35396 POINT(29.793398257483595 -95.96348443091141) bank35396 +35397 POINT(29.90819635875961 -96.2407187358815) bank35397 +35398 POINT(29.01806131444023 -95.15448114859667) bank35398 +35399 POINT(29.571982663209226 -94.8347367511751) bank35399 +35400 POINT(30.28817281914629 -94.98994597545345) bank35400 +35401 POINT(30.29187407434612 -95.03013761847541) bank35401 +35402 POINT(29.110941695466003 -95.9291009844423) bank35402 +35403 POINT(29.833110300492354 -95.75892063090387) bank35403 +35404 POINT(30.660435427722074 -94.87841454918986) bank35404 +35405 POINT(29.840469826911605 -95.75446700946604) bank35405 +35406 POINT(30.716207312524787 -95.2018411510357) bank35406 +35407 POINT(30.30700552545768 -96.30691014102302) bank35407 +35408 POINT(29.745891551078902 -95.53091011010571) bank35408 +35409 POINT(30.6826844743676 -96.10255771093034) bank35409 +35410 POINT(29.452906564682944 -95.37790825058671) bank35410 +35411 POINT(29.861836831571615 -96.21595901799866) bank35411 +35412 POINT(29.62585291429778 -94.39451441755618) bank35412 +35413 POINT(29.95650130434378 -96.31280338591174) bank35413 +35414 POINT(30.561007865352934 -95.67048170582558) bank35414 +35415 POINT(30.31959334116149 -95.60502537520796) bank35415 +35416 POINT(30.759140354992518 -96.34879727703637) bank35416 +35417 POINT(30.607536254260495 -95.6227670181202) bank35417 +35418 POINT(30.29924510745916 -96.02099883131294) bank35418 +35419 POINT(29.395200339769968 -95.39886777773093) bank35419 +35420 POINT(29.59494839077218 -95.03878834419108) bank35420 +35421 POINT(30.379127859575 -94.92704592951486) bank35421 +35422 POINT(29.094937823176096 -95.24680104457212) bank35422 +35423 POINT(29.826034556982144 -94.63082667723681) bank35423 +35424 POINT(30.292618468075034 -95.83997225328584) bank35424 +35425 POINT(29.23269458205732 -96.30521992707999) bank35425 +35426 POINT(29.91256390052731 -95.30678495207991) bank35426 +35427 POINT(30.43679648592083 -95.84063264071149) bank35427 +35428 POINT(30.23932810840623 -95.1571891215695) bank35428 +35429 POINT(28.999361959460238 -94.66325952972623) bank35429 +35430 POINT(29.248792686876183 -94.78804693705199) bank35430 +35431 POINT(30.636239181287763 -95.45572903722588) bank35431 +35432 POINT(30.084836430295226 -96.18101243262392) bank35432 +35433 POINT(30.18371724692624 -96.20288014470525) bank35433 +35434 POINT(30.538309241564377 -94.74167874502984) bank35434 +35435 POINT(28.909248376194576 -96.33685192743334) bank35435 +35436 POINT(28.793005925644913 -94.65554325706158) bank35436 +35437 POINT(29.416702509281457 -95.6880804411244) bank35437 +35438 POINT(29.221701150957724 -95.84524797049554) bank35438 +35439 POINT(28.95312363883023 -96.30010747994709) bank35439 +35440 POINT(29.939179657320842 -95.75069064231865) bank35440 +35441 POINT(29.480116402081695 -94.43451229068062) bank35441 +35442 POINT(30.558669907882106 -96.15997677379049) bank35442 +35443 POINT(30.018599545701253 -96.32573410671927) bank35443 +35444 POINT(30.17773035256484 -94.3778592302543) bank35444 +35445 POINT(29.7147379965352 -96.1064137681766) bank35445 +35446 POINT(29.279078563002223 -94.37732867047616) bank35446 +35447 POINT(30.436630280904975 -96.31803859307583) bank35447 +35448 POINT(29.06023375710718 -96.00110316864085) bank35448 +35449 POINT(29.512730690201355 -95.03343067615549) bank35449 +35450 POINT(30.42056712227709 -95.34755764081508) bank35450 +35451 POINT(29.277934936442556 -94.93944759288891) bank35451 +35452 POINT(30.75694270691077 -95.30422299101825) bank35452 +35453 POINT(29.25135005587709 -95.66863805680647) bank35453 +35454 POINT(29.332681954209107 -95.94293050384225) bank35454 +35455 POINT(29.129953259810254 -94.42485654535868) bank35455 +35456 POINT(29.52544715634785 -94.73361578115433) bank35456 +35457 POINT(30.07971047986482 -95.1278235335828) bank35457 +35458 POINT(29.22087310138303 -96.0037903788638) bank35458 +35459 POINT(29.072683714659362 -95.21927240915467) bank35459 +35460 POINT(29.365519544167284 -94.8792014302108) bank35460 +35461 POINT(30.30416109030704 -95.20001339267046) bank35461 +35462 POINT(28.954220632364535 -95.98839317018378) bank35462 +35463 POINT(30.275979796643167 -95.3849999510417) bank35463 +35464 POINT(29.266945534636353 -95.84967877690511) bank35464 +35465 POINT(30.425663735814727 -95.90909854555879) bank35465 +35466 POINT(30.40968251566281 -96.32356235805253) bank35466 +35467 POINT(30.43102007535458 -94.66379272866584) bank35467 +35468 POINT(29.9041220785579 -95.70761887382983) bank35468 +35469 POINT(28.83078795602061 -94.51259766220545) bank35469 +35470 POINT(29.869918113901186 -95.8788191629409) bank35470 +35471 POINT(28.886063066976472 -96.11848510479815) bank35471 +35472 POINT(29.185923984570216 -95.53237321723121) bank35472 +35473 POINT(30.29071645004724 -94.56258944489028) bank35473 +35474 POINT(29.77976650936116 -95.08641268472178) bank35474 +35475 POINT(29.324547529256527 -95.85653863838515) bank35475 +35476 POINT(29.024660125411497 -96.31134649752234) bank35476 +35477 POINT(29.288008378186998 -96.31344220184305) bank35477 +35478 POINT(30.048959940977646 -94.82736605082683) bank35478 +35479 POINT(30.263730217016384 -95.59284501305835) bank35479 +35480 POINT(29.067112630517006 -95.65778073504873) bank35480 +35481 POINT(30.491772633296115 -96.16929325321551) bank35481 +35482 POINT(28.884282627152814 -95.63968286669382) bank35482 +35483 POINT(30.454173652047565 -96.04335184579047) bank35483 +35484 POINT(30.446840720305207 -94.9521477541851) bank35484 +35485 POINT(29.106860805205407 -94.88281338069906) bank35485 +35486 POINT(29.501362441658102 -95.7731836571931) bank35486 +35487 POINT(30.71266166134915 -96.162059371985) bank35487 +35488 POINT(30.65414169330961 -94.49371220513025) bank35488 +35489 POINT(29.94436665401122 -96.27642459814477) bank35489 +35490 POINT(28.990462614279885 -95.89840085018827) bank35490 +35491 POINT(28.858978187470292 -95.76556176105748) bank35491 +35492 POINT(30.079617428948534 -96.08533859552661) bank35492 +35493 POINT(29.401328005595712 -96.26604164886936) bank35493 +35494 POINT(28.931514971184104 -94.44521466137442) bank35494 +35495 POINT(30.629727691773674 -94.75634240808563) bank35495 +35496 POINT(30.41512905466346 -94.38847274389734) bank35496 +35497 POINT(29.39008233484489 -95.57468995290127) bank35497 +35498 POINT(29.990179166712434 -95.68250591714688) bank35498 +35499 POINT(28.94157279698594 -96.20548607462383) bank35499 +35500 POINT(29.517432128227494 -94.90987002634051) bank35500 +35501 POINT(29.415974660962146 -95.971698682207) bank35501 +35502 POINT(28.99599774744136 -95.30207511354158) bank35502 +35503 POINT(30.12527519538334 -94.84019430617036) bank35503 +35504 POINT(29.946624479297174 -96.02450339106947) bank35504 +35505 POINT(30.47861748119923 -95.95765078549093) bank35505 +35506 POINT(30.21173368138759 -95.9398933805812) bank35506 +35507 POINT(29.51907754410981 -96.21971240234136) bank35507 +35508 POINT(29.30707619384985 -95.90228956596589) bank35508 +35509 POINT(29.49743055383685 -95.21958842580072) bank35509 +35510 POINT(30.35420397642114 -95.6759977443802) bank35510 +35511 POINT(30.010593779453565 -95.51320108919111) bank35511 +35512 POINT(29.66624503537602 -94.88074887891084) bank35512 +35513 POINT(29.87605156234645 -95.67125184187881) bank35513 +35514 POINT(29.782917165736855 -95.25426515096825) bank35514 +35515 POINT(30.467750732454036 -94.59939502354824) bank35515 +35516 POINT(29.203330708136985 -95.4326282868553) bank35516 +35517 POINT(30.05231553329679 -94.43434979610842) bank35517 +35518 POINT(30.007866813160494 -96.04995365195302) bank35518 +35519 POINT(30.240770354189902 -95.20918782946607) bank35519 +35520 POINT(29.639143301514256 -94.94091026543754) bank35520 +35521 POINT(29.221513305409733 -96.2962447522504) bank35521 +35522 POINT(30.66359679284826 -95.01335516807004) bank35522 +35523 POINT(30.707655733167115 -95.7674465374236) bank35523 +35524 POINT(29.685638617829703 -95.0697413141906) bank35524 +35525 POINT(28.938645095192786 -96.10377916360396) bank35525 +35526 POINT(29.598830548150268 -95.23231543821765) bank35526 +35527 POINT(29.95411843698799 -94.6502941386023) bank35527 +35528 POINT(28.989526366785192 -94.58361901936011) bank35528 +35529 POINT(28.90127091083118 -95.66465065604318) bank35529 +35530 POINT(29.53415595226798 -94.91304780824302) bank35530 +35531 POINT(30.500712549740165 -95.18517593434886) bank35531 +35532 POINT(29.048782648138786 -95.80079512420181) bank35532 +35533 POINT(30.640291786380537 -95.88926278662582) bank35533 +35534 POINT(30.538234480543252 -95.69383724946213) bank35534 +35535 POINT(28.85055081920222 -95.2744468604716) bank35535 +35536 POINT(30.216289162294974 -96.07336186964315) bank35536 +35537 POINT(29.579217162635018 -95.3267457779443) bank35537 +35538 POINT(30.320292282657373 -94.39286944036151) bank35538 +35539 POINT(29.958275360532944 -95.33004793322549) bank35539 +35540 POINT(30.230074789382115 -94.90291764855539) bank35540 +35541 POINT(30.408414561192334 -95.61271609002874) bank35541 +35542 POINT(29.44848561476318 -96.22048759818858) bank35542 +35543 POINT(29.429893302597563 -95.70458922768381) bank35543 +35544 POINT(29.04890460793458 -95.70192515263578) bank35544 +35545 POINT(29.0475382305995 -96.07833155181126) bank35545 +35546 POINT(28.9896255503139 -95.18375618034936) bank35546 +35547 POINT(29.62791504423457 -95.84815490088094) bank35547 +35548 POINT(28.85616250604912 -94.96185975969293) bank35548 +35549 POINT(29.193950641463264 -95.8723076043074) bank35549 +35550 POINT(29.81538171330232 -95.32762558484504) bank35550 +35551 POINT(28.844182260987232 -94.96704985836747) bank35551 +35552 POINT(30.64790373984323 -94.71988229693605) bank35552 +35553 POINT(29.194851184973558 -94.78594424688319) bank35553 +35554 POINT(28.800364214944274 -94.84841064592781) bank35554 +35555 POINT(29.26626846002402 -95.19236549525371) bank35555 +35556 POINT(29.741740305118153 -95.58800252833386) bank35556 +35557 POINT(29.019752836391373 -94.46168130233838) bank35557 +35558 POINT(30.174376348541188 -94.72732604824883) bank35558 +35559 POINT(29.544688011199693 -95.26106919423435) bank35559 +35560 POINT(30.2847405475085 -94.89935751955778) bank35560 +35561 POINT(29.996976087989324 -95.66319945798178) bank35561 +35562 POINT(30.581660282752726 -95.08632452589548) bank35562 +35563 POINT(29.69115619741163 -94.50682342475366) bank35563 +35564 POINT(29.24578064679216 -95.26066471824642) bank35564 +35565 POINT(28.794864128006473 -95.26875842541186) bank35565 +35566 POINT(30.416103440069655 -96.01982144752539) bank35566 +35567 POINT(29.307636450048943 -96.12780572179632) bank35567 +35568 POINT(30.08138460679493 -95.58104230019688) bank35568 +35569 POINT(29.08969890334971 -95.11411002225026) bank35569 +35570 POINT(30.52500792202576 -95.53200492220576) bank35570 +35571 POINT(30.549137084926482 -95.72163227202103) bank35571 +35572 POINT(30.55365304259167 -94.37190026918908) bank35572 +35573 POINT(29.55521912663668 -96.03014280268346) bank35573 +35574 POINT(28.93505863324986 -95.34064609088145) bank35574 +35575 POINT(30.306773801456767 -94.53338230637054) bank35575 +35576 POINT(28.876619052281875 -94.83691511265145) bank35576 +35577 POINT(30.070288327170537 -96.19563644909569) bank35577 +35578 POINT(29.129282009504383 -96.1739959003193) bank35578 +35579 POINT(30.02287074453144 -95.29045019930167) bank35579 +35580 POINT(29.205084932595682 -95.34484989926125) bank35580 +35581 POINT(30.356899526737404 -94.61615733243485) bank35581 +35582 POINT(29.03996956852463 -94.38801521778633) bank35582 +35583 POINT(29.473975211120027 -95.26269484632846) bank35583 +35584 POINT(28.81305678132697 -95.2796314016043) bank35584 +35585 POINT(30.328319731302702 -95.60276945287744) bank35585 +35586 POINT(30.257111777985436 -95.59990454290823) bank35586 +35587 POINT(29.961728037095316 -94.96235851037639) bank35587 +35588 POINT(29.73159794111422 -96.34857680340926) bank35588 +35589 POINT(29.546918977099804 -94.86603107691165) bank35589 +35590 POINT(29.398954438868593 -96.09598306071317) bank35590 +35591 POINT(28.769811939171912 -94.4698966207586) bank35591 +35592 POINT(30.470168551599446 -96.36670839580515) bank35592 +35593 POINT(30.592116800328128 -95.98530253081233) bank35593 +35594 POINT(29.01234219228387 -94.59818130009032) bank35594 +35595 POINT(29.712031854991658 -95.91471584902132) bank35595 +35596 POINT(28.9967591348194 -95.26575539660996) bank35596 +35597 POINT(30.32338085968065 -95.17857833481945) bank35597 +35598 POINT(29.746010225695446 -95.3805320229666) bank35598 +35599 POINT(29.83350449064798 -96.10009019919846) bank35599 +35600 POINT(30.122769315167403 -95.21816141501768) bank35600 +35601 POINT(29.238414181905096 -94.75753026120508) bank35601 +35602 POINT(30.33934306036675 -95.39982753738923) bank35602 +35603 POINT(29.692173329064715 -95.30372150860704) bank35603 +35604 POINT(30.75096085312326 -96.22587364231357) bank35604 +35605 POINT(28.808956036681128 -95.5374184869437) bank35605 +35606 POINT(28.961764363238267 -94.50628145234043) bank35606 +35607 POINT(29.165408935111955 -94.52912412869472) bank35607 +35608 POINT(30.598072479304413 -95.735212769844) bank35608 +35609 POINT(29.3509949946867 -94.58059166109977) bank35609 +35610 POINT(30.29924823308947 -95.86707771326698) bank35610 +35611 POINT(29.040275024871477 -94.78349805129355) bank35611 +35612 POINT(30.011923398507953 -95.08023415666511) bank35612 +35613 POINT(29.29650255880482 -96.00182529227101) bank35613 +35614 POINT(29.22127710947275 -95.76992582228915) bank35614 +35615 POINT(30.012547478619396 -96.01184297481947) bank35615 +35616 POINT(29.605843465182865 -95.86599676838934) bank35616 +35617 POINT(29.33305019456224 -96.17385184726056) bank35617 +35618 POINT(30.74684716683502 -96.12292560810049) bank35618 +35619 POINT(30.500363862621327 -95.39901857481253) bank35619 +35620 POINT(30.1449450942044 -94.6333782590332) bank35620 +35621 POINT(29.14451469387045 -95.37793707872277) bank35621 +35622 POINT(30.229665012308722 -94.53722056328557) bank35622 +35623 POINT(30.72865258546189 -95.1435146448775) bank35623 +35624 POINT(30.574230451392378 -96.08055610966727) bank35624 +35625 POINT(30.694776725089564 -94.93923478358145) bank35625 +35626 POINT(29.12692119475242 -94.99237694655898) bank35626 +35627 POINT(29.676444067952648 -96.30131273795811) bank35627 +35628 POINT(30.29242987179863 -95.00830329425261) bank35628 +35629 POINT(28.930579553339467 -95.05840333172715) bank35629 +35630 POINT(29.673634064122673 -95.19481299822257) bank35630 +35631 POINT(30.309530773511263 -95.65469441474268) bank35631 +35632 POINT(30.11720215536306 -94.56734697053086) bank35632 +35633 POINT(29.676839975143945 -95.70646593257335) bank35633 +35634 POINT(30.48206301519856 -95.2821503616321) bank35634 +35635 POINT(28.79605679222548 -96.1765747361433) bank35635 +35636 POINT(30.21857806646993 -94.41817666112148) bank35636 +35637 POINT(28.961641301576897 -95.96754135214779) bank35637 +35638 POINT(30.366741963189604 -94.70276622997466) bank35638 +35639 POINT(30.71828472477489 -95.00585246676621) bank35639 +35640 POINT(29.868367066711635 -96.11934868046686) bank35640 +35641 POINT(30.459037049910414 -94.71267442276914) bank35641 +35642 POINT(29.470224383422828 -96.29009871265546) bank35642 +35643 POINT(30.072524082881245 -95.44889076226042) bank35643 +35644 POINT(30.757237815839304 -96.35603857928663) bank35644 +35645 POINT(29.534547274943275 -95.63607276074532) bank35645 +35646 POINT(29.744882147854444 -95.37869735260944) bank35646 +35647 POINT(28.87501654697853 -95.33855415668684) bank35647 +35648 POINT(29.44648055246878 -95.37380792822213) bank35648 +35649 POINT(29.971503079411498 -94.81808849764514) bank35649 +35650 POINT(30.21105901356897 -95.61471925761786) bank35650 +35651 POINT(30.563203151403762 -95.72084231585717) bank35651 +35652 POINT(30.41418204505875 -95.04413952739398) bank35652 +35653 POINT(30.696112461958702 -94.87796525491372) bank35653 +35654 POINT(29.153840218993185 -95.48268012652875) bank35654 +35655 POINT(30.31271215731749 -95.57146155604121) bank35655 +35656 POINT(29.823742066736944 -96.36742006100334) bank35656 +35657 POINT(30.417419068702806 -96.20949244177082) bank35657 +35658 POINT(29.710455275253548 -96.09247064503066) bank35658 +35659 POINT(30.39455652838032 -95.33002854490053) bank35659 +35660 POINT(30.041368564646138 -94.60168778435992) bank35660 +35661 POINT(29.158652940288547 -95.39266653659465) bank35661 +35662 POINT(29.75563276801808 -95.2341191400196) bank35662 +35663 POINT(29.41185328238982 -96.30652668234086) bank35663 +35664 POINT(30.021259382843528 -95.13089870020313) bank35664 +35665 POINT(30.67967148401786 -95.4610226892972) bank35665 +35666 POINT(29.7611989966755 -96.208026143372) bank35666 +35667 POINT(30.372920196754027 -95.92990612060443) bank35667 +35668 POINT(30.253056169542713 -94.61012153125154) bank35668 +35669 POINT(29.127930647685602 -94.78311118601185) bank35669 +35670 POINT(30.548987525460966 -95.38664269817818) bank35670 +35671 POINT(29.48621031574208 -95.21855020972812) bank35671 +35672 POINT(28.868852959982824 -94.53106686270658) bank35672 +35673 POINT(30.614482230343416 -95.06707705657024) bank35673 +35674 POINT(30.22380826334156 -95.86651785419036) bank35674 +35675 POINT(29.924312133184632 -95.61643707832357) bank35675 +35676 POINT(30.199643054153924 -96.07979414517037) bank35676 +35677 POINT(30.446443447521304 -95.78054365038182) bank35677 +35678 POINT(30.510063722826068 -95.42578922554634) bank35678 +35679 POINT(30.11081436779852 -95.1216670731435) bank35679 +35680 POINT(29.746058193331248 -94.48322461833229) bank35680 +35681 POINT(29.59376980509642 -95.93678677123083) bank35681 +35682 POINT(30.336063201701357 -95.7618813104824) bank35682 +35683 POINT(30.103224564174525 -95.19239930770966) bank35683 +35684 POINT(29.32831665661975 -95.89578851546563) bank35684 +35685 POINT(30.525191138326 -96.16643585555776) bank35685 +35686 POINT(29.96879329064823 -94.62537266998352) bank35686 +35687 POINT(28.823467288722462 -95.32078358839973) bank35687 +35688 POINT(30.526835541014115 -94.624503739196) bank35688 +35689 POINT(29.673668690447638 -95.55857811006103) bank35689 +35690 POINT(30.456215376652807 -94.80689831652451) bank35690 +35691 POINT(30.085192967968275 -96.20375305241161) bank35691 +35692 POINT(30.465032281133325 -94.87707533781625) bank35692 +35693 POINT(30.404755890605273 -95.99778749166965) bank35693 +35694 POINT(29.12663989244022 -94.65369241288059) bank35694 +35695 POINT(29.409578704056486 -96.10766214995878) bank35695 +35696 POINT(29.682792551204646 -94.93327244383646) bank35696 +35697 POINT(30.16497028205512 -95.07947138505429) bank35697 +35698 POINT(29.75870109904099 -95.43823599885948) bank35698 +35699 POINT(29.391213165746073 -95.87855175693521) bank35699 +35700 POINT(29.517606089048034 -94.3815039607718) bank35700 +35701 POINT(28.862843146461124 -95.29554663218947) bank35701 +35702 POINT(30.110723635172743 -95.78247352474462) bank35702 +35703 POINT(30.75326640804283 -96.26088210971554) bank35703 +35704 POINT(28.781979635256995 -95.53637686707066) bank35704 +35705 POINT(29.334766593851427 -94.61834448597446) bank35705 +35706 POINT(29.459423518752512 -95.5115048445791) bank35706 +35707 POINT(30.08777470214822 -95.8165424406733) bank35707 +35708 POINT(30.26311344497225 -94.40479400076482) bank35708 +35709 POINT(29.467599765571368 -94.73881286385588) bank35709 +35710 POINT(30.66021534683238 -95.51464934391495) bank35710 +35711 POINT(29.269452412849695 -96.06540622132293) bank35711 +35712 POINT(30.08654637877129 -96.03300008217737) bank35712 +35713 POINT(29.48319670611753 -95.9660264883598) bank35713 +35714 POINT(28.891088881287214 -95.81154537318916) bank35714 +35715 POINT(30.752328436744207 -95.64280445047666) bank35715 +35716 POINT(29.80339488489809 -94.49971311835198) bank35716 +35717 POINT(30.600039819317786 -95.37989689009139) bank35717 +35718 POINT(29.631433253365874 -96.21555615263743) bank35718 +35719 POINT(30.02954090593582 -95.53916755388049) bank35719 +35720 POINT(29.496714540247915 -96.35803985464196) bank35720 +35721 POINT(29.671139548771635 -96.0553860685375) bank35721 +35722 POINT(30.285981412776223 -95.4536024954384) bank35722 +35723 POINT(29.99365612262334 -96.25696047109068) bank35723 +35724 POINT(29.64033326266262 -95.12049791524085) bank35724 +35725 POINT(29.6298975229448 -95.69458908840251) bank35725 +35726 POINT(29.586062300113714 -96.2187624486063) bank35726 +35727 POINT(30.036461992796564 -96.06046970088025) bank35727 +35728 POINT(30.649837736959867 -96.20093599361736) bank35728 +35729 POINT(29.085167994767165 -95.10303152104072) bank35729 +35730 POINT(29.802392300027453 -94.37816317776955) bank35730 +35731 POINT(29.832752432683733 -95.42560770821494) bank35731 +35732 POINT(30.151987676557873 -95.60003919295613) bank35732 +35733 POINT(30.307493217262824 -95.15905211230054) bank35733 +35734 POINT(30.416401151371428 -94.89369591572859) bank35734 +35735 POINT(30.65050104708328 -95.50785511138264) bank35735 +35736 POINT(29.710414827930762 -96.28239387057499) bank35736 +35737 POINT(28.762812087060084 -94.6042328626832) bank35737 +35738 POINT(30.460373965273714 -94.60732857140349) bank35738 +35739 POINT(29.89502043132842 -96.22190880820273) bank35739 +35740 POINT(29.45052356667372 -94.8149438023622) bank35740 +35741 POINT(30.75388851223397 -95.37579531203461) bank35741 +35742 POINT(30.370054791877415 -94.72342109134809) bank35742 +35743 POINT(29.03026585616974 -95.70006043585579) bank35743 +35744 POINT(30.73495399378143 -96.27578811101816) bank35744 +35745 POINT(30.753953546981236 -94.59882508466627) bank35745 +35746 POINT(29.336930347245602 -94.53014580270845) bank35746 +35747 POINT(30.587477261394152 -95.22404289489202) bank35747 +35748 POINT(29.640389886332073 -94.73520602279805) bank35748 +35749 POINT(29.86094350712558 -96.36663066480145) bank35749 +35750 POINT(29.501757919857155 -94.83468414524961) bank35750 +35751 POINT(29.35908150761928 -94.54268035945655) bank35751 +35752 POINT(29.39364328770274 -96.31502399091251) bank35752 +35753 POINT(30.65061933365859 -95.01113580234174) bank35753 +35754 POINT(29.02718794343445 -95.51969776051874) bank35754 +35755 POINT(30.703874338989937 -94.68092206624088) bank35755 +35756 POINT(29.815932686089564 -94.45050906076271) bank35756 +35757 POINT(30.48242459305491 -94.85505276914924) bank35757 +35758 POINT(29.17802366943121 -94.83969334806342) bank35758 +35759 POINT(29.84445215032487 -96.23103801450024) bank35759 +35760 POINT(29.501936037008402 -96.24243460821927) bank35760 +35761 POINT(30.048496042921734 -95.69447197399202) bank35761 +35762 POINT(29.210352009005316 -94.76722742463329) bank35762 +35763 POINT(29.482908209435756 -94.60955252907851) bank35763 +35764 POINT(29.064531562088206 -94.71944617742548) bank35764 +35765 POINT(30.369881866406647 -95.4915827580737) bank35765 +35766 POINT(29.094223568185093 -95.54220403880757) bank35766 +35767 POINT(29.648648545869822 -95.85959289998326) bank35767 +35768 POINT(30.154039646464618 -94.54309618206935) bank35768 +35769 POINT(30.219195288485462 -94.90676773069667) bank35769 +35770 POINT(29.31285891570835 -94.77143386611215) bank35770 +35771 POINT(28.819052981193476 -95.97486995478528) bank35771 +35772 POINT(28.90238852448091 -96.3579868098304) bank35772 +35773 POINT(29.541236742379407 -95.47656922943705) bank35773 +35774 POINT(29.051684155128214 -94.62419702962963) bank35774 +35775 POINT(29.61632683406604 -96.04442063487265) bank35775 +35776 POINT(29.35155915779232 -94.66481604002605) bank35776 +35777 POINT(30.585844209322726 -95.92589819470652) bank35777 +35778 POINT(29.94973408264242 -94.46908534321145) bank35778 +35779 POINT(30.258550474903146 -94.9135808377311) bank35779 +35780 POINT(29.130732719406474 -95.07902736129951) bank35780 +35781 POINT(29.917633815189383 -95.59072070470349) bank35781 +35782 POINT(29.129411807492602 -95.67051016928939) bank35782 +35783 POINT(29.229376439268563 -94.96999628582049) bank35783 +35784 POINT(29.82425272902678 -95.7040270431697) bank35784 +35785 POINT(29.76067826970341 -95.90553865969453) bank35785 +35786 POINT(29.593634197767688 -94.72623158156098) bank35786 +35787 POINT(30.352259117508094 -96.11040505426861) bank35787 +35788 POINT(29.919844631856584 -95.53965912409001) bank35788 +35789 POINT(30.724133980454308 -95.79819619808866) bank35789 +35790 POINT(29.133734713036727 -95.94150247364783) bank35790 +35791 POINT(30.281250651953425 -94.3998241588598) bank35791 +35792 POINT(28.878290297630333 -95.01243784943723) bank35792 +35793 POINT(29.500653758887918 -95.30332512850912) bank35793 +35794 POINT(29.277977092481816 -94.6101624325623) bank35794 +35795 POINT(28.958654768042646 -95.36533534959506) bank35795 +35796 POINT(30.241320613844962 -96.19647677542315) bank35796 +35797 POINT(30.452377870886735 -95.81378181162303) bank35797 +35798 POINT(30.665190043846707 -95.29068128076233) bank35798 +35799 POINT(30.136320294705605 -94.77911222239959) bank35799 +35800 POINT(30.483646195320574 -94.40625417884027) bank35800 +35801 POINT(29.72114611243022 -95.13204062853234) bank35801 +35802 POINT(28.943285122652686 -94.55080482183074) bank35802 +35803 POINT(30.49011440636989 -95.53083094306128) bank35803 +35804 POINT(30.32674225569803 -95.81797783930081) bank35804 +35805 POINT(29.773394827920537 -94.51851506985365) bank35805 +35806 POINT(29.03733357898665 -96.19659364409281) bank35806 +35807 POINT(29.620557270213098 -94.85830726594733) bank35807 +35808 POINT(28.791834637145683 -95.00133391632579) bank35808 +35809 POINT(29.776865540300417 -95.39218757369629) bank35809 +35810 POINT(28.85771302831281 -94.66000486105278) bank35810 +35811 POINT(29.05270699286579 -94.8945212172968) bank35811 +35812 POINT(30.020330761093973 -96.04627329160154) bank35812 +35813 POINT(30.366780828851766 -94.98882260559476) bank35813 +35814 POINT(30.14386682860597 -94.62913843042988) bank35814 +35815 POINT(29.001046675740906 -95.95261610657941) bank35815 +35816 POINT(28.964498643902672 -94.828440115238) bank35816 +35817 POINT(29.68492600185488 -95.33482979587546) bank35817 +35818 POINT(30.49939519235472 -95.86347949665307) bank35818 +35819 POINT(29.854109558362932 -95.89422012251629) bank35819 +35820 POINT(29.081775782438278 -95.43537179703509) bank35820 +35821 POINT(30.11211862469106 -95.37029960647669) bank35821 +35822 POINT(30.170182086249834 -94.75883833011702) bank35822 +35823 POINT(30.09972358126091 -95.35753727375064) bank35823 +35824 POINT(29.93872139280772 -94.963078534891) bank35824 +35825 POINT(30.13466951420045 -94.75284892705167) bank35825 +35826 POINT(30.16695561686199 -96.29723450709844) bank35826 +35827 POINT(29.371124327491607 -95.34590508670597) bank35827 +35828 POINT(29.491721270903565 -95.08491815785807) bank35828 +35829 POINT(28.882984836910104 -95.1050472742195) bank35829 +35830 POINT(28.943781431775072 -94.56625196918097) bank35830 +35831 POINT(29.48180768987199 -94.84180493895887) bank35831 +35832 POINT(29.683721710673584 -95.92547550082786) bank35832 +35833 POINT(30.548722764616745 -95.52781539070918) bank35833 +35834 POINT(30.253718322434032 -95.72704959611812) bank35834 +35835 POINT(29.43735906701892 -95.738219591849) bank35835 +35836 POINT(30.012211890752308 -95.74688018273227) bank35836 +35837 POINT(30.689859177453737 -94.55600540565429) bank35837 +35838 POINT(29.610277516212935 -94.8814297664224) bank35838 +35839 POINT(30.340098524488003 -94.632443221906) bank35839 +35840 POINT(28.997521954810068 -95.84756710263026) bank35840 +35841 POINT(29.75777372069765 -96.20736555868922) bank35841 +35842 POINT(29.03677308141138 -95.47328135678873) bank35842 +35843 POINT(29.85598164521321 -96.34378051176898) bank35843 +35844 POINT(28.976655530894284 -94.71145755842984) bank35844 +35845 POINT(29.828978369645462 -95.88655503288501) bank35845 +35846 POINT(29.807606708950292 -94.37389255598625) bank35846 +35847 POINT(28.973171793422363 -94.86268207159156) bank35847 +35848 POINT(29.75270135369834 -95.30609128304637) bank35848 +35849 POINT(30.684706241568946 -94.68133643266256) bank35849 +35850 POINT(29.567575559185666 -94.7500412425013) bank35850 +35851 POINT(29.12688165402462 -94.80736297632306) bank35851 +35852 POINT(30.055728771880393 -95.71656234879065) bank35852 +35853 POINT(29.40770920434805 -94.43999545557541) bank35853 +35854 POINT(30.153959310455242 -95.99840171581205) bank35854 +35855 POINT(30.38398284548182 -95.66519865599221) bank35855 +35856 POINT(29.192753158535965 -94.96394032555696) bank35856 +35857 POINT(30.23012013991444 -95.0294298481354) bank35857 +35858 POINT(30.454046999662324 -94.78815175403477) bank35858 +35859 POINT(30.48590492711715 -94.90096091309326) bank35859 +35860 POINT(30.237178541668463 -96.35235267415685) bank35860 +35861 POINT(28.861211075064823 -95.06608886831975) bank35861 +35862 POINT(29.139907898223758 -96.31011037730659) bank35862 +35863 POINT(30.75260057951481 -94.79409849554625) bank35863 +35864 POINT(29.48447368732462 -95.16972998845968) bank35864 +35865 POINT(29.05045476244113 -94.43505258121951) bank35865 +35866 POINT(29.41885070825131 -95.1743509874932) bank35866 +35867 POINT(29.167156977790782 -95.61711520895281) bank35867 +35868 POINT(30.546087489926425 -95.63770151109517) bank35868 +35869 POINT(29.85217915663274 -94.7509516448692) bank35869 +35870 POINT(29.05166767856035 -95.62487290326622) bank35870 +35871 POINT(30.51939722100612 -95.52778676669352) bank35871 +35872 POINT(29.57102848311831 -95.18184532728198) bank35872 +35873 POINT(30.233582590863822 -94.98003348579155) bank35873 +35874 POINT(30.534440507158752 -95.01126209857915) bank35874 +35875 POINT(29.546820222805497 -95.84767762325629) bank35875 +35876 POINT(28.8698813457209 -95.37087684857048) bank35876 +35877 POINT(30.22689803299772 -95.5193477496342) bank35877 +35878 POINT(28.79087714512615 -95.19041004808257) bank35878 +35879 POINT(30.13008189089095 -95.81336818825994) bank35879 +35880 POINT(29.67561897113733 -95.91869282956037) bank35880 +35881 POINT(29.913317569120274 -95.65544925959472) bank35881 +35882 POINT(29.61504188280639 -95.93919313205328) bank35882 +35883 POINT(28.926143883559934 -95.44669788635036) bank35883 +35884 POINT(30.727223458160147 -94.93425339975981) bank35884 +35885 POINT(30.513843025900734 -94.79191674754993) bank35885 +35886 POINT(29.906052077898366 -96.30685612143604) bank35886 +35887 POINT(30.376195370101176 -95.6454101364943) bank35887 +35888 POINT(28.857475349355273 -94.69968274381786) bank35888 +35889 POINT(30.39571066445166 -95.82244771636587) bank35889 +35890 POINT(29.277225042766442 -94.42093071796734) bank35890 +35891 POINT(29.863782680048924 -94.46871202138583) bank35891 +35892 POINT(29.001504938894307 -94.63313053109579) bank35892 +35893 POINT(30.25609563531096 -95.32866040800236) bank35893 +35894 POINT(29.56129301108986 -95.71438858283705) bank35894 +35895 POINT(29.991319117723787 -95.02811219891532) bank35895 +35896 POINT(30.705218890284833 -95.22762156248923) bank35896 +35897 POINT(30.136141895994797 -95.93976402951411) bank35897 +35898 POINT(30.380372508912657 -95.07281809688624) bank35898 +35899 POINT(28.844963141528574 -94.84238627039676) bank35899 +35900 POINT(29.571117139487548 -95.21439096020397) bank35900 +35901 POINT(29.339676053026835 -95.3701583223271) bank35901 +35902 POINT(30.23590845300738 -95.46923110417526) bank35902 +35903 POINT(30.35584124040173 -95.90171156094) bank35903 +35904 POINT(29.47443502969045 -94.508179722381) bank35904 +35905 POINT(29.228938116207498 -94.98221588058561) bank35905 +35906 POINT(29.82880792030656 -96.0388776450137) bank35906 +35907 POINT(29.37853394509433 -95.73737837606559) bank35907 +35908 POINT(29.69274678847583 -95.39961253187371) bank35908 +35909 POINT(30.726667594197966 -95.06503385932999) bank35909 +35910 POINT(29.352506466175043 -94.8440311383043) bank35910 +35911 POINT(29.005005274447033 -94.79485071254157) bank35911 +35912 POINT(30.609115085705646 -94.4218973746484) bank35912 +35913 POINT(29.91380511717101 -95.75004600236927) bank35913 +35914 POINT(29.066716042790194 -95.31224228587777) bank35914 +35915 POINT(30.500960430877697 -95.78674164986467) bank35915 +35916 POINT(29.09296957577127 -96.32620785467974) bank35916 +35917 POINT(29.331593556881003 -95.85863521944113) bank35917 +35918 POINT(29.60649748086466 -95.4974123987949) bank35918 +35919 POINT(30.655852876119585 -95.68141028768386) bank35919 +35920 POINT(30.031054357338423 -95.493532234807) bank35920 +35921 POINT(30.114843485828477 -95.59916279767705) bank35921 +35922 POINT(29.838569307332357 -96.244584327292) bank35922 +35923 POINT(28.96170956662754 -95.35264679421323) bank35923 +35924 POINT(29.505018548865596 -96.22021781796055) bank35924 +35925 POINT(29.054067063720066 -94.79255518436946) bank35925 +35926 POINT(29.69181763605112 -95.80699902057022) bank35926 +35927 POINT(29.650638664777198 -95.58240013494463) bank35927 +35928 POINT(30.697877154545026 -94.95388158775984) bank35928 +35929 POINT(29.295937013475896 -95.72342858082433) bank35929 +35930 POINT(29.2809562967462 -94.8986572955098) bank35930 +35931 POINT(28.974172219405187 -95.58096911877135) bank35931 +35932 POINT(30.441176652898083 -95.70126967669627) bank35932 +35933 POINT(28.877733075119476 -95.33364153381237) bank35933 +35934 POINT(28.898452197648925 -95.68161941514728) bank35934 +35935 POINT(30.690825974472197 -94.96910919372698) bank35935 +35936 POINT(29.044257233969702 -95.72107346971207) bank35936 +35937 POINT(29.90036834449623 -96.00468523306662) bank35937 +35938 POINT(28.890048389714003 -96.2753917892773) bank35938 +35939 POINT(28.821249193898662 -95.39135834086774) bank35939 +35940 POINT(29.278181727317374 -95.40768002623462) bank35940 +35941 POINT(30.671296930334396 -95.71070066665571) bank35941 +35942 POINT(29.171153635218577 -95.72488281653864) bank35942 +35943 POINT(29.26045775079247 -94.82054205648222) bank35943 +35944 POINT(30.072024982898874 -96.20712220667956) bank35944 +35945 POINT(28.921407276316796 -94.85704941328777) bank35945 +35946 POINT(29.601745360769282 -94.47980699624529) bank35946 +35947 POINT(29.070932945827224 -95.39035252497055) bank35947 +35948 POINT(30.167409335025056 -94.44786522714676) bank35948 +35949 POINT(29.054784586201315 -95.29034919911874) bank35949 +35950 POINT(29.900244624268712 -95.80384901937123) bank35950 +35951 POINT(30.583330920108203 -94.57823247178206) bank35951 +35952 POINT(30.003221755277934 -95.82353929343671) bank35952 +35953 POINT(30.49878840271213 -94.8953658381065) bank35953 +35954 POINT(30.446673763517097 -94.94264759456908) bank35954 +35955 POINT(29.666608442615857 -95.70206897896882) bank35955 +35956 POINT(30.144360100196852 -96.07707651599654) bank35956 +35957 POINT(30.402927458482985 -95.9137719603457) bank35957 +35958 POINT(28.88917579087389 -95.42444295581912) bank35958 +35959 POINT(30.144561070827645 -96.14827379007248) bank35959 +35960 POINT(30.197597713759514 -94.64929111730125) bank35960 +35961 POINT(30.458275626889282 -96.09625111355422) bank35961 +35962 POINT(29.79191720759573 -94.84737007953761) bank35962 +35963 POINT(30.46417594443906 -95.60660591902356) bank35963 +35964 POINT(29.400749766563102 -96.32087708580943) bank35964 +35965 POINT(30.211409830808286 -95.54610019774572) bank35965 +35966 POINT(30.184357684860483 -95.76784766233838) bank35966 +35967 POINT(29.538591750245143 -94.86764534668126) bank35967 +35968 POINT(30.073951336155737 -96.34203126420671) bank35968 +35969 POINT(29.722956591829792 -95.25970957740847) bank35969 +35970 POINT(30.7054845289998 -95.87482166370944) bank35970 +35971 POINT(29.105999363699276 -95.06704922487603) bank35971 +35972 POINT(29.467619760146707 -95.25542939844298) bank35972 +35973 POINT(30.03001721168585 -95.83626652208936) bank35973 +35974 POINT(30.5028230903726 -95.93241339793366) bank35974 +35975 POINT(28.93426246409493 -95.70429428665997) bank35975 +35976 POINT(30.0538354028906 -94.76741113209222) bank35976 +35977 POINT(30.151909368287235 -95.41069716585471) bank35977 +35978 POINT(29.092280784119694 -95.85992721441733) bank35978 +35979 POINT(30.47337598106223 -94.79224877628297) bank35979 +35980 POINT(29.343333452625444 -95.48552881657977) bank35980 +35981 POINT(28.906526235127444 -95.73865459181594) bank35981 +35982 POINT(30.034632680524552 -95.95560533484041) bank35982 +35983 POINT(28.9613125776108 -94.528244214649) bank35983 +35984 POINT(29.39382998473549 -95.61896824918789) bank35984 +35985 POINT(29.19984936314953 -94.66656152053008) bank35985 +35986 POINT(29.02783266255953 -96.33349803459113) bank35986 +35987 POINT(29.26679956177276 -95.22596193387217) bank35987 +35988 POINT(28.9090392748458 -95.78719675214631) bank35988 +35989 POINT(30.32319586620768 -95.38328759452577) bank35989 +35990 POINT(30.745877167885784 -94.46779304153978) bank35990 +35991 POINT(30.558663976077217 -94.58387117916602) bank35991 +35992 POINT(29.590410641762478 -95.53947249868034) bank35992 +35993 POINT(29.891402903779763 -95.83821437469196) bank35993 +35994 POINT(29.963510671001643 -95.00076063009534) bank35994 +35995 POINT(29.989332333709307 -95.7188220799217) bank35995 +35996 POINT(30.593060175702973 -96.24621248455662) bank35996 +35997 POINT(30.61552516149094 -96.05031705090258) bank35997 +35998 POINT(28.76320056895388 -94.41759979175401) bank35998 +35999 POINT(29.15891090020386 -96.15283111124658) bank35999 +36000 POINT(28.79328987987502 -96.07612275455509) bank36000 +36001 POINT(29.732112235054764 -95.83882840331385) bank36001 +36002 POINT(28.803620558772387 -95.2394916386049) bank36002 +36003 POINT(28.94602962180188 -94.8511567468831) bank36003 +36004 POINT(29.99211926145571 -95.05483592510807) bank36004 +36005 POINT(30.731799612204078 -96.30529357112934) bank36005 +36006 POINT(29.855528669556396 -94.57181158641556) bank36006 +36007 POINT(30.391951338416277 -96.21680445404914) bank36007 +36008 POINT(30.19778515939032 -96.19665531946363) bank36008 +36009 POINT(30.273047302415396 -95.78135377973571) bank36009 +36010 POINT(29.803474948978078 -95.51821681238991) bank36010 +36011 POINT(28.892051745922203 -94.86608444683912) bank36011 +36012 POINT(29.61978134852064 -94.3719775093714) bank36012 +36013 POINT(28.918885167715775 -95.60754522342549) bank36013 +36014 POINT(29.772371653687415 -94.61187855548488) bank36014 +36015 POINT(29.738597626912284 -95.67591227794979) bank36015 +36016 POINT(29.5621270281503 -96.33039446319557) bank36016 +36017 POINT(30.342945229237127 -94.68820823104717) bank36017 +36018 POINT(29.188603532192374 -94.93762709239134) bank36018 +36019 POINT(30.09541643116335 -95.85267060830193) bank36019 +36020 POINT(30.730142602356906 -95.9692429460712) bank36020 +36021 POINT(29.02490160280411 -95.07188521522227) bank36021 +36022 POINT(30.418826316536247 -95.23335371496576) bank36022 +36023 POINT(29.66521311478251 -96.19656974212498) bank36023 +36024 POINT(29.211765969930774 -95.67406678367902) bank36024 +36025 POINT(30.35164104700405 -94.56817354944194) bank36025 +36026 POINT(30.014775577640844 -95.01595037808583) bank36026 +36027 POINT(28.769646947211974 -95.14698827626685) bank36027 +36028 POINT(28.90911935512633 -94.47512380183514) bank36028 +36029 POINT(29.344160535094908 -95.25547184122595) bank36029 +36030 POINT(29.119949076860866 -94.62830204301713) bank36030 +36031 POINT(30.215729601201055 -95.99021863779426) bank36031 +36032 POINT(29.121208587016596 -95.60095997410042) bank36032 +36033 POINT(29.170177046876013 -95.26927242151302) bank36033 +36034 POINT(29.78438209137024 -96.1347886178523) bank36034 +36035 POINT(29.17969967107019 -96.34599190378512) bank36035 +36036 POINT(30.027082484688847 -94.58624265888778) bank36036 +36037 POINT(30.050068273705957 -94.65529842835517) bank36037 +36038 POINT(29.74473846248368 -94.42496268372024) bank36038 +36039 POINT(29.961194436928473 -94.88746824962736) bank36039 +36040 POINT(29.686146506594522 -94.96624801668074) bank36040 +36041 POINT(29.273072814275118 -95.92118527439241) bank36041 +36042 POINT(29.36248262013374 -96.12806885464924) bank36042 +36043 POINT(28.989506620827118 -95.21466321807799) bank36043 +36044 POINT(29.14333380001465 -95.86867123661243) bank36044 +36045 POINT(29.391100001259964 -94.51146688515847) bank36045 +36046 POINT(30.57761460644796 -94.79575352172961) bank36046 +36047 POINT(29.62482392163639 -94.83274349928476) bank36047 +36048 POINT(29.15884108769439 -96.12042425416422) bank36048 +36049 POINT(29.560967345652745 -95.33467466373119) bank36049 +36050 POINT(30.63181116628008 -95.68807383304222) bank36050 +36051 POINT(29.670671870616502 -95.56323177605701) bank36051 +36052 POINT(29.72703538739671 -95.68956153338972) bank36052 +36053 POINT(29.680235972551454 -94.81116033486876) bank36053 +36054 POINT(29.62700045922768 -94.89166840429206) bank36054 +36055 POINT(30.347349251332005 -95.07049157907436) bank36055 +36056 POINT(29.31345602795584 -95.89768415261378) bank36056 +36057 POINT(30.712846995729244 -94.68507983409819) bank36057 +36058 POINT(30.717032011295935 -94.93321152576645) bank36058 +36059 POINT(29.58322252815913 -94.79239417608682) bank36059 +36060 POINT(29.701384043797166 -94.40955443396203) bank36060 +36061 POINT(30.547616521888667 -95.10724952320506) bank36061 +36062 POINT(28.810445245392483 -94.89525383167913) bank36062 +36063 POINT(30.374656531642767 -95.8157736795429) bank36063 +36064 POINT(30.292636684028352 -95.70414913049117) bank36064 +36065 POINT(29.03891111056589 -94.70998589562016) bank36065 +36066 POINT(29.392008628854327 -95.72310831757795) bank36066 +36067 POINT(30.0299619752625 -95.60423280940472) bank36067 +36068 POINT(30.010918634062527 -94.79345057447662) bank36068 +36069 POINT(29.66724191356997 -95.8997831785141) bank36069 +36070 POINT(29.25185576143071 -94.68342065421713) bank36070 +36071 POINT(29.374291807966383 -95.60257809410888) bank36071 +36072 POINT(29.503946705494634 -94.59871445832908) bank36072 +36073 POINT(30.71741875834015 -95.11980568977884) bank36073 +36074 POINT(28.884289655393133 -94.45551714177695) bank36074 +36075 POINT(28.813791631653476 -95.10055296878863) bank36075 +36076 POINT(29.78266552139729 -95.66017307069885) bank36076 +36077 POINT(30.64742962237354 -95.86567894712564) bank36077 +36078 POINT(29.129912888974445 -95.82731045074729) bank36078 +36079 POINT(30.74868382047412 -94.8989271004683) bank36079 +36080 POINT(30.697020850622312 -96.13353527729946) bank36080 +36081 POINT(30.250761175177733 -95.68240118357393) bank36081 +36082 POINT(29.543653222026943 -96.21214154891432) bank36082 +36083 POINT(30.680016536217913 -95.85509811930193) bank36083 +36084 POINT(30.4197017932676 -95.77299638787888) bank36084 +36085 POINT(30.38829373588579 -95.4358230354964) bank36085 +36086 POINT(28.908537649978193 -95.45983045633204) bank36086 +36087 POINT(29.063845087912288 -95.9986278012302) bank36087 +36088 POINT(30.463612634216602 -94.88127936443301) bank36088 +36089 POINT(29.12120742259557 -95.99141599113238) bank36089 +36090 POINT(29.253573203234076 -95.51731066422984) bank36090 +36091 POINT(29.866793354504193 -94.7765541149852) bank36091 +36092 POINT(30.239026691052672 -94.92224858054335) bank36092 +36093 POINT(29.910839022795816 -96.15479798800234) bank36093 +36094 POINT(30.434715346904426 -94.45450969203056) bank36094 +36095 POINT(29.82588436195461 -95.89216298553329) bank36095 +36096 POINT(29.85497981594773 -95.89663971253505) bank36096 +36097 POINT(29.42622667849207 -95.14466092370887) bank36097 +36098 POINT(28.76223223037631 -94.82582815607597) bank36098 +36099 POINT(28.90224250458717 -94.6783421431349) bank36099 +36100 POINT(30.298446272460037 -95.12774331188848) bank36100 +36101 POINT(29.998416001674972 -95.28676265006239) bank36101 +36102 POINT(29.758598514508986 -95.91849596445937) bank36102 +36103 POINT(29.260203562707797 -95.18856776765371) bank36103 +36104 POINT(30.588362801573076 -94.43893758156868) bank36104 +36105 POINT(28.890191137916442 -96.23946912720763) bank36105 +36106 POINT(30.071349625999094 -94.89434646684728) bank36106 +36107 POINT(29.35689416989194 -95.70490563610315) bank36107 +36108 POINT(29.049832388300576 -94.80836727274533) bank36108 +36109 POINT(29.322495954346767 -94.99197616065933) bank36109 +36110 POINT(30.408987096456045 -94.65295120900102) bank36110 +36111 POINT(29.675438490375843 -94.38638222016708) bank36111 +36112 POINT(29.801613609199492 -95.09442627259409) bank36112 +36113 POINT(28.784588416305784 -95.18008160666248) bank36113 +36114 POINT(29.66613365559994 -94.6970324131342) bank36114 +36115 POINT(30.3975715179748 -95.66963112682576) bank36115 +36116 POINT(29.67182471776362 -95.08229572233486) bank36116 +36117 POINT(28.772659044654745 -95.01149280165785) bank36117 +36118 POINT(29.97484977927641 -94.80022254397127) bank36118 +36119 POINT(29.234187854136998 -95.94999723397677) bank36119 +36120 POINT(29.962641715401432 -95.88789952378079) bank36120 +36121 POINT(29.08447593722802 -95.0532907684763) bank36121 +36122 POINT(29.573758326125773 -94.93172806506104) bank36122 +36123 POINT(30.592879238402745 -96.24551648549863) bank36123 +36124 POINT(29.48569523406057 -94.6529835155059) bank36124 +36125 POINT(30.542793213620346 -96.19182329641279) bank36125 +36126 POINT(30.54458175074221 -96.25103759228878) bank36126 +36127 POINT(30.64047041600991 -94.6165667582888) bank36127 +36128 POINT(29.631348030390093 -95.81226183873324) bank36128 +36129 POINT(29.81021482389789 -95.5524021840157) bank36129 +36130 POINT(29.614705817528623 -96.17287499971204) bank36130 +36131 POINT(30.414150029317984 -94.595765141704) bank36131 +36132 POINT(29.566295941200266 -95.68670408196691) bank36132 +36133 POINT(29.97200973169128 -95.77911530826513) bank36133 +36134 POINT(30.35688763169778 -96.26865686827061) bank36134 +36135 POINT(29.63005305126525 -95.26561909116441) bank36135 +36136 POINT(29.29170829458731 -95.81022494829763) bank36136 +36137 POINT(30.09311035336797 -95.8913820699007) bank36137 +36138 POINT(29.328955639918053 -96.30296818751361) bank36138 +36139 POINT(29.588753833755128 -96.2007655583718) bank36139 +36140 POINT(29.250482640618998 -95.81588457560589) bank36140 +36141 POINT(29.252158051578355 -95.72697838800016) bank36141 +36142 POINT(29.829950313681476 -94.6717177501165) bank36142 +36143 POINT(29.234304080339236 -95.68674916042602) bank36143 +36144 POINT(29.0177213480857 -95.44777484510031) bank36144 +36145 POINT(30.662145157589674 -94.60215939601859) bank36145 +36146 POINT(29.653265318115007 -94.9289792535823) bank36146 +36147 POINT(30.111710592479263 -96.1422721318814) bank36147 +36148 POINT(30.159061681076658 -95.9646131584824) bank36148 +36149 POINT(29.29355393559227 -95.73835533259837) bank36149 +36150 POINT(30.662582476172535 -95.74109765858117) bank36150 +36151 POINT(30.29894161955282 -94.82253020343849) bank36151 +36152 POINT(30.478814779334808 -96.24861025683684) bank36152 +36153 POINT(29.33532897962754 -94.46156932935381) bank36153 +36154 POINT(30.145822155930713 -94.42900721452328) bank36154 +36155 POINT(30.503661645489103 -95.91188800644679) bank36155 +36156 POINT(29.34235272504625 -95.26833127430768) bank36156 +36157 POINT(30.60303280816458 -94.50270716843693) bank36157 +36158 POINT(30.581764504542917 -95.51619888824649) bank36158 +36159 POINT(30.48558262504831 -94.49749827701929) bank36159 +36160 POINT(29.418934561600114 -94.8286484813422) bank36160 +36161 POINT(30.074065219373878 -95.18015475701709) bank36161 +36162 POINT(28.924332841585468 -94.49497560712089) bank36162 +36163 POINT(30.017932630993023 -94.41297044343195) bank36163 +36164 POINT(29.84235923319616 -95.08951180004664) bank36164 +36165 POINT(28.879391169607416 -96.11811492667023) bank36165 +36166 POINT(29.35893920212916 -96.21089115900806) bank36166 +36167 POINT(30.043658680070568 -94.82388955489235) bank36167 +36168 POINT(30.051267608567134 -95.0971019523993) bank36168 +36169 POINT(29.139208084082806 -94.6229227194447) bank36169 +36170 POINT(29.30810000906541 -95.63024560085474) bank36170 +36171 POINT(29.326955753244924 -96.32888786406092) bank36171 +36172 POINT(29.973796479077468 -96.28433567769568) bank36172 +36173 POINT(29.45655211501548 -96.16177343658731) bank36173 +36174 POINT(30.377782614541307 -95.63319019305592) bank36174 +36175 POINT(29.122056835852938 -94.730043283638) bank36175 +36176 POINT(30.669779776300874 -96.13519984106733) bank36176 +36177 POINT(28.80625144460853 -94.60710385095028) bank36177 +36178 POINT(29.570880073454347 -96.36208026957452) bank36178 +36179 POINT(29.919738702407667 -95.50208436516456) bank36179 +36180 POINT(30.69611451874231 -94.4819806699211) bank36180 +36181 POINT(29.576960713229376 -94.50407145440471) bank36181 +36182 POINT(29.077249393113597 -95.93041301462226) bank36182 +36183 POINT(28.998575422994705 -95.87023755577083) bank36183 +36184 POINT(30.251030537883857 -95.19321174182086) bank36184 +36185 POINT(30.488735258690298 -95.56084229440815) bank36185 +36186 POINT(30.614816150373045 -95.50054699759325) bank36186 +36187 POINT(30.42656521296957 -94.50262883807245) bank36187 +36188 POINT(30.589573256563433 -94.58927717429788) bank36188 +36189 POINT(29.899478976852226 -94.73639516652757) bank36189 +36190 POINT(30.748436132407523 -95.0908902880176) bank36190 +36191 POINT(30.622595752865198 -94.38660865600094) bank36191 +36192 POINT(30.283321639349996 -95.26320472322536) bank36192 +36193 POINT(29.47451436735885 -95.06811563103498) bank36193 +36194 POINT(29.3376070176537 -94.9323477963704) bank36194 +36195 POINT(30.681736513504532 -96.04542855692209) bank36195 +36196 POINT(30.755827277603213 -95.46603268724391) bank36196 +36197 POINT(29.9353556798804 -95.09916346286258) bank36197 +36198 POINT(30.321474741329393 -95.67125578272876) bank36198 +36199 POINT(30.253458441498626 -95.96656118582169) bank36199 +36200 POINT(29.557006104078432 -95.5529519935058) bank36200 +36201 POINT(29.83857246196227 -95.48637191148913) bank36201 +36202 POINT(29.750454783935247 -94.41742441999881) bank36202 +36203 POINT(30.540472754869835 -96.13722030422167) bank36203 +36204 POINT(29.221140710604736 -95.86429054442306) bank36204 +36205 POINT(29.125926221764672 -96.1600390887769) bank36205 +36206 POINT(30.572782117499216 -96.31210280440061) bank36206 +36207 POINT(30.050747627909356 -96.28168184927723) bank36207 +36208 POINT(30.254395791948443 -96.22231091895418) bank36208 +36209 POINT(29.286980468198546 -94.68679476805391) bank36209 +36210 POINT(29.12577904690456 -96.1302533705934) bank36210 +36211 POINT(29.76840334904332 -95.9761260304357) bank36211 +36212 POINT(29.69044354985977 -96.32637893678147) bank36212 +36213 POINT(29.714489314889512 -96.32486137085564) bank36213 +36214 POINT(29.32054074850853 -95.45084912907836) bank36214 +36215 POINT(29.63310629943925 -95.50163070668398) bank36215 +36216 POINT(29.646375282771963 -95.7041774195285) bank36216 +36217 POINT(30.57036067654785 -95.6906401103798) bank36217 +36218 POINT(29.65955260748366 -96.30499127352014) bank36218 +36219 POINT(29.675475313827 -94.48541839117613) bank36219 +36220 POINT(30.660459130421124 -95.08789472924009) bank36220 +36221 POINT(30.609608252303527 -94.43307839667237) bank36221 +36222 POINT(29.470412402794942 -96.1941154407503) bank36222 +36223 POINT(28.80298132213718 -94.66915109683069) bank36223 +36224 POINT(29.815385523988713 -95.1883815893088) bank36224 +36225 POINT(29.188977258841547 -95.39896394543536) bank36225 +36226 POINT(29.97171823839737 -96.25938225585064) bank36226 +36227 POINT(29.477584940831242 -94.45321848322686) bank36227 +36228 POINT(29.517184897281652 -95.85108326920299) bank36228 +36229 POINT(29.19938144724037 -95.5983553209155) bank36229 +36230 POINT(30.590643040335348 -95.04312351560533) bank36230 +36231 POINT(30.40732776199333 -94.45914092070468) bank36231 +36232 POINT(29.91614154642896 -96.30877190668008) bank36232 +36233 POINT(29.75706018056331 -94.53362226227372) bank36233 +36234 POINT(29.69095550128465 -95.94662927807582) bank36234 +36235 POINT(29.020936065497374 -96.1994244908276) bank36235 +36236 POINT(29.525221092817116 -96.30382333209057) bank36236 +36237 POINT(29.805074122259782 -94.3851359232933) bank36237 +36238 POINT(29.429198157082713 -95.2726720070776) bank36238 +36239 POINT(30.512998369300412 -96.33201537129196) bank36239 +36240 POINT(30.586712849669976 -95.10764794409556) bank36240 +36241 POINT(29.626299074414916 -96.35595858027142) bank36241 +36242 POINT(29.976716544138235 -94.67773194762442) bank36242 +36243 POINT(29.13438340217364 -95.21212671685163) bank36243 +36244 POINT(30.208533217281342 -94.68484971021893) bank36244 +36245 POINT(29.07174861608096 -95.83170809005593) bank36245 +36246 POINT(30.082354145616232 -94.99633275649214) bank36246 +36247 POINT(29.604484587090102 -95.0325058353364) bank36247 +36248 POINT(30.201735420296814 -94.79076066038596) bank36248 +36249 POINT(30.001702768028856 -95.74601335520606) bank36249 +36250 POINT(30.68338793012362 -94.40910116926176) bank36250 +36251 POINT(29.041624060562164 -95.7542664438987) bank36251 +36252 POINT(29.762664751831007 -95.08418493126584) bank36252 +36253 POINT(29.98951223798744 -96.03774084648099) bank36253 +36254 POINT(30.427740955011707 -96.07678546445963) bank36254 +36255 POINT(29.846180902717744 -94.55811368937091) bank36255 +36256 POINT(30.07198236366857 -94.56220025852372) bank36256 +36257 POINT(30.49855009609933 -95.66557175158704) bank36257 +36258 POINT(28.83292498649179 -94.54042936423227) bank36258 +36259 POINT(30.757475978836958 -96.07992689363152) bank36259 +36260 POINT(29.459214722754385 -94.94521876215097) bank36260 +36261 POINT(30.59812511656552 -94.44374877778155) bank36261 +36262 POINT(30.73622408610089 -94.72618535119955) bank36262 +36263 POINT(29.135060586210546 -95.54840735997699) bank36263 +36264 POINT(30.390812136282577 -94.66870345123516) bank36264 +36265 POINT(29.42148027375262 -95.152045121702) bank36265 +36266 POINT(28.9720797902684 -95.06160197679233) bank36266 +36267 POINT(29.948697568803528 -95.62958823293876) bank36267 +36268 POINT(30.44413978672493 -94.9453769195626) bank36268 +36269 POINT(29.367131079640696 -94.97794185660929) bank36269 +36270 POINT(30.346359975744978 -94.7009852589097) bank36270 +36271 POINT(29.87167768980252 -95.91993107677327) bank36271 +36272 POINT(30.17015282388473 -95.86972035299951) bank36272 +36273 POINT(29.467020236637982 -95.90343952362971) bank36273 +36274 POINT(29.673757112057906 -94.89004345330059) bank36274 +36275 POINT(29.180631381604435 -95.58572197745585) bank36275 +36276 POINT(30.430524249641856 -95.63295772433122) bank36276 +36277 POINT(30.62114214662549 -95.61203874399108) bank36277 +36278 POINT(30.143221223212358 -95.60843442944855) bank36278 +36279 POINT(29.751858778266108 -94.79750871999909) bank36279 +36280 POINT(30.43672599844879 -94.9450021032541) bank36280 +36281 POINT(29.585560045479138 -95.71642235045547) bank36281 +36282 POINT(30.62010439882942 -95.57848294450908) bank36282 +36283 POINT(29.763986675472882 -96.13992511379755) bank36283 +36284 POINT(29.968474111201516 -95.10019344283425) bank36284 +36285 POINT(29.68874413293322 -96.33740374276849) bank36285 +36286 POINT(30.402842660272118 -94.94275272915107) bank36286 +36287 POINT(30.36207548332857 -94.49242747401297) bank36287 +36288 POINT(30.515857166614296 -95.25561698854676) bank36288 +36289 POINT(29.9142538676877 -94.61518179192268) bank36289 +36290 POINT(30.008461985343764 -96.31712459770934) bank36290 +36291 POINT(28.99367513349641 -96.15865915722823) bank36291 +36292 POINT(29.684557280524057 -96.2703634398974) bank36292 +36293 POINT(29.61157472873235 -94.73548740199159) bank36293 +36294 POINT(30.752054442528422 -95.08660381691494) bank36294 +36295 POINT(29.32691371331715 -95.92243467610471) bank36295 +36296 POINT(29.437885702761363 -94.64187795946278) bank36296 +36297 POINT(29.65538377938258 -96.35589599125063) bank36297 +36298 POINT(29.622878068550623 -94.97682803066651) bank36298 +36299 POINT(30.367610190587754 -96.31486694453848) bank36299 +36300 POINT(29.186053550138563 -95.45642784191284) bank36300 +36301 POINT(30.48329718550662 -94.96335584745378) bank36301 +36302 POINT(29.521047768512766 -95.31902922797141) bank36302 +36303 POINT(30.462166740496325 -94.84092865211542) bank36303 +36304 POINT(30.024334828448 -95.60083605675227) bank36304 +36305 POINT(30.738659041518368 -95.76859240383287) bank36305 +36306 POINT(29.271191816686116 -95.7631059885873) bank36306 +36307 POINT(29.712789914375023 -95.11253748248579) bank36307 +36308 POINT(29.935839150867842 -96.06138643463994) bank36308 +36309 POINT(29.26733176841385 -95.22640301514942) bank36309 +36310 POINT(30.508916107817406 -94.70974431165402) bank36310 +36311 POINT(28.83325115495779 -94.86704917703356) bank36311 +36312 POINT(29.7810487006014 -94.71377815506979) bank36312 +36313 POINT(30.441014089861778 -95.38405317537509) bank36313 +36314 POINT(29.872047816805406 -95.15178248782553) bank36314 +36315 POINT(28.820902816814986 -95.55072485898724) bank36315 +36316 POINT(29.654279172848245 -96.11875639506322) bank36316 +36317 POINT(28.937112565080856 -95.34021456853141) bank36317 +36318 POINT(29.06924711515033 -94.88923097521656) bank36318 +36319 POINT(30.31411947919125 -95.33026779401109) bank36319 +36320 POINT(29.739836019411676 -96.0185505159975) bank36320 +36321 POINT(30.61041593927147 -95.93692553451858) bank36321 +36322 POINT(28.84446926171143 -95.08140942490591) bank36322 +36323 POINT(30.02641410320509 -94.90022404863895) bank36323 +36324 POINT(29.80773136979484 -95.61391984346815) bank36324 +36325 POINT(30.254690169875552 -94.91102625739695) bank36325 +36326 POINT(30.35132786716558 -94.89190140550113) bank36326 +36327 POINT(29.651500831526217 -95.4315742751318) bank36327 +36328 POINT(30.20247224925035 -95.24252284633435) bank36328 +36329 POINT(30.218568964033704 -96.07905499100644) bank36329 +36330 POINT(29.274456505597524 -94.95397591858058) bank36330 +36331 POINT(28.852691966865404 -95.33362185056885) bank36331 +36332 POINT(29.76164670013828 -94.57205512522744) bank36332 +36333 POINT(30.001196555451138 -94.83494466359645) bank36333 +36334 POINT(30.39985467138556 -94.57797725293429) bank36334 +36335 POINT(28.98358394795288 -95.68491785399196) bank36335 +36336 POINT(28.99711491096482 -94.45006870957221) bank36336 +36337 POINT(28.770398132378748 -95.71093536519612) bank36337 +36338 POINT(30.35989763095035 -94.56159720485657) bank36338 +36339 POINT(29.248860372073 -95.29241040208838) bank36339 +36340 POINT(30.21047057138573 -96.14681495163836) bank36340 +36341 POINT(29.29356315755968 -95.35203035713711) bank36341 +36342 POINT(30.32740540714885 -96.20884828223797) bank36342 +36343 POINT(30.65815174975565 -94.78702803390102) bank36343 +36344 POINT(30.080026100972887 -95.88417991793018) bank36344 +36345 POINT(29.567590748206076 -94.44203046003842) bank36345 +36346 POINT(30.670040670488234 -95.77670174176346) bank36346 +36347 POINT(30.171436482428746 -95.99072335677076) bank36347 +36348 POINT(30.403860145224712 -94.96535857627963) bank36348 +36349 POINT(30.164812219548566 -94.41849164341764) bank36349 +36350 POINT(29.178237836787424 -95.32217447363938) bank36350 +36351 POINT(29.145191185979492 -95.10207002870324) bank36351 +36352 POINT(30.234537740146166 -94.69417921302924) bank36352 +36353 POINT(28.91193983764394 -94.55755017355435) bank36353 +36354 POINT(29.313450673026065 -94.77846944560544) bank36354 +36355 POINT(29.47086299758868 -95.37931796810769) bank36355 +36356 POINT(30.370332454171596 -95.5479905229206) bank36356 +36357 POINT(30.637264482212295 -95.93010178407854) bank36357 +36358 POINT(30.064453212233094 -94.60790800748642) bank36358 +36359 POINT(29.58827135983789 -95.94025003911507) bank36359 +36360 POINT(29.11920650154797 -95.71147971948756) bank36360 +36361 POINT(28.830717421918617 -95.92050980633023) bank36361 +36362 POINT(30.453722561711114 -96.03819295640287) bank36362 +36363 POINT(29.142963210441806 -96.189166512496) bank36363 +36364 POINT(30.160231528806893 -94.68889935897673) bank36364 +36365 POINT(30.697475525369132 -94.54817344773844) bank36365 +36366 POINT(29.86983656140283 -94.96541131637234) bank36366 +36367 POINT(30.68749292504272 -95.95155939101936) bank36367 +36368 POINT(29.799246668361416 -95.30757573449685) bank36368 +36369 POINT(29.502247034198945 -96.13790829186959) bank36369 +36370 POINT(29.837318910713524 -96.18841649395024) bank36370 +36371 POINT(29.022994913946757 -94.56991820671016) bank36371 +36372 POINT(29.816250949863782 -95.7241334237471) bank36372 +36373 POINT(30.377241835569805 -95.40810276135677) bank36373 +36374 POINT(30.6512479284214 -95.4788511678182) bank36374 +36375 POINT(30.508126772617118 -94.77680284807656) bank36375 +36376 POINT(29.421976740180103 -94.93438137309785) bank36376 +36377 POINT(29.132662170984613 -95.88341495970556) bank36377 +36378 POINT(29.985931139598012 -95.56971805605698) bank36378 +36379 POINT(29.156724336424933 -96.1912355149497) bank36379 +36380 POINT(30.33669133809986 -95.07037200522078) bank36380 +36381 POINT(30.036624465846035 -95.86494882967727) bank36381 +36382 POINT(30.1022943084603 -95.46467897620003) bank36382 +36383 POINT(29.742282398380453 -94.51895334812667) bank36383 +36384 POINT(29.064246703978522 -94.52270839676639) bank36384 +36385 POINT(30.443348833522556 -96.06333273422383) bank36385 +36386 POINT(28.94480601944241 -96.34883700227006) bank36386 +36387 POINT(30.486982933446278 -94.8265587235114) bank36387 +36388 POINT(30.32169416310796 -95.07339453501842) bank36388 +36389 POINT(29.82068782208385 -95.55848712101665) bank36389 +36390 POINT(29.39145287998129 -96.08537062421716) bank36390 +36391 POINT(30.73871152276241 -95.42530746936558) bank36391 +36392 POINT(29.586878759477614 -95.26960667178845) bank36392 +36393 POINT(30.759498549691823 -96.0675899971398) bank36393 +36394 POINT(29.650572413838518 -95.49670130212579) bank36394 +36395 POINT(29.633122120476376 -94.55971789665747) bank36395 +36396 POINT(30.154307014855124 -95.54154938306237) bank36396 +36397 POINT(30.666788766629328 -94.86199031113931) bank36397 +36398 POINT(30.684069201983775 -95.6106915612626) bank36398 +36399 POINT(30.60942257218649 -94.4440282510109) bank36399 +36400 POINT(30.021649276254458 -96.2339761009791) bank36400 +36401 POINT(30.425228231573815 -96.07573272871664) bank36401 +36402 POINT(29.67899991244326 -95.72236842619103) bank36402 +36403 POINT(30.080746340856003 -95.37594325369004) bank36403 +36404 POINT(29.92288905548598 -96.20440209554661) bank36404 +36405 POINT(30.277787703071922 -96.30666192977087) bank36405 +36406 POINT(29.485325041512237 -94.6512246129469) bank36406 +36407 POINT(30.694226884503845 -95.16454368135855) bank36407 +36408 POINT(29.985656416563277 -94.69840556095893) bank36408 +36409 POINT(29.51987840194107 -95.29842812229346) bank36409 +36410 POINT(28.932176579476568 -94.62772318108773) bank36410 +36411 POINT(30.2624253580723 -95.74302681725844) bank36411 +36412 POINT(29.374364945364064 -95.48473695864251) bank36412 +36413 POINT(29.925477540553512 -95.13857282575049) bank36413 +36414 POINT(29.179343462521572 -94.87873154095914) bank36414 +36415 POINT(29.275341630419813 -95.62411053605102) bank36415 +36416 POINT(28.80649825503889 -96.05243394054898) bank36416 +36417 POINT(30.622756833106134 -94.3801347871493) bank36417 +36418 POINT(29.786321965007605 -95.4228830197556) bank36418 +36419 POINT(30.146420800610688 -94.44806776717313) bank36419 +36420 POINT(30.64576205490305 -94.77281842232303) bank36420 +36421 POINT(28.940740342827635 -96.05293254311898) bank36421 +36422 POINT(30.66537882551427 -95.03081166290065) bank36422 +36423 POINT(30.640547953338164 -95.20328330280991) bank36423 +36424 POINT(29.935369971262098 -94.63766324341613) bank36424 +36425 POINT(29.090604627439582 -96.14804638692158) bank36425 +36426 POINT(29.003654669866524 -95.90083434278368) bank36426 +36427 POINT(29.71853734204695 -94.53222171697554) bank36427 +36428 POINT(28.808562381940316 -95.33002474258834) bank36428 +36429 POINT(30.02301396338766 -94.6395886187337) bank36429 +36430 POINT(29.75036196557362 -94.73395261289832) bank36430 +36431 POINT(29.256815203893343 -95.5903960455142) bank36431 +36432 POINT(29.982681043031874 -95.61207372890784) bank36432 +36433 POINT(29.29063720534836 -94.3733605880146) bank36433 +36434 POINT(29.187884741491427 -94.60009448654735) bank36434 +36435 POINT(29.233729244582882 -94.92902979156734) bank36435 +36436 POINT(29.604927440953297 -94.6853566923971) bank36436 +36437 POINT(29.301767941649555 -94.72270045435116) bank36437 +36438 POINT(30.241567786066504 -94.74910609767261) bank36438 +36439 POINT(30.250906484611125 -94.86168679650008) bank36439 +36440 POINT(29.67043884808104 -95.19492933347298) bank36440 +36441 POINT(30.341490361094387 -94.58698205380594) bank36441 +36442 POINT(28.941522360960327 -95.37836327484763) bank36442 +36443 POINT(28.830669371016633 -95.1644240969777) bank36443 +36444 POINT(30.70018190327388 -95.7626616345923) bank36444 +36445 POINT(30.06355773231092 -94.7084252101186) bank36445 +36446 POINT(29.871851324417847 -95.0383008154515) bank36446 +36447 POINT(29.75849198489584 -95.2295170422372) bank36447 +36448 POINT(29.475567890820244 -96.10740920208802) bank36448 +36449 POINT(29.803093754623262 -94.49006013234357) bank36449 +36450 POINT(29.149769103547115 -95.95979575329093) bank36450 +36451 POINT(29.703973891635012 -95.91757038250032) bank36451 +36452 POINT(29.118218144967916 -95.40825318859815) bank36452 +36453 POINT(30.7516738524614 -95.39831025971613) bank36453 +36454 POINT(30.377819327707186 -96.09902409317539) bank36454 +36455 POINT(29.01145770896798 -96.28360599728055) bank36455 +36456 POINT(30.0361364436087 -95.5605749719814) bank36456 +36457 POINT(29.15158082625353 -96.35961680160003) bank36457 +36458 POINT(30.577778807556573 -95.12184866500715) bank36458 +36459 POINT(29.19700428470843 -95.57122578591434) bank36459 +36460 POINT(29.65512063538538 -95.64993613044223) bank36460 +36461 POINT(29.210475964480082 -95.47603247902572) bank36461 +36462 POINT(28.989702327498428 -96.19398524948588) bank36462 +36463 POINT(30.02780505592273 -94.93830186677556) bank36463 +36464 POINT(30.31798992675291 -96.22391499632077) bank36464 +36465 POINT(29.94813169653044 -96.15858554380563) bank36465 +36466 POINT(30.086147462043026 -94.70741582170194) bank36466 +36467 POINT(28.785000157859564 -94.67959329535954) bank36467 +36468 POINT(30.744547277141503 -96.27101784223497) bank36468 +36469 POINT(29.124349323929355 -94.3978149525144) bank36469 +36470 POINT(30.093492899774475 -94.7787360202731) bank36470 +36471 POINT(29.841439882493596 -95.1232013693957) bank36471 +36472 POINT(30.424865400651818 -94.45095818810584) bank36472 +36473 POINT(30.04944249725446 -94.71575368428147) bank36473 +36474 POINT(29.049433920413932 -95.00962721887433) bank36474 +36475 POINT(30.046789056734017 -96.03881082371245) bank36475 +36476 POINT(28.89866241566813 -95.18728309368998) bank36476 +36477 POINT(29.445128237275885 -94.61934082104483) bank36477 +36478 POINT(30.394245842783523 -94.69261854986696) bank36478 +36479 POINT(29.981511175463222 -95.982938978836) bank36479 +36480 POINT(29.181872266943707 -95.21374013467693) bank36480 +36481 POINT(29.124612534888644 -95.42612465181003) bank36481 +36482 POINT(30.418367806691137 -95.21606506242253) bank36482 +36483 POINT(28.81480530665633 -95.9673353774033) bank36483 +36484 POINT(30.10050733266576 -94.73055824240382) bank36484 +36485 POINT(30.2444684935173 -95.88271067573973) bank36485 +36486 POINT(29.452512061195424 -94.52890294505278) bank36486 +36487 POINT(30.569365334143583 -94.39091283499245) bank36487 +36488 POINT(29.017519446241902 -95.54298910578397) bank36488 +36489 POINT(28.943503256593807 -94.53653816518454) bank36489 +36490 POINT(29.77361372850065 -94.4912458663705) bank36490 +36491 POINT(30.003278128286407 -96.15875463352862) bank36491 +36492 POINT(29.349680897435164 -94.91611193364422) bank36492 +36493 POINT(30.466950700742593 -95.22819010325959) bank36493 +36494 POINT(30.128178621951392 -95.56835955370266) bank36494 +36495 POINT(28.97227943398735 -95.83742089482952) bank36495 +36496 POINT(30.24591790410282 -96.01002500553115) bank36496 +36497 POINT(29.703562923492548 -94.93344045542108) bank36497 +36498 POINT(29.50221327103378 -95.13761572637318) bank36498 +36499 POINT(30.69713484204873 -95.73786204588113) bank36499 +36500 POINT(30.419154180617717 -95.77896353840006) bank36500 +36501 POINT(29.275080434635335 -95.18039871988495) bank36501 +36502 POINT(30.366603784605964 -94.98095869012423) bank36502 +36503 POINT(28.99541936536032 -95.16803536258321) bank36503 +36504 POINT(29.929833218683406 -96.1649756662606) bank36504 +36505 POINT(30.327429551433063 -96.3455019979704) bank36505 +36506 POINT(30.575898036185517 -94.42456445585023) bank36506 +36507 POINT(28.89386155786467 -96.28223905170414) bank36507 +36508 POINT(29.98261942679378 -94.39737939556731) bank36508 +36509 POINT(29.387746115822498 -95.88084334002895) bank36509 +36510 POINT(29.709084483496756 -95.71969920185968) bank36510 +36511 POINT(30.33442544193289 -95.40500413279332) bank36511 +36512 POINT(28.83216608197807 -94.54550582612261) bank36512 +36513 POINT(30.346146553777395 -96.36860435531433) bank36513 +36514 POINT(29.950304539809935 -95.76183831570695) bank36514 +36515 POINT(28.94612655696579 -95.21829124276049) bank36515 +36516 POINT(30.115321757525088 -94.51769930896252) bank36516 +36517 POINT(29.89731254407895 -94.3822837408139) bank36517 +36518 POINT(29.222923631135238 -96.35599337919832) bank36518 +36519 POINT(28.8132544532063 -95.34126973616286) bank36519 +36520 POINT(29.214243037384556 -94.88935818352712) bank36520 +36521 POINT(29.00386351266846 -95.88018481033744) bank36521 +36522 POINT(30.162388856306496 -95.93927057420973) bank36522 +36523 POINT(28.817337371430646 -94.79891442652223) bank36523 +36524 POINT(30.14360877356536 -95.99199223913297) bank36524 +36525 POINT(30.226406089903435 -94.53915230132282) bank36525 +36526 POINT(29.746786028392684 -94.63432068625163) bank36526 +36527 POINT(30.486208478132916 -95.11851444612961) bank36527 +36528 POINT(29.600104493584745 -95.2653051173568) bank36528 +36529 POINT(30.747539818655444 -95.36330277223563) bank36529 +36530 POINT(30.530096304026284 -94.84808865200866) bank36530 +36531 POINT(30.664997408293598 -95.16300835973335) bank36531 +36532 POINT(30.42800185846042 -94.9402702991894) bank36532 +36533 POINT(29.443154090753104 -95.36925930230919) bank36533 +36534 POINT(28.821767378234682 -95.77087041070857) bank36534 +36535 POINT(29.90347596371477 -94.9154416017069) bank36535 +36536 POINT(30.45198481842627 -95.42498432955077) bank36536 +36537 POINT(30.706611714408833 -95.63079072575141) bank36537 +36538 POINT(29.611927621637253 -96.186563326784) bank36538 +36539 POINT(29.26151398826558 -96.12183707868819) bank36539 +36540 POINT(29.280779681881523 -96.29373344191798) bank36540 +36541 POINT(29.68682997812129 -96.05510520473548) bank36541 +36542 POINT(30.154318059696642 -95.58702176242231) bank36542 +36543 POINT(28.821760915004635 -95.39852426921604) bank36543 +36544 POINT(30.40531555029266 -95.80908046982431) bank36544 +36545 POINT(30.341664328207568 -95.48939042207799) bank36545 +36546 POINT(29.022301142957613 -95.22391838392186) bank36546 +36547 POINT(29.943573744148377 -94.67775910560749) bank36547 +36548 POINT(29.75725512129887 -96.26861147131925) bank36548 +36549 POINT(30.653959483186853 -95.43137246045167) bank36549 +36550 POINT(29.624963995444716 -95.3378266685268) bank36550 +36551 POINT(30.28046367355531 -95.57003692391402) bank36551 +36552 POINT(29.996795502632462 -94.99041962209957) bank36552 +36553 POINT(29.44153922372886 -96.16267888945833) bank36553 +36554 POINT(28.762742089458115 -96.27238367501161) bank36554 +36555 POINT(28.84946352234787 -94.9617738081411) bank36555 +36556 POINT(29.0867920672031 -95.43847944166457) bank36556 +36557 POINT(29.70395475821402 -96.25756917299331) bank36557 +36558 POINT(29.325222654956626 -94.49933528912003) bank36558 +36559 POINT(29.021975858045405 -96.31009031584325) bank36559 +36560 POINT(29.456706693351286 -95.89196700363965) bank36560 +36561 POINT(30.03651109876148 -96.17682759305809) bank36561 +36562 POINT(29.883016579584567 -95.12790405092035) bank36562 +36563 POINT(28.848736184035758 -94.43115681159163) bank36563 +36564 POINT(29.3479357296551 -96.10138499275497) bank36564 +36565 POINT(29.553721935540253 -95.89375075924296) bank36565 +36566 POINT(29.56686708025556 -94.6605570851121) bank36566 +36567 POINT(30.508621414909886 -95.97707515179164) bank36567 +36568 POINT(30.138268101672573 -94.68706160517323) bank36568 +36569 POINT(29.45091810855523 -96.03886279569348) bank36569 +36570 POINT(29.643503504593685 -95.14445023008302) bank36570 +36571 POINT(30.329764748316574 -95.30447111382914) bank36571 +36572 POINT(28.827279705376718 -94.9941519871134) bank36572 +36573 POINT(29.885404328297295 -96.18918121713338) bank36573 +36574 POINT(29.167239894075347 -96.22376834269251) bank36574 +36575 POINT(30.513851776785973 -95.37299603023983) bank36575 +36576 POINT(29.071228319764536 -94.61521246355957) bank36576 +36577 POINT(30.53578491330609 -94.81543031397229) bank36577 +36578 POINT(29.15932786770918 -94.85254832164607) bank36578 +36579 POINT(30.040450750452933 -95.99714127657892) bank36579 +36580 POINT(30.487367330785794 -94.62099936159093) bank36580 +36581 POINT(30.066916786321112 -95.82663949691167) bank36581 +36582 POINT(29.95028678320804 -95.65994622373057) bank36582 +36583 POINT(29.752538439124045 -95.29098954712178) bank36583 +36584 POINT(30.629817044358564 -94.45827159341633) bank36584 +36585 POINT(30.10031831597362 -94.82205184883308) bank36585 +36586 POINT(29.884206820551803 -94.41472729403674) bank36586 +36587 POINT(30.135824221284327 -95.24631683226333) bank36587 +36588 POINT(29.400083450002754 -94.52193429457266) bank36588 +36589 POINT(29.435723931135666 -94.69108308107477) bank36589 +36590 POINT(28.93161243642127 -95.39406922559294) bank36590 +36591 POINT(29.7564660257218 -94.39849025893253) bank36591 +36592 POINT(29.404930008028995 -94.87691331043523) bank36592 +36593 POINT(28.97771707236326 -95.30049238017116) bank36593 +36594 POINT(30.38135444667019 -95.13192151112936) bank36594 +36595 POINT(29.83470362795012 -94.77888567758248) bank36595 +36596 POINT(29.255590964962405 -95.6416372824308) bank36596 +36597 POINT(29.093271648339424 -94.64687442715896) bank36597 +36598 POINT(29.853154463231185 -95.46612152098456) bank36598 +36599 POINT(29.273731591498468 -96.18194823265706) bank36599 +36600 POINT(28.967906832274853 -94.87395127861708) bank36600 +36601 POINT(29.942229096681128 -94.71159129758946) bank36601 +36602 POINT(30.58324877470553 -96.0428097590737) bank36602 +36603 POINT(30.33692125459903 -95.7173585019812) bank36603 +36604 POINT(29.180910626507305 -94.40878068566252) bank36604 +36605 POINT(28.83887953668441 -94.8294172493317) bank36605 +36606 POINT(30.087595831372024 -95.65011103570423) bank36606 +36607 POINT(29.83861717356006 -94.89360553329604) bank36607 +36608 POINT(30.070636376395306 -96.20509273881089) bank36608 +36609 POINT(30.31051610656066 -94.57669951884145) bank36609 +36610 POINT(29.354079768479593 -94.51073347335546) bank36610 +36611 POINT(30.65169482952565 -96.13629163191523) bank36611 +36612 POINT(30.269143017524147 -95.57950174643595) bank36612 +36613 POINT(30.01991184735883 -95.98073060446833) bank36613 +36614 POINT(29.833033535449857 -94.51376897381854) bank36614 +36615 POINT(30.43327564108528 -94.78185067457429) bank36615 +36616 POINT(29.952022780017128 -94.86319123563015) bank36616 +36617 POINT(30.685659630464677 -96.1452162041332) bank36617 +36618 POINT(30.661067853496796 -95.93216625592004) bank36618 +36619 POINT(30.737692667473585 -96.00043164584898) bank36619 +36620 POINT(28.889223713376673 -94.65746077042682) bank36620 +36621 POINT(30.19211552762043 -95.02943843751937) bank36621 +36622 POINT(28.780992008360506 -95.58824780993139) bank36622 +36623 POINT(30.533947728185893 -94.69112087051695) bank36623 +36624 POINT(30.009738482915843 -96.27613064791531) bank36624 +36625 POINT(29.37368382644227 -95.16941330450811) bank36625 +36626 POINT(30.544236606951102 -95.74462063055401) bank36626 +36627 POINT(29.519042889620696 -94.47871867360406) bank36627 +36628 POINT(29.470489298653373 -95.27512231787883) bank36628 +36629 POINT(30.39644587174065 -96.09314487698683) bank36629 +36630 POINT(29.916611655523713 -95.10093672248283) bank36630 +36631 POINT(29.400559750712233 -95.01324410561892) bank36631 +36632 POINT(29.42533330887345 -95.14182380666537) bank36632 +36633 POINT(30.500746854127726 -96.09595849366166) bank36633 +36634 POINT(30.243803392140105 -96.00391302065427) bank36634 +36635 POINT(30.431445847721125 -94.37818076482391) bank36635 +36636 POINT(30.733858205525916 -96.05696907493339) bank36636 +36637 POINT(30.1546188066927 -94.50978342231791) bank36637 +36638 POINT(29.32034128226883 -95.91706394183765) bank36638 +36639 POINT(30.747727966664446 -94.63182639085377) bank36639 +36640 POINT(30.57046005882134 -95.27546816146709) bank36640 +36641 POINT(29.2797569059817 -95.90909091747383) bank36641 +36642 POINT(30.683440928711175 -94.63297987397601) bank36642 +36643 POINT(30.2690738328906 -96.26575859714912) bank36643 +36644 POINT(29.752064916790324 -95.23761983777156) bank36644 +36645 POINT(30.65621868511253 -95.89991478213541) bank36645 +36646 POINT(30.02939877072994 -95.40232707371393) bank36646 +36647 POINT(30.748085196863272 -95.26486046654242) bank36647 +36648 POINT(29.32847922624231 -94.70018508151095) bank36648 +36649 POINT(29.671264453754006 -95.12941978691177) bank36649 +36650 POINT(29.908082607432902 -94.59780871311719) bank36650 +36651 POINT(29.770395383028923 -95.39414642011994) bank36651 +36652 POINT(30.025691500505392 -96.11952071914557) bank36652 +36653 POINT(30.404285544594188 -96.21375137791311) bank36653 +36654 POINT(30.739639756225632 -94.97037493145567) bank36654 +36655 POINT(30.713032090001683 -96.25428696564872) bank36655 +36656 POINT(29.89538819918285 -95.0755341423233) bank36656 +36657 POINT(30.605557922109707 -94.56223482749766) bank36657 +36658 POINT(30.73950148201383 -95.54290979551132) bank36658 +36659 POINT(30.44526057404847 -94.5291097245013) bank36659 +36660 POINT(29.905445580692252 -95.10091510517185) bank36660 +36661 POINT(29.93387058851295 -95.40786177198267) bank36661 +36662 POINT(29.762826128258958 -95.5975562182646) bank36662 +36663 POINT(28.820873569067743 -96.24371087059266) bank36663 +36664 POINT(29.399710904290558 -95.01199521863168) bank36664 +36665 POINT(29.285244415705094 -94.88334350103793) bank36665 +36666 POINT(29.165040871402503 -95.4838570195179) bank36666 +36667 POINT(30.142704306151153 -95.22772341793096) bank36667 +36668 POINT(29.689033289679884 -95.78709345017612) bank36668 +36669 POINT(29.649257809333033 -95.4948019624101) bank36669 +36670 POINT(30.498394169995933 -95.51255134956189) bank36670 +36671 POINT(30.00879418544804 -94.6650516465541) bank36671 +36672 POINT(29.495873587882016 -94.71576254220089) bank36672 +36673 POINT(29.911597309475148 -95.3872105909438) bank36673 +36674 POINT(30.294640945097022 -94.52043036013636) bank36674 +36675 POINT(28.871534639811486 -95.60238980152315) bank36675 +36676 POINT(29.386793672218143 -96.33629073132418) bank36676 +36677 POINT(30.15411992307179 -95.78519728098935) bank36677 +36678 POINT(28.97708845253172 -94.95101324409059) bank36678 +36679 POINT(30.40540847413683 -95.7265054108448) bank36679 +36680 POINT(30.5084574997154 -95.92039825449656) bank36680 +36681 POINT(29.935677892146046 -94.38014026222649) bank36681 +36682 POINT(29.33247870808959 -96.29092642173923) bank36682 +36683 POINT(29.506724189446327 -95.73273642927919) bank36683 +36684 POINT(30.403034048086038 -95.71381251126861) bank36684 +36685 POINT(30.528484825785085 -94.8935670980865) bank36685 +36686 POINT(29.040770778277192 -95.53232486567171) bank36686 +36687 POINT(30.070511550028026 -95.22372604588118) bank36687 +36688 POINT(30.47586043431098 -95.96910433671324) bank36688 +36689 POINT(29.006704501934014 -95.00497670618323) bank36689 +36690 POINT(30.13881107853613 -95.24980620417371) bank36690 +36691 POINT(29.338577090730716 -96.2224925165858) bank36691 +36692 POINT(29.941145837635755 -95.57450378508061) bank36692 +36693 POINT(29.8904061273385 -96.30443910291402) bank36693 +36694 POINT(30.46368457951739 -95.7870108504459) bank36694 +36695 POINT(30.44361264243969 -96.05880862706307) bank36695 +36696 POINT(30.69161553245795 -94.75498805455639) bank36696 +36697 POINT(29.54913487181152 -95.93829555585027) bank36697 +36698 POINT(29.30944033709742 -95.43427814051718) bank36698 +36699 POINT(30.48218298133154 -95.73293615352583) bank36699 +36700 POINT(29.983182745927223 -95.06697979324046) bank36700 +36701 POINT(29.414138295256308 -96.19751561208344) bank36701 +36702 POINT(30.647387029395553 -95.35310098646198) bank36702 +36703 POINT(30.1907344195418 -95.02114634041041) bank36703 +36704 POINT(29.98803421273433 -94.82793162316541) bank36704 +36705 POINT(30.538655091259375 -96.33340145858996) bank36705 +36706 POINT(28.992132583928033 -95.18654535937142) bank36706 +36707 POINT(30.051261414488593 -95.20481093753997) bank36707 +36708 POINT(29.3828354807786 -95.25691727864573) bank36708 +36709 POINT(30.42809078200713 -95.12106155951868) bank36709 +36710 POINT(30.417186460101952 -94.59077582834551) bank36710 +36711 POINT(29.229702900776164 -96.13109443006061) bank36711 +36712 POINT(29.215092648011396 -95.18343219058687) bank36712 +36713 POINT(29.12754081890319 -95.3442555224514) bank36713 +36714 POINT(29.306652333405818 -94.38534330354116) bank36714 +36715 POINT(29.60928953641367 -95.26675591939488) bank36715 +36716 POINT(29.401068214930795 -96.26030622271027) bank36716 +36717 POINT(29.09035771577428 -95.46105945628463) bank36717 +36718 POINT(29.369808904796184 -96.29854380963383) bank36718 +36719 POINT(29.797516176836254 -94.5671019660696) bank36719 +36720 POINT(29.17975909677297 -95.29276916425184) bank36720 +36721 POINT(29.029029439314826 -94.90082903298887) bank36721 +36722 POINT(29.460814122562315 -94.91898689212807) bank36722 +36723 POINT(30.301669277973527 -94.4197930266158) bank36723 +36724 POINT(30.553912219281454 -95.50159521046517) bank36724 +36725 POINT(30.44391753229905 -95.62303490582113) bank36725 +36726 POINT(29.29638510376295 -94.71673859837658) bank36726 +36727 POINT(30.116366389842888 -94.9519845803719) bank36727 +36728 POINT(28.969689613722803 -95.84268468661186) bank36728 +36729 POINT(29.2927885624349 -95.02693164642132) bank36729 +36730 POINT(30.30761451002928 -94.59581443333377) bank36730 +36731 POINT(30.59869669632526 -94.82812181117988) bank36731 +36732 POINT(29.027857162233055 -95.67782816736384) bank36732 +36733 POINT(30.033838043255045 -94.9448842390857) bank36733 +36734 POINT(30.55086790926313 -95.2477115263058) bank36734 +36735 POINT(30.26262012701884 -96.11752374275203) bank36735 +36736 POINT(29.504159428103193 -95.43002889518529) bank36736 +36737 POINT(29.43594004420085 -95.03186475262991) bank36737 +36738 POINT(29.79932217068375 -95.58711291298134) bank36738 +36739 POINT(29.21541776843074 -95.19025428704025) bank36739 +36740 POINT(30.660282172350623 -95.09714531728082) bank36740 +36741 POINT(30.247516159436316 -95.59221295840067) bank36741 +36742 POINT(30.102981176973433 -95.53480135903243) bank36742 +36743 POINT(30.01487542808489 -94.51296327754964) bank36743 +36744 POINT(30.35923250322631 -95.0632254894755) bank36744 +36745 POINT(30.53333662693698 -94.85667490406712) bank36745 +36746 POINT(30.376250831431232 -94.92648249435311) bank36746 +36747 POINT(30.305958672725804 -95.94284784588218) bank36747 +36748 POINT(30.357686231642653 -96.18050402778267) bank36748 +36749 POINT(29.483134405745492 -95.42121960994307) bank36749 +36750 POINT(30.051212271251565 -94.41409830534212) bank36750 +36751 POINT(30.176557490547523 -94.72940209538375) bank36751 +36752 POINT(30.53334562073069 -95.39373654670787) bank36752 +36753 POINT(29.731425273511398 -95.6054983293382) bank36753 +36754 POINT(29.18398815977075 -96.19930031997465) bank36754 +36755 POINT(30.170708996228832 -95.26919735357275) bank36755 +36756 POINT(29.8083472443323 -94.94098741974342) bank36756 +36757 POINT(30.635178093860564 -94.67117596576395) bank36757 +36758 POINT(30.44997408641248 -96.2980587057941) bank36758 +36759 POINT(30.716611235630683 -94.97203398175255) bank36759 +36760 POINT(29.0618113044133 -94.6160533257938) bank36760 +36761 POINT(29.38775122013493 -95.13787045067916) bank36761 +36762 POINT(30.631226599144423 -95.5588967434412) bank36762 +36763 POINT(29.3723258855098 -96.10820862102749) bank36763 +36764 POINT(30.574705427480744 -95.1004286469023) bank36764 +36765 POINT(29.271212671065918 -95.68652583258839) bank36765 +36766 POINT(29.83575538023919 -95.42446149654363) bank36766 +36767 POINT(30.467815936028273 -95.31342084246239) bank36767 +36768 POINT(30.118903061261765 -95.05285811255213) bank36768 +36769 POINT(29.996810975271746 -95.04823940035875) bank36769 +36770 POINT(29.897459374230056 -94.7026238610943) bank36770 +36771 POINT(30.16885342042882 -95.99744449615721) bank36771 +36772 POINT(30.654288212593755 -96.34446550154841) bank36772 +36773 POINT(29.690060801123266 -95.28653484982517) bank36773 +36774 POINT(29.00142428230753 -94.9613975882529) bank36774 +36775 POINT(30.55044481950073 -95.55150629072097) bank36775 +36776 POINT(29.544471953400187 -95.5522745440842) bank36776 +36777 POINT(30.6280853310449 -95.29080927138337) bank36777 +36778 POINT(28.975517807168266 -96.11991986245044) bank36778 +36779 POINT(29.32288193637734 -94.45025007438703) bank36779 +36780 POINT(29.171743772509345 -95.65274223130321) bank36780 +36781 POINT(30.40870306013322 -96.30282691661536) bank36781 +36782 POINT(29.16861227821402 -95.31089832781103) bank36782 +36783 POINT(29.956511603403868 -96.17845761361549) bank36783 +36784 POINT(30.013234003609494 -95.59491798391616) bank36784 +36785 POINT(29.51278707899741 -94.6009523811022) bank36785 +36786 POINT(29.56354832384537 -96.14167251288838) bank36786 +36787 POINT(28.96604641642275 -95.9759830477484) bank36787 +36788 POINT(29.23687030056217 -96.11486727438918) bank36788 +36789 POINT(29.23563221028668 -96.28213712382043) bank36789 +36790 POINT(29.385091306610803 -96.33715049820108) bank36790 +36791 POINT(29.614897575806545 -95.28770171582934) bank36791 +36792 POINT(29.039676015204236 -94.91756494935119) bank36792 +36793 POINT(29.73944909468251 -96.0684249539379) bank36793 +36794 POINT(29.878042688443376 -95.30646306387878) bank36794 +36795 POINT(29.276237998203996 -95.40175777074126) bank36795 +36796 POINT(28.784254771751673 -94.89737758758503) bank36796 +36797 POINT(29.57315146501054 -95.07374755717474) bank36797 +36798 POINT(29.96021024813246 -95.14951371877038) bank36798 +36799 POINT(30.58392234854897 -94.54547834491082) bank36799 +36800 POINT(29.265873660593403 -95.27387269666616) bank36800 +36801 POINT(30.06137380966617 -95.50180717628831) bank36801 +36802 POINT(29.247200344656886 -95.81630826644098) bank36802 +36803 POINT(30.230965001247508 -94.72230468985123) bank36803 +36804 POINT(28.97080403450262 -96.26780454680382) bank36804 +36805 POINT(30.534395321554292 -96.02785765275027) bank36805 +36806 POINT(28.772048488178783 -94.53411634287255) bank36806 +36807 POINT(29.504406761402134 -95.6758962001336) bank36807 +36808 POINT(30.11208285409035 -94.99458745739922) bank36808 +36809 POINT(29.68323371909205 -95.11794205182542) bank36809 +36810 POINT(30.53232651940237 -94.38974226746925) bank36810 +36811 POINT(29.48416409088079 -94.86448704516815) bank36811 +36812 POINT(28.99100717280521 -96.00743264956547) bank36812 +36813 POINT(29.025354548783017 -95.71071134002702) bank36813 +36814 POINT(29.154698118266793 -95.36067336070245) bank36814 +36815 POINT(30.681590116911547 -94.55646426218662) bank36815 +36816 POINT(30.388454957466738 -95.8725381941156) bank36816 +36817 POINT(29.48610934857237 -95.0873409728759) bank36817 +36818 POINT(29.7682260116912 -95.4444003618715) bank36818 +36819 POINT(29.610890818396637 -95.54263474215968) bank36819 +36820 POINT(28.810505228970836 -94.86132951198219) bank36820 +36821 POINT(29.90811301330339 -95.8925304553781) bank36821 +36822 POINT(29.114469706206723 -96.31243804287979) bank36822 +36823 POINT(29.96293662837717 -94.57991651765943) bank36823 +36824 POINT(30.478107952933172 -95.94685843107075) bank36824 +36825 POINT(29.487089780192314 -95.04895792221855) bank36825 +36826 POINT(29.499871059399034 -96.02652283677841) bank36826 +36827 POINT(29.648416166207994 -95.34765964963532) bank36827 +36828 POINT(28.896309735397413 -95.3375863085924) bank36828 +36829 POINT(29.795333905326753 -95.68547904602295) bank36829 +36830 POINT(29.659424300047327 -96.25464223540727) bank36830 +36831 POINT(29.185927619584444 -95.2376202464696) bank36831 +36832 POINT(29.23736486708297 -94.97483811292865) bank36832 +36833 POINT(30.197308792683167 -95.47247344727151) bank36833 +36834 POINT(30.736171223721747 -95.4449778071828) bank36834 +36835 POINT(29.36536504330682 -95.33068594232415) bank36835 +36836 POINT(29.35147499759092 -96.02265719030731) bank36836 +36837 POINT(30.356328950285075 -95.09111693908353) bank36837 +36838 POINT(28.81312264890582 -94.82806418284159) bank36838 +36839 POINT(30.196748857571304 -95.10696472515906) bank36839 +36840 POINT(30.514950507235337 -96.14317448623127) bank36840 +36841 POINT(29.4115137384114 -94.76445536002551) bank36841 +36842 POINT(30.72377836236911 -94.51523830905654) bank36842 +36843 POINT(29.425177544411607 -95.09204808403169) bank36843 +36844 POINT(30.572374225851473 -96.20800347246283) bank36844 +36845 POINT(30.372820216409448 -95.5605828623745) bank36845 +36846 POINT(30.235728161351446 -94.61288556211422) bank36846 +36847 POINT(29.121011144176205 -96.21153244058782) bank36847 +36848 POINT(30.74062989296881 -94.51287450627021) bank36848 +36849 POINT(29.73786131566558 -96.23231180352381) bank36849 +36850 POINT(30.307917358784355 -95.8774373800453) bank36850 +36851 POINT(30.728653089527576 -94.73324011175168) bank36851 +36852 POINT(29.335495759443084 -96.28144943011219) bank36852 +36853 POINT(29.60190075660782 -94.70092405316592) bank36853 +36854 POINT(29.103346759111314 -95.86647018469073) bank36854 +36855 POINT(30.059884203644074 -96.27702811388352) bank36855 +36856 POINT(28.847591074794334 -94.53753055779724) bank36856 +36857 POINT(29.596970465058106 -96.33232865673294) bank36857 +36858 POINT(29.03829061779469 -95.67161134148273) bank36858 +36859 POINT(29.156956065608625 -95.89783094177565) bank36859 +36860 POINT(28.940528242115178 -94.8442247117288) bank36860 +36861 POINT(29.03658018948085 -94.62521010352616) bank36861 +36862 POINT(29.821984721333845 -95.47157465756685) bank36862 +36863 POINT(30.142587396058975 -94.84262313274664) bank36863 +36864 POINT(28.785416299798335 -96.25752901796113) bank36864 +36865 POINT(30.307951754595504 -96.29267503990793) bank36865 +36866 POINT(30.348597342328063 -96.21483023847064) bank36866 +36867 POINT(29.820428900372956 -95.00864641657883) bank36867 +36868 POINT(29.68635019973052 -95.86043998855878) bank36868 +36869 POINT(30.617041832987727 -95.86466428405579) bank36869 +36870 POINT(30.3565492202399 -94.39593328829017) bank36870 +36871 POINT(30.3796233501385 -96.1035130641292) bank36871 +36872 POINT(30.552615193972237 -95.92371982093317) bank36872 +36873 POINT(30.513335745357082 -95.41070573616219) bank36873 +36874 POINT(30.299145309346 -95.58530870592581) bank36874 +36875 POINT(29.758445503408964 -94.8256120780052) bank36875 +36876 POINT(30.613101903805603 -95.53153123437751) bank36876 +36877 POINT(29.74334396015878 -95.48037932754663) bank36877 +36878 POINT(30.616151140191405 -94.42085446526247) bank36878 +36879 POINT(29.737563495000153 -96.02995972165311) bank36879 +36880 POINT(30.7586031016288 -95.54828086388551) bank36880 +36881 POINT(29.6255539987639 -95.78019608721715) bank36881 +36882 POINT(30.19710253703506 -95.88707510886452) bank36882 +36883 POINT(29.894054376596916 -95.95246575495685) bank36883 +36884 POINT(28.810973831152516 -95.73175181491635) bank36884 +36885 POINT(28.792289307302866 -95.43654019368832) bank36885 +36886 POINT(29.55889011276286 -94.57291733534034) bank36886 +36887 POINT(29.483583049765723 -95.8329214683593) bank36887 +36888 POINT(30.63084718948071 -95.1361390873424) bank36888 +36889 POINT(29.605871310681643 -96.10886025594904) bank36889 +36890 POINT(28.894139716771367 -94.9172402131194) bank36890 +36891 POINT(30.581147482098288 -94.621414396825) bank36891 +36892 POINT(30.144207124594384 -94.75567458666751) bank36892 +36893 POINT(29.859132945813545 -96.18057827458887) bank36893 +36894 POINT(29.623775677604538 -95.70071134605753) bank36894 +36895 POINT(29.235927157554716 -96.18737254623335) bank36895 +36896 POINT(29.092440626991085 -96.30429170378095) bank36896 +36897 POINT(30.59505547395117 -95.851496606789) bank36897 +36898 POINT(28.874031147932342 -96.22963019304274) bank36898 +36899 POINT(29.948536916262228 -94.5429052025887) bank36899 +36900 POINT(30.316860958371503 -95.15322644609306) bank36900 +36901 POINT(29.731037937756454 -95.7888451627815) bank36901 +36902 POINT(30.097730961531475 -95.36506603513189) bank36902 +36903 POINT(29.439848156197186 -95.78988051512226) bank36903 +36904 POINT(29.37207791857521 -95.60754942427245) bank36904 +36905 POINT(29.078912288220423 -95.18524017587923) bank36905 +36906 POINT(29.187112436452793 -96.35486787244085) bank36906 +36907 POINT(29.508720283018967 -95.31564975578485) bank36907 +36908 POINT(28.878297873293658 -95.11596339721146) bank36908 +36909 POINT(29.337933792078793 -95.58521075160702) bank36909 +36910 POINT(29.977325626774956 -94.76003130681313) bank36910 +36911 POINT(30.47203812691452 -95.84592257285672) bank36911 +36912 POINT(30.20804556747301 -96.29942468058069) bank36912 +36913 POINT(29.4122898126965 -94.5148046995468) bank36913 +36914 POINT(28.97876432174364 -95.8825663575077) bank36914 +36915 POINT(29.966975871368238 -94.5263541507257) bank36915 +36916 POINT(29.008421836269683 -94.64469255772423) bank36916 +36917 POINT(29.49146518426932 -95.11368842811503) bank36917 +36918 POINT(28.920307971018026 -95.56729064696415) bank36918 +36919 POINT(30.325708565700538 -94.85390365946245) bank36919 +36920 POINT(28.87514712031191 -94.86000615562907) bank36920 +36921 POINT(29.17709648022458 -96.02799021874753) bank36921 +36922 POINT(29.917148293560548 -95.56672798475752) bank36922 +36923 POINT(29.756682593638207 -94.51476047106664) bank36923 +36924 POINT(29.674017523350443 -94.80558761450172) bank36924 +36925 POINT(29.55995291458063 -95.21782325107843) bank36925 +36926 POINT(30.758987232448177 -96.32656669952458) bank36926 +36927 POINT(29.18999348022159 -95.15288975163269) bank36927 +36928 POINT(29.510783178514096 -95.47150472461351) bank36928 +36929 POINT(29.78213291993881 -95.52850186925151) bank36929 +36930 POINT(30.18269945487155 -95.28485855965171) bank36930 +36931 POINT(29.30603005802957 -96.27554364618295) bank36931 +36932 POINT(28.91732347193378 -96.01008412867374) bank36932 +36933 POINT(30.12146465133517 -94.50483140455654) bank36933 +36934 POINT(29.39947901936572 -96.13498909173151) bank36934 +36935 POINT(29.16968031954964 -96.31088745947189) bank36935 +36936 POINT(30.710873697925706 -94.61128731853708) bank36936 +36937 POINT(30.28177530530577 -95.76886291352419) bank36937 +36938 POINT(29.0172596769458 -95.62276384695782) bank36938 +36939 POINT(29.179012417710343 -95.75922744410487) bank36939 +36940 POINT(29.52996505916833 -95.05346920001921) bank36940 +36941 POINT(30.003423186575993 -95.43377983284992) bank36941 +36942 POINT(29.50714148838245 -95.49099544950111) bank36942 +36943 POINT(30.17649986117059 -96.04725015017029) bank36943 +36944 POINT(28.84784597354718 -94.76095421539931) bank36944 +36945 POINT(28.96573446892107 -95.34037777590792) bank36945 +36946 POINT(29.21956735626532 -95.80188462828184) bank36946 +36947 POINT(29.49443503216225 -96.28810452028361) bank36947 +36948 POINT(29.614128683177977 -96.06490565754628) bank36948 +36949 POINT(30.236396223423235 -95.7835010786124) bank36949 +36950 POINT(28.906273556151 -95.63957711340244) bank36950 +36951 POINT(28.80745006023757 -94.75788663388472) bank36951 +36952 POINT(29.063690375564864 -94.72941802334073) bank36952 +36953 POINT(29.339970048600772 -95.77134459300446) bank36953 +36954 POINT(29.663410306753182 -94.45295995167409) bank36954 +36955 POINT(30.527077764471244 -95.52625037787688) bank36955 +36956 POINT(30.58176620709445 -95.7987420854605) bank36956 +36957 POINT(30.624888393428574 -95.24008043775369) bank36957 +36958 POINT(29.185484714078147 -94.58937349275547) bank36958 +36959 POINT(29.54442992507139 -95.00684807322992) bank36959 +36960 POINT(29.686535753573356 -96.06794976043497) bank36960 +36961 POINT(28.86183909781682 -94.449772320269) bank36961 +36962 POINT(30.445957679030947 -95.45068675095644) bank36962 +36963 POINT(28.9020470403066 -96.2536837635108) bank36963 +36964 POINT(29.802530996658447 -94.73426171120666) bank36964 +36965 POINT(28.874131359504656 -94.91746913813358) bank36965 +36966 POINT(29.379305020348298 -94.76724526367057) bank36966 +36967 POINT(29.347796473757292 -95.21593511193686) bank36967 +36968 POINT(29.177961967371115 -96.15415205621208) bank36968 +36969 POINT(28.9539171575428 -95.88573659918947) bank36969 +36970 POINT(28.994355548456536 -95.50203127954735) bank36970 +36971 POINT(28.999732008611627 -95.60856137738051) bank36971 +36972 POINT(29.06588330865061 -96.13463075181004) bank36972 +36973 POINT(29.162837957143537 -96.09825738113327) bank36973 +36974 POINT(29.28045343929238 -95.5178510375459) bank36974 +36975 POINT(30.376842053374318 -95.00463442158392) bank36975 +36976 POINT(29.02212360669345 -96.2604732991873) bank36976 +36977 POINT(29.120442670337745 -96.06298751480847) bank36977 +36978 POINT(29.87313939029097 -94.40236269235683) bank36978 +36979 POINT(29.57111484141534 -94.42158821215827) bank36979 +36980 POINT(30.662597864161576 -94.88442443840559) bank36980 +36981 POINT(30.552114265995694 -95.37575917572111) bank36981 +36982 POINT(30.594941485014413 -95.36111898588655) bank36982 +36983 POINT(29.051470338294017 -95.52863472312295) bank36983 +36984 POINT(29.649561691633483 -95.6662375682394) bank36984 +36985 POINT(30.046022920019215 -95.45076627709723) bank36985 +36986 POINT(29.437889201469627 -95.58222107890438) bank36986 +36987 POINT(29.453279862307568 -94.4415339155473) bank36987 +36988 POINT(29.56826072484323 -96.21381249814141) bank36988 +36989 POINT(29.37166946289001 -95.23341541056226) bank36989 +36990 POINT(30.36946870899917 -94.45449814473858) bank36990 +36991 POINT(30.038582805015658 -95.7352744375336) bank36991 +36992 POINT(30.28942677598482 -95.12123627892004) bank36992 +36993 POINT(29.301034082768524 -96.32882134460137) bank36993 +36994 POINT(29.58171771802363 -94.74681340001761) bank36994 +36995 POINT(29.283012009806725 -95.81895764756393) bank36995 +36996 POINT(29.74016564258274 -95.96934559744317) bank36996 +36997 POINT(30.72161387945388 -94.80451466985718) bank36997 +36998 POINT(30.11306697411878 -95.54953770247087) bank36998 +36999 POINT(29.446523067595322 -95.13217257366749) bank36999 +37000 POINT(30.369183759499254 -95.61881932542988) bank37000 +37001 POINT(29.10589776040277 -95.40708985339909) bank37001 +37002 POINT(29.42443573706119 -96.00111655647149) bank37002 +37003 POINT(28.84601218702766 -96.10626261797223) bank37003 +37004 POINT(30.665716495644517 -94.63604209990187) bank37004 +37005 POINT(29.39682890420432 -94.56607826159569) bank37005 +37006 POINT(29.130368760482824 -96.2390546104441) bank37006 +37007 POINT(29.11699049219838 -95.47480020863624) bank37007 +37008 POINT(29.860805939682006 -95.80516972808367) bank37008 +37009 POINT(30.36670847378418 -95.83091357042285) bank37009 +37010 POINT(28.772150413893087 -96.3559504867318) bank37010 +37011 POINT(30.072635489144055 -95.34878067085805) bank37011 +37012 POINT(29.155150549277042 -95.40332353704159) bank37012 +37013 POINT(29.60408963702791 -95.62079868793086) bank37013 +37014 POINT(30.30772969481061 -96.0955588105625) bank37014 +37015 POINT(30.4804362085606 -94.59844908117388) bank37015 +37016 POINT(29.67433328567108 -95.68782000198951) bank37016 +37017 POINT(29.41690008985154 -95.23509493575168) bank37017 +37018 POINT(29.323205511016695 -95.34589634586393) bank37018 +37019 POINT(29.575053552100083 -95.46654855311343) bank37019 +37020 POINT(29.577436202453047 -94.82786150204663) bank37020 +37021 POINT(29.52842796288174 -94.66966263120993) bank37021 +37022 POINT(29.460261050934314 -95.2416401881973) bank37022 +37023 POINT(29.091007770955905 -95.61352526155301) bank37023 +37024 POINT(29.279671360407832 -96.22084838145804) bank37024 +37025 POINT(29.38574078055772 -95.10326398040239) bank37025 +37026 POINT(30.635636684430448 -94.52806079491354) bank37026 +37027 POINT(28.915746713286914 -94.49991191232596) bank37027 +37028 POINT(30.09201905447347 -95.1497347839455) bank37028 +37029 POINT(30.538358570281375 -94.97991426602725) bank37029 +37030 POINT(29.911200349995752 -95.35683857006201) bank37030 +37031 POINT(30.612302124714674 -95.09144852218849) bank37031 +37032 POINT(30.64512976600351 -95.4963946242309) bank37032 +37033 POINT(29.968724641071002 -95.9936536209732) bank37033 +37034 POINT(29.54143128879642 -95.57609789121557) bank37034 +37035 POINT(28.91375096933242 -94.96823731545297) bank37035 +37036 POINT(29.992994240487374 -95.62838002717888) bank37036 +37037 POINT(30.540841364704335 -95.23227918387298) bank37037 +37038 POINT(29.86466894545172 -95.17458386757056) bank37038 +37039 POINT(30.43912718702192 -95.87907595374897) bank37039 +37040 POINT(29.0488236392901 -95.87948487327809) bank37040 +37041 POINT(28.9751091645533 -96.01659699486254) bank37041 +37042 POINT(29.40891790153184 -95.67328811433069) bank37042 +37043 POINT(29.1851755101068 -95.15968746084579) bank37043 +37044 POINT(29.00148971333243 -95.72108993552688) bank37044 +37045 POINT(28.81883777367021 -95.18976429095058) bank37045 +37046 POINT(29.168415469855038 -94.40412745935768) bank37046 +37047 POINT(29.86517856439817 -95.55339689348551) bank37047 +37048 POINT(29.288035118731234 -94.61176058860286) bank37048 +37049 POINT(29.68680590416794 -95.07924701920591) bank37049 +37050 POINT(30.309501646582586 -95.66397303010011) bank37050 +37051 POINT(29.992322947598442 -94.80927854232446) bank37051 +37052 POINT(30.401684510888185 -95.70751688995001) bank37052 +37053 POINT(29.654169711447814 -94.87031002264315) bank37053 +37054 POINT(29.491234526248434 -94.56642267147932) bank37054 +37055 POINT(30.33485671490761 -96.22722088207385) bank37055 +37056 POINT(28.785289206371164 -95.02503654021412) bank37056 +37057 POINT(29.95018777931202 -96.24056575287848) bank37057 +37058 POINT(30.412163928699627 -95.79438239805081) bank37058 +37059 POINT(29.570784339633647 -95.5906550552033) bank37059 +37060 POINT(29.466373321100313 -95.16869624033897) bank37060 +37061 POINT(30.3308906720183 -96.21796349394532) bank37061 +37062 POINT(29.441997703510914 -95.67855580792404) bank37062 +37063 POINT(30.236398844932662 -95.96978774669128) bank37063 +37064 POINT(29.534357640628137 -95.50935441355969) bank37064 +37065 POINT(30.369068530422254 -94.60612008871239) bank37065 +37066 POINT(30.482389358310538 -96.2229915962747) bank37066 +37067 POINT(29.61550265583503 -94.59945396822768) bank37067 +37068 POINT(30.634319085936628 -95.58970503205971) bank37068 +37069 POINT(28.90025505201299 -95.46268714920316) bank37069 +37070 POINT(30.026268331009597 -95.07668610927014) bank37070 +37071 POINT(29.08733945640957 -94.48113789056168) bank37071 +37072 POINT(30.09083652168 -96.2652064867846) bank37072 +37073 POINT(30.57193368940347 -94.47824426115776) bank37073 +37074 POINT(29.659376177994893 -95.0990202234622) bank37074 +37075 POINT(29.8067963583532 -95.08316970034353) bank37075 +37076 POINT(30.735309604318516 -95.37885030153959) bank37076 +37077 POINT(28.935260154023695 -94.8683947188124) bank37077 +37078 POINT(30.75742193545047 -96.31351786616796) bank37078 +37079 POINT(29.627641834781805 -95.74633821074258) bank37079 +37080 POINT(29.33932956795091 -95.65560090629788) bank37080 +37081 POINT(30.146629955748978 -95.48347415847675) bank37081 +37082 POINT(29.45742249980369 -95.98964279028804) bank37082 +37083 POINT(28.996104415234 -95.68739787973628) bank37083 +37084 POINT(29.122670237889796 -96.36382932027881) bank37084 +37085 POINT(30.609497695534106 -95.9344232375192) bank37085 +37086 POINT(29.321739767458936 -96.10845415292212) bank37086 +37087 POINT(30.37184204908687 -95.12875732021284) bank37087 +37088 POINT(29.75252688413028 -95.00573061728433) bank37088 +37089 POINT(30.071014749738637 -95.31082014707408) bank37089 +37090 POINT(30.52040396625952 -95.09421657114423) bank37090 +37091 POINT(29.895751494341525 -96.24606946758537) bank37091 +37092 POINT(30.05550754852839 -95.2511212237451) bank37092 +37093 POINT(29.923561814533134 -94.45669437306485) bank37093 +37094 POINT(29.31098670847547 -94.99753499404943) bank37094 +37095 POINT(30.039206529618415 -94.47855812540297) bank37095 +37096 POINT(29.00705320430575 -95.79123067685086) bank37096 +37097 POINT(29.694008340136318 -96.09900426809696) bank37097 +37098 POINT(30.14680425837812 -95.86248091087599) bank37098 +37099 POINT(30.116966379899548 -94.48068008491715) bank37099 +37100 POINT(29.97277862604126 -95.49652993585265) bank37100 +37101 POINT(28.93614960958986 -95.64715080427736) bank37101 +37102 POINT(30.338972243686417 -94.3996861421941) bank37102 +37103 POINT(29.899914527600302 -94.78169176143102) bank37103 +37104 POINT(30.518943401342323 -95.94378662407469) bank37104 +37105 POINT(29.488455152298787 -95.43467372094094) bank37105 +37106 POINT(29.94600503654607 -95.10539890079255) bank37106 +37107 POINT(29.945344230167738 -96.25509048675332) bank37107 +37108 POINT(29.255600974168363 -94.44513992155831) bank37108 +37109 POINT(29.498858180930977 -95.74645584547076) bank37109 +37110 POINT(29.331330931202157 -95.29500723815333) bank37110 +37111 POINT(29.086043950710227 -94.75811696101239) bank37111 +37112 POINT(29.79665980806119 -95.41775861985163) bank37112 +37113 POINT(29.03035010619463 -95.80879959381049) bank37113 +37114 POINT(29.08841218475306 -94.83673598648207) bank37114 +37115 POINT(29.303759884235294 -95.36408483732585) bank37115 +37116 POINT(29.054728504581803 -94.97562979643753) bank37116 +37117 POINT(29.883091042977195 -94.6348878765568) bank37117 +37118 POINT(30.335457936754988 -96.31645866696968) bank37118 +37119 POINT(29.691546359709168 -95.33973423593866) bank37119 +37120 POINT(30.752293130179588 -95.9840104323425) bank37120 +37121 POINT(29.841767898248744 -95.72249354215799) bank37121 +37122 POINT(28.991499877025337 -95.78722959727187) bank37122 +37123 POINT(30.499660929046094 -95.3953316227652) bank37123 +37124 POINT(29.199884799437353 -95.45276309354126) bank37124 +37125 POINT(30.076205341044894 -95.85229315283644) bank37125 +37126 POINT(30.2056568695174 -94.90081230102986) bank37126 +37127 POINT(29.892142003003684 -95.97885256435194) bank37127 +37128 POINT(28.971859988347763 -95.97555840037289) bank37128 +37129 POINT(30.37135494150653 -95.61735615168577) bank37129 +37130 POINT(30.343565740512645 -96.00615942090559) bank37130 +37131 POINT(30.68546382831846 -95.84560020155175) bank37131 +37132 POINT(29.957359352425428 -95.29149989068418) bank37132 +37133 POINT(29.554731133530325 -94.4156215043262) bank37133 +37134 POINT(29.84959861055811 -95.02341209982428) bank37134 +37135 POINT(30.147774344701556 -96.11089065724671) bank37135 +37136 POINT(30.305961968001984 -94.70932600873194) bank37136 +37137 POINT(29.935903806203076 -95.16175005879008) bank37137 +37138 POINT(29.03710081908664 -95.7798002124116) bank37138 +37139 POINT(30.23620955236657 -95.01863985943895) bank37139 +37140 POINT(30.35889184643106 -96.03118146170605) bank37140 +37141 POINT(30.469005683909348 -96.05765534440052) bank37141 +37142 POINT(30.005702546270644 -95.94256297642175) bank37142 +37143 POINT(30.323826269538273 -96.09579714520834) bank37143 +37144 POINT(29.370173863427283 -94.88550745668597) bank37144 +37145 POINT(30.648894702696428 -96.34434081435025) bank37145 +37146 POINT(29.015583466721232 -94.67213159793181) bank37146 +37147 POINT(28.84525281169562 -94.80626349263524) bank37147 +37148 POINT(30.04960474954154 -95.14339109167548) bank37148 +37149 POINT(29.0237025264748 -96.35720547167173) bank37149 +37150 POINT(30.124665455564003 -94.50334993334965) bank37150 +37151 POINT(29.57033823072137 -95.39882074901757) bank37151 +37152 POINT(29.963421947357748 -94.43977218694842) bank37152 +37153 POINT(29.096021147892323 -95.61244074391641) bank37153 +37154 POINT(30.65300459266132 -95.89850631578436) bank37154 +37155 POINT(29.253673292379794 -94.50799373425738) bank37155 +37156 POINT(30.07494686456994 -94.49563642706394) bank37156 +37157 POINT(29.571275025381688 -95.47659726822052) bank37157 +37158 POINT(30.628805817715033 -95.18156016122651) bank37158 +37159 POINT(29.982030325098485 -96.2388615756919) bank37159 +37160 POINT(30.174290170769893 -95.09707988677923) bank37160 +37161 POINT(30.083324484934487 -94.6929684558064) bank37161 +37162 POINT(29.60824472459279 -94.99880667968992) bank37162 +37163 POINT(28.805873779184164 -94.90617392929202) bank37163 +37164 POINT(29.740572058056667 -94.82975655287532) bank37164 +37165 POINT(29.222389465236596 -96.35591895551057) bank37165 +37166 POINT(29.77128763772094 -94.47952889601132) bank37166 +37167 POINT(30.658765726576824 -95.74673053276425) bank37167 +37168 POINT(29.851997763196128 -95.0313542390581) bank37168 +37169 POINT(30.385804367504992 -95.99561829480052) bank37169 +37170 POINT(30.31197968175064 -96.02822422180077) bank37170 +37171 POINT(30.19390347375885 -96.15890262291889) bank37171 +37172 POINT(29.607181411940473 -95.01350116790525) bank37172 +37173 POINT(29.20388735336114 -95.91090354637299) bank37173 +37174 POINT(30.584620678764733 -95.69564225000222) bank37174 +37175 POINT(29.62656111359555 -94.53928403747948) bank37175 +37176 POINT(29.606923157137448 -95.59282348259079) bank37176 +37177 POINT(30.507476542615564 -94.64518571760188) bank37177 +37178 POINT(29.595722621366196 -96.2679434922334) bank37178 +37179 POINT(29.6896203202496 -94.43593066256777) bank37179 +37180 POINT(30.476832980166524 -94.88204055077374) bank37180 +37181 POINT(30.51229694281919 -95.04113688035956) bank37181 +37182 POINT(30.684614636980566 -94.65229205939897) bank37182 +37183 POINT(29.146178210463887 -95.50623396793867) bank37183 +37184 POINT(30.598299815911503 -95.94997031291697) bank37184 +37185 POINT(29.869366409714267 -94.99293726568158) bank37185 +37186 POINT(28.920785640555973 -94.56818925518978) bank37186 +37187 POINT(29.77669079842836 -95.92083433854864) bank37187 +37188 POINT(30.06470253585838 -96.30561538900928) bank37188 +37189 POINT(30.2386554360139 -95.4937247225422) bank37189 +37190 POINT(29.35338556110947 -94.52272979445823) bank37190 +37191 POINT(30.147040639480494 -95.29355503502909) bank37191 +37192 POINT(30.378178242386426 -94.42174419428879) bank37192 +37193 POINT(29.30299038801262 -95.44710245584467) bank37193 +37194 POINT(29.34005189336861 -96.17389509426276) bank37194 +37195 POINT(29.623510099437386 -95.70446890484865) bank37195 +37196 POINT(29.728786694269047 -96.04062384346877) bank37196 +37197 POINT(29.38411533445567 -95.49314551011832) bank37197 +37198 POINT(30.00189454096514 -94.96780957933761) bank37198 +37199 POINT(29.48432085751577 -95.06788428326873) bank37199 +37200 POINT(29.06857246837928 -95.77025801225196) bank37200 +37201 POINT(30.638221904124087 -94.40018377489726) bank37201 +37202 POINT(30.120867289788247 -95.95287619773704) bank37202 +37203 POINT(30.496414207337953 -96.1636068813947) bank37203 +37204 POINT(29.464037200462677 -94.75606131102262) bank37204 +37205 POINT(30.729831696722346 -95.52168079658468) bank37205 +37206 POINT(30.59902193026895 -95.43212903381443) bank37206 +37207 POINT(30.116549277513187 -95.0940318259991) bank37207 +37208 POINT(30.476482308462977 -95.10335517346849) bank37208 +37209 POINT(30.3574712984543 -95.16592002169185) bank37209 +37210 POINT(29.153599053927113 -96.28702571117071) bank37210 +37211 POINT(28.867182942898573 -94.69015892237859) bank37211 +37212 POINT(30.49556906902019 -95.33250593198665) bank37212 +37213 POINT(29.393408517738433 -94.39723494444884) bank37213 +37214 POINT(29.40867809674334 -94.90392545816805) bank37214 +37215 POINT(28.973815827476685 -95.04457422640641) bank37215 +37216 POINT(29.990639192475815 -95.97998429074136) bank37216 +37217 POINT(29.668303285425445 -95.09991473584367) bank37217 +37218 POINT(30.654190467658406 -95.13747179327815) bank37218 +37219 POINT(28.99392658641007 -96.36272677641277) bank37219 +37220 POINT(29.178794523358945 -95.37405402367177) bank37220 +37221 POINT(28.864610397423572 -96.1953648216924) bank37221 +37222 POINT(29.042560989179716 -95.56036982816399) bank37222 +37223 POINT(30.195665260894273 -95.5769366527818) bank37223 +37224 POINT(29.571243414082833 -95.90294258624519) bank37224 +37225 POINT(30.675901864764192 -94.55782830040904) bank37225 +37226 POINT(30.34052935143149 -96.16965935432691) bank37226 +37227 POINT(28.81303561570722 -94.89815897485765) bank37227 +37228 POINT(30.059876560768906 -95.25361759554534) bank37228 +37229 POINT(30.361926202939898 -94.8558049230219) bank37229 +37230 POINT(30.34221148745551 -94.7762995412281) bank37230 +37231 POINT(30.69022934627823 -95.02519121532445) bank37231 +37232 POINT(30.20778533312819 -95.28384176914464) bank37232 +37233 POINT(30.49473671552356 -95.50362951604637) bank37233 +37234 POINT(29.16253027497777 -96.35296219022555) bank37234 +37235 POINT(30.32760495729227 -95.9135885754311) bank37235 +37236 POINT(30.445355309407006 -94.81602098678913) bank37236 +37237 POINT(29.30352404728659 -96.3471498128306) bank37237 +37238 POINT(29.380212642536257 -95.68144195831516) bank37238 +37239 POINT(30.635900191270647 -95.25230593168608) bank37239 +37240 POINT(29.5001759106796 -96.30502419092272) bank37240 +37241 POINT(30.016236622205952 -95.7194962217888) bank37241 +37242 POINT(30.733441956683023 -94.81412029673558) bank37242 +37243 POINT(29.993159368266728 -95.04422500314041) bank37243 +37244 POINT(28.82853870131549 -95.1252250947347) bank37244 +37245 POINT(28.82436881763432 -95.33147808242917) bank37245 +37246 POINT(30.493965682325094 -95.28057899260614) bank37246 +37247 POINT(29.1956544435714 -94.462182380929) bank37247 +37248 POINT(28.965805054961102 -95.45108254056669) bank37248 +37249 POINT(29.077419787982567 -94.7539135310299) bank37249 +37250 POINT(29.83324859705673 -94.68597892924485) bank37250 +37251 POINT(29.368092489039793 -95.87745508529675) bank37251 +37252 POINT(29.64173039009646 -94.9114693377914) bank37252 +37253 POINT(30.015029982011406 -95.02120701886201) bank37253 +37254 POINT(30.555856548020852 -95.71395161580642) bank37254 +37255 POINT(30.294791349032202 -94.76023335751634) bank37255 +37256 POINT(28.84000668846452 -94.74370796813153) bank37256 +37257 POINT(29.590778657050308 -95.97488267201801) bank37257 +37258 POINT(30.655330068232587 -96.07117361227512) bank37258 +37259 POINT(29.569375279828897 -95.06999993050297) bank37259 +37260 POINT(30.087422458259674 -95.20228564506961) bank37260 +37261 POINT(28.766072689600765 -95.0460069843465) bank37261 +37262 POINT(29.992450857353084 -96.35435025352555) bank37262 +37263 POINT(29.676107852377843 -96.36185036741763) bank37263 +37264 POINT(30.1228229278045 -96.1076819124478) bank37264 +37265 POINT(29.34165600012775 -94.90198875527877) bank37265 +37266 POINT(29.91555123808029 -95.69894787579202) bank37266 +37267 POINT(29.48495398474415 -96.0109660280351) bank37267 +37268 POINT(29.304811051087054 -96.31667996231715) bank37268 +37269 POINT(29.259863244018618 -94.61541075874406) bank37269 +37270 POINT(29.06523096014142 -96.34263013664093) bank37270 +37271 POINT(30.019290340594104 -96.04167738311332) bank37271 +37272 POINT(30.189819720294054 -95.05478863424959) bank37272 +37273 POINT(29.070924686011438 -94.71670607367749) bank37273 +37274 POINT(28.807983022316925 -95.73572312305433) bank37274 +37275 POINT(29.72678115019691 -95.85829341705096) bank37275 +37276 POINT(30.36962876240042 -95.66919332574925) bank37276 +37277 POINT(29.208079161450453 -95.72694466615651) bank37277 +37278 POINT(29.78022140518729 -95.56237951825028) bank37278 +37279 POINT(29.848225655502258 -95.16634852599786) bank37279 +37280 POINT(29.604305100966798 -95.20708132602417) bank37280 +37281 POINT(29.795966776817124 -95.71406810849697) bank37281 +37282 POINT(28.82718982809182 -95.61990296330809) bank37282 +37283 POINT(29.81531995866221 -95.9429089630612) bank37283 +37284 POINT(30.55155736307788 -95.06601152592617) bank37284 +37285 POINT(29.305011903078732 -95.18029633979508) bank37285 +37286 POINT(30.119946463776515 -95.22359214558404) bank37286 +37287 POINT(29.0354287759883 -95.64838167124199) bank37287 +37288 POINT(30.3617182908159 -95.42140754528856) bank37288 +37289 POINT(29.72770226239229 -96.18503807397158) bank37289 +37290 POINT(29.499369168443906 -95.34410054077725) bank37290 +37291 POINT(30.056442797690544 -95.78917931511877) bank37291 +37292 POINT(28.97506882204618 -94.54664547239751) bank37292 +37293 POINT(30.14290471469809 -94.92733300831755) bank37293 +37294 POINT(29.269847997449236 -95.76379195789255) bank37294 +37295 POINT(29.39013984294381 -95.31824832335296) bank37295 +37296 POINT(29.09105604393098 -94.53859872611397) bank37296 +37297 POINT(30.756220961736947 -95.73701006891027) bank37297 +37298 POINT(30.077615346605963 -95.73569840818138) bank37298 +37299 POINT(29.170607109314744 -96.36296470243337) bank37299 +37300 POINT(28.892031186396004 -95.71766254163926) bank37300 +37301 POINT(29.477995018156605 -96.0710508840203) bank37301 +37302 POINT(29.430994887573277 -94.60044515180668) bank37302 +37303 POINT(29.31230243728626 -95.61894838524611) bank37303 +37304 POINT(29.216653399262846 -95.38605551409414) bank37304 +37305 POINT(30.6587936552604 -95.58797661575149) bank37305 +37306 POINT(30.2955761204703 -96.33212112062505) bank37306 +37307 POINT(30.455188505436617 -94.60822325601691) bank37307 +37308 POINT(29.27191951935504 -96.0694013137274) bank37308 +37309 POINT(29.05768305578294 -95.5336235668827) bank37309 +37310 POINT(29.203413386209935 -95.64160135541991) bank37310 +37311 POINT(29.820653867611284 -94.99107979476314) bank37311 +37312 POINT(30.646627422762418 -95.04107883168602) bank37312 +37313 POINT(29.075847971142082 -95.27493297248144) bank37313 +37314 POINT(28.918118356777267 -94.73350349146888) bank37314 +37315 POINT(30.103541959799433 -95.5106347137363) bank37315 +37316 POINT(30.67732640439125 -94.56466781153297) bank37316 +37317 POINT(30.097051477099743 -95.0054387064309) bank37317 +37318 POINT(30.223615673437465 -96.12636556200216) bank37318 +37319 POINT(30.586419477893617 -95.17191041832132) bank37319 +37320 POINT(29.161443442483574 -95.47774022463169) bank37320 +37321 POINT(29.432615137715175 -95.79359290185795) bank37321 +37322 POINT(30.597783275649377 -95.94756852197766) bank37322 +37323 POINT(30.062466666881708 -96.08460270997973) bank37323 +37324 POINT(30.251825922987525 -95.06963157076284) bank37324 +37325 POINT(29.28067236304413 -95.39937734666768) bank37325 +37326 POINT(28.977468581425832 -96.18050052894193) bank37326 +37327 POINT(29.339922338987897 -95.44885190624134) bank37327 +37328 POINT(29.077834988139504 -95.09419376986784) bank37328 +37329 POINT(30.44364865657927 -95.58949593043316) bank37329 +37330 POINT(29.04077771474373 -95.56117454773218) bank37330 +37331 POINT(28.91829858043913 -94.75054168636504) bank37331 +37332 POINT(30.349930457632393 -95.79876857992843) bank37332 +37333 POINT(29.81706004658372 -95.00476061973539) bank37333 +37334 POINT(29.992379981044035 -94.94415876060025) bank37334 +37335 POINT(29.320124314738088 -94.50997669296112) bank37335 +37336 POINT(29.87599132581384 -94.53359974465373) bank37336 +37337 POINT(29.188306236571254 -96.3290632254649) bank37337 +37338 POINT(29.269439960022783 -94.55163347203553) bank37338 +37339 POINT(30.399918248644124 -95.18069393798692) bank37339 +37340 POINT(29.03397377934876 -94.47996915512653) bank37340 +37341 POINT(29.376119891855993 -94.71496307568317) bank37341 +37342 POINT(30.386497249647196 -96.21095769371765) bank37342 +37343 POINT(29.71946782543331 -94.50335419883166) bank37343 +37344 POINT(29.675983388599622 -95.45277580856227) bank37344 +37345 POINT(29.210759977097506 -95.47681957508655) bank37345 +37346 POINT(28.908230183944546 -96.28496529667795) bank37346 +37347 POINT(29.71464596849702 -95.48545553163159) bank37347 +37348 POINT(30.460281642063265 -94.81228679292579) bank37348 +37349 POINT(29.328992588990697 -95.74893183239357) bank37349 +37350 POINT(28.854713780788053 -94.61769994213074) bank37350 +37351 POINT(28.93865945437755 -94.91836297981615) bank37351 +37352 POINT(30.619135854679794 -94.95089796660184) bank37352 +37353 POINT(30.50309757770572 -96.22597771985032) bank37353 +37354 POINT(30.589826461272626 -96.22974334844292) bank37354 +37355 POINT(29.857137837561933 -95.9638393019357) bank37355 +37356 POINT(29.398272278225754 -95.56717795098469) bank37356 +37357 POINT(29.814682772107062 -94.66232536626912) bank37357 +37358 POINT(28.769367103694485 -94.94384276583712) bank37358 +37359 POINT(30.48616359976957 -95.87818632888558) bank37359 +37360 POINT(30.491641821373353 -95.67276514621778) bank37360 +37361 POINT(29.67900236854773 -95.45288972865585) bank37361 +37362 POINT(29.660522942633037 -95.05159170697476) bank37362 +37363 POINT(28.94940102170827 -94.3945227533639) bank37363 +37364 POINT(29.130209025777486 -95.80370640347884) bank37364 +37365 POINT(30.654314951397982 -94.51054244701312) bank37365 +37366 POINT(29.344238113244764 -94.47143808109762) bank37366 +37367 POINT(28.946268659150576 -95.91545130231925) bank37367 +37368 POINT(29.92759768349642 -95.5352146466369) bank37368 +37369 POINT(29.761061694336057 -95.83616883489877) bank37369 +37370 POINT(29.499173991169748 -96.12773756420647) bank37370 +37371 POINT(29.165618018959904 -95.6966907551384) bank37371 +37372 POINT(29.100724701058976 -95.9457158878117) bank37372 +37373 POINT(29.62583029735294 -94.78200894331808) bank37373 +37374 POINT(30.382598821238787 -95.71348702217742) bank37374 +37375 POINT(28.979380099027438 -95.07395283682973) bank37375 +37376 POINT(28.91772038567318 -95.31634305793717) bank37376 +37377 POINT(30.426035618746912 -95.15094935833626) bank37377 +37378 POINT(29.01243428526354 -96.10185490249849) bank37378 +37379 POINT(29.30271190653127 -95.18489984680929) bank37379 +37380 POINT(30.414157619585858 -96.08530749762065) bank37380 +37381 POINT(30.534139312473744 -96.10737273962228) bank37381 +37382 POINT(29.458813332586963 -94.56794800161559) bank37382 +37383 POINT(28.966876386347796 -96.17108946410161) bank37383 +37384 POINT(30.715728452783924 -95.33479311817733) bank37384 +37385 POINT(29.060051765183808 -95.06768747465009) bank37385 +37386 POINT(28.954981918213818 -96.36618923066553) bank37386 +37387 POINT(30.325740491848144 -95.71740082898899) bank37387 +37388 POINT(30.024981172361883 -94.45952512600226) bank37388 +37389 POINT(30.670172433284907 -95.39495166204453) bank37389 +37390 POINT(30.092479801103394 -95.76243335275457) bank37390 +37391 POINT(30.3183232762548 -95.28982675574449) bank37391 +37392 POINT(28.77013333119002 -95.84643564743294) bank37392 +37393 POINT(29.113894940799714 -95.5395226176556) bank37393 +37394 POINT(29.76040182805831 -95.5245625078527) bank37394 +37395 POINT(30.559506189304532 -96.23656371200533) bank37395 +37396 POINT(28.931728322120186 -94.61437177605032) bank37396 +37397 POINT(30.742332406871682 -95.51848335627639) bank37397 +37398 POINT(29.331712109125966 -96.03346811049913) bank37398 +37399 POINT(28.846383358236434 -95.995189904277) bank37399 +37400 POINT(28.858915431334754 -94.61998259623097) bank37400 +37401 POINT(30.19244138728128 -94.84935173059552) bank37401 +37402 POINT(30.54182054082064 -96.05880867095142) bank37402 +37403 POINT(30.046537536612576 -95.84780905304987) bank37403 +37404 POINT(29.586200438206404 -95.33053933145187) bank37404 +37405 POINT(30.54906930496762 -94.68492289260651) bank37405 +37406 POINT(30.053908374140246 -95.64641224872196) bank37406 +37407 POINT(29.241759400140282 -95.09772248023415) bank37407 +37408 POINT(29.163989495095997 -94.86658152609853) bank37408 +37409 POINT(29.529072723050206 -95.07586514060145) bank37409 +37410 POINT(29.757473524068807 -94.6458256064487) bank37410 +37411 POINT(30.259127266906315 -95.52516860822271) bank37411 +37412 POINT(29.125830903935803 -95.57760677115374) bank37412 +37413 POINT(30.590581566026792 -94.85342703735422) bank37413 +37414 POINT(30.747510311883033 -95.85002921972057) bank37414 +37415 POINT(29.745002852792275 -95.95731089971785) bank37415 +37416 POINT(30.698757077005332 -96.06861422831668) bank37416 +37417 POINT(30.66681867315472 -94.71566999589733) bank37417 +37418 POINT(28.92752035952149 -94.93545082085814) bank37418 +37419 POINT(29.123167673533004 -94.68087000691392) bank37419 +37420 POINT(28.981992778729005 -94.51227958895805) bank37420 +37421 POINT(30.293166253060544 -94.76128436824624) bank37421 +37422 POINT(29.36204177112281 -95.1496607184691) bank37422 +37423 POINT(30.18282564281138 -95.51653158652029) bank37423 +37424 POINT(30.04620864208141 -95.16597128278224) bank37424 +37425 POINT(29.140097796820655 -96.05951735915582) bank37425 +37426 POINT(29.80952920232284 -95.85131713143122) bank37426 +37427 POINT(29.728638502226065 -95.76338686621122) bank37427 +37428 POINT(29.887679168584793 -95.72342518472071) bank37428 +37429 POINT(30.656193932280814 -95.0228114301484) bank37429 +37430 POINT(29.154185853022103 -94.63926980290856) bank37430 +37431 POINT(29.477282049030723 -96.00089587997991) bank37431 +37432 POINT(30.013765270554252 -95.87118313626804) bank37432 +37433 POINT(30.339059660825363 -96.17025846764803) bank37433 +37434 POINT(29.780917561719413 -95.22421391928306) bank37434 +37435 POINT(28.79473840088263 -94.45059823013574) bank37435 +37436 POINT(29.780755602937752 -94.53148169953175) bank37436 +37437 POINT(28.89727004870774 -96.07542039938251) bank37437 +37438 POINT(28.871863373904613 -94.8012954679167) bank37438 +37439 POINT(28.797500577050947 -95.07529441943714) bank37439 +37440 POINT(29.07054554469094 -94.49129570311884) bank37440 +37441 POINT(30.486424573639823 -94.53944394613238) bank37441 +37442 POINT(29.15065944316426 -95.05910351817747) bank37442 +37443 POINT(30.271464875224947 -96.1297398815032) bank37443 +37444 POINT(28.92369783941901 -94.75772190350776) bank37444 +37445 POINT(29.987862001385484 -96.09208386266161) bank37445 +37446 POINT(30.065733671476103 -94.57497938380878) bank37446 +37447 POINT(30.47544178160499 -94.59703567150459) bank37447 +37448 POINT(30.165528334255043 -95.69168683607609) bank37448 +37449 POINT(30.25754318636784 -96.3567949030021) bank37449 +37450 POINT(28.938880942841006 -95.52167332684868) bank37450 +37451 POINT(29.570403038752115 -94.57869675851205) bank37451 +37452 POINT(28.873137457361654 -95.40071011983622) bank37452 +37453 POINT(29.070558027680846 -96.13860857372744) bank37453 +37454 POINT(29.20316128621112 -94.62645135795759) bank37454 +37455 POINT(30.15324131760264 -96.14161109832736) bank37455 +37456 POINT(30.680696848316416 -95.02317994261034) bank37456 +37457 POINT(30.74716989932375 -95.06200703965223) bank37457 +37458 POINT(29.531123636399276 -95.87225023360048) bank37458 +37459 POINT(30.561248728992847 -96.2533453411451) bank37459 +37460 POINT(29.6709826811052 -96.22687683299337) bank37460 +37461 POINT(30.46788024409822 -95.65198555012658) bank37461 +37462 POINT(30.374260372810767 -95.6774202244992) bank37462 +37463 POINT(29.68811965175377 -95.42984976415018) bank37463 +37464 POINT(29.687927944265297 -95.37797531568062) bank37464 +37465 POINT(28.994683705236156 -95.08727891238324) bank37465 +37466 POINT(29.741377118457496 -94.92793481779465) bank37466 +37467 POINT(29.755766906131115 -94.52028383181685) bank37467 +37468 POINT(28.98442245157709 -94.43767826504987) bank37468 +37469 POINT(30.1680049030588 -95.15351236340723) bank37469 +37470 POINT(29.02879810556185 -95.68196949570604) bank37470 +37471 POINT(30.319868148326872 -95.67525671877571) bank37471 +37472 POINT(30.15001394502741 -94.58524888932911) bank37472 +37473 POINT(30.636143099689917 -94.69046015475698) bank37473 +37474 POINT(30.583740501078186 -94.67327509581433) bank37474 +37475 POINT(29.337107347657195 -94.56565312693505) bank37475 +37476 POINT(30.751177269799552 -94.94243645333373) bank37476 +37477 POINT(30.56946263687664 -95.44390303859527) bank37477 +37478 POINT(29.27495253206549 -94.48624225591708) bank37478 +37479 POINT(29.158193090987826 -96.03334446351073) bank37479 +37480 POINT(30.68933260574944 -96.18148804155676) bank37480 +37481 POINT(29.19727522459157 -95.02323773649015) bank37481 +37482 POINT(30.751839631898527 -95.68712818206649) bank37482 +37483 POINT(30.623674929792852 -95.2243596030495) bank37483 +37484 POINT(30.289295326773246 -95.09036381692933) bank37484 +37485 POINT(30.281550675224864 -95.11188995130408) bank37485 +37486 POINT(30.66316766762904 -94.79497013415921) bank37486 +37487 POINT(30.70391807710933 -94.76745732659306) bank37487 +37488 POINT(30.173349552026853 -95.41937222324522) bank37488 +37489 POINT(30.39001658313315 -95.44819428271754) bank37489 +37490 POINT(29.520661286135127 -95.53035083046993) bank37490 +37491 POINT(29.34987704367703 -95.84322016666593) bank37491 +37492 POINT(28.963622834746204 -95.68762904649545) bank37492 +37493 POINT(28.98409391043794 -95.03271946695493) bank37493 +37494 POINT(29.154231057493156 -96.14802840977) bank37494 +37495 POINT(29.753515543218093 -94.69298896342981) bank37495 +37496 POINT(29.09968083384778 -95.68222732142631) bank37496 +37497 POINT(30.622474805831253 -95.70312182351901) bank37497 +37498 POINT(29.159964241039464 -94.79820188760357) bank37498 +37499 POINT(30.213958820734604 -95.6610950589907) bank37499 +37500 POINT(29.98525768545166 -94.87522950166236) bank37500 +37501 POINT(29.600657553858277 -94.8889839298607) bank37501 +37502 POINT(29.965814722656603 -94.65421766019313) bank37502 +37503 POINT(29.860373339774824 -95.38523766150365) bank37503 +37504 POINT(29.756894172938903 -96.14837083423879) bank37504 +37505 POINT(28.81779931624569 -94.98254691562481) bank37505 +37506 POINT(28.83277600777159 -95.58674861188564) bank37506 +37507 POINT(29.712974713868917 -95.05327583965804) bank37507 +37508 POINT(29.078992842528233 -95.68417518425763) bank37508 +37509 POINT(29.949159517847235 -95.88643855210238) bank37509 +37510 POINT(28.849846062259395 -95.47943400662436) bank37510 +37511 POINT(29.96573313480478 -96.18592847287273) bank37511 +37512 POINT(30.071775126688003 -95.10568942436238) bank37512 +37513 POINT(29.984133789612248 -95.64708697022984) bank37513 +37514 POINT(29.04031570018247 -94.60406559075724) bank37514 +37515 POINT(29.769010293731974 -95.63286593964027) bank37515 +37516 POINT(30.49175467923748 -94.47845029815535) bank37516 +37517 POINT(28.967127718954483 -95.7254701749133) bank37517 +37518 POINT(30.64612118335883 -94.87307692699893) bank37518 +37519 POINT(30.10155120126159 -94.60161658291155) bank37519 +37520 POINT(30.05109295060069 -96.0155741809261) bank37520 +37521 POINT(29.39567610284713 -95.52963990615969) bank37521 +37522 POINT(29.665882622341204 -95.10968601888567) bank37522 +37523 POINT(29.72732837305948 -95.60030486365596) bank37523 +37524 POINT(29.764152660113947 -94.9584048197572) bank37524 +37525 POINT(30.533398136256857 -95.48126343055465) bank37525 +37526 POINT(30.67593081109097 -94.76905629047356) bank37526 +37527 POINT(29.139345664592724 -94.55786632298411) bank37527 +37528 POINT(29.03951745871503 -95.0709639357393) bank37528 +37529 POINT(30.617919328859198 -96.18250590678264) bank37529 +37530 POINT(29.10870951189887 -96.2193270875225) bank37530 +37531 POINT(30.518959432824655 -94.73212287809253) bank37531 +37532 POINT(29.527217682802068 -95.06326980384165) bank37532 +37533 POINT(30.29444740636713 -96.23219277271689) bank37533 +37534 POINT(30.265678678030035 -95.99161389351649) bank37534 +37535 POINT(29.535943544246088 -95.56404540812865) bank37535 +37536 POINT(30.353774895190305 -95.0880812397803) bank37536 +37537 POINT(29.97360240348122 -95.08761693858355) bank37537 +37538 POINT(30.509333972454552 -95.51817579480829) bank37538 +37539 POINT(30.02250359407892 -94.54262821719038) bank37539 +37540 POINT(28.881871852887784 -96.20487361713661) bank37540 +37541 POINT(29.496653931838367 -95.24455376287798) bank37541 +37542 POINT(29.647731790610614 -94.82818088860344) bank37542 +37543 POINT(30.102046381574265 -94.61004814184427) bank37543 +37544 POINT(30.30723540542843 -94.5870532840979) bank37544 +37545 POINT(29.421410567614952 -96.32980272869132) bank37545 +37546 POINT(29.269396984195033 -95.2376739953184) bank37546 +37547 POINT(29.51661132975692 -95.44087207225249) bank37547 +37548 POINT(29.329391163257327 -95.30987506266008) bank37548 +37549 POINT(29.481953091873248 -95.01301114886991) bank37549 +37550 POINT(29.9792743621355 -94.93947690937058) bank37550 +37551 POINT(30.706293468347557 -94.42891553566757) bank37551 +37552 POINT(30.54624991238653 -95.73747711214902) bank37552 +37553 POINT(30.63814061357381 -95.75098232033169) bank37553 +37554 POINT(30.060806739801443 -96.14410869640193) bank37554 +37555 POINT(30.510623045945557 -94.64124445824983) bank37555 +37556 POINT(29.972206369770447 -95.2337345218011) bank37556 +37557 POINT(30.021175330425063 -95.59805654058582) bank37557 +37558 POINT(28.98777356941082 -96.21941402201315) bank37558 +37559 POINT(29.52578258540657 -95.2895277592784) bank37559 +37560 POINT(29.319061530872244 -94.48950070369591) bank37560 +37561 POINT(29.66384509532272 -95.11694724104858) bank37561 +37562 POINT(30.321949785385335 -96.23216950587567) bank37562 +37563 POINT(29.90484107434729 -95.05781830955169) bank37563 +37564 POINT(30.645292994543556 -95.56347138974499) bank37564 +37565 POINT(30.72627856255949 -95.67937614981521) bank37565 +37566 POINT(29.254845824605113 -95.844701241319) bank37566 +37567 POINT(29.383316872801416 -96.25050314585935) bank37567 +37568 POINT(29.437037892719555 -94.5152503245548) bank37568 +37569 POINT(30.168246572636342 -94.44153005759014) bank37569 +37570 POINT(29.240913640988072 -95.97507247036548) bank37570 +37571 POINT(30.061880576727386 -95.26780982147255) bank37571 +37572 POINT(30.715700122584575 -95.31372378524922) bank37572 +37573 POINT(30.53216941870656 -95.64452529900177) bank37573 +37574 POINT(29.14435083258079 -96.07845758962603) bank37574 +37575 POINT(29.97262573552367 -94.43197750867994) bank37575 +37576 POINT(28.955961796748024 -94.92800302234049) bank37576 +37577 POINT(29.411458074684376 -96.17403606901365) bank37577 +37578 POINT(29.591182602743746 -95.6784541066584) bank37578 +37579 POINT(30.466819164502887 -95.65404903429312) bank37579 +37580 POINT(29.0365852866247 -96.15331953286994) bank37580 +37581 POINT(29.158561279189758 -94.9347540732476) bank37581 +37582 POINT(30.51367566365917 -94.91042025861707) bank37582 +37583 POINT(29.274089348391175 -95.99338795934194) bank37583 +37584 POINT(29.37124035994818 -94.59535691189988) bank37584 +37585 POINT(28.89320460333923 -95.08115685332744) bank37585 +37586 POINT(29.35367130688695 -94.67868972693923) bank37586 +37587 POINT(29.82298511785971 -95.11202821331543) bank37587 +37588 POINT(30.657760012104205 -95.22347969402426) bank37588 +37589 POINT(29.28235659931629 -95.03721374416864) bank37589 +37590 POINT(30.13683886995977 -96.01870194547335) bank37590 +37591 POINT(29.273185991522485 -95.61386849063825) bank37591 +37592 POINT(29.72059423200316 -94.45243940071322) bank37592 +37593 POINT(29.437436359345334 -95.04566667921696) bank37593 +37594 POINT(28.950431744988677 -94.90802135266736) bank37594 +37595 POINT(29.64371955596438 -94.91906234803517) bank37595 +37596 POINT(29.898759697269043 -94.49786801826454) bank37596 +37597 POINT(29.424432258227174 -95.5804999488301) bank37597 +37598 POINT(29.58812327782645 -95.27139407089403) bank37598 +37599 POINT(30.094353372195695 -95.4988766079161) bank37599 +37600 POINT(30.45399312135228 -94.527331176134) bank37600 +37601 POINT(29.88561547595164 -95.48622241923526) bank37601 +37602 POINT(29.59248797819827 -96.01090507425843) bank37602 +37603 POINT(28.99904891240689 -94.7630381075454) bank37603 +37604 POINT(30.087866086824466 -94.47350024551895) bank37604 +37605 POINT(30.36691296502868 -94.61788666453798) bank37605 +37606 POINT(30.17897546003104 -94.54325007709313) bank37606 +37607 POINT(30.578839890047103 -94.71485304501805) bank37607 +37608 POINT(29.340476494250822 -95.03055672413593) bank37608 +37609 POINT(29.37644313181863 -96.01172454035716) bank37609 +37610 POINT(30.42210619629497 -95.15973982515376) bank37610 +37611 POINT(28.835917744596006 -94.5565192655328) bank37611 +37612 POINT(30.487149862970224 -95.95884114831367) bank37612 +37613 POINT(29.287553499235752 -94.57994522699882) bank37613 +37614 POINT(29.29957427427301 -94.52085241380365) bank37614 +37615 POINT(30.59526290477583 -95.93679355663687) bank37615 +37616 POINT(29.50984576055393 -95.26552026283706) bank37616 +37617 POINT(29.07123162899518 -95.68433956210092) bank37617 +37618 POINT(30.11744342680574 -94.92813985614812) bank37618 +37619 POINT(29.09406378118491 -94.9096816965448) bank37619 +37620 POINT(29.335438679766874 -96.17270583876764) bank37620 +37621 POINT(29.174577700742983 -95.14560315327827) bank37621 +37622 POINT(30.681566442872107 -94.6872095973271) bank37622 +37623 POINT(30.2660624074654 -95.00505548423162) bank37623 +37624 POINT(30.350748422851794 -95.13854864141067) bank37624 +37625 POINT(30.140499961333518 -94.90557656695572) bank37625 +37626 POINT(29.48978023872444 -96.26630113150597) bank37626 +37627 POINT(30.21424665284694 -95.03213600788366) bank37627 +37628 POINT(30.105393114238662 -96.23081297024422) bank37628 +37629 POINT(29.289654217383383 -95.20186246999864) bank37629 +37630 POINT(29.062735105019705 -94.9551959651431) bank37630 +37631 POINT(30.466472029203416 -95.58524935708394) bank37631 +37632 POINT(29.19253593439086 -95.93766363398868) bank37632 +37633 POINT(29.781589403660757 -95.27248715735556) bank37633 +37634 POINT(30.35417527689404 -95.17199948866791) bank37634 +37635 POINT(29.03718976231276 -94.71481773816424) bank37635 +37636 POINT(29.740202909329586 -95.17721465521511) bank37636 +37637 POINT(30.397264362516864 -94.77473658649666) bank37637 +37638 POINT(29.63679013408874 -95.00019585149312) bank37638 +37639 POINT(30.24370101321429 -95.89232713482231) bank37639 +37640 POINT(29.651509301936514 -96.19068555599779) bank37640 +37641 POINT(29.034642360956468 -95.26836781665054) bank37641 +37642 POINT(28.849624462366563 -94.99576955100508) bank37642 +37643 POINT(28.83965227950661 -96.00780605278082) bank37643 +37644 POINT(29.27534081215391 -94.72280920636341) bank37644 +37645 POINT(28.797659008317098 -95.05924290061587) bank37645 +37646 POINT(29.96830620651872 -95.11498682396535) bank37646 +37647 POINT(30.17391365891294 -94.44584753945554) bank37647 +37648 POINT(30.324361526869797 -96.0574483972528) bank37648 +37649 POINT(30.607964103130396 -95.80340793952031) bank37649 +37650 POINT(29.45715799544729 -96.1724754983133) bank37650 +37651 POINT(30.704934002533193 -95.5082284770982) bank37651 +37652 POINT(30.70061837831303 -95.49894810927573) bank37652 +37653 POINT(29.018711960010883 -94.52779104359327) bank37653 +37654 POINT(30.230182374775886 -96.11037051876693) bank37654 +37655 POINT(30.297049787958 -94.71144901382529) bank37655 +37656 POINT(29.747635129997143 -95.10450113682559) bank37656 +37657 POINT(29.38038735705508 -95.91634634517034) bank37657 +37658 POINT(30.396933153192116 -95.56017340857497) bank37658 +37659 POINT(28.872398034302194 -95.17172228077702) bank37659 +37660 POINT(29.272586067833423 -95.40821779211682) bank37660 +37661 POINT(28.77352145368342 -94.76473313296079) bank37661 +37662 POINT(29.603642616738316 -95.47505440854087) bank37662 +37663 POINT(29.380246343760202 -95.50015639874435) bank37663 +37664 POINT(29.525409244270424 -95.21834458759531) bank37664 +37665 POINT(29.977387812355456 -95.7182055045686) bank37665 +37666 POINT(28.855000925327044 -95.65099369816353) bank37666 +37667 POINT(30.35329524468897 -94.46748354582594) bank37667 +37668 POINT(29.359650248699847 -95.88386036977444) bank37668 +37669 POINT(29.434747324395488 -95.68295920998585) bank37669 +37670 POINT(29.516023833184615 -96.25112104674845) bank37670 +37671 POINT(29.58764592194134 -94.8778729935893) bank37671 +37672 POINT(30.48618098582184 -95.10677587236152) bank37672 +37673 POINT(29.76771176052173 -95.38694619817961) bank37673 +37674 POINT(30.270912655394085 -94.64325643639222) bank37674 +37675 POINT(30.654254049943013 -96.32210675711532) bank37675 +37676 POINT(29.407702889102865 -94.872737522289) bank37676 +37677 POINT(29.51503092959421 -95.96892565806772) bank37677 +37678 POINT(29.549726835195706 -95.074952257773) bank37678 +37679 POINT(30.098563002349746 -96.24638971490981) bank37679 +37680 POINT(30.59620646702989 -94.5301093888119) bank37680 +37681 POINT(29.63154257412411 -94.40191345966046) bank37681 +37682 POINT(30.647763366757413 -95.61382221634638) bank37682 +37683 POINT(30.74004797813345 -94.69682879876768) bank37683 +37684 POINT(29.00878248756985 -94.85883088154223) bank37684 +37685 POINT(29.90543308465404 -95.07924117254879) bank37685 +37686 POINT(29.06037083265575 -96.10505650039543) bank37686 +37687 POINT(29.453065627808854 -94.70723465167052) bank37687 +37688 POINT(29.45177639044247 -95.55238987988031) bank37688 +37689 POINT(29.64767529101151 -94.87298150848466) bank37689 +37690 POINT(29.667653846452588 -95.3122300217989) bank37690 +37691 POINT(29.079786207995458 -95.80819968542727) bank37691 +37692 POINT(28.93574779418346 -96.3509038250049) bank37692 +37693 POINT(29.869975070256388 -95.24823045707305) bank37693 +37694 POINT(30.103215035932024 -94.99673991034734) bank37694 +37695 POINT(29.39819859765118 -96.34375529808995) bank37695 +37696 POINT(29.093576647487005 -96.33681964220914) bank37696 +37697 POINT(29.070915543404677 -95.66127834687717) bank37697 +37698 POINT(29.951718758049005 -94.76031645654209) bank37698 +37699 POINT(30.627791391741212 -95.25040606535589) bank37699 +37700 POINT(30.054062691142448 -95.78703849252486) bank37700 +37701 POINT(29.603805204255995 -95.33717626981428) bank37701 +37702 POINT(29.872783281553037 -95.3580028743722) bank37702 +37703 POINT(30.154011523867425 -95.2949933737632) bank37703 +37704 POINT(29.723921392864916 -95.13631432708937) bank37704 +37705 POINT(29.470580975556434 -95.49025381462845) bank37705 +37706 POINT(29.632570687107396 -95.21468672527155) bank37706 +37707 POINT(29.88537901196964 -95.03487592339536) bank37707 +37708 POINT(30.669248022327505 -95.87582738857846) bank37708 +37709 POINT(30.003758735543382 -94.92142915810324) bank37709 +37710 POINT(29.813981599938447 -95.81028629753655) bank37710 +37711 POINT(30.27854181274087 -95.42646320498994) bank37711 +37712 POINT(29.45454259562545 -95.74113672803318) bank37712 +37713 POINT(29.609726958603073 -95.68681132107471) bank37713 +37714 POINT(29.614893709244328 -95.15490256177907) bank37714 +37715 POINT(29.903825986867236 -96.29507306129038) bank37715 +37716 POINT(30.062554111537203 -95.75188720444216) bank37716 +37717 POINT(29.32816905399413 -96.12899667551072) bank37717 +37718 POINT(29.663587857519722 -95.4987519756616) bank37718 +37719 POINT(30.538986503151676 -94.95955072351039) bank37719 +37720 POINT(28.78084769186934 -95.12763990671016) bank37720 +37721 POINT(30.18639412954392 -95.53273549048626) bank37721 +37722 POINT(30.57625735491335 -94.37719516410336) bank37722 +37723 POINT(28.863321356937252 -96.26367728016152) bank37723 +37724 POINT(30.16886244357528 -95.19851380125748) bank37724 +37725 POINT(29.730807530456165 -94.7462249075216) bank37725 +37726 POINT(29.820627079839593 -95.70173664790252) bank37726 +37727 POINT(30.101051055395658 -95.32722568013054) bank37727 +37728 POINT(28.821959336283957 -94.63133083650878) bank37728 +37729 POINT(28.85976468217002 -95.27276236270579) bank37729 +37730 POINT(28.862077645537962 -94.58481392914462) bank37730 +37731 POINT(29.074184208442546 -94.63591147332106) bank37731 +37732 POINT(30.280322849592885 -95.22906432195647) bank37732 +37733 POINT(30.468892212484096 -94.47597601305614) bank37733 +37734 POINT(29.47394706998848 -95.76110802368461) bank37734 +37735 POINT(30.274028395778767 -95.0424070932441) bank37735 +37736 POINT(29.62216210024556 -95.69196371284745) bank37736 +37737 POINT(29.027049258008766 -95.68901915118327) bank37737 +37738 POINT(28.95725043621482 -95.00942423214009) bank37738 +37739 POINT(29.793116710707903 -94.78899369066552) bank37739 +37740 POINT(30.390521793755486 -94.37051809372782) bank37740 +37741 POINT(30.327099966483857 -95.91196145930543) bank37741 +37742 POINT(29.330288675983173 -95.23436756306823) bank37742 +37743 POINT(29.774542972747014 -95.8751266850719) bank37743 +37744 POINT(30.529026046361363 -95.67037289755883) bank37744 +37745 POINT(30.249327470558853 -96.0207876449655) bank37745 +37746 POINT(30.081054931521386 -94.77973363131079) bank37746 +37747 POINT(29.22536788279659 -94.42304473583667) bank37747 +37748 POINT(30.21688235058252 -95.6969184166727) bank37748 +37749 POINT(29.429514852534865 -94.85139467736485) bank37749 +37750 POINT(29.7965123827969 -94.9264589042713) bank37750 +37751 POINT(29.753563342630404 -96.21576342804362) bank37751 +37752 POINT(29.158912967894963 -96.15803377561328) bank37752 +37753 POINT(30.35453049145543 -95.7391145863812) bank37753 +37754 POINT(30.70304696031722 -95.30866194827082) bank37754 +37755 POINT(29.25032376675529 -95.99211222837674) bank37755 +37756 POINT(30.423226622610805 -95.64980664812119) bank37756 +37757 POINT(29.20996351810194 -94.51485282236432) bank37757 +37758 POINT(29.39057352879487 -94.58061210013226) bank37758 +37759 POINT(29.35232708294592 -95.52463719515173) bank37759 +37760 POINT(28.82416658957972 -95.81821202018632) bank37760 +37761 POINT(29.12130619243902 -95.15061854566561) bank37761 +37762 POINT(30.482792233404332 -94.8307632511923) bank37762 +37763 POINT(30.672471820657535 -94.53454488251072) bank37763 +37764 POINT(29.453634443152477 -95.57444735509331) bank37764 +37765 POINT(30.250771378150095 -95.98621066207423) bank37765 +37766 POINT(30.470986210261565 -96.0823650038491) bank37766 +37767 POINT(30.34161983592954 -94.52649817764348) bank37767 +37768 POINT(28.9443055009339 -95.01273884269584) bank37768 +37769 POINT(28.991076760654114 -95.61147124349908) bank37769 +37770 POINT(29.134012374626284 -96.10538224662565) bank37770 +37771 POINT(30.58585660341004 -95.55355440561604) bank37771 +37772 POINT(30.212415913721166 -95.6985192007933) bank37772 +37773 POINT(28.99928161722354 -95.29168899333368) bank37773 +37774 POINT(29.983664998173815 -96.17235881325226) bank37774 +37775 POINT(28.94809638320163 -95.17284896976916) bank37775 +37776 POINT(30.39318591444452 -95.41907009827459) bank37776 +37777 POINT(29.14305274043965 -94.37042386593522) bank37777 +37778 POINT(30.150750070632018 -95.17744509828242) bank37778 +37779 POINT(29.018916756162003 -94.69261509470692) bank37779 +37780 POINT(30.035976174948377 -95.20559363914042) bank37780 +37781 POINT(30.19075448261402 -96.26893803137095) bank37781 +37782 POINT(28.80938716016512 -95.04162746119125) bank37782 +37783 POINT(30.73392986092402 -94.48095949019991) bank37783 +37784 POINT(29.934098167403533 -94.70173733172597) bank37784 +37785 POINT(29.750219034791055 -96.3501444091312) bank37785 +37786 POINT(29.952446818637615 -96.32776840645903) bank37786 +37787 POINT(29.584165498958765 -95.97125688740446) bank37787 +37788 POINT(29.978487789416036 -95.94501081725655) bank37788 +37789 POINT(29.093252703131476 -94.8930057458709) bank37789 +37790 POINT(30.46105903735574 -94.68376772396658) bank37790 +37791 POINT(30.7175472243481 -95.96967092172468) bank37791 +37792 POINT(29.2749888666149 -94.59591625884265) bank37792 +37793 POINT(29.865566940667186 -95.23409664400431) bank37793 +37794 POINT(29.005102743682414 -96.11278158607607) bank37794 +37795 POINT(29.34561896371537 -94.79532973211963) bank37795 +37796 POINT(29.75777928342961 -94.88337736812619) bank37796 +37797 POINT(29.79750807586933 -94.45838523281078) bank37797 +37798 POINT(29.507456992635202 -95.12978238381174) bank37798 +37799 POINT(29.241131503580736 -94.55006214698938) bank37799 +37800 POINT(29.725379434600754 -94.69537253778577) bank37800 +37801 POINT(29.834656878098635 -94.72558254843763) bank37801 +37802 POINT(29.637776693910986 -94.80227386003514) bank37802 +37803 POINT(28.988815801861804 -94.80474962959788) bank37803 +37804 POINT(29.194568451755377 -94.56431721312626) bank37804 +37805 POINT(30.05222194569668 -96.36784359163197) bank37805 +37806 POINT(29.090147873453155 -96.30919492946565) bank37806 +37807 POINT(29.068882871055628 -94.68572005056272) bank37807 +37808 POINT(30.71953432057598 -94.581782142268) bank37808 +37809 POINT(29.80229260955467 -96.31012668566952) bank37809 +37810 POINT(30.106471274155574 -95.70559588645752) bank37810 +37811 POINT(29.47775343278955 -94.57960353282826) bank37811 +37812 POINT(29.875505470916362 -94.47036002741632) bank37812 +37813 POINT(29.307640411880268 -94.37171083505295) bank37813 +37814 POINT(30.246401062304788 -95.12509627690059) bank37814 +37815 POINT(29.220410667318237 -95.24362337029885) bank37815 +37816 POINT(28.86655577881189 -95.29078393961022) bank37816 +37817 POINT(29.710903503549837 -95.06802819486987) bank37817 +37818 POINT(29.450782997563472 -95.4536461654166) bank37818 +37819 POINT(30.349120535026803 -95.02039425131215) bank37819 +37820 POINT(30.59251910359888 -95.45080394838601) bank37820 +37821 POINT(29.74292906612253 -95.6730697514884) bank37821 +37822 POINT(28.790605151062554 -94.51515593468646) bank37822 +37823 POINT(29.43201752325926 -95.97999270739423) bank37823 +37824 POINT(30.117344295930568 -95.53567450047709) bank37824 +37825 POINT(29.477829247418196 -95.01402370475675) bank37825 +37826 POINT(30.40921287631859 -96.23093175803315) bank37826 +37827 POINT(29.624399442339612 -95.3561747450236) bank37827 +37828 POINT(30.69242756944854 -95.0325187733641) bank37828 +37829 POINT(28.802718011665355 -95.72236767101067) bank37829 +37830 POINT(29.128661715211646 -95.18600855052331) bank37830 +37831 POINT(29.36519173836546 -95.92389413011917) bank37831 +37832 POINT(28.8111298067792 -94.76658647829818) bank37832 +37833 POINT(29.884615318905215 -94.96701367369009) bank37833 +37834 POINT(29.21654568059976 -94.9714731090377) bank37834 +37835 POINT(29.842062487110567 -95.25705397300437) bank37835 +37836 POINT(29.349127778352944 -95.22512159958464) bank37836 +37837 POINT(30.51596111363964 -96.28054356279095) bank37837 +37838 POINT(29.54593807950478 -95.22156287351254) bank37838 +37839 POINT(28.952263724086784 -94.8509670580866) bank37839 +37840 POINT(30.216833878340186 -94.88117833182929) bank37840 +37841 POINT(29.388886154342938 -94.5229188837346) bank37841 +37842 POINT(29.098724846475886 -96.23925706505058) bank37842 +37843 POINT(29.0045416556832 -95.25992425906642) bank37843 +37844 POINT(29.123240633633202 -95.54350050820695) bank37844 +37845 POINT(29.020452108886943 -96.07682164571793) bank37845 +37846 POINT(29.023807766914334 -95.45821717791148) bank37846 +37847 POINT(28.830845414168977 -96.12205991762816) bank37847 +37848 POINT(30.649467643561483 -94.76216432966983) bank37848 +37849 POINT(30.366493757716448 -95.39702510792672) bank37849 +37850 POINT(29.795189943848413 -95.20582569509158) bank37850 +37851 POINT(29.47916394117739 -94.53794885639185) bank37851 +37852 POINT(30.673289040321645 -95.16646150093993) bank37852 +37853 POINT(29.077558350562384 -95.04346432958228) bank37853 +37854 POINT(30.29524466272302 -95.65730949513883) bank37854 +37855 POINT(29.95992131486309 -95.8800088102064) bank37855 +37856 POINT(29.84008375504662 -94.96326523147054) bank37856 +37857 POINT(30.6660817447081 -94.67296185639013) bank37857 +37858 POINT(30.003849963779153 -94.49028922983715) bank37858 +37859 POINT(29.889497431428882 -96.0216799582528) bank37859 +37860 POINT(30.520050277015493 -95.32237420580464) bank37860 +37861 POINT(30.639346613587104 -94.82000687625413) bank37861 +37862 POINT(29.697727417421948 -96.28965004227211) bank37862 +37863 POINT(29.195218143930326 -95.56575825541414) bank37863 +37864 POINT(28.828467856264954 -94.63441382545491) bank37864 +37865 POINT(29.683118848838365 -94.56038244843047) bank37865 +37866 POINT(29.078345339674836 -94.91407673369679) bank37866 +37867 POINT(30.66250665497465 -95.39084057477463) bank37867 +37868 POINT(28.7903619525887 -94.61440888936893) bank37868 +37869 POINT(29.617654921631807 -94.96465146667892) bank37869 +37870 POINT(29.326298284103604 -94.87581469579547) bank37870 +37871 POINT(29.637883244945744 -95.82846198109308) bank37871 +37872 POINT(30.305991051224336 -96.17361395112245) bank37872 +37873 POINT(29.960494930685112 -96.34096224771618) bank37873 +37874 POINT(29.496751627309962 -94.53295930442052) bank37874 +37875 POINT(29.394306644335497 -96.05416808784493) bank37875 +37876 POINT(30.74542414733992 -95.23179957602628) bank37876 +37877 POINT(29.32826076322169 -94.8447037006923) bank37877 +37878 POINT(30.06637528837141 -94.59344024893251) bank37878 +37879 POINT(30.095827874599077 -95.65427530073384) bank37879 +37880 POINT(29.574067343711842 -94.38059680301157) bank37880 +37881 POINT(30.661764857058753 -95.75238525723121) bank37881 +37882 POINT(30.645027395934704 -96.0087249459815) bank37882 +37883 POINT(28.88794567249344 -96.24400488790306) bank37883 +37884 POINT(30.281608452555997 -94.45434001168262) bank37884 +37885 POINT(29.6807440481548 -95.13409755588576) bank37885 +37886 POINT(30.305912155174564 -96.35108096866887) bank37886 +37887 POINT(29.655857948176156 -94.95240669520759) bank37887 +37888 POINT(29.394582925311177 -96.04447404952091) bank37888 +37889 POINT(30.20243247206249 -95.00283638638318) bank37889 +37890 POINT(30.55847018265246 -95.3087108201674) bank37890 +37891 POINT(30.29038188246639 -94.82660467977205) bank37891 +37892 POINT(30.651342954388184 -94.67088554600961) bank37892 +37893 POINT(30.04242863635058 -94.51426827980501) bank37893 +37894 POINT(29.906541486919828 -94.64680998999961) bank37894 +37895 POINT(29.141361297385878 -95.78528977086403) bank37895 +37896 POINT(29.258083912745853 -95.19954673307306) bank37896 +37897 POINT(29.82519924778345 -96.01487136559923) bank37897 +37898 POINT(30.480982538623415 -96.29210913647859) bank37898 +37899 POINT(29.27485682834325 -95.27469690914498) bank37899 +37900 POINT(30.354671741385584 -95.8541709372959) bank37900 +37901 POINT(29.226447197997324 -94.68617987368435) bank37901 +37902 POINT(30.72507562636408 -95.22111403492282) bank37902 +37903 POINT(30.117473497295247 -95.52241099925189) bank37903 +37904 POINT(30.366330933381473 -96.28538369712277) bank37904 +37905 POINT(30.23870539238948 -95.3850147061498) bank37905 +37906 POINT(29.106417560352124 -96.18643475331055) bank37906 +37907 POINT(29.901484668548864 -95.79883525983297) bank37907 +37908 POINT(28.946668896751007 -94.98719100737425) bank37908 +37909 POINT(29.99352743830275 -95.75681696185387) bank37909 +37910 POINT(30.276747053975285 -95.89162993716967) bank37910 +37911 POINT(29.603563743612018 -94.47505383231635) bank37911 +37912 POINT(29.965933393177174 -95.332854261496) bank37912 +37913 POINT(30.092923547560808 -94.49478530309119) bank37913 +37914 POINT(29.047566081840273 -94.88888404119983) bank37914 +37915 POINT(30.730393216586304 -94.53282999390616) bank37915 +37916 POINT(29.135343099446644 -94.54206312893642) bank37916 +37917 POINT(30.283682386309533 -95.89926497353382) bank37917 +37918 POINT(28.89135568765608 -96.09533373255452) bank37918 +37919 POINT(30.119301142990825 -94.85297057635644) bank37919 +37920 POINT(30.271051530783218 -95.06140913227999) bank37920 +37921 POINT(29.4658382862731 -96.1179571001654) bank37921 +37922 POINT(29.187651949814153 -95.4994879431485) bank37922 +37923 POINT(29.773061928897704 -94.69636668242335) bank37923 +37924 POINT(29.183096519751725 -96.1707460950982) bank37924 +37925 POINT(28.805297670663464 -95.63565938307401) bank37925 +37926 POINT(29.74584939245087 -96.17919118546976) bank37926 +37927 POINT(30.234911811102165 -94.74437650122462) bank37927 +37928 POINT(28.975793409424476 -94.62685539738975) bank37928 +37929 POINT(28.808289588672572 -96.07922696269772) bank37929 +37930 POINT(30.407416325170185 -96.08551144652094) bank37930 +37931 POINT(30.308186877258112 -95.77337689116023) bank37931 +37932 POINT(30.580330418121555 -94.59759747680869) bank37932 +37933 POINT(29.8219315079868 -94.9989632415763) bank37933 +37934 POINT(29.95824716504221 -95.18658504545903) bank37934 +37935 POINT(29.208559911010557 -94.67713995411408) bank37935 +37936 POINT(30.12863342560489 -95.96326686592029) bank37936 +37937 POINT(29.637285868172757 -96.05473160081465) bank37937 +37938 POINT(29.425072935608107 -96.30417943459125) bank37938 +37939 POINT(30.174828220469628 -96.13134913270201) bank37939 +37940 POINT(29.680402432974713 -96.28003573760027) bank37940 +37941 POINT(29.088880343945654 -96.22334974124328) bank37941 +37942 POINT(29.11484181199598 -95.01059625697667) bank37942 +37943 POINT(30.174842440560987 -95.35010829723696) bank37943 +37944 POINT(30.213721506284152 -95.27884281420366) bank37944 +37945 POINT(29.05540039060344 -95.0023655338262) bank37945 +37946 POINT(30.461386389171327 -95.40717338705898) bank37946 +37947 POINT(30.260045160241457 -96.23961494778604) bank37947 +37948 POINT(29.8187652103677 -94.49153889974887) bank37948 +37949 POINT(28.881678486852266 -95.68268656661952) bank37949 +37950 POINT(30.659771216479463 -95.9199196475293) bank37950 +37951 POINT(29.179564568781693 -94.54532064668612) bank37951 +37952 POINT(30.75100731543661 -95.4268100532002) bank37952 +37953 POINT(29.188531854849966 -96.1499159011849) bank37953 +37954 POINT(29.95256648780413 -95.98712351661344) bank37954 +37955 POINT(29.430459354988972 -94.9116351780844) bank37955 +37956 POINT(30.065003554786422 -94.54533685581305) bank37956 +37957 POINT(29.054072675528857 -95.403446122919) bank37957 +37958 POINT(28.772496274338486 -95.06440980211659) bank37958 +37959 POINT(30.460081171468484 -95.58877456040672) bank37959 +37960 POINT(29.0965046757684 -95.51382949902846) bank37960 +37961 POINT(29.05450075643893 -95.3696342730175) bank37961 +37962 POINT(29.927246365361107 -94.63228309412371) bank37962 +37963 POINT(29.254374138029426 -95.16615638346347) bank37963 +37964 POINT(28.863434050154947 -94.53298229618022) bank37964 +37965 POINT(30.306023221817707 -96.33340000974088) bank37965 +37966 POINT(29.753195583237865 -95.81371364242149) bank37966 +37967 POINT(28.877587791688985 -95.74491950975538) bank37967 +37968 POINT(29.60752818450065 -94.61339535424085) bank37968 +37969 POINT(29.71578572820139 -95.23684336582983) bank37969 +37970 POINT(30.134727442616335 -96.23470012579014) bank37970 +37971 POINT(29.499344993160914 -94.8592766406926) bank37971 +37972 POINT(29.21431078131304 -95.18160608190298) bank37972 +37973 POINT(29.45689779850684 -95.01473673054498) bank37973 +37974 POINT(28.985511985037775 -94.9749617226506) bank37974 +37975 POINT(30.465260105549874 -96.03214624744828) bank37975 +37976 POINT(29.025855220909968 -96.0670487177066) bank37976 +37977 POINT(30.292906421010453 -95.51949538743489) bank37977 +37978 POINT(30.44653807485457 -95.1460394192424) bank37978 +37979 POINT(30.084557369220587 -94.47008474968752) bank37979 +37980 POINT(29.518107787510562 -95.07594897740172) bank37980 +37981 POINT(30.111054079215798 -94.77352583591718) bank37981 +37982 POINT(30.5324984064247 -94.97468218163627) bank37982 +37983 POINT(28.828416758091695 -94.47738558412513) bank37983 +37984 POINT(30.40154522283751 -95.71667931756768) bank37984 +37985 POINT(30.451163161475304 -95.76310859135671) bank37985 +37986 POINT(30.648542898147394 -94.82500701673328) bank37986 +37987 POINT(29.650600698472072 -95.18272525715079) bank37987 +37988 POINT(28.797160744518795 -94.83290133983948) bank37988 +37989 POINT(28.914874787770835 -94.77582257962375) bank37989 +37990 POINT(28.839759019158187 -94.58721014160992) bank37990 +37991 POINT(28.81994059023818 -95.44448723682336) bank37991 +37992 POINT(30.045653474983073 -96.03616334172207) bank37992 +37993 POINT(29.158613204401938 -94.97795717238854) bank37993 +37994 POINT(30.467899905057383 -95.64326537351965) bank37994 +37995 POINT(29.35164836476853 -95.71747291527757) bank37995 +37996 POINT(30.1687645194082 -94.76307919121706) bank37996 +37997 POINT(29.35399879490895 -95.64643091098947) bank37997 +37998 POINT(29.347365276081234 -94.76669214699416) bank37998 +37999 POINT(30.516911589656587 -96.12923138418313) bank37999 +38000 POINT(29.115199836237927 -96.28224165909654) bank38000 +38001 POINT(29.96005555610744 -95.57691335167922) bank38001 +38002 POINT(29.95027877441782 -96.35062717721335) bank38002 +38003 POINT(29.061233257102344 -94.38674147748446) bank38003 +38004 POINT(29.62583769496087 -95.02584841315736) bank38004 +38005 POINT(29.181418963083082 -96.11210623540532) bank38005 +38006 POINT(29.46356864403199 -94.57523425495995) bank38006 +38007 POINT(30.030587692403433 -96.19890133883764) bank38007 +38008 POINT(28.845366309463927 -95.93833734722499) bank38008 +38009 POINT(29.65992936894282 -95.29897983270024) bank38009 +38010 POINT(30.73968535595024 -95.38095938186122) bank38010 +38011 POINT(29.152925904801524 -94.93440817208509) bank38011 +38012 POINT(29.609711153884877 -95.65231633943161) bank38012 +38013 POINT(30.00464154154099 -94.4031518164302) bank38013 +38014 POINT(29.446956093425225 -95.99074795848087) bank38014 +38015 POINT(29.185570716532172 -94.81646698843778) bank38015 +38016 POINT(29.426879234599767 -96.21531382645809) bank38016 +38017 POINT(28.944538625909797 -95.40697435388633) bank38017 +38018 POINT(30.440847411065494 -96.07641228393017) bank38018 +38019 POINT(30.514386820552335 -94.54795594063069) bank38019 +38020 POINT(29.631230028485856 -94.59713905350893) bank38020 +38021 POINT(29.77101710547513 -95.75358015667302) bank38021 +38022 POINT(30.228402779687862 -94.82192535827335) bank38022 +38023 POINT(29.5908593018463 -94.611191117772) bank38023 +38024 POINT(29.25604208434862 -96.02618144479952) bank38024 +38025 POINT(29.60995457823278 -94.45437312387841) bank38025 +38026 POINT(30.643184135654405 -94.64240249791192) bank38026 +38027 POINT(30.057703468066016 -96.20205306168597) bank38027 +38028 POINT(29.001625119593058 -96.28461125351225) bank38028 +38029 POINT(30.105316482048558 -94.98729512102352) bank38029 +38030 POINT(29.33875606646165 -95.90623147061584) bank38030 +38031 POINT(28.848730761329783 -95.55539389028276) bank38031 +38032 POINT(29.315106661511194 -95.57395532816855) bank38032 +38033 POINT(29.713865571596816 -94.45633021173616) bank38033 +38034 POINT(29.5290570656841 -95.7903858330602) bank38034 +38035 POINT(30.569574247513724 -96.18785549572641) bank38035 +38036 POINT(28.797464719566644 -94.76175289910601) bank38036 +38037 POINT(30.434153270378687 -94.64483266529523) bank38037 +38038 POINT(30.634932818406966 -94.48792533743764) bank38038 +38039 POINT(29.406099426729245 -94.77757455612448) bank38039 +38040 POINT(30.026953522564863 -95.57276339786412) bank38040 +38041 POINT(29.862451479241653 -94.86865504961168) bank38041 +38042 POINT(30.240021995145273 -96.0827153014262) bank38042 +38043 POINT(30.586610185924712 -95.32234949543046) bank38043 +38044 POINT(30.280285754460664 -96.2992221320862) bank38044 +38045 POINT(28.878744776954587 -94.40229974049035) bank38045 +38046 POINT(30.327255704785763 -95.62188935508985) bank38046 +38047 POINT(29.60335315167653 -95.21560158607296) bank38047 +38048 POINT(29.39809167457145 -94.72349786360549) bank38048 +38049 POINT(30.534371408178067 -94.40284180513957) bank38049 +38050 POINT(29.8150808717414 -94.8715343401791) bank38050 +38051 POINT(30.185498428019272 -95.43550752747234) bank38051 +38052 POINT(29.834592735878314 -94.58812140413323) bank38052 +38053 POINT(30.373388033789873 -95.52819534653298) bank38053 +38054 POINT(29.861625921029194 -94.70185812146245) bank38054 +38055 POINT(29.057501246068018 -96.16840499170023) bank38055 +38056 POINT(30.03814911564001 -95.09953686293565) bank38056 +38057 POINT(29.098633710642435 -94.82082996430324) bank38057 +38058 POINT(30.370214331015205 -94.81319893241485) bank38058 +38059 POINT(30.320428898652782 -94.67026247374822) bank38059 +38060 POINT(29.301885296593564 -94.66375652343424) bank38060 +38061 POINT(29.0806427436929 -95.04238275599123) bank38061 +38062 POINT(30.29509123095206 -95.34896907120869) bank38062 +38063 POINT(29.35547165901467 -94.67969819639576) bank38063 +38064 POINT(30.075509709590953 -94.75295576100173) bank38064 +38065 POINT(28.77696295070294 -96.06591673620292) bank38065 +38066 POINT(29.64439275890947 -95.39413146285018) bank38066 +38067 POINT(30.690333439816314 -95.32852308631003) bank38067 +38068 POINT(29.262350595964783 -96.24843834742833) bank38068 +38069 POINT(30.54474330452117 -95.33308645431306) bank38069 +38070 POINT(30.201379728418296 -96.31556089216687) bank38070 +38071 POINT(30.726307280368975 -96.33170149975184) bank38071 +38072 POINT(29.834688542536817 -95.88160989692503) bank38072 +38073 POINT(30.50079550042241 -95.21568468458786) bank38073 +38074 POINT(28.89742346351527 -95.41918784181753) bank38074 +38075 POINT(29.4913170430179 -95.56821111222345) bank38075 +38076 POINT(29.666101473989862 -94.62717946138501) bank38076 +38077 POINT(29.531717854799222 -95.34139989169701) bank38077 +38078 POINT(30.039328538977156 -96.1470888452326) bank38078 +38079 POINT(30.326819652274768 -94.84895470292537) bank38079 +38080 POINT(29.593746995222837 -94.74419722567438) bank38080 +38081 POINT(29.194817056482812 -96.18316949016486) bank38081 +38082 POINT(30.72812350107737 -96.31660173503084) bank38082 +38083 POINT(29.3795620147014 -95.98679715732219) bank38083 +38084 POINT(30.19430469663361 -96.34216316270543) bank38084 +38085 POINT(29.522006160255767 -94.60308413977599) bank38085 +38086 POINT(30.60010861814353 -94.45604546763226) bank38086 +38087 POINT(28.840209824364486 -95.66481480406455) bank38087 +38088 POINT(30.370894477923176 -95.63011151428776) bank38088 +38089 POINT(30.413605028685144 -95.62109342199201) bank38089 +38090 POINT(30.371307820201913 -95.1932594994466) bank38090 +38091 POINT(30.151040591755734 -96.1849711810481) bank38091 +38092 POINT(30.386028853161818 -95.85458065298941) bank38092 +38093 POINT(29.21739245004997 -95.25146190695116) bank38093 +38094 POINT(30.255154727899345 -94.41197452112705) bank38094 +38095 POINT(28.84641370218752 -95.27656975813339) bank38095 +38096 POINT(28.99509851870977 -94.37726305227783) bank38096 +38097 POINT(29.50848774701758 -95.18126597325296) bank38097 +38098 POINT(30.491968769918923 -96.24438688434678) bank38098 +38099 POINT(30.640492899266633 -95.99877127893545) bank38099 +38100 POINT(30.734745228775985 -95.30983085456353) bank38100 +38101 POINT(28.790713270462202 -94.88805970453092) bank38101 +38102 POINT(28.81101462362799 -95.29250389265664) bank38102 +38103 POINT(29.838310067619823 -95.87717908737619) bank38103 +38104 POINT(29.63911403019638 -96.12329179455676) bank38104 +38105 POINT(30.112295881912154 -95.65240445368119) bank38105 +38106 POINT(29.531509284718222 -96.02971416320717) bank38106 +38107 POINT(29.43527446503437 -95.49877080733187) bank38107 +38108 POINT(29.644162020102126 -94.76977723358378) bank38108 +38109 POINT(29.004171218340346 -96.14727331953347) bank38109 +38110 POINT(28.944560988484987 -95.48451614700632) bank38110 +38111 POINT(29.209879222753603 -94.43597572663938) bank38111 +38112 POINT(29.812540922452868 -95.25614180309637) bank38112 +38113 POINT(29.747804646283008 -95.05816640441942) bank38113 +38114 POINT(30.06347014751261 -95.50732460557991) bank38114 +38115 POINT(29.326172052927532 -94.59225478080876) bank38115 +38116 POINT(29.66884101857059 -95.01897400506031) bank38116 +38117 POINT(29.406019370878706 -95.19094332189925) bank38117 +38118 POINT(30.27782333266068 -96.21394140206733) bank38118 +38119 POINT(29.228574852394907 -95.16978289983274) bank38119 +38120 POINT(30.094902225945148 -95.65368787368003) bank38120 +38121 POINT(29.067468260450124 -94.67176116839055) bank38121 +38122 POINT(30.525017649904324 -95.37291101940679) bank38122 +38123 POINT(30.04315922539851 -94.52031962507549) bank38123 +38124 POINT(28.91714521068525 -95.83119121492211) bank38124 +38125 POINT(29.217097807935385 -95.0953976206495) bank38125 +38126 POINT(29.5754268452496 -95.60895531587951) bank38126 +38127 POINT(30.676545940367408 -95.5135786586022) bank38127 +38128 POINT(30.600171547461983 -95.80524078071853) bank38128 +38129 POINT(29.41290289782419 -94.79506857167878) bank38129 +38130 POINT(29.438821517979225 -95.16638563692126) bank38130 +38131 POINT(30.3927775557816 -95.46410154105511) bank38131 +38132 POINT(30.34933998830485 -96.18948781685157) bank38132 +38133 POINT(29.106943549868966 -94.39936898773115) bank38133 +38134 POINT(30.67413108825418 -94.50743093379846) bank38134 +38135 POINT(30.18839799802747 -96.28510763902217) bank38135 +38136 POINT(29.293321005700772 -96.27458959301941) bank38136 +38137 POINT(28.940512947041338 -95.49078104025473) bank38137 +38138 POINT(30.28314558934932 -94.62750089672882) bank38138 +38139 POINT(29.89129527585173 -95.53897812788728) bank38139 +38140 POINT(29.540402777027484 -96.33069579245016) bank38140 +38141 POINT(29.866351529366604 -96.06571613882723) bank38141 +38142 POINT(28.7827074158816 -95.63104004102922) bank38142 +38143 POINT(29.01548314669839 -95.43084163093289) bank38143 +38144 POINT(30.213382204325278 -95.95404446462378) bank38144 +38145 POINT(28.99811854113799 -95.2945152344902) bank38145 +38146 POINT(29.716163319990056 -95.59086569782822) bank38146 +38147 POINT(30.2891575273617 -95.8332778617828) bank38147 +38148 POINT(29.242267034873535 -96.26537940075522) bank38148 +38149 POINT(29.398540240556123 -95.6816901297086) bank38149 +38150 POINT(29.16896551481177 -94.96942487745885) bank38150 +38151 POINT(30.10643747000928 -95.51819635021033) bank38151 +38152 POINT(29.10253745343495 -96.03916399085023) bank38152 +38153 POINT(29.80064252415816 -96.12005918870845) bank38153 +38154 POINT(30.492732132792035 -96.26768334087345) bank38154 +38155 POINT(29.398237858507233 -94.71801452273189) bank38155 +38156 POINT(30.60100050033265 -95.3925814257593) bank38156 +38157 POINT(30.236910762532265 -94.9528813438496) bank38157 +38158 POINT(28.914027752978562 -96.16164150894345) bank38158 +38159 POINT(29.229967924797954 -95.11391552933846) bank38159 +38160 POINT(28.846875162238767 -95.96394220469043) bank38160 +38161 POINT(29.43362123941363 -94.7390509053711) bank38161 +38162 POINT(29.73139705731129 -95.54557003696118) bank38162 +38163 POINT(30.573664211875975 -95.4130103057137) bank38163 +38164 POINT(30.195460968916766 -95.44897909451915) bank38164 +38165 POINT(29.977007914481053 -95.30568076865598) bank38165 +38166 POINT(29.398418778778694 -95.0458925359927) bank38166 +38167 POINT(29.67321083496063 -94.39360636087792) bank38167 +38168 POINT(28.762541524772566 -94.86439609147237) bank38168 +38169 POINT(30.739391985728776 -94.8410752585322) bank38169 +38170 POINT(30.118588546476182 -94.93013834743819) bank38170 +38171 POINT(30.275648416639356 -95.03432208093814) bank38171 +38172 POINT(29.23716883139912 -95.17314858516987) bank38172 +38173 POINT(29.039254079242582 -95.61806916289119) bank38173 +38174 POINT(29.846885498391245 -95.12714491348451) bank38174 +38175 POINT(30.680654884158404 -94.88717947767344) bank38175 +38176 POINT(30.581462097608537 -95.01557236086369) bank38176 +38177 POINT(29.933079992583664 -96.1744439515391) bank38177 +38178 POINT(29.96857086697139 -95.76764372956195) bank38178 +38179 POINT(30.28664283192422 -94.82148573738732) bank38179 +38180 POINT(30.47595722366873 -95.99211964973824) bank38180 +38181 POINT(29.679233560446118 -95.29239165861249) bank38181 +38182 POINT(30.547664978990706 -94.37332360766776) bank38182 +38183 POINT(29.704044878291676 -95.85116740409333) bank38183 +38184 POINT(30.428904053029044 -94.81948591598889) bank38184 +38185 POINT(29.70053584816258 -94.53742875129556) bank38185 +38186 POINT(29.85637995472703 -94.58091501346998) bank38186 +38187 POINT(29.020457698990803 -95.40673490569563) bank38187 +38188 POINT(29.855224979721047 -96.16074451713105) bank38188 +38189 POINT(29.0770474415129 -94.74926453817542) bank38189 +38190 POINT(28.827568374534987 -95.70464298775686) bank38190 +38191 POINT(28.873374687803178 -96.17749186835752) bank38191 +38192 POINT(30.38205719124041 -95.07280982632089) bank38192 +38193 POINT(30.659091971539013 -95.28920675573484) bank38193 +38194 POINT(30.299857591238954 -94.5045093320041) bank38194 +38195 POINT(30.190148205003972 -95.5020902273254) bank38195 +38196 POINT(29.93784272267983 -95.66204335044478) bank38196 +38197 POINT(30.51170600510359 -94.74882909650832) bank38197 +38198 POINT(30.091277724615253 -96.32582194383379) bank38198 +38199 POINT(29.721376008205326 -96.19365792018225) bank38199 +38200 POINT(28.902085873402235 -95.71498968110056) bank38200 +38201 POINT(30.2155219720265 -95.26079220126215) bank38201 +38202 POINT(30.7370488930127 -94.78185932094698) bank38202 +38203 POINT(29.94956108964019 -95.52890752616332) bank38203 +38204 POINT(29.50658588925421 -96.16357506542428) bank38204 +38205 POINT(30.188113203847262 -96.17535375844159) bank38205 +38206 POINT(29.99588653330422 -94.81415780056292) bank38206 +38207 POINT(29.213642283341343 -95.24017323492042) bank38207 +38208 POINT(30.222152099553103 -96.03564807943883) bank38208 +38209 POINT(28.85787394546794 -95.0798080966903) bank38209 +38210 POINT(30.614215382878946 -94.4415162174161) bank38210 +38211 POINT(30.05918747543539 -95.45979894235741) bank38211 +38212 POINT(29.573597490707556 -94.76793575535653) bank38212 +38213 POINT(30.607738680971394 -95.58895035819626) bank38213 +38214 POINT(30.170871433235547 -96.3102560827845) bank38214 +38215 POINT(29.70431006338882 -94.86144870747577) bank38215 +38216 POINT(29.627747684270627 -96.25042775330549) bank38216 +38217 POINT(28.799935082923447 -95.83425064645526) bank38217 +38218 POINT(30.443544647432503 -95.88901054259934) bank38218 +38219 POINT(29.719006600559197 -94.93207369031731) bank38219 +38220 POINT(29.592123955397334 -94.9788550336579) bank38220 +38221 POINT(29.123646999318485 -95.21170219185795) bank38221 +38222 POINT(29.10073379203466 -94.6083952062595) bank38222 +38223 POINT(30.429105607000132 -94.77666711195798) bank38223 +38224 POINT(29.694581310273154 -95.99347030317112) bank38224 +38225 POINT(30.02485056278602 -96.0387051050211) bank38225 +38226 POINT(29.2278254524899 -96.2281542350982) bank38226 +38227 POINT(29.14809085726544 -95.6720650105706) bank38227 +38228 POINT(30.348225775343774 -95.22475092938679) bank38228 +38229 POINT(29.568726421591574 -95.5400403237811) bank38229 +38230 POINT(29.25142339922191 -95.90466815721612) bank38230 +38231 POINT(28.798498169668044 -94.6536457401549) bank38231 +38232 POINT(29.65630422477482 -94.79248609764763) bank38232 +38233 POINT(30.572554343852556 -95.68098659965739) bank38233 +38234 POINT(29.99240928699289 -95.640493296948) bank38234 +38235 POINT(29.60974229501066 -95.55473056005422) bank38235 +38236 POINT(30.1184188975357 -95.98053193079043) bank38236 +38237 POINT(29.445030290792356 -95.85668028410309) bank38237 +38238 POINT(30.416624488513037 -96.08768075636264) bank38238 +38239 POINT(29.597401200061107 -95.56766521654369) bank38239 +38240 POINT(30.610829459538337 -96.36461409929281) bank38240 +38241 POINT(30.02673066724992 -94.51936868108623) bank38241 +38242 POINT(29.20368317272938 -95.84112678658971) bank38242 +38243 POINT(29.40533641555476 -94.60781618331725) bank38243 +38244 POINT(30.511409125859625 -95.51696894332511) bank38244 +38245 POINT(29.76486162517079 -95.91837620256904) bank38245 +38246 POINT(29.135945028666463 -96.04063489017756) bank38246 +38247 POINT(29.741446046126864 -95.83634177397387) bank38247 +38248 POINT(29.956659398875725 -94.61130845083392) bank38248 +38249 POINT(29.76590236413102 -96.04053768925287) bank38249 +38250 POINT(29.651782893992834 -96.06459147883713) bank38250 +38251 POINT(29.705369934111182 -94.41523582294775) bank38251 +38252 POINT(29.919440967639236 -95.70493731686607) bank38252 +38253 POINT(29.780389989375262 -94.56667409132889) bank38253 +38254 POINT(29.378626422768725 -95.33854176778405) bank38254 +38255 POINT(30.568232206023417 -96.32445310626298) bank38255 +38256 POINT(30.47538391895636 -95.89569568438169) bank38256 +38257 POINT(29.368379594577156 -94.96832750371244) bank38257 +38258 POINT(29.669616968781572 -96.09309663222965) bank38258 +38259 POINT(29.558227967448392 -95.7083598034624) bank38259 +38260 POINT(29.790147898970076 -95.70842529140134) bank38260 +38261 POINT(30.316314070341576 -96.22904808945421) bank38261 +38262 POINT(30.740272188435263 -96.05728922396128) bank38262 +38263 POINT(29.203950511276958 -94.45682177055176) bank38263 +38264 POINT(30.018603954288675 -94.82381238702051) bank38264 +38265 POINT(29.286393676484696 -94.63513062302272) bank38265 +38266 POINT(29.44992602124032 -95.84200885608705) bank38266 +38267 POINT(30.04122664357246 -95.02858056158026) bank38267 +38268 POINT(30.313998227320745 -94.97361929121017) bank38268 +38269 POINT(29.62689622399604 -95.19585385434782) bank38269 +38270 POINT(30.73419607655621 -95.27815193825286) bank38270 +38271 POINT(30.62856292803716 -95.20688426999851) bank38271 +38272 POINT(30.596323566737734 -95.80011155488353) bank38272 +38273 POINT(29.254172697844016 -95.80993156110424) bank38273 +38274 POINT(30.474530485356315 -94.9202748352301) bank38274 +38275 POINT(30.604546987181802 -94.80334502350212) bank38275 +38276 POINT(30.120632725036625 -96.30574092896161) bank38276 +38277 POINT(29.215010643712596 -94.75500235425262) bank38277 +38278 POINT(29.653659276196876 -96.33457790173597) bank38278 +38279 POINT(29.439567577277987 -95.6507098950073) bank38279 +38280 POINT(29.004647591873834 -95.48087912238067) bank38280 +38281 POINT(29.95538615048657 -95.16236745787506) bank38281 +38282 POINT(29.01503374606477 -94.7688588708899) bank38282 +38283 POINT(29.663021293225572 -95.16626579211818) bank38283 +38284 POINT(29.03828112014822 -94.68926229056797) bank38284 +38285 POINT(28.990287838902802 -94.53660473711952) bank38285 +38286 POINT(28.847806955170473 -94.40646105359806) bank38286 +38287 POINT(29.35781029554972 -94.65644620866343) bank38287 +38288 POINT(29.551538094763547 -94.72029614437841) bank38288 +38289 POINT(28.81057338641614 -94.53496752282702) bank38289 +38290 POINT(29.867586500384043 -94.54207883959361) bank38290 +38291 POINT(30.17158582261108 -94.73457688593085) bank38291 +38292 POINT(29.53568541490173 -94.89436858172743) bank38292 +38293 POINT(30.221866700879747 -95.31913585925176) bank38293 +38294 POINT(29.48105817427611 -96.27388394013795) bank38294 +38295 POINT(30.454315525094756 -96.0979143430444) bank38295 +38296 POINT(29.46397411125069 -95.33878448998303) bank38296 +38297 POINT(30.150328904139492 -95.62682645631615) bank38297 +38298 POINT(28.866956755267577 -95.71779683454953) bank38298 +38299 POINT(30.745908421882515 -95.49227392625859) bank38299 +38300 POINT(29.107227533780758 -96.15902565981561) bank38300 +38301 POINT(30.51389441559687 -94.61233983937234) bank38301 +38302 POINT(28.771170857018 -96.20013029964338) bank38302 +38303 POINT(30.34403610920252 -96.1780048084445) bank38303 +38304 POINT(29.250654914380707 -96.36908708839405) bank38304 +38305 POINT(30.351289601960453 -96.20272822147686) bank38305 +38306 POINT(29.011382470573096 -94.99609591438595) bank38306 +38307 POINT(28.814593310937198 -95.30614889708178) bank38307 +38308 POINT(29.43614948112187 -94.99981564289986) bank38308 +38309 POINT(30.52993655030755 -95.3753566493799) bank38309 +38310 POINT(30.7250246074523 -95.26919810886314) bank38310 +38311 POINT(28.966405610164227 -94.53931229170318) bank38311 +38312 POINT(29.76433573529309 -96.18255738290522) bank38312 +38313 POINT(28.97624516310726 -95.90144832645883) bank38313 +38314 POINT(28.76523100194091 -96.02920129473952) bank38314 +38315 POINT(29.751936115695365 -94.73176453167717) bank38315 +38316 POINT(29.63570913878184 -95.92218721166644) bank38316 +38317 POINT(29.497532343903067 -95.83394103639559) bank38317 +38318 POINT(30.614866833239436 -94.89364479972238) bank38318 +38319 POINT(28.832859349056953 -95.92707978165366) bank38319 +38320 POINT(30.227731939918222 -95.53918159696515) bank38320 +38321 POINT(30.663711083947987 -94.46496506688985) bank38321 +38322 POINT(29.300812177896386 -96.27591997602961) bank38322 +38323 POINT(30.73261457790564 -94.99616218512105) bank38323 +38324 POINT(29.44773497604192 -96.05762741254836) bank38324 +38325 POINT(29.485962858976105 -96.33027192649757) bank38325 +38326 POINT(29.915752047331413 -95.58486430021865) bank38326 +38327 POINT(30.706001460753207 -94.77523004964456) bank38327 +38328 POINT(29.59165112559376 -96.03593245844365) bank38328 +38329 POINT(30.606943101566014 -94.41754704103738) bank38329 +38330 POINT(28.772399974351522 -94.82304870794725) bank38330 +38331 POINT(30.755925056485143 -95.39552633147375) bank38331 +38332 POINT(29.038976394577908 -94.79702312522171) bank38332 +38333 POINT(29.068681332195517 -96.31714718307504) bank38333 +38334 POINT(28.85505703785859 -95.55213666682174) bank38334 +38335 POINT(30.06915619478673 -94.60338411451886) bank38335 +38336 POINT(29.31235605330394 -95.97676382154829) bank38336 +38337 POINT(29.19936636552009 -95.50197637658702) bank38337 +38338 POINT(29.812461355523233 -95.00392433456179) bank38338 +38339 POINT(29.62409722122266 -95.28515978671312) bank38339 +38340 POINT(29.845311089779532 -95.33544485537533) bank38340 +38341 POINT(28.891497353173648 -95.13466181323257) bank38341 +38342 POINT(30.663818931419403 -96.13435879990043) bank38342 +38343 POINT(29.1220745853246 -95.4980737159439) bank38343 +38344 POINT(29.466392330550335 -96.12159496012299) bank38344 +38345 POINT(30.201800368399947 -95.84468470787013) bank38345 +38346 POINT(29.11294029435871 -96.0468311407597) bank38346 +38347 POINT(30.08509294110654 -94.90521424929658) bank38347 +38348 POINT(30.68340707716215 -95.26951454118645) bank38348 +38349 POINT(29.17965280599388 -95.22252497188234) bank38349 +38350 POINT(30.559692280678703 -95.50700136824591) bank38350 +38351 POINT(30.586468060803305 -94.8384661529297) bank38351 +38352 POINT(29.82001841987268 -94.5694250000196) bank38352 +38353 POINT(28.929456426010617 -95.71365927638449) bank38353 +38354 POINT(29.460823700583724 -95.78127749291606) bank38354 +38355 POINT(29.754279011005544 -94.64441770715855) bank38355 +38356 POINT(29.376220003715268 -95.61452195129779) bank38356 +38357 POINT(30.411316557791405 -94.97218653436305) bank38357 +38358 POINT(29.88442631794386 -94.92762913983034) bank38358 +38359 POINT(30.440861373142873 -95.72461629126357) bank38359 +38360 POINT(28.927997415649116 -95.34255489208329) bank38360 +38361 POINT(29.761284487688595 -95.28786117959773) bank38361 +38362 POINT(29.04699790768491 -94.97578128329609) bank38362 +38363 POINT(30.065150000238237 -96.10261555148466) bank38363 +38364 POINT(30.47780365893527 -96.34134213037142) bank38364 +38365 POINT(29.15344945184658 -94.83000998621216) bank38365 +38366 POINT(30.365284534299015 -95.25520456400972) bank38366 +38367 POINT(28.799634647801874 -96.10774494362663) bank38367 +38368 POINT(28.76685035391777 -95.19329244803312) bank38368 +38369 POINT(30.64919197125643 -94.82316543771346) bank38369 +38370 POINT(30.2440958371588 -95.39837612249207) bank38370 +38371 POINT(30.38141565764801 -94.57846390631087) bank38371 +38372 POINT(29.9976766083412 -95.48669679413207) bank38372 +38373 POINT(29.045181823799393 -96.21725674303862) bank38373 +38374 POINT(28.95166252798947 -94.9376875653605) bank38374 +38375 POINT(29.690489734385586 -95.24041782708096) bank38375 +38376 POINT(30.262056325409585 -95.4254124481753) bank38376 +38377 POINT(28.880058684321064 -95.40740471779414) bank38377 +38378 POINT(29.592658799419944 -95.47036011964391) bank38378 +38379 POINT(30.369898138348283 -94.52309306479687) bank38379 +38380 POINT(28.9464050633235 -95.2326164689884) bank38380 +38381 POINT(30.43195457576096 -95.30999900729321) bank38381 +38382 POINT(30.06897366469261 -96.12882742563913) bank38382 +38383 POINT(30.60107579904985 -95.12144789088) bank38383 +38384 POINT(29.041556115561555 -95.12893590666117) bank38384 +38385 POINT(29.036631786169288 -94.48694204180363) bank38385 +38386 POINT(29.054358366596183 -95.03624568032087) bank38386 +38387 POINT(30.702161999765988 -94.88275513459573) bank38387 +38388 POINT(29.052075933932414 -95.28699826763385) bank38388 +38389 POINT(29.644807336734914 -96.2740583660888) bank38389 +38390 POINT(28.91118381732022 -96.1395013705669) bank38390 +38391 POINT(29.083361837011225 -95.11931977321868) bank38391 +38392 POINT(29.95838237092741 -94.83669057452506) bank38392 +38393 POINT(30.337651620705035 -94.39425327713072) bank38393 +38394 POINT(30.046801025378596 -95.90993585849277) bank38394 +38395 POINT(29.54782999281933 -94.65628431379359) bank38395 +38396 POINT(29.07304946053156 -94.92830543751911) bank38396 +38397 POINT(29.151039781022014 -94.52745821670156) bank38397 +38398 POINT(29.242625508473473 -95.1566625935245) bank38398 +38399 POINT(29.991320756711414 -95.34127602301254) bank38399 +38400 POINT(29.855285724072058 -95.74592330196336) bank38400 +38401 POINT(29.48026800967366 -95.18906215361412) bank38401 +38402 POINT(30.700678524684268 -95.62225152694903) bank38402 +38403 POINT(30.62264629965884 -94.88065892665983) bank38403 +38404 POINT(30.282995158600855 -95.38326793858403) bank38404 +38405 POINT(30.527019254946715 -95.24362021534452) bank38405 +38406 POINT(29.23630648147476 -94.83345858851474) bank38406 +38407 POINT(29.703191384872984 -94.9514318214139) bank38407 +38408 POINT(29.262801899608633 -94.65028997195762) bank38408 +38409 POINT(29.885793314360413 -95.99653452626004) bank38409 +38410 POINT(29.56297520172229 -95.47167622153646) bank38410 +38411 POINT(29.86631391867011 -96.2471670160671) bank38411 +38412 POINT(29.312939774508838 -94.97938981937581) bank38412 +38413 POINT(29.890688136172393 -94.53473028048192) bank38413 +38414 POINT(30.02904538242793 -94.6371529274007) bank38414 +38415 POINT(30.39064522564351 -95.37942465360081) bank38415 +38416 POINT(29.311023582247536 -95.21571715139324) bank38416 +38417 POINT(30.114211718662872 -94.74970703666082) bank38417 +38418 POINT(30.34234149993554 -95.13207483052518) bank38418 +38419 POINT(29.583134861112725 -96.27950969386195) bank38419 +38420 POINT(28.984988639056542 -96.33039009426564) bank38420 +38421 POINT(30.670351844101337 -95.77393510023019) bank38421 +38422 POINT(30.041490998498915 -95.93005766930021) bank38422 +38423 POINT(30.565572261680956 -96.10987852429574) bank38423 +38424 POINT(29.819939999176466 -95.5671530374805) bank38424 +38425 POINT(30.653309342620553 -95.33788910709652) bank38425 +38426 POINT(29.59628792904383 -95.67819323512032) bank38426 +38427 POINT(29.339856051048955 -96.18904719105514) bank38427 +38428 POINT(30.75202964571469 -94.61522309817136) bank38428 +38429 POINT(30.327447490252666 -95.81670678914872) bank38429 +38430 POINT(29.030432741569307 -94.74727547776205) bank38430 +38431 POINT(29.225672508086458 -94.37072899782501) bank38431 +38432 POINT(30.270281025128206 -95.60992726125461) bank38432 +38433 POINT(30.22349301831574 -96.03260154666071) bank38433 +38434 POINT(30.358332761838668 -94.7615798524443) bank38434 +38435 POINT(30.460435442823787 -96.0514747228745) bank38435 +38436 POINT(28.91952232364326 -94.89023014953084) bank38436 +38437 POINT(29.22559472049283 -95.83151754562294) bank38437 +38438 POINT(29.879303962518186 -95.54350829381589) bank38438 +38439 POINT(30.0581686300488 -95.07311711150564) bank38439 +38440 POINT(30.537674723064573 -94.91159458966068) bank38440 +38441 POINT(28.894534863427644 -96.05491176250315) bank38441 +38442 POINT(30.343370624732312 -94.67948804205895) bank38442 +38443 POINT(30.13877517860796 -94.36992154735063) bank38443 +38444 POINT(29.507731783225626 -94.88883603451507) bank38444 +38445 POINT(29.97168391378393 -95.01432791503537) bank38445 +38446 POINT(30.311113968085337 -94.50763026109293) bank38446 +38447 POINT(30.23917205595691 -95.70345454018278) bank38447 +38448 POINT(29.233227652072085 -95.09118354453221) bank38448 +38449 POINT(30.622227352218363 -94.87984018109383) bank38449 +38450 POINT(28.93337006126584 -94.7784612528314) bank38450 +38451 POINT(30.70383985770655 -95.70996210619084) bank38451 +38452 POINT(29.782273996980575 -95.85906268932283) bank38452 +38453 POINT(30.54125939996508 -94.47582072927077) bank38453 +38454 POINT(28.902443047064033 -94.95906073524439) bank38454 +38455 POINT(30.613311094369685 -95.89862085233635) bank38455 +38456 POINT(30.554659766891525 -94.48629808206445) bank38456 +38457 POINT(30.573462909451692 -96.12557114263475) bank38457 +38458 POINT(29.852960621569622 -95.94376027327094) bank38458 +38459 POINT(30.27279701038681 -95.20057949035828) bank38459 +38460 POINT(30.627504371986557 -94.59290850796661) bank38460 +38461 POINT(29.457501130642488 -95.24201099389448) bank38461 +38462 POINT(29.33755761266257 -95.0791336581442) bank38462 +38463 POINT(30.658103960975154 -94.75028170281472) bank38463 +38464 POINT(29.467849182868928 -96.3361846855034) bank38464 +38465 POINT(29.359624932358336 -96.31283425481371) bank38465 +38466 POINT(28.99518779835076 -94.59796542495411) bank38466 +38467 POINT(29.29262189594038 -95.97219958337016) bank38467 +38468 POINT(30.22835889659231 -94.42991196284201) bank38468 +38469 POINT(29.34553196279486 -94.37014018874069) bank38469 +38470 POINT(29.86355372873073 -95.70741598105853) bank38470 +38471 POINT(30.267939854723885 -95.19720837335622) bank38471 +38472 POINT(28.97839272458744 -95.23422955078803) bank38472 +38473 POINT(29.28765691046046 -95.41286882529677) bank38473 +38474 POINT(28.83262542824518 -96.25095667030445) bank38474 +38475 POINT(30.595665581421578 -95.10737098606495) bank38475 +38476 POINT(30.55921833429073 -94.89858858054251) bank38476 +38477 POINT(29.69779373976274 -96.12779036106775) bank38477 +38478 POINT(30.270909209182097 -95.48992171360959) bank38478 +38479 POINT(28.98762142894071 -96.32426839816311) bank38479 +38480 POINT(29.809779290620728 -95.4714986608747) bank38480 +38481 POINT(29.521388337164307 -94.498877059644) bank38481 +38482 POINT(29.09990166169875 -95.81589076437442) bank38482 +38483 POINT(29.007418143238695 -94.85498023122757) bank38483 +38484 POINT(30.612460610295322 -95.63236472530787) bank38484 +38485 POINT(29.005939334517496 -95.48666459820473) bank38485 +38486 POINT(29.674937143184685 -95.28211055148729) bank38486 +38487 POINT(29.721762794823633 -95.98729507588746) bank38487 +38488 POINT(29.61980889954855 -94.37486354197979) bank38488 +38489 POINT(29.534127096823678 -95.42724246459794) bank38489 +38490 POINT(29.38275077526668 -96.13074756029195) bank38490 +38491 POINT(28.77859201470129 -96.2933926561342) bank38491 +38492 POINT(29.252029053124726 -96.06590802586682) bank38492 +38493 POINT(29.200981047686305 -94.51875714550113) bank38493 +38494 POINT(29.394812904183574 -96.11455167068137) bank38494 +38495 POINT(29.551622180617475 -94.61820730451164) bank38495 +38496 POINT(29.48862873802389 -95.99544434664698) bank38496 +38497 POINT(30.319185035464162 -96.17820407465705) bank38497 +38498 POINT(29.056192731732274 -94.9669743673522) bank38498 +38499 POINT(29.957124125989758 -96.03423810379505) bank38499 +38500 POINT(29.283931078584523 -94.59717625033991) bank38500 +38501 POINT(29.97030788081205 -95.49412734173106) bank38501 +38502 POINT(30.18058308496129 -94.73833621412945) bank38502 +38503 POINT(29.389822914607223 -95.279022706057) bank38503 +38504 POINT(30.277557263497897 -95.52660411655705) bank38504 +38505 POINT(29.525391633004336 -96.2988023439685) bank38505 +38506 POINT(30.10004038375916 -95.6716203153868) bank38506 +38507 POINT(30.36332755288872 -95.1067364279761) bank38507 +38508 POINT(28.828720400912943 -95.52658682657349) bank38508 +38509 POINT(29.229148551827617 -96.30810130090555) bank38509 +38510 POINT(29.636354732646893 -94.7552563071065) bank38510 +38511 POINT(29.771367522530436 -96.03193497843291) bank38511 +38512 POINT(29.03268926890172 -95.24473003171273) bank38512 +38513 POINT(30.32212303205712 -94.65611304639478) bank38513 +38514 POINT(29.098100113001507 -94.42705568463995) bank38514 +38515 POINT(29.022631738029293 -94.77636998012645) bank38515 +38516 POINT(28.83764033569425 -96.27202387702731) bank38516 +38517 POINT(29.052840368222814 -94.63642614701311) bank38517 +38518 POINT(29.7431404222851 -94.93721406950185) bank38518 +38519 POINT(28.864493334485356 -94.83931791245327) bank38519 +38520 POINT(29.925967847767843 -96.21750341360298) bank38520 +38521 POINT(29.64992628575932 -94.3723413048447) bank38521 +38522 POINT(29.872170551300268 -95.41723335370271) bank38522 +38523 POINT(29.03039980355997 -95.80987270539234) bank38523 +38524 POINT(29.343874848232062 -95.42191632485256) bank38524 +38525 POINT(30.440054999780664 -95.19065610615897) bank38525 +38526 POINT(28.916224589727513 -95.47823738456358) bank38526 +38527 POINT(29.419518919009697 -95.88916075347885) bank38527 +38528 POINT(29.608475560745372 -96.11782856018145) bank38528 +38529 POINT(30.20229030395538 -94.80103789176358) bank38529 +38530 POINT(30.107829399932463 -95.74460359864605) bank38530 +38531 POINT(28.80681991837898 -96.06839205784932) bank38531 +38532 POINT(30.5144208563809 -94.54763427733667) bank38532 +38533 POINT(30.21884882833632 -95.3604025295938) bank38533 +38534 POINT(30.558792653731906 -96.29645916640361) bank38534 +38535 POINT(29.852497551183113 -95.43072232530389) bank38535 +38536 POINT(29.24864411474233 -95.61217223894434) bank38536 +38537 POINT(30.557174046943814 -95.06508317750988) bank38537 +38538 POINT(30.205830587913717 -95.19816379969579) bank38538 +38539 POINT(29.6390099604597 -96.16673521465376) bank38539 +38540 POINT(30.126832620115426 -94.77827856247556) bank38540 +38541 POINT(30.74719203781363 -95.77621110104954) bank38541 +38542 POINT(30.019577400498235 -95.71903060560426) bank38542 +38543 POINT(29.191289138165253 -96.06704767618507) bank38543 +38544 POINT(29.77199370230878 -94.75774696732205) bank38544 +38545 POINT(30.37394199019811 -96.19195121335008) bank38545 +38546 POINT(29.314171157451543 -95.15439501528587) bank38546 +38547 POINT(29.966208177662804 -94.64650304967353) bank38547 +38548 POINT(28.995911651471594 -95.65611441897715) bank38548 +38549 POINT(30.570419765932733 -96.29145294191308) bank38549 +38550 POINT(29.49849614196158 -96.02708360622009) bank38550 +38551 POINT(29.899630463769643 -94.86225918435667) bank38551 +38552 POINT(29.16272237380389 -95.34927678061977) bank38552 +38553 POINT(29.831271459903956 -96.06495328350756) bank38553 +38554 POINT(29.743185089499175 -95.20533391576738) bank38554 +38555 POINT(29.35968581344287 -96.06641027369439) bank38555 +38556 POINT(29.57319436549248 -94.50679868192813) bank38556 +38557 POINT(29.036467258484123 -95.41510685042633) bank38557 +38558 POINT(30.62700504789754 -94.57738954917029) bank38558 +38559 POINT(29.322693578935937 -96.22755865830429) bank38559 +38560 POINT(30.420109281508736 -95.61796838873185) bank38560 +38561 POINT(30.324057451253832 -94.88671419388955) bank38561 +38562 POINT(30.32171426500001 -94.87854412581275) bank38562 +38563 POINT(28.806043880199397 -95.01545260511178) bank38563 +38564 POINT(29.3668248165195 -95.65490778777092) bank38564 +38565 POINT(29.06027739636949 -94.64227360435287) bank38565 +38566 POINT(29.800354792623235 -94.51656047719325) bank38566 +38567 POINT(30.746593147735954 -95.73950335016471) bank38567 +38568 POINT(29.149535420471818 -94.46079647847714) bank38568 +38569 POINT(29.728136396508535 -95.97082085977455) bank38569 +38570 POINT(29.484627570228188 -94.74235436072243) bank38570 +38571 POINT(29.73541090836837 -95.36753650769874) bank38571 +38572 POINT(29.527277358942346 -95.8989129519453) bank38572 +38573 POINT(29.39434910291935 -95.74412469689753) bank38573 +38574 POINT(30.325445737176864 -95.61599015740059) bank38574 +38575 POINT(29.315347769369886 -95.4466523840773) bank38575 +38576 POINT(30.69561054481652 -95.06797786903556) bank38576 +38577 POINT(30.055810842214814 -95.95110877749221) bank38577 +38578 POINT(28.938561529026476 -95.83607957533651) bank38578 +38579 POINT(29.62381533266153 -95.61612688019348) bank38579 +38580 POINT(28.982508604016818 -95.28201196764448) bank38580 +38581 POINT(29.582480427915957 -94.43056480491904) bank38581 +38582 POINT(29.81402285256102 -95.35579724709548) bank38582 +38583 POINT(30.60400830241475 -94.6041889810146) bank38583 +38584 POINT(29.755174827768663 -94.59529697977759) bank38584 +38585 POINT(29.82818332679276 -95.48923579116816) bank38585 +38586 POINT(29.621428149492303 -94.51539629604729) bank38586 +38587 POINT(30.24938119750963 -94.43746274035064) bank38587 +38588 POINT(30.342964042805274 -96.32770884731103) bank38588 +38589 POINT(30.580184842267094 -95.41513897728306) bank38589 +38590 POINT(29.490793892958642 -95.40753529857103) bank38590 +38591 POINT(30.329257308898484 -95.43528184799261) bank38591 +38592 POINT(29.8213556995023 -95.37105877954244) bank38592 +38593 POINT(29.50191501099852 -96.08252831733314) bank38593 +38594 POINT(29.469663858198842 -96.11128260754278) bank38594 +38595 POINT(30.11547927371775 -94.74214266707727) bank38595 +38596 POINT(29.458454694334975 -95.39156885844984) bank38596 +38597 POINT(29.2028717819426 -96.22507195141081) bank38597 +38598 POINT(30.733042596390348 -96.03169634467257) bank38598 +38599 POINT(30.618436168259787 -95.33765840088209) bank38599 +38600 POINT(30.6138026358282 -94.42311073992697) bank38600 +38601 POINT(29.576737801979853 -94.76792806773683) bank38601 +38602 POINT(28.881929653159386 -94.79355216894474) bank38602 +38603 POINT(28.862026112775794 -96.06388389382532) bank38603 +38604 POINT(30.155885616105785 -94.54182107470011) bank38604 +38605 POINT(30.237254106243103 -95.34767544839183) bank38605 +38606 POINT(29.405346538057145 -95.83408089696744) bank38606 +38607 POINT(29.602726658143787 -95.28569575232136) bank38607 +38608 POINT(29.466453820319664 -94.8891266142798) bank38608 +38609 POINT(30.26899984166027 -94.63843011196953) bank38609 +38610 POINT(28.79541446195465 -95.90091594179711) bank38610 +38611 POINT(29.46825726040023 -95.81885129867666) bank38611 +38612 POINT(30.45069790841918 -94.98893455730126) bank38612 +38613 POINT(30.112153325220415 -94.90146840218607) bank38613 +38614 POINT(30.313543139508567 -95.98209437644473) bank38614 +38615 POINT(30.270131115456877 -95.73706927545551) bank38615 +38616 POINT(30.41529221979476 -95.20419479340063) bank38616 +38617 POINT(29.997299000938355 -95.1916708220274) bank38617 +38618 POINT(29.531988030732588 -96.2540006137795) bank38618 +38619 POINT(29.72114041892506 -95.4040305233799) bank38619 +38620 POINT(30.087005775738092 -95.94088133378939) bank38620 +38621 POINT(30.338974060198257 -94.41947066303311) bank38621 +38622 POINT(28.866077961573904 -94.62884740329939) bank38622 +38623 POINT(30.263161895899916 -94.61399381725353) bank38623 +38624 POINT(29.377719033514435 -96.20074036162335) bank38624 +38625 POINT(29.351429147785822 -95.15575966235792) bank38625 +38626 POINT(30.24721567893827 -94.46068413053774) bank38626 +38627 POINT(30.60008547847704 -95.1400352177171) bank38627 +38628 POINT(29.21533072087909 -96.11603531764969) bank38628 +38629 POINT(28.872312039998356 -96.0669405343121) bank38629 +38630 POINT(30.18465343356092 -95.87575401448598) bank38630 +38631 POINT(30.569848154219294 -94.74705907562964) bank38631 +38632 POINT(30.004177436815482 -95.51722742901535) bank38632 +38633 POINT(28.799099392671295 -94.97661561448692) bank38633 +38634 POINT(29.44768586469056 -95.21602501790564) bank38634 +38635 POINT(30.027312840750938 -94.74765007351029) bank38635 +38636 POINT(30.4603590009775 -94.61390441031911) bank38636 +38637 POINT(30.640733284001293 -95.16649788344193) bank38637 +38638 POINT(28.83178491725848 -94.50365025280885) bank38638 +38639 POINT(29.041135028170796 -95.02902817381393) bank38639 +38640 POINT(30.2153822593491 -96.1105939460025) bank38640 +38641 POINT(29.688373076583474 -96.30686688029239) bank38641 +38642 POINT(28.877194812740818 -95.32170316269796) bank38642 +38643 POINT(30.49174121708602 -95.83953710056299) bank38643 +38644 POINT(29.381329176728663 -95.8566550997106) bank38644 +38645 POINT(29.172410643455308 -94.7916907231858) bank38645 +38646 POINT(29.662592636510983 -96.05655619255482) bank38646 +38647 POINT(29.56154373906558 -94.44049697134271) bank38647 +38648 POINT(29.513702641840922 -94.99523366997005) bank38648 +38649 POINT(30.08573061813883 -95.63754098004415) bank38649 +38650 POINT(28.921911222681054 -94.79593948010121) bank38650 +38651 POINT(30.37231972168166 -94.73132617607979) bank38651 +38652 POINT(30.27203538288256 -96.35067497368155) bank38652 +38653 POINT(29.497546833736408 -95.32364907695131) bank38653 +38654 POINT(30.48260691254879 -96.32538156936857) bank38654 +38655 POINT(29.345370661554572 -96.01862736354063) bank38655 +38656 POINT(28.778023399688298 -95.00132600260959) bank38656 +38657 POINT(29.6874997929418 -94.65915511228899) bank38657 +38658 POINT(29.304260102745562 -94.40054746620223) bank38658 +38659 POINT(29.659525792380478 -95.53373628263307) bank38659 +38660 POINT(30.70453656771557 -96.0643003166702) bank38660 +38661 POINT(30.016519698955534 -96.03108710277421) bank38661 +38662 POINT(29.63998133734134 -94.70025073202798) bank38662 +38663 POINT(30.088534472794574 -95.40725127444647) bank38663 +38664 POINT(28.877618677505488 -94.40189193188505) bank38664 +38665 POINT(29.91273061266077 -95.4753593370446) bank38665 +38666 POINT(30.63302545388732 -94.88928230774378) bank38666 +38667 POINT(30.04094300904944 -95.36503809731491) bank38667 +38668 POINT(29.990253716282222 -95.39163578785308) bank38668 +38669 POINT(29.049770919013152 -96.24765581591242) bank38669 +38670 POINT(30.659692870714213 -95.95303181599795) bank38670 +38671 POINT(30.004221146850654 -94.75338763231588) bank38671 +38672 POINT(29.977747619851282 -96.30527505907298) bank38672 +38673 POINT(30.646865054333663 -95.58064951228619) bank38673 +38674 POINT(29.541802912502632 -94.82059219681125) bank38674 +38675 POINT(29.06111163846441 -95.05531280009865) bank38675 +38676 POINT(29.34738746666861 -96.02944781907597) bank38676 +38677 POINT(30.15921100446437 -96.17749549584816) bank38677 +38678 POINT(29.422413114850563 -94.63869862582206) bank38678 +38679 POINT(29.630591341683918 -94.95805856633469) bank38679 +38680 POINT(30.252290516332085 -95.86726363205874) bank38680 +38681 POINT(30.47696787804812 -95.95007605536348) bank38681 +38682 POINT(29.295707584922813 -94.47505572531368) bank38682 +38683 POINT(29.956344410276152 -95.02352964929949) bank38683 +38684 POINT(30.638628804582694 -94.99887587636846) bank38684 +38685 POINT(30.67375440981513 -94.99380393957533) bank38685 +38686 POINT(28.821078915871617 -95.75941401193654) bank38686 +38687 POINT(29.922429992466743 -94.66616548286481) bank38687 +38688 POINT(29.006490059134716 -95.3809322767819) bank38688 +38689 POINT(30.290589383364907 -94.38950247741538) bank38689 +38690 POINT(30.629674513157557 -96.09668531946896) bank38690 +38691 POINT(29.841131915908505 -94.4877874499437) bank38691 +38692 POINT(30.242266884782293 -95.90265009879501) bank38692 +38693 POINT(30.366909757905223 -95.86265779018902) bank38693 +38694 POINT(29.678624178901583 -94.74421667322396) bank38694 +38695 POINT(30.72013316775503 -96.30183655651938) bank38695 +38696 POINT(30.38844356333998 -94.70696755474788) bank38696 +38697 POINT(30.579408277217773 -95.73630596718434) bank38697 +38698 POINT(29.820665006314307 -94.5515715061009) bank38698 +38699 POINT(30.188575931425763 -96.18118523584835) bank38699 +38700 POINT(30.497456319023428 -95.16358538663742) bank38700 +38701 POINT(30.74396655788063 -95.0854797694004) bank38701 +38702 POINT(30.58928167429655 -96.09719720984528) bank38702 +38703 POINT(29.560134959500132 -95.92859687586072) bank38703 +38704 POINT(29.38443297756549 -95.37493611131573) bank38704 +38705 POINT(30.53521249284355 -94.9261393554313) bank38705 +38706 POINT(29.433073193532355 -95.3137928066429) bank38706 +38707 POINT(29.77212463100394 -94.55822571038793) bank38707 +38708 POINT(29.213135037915624 -96.28372924956464) bank38708 +38709 POINT(29.818273931926946 -95.26996729537942) bank38709 +38710 POINT(30.016384467751184 -96.04621919834906) bank38710 +38711 POINT(29.04693062891045 -96.33330945742804) bank38711 +38712 POINT(29.588880111155458 -95.47417255240265) bank38712 +38713 POINT(29.12859782772872 -95.65653918846017) bank38713 +38714 POINT(28.81493656260578 -95.08774193304933) bank38714 +38715 POINT(30.465078790407535 -94.7620586231871) bank38715 +38716 POINT(30.051523147524282 -96.18674165818516) bank38716 +38717 POINT(29.04852299586079 -95.4702682818996) bank38717 +38718 POINT(29.89563323319577 -94.92876683164218) bank38718 +38719 POINT(29.286798383926193 -96.04478087589898) bank38719 +38720 POINT(29.1829938897344 -94.53191547845688) bank38720 +38721 POINT(29.74341242673996 -94.74227688501296) bank38721 +38722 POINT(29.3135654915327 -94.94317986153453) bank38722 +38723 POINT(30.375232373103785 -95.51685011589473) bank38723 +38724 POINT(29.02826295277513 -96.25949825829511) bank38724 +38725 POINT(29.131855579724657 -95.77186151347374) bank38725 +38726 POINT(28.763452469080296 -95.41815487506892) bank38726 +38727 POINT(29.240114621940492 -95.83551556413795) bank38727 +38728 POINT(28.86168420975096 -95.63471173188051) bank38728 +38729 POINT(29.538280155257326 -95.55596721342467) bank38729 +38730 POINT(30.208059641942036 -94.68960996108623) bank38730 +38731 POINT(30.49652507310638 -95.64854924046783) bank38731 +38732 POINT(30.72480110638379 -96.21957721520874) bank38732 +38733 POINT(30.02452765324383 -95.8130166760266) bank38733 +38734 POINT(30.621054004040715 -94.86670865466361) bank38734 +38735 POINT(29.11471067113989 -95.10914368915115) bank38735 +38736 POINT(29.023903164484476 -95.19084087859112) bank38736 +38737 POINT(29.792525258721643 -94.64346980398123) bank38737 +38738 POINT(29.24351619091367 -95.08901975102393) bank38738 +38739 POINT(29.93880820217137 -95.74069628558927) bank38739 +38740 POINT(30.409946043327697 -96.13355645385882) bank38740 +38741 POINT(30.19890849183381 -95.33386968729661) bank38741 +38742 POINT(30.12214219440658 -94.53053456688946) bank38742 +38743 POINT(28.99115803325477 -96.36191264941316) bank38743 +38744 POINT(29.85620923130384 -94.82676105355368) bank38744 +38745 POINT(30.67329465358577 -95.94948448358551) bank38745 +38746 POINT(30.175033036640308 -96.09666349086861) bank38746 +38747 POINT(30.48072752963674 -94.86609207551517) bank38747 +38748 POINT(28.77174211022709 -94.91538517853878) bank38748 +38749 POINT(30.51870918551078 -95.22791610367743) bank38749 +38750 POINT(29.532858490298736 -95.8901936470924) bank38750 +38751 POINT(29.105806185262182 -94.95920171038425) bank38751 +38752 POINT(30.30530607762422 -95.25205958336016) bank38752 +38753 POINT(30.62898367488082 -95.33939104780026) bank38753 +38754 POINT(28.95931778833393 -95.52406003817424) bank38754 +38755 POINT(30.42688218617526 -95.75853674410271) bank38755 +38756 POINT(30.167452975363386 -94.50271558878168) bank38756 +38757 POINT(29.375252497470825 -96.11581768324355) bank38757 +38758 POINT(30.221809761460342 -94.76456836416014) bank38758 +38759 POINT(29.785297449669603 -95.74373569145268) bank38759 +38760 POINT(29.18775131066935 -96.22159148321896) bank38760 +38761 POINT(29.217164915412592 -95.26840625066775) bank38761 +38762 POINT(29.228814838017243 -95.69958882677918) bank38762 +38763 POINT(29.365396004621036 -94.70169628824269) bank38763 +38764 POINT(28.83053179712393 -94.45871402343687) bank38764 +38765 POINT(29.219112852555114 -94.84651997103742) bank38765 +38766 POINT(30.509030482900986 -94.82120440435126) bank38766 +38767 POINT(29.1946590726822 -95.10163984971187) bank38767 +38768 POINT(30.71955506888574 -95.71527827281157) bank38768 +38769 POINT(29.874357755149372 -95.35892469848807) bank38769 +38770 POINT(29.426577117475652 -94.76825748464505) bank38770 +38771 POINT(29.087061306472975 -95.79082384286063) bank38771 +38772 POINT(30.289414568886873 -95.72361342740706) bank38772 +38773 POINT(29.315901515961812 -96.36368652650914) bank38773 +38774 POINT(28.825574916931835 -95.788554282841) bank38774 +38775 POINT(28.83153540550684 -94.80294882058314) bank38775 +38776 POINT(30.304984949633 -95.03570675160437) bank38776 +38777 POINT(28.874364900717058 -95.55412940686296) bank38777 +38778 POINT(30.429383244038416 -94.79416337384501) bank38778 +38779 POINT(28.77030725096948 -96.0130630950967) bank38779 +38780 POINT(30.631128658136042 -95.85281393837538) bank38780 +38781 POINT(29.075496974032795 -96.18839726848573) bank38781 +38782 POINT(29.836461401554114 -94.65031816539468) bank38782 +38783 POINT(29.542570168469588 -94.83876870302781) bank38783 +38784 POINT(29.882732765218094 -94.95558209045618) bank38784 +38785 POINT(29.125391318131534 -94.89552935819155) bank38785 +38786 POINT(30.13711365456398 -95.05266519448973) bank38786 +38787 POINT(29.226440474561706 -95.49844726346191) bank38787 +38788 POINT(30.75380121985178 -95.60898741124882) bank38788 +38789 POINT(29.596495160030493 -95.51617886798043) bank38789 +38790 POINT(29.167347664844023 -94.42852729506416) bank38790 +38791 POINT(30.678564140198116 -94.90662122599967) bank38791 +38792 POINT(28.801186479004755 -94.51751257882854) bank38792 +38793 POINT(29.18927326228279 -95.08986498979836) bank38793 +38794 POINT(30.49680991058868 -95.76184317975951) bank38794 +38795 POINT(28.807792283787943 -94.76695428608453) bank38795 +38796 POINT(30.46645310384823 -96.20753249119278) bank38796 +38797 POINT(29.695650807437 -94.48454116023977) bank38797 +38798 POINT(29.790174726127315 -95.00800658289417) bank38798 +38799 POINT(30.369882519006307 -94.88216133674703) bank38799 +38800 POINT(29.55190170141162 -95.48571288656514) bank38800 +38801 POINT(29.95412791662473 -95.47198057465069) bank38801 +38802 POINT(29.819240810990273 -96.32007072804961) bank38802 +38803 POINT(30.32213237716506 -96.3598144693112) bank38803 +38804 POINT(29.905167409018613 -94.83110370225205) bank38804 +38805 POINT(30.466758048595327 -95.41332926976072) bank38805 +38806 POINT(29.616659842823548 -96.24237793768414) bank38806 +38807 POINT(30.25488945364397 -95.25393721116849) bank38807 +38808 POINT(29.959434909857592 -95.20366663907983) bank38808 +38809 POINT(30.595931522835613 -95.19606901188662) bank38809 +38810 POINT(29.851809836734155 -94.802878860919) bank38810 +38811 POINT(30.367611697178535 -95.9695172115336) bank38811 +38812 POINT(30.708128535275257 -94.704724756064) bank38812 +38813 POINT(30.630246910527298 -95.16630891475029) bank38813 +38814 POINT(28.9668427491529 -96.24220399991276) bank38814 +38815 POINT(30.19840835733028 -95.55566209729592) bank38815 +38816 POINT(29.264255681141492 -95.28028071183566) bank38816 +38817 POINT(29.216212516150065 -95.67291499004986) bank38817 +38818 POINT(30.13558880665037 -94.65013781746939) bank38818 +38819 POINT(29.37648797910904 -94.81627449586928) bank38819 +38820 POINT(29.505626903343984 -94.53288582945423) bank38820 +38821 POINT(28.91120195367428 -94.50146800998797) bank38821 +38822 POINT(30.588790409687135 -95.21221648452253) bank38822 +38823 POINT(28.9574337633838 -94.84578763623942) bank38823 +38824 POINT(29.509905499950257 -95.83515518197638) bank38824 +38825 POINT(30.122279874477464 -95.05794173654135) bank38825 +38826 POINT(29.024838285600932 -94.99877888300067) bank38826 +38827 POINT(29.325840972256934 -94.49290202591706) bank38827 +38828 POINT(30.42531045806763 -95.34762713353287) bank38828 +38829 POINT(30.3689703807821 -95.76208208734597) bank38829 +38830 POINT(30.746024137475946 -94.44769058318667) bank38830 +38831 POINT(30.731746836213926 -95.58358622849475) bank38831 +38832 POINT(29.00131078224673 -95.10562399804887) bank38832 +38833 POINT(28.868214325094677 -94.73616714304005) bank38833 +38834 POINT(30.488218944211557 -95.15966427675508) bank38834 +38835 POINT(30.41743182596791 -94.49596498840067) bank38835 +38836 POINT(29.589283730713447 -96.1495076427956) bank38836 +38837 POINT(28.848610396201604 -96.11257318500881) bank38837 +38838 POINT(30.534707552073492 -95.65686199579888) bank38838 +38839 POINT(29.404332017763707 -95.6226387860638) bank38839 +38840 POINT(29.383377764269962 -96.11178913722867) bank38840 +38841 POINT(29.64831521478878 -96.18007782156505) bank38841 +38842 POINT(29.60499693730763 -95.19412574701971) bank38842 +38843 POINT(29.597117364428197 -95.02920486121184) bank38843 +38844 POINT(29.238647826951983 -94.95881062917555) bank38844 +38845 POINT(28.947321387732284 -95.81708891144858) bank38845 +38846 POINT(29.75779076428485 -95.68055660609022) bank38846 +38847 POINT(30.623000139809996 -94.5588837489072) bank38847 +38848 POINT(28.86688569061165 -95.37840658542248) bank38848 +38849 POINT(29.203527524247633 -95.96815520931847) bank38849 +38850 POINT(29.09333189235015 -94.60967715318972) bank38850 +38851 POINT(29.65129555458513 -95.48241391034223) bank38851 +38852 POINT(30.092795682379226 -95.61971785658152) bank38852 +38853 POINT(30.41403757724738 -95.9773490806634) bank38853 +38854 POINT(30.047627529986922 -95.72819110711596) bank38854 +38855 POINT(29.14615914424832 -94.62832391925083) bank38855 +38856 POINT(29.41053472371246 -96.14387054750998) bank38856 +38857 POINT(29.106922231063297 -96.09677017036994) bank38857 +38858 POINT(30.271211501282323 -94.87531360435769) bank38858 +38859 POINT(30.460478652786957 -95.3980742141163) bank38859 +38860 POINT(30.522686640247088 -95.89179068282996) bank38860 +38861 POINT(30.73725352423271 -94.40880155070508) bank38861 +38862 POINT(30.337452407563877 -95.42126129860931) bank38862 +38863 POINT(29.929414995540007 -94.8747281506199) bank38863 +38864 POINT(29.75097132788998 -94.41089270400205) bank38864 +38865 POINT(29.456949506313308 -95.22975868668874) bank38865 +38866 POINT(29.816326452797068 -96.14692797230155) bank38866 +38867 POINT(29.730243279605933 -96.06148053529273) bank38867 +38868 POINT(30.440528648112704 -94.62542245848424) bank38868 +38869 POINT(29.914473273164262 -96.33324470906969) bank38869 +38870 POINT(29.40490267956718 -96.13968280136875) bank38870 +38871 POINT(29.828473751808154 -94.88708428677636) bank38871 +38872 POINT(30.746862928780345 -95.45538591512573) bank38872 +38873 POINT(30.134581894650445 -95.78954937730506) bank38873 +38874 POINT(29.425649213922256 -95.22609152972888) bank38874 +38875 POINT(30.440846781937907 -96.23282072315689) bank38875 +38876 POINT(30.698629558083983 -95.69311619349727) bank38876 +38877 POINT(29.69963744594983 -95.03205777090176) bank38877 +38878 POINT(30.590029424252084 -95.05721564681527) bank38878 +38879 POINT(29.882667893193045 -94.66754896508512) bank38879 +38880 POINT(29.05056132084882 -94.83259676292262) bank38880 +38881 POINT(29.35980681634891 -96.15458371021) bank38881 +38882 POINT(29.668672851390564 -95.31417671919728) bank38882 +38883 POINT(29.676763171517187 -94.89296404259977) bank38883 +38884 POINT(30.445566010143743 -95.50964340732429) bank38884 +38885 POINT(30.351862128150323 -96.21287635642845) bank38885 +38886 POINT(30.013754359017206 -95.78370872614627) bank38886 +38887 POINT(28.797348469359783 -94.59563236560257) bank38887 +38888 POINT(30.32773608554234 -95.97864344099855) bank38888 +38889 POINT(29.758120937750782 -94.83804425051186) bank38889 +38890 POINT(29.543305645784407 -94.78437288144035) bank38890 +38891 POINT(29.22701783527467 -96.2838679309217) bank38891 +38892 POINT(29.715834437650624 -94.7272758142988) bank38892 +38893 POINT(29.06546524439503 -95.56790672426762) bank38893 +38894 POINT(30.365451172491767 -95.77868045659476) bank38894 +38895 POINT(30.16212261603596 -94.48838989119307) bank38895 +38896 POINT(30.485491889583585 -94.95807103100965) bank38896 +38897 POINT(30.340045592079484 -95.43981681034968) bank38897 +38898 POINT(29.824983965061875 -95.63894398755075) bank38898 +38899 POINT(29.803308579011983 -94.71390457397901) bank38899 +38900 POINT(30.143436514904003 -95.0321873248821) bank38900 +38901 POINT(30.103680610515536 -94.42921125982964) bank38901 +38902 POINT(28.807865004472916 -94.75534855246353) bank38902 +38903 POINT(29.249610622937624 -95.12927741527527) bank38903 +38904 POINT(29.27988757770097 -95.58742283168041) bank38904 +38905 POINT(30.560369874905085 -96.09770902897421) bank38905 +38906 POINT(29.855510238967895 -95.48894592866321) bank38906 +38907 POINT(29.109493549861618 -96.17507871865625) bank38907 +38908 POINT(28.86517078299604 -95.51838031453516) bank38908 +38909 POINT(29.785685595822994 -95.1071108289294) bank38909 +38910 POINT(30.225400624105323 -94.93896426597568) bank38910 +38911 POINT(30.404599864288123 -94.45189232129697) bank38911 +38912 POINT(29.219965108469427 -94.76364787602836) bank38912 +38913 POINT(29.257569577762933 -96.30885188941907) bank38913 +38914 POINT(30.62536580603984 -94.72146557287395) bank38914 +38915 POINT(30.23320991519417 -94.94249750929598) bank38915 +38916 POINT(29.179681740521673 -95.0482162097903) bank38916 +38917 POINT(29.20454566186701 -94.6958166510672) bank38917 +38918 POINT(29.599363894687826 -94.93498648706831) bank38918 +38919 POINT(28.998109565837762 -95.85501831662184) bank38919 +38920 POINT(30.74379995608522 -94.70905424170012) bank38920 +38921 POINT(29.272152614926412 -95.255749829418) bank38921 +38922 POINT(30.02685151505382 -94.55666206752466) bank38922 +38923 POINT(30.734402770469835 -96.00989509929002) bank38923 +38924 POINT(29.595411062412275 -94.57594309646247) bank38924 +38925 POINT(30.66865758190816 -95.66357700736738) bank38925 +38926 POINT(30.07710061480142 -94.41516178826349) bank38926 +38927 POINT(29.63730878220941 -95.93252031791276) bank38927 +38928 POINT(28.955855182854634 -95.27652061405568) bank38928 +38929 POINT(30.215123734759633 -94.70255711404961) bank38929 +38930 POINT(30.739433086920087 -96.2495261510287) bank38930 +38931 POINT(29.504787716886558 -94.91458716471561) bank38931 +38932 POINT(29.688712288854386 -95.97303097093952) bank38932 +38933 POINT(30.443159508999564 -94.87524431836277) bank38933 +38934 POINT(29.81598671560474 -95.06976015751299) bank38934 +38935 POINT(29.95074872812193 -95.51013281380249) bank38935 +38936 POINT(29.876468456287775 -95.15622487487006) bank38936 +38937 POINT(29.891953166195915 -95.26571875253387) bank38937 +38938 POINT(29.769751448833293 -96.17730861451105) bank38938 +38939 POINT(29.97311649559993 -94.5040486371359) bank38939 +38940 POINT(30.66616364390778 -94.90086866910538) bank38940 +38941 POINT(28.85574652659239 -95.11349845958556) bank38941 +38942 POINT(30.071103823970596 -95.10697885899953) bank38942 +38943 POINT(29.064991142316973 -94.93854246226392) bank38943 +38944 POINT(29.153804062168902 -95.29669227689224) bank38944 +38945 POINT(30.236404608276498 -94.62119840145077) bank38945 +38946 POINT(29.364315040669805 -96.34243837269712) bank38946 +38947 POINT(30.572519027001793 -94.61907108677937) bank38947 +38948 POINT(29.310595718990427 -94.83842839720093) bank38948 +38949 POINT(30.08708985763214 -95.47024798510157) bank38949 +38950 POINT(29.70610578161764 -94.4660496280328) bank38950 +38951 POINT(30.293294976488834 -96.27735889377496) bank38951 +38952 POINT(30.47390159966525 -95.07535508474976) bank38952 +38953 POINT(29.651945582723926 -95.01758855342746) bank38953 +38954 POINT(29.476029240721477 -95.10291647677397) bank38954 +38955 POINT(29.369034729057223 -95.43787122931857) bank38955 +38956 POINT(29.086073597984754 -95.79609374682593) bank38956 +38957 POINT(30.035967970929477 -96.31738121936775) bank38957 +38958 POINT(30.726213360314972 -95.861431537861) bank38958 +38959 POINT(29.924901018019597 -95.84239917148012) bank38959 +38960 POINT(30.30306826439814 -94.58058798697647) bank38960 +38961 POINT(29.74533331805097 -95.51980772706975) bank38961 +38962 POINT(30.423440341751878 -95.0332871933974) bank38962 +38963 POINT(28.857068387698764 -95.18134126928398) bank38963 +38964 POINT(30.19021102313881 -94.61651092711939) bank38964 +38965 POINT(29.200976234029074 -94.95229211041668) bank38965 +38966 POINT(28.956539190051775 -96.18892792684281) bank38966 +38967 POINT(29.17184737251055 -94.39427999529069) bank38967 +38968 POINT(28.81786168253711 -95.97181916482789) bank38968 +38969 POINT(29.671098173724836 -96.27796344188252) bank38969 +38970 POINT(30.61062598105235 -94.94535995138789) bank38970 +38971 POINT(30.24147422321621 -94.92219758783968) bank38971 +38972 POINT(29.863683854273372 -96.16542462879124) bank38972 +38973 POINT(30.571146772684333 -96.16487456149771) bank38973 +38974 POINT(30.502615833820364 -94.98391782872335) bank38974 +38975 POINT(29.56358340386195 -96.28800775251132) bank38975 +38976 POINT(30.202427717924596 -94.65232617543617) bank38976 +38977 POINT(29.557151048778547 -95.60177015854418) bank38977 +38978 POINT(30.404460206812878 -96.15543089835516) bank38978 +38979 POINT(29.541883430156034 -95.82670003531169) bank38979 +38980 POINT(30.173802893451796 -95.5400341715179) bank38980 +38981 POINT(30.180833001215593 -94.95390978876324) bank38981 +38982 POINT(28.92754794214315 -94.88407306391714) bank38982 +38983 POINT(29.50434961944156 -95.37160677264406) bank38983 +38984 POINT(30.68724424768002 -95.19116169912499) bank38984 +38985 POINT(30.251514051062077 -95.33029801924651) bank38985 +38986 POINT(29.53215489911672 -96.12970625490712) bank38986 +38987 POINT(29.895581208084018 -95.47674248627979) bank38987 +38988 POINT(29.787641421900542 -95.88957410020234) bank38988 +38989 POINT(28.839176762783175 -95.186836801322) bank38989 +38990 POINT(30.72065855559759 -96.29264746764112) bank38990 +38991 POINT(29.725417162881005 -95.78250452399382) bank38991 +38992 POINT(29.023895165455507 -95.46663837266938) bank38992 +38993 POINT(28.997516591505608 -95.78101801588997) bank38993 +38994 POINT(29.068712945023947 -95.94889726996462) bank38994 +38995 POINT(30.523513327273328 -94.63241668629128) bank38995 +38996 POINT(30.130050653470747 -94.50313255585932) bank38996 +38997 POINT(29.21025770709915 -95.45981293556044) bank38997 +38998 POINT(30.191191190609125 -94.51341229832063) bank38998 +38999 POINT(29.73742535260034 -95.7840180780005) bank38999 +39000 POINT(29.732789522474334 -94.7292556292201) bank39000 +39001 POINT(30.54464734643236 -95.29947483803397) bank39001 +39002 POINT(30.63393647968241 -95.62690171928803) bank39002 +39003 POINT(29.82842341683784 -96.16312791191012) bank39003 +39004 POINT(29.88657280252842 -94.88155334141742) bank39004 +39005 POINT(30.692466142947975 -95.1997919933746) bank39005 +39006 POINT(30.427076994996685 -95.54053764962259) bank39006 +39007 POINT(30.21515094493981 -95.24450265542121) bank39007 +39008 POINT(29.27974721799219 -94.89106549306659) bank39008 +39009 POINT(29.36720039828156 -96.35910193618734) bank39009 +39010 POINT(28.940302750139047 -95.8095385475984) bank39010 +39011 POINT(29.13207484427116 -96.03845285364129) bank39011 +39012 POINT(29.03647463773848 -95.84980043223462) bank39012 +39013 POINT(29.76661961906207 -96.36309032260029) bank39013 +39014 POINT(29.751861790894928 -94.83295873472427) bank39014 +39015 POINT(29.81524510726196 -94.76042106840485) bank39015 +39016 POINT(28.78604241649137 -95.25219120030631) bank39016 +39017 POINT(29.61289852730006 -94.49102503394448) bank39017 +39018 POINT(30.285075869592514 -94.67963925575056) bank39018 +39019 POINT(29.810240888129357 -95.7765009088514) bank39019 +39020 POINT(29.287738416037477 -94.41365649849321) bank39020 +39021 POINT(30.56304298411564 -95.18967686176498) bank39021 +39022 POINT(29.48444728295756 -96.15721705495442) bank39022 +39023 POINT(30.452192192967498 -94.77929662560976) bank39023 +39024 POINT(30.227231085197335 -95.25167763289244) bank39024 +39025 POINT(29.66531723620161 -95.87183594470362) bank39025 +39026 POINT(29.972600995479425 -94.55133991664204) bank39026 +39027 POINT(28.992590407198758 -94.55117171378515) bank39027 +39028 POINT(29.874073120736853 -94.48363572849854) bank39028 +39029 POINT(29.97019721364566 -95.70427701997635) bank39029 +39030 POINT(30.315281753684772 -95.7348094523711) bank39030 +39031 POINT(30.649026651771713 -94.43288810482665) bank39031 +39032 POINT(30.220661822386223 -94.44769752966594) bank39032 +39033 POINT(30.7230633642736 -94.77465994319495) bank39033 +39034 POINT(30.706676908990097 -95.99909034477528) bank39034 +39035 POINT(29.553125967181504 -95.37707959599126) bank39035 +39036 POINT(29.167134224930738 -96.11746305063659) bank39036 +39037 POINT(29.761571869451284 -94.58666432195703) bank39037 +39038 POINT(30.17687019895086 -96.11617672857454) bank39038 +39039 POINT(29.752094232617022 -96.02211618886815) bank39039 +39040 POINT(30.290102418251784 -95.90405620522738) bank39040 +39041 POINT(30.53911164909354 -94.8627737880007) bank39041 +39042 POINT(29.318410313809682 -95.83434514498077) bank39042 +39043 POINT(29.124204459689956 -95.18540661463132) bank39043 +39044 POINT(30.09364385819979 -95.24890402316535) bank39044 +39045 POINT(29.273612973278592 -94.53298683872514) bank39045 +39046 POINT(29.428837305754406 -94.80531992553489) bank39046 +39047 POINT(29.77132697401118 -96.03777373721232) bank39047 +39048 POINT(29.543857088876944 -94.87695122769416) bank39048 +39049 POINT(29.699344572533796 -95.91133229301188) bank39049 +39050 POINT(30.713824209569715 -95.04331764043968) bank39050 +39051 POINT(28.832488589190493 -95.30109872412343) bank39051 +39052 POINT(29.08118792108021 -95.18669198601819) bank39052 +39053 POINT(29.48788589614537 -94.55241796582064) bank39053 +39054 POINT(30.12436429446166 -95.66696227392673) bank39054 +39055 POINT(29.527414031396585 -94.91872938832566) bank39055 +39056 POINT(30.286906314195424 -95.25084901273208) bank39056 +39057 POINT(30.529416016699884 -94.67768136745838) bank39057 +39058 POINT(29.986263472324822 -94.56162539133462) bank39058 +39059 POINT(29.177498381743693 -95.63203823250645) bank39059 +39060 POINT(30.278701185619884 -95.48982607993219) bank39060 +39061 POINT(28.903612746568047 -96.19516536034178) bank39061 +39062 POINT(30.500902935794965 -95.3251687126646) bank39062 +39063 POINT(29.226487381715415 -96.15394299066347) bank39063 +39064 POINT(29.623655303093642 -94.69883863202331) bank39064 +39065 POINT(29.694155859104345 -95.8129778645252) bank39065 +39066 POINT(30.60582221006195 -94.56048803654211) bank39066 +39067 POINT(29.33526940013779 -94.3901830188716) bank39067 +39068 POINT(29.501631990836508 -94.67504916990373) bank39068 +39069 POINT(30.40061496304264 -94.55454506775109) bank39069 +39070 POINT(29.078879533433 -95.47911818075885) bank39070 +39071 POINT(29.23800154542449 -94.73156826857198) bank39071 +39072 POINT(30.317606387075454 -94.61730304011196) bank39072 +39073 POINT(29.380243500874116 -95.101148596364) bank39073 +39074 POINT(29.875602218728936 -94.56672766006064) bank39074 +39075 POINT(29.489016822953765 -95.93925963755976) bank39075 +39076 POINT(30.24502682611338 -96.18023622535533) bank39076 +39077 POINT(30.249393904946473 -95.38097082311069) bank39077 +39078 POINT(29.052581806851773 -95.37110587998438) bank39078 +39079 POINT(28.80052395015402 -96.07792236239943) bank39079 +39080 POINT(29.161875248050535 -96.2801975973754) bank39080 +39081 POINT(28.782247037748323 -95.68031780335104) bank39081 +39082 POINT(29.873014399556805 -94.90054811245528) bank39082 +39083 POINT(28.81866405898707 -95.24344135386384) bank39083 +39084 POINT(30.478503842222135 -94.56173496483821) bank39084 +39085 POINT(29.03651339825994 -94.91886903845119) bank39085 +39086 POINT(28.864226988977464 -95.17562787245592) bank39086 +39087 POINT(30.3244788060133 -95.91049788807982) bank39087 +39088 POINT(28.846584393895167 -95.42200838244952) bank39088 +39089 POINT(30.29714315264303 -96.21613240898094) bank39089 +39090 POINT(28.992556531257435 -96.09288755120302) bank39090 +39091 POINT(29.20391583040535 -95.45566199224596) bank39091 +39092 POINT(29.48298943192933 -94.97068816170564) bank39092 +39093 POINT(30.4681636713305 -94.73660773164701) bank39093 +39094 POINT(29.388504024345213 -94.47809535609849) bank39094 +39095 POINT(28.859212416306715 -96.18410929896628) bank39095 +39096 POINT(29.295599588259556 -94.78455658673717) bank39096 +39097 POINT(30.345325242493935 -94.44116492141312) bank39097 +39098 POINT(30.5682532942808 -94.88106000419678) bank39098 +39099 POINT(30.252769048108007 -96.15560739767501) bank39099 +39100 POINT(29.780718895409688 -96.00127012115676) bank39100 +39101 POINT(30.01976439039298 -95.96706206513574) bank39101 +39102 POINT(30.507807831948444 -94.92838803156549) bank39102 +39103 POINT(30.66137294366574 -94.75226556862876) bank39103 +39104 POINT(30.50937756029357 -95.99589294622747) bank39104 +39105 POINT(28.88532496543982 -94.79421088646465) bank39105 +39106 POINT(30.34187803169561 -95.40429539910598) bank39106 +39107 POINT(29.613555279169255 -95.45993663637448) bank39107 +39108 POINT(29.922693258422182 -94.47346402570018) bank39108 +39109 POINT(29.81612691136842 -95.15015545354709) bank39109 +39110 POINT(30.75900016261919 -95.5744551852681) bank39110 +39111 POINT(29.994459977718094 -96.15281246142148) bank39111 +39112 POINT(29.964757174629657 -95.40590998314121) bank39112 +39113 POINT(29.540414904280816 -94.75290360804885) bank39113 +39114 POINT(29.315594581942705 -96.18968553756552) bank39114 +39115 POINT(29.946768485307327 -94.69366089836464) bank39115 +39116 POINT(30.514654455462644 -96.17171060274804) bank39116 +39117 POINT(28.848337253780723 -96.09476953653842) bank39117 +39118 POINT(28.85402481575386 -95.26814637927856) bank39118 +39119 POINT(29.531299669278777 -95.28182928013547) bank39119 +39120 POINT(30.301121549727096 -95.0995321168371) bank39120 +39121 POINT(29.873471216190925 -95.75646823039801) bank39121 +39122 POINT(30.731347884135072 -95.00727821651587) bank39122 +39123 POINT(30.654051592525686 -95.28538922019155) bank39123 +39124 POINT(30.293566814662277 -94.75237700832658) bank39124 +39125 POINT(28.777791918189614 -94.87244085680307) bank39125 +39126 POINT(29.303682475032684 -96.30860129061861) bank39126 +39127 POINT(29.168717165665456 -96.1759771166365) bank39127 +39128 POINT(29.019114985714477 -95.05837888552533) bank39128 +39129 POINT(30.513050670606514 -95.93339612599948) bank39129 +39130 POINT(28.7778131710506 -95.5776838161979) bank39130 +39131 POINT(29.119057731737982 -95.7396402635719) bank39131 +39132 POINT(29.52956015892949 -94.38556599215065) bank39132 +39133 POINT(29.122921442476297 -94.3758487717521) bank39133 +39134 POINT(29.796284617362353 -94.51498482610187) bank39134 +39135 POINT(30.531096286793872 -94.42988846488073) bank39135 +39136 POINT(29.924678527256326 -96.16590950682402) bank39136 +39137 POINT(30.477530439974686 -96.30823449797282) bank39137 +39138 POINT(30.347383268019456 -96.11806635639802) bank39138 +39139 POINT(30.34964813259141 -94.56222661771555) bank39139 +39140 POINT(30.006637976501334 -96.26050061318571) bank39140 +39141 POINT(29.580126038192603 -94.70153591517051) bank39141 +39142 POINT(30.00761497415541 -94.59952330216542) bank39142 +39143 POINT(30.158700371874232 -95.39632210435518) bank39143 +39144 POINT(29.045388257656096 -96.19816577525131) bank39144 +39145 POINT(29.611487908256702 -94.6665123641013) bank39145 +39146 POINT(29.929568479203496 -95.02065749845693) bank39146 +39147 POINT(29.671719087661636 -95.99198919168555) bank39147 +39148 POINT(29.175813325214804 -94.91760696884849) bank39148 +39149 POINT(29.90655625895433 -95.15948033945855) bank39149 +39150 POINT(29.336362171379157 -94.87319726699627) bank39150 +39151 POINT(28.949944781949256 -95.09104896933177) bank39151 +39152 POINT(30.16786631483104 -95.83029923146475) bank39152 +39153 POINT(29.79801826040177 -94.71378260538651) bank39153 +39154 POINT(30.573732312460304 -95.3239814228859) bank39154 +39155 POINT(29.089715127838655 -94.85029677997004) bank39155 +39156 POINT(29.282412106617162 -95.75309341928849) bank39156 +39157 POINT(30.023681606858577 -95.54586604002348) bank39157 +39158 POINT(30.617604109513717 -94.57439121377055) bank39158 +39159 POINT(29.26994025083496 -95.65313100056301) bank39159 +39160 POINT(28.894394155787154 -94.93921742198192) bank39160 +39161 POINT(29.62185819214266 -95.75454379275789) bank39161 +39162 POINT(28.935834163232542 -95.86684128583286) bank39162 +39163 POINT(30.01143079185047 -95.54373137236169) bank39163 +39164 POINT(29.898325261008075 -96.0294637366102) bank39164 +39165 POINT(30.111836422299028 -95.54887765532587) bank39165 +39166 POINT(29.000916576751855 -95.68549799051335) bank39166 +39167 POINT(28.85210443769574 -94.75366356990055) bank39167 +39168 POINT(29.120606929314803 -96.21847971146927) bank39168 +39169 POINT(29.911677440111553 -95.5263741861923) bank39169 +39170 POINT(29.484288764656135 -94.94076681735795) bank39170 +39171 POINT(30.7293920977406 -95.82549409145896) bank39171 +39172 POINT(29.396263025310965 -94.90649085663385) bank39172 +39173 POINT(30.693374705088527 -95.6056115869958) bank39173 +39174 POINT(29.952046756325576 -95.41334652961493) bank39174 +39175 POINT(29.950683103000156 -94.71842917591414) bank39175 +39176 POINT(30.13037859191518 -95.73808912261349) bank39176 +39177 POINT(28.89683204027461 -95.04191367708626) bank39177 +39178 POINT(30.617556182158797 -94.42716170163393) bank39178 +39179 POINT(30.11809861108533 -94.93808780897956) bank39179 +39180 POINT(29.220401937378337 -95.08948702372182) bank39180 +39181 POINT(30.335777385039233 -94.46482247491046) bank39181 +39182 POINT(29.85853118866736 -94.53565048224831) bank39182 +39183 POINT(29.1554817249405 -96.03681116806831) bank39183 +39184 POINT(30.754555895642532 -95.56377363086845) bank39184 +39185 POINT(29.461741646034675 -95.36017313738635) bank39185 +39186 POINT(28.806046089146037 -96.16300086405005) bank39186 +39187 POINT(29.859882400453568 -96.12065038590931) bank39187 +39188 POINT(29.5369691845575 -94.59983144936048) bank39188 +39189 POINT(30.48871799255735 -96.2406195882193) bank39189 +39190 POINT(29.724041325247466 -95.35696708444436) bank39190 +39191 POINT(29.31429944462522 -94.6215313129651) bank39191 +39192 POINT(30.074731617564467 -95.84116630193499) bank39192 +39193 POINT(29.87015194444193 -95.38145610642864) bank39193 +39194 POINT(29.290808994202635 -96.04440975408542) bank39194 +39195 POINT(28.92682790029106 -95.99831906573215) bank39195 +39196 POINT(29.834062575534354 -95.52181715464425) bank39196 +39197 POINT(29.170693403971207 -95.45068624445382) bank39197 +39198 POINT(29.965337240531973 -95.52765925343105) bank39198 +39199 POINT(29.439730719132527 -94.4762890479826) bank39199 +39200 POINT(29.34277618950549 -95.15549295431272) bank39200 +39201 POINT(30.499724425291955 -94.47684962790913) bank39201 +39202 POINT(29.327728844322856 -96.22997282329959) bank39202 +39203 POINT(29.44531423778264 -95.82408063821774) bank39203 +39204 POINT(30.263840392048348 -94.902443517866) bank39204 +39205 POINT(29.24611745228725 -94.38182666615205) bank39205 +39206 POINT(29.105151704161614 -94.881121341138) bank39206 +39207 POINT(29.14663164485878 -94.8472570614115) bank39207 +39208 POINT(28.965703875667096 -96.13725552364613) bank39208 +39209 POINT(30.73255778846153 -95.29963746439209) bank39209 +39210 POINT(29.75488510106395 -94.5836198024119) bank39210 +39211 POINT(29.829768556130443 -95.90752480674887) bank39211 +39212 POINT(30.09815773243683 -94.4778303642683) bank39212 +39213 POINT(30.086491732889833 -95.19414733841981) bank39213 +39214 POINT(29.851620048437365 -96.328495526373) bank39214 +39215 POINT(30.692066430355375 -94.63103902507015) bank39215 +39216 POINT(29.179807960833372 -94.58927398777674) bank39216 +39217 POINT(30.22769817313567 -94.98980933511518) bank39217 +39218 POINT(30.57786476924265 -94.79716852449073) bank39218 +39219 POINT(29.27653416682628 -95.62333529752551) bank39219 +39220 POINT(29.663627460258603 -95.88288959712605) bank39220 +39221 POINT(29.552431777241484 -94.85419694445173) bank39221 +39222 POINT(30.364823766008627 -95.00766815995453) bank39222 +39223 POINT(30.162269367795528 -94.40965198028432) bank39223 +39224 POINT(29.562226887415388 -95.01885568437231) bank39224 +39225 POINT(30.572559002842176 -95.58443559520194) bank39225 +39226 POINT(30.45420645848481 -95.09895803545129) bank39226 +39227 POINT(29.323043228401385 -95.46215288240363) bank39227 +39228 POINT(28.86014684164382 -96.21224814374365) bank39228 +39229 POINT(29.68757788701764 -96.026780918537) bank39229 +39230 POINT(29.439898279195813 -94.4184858633951) bank39230 +39231 POINT(29.689697399554944 -95.98247133218794) bank39231 +39232 POINT(30.601636836944117 -95.84414155426563) bank39232 +39233 POINT(30.7391379196104 -95.80538807492691) bank39233 +39234 POINT(29.77231943292763 -94.57610479472919) bank39234 +39235 POINT(28.87619043030048 -95.67565195178675) bank39235 +39236 POINT(30.417017341085202 -95.47933853852368) bank39236 +39237 POINT(28.88488719919637 -94.91509824607455) bank39237 +39238 POINT(30.660831098714493 -94.97701200926888) bank39238 +39239 POINT(29.812257791111765 -94.82757872224806) bank39239 +39240 POINT(29.532638476774984 -95.16465978985168) bank39240 +39241 POINT(29.299217917576712 -94.5931878920193) bank39241 +39242 POINT(28.92053634033205 -95.45493815132848) bank39242 +39243 POINT(29.53879251684509 -94.49153429430608) bank39243 +39244 POINT(30.258504699064073 -94.90356258959301) bank39244 +39245 POINT(29.955715428595376 -94.51213000460568) bank39245 +39246 POINT(29.058106304037945 -95.24317019230772) bank39246 +39247 POINT(30.201889329124693 -94.51781744920834) bank39247 +39248 POINT(30.191827557634493 -96.21043833981929) bank39248 +39249 POINT(29.821602626593272 -94.70173636238768) bank39249 +39250 POINT(30.394444444704725 -95.60357217109944) bank39250 +39251 POINT(30.733342396050375 -94.6135353878245) bank39251 +39252 POINT(30.551563967878568 -95.19508018726437) bank39252 +39253 POINT(30.735683182228083 -96.15812238931409) bank39253 +39254 POINT(30.188642827563864 -95.02845803935803) bank39254 +39255 POINT(30.242859047610164 -95.45526445163941) bank39255 +39256 POINT(29.97729680490498 -95.11945239377938) bank39256 +39257 POINT(30.565499667375807 -95.34288844796191) bank39257 +39258 POINT(29.995959207409193 -94.63712199525885) bank39258 +39259 POINT(30.231079691352843 -96.33332804610922) bank39259 +39260 POINT(29.20557230621867 -95.33303937201369) bank39260 +39261 POINT(29.301529487534896 -96.25324850862373) bank39261 +39262 POINT(30.720986009199393 -95.5270120356497) bank39262 +39263 POINT(30.627816384668773 -95.72215925866332) bank39263 +39264 POINT(30.03237688011974 -94.79016044136753) bank39264 +39265 POINT(29.99388319627543 -95.1474600051651) bank39265 +39266 POINT(28.850270304834677 -96.23628006145798) bank39266 +39267 POINT(28.9171342768335 -96.13905583933122) bank39267 +39268 POINT(29.1150472632242 -95.99066328518626) bank39268 +39269 POINT(29.888159115449763 -94.73657176877535) bank39269 +39270 POINT(30.30668146140549 -94.6447262340453) bank39270 +39271 POINT(30.7446984100659 -95.02431457644273) bank39271 +39272 POINT(29.72948543469162 -95.67491307484508) bank39272 +39273 POINT(30.003909870759138 -94.82262357668857) bank39273 +39274 POINT(29.530232492212495 -94.97359415341835) bank39274 +39275 POINT(30.39303985588905 -95.64040893608437) bank39275 +39276 POINT(30.434761847835055 -94.91833353072803) bank39276 +39277 POINT(30.142825902585074 -94.42455456309484) bank39277 +39278 POINT(30.374443155785816 -95.71956529596596) bank39278 +39279 POINT(28.970682357529316 -94.41062766909931) bank39279 +39280 POINT(30.74448886136462 -94.8455629585504) bank39280 +39281 POINT(29.678847569143006 -95.88514946195606) bank39281 +39282 POINT(28.98765655715919 -96.32743074474304) bank39282 +39283 POINT(29.786717577246414 -95.4964799756487) bank39283 +39284 POINT(30.256467609559 -96.1806787036011) bank39284 +39285 POINT(29.61751625241032 -94.7919587837725) bank39285 +39286 POINT(30.130368088333434 -96.29443050081228) bank39286 +39287 POINT(29.131515724708365 -95.24358219795495) bank39287 +39288 POINT(30.58407676429558 -94.63240810462246) bank39288 +39289 POINT(30.591523288418237 -96.3365828641969) bank39289 +39290 POINT(30.371965617592117 -94.43573297091807) bank39290 +39291 POINT(30.38771123837645 -96.15182973104704) bank39291 +39292 POINT(30.70633559471653 -95.56400570680826) bank39292 +39293 POINT(29.255006779483413 -95.20031494851543) bank39293 +39294 POINT(29.230208264022234 -94.513253810444) bank39294 +39295 POINT(28.76965519001889 -96.34814691481925) bank39295 +39296 POINT(30.089784077954576 -95.57671872171274) bank39296 +39297 POINT(29.61095339391674 -94.72398298444288) bank39297 +39298 POINT(28.893404802867 -95.13524167190765) bank39298 +39299 POINT(28.960569362252798 -94.73913990559414) bank39299 +39300 POINT(29.954828879990586 -94.95143975149811) bank39300 +39301 POINT(29.45229117367773 -94.45130892899715) bank39301 +39302 POINT(30.03584433889996 -95.921160311486) bank39302 +39303 POINT(30.367692257258984 -94.42514600741603) bank39303 +39304 POINT(30.54640982197765 -95.59342324020761) bank39304 +39305 POINT(29.80101393622296 -95.14322441320506) bank39305 +39306 POINT(30.7369681813095 -95.88188335286377) bank39306 +39307 POINT(30.418361513828536 -95.95034286608684) bank39307 +39308 POINT(29.94310550012014 -94.37545850789071) bank39308 +39309 POINT(30.00351340948889 -95.34331698094597) bank39309 +39310 POINT(30.320332369024545 -95.69741663367536) bank39310 +39311 POINT(29.076667552250882 -95.77283531562891) bank39311 +39312 POINT(30.56759577919732 -94.98076276387933) bank39312 +39313 POINT(29.091575884203394 -94.81958707742592) bank39313 +39314 POINT(29.323320446845642 -95.35347037402678) bank39314 +39315 POINT(28.95273069672827 -95.01555725393172) bank39315 +39316 POINT(28.801114359961115 -95.47245370991675) bank39316 +39317 POINT(30.422459991607063 -95.89477244865495) bank39317 +39318 POINT(29.476446892729996 -95.2800085333924) bank39318 +39319 POINT(30.349956830731323 -95.7296031993302) bank39319 +39320 POINT(29.401190132513157 -95.36704989256319) bank39320 +39321 POINT(30.718227431915814 -94.90258085840127) bank39321 +39322 POINT(30.638856861747943 -96.03984182041529) bank39322 +39323 POINT(29.53840199759653 -95.44072214317441) bank39323 +39324 POINT(30.553453364871615 -95.64693396055718) bank39324 +39325 POINT(30.061084392156186 -96.27851969064261) bank39325 +39326 POINT(29.44711175460031 -96.28523662309641) bank39326 +39327 POINT(28.92427831625195 -95.75437077511413) bank39327 +39328 POINT(29.93176858212961 -95.76617605531793) bank39328 +39329 POINT(30.38950484600848 -95.6404525243443) bank39329 +39330 POINT(30.31743942692046 -96.01255021036344) bank39330 +39331 POINT(29.836299950551076 -95.3813849687031) bank39331 +39332 POINT(30.660181053781812 -95.91473885628929) bank39332 +39333 POINT(29.782382266658185 -94.56436500691325) bank39333 +39334 POINT(30.63035711171417 -94.50573853278378) bank39334 +39335 POINT(28.946505341415975 -96.08196939644303) bank39335 +39336 POINT(28.988452119882034 -94.58648525509696) bank39336 +39337 POINT(28.889749334133956 -96.131747529046) bank39337 +39338 POINT(29.285183504051158 -95.35367034389458) bank39338 +39339 POINT(29.597427363691267 -95.51914143597412) bank39339 +39340 POINT(30.395161265379663 -95.1320088724522) bank39340 +39341 POINT(30.354091721285943 -94.9282730987032) bank39341 +39342 POINT(29.228890511019188 -94.37825307571028) bank39342 +39343 POINT(29.100234126318195 -96.34030716131956) bank39343 +39344 POINT(29.489543163277247 -95.50958988103218) bank39344 +39345 POINT(30.125075245619072 -96.31981427517145) bank39345 +39346 POINT(30.24024967450499 -95.48087726265071) bank39346 +39347 POINT(29.66238225900084 -95.52992354003639) bank39347 +39348 POINT(30.1521972492274 -95.4545633605738) bank39348 +39349 POINT(30.20394311372619 -94.60831967123669) bank39349 +39350 POINT(30.572035627625485 -95.41185225734125) bank39350 +39351 POINT(28.84929581727516 -95.498051646269) bank39351 +39352 POINT(30.243979638246735 -95.15275227714577) bank39352 +39353 POINT(29.323545155707524 -95.7421884728539) bank39353 +39354 POINT(29.659565776797965 -96.00687792265596) bank39354 +39355 POINT(29.66809935083043 -95.80382109246455) bank39355 +39356 POINT(30.533736522214824 -94.74377550632147) bank39356 +39357 POINT(28.825116251488325 -95.13731877204069) bank39357 +39358 POINT(30.129810431337074 -94.49750816684165) bank39358 +39359 POINT(29.133361486442116 -95.84137384469015) bank39359 +39360 POINT(29.304548739614603 -94.63153645641438) bank39360 +39361 POINT(29.962360194263958 -94.79795698337428) bank39361 +39362 POINT(30.343792218285778 -95.53442513056437) bank39362 +39363 POINT(30.645234544683348 -96.0073010096456) bank39363 +39364 POINT(29.507945030565473 -94.38820336713584) bank39364 +39365 POINT(28.824358710170937 -94.81135680893347) bank39365 +39366 POINT(29.178014773488332 -94.92622000450966) bank39366 +39367 POINT(30.71593635673767 -94.39279934293859) bank39367 +39368 POINT(30.617499810352168 -95.31763032077357) bank39368 +39369 POINT(30.576454961549565 -96.00516256365276) bank39369 +39370 POINT(30.33283948051817 -94.38335431526329) bank39370 +39371 POINT(29.37688796280762 -95.02110983622512) bank39371 +39372 POINT(29.25723876227914 -95.22776785529277) bank39372 +39373 POINT(28.841176044096215 -96.22690807788395) bank39373 +39374 POINT(29.985010925651707 -95.63613132244923) bank39374 +39375 POINT(30.368945249950198 -95.26706286886699) bank39375 +39376 POINT(30.145725268234923 -94.41673238149176) bank39376 +39377 POINT(28.839457855692366 -96.3422207897039) bank39377 +39378 POINT(29.269958598428463 -95.85257381978656) bank39378 +39379 POINT(30.076254530760906 -95.66470005443801) bank39379 +39380 POINT(29.928167702129457 -96.03604522718273) bank39380 +39381 POINT(28.861558948522152 -95.04959403722587) bank39381 +39382 POINT(30.03731077024431 -96.27709238226413) bank39382 +39383 POINT(28.769481721221062 -94.44961472879206) bank39383 +39384 POINT(29.52138503636688 -94.88933394284273) bank39384 +39385 POINT(29.899988546105774 -94.65178457733104) bank39385 +39386 POINT(28.85318930093782 -95.09288773700487) bank39386 +39387 POINT(30.518863815903345 -96.25525886621065) bank39387 +39388 POINT(30.613304683897294 -94.70858120402093) bank39388 +39389 POINT(29.31365677816471 -95.6995693009243) bank39389 +39390 POINT(30.63776509507694 -95.2836797235502) bank39390 +39391 POINT(30.21848628572291 -95.47567384955957) bank39391 +39392 POINT(29.985132014600502 -95.91260469268586) bank39392 +39393 POINT(29.362379128127454 -95.95164866908475) bank39393 +39394 POINT(30.29569847252004 -95.35307345071315) bank39394 +39395 POINT(30.54452868606249 -95.29463637975654) bank39395 +39396 POINT(29.375976833996454 -94.68704742565788) bank39396 +39397 POINT(30.290004557876347 -94.64954124590467) bank39397 +39398 POINT(29.83849149180365 -94.99075038278606) bank39398 +39399 POINT(29.0305733849668 -96.0019806187453) bank39399 +39400 POINT(29.531332289172916 -95.78263079937246) bank39400 +39401 POINT(30.481619654395686 -94.47574507446237) bank39401 +39402 POINT(29.99368087644786 -95.31701564303204) bank39402 +39403 POINT(30.53937953639645 -96.14771250444831) bank39403 +39404 POINT(29.70190351332006 -94.48600269090933) bank39404 +39405 POINT(29.867144107052894 -96.24794480378618) bank39405 +39406 POINT(30.464933428644656 -94.83076780364141) bank39406 +39407 POINT(29.329176709326862 -95.24413586904718) bank39407 +39408 POINT(29.154931123184443 -95.92968469286184) bank39408 +39409 POINT(29.139062082061013 -94.66122014415) bank39409 +39410 POINT(30.52884383126096 -95.34741906193963) bank39410 +39411 POINT(29.95244252986116 -96.23398490684237) bank39411 +39412 POINT(29.5978955435331 -96.31543975080262) bank39412 +39413 POINT(29.56158478772686 -95.9327292417448) bank39413 +39414 POINT(28.823057865784442 -95.78115514899712) bank39414 +39415 POINT(30.26420016601701 -96.08923822912362) bank39415 +39416 POINT(29.153255746227423 -94.9634435953096) bank39416 +39417 POINT(30.246065554677493 -95.79519989744446) bank39417 +39418 POINT(29.677986230763715 -94.85015411445289) bank39418 +39419 POINT(30.103719437239334 -96.0947871186204) bank39419 +39420 POINT(29.652614425241957 -95.8899981338184) bank39420 +39421 POINT(30.237076749032067 -95.4019879705578) bank39421 +39422 POINT(29.52923253863492 -96.30066181235458) bank39422 +39423 POINT(29.5429545232051 -96.05082482414629) bank39423 +39424 POINT(29.975042048400944 -95.8512824087075) bank39424 +39425 POINT(29.70618770585071 -94.66072300098914) bank39425 +39426 POINT(28.7657305327722 -95.38639022533161) bank39426 +39427 POINT(29.654242415043193 -95.7655803626987) bank39427 +39428 POINT(30.505887720841724 -95.16288094401024) bank39428 +39429 POINT(30.529811593257552 -95.84468353212118) bank39429 +39430 POINT(30.17224165985857 -94.74246413481451) bank39430 +39431 POINT(29.403843926603525 -96.29261710030795) bank39431 +39432 POINT(29.829043645509667 -96.32671939083258) bank39432 +39433 POINT(29.241257914467766 -95.65762171173684) bank39433 +39434 POINT(29.591338392495466 -96.27966771859491) bank39434 +39435 POINT(29.89995439743875 -95.7276620562463) bank39435 +39436 POINT(29.551602791219082 -96.11919031685447) bank39436 +39437 POINT(29.28855383931915 -95.76076160414559) bank39437 +39438 POINT(29.233354524976708 -95.73459767484378) bank39438 +39439 POINT(29.177989394573856 -96.08508492285317) bank39439 +39440 POINT(28.8614466296761 -96.16795350957199) bank39440 +39441 POINT(29.37714353973422 -95.94958375169942) bank39441 +39442 POINT(30.682494073413647 -96.07197184171282) bank39442 +39443 POINT(28.902467708828905 -94.636272118168) bank39443 +39444 POINT(30.35636005346503 -94.97437556629775) bank39444 +39445 POINT(29.72288988153453 -94.81011097966201) bank39445 +39446 POINT(29.242913801984773 -95.72777974902075) bank39446 +39447 POINT(29.219248155181507 -96.15443896015745) bank39447 +39448 POINT(28.911199901880433 -95.12840407269901) bank39448 +39449 POINT(29.41249816281427 -95.70350055869527) bank39449 +39450 POINT(29.1043940842509 -95.65720615170545) bank39450 +39451 POINT(29.355298795453784 -95.82359176338595) bank39451 +39452 POINT(30.48532695367621 -95.64356294458833) bank39452 +39453 POINT(29.489360474062003 -95.0138519498843) bank39453 +39454 POINT(28.836175404735016 -95.40068906475874) bank39454 +39455 POINT(29.88102188577644 -95.41445771460583) bank39455 +39456 POINT(28.90075033079305 -96.22044306603672) bank39456 +39457 POINT(29.452447042990908 -95.17787818770448) bank39457 +39458 POINT(28.843694078710225 -95.15825288782958) bank39458 +39459 POINT(30.183940510146403 -96.35383081929068) bank39459 +39460 POINT(30.44986306047513 -95.45203054225394) bank39460 +39461 POINT(29.241056482077255 -95.82563156394077) bank39461 +39462 POINT(30.461699758502455 -95.15456429012012) bank39462 +39463 POINT(30.2553409631102 -94.65384946826879) bank39463 +39464 POINT(30.01824695396283 -94.42056729869819) bank39464 +39465 POINT(30.513734997271364 -95.38656104862936) bank39465 +39466 POINT(29.145104712393458 -95.92025160029414) bank39466 +39467 POINT(29.35036666468882 -95.09755535880838) bank39467 +39468 POINT(30.065166157671218 -95.53735213506052) bank39468 +39469 POINT(29.279713368992386 -94.90710680347601) bank39469 +39470 POINT(30.03578873204101 -95.33199401874012) bank39470 +39471 POINT(30.34547598624453 -94.40367575625311) bank39471 +39472 POINT(28.81646652975219 -94.71075733432806) bank39472 +39473 POINT(28.843059953383044 -95.38774390392362) bank39473 +39474 POINT(30.21035464828661 -94.72650623867152) bank39474 +39475 POINT(28.914281648005794 -95.89512917024926) bank39475 +39476 POINT(29.956510882620346 -94.65351016258298) bank39476 +39477 POINT(28.969061266504365 -95.26881091150108) bank39477 +39478 POINT(29.60638787480419 -95.72260026915518) bank39478 +39479 POINT(29.638844558528035 -94.94170397936615) bank39479 +39480 POINT(29.861028807606814 -96.26318885845427) bank39480 +39481 POINT(29.23684599959323 -96.2106167734618) bank39481 +39482 POINT(29.142913677453045 -96.07984584835576) bank39482 +39483 POINT(29.064724273922693 -94.80837993080813) bank39483 +39484 POINT(30.61465212254579 -95.81172755605542) bank39484 +39485 POINT(30.142573443159797 -96.32476952224302) bank39485 +39486 POINT(29.211958241646887 -94.4257869285202) bank39486 +39487 POINT(30.67552174160306 -94.73652780024756) bank39487 +39488 POINT(30.66912224162447 -96.12356828019793) bank39488 +39489 POINT(29.022227410516106 -94.64345767426278) bank39489 +39490 POINT(30.207147257467184 -96.16954318331331) bank39490 +39491 POINT(28.96565239923496 -95.16909033521087) bank39491 +39492 POINT(29.980708823767245 -95.53571129891266) bank39492 +39493 POINT(30.161119458853953 -94.67737590591565) bank39493 +39494 POINT(30.145014970455662 -94.59857536984424) bank39494 +39495 POINT(30.18227078404154 -94.92671538925674) bank39495 +39496 POINT(29.38182724119757 -94.6176026607908) bank39496 +39497 POINT(30.599242866196935 -94.49260435627721) bank39497 +39498 POINT(30.14745959423312 -94.96683209913718) bank39498 +39499 POINT(28.984271749733796 -94.89437332998446) bank39499 +39500 POINT(30.20627952066451 -95.94938819203516) bank39500 +39501 POINT(29.86692945757172 -95.70166473672562) bank39501 +39502 POINT(29.0892139348776 -95.17826688230042) bank39502 +39503 POINT(29.642674436929497 -94.45609065615562) bank39503 +39504 POINT(29.681209992068855 -95.8802022469523) bank39504 +39505 POINT(30.129533773310914 -96.05603633808734) bank39505 +39506 POINT(29.34917389487391 -94.8792167401857) bank39506 +39507 POINT(29.006776373925163 -96.07459107244051) bank39507 +39508 POINT(29.342632373805433 -94.61174982929187) bank39508 +39509 POINT(29.089561222698812 -94.63926216733694) bank39509 +39510 POINT(29.631909022458153 -95.60765569841905) bank39510 +39511 POINT(29.449858764332912 -94.88095420293944) bank39511 +39512 POINT(29.37561529576482 -95.28146843721427) bank39512 +39513 POINT(28.79414989038158 -94.44262928855319) bank39513 +39514 POINT(29.697571504828264 -95.74878056000036) bank39514 +39515 POINT(29.643138366640738 -95.9710901697058) bank39515 +39516 POINT(30.504615124171238 -96.28911780709235) bank39516 +39517 POINT(29.419980409935278 -95.62407800542807) bank39517 +39518 POINT(29.871219023516634 -95.01360367312836) bank39518 +39519 POINT(30.242652683634862 -96.3292026925582) bank39519 +39520 POINT(30.103619157809238 -94.47010370053798) bank39520 +39521 POINT(29.5731422390985 -94.83615939478139) bank39521 +39522 POINT(30.51786765191624 -94.47177123110512) bank39522 +39523 POINT(30.23427895009749 -95.14544432021928) bank39523 +39524 POINT(29.853880756916496 -95.81053526882967) bank39524 +39525 POINT(30.534284665619754 -95.08325006379509) bank39525 +39526 POINT(29.668138849161668 -95.82726269242464) bank39526 +39527 POINT(30.559759861597566 -94.9770230267036) bank39527 +39528 POINT(30.671511010872308 -96.3057772206975) bank39528 +39529 POINT(29.492937463703125 -95.20889353823408) bank39529 +39530 POINT(29.617674791026126 -94.87128962129817) bank39530 +39531 POINT(29.920359816907414 -94.69385872419377) bank39531 +39532 POINT(29.416443525990097 -94.52759941381521) bank39532 +39533 POINT(29.37476739813526 -96.25158369880839) bank39533 +39534 POINT(29.775446889435713 -95.3711245493328) bank39534 +39535 POINT(29.727145524283763 -94.81426131991066) bank39535 +39536 POINT(29.22731766513859 -95.68836326299936) bank39536 +39537 POINT(29.306176850687567 -95.58712662270955) bank39537 +39538 POINT(29.151151264387863 -95.5317749235445) bank39538 +39539 POINT(30.25057015233185 -94.71213275860842) bank39539 +39540 POINT(30.16843895685606 -95.48637448461724) bank39540 +39541 POINT(30.47447181314991 -94.73447521102004) bank39541 +39542 POINT(29.452966693364683 -94.61280050625251) bank39542 +39543 POINT(30.407207056824095 -95.18135066039275) bank39543 +39544 POINT(30.238480582871496 -95.59515670996446) bank39544 +39545 POINT(30.65395425679306 -95.89961453547333) bank39545 +39546 POINT(29.581005658678404 -95.72934862021489) bank39546 +39547 POINT(30.008726330381567 -95.11198442949559) bank39547 +39548 POINT(29.212191663353444 -95.26292087770774) bank39548 +39549 POINT(29.172623768004723 -94.87541213358024) bank39549 +39550 POINT(29.70369149580212 -95.8939135572413) bank39550 +39551 POINT(30.076698231162364 -95.63490740283541) bank39551 +39552 POINT(29.275103439874233 -96.08284678104395) bank39552 +39553 POINT(30.169736295898137 -96.1351024527866) bank39553 +39554 POINT(29.73716448507141 -95.76720130513057) bank39554 +39555 POINT(29.82718371852641 -95.42926654862507) bank39555 +39556 POINT(29.916130287060174 -95.4265071051968) bank39556 +39557 POINT(29.275203389805114 -95.50879065888904) bank39557 +39558 POINT(29.912288212318785 -94.92421912175436) bank39558 +39559 POINT(30.542514514152042 -96.31867464844113) bank39559 +39560 POINT(29.464222739372083 -95.78997963617937) bank39560 +39561 POINT(30.610720984373117 -96.34235816775417) bank39561 +39562 POINT(30.395445061843837 -96.05361843836931) bank39562 +39563 POINT(29.000903239881723 -94.67301554486806) bank39563 +39564 POINT(28.82773085207125 -94.6795819368013) bank39564 +39565 POINT(29.692291248556344 -94.47291597599408) bank39565 +39566 POINT(30.099340733324706 -94.43066904092498) bank39566 +39567 POINT(30.175501909928474 -96.16013612022937) bank39567 +39568 POINT(29.576504592037292 -95.84370466414204) bank39568 +39569 POINT(29.823204940345008 -94.59833939242134) bank39569 +39570 POINT(29.745938825928594 -95.71267599162837) bank39570 +39571 POINT(30.388762966561124 -95.31563869803746) bank39571 +39572 POINT(30.369747627320873 -96.33413231372506) bank39572 +39573 POINT(29.693292559524888 -96.12220369410213) bank39573 +39574 POINT(28.778984970839225 -94.75772552771119) bank39574 +39575 POINT(29.172805614657108 -96.19163040733275) bank39575 +39576 POINT(30.039319928948537 -95.95948404779219) bank39576 +39577 POINT(30.56293457429653 -95.05995984968348) bank39577 +39578 POINT(28.964555485400393 -95.11380378588852) bank39578 +39579 POINT(29.01337597491062 -96.25561634059903) bank39579 +39580 POINT(30.748217251117154 -95.47937017457687) bank39580 +39581 POINT(30.60448636380666 -96.08821983683217) bank39581 +39582 POINT(29.857538113273662 -94.70151397262414) bank39582 +39583 POINT(29.92136463158661 -94.63362403198977) bank39583 +39584 POINT(30.195933412135208 -94.88815572279776) bank39584 +39585 POINT(29.621888043389337 -96.3467407120421) bank39585 +39586 POINT(30.42688594951809 -95.66782420081957) bank39586 +39587 POINT(28.988487269535497 -96.2298327451889) bank39587 +39588 POINT(30.047557550031122 -94.69274871808331) bank39588 +39589 POINT(28.785123978423584 -95.87785211457836) bank39589 +39590 POINT(30.17725066416271 -95.8924361806735) bank39590 +39591 POINT(29.62720031672672 -96.28329306304963) bank39591 +39592 POINT(30.706791213242536 -95.99707557541942) bank39592 +39593 POINT(30.132883322206872 -95.69511260792291) bank39593 +39594 POINT(30.27169265776372 -94.60098221598427) bank39594 +39595 POINT(29.417287163846122 -96.11684912892022) bank39595 +39596 POINT(29.302109898404222 -95.02900977411998) bank39596 +39597 POINT(29.400768047278305 -94.38202251935182) bank39597 +39598 POINT(30.431679353020908 -95.929887399729) bank39598 +39599 POINT(29.36846056945704 -95.0746413272738) bank39599 +39600 POINT(29.583241724415704 -94.82916721899744) bank39600 +39601 POINT(30.13018206315155 -95.34179611426994) bank39601 +39602 POINT(28.80276535279327 -95.19365064712007) bank39602 +39603 POINT(28.798873982983434 -95.29498586414512) bank39603 +39604 POINT(30.21344565124139 -95.88061933472233) bank39604 +39605 POINT(28.828752707789214 -95.3901532229146) bank39605 +39606 POINT(29.64234882007542 -94.47319005942535) bank39606 +39607 POINT(29.607470616403333 -95.24717972130573) bank39607 +39608 POINT(30.446107579149345 -96.35482071783481) bank39608 +39609 POINT(29.07632922945141 -95.57600812435062) bank39609 +39610 POINT(29.780292183890673 -95.86087016138005) bank39610 +39611 POINT(29.117450981805476 -96.05535243673324) bank39611 +39612 POINT(30.149334545171072 -94.76598561474862) bank39612 +39613 POINT(29.977205238693116 -94.5131263851054) bank39613 +39614 POINT(30.329377239518497 -95.872398041235) bank39614 +39615 POINT(29.90864147708556 -95.16636971755591) bank39615 +39616 POINT(29.556492955002618 -94.69304005323272) bank39616 +39617 POINT(29.752095365289637 -95.13744923005163) bank39617 +39618 POINT(30.654033148878526 -94.887720347557) bank39618 +39619 POINT(29.98197682545904 -96.09588162623533) bank39619 +39620 POINT(30.40520041140617 -95.30194516994645) bank39620 +39621 POINT(30.152450656944072 -95.76605624232428) bank39621 +39622 POINT(29.76277108545781 -95.6814947158241) bank39622 +39623 POINT(28.806982636335288 -95.02033843442878) bank39623 +39624 POINT(29.829786179553373 -95.70748113796517) bank39624 +39625 POINT(30.692528169369066 -94.83827423047526) bank39625 +39626 POINT(30.35951879886054 -94.85320538335709) bank39626 +39627 POINT(29.42926964336411 -95.52004722203928) bank39627 +39628 POINT(29.26873446532931 -95.58195577625825) bank39628 +39629 POINT(30.07283619894347 -95.05771223604665) bank39629 +39630 POINT(29.21615777467157 -94.61874730881304) bank39630 +39631 POINT(30.26241683238955 -96.08898018313111) bank39631 +39632 POINT(29.542855114679977 -95.75450207257214) bank39632 +39633 POINT(29.34137814797778 -94.90024860222539) bank39633 +39634 POINT(29.674129551395033 -96.30780216869891) bank39634 +39635 POINT(30.2187237439834 -95.29007757372213) bank39635 +39636 POINT(30.3243492293428 -95.06579392943394) bank39636 +39637 POINT(30.392645719789684 -94.42576405979591) bank39637 +39638 POINT(30.585943085214655 -95.99167395419916) bank39638 +39639 POINT(30.270368225438418 -94.85524546039552) bank39639 +39640 POINT(30.355563001372566 -96.24767993152726) bank39640 +39641 POINT(29.29405263166172 -95.87133356836432) bank39641 +39642 POINT(29.974480885749436 -96.112162956544) bank39642 +39643 POINT(28.987587692512637 -94.71009234857141) bank39643 +39644 POINT(29.279352786218272 -95.78257682465355) bank39644 +39645 POINT(29.3731204716395 -95.60862680898173) bank39645 +39646 POINT(28.78986289595355 -94.38134619001187) bank39646 +39647 POINT(28.789035182188474 -96.0599622411425) bank39647 +39648 POINT(30.12871882618377 -94.96724225043141) bank39648 +39649 POINT(30.383907997916026 -95.26107928818035) bank39649 +39650 POINT(30.698245598948095 -94.64741470641196) bank39650 +39651 POINT(30.28148118619195 -94.90627842418407) bank39651 +39652 POINT(29.5493795393229 -95.25574902715849) bank39652 +39653 POINT(30.021845654514976 -96.17355922294081) bank39653 +39654 POINT(30.62716012448927 -96.27248714193072) bank39654 +39655 POINT(30.311932261567154 -95.58494289854974) bank39655 +39656 POINT(30.672343711562593 -94.8395758835472) bank39656 +39657 POINT(30.386816288523068 -94.55218532624338) bank39657 +39658 POINT(30.5582335030206 -94.95837309007135) bank39658 +39659 POINT(30.29279069902835 -96.23558876165264) bank39659 +39660 POINT(30.108954620293087 -95.14343658875171) bank39660 +39661 POINT(29.85265634820563 -94.60909998924133) bank39661 +39662 POINT(29.608728806504 -95.3082625996201) bank39662 +39663 POINT(30.554737398903406 -94.99137274067185) bank39663 +39664 POINT(30.097707142160964 -96.07746916158837) bank39664 +39665 POINT(30.269422722593468 -94.79259691501707) bank39665 +39666 POINT(30.152279781678487 -95.6714888853368) bank39666 +39667 POINT(30.059991656627492 -95.03319670624333) bank39667 +39668 POINT(29.682916492534464 -96.1820919097548) bank39668 +39669 POINT(30.297407778543793 -94.71665208582785) bank39669 +39670 POINT(29.141931880100525 -96.31264011993862) bank39670 +39671 POINT(30.525276420522005 -94.47470454521687) bank39671 +39672 POINT(29.903637361465993 -96.0758666348309) bank39672 +39673 POINT(30.2998940647016 -95.30663290863981) bank39673 +39674 POINT(30.648029753798976 -95.59284873181059) bank39674 +39675 POINT(28.971427515864455 -95.3570498413549) bank39675 +39676 POINT(29.933759605636066 -94.6350053382663) bank39676 +39677 POINT(30.423630597821987 -94.43253924645495) bank39677 +39678 POINT(28.782468305533264 -95.95791354515615) bank39678 +39679 POINT(30.54426018505891 -94.5431020003486) bank39679 +39680 POINT(29.482776498982734 -94.49946849999868) bank39680 +39681 POINT(29.64959081548855 -94.55892918959258) bank39681 +39682 POINT(29.54000020431229 -95.56719171305218) bank39682 +39683 POINT(29.915013011909195 -94.67686437253641) bank39683 +39684 POINT(29.756803842643944 -95.05263815010024) bank39684 +39685 POINT(29.05841879254035 -94.94363558602757) bank39685 +39686 POINT(30.535617094864882 -95.61090738505989) bank39686 +39687 POINT(29.425835108711507 -94.77188293221279) bank39687 +39688 POINT(28.98635560988845 -95.646256469861) bank39688 +39689 POINT(29.64629867538114 -94.4064756183541) bank39689 +39690 POINT(30.44414638949294 -94.47318460835618) bank39690 +39691 POINT(29.172006070234044 -95.94897432325104) bank39691 +39692 POINT(28.860285601491633 -96.28912866990983) bank39692 +39693 POINT(28.894073026706803 -94.6819448675911) bank39693 +39694 POINT(30.3271041487273 -95.46010079438858) bank39694 +39695 POINT(30.53154301906936 -94.85078075146309) bank39695 +39696 POINT(28.96487380452534 -96.05302596682176) bank39696 +39697 POINT(29.621303478339588 -96.13221841154217) bank39697 +39698 POINT(30.48977634943556 -95.22196269866289) bank39698 +39699 POINT(29.796148855228065 -94.85853018455852) bank39699 +39700 POINT(29.823434346186097 -96.0389520565346) bank39700 +39701 POINT(29.7970008174382 -94.51199419538561) bank39701 +39702 POINT(29.567072315354572 -95.11820605782752) bank39702 +39703 POINT(29.691920638357587 -94.54998914844714) bank39703 +39704 POINT(29.66931639032538 -94.53689663716958) bank39704 +39705 POINT(30.424101790665237 -95.96469879331859) bank39705 +39706 POINT(30.067097229788157 -94.71435831626516) bank39706 +39707 POINT(30.576177642275074 -94.3824318447313) bank39707 +39708 POINT(30.014438653127613 -94.58765682085355) bank39708 +39709 POINT(30.0182155883494 -95.88342233280973) bank39709 +39710 POINT(29.091531850075466 -95.24046902249422) bank39710 +39711 POINT(28.916091544632465 -94.44636551685086) bank39711 +39712 POINT(30.313019412296793 -94.8479735696896) bank39712 +39713 POINT(29.216902005030576 -95.39947389594076) bank39713 +39714 POINT(28.801995221152325 -96.10722402586097) bank39714 +39715 POINT(30.168569075874043 -95.79223460847356) bank39715 +39716 POINT(30.757585939079902 -95.79187103064405) bank39716 +39717 POINT(29.221943719063024 -95.7889201474934) bank39717 +39718 POINT(30.244616448504537 -95.10878760231225) bank39718 +39719 POINT(29.544537817611012 -94.92730455409412) bank39719 +39720 POINT(28.873049498544457 -95.3391437403536) bank39720 +39721 POINT(29.545877967670364 -95.68469739438287) bank39721 +39722 POINT(29.47308674822594 -96.03398594237143) bank39722 +39723 POINT(29.095548408566646 -95.2089563043364) bank39723 +39724 POINT(28.841404948101733 -95.47646527398553) bank39724 +39725 POINT(30.739108288214254 -96.26969660887276) bank39725 +39726 POINT(29.215624217504743 -95.73725422901887) bank39726 +39727 POINT(30.451574835230932 -96.29218504098355) bank39727 +39728 POINT(29.148099075154672 -95.16811502588715) bank39728 +39729 POINT(30.40876182190458 -94.72561397067956) bank39729 +39730 POINT(29.914879965754192 -94.51252693616699) bank39730 +39731 POINT(28.972655474576598 -95.2525027913566) bank39731 +39732 POINT(30.438106893913243 -96.27272562127321) bank39732 +39733 POINT(29.122006041344523 -95.43095154806309) bank39733 +39734 POINT(30.348300379931075 -96.12643118139817) bank39734 +39735 POINT(30.051722037145247 -94.3816207397159) bank39735 +39736 POINT(30.088232086824437 -94.71239191593348) bank39736 +39737 POINT(29.2095067714199 -95.206841698683) bank39737 +39738 POINT(29.61027031624439 -96.26299341217921) bank39738 +39739 POINT(29.39330279749581 -95.29153815340503) bank39739 +39740 POINT(30.176150004138695 -96.23855817461585) bank39740 +39741 POINT(29.408363702557196 -96.02943373261502) bank39741 +39742 POINT(30.268748672448126 -95.42814682968294) bank39742 +39743 POINT(30.15955912032861 -94.96625973558456) bank39743 +39744 POINT(29.6725764136306 -94.55514925524938) bank39744 +39745 POINT(29.658579528646143 -95.42350474099418) bank39745 +39746 POINT(29.621852689271318 -95.73497966800424) bank39746 +39747 POINT(29.818504433247952 -94.48987486786702) bank39747 +39748 POINT(28.984198049480362 -95.33889644127397) bank39748 +39749 POINT(30.030966438157233 -94.68586941399475) bank39749 +39750 POINT(30.62815737987078 -95.95597448796035) bank39750 +39751 POINT(30.64975900791167 -96.20891794107591) bank39751 +39752 POINT(30.740528429101563 -94.46609147517107) bank39752 +39753 POINT(30.468082231255618 -96.28043829142854) bank39753 +39754 POINT(29.055722949245126 -95.2644881978817) bank39754 +39755 POINT(29.458768950380655 -94.88038834266432) bank39755 +39756 POINT(30.497414339331286 -95.89693703156192) bank39756 +39757 POINT(30.71585929660088 -95.20192307484996) bank39757 +39758 POINT(28.951167346291466 -95.3717710472669) bank39758 +39759 POINT(28.81150796426512 -94.69364899898456) bank39759 +39760 POINT(30.25182204185491 -95.3141018711355) bank39760 +39761 POINT(30.56428326851365 -95.67880438394539) bank39761 +39762 POINT(28.927421164150115 -95.00726201039784) bank39762 +39763 POINT(30.741319521861687 -95.84673053188087) bank39763 +39764 POINT(30.316311688235455 -94.37922510740759) bank39764 +39765 POINT(30.612283589887955 -96.24635290731472) bank39765 +39766 POINT(30.362001113539097 -95.33964473882864) bank39766 +39767 POINT(30.382636385851786 -95.0999828108051) bank39767 +39768 POINT(29.390441728881605 -94.83890917166977) bank39768 +39769 POINT(29.85744992784843 -95.62437051883524) bank39769 +39770 POINT(29.953165942873937 -95.8083686859979) bank39770 +39771 POINT(30.099749697131465 -94.77472303505542) bank39771 +39772 POINT(30.412648682494634 -94.38612860682936) bank39772 +39773 POINT(28.81229734472908 -95.70331119669456) bank39773 +39774 POINT(29.853891014323263 -94.96331154936928) bank39774 +39775 POINT(29.424095525249676 -95.7798546651092) bank39775 +39776 POINT(30.70085962985628 -95.79826906651189) bank39776 +39777 POINT(29.63049453721026 -95.24882371662102) bank39777 +39778 POINT(30.749783455245794 -96.0816451023079) bank39778 +39779 POINT(29.761803586670496 -94.7472743018945) bank39779 +39780 POINT(29.062369644194312 -95.13219902841405) bank39780 +39781 POINT(28.79149080226763 -95.61867185922968) bank39781 +39782 POINT(30.46867847854128 -96.09680606874804) bank39782 +39783 POINT(30.30971280729164 -94.68647502433988) bank39783 +39784 POINT(28.842506605015696 -96.28690152133686) bank39784 +39785 POINT(30.663925772512467 -94.83833362847881) bank39785 +39786 POINT(29.707872734834634 -96.21525704150834) bank39786 +39787 POINT(29.1876981024611 -94.90650408268823) bank39787 +39788 POINT(30.267750499427898 -94.63519570958809) bank39788 +39789 POINT(29.159196063268606 -95.56182827303019) bank39789 +39790 POINT(29.932162516922617 -95.60131002596395) bank39790 +39791 POINT(29.364425489393575 -94.54154172329577) bank39791 +39792 POINT(29.284430216954934 -94.71716358658271) bank39792 +39793 POINT(30.357855355243547 -95.11612450544561) bank39793 +39794 POINT(29.858791002287074 -94.48107716081714) bank39794 +39795 POINT(29.684549294788628 -95.06359366657087) bank39795 +39796 POINT(29.97247502013279 -95.26704709050811) bank39796 +39797 POINT(30.63306246223444 -95.29994063008898) bank39797 +39798 POINT(30.622905424490163 -95.06460498051734) bank39798 +39799 POINT(30.279095236666674 -96.22607847176346) bank39799 +39800 POINT(29.631108165959155 -94.37455352709509) bank39800 +39801 POINT(30.12611162113982 -95.45646704543636) bank39801 +39802 POINT(30.152030314483543 -95.88154585940892) bank39802 +39803 POINT(30.187104408611955 -96.09634884410094) bank39803 +39804 POINT(28.941871860450604 -95.55525656880229) bank39804 +39805 POINT(29.355602261938778 -94.84122937601123) bank39805 +39806 POINT(30.059208956605058 -96.25078771137161) bank39806 +39807 POINT(29.724959566064623 -96.10426437180163) bank39807 +39808 POINT(29.160060058671515 -96.35403912559043) bank39808 +39809 POINT(30.284433647805336 -94.47606022868342) bank39809 +39810 POINT(29.56946796657281 -94.55803051864602) bank39810 +39811 POINT(30.617489118446237 -94.96560828979727) bank39811 +39812 POINT(28.855406044382214 -94.50447181023553) bank39812 +39813 POINT(30.241145130936427 -95.72235060344354) bank39813 +39814 POINT(29.463109901429625 -96.1323981111046) bank39814 +39815 POINT(29.729727032368277 -95.53348047325285) bank39815 +39816 POINT(29.504115167485466 -95.16208106832748) bank39816 +39817 POINT(30.057283343151067 -95.86823839123701) bank39817 +39818 POINT(28.91449572766941 -94.94318274588971) bank39818 +39819 POINT(30.64347658552189 -94.4770547926703) bank39819 +39820 POINT(29.282677895899134 -96.33717111607145) bank39820 +39821 POINT(29.41714994996628 -94.4164344255479) bank39821 +39822 POINT(28.76179711898987 -95.5085588480201) bank39822 +39823 POINT(30.563489906161955 -95.73626196478475) bank39823 +39824 POINT(30.017457000773625 -95.78877351891936) bank39824 +39825 POINT(30.603781910319437 -95.85026044904447) bank39825 +39826 POINT(30.042703277976923 -94.97115196250802) bank39826 +39827 POINT(29.611217857108745 -94.73425374294706) bank39827 +39828 POINT(30.04917629404464 -95.57335177065691) bank39828 +39829 POINT(29.835982845449962 -95.99823440233895) bank39829 +39830 POINT(30.21395567455234 -94.86600140148008) bank39830 +39831 POINT(30.35203315791949 -94.8542681779938) bank39831 +39832 POINT(29.904679711041407 -96.23297330161148) bank39832 +39833 POINT(30.693991764835918 -95.22029061218001) bank39833 +39834 POINT(30.11633246995247 -96.34070087516379) bank39834 +39835 POINT(29.21689831183883 -94.37279129041657) bank39835 +39836 POINT(29.360935096258164 -94.86747523446559) bank39836 +39837 POINT(30.735787351331417 -95.106193694684) bank39837 +39838 POINT(30.26111991278026 -95.61182858282108) bank39838 +39839 POINT(29.14008683725875 -96.22995240212629) bank39839 +39840 POINT(30.735407248422447 -95.53419975675544) bank39840 +39841 POINT(30.172418733848406 -96.04937020105095) bank39841 +39842 POINT(29.941753372390366 -94.5674333808586) bank39842 +39843 POINT(30.14975832515455 -96.01560507694052) bank39843 +39844 POINT(29.929874220590257 -96.16143041693786) bank39844 +39845 POINT(30.71355517122262 -95.19934243022625) bank39845 +39846 POINT(30.1116388631896 -96.35772016263303) bank39846 +39847 POINT(29.202847257017368 -95.73600991451832) bank39847 +39848 POINT(29.11212022458888 -95.41602734150423) bank39848 +39849 POINT(30.73317271557962 -95.81555209292073) bank39849 +39850 POINT(29.333096438950413 -96.13718870181695) bank39850 +39851 POINT(30.48636932524823 -95.00498301691093) bank39851 +39852 POINT(30.447034382987283 -95.66507880481747) bank39852 +39853 POINT(29.069645880798042 -96.13247256679236) bank39853 +39854 POINT(30.371015983628503 -96.28279850523474) bank39854 +39855 POINT(29.85665803267087 -94.65691736577617) bank39855 +39856 POINT(29.641417104057368 -95.89553529359648) bank39856 +39857 POINT(29.888016574072566 -94.45489383644669) bank39857 +39858 POINT(29.255016612726347 -94.9492228446007) bank39858 +39859 POINT(29.61853087851345 -95.49581375910509) bank39859 +39860 POINT(29.494640959761448 -95.78960154001543) bank39860 +39861 POINT(30.38274419495632 -95.30637281910943) bank39861 +39862 POINT(30.569195854525603 -94.86715427704692) bank39862 +39863 POINT(30.10766362199092 -94.81674381779908) bank39863 +39864 POINT(29.92859418992195 -94.85283619556473) bank39864 +39865 POINT(30.235034210744235 -95.88919796626186) bank39865 +39866 POINT(30.726787399044184 -94.72605116550751) bank39866 +39867 POINT(29.456139610335853 -95.85991759745548) bank39867 +39868 POINT(29.419359131843088 -95.1930799703679) bank39868 +39869 POINT(30.405090396842233 -95.2398444121718) bank39869 +39870 POINT(29.57641704918384 -94.98103130803626) bank39870 +39871 POINT(28.85062874192117 -95.40645386715438) bank39871 +39872 POINT(30.17767170156898 -95.34551392536827) bank39872 +39873 POINT(29.8103303114241 -96.17320448195329) bank39873 +39874 POINT(29.733357398567957 -95.8343542985899) bank39874 +39875 POINT(30.6214582565938 -95.3807275459685) bank39875 +39876 POINT(30.50765843760744 -96.2053352311557) bank39876 +39877 POINT(29.34196534774805 -94.40773051559059) bank39877 +39878 POINT(28.856677523982924 -95.37085308810744) bank39878 +39879 POINT(29.49673602491141 -94.60498274596783) bank39879 +39880 POINT(30.11894935824261 -95.39307971880079) bank39880 +39881 POINT(29.324994749937346 -94.4123314914612) bank39881 +39882 POINT(29.657798862715783 -95.42732989411702) bank39882 +39883 POINT(30.08490586297958 -94.86706645732241) bank39883 +39884 POINT(28.887890891076573 -95.08420528131734) bank39884 +39885 POINT(29.46750067748553 -95.39472601236893) bank39885 +39886 POINT(28.88172400365658 -95.10051023223699) bank39886 +39887 POINT(30.737495801333008 -94.510106570561) bank39887 +39888 POINT(30.219438878998798 -95.10565952730187) bank39888 +39889 POINT(29.781922646184576 -94.55843508074322) bank39889 +39890 POINT(29.722259831853574 -94.62165131277355) bank39890 +39891 POINT(29.177544274883264 -96.11056039009958) bank39891 +39892 POINT(30.246968178143504 -96.2760418037587) bank39892 +39893 POINT(29.728828275971342 -94.87878634068011) bank39893 +39894 POINT(29.579936485300063 -95.70390432306563) bank39894 +39895 POINT(29.372518525273517 -95.92673655302775) bank39895 +39896 POINT(30.665830369597273 -94.56328269916155) bank39896 +39897 POINT(29.253720584666485 -95.9327157707621) bank39897 +39898 POINT(29.477873618910493 -96.07347001330808) bank39898 +39899 POINT(29.069191738445976 -94.4368371320766) bank39899 +39900 POINT(30.52607149694051 -96.08864512255562) bank39900 +39901 POINT(30.1426283686626 -94.5776900156681) bank39901 +39902 POINT(29.00646243257169 -95.50514612546019) bank39902 +39903 POINT(30.161129347928977 -94.57079377191221) bank39903 +39904 POINT(30.33806334505276 -94.7984981222593) bank39904 +39905 POINT(29.19178733765352 -95.33386441951608) bank39905 +39906 POINT(30.62382375120763 -94.44565584150834) bank39906 +39907 POINT(30.433711009436138 -95.69851963635449) bank39907 +39908 POINT(29.488108192588548 -94.6618426810185) bank39908 +39909 POINT(29.154946090595143 -95.54226538316419) bank39909 +39910 POINT(29.34596114380922 -95.1286190340902) bank39910 +39911 POINT(29.706090333752563 -95.63668205661631) bank39911 +39912 POINT(29.048472795497204 -95.76679097409647) bank39912 +39913 POINT(30.019231239076433 -94.71708245820157) bank39913 +39914 POINT(29.929237876588484 -94.5918987851245) bank39914 +39915 POINT(30.092158829533012 -95.57219067260833) bank39915 +39916 POINT(29.11500007390712 -94.72047262898263) bank39916 +39917 POINT(29.604535595681835 -96.03882073856943) bank39917 +39918 POINT(30.64634072484799 -94.73870281733032) bank39918 +39919 POINT(29.012439781394683 -95.69616576717411) bank39919 +39920 POINT(28.83531155464034 -95.16150629127719) bank39920 +39921 POINT(29.28876425581555 -95.59910181121353) bank39921 +39922 POINT(30.351148014114845 -95.79979127786126) bank39922 +39923 POINT(30.05726413473948 -96.27093225737309) bank39923 +39924 POINT(30.53095796150591 -94.83520767931607) bank39924 +39925 POINT(29.230874147494823 -96.21162199941799) bank39925 +39926 POINT(29.744434852972585 -94.56926390939492) bank39926 +39927 POINT(29.66724117062976 -95.2052410627762) bank39927 +39928 POINT(29.263174083908808 -95.98337025856232) bank39928 +39929 POINT(30.493100042137623 -94.76422280796116) bank39929 +39930 POINT(30.689657419731642 -95.15833856346289) bank39930 +39931 POINT(29.36670815720491 -94.43842153427826) bank39931 +39932 POINT(29.819755071631857 -95.4503472683077) bank39932 +39933 POINT(29.222978428335427 -96.1574909606767) bank39933 +39934 POINT(30.71528759840591 -95.50924341040452) bank39934 +39935 POINT(30.18730120701707 -95.3238851229837) bank39935 +39936 POINT(30.3322364991659 -94.67836057431566) bank39936 +39937 POINT(28.77302747285972 -96.28110418674014) bank39937 +39938 POINT(30.7319884614421 -94.58945021787835) bank39938 +39939 POINT(30.36309413029495 -94.6627754945573) bank39939 +39940 POINT(30.226660190098738 -95.90719485510262) bank39940 +39941 POINT(30.735666418683998 -96.35196437193093) bank39941 +39942 POINT(30.39443299543142 -94.49328484047084) bank39942 +39943 POINT(30.39324058009188 -95.85273378056542) bank39943 +39944 POINT(29.23920785064225 -95.30878932170727) bank39944 +39945 POINT(29.131313982456884 -95.0370206093242) bank39945 +39946 POINT(29.229733698458432 -95.40737120742908) bank39946 +39947 POINT(29.770410851166854 -95.0053217597007) bank39947 +39948 POINT(29.989458944003687 -95.71176362092557) bank39948 +39949 POINT(30.132560377890513 -96.02378134810122) bank39949 +39950 POINT(30.62878256317059 -95.29410810717435) bank39950 +39951 POINT(30.726689718055237 -96.0122362023976) bank39951 +39952 POINT(30.39838500748866 -95.35954484227862) bank39952 +39953 POINT(30.469575183333692 -96.19133672157535) bank39953 +39954 POINT(29.45793063750526 -95.90745222935887) bank39954 +39955 POINT(29.680171201681922 -96.02490832509156) bank39955 +39956 POINT(30.612122619890332 -95.10546915510122) bank39956 +39957 POINT(29.00367414353993 -95.79546977012214) bank39957 +39958 POINT(29.473091399621676 -94.46751897011805) bank39958 +39959 POINT(30.32099706473932 -94.57509410147073) bank39959 +39960 POINT(29.515164134803502 -94.48446391342073) bank39960 +39961 POINT(30.000475186822673 -95.32181856364471) bank39961 +39962 POINT(30.46116934623883 -95.12759099148457) bank39962 +39963 POINT(29.855801548502242 -95.3693357632289) bank39963 +39964 POINT(30.278635260180558 -95.74055138155629) bank39964 +39965 POINT(28.960180915319388 -95.68736759021388) bank39965 +39966 POINT(29.52819068992057 -95.2401840008503) bank39966 +39967 POINT(29.874582411411094 -94.62180450561502) bank39967 +39968 POINT(28.937814080579173 -95.53621081259294) bank39968 +39969 POINT(29.412282432838605 -94.62231621098205) bank39969 +39970 POINT(29.417225055970004 -96.08992131075378) bank39970 +39971 POINT(29.43024194657722 -94.79566065286899) bank39971 +39972 POINT(30.50483466051354 -94.61846910016101) bank39972 +39973 POINT(28.911306104096994 -94.6408652170003) bank39973 +39974 POINT(29.264837620831027 -94.93280298775855) bank39974 +39975 POINT(30.415428059273708 -95.4086022461557) bank39975 +39976 POINT(29.707505651711067 -95.5183389820503) bank39976 +39977 POINT(29.493769989562917 -95.96378526977014) bank39977 +39978 POINT(30.533953414594063 -96.25074541919649) bank39978 +39979 POINT(29.310294522790457 -94.94424599166871) bank39979 +39980 POINT(29.212497028491953 -94.88932555082937) bank39980 +39981 POINT(28.86733624852732 -95.47194736234097) bank39981 +39982 POINT(29.96080910705287 -95.18757317928527) bank39982 +39983 POINT(29.726504804429503 -96.11445000623176) bank39983 +39984 POINT(29.665268384461577 -94.78729747733102) bank39984 +39985 POINT(29.697893387922203 -95.23719932726732) bank39985 +39986 POINT(29.760743718394405 -95.72525325465449) bank39986 +39987 POINT(30.45117346504527 -95.73117462960026) bank39987 +39988 POINT(29.26672758437908 -95.047013700988) bank39988 +39989 POINT(30.62375785865762 -95.76492562022685) bank39989 +39990 POINT(29.379558396334502 -95.82317497763948) bank39990 +39991 POINT(30.628375616593292 -94.62877203015383) bank39991 +39992 POINT(29.078964962211717 -96.29312337701646) bank39992 +39993 POINT(28.818920214821915 -96.34891208796942) bank39993 +39994 POINT(29.71598969448182 -95.32882033344438) bank39994 +39995 POINT(30.175950766658485 -95.23366081915763) bank39995 +39996 POINT(29.135795453433058 -94.74494833324346) bank39996 +39997 POINT(28.889551877682518 -96.35200817459392) bank39997 +39998 POINT(30.342708660542954 -94.75644206549443) bank39998 +39999 POINT(29.952850534553654 -95.25322367172934) bank39999 +40000 POINT(29.81017482803747 -95.38330093448867) bank40000 +40001 POINT(29.78098358036062 -95.1373245704389) bank40001 +40002 POINT(30.736025341171523 -95.0613832080765) bank40002 +40003 POINT(30.361694695537082 -95.0756610941443) bank40003 +40004 POINT(29.023953674110714 -95.41194484204878) bank40004 +40005 POINT(29.838337288166198 -95.85758185136953) bank40005 +40006 POINT(29.59125936368028 -94.54943650868476) bank40006 +40007 POINT(30.010448713636897 -96.19896086633328) bank40007 +40008 POINT(29.549517992221467 -94.43264876763824) bank40008 +40009 POINT(29.807533986168952 -95.40225489745778) bank40009 +40010 POINT(28.959751687802623 -96.29217126774647) bank40010 +40011 POINT(30.118852030751995 -96.2426684740913) bank40011 +40012 POINT(29.35047262132247 -95.90418129130207) bank40012 +40013 POINT(29.45605681127605 -96.02293256979033) bank40013 +40014 POINT(29.511451598899647 -95.39390929375698) bank40014 +40015 POINT(29.838957088850044 -95.51623198793365) bank40015 +40016 POINT(30.52739676455509 -95.22507770619669) bank40016 +40017 POINT(29.154770368108657 -95.0162727774011) bank40017 +40018 POINT(29.077286436710416 -96.01631548042936) bank40018 +40019 POINT(29.430626509189874 -94.5825461715471) bank40019 +40020 POINT(29.093636437595368 -96.09072748056357) bank40020 +40021 POINT(30.23627027846716 -94.69659539295448) bank40021 +40022 POINT(29.338464031415306 -95.67390707386866) bank40022 +40023 POINT(30.580499148111706 -95.7675306261684) bank40023 +40024 POINT(30.637767472182656 -96.30989224388978) bank40024 +40025 POINT(30.39376471107249 -96.00339277511422) bank40025 +40026 POINT(29.718917884534584 -95.45615952689654) bank40026 +40027 POINT(29.43142632559364 -95.45936405335755) bank40027 +40028 POINT(30.741248103452744 -95.00943627761481) bank40028 +40029 POINT(30.07182255228877 -96.3382529363773) bank40029 +40030 POINT(29.921961381380665 -94.51054780045877) bank40030 +40031 POINT(29.86805957849511 -95.13951066008593) bank40031 +40032 POINT(29.25513733988119 -96.32987591674197) bank40032 +40033 POINT(30.45227223157345 -95.27217893598845) bank40033 +40034 POINT(29.36096600362837 -95.46180510834955) bank40034 +40035 POINT(29.530407570593663 -94.56601573767472) bank40035 +40036 POINT(29.852473926285374 -95.39361077069552) bank40036 +40037 POINT(30.02283743391017 -96.36007366569424) bank40037 +40038 POINT(30.734759002956935 -95.22481682581105) bank40038 +40039 POINT(29.723638504171646 -94.81230531857621) bank40039 +40040 POINT(28.919446418322494 -96.29814527323894) bank40040 +40041 POINT(30.683927629810245 -96.13609577614879) bank40041 +40042 POINT(29.66983524268508 -95.74768637815089) bank40042 +40043 POINT(29.988919575609938 -95.43129260337248) bank40043 +40044 POINT(29.564301015166784 -94.81334558383845) bank40044 +40045 POINT(29.420901414996617 -95.30489069610881) bank40045 +40046 POINT(29.815855611401464 -95.74232766322126) bank40046 +40047 POINT(30.24279737979381 -94.8989338602069) bank40047 +40048 POINT(28.90866512414782 -94.3784986512171) bank40048 +40049 POINT(28.926877022278116 -94.39401403551713) bank40049 +40050 POINT(28.78575497765947 -94.73213288187999) bank40050 +40051 POINT(28.959479696400873 -94.82979611224884) bank40051 +40052 POINT(29.622323588512543 -95.58801077871219) bank40052 +40053 POINT(30.252666713623697 -95.38150356406331) bank40053 +40054 POINT(29.294248197897154 -95.10820365779581) bank40054 +40055 POINT(29.20105637923813 -94.86830633414782) bank40055 +40056 POINT(29.473009282856047 -95.18342842515773) bank40056 +40057 POINT(29.266784263649384 -95.63365386788729) bank40057 +40058 POINT(29.307440891400447 -95.8314319157245) bank40058 +40059 POINT(30.61909070251749 -95.90926761071168) bank40059 +40060 POINT(28.76471746354294 -95.67739266605605) bank40060 +40061 POINT(29.919479444096226 -95.80199411171056) bank40061 +40062 POINT(28.82351391143058 -96.10018761352899) bank40062 +40063 POINT(29.68982620451112 -95.79006614026595) bank40063 +40064 POINT(28.839899714306135 -96.08276386057089) bank40064 +40065 POINT(29.93613670123345 -94.9626062120617) bank40065 +40066 POINT(29.816673744160397 -96.25996851619702) bank40066 +40067 POINT(30.566262793128683 -94.91751329142382) bank40067 +40068 POINT(29.751339340875415 -95.40412435874326) bank40068 +40069 POINT(30.505620639249916 -95.33967986854722) bank40069 +40070 POINT(28.94625020918154 -95.20617230279444) bank40070 +40071 POINT(28.858327637743127 -95.26998777947804) bank40071 +40072 POINT(30.608313422017005 -95.30501050845766) bank40072 +40073 POINT(29.062743877667227 -96.31551019789632) bank40073 +40074 POINT(30.134935483656832 -94.8375996990524) bank40074 +40075 POINT(28.777109938165353 -96.12602897814878) bank40075 +40076 POINT(29.998311769607724 -95.81529557773645) bank40076 +40077 POINT(29.689212940862998 -95.92419145570202) bank40077 +40078 POINT(29.621269461087486 -95.01746690905992) bank40078 +40079 POINT(30.591416557437928 -95.2171479767654) bank40079 +40080 POINT(29.242101706048224 -96.18595569890424) bank40080 +40081 POINT(29.497634834460946 -94.55623695745388) bank40081 +40082 POINT(29.564507181466965 -94.78890700611066) bank40082 +40083 POINT(28.82707084617347 -95.12835145737174) bank40083 +40084 POINT(29.808484154781723 -94.5472690517689) bank40084 +40085 POINT(30.22208638595223 -96.25509246938381) bank40085 +40086 POINT(30.116018563448968 -94.62931468282255) bank40086 +40087 POINT(29.114000142069795 -94.94157777708716) bank40087 +40088 POINT(29.541085852077966 -95.8539958689035) bank40088 +40089 POINT(30.56988575016122 -94.76608019710696) bank40089 +40090 POINT(29.60619141402325 -94.79861452303858) bank40090 +40091 POINT(29.030723436504967 -95.60534935477078) bank40091 +40092 POINT(29.072024433113153 -94.97879593603268) bank40092 +40093 POINT(29.457437035275813 -95.24776404124933) bank40093 +40094 POINT(30.07356315043729 -95.90528046047059) bank40094 +40095 POINT(30.29661335410308 -96.13902850962631) bank40095 +40096 POINT(29.702152803894844 -96.27414359957143) bank40096 +40097 POINT(30.121351860502372 -95.3728821196403) bank40097 +40098 POINT(29.294331055938233 -95.15622898111519) bank40098 +40099 POINT(30.026360716359072 -94.58536221200171) bank40099 +40100 POINT(30.233996301183492 -94.48957706645015) bank40100 +40101 POINT(29.905400976251194 -94.99152789993438) bank40101 +40102 POINT(29.32591446756387 -94.85070422223426) bank40102 +40103 POINT(30.04778915410515 -95.23841729055705) bank40103 +40104 POINT(29.613052100129078 -94.45057843674886) bank40104 +40105 POINT(29.489905189779407 -94.98511544874785) bank40105 +40106 POINT(29.792377473031717 -95.16375775094278) bank40106 +40107 POINT(30.657578342827644 -95.23367985839086) bank40107 +40108 POINT(29.38088191799799 -95.73888865043591) bank40108 +40109 POINT(28.996345535747768 -96.27473406557414) bank40109 +40110 POINT(29.49218554645902 -95.40428641774024) bank40110 +40111 POINT(29.72715419188971 -94.77448391021434) bank40111 +40112 POINT(29.93554083919186 -94.94060620524625) bank40112 +40113 POINT(29.042772183134904 -96.32146540948965) bank40113 +40114 POINT(28.884929913183942 -96.12062171577688) bank40114 +40115 POINT(28.880049819726626 -95.0584670301218) bank40115 +40116 POINT(29.70256879781665 -96.15522556528083) bank40116 +40117 POINT(29.873400342624258 -95.92593911511985) bank40117 +40118 POINT(29.734608301888404 -94.61325560579543) bank40118 +40119 POINT(29.408227723501664 -96.21963014053036) bank40119 +40120 POINT(29.308108778473777 -94.88934953492999) bank40120 +40121 POINT(29.226861487128172 -94.83211783325187) bank40121 +40122 POINT(29.83820634400393 -94.76616480530524) bank40122 +40123 POINT(30.50464954351307 -94.753162080947) bank40123 +40124 POINT(30.54277172619084 -95.85286525480294) bank40124 +40125 POINT(29.330391131476855 -95.33178956680221) bank40125 +40126 POINT(29.9533711135587 -95.35973392676414) bank40126 +40127 POINT(29.314129519314207 -94.92869817118634) bank40127 +40128 POINT(30.731407235798788 -95.31092577345368) bank40128 +40129 POINT(29.96082516453003 -94.7416674747538) bank40129 +40130 POINT(30.229363685075207 -95.28415499009354) bank40130 +40131 POINT(29.439156870286585 -96.31547267057887) bank40131 +40132 POINT(29.645302503346425 -94.75365764688304) bank40132 +40133 POINT(29.982088094835586 -94.66843746915443) bank40133 +40134 POINT(29.21328030361793 -94.73773623096933) bank40134 +40135 POINT(29.097033210587007 -96.14473147301474) bank40135 +40136 POINT(30.453408320254926 -96.2562815480062) bank40136 +40137 POINT(29.98670539929676 -95.33295782911313) bank40137 +40138 POINT(30.07594433474628 -95.09312767201766) bank40138 +40139 POINT(29.000040137693745 -95.89000045303126) bank40139 +40140 POINT(30.27145649892022 -94.6563306833882) bank40140 +40141 POINT(30.658779294360123 -94.80057175027889) bank40141 +40142 POINT(29.983334779583647 -95.76989666663081) bank40142 +40143 POINT(28.805604838443237 -94.65705772331863) bank40143 +40144 POINT(29.67368133617047 -94.89046075408487) bank40144 +40145 POINT(30.157961753083175 -95.806938679719) bank40145 +40146 POINT(30.22557709124748 -94.79193558628074) bank40146 +40147 POINT(29.54139136155274 -94.77332778891007) bank40147 +40148 POINT(30.12292802600733 -94.93632073327637) bank40148 +40149 POINT(30.379968632365717 -94.81451792658162) bank40149 +40150 POINT(30.60157081311454 -95.52810973112685) bank40150 +40151 POINT(29.7272079779422 -95.26580862066311) bank40151 +40152 POINT(29.230372005150404 -95.27235634858779) bank40152 +40153 POINT(28.772362356119327 -94.87367597920365) bank40153 +40154 POINT(30.550603758287018 -95.58722073502673) bank40154 +40155 POINT(30.40931667716178 -95.83434323076175) bank40155 +40156 POINT(28.85808943501506 -96.24242008417694) bank40156 +40157 POINT(28.806053745322597 -94.96772739019859) bank40157 +40158 POINT(29.91539170120807 -95.55834177803624) bank40158 +40159 POINT(29.128970787543068 -96.21447874867434) bank40159 +40160 POINT(30.575447758719598 -94.58606921337879) bank40160 +40161 POINT(30.25579815424432 -95.03676234248572) bank40161 +40162 POINT(30.199331491354013 -95.63485597965611) bank40162 +40163 POINT(29.399335092552608 -95.45870620673907) bank40163 +40164 POINT(29.919207664709138 -94.58272606775348) bank40164 +40165 POINT(30.092849500129784 -95.4830741783963) bank40165 +40166 POINT(29.67318326550592 -95.28931564152147) bank40166 +40167 POINT(29.803153873560568 -95.20178546097593) bank40167 +40168 POINT(29.381173087267968 -95.55043991785236) bank40168 +40169 POINT(29.389516602372634 -96.16723377633177) bank40169 +40170 POINT(30.315508282952315 -95.18388373572738) bank40170 +40171 POINT(29.10324149947896 -95.77124807619651) bank40171 +40172 POINT(29.608788760776267 -95.76288839824103) bank40172 +40173 POINT(29.70375799951087 -95.02961408844067) bank40173 +40174 POINT(29.17120887279481 -96.32369437878202) bank40174 +40175 POINT(29.64572233049773 -94.59030047200783) bank40175 +40176 POINT(28.807748478247195 -95.07675081251574) bank40176 +40177 POINT(29.042235080910835 -94.959635739363) bank40177 +40178 POINT(29.779686908585088 -94.64773941019557) bank40178 +40179 POINT(30.222766312594732 -95.3230454013895) bank40179 +40180 POINT(30.013944369938724 -96.10860019069835) bank40180 +40181 POINT(29.82784407483181 -95.85952219654406) bank40181 +40182 POINT(29.397892727521533 -96.05686092819224) bank40182 +40183 POINT(29.060570207237014 -94.95610888139329) bank40183 +40184 POINT(29.37120412146446 -95.86341472121056) bank40184 +40185 POINT(30.532837709212693 -96.34294740514197) bank40185 +40186 POINT(30.734444904844423 -94.64350151735738) bank40186 +40187 POINT(29.662036866105684 -94.50187631069296) bank40187 +40188 POINT(29.917443381923317 -94.61460981206672) bank40188 +40189 POINT(30.374888106131365 -95.30270686566669) bank40189 +40190 POINT(29.466996928507584 -94.52718858517859) bank40190 +40191 POINT(30.19733357242803 -95.31749629208565) bank40191 +40192 POINT(29.01998925259523 -96.08342221738916) bank40192 +40193 POINT(29.865040595772637 -95.76869349554406) bank40193 +40194 POINT(29.583882027594562 -94.41442864970598) bank40194 +40195 POINT(29.62876336483868 -96.05552812401024) bank40195 +40196 POINT(29.809141169017014 -94.81907833406783) bank40196 +40197 POINT(30.708451658194075 -95.08541543770782) bank40197 +40198 POINT(30.069151763842072 -95.00717120768992) bank40198 +40199 POINT(28.987716466810827 -94.52224013194707) bank40199 +40200 POINT(30.482878094486857 -94.44046391029072) bank40200 +40201 POINT(29.784420260896017 -95.42435485508153) bank40201 +40202 POINT(30.144379210743278 -94.91279823026416) bank40202 +40203 POINT(29.969239488610427 -96.32544007741647) bank40203 +40204 POINT(29.40141288548911 -96.12193196389025) bank40204 +40205 POINT(29.694260741036732 -94.98550545833554) bank40205 +40206 POINT(30.414716887511776 -94.72200984838328) bank40206 +40207 POINT(30.34546077703861 -94.40943019386621) bank40207 +40208 POINT(30.209937492889793 -94.4231167250726) bank40208 +40209 POINT(30.329409880770896 -94.49845462574851) bank40209 +40210 POINT(30.186771026572895 -95.60364485421492) bank40210 +40211 POINT(30.592733489110763 -95.83717040116984) bank40211 +40212 POINT(28.929293267697684 -95.41464417192252) bank40212 +40213 POINT(30.176365104076368 -94.55349331202923) bank40213 +40214 POINT(30.66658475068958 -94.74667713497622) bank40214 +40215 POINT(29.324987510763673 -95.38144277296304) bank40215 +40216 POINT(30.611207177280647 -95.69530404284843) bank40216 +40217 POINT(29.691672153679654 -95.37689251724137) bank40217 +40218 POINT(30.3336986731604 -95.70585557644436) bank40218 +40219 POINT(30.546285593058055 -96.33777093016964) bank40219 +40220 POINT(28.855092838148984 -95.57169615691923) bank40220 +40221 POINT(28.999701340060476 -96.0491355227796) bank40221 +40222 POINT(30.56221143507471 -96.03382597225546) bank40222 +40223 POINT(29.395040488916965 -96.01481817046913) bank40223 +40224 POINT(30.457402149050193 -95.73035606044783) bank40224 +40225 POINT(29.809367088657293 -95.93134960405199) bank40225 +40226 POINT(30.66832141244037 -94.83018998921949) bank40226 +40227 POINT(29.21258963276135 -95.5944878521869) bank40227 +40228 POINT(29.33658728846334 -95.14575842948382) bank40228 +40229 POINT(29.3744241817609 -94.55032006943216) bank40229 +40230 POINT(30.64066416148462 -95.7223824691706) bank40230 +40231 POINT(29.02417965702178 -95.87898068899652) bank40231 +40232 POINT(29.74579949741216 -95.44281844658443) bank40232 +40233 POINT(30.045967128520847 -94.64801288698351) bank40233 +40234 POINT(29.943383166409326 -95.13499417887753) bank40234 +40235 POINT(29.470721091448553 -96.03855941219848) bank40235 +40236 POINT(30.506914811864466 -95.01770256092813) bank40236 +40237 POINT(30.44788428875297 -95.51323026989179) bank40237 +40238 POINT(29.945523636345044 -95.46963080347331) bank40238 +40239 POINT(30.4388994083667 -94.82754905425008) bank40239 +40240 POINT(29.597611614961462 -94.44663062226704) bank40240 +40241 POINT(30.5672208570751 -94.51378819055346) bank40241 +40242 POINT(29.046810086957 -95.26701172887503) bank40242 +40243 POINT(29.2252995255027 -94.81442065802135) bank40243 +40244 POINT(30.106128686472065 -96.29263019473854) bank40244 +40245 POINT(29.41460015351743 -95.44603140514663) bank40245 +40246 POINT(30.258580688092255 -96.10484470421211) bank40246 +40247 POINT(29.99016009724887 -95.64950784933451) bank40247 +40248 POINT(29.664919581840167 -95.07646132202004) bank40248 +40249 POINT(29.487972459599366 -94.46081467169736) bank40249 +40250 POINT(29.862840442232017 -94.46507295627198) bank40250 +40251 POINT(29.351522436498016 -96.10108653836022) bank40251 +40252 POINT(29.27945725787983 -94.59225602035782) bank40252 +40253 POINT(28.82070519207721 -95.6087354997843) bank40253 +40254 POINT(29.92911210649579 -95.52777376007057) bank40254 +40255 POINT(30.160873883582546 -94.49378397612281) bank40255 +40256 POINT(30.749692006415362 -95.88926776156467) bank40256 +40257 POINT(30.461824219463473 -96.04999939944423) bank40257 +40258 POINT(30.546311608114788 -95.25202365162978) bank40258 +40259 POINT(29.26571384938372 -96.25944144438117) bank40259 +40260 POINT(29.394467572025995 -96.01366358131683) bank40260 +40261 POINT(29.660318922676336 -94.90213750560574) bank40261 +40262 POINT(29.723376060917563 -94.96216760450643) bank40262 +40263 POINT(29.528334556626458 -94.75049923115898) bank40263 +40264 POINT(29.020242079849986 -95.70870324755097) bank40264 +40265 POINT(29.293169213957622 -96.31122321910422) bank40265 +40266 POINT(30.64881433386143 -95.12912229438777) bank40266 +40267 POINT(30.428479880291054 -95.38341055959393) bank40267 +40268 POINT(29.22659181282146 -95.97220260888396) bank40268 +40269 POINT(30.243654009400007 -95.84642985014027) bank40269 +40270 POINT(29.061898753067414 -94.702283109198) bank40270 +40271 POINT(28.993389513780752 -94.41993194273513) bank40271 +40272 POINT(29.590094216226266 -95.15660801993127) bank40272 +40273 POINT(29.25148439866494 -95.54481435359287) bank40273 +40274 POINT(30.09166807947187 -94.70226057892124) bank40274 +40275 POINT(30.08952911976412 -95.1263358674688) bank40275 +40276 POINT(30.528355223453186 -95.60411005529294) bank40276 +40277 POINT(30.512321054805867 -94.80673599756315) bank40277 +40278 POINT(30.092488045708492 -95.92129210838407) bank40278 +40279 POINT(29.72886563743133 -95.91638059677483) bank40279 +40280 POINT(30.0357910330952 -96.1714440229357) bank40280 +40281 POINT(29.507410077984623 -94.8700853901236) bank40281 +40282 POINT(29.99955597867816 -96.13278032624602) bank40282 +40283 POINT(29.73429961103315 -94.9305858042821) bank40283 +40284 POINT(29.04217855143221 -95.38807736903662) bank40284 +40285 POINT(30.4401061328399 -96.33718986041382) bank40285 +40286 POINT(29.98768299774087 -95.29175238642888) bank40286 +40287 POINT(29.821178585391344 -96.13766754871828) bank40287 +40288 POINT(29.361221290941526 -94.56826953384743) bank40288 +40289 POINT(29.78857998346431 -95.06062480783527) bank40289 +40290 POINT(30.53489698612775 -95.69699040612811) bank40290 +40291 POINT(29.289301829342765 -94.88081971420725) bank40291 +40292 POINT(30.091352246561996 -94.8640055609686) bank40292 +40293 POINT(29.8046503377268 -96.14099963649035) bank40293 +40294 POINT(29.564490839895633 -95.75951010502851) bank40294 +40295 POINT(29.33054760912352 -94.50582445806485) bank40295 +40296 POINT(30.36732940307849 -95.87554155801783) bank40296 +40297 POINT(29.005934688377 -94.37546922855792) bank40297 +40298 POINT(29.249049855683378 -94.87163471247797) bank40298 +40299 POINT(30.312290689107307 -95.21961483546269) bank40299 +40300 POINT(29.306166549800928 -94.43211662085514) bank40300 +40301 POINT(29.43136224578557 -94.94516060040492) bank40301 +40302 POINT(29.52143410711571 -95.71346594918614) bank40302 +40303 POINT(30.65724673015844 -94.54158901496474) bank40303 +40304 POINT(30.314274434923124 -94.85767554690449) bank40304 +40305 POINT(29.86521872965729 -95.06787036421787) bank40305 +40306 POINT(29.032104881450348 -95.60005936558643) bank40306 +40307 POINT(29.717409115937055 -95.36078893355024) bank40307 +40308 POINT(30.075115880198737 -95.58107968443102) bank40308 +40309 POINT(30.474892550774314 -94.58985070418402) bank40309 +40310 POINT(28.993015335027316 -95.95922461127385) bank40310 +40311 POINT(29.060536985536718 -94.9669776007765) bank40311 +40312 POINT(29.327638432122225 -94.43050687939196) bank40312 +40313 POINT(29.9866541850009 -96.35135958451066) bank40313 +40314 POINT(28.881106625625275 -94.45489614530918) bank40314 +40315 POINT(28.859875313995808 -95.5279718219498) bank40315 +40316 POINT(29.024616603595053 -95.38858906793267) bank40316 +40317 POINT(29.946281639924837 -94.67050857741094) bank40317 +40318 POINT(29.110255493889902 -94.56928711580318) bank40318 +40319 POINT(30.61320082457135 -95.43341686219398) bank40319 +40320 POINT(29.455860471229098 -96.0860247775146) bank40320 +40321 POINT(30.595959793245637 -96.32720448554761) bank40321 +40322 POINT(29.28955040518573 -95.89899767107597) bank40322 +40323 POINT(29.245797179359275 -94.65464752750627) bank40323 +40324 POINT(29.9651695312184 -96.17229514973805) bank40324 +40325 POINT(29.55594396901345 -94.66710569923421) bank40325 +40326 POINT(30.646464763943236 -95.97552143032925) bank40326 +40327 POINT(30.628916898262112 -95.8896039464196) bank40327 +40328 POINT(28.76773952836734 -95.26144187770589) bank40328 +40329 POINT(28.81644519920451 -95.59332759596975) bank40329 +40330 POINT(30.43178508304045 -95.75734739279136) bank40330 +40331 POINT(30.316639499359002 -96.05998096274358) bank40331 +40332 POINT(28.899387212936603 -95.71543964798838) bank40332 +40333 POINT(29.759717756287657 -95.0516836296681) bank40333 +40334 POINT(29.52115918451518 -95.38373961505053) bank40334 +40335 POINT(29.646808108633447 -95.40665477423855) bank40335 +40336 POINT(29.142743676521764 -94.67622681081727) bank40336 +40337 POINT(29.420330355846374 -94.65875160882091) bank40337 +40338 POINT(28.97191082105373 -94.43381519156314) bank40338 +40339 POINT(30.009731124153237 -94.48366042551416) bank40339 +40340 POINT(29.436566362214588 -96.1317037968801) bank40340 +40341 POINT(29.80688449169724 -95.07001849662521) bank40341 +40342 POINT(29.29966789114824 -96.03235398486261) bank40342 +40343 POINT(29.820746591763015 -95.58579657339345) bank40343 +40344 POINT(30.642798375772813 -95.53315974755759) bank40344 +40345 POINT(29.502253488008964 -95.78163310486313) bank40345 +40346 POINT(29.942718444893714 -94.47842561602438) bank40346 +40347 POINT(29.67277575724388 -95.91114958454708) bank40347 +40348 POINT(29.404003588116865 -94.85186918956096) bank40348 +40349 POINT(30.242153838229918 -94.78876945271) bank40349 +40350 POINT(30.59690128418918 -94.69384456770496) bank40350 +40351 POINT(30.228542318110183 -94.67015398404864) bank40351 +40352 POINT(28.852321133486818 -95.32445052879726) bank40352 +40353 POINT(30.05342088251097 -94.69380809597084) bank40353 +40354 POINT(29.533342459577682 -95.71701801424047) bank40354 +40355 POINT(29.714011805412134 -95.64286048457363) bank40355 +40356 POINT(28.838883497441813 -94.87350332559213) bank40356 +40357 POINT(29.180936293676336 -95.17922361954902) bank40357 +40358 POINT(30.707880016008474 -96.19008175572458) bank40358 +40359 POINT(30.147222037485808 -94.97273664038707) bank40359 +40360 POINT(30.676274389035754 -95.88222862247186) bank40360 +40361 POINT(29.243888110310234 -95.37375303350608) bank40361 +40362 POINT(29.94739198384397 -94.51200605319474) bank40362 +40363 POINT(28.921423770017398 -94.87360307099411) bank40363 +40364 POINT(30.452813184378225 -94.93689347873264) bank40364 +40365 POINT(30.71770729328208 -95.35162330626245) bank40365 +40366 POINT(30.255856871425504 -95.34428327438765) bank40366 +40367 POINT(28.972020459242493 -95.57692043371559) bank40367 +40368 POINT(30.692762120037298 -96.05539761503341) bank40368 +40369 POINT(30.65325310382949 -95.4992025465818) bank40369 +40370 POINT(28.824306098966822 -95.82175555987784) bank40370 +40371 POINT(29.313333097679564 -95.7661262208299) bank40371 +40372 POINT(29.49650004012507 -96.12768319671912) bank40372 +40373 POINT(29.941693547715925 -94.54380915986795) bank40373 +40374 POINT(29.05710093611995 -95.68669931686962) bank40374 +40375 POINT(29.362759538247868 -95.76813804877158) bank40375 +40376 POINT(29.90391583551071 -95.58824131301793) bank40376 +40377 POINT(30.672016475601612 -95.73494319186617) bank40377 +40378 POINT(30.6742959129635 -95.62587429131473) bank40378 +40379 POINT(28.962388764306567 -94.4924091941009) bank40379 +40380 POINT(28.797772832351026 -94.53008634049924) bank40380 +40381 POINT(29.964646703744368 -95.42441599334163) bank40381 +40382 POINT(30.013741643485265 -96.13739789750429) bank40382 +40383 POINT(29.011290857871423 -96.30060818644458) bank40383 +40384 POINT(29.957039916139024 -95.35846561438987) bank40384 +40385 POINT(29.828597545939182 -94.92992876998247) bank40385 +40386 POINT(30.50878313762878 -95.54685603483911) bank40386 +40387 POINT(30.458380863864335 -95.39705353175) bank40387 +40388 POINT(28.81981838605075 -94.79012829863149) bank40388 +40389 POINT(28.848185138109162 -95.26298321584096) bank40389 +40390 POINT(29.468672790183145 -95.18313067634708) bank40390 +40391 POINT(29.861824783362003 -95.22718761358055) bank40391 +40392 POINT(29.118925039686296 -95.20277215950814) bank40392 +40393 POINT(30.511701857710598 -95.76423779108492) bank40393 +40394 POINT(30.55708379404479 -96.13054471930576) bank40394 +40395 POINT(30.565164726304427 -96.30787587200767) bank40395 +40396 POINT(30.549163566440463 -95.44203212185941) bank40396 +40397 POINT(30.168109134011107 -96.05680548204093) bank40397 +40398 POINT(30.224602838161008 -94.92126144796593) bank40398 +40399 POINT(30.484087190926292 -95.1437334932477) bank40399 +40400 POINT(28.859551482316782 -94.8084661200933) bank40400 +40401 POINT(30.14341381386041 -94.4388536921435) bank40401 +40402 POINT(29.43396249299428 -94.44860340418994) bank40402 +40403 POINT(28.98556712114058 -94.60847202682281) bank40403 +40404 POINT(28.928653096416724 -95.31030552817302) bank40404 +40405 POINT(30.55652294594304 -95.62443867413391) bank40405 +40406 POINT(29.606025666589495 -94.88769665938972) bank40406 +40407 POINT(29.862111031951375 -95.56808894361407) bank40407 +40408 POINT(28.96607066686182 -94.94798993325955) bank40408 +40409 POINT(29.9270046542182 -95.1911156414637) bank40409 +40410 POINT(29.44894636451076 -95.6211212859573) bank40410 +40411 POINT(28.845980551330758 -94.40424623395154) bank40411 +40412 POINT(29.029252625259762 -95.35588136580094) bank40412 +40413 POINT(29.158317258934197 -96.35462924397235) bank40413 +40414 POINT(29.911725323163132 -96.09677354310875) bank40414 +40415 POINT(29.1832097906624 -96.11045429108574) bank40415 +40416 POINT(29.207805618389553 -94.47501419032592) bank40416 +40417 POINT(30.316324827662292 -95.6639417699084) bank40417 +40418 POINT(30.356611237334896 -95.42779463515062) bank40418 +40419 POINT(29.81125386772443 -95.54895979826047) bank40419 +40420 POINT(30.567324605120913 -95.24881611929591) bank40420 +40421 POINT(30.27416718730016 -95.60533674219387) bank40421 +40422 POINT(28.96059426599955 -96.18636859788828) bank40422 +40423 POINT(29.117001434587362 -95.00432920906084) bank40423 +40424 POINT(29.298977459054758 -95.2164981615678) bank40424 +40425 POINT(29.787852159575653 -94.94849654469014) bank40425 +40426 POINT(29.165115332384723 -96.2604502789322) bank40426 +40427 POINT(30.739735383295653 -96.33096203383954) bank40427 +40428 POINT(29.97928301284717 -95.9611862532782) bank40428 +40429 POINT(29.943387283021387 -95.95623557755228) bank40429 +40430 POINT(29.217432356374296 -94.91313621763099) bank40430 +40431 POINT(29.45602982947605 -95.58888434426031) bank40431 +40432 POINT(30.298736921286025 -95.53779469037237) bank40432 +40433 POINT(30.389196979189574 -95.18014850904302) bank40433 +40434 POINT(28.917414191541006 -94.58613249352616) bank40434 +40435 POINT(29.750120394005815 -95.56116432042651) bank40435 +40436 POINT(29.230306609170164 -95.47255460785392) bank40436 +40437 POINT(30.35937445297365 -95.6260997805757) bank40437 +40438 POINT(30.256892144136444 -95.38597370860923) bank40438 +40439 POINT(30.52957847540227 -95.71592064811614) bank40439 +40440 POINT(29.580030288597303 -95.30006750453366) bank40440 +40441 POINT(30.328632017506255 -94.79753667909877) bank40441 +40442 POINT(30.037918366644014 -95.44751724015292) bank40442 +40443 POINT(29.357001185062586 -95.90742763308671) bank40443 +40444 POINT(29.957013012734784 -94.4509218186117) bank40444 +40445 POINT(30.457653533019236 -96.125532602258) bank40445 +40446 POINT(30.68050123419906 -96.03607464584552) bank40446 +40447 POINT(28.977769026318548 -94.45351351343682) bank40447 +40448 POINT(30.56944774411231 -95.85153543431048) bank40448 +40449 POINT(28.846711799949517 -95.15333295898547) bank40449 +40450 POINT(29.96239288561128 -96.36123301263122) bank40450 +40451 POINT(30.187338451964774 -94.48468431073252) bank40451 +40452 POINT(30.158215947787944 -95.18783698436067) bank40452 +40453 POINT(29.624609141981978 -95.448708023958) bank40453 +40454 POINT(30.15302098991481 -96.00234533527914) bank40454 +40455 POINT(29.99390450035898 -94.86237611507728) bank40455 +40456 POINT(30.221700516381105 -95.31494565098095) bank40456 +40457 POINT(30.592544153841583 -95.52037503098826) bank40457 +40458 POINT(29.9613093828591 -95.91528204672939) bank40458 +40459 POINT(29.00457155838063 -95.34719477898021) bank40459 +40460 POINT(29.925631227567322 -96.01346373860525) bank40460 +40461 POINT(29.93640636619984 -95.75021657262731) bank40461 +40462 POINT(28.852230266857106 -94.59494017757333) bank40462 +40463 POINT(30.11872988250921 -96.06150030192943) bank40463 +40464 POINT(29.066303829424722 -95.59958629665388) bank40464 +40465 POINT(29.826766746761233 -95.40515147624734) bank40465 +40466 POINT(29.590586078387236 -96.23391662355358) bank40466 +40467 POINT(28.761512885763896 -94.90817189968774) bank40467 +40468 POINT(28.882117082251852 -95.73530545281399) bank40468 +40469 POINT(30.58940144141043 -95.15589604581488) bank40469 +40470 POINT(29.479682870887594 -94.41224623128247) bank40470 +40471 POINT(29.693562526059303 -96.26453214544779) bank40471 +40472 POINT(29.667474668944937 -95.83991241369513) bank40472 +40473 POINT(29.142599923781898 -95.7435008899406) bank40473 +40474 POINT(30.362991538273473 -94.37916942323032) bank40474 +40475 POINT(30.70582367624122 -95.2950045765812) bank40475 +40476 POINT(28.877417248732968 -95.36256311459817) bank40476 +40477 POINT(28.993671181608097 -94.41297680912547) bank40477 +40478 POINT(30.659035604952518 -95.12932315734521) bank40478 +40479 POINT(30.69099360142416 -95.85825446917828) bank40479 +40480 POINT(30.066980626080984 -95.17370357206458) bank40480 +40481 POINT(29.532850127932406 -94.55214136572717) bank40481 +40482 POINT(30.7228395395091 -95.11789705160561) bank40482 +40483 POINT(30.387340483622044 -95.14067673327614) bank40483 +40484 POINT(28.915028407490436 -96.10428373227586) bank40484 +40485 POINT(30.7501573852895 -96.16936574134203) bank40485 +40486 POINT(30.47222019275995 -95.84440822551107) bank40486 +40487 POINT(29.15558163976559 -96.03733413122616) bank40487 +40488 POINT(30.491998000341347 -96.0435335694798) bank40488 +40489 POINT(30.131439286504897 -95.8842057449452) bank40489 +40490 POINT(29.223865897000852 -95.8815162189501) bank40490 +40491 POINT(30.529079998041404 -95.4410078717786) bank40491 +40492 POINT(30.450262111572755 -95.9341682700912) bank40492 +40493 POINT(29.476106873412927 -95.21017488179936) bank40493 +40494 POINT(29.670483154630837 -96.01661173069247) bank40494 +40495 POINT(30.4936780529093 -95.62082405754812) bank40495 +40496 POINT(28.918886110199317 -95.8780101996801) bank40496 +40497 POINT(29.800447441499017 -95.62199015525997) bank40497 +40498 POINT(29.356266049180977 -94.38402773594872) bank40498 +40499 POINT(29.22416225829442 -95.22767339953087) bank40499 +40500 POINT(30.268869014565617 -96.24246076988032) bank40500 +40501 POINT(29.979424197590642 -94.70809275134195) bank40501 +40502 POINT(29.753815315388927 -95.5606562796323) bank40502 +40503 POINT(30.238390004576907 -95.9837532135479) bank40503 +40504 POINT(29.90563657874138 -96.28330894217243) bank40504 +40505 POINT(30.46118653519443 -94.47128703165941) bank40505 +40506 POINT(29.20977572115938 -95.73807322751858) bank40506 +40507 POINT(29.28708289577206 -95.05140604227248) bank40507 +40508 POINT(30.730119758597162 -96.10835541875794) bank40508 +40509 POINT(29.645739267483872 -94.42464947038708) bank40509 +40510 POINT(29.83855608874246 -94.82287277226541) bank40510 +40511 POINT(30.087600880777483 -94.77773623896185) bank40511 +40512 POINT(29.240664688588808 -95.83791050313856) bank40512 +40513 POINT(29.645214648768025 -95.77971250126495) bank40513 +40514 POINT(29.309441269418095 -94.83449517183185) bank40514 +40515 POINT(29.482956727643213 -96.13712166426917) bank40515 +40516 POINT(29.292686831325522 -96.12728785854952) bank40516 +40517 POINT(29.552702555140776 -95.5656417946863) bank40517 +40518 POINT(29.12488812693801 -95.13196845845734) bank40518 +40519 POINT(30.159894020029803 -95.50627536258204) bank40519 +40520 POINT(29.295535725915073 -94.97834759370389) bank40520 +40521 POINT(29.98789428877268 -95.02556604285611) bank40521 +40522 POINT(30.118440335823742 -95.27331265660185) bank40522 +40523 POINT(29.97836259810589 -96.02771990389377) bank40523 +40524 POINT(29.408685531280017 -95.98619033723809) bank40524 +40525 POINT(30.610382050856067 -95.4510695776667) bank40525 +40526 POINT(29.529259693630696 -96.14427830152613) bank40526 +40527 POINT(29.48176762693682 -95.3933973780766) bank40527 +40528 POINT(28.888870025151036 -95.21050705759771) bank40528 +40529 POINT(30.686493923223516 -95.74293158868646) bank40529 +40530 POINT(28.899193386471975 -96.04849225142931) bank40530 +40531 POINT(29.403196036908955 -95.31590548302238) bank40531 +40532 POINT(30.732089936462355 -95.1453079470606) bank40532 +40533 POINT(30.26331632421851 -96.07327414224032) bank40533 +40534 POINT(30.163474632295706 -94.59888284664231) bank40534 +40535 POINT(29.306587927734242 -95.99171803328446) bank40535 +40536 POINT(29.642067499661568 -94.50939287592597) bank40536 +40537 POINT(29.290972507179426 -95.80646253589516) bank40537 +40538 POINT(30.364902694751105 -94.94037910369306) bank40538 +40539 POINT(28.976317866725143 -96.06567415656504) bank40539 +40540 POINT(29.209549192865182 -94.90113958719326) bank40540 +40541 POINT(29.136487106543104 -96.0666868435926) bank40541 +40542 POINT(30.551895137460104 -96.19305815427978) bank40542 +40543 POINT(29.67632944592485 -96.29201090227647) bank40543 +40544 POINT(29.740983514532452 -94.92559153303397) bank40544 +40545 POINT(29.151708642896676 -95.91297017638246) bank40545 +40546 POINT(29.33341244883018 -95.47732323208334) bank40546 +40547 POINT(29.28037091358039 -94.91161316139271) bank40547 +40548 POINT(29.315007791846163 -95.2660089088991) bank40548 +40549 POINT(29.007511063129236 -95.00944544490936) bank40549 +40550 POINT(30.456221718677003 -94.92793845914781) bank40550 +40551 POINT(29.506267795003264 -95.63769058850234) bank40551 +40552 POINT(29.290056411731648 -94.42666019815177) bank40552 +40553 POINT(29.23780086670958 -95.09902433975661) bank40553 +40554 POINT(30.197760165867454 -95.02203209717145) bank40554 +40555 POINT(29.37530391827743 -96.33303606141835) bank40555 +40556 POINT(29.934277328221086 -94.87586124048683) bank40556 +40557 POINT(30.589185153489936 -94.88171628511232) bank40557 +40558 POINT(29.766733815645352 -95.64941448589336) bank40558 +40559 POINT(29.640955886281592 -94.39774110554117) bank40559 +40560 POINT(29.358506555394722 -94.64738534862927) bank40560 +40561 POINT(29.292623568696545 -96.28201982108625) bank40561 +40562 POINT(29.98359189162851 -95.4881430859798) bank40562 +40563 POINT(29.146000189871494 -95.30672145101845) bank40563 +40564 POINT(28.8686639616422 -95.7299428428416) bank40564 +40565 POINT(29.585670933355207 -96.22409833076198) bank40565 +40566 POINT(30.075871995462013 -95.71769768839923) bank40566 +40567 POINT(29.208805321725382 -94.79522542140217) bank40567 +40568 POINT(30.60774139062334 -95.11722922207252) bank40568 +40569 POINT(29.39053856377966 -94.49065620943502) bank40569 +40570 POINT(30.707586155382213 -94.94636871399564) bank40570 +40571 POINT(30.721852316484064 -95.26263086457017) bank40571 +40572 POINT(28.922578393682116 -94.72223945544486) bank40572 +40573 POINT(30.096511917599504 -95.77398153294307) bank40573 +40574 POINT(30.09453707287334 -95.39207873229334) bank40574 +40575 POINT(29.708857715643394 -94.7586158531525) bank40575 +40576 POINT(29.70431032161986 -96.3283175548869) bank40576 +40577 POINT(28.910447647387745 -95.61229307209422) bank40577 +40578 POINT(29.951060689386715 -94.77232375073577) bank40578 +40579 POINT(30.176337931854043 -95.68047508203861) bank40579 +40580 POINT(30.294107986714355 -95.86828478162893) bank40580 +40581 POINT(29.709539409020387 -96.20033451420419) bank40581 +40582 POINT(28.820727567335584 -95.03820964303767) bank40582 +40583 POINT(30.21798735277033 -94.77972427511219) bank40583 +40584 POINT(29.89052352024976 -95.42456692510638) bank40584 +40585 POINT(29.389272953852206 -94.7597727473971) bank40585 +40586 POINT(29.352577596776623 -95.19608079370109) bank40586 +40587 POINT(29.352705637716756 -95.33437091988128) bank40587 +40588 POINT(29.737924171827675 -95.57848999922729) bank40588 +40589 POINT(30.18534770831439 -96.01409422477104) bank40589 +40590 POINT(30.543282881495468 -94.79806682318016) bank40590 +40591 POINT(29.144057018883977 -95.50434297687968) bank40591 +40592 POINT(30.07598445198766 -95.06557871717474) bank40592 +40593 POINT(30.019588019602587 -94.64577136179233) bank40593 +40594 POINT(29.119444716803855 -95.94480949842153) bank40594 +40595 POINT(29.73763948352854 -94.6744754909748) bank40595 +40596 POINT(28.964317254593322 -95.52201181589362) bank40596 +40597 POINT(29.504195667811807 -95.45829337887305) bank40597 +40598 POINT(30.425267309347817 -96.0554974439757) bank40598 +40599 POINT(29.902787748148125 -94.45845939205479) bank40599 +40600 POINT(29.10421081690262 -95.65748018660484) bank40600 +40601 POINT(29.290123629256524 -95.65412942493936) bank40601 +40602 POINT(28.83985760538566 -94.65343872743844) bank40602 +40603 POINT(28.836480510503442 -95.74297103773186) bank40603 +40604 POINT(28.949277872233715 -96.11769308837033) bank40604 +40605 POINT(30.045947580930427 -95.96932311548602) bank40605 +40606 POINT(29.983590563793175 -94.96243535132997) bank40606 +40607 POINT(29.103245661874347 -96.33790394919666) bank40607 +40608 POINT(29.45209258290139 -96.16574927793191) bank40608 +40609 POINT(29.301963512770882 -96.30417068146055) bank40609 +40610 POINT(29.39580726171809 -95.45550928301245) bank40610 +40611 POINT(29.589993811195765 -94.40885243027898) bank40611 +40612 POINT(30.051691611322067 -95.8066860561592) bank40612 +40613 POINT(29.18530817103232 -94.47309016313557) bank40613 +40614 POINT(28.881016158882822 -94.42297323957068) bank40614 +40615 POINT(28.96251176306518 -95.54802409883797) bank40615 +40616 POINT(30.412414691745667 -94.57230916588858) bank40616 +40617 POINT(30.42445110959093 -96.05084710379947) bank40617 +40618 POINT(29.52188574053621 -95.32308446419812) bank40618 +40619 POINT(29.606699261671167 -94.69946920955653) bank40619 +40620 POINT(28.884334565257703 -95.03973975213223) bank40620 +40621 POINT(29.39248086289593 -95.07756255909212) bank40621 +40622 POINT(29.464394128615677 -94.61192436415723) bank40622 +40623 POINT(29.66557856095562 -95.89118009283747) bank40623 +40624 POINT(29.823208023184716 -94.54354269735953) bank40624 +40625 POINT(30.491857977416895 -95.11641606148662) bank40625 +40626 POINT(30.538184165599063 -95.99500931412796) bank40626 +40627 POINT(29.653295165334235 -95.57206930648584) bank40627 +40628 POINT(29.997713749803346 -96.21933055633677) bank40628 +40629 POINT(30.231509118791514 -95.71262818245215) bank40629 +40630 POINT(28.789901139594832 -95.89208523366676) bank40630 +40631 POINT(29.612256510482116 -95.59457316510044) bank40631 +40632 POINT(29.44755012897686 -94.51604917225126) bank40632 +40633 POINT(28.896839141927707 -95.07906883607595) bank40633 +40634 POINT(30.72214729405581 -94.43226659176626) bank40634 +40635 POINT(30.297159254049888 -95.35489519162095) bank40635 +40636 POINT(30.22221686383216 -94.62772203947118) bank40636 +40637 POINT(30.20082980700066 -96.20162789193073) bank40637 +40638 POINT(29.400391559363957 -95.09033334313945) bank40638 +40639 POINT(29.00132054551277 -96.0566576751931) bank40639 +40640 POINT(30.277250290521287 -94.44965836777506) bank40640 +40641 POINT(29.65979283031427 -95.4311245347268) bank40641 +40642 POINT(28.92740458783857 -96.09497434613303) bank40642 +40643 POINT(30.63032859271342 -96.0893455860839) bank40643 +40644 POINT(29.580059155878732 -95.45220438398034) bank40644 +40645 POINT(30.32254445728899 -94.48870653901186) bank40645 +40646 POINT(28.9840954589582 -95.35387111704277) bank40646 +40647 POINT(30.483558408191275 -95.11550870033773) bank40647 +40648 POINT(30.12171997121811 -95.25653631856909) bank40648 +40649 POINT(29.26535265405499 -96.14873708714448) bank40649 +40650 POINT(29.35117099289305 -95.74896412136957) bank40650 +40651 POINT(29.51681492521573 -94.52089411019611) bank40651 +40652 POINT(28.92704947901445 -95.74644932598288) bank40652 +40653 POINT(29.300325299952142 -96.15120779632724) bank40653 +40654 POINT(30.051286324301916 -96.1759512375852) bank40654 +40655 POINT(28.89593401087423 -95.18096725643423) bank40655 +40656 POINT(29.8353604623328 -95.08768429195835) bank40656 +40657 POINT(29.985515088449766 -94.97455297616673) bank40657 +40658 POINT(30.464844799317827 -95.40227495699988) bank40658 +40659 POINT(30.308979258779413 -95.97713663212849) bank40659 +40660 POINT(29.225360359559293 -95.01491014616525) bank40660 +40661 POINT(29.773866558738725 -94.91983034888075) bank40661 +40662 POINT(29.629831627882428 -94.88293329748032) bank40662 +40663 POINT(29.193802687864558 -95.98259723109784) bank40663 +40664 POINT(29.492127975518482 -94.72657276422123) bank40664 +40665 POINT(30.52235839226632 -95.95307675239007) bank40665 +40666 POINT(28.882217430375384 -96.06043502770464) bank40666 +40667 POINT(29.081237769441785 -95.59870001384898) bank40667 +40668 POINT(30.045582909384432 -94.55309399988019) bank40668 +40669 POINT(29.190934657539067 -96.3378805304847) bank40669 +40670 POINT(29.041896071746123 -94.47745293274534) bank40670 +40671 POINT(30.563964280247323 -95.12522302518417) bank40671 +40672 POINT(29.712850543534383 -94.42196609909557) bank40672 +40673 POINT(29.32950729038655 -94.76903886146157) bank40673 +40674 POINT(29.50290686299118 -95.47559832292565) bank40674 +40675 POINT(29.12568668802958 -95.41793268920071) bank40675 +40676 POINT(29.283079849734705 -95.99270628870505) bank40676 +40677 POINT(29.92882785567184 -96.1692682513193) bank40677 +40678 POINT(29.68918937350559 -95.00836456062113) bank40678 +40679 POINT(29.15449995921344 -95.54503272729244) bank40679 +40680 POINT(30.010772272686506 -96.19770237729801) bank40680 +40681 POINT(30.257077353391168 -96.15149087889739) bank40681 +40682 POINT(30.637258225521812 -95.88452375643566) bank40682 +40683 POINT(29.94421860599626 -95.69667683831781) bank40683 +40684 POINT(29.625951172076146 -94.8092074163768) bank40684 +40685 POINT(30.673027296466657 -94.98703553225626) bank40685 +40686 POINT(30.251608457664993 -96.23707716608006) bank40686 +40687 POINT(30.101607855383175 -95.30740267259162) bank40687 +40688 POINT(30.247697560439754 -95.405234702914) bank40688 +40689 POINT(30.351828792485634 -94.43847654422551) bank40689 +40690 POINT(29.443457001338153 -94.96669003518917) bank40690 +40691 POINT(30.432423605355655 -95.82905730726824) bank40691 +40692 POINT(29.83596736687461 -96.19247776660939) bank40692 +40693 POINT(30.047201978750742 -96.06100200325949) bank40693 +40694 POINT(30.220605298114297 -95.38618836807586) bank40694 +40695 POINT(29.78646703129418 -95.99221922665348) bank40695 +40696 POINT(30.741496662654246 -94.61381761141156) bank40696 +40697 POINT(29.338380992269382 -96.24699525688928) bank40697 +40698 POINT(30.578898643029287 -95.74222422531199) bank40698 +40699 POINT(30.332990383056213 -95.38726486088962) bank40699 +40700 POINT(29.074926843318632 -95.92964939795537) bank40700 +40701 POINT(28.95335832553838 -95.9523297312005) bank40701 +40702 POINT(30.657889124412076 -94.52162572173627) bank40702 +40703 POINT(29.489101428795266 -96.32079144328263) bank40703 +40704 POINT(30.017070641863533 -94.73881244845718) bank40704 +40705 POINT(30.300965341507457 -95.70538366335698) bank40705 +40706 POINT(29.367332181790623 -94.4714543302379) bank40706 +40707 POINT(29.827917078961562 -94.83412185402024) bank40707 +40708 POINT(29.02489203427238 -95.2324354174376) bank40708 +40709 POINT(28.88665915242546 -94.80938656088706) bank40709 +40710 POINT(30.542087986059173 -95.34891950586564) bank40710 +40711 POINT(30.75237306103933 -94.52269404411427) bank40711 +40712 POINT(30.163797571045727 -95.39731901625633) bank40712 +40713 POINT(30.615961357647226 -94.91525412985588) bank40713 +40714 POINT(30.144558302653426 -95.63969334661654) bank40714 +40715 POINT(29.388057451122318 -94.60423412762972) bank40715 +40716 POINT(28.81263696992681 -95.8050090919091) bank40716 +40717 POINT(30.024061096069104 -94.48173008629048) bank40717 +40718 POINT(30.72838997920714 -95.91375942222369) bank40718 +40719 POINT(30.30916329302609 -95.24149336471935) bank40719 +40720 POINT(29.906531552069037 -94.83768058855568) bank40720 +40721 POINT(29.189926941845435 -96.29924562546987) bank40721 +40722 POINT(28.974152296153683 -95.18450120449923) bank40722 +40723 POINT(28.942272998571568 -95.06673386498137) bank40723 +40724 POINT(30.126201655035562 -95.36325664673855) bank40724 +40725 POINT(28.789276347079433 -95.35695415956104) bank40725 +40726 POINT(30.215745200742724 -95.30205448508829) bank40726 +40727 POINT(28.86528354226355 -94.83961811404944) bank40727 +40728 POINT(30.410702378590923 -95.54011245210354) bank40728 +40729 POINT(29.41749688800819 -95.04155905491174) bank40729 +40730 POINT(28.810440495132184 -95.6865308422391) bank40730 +40731 POINT(30.565777449770238 -94.52521896683294) bank40731 +40732 POINT(29.121100647217375 -95.41887924156569) bank40732 +40733 POINT(29.132569108842915 -94.83159024836432) bank40733 +40734 POINT(28.837255689146808 -95.85355351177076) bank40734 +40735 POINT(29.4221741919687 -96.23787513891956) bank40735 +40736 POINT(28.853032611485936 -95.78961950143615) bank40736 +40737 POINT(30.759982160492736 -95.37487116830778) bank40737 +40738 POINT(29.541426314229184 -95.10956719960181) bank40738 +40739 POINT(29.304708636973896 -96.26377324724157) bank40739 +40740 POINT(28.82813089867234 -95.74272040308924) bank40740 +40741 POINT(30.374353956992895 -95.38663247146738) bank40741 +40742 POINT(29.518195396042852 -96.05396211207976) bank40742 +40743 POINT(29.121451517939175 -95.28109165670404) bank40743 +40744 POINT(29.75737996418253 -95.59096194580316) bank40744 +40745 POINT(30.200254460278533 -96.1826875813286) bank40745 +40746 POINT(29.900381595943184 -95.07188183272652) bank40746 +40747 POINT(30.583457837722744 -94.98441973823492) bank40747 +40748 POINT(29.30639048664987 -95.48425145399045) bank40748 +40749 POINT(29.253464749712535 -96.12224920810759) bank40749 +40750 POINT(29.24227429548008 -95.486760540658) bank40750 +40751 POINT(28.963264555791422 -95.69046970345325) bank40751 +40752 POINT(29.999532879663402 -95.946900816079) bank40752 +40753 POINT(29.7484898406975 -95.52766125805158) bank40753 +40754 POINT(28.98232388786667 -94.76002387503038) bank40754 +40755 POINT(29.94043069555519 -95.9644398283579) bank40755 +40756 POINT(29.758805528287716 -95.59006811363658) bank40756 +40757 POINT(29.010155661182058 -95.65902616956124) bank40757 +40758 POINT(29.75601806004482 -94.41325771362706) bank40758 +40759 POINT(29.766213942533458 -94.6666964935795) bank40759 +40760 POINT(30.100019926755778 -95.00109486745617) bank40760 +40761 POINT(30.10670599085951 -95.18613384718898) bank40761 +40762 POINT(29.943323691838287 -96.11423395778772) bank40762 +40763 POINT(30.520516533166095 -95.85046419730784) bank40763 +40764 POINT(30.089405465176508 -96.04562784771402) bank40764 +40765 POINT(30.550911439358767 -94.41579052934915) bank40765 +40766 POINT(30.068168282009502 -96.26616726490244) bank40766 +40767 POINT(28.931505730641565 -95.57091517603445) bank40767 +40768 POINT(30.2023686936731 -94.97209048747591) bank40768 +40769 POINT(28.77059858906046 -95.16961685651367) bank40769 +40770 POINT(29.719270292857544 -95.65144245971464) bank40770 +40771 POINT(30.445459607461125 -94.51328645918764) bank40771 +40772 POINT(30.319693871684798 -96.28956247858025) bank40772 +40773 POINT(29.248337347022495 -96.04744853605965) bank40773 +40774 POINT(30.562362195036975 -94.8509741054076) bank40774 +40775 POINT(29.36499858551379 -96.16570701140797) bank40775 +40776 POINT(29.34148861747652 -95.55957953505593) bank40776 +40777 POINT(30.20709713887478 -94.71233455375612) bank40777 +40778 POINT(30.428189732000895 -95.67963886945495) bank40778 +40779 POINT(29.927785704057595 -95.45356625406319) bank40779 +40780 POINT(28.86978300555615 -95.54256392843494) bank40780 +40781 POINT(30.717807045701495 -95.61556700401582) bank40781 +40782 POINT(29.571342576076283 -96.10588526829314) bank40782 +40783 POINT(29.347342191659166 -96.3022709885296) bank40783 +40784 POINT(29.04519983139119 -94.43312212183073) bank40784 +40785 POINT(29.70311893864052 -94.70219832548638) bank40785 +40786 POINT(28.87094455032516 -94.95617094303685) bank40786 +40787 POINT(29.72917962798971 -95.0966120685812) bank40787 +40788 POINT(30.72070537764366 -95.15865750650352) bank40788 +40789 POINT(29.688653274829182 -94.95976437258993) bank40789 +40790 POINT(30.252269755623033 -95.4957750777229) bank40790 +40791 POINT(29.37591958879892 -94.76218853153898) bank40791 +40792 POINT(30.05311568843255 -95.06974058962487) bank40792 +40793 POINT(29.766109828697545 -94.95718902665068) bank40793 +40794 POINT(29.935089547170364 -95.82454895184834) bank40794 +40795 POINT(28.81617514744304 -95.08775363009005) bank40795 +40796 POINT(29.34775693449737 -95.22935698031611) bank40796 +40797 POINT(30.353978696223038 -94.75062912714313) bank40797 +40798 POINT(29.093380242082404 -94.91511147098639) bank40798 +40799 POINT(29.259243408108706 -96.36659443057287) bank40799 +40800 POINT(30.16400167050559 -94.80785146248725) bank40800 +40801 POINT(29.487585387312972 -96.08572381388969) bank40801 +40802 POINT(29.577214717554774 -94.9320422654579) bank40802 +40803 POINT(28.931466455499457 -94.41302510547766) bank40803 +40804 POINT(29.329540880161773 -95.6523029350573) bank40804 +40805 POINT(29.46673452173253 -95.9718509922276) bank40805 +40806 POINT(29.939540902152057 -95.64322918685309) bank40806 +40807 POINT(29.34242213993999 -94.57633314334203) bank40807 +40808 POINT(29.78710887916473 -94.79521696063222) bank40808 +40809 POINT(30.34165924936439 -95.08535796202722) bank40809 +40810 POINT(30.566734914663822 -95.5178580933173) bank40810 +40811 POINT(29.219379494842364 -95.37261494074929) bank40811 +40812 POINT(29.29643502942006 -95.95120145929798) bank40812 +40813 POINT(29.161297902432022 -96.18706211771551) bank40813 +40814 POINT(30.3011247499589 -94.60640821465407) bank40814 +40815 POINT(29.376024423098123 -96.32347853972394) bank40815 +40816 POINT(28.813348929624745 -94.65644007715443) bank40816 +40817 POINT(30.73031956474617 -94.79989087333324) bank40817 +40818 POINT(29.976738870438812 -95.6990826046208) bank40818 +40819 POINT(29.065067962324704 -96.31215804803334) bank40819 +40820 POINT(29.84242124624166 -95.95730332562205) bank40820 +40821 POINT(30.501432711718806 -94.39703125723221) bank40821 +40822 POINT(29.826808061739953 -96.28473276463332) bank40822 +40823 POINT(29.322569924646018 -94.71258823882222) bank40823 +40824 POINT(28.87525715208455 -95.75339966474812) bank40824 +40825 POINT(29.024619354367452 -94.37793665675554) bank40825 +40826 POINT(30.557651125148208 -96.32803341809615) bank40826 +40827 POINT(29.977358285681518 -95.39068709788313) bank40827 +40828 POINT(29.29093452240097 -95.18714028738655) bank40828 +40829 POINT(30.372487265979892 -94.79056324026621) bank40829 +40830 POINT(29.254566598341718 -95.17083978858545) bank40830 +40831 POINT(29.643152888138363 -96.30214469255817) bank40831 +40832 POINT(29.877920153074506 -94.81903209310364) bank40832 +40833 POINT(29.78942152537219 -95.46506704839686) bank40833 +40834 POINT(30.01488649844105 -96.26877366465811) bank40834 +40835 POINT(30.0998649190549 -96.2685350777274) bank40835 +40836 POINT(30.663166163863487 -94.3839628077086) bank40836 +40837 POINT(29.701853153599135 -95.49410319324784) bank40837 +40838 POINT(28.940587419609894 -94.82700162524112) bank40838 +40839 POINT(30.194698982101652 -95.02469461373458) bank40839 +40840 POINT(29.013275149350214 -95.28119255189834) bank40840 +40841 POINT(30.5463345235671 -95.49865924859397) bank40841 +40842 POINT(30.709113400323364 -95.30856082674087) bank40842 +40843 POINT(29.323249312421375 -96.18125798960833) bank40843 +40844 POINT(30.679265547449656 -96.03551613573309) bank40844 +40845 POINT(29.682081612705574 -94.38673137006921) bank40845 +40846 POINT(29.24515153618306 -95.19628262883128) bank40846 +40847 POINT(29.549256876339697 -95.86496417296561) bank40847 +40848 POINT(28.886572226029575 -94.97646568077442) bank40848 +40849 POINT(30.39546645949046 -94.41595681365217) bank40849 +40850 POINT(29.479845211362782 -95.15798234246407) bank40850 +40851 POINT(30.45685801464591 -95.33460062687627) bank40851 +40852 POINT(29.291442612164676 -95.0435935714058) bank40852 +40853 POINT(30.17726690666614 -94.64072247727114) bank40853 +40854 POINT(29.798687211799695 -95.28545399059361) bank40854 +40855 POINT(30.427750149900024 -94.56373172591161) bank40855 +40856 POINT(30.51570005628011 -95.61797186493806) bank40856 +40857 POINT(29.288269513150453 -96.24457539084472) bank40857 +40858 POINT(28.782999892181685 -96.03730887280233) bank40858 +40859 POINT(30.40440655322621 -95.44788421231164) bank40859 +40860 POINT(29.672968291636295 -95.08529238878849) bank40860 +40861 POINT(29.613643699159365 -94.5707546824666) bank40861 +40862 POINT(30.38567188897024 -95.80896025091792) bank40862 +40863 POINT(29.32804454515295 -95.47157658979239) bank40863 +40864 POINT(29.405445002408676 -95.54339385183897) bank40864 +40865 POINT(30.274095090614548 -94.56245294853832) bank40865 +40866 POINT(30.75883699866697 -94.40240650047521) bank40866 +40867 POINT(29.284334039804655 -94.69995100334077) bank40867 +40868 POINT(29.942448900002386 -95.00605855320603) bank40868 +40869 POINT(29.8555393146551 -95.2496647630147) bank40869 +40870 POINT(28.925688424055814 -96.00527599002459) bank40870 +40871 POINT(29.10906811378922 -95.0126542790631) bank40871 +40872 POINT(29.483203808936103 -94.38218589599747) bank40872 +40873 POINT(28.86933997195525 -95.99722961642581) bank40873 +40874 POINT(29.43891595729909 -95.31238819730218) bank40874 +40875 POINT(29.112871432444376 -96.2754237583228) bank40875 +40876 POINT(30.396819928946876 -94.49772526334755) bank40876 +40877 POINT(30.0502751829956 -96.11034966365791) bank40877 +40878 POINT(30.191159328443387 -95.8901603843467) bank40878 +40879 POINT(30.716066732029603 -95.1150130938216) bank40879 +40880 POINT(29.252156075716922 -94.87500497456715) bank40880 +40881 POINT(29.19070368162671 -96.2393436482439) bank40881 +40882 POINT(29.749151914204297 -94.4470708109442) bank40882 +40883 POINT(29.8539735610872 -95.57965408599986) bank40883 +40884 POINT(29.459189967185743 -94.49343766664319) bank40884 +40885 POINT(30.64291539176289 -95.78202128502532) bank40885 +40886 POINT(29.399433220643104 -96.25410786587281) bank40886 +40887 POINT(30.715918660497984 -95.58102713741876) bank40887 +40888 POINT(29.204296338610195 -95.11634757689242) bank40888 +40889 POINT(30.49712157461447 -95.94521229188454) bank40889 +40890 POINT(29.376689619437546 -94.59309355959533) bank40890 +40891 POINT(29.7365685946299 -96.11435583329268) bank40891 +40892 POINT(29.5785771956173 -95.58119171882792) bank40892 +40893 POINT(29.372366233261207 -95.28248603154441) bank40893 +40894 POINT(30.11779217352661 -96.34623467767389) bank40894 +40895 POINT(28.983713989471788 -94.96631330986861) bank40895 +40896 POINT(29.456030669855608 -95.97652698698683) bank40896 +40897 POINT(29.878937568174162 -94.88200901250235) bank40897 +40898 POINT(28.85766646607666 -95.48647771939757) bank40898 +40899 POINT(29.357321281656787 -95.63979132386788) bank40899 +40900 POINT(30.65023673676399 -94.93495034553781) bank40900 +40901 POINT(30.709136156419262 -95.80792493093149) bank40901 +40902 POINT(28.97270079676305 -95.01860773141615) bank40902 +40903 POINT(30.33353678886351 -95.08980008790512) bank40903 +40904 POINT(29.726135679884806 -94.48653077426994) bank40904 +40905 POINT(30.6685734994819 -94.98574444049198) bank40905 +40906 POINT(29.685064619038652 -95.38262534427146) bank40906 +40907 POINT(30.13487049039153 -96.01216470007323) bank40907 +40908 POINT(29.405484254347122 -95.60484296491629) bank40908 +40909 POINT(30.053491805922786 -95.14110223609462) bank40909 +40910 POINT(30.091228486698526 -94.57556266074486) bank40910 +40911 POINT(30.076777248733727 -95.11328384472408) bank40911 +40912 POINT(30.569030504992533 -95.80084739980322) bank40912 +40913 POINT(30.639404268432017 -95.3675153616407) bank40913 +40914 POINT(30.489491285843584 -95.78703220664427) bank40914 +40915 POINT(29.039579520239023 -95.49919732511411) bank40915 +40916 POINT(29.315507466596877 -95.15556524069034) bank40916 +40917 POINT(30.550362625305773 -96.34615881808338) bank40917 +40918 POINT(30.412674400394714 -95.92589893964451) bank40918 +40919 POINT(29.41806810009987 -95.6978889403762) bank40919 +40920 POINT(30.41359815995742 -96.28599712051984) bank40920 +40921 POINT(29.906532290107908 -95.34686698990521) bank40921 +40922 POINT(28.96799852571459 -95.7457805667267) bank40922 +40923 POINT(29.57806188426025 -94.81778618505399) bank40923 +40924 POINT(29.529874247974078 -95.59669529097923) bank40924 +40925 POINT(29.49332643431919 -95.51900640481722) bank40925 +40926 POINT(30.425719705820963 -94.37467150461879) bank40926 +40927 POINT(29.762167370718416 -95.96084087558846) bank40927 +40928 POINT(28.971037174721147 -95.40253364082916) bank40928 +40929 POINT(28.95518700539646 -95.15259132732481) bank40929 +40930 POINT(29.47448732277986 -95.8548509615895) bank40930 +40931 POINT(30.139725946313785 -94.38158071117317) bank40931 +40932 POINT(29.84599820162664 -95.17595575279854) bank40932 +40933 POINT(29.56657874884932 -95.10609161680763) bank40933 +40934 POINT(29.847314322441374 -95.40162494922666) bank40934 +40935 POINT(30.559371578733554 -95.55324212712343) bank40935 +40936 POINT(29.12544393856787 -95.52987270698463) bank40936 +40937 POINT(29.943442300663616 -94.70382067611098) bank40937 +40938 POINT(29.31806734317384 -95.9061039503435) bank40938 +40939 POINT(30.23795970296171 -95.47972668978852) bank40939 +40940 POINT(30.222406308824382 -94.73202813021645) bank40940 +40941 POINT(29.688302249695564 -94.96639202821972) bank40941 +40942 POINT(28.971448401433975 -96.00117107381314) bank40942 +40943 POINT(30.116715925230057 -94.77600101910795) bank40943 +40944 POINT(29.802992544449143 -95.73350462179309) bank40944 +40945 POINT(29.7704748833251 -95.65520043133611) bank40945 +40946 POINT(30.288990014529208 -96.19343138782249) bank40946 +40947 POINT(30.568383350755084 -95.17884636206743) bank40947 +40948 POINT(30.68583135692884 -95.89710618551979) bank40948 +40949 POINT(30.497566343878983 -94.60472181866562) bank40949 +40950 POINT(29.827533891192118 -95.09657807358444) bank40950 +40951 POINT(29.96213503169183 -96.2051898639324) bank40951 +40952 POINT(30.246511145123876 -95.64436016832968) bank40952 +40953 POINT(29.346084759516312 -94.60029795027316) bank40953 +40954 POINT(30.492897218382584 -95.60964497401328) bank40954 +40955 POINT(29.45441281441558 -95.37273982096794) bank40955 +40956 POINT(29.077658286972927 -95.45671685711505) bank40956 +40957 POINT(29.38128330356857 -95.27700393655932) bank40957 +40958 POINT(29.460807896884948 -95.35615787860817) bank40958 +40959 POINT(29.341932296825462 -95.84205330681539) bank40959 +40960 POINT(30.128040075469336 -94.84570803279304) bank40960 +40961 POINT(30.340559093703796 -95.06276839019341) bank40961 +40962 POINT(30.37105356566215 -96.04716398918514) bank40962 +40963 POINT(30.200170751579574 -96.06661044160782) bank40963 +40964 POINT(29.730953213222655 -95.42997379546506) bank40964 +40965 POINT(30.55642888329877 -95.35301699645504) bank40965 +40966 POINT(29.75771755578007 -96.33882153004886) bank40966 +40967 POINT(30.329265870612367 -94.79685414739646) bank40967 +40968 POINT(29.59130817227869 -95.04596144749722) bank40968 +40969 POINT(30.596540561917113 -94.45924225448546) bank40969 +40970 POINT(30.56030814748302 -95.17740407162194) bank40970 +40971 POINT(30.698329046323522 -95.61410429552548) bank40971 +40972 POINT(29.019451471966242 -96.17355205206678) bank40972 +40973 POINT(29.888873218713346 -94.57921577300633) bank40973 +40974 POINT(29.120161424635587 -94.51915789551676) bank40974 +40975 POINT(29.67019778094882 -95.93913238633276) bank40975 +40976 POINT(29.96720192978213 -95.4907873254737) bank40976 +40977 POINT(29.065467608510385 -94.85243182075473) bank40977 +40978 POINT(29.00437950148015 -95.55115879967232) bank40978 +40979 POINT(29.240603338816822 -95.074445627741) bank40979 +40980 POINT(30.26233438424484 -94.85772028279918) bank40980 +40981 POINT(29.673412866751857 -94.49099540735222) bank40981 +40982 POINT(29.300903594930805 -95.20859613576884) bank40982 +40983 POINT(29.95116736701391 -95.05863876000281) bank40983 +40984 POINT(30.01185701433489 -94.47750628142515) bank40984 +40985 POINT(29.22382272088609 -94.91749228416778) bank40985 +40986 POINT(29.634256035658083 -94.63320042851127) bank40986 +40987 POINT(28.991885342822194 -95.51332950936174) bank40987 +40988 POINT(30.209282476040467 -94.66793418120427) bank40988 +40989 POINT(29.719231805811816 -96.16003159163557) bank40989 +40990 POINT(29.514373692855074 -95.61257653326402) bank40990 +40991 POINT(29.110637310572734 -95.79942889679987) bank40991 +40992 POINT(29.346690105648428 -95.637211202433) bank40992 +40993 POINT(29.810067073349476 -95.55292089447913) bank40993 +40994 POINT(29.223517874908556 -95.11429543176632) bank40994 +40995 POINT(28.947838329765492 -95.03806417527625) bank40995 +40996 POINT(30.60718351042383 -95.11809704628827) bank40996 +40997 POINT(29.744096377583524 -94.69826414447549) bank40997 +40998 POINT(29.160265836663886 -95.35602338637548) bank40998 +40999 POINT(28.839269941833003 -95.09877748575046) bank40999 +41000 POINT(29.563351268793028 -95.59091865133968) bank41000 +41001 POINT(28.960545930128497 -95.5497228551823) bank41001 +41002 POINT(29.488568139640453 -95.48516788851352) bank41002 +41003 POINT(29.64494963746432 -95.26584247533465) bank41003 +41004 POINT(30.749361436643184 -94.68351811541538) bank41004 +41005 POINT(30.50738260875811 -95.24747139040743) bank41005 +41006 POINT(29.311500503362208 -94.52163223321233) bank41006 +41007 POINT(30.364767790837043 -94.53163720586859) bank41007 +41008 POINT(28.994911713006683 -95.02224414381458) bank41008 +41009 POINT(28.866794586170577 -95.2458477342206) bank41009 +41010 POINT(30.230476836013374 -95.60450661290275) bank41010 +41011 POINT(30.56324933462778 -95.49545689025143) bank41011 +41012 POINT(29.60469218078734 -95.78086291953866) bank41012 +41013 POINT(29.959708228828315 -94.46564573634973) bank41013 +41014 POINT(30.710093918517945 -95.32358895057787) bank41014 +41015 POINT(30.47032588057424 -95.97594402501984) bank41015 +41016 POINT(29.887468560359746 -95.84231396510415) bank41016 +41017 POINT(28.828883883073267 -96.10119941933753) bank41017 +41018 POINT(29.114370225058966 -94.81357882576415) bank41018 +41019 POINT(28.77045771088065 -94.7635853522965) bank41019 +41020 POINT(29.35177397050465 -94.85358183389363) bank41020 +41021 POINT(29.43091822484807 -95.8598419381221) bank41021 +41022 POINT(29.232906226621548 -95.10256754436034) bank41022 +41023 POINT(29.432616409787006 -94.98505929863377) bank41023 +41024 POINT(28.820593168911522 -95.33395375244864) bank41024 +41025 POINT(30.23876257284986 -94.863997066895) bank41025 +41026 POINT(29.838419790604824 -95.48796378457358) bank41026 +41027 POINT(29.962656175127325 -94.7852851717564) bank41027 +41028 POINT(30.079571308571524 -95.23001593558821) bank41028 +41029 POINT(30.207209559510435 -95.4776557310635) bank41029 +41030 POINT(29.507279863826604 -95.87947449596754) bank41030 +41031 POINT(29.881197371100107 -94.51512565429678) bank41031 +41032 POINT(30.3935334056747 -94.40945238429325) bank41032 +41033 POINT(30.078474639958664 -94.80530473684757) bank41033 +41034 POINT(30.20913121971978 -94.38807917529807) bank41034 +41035 POINT(29.895136631647482 -95.99120692824845) bank41035 +41036 POINT(29.15438549217378 -95.79947355174896) bank41036 +41037 POINT(29.59428173253983 -94.93147745379606) bank41037 +41038 POINT(29.41634319275472 -94.90428883016051) bank41038 +41039 POINT(28.889292269247655 -95.76141462569618) bank41039 +41040 POINT(29.403287732449606 -94.72851651593035) bank41040 +41041 POINT(29.91701154661776 -95.83448741035956) bank41041 +41042 POINT(29.441916319074323 -96.193569797609) bank41042 +41043 POINT(30.731924609914476 -94.4410710037925) bank41043 +41044 POINT(28.82722577051633 -94.62271923543526) bank41044 +41045 POINT(28.88575905443706 -94.47887311009414) bank41045 +41046 POINT(29.078830012644897 -96.00232668223427) bank41046 +41047 POINT(29.59693055808638 -94.77570604231869) bank41047 +41048 POINT(29.96758645659632 -95.26222915354529) bank41048 +41049 POINT(30.160889753505586 -94.46954488501181) bank41049 +41050 POINT(30.040171651221684 -94.53913953189446) bank41050 +41051 POINT(29.07048796406369 -94.93678459843836) bank41051 +41052 POINT(30.643252383061498 -95.95185081806007) bank41052 +41053 POINT(29.945859097731805 -95.71019615780826) bank41053 +41054 POINT(30.441911139905468 -95.94800445777557) bank41054 +41055 POINT(30.05673094230296 -95.02915314214931) bank41055 +41056 POINT(30.41022344840752 -94.89876756621871) bank41056 +41057 POINT(29.54045916041423 -95.63407758887486) bank41057 +41058 POINT(29.901239120718564 -95.59382224197284) bank41058 +41059 POINT(30.47838143800869 -96.368527870793) bank41059 +41060 POINT(30.412001560552646 -95.73702711432345) bank41060 +41061 POINT(29.918910335532793 -94.45374412346558) bank41061 +41062 POINT(30.737621090556278 -94.47586002794782) bank41062 +41063 POINT(30.597207702817848 -96.04178664441531) bank41063 +41064 POINT(29.33033046289771 -94.41550673589225) bank41064 +41065 POINT(29.494355125709703 -95.4429469973716) bank41065 +41066 POINT(29.75256866253702 -95.40616497045639) bank41066 +41067 POINT(29.860876730786035 -94.65718254482205) bank41067 +41068 POINT(30.486050834896464 -96.21931683272048) bank41068 +41069 POINT(30.25518813075253 -94.5664454878861) bank41069 +41070 POINT(29.550585405011635 -95.11571698958642) bank41070 +41071 POINT(29.71970060847254 -94.91118267104838) bank41071 +41072 POINT(30.401367766316483 -95.58548764401583) bank41072 +41073 POINT(29.79490956524171 -96.2785699595852) bank41073 +41074 POINT(29.857010216530032 -95.63580292364695) bank41074 +41075 POINT(28.971911416800783 -95.14640067674715) bank41075 +41076 POINT(30.28394537438482 -96.0958806268244) bank41076 +41077 POINT(29.117175107347176 -95.36612681747725) bank41077 +41078 POINT(28.85944853881681 -95.78235074894029) bank41078 +41079 POINT(28.777511824254386 -95.1384568661316) bank41079 +41080 POINT(30.136746926670156 -96.14469474000425) bank41080 +41081 POINT(30.35102694129036 -96.36224085661897) bank41081 +41082 POINT(29.629425570343063 -95.92804697182788) bank41082 +41083 POINT(28.869801178215575 -95.67561685600457) bank41083 +41084 POINT(29.142766125823414 -94.58238841618638) bank41084 +41085 POINT(29.115634347820865 -95.61246292503517) bank41085 +41086 POINT(29.337132409248657 -95.7207219036014) bank41086 +41087 POINT(28.821789736013642 -96.03872262746692) bank41087 +41088 POINT(29.56942006757527 -94.72984226617659) bank41088 +41089 POINT(29.522443792843987 -95.20857172316684) bank41089 +41090 POINT(29.039955228664585 -94.6316671432941) bank41090 +41091 POINT(29.251311123542674 -95.95053033821516) bank41091 +41092 POINT(29.35192584101664 -95.8783662496266) bank41092 +41093 POINT(30.129000946019787 -94.42397370089071) bank41093 +41094 POINT(30.634521256013716 -94.64398205930478) bank41094 +41095 POINT(30.365914668001675 -95.97833721760311) bank41095 +41096 POINT(29.134564955642354 -95.78387827601263) bank41096 +41097 POINT(29.340607446769408 -96.18683579067816) bank41097 +41098 POINT(29.514792848732668 -95.54537999689809) bank41098 +41099 POINT(30.575825574728537 -95.48231056157645) bank41099 +41100 POINT(28.959022170792625 -94.49241490017464) bank41100 +41101 POINT(29.973007881776912 -95.69416726987151) bank41101 +41102 POINT(29.241268170500412 -95.9013763700443) bank41102 +41103 POINT(29.80255415679203 -94.46748018336977) bank41103 +41104 POINT(30.128031079893976 -95.67379155513365) bank41104 +41105 POINT(29.111923280689012 -94.42639493900784) bank41105 +41106 POINT(30.72448663676391 -96.04528519970083) bank41106 +41107 POINT(30.114474376622482 -95.12149531842303) bank41107 +41108 POINT(30.127604404194322 -95.82683224252457) bank41108 +41109 POINT(29.271290195959637 -94.7113693466082) bank41109 +41110 POINT(29.31337041684989 -95.93977655474406) bank41110 +41111 POINT(30.285688669046905 -96.34165931348247) bank41111 +41112 POINT(29.733732378037615 -94.8314227715702) bank41112 +41113 POINT(29.240758115766432 -95.96958882546596) bank41113 +41114 POINT(29.665838086583964 -94.50956569526721) bank41114 +41115 POINT(29.532698614229492 -96.136823277941) bank41115 +41116 POINT(29.532496934368638 -95.96885491530111) bank41116 +41117 POINT(30.745570102793312 -94.94162070959152) bank41117 +41118 POINT(29.689064819606674 -94.64854390904695) bank41118 +41119 POINT(28.882264763365836 -94.42786521734773) bank41119 +41120 POINT(29.20828532373282 -96.29682655599946) bank41120 +41121 POINT(28.787224883847557 -94.53787174078425) bank41121 +41122 POINT(29.890959105669932 -94.88695603243342) bank41122 +41123 POINT(30.25862604833033 -96.34780059891756) bank41123 +41124 POINT(29.427454436335147 -95.94734218044303) bank41124 +41125 POINT(30.395841525851672 -95.54370152037875) bank41125 +41126 POINT(29.881129028860833 -94.84291221872893) bank41126 +41127 POINT(29.53339039838113 -95.8762566588859) bank41127 +41128 POINT(29.29212468652316 -95.12077297551075) bank41128 +41129 POINT(28.873214803129766 -95.08825404274047) bank41129 +41130 POINT(30.079508449689634 -95.85000823354679) bank41130 +41131 POINT(29.274213737775415 -94.42901356068141) bank41131 +41132 POINT(29.760550965842718 -96.09607137444215) bank41132 +41133 POINT(29.969704898993967 -95.46533759705747) bank41133 +41134 POINT(28.914470342426604 -95.97257201285026) bank41134 +41135 POINT(29.39564344773189 -94.89925344784034) bank41135 +41136 POINT(30.56077354584339 -96.31136525796458) bank41136 +41137 POINT(29.254375191047142 -95.77449491716126) bank41137 +41138 POINT(29.465807877087315 -96.23650677468994) bank41138 +41139 POINT(29.14929697653227 -95.31994494541124) bank41139 +41140 POINT(30.437269974108595 -94.97639246939065) bank41140 +41141 POINT(29.391696327156076 -94.58111290502926) bank41141 +41142 POINT(30.03004885590856 -95.81228419987599) bank41142 +41143 POINT(29.412642355465525 -95.2811453002036) bank41143 +41144 POINT(30.60719358814391 -94.83224245316862) bank41144 +41145 POINT(28.967419676265177 -96.07326433068677) bank41145 +41146 POINT(30.431938735264083 -94.53047182223652) bank41146 +41147 POINT(29.237314761572897 -95.42919982788493) bank41147 +41148 POINT(30.07662818473881 -96.08849444019) bank41148 +41149 POINT(29.62909081482787 -96.0282826833296) bank41149 +41150 POINT(29.42135449409989 -95.94832247398718) bank41150 +41151 POINT(30.74913021325741 -95.81284032233519) bank41151 +41152 POINT(28.89660547089937 -94.68882160756468) bank41152 +41153 POINT(29.838757466534712 -96.18730006812626) bank41153 +41154 POINT(29.377796778540304 -94.60090612057517) bank41154 +41155 POINT(29.90116171464138 -95.11985264466442) bank41155 +41156 POINT(29.70292821888547 -94.38892461920508) bank41156 +41157 POINT(30.442987180799985 -96.30443433898499) bank41157 +41158 POINT(29.728548854827427 -95.46781240009904) bank41158 +41159 POINT(29.653299680193264 -94.64499426607198) bank41159 +41160 POINT(29.555586542660766 -96.15442122830926) bank41160 +41161 POINT(30.507988441902803 -95.96370801403583) bank41161 +41162 POINT(30.21940329301403 -95.1582580901038) bank41162 +41163 POINT(30.172420897793973 -95.3384951177966) bank41163 +41164 POINT(30.549396383603273 -95.03898345393061) bank41164 +41165 POINT(30.026460761568607 -95.94581865922851) bank41165 +41166 POINT(29.42823732495759 -96.07098509234572) bank41166 +41167 POINT(28.98126895638477 -94.98679857214404) bank41167 +41168 POINT(30.066369144425966 -94.7888170022272) bank41168 +41169 POINT(30.622515561347697 -94.45372185337821) bank41169 +41170 POINT(28.868376846020325 -95.70813225410612) bank41170 +41171 POINT(30.465236615253417 -95.9094211247117) bank41171 +41172 POINT(30.63838790065567 -95.1638260060345) bank41172 +41173 POINT(29.589965323671407 -95.85802199398964) bank41173 +41174 POINT(28.790332072452625 -95.11880986357359) bank41174 +41175 POINT(29.099400591559117 -94.65330965978947) bank41175 +41176 POINT(30.170411975160057 -94.66211527120558) bank41176 +41177 POINT(29.97571676632862 -94.70225207146716) bank41177 +41178 POINT(29.59894224348848 -96.01044120864115) bank41178 +41179 POINT(29.065162582570075 -96.34697517212238) bank41179 +41180 POINT(29.281707152908343 -95.75668936737598) bank41180 +41181 POINT(30.614662120973314 -95.70401984592729) bank41181 +41182 POINT(29.139976436362296 -95.75024584031324) bank41182 +41183 POINT(29.39157292451037 -94.63251560323589) bank41183 +41184 POINT(30.158565178119154 -96.27869933476366) bank41184 +41185 POINT(29.115016545784215 -96.28288930132526) bank41185 +41186 POINT(29.57139715748459 -94.90563092382347) bank41186 +41187 POINT(28.772622477704925 -94.5598672118323) bank41187 +41188 POINT(29.99716049273798 -94.62174429220755) bank41188 +41189 POINT(29.944777825652814 -96.05871447624226) bank41189 +41190 POINT(28.902540019133227 -96.27020300003215) bank41190 +41191 POINT(29.716977147237927 -94.6454112298446) bank41191 +41192 POINT(30.384603215276734 -95.14924823083335) bank41192 +41193 POINT(28.91973153851657 -95.68033694424857) bank41193 +41194 POINT(30.72246548440483 -95.38766642773663) bank41194 +41195 POINT(30.48859932929095 -96.05386627258223) bank41195 +41196 POINT(29.13878365197468 -96.00883023317557) bank41196 +41197 POINT(29.171859629906773 -95.81125249574778) bank41197 +41198 POINT(29.392369721258795 -95.18025533857822) bank41198 +41199 POINT(29.532314284294063 -94.76127908267998) bank41199 +41200 POINT(29.747510481268243 -94.71010551760243) bank41200 +41201 POINT(30.317176285887903 -95.58055874733505) bank41201 +41202 POINT(30.42890116001485 -95.55405773829918) bank41202 +41203 POINT(28.80994889975297 -96.21787149421189) bank41203 +41204 POINT(29.670107368817447 -94.595244155788) bank41204 +41205 POINT(29.1135080804715 -96.02698569361284) bank41205 +41206 POINT(30.445808625239696 -95.64974978679547) bank41206 +41207 POINT(29.176269232510936 -95.94403942866735) bank41207 +41208 POINT(30.43255475510373 -95.6400811674432) bank41208 +41209 POINT(30.11267167109821 -94.54843326181226) bank41209 +41210 POINT(28.978244430762974 -96.14296768666004) bank41210 +41211 POINT(30.711131863610564 -95.34684301828595) bank41211 +41212 POINT(30.731949978054224 -95.07813378231857) bank41212 +41213 POINT(30.411694094458745 -94.42606120377734) bank41213 +41214 POINT(29.042159748281463 -94.498117085723) bank41214 +41215 POINT(29.13334718523121 -94.59297819054622) bank41215 +41216 POINT(28.99128468446159 -94.87913647583846) bank41216 +41217 POINT(29.58041431199185 -95.03796709320675) bank41217 +41218 POINT(30.647632691895083 -94.47060785105) bank41218 +41219 POINT(30.678699351949234 -95.85359380649776) bank41219 +41220 POINT(30.05371308917037 -95.07406866371896) bank41220 +41221 POINT(30.15651358850514 -95.30636964794014) bank41221 +41222 POINT(29.33932082334708 -94.96844891403833) bank41222 +41223 POINT(29.488035857902695 -95.04438892563327) bank41223 +41224 POINT(30.50421177460727 -94.7517064146347) bank41224 +41225 POINT(29.443778193879883 -94.52341356980435) bank41225 +41226 POINT(30.500536811935586 -96.18738042084536) bank41226 +41227 POINT(30.006329489820516 -96.09032392667436) bank41227 +41228 POINT(30.065499469332195 -94.87965633209745) bank41228 +41229 POINT(29.11177412740082 -95.63528150537378) bank41229 +41230 POINT(29.397578836504525 -95.6749011087315) bank41230 +41231 POINT(30.219018532658747 -95.68086514093186) bank41231 +41232 POINT(29.70719820187822 -95.99466993224671) bank41232 +41233 POINT(30.57482574863156 -96.17265985497691) bank41233 +41234 POINT(29.66116671536177 -96.06632764143707) bank41234 +41235 POINT(28.88038901421511 -95.9387671836366) bank41235 +41236 POINT(30.61381803408866 -94.6566839610588) bank41236 +41237 POINT(30.03198957331707 -95.2424001425659) bank41237 +41238 POINT(30.0943595871816 -94.40274801534679) bank41238 +41239 POINT(30.11403267947831 -95.97408770607694) bank41239 +41240 POINT(29.829296439902862 -95.27761822010561) bank41240 +41241 POINT(29.331991272452367 -95.85696509367729) bank41241 +41242 POINT(29.768003170902276 -95.5315261725541) bank41242 +41243 POINT(29.737380699869643 -95.28796729810557) bank41243 +41244 POINT(28.821065954233624 -96.00548638971726) bank41244 +41245 POINT(29.618463889068806 -95.23553211042442) bank41245 +41246 POINT(30.22061060969177 -96.05375216365567) bank41246 +41247 POINT(30.593445737305455 -95.211479998218) bank41247 +41248 POINT(30.401414335002933 -95.7507791190298) bank41248 +41249 POINT(29.018290093596047 -95.8725169083864) bank41249 +41250 POINT(30.12938752724412 -94.90259553564694) bank41250 +41251 POINT(29.560794783654956 -96.32555607269573) bank41251 +41252 POINT(30.722179846103188 -94.85063690901535) bank41252 +41253 POINT(29.07615448062902 -94.94664894068302) bank41253 +41254 POINT(29.036921749451636 -94.85540065024412) bank41254 +41255 POINT(29.772928950409394 -96.05437502284614) bank41255 +41256 POINT(30.57092447748669 -95.41537353132298) bank41256 +41257 POINT(29.15988332941093 -94.53275460470647) bank41257 +41258 POINT(29.007228907932813 -95.85430610998604) bank41258 +41259 POINT(29.732101547240347 -95.14160339383793) bank41259 +41260 POINT(29.59902228486805 -94.44325131808503) bank41260 +41261 POINT(29.62927638314867 -96.02823279311666) bank41261 +41262 POINT(29.938129432724832 -96.27257847235227) bank41262 +41263 POINT(29.42671213182765 -95.26260824468771) bank41263 +41264 POINT(30.15175044047964 -95.96384194304711) bank41264 +41265 POINT(30.336666676151697 -95.95571519453519) bank41265 +41266 POINT(29.269076797636615 -94.54723414360866) bank41266 +41267 POINT(29.21992124293887 -94.47799070668056) bank41267 +41268 POINT(29.661364760281657 -94.82748334272378) bank41268 +41269 POINT(29.547647375500468 -96.08180167200763) bank41269 +41270 POINT(30.14883585249147 -96.00731977932908) bank41270 +41271 POINT(29.22899510059492 -94.85184696241954) bank41271 +41272 POINT(30.446663269261272 -94.90814333871637) bank41272 +41273 POINT(30.414043831140916 -95.50789936926104) bank41273 +41274 POINT(30.35514046641755 -95.34645658335329) bank41274 +41275 POINT(29.371581972619175 -96.04461466639515) bank41275 +41276 POINT(29.35268509485747 -94.83235471329515) bank41276 +41277 POINT(29.707231310643852 -94.79477779258893) bank41277 +41278 POINT(29.986806185473664 -95.79870635501295) bank41278 +41279 POINT(29.093052738314665 -95.62196808969989) bank41279 +41280 POINT(29.273367860240754 -95.27605195673016) bank41280 +41281 POINT(29.423235851305193 -94.69543912880559) bank41281 +41282 POINT(30.49446055807063 -94.37836585897057) bank41282 +41283 POINT(29.247805249475046 -96.16038058376978) bank41283 +41284 POINT(30.372987600107276 -94.73985049849261) bank41284 +41285 POINT(29.62648263315732 -95.7059053811936) bank41285 +41286 POINT(30.197767424563395 -96.12390583037035) bank41286 +41287 POINT(28.86335080553862 -95.3486135938128) bank41287 +41288 POINT(30.52571155652906 -94.76453826620525) bank41288 +41289 POINT(29.44008848789402 -95.59465887695785) bank41289 +41290 POINT(29.985183171932434 -95.69781759574599) bank41290 +41291 POINT(30.16981391418795 -96.23266821741474) bank41291 +41292 POINT(29.734562015281277 -95.50939363931276) bank41292 +41293 POINT(28.924121697638157 -95.7018336843183) bank41293 +41294 POINT(29.6729886772305 -95.22997014850046) bank41294 +41295 POINT(28.81254781989572 -95.26871058850422) bank41295 +41296 POINT(29.69118362799783 -95.35424547929351) bank41296 +41297 POINT(30.406697427810588 -94.63768678037088) bank41297 +41298 POINT(29.68868473373855 -96.21961387852143) bank41298 +41299 POINT(29.13172528573496 -96.35075152034958) bank41299 +41300 POINT(29.44390954854941 -95.8696831312646) bank41300 +41301 POINT(30.131344879258265 -95.03494805279045) bank41301 +41302 POINT(30.060425690371826 -96.367511402821) bank41302 +41303 POINT(28.7769718497093 -94.91425200986657) bank41303 +41304 POINT(29.085400631921384 -95.71319127888275) bank41304 +41305 POINT(30.003771910462575 -95.32868785464396) bank41305 +41306 POINT(29.226326819371007 -95.73178866947026) bank41306 +41307 POINT(29.15824281909238 -96.00862441384857) bank41307 +41308 POINT(30.082751147752333 -94.99330908826869) bank41308 +41309 POINT(29.659867295498337 -94.56130028491161) bank41309 +41310 POINT(30.21449184909024 -94.76751043974063) bank41310 +41311 POINT(29.207348983241417 -96.3115758795017) bank41311 +41312 POINT(28.974877855760898 -95.73581091744354) bank41312 +41313 POINT(30.13216544488565 -94.6199086038564) bank41313 +41314 POINT(28.7733483566792 -95.29439042309744) bank41314 +41315 POINT(29.618737587270303 -95.06192659800556) bank41315 +41316 POINT(30.036577156025523 -95.35617602625013) bank41316 +41317 POINT(30.378008122831744 -95.51784676066688) bank41317 +41318 POINT(29.31188059344804 -94.95754802606542) bank41318 +41319 POINT(30.649089659482758 -96.13325050147112) bank41319 +41320 POINT(28.97280912498577 -96.29043938065561) bank41320 +41321 POINT(30.37611657437712 -94.51198325380871) bank41321 +41322 POINT(29.7751173247834 -94.65656407020681) bank41322 +41323 POINT(29.841196590103518 -96.14268301889649) bank41323 +41324 POINT(28.919406636355745 -95.9361426753197) bank41324 +41325 POINT(28.957586440293056 -95.17813904425219) bank41325 +41326 POINT(29.908809742464314 -95.82888259205923) bank41326 +41327 POINT(29.128395452380555 -95.39107045324977) bank41327 +41328 POINT(29.922590641530444 -95.95921712517183) bank41328 +41329 POINT(28.835096698597514 -94.7819639806667) bank41329 +41330 POINT(30.023798763978714 -94.98091707831288) bank41330 +41331 POINT(30.094217756754386 -95.84548560826781) bank41331 +41332 POINT(30.636282901228448 -96.21607424844031) bank41332 +41333 POINT(30.518173491293414 -95.6855123190209) bank41333 +41334 POINT(29.432438283457 -95.38813623773062) bank41334 +41335 POINT(28.87279973411048 -95.04670462629957) bank41335 +41336 POINT(29.00962778682056 -94.78820465175723) bank41336 +41337 POINT(30.45474448241542 -95.35375177971711) bank41337 +41338 POINT(30.669235180008773 -94.3841315239653) bank41338 +41339 POINT(28.870660373783377 -94.47353074754272) bank41339 +41340 POINT(30.6949600748369 -95.26288570303083) bank41340 +41341 POINT(29.40780668508344 -95.75306442245275) bank41341 +41342 POINT(28.969700564869296 -95.8294610995678) bank41342 +41343 POINT(29.409312942592692 -95.52437309814516) bank41343 +41344 POINT(30.295240160525246 -96.09162860966252) bank41344 +41345 POINT(30.117180366286444 -95.09091427220915) bank41345 +41346 POINT(30.01386835724992 -95.56087239007098) bank41346 +41347 POINT(29.992935743613007 -96.14183077959164) bank41347 +41348 POINT(30.005923255422466 -95.54909936803675) bank41348 +41349 POINT(28.872946882420386 -95.37497168829212) bank41349 +41350 POINT(29.5736902900192 -96.23204374385529) bank41350 +41351 POINT(29.28687824109505 -95.75880416077231) bank41351 +41352 POINT(29.91330766992991 -94.63121700965362) bank41352 +41353 POINT(29.695416838002814 -96.34197212631794) bank41353 +41354 POINT(29.364261806028626 -95.71511092187578) bank41354 +41355 POINT(29.979327276383668 -95.44323400074474) bank41355 +41356 POINT(29.94460865321171 -95.48607878912273) bank41356 +41357 POINT(28.931866855127375 -96.0918243254044) bank41357 +41358 POINT(29.477747334952973 -94.98738794687821) bank41358 +41359 POINT(30.710538124738502 -95.14983849532682) bank41359 +41360 POINT(29.090216173951628 -95.41473345247736) bank41360 +41361 POINT(30.493057972340246 -96.18924726657455) bank41361 +41362 POINT(30.238265404409354 -94.97757937256945) bank41362 +41363 POINT(29.70960247304516 -96.36904151171932) bank41363 +41364 POINT(30.673512327811103 -94.99430504379288) bank41364 +41365 POINT(29.069251610296376 -95.58905837675391) bank41365 +41366 POINT(30.64836143220691 -95.76727173121859) bank41366 +41367 POINT(30.10361812589961 -94.4161807092434) bank41367 +41368 POINT(29.28212362064776 -95.54176326448402) bank41368 +41369 POINT(28.870607486047444 -94.4919809749456) bank41369 +41370 POINT(28.87813671943305 -95.96428419753649) bank41370 +41371 POINT(30.14418873741952 -96.0807313812266) bank41371 +41372 POINT(29.49107751787912 -95.2808694028289) bank41372 +41373 POINT(30.00496861465646 -95.35815296252672) bank41373 +41374 POINT(29.245665806260792 -95.0451734444805) bank41374 +41375 POINT(29.677024667949347 -95.78962328931168) bank41375 +41376 POINT(30.753842899840436 -95.53036393472044) bank41376 +41377 POINT(30.59872486485459 -95.05280507988616) bank41377 +41378 POINT(29.953722669114054 -95.1488445679561) bank41378 +41379 POINT(30.189111309165842 -95.11535270804849) bank41379 +41380 POINT(29.412741643534428 -96.07990256611835) bank41380 +41381 POINT(29.164405097549277 -96.01346480452003) bank41381 +41382 POINT(30.25586283924297 -95.13635625589748) bank41382 +41383 POINT(29.57835875244175 -94.42741448905055) bank41383 +41384 POINT(29.03882088370714 -96.18068891770135) bank41384 +41385 POINT(28.90502025698716 -94.79140788918833) bank41385 +41386 POINT(29.674549234408605 -95.25952767060762) bank41386 +41387 POINT(28.800914436569627 -95.9454014169213) bank41387 +41388 POINT(29.184394949631823 -95.06434342839562) bank41388 +41389 POINT(30.383944069039096 -94.80040669149881) bank41389 +41390 POINT(30.53683983053892 -94.67858190759749) bank41390 +41391 POINT(30.673948131096147 -94.38607429343142) bank41391 +41392 POINT(29.48663668276121 -95.54015916552441) bank41392 +41393 POINT(29.95834290877475 -94.94713730073643) bank41393 +41394 POINT(30.039625033826653 -96.14968749386476) bank41394 +41395 POINT(30.63070060709719 -95.29436076351948) bank41395 +41396 POINT(28.916927030135653 -95.18591827203937) bank41396 +41397 POINT(28.9689056420592 -94.92700435439555) bank41397 +41398 POINT(30.02714084565623 -94.39920239348577) bank41398 +41399 POINT(29.135256861200773 -94.44299365029936) bank41399 +41400 POINT(30.50341916171074 -95.30988678972608) bank41400 +41401 POINT(29.373058123924558 -95.96723148490456) bank41401 +41402 POINT(30.236946278300074 -96.01886838189769) bank41402 +41403 POINT(30.35490890956562 -96.25935355824276) bank41403 +41404 POINT(30.617950944593677 -94.74760892249911) bank41404 +41405 POINT(30.276683192578087 -95.27778469670416) bank41405 +41406 POINT(29.292501678623502 -96.0907791554146) bank41406 +41407 POINT(30.166886827210632 -95.90669330690088) bank41407 +41408 POINT(29.196071727491454 -96.1766377329964) bank41408 +41409 POINT(29.860586696731332 -95.05116785542498) bank41409 +41410 POINT(30.018313258533293 -94.89752035035082) bank41410 +41411 POINT(28.953112235658498 -95.62741209132933) bank41411 +41412 POINT(29.04274196248751 -95.47355244328263) bank41412 +41413 POINT(29.054069209775843 -95.77827939646056) bank41413 +41414 POINT(30.314604021906412 -95.58068481032316) bank41414 +41415 POINT(29.05001354674283 -94.53047273447004) bank41415 +41416 POINT(29.013332763089377 -95.81588354750377) bank41416 +41417 POINT(29.608867889772114 -94.41424285615292) bank41417 +41418 POINT(29.59965277581293 -95.37780515778323) bank41418 +41419 POINT(30.096954095123895 -94.37568374918176) bank41419 +41420 POINT(30.1058686081678 -96.14062281792496) bank41420 +41421 POINT(28.922793595031404 -94.72566003836434) bank41421 +41422 POINT(29.550513091895517 -95.03237525074454) bank41422 +41423 POINT(30.11810120602663 -95.36823687247764) bank41423 +41424 POINT(29.71155786236391 -96.25441177489543) bank41424 +41425 POINT(29.362471465769637 -95.31145382508062) bank41425 +41426 POINT(29.89676445429847 -95.78850115976564) bank41426 +41427 POINT(28.827767902845086 -96.34458279209905) bank41427 +41428 POINT(29.65223541100332 -94.53244686162158) bank41428 +41429 POINT(30.022981488435658 -95.8025927916955) bank41429 +41430 POINT(30.272017488678685 -96.36697724883236) bank41430 +41431 POINT(29.27937569422029 -95.18051453272888) bank41431 +41432 POINT(29.247364133749166 -94.45540347264934) bank41432 +41433 POINT(30.078554303156242 -95.32180984802997) bank41433 +41434 POINT(29.895482857101364 -95.36178494631818) bank41434 +41435 POINT(29.34086656830619 -95.27415436426976) bank41435 +41436 POINT(29.70641153633927 -95.46741065766167) bank41436 +41437 POINT(29.056115918355296 -96.33464845370305) bank41437 +41438 POINT(30.385884636118153 -95.6673032194609) bank41438 +41439 POINT(30.3605818650625 -96.35443137681315) bank41439 +41440 POINT(29.95398786553096 -96.3365573878757) bank41440 +41441 POINT(29.490455769232447 -94.72722604883151) bank41441 +41442 POINT(29.743586478813814 -95.03633509423629) bank41442 +41443 POINT(29.964617641349648 -96.02859647093028) bank41443 +41444 POINT(30.262850630036233 -96.16211706893444) bank41444 +41445 POINT(30.202471840327913 -94.42721069786596) bank41445 +41446 POINT(29.423389429301814 -96.2719400464403) bank41446 +41447 POINT(29.11451280049726 -94.94912951925875) bank41447 +41448 POINT(29.055055101068742 -94.75110721953784) bank41448 +41449 POINT(29.92664038016479 -96.14670527643484) bank41449 +41450 POINT(29.445030851443384 -95.2703649419648) bank41450 +41451 POINT(29.429445409700055 -95.27990259734949) bank41451 +41452 POINT(30.112669656187403 -95.44676275123717) bank41452 +41453 POINT(29.96325257105179 -95.5735126323685) bank41453 +41454 POINT(29.980412796269878 -95.40761764624028) bank41454 +41455 POINT(30.154161280291902 -96.06015633187377) bank41455 +41456 POINT(29.67185706638447 -96.27916307326224) bank41456 +41457 POINT(29.34874089283696 -95.86540147501752) bank41457 +41458 POINT(30.579588051206755 -95.28888134971302) bank41458 +41459 POINT(30.053762201428576 -94.8806435555821) bank41459 +41460 POINT(30.388299032689407 -94.74064971693105) bank41460 +41461 POINT(29.353752250415297 -94.69342749375203) bank41461 +41462 POINT(29.278057498245204 -94.47496387671515) bank41462 +41463 POINT(29.004351174874376 -95.07448828028153) bank41463 +41464 POINT(30.643650651389443 -94.59628024864863) bank41464 +41465 POINT(29.233660098051708 -94.95388798530917) bank41465 +41466 POINT(29.90707418754544 -95.71998930275164) bank41466 +41467 POINT(28.98463722762608 -96.2327032454032) bank41467 +41468 POINT(30.036329727013026 -95.12850990652886) bank41468 +41469 POINT(29.277787856076433 -95.21542223541867) bank41469 +41470 POINT(30.128140912328135 -94.52315164549653) bank41470 +41471 POINT(29.85911782569361 -95.86633833881842) bank41471 +41472 POINT(29.73914209582897 -95.74968557049976) bank41472 +41473 POINT(30.166065096692705 -95.54635030917525) bank41473 +41474 POINT(29.557439369986 -96.20846711100938) bank41474 +41475 POINT(30.118252047743834 -95.00270221492529) bank41475 +41476 POINT(29.765636620710893 -95.79940720745596) bank41476 +41477 POINT(30.573631520864218 -95.05161076392261) bank41477 +41478 POINT(29.48891560297415 -95.80192284579054) bank41478 +41479 POINT(29.203082358053088 -96.1536142185404) bank41479 +41480 POINT(28.972233163618153 -96.30791437696506) bank41480 +41481 POINT(29.49607450469591 -94.85597736895774) bank41481 +41482 POINT(30.509035450801207 -95.05084713671152) bank41482 +41483 POINT(29.742341858404313 -94.53845715375917) bank41483 +41484 POINT(28.783002211910954 -95.85385249235951) bank41484 +41485 POINT(30.0124811022962 -95.57794886440827) bank41485 +41486 POINT(30.257006075415074 -96.35976972625065) bank41486 +41487 POINT(29.128851136405327 -95.48230667119172) bank41487 +41488 POINT(29.277906561753046 -95.93917655840342) bank41488 +41489 POINT(30.70383822751159 -95.16689159817209) bank41489 +41490 POINT(30.418217205919998 -96.09532331309498) bank41490 +41491 POINT(29.719316790355926 -95.56598286354586) bank41491 +41492 POINT(30.37187312088693 -95.29261126700648) bank41492 +41493 POINT(30.642181395453942 -95.58748114844643) bank41493 +41494 POINT(30.218461459374094 -95.65451389014856) bank41494 +41495 POINT(28.882065346625286 -96.05882799773416) bank41495 +41496 POINT(30.473684357488974 -95.04041866601982) bank41496 +41497 POINT(28.783494793267035 -95.1518957310954) bank41497 +41498 POINT(28.953140151081524 -95.5233086220338) bank41498 +41499 POINT(30.565343714026167 -96.28428077577603) bank41499 +41500 POINT(30.536148021545973 -94.79080934322904) bank41500 +41501 POINT(29.27422811404472 -96.36960042252555) bank41501 +41502 POINT(29.369178998374426 -95.11421451919323) bank41502 +41503 POINT(30.30913096771529 -94.8248286442465) bank41503 +41504 POINT(29.196112051085674 -96.12274715423833) bank41504 +41505 POINT(30.076887685671366 -95.86925326260463) bank41505 +41506 POINT(28.77108897700819 -95.86472709372508) bank41506 +41507 POINT(28.95146413733731 -94.52555370904835) bank41507 +41508 POINT(29.351776138564507 -95.75370278945789) bank41508 +41509 POINT(30.344527990129595 -95.17074348384897) bank41509 +41510 POINT(30.13580564798791 -94.52584466649272) bank41510 +41511 POINT(29.68765031228145 -96.02303496193706) bank41511 +41512 POINT(30.02742074103457 -94.74046778855508) bank41512 +41513 POINT(30.26985745285715 -94.44278867569277) bank41513 +41514 POINT(30.59872204329986 -96.26797868786892) bank41514 +41515 POINT(30.277168303176595 -94.68336417893802) bank41515 +41516 POINT(29.534872333967307 -94.4577400497093) bank41516 +41517 POINT(29.704431787085202 -95.47743380962054) bank41517 +41518 POINT(30.12114859795731 -95.32772207612665) bank41518 +41519 POINT(30.487867556019385 -95.87634627562274) bank41519 +41520 POINT(29.211958873406523 -96.10627369625328) bank41520 +41521 POINT(30.33221599903688 -95.09923377518612) bank41521 +41522 POINT(29.646460918191497 -95.4233688483977) bank41522 +41523 POINT(28.89487685981196 -95.72754164325535) bank41523 +41524 POINT(29.575153242493617 -95.54933744429387) bank41524 +41525 POINT(28.871400904516175 -94.78559821292748) bank41525 +41526 POINT(29.98074693846355 -94.76851541789594) bank41526 +41527 POINT(29.503689203034472 -95.37728786993101) bank41527 +41528 POINT(28.910605523219253 -95.93683681771886) bank41528 +41529 POINT(29.57458891434989 -94.74208170778323) bank41529 +41530 POINT(29.302277898420666 -95.02759507764847) bank41530 +41531 POINT(30.18083749815008 -96.27824819250088) bank41531 +41532 POINT(29.443981726611558 -95.96001213167138) bank41532 +41533 POINT(30.72874993367294 -95.68308183376799) bank41533 +41534 POINT(30.54333819875377 -95.53297460022804) bank41534 +41535 POINT(29.62437090091378 -95.59435687535537) bank41535 +41536 POINT(30.505299387782546 -94.92621821705458) bank41536 +41537 POINT(30.315146883638146 -95.05470591992733) bank41537 +41538 POINT(30.359406680344467 -94.9719985549325) bank41538 +41539 POINT(30.29777836381727 -95.9229608288749) bank41539 +41540 POINT(28.96897573499692 -94.58673168835105) bank41540 +41541 POINT(30.21767106188045 -96.09395183248593) bank41541 +41542 POINT(29.73510195826594 -95.8343552613776) bank41542 +41543 POINT(29.22717836219197 -95.60705370863649) bank41543 +41544 POINT(30.48888037941184 -95.62551358868731) bank41544 +41545 POINT(29.46124520816771 -95.25800151569914) bank41545 +41546 POINT(29.69263100838136 -94.8835281975467) bank41546 +41547 POINT(30.061856365189005 -96.31038485974304) bank41547 +41548 POINT(30.467328703750002 -95.47591794423575) bank41548 +41549 POINT(29.716121561438992 -95.04712759298258) bank41549 +41550 POINT(28.955884844035157 -95.57931126851322) bank41550 +41551 POINT(30.63344255412794 -95.29399877088247) bank41551 +41552 POINT(30.336883350803234 -94.83144122856025) bank41552 +41553 POINT(30.031590469985566 -94.81648077327094) bank41553 +41554 POINT(29.448134169054992 -94.60036953617397) bank41554 +41555 POINT(30.183177645892687 -95.12682629381028) bank41555 +41556 POINT(30.243616438449354 -96.1847667492078) bank41556 +41557 POINT(29.934732589837022 -94.86841619261936) bank41557 +41558 POINT(30.121811758569628 -95.6515940143563) bank41558 +41559 POINT(30.091942460797355 -96.12749921564608) bank41559 +41560 POINT(29.453964755263556 -94.39790306695663) bank41560 +41561 POINT(28.895273083098694 -95.16989412805242) bank41561 +41562 POINT(29.54523561956104 -95.82615452794586) bank41562 +41563 POINT(29.2758184177833 -95.21746456851662) bank41563 +41564 POINT(29.08725394675727 -95.74328245916654) bank41564 +41565 POINT(28.92667652359855 -95.2953987541213) bank41565 +41566 POINT(29.724202427989503 -96.32650953058344) bank41566 +41567 POINT(29.09570693715466 -96.25475622688167) bank41567 +41568 POINT(29.752150070465134 -96.06423688214736) bank41568 +41569 POINT(29.90467476750562 -94.96060237943875) bank41569 +41570 POINT(30.67858751186215 -95.52122393486859) bank41570 +41571 POINT(28.98125658336148 -95.34838314610924) bank41571 +41572 POINT(30.00457459809147 -94.49822802539329) bank41572 +41573 POINT(30.51994164227431 -96.30749169403629) bank41573 +41574 POINT(28.866467793005636 -95.84927640846058) bank41574 +41575 POINT(29.597463335619985 -94.91632297641225) bank41575 +41576 POINT(29.53375513329437 -94.87176406874413) bank41576 +41577 POINT(30.21170715174539 -95.84025442222013) bank41577 +41578 POINT(30.109219022304682 -95.18255717199563) bank41578 +41579 POINT(29.533841237072302 -95.89216230563365) bank41579 +41580 POINT(28.938434147603346 -94.61492240469052) bank41580 +41581 POINT(30.06702150251601 -95.0696091420023) bank41581 +41582 POINT(28.840374382054897 -94.67167858973237) bank41582 +41583 POINT(29.37536223012172 -96.12998325892286) bank41583 +41584 POINT(28.88606963049173 -95.96663954046507) bank41584 +41585 POINT(29.596295296511194 -95.30111877155953) bank41585 +41586 POINT(29.74764657676964 -95.81503956618722) bank41586 +41587 POINT(30.221859653400514 -96.3646882760551) bank41587 +41588 POINT(28.93069141542253 -94.465214611481) bank41588 +41589 POINT(30.32546372830088 -94.50280563979314) bank41589 +41590 POINT(29.745861317194613 -95.21462895913989) bank41590 +41591 POINT(30.190281869916483 -95.22005042666136) bank41591 +41592 POINT(29.872670888915447 -95.97162961203857) bank41592 +41593 POINT(29.891872070361917 -95.69378371667881) bank41593 +41594 POINT(29.70414407863951 -96.15014691091001) bank41594 +41595 POINT(30.44850861575085 -95.96646530214153) bank41595 +41596 POINT(30.067090316487572 -94.51817472372589) bank41596 +41597 POINT(30.333710231805757 -95.65845522922645) bank41597 +41598 POINT(29.897514126034594 -96.28909251255402) bank41598 +41599 POINT(29.29829139716311 -95.8633354253387) bank41599 +41600 POINT(30.20640212542719 -94.43116822390425) bank41600 +41601 POINT(30.45948773546633 -95.63044080556158) bank41601 +41602 POINT(28.785602829018654 -94.95148336664143) bank41602 +41603 POINT(30.38111284889529 -96.1668587045559) bank41603 +41604 POINT(29.827121400011748 -95.95270218343268) bank41604 +41605 POINT(29.055683024179977 -95.60392003790754) bank41605 +41606 POINT(30.740186230205186 -95.25048132447941) bank41606 +41607 POINT(30.291145064692767 -94.91807265940663) bank41607 +41608 POINT(29.956832426002258 -94.96510978977074) bank41608 +41609 POINT(29.35373067291391 -95.2624947414611) bank41609 +41610 POINT(28.90652143672769 -95.45437488909052) bank41610 +41611 POINT(28.777831594870747 -95.56218658045935) bank41611 +41612 POINT(29.15441704338943 -94.98228179299922) bank41612 +41613 POINT(29.67805826285845 -96.06128240782931) bank41613 +41614 POINT(30.43747035580799 -95.84323830863218) bank41614 +41615 POINT(30.161621174456513 -95.96614891681673) bank41615 +41616 POINT(29.520054499134247 -95.74385139168851) bank41616 +41617 POINT(29.26269319196036 -94.4529417782894) bank41617 +41618 POINT(29.659864239061225 -96.03496147344165) bank41618 +41619 POINT(29.845996743928147 -94.43045174915913) bank41619 +41620 POINT(30.401978130439616 -95.24389190736586) bank41620 +41621 POINT(29.7360961136112 -95.94536772714923) bank41621 +41622 POINT(29.681768528351594 -95.45468636841287) bank41622 +41623 POINT(29.896252236227784 -94.66679584815073) bank41623 +41624 POINT(29.621023450062378 -94.37582585225323) bank41624 +41625 POINT(30.092590056026584 -95.35566447858221) bank41625 +41626 POINT(30.606934654345356 -96.05573552461327) bank41626 +41627 POINT(30.49786749644176 -95.05189617223972) bank41627 +41628 POINT(29.799328688324024 -95.09341691655925) bank41628 +41629 POINT(29.844225864289996 -95.87872486260282) bank41629 +41630 POINT(29.79570964827805 -96.09754886699082) bank41630 +41631 POINT(30.66761095959497 -96.28349644379567) bank41631 +41632 POINT(30.01345451313297 -96.2269628439077) bank41632 +41633 POINT(30.30840856268077 -94.5919654862022) bank41633 +41634 POINT(29.01405978934834 -95.71341910489748) bank41634 +41635 POINT(29.213351469941575 -96.2006274268415) bank41635 +41636 POINT(29.859681952208355 -94.72377454851122) bank41636 +41637 POINT(30.618164097264923 -95.23995023355234) bank41637 +41638 POINT(29.270664536336263 -95.5614426666329) bank41638 +41639 POINT(28.765754148514922 -95.06706003186432) bank41639 +41640 POINT(30.568821973340814 -96.01209028089346) bank41640 +41641 POINT(30.468691243342754 -95.08553525122858) bank41641 +41642 POINT(30.432774429125946 -94.4395384818795) bank41642 +41643 POINT(28.813922607377844 -95.94857666203886) bank41643 +41644 POINT(28.95874010985122 -94.9866583838118) bank41644 +41645 POINT(29.3519771152692 -95.39284579638078) bank41645 +41646 POINT(29.640687128749427 -95.31190204626685) bank41646 +41647 POINT(30.11139295873979 -94.80592282164764) bank41647 +41648 POINT(29.80353930629597 -96.10370922085788) bank41648 +41649 POINT(30.04307304387989 -95.4638777955366) bank41649 +41650 POINT(29.470101122761253 -94.46228593545989) bank41650 +41651 POINT(28.92399924134738 -94.48872728419582) bank41651 +41652 POINT(30.343163583702015 -95.25077243271755) bank41652 +41653 POINT(29.766469207247123 -95.79760235817311) bank41653 +41654 POINT(30.549004668899116 -95.72155315464317) bank41654 +41655 POINT(30.16449510936907 -95.14222742062901) bank41655 +41656 POINT(29.763141446402873 -95.94645934535343) bank41656 +41657 POINT(29.157427771895733 -95.34169551718082) bank41657 +41658 POINT(30.10961430341731 -95.72396621437953) bank41658 +41659 POINT(30.312666500819187 -96.16634547698284) bank41659 +41660 POINT(30.11689333323029 -96.14889646564887) bank41660 +41661 POINT(29.582414130885823 -95.06034871276476) bank41661 +41662 POINT(29.703577571952806 -94.63762618748753) bank41662 +41663 POINT(29.581780307711483 -95.74771732179413) bank41663 +41664 POINT(30.47612083102246 -94.96764030776222) bank41664 +41665 POINT(29.601060143431056 -95.62838305136738) bank41665 +41666 POINT(29.38029493188339 -95.98529847549352) bank41666 +41667 POINT(29.93253002221934 -94.94728173254956) bank41667 +41668 POINT(30.055789980131326 -94.80110800396331) bank41668 +41669 POINT(30.592545564963352 -94.87391180129707) bank41669 +41670 POINT(29.781791238199308 -96.08166142556558) bank41670 +41671 POINT(30.13405702314085 -96.34698636484998) bank41671 +41672 POINT(29.108972603062437 -95.21370693373929) bank41672 +41673 POINT(28.96967205714826 -96.2115175862096) bank41673 +41674 POINT(29.385291285354164 -94.9522222667509) bank41674 +41675 POINT(30.240462601425186 -94.4868154617631) bank41675 +41676 POINT(29.088445536692085 -95.53897478531438) bank41676 +41677 POINT(30.65040664630758 -95.01030772331164) bank41677 +41678 POINT(29.343640471291888 -96.15475109470495) bank41678 +41679 POINT(29.058689969729713 -96.35370814939371) bank41679 +41680 POINT(29.78163990059058 -94.59441165603921) bank41680 +41681 POINT(29.511056521797144 -96.14649972741404) bank41681 +41682 POINT(30.517370618865368 -94.64448404158705) bank41682 +41683 POINT(29.532697367694862 -95.84031461432718) bank41683 +41684 POINT(29.6125156795759 -94.99121568722892) bank41684 +41685 POINT(30.229429262309928 -96.14827592767963) bank41685 +41686 POINT(29.49155094200718 -95.34989360422396) bank41686 +41687 POINT(30.4891153433006 -94.82701406613837) bank41687 +41688 POINT(29.926035157650098 -94.89422047753524) bank41688 +41689 POINT(30.537942781477167 -96.0948064569911) bank41689 +41690 POINT(30.43825487012558 -95.13452840056668) bank41690 +41691 POINT(30.01997139332497 -95.28667133146327) bank41691 +41692 POINT(29.677465183816146 -94.9930385966339) bank41692 +41693 POINT(29.883152602415525 -94.7023122289092) bank41693 +41694 POINT(29.1418231674527 -96.00886686008806) bank41694 +41695 POINT(30.465581838584527 -94.84467235221163) bank41695 +41696 POINT(30.26079644771733 -96.25001887883722) bank41696 +41697 POINT(29.561295531729517 -94.46250081296874) bank41697 +41698 POINT(29.49979613666104 -95.523580034088) bank41698 +41699 POINT(30.41022364024072 -95.94216216337192) bank41699 +41700 POINT(30.74815081699788 -94.85194374284178) bank41700 +41701 POINT(29.596191535017578 -96.31045026214252) bank41701 +41702 POINT(30.060571313416776 -95.79442960947665) bank41702 +41703 POINT(29.48645099787464 -94.96182709408859) bank41703 +41704 POINT(30.667039926311922 -95.02745328016013) bank41704 +41705 POINT(29.163421883289782 -95.84965151975351) bank41705 +41706 POINT(29.57326188379762 -95.76966747929703) bank41706 +41707 POINT(29.178837774770535 -94.54131014567099) bank41707 +41708 POINT(29.826744602484375 -96.36874477385783) bank41708 +41709 POINT(29.411374239251227 -95.97136436775605) bank41709 +41710 POINT(29.81024060163511 -95.53615617222079) bank41710 +41711 POINT(29.874544286529115 -95.89655591356649) bank41711 +41712 POINT(28.784104835042207 -95.4075397821254) bank41712 +41713 POINT(29.2995419089939 -96.35450377269821) bank41713 +41714 POINT(28.963072617117547 -94.96503743970919) bank41714 +41715 POINT(30.278783650322474 -94.84240589410912) bank41715 +41716 POINT(30.39631020811327 -94.75195254750689) bank41716 +41717 POINT(30.352992743431457 -96.08004848158139) bank41717 +41718 POINT(29.864348883474086 -95.91536595808793) bank41718 +41719 POINT(29.759508961145983 -96.08065153929394) bank41719 +41720 POINT(29.033708639545893 -95.79994710355957) bank41720 +41721 POINT(29.05106365267404 -94.8786142476727) bank41721 +41722 POINT(29.803277348938632 -94.81689422456232) bank41722 +41723 POINT(28.950391553077644 -96.12511952349297) bank41723 +41724 POINT(29.557944505988893 -95.40993763499108) bank41724 +41725 POINT(29.754999179850216 -96.1099279411827) bank41725 +41726 POINT(28.881055385894577 -95.13940287718938) bank41726 +41727 POINT(30.657187828015193 -96.03740174386012) bank41727 +41728 POINT(30.24561557492555 -95.04983492504448) bank41728 +41729 POINT(30.459795917965863 -94.38643596327688) bank41729 +41730 POINT(29.220283372204758 -95.86499691637391) bank41730 +41731 POINT(29.855030422585116 -95.92417482908738) bank41731 +41732 POINT(29.29687165124339 -95.84011560733816) bank41732 +41733 POINT(29.575554854586645 -95.98092835203177) bank41733 +41734 POINT(29.791076753682276 -96.25291747413469) bank41734 +41735 POINT(30.67914407684901 -95.52379564868833) bank41735 +41736 POINT(29.466270752420307 -95.8966088394399) bank41736 +41737 POINT(28.924573073777093 -95.17045061536166) bank41737 +41738 POINT(28.93568948978539 -95.17812517277575) bank41738 +41739 POINT(29.794684846599015 -95.45701301181319) bank41739 +41740 POINT(30.63595586996493 -96.23000681427244) bank41740 +41741 POINT(29.467940678448056 -95.10141560988279) bank41741 +41742 POINT(29.267537445749976 -94.51676607621069) bank41742 +41743 POINT(30.666026803998456 -96.066575552873) bank41743 +41744 POINT(30.747635793563585 -95.79074368213799) bank41744 +41745 POINT(29.35054507616321 -95.85015686073196) bank41745 +41746 POINT(30.58724854450256 -94.87612403973019) bank41746 +41747 POINT(29.100371001175596 -95.24895149931639) bank41747 +41748 POINT(29.848464440228412 -95.69252233310864) bank41748 +41749 POINT(30.357702324086898 -95.06595377129898) bank41749 +41750 POINT(28.814874844828292 -95.63498403136948) bank41750 +41751 POINT(30.42022161993402 -96.26700712243937) bank41751 +41752 POINT(30.592635354906037 -94.96310390946186) bank41752 +41753 POINT(30.720347502297933 -94.7460880176905) bank41753 +41754 POINT(29.625261646333886 -96.14201672986644) bank41754 +41755 POINT(29.41917554799748 -94.55870379257594) bank41755 +41756 POINT(29.20814613533475 -96.16504489748124) bank41756 +41757 POINT(29.839842855256716 -94.77561544437573) bank41757 +41758 POINT(28.92446588753237 -95.87447134261947) bank41758 +41759 POINT(29.187414911432583 -95.13303437978395) bank41759 +41760 POINT(29.513577452183373 -94.6065855492635) bank41760 +41761 POINT(29.887161280161017 -94.74178236770005) bank41761 +41762 POINT(28.995325903179836 -94.54886197335841) bank41762 +41763 POINT(29.53256918352208 -95.35058707418764) bank41763 +41764 POINT(30.685761430409816 -95.66139223284614) bank41764 +41765 POINT(28.956901603874215 -95.5251156763422) bank41765 +41766 POINT(29.552304074697425 -95.27205198867885) bank41766 +41767 POINT(30.311103207423866 -95.76449201199924) bank41767 +41768 POINT(29.286468834489632 -95.41666120195855) bank41768 +41769 POINT(28.993985363535565 -96.28509875775228) bank41769 +41770 POINT(29.14500202050159 -96.06561939007936) bank41770 +41771 POINT(30.626551627924172 -94.50561964865558) bank41771 +41772 POINT(29.824834850626086 -96.30766989915017) bank41772 +41773 POINT(29.540295387942162 -95.73132371362591) bank41773 +41774 POINT(30.73686633593539 -95.44187544675562) bank41774 +41775 POINT(29.57361011853321 -94.74836570384633) bank41775 +41776 POINT(29.804022311282054 -95.32865703208007) bank41776 +41777 POINT(28.90821387676798 -96.2499166278656) bank41777 +41778 POINT(28.890084001261243 -96.36179221622865) bank41778 +41779 POINT(29.70485391441499 -95.73826791416025) bank41779 +41780 POINT(30.483323487250427 -94.6115772695372) bank41780 +41781 POINT(28.80395904510208 -95.23834539564403) bank41781 +41782 POINT(29.576703783777702 -95.18664703490143) bank41782 +41783 POINT(30.68922547260209 -94.39845752593922) bank41783 +41784 POINT(29.3162791098436 -94.96384376141671) bank41784 +41785 POINT(29.86142373443807 -94.86593274384245) bank41785 +41786 POINT(30.06087477213513 -94.78585064403573) bank41786 +41787 POINT(29.93806559228578 -95.47313054706683) bank41787 +41788 POINT(28.8463733415973 -94.78169459020899) bank41788 +41789 POINT(30.050133704069413 -94.83429612270886) bank41789 +41790 POINT(28.96217857700156 -94.69440003551134) bank41790 +41791 POINT(29.7037827771413 -96.21462516579277) bank41791 +41792 POINT(29.14272620120124 -95.93320252697809) bank41792 +41793 POINT(30.10488552142343 -95.84470468452682) bank41793 +41794 POINT(30.02030770067055 -95.21897012142097) bank41794 +41795 POINT(30.435051391031493 -94.74468313559254) bank41795 +41796 POINT(29.000524081617062 -95.90305058346418) bank41796 +41797 POINT(29.65347143385532 -95.83589360968675) bank41797 +41798 POINT(30.406444438649867 -96.02637777267854) bank41798 +41799 POINT(29.90001268490913 -96.00971051896246) bank41799 +41800 POINT(30.670286010575666 -95.40403200503157) bank41800 +41801 POINT(29.949875841216063 -95.37830958675458) bank41801 +41802 POINT(30.363947672887964 -95.1480676314412) bank41802 +41803 POINT(29.43935904862997 -94.57643140133428) bank41803 +41804 POINT(29.96645185986621 -95.41971873214527) bank41804 +41805 POINT(28.952358569454624 -95.71987191194432) bank41805 +41806 POINT(30.409759496718287 -96.35574379327271) bank41806 +41807 POINT(29.109009308604808 -94.65748390561357) bank41807 +41808 POINT(28.775715855183545 -96.02011685209895) bank41808 +41809 POINT(30.248160808015353 -94.85676060818865) bank41809 +41810 POINT(28.972598471592917 -95.26785517457535) bank41810 +41811 POINT(29.09813278709407 -95.51834672207777) bank41811 +41812 POINT(30.665925903120993 -96.02318610890138) bank41812 +41813 POINT(29.703155147508777 -95.86922958789681) bank41813 +41814 POINT(28.98806047781774 -95.81955144187302) bank41814 +41815 POINT(29.025762757644117 -95.93537750893083) bank41815 +41816 POINT(29.392934176217647 -96.30981654900128) bank41816 +41817 POINT(29.66157570328268 -95.6475279808923) bank41817 +41818 POINT(28.98164263003043 -95.74551556961006) bank41818 +41819 POINT(30.056644116738216 -95.44505328958508) bank41819 +41820 POINT(29.74748950748762 -94.71140515689484) bank41820 +41821 POINT(30.53069563872727 -95.19607376428253) bank41821 +41822 POINT(29.305029845447436 -94.70561789478447) bank41822 +41823 POINT(30.69223106426928 -95.22753210364901) bank41823 +41824 POINT(29.840440926811034 -95.4131165728791) bank41824 +41825 POINT(30.02567791289274 -95.69832829339126) bank41825 +41826 POINT(30.058752585638793 -95.53393615909843) bank41826 +41827 POINT(29.288635125556794 -94.7432978921926) bank41827 +41828 POINT(29.479354860834317 -94.73166370699477) bank41828 +41829 POINT(29.340838496936573 -96.27684349738139) bank41829 +41830 POINT(30.107935944907524 -96.08152266037142) bank41830 +41831 POINT(28.91863551795537 -96.28413803605044) bank41831 +41832 POINT(28.82757657293703 -95.25533721120443) bank41832 +41833 POINT(30.20886430821959 -95.526535674738) bank41833 +41834 POINT(29.713824855463688 -94.50342167130579) bank41834 +41835 POINT(30.105884447136155 -94.83344715919515) bank41835 +41836 POINT(29.003780685175474 -95.95084720348008) bank41836 +41837 POINT(28.796822628842797 -95.5133094706094) bank41837 +41838 POINT(29.05142162042704 -95.97254478904044) bank41838 +41839 POINT(28.780544821797882 -94.57358458610828) bank41839 +41840 POINT(29.495784511861135 -96.18643497037738) bank41840 +41841 POINT(30.458021494348973 -96.33128931888133) bank41841 +41842 POINT(29.26983523883972 -95.15114361170016) bank41842 +41843 POINT(30.386466374494574 -95.64074758621712) bank41843 +41844 POINT(29.472813422343037 -95.9843445686286) bank41844 +41845 POINT(30.393846123160692 -95.51523835666819) bank41845 +41846 POINT(29.66459759148563 -96.14391637697672) bank41846 +41847 POINT(28.954230897469728 -94.83330660676155) bank41847 +41848 POINT(30.25892398814112 -95.93552139299345) bank41848 +41849 POINT(30.477882463287617 -94.72423765766409) bank41849 +41850 POINT(29.259752498964012 -94.41697945514129) bank41850 +41851 POINT(28.99475940590896 -95.37765737248563) bank41851 +41852 POINT(30.335152576336043 -96.33464709184297) bank41852 +41853 POINT(29.867651771864036 -95.00001886176581) bank41853 +41854 POINT(30.218972405052636 -94.38884806061691) bank41854 +41855 POINT(29.954461948812202 -94.69843159565043) bank41855 +41856 POINT(30.608638872790195 -95.2451273564769) bank41856 +41857 POINT(30.190016784424245 -96.21206045703288) bank41857 +41858 POINT(30.039594590352266 -95.09216935785841) bank41858 +41859 POINT(29.710454132703617 -94.49075383503013) bank41859 +41860 POINT(28.93311795080482 -95.21219080162186) bank41860 +41861 POINT(29.35541674474789 -95.15860947001481) bank41861 +41862 POINT(28.873350980651228 -95.20795632006684) bank41862 +41863 POINT(29.09036976226682 -95.08769168987021) bank41863 +41864 POINT(30.498784281836922 -94.75988105679546) bank41864 +41865 POINT(28.764133655738554 -94.54439532550225) bank41865 +41866 POINT(29.915496942572915 -96.16661882253786) bank41866 +41867 POINT(30.616729474380083 -95.37460314639269) bank41867 +41868 POINT(28.795290709903846 -94.70676375795387) bank41868 +41869 POINT(29.857433459440664 -94.75009027349878) bank41869 +41870 POINT(29.036916437591017 -94.79195627725032) bank41870 +41871 POINT(30.571804693227442 -96.0729211377205) bank41871 +41872 POINT(28.788007465940858 -96.15614292347487) bank41872 +41873 POINT(29.707163381577036 -95.84447621051052) bank41873 +41874 POINT(30.22864183569271 -94.538349162079) bank41874 +41875 POINT(29.459555196986514 -94.51309769811303) bank41875 +41876 POINT(29.63408554941384 -95.92046567376845) bank41876 +41877 POINT(28.82700818695476 -94.80482484599747) bank41877 +41878 POINT(28.990258080243603 -95.6463046320395) bank41878 +41879 POINT(29.724935291226494 -96.35861773141227) bank41879 +41880 POINT(30.00047246423884 -94.71109765546409) bank41880 +41881 POINT(30.581976047060508 -95.4839586625324) bank41881 +41882 POINT(29.764353657635706 -95.4659341373238) bank41882 +41883 POINT(28.960247452221026 -95.7097542263883) bank41883 +41884 POINT(29.19987929402093 -96.20214121460299) bank41884 +41885 POINT(30.720133219434356 -95.55481496525596) bank41885 +41886 POINT(30.748932227798306 -94.83825138294367) bank41886 +41887 POINT(29.899159089954743 -95.91843236287112) bank41887 +41888 POINT(29.28949350374117 -95.8071148323224) bank41888 +41889 POINT(28.920480640740838 -94.45155892949644) bank41889 +41890 POINT(29.298528602964012 -94.93375779143001) bank41890 +41891 POINT(28.970461507622833 -95.68370131274945) bank41891 +41892 POINT(29.216783707240573 -94.54173878886084) bank41892 +41893 POINT(29.68259441479197 -95.09568453939626) bank41893 +41894 POINT(30.34587904141091 -95.94244446506495) bank41894 +41895 POINT(28.890837635391623 -95.53777332309015) bank41895 +41896 POINT(30.470143674566703 -95.55205623314174) bank41896 +41897 POINT(29.890673677667454 -95.3171758069018) bank41897 +41898 POINT(30.230449485951706 -95.27524650617694) bank41898 +41899 POINT(29.765840931195665 -95.29804687867902) bank41899 +41900 POINT(30.280543857376706 -94.45779743696976) bank41900 +41901 POINT(28.785697293069404 -95.2122622536161) bank41901 +41902 POINT(29.48318001298957 -94.70853763099356) bank41902 +41903 POINT(29.31607143036644 -96.09300455803144) bank41903 +41904 POINT(29.281171712217557 -96.12921274536636) bank41904 +41905 POINT(29.81658521484755 -95.47296555233892) bank41905 +41906 POINT(29.427533237357096 -95.1774353192448) bank41906 +41907 POINT(29.717281920239117 -95.3692873065923) bank41907 +41908 POINT(30.119856441413027 -95.36883326735239) bank41908 +41909 POINT(30.14632635486861 -95.18498753928313) bank41909 +41910 POINT(29.499209225841057 -95.92210554613261) bank41910 +41911 POINT(30.083811769148543 -95.06957901199986) bank41911 +41912 POINT(30.390159380583885 -94.98709443962018) bank41912 +41913 POINT(30.633251958260917 -95.92708369008892) bank41913 +41914 POINT(29.907478547986788 -95.88434299698817) bank41914 +41915 POINT(30.55134395901043 -95.817098915216) bank41915 +41916 POINT(30.083228737563708 -95.66064604655472) bank41916 +41917 POINT(29.23152259946113 -95.61117284830279) bank41917 +41918 POINT(28.78982594285541 -95.81582739745213) bank41918 +41919 POINT(29.021244285305425 -95.97279724710086) bank41919 +41920 POINT(29.89664073893559 -95.68583256791668) bank41920 +41921 POINT(30.035421800135733 -94.73715601821364) bank41921 +41922 POINT(29.62237972613797 -95.43122902870358) bank41922 +41923 POINT(29.970496335116664 -94.37570580052349) bank41923 +41924 POINT(29.69228188636284 -94.71892012128308) bank41924 +41925 POINT(29.411668738811173 -95.0134758956592) bank41925 +41926 POINT(30.75160175243003 -96.31759455359126) bank41926 +41927 POINT(29.194232665780568 -95.12774024615868) bank41927 +41928 POINT(29.86064658286307 -95.74191674557387) bank41928 +41929 POINT(29.051660746767116 -94.59129120268366) bank41929 +41930 POINT(29.270774270811586 -95.45295842971116) bank41930 +41931 POINT(30.13150723332307 -94.72523466247351) bank41931 +41932 POINT(30.509515505389725 -95.66926092205746) bank41932 +41933 POINT(29.002485036887755 -95.40556028592825) bank41933 +41934 POINT(30.554287907482287 -95.49089996506008) bank41934 +41935 POINT(29.057244974479016 -94.6117760355882) bank41935 +41936 POINT(30.35053649103316 -95.57728061103177) bank41936 +41937 POINT(28.969626937513944 -96.32305735263786) bank41937 +41938 POINT(30.632666563477354 -95.65845761151174) bank41938 +41939 POINT(30.746689430914557 -95.65976496635433) bank41939 +41940 POINT(29.707686600161754 -94.48829449980866) bank41940 +41941 POINT(29.757337732708972 -96.00540625502128) bank41941 +41942 POINT(29.872024360232814 -95.95929728085714) bank41942 +41943 POINT(30.058075813353145 -94.81948151796918) bank41943 +41944 POINT(29.616565302243515 -95.696725588711) bank41944 +41945 POINT(29.305875399242844 -95.26394476677665) bank41945 +41946 POINT(29.880928250802384 -96.26749007571414) bank41946 +41947 POINT(29.426730445387378 -95.00336337328159) bank41947 +41948 POINT(29.088560496400202 -95.43835846943624) bank41948 +41949 POINT(30.419711143607454 -96.00414405986336) bank41949 +41950 POINT(29.615873728614684 -96.18308866507037) bank41950 +41951 POINT(30.185665326615936 -95.36138219507112) bank41951 +41952 POINT(29.037195682228724 -96.2464392183168) bank41952 +41953 POINT(30.707826696369644 -94.72269347779387) bank41953 +41954 POINT(29.348091977387103 -94.98642187708518) bank41954 +41955 POINT(30.261184304386337 -95.5789220404236) bank41955 +41956 POINT(30.08882267495737 -96.15429601760405) bank41956 +41957 POINT(29.256768407197356 -95.2009554370616) bank41957 +41958 POINT(29.838745821593932 -95.69662064630874) bank41958 +41959 POINT(30.57433967983237 -95.2550781718092) bank41959 +41960 POINT(30.662178982378737 -94.57459797474762) bank41960 +41961 POINT(30.092500402485 -96.29525317862205) bank41961 +41962 POINT(29.311185181318347 -95.84509752126122) bank41962 +41963 POINT(29.719751872965503 -95.18237510388576) bank41963 +41964 POINT(30.202457220648302 -94.95645507691935) bank41964 +41965 POINT(30.06021296908886 -96.11225813805214) bank41965 +41966 POINT(29.792168119866556 -95.70208238672512) bank41966 +41967 POINT(29.33130961358725 -96.22213752784369) bank41967 +41968 POINT(28.78591887319331 -94.77171012919999) bank41968 +41969 POINT(30.173649279938726 -96.16716395170603) bank41969 +41970 POINT(29.330646226129108 -95.2740734164448) bank41970 +41971 POINT(30.591577120195705 -94.75185623733515) bank41971 +41972 POINT(30.233796875727798 -95.5344200404491) bank41972 +41973 POINT(30.59366294235317 -96.0145827066946) bank41973 +41974 POINT(29.27061951404326 -96.26084783478102) bank41974 +41975 POINT(29.479271892580943 -95.14761697767445) bank41975 +41976 POINT(30.372312816191023 -96.13036472924594) bank41976 +41977 POINT(29.956103495441262 -94.4756624349881) bank41977 +41978 POINT(29.07812605956161 -95.5569176630592) bank41978 +41979 POINT(29.71501163723794 -94.82764254500123) bank41979 +41980 POINT(30.75760147919285 -95.30606359904746) bank41980 +41981 POINT(30.218162506339155 -94.56723042051631) bank41981 +41982 POINT(29.98199219701297 -95.98102731677089) bank41982 +41983 POINT(29.778186760191463 -95.89696302508318) bank41983 +41984 POINT(29.573428379588623 -94.6654209122409) bank41984 +41985 POINT(28.802422966552644 -95.04397611810236) bank41985 +41986 POINT(29.39409996322742 -95.21335659494314) bank41986 +41987 POINT(30.301515366277524 -96.10995986731858) bank41987 +41988 POINT(30.49048347391881 -95.57921855497769) bank41988 +41989 POINT(30.606995862477493 -94.94002984375408) bank41989 +41990 POINT(29.803934003255723 -95.83346867827389) bank41990 +41991 POINT(29.02333758653891 -95.81831317340644) bank41991 +41992 POINT(28.82881511283406 -95.35739782583009) bank41992 +41993 POINT(30.052830651373437 -95.67836016558063) bank41993 +41994 POINT(30.730608901281688 -94.49686659884907) bank41994 +41995 POINT(29.05247288670316 -95.46396439631587) bank41995 +41996 POINT(29.5031501739296 -95.23348666487173) bank41996 +41997 POINT(29.420320096081312 -95.87717410830564) bank41997 +41998 POINT(30.216764647522606 -95.2345693157162) bank41998 +41999 POINT(28.989562428626552 -94.81512541035522) bank41999 +42000 POINT(28.919346788507927 -96.31424977920423) bank42000 +42001 POINT(29.792654973375768 -96.20209086105712) bank42001 +42002 POINT(30.052106480120734 -94.8885575578438) bank42002 +42003 POINT(30.286036415234445 -96.35740474581404) bank42003 +42004 POINT(29.179578552411947 -95.42025031050228) bank42004 +42005 POINT(30.65547817083447 -94.92946421978785) bank42005 +42006 POINT(29.13702122754819 -94.60679132189283) bank42006 +42007 POINT(29.509837544711676 -94.68879011931563) bank42007 +42008 POINT(29.249113363952205 -95.92455340450697) bank42008 +42009 POINT(29.07983624107953 -94.49888858764925) bank42009 +42010 POINT(29.048829233650622 -96.04012195722852) bank42010 +42011 POINT(29.420906617445805 -95.39731135777775) bank42011 +42012 POINT(30.154985200702328 -96.02707245724626) bank42012 +42013 POINT(29.670394840673772 -95.48853200830317) bank42013 +42014 POINT(29.174178310296742 -94.47646533412863) bank42014 +42015 POINT(30.31290292533248 -95.72614181731576) bank42015 +42016 POINT(29.608448490569966 -94.88928374934319) bank42016 +42017 POINT(30.180764193664174 -95.69834242751844) bank42017 +42018 POINT(30.567680944882603 -95.70659209983893) bank42018 +42019 POINT(29.73408737007489 -94.91398889098501) bank42019 +42020 POINT(30.353145829259 -95.48491974803619) bank42020 +42021 POINT(29.67492334253769 -96.28577428870227) bank42021 +42022 POINT(30.508111964353514 -95.12478042743076) bank42022 +42023 POINT(30.09638822804081 -94.53985817743592) bank42023 +42024 POINT(30.116843203060448 -95.81836991917781) bank42024 +42025 POINT(29.56622579237414 -95.17722880032629) bank42025 +42026 POINT(30.36343402758571 -95.94585245860581) bank42026 +42027 POINT(29.966644102516067 -96.20695428051768) bank42027 +42028 POINT(29.232492567177935 -95.11905265341618) bank42028 +42029 POINT(28.892383940879718 -96.02350618946478) bank42029 +42030 POINT(29.70558255916842 -95.88574182174196) bank42030 +42031 POINT(30.704518563542564 -96.06317987771465) bank42031 +42032 POINT(29.691353628000904 -94.45622102086395) bank42032 +42033 POINT(29.436188801120885 -95.12247702905644) bank42033 +42034 POINT(29.87808740828681 -95.22469490189893) bank42034 +42035 POINT(29.980318081362686 -94.88605664885631) bank42035 +42036 POINT(30.675507905412307 -94.8876411825506) bank42036 +42037 POINT(29.91365135991647 -94.90597863012015) bank42037 +42038 POINT(28.850305378606162 -96.02159764253571) bank42038 +42039 POINT(30.191979912443244 -95.03076323614327) bank42039 +42040 POINT(28.964707654854568 -94.63455439389345) bank42040 +42041 POINT(29.634669226399705 -96.07422019627015) bank42041 +42042 POINT(29.98816075581826 -95.38962934900044) bank42042 +42043 POINT(28.769909149118128 -94.77686555040049) bank42043 +42044 POINT(29.116858437922794 -95.41767708541738) bank42044 +42045 POINT(30.647672189225965 -95.65282363826991) bank42045 +42046 POINT(30.227667608377313 -94.38818563880564) bank42046 +42047 POINT(30.275335514081366 -96.20299549898068) bank42047 +42048 POINT(29.280417122595875 -95.39603974245854) bank42048 +42049 POINT(29.670702112455178 -95.84584961210163) bank42049 +42050 POINT(30.372836030673117 -94.64387548206021) bank42050 +42051 POINT(29.608596532252548 -95.32289542542881) bank42051 +42052 POINT(28.943970140079433 -94.96731580894064) bank42052 +42053 POINT(29.2562166521648 -95.98342437946097) bank42053 +42054 POINT(29.685670091790954 -95.17536552286032) bank42054 +42055 POINT(29.129440496224326 -95.04590737715698) bank42055 +42056 POINT(28.807355953561736 -94.91624711627937) bank42056 +42057 POINT(29.576143258313216 -95.97569337270502) bank42057 +42058 POINT(28.83308400124081 -95.17078513050403) bank42058 +42059 POINT(29.719581779433618 -96.06359489464607) bank42059 +42060 POINT(29.04476727942163 -95.21935954263884) bank42060 +42061 POINT(28.850961539410914 -94.94034580356438) bank42061 +42062 POINT(30.077846614101055 -95.27446288311513) bank42062 +42063 POINT(29.405520331618256 -94.875151299468) bank42063 +42064 POINT(30.51396168129562 -95.65669430381722) bank42064 +42065 POINT(30.62496272834852 -95.05117150632252) bank42065 +42066 POINT(29.87292567178175 -95.58084122635412) bank42066 +42067 POINT(29.077662792728628 -95.74726857861302) bank42067 +42068 POINT(30.48030342778203 -95.81122689320294) bank42068 +42069 POINT(30.604266499234694 -94.40295594277706) bank42069 +42070 POINT(29.663341647027607 -94.97187030848215) bank42070 +42071 POINT(29.295246254983503 -96.35701865085078) bank42071 +42072 POINT(30.17429592125057 -95.04458673640524) bank42072 +42073 POINT(30.547199200715287 -95.12579232155112) bank42073 +42074 POINT(29.329822488047576 -96.0427845858407) bank42074 +42075 POINT(30.559879195545136 -96.18523817823197) bank42075 +42076 POINT(30.289911023956428 -95.57686415450992) bank42076 +42077 POINT(28.90105764367916 -95.51901006120536) bank42077 +42078 POINT(29.39815933828867 -95.23677845569225) bank42078 +42079 POINT(30.585670881531552 -96.10064728973694) bank42079 +42080 POINT(30.719214393682456 -95.3874436097837) bank42080 +42081 POINT(30.01043024428217 -94.40686602233271) bank42081 +42082 POINT(29.02823014827418 -95.3043061295779) bank42082 +42083 POINT(30.401255751014332 -95.87415370337015) bank42083 +42084 POINT(29.599614086091535 -94.55827000477628) bank42084 +42085 POINT(30.082184819209402 -95.29997739769097) bank42085 +42086 POINT(29.957322752953594 -96.07643666564672) bank42086 +42087 POINT(29.971282243303268 -95.64834832632174) bank42087 +42088 POINT(30.63418031208179 -96.2777637510435) bank42088 +42089 POINT(30.29371735628394 -94.592413015139) bank42089 +42090 POINT(30.520039921006973 -96.05077330029273) bank42090 +42091 POINT(30.578236038553225 -95.19041880653603) bank42091 +42092 POINT(30.303406573836327 -95.57654175792973) bank42092 +42093 POINT(29.51004109070481 -94.37118162817168) bank42093 +42094 POINT(30.592466783092576 -95.15362108482213) bank42094 +42095 POINT(29.736858820412195 -94.46721367631048) bank42095 +42096 POINT(30.0739672474953 -95.1148148400173) bank42096 +42097 POINT(29.2156095728957 -96.04751964403091) bank42097 +42098 POINT(29.985296415647827 -95.75681733286858) bank42098 +42099 POINT(29.93306874219382 -95.18539094512617) bank42099 +42100 POINT(30.217165650427965 -95.21085092787023) bank42100 +42101 POINT(30.647356537894737 -95.86366625311493) bank42101 +42102 POINT(30.137118665293357 -94.82986923750929) bank42102 +42103 POINT(29.73689259477685 -95.58436271716161) bank42103 +42104 POINT(30.709655858263208 -94.38636140893367) bank42104 +42105 POINT(30.346649765974732 -96.11080575432159) bank42105 +42106 POINT(28.94392091191938 -95.04308730261153) bank42106 +42107 POINT(29.203809017868267 -94.63913206649397) bank42107 +42108 POINT(28.816573065843997 -96.1206878849467) bank42108 +42109 POINT(30.582819283265525 -94.88552026064146) bank42109 +42110 POINT(30.33350246769027 -96.33508920085866) bank42110 +42111 POINT(30.326445455570852 -94.97510908889245) bank42111 +42112 POINT(29.273652862044617 -94.54006689297992) bank42112 +42113 POINT(30.743510613715046 -95.21652720269826) bank42113 +42114 POINT(29.672280421398472 -95.90599837973704) bank42114 +42115 POINT(29.665107910654598 -95.7823978919985) bank42115 +42116 POINT(29.714690649841906 -94.6358978961057) bank42116 +42117 POINT(30.185148905494618 -94.98755754816402) bank42117 +42118 POINT(29.42124081690391 -96.21701300683573) bank42118 +42119 POINT(30.061263852542965 -94.51980677807464) bank42119 +42120 POINT(30.632901048056443 -96.03591583724798) bank42120 +42121 POINT(29.073677387894254 -96.22634039044762) bank42121 +42122 POINT(28.874494782961534 -95.60576401895793) bank42122 +42123 POINT(29.78044244546013 -95.28953022987454) bank42123 +42124 POINT(30.672210847183024 -95.00433496510334) bank42124 +42125 POINT(29.587102352551682 -95.92943583981338) bank42125 +42126 POINT(29.925445938537187 -95.81564994486286) bank42126 +42127 POINT(30.424319272881 -95.36412513494261) bank42127 +42128 POINT(28.93693122749552 -95.95922706962344) bank42128 +42129 POINT(30.15528518655823 -95.64672710789372) bank42129 +42130 POINT(30.096351749894716 -95.5179297269683) bank42130 +42131 POINT(29.239511536710232 -94.4449717807162) bank42131 +42132 POINT(29.485547283212444 -95.11348295981408) bank42132 +42133 POINT(29.52437913114257 -95.16981627849692) bank42133 +42134 POINT(29.39646217220247 -94.82909156138854) bank42134 +42135 POINT(29.442983770259495 -94.60784794279361) bank42135 +42136 POINT(30.332015816873316 -96.21832768636764) bank42136 +42137 POINT(28.937458182789946 -95.72438832030555) bank42137 +42138 POINT(30.49566095373486 -95.38851450948742) bank42138 +42139 POINT(29.31889484462449 -94.83426139727531) bank42139 +42140 POINT(30.45533285494196 -94.54541285308228) bank42140 +42141 POINT(29.348315839087526 -94.57845722579341) bank42141 +42142 POINT(29.068020492763694 -94.53030988872575) bank42142 +42143 POINT(29.751610548991437 -95.49522052789376) bank42143 +42144 POINT(29.188621768511155 -95.7322259467029) bank42144 +42145 POINT(29.906364708446972 -94.58218121296851) bank42145 +42146 POINT(30.34785194978613 -95.64084671427169) bank42146 +42147 POINT(29.580863808166974 -95.15138696561826) bank42147 +42148 POINT(29.07932321001394 -95.26132189254206) bank42148 +42149 POINT(29.115507095235618 -94.38434913806181) bank42149 +42150 POINT(30.312133824320068 -95.13626802469845) bank42150 +42151 POINT(29.06105440809074 -96.1264968793976) bank42151 +42152 POINT(29.748240086368806 -95.19328208884772) bank42152 +42153 POINT(30.27949426195592 -95.40556371271545) bank42153 +42154 POINT(28.823687275021673 -95.35733889473508) bank42154 +42155 POINT(30.08975334298076 -95.26720513326879) bank42155 +42156 POINT(30.538243659463024 -94.47856136654325) bank42156 +42157 POINT(30.253498683203624 -94.78434362803682) bank42157 +42158 POINT(30.6514212097319 -94.37533457700314) bank42158 +42159 POINT(29.111979053177432 -95.85709109653845) bank42159 +42160 POINT(29.936252712649747 -94.70832409161149) bank42160 +42161 POINT(29.822666113523535 -95.56614957340943) bank42161 +42162 POINT(30.242784751411335 -96.02024917276957) bank42162 +42163 POINT(30.188441082071712 -96.23626572104591) bank42163 +42164 POINT(29.876860780119767 -95.67297927183924) bank42164 +42165 POINT(29.05609829955005 -94.47595708332517) bank42165 +42166 POINT(28.940754936443344 -95.16184451334365) bank42166 +42167 POINT(30.19177305281819 -94.41434946814168) bank42167 +42168 POINT(29.50848865516465 -95.09561711777333) bank42168 +42169 POINT(30.006254975069623 -94.650639449245) bank42169 +42170 POINT(29.80730085722247 -94.5727225898275) bank42170 +42171 POINT(29.86264488326993 -94.72563817976604) bank42171 +42172 POINT(30.273116260043075 -95.79325816299718) bank42172 +42173 POINT(29.247190831160566 -94.58262625077758) bank42173 +42174 POINT(29.046664008706987 -94.84969914701075) bank42174 +42175 POINT(28.94707570403601 -95.29358616756427) bank42175 +42176 POINT(28.956414926957308 -95.39116762396353) bank42176 +42177 POINT(28.806861133010965 -94.63809810581613) bank42177 +42178 POINT(29.427040629040015 -96.22701256135798) bank42178 +42179 POINT(29.90832596011667 -94.7766934119203) bank42179 +42180 POINT(28.852337149503636 -96.11277614410335) bank42180 +42181 POINT(29.485030584154618 -94.37587499636722) bank42181 +42182 POINT(30.759815716773243 -94.49158686605986) bank42182 +42183 POINT(30.138338161179885 -94.40760022509919) bank42183 +42184 POINT(30.572275907598378 -95.34022840818527) bank42184 +42185 POINT(30.22422345433263 -96.35882253181539) bank42185 +42186 POINT(30.445119291093835 -95.33380628989933) bank42186 +42187 POINT(29.818909026740183 -95.56373375126259) bank42187 +42188 POINT(29.464586693081255 -94.92224735096372) bank42188 +42189 POINT(28.80705321262293 -95.59915402769825) bank42189 +42190 POINT(30.472673599592483 -96.32902115137999) bank42190 +42191 POINT(29.200322536405814 -94.48779478249898) bank42191 +42192 POINT(30.24351214402818 -96.09204356878317) bank42192 +42193 POINT(28.88949893463254 -95.17651496075791) bank42193 +42194 POINT(29.626660798834884 -94.40049147307026) bank42194 +42195 POINT(29.998158915921405 -95.01603838761385) bank42195 +42196 POINT(29.970443036336444 -94.80686685276528) bank42196 +42197 POINT(29.574281876140677 -95.93906655782885) bank42197 +42198 POINT(29.723907245913985 -95.81606071288456) bank42198 +42199 POINT(30.215478034838167 -95.20532494081587) bank42199 +42200 POINT(29.489609471079646 -95.97037941175387) bank42200 +42201 POINT(30.4191623451963 -94.94869404968509) bank42201 +42202 POINT(30.150489060947304 -95.99914628319137) bank42202 +42203 POINT(29.699779452989322 -95.12516067758193) bank42203 +42204 POINT(28.88728441822952 -94.42666135847031) bank42204 +42205 POINT(30.264577798978866 -96.28600113989846) bank42205 +42206 POINT(28.907422190376824 -95.51923776806846) bank42206 +42207 POINT(29.39400465311469 -95.99667138376063) bank42207 +42208 POINT(29.29111711475265 -96.25945117744286) bank42208 +42209 POINT(29.51963215920582 -96.24613530472827) bank42209 +42210 POINT(29.38571953016819 -94.95203189548789) bank42210 +42211 POINT(30.73743630341567 -96.10854397281061) bank42211 +42212 POINT(29.196760946308366 -96.33953933480345) bank42212 +42213 POINT(29.239939810376782 -95.43866546468637) bank42213 +42214 POINT(30.41351326509577 -94.37892200399362) bank42214 +42215 POINT(29.166434457857672 -95.78665216189566) bank42215 +42216 POINT(29.11625449275587 -95.36743292250264) bank42216 +42217 POINT(30.49876798096235 -94.4942327162863) bank42217 +42218 POINT(30.134571436073752 -94.89722274705089) bank42218 +42219 POINT(29.774221451145934 -95.36875951348438) bank42219 +42220 POINT(30.735721862285057 -94.95363069347157) bank42220 +42221 POINT(29.397074478084036 -95.54327063884472) bank42221 +42222 POINT(29.86464836292822 -94.5506720623059) bank42222 +42223 POINT(29.4220660317796 -94.97706079604434) bank42223 +42224 POINT(30.020312129111378 -96.1192738477145) bank42224 +42225 POINT(29.019997504395544 -96.29943246174268) bank42225 +42226 POINT(28.782543667860566 -95.75121630662926) bank42226 +42227 POINT(30.147015332815062 -94.44479634858999) bank42227 +42228 POINT(29.504263397335325 -94.39214874723223) bank42228 +42229 POINT(30.318207316607847 -94.92913371337195) bank42229 +42230 POINT(30.18727128660557 -96.18507479136292) bank42230 +42231 POINT(29.013012864213415 -94.57805849881125) bank42231 +42232 POINT(30.732887775652344 -94.70390328600048) bank42232 +42233 POINT(30.711969879777588 -95.84810215512685) bank42233 +42234 POINT(30.086970299405735 -96.17578604659097) bank42234 +42235 POINT(29.756079070404358 -94.71291869482663) bank42235 +42236 POINT(30.11774161910032 -95.86202220494619) bank42236 +42237 POINT(30.65526795234822 -96.21213195935042) bank42237 +42238 POINT(28.991258338474733 -94.3922169832469) bank42238 +42239 POINT(29.837115721058545 -96.0637723476036) bank42239 +42240 POINT(28.77937388284798 -96.36584869134137) bank42240 +42241 POINT(30.373329166140703 -95.86108749798716) bank42241 +42242 POINT(29.717689407472154 -95.8611813420926) bank42242 +42243 POINT(29.432958105770396 -95.3871848827368) bank42243 +42244 POINT(29.152515268629095 -94.76341380328037) bank42244 +42245 POINT(30.473589383073286 -95.17501117318814) bank42245 +42246 POINT(29.25960544865248 -95.68096377625263) bank42246 +42247 POINT(30.685141893026866 -94.94173200867235) bank42247 +42248 POINT(29.208528756528267 -96.07222863363575) bank42248 +42249 POINT(30.06252486600758 -94.8626829192859) bank42249 +42250 POINT(28.837323833517033 -95.05201111087032) bank42250 +42251 POINT(29.91545603085233 -95.62173721727316) bank42251 +42252 POINT(30.467726498430654 -95.32772249202948) bank42252 +42253 POINT(29.32281995218664 -96.23558880277172) bank42253 +42254 POINT(30.753330704067775 -95.58323937538138) bank42254 +42255 POINT(30.654191841278866 -95.60748050564011) bank42255 +42256 POINT(28.862491022774538 -96.03140083022203) bank42256 +42257 POINT(29.81738949733667 -94.62469038304543) bank42257 +42258 POINT(28.950832047927797 -95.93282341101003) bank42258 +42259 POINT(30.01260176351355 -94.7379881634164) bank42259 +42260 POINT(29.038179837931715 -95.77396776791876) bank42260 +42261 POINT(29.496732741321985 -94.7535644050742) bank42261 +42262 POINT(28.83770734850452 -94.7412574523132) bank42262 +42263 POINT(30.59174996505762 -94.98088435534798) bank42263 +42264 POINT(29.626697560503874 -94.65692837051986) bank42264 +42265 POINT(30.42540577160058 -95.65710452111784) bank42265 +42266 POINT(30.479053477228906 -95.9490568130857) bank42266 +42267 POINT(29.499653642608767 -95.7430475884287) bank42267 +42268 POINT(29.365054386964083 -95.85671186605745) bank42268 +42269 POINT(29.91524388295751 -95.02167273308963) bank42269 +42270 POINT(30.35745169804034 -95.93543751061219) bank42270 +42271 POINT(29.553508356913273 -95.77948719987036) bank42271 +42272 POINT(29.87345203373972 -96.11075830444548) bank42272 +42273 POINT(29.70451021595557 -95.91875062225759) bank42273 +42274 POINT(30.565191173145735 -96.35568709295627) bank42274 +42275 POINT(29.942842580180592 -95.7726576301941) bank42275 +42276 POINT(30.715889588488686 -95.36608950031172) bank42276 +42277 POINT(30.22399630049133 -96.30629646114878) bank42277 +42278 POINT(28.85870855319488 -95.02088658734121) bank42278 +42279 POINT(30.268343842586965 -95.77878835595916) bank42279 +42280 POINT(29.99051879875235 -96.18882058539751) bank42280 +42281 POINT(29.168371222584828 -95.06033436899673) bank42281 +42282 POINT(29.79644941979325 -95.46070870375313) bank42282 +42283 POINT(29.309049659425526 -95.56250393102782) bank42283 +42284 POINT(30.363926296972487 -94.98639166941769) bank42284 +42285 POINT(29.767261360964365 -94.59406587087798) bank42285 +42286 POINT(29.141199359154285 -94.88326144023225) bank42286 +42287 POINT(30.210397514837496 -95.4662724119097) bank42287 +42288 POINT(30.42234456322913 -94.81667209269004) bank42288 +42289 POINT(29.133108298525872 -95.30506042120919) bank42289 +42290 POINT(29.485938600352103 -94.44383084303732) bank42290 +42291 POINT(30.338562193134415 -95.84813587693316) bank42291 +42292 POINT(30.46309041192197 -96.05198409720758) bank42292 +42293 POINT(29.418805939120965 -95.76681759966226) bank42293 +42294 POINT(30.4428668085913 -94.52466982771949) bank42294 +42295 POINT(28.76985135028676 -95.60312337414908) bank42295 +42296 POINT(30.599789563011086 -96.01964200981158) bank42296 +42297 POINT(30.008010304465444 -94.42757627619143) bank42297 +42298 POINT(29.684395140665476 -94.82366154320752) bank42298 +42299 POINT(30.139256235900945 -96.15699276343557) bank42299 +42300 POINT(30.492639268599785 -95.77816051213917) bank42300 +42301 POINT(30.662543226593478 -96.0357116315668) bank42301 +42302 POINT(29.660734294041582 -96.0556046210687) bank42302 +42303 POINT(30.636398782437098 -95.02599997181952) bank42303 +42304 POINT(30.004824139139657 -95.51558428709295) bank42304 +42305 POINT(30.150172435127043 -95.64703524846625) bank42305 +42306 POINT(29.65773270727914 -95.35234288124263) bank42306 +42307 POINT(29.43179930702988 -95.3959171330651) bank42307 +42308 POINT(30.355872560657208 -95.56653552687331) bank42308 +42309 POINT(29.101135469603538 -95.50597699471987) bank42309 +42310 POINT(29.68213033699291 -94.69699463337396) bank42310 +42311 POINT(30.687517514105995 -95.62765084043232) bank42311 +42312 POINT(29.865473712058126 -94.58357302300705) bank42312 +42313 POINT(29.847165238027408 -94.78437703011144) bank42313 +42314 POINT(29.248976123320126 -96.31441369478566) bank42314 +42315 POINT(30.42449438439158 -95.57590269649747) bank42315 +42316 POINT(29.136121091686316 -96.22762212882783) bank42316 +42317 POINT(29.92904587628664 -95.16705476515727) bank42317 +42318 POINT(30.486162800986214 -94.8618710978423) bank42318 +42319 POINT(28.928573882291982 -94.59997155091665) bank42319 +42320 POINT(30.462712494698202 -96.26654679179103) bank42320 +42321 POINT(30.24357061653196 -94.61034007715506) bank42321 +42322 POINT(29.071801695090162 -96.12181132998138) bank42322 +42323 POINT(29.224582826847424 -95.39119925565092) bank42323 +42324 POINT(29.738641288911793 -94.4917869509809) bank42324 +42325 POINT(30.73301490086783 -95.37288895275726) bank42325 +42326 POINT(29.65901691252504 -96.06231944475775) bank42326 +42327 POINT(29.957322509728925 -94.39467793947361) bank42327 +42328 POINT(28.80598853241523 -95.96527795824433) bank42328 +42329 POINT(29.123767310426196 -95.31285470145082) bank42329 +42330 POINT(30.066206963918106 -94.76188521038152) bank42330 +42331 POINT(28.888527682566977 -96.25416490084244) bank42331 +42332 POINT(30.01525252537892 -95.21187489481221) bank42332 +42333 POINT(30.749506213917037 -95.76240175546059) bank42333 +42334 POINT(29.774552110626626 -94.61970670897043) bank42334 +42335 POINT(30.46812457541249 -95.73239865600456) bank42335 +42336 POINT(29.303837504168047 -95.1748806903464) bank42336 +42337 POINT(30.503291462369205 -94.6048352768886) bank42337 +42338 POINT(30.667998795808646 -94.6981290295381) bank42338 +42339 POINT(29.096514683934632 -95.21028864257117) bank42339 +42340 POINT(29.40011832298874 -96.32718146764607) bank42340 +42341 POINT(29.61926327661597 -96.19123164002438) bank42341 +42342 POINT(29.69544778745981 -94.91158513653853) bank42342 +42343 POINT(30.140374804983352 -95.10359029572064) bank42343 +42344 POINT(30.238968601014292 -96.30142476921544) bank42344 +42345 POINT(30.29626827603688 -94.92280207103462) bank42345 +42346 POINT(30.55911931540417 -96.140132746003) bank42346 +42347 POINT(30.18003983819024 -96.20730173179892) bank42347 +42348 POINT(29.463857753187753 -94.69482986808437) bank42348 +42349 POINT(29.797866807237753 -95.65928303670287) bank42349 +42350 POINT(30.112825197371205 -95.22821892115904) bank42350 +42351 POINT(30.06312427005434 -95.99539869361716) bank42351 +42352 POINT(29.649831507270925 -96.2069281132307) bank42352 +42353 POINT(29.05726833030766 -95.69294314624221) bank42353 +42354 POINT(30.17609583635226 -95.99507009393425) bank42354 +42355 POINT(29.67442078719087 -95.53611611959306) bank42355 +42356 POINT(29.94966467792864 -95.7889303960477) bank42356 +42357 POINT(29.93870764682892 -95.11561235419705) bank42357 +42358 POINT(29.82063190913362 -96.01313568793125) bank42358 +42359 POINT(29.871148220176583 -96.33684753868475) bank42359 +42360 POINT(30.079726274531687 -96.34109869562408) bank42360 +42361 POINT(30.037250207004526 -95.67192871283977) bank42361 +42362 POINT(29.44910638013844 -95.70001190882338) bank42362 +42363 POINT(29.110534860484687 -94.89858426410402) bank42363 +42364 POINT(29.409823903984346 -95.39645806457209) bank42364 +42365 POINT(29.14655185282651 -94.80370073504471) bank42365 +42366 POINT(30.1127420352455 -94.41966708262575) bank42366 +42367 POINT(28.851296811298067 -94.81292298542469) bank42367 +42368 POINT(30.60817749256651 -96.1656149616495) bank42368 +42369 POINT(30.3411572724541 -94.54000052857813) bank42369 +42370 POINT(30.019314354893822 -96.28743282483735) bank42370 +42371 POINT(28.91837504639376 -95.52624628360414) bank42371 +42372 POINT(29.059357650087335 -95.09924442948639) bank42372 +42373 POINT(30.444319061698014 -95.44425159963127) bank42373 +42374 POINT(29.720450661801046 -94.9783730419415) bank42374 +42375 POINT(30.538743088054094 -95.01251702436956) bank42375 +42376 POINT(30.4577045553716 -95.23588091399168) bank42376 +42377 POINT(29.631507782358554 -94.37558677079122) bank42377 +42378 POINT(29.224639968397142 -94.56007156116888) bank42378 +42379 POINT(29.79496760233535 -95.50195409911872) bank42379 +42380 POINT(30.30219610571782 -95.74304058317959) bank42380 +42381 POINT(30.404213298995185 -95.17552301635415) bank42381 +42382 POINT(30.7058866506202 -96.25121089766334) bank42382 +42383 POINT(30.598439136341707 -94.68699594963054) bank42383 +42384 POINT(30.636496416942627 -95.70293246943487) bank42384 +42385 POINT(30.39096423440379 -95.53023914887083) bank42385 +42386 POINT(28.79444214810634 -94.99829658097585) bank42386 +42387 POINT(29.925735621429897 -96.173533551747) bank42387 +42388 POINT(29.38596626747514 -95.92532215348562) bank42388 +42389 POINT(30.478760107022502 -95.98544568047872) bank42389 +42390 POINT(30.654617922213504 -95.97468958881949) bank42390 +42391 POINT(29.805656051154116 -96.0521115441247) bank42391 +42392 POINT(28.837255869206558 -95.08676807706667) bank42392 +42393 POINT(28.933558798002736 -95.45814427328109) bank42393 +42394 POINT(30.3190132995735 -96.15553360588616) bank42394 +42395 POINT(29.050489690792705 -95.63760923316464) bank42395 +42396 POINT(30.198668753717524 -95.67392534894547) bank42396 +42397 POINT(30.40177141497152 -94.9114056562314) bank42397 +42398 POINT(30.184599394048917 -95.4632698771996) bank42398 +42399 POINT(30.124618692707262 -95.12792618070726) bank42399 +42400 POINT(29.7428334629287 -96.11931929932322) bank42400 +42401 POINT(30.248160451830884 -95.0610488906901) bank42401 +42402 POINT(29.553069231509696 -96.26215515969302) bank42402 +42403 POINT(29.115248152194035 -96.09196865317885) bank42403 +42404 POINT(30.671174004714032 -95.25748202167352) bank42404 +42405 POINT(30.65068583954656 -94.91547377911338) bank42405 +42406 POINT(30.70706798575453 -95.75896621278919) bank42406 +42407 POINT(29.633291619986583 -95.91024220523694) bank42407 +42408 POINT(30.526510799902457 -95.81607566336494) bank42408 +42409 POINT(28.82086274120842 -95.14553933545326) bank42409 +42410 POINT(30.241519918389763 -95.11271431489929) bank42410 +42411 POINT(29.66387459480645 -95.32196019448413) bank42411 +42412 POINT(29.090117956900396 -96.1076426412602) bank42412 +42413 POINT(29.971081473949425 -96.05858486664735) bank42413 +42414 POINT(28.846973788282643 -95.75575761406873) bank42414 +42415 POINT(29.235892557044902 -95.55183817058473) bank42415 +42416 POINT(28.82486174548638 -95.37583471726053) bank42416 +42417 POINT(29.30260481946793 -95.73046082236293) bank42417 +42418 POINT(30.01122078270237 -94.72356340459865) bank42418 +42419 POINT(29.854630772495682 -95.82622332694436) bank42419 +42420 POINT(30.323010783342838 -95.78336235498558) bank42420 +42421 POINT(29.475049071011213 -95.47791275637528) bank42421 +42422 POINT(30.155457439677274 -94.6745689162705) bank42422 +42423 POINT(29.094424019992147 -95.34543335186342) bank42423 +42424 POINT(30.168886665852163 -95.9881770645093) bank42424 +42425 POINT(29.647385083618367 -95.42500924463847) bank42425 +42426 POINT(29.469735697154587 -95.88907193454563) bank42426 +42427 POINT(30.495134720931453 -96.0165974742712) bank42427 +42428 POINT(30.05209234253028 -94.48613913771982) bank42428 +42429 POINT(29.88586690326477 -95.48412718131979) bank42429 +42430 POINT(30.465751114140684 -95.43302845195794) bank42430 +42431 POINT(29.20203803020974 -95.37787375459304) bank42431 +42432 POINT(28.839875802184515 -94.79265027790157) bank42432 +42433 POINT(30.377450543558666 -95.65486030344984) bank42433 +42434 POINT(29.289639404445868 -96.27038757040027) bank42434 +42435 POINT(30.250770636167545 -94.90206745424233) bank42435 +42436 POINT(28.864941981895075 -96.0608305963584) bank42436 +42437 POINT(29.436272247042005 -95.72975320789598) bank42437 +42438 POINT(30.318082838117 -95.50302383135659) bank42438 +42439 POINT(30.529072351081066 -95.89448696787488) bank42439 +42440 POINT(30.38235852683345 -94.72527819975207) bank42440 +42441 POINT(29.594174432853734 -94.63187782909993) bank42441 +42442 POINT(30.301263523757935 -94.59224250784517) bank42442 +42443 POINT(30.426343119406816 -95.23772729056925) bank42443 +42444 POINT(30.0591145605709 -94.90856925794606) bank42444 +42445 POINT(28.895971146276537 -94.91849465511062) bank42445 +42446 POINT(29.0858918563075 -94.44849160537674) bank42446 +42447 POINT(29.224528435203634 -95.76276030030093) bank42447 +42448 POINT(29.320258980219478 -94.82666744955307) bank42448 +42449 POINT(29.943712065269168 -95.25778721195388) bank42449 +42450 POINT(28.763194225962966 -94.76828693517818) bank42450 +42451 POINT(29.90021526964574 -94.593391980749) bank42451 +42452 POINT(29.21325994402297 -96.03552739254437) bank42452 +42453 POINT(28.99981259213628 -95.45696975594257) bank42453 +42454 POINT(29.908279687190035 -95.93616780635875) bank42454 +42455 POINT(30.428574575588573 -94.67214839212515) bank42455 +42456 POINT(29.130766059509604 -95.02719762199845) bank42456 +42457 POINT(29.641492628860266 -94.75341903636443) bank42457 +42458 POINT(28.86220676663583 -95.60479106865813) bank42458 +42459 POINT(29.156349623741054 -95.7451782823072) bank42459 +42460 POINT(29.492130013080544 -95.87282153437611) bank42460 +42461 POINT(30.247813510235495 -95.76917195293356) bank42461 +42462 POINT(29.778111728486206 -94.46866099359245) bank42462 +42463 POINT(30.063231943046343 -96.36908827178726) bank42463 +42464 POINT(30.24595874633572 -95.63215972440088) bank42464 +42465 POINT(29.59525408149315 -94.6636181962909) bank42465 +42466 POINT(29.333885532331188 -95.0647410734022) bank42466 +42467 POINT(28.84308302794635 -96.25786449667932) bank42467 +42468 POINT(29.79422199801828 -94.44383078243561) bank42468 +42469 POINT(30.09913256279315 -95.62850146162212) bank42469 +42470 POINT(29.537616758537197 -95.99796483662585) bank42470 +42471 POINT(28.967950419160747 -94.65856823256719) bank42471 +42472 POINT(29.566917095884484 -94.76204485073413) bank42472 +42473 POINT(30.464406591109494 -95.67243014188628) bank42473 +42474 POINT(29.479208113168067 -94.92234892898415) bank42474 +42475 POINT(30.31359369333809 -94.67044571168833) bank42475 +42476 POINT(30.23815958655959 -95.86204250037727) bank42476 +42477 POINT(30.230310564721485 -96.25867328618988) bank42477 +42478 POINT(30.425564676478896 -95.46567449487517) bank42478 +42479 POINT(30.67715323924382 -94.65706102163347) bank42479 +42480 POINT(30.69073292533785 -95.75117625374087) bank42480 +42481 POINT(29.03022426843932 -95.52274965868722) bank42481 +42482 POINT(29.366487017989662 -96.3244774022878) bank42482 +42483 POINT(29.977028282907856 -94.68301096400718) bank42483 +42484 POINT(30.379934082305354 -95.66107414424177) bank42484 +42485 POINT(29.52394370312829 -95.39707721022066) bank42485 +42486 POINT(30.484885575251035 -94.94687804415125) bank42486 +42487 POINT(28.899830406132207 -96.08086217208088) bank42487 +42488 POINT(30.66810432368911 -96.19775494378648) bank42488 +42489 POINT(28.762704415657968 -95.88398631892751) bank42489 +42490 POINT(30.062614506164692 -95.72334159365568) bank42490 +42491 POINT(29.01199117911008 -95.74832180213319) bank42491 +42492 POINT(29.434885342855324 -95.7271331642106) bank42492 +42493 POINT(28.85149949777457 -94.67870255199203) bank42493 +42494 POINT(29.539029516488053 -96.3074740318707) bank42494 +42495 POINT(30.199146166266722 -94.43168951762739) bank42495 +42496 POINT(30.017948436759916 -94.91775322004182) bank42496 +42497 POINT(28.826533902690606 -95.3588865204802) bank42497 +42498 POINT(28.958928631319427 -95.4724237367573) bank42498 +42499 POINT(29.831020547747812 -96.2780020559633) bank42499 +42500 POINT(29.419804145291145 -94.80654157353177) bank42500 +42501 POINT(30.246736076202154 -95.12564118222707) bank42501 +42502 POINT(29.1535333114138 -96.081926907289) bank42502 +42503 POINT(29.546685397534215 -95.9207966022954) bank42503 +42504 POINT(29.92564202655068 -95.20468805264804) bank42504 +42505 POINT(29.61234085055703 -96.17704828435186) bank42505 +42506 POINT(30.201401213410815 -95.7417812109763) bank42506 +42507 POINT(30.616456260794635 -94.55151518108305) bank42507 +42508 POINT(30.674018705351084 -96.36303629969802) bank42508 +42509 POINT(30.148607370477116 -95.55199474625748) bank42509 +42510 POINT(29.006076327995014 -96.19588971270639) bank42510 +42511 POINT(29.61132567218206 -94.45662163979799) bank42511 +42512 POINT(30.028695723362326 -95.48977196482893) bank42512 +42513 POINT(30.032443713613155 -94.54374791261736) bank42513 +42514 POINT(29.71486011279748 -95.0055630194062) bank42514 +42515 POINT(29.19848073578584 -94.7797823375683) bank42515 +42516 POINT(28.881246393878047 -95.57978900168938) bank42516 +42517 POINT(28.855066038445013 -96.00517033162723) bank42517 +42518 POINT(29.12842938600373 -95.9251361575741) bank42518 +42519 POINT(30.701569664516942 -95.25436361871687) bank42519 +42520 POINT(28.824874262557387 -96.25645616199581) bank42520 +42521 POINT(29.93333424337275 -95.01743003174487) bank42521 +42522 POINT(28.954146129744096 -95.98986025960087) bank42522 +42523 POINT(29.620148978780644 -95.03343600135663) bank42523 +42524 POINT(29.678695143352986 -94.91335419248915) bank42524 +42525 POINT(28.87324548035717 -95.70555646356483) bank42525 +42526 POINT(30.151723804167347 -95.04414937291997) bank42526 +42527 POINT(29.668361152037356 -95.71890843817178) bank42527 +42528 POINT(29.585201769464636 -95.19984207789618) bank42528 +42529 POINT(29.636326157996333 -94.75765938439349) bank42529 +42530 POINT(29.725314625054658 -95.1185215338408) bank42530 +42531 POINT(30.128682406806927 -94.92647948761957) bank42531 +42532 POINT(28.913842596210532 -95.45307305150304) bank42532 +42533 POINT(29.793699399870235 -94.59193944398426) bank42533 +42534 POINT(29.216667439748115 -96.33239076698742) bank42534 +42535 POINT(29.647844706977807 -95.74169663601589) bank42535 +42536 POINT(29.167334728500062 -96.35196731684248) bank42536 +42537 POINT(30.69675676369496 -95.16604478950549) bank42537 +42538 POINT(28.880781327577186 -94.92410014152972) bank42538 +42539 POINT(29.885587952449498 -95.81220154534807) bank42539 +42540 POINT(29.941367626218046 -94.7283717043577) bank42540 +42541 POINT(29.153619442077243 -95.30066989171476) bank42541 +42542 POINT(29.61494425451876 -95.39569327787082) bank42542 +42543 POINT(29.695260772074185 -94.46705354283485) bank42543 +42544 POINT(28.956611113297658 -96.21923050105389) bank42544 +42545 POINT(30.52820471664308 -96.35389200887366) bank42545 +42546 POINT(29.258561135632117 -94.77171453503585) bank42546 +42547 POINT(29.178086823308323 -95.3904117236704) bank42547 +42548 POINT(28.771285843379527 -95.36088190127948) bank42548 +42549 POINT(29.996610555910713 -96.335047222396) bank42549 +42550 POINT(29.00671555022604 -95.91124220023802) bank42550 +42551 POINT(29.395957002395274 -94.97384025039509) bank42551 +42552 POINT(28.783898962880542 -94.53505908700372) bank42552 +42553 POINT(28.93855010235402 -95.03350370584273) bank42553 +42554 POINT(30.081710373910877 -94.58644779616908) bank42554 +42555 POINT(28.988251905670957 -95.31372463883837) bank42555 +42556 POINT(30.69607075568222 -95.54531091870157) bank42556 +42557 POINT(30.389516514362498 -96.21138055595581) bank42557 +42558 POINT(29.360103189261977 -95.07867116828444) bank42558 +42559 POINT(29.604656669271943 -95.26162770270246) bank42559 +42560 POINT(29.09167393401784 -95.49704938231685) bank42560 +42561 POINT(29.49541417924896 -96.02327939562723) bank42561 +42562 POINT(29.96977572007687 -96.03260082826439) bank42562 +42563 POINT(29.737860858965867 -95.0491341635168) bank42563 +42564 POINT(28.816147557369156 -94.98078619947225) bank42564 +42565 POINT(30.072349130397214 -95.56064059329542) bank42565 +42566 POINT(29.46900879936788 -94.84614118409606) bank42566 +42567 POINT(30.7190736376285 -94.57557282163705) bank42567 +42568 POINT(30.154474665993604 -94.97601723786592) bank42568 +42569 POINT(29.336012041019707 -94.39786690153412) bank42569 +42570 POINT(28.807805541353087 -95.05610018678536) bank42570 +42571 POINT(29.46275969643924 -94.67984356897577) bank42571 +42572 POINT(30.46408721361986 -94.84439812111614) bank42572 +42573 POINT(29.997585965346403 -95.30204679069313) bank42573 +42574 POINT(30.627399612577374 -95.39540089823176) bank42574 +42575 POINT(30.25416663421201 -95.62042071454378) bank42575 +42576 POINT(29.03575694318454 -95.59791179932122) bank42576 +42577 POINT(29.26778923259843 -96.00323666314068) bank42577 +42578 POINT(30.60414712276002 -95.96842486203633) bank42578 +42579 POINT(29.117891795559352 -95.21165663677286) bank42579 +42580 POINT(29.063179596220216 -95.38450555276866) bank42580 +42581 POINT(29.14025928900235 -94.82097698648964) bank42581 +42582 POINT(29.739991997870373 -94.5503742233007) bank42582 +42583 POINT(29.960961420176883 -96.13639288937493) bank42583 +42584 POINT(30.661537278461285 -95.3145517903781) bank42584 +42585 POINT(30.433594944644554 -95.36357330817007) bank42585 +42586 POINT(30.607091836022583 -94.81093436232614) bank42586 +42587 POINT(29.732262048968177 -94.86842318867161) bank42587 +42588 POINT(30.749159546230153 -95.69700502133979) bank42588 +42589 POINT(30.392002880799613 -94.60728359810105) bank42589 +42590 POINT(30.69133405265746 -94.73629303064288) bank42590 +42591 POINT(29.67458357880806 -95.43062856712191) bank42591 +42592 POINT(30.295458956074025 -96.09296658663752) bank42592 +42593 POINT(30.670326745637322 -95.208690156273) bank42593 +42594 POINT(30.216161097357904 -95.57868900760587) bank42594 +42595 POINT(29.787157166000494 -96.08542026962358) bank42595 +42596 POINT(28.973529561690576 -94.56607075006485) bank42596 +42597 POINT(30.162665956983552 -96.34503465432168) bank42597 +42598 POINT(29.976352028231922 -96.36563745592633) bank42598 +42599 POINT(28.764630078031782 -95.46314096494044) bank42599 +42600 POINT(30.48681545949876 -94.9003611448139) bank42600 +42601 POINT(29.74196402600642 -94.5755500113726) bank42601 +42602 POINT(28.82168163915615 -95.81997876673715) bank42602 +42603 POINT(29.33374270855446 -96.11471120540446) bank42603 +42604 POINT(29.49146202582852 -96.2234810815228) bank42604 +42605 POINT(29.623074785586958 -95.14810785500725) bank42605 +42606 POINT(29.466553498092622 -95.90767036314482) bank42606 +42607 POINT(30.347678454598935 -96.35841719933723) bank42607 +42608 POINT(28.911193824435845 -94.84272380103548) bank42608 +42609 POINT(28.84472260137052 -95.17186112500643) bank42609 +42610 POINT(29.747695695618976 -95.99118115575398) bank42610 +42611 POINT(30.19031122502981 -94.5179007614578) bank42611 +42612 POINT(29.40561050185201 -95.50582548100539) bank42612 +42613 POINT(29.428598448805165 -95.96577514393505) bank42613 +42614 POINT(28.941583307318137 -95.9400891966548) bank42614 +42615 POINT(30.583573154544727 -95.08467937501831) bank42615 +42616 POINT(29.283019668533196 -94.4597796446925) bank42616 +42617 POINT(30.275365966433647 -95.48805891045762) bank42617 +42618 POINT(30.41212502889509 -95.28846265753765) bank42618 +42619 POINT(29.495700016485824 -94.90137813048419) bank42619 +42620 POINT(30.035466209965648 -95.39678758392873) bank42620 +42621 POINT(29.21561861436237 -95.67787880074252) bank42621 +42622 POINT(29.668175296893455 -95.60862910484177) bank42622 +42623 POINT(29.326956831410413 -95.42189169545014) bank42623 +42624 POINT(29.66452087794644 -95.43089098269617) bank42624 +42625 POINT(29.6475218123476 -96.05078324466479) bank42625 +42626 POINT(30.000217942368398 -95.01529861868896) bank42626 +42627 POINT(29.31506415630538 -95.54864607909515) bank42627 +42628 POINT(29.597980481481585 -95.65210724998693) bank42628 +42629 POINT(29.949113384485955 -95.58927212318405) bank42629 +42630 POINT(29.0161066671219 -95.95346711095766) bank42630 +42631 POINT(29.145543243900303 -94.87212180862443) bank42631 +42632 POINT(29.29415650265078 -95.62036717008621) bank42632 +42633 POINT(29.9809488833501 -95.28998010050609) bank42633 +42634 POINT(29.21575659206979 -94.41258794512112) bank42634 +42635 POINT(28.969795146940896 -96.17551256096634) bank42635 +42636 POINT(30.54433653709056 -94.79117757134254) bank42636 +42637 POINT(30.15105755304957 -95.83565834934257) bank42637 +42638 POINT(30.359695329763998 -95.63961795863297) bank42638 +42639 POINT(29.81388824593305 -95.72429539362572) bank42639 +42640 POINT(29.778960025023633 -96.35680851955203) bank42640 +42641 POINT(30.102440215785165 -95.36807562499371) bank42641 +42642 POINT(30.26852496777843 -95.49665876035682) bank42642 +42643 POINT(30.678593101493774 -94.82249858626491) bank42643 +42644 POINT(30.747410157393052 -95.61960784137786) bank42644 +42645 POINT(30.400503468501388 -94.7646435519105) bank42645 +42646 POINT(30.550741210561956 -96.10177541641694) bank42646 +42647 POINT(30.5490045267326 -94.49099784785882) bank42647 +42648 POINT(30.499441165085994 -95.47648788005228) bank42648 +42649 POINT(29.22556393518371 -95.45178872329579) bank42649 +42650 POINT(30.713279189081923 -95.1078765035029) bank42650 +42651 POINT(28.86192180566105 -95.67611412196236) bank42651 +42652 POINT(29.324302929160233 -95.79323858861483) bank42652 +42653 POINT(30.56194554053146 -95.56830231524123) bank42653 +42654 POINT(29.99169691201608 -95.26344131767267) bank42654 +42655 POINT(30.06666018245782 -95.00067049255941) bank42655 +42656 POINT(29.72922369055838 -94.47131914779214) bank42656 +42657 POINT(29.264926663345882 -95.0041142510206) bank42657 +42658 POINT(30.090030090805218 -96.17900572272873) bank42658 +42659 POINT(30.292010128142444 -95.65577839924775) bank42659 +42660 POINT(30.142963960003687 -96.22073007453801) bank42660 +42661 POINT(29.560639837845418 -95.50426423298326) bank42661 +42662 POINT(30.679519699590205 -95.06326257576083) bank42662 +42663 POINT(30.082214802676724 -95.47619302381365) bank42663 +42664 POINT(29.724035334695742 -96.15049302893844) bank42664 +42665 POINT(30.479799460949415 -95.4192475332745) bank42665 +42666 POINT(29.738311079487175 -95.0376607619681) bank42666 +42667 POINT(30.464396890909647 -95.52504330184873) bank42667 +42668 POINT(30.675683602050814 -95.33996817917455) bank42668 +42669 POINT(28.941041647638606 -96.0408034426667) bank42669 +42670 POINT(30.109824236559874 -95.99185083699999) bank42670 +42671 POINT(29.95273048438926 -96.33352316798545) bank42671 +42672 POINT(30.255302680527763 -95.47663557631962) bank42672 +42673 POINT(30.661917816632673 -94.48053598075155) bank42673 +42674 POINT(29.35286616253052 -95.83215023907384) bank42674 +42675 POINT(30.160684159209264 -96.02656238125539) bank42675 +42676 POINT(30.1351165734262 -96.20418326684359) bank42676 +42677 POINT(28.864345309464852 -95.62036164110677) bank42677 +42678 POINT(29.475668151869254 -94.70374345237572) bank42678 +42679 POINT(30.26206496541861 -95.37164568864127) bank42679 +42680 POINT(30.711517666864207 -95.61375769960974) bank42680 +42681 POINT(28.79632395361366 -95.23375233964504) bank42681 +42682 POINT(30.668852209940738 -95.67917004434483) bank42682 +42683 POINT(29.12640683767739 -96.05645570138034) bank42683 +42684 POINT(29.295060932665358 -95.5681632946568) bank42684 +42685 POINT(29.75107264963239 -96.05489054559813) bank42685 +42686 POINT(29.83883307488483 -94.87647770871943) bank42686 +42687 POINT(29.15439973910593 -96.34998210983306) bank42687 +42688 POINT(30.09199320106378 -94.48646231251449) bank42688 +42689 POINT(30.017518269333337 -94.6444325999525) bank42689 +42690 POINT(29.833888347478663 -95.4727169246421) bank42690 +42691 POINT(29.631353521366083 -94.86667608723215) bank42691 +42692 POINT(30.523388542867075 -95.01424338489548) bank42692 +42693 POINT(28.81679526640566 -95.13346480919927) bank42693 +42694 POINT(30.475179775830767 -95.14058536426401) bank42694 +42695 POINT(30.054148247136066 -95.66314466628913) bank42695 +42696 POINT(28.91638877339195 -95.1243396852627) bank42696 +42697 POINT(30.708565899362135 -95.19727963533883) bank42697 +42698 POINT(30.196307111333034 -94.48467657302419) bank42698 +42699 POINT(29.71538510142981 -94.53858475735737) bank42699 +42700 POINT(28.79916306088091 -95.52302412901314) bank42700 +42701 POINT(28.78762417386036 -95.46995472984705) bank42701 +42702 POINT(30.38857290562695 -96.35065644926037) bank42702 +42703 POINT(30.533971119941945 -95.51342619171515) bank42703 +42704 POINT(29.44601383397772 -95.68050313387292) bank42704 +42705 POINT(30.13796148010152 -94.78536564272724) bank42705 +42706 POINT(28.903202395570297 -95.21232827573505) bank42706 +42707 POINT(29.59150154685062 -96.16146988852715) bank42707 +42708 POINT(29.661898951377154 -95.01924853563585) bank42708 +42709 POINT(30.73916907708598 -94.84398291343985) bank42709 +42710 POINT(30.580949385588887 -95.87303612395198) bank42710 +42711 POINT(29.224218562909662 -94.8953242117911) bank42711 +42712 POINT(28.95604956138072 -94.66141070312955) bank42712 +42713 POINT(30.342866991479745 -95.45325725310174) bank42713 +42714 POINT(28.901458154951655 -95.60596753163048) bank42714 +42715 POINT(28.807023024206106 -94.60022315653283) bank42715 +42716 POINT(28.97119549010885 -96.18676475365368) bank42716 +42717 POINT(28.871885576795094 -95.75631619967251) bank42717 +42718 POINT(30.70779071695087 -95.54851754287219) bank42718 +42719 POINT(29.589476942669783 -96.00383167836391) bank42719 +42720 POINT(28.871179334020162 -95.13012546448894) bank42720 +42721 POINT(29.73902486253708 -96.16135214871437) bank42721 +42722 POINT(30.232925119153897 -95.4991159594235) bank42722 +42723 POINT(28.961086323926544 -94.54328371725835) bank42723 +42724 POINT(28.948566033649183 -94.77243623985895) bank42724 +42725 POINT(28.8762842098979 -94.79223115712709) bank42725 +42726 POINT(29.39850111223661 -96.0123431855488) bank42726 +42727 POINT(29.29773785992395 -96.07204535539789) bank42727 +42728 POINT(29.20102370773431 -94.46527177905446) bank42728 +42729 POINT(30.06384853419491 -95.2736865267541) bank42729 +42730 POINT(30.52590908647246 -95.71624694247105) bank42730 +42731 POINT(29.393086303426813 -95.12882546515007) bank42731 +42732 POINT(29.89706045880127 -95.22909794643861) bank42732 +42733 POINT(30.66815144330707 -96.02051452628521) bank42733 +42734 POINT(29.968541148863366 -95.7287624903954) bank42734 +42735 POINT(28.913789358668122 -95.01969303382775) bank42735 +42736 POINT(29.033410747250066 -94.98984176510724) bank42736 +42737 POINT(29.503804532945715 -96.19799753131835) bank42737 +42738 POINT(29.709213726458206 -94.52231234943903) bank42738 +42739 POINT(29.534607287958718 -95.58403633585665) bank42739 +42740 POINT(30.121427960119668 -95.20801498516452) bank42740 +42741 POINT(28.839818317585884 -95.45941078055895) bank42741 +42742 POINT(30.689108046377502 -96.0548349290339) bank42742 +42743 POINT(28.96451802653798 -95.90777298277278) bank42743 +42744 POINT(30.077622903959025 -94.41012645628132) bank42744 +42745 POINT(30.70948791683049 -95.30682612601639) bank42745 +42746 POINT(30.685910815128327 -94.58859933854899) bank42746 +42747 POINT(29.12276115502953 -96.22566035042958) bank42747 +42748 POINT(28.83913854512851 -94.42817339381237) bank42748 +42749 POINT(29.86636680382531 -96.3029534665001) bank42749 +42750 POINT(29.017763302157377 -94.78675618117133) bank42750 +42751 POINT(29.691364473770932 -95.73107242072199) bank42751 +42752 POINT(30.078386623192205 -95.00395599591184) bank42752 +42753 POINT(30.12563748725425 -95.91300981465147) bank42753 +42754 POINT(29.661680810046846 -95.8703530457591) bank42754 +42755 POINT(29.613297879844733 -94.89099650236956) bank42755 +42756 POINT(29.2667078523791 -95.50009480797422) bank42756 +42757 POINT(28.944597230622 -94.72457619395375) bank42757 +42758 POINT(30.57158612977845 -95.72642322645503) bank42758 +42759 POINT(29.241536487646545 -96.0743950419568) bank42759 +42760 POINT(29.446229100011507 -95.18739605406878) bank42760 +42761 POINT(30.17406087819694 -95.57188265563077) bank42761 +42762 POINT(29.25876184814326 -95.22726760195337) bank42762 +42763 POINT(29.213651274798302 -95.1181917461075) bank42763 +42764 POINT(29.456788382251293 -96.31858596914479) bank42764 +42765 POINT(29.84608538373732 -94.91156432237383) bank42765 +42766 POINT(30.433347398377997 -95.20968242261492) bank42766 +42767 POINT(30.603628350625407 -96.15580092363078) bank42767 +42768 POINT(29.04944080743091 -95.83229951392802) bank42768 +42769 POINT(30.613778360843657 -95.08375733864666) bank42769 +42770 POINT(28.780775112643305 -96.28415835667704) bank42770 +42771 POINT(29.53467704603476 -94.4296955437544) bank42771 +42772 POINT(29.41676049353953 -94.56069749042767) bank42772 +42773 POINT(30.634634390720198 -94.92922427260048) bank42773 +42774 POINT(29.260403025580917 -95.25233786809063) bank42774 +42775 POINT(30.405750261262124 -94.53560006008203) bank42775 +42776 POINT(29.21561731585133 -94.92617395734234) bank42776 +42777 POINT(30.254253299539595 -95.16615765228082) bank42777 +42778 POINT(30.173216110694913 -95.59045757811008) bank42778 +42779 POINT(29.407634288120608 -95.02466036592361) bank42779 +42780 POINT(29.439154008096175 -96.2380553951794) bank42780 +42781 POINT(29.241146203567965 -95.213635752195) bank42781 +42782 POINT(30.40405423251681 -94.6045665667186) bank42782 +42783 POINT(30.516343389678855 -96.08891819719605) bank42783 +42784 POINT(30.034986714077533 -94.37907241861645) bank42784 +42785 POINT(30.02333204462985 -94.56018319867994) bank42785 +42786 POINT(29.697119806469782 -96.1939500589951) bank42786 +42787 POINT(29.76210874309843 -94.97059630025973) bank42787 +42788 POINT(29.19814200957484 -96.0039663438732) bank42788 +42789 POINT(30.10737260411778 -94.5156165077327) bank42789 +42790 POINT(29.366416162885358 -95.47284079501466) bank42790 +42791 POINT(30.255621726057253 -95.8527209845297) bank42791 +42792 POINT(30.300549343293316 -95.40410040521158) bank42792 +42793 POINT(29.40691747282308 -94.8036340190161) bank42793 +42794 POINT(29.13963053084537 -94.37157459624498) bank42794 +42795 POINT(29.601793819655633 -95.0284518921899) bank42795 +42796 POINT(29.601522719132312 -94.6692045101317) bank42796 +42797 POINT(29.582784144691853 -95.2870124416239) bank42797 +42798 POINT(30.01829411949149 -95.9059189026645) bank42798 +42799 POINT(29.717266931814205 -94.74471131278884) bank42799 +42800 POINT(29.275846117487458 -95.34299001942645) bank42800 +42801 POINT(29.73441180134602 -96.23834812698247) bank42801 +42802 POINT(29.556246030779402 -95.76392978824516) bank42802 +42803 POINT(28.998562907148287 -95.73425347878315) bank42803 +42804 POINT(28.85610822065178 -94.72387083590382) bank42804 +42805 POINT(30.174826581621733 -94.41926676285293) bank42805 +42806 POINT(29.59833663459675 -95.62503264279461) bank42806 +42807 POINT(30.159108848360066 -95.33792540856076) bank42807 +42808 POINT(29.617230851124834 -95.84879713311238) bank42808 +42809 POINT(28.87643041792303 -94.50524925035931) bank42809 +42810 POINT(29.6533933459254 -94.41404137663446) bank42810 +42811 POINT(29.333002529413477 -96.2221580143984) bank42811 +42812 POINT(30.37241033738795 -94.96792663154625) bank42812 +42813 POINT(29.978554541477713 -96.08531573284952) bank42813 +42814 POINT(28.992744376069872 -94.52122013306571) bank42814 +42815 POINT(30.404539932208404 -94.56499030759409) bank42815 +42816 POINT(30.452077337910982 -94.96194171554625) bank42816 +42817 POINT(30.104732071933757 -95.09846987421082) bank42817 +42818 POINT(30.306676484193236 -96.0204474519638) bank42818 +42819 POINT(29.774227231592526 -96.2313852065203) bank42819 +42820 POINT(30.47911767739635 -96.28029219367313) bank42820 +42821 POINT(29.989743190870673 -94.9196999619392) bank42821 +42822 POINT(28.990831348569216 -95.6965303018917) bank42822 +42823 POINT(29.026522784770126 -95.28680135318953) bank42823 +42824 POINT(29.53375638984349 -95.37269315364153) bank42824 +42825 POINT(29.804314859824363 -94.59335587169343) bank42825 +42826 POINT(29.147466946004588 -96.3364629091025) bank42826 +42827 POINT(28.98150471698906 -95.31986010791316) bank42827 +42828 POINT(29.48910924640214 -96.00715271578105) bank42828 +42829 POINT(28.890010202954475 -95.33469545827515) bank42829 +42830 POINT(29.45808682465358 -94.50394665935023) bank42830 +42831 POINT(28.88796657000656 -95.18450690898182) bank42831 +42832 POINT(30.331244372229346 -96.07344248067447) bank42832 +42833 POINT(29.346459936707635 -94.63334042335917) bank42833 +42834 POINT(28.885709597508118 -95.74471262616224) bank42834 +42835 POINT(29.852714404821892 -96.27177849714603) bank42835 +42836 POINT(30.74031288660722 -96.31106304999845) bank42836 +42837 POINT(30.09455071711084 -94.80717483596736) bank42837 +42838 POINT(29.658500448132298 -95.61022294223817) bank42838 +42839 POINT(29.01777620804828 -94.70425213406313) bank42839 +42840 POINT(30.05711570839635 -95.95995219700727) bank42840 +42841 POINT(30.632525895659608 -95.728806500988) bank42841 +42842 POINT(30.278945285906335 -94.72103096449646) bank42842 +42843 POINT(28.77928961252191 -95.165577045464) bank42843 +42844 POINT(29.002890857354824 -94.78474109979857) bank42844 +42845 POINT(29.95085035110106 -94.68149242234698) bank42845 +42846 POINT(29.772405248856444 -94.96637127452388) bank42846 +42847 POINT(30.410172013603948 -94.57283824613805) bank42847 +42848 POINT(29.276347022242657 -94.90258120412061) bank42848 +42849 POINT(29.30945712602322 -95.64766437865951) bank42849 +42850 POINT(30.057170756852166 -96.0539533760442) bank42850 +42851 POINT(30.516860505761827 -94.5087313655815) bank42851 +42852 POINT(29.752351525540497 -94.45918248463347) bank42852 +42853 POINT(30.109454055762587 -95.2474821044419) bank42853 +42854 POINT(29.876241436740706 -94.82186659475263) bank42854 +42855 POINT(29.12970480669476 -95.47177881263612) bank42855 +42856 POINT(30.426448656747937 -94.50173875681739) bank42856 +42857 POINT(29.428901482113194 -94.47824760563135) bank42857 +42858 POINT(30.007273966603886 -95.94796677193828) bank42858 +42859 POINT(29.02628485543712 -95.8643164640203) bank42859 +42860 POINT(29.452249462905694 -95.96270108873378) bank42860 +42861 POINT(30.65543241889941 -96.02250226487409) bank42861 +42862 POINT(29.1012156663808 -96.25681479470548) bank42862 +42863 POINT(30.279356509255184 -94.9957715183827) bank42863 +42864 POINT(29.683477208759783 -95.94673674124269) bank42864 +42865 POINT(30.494791252484415 -95.6153272009555) bank42865 +42866 POINT(29.665608796110917 -94.63816883260414) bank42866 +42867 POINT(29.231812683233798 -96.08037166382725) bank42867 +42868 POINT(29.05734653871286 -95.66169508836681) bank42868 +42869 POINT(30.116049603962065 -95.97778862571074) bank42869 +42870 POINT(29.85650529262045 -94.4990081795311) bank42870 +42871 POINT(29.53067737974459 -94.8520039434179) bank42871 +42872 POINT(29.925462249728987 -96.09328584735727) bank42872 +42873 POINT(29.072577154438406 -95.31007291854249) bank42873 +42874 POINT(29.438453198814933 -95.40339356548039) bank42874 +42875 POINT(30.202871399729627 -96.01920871675316) bank42875 +42876 POINT(30.462429468478106 -96.284042303831) bank42876 +42877 POINT(30.3654797421072 -94.48619357431636) bank42877 +42878 POINT(30.696018032215314 -95.95009463921843) bank42878 +42879 POINT(29.48186138885082 -94.82993777525904) bank42879 +42880 POINT(29.336125019178862 -94.6222712279606) bank42880 +42881 POINT(29.025384532367895 -95.43259670396057) bank42881 +42882 POINT(29.741076593594638 -94.96800743517383) bank42882 +42883 POINT(29.558251101781707 -95.15296245518442) bank42883 +42884 POINT(29.04686223345515 -95.80238440714602) bank42884 +42885 POINT(30.729121113898714 -94.9146673805651) bank42885 +42886 POINT(29.975227443599596 -94.64428873407851) bank42886 +42887 POINT(29.1360725789829 -94.66922867784447) bank42887 +42888 POINT(29.998259777506206 -95.96695411732846) bank42888 +42889 POINT(29.873612437536224 -95.3686628497776) bank42889 +42890 POINT(29.811553863490875 -95.25028478538242) bank42890 +42891 POINT(30.26279437560511 -95.54876231818186) bank42891 +42892 POINT(29.642021555741476 -94.37194434687723) bank42892 +42893 POINT(29.236195224496413 -96.15201499841012) bank42893 +42894 POINT(28.999564425619518 -94.54144443051483) bank42894 +42895 POINT(29.120072609676853 -94.64609712963065) bank42895 +42896 POINT(29.319582129447415 -94.97914836702233) bank42896 +42897 POINT(29.6467569434051 -96.31917863679843) bank42897 +42898 POINT(29.54601197861417 -95.99329503335514) bank42898 +42899 POINT(29.892473047114613 -94.74765589522522) bank42899 +42900 POINT(30.41289349852391 -95.31629799219996) bank42900 +42901 POINT(30.434619446245712 -95.85670687000818) bank42901 +42902 POINT(30.38975545587228 -94.80666001491042) bank42902 +42903 POINT(28.936101802027196 -95.0301339924485) bank42903 +42904 POINT(30.436157122621072 -96.24686378783616) bank42904 +42905 POINT(30.37042529099679 -95.5624666410555) bank42905 +42906 POINT(29.5617558972774 -95.18783047724129) bank42906 +42907 POINT(29.587561929041765 -96.31268601784664) bank42907 +42908 POINT(29.42881384112111 -95.9627468758022) bank42908 +42909 POINT(29.803741302525616 -96.35335943315577) bank42909 +42910 POINT(30.136018585364987 -95.51682558127308) bank42910 +42911 POINT(29.823968236940985 -94.46131763194842) bank42911 +42912 POINT(28.795709008343735 -95.43276675257401) bank42912 +42913 POINT(29.753829323758286 -95.87257298197481) bank42913 +42914 POINT(29.55141687529697 -96.27168062754436) bank42914 +42915 POINT(28.969202339092064 -95.10234524922096) bank42915 +42916 POINT(29.06203618626151 -96.08957106916058) bank42916 +42917 POINT(30.73917144039901 -95.30201028763172) bank42917 +42918 POINT(29.641083112347022 -94.44398566191823) bank42918 +42919 POINT(30.219849667330415 -95.27787572143191) bank42919 +42920 POINT(29.342355002451967 -95.27940282593184) bank42920 +42921 POINT(30.122975999933555 -96.06701714003687) bank42921 +42922 POINT(29.27409050998712 -95.44424594199927) bank42922 +42923 POINT(30.683519183495815 -94.41576028935896) bank42923 +42924 POINT(29.53861647408705 -95.90756119422197) bank42924 +42925 POINT(29.26973062478871 -96.21832680328913) bank42925 +42926 POINT(29.20134300840001 -94.94589305603547) bank42926 +42927 POINT(29.08017290138816 -94.87897786055314) bank42927 +42928 POINT(29.525658964699705 -95.35027106976803) bank42928 +42929 POINT(30.176986200400176 -94.87337630208987) bank42929 +42930 POINT(30.06769547023964 -95.9308281076747) bank42930 +42931 POINT(30.592369919962003 -95.00129289309672) bank42931 +42932 POINT(30.272639250926613 -95.69335442627244) bank42932 +42933 POINT(29.241080916082332 -95.53556560645592) bank42933 +42934 POINT(29.009560404105265 -94.88384734272586) bank42934 +42935 POINT(29.484491319921855 -94.91756156879451) bank42935 +42936 POINT(30.6712838381815 -95.3256339839784) bank42936 +42937 POINT(29.565695384884265 -95.06616878461666) bank42937 +42938 POINT(29.451573930263258 -96.29919288376041) bank42938 +42939 POINT(28.87111108066641 -95.95003502875431) bank42939 +42940 POINT(30.379291367054762 -95.59742027544614) bank42940 +42941 POINT(29.925040214979095 -95.62703303626458) bank42941 +42942 POINT(30.623454501489043 -94.72601258912397) bank42942 +42943 POINT(29.806565387915906 -94.93436839725865) bank42943 +42944 POINT(29.038221638115612 -94.48842012483809) bank42944 +42945 POINT(30.686685670883357 -94.41445031462219) bank42945 +42946 POINT(29.999896512358532 -95.85737117254882) bank42946 +42947 POINT(29.334722675402958 -94.94621566366341) bank42947 +42948 POINT(29.218081374521468 -95.61108474543005) bank42948 +42949 POINT(29.505924224160708 -96.35916316226461) bank42949 +42950 POINT(30.586175114579625 -95.3832986081983) bank42950 +42951 POINT(30.461082581514063 -96.36809108651221) bank42951 +42952 POINT(29.043219318411335 -95.11850522638629) bank42952 +42953 POINT(29.401899481399173 -94.60650048310453) bank42953 +42954 POINT(30.40931130649375 -96.28881649031179) bank42954 +42955 POINT(28.7929817809461 -94.91202241131144) bank42955 +42956 POINT(30.187793391817564 -95.7909190858011) bank42956 +42957 POINT(30.558015457130928 -94.4621252104886) bank42957 +42958 POINT(30.040921285519474 -94.85698647774572) bank42958 +42959 POINT(29.826265217630755 -94.6202866044044) bank42959 +42960 POINT(30.531398457827105 -95.5711363358986) bank42960 +42961 POINT(29.32695657078141 -95.53729805521276) bank42961 +42962 POINT(29.374277699460073 -94.54682994816604) bank42962 +42963 POINT(30.167221232831565 -95.25614014921254) bank42963 +42964 POINT(30.144690451186992 -95.06506865369784) bank42964 +42965 POINT(29.553232003109553 -95.03796095095723) bank42965 +42966 POINT(29.756148710881906 -94.59244303219162) bank42966 +42967 POINT(29.03120783487682 -95.6518634590415) bank42967 +42968 POINT(29.628848502250264 -95.2275793435734) bank42968 +42969 POINT(30.57386495711724 -95.2739406215103) bank42969 +42970 POINT(28.839201464757846 -96.3013608944246) bank42970 +42971 POINT(29.34649352375466 -95.51236950309192) bank42971 +42972 POINT(28.999959040264187 -94.53174920571256) bank42972 +42973 POINT(28.89425106323631 -95.3983657702492) bank42973 +42974 POINT(29.447370245932717 -96.1647882481314) bank42974 +42975 POINT(29.152859990031885 -95.1762149478988) bank42975 +42976 POINT(29.049653687355963 -95.65021489727535) bank42976 +42977 POINT(29.87434586148489 -95.70184672459511) bank42977 +42978 POINT(30.436743377778374 -95.2904250174989) bank42978 +42979 POINT(28.935486017915 -95.22975947757148) bank42979 +42980 POINT(30.426062921773717 -94.60865492717438) bank42980 +42981 POINT(30.514816374800144 -94.79510463394294) bank42981 +42982 POINT(29.249241566935556 -95.69018788258799) bank42982 +42983 POINT(29.238902772661437 -94.68978419500651) bank42983 +42984 POINT(30.64506821645516 -95.27030782311388) bank42984 +42985 POINT(29.872962863948036 -94.91001526501782) bank42985 +42986 POINT(30.712360269712406 -95.88762131964322) bank42986 +42987 POINT(30.224326130247753 -95.0402576295256) bank42987 +42988 POINT(30.50741405898795 -96.14612088789156) bank42988 +42989 POINT(29.31668350267385 -95.06892034788252) bank42989 +42990 POINT(30.163054320467822 -96.1806124238473) bank42990 +42991 POINT(29.09097626323157 -94.49312603122995) bank42991 +42992 POINT(30.469923470309794 -94.67486909056547) bank42992 +42993 POINT(29.577106332149018 -96.17707255953395) bank42993 +42994 POINT(29.32696698090239 -95.20912782461184) bank42994 +42995 POINT(30.645580938254376 -94.69343838318052) bank42995 +42996 POINT(29.71757000484404 -95.52933875579511) bank42996 +42997 POINT(29.844116560414786 -94.48879610882837) bank42997 +42998 POINT(30.635312726336124 -94.6904717413738) bank42998 +42999 POINT(30.353656902982262 -95.79428566881103) bank42999 +43000 POINT(30.264858987378314 -94.60806892440517) bank43000 +43001 POINT(30.732129535800507 -95.82259606438822) bank43001 +43002 POINT(28.90697730045117 -95.90863200843052) bank43002 +43003 POINT(30.33516916279003 -95.34672532363967) bank43003 +43004 POINT(30.225104478626633 -95.22099575447974) bank43004 +43005 POINT(30.168966274528596 -95.08497549405052) bank43005 +43006 POINT(28.890310036484724 -94.45453807030144) bank43006 +43007 POINT(29.439722074465916 -95.838329336603) bank43007 +43008 POINT(29.493268853590273 -95.2575679329027) bank43008 +43009 POINT(29.722359303160538 -95.0678935250631) bank43009 +43010 POINT(29.81870866292327 -96.22803704873698) bank43010 +43011 POINT(29.639436119928746 -94.75676064041461) bank43011 +43012 POINT(29.02041239425542 -95.40204560669328) bank43012 +43013 POINT(30.346853117536106 -95.65238122163535) bank43013 +43014 POINT(29.026521970976837 -95.49728132457886) bank43014 +43015 POINT(30.401851398989955 -95.3501567996186) bank43015 +43016 POINT(29.699697829818984 -95.99147012897834) bank43016 +43017 POINT(28.89557385012523 -95.09149060584072) bank43017 +43018 POINT(29.32844758996759 -94.9855170607235) bank43018 +43019 POINT(30.6032802210172 -95.73102841761644) bank43019 +43020 POINT(30.593659127678684 -96.03905850642239) bank43020 +43021 POINT(30.336367636342995 -96.24956388632211) bank43021 +43022 POINT(30.671002286031772 -94.97736123418004) bank43022 +43023 POINT(28.850841478059316 -95.31935228134843) bank43023 +43024 POINT(29.1181630445013 -95.96843563400822) bank43024 +43025 POINT(30.346769328052247 -96.02748319976064) bank43025 +43026 POINT(29.320060856075866 -95.86636570818634) bank43026 +43027 POINT(30.46835637838156 -96.34913822657703) bank43027 +43028 POINT(29.49895624714635 -95.58534951706925) bank43028 +43029 POINT(29.99217519117243 -94.57646869679202) bank43029 +43030 POINT(30.74163981827608 -96.17648523912902) bank43030 +43031 POINT(28.93463969741155 -94.73220319111039) bank43031 +43032 POINT(30.626592444139458 -94.7989000218155) bank43032 +43033 POINT(29.978295427320656 -94.51887833759581) bank43033 +43034 POINT(29.309943752575776 -95.2690419431798) bank43034 +43035 POINT(29.08889955452623 -95.91023178477903) bank43035 +43036 POINT(30.248777728453483 -95.82159057421039) bank43036 +43037 POINT(30.07676920925934 -95.56546102957404) bank43037 +43038 POINT(29.412053762609748 -94.75523572049599) bank43038 +43039 POINT(29.984329113269208 -96.3526166742882) bank43039 +43040 POINT(30.23828088883333 -95.78595294992039) bank43040 +43041 POINT(30.27536326806152 -95.1259191248545) bank43041 +43042 POINT(30.218554639533217 -96.33763783502972) bank43042 +43043 POINT(30.075296527864957 -96.1723875555378) bank43043 +43044 POINT(30.22224514786009 -94.53661854969495) bank43044 +43045 POINT(30.416823253614023 -95.72724588119596) bank43045 +43046 POINT(29.55308580071434 -95.70223919168757) bank43046 +43047 POINT(29.888805572399917 -94.91104925411058) bank43047 +43048 POINT(30.133511750422365 -94.80317131156956) bank43048 +43049 POINT(28.99993218936571 -94.71120879466119) bank43049 +43050 POINT(29.539567666532125 -95.36870136594146) bank43050 +43051 POINT(29.242475159243376 -95.02378818566946) bank43051 +43052 POINT(30.478237297781618 -96.27949750965504) bank43052 +43053 POINT(29.281156300352997 -94.89665848781132) bank43053 +43054 POINT(30.50429076340344 -95.70500027303156) bank43054 +43055 POINT(29.175195883189485 -94.7815469127444) bank43055 +43056 POINT(29.727257355145273 -95.1215223950942) bank43056 +43057 POINT(28.85035661752254 -95.08981625843022) bank43057 +43058 POINT(29.221304510446288 -96.13295993925108) bank43058 +43059 POINT(29.90577473045524 -96.35561729141358) bank43059 +43060 POINT(30.39920029919706 -95.48900296672387) bank43060 +43061 POINT(30.42205168789494 -94.98222205917318) bank43061 +43062 POINT(29.68101645531703 -94.74355429652334) bank43062 +43063 POINT(30.615205785972346 -94.72947819173167) bank43063 +43064 POINT(30.25412689340451 -94.69270730130339) bank43064 +43065 POINT(29.397169887426536 -96.00407032376758) bank43065 +43066 POINT(29.447783971176296 -94.70151366043696) bank43066 +43067 POINT(30.25018564267045 -95.19981863170531) bank43067 +43068 POINT(29.7185818801309 -95.33730669640967) bank43068 +43069 POINT(30.73195050229746 -94.93051843505913) bank43069 +43070 POINT(29.718989875308978 -95.00948934637628) bank43070 +43071 POINT(29.024795134637845 -95.89444307864993) bank43071 +43072 POINT(30.741442676406297 -95.76071114838672) bank43072 +43073 POINT(30.06768963354614 -96.05353248067838) bank43073 +43074 POINT(30.491901770412255 -94.3737662858515) bank43074 +43075 POINT(29.792876442680747 -96.21625004202063) bank43075 +43076 POINT(29.7995472391433 -94.57801015607106) bank43076 +43077 POINT(28.938290195677936 -94.61817053905007) bank43077 +43078 POINT(29.421538112352344 -94.66279863414793) bank43078 +43079 POINT(29.331498877825045 -94.38607789477108) bank43079 +43080 POINT(28.783104959686813 -94.94039893852916) bank43080 +43081 POINT(29.40511809602972 -96.29222786778578) bank43081 +43082 POINT(30.737262766194593 -94.41758549627015) bank43082 +43083 POINT(29.214241821072406 -95.80664211570277) bank43083 +43084 POINT(30.114544658699216 -94.85199219908037) bank43084 +43085 POINT(30.149424159121086 -95.41656000135312) bank43085 +43086 POINT(29.988284286424044 -95.4876465621624) bank43086 +43087 POINT(29.82420653035812 -95.94930997383888) bank43087 +43088 POINT(29.338874976509683 -94.45256615250523) bank43088 +43089 POINT(30.53342879252812 -95.75528739965962) bank43089 +43090 POINT(30.603231446842702 -96.28992549844783) bank43090 +43091 POINT(29.09296630583495 -95.05293638849018) bank43091 +43092 POINT(28.825861833383186 -94.52598650953756) bank43092 +43093 POINT(29.899157784005542 -96.205955081939) bank43093 +43094 POINT(29.18795418961016 -96.10313205734926) bank43094 +43095 POINT(28.82634871467094 -94.46302315789684) bank43095 +43096 POINT(30.725173259317483 -95.47740664111578) bank43096 +43097 POINT(29.426072676222255 -96.13949294395779) bank43097 +43098 POINT(30.72003031743955 -95.62736156192922) bank43098 +43099 POINT(29.43709230281358 -96.30764116903325) bank43099 +43100 POINT(29.59554946377356 -96.04397947339685) bank43100 +43101 POINT(28.77412170060409 -95.63377875299108) bank43101 +43102 POINT(29.39240132202731 -95.47990510378116) bank43102 +43103 POINT(30.512538352688406 -96.22478906626988) bank43103 +43104 POINT(29.110134997777543 -94.94592836280209) bank43104 +43105 POINT(30.328196921126317 -95.22464950198365) bank43105 +43106 POINT(30.63386868101744 -96.20638608213648) bank43106 +43107 POINT(29.708551682172978 -95.54368176325956) bank43107 +43108 POINT(29.22136869603557 -94.47654298772753) bank43108 +43109 POINT(29.524057124443857 -94.70174453743273) bank43109 +43110 POINT(29.65351853678535 -95.91369424164169) bank43110 +43111 POINT(30.164699711844452 -94.73303626991917) bank43111 +43112 POINT(29.992115117425424 -95.35709659908626) bank43112 +43113 POINT(28.936833991839702 -94.43585468735348) bank43113 +43114 POINT(29.100068521369266 -95.1780534319688) bank43114 +43115 POINT(30.478878182869234 -96.15439763500133) bank43115 +43116 POINT(30.12276723956462 -95.3301984571132) bank43116 +43117 POINT(29.303536269087715 -94.54232758036801) bank43117 +43118 POINT(30.58744676015307 -94.87033106677443) bank43118 +43119 POINT(28.789623960559393 -95.61210951335582) bank43119 +43120 POINT(30.190128292882523 -95.28830473594626) bank43120 +43121 POINT(29.44261549034834 -95.60568436887877) bank43121 +43122 POINT(29.864924153549143 -95.58373662897668) bank43122 +43123 POINT(29.84600747107911 -94.77884304542106) bank43123 +43124 POINT(29.439931350757856 -94.75772782182842) bank43124 +43125 POINT(30.700300887587 -94.84025452827177) bank43125 +43126 POINT(29.47252640345428 -94.96699354494005) bank43126 +43127 POINT(28.851498789255093 -94.95524875075158) bank43127 +43128 POINT(30.39719920052067 -96.13679106754154) bank43128 +43129 POINT(29.5422906083432 -95.48873500649954) bank43129 +43130 POINT(30.351657206687083 -95.84512114032934) bank43130 +43131 POINT(29.397057248096104 -95.75299538044447) bank43131 +43132 POINT(28.952992685631088 -94.77000110742784) bank43132 +43133 POINT(28.894987167441553 -95.39471002823038) bank43133 +43134 POINT(30.171857640472982 -94.68367782786333) bank43134 +43135 POINT(30.65395441134627 -95.06636346816468) bank43135 +43136 POINT(30.25311244376709 -94.41407543421774) bank43136 +43137 POINT(30.259139374091976 -94.56592769204937) bank43137 +43138 POINT(29.41240653376219 -94.64416250028124) bank43138 +43139 POINT(29.995505287760455 -94.52140958526421) bank43139 +43140 POINT(29.205539632309993 -94.92787250267314) bank43140 +43141 POINT(29.040156104788892 -95.35711210499045) bank43141 +43142 POINT(29.23512555262406 -95.03240944788053) bank43142 +43143 POINT(29.59212533125477 -95.98402059086537) bank43143 +43144 POINT(30.443511152502744 -95.41898275434697) bank43144 +43145 POINT(29.467725213023208 -94.88455138640033) bank43145 +43146 POINT(30.303404994934738 -95.98997263069785) bank43146 +43147 POINT(29.46272442931544 -94.612791917447) bank43147 +43148 POINT(29.293508493708725 -94.40631697564045) bank43148 +43149 POINT(30.378322845295813 -95.79404994080912) bank43149 +43150 POINT(30.5781252013067 -94.5025766942994) bank43150 +43151 POINT(28.87887363376156 -94.81435967652405) bank43151 +43152 POINT(30.524526134399913 -95.16420924169476) bank43152 +43153 POINT(30.398794660237616 -95.73122010183481) bank43153 +43154 POINT(29.824442007938053 -95.55154383419598) bank43154 +43155 POINT(30.549503234771617 -94.81931687988094) bank43155 +43156 POINT(30.475542969244774 -96.30862507659364) bank43156 +43157 POINT(29.151528909274184 -95.40422313516899) bank43157 +43158 POINT(30.716381844197972 -94.99064030090845) bank43158 +43159 POINT(30.466201717294002 -95.09391420467077) bank43159 +43160 POINT(28.85196999405985 -95.96839442882101) bank43160 +43161 POINT(29.82836915016931 -95.65153394381865) bank43161 +43162 POINT(28.989442437884353 -96.15887651136009) bank43162 +43163 POINT(29.995232107757754 -95.05280859866252) bank43163 +43164 POINT(29.437796172960805 -94.62364601272209) bank43164 +43165 POINT(29.594556176675933 -94.75505440022118) bank43165 +43166 POINT(30.04906782943573 -96.1895555733864) bank43166 +43167 POINT(29.847852101244882 -95.34700819333864) bank43167 +43168 POINT(30.096245537897335 -95.7769258441277) bank43168 +43169 POINT(30.715323199666937 -94.7501901129105) bank43169 +43170 POINT(29.929647162792755 -95.09622090032903) bank43170 +43171 POINT(28.975718408845275 -96.21016956604744) bank43171 +43172 POINT(29.497924881105675 -95.98014054444266) bank43172 +43173 POINT(30.284990492424168 -94.63325659573471) bank43173 +43174 POINT(28.941657570414918 -94.76697537127224) bank43174 +43175 POINT(30.505402587486667 -96.0901027996726) bank43175 +43176 POINT(30.58479405966879 -96.30489610986896) bank43176 +43177 POINT(30.700728565338565 -94.67121266896031) bank43177 +43178 POINT(30.091946856338282 -95.38517685952597) bank43178 +43179 POINT(29.3959911558532 -96.3225985351664) bank43179 +43180 POINT(30.41126837187382 -95.47922726356146) bank43180 +43181 POINT(29.118173333906668 -96.16761452517298) bank43181 +43182 POINT(30.4813693740575 -96.22342561553712) bank43182 +43183 POINT(30.07479890833617 -95.0449313326424) bank43183 +43184 POINT(28.99690214944184 -96.27154990810969) bank43184 +43185 POINT(30.01374962871835 -95.47319724167124) bank43185 +43186 POINT(29.97659296008528 -96.3224967714265) bank43186 +43187 POINT(29.027855731109867 -96.23796742717568) bank43187 +43188 POINT(30.406198597114606 -94.41489293087166) bank43188 +43189 POINT(29.371889494066934 -94.96090810104302) bank43189 +43190 POINT(29.063364691838437 -95.66984232521577) bank43190 +43191 POINT(29.14120967156229 -94.9361050979496) bank43191 +43192 POINT(30.200579303215257 -95.46861528145303) bank43192 +43193 POINT(30.11349102200701 -95.73226873849214) bank43193 +43194 POINT(29.144380309511106 -95.33205539528655) bank43194 +43195 POINT(28.883515333894017 -95.4968115783933) bank43195 +43196 POINT(30.118094948473903 -96.0316214469237) bank43196 +43197 POINT(29.69827933499892 -95.35004494919077) bank43197 +43198 POINT(29.152095018903463 -96.33103784782386) bank43198 +43199 POINT(30.203184799080002 -95.21956875145536) bank43199 +43200 POINT(29.987254849607844 -95.63434800803803) bank43200 +43201 POINT(29.71458838258161 -95.9026684536486) bank43201 +43202 POINT(29.537830294195984 -95.0080187801887) bank43202 +43203 POINT(30.36561228064026 -94.6337534674882) bank43203 +43204 POINT(30.184724897629863 -95.35459941606447) bank43204 +43205 POINT(28.871120305792687 -95.84547197732616) bank43205 +43206 POINT(29.681391144689368 -96.3331012317858) bank43206 +43207 POINT(29.47865551509212 -95.25353701151812) bank43207 +43208 POINT(29.705103476541424 -96.19057677663253) bank43208 +43209 POINT(30.01515802089783 -94.63320128252934) bank43209 +43210 POINT(29.8867051617353 -95.944394966705) bank43210 +43211 POINT(29.36570842850525 -94.84313314411459) bank43211 +43212 POINT(29.868273683895417 -94.53461595613464) bank43212 +43213 POINT(28.8330168874158 -94.87737691481529) bank43213 +43214 POINT(30.022254865586458 -94.78312045690797) bank43214 +43215 POINT(30.544284290285773 -95.5036682247056) bank43215 +43216 POINT(29.02121988030107 -95.8491567600419) bank43216 +43217 POINT(29.059816314199097 -94.98579680695255) bank43217 +43218 POINT(30.51842000801078 -96.08044008613345) bank43218 +43219 POINT(29.224369460862405 -95.26691026143158) bank43219 +43220 POINT(29.76684718825303 -95.01264563209111) bank43220 +43221 POINT(28.814441076870654 -95.82561037636117) bank43221 +43222 POINT(30.058706742664114 -96.01208605958064) bank43222 +43223 POINT(30.64784672266262 -96.17306436940314) bank43223 +43224 POINT(29.741442625972443 -96.24940472009231) bank43224 +43225 POINT(29.01189846285665 -95.84799707538376) bank43225 +43226 POINT(29.594336118485 -95.66742594101333) bank43226 +43227 POINT(29.67219758220415 -96.2513527985286) bank43227 +43228 POINT(30.715260545410047 -95.63506096751512) bank43228 +43229 POINT(30.48204833406051 -95.78114795662991) bank43229 +43230 POINT(29.153034805934332 -94.50814285679837) bank43230 +43231 POINT(29.836749621129275 -95.19587652303682) bank43231 +43232 POINT(29.45313165769187 -95.48004644458601) bank43232 +43233 POINT(29.62946168311491 -94.73870003006077) bank43233 +43234 POINT(29.177437508183854 -94.78318087285611) bank43234 +43235 POINT(30.137888835352467 -94.93601902913602) bank43235 +43236 POINT(30.69618196048608 -95.16285117910736) bank43236 +43237 POINT(30.25174768595977 -95.39273190326419) bank43237 +43238 POINT(29.368909413883458 -95.77154833922614) bank43238 +43239 POINT(30.23139831915183 -96.27242166622867) bank43239 +43240 POINT(28.946145227063404 -96.0517278181138) bank43240 +43241 POINT(29.219280433637067 -95.57611142878427) bank43241 +43242 POINT(28.773826641780996 -95.00686371210011) bank43242 +43243 POINT(30.759579317149733 -94.96362378237643) bank43243 +43244 POINT(29.251361329606485 -95.43708917935636) bank43244 +43245 POINT(28.92812962603309 -96.02183883191805) bank43245 +43246 POINT(30.411876957504983 -95.84547127384211) bank43246 +43247 POINT(29.442508956494112 -95.74287166439781) bank43247 +43248 POINT(29.484221643301794 -94.95624645888459) bank43248 +43249 POINT(29.77993301559724 -94.71738222036012) bank43249 +43250 POINT(29.20352090071767 -94.44783727484763) bank43250 +43251 POINT(29.00284964973278 -94.91554869086445) bank43251 +43252 POINT(29.025118419298835 -96.09693798063971) bank43252 +43253 POINT(29.281214025855807 -94.81324844910785) bank43253 +43254 POINT(29.955284382084706 -95.7546742776064) bank43254 +43255 POINT(28.823591793527303 -94.78787285821814) bank43255 +43256 POINT(29.86449568998555 -95.89394405033413) bank43256 +43257 POINT(30.292427950993716 -95.2137468971157) bank43257 +43258 POINT(29.575737999968624 -94.38328578237171) bank43258 +43259 POINT(29.27388357601968 -95.19335563992115) bank43259 +43260 POINT(29.91113394132181 -96.11900560599938) bank43260 +43261 POINT(30.258372817567444 -95.33350239967235) bank43261 +43262 POINT(28.847785004622228 -95.07352921900943) bank43262 +43263 POINT(29.889639514944733 -94.54303533329455) bank43263 +43264 POINT(28.865542808363713 -94.900732446009) bank43264 +43265 POINT(29.558341444588446 -95.15152233309496) bank43265 +43266 POINT(29.07200106150039 -95.55530330697066) bank43266 +43267 POINT(28.865781506479802 -94.42931947907606) bank43267 +43268 POINT(30.14941122919963 -94.71684533798039) bank43268 +43269 POINT(30.303835789323553 -95.93988447145666) bank43269 +43270 POINT(30.59479052731639 -94.52283260378155) bank43270 +43271 POINT(30.592109638715094 -95.83211153898449) bank43271 +43272 POINT(28.811021140344355 -95.08838835047644) bank43272 +43273 POINT(30.25535578718972 -95.21718800545104) bank43273 +43274 POINT(28.903881450740244 -96.11634610751786) bank43274 +43275 POINT(30.661150293567943 -94.45385337460733) bank43275 +43276 POINT(30.43293623316492 -94.82959855036643) bank43276 +43277 POINT(30.416195409601457 -96.1966333345138) bank43277 +43278 POINT(29.83706657431764 -94.40028993161903) bank43278 +43279 POINT(30.445549345311964 -94.64762304795045) bank43279 +43280 POINT(29.08277862088581 -95.96280399478397) bank43280 +43281 POINT(29.42492669661946 -95.73466723910256) bank43281 +43282 POINT(30.690738027691655 -95.96724884030054) bank43282 +43283 POINT(30.71305626344586 -95.84336461681427) bank43283 +43284 POINT(29.475603387367244 -94.63428378267294) bank43284 +43285 POINT(28.860494208665894 -94.40297500829377) bank43285 +43286 POINT(29.985874433251034 -95.23805303900446) bank43286 +43287 POINT(28.875753312988397 -95.68908419995448) bank43287 +43288 POINT(29.711007154507694 -95.22497634762384) bank43288 +43289 POINT(30.60489649962205 -95.38899524307611) bank43289 +43290 POINT(30.09943829068719 -94.94845328230373) bank43290 +43291 POINT(28.797334423471103 -94.42286561877346) bank43291 +43292 POINT(28.858885315203455 -94.61272611111379) bank43292 +43293 POINT(30.647588827619852 -94.79916292862112) bank43293 +43294 POINT(28.895897395880468 -94.4099683841681) bank43294 +43295 POINT(29.30514719940961 -95.85494794744277) bank43295 +43296 POINT(30.55018237592681 -94.88670239809596) bank43296 +43297 POINT(29.165261189201892 -96.22599234412503) bank43297 +43298 POINT(29.45214116064185 -94.57698605947883) bank43298 +43299 POINT(30.44105083251445 -95.13726402741332) bank43299 +43300 POINT(29.101027562482543 -96.30041121641965) bank43300 +43301 POINT(30.515595737887157 -94.5238750094441) bank43301 +43302 POINT(29.768242926389185 -94.74897648225479) bank43302 +43303 POINT(29.25642714776415 -96.21828812512761) bank43303 +43304 POINT(29.705581736779944 -94.8741690485042) bank43304 +43305 POINT(30.440440223235406 -95.82859470236457) bank43305 +43306 POINT(30.73239512905635 -95.25313505866929) bank43306 +43307 POINT(30.180245315939356 -95.86534599369048) bank43307 +43308 POINT(29.701485579300915 -95.59391414231779) bank43308 +43309 POINT(29.24988598752585 -95.80162882839343) bank43309 +43310 POINT(28.8150772933334 -95.30022766177302) bank43310 +43311 POINT(30.471262952074746 -96.07167993561133) bank43311 +43312 POINT(29.209457099484048 -94.78309772210153) bank43312 +43313 POINT(30.46691648753723 -94.78961352396823) bank43313 +43314 POINT(30.180697098501952 -94.5412654599053) bank43314 +43315 POINT(29.93987440800926 -95.68734200159037) bank43315 +43316 POINT(28.954061836365593 -95.43567094796424) bank43316 +43317 POINT(30.063792249935723 -95.29413457458809) bank43317 +43318 POINT(30.097941532081933 -94.95980440444326) bank43318 +43319 POINT(30.408486780656578 -95.84392351095524) bank43319 +43320 POINT(30.188730279804517 -94.78586501240515) bank43320 +43321 POINT(30.33944363028875 -96.0259076509194) bank43321 +43322 POINT(29.241291997120417 -94.66985239435176) bank43322 +43323 POINT(29.15658452310184 -96.23024391390679) bank43323 +43324 POINT(30.007150133156816 -96.00792953246803) bank43324 +43325 POINT(29.145215897942588 -95.42075116116561) bank43325 +43326 POINT(28.98206759498776 -95.97531365389418) bank43326 +43327 POINT(30.5598182467196 -95.5711646765845) bank43327 +43328 POINT(29.817761396139666 -95.96977837075707) bank43328 +43329 POINT(30.578962073700872 -94.92074587765973) bank43329 +43330 POINT(30.75950850536177 -94.50398092935426) bank43330 +43331 POINT(29.558654482516385 -94.67328701581447) bank43331 +43332 POINT(30.64993395747865 -94.72364372922029) bank43332 +43333 POINT(29.82912000581819 -96.03002238033311) bank43333 +43334 POINT(29.491128404188682 -95.7115762198812) bank43334 +43335 POINT(29.914777317899656 -94.67411676615777) bank43335 +43336 POINT(29.084397294305226 -95.29160594382805) bank43336 +43337 POINT(30.09978633979915 -95.94468427504663) bank43337 +43338 POINT(30.58794546388839 -96.07781522989931) bank43338 +43339 POINT(30.734621237672528 -95.92301082241801) bank43339 +43340 POINT(29.42171094527817 -95.5451958115798) bank43340 +43341 POINT(29.690482900702737 -94.38803266813373) bank43341 +43342 POINT(30.289291082137165 -95.66583380825715) bank43342 +43343 POINT(29.718044025751183 -95.3476447981024) bank43343 +43344 POINT(29.99970473688936 -96.33302426000046) bank43344 +43345 POINT(28.999944443315727 -95.50131898401587) bank43345 +43346 POINT(29.860944352208772 -94.53455429390897) bank43346 +43347 POINT(29.66836669417698 -95.67713649967783) bank43347 +43348 POINT(29.39941030111886 -96.24604879213003) bank43348 +43349 POINT(29.567655234334197 -95.58063533361599) bank43349 +43350 POINT(29.946329785550034 -94.73129544307552) bank43350 +43351 POINT(30.35467541887973 -95.42770748580865) bank43351 +43352 POINT(29.596173854352582 -94.84645421683373) bank43352 +43353 POINT(29.005506166688637 -95.52188120271296) bank43353 +43354 POINT(29.33127367577328 -95.25928595900098) bank43354 +43355 POINT(28.811082053740037 -95.76761654182344) bank43355 +43356 POINT(30.430086841137875 -95.65284020834476) bank43356 +43357 POINT(30.24582994165376 -96.22064566694996) bank43357 +43358 POINT(30.38746903224435 -96.04431251289537) bank43358 +43359 POINT(29.201726796344648 -95.31342349817496) bank43359 +43360 POINT(29.558005042050056 -94.98850501807166) bank43360 +43361 POINT(30.31966439854459 -94.49885385544448) bank43361 +43362 POINT(29.67294273927312 -95.41948077006037) bank43362 +43363 POINT(29.598453915526733 -95.67264382788112) bank43363 +43364 POINT(30.689418131004675 -96.04527295054689) bank43364 +43365 POINT(29.356595916805436 -94.89450239768375) bank43365 +43366 POINT(30.55435341131559 -94.89959510698493) bank43366 +43367 POINT(29.659763539761432 -94.47275554593098) bank43367 +43368 POINT(29.96629830718064 -96.20634623944261) bank43368 +43369 POINT(30.56955737267523 -95.98239177452156) bank43369 +43370 POINT(29.567778269724617 -95.79534654809675) bank43370 +43371 POINT(30.6596070723748 -94.49815596998839) bank43371 +43372 POINT(29.100075962484958 -95.11610954156804) bank43372 +43373 POINT(28.9809279001698 -95.15071288592081) bank43373 +43374 POINT(30.038707070893153 -94.77950530267799) bank43374 +43375 POINT(30.026205225604063 -95.45158221041726) bank43375 +43376 POINT(28.99260689588896 -95.18985080688968) bank43376 +43377 POINT(29.440049772829024 -94.79378079200275) bank43377 +43378 POINT(28.854207056592188 -95.65189214661527) bank43378 +43379 POINT(29.368700505797698 -94.75937051873129) bank43379 +43380 POINT(28.876911995097224 -96.09070838923489) bank43380 +43381 POINT(30.288109214793295 -94.43010107786253) bank43381 +43382 POINT(29.969954552954434 -94.96568639985486) bank43382 +43383 POINT(29.003784063117234 -95.31000249009446) bank43383 +43384 POINT(30.45414850666568 -94.73809470952911) bank43384 +43385 POINT(29.47476960469049 -95.16740387856044) bank43385 +43386 POINT(30.58386579377726 -95.99614888811529) bank43386 +43387 POINT(29.466320550111295 -95.36596781808623) bank43387 +43388 POINT(29.347297811322946 -94.66876406274322) bank43388 +43389 POINT(30.07193564968282 -95.83894374370117) bank43389 +43390 POINT(29.218184491833004 -96.33706745514091) bank43390 +43391 POINT(29.478345767381708 -96.31197706072068) bank43391 +43392 POINT(30.40481720226985 -95.78267653333631) bank43392 +43393 POINT(29.663117863061046 -95.38357801808955) bank43393 +43394 POINT(29.10967841044108 -95.38432858452138) bank43394 +43395 POINT(28.78107223371178 -96.33147501050226) bank43395 +43396 POINT(29.547966789279204 -96.27078039977776) bank43396 +43397 POINT(30.364959179466112 -95.28639700897841) bank43397 +43398 POINT(30.34996106286958 -95.59098358188166) bank43398 +43399 POINT(29.122642473692107 -94.61488447666304) bank43399 +43400 POINT(29.13771394171729 -95.27540366058591) bank43400 +43401 POINT(30.236726408412668 -95.50863039963721) bank43401 +43402 POINT(29.450643533057153 -96.19144044616104) bank43402 +43403 POINT(28.801580320578605 -94.67799336379338) bank43403 +43404 POINT(29.68873080903123 -96.3269719307075) bank43404 +43405 POINT(30.143664310029216 -96.24642718836904) bank43405 +43406 POINT(29.369063234174135 -94.40694257442222) bank43406 +43407 POINT(28.903315968707208 -95.53312113661913) bank43407 +43408 POINT(30.071477278988986 -94.67208871057771) bank43408 +43409 POINT(28.871637470941675 -94.66572759181318) bank43409 +43410 POINT(30.309279764482966 -95.8940523009139) bank43410 +43411 POINT(30.475185809712144 -94.42574517947313) bank43411 +43412 POINT(29.38401326192335 -96.16945595926663) bank43412 +43413 POINT(29.708726075798207 -95.94467622825691) bank43413 +43414 POINT(29.9849747461165 -94.97826583353599) bank43414 +43415 POINT(29.957953444747695 -94.40582908898764) bank43415 +43416 POINT(30.729285018111952 -96.2849263546381) bank43416 +43417 POINT(29.79586165020284 -94.96591042857969) bank43417 +43418 POINT(30.33447871522037 -94.94800814678844) bank43418 +43419 POINT(29.833026692396796 -95.66991153179902) bank43419 +43420 POINT(29.779174944480488 -96.00785689161533) bank43420 +43421 POINT(29.62089921595489 -95.83994600876817) bank43421 +43422 POINT(29.451718412383343 -96.16586196893844) bank43422 +43423 POINT(28.815316729986737 -94.4166599694405) bank43423 +43424 POINT(29.793508460010813 -94.54168704464945) bank43424 +43425 POINT(29.586983388319478 -94.7088043538637) bank43425 +43426 POINT(28.884123398261252 -95.6478984940711) bank43426 +43427 POINT(29.077971905090987 -94.56906946027797) bank43427 +43428 POINT(30.603291264926227 -95.46368278582665) bank43428 +43429 POINT(29.284209443154882 -95.88577979475927) bank43429 +43430 POINT(29.46112737737091 -96.18454932038892) bank43430 +43431 POINT(30.169766659425633 -95.10819513019052) bank43431 +43432 POINT(29.638919369100357 -95.61691993549096) bank43432 +43433 POINT(29.862342187955363 -95.19119357500243) bank43433 +43434 POINT(29.928273300187588 -95.92065911752586) bank43434 +43435 POINT(29.737142136455716 -94.41540712636302) bank43435 +43436 POINT(30.298918090790753 -95.75676260047268) bank43436 +43437 POINT(29.045141708245964 -94.62697457788818) bank43437 +43438 POINT(29.0936144031581 -95.28044126046548) bank43438 +43439 POINT(29.269738060884386 -94.62967686074154) bank43439 +43440 POINT(29.857557658093924 -94.90811716149531) bank43440 +43441 POINT(30.009216956727432 -96.08502747484137) bank43441 +43442 POINT(30.64593262111867 -95.64086503977113) bank43442 +43443 POINT(28.94724086254728 -95.75847054625251) bank43443 +43444 POINT(29.911885837229217 -94.46647857824597) bank43444 +43445 POINT(30.567434736747412 -95.15628183325917) bank43445 +43446 POINT(29.662709580466686 -94.40136575168628) bank43446 +43447 POINT(29.26304029032853 -96.20361499150088) bank43447 +43448 POINT(30.01580402867623 -95.33670881827258) bank43448 +43449 POINT(29.741397436615618 -95.11075425654596) bank43449 +43450 POINT(29.360369129922795 -95.17780032824673) bank43450 +43451 POINT(29.79059068439598 -95.07669028965738) bank43451 +43452 POINT(30.391318624725294 -94.6282966546189) bank43452 +43453 POINT(29.463244581183947 -94.48527043415974) bank43453 +43454 POINT(30.427741016261002 -94.66925635359134) bank43454 +43455 POINT(29.317322379107736 -94.67831990301283) bank43455 +43456 POINT(29.4881071189598 -95.84499282649648) bank43456 +43457 POINT(30.24084621999254 -95.78048597901345) bank43457 +43458 POINT(30.083372044782635 -94.60738068072767) bank43458 +43459 POINT(30.521148161528743 -95.90135671027616) bank43459 +43460 POINT(28.985572071749647 -95.04160889418914) bank43460 +43461 POINT(29.25675066377511 -95.66465040359317) bank43461 +43462 POINT(28.941967213628484 -95.03180733302462) bank43462 +43463 POINT(29.10398117175766 -94.73222215911845) bank43463 +43464 POINT(30.472471432502555 -95.55376027124612) bank43464 +43465 POINT(29.140351555772572 -96.28487501405286) bank43465 +43466 POINT(29.25639387591307 -95.74215558457259) bank43466 +43467 POINT(28.802227053101188 -94.932865155013) bank43467 +43468 POINT(30.528584910028176 -96.13067016830833) bank43468 +43469 POINT(30.39178525670388 -94.98197820554716) bank43469 +43470 POINT(30.28677208883896 -95.64310686955383) bank43470 +43471 POINT(30.718317776642962 -94.74880634870587) bank43471 +43472 POINT(30.528054042204413 -95.08328495900288) bank43472 +43473 POINT(29.73419750949039 -94.74735318043034) bank43473 +43474 POINT(28.88422860971949 -95.96746700005427) bank43474 +43475 POINT(29.050045117657085 -95.43409130064254) bank43475 +43476 POINT(30.459812628111816 -95.90245586526729) bank43476 +43477 POINT(29.600916636713322 -95.11818920342932) bank43477 +43478 POINT(30.64084528374539 -96.11860137977513) bank43478 +43479 POINT(29.01273739620612 -95.6227120036324) bank43479 +43480 POINT(30.690803121745354 -96.2723499895159) bank43480 +43481 POINT(29.09863163763132 -95.7101714147196) bank43481 +43482 POINT(29.21267038314227 -95.2824561165308) bank43482 +43483 POINT(29.911609876077648 -94.71000827382929) bank43483 +43484 POINT(30.735980512492056 -95.07594413179979) bank43484 +43485 POINT(30.348962032569023 -96.06194511244635) bank43485 +43486 POINT(30.051331134557604 -96.00559240565242) bank43486 +43487 POINT(30.746281122163165 -95.22976572442094) bank43487 +43488 POINT(28.7922181538288 -95.62836810590149) bank43488 +43489 POINT(29.762501592382137 -95.06717821004459) bank43489 +43490 POINT(29.406865356320466 -94.649020611444) bank43490 +43491 POINT(29.355940428437954 -94.73645938854054) bank43491 +43492 POINT(30.542874281329805 -96.07717781447714) bank43492 +43493 POINT(30.28950873088222 -94.40474856422097) bank43493 +43494 POINT(30.61331837307914 -95.6331284506926) bank43494 +43495 POINT(30.455364953706802 -96.33711108669556) bank43495 +43496 POINT(29.377018674237206 -96.17781852105371) bank43496 +43497 POINT(29.9184436126344 -94.40171869533378) bank43497 +43498 POINT(30.07331193749266 -96.30954964060729) bank43498 +43499 POINT(30.38589989820395 -95.83294051633713) bank43499 +43500 POINT(29.394702333409903 -94.5806693982429) bank43500 +43501 POINT(30.02408633634103 -95.2145576530697) bank43501 +43502 POINT(30.126589030213914 -95.7286255319615) bank43502 +43503 POINT(29.635204456458393 -94.52179397613126) bank43503 +43504 POINT(28.891717526675414 -94.9275995291621) bank43504 +43505 POINT(29.605905687199375 -96.08628007743452) bank43505 +43506 POINT(29.20317919083014 -95.74623941160617) bank43506 +43507 POINT(29.58076097936643 -95.94429099224014) bank43507 +43508 POINT(28.76938465853405 -94.56832307381413) bank43508 +43509 POINT(29.402323930702778 -95.81841983479141) bank43509 +43510 POINT(30.528688578545538 -95.77521474145765) bank43510 +43511 POINT(30.5562320325592 -95.01741898913289) bank43511 +43512 POINT(30.114281672061615 -94.5228552108788) bank43512 +43513 POINT(29.80289991236587 -94.64208544411571) bank43513 +43514 POINT(29.67518393679004 -94.52024411755295) bank43514 +43515 POINT(28.87338104319598 -96.0119978738536) bank43515 +43516 POINT(29.599484674751384 -96.24890553714361) bank43516 +43517 POINT(30.45618157816039 -96.0833899784108) bank43517 +43518 POINT(29.842847498633834 -95.65120974795026) bank43518 +43519 POINT(29.42612182760796 -94.8257442492891) bank43519 +43520 POINT(29.730756419120155 -95.5452562330473) bank43520 +43521 POINT(30.52356176929085 -96.18729863495673) bank43521 +43522 POINT(29.41569769886242 -94.85311862612063) bank43522 +43523 POINT(30.56820488638728 -94.8914811389409) bank43523 +43524 POINT(28.891368068173104 -96.19753246599198) bank43524 +43525 POINT(30.263937859450913 -94.99668060455936) bank43525 +43526 POINT(29.947760976527945 -94.37798296624037) bank43526 +43527 POINT(29.19456429910186 -94.53175013239469) bank43527 +43528 POINT(30.265774232120474 -94.5629418008742) bank43528 +43529 POINT(29.685260559437964 -95.4298222890429) bank43529 +43530 POINT(30.637314769880817 -96.0432723953871) bank43530 +43531 POINT(29.448267536294424 -96.15147673522712) bank43531 +43532 POINT(30.012837488629746 -94.72732446519568) bank43532 +43533 POINT(29.558707025116004 -95.84932397237023) bank43533 +43534 POINT(30.71103644374145 -94.89265766315539) bank43534 +43535 POINT(29.63477235992003 -95.29125342364308) bank43535 +43536 POINT(30.726023829577915 -94.47828371037224) bank43536 +43537 POINT(28.76493437007211 -95.52612389504633) bank43537 +43538 POINT(30.68039970130891 -95.17580827672869) bank43538 +43539 POINT(30.055225715895432 -95.55932729530532) bank43539 +43540 POINT(30.519901672157726 -95.14215551673944) bank43540 +43541 POINT(29.65973116786471 -95.34306807216) bank43541 +43542 POINT(28.936199241181097 -95.343922775181) bank43542 +43543 POINT(30.399443008512606 -95.86859070647138) bank43543 +43544 POINT(29.5997583177478 -95.43935415974832) bank43544 +43545 POINT(29.315797630943152 -94.62488395079632) bank43545 +43546 POINT(29.499768147393773 -94.64860532972885) bank43546 +43547 POINT(29.806721034072396 -95.31267314519604) bank43547 +43548 POINT(30.427820818259665 -96.10090567945518) bank43548 +43549 POINT(30.696242951720095 -95.84534058288402) bank43549 +43550 POINT(29.78290096922239 -95.85409009638413) bank43550 +43551 POINT(29.756042271931886 -94.477634996789) bank43551 +43552 POINT(30.634461312071537 -95.32746626399282) bank43552 +43553 POINT(29.103434006509534 -95.93095182614766) bank43553 +43554 POINT(29.440916692996318 -95.16734906891573) bank43554 +43555 POINT(30.25350762353365 -95.05478660898157) bank43555 +43556 POINT(29.759148009662102 -95.55164148254292) bank43556 +43557 POINT(28.832261103704084 -95.12021739556455) bank43557 +43558 POINT(30.61929042669748 -94.70723644490774) bank43558 +43559 POINT(29.117530708474565 -95.82493460594279) bank43559 +43560 POINT(30.691686032636305 -96.0385621902835) bank43560 +43561 POINT(29.408242990261744 -95.35773607830002) bank43561 +43562 POINT(29.36488539034032 -96.13686771231231) bank43562 +43563 POINT(29.837701832418755 -94.79169363525125) bank43563 +43564 POINT(30.087691058423875 -95.67713699350416) bank43564 +43565 POINT(29.811245976869426 -95.78603907073138) bank43565 +43566 POINT(29.38287198293523 -95.84225131501788) bank43566 +43567 POINT(30.7302003199295 -95.98854784235435) bank43567 +43568 POINT(29.54198253872425 -94.85276002740086) bank43568 +43569 POINT(29.25943517761046 -95.78300217006958) bank43569 +43570 POINT(29.452102751708765 -96.25886517244233) bank43570 +43571 POINT(29.571629568830993 -94.54101449976942) bank43571 +43572 POINT(28.83839833761027 -95.09273050250683) bank43572 +43573 POINT(29.020385329566153 -95.61679777596613) bank43573 +43574 POINT(29.80081905399226 -94.5975319939675) bank43574 +43575 POINT(30.10484950682696 -96.01893762568396) bank43575 +43576 POINT(29.943399285098703 -95.57128103340635) bank43576 +43577 POINT(30.33667217489267 -96.30170171940105) bank43577 +43578 POINT(30.42337187250793 -95.96619383919274) bank43578 +43579 POINT(30.107190302505387 -94.63628384815023) bank43579 +43580 POINT(29.19484548216016 -95.27396196937829) bank43580 +43581 POINT(29.339695292070157 -94.38570793194398) bank43581 +43582 POINT(29.413331810100082 -96.21727050270596) bank43582 +43583 POINT(29.04103586570047 -95.97422631133551) bank43583 +43584 POINT(30.272308852739663 -95.29326255195431) bank43584 +43585 POINT(30.48745294428035 -95.5156974644797) bank43585 +43586 POINT(30.077241329807183 -95.37378083678556) bank43586 +43587 POINT(30.499387716454752 -94.39062765770218) bank43587 +43588 POINT(29.15777387192047 -95.75134237754752) bank43588 +43589 POINT(29.296962999988086 -95.64965715297778) bank43589 +43590 POINT(29.49874250470657 -94.74515071849198) bank43590 +43591 POINT(29.29131370069168 -96.080425180635) bank43591 +43592 POINT(29.559053896580245 -95.81379046662099) bank43592 +43593 POINT(28.972446785035988 -95.65733418390822) bank43593 +43594 POINT(29.591668429751863 -96.31793985611864) bank43594 +43595 POINT(29.52229137667937 -95.73249762168828) bank43595 +43596 POINT(30.641115157488684 -95.89486129073461) bank43596 +43597 POINT(28.834408386471207 -94.91802957984628) bank43597 +43598 POINT(29.732308205596677 -95.87193489752927) bank43598 +43599 POINT(29.464094311831463 -96.31205172586733) bank43599 +43600 POINT(29.171646858266403 -95.02281297051657) bank43600 +43601 POINT(30.535390755929747 -94.74459184230561) bank43601 +43602 POINT(29.59430304809544 -96.06110267754693) bank43602 +43603 POINT(29.33925486929184 -95.11102575927065) bank43603 +43604 POINT(29.159064368076976 -94.95702651676865) bank43604 +43605 POINT(29.746166659862872 -96.2221248551827) bank43605 +43606 POINT(28.83539131022399 -96.3085971251068) bank43606 +43607 POINT(30.48218645066152 -95.87903417197357) bank43607 +43608 POINT(29.472169497792137 -94.83603107432965) bank43608 +43609 POINT(29.772303679471207 -94.76877832912956) bank43609 +43610 POINT(30.055224447000885 -94.5791891933204) bank43610 +43611 POINT(29.430240994561448 -95.8141289894396) bank43611 +43612 POINT(28.985188926409087 -95.52510204608747) bank43612 +43613 POINT(29.406084965280037 -96.2018943602479) bank43613 +43614 POINT(30.104141787087762 -94.67430305361387) bank43614 +43615 POINT(29.214653796759446 -94.48733443540647) bank43615 +43616 POINT(30.316087220095753 -94.66410223740804) bank43616 +43617 POINT(30.62412107085719 -95.26813987709818) bank43617 +43618 POINT(30.60271225738443 -95.58404490884455) bank43618 +43619 POINT(29.004198846872665 -95.46803688509628) bank43619 +43620 POINT(29.339870047677884 -96.0349608041965) bank43620 +43621 POINT(30.582963659199148 -95.68683027371691) bank43621 +43622 POINT(29.54050586835625 -95.28416670980515) bank43622 +43623 POINT(29.78873951599948 -94.52911889271707) bank43623 +43624 POINT(30.4700824916831 -94.99634060063326) bank43624 +43625 POINT(29.780356295168858 -94.71116574663333) bank43625 +43626 POINT(29.66958074317819 -95.0662450450882) bank43626 +43627 POINT(29.232110649002298 -94.71170678387206) bank43627 +43628 POINT(30.395738028790475 -95.91742786633854) bank43628 +43629 POINT(30.234447483022024 -95.40032191022948) bank43629 +43630 POINT(29.10832587492691 -94.93533934045976) bank43630 +43631 POINT(30.2800569777804 -95.0627760443388) bank43631 +43632 POINT(29.649325690706437 -95.69047824430821) bank43632 +43633 POINT(29.893118911473543 -96.10260768703134) bank43633 +43634 POINT(30.57257162800361 -95.1131157062363) bank43634 +43635 POINT(29.083721988151 -96.23474014976549) bank43635 +43636 POINT(29.051113699357508 -94.70876462193351) bank43636 +43637 POINT(30.71957679860891 -94.75961528154761) bank43637 +43638 POINT(29.064626683650815 -94.782047936714) bank43638 +43639 POINT(30.38028878210594 -95.71289491518083) bank43639 +43640 POINT(28.768818162053304 -95.07662039265462) bank43640 +43641 POINT(30.63008260426334 -95.97787793598874) bank43641 +43642 POINT(29.61927350444925 -94.48447998035816) bank43642 +43643 POINT(29.168021357595645 -95.00868058075554) bank43643 +43644 POINT(29.525549790046934 -95.17803382100541) bank43644 +43645 POINT(30.151887386377442 -95.81270116681) bank43645 +43646 POINT(30.46479307960431 -95.2452066667801) bank43646 +43647 POINT(29.85439735492254 -94.51084123568212) bank43647 +43648 POINT(29.026151523721996 -96.20074133347549) bank43648 +43649 POINT(29.70082070940309 -95.678052929403) bank43649 +43650 POINT(30.367540572818964 -95.9235836075178) bank43650 +43651 POINT(29.611008968813337 -95.28237608664213) bank43651 +43652 POINT(29.11341083475927 -95.34112441445428) bank43652 +43653 POINT(29.2801029534022 -96.35245041433846) bank43653 +43654 POINT(28.880654255878255 -95.77154740650691) bank43654 +43655 POINT(30.09976356628993 -96.25601758079893) bank43655 +43656 POINT(29.38415850689831 -95.33559162057307) bank43656 +43657 POINT(28.8881844257913 -96.057902945046) bank43657 +43658 POINT(28.88305984314681 -94.82366674246794) bank43658 +43659 POINT(30.426450331362002 -95.37620602784524) bank43659 +43660 POINT(29.479703652501755 -95.69241232490224) bank43660 +43661 POINT(30.025868618027367 -94.37313727688) bank43661 +43662 POINT(29.60021523101414 -95.74480397278775) bank43662 +43663 POINT(30.20422635812725 -95.20315353155675) bank43663 +43664 POINT(30.57127842781039 -96.32073503397878) bank43664 +43665 POINT(29.242620267503685 -94.44916249446513) bank43665 +43666 POINT(29.57366610883284 -94.99290782688871) bank43666 +43667 POINT(29.674011210889173 -94.38308137535304) bank43667 +43668 POINT(29.331139757417546 -96.22180693434838) bank43668 +43669 POINT(29.078664973145106 -94.94391407992214) bank43669 +43670 POINT(29.847435038939835 -95.58995543334267) bank43670 +43671 POINT(30.02245748960567 -95.16170081026453) bank43671 +43672 POINT(30.024765899652284 -94.63491734172355) bank43672 +43673 POINT(30.66867357739114 -94.54606536202854) bank43673 +43674 POINT(30.422895028234656 -95.45707964282894) bank43674 +43675 POINT(30.33140230929764 -94.6490462187668) bank43675 +43676 POINT(29.97943458284933 -94.92028028820268) bank43676 +43677 POINT(29.86401780645495 -94.55278690511258) bank43677 +43678 POINT(30.50803291185338 -95.4528858932804) bank43678 +43679 POINT(29.793324456959393 -95.32589584296336) bank43679 +43680 POINT(29.414701984363113 -95.4867637224551) bank43680 +43681 POINT(28.790313757920398 -94.7630858884815) bank43681 +43682 POINT(29.981038597324922 -95.02913881902333) bank43682 +43683 POINT(30.018046007896757 -95.83732795350724) bank43683 +43684 POINT(30.295358710355146 -95.2737390119084) bank43684 +43685 POINT(29.01253703592563 -95.35157165614834) bank43685 +43686 POINT(29.630853118930094 -94.72333236809312) bank43686 +43687 POINT(29.18389141115113 -94.60122208893627) bank43687 +43688 POINT(29.273717625174434 -94.71903252069697) bank43688 +43689 POINT(29.86205586939137 -95.1444381757799) bank43689 +43690 POINT(30.701821101214342 -95.82803298413555) bank43690 +43691 POINT(29.377512746057313 -96.01692437627644) bank43691 +43692 POINT(30.75178383663898 -94.93921269681098) bank43692 +43693 POINT(29.08025917803282 -94.91137511989012) bank43693 +43694 POINT(29.038002057479535 -96.16255227062324) bank43694 +43695 POINT(30.33417263122901 -94.44544170894456) bank43695 +43696 POINT(30.581507213537158 -94.86361849690411) bank43696 +43697 POINT(30.636821916420594 -94.78768099623548) bank43697 +43698 POINT(29.207471717187335 -95.50385391132978) bank43698 +43699 POINT(28.801858184373824 -95.68842920239777) bank43699 +43700 POINT(30.086178016176014 -95.51528243117886) bank43700 +43701 POINT(30.17756026345852 -95.41085011938588) bank43701 +43702 POINT(30.61317660164255 -95.53682284090542) bank43702 +43703 POINT(29.46978934498687 -96.34841880964406) bank43703 +43704 POINT(30.695074383265865 -95.30806239908163) bank43704 +43705 POINT(29.65774995014872 -95.4457088373889) bank43705 +43706 POINT(30.3049964266418 -95.69116333385917) bank43706 +43707 POINT(29.65246445663208 -96.33714073056393) bank43707 +43708 POINT(29.734679052567852 -96.22491917713103) bank43708 +43709 POINT(29.6181089720327 -95.92727715214582) bank43709 +43710 POINT(29.710110220920182 -94.57882680228964) bank43710 +43711 POINT(29.875826757596656 -94.83415876668215) bank43711 +43712 POINT(29.684651068433606 -94.60326598303212) bank43712 +43713 POINT(29.68809566682352 -95.64860529385628) bank43713 +43714 POINT(30.62787794330756 -95.31348764597713) bank43714 +43715 POINT(29.2494635760225 -94.7814414175817) bank43715 +43716 POINT(30.760395724543155 -94.55109921842107) bank43716 +43717 POINT(29.828230930572726 -94.77025185326863) bank43717 +43718 POINT(29.575718322940762 -95.3298765504587) bank43718 +43719 POINT(30.4163059833171 -94.47429249307359) bank43719 +43720 POINT(29.228214378112785 -94.89293520773182) bank43720 +43721 POINT(29.868634263252396 -95.13492778542702) bank43721 +43722 POINT(29.429609910402863 -95.97615398015701) bank43722 +43723 POINT(29.771154906140087 -94.70387040361142) bank43723 +43724 POINT(29.214760908046898 -94.95930424690253) bank43724 +43725 POINT(30.403664804483938 -95.78843650957465) bank43725 +43726 POINT(29.905490093792686 -94.56953019335663) bank43726 +43727 POINT(29.645591531379186 -95.02282384601051) bank43727 +43728 POINT(29.668188639506383 -95.60342337806664) bank43728 +43729 POINT(29.821313602745725 -94.74890192830084) bank43729 +43730 POINT(30.670269997303876 -96.10255152517662) bank43730 +43731 POINT(30.480211021704605 -95.26060194078383) bank43731 +43732 POINT(29.146241532079657 -94.91436286999803) bank43732 +43733 POINT(29.22383847193316 -95.07045456371925) bank43733 +43734 POINT(29.937207116519033 -95.71689594453572) bank43734 +43735 POINT(30.005897948234615 -94.40061254495062) bank43735 +43736 POINT(29.402492597950662 -96.10731713240983) bank43736 +43737 POINT(30.24157957414971 -94.78400207176107) bank43737 +43738 POINT(29.66071931737331 -95.3902964565275) bank43738 +43739 POINT(28.906603156401236 -95.70578552873931) bank43739 +43740 POINT(29.309583060960545 -95.36036531425094) bank43740 +43741 POINT(29.694186334395532 -96.15975580381054) bank43741 +43742 POINT(29.57187995748137 -94.93813619210759) bank43742 +43743 POINT(29.60832540223133 -94.69753206139542) bank43743 +43744 POINT(29.95055227895655 -95.11578494295063) bank43744 +43745 POINT(30.549841165703185 -95.49055856954881) bank43745 +43746 POINT(29.292743482662992 -94.88546904666647) bank43746 +43747 POINT(28.81729824055038 -96.28874235561949) bank43747 +43748 POINT(30.26886888641811 -94.44651599712239) bank43748 +43749 POINT(29.73193597988076 -94.51557220378834) bank43749 +43750 POINT(29.266770211943367 -96.21945494370193) bank43750 +43751 POINT(29.760563059591195 -95.49231064045262) bank43751 +43752 POINT(29.377069451450765 -95.05477366271046) bank43752 +43753 POINT(29.087250646581623 -95.01160080946366) bank43753 +43754 POINT(30.24362437240679 -95.59040445721658) bank43754 +43755 POINT(28.99754483302151 -95.22238275044496) bank43755 +43756 POINT(30.000594756960485 -94.5246395286372) bank43756 +43757 POINT(29.728858650872997 -94.41489164743656) bank43757 +43758 POINT(30.172722861488335 -95.81826159357783) bank43758 +43759 POINT(30.065190009367566 -96.10661644209287) bank43759 +43760 POINT(29.390561280382727 -95.68047674588144) bank43760 +43761 POINT(30.603539653154485 -95.96019907917648) bank43761 +43762 POINT(30.202410210402373 -95.68874228280272) bank43762 +43763 POINT(29.971245205282877 -96.35599720361083) bank43763 +43764 POINT(30.613620556958196 -94.44409450113463) bank43764 +43765 POINT(28.83781015009842 -94.8653595747584) bank43765 +43766 POINT(29.47774720649407 -95.32640882608003) bank43766 +43767 POINT(29.179238802919297 -94.83795746543822) bank43767 +43768 POINT(30.71339058141748 -95.78065872952796) bank43768 +43769 POINT(30.182625451020556 -95.22062572785475) bank43769 +43770 POINT(29.83385000033307 -95.63393322765404) bank43770 +43771 POINT(28.96684638166503 -96.3100069293532) bank43771 +43772 POINT(29.39872873453566 -95.88987776834831) bank43772 +43773 POINT(29.342633084128323 -95.55382854275392) bank43773 +43774 POINT(29.04499552382173 -94.37868368997071) bank43774 +43775 POINT(30.67187240268268 -94.49239070558893) bank43775 +43776 POINT(29.55101038805075 -95.26951033038557) bank43776 +43777 POINT(29.92704207602979 -94.73726493372043) bank43777 +43778 POINT(28.772520796834037 -95.40526019747188) bank43778 +43779 POINT(28.830582930314932 -95.89178242026266) bank43779 +43780 POINT(29.944602131785093 -95.74728330388946) bank43780 +43781 POINT(28.90808721891443 -95.93759225907941) bank43781 +43782 POINT(30.439050534157793 -94.8743825027605) bank43782 +43783 POINT(29.459493278158508 -95.58061341527113) bank43783 +43784 POINT(28.977792650056347 -96.32002776237174) bank43784 +43785 POINT(30.502640326116943 -96.22742845558581) bank43785 +43786 POINT(30.710720590216244 -94.62660499683096) bank43786 +43787 POINT(30.011322711123547 -95.4769995078333) bank43787 +43788 POINT(29.169608841875394 -96.36524635589382) bank43788 +43789 POINT(30.712100070509592 -94.76769247456897) bank43789 +43790 POINT(29.713204448611666 -95.65637448948564) bank43790 +43791 POINT(30.230671217667766 -96.22450261714562) bank43791 +43792 POINT(29.17038635816793 -95.17854295230227) bank43792 +43793 POINT(30.02594444122346 -95.31032222972246) bank43793 +43794 POINT(29.98361081282158 -95.75631874357714) bank43794 +43795 POINT(29.060849627841773 -94.5917215239466) bank43795 +43796 POINT(30.541470074032574 -95.96173004954495) bank43796 +43797 POINT(30.60027483915245 -95.49166039457025) bank43797 +43798 POINT(30.015419814953233 -94.90199735623456) bank43798 +43799 POINT(29.28063394869741 -95.47837521479995) bank43799 +43800 POINT(30.280685078755834 -96.32324420083009) bank43800 +43801 POINT(28.815780733506596 -94.45283524527309) bank43801 +43802 POINT(29.501152871460963 -96.29426578006898) bank43802 +43803 POINT(30.340070199661717 -95.71314507734793) bank43803 +43804 POINT(30.13331120727255 -95.18891322166152) bank43804 +43805 POINT(29.75724715193822 -95.5509445151321) bank43805 +43806 POINT(28.95319049259457 -96.33928879898532) bank43806 +43807 POINT(30.123764880134857 -95.0038624379095) bank43807 +43808 POINT(28.84568874314026 -95.93708695301669) bank43808 +43809 POINT(30.37523183273991 -95.07595843516364) bank43809 +43810 POINT(29.735616397877653 -95.43591248459738) bank43810 +43811 POINT(29.623069012520105 -94.59294508167717) bank43811 +43812 POINT(29.326015414055753 -96.0866951614968) bank43812 +43813 POINT(29.246805791534115 -94.86499781024268) bank43813 +43814 POINT(29.06886737488849 -94.6206871551112) bank43814 +43815 POINT(29.545077590562386 -95.86141881327677) bank43815 +43816 POINT(29.660750764063298 -94.57566853009119) bank43816 +43817 POINT(29.568386736357752 -94.69339099073113) bank43817 +43818 POINT(30.32448444747681 -96.28037694704915) bank43818 +43819 POINT(30.657691144765252 -95.22867661664475) bank43819 +43820 POINT(30.444352412410637 -94.37601290694096) bank43820 +43821 POINT(30.09316846980215 -95.10927021471367) bank43821 +43822 POINT(29.717245276056673 -95.3110474883182) bank43822 +43823 POINT(29.88027500217371 -94.72651945493512) bank43823 +43824 POINT(29.6280987164776 -94.50825419220962) bank43824 +43825 POINT(29.35803970508815 -95.73057434756308) bank43825 +43826 POINT(30.34685199599663 -95.37036617610676) bank43826 +43827 POINT(29.452288284028786 -95.70917074340912) bank43827 +43828 POINT(30.442633606089384 -96.1658513587033) bank43828 +43829 POINT(29.159835930520952 -95.94396993686225) bank43829 +43830 POINT(30.542362594984898 -94.83666627957709) bank43830 +43831 POINT(29.4373099946372 -94.5177921652384) bank43831 +43832 POINT(30.24647132470789 -94.69650160900851) bank43832 +43833 POINT(29.012293010041027 -95.2709885936246) bank43833 +43834 POINT(29.36226487190407 -95.72849939172727) bank43834 +43835 POINT(30.704063269948296 -94.62774552679303) bank43835 +43836 POINT(29.609843495180428 -94.44575491699389) bank43836 +43837 POINT(30.752410884058914 -94.65588611262962) bank43837 +43838 POINT(29.205179288329873 -95.11771142636312) bank43838 +43839 POINT(28.933545228190127 -95.95874479083467) bank43839 +43840 POINT(29.02005964154717 -94.4166233786777) bank43840 +43841 POINT(29.83945968035682 -95.91381375392758) bank43841 +43842 POINT(30.726036325008497 -95.87980271168766) bank43842 +43843 POINT(30.22494095193816 -95.29082988604983) bank43843 +43844 POINT(29.538554523698917 -95.66114857372551) bank43844 +43845 POINT(30.702480646613026 -96.15587932810512) bank43845 +43846 POINT(29.488678658810183 -94.80236161527054) bank43846 +43847 POINT(30.21174201190784 -96.36126241792952) bank43847 +43848 POINT(30.41092648459891 -96.18646075551601) bank43848 +43849 POINT(29.258264883569893 -95.86254724480382) bank43849 +43850 POINT(30.75078960935055 -96.19241322482189) bank43850 +43851 POINT(28.93026772569406 -94.49888139812836) bank43851 +43852 POINT(29.393778876336626 -95.05252562216089) bank43852 +43853 POINT(28.95221278222119 -95.01536382371563) bank43853 +43854 POINT(30.195459830908803 -95.28221655594932) bank43854 +43855 POINT(29.52593431069716 -94.57183561469463) bank43855 +43856 POINT(30.136069912216822 -95.51902862251762) bank43856 +43857 POINT(30.297013090740464 -94.77734334392412) bank43857 +43858 POINT(30.319187130977507 -94.95250260718633) bank43858 +43859 POINT(30.48843434868806 -94.45631871610546) bank43859 +43860 POINT(30.55717054441192 -94.6973633027905) bank43860 +43861 POINT(30.61102311449849 -96.2954913596787) bank43861 +43862 POINT(30.757379251490264 -94.67888353461666) bank43862 +43863 POINT(28.76480713124676 -94.6027464048753) bank43863 +43864 POINT(30.270012871650167 -94.80219110436012) bank43864 +43865 POINT(29.922797294954304 -95.13985329167447) bank43865 +43866 POINT(29.297526074714163 -94.78383857987252) bank43866 +43867 POINT(28.92908820173792 -95.01389867871265) bank43867 +43868 POINT(29.978214679444086 -94.66769608178573) bank43868 +43869 POINT(30.248142915267525 -96.13507861057094) bank43869 +43870 POINT(30.48115042954915 -96.24636554073204) bank43870 +43871 POINT(29.69718524183996 -95.0614975070605) bank43871 +43872 POINT(29.861578137267763 -96.12008777976025) bank43872 +43873 POINT(30.184330041307053 -95.6174353133574) bank43873 +43874 POINT(30.712364430283383 -96.1411951667657) bank43874 +43875 POINT(30.246681274188127 -94.90563810253818) bank43875 +43876 POINT(30.127480823160838 -95.91215017630569) bank43876 +43877 POINT(29.794865719013927 -95.84314444777077) bank43877 +43878 POINT(30.27267369727901 -95.23292980801604) bank43878 +43879 POINT(29.07767881036974 -94.58080355054106) bank43879 +43880 POINT(29.256118723291323 -95.91190339532403) bank43880 +43881 POINT(29.839791136363438 -95.13289456599303) bank43881 +43882 POINT(29.016189100612138 -95.07800809038224) bank43882 +43883 POINT(29.973199165343317 -96.2401426269494) bank43883 +43884 POINT(30.044329513905303 -94.45750988490079) bank43884 +43885 POINT(30.441648606057665 -95.63911597439312) bank43885 +43886 POINT(30.719941549166435 -95.12279544990469) bank43886 +43887 POINT(28.982539757778113 -94.98081665459794) bank43887 +43888 POINT(28.775924023770063 -95.50503087239328) bank43888 +43889 POINT(30.71047977977892 -95.75255366098283) bank43889 +43890 POINT(29.27110288071148 -95.2309710632391) bank43890 +43891 POINT(28.852669504485334 -95.60746287189286) bank43891 +43892 POINT(28.986748954899173 -95.31470980909641) bank43892 +43893 POINT(30.66886693325399 -95.01776683055297) bank43893 +43894 POINT(30.077807997491668 -94.85613722485914) bank43894 +43895 POINT(28.881305313432463 -94.83702911754811) bank43895 +43896 POINT(30.456718707947363 -95.89262891770763) bank43896 +43897 POINT(29.139964926521245 -95.42510210973239) bank43897 +43898 POINT(28.968089112046844 -95.13932739816069) bank43898 +43899 POINT(30.15013717846533 -95.32973780606645) bank43899 +43900 POINT(29.581736556125648 -94.84644927846723) bank43900 +43901 POINT(29.247540752850124 -95.97613345608465) bank43901 +43902 POINT(28.99426721125284 -94.55985741246043) bank43902 +43903 POINT(30.324969686691336 -96.1652126656297) bank43903 +43904 POINT(29.080392941237168 -95.64835273693228) bank43904 +43905 POINT(29.672396752216116 -96.3536899651256) bank43905 +43906 POINT(30.179615219479082 -94.81293370615482) bank43906 +43907 POINT(29.702082971123406 -94.68248362175868) bank43907 +43908 POINT(30.6356384298998 -96.08785078781858) bank43908 +43909 POINT(30.17683513726428 -95.71520154246372) bank43909 +43910 POINT(29.01747211225631 -94.58508658373691) bank43910 +43911 POINT(30.735682024789803 -96.2440220201653) bank43911 +43912 POINT(29.65419934157781 -94.56763621448899) bank43912 +43913 POINT(28.86335448699216 -95.65828674714062) bank43913 +43914 POINT(29.66111426172376 -95.42604853893454) bank43914 +43915 POINT(30.65531104744268 -95.23536698436557) bank43915 +43916 POINT(30.682339830857106 -94.39285336930077) bank43916 +43917 POINT(29.6592525927233 -94.54962876474961) bank43917 +43918 POINT(29.040113739003854 -95.01251408505289) bank43918 +43919 POINT(30.255607953098554 -95.50147510324733) bank43919 +43920 POINT(29.704521668535282 -94.77891233857359) bank43920 +43921 POINT(30.476841977647855 -95.42361772808908) bank43921 +43922 POINT(29.916410007177475 -96.11164427248328) bank43922 +43923 POINT(29.153938861908607 -96.34804560029663) bank43923 +43924 POINT(30.477350863318986 -95.43924727011662) bank43924 +43925 POINT(29.971300612537345 -95.4895082498442) bank43925 +43926 POINT(30.198227544163565 -94.4297176768118) bank43926 +43927 POINT(29.144274662766108 -94.41362219977958) bank43927 +43928 POINT(30.61920097860392 -95.42786478271995) bank43928 +43929 POINT(29.794099415747752 -94.62607352912127) bank43929 +43930 POINT(29.768820911966717 -96.14940245029064) bank43930 +43931 POINT(29.37754319940017 -94.67008614440142) bank43931 +43932 POINT(30.228364298977244 -96.28207042210057) bank43932 +43933 POINT(28.946711010140312 -95.02070228364825) bank43933 +43934 POINT(30.635002935752443 -95.72908459616782) bank43934 +43935 POINT(30.546718090815393 -96.03512058115341) bank43935 +43936 POINT(30.137881000529745 -94.41302847072147) bank43936 +43937 POINT(30.435822930812787 -94.5577534482003) bank43937 +43938 POINT(29.854686302829453 -96.06131435609572) bank43938 +43939 POINT(29.61903219800472 -96.3490018357326) bank43939 +43940 POINT(30.18247633213593 -95.79012977824962) bank43940 +43941 POINT(30.14317313122903 -95.52845688086973) bank43941 +43942 POINT(30.038593279407486 -94.96427417559102) bank43942 +43943 POINT(28.783399649903068 -95.240723176466) bank43943 +43944 POINT(29.132916545459924 -96.28518842983146) bank43944 +43945 POINT(28.945954246960795 -95.7805288250007) bank43945 +43946 POINT(29.52759082194172 -96.32149996059195) bank43946 +43947 POINT(30.18202794356833 -95.32273829788764) bank43947 +43948 POINT(29.277820541607205 -96.21619540771302) bank43948 +43949 POINT(28.9868772521854 -94.5559843078007) bank43949 +43950 POINT(30.33752788156438 -94.77335378497135) bank43950 +43951 POINT(29.662567647177113 -95.06157102784374) bank43951 +43952 POINT(29.089979995098744 -95.28438844990448) bank43952 +43953 POINT(29.85704822266415 -95.44060262695307) bank43953 +43954 POINT(29.24653836077144 -94.65300153073076) bank43954 +43955 POINT(29.294223049102605 -94.83608081738703) bank43955 +43956 POINT(29.826461642615445 -94.81940017953988) bank43956 +43957 POINT(30.084226838759772 -94.43108661730916) bank43957 +43958 POINT(29.20233682968099 -94.58922692151202) bank43958 +43959 POINT(29.622685347036303 -95.29323873401408) bank43959 +43960 POINT(28.879917605465053 -94.48588379193832) bank43960 +43961 POINT(30.57622273004682 -95.05550350249904) bank43961 +43962 POINT(30.14758246242226 -95.18825528411017) bank43962 +43963 POINT(30.594207328554585 -96.31368930039005) bank43963 +43964 POINT(30.293133665103923 -95.47276600119376) bank43964 +43965 POINT(29.45261740007158 -94.61398050993228) bank43965 +43966 POINT(30.366451041370063 -96.19079549346844) bank43966 +43967 POINT(29.775510155307682 -95.115403133743) bank43967 +43968 POINT(29.41711453587466 -95.83771666353269) bank43968 +43969 POINT(29.66847489548787 -94.80698025613918) bank43969 +43970 POINT(30.064754599836892 -96.3089672692134) bank43970 +43971 POINT(29.159674515768053 -95.09891791237825) bank43971 +43972 POINT(30.207181187334417 -94.60721689797214) bank43972 +43973 POINT(29.21297959417442 -95.7084946011774) bank43973 +43974 POINT(29.55686827598262 -96.2004847919798) bank43974 +43975 POINT(28.96477785569328 -95.67914136572759) bank43975 +43976 POINT(29.089735250660937 -96.10867703220025) bank43976 +43977 POINT(29.71791819067565 -96.24241094484711) bank43977 +43978 POINT(29.924789882432716 -94.59659203758041) bank43978 +43979 POINT(29.769457243170113 -94.4729068471303) bank43979 +43980 POINT(28.797200814285045 -95.05788515087865) bank43980 +43981 POINT(29.1637675953794 -95.72554137388407) bank43981 +43982 POINT(29.56586947465554 -95.10704827598033) bank43982 +43983 POINT(30.738216143464232 -95.34608826147526) bank43983 +43984 POINT(30.759474103668335 -95.8763002250396) bank43984 +43985 POINT(30.552558849280338 -94.45240288865702) bank43985 +43986 POINT(29.958822970115953 -95.37672598775393) bank43986 +43987 POINT(30.332489724470804 -95.62201837435182) bank43987 +43988 POINT(29.505212051869716 -95.23069675671022) bank43988 +43989 POINT(28.89929220271429 -95.92747371559399) bank43989 +43990 POINT(29.659608198550128 -95.42262374863567) bank43990 +43991 POINT(29.963514336292704 -94.96961055563455) bank43991 +43992 POINT(29.788261215153856 -95.94069654920816) bank43992 +43993 POINT(30.641205236954107 -95.04981652544322) bank43993 +43994 POINT(29.88379180760655 -95.59242941819994) bank43994 +43995 POINT(29.122628678741503 -96.21467788848311) bank43995 +43996 POINT(30.362034614734853 -94.93362552251175) bank43996 +43997 POINT(29.58212576676457 -94.69065164873219) bank43997 +43998 POINT(29.953384069567512 -96.11099858059958) bank43998 +43999 POINT(29.322388032694466 -95.590161091422) bank43999 +44000 POINT(29.266579127274102 -95.60936067153406) bank44000 +44001 POINT(29.118720479181263 -94.91982281667488) bank44001 +44002 POINT(29.752448379587253 -95.00887424675135) bank44002 +44003 POINT(30.75915785148866 -96.25108618610412) bank44003 +44004 POINT(30.31760978893537 -95.03682975243724) bank44004 +44005 POINT(29.374479777082577 -95.32815168344135) bank44005 +44006 POINT(29.27052027884324 -95.45861271469217) bank44006 +44007 POINT(30.08346616258323 -95.1437083463796) bank44007 +44008 POINT(29.239385084830133 -95.15760299723625) bank44008 +44009 POINT(30.293372106757534 -95.28326605222891) bank44009 +44010 POINT(29.77490628596362 -94.67432446562164) bank44010 +44011 POINT(29.793274381665444 -94.42129979501877) bank44011 +44012 POINT(29.48751721014751 -95.02308636507938) bank44012 +44013 POINT(29.95210277242587 -95.35959864532828) bank44013 +44014 POINT(30.55536950402191 -95.97721476113752) bank44014 +44015 POINT(29.489429313276027 -96.30808290426293) bank44015 +44016 POINT(29.39661826125543 -96.25880166807457) bank44016 +44017 POINT(30.427604538236235 -95.81674923067527) bank44017 +44018 POINT(30.35998473848593 -95.7547228192031) bank44018 +44019 POINT(29.600997699167127 -96.18009044808076) bank44019 +44020 POINT(28.79208852063162 -95.2813474812554) bank44020 +44021 POINT(30.420834647603286 -95.99108054804009) bank44021 +44022 POINT(30.715339572873404 -94.71134697800251) bank44022 +44023 POINT(28.93446048627188 -96.33597034637548) bank44023 +44024 POINT(30.69523824706011 -95.50418829346829) bank44024 +44025 POINT(30.047419116984557 -94.88340056222974) bank44025 +44026 POINT(30.743373631570766 -95.9593087974773) bank44026 +44027 POINT(29.648494820906844 -94.9106542624634) bank44027 +44028 POINT(29.818135510353454 -95.1089475740994) bank44028 +44029 POINT(29.890234623308597 -95.62265645567446) bank44029 +44030 POINT(29.14709477084119 -95.62376253175127) bank44030 +44031 POINT(30.420157145116093 -96.05115372174915) bank44031 +44032 POINT(29.0502497753705 -94.5396881771126) bank44032 +44033 POINT(30.247947256973994 -95.37787155006156) bank44033 +44034 POINT(30.61562719936605 -95.1251017592237) bank44034 +44035 POINT(28.966351452289953 -94.52329920060167) bank44035 +44036 POINT(30.599482564298658 -95.48904369036224) bank44036 +44037 POINT(29.853912405030165 -96.30755429786102) bank44037 +44038 POINT(29.533154474511488 -96.30610687380074) bank44038 +44039 POINT(30.301945957978198 -95.6746762613962) bank44039 +44040 POINT(29.12777435146682 -95.49786327824393) bank44040 +44041 POINT(30.56302434109055 -95.27516247925094) bank44041 +44042 POINT(28.93889262167259 -96.01607549139442) bank44042 +44043 POINT(28.887621868989612 -95.70936782554315) bank44043 +44044 POINT(29.05138880031411 -95.23906055256344) bank44044 +44045 POINT(30.42009864850398 -95.83569310740101) bank44045 +44046 POINT(29.516318955868364 -95.57881441853898) bank44046 +44047 POINT(29.302393904450756 -94.38448850755319) bank44047 +44048 POINT(28.898687295288035 -95.4037544776946) bank44048 +44049 POINT(30.297880191486616 -95.53396763630515) bank44049 +44050 POINT(30.579756295449894 -95.20996497964592) bank44050 +44051 POINT(29.406796037621927 -95.05187671555768) bank44051 +44052 POINT(29.48116248600185 -94.57459233210793) bank44052 +44053 POINT(30.467205110226327 -96.18641135853946) bank44053 +44054 POINT(30.286308809266625 -95.07193794698706) bank44054 +44055 POINT(28.906375387613963 -95.02112408894077) bank44055 +44056 POINT(30.354217997612935 -96.29101285700676) bank44056 +44057 POINT(30.369910172344028 -95.69526512443528) bank44057 +44058 POINT(29.857462359527105 -94.51663332043506) bank44058 +44059 POINT(29.00569196393309 -94.91458432851947) bank44059 +44060 POINT(30.051929152735934 -95.71022983329836) bank44060 +44061 POINT(28.815969317303434 -96.19444646714037) bank44061 +44062 POINT(29.595054916285648 -94.58649580292172) bank44062 +44063 POINT(29.3527410254989 -95.6087933437525) bank44063 +44064 POINT(28.990015958294716 -95.66400008530051) bank44064 +44065 POINT(28.921829474373826 -95.76939017676416) bank44065 +44066 POINT(29.799534189102264 -95.56281143543175) bank44066 +44067 POINT(29.44216580898881 -94.62584847532436) bank44067 +44068 POINT(30.111475458146856 -95.7810206038054) bank44068 +44069 POINT(29.334606952573562 -95.4982498306879) bank44069 +44070 POINT(30.208029121962877 -96.07521237899961) bank44070 +44071 POINT(29.112703508213304 -96.33808558889065) bank44071 +44072 POINT(30.592114988881324 -95.37837506231574) bank44072 +44073 POINT(29.839165658088614 -94.39661626210774) bank44073 +44074 POINT(30.682527111070346 -96.24187220504807) bank44074 +44075 POINT(28.91769454439186 -95.76004813025347) bank44075 +44076 POINT(30.306371334364048 -95.20640779587279) bank44076 +44077 POINT(28.986567127179686 -94.96655385496487) bank44077 +44078 POINT(29.292531910133988 -94.98328706711085) bank44078 +44079 POINT(29.12760427271166 -96.31740127044499) bank44079 +44080 POINT(28.801854567235004 -96.27362863868439) bank44080 +44081 POINT(30.09666788476865 -94.61657338532642) bank44081 +44082 POINT(29.551000252996285 -95.374404254283) bank44082 +44083 POINT(29.834706602424966 -94.68822314667594) bank44083 +44084 POINT(29.264018579767477 -95.42510456406814) bank44084 +44085 POINT(29.12309555070597 -95.31750612213642) bank44085 +44086 POINT(30.441091573634914 -94.5427410108639) bank44086 +44087 POINT(29.199978013475796 -95.79508996606498) bank44087 +44088 POINT(29.352810652273778 -96.13517566670733) bank44088 +44089 POINT(28.87104097483147 -94.46862939351853) bank44089 +44090 POINT(30.457116322499292 -94.59487842717522) bank44090 +44091 POINT(29.972347870784148 -94.63193038411609) bank44091 +44092 POINT(29.3011561460456 -94.98583936921355) bank44092 +44093 POINT(29.831880301248265 -95.38597462930166) bank44093 +44094 POINT(30.56899217383783 -95.7369600852374) bank44094 +44095 POINT(29.438586575671067 -94.41329195554171) bank44095 +44096 POINT(30.257126979786772 -95.6895747558304) bank44096 +44097 POINT(30.2900621286324 -95.14825812208026) bank44097 +44098 POINT(30.68320318758924 -94.74561721355825) bank44098 +44099 POINT(29.03436115751137 -95.52922464531225) bank44099 +44100 POINT(29.25346551696922 -95.66057790595535) bank44100 +44101 POINT(30.34604189946685 -94.95621176643914) bank44101 +44102 POINT(30.276356610792362 -96.28860613010109) bank44102 +44103 POINT(29.3763971855094 -94.79508753124196) bank44103 +44104 POINT(30.419974075228666 -95.48258389352094) bank44104 +44105 POINT(29.33862309263365 -95.06053520356025) bank44105 +44106 POINT(30.05574516427073 -95.24609079485322) bank44106 +44107 POINT(29.298456684438285 -96.01452417830887) bank44107 +44108 POINT(29.250056463934854 -94.86610182981053) bank44108 +44109 POINT(29.261809523700897 -95.08132388118506) bank44109 +44110 POINT(29.131427512007264 -95.03053664710134) bank44110 +44111 POINT(30.62681284096964 -95.36151893862339) bank44111 +44112 POINT(28.7751633378388 -96.16668258522209) bank44112 +44113 POINT(30.058053425472252 -96.25375335779052) bank44113 +44114 POINT(28.763917455892162 -95.86258740887507) bank44114 +44115 POINT(30.38109289196921 -95.55590255897647) bank44115 +44116 POINT(29.107666795301103 -94.55972935437607) bank44116 +44117 POINT(29.69646038196699 -96.269958735479) bank44117 +44118 POINT(29.71138259568623 -95.36033228020396) bank44118 +44119 POINT(30.049832745802586 -95.10605604070123) bank44119 +44120 POINT(30.438848288891897 -94.4886645399362) bank44120 +44121 POINT(30.750145747515983 -95.35391577894593) bank44121 +44122 POINT(30.273292144951743 -95.3787223757017) bank44122 +44123 POINT(28.908609344128838 -95.4230382584947) bank44123 +44124 POINT(30.423742950943158 -95.77159744231331) bank44124 +44125 POINT(29.57190152533226 -94.99187241882132) bank44125 +44126 POINT(29.125720431955365 -94.5204826286966) bank44126 +44127 POINT(28.992387004148384 -96.0733901886051) bank44127 +44128 POINT(30.660524059022745 -94.45781290403772) bank44128 +44129 POINT(29.69522138206175 -95.8638203086542) bank44129 +44130 POINT(28.99262148389137 -95.82215065597751) bank44130 +44131 POINT(30.46820758206089 -96.02428952585778) bank44131 +44132 POINT(29.672037796016173 -95.85645859688333) bank44132 +44133 POINT(30.30646050295072 -95.4530825219659) bank44133 +44134 POINT(28.998645071582622 -95.62186325040172) bank44134 +44135 POINT(29.815728838493914 -95.25745662362752) bank44135 +44136 POINT(28.977258624647945 -94.69850088040553) bank44136 +44137 POINT(30.04337213447862 -94.8111393858454) bank44137 +44138 POINT(28.7850211517681 -95.74235599809657) bank44138 +44139 POINT(30.411586916796963 -96.12968355654905) bank44139 +44140 POINT(30.065130930165974 -94.73047763492868) bank44140 +44141 POINT(29.26634393233062 -94.69890355934176) bank44141 +44142 POINT(29.39435845997625 -94.9966526121348) bank44142 +44143 POINT(29.809402596063062 -94.80186789773228) bank44143 +44144 POINT(30.56633769188567 -94.67651307204659) bank44144 +44145 POINT(28.76204862783267 -95.77286411828796) bank44145 +44146 POINT(29.517730629164227 -96.06096387699361) bank44146 +44147 POINT(29.97597978894727 -95.62032491283816) bank44147 +44148 POINT(30.387920015150907 -95.1478013906279) bank44148 +44149 POINT(29.404442045413113 -94.42678643685218) bank44149 +44150 POINT(29.67663149206162 -95.68508502488216) bank44150 +44151 POINT(30.45552213786275 -95.78263787935792) bank44151 +44152 POINT(29.556732170642928 -94.79163437235715) bank44152 +44153 POINT(29.28578918826334 -95.02534505993387) bank44153 +44154 POINT(29.720676377744514 -95.18608549654645) bank44154 +44155 POINT(29.519374107237056 -94.97996728696036) bank44155 +44156 POINT(29.383784160234406 -96.26426975601433) bank44156 +44157 POINT(30.177631212180334 -95.58392594108152) bank44157 +44158 POINT(29.036641274696237 -95.14819943706654) bank44158 +44159 POINT(30.58790915357597 -95.18292457392552) bank44159 +44160 POINT(30.68588619293424 -94.95115392720697) bank44160 +44161 POINT(30.393460017402337 -94.73770998185354) bank44161 +44162 POINT(29.5425990816207 -94.86676350458598) bank44162 +44163 POINT(30.426398715026426 -95.58277804223168) bank44163 +44164 POINT(28.9347044475649 -94.57603088522063) bank44164 +44165 POINT(28.957721992276216 -95.62685791917401) bank44165 +44166 POINT(29.436633955881994 -95.76499705207475) bank44166 +44167 POINT(29.290075894694915 -95.88369337339891) bank44167 +44168 POINT(29.651848174590885 -94.38359194251711) bank44168 +44169 POINT(30.600499066675628 -94.70000828903397) bank44169 +44170 POINT(30.29531556029649 -94.38791322732871) bank44170 +44171 POINT(29.437234119978605 -95.52051604466071) bank44171 +44172 POINT(29.216062887142236 -95.96028606062939) bank44172 +44173 POINT(29.70222082688643 -96.28530501601688) bank44173 +44174 POINT(29.67949477257973 -95.04434382854329) bank44174 +44175 POINT(29.377177209539703 -94.62880434673954) bank44175 +44176 POINT(29.6367868742374 -95.94395468644173) bank44176 +44177 POINT(30.279295552015125 -95.24401946518616) bank44177 +44178 POINT(29.06437162846838 -94.51066257376735) bank44178 +44179 POINT(29.84363110654104 -96.19963599592963) bank44179 +44180 POINT(29.404192030527106 -95.0256645971457) bank44180 +44181 POINT(28.78688587279083 -95.62593864280235) bank44181 +44182 POINT(29.10805379999782 -95.62345573963468) bank44182 +44183 POINT(29.702698767948327 -95.04250958725213) bank44183 +44184 POINT(30.607148598756446 -95.33862785592837) bank44184 +44185 POINT(29.148869770074736 -95.1065449908344) bank44185 +44186 POINT(29.38923479472065 -94.9471318321757) bank44186 +44187 POINT(30.566786669052455 -95.86409849112351) bank44187 +44188 POINT(30.444111621649057 -94.84819269744939) bank44188 +44189 POINT(29.337810515464312 -95.26450254549891) bank44189 +44190 POINT(30.172683614437247 -95.97358409804988) bank44190 +44191 POINT(30.19039991712517 -94.51473641789984) bank44191 +44192 POINT(29.13008984673642 -95.49017483147671) bank44192 +44193 POINT(29.399909635974158 -94.75712460373363) bank44193 +44194 POINT(30.526060072364086 -95.98422745241905) bank44194 +44195 POINT(30.347865035504864 -96.18589283635406) bank44195 +44196 POINT(30.215447171428703 -95.37324810496158) bank44196 +44197 POINT(29.741055642654032 -95.7888059760827) bank44197 +44198 POINT(29.76317763332157 -94.41152703145015) bank44198 +44199 POINT(29.556493180177366 -94.73751942878918) bank44199 +44200 POINT(30.05974613843429 -96.09033934438403) bank44200 +44201 POINT(30.384912197724763 -95.58362462275272) bank44201 +44202 POINT(29.29451685921355 -95.79147737037874) bank44202 +44203 POINT(30.116503735899155 -94.42995661750533) bank44203 +44204 POINT(29.806951990867766 -94.8538646301741) bank44204 +44205 POINT(30.056788252349982 -95.37496243153619) bank44205 +44206 POINT(30.518365047105885 -96.31717643126963) bank44206 +44207 POINT(30.180855120876068 -94.76197912563774) bank44207 +44208 POINT(29.336778669097846 -95.4343550206095) bank44208 +44209 POINT(30.191512128270585 -95.60570099579165) bank44209 +44210 POINT(30.632554178895855 -95.56356576731045) bank44210 +44211 POINT(29.565139057809844 -95.67754732175912) bank44211 +44212 POINT(28.858325750444838 -96.05788744764692) bank44212 +44213 POINT(29.651934468212794 -95.74605107990152) bank44213 +44214 POINT(29.208322279176347 -95.68047501803642) bank44214 +44215 POINT(29.96087132975657 -95.0461703731205) bank44215 +44216 POINT(29.70209915373183 -94.86339676549312) bank44216 +44217 POINT(29.95654713276603 -96.19423335096376) bank44217 +44218 POINT(30.22179957320176 -95.75856718966209) bank44218 +44219 POINT(29.04028349326296 -94.94547029683267) bank44219 +44220 POINT(29.925348662532596 -96.20853841378683) bank44220 +44221 POINT(29.51939067280138 -94.71349436265503) bank44221 +44222 POINT(30.432693684729227 -95.34678147316977) bank44222 +44223 POINT(29.24439334466971 -95.54415173116246) bank44223 +44224 POINT(29.030270506213597 -94.65939261877445) bank44224 +44225 POINT(30.1807563370756 -95.66275418606376) bank44225 +44226 POINT(30.218540789262967 -95.10688761590086) bank44226 +44227 POINT(30.66974620291544 -95.67379268833136) bank44227 +44228 POINT(29.518872534590336 -95.53286912482253) bank44228 +44229 POINT(29.97192889452442 -94.76679001550407) bank44229 +44230 POINT(30.093370566382198 -95.02456881883444) bank44230 +44231 POINT(30.00840626555575 -95.36705241261) bank44231 +44232 POINT(30.314613446340303 -94.82970336912744) bank44232 +44233 POINT(30.688995606658455 -95.27582368304309) bank44233 +44234 POINT(29.16457401639519 -94.58718739831124) bank44234 +44235 POINT(30.58044165029684 -96.12775538457072) bank44235 +44236 POINT(30.02848271874408 -95.37829165149384) bank44236 +44237 POINT(29.633940228926587 -95.0149532513561) bank44237 +44238 POINT(30.077079953439185 -95.08059667180942) bank44238 +44239 POINT(29.980794656504603 -96.20151032107437) bank44239 +44240 POINT(30.176360332670516 -94.43875052896345) bank44240 +44241 POINT(30.100319330640865 -95.5714597974228) bank44241 +44242 POINT(29.316803332271753 -95.43907109995969) bank44242 +44243 POINT(30.637700217562067 -94.94157672702063) bank44243 +44244 POINT(30.388202885727335 -95.17573182535841) bank44244 +44245 POINT(30.55713919310626 -95.22587771740008) bank44245 +44246 POINT(30.194562412477346 -94.8094567774669) bank44246 +44247 POINT(29.73273745559013 -96.34057162858575) bank44247 +44248 POINT(28.916189234881955 -96.13518218907873) bank44248 +44249 POINT(30.385045013159292 -94.76686198909765) bank44249 +44250 POINT(29.002994640087874 -95.78228694697165) bank44250 +44251 POINT(30.15612315610086 -95.4792981011799) bank44251 +44252 POINT(30.46074032540257 -95.97216269959102) bank44252 +44253 POINT(30.04123346314486 -95.57309349294631) bank44253 +44254 POINT(30.31444682957665 -94.72807541167425) bank44254 +44255 POINT(29.20487746405672 -95.26896004206093) bank44255 +44256 POINT(29.498890060938088 -95.35991364448077) bank44256 +44257 POINT(30.15788216525382 -95.9117230816128) bank44257 +44258 POINT(29.400451605849184 -95.25767657364379) bank44258 +44259 POINT(28.923073811223333 -94.99333084474415) bank44259 +44260 POINT(30.360933692244885 -96.36947278762982) bank44260 +44261 POINT(29.654351507697033 -96.07813807893625) bank44261 +44262 POINT(29.01324923373904 -94.83579427463415) bank44262 +44263 POINT(30.732237612499095 -95.69627979108498) bank44263 +44264 POINT(30.138669912055057 -95.96054163278917) bank44264 +44265 POINT(29.179361970653037 -95.1255765740046) bank44265 +44266 POINT(29.528951502505713 -94.44018849610165) bank44266 +44267 POINT(29.46022234916051 -96.19418849396135) bank44267 +44268 POINT(30.513737467573087 -94.6557206294117) bank44268 +44269 POINT(29.651872466239634 -96.15626060649598) bank44269 +44270 POINT(30.577686738878807 -94.86608434366944) bank44270 +44271 POINT(29.12590784248988 -94.46187184724747) bank44271 +44272 POINT(30.441756498379203 -94.40523146782225) bank44272 +44273 POINT(30.095543196271663 -95.80544014477368) bank44273 +44274 POINT(29.62102256264653 -96.3145355979523) bank44274 +44275 POINT(30.645583782284458 -94.9801798759737) bank44275 +44276 POINT(29.992276437946607 -95.91371015191139) bank44276 +44277 POINT(30.13102658402333 -95.74169225084623) bank44277 +44278 POINT(29.954942103214627 -94.990110770805) bank44278 +44279 POINT(29.209053139929924 -95.90734839243076) bank44279 +44280 POINT(30.058247052130593 -95.68090369515754) bank44280 +44281 POINT(29.183472051729936 -95.93670465413857) bank44281 +44282 POINT(30.73258513766755 -96.17656608615565) bank44282 +44283 POINT(29.787130424681855 -95.1851499447825) bank44283 +44284 POINT(30.0099246964076 -94.55319871106532) bank44284 +44285 POINT(29.641469951832573 -94.4205672060247) bank44285 +44286 POINT(29.96022226202732 -94.74847050675373) bank44286 +44287 POINT(29.40728937481847 -94.56928739555396) bank44287 +44288 POINT(29.07473724438766 -95.50330941923472) bank44288 +44289 POINT(29.797887478025558 -95.27998994666119) bank44289 +44290 POINT(28.992310716794137 -95.48779798547955) bank44290 +44291 POINT(29.937190260672633 -95.69590691213486) bank44291 +44292 POINT(29.47652293778598 -95.81771374656398) bank44292 +44293 POINT(29.61957625967321 -96.09255642321534) bank44293 +44294 POINT(30.62713275989506 -96.3059291270485) bank44294 +44295 POINT(29.56816613615281 -94.51331904876797) bank44295 +44296 POINT(28.939321734146546 -94.88993342808962) bank44296 +44297 POINT(29.52444778931456 -94.58191529024727) bank44297 +44298 POINT(29.398509920546037 -95.16442345038313) bank44298 +44299 POINT(29.898700862568564 -94.70889435898725) bank44299 +44300 POINT(29.028725911291637 -94.61137921005722) bank44300 +44301 POINT(29.012821575650953 -95.49754930221124) bank44301 +44302 POINT(30.75673985799594 -96.26615471781987) bank44302 +44303 POINT(29.894185281488895 -95.27522565804627) bank44303 +44304 POINT(30.53009639581671 -96.12452707565794) bank44304 +44305 POINT(30.544426883540293 -95.70439842467127) bank44305 +44306 POINT(29.900470231849962 -95.85119660598393) bank44306 +44307 POINT(30.179939788585433 -96.21545577644011) bank44307 +44308 POINT(30.70432318660777 -94.86319755235644) bank44308 +44309 POINT(29.936254405034823 -95.92201753999953) bank44309 +44310 POINT(30.38237649995593 -95.64999661503411) bank44310 +44311 POINT(30.657548089476265 -95.21946893264827) bank44311 +44312 POINT(29.55855210589993 -95.16050802817097) bank44312 +44313 POINT(30.289396415434265 -94.5126942570249) bank44313 +44314 POINT(29.919297807696953 -94.97040813495977) bank44314 +44315 POINT(29.049818778673668 -95.18250958032223) bank44315 +44316 POINT(28.81425033422861 -95.56888436640624) bank44316 +44317 POINT(29.204913510386096 -94.64006765701006) bank44317 +44318 POINT(29.24328508462855 -94.79107941864113) bank44318 +44319 POINT(29.950588233521348 -95.46792596976417) bank44319 +44320 POINT(29.788389558198418 -95.01623402303794) bank44320 +44321 POINT(28.969418229045605 -95.9529364422859) bank44321 +44322 POINT(28.79047149211023 -96.36661787236301) bank44322 +44323 POINT(30.07293918732142 -94.61128606911761) bank44323 +44324 POINT(30.24147330950416 -96.2122135545822) bank44324 +44325 POINT(30.053613037660348 -95.79112568657523) bank44325 +44326 POINT(29.160426466728055 -96.08794747614535) bank44326 +44327 POINT(30.037879108621272 -96.2664078894982) bank44327 +44328 POINT(29.885825405862978 -95.20608940399305) bank44328 +44329 POINT(29.77853215914215 -95.32129509444388) bank44329 +44330 POINT(28.77665439367818 -95.38520308203587) bank44330 +44331 POINT(29.579847609011168 -94.88996781408918) bank44331 +44332 POINT(29.854847061999028 -94.84194409436988) bank44332 +44333 POINT(30.12614646536099 -94.89339640627506) bank44333 +44334 POINT(30.232981114873972 -95.7299023173815) bank44334 +44335 POINT(29.068202923446545 -95.49420090465235) bank44335 +44336 POINT(30.659115603158806 -96.11343259238961) bank44336 +44337 POINT(29.117763521266205 -95.94725920353358) bank44337 +44338 POINT(29.55091964751934 -95.04829060780482) bank44338 +44339 POINT(30.71109218853395 -94.44148163559855) bank44339 +44340 POINT(29.529765279853443 -95.1209550155013) bank44340 +44341 POINT(30.311379516838866 -95.23687689863506) bank44341 +44342 POINT(30.185722466148704 -94.90519931982779) bank44342 +44343 POINT(30.68935418861343 -95.40395501089266) bank44343 +44344 POINT(30.46655440107244 -95.42148441105466) bank44344 +44345 POINT(29.76949213946515 -95.36771288960098) bank44345 +44346 POINT(29.854773803044164 -95.34148721759698) bank44346 +44347 POINT(30.5594761605406 -94.8825561701367) bank44347 +44348 POINT(30.636172241418638 -94.46147886147473) bank44348 +44349 POINT(29.01437385726729 -94.83237346060545) bank44349 +44350 POINT(29.623023309593215 -96.26088303352363) bank44350 +44351 POINT(30.456916380433842 -95.28112425564055) bank44351 +44352 POINT(29.838635968080524 -95.3180519141743) bank44352 +44353 POINT(29.42079278718999 -95.95594500914723) bank44353 +44354 POINT(29.38455186298786 -96.0255924566501) bank44354 +44355 POINT(30.500906381825025 -94.73668960159642) bank44355 +44356 POINT(29.77637982439596 -96.25375788716342) bank44356 +44357 POINT(29.462864815754063 -96.17196847421687) bank44357 +44358 POINT(29.345835148438468 -95.52205490490454) bank44358 +44359 POINT(29.967552183380676 -95.06552734022269) bank44359 +44360 POINT(29.385297335220653 -94.9427391819329) bank44360 +44361 POINT(28.768584493830456 -95.51852659394838) bank44361 +44362 POINT(29.092435116523845 -94.43974268311095) bank44362 +44363 POINT(30.03501664690581 -95.31741301874763) bank44363 +44364 POINT(29.622763912561727 -94.64620047371673) bank44364 +44365 POINT(30.363283252176092 -96.294281900978) bank44365 +44366 POINT(29.458925936860734 -95.87052738157072) bank44366 +44367 POINT(29.83333316656916 -95.8685801623487) bank44367 +44368 POINT(30.177764511878298 -95.71109712735137) bank44368 +44369 POINT(30.2312516664024 -94.70701105309273) bank44369 +44370 POINT(30.346673570475204 -95.24447787721404) bank44370 +44371 POINT(29.809352464017206 -96.28758474646598) bank44371 +44372 POINT(29.672740297928936 -94.45796703113952) bank44372 +44373 POINT(29.91467619852225 -95.84270450677528) bank44373 +44374 POINT(30.242144183396135 -94.74910712156239) bank44374 +44375 POINT(29.94723256079032 -95.57795997926443) bank44375 +44376 POINT(29.86732417515343 -94.37453607518461) bank44376 +44377 POINT(30.062089823249906 -96.0340673205737) bank44377 +44378 POINT(30.36991183330607 -95.74918388113058) bank44378 +44379 POINT(30.52288455939286 -96.09804119265497) bank44379 +44380 POINT(29.761172609009083 -95.62175591960653) bank44380 +44381 POINT(29.231709147196224 -96.09867838830154) bank44381 +44382 POINT(28.84882669831265 -94.49722762199329) bank44382 +44383 POINT(29.613671968286013 -94.99216280414362) bank44383 +44384 POINT(29.672857052903847 -95.72180417096537) bank44384 +44385 POINT(29.02338679745432 -94.63375536716448) bank44385 +44386 POINT(30.219219462033674 -94.61422552566525) bank44386 +44387 POINT(29.445954156832148 -96.03545230391559) bank44387 +44388 POINT(29.552930380052697 -96.02058360251795) bank44388 +44389 POINT(29.140299267447343 -94.47634846012343) bank44389 +44390 POINT(28.90111586174122 -95.63207948176017) bank44390 +44391 POINT(28.96823297375273 -96.11298819308321) bank44391 +44392 POINT(30.616300757223428 -94.66215221522765) bank44392 +44393 POINT(29.943598834965606 -94.45339333000041) bank44393 +44394 POINT(29.01475250526054 -94.81766146905875) bank44394 +44395 POINT(28.96125291107759 -95.63256374688429) bank44395 +44396 POINT(28.828904991683 -96.08378167650923) bank44396 +44397 POINT(29.76467108617192 -94.96024271747523) bank44397 +44398 POINT(30.63432813849792 -95.45985114359206) bank44398 +44399 POINT(29.574916109644946 -95.73689037232779) bank44399 +44400 POINT(30.691924647117883 -95.73279093482373) bank44400 +44401 POINT(29.553467473137133 -95.36985620448998) bank44401 +44402 POINT(28.952632505086868 -94.43410202748866) bank44402 +44403 POINT(30.685687211666814 -94.49406118705082) bank44403 +44404 POINT(29.707157856807513 -94.44986922748726) bank44404 +44405 POINT(29.226664688888253 -95.27324375487242) bank44405 +44406 POINT(29.76017623445567 -95.42173962698996) bank44406 +44407 POINT(28.79009582956409 -96.3517101789599) bank44407 +44408 POINT(30.270912297318155 -96.03990962640327) bank44408 +44409 POINT(29.936821101920163 -95.52962156510823) bank44409 +44410 POINT(28.949959254569755 -95.23900327077858) bank44410 +44411 POINT(28.967132368493324 -96.13137473938687) bank44411 +44412 POINT(28.910748253529007 -95.21261009587197) bank44412 +44413 POINT(30.62252304039781 -95.93629806891597) bank44413 +44414 POINT(29.171394415801856 -95.43601130481876) bank44414 +44415 POINT(29.824886709266725 -95.17612179374085) bank44415 +44416 POINT(29.776157041990842 -94.84173476806389) bank44416 +44417 POINT(29.382863124016446 -95.62168801859987) bank44417 +44418 POINT(29.149564260713632 -95.51609851281549) bank44418 +44419 POINT(29.827917147755546 -96.21653426050992) bank44419 +44420 POINT(28.91960487287343 -94.4182115917083) bank44420 +44421 POINT(29.15641103471336 -96.20802064393463) bank44421 +44422 POINT(29.42743391580587 -94.83003545378628) bank44422 +44423 POINT(30.426540071579 -96.12315118775349) bank44423 +44424 POINT(29.344399264154752 -95.72817030671852) bank44424 +44425 POINT(28.760975349367623 -95.38362421210593) bank44425 +44426 POINT(30.372165099547217 -95.08988146532067) bank44426 +44427 POINT(29.839610660866416 -95.52922410727457) bank44427 +44428 POINT(29.68167624193291 -95.0551197898253) bank44428 +44429 POINT(29.48354220659445 -95.58384447970057) bank44429 +44430 POINT(30.15721258479925 -94.6749402964312) bank44430 +44431 POINT(30.0163318365 -95.1791637356905) bank44431 +44432 POINT(30.104306158052793 -96.23533849385517) bank44432 +44433 POINT(29.11901395477367 -95.02466182321817) bank44433 +44434 POINT(30.45948311508189 -96.09448907038805) bank44434 +44435 POINT(28.799776312526216 -94.38810677134887) bank44435 +44436 POINT(29.43732232125737 -94.72421583522762) bank44436 +44437 POINT(29.832398945908484 -95.62068472314819) bank44437 +44438 POINT(29.617855380243494 -94.98099291502501) bank44438 +44439 POINT(30.22252458299562 -95.59516282264065) bank44439 +44440 POINT(30.1728951523159 -95.47744943863023) bank44440 +44441 POINT(30.04468579049733 -94.60275210022537) bank44441 +44442 POINT(30.03173141519256 -95.03646722769395) bank44442 +44443 POINT(30.61777894099552 -94.9400434857448) bank44443 +44444 POINT(30.221557523964957 -94.80522413012227) bank44444 +44445 POINT(30.364596457207504 -95.63449892223042) bank44445 +44446 POINT(29.304242596583 -95.55372221846764) bank44446 +44447 POINT(29.036857428661644 -94.47193149872002) bank44447 +44448 POINT(29.210570301327365 -94.8397199627451) bank44448 +44449 POINT(30.251835754869838 -96.36173971409143) bank44449 +44450 POINT(29.704931393997924 -94.82247028817736) bank44450 +44451 POINT(29.40156695541395 -95.45192552418462) bank44451 +44452 POINT(30.59261188618957 -95.46124535061516) bank44452 +44453 POINT(30.07979221854833 -95.52561038040812) bank44453 +44454 POINT(30.555000889976423 -95.60981324545946) bank44454 +44455 POINT(30.483777835209427 -94.55158496714797) bank44455 +44456 POINT(30.638571197087533 -95.47107068563) bank44456 +44457 POINT(29.75372484758737 -95.91293236887893) bank44457 +44458 POINT(30.484534552830777 -94.56287940040431) bank44458 +44459 POINT(30.343110929608436 -95.47775894506037) bank44459 +44460 POINT(30.581958667478265 -96.09412134932734) bank44460 +44461 POINT(28.855287404734884 -95.0728928133191) bank44461 +44462 POINT(29.761465477687953 -95.69374858807191) bank44462 +44463 POINT(29.586558260635016 -95.62096161203061) bank44463 +44464 POINT(29.518667790220693 -95.739116337394) bank44464 +44465 POINT(29.981301348153565 -94.65415002244909) bank44465 +44466 POINT(30.410586454117855 -96.2756125937395) bank44466 +44467 POINT(30.237603670502395 -94.4352656487625) bank44467 +44468 POINT(30.163262670632065 -95.24469950875579) bank44468 +44469 POINT(30.17928948953315 -95.82717474577063) bank44469 +44470 POINT(29.65150472290372 -94.68617517501983) bank44470 +44471 POINT(30.052584384066822 -95.04884147124878) bank44471 +44472 POINT(30.010883011268835 -95.52890770604955) bank44472 +44473 POINT(29.51706975379661 -95.25259656590237) bank44473 +44474 POINT(29.69705461012487 -95.67100935357898) bank44474 +44475 POINT(29.760999442778147 -94.72732919272038) bank44475 +44476 POINT(29.10365388732251 -95.0028536471789) bank44476 +44477 POINT(28.94807146135547 -94.86694708115257) bank44477 +44478 POINT(30.503388551494194 -95.8467310757188) bank44478 +44479 POINT(29.930787619388425 -94.94861120468825) bank44479 +44480 POINT(30.661166443854633 -94.5110407397373) bank44480 +44481 POINT(29.485590478946516 -95.44458891844323) bank44481 +44482 POINT(30.21622799155757 -94.75818379464955) bank44482 +44483 POINT(30.413964722645446 -95.19385474812582) bank44483 +44484 POINT(30.662618460874178 -95.07899857199921) bank44484 +44485 POINT(29.969713961142794 -94.40567035871376) bank44485 +44486 POINT(30.052667113142867 -95.46392207062217) bank44486 +44487 POINT(30.313731937569028 -95.55791957468395) bank44487 +44488 POINT(30.500024682341117 -96.18800319151379) bank44488 +44489 POINT(30.62600245266035 -95.81280302905009) bank44489 +44490 POINT(29.75453182667886 -96.2697277151961) bank44490 +44491 POINT(29.56916086098942 -94.42348543346321) bank44491 +44492 POINT(30.0880167956651 -94.4440823423543) bank44492 +44493 POINT(29.1453750839814 -94.45363397443465) bank44493 +44494 POINT(30.14127783686622 -95.40104193835262) bank44494 +44495 POINT(30.15709224326652 -95.93663064599177) bank44495 +44496 POINT(29.393267983045448 -94.37950065092096) bank44496 +44497 POINT(28.8044319068086 -95.5558320587414) bank44497 +44498 POINT(29.18754375043588 -95.22938734171154) bank44498 +44499 POINT(30.65014926655224 -94.66583661701961) bank44499 +44500 POINT(29.382717719974718 -96.22095032146903) bank44500 +44501 POINT(29.956149082518145 -95.54201777335118) bank44501 +44502 POINT(29.990889205213968 -94.79314162856035) bank44502 +44503 POINT(30.324799608378832 -95.53142853728913) bank44503 +44504 POINT(30.119347200979718 -96.30360438512906) bank44504 +44505 POINT(29.764087809021817 -95.6749512153591) bank44505 +44506 POINT(30.370756026732533 -95.04543305691833) bank44506 +44507 POINT(30.084476206345133 -96.03579318941192) bank44507 +44508 POINT(30.04415982059032 -94.60446517488434) bank44508 +44509 POINT(30.469030170294143 -94.81452056242037) bank44509 +44510 POINT(29.860565687354917 -94.46387389929927) bank44510 +44511 POINT(30.135305549432687 -94.7402497288494) bank44511 +44512 POINT(28.85204254126375 -94.39579676210151) bank44512 +44513 POINT(29.24323884709318 -95.10821512428794) bank44513 +44514 POINT(29.077661873005237 -96.24654597825791) bank44514 +44515 POINT(30.22460010927735 -94.43784242457593) bank44515 +44516 POINT(29.91207006047332 -94.89971219569226) bank44516 +44517 POINT(30.105907122704412 -96.0157893800149) bank44517 +44518 POINT(29.016747334532496 -95.05676896135884) bank44518 +44519 POINT(30.565290463792497 -96.26930549558881) bank44519 +44520 POINT(29.3837427206948 -96.01041346460819) bank44520 +44521 POINT(29.28031908872928 -95.13071472496546) bank44521 +44522 POINT(29.170271282935264 -95.55781956427047) bank44522 +44523 POINT(30.50473720276658 -95.8959754812279) bank44523 +44524 POINT(30.079387474496748 -96.21582872689156) bank44524 +44525 POINT(30.26293644601712 -95.34241486378582) bank44525 +44526 POINT(29.057032316343655 -95.30266059451462) bank44526 +44527 POINT(30.012792503783643 -95.08622322013586) bank44527 +44528 POINT(30.401502529257233 -94.63904703305116) bank44528 +44529 POINT(29.819841677790105 -95.66061217818348) bank44529 +44530 POINT(29.473363704422045 -96.05799853208548) bank44530 +44531 POINT(30.02736783087134 -94.58685514646302) bank44531 +44532 POINT(29.61914208468509 -94.52924567904519) bank44532 +44533 POINT(29.839789722132174 -96.18611501932575) bank44533 +44534 POINT(29.249850962173035 -94.79883773736356) bank44534 +44535 POINT(30.193275524512487 -95.10171239939103) bank44535 +44536 POINT(29.22957126433163 -95.92924954982792) bank44536 +44537 POINT(29.167661367129167 -94.96726447644986) bank44537 +44538 POINT(29.277585563235558 -94.80174951276564) bank44538 +44539 POINT(30.415041059964533 -94.68790111652949) bank44539 +44540 POINT(29.84413714813223 -96.00353728556222) bank44540 +44541 POINT(29.861521279949542 -95.69215650296213) bank44541 +44542 POINT(30.65011982454592 -96.04119084195351) bank44542 +44543 POINT(30.37955817509732 -95.97997783093031) bank44543 +44544 POINT(29.017598501594943 -95.80037511034872) bank44544 +44545 POINT(30.342533404114416 -95.32482790693409) bank44545 +44546 POINT(28.86442419790235 -94.7327463257966) bank44546 +44547 POINT(29.982412337807705 -95.27776142821102) bank44547 +44548 POINT(30.750380928944622 -96.25226705234674) bank44548 +44549 POINT(30.512710896692685 -95.38009839453005) bank44549 +44550 POINT(29.849655571674774 -94.5183087459508) bank44550 +44551 POINT(30.5886498737991 -94.71717656677724) bank44551 +44552 POINT(30.38052019019195 -94.44090057855608) bank44552 +44553 POINT(29.68187238361914 -95.95355237680022) bank44553 +44554 POINT(28.87417136636538 -94.44308858763407) bank44554 +44555 POINT(28.783589252010625 -95.5211159709279) bank44555 +44556 POINT(29.917297646546665 -96.34480278986192) bank44556 +44557 POINT(30.39756471749011 -96.29851168196045) bank44557 +44558 POINT(28.88734236231969 -95.8582249543333) bank44558 +44559 POINT(29.042839145753952 -95.95006650564795) bank44559 +44560 POINT(29.87735813221889 -95.97324099470062) bank44560 +44561 POINT(30.56538067107432 -94.54180930089275) bank44561 +44562 POINT(28.788618971515117 -96.25657295779955) bank44562 +44563 POINT(29.028564546194236 -95.78638600489809) bank44563 +44564 POINT(29.52292589591667 -95.45727582282801) bank44564 +44565 POINT(29.59297089779847 -96.06141059961722) bank44565 +44566 POINT(30.44338884232212 -95.90105506165358) bank44566 +44567 POINT(29.3513169319576 -96.18525388948156) bank44567 +44568 POINT(28.928744286662408 -94.73611463356423) bank44568 +44569 POINT(30.231478739558995 -94.8878071406632) bank44569 +44570 POINT(30.35861311136118 -96.31624058889658) bank44570 +44571 POINT(30.096514236170698 -95.64825605826664) bank44571 +44572 POINT(29.749281084166558 -95.40688778313138) bank44572 +44573 POINT(28.778951566214513 -95.58520190018376) bank44573 +44574 POINT(30.023272311851386 -96.17778978108898) bank44574 +44575 POINT(29.912145937080556 -94.87357035697694) bank44575 +44576 POINT(30.607156694292282 -95.46981701405167) bank44576 +44577 POINT(29.853865037914716 -94.5763389786641) bank44577 +44578 POINT(30.053429051468292 -95.88610759353104) bank44578 +44579 POINT(30.663100758032613 -94.52624829381283) bank44579 +44580 POINT(30.08292545495191 -95.7886361285924) bank44580 +44581 POINT(29.40473766315565 -95.95052510111715) bank44581 +44582 POINT(29.882293225289942 -94.71222235655749) bank44582 +44583 POINT(29.62139294499887 -96.32579310368304) bank44583 +44584 POINT(30.40064746884372 -95.57113838711462) bank44584 +44585 POINT(30.59818659057637 -94.8522849195791) bank44585 +44586 POINT(29.128942329792366 -95.91902822820643) bank44586 +44587 POINT(29.206449652666098 -94.84536401260765) bank44587 +44588 POINT(30.728624018644528 -94.8994221674858) bank44588 +44589 POINT(28.821454707722527 -95.95491057180598) bank44589 +44590 POINT(29.978192882554502 -96.03505868395479) bank44590 +44591 POINT(30.300561529521282 -94.59585571577692) bank44591 +44592 POINT(28.888933882643954 -95.13716174212887) bank44592 +44593 POINT(29.94580717227057 -94.55940601277231) bank44593 +44594 POINT(29.59733012428655 -95.59702717610193) bank44594 +44595 POINT(30.537251076469218 -95.54800825398448) bank44595 +44596 POINT(30.30673794149059 -94.41946022852612) bank44596 +44597 POINT(30.02179383851341 -96.08274422489174) bank44597 +44598 POINT(29.407585997153845 -95.23916767231007) bank44598 +44599 POINT(30.37928898447849 -95.75447765626585) bank44599 +44600 POINT(29.89238801471933 -95.44530967840623) bank44600 +44601 POINT(28.98295923858879 -94.81061803765301) bank44601 +44602 POINT(30.558248105361038 -96.1390505431721) bank44602 +44603 POINT(28.87845445092426 -96.21821832498631) bank44603 +44604 POINT(29.603320685741444 -94.40156054503905) bank44604 +44605 POINT(29.51850354834258 -94.62586208448631) bank44605 +44606 POINT(29.81368673435467 -96.08871305496713) bank44606 +44607 POINT(29.06506796362728 -96.10861194796593) bank44607 +44608 POINT(30.739547508751002 -95.8210414829837) bank44608 +44609 POINT(29.712312999444656 -94.59118871489886) bank44609 +44610 POINT(29.782603831981618 -95.07480673125235) bank44610 +44611 POINT(28.887815481723024 -94.83195400136054) bank44611 +44612 POINT(30.657239856184912 -95.68842195065949) bank44612 +44613 POINT(29.934017579045356 -94.95428264507001) bank44613 +44614 POINT(30.624372265717863 -94.95220889678451) bank44614 +44615 POINT(30.4354926953781 -96.21930042060649) bank44615 +44616 POINT(29.015771160291905 -95.15155320991794) bank44616 +44617 POINT(30.688992050631196 -95.58630907075302) bank44617 +44618 POINT(29.799816709128866 -95.09797112478988) bank44618 +44619 POINT(29.410965253726623 -95.72813507306873) bank44619 +44620 POINT(29.19072548714125 -94.88956448055735) bank44620 +44621 POINT(30.279000392500013 -94.9222342306952) bank44621 +44622 POINT(30.36501004734009 -94.56045518697113) bank44622 +44623 POINT(29.44745821505708 -94.80030089652458) bank44623 +44624 POINT(30.22659234376467 -94.91691449381139) bank44624 +44625 POINT(29.3086196187153 -94.7084841733121) bank44625 +44626 POINT(30.731612935210777 -95.05785199422542) bank44626 +44627 POINT(29.998728450103407 -94.77024887899064) bank44627 +44628 POINT(29.92073892120044 -95.15344232721476) bank44628 +44629 POINT(29.49958625365676 -95.29418867986185) bank44629 +44630 POINT(29.12697178138817 -95.80190890254497) bank44630 +44631 POINT(29.263919740642937 -95.49818862517554) bank44631 +44632 POINT(29.12410554791947 -96.21416695031324) bank44632 +44633 POINT(29.457854492433455 -95.10920138282142) bank44633 +44634 POINT(29.2500074068793 -95.62652939146125) bank44634 +44635 POINT(29.84763095281002 -96.31757554591027) bank44635 +44636 POINT(28.880297191899526 -94.90031967343742) bank44636 +44637 POINT(29.823061132027473 -95.58504942991803) bank44637 +44638 POINT(29.241522162659805 -96.33432359478692) bank44638 +44639 POINT(29.541652186448218 -94.88817837795092) bank44639 +44640 POINT(29.285225497690867 -95.82830134782562) bank44640 +44641 POINT(30.32656134308419 -95.03823302486464) bank44641 +44642 POINT(28.815252959113046 -95.39242545311645) bank44642 +44643 POINT(28.90521117600706 -96.24121330710234) bank44643 +44644 POINT(30.202753086774432 -95.55749600375817) bank44644 +44645 POINT(30.4504449826993 -95.2811915072173) bank44645 +44646 POINT(29.886530883784157 -94.79777340751212) bank44646 +44647 POINT(29.471523027242377 -94.59686751980956) bank44647 +44648 POINT(29.666462085226446 -95.90154397187942) bank44648 +44649 POINT(29.392509686626195 -95.27619151169779) bank44649 +44650 POINT(29.93914180007999 -96.09546439048852) bank44650 +44651 POINT(29.721572437927698 -94.50275343916562) bank44651 +44652 POINT(29.983619743882638 -95.1186598605536) bank44652 +44653 POINT(29.215156745042247 -96.2816350232918) bank44653 +44654 POINT(29.6358694250907 -94.77001792158815) bank44654 +44655 POINT(28.847235028427644 -95.34494163336905) bank44655 +44656 POINT(30.477064322997464 -94.4146238742433) bank44656 +44657 POINT(29.55674635745383 -95.64738976152626) bank44657 +44658 POINT(29.147798115538876 -96.27745498409352) bank44658 +44659 POINT(29.359155400894725 -95.72627646890577) bank44659 +44660 POINT(29.23719522801428 -96.23369818673665) bank44660 +44661 POINT(29.26994463098152 -96.16737568831141) bank44661 +44662 POINT(29.630828595373213 -96.0736099676898) bank44662 +44663 POINT(30.14199299994094 -95.17501331343777) bank44663 +44664 POINT(29.443497673579618 -94.48740863998835) bank44664 +44665 POINT(29.796477555160827 -95.51232541428914) bank44665 +44666 POINT(29.32067811966826 -95.5091692251668) bank44666 +44667 POINT(28.820290548937123 -95.29200760867329) bank44667 +44668 POINT(29.17145960374144 -95.74414349743184) bank44668 +44669 POINT(29.996824337307963 -94.61110847918044) bank44669 +44670 POINT(29.04074615484621 -95.83745089491897) bank44670 +44671 POINT(30.205966526282385 -94.83982592808611) bank44671 +44672 POINT(29.945309041896436 -94.76550298276157) bank44672 +44673 POINT(30.452777853343445 -95.39518910123469) bank44673 +44674 POINT(29.835984567393634 -94.38611411377671) bank44674 +44675 POINT(29.132962062706277 -95.2881358118395) bank44675 +44676 POINT(30.74071308948032 -94.71069516315062) bank44676 +44677 POINT(29.650633020675844 -96.3250792186855) bank44677 +44678 POINT(29.25070488680648 -94.5290040133166) bank44678 +44679 POINT(29.139305973561857 -94.7981223261098) bank44679 +44680 POINT(29.427530658629514 -95.22202086820887) bank44680 +44681 POINT(30.22745124373424 -94.58614956956828) bank44681 +44682 POINT(30.01319030770962 -94.67415633017303) bank44682 +44683 POINT(28.971256753814668 -96.16860033579388) bank44683 +44684 POINT(29.954340116842225 -94.42014593609541) bank44684 +44685 POINT(29.093335773941973 -95.79025243073767) bank44685 +44686 POINT(30.264212417544073 -95.27168493314511) bank44686 +44687 POINT(30.521954677419686 -95.38293412981123) bank44687 +44688 POINT(28.92619833328671 -94.39282320345737) bank44688 +44689 POINT(29.000490408532915 -94.40354263756397) bank44689 +44690 POINT(29.528791903153586 -96.33801299913254) bank44690 +44691 POINT(28.83336731638357 -95.79573930135135) bank44691 +44692 POINT(30.285407031008027 -94.50125401670785) bank44692 +44693 POINT(30.59866661029931 -94.98219709594106) bank44693 +44694 POINT(29.066178778273336 -94.82869682133168) bank44694 +44695 POINT(29.943633836433396 -96.22659393070863) bank44695 +44696 POINT(30.099830987041084 -95.27491892909879) bank44696 +44697 POINT(30.214242927985232 -95.16971536322771) bank44697 +44698 POINT(28.961477764368905 -95.48496039971499) bank44698 +44699 POINT(29.395263333954386 -94.50774062057909) bank44699 +44700 POINT(29.001081045176687 -94.5414616779257) bank44700 +44701 POINT(29.50954315269629 -95.63894679140746) bank44701 +44702 POINT(29.848899040288693 -96.20405927581422) bank44702 +44703 POINT(30.21823790905296 -95.39142083860366) bank44703 +44704 POINT(30.47868203188214 -96.07161539339377) bank44704 +44705 POINT(30.359574356979767 -95.27512406995096) bank44705 +44706 POINT(30.236353619533148 -95.21485958533675) bank44706 +44707 POINT(30.398714706674223 -94.67050457379781) bank44707 +44708 POINT(29.09184872150922 -94.89038552019952) bank44708 +44709 POINT(29.327125347394787 -94.8554095478002) bank44709 +44710 POINT(29.519372279150936 -95.96546825371868) bank44710 +44711 POINT(29.789608021332416 -95.66003225982797) bank44711 +44712 POINT(29.429111895554552 -95.69611018128987) bank44712 +44713 POINT(29.30355896195507 -94.47982379772438) bank44713 +44714 POINT(29.848791178868403 -95.06623134302846) bank44714 +44715 POINT(30.293965284467426 -94.48385305236853) bank44715 +44716 POINT(30.745271254622896 -96.02770299819207) bank44716 +44717 POINT(29.46516161097201 -94.74997884748092) bank44717 +44718 POINT(29.231667638946252 -94.89351580008872) bank44718 +44719 POINT(30.411039197546593 -94.41559137711133) bank44719 +44720 POINT(29.200771217529706 -96.27202536939575) bank44720 +44721 POINT(29.674215067819425 -94.68319645298864) bank44721 +44722 POINT(30.47269511666936 -96.00775398357939) bank44722 +44723 POINT(29.486597480668173 -94.74569540405902) bank44723 +44724 POINT(30.704391454452168 -95.58855336188141) bank44724 +44725 POINT(29.824533127607953 -94.72270086198988) bank44725 +44726 POINT(30.409221948109323 -95.63104076377357) bank44726 +44727 POINT(29.788139248697533 -94.85667304552035) bank44727 +44728 POINT(29.145773181022918 -95.1945677900555) bank44728 +44729 POINT(29.965430713065167 -95.51947199752094) bank44729 +44730 POINT(29.657119696490394 -95.04624891096837) bank44730 +44731 POINT(29.54274000075587 -95.88646213730627) bank44731 +44732 POINT(28.85364003997554 -94.73329905691799) bank44732 +44733 POINT(29.360727054245036 -96.33370000507314) bank44733 +44734 POINT(29.63302653824414 -95.72129248277648) bank44734 +44735 POINT(28.839637982021824 -94.8873686701048) bank44735 +44736 POINT(29.58763491754884 -94.65765623684973) bank44736 +44737 POINT(30.187198711276732 -96.32887523930445) bank44737 +44738 POINT(29.12120998154319 -95.72482104717545) bank44738 +44739 POINT(30.57051255297807 -94.88592914644988) bank44739 +44740 POINT(30.08169340946733 -95.89986378442951) bank44740 +44741 POINT(28.978651336225862 -94.76233779479294) bank44741 +44742 POINT(29.422592580097852 -94.94643433141596) bank44742 +44743 POINT(28.762335497477952 -96.0495936935619) bank44743 +44744 POINT(29.277158837260284 -96.33725064021908) bank44744 +44745 POINT(30.737413154588957 -96.09449983578286) bank44745 +44746 POINT(30.253966170839455 -95.94337915784524) bank44746 +44747 POINT(29.319302841225433 -95.60735045905518) bank44747 +44748 POINT(29.78838937757336 -94.60126413452815) bank44748 +44749 POINT(29.39338356343169 -94.51874365243236) bank44749 +44750 POINT(28.904694872637876 -95.16962772669712) bank44750 +44751 POINT(29.942180274953543 -94.9656281273541) bank44751 +44752 POINT(29.613025412680635 -94.77759208818235) bank44752 +44753 POINT(29.53122065517257 -95.79456401831357) bank44753 +44754 POINT(30.307225136174097 -94.57233569285292) bank44754 +44755 POINT(29.404457070169578 -95.08965429764713) bank44755 +44756 POINT(30.465294756873874 -96.02929280271213) bank44756 +44757 POINT(29.097339679269055 -95.36163829750033) bank44757 +44758 POINT(29.968114943458193 -95.01101657618155) bank44758 +44759 POINT(29.128600015741164 -94.80611546999803) bank44759 +44760 POINT(29.02733494550984 -95.22321086309474) bank44760 +44761 POINT(29.9288158422125 -96.04788934769314) bank44761 +44762 POINT(29.32167853550103 -95.37774618351477) bank44762 +44763 POINT(30.58290638872612 -95.74210598885404) bank44763 +44764 POINT(30.49760418292352 -95.81398180777536) bank44764 +44765 POINT(29.23995458994801 -95.04529139856645) bank44765 +44766 POINT(28.993840422584427 -96.00877377080903) bank44766 +44767 POINT(30.65830725303166 -96.04432716080912) bank44767 +44768 POINT(30.17863019497517 -96.11886299841794) bank44768 +44769 POINT(29.63914102741058 -96.13653795926055) bank44769 +44770 POINT(30.003515806250263 -94.50085698193494) bank44770 +44771 POINT(29.22857414312888 -94.59046256007092) bank44771 +44772 POINT(30.318699424311713 -95.50120035387712) bank44772 +44773 POINT(30.70359418096868 -95.0275841015647) bank44773 +44774 POINT(29.528257954094393 -95.15420809170782) bank44774 +44775 POINT(30.52195186724978 -95.19690492358302) bank44775 +44776 POINT(30.03509609607819 -95.66823382699758) bank44776 +44777 POINT(29.26597390840469 -96.13670160702249) bank44777 +44778 POINT(29.824248478262735 -95.9868778924563) bank44778 +44779 POINT(29.667963463805496 -96.08441091555916) bank44779 +44780 POINT(30.026583680488997 -95.4531014954729) bank44780 +44781 POINT(29.020541296168066 -94.78202080477232) bank44781 +44782 POINT(29.756662676169896 -95.23695887260068) bank44782 +44783 POINT(29.566426637893017 -96.14874228745379) bank44783 +44784 POINT(30.22055692976891 -95.29878752416997) bank44784 +44785 POINT(28.932029720569034 -96.23717137886072) bank44785 +44786 POINT(29.60962911584877 -94.9545097245374) bank44786 +44787 POINT(28.964944852990715 -95.92782742322414) bank44787 +44788 POINT(29.251145827851712 -94.84400246934933) bank44788 +44789 POINT(29.142384439487245 -95.27450203287017) bank44789 +44790 POINT(30.06893141394717 -94.95005873719592) bank44790 +44791 POINT(30.15991997593531 -95.74472041431086) bank44791 +44792 POINT(28.999061943103595 -95.61714724714203) bank44792 +44793 POINT(30.08904825879043 -96.30010692596467) bank44793 +44794 POINT(28.96245780913923 -96.01627096829223) bank44794 +44795 POINT(30.59041659977091 -94.84602360777768) bank44795 +44796 POINT(30.566498315108642 -96.03041736992505) bank44796 +44797 POINT(29.595148147512997 -96.05129146041057) bank44797 +44798 POINT(28.875193790099363 -94.58613497256181) bank44798 +44799 POINT(28.853959034253844 -95.84679414605176) bank44799 +44800 POINT(28.954130365961177 -94.8956244992061) bank44800 +44801 POINT(29.725820875259075 -94.62572248257307) bank44801 +44802 POINT(30.07060608291649 -95.55335520070763) bank44802 +44803 POINT(28.914067366071194 -94.55985283849985) bank44803 +44804 POINT(28.94692920926659 -94.54530912604967) bank44804 +44805 POINT(29.96877493362844 -95.25166447821948) bank44805 +44806 POINT(30.06923888559438 -95.90417456655739) bank44806 +44807 POINT(28.909175690584078 -96.27053601108767) bank44807 +44808 POINT(29.642036742573303 -94.42318134972895) bank44808 +44809 POINT(29.90342570070063 -95.74667964009556) bank44809 +44810 POINT(28.802108751948595 -94.68212272640784) bank44810 +44811 POINT(30.61088052040644 -96.0253803398663) bank44811 +44812 POINT(29.18146690092523 -94.47410578812037) bank44812 +44813 POINT(30.188929659289578 -95.00807855576396) bank44813 +44814 POINT(30.74882670654368 -95.620350447583) bank44814 +44815 POINT(30.630296338044285 -95.99317580849377) bank44815 +44816 POINT(28.932868709571828 -96.10108889297358) bank44816 +44817 POINT(29.755048646269078 -94.62968817307504) bank44817 +44818 POINT(29.238154745413922 -95.51814473625916) bank44818 +44819 POINT(28.81462500678849 -94.51753350283957) bank44819 +44820 POINT(29.159993791017193 -95.7304264151388) bank44820 +44821 POINT(30.418191261034085 -95.99127522125764) bank44821 +44822 POINT(28.998095897224413 -94.79593510497193) bank44822 +44823 POINT(28.78930505274419 -96.06500936461717) bank44823 +44824 POINT(30.461732440441654 -95.7725998555332) bank44824 +44825 POINT(29.77290975548774 -95.67480773395236) bank44825 +44826 POINT(30.534706925284148 -94.4685501711082) bank44826 +44827 POINT(30.641943631244388 -95.96939003458728) bank44827 +44828 POINT(30.432622540750234 -94.78535057870468) bank44828 +44829 POINT(29.048029746233052 -96.24747781291823) bank44829 +44830 POINT(29.340275702545632 -94.39690808534239) bank44830 +44831 POINT(29.611746073036308 -95.92764598799127) bank44831 +44832 POINT(30.32286918820688 -95.66506650991647) bank44832 +44833 POINT(30.269101280638537 -94.59189032990311) bank44833 +44834 POINT(28.8153571932504 -95.31811615212808) bank44834 +44835 POINT(30.54028282437211 -95.01567083343433) bank44835 +44836 POINT(29.23660575726845 -95.68474874013035) bank44836 +44837 POINT(29.813103697031647 -94.37942965078108) bank44837 +44838 POINT(30.689952596795916 -95.98934807057122) bank44838 +44839 POINT(30.458548555985175 -94.59342799596203) bank44839 +44840 POINT(29.194723280623425 -95.73012195733702) bank44840 +44841 POINT(29.793018169999275 -95.796222215062) bank44841 +44842 POINT(29.747823890884835 -95.65372671751102) bank44842 +44843 POINT(30.707697298334768 -94.93534835691052) bank44843 +44844 POINT(29.519324588088292 -95.90428683560923) bank44844 +44845 POINT(28.929603397059314 -96.26465556638934) bank44845 +44846 POINT(29.695615942164437 -95.60626530330568) bank44846 +44847 POINT(29.53880616013455 -95.57041894291777) bank44847 +44848 POINT(29.35033231623757 -94.53506688971858) bank44848 +44849 POINT(30.4690440139852 -95.61800469882645) bank44849 +44850 POINT(28.987208342760354 -94.60194354894335) bank44850 +44851 POINT(29.552096330029087 -96.3387029337522) bank44851 +44852 POINT(29.196696469131076 -95.2223863113039) bank44852 +44853 POINT(30.471482055995438 -94.91011595439241) bank44853 +44854 POINT(30.225191502646048 -95.56164760371526) bank44854 +44855 POINT(30.45054390022374 -95.35626676997789) bank44855 +44856 POINT(28.93611295963261 -95.92509249854282) bank44856 +44857 POINT(29.972958472807875 -95.22997734373821) bank44857 +44858 POINT(28.90399110063727 -95.55607104267082) bank44858 +44859 POINT(30.350459889010832 -95.93147938533446) bank44859 +44860 POINT(30.130954437498616 -95.51430473801538) bank44860 +44861 POINT(28.88965314919904 -96.20506941477537) bank44861 +44862 POINT(29.98566828893803 -94.85539663265868) bank44862 +44863 POINT(28.768077991434886 -95.33017405304491) bank44863 +44864 POINT(29.39013213935317 -94.89514717789159) bank44864 +44865 POINT(30.40794043432239 -95.7818242900584) bank44865 +44866 POINT(30.677704720593656 -94.43065103765964) bank44866 +44867 POINT(30.496678091988976 -95.65946556819989) bank44867 +44868 POINT(30.12774158649631 -95.51233648759782) bank44868 +44869 POINT(30.121025170253308 -95.02710196311823) bank44869 +44870 POINT(30.23749161104811 -95.7936540241296) bank44870 +44871 POINT(30.733462130757857 -95.42976631265896) bank44871 +44872 POINT(30.122256970085107 -95.52832165548621) bank44872 +44873 POINT(29.684419301227102 -94.95232320225038) bank44873 +44874 POINT(28.869432989377678 -94.77775145207393) bank44874 +44875 POINT(29.08513341816714 -95.33825899934266) bank44875 +44876 POINT(30.558872552772165 -95.75378920254929) bank44876 +44877 POINT(29.90303109973373 -95.84926761729608) bank44877 +44878 POINT(30.072971373105755 -94.634318909376) bank44878 +44879 POINT(29.14961686469565 -94.6457091203785) bank44879 +44880 POINT(30.66184553080632 -95.79610175337734) bank44880 +44881 POINT(28.97677599561255 -96.00204861992641) bank44881 +44882 POINT(29.13360474862742 -95.48361139090046) bank44882 +44883 POINT(29.892564254414843 -94.59586359453046) bank44883 +44884 POINT(30.630681225057366 -95.20679308202348) bank44884 +44885 POINT(29.85339174366462 -94.89140921185331) bank44885 +44886 POINT(30.384503540439805 -96.26676840371775) bank44886 +44887 POINT(30.13315628959822 -95.7612384996932) bank44887 +44888 POINT(29.203843522584616 -94.67830226578091) bank44888 +44889 POINT(30.70011094052463 -94.59841948993248) bank44889 +44890 POINT(30.32474356693281 -96.14936941307688) bank44890 +44891 POINT(30.193974990382266 -95.1697850915467) bank44891 +44892 POINT(30.18067312193656 -95.715116076081) bank44892 +44893 POINT(30.6259764718177 -96.28470230768578) bank44893 +44894 POINT(28.873837548057907 -95.15157147071868) bank44894 +44895 POINT(30.232715026137118 -96.03674075527816) bank44895 +44896 POINT(30.43687616353634 -95.80104111187057) bank44896 +44897 POINT(29.491286427628552 -95.39440898364364) bank44897 +44898 POINT(30.201127609168523 -95.90926777280414) bank44898 +44899 POINT(29.258523738564875 -95.00911205599807) bank44899 +44900 POINT(29.322811228850643 -95.22826669059364) bank44900 +44901 POINT(30.632658172743895 -94.97019330379709) bank44901 +44902 POINT(29.830716721768912 -94.86474371215559) bank44902 +44903 POINT(29.01705843418859 -94.42436345498415) bank44903 +44904 POINT(29.64309016535372 -94.40008192457827) bank44904 +44905 POINT(28.994109060102375 -95.78801968732061) bank44905 +44906 POINT(30.19020569898156 -95.54225829632392) bank44906 +44907 POINT(28.992514332179066 -94.77023655216131) bank44907 +44908 POINT(30.382055535296786 -95.55705492142306) bank44908 +44909 POINT(29.10722334262914 -96.21706140173383) bank44909 +44910 POINT(28.95548442589304 -95.64532406110534) bank44910 +44911 POINT(29.44161339281955 -95.36966923196778) bank44911 +44912 POINT(30.66032003790231 -94.56107054013107) bank44912 +44913 POINT(29.98992759216401 -95.17965539465945) bank44913 +44914 POINT(30.184304956908683 -94.78995459788688) bank44914 +44915 POINT(28.922633717281368 -95.260488311068) bank44915 +44916 POINT(28.86192647280113 -95.54122489945249) bank44916 +44917 POINT(29.66313715290827 -95.44100005330482) bank44917 +44918 POINT(29.706228819638664 -95.75884852655103) bank44918 +44919 POINT(29.057269696043026 -96.12259158101004) bank44919 +44920 POINT(29.468048556829213 -94.56432714335222) bank44920 +44921 POINT(28.929987125790202 -94.38726461563142) bank44921 +44922 POINT(30.347239689122397 -94.9690524996463) bank44922 +44923 POINT(30.27286887648533 -95.33719348402138) bank44923 +44924 POINT(29.784794639030213 -95.99183527941202) bank44924 +44925 POINT(28.97433793981391 -95.35253903987004) bank44925 +44926 POINT(29.132626228506624 -94.7836909049476) bank44926 +44927 POINT(30.031658941350784 -95.26457230317068) bank44927 +44928 POINT(30.17025576220606 -95.917471682086) bank44928 +44929 POINT(30.145458981367153 -94.649015829078) bank44929 +44930 POINT(28.990341492153874 -95.97389927995455) bank44930 +44931 POINT(28.99962099957228 -94.6718367080581) bank44931 +44932 POINT(28.803759535061303 -96.16927339140577) bank44932 +44933 POINT(30.06566707797011 -94.90410309640455) bank44933 +44934 POINT(29.077450460484656 -95.30607971832312) bank44934 +44935 POINT(30.41241946342565 -94.62701919143348) bank44935 +44936 POINT(29.646869173601143 -95.65945069680774) bank44936 +44937 POINT(30.32926258359521 -94.41511126051935) bank44937 +44938 POINT(30.690693991968534 -95.84263420774693) bank44938 +44939 POINT(29.28904121667086 -96.0441794886011) bank44939 +44940 POINT(28.983177917898306 -95.08570985784198) bank44940 +44941 POINT(30.35869423040991 -95.15868571826253) bank44941 +44942 POINT(28.807851304484586 -96.14810136165042) bank44942 +44943 POINT(28.85410345239554 -95.46454501707603) bank44943 +44944 POINT(29.931902529448013 -95.68880259961752) bank44944 +44945 POINT(29.06485933478352 -95.55156073905958) bank44945 +44946 POINT(29.19247172632174 -95.45795988042241) bank44946 +44947 POINT(29.339774774949426 -96.20667725995935) bank44947 +44948 POINT(28.85216174879789 -94.88820540554288) bank44948 +44949 POINT(29.447420797800632 -96.33394146224032) bank44949 +44950 POINT(29.676766890995275 -95.74579777024671) bank44950 +44951 POINT(29.36286217198375 -96.1121498047766) bank44951 +44952 POINT(30.128708754063624 -96.01442368953049) bank44952 +44953 POINT(30.630854232765046 -94.7196832944697) bank44953 +44954 POINT(30.602348540172716 -95.32292271969376) bank44954 +44955 POINT(30.029745174110683 -95.53302478556472) bank44955 +44956 POINT(30.334353408981563 -95.2923387362276) bank44956 +44957 POINT(28.766825849870443 -95.79480479919579) bank44957 +44958 POINT(30.44856458094155 -95.77919388218977) bank44958 +44959 POINT(29.960646968162997 -95.20273291161874) bank44959 +44960 POINT(28.89526951243622 -94.91398583317977) bank44960 +44961 POINT(30.75566944982693 -95.64390894181194) bank44961 +44962 POINT(30.139818135634464 -95.46269260218558) bank44962 +44963 POINT(30.22036042066733 -95.98527722351136) bank44963 +44964 POINT(30.681230808213208 -94.90596611523759) bank44964 +44965 POINT(29.299402627705845 -95.44757832937782) bank44965 +44966 POINT(29.390130008085634 -96.07227337736877) bank44966 +44967 POINT(30.574792845754967 -94.84501388297167) bank44967 +44968 POINT(30.70919018612076 -95.43558339500028) bank44968 +44969 POINT(29.027793187748617 -95.0785633280944) bank44969 +44970 POINT(29.786711951355407 -94.68887306093653) bank44970 +44971 POINT(30.71896752535045 -94.63895911252197) bank44971 +44972 POINT(30.739923743222338 -94.72572920525334) bank44972 +44973 POINT(29.74273022385071 -95.36476340645063) bank44973 +44974 POINT(29.284256484705438 -95.41743464282013) bank44974 +44975 POINT(28.84048115842149 -95.33436997559095) bank44975 +44976 POINT(30.19541278728622 -94.64765639474034) bank44976 +44977 POINT(30.322199682039845 -94.51498035388717) bank44977 +44978 POINT(30.21621443813486 -95.57565659868344) bank44978 +44979 POINT(30.680054550113844 -94.47627574864148) bank44979 +44980 POINT(30.172889178138185 -96.0599569822399) bank44980 +44981 POINT(30.19598280165865 -95.2082981484503) bank44981 +44982 POINT(30.098717770605035 -95.01162446645958) bank44982 +44983 POINT(29.262670517226827 -94.38483417684309) bank44983 +44984 POINT(29.537989070971555 -95.68951628292952) bank44984 +44985 POINT(28.820543203193672 -95.60215943438418) bank44985 +44986 POINT(29.334780112873727 -94.56730594288153) bank44986 +44987 POINT(29.90906774156502 -95.597653435145) bank44987 +44988 POINT(30.623995533192137 -96.05030318335194) bank44988 +44989 POINT(30.69636825142954 -95.92586118250577) bank44989 +44990 POINT(30.609171896809645 -94.84481041765855) bank44990 +44991 POINT(30.235522921820785 -94.52166648809882) bank44991 +44992 POINT(30.21581155462454 -94.71921127174234) bank44992 +44993 POINT(29.68238504169141 -95.56344451000705) bank44993 +44994 POINT(30.67726725739295 -95.17167805413851) bank44994 +44995 POINT(29.3527706894532 -95.53946443217721) bank44995 +44996 POINT(29.538709326148993 -94.97823795205198) bank44996 +44997 POINT(30.208937518886486 -95.05531288620277) bank44997 +44998 POINT(29.25260490107501 -94.48987764945086) bank44998 +44999 POINT(29.661308196556647 -95.88701170867456) bank44999 +45000 POINT(30.06851235314486 -96.3120120594617) bank45000 +45001 POINT(29.41785604902679 -96.30169456190247) bank45001 +45002 POINT(30.60630190635017 -96.36832247446745) bank45002 +45003 POINT(29.096558781103788 -94.71442327583685) bank45003 +45004 POINT(29.82493186202212 -94.9865534709933) bank45004 +45005 POINT(29.521098439443 -94.69923205873698) bank45005 +45006 POINT(29.926504579512315 -95.02656237877709) bank45006 +45007 POINT(28.794186471872262 -96.30549588636784) bank45007 +45008 POINT(30.363128355141626 -94.71482528694538) bank45008 +45009 POINT(29.452107621390425 -94.61114099370974) bank45009 +45010 POINT(30.431358507805104 -96.06459588442748) bank45010 +45011 POINT(30.26104880772863 -94.68109013023582) bank45011 +45012 POINT(29.356395449452542 -94.64493574593497) bank45012 +45013 POINT(30.277709538157367 -95.19819896766641) bank45013 +45014 POINT(30.365195857508613 -95.69552608505306) bank45014 +45015 POINT(29.213633194165517 -95.38897641944563) bank45015 +45016 POINT(30.72944037651877 -95.94337697240918) bank45016 +45017 POINT(29.777607964740984 -95.512724530229) bank45017 +45018 POINT(30.648491159009232 -95.33956327251589) bank45018 +45019 POINT(30.075373225766477 -95.67607354511686) bank45019 +45020 POINT(30.320032525101922 -95.30557977968317) bank45020 +45021 POINT(30.637407727147902 -95.11699594878864) bank45021 +45022 POINT(30.533040585090713 -95.66447036021137) bank45022 +45023 POINT(29.19711696816542 -95.30453356389062) bank45023 +45024 POINT(30.678674779588142 -94.61593441411009) bank45024 +45025 POINT(30.251783987711313 -95.6596963451248) bank45025 +45026 POINT(30.697541289014456 -96.0750690966416) bank45026 +45027 POINT(29.725533268008498 -95.40167078057783) bank45027 +45028 POINT(30.23602175185553 -94.99463043592783) bank45028 +45029 POINT(29.010296183623637 -94.68459841521977) bank45029 +45030 POINT(29.371378033870037 -95.46638415407728) bank45030 +45031 POINT(30.167541345602665 -94.41371915649881) bank45031 +45032 POINT(30.18716468168918 -95.39150688747523) bank45032 +45033 POINT(29.017822622465022 -95.47104340488819) bank45033 +45034 POINT(29.348976233444425 -94.9492169432942) bank45034 +45035 POINT(28.85723025572251 -95.37325517223401) bank45035 +45036 POINT(28.870032337708967 -96.05593857808147) bank45036 +45037 POINT(30.6622146855882 -94.89201046328054) bank45037 +45038 POINT(28.984684069647273 -96.29068657516626) bank45038 +45039 POINT(30.18024904901752 -94.75969091241029) bank45039 +45040 POINT(29.79339904327404 -95.75745873409076) bank45040 +45041 POINT(29.196451121467728 -95.05166821000236) bank45041 +45042 POINT(30.17562290510921 -95.0366733508743) bank45042 +45043 POINT(30.261403478850756 -94.74317314019062) bank45043 +45044 POINT(30.138353082887253 -95.19765531911223) bank45044 +45045 POINT(29.317117229283816 -94.80619495450931) bank45045 +45046 POINT(29.641956236572288 -95.80500025722507) bank45046 +45047 POINT(29.66032637033562 -95.70589803954584) bank45047 +45048 POINT(30.397753932878146 -95.09490933319124) bank45048 +45049 POINT(28.84128789397496 -95.7703790671379) bank45049 +45050 POINT(30.001722297192813 -95.56718259080702) bank45050 +45051 POINT(29.681269115393004 -94.59198176518488) bank45051 +45052 POINT(30.555457992695242 -96.30661700723974) bank45052 +45053 POINT(30.08115521921324 -95.02199493631574) bank45053 +45054 POINT(29.650286458123944 -95.95932910932989) bank45054 +45055 POINT(30.599250637702358 -95.73821031633577) bank45055 +45056 POINT(29.795514106947635 -95.72927297378305) bank45056 +45057 POINT(30.224023406186365 -95.2643767502082) bank45057 +45058 POINT(29.927304632632868 -96.29838624862226) bank45058 +45059 POINT(30.11205658102878 -95.50694212032455) bank45059 +45060 POINT(30.637393016523045 -94.8969615345633) bank45060 +45061 POINT(30.712810468588117 -96.06074580910261) bank45061 +45062 POINT(29.501612480045694 -94.83969079687334) bank45062 +45063 POINT(29.833302345247837 -95.23532833210737) bank45063 +45064 POINT(30.355894229202498 -94.96694902928718) bank45064 +45065 POINT(29.86402286279458 -95.71969663431005) bank45065 +45066 POINT(29.999781809501076 -95.34661823004168) bank45066 +45067 POINT(30.213468087080685 -95.08297769397251) bank45067 +45068 POINT(30.65651480309696 -96.17277876369953) bank45068 +45069 POINT(30.546961300244156 -95.8317065502936) bank45069 +45070 POINT(30.59326158040057 -96.00560809439739) bank45070 +45071 POINT(29.20729481118482 -96.33173589818881) bank45071 +45072 POINT(29.111565643217823 -95.93345328326977) bank45072 +45073 POINT(29.459620022494285 -96.23393336229452) bank45073 +45074 POINT(29.525746117154654 -94.8626274224915) bank45074 +45075 POINT(29.36628757854703 -95.68343443392027) bank45075 +45076 POINT(30.0985268542745 -94.8073028222374) bank45076 +45077 POINT(28.974155258252132 -95.63048167721541) bank45077 +45078 POINT(29.08019439787788 -94.90914335802242) bank45078 +45079 POINT(29.18698590543106 -95.89859130961051) bank45079 +45080 POINT(30.210231246401218 -95.95192210736002) bank45080 +45081 POINT(29.782047644430012 -95.53805600948006) bank45081 +45082 POINT(30.02741870175017 -95.30089681598405) bank45082 +45083 POINT(29.084749222101863 -95.7906501617858) bank45083 +45084 POINT(30.191903985295795 -95.8927840589414) bank45084 +45085 POINT(30.74054946421264 -96.00298169865505) bank45085 +45086 POINT(29.050963896783244 -94.83000678451208) bank45086 +45087 POINT(29.15335612017814 -94.38413438154464) bank45087 +45088 POINT(30.713597344661302 -95.44333229909034) bank45088 +45089 POINT(30.23286535214745 -96.02545954077596) bank45089 +45090 POINT(30.167819132636208 -95.2312717387551) bank45090 +45091 POINT(29.50746875287669 -94.72946586757398) bank45091 +45092 POINT(30.029787705692666 -95.13065270868519) bank45092 +45093 POINT(30.023754412626836 -94.89772046384624) bank45093 +45094 POINT(30.43532364680091 -95.61716272732745) bank45094 +45095 POINT(29.24821875382007 -94.50472099082236) bank45095 +45096 POINT(29.55557872292995 -95.40433676270648) bank45096 +45097 POINT(29.90627177123867 -94.49303425983673) bank45097 +45098 POINT(28.79775509794326 -94.52694888513241) bank45098 +45099 POINT(30.74318078295655 -96.24060603061281) bank45099 +45100 POINT(29.741298046480967 -95.7531290675065) bank45100 +45101 POINT(29.155671596935807 -96.00556437471506) bank45101 +45102 POINT(30.631993941787382 -94.94597834752028) bank45102 +45103 POINT(28.805369057053397 -95.97344575783521) bank45103 +45104 POINT(30.66581834337699 -94.6844204503361) bank45104 +45105 POINT(29.050139979642275 -95.6189123172813) bank45105 +45106 POINT(30.55399298315287 -96.28380253129343) bank45106 +45107 POINT(29.15170269169792 -94.4284802231027) bank45107 +45108 POINT(29.318813479546442 -96.04163483249077) bank45108 +45109 POINT(29.284273548106352 -95.86516092658509) bank45109 +45110 POINT(29.496440416343905 -95.09387430663492) bank45110 +45111 POINT(30.67037094389683 -95.11704121090797) bank45111 +45112 POINT(29.940903697421952 -94.47021228588136) bank45112 +45113 POINT(30.156757606554375 -95.91225116437802) bank45113 +45114 POINT(28.949191527555488 -96.35405827155499) bank45114 +45115 POINT(29.164695863573378 -95.26987541003223) bank45115 +45116 POINT(29.60679182377869 -95.68441014742811) bank45116 +45117 POINT(29.561283675950435 -96.29972688716667) bank45117 +45118 POINT(29.76481116521112 -95.41345484933629) bank45118 +45119 POINT(29.91338547133996 -94.85133717832116) bank45119 +45120 POINT(30.19707913564127 -95.59156535947592) bank45120 +45121 POINT(28.975872926482722 -95.42126067937811) bank45121 +45122 POINT(29.005760346642298 -95.60515126951783) bank45122 +45123 POINT(29.59908202412899 -96.25999207929577) bank45123 +45124 POINT(29.26852252809758 -95.33852453356354) bank45124 +45125 POINT(29.057548022948694 -95.22986734135395) bank45125 +45126 POINT(30.080825342543424 -95.03392105745172) bank45126 +45127 POINT(29.078703109814857 -95.96379954014427) bank45127 +45128 POINT(29.761366141711306 -95.13361410999656) bank45128 +45129 POINT(30.195616862786636 -95.29470892115329) bank45129 +45130 POINT(29.480813644065943 -94.73419840788156) bank45130 +45131 POINT(29.20279426966784 -95.92835036442251) bank45131 +45132 POINT(29.45116578735974 -95.28095007882216) bank45132 +45133 POINT(30.034936075717166 -95.06868585177416) bank45133 +45134 POINT(29.44209149501797 -95.08261137770997) bank45134 +45135 POINT(29.71526759974045 -95.12991794004215) bank45135 +45136 POINT(29.127042520195005 -95.65817062189981) bank45136 +45137 POINT(30.326732279080183 -95.4735916342102) bank45137 +45138 POINT(29.569262667571163 -95.86085039589342) bank45138 +45139 POINT(29.06261088382792 -96.23115524550622) bank45139 +45140 POINT(28.90842298413759 -95.80656180553328) bank45140 +45141 POINT(30.212732954080362 -96.17822419281097) bank45141 +45142 POINT(30.074995005465112 -95.08309124738022) bank45142 +45143 POINT(30.606799953172736 -94.96210608290882) bank45143 +45144 POINT(30.187386186004193 -95.07510841630513) bank45144 +45145 POINT(29.459591617907265 -95.12800583042934) bank45145 +45146 POINT(29.012376280903705 -95.87865805906247) bank45146 +45147 POINT(29.896075397681848 -96.36608806922123) bank45147 +45148 POINT(29.283470232756752 -94.66460842425539) bank45148 +45149 POINT(30.18646952641052 -95.38273677781564) bank45149 +45150 POINT(29.064543625428563 -95.05256887607113) bank45150 +45151 POINT(30.014754205431338 -96.27368032939327) bank45151 +45152 POINT(29.611254594989948 -96.32795766152783) bank45152 +45153 POINT(29.92675843980075 -95.34399180046759) bank45153 +45154 POINT(28.996048801670554 -95.09429583750422) bank45154 +45155 POINT(30.564739199817062 -95.6189242285075) bank45155 +45156 POINT(29.253456763818928 -94.68584898082354) bank45156 +45157 POINT(29.71194422176107 -96.16152107460361) bank45157 +45158 POINT(30.57091581742998 -95.71394033893085) bank45158 +45159 POINT(30.559593934488905 -94.81344842144614) bank45159 +45160 POINT(29.029411597956162 -95.42056922884274) bank45160 +45161 POINT(30.221275265866424 -94.59156295531817) bank45161 +45162 POINT(28.8369277524117 -94.43534617350386) bank45162 +45163 POINT(29.769632684447306 -94.58046971531128) bank45163 +45164 POINT(29.940989293663446 -96.05027506426048) bank45164 +45165 POINT(29.218646260206114 -95.4834006007019) bank45165 +45166 POINT(30.424306577664993 -94.46561007349788) bank45166 +45167 POINT(29.355233607706968 -95.17567553419727) bank45167 +45168 POINT(29.383554027702424 -95.10977545201406) bank45168 +45169 POINT(29.197427662477484 -95.95324584206267) bank45169 +45170 POINT(30.260399954866205 -95.83632414137291) bank45170 +45171 POINT(29.58381312930559 -96.21741474838093) bank45171 +45172 POINT(28.812459927233128 -95.89003334832177) bank45172 +45173 POINT(29.68275771574605 -95.92778868570065) bank45173 +45174 POINT(29.5706845677962 -94.48721660427628) bank45174 +45175 POINT(29.95726687870677 -95.06163672995902) bank45175 +45176 POINT(30.02119048092857 -94.74345294231112) bank45176 +45177 POINT(29.62307253825218 -95.28653643733621) bank45177 +45178 POINT(30.425744992944438 -94.87455278987755) bank45178 +45179 POINT(30.475379557324175 -94.40790726987713) bank45179 +45180 POINT(29.49539188619725 -95.20520903332734) bank45180 +45181 POINT(29.041080172085803 -95.93054234335813) bank45181 +45182 POINT(29.927844704339872 -95.47270704274139) bank45182 +45183 POINT(30.373713648514386 -95.54316665522701) bank45183 +45184 POINT(29.78850716367704 -94.96785354498232) bank45184 +45185 POINT(28.898324024454336 -95.91381039291875) bank45185 +45186 POINT(29.248708411656022 -95.15790565533445) bank45186 +45187 POINT(29.512700919815376 -94.79870534329007) bank45187 +45188 POINT(29.05080219209262 -96.32109063012157) bank45188 +45189 POINT(29.41955434679966 -94.55446373878291) bank45189 +45190 POINT(30.71588069802285 -94.619311016093) bank45190 +45191 POINT(30.266749965125353 -94.78007523533662) bank45191 +45192 POINT(29.19420809186988 -95.61236035848948) bank45192 +45193 POINT(28.93574845248756 -94.97347252938292) bank45193 +45194 POINT(29.07029779623624 -96.28099092346659) bank45194 +45195 POINT(30.332952425538725 -94.76498270435339) bank45195 +45196 POINT(29.119687988122823 -96.25249527109669) bank45196 +45197 POINT(30.1255160142944 -95.29638286128915) bank45197 +45198 POINT(28.840784516461543 -95.64583274397512) bank45198 +45199 POINT(30.586099581862427 -94.60468937995844) bank45199 +45200 POINT(30.230189752227012 -94.44236932290616) bank45200 +45201 POINT(30.249402954464426 -95.56639634181559) bank45201 +45202 POINT(28.792402677116172 -95.74209622330484) bank45202 +45203 POINT(30.678512941495306 -94.86701578311374) bank45203 +45204 POINT(30.083930461617495 -95.60140810925958) bank45204 +45205 POINT(30.11313518665947 -96.06424641634477) bank45205 +45206 POINT(30.025140734318924 -94.59432625052715) bank45206 +45207 POINT(29.882462389248854 -95.72894042978642) bank45207 +45208 POINT(30.521992444109742 -95.36249548036935) bank45208 +45209 POINT(29.73940318825687 -95.8484525063841) bank45209 +45210 POINT(30.157854444223116 -94.63528535120886) bank45210 +45211 POINT(30.056689357419607 -94.8488525746113) bank45211 +45212 POINT(28.762026342496394 -94.47196435373374) bank45212 +45213 POINT(30.405817372610066 -96.01007031110518) bank45213 +45214 POINT(30.385950234027394 -95.69495625590837) bank45214 +45215 POINT(30.071556550190703 -94.3814939244903) bank45215 +45216 POINT(30.320144484949022 -96.22214778300273) bank45216 +45217 POINT(29.023075637144117 -95.48212891543353) bank45217 +45218 POINT(30.412048513228978 -95.50944659990951) bank45218 +45219 POINT(28.791530009522443 -96.03961444794037) bank45219 +45220 POINT(28.9421239404583 -95.08047825970377) bank45220 +45221 POINT(30.120746714935677 -94.89495049577944) bank45221 +45222 POINT(30.557987154384602 -95.18982364498451) bank45222 +45223 POINT(29.11808892541732 -96.3426860080304) bank45223 +45224 POINT(29.43364055011237 -95.10160007710036) bank45224 +45225 POINT(30.18121494061663 -94.54771410490335) bank45225 +45226 POINT(29.69431549387039 -96.21295826234018) bank45226 +45227 POINT(29.1748756156932 -94.89874429175049) bank45227 +45228 POINT(30.2815293787396 -95.5698876656029) bank45228 +45229 POINT(29.40693122896096 -95.5395730512409) bank45229 +45230 POINT(29.4318673006686 -95.7726780652706) bank45230 +45231 POINT(29.00705386464012 -94.93758683449768) bank45231 +45232 POINT(29.863514261472986 -95.93761079115653) bank45232 +45233 POINT(30.479014531874427 -94.40621109065442) bank45233 +45234 POINT(29.95067694612724 -96.10771078000903) bank45234 +45235 POINT(28.956756062677847 -96.1256398025885) bank45235 +45236 POINT(30.53346350501047 -95.70553387626508) bank45236 +45237 POINT(29.723627686670483 -96.33492014794639) bank45237 +45238 POINT(28.996575604939537 -95.94039298210514) bank45238 +45239 POINT(28.841132914045666 -94.70367073333782) bank45239 +45240 POINT(29.21781365829992 -95.69925710825176) bank45240 +45241 POINT(30.67611857746558 -96.13029107371258) bank45241 +45242 POINT(29.319735674089415 -95.62635395684752) bank45242 +45243 POINT(29.46424367590287 -96.01367644506573) bank45243 +45244 POINT(29.814831534205165 -95.09730276107088) bank45244 +45245 POINT(29.671764387934 -95.36947465761779) bank45245 +45246 POINT(28.980919932135382 -94.47509342539905) bank45246 +45247 POINT(30.250191400423866 -94.45609765641535) bank45247 +45248 POINT(30.624509102600957 -95.08916807014137) bank45248 +45249 POINT(29.935722339205228 -95.08732481967812) bank45249 +45250 POINT(29.029333800492605 -94.76044158545139) bank45250 +45251 POINT(30.17864814025087 -95.39919714571315) bank45251 +45252 POINT(29.494815409299665 -94.86063413891469) bank45252 +45253 POINT(29.526871109187308 -95.85171638000705) bank45253 +45254 POINT(30.59132182217675 -95.28050389753567) bank45254 +45255 POINT(30.05854143876294 -95.14480058932179) bank45255 +45256 POINT(28.959750296651748 -95.05393816282921) bank45256 +45257 POINT(28.82084586408763 -95.73334010362575) bank45257 +45258 POINT(29.63011477269232 -95.23101340110692) bank45258 +45259 POINT(30.47881721751875 -96.34128279772948) bank45259 +45260 POINT(29.752588802486894 -95.64770377632605) bank45260 +45261 POINT(30.158831634632715 -95.65989852681199) bank45261 +45262 POINT(29.09558585602196 -95.88388188035157) bank45262 +45263 POINT(29.784640383544804 -95.81534812868621) bank45263 +45264 POINT(29.02015322886556 -95.63443744737236) bank45264 +45265 POINT(30.109340710910477 -95.0185806372047) bank45265 +45266 POINT(30.439041648922714 -94.63530749502122) bank45266 +45267 POINT(29.951317607845148 -95.8153284444793) bank45267 +45268 POINT(28.98108942777888 -96.00997350987014) bank45268 +45269 POINT(29.393769190184408 -95.47410627605251) bank45269 +45270 POINT(29.504180783497834 -95.00835820867132) bank45270 +45271 POINT(30.529742757558683 -94.37523242724386) bank45271 +45272 POINT(29.005500422062674 -94.79439510205212) bank45272 +45273 POINT(30.100561438326057 -94.44501400377875) bank45273 +45274 POINT(29.33321960774119 -95.4629730967711) bank45274 +45275 POINT(29.556526917564565 -95.84024784558576) bank45275 +45276 POINT(30.540925529989828 -95.81477717917177) bank45276 +45277 POINT(30.05497448611613 -95.36341689651803) bank45277 +45278 POINT(30.000414458174102 -95.56180760400517) bank45278 +45279 POINT(30.318807819252932 -94.6739125554316) bank45279 +45280 POINT(29.11542125406607 -94.79398465973486) bank45280 +45281 POINT(29.658414416111018 -95.40585575868103) bank45281 +45282 POINT(29.36682802839216 -95.30886652824933) bank45282 +45283 POINT(29.26814910318858 -95.20424263932654) bank45283 +45284 POINT(29.35877617703525 -94.99253124470724) bank45284 +45285 POINT(30.285004139073216 -96.0901742587669) bank45285 +45286 POINT(30.231645901863452 -96.36471761016924) bank45286 +45287 POINT(30.01826623066695 -95.61782700351789) bank45287 +45288 POINT(29.539567425485444 -96.18082737232285) bank45288 +45289 POINT(30.49471406421401 -95.6091266710733) bank45289 +45290 POINT(30.147034278526124 -95.47494778361246) bank45290 +45291 POINT(29.492486354547943 -96.25137065844424) bank45291 +45292 POINT(30.605970022034214 -95.80921985732999) bank45292 +45293 POINT(30.740566237028872 -95.73906724172718) bank45293 +45294 POINT(28.964204107882274 -94.83344189727568) bank45294 +45295 POINT(28.78149055984914 -95.18474101170717) bank45295 +45296 POINT(29.562016384665913 -95.02465710336357) bank45296 +45297 POINT(30.1154160923122 -95.92597141654039) bank45297 +45298 POINT(30.469016671982878 -95.42382868540663) bank45298 +45299 POINT(29.666509978296183 -94.47682949995993) bank45299 +45300 POINT(29.391835565216816 -94.88027604947092) bank45300 +45301 POINT(30.468600233788653 -95.09579206688959) bank45301 +45302 POINT(30.14583891032046 -96.3140724080997) bank45302 +45303 POINT(29.26401488881121 -95.53590127147257) bank45303 +45304 POINT(30.436045759979656 -94.82373531161264) bank45304 +45305 POINT(29.81106424171239 -96.07888581981595) bank45305 +45306 POINT(29.56470775433187 -95.37845976924962) bank45306 +45307 POINT(29.431585217325246 -95.77401137448255) bank45307 +45308 POINT(30.174341694155363 -95.25000078635323) bank45308 +45309 POINT(30.2591161929101 -94.43943577150137) bank45309 +45310 POINT(30.722984459247467 -94.92818372496387) bank45310 +45311 POINT(29.9149483635695 -94.69677396900346) bank45311 +45312 POINT(28.987026680621025 -95.549758873915) bank45312 +45313 POINT(29.740508393671416 -94.62570839146935) bank45313 +45314 POINT(29.896106659995617 -96.17801920172603) bank45314 +45315 POINT(29.21165525184957 -96.0769300119347) bank45315 +45316 POINT(29.843171074757063 -96.17139882533024) bank45316 +45317 POINT(30.458931136063462 -94.8681517596435) bank45317 +45318 POINT(29.548735349547293 -95.16746350566764) bank45318 +45319 POINT(29.534728090968613 -95.85600012571703) bank45319 +45320 POINT(29.54282816236254 -94.56308123667841) bank45320 +45321 POINT(30.325025787047828 -95.20603687022351) bank45321 +45322 POINT(30.713621160668414 -94.6467313370337) bank45322 +45323 POINT(30.628964031435583 -94.96793253871327) bank45323 +45324 POINT(29.263190181001313 -95.94093817141314) bank45324 +45325 POINT(29.354137789148652 -95.49421434737341) bank45325 +45326 POINT(29.795776533521842 -94.89721498940385) bank45326 +45327 POINT(28.934363683405646 -96.29676913678348) bank45327 +45328 POINT(30.35932674591219 -96.2041380253867) bank45328 +45329 POINT(30.10705641624781 -94.69636541771781) bank45329 +45330 POINT(29.141624405844343 -95.43699753881499) bank45330 +45331 POINT(29.59544268127479 -95.26944360333445) bank45331 +45332 POINT(29.37177408146149 -94.42784391670685) bank45332 +45333 POINT(30.489726452078273 -95.39614130993687) bank45333 +45334 POINT(29.125545273112028 -94.76621314548376) bank45334 +45335 POINT(29.19671885949192 -95.70351041792536) bank45335 +45336 POINT(28.791199161542192 -95.48657164412195) bank45336 +45337 POINT(29.306248621044013 -96.11517735573234) bank45337 +45338 POINT(28.963147609589083 -94.4042760876176) bank45338 +45339 POINT(29.49913230884475 -94.75453775117018) bank45339 +45340 POINT(29.914853770188238 -95.4494006803343) bank45340 +45341 POINT(29.45405283136344 -94.49328533346076) bank45341 +45342 POINT(30.69493003061755 -94.42713779379605) bank45342 +45343 POINT(29.144465244888472 -96.2173456015618) bank45343 +45344 POINT(28.975871031915865 -96.34466822539498) bank45344 +45345 POINT(29.389350984736264 -95.55177217233022) bank45345 +45346 POINT(29.753206156858965 -95.86162022519918) bank45346 +45347 POINT(29.24715278694829 -95.73762816789872) bank45347 +45348 POINT(29.97250293074485 -95.69397836045844) bank45348 +45349 POINT(29.136733301299326 -96.19224614094779) bank45349 +45350 POINT(30.209301426910177 -96.20148031699635) bank45350 +45351 POINT(29.123741793792153 -94.55479821236608) bank45351 +45352 POINT(28.79033852743586 -95.2138431442638) bank45352 +45353 POINT(28.8465791784885 -96.35107745711898) bank45353 +45354 POINT(29.79174261585524 -94.94153253692767) bank45354 +45355 POINT(29.988092534152706 -94.44676497425966) bank45355 +45356 POINT(30.58298552277826 -96.3156087126808) bank45356 +45357 POINT(30.21237649648233 -95.80171181552565) bank45357 +45358 POINT(28.793047416358664 -94.78155673054445) bank45358 +45359 POINT(29.46698012507965 -96.23653505784003) bank45359 +45360 POINT(30.241038312132275 -96.15016550578248) bank45360 +45361 POINT(28.771859196599976 -96.1018713621876) bank45361 +45362 POINT(29.061121012683955 -95.85993444279137) bank45362 +45363 POINT(28.918203241607205 -95.24947518090009) bank45363 +45364 POINT(29.957539976678433 -96.21751276523794) bank45364 +45365 POINT(29.71198502798037 -94.39528182838002) bank45365 +45366 POINT(28.81463607600817 -96.28989249903634) bank45366 +45367 POINT(29.742317833430832 -96.33070213477293) bank45367 +45368 POINT(30.343648684586178 -95.07818506172094) bank45368 +45369 POINT(30.62869829812354 -95.08178503768545) bank45369 +45370 POINT(28.910335953988692 -96.21841890530997) bank45370 +45371 POINT(28.852938199856027 -95.54836297239964) bank45371 +45372 POINT(28.955822557240822 -95.52397902760087) bank45372 +45373 POINT(29.698151650540282 -94.41567149666295) bank45373 +45374 POINT(30.091734226868635 -96.11217536263935) bank45374 +45375 POINT(29.28898284744653 -96.22138371061925) bank45375 +45376 POINT(29.348449665368918 -95.75156697782874) bank45376 +45377 POINT(29.663503372274896 -95.24546418692401) bank45377 +45378 POINT(30.149882929170616 -96.20058380658911) bank45378 +45379 POINT(29.895690632076317 -94.42196634070446) bank45379 +45380 POINT(30.440056702407645 -95.61352616347527) bank45380 +45381 POINT(30.667340099475116 -96.22146921730716) bank45381 +45382 POINT(29.888813848820256 -95.67392172384126) bank45382 +45383 POINT(29.626186141571424 -95.81264559818668) bank45383 +45384 POINT(28.98848388596963 -94.79261272996978) bank45384 +45385 POINT(30.151017177577042 -96.35916125931541) bank45385 +45386 POINT(29.199058414378737 -95.5597339183718) bank45386 +45387 POINT(30.335016030176053 -95.49288477274978) bank45387 +45388 POINT(29.41033595490992 -95.03880469109147) bank45388 +45389 POINT(30.39907208633906 -95.69340143596105) bank45389 +45390 POINT(29.83466098925396 -94.87080559508945) bank45390 +45391 POINT(30.654123184591047 -95.8582218697654) bank45391 +45392 POINT(30.652875330866973 -94.49234564668252) bank45392 +45393 POINT(29.391039589850006 -94.68553416544385) bank45393 +45394 POINT(29.458566960088177 -94.54398517745226) bank45394 +45395 POINT(29.279478142771463 -94.76110563754682) bank45395 +45396 POINT(30.37002213738777 -94.97679720463191) bank45396 +45397 POINT(30.474896325141692 -96.3613275706559) bank45397 +45398 POINT(29.255207914223917 -95.85261064794588) bank45398 +45399 POINT(30.307263720011463 -95.81358028398351) bank45399 +45400 POINT(30.612058357858455 -94.7274288679833) bank45400 +45401 POINT(29.609496538704914 -94.52840050865838) bank45401 +45402 POINT(29.2252220178569 -95.34109434305094) bank45402 +45403 POINT(29.850688070340752 -94.59397643530367) bank45403 +45404 POINT(29.118396845647588 -95.0857464894016) bank45404 +45405 POINT(28.95565824213163 -94.37503295586862) bank45405 +45406 POINT(29.000834903806503 -94.49748071511326) bank45406 +45407 POINT(29.45342129895717 -96.06931523138364) bank45407 +45408 POINT(30.35551359436891 -94.87343083329391) bank45408 +45409 POINT(29.71017578916697 -94.9037839618294) bank45409 +45410 POINT(29.1293627496253 -95.13793993591266) bank45410 +45411 POINT(29.14218655010125 -95.25227860070721) bank45411 +45412 POINT(30.494734172142856 -96.07543608357217) bank45412 +45413 POINT(29.14098375353368 -94.39438875668974) bank45413 +45414 POINT(30.260754309566043 -95.35712853510624) bank45414 +45415 POINT(29.001508432298674 -96.25881348780557) bank45415 +45416 POINT(29.934344532843888 -94.78103123633808) bank45416 +45417 POINT(30.682017294260255 -96.04403594214759) bank45417 +45418 POINT(30.171617315118024 -95.55095160655847) bank45418 +45419 POINT(29.29002932318662 -96.06933803654056) bank45419 +45420 POINT(28.92470506886207 -95.24909428711703) bank45420 +45421 POINT(29.75178993950862 -94.8280071737941) bank45421 +45422 POINT(29.392755858286918 -96.2857171269817) bank45422 +45423 POINT(29.602808802620732 -95.92458758061993) bank45423 +45424 POINT(29.649191369353527 -94.42737133458283) bank45424 +45425 POINT(30.353480047755625 -95.90854886965407) bank45425 +45426 POINT(29.010640883208207 -96.34868042591849) bank45426 +45427 POINT(29.845894175248304 -95.28424432178952) bank45427 +45428 POINT(30.512890812509823 -94.56198971800758) bank45428 +45429 POINT(30.025595232267236 -95.24178513643223) bank45429 +45430 POINT(30.21510061387465 -95.26689774655395) bank45430 +45431 POINT(29.184024571929495 -95.55401253631581) bank45431 +45432 POINT(29.347263431169978 -94.9765963319882) bank45432 +45433 POINT(29.844771569568362 -94.60674467851295) bank45433 +45434 POINT(29.729314114983776 -95.97425408334269) bank45434 +45435 POINT(28.862147565682513 -95.89201407267186) bank45435 +45436 POINT(30.013280411840533 -96.0262084108682) bank45436 +45437 POINT(30.675397776852677 -94.72781899098058) bank45437 +45438 POINT(29.53555994898954 -94.97157749050875) bank45438 +45439 POINT(30.573829364209914 -94.61467773767141) bank45439 +45440 POINT(29.93447825368024 -95.02389514944733) bank45440 +45441 POINT(30.170840797640572 -95.66626497449955) bank45441 +45442 POINT(30.212910215253387 -94.96648036443034) bank45442 +45443 POINT(29.88683571725389 -95.12889889177482) bank45443 +45444 POINT(30.039031731677795 -95.56690496718758) bank45444 +45445 POINT(29.170388373563654 -96.30616496777526) bank45445 +45446 POINT(29.720422833766875 -94.94796082095968) bank45446 +45447 POINT(29.067019574930118 -95.41038017632437) bank45447 +45448 POINT(30.061035276879565 -95.27689432656564) bank45448 +45449 POINT(30.37681368695113 -95.6553908390702) bank45449 +45450 POINT(30.193209974718265 -95.64851178080701) bank45450 +45451 POINT(29.3552534121863 -94.64738702121002) bank45451 +45452 POINT(30.147142564951697 -95.28043817987611) bank45452 +45453 POINT(29.6916719200362 -94.62060564835203) bank45453 +45454 POINT(29.052952465240697 -94.89540365936213) bank45454 +45455 POINT(29.434711709303862 -94.75382691026577) bank45455 +45456 POINT(29.358554343694195 -94.77532675277489) bank45456 +45457 POINT(30.541905413463695 -94.74737678900733) bank45457 +45458 POINT(28.850882932103143 -95.54678422210283) bank45458 +45459 POINT(30.396522168965923 -94.82056411012604) bank45459 +45460 POINT(28.76851550351904 -95.62499596013976) bank45460 +45461 POINT(29.709856303432396 -94.46055377122926) bank45461 +45462 POINT(29.08681719297594 -95.38511031554238) bank45462 +45463 POINT(30.488009536883528 -94.42818267615974) bank45463 +45464 POINT(29.817349320041462 -95.99839655551847) bank45464 +45465 POINT(30.095163070353237 -94.67693273244787) bank45465 +45466 POINT(28.843084230707728 -94.40696444807162) bank45466 +45467 POINT(29.29236369419208 -95.97465290219179) bank45467 +45468 POINT(29.43790317769244 -95.74801575546502) bank45468 +45469 POINT(29.026447753200564 -94.88469174404761) bank45469 +45470 POINT(28.81149790145913 -95.46971241058043) bank45470 +45471 POINT(29.28004292734721 -95.24025923328574) bank45471 +45472 POINT(29.903177593529186 -96.2591875247688) bank45472 +45473 POINT(30.455175593659003 -96.17909248777917) bank45473 +45474 POINT(28.957800503714296 -95.88537001164848) bank45474 +45475 POINT(30.00290664831364 -94.57302845215818) bank45475 +45476 POINT(30.56406164247988 -95.19581423050579) bank45476 +45477 POINT(29.45467584698067 -95.4669996891625) bank45477 +45478 POINT(28.808709165349473 -94.55097468854903) bank45478 +45479 POINT(29.014523812215792 -95.51235397079822) bank45479 +45480 POINT(29.677359503883384 -96.10664688917625) bank45480 +45481 POINT(29.76882812403293 -95.49441261551485) bank45481 +45482 POINT(29.30314740661054 -95.44944414049412) bank45482 +45483 POINT(30.20992380364865 -95.75819835693112) bank45483 +45484 POINT(29.374033500261884 -95.70013831984755) bank45484 +45485 POINT(28.961409115490145 -94.46937920844995) bank45485 +45486 POINT(29.181551765067084 -94.5997148245683) bank45486 +45487 POINT(29.783921469896494 -96.00876480204596) bank45487 +45488 POINT(29.686932282157105 -95.00632323043018) bank45488 +45489 POINT(30.568687373217994 -94.48645010799909) bank45489 +45490 POINT(29.0574988947163 -95.47353738597849) bank45490 +45491 POINT(29.372637349063247 -95.52986806211239) bank45491 +45492 POINT(30.08266145203871 -95.74676061037897) bank45492 +45493 POINT(30.043494150250012 -95.5886361536499) bank45493 +45494 POINT(29.890034743963337 -95.52007192349753) bank45494 +45495 POINT(29.541790788451134 -94.54777024058788) bank45495 +45496 POINT(28.858266668796524 -95.55457263887062) bank45496 +45497 POINT(30.225763395445767 -95.26789964153265) bank45497 +45498 POINT(28.832838572730676 -94.56880567542333) bank45498 +45499 POINT(29.421217922402406 -95.53492814507715) bank45499 +45500 POINT(30.72857877406328 -95.18851418674136) bank45500 +45501 POINT(29.928485482009883 -96.31660014153724) bank45501 +45502 POINT(30.36225248807075 -95.2132048841717) bank45502 +45503 POINT(29.05623393048799 -96.02350750424159) bank45503 +45504 POINT(29.901517626223935 -96.32655495830114) bank45504 +45505 POINT(30.4778423742662 -95.01641140698251) bank45505 +45506 POINT(30.237434838481647 -94.80524161032454) bank45506 +45507 POINT(28.958856950525764 -94.84663906417876) bank45507 +45508 POINT(30.11061183005624 -95.04275018068368) bank45508 +45509 POINT(29.41842443074584 -95.80434331523112) bank45509 +45510 POINT(30.689113385517548 -95.66830682468299) bank45510 +45511 POINT(30.132256639327185 -96.36814095337965) bank45511 +45512 POINT(28.88869129818007 -94.99601131362834) bank45512 +45513 POINT(29.833305584244076 -95.71988306618192) bank45513 +45514 POINT(29.624899914329927 -95.64257010872961) bank45514 +45515 POINT(29.154278153155428 -96.14373954424501) bank45515 +45516 POINT(29.833651556721527 -96.3627841001721) bank45516 +45517 POINT(30.470619153937037 -94.67996888621037) bank45517 +45518 POINT(30.069213395824562 -96.0922162619235) bank45518 +45519 POINT(29.17966850010486 -95.38784520257066) bank45519 +45520 POINT(30.55581695801396 -95.46496531763273) bank45520 +45521 POINT(30.0095624188658 -95.85484696211056) bank45521 +45522 POINT(30.09576988867833 -94.60343267221495) bank45522 +45523 POINT(29.590900095444713 -94.3709262825743) bank45523 +45524 POINT(28.969953562634306 -95.50833828182341) bank45524 +45525 POINT(28.812284151685276 -94.94363757177697) bank45525 +45526 POINT(29.071951719374866 -95.44886736877463) bank45526 +45527 POINT(30.302994734875554 -96.16996955195084) bank45527 +45528 POINT(30.52654197982318 -95.8072727907037) bank45528 +45529 POINT(28.91426442040118 -95.66281509134464) bank45529 +45530 POINT(30.708873583250693 -95.96969214186208) bank45530 +45531 POINT(30.451254435511604 -94.38877522437004) bank45531 +45532 POINT(30.567688718939852 -95.72861730706866) bank45532 +45533 POINT(29.13680900107325 -94.99531816698554) bank45533 +45534 POINT(30.459436527738774 -95.6848499675186) bank45534 +45535 POINT(29.84151198436162 -96.13719021101058) bank45535 +45536 POINT(29.305572763108948 -94.50837620580026) bank45536 +45537 POINT(29.722419718422522 -96.34178432951099) bank45537 +45538 POINT(29.109197285242416 -95.4692720164851) bank45538 +45539 POINT(29.52535312278039 -95.44872654470979) bank45539 +45540 POINT(30.42087841929409 -95.75584947189387) bank45540 +45541 POINT(28.768997697407343 -95.96372473917837) bank45541 +45542 POINT(29.416680201937616 -94.62035820465447) bank45542 +45543 POINT(30.362090691630073 -94.50852426067158) bank45543 +45544 POINT(29.05142787809281 -95.57893309654635) bank45544 +45545 POINT(29.239780693894403 -94.8585907102872) bank45545 +45546 POINT(30.615820801792044 -94.5178131264846) bank45546 +45547 POINT(29.31119028225914 -96.23944134380243) bank45547 +45548 POINT(30.233770086550805 -96.14265715707458) bank45548 +45549 POINT(29.19923513152759 -96.23522988103936) bank45549 +45550 POINT(30.04905105078558 -94.92499372385929) bank45550 +45551 POINT(29.775378934357196 -96.31207995863518) bank45551 +45552 POINT(29.058779438703635 -96.09173905214227) bank45552 +45553 POINT(30.430736498138383 -95.73304909456772) bank45553 +45554 POINT(28.859258945436466 -95.00363503597004) bank45554 +45555 POINT(30.013805650304526 -96.1449453475352) bank45555 +45556 POINT(29.312458668574823 -96.03084006182819) bank45556 +45557 POINT(29.248363162803646 -95.51373047389053) bank45557 +45558 POINT(30.246982405383832 -95.0058363993537) bank45558 +45559 POINT(28.86735346942457 -96.34520969005665) bank45559 +45560 POINT(30.61072310520122 -96.11924361870894) bank45560 +45561 POINT(28.79767892009371 -94.74976115460329) bank45561 +45562 POINT(29.47362483251727 -94.9024801546767) bank45562 +45563 POINT(30.34083817891823 -95.70234290075685) bank45563 +45564 POINT(29.551537015318523 -96.02157656067146) bank45564 +45565 POINT(28.76652233167712 -96.18651856047572) bank45565 +45566 POINT(30.419088824222257 -94.62667127425374) bank45566 +45567 POINT(30.040319723746077 -94.57206638476666) bank45567 +45568 POINT(28.891815769308764 -96.13096362721973) bank45568 +45569 POINT(28.862831631372167 -95.45615168364391) bank45569 +45570 POINT(30.045680415794326 -95.66111421750253) bank45570 +45571 POINT(29.887247892418685 -96.11414669366431) bank45571 +45572 POINT(30.691346895431142 -95.69219045561279) bank45572 +45573 POINT(29.111838216130526 -95.94501936897214) bank45573 +45574 POINT(30.617230612096176 -95.8085965170725) bank45574 +45575 POINT(29.825698533389097 -95.52443498137407) bank45575 +45576 POINT(29.67445838991924 -95.13489745593535) bank45576 +45577 POINT(29.650184134043784 -95.30051312148329) bank45577 +45578 POINT(29.693547206924887 -94.83942753360569) bank45578 +45579 POINT(29.935759859094723 -96.15523825672997) bank45579 +45580 POINT(30.60612972189155 -95.00641063853836) bank45580 +45581 POINT(30.351747744441365 -95.70687689635524) bank45581 +45582 POINT(30.741964929388754 -96.33007519078639) bank45582 +45583 POINT(30.37691566622554 -96.31970548269301) bank45583 +45584 POINT(30.55234061281851 -94.64966527454416) bank45584 +45585 POINT(29.420029141716228 -94.38279203549897) bank45585 +45586 POINT(30.049882247897486 -94.8257574271367) bank45586 +45587 POINT(28.864086131957574 -95.84898828401734) bank45587 +45588 POINT(29.840531760178745 -95.31895547384073) bank45588 +45589 POINT(28.852437911691755 -96.24855416687075) bank45589 +45590 POINT(29.1430323690998 -96.25577402014504) bank45590 +45591 POINT(30.219327679894466 -95.22840930079387) bank45591 +45592 POINT(29.46184486889896 -95.0616090077342) bank45592 +45593 POINT(29.95057120678984 -96.25507291146208) bank45593 +45594 POINT(29.747510307867522 -95.13105052149734) bank45594 +45595 POINT(28.88756775672782 -95.49415785388643) bank45595 +45596 POINT(30.186832741762174 -96.10458667916029) bank45596 +45597 POINT(29.620708306604307 -96.15440695960518) bank45597 +45598 POINT(28.839829937027137 -95.96944487853607) bank45598 +45599 POINT(30.72162175527736 -94.56945203412674) bank45599 +45600 POINT(29.43309927465808 -94.76164457301607) bank45600 +45601 POINT(29.828893798513814 -95.36185314193095) bank45601 +45602 POINT(29.0224434008888 -95.31841784182396) bank45602 +45603 POINT(28.90094117497035 -95.7032310860164) bank45603 +45604 POINT(29.83576525755086 -95.19808867603592) bank45604 +45605 POINT(30.30027548726921 -95.46707593596118) bank45605 +45606 POINT(29.78511602685493 -96.15173592002412) bank45606 +45607 POINT(28.8079361200839 -95.37566937051369) bank45607 +45608 POINT(29.5634320202038 -95.42829918519668) bank45608 +45609 POINT(30.549372230891365 -96.117881279084) bank45609 +45610 POINT(29.86439227577179 -95.08795414757566) bank45610 +45611 POINT(29.89637652743233 -95.55831315196909) bank45611 +45612 POINT(29.486082605458172 -95.35630900835747) bank45612 +45613 POINT(30.696713047218307 -96.16412870426929) bank45613 +45614 POINT(29.063995383851868 -95.54582900917197) bank45614 +45615 POINT(29.764115486929978 -94.53452912779458) bank45615 +45616 POINT(29.01170283323259 -94.76297161752767) bank45616 +45617 POINT(29.66560285177278 -95.89745929608287) bank45617 +45618 POINT(30.402799958075995 -95.66090676962114) bank45618 +45619 POINT(29.44876417452584 -94.54878738937602) bank45619 +45620 POINT(29.551024313591306 -94.72912559130403) bank45620 +45621 POINT(28.824848976906353 -95.78560541077782) bank45621 +45622 POINT(30.217437153717917 -94.82121380009593) bank45622 +45623 POINT(28.907769628813536 -95.66198497926122) bank45623 +45624 POINT(29.525095305104852 -95.07304027229634) bank45624 +45625 POINT(29.761030719181257 -96.26454704245252) bank45625 +45626 POINT(29.268667006833653 -94.86309947306252) bank45626 +45627 POINT(29.58843408263059 -95.71356570190322) bank45627 +45628 POINT(29.103832411935077 -94.78626927806131) bank45628 +45629 POINT(29.669246693473276 -94.56117300990104) bank45629 +45630 POINT(29.80870250727692 -94.86281459392393) bank45630 +45631 POINT(29.66156383867925 -96.20131521862237) bank45631 +45632 POINT(29.893938530162732 -95.88365764330112) bank45632 +45633 POINT(28.800751600922233 -95.37184936101644) bank45633 +45634 POINT(30.657739628806553 -95.19548884194903) bank45634 +45635 POINT(28.818634278636985 -96.26114763627399) bank45635 +45636 POINT(29.209475643742277 -95.92214012576078) bank45636 +45637 POINT(30.414454215808068 -96.14544463773815) bank45637 +45638 POINT(30.650129823350998 -94.84886340472745) bank45638 +45639 POINT(30.63340734873573 -95.17692432048479) bank45639 +45640 POINT(30.152129962952728 -95.0716747362536) bank45640 +45641 POINT(30.728073687556883 -95.09036992770616) bank45641 +45642 POINT(28.763511705972817 -94.7113122564562) bank45642 +45643 POINT(30.15919781276757 -95.94411475080138) bank45643 +45644 POINT(29.467101855311547 -95.64111722844046) bank45644 +45645 POINT(29.526916555276834 -95.49586018437255) bank45645 +45646 POINT(29.751107816132482 -94.57196113742205) bank45646 +45647 POINT(29.651674559919467 -94.77716143832377) bank45647 +45648 POINT(28.768045156268165 -96.24569176394157) bank45648 +45649 POINT(30.718951377303572 -96.22427551072906) bank45649 +45650 POINT(29.11172628083245 -94.76315772690633) bank45650 +45651 POINT(30.20574392045808 -95.35141247429476) bank45651 +45652 POINT(29.81682043579805 -95.6813310572511) bank45652 +45653 POINT(29.569490382969917 -96.2648479417694) bank45653 +45654 POINT(30.52771375009888 -94.58754268704901) bank45654 +45655 POINT(29.246879708245377 -95.17286444438882) bank45655 +45656 POINT(29.074332687226732 -94.63813549810365) bank45656 +45657 POINT(30.150873132966463 -94.89080116613907) bank45657 +45658 POINT(30.28079351925489 -96.26607140712443) bank45658 +45659 POINT(29.849811280532744 -95.06092739696777) bank45659 +45660 POINT(29.954230792669716 -96.01997081357247) bank45660 +45661 POINT(29.32211783796371 -94.79258785504217) bank45661 +45662 POINT(30.524541888106302 -95.80165310626296) bank45662 +45663 POINT(30.463290058330053 -95.33673881786298) bank45663 +45664 POINT(28.9872913317414 -95.22972440211265) bank45664 +45665 POINT(30.110703927689453 -94.40978727505326) bank45665 +45666 POINT(30.120519097322504 -96.22481979702142) bank45666 +45667 POINT(29.257793490139047 -94.71219934324346) bank45667 +45668 POINT(29.600964856537942 -95.76418176386503) bank45668 +45669 POINT(29.4449120360961 -95.66691977447856) bank45669 +45670 POINT(29.161499007487627 -95.182850779501) bank45670 +45671 POINT(30.366789478068075 -95.83138588951319) bank45671 +45672 POINT(30.083258868654124 -96.20525181002508) bank45672 +45673 POINT(29.094722352101538 -94.38318964397865) bank45673 +45674 POINT(30.02879494308769 -95.07367792916963) bank45674 +45675 POINT(30.381069815412474 -96.26590247395893) bank45675 +45676 POINT(28.768460202534715 -94.41378630483032) bank45676 +45677 POINT(29.544319426512725 -96.30266987252342) bank45677 +45678 POINT(28.832600534043205 -95.81719358595015) bank45678 +45679 POINT(30.679077376969573 -96.34799013522381) bank45679 +45680 POINT(29.58480214615097 -94.6218837913065) bank45680 +45681 POINT(30.4515485668175 -94.59098828069301) bank45681 +45682 POINT(29.604642627351346 -95.53889795921097) bank45682 +45683 POINT(30.667708815528936 -96.14164032724) bank45683 +45684 POINT(30.298271659852713 -94.56405643036327) bank45684 +45685 POINT(30.250183284499315 -96.28365860011961) bank45685 +45686 POINT(29.645594751401465 -95.48574349831752) bank45686 +45687 POINT(30.468806689004367 -96.11531999172368) bank45687 +45688 POINT(29.196572827404545 -95.83765898451905) bank45688 +45689 POINT(29.38677189230951 -94.9565054438839) bank45689 +45690 POINT(30.42856611745685 -95.08238288681433) bank45690 +45691 POINT(30.455059302079007 -95.8040504475673) bank45691 +45692 POINT(29.056838266291162 -96.12370737184737) bank45692 +45693 POINT(29.63738301168889 -94.58480461852389) bank45693 +45694 POINT(28.899806723121337 -95.36698989470551) bank45694 +45695 POINT(29.217599536737957 -94.83988943519101) bank45695 +45696 POINT(30.413598642466283 -94.94771015674621) bank45696 +45697 POINT(30.28819861734229 -95.50785182268157) bank45697 +45698 POINT(28.98179462161606 -95.87985024291768) bank45698 +45699 POINT(30.421158374273617 -95.66872148295298) bank45699 +45700 POINT(29.643763154873657 -94.38736595105796) bank45700 +45701 POINT(30.530056197147502 -95.81439666310055) bank45701 +45702 POINT(29.646743432742102 -94.94776015952485) bank45702 +45703 POINT(28.877882462697134 -95.74275675862043) bank45703 +45704 POINT(30.352267925151875 -94.97605112899694) bank45704 +45705 POINT(29.882784081898283 -95.83864830645129) bank45705 +45706 POINT(29.27826561894352 -95.65129777827362) bank45706 +45707 POINT(29.8030171842415 -95.62579110071908) bank45707 +45708 POINT(28.98294402564692 -95.01861735489068) bank45708 +45709 POINT(30.47449181213704 -95.84906095498185) bank45709 +45710 POINT(30.406684109441095 -96.21253669088215) bank45710 +45711 POINT(29.745316203984746 -95.66504144894411) bank45711 +45712 POINT(29.63001525314758 -95.61257531505547) bank45712 +45713 POINT(29.728814544747 -94.91965060258144) bank45713 +45714 POINT(30.66477059506785 -95.2746919199248) bank45714 +45715 POINT(29.509530372745644 -95.62922157985986) bank45715 +45716 POINT(29.741842802600534 -95.90271111676363) bank45716 +45717 POINT(29.975753398132472 -95.07587168908866) bank45717 +45718 POINT(29.170899926453323 -96.03384969735815) bank45718 +45719 POINT(30.020700788076294 -96.1729693477244) bank45719 +45720 POINT(29.03113081644724 -95.16774035536773) bank45720 +45721 POINT(29.344636035535114 -95.75709557498222) bank45721 +45722 POINT(29.297332677080565 -96.25076136585282) bank45722 +45723 POINT(29.203888571347445 -94.8869452613108) bank45723 +45724 POINT(29.021563878698686 -96.14590312743063) bank45724 +45725 POINT(29.514658020554027 -95.09442645329369) bank45725 +45726 POINT(29.625800157376446 -94.39114238294165) bank45726 +45727 POINT(30.332314316822565 -95.81720188115683) bank45727 +45728 POINT(29.33349869895825 -96.10414524213066) bank45728 +45729 POINT(30.429827586882883 -94.75638146485117) bank45729 +45730 POINT(30.433312850495675 -95.5356842197408) bank45730 +45731 POINT(28.886249419325555 -95.90759041974891) bank45731 +45732 POINT(30.288229069508013 -95.49119450446464) bank45732 +45733 POINT(29.745701647944287 -94.62465642059023) bank45733 +45734 POINT(29.51264279181137 -94.45220835757392) bank45734 +45735 POINT(30.375373051038007 -96.23867514912214) bank45735 +45736 POINT(30.13437502040424 -95.21201885323117) bank45736 +45737 POINT(29.38699930006963 -95.73914976353349) bank45737 +45738 POINT(29.06476231937981 -94.82525394001544) bank45738 +45739 POINT(29.437459579749465 -96.34081250363462) bank45739 +45740 POINT(29.13969410544508 -95.57009677642696) bank45740 +45741 POINT(30.03553825527081 -94.48417043935576) bank45741 +45742 POINT(30.473858511637566 -95.88650753971338) bank45742 +45743 POINT(30.10339095777707 -95.25378484183433) bank45743 +45744 POINT(29.295407182748715 -95.5269997530198) bank45744 +45745 POINT(29.924498372680286 -95.18776001253566) bank45745 +45746 POINT(29.364249280545494 -95.54668127494072) bank45746 +45747 POINT(28.935884682033404 -95.5598484306871) bank45747 +45748 POINT(28.87471473079436 -95.57544972996044) bank45748 +45749 POINT(29.47570201096878 -95.32129309615206) bank45749 +45750 POINT(29.214265907789855 -95.99177298935201) bank45750 +45751 POINT(30.11794807942127 -94.8758821187376) bank45751 +45752 POINT(28.76888590810362 -94.95309267731103) bank45752 +45753 POINT(30.009419991834875 -96.14211192695728) bank45753 +45754 POINT(30.00685276321283 -95.91964043845162) bank45754 +45755 POINT(29.518521219960995 -95.76001408417778) bank45755 +45756 POINT(30.099658951810106 -96.31945518658235) bank45756 +45757 POINT(30.501739806107324 -94.58177450345862) bank45757 +45758 POINT(30.333617293814864 -95.36193320667697) bank45758 +45759 POINT(29.906471925641704 -95.25475071965349) bank45759 +45760 POINT(28.765844635166243 -95.59188438800547) bank45760 +45761 POINT(29.392443857346272 -94.44294467958737) bank45761 +45762 POINT(29.067676901225152 -96.0997993709488) bank45762 +45763 POINT(30.12717298058213 -94.92534609941123) bank45763 +45764 POINT(29.065762603830088 -96.3676989654049) bank45764 +45765 POINT(29.789579073890348 -94.97503632719223) bank45765 +45766 POINT(30.242270953495396 -95.1559078117678) bank45766 +45767 POINT(30.37371715774083 -94.93922623048923) bank45767 +45768 POINT(28.97681761513037 -95.61062401284994) bank45768 +45769 POINT(29.196417793143603 -95.93113132559724) bank45769 +45770 POINT(29.3866718844807 -96.00225713505826) bank45770 +45771 POINT(29.125143154151292 -95.77881138406975) bank45771 +45772 POINT(29.410456931621784 -95.59956430161967) bank45772 +45773 POINT(28.952157864945992 -94.7742594993063) bank45773 +45774 POINT(29.43851518092017 -95.98588526204847) bank45774 +45775 POINT(30.245285873040224 -94.52020752077406) bank45775 +45776 POINT(30.115904774993624 -94.52970749984694) bank45776 +45777 POINT(29.84876781896774 -95.92705612060233) bank45777 +45778 POINT(30.003973613447258 -95.33941062835761) bank45778 +45779 POINT(30.32639993727391 -94.80896490736372) bank45779 +45780 POINT(30.25151018611441 -96.22975810474594) bank45780 +45781 POINT(30.44175881814209 -94.89043433426862) bank45781 +45782 POINT(29.537093003059027 -96.09343612157697) bank45782 +45783 POINT(30.41579483276734 -95.02204582973062) bank45783 +45784 POINT(29.168747219027576 -95.6475997230984) bank45784 +45785 POINT(28.781001228117127 -95.44359078288505) bank45785 +45786 POINT(29.060756757867903 -96.11724331524186) bank45786 +45787 POINT(28.95146157946281 -95.56275678653222) bank45787 +45788 POINT(29.677862117681055 -95.32485504881093) bank45788 +45789 POINT(30.47210343542121 -95.36349481235942) bank45789 +45790 POINT(29.682261470847756 -95.69672212105277) bank45790 +45791 POINT(30.538337767652475 -94.63599676618826) bank45791 +45792 POINT(29.97908823943381 -95.46951592032586) bank45792 +45793 POINT(29.16648398500358 -95.33997461990957) bank45793 +45794 POINT(29.20025890288513 -95.07794005454453) bank45794 +45795 POINT(29.418247337201105 -94.48877532612728) bank45795 +45796 POINT(29.11775101910638 -95.43079318177773) bank45796 +45797 POINT(30.51973095123486 -96.0349422126423) bank45797 +45798 POINT(30.11404709451683 -94.5578995383459) bank45798 +45799 POINT(30.165737479244676 -94.80216393247152) bank45799 +45800 POINT(28.945696065851553 -94.8178542420114) bank45800 +45801 POINT(29.692527252837742 -95.05901063077798) bank45801 +45802 POINT(28.885540666123543 -96.00468550375797) bank45802 +45803 POINT(29.794149794632496 -94.63014179685163) bank45803 +45804 POINT(29.74449455181177 -94.40627308018014) bank45804 +45805 POINT(29.857882189146068 -95.21668913626145) bank45805 +45806 POINT(29.804084596528 -95.30857565018566) bank45806 +45807 POINT(29.61486229521604 -96.266081668234) bank45807 +45808 POINT(30.280945963300354 -96.08459763256512) bank45808 +45809 POINT(29.418219449441384 -94.3745694170704) bank45809 +45810 POINT(29.317639309172634 -96.30451104221892) bank45810 +45811 POINT(29.917549458514596 -95.69419431017322) bank45811 +45812 POINT(29.817111337045144 -95.28578430493634) bank45812 +45813 POINT(28.878476720692532 -94.45284397636165) bank45813 +45814 POINT(29.560100596523096 -96.08901456828231) bank45814 +45815 POINT(28.934379566257757 -94.90161442688331) bank45815 +45816 POINT(29.382221710501856 -95.17545595180859) bank45816 +45817 POINT(29.700067293377785 -94.45082165450859) bank45817 +45818 POINT(29.20446783416385 -94.89276070262937) bank45818 +45819 POINT(29.676267214010863 -94.98529385741602) bank45819 +45820 POINT(28.906879056574738 -95.01220427199951) bank45820 +45821 POINT(30.430631971088022 -95.93237552940268) bank45821 +45822 POINT(30.528111946379273 -95.55550601194176) bank45822 +45823 POINT(30.17455335675415 -94.38520891361544) bank45823 +45824 POINT(29.588783942937606 -95.59596479795525) bank45824 +45825 POINT(29.746537485031862 -95.58716288958098) bank45825 +45826 POINT(30.133956271224157 -96.18539007001652) bank45826 +45827 POINT(29.875209259232644 -95.70542162477034) bank45827 +45828 POINT(29.708867202559148 -95.8220856378617) bank45828 +45829 POINT(30.1977444273018 -96.26036427294547) bank45829 +45830 POINT(29.9307419457594 -95.55411137257734) bank45830 +45831 POINT(29.621704935000427 -95.21975530258038) bank45831 +45832 POINT(29.055324008239566 -95.78535011903959) bank45832 +45833 POINT(28.844256524009896 -95.73714769214713) bank45833 +45834 POINT(29.271895559089874 -94.93154636234055) bank45834 +45835 POINT(29.10756878482245 -95.94450550404657) bank45835 +45836 POINT(29.358205780547394 -95.43460529167864) bank45836 +45837 POINT(29.498554378652816 -95.79915355387924) bank45837 +45838 POINT(29.258107692634578 -95.44764609078001) bank45838 +45839 POINT(29.77772558270201 -94.88371157852099) bank45839 +45840 POINT(30.50381723996602 -95.38957714001148) bank45840 +45841 POINT(28.948115735717654 -94.61107663894971) bank45841 +45842 POINT(30.40553085032855 -95.494820465564) bank45842 +45843 POINT(29.185604579189757 -95.50038832736315) bank45843 +45844 POINT(30.426918108732156 -94.90405683388009) bank45844 +45845 POINT(30.521594336635946 -95.5352097032635) bank45845 +45846 POINT(30.526599085985282 -95.66315191975337) bank45846 +45847 POINT(28.880851560231477 -94.78110719010765) bank45847 +45848 POINT(30.109845455268363 -95.80767083422178) bank45848 +45849 POINT(30.342340335752617 -95.59553571039183) bank45849 +45850 POINT(30.53430579905736 -95.80902260750041) bank45850 +45851 POINT(29.353009345582166 -94.5023725948802) bank45851 +45852 POINT(30.409784016921737 -94.93382160317461) bank45852 +45853 POINT(29.784111475232343 -94.77728153221786) bank45853 +45854 POINT(30.141915007437046 -94.53340766397952) bank45854 +45855 POINT(30.537899303214843 -95.13607267349238) bank45855 +45856 POINT(30.107106448565048 -94.72876560915827) bank45856 +45857 POINT(29.587479349893385 -95.13579128806272) bank45857 +45858 POINT(30.10997883516067 -94.71044733107038) bank45858 +45859 POINT(30.62757029340069 -96.36462935970559) bank45859 +45860 POINT(30.233028604697573 -95.73791961786264) bank45860 +45861 POINT(30.086164832168745 -94.74146565631727) bank45861 +45862 POINT(30.56801591373715 -94.68650055096901) bank45862 +45863 POINT(30.469518165503054 -95.46476876095176) bank45863 +45864 POINT(29.580073561761754 -95.35430430398905) bank45864 +45865 POINT(30.485809826291543 -95.03472820174085) bank45865 +45866 POINT(29.8172891590509 -96.15087817510351) bank45866 +45867 POINT(29.6033609561895 -95.49430807758878) bank45867 +45868 POINT(29.582080581429967 -95.24286881751344) bank45868 +45869 POINT(29.010502021789446 -95.65344148914532) bank45869 +45870 POINT(30.02542354205168 -94.6812692311502) bank45870 +45871 POINT(30.738158538152287 -94.52199290562788) bank45871 +45872 POINT(29.519712624573316 -94.74742244403346) bank45872 +45873 POINT(30.730666693833946 -95.57433267613581) bank45873 +45874 POINT(28.912125269248282 -95.5477806192929) bank45874 +45875 POINT(29.991523561877166 -96.1828121832963) bank45875 +45876 POINT(29.236362520332527 -95.11862943711613) bank45876 +45877 POINT(30.018143441373397 -95.88302380646958) bank45877 +45878 POINT(29.764352369702504 -95.25848724413007) bank45878 +45879 POINT(30.70788883568103 -94.72469686245076) bank45879 +45880 POINT(30.47634408926868 -95.76707700169253) bank45880 +45881 POINT(30.744485922470144 -95.70580581253613) bank45881 +45882 POINT(30.31330268688436 -95.7596290688798) bank45882 +45883 POINT(28.945430868456416 -94.42537690815217) bank45883 +45884 POINT(30.153586763392244 -94.70554908876028) bank45884 +45885 POINT(30.644246484681332 -95.85197774739541) bank45885 +45886 POINT(30.517764718536068 -95.88636490773995) bank45886 +45887 POINT(30.19565272254794 -94.54642841291658) bank45887 +45888 POINT(28.957370614200645 -96.07319683573053) bank45888 +45889 POINT(29.379228486251236 -95.42777972236408) bank45889 +45890 POINT(30.458023938585885 -94.77112332579105) bank45890 +45891 POINT(30.613070974280255 -95.84326640706612) bank45891 +45892 POINT(30.533362579823272 -94.37601274476552) bank45892 +45893 POINT(28.91213028184188 -95.12936420951758) bank45893 +45894 POINT(29.372525272521585 -95.27967507888621) bank45894 +45895 POINT(28.954330330552104 -95.2068945840473) bank45895 +45896 POINT(29.065696011381174 -94.5019375810615) bank45896 +45897 POINT(30.083640706311137 -95.22727559653823) bank45897 +45898 POINT(29.885076829069575 -95.96904576992421) bank45898 +45899 POINT(29.21617879568185 -96.30433533340914) bank45899 +45900 POINT(29.967673864583535 -96.32193102827772) bank45900 +45901 POINT(30.748505305646653 -96.2581895908468) bank45901 +45902 POINT(30.285145879041124 -94.4796321139246) bank45902 +45903 POINT(29.51921583564452 -95.53577145825201) bank45903 +45904 POINT(29.084911228583977 -94.84874130869164) bank45904 +45905 POINT(30.020052589130906 -95.68273490988393) bank45905 +45906 POINT(30.273879304682573 -95.90397688434679) bank45906 +45907 POINT(29.392552817432847 -95.00969442780763) bank45907 +45908 POINT(30.63933912562122 -94.53907840952438) bank45908 +45909 POINT(29.017415447207178 -96.35598768758295) bank45909 +45910 POINT(29.424143241950453 -95.11059816433729) bank45910 +45911 POINT(29.49620658399229 -94.53362293672731) bank45911 +45912 POINT(29.096268681470725 -94.95529293888926) bank45912 +45913 POINT(29.45502208816911 -94.46149524114995) bank45913 +45914 POINT(29.59878509941704 -94.87614425318166) bank45914 +45915 POINT(29.03691413329257 -96.28715346713962) bank45915 +45916 POINT(29.452316197939783 -96.28839566156047) bank45916 +45917 POINT(30.016703647906635 -94.75107818470347) bank45917 +45918 POINT(28.7841830594586 -94.86078227739715) bank45918 +45919 POINT(30.280163299221307 -94.65737436806826) bank45919 +45920 POINT(30.698509392862626 -95.54222817521213) bank45920 +45921 POINT(29.4806629626912 -95.19650817261903) bank45921 +45922 POINT(30.35330447829546 -96.24630021072647) bank45922 +45923 POINT(30.00545010442486 -96.24233219747369) bank45923 +45924 POINT(29.5822882047233 -95.08645432753026) bank45924 +45925 POINT(28.896901739315503 -95.63244522831957) bank45925 +45926 POINT(29.932656270346783 -95.79764728430014) bank45926 +45927 POINT(30.20543915525673 -94.51666325809505) bank45927 +45928 POINT(29.63358946234858 -94.73125287979423) bank45928 +45929 POINT(29.882299545886585 -96.11360148423778) bank45929 +45930 POINT(30.225491568115 -96.33892802087897) bank45930 +45931 POINT(30.38472830554594 -95.90096746439849) bank45931 +45932 POINT(29.390003107501663 -95.78870556636069) bank45932 +45933 POINT(30.07889036809362 -96.10195231993985) bank45933 +45934 POINT(30.38606093351241 -94.88552436868599) bank45934 +45935 POINT(28.810044460936137 -95.08873816599719) bank45935 +45936 POINT(30.25484495603844 -95.49912684772092) bank45936 +45937 POINT(30.369977322249664 -94.59194123345667) bank45937 +45938 POINT(29.278382254271122 -94.44895655492125) bank45938 +45939 POINT(29.284294661542866 -94.63309591302323) bank45939 +45940 POINT(30.050378324921926 -95.9045075660766) bank45940 +45941 POINT(29.41524190149948 -96.06458147990801) bank45941 +45942 POINT(30.457742627572564 -95.22907828192707) bank45942 +45943 POINT(29.169237616976762 -94.92462036731803) bank45943 +45944 POINT(29.876142955664175 -96.34124601081946) bank45944 +45945 POINT(30.005410639806623 -95.48040250704086) bank45945 +45946 POINT(29.445641117399074 -95.73848806583399) bank45946 +45947 POINT(29.47624588944002 -94.60116886925891) bank45947 +45948 POINT(30.717797571294913 -95.55889166915826) bank45948 +45949 POINT(29.10625373816353 -94.98173100437683) bank45949 +45950 POINT(29.109326863150862 -94.82502989292959) bank45950 +45951 POINT(30.290234682574507 -95.73342613134962) bank45951 +45952 POINT(29.174280723791558 -95.46306435543981) bank45952 +45953 POINT(30.191643074565494 -95.75036726005182) bank45953 +45954 POINT(29.936510182002117 -94.8657174586845) bank45954 +45955 POINT(29.705365251026997 -94.44890416106205) bank45955 +45956 POINT(29.764471845855734 -95.01324262807833) bank45956 +45957 POINT(30.26652849260986 -94.67070740640685) bank45957 +45958 POINT(29.70279452325143 -96.15978174666232) bank45958 +45959 POINT(30.156239419811193 -96.01718092784168) bank45959 +45960 POINT(29.233639324349266 -95.56888705027599) bank45960 +45961 POINT(30.523261001368024 -94.94214543927012) bank45961 +45962 POINT(30.637947951612635 -94.81143963311526) bank45962 +45963 POINT(30.653896062387606 -94.83637716842193) bank45963 +45964 POINT(29.060506436028817 -95.35058258051366) bank45964 +45965 POINT(30.6896998296242 -94.58067490133907) bank45965 +45966 POINT(29.032466282088105 -95.20314317509641) bank45966 +45967 POINT(29.806534144982642 -94.78037565743449) bank45967 +45968 POINT(30.625983061384183 -94.52681701411537) bank45968 +45969 POINT(28.891703720275647 -95.51124586638389) bank45969 +45970 POINT(30.096302819353546 -95.84320021537201) bank45970 +45971 POINT(30.40255782653157 -96.12497693117952) bank45971 +45972 POINT(30.40091641969948 -95.05559917206875) bank45972 +45973 POINT(30.349086789483508 -95.26030882721423) bank45973 +45974 POINT(30.587437628665135 -94.75360791238022) bank45974 +45975 POINT(29.435392433410065 -95.35527287069866) bank45975 +45976 POINT(29.84855704684812 -94.70348507441126) bank45976 +45977 POINT(30.438395358846677 -94.66317400817542) bank45977 +45978 POINT(30.581946104940883 -96.10532902304355) bank45978 +45979 POINT(30.629721388736833 -95.8096047901266) bank45979 +45980 POINT(29.315019967803423 -95.12427568915776) bank45980 +45981 POINT(29.97146065313207 -95.10376550399262) bank45981 +45982 POINT(29.81407951776738 -95.02040594552133) bank45982 +45983 POINT(29.225062304344362 -95.15374709941179) bank45983 +45984 POINT(29.278926593032736 -94.43613878986635) bank45984 +45985 POINT(28.999591917047077 -94.60076762399692) bank45985 +45986 POINT(30.682697647236722 -95.36073166562974) bank45986 +45987 POINT(29.11440328902933 -94.45433266163207) bank45987 +45988 POINT(30.04431676877074 -94.44218202335853) bank45988 +45989 POINT(29.104946517820174 -95.28070842966865) bank45989 +45990 POINT(29.53606115183076 -95.15250111335955) bank45990 +45991 POINT(29.79737953596548 -95.78287960344838) bank45991 +45992 POINT(30.007401967971084 -94.70113154088256) bank45992 +45993 POINT(30.63022597863539 -94.46798061271193) bank45993 +45994 POINT(29.992579929942387 -94.81706578660435) bank45994 +45995 POINT(30.249730735420457 -96.10573795725404) bank45995 +45996 POINT(29.189908232950696 -96.29155103507405) bank45996 +45997 POINT(28.986462353823516 -95.37419404451765) bank45997 +45998 POINT(30.687898432174055 -94.58520132475306) bank45998 +45999 POINT(30.198065963156797 -94.4719432978021) bank45999 +46000 POINT(29.298078770104365 -96.21780572919216) bank46000 +46001 POINT(29.949635680077588 -95.96137388849925) bank46001 +46002 POINT(29.137455746424223 -95.64138506545623) bank46002 +46003 POINT(30.605697448502724 -95.70917115673893) bank46003 +46004 POINT(28.998716724019893 -94.60466600829007) bank46004 +46005 POINT(30.741994845965397 -95.42599589505674) bank46005 +46006 POINT(29.699464656261934 -94.4355448966798) bank46006 +46007 POINT(29.201985993460283 -94.53891514092533) bank46007 +46008 POINT(29.29289371228246 -95.6757595381218) bank46008 +46009 POINT(29.606069127898262 -95.07139970714579) bank46009 +46010 POINT(28.770119895650843 -96.35398766398362) bank46010 +46011 POINT(28.871239142871072 -95.62642750754047) bank46011 +46012 POINT(29.516005732903466 -95.07392184412487) bank46012 +46013 POINT(30.648609117802344 -95.83537717240566) bank46013 +46014 POINT(30.005926274690573 -94.78710250720235) bank46014 +46015 POINT(29.49912506551508 -95.10901126009098) bank46015 +46016 POINT(28.840125501200113 -95.74984551150288) bank46016 +46017 POINT(30.328132794856078 -95.41242444734252) bank46017 +46018 POINT(30.70008702389696 -94.96399972437412) bank46018 +46019 POINT(29.478265981522824 -94.48819561054383) bank46019 +46020 POINT(30.18372033193482 -94.81967430329422) bank46020 +46021 POINT(29.09592378917691 -94.41406568784359) bank46021 +46022 POINT(29.25900929812056 -94.80325810906466) bank46022 +46023 POINT(30.30841943125136 -95.08517401670599) bank46023 +46024 POINT(29.22349720210174 -96.23171710197377) bank46024 +46025 POINT(29.510083581545025 -95.5662577313998) bank46025 +46026 POINT(30.07464461380401 -94.95294666302213) bank46026 +46027 POINT(29.535798007246573 -95.53376076032399) bank46027 +46028 POINT(30.357460842696575 -94.58236881445994) bank46028 +46029 POINT(30.669427550592182 -95.31228612620673) bank46029 +46030 POINT(29.535087955177854 -95.16829258841537) bank46030 +46031 POINT(30.668837009031837 -95.85914858210778) bank46031 +46032 POINT(29.432458315765228 -96.02839544085639) bank46032 +46033 POINT(30.164721390371437 -95.0448442947806) bank46033 +46034 POINT(29.052129508274508 -95.01492004904895) bank46034 +46035 POINT(29.55587320965799 -94.49081028318786) bank46035 +46036 POINT(29.106215596391298 -96.20219675820242) bank46036 +46037 POINT(29.470307553191677 -95.86626826327876) bank46037 +46038 POINT(30.320818831927593 -96.28638877369546) bank46038 +46039 POINT(29.52365391273217 -95.98270565851107) bank46039 +46040 POINT(30.247735656337298 -95.1107607981386) bank46040 +46041 POINT(29.923786038700936 -95.84476307533083) bank46041 +46042 POINT(30.50128246566502 -94.76661676433696) bank46042 +46043 POINT(30.225366892905214 -95.72057560947468) bank46043 +46044 POINT(29.258311476644163 -95.16193327347145) bank46044 +46045 POINT(30.247061595573935 -94.64934206080373) bank46045 +46046 POINT(29.0981884828858 -94.93821760790884) bank46046 +46047 POINT(29.490638093640147 -95.90789181001439) bank46047 +46048 POINT(30.398451097596176 -95.44228182937958) bank46048 +46049 POINT(29.070640425246474 -95.95748181437338) bank46049 +46050 POINT(30.56998998985458 -95.44314321380561) bank46050 +46051 POINT(29.72723934312778 -95.21657421116751) bank46051 +46052 POINT(30.528443984706055 -94.41741685574945) bank46052 +46053 POINT(30.239817588993485 -95.09845311838407) bank46053 +46054 POINT(29.729869691632633 -95.08068753597534) bank46054 +46055 POINT(30.10047001406587 -95.03340454020831) bank46055 +46056 POINT(29.25956565019319 -94.79246282226111) bank46056 +46057 POINT(30.576699400907355 -95.67197073219386) bank46057 +46058 POINT(29.286698821508 -94.77227288728398) bank46058 +46059 POINT(29.252335782232034 -95.43027456922529) bank46059 +46060 POINT(30.211586320855403 -95.73136254490794) bank46060 +46061 POINT(29.682760087206198 -94.73378146642362) bank46061 +46062 POINT(30.739327283592885 -95.56302968496674) bank46062 +46063 POINT(30.465525681276496 -96.18083529731767) bank46063 +46064 POINT(30.529243428390348 -94.6005052668172) bank46064 +46065 POINT(30.14687140582045 -94.7198352085525) bank46065 +46066 POINT(29.81394111862746 -94.97092079773978) bank46066 +46067 POINT(30.13376692587669 -95.76492379466464) bank46067 +46068 POINT(29.348810470437652 -94.82779274078744) bank46068 +46069 POINT(29.396203559797215 -95.43697381219881) bank46069 +46070 POINT(29.405843625338587 -96.11276820913517) bank46070 +46071 POINT(28.852152584902825 -94.40084252365126) bank46071 +46072 POINT(30.711532094848298 -95.70641358214048) bank46072 +46073 POINT(29.798713676725658 -95.06642349514082) bank46073 +46074 POINT(29.526698565798984 -95.66176857901127) bank46074 +46075 POINT(28.855627353839512 -95.47913353314324) bank46075 +46076 POINT(30.410383544289207 -95.03147188386438) bank46076 +46077 POINT(29.318807887240137 -95.48044901619394) bank46077 +46078 POINT(30.687992542996817 -96.16834782058767) bank46078 +46079 POINT(29.54836206515779 -94.42025056644958) bank46079 +46080 POINT(29.75339914412361 -94.77292456738384) bank46080 +46081 POINT(29.991319388224767 -95.35458426041633) bank46081 +46082 POINT(29.235505609493014 -95.19054557938202) bank46082 +46083 POINT(29.84731766237293 -94.60013579348752) bank46083 +46084 POINT(30.646780203632847 -95.19522803250553) bank46084 +46085 POINT(29.36740616628149 -95.91654177836072) bank46085 +46086 POINT(29.502665385939906 -95.94603795317997) bank46086 +46087 POINT(30.323731952201015 -95.180245804566) bank46087 +46088 POINT(29.506731693261685 -94.54961902007678) bank46088 +46089 POINT(29.983024923353312 -96.27206531497325) bank46089 +46090 POINT(29.53023809477568 -95.81683390159338) bank46090 +46091 POINT(29.652238231701993 -95.1482482370579) bank46091 +46092 POINT(30.275656809433467 -94.76026992901348) bank46092 +46093 POINT(29.67277713939282 -96.3556815031857) bank46093 +46094 POINT(30.00631454610158 -95.99543915531125) bank46094 +46095 POINT(29.038410556506633 -94.45079775916145) bank46095 +46096 POINT(30.67983268083371 -94.8277776496791) bank46096 +46097 POINT(30.133037741365673 -94.97889360402223) bank46097 +46098 POINT(29.097344292329385 -94.77914599982707) bank46098 +46099 POINT(28.99709638377911 -94.79925525690922) bank46099 +46100 POINT(29.546982849894977 -95.3794755487724) bank46100 +46101 POINT(29.30161634311368 -96.0407443493699) bank46101 +46102 POINT(29.001840951398922 -95.83985225158699) bank46102 +46103 POINT(28.835874364715107 -95.60881375177198) bank46103 +46104 POINT(29.233338339295855 -94.87672617126948) bank46104 +46105 POINT(29.35711620353793 -95.87130494728937) bank46105 +46106 POINT(29.522323567404197 -95.42847461740536) bank46106 +46107 POINT(29.718473327838986 -94.84943312572946) bank46107 +46108 POINT(30.222191680282844 -95.52341194220783) bank46108 +46109 POINT(29.55582971763886 -95.8006504095075) bank46109 +46110 POINT(30.001410684674994 -94.78999069630792) bank46110 +46111 POINT(28.981812830683396 -95.22430817078404) bank46111 +46112 POINT(29.85291149857268 -95.41482988788303) bank46112 +46113 POINT(30.141855289858267 -94.43465115299088) bank46113 +46114 POINT(30.357916142997738 -95.74181494886882) bank46114 +46115 POINT(30.202703701000228 -96.33283858072735) bank46115 +46116 POINT(29.446469055751916 -95.26659066349882) bank46116 +46117 POINT(29.754104480646404 -94.74748210915646) bank46117 +46118 POINT(30.36582786455555 -94.8064629617283) bank46118 +46119 POINT(29.827081953917506 -94.53098012582991) bank46119 +46120 POINT(29.994729930697943 -95.25449557248066) bank46120 +46121 POINT(29.465605914825737 -95.71637583684361) bank46121 +46122 POINT(30.552429918019183 -94.71022878214644) bank46122 +46123 POINT(30.172416771587805 -94.5749954932439) bank46123 +46124 POINT(30.526071612066083 -95.39771976175675) bank46124 +46125 POINT(29.254387161159865 -94.52355053884844) bank46125 +46126 POINT(30.057412023225368 -95.98844140845227) bank46126 +46127 POINT(30.522019043235037 -95.62810121054352) bank46127 +46128 POINT(30.590121579014486 -94.82771130795085) bank46128 +46129 POINT(30.33401086955748 -96.01511321062291) bank46129 +46130 POINT(30.02685492071014 -95.43478546566823) bank46130 +46131 POINT(29.264697542901693 -95.06324961430902) bank46131 +46132 POINT(28.80671274666284 -95.47145824768302) bank46132 +46133 POINT(29.931703686510346 -95.59580110892422) bank46133 +46134 POINT(29.750866980285632 -94.39390146129249) bank46134 +46135 POINT(29.919360594210765 -95.01397053590613) bank46135 +46136 POINT(29.81825947098469 -94.38325407545638) bank46136 +46137 POINT(30.630642383218646 -95.297645759303) bank46137 +46138 POINT(30.070106120002873 -95.25793480120738) bank46138 +46139 POINT(29.298400700208283 -95.42928319089688) bank46139 +46140 POINT(30.01999079134151 -95.9481232149759) bank46140 +46141 POINT(28.89508829141562 -96.21651026390187) bank46141 +46142 POINT(30.07804372243087 -94.65890545328742) bank46142 +46143 POINT(30.461759901970247 -95.8377601186425) bank46143 +46144 POINT(29.828956910515657 -94.87216150817842) bank46144 +46145 POINT(29.51978223729959 -95.13250084606585) bank46145 +46146 POINT(30.172415274489985 -94.5073474244686) bank46146 +46147 POINT(29.96968029983525 -95.31042203118722) bank46147 +46148 POINT(28.819996959840825 -96.21686261642621) bank46148 +46149 POINT(30.028398757942885 -94.73592086489404) bank46149 +46150 POINT(28.827776752747397 -95.34217309320893) bank46150 +46151 POINT(29.722122689331805 -94.73121725879507) bank46151 +46152 POINT(29.344133593737755 -94.37360057469158) bank46152 +46153 POINT(29.688070172195687 -94.3860529145368) bank46153 +46154 POINT(29.73138824587799 -94.57791210410204) bank46154 +46155 POINT(29.27241021839155 -95.86179548277508) bank46155 +46156 POINT(29.668416618384605 -94.80345853944928) bank46156 +46157 POINT(29.0661476384456 -94.61701239524437) bank46157 +46158 POINT(30.18949745798782 -95.19901557635303) bank46158 +46159 POINT(30.27844295010932 -95.63397249359755) bank46159 +46160 POINT(29.757806240828856 -95.31591783404343) bank46160 +46161 POINT(28.897053791790814 -95.61930039691002) bank46161 +46162 POINT(29.351434300822064 -94.64187220152172) bank46162 +46163 POINT(30.24000681917694 -95.65334423034503) bank46163 +46164 POINT(30.713258633928348 -95.68190945067681) bank46164 +46165 POINT(29.91272882320229 -95.55721067730933) bank46165 +46166 POINT(30.590221660837184 -94.48021009351737) bank46166 +46167 POINT(29.892938840552382 -95.5879264769773) bank46167 +46168 POINT(29.70132568947533 -94.58439123648554) bank46168 +46169 POINT(29.45000180544185 -95.83065118955551) bank46169 +46170 POINT(28.96971724441739 -96.22733592189563) bank46170 +46171 POINT(29.166007894401787 -95.55454365684525) bank46171 +46172 POINT(30.66385814664701 -95.42798215242613) bank46172 +46173 POINT(29.329189456522293 -95.68875595655481) bank46173 +46174 POINT(29.712054265427117 -96.00068404977044) bank46174 +46175 POINT(29.301160599331 -94.88097178488225) bank46175 +46176 POINT(28.987450822409333 -95.70575650854973) bank46176 +46177 POINT(29.42165520373418 -94.8646749053189) bank46177 +46178 POINT(29.101415885079934 -95.75282336678346) bank46178 +46179 POINT(29.471077921556738 -95.27152978371916) bank46179 +46180 POINT(29.411188999905878 -95.40137711229701) bank46180 +46181 POINT(28.7853403407315 -95.27310601179374) bank46181 +46182 POINT(29.634214802566415 -94.77684381215775) bank46182 +46183 POINT(28.829055667620388 -94.78998117069813) bank46183 +46184 POINT(29.17842054494352 -96.12261046971491) bank46184 +46185 POINT(29.563251725939562 -94.53791952063112) bank46185 +46186 POINT(30.481439611456967 -95.27226649692271) bank46186 +46187 POINT(28.838010185776337 -95.77607724089248) bank46187 +46188 POINT(29.5290858434811 -95.84189872272324) bank46188 +46189 POINT(30.64483213573982 -94.85952216342721) bank46189 +46190 POINT(28.77868578985586 -95.71313162282974) bank46190 +46191 POINT(29.994892704258753 -95.39123992231711) bank46191 +46192 POINT(30.160401971124294 -95.98175234612914) bank46192 +46193 POINT(29.744976362599488 -95.3972972403647) bank46193 +46194 POINT(29.573251163302686 -96.08286727401342) bank46194 +46195 POINT(30.036210225236744 -94.87338317734853) bank46195 +46196 POINT(30.246979937344495 -94.88165916833161) bank46196 +46197 POINT(30.0517254653196 -95.2054934084533) bank46197 +46198 POINT(30.216181643960265 -95.21283958356673) bank46198 +46199 POINT(30.529943507825507 -95.53839163752711) bank46199 +46200 POINT(30.440511881178757 -95.79313807731138) bank46200 +46201 POINT(28.98651808429711 -94.43453429511594) bank46201 +46202 POINT(29.82121075563907 -95.83307638858703) bank46202 +46203 POINT(28.996232239156296 -94.52269310620449) bank46203 +46204 POINT(30.476708271648437 -94.63086884116875) bank46204 +46205 POINT(30.33056396692395 -94.51307264664177) bank46205 +46206 POINT(29.732084172389406 -95.86576693589353) bank46206 +46207 POINT(29.82937752179687 -95.00313046933043) bank46207 +46208 POINT(30.074454770950137 -95.95455793337096) bank46208 +46209 POINT(30.133106134090564 -96.16413098518282) bank46209 +46210 POINT(30.454072394443802 -95.78341807844262) bank46210 +46211 POINT(29.6510809305078 -95.9871908728491) bank46211 +46212 POINT(30.481671568355004 -95.01758308856115) bank46212 +46213 POINT(30.561684911359276 -95.34889714734517) bank46213 +46214 POINT(29.081290784256968 -95.10829329080171) bank46214 +46215 POINT(29.7187050459968 -95.12677783681114) bank46215 +46216 POINT(29.791516498564135 -96.1202233983554) bank46216 +46217 POINT(30.501317668028666 -96.34936635605841) bank46217 +46218 POINT(29.14679714494197 -95.08376388167933) bank46218 +46219 POINT(30.728835601547136 -95.2086715552329) bank46219 +46220 POINT(29.05782543891841 -96.05893142811172) bank46220 +46221 POINT(29.880519971867002 -94.72901235307329) bank46221 +46222 POINT(28.779636310136723 -94.96876974448266) bank46222 +46223 POINT(29.586984234417063 -94.57421191829337) bank46223 +46224 POINT(30.358058274316214 -94.68495634166345) bank46224 +46225 POINT(29.749302415992243 -95.03545102427815) bank46225 +46226 POINT(30.566060184916164 -95.83733453400477) bank46226 +46227 POINT(29.026013460916428 -95.61876484699084) bank46227 +46228 POINT(30.283904678971908 -95.99855608923096) bank46228 +46229 POINT(30.50864810510894 -94.72152527110515) bank46229 +46230 POINT(30.566123440013655 -95.00081446790135) bank46230 +46231 POINT(28.912570111595564 -95.57116547218355) bank46231 +46232 POINT(29.70411751345912 -95.11205976586702) bank46232 +46233 POINT(29.457398972686903 -95.99487746400523) bank46233 +46234 POINT(30.108130257537432 -94.4890958849074) bank46234 +46235 POINT(29.540797525186797 -94.79068127188238) bank46235 +46236 POINT(30.442627325345683 -94.43861654532003) bank46236 +46237 POINT(30.695673657950582 -95.11476185149968) bank46237 +46238 POINT(28.96751929149064 -96.36730037828998) bank46238 +46239 POINT(30.693768654490466 -95.80611913553169) bank46239 +46240 POINT(30.33110496341036 -94.74926516268692) bank46240 +46241 POINT(29.493415780002092 -94.78844472477284) bank46241 +46242 POINT(29.671561979908667 -94.90196629488575) bank46242 +46243 POINT(30.484833708113836 -95.7283002430251) bank46243 +46244 POINT(30.67155503057136 -94.87431008503118) bank46244 +46245 POINT(30.14643082903703 -95.07194004197197) bank46245 +46246 POINT(30.27938473837606 -95.00887932629507) bank46246 +46247 POINT(29.373216586099648 -94.41909116138198) bank46247 +46248 POINT(29.692074264097574 -96.0772482079378) bank46248 +46249 POINT(29.22583438276348 -96.29575423674015) bank46249 +46250 POINT(30.007071431550376 -94.48602099594481) bank46250 +46251 POINT(28.947103408049642 -95.92134363546644) bank46251 +46252 POINT(28.896883571361656 -95.19639392625447) bank46252 +46253 POINT(29.199548714587674 -94.5561340993943) bank46253 +46254 POINT(29.40263636325719 -95.10572403089904) bank46254 +46255 POINT(29.95527497892523 -96.06515514932084) bank46255 +46256 POINT(30.11092259062576 -95.64486332054409) bank46256 +46257 POINT(29.255075672657846 -95.05236328042335) bank46257 +46258 POINT(29.741388834554144 -95.46987433383933) bank46258 +46259 POINT(30.059150026147428 -96.10965025496932) bank46259 +46260 POINT(29.78426316492982 -95.97481968048378) bank46260 +46261 POINT(30.32368337140756 -96.25316251274414) bank46261 +46262 POINT(29.08073281565172 -95.3560536912762) bank46262 +46263 POINT(29.587250005445746 -94.65847712677902) bank46263 +46264 POINT(29.95595503100807 -95.2748776087981) bank46264 +46265 POINT(30.734350646879143 -95.66972278778903) bank46265 +46266 POINT(30.066044613401232 -95.96440183403689) bank46266 +46267 POINT(28.96742272279856 -95.17624544541732) bank46267 +46268 POINT(29.048907677811993 -94.58822585837558) bank46268 +46269 POINT(30.625461459730488 -95.1329217493568) bank46269 +46270 POINT(29.038754866764222 -94.37421856791337) bank46270 +46271 POINT(29.30243341022617 -94.49654321927494) bank46271 +46272 POINT(30.65729987229079 -94.47867959809072) bank46272 +46273 POINT(30.494689451441236 -95.49489134186098) bank46273 +46274 POINT(29.746620262534677 -95.20146578996952) bank46274 +46275 POINT(29.455873891975617 -96.02341926712947) bank46275 +46276 POINT(30.495831054207063 -94.82665416415252) bank46276 +46277 POINT(30.706799734787886 -96.27407673115438) bank46277 +46278 POINT(30.18210550504591 -94.6170546613846) bank46278 +46279 POINT(30.220790247454108 -95.84070403877561) bank46279 +46280 POINT(29.608815007531874 -96.30703817743402) bank46280 +46281 POINT(29.327612549887533 -96.33282389134158) bank46281 +46282 POINT(29.8409943016202 -94.60257535647726) bank46282 +46283 POINT(29.240742848241357 -95.62811154369827) bank46283 +46284 POINT(30.43747338135859 -96.24142373764838) bank46284 +46285 POINT(28.917694644518363 -95.54835939467799) bank46285 +46286 POINT(30.1788906878025 -96.36040710715578) bank46286 +46287 POINT(29.613539578104103 -96.34638007196297) bank46287 +46288 POINT(30.225966655049533 -95.0414494372386) bank46288 +46289 POINT(29.54189952331179 -95.80925744864972) bank46289 +46290 POINT(30.573534523753672 -94.78338835553949) bank46290 +46291 POINT(30.55538854534109 -96.31126681692714) bank46291 +46292 POINT(29.15614750459751 -95.13488129770363) bank46292 +46293 POINT(30.73134186533681 -95.60565638662804) bank46293 +46294 POINT(29.728430283127548 -96.04364416839535) bank46294 +46295 POINT(29.689873748104635 -96.17088007269633) bank46295 +46296 POINT(28.776285986677923 -95.1284237899227) bank46296 +46297 POINT(29.836624386125575 -95.53373049318526) bank46297 +46298 POINT(28.836011298591817 -95.79549162436109) bank46298 +46299 POINT(30.682001416430943 -94.78621866389528) bank46299 +46300 POINT(28.782574771482384 -95.72419593595161) bank46300 +46301 POINT(30.07858860398926 -95.86790870298044) bank46301 +46302 POINT(30.20085650237005 -95.48236374054548) bank46302 +46303 POINT(29.14785698659134 -95.77490394917226) bank46303 +46304 POINT(30.420821014684538 -94.93081800989927) bank46304 +46305 POINT(30.247291018603132 -95.69526444160134) bank46305 +46306 POINT(30.440446553055835 -94.36986125656658) bank46306 +46307 POINT(30.381499936430014 -94.68153112632126) bank46307 +46308 POINT(29.43498843276855 -95.67556334785122) bank46308 +46309 POINT(30.434642419199548 -95.87392143253727) bank46309 +46310 POINT(28.95301074525328 -94.99859431813637) bank46310 +46311 POINT(29.897898430684663 -96.19971680805286) bank46311 +46312 POINT(29.53077344361077 -94.48593332728701) bank46312 +46313 POINT(30.655098476222598 -96.11712952990206) bank46313 +46314 POINT(29.574684262184665 -94.94426573634975) bank46314 +46315 POINT(29.982431265618747 -94.49261181883034) bank46315 +46316 POINT(29.25808815538659 -95.53602970056659) bank46316 +46317 POINT(30.160090703427844 -94.7958337027559) bank46317 +46318 POINT(30.2925858983602 -95.40549992047085) bank46318 +46319 POINT(29.467402039308467 -95.14961810469259) bank46319 +46320 POINT(29.031079848581918 -94.39669435043066) bank46320 +46321 POINT(30.308538593083068 -96.11780898172104) bank46321 +46322 POINT(29.444219157169936 -95.3805627961027) bank46322 +46323 POINT(29.473280308956827 -95.14633710521134) bank46323 +46324 POINT(29.385064382124163 -95.54998625055494) bank46324 +46325 POINT(30.6381808546137 -95.18498903817046) bank46325 +46326 POINT(29.70937148424655 -95.16347937708431) bank46326 +46327 POINT(30.217830393331955 -94.67688118995785) bank46327 +46328 POINT(30.047460042434288 -94.861422712676) bank46328 +46329 POINT(30.072695857256196 -95.11570110820465) bank46329 +46330 POINT(29.921291807510933 -94.63976620870069) bank46330 +46331 POINT(30.02830544621131 -94.8275841279215) bank46331 +46332 POINT(29.47960350608867 -94.58598223153544) bank46332 +46333 POINT(28.81531252967615 -95.04411181680138) bank46333 +46334 POINT(28.953494045944904 -95.4695763827793) bank46334 +46335 POINT(30.123162097576195 -94.7498932749582) bank46335 +46336 POINT(30.198970631398225 -94.53354680125808) bank46336 +46337 POINT(29.637815211062158 -96.01446338753759) bank46337 +46338 POINT(29.096415110986854 -94.57919976739271) bank46338 +46339 POINT(28.943284380268345 -95.85051573364483) bank46339 +46340 POINT(29.73919683173713 -94.88084075627242) bank46340 +46341 POINT(29.57832641122099 -94.41644238072473) bank46341 +46342 POINT(29.16650682196819 -96.04757730817308) bank46342 +46343 POINT(29.220539349345362 -96.29971344341607) bank46343 +46344 POINT(29.889065957129514 -94.67189242642245) bank46344 +46345 POINT(30.21992069201757 -94.7852270234959) bank46345 +46346 POINT(30.100908076478063 -94.93191988036395) bank46346 +46347 POINT(30.07597385555506 -94.63072496956198) bank46347 +46348 POINT(29.912528262321718 -96.31117987286754) bank46348 +46349 POINT(29.267857628790285 -95.9019519344233) bank46349 +46350 POINT(29.1908417093242 -96.15202849396475) bank46350 +46351 POINT(29.63915884750601 -95.39553548455719) bank46351 +46352 POINT(29.66191950268523 -95.38247222254621) bank46352 +46353 POINT(28.79328765457692 -94.46258757773026) bank46353 +46354 POINT(30.65660042917336 -94.42155745707483) bank46354 +46355 POINT(29.910766602385458 -94.86700043387468) bank46355 +46356 POINT(28.948016286158857 -96.17542871515207) bank46356 +46357 POINT(30.433914722128968 -96.03285810305447) bank46357 +46358 POINT(29.899519194312465 -94.80376155437872) bank46358 +46359 POINT(29.467600213068888 -95.11581310145004) bank46359 +46360 POINT(30.696633547219413 -95.68343072017028) bank46360 +46361 POINT(29.846434613024563 -94.59768980921622) bank46361 +46362 POINT(29.60305772963129 -95.53651384951552) bank46362 +46363 POINT(30.65606336374473 -96.36512273454302) bank46363 +46364 POINT(29.170950851760235 -96.02072953156964) bank46364 +46365 POINT(30.280296042950276 -96.34101004444145) bank46365 +46366 POINT(28.85112676338557 -95.49531861222123) bank46366 +46367 POINT(29.368763189708456 -94.91519374226971) bank46367 +46368 POINT(30.745703695644245 -95.69820536485288) bank46368 +46369 POINT(30.523870427032307 -95.8541527961464) bank46369 +46370 POINT(28.90580881010054 -96.31049117823375) bank46370 +46371 POINT(30.68829009833185 -95.07335549952171) bank46371 +46372 POINT(29.916504988042927 -94.92082471475442) bank46372 +46373 POINT(29.289965405382137 -95.03248940738298) bank46373 +46374 POINT(30.27967487190922 -95.85277418055624) bank46374 +46375 POINT(28.84648463265655 -95.45952295489526) bank46375 +46376 POINT(30.029694195572265 -95.23504290723137) bank46376 +46377 POINT(30.738378690306345 -95.53071555616962) bank46377 +46378 POINT(30.066261918726756 -94.91416616002634) bank46378 +46379 POINT(30.441521062948738 -95.87814677285027) bank46379 +46380 POINT(30.496427501651574 -95.48307627646682) bank46380 +46381 POINT(29.12141416355003 -95.67063495704413) bank46381 +46382 POINT(29.962681449007718 -96.31109758073731) bank46382 +46383 POINT(30.550186944445706 -95.41356049790602) bank46383 +46384 POINT(29.550993626198018 -95.69200869381295) bank46384 +46385 POINT(29.913991204408706 -94.86220737825221) bank46385 +46386 POINT(30.61630596141526 -96.36870814852018) bank46386 +46387 POINT(28.9892667173758 -95.1188006568509) bank46387 +46388 POINT(29.808957348992557 -95.4240859092081) bank46388 +46389 POINT(29.532613973211557 -95.30185101396853) bank46389 +46390 POINT(29.091451704399198 -95.27328587181924) bank46390 +46391 POINT(29.648278891813554 -94.47606035381544) bank46391 +46392 POINT(30.595380299324297 -94.47659754671201) bank46392 +46393 POINT(30.021035198111818 -96.1568246044921) bank46393 +46394 POINT(29.67902177746509 -96.14142951695024) bank46394 +46395 POINT(29.09005530773193 -94.7658861095253) bank46395 +46396 POINT(30.435820523004924 -96.10794023721708) bank46396 +46397 POINT(29.976878190767863 -95.10303118694614) bank46397 +46398 POINT(29.34095895641751 -94.46916421431821) bank46398 +46399 POINT(29.775987093063073 -95.6293197309961) bank46399 +46400 POINT(30.729724027569013 -95.23456919930284) bank46400 +46401 POINT(29.697895322121436 -95.60906370061575) bank46401 +46402 POINT(29.864010168115996 -95.48923737410657) bank46402 +46403 POINT(30.236132258511557 -95.67195086785239) bank46403 +46404 POINT(28.90192189965384 -95.36911312984668) bank46404 +46405 POINT(29.98931539947141 -94.96149999714615) bank46405 +46406 POINT(28.847585088728582 -95.56893177974845) bank46406 +46407 POINT(29.431168991784702 -95.22879725683987) bank46407 +46408 POINT(30.297819430343697 -96.07960732000903) bank46408 +46409 POINT(29.42205076205787 -96.05585863012374) bank46409 +46410 POINT(30.6327453615684 -95.23925450059944) bank46410 +46411 POINT(29.154533682553225 -95.99250267217003) bank46411 +46412 POINT(30.556318536850448 -95.95718706057505) bank46412 +46413 POINT(29.02687829017632 -96.1181997852204) bank46413 +46414 POINT(30.11575666073638 -95.72688515712086) bank46414 +46415 POINT(29.896469178375774 -95.7936934148633) bank46415 +46416 POINT(29.650286017458 -95.61132424905969) bank46416 +46417 POINT(29.142249921133665 -95.80104479559593) bank46417 +46418 POINT(29.5888653705606 -96.11911030875883) bank46418 +46419 POINT(29.220542771873635 -95.69330667322613) bank46419 +46420 POINT(29.809796762547727 -95.44281337268873) bank46420 +46421 POINT(29.081237284345576 -95.4394586903168) bank46421 +46422 POINT(30.415134129318457 -94.61371916007602) bank46422 +46423 POINT(30.069446737066563 -94.88539279812721) bank46423 +46424 POINT(29.698854925919186 -94.5842720559348) bank46424 +46425 POINT(30.07258964472731 -95.80741080295704) bank46425 +46426 POINT(29.286119033777837 -95.04138550246185) bank46426 +46427 POINT(29.344400604882182 -95.16318605086659) bank46427 +46428 POINT(30.504849492571054 -95.81563620580074) bank46428 +46429 POINT(30.701797266085155 -95.29774386752989) bank46429 +46430 POINT(29.040158735221368 -95.69202200276247) bank46430 +46431 POINT(29.54538382002198 -95.1224090145472) bank46431 +46432 POINT(29.050214206628947 -95.99350218154252) bank46432 +46433 POINT(30.50386649721483 -95.53251385982327) bank46433 +46434 POINT(30.26714693394611 -95.97245409722571) bank46434 +46435 POINT(28.92822801540341 -95.38727668617553) bank46435 +46436 POINT(30.488127518375524 -96.2691209872661) bank46436 +46437 POINT(29.965907172181936 -94.78636613378657) bank46437 +46438 POINT(30.322406477950476 -95.23478542983659) bank46438 +46439 POINT(30.59420265618564 -95.17627214124053) bank46439 +46440 POINT(29.62492760442204 -95.71499363047846) bank46440 +46441 POINT(30.63731666280097 -94.9101753421603) bank46441 +46442 POINT(29.80462568244754 -95.77583415043817) bank46442 +46443 POINT(28.785353790622562 -94.81812657113886) bank46443 +46444 POINT(29.89380868374714 -96.16509324909055) bank46444 +46445 POINT(30.310198104777918 -95.59887542930548) bank46445 +46446 POINT(30.11079150949908 -94.9465256775332) bank46446 +46447 POINT(29.887056050036957 -94.59242473360615) bank46447 +46448 POINT(29.10687641040545 -94.3759514810889) bank46448 +46449 POINT(30.40216192701634 -96.01280548477848) bank46449 +46450 POINT(30.274840999105955 -95.14957747566083) bank46450 +46451 POINT(29.22938096157169 -96.00691459318098) bank46451 +46452 POINT(30.29091886313205 -95.3596259673503) bank46452 +46453 POINT(29.8495403884975 -94.65521673113471) bank46453 +46454 POINT(29.940433623191435 -94.69090687945203) bank46454 +46455 POINT(29.576229850853775 -96.17222796713018) bank46455 +46456 POINT(30.31866083179163 -95.89635444937213) bank46456 +46457 POINT(29.677564281255876 -96.18073147332414) bank46457 +46458 POINT(29.64397799209511 -96.34921335234277) bank46458 +46459 POINT(30.23010120022716 -94.69959936801268) bank46459 +46460 POINT(29.269829976839326 -94.59816213525842) bank46460 +46461 POINT(29.96151715133482 -96.1390349804142) bank46461 +46462 POINT(30.179589133242587 -95.21510382868638) bank46462 +46463 POINT(29.23124441096394 -94.98830073530165) bank46463 +46464 POINT(29.619199357319474 -95.38880914801972) bank46464 +46465 POINT(29.821583116072663 -94.77011417788545) bank46465 +46466 POINT(29.707327482090264 -96.07433576130448) bank46466 +46467 POINT(28.945486457628796 -94.63575690914453) bank46467 +46468 POINT(30.313026510367337 -96.10378018251689) bank46468 +46469 POINT(30.69640313758091 -94.664872490201) bank46469 +46470 POINT(30.397090110522733 -95.18759638759656) bank46470 +46471 POINT(28.979512340631594 -94.57405730400936) bank46471 +46472 POINT(30.71117614031938 -96.2815282950968) bank46472 +46473 POINT(29.66524664122132 -95.05336973555828) bank46473 +46474 POINT(30.113234409031744 -95.27643119260203) bank46474 +46475 POINT(29.69572073000437 -95.52393988128976) bank46475 +46476 POINT(30.185823865081705 -95.5418870247332) bank46476 +46477 POINT(30.265575847108256 -96.06636959528036) bank46477 +46478 POINT(30.219108823291542 -95.81701098945435) bank46478 +46479 POINT(30.112361941175486 -95.6945006645727) bank46479 +46480 POINT(28.943055735714683 -94.79597434943248) bank46480 +46481 POINT(29.12832346732646 -96.0950590783982) bank46481 +46482 POINT(29.418414065541434 -94.7340553953856) bank46482 +46483 POINT(29.844506739360842 -94.84919745203642) bank46483 +46484 POINT(28.774337878988504 -95.1095673347881) bank46484 +46485 POINT(30.423462295999617 -94.55186508344748) bank46485 +46486 POINT(30.563936500309726 -95.14061042983131) bank46486 +46487 POINT(29.003499281431456 -95.46857910529978) bank46487 +46488 POINT(29.673773584044437 -95.02600679057132) bank46488 +46489 POINT(28.997393154853214 -95.09827847720742) bank46489 +46490 POINT(29.100589979797718 -95.34472324568873) bank46490 +46491 POINT(29.42824009939782 -95.49594940620717) bank46491 +46492 POINT(29.92997835471786 -96.21120707110022) bank46492 +46493 POINT(30.491923969706992 -95.11418053799916) bank46493 +46494 POINT(30.33711272116956 -96.2829692963671) bank46494 +46495 POINT(29.21234361493649 -95.88200496703485) bank46495 +46496 POINT(29.451345098073084 -94.76760072794701) bank46496 +46497 POINT(29.830679574187634 -94.42537033145794) bank46497 +46498 POINT(29.767611903305493 -94.84808929704158) bank46498 +46499 POINT(29.633567668423176 -95.50084491924956) bank46499 +46500 POINT(28.958528105750982 -95.84594230385359) bank46500 +46501 POINT(29.46206611550938 -95.32405625196687) bank46501 +46502 POINT(30.128888672990165 -95.81344042613199) bank46502 +46503 POINT(29.560050999770308 -95.23153150993653) bank46503 +46504 POINT(30.201978758693524 -94.97828101486162) bank46504 +46505 POINT(30.484382249080927 -94.81792443639347) bank46505 +46506 POINT(29.699479706203384 -95.36851708167595) bank46506 +46507 POINT(30.36763198071901 -95.73399280399747) bank46507 +46508 POINT(29.507620375936263 -96.03483459008999) bank46508 +46509 POINT(29.669329563996843 -94.39827659764472) bank46509 +46510 POINT(29.52901853802357 -95.800047858982) bank46510 +46511 POINT(30.466608632715495 -94.73818958250547) bank46511 +46512 POINT(29.90333050986258 -95.23342848057759) bank46512 +46513 POINT(29.861551004628474 -96.03416605666654) bank46513 +46514 POINT(30.75244060169005 -96.17349749871659) bank46514 +46515 POINT(30.391537769536995 -94.60036945731534) bank46515 +46516 POINT(28.89573307874126 -95.78459031830923) bank46516 +46517 POINT(30.26585378514193 -94.44378184760055) bank46517 +46518 POINT(30.241863612544726 -95.27789422425931) bank46518 +46519 POINT(28.91643804879012 -95.4358762395143) bank46519 +46520 POINT(30.702068920250575 -94.39722482409049) bank46520 +46521 POINT(30.523522151857087 -94.43671322956841) bank46521 +46522 POINT(30.37129539337124 -94.87604392565059) bank46522 +46523 POINT(30.530278263421113 -96.05134868515277) bank46523 +46524 POINT(29.60213046954086 -95.10773957851676) bank46524 +46525 POINT(29.762516938418926 -95.70304023764291) bank46525 +46526 POINT(29.27843850433888 -94.58324226171351) bank46526 +46527 POINT(29.054343569063654 -96.24269209902705) bank46527 +46528 POINT(30.161786490312572 -95.67577475443723) bank46528 +46529 POINT(29.47227552224801 -95.18559781248631) bank46529 +46530 POINT(28.9391721605447 -94.37455454537441) bank46530 +46531 POINT(29.883529615304546 -94.69475037557986) bank46531 +46532 POINT(29.394410074539927 -96.14964764108107) bank46532 +46533 POINT(30.13368073940891 -94.41839313081093) bank46533 +46534 POINT(28.928339431149404 -95.50819358900317) bank46534 +46535 POINT(30.700446055264187 -95.01745077144253) bank46535 +46536 POINT(30.71637474492833 -95.66038159801136) bank46536 +46537 POINT(30.38430733333009 -95.90971301568196) bank46537 +46538 POINT(30.678402235476774 -96.20821628702929) bank46538 +46539 POINT(30.24755293225158 -95.635522064178) bank46539 +46540 POINT(29.805690740889368 -95.27468320129117) bank46540 +46541 POINT(28.764354205875446 -96.24083895261086) bank46541 +46542 POINT(29.927164048588388 -96.00776008702492) bank46542 +46543 POINT(29.731382195090255 -96.23804747927241) bank46543 +46544 POINT(29.614231716928103 -94.58477573467754) bank46544 +46545 POINT(30.692308431893505 -96.17860895166748) bank46545 +46546 POINT(30.434848479177628 -94.57749011832435) bank46546 +46547 POINT(28.986217729228763 -95.3310712648794) bank46547 +46548 POINT(30.20624173109839 -95.5668610858415) bank46548 +46549 POINT(29.638707039159375 -95.0768608782801) bank46549 +46550 POINT(28.944747405810556 -95.39924886901396) bank46550 +46551 POINT(30.598344280013134 -95.37774700966617) bank46551 +46552 POINT(29.74566529576031 -95.79692611207022) bank46552 +46553 POINT(30.242925058110583 -94.47991138097176) bank46553 +46554 POINT(30.04883626869413 -95.63016254732895) bank46554 +46555 POINT(29.43829888158771 -94.68850867664413) bank46555 +46556 POINT(29.54202579578053 -94.51262712693693) bank46556 +46557 POINT(30.68462297458124 -95.90068554057825) bank46557 +46558 POINT(29.54248912553237 -96.0552161367842) bank46558 +46559 POINT(29.701112044900796 -95.90397853917743) bank46559 +46560 POINT(29.30407642315399 -96.30535617761554) bank46560 +46561 POINT(29.492745558004998 -94.79952954596791) bank46561 +46562 POINT(29.080945689777064 -95.15309712547123) bank46562 +46563 POINT(29.56379590234259 -94.91245913776329) bank46563 +46564 POINT(29.50812490957413 -95.46252472806542) bank46564 +46565 POINT(29.847778953726987 -95.64356536434472) bank46565 +46566 POINT(29.61609088438379 -96.13232191872102) bank46566 +46567 POINT(30.21743220005647 -96.20110541477209) bank46567 +46568 POINT(30.054416689126775 -95.60890540425889) bank46568 +46569 POINT(30.366603270238677 -94.51950997283065) bank46569 +46570 POINT(30.28832546648484 -95.99823410347284) bank46570 +46571 POINT(29.659732890918367 -96.15494208662273) bank46571 +46572 POINT(29.192839460185862 -95.04992038737905) bank46572 +46573 POINT(30.02704754849995 -95.10986295872088) bank46573 +46574 POINT(30.465183986308748 -95.19812639917433) bank46574 +46575 POINT(30.718696278983685 -94.90032169401509) bank46575 +46576 POINT(29.19421136597114 -94.39725677036867) bank46576 +46577 POINT(29.292459450424047 -94.7682676129306) bank46577 +46578 POINT(29.80206351204902 -95.79999650516584) bank46578 +46579 POINT(28.818962463039274 -95.20413098338243) bank46579 +46580 POINT(30.10605241273257 -95.69683451598546) bank46580 +46581 POINT(30.272563619417905 -96.21057400887119) bank46581 +46582 POINT(30.69572053664492 -94.60190944769803) bank46582 +46583 POINT(29.30807797312472 -95.67164117390546) bank46583 +46584 POINT(29.563149702769874 -96.13533894280556) bank46584 +46585 POINT(29.214196472279124 -95.51900160549242) bank46585 +46586 POINT(28.96182655471307 -95.55929548596127) bank46586 +46587 POINT(30.56437838337112 -95.57302474813613) bank46587 +46588 POINT(29.083355520220447 -95.88658803243294) bank46588 +46589 POINT(30.100484915835306 -95.49235959937677) bank46589 +46590 POINT(29.82699054692846 -94.70317795617841) bank46590 +46591 POINT(29.02402742334274 -96.16643137546122) bank46591 +46592 POINT(29.390563610069577 -95.20992255588486) bank46592 +46593 POINT(29.97141315681299 -95.062019218375) bank46593 +46594 POINT(30.61629769632722 -96.18535969005505) bank46594 +46595 POINT(28.99371609267673 -95.3122754372293) bank46595 +46596 POINT(29.90603683373174 -94.49567579934059) bank46596 +46597 POINT(29.526821878683158 -95.24558259147157) bank46597 +46598 POINT(29.98275862929097 -95.59203038898984) bank46598 +46599 POINT(29.334563708990768 -95.06504413447803) bank46599 +46600 POINT(29.22273156991236 -95.13595861696422) bank46600 +46601 POINT(28.966761845975444 -94.7342672250646) bank46601 +46602 POINT(28.833966833277604 -94.8612716105882) bank46602 +46603 POINT(29.338451238614763 -95.31018794627967) bank46603 +46604 POINT(29.81894140015089 -95.28918429665232) bank46604 +46605 POINT(29.307101850234453 -95.68979733617743) bank46605 +46606 POINT(30.479575694726922 -94.60866271979887) bank46606 +46607 POINT(29.535617281751236 -95.12123401966697) bank46607 +46608 POINT(30.161894818912337 -95.51745367489482) bank46608 +46609 POINT(29.481159709869207 -95.0628171810118) bank46609 +46610 POINT(29.491557740243532 -96.20315721326105) bank46610 +46611 POINT(29.592828473645817 -94.52236578435763) bank46611 +46612 POINT(28.964426421869394 -96.17957099221657) bank46612 +46613 POINT(30.48685545048591 -95.64572666185974) bank46613 +46614 POINT(30.179295702522655 -94.70613281314591) bank46614 +46615 POINT(30.395694233127593 -95.08575529616816) bank46615 +46616 POINT(30.492372709882517 -96.34592845346765) bank46616 +46617 POINT(30.326089656261754 -95.88901425880113) bank46617 +46618 POINT(30.503311427458566 -94.66176452074305) bank46618 +46619 POINT(29.64752982501925 -95.7947728393406) bank46619 +46620 POINT(29.1973231517427 -95.97093438663963) bank46620 +46621 POINT(29.878828122467056 -95.19058344306227) bank46621 +46622 POINT(29.624345429580487 -95.0236536877654) bank46622 +46623 POINT(30.013703424249652 -95.88699717096924) bank46623 +46624 POINT(29.89245352491947 -95.96815389538371) bank46624 +46625 POINT(29.524202107086047 -95.91132723689687) bank46625 +46626 POINT(30.717025647274856 -94.68583308820328) bank46626 +46627 POINT(29.914468235363426 -95.05721681841445) bank46627 +46628 POINT(30.438200727629205 -94.81654379387459) bank46628 +46629 POINT(30.132953891562853 -96.00611701722744) bank46629 +46630 POINT(29.770397619789023 -94.85128652877255) bank46630 +46631 POINT(29.32764904489181 -95.29381424211998) bank46631 +46632 POINT(29.780083819144625 -94.59478363443152) bank46632 +46633 POINT(30.432121477800788 -96.04532445770867) bank46633 +46634 POINT(29.822886459895255 -96.1433386924371) bank46634 +46635 POINT(30.50426517544538 -95.0456651338674) bank46635 +46636 POINT(29.15159414716012 -96.15585951296194) bank46636 +46637 POINT(28.815604936507444 -95.5131799283416) bank46637 +46638 POINT(29.55667747253162 -94.94207266897543) bank46638 +46639 POINT(30.262138227265 -95.76309193513423) bank46639 +46640 POINT(29.75024522837083 -95.97322643320922) bank46640 +46641 POINT(29.14021446391661 -96.27610824260022) bank46641 +46642 POINT(30.633150546802103 -95.33759231954195) bank46642 +46643 POINT(30.552185895771423 -95.90605710573693) bank46643 +46644 POINT(30.53107558382614 -96.22942468788742) bank46644 +46645 POINT(29.641815536507856 -96.11957393812409) bank46645 +46646 POINT(30.167861599087008 -95.03981908168149) bank46646 +46647 POINT(30.3405660660971 -94.70444422882538) bank46647 +46648 POINT(30.062454558556325 -95.5256877097147) bank46648 +46649 POINT(29.34379727934723 -95.92897541280189) bank46649 +46650 POINT(30.046938373605226 -94.61456570606842) bank46650 +46651 POINT(30.206891773954787 -95.62071124630575) bank46651 +46652 POINT(28.79809771230002 -95.68013783290992) bank46652 +46653 POINT(30.34637635887345 -95.97067832566779) bank46653 +46654 POINT(30.68764571870738 -96.14316229465416) bank46654 +46655 POINT(30.379458322241746 -94.66272774444305) bank46655 +46656 POINT(29.991621026653448 -96.10082366892587) bank46656 +46657 POINT(29.93873614349222 -95.41115390253754) bank46657 +46658 POINT(29.9830314778091 -96.27800099934458) bank46658 +46659 POINT(30.663920724148603 -95.90674787200861) bank46659 +46660 POINT(29.078368912412316 -96.099738756006) bank46660 +46661 POINT(29.548584605968273 -95.97172542686602) bank46661 +46662 POINT(30.513352156815145 -94.71890922987065) bank46662 +46663 POINT(29.059589664993585 -94.84695042461625) bank46663 +46664 POINT(29.295172179184622 -94.49021591963289) bank46664 +46665 POINT(30.67344327653062 -95.2747622546769) bank46665 +46666 POINT(29.36418887951435 -95.9335932694488) bank46666 +46667 POINT(29.271427982769236 -95.84890485904623) bank46667 +46668 POINT(30.33699211509615 -96.12875627040438) bank46668 +46669 POINT(30.097008800862234 -95.71822193820095) bank46669 +46670 POINT(29.666148014414556 -94.40052195624062) bank46670 +46671 POINT(29.45235026926402 -95.98458230014522) bank46671 +46672 POINT(30.064928375998104 -94.37074680784052) bank46672 +46673 POINT(29.27610774248222 -94.39239569654458) bank46673 +46674 POINT(29.733220318600754 -94.78571292018438) bank46674 +46675 POINT(30.19861798626804 -94.96036443619745) bank46675 +46676 POINT(30.264037602163537 -94.83079182672968) bank46676 +46677 POINT(29.056201727960104 -96.09921350226746) bank46677 +46678 POINT(29.401522499322077 -95.56553362113317) bank46678 +46679 POINT(29.92633954390134 -95.02518089384357) bank46679 +46680 POINT(29.884787605266737 -94.45753758803151) bank46680 +46681 POINT(30.235415085773454 -95.26176382790456) bank46681 +46682 POINT(30.13659367850961 -96.20964588780355) bank46682 +46683 POINT(29.330193906397426 -95.68746108016735) bank46683 +46684 POINT(29.823612045159113 -95.00345680002623) bank46684 +46685 POINT(30.581620333132303 -95.58484863437195) bank46685 +46686 POINT(30.184235613019958 -95.42790228092021) bank46686 +46687 POINT(29.56463292407003 -94.83508540345353) bank46687 +46688 POINT(29.480339865346092 -96.30454536746586) bank46688 +46689 POINT(29.232176140096335 -95.65591885599052) bank46689 +46690 POINT(30.61708403329748 -95.16140914568835) bank46690 +46691 POINT(30.568576745383996 -95.43640207219053) bank46691 +46692 POINT(30.48567738161866 -94.92564090578443) bank46692 +46693 POINT(30.742247325426096 -95.78793847066353) bank46693 +46694 POINT(29.611718469667036 -95.28632297654917) bank46694 +46695 POINT(29.878721128832694 -95.67864800480208) bank46695 +46696 POINT(30.55555402347855 -94.73715353293741) bank46696 +46697 POINT(28.804753706996543 -96.15169638913119) bank46697 +46698 POINT(30.22723625428768 -96.074779154423) bank46698 +46699 POINT(28.76350707553236 -94.74631269050182) bank46699 +46700 POINT(29.845223156052025 -94.42617740755733) bank46700 +46701 POINT(29.344661526208718 -94.66732774373715) bank46701 +46702 POINT(29.265619629077506 -94.6219548005911) bank46702 +46703 POINT(28.9322406726766 -95.47582363114985) bank46703 +46704 POINT(29.975893102218944 -95.71116040965043) bank46704 +46705 POINT(30.701156117792426 -95.26873949714792) bank46705 +46706 POINT(29.800242460727628 -96.24794167144364) bank46706 +46707 POINT(28.76049579294064 -95.71005732372366) bank46707 +46708 POINT(30.6618245042176 -95.59236685820885) bank46708 +46709 POINT(29.429826370367568 -96.2200892724156) bank46709 +46710 POINT(29.478987452304693 -94.80682420046602) bank46710 +46711 POINT(28.870916704743724 -95.4973812613333) bank46711 +46712 POINT(29.673711172302944 -95.53545563396476) bank46712 +46713 POINT(30.408425856353546 -95.2535507776621) bank46713 +46714 POINT(30.228685236535874 -96.31507500031144) bank46714 +46715 POINT(29.525591599949205 -95.5848127219717) bank46715 +46716 POINT(29.87279014473359 -96.28504381846474) bank46716 +46717 POINT(29.80602584378062 -96.15353400412597) bank46717 +46718 POINT(30.0792298626155 -94.52971021214826) bank46718 +46719 POINT(30.618721219445096 -94.44251759640237) bank46719 +46720 POINT(30.64000608993192 -95.19629187943268) bank46720 +46721 POINT(30.301937462925427 -95.50834654249056) bank46721 +46722 POINT(30.54375263948149 -95.24156226382956) bank46722 +46723 POINT(30.58169601996708 -95.70201635964087) bank46723 +46724 POINT(29.959405469574957 -95.58417909830253) bank46724 +46725 POINT(30.547102883056564 -95.27374920656057) bank46725 +46726 POINT(29.802859458811184 -95.43320335123188) bank46726 +46727 POINT(29.6150547540901 -94.44189247734789) bank46727 +46728 POINT(30.44979082743026 -95.25772258008736) bank46728 +46729 POINT(28.76681781822847 -94.66163131391954) bank46729 +46730 POINT(30.296234132710474 -95.02338087066404) bank46730 +46731 POINT(30.332706677842005 -95.3541745713056) bank46731 +46732 POINT(29.00700790529666 -96.13906557673891) bank46732 +46733 POINT(30.70093659886971 -95.49874167025663) bank46733 +46734 POINT(29.35131260648615 -94.42672186707897) bank46734 +46735 POINT(30.52147081846661 -94.52306268349375) bank46735 +46736 POINT(29.189185483540303 -95.71949658170779) bank46736 +46737 POINT(30.70192216975692 -94.39531079717388) bank46737 +46738 POINT(28.9182566078779 -94.93672227275101) bank46738 +46739 POINT(30.42118376538433 -95.00974627388429) bank46739 +46740 POINT(30.725109680798415 -94.37403008300696) bank46740 +46741 POINT(30.51120705366516 -95.54051331151771) bank46741 +46742 POINT(30.091699725703343 -95.53226238106839) bank46742 +46743 POINT(28.936054417519717 -96.2935817187306) bank46743 +46744 POINT(29.940250071378617 -96.33916875413473) bank46744 +46745 POINT(29.096620371546017 -94.91773885672679) bank46745 +46746 POINT(30.655312053597267 -94.87561452868532) bank46746 +46747 POINT(30.11933418500226 -95.7174882995431) bank46747 +46748 POINT(30.005236463825042 -96.02241590050822) bank46748 +46749 POINT(30.423695912254228 -95.26529569583703) bank46749 +46750 POINT(29.832758725556268 -94.40601592234333) bank46750 +46751 POINT(29.942755730606066 -94.72445377739092) bank46751 +46752 POINT(30.707715898602235 -96.2306795204621) bank46752 +46753 POINT(29.91688709655 -96.06094068245864) bank46753 +46754 POINT(29.954484758020644 -95.97745264546029) bank46754 +46755 POINT(28.846088693224477 -96.29286505188803) bank46755 +46756 POINT(29.93110134018564 -94.86224597052474) bank46756 +46757 POINT(29.3525818593469 -94.6818768137708) bank46757 +46758 POINT(30.124461348660795 -95.23648892401812) bank46758 +46759 POINT(29.753823107679196 -95.44417869949656) bank46759 +46760 POINT(29.826974483521575 -95.35227644904502) bank46760 +46761 POINT(28.947982870048715 -94.99900507026322) bank46761 +46762 POINT(29.985196310120894 -95.48014540447782) bank46762 +46763 POINT(29.384094670090608 -94.94386361782801) bank46763 +46764 POINT(30.122622099389435 -95.05124963406041) bank46764 +46765 POINT(30.736240277063704 -95.27097706912095) bank46765 +46766 POINT(29.811816743598104 -94.79013168321129) bank46766 +46767 POINT(29.949230530680836 -94.3837550250271) bank46767 +46768 POINT(30.123197144436126 -95.34357169114546) bank46768 +46769 POINT(29.79066908357399 -96.19725301720354) bank46769 +46770 POINT(30.722011839095877 -94.90133256467645) bank46770 +46771 POINT(30.711282314130504 -95.78215040101107) bank46771 +46772 POINT(29.727970635347777 -95.24253096191318) bank46772 +46773 POINT(29.48058889721027 -95.57602133459808) bank46773 +46774 POINT(29.87514556156878 -95.22137971029684) bank46774 +46775 POINT(29.10662790209106 -95.14009497564513) bank46775 +46776 POINT(30.36666059626762 -94.6058440590534) bank46776 +46777 POINT(30.237734788527092 -95.42572043897418) bank46777 +46778 POINT(30.503398122523294 -94.54360012597063) bank46778 +46779 POINT(30.192925055725546 -95.59550169933648) bank46779 +46780 POINT(29.50147814187647 -95.92447367675462) bank46780 +46781 POINT(28.908088439884573 -95.03952923168796) bank46781 +46782 POINT(29.477862235476668 -96.28127816408305) bank46782 +46783 POINT(30.446455368199178 -95.37915839436492) bank46783 +46784 POINT(30.077693577859844 -95.71080304166497) bank46784 +46785 POINT(30.14900370388151 -95.71039729537047) bank46785 +46786 POINT(29.497833887324656 -95.3052584871313) bank46786 +46787 POINT(29.584692207577522 -94.81080001757584) bank46787 +46788 POINT(30.01739886096805 -95.4380208709434) bank46788 +46789 POINT(30.31312068981633 -95.6919142266646) bank46789 +46790 POINT(29.977465334415662 -94.97789624736646) bank46790 +46791 POINT(30.14851421216053 -96.19695329656057) bank46791 +46792 POINT(29.02164275224518 -95.17780259114184) bank46792 +46793 POINT(28.80415127299407 -95.92635906209226) bank46793 +46794 POINT(30.189212847897622 -95.02754197084359) bank46794 +46795 POINT(30.357663347284785 -94.77544078987512) bank46795 +46796 POINT(29.073651698910655 -95.74943839847418) bank46796 +46797 POINT(30.140370269245448 -95.37950569165865) bank46797 +46798 POINT(29.19998126860241 -96.30149699690331) bank46798 +46799 POINT(30.08641925652629 -96.21108502329439) bank46799 +46800 POINT(29.21279534070926 -94.99397366509999) bank46800 +46801 POINT(29.747688858333788 -94.48687162344594) bank46801 +46802 POINT(30.31552507199647 -96.20588971226196) bank46802 +46803 POINT(30.287846236745818 -95.6276122560436) bank46803 +46804 POINT(29.589700184632736 -94.56447417008992) bank46804 +46805 POINT(29.56444717709991 -96.18688559008898) bank46805 +46806 POINT(30.48458941214105 -94.70164912906164) bank46806 +46807 POINT(30.229402344482168 -95.8540181933373) bank46807 +46808 POINT(30.199242813943094 -95.4088258900389) bank46808 +46809 POINT(29.023634262905404 -96.32762757280483) bank46809 +46810 POINT(29.56590335021552 -95.90568582075905) bank46810 +46811 POINT(29.405411777548622 -95.13893843324567) bank46811 +46812 POINT(28.97771664730721 -95.9999920750087) bank46812 +46813 POINT(29.919413099997314 -95.8183480380585) bank46813 +46814 POINT(28.958908599146948 -95.90029739681083) bank46814 +46815 POINT(30.72633356224002 -95.11012998381788) bank46815 +46816 POINT(30.57585091229196 -96.21887026165389) bank46816 +46817 POINT(28.819060420550382 -94.45228431535021) bank46817 +46818 POINT(29.683285409183295 -94.4460907771538) bank46818 +46819 POINT(28.97822443183897 -95.60048800574764) bank46819 +46820 POINT(29.415467946114237 -96.23841073349352) bank46820 +46821 POINT(30.359856027125815 -95.01095986950317) bank46821 +46822 POINT(30.362845940113093 -95.43599687029322) bank46822 +46823 POINT(29.077588189678906 -95.03308497448376) bank46823 +46824 POINT(29.45110551874843 -95.9744727500842) bank46824 +46825 POINT(29.621435505505026 -94.7128835367334) bank46825 +46826 POINT(28.96967787021417 -95.02991566962415) bank46826 +46827 POINT(29.143328611370528 -95.13062937499272) bank46827 +46828 POINT(28.865470616927485 -95.23544584448476) bank46828 +46829 POINT(30.347235675266752 -96.02419390154617) bank46829 +46830 POINT(29.419019997479868 -94.76700004837772) bank46830 +46831 POINT(30.299571451925562 -95.38548684852431) bank46831 +46832 POINT(30.335498384018354 -95.67501395951547) bank46832 +46833 POINT(30.633049377125744 -95.74763280371855) bank46833 +46834 POINT(29.230125359755235 -94.55484992431431) bank46834 +46835 POINT(29.85966995083342 -94.67540267852853) bank46835 +46836 POINT(30.33405548433663 -96.36802183359632) bank46836 +46837 POINT(29.972553745478084 -95.2134400641675) bank46837 +46838 POINT(30.470374923028974 -96.2076863649084) bank46838 +46839 POINT(30.24619324181383 -95.11036275687887) bank46839 +46840 POINT(30.687582712026188 -95.936374722138) bank46840 +46841 POINT(29.373215792520366 -96.08658824993145) bank46841 +46842 POINT(29.817699807263782 -96.04515508234549) bank46842 +46843 POINT(30.435740831848364 -94.76883548121292) bank46843 +46844 POINT(29.910020593497908 -94.74449723286286) bank46844 +46845 POINT(29.76896731588239 -94.87190603600632) bank46845 +46846 POINT(30.332553026558454 -95.7566658594049) bank46846 +46847 POINT(30.306071469958937 -95.81398110257302) bank46847 +46848 POINT(30.60776775284404 -96.23368448594391) bank46848 +46849 POINT(29.527071436565834 -96.34258813418118) bank46849 +46850 POINT(30.205914880391525 -94.96411166608081) bank46850 +46851 POINT(28.901401068205796 -94.64769774358778) bank46851 +46852 POINT(29.07549636869505 -94.78573256799528) bank46852 +46853 POINT(29.366440014839252 -96.28917689160764) bank46853 +46854 POINT(28.849703402317413 -96.01635737956494) bank46854 +46855 POINT(29.63910863113821 -95.53424578438371) bank46855 +46856 POINT(30.12107755865941 -95.84612759397021) bank46856 +46857 POINT(29.459906338918895 -95.93587192843043) bank46857 +46858 POINT(28.90946923153667 -95.97926959023741) bank46858 +46859 POINT(29.694989178642384 -94.88985846151188) bank46859 +46860 POINT(30.630398510923555 -96.05099084782806) bank46860 +46861 POINT(29.177399220506935 -95.11271337117175) bank46861 +46862 POINT(29.82922095895687 -95.63221634973391) bank46862 +46863 POINT(30.711498505762627 -94.46604278536478) bank46863 +46864 POINT(29.85770366670923 -95.51252005868837) bank46864 +46865 POINT(30.126422563824086 -95.25769736518718) bank46865 +46866 POINT(30.23084516680893 -95.17951987767205) bank46866 +46867 POINT(29.056352777795894 -95.96268317818343) bank46867 +46868 POINT(30.35296526094425 -95.01213123733095) bank46868 +46869 POINT(28.85449181107539 -96.03115713020175) bank46869 +46870 POINT(28.91870788959447 -96.00572956825152) bank46870 +46871 POINT(30.68928811904863 -95.31765738188201) bank46871 +46872 POINT(30.568670760268343 -94.97157358166471) bank46872 +46873 POINT(29.517148243234622 -95.30273321090262) bank46873 +46874 POINT(30.022736986662444 -94.640399597322) bank46874 +46875 POINT(30.652862688396283 -94.75739970142902) bank46875 +46876 POINT(29.020748810116544 -95.38748777410504) bank46876 +46877 POINT(28.861913144926156 -96.1556509104765) bank46877 +46878 POINT(29.24114632505209 -95.43394319189136) bank46878 +46879 POINT(29.916233201426536 -96.17493321710981) bank46879 +46880 POINT(30.428040020006456 -95.27956709809337) bank46880 +46881 POINT(29.5742751255155 -95.93180308667095) bank46881 +46882 POINT(30.165014972275674 -94.57850778412607) bank46882 +46883 POINT(29.218530646915642 -95.78062925131825) bank46883 +46884 POINT(29.665243437970314 -94.71178356574434) bank46884 +46885 POINT(29.735160554305875 -94.70454274747507) bank46885 +46886 POINT(29.651396178133997 -95.15039191291797) bank46886 +46887 POINT(30.24074712143312 -95.13268803300049) bank46887 +46888 POINT(29.634181692160762 -96.08513922570731) bank46888 +46889 POINT(29.626858521086895 -95.41120464721203) bank46889 +46890 POINT(28.886717527707127 -94.4265802736649) bank46890 +46891 POINT(29.54279171851997 -94.49296714250751) bank46891 +46892 POINT(30.114092478215806 -94.38046263575342) bank46892 +46893 POINT(28.92830849089801 -95.05702793318179) bank46893 +46894 POINT(30.15526008880088 -95.41095503505458) bank46894 +46895 POINT(29.973773567590218 -95.32829481117214) bank46895 +46896 POINT(28.776255363889014 -94.85868753484763) bank46896 +46897 POINT(30.28612171469985 -95.69750114938945) bank46897 +46898 POINT(29.54979001956328 -95.80556467006986) bank46898 +46899 POINT(29.066625227138456 -94.8343790712725) bank46899 +46900 POINT(29.47505769385121 -95.88267585618169) bank46900 +46901 POINT(30.016126242163036 -95.50500242939364) bank46901 +46902 POINT(28.78787738164867 -95.81571215502007) bank46902 +46903 POINT(30.34685538741348 -95.92896661564626) bank46903 +46904 POINT(30.576540699860274 -95.43482933873636) bank46904 +46905 POINT(29.921560168003353 -96.32552040712392) bank46905 +46906 POINT(28.82566215805501 -95.84480873186816) bank46906 +46907 POINT(28.89496326380602 -94.50302835046412) bank46907 +46908 POINT(29.22485208193838 -95.0130057433846) bank46908 +46909 POINT(29.252330818534016 -95.90536263930211) bank46909 +46910 POINT(30.756236781563686 -94.97394108216726) bank46910 +46911 POINT(29.471280293045794 -95.28312471559435) bank46911 +46912 POINT(28.841158762710286 -95.64724088084394) bank46912 +46913 POINT(30.470526309685827 -94.51103442338912) bank46913 +46914 POINT(30.576837004328688 -95.5241414545305) bank46914 +46915 POINT(30.531358345213306 -96.14347470209678) bank46915 +46916 POINT(29.530526099266403 -95.33653557455142) bank46916 +46917 POINT(28.876927337759263 -95.2464929463847) bank46917 +46918 POINT(29.980699888502432 -95.04529441328086) bank46918 +46919 POINT(30.49863150134251 -95.31999834180355) bank46919 +46920 POINT(30.24312268594011 -96.27964780082402) bank46920 +46921 POINT(29.947656821450796 -96.12942451627764) bank46921 +46922 POINT(29.747087187457367 -95.10309016055024) bank46922 +46923 POINT(30.74598590269482 -94.4195987228327) bank46923 +46924 POINT(29.22750352309049 -96.02800560551164) bank46924 +46925 POINT(29.970734878932824 -94.93260017598716) bank46925 +46926 POINT(30.575360949594643 -94.96293695064136) bank46926 +46927 POINT(29.571093868581396 -95.01878483779568) bank46927 +46928 POINT(28.79169409588757 -96.32028017918097) bank46928 +46929 POINT(29.76633136894627 -95.34486351464086) bank46929 +46930 POINT(29.188755591196934 -95.95303101276171) bank46930 +46931 POINT(29.805295374220133 -94.49764663398349) bank46931 +46932 POINT(30.211274814858857 -95.07426019244198) bank46932 +46933 POINT(28.97648524057438 -95.50091413779526) bank46933 +46934 POINT(30.17685941100711 -94.59660730338366) bank46934 +46935 POINT(28.86351158675652 -95.65324687328835) bank46935 +46936 POINT(29.489069329276965 -96.26331615969312) bank46936 +46937 POINT(29.449907611276437 -96.21438275080617) bank46937 +46938 POINT(28.87198317552146 -94.6328012561267) bank46938 +46939 POINT(29.440256476768045 -94.678991773649) bank46939 +46940 POINT(30.438960782883807 -94.73814631070749) bank46940 +46941 POINT(29.49510430787683 -94.75501899309327) bank46941 +46942 POINT(30.326276118701244 -95.92881634599374) bank46942 +46943 POINT(30.21500734048361 -94.57354575715829) bank46943 +46944 POINT(29.42391372316761 -94.9534132789205) bank46944 +46945 POINT(30.211747883283685 -96.08028574831621) bank46945 +46946 POINT(29.170428975515907 -96.16887596881259) bank46946 +46947 POINT(29.391317511133174 -96.28563025593584) bank46947 +46948 POINT(28.820105809185588 -94.95462002110972) bank46948 +46949 POINT(29.843933868408527 -95.36553455283301) bank46949 +46950 POINT(29.967991363104407 -94.97477392198024) bank46950 +46951 POINT(29.085466076827803 -95.47193228772846) bank46951 +46952 POINT(29.566928233097514 -95.56865444186882) bank46952 +46953 POINT(28.804972962403333 -95.06627231495581) bank46953 +46954 POINT(29.948932922769878 -95.06912415139641) bank46954 +46955 POINT(29.58197817370064 -95.43878626302424) bank46955 +46956 POINT(29.437771774843004 -94.37488532286388) bank46956 +46957 POINT(29.7582366540988 -95.92504381201047) bank46957 +46958 POINT(30.57510805315284 -94.38046365728556) bank46958 +46959 POINT(29.945404449145475 -94.54556687912044) bank46959 +46960 POINT(30.726540399854148 -95.20483291626053) bank46960 +46961 POINT(29.198920501795204 -94.57918006419266) bank46961 +46962 POINT(30.444799807519146 -94.63691681805523) bank46962 +46963 POINT(30.362414551629257 -95.6391750307836) bank46963 +46964 POINT(29.97325784822467 -94.48396085013214) bank46964 +46965 POINT(30.621532298283217 -94.57995832546437) bank46965 +46966 POINT(28.849748057942115 -94.38868830099632) bank46966 +46967 POINT(29.959211077615848 -94.47763896791948) bank46967 +46968 POINT(29.552038257585224 -95.61492752784456) bank46968 +46969 POINT(30.045463261500586 -96.04817156551236) bank46969 +46970 POINT(30.08224842157887 -96.34903668778007) bank46970 +46971 POINT(30.387150787062907 -94.89566822220691) bank46971 +46972 POINT(29.297981657821406 -95.90656234560215) bank46972 +46973 POINT(30.254486862803518 -95.00536766874058) bank46973 +46974 POINT(30.201160685499357 -94.4511401352688) bank46974 +46975 POINT(28.900090814613666 -94.57139915709584) bank46975 +46976 POINT(29.123611530095253 -96.08280286610099) bank46976 +46977 POINT(29.473220277720312 -94.79181729356424) bank46977 +46978 POINT(29.95971201969059 -96.07188806197169) bank46978 +46979 POINT(29.028081509903465 -94.71729113746633) bank46979 +46980 POINT(29.957390093368314 -96.13511363933982) bank46980 +46981 POINT(28.98113596610617 -94.85036677970301) bank46981 +46982 POINT(29.45916738753062 -94.38493826600299) bank46982 +46983 POINT(29.52392875899947 -95.08671349532328) bank46983 +46984 POINT(29.98428482939931 -95.57570730586943) bank46984 +46985 POINT(28.8188270823844 -95.91673316988984) bank46985 +46986 POINT(30.156671004520888 -95.78515789943772) bank46986 +46987 POINT(29.486413395120753 -94.62360650054228) bank46987 +46988 POINT(30.238699598040107 -94.43870607883679) bank46988 +46989 POINT(30.351735242650502 -96.23639873270189) bank46989 +46990 POINT(30.613955780092986 -94.76142672588973) bank46990 +46991 POINT(29.10019789617271 -95.35330179054048) bank46991 +46992 POINT(29.549115504414548 -95.62496663449757) bank46992 +46993 POINT(29.51350150402195 -95.47383340026838) bank46993 +46994 POINT(30.30024570779619 -95.58113372793626) bank46994 +46995 POINT(30.33249080941036 -94.56865551959808) bank46995 +46996 POINT(29.499168638496368 -95.64076744337785) bank46996 +46997 POINT(30.034128613768416 -94.84897923044272) bank46997 +46998 POINT(30.589111890917682 -96.23065854485955) bank46998 +46999 POINT(29.804789877273663 -95.74375474140881) bank46999 +47000 POINT(30.6183185774363 -94.61223712636416) bank47000 +47001 POINT(29.81153173889035 -96.21760430625885) bank47001 +47002 POINT(29.721268715219374 -95.67280019871079) bank47002 +47003 POINT(29.428260092547045 -95.83136129420554) bank47003 +47004 POINT(29.761054844484548 -94.90755742948288) bank47004 +47005 POINT(30.496172300875205 -95.25288606419622) bank47005 +47006 POINT(29.494818680026476 -94.51597367427753) bank47006 +47007 POINT(30.34583601023653 -94.55940161764194) bank47007 +47008 POINT(29.89631691826463 -95.38591374834844) bank47008 +47009 POINT(30.27462814997648 -94.42835799124475) bank47009 +47010 POINT(29.226704759381075 -94.90723131851864) bank47010 +47011 POINT(30.434236181600976 -95.20347705506074) bank47011 +47012 POINT(29.997073244451947 -95.01334701179027) bank47012 +47013 POINT(30.756144126897425 -95.28706420163606) bank47013 +47014 POINT(29.450687437918955 -94.37032480570836) bank47014 +47015 POINT(29.867278638858288 -95.97981951572292) bank47015 +47016 POINT(29.750352530454887 -96.18761863848009) bank47016 +47017 POINT(29.847799087309102 -95.5678200721539) bank47017 +47018 POINT(29.56063498298826 -95.97096448982117) bank47018 +47019 POINT(30.284199833126603 -95.21398427031995) bank47019 +47020 POINT(29.91798397417397 -95.90872720796824) bank47020 +47021 POINT(29.225732859816105 -96.1985273922304) bank47021 +47022 POINT(30.546436933160326 -95.70256947507487) bank47022 +47023 POINT(29.79255366547218 -94.40378783550834) bank47023 +47024 POINT(29.87608253908346 -95.08536331753666) bank47024 +47025 POINT(29.234650852842684 -94.69251270285781) bank47025 +47026 POINT(29.33656742519561 -96.22315141210522) bank47026 +47027 POINT(30.569294694174616 -96.18105446838148) bank47027 +47028 POINT(29.05930514516624 -94.92578423812263) bank47028 +47029 POINT(30.248587359874737 -96.06118589482512) bank47029 +47030 POINT(30.00108567427108 -95.88462368073273) bank47030 +47031 POINT(29.91808469073246 -95.70798778245111) bank47031 +47032 POINT(29.26807143950736 -94.90522322702205) bank47032 +47033 POINT(30.29105422493451 -95.55386381276332) bank47033 +47034 POINT(30.27860814882318 -95.05050650046921) bank47034 +47035 POINT(30.394878069585328 -95.86227457788773) bank47035 +47036 POINT(30.740274593968387 -94.49710870776364) bank47036 +47037 POINT(30.201299701939533 -95.15254869412325) bank47037 +47038 POINT(30.07910576345521 -94.60991836775388) bank47038 +47039 POINT(29.013692784427473 -95.85381857297236) bank47039 +47040 POINT(29.107346943635182 -95.20948675072871) bank47040 +47041 POINT(29.71680184477017 -94.71607949317064) bank47041 +47042 POINT(30.354900760379174 -94.86099493237599) bank47042 +47043 POINT(30.36816725876948 -95.36102464663117) bank47043 +47044 POINT(30.37308287722063 -94.74475278878047) bank47044 +47045 POINT(29.747611180937817 -95.01338417765547) bank47045 +47046 POINT(30.341350674904714 -96.3018750354354) bank47046 +47047 POINT(29.189768259577754 -95.55108263324054) bank47047 +47048 POINT(30.276586962633775 -95.47344880794346) bank47048 +47049 POINT(29.476444022520038 -94.53371577199742) bank47049 +47050 POINT(30.296796878649698 -94.82203666320248) bank47050 +47051 POINT(29.800068303708027 -94.68312786502565) bank47051 +47052 POINT(29.782834523485132 -94.43827401330245) bank47052 +47053 POINT(29.027111515427656 -96.07737653400372) bank47053 +47054 POINT(30.074650679210198 -96.17148292613807) bank47054 +47055 POINT(28.920676858120913 -96.297098041521) bank47055 +47056 POINT(29.543873893601365 -95.45985996759696) bank47056 +47057 POINT(29.44382731782511 -95.78087590344622) bank47057 +47058 POINT(30.425376398598623 -95.29697791139392) bank47058 +47059 POINT(29.613700475767782 -96.10541257692923) bank47059 +47060 POINT(30.533466063357935 -96.27528330822251) bank47060 +47061 POINT(30.151688059795497 -95.87699829419647) bank47061 +47062 POINT(29.5077973321646 -94.45077256374799) bank47062 +47063 POINT(28.76380476902555 -94.91175477973987) bank47063 +47064 POINT(28.922127791024824 -95.3758614824627) bank47064 +47065 POINT(30.503816816309136 -96.05837939426958) bank47065 +47066 POINT(28.97626596475604 -94.43648670284054) bank47066 +47067 POINT(29.669290500238905 -95.0395695784403) bank47067 +47068 POINT(29.013867380540603 -95.24961767166465) bank47068 +47069 POINT(29.407190547878418 -95.61062386877798) bank47069 +47070 POINT(29.450521943097694 -95.79906568091128) bank47070 +47071 POINT(29.31527601894782 -95.88513145019795) bank47071 +47072 POINT(30.37872588232203 -94.95926617380128) bank47072 +47073 POINT(29.25633415306407 -94.48736155875136) bank47073 +47074 POINT(30.661843002215964 -95.26216167666679) bank47074 +47075 POINT(29.9226241935428 -96.15127106147344) bank47075 +47076 POINT(30.416956518707813 -96.1729184859312) bank47076 +47077 POINT(30.192975432658283 -94.9690177322381) bank47077 +47078 POINT(30.212153447513366 -95.43483157801174) bank47078 +47079 POINT(30.321945265570918 -94.84232151535613) bank47079 +47080 POINT(29.376196360267762 -95.75296477851224) bank47080 +47081 POINT(29.528542656702722 -95.14348793777117) bank47081 +47082 POINT(29.372531393910204 -94.46055724483166) bank47082 +47083 POINT(29.682215371902355 -95.19026929899834) bank47083 +47084 POINT(29.84725975113209 -94.80606372047198) bank47084 +47085 POINT(30.695540559433827 -96.06106627501188) bank47085 +47086 POINT(30.340500905554688 -94.72946494699853) bank47086 +47087 POINT(29.874951326632697 -95.56812513711611) bank47087 +47088 POINT(29.480044744599798 -94.46376100652768) bank47088 +47089 POINT(28.963400927656817 -94.70274838301152) bank47089 +47090 POINT(29.76861976366827 -95.86908733432406) bank47090 +47091 POINT(28.909423308442065 -95.55959516146264) bank47091 +47092 POINT(30.13384553095912 -94.72876272263424) bank47092 +47093 POINT(30.278570074840996 -95.81365655209144) bank47093 +47094 POINT(29.73546190410469 -94.83999645333866) bank47094 +47095 POINT(29.318092242495112 -94.75018515576296) bank47095 +47096 POINT(30.755875226256276 -94.50961480624834) bank47096 +47097 POINT(29.246791364092722 -95.1723505679095) bank47097 +47098 POINT(30.62419307050523 -95.6857384642934) bank47098 +47099 POINT(30.31426891379207 -94.49523382533319) bank47099 +47100 POINT(29.919413003025614 -96.35255812564304) bank47100 +47101 POINT(30.75671395870984 -95.40541495910539) bank47101 +47102 POINT(30.34345215710992 -96.13117381626239) bank47102 +47103 POINT(28.878784713817513 -96.07635182728846) bank47103 +47104 POINT(28.883151214787183 -95.63562127880142) bank47104 +47105 POINT(29.25148386844527 -94.65310651243587) bank47105 +47106 POINT(29.722572873644953 -96.22893959814189) bank47106 +47107 POINT(28.783224266786817 -95.83149619264245) bank47107 +47108 POINT(28.813892999979565 -96.07241983992996) bank47108 +47109 POINT(29.25166409308324 -95.7588229190757) bank47109 +47110 POINT(29.57042750391295 -95.4325266589589) bank47110 +47111 POINT(29.64534173334447 -94.43745980234748) bank47111 +47112 POINT(30.740185358985254 -95.1714027691149) bank47112 +47113 POINT(29.532850178613213 -96.19774427655912) bank47113 +47114 POINT(29.295081106467418 -95.94280425502352) bank47114 +47115 POINT(30.527812616981823 -94.5925429155999) bank47115 +47116 POINT(30.450274142728038 -94.82768466274065) bank47116 +47117 POINT(30.13579267864011 -94.86820098282938) bank47117 +47118 POINT(30.479322554084064 -95.01451166281888) bank47118 +47119 POINT(29.20060530987235 -95.75103534762695) bank47119 +47120 POINT(30.180364711575145 -95.24440532086118) bank47120 +47121 POINT(30.193973379113288 -95.07502186150172) bank47121 +47122 POINT(29.594129421138074 -95.70879799683965) bank47122 +47123 POINT(29.270980921213543 -94.47925945645532) bank47123 +47124 POINT(30.132801252152127 -96.2479715939347) bank47124 +47125 POINT(29.966376487100664 -94.69522726162685) bank47125 +47126 POINT(30.15032082015169 -95.26717837050452) bank47126 +47127 POINT(29.914459825008915 -95.66355673163336) bank47127 +47128 POINT(30.077018060460396 -94.54157712082997) bank47128 +47129 POINT(28.958844755132976 -95.34638061982322) bank47129 +47130 POINT(29.98594146785142 -95.98824980579495) bank47130 +47131 POINT(28.996732717156718 -95.1096101531448) bank47131 +47132 POINT(29.482889029659848 -95.61155134048087) bank47132 +47133 POINT(29.77257165525663 -94.54309120331642) bank47133 +47134 POINT(30.292944836477506 -95.81046936375546) bank47134 +47135 POINT(30.398629257619483 -94.39398232351395) bank47135 +47136 POINT(30.665336942667437 -95.25012252491823) bank47136 +47137 POINT(29.759584466867455 -95.21501181770185) bank47137 +47138 POINT(30.440581788364767 -95.60738247789678) bank47138 +47139 POINT(30.43614340197435 -96.15536854310193) bank47139 +47140 POINT(30.496566742879455 -95.08843307607337) bank47140 +47141 POINT(30.10107149157902 -95.54304257318341) bank47141 +47142 POINT(29.852760592035246 -96.04407289242853) bank47142 +47143 POINT(29.16042591556903 -95.1447103506253) bank47143 +47144 POINT(29.811663910926846 -95.19127533745333) bank47144 +47145 POINT(29.737422480078298 -94.4594013912343) bank47145 +47146 POINT(30.566790269597334 -96.36063930873486) bank47146 +47147 POINT(29.09337781599738 -95.3613618752115) bank47147 +47148 POINT(29.09502303949699 -96.31250032763016) bank47148 +47149 POINT(28.761992042672713 -96.00417747656233) bank47149 +47150 POINT(29.275184403094045 -95.11472226176342) bank47150 +47151 POINT(29.292956108415943 -95.18377453964588) bank47151 +47152 POINT(29.68872720199667 -94.86650530441435) bank47152 +47153 POINT(29.15774595169893 -94.80025504743803) bank47153 +47154 POINT(29.11591097336594 -94.48829479194339) bank47154 +47155 POINT(29.67381306479045 -95.38568172110844) bank47155 +47156 POINT(30.480850042346194 -95.85156401433203) bank47156 +47157 POINT(28.761631334635965 -95.25138600515403) bank47157 +47158 POINT(29.860751259585633 -96.2763067628424) bank47158 +47159 POINT(29.436582241287834 -95.42824870548864) bank47159 +47160 POINT(29.572262044602223 -94.84041239752625) bank47160 +47161 POINT(29.821478144975877 -96.15148764013662) bank47161 +47162 POINT(30.29265589354698 -96.11031523534488) bank47162 +47163 POINT(30.31489520691189 -95.66759002605224) bank47163 +47164 POINT(30.483434625945577 -94.41634018266708) bank47164 +47165 POINT(29.555628128844365 -95.45386641833404) bank47165 +47166 POINT(29.04013705482182 -94.86529800526897) bank47166 +47167 POINT(30.264912070831993 -95.06906001736309) bank47167 +47168 POINT(30.709958796346882 -95.19488594912205) bank47168 +47169 POINT(29.02714510150257 -94.46342482066605) bank47169 +47170 POINT(29.479119667294572 -96.04177001643141) bank47170 +47171 POINT(29.311225982830077 -95.39942292433923) bank47171 +47172 POINT(29.972727404941185 -94.98311725095033) bank47172 +47173 POINT(29.313581137239474 -96.34853774056306) bank47173 +47174 POINT(30.626242013121466 -96.1786926399343) bank47174 +47175 POINT(30.173014765462494 -95.17732193688049) bank47175 +47176 POINT(29.744065522704933 -94.65547352020201) bank47176 +47177 POINT(29.4704996574919 -94.56299441961767) bank47177 +47178 POINT(29.0673039070978 -95.29595604337831) bank47178 +47179 POINT(29.18820063312806 -96.34281783475538) bank47179 +47180 POINT(29.771242100145713 -95.51318383170401) bank47180 +47181 POINT(28.915100625310064 -94.54042754470353) bank47181 +47182 POINT(30.475162733183645 -95.86589681683947) bank47182 +47183 POINT(30.466087982746863 -96.13439624058904) bank47183 +47184 POINT(29.954702541679207 -94.41235350398509) bank47184 +47185 POINT(29.899921931643465 -96.03205011921779) bank47185 +47186 POINT(28.86753236033725 -94.77969128725591) bank47186 +47187 POINT(28.939797684019176 -95.28552667433759) bank47187 +47188 POINT(29.85396742896755 -94.63160667553953) bank47188 +47189 POINT(28.918940593685956 -95.6172770397407) bank47189 +47190 POINT(30.313760571895884 -94.85255201861028) bank47190 +47191 POINT(29.76672665246346 -94.55551213819085) bank47191 +47192 POINT(29.206525113188984 -95.7658408910606) bank47192 +47193 POINT(29.872878507405936 -95.76899475972976) bank47193 +47194 POINT(29.39414971235059 -95.96221758592783) bank47194 +47195 POINT(29.303683714163725 -95.1694138014379) bank47195 +47196 POINT(30.52037031232439 -95.47421008183994) bank47196 +47197 POINT(30.44096688093337 -96.06941960057566) bank47197 +47198 POINT(29.405420407065062 -95.83656576313643) bank47198 +47199 POINT(30.407922971873155 -95.51080352170463) bank47199 +47200 POINT(30.657877202539385 -95.39456578255171) bank47200 +47201 POINT(29.917647382506743 -95.82259495780531) bank47201 +47202 POINT(28.859008156478446 -96.35636667605601) bank47202 +47203 POINT(29.79720095019612 -94.85829795834489) bank47203 +47204 POINT(29.328451234029945 -94.60947483839841) bank47204 +47205 POINT(29.535843005008108 -95.24482628403045) bank47205 +47206 POINT(30.216519149010836 -94.90178133127165) bank47206 +47207 POINT(30.628404317034757 -95.83584527825903) bank47207 +47208 POINT(29.509465427458398 -95.27535068463338) bank47208 +47209 POINT(30.350249160939327 -96.27417371328899) bank47209 +47210 POINT(29.223284046519957 -94.85123085739266) bank47210 +47211 POINT(29.2116340687895 -94.85760374680605) bank47211 +47212 POINT(30.51601377507448 -96.167647295375) bank47212 +47213 POINT(30.196646042196022 -96.3640716934554) bank47213 +47214 POINT(30.381702859271428 -95.76097747276414) bank47214 +47215 POINT(30.063937970935797 -95.8257171092035) bank47215 +47216 POINT(29.644563256259456 -95.70101357963387) bank47216 +47217 POINT(29.623203281966376 -94.61181669923673) bank47217 +47218 POINT(29.9005234394288 -95.92376704393791) bank47218 +47219 POINT(29.728584382677937 -94.43477849995918) bank47219 +47220 POINT(29.43597443957734 -95.77459688144252) bank47220 +47221 POINT(30.64589769884497 -94.86972860195435) bank47221 +47222 POINT(29.19435916471744 -95.41365463785442) bank47222 +47223 POINT(29.681600304732683 -94.60964181589233) bank47223 +47224 POINT(29.50793320259308 -94.64743580379424) bank47224 +47225 POINT(30.73566470559887 -95.20722230445485) bank47225 +47226 POINT(28.9778538558741 -95.64051345793735) bank47226 +47227 POINT(30.464221462518207 -96.07485603541416) bank47227 +47228 POINT(29.344967793465052 -94.37552024840039) bank47228 +47229 POINT(29.53381338396723 -94.71867231307583) bank47229 +47230 POINT(29.49493225206665 -94.92781801250574) bank47230 +47231 POINT(29.925952664311268 -94.40409953161677) bank47231 +47232 POINT(30.24141345790855 -96.16511898734227) bank47232 +47233 POINT(29.51841092744929 -95.23034040886121) bank47233 +47234 POINT(29.774875868306086 -96.31797155680286) bank47234 +47235 POINT(29.103789892999465 -94.95036533523665) bank47235 +47236 POINT(29.247402584663195 -95.6170409851302) bank47236 +47237 POINT(30.459137992696192 -95.19371523131596) bank47237 +47238 POINT(28.962628044079416 -95.59655344931835) bank47238 +47239 POINT(30.235907882974633 -96.1777725906482) bank47239 +47240 POINT(30.60220521771171 -95.79418419038618) bank47240 +47241 POINT(29.14874804202618 -96.18993472427458) bank47241 +47242 POINT(30.625209281729102 -95.38908080553198) bank47242 +47243 POINT(29.861734528822566 -95.35422589871112) bank47243 +47244 POINT(29.60388111523682 -95.49314941102163) bank47244 +47245 POINT(30.530512690445 -95.9942595393627) bank47245 +47246 POINT(28.771026697830735 -94.72384492318143) bank47246 +47247 POINT(29.089485818964317 -95.3312881532895) bank47247 +47248 POINT(30.102215022078745 -95.70009967214807) bank47248 +47249 POINT(29.312748293865997 -96.31940406662959) bank47249 +47250 POINT(30.040907774764964 -94.51127043656349) bank47250 +47251 POINT(28.989234912853213 -94.61300432923935) bank47251 +47252 POINT(29.770951957641987 -95.02007888234134) bank47252 +47253 POINT(30.14226261930649 -94.93054391479374) bank47253 +47254 POINT(29.6487563262216 -96.05691017866542) bank47254 +47255 POINT(29.520044826863355 -94.58737748606515) bank47255 +47256 POINT(29.14018103085508 -95.45637397225943) bank47256 +47257 POINT(28.83817229967996 -94.65648538806222) bank47257 +47258 POINT(28.977517028545478 -94.67138144773305) bank47258 +47259 POINT(30.674713652068103 -94.60033458081763) bank47259 +47260 POINT(29.93280717976371 -95.57568318476696) bank47260 +47261 POINT(29.3871260341085 -95.32571566370402) bank47261 +47262 POINT(29.70630300392016 -95.69864707451983) bank47262 +47263 POINT(29.070319661838074 -96.05417474382322) bank47263 +47264 POINT(29.722371548399963 -95.92877606268026) bank47264 +47265 POINT(28.975517318362005 -95.05143334927325) bank47265 +47266 POINT(29.696462988804758 -95.39467371319567) bank47266 +47267 POINT(29.204682109938645 -96.21659647725446) bank47267 +47268 POINT(29.162105466982347 -95.80235839592932) bank47268 +47269 POINT(30.093683737426257 -94.53271390723212) bank47269 +47270 POINT(30.43372630957358 -95.70465180033747) bank47270 +47271 POINT(30.188248008912765 -95.58392075269359) bank47271 +47272 POINT(30.33259766155104 -94.75527346318367) bank47272 +47273 POINT(30.510178377605094 -95.6277024390801) bank47273 +47274 POINT(30.756186928939833 -94.5393242324926) bank47274 +47275 POINT(30.615450782517552 -96.19569278964742) bank47275 +47276 POINT(28.80272103093211 -94.4295457042924) bank47276 +47277 POINT(30.529245511595075 -95.44052774605683) bank47277 +47278 POINT(30.07347814877351 -94.55906650865772) bank47278 +47279 POINT(29.57642053582198 -96.03839470312668) bank47279 +47280 POINT(29.14434952118739 -96.36377168707689) bank47280 +47281 POINT(29.718080065970202 -96.25849376926386) bank47281 +47282 POINT(30.71699929774943 -94.41770170635105) bank47282 +47283 POINT(29.76587334184583 -96.35478972881741) bank47283 +47284 POINT(30.43993613916886 -95.82235452357678) bank47284 +47285 POINT(30.117861715958547 -96.15317817207941) bank47285 +47286 POINT(29.121630156417165 -95.75484883009855) bank47286 +47287 POINT(29.333347941088856 -95.6963559543219) bank47287 +47288 POINT(30.3681227150488 -96.35835388834295) bank47288 +47289 POINT(29.12285713359086 -95.53474589597604) bank47289 +47290 POINT(29.298905996487825 -94.67671853290717) bank47290 +47291 POINT(30.254301229935216 -95.33559164824854) bank47291 +47292 POINT(29.12051553144223 -95.6817526413229) bank47292 +47293 POINT(30.730423818407992 -96.36945794389787) bank47293 +47294 POINT(30.62268309580516 -95.30625348990947) bank47294 +47295 POINT(29.33512435632031 -95.34996025306141) bank47295 +47296 POINT(30.358031954679863 -95.90138805692253) bank47296 +47297 POINT(30.132145956185152 -95.98820592030552) bank47297 +47298 POINT(29.265901668290223 -95.9325759696124) bank47298 +47299 POINT(28.979728326777384 -94.43190041069079) bank47299 +47300 POINT(29.33745918087564 -94.688539681139) bank47300 +47301 POINT(29.763937176488874 -96.3336857030952) bank47301 +47302 POINT(29.400482219299505 -95.22450584476495) bank47302 +47303 POINT(28.870853837798776 -95.50473076979068) bank47303 +47304 POINT(29.23618981735782 -96.2503898166163) bank47304 +47305 POINT(29.716534970155905 -94.9542751790761) bank47305 +47306 POINT(30.514395002721674 -95.88699421926246) bank47306 +47307 POINT(29.046589144777847 -94.72734587896343) bank47307 +47308 POINT(30.07453033187781 -94.96104407908408) bank47308 +47309 POINT(30.047358844736628 -95.65959251256511) bank47309 +47310 POINT(29.55051944152791 -94.97459003632243) bank47310 +47311 POINT(30.105576545967082 -94.49362528854383) bank47311 +47312 POINT(30.59972774034506 -95.36054477681921) bank47312 +47313 POINT(29.127415128561083 -94.65063343110899) bank47313 +47314 POINT(30.357797269124063 -95.76659573771462) bank47314 +47315 POINT(29.802813035592425 -94.4033690513716) bank47315 +47316 POINT(29.34699447757353 -95.98315039901867) bank47316 +47317 POINT(29.427020642863035 -96.34711295225873) bank47317 +47318 POINT(28.816313369400213 -96.10592702059567) bank47318 +47319 POINT(29.04131442358157 -95.40775507936043) bank47319 +47320 POINT(29.738735704746137 -94.62022437857357) bank47320 +47321 POINT(28.905401519912925 -94.8315337684897) bank47321 +47322 POINT(29.284374614984714 -96.04315902980393) bank47322 +47323 POINT(28.943074985992777 -95.92666808587585) bank47323 +47324 POINT(29.40484183009418 -95.99970671777918) bank47324 +47325 POINT(29.2033869667462 -94.5269141886961) bank47325 +47326 POINT(29.88021115708517 -95.37060761269214) bank47326 +47327 POINT(29.891402675878204 -94.44202799981493) bank47327 +47328 POINT(30.381797081332085 -95.28470998171967) bank47328 +47329 POINT(29.499689074278834 -96.32890301453197) bank47329 +47330 POINT(29.239450809863328 -95.46626128755062) bank47330 +47331 POINT(30.661118324396607 -95.53904212037894) bank47331 +47332 POINT(30.69282467405652 -94.83947271539598) bank47332 +47333 POINT(29.653188004356355 -95.4014841234359) bank47333 +47334 POINT(30.263079127016717 -95.94727224915395) bank47334 +47335 POINT(30.432963519424064 -95.21312520016936) bank47335 +47336 POINT(29.432342805045934 -96.17835687487906) bank47336 +47337 POINT(29.531934774863593 -96.14686069782427) bank47337 +47338 POINT(29.921707883575152 -94.50346462619852) bank47338 +47339 POINT(29.057603152899688 -94.77305101156001) bank47339 +47340 POINT(29.563651303101146 -95.01994832133464) bank47340 +47341 POINT(30.038694485102084 -95.83727157852537) bank47341 +47342 POINT(28.947411391553548 -95.30839036056165) bank47342 +47343 POINT(30.13909784054585 -95.90089316565185) bank47343 +47344 POINT(29.10223269185574 -95.24408990992775) bank47344 +47345 POINT(29.184078835177395 -94.48697575026226) bank47345 +47346 POINT(29.051211677094933 -94.86763150926191) bank47346 +47347 POINT(29.33210350372035 -94.73068702752711) bank47347 +47348 POINT(29.526935253812837 -96.12172350804292) bank47348 +47349 POINT(29.190876296025614 -94.49663108352111) bank47349 +47350 POINT(29.674838532125793 -94.73632760501378) bank47350 +47351 POINT(30.704303401531977 -95.14624281867117) bank47351 +47352 POINT(30.080044317375524 -95.09942356504129) bank47352 +47353 POINT(29.01654238242323 -95.85335816291773) bank47353 +47354 POINT(30.44769700184556 -94.65944220635645) bank47354 +47355 POINT(29.186203965763028 -95.79588739614852) bank47355 +47356 POINT(30.31966440513781 -95.19446371073153) bank47356 +47357 POINT(29.012443325774942 -95.07938241905019) bank47357 +47358 POINT(29.35630642702648 -94.67473532179918) bank47358 +47359 POINT(30.08721145856752 -94.83087107518419) bank47359 +47360 POINT(29.60639283083893 -95.27345553869858) bank47360 +47361 POINT(30.37083876144194 -95.33770375548872) bank47361 +47362 POINT(29.277762005202877 -94.67140233086405) bank47362 +47363 POINT(30.088014228863607 -95.98784834775444) bank47363 +47364 POINT(30.385052086341474 -95.69751888791217) bank47364 +47365 POINT(29.18840234784985 -95.45859109216879) bank47365 +47366 POINT(29.706028147407814 -96.06505775578427) bank47366 +47367 POINT(30.577500551915083 -96.24811767357735) bank47367 +47368 POINT(30.009394680843858 -95.78341267257697) bank47368 +47369 POINT(29.439399503766055 -96.09483569442793) bank47369 +47370 POINT(30.006099747056826 -95.58487059797967) bank47370 +47371 POINT(30.44122292721674 -94.58449945100654) bank47371 +47372 POINT(28.821660301114225 -96.0963652808862) bank47372 +47373 POINT(29.07634469916088 -95.01106315480123) bank47373 +47374 POINT(29.005920135425278 -95.32123762953081) bank47374 +47375 POINT(30.096596931467527 -95.48199650124724) bank47375 +47376 POINT(30.026634551127508 -94.44657215205368) bank47376 +47377 POINT(30.485876499676408 -94.77103511493644) bank47377 +47378 POINT(29.437779366867 -94.80282190143114) bank47378 +47379 POINT(29.377417801516675 -95.62894656633912) bank47379 +47380 POINT(30.14838026665559 -95.89334486092677) bank47380 +47381 POINT(28.946046155852077 -95.86484938197688) bank47381 +47382 POINT(30.672234682754144 -94.55504614502995) bank47382 +47383 POINT(29.159879638171397 -96.01547016884052) bank47383 +47384 POINT(30.576088318964366 -94.94811478466752) bank47384 +47385 POINT(29.855012029313794 -95.44025120984449) bank47385 +47386 POINT(28.953063732023033 -96.29275846279337) bank47386 +47387 POINT(29.53504115685122 -94.7460307883602) bank47387 +47388 POINT(30.61404673949793 -94.85628217545957) bank47388 +47389 POINT(30.274009986190066 -95.34359652448833) bank47389 +47390 POINT(29.399977786182482 -94.51074063189775) bank47390 +47391 POINT(30.27471376361207 -95.89562342435617) bank47391 +47392 POINT(30.525072701780484 -95.74411731785443) bank47392 +47393 POINT(29.732187188980546 -96.28698186860422) bank47393 +47394 POINT(29.138323613802903 -94.48237981584188) bank47394 +47395 POINT(28.87074304813437 -95.47798514947019) bank47395 +47396 POINT(29.328395617095676 -96.08550258568496) bank47396 +47397 POINT(29.14006635227165 -94.86134866961419) bank47397 +47398 POINT(29.352302672958963 -96.30514783480422) bank47398 +47399 POINT(28.90168899409459 -96.27331040572987) bank47399 +47400 POINT(29.744123163984224 -95.24139470227908) bank47400 +47401 POINT(29.734468415280972 -95.04316973210273) bank47401 +47402 POINT(29.108128536070623 -94.82341107955169) bank47402 +47403 POINT(30.272455795201495 -95.79034497649565) bank47403 +47404 POINT(28.881987240357933 -94.73197772277251) bank47404 +47405 POINT(29.403974049313046 -94.97473390002523) bank47405 +47406 POINT(29.385576854843173 -94.37642407397435) bank47406 +47407 POINT(29.08649139707524 -94.90555374231965) bank47407 +47408 POINT(28.808672385074498 -94.97039506070092) bank47408 +47409 POINT(29.590058066267964 -94.83993518611099) bank47409 +47410 POINT(30.530199612329163 -95.20949835977603) bank47410 +47411 POINT(28.853759513138645 -96.18839676387806) bank47411 +47412 POINT(29.78241652762002 -95.62129899618546) bank47412 +47413 POINT(30.429833466621634 -95.85870430396007) bank47413 +47414 POINT(29.762943268564317 -96.26725718235372) bank47414 +47415 POINT(29.97380097809674 -96.3549099027064) bank47415 +47416 POINT(30.1400006149135 -94.72600850870819) bank47416 +47417 POINT(30.22901500028169 -94.90622786142262) bank47417 +47418 POINT(30.518008900213438 -95.59861228664894) bank47418 +47419 POINT(29.353168584726895 -94.96253178246934) bank47419 +47420 POINT(30.206460430546258 -95.64123384718285) bank47420 +47421 POINT(29.63153980123562 -95.31926840499247) bank47421 +47422 POINT(30.11078548934821 -95.03508593062017) bank47422 +47423 POINT(30.47909046914566 -95.5956900472277) bank47423 +47424 POINT(29.146937811261008 -94.73480240114341) bank47424 +47425 POINT(30.03995212843111 -95.86787790738528) bank47425 +47426 POINT(30.446753417734868 -95.03161851968461) bank47426 +47427 POINT(30.14127330939808 -95.41584237688696) bank47427 +47428 POINT(29.344882132446887 -95.17042375821299) bank47428 +47429 POINT(30.377445151318188 -95.29363378264851) bank47429 +47430 POINT(29.157002763301378 -95.4196911766481) bank47430 +47431 POINT(30.025398696999773 -95.68750443407261) bank47431 +47432 POINT(28.78104932003721 -95.04379898777884) bank47432 +47433 POINT(30.38639592501875 -96.10041608299794) bank47433 +47434 POINT(30.62743313563898 -95.79310788564572) bank47434 +47435 POINT(30.754636519242286 -95.03013722169662) bank47435 +47436 POINT(29.156543085137912 -94.51072227265772) bank47436 +47437 POINT(30.464223765233594 -95.91196826133354) bank47437 +47438 POINT(29.042659033207414 -94.52050585351962) bank47438 +47439 POINT(30.47721131843123 -95.76819468186416) bank47439 +47440 POINT(30.03992399481518 -95.40991628970316) bank47440 +47441 POINT(30.489655117431077 -95.38103910368963) bank47441 +47442 POINT(30.56413743692977 -94.60924452749936) bank47442 +47443 POINT(30.53624106013191 -95.03816481066531) bank47443 +47444 POINT(28.949218126107425 -94.886873655613) bank47444 +47445 POINT(29.225593983579884 -95.0102361195569) bank47445 +47446 POINT(30.686489133111593 -95.58425667065008) bank47446 +47447 POINT(29.51814890938957 -94.73872948240036) bank47447 +47448 POINT(30.23407029625442 -95.33255061076726) bank47448 +47449 POINT(29.616442389296225 -95.42576429749776) bank47449 +47450 POINT(30.247411260405134 -94.41686610454292) bank47450 +47451 POINT(30.11004518864906 -95.19167520178618) bank47451 +47452 POINT(30.64936643040772 -96.00251075670276) bank47452 +47453 POINT(29.34709277605151 -95.12071224033616) bank47453 +47454 POINT(28.76558062801395 -95.76766183958091) bank47454 +47455 POINT(29.15713951649331 -95.07291957979517) bank47455 +47456 POINT(30.302030742240383 -95.68768530226326) bank47456 +47457 POINT(29.226971547227112 -94.65804194730886) bank47457 +47458 POINT(29.27167547904162 -94.92554915259218) bank47458 +47459 POINT(29.20039273193684 -95.98954528907439) bank47459 +47460 POINT(29.229836706482114 -96.11540316000819) bank47460 +47461 POINT(30.67308215057252 -95.38999936798065) bank47461 +47462 POINT(30.58060101885962 -95.14576090207052) bank47462 +47463 POINT(30.327114323001247 -95.07012095298236) bank47463 +47464 POINT(30.552068404247404 -94.72167386096353) bank47464 +47465 POINT(29.33458115978991 -95.66869091401878) bank47465 +47466 POINT(30.711171363451626 -95.62157324594821) bank47466 +47467 POINT(28.981069712902247 -94.99390204226577) bank47467 +47468 POINT(29.148965850121883 -95.39748909746012) bank47468 +47469 POINT(29.551680662940733 -94.6214938370791) bank47469 +47470 POINT(30.15461156950248 -94.41101398676334) bank47470 +47471 POINT(30.41160853016766 -96.02915546165002) bank47471 +47472 POINT(29.057340027912886 -95.38767504771366) bank47472 +47473 POINT(30.35811516306739 -94.6809609316767) bank47473 +47474 POINT(30.168142676729403 -95.58757890829203) bank47474 +47475 POINT(29.393998892395885 -95.25737776418703) bank47475 +47476 POINT(30.41402304093787 -94.97418392502878) bank47476 +47477 POINT(29.069813420722184 -95.19713676393512) bank47477 +47478 POINT(29.879525839605407 -96.1745497129414) bank47478 +47479 POINT(30.52866896070569 -95.15626350713843) bank47479 +47480 POINT(28.82578775946601 -96.05871005632216) bank47480 +47481 POINT(30.560936629017547 -95.42567047735321) bank47481 +47482 POINT(29.03715215907652 -94.8381525441503) bank47482 +47483 POINT(30.2384547546973 -95.96324319121938) bank47483 +47484 POINT(28.86182921823298 -95.79102670531228) bank47484 +47485 POINT(28.778442114789442 -95.32460683439028) bank47485 +47486 POINT(30.459941730415085 -94.63624213274346) bank47486 +47487 POINT(30.391955760600986 -96.31697299092608) bank47487 +47488 POINT(29.989282068160104 -95.37536858176443) bank47488 +47489 POINT(28.794442526054855 -94.67582239053597) bank47489 +47490 POINT(28.849055388244583 -94.80726963226262) bank47490 +47491 POINT(30.071629026366654 -95.60833979548798) bank47491 +47492 POINT(30.415661438467914 -95.72222533797604) bank47492 +47493 POINT(30.69898526498031 -96.35999218014601) bank47493 +47494 POINT(29.90802741892942 -95.4988831080546) bank47494 +47495 POINT(29.60683664871836 -96.3328217834505) bank47495 +47496 POINT(29.872275848218717 -95.25190355845817) bank47496 +47497 POINT(28.956838038618685 -94.76192584814463) bank47497 +47498 POINT(28.99206731166004 -95.77056041742945) bank47498 +47499 POINT(29.53478952918754 -94.51721186811751) bank47499 +47500 POINT(29.607826145192384 -95.72295997892765) bank47500 +47501 POINT(29.078008386873577 -95.92713891520818) bank47501 +47502 POINT(30.190969853957355 -95.38147743960992) bank47502 +47503 POINT(28.91317739581695 -95.62783271804386) bank47503 +47504 POINT(29.579300538244215 -95.43020682165769) bank47504 +47505 POINT(29.052660800212628 -94.57759627729857) bank47505 +47506 POINT(28.82676474443286 -95.04899997875702) bank47506 +47507 POINT(30.16211646949588 -95.65230192061867) bank47507 +47508 POINT(30.446399178498954 -96.35375392077526) bank47508 +47509 POINT(29.104204034774114 -94.97498764414266) bank47509 +47510 POINT(28.772425752915385 -95.7306353442528) bank47510 +47511 POINT(30.407963890957227 -95.52030449185202) bank47511 +47512 POINT(28.795869765792773 -95.65768169027227) bank47512 +47513 POINT(29.28333215525181 -94.79727123450753) bank47513 +47514 POINT(30.095271495857336 -95.64678028063454) bank47514 +47515 POINT(29.45804251957806 -95.29905065208814) bank47515 +47516 POINT(29.601929299459677 -95.09655432463872) bank47516 +47517 POINT(29.111057800432032 -95.54837652921387) bank47517 +47518 POINT(30.225728543584374 -96.1151823807829) bank47518 +47519 POINT(29.228283690250002 -94.84297652258867) bank47519 +47520 POINT(29.963970615650464 -95.22599450434406) bank47520 +47521 POINT(30.226515841816482 -96.11307392397015) bank47521 +47522 POINT(29.604244465904785 -94.5733792631322) bank47522 +47523 POINT(30.63384880931196 -94.65354078233193) bank47523 +47524 POINT(30.193432135028345 -94.37540211691396) bank47524 +47525 POINT(29.49218173282756 -95.9726994145491) bank47525 +47526 POINT(30.15785940677629 -95.98723686435979) bank47526 +47527 POINT(28.791977257664954 -95.23491902602956) bank47527 +47528 POINT(29.748175468197058 -95.62938981002497) bank47528 +47529 POINT(30.305884807175474 -96.00912719647668) bank47529 +47530 POINT(30.325920035201875 -96.15858143430312) bank47530 +47531 POINT(29.909756921814783 -96.18742278571813) bank47531 +47532 POINT(29.69680778639806 -96.18115070142157) bank47532 +47533 POINT(30.657131595466776 -95.6401288953471) bank47533 +47534 POINT(29.021387995797237 -95.74870729492115) bank47534 +47535 POINT(29.60875901320298 -96.1413530007811) bank47535 +47536 POINT(30.308691142850044 -94.86027699035314) bank47536 +47537 POINT(29.710034506190738 -95.63327365789283) bank47537 +47538 POINT(30.387629317000957 -94.45741898662367) bank47538 +47539 POINT(30.268280187074314 -96.26801479508065) bank47539 +47540 POINT(29.50039346000549 -95.425049101778) bank47540 +47541 POINT(29.254062346437955 -95.28702131627072) bank47541 +47542 POINT(30.355004918034606 -95.43591398325327) bank47542 +47543 POINT(29.548085856282952 -95.37716058457866) bank47543 +47544 POINT(30.409231621308827 -95.37630037411309) bank47544 +47545 POINT(28.76993212535344 -95.31291390541453) bank47545 +47546 POINT(30.010534626685402 -95.0897021192886) bank47546 +47547 POINT(30.46643428348518 -95.98540546048424) bank47547 +47548 POINT(29.257526151216346 -94.72746036609755) bank47548 +47549 POINT(29.861676430433825 -94.61733257995121) bank47549 +47550 POINT(30.19757913557954 -94.75143201172622) bank47550 +47551 POINT(30.559640366000934 -95.94695368812916) bank47551 +47552 POINT(30.59692068258576 -95.52841193906256) bank47552 +47553 POINT(29.340901547642428 -95.02400759439142) bank47553 +47554 POINT(30.07624509215876 -95.0950774583274) bank47554 +47555 POINT(29.893794490474974 -95.19995831308148) bank47555 +47556 POINT(28.77407359142771 -94.48536884454964) bank47556 +47557 POINT(29.69667381337424 -96.3448096983482) bank47557 +47558 POINT(29.568355769459714 -94.600167555952) bank47558 +47559 POINT(28.771980769954695 -94.9191265663472) bank47559 +47560 POINT(30.508994549978546 -95.23993123594914) bank47560 +47561 POINT(29.256235228442566 -95.94103696292463) bank47561 +47562 POINT(30.628764496897873 -94.94001379327646) bank47562 +47563 POINT(28.880885567952006 -94.86666381864126) bank47563 +47564 POINT(30.04840279215154 -94.48877951047469) bank47564 +47565 POINT(30.24696195202245 -95.59466215739218) bank47565 +47566 POINT(30.63522010145764 -96.31251129823633) bank47566 +47567 POINT(28.87641096412429 -94.48333001732486) bank47567 +47568 POINT(30.62781514872049 -94.62005556518203) bank47568 +47569 POINT(29.615751452680023 -95.28742818745069) bank47569 +47570 POINT(30.61350410592512 -95.11900848461671) bank47570 +47571 POINT(29.58392981351181 -95.63236822055283) bank47571 +47572 POINT(29.78966032536146 -95.45461438117829) bank47572 +47573 POINT(30.520445166347603 -94.84935374316235) bank47573 +47574 POINT(29.39678389909919 -94.64064236330542) bank47574 +47575 POINT(28.913929335867692 -94.70215465506325) bank47575 +47576 POINT(29.87561966150415 -94.5054834340947) bank47576 +47577 POINT(28.968304530337136 -95.41390150112349) bank47577 +47578 POINT(30.159857779086067 -95.38914765171654) bank47578 +47579 POINT(29.989865260067837 -94.46553533857495) bank47579 +47580 POINT(29.788251289109745 -95.67889205096353) bank47580 +47581 POINT(30.032099711132087 -94.94534578090817) bank47581 +47582 POINT(29.096019645300615 -95.01525027627606) bank47582 +47583 POINT(29.528099068626286 -95.78752873547944) bank47583 +47584 POINT(28.847995850991854 -94.60558772137563) bank47584 +47585 POINT(28.963203837253165 -96.11632514761558) bank47585 +47586 POINT(30.00787322729307 -94.77637045969617) bank47586 +47587 POINT(30.739110241823102 -95.20284396021229) bank47587 +47588 POINT(29.463260854079028 -95.57570977484728) bank47588 +47589 POINT(29.23292413992189 -94.47051317006635) bank47589 +47590 POINT(29.001785085773662 -96.01630372003052) bank47590 +47591 POINT(29.412040663290348 -94.67708778488851) bank47591 +47592 POINT(29.223787958841072 -95.74456184047015) bank47592 +47593 POINT(30.23776809173992 -95.19640923491963) bank47593 +47594 POINT(28.903446927165273 -95.22466055193982) bank47594 +47595 POINT(30.229101854701174 -96.23094929892272) bank47595 +47596 POINT(30.698532087351044 -94.83209651770754) bank47596 +47597 POINT(28.929736239843606 -94.52100847140345) bank47597 +47598 POINT(30.064407097957798 -96.0620706108868) bank47598 +47599 POINT(30.342430066957146 -95.5338013533293) bank47599 +47600 POINT(30.22235938711336 -96.2745301196203) bank47600 +47601 POINT(29.599740432145122 -95.4640229640548) bank47601 +47602 POINT(30.065012010669935 -95.54216304978753) bank47602 +47603 POINT(29.173556428535765 -94.57793254501141) bank47603 +47604 POINT(29.88156318030394 -94.85762436626293) bank47604 +47605 POINT(29.34639251308153 -96.04569687098822) bank47605 +47606 POINT(28.962202594293487 -96.13994680778382) bank47606 +47607 POINT(30.72105883490516 -94.90903828327046) bank47607 +47608 POINT(30.07265289616789 -96.09669723459758) bank47608 +47609 POINT(28.960305019909978 -94.83924492581562) bank47609 +47610 POINT(30.0685285705271 -94.69349567729293) bank47610 +47611 POINT(30.099008170395226 -94.57516399572785) bank47611 +47612 POINT(30.398572750066794 -96.35463995001967) bank47612 +47613 POINT(29.568878783475316 -95.04865419498881) bank47613 +47614 POINT(30.272155746328018 -94.70497911571265) bank47614 +47615 POINT(29.88409113785005 -94.65427559971131) bank47615 +47616 POINT(29.0830083329353 -95.46851485345344) bank47616 +47617 POINT(29.04995590744987 -96.05860226238016) bank47617 +47618 POINT(29.675759359505477 -95.95604581189619) bank47618 +47619 POINT(29.326513057269675 -96.07977678605256) bank47619 +47620 POINT(29.350154924732223 -95.77208736567202) bank47620 +47621 POINT(29.494705625643437 -95.2483494538044) bank47621 +47622 POINT(29.726650644617223 -95.25179411836366) bank47622 +47623 POINT(29.88248391144912 -94.71164277719119) bank47623 +47624 POINT(30.148622234903517 -95.40274451940088) bank47624 +47625 POINT(28.79197666198526 -95.83048936648662) bank47625 +47626 POINT(29.353646641481745 -95.96024636500479) bank47626 +47627 POINT(29.61585257563211 -96.05574815830312) bank47627 +47628 POINT(30.43428621762801 -94.72573284402725) bank47628 +47629 POINT(29.844395365692325 -95.57060048778992) bank47629 +47630 POINT(30.603939338541533 -94.63253373941286) bank47630 +47631 POINT(30.069413487726006 -95.86292479596149) bank47631 +47632 POINT(29.893065930792037 -94.95125416211935) bank47632 +47633 POINT(29.39639191668377 -95.691237946482) bank47633 +47634 POINT(30.17018083371104 -94.93540341105647) bank47634 +47635 POINT(29.34999428218901 -94.38609031151162) bank47635 +47636 POINT(28.86371579035595 -95.92755538098274) bank47636 +47637 POINT(30.415056697811785 -94.8364444910017) bank47637 +47638 POINT(30.523189696504232 -94.4963760070885) bank47638 +47639 POINT(30.579486226157375 -95.95318977538228) bank47639 +47640 POINT(29.682223259878842 -95.87766759303223) bank47640 +47641 POINT(29.345191042475236 -95.00595469468246) bank47641 +47642 POINT(29.50056082808072 -95.35897831030597) bank47642 +47643 POINT(29.606760902342007 -94.8110358962817) bank47643 +47644 POINT(30.58760585663602 -95.93712386358624) bank47644 +47645 POINT(28.89759190312498 -95.82203883328737) bank47645 +47646 POINT(30.608768863201014 -94.5007054373921) bank47646 +47647 POINT(29.15284761497709 -95.0757330356097) bank47647 +47648 POINT(29.837267502193704 -94.63610804741289) bank47648 +47649 POINT(28.973223192270247 -96.3344025926397) bank47649 +47650 POINT(29.818186657747127 -94.4988018491301) bank47650 +47651 POINT(29.551078560355847 -95.37532030085303) bank47651 +47652 POINT(29.37209100462542 -95.8092963299502) bank47652 +47653 POINT(29.30921837956445 -94.6108852115625) bank47653 +47654 POINT(30.137299290068217 -95.87522836309911) bank47654 +47655 POINT(29.18500148052473 -95.26198758634749) bank47655 +47656 POINT(29.211284843463112 -96.2024894867501) bank47656 +47657 POINT(29.880079132092753 -96.3416391101726) bank47657 +47658 POINT(29.427230018303362 -96.03967799553675) bank47658 +47659 POINT(30.097162649500536 -95.33207183827066) bank47659 +47660 POINT(30.418267991624614 -95.28746692160622) bank47660 +47661 POINT(29.815524447948583 -96.22314624485796) bank47661 +47662 POINT(29.612166773932387 -95.365066905085) bank47662 +47663 POINT(29.196498346565498 -95.7280678194005) bank47663 +47664 POINT(30.603838684471718 -94.77390429865942) bank47664 +47665 POINT(29.456899818136876 -96.22119675474597) bank47665 +47666 POINT(30.537938318797373 -96.1957688332557) bank47666 +47667 POINT(29.70807413338568 -95.82509155224277) bank47667 +47668 POINT(29.803460724296862 -95.43397061303365) bank47668 +47669 POINT(29.23960179538593 -95.77708183030433) bank47669 +47670 POINT(28.810694628800515 -95.84687518061627) bank47670 +47671 POINT(29.961639367097472 -95.62327877297349) bank47671 +47672 POINT(29.776493519970266 -94.78846452581156) bank47672 +47673 POINT(29.934941358196365 -94.82975205814192) bank47673 +47674 POINT(29.89596090067384 -95.67509425325775) bank47674 +47675 POINT(29.725710591775734 -94.87063301067134) bank47675 +47676 POINT(30.107762605342646 -96.02722736810794) bank47676 +47677 POINT(30.28634967725986 -94.39916273049629) bank47677 +47678 POINT(28.96968991374281 -96.17564109834542) bank47678 +47679 POINT(29.016225234073342 -95.41780004396196) bank47679 +47680 POINT(30.03957228765502 -96.04895552770812) bank47680 +47681 POINT(28.849241114762584 -94.54432336355995) bank47681 +47682 POINT(30.280748654640856 -94.79395577942596) bank47682 +47683 POINT(30.030183031837957 -96.10410141140713) bank47683 +47684 POINT(29.819986251881556 -95.4265072386813) bank47684 +47685 POINT(29.39453992795353 -96.03150070448548) bank47685 +47686 POINT(29.812361359364385 -95.54324806994133) bank47686 +47687 POINT(29.53427008921599 -96.18687336746858) bank47687 +47688 POINT(29.398765121516043 -95.18895414160028) bank47688 +47689 POINT(29.478631963764865 -94.51628920106143) bank47689 +47690 POINT(29.133458883803645 -95.99166428554337) bank47690 +47691 POINT(29.258276077232612 -96.03947611980657) bank47691 +47692 POINT(30.325423031275136 -94.4978025993801) bank47692 +47693 POINT(30.005645940408964 -95.22992749727167) bank47693 +47694 POINT(30.663854299475982 -95.14774566460866) bank47694 +47695 POINT(30.074799325567508 -95.32618506625457) bank47695 +47696 POINT(28.992551433647986 -94.60198302021753) bank47696 +47697 POINT(28.888637191367273 -94.70560904874715) bank47697 +47698 POINT(29.33407235984641 -94.81257647844768) bank47698 +47699 POINT(28.846718663603028 -94.57861894695819) bank47699 +47700 POINT(29.273566136887652 -96.14804719052559) bank47700 +47701 POINT(30.15092447971583 -95.46644468063441) bank47701 +47702 POINT(29.557267445912434 -95.0783099109276) bank47702 +47703 POINT(29.47113362142626 -95.2287873545473) bank47703 +47704 POINT(30.691292180099488 -95.68183242684587) bank47704 +47705 POINT(29.735201102210937 -95.06625550901276) bank47705 +47706 POINT(30.095773219852706 -94.81087772965643) bank47706 +47707 POINT(29.490614970978804 -96.16410308926011) bank47707 +47708 POINT(29.130034505067893 -94.79066699195366) bank47708 +47709 POINT(29.96814977905028 -95.20822082529338) bank47709 +47710 POINT(29.790493702138082 -94.7920225809675) bank47710 +47711 POINT(30.13730152504787 -95.57721517248137) bank47711 +47712 POINT(30.3116432975289 -95.02695249241977) bank47712 +47713 POINT(29.58558998664558 -95.98973282548396) bank47713 +47714 POINT(29.32732063275032 -95.6518630294088) bank47714 +47715 POINT(29.415227738257975 -95.7253247262008) bank47715 +47716 POINT(29.30830809715072 -96.257842230988) bank47716 +47717 POINT(28.848900203648874 -94.79234095771821) bank47717 +47718 POINT(29.732397476292082 -96.15419167132958) bank47718 +47719 POINT(29.72570768413859 -95.00179951338112) bank47719 +47720 POINT(28.773320477731662 -94.54063504750803) bank47720 +47721 POINT(29.697111745228387 -95.15044411032878) bank47721 +47722 POINT(30.680827329635832 -95.69687439661686) bank47722 +47723 POINT(29.879133563187576 -94.98899187105145) bank47723 +47724 POINT(30.438975630946498 -95.42274573263829) bank47724 +47725 POINT(29.28572153237917 -96.11937042293069) bank47725 +47726 POINT(29.139833308090807 -95.41936549683213) bank47726 +47727 POINT(29.88420047561406 -94.67016080930907) bank47727 +47728 POINT(30.349350001221666 -94.61504993402313) bank47728 +47729 POINT(29.752914861691398 -94.4515485260794) bank47729 +47730 POINT(29.54695315699307 -94.40977853066194) bank47730 +47731 POINT(30.172620780030215 -95.51889928402741) bank47731 +47732 POINT(30.574065407871924 -95.758082716303) bank47732 +47733 POINT(30.46669047298591 -96.13711807761275) bank47733 +47734 POINT(29.379372627922105 -95.53136990601058) bank47734 +47735 POINT(29.41788122089441 -94.64362782935066) bank47735 +47736 POINT(30.71605475450156 -95.9168095119866) bank47736 +47737 POINT(29.167158351656145 -95.37352663278209) bank47737 +47738 POINT(30.207405685172695 -94.8505271997997) bank47738 +47739 POINT(28.81194174569109 -94.65566733669222) bank47739 +47740 POINT(30.074575785520782 -94.91478273828822) bank47740 +47741 POINT(28.801902696281328 -95.7786066277489) bank47741 +47742 POINT(28.762841273428176 -96.33613166467991) bank47742 +47743 POINT(30.214282673560277 -94.69485828887646) bank47743 +47744 POINT(29.598636813271476 -95.39945964076634) bank47744 +47745 POINT(28.893116279928186 -95.26348359202346) bank47745 +47746 POINT(29.06789875604609 -94.38111891210504) bank47746 +47747 POINT(29.98845660048239 -95.84882011150545) bank47747 +47748 POINT(29.057639961672077 -94.6158065920919) bank47748 +47749 POINT(29.476014705438256 -96.20014694441687) bank47749 +47750 POINT(28.865305249289705 -95.87785897943866) bank47750 +47751 POINT(29.867437117731917 -95.22742560961069) bank47751 +47752 POINT(29.537946026617153 -94.75467764479725) bank47752 +47753 POINT(30.396230698915375 -95.42197207481601) bank47753 +47754 POINT(30.71791894279438 -95.95703895966997) bank47754 +47755 POINT(30.24684880508167 -94.55048139801333) bank47755 +47756 POINT(29.856513639192386 -96.31544692292195) bank47756 +47757 POINT(29.886189614224886 -94.99997399410275) bank47757 +47758 POINT(30.636152257063486 -96.20889861068062) bank47758 +47759 POINT(29.900728057598556 -96.14278969722986) bank47759 +47760 POINT(29.96644191746844 -95.62796097789216) bank47760 +47761 POINT(29.90078807731337 -96.06617307136737) bank47761 +47762 POINT(29.054696325112367 -96.06612511966732) bank47762 +47763 POINT(30.502426892129 -94.65678279820273) bank47763 +47764 POINT(30.432956237031117 -95.23550801705416) bank47764 +47765 POINT(29.052083092223725 -96.23432996315488) bank47765 +47766 POINT(30.216477787541088 -94.6201958115) bank47766 +47767 POINT(29.47267722726737 -95.59151521617363) bank47767 +47768 POINT(30.23150144583128 -94.45946549697383) bank47768 +47769 POINT(30.41831859660957 -95.12373929770126) bank47769 +47770 POINT(30.714685367082463 -96.01193833063994) bank47770 +47771 POINT(29.73455520266259 -96.00663237228315) bank47771 +47772 POINT(30.221203777652917 -94.6052233066649) bank47772 +47773 POINT(29.83228512190628 -95.56968285217579) bank47773 +47774 POINT(30.164640152988003 -95.5836557175352) bank47774 +47775 POINT(29.962655739308484 -95.43704124472124) bank47775 +47776 POINT(30.156585191426657 -96.2921816820331) bank47776 +47777 POINT(29.658110622223724 -95.29416423023363) bank47777 +47778 POINT(30.059806500696347 -95.41565988265516) bank47778 +47779 POINT(29.853327592519314 -94.65748612044834) bank47779 +47780 POINT(29.91046108124772 -95.58915184704702) bank47780 +47781 POINT(29.038591703107016 -94.8724199127929) bank47781 +47782 POINT(30.057781426719938 -95.52815286415505) bank47782 +47783 POINT(29.667226510788108 -94.84615098926724) bank47783 +47784 POINT(29.178321614486048 -94.62129403647937) bank47784 +47785 POINT(29.176337407409378 -95.0710350956986) bank47785 +47786 POINT(29.64562988623599 -95.82134436988129) bank47786 +47787 POINT(28.81521319205155 -95.43192327095797) bank47787 +47788 POINT(30.483836345578077 -96.0294028648207) bank47788 +47789 POINT(30.59157786274291 -96.09621776390995) bank47789 +47790 POINT(28.789761453452595 -94.93163011655004) bank47790 +47791 POINT(29.71610294106798 -94.89804227645416) bank47791 +47792 POINT(28.76987686196317 -94.67168810013806) bank47792 +47793 POINT(30.53339542401429 -95.99793788264208) bank47793 +47794 POINT(30.65602526232047 -95.82792008281614) bank47794 +47795 POINT(30.40439484557863 -94.66842370082814) bank47795 +47796 POINT(30.181315546541317 -94.92847388416529) bank47796 +47797 POINT(29.950668624639633 -94.40146506846351) bank47797 +47798 POINT(29.109942848815727 -95.7240168436136) bank47798 +47799 POINT(30.05138643650786 -95.3236609876785) bank47799 +47800 POINT(30.73750330729522 -95.81666018759007) bank47800 +47801 POINT(28.997681641146023 -94.63521543913335) bank47801 +47802 POINT(29.489600055552202 -95.75192391743961) bank47802 +47803 POINT(30.29830911119345 -95.29714539221104) bank47803 +47804 POINT(30.55959574586911 -96.02485068098127) bank47804 +47805 POINT(30.74717455285095 -94.73599901237975) bank47805 +47806 POINT(29.23116975430897 -95.78388433726936) bank47806 +47807 POINT(28.801554949440906 -95.38271881407931) bank47807 +47808 POINT(30.67823933361895 -96.31075832089056) bank47808 +47809 POINT(29.087881438626034 -95.57739156758274) bank47809 +47810 POINT(30.261800977106745 -96.1710974374102) bank47810 +47811 POINT(29.115280065951072 -94.48656135525353) bank47811 +47812 POINT(29.568699579598732 -95.79109888475884) bank47812 +47813 POINT(29.54247703112582 -95.15284471810465) bank47813 +47814 POINT(29.4944900655131 -94.77779968034218) bank47814 +47815 POINT(29.274348498022025 -95.0108595144827) bank47815 +47816 POINT(30.383529566822624 -96.20794558263714) bank47816 +47817 POINT(28.95511595346106 -95.76142706649827) bank47817 +47818 POINT(29.226158967098634 -95.51858037461997) bank47818 +47819 POINT(30.158447048182367 -95.99314306079363) bank47819 +47820 POINT(30.21519363246585 -94.43129383161019) bank47820 +47821 POINT(30.400780515665815 -95.48674541419346) bank47821 +47822 POINT(30.210620627109357 -95.84699875915294) bank47822 +47823 POINT(30.4981303040156 -94.90958693059993) bank47823 +47824 POINT(30.361748197182813 -95.88929504780793) bank47824 +47825 POINT(29.977433213300333 -96.10027399460995) bank47825 +47826 POINT(30.493167669366944 -95.48175149921197) bank47826 +47827 POINT(29.49632527573296 -95.64492938976801) bank47827 +47828 POINT(29.28297315660166 -95.67960840967712) bank47828 +47829 POINT(29.81833188177928 -96.05231959641453) bank47829 +47830 POINT(30.55101995282653 -94.69527793534341) bank47830 +47831 POINT(30.383448035928758 -95.31721912549192) bank47831 +47832 POINT(30.707349694191866 -95.02065919248953) bank47832 +47833 POINT(30.701140537142223 -94.95231637417074) bank47833 +47834 POINT(29.368590030222975 -96.2022674201695) bank47834 +47835 POINT(30.10313536656446 -95.98190361924551) bank47835 +47836 POINT(30.23538342688037 -95.5914233423751) bank47836 +47837 POINT(29.750312012607093 -95.55450557914716) bank47837 +47838 POINT(30.218654681471552 -95.30310619529418) bank47838 +47839 POINT(30.522533599798845 -95.95722255042955) bank47839 +47840 POINT(28.942665605236694 -95.25886315113819) bank47840 +47841 POINT(30.685645104144694 -95.30732754982049) bank47841 +47842 POINT(30.407636722347565 -95.57101147063037) bank47842 +47843 POINT(29.098882647075616 -95.42400313562719) bank47843 +47844 POINT(29.49097200349642 -95.36896425275344) bank47844 +47845 POINT(29.764162976895793 -95.45940444636847) bank47845 +47846 POINT(29.849268274734747 -95.79658552349547) bank47846 +47847 POINT(29.708770846141707 -95.22487135315991) bank47847 +47848 POINT(29.122236119838703 -95.21954450470213) bank47848 +47849 POINT(29.862738762928128 -94.53911261633684) bank47849 +47850 POINT(30.641947323268372 -95.6791048924013) bank47850 +47851 POINT(29.11197476408569 -95.52804086347443) bank47851 +47852 POINT(30.56704451065764 -94.4605803122316) bank47852 +47853 POINT(29.246927278267584 -95.21239312657751) bank47853 +47854 POINT(30.733101665689308 -96.13541495276982) bank47854 +47855 POINT(29.888425301142842 -95.51045433821453) bank47855 +47856 POINT(30.72558325882034 -95.72482015256975) bank47856 +47857 POINT(29.904165209580462 -95.54296384517573) bank47857 +47858 POINT(30.078778646136502 -95.94616209711829) bank47858 +47859 POINT(30.31893202291275 -95.53503854971078) bank47859 +47860 POINT(28.841834913332942 -96.36774255255634) bank47860 +47861 POINT(30.304329421995448 -95.76501095276191) bank47861 +47862 POINT(28.906312623561398 -94.42122083367742) bank47862 +47863 POINT(30.1765826600815 -95.92742291810494) bank47863 +47864 POINT(29.539414821864927 -95.18504459730266) bank47864 +47865 POINT(30.73755673451368 -95.55963933475515) bank47865 +47866 POINT(29.868913684649755 -95.72897553886942) bank47866 +47867 POINT(29.639539600066144 -94.80960528819315) bank47867 +47868 POINT(29.133334790827117 -94.49278407592966) bank47868 +47869 POINT(30.143443192654633 -95.33847528261653) bank47869 +47870 POINT(28.89640959156657 -96.22351768888201) bank47870 +47871 POINT(30.68711457899785 -94.98437448834962) bank47871 +47872 POINT(30.084111510334452 -95.51049333063885) bank47872 +47873 POINT(29.921768591530338 -96.31327886809822) bank47873 +47874 POINT(29.623082180101306 -94.89506391042917) bank47874 +47875 POINT(28.84936109840165 -96.27954202031322) bank47875 +47876 POINT(30.55443421496617 -95.35624041548829) bank47876 +47877 POINT(30.26258343562428 -95.84412726051393) bank47877 +47878 POINT(29.450946334836864 -94.46957319971048) bank47878 +47879 POINT(29.55734219692897 -94.88188902489533) bank47879 +47880 POINT(29.021266460792695 -96.0882866563484) bank47880 +47881 POINT(29.893096309967383 -94.81798266066352) bank47881 +47882 POINT(29.662571815834834 -96.24510938909415) bank47882 +47883 POINT(28.985521195076707 -94.87370239265258) bank47883 +47884 POINT(30.045034577414054 -95.15978016514319) bank47884 +47885 POINT(29.616743132936765 -95.2297144819552) bank47885 +47886 POINT(28.852314732312326 -94.50487260525706) bank47886 +47887 POINT(28.984839361588513 -95.07136654315089) bank47887 +47888 POINT(28.92316153356692 -95.22076194872065) bank47888 +47889 POINT(29.212640287684806 -96.0855427134383) bank47889 +47890 POINT(30.406070606495682 -94.49364520374816) bank47890 +47891 POINT(29.035270767824763 -95.44833738735848) bank47891 +47892 POINT(30.18802879237737 -95.43608759245095) bank47892 +47893 POINT(30.344009851851464 -95.05901357873812) bank47893 +47894 POINT(29.334783292411718 -94.41883203836655) bank47894 +47895 POINT(29.14585879782824 -95.40578103778212) bank47895 +47896 POINT(29.359060306957577 -94.78741482824502) bank47896 +47897 POINT(30.269146587938362 -94.67124907730086) bank47897 +47898 POINT(28.780853193098583 -95.65309545995977) bank47898 +47899 POINT(29.00079956278164 -95.66670647137677) bank47899 +47900 POINT(28.828454195332643 -95.82822262438124) bank47900 +47901 POINT(29.609007388938068 -94.42183534156159) bank47901 +47902 POINT(29.380713356682396 -94.78358361493058) bank47902 +47903 POINT(29.79769769362721 -94.47836872856497) bank47903 +47904 POINT(29.264763315280653 -95.87542350344839) bank47904 +47905 POINT(29.199986097174186 -96.05493978157494) bank47905 +47906 POINT(29.8916890764467 -95.62181718476585) bank47906 +47907 POINT(29.992798341972808 -95.90368079721483) bank47907 +47908 POINT(29.07093184852485 -94.6889444306084) bank47908 +47909 POINT(29.412058124110903 -95.93271880600832) bank47909 +47910 POINT(30.04621092888653 -95.89243709015395) bank47910 +47911 POINT(29.96680358314607 -95.99103682314049) bank47911 +47912 POINT(28.882291161426956 -95.51246067124697) bank47912 +47913 POINT(30.015809582290053 -95.6990247076264) bank47913 +47914 POINT(29.968846633884645 -95.27690324832135) bank47914 +47915 POINT(29.681474168174972 -94.76859253361796) bank47915 +47916 POINT(29.40534320341862 -95.33509020293693) bank47916 +47917 POINT(29.607753289476378 -94.4294796432615) bank47917 +47918 POINT(29.80879527036482 -95.35189773052223) bank47918 +47919 POINT(30.703849178541088 -96.27046924539295) bank47919 +47920 POINT(30.680196486030955 -94.89163149793012) bank47920 +47921 POINT(28.860853296724684 -95.4188096698141) bank47921 +47922 POINT(30.262669023948387 -94.58234220740907) bank47922 +47923 POINT(29.831794094565165 -95.16410977963967) bank47923 +47924 POINT(30.64303799368621 -96.11954218347937) bank47924 +47925 POINT(29.27698780341786 -95.89491529122303) bank47925 +47926 POINT(30.002110694224207 -94.65276804001819) bank47926 +47927 POINT(29.64217533708611 -95.05388703534845) bank47927 +47928 POINT(28.9424560824896 -95.03772653393114) bank47928 +47929 POINT(30.2697108293802 -95.49151651674393) bank47929 +47930 POINT(28.774486677744104 -95.54322639415221) bank47930 +47931 POINT(28.79058173686389 -94.64061198917241) bank47931 +47932 POINT(29.534209880329428 -95.12841177210485) bank47932 +47933 POINT(28.97721175762417 -96.1585942098109) bank47933 +47934 POINT(29.973391776639914 -95.13045116412357) bank47934 +47935 POINT(30.5535622908485 -95.06794161207937) bank47935 +47936 POINT(29.81303079344725 -95.82607824386173) bank47936 +47937 POINT(29.33856052170685 -96.17680216429254) bank47937 +47938 POINT(30.549655154662684 -94.66600467948001) bank47938 +47939 POINT(29.28091928006091 -94.89807544083577) bank47939 +47940 POINT(30.51465047175886 -95.72788984496916) bank47940 +47941 POINT(30.222976823238607 -94.41732773911905) bank47941 +47942 POINT(30.676819323650058 -95.06637465103964) bank47942 +47943 POINT(29.170128411186077 -95.83189198011534) bank47943 +47944 POINT(29.224050161328314 -95.86298090108363) bank47944 +47945 POINT(28.7952310980103 -95.69305158265202) bank47945 +47946 POINT(30.286614569430718 -95.97144725552457) bank47946 +47947 POINT(28.890302496833684 -95.16398770826697) bank47947 +47948 POINT(30.274453979492506 -94.58032636247371) bank47948 +47949 POINT(29.617597957189666 -94.72684457550656) bank47949 +47950 POINT(28.916263060522642 -94.88511551078362) bank47950 +47951 POINT(29.037533728277676 -94.7478732367029) bank47951 +47952 POINT(29.45827002628299 -94.79994355304818) bank47952 +47953 POINT(30.11032965811724 -94.5414035698038) bank47953 +47954 POINT(29.523192540435794 -94.88021373719558) bank47954 +47955 POINT(29.968818805975403 -94.84507279180717) bank47955 +47956 POINT(28.929788235628887 -94.3828336397942) bank47956 +47957 POINT(29.145799531761536 -95.31880987015515) bank47957 +47958 POINT(29.980978977714262 -94.59615252108877) bank47958 +47959 POINT(30.692599276389178 -95.51874510255846) bank47959 +47960 POINT(30.314404565170705 -95.13336757842586) bank47960 +47961 POINT(30.64054824595399 -95.96608948926705) bank47961 +47962 POINT(29.927671137333256 -95.51507189753602) bank47962 +47963 POINT(30.30403224377775 -95.85343951896368) bank47963 +47964 POINT(29.376955917693774 -95.22336150422657) bank47964 +47965 POINT(29.0169030693705 -94.55408843668704) bank47965 +47966 POINT(29.113058359142336 -95.74518621975339) bank47966 +47967 POINT(30.636669820848404 -94.94975403366026) bank47967 +47968 POINT(29.451303820128157 -95.13181455055192) bank47968 +47969 POINT(30.41682550892153 -95.31527697118648) bank47969 +47970 POINT(30.48732367339176 -95.49656887340083) bank47970 +47971 POINT(29.631181769694038 -94.8879993031594) bank47971 +47972 POINT(29.545501931223946 -96.20005302235411) bank47972 +47973 POINT(28.80860518431876 -95.97587058388073) bank47973 +47974 POINT(29.86478730090115 -95.25385052137243) bank47974 +47975 POINT(29.68282329348077 -94.74852839327838) bank47975 +47976 POINT(30.489154443311218 -96.2586500139891) bank47976 +47977 POINT(29.60093218821268 -95.16961278918288) bank47977 +47978 POINT(29.05840657947332 -94.91762442252373) bank47978 +47979 POINT(28.826013604690672 -96.0326283596116) bank47979 +47980 POINT(29.443582672887977 -95.6274905547045) bank47980 +47981 POINT(29.236071427114865 -96.10864374800452) bank47981 +47982 POINT(29.523267004874096 -94.74808679369562) bank47982 +47983 POINT(30.321290012831593 -96.1700205262366) bank47983 +47984 POINT(29.676392193824167 -96.00832852808881) bank47984 +47985 POINT(29.119970888620784 -95.01202738100194) bank47985 +47986 POINT(30.730818235083984 -95.79805932147787) bank47986 +47987 POINT(29.309409271393292 -96.21418945797875) bank47987 +47988 POINT(29.704425907488766 -95.63321895757316) bank47988 +47989 POINT(30.344972006396116 -94.63967614404032) bank47989 +47990 POINT(29.84245553116904 -96.01811965344861) bank47990 +47991 POINT(30.571613688673807 -95.26356534037896) bank47991 +47992 POINT(29.911850329358952 -96.30146585170617) bank47992 +47993 POINT(30.65576582252659 -95.69292269398106) bank47993 +47994 POINT(29.58930740873393 -95.65224603670426) bank47994 +47995 POINT(29.696870541049908 -96.02368605641334) bank47995 +47996 POINT(29.062813805905368 -96.17966407579165) bank47996 +47997 POINT(30.0962738172431 -95.77552401069167) bank47997 +47998 POINT(28.80849929878839 -95.28362524663244) bank47998 +47999 POINT(29.712303702961595 -95.00017570681626) bank47999 +48000 POINT(28.831220980495154 -95.11742590513533) bank48000 +48001 POINT(29.45629256334886 -96.01081317726138) bank48001 +48002 POINT(29.819447096918548 -95.93790596986173) bank48002 +48003 POINT(30.15665447736577 -96.01672370232475) bank48003 +48004 POINT(30.142081059037398 -94.98954930765109) bank48004 +48005 POINT(29.341771454383657 -94.71061824598252) bank48005 +48006 POINT(29.753120551155803 -95.73646587643597) bank48006 +48007 POINT(29.44624937727834 -95.8901501370501) bank48007 +48008 POINT(30.19354126406106 -95.86623928840845) bank48008 +48009 POINT(29.780682691675143 -94.61791239540881) bank48009 +48010 POINT(29.926179714033463 -95.10423506614322) bank48010 +48011 POINT(29.017562065167283 -95.40171484469515) bank48011 +48012 POINT(29.142426085143548 -96.02935193260846) bank48012 +48013 POINT(29.257536666571337 -96.08959481728519) bank48013 +48014 POINT(29.515406856448845 -96.20138876750767) bank48014 +48015 POINT(30.057606329630087 -94.72885900416283) bank48015 +48016 POINT(28.993298596695496 -96.26811430042765) bank48016 +48017 POINT(29.843310827657128 -96.00058954464309) bank48017 +48018 POINT(30.08952092995744 -95.28662261223842) bank48018 +48019 POINT(29.806250714376006 -95.07902653415384) bank48019 +48020 POINT(30.568065052709645 -95.57745041297605) bank48020 +48021 POINT(29.849611121936082 -95.73231611820478) bank48021 +48022 POINT(30.424774492361326 -95.71357706706786) bank48022 +48023 POINT(29.044809318014114 -95.07913735802906) bank48023 +48024 POINT(29.827913623047536 -96.305775341562) bank48024 +48025 POINT(30.25148037275759 -95.47522778718154) bank48025 +48026 POINT(29.07483097274904 -96.0460786664852) bank48026 +48027 POINT(29.95376288317534 -95.676191006918) bank48027 +48028 POINT(29.445659684747294 -94.84513356107419) bank48028 +48029 POINT(30.22281056356791 -95.02179665973411) bank48029 +48030 POINT(29.04537085390883 -95.26994010317456) bank48030 +48031 POINT(30.026010605957037 -96.32500027727812) bank48031 +48032 POINT(29.684014172034825 -95.34730983914451) bank48032 +48033 POINT(28.766731622459943 -96.3045080574718) bank48033 +48034 POINT(30.298825483563867 -94.50753595335306) bank48034 +48035 POINT(29.280484200453582 -94.75497174560596) bank48035 +48036 POINT(29.619120812194865 -95.14126023734028) bank48036 +48037 POINT(29.39492683313589 -95.10879833373036) bank48037 +48038 POINT(30.222649465965215 -94.7776498426311) bank48038 +48039 POINT(28.951172535289196 -95.71127142175646) bank48039 +48040 POINT(30.303676332055794 -95.92886884720711) bank48040 +48041 POINT(28.82377671252451 -95.83388241794675) bank48041 +48042 POINT(29.65919353563896 -95.17264977392769) bank48042 +48043 POINT(29.873092893468524 -95.40376387382966) bank48043 +48044 POINT(30.214827000257834 -94.39600792305043) bank48044 +48045 POINT(30.409887646465886 -95.09016703131181) bank48045 +48046 POINT(29.320423257466185 -95.57926034423333) bank48046 +48047 POINT(29.02460828878176 -94.42883127338138) bank48047 +48048 POINT(29.31040915336654 -95.20295509657282) bank48048 +48049 POINT(29.65352263568402 -96.10572661741666) bank48049 +48050 POINT(29.054904864650638 -95.26997781789585) bank48050 +48051 POINT(29.159038623353727 -94.40275374307474) bank48051 +48052 POINT(30.176299328273654 -96.26337440497073) bank48052 +48053 POINT(30.45489614930268 -94.99610029678993) bank48053 +48054 POINT(29.12648392298871 -95.52503678027314) bank48054 +48055 POINT(30.369027373745357 -94.98610352424633) bank48055 +48056 POINT(30.285065473450505 -94.55239529391199) bank48056 +48057 POINT(30.731214487616928 -95.80103963862443) bank48057 +48058 POINT(30.609979766732966 -95.4513305375861) bank48058 +48059 POINT(30.714741164249897 -94.61055706970411) bank48059 +48060 POINT(28.97596285793078 -94.47211729985516) bank48060 +48061 POINT(29.591437573587143 -94.86066181726665) bank48061 +48062 POINT(29.994007769625124 -95.76555346239343) bank48062 +48063 POINT(30.143266212303832 -95.28796693182434) bank48063 +48064 POINT(30.70507590404601 -95.53260453629224) bank48064 +48065 POINT(30.33599530836197 -94.57316864341851) bank48065 +48066 POINT(29.527968090083977 -96.13840983092953) bank48066 +48067 POINT(30.0573935912076 -94.66971103621403) bank48067 +48068 POINT(30.37302730133749 -96.2110492032745) bank48068 +48069 POINT(28.88604624081899 -95.58768122410147) bank48069 +48070 POINT(30.630437330840632 -96.08464861843434) bank48070 +48071 POINT(30.26762900618332 -96.08825470087687) bank48071 +48072 POINT(30.097026114120577 -95.07535520116976) bank48072 +48073 POINT(28.83532131835915 -95.76389496620749) bank48073 +48074 POINT(30.45949813277752 -94.64263442783057) bank48074 +48075 POINT(30.48477057805346 -95.96646255477785) bank48075 +48076 POINT(30.731363300546843 -94.4371360980744) bank48076 +48077 POINT(30.31377257774887 -95.96822982298985) bank48077 +48078 POINT(29.557018887301734 -95.94797915707434) bank48078 +48079 POINT(29.201453226871664 -95.73193854808909) bank48079 +48080 POINT(29.330840009635068 -96.19090589130897) bank48080 +48081 POINT(29.198818513729083 -94.47609392336524) bank48081 +48082 POINT(29.4155647400166 -95.55584504989686) bank48082 +48083 POINT(29.93836421507515 -94.71333723914361) bank48083 +48084 POINT(29.459610575908485 -94.62788394561592) bank48084 +48085 POINT(29.553689278743022 -96.04275845872677) bank48085 +48086 POINT(29.86485860485616 -94.97033183977162) bank48086 +48087 POINT(30.18998972069908 -95.11609836940407) bank48087 +48088 POINT(29.403078855748674 -94.79168997112471) bank48088 +48089 POINT(30.34775903613259 -95.78205116400797) bank48089 +48090 POINT(30.07296288778449 -95.29586396283031) bank48090 +48091 POINT(30.008391403831798 -95.73946561753901) bank48091 +48092 POINT(30.541679659286668 -95.8815907757109) bank48092 +48093 POINT(28.998648203230196 -96.28990686645822) bank48093 +48094 POINT(29.568320758652682 -95.84020789824427) bank48094 +48095 POINT(30.653065452964004 -94.85855895510494) bank48095 +48096 POINT(30.412845226647764 -95.65287170737379) bank48096 +48097 POINT(30.46273272592281 -95.04215970462586) bank48097 +48098 POINT(30.01889911960497 -95.47147765768271) bank48098 +48099 POINT(30.685369191820804 -96.15724012022132) bank48099 +48100 POINT(30.45736743071307 -94.94520655396263) bank48100 +48101 POINT(29.57179797404075 -95.28109365777571) bank48101 +48102 POINT(28.86195188766792 -96.10348126313292) bank48102 +48103 POINT(29.357154477176902 -95.15982457107492) bank48103 +48104 POINT(29.376776118167076 -94.83750506156014) bank48104 +48105 POINT(30.422998393553158 -95.26296438654886) bank48105 +48106 POINT(29.31314738081505 -95.60136990442722) bank48106 +48107 POINT(28.969226758785926 -95.09976865823523) bank48107 +48108 POINT(30.210958603333452 -94.67639124121659) bank48108 +48109 POINT(29.520338446483553 -95.55798376736745) bank48109 +48110 POINT(29.07454306117566 -94.62044258381462) bank48110 +48111 POINT(30.630178868823492 -95.2253268144878) bank48111 +48112 POINT(30.639268389893708 -94.58815110868478) bank48112 +48113 POINT(29.94351732653402 -94.73188783731189) bank48113 +48114 POINT(29.289715827387344 -94.9876523985028) bank48114 +48115 POINT(28.980087918536746 -96.22334855159946) bank48115 +48116 POINT(30.266995685632153 -94.85701067307066) bank48116 +48117 POINT(28.81938387349761 -95.49365513093825) bank48117 +48118 POINT(29.07202762438403 -95.00638149053277) bank48118 +48119 POINT(28.9793279950136 -94.76197664095379) bank48119 +48120 POINT(30.38077806003404 -96.32979851771016) bank48120 +48121 POINT(30.379531950554608 -95.93247899243393) bank48121 +48122 POINT(28.936035825871052 -95.30894352979976) bank48122 +48123 POINT(29.229098567613377 -95.88199481892883) bank48123 +48124 POINT(30.517820525419147 -96.0473673691097) bank48124 +48125 POINT(29.972615327505906 -95.85567694761063) bank48125 +48126 POINT(30.0922897792013 -96.02270080758476) bank48126 +48127 POINT(30.559328386537505 -95.33548207853114) bank48127 +48128 POINT(29.58220099881387 -96.12186293028823) bank48128 +48129 POINT(29.64352361745735 -95.93506230625994) bank48129 +48130 POINT(30.102970343155203 -95.39112659605102) bank48130 +48131 POINT(30.095024281119407 -95.3443392836346) bank48131 +48132 POINT(29.80372435158115 -96.25035711172535) bank48132 +48133 POINT(30.549942689166745 -94.69730830297597) bank48133 +48134 POINT(30.326465308683545 -94.40001519898941) bank48134 +48135 POINT(30.306523593312598 -94.8186264575209) bank48135 +48136 POINT(30.017591680271735 -94.78523553282908) bank48136 +48137 POINT(30.519670242602405 -95.58502091339439) bank48137 +48138 POINT(30.202828033609926 -95.23761853321858) bank48138 +48139 POINT(29.151084037674455 -94.77239448297118) bank48139 +48140 POINT(29.05909650488496 -95.50437725157566) bank48140 +48141 POINT(29.138473385892084 -94.99341497064178) bank48141 +48142 POINT(29.79494492031077 -96.24446792533742) bank48142 +48143 POINT(29.326982641655444 -94.48099394968324) bank48143 +48144 POINT(28.80160404692718 -94.53393758312171) bank48144 +48145 POINT(30.119625179625874 -95.69728020726687) bank48145 +48146 POINT(30.730854744144928 -95.58471685294174) bank48146 +48147 POINT(29.443791061220082 -95.90586833074902) bank48147 +48148 POINT(30.110280213538346 -95.24705330837716) bank48148 +48149 POINT(28.76489918511349 -96.26733507669039) bank48149 +48150 POINT(29.598653533419554 -96.0124817229254) bank48150 +48151 POINT(29.963854606058657 -96.33686898770658) bank48151 +48152 POINT(29.393064162355703 -94.74778961376441) bank48152 +48153 POINT(30.012714543341453 -95.91701029588826) bank48153 +48154 POINT(29.868820969183766 -96.07790116049982) bank48154 +48155 POINT(30.02217762062135 -94.91110353222278) bank48155 +48156 POINT(29.403240350946813 -96.15952498326767) bank48156 +48157 POINT(29.57001296313661 -95.01897763522159) bank48157 +48158 POINT(29.201166496843317 -95.78090890773213) bank48158 +48159 POINT(30.20545627377701 -95.337664815902) bank48159 +48160 POINT(29.72145340193706 -95.79362035876134) bank48160 +48161 POINT(30.45377688792548 -94.92519013069294) bank48161 +48162 POINT(29.114281721850745 -94.5844697428501) bank48162 +48163 POINT(30.46685702853375 -95.16208483369094) bank48163 +48164 POINT(29.675716931003947 -95.98813424980723) bank48164 +48165 POINT(29.161749272560627 -96.18160194418886) bank48165 +48166 POINT(28.83235582392352 -94.9976939116824) bank48166 +48167 POINT(30.211822114105804 -95.46398381286589) bank48167 +48168 POINT(29.766019218261352 -94.91555700378964) bank48168 +48169 POINT(29.515692698842265 -96.06825845629847) bank48169 +48170 POINT(29.847920261514822 -96.03279510858812) bank48170 +48171 POINT(29.85982071566935 -95.05066647324696) bank48171 +48172 POINT(28.94472597549326 -95.61712403845573) bank48172 +48173 POINT(30.718229850669868 -95.90229915620777) bank48173 +48174 POINT(29.65263930993528 -96.04069184710023) bank48174 +48175 POINT(28.850599351918103 -94.69566248604019) bank48175 +48176 POINT(30.35892963369363 -94.53477692505247) bank48176 +48177 POINT(29.31970983162201 -95.99953886216437) bank48177 +48178 POINT(29.32522702413641 -96.35861149292366) bank48178 +48179 POINT(30.311856079566063 -94.90442844782076) bank48179 +48180 POINT(28.79248073059777 -95.24959560789692) bank48180 +48181 POINT(30.04661011816802 -94.39107156239469) bank48181 +48182 POINT(28.785308567421207 -95.31967085738988) bank48182 +48183 POINT(30.673637040680795 -94.61300487052469) bank48183 +48184 POINT(29.662816445578848 -95.7725130255051) bank48184 +48185 POINT(28.788121617065975 -94.72168152878533) bank48185 +48186 POINT(30.14847832503324 -96.3008046582391) bank48186 +48187 POINT(30.680896528734475 -96.01695206807445) bank48187 +48188 POINT(30.600017241681826 -94.77355150155377) bank48188 +48189 POINT(28.851168241779984 -95.582949454465) bank48189 +48190 POINT(30.32320284234673 -95.60477696665913) bank48190 +48191 POINT(30.53848144980814 -95.07440643601984) bank48191 +48192 POINT(30.607572370534918 -95.47712644176407) bank48192 +48193 POINT(29.402171974095378 -95.7405213214348) bank48193 +48194 POINT(30.66860619149186 -95.00888071719282) bank48194 +48195 POINT(29.389195329054715 -95.04101162546067) bank48195 +48196 POINT(29.923323532981968 -96.21581211770223) bank48196 +48197 POINT(29.402423170752314 -94.54404910190956) bank48197 +48198 POINT(28.833025358331177 -96.0953138613257) bank48198 +48199 POINT(29.37861493894528 -95.28503244979977) bank48199 +48200 POINT(29.349002681954506 -95.40603327694647) bank48200 +48201 POINT(30.364875041093672 -96.17674362383923) bank48201 +48202 POINT(29.93208230527254 -95.73306292815982) bank48202 +48203 POINT(30.638069095138345 -95.859903572485) bank48203 +48204 POINT(29.07253357296706 -96.17872238684065) bank48204 +48205 POINT(29.807932331739735 -95.34928024323584) bank48205 +48206 POINT(29.808825662582077 -94.65357885419101) bank48206 +48207 POINT(30.622742400289717 -95.260639761281) bank48207 +48208 POINT(29.12525528459046 -95.01184462880117) bank48208 +48209 POINT(30.564744331001904 -95.99043149986434) bank48209 +48210 POINT(29.58183266979794 -95.14022137952826) bank48210 +48211 POINT(28.823756972442787 -95.1420149263564) bank48211 +48212 POINT(28.7885276288862 -95.67602633374807) bank48212 +48213 POINT(29.561868332412843 -94.85314088129789) bank48213 +48214 POINT(28.96260656081533 -94.72849352081242) bank48214 +48215 POINT(29.111428993478427 -95.5667200875034) bank48215 +48216 POINT(28.99927818430883 -95.08556481168009) bank48216 +48217 POINT(29.348270175404938 -95.62162678262067) bank48217 +48218 POINT(28.974164588124115 -94.59582841317687) bank48218 +48219 POINT(29.469112143738023 -95.54253375700483) bank48219 +48220 POINT(29.342451268445597 -94.5509414916658) bank48220 +48221 POINT(28.77330152899312 -94.95628989136287) bank48221 +48222 POINT(29.502157007773057 -95.03911680765576) bank48222 +48223 POINT(28.810877024061867 -94.71255869647379) bank48223 +48224 POINT(29.218217678685463 -94.39701539360068) bank48224 +48225 POINT(29.838668091646202 -95.60649796298887) bank48225 +48226 POINT(29.11934728903503 -94.72185495316019) bank48226 +48227 POINT(29.345344021700022 -94.93291892880707) bank48227 +48228 POINT(29.54004075346055 -94.45716116327146) bank48228 +48229 POINT(29.339217378495935 -95.14589189191523) bank48229 +48230 POINT(30.520499802530086 -95.1549247764587) bank48230 +48231 POINT(29.60669541430971 -96.04920149662043) bank48231 +48232 POINT(30.314294587682202 -95.21064748720985) bank48232 +48233 POINT(29.35212238901295 -95.04023059281461) bank48233 +48234 POINT(29.572034044304527 -95.34830059392714) bank48234 +48235 POINT(30.305047372505 -95.46233305291952) bank48235 +48236 POINT(29.81657944487081 -96.0607973767855) bank48236 +48237 POINT(29.50472195926334 -95.37845091574711) bank48237 +48238 POINT(30.545537845259986 -94.57450052437342) bank48238 +48239 POINT(30.65579347572312 -94.85680950898941) bank48239 +48240 POINT(29.972050155537545 -95.34046249993196) bank48240 +48241 POINT(30.70321714354764 -95.89043101355234) bank48241 +48242 POINT(28.99612216995077 -95.90152720718706) bank48242 +48243 POINT(30.25618991354911 -94.8999588637568) bank48243 +48244 POINT(30.553032234423277 -95.2577416207049) bank48244 +48245 POINT(29.133320806811252 -95.02933326369569) bank48245 +48246 POINT(28.825999493197344 -95.60821802529713) bank48246 +48247 POINT(29.17881138752803 -95.89657652612263) bank48247 +48248 POINT(29.860050061331545 -94.82156163726162) bank48248 +48249 POINT(30.279329883927794 -95.46964466449943) bank48249 +48250 POINT(29.830283166231162 -95.43586559494418) bank48250 +48251 POINT(30.576417704011217 -95.93587614316347) bank48251 +48252 POINT(29.525450887016163 -94.40978151672934) bank48252 +48253 POINT(28.762333877492075 -94.67682126182596) bank48253 +48254 POINT(30.419647646448478 -95.39090865348835) bank48254 +48255 POINT(30.18265449584284 -94.6827091339059) bank48255 +48256 POINT(30.438509362879252 -94.4783591016594) bank48256 +48257 POINT(29.62837061389888 -96.12214675903702) bank48257 +48258 POINT(29.79454499391302 -94.6416236067141) bank48258 +48259 POINT(30.045414509960963 -95.94940203894099) bank48259 +48260 POINT(30.043980937197407 -96.09828756672313) bank48260 +48261 POINT(29.97443388795801 -94.88321832036237) bank48261 +48262 POINT(28.776814180902413 -96.00600054437551) bank48262 +48263 POINT(30.33257883940266 -94.67909842408145) bank48263 +48264 POINT(28.765400666334568 -96.28336163445404) bank48264 +48265 POINT(30.09514166015781 -95.62187874930216) bank48265 +48266 POINT(30.691969346262752 -94.85186283337637) bank48266 +48267 POINT(29.113827287282728 -96.36892274941117) bank48267 +48268 POINT(29.25727453719936 -96.23924894634997) bank48268 +48269 POINT(29.029261117732414 -95.10611901424546) bank48269 +48270 POINT(30.628198298426582 -95.04325313720749) bank48270 +48271 POINT(28.821331914624402 -95.69218206346088) bank48271 +48272 POINT(29.673106124330772 -95.22450867324609) bank48272 +48273 POINT(29.853989337625713 -96.31415677576804) bank48273 +48274 POINT(29.682714603229577 -96.14609670127173) bank48274 +48275 POINT(28.80424655850317 -94.39577601589306) bank48275 +48276 POINT(29.10009275738875 -95.20374704819709) bank48276 +48277 POINT(30.020147184851687 -94.97469522137621) bank48277 +48278 POINT(29.79438232170503 -95.87185188543673) bank48278 +48279 POINT(30.588921035297737 -94.765681368087) bank48279 +48280 POINT(29.191950459859832 -95.5530358564535) bank48280 +48281 POINT(29.042570724995095 -95.28828736752006) bank48281 +48282 POINT(30.241643991654954 -94.70772219675149) bank48282 +48283 POINT(29.696108461506316 -95.4774003881653) bank48283 +48284 POINT(29.516866332905334 -94.62389840609336) bank48284 +48285 POINT(29.84250097199305 -96.33019288074144) bank48285 +48286 POINT(28.82696613797502 -94.59564218334937) bank48286 +48287 POINT(30.578607415968428 -95.93109960560147) bank48287 +48288 POINT(29.852366192035383 -95.45624640302813) bank48288 +48289 POINT(29.116652233232255 -95.56714227081028) bank48289 +48290 POINT(29.629137737831673 -95.87451812817099) bank48290 +48291 POINT(30.554463036497452 -95.31637641650858) bank48291 +48292 POINT(29.415159751703914 -96.32579024841793) bank48292 +48293 POINT(28.965845138072083 -94.59504632896119) bank48293 +48294 POINT(30.44190536961844 -95.99583041795073) bank48294 +48295 POINT(29.380755408750147 -95.89952654035497) bank48295 +48296 POINT(29.08413009853384 -96.27289858999951) bank48296 +48297 POINT(29.28510650502687 -95.70800044455291) bank48297 +48298 POINT(28.873589862898882 -94.98361856893753) bank48298 +48299 POINT(30.522013302379367 -95.28451100012366) bank48299 +48300 POINT(30.118691316908667 -96.36038659518518) bank48300 +48301 POINT(29.820530551812116 -95.81297459483469) bank48301 +48302 POINT(28.923319323292695 -95.05770929901134) bank48302 +48303 POINT(29.404779747543436 -95.01414700506265) bank48303 +48304 POINT(29.634195032259868 -95.44566593810146) bank48304 +48305 POINT(30.404038911573146 -94.89392811539481) bank48305 +48306 POINT(29.595194221934676 -96.30397855223683) bank48306 +48307 POINT(30.454197975724366 -95.08080543561925) bank48307 +48308 POINT(30.0092102139757 -96.08621714100502) bank48308 +48309 POINT(29.418866217245437 -96.26125504849719) bank48309 +48310 POINT(30.366945650094934 -94.51288062425758) bank48310 +48311 POINT(29.43881803440658 -96.19250956917718) bank48311 +48312 POINT(30.564678137603313 -95.39201777256176) bank48312 +48313 POINT(30.159377976165352 -95.86377308832277) bank48313 +48314 POINT(29.248106729136268 -94.748251528845) bank48314 +48315 POINT(29.983080670980126 -95.86880385199554) bank48315 +48316 POINT(29.84929530205554 -95.46685135297648) bank48316 +48317 POINT(29.990568904727667 -95.87205916179457) bank48317 +48318 POINT(29.697072569912283 -95.6171416618868) bank48318 +48319 POINT(29.786420244082418 -95.91780489555008) bank48319 +48320 POINT(30.495045254751563 -95.3124336919244) bank48320 +48321 POINT(29.73892033906805 -94.6968069909546) bank48321 +48322 POINT(29.465770470244383 -95.07365191703892) bank48322 +48323 POINT(29.446898247382112 -95.54385027170422) bank48323 +48324 POINT(29.161821040533198 -95.80735075918328) bank48324 +48325 POINT(29.70082438165599 -95.42462041071954) bank48325 +48326 POINT(29.49815041761899 -95.5224811012161) bank48326 +48327 POINT(29.91192527842613 -94.72430237075227) bank48327 +48328 POINT(30.381707872937596 -94.61799725779642) bank48328 +48329 POINT(29.75749427243857 -95.26791566823854) bank48329 +48330 POINT(29.53420582102226 -95.18041041686831) bank48330 +48331 POINT(29.915827083545082 -95.07496387334538) bank48331 +48332 POINT(28.93651975022793 -95.96795959923931) bank48332 +48333 POINT(28.87288866600123 -96.36636761732596) bank48333 +48334 POINT(30.488514594860685 -96.15390495846515) bank48334 +48335 POINT(29.289539033323827 -95.32060512273418) bank48335 +48336 POINT(29.20746788233413 -96.18876990121434) bank48336 +48337 POINT(28.866276592788672 -94.79234678231751) bank48337 +48338 POINT(29.499320497017194 -94.83370007331658) bank48338 +48339 POINT(30.330988350109152 -96.03594198265554) bank48339 +48340 POINT(30.542049794905314 -95.8153047384098) bank48340 +48341 POINT(30.046081732428462 -94.93937630007852) bank48341 +48342 POINT(30.20386622157777 -94.6399115462585) bank48342 +48343 POINT(28.79235409074584 -95.56711289694572) bank48343 +48344 POINT(29.884625704585524 -94.91492501720839) bank48344 +48345 POINT(28.941639537847184 -95.86951889068871) bank48345 +48346 POINT(29.23199781058516 -96.13456100373386) bank48346 +48347 POINT(30.612966876941112 -95.36435204220537) bank48347 +48348 POINT(28.83225061865248 -95.61884149199066) bank48348 +48349 POINT(29.081087161375418 -94.76949546479707) bank48349 +48350 POINT(29.16090568397918 -95.95499781433458) bank48350 +48351 POINT(30.56483820659659 -95.49631792503115) bank48351 +48352 POINT(30.109240497872808 -94.59676075476318) bank48352 +48353 POINT(30.3965321898011 -95.79511412423025) bank48353 +48354 POINT(29.552559611722625 -96.04333859905698) bank48354 +48355 POINT(29.29331003753161 -95.25013412350577) bank48355 +48356 POINT(29.7167219299904 -96.0184445382064) bank48356 +48357 POINT(30.14145880446836 -94.7382990441378) bank48357 +48358 POINT(29.919166622266506 -94.96068548160723) bank48358 +48359 POINT(29.933045626225876 -95.97044761650567) bank48359 +48360 POINT(30.752983592945924 -95.5886646072436) bank48360 +48361 POINT(28.83269046987069 -95.86464548301176) bank48361 +48362 POINT(30.163996375318316 -95.59670243333034) bank48362 +48363 POINT(30.04976410974369 -95.3040451471806) bank48363 +48364 POINT(30.71895292253447 -95.35579865257107) bank48364 +48365 POINT(28.773019253522126 -94.75611746623814) bank48365 +48366 POINT(30.69226935389516 -95.29025460143585) bank48366 +48367 POINT(30.615211188445492 -95.3569465792301) bank48367 +48368 POINT(29.294021396425837 -96.32812372363477) bank48368 +48369 POINT(29.5168072982821 -95.01230039234294) bank48369 +48370 POINT(29.83654603551961 -95.03284237718208) bank48370 +48371 POINT(29.314673331433504 -95.771913512821) bank48371 +48372 POINT(29.927314771413208 -96.16773774823987) bank48372 +48373 POINT(30.35758188651686 -95.66181087399178) bank48373 +48374 POINT(28.874398850539706 -96.27942493823628) bank48374 +48375 POINT(28.8580675689053 -95.09334442972627) bank48375 +48376 POINT(29.84945180740101 -95.1166270808879) bank48376 +48377 POINT(30.62535700438524 -94.6102062330828) bank48377 +48378 POINT(29.26074276862832 -95.60334121666132) bank48378 +48379 POINT(30.195403924436604 -95.61825129719651) bank48379 +48380 POINT(30.16277320767551 -96.13072449844135) bank48380 +48381 POINT(30.443453174325754 -95.55390162932935) bank48381 +48382 POINT(28.94942953202588 -95.95472499731615) bank48382 +48383 POINT(30.06007213996124 -95.19022209174717) bank48383 +48384 POINT(29.00311057072888 -94.96347390407435) bank48384 +48385 POINT(29.277831578507676 -95.76397085913125) bank48385 +48386 POINT(29.124854078107877 -94.72191408942159) bank48386 +48387 POINT(30.497461104849354 -95.29840529121766) bank48387 +48388 POINT(28.78527703903852 -95.12722996808463) bank48388 +48389 POINT(30.446358825413938 -95.25828768809538) bank48389 +48390 POINT(28.929879759926763 -95.54308362257734) bank48390 +48391 POINT(29.925403434891322 -95.7893120629457) bank48391 +48392 POINT(30.081663403603027 -96.19568658441855) bank48392 +48393 POINT(30.064031459609048 -95.20283128457756) bank48393 +48394 POINT(30.407318472884867 -94.95871306007736) bank48394 +48395 POINT(28.835965167803682 -94.6086911802471) bank48395 +48396 POINT(29.824263823312755 -94.51690937001983) bank48396 +48397 POINT(29.937547361132257 -95.51600861090824) bank48397 +48398 POINT(28.830940356035097 -95.79183513324655) bank48398 +48399 POINT(29.251124270803654 -95.50815155177945) bank48399 +48400 POINT(28.995150157711127 -94.97990959353602) bank48400 +48401 POINT(30.194739785883424 -95.59324465239466) bank48401 +48402 POINT(28.86552279320369 -96.0431116854218) bank48402 +48403 POINT(29.826088619779235 -94.9152539632764) bank48403 +48404 POINT(29.620716652517547 -94.52028285402142) bank48404 +48405 POINT(30.717833478134427 -95.56357817773592) bank48405 +48406 POINT(30.707224639502282 -95.61472863716965) bank48406 +48407 POINT(29.59011367439823 -95.2303636021248) bank48407 +48408 POINT(29.106673550283876 -95.6791559430225) bank48408 +48409 POINT(28.933128577797188 -95.90797967771731) bank48409 +48410 POINT(29.26022505881335 -95.44336073036058) bank48410 +48411 POINT(29.801267390347327 -95.18670525090252) bank48411 +48412 POINT(29.322968691797083 -95.80447483067461) bank48412 +48413 POINT(30.69554319352006 -96.13744749770872) bank48413 +48414 POINT(29.066065120013334 -94.81033315672376) bank48414 +48415 POINT(29.162783880552855 -95.40491642982624) bank48415 +48416 POINT(30.254801129599738 -95.22367008495455) bank48416 +48417 POINT(29.75568093108227 -95.48112551441348) bank48417 +48418 POINT(29.180938181247285 -94.81956969959043) bank48418 +48419 POINT(29.300504438645284 -96.14398821818104) bank48419 +48420 POINT(30.190729359828843 -96.07348673450804) bank48420 +48421 POINT(30.357725277445283 -95.77674409646384) bank48421 +48422 POINT(30.048622163307712 -95.29044436776933) bank48422 +48423 POINT(29.701265656214293 -95.7812743059546) bank48423 +48424 POINT(29.268607762345905 -95.45229968706892) bank48424 +48425 POINT(30.1991320556383 -95.57492821660347) bank48425 +48426 POINT(30.578684861017457 -94.53169397297043) bank48426 +48427 POINT(30.18523936413498 -95.35230458326735) bank48427 +48428 POINT(28.957103717111536 -94.55489274478744) bank48428 +48429 POINT(30.680251045032662 -95.61460851637975) bank48429 +48430 POINT(29.007990468164486 -95.98854699196734) bank48430 +48431 POINT(29.783894275193767 -96.08462550701914) bank48431 +48432 POINT(29.841655209531165 -94.95574977368464) bank48432 +48433 POINT(29.5540461132483 -94.43833252054323) bank48433 +48434 POINT(30.442651995473767 -96.18483308987514) bank48434 +48435 POINT(29.62901590158418 -94.57950937064263) bank48435 +48436 POINT(30.34081124586325 -94.92423078655324) bank48436 +48437 POINT(30.06744219490708 -95.45025628060449) bank48437 +48438 POINT(30.170018283579903 -95.87261394585944) bank48438 +48439 POINT(30.104680630808975 -95.01834362362979) bank48439 +48440 POINT(29.683319704163907 -96.22770648218993) bank48440 +48441 POINT(30.36247327226859 -95.47700181947455) bank48441 +48442 POINT(30.749279668031598 -94.45929805768965) bank48442 +48443 POINT(29.424207513967204 -94.64927778724795) bank48443 +48444 POINT(30.56933852502138 -94.75135393663204) bank48444 +48445 POINT(29.05318645480623 -94.392578086652) bank48445 +48446 POINT(30.547028805728402 -94.6243854285) bank48446 +48447 POINT(29.79528541137933 -95.81455355143562) bank48447 +48448 POINT(29.21987032873796 -96.28486999439) bank48448 +48449 POINT(30.09193335575657 -95.54939402032818) bank48449 +48450 POINT(29.736861244995627 -96.29293695322751) bank48450 +48451 POINT(28.958121073248773 -95.0792529722608) bank48451 +48452 POINT(29.013378254237242 -94.79507779299692) bank48452 +48453 POINT(30.53950306575829 -94.45945457826471) bank48453 +48454 POINT(30.63622239623677 -94.7466067026213) bank48454 +48455 POINT(29.175878336867807 -95.43728337406519) bank48455 +48456 POINT(29.048964236050008 -94.53324609784681) bank48456 +48457 POINT(29.263999822589522 -94.4760996384939) bank48457 +48458 POINT(29.21494422743931 -94.76219545194557) bank48458 +48459 POINT(28.791633976186688 -94.4694818827596) bank48459 +48460 POINT(30.39637625618561 -95.84872725016423) bank48460 +48461 POINT(28.793359301657922 -95.59656562317247) bank48461 +48462 POINT(30.42971074617031 -96.1294526478876) bank48462 +48463 POINT(29.431517833185556 -95.86853851352241) bank48463 +48464 POINT(28.951294138295612 -94.78414859736495) bank48464 +48465 POINT(28.84605839260166 -94.81321521033941) bank48465 +48466 POINT(29.155921983885516 -96.16366000656114) bank48466 +48467 POINT(29.76240754510838 -95.89436557925147) bank48467 +48468 POINT(29.436720394863592 -95.7193880246341) bank48468 +48469 POINT(28.90588124835197 -95.01762164497225) bank48469 +48470 POINT(29.815093295105694 -95.67792455795654) bank48470 +48471 POINT(30.61124779737601 -95.76283943019912) bank48471 +48472 POINT(30.10220608831493 -94.82793294281149) bank48472 +48473 POINT(30.187405841694787 -94.79266479714674) bank48473 +48474 POINT(30.69118543004895 -94.50620047694188) bank48474 +48475 POINT(30.156308004315477 -95.44348057383326) bank48475 +48476 POINT(29.671901888216134 -95.21985400409392) bank48476 +48477 POINT(29.799448462050844 -95.15708661079256) bank48477 +48478 POINT(28.82287431952487 -94.85100349310633) bank48478 +48479 POINT(28.81038548767521 -95.49358633751864) bank48479 +48480 POINT(30.070228422019653 -95.32358240438792) bank48480 +48481 POINT(30.390849124564163 -95.05482220414912) bank48481 +48482 POINT(30.424893758477097 -95.16053714699738) bank48482 +48483 POINT(29.033120038452715 -95.51833143507021) bank48483 +48484 POINT(28.92944236667115 -95.04378018132226) bank48484 +48485 POINT(30.00622459792746 -94.98981516938402) bank48485 +48486 POINT(30.619912845168628 -95.22329915226028) bank48486 +48487 POINT(29.67937222962484 -94.99048736106415) bank48487 +48488 POINT(30.120866346193793 -95.12343979826817) bank48488 +48489 POINT(29.18016712504076 -94.7904125696925) bank48489 +48490 POINT(29.225737542685263 -95.31510610030166) bank48490 +48491 POINT(29.66587730164519 -95.6563920081359) bank48491 +48492 POINT(28.984235973526836 -94.70404153414437) bank48492 +48493 POINT(30.568068574543542 -94.67964842874315) bank48493 +48494 POINT(29.461266201519912 -95.53971930207025) bank48494 +48495 POINT(30.278414510317194 -94.62556799386448) bank48495 +48496 POINT(29.690562318842446 -94.60126434499817) bank48496 +48497 POINT(29.528548184736923 -95.6160536919573) bank48497 +48498 POINT(28.84592379456755 -94.69373988458524) bank48498 +48499 POINT(29.79210639669122 -95.87707817144148) bank48499 +48500 POINT(29.560501459706202 -94.68486164118303) bank48500 +48501 POINT(28.87632693856109 -95.91125468370977) bank48501 +48502 POINT(30.01921250370534 -94.78439909468112) bank48502 +48503 POINT(28.98100925233878 -95.43078851562345) bank48503 +48504 POINT(28.960809602250563 -95.12453076751831) bank48504 +48505 POINT(28.879546266418824 -95.25025523031204) bank48505 +48506 POINT(29.1535396872873 -95.87754765147136) bank48506 +48507 POINT(29.404380981121754 -96.00095104589559) bank48507 +48508 POINT(30.310947288285174 -96.2228123124595) bank48508 +48509 POINT(29.99985824289648 -95.45676150784033) bank48509 +48510 POINT(28.803535148109578 -96.11294841991375) bank48510 +48511 POINT(30.63346293294117 -95.93466836634951) bank48511 +48512 POINT(29.13190272693135 -94.54879348242783) bank48512 +48513 POINT(30.54230101527772 -96.17624474846775) bank48513 +48514 POINT(29.983685452676866 -95.42169289258449) bank48514 +48515 POINT(29.17551966313968 -94.80419748168573) bank48515 +48516 POINT(29.426468707627027 -95.06528424566646) bank48516 +48517 POINT(29.02761183782965 -94.42397581565346) bank48517 +48518 POINT(30.570412639415405 -94.52913241339793) bank48518 +48519 POINT(29.220586697262203 -95.54164414779721) bank48519 +48520 POINT(29.88444132961054 -95.72964326625983) bank48520 +48521 POINT(30.171709777139323 -95.51027882161317) bank48521 +48522 POINT(29.5675800358562 -95.33793045941745) bank48522 +48523 POINT(29.14238061815261 -94.97900396866537) bank48523 +48524 POINT(30.709697741924522 -94.62484020032453) bank48524 +48525 POINT(29.414991041984873 -95.08480121088733) bank48525 +48526 POINT(30.724172406510082 -95.72582255413393) bank48526 +48527 POINT(28.83679108135358 -94.94983975681407) bank48527 +48528 POINT(30.54755983314673 -94.96232279949697) bank48528 +48529 POINT(30.126844010615905 -95.66568002650118) bank48529 +48530 POINT(29.62460141661646 -94.55523358356403) bank48530 +48531 POINT(29.81573897411866 -94.93928898909718) bank48531 +48532 POINT(29.835806852063545 -95.3469515726808) bank48532 +48533 POINT(30.360600932207447 -95.60730131941497) bank48533 +48534 POINT(30.515130476407926 -95.81584413532507) bank48534 +48535 POINT(30.68494212268671 -96.2854882137379) bank48535 +48536 POINT(29.493729357560845 -94.82229617605287) bank48536 +48537 POINT(29.107542562704353 -95.98717594469326) bank48537 +48538 POINT(30.60056988400358 -94.68119016003357) bank48538 +48539 POINT(30.320003768451873 -96.294257227437) bank48539 +48540 POINT(29.612746931376044 -94.41503176000121) bank48540 +48541 POINT(28.928175091923606 -96.30065814029535) bank48541 +48542 POINT(29.369089886920438 -95.48949561834252) bank48542 +48543 POINT(29.005740932905503 -94.39520069200677) bank48543 +48544 POINT(29.422758178566035 -95.70642885106659) bank48544 +48545 POINT(30.169539526135217 -94.40213455827171) bank48545 +48546 POINT(30.247834591755126 -94.89083050974747) bank48546 +48547 POINT(29.60407475280283 -96.19150532595529) bank48547 +48548 POINT(28.92243884004896 -96.01792798671708) bank48548 +48549 POINT(29.48217541592777 -96.00397687848091) bank48549 +48550 POINT(29.678692189542204 -95.2244307836152) bank48550 +48551 POINT(30.05029602284026 -95.3865025948549) bank48551 +48552 POINT(29.82090023535019 -94.5248064655935) bank48552 +48553 POINT(28.999696111753003 -94.90712912419797) bank48553 +48554 POINT(30.5320815823966 -96.15266475650327) bank48554 +48555 POINT(29.58001964832222 -94.59059835904057) bank48555 +48556 POINT(30.44112471890858 -94.61671311373111) bank48556 +48557 POINT(29.39100397903044 -95.64665193742829) bank48557 +48558 POINT(30.475763850115456 -94.96722462077517) bank48558 +48559 POINT(29.548458587948854 -95.41159471245848) bank48559 +48560 POINT(30.12868983505705 -95.57810469295978) bank48560 +48561 POINT(29.3955625713575 -95.43239632317979) bank48561 +48562 POINT(30.561785112265596 -95.973947170393) bank48562 +48563 POINT(29.357794864216554 -95.88425777918468) bank48563 +48564 POINT(29.015185535613778 -95.22658837925664) bank48564 +48565 POINT(30.70186021794006 -95.26442587326535) bank48565 +48566 POINT(29.659647032123765 -95.3955868506602) bank48566 +48567 POINT(30.34308917746365 -94.81612912628968) bank48567 +48568 POINT(30.086302586753423 -95.50313666314744) bank48568 +48569 POINT(30.229152418164105 -95.08923150694912) bank48569 +48570 POINT(29.04956878713504 -94.65440415220684) bank48570 +48571 POINT(30.011055193400733 -95.33523376380097) bank48571 +48572 POINT(30.421901907941688 -94.8351158654578) bank48572 +48573 POINT(30.246547936052497 -95.25653189565061) bank48573 +48574 POINT(29.030225467978827 -94.71395753314006) bank48574 +48575 POINT(29.524039123660398 -94.39171860021904) bank48575 +48576 POINT(29.67599289750531 -94.46273693191263) bank48576 +48577 POINT(30.33709939137884 -95.70737764353396) bank48577 +48578 POINT(29.862771516675533 -94.87836184983153) bank48578 +48579 POINT(30.066995859346196 -95.36127972533055) bank48579 +48580 POINT(29.295250786765575 -94.62424585618788) bank48580 +48581 POINT(29.69594178012419 -95.23952479817228) bank48581 +48582 POINT(30.291299449515915 -94.65463049388546) bank48582 +48583 POINT(29.699837938627947 -95.89373691605702) bank48583 +48584 POINT(30.08855786750539 -94.79240352090774) bank48584 +48585 POINT(29.49460676843496 -95.80558642526361) bank48585 +48586 POINT(28.939889325074237 -95.30914535802098) bank48586 +48587 POINT(29.43707643021039 -95.45329324067377) bank48587 +48588 POINT(29.03458535114526 -95.21466273102894) bank48588 +48589 POINT(30.262702991851054 -96.11771981042997) bank48589 +48590 POINT(30.757333477749384 -94.80982032713041) bank48590 +48591 POINT(29.52914388146017 -94.54974569935216) bank48591 +48592 POINT(30.322780152019583 -94.37455904808913) bank48592 +48593 POINT(29.38340132839584 -95.6586027540694) bank48593 +48594 POINT(29.117367961611095 -95.17549983043729) bank48594 +48595 POINT(30.413564268433134 -96.36227774201281) bank48595 +48596 POINT(30.62275118234677 -94.95350206817825) bank48596 +48597 POINT(28.994334696289002 -95.22124025956981) bank48597 +48598 POINT(28.89807524523081 -96.06350006729679) bank48598 +48599 POINT(29.49401733156927 -94.96553468849008) bank48599 +48600 POINT(30.18907501334487 -96.1865307171026) bank48600 +48601 POINT(29.16548768344544 -95.15344901520078) bank48601 +48602 POINT(29.621500384298198 -95.57442482372961) bank48602 +48603 POINT(30.074042667074508 -94.80655862786324) bank48603 +48604 POINT(29.388811596597797 -95.35102356320667) bank48604 +48605 POINT(28.820638239783225 -95.9668673606628) bank48605 +48606 POINT(29.240674934431784 -95.49713695405954) bank48606 +48607 POINT(28.91335095689156 -95.1006995345791) bank48607 +48608 POINT(30.433233113717133 -96.16302776460631) bank48608 +48609 POINT(28.831998521352123 -95.52030160397892) bank48609 +48610 POINT(30.102089457785656 -94.49808266361931) bank48610 +48611 POINT(29.268135424291057 -94.85917945500594) bank48611 +48612 POINT(30.758023683020944 -94.47407217641961) bank48612 +48613 POINT(30.499387475314997 -95.8394266198322) bank48613 +48614 POINT(29.213870976632535 -95.01286581788838) bank48614 +48615 POINT(29.159494020959002 -95.75336533127614) bank48615 +48616 POINT(29.736498942248296 -95.54005447802199) bank48616 +48617 POINT(29.086658953853284 -95.99915727581524) bank48617 +48618 POINT(29.806193561565244 -95.49113204870044) bank48618 +48619 POINT(29.538001283199108 -94.41826081728107) bank48619 +48620 POINT(29.972014007264537 -95.35115506927079) bank48620 +48621 POINT(30.550886760109332 -94.97529920708013) bank48621 +48622 POINT(29.59333191513717 -94.66276657923686) bank48622 +48623 POINT(29.962666182206856 -94.70000369781711) bank48623 +48624 POINT(29.703903540773815 -94.61477297241105) bank48624 +48625 POINT(30.173181880550388 -95.66968279686799) bank48625 +48626 POINT(30.734566924136047 -94.66335672774716) bank48626 +48627 POINT(29.287556632391052 -96.04385432908894) bank48627 +48628 POINT(29.100456165874512 -96.10466411465963) bank48628 +48629 POINT(30.132297903419385 -94.93666626999355) bank48629 +48630 POINT(29.694097270325535 -94.76482738375756) bank48630 +48631 POINT(30.473722334311677 -95.52030049107701) bank48631 +48632 POINT(29.48298502579994 -96.12119819751176) bank48632 +48633 POINT(30.009747020181603 -94.80335125976593) bank48633 +48634 POINT(29.39738551710846 -95.95731804903316) bank48634 +48635 POINT(29.909204451684857 -96.3480023129366) bank48635 +48636 POINT(30.412318649244376 -96.11017818916432) bank48636 +48637 POINT(29.860305879208404 -94.41065678273463) bank48637 +48638 POINT(29.879804105481092 -94.43612030607439) bank48638 +48639 POINT(30.360441278045336 -94.41383996945197) bank48639 +48640 POINT(29.112154804616342 -94.81581630714398) bank48640 +48641 POINT(29.88673301249728 -95.52599291553243) bank48641 +48642 POINT(30.358829989774403 -95.02213180099132) bank48642 +48643 POINT(30.236588594245436 -94.61572320186544) bank48643 +48644 POINT(29.362900831665698 -96.11717726432359) bank48644 +48645 POINT(30.417442607643462 -94.87243805601952) bank48645 +48646 POINT(28.85211334518946 -95.75855818964494) bank48646 +48647 POINT(30.30788959135973 -95.72816018979849) bank48647 +48648 POINT(29.070976547064504 -95.83402226343674) bank48648 +48649 POINT(30.453804865322766 -95.27710583354026) bank48649 +48650 POINT(30.746130653230065 -95.817515792382) bank48650 +48651 POINT(30.099238549201857 -94.86364681939655) bank48651 +48652 POINT(29.177921559742472 -94.72654133626929) bank48652 +48653 POINT(29.277428431644076 -96.02887017435299) bank48653 +48654 POINT(30.257124905368695 -95.6461094276287) bank48654 +48655 POINT(29.174290958641443 -95.72537797491941) bank48655 +48656 POINT(30.322990822889185 -95.32327500116548) bank48656 +48657 POINT(29.940467028983495 -95.75232319203215) bank48657 +48658 POINT(29.341020113911103 -94.87601026091484) bank48658 +48659 POINT(30.623553115172868 -96.19528004570132) bank48659 +48660 POINT(29.875269674083153 -96.0258420036296) bank48660 +48661 POINT(30.63385201656749 -94.60402406514717) bank48661 +48662 POINT(29.019440529817732 -96.11192428627115) bank48662 +48663 POINT(30.31405583737342 -95.97918275058102) bank48663 +48664 POINT(29.031210014983884 -95.28669487479534) bank48664 +48665 POINT(30.583446591623414 -96.14558507337135) bank48665 +48666 POINT(28.82753385196858 -95.80013458780282) bank48666 +48667 POINT(29.67380426376228 -95.95095594633352) bank48667 +48668 POINT(30.352942581455537 -95.4062397304978) bank48668 +48669 POINT(30.35041006462984 -95.12222751127321) bank48669 +48670 POINT(29.824244576231003 -96.27780585983753) bank48670 +48671 POINT(30.449092739060415 -96.1528346669754) bank48671 +48672 POINT(28.839834218874152 -95.40283003398658) bank48672 +48673 POINT(29.96375074570141 -96.18054647802974) bank48673 +48674 POINT(29.021407425158923 -95.603658622497) bank48674 +48675 POINT(29.131820480785382 -94.88252116565535) bank48675 +48676 POINT(30.558724674654393 -96.05672751921696) bank48676 +48677 POINT(30.472460912452043 -94.88275674841236) bank48677 +48678 POINT(29.064669930943325 -95.5679760281932) bank48678 +48679 POINT(30.201689280287468 -95.42907647229816) bank48679 +48680 POINT(30.37559937180552 -96.17806562488278) bank48680 +48681 POINT(28.89545397748214 -94.69179408163397) bank48681 +48682 POINT(29.532977830732705 -95.30327812874772) bank48682 +48683 POINT(29.59782241016942 -95.21546476058039) bank48683 +48684 POINT(29.11943792079563 -94.90475363479445) bank48684 +48685 POINT(30.557480284633783 -95.00794054793218) bank48685 +48686 POINT(30.35162568959469 -95.24163302323585) bank48686 +48687 POINT(29.23434607212929 -94.94168799125391) bank48687 +48688 POINT(30.70440953502663 -94.46131497610796) bank48688 +48689 POINT(30.734484355685627 -95.75345284180352) bank48689 +48690 POINT(30.451772918396788 -96.09847195742852) bank48690 +48691 POINT(30.382792232428475 -95.13911187310235) bank48691 +48692 POINT(29.516139376120666 -95.19150072027674) bank48692 +48693 POINT(30.645353627420878 -95.80507585818553) bank48693 +48694 POINT(28.946066128313607 -95.34983328281407) bank48694 +48695 POINT(28.763542681461338 -94.5583070378663) bank48695 +48696 POINT(29.780928584637074 -94.4631763648057) bank48696 +48697 POINT(30.1509223214141 -94.93612687248321) bank48697 +48698 POINT(29.922250321161847 -94.50286994222937) bank48698 +48699 POINT(29.348845820269876 -95.14123406812351) bank48699 +48700 POINT(29.7259249641511 -94.85838837761219) bank48700 +48701 POINT(30.683190128389864 -94.40181003758404) bank48701 +48702 POINT(30.51240373034296 -95.19392382689684) bank48702 +48703 POINT(29.09028505373268 -95.10532338508793) bank48703 +48704 POINT(29.234934594683516 -94.85028026982667) bank48704 +48705 POINT(30.203494972911113 -95.29936354197481) bank48705 +48706 POINT(30.256051868501782 -96.23751263379008) bank48706 +48707 POINT(29.10883817767891 -95.55542282019194) bank48707 +48708 POINT(29.858337447621476 -95.71584720850693) bank48708 +48709 POINT(30.338281594872008 -95.59562346077453) bank48709 +48710 POINT(30.472916219963793 -95.05122716893393) bank48710 +48711 POINT(29.859216688810807 -95.3290916793031) bank48711 +48712 POINT(30.257165295065604 -95.33065329210646) bank48712 +48713 POINT(30.102143059021515 -94.89006481979855) bank48713 +48714 POINT(29.193435643457114 -94.81947488050903) bank48714 +48715 POINT(28.972549663098977 -94.95309728009882) bank48715 +48716 POINT(30.61393993230356 -96.2082153735303) bank48716 +48717 POINT(29.28745373127566 -95.27670250323781) bank48717 +48718 POINT(29.616880334021978 -94.902291668912) bank48718 +48719 POINT(30.45582357574144 -95.46969769755276) bank48719 +48720 POINT(30.418710642252 -95.89485300085293) bank48720 +48721 POINT(30.135744665647387 -95.58838832718041) bank48721 +48722 POINT(30.052988412468423 -94.43232692761866) bank48722 +48723 POINT(30.654314713884382 -95.14364546450176) bank48723 +48724 POINT(30.081408962721177 -94.5824701966167) bank48724 +48725 POINT(30.601577113701055 -96.32210529624905) bank48725 +48726 POINT(28.820343067860097 -94.4249769591438) bank48726 +48727 POINT(29.053118390759405 -95.87176992018559) bank48727 +48728 POINT(30.29975564994248 -95.86379295272535) bank48728 +48729 POINT(30.582655155481426 -95.92334718976936) bank48729 +48730 POINT(29.498373100263557 -94.51889668163238) bank48730 +48731 POINT(29.16320730604217 -96.2917755040884) bank48731 +48732 POINT(29.03319917955612 -94.47516850829825) bank48732 +48733 POINT(29.841661054598102 -95.945343587436) bank48733 +48734 POINT(29.035288935366776 -95.1510053640874) bank48734 +48735 POINT(29.19810600419867 -94.55159901340973) bank48735 +48736 POINT(29.94334463728366 -94.64725532308967) bank48736 +48737 POINT(30.586287981864345 -95.04204934498227) bank48737 +48738 POINT(30.158623011855312 -94.86168711913919) bank48738 +48739 POINT(29.156226779463424 -96.24080848691521) bank48739 +48740 POINT(29.2706131067109 -95.03220429468044) bank48740 +48741 POINT(30.41860065895962 -96.05202109239845) bank48741 +48742 POINT(29.26210913025681 -95.22299673778672) bank48742 +48743 POINT(29.513526303587522 -95.66519338334956) bank48743 +48744 POINT(30.681704036837658 -95.62988576923982) bank48744 +48745 POINT(30.585604104320257 -94.71430913165538) bank48745 +48746 POINT(30.635337913859406 -96.26381904881278) bank48746 +48747 POINT(30.260336609585305 -94.969067547512) bank48747 +48748 POINT(30.122750828945712 -96.17626694429507) bank48748 +48749 POINT(29.924981440794834 -94.94193438393614) bank48749 +48750 POINT(29.717254678104695 -94.61662649493483) bank48750 +48751 POINT(29.693398371873 -95.69400757265473) bank48751 +48752 POINT(29.381370474450094 -96.21065368003228) bank48752 +48753 POINT(29.018816916733158 -95.79113609103568) bank48753 +48754 POINT(29.22923720224006 -95.20375970296767) bank48754 +48755 POINT(30.441656896256255 -94.92697159820435) bank48755 +48756 POINT(29.01510090048009 -94.80850897520746) bank48756 +48757 POINT(29.292483076337565 -95.8840245072831) bank48757 +48758 POINT(30.228178730658257 -96.02965794815796) bank48758 +48759 POINT(30.454754917115498 -94.57527468450505) bank48759 +48760 POINT(29.538021869335925 -95.97793290245573) bank48760 +48761 POINT(29.358537469432168 -95.50900995705683) bank48761 +48762 POINT(29.096595969729687 -94.45716130258785) bank48762 +48763 POINT(29.40062557622439 -95.40908614438862) bank48763 +48764 POINT(30.447412981905817 -95.81347599810549) bank48764 +48765 POINT(29.62973491675852 -94.80650322325506) bank48765 +48766 POINT(29.877051018652658 -95.76701954783624) bank48766 +48767 POINT(30.672831019086786 -95.5025236773358) bank48767 +48768 POINT(29.55638759289209 -95.06042334826189) bank48768 +48769 POINT(30.541694382415454 -95.23270693305689) bank48769 +48770 POINT(29.42325946833031 -95.3972266917143) bank48770 +48771 POINT(29.102826217176595 -95.87391560631232) bank48771 +48772 POINT(29.1461811403874 -95.54009149815532) bank48772 +48773 POINT(29.071011903158613 -95.88381321529222) bank48773 +48774 POINT(30.55993150512868 -94.76394109046518) bank48774 +48775 POINT(29.191211517460545 -95.29099285973118) bank48775 +48776 POINT(29.674538216408987 -95.49030929321484) bank48776 +48777 POINT(29.738167604880573 -96.36414689075247) bank48777 +48778 POINT(29.995897916809117 -95.32179958749528) bank48778 +48779 POINT(30.668548073313048 -95.50185811036691) bank48779 +48780 POINT(29.237107135614007 -94.87277274604128) bank48780 +48781 POINT(30.617741259848398 -95.6541906960449) bank48781 +48782 POINT(30.74859208724586 -94.65208632075206) bank48782 +48783 POINT(28.774098673519166 -95.07063549690932) bank48783 +48784 POINT(30.1344965978582 -95.99625263617116) bank48784 +48785 POINT(29.35154459175853 -95.12105360828893) bank48785 +48786 POINT(30.102032509586614 -94.87007817221952) bank48786 +48787 POINT(30.444616691993055 -95.34969393846555) bank48787 +48788 POINT(28.912688682378032 -96.25725842546086) bank48788 +48789 POINT(29.244319653480648 -95.47690230302405) bank48789 +48790 POINT(29.76967306011528 -95.26507245230981) bank48790 +48791 POINT(30.0247682543591 -96.04191802400389) bank48791 +48792 POINT(30.519526984664846 -94.89173323959714) bank48792 +48793 POINT(30.362574847354438 -96.19182308617535) bank48793 +48794 POINT(29.339579330588368 -95.31911735856332) bank48794 +48795 POINT(30.151521159166244 -95.17488619553063) bank48795 +48796 POINT(29.429778146206335 -96.35870973907659) bank48796 +48797 POINT(29.08388308420755 -95.33016818403286) bank48797 +48798 POINT(30.72448368183812 -94.69021811208228) bank48798 +48799 POINT(29.733431800116485 -94.41422751095624) bank48799 +48800 POINT(30.473280572443336 -94.58323203212743) bank48800 +48801 POINT(30.69712989637254 -94.93860184953874) bank48801 +48802 POINT(29.8854652825787 -96.13057905613171) bank48802 +48803 POINT(29.694520484009246 -95.25661554792995) bank48803 +48804 POINT(30.317536137739488 -95.3140710657917) bank48804 +48805 POINT(30.450023718645078 -94.52459064443376) bank48805 +48806 POINT(28.885222621847618 -94.93576518792364) bank48806 +48807 POINT(29.828193526123528 -95.94407159959079) bank48807 +48808 POINT(30.733272454410532 -95.27963793906179) bank48808 +48809 POINT(30.51017747033518 -95.457453665413) bank48809 +48810 POINT(29.79068628169586 -95.90092155087385) bank48810 +48811 POINT(29.153584168169846 -96.20664347747758) bank48811 +48812 POINT(29.319450272353013 -95.31097103820534) bank48812 +48813 POINT(30.62253029028122 -95.78384289200334) bank48813 +48814 POINT(30.349660566515624 -94.58372673898278) bank48814 +48815 POINT(29.317377493566195 -94.75185810924881) bank48815 +48816 POINT(29.46810443016732 -94.71507342343196) bank48816 +48817 POINT(29.355452611207568 -96.11206326925542) bank48817 +48818 POINT(30.074510141028615 -94.93054656408161) bank48818 +48819 POINT(29.078351301271862 -95.01768791040189) bank48819 +48820 POINT(29.73199899975149 -94.84881793687461) bank48820 +48821 POINT(30.449104778243864 -94.9881424881979) bank48821 +48822 POINT(28.955403480809895 -96.27885440842581) bank48822 +48823 POINT(29.69010271549325 -95.99828528252813) bank48823 +48824 POINT(28.82204202336965 -94.7638246873629) bank48824 +48825 POINT(29.346760831981875 -95.98112491813356) bank48825 +48826 POINT(30.079824757714466 -94.9172283790667) bank48826 +48827 POINT(30.66718565060288 -96.3267998228112) bank48827 +48828 POINT(29.279160632294907 -95.82095324412775) bank48828 +48829 POINT(30.03990830330448 -94.9564292004967) bank48829 +48830 POINT(30.44459177742927 -95.16516852753429) bank48830 +48831 POINT(29.668844127857046 -95.60463693826468) bank48831 +48832 POINT(30.06046279062608 -94.41605334026099) bank48832 +48833 POINT(30.021144985178285 -96.09668877997885) bank48833 +48834 POINT(30.01305449180091 -95.47138978823597) bank48834 +48835 POINT(29.701124339285037 -96.21405852147178) bank48835 +48836 POINT(28.802981176649453 -94.87219069865981) bank48836 +48837 POINT(29.423337609454247 -96.29823388941028) bank48837 +48838 POINT(29.295578456185243 -95.39779642734037) bank48838 +48839 POINT(28.9728359162374 -95.18524701744516) bank48839 +48840 POINT(28.867519136504306 -95.98260400178393) bank48840 +48841 POINT(30.628164654559754 -96.07114351429006) bank48841 +48842 POINT(30.23892374925512 -95.40836562124257) bank48842 +48843 POINT(29.466216278573793 -96.24993957224997) bank48843 +48844 POINT(30.505929378257004 -94.73890036690904) bank48844 +48845 POINT(28.76873508391806 -94.55131276387077) bank48845 +48846 POINT(30.138492069163636 -95.00377094003944) bank48846 +48847 POINT(29.954974808942275 -95.59498365873121) bank48847 +48848 POINT(30.349352108035276 -96.34320781067794) bank48848 +48849 POINT(29.495829510706713 -94.4320674714444) bank48849 +48850 POINT(29.67668741815613 -95.14523670948876) bank48850 +48851 POINT(28.944293872715996 -95.02632214530152) bank48851 +48852 POINT(29.99875637725785 -95.29294269729388) bank48852 +48853 POINT(30.555505670596773 -96.17162362238945) bank48853 +48854 POINT(29.689096698183764 -96.2581719486497) bank48854 +48855 POINT(29.73269118465753 -95.96879078314547) bank48855 +48856 POINT(30.206212961127708 -95.05431813349135) bank48856 +48857 POINT(28.863237318009936 -94.44150749117703) bank48857 +48858 POINT(29.33499667415408 -94.82159929347885) bank48858 +48859 POINT(29.351875600075427 -95.1793847836487) bank48859 +48860 POINT(29.692972272242457 -94.61248543711204) bank48860 +48861 POINT(28.862214515820117 -96.01961581253518) bank48861 +48862 POINT(29.358452646879012 -94.49592879544876) bank48862 +48863 POINT(29.368189723451877 -95.84599981024475) bank48863 +48864 POINT(30.19794646901527 -94.43903350001172) bank48864 +48865 POINT(28.899923012713405 -94.56143267297264) bank48865 +48866 POINT(29.23625052686247 -94.62300468655582) bank48866 +48867 POINT(30.404449205159835 -95.47860504225955) bank48867 +48868 POINT(28.857413139477377 -96.0954878555431) bank48868 +48869 POINT(30.431084837038135 -96.35520324891655) bank48869 +48870 POINT(29.946358896572804 -95.07386704547719) bank48870 +48871 POINT(29.601781940349134 -95.3893976692552) bank48871 +48872 POINT(29.346905676673522 -95.88727217706851) bank48872 +48873 POINT(29.152302899746946 -94.39179203318557) bank48873 +48874 POINT(30.451816357653854 -94.50340916913541) bank48874 +48875 POINT(30.407896611748846 -95.9225547973596) bank48875 +48876 POINT(30.03751293435871 -94.92650215423707) bank48876 +48877 POINT(30.32473781575345 -95.63650569649072) bank48877 +48878 POINT(29.506135747045533 -96.10351181287776) bank48878 +48879 POINT(29.69313383782376 -96.32193883043446) bank48879 +48880 POINT(28.90516169569858 -95.0269708812163) bank48880 +48881 POINT(30.154137514537243 -95.36634879616959) bank48881 +48882 POINT(30.009872844576762 -95.27960893652369) bank48882 +48883 POINT(29.135097802361813 -94.65395095188194) bank48883 +48884 POINT(29.716978027278618 -94.83180111120774) bank48884 +48885 POINT(29.229801702877012 -95.21088371904946) bank48885 +48886 POINT(29.66322618120798 -96.0189769335669) bank48886 +48887 POINT(29.616928101972984 -95.27905449596693) bank48887 +48888 POINT(30.02031763991744 -95.786294415717) bank48888 +48889 POINT(29.740751270015597 -95.85091593932239) bank48889 +48890 POINT(29.80970292712455 -96.15322925857883) bank48890 +48891 POINT(29.19068817607297 -96.30730449660153) bank48891 +48892 POINT(29.3712661045178 -95.84333591530051) bank48892 +48893 POINT(29.40420874575568 -95.86819777035178) bank48893 +48894 POINT(28.85965457435008 -95.5109888989344) bank48894 +48895 POINT(29.219732872578117 -95.07432603129526) bank48895 +48896 POINT(29.527359089840004 -94.75458800277217) bank48896 +48897 POINT(29.522429419218604 -95.87632420642775) bank48897 +48898 POINT(30.51761486062643 -95.63441392771303) bank48898 +48899 POINT(29.01889983742088 -95.04705373364284) bank48899 +48900 POINT(29.76668722281633 -95.3494611409831) bank48900 +48901 POINT(29.498198157977363 -95.18321861815821) bank48901 +48902 POINT(30.099343164767614 -95.69828020515784) bank48902 +48903 POINT(29.816611757377945 -94.3849038211709) bank48903 +48904 POINT(29.96250759096451 -96.23897585485679) bank48904 +48905 POINT(29.389786499075992 -95.19602432277378) bank48905 +48906 POINT(29.08439973436799 -95.66787042627546) bank48906 +48907 POINT(30.242367543322445 -95.23598556436706) bank48907 +48908 POINT(30.32864820187083 -94.75117910749566) bank48908 +48909 POINT(29.023295898496556 -94.60595987615643) bank48909 +48910 POINT(29.278489520392075 -95.96565685172779) bank48910 +48911 POINT(30.615778119499605 -95.77425577630021) bank48911 +48912 POINT(30.050342974504847 -95.7488824659335) bank48912 +48913 POINT(29.29570125919495 -94.47848481069842) bank48913 +48914 POINT(29.5738940362253 -95.69150262738768) bank48914 +48915 POINT(29.695884745052957 -96.03799966950479) bank48915 +48916 POINT(30.727759554595362 -95.07319038243101) bank48916 +48917 POINT(29.798977876360453 -94.63783983909606) bank48917 +48918 POINT(29.907631169699073 -94.74406354055057) bank48918 +48919 POINT(29.66118827563302 -95.25086397290778) bank48919 +48920 POINT(28.86063249484513 -96.19437703399055) bank48920 +48921 POINT(29.247371514567824 -94.86467881123872) bank48921 +48922 POINT(29.37716924815013 -95.63385310676077) bank48922 +48923 POINT(30.697302751464065 -94.71580116702394) bank48923 +48924 POINT(29.226796373598305 -95.99316625903023) bank48924 +48925 POINT(29.635065762889898 -96.16362674825051) bank48925 +48926 POINT(29.070467796957963 -95.43248273866014) bank48926 +48927 POINT(30.383801062494037 -95.35898786092275) bank48927 +48928 POINT(28.836401790377074 -96.12168740620216) bank48928 +48929 POINT(29.520998230143285 -95.62762228802762) bank48929 +48930 POINT(28.924437234303213 -95.44273820318897) bank48930 +48931 POINT(30.337204669210838 -96.01573572888594) bank48931 +48932 POINT(29.32020514602904 -95.54413690941927) bank48932 +48933 POINT(30.641039128057226 -95.66954482356834) bank48933 +48934 POINT(28.83840808650549 -95.53849539801159) bank48934 +48935 POINT(30.30611560153871 -95.10822441878437) bank48935 +48936 POINT(30.342333244312574 -95.63396295521511) bank48936 +48937 POINT(29.060199410869128 -94.73414366366389) bank48937 +48938 POINT(29.33126452073876 -95.29441824013666) bank48938 +48939 POINT(29.20312584466176 -94.76141668669139) bank48939 +48940 POINT(29.443309808483523 -94.8047741858811) bank48940 +48941 POINT(29.046354787819663 -96.04216440787299) bank48941 +48942 POINT(29.31613114930683 -95.62094119461702) bank48942 +48943 POINT(28.855573807980488 -95.17023958714593) bank48943 +48944 POINT(29.597915315267766 -94.3877399145134) bank48944 +48945 POINT(29.985084917286944 -94.98365193363753) bank48945 +48946 POINT(29.673035560213936 -96.22484191138996) bank48946 +48947 POINT(30.33585227207404 -94.95965491940987) bank48947 +48948 POINT(28.996446451803816 -95.26974989150307) bank48948 +48949 POINT(30.480420929126456 -94.9185263401212) bank48949 +48950 POINT(29.662587710376084 -95.11318292559733) bank48950 +48951 POINT(29.736776104497224 -95.50057925044447) bank48951 +48952 POINT(29.962023460628075 -95.49578714495797) bank48952 +48953 POINT(29.0003196827162 -95.73391239029154) bank48953 +48954 POINT(29.551275819023754 -95.77862130174012) bank48954 +48955 POINT(29.19697076004689 -96.1476741586967) bank48955 +48956 POINT(29.780659784031652 -94.6068923429982) bank48956 +48957 POINT(29.95870619358117 -95.29939389981901) bank48957 +48958 POINT(30.555635877247738 -94.7149941136942) bank48958 +48959 POINT(30.538694780713968 -96.13412578452937) bank48959 +48960 POINT(30.666065692358586 -96.02964537900573) bank48960 +48961 POINT(29.28704747244348 -94.46611762191996) bank48961 +48962 POINT(28.973268405301 -94.78908180917075) bank48962 +48963 POINT(28.85444846081345 -94.45315819633737) bank48963 +48964 POINT(30.03685665960334 -96.2201513878907) bank48964 +48965 POINT(28.964525409134772 -95.06857297732137) bank48965 +48966 POINT(29.28038544637066 -94.94698432245848) bank48966 +48967 POINT(29.90809468254501 -95.051415332533) bank48967 +48968 POINT(29.662703846040028 -96.3109992683988) bank48968 +48969 POINT(30.179937187523144 -94.75207807728277) bank48969 +48970 POINT(30.30451661025989 -95.36184727810638) bank48970 +48971 POINT(30.46856516124243 -95.80410518847879) bank48971 +48972 POINT(29.04444959166202 -95.41933932872922) bank48972 +48973 POINT(30.571717159731183 -96.22967775297683) bank48973 +48974 POINT(28.908425688528446 -94.39018079997398) bank48974 +48975 POINT(28.815963057424092 -95.85370854587332) bank48975 +48976 POINT(28.882238949823165 -96.13783074005644) bank48976 +48977 POINT(29.11482860508031 -95.40554871216902) bank48977 +48978 POINT(29.13106833301925 -95.86573598605825) bank48978 +48979 POINT(30.37824290802635 -94.46561466924598) bank48979 +48980 POINT(28.790610602973395 -95.26695086243957) bank48980 +48981 POINT(28.833068368689283 -95.37423633826764) bank48981 +48982 POINT(29.98634760332601 -94.51974609928779) bank48982 +48983 POINT(30.18316295911022 -95.6317852263536) bank48983 +48984 POINT(30.207677017578103 -95.88963998408911) bank48984 +48985 POINT(29.722054910456155 -96.02113961903324) bank48985 +48986 POINT(29.95555920081234 -96.26193108588382) bank48986 +48987 POINT(30.39837183240863 -95.90313105090958) bank48987 +48988 POINT(29.332228082117716 -95.8728830921994) bank48988 +48989 POINT(28.940629406256548 -94.81688384272482) bank48989 +48990 POINT(29.272929563137108 -95.12518971088522) bank48990 +48991 POINT(30.406803335250487 -95.44289093616773) bank48991 +48992 POINT(29.162426264675346 -96.3029045661148) bank48992 +48993 POINT(30.00538032639998 -94.890037634096) bank48993 +48994 POINT(28.926950558449917 -94.69407105797299) bank48994 +48995 POINT(29.666626724393517 -95.78125589002016) bank48995 +48996 POINT(30.172755277449717 -94.63046246996214) bank48996 +48997 POINT(29.914794225111525 -95.83906491876942) bank48997 +48998 POINT(29.648062404805923 -95.4794477640265) bank48998 +48999 POINT(28.76561540334193 -94.4529376046309) bank48999 +49000 POINT(30.547934304516815 -95.58073391806) bank49000 +49001 POINT(30.596958754933958 -94.55471019004976) bank49001 +49002 POINT(30.048388565759385 -95.30060216598841) bank49002 +49003 POINT(29.010570426564065 -96.17026362346685) bank49003 +49004 POINT(28.801541811463736 -95.72910142220681) bank49004 +49005 POINT(30.070817557851147 -95.16396453457907) bank49005 +49006 POINT(29.109769251262424 -94.81986458272041) bank49006 +49007 POINT(30.250151136469707 -96.13481691809623) bank49007 +49008 POINT(30.267685109010596 -96.28425724092435) bank49008 +49009 POINT(29.58255254643139 -95.89242005706852) bank49009 +49010 POINT(30.45710568981017 -96.12602021943886) bank49010 +49011 POINT(29.674074042733004 -95.6555559965699) bank49011 +49012 POINT(30.08386074415999 -95.36673091579999) bank49012 +49013 POINT(29.206537311041924 -95.21886252331282) bank49013 +49014 POINT(30.42234845483164 -96.00358048835965) bank49014 +49015 POINT(29.83480139454762 -96.36945249271226) bank49015 +49016 POINT(30.462724124192228 -95.36256793322028) bank49016 +49017 POINT(29.79873094083266 -95.90359868726087) bank49017 +49018 POINT(29.098411056492132 -95.897029235394) bank49018 +49019 POINT(30.652258933466204 -94.8884874998286) bank49019 +49020 POINT(30.264643657076828 -95.46054482078708) bank49020 +49021 POINT(29.470787000100245 -94.78998355347643) bank49021 +49022 POINT(29.263366537739024 -96.3175593691827) bank49022 +49023 POINT(29.280285110835933 -96.01661948884976) bank49023 +49024 POINT(30.074353972683543 -95.45416620885901) bank49024 +49025 POINT(28.923162395657638 -94.79183619432663) bank49025 +49026 POINT(29.59233815636583 -95.88450462821567) bank49026 +49027 POINT(29.188235502584735 -95.06412480473537) bank49027 +49028 POINT(28.993082072785484 -96.21068230858033) bank49028 +49029 POINT(29.611521137045493 -95.81848413037694) bank49029 +49030 POINT(29.061447806235492 -95.11015956824474) bank49030 +49031 POINT(29.38885115873356 -95.51192334965829) bank49031 +49032 POINT(29.432736137868336 -95.20287795225421) bank49032 +49033 POINT(29.786478015210434 -95.6313952023228) bank49033 +49034 POINT(30.35801510694189 -95.19414056703879) bank49034 +49035 POINT(30.72417440997539 -95.76902695317254) bank49035 +49036 POINT(29.802361022847293 -95.45798184198442) bank49036 +49037 POINT(29.950883994120833 -95.65624008657568) bank49037 +49038 POINT(29.815569673684227 -96.23142653175064) bank49038 +49039 POINT(29.37821885377776 -94.67320305337117) bank49039 +49040 POINT(28.782477313707307 -94.81881582656955) bank49040 +49041 POINT(30.401728937028423 -95.43605821564134) bank49041 +49042 POINT(29.661240338212252 -96.33931074497588) bank49042 +49043 POINT(28.771496147141733 -96.35318658626453) bank49043 +49044 POINT(29.525993719924774 -95.53726637350883) bank49044 +49045 POINT(29.95126894281606 -95.03471979948054) bank49045 +49046 POINT(28.971345873759763 -94.98560694928145) bank49046 +49047 POINT(29.62906231136875 -94.65734524335763) bank49047 +49048 POINT(29.311963694264076 -95.20650923252501) bank49048 +49049 POINT(29.569132929097083 -96.18920114640011) bank49049 +49050 POINT(29.532754926570178 -95.52712226529022) bank49050 +49051 POINT(29.994778063996993 -94.63685872221048) bank49051 +49052 POINT(30.613811917339042 -95.36631592265336) bank49052 +49053 POINT(29.89497480338748 -96.03513341678676) bank49053 +49054 POINT(30.684565144858595 -94.58335429058594) bank49054 +49055 POINT(29.08114090842905 -94.54527371622241) bank49055 +49056 POINT(29.664778706636522 -94.62085032936245) bank49056 +49057 POINT(29.96971894441054 -94.52166538838287) bank49057 +49058 POINT(28.969991352933413 -95.87384478219109) bank49058 +49059 POINT(30.356394782535673 -95.75579351195094) bank49059 +49060 POINT(29.840035090483376 -95.03743373139339) bank49060 +49061 POINT(30.604042424208842 -94.9802150693826) bank49061 +49062 POINT(29.455687301938283 -94.54755140248503) bank49062 +49063 POINT(30.450098013953603 -94.87064849168151) bank49063 +49064 POINT(30.091872276448317 -94.92023185079312) bank49064 +49065 POINT(29.879874213263026 -95.28726546166821) bank49065 +49066 POINT(30.371543662187857 -94.37098124371695) bank49066 +49067 POINT(29.986414384141696 -94.37181156121939) bank49067 +49068 POINT(30.701189026883206 -94.48079411863688) bank49068 +49069 POINT(30.051691086879345 -95.06995685079183) bank49069 +49070 POINT(29.236181248967558 -95.76211740175293) bank49070 +49071 POINT(28.81451961314338 -95.45789900028451) bank49071 +49072 POINT(29.40327492176553 -96.29232676512079) bank49072 +49073 POINT(30.481597162011738 -95.76023735639413) bank49073 +49074 POINT(29.210417208963808 -94.72883981813158) bank49074 +49075 POINT(29.656421253317564 -95.66590173780742) bank49075 +49076 POINT(30.6847044113684 -95.86990643341836) bank49076 +49077 POINT(30.570607292248095 -95.84908506347554) bank49077 +49078 POINT(30.587081735723135 -96.3645142567357) bank49078 +49079 POINT(29.966581444904953 -94.38225277867765) bank49079 +49080 POINT(29.144197685582057 -94.63219257760323) bank49080 +49081 POINT(28.769707540119082 -95.64083407225576) bank49081 +49082 POINT(29.95202537872979 -95.72713150029662) bank49082 +49083 POINT(29.394177739617497 -95.54613085335097) bank49083 +49084 POINT(30.18013720287014 -95.8923143013277) bank49084 +49085 POINT(29.025430424038078 -95.75405714382892) bank49085 +49086 POINT(30.652647794677534 -95.25953137104516) bank49086 +49087 POINT(30.52844874683093 -95.04094496209248) bank49087 +49088 POINT(30.505065921119897 -94.93049852035296) bank49088 +49089 POINT(29.61246122057472 -94.38979003811644) bank49089 +49090 POINT(30.56033915686325 -96.31427421554379) bank49090 +49091 POINT(29.127418030928666 -96.25606394783978) bank49091 +49092 POINT(29.07145156022286 -94.4843944320708) bank49092 +49093 POINT(29.284408359496194 -95.77946282208458) bank49093 +49094 POINT(30.550309014819163 -95.43266613489173) bank49094 +49095 POINT(29.504059092079515 -94.48206301056312) bank49095 +49096 POINT(29.747574183895274 -95.97143100532688) bank49096 +49097 POINT(29.336503187989386 -95.26261764817082) bank49097 +49098 POINT(30.586854115466057 -95.42744626540663) bank49098 +49099 POINT(28.971422876167686 -94.6524998581577) bank49099 +49100 POINT(30.04443303333479 -95.45621126052548) bank49100 +49101 POINT(29.64177192416097 -95.88749788725076) bank49101 +49102 POINT(28.880273903940235 -94.51944265123267) bank49102 +49103 POINT(29.30900842130217 -95.74585795793978) bank49103 +49104 POINT(30.091392670748956 -95.10912345760562) bank49104 +49105 POINT(30.400259725674683 -95.85247506702508) bank49105 +49106 POINT(29.80768014448747 -94.66842095017942) bank49106 +49107 POINT(28.977503615094623 -95.2669571413086) bank49107 +49108 POINT(29.983164848540866 -96.04771639066814) bank49108 +49109 POINT(28.92880706970992 -94.77854150682339) bank49109 +49110 POINT(29.707795722271598 -96.269081153164) bank49110 +49111 POINT(29.394073093373002 -94.40386875140723) bank49111 +49112 POINT(28.862773706814018 -95.2603964412046) bank49112 +49113 POINT(29.85701475984459 -96.16871776235793) bank49113 +49114 POINT(29.100096939394504 -95.48126066044694) bank49114 +49115 POINT(30.663107395349865 -95.95023700573039) bank49115 +49116 POINT(30.39620128232486 -95.60473537960627) bank49116 +49117 POINT(30.216443596818596 -95.9032875158743) bank49117 +49118 POINT(30.09961072868835 -96.10926496687613) bank49118 +49119 POINT(29.82723686044416 -95.4492009195025) bank49119 +49120 POINT(29.70473453191673 -94.74513397021072) bank49120 +49121 POINT(30.203217248173257 -96.04991115260908) bank49121 +49122 POINT(29.15373708757378 -96.25030081351326) bank49122 +49123 POINT(30.387548522386552 -95.4705725903553) bank49123 +49124 POINT(30.404327844677105 -94.80371549993309) bank49124 +49125 POINT(30.096811230559666 -95.22435108048282) bank49125 +49126 POINT(30.300423897897975 -95.87677921589093) bank49126 +49127 POINT(28.872192355064115 -95.21379473688474) bank49127 +49128 POINT(30.73255163268156 -94.75709545514884) bank49128 +49129 POINT(30.282523533362593 -95.68318356414932) bank49129 +49130 POINT(30.00485305962152 -94.3744770631925) bank49130 +49131 POINT(29.985735589424106 -94.38093714357076) bank49131 +49132 POINT(28.803101187314404 -94.37569167801654) bank49132 +49133 POINT(29.301326638323786 -95.77643758746862) bank49133 +49134 POINT(28.986540111901757 -94.80919004984717) bank49134 +49135 POINT(29.582770159850615 -95.56780426305029) bank49135 +49136 POINT(29.335190841980957 -96.16885998594313) bank49136 +49137 POINT(28.89975272160308 -94.83446022557635) bank49137 +49138 POINT(29.17852961491183 -95.36875529956622) bank49138 +49139 POINT(29.6406061099677 -95.94777064625988) bank49139 +49140 POINT(30.133422032429042 -94.8415064562326) bank49140 +49141 POINT(29.509697887547972 -95.65880435698442) bank49141 +49142 POINT(29.16422951150339 -95.5809837554221) bank49142 +49143 POINT(29.485800309817435 -95.23050018954022) bank49143 +49144 POINT(28.952554931838247 -95.08227415961039) bank49144 +49145 POINT(28.999296443710822 -95.73046585090513) bank49145 +49146 POINT(29.639688283050422 -96.32153564555487) bank49146 +49147 POINT(29.392933044346577 -95.83045970996358) bank49147 +49148 POINT(28.902154336575947 -94.80980319319828) bank49148 +49149 POINT(29.880808791715403 -95.82320030795871) bank49149 +49150 POINT(29.928635052220557 -94.51109268756466) bank49150 +49151 POINT(28.948844681512508 -96.31210045588404) bank49151 +49152 POINT(29.91114621060976 -95.34251950405276) bank49152 +49153 POINT(30.371302998821637 -95.70932410362359) bank49153 +49154 POINT(29.433132307544756 -94.89998764093508) bank49154 +49155 POINT(28.863722697818876 -94.56682230196624) bank49155 +49156 POINT(30.331579985687156 -94.65617104866051) bank49156 +49157 POINT(30.687815887729002 -96.32733003521135) bank49157 +49158 POINT(29.039619242896947 -94.5376919318934) bank49158 +49159 POINT(30.577573023145256 -96.094524514181) bank49159 +49160 POINT(28.796040429105293 -95.72799444477643) bank49160 +49161 POINT(30.24115950761131 -95.25067562652004) bank49161 +49162 POINT(30.752532856791866 -94.63872825744649) bank49162 +49163 POINT(29.003443174541204 -95.05028921344531) bank49163 +49164 POINT(30.385078624970944 -94.51473963643166) bank49164 +49165 POINT(28.988888161173634 -94.50742957836604) bank49165 +49166 POINT(29.246115379257805 -96.04679251001403) bank49166 +49167 POINT(30.371979438731593 -94.58186559165324) bank49167 +49168 POINT(30.1335057784397 -95.02494978751558) bank49168 +49169 POINT(29.672849497074566 -95.75574977188377) bank49169 +49170 POINT(28.795942364934596 -94.87267631213089) bank49170 +49171 POINT(29.450872703915124 -95.98802358747312) bank49171 +49172 POINT(30.015938155739185 -95.16552948116151) bank49172 +49173 POINT(28.95291155524429 -94.63668136594649) bank49173 +49174 POINT(30.25938261619002 -95.02164976230908) bank49174 +49175 POINT(30.48525616934345 -94.8833680311452) bank49175 +49176 POINT(30.205865050151544 -94.48646935739595) bank49176 +49177 POINT(29.333639170867844 -94.88089055643428) bank49177 +49178 POINT(30.018757726866195 -94.42039196130584) bank49178 +49179 POINT(29.734434934125343 -95.23871908787706) bank49179 +49180 POINT(30.526724488153885 -94.67308815863016) bank49180 +49181 POINT(29.519241649414862 -95.68627391395027) bank49181 +49182 POINT(30.186897260535204 -95.74224706755146) bank49182 +49183 POINT(29.045404513496347 -95.49637557522783) bank49183 +49184 POINT(30.561567242803154 -96.03083058730088) bank49184 +49185 POINT(30.598223358322013 -95.06281367084743) bank49185 +49186 POINT(30.087524311107046 -96.2657244387555) bank49186 +49187 POINT(30.624222977354076 -95.03079968642065) bank49187 +49188 POINT(29.06461290524142 -94.77962387235186) bank49188 +49189 POINT(29.117490900537423 -94.37379627215986) bank49189 +49190 POINT(30.036371091117122 -95.96991723725327) bank49190 +49191 POINT(29.402444407592682 -94.53238197761051) bank49191 +49192 POINT(29.49491590201853 -95.41029077265195) bank49192 +49193 POINT(29.708993763157608 -95.33550419619303) bank49193 +49194 POINT(29.920501068302354 -94.57482909418374) bank49194 +49195 POINT(29.915193143321034 -95.41318683312569) bank49195 +49196 POINT(30.135500342208683 -96.32742698709608) bank49196 +49197 POINT(30.110318214001154 -94.46733854838429) bank49197 +49198 POINT(29.71383423811198 -94.7314631709663) bank49198 +49199 POINT(29.035923621141563 -94.96636301719069) bank49199 +49200 POINT(29.021179670378668 -95.2816683190433) bank49200 +49201 POINT(28.78117215771633 -95.76676606017723) bank49201 +49202 POINT(30.29265680621881 -95.6743304999042) bank49202 +49203 POINT(30.387768886348464 -94.86943317058774) bank49203 +49204 POINT(28.995260844826294 -94.48262312605013) bank49204 +49205 POINT(29.331103722436822 -95.365131411516) bank49205 +49206 POINT(28.892469978081294 -95.26260337962162) bank49206 +49207 POINT(29.066923417435433 -94.74527090556906) bank49207 +49208 POINT(29.391885216214593 -94.47562666434312) bank49208 +49209 POINT(30.395861521906777 -94.80629814675527) bank49209 +49210 POINT(30.083816229349527 -94.49756412007481) bank49210 +49211 POINT(28.949187694443978 -95.08770617359309) bank49211 +49212 POINT(30.640311280280745 -95.17448119323657) bank49212 +49213 POINT(29.847935315360512 -95.62017088895982) bank49213 +49214 POINT(29.175377008974298 -94.40859193082021) bank49214 +49215 POINT(30.27359094540583 -95.97394298413978) bank49215 +49216 POINT(30.135756551516806 -96.07335493006643) bank49216 +49217 POINT(30.17549854325889 -95.54320392773194) bank49217 +49218 POINT(30.230482845795272 -94.54151537843771) bank49218 +49219 POINT(30.01113207356391 -96.21744163461938) bank49219 +49220 POINT(30.455909508835564 -96.16123787530289) bank49220 +49221 POINT(29.66004778833246 -94.81263393852826) bank49221 +49222 POINT(30.14592150886488 -96.04731239680424) bank49222 +49223 POINT(30.669104629056374 -95.04084202443146) bank49223 +49224 POINT(30.429047637091454 -94.49298528981952) bank49224 +49225 POINT(29.771151455005693 -94.7875088507489) bank49225 +49226 POINT(28.8415645902048 -95.9214703463905) bank49226 +49227 POINT(30.040381036987608 -95.33999077890577) bank49227 +49228 POINT(29.372396431686187 -94.78426227871171) bank49228 +49229 POINT(30.316740802418174 -94.461256569556) bank49229 +49230 POINT(30.622659693093397 -94.66271581205089) bank49230 +49231 POINT(29.295708713749942 -94.72281916100621) bank49231 +49232 POINT(30.687724569781835 -96.12206090459642) bank49232 +49233 POINT(30.279617200214542 -95.05903646935813) bank49233 +49234 POINT(29.843186052592173 -94.82258531305041) bank49234 +49235 POINT(29.506333787914432 -95.51755129872484) bank49235 +49236 POINT(30.560271783820728 -95.96393739895426) bank49236 +49237 POINT(29.863332638571407 -95.65450261300664) bank49237 +49238 POINT(29.060007747894552 -95.1360666328431) bank49238 +49239 POINT(30.11030891561961 -95.8661720199499) bank49239 +49240 POINT(29.163027289513707 -96.34012356779846) bank49240 +49241 POINT(30.138979237151197 -95.55320124359277) bank49241 +49242 POINT(30.10004173137396 -95.38313167351866) bank49242 +49243 POINT(29.865984306325352 -95.74100297561974) bank49243 +49244 POINT(30.64227201733207 -95.56424813414468) bank49244 +49245 POINT(30.619566356924498 -95.27392804388948) bank49245 +49246 POINT(28.770718373858333 -95.83668682602696) bank49246 +49247 POINT(29.557780521978952 -94.57780164919937) bank49247 +49248 POINT(30.3392719482343 -95.37569406523022) bank49248 +49249 POINT(30.758080316454404 -96.20348203046814) bank49249 +49250 POINT(29.92387555856162 -96.27527819251027) bank49250 +49251 POINT(29.035183403127625 -96.29972566629874) bank49251 +49252 POINT(30.585573208583053 -95.7255935456617) bank49252 +49253 POINT(29.037491398827015 -96.2517030029808) bank49253 +49254 POINT(29.172236323154905 -95.4064801061745) bank49254 +49255 POINT(30.40259849827364 -95.24980631335127) bank49255 +49256 POINT(28.86646988298239 -95.4970523948267) bank49256 +49257 POINT(30.063608588230558 -96.30726719556236) bank49257 +49258 POINT(29.21928336670686 -95.62607517126459) bank49258 +49259 POINT(28.941319079154272 -96.3025190917195) bank49259 +49260 POINT(29.03709797098508 -95.26430641648629) bank49260 +49261 POINT(29.636133170111307 -94.47760968603963) bank49261 +49262 POINT(29.006864370657848 -95.20637138498395) bank49262 +49263 POINT(29.475878989881394 -95.63465690294555) bank49263 +49264 POINT(29.92556639611972 -95.66385124875531) bank49264 +49265 POINT(29.156291258139255 -96.18534151016978) bank49265 +49266 POINT(30.040261926516933 -96.07314543867466) bank49266 +49267 POINT(30.60669816349814 -95.30389360836591) bank49267 +49268 POINT(30.342384172993974 -96.21122543817937) bank49268 +49269 POINT(29.167957668417323 -95.09347134194542) bank49269 +49270 POINT(30.38750071585188 -95.76073397623153) bank49270 +49271 POINT(29.99147752309103 -94.6208009529148) bank49271 +49272 POINT(29.52338394123936 -95.9692015552984) bank49272 +49273 POINT(29.4487253997611 -94.78692432662842) bank49273 +49274 POINT(29.78207246280544 -95.17903721494298) bank49274 +49275 POINT(29.599570392153495 -94.9608535987321) bank49275 +49276 POINT(30.703278148029714 -95.11197739041347) bank49276 +49277 POINT(30.6529884797829 -96.01946833471408) bank49277 +49278 POINT(29.816647898864968 -96.03686170133209) bank49278 +49279 POINT(30.077310787021304 -96.07378359650963) bank49279 +49280 POINT(30.67172591759424 -95.74077040062407) bank49280 +49281 POINT(29.08781054297302 -95.06642107337147) bank49281 +49282 POINT(29.815771072286907 -95.06568220314502) bank49282 +49283 POINT(30.29798914459513 -94.99420658907333) bank49283 +49284 POINT(30.232868345543345 -94.8238068125862) bank49284 +49285 POINT(28.871146289788 -95.68343689877192) bank49285 +49286 POINT(29.445089862554536 -95.08574586168668) bank49286 +49287 POINT(30.443446824986292 -95.64524043174902) bank49287 +49288 POINT(29.785123483293226 -96.03813149317065) bank49288 +49289 POINT(28.91960917268215 -95.78707946064661) bank49289 +49290 POINT(29.01619118723473 -95.64060014165699) bank49290 +49291 POINT(29.28766430184862 -94.70489328051676) bank49291 +49292 POINT(30.506737286042238 -95.94053470895662) bank49292 +49293 POINT(30.543603036311833 -95.82437495777042) bank49293 +49294 POINT(30.616553962414148 -94.88021369344557) bank49294 +49295 POINT(29.546254098802656 -95.6858059459116) bank49295 +49296 POINT(30.385766067399697 -94.90822728045725) bank49296 +49297 POINT(30.222256795020364 -95.00075813112697) bank49297 +49298 POINT(30.123974635066695 -96.22031315731473) bank49298 +49299 POINT(30.396941710624468 -94.7026365674266) bank49299 +49300 POINT(30.439851726135853 -96.13940590962251) bank49300 +49301 POINT(28.77858926496682 -95.9344743174492) bank49301 +49302 POINT(28.9659450593603 -95.06877404411226) bank49302 +49303 POINT(30.581876518940522 -95.48978479375063) bank49303 +49304 POINT(30.248428963958823 -96.28193543474991) bank49304 +49305 POINT(30.33403673167741 -95.60558137363473) bank49305 +49306 POINT(30.403087051742084 -96.33845501739526) bank49306 +49307 POINT(30.15527659135657 -95.17784824066806) bank49307 +49308 POINT(29.811468354242617 -95.96175324724635) bank49308 +49309 POINT(30.411970526314015 -95.26410471220306) bank49309 +49310 POINT(29.754036267688843 -95.2037858894501) bank49310 +49311 POINT(29.600482456862967 -95.51945482483545) bank49311 +49312 POINT(30.566187036002475 -94.60245258935245) bank49312 +49313 POINT(30.725514691881813 -94.92332278683361) bank49313 +49314 POINT(30.38387331371637 -94.5887268724801) bank49314 +49315 POINT(29.025070772842405 -94.74273406832451) bank49315 +49316 POINT(28.84463245154413 -94.47245544098614) bank49316 +49317 POINT(30.547659122840095 -95.78129358296273) bank49317 +49318 POINT(28.78007394579328 -96.0117663775096) bank49318 +49319 POINT(29.770038984149412 -95.4497932219891) bank49319 +49320 POINT(29.426490635749214 -95.99909067013904) bank49320 +49321 POINT(29.855230269431367 -95.91253526043096) bank49321 +49322 POINT(29.519799255271078 -94.47028570681397) bank49322 +49323 POINT(29.439862446896907 -96.16198993698423) bank49323 +49324 POINT(30.070643045575967 -95.61392661921431) bank49324 +49325 POINT(29.455826614722685 -96.29462006591169) bank49325 +49326 POINT(29.926672451002595 -95.43322999012706) bank49326 +49327 POINT(29.090184209413458 -95.72247523237509) bank49327 +49328 POINT(30.249165785811563 -95.34065820294957) bank49328 +49329 POINT(29.76776541096608 -96.23910316111476) bank49329 +49330 POINT(30.153092379259963 -94.94515073699802) bank49330 +49331 POINT(30.61385449046792 -95.31234657451716) bank49331 +49332 POINT(30.760064252253766 -95.76304357646573) bank49332 +49333 POINT(29.56915884655509 -95.75543267519335) bank49333 +49334 POINT(30.74162234807857 -94.53718996717947) bank49334 +49335 POINT(30.604879897973976 -94.81173328652669) bank49335 +49336 POINT(29.123247122403974 -96.05972293031832) bank49336 +49337 POINT(30.62268131048353 -95.47423183927907) bank49337 +49338 POINT(30.40513363968967 -94.43924844153463) bank49338 +49339 POINT(29.37229961523232 -95.16727433580722) bank49339 +49340 POINT(28.852286360848456 -95.66097106670509) bank49340 +49341 POINT(29.764021636271043 -94.61606369102472) bank49341 +49342 POINT(30.58214287931012 -94.84959801921435) bank49342 +49343 POINT(29.55618885885216 -94.3914125644068) bank49343 +49344 POINT(30.60725028030909 -95.26071736050463) bank49344 +49345 POINT(29.11217584973889 -95.64225124917631) bank49345 +49346 POINT(29.40521804258471 -95.0557644280172) bank49346 +49347 POINT(30.501504849540638 -95.69569092633861) bank49347 +49348 POINT(30.66635668560102 -96.3511695828146) bank49348 +49349 POINT(30.360511236955738 -95.14940962410311) bank49349 +49350 POINT(29.114122552551006 -95.48230491323267) bank49350 +49351 POINT(29.226609518815966 -95.01801903171065) bank49351 +49352 POINT(29.62000926632186 -95.60073563217452) bank49352 +49353 POINT(28.949810743336 -94.62455307742428) bank49353 +49354 POINT(30.686427907576565 -94.38929606220908) bank49354 +49355 POINT(29.1336081966629 -95.37424883308535) bank49355 +49356 POINT(30.22310140330656 -95.47964284278277) bank49356 +49357 POINT(29.54659413497711 -94.90420968537921) bank49357 +49358 POINT(29.48858136603675 -94.47929707618171) bank49358 +49359 POINT(29.454220611764878 -95.5503830413478) bank49359 +49360 POINT(30.449000124235415 -95.15505406493749) bank49360 +49361 POINT(29.531102857770957 -96.31709695907509) bank49361 +49362 POINT(29.832974146761313 -96.07058166851566) bank49362 +49363 POINT(29.958368195748058 -96.30321717213104) bank49363 +49364 POINT(30.12518336081074 -96.12469065979614) bank49364 +49365 POINT(30.23707041881221 -94.99095891809795) bank49365 +49366 POINT(29.707424763227237 -96.24373417248295) bank49366 +49367 POINT(30.204237389941003 -96.30251067173889) bank49367 +49368 POINT(29.347632417613475 -95.63812539667737) bank49368 +49369 POINT(30.687756363721874 -96.28802476266499) bank49369 +49370 POINT(29.62298586584108 -95.28815367400625) bank49370 +49371 POINT(30.7334253351595 -95.20367724493958) bank49371 +49372 POINT(29.48810074094895 -95.89084022072613) bank49372 +49373 POINT(30.470534062015204 -95.68142153615912) bank49373 +49374 POINT(29.001258103364222 -95.0276369683298) bank49374 +49375 POINT(29.029402766224038 -95.76077401301642) bank49375 +49376 POINT(29.487437581136373 -96.1037496118397) bank49376 +49377 POINT(29.262640954491776 -95.88488382794716) bank49377 +49378 POINT(29.186814878657152 -94.64245250867855) bank49378 +49379 POINT(28.9653746683009 -96.33067496790045) bank49379 +49380 POINT(29.686461495172267 -96.2117168491556) bank49380 +49381 POINT(30.31145115329297 -95.51299519556858) bank49381 +49382 POINT(30.351898749850736 -95.24516960099396) bank49382 +49383 POINT(29.81523127971489 -94.68504810630846) bank49383 +49384 POINT(28.85406631232115 -95.24471123775464) bank49384 +49385 POINT(28.885791496080788 -94.60530353660843) bank49385 +49386 POINT(29.741651066036663 -95.37641718794404) bank49386 +49387 POINT(30.719885921016527 -95.48135963829084) bank49387 +49388 POINT(28.89932117980886 -96.14075694601588) bank49388 +49389 POINT(30.54039673550691 -96.3154268289656) bank49389 +49390 POINT(29.384063810542393 -95.58130361125714) bank49390 +49391 POINT(29.46380823529191 -94.78191276720256) bank49391 +49392 POINT(30.25339323992138 -94.78817518892308) bank49392 +49393 POINT(29.89059003583521 -95.15050743208785) bank49393 +49394 POINT(29.875376050043972 -94.40174243068016) bank49394 +49395 POINT(29.487484504945073 -95.97983120169106) bank49395 +49396 POINT(30.27078499635341 -95.87829325970407) bank49396 +49397 POINT(29.270078145551146 -95.84879749271336) bank49397 +49398 POINT(30.312343502642236 -95.71138684628733) bank49398 +49399 POINT(30.230733354057797 -95.36053680674469) bank49399 +49400 POINT(30.416717320155826 -95.34861850443717) bank49400 +49401 POINT(29.956829767077487 -96.0032985186961) bank49401 +49402 POINT(29.598009108021035 -96.1721392518675) bank49402 +49403 POINT(29.01463233456413 -95.92976088357094) bank49403 +49404 POINT(29.22333680561324 -95.86635732496498) bank49404 +49405 POINT(30.41150628580302 -95.45370289364577) bank49405 +49406 POINT(29.26772274148094 -94.61114958634616) bank49406 +49407 POINT(29.39527929005087 -95.91167073242461) bank49407 +49408 POINT(30.056578434248785 -95.47657340833963) bank49408 +49409 POINT(29.77025697474891 -95.73045468124342) bank49409 +49410 POINT(29.194564789084833 -96.33223276573419) bank49410 +49411 POINT(29.936395457496776 -95.36676878909576) bank49411 +49412 POINT(29.737035236884044 -94.86949328940318) bank49412 +49413 POINT(29.162497427516616 -96.30690337283791) bank49413 +49414 POINT(30.02864970838453 -94.4940495010788) bank49414 +49415 POINT(29.8243012582696 -96.3293199965112) bank49415 +49416 POINT(29.220477560089392 -95.95006151628638) bank49416 +49417 POINT(29.0563027692542 -94.98358610876907) bank49417 +49418 POINT(29.866498631887122 -95.28667040479507) bank49418 +49419 POINT(29.709834278095247 -96.02237188297511) bank49419 +49420 POINT(30.484366020403183 -95.16210325598664) bank49420 +49421 POINT(29.09408089754444 -95.86110773266022) bank49421 +49422 POINT(29.656759043854578 -96.03015223577896) bank49422 +49423 POINT(29.94028751203349 -94.71590211614271) bank49423 +49424 POINT(30.277606758046574 -94.4754430201759) bank49424 +49425 POINT(29.988686670955417 -95.77753365033982) bank49425 +49426 POINT(30.70132883841494 -94.73568067561425) bank49426 +49427 POINT(30.53117203815267 -96.36608608290615) bank49427 +49428 POINT(29.072005728764726 -94.53874087274855) bank49428 +49429 POINT(28.87121606503039 -95.85024627334232) bank49429 +49430 POINT(30.180677851362088 -95.77145709065987) bank49430 +49431 POINT(28.85254419878211 -94.54608051329278) bank49431 +49432 POINT(29.561604053065402 -94.38193536206488) bank49432 +49433 POINT(28.97375150785078 -94.63828051786555) bank49433 +49434 POINT(30.374189127539474 -96.12074277574848) bank49434 +49435 POINT(30.456268757693554 -94.99313554938925) bank49435 +49436 POINT(29.288032274197676 -94.5963499730567) bank49436 +49437 POINT(29.483600902955803 -95.89297482728114) bank49437 +49438 POINT(30.353204370244722 -96.00745845026621) bank49438 +49439 POINT(30.320111720407958 -95.08023748388678) bank49439 +49440 POINT(30.678565998341032 -96.36875029476187) bank49440 +49441 POINT(29.06765232173663 -94.90671355139092) bank49441 +49442 POINT(29.976734697521632 -96.12755027943813) bank49442 +49443 POINT(30.433682502623416 -95.28395672508813) bank49443 +49444 POINT(30.15114249182744 -94.47383048098277) bank49444 +49445 POINT(29.887991044953527 -95.90529880764073) bank49445 +49446 POINT(30.43810620878657 -95.41846277886957) bank49446 +49447 POINT(30.608766284379076 -95.88664822496352) bank49447 +49448 POINT(29.59229672653506 -96.10014539938284) bank49448 +49449 POINT(30.341445768175053 -95.51802277957725) bank49449 +49450 POINT(29.545152188332608 -95.7923267983479) bank49450 +49451 POINT(29.64573682246344 -94.48437867585072) bank49451 +49452 POINT(29.58398518964933 -95.68901261509052) bank49452 +49453 POINT(30.6209422711746 -94.67554206724233) bank49453 +49454 POINT(30.308080569693786 -94.82799825566244) bank49454 +49455 POINT(30.439684950717922 -95.43952330196468) bank49455 +49456 POINT(30.63518779651119 -95.16336177921004) bank49456 +49457 POINT(30.5054234686994 -94.38840162020081) bank49457 +49458 POINT(30.433290776168707 -95.99964226844973) bank49458 +49459 POINT(30.452261404045824 -95.59789104897544) bank49459 +49460 POINT(29.73653767491908 -94.55916272550958) bank49460 +49461 POINT(29.124254337902673 -95.61231771336826) bank49461 +49462 POINT(30.268884203876233 -94.79002702049111) bank49462 +49463 POINT(29.589220023579205 -95.99419024805377) bank49463 +49464 POINT(30.19280958931458 -94.95389910359631) bank49464 +49465 POINT(30.323150224203452 -95.68108753704809) bank49465 +49466 POINT(29.13028321749886 -95.10673830132417) bank49466 +49467 POINT(30.23487166283255 -94.77932874159801) bank49467 +49468 POINT(29.396731313914223 -96.0182942560549) bank49468 +49469 POINT(30.351224862729314 -94.66060146991741) bank49469 +49470 POINT(28.984851192181086 -96.04432746326628) bank49470 +49471 POINT(29.98224467349151 -96.25272758725802) bank49471 +49472 POINT(29.355413343617265 -96.04700053091011) bank49472 +49473 POINT(30.36492295272771 -95.05919141015535) bank49473 +49474 POINT(29.549710441378483 -94.66993206324405) bank49474 +49475 POINT(29.689586495089475 -95.5683253478556) bank49475 +49476 POINT(29.913349047840132 -96.20059885561545) bank49476 +49477 POINT(30.32536146760152 -95.20109711593389) bank49477 +49478 POINT(30.38375248434705 -96.15262249393695) bank49478 +49479 POINT(30.13969609548521 -94.93213788850734) bank49479 +49480 POINT(29.520087107435085 -95.63680988749294) bank49480 +49481 POINT(30.21313719360402 -95.52206719234144) bank49481 +49482 POINT(29.868687558562765 -94.38520113776733) bank49482 +49483 POINT(30.679939618547692 -96.36643917063846) bank49483 +49484 POINT(29.228896659126114 -95.5464815500788) bank49484 +49485 POINT(28.87929879009284 -96.06587560925216) bank49485 +49486 POINT(29.114734243858706 -94.4143515417426) bank49486 +49487 POINT(29.605126721737655 -94.80969128995777) bank49487 +49488 POINT(30.041972212878605 -95.43146634807093) bank49488 +49489 POINT(30.747095084854774 -95.32364708047974) bank49489 +49490 POINT(28.84767917766003 -96.23606018100395) bank49490 +49491 POINT(29.980928493664415 -95.5515358614793) bank49491 +49492 POINT(30.651309649709912 -95.319420789386) bank49492 +49493 POINT(30.36192485492528 -96.00224965134167) bank49493 +49494 POINT(30.029643349970165 -94.41469586161685) bank49494 +49495 POINT(29.237537207025458 -95.29624463444868) bank49495 +49496 POINT(29.7068582021324 -94.82411343357865) bank49496 +49497 POINT(29.824150248711746 -95.32045613628853) bank49497 +49498 POINT(30.73292640630325 -95.9627344695615) bank49498 +49499 POINT(29.287222317683295 -94.6578079558399) bank49499 +49500 POINT(29.01800874013809 -95.04163836698915) bank49500 +49501 POINT(30.12717642119343 -96.00233215257713) bank49501 +49502 POINT(29.297152702531548 -94.38639390084116) bank49502 +49503 POINT(30.29203071952934 -94.69003931680533) bank49503 +49504 POINT(29.47571596850231 -95.79369690763966) bank49504 +49505 POINT(30.243792173796447 -94.78796866829225) bank49505 +49506 POINT(30.271708745061915 -95.93905544752327) bank49506 +49507 POINT(29.824198679084663 -96.3554572056843) bank49507 +49508 POINT(30.374818655227934 -94.71967833462689) bank49508 +49509 POINT(30.629013783601703 -96.2985677026393) bank49509 +49510 POINT(30.651547076096733 -94.73753568802914) bank49510 +49511 POINT(29.64237560726679 -95.68028822594513) bank49511 +49512 POINT(30.203431255874428 -95.65249216715362) bank49512 +49513 POINT(29.42449857049654 -96.30820027784704) bank49513 +49514 POINT(30.227490410030338 -95.9364950523997) bank49514 +49515 POINT(29.4320949051526 -95.57073020536934) bank49515 +49516 POINT(29.723928690227805 -94.83627905517348) bank49516 +49517 POINT(30.36952122679776 -95.92914927982454) bank49517 +49518 POINT(29.148735860151188 -94.76740894220626) bank49518 +49519 POINT(30.18371394220593 -94.85316133219365) bank49519 +49520 POINT(30.49340649011509 -95.09049129198024) bank49520 +49521 POINT(30.033828090390266 -95.25937936619493) bank49521 +49522 POINT(30.43941862515937 -95.86178746482555) bank49522 +49523 POINT(29.943325444207535 -95.92783502792545) bank49523 +49524 POINT(29.428720456748774 -95.46117920963066) bank49524 +49525 POINT(29.693240483250413 -95.59997076785045) bank49525 +49526 POINT(29.01891474076993 -96.29903813386201) bank49526 +49527 POINT(30.616769368942425 -95.02236607858607) bank49527 +49528 POINT(29.86672010789539 -96.18687203492314) bank49528 +49529 POINT(29.82505543874823 -96.07078171722917) bank49529 +49530 POINT(30.581628907956475 -95.99018089489138) bank49530 +49531 POINT(30.22672820048385 -95.79861499082152) bank49531 +49532 POINT(29.63154214814809 -95.99584570809878) bank49532 +49533 POINT(30.076681363705614 -95.8886704743873) bank49533 +49534 POINT(29.459846511006432 -95.16062101343267) bank49534 +49535 POINT(29.558352061923543 -96.04351638765235) bank49535 +49536 POINT(29.205957332027296 -95.14626315228791) bank49536 +49537 POINT(29.33001781012373 -95.4450616055772) bank49537 +49538 POINT(29.7337691536165 -94.8041054859711) bank49538 +49539 POINT(30.426176026947626 -94.9347678732587) bank49539 +49540 POINT(30.22511992787873 -95.33626253601486) bank49540 +49541 POINT(30.119648893816954 -95.93839554311457) bank49541 +49542 POINT(28.92127746545047 -94.56053083498519) bank49542 +49543 POINT(29.862840907367364 -96.04540750085721) bank49543 +49544 POINT(29.099158359491824 -95.010234292158) bank49544 +49545 POINT(28.898173990133966 -95.21636961450987) bank49545 +49546 POINT(29.673819738151096 -95.20388790649582) bank49546 +49547 POINT(30.09429755760275 -95.98603897479998) bank49547 +49548 POINT(29.750589177763462 -94.637621779963) bank49548 +49549 POINT(28.761971869858243 -96.20915238313596) bank49549 +49550 POINT(30.464413497691623 -96.07195960628222) bank49550 +49551 POINT(29.174003082953558 -94.52416395008701) bank49551 +49552 POINT(29.810169894239408 -95.2274544616401) bank49552 +49553 POINT(29.96225718301458 -94.51212212903242) bank49553 +49554 POINT(28.902503824705036 -95.16140094135999) bank49554 +49555 POINT(30.311089278225445 -95.94326549445022) bank49555 +49556 POINT(29.628473652473375 -94.41536806987445) bank49556 +49557 POINT(29.005255246869265 -94.48188859613448) bank49557 +49558 POINT(28.795902639505847 -95.53232958867233) bank49558 +49559 POINT(29.06756231157922 -96.05598923461258) bank49559 +49560 POINT(29.466275238070004 -94.67213074758088) bank49560 +49561 POINT(29.426576871353443 -95.30641965229593) bank49561 +49562 POINT(29.074756438719145 -96.23030974837049) bank49562 +49563 POINT(28.900929055136146 -96.33029131866961) bank49563 +49564 POINT(29.929217054447086 -95.36155427567618) bank49564 +49565 POINT(29.32401013950926 -95.69480617925107) bank49565 +49566 POINT(28.79443556910161 -94.45783277641321) bank49566 +49567 POINT(29.258344957140128 -95.87060982943991) bank49567 +49568 POINT(29.08032192838217 -95.29366178688859) bank49568 +49569 POINT(30.000944313285324 -94.56650426825665) bank49569 +49570 POINT(30.6652565554534 -96.19649629138524) bank49570 +49571 POINT(29.182701421164587 -95.09671699751408) bank49571 +49572 POINT(28.959817069918557 -95.57149498525946) bank49572 +49573 POINT(28.822487309273647 -94.38385355925409) bank49573 +49574 POINT(29.53721204371413 -95.58539332697801) bank49574 +49575 POINT(28.78858098892955 -96.0102277614575) bank49575 +49576 POINT(29.2672753967648 -95.1818325341342) bank49576 +49577 POINT(29.243077469494295 -95.16128523107749) bank49577 +49578 POINT(29.63747359073056 -94.53624313047631) bank49578 +49579 POINT(29.627941581035373 -94.46213412614982) bank49579 +49580 POINT(30.541741953524493 -94.76792309004848) bank49580 +49581 POINT(29.060734714485672 -94.85140957317594) bank49581 +49582 POINT(30.543007075796833 -96.18453544289346) bank49582 +49583 POINT(30.554219408727725 -95.12267184246272) bank49583 +49584 POINT(30.270291424002448 -96.28945396124053) bank49584 +49585 POINT(28.850880643793936 -95.62248682563222) bank49585 +49586 POINT(29.605746925671934 -95.51161198255454) bank49586 +49587 POINT(28.968414821352027 -96.17916882270681) bank49587 +49588 POINT(29.032577034308538 -95.56130427492873) bank49588 +49589 POINT(29.534960119929167 -95.73620942377627) bank49589 +49590 POINT(30.450216265733072 -94.82335979365917) bank49590 +49591 POINT(29.56148879892034 -94.48204344322637) bank49591 +49592 POINT(28.867965804762065 -95.5850708548648) bank49592 +49593 POINT(29.5143972852375 -95.61730786646183) bank49593 +49594 POINT(29.207058463377766 -95.09789671456262) bank49594 +49595 POINT(29.30555187324044 -95.00328395708605) bank49595 +49596 POINT(30.620838489837716 -95.28051701533455) bank49596 +49597 POINT(28.824441936154567 -96.1147560796514) bank49597 +49598 POINT(29.459430775847068 -96.25694556225231) bank49598 +49599 POINT(29.660225166567535 -95.39063217470924) bank49599 +49600 POINT(29.198635690519772 -95.95187182124207) bank49600 +49601 POINT(28.76975284189056 -95.44043941228287) bank49601 +49602 POINT(30.404127681879586 -94.53261688552932) bank49602 +49603 POINT(28.851077262084875 -94.97053888427367) bank49603 +49604 POINT(30.25636718480893 -96.21753104282678) bank49604 +49605 POINT(30.755472303789723 -96.01894309047331) bank49605 +49606 POINT(30.18539289033449 -94.60631266456019) bank49606 +49607 POINT(30.103657310187224 -95.24666417978875) bank49607 +49608 POINT(30.537763059240834 -96.13316301125367) bank49608 +49609 POINT(29.377415443144834 -95.83424253371477) bank49609 +49610 POINT(29.469511171150142 -96.11529665483413) bank49610 +49611 POINT(30.112033372087904 -95.50292475650498) bank49611 +49612 POINT(29.26974684061025 -95.44746966899916) bank49612 +49613 POINT(29.760011166225897 -94.48327248035362) bank49613 +49614 POINT(29.30940362573495 -95.25141098589306) bank49614 +49615 POINT(29.348900714988563 -95.75660362169323) bank49615 +49616 POINT(29.803866484892815 -95.43408618732762) bank49616 +49617 POINT(30.009660901575366 -94.67425423543968) bank49617 +49618 POINT(30.623605526396105 -95.65259964870633) bank49618 +49619 POINT(29.138624580594403 -95.93668096660562) bank49619 +49620 POINT(30.485257765786166 -94.96086739730754) bank49620 +49621 POINT(30.45514958269259 -94.89151020867638) bank49621 +49622 POINT(30.504740790996696 -95.38546575910087) bank49622 +49623 POINT(30.70017719151178 -95.2596762667833) bank49623 +49624 POINT(30.425052554868614 -94.50933847533916) bank49624 +49625 POINT(29.823824436791444 -95.99236995923816) bank49625 +49626 POINT(30.09440153447054 -95.38417292419987) bank49626 +49627 POINT(29.43034840027961 -95.39584702290237) bank49627 +49628 POINT(28.763352978633 -96.25877149949396) bank49628 +49629 POINT(29.63040227862315 -96.31901511951125) bank49629 +49630 POINT(29.80981430312502 -95.9170803030374) bank49630 +49631 POINT(30.061574451091037 -95.7621070732326) bank49631 +49632 POINT(29.022747881831155 -94.92950112227514) bank49632 +49633 POINT(30.183948359284724 -95.88142116906462) bank49633 +49634 POINT(30.74871795009171 -95.07898684815541) bank49634 +49635 POINT(30.113960951932906 -95.4083777187024) bank49635 +49636 POINT(29.883712433120117 -94.40710052915571) bank49636 +49637 POINT(29.0194882090862 -95.30149750620116) bank49637 +49638 POINT(29.89685758006036 -96.19603678906311) bank49638 +49639 POINT(30.188244697447114 -96.15984215595746) bank49639 +49640 POINT(29.08794692976451 -96.03125446774405) bank49640 +49641 POINT(28.98592938847278 -94.50135198703691) bank49641 +49642 POINT(30.70324029730194 -96.0826789660654) bank49642 +49643 POINT(30.71517241317337 -94.60252313217511) bank49643 +49644 POINT(29.976333983541906 -94.81834278447268) bank49644 +49645 POINT(30.384241771252217 -94.88152599974957) bank49645 +49646 POINT(29.62106474258693 -95.42313989916809) bank49646 +49647 POINT(29.466304047078374 -95.73181893037399) bank49647 +49648 POINT(30.255337115848572 -96.30220748726738) bank49648 +49649 POINT(29.31829267403318 -94.67054015454093) bank49649 +49650 POINT(28.805914505428596 -96.31739991368255) bank49650 +49651 POINT(29.330110228647822 -95.0437800820463) bank49651 +49652 POINT(28.88355243958028 -94.53145694581858) bank49652 +49653 POINT(29.26514003414326 -94.59181312851011) bank49653 +49654 POINT(29.469962498336727 -94.80413208661896) bank49654 +49655 POINT(30.713154356664337 -96.05280503784029) bank49655 +49656 POINT(28.83696190809971 -95.35938965479453) bank49656 +49657 POINT(30.75781533574544 -95.41478624683945) bank49657 +49658 POINT(28.82448273974873 -95.29084853255877) bank49658 +49659 POINT(29.22679944714498 -95.64415391103874) bank49659 +49660 POINT(29.46986131965103 -95.70935016262467) bank49660 +49661 POINT(29.886875885219677 -94.40844739681107) bank49661 +49662 POINT(30.57418997257919 -96.23943702752807) bank49662 +49663 POINT(30.038223077874274 -95.13710430911934) bank49663 +49664 POINT(29.173914639718866 -95.10814384635859) bank49664 +49665 POINT(28.845259261154546 -95.81776321546245) bank49665 +49666 POINT(30.45135495558452 -94.66727832544943) bank49666 +49667 POINT(29.54099506826295 -95.7868956615866) bank49667 +49668 POINT(28.983850623442763 -94.5861241489032) bank49668 +49669 POINT(30.34316738960668 -95.7007841962897) bank49669 +49670 POINT(29.381371849847365 -94.37008168375296) bank49670 +49671 POINT(30.715718908122675 -94.82853449642401) bank49671 +49672 POINT(29.667109809718294 -95.841003066554) bank49672 +49673 POINT(30.20882213691478 -94.52609722224504) bank49673 +49674 POINT(29.545644775718277 -95.58344319794553) bank49674 +49675 POINT(28.76059139091955 -95.85939214464126) bank49675 +49676 POINT(29.100278862181423 -95.75877733640912) bank49676 +49677 POINT(30.28468664338119 -95.23403331915723) bank49677 +49678 POINT(30.239755701420428 -95.79584652348551) bank49678 +49679 POINT(29.671687796831236 -95.19402966435003) bank49679 +49680 POINT(30.31697188633033 -95.30097665063684) bank49680 +49681 POINT(30.389234401521954 -95.04165169157683) bank49681 +49682 POINT(29.282236485573744 -96.11092402070929) bank49682 +49683 POINT(30.030850581219422 -94.45478350693121) bank49683 +49684 POINT(30.401682886882718 -96.3619015194013) bank49684 +49685 POINT(30.160654608301364 -96.14382546556466) bank49685 +49686 POINT(29.33888941467636 -95.9467160022547) bank49686 +49687 POINT(30.447200684086052 -96.12123501037583) bank49687 +49688 POINT(30.34081202257161 -95.63213999104983) bank49688 +49689 POINT(30.328483344146065 -95.64388776584894) bank49689 +49690 POINT(29.96790110162072 -94.62060267026021) bank49690 +49691 POINT(29.979662265376056 -95.04947785940118) bank49691 +49692 POINT(30.227142609921064 -95.67438133105385) bank49692 +49693 POINT(30.710725508435626 -95.17966414630133) bank49693 +49694 POINT(29.51905427353867 -95.70289334618894) bank49694 +49695 POINT(28.768709610738334 -95.548616879102) bank49695 +49696 POINT(29.602297291174324 -95.78659959543505) bank49696 +49697 POINT(30.116605810023117 -96.15946288988323) bank49697 +49698 POINT(28.81717863779297 -95.0424171061519) bank49698 +49699 POINT(28.945029235478692 -96.3295243722416) bank49699 +49700 POINT(30.42647334464541 -95.01641487337444) bank49700 +49701 POINT(29.24803228905332 -95.68109602183839) bank49701 +49702 POINT(29.5291018381224 -94.86258360457384) bank49702 +49703 POINT(30.258649691711714 -96.18071704890555) bank49703 +49704 POINT(29.859443724679235 -95.32086034693742) bank49704 +49705 POINT(29.971282020973447 -95.15684285407046) bank49705 +49706 POINT(29.517604283180543 -95.76958958972834) bank49706 +49707 POINT(29.4975729180659 -95.38116506256632) bank49707 +49708 POINT(29.58484457708043 -95.79392345995771) bank49708 +49709 POINT(30.233496189829665 -95.49314523449267) bank49709 +49710 POINT(28.888941981836464 -94.40354141499874) bank49710 +49711 POINT(29.757202010546198 -95.71112144636633) bank49711 +49712 POINT(29.71499069994317 -96.16572085818994) bank49712 +49713 POINT(29.581000365680154 -94.55776298980257) bank49713 +49714 POINT(30.2371218866284 -95.59912062368586) bank49714 +49715 POINT(30.129287367479197 -96.09092627658754) bank49715 +49716 POINT(30.592871856055474 -95.52426426021991) bank49716 +49717 POINT(29.80070077050311 -96.13468388378158) bank49717 +49718 POINT(29.5033915613926 -94.81094267138486) bank49718 +49719 POINT(29.70150458540666 -94.58209835070582) bank49719 +49720 POINT(30.70633628018929 -95.22045526664975) bank49720 +49721 POINT(29.620877178508422 -94.75238141739085) bank49721 +49722 POINT(30.518180835662147 -95.22975678505371) bank49722 +49723 POINT(29.841405060917783 -96.03898440004785) bank49723 +49724 POINT(30.488950387101777 -95.46244919828864) bank49724 +49725 POINT(29.197148761296848 -95.46426408619124) bank49725 +49726 POINT(29.52179148155936 -94.86874652002425) bank49726 +49727 POINT(28.8572629748656 -94.9373930727936) bank49727 +49728 POINT(29.934461878506863 -94.50525141706606) bank49728 +49729 POINT(30.06197723424856 -94.71786278410885) bank49729 +49730 POINT(30.51540219969183 -95.80463862000383) bank49730 +49731 POINT(29.24256983392971 -95.30817295876358) bank49731 +49732 POINT(29.806195492640285 -96.32978681229062) bank49732 +49733 POINT(30.62594624797202 -95.80906330113599) bank49733 +49734 POINT(30.12133272104395 -94.52302387913491) bank49734 +49735 POINT(30.61853787731733 -94.42889003336536) bank49735 +49736 POINT(30.06969875235521 -94.42128103262029) bank49736 +49737 POINT(30.16079979872595 -95.28604302143268) bank49737 +49738 POINT(29.431858525591725 -94.38896025908903) bank49738 +49739 POINT(30.561368343489796 -95.6867851859019) bank49739 +49740 POINT(30.348946146360035 -95.67978906040692) bank49740 +49741 POINT(29.108064309244224 -94.73999089159376) bank49741 +49742 POINT(30.141455288800202 -95.36748758568025) bank49742 +49743 POINT(29.45481108885535 -95.24476006514661) bank49743 +49744 POINT(29.46361522135203 -94.86838893706965) bank49744 +49745 POINT(29.548048420781562 -95.0444367685999) bank49745 +49746 POINT(29.622430881781 -95.05942651719114) bank49746 +49747 POINT(30.268308819054763 -95.01611170745639) bank49747 +49748 POINT(29.274298956834404 -95.86831362575143) bank49748 +49749 POINT(29.74743226406272 -95.85030369007467) bank49749 +49750 POINT(28.85134241544662 -95.21507462471449) bank49750 +49751 POINT(28.824991320619688 -95.6278483669405) bank49751 +49752 POINT(30.352393601499795 -95.38728107419664) bank49752 +49753 POINT(29.284706187446805 -96.29450155158311) bank49753 +49754 POINT(29.33645612716121 -95.3490748070026) bank49754 +49755 POINT(29.118287515508356 -94.7065601762355) bank49755 +49756 POINT(28.840504893525065 -96.29111385328783) bank49756 +49757 POINT(30.468057485067973 -96.35361472701548) bank49757 +49758 POINT(29.384296461523142 -96.1915701364155) bank49758 +49759 POINT(29.70700258945596 -94.90314533194974) bank49759 +49760 POINT(29.74856391159998 -95.75207177802558) bank49760 +49761 POINT(30.340882290827828 -96.30596456499997) bank49761 +49762 POINT(29.83598029224053 -95.17054756987837) bank49762 +49763 POINT(30.680705014552103 -96.13937988049832) bank49763 +49764 POINT(29.486316411139 -95.44658801807014) bank49764 +49765 POINT(29.707222200514188 -94.57216031455256) bank49765 +49766 POINT(29.59474611949865 -96.22567376505387) bank49766 +49767 POINT(30.443498384608475 -95.86211081642435) bank49767 +49768 POINT(29.886060043803287 -94.98656339026844) bank49768 +49769 POINT(29.127086593346736 -95.18858845794475) bank49769 +49770 POINT(30.554791768215388 -96.04714363229368) bank49770 +49771 POINT(29.05601160257426 -95.31049083490815) bank49771 +49772 POINT(29.489275125073924 -96.32320632540608) bank49772 +49773 POINT(29.957292981720396 -95.57896827369728) bank49773 +49774 POINT(28.933253397493054 -94.94850854913224) bank49774 +49775 POINT(29.662935562835447 -94.82876088095357) bank49775 +49776 POINT(28.9386500213755 -96.12078154400508) bank49776 +49777 POINT(29.58270787135037 -95.17144517866647) bank49777 +49778 POINT(29.369470144784856 -96.23921730177875) bank49778 +49779 POINT(28.990740469155387 -96.08149918968279) bank49779 +49780 POINT(29.5225036703382 -94.47007615226042) bank49780 +49781 POINT(30.15703649905153 -95.9191093256833) bank49781 +49782 POINT(30.704961967699198 -94.45859558666068) bank49782 +49783 POINT(28.789852565272447 -96.2024318696652) bank49783 +49784 POINT(29.254050531140777 -95.00842074883288) bank49784 +49785 POINT(28.844476562407948 -95.37936662188301) bank49785 +49786 POINT(29.5203295429661 -96.12418770467468) bank49786 +49787 POINT(30.510995181944462 -94.43321588395375) bank49787 +49788 POINT(29.79695877532203 -94.7180255613339) bank49788 +49789 POINT(30.317853408168137 -96.331092212968) bank49789 +49790 POINT(29.62350273881363 -95.20145923643875) bank49790 +49791 POINT(30.73218176038258 -95.93625571876669) bank49791 +49792 POINT(30.660983016654697 -95.54934819296113) bank49792 +49793 POINT(29.561354905883814 -96.1904619894053) bank49793 +49794 POINT(30.165929197809596 -94.87900996739057) bank49794 +49795 POINT(29.426953756988567 -95.91830205837248) bank49795 +49796 POINT(30.3960935677363 -94.9879707202312) bank49796 +49797 POINT(29.473286135361217 -94.81867555075928) bank49797 +49798 POINT(29.93778824956206 -95.43616213885714) bank49798 +49799 POINT(29.323945729967726 -95.49678920103004) bank49799 +49800 POINT(30.268148461628776 -95.06348644999925) bank49800 +49801 POINT(29.01050646534947 -94.39608323181845) bank49801 +49802 POINT(30.590748691186672 -96.02544542940055) bank49802 +49803 POINT(29.926893351943367 -94.71403947084886) bank49803 +49804 POINT(29.159508255047545 -95.70083579549454) bank49804 +49805 POINT(29.661157632138092 -94.47466483264463) bank49805 +49806 POINT(30.68849037541025 -94.56277383450869) bank49806 +49807 POINT(30.036913530684934 -96.32285034205853) bank49807 +49808 POINT(28.811835652131567 -96.03262775923422) bank49808 +49809 POINT(30.680706921069486 -96.21442929513726) bank49809 +49810 POINT(29.627106586349804 -95.00888981809729) bank49810 +49811 POINT(30.373758046087644 -94.56740444551167) bank49811 +49812 POINT(29.970109006877433 -95.69723583032152) bank49812 +49813 POINT(30.143125905295634 -95.82064181701274) bank49813 +49814 POINT(30.00086930647935 -95.93196810030645) bank49814 +49815 POINT(30.697712137741792 -94.56587863585534) bank49815 +49816 POINT(30.29180601793184 -94.40706678009683) bank49816 +49817 POINT(30.541732122268257 -95.73115103947215) bank49817 +49818 POINT(30.368884665960376 -94.69663517218842) bank49818 +49819 POINT(30.640199960165845 -94.49457399928956) bank49819 +49820 POINT(30.627229810858978 -94.87045191923755) bank49820 +49821 POINT(28.952005883206425 -95.78623553677586) bank49821 +49822 POINT(29.676645716741238 -94.46308906488747) bank49822 +49823 POINT(30.492887151503922 -95.95634713942002) bank49823 +49824 POINT(29.148584597699937 -95.44193665260404) bank49824 +49825 POINT(30.191289870265006 -96.00841244598723) bank49825 +49826 POINT(29.30394730203899 -95.56653265195361) bank49826 +49827 POINT(29.135691008034925 -94.49141098225311) bank49827 +49828 POINT(29.12073938432673 -95.26865602554308) bank49828 +49829 POINT(29.5496530909666 -95.7310860924939) bank49829 +49830 POINT(30.586248448600536 -96.14081356979361) bank49830 +49831 POINT(30.57349913326027 -94.39239919235911) bank49831 +49832 POINT(29.386369219849406 -95.5595383867591) bank49832 +49833 POINT(29.650616052914888 -96.08940157746788) bank49833 +49834 POINT(29.334763490562466 -96.34293583135933) bank49834 +49835 POINT(30.497624337330773 -95.82884885854158) bank49835 +49836 POINT(30.0589995306413 -95.80336472164923) bank49836 +49837 POINT(29.211620502686763 -95.33231752773189) bank49837 +49838 POINT(30.496623171237687 -95.51409323835526) bank49838 +49839 POINT(29.54621799092616 -95.77374143658386) bank49839 +49840 POINT(28.96045721388557 -95.89764337007608) bank49840 +49841 POINT(29.518371083932227 -96.08212514824413) bank49841 +49842 POINT(30.083611671011386 -95.10200260286032) bank49842 +49843 POINT(30.240876627927406 -96.36025098883927) bank49843 +49844 POINT(30.404540420127475 -95.71669297927168) bank49844 +49845 POINT(29.859444820333124 -95.2292378290473) bank49845 +49846 POINT(30.503594679437928 -95.33829900121164) bank49846 +49847 POINT(29.387088749071733 -96.34267211007247) bank49847 +49848 POINT(30.39503888902746 -95.28701652658764) bank49848 +49849 POINT(30.272411229436035 -94.3836742519932) bank49849 +49850 POINT(29.12707171090374 -94.78692631897086) bank49850 +49851 POINT(30.609383949715646 -95.64855659666729) bank49851 +49852 POINT(29.788683772458317 -94.54748628083833) bank49852 +49853 POINT(29.330813698961762 -96.07440756883327) bank49853 +49854 POINT(30.613957662333945 -96.3287860411265) bank49854 +49855 POINT(30.310471712315934 -95.80302506496434) bank49855 +49856 POINT(29.877625900583652 -95.15371395044421) bank49856 +49857 POINT(28.826715425958838 -95.91007310302552) bank49857 +49858 POINT(29.727781152134465 -95.77214749105421) bank49858 +49859 POINT(29.632994705638456 -95.57234888423793) bank49859 +49860 POINT(30.156788165343706 -95.00978050148639) bank49860 +49861 POINT(29.611025322427647 -95.04396849228205) bank49861 +49862 POINT(29.307443471229202 -94.69054620351791) bank49862 +49863 POINT(30.17197773954864 -95.5112188934178) bank49863 +49864 POINT(30.341053624064948 -95.88912474803358) bank49864 +49865 POINT(29.959661035637858 -96.28269395686787) bank49865 +49866 POINT(29.945760981430936 -95.98830721841543) bank49866 +49867 POINT(29.188458496829597 -96.2492651438555) bank49867 +49868 POINT(29.897954547568876 -95.77858887654114) bank49868 +49869 POINT(29.181165315456425 -95.55215391193924) bank49869 +49870 POINT(30.437061968485015 -95.83000046648357) bank49870 +49871 POINT(28.830373730128827 -95.01160075768239) bank49871 +49872 POINT(30.08894530658408 -95.9225235761702) bank49872 +49873 POINT(30.031997404583564 -94.70689833382136) bank49873 +49874 POINT(29.679782685038198 -96.28270397618937) bank49874 +49875 POINT(29.76701524327127 -95.10953150136646) bank49875 +49876 POINT(29.238417804508792 -94.99493378033019) bank49876 +49877 POINT(30.168673061791086 -95.51909127137085) bank49877 +49878 POINT(29.092646355762845 -94.56434562073291) bank49878 +49879 POINT(30.47195013311772 -94.63566350392237) bank49879 +49880 POINT(30.37776765266202 -96.19553297829508) bank49880 +49881 POINT(29.42189629306342 -95.2144435049163) bank49881 +49882 POINT(30.598025643686114 -95.15425347745655) bank49882 +49883 POINT(29.659090970221776 -94.50375233792741) bank49883 +49884 POINT(30.72609438419392 -95.42642476111718) bank49884 +49885 POINT(30.73774570291807 -94.93297763531577) bank49885 +49886 POINT(29.502467437310948 -94.79204022183323) bank49886 +49887 POINT(29.301864976592835 -95.74871909696701) bank49887 +49888 POINT(29.73081609848073 -95.53158086794356) bank49888 +49889 POINT(29.257038366266826 -94.83046612729902) bank49889 +49890 POINT(29.58676915897801 -95.90523706094268) bank49890 +49891 POINT(29.270395841195047 -95.98925216712257) bank49891 +49892 POINT(29.734459810083916 -95.3918087363501) bank49892 +49893 POINT(29.033017509720576 -95.2423220867846) bank49893 +49894 POINT(29.38210999018433 -95.61397366142421) bank49894 +49895 POINT(29.41588925622272 -96.27874883139778) bank49895 +49896 POINT(29.414987163715438 -95.13578740081256) bank49896 +49897 POINT(29.385543634687718 -94.78203550099602) bank49897 +49898 POINT(29.908170012492597 -95.09765413270097) bank49898 +49899 POINT(28.76221259844534 -94.49424742415486) bank49899 +49900 POINT(29.05876540266794 -96.23172958286821) bank49900 +49901 POINT(29.404122569009225 -95.70050435024977) bank49901 +49902 POINT(30.136724325292448 -95.78318807405412) bank49902 +49903 POINT(30.10012403176278 -94.37864332404757) bank49903 +49904 POINT(29.212703193228005 -95.59707750792683) bank49904 +49905 POINT(29.85472130865008 -94.50643949200682) bank49905 +49906 POINT(29.7928961708317 -95.90123785424068) bank49906 +49907 POINT(30.286103620224804 -94.71188678281435) bank49907 +49908 POINT(30.62058682011521 -96.04354074123576) bank49908 +49909 POINT(29.373420709178873 -94.93631922993113) bank49909 +49910 POINT(29.069565067129805 -95.41831154372174) bank49910 +49911 POINT(30.11918375989867 -95.53525651024125) bank49911 +49912 POINT(29.933837038855888 -95.36699285733465) bank49912 +49913 POINT(30.323081710034696 -94.49650084153265) bank49913 +49914 POINT(29.161022775013066 -95.57088254422509) bank49914 +49915 POINT(29.2858745997518 -95.84921384413366) bank49915 +49916 POINT(29.166029206972254 -95.84931784505959) bank49916 +49917 POINT(29.917362920646536 -95.66983567003577) bank49917 +49918 POINT(29.16136142447316 -95.94122971141154) bank49918 +49919 POINT(28.76974188775534 -95.60829792822273) bank49919 +49920 POINT(28.817355682065582 -94.37109453205812) bank49920 +49921 POINT(29.92367923345804 -95.60354359551182) bank49921 +49922 POINT(29.1512036017595 -96.34269290831264) bank49922 +49923 POINT(30.087753506068086 -94.53791797971249) bank49923 +49924 POINT(30.211609533648865 -96.30275521494744) bank49924 +49925 POINT(28.985700248319695 -95.6998423218643) bank49925 +49926 POINT(29.317028589229075 -95.05520028081435) bank49926 +49927 POINT(30.455481983178974 -94.47145735303147) bank49927 +49928 POINT(28.792701586212583 -95.66816065230734) bank49928 +49929 POINT(29.40921844187801 -95.54152815975772) bank49929 +49930 POINT(30.3886298345464 -96.30315298662285) bank49930 +49931 POINT(29.15215321670663 -96.19821683897442) bank49931 +49932 POINT(29.017047097875473 -95.7943111644789) bank49932 +49933 POINT(30.266992443175976 -96.04856418273707) bank49933 +49934 POINT(30.498195746164672 -95.16594983409234) bank49934 +49935 POINT(29.889450102664192 -95.03688408644953) bank49935 +49936 POINT(28.809055101285015 -95.03357433781566) bank49936 +49937 POINT(29.662551292457156 -94.66986127157747) bank49937 +49938 POINT(29.247090819017245 -94.97709069043309) bank49938 +49939 POINT(29.07440031672185 -95.52568886420525) bank49939 +49940 POINT(29.906724589559506 -96.35234834594041) bank49940 +49941 POINT(29.240547422257187 -95.19391988609364) bank49941 +49942 POINT(29.19068607135031 -95.75219838096069) bank49942 +49943 POINT(30.705441911198605 -94.62622214828109) bank49943 +49944 POINT(29.770538228839822 -96.086646185809) bank49944 +49945 POINT(29.58798380555347 -94.38458078154278) bank49945 +49946 POINT(29.764611873697497 -95.14664846858109) bank49946 +49947 POINT(30.6956301092865 -96.05743702869587) bank49947 +49948 POINT(29.808589626912447 -96.26102500453406) bank49948 +49949 POINT(30.20695677906512 -94.55864158808704) bank49949 +49950 POINT(29.189999013985844 -94.98873932628884) bank49950 +49951 POINT(30.001330822598756 -95.66545478814203) bank49951 +49952 POINT(29.23195060093126 -94.9572013341912) bank49952 +49953 POINT(28.838394945396583 -96.35556794744947) bank49953 +49954 POINT(29.97088328526805 -95.68023835211906) bank49954 +49955 POINT(30.55452072288799 -95.15908649093524) bank49955 +49956 POINT(29.816290973029606 -95.05524665631361) bank49956 +49957 POINT(30.737813407051615 -95.11543977931771) bank49957 +49958 POINT(30.193077622635613 -95.39317646741958) bank49958 +49959 POINT(29.25293420877582 -95.72109682512892) bank49959 +49960 POINT(30.618714964235703 -96.17694373871042) bank49960 +49961 POINT(29.86117172258418 -95.03085284002375) bank49961 +49962 POINT(28.776339812491436 -95.19265129331491) bank49962 +49963 POINT(30.69090645839154 -95.45903234973322) bank49963 +49964 POINT(29.877640719622317 -94.99634565849371) bank49964 +49965 POINT(29.829114139804844 -95.95672001672914) bank49965 +49966 POINT(29.230174174932337 -95.50468212572032) bank49966 +49967 POINT(29.072593663307444 -95.08094955396419) bank49967 +49968 POINT(29.14900935444724 -94.95622139537664) bank49968 +49969 POINT(29.578305835855286 -94.44417398230337) bank49969 +49970 POINT(30.556837707898325 -95.05232433066827) bank49970 +49971 POINT(30.209200869456076 -95.08537178234842) bank49971 +49972 POINT(29.295587167031997 -94.60494327062331) bank49972 +49973 POINT(30.30647047256735 -96.06387881770057) bank49973 +49974 POINT(29.863661837929833 -94.58839471435049) bank49974 +49975 POINT(30.010610037394333 -94.8996323242628) bank49975 +49976 POINT(29.9391608671174 -94.86249460328969) bank49976 +49977 POINT(30.122074028528534 -95.76179197567679) bank49977 +49978 POINT(29.01687921341859 -95.35471655769811) bank49978 +49979 POINT(29.84637001924505 -94.70886792551084) bank49979 +49980 POINT(29.573521397355663 -94.95433429580842) bank49980 +49981 POINT(29.83952464362799 -94.86425592865751) bank49981 +49982 POINT(30.077656607084617 -94.84207757074343) bank49982 +49983 POINT(30.05594983695126 -94.98892427817455) bank49983 +49984 POINT(28.96188684783452 -95.8999803803434) bank49984 +49985 POINT(30.485099506230785 -96.0391696692844) bank49985 +49986 POINT(30.586571680322056 -95.41111848186092) bank49986 +49987 POINT(29.477412551313737 -95.7563189846922) bank49987 +49988 POINT(30.419303036862498 -95.57323641877433) bank49988 +49989 POINT(30.60805739207375 -95.29389681728652) bank49989 +49990 POINT(28.79699624844847 -95.27105204343064) bank49990 +49991 POINT(28.927261248288975 -95.13120447622823) bank49991 +49992 POINT(28.822444152831327 -95.58723734951525) bank49992 +49993 POINT(30.40380458647563 -95.09059917280081) bank49993 +49994 POINT(29.556093348861 -95.3378064356405) bank49994 +49995 POINT(30.13070019456836 -94.51061098097838) bank49995 +49996 POINT(29.555436447291623 -94.4804558751675) bank49996 +49997 POINT(29.697243319509393 -95.8189568804895) bank49997 +49998 POINT(30.722246883019867 -96.25179495340223) bank49998 +49999 POINT(30.736594776260418 -95.49904262852537) bank49999 +50000 POINT(30.690238270669912 -95.50266336786737) bank50000 +50001 POINT(29.098823103461385 -95.76861786593823) bank50001 +50002 POINT(29.794274732771324 -95.45742673949073) bank50002 +50003 POINT(30.733380969642983 -95.28591899332669) bank50003 +50004 POINT(30.54679918140628 -95.20792813168302) bank50004 +50005 POINT(28.943652775489696 -95.64747896769194) bank50005 +50006 POINT(30.367897457498263 -95.38081954489631) bank50006 +50007 POINT(30.609888670289735 -95.41654549098855) bank50007 +50008 POINT(29.44671490582963 -95.68344299954151) bank50008 +50009 POINT(30.22078042122138 -94.48582722859706) bank50009 +50010 POINT(30.40655429517241 -95.54956353532128) bank50010 +50011 POINT(29.620518532474033 -94.7433481310083) bank50011 +50012 POINT(30.5030929222953 -94.98168027891312) bank50012 +50013 POINT(29.024686183402558 -95.00848618139072) bank50013 +50014 POINT(28.769314739110794 -94.42615709574677) bank50014 +50015 POINT(29.65143916997077 -96.16224686743823) bank50015 +50016 POINT(29.16085118629824 -94.72590709398207) bank50016 +50017 POINT(29.039768739874546 -94.6326584992706) bank50017 +50018 POINT(29.68839812518265 -95.74984450468531) bank50018 +50019 POINT(30.2939440418274 -94.49017333858963) bank50019 +50020 POINT(29.437547949038326 -94.93461130364655) bank50020 +50021 POINT(29.21794861697521 -94.63188892809612) bank50021 +50022 POINT(29.852102949361154 -95.24670479158627) bank50022 +50023 POINT(29.351542661485922 -94.64524124177463) bank50023 +50024 POINT(29.57792973050636 -95.41377443369112) bank50024 +50025 POINT(29.506636967008713 -95.81997798894243) bank50025 +50026 POINT(30.756221634415674 -94.73459032654368) bank50026 +50027 POINT(30.280260641072292 -96.24221031779007) bank50027 +50028 POINT(29.53830201969943 -94.52458387947041) bank50028 +50029 POINT(29.587208606457587 -95.47417165740062) bank50029 +50030 POINT(30.469695333678803 -94.74257742729388) bank50030 +50031 POINT(28.848655464384265 -95.42393421033421) bank50031 +50032 POINT(29.159380791820237 -94.87901882968893) bank50032 +50033 POINT(29.222462299693362 -94.49836772011535) bank50033 +50034 POINT(30.150223010374248 -95.04587515287866) bank50034 +50035 POINT(30.136468200512304 -94.60567189035086) bank50035 +50036 POINT(30.350124182017005 -96.08259886550283) bank50036 +50037 POINT(29.883622341395473 -96.30920422903228) bank50037 +50038 POINT(30.732857534681635 -95.08640295405696) bank50038 +50039 POINT(30.181124807520067 -94.70187565060479) bank50039 +50040 POINT(29.218141282882225 -94.48904816381187) bank50040 +50041 POINT(29.337012448099962 -95.6837263695697) bank50041 +50042 POINT(30.376693287727086 -96.3030250664899) bank50042 +50043 POINT(30.27167536659402 -94.78587856024464) bank50043 +50044 POINT(29.156769835831884 -94.4394634008551) bank50044 +50045 POINT(30.22159708635141 -95.07873104741999) bank50045 +50046 POINT(30.031929410354856 -94.45269023282842) bank50046 +50047 POINT(29.790446890340334 -95.35836191830043) bank50047 +50048 POINT(30.124163873025346 -96.26028416112587) bank50048 +50049 POINT(30.28038382417414 -95.38990040836622) bank50049 +50050 POINT(30.392341155791446 -94.77123441316493) bank50050 +50051 POINT(30.269199440554925 -94.81224135421047) bank50051 +50052 POINT(30.749400788737226 -95.07325064103897) bank50052 +50053 POINT(29.56700222979515 -95.18886699138848) bank50053 +50054 POINT(30.459934659015847 -94.94065125853514) bank50054 +50055 POINT(30.217245978862547 -95.49547162473695) bank50055 +50056 POINT(29.11421675415781 -94.562608004786) bank50056 +50057 POINT(29.70524130185664 -95.39009244579542) bank50057 +50058 POINT(29.410019279344677 -94.83605705610874) bank50058 +50059 POINT(29.740287034232875 -95.6298896434619) bank50059 +50060 POINT(29.575562163650076 -95.9475602002864) bank50060 +50061 POINT(30.440873524717603 -94.55986293475338) bank50061 +50062 POINT(30.6307051079654 -94.4725158135446) bank50062 +50063 POINT(30.030719554991837 -95.50046749095769) bank50063 +50064 POINT(29.21097332149009 -95.73893673579512) bank50064 +50065 POINT(29.23833974646034 -95.79297740687431) bank50065 +50066 POINT(30.09450032014452 -94.95198968584612) bank50066 +50067 POINT(29.845441326319598 -95.58364859437508) bank50067 +50068 POINT(30.29903349454844 -96.3024874933373) bank50068 +50069 POINT(30.607628467810116 -94.48397233055188) bank50069 +50070 POINT(30.13175399568793 -96.16918478102639) bank50070 +50071 POINT(28.995866675066964 -94.42112482751618) bank50071 +50072 POINT(30.708663760979782 -95.43766323265986) bank50072 +50073 POINT(29.978050661530812 -96.16008845237002) bank50073 +50074 POINT(29.980552955735927 -96.26369001303061) bank50074 +50075 POINT(30.694590888660674 -95.54874042866062) bank50075 +50076 POINT(30.1061836401571 -95.45624916647766) bank50076 +50077 POINT(30.54871285926912 -95.39744412175611) bank50077 +50078 POINT(30.31589573563753 -95.28263619779632) bank50078 +50079 POINT(28.837889666074158 -95.58893489388058) bank50079 +50080 POINT(30.39292724247786 -96.17685951476183) bank50080 +50081 POINT(30.45117885766407 -95.71605975561872) bank50081 +50082 POINT(29.293862079957545 -95.41830490594663) bank50082 +50083 POINT(30.58776899423803 -95.6480901013415) bank50083 +50084 POINT(29.833845024418537 -94.71839815656836) bank50084 +50085 POINT(29.64222052769225 -95.94661813043574) bank50085 +50086 POINT(30.28275857990822 -95.79142012962184) bank50086 +50087 POINT(29.38540018974289 -96.32290343603286) bank50087 +50088 POINT(30.60301091671741 -95.76387991774153) bank50088 +50089 POINT(30.384891671412976 -95.88883511335625) bank50089 +50090 POINT(30.407037645205744 -95.8669291185709) bank50090 +50091 POINT(30.357547637710905 -95.53504677354445) bank50091 +50092 POINT(29.96196400383805 -94.68175847300485) bank50092 +50093 POINT(30.52603951002169 -95.03376381867658) bank50093 +50094 POINT(30.305997713938066 -94.87617705760815) bank50094 +50095 POINT(30.04340142952554 -95.63639923834346) bank50095 +50096 POINT(29.516113633372623 -95.84359467519894) bank50096 +50097 POINT(30.628578402933176 -94.41778176562751) bank50097 +50098 POINT(30.558904211750082 -95.61758824156558) bank50098 +50099 POINT(28.937958530748478 -95.18601989369633) bank50099 +50100 POINT(28.87331653180627 -95.27558150817083) bank50100 +50101 POINT(29.515087573978477 -95.04097486955745) bank50101 +50102 POINT(30.651983389063517 -95.23171526490646) bank50102 +50103 POINT(29.02972572727591 -94.58663286511326) bank50103 +50104 POINT(29.92783354960051 -95.84123982894359) bank50104 +50105 POINT(30.18975688586849 -94.99898359712593) bank50105 +50106 POINT(28.79024719238117 -94.54416419626492) bank50106 +50107 POINT(29.8794696144716 -95.62066882180775) bank50107 +50108 POINT(29.20559137836401 -96.06161933335318) bank50108 +50109 POINT(29.988797845465623 -94.57650066577176) bank50109 +50110 POINT(30.294016120166248 -95.66209234278465) bank50110 +50111 POINT(29.184134007773125 -94.5233570474049) bank50111 +50112 POINT(28.992121545053987 -95.40174886208692) bank50112 +50113 POINT(29.467752200257618 -95.41639594565508) bank50113 +50114 POINT(30.30680996755758 -95.19385402397293) bank50114 +50115 POINT(29.951624670792015 -94.58537032076919) bank50115 +50116 POINT(28.89516934458167 -95.26558306129343) bank50116 +50117 POINT(30.70120327410546 -94.47075045642607) bank50117 +50118 POINT(29.604903077197356 -94.6939089390348) bank50118 +50119 POINT(30.222076076766403 -94.66351712858832) bank50119 +50120 POINT(29.004880296114116 -95.51591492882162) bank50120 +50121 POINT(29.579847072938307 -95.47614106902105) bank50121 +50122 POINT(29.783852329349738 -95.9904002940666) bank50122 +50123 POINT(30.171318120022498 -95.11126718097069) bank50123 +50124 POINT(29.53094257967305 -95.87356812132937) bank50124 +50125 POINT(29.825714087975342 -96.33428451166628) bank50125 +50126 POINT(29.34870141601319 -96.25390051731023) bank50126 +50127 POINT(29.086809014042522 -95.18481295504056) bank50127 +50128 POINT(30.25820160377378 -95.89201016539637) bank50128 +50129 POINT(30.502578732664794 -94.5192699668994) bank50129 +50130 POINT(29.785023631289135 -95.86141166117528) bank50130 +50131 POINT(30.271644980080527 -95.16667171089749) bank50131 +50132 POINT(29.978538196429156 -95.78778645696052) bank50132 +50133 POINT(29.512977423684656 -94.82139024777787) bank50133 +50134 POINT(29.933435120973186 -96.06281638472255) bank50134 +50135 POINT(30.532824767588767 -96.29149640611608) bank50135 +50136 POINT(29.625188657270353 -95.36777087670079) bank50136 +50137 POINT(28.797289129817557 -95.08278719540368) bank50137 +50138 POINT(28.832767248354344 -94.94743942587822) bank50138 +50139 POINT(30.21200440894564 -95.39081062854926) bank50139 +50140 POINT(29.40217357717495 -95.17597190157468) bank50140 +50141 POINT(30.382508411202142 -96.05912948742021) bank50141 +50142 POINT(28.811415085878746 -95.14877574769996) bank50142 +50143 POINT(29.553260323953406 -96.32126472438856) bank50143 +50144 POINT(29.565623344751387 -96.35490557817727) bank50144 +50145 POINT(28.798100872976864 -94.60925485334776) bank50145 +50146 POINT(30.17879517301251 -94.57201050610712) bank50146 +50147 POINT(29.938644662904615 -94.44214626032397) bank50147 +50148 POINT(29.536158450215908 -95.78159344279416) bank50148 +50149 POINT(29.9275117578128 -95.23185361462906) bank50149 +50150 POINT(30.028887814482136 -95.70342867629856) bank50150 +50151 POINT(30.505221446440075 -94.53492464566594) bank50151 +50152 POINT(28.923235360782147 -95.52066536893027) bank50152 +50153 POINT(29.752491432889308 -96.36421284140823) bank50153 +50154 POINT(30.40218144072491 -95.32279455391077) bank50154 +50155 POINT(30.16244794227574 -94.68541333471794) bank50155 +50156 POINT(29.093636868959415 -95.80607169833041) bank50156 +50157 POINT(30.408892549436537 -95.75826941888813) bank50157 +50158 POINT(30.74969137890733 -94.97280852113508) bank50158 +50159 POINT(29.03414235289099 -95.05536960683662) bank50159 +50160 POINT(29.53398382954869 -95.42807956034163) bank50160 +50161 POINT(30.306375563840266 -94.98471920429232) bank50161 +50162 POINT(30.299960648228726 -95.00426908378596) bank50162 +50163 POINT(29.295540890179414 -95.48014503085231) bank50163 +50164 POINT(29.5472883368596 -96.36936875666976) bank50164 +50165 POINT(29.158194000364695 -94.4522013947811) bank50165 +50166 POINT(30.704924232128487 -95.48434517451408) bank50166 +50167 POINT(30.062160131151355 -94.92214381790644) bank50167 +50168 POINT(29.444101825644953 -95.19760743027697) bank50168 +50169 POINT(29.22317289939604 -96.03396842282243) bank50169 +50170 POINT(30.074035300430957 -95.1207241938894) bank50170 +50171 POINT(30.47986118315546 -95.08984174184502) bank50171 +50172 POINT(28.872249887464314 -96.25986131534282) bank50172 +50173 POINT(30.748832573555365 -94.66277959660175) bank50173 +50174 POINT(28.863667330605065 -94.70054545875891) bank50174 +50175 POINT(29.4988347318973 -95.2990489345434) bank50175 +50176 POINT(29.11095376662748 -94.8870271719239) bank50176 +50177 POINT(30.254639193693937 -96.2457070801182) bank50177 +50178 POINT(29.9539717653872 -95.0437765516916) bank50178 +50179 POINT(28.902037739382347 -95.76215694922246) bank50179 +50180 POINT(30.389923151986068 -95.97395058718865) bank50180 +50181 POINT(29.837777507951984 -94.50823987500641) bank50181 +50182 POINT(29.484764845251025 -96.15725449711266) bank50182 +50183 POINT(29.508621941649523 -95.91120959253792) bank50183 +50184 POINT(29.43253039288576 -94.7693786222099) bank50184 +50185 POINT(29.862948870198615 -94.51354619871988) bank50185 +50186 POINT(29.573338917332233 -96.31593049037933) bank50186 +50187 POINT(30.30789740913469 -95.21123011991382) bank50187 +50188 POINT(29.69288441946601 -96.21113950665979) bank50188 +50189 POINT(30.467471338657248 -95.91083486495654) bank50189 +50190 POINT(30.28647918538736 -94.47617868815578) bank50190 +50191 POINT(29.34694688258463 -94.79442738815953) bank50191 +50192 POINT(30.038572271560298 -94.82024919858797) bank50192 +50193 POINT(28.876962579840807 -94.7510203034723) bank50193 +50194 POINT(30.383955757772593 -96.02230091431231) bank50194 +50195 POINT(30.717792619658898 -95.30320505475306) bank50195 +50196 POINT(30.55252501513708 -94.55052891311225) bank50196 +50197 POINT(30.483872500678665 -95.78147134678652) bank50197 +50198 POINT(30.5083314587201 -94.56350296569471) bank50198 +50199 POINT(30.23190030319185 -94.88481447333385) bank50199 +50200 POINT(28.948834604315238 -94.7951983554148) bank50200 +50201 POINT(29.325528058004615 -95.12462179616037) bank50201 +50202 POINT(30.34524031540983 -95.27402256248304) bank50202 +50203 POINT(28.799437906942394 -95.36900910240507) bank50203 +50204 POINT(29.625251049069345 -95.85510870319142) bank50204 +50205 POINT(30.013452661752225 -94.79035305725313) bank50205 +50206 POINT(29.393020495368823 -95.59233865311398) bank50206 +50207 POINT(30.162433542005907 -94.62153060363623) bank50207 +50208 POINT(30.671471887748922 -96.04260049356176) bank50208 +50209 POINT(29.434995583110826 -94.59388117319178) bank50209 +50210 POINT(29.270495902474533 -94.56137513073911) bank50210 +50211 POINT(30.665548655524027 -94.87495482996798) bank50211 +50212 POINT(30.232594341481555 -96.05682671403876) bank50212 +50213 POINT(30.417756867734283 -94.44047468744827) bank50213 +50214 POINT(30.251718217467992 -94.99679306983633) bank50214 +50215 POINT(30.3061152547065 -95.1562403046504) bank50215 +50216 POINT(28.947497212008194 -94.84548926063918) bank50216 +50217 POINT(29.1282456371158 -94.93900364624895) bank50217 +50218 POINT(29.67694700205591 -96.16179156537913) bank50218 +50219 POINT(29.788987533767738 -95.71909849458801) bank50219 +50220 POINT(30.26743512740256 -95.45078247241388) bank50220 +50221 POINT(29.516421663680777 -94.8122803768684) bank50221 +50222 POINT(28.961240952503314 -96.25452782161186) bank50222 +50223 POINT(28.835800976799092 -96.25375266189809) bank50223 +50224 POINT(30.224219012956674 -95.4559716726371) bank50224 +50225 POINT(29.837707358987878 -95.95080463780886) bank50225 +50226 POINT(30.157932456486424 -95.35735432216988) bank50226 +50227 POINT(29.664425677219015 -94.96620086672816) bank50227 +50228 POINT(30.6212999570516 -96.24699842594082) bank50228 +50229 POINT(30.753258844510224 -95.70110785644691) bank50229 +50230 POINT(30.741286810514033 -95.5366693068693) bank50230 +50231 POINT(29.87139642044292 -95.24279007663192) bank50231 +50232 POINT(30.364197032530097 -94.41770666776382) bank50232 +50233 POINT(30.137660484538532 -95.28397840555033) bank50233 +50234 POINT(28.819888068782166 -95.204480430927) bank50234 +50235 POINT(29.917140074814277 -94.96583494845832) bank50235 +50236 POINT(29.622065235091778 -96.07230844797357) bank50236 +50237 POINT(29.73241197220766 -96.21744765518208) bank50237 +50238 POINT(30.396326002986143 -94.57954753141735) bank50238 +50239 POINT(30.546114040714077 -95.2373362423454) bank50239 +50240 POINT(28.87627037204928 -95.79651888605395) bank50240 +50241 POINT(29.65673622903222 -95.24257847163983) bank50241 +50242 POINT(29.982488829554086 -95.2553173131638) bank50242 +50243 POINT(29.018382065567337 -94.67634216797579) bank50243 +50244 POINT(29.831925905827266 -94.66525020493064) bank50244 +50245 POINT(29.778171567458603 -96.20501750115606) bank50245 +50246 POINT(30.5372396678949 -95.77387530394982) bank50246 +50247 POINT(28.824573876788563 -95.07942494126883) bank50247 +50248 POINT(29.59276491368503 -96.03112736736965) bank50248 +50249 POINT(29.99110062633597 -94.8742404904571) bank50249 +50250 POINT(29.60635867300565 -95.75595802190597) bank50250 +50251 POINT(30.644623895867095 -94.3746207146563) bank50251 +50252 POINT(30.702738467762188 -94.83030482704594) bank50252 +50253 POINT(28.795305281451427 -95.65767896294321) bank50253 +50254 POINT(29.554155613239445 -94.69946762487413) bank50254 +50255 POINT(30.43835036416065 -94.79603038638686) bank50255 +50256 POINT(29.69963001034713 -94.39919460300202) bank50256 +50257 POINT(30.139763193761844 -94.78657957416112) bank50257 +50258 POINT(30.114784654546863 -94.460608454235) bank50258 +50259 POINT(30.153992357679005 -96.20738485058172) bank50259 +50260 POINT(30.220589284031657 -95.4830316069344) bank50260 +50261 POINT(29.123648594068143 -94.50814396216363) bank50261 +50262 POINT(28.961090149905967 -94.66414095166199) bank50262 +50263 POINT(30.19969641843784 -95.68876199657748) bank50263 +50264 POINT(28.944863152614587 -96.32282421713764) bank50264 +50265 POINT(29.254468052039716 -95.81442207667753) bank50265 +50266 POINT(29.8192816763008 -96.13603639826466) bank50266 +50267 POINT(29.58891798243382 -95.33615561521896) bank50267 +50268 POINT(28.865288696235357 -94.52228448492222) bank50268 +50269 POINT(30.545771079085032 -94.53821045340813) bank50269 +50270 POINT(30.127134118979445 -94.58110976143523) bank50270 +50271 POINT(29.139924435266437 -94.43965887959317) bank50271 +50272 POINT(30.11424768524234 -96.22326495735888) bank50272 +50273 POINT(30.543547123845403 -94.92956127629422) bank50273 +50274 POINT(29.339834845812014 -94.49757837412206) bank50274 +50275 POINT(29.70500778150293 -95.21164486756902) bank50275 +50276 POINT(30.014602213385672 -94.69341724383635) bank50276 +50277 POINT(29.786598567487598 -96.15243877537834) bank50277 +50278 POINT(30.285441646808096 -95.43740179757258) bank50278 +50279 POINT(30.068246157414162 -94.44852478565485) bank50279 +50280 POINT(30.510366423733608 -94.57894417710816) bank50280 +50281 POINT(29.47025435642927 -94.54941853005997) bank50281 +50282 POINT(30.30033144388905 -95.51619611564391) bank50282 +50283 POINT(29.584908755914157 -94.51310584315637) bank50283 +50284 POINT(29.467324430985556 -96.13480515503778) bank50284 +50285 POINT(28.77058743685909 -94.6066520092052) bank50285 +50286 POINT(30.668138028268306 -95.38384879026543) bank50286 +50287 POINT(29.80114835685553 -94.71243391970813) bank50287 +50288 POINT(29.99445924459919 -94.59118560322581) bank50288 +50289 POINT(30.46270105192776 -94.89793585096014) bank50289 +50290 POINT(29.42228739446154 -95.14102044187699) bank50290 +50291 POINT(28.962771464470574 -95.80667998979938) bank50291 +50292 POINT(29.901615008111055 -96.00580005480289) bank50292 +50293 POINT(29.64504401358367 -94.97112249840275) bank50293 +50294 POINT(29.162239740491668 -95.66993878826257) bank50294 +50295 POINT(29.220316053597774 -95.17256984665201) bank50295 +50296 POINT(29.369138721690923 -95.2355182362736) bank50296 +50297 POINT(30.11111222208861 -96.00551434795264) bank50297 +50298 POINT(28.98918819928052 -94.86808167226128) bank50298 +50299 POINT(29.758293429582913 -96.3418062546173) bank50299 +50300 POINT(30.322537906845163 -94.88085581781901) bank50300 +50301 POINT(29.747679692256845 -95.28770537875438) bank50301 +50302 POINT(29.771391810726065 -96.2621015055831) bank50302 +50303 POINT(28.85078600209461 -95.831158364584) bank50303 +50304 POINT(29.59875768930747 -96.28897023558649) bank50304 +50305 POINT(29.38527156216764 -95.22225698984728) bank50305 +50306 POINT(29.727541529389608 -94.78706414391691) bank50306 +50307 POINT(30.617970349927983 -96.35742523221477) bank50307 +50308 POINT(30.007835651956963 -95.80438772379873) bank50308 +50309 POINT(30.315665021079706 -94.6052841784193) bank50309 +50310 POINT(30.26867004788867 -94.4665002216703) bank50310 +50311 POINT(28.836194808610525 -95.64569691446074) bank50311 +50312 POINT(29.416874086024446 -96.09178757410893) bank50312 +50313 POINT(30.494475971311783 -96.366393267011) bank50313 +50314 POINT(30.394513403151535 -94.63331871617726) bank50314 +50315 POINT(29.256538616301853 -94.83994061726217) bank50315 +50316 POINT(30.19471828253921 -95.16273139591648) bank50316 +50317 POINT(29.397904943798352 -94.49883319489821) bank50317 +50318 POINT(29.370815039032554 -95.30609141982238) bank50318 +50319 POINT(28.815389409767562 -94.70285172219714) bank50319 +50320 POINT(30.322320606589823 -95.50503663839265) bank50320 +50321 POINT(29.44143503079815 -95.4930619935345) bank50321 +50322 POINT(29.7065753996203 -96.31484914794028) bank50322 +50323 POINT(29.946594784641732 -95.08840445915337) bank50323 +50324 POINT(30.668642827751682 -95.2113458346422) bank50324 +50325 POINT(29.229403838677847 -95.84211532245341) bank50325 +50326 POINT(29.675905388799023 -95.70288339455978) bank50326 +50327 POINT(30.114095705200423 -94.91714223398793) bank50327 +50328 POINT(29.09625677582673 -95.4811168950821) bank50328 +50329 POINT(29.793919050073228 -95.32831799500478) bank50329 +50330 POINT(29.150911304325085 -94.57839774930896) bank50330 +50331 POINT(30.57285098114886 -94.9610952869874) bank50331 +50332 POINT(29.408909114717392 -96.11908741822229) bank50332 +50333 POINT(28.923716010653028 -95.29273882028266) bank50333 +50334 POINT(29.57579852738185 -96.08043435056732) bank50334 +50335 POINT(30.539464144049365 -95.40078815401466) bank50335 +50336 POINT(28.88871715119715 -95.77295910787748) bank50336 +50337 POINT(28.885029389235772 -95.8142192963529) bank50337 +50338 POINT(30.122860897724273 -94.37717429242375) bank50338 +50339 POINT(30.43351741773572 -96.1294086697658) bank50339 +50340 POINT(29.966350707270692 -95.29853777639765) bank50340 +50341 POINT(28.83097618999936 -95.13093301028783) bank50341 +50342 POINT(30.743668666395568 -94.71817766116833) bank50342 +50343 POINT(29.533958923589218 -96.3685885517434) bank50343 +50344 POINT(30.525082041254535 -95.86894852216693) bank50344 +50345 POINT(29.257490029423636 -95.94280022226408) bank50345 +50346 POINT(29.93098462475165 -94.50350184664113) bank50346 +50347 POINT(30.345691076493384 -94.6110669348655) bank50347 +50348 POINT(29.93690597463902 -95.8260902219114) bank50348 +50349 POINT(28.955151850823185 -96.36152055117424) bank50349 +50350 POINT(29.754456548047315 -94.52637723932644) bank50350 +50351 POINT(30.426639403544986 -95.83207528034565) bank50351 +50352 POINT(30.323913097939283 -94.54238970962746) bank50352 +50353 POINT(30.38705090908581 -96.15811024082636) bank50353 +50354 POINT(30.662391402283493 -96.13225487312599) bank50354 +50355 POINT(29.85501176856832 -94.41297516456139) bank50355 +50356 POINT(29.737511419372115 -95.79053865977238) bank50356 +50357 POINT(29.659286934771675 -96.29879421100034) bank50357 +50358 POINT(29.388764220623568 -94.43141048567819) bank50358 +50359 POINT(30.660762622779284 -95.9184086368812) bank50359 +50360 POINT(30.401780497670703 -95.5360839253627) bank50360 +50361 POINT(29.30193412116071 -96.2314278147416) bank50361 +50362 POINT(29.140548371302444 -95.84585181923244) bank50362 +50363 POINT(29.08867080058085 -94.74456818479187) bank50363 +50364 POINT(29.19669683607027 -95.62454570177502) bank50364 +50365 POINT(29.910346745964414 -95.57942821308323) bank50365 +50366 POINT(30.72427314756418 -95.65823128392198) bank50366 +50367 POINT(28.870314929422225 -94.63587939028119) bank50367 +50368 POINT(29.849655093994407 -94.97441632335469) bank50368 +50369 POINT(28.985229413457162 -94.83033418268838) bank50369 +50370 POINT(29.55772717505424 -96.3436780326298) bank50370 +50371 POINT(29.882622224254554 -95.617973589446) bank50371 +50372 POINT(30.143130262747277 -94.50580154212798) bank50372 +50373 POINT(29.119213871633416 -94.73372255483415) bank50373 +50374 POINT(30.148686429430388 -96.36709128158427) bank50374 +50375 POINT(30.283225086329423 -95.75455870188873) bank50375 +50376 POINT(29.22193730773848 -94.94608458558685) bank50376 +50377 POINT(30.087445888372798 -94.48373299492661) bank50377 +50378 POINT(29.163604497339566 -96.35201123036877) bank50378 +50379 POINT(30.11448141685867 -94.39058976978683) bank50379 +50380 POINT(29.2766280010866 -96.33948701431443) bank50380 +50381 POINT(30.10320649438916 -95.78000603488664) bank50381 +50382 POINT(29.397414812417328 -95.71721024983685) bank50382 +50383 POINT(29.206802064835152 -94.65681524345898) bank50383 +50384 POINT(29.434485301388484 -96.00331467944368) bank50384 +50385 POINT(29.6365023127963 -95.09377327881003) bank50385 +50386 POINT(29.95272532708118 -95.97230203040507) bank50386 +50387 POINT(29.663484238442166 -95.39851328093182) bank50387 +50388 POINT(29.459512659239962 -94.53099126044842) bank50388 +50389 POINT(30.681447239141434 -95.18519284453735) bank50389 +50390 POINT(29.870901925607026 -95.6731106486978) bank50390 +50391 POINT(30.05417264652671 -94.72243078601248) bank50391 +50392 POINT(29.462834300428476 -95.96821536974154) bank50392 +50393 POINT(30.726064175020372 -94.52117033824054) bank50393 +50394 POINT(30.45917558154906 -95.90066549748468) bank50394 +50395 POINT(29.563648756037598 -95.68226376018595) bank50395 +50396 POINT(30.395037819622342 -95.07177472446544) bank50396 +50397 POINT(29.013504477793433 -95.92742148442309) bank50397 +50398 POINT(29.999586513772172 -95.96719409655081) bank50398 +50399 POINT(29.300699759342532 -95.41510229304265) bank50399 +50400 POINT(30.53505478710433 -94.50437012796382) bank50400 +50401 POINT(29.045423467650668 -94.87492030378935) bank50401 +50402 POINT(29.431307946733035 -96.3515678470903) bank50402 +50403 POINT(30.116655504536418 -96.26001965603308) bank50403 +50404 POINT(29.09833308594763 -95.77902934363138) bank50404 +50405 POINT(29.60431129116146 -95.39393454249146) bank50405 +50406 POINT(29.1897541973769 -96.28911632884355) bank50406 +50407 POINT(29.565198951931862 -94.6881897904155) bank50407 +50408 POINT(30.502171578708772 -96.34846643605187) bank50408 +50409 POINT(30.426247514739185 -95.41898930729548) bank50409 +50410 POINT(29.527079550277804 -96.20996164323685) bank50410 +50411 POINT(30.33764994114201 -95.41074777204008) bank50411 +50412 POINT(29.599469813830243 -96.1334243538621) bank50412 +50413 POINT(29.068496324954726 -95.4399267988323) bank50413 +50414 POINT(29.152097254482612 -95.19617666975428) bank50414 +50415 POINT(29.681690243270303 -95.14658686454439) bank50415 +50416 POINT(29.90098262169346 -95.01797152558409) bank50416 +50417 POINT(29.704325963228584 -96.17605782396762) bank50417 +50418 POINT(29.591870714136455 -94.81488164175039) bank50418 +50419 POINT(29.19107662640902 -95.21915160544907) bank50419 +50420 POINT(29.743111365819995 -95.61104189026666) bank50420 +50421 POINT(30.239447987032598 -94.42929087910426) bank50421 +50422 POINT(29.29960689856617 -94.87114981152779) bank50422 +50423 POINT(30.014586562674786 -96.18614400174867) bank50423 +50424 POINT(28.795585491366175 -95.80192525040796) bank50424 +50425 POINT(29.203635857051413 -96.20736377618148) bank50425 +50426 POINT(30.41057302348023 -95.3429755354197) bank50426 +50427 POINT(29.629730163814468 -96.05080411400643) bank50427 +50428 POINT(29.708675390989608 -96.24063712408324) bank50428 +50429 POINT(30.431748498669574 -95.11057278727114) bank50429 +50430 POINT(29.528555856507005 -95.44184399178269) bank50430 +50431 POINT(29.458181631229643 -95.4747722801715) bank50431 +50432 POINT(30.268602272416835 -94.49860272152436) bank50432 +50433 POINT(30.08061515310823 -95.52734230627954) bank50433 +50434 POINT(29.226223421857338 -95.31728190308462) bank50434 +50435 POINT(29.60177007887042 -95.128118924163) bank50435 +50436 POINT(30.481277236666475 -95.58292552944962) bank50436 +50437 POINT(30.087463509162273 -96.15144805980123) bank50437 +50438 POINT(29.346183458296554 -94.50555566153562) bank50438 +50439 POINT(30.062269744290546 -94.72536543338862) bank50439 +50440 POINT(29.57548074676114 -94.69507676471919) bank50440 +50441 POINT(29.322633528272814 -95.73196093589202) bank50441 +50442 POINT(30.19434042031182 -95.0600385284492) bank50442 +50443 POINT(28.835539228230502 -95.00241931143196) bank50443 +50444 POINT(30.274155342935384 -95.82154396562969) bank50444 +50445 POINT(30.471610532024727 -95.96422677256274) bank50445 +50446 POINT(29.444330015377933 -96.05936304103388) bank50446 +50447 POINT(29.43530862228184 -94.54524491636332) bank50447 +50448 POINT(30.419954257028145 -95.49948902017681) bank50448 +50449 POINT(30.514703224413367 -94.6137023810269) bank50449 +50450 POINT(29.53707212636261 -95.09061027794964) bank50450 +50451 POINT(28.880445499042157 -94.53956500882724) bank50451 +50452 POINT(30.03853328448035 -95.24780038825322) bank50452 +50453 POINT(29.482816332750527 -95.5757078755343) bank50453 +50454 POINT(30.27118556544981 -94.7471484732093) bank50454 +50455 POINT(30.13538198323311 -95.65100726169165) bank50455 +50456 POINT(29.72193025007211 -95.71239906741523) bank50456 +50457 POINT(30.119115359194797 -95.18634667476393) bank50457 +50458 POINT(29.48283563531919 -95.38035060026776) bank50458 +50459 POINT(29.20095818660497 -94.95067968847445) bank50459 +50460 POINT(29.49089920942833 -96.30317200377964) bank50460 +50461 POINT(29.634808130386233 -95.09501920371048) bank50461 +50462 POINT(29.245848741726604 -95.2690279773962) bank50462 +50463 POINT(30.365967477763945 -95.10973766573038) bank50463 +50464 POINT(30.102198581133212 -94.42896693478512) bank50464 +50465 POINT(30.58101968543478 -94.46754706014336) bank50465 +50466 POINT(29.12473983912727 -96.19099157492852) bank50466 +50467 POINT(30.717191056629204 -94.66405672310322) bank50467 +50468 POINT(30.14307814019163 -94.6415039616444) bank50468 +50469 POINT(29.85603724394888 -94.43826458989001) bank50469 +50470 POINT(28.808277124303796 -95.26817864813553) bank50470 +50471 POINT(30.593693337697044 -94.47749868022039) bank50471 +50472 POINT(29.370121788227543 -94.80719370665459) bank50472 +50473 POINT(29.296783114168893 -95.90061595717864) bank50473 +50474 POINT(30.58477638016621 -95.62656316239328) bank50474 +50475 POINT(30.205684486902907 -95.58991359414317) bank50475 +50476 POINT(29.7237603663195 -95.12028722486332) bank50476 +50477 POINT(30.18876419595784 -94.96504117857208) bank50477 +50478 POINT(30.65626562913059 -95.3109547292984) bank50478 +50479 POINT(29.157397522058933 -95.16545160966545) bank50479 +50480 POINT(28.76922429507691 -94.75624269099951) bank50480 +50481 POINT(29.228442835385007 -94.42283393666457) bank50481 +50482 POINT(30.679050895530537 -94.60437858730211) bank50482 +50483 POINT(29.145951239149966 -95.07772632251947) bank50483 +50484 POINT(29.90967182544468 -95.667067002166) bank50484 +50485 POINT(29.170294227672713 -94.37346234715156) bank50485 +50486 POINT(30.287219110089744 -95.15078380484508) bank50486 +50487 POINT(29.219006068352698 -94.60670140855042) bank50487 +50488 POINT(29.588117602683766 -94.9854626675099) bank50488 +50489 POINT(30.554200429207995 -94.97326527795799) bank50489 +50490 POINT(28.776390306776698 -95.75691174249206) bank50490 +50491 POINT(29.77252280931489 -96.34360967502606) bank50491 +50492 POINT(29.92139551931264 -94.61767162579417) bank50492 +50493 POINT(28.788326326256495 -95.50056279479081) bank50493 +50494 POINT(30.32535226951261 -94.87943788576314) bank50494 +50495 POINT(30.601522841283625 -94.898356370805) bank50495 +50496 POINT(29.06834473316988 -95.75916857260194) bank50496 +50497 POINT(29.563404661260673 -95.49106228486058) bank50497 +50498 POINT(29.10337666133566 -96.2015986492196) bank50498 +50499 POINT(29.191955821056606 -94.41959994013502) bank50499 +50500 POINT(29.40989898038079 -95.07198740819493) bank50500 +50501 POINT(30.521117370478144 -96.1661061057394) bank50501 +50502 POINT(29.970737481577526 -96.10596162313757) bank50502 +50503 POINT(30.676547691241197 -95.58942692951123) bank50503 +50504 POINT(30.635975638434495 -95.20873022095313) bank50504 +50505 POINT(29.51230015037796 -95.78667254263615) bank50505 +50506 POINT(29.626794376058083 -95.5700658017431) bank50506 +50507 POINT(30.159957392897468 -95.92338108288484) bank50507 +50508 POINT(30.345995240958754 -95.05587756677599) bank50508 +50509 POINT(29.674965810780225 -94.45699556861773) bank50509 +50510 POINT(29.66260453278432 -94.6065544883774) bank50510 +50511 POINT(30.231330509196393 -96.2585943483915) bank50511 +50512 POINT(30.462075890620415 -95.32440605124545) bank50512 +50513 POINT(30.552682314951156 -96.13647700003075) bank50513 +50514 POINT(30.758809860900122 -94.71588949241621) bank50514 +50515 POINT(29.21533154497158 -95.87923437481334) bank50515 +50516 POINT(29.94577401115705 -95.45104114630512) bank50516 +50517 POINT(29.74588889186428 -95.91455313103984) bank50517 +50518 POINT(29.264020455657086 -95.73962847076938) bank50518 +50519 POINT(30.487139373404837 -95.15830596979275) bank50519 +50520 POINT(28.82495213197068 -94.642669287311) bank50520 +50521 POINT(29.77344830080076 -94.63489967983467) bank50521 +50522 POINT(30.49359846120622 -95.91097101405302) bank50522 +50523 POINT(29.513944100040195 -94.93954434429158) bank50523 +50524 POINT(30.675736684725187 -94.67121349463433) bank50524 +50525 POINT(29.527434336850718 -94.77143505845937) bank50525 +50526 POINT(30.52622578547538 -96.17585793897443) bank50526 +50527 POINT(29.84305732881649 -94.708394108508) bank50527 +50528 POINT(28.8629654552892 -95.09446806314169) bank50528 +50529 POINT(29.18126912102604 -95.98584606138932) bank50529 +50530 POINT(28.95715712590342 -95.18303647805952) bank50530 +50531 POINT(28.87305802519692 -94.37071696143484) bank50531 +50532 POINT(30.206865648251167 -95.1723506804319) bank50532 +50533 POINT(30.24179249319627 -95.94026001720884) bank50533 +50534 POINT(30.33002826392095 -94.99993902798742) bank50534 +50535 POINT(30.10336930194224 -94.49749069327555) bank50535 +50536 POINT(28.874076868626737 -95.22969563531252) bank50536 +50537 POINT(29.580093066544414 -94.66529196884079) bank50537 +50538 POINT(30.153286795228084 -94.48737503925331) bank50538 +50539 POINT(30.44158393757449 -94.6814645523384) bank50539 +50540 POINT(29.003102804416645 -95.59519448828743) bank50540 +50541 POINT(29.931474891341235 -94.66277579143578) bank50541 +50542 POINT(29.26819519577565 -95.1387687160524) bank50542 +50543 POINT(28.986613550141403 -95.15945423173285) bank50543 +50544 POINT(29.641253293725857 -94.887107374714) bank50544 +50545 POINT(30.04348021107144 -94.77620748920558) bank50545 +50546 POINT(30.13590282610269 -95.53595399464035) bank50546 +50547 POINT(29.072495163458107 -95.80465226285403) bank50547 +50548 POINT(28.87351477246943 -96.29326196095712) bank50548 +50549 POINT(29.52872720763952 -96.14787511126214) bank50549 +50550 POINT(30.461565459941987 -95.3446571487811) bank50550 +50551 POINT(29.238611408163706 -95.74809510389005) bank50551 +50552 POINT(30.058583038995028 -94.7455953246019) bank50552 +50553 POINT(29.151859372255956 -94.51166201768638) bank50553 +50554 POINT(29.484327901573312 -95.65889376675854) bank50554 +50555 POINT(30.02635423959645 -95.79488669409028) bank50555 +50556 POINT(29.406886505682994 -95.60702947387647) bank50556 +50557 POINT(29.419498519591148 -94.512190398127) bank50557 +50558 POINT(29.542942958363994 -94.3714682894981) bank50558 +50559 POINT(30.489401645678353 -96.2256503505847) bank50559 +50560 POINT(30.23265764234057 -96.25751605751833) bank50560 +50561 POINT(29.936359377803086 -94.49000677132355) bank50561 +50562 POINT(29.098659770440857 -96.08401313225747) bank50562 +50563 POINT(30.322596215180297 -95.40548872077292) bank50563 +50564 POINT(29.970733380187635 -95.72328255335614) bank50564 +50565 POINT(29.727476121725644 -96.33448790873582) bank50565 +50566 POINT(29.630258315160827 -95.74917219481353) bank50566 +50567 POINT(29.48062315419835 -95.96686301684052) bank50567 +50568 POINT(30.19399228162363 -94.71560200532753) bank50568 +50569 POINT(29.76918808639377 -94.79714501930452) bank50569 +50570 POINT(29.725347506766845 -96.33725025320587) bank50570 +50571 POINT(29.844693401374155 -95.3309993426921) bank50571 +50572 POINT(29.812213842594883 -95.03692970472412) bank50572 +50573 POINT(29.45685477882864 -95.35445675086756) bank50573 +50574 POINT(30.694722759662106 -95.84492195759363) bank50574 +50575 POINT(29.8221451472587 -95.12030104495977) bank50575 +50576 POINT(30.21454090106563 -94.89466841035023) bank50576 +50577 POINT(28.78041043715152 -95.5170888229139) bank50577 +50578 POINT(29.93807563932496 -95.41679614125378) bank50578 +50579 POINT(30.69078294383006 -95.0839590332743) bank50579 +50580 POINT(29.203160929348737 -95.67801774227716) bank50580 +50581 POINT(29.775914859092772 -95.45721269427365) bank50581 +50582 POINT(29.17907881155467 -95.5304674311967) bank50582 +50583 POINT(28.904101223327935 -94.66719927877216) bank50583 +50584 POINT(30.28435939332315 -95.41096958660702) bank50584 +50585 POINT(29.541076768071612 -95.14295361524468) bank50585 +50586 POINT(30.717249047619678 -94.55524192416807) bank50586 +50587 POINT(30.25892119374118 -95.54896323531392) bank50587 +50588 POINT(29.611369149530837 -94.39131898978721) bank50588 +50589 POINT(30.40643043693397 -95.85163186346851) bank50589 +50590 POINT(29.100696840384437 -94.76343830008939) bank50590 +50591 POINT(28.876993952094267 -95.9757875657597) bank50591 +50592 POINT(30.11725945679936 -95.82607594418964) bank50592 +50593 POINT(29.835132610299095 -94.51950370645818) bank50593 +50594 POINT(30.21043319972781 -94.71933262221835) bank50594 +50595 POINT(29.40478952679603 -95.3919440582326) bank50595 +50596 POINT(28.79262354726821 -95.18236981082646) bank50596 +50597 POINT(30.666339103002027 -95.71909385904338) bank50597 +50598 POINT(28.83589652621037 -95.40779000329994) bank50598 +50599 POINT(30.603305148608733 -96.25684341818925) bank50599 +50600 POINT(30.75062800068831 -95.74503315616242) bank50600 +50601 POINT(30.560201693851326 -94.97104301027264) bank50601 +50602 POINT(28.76494503256338 -94.72563429939784) bank50602 +50603 POINT(30.323743611090965 -94.4245710263923) bank50603 +50604 POINT(28.911688129967295 -94.65067956921668) bank50604 +50605 POINT(30.03420765080402 -94.84028865212666) bank50605 +50606 POINT(30.126455471488317 -94.65324397356581) bank50606 +50607 POINT(29.012133123432328 -95.87865185685605) bank50607 +50608 POINT(30.337019362948958 -94.64042541844408) bank50608 +50609 POINT(28.82631046319985 -96.32927805212552) bank50609 +50610 POINT(30.482402097337463 -95.49755516038151) bank50610 +50611 POINT(29.572623313059605 -96.00706889236034) bank50611 +50612 POINT(29.122856543432814 -95.82474367195005) bank50612 +50613 POINT(29.01581812303521 -95.34202158599405) bank50613 +50614 POINT(29.412631806028614 -95.08301619877153) bank50614 +50615 POINT(28.77922902968473 -94.91121913431301) bank50615 +50616 POINT(29.010751894686173 -94.42474225042015) bank50616 +50617 POINT(28.7898265675471 -96.23480555393317) bank50617 +50618 POINT(29.645962581417916 -94.96445526611525) bank50618 +50619 POINT(29.908389253872183 -95.1016324616394) bank50619 +50620 POINT(30.54154567078264 -94.40301545150155) bank50620 +50621 POINT(29.08733384838616 -94.5076962223214) bank50621 +50622 POINT(29.32362987231725 -95.09088665171141) bank50622 +50623 POINT(29.66931168939648 -95.28473915378838) bank50623 +50624 POINT(29.11667286619393 -95.64213766968952) bank50624 +50625 POINT(30.130285374379977 -96.21015309879279) bank50625 +50626 POINT(29.674473768831444 -95.6383168138897) bank50626 +50627 POINT(30.442474998013765 -95.00957040624507) bank50627 +50628 POINT(29.785326264034683 -96.13559802830702) bank50628 +50629 POINT(30.414704765329155 -95.8409323133145) bank50629 +50630 POINT(30.44821758072634 -95.57521621614255) bank50630 +50631 POINT(28.82243973293191 -95.05715325562191) bank50631 +50632 POINT(30.744286404857423 -96.05025382677313) bank50632 +50633 POINT(29.225270908216153 -94.59515349281878) bank50633 +50634 POINT(29.218182172609833 -96.26084047124063) bank50634 +50635 POINT(29.30206872775664 -95.80623830015733) bank50635 +50636 POINT(29.770609787900966 -95.73990398158584) bank50636 +50637 POINT(29.036482650517794 -96.10752045992383) bank50637 +50638 POINT(30.02388076431376 -94.54217866302946) bank50638 +50639 POINT(30.75680228661895 -95.54935899608472) bank50639 +50640 POINT(30.028470866698306 -94.80269157810513) bank50640 +50641 POINT(29.902121544099387 -94.41586817373017) bank50641 +50642 POINT(30.058486917592155 -96.06016828158852) bank50642 +50643 POINT(29.002752485097428 -95.21665010543056) bank50643 +50644 POINT(29.2479856162213 -94.77878807004798) bank50644 +50645 POINT(29.955938447389208 -96.20159443205905) bank50645 +50646 POINT(30.29374664819908 -96.06422801875308) bank50646 +50647 POINT(29.02529501491902 -95.94808464203184) bank50647 +50648 POINT(29.30078014523276 -96.00725923089787) bank50648 +50649 POINT(30.696710901192546 -94.73014908886914) bank50649 +50650 POINT(29.092885148576613 -94.79046333059074) bank50650 +50651 POINT(30.178987049023817 -95.1098289261721) bank50651 +50652 POINT(29.20324556929183 -95.06945522851484) bank50652 +50653 POINT(29.436297630165654 -94.7789361200298) bank50653 +50654 POINT(30.163047269276614 -96.30868837167783) bank50654 +50655 POINT(29.634042694674026 -94.84434630605584) bank50655 +50656 POINT(29.856632607042535 -94.91364516284969) bank50656 +50657 POINT(29.443370916822573 -95.04615971494701) bank50657 +50658 POINT(29.878447765606865 -94.65001105150289) bank50658 +50659 POINT(30.62315629698437 -95.62953831431356) bank50659 +50660 POINT(28.949784287505306 -94.58762773767087) bank50660 +50661 POINT(29.23875053550046 -95.06343684095434) bank50661 +50662 POINT(28.910903448283708 -95.16345731631121) bank50662 +50663 POINT(30.0140209105393 -94.73654913762779) bank50663 +50664 POINT(30.240711909319764 -95.92549304539854) bank50664 +50665 POINT(30.1862980694367 -95.0712813443596) bank50665 +50666 POINT(29.755714104376487 -95.44641886605686) bank50666 +50667 POINT(29.134685021294352 -94.43704786175986) bank50667 +50668 POINT(30.722033224277894 -96.0897170392218) bank50668 +50669 POINT(29.510773800536768 -95.83710667724434) bank50669 +50670 POINT(28.770147643841433 -95.55079139829705) bank50670 +50671 POINT(29.591327161616213 -94.79573305450808) bank50671 +50672 POINT(30.089060965675277 -95.54554544114056) bank50672 +50673 POINT(30.756756065004144 -95.30720712362853) bank50673 +50674 POINT(28.990624223367167 -94.95913388627402) bank50674 +50675 POINT(30.028539303455133 -94.90113232293723) bank50675 +50676 POINT(30.529458693244628 -94.7980829950073) bank50676 +50677 POINT(30.66106662341686 -95.681467540667) bank50677 +50678 POINT(30.422641698545327 -95.13913313309928) bank50678 +50679 POINT(30.03521618089691 -95.6051474881091) bank50679 +50680 POINT(29.51387174069931 -95.70526366973709) bank50680 +50681 POINT(30.07078176619659 -94.795694577563) bank50681 +50682 POINT(30.577003061696796 -95.6189172974747) bank50682 +50683 POINT(29.152011987910388 -94.7647647399005) bank50683 +50684 POINT(29.542378958875112 -95.70529754893334) bank50684 +50685 POINT(29.358069870280783 -94.71263365228843) bank50685 +50686 POINT(29.620535395483685 -95.86399810975517) bank50686 +50687 POINT(29.077872668406687 -95.13493463431311) bank50687 +50688 POINT(29.568663237417795 -96.11237301597106) bank50688 +50689 POINT(30.664893351423753 -95.43309403034573) bank50689 +50690 POINT(29.317180356550004 -95.83984481679781) bank50690 +50691 POINT(28.900432947020974 -96.07137546939134) bank50691 +50692 POINT(29.15025059178085 -95.09228475131502) bank50692 +50693 POINT(30.17819750241578 -94.71094263791439) bank50693 +50694 POINT(30.573987327175896 -94.67555338215134) bank50694 +50695 POINT(29.5074369464415 -94.87630128006376) bank50695 +50696 POINT(29.182526809781113 -95.41739978217329) bank50696 +50697 POINT(29.25643434128448 -95.73897907181889) bank50697 +50698 POINT(29.150933377561294 -94.58857028420462) bank50698 +50699 POINT(30.001427978776533 -95.55384919902843) bank50699 +50700 POINT(29.60962344927891 -94.6267991757833) bank50700 +50701 POINT(29.62242208143233 -95.55727859577308) bank50701 +50702 POINT(30.354779008133054 -95.1219856570936) bank50702 +50703 POINT(30.743830156353177 -95.60172322511102) bank50703 +50704 POINT(30.46743158115875 -95.57947308869944) bank50704 +50705 POINT(29.322459880188127 -95.99855487970224) bank50705 +50706 POINT(29.012378674279887 -95.75905763021271) bank50706 +50707 POINT(29.294990333533537 -95.32151024136824) bank50707 +50708 POINT(29.700013788679062 -96.31890768774367) bank50708 +50709 POINT(29.663746093098023 -95.3649202604325) bank50709 +50710 POINT(30.541010635406362 -95.5004969160971) bank50710 +50711 POINT(30.047390235831987 -94.8168799642786) bank50711 +50712 POINT(30.531269274084096 -95.97284250741617) bank50712 +50713 POINT(28.880810350420774 -95.62193080151776) bank50713 +50714 POINT(30.07857301425025 -94.63548989334666) bank50714 +50715 POINT(30.333633538746927 -95.70839375984269) bank50715 +50716 POINT(30.376933490647637 -95.94626015297104) bank50716 +50717 POINT(30.741998187668777 -94.64254992743294) bank50717 +50718 POINT(29.387901023544238 -95.12437439727356) bank50718 +50719 POINT(29.515755175877942 -95.93804923149648) bank50719 +50720 POINT(29.859774650560396 -95.82885812898343) bank50720 +50721 POINT(29.688697989713518 -95.26535007749474) bank50721 +50722 POINT(29.487547918998203 -96.21852082348703) bank50722 +50723 POINT(29.25783150745775 -95.20170215354904) bank50723 +50724 POINT(29.505723196856405 -95.32636016688521) bank50724 +50725 POINT(29.832246920856264 -95.65041727707072) bank50725 +50726 POINT(29.253491646353815 -96.01534929806635) bank50726 +50727 POINT(29.44237154615174 -94.95001980127809) bank50727 +50728 POINT(28.964433693895412 -95.87517639362933) bank50728 +50729 POINT(29.552485725622763 -95.05428546890877) bank50729 +50730 POINT(29.93455935573614 -95.16949609771787) bank50730 +50731 POINT(29.68141028166371 -95.09802075787795) bank50731 +50732 POINT(30.033387011246734 -95.63182562899061) bank50732 +50733 POINT(29.26463440534718 -96.31923164383606) bank50733 +50734 POINT(29.388948290248727 -94.91062138516536) bank50734 +50735 POINT(29.80847013983158 -95.68582896596784) bank50735 +50736 POINT(30.126762226934638 -95.37884767494353) bank50736 +50737 POINT(29.14598406809709 -94.7006008531034) bank50737 +50738 POINT(29.625722884044958 -96.30735992376026) bank50738 +50739 POINT(30.217892354660933 -94.44993416478941) bank50739 +50740 POINT(30.728181371723423 -95.45364725415047) bank50740 +50741 POINT(30.41047797631865 -94.79851864434026) bank50741 +50742 POINT(30.176016410590133 -95.51961029472201) bank50742 +50743 POINT(29.664754198884932 -95.57297159233359) bank50743 +50744 POINT(28.83338105402844 -95.68303914804657) bank50744 +50745 POINT(29.524522977807962 -95.7449740388861) bank50745 +50746 POINT(28.917436873577238 -95.60382574274986) bank50746 +50747 POINT(29.7014221646573 -95.18907153648553) bank50747 +50748 POINT(29.026627392736813 -95.9780789569482) bank50748 +50749 POINT(29.96370326086004 -94.77054191821377) bank50749 +50750 POINT(29.308055153874648 -95.87599388065084) bank50750 +50751 POINT(30.143751371668007 -95.72227936009214) bank50751 +50752 POINT(30.479406350919387 -95.83509678427616) bank50752 +50753 POINT(28.80679795596023 -95.42842129854044) bank50753 +50754 POINT(30.14290799329857 -95.41388176296647) bank50754 +50755 POINT(30.295537573516583 -95.87120569913597) bank50755 +50756 POINT(30.44875129530621 -95.27659951079646) bank50756 +50757 POINT(30.437244415163622 -94.53836014383667) bank50757 +50758 POINT(29.63115744976146 -95.58805071839967) bank50758 +50759 POINT(29.852225868475895 -94.94689052064054) bank50759 +50760 POINT(29.92371839941706 -96.26109426628855) bank50760 +50761 POINT(30.109562714815876 -94.80134263667198) bank50761 +50762 POINT(30.446155820624856 -94.40854571747255) bank50762 +50763 POINT(30.077203247222226 -95.17380592468442) bank50763 +50764 POINT(29.469504713245918 -94.4336950667726) bank50764 +50765 POINT(29.89399370709899 -95.0714279892994) bank50765 +50766 POINT(29.818672115453207 -94.80290812456185) bank50766 +50767 POINT(29.497826879318858 -95.43121540731602) bank50767 +50768 POINT(29.5433282021276 -95.08189257650614) bank50768 +50769 POINT(29.92277116475524 -95.32571073268684) bank50769 +50770 POINT(29.95622652796912 -94.43726985196578) bank50770 +50771 POINT(30.610731218078513 -95.52608382794716) bank50771 +50772 POINT(29.646734402491457 -95.78817931517241) bank50772 +50773 POINT(30.301686299690385 -95.85436031876588) bank50773 +50774 POINT(30.06200964516638 -95.20270274792266) bank50774 +50775 POINT(28.972203829383478 -95.46970047360841) bank50775 +50776 POINT(30.620552865858826 -95.02122399997823) bank50776 +50777 POINT(28.887153424545385 -96.20123340650268) bank50777 +50778 POINT(30.26942251151863 -94.87467276752726) bank50778 +50779 POINT(30.013052362719687 -95.03387050918676) bank50779 +50780 POINT(30.228389749087462 -95.16372684434559) bank50780 +50781 POINT(29.679138782095986 -95.97202488453702) bank50781 +50782 POINT(29.77655398182115 -94.51290993625517) bank50782 +50783 POINT(30.305994347423272 -95.82510268912426) bank50783 +50784 POINT(29.780135209201095 -94.84995852497381) bank50784 +50785 POINT(28.95972385049409 -94.66240665754984) bank50785 +50786 POINT(30.307939116286676 -95.88184993189843) bank50786 +50787 POINT(30.161013471120338 -96.3386343785102) bank50787 +50788 POINT(30.707907589280452 -94.79579156127177) bank50788 +50789 POINT(28.854209727698365 -94.74593918023638) bank50789 +50790 POINT(29.492020315178888 -95.73632988594318) bank50790 +50791 POINT(29.05359787322306 -95.72020465237198) bank50791 +50792 POINT(30.21368452797444 -94.78186205114787) bank50792 +50793 POINT(30.440716746297515 -95.37092080658206) bank50793 +50794 POINT(29.94292341001057 -95.98306982650361) bank50794 +50795 POINT(30.10260120359156 -94.95222996419456) bank50795 +50796 POINT(30.14480210514652 -95.84666423333827) bank50796 +50797 POINT(28.92990255167687 -96.20229635658643) bank50797 +50798 POINT(30.285102485850985 -94.65429560893028) bank50798 +50799 POINT(30.59005938197963 -94.94118308783985) bank50799 +50800 POINT(30.17574436375241 -95.87495993220179) bank50800 +50801 POINT(29.43799228941297 -96.15862463947285) bank50801 +50802 POINT(28.929087973768358 -94.94654728206285) bank50802 +50803 POINT(30.64048365105754 -95.28142086792197) bank50803 +50804 POINT(30.532090415599914 -95.88193687490893) bank50804 +50805 POINT(29.364877883661244 -94.64654903751385) bank50805 +50806 POINT(28.848381600600273 -95.11671433874595) bank50806 +50807 POINT(30.481604186245132 -95.06108943412723) bank50807 +50808 POINT(29.87925183257674 -95.14022409549925) bank50808 +50809 POINT(29.109575583616397 -94.73635283096542) bank50809 +50810 POINT(29.06860158491006 -95.1083057831201) bank50810 +50811 POINT(30.065212923336624 -96.33999096273853) bank50811 +50812 POINT(30.34111020218093 -95.28159383115629) bank50812 +50813 POINT(30.102047787328633 -95.19454115269436) bank50813 +50814 POINT(30.44985862735077 -96.04948328410556) bank50814 +50815 POINT(28.9525807108274 -94.70389341942392) bank50815 +50816 POINT(28.98655660477762 -95.87194509339771) bank50816 +50817 POINT(30.50178116476729 -95.30355253785012) bank50817 +50818 POINT(30.322980494786005 -96.21905851024665) bank50818 +50819 POINT(30.142556838883458 -95.81014869379018) bank50819 +50820 POINT(29.92100038562936 -95.24304250601892) bank50820 +50821 POINT(29.117312489104098 -94.82079068612113) bank50821 +50822 POINT(30.204019227986528 -94.95209395824072) bank50822 +50823 POINT(30.751109426847275 -95.38557541766474) bank50823 +50824 POINT(30.035713087467954 -95.44201743207954) bank50824 +50825 POINT(28.882387232968483 -95.38231145611846) bank50825 +50826 POINT(29.6212783477605 -94.74814498672437) bank50826 +50827 POINT(29.335016401213114 -96.11744099155402) bank50827 +50828 POINT(29.09737112757233 -95.82397831377163) bank50828 +50829 POINT(29.43865504599779 -94.76386890886826) bank50829 +50830 POINT(30.394073964199332 -94.71562744792615) bank50830 +50831 POINT(28.99471117164417 -96.04340163671777) bank50831 +50832 POINT(29.561054268608284 -96.28805102933138) bank50832 +50833 POINT(28.944331537656517 -94.53721748627898) bank50833 +50834 POINT(29.18941607458531 -95.8635681059758) bank50834 +50835 POINT(30.63388017637018 -96.02566686563159) bank50835 +50836 POINT(29.411709925536247 -95.23660623471055) bank50836 +50837 POINT(29.160946925667012 -96.32874800449787) bank50837 +50838 POINT(29.055378388453295 -95.55897538903021) bank50838 +50839 POINT(30.597876032308523 -95.52051546559473) bank50839 +50840 POINT(29.11113031919408 -95.46089413378857) bank50840 +50841 POINT(30.653065637813157 -94.92646409919158) bank50841 +50842 POINT(30.68877905162803 -96.32259058525852) bank50842 +50843 POINT(30.510709971791222 -94.59977532505378) bank50843 +50844 POINT(29.54511165243755 -95.531757504688) bank50844 +50845 POINT(28.77180696702651 -94.67143462793302) bank50845 +50846 POINT(30.003005840315847 -94.70907524618376) bank50846 +50847 POINT(29.654265397811987 -95.17935157243248) bank50847 +50848 POINT(29.75597045086405 -94.80501687188168) bank50848 +50849 POINT(29.602879645895023 -94.77511282514148) bank50849 +50850 POINT(29.61812013475113 -95.45807644360572) bank50850 +50851 POINT(29.02452890609137 -94.77759319609841) bank50851 +50852 POINT(29.887427217509043 -95.42424011774266) bank50852 +50853 POINT(30.487126460495414 -94.88802512080807) bank50853 +50854 POINT(29.212273306889905 -95.14791775756741) bank50854 +50855 POINT(29.343092490114437 -95.17771153765595) bank50855 +50856 POINT(29.842716667377683 -94.96102033048432) bank50856 +50857 POINT(30.237213104923867 -94.89782965361223) bank50857 +50858 POINT(29.993225660593062 -96.35243593176995) bank50858 +50859 POINT(29.648557364581958 -95.86009834736566) bank50859 +50860 POINT(30.643302333932883 -95.45801864559229) bank50860 +50861 POINT(30.727756159692635 -95.51194244579325) bank50861 +50862 POINT(29.385868523285765 -95.32065020870488) bank50862 +50863 POINT(30.031973710428403 -94.48564214244651) bank50863 +50864 POINT(28.948630725537125 -95.63662067614545) bank50864 +50865 POINT(30.184193543686867 -95.99328200834304) bank50865 +50866 POINT(30.161604973016278 -95.76831587715031) bank50866 +50867 POINT(28.91480003570265 -95.5462468417234) bank50867 +50868 POINT(30.74892217222522 -95.83780927045491) bank50868 +50869 POINT(29.751974606291643 -95.9874131029972) bank50869 +50870 POINT(30.26763923251145 -95.81395950089106) bank50870 +50871 POINT(29.018889650341617 -95.99557726222757) bank50871 +50872 POINT(29.93897310461407 -95.68638997433314) bank50872 +50873 POINT(30.648911195282306 -95.21376070945158) bank50873 +50874 POINT(30.619579411222155 -95.06887924644938) bank50874 +50875 POINT(29.11557099324177 -94.9082584824744) bank50875 +50876 POINT(30.64981741896966 -95.9247348842036) bank50876 +50877 POINT(29.8578978612499 -95.31088189623073) bank50877 +50878 POINT(28.824892332977583 -94.68252667201472) bank50878 +50879 POINT(30.599872641223254 -94.48367572909173) bank50879 +50880 POINT(30.74612845668779 -96.25314778366904) bank50880 +50881 POINT(29.153740743967678 -94.60277077497935) bank50881 +50882 POINT(30.668284353570257 -96.1367305314057) bank50882 +50883 POINT(28.915795814683246 -95.08727154880903) bank50883 +50884 POINT(29.52707967511987 -94.85860322574477) bank50884 +50885 POINT(30.653120718908255 -95.56407216730925) bank50885 +50886 POINT(29.683704216558997 -95.559139546583) bank50886 +50887 POINT(29.302905163157213 -95.92059639585372) bank50887 +50888 POINT(30.48030849164143 -95.27734098896985) bank50888 +50889 POINT(30.6355517447659 -94.75085229816763) bank50889 +50890 POINT(30.343848558904746 -95.86860855605345) bank50890 +50891 POINT(28.979974978839667 -95.42347190146938) bank50891 +50892 POINT(28.78282249048385 -95.36539066730494) bank50892 +50893 POINT(29.54455451791201 -95.54115932586328) bank50893 +50894 POINT(29.530392858149625 -95.99120341685573) bank50894 +50895 POINT(30.189274759961624 -94.58019512649498) bank50895 +50896 POINT(30.7336264468107 -96.06084875720786) bank50896 +50897 POINT(29.455977494110385 -95.18116485864944) bank50897 +50898 POINT(29.103214069625576 -95.22967311272761) bank50898 +50899 POINT(30.22483391895306 -95.54686136340399) bank50899 +50900 POINT(30.27394884394275 -95.40028794917522) bank50900 +50901 POINT(29.007721963921515 -94.80156233202524) bank50901 +50902 POINT(30.147160084025177 -94.94213003025003) bank50902 +50903 POINT(28.918472420643095 -94.81879884816944) bank50903 +50904 POINT(28.848555101585138 -95.46618228532606) bank50904 +50905 POINT(29.914534489542 -94.93421448172944) bank50905 +50906 POINT(30.434967771580823 -94.48217276650105) bank50906 +50907 POINT(28.843616132285664 -96.33890867750618) bank50907 +50908 POINT(30.53988254127345 -96.03597083501867) bank50908 +50909 POINT(29.94743879324595 -96.08661753670687) bank50909 +50910 POINT(29.061627341234455 -95.21424026032577) bank50910 +50911 POINT(29.794386232350167 -95.44834698853766) bank50911 +50912 POINT(29.91897526371716 -95.17137692743465) bank50912 +50913 POINT(30.432416904979284 -96.01554325989221) bank50913 +50914 POINT(28.8895340595324 -94.90474205864693) bank50914 +50915 POINT(29.599055687607972 -95.33097481570407) bank50915 +50916 POINT(29.263600486561092 -95.08473979522947) bank50916 +50917 POINT(29.796886826729903 -95.17806146504712) bank50917 +50918 POINT(29.91667693325952 -94.80318600504677) bank50918 +50919 POINT(29.428929524837347 -95.26111492007179) bank50919 +50920 POINT(30.71056746341781 -95.72408992339405) bank50920 +50921 POINT(29.62454592059925 -95.31509857415031) bank50921 +50922 POINT(29.54687297556472 -94.71067309077638) bank50922 +50923 POINT(29.39640922668447 -95.00559689897594) bank50923 +50924 POINT(28.998742697116132 -94.90260081916477) bank50924 +50925 POINT(30.746085659573573 -96.02906262889387) bank50925 +50926 POINT(30.337526013076747 -94.78208057306418) bank50926 +50927 POINT(29.81761344977846 -94.82185595556078) bank50927 +50928 POINT(30.37621944016312 -96.040783362861) bank50928 +50929 POINT(30.413704895530874 -95.12919703063066) bank50929 +50930 POINT(29.960711446533434 -95.99129294401388) bank50930 +50931 POINT(29.810954124708346 -96.07674007583194) bank50931 +50932 POINT(30.393041998660138 -95.27081008086661) bank50932 +50933 POINT(29.060698150386443 -95.42847651953116) bank50933 +50934 POINT(29.88232861194531 -95.1965075593966) bank50934 +50935 POINT(29.172625058751965 -94.86750883304592) bank50935 +50936 POINT(28.902888873226257 -95.9701751004089) bank50936 +50937 POINT(29.1311535944918 -95.16873810437738) bank50937 +50938 POINT(30.492728163933528 -95.19006283088946) bank50938 +50939 POINT(28.989632927458757 -94.424537394582) bank50939 +50940 POINT(29.668086559736295 -94.68934382271954) bank50940 +50941 POINT(30.703288420653024 -95.05939815635152) bank50941 +50942 POINT(30.61947709484188 -96.05009974157164) bank50942 +50943 POINT(30.509927239973983 -94.92101239256563) bank50943 +50944 POINT(28.8238455053038 -95.6450822453113) bank50944 +50945 POINT(30.59321387761717 -94.69688375907499) bank50945 +50946 POINT(29.394505073726563 -94.49502150120159) bank50946 +50947 POINT(29.43557159859035 -96.19499483057021) bank50947 +50948 POINT(30.61374803576725 -94.65716843234667) bank50948 +50949 POINT(30.69615238089399 -95.78686328507746) bank50949 +50950 POINT(29.27286048400556 -94.38430631753027) bank50950 +50951 POINT(30.291361317047137 -95.41040121599114) bank50951 +50952 POINT(29.06313527507371 -94.5165717181019) bank50952 +50953 POINT(29.78271593517961 -96.05177265116639) bank50953 +50954 POINT(28.79893935236235 -94.46116614169395) bank50954 +50955 POINT(29.487451689074668 -95.32641008563289) bank50955 +50956 POINT(30.309298370118583 -95.55161337837818) bank50956 +50957 POINT(30.74685103976554 -95.32906015946018) bank50957 +50958 POINT(29.3528562326016 -95.48466583766051) bank50958 +50959 POINT(30.312539033391914 -95.52992220403237) bank50959 +50960 POINT(30.294087524392776 -96.31728972232533) bank50960 +50961 POINT(30.483163680780343 -94.4148256523319) bank50961 +50962 POINT(29.960043391528245 -94.44532448327374) bank50962 +50963 POINT(30.57686734850023 -94.46125678677933) bank50963 +50964 POINT(30.197214848590267 -95.97098059594883) bank50964 +50965 POINT(29.42045286310797 -94.75706791856284) bank50965 +50966 POINT(29.012092773435683 -94.71144015582742) bank50966 +50967 POINT(29.063010005750687 -94.85202911928742) bank50967 +50968 POINT(29.432611766596963 -94.99789994458853) bank50968 +50969 POINT(30.59829220997882 -94.51097979373905) bank50969 +50970 POINT(30.278579508466528 -94.42451623792432) bank50970 +50971 POINT(29.679941046435857 -95.62324371409514) bank50971 +50972 POINT(29.16003505598094 -96.02199743901208) bank50972 +50973 POINT(29.82219779280865 -95.32345893456467) bank50973 +50974 POINT(30.376671675816915 -95.47570016936476) bank50974 +50975 POINT(28.938631901814738 -94.7127607858993) bank50975 +50976 POINT(30.468843296547035 -94.7101013952634) bank50976 +50977 POINT(29.340745973619093 -95.01029479256317) bank50977 +50978 POINT(29.217464741112504 -95.21636064833626) bank50978 +50979 POINT(30.54091957995483 -94.52524662966773) bank50979 +50980 POINT(30.30819522854019 -95.43840534752451) bank50980 +50981 POINT(29.704773438618744 -94.97865702495825) bank50981 +50982 POINT(29.251339291408083 -94.8026040617125) bank50982 +50983 POINT(30.014054014188012 -95.45292334212203) bank50983 +50984 POINT(30.654786120248545 -94.38976231013723) bank50984 +50985 POINT(29.460818488519976 -96.0434070513276) bank50985 +50986 POINT(30.30870040192397 -95.10181082479461) bank50986 +50987 POINT(30.513766937961545 -95.57622676770714) bank50987 +50988 POINT(30.65995494244153 -95.08396348723959) bank50988 +50989 POINT(30.054782774538104 -95.21982507325208) bank50989 +50990 POINT(29.27745051044169 -95.35158850405429) bank50990 +50991 POINT(30.45702244465957 -95.37897404314293) bank50991 +50992 POINT(29.630703565376855 -95.94804556321999) bank50992 +50993 POINT(29.232764830251142 -95.85191086409142) bank50993 +50994 POINT(30.154159112705486 -95.56886147005581) bank50994 +50995 POINT(29.453274588643005 -95.3511484540141) bank50995 +50996 POINT(30.138720441995115 -96.16214825934402) bank50996 +50997 POINT(29.129497189047626 -95.82505462439352) bank50997 +50998 POINT(29.21512275677624 -95.6545690713545) bank50998 +50999 POINT(29.176751490960868 -95.79254288169241) bank50999 +51000 POINT(28.84472392972461 -96.18988767247458) bank51000 +51001 POINT(28.975556372096566 -95.80302176578856) bank51001 +51002 POINT(29.203450962214106 -96.30470961318132) bank51002 +51003 POINT(28.83112982776606 -94.93899745455778) bank51003 +51004 POINT(29.187110988723568 -95.62916790681092) bank51004 +51005 POINT(29.40772410492322 -95.34311363210256) bank51005 +51006 POINT(30.225607766229754 -95.7964204062182) bank51006 +51007 POINT(28.787561267807995 -95.52492385980926) bank51007 +51008 POINT(29.854652412545345 -96.01191407169738) bank51008 +51009 POINT(30.559165739949297 -95.95800577487597) bank51009 +51010 POINT(28.853553461025232 -94.97655608185627) bank51010 +51011 POINT(29.459512210244075 -95.33829129183222) bank51011 +51012 POINT(30.473087122829003 -95.69737280312218) bank51012 +51013 POINT(29.252699500279427 -95.34937317796332) bank51013 +51014 POINT(30.01941404043125 -95.01072955450367) bank51014 +51015 POINT(29.003728976135612 -94.77550744254076) bank51015 +51016 POINT(29.866532167383493 -94.55533939981407) bank51016 +51017 POINT(30.61341231293983 -94.37395875516454) bank51017 +51018 POINT(29.442211192761306 -94.55322877501567) bank51018 +51019 POINT(30.35694429525684 -95.13569537008348) bank51019 +51020 POINT(30.139939147873257 -96.12768123927418) bank51020 +51021 POINT(29.501719620869295 -94.66718302685824) bank51021 +51022 POINT(29.651723318075327 -94.71151588222767) bank51022 +51023 POINT(28.97636073575148 -95.19413844148227) bank51023 +51024 POINT(29.8610033621226 -96.05412767662318) bank51024 +51025 POINT(29.889797747869473 -96.00720145013955) bank51025 +51026 POINT(29.684028905858177 -95.71027960332654) bank51026 +51027 POINT(28.847876626220575 -95.00965684462935) bank51027 +51028 POINT(30.046814602671514 -96.27769699164008) bank51028 +51029 POINT(30.442652543881604 -96.00685756026462) bank51029 +51030 POINT(30.31476133338582 -94.43461794391227) bank51030 +51031 POINT(30.69404924875031 -96.06576568290002) bank51031 +51032 POINT(29.602498081309168 -96.30275654134181) bank51032 +51033 POINT(29.17324138049281 -95.19882993770729) bank51033 +51034 POINT(30.4472001193667 -96.33090874411444) bank51034 +51035 POINT(29.38027342516481 -94.77155845879142) bank51035 +51036 POINT(29.834649340492387 -95.93285546587168) bank51036 +51037 POINT(29.744360085262283 -96.29487266144213) bank51037 +51038 POINT(30.736568696068296 -95.04833513598972) bank51038 +51039 POINT(29.807768476000117 -95.89242603596806) bank51039 +51040 POINT(29.318626763578685 -96.02646310927037) bank51040 +51041 POINT(30.368917258091425 -95.85378819421959) bank51041 +51042 POINT(30.26436419830879 -94.79186531667443) bank51042 +51043 POINT(30.123452327417677 -95.26439945239578) bank51043 +51044 POINT(30.03252907207491 -95.60813311752602) bank51044 +51045 POINT(29.18267098416915 -95.62074534973296) bank51045 +51046 POINT(28.9547291179107 -94.93202229970505) bank51046 +51047 POINT(29.661037329120205 -96.3331160713547) bank51047 +51048 POINT(30.077991211677706 -95.86317986465475) bank51048 +51049 POINT(29.294882210341413 -95.66990519314756) bank51049 +51050 POINT(30.498471133562226 -95.83998424296735) bank51050 +51051 POINT(29.71670649323176 -96.0838851043597) bank51051 +51052 POINT(30.653854652242686 -95.34425357551744) bank51052 +51053 POINT(30.619491909473094 -95.78174282113606) bank51053 +51054 POINT(29.712520157337668 -96.04539042350284) bank51054 +51055 POINT(30.518325737975683 -94.75680870332525) bank51055 +51056 POINT(29.670199775308845 -95.03746628351456) bank51056 +51057 POINT(29.151067779101076 -95.34049025448756) bank51057 +51058 POINT(30.098195879660746 -96.15296884005222) bank51058 +51059 POINT(30.12150927255462 -94.44605228354212) bank51059 +51060 POINT(29.759475971926086 -94.42300533925412) bank51060 +51061 POINT(28.86448081768376 -95.54281787510243) bank51061 +51062 POINT(29.367835312785196 -95.96456776097037) bank51062 +51063 POINT(30.075646493641592 -95.06224466562252) bank51063 +51064 POINT(29.908530438387306 -94.72080278413057) bank51064 +51065 POINT(30.48593847927907 -94.5255695342499) bank51065 +51066 POINT(30.15076237352675 -94.39494083893412) bank51066 +51067 POINT(30.397847155632917 -96.25848037859092) bank51067 +51068 POINT(30.608871661289097 -94.53403425910906) bank51068 +51069 POINT(29.250918759447565 -94.46951696580065) bank51069 +51070 POINT(30.062773866319187 -94.52582982985653) bank51070 +51071 POINT(30.068467645222785 -95.27220269947355) bank51071 +51072 POINT(29.23063966892534 -95.38304981197271) bank51072 +51073 POINT(30.56070540149713 -94.8038136064859) bank51073 +51074 POINT(29.293776631544095 -95.38676955471787) bank51074 +51075 POINT(29.16907261866958 -95.85445912774424) bank51075 +51076 POINT(30.529091275373098 -95.00809048270584) bank51076 +51077 POINT(30.474340506256954 -95.89562887681633) bank51077 +51078 POINT(29.29079505247136 -96.35258622972282) bank51078 +51079 POINT(30.270292218787755 -94.91688264070851) bank51079 +51080 POINT(29.76789907481178 -96.112053414974) bank51080 +51081 POINT(28.920878384079113 -95.32663870888908) bank51081 +51082 POINT(29.06304601445669 -94.8817506327394) bank51082 +51083 POINT(29.381113311304762 -95.19269270233966) bank51083 +51084 POINT(30.542961096043996 -94.37743567783689) bank51084 +51085 POINT(29.381497418391145 -95.91527382068202) bank51085 +51086 POINT(29.52389033896576 -94.4082527334292) bank51086 +51087 POINT(29.074944592976763 -95.08437268573971) bank51087 +51088 POINT(29.59599444479565 -95.56833972273773) bank51088 +51089 POINT(30.69461757625929 -94.86308776076679) bank51089 +51090 POINT(30.352886591052865 -95.5805803133762) bank51090 +51091 POINT(29.381265278713546 -96.2627898774429) bank51091 +51092 POINT(29.93565847640272 -95.54810742614315) bank51092 +51093 POINT(29.16259826918756 -96.31316641822016) bank51093 +51094 POINT(29.41784996577115 -95.19429842509612) bank51094 +51095 POINT(30.679254325230218 -94.62707541830584) bank51095 +51096 POINT(30.305489644755884 -94.52793635086891) bank51096 +51097 POINT(29.512181246264483 -95.35143493554499) bank51097 +51098 POINT(29.02877254165927 -96.1618200684086) bank51098 +51099 POINT(29.764278172145342 -94.63823211605624) bank51099 +51100 POINT(29.56815749400935 -95.29561251461493) bank51100 +51101 POINT(29.208120071765894 -95.27923166027203) bank51101 +51102 POINT(29.06297568875922 -94.64965550189966) bank51102 +51103 POINT(28.85452112932781 -95.15449619382623) bank51103 +51104 POINT(29.94586267818919 -95.49122554715787) bank51104 +51105 POINT(30.512310705854556 -95.68116636392963) bank51105 +51106 POINT(29.172715839072353 -95.37163428395948) bank51106 +51107 POINT(29.393017731357226 -95.04147760070643) bank51107 +51108 POINT(28.945953371942135 -94.65432425487975) bank51108 +51109 POINT(30.36481764516567 -95.06600361506054) bank51109 +51110 POINT(29.251126484029093 -94.98114637607257) bank51110 +51111 POINT(29.348202734474036 -95.19351137193115) bank51111 +51112 POINT(29.709170760718944 -95.23867468067478) bank51112 +51113 POINT(29.06375208807446 -95.2254082474022) bank51113 +51114 POINT(29.1181688832329 -95.77652298265372) bank51114 +51115 POINT(29.888888384597607 -96.12617476095924) bank51115 +51116 POINT(30.533596901353125 -95.71105169920855) bank51116 +51117 POINT(29.127538621375578 -94.95895124304093) bank51117 +51118 POINT(29.00635142347845 -95.48620757149493) bank51118 +51119 POINT(29.873965614263863 -95.31546146702993) bank51119 +51120 POINT(29.207160824731883 -95.53218622223298) bank51120 +51121 POINT(29.158649409779123 -94.77382293069246) bank51121 +51122 POINT(29.50211648709094 -95.26846867135632) bank51122 +51123 POINT(29.60933093088816 -96.28944610929912) bank51123 +51124 POINT(30.230213275494844 -95.85149349191931) bank51124 +51125 POINT(29.81704426470984 -96.08123555628949) bank51125 +51126 POINT(28.85693820030238 -94.38994743686118) bank51126 +51127 POINT(28.838539755189775 -94.71445403949525) bank51127 +51128 POINT(30.549309059134426 -95.58194181793826) bank51128 +51129 POINT(28.93663986801511 -95.83534610089794) bank51129 +51130 POINT(29.699002049001066 -94.77630286223142) bank51130 +51131 POINT(29.200423516798534 -94.37401655862651) bank51131 +51132 POINT(30.65292625095797 -94.78547682392356) bank51132 +51133 POINT(29.59586891284068 -95.5392219378448) bank51133 +51134 POINT(30.076984980849325 -96.33753712956494) bank51134 +51135 POINT(30.085542746176646 -95.01863888477088) bank51135 +51136 POINT(29.814752744936676 -94.39402661651786) bank51136 +51137 POINT(28.983810261994613 -95.36485564558234) bank51137 +51138 POINT(30.144822529296807 -95.03873576624083) bank51138 +51139 POINT(29.9536851199642 -95.29562979234655) bank51139 +51140 POINT(28.845319690833467 -94.41609975674295) bank51140 +51141 POINT(30.198592466743765 -95.29689380060334) bank51141 +51142 POINT(30.043209212595418 -94.56430745587762) bank51142 +51143 POINT(29.964970441159 -94.48592107255568) bank51143 +51144 POINT(29.060191887581627 -95.5971358104114) bank51144 +51145 POINT(29.28951587017735 -95.20375244116966) bank51145 +51146 POINT(29.902993652388123 -95.39037731394366) bank51146 +51147 POINT(30.4834652114947 -94.6446160437778) bank51147 +51148 POINT(29.30096242436968 -95.04340946610073) bank51148 +51149 POINT(29.27115024381131 -95.14143859511178) bank51149 +51150 POINT(30.43598147553189 -95.18275434480675) bank51150 +51151 POINT(29.222372504506485 -95.8716305201122) bank51151 +51152 POINT(30.759067067915414 -94.50308351981822) bank51152 +51153 POINT(30.679130813725436 -95.11651064895219) bank51153 +51154 POINT(29.935392195979766 -94.48575361816818) bank51154 +51155 POINT(30.35141175017223 -94.64993774347175) bank51155 +51156 POINT(28.765263549426923 -95.92074292811861) bank51156 +51157 POINT(28.820060705575386 -96.05925497131408) bank51157 +51158 POINT(29.597402895116847 -95.00643251701426) bank51158 +51159 POINT(29.918595882995877 -96.15073802023066) bank51159 +51160 POINT(29.60625576702403 -96.03927726122222) bank51160 +51161 POINT(30.663342523167145 -96.26268726803734) bank51161 +51162 POINT(28.962197268572446 -94.66930885302908) bank51162 +51163 POINT(30.410983574654722 -95.99603783141153) bank51163 +51164 POINT(30.40299089423587 -95.54524179300748) bank51164 +51165 POINT(29.906223171473943 -95.27122543760093) bank51165 +51166 POINT(30.46210383500718 -94.84409450577638) bank51166 +51167 POINT(29.943783591879914 -94.92959408719722) bank51167 +51168 POINT(28.900084063288507 -96.08367309388487) bank51168 +51169 POINT(30.44645411498176 -95.28570263374017) bank51169 +51170 POINT(29.89608431635471 -94.9223660056214) bank51170 +51171 POINT(30.37234826287109 -94.65575501030064) bank51171 +51172 POINT(28.773203604357217 -94.88314146477455) bank51172 +51173 POINT(29.863727389593233 -94.7052238412632) bank51173 +51174 POINT(29.231783554000913 -94.45404919803384) bank51174 +51175 POINT(30.117172385078824 -94.66648428819096) bank51175 +51176 POINT(29.98797473508039 -95.19715254953226) bank51176 +51177 POINT(30.29556561461257 -96.06858666821249) bank51177 +51178 POINT(30.32091805658246 -95.64133647606076) bank51178 +51179 POINT(30.118354482451693 -95.59024418856099) bank51179 +51180 POINT(29.560975997667807 -94.72383789013143) bank51180 +51181 POINT(30.293413625356 -96.24578245170171) bank51181 +51182 POINT(28.83443634946118 -94.45573197654858) bank51182 +51183 POINT(29.633033321088107 -94.64289424437453) bank51183 +51184 POINT(28.94100146476971 -95.8105558633836) bank51184 +51185 POINT(30.477114084983782 -95.02583579137648) bank51185 +51186 POINT(29.131218011824032 -94.55663041025085) bank51186 +51187 POINT(30.647608045502427 -94.92119699567498) bank51187 +51188 POINT(29.423471126272997 -94.97289033331013) bank51188 +51189 POINT(30.131658478919714 -94.9110538121528) bank51189 +51190 POINT(29.047095457110114 -95.36166326605148) bank51190 +51191 POINT(30.441765206594756 -96.34965918792595) bank51191 +51192 POINT(30.52678763745806 -94.78562481143427) bank51192 +51193 POINT(30.45124160799603 -94.4200056064978) bank51193 +51194 POINT(29.913388464990796 -95.0134010084356) bank51194 +51195 POINT(29.697512733510326 -94.43810484560593) bank51195 +51196 POINT(29.81118465301301 -94.92793992162235) bank51196 +51197 POINT(30.512863782315392 -95.83570534905171) bank51197 +51198 POINT(30.266545628180012 -94.83082686530275) bank51198 +51199 POINT(29.81026532753574 -94.79347256503323) bank51199 +51200 POINT(30.727826191067507 -94.79171542586738) bank51200 +51201 POINT(30.019525602898657 -95.71247839138415) bank51201 +51202 POINT(30.263311045284414 -94.56233661593299) bank51202 +51203 POINT(29.130192550808705 -95.39749461924977) bank51203 +51204 POINT(28.939169860309814 -96.07817982020225) bank51204 +51205 POINT(29.64968604706878 -95.47446188329454) bank51205 +51206 POINT(29.71623481883243 -95.87676092899505) bank51206 +51207 POINT(30.523433955383332 -95.96747505318781) bank51207 +51208 POINT(30.14746680163966 -96.35525029603178) bank51208 +51209 POINT(30.20943633782558 -94.9786114687974) bank51209 +51210 POINT(30.746028619281628 -94.66706823902676) bank51210 +51211 POINT(28.927666743740478 -95.5218630435754) bank51211 +51212 POINT(29.87193604990777 -95.87426542439063) bank51212 +51213 POINT(29.357230254553244 -96.21648725339087) bank51213 +51214 POINT(30.5656763129274 -95.15089973923851) bank51214 +51215 POINT(29.219005090993743 -95.56529137185225) bank51215 +51216 POINT(29.246688625131227 -95.13516003923128) bank51216 +51217 POINT(30.328310308291204 -96.098754529575) bank51217 +51218 POINT(29.279932559125253 -95.19711089923936) bank51218 +51219 POINT(30.552111297835353 -94.44730173197186) bank51219 +51220 POINT(29.773239099052358 -95.51685116516855) bank51220 +51221 POINT(30.21137317123977 -95.67641513095126) bank51221 +51222 POINT(30.1648082453434 -96.01662735030001) bank51222 +51223 POINT(29.103561540727778 -95.65816091971239) bank51223 +51224 POINT(30.542567419299395 -94.7743458803542) bank51224 +51225 POINT(30.621336876166424 -95.64532430785717) bank51225 +51226 POINT(30.1227783813933 -96.14541222532966) bank51226 +51227 POINT(30.36429663173821 -95.8871815616621) bank51227 +51228 POINT(30.442652496798644 -96.17424776696181) bank51228 +51229 POINT(28.906540321438367 -95.65764897216492) bank51229 +51230 POINT(28.847480919286077 -95.92193480054712) bank51230 +51231 POINT(29.66527013105693 -95.18534712788195) bank51231 +51232 POINT(30.29449815456143 -96.2506857566636) bank51232 +51233 POINT(30.191638207026898 -94.45266913944822) bank51233 +51234 POINT(28.850114031273982 -95.60645981386637) bank51234 +51235 POINT(28.790920683229764 -95.33517056791491) bank51235 +51236 POINT(30.75656752678634 -96.01773543521757) bank51236 +51237 POINT(30.052864799781613 -95.71953916377069) bank51237 +51238 POINT(29.201028245106727 -95.78010571342564) bank51238 +51239 POINT(29.09912082333541 -96.12795388771144) bank51239 +51240 POINT(30.221852172550257 -96.01225194083023) bank51240 +51241 POINT(29.865659065833952 -96.09462489416778) bank51241 +51242 POINT(29.062291334623474 -95.63739348213247) bank51242 +51243 POINT(30.637312842858414 -96.20313978160509) bank51243 +51244 POINT(30.496621662304083 -94.46874421845213) bank51244 +51245 POINT(29.60951950884356 -96.28867093992872) bank51245 +51246 POINT(29.663711889388956 -95.32504185150358) bank51246 +51247 POINT(29.166960289591884 -95.29998793386116) bank51247 +51248 POINT(30.488236223984337 -96.22540491007533) bank51248 +51249 POINT(30.664874317788502 -94.82598558329342) bank51249 +51250 POINT(28.983518771786688 -95.3636458928559) bank51250 +51251 POINT(29.37108657270866 -95.75395701432495) bank51251 +51252 POINT(30.020945176163494 -96.24655277727092) bank51252 +51253 POINT(28.766163459090002 -95.02401736689733) bank51253 +51254 POINT(29.312768672561585 -94.80796950788097) bank51254 +51255 POINT(29.440953696446744 -94.8366836411772) bank51255 +51256 POINT(29.988038591978345 -96.164752152338) bank51256 +51257 POINT(30.62261048012665 -95.40593142733005) bank51257 +51258 POINT(29.614247393590286 -94.86247194073634) bank51258 +51259 POINT(28.775389456394674 -96.28580599405545) bank51259 +51260 POINT(30.701606899072367 -94.44755854576259) bank51260 +51261 POINT(29.74267964061671 -95.14189348529057) bank51261 +51262 POINT(30.504313682570984 -96.22810270523524) bank51262 +51263 POINT(29.235336390840498 -95.34031587543063) bank51263 +51264 POINT(30.66603727618967 -94.53832292715792) bank51264 +51265 POINT(30.374528012764507 -96.07664468437785) bank51265 +51266 POINT(30.261949252461715 -95.26900864019147) bank51266 +51267 POINT(29.991623592315925 -96.34342140042648) bank51267 +51268 POINT(29.84863293211448 -94.50398409918337) bank51268 +51269 POINT(30.61008911294853 -95.88398593255738) bank51269 +51270 POINT(30.647782901357964 -94.85305946507368) bank51270 +51271 POINT(30.30733214604113 -95.17822649109758) bank51271 +51272 POINT(29.18179520834623 -95.07194928071834) bank51272 +51273 POINT(30.412863510733672 -95.03561678398219) bank51273 +51274 POINT(30.11857558344216 -94.70162524115598) bank51274 +51275 POINT(30.21775081399366 -94.76804000745875) bank51275 +51276 POINT(30.34415836594148 -95.08724763427468) bank51276 +51277 POINT(29.20465870718154 -95.84842930466576) bank51277 +51278 POINT(28.939729900292605 -95.6911717479157) bank51278 +51279 POINT(29.14053370694305 -96.33840410111002) bank51279 +51280 POINT(29.54825386049409 -94.39267918946504) bank51280 +51281 POINT(30.50467229375427 -96.35955043633199) bank51281 +51282 POINT(30.663917732448205 -95.69460361135704) bank51282 +51283 POINT(29.130753826985462 -95.55367766937945) bank51283 +51284 POINT(29.915848659052536 -95.20359372152566) bank51284 +51285 POINT(30.35519378687977 -96.33529477464425) bank51285 +51286 POINT(29.32107114794291 -94.43509193109284) bank51286 +51287 POINT(29.738337110778332 -94.6011165234013) bank51287 +51288 POINT(30.156580991247207 -95.89993793934212) bank51288 +51289 POINT(29.61254438887749 -94.6020793406053) bank51289 +51290 POINT(29.58507925472034 -95.12535825508827) bank51290 +51291 POINT(30.305254159977814 -95.3929172280158) bank51291 +51292 POINT(28.787603213255263 -96.31519499667685) bank51292 +51293 POINT(30.00271671275711 -95.40117147925996) bank51293 +51294 POINT(30.205250044555648 -94.97108966143502) bank51294 +51295 POINT(30.25665108161109 -94.90469185298987) bank51295 +51296 POINT(30.746158639693377 -96.30642638596731) bank51296 +51297 POINT(29.557423597173376 -94.66638105996431) bank51297 +51298 POINT(30.429769953210908 -94.54649445979922) bank51298 +51299 POINT(29.66809403017079 -94.80293013657594) bank51299 +51300 POINT(29.7118113783787 -95.86962033781109) bank51300 +51301 POINT(29.76795642816274 -95.64279759762049) bank51301 +51302 POINT(30.399303377657272 -95.57728220415214) bank51302 +51303 POINT(30.59471931202786 -95.90334153429363) bank51303 +51304 POINT(28.871781102621657 -95.79265732890086) bank51304 +51305 POINT(29.519773446043416 -94.68605872241756) bank51305 +51306 POINT(28.85412420576357 -95.69509979223793) bank51306 +51307 POINT(29.64744359541935 -96.1147681003012) bank51307 +51308 POINT(30.74962493760025 -95.29102460986032) bank51308 +51309 POINT(28.920758521174672 -95.49115390283407) bank51309 +51310 POINT(29.97071580269816 -95.40521443323972) bank51310 +51311 POINT(29.711672973201733 -95.29280642372925) bank51311 +51312 POINT(29.47533421911191 -94.42764067970319) bank51312 +51313 POINT(30.17245900501602 -96.08479289954457) bank51313 +51314 POINT(30.11675531930404 -95.33641524448886) bank51314 +51315 POINT(29.153882610170825 -95.54593193690913) bank51315 +51316 POINT(29.919143632304 -94.89138299166459) bank51316 +51317 POINT(28.943124191303216 -95.54731270963292) bank51317 +51318 POINT(29.233162446181016 -94.41937315611855) bank51318 +51319 POINT(30.2228529034318 -95.42978780381424) bank51319 +51320 POINT(28.881887867955676 -95.57321901702191) bank51320 +51321 POINT(30.190094810147393 -95.11179155872878) bank51321 +51322 POINT(28.91796111611167 -94.95866331020437) bank51322 +51323 POINT(29.77770916865136 -95.50834485603087) bank51323 +51324 POINT(29.61085315976276 -94.69923912027159) bank51324 +51325 POINT(29.518814470652188 -95.68501972783898) bank51325 +51326 POINT(29.843058130683985 -95.60063040017414) bank51326 +51327 POINT(29.656568444929384 -96.34357492451709) bank51327 +51328 POINT(29.209208145249836 -95.09838936888428) bank51328 +51329 POINT(29.559518051074683 -95.59211734078355) bank51329 +51330 POINT(29.696055265464295 -94.71238602390217) bank51330 +51331 POINT(29.687959855310506 -94.5937358946228) bank51331 +51332 POINT(30.08374904483578 -94.6313619452933) bank51332 +51333 POINT(29.81286447951822 -94.61218209957714) bank51333 +51334 POINT(29.5010627804476 -94.75988749777439) bank51334 +51335 POINT(29.857419533981545 -94.96618145495134) bank51335 +51336 POINT(30.396839320290603 -94.45714729454316) bank51336 +51337 POINT(29.506534924630767 -95.46313350928409) bank51337 +51338 POINT(28.801849005135363 -94.76746202890561) bank51338 +51339 POINT(30.353500525277454 -95.32857338013847) bank51339 +51340 POINT(29.388791409423323 -95.60816998960509) bank51340 +51341 POINT(30.439408152989174 -96.13638699599628) bank51341 +51342 POINT(30.3352573692182 -95.98475118491193) bank51342 +51343 POINT(28.870352237923957 -95.17494140259154) bank51343 +51344 POINT(29.690540802862596 -94.93711952817368) bank51344 +51345 POINT(29.90276070871287 -95.67220978297824) bank51345 +51346 POINT(29.030202597997935 -94.56839769739614) bank51346 +51347 POINT(30.564541685179794 -94.71125792344337) bank51347 +51348 POINT(29.98882276720157 -94.67124546590932) bank51348 +51349 POINT(30.139784037944636 -94.81779265226821) bank51349 +51350 POINT(29.051175295341313 -95.66683243563388) bank51350 +51351 POINT(30.669584086089213 -94.88510157636003) bank51351 +51352 POINT(29.7196451169631 -94.76353365531413) bank51352 +51353 POINT(30.301424025515193 -95.50566824897354) bank51353 +51354 POINT(29.65355885985811 -96.0162759143924) bank51354 +51355 POINT(30.444557133256968 -95.59579065409454) bank51355 +51356 POINT(30.455934671365384 -95.54866573003395) bank51356 +51357 POINT(30.296532296504918 -94.86582580312302) bank51357 +51358 POINT(29.226186109507733 -95.94074062636852) bank51358 +51359 POINT(29.535400992712233 -95.31960850344734) bank51359 +51360 POINT(30.030841026454095 -94.99521492881804) bank51360 +51361 POINT(29.701109524612896 -95.68546588202308) bank51361 +51362 POINT(29.992927467960783 -95.96454532189749) bank51362 +51363 POINT(28.7801650691471 -95.57949877363647) bank51363 +51364 POINT(29.614862416219324 -96.07650064218248) bank51364 +51365 POINT(30.33631480851738 -94.68306753902675) bank51365 +51366 POINT(28.8202855336588 -96.28909743912303) bank51366 +51367 POINT(29.101545220387585 -95.81178579733015) bank51367 +51368 POINT(28.874757447586106 -94.85972710998219) bank51368 +51369 POINT(30.116203878304326 -95.13950174250616) bank51369 +51370 POINT(30.599135499613485 -94.70807034648489) bank51370 +51371 POINT(29.695248691043705 -95.53016362595369) bank51371 +51372 POINT(30.5645705897641 -96.05787759564286) bank51372 +51373 POINT(28.901121742408407 -95.23263187550945) bank51373 +51374 POINT(29.95480306806491 -95.83863483056872) bank51374 +51375 POINT(29.483707894025112 -95.14537473197245) bank51375 +51376 POINT(28.98875193464521 -95.92213773687878) bank51376 +51377 POINT(30.280695095182885 -95.05703461230252) bank51377 +51378 POINT(30.153438188490068 -95.31018101203087) bank51378 +51379 POINT(29.760654929830345 -95.30369262456395) bank51379 +51380 POINT(29.419727087045988 -94.4923084487492) bank51380 +51381 POINT(30.556228741749205 -94.50786496532676) bank51381 +51382 POINT(30.11211087218635 -96.29181633335917) bank51382 +51383 POINT(28.914365115964923 -95.81797195660606) bank51383 +51384 POINT(29.624970915189675 -94.7224761953498) bank51384 +51385 POINT(30.451307713032595 -95.34350692452806) bank51385 +51386 POINT(28.836568538762773 -95.68192605605557) bank51386 +51387 POINT(28.81667726367137 -94.96578730461779) bank51387 +51388 POINT(30.271741061023643 -95.34870572779865) bank51388 +51389 POINT(29.836216206301042 -95.86471633517627) bank51389 +51390 POINT(28.83811420678644 -95.4216808486013) bank51390 +51391 POINT(30.3234625869486 -95.74232332616768) bank51391 +51392 POINT(29.5233912473806 -94.91003845844949) bank51392 +51393 POINT(30.1030355332393 -95.69948506773127) bank51393 +51394 POINT(30.336063333849843 -95.34469982726695) bank51394 +51395 POINT(30.172620413936546 -95.74324555882886) bank51395 +51396 POINT(28.92031622669421 -95.42702527648132) bank51396 +51397 POINT(28.817553060370976 -94.47166880199357) bank51397 +51398 POINT(30.555147827980647 -95.22987410580889) bank51398 +51399 POINT(30.004583824872192 -94.8732329074756) bank51399 +51400 POINT(28.976379777202947 -94.57636868708633) bank51400 +51401 POINT(29.282248240317113 -95.85559126459677) bank51401 +51402 POINT(30.23099980585534 -94.52323996581865) bank51402 +51403 POINT(30.149188611017678 -95.1343338030332) bank51403 +51404 POINT(29.08220702465655 -94.79876215243496) bank51404 +51405 POINT(30.50467466604542 -94.81829808374806) bank51405 +51406 POINT(30.711978231100897 -95.22123363317687) bank51406 +51407 POINT(29.826420754606318 -95.20343812584338) bank51407 +51408 POINT(29.067002764736102 -96.12594703056868) bank51408 +51409 POINT(29.699051144835096 -95.21029064145118) bank51409 +51410 POINT(30.66904114571874 -95.7280445458957) bank51410 +51411 POINT(29.835820542562278 -95.37436434920498) bank51411 +51412 POINT(28.83783898381163 -95.59956797319738) bank51412 +51413 POINT(28.99703232357344 -95.6669745951608) bank51413 +51414 POINT(30.67059834885973 -96.31646539870133) bank51414 +51415 POINT(30.584894336048833 -95.83951350277049) bank51415 +51416 POINT(30.301523665554345 -94.86851792230125) bank51416 +51417 POINT(29.922611165733016 -94.6775100762485) bank51417 +51418 POINT(30.67444860279963 -95.11892316922471) bank51418 +51419 POINT(30.657548755985545 -94.39917283524389) bank51419 +51420 POINT(29.93394149845959 -95.66485775308392) bank51420 +51421 POINT(29.643909456851457 -96.26119274056285) bank51421 +51422 POINT(30.537607456613387 -96.12115997105886) bank51422 +51423 POINT(30.14198885208989 -94.71990880536094) bank51423 +51424 POINT(30.26548295559553 -95.03388051373801) bank51424 +51425 POINT(29.43248997748396 -94.98973105967724) bank51425 +51426 POINT(30.1602047393881 -95.22634228235337) bank51426 +51427 POINT(29.690944632485056 -96.13207720260566) bank51427 +51428 POINT(28.8705752478283 -95.31591836707757) bank51428 +51429 POINT(30.20772753152015 -94.85916579877563) bank51429 +51430 POINT(30.24496227710801 -96.02414804062734) bank51430 +51431 POINT(29.105224168328963 -95.31273095398053) bank51431 +51432 POINT(30.502318164583563 -95.9538535008844) bank51432 +51433 POINT(29.70961011370755 -95.89022284572448) bank51433 +51434 POINT(29.767944409764397 -95.21271508866906) bank51434 +51435 POINT(30.654165387065955 -96.20884946183668) bank51435 +51436 POINT(29.154226889420364 -94.6054361543783) bank51436 +51437 POINT(30.056623611113455 -95.61975704612512) bank51437 +51438 POINT(29.665739234334104 -95.19052923909369) bank51438 +51439 POINT(30.437312183532796 -95.85428732369199) bank51439 +51440 POINT(29.148142792927903 -96.20573993362919) bank51440 +51441 POINT(30.55511616391255 -95.9992614655958) bank51441 +51442 POINT(29.26521418145884 -94.38204322639308) bank51442 +51443 POINT(29.111191869977542 -95.76092764823663) bank51443 +51444 POINT(29.10465696001834 -95.62081574519702) bank51444 +51445 POINT(29.032346537971367 -95.43992800618922) bank51445 +51446 POINT(29.230579706445095 -95.7458656021631) bank51446 +51447 POINT(30.241454061771908 -94.65642237887138) bank51447 +51448 POINT(28.988060080694122 -95.94155544007971) bank51448 +51449 POINT(30.740649869208745 -94.43704586593147) bank51449 +51450 POINT(29.550773881358097 -96.27988849305494) bank51450 +51451 POINT(30.141702762752814 -95.3512236384635) bank51451 +51452 POINT(30.315724417717547 -96.16302349026691) bank51452 +51453 POINT(30.75548060837986 -96.19624977311965) bank51453 +51454 POINT(30.153342152918587 -95.43284588182502) bank51454 +51455 POINT(29.94345676933319 -95.7915859960763) bank51455 +51456 POINT(29.70806914269707 -96.16911016661507) bank51456 +51457 POINT(30.74277365129275 -94.81377640609837) bank51457 +51458 POINT(29.91755771019887 -94.68374383180435) bank51458 +51459 POINT(29.935944301532874 -96.32479036582565) bank51459 +51460 POINT(29.124520449687225 -94.89051283970181) bank51460 +51461 POINT(28.849055990324324 -95.11835047590006) bank51461 +51462 POINT(30.367863963143535 -95.39899384671332) bank51462 +51463 POINT(29.11010281356537 -96.21028344067348) bank51463 +51464 POINT(28.787300934189762 -94.43624184701196) bank51464 +51465 POINT(29.54184116866617 -95.41945607554702) bank51465 +51466 POINT(29.239382342103074 -95.68807566815032) bank51466 +51467 POINT(28.91594104037423 -95.90367297714242) bank51467 +51468 POINT(28.803688631717417 -95.48434150098657) bank51468 +51469 POINT(29.482859358707543 -94.4818179810961) bank51469 +51470 POINT(29.89118231794357 -96.19544778689014) bank51470 +51471 POINT(29.033031301095033 -96.05843027062124) bank51471 +51472 POINT(29.388972031575754 -96.09605547152566) bank51472 +51473 POINT(29.22507365579637 -94.86424996312678) bank51473 +51474 POINT(29.99859023230683 -94.61515212615552) bank51474 +51475 POINT(29.745280422723113 -94.67717491827605) bank51475 +51476 POINT(30.271588303494173 -94.48879222254062) bank51476 +51477 POINT(30.623611324308172 -95.35964773537151) bank51477 +51478 POINT(29.759273608231794 -94.73780107309176) bank51478 +51479 POINT(29.62399570637696 -95.47593230648748) bank51479 +51480 POINT(30.128601600644938 -96.14314586484947) bank51480 +51481 POINT(29.23403220585888 -96.01529926098115) bank51481 +51482 POINT(29.41351520520432 -94.4391519016337) bank51482 +51483 POINT(29.959253121139056 -95.99785983399602) bank51483 +51484 POINT(30.316093306083143 -95.83460134682532) bank51484 +51485 POINT(29.347910547639152 -95.02853555548201) bank51485 +51486 POINT(29.502076717312843 -95.75408246832936) bank51486 +51487 POINT(29.936597154990896 -95.83500880758316) bank51487 +51488 POINT(29.27090191453542 -95.37397304688615) bank51488 +51489 POINT(29.13477514341488 -96.07252409961843) bank51489 +51490 POINT(30.12900222415265 -95.60493927556814) bank51490 +51491 POINT(29.011537736363927 -96.17151316683955) bank51491 +51492 POINT(30.40820071792305 -95.470111109505) bank51492 +51493 POINT(29.64625293614813 -95.62700201435058) bank51493 +51494 POINT(30.23163439809663 -95.81688816125647) bank51494 +51495 POINT(28.98888103556335 -94.70394627276916) bank51495 +51496 POINT(29.591326047195494 -95.33970633571946) bank51496 +51497 POINT(28.887949368615416 -95.5302493185517) bank51497 +51498 POINT(30.32946795789696 -94.87596753353891) bank51498 +51499 POINT(28.985957378100863 -95.58538957073574) bank51499 +51500 POINT(29.403412790814496 -96.13978879940291) bank51500 +51501 POINT(29.799507311265945 -96.02046698164365) bank51501 +51502 POINT(28.885923752391886 -96.25367080729217) bank51502 +51503 POINT(29.251258576448457 -95.38222095370406) bank51503 +51504 POINT(30.408878641084215 -95.34143520037443) bank51504 +51505 POINT(29.305581381543547 -95.61849092219029) bank51505 +51506 POINT(30.23623634823662 -94.90195348627549) bank51506 +51507 POINT(29.290607517588818 -96.29849813881711) bank51507 +51508 POINT(29.248418905587037 -95.20982317166727) bank51508 +51509 POINT(30.694561098429975 -95.02365902595882) bank51509 +51510 POINT(29.07257401037464 -94.77690959634734) bank51510 +51511 POINT(29.134724441454143 -94.69494272087026) bank51511 +51512 POINT(29.266374618809007 -95.85195236744683) bank51512 +51513 POINT(30.370256143828957 -95.93109757677551) bank51513 +51514 POINT(29.808023120166048 -95.76988766912584) bank51514 +51515 POINT(29.289022454892123 -94.80260878967097) bank51515 +51516 POINT(29.004929842984552 -95.44323408268966) bank51516 +51517 POINT(29.78694498163697 -94.42309497626411) bank51517 +51518 POINT(29.532749592346747 -95.87608499658637) bank51518 +51519 POINT(29.369741520723093 -95.89345286398373) bank51519 +51520 POINT(29.978009229470942 -94.45027278326461) bank51520 +51521 POINT(28.987241088245295 -95.35874857644772) bank51521 +51522 POINT(29.539720491808886 -95.58017763280732) bank51522 +51523 POINT(29.571457436223493 -96.30674792765684) bank51523 +51524 POINT(28.865118845206503 -94.81137231613896) bank51524 +51525 POINT(30.040220063376605 -94.87972812511413) bank51525 +51526 POINT(29.514284291648995 -94.8605939838733) bank51526 +51527 POINT(29.72145705811616 -95.06651370464473) bank51527 +51528 POINT(30.49679498615397 -96.12150318699904) bank51528 +51529 POINT(28.967646879863512 -96.3345703919116) bank51529 +51530 POINT(29.727924505012396 -96.118836411959) bank51530 +51531 POINT(28.76135667447464 -94.99538844974849) bank51531 +51532 POINT(30.478139200199987 -95.1277027493699) bank51532 +51533 POINT(29.774274510096763 -95.6583646099054) bank51533 +51534 POINT(28.967627394116857 -95.8993518778594) bank51534 +51535 POINT(29.39906073463071 -94.48698434596756) bank51535 +51536 POINT(29.77519046851676 -95.77355444732899) bank51536 +51537 POINT(29.772218405269687 -96.35551830194625) bank51537 +51538 POINT(29.426657492108337 -95.63578030832913) bank51538 +51539 POINT(29.27401639666255 -96.30990522628275) bank51539 +51540 POINT(29.131588860648723 -95.63402910753959) bank51540 +51541 POINT(29.686671635441588 -95.56209299372098) bank51541 +51542 POINT(29.120807601175954 -96.03893456250255) bank51542 +51543 POINT(30.733176147888283 -96.24946605377872) bank51543 +51544 POINT(30.065093789695688 -94.4835706726434) bank51544 +51545 POINT(30.061804602500708 -94.72001942155667) bank51545 +51546 POINT(29.452569008788977 -95.44482028032192) bank51546 +51547 POINT(28.952820748482107 -95.65323706765653) bank51547 +51548 POINT(29.15032932395098 -95.0315587175761) bank51548 +51549 POINT(29.641949873965796 -96.13726365421148) bank51549 +51550 POINT(30.677123532661412 -95.69716170292931) bank51550 +51551 POINT(30.37425729934224 -95.43624394988112) bank51551 +51552 POINT(30.450581148016276 -95.40719190113101) bank51552 +51553 POINT(29.913975241483833 -94.4932240110506) bank51553 +51554 POINT(29.13913025077654 -95.52934248900945) bank51554 +51555 POINT(28.770153271632605 -95.9509871818669) bank51555 +51556 POINT(29.665047979269012 -96.23396542693088) bank51556 +51557 POINT(29.85611004932773 -94.7078241970315) bank51557 +51558 POINT(29.765617732008824 -95.55722548072995) bank51558 +51559 POINT(30.04089803324341 -96.35063611389458) bank51559 +51560 POINT(30.006144695683034 -94.57515752642696) bank51560 +51561 POINT(29.054915765517798 -96.33966742846911) bank51561 +51562 POINT(29.213955052204724 -95.56388919724567) bank51562 +51563 POINT(29.063391122180317 -96.17035190317199) bank51563 +51564 POINT(29.63456208723864 -96.06871117731878) bank51564 +51565 POINT(30.078249369367043 -96.09562366189472) bank51565 +51566 POINT(29.662887152039364 -95.23593363291087) bank51566 +51567 POINT(29.888886775994166 -94.80476222224108) bank51567 +51568 POINT(29.716183445303688 -94.94510935531991) bank51568 +51569 POINT(28.826500890112918 -94.62025840220686) bank51569 +51570 POINT(30.407151482814204 -95.7929847851253) bank51570 +51571 POINT(28.810416115647907 -95.56105275524075) bank51571 +51572 POINT(30.659277773092075 -95.39941406830549) bank51572 +51573 POINT(29.41375044522571 -95.19187076009148) bank51573 +51574 POINT(29.474960124717636 -94.80606278425519) bank51574 +51575 POINT(29.96334694894702 -95.74988477238479) bank51575 +51576 POINT(29.730991492892244 -94.99530345358791) bank51576 +51577 POINT(29.86633933121833 -95.83389472034087) bank51577 +51578 POINT(29.790230112519318 -96.03291231162027) bank51578 +51579 POINT(30.469398558709415 -94.45703707739477) bank51579 +51580 POINT(30.130024083201775 -94.91954298136378) bank51580 +51581 POINT(30.169846343409805 -95.51012690409819) bank51581 +51582 POINT(30.215879502995143 -95.03234345854254) bank51582 +51583 POINT(30.43270898213312 -95.38853730656659) bank51583 +51584 POINT(30.533064397057483 -96.27125442657255) bank51584 +51585 POINT(30.38662773572504 -95.8057075883771) bank51585 +51586 POINT(30.699457540102813 -96.1075092822887) bank51586 +51587 POINT(28.931953522187353 -95.09733236648566) bank51587 +51588 POINT(29.457401756364238 -94.53155971506544) bank51588 +51589 POINT(29.105495332058013 -94.84220943376444) bank51589 +51590 POINT(30.02829907137476 -95.33758218227482) bank51590 +51591 POINT(29.698850402644116 -94.92806948571364) bank51591 +51592 POINT(30.10543441370805 -94.90938243631835) bank51592 +51593 POINT(28.830384027275194 -96.24261567568695) bank51593 +51594 POINT(30.133814323249442 -95.06642615826885) bank51594 +51595 POINT(29.933498228049846 -95.01031753125228) bank51595 +51596 POINT(28.794566826795755 -95.59978394631803) bank51596 +51597 POINT(28.890719250644107 -95.21404941321022) bank51597 +51598 POINT(29.527888950181612 -94.74653273192642) bank51598 +51599 POINT(30.695294026977002 -95.820643768177) bank51599 +51600 POINT(30.44974396515226 -95.71184797653947) bank51600 +51601 POINT(29.373647312584968 -96.06492238096125) bank51601 +51602 POINT(30.74672284969579 -96.23922433306838) bank51602 +51603 POINT(29.3041310433497 -95.56642486361518) bank51603 +51604 POINT(30.745273825261783 -96.3410667639243) bank51604 +51605 POINT(29.27726577417244 -94.61605018087127) bank51605 +51606 POINT(30.42405933080438 -95.75154633158857) bank51606 +51607 POINT(30.217745959941205 -95.51331521337909) bank51607 +51608 POINT(29.081619031979468 -94.38246953257632) bank51608 +51609 POINT(30.238407722491267 -95.46223215872921) bank51609 +51610 POINT(28.803526333276174 -94.55079993971553) bank51610 +51611 POINT(29.981991298553748 -95.9528926778481) bank51611 +51612 POINT(29.147407369895067 -95.00869849008394) bank51612 +51613 POINT(29.53329719025792 -94.44586046885591) bank51613 +51614 POINT(29.373814289881913 -94.89187117357754) bank51614 +51615 POINT(29.235339913099303 -94.76108152774384) bank51615 +51616 POINT(29.380362654694558 -95.90392096792269) bank51616 +51617 POINT(29.884892741579844 -94.37390908515326) bank51617 +51618 POINT(30.1657668767114 -96.0227409713931) bank51618 +51619 POINT(30.213661996136178 -95.39353655338223) bank51619 +51620 POINT(30.140989961768188 -95.11526589527524) bank51620 +51621 POINT(29.287122030089428 -95.70649705720858) bank51621 +51622 POINT(29.03919639097904 -95.2569969912511) bank51622 +51623 POINT(29.19172352013897 -95.50819181085035) bank51623 +51624 POINT(30.711256355753235 -94.48369381690162) bank51624 +51625 POINT(28.9126018021363 -94.40099240856608) bank51625 +51626 POINT(30.437219595623333 -96.11093285682863) bank51626 +51627 POINT(30.707473332879708 -96.05297986187648) bank51627 +51628 POINT(30.185471524197233 -95.08031308347748) bank51628 +51629 POINT(29.89482874177692 -95.10274369844963) bank51629 +51630 POINT(30.696234216381583 -95.02499813569884) bank51630 +51631 POINT(29.63317552482635 -94.38807662629404) bank51631 +51632 POINT(29.3997513851479 -95.27098950625415) bank51632 +51633 POINT(30.105450486033035 -94.87290700430378) bank51633 +51634 POINT(30.42363404306074 -94.75433412525695) bank51634 +51635 POINT(29.93508877799941 -95.56935716930543) bank51635 +51636 POINT(28.88485563784954 -95.24820307749474) bank51636 +51637 POINT(30.175637937990004 -96.30436686331129) bank51637 +51638 POINT(29.78806773810327 -95.79772700604988) bank51638 +51639 POINT(30.303841430097233 -95.93781924560517) bank51639 +51640 POINT(30.325846150891053 -96.07158947709786) bank51640 +51641 POINT(29.635482574545385 -95.05985354255508) bank51641 +51642 POINT(30.22335250622307 -94.44096126060477) bank51642 +51643 POINT(30.31112375786029 -96.29833915794501) bank51643 +51644 POINT(28.92984393688793 -95.96678243909021) bank51644 +51645 POINT(30.760095757018007 -94.78304842486071) bank51645 +51646 POINT(30.16696712551695 -96.27410303479414) bank51646 +51647 POINT(29.71279139155047 -95.79509742637572) bank51647 +51648 POINT(29.709749063690847 -95.16463484266038) bank51648 +51649 POINT(29.141926582656282 -96.18476142786771) bank51649 +51650 POINT(29.037843873437488 -95.49607886799811) bank51650 +51651 POINT(30.374756020539007 -94.63562009606454) bank51651 +51652 POINT(29.330172212747517 -96.23517615451584) bank51652 +51653 POINT(29.617559380461202 -94.46171676787968) bank51653 +51654 POINT(30.059435135881678 -96.08018977194982) bank51654 +51655 POINT(30.4712722586665 -94.93491396778305) bank51655 +51656 POINT(29.009852452191847 -95.42826684655968) bank51656 +51657 POINT(29.439979537704428 -95.56024803098762) bank51657 +51658 POINT(30.003913004578266 -95.28308542843132) bank51658 +51659 POINT(29.88739288159207 -94.97720959584696) bank51659 +51660 POINT(30.406007321872334 -94.5563728015743) bank51660 +51661 POINT(29.582660089354054 -94.553994266566) bank51661 +51662 POINT(28.82065672331325 -96.09403572388165) bank51662 +51663 POINT(29.346295631940347 -95.65235503327638) bank51663 +51664 POINT(29.600382129660304 -94.85216414488741) bank51664 +51665 POINT(30.50932743893008 -94.6653010419) bank51665 +51666 POINT(29.752762255483976 -95.72192139347324) bank51666 +51667 POINT(29.752280294047914 -95.0768592914265) bank51667 +51668 POINT(29.76288971404623 -95.11811880585014) bank51668 +51669 POINT(30.190610005478344 -96.33969731225517) bank51669 +51670 POINT(30.33860088287363 -96.34601702369827) bank51670 +51671 POINT(29.8218849733358 -95.08304850736366) bank51671 +51672 POINT(30.089146333513472 -95.4037599595963) bank51672 +51673 POINT(29.16728953732379 -94.54303729535499) bank51673 +51674 POINT(30.43431449688321 -95.91948966050603) bank51674 +51675 POINT(30.596051688954578 -95.87703820402142) bank51675 +51676 POINT(30.601368504035655 -94.78127229445802) bank51676 +51677 POINT(29.35633825011341 -94.92023348427948) bank51677 +51678 POINT(30.27834243653839 -95.79992817106766) bank51678 +51679 POINT(30.703066335729204 -96.13370981324489) bank51679 +51680 POINT(30.100971877465867 -96.15345146340688) bank51680 +51681 POINT(29.437205831657227 -95.5660891909515) bank51681 +51682 POINT(29.03158299731349 -95.79546785128629) bank51682 +51683 POINT(29.233426120585776 -95.9384285854892) bank51683 +51684 POINT(29.862129455038154 -96.26142693972088) bank51684 +51685 POINT(30.002557629855218 -95.34923939960319) bank51685 +51686 POINT(29.275263020688953 -94.6500933346358) bank51686 +51687 POINT(29.246473298727718 -95.47890513274127) bank51687 +51688 POINT(29.62520802692516 -94.4506276936053) bank51688 +51689 POINT(29.37672645531748 -95.1613356490667) bank51689 +51690 POINT(29.684182514909658 -94.38021716953068) bank51690 +51691 POINT(30.301078534860796 -95.11017664543125) bank51691 +51692 POINT(30.557144158300403 -95.43696316735428) bank51692 +51693 POINT(29.731779007774616 -95.76281848598623) bank51693 +51694 POINT(30.72637557324364 -95.83620434807786) bank51694 +51695 POINT(29.201990494971493 -95.97206859801243) bank51695 +51696 POINT(30.252148950031298 -94.5862130441107) bank51696 +51697 POINT(29.84730033035472 -95.92761839390324) bank51697 +51698 POINT(30.416127112100625 -94.435903060376) bank51698 +51699 POINT(30.146995952103165 -95.74974143301701) bank51699 +51700 POINT(30.20965896141799 -96.1538071286181) bank51700 +51701 POINT(29.25407958498054 -95.00771429291551) bank51701 +51702 POINT(28.90718026488242 -94.47839276410865) bank51702 +51703 POINT(30.30249947045943 -95.3337146207704) bank51703 +51704 POINT(29.445214728019558 -95.77690708015625) bank51704 +51705 POINT(28.772078983038238 -95.25521330733282) bank51705 +51706 POINT(29.60847825570979 -95.2942156766898) bank51706 +51707 POINT(29.09626528289727 -96.35365186432558) bank51707 +51708 POINT(30.543659381398157 -96.31603419318878) bank51708 +51709 POINT(30.462408357116768 -94.84431393524552) bank51709 +51710 POINT(30.424662455753946 -95.8443689048918) bank51710 +51711 POINT(29.15258665611946 -95.6526440515681) bank51711 +51712 POINT(30.60597737824371 -95.09689813205833) bank51712 +51713 POINT(29.014355933602857 -95.91043523238271) bank51713 +51714 POINT(29.635119160279057 -95.37719827147069) bank51714 +51715 POINT(29.921973378117663 -94.50122663244275) bank51715 +51716 POINT(29.205956759203634 -96.1034065368136) bank51716 +51717 POINT(28.955713908835733 -95.46109698099362) bank51717 +51718 POINT(30.470016417482853 -95.75976263857567) bank51718 +51719 POINT(28.91829789479542 -95.54272010716774) bank51719 +51720 POINT(29.846596050392964 -94.93063968110613) bank51720 +51721 POINT(29.349669612411112 -95.55497625447552) bank51721 +51722 POINT(30.340778042245528 -95.83387668825324) bank51722 +51723 POINT(30.64376553670661 -96.36548857365976) bank51723 +51724 POINT(29.195617025363745 -95.06278735279457) bank51724 +51725 POINT(30.622989667012146 -95.59938723344588) bank51725 +51726 POINT(28.942425356816877 -95.95284379902053) bank51726 +51727 POINT(30.6350381400185 -96.29517792983765) bank51727 +51728 POINT(29.001792927362622 -94.52558463517109) bank51728 +51729 POINT(30.591532323847087 -94.64510957015884) bank51729 +51730 POINT(29.135056778794176 -94.37690785825521) bank51730 +51731 POINT(29.351551618807875 -95.12283960317973) bank51731 +51732 POINT(29.999823005247 -95.00661708802862) bank51732 +51733 POINT(29.214206491387998 -95.0185259426582) bank51733 +51734 POINT(30.070665244745335 -96.28157128105912) bank51734 +51735 POINT(28.778175563659946 -96.33423444211296) bank51735 +51736 POINT(29.976396264734586 -94.49743384200116) bank51736 +51737 POINT(29.06910644892997 -95.39988511041835) bank51737 +51738 POINT(30.41701132869259 -94.74998083842014) bank51738 +51739 POINT(29.123176955959643 -96.14565737181528) bank51739 +51740 POINT(30.58619272410608 -96.03743123101994) bank51740 +51741 POINT(29.684892946002385 -94.47069193827295) bank51741 +51742 POINT(29.67410526425039 -94.85295660372476) bank51742 +51743 POINT(29.60380271774711 -95.03609089183408) bank51743 +51744 POINT(29.738526045278057 -94.8060774780316) bank51744 +51745 POINT(30.6047064992476 -95.36517730034862) bank51745 +51746 POINT(30.03274700426348 -95.52608552707152) bank51746 +51747 POINT(30.421668623797917 -94.56169059788195) bank51747 +51748 POINT(29.638607976382826 -95.8283140397016) bank51748 +51749 POINT(28.773580047723975 -96.36295788941717) bank51749 +51750 POINT(29.406170123899116 -95.4031186922605) bank51750 +51751 POINT(29.014872033978854 -94.48431855992423) bank51751 +51752 POINT(29.92838235941974 -96.11885131727229) bank51752 +51753 POINT(29.42012380287858 -94.90869548270234) bank51753 +51754 POINT(29.732276702093298 -95.19501106419912) bank51754 +51755 POINT(30.312508106473967 -96.33055406311588) bank51755 +51756 POINT(29.290084590021614 -95.7268123357356) bank51756 +51757 POINT(30.666802944043237 -95.99905161198409) bank51757 +51758 POINT(29.91531473402026 -95.03173285822861) bank51758 +51759 POINT(29.778286283208153 -94.39533208771223) bank51759 +51760 POINT(29.17105915156783 -95.7355908156369) bank51760 +51761 POINT(30.187459222791038 -96.05469244249262) bank51761 +51762 POINT(30.003930590340758 -94.87703999001951) bank51762 +51763 POINT(30.294855070453995 -95.50867844331285) bank51763 +51764 POINT(29.916866577675293 -95.5707741574992) bank51764 +51765 POINT(29.79307583315415 -95.72105127809415) bank51765 +51766 POINT(28.806424080398305 -95.71835088512891) bank51766 +51767 POINT(29.68982962553577 -95.85352010320621) bank51767 +51768 POINT(30.43020779267139 -94.94150669398111) bank51768 +51769 POINT(28.906916527014225 -95.42400932015498) bank51769 +51770 POINT(30.71882595986781 -95.6304128969324) bank51770 +51771 POINT(29.254541968331623 -95.31203890382568) bank51771 +51772 POINT(30.71400411893073 -94.45826079766503) bank51772 +51773 POINT(29.09579727139811 -95.3459945116545) bank51773 +51774 POINT(29.12386585368438 -94.76888068828663) bank51774 +51775 POINT(29.435011130623703 -95.73503569683577) bank51775 +51776 POINT(30.604560874377704 -96.11434561204504) bank51776 +51777 POINT(29.152909656598037 -96.26676462514448) bank51777 +51778 POINT(29.68463931236237 -96.02161436113244) bank51778 +51779 POINT(29.506269965192338 -96.25716784233136) bank51779 +51780 POINT(28.881840587644895 -95.42194627401645) bank51780 +51781 POINT(28.983024685087123 -95.20210384872306) bank51781 +51782 POINT(30.61311009654583 -95.99683342363112) bank51782 +51783 POINT(30.39731519786508 -94.95296819904759) bank51783 +51784 POINT(30.299270392904457 -96.33332168977222) bank51784 +51785 POINT(29.449886122512407 -96.12416765181513) bank51785 +51786 POINT(29.971966300040563 -95.14033123442383) bank51786 +51787 POINT(29.518673466600227 -95.41089037177044) bank51787 +51788 POINT(28.768455207563342 -94.57145716160436) bank51788 +51789 POINT(30.15526824563516 -96.09700596125474) bank51789 +51790 POINT(29.118685655874472 -95.3430490700521) bank51790 +51791 POINT(29.70742175092004 -95.31352062626866) bank51791 +51792 POINT(30.081215362895005 -96.01758858695193) bank51792 +51793 POINT(29.340592277423227 -95.24891768710494) bank51793 +51794 POINT(30.068960335577533 -94.76460068644856) bank51794 +51795 POINT(30.088119414044087 -95.7535520158268) bank51795 +51796 POINT(30.271090086298006 -94.93337416225769) bank51796 +51797 POINT(30.36541813435968 -95.27963795086112) bank51797 +51798 POINT(29.765685905007096 -94.4991294922908) bank51798 +51799 POINT(29.38864385797675 -96.12404736385545) bank51799 +51800 POINT(30.617204238472258 -95.90857055130985) bank51800 +51801 POINT(30.348449223113047 -96.3349714410508) bank51801 +51802 POINT(29.604361521642762 -95.81963626345639) bank51802 +51803 POINT(28.983431058862458 -95.69410866260958) bank51803 +51804 POINT(30.554373996683776 -94.73991368114298) bank51804 +51805 POINT(30.53755550164985 -95.16975307149242) bank51805 +51806 POINT(30.511534359785834 -96.13351717125673) bank51806 +51807 POINT(29.780338819643095 -95.61846346710249) bank51807 +51808 POINT(30.670028224479573 -96.1163095045567) bank51808 +51809 POINT(29.86731479925921 -95.79588535052775) bank51809 +51810 POINT(30.66071157496806 -94.57065767130452) bank51810 +51811 POINT(29.349542343449162 -94.81013584364734) bank51811 +51812 POINT(28.860722453620756 -96.17418942262204) bank51812 +51813 POINT(28.895109737884656 -96.18187596819484) bank51813 +51814 POINT(30.247036473909393 -96.19719534076148) bank51814 +51815 POINT(30.012206609761538 -95.83036618240102) bank51815 +51816 POINT(29.015086078135404 -95.32376558810579) bank51816 +51817 POINT(28.777397523324332 -95.59936034306425) bank51817 +51818 POINT(28.976079171890603 -95.84360613796562) bank51818 +51819 POINT(28.958661689350542 -96.08552516564458) bank51819 +51820 POINT(30.00098394009585 -96.06905639562603) bank51820 +51821 POINT(30.276102137292103 -96.32323159217543) bank51821 +51822 POINT(30.26651177062739 -95.71689205327301) bank51822 +51823 POINT(30.205667817732994 -95.37669623813964) bank51823 +51824 POINT(30.375446847157423 -96.02595694390108) bank51824 +51825 POINT(30.232900429546753 -96.15106665019752) bank51825 +51826 POINT(30.458338375871186 -94.50878045960997) bank51826 +51827 POINT(30.736410364413945 -94.37795331345208) bank51827 +51828 POINT(28.81722095051534 -94.69852771858626) bank51828 +51829 POINT(29.130271512014048 -95.20254947353389) bank51829 +51830 POINT(29.920023836989337 -95.86633998631709) bank51830 +51831 POINT(30.099134225006537 -95.72894229200998) bank51831 +51832 POINT(28.966886462432974 -96.23940217398454) bank51832 +51833 POINT(30.27236047583556 -96.05373124239443) bank51833 +51834 POINT(29.463671518340444 -95.80483199299215) bank51834 +51835 POINT(28.960892715531006 -96.07339962130847) bank51835 +51836 POINT(29.299958499052046 -94.89760891278299) bank51836 +51837 POINT(30.489085544183276 -95.9296449760797) bank51837 +51838 POINT(30.25314645532355 -94.40901090054724) bank51838 +51839 POINT(30.637446915799572 -94.9256804754018) bank51839 +51840 POINT(30.524015293747112 -95.05807358697358) bank51840 +51841 POINT(30.23830538073022 -95.12089389584743) bank51841 +51842 POINT(30.536881771169217 -95.42626118474602) bank51842 +51843 POINT(30.215130434019496 -94.76273653622557) bank51843 +51844 POINT(29.943460247532613 -95.0857329958583) bank51844 +51845 POINT(30.129071605614925 -96.03089366602579) bank51845 +51846 POINT(30.67153159278344 -96.08752300474106) bank51846 +51847 POINT(30.12799325737193 -94.41360225351626) bank51847 +51848 POINT(29.855402777105756 -95.49379575259347) bank51848 +51849 POINT(29.088182904861892 -94.45709792622739) bank51849 +51850 POINT(29.48033289401936 -96.21540414110561) bank51850 +51851 POINT(29.7303050318842 -96.03576422662775) bank51851 +51852 POINT(28.944006252497967 -94.86551189350439) bank51852 +51853 POINT(29.64416760942963 -95.73921158356848) bank51853 +51854 POINT(29.604932347655033 -94.61675560430321) bank51854 +51855 POINT(29.52594872724987 -95.5455866707818) bank51855 +51856 POINT(29.992052177085213 -95.88613446249961) bank51856 +51857 POINT(30.555632086689766 -94.79568172907034) bank51857 +51858 POINT(30.15010105249705 -95.16226725161424) bank51858 +51859 POINT(30.349624777283033 -95.01075234082442) bank51859 +51860 POINT(29.23014467211477 -95.83955760546534) bank51860 +51861 POINT(30.066760075049 -96.04912814944227) bank51861 +51862 POINT(29.282294138202253 -94.71917504674097) bank51862 +51863 POINT(29.783590480198363 -94.78604389019954) bank51863 +51864 POINT(30.454213545178355 -94.80844789285206) bank51864 +51865 POINT(30.29768692441863 -94.95815962278844) bank51865 +51866 POINT(29.697185229989923 -95.81813471206902) bank51866 +51867 POINT(30.079249136189496 -95.23089417575946) bank51867 +51868 POINT(29.946759626957284 -96.34490506523838) bank51868 +51869 POINT(29.80055445709118 -96.36947125957144) bank51869 +51870 POINT(29.200842453690278 -95.30590858840435) bank51870 +51871 POINT(30.56344615781468 -95.2457296936751) bank51871 +51872 POINT(30.51041933029359 -95.4708199875527) bank51872 +51873 POINT(29.632126509420505 -95.53417026241667) bank51873 +51874 POINT(29.58671145723993 -94.67628940597865) bank51874 +51875 POINT(28.916027893156393 -95.35662464342587) bank51875 +51876 POINT(29.72264763857402 -96.14126162501746) bank51876 +51877 POINT(30.089723982019528 -95.18624761959481) bank51877 +51878 POINT(29.366503429807942 -96.13394121458862) bank51878 +51879 POINT(30.384990160699694 -96.030509925791) bank51879 +51880 POINT(29.960778949415744 -94.70776419340937) bank51880 +51881 POINT(29.146128765419643 -94.97054760320634) bank51881 +51882 POINT(29.65576667496721 -95.9987939738188) bank51882 +51883 POINT(30.089152212281455 -95.36347758835467) bank51883 +51884 POINT(30.406308982498093 -94.53663887829136) bank51884 +51885 POINT(29.09983352763096 -94.76714354392108) bank51885 +51886 POINT(30.719033032073888 -96.05417921487278) bank51886 +51887 POINT(28.798574627036686 -94.6221046443086) bank51887 +51888 POINT(29.79656653185667 -94.96653043483558) bank51888 +51889 POINT(28.95878903398861 -95.71001428968124) bank51889 +51890 POINT(30.1972014056318 -94.67330666849199) bank51890 +51891 POINT(29.746028564331876 -96.35437005730547) bank51891 +51892 POINT(30.627942484679387 -95.26614937491509) bank51892 +51893 POINT(29.884956644603488 -95.7890781360199) bank51893 +51894 POINT(29.494827593166647 -95.43049284468574) bank51894 +51895 POINT(30.69010283646013 -94.5891317238051) bank51895 +51896 POINT(30.147321697368103 -95.94492340982806) bank51896 +51897 POINT(29.467846695288994 -96.1228669092892) bank51897 +51898 POINT(29.10248565699601 -94.89199430762342) bank51898 +51899 POINT(29.27864772444626 -96.3463560345368) bank51899 +51900 POINT(30.61783334426337 -95.28856239254274) bank51900 +51901 POINT(29.11644190501138 -96.15093976591417) bank51901 +51902 POINT(29.433247658493897 -95.41742639827986) bank51902 +51903 POINT(30.18872097196178 -95.26719251805304) bank51903 +51904 POINT(30.68306643922451 -95.79961686543493) bank51904 +51905 POINT(30.353314386225716 -94.40067017100426) bank51905 +51906 POINT(29.659703513743125 -94.85128910776731) bank51906 +51907 POINT(30.37220841087926 -95.46360713942157) bank51907 +51908 POINT(30.632692697654065 -95.69283449391634) bank51908 +51909 POINT(28.820558520148992 -95.94985887702938) bank51909 +51910 POINT(29.435712486769717 -96.02965135952867) bank51910 +51911 POINT(29.044350337560243 -96.03523522418853) bank51911 +51912 POINT(29.052498621921018 -95.00187196685333) bank51912 +51913 POINT(29.558213544612197 -94.37567619818522) bank51913 +51914 POINT(29.733413663547005 -95.1213474686149) bank51914 +51915 POINT(30.250529466361574 -95.68491150260643) bank51915 +51916 POINT(28.94207751073601 -96.04119437131371) bank51916 +51917 POINT(28.905303222187893 -95.80447120166059) bank51917 +51918 POINT(30.194387054620773 -96.04463044938706) bank51918 +51919 POINT(29.08292647766764 -95.54157739334651) bank51919 +51920 POINT(30.256738101952482 -95.18959202653475) bank51920 +51921 POINT(30.464974548659487 -95.79474021741045) bank51921 +51922 POINT(28.92973524294503 -95.3106073357674) bank51922 +51923 POINT(29.301996530021214 -95.1200606599253) bank51923 +51924 POINT(30.691022203302342 -94.41309686746033) bank51924 +51925 POINT(29.690014155191207 -95.27233807451375) bank51925 +51926 POINT(30.14550924937174 -95.284198895418) bank51926 +51927 POINT(29.346402602826974 -95.04436270633163) bank51927 +51928 POINT(29.52840823707886 -96.01013477966825) bank51928 +51929 POINT(28.86306059628631 -96.13101445530744) bank51929 +51930 POINT(28.8250548197758 -94.48150071516625) bank51930 +51931 POINT(29.53404634008195 -95.64892391361117) bank51931 +51932 POINT(30.551852043658837 -96.28838602438411) bank51932 +51933 POINT(29.81554919475574 -96.0639990402909) bank51933 +51934 POINT(30.43888682304664 -95.57042196097507) bank51934 +51935 POINT(29.017678015112168 -95.30194572520145) bank51935 +51936 POINT(28.980647958662612 -95.9800700849014) bank51936 +51937 POINT(29.892138923219463 -95.4389537516359) bank51937 +51938 POINT(28.820975563195827 -95.31145723776878) bank51938 +51939 POINT(29.187899988814863 -95.24857778249937) bank51939 +51940 POINT(28.98820557404713 -95.75800925432318) bank51940 +51941 POINT(29.813049692084423 -95.91665463513037) bank51941 +51942 POINT(29.90468804016952 -94.64007879873118) bank51942 +51943 POINT(29.983412087771313 -94.61664031182673) bank51943 +51944 POINT(29.82615410724605 -95.98511968200091) bank51944 +51945 POINT(30.4648409802436 -95.02204491620417) bank51945 +51946 POINT(29.38068103978579 -96.23600834841274) bank51946 +51947 POINT(30.232087151650596 -96.15947261710475) bank51947 +51948 POINT(29.826444103279364 -95.50287893337033) bank51948 +51949 POINT(29.785328765140036 -94.70776495816455) bank51949 +51950 POINT(28.98940428346734 -94.92794812499768) bank51950 +51951 POINT(30.126848168908655 -95.454727848943) bank51951 +51952 POINT(30.39222215193259 -95.50151187693021) bank51952 +51953 POINT(29.867081168022413 -95.86287897373583) bank51953 +51954 POINT(29.26872602676396 -95.30831107250316) bank51954 +51955 POINT(29.746449664713108 -94.69401635667325) bank51955 +51956 POINT(29.382071727020517 -96.33775874012791) bank51956 +51957 POINT(30.11656755916778 -95.00633258359818) bank51957 +51958 POINT(29.96752728881716 -95.64575848096433) bank51958 +51959 POINT(30.348628204162026 -96.2829355521822) bank51959 +51960 POINT(29.410112416979924 -95.11069362019025) bank51960 +51961 POINT(29.666226182988883 -95.40501946158057) bank51961 +51962 POINT(29.412628027790962 -96.23387242528594) bank51962 +51963 POINT(29.35461322367369 -95.70814029125924) bank51963 +51964 POINT(29.20833648111864 -95.32376032340558) bank51964 +51965 POINT(30.228816784474944 -95.93633276560591) bank51965 +51966 POINT(29.280644509818835 -94.7993547494573) bank51966 +51967 POINT(28.84408724807009 -95.63775561323483) bank51967 +51968 POINT(28.981237234644674 -94.79321657679043) bank51968 +51969 POINT(28.762295395755793 -95.77271064572132) bank51969 +51970 POINT(28.778727254458243 -95.98399325230808) bank51970 +51971 POINT(29.127794816260717 -95.28428243917052) bank51971 +51972 POINT(29.724435431516103 -95.08063601459486) bank51972 +51973 POINT(30.697634229279405 -94.54924505263845) bank51973 +51974 POINT(29.096998883810578 -94.58276723382377) bank51974 +51975 POINT(29.047122692878403 -95.59093528263804) bank51975 +51976 POINT(29.15515956106091 -94.74832202556429) bank51976 +51977 POINT(29.86668544965776 -95.02031244478533) bank51977 +51978 POINT(29.62960927507582 -96.3583677702171) bank51978 +51979 POINT(29.5982096075994 -94.56377638856496) bank51979 +51980 POINT(30.22807528371706 -95.00820931514514) bank51980 +51981 POINT(28.770394865550966 -96.15370908809729) bank51981 +51982 POINT(30.420012418089556 -96.16992302635695) bank51982 +51983 POINT(30.410574417714752 -95.31972681103295) bank51983 +51984 POINT(30.25448011697633 -95.1352617147512) bank51984 +51985 POINT(30.12646634141832 -95.84213591201834) bank51985 +51986 POINT(29.174718776211083 -95.22362441445594) bank51986 +51987 POINT(28.86528323471235 -96.03313842084546) bank51987 +51988 POINT(28.807988798255266 -94.51637494659964) bank51988 +51989 POINT(29.66822498634019 -94.57405629688526) bank51989 +51990 POINT(29.919507217272443 -95.61754938463142) bank51990 +51991 POINT(28.89878268776356 -95.5305306364554) bank51991 +51992 POINT(30.16499894055702 -94.48405162350404) bank51992 +51993 POINT(29.326558594884286 -94.47194704730094) bank51993 +51994 POINT(29.686771457738043 -95.97898284204385) bank51994 +51995 POINT(29.816083322736183 -95.47572743200234) bank51995 +51996 POINT(30.367723402574402 -95.82556624157203) bank51996 +51997 POINT(30.269326535009135 -95.62975632644725) bank51997 +51998 POINT(29.793199793384304 -94.75835681192014) bank51998 +51999 POINT(29.384015129307315 -95.0130300975861) bank51999 +52000 POINT(29.854064660564664 -95.9328306315108) bank52000 +52001 POINT(30.36070578738421 -94.88614514977078) bank52001 +52002 POINT(30.67271930167961 -96.2505610205149) bank52002 +52003 POINT(29.076343541707466 -95.73512265377819) bank52003 +52004 POINT(30.037947285210745 -95.39264044176201) bank52004 +52005 POINT(29.951860209955093 -95.46631168842168) bank52005 +52006 POINT(30.319940154906654 -95.19346362127116) bank52006 +52007 POINT(29.127974647126965 -94.76845284949104) bank52007 +52008 POINT(29.381703129477184 -96.07055922038546) bank52008 +52009 POINT(30.715896432167384 -96.1678357118513) bank52009 +52010 POINT(29.957042063407172 -95.49436084905508) bank52010 +52011 POINT(29.226809088371166 -96.33151185123499) bank52011 +52012 POINT(30.421406073767386 -95.98382863608062) bank52012 +52013 POINT(30.265036356612768 -94.76905777905809) bank52013 +52014 POINT(30.632250964090282 -94.93575584418204) bank52014 +52015 POINT(30.463852574906028 -95.75604970165247) bank52015 +52016 POINT(28.91915364905593 -94.43532671326633) bank52016 +52017 POINT(30.018937428076683 -96.14438232122171) bank52017 +52018 POINT(28.76772087483441 -95.32254389537312) bank52018 +52019 POINT(29.361352546577407 -95.3566563658892) bank52019 +52020 POINT(28.83201763786917 -95.00161579633307) bank52020 +52021 POINT(29.103468394410694 -95.73255391034378) bank52021 +52022 POINT(29.619562226568 -95.91291957200586) bank52022 +52023 POINT(29.02611108834029 -95.36264575397036) bank52023 +52024 POINT(30.056446954833362 -95.61556495683634) bank52024 +52025 POINT(29.040630286084006 -94.47432445541054) bank52025 +52026 POINT(29.593540794320024 -94.63315448369505) bank52026 +52027 POINT(29.186982376579287 -94.86670745207901) bank52027 +52028 POINT(28.958618705487496 -95.66499734921355) bank52028 +52029 POINT(28.802300195326122 -95.8685470094868) bank52029 +52030 POINT(30.331001883221226 -96.04889708203427) bank52030 +52031 POINT(30.188055880077126 -94.74709175199803) bank52031 +52032 POINT(29.96205256402498 -95.60235783822712) bank52032 +52033 POINT(30.602694030780377 -95.60424093123632) bank52033 +52034 POINT(29.18863484104014 -94.50474219142721) bank52034 +52035 POINT(29.853810910922995 -95.34839418293919) bank52035 +52036 POINT(30.188163316894606 -95.41701604793927) bank52036 +52037 POINT(30.51482016373322 -94.58877582052473) bank52037 +52038 POINT(29.282500584099445 -94.45544923766785) bank52038 +52039 POINT(29.345333764359214 -96.25478484331576) bank52039 +52040 POINT(29.563716875317336 -95.95348572924779) bank52040 +52041 POINT(29.29441660429181 -94.76121694080433) bank52041 +52042 POINT(29.780095498742714 -95.31527534531921) bank52042 +52043 POINT(28.89266342876733 -96.12143106227118) bank52043 +52044 POINT(30.330231628744453 -94.53827303144475) bank52044 +52045 POINT(29.65710943337051 -94.74297087442696) bank52045 +52046 POINT(30.694769265133225 -95.88648500659659) bank52046 +52047 POINT(30.750544866706434 -94.8782588541794) bank52047 +52048 POINT(29.94857463087169 -96.13524732860037) bank52048 +52049 POINT(29.259195973758764 -95.8529555779283) bank52049 +52050 POINT(30.192178821573766 -94.54203291597196) bank52050 +52051 POINT(29.168779772991094 -95.99574403919667) bank52051 +52052 POINT(29.00540137981841 -94.87095078939929) bank52052 +52053 POINT(29.285156709797842 -94.83359602785012) bank52053 +52054 POINT(30.681290685385083 -95.22012580200037) bank52054 +52055 POINT(29.934302163405995 -95.6667531129549) bank52055 +52056 POINT(29.507690379412196 -95.74511928555215) bank52056 +52057 POINT(30.406962069113312 -95.55057587469766) bank52057 +52058 POINT(30.380172514760844 -95.41427499470622) bank52058 +52059 POINT(29.3489729105173 -94.86392031353104) bank52059 +52060 POINT(29.856657800862294 -94.8398899840852) bank52060 +52061 POINT(29.963951051198933 -95.03208396372499) bank52061 +52062 POINT(29.785338400191147 -95.72204821559217) bank52062 +52063 POINT(29.341291225481328 -96.06987065021383) bank52063 +52064 POINT(29.114700305781366 -95.74536904735379) bank52064 +52065 POINT(30.708946173502756 -96.15344893131628) bank52065 +52066 POINT(30.754585019440853 -95.0016226217108) bank52066 +52067 POINT(30.237907524274917 -95.74549653109591) bank52067 +52068 POINT(29.94829111530396 -95.82890006934022) bank52068 +52069 POINT(30.163681800358713 -96.20338841896312) bank52069 +52070 POINT(29.863789764979952 -96.17775671736237) bank52070 +52071 POINT(30.526065175829356 -95.5639285597227) bank52071 +52072 POINT(30.56567232226041 -95.83611385326417) bank52072 +52073 POINT(29.837043582804558 -94.83728512964034) bank52073 +52074 POINT(29.166396986382065 -95.65869444185543) bank52074 +52075 POINT(29.930255308621945 -94.781656664289) bank52075 +52076 POINT(30.646887130519204 -94.8705207255556) bank52076 +52077 POINT(30.02499288063759 -95.02350940075709) bank52077 +52078 POINT(29.631187760858737 -95.39684614530583) bank52078 +52079 POINT(29.92498745445709 -95.32437711781) bank52079 +52080 POINT(29.339201040334096 -96.14226636785719) bank52080 +52081 POINT(30.313773475857545 -95.94283462667796) bank52081 +52082 POINT(29.968613175049313 -94.4507264496196) bank52082 +52083 POINT(29.56992624741149 -96.28197380409772) bank52083 +52084 POINT(29.028613264125703 -96.23096637669916) bank52084 +52085 POINT(30.67389747289197 -94.9829539395015) bank52085 +52086 POINT(28.820308741884833 -94.87226804142776) bank52086 +52087 POINT(29.788698815580506 -96.26495909887613) bank52087 +52088 POINT(29.53003619699621 -96.03696045857015) bank52088 +52089 POINT(29.480580089024834 -95.25704336551699) bank52089 +52090 POINT(29.383405131999005 -95.96050541558851) bank52090 +52091 POINT(29.044742820005943 -95.7419114933124) bank52091 +52092 POINT(29.475802664571123 -95.79375071242745) bank52092 +52093 POINT(29.693396596003094 -94.94608833532588) bank52093 +52094 POINT(28.84140844152291 -94.60013209919029) bank52094 +52095 POINT(30.374256319207653 -95.358848988814) bank52095 +52096 POINT(29.993416914666206 -94.57985542411414) bank52096 +52097 POINT(30.696135745843144 -94.45066614305077) bank52097 +52098 POINT(29.30602054154712 -95.87629583643458) bank52098 +52099 POINT(29.546365580812726 -95.38590327019216) bank52099 +52100 POINT(30.258148337149652 -94.45927033228078) bank52100 +52101 POINT(29.56488386081234 -94.86672277008951) bank52101 +52102 POINT(29.337191512010126 -96.224183441703) bank52102 +52103 POINT(29.244274243507228 -95.16812595680655) bank52103 +52104 POINT(29.68906975255471 -94.9578713659635) bank52104 +52105 POINT(29.752778010163887 -95.16633831101893) bank52105 +52106 POINT(29.722881612924283 -94.452253721835) bank52106 +52107 POINT(28.915708459599802 -95.86329164285924) bank52107 +52108 POINT(30.45592062153153 -94.72710089574659) bank52108 +52109 POINT(29.82498531530856 -94.47708168441095) bank52109 +52110 POINT(29.33951331805974 -96.19475347216543) bank52110 +52111 POINT(29.664940951975638 -95.13857143051402) bank52111 +52112 POINT(29.86675186989209 -96.14741745145604) bank52112 +52113 POINT(29.778846990650695 -95.28339350840449) bank52113 +52114 POINT(29.504126014426923 -95.53020191319315) bank52114 +52115 POINT(30.73906518357777 -95.95304766240761) bank52115 +52116 POINT(29.293189021264734 -94.74000016653861) bank52116 +52117 POINT(29.16859081138795 -96.34850299755402) bank52117 +52118 POINT(30.133212545252363 -95.21499010694855) bank52118 +52119 POINT(30.203989300559975 -94.65817619195113) bank52119 +52120 POINT(30.370588692575776 -94.69320084642692) bank52120 +52121 POINT(29.21404757657582 -95.75854443927422) bank52121 +52122 POINT(29.405702547539118 -95.30429625425161) bank52122 +52123 POINT(30.381303685213155 -94.55009452890496) bank52123 +52124 POINT(29.27708165154791 -95.51560293992561) bank52124 +52125 POINT(29.244691137981437 -96.09527431378987) bank52125 +52126 POINT(30.408962828445773 -95.92586535207187) bank52126 +52127 POINT(30.717722380798037 -94.6497795492785) bank52127 +52128 POINT(28.93819533827894 -95.11140507311669) bank52128 +52129 POINT(30.734776411354407 -96.01431983246489) bank52129 +52130 POINT(29.302359528864276 -95.59056106985821) bank52130 +52131 POINT(29.082167423008016 -94.80998196116323) bank52131 +52132 POINT(29.435005034468396 -95.78478904533267) bank52132 +52133 POINT(29.54289266772336 -95.99125740246868) bank52133 +52134 POINT(29.695871709368816 -95.11652055077865) bank52134 +52135 POINT(30.37349652290958 -95.718097417741) bank52135 +52136 POINT(30.075255619320334 -94.70989418544717) bank52136 +52137 POINT(29.678684064372703 -94.54760000562636) bank52137 +52138 POINT(30.006524670048933 -95.11451244898768) bank52138 +52139 POINT(28.89550115465196 -94.6486539004553) bank52139 +52140 POINT(30.49119474820256 -95.28006864817705) bank52140 +52141 POINT(29.827486516656844 -95.80288968658388) bank52141 +52142 POINT(28.901280976667113 -95.25205724335912) bank52142 +52143 POINT(29.1875454809143 -95.52018805348537) bank52143 +52144 POINT(30.603014369712316 -95.33105478627195) bank52144 +52145 POINT(30.44354728434638 -96.17800173552268) bank52145 +52146 POINT(29.72658325499401 -95.10860980216154) bank52146 +52147 POINT(29.92284422347604 -95.16836104609123) bank52147 +52148 POINT(29.683813325948318 -96.07361231420141) bank52148 +52149 POINT(29.886608456118402 -95.99752245277212) bank52149 +52150 POINT(29.02070760510927 -95.89229322487436) bank52150 +52151 POINT(30.389579310734845 -95.38576423804167) bank52151 +52152 POINT(29.497054074587396 -94.9224442806798) bank52152 +52153 POINT(29.833310847148113 -94.6609567927904) bank52153 +52154 POINT(30.192820686720378 -94.58656702168268) bank52154 +52155 POINT(29.58496913058715 -96.00579486149296) bank52155 +52156 POINT(28.988497704922736 -96.01979515239776) bank52156 +52157 POINT(29.84196373950184 -94.62174357870774) bank52157 +52158 POINT(29.573461326677062 -95.59378971834948) bank52158 +52159 POINT(29.219253596932763 -94.74557884091804) bank52159 +52160 POINT(29.093566524617174 -95.1594001532919) bank52160 +52161 POINT(29.166870797734877 -96.14263445431965) bank52161 +52162 POINT(30.122762887451888 -95.78728301435456) bank52162 +52163 POINT(30.70029416626901 -96.21418528367171) bank52163 +52164 POINT(29.619440301122673 -95.2915324852838) bank52164 +52165 POINT(29.486018984960772 -94.70964688074592) bank52165 +52166 POINT(30.759763529257807 -94.99316318540944) bank52166 +52167 POINT(29.652826462882622 -94.77461211284138) bank52167 +52168 POINT(29.074276538380957 -95.64090868960885) bank52168 +52169 POINT(29.0469092512235 -94.95339567943896) bank52169 +52170 POINT(30.496488297973418 -94.96821953030201) bank52170 +52171 POINT(30.580008613614506 -94.39701031006182) bank52171 +52172 POINT(30.125657157802518 -95.87055033145003) bank52172 +52173 POINT(30.71748633047209 -95.5707346853883) bank52173 +52174 POINT(29.503695643667676 -95.90753557159898) bank52174 +52175 POINT(29.48193781375066 -94.62431363718389) bank52175 +52176 POINT(29.084475439081203 -94.4662839058293) bank52176 +52177 POINT(30.509805433623693 -96.11139000177215) bank52177 +52178 POINT(29.278556629554384 -96.05075292565026) bank52178 +52179 POINT(30.006071267813965 -94.95384483247192) bank52179 +52180 POINT(29.63817408668156 -95.37122850546126) bank52180 +52181 POINT(29.786292071444446 -94.75967663052417) bank52181 +52182 POINT(29.327227759910674 -94.40595682763357) bank52182 +52183 POINT(30.162987021361477 -94.41624527297033) bank52183 +52184 POINT(30.32979794675978 -95.40844197794814) bank52184 +52185 POINT(30.549325303115477 -95.34725443593254) bank52185 +52186 POINT(30.366778514390166 -96.3388736041336) bank52186 +52187 POINT(29.944369299932802 -94.96136945978255) bank52187 +52188 POINT(29.528715822479217 -95.7596043074984) bank52188 +52189 POINT(29.051981017303447 -94.78384342659004) bank52189 +52190 POINT(29.472555164304158 -95.88005595667752) bank52190 +52191 POINT(29.20117657901482 -95.64532323431067) bank52191 +52192 POINT(29.196447046992848 -96.22617935078539) bank52192 +52193 POINT(29.13433181897437 -95.79929725323042) bank52193 +52194 POINT(30.265815682867768 -95.13959538795186) bank52194 +52195 POINT(28.85300114447599 -94.922232972441) bank52195 +52196 POINT(29.410678702949074 -94.96417528981989) bank52196 +52197 POINT(30.130280675785666 -95.45462004620273) bank52197 +52198 POINT(30.45972187968517 -94.48490397195941) bank52198 +52199 POINT(29.20332777995423 -95.99353048766343) bank52199 +52200 POINT(30.433294936983565 -94.87429710317754) bank52200 +52201 POINT(30.67408630893086 -94.52092102291937) bank52201 +52202 POINT(29.890639872869674 -94.6445906281276) bank52202 +52203 POINT(29.46097759426026 -94.58562251695045) bank52203 +52204 POINT(30.598229355067552 -95.72888456103496) bank52204 +52205 POINT(29.270958428601112 -94.99807197868203) bank52205 +52206 POINT(30.079798355721294 -94.52776984519737) bank52206 +52207 POINT(29.592414413253373 -95.84870311351867) bank52207 +52208 POINT(30.093626112852434 -96.36668609887745) bank52208 +52209 POINT(28.89551102621365 -95.14029480272794) bank52209 +52210 POINT(30.557583353200553 -94.40114827060977) bank52210 +52211 POINT(30.23971430272509 -96.18305563131015) bank52211 +52212 POINT(28.911410234959924 -95.8353459940853) bank52212 +52213 POINT(29.835512296631713 -94.75592222167202) bank52213 +52214 POINT(29.55717680123337 -94.65150902238145) bank52214 +52215 POINT(30.67277088840608 -94.4855149011883) bank52215 +52216 POINT(29.986949599695585 -95.24553145851881) bank52216 +52217 POINT(29.403742638496553 -95.21620331028379) bank52217 +52218 POINT(30.744811303790332 -96.32326538160942) bank52218 +52219 POINT(28.984876686765475 -94.71078155189663) bank52219 +52220 POINT(29.993921316222973 -94.42956690149848) bank52220 +52221 POINT(30.548806849410468 -96.05031748587645) bank52221 +52222 POINT(29.15552618086703 -96.36179982644914) bank52222 +52223 POINT(30.488141094969354 -95.76467715025403) bank52223 +52224 POINT(28.7893587878187 -94.45871564287152) bank52224 +52225 POINT(29.88799277943129 -96.17995923780451) bank52225 +52226 POINT(29.331279379783577 -94.79116092863231) bank52226 +52227 POINT(28.9618974560181 -94.85691228574906) bank52227 +52228 POINT(30.51517306203869 -95.62443601588625) bank52228 +52229 POINT(29.305812059947073 -94.37610189953638) bank52229 +52230 POINT(28.841362031128458 -95.10007749261206) bank52230 +52231 POINT(30.73688446173255 -94.9480997948459) bank52231 +52232 POINT(30.359293448719907 -95.3311007068317) bank52232 +52233 POINT(28.97297523772385 -96.04766631722408) bank52233 +52234 POINT(29.85556071362216 -95.4277897920714) bank52234 +52235 POINT(29.214052879429232 -94.51151332514831) bank52235 +52236 POINT(29.11193251567902 -95.4637325953961) bank52236 +52237 POINT(28.986424350143388 -95.49827754781273) bank52237 +52238 POINT(30.10978242044663 -95.35376388741007) bank52238 +52239 POINT(30.59745592831831 -94.44825696515638) bank52239 +52240 POINT(30.688944433821927 -94.5492080573095) bank52240 +52241 POINT(29.043259337844383 -95.2396281350558) bank52241 +52242 POINT(29.971161599848593 -96.3633512358431) bank52242 +52243 POINT(30.22462003964015 -95.69173052184107) bank52243 +52244 POINT(30.413530604009516 -94.8930826031913) bank52244 +52245 POINT(30.61297797574659 -95.22473669949169) bank52245 +52246 POINT(29.85102484360272 -96.1999196306377) bank52246 +52247 POINT(29.424303841696254 -96.11217565387976) bank52247 +52248 POINT(30.136102159839375 -94.90126026145691) bank52248 +52249 POINT(29.744030402119815 -94.95777253719196) bank52249 +52250 POINT(28.770708623937793 -96.01014878780073) bank52250 +52251 POINT(28.968236157252452 -95.86616037069695) bank52251 +52252 POINT(30.1027643055077 -95.84922475525084) bank52252 +52253 POINT(29.811501243901564 -96.07041451655165) bank52253 +52254 POINT(29.25512138677889 -95.04303538550595) bank52254 +52255 POINT(29.286545657278513 -94.67577188063399) bank52255 +52256 POINT(29.000690286449586 -96.17170543360254) bank52256 +52257 POINT(29.615864564220796 -96.02529717992627) bank52257 +52258 POINT(29.993046278117948 -95.80051028033176) bank52258 +52259 POINT(29.364391831463028 -96.2671030653768) bank52259 +52260 POINT(29.13164015131626 -95.69258560262995) bank52260 +52261 POINT(30.747060961816402 -94.61303243309469) bank52261 +52262 POINT(30.134802540559594 -96.03474471404894) bank52262 +52263 POINT(30.514357719171265 -95.83502585796349) bank52263 +52264 POINT(29.66176653781396 -95.09626131780288) bank52264 +52265 POINT(29.703495120968896 -96.36825015864216) bank52265 +52266 POINT(30.534687414165703 -94.92894041988504) bank52266 +52267 POINT(29.958368837553696 -95.07803232428456) bank52267 +52268 POINT(29.757294775619638 -95.0293098326567) bank52268 +52269 POINT(28.898758377226113 -96.1076895800617) bank52269 +52270 POINT(29.297445228496464 -96.08743468702353) bank52270 +52271 POINT(30.39190908726316 -95.4330780059023) bank52271 +52272 POINT(29.36965336142979 -94.42907661495477) bank52272 +52273 POINT(29.259206320928172 -95.27093705880426) bank52273 +52274 POINT(29.070613170987674 -95.03715661561078) bank52274 +52275 POINT(29.861924330840367 -95.38640357027091) bank52275 +52276 POINT(30.4999768307383 -95.94301893588937) bank52276 +52277 POINT(30.633818167470906 -95.012617589872) bank52277 +52278 POINT(29.96754494667838 -94.67039826469905) bank52278 +52279 POINT(30.723406784224572 -95.04646012319098) bank52279 +52280 POINT(29.768163913003875 -94.68277602703945) bank52280 +52281 POINT(29.424041381821105 -95.57497002397342) bank52281 +52282 POINT(28.77528842117471 -96.05689070368862) bank52282 +52283 POINT(30.134405525091218 -94.66042464120223) bank52283 +52284 POINT(30.05036089372152 -95.90153462474942) bank52284 +52285 POINT(29.849242806105295 -96.32872477802138) bank52285 +52286 POINT(29.88026093333188 -94.71149672448276) bank52286 +52287 POINT(29.205831087901824 -95.15794291874236) bank52287 +52288 POINT(29.728503139542084 -95.8767602489319) bank52288 +52289 POINT(29.155966088788357 -96.07629620582766) bank52289 +52290 POINT(29.617544683662903 -95.00388521697683) bank52290 +52291 POINT(30.63209046221192 -94.43137881685202) bank52291 +52292 POINT(29.101690161788632 -96.0468888487235) bank52292 +52293 POINT(30.281569408886096 -96.1789744924889) bank52293 +52294 POINT(29.036943050894596 -96.01700410042783) bank52294 +52295 POINT(29.138774854783502 -95.42101499303126) bank52295 +52296 POINT(28.818467493209425 -95.70187457679334) bank52296 +52297 POINT(30.457064031460064 -95.77328316670842) bank52297 +52298 POINT(28.943863161163957 -96.2029951198735) bank52298 +52299 POINT(29.82497507699603 -95.23403989660913) bank52299 +52300 POINT(29.920579367018036 -95.49114275331009) bank52300 +52301 POINT(29.41944058011808 -94.7658143405208) bank52301 +52302 POINT(30.075912838095356 -95.57231688005201) bank52302 +52303 POINT(29.154264015640287 -94.78683954409935) bank52303 +52304 POINT(29.85874549062543 -95.30749065830804) bank52304 +52305 POINT(29.968503620256723 -96.2054328320114) bank52305 +52306 POINT(28.8845269372465 -96.15119391662043) bank52306 +52307 POINT(29.288171044770547 -95.56933386225107) bank52307 +52308 POINT(30.59881165996331 -94.82460239597786) bank52308 +52309 POINT(29.089528743875555 -94.6710821532426) bank52309 +52310 POINT(29.843312925854704 -94.84907497728716) bank52310 +52311 POINT(30.448253968867593 -95.02293313977007) bank52311 +52312 POINT(29.21397585039385 -95.19654335170607) bank52312 +52313 POINT(30.266968134289268 -94.61397800324728) bank52313 +52314 POINT(29.45901772982864 -95.93605699598737) bank52314 +52315 POINT(29.35226947355899 -96.15573739758784) bank52315 +52316 POINT(29.22474780848047 -95.96378211692726) bank52316 +52317 POINT(29.502150719999644 -95.35997491717208) bank52317 +52318 POINT(29.370265357995024 -94.37675811800051) bank52318 +52319 POINT(30.04273142928198 -96.05742651685992) bank52319 +52320 POINT(30.218050596248098 -94.74953998557014) bank52320 +52321 POINT(29.00208386455531 -95.55212030104225) bank52321 +52322 POINT(29.016842972064644 -96.03172600941008) bank52322 +52323 POINT(28.97450972687948 -95.294048310632) bank52323 +52324 POINT(29.92443012400536 -95.34124942181036) bank52324 +52325 POINT(29.98456932282138 -95.20047234675171) bank52325 +52326 POINT(29.92053167019836 -94.66611283650039) bank52326 +52327 POINT(29.53687785542367 -95.08585887676313) bank52327 +52328 POINT(30.3186466036349 -94.88045438232622) bank52328 +52329 POINT(29.8967990129306 -95.19380536544215) bank52329 +52330 POINT(30.720585400662877 -95.67627833514771) bank52330 +52331 POINT(30.04575293747823 -95.72828256099108) bank52331 +52332 POINT(30.431413231867175 -95.55700645627523) bank52332 +52333 POINT(29.678265476560824 -95.56696804935622) bank52333 +52334 POINT(30.482278868106775 -96.23049235979053) bank52334 +52335 POINT(30.593579846152583 -94.57971860107585) bank52335 +52336 POINT(29.71257309618852 -95.19382304058534) bank52336 +52337 POINT(30.42906143486459 -94.84725171404982) bank52337 +52338 POINT(30.684797293308982 -96.05175259154227) bank52338 +52339 POINT(30.46938701366065 -95.24919070402866) bank52339 +52340 POINT(29.83025178759633 -95.27589008012357) bank52340 +52341 POINT(29.731611215141015 -96.02953592426961) bank52341 +52342 POINT(29.2189267754796 -94.8342037535193) bank52342 +52343 POINT(29.062612404093816 -94.82964183564293) bank52343 +52344 POINT(30.100303553875772 -95.0061319461802) bank52344 +52345 POINT(29.174824248747775 -95.0923165734555) bank52345 +52346 POINT(30.2822741032124 -94.59601953777637) bank52346 +52347 POINT(29.451643720825363 -95.55993588446903) bank52347 +52348 POINT(28.783697558249973 -95.97936765176892) bank52348 +52349 POINT(29.09061711649383 -94.86094595051259) bank52349 +52350 POINT(29.507448318170585 -95.9204117668926) bank52350 +52351 POINT(29.059867451271 -95.29723506714716) bank52351 +52352 POINT(29.326254710405085 -95.71489526576738) bank52352 +52353 POINT(30.55991627938912 -95.30488038374709) bank52353 +52354 POINT(30.241076713474214 -96.14420890549111) bank52354 +52355 POINT(30.513780955757785 -96.05446871187603) bank52355 +52356 POINT(29.310213346222866 -95.7008345174249) bank52356 +52357 POINT(29.08611530055939 -96.23381475957697) bank52357 +52358 POINT(29.53121718716689 -95.69298980881477) bank52358 +52359 POINT(29.931468250912854 -95.38091773853287) bank52359 +52360 POINT(30.251801355574397 -94.95068533235937) bank52360 +52361 POINT(28.994038843437153 -95.29194214280088) bank52361 +52362 POINT(29.979417569765094 -96.35807482749489) bank52362 +52363 POINT(30.163158452162616 -95.63887979882459) bank52363 +52364 POINT(29.562580955201547 -95.64621395882823) bank52364 +52365 POINT(29.95873903379299 -95.82319792545216) bank52365 +52366 POINT(30.470212629742704 -96.10377761711312) bank52366 +52367 POINT(29.81192749620638 -94.62791341757433) bank52367 +52368 POINT(29.12256728759849 -95.24553331415817) bank52368 +52369 POINT(29.29059184177996 -96.16760192154885) bank52369 +52370 POINT(29.7925110744289 -95.68513651156901) bank52370 +52371 POINT(30.67779581485276 -94.77199639199914) bank52371 +52372 POINT(30.51262918423148 -96.20585294742291) bank52372 +52373 POINT(30.73169159638256 -95.13336741035798) bank52373 +52374 POINT(30.216093461067143 -94.43460836041571) bank52374 +52375 POINT(29.525071082626 -95.03681174494457) bank52375 +52376 POINT(28.8138774533166 -94.82000905950186) bank52376 +52377 POINT(30.31275562335987 -95.81107807551078) bank52377 +52378 POINT(30.586077005421643 -96.10928862481774) bank52378 +52379 POINT(28.780644356775465 -95.17906119361056) bank52379 +52380 POINT(29.397627273897257 -96.05039291176494) bank52380 +52381 POINT(28.8928823035167 -95.87597154867669) bank52381 +52382 POINT(29.863751331586734 -96.03134057411978) bank52382 +52383 POINT(29.157744658946587 -95.61168270890343) bank52383 +52384 POINT(29.041893338615388 -96.31040798507426) bank52384 +52385 POINT(30.00353946571203 -95.854560010283) bank52385 +52386 POINT(29.931869610119858 -96.28134410026409) bank52386 +52387 POINT(28.808174033086033 -94.91323650721976) bank52387 +52388 POINT(29.551223891172068 -94.59874607092028) bank52388 +52389 POINT(29.662475862741843 -94.97408459698943) bank52389 +52390 POINT(30.07155018973649 -95.33014440013544) bank52390 +52391 POINT(29.130274904915034 -96.27167483263061) bank52391 +52392 POINT(30.698236030552835 -95.22439403132498) bank52392 +52393 POINT(29.914025761778362 -95.06162236367685) bank52393 +52394 POINT(29.19128621153597 -96.0439069867812) bank52394 +52395 POINT(30.008575541904097 -94.89611222282194) bank52395 +52396 POINT(30.311672104595395 -95.86399900486764) bank52396 +52397 POINT(28.772528655287488 -94.6169775932935) bank52397 +52398 POINT(30.351645635085323 -95.72793801572713) bank52398 +52399 POINT(29.37519897350001 -94.75036739224613) bank52399 +52400 POINT(30.001781102459464 -96.28077656865487) bank52400 +52401 POINT(28.933677713193024 -95.20299701525855) bank52401 +52402 POINT(30.480140178963808 -95.07026957816306) bank52402 +52403 POINT(29.207901014854073 -94.55368343546255) bank52403 +52404 POINT(29.561336941343697 -95.46091093698463) bank52404 +52405 POINT(29.761895291158893 -94.70215727607848) bank52405 +52406 POINT(28.866201514961393 -94.98538919973699) bank52406 +52407 POINT(29.808105442568632 -96.26343670030067) bank52407 +52408 POINT(30.027092992684864 -96.27471187432567) bank52408 +52409 POINT(29.990142018769472 -95.33164183168525) bank52409 +52410 POINT(29.38847589562245 -95.89923103857662) bank52410 +52411 POINT(29.273666629545676 -95.24055673800287) bank52411 +52412 POINT(29.83354987438822 -95.69864699479837) bank52412 +52413 POINT(30.725036821795694 -95.36557866043343) bank52413 +52414 POINT(29.82063712172944 -95.44174176433867) bank52414 +52415 POINT(29.58869524193346 -95.75265292433409) bank52415 +52416 POINT(29.122361305043604 -94.95309650312907) bank52416 +52417 POINT(30.250602110361108 -95.86893698887653) bank52417 +52418 POINT(30.507081883082144 -95.80707742030468) bank52418 +52419 POINT(30.260653388440783 -95.23031244960924) bank52419 +52420 POINT(28.831480325827677 -94.63091832066156) bank52420 +52421 POINT(30.104110535302237 -95.66900248461228) bank52421 +52422 POINT(30.117354869307228 -95.14555722148653) bank52422 +52423 POINT(28.847782973690656 -94.88080541869721) bank52423 +52424 POINT(28.761826011031587 -96.04339477806992) bank52424 +52425 POINT(30.290395539204745 -95.02691545053239) bank52425 +52426 POINT(28.848290839798334 -95.11880046151525) bank52426 +52427 POINT(30.637537099012924 -94.86133496175627) bank52427 +52428 POINT(30.19530750937203 -95.94785461252538) bank52428 +52429 POINT(28.89300085427231 -95.07947661574316) bank52429 +52430 POINT(28.946690207640625 -96.0051509530057) bank52430 +52431 POINT(30.336570220125367 -95.25698830908833) bank52431 +52432 POINT(30.009928802101886 -96.17338794402106) bank52432 +52433 POINT(30.417491280813664 -94.44970846193452) bank52433 +52434 POINT(28.87763631716679 -95.47182750471691) bank52434 +52435 POINT(29.07459270448539 -95.28478521762025) bank52435 +52436 POINT(30.352859083570767 -96.30358734478132) bank52436 +52437 POINT(30.536549493541205 -95.6744709302758) bank52437 +52438 POINT(30.215788406330937 -94.49311498527265) bank52438 +52439 POINT(29.118450654119012 -94.4587114539548) bank52439 +52440 POINT(28.825259418368063 -96.09848532339002) bank52440 +52441 POINT(30.187500712331563 -96.33861489506751) bank52441 +52442 POINT(29.477932053271335 -96.13782778984861) bank52442 +52443 POINT(28.925861947861453 -95.80518898468219) bank52443 +52444 POINT(29.10522437199798 -96.1730999325306) bank52444 +52445 POINT(29.50736993218673 -96.3174449216766) bank52445 +52446 POINT(29.35696824657624 -95.35519600406387) bank52446 +52447 POINT(30.15601424904086 -94.6068679739147) bank52447 +52448 POINT(29.759627906411694 -95.67775908047373) bank52448 +52449 POINT(30.21100894861472 -94.67325229386773) bank52449 +52450 POINT(30.008657970965366 -94.61027543788384) bank52450 +52451 POINT(28.773133618869444 -96.07221895614994) bank52451 +52452 POINT(29.487825161606985 -95.25939977665188) bank52452 +52453 POINT(30.21919580596071 -95.75785063055382) bank52453 +52454 POINT(29.582970955737547 -94.79039477538272) bank52454 +52455 POINT(30.520445774015013 -94.65268662026696) bank52455 +52456 POINT(30.268572252434904 -95.77164021708798) bank52456 +52457 POINT(29.317389697517548 -95.9837234918053) bank52457 +52458 POINT(30.271145098365736 -96.36541671028678) bank52458 +52459 POINT(29.36088032053054 -94.82594610725675) bank52459 +52460 POINT(30.735273953143082 -95.31684522247274) bank52460 +52461 POINT(30.172056025801506 -95.19967220671836) bank52461 +52462 POINT(30.406942205451873 -94.79261867973253) bank52462 +52463 POINT(30.21365627596958 -94.94596057135185) bank52463 +52464 POINT(29.905306517050985 -95.80856838316065) bank52464 +52465 POINT(29.277915317355856 -94.83141921858203) bank52465 +52466 POINT(30.231779820026365 -95.92099691594939) bank52466 +52467 POINT(29.809278782926967 -94.72369548736773) bank52467 +52468 POINT(30.02052217573065 -95.28975556940216) bank52468 +52469 POINT(30.49531310408801 -95.26138026617524) bank52469 +52470 POINT(30.094768891854496 -96.15779448773549) bank52470 +52471 POINT(28.920028000012632 -95.12425202726062) bank52471 +52472 POINT(29.951694329201246 -94.5220183779762) bank52472 +52473 POINT(28.905155596351904 -95.95721880548341) bank52473 +52474 POINT(30.312483647944017 -96.01662157775951) bank52474 +52475 POINT(28.923655914136237 -95.4733016332057) bank52475 +52476 POINT(30.47978126353266 -95.8079746187663) bank52476 +52477 POINT(28.801056837266923 -94.73440225136159) bank52477 +52478 POINT(29.589603691286566 -94.54952937402773) bank52478 +52479 POINT(30.225723527197655 -95.07310385217758) bank52479 +52480 POINT(29.784117818808127 -96.18602012051342) bank52480 +52481 POINT(29.40951338205203 -95.31485200571018) bank52481 +52482 POINT(29.529204106827898 -95.90714728269381) bank52482 +52483 POINT(29.11649855320728 -95.28900336261177) bank52483 +52484 POINT(30.2169042320715 -95.92219935332625) bank52484 +52485 POINT(29.617388241994036 -95.22007003420448) bank52485 +52486 POINT(29.963070914630386 -96.30580786187753) bank52486 +52487 POINT(29.841827307150417 -96.31154806558166) bank52487 +52488 POINT(29.69124530593992 -94.84213161095212) bank52488 +52489 POINT(29.589664844535836 -95.7197609550359) bank52489 +52490 POINT(29.895416235436596 -95.7776162341027) bank52490 +52491 POINT(29.380629260888274 -95.10174907644644) bank52491 +52492 POINT(30.135708106241218 -95.36723093777974) bank52492 +52493 POINT(29.483449933248817 -94.66170081853727) bank52493 +52494 POINT(30.04071093979103 -95.68052888358379) bank52494 +52495 POINT(30.22429719656106 -94.56869703725931) bank52495 +52496 POINT(29.12023312283518 -95.58458926679826) bank52496 +52497 POINT(29.74152519061242 -95.59496320538105) bank52497 +52498 POINT(29.758474020482446 -95.69747962215952) bank52498 +52499 POINT(30.08438477037021 -95.2587009247763) bank52499 +52500 POINT(29.437171096718572 -95.94188766624089) bank52500 +52501 POINT(30.68547468045128 -95.80868816547597) bank52501 +52502 POINT(30.318340664917415 -94.46991763391118) bank52502 +52503 POINT(29.425776407297402 -95.54106064253303) bank52503 +52504 POINT(30.389807255766076 -95.92040665436318) bank52504 +52505 POINT(30.276901964101857 -94.87327879804798) bank52505 +52506 POINT(29.10240574458582 -94.46026147263447) bank52506 +52507 POINT(29.922955821109618 -95.67352460946682) bank52507 +52508 POINT(30.651690470514087 -95.90899935838911) bank52508 +52509 POINT(30.200637877793948 -94.70101116848221) bank52509 +52510 POINT(30.580544153018568 -95.28958553524045) bank52510 +52511 POINT(29.191381040531525 -95.8250205225544) bank52511 +52512 POINT(29.999980034359563 -95.54910328307719) bank52512 +52513 POINT(30.488679704470286 -95.493258680396) bank52513 +52514 POINT(30.155902035345814 -94.55280861168798) bank52514 +52515 POINT(30.715988816851656 -95.08917331367523) bank52515 +52516 POINT(29.124948382496072 -94.90567572314475) bank52516 +52517 POINT(29.37009139994899 -95.97563355540252) bank52517 +52518 POINT(28.789158486029834 -96.03796670427579) bank52518 +52519 POINT(30.033532365111533 -95.73982132980427) bank52519 +52520 POINT(29.861215830168916 -94.57314752362663) bank52520 +52521 POINT(29.461399268201163 -96.02671214318006) bank52521 +52522 POINT(29.106796393131834 -94.7892270649427) bank52522 +52523 POINT(29.383971116263233 -94.53778609912716) bank52523 +52524 POINT(30.378199855768475 -96.01816932618898) bank52524 +52525 POINT(30.212181092413054 -95.73024543780248) bank52525 +52526 POINT(28.842313729497675 -95.0626703573792) bank52526 +52527 POINT(30.62308125834322 -96.17519617314991) bank52527 +52528 POINT(29.850171816617678 -94.7227996614035) bank52528 +52529 POINT(30.683420416409593 -95.87753077753824) bank52529 +52530 POINT(30.063069541205177 -96.3430228868203) bank52530 +52531 POINT(28.899071777278884 -96.17593753514089) bank52531 +52532 POINT(29.71824463984042 -96.36515919179175) bank52532 +52533 POINT(30.1027391663703 -95.21553231940406) bank52533 +52534 POINT(30.467186711079577 -94.99068485910921) bank52534 +52535 POINT(29.71931769341392 -95.57834292169002) bank52535 +52536 POINT(29.499004935552804 -94.72338224944608) bank52536 +52537 POINT(29.79068627034317 -95.96238095264803) bank52537 +52538 POINT(30.52492407634777 -94.56936106549581) bank52538 +52539 POINT(29.08298883141226 -95.49478344376656) bank52539 +52540 POINT(30.071397144125203 -96.04172412517491) bank52540 +52541 POINT(28.79097373926133 -95.53485095789384) bank52541 +52542 POINT(30.311988134708635 -96.03724207212838) bank52542 +52543 POINT(29.258605027419176 -95.5325153966617) bank52543 +52544 POINT(29.122813565332102 -94.45926735683746) bank52544 +52545 POINT(30.536597943487372 -95.48788179201036) bank52545 +52546 POINT(30.000444120843774 -94.96243315990755) bank52546 +52547 POINT(29.047508652842136 -94.99144666711148) bank52547 +52548 POINT(29.990781657339966 -95.86932152515787) bank52548 +52549 POINT(30.45138100721111 -95.93429971906212) bank52549 +52550 POINT(28.915779220633105 -94.66086199489814) bank52550 +52551 POINT(29.965452776472524 -94.91018896920083) bank52551 +52552 POINT(30.641592099845436 -96.17935696372774) bank52552 +52553 POINT(29.659636348684984 -95.83999055353759) bank52553 +52554 POINT(28.990125920321145 -95.53468763201526) bank52554 +52555 POINT(29.8249163913692 -95.68290562646436) bank52555 +52556 POINT(29.080776236968028 -94.83591812463291) bank52556 +52557 POINT(28.79062667730435 -96.10932444522561) bank52557 +52558 POINT(29.40251526629949 -95.3789087617176) bank52558 +52559 POINT(30.483034681984734 -94.55738446835039) bank52559 +52560 POINT(30.060288834486776 -95.02363146952773) bank52560 +52561 POINT(30.0367153886915 -95.2022439139001) bank52561 +52562 POINT(29.013165160426965 -95.83407439767849) bank52562 +52563 POINT(29.937108749742578 -95.71253612670216) bank52563 +52564 POINT(30.672092607415642 -95.44790898416092) bank52564 +52565 POINT(29.9336657785759 -95.84389634834226) bank52565 +52566 POINT(30.023181765518782 -96.15404759063883) bank52566 +52567 POINT(29.885707882878474 -94.98806041059414) bank52567 +52568 POINT(29.266963981211376 -95.06362454184799) bank52568 +52569 POINT(28.778016321380246 -96.03980814844218) bank52569 +52570 POINT(29.518432538195132 -96.00006831201497) bank52570 +52571 POINT(30.46129947068504 -94.51511064170958) bank52571 +52572 POINT(30.686408667650046 -96.04546178723264) bank52572 +52573 POINT(30.58141081835318 -94.94206049103514) bank52573 +52574 POINT(28.865201105448318 -95.53334746665159) bank52574 +52575 POINT(28.855091123201284 -95.45575416357258) bank52575 +52576 POINT(28.97282390077245 -95.14944752842698) bank52576 +52577 POINT(28.889447125896446 -94.55024400335262) bank52577 +52578 POINT(29.27227127876274 -94.84640793465782) bank52578 +52579 POINT(29.399887631169754 -95.16835486762278) bank52579 +52580 POINT(30.010960369546055 -96.08431772628062) bank52580 +52581 POINT(29.96040926851916 -95.61292030843164) bank52581 +52582 POINT(29.0112889181069 -94.48474545226185) bank52582 +52583 POINT(30.73063201905088 -95.7292447161917) bank52583 +52584 POINT(29.21407629717487 -94.61813036227305) bank52584 +52585 POINT(29.854484149795375 -95.5511286273188) bank52585 +52586 POINT(29.42689329755115 -96.05678897018704) bank52586 +52587 POINT(30.25374426520313 -95.39737820971249) bank52587 +52588 POINT(30.333046495983893 -94.6782363765698) bank52588 +52589 POINT(29.9252181688388 -95.78230821382367) bank52589 +52590 POINT(29.276027996462197 -95.95729863146408) bank52590 +52591 POINT(29.242639636014264 -95.95405060602585) bank52591 +52592 POINT(29.906210346952562 -95.48835768792148) bank52592 +52593 POINT(30.014605663900713 -95.11638886696274) bank52593 +52594 POINT(30.189702965611655 -96.14742098805901) bank52594 +52595 POINT(29.323557049996065 -94.59076072748904) bank52595 +52596 POINT(30.6025025670247 -94.5895236637878) bank52596 +52597 POINT(30.502110882894023 -94.58102495815764) bank52597 +52598 POINT(30.43186360687773 -94.86138296170178) bank52598 +52599 POINT(29.513724920338646 -94.40772731553834) bank52599 +52600 POINT(30.530684720278114 -95.56675494268627) bank52600 +52601 POINT(29.276037628929146 -95.77559803244796) bank52601 +52602 POINT(29.616063109535347 -95.29991917770572) bank52602 +52603 POINT(30.499295250424094 -95.76490155697562) bank52603 +52604 POINT(30.553554944647356 -95.30125839869238) bank52604 +52605 POINT(29.242783326159127 -94.3860845858053) bank52605 +52606 POINT(30.07566029882706 -95.52222528332521) bank52606 +52607 POINT(29.099815190504373 -95.02219368524277) bank52607 +52608 POINT(29.17868625683211 -94.60165296528021) bank52608 +52609 POINT(28.988857243761032 -94.85434875951837) bank52609 +52610 POINT(29.78884612232699 -95.05271201358858) bank52610 +52611 POINT(29.04757541010345 -94.5523281199546) bank52611 +52612 POINT(29.050407121366938 -94.93898903768992) bank52612 +52613 POINT(28.85785138043117 -95.06644338221477) bank52613 +52614 POINT(30.397798818527352 -95.51893538823069) bank52614 +52615 POINT(29.481510614125312 -95.73093656673709) bank52615 +52616 POINT(29.40108949437187 -95.68292299932021) bank52616 +52617 POINT(29.69043853861817 -94.7690344136959) bank52617 +52618 POINT(29.52564803976565 -94.7411940070934) bank52618 +52619 POINT(30.683311077620214 -94.57448364004975) bank52619 +52620 POINT(30.306392944152346 -96.31821776767995) bank52620 +52621 POINT(29.18456065274288 -94.65862034382126) bank52621 +52622 POINT(30.44515914034599 -94.99708092303375) bank52622 +52623 POINT(29.24085926566358 -95.16374276798395) bank52623 +52624 POINT(29.40583477626715 -96.1269183387633) bank52624 +52625 POINT(29.254484923534978 -94.87749447769004) bank52625 +52626 POINT(29.059515643044875 -95.24527913929273) bank52626 +52627 POINT(29.529886147236045 -96.27723697443203) bank52627 +52628 POINT(30.293485956865435 -95.83043407097162) bank52628 +52629 POINT(29.60882710713967 -95.27097713917553) bank52629 +52630 POINT(29.774507275430736 -95.7887885288786) bank52630 +52631 POINT(30.49955318605003 -94.92333640253148) bank52631 +52632 POINT(30.291095419925952 -95.957996658488) bank52632 +52633 POINT(28.873089334655404 -95.42185826370917) bank52633 +52634 POINT(30.097031445060324 -94.89468575636361) bank52634 +52635 POINT(29.340677601571308 -94.55281718081568) bank52635 +52636 POINT(30.470078872982807 -96.26677186782153) bank52636 +52637 POINT(29.626546658347856 -94.55640665935883) bank52637 +52638 POINT(30.111460742427216 -95.70303734928041) bank52638 +52639 POINT(30.341449361434666 -94.40334960302577) bank52639 +52640 POINT(30.06158202806575 -95.29526109525057) bank52640 +52641 POINT(29.31178228703756 -94.54834641068976) bank52641 +52642 POINT(28.93702111017983 -95.67925654315907) bank52642 +52643 POINT(28.978378756547226 -94.88600214710722) bank52643 +52644 POINT(29.162486035403898 -95.3687645213901) bank52644 +52645 POINT(30.196696662511275 -94.93501078851578) bank52645 +52646 POINT(30.49607330426004 -95.43635120506447) bank52646 +52647 POINT(29.680533874448916 -95.33466582941135) bank52647 +52648 POINT(29.257383070157676 -95.00090908212091) bank52648 +52649 POINT(30.308440830895265 -94.76078068830904) bank52649 +52650 POINT(30.39759045837531 -95.61566197018637) bank52650 +52651 POINT(28.972224510588457 -95.14101692615701) bank52651 +52652 POINT(29.436133133758876 -94.56958730861575) bank52652 +52653 POINT(29.679069695129442 -96.1382029284915) bank52653 +52654 POINT(29.600920154404346 -94.92425613897773) bank52654 +52655 POINT(29.64843806614475 -95.60272611457239) bank52655 +52656 POINT(28.897703832378905 -94.37274763938213) bank52656 +52657 POINT(30.16187371682981 -95.48122950436148) bank52657 +52658 POINT(29.689832116836257 -94.60060428425255) bank52658 +52659 POINT(30.2826444000036 -94.46901903818376) bank52659 +52660 POINT(29.88484739439037 -96.09337151994963) bank52660 +52661 POINT(29.37424003334772 -95.24503989835773) bank52661 +52662 POINT(29.655636148334747 -95.92532671430386) bank52662 +52663 POINT(29.969567378375682 -96.04891160831022) bank52663 +52664 POINT(30.487505040664804 -95.5158043289475) bank52664 +52665 POINT(29.794626110726917 -95.3387219121263) bank52665 +52666 POINT(29.058921121810123 -95.2579237675764) bank52666 +52667 POINT(30.723896202245545 -95.75151710008987) bank52667 +52668 POINT(29.54847981775663 -95.94165539083902) bank52668 +52669 POINT(29.709194860648424 -94.3973659809531) bank52669 +52670 POINT(29.28072884803318 -95.32337739295066) bank52670 +52671 POINT(29.230002433374263 -94.8036858563388) bank52671 +52672 POINT(30.09638373055831 -96.00293048946105) bank52672 +52673 POINT(29.83895657918894 -94.7753669614115) bank52673 +52674 POINT(29.214742522373875 -95.25399595789227) bank52674 +52675 POINT(29.458036629680947 -94.64737083597284) bank52675 +52676 POINT(29.828263594995217 -95.18020983581741) bank52676 +52677 POINT(29.962383571134534 -94.94729671956759) bank52677 +52678 POINT(30.2904953814697 -95.08822077541699) bank52678 +52679 POINT(30.094503717972813 -95.93445494341844) bank52679 +52680 POINT(29.302869621652608 -95.03891887303143) bank52680 +52681 POINT(30.470375862708472 -95.42107503797484) bank52681 +52682 POINT(30.036724755529992 -96.32629573287478) bank52682 +52683 POINT(29.9150391710971 -95.19028674702027) bank52683 +52684 POINT(29.540110673501687 -95.35148690418615) bank52684 +52685 POINT(30.300995343287514 -95.59574297081853) bank52685 +52686 POINT(29.054764922231257 -95.086711311815) bank52686 +52687 POINT(29.511233450914677 -95.32414604762583) bank52687 +52688 POINT(30.391568581723142 -95.20269951663082) bank52688 +52689 POINT(30.281506683230806 -95.97439748795951) bank52689 +52690 POINT(30.041331276059754 -94.76362321401314) bank52690 +52691 POINT(30.434967549863988 -94.7346031312573) bank52691 +52692 POINT(28.989789389065432 -96.2453798629303) bank52692 +52693 POINT(30.160928392919672 -94.6380203455051) bank52693 +52694 POINT(30.442817842351115 -94.78143955198641) bank52694 +52695 POINT(29.254848509624587 -96.06103629529316) bank52695 +52696 POINT(29.965035592903988 -95.34512572738454) bank52696 +52697 POINT(30.052349525691803 -95.9818615275858) bank52697 +52698 POINT(29.83571996439681 -94.50161402952604) bank52698 +52699 POINT(30.12942331500086 -95.22279234423557) bank52699 +52700 POINT(29.645732285159156 -94.94581218547631) bank52700 +52701 POINT(30.439960610994554 -95.11837918287513) bank52701 +52702 POINT(30.29328030378193 -95.78108123495481) bank52702 +52703 POINT(30.175882095142583 -94.69894242773232) bank52703 +52704 POINT(29.99171634518385 -94.41008602966897) bank52704 +52705 POINT(30.007032925357695 -96.11428090841362) bank52705 +52706 POINT(30.09119024068438 -96.26868861240608) bank52706 +52707 POINT(30.588630784353562 -94.54041710734448) bank52707 +52708 POINT(30.221978382356195 -96.23769477385972) bank52708 +52709 POINT(28.879157279195436 -96.26405438808924) bank52709 +52710 POINT(30.53487501532164 -96.23197720614284) bank52710 +52711 POINT(28.99109275204869 -95.27206136934687) bank52711 +52712 POINT(29.141004039317785 -95.83320888299266) bank52712 +52713 POINT(29.20979835662906 -96.08607680515632) bank52713 +52714 POINT(30.44586648424485 -95.0985165421981) bank52714 +52715 POINT(28.882088826910106 -94.4295569929626) bank52715 +52716 POINT(29.875095724853892 -95.6798337887491) bank52716 +52717 POINT(30.15569898104963 -94.80978211691101) bank52717 +52718 POINT(29.01929126230642 -94.81181590599921) bank52718 +52719 POINT(29.810758330441107 -94.69816888585093) bank52719 +52720 POINT(29.02516255528065 -95.0173052049135) bank52720 +52721 POINT(29.486913502892513 -94.84702212026872) bank52721 +52722 POINT(29.457570529029425 -95.15052232447067) bank52722 +52723 POINT(29.502838247450292 -96.09715284826045) bank52723 +52724 POINT(29.247267588162934 -94.42510186382671) bank52724 +52725 POINT(29.90640339485123 -94.56237430422203) bank52725 +52726 POINT(29.83505171920043 -94.73034850287418) bank52726 +52727 POINT(30.636867490841617 -95.30646225036816) bank52727 +52728 POINT(29.353679880021907 -96.06312888767222) bank52728 +52729 POINT(29.41485834108106 -94.93284536227434) bank52729 +52730 POINT(29.846196657053845 -94.37805515923519) bank52730 +52731 POINT(29.23699870054885 -94.768007990526) bank52731 +52732 POINT(29.384847567923103 -94.54921996497576) bank52732 +52733 POINT(30.61652839836463 -95.30765948459927) bank52733 +52734 POINT(29.97632504869409 -94.57389802553513) bank52734 +52735 POINT(30.45353027140954 -94.63657493327837) bank52735 +52736 POINT(29.538569470531474 -95.34720346937989) bank52736 +52737 POINT(28.930377065971907 -94.80018986057968) bank52737 +52738 POINT(29.671419392555975 -95.66648507431377) bank52738 +52739 POINT(29.63908909024768 -95.48628620364786) bank52739 +52740 POINT(30.215670895456 -96.23816183528412) bank52740 +52741 POINT(29.028766620868907 -94.40156394473618) bank52741 +52742 POINT(30.645803202589242 -95.08937990699927) bank52742 +52743 POINT(30.206109353462164 -95.91182887885716) bank52743 +52744 POINT(30.39950400983912 -96.31704239457133) bank52744 +52745 POINT(29.161102670563537 -94.73671429334931) bank52745 +52746 POINT(30.72126091924159 -96.21429747596508) bank52746 +52747 POINT(29.563501055520764 -94.74597138178605) bank52747 +52748 POINT(29.265751860380043 -96.34620454949301) bank52748 +52749 POINT(30.486878963671703 -94.88922894332624) bank52749 +52750 POINT(29.292458735372637 -95.29902448337256) bank52750 +52751 POINT(28.881526334290932 -95.17696386418315) bank52751 +52752 POINT(29.290829972496546 -95.25932576342589) bank52752 +52753 POINT(29.21803269589785 -94.72874814733265) bank52753 +52754 POINT(30.130995397878888 -95.69737276112235) bank52754 +52755 POINT(30.437714588736434 -94.68509337765659) bank52755 +52756 POINT(29.744285788199893 -95.23622001412855) bank52756 +52757 POINT(28.959688853147753 -94.48115920258506) bank52757 +52758 POINT(30.716317791336035 -95.74059868110982) bank52758 +52759 POINT(29.162634532294934 -94.37876459282337) bank52759 +52760 POINT(29.004577426816507 -95.59372942070978) bank52760 +52761 POINT(28.930906253131976 -94.37469222864142) bank52761 +52762 POINT(30.195903448450643 -95.64840817372273) bank52762 +52763 POINT(30.687021044840872 -94.55917275296743) bank52763 +52764 POINT(30.395548829620612 -96.16269464777373) bank52764 +52765 POINT(30.074098292144434 -95.99469473365124) bank52765 +52766 POINT(30.7316658541048 -95.8326708555279) bank52766 +52767 POINT(29.17736606842993 -95.26651691897676) bank52767 +52768 POINT(29.09947284596907 -96.00601615226593) bank52768 +52769 POINT(29.155588434287292 -95.93646145289594) bank52769 +52770 POINT(29.559471977310697 -94.41128763183744) bank52770 +52771 POINT(30.617922135963486 -95.08141129208366) bank52771 +52772 POINT(29.599282159125867 -94.42761827307712) bank52772 +52773 POINT(29.971907457775508 -96.27663248390914) bank52773 +52774 POINT(29.344319371353915 -95.67894854031329) bank52774 +52775 POINT(28.798044505247802 -95.68731899858182) bank52775 +52776 POINT(29.840888110938135 -95.60835944819735) bank52776 +52777 POINT(29.168648869562592 -94.65848748688184) bank52777 +52778 POINT(30.708401616276376 -96.29768631458036) bank52778 +52779 POINT(29.44263797462022 -95.37690233726786) bank52779 +52780 POINT(28.810690003299385 -94.63290301902344) bank52780 +52781 POINT(30.321448056533896 -95.68894621960695) bank52781 +52782 POINT(29.20170857255122 -95.32061772359462) bank52782 +52783 POINT(29.408114730292343 -95.11594481155161) bank52783 +52784 POINT(30.73667911403294 -95.63774770382057) bank52784 +52785 POINT(29.125749356245258 -96.22833356344626) bank52785 +52786 POINT(29.115159108425004 -94.54126736882851) bank52786 +52787 POINT(30.37797427208076 -95.90656283431662) bank52787 +52788 POINT(30.46210766431263 -95.99560186492432) bank52788 +52789 POINT(30.498290147592776 -94.94084411010542) bank52789 +52790 POINT(29.217146082587348 -94.46590763087055) bank52790 +52791 POINT(30.653437126539274 -95.5818943989579) bank52791 +52792 POINT(29.781946440076414 -95.94436705413582) bank52792 +52793 POINT(29.600602924824926 -95.35335586659895) bank52793 +52794 POINT(29.898938331969966 -95.0519055286009) bank52794 +52795 POINT(29.52794499690427 -94.68886303878976) bank52795 +52796 POINT(30.184315742298605 -94.62238419085605) bank52796 +52797 POINT(29.535651816725636 -96.04183925261218) bank52797 +52798 POINT(28.769860885062975 -95.13308536047768) bank52798 +52799 POINT(29.835011527945063 -95.28362971520149) bank52799 +52800 POINT(29.616259455908736 -94.71149292675224) bank52800 +52801 POINT(30.44293923840489 -95.13662421566939) bank52801 +52802 POINT(29.188540708245814 -95.75839977448587) bank52802 +52803 POINT(29.757182364628484 -94.85004739999053) bank52803 +52804 POINT(30.12322190936971 -95.20385877421494) bank52804 +52805 POINT(29.846254451412737 -95.18927864651376) bank52805 +52806 POINT(29.168319913773335 -94.44913823069767) bank52806 +52807 POINT(29.42752096810337 -94.79081309926909) bank52807 +52808 POINT(29.93918324170698 -95.33143053365865) bank52808 +52809 POINT(30.72929786302101 -95.01290205143023) bank52809 +52810 POINT(29.475879088196166 -95.71208566599165) bank52810 +52811 POINT(30.01182253444861 -94.60913181334395) bank52811 +52812 POINT(29.00440798703213 -96.15563796786053) bank52812 +52813 POINT(29.25943097543829 -95.92736559641608) bank52813 +52814 POINT(29.78651411587876 -94.71002145374284) bank52814 +52815 POINT(30.10941745922341 -95.48519858169023) bank52815 +52816 POINT(29.36018646993413 -95.0495878155089) bank52816 +52817 POINT(29.87750993187159 -95.55006440059637) bank52817 +52818 POINT(28.8466249030918 -96.04871205566299) bank52818 +52819 POINT(29.599312273803594 -95.7992496617853) bank52819 +52820 POINT(29.24104753439208 -95.21711033096553) bank52820 +52821 POINT(28.95707533323084 -95.3878991514874) bank52821 +52822 POINT(29.125277804941582 -94.73315620531118) bank52822 +52823 POINT(29.67132032039337 -95.98364932625553) bank52823 +52824 POINT(28.853244158828 -95.2108509331728) bank52824 +52825 POINT(30.495281410138542 -95.7375503638706) bank52825 +52826 POINT(29.86031202385849 -95.76957213229046) bank52826 +52827 POINT(29.788799928367872 -95.70069095235078) bank52827 +52828 POINT(29.49566865288665 -94.86076333169981) bank52828 +52829 POINT(29.9175031079735 -94.48052173525326) bank52829 +52830 POINT(28.898926432676426 -96.19277916631381) bank52830 +52831 POINT(29.308297939974402 -94.61550194291615) bank52831 +52832 POINT(30.352324992594657 -95.88596388492272) bank52832 +52833 POINT(29.436407493450037 -95.39995087285699) bank52833 +52834 POINT(30.34420460389076 -95.95459487306653) bank52834 +52835 POINT(29.5181766899796 -95.06509697093449) bank52835 +52836 POINT(29.47139754481377 -95.90401481918927) bank52836 +52837 POINT(29.12729502120094 -95.58612675177946) bank52837 +52838 POINT(30.27901583075503 -95.03898205333694) bank52838 +52839 POINT(30.031726810107426 -94.45967273560528) bank52839 +52840 POINT(29.99092638433959 -94.57372632477363) bank52840 +52841 POINT(30.659646782394393 -96.0573152902755) bank52841 +52842 POINT(28.848105396555084 -95.91422209206479) bank52842 +52843 POINT(29.42943813357189 -95.48248655136113) bank52843 +52844 POINT(30.681526621997975 -95.62512526764479) bank52844 +52845 POINT(29.46102254290856 -94.9280659426805) bank52845 +52846 POINT(29.253411615013814 -95.49550927105439) bank52846 +52847 POINT(29.40314767708523 -94.65387058838762) bank52847 +52848 POINT(30.35186018820243 -95.00774746117379) bank52848 +52849 POINT(29.94352127436764 -94.81874354386373) bank52849 +52850 POINT(30.23144653162182 -95.71572531309548) bank52850 +52851 POINT(29.606893762674034 -95.66779258092504) bank52851 +52852 POINT(30.328730207974736 -95.82691514850744) bank52852 +52853 POINT(29.166154862990716 -95.7983386451656) bank52853 +52854 POINT(30.54381436203465 -94.40533044170604) bank52854 +52855 POINT(29.36352780165798 -94.78387631353439) bank52855 +52856 POINT(29.681982937500756 -95.1424652241508) bank52856 +52857 POINT(30.348298379605552 -95.4769562017382) bank52857 +52858 POINT(29.741964177092854 -95.25429101921661) bank52858 +52859 POINT(30.70107586600632 -95.01835489512366) bank52859 +52860 POINT(30.130176868778772 -95.79509012291818) bank52860 +52861 POINT(29.818773071461862 -96.11968241191008) bank52861 +52862 POINT(29.97010486962118 -95.91190790346668) bank52862 +52863 POINT(28.811595605400328 -95.61989172927402) bank52863 +52864 POINT(29.301135376658944 -94.94560602550031) bank52864 +52865 POINT(29.618252907933346 -95.86161929469102) bank52865 +52866 POINT(29.85094294139157 -95.88878257377469) bank52866 +52867 POINT(30.500586837898048 -95.68310015954012) bank52867 +52868 POINT(30.297178210699215 -94.41849240934255) bank52868 +52869 POINT(30.25305898129365 -96.31121972056384) bank52869 +52870 POINT(28.954491290549957 -95.36897058426422) bank52870 +52871 POINT(30.29576155579298 -95.02773062981822) bank52871 +52872 POINT(30.73314931355541 -94.93596880767076) bank52872 +52873 POINT(30.008199434203302 -95.46233590075343) bank52873 +52874 POINT(30.75502424354243 -95.15574800989774) bank52874 +52875 POINT(29.85594043342296 -94.50661556264764) bank52875 +52876 POINT(29.909769309322076 -94.47138697701456) bank52876 +52877 POINT(29.231697354246773 -95.46886611715091) bank52877 +52878 POINT(29.94748468557063 -95.86633022160815) bank52878 +52879 POINT(30.173065572351828 -95.86078755140562) bank52879 +52880 POINT(29.518739256978986 -95.5384796835675) bank52880 +52881 POINT(29.574992674128467 -95.44860536709406) bank52881 +52882 POINT(29.44465586682902 -95.83467224500473) bank52882 +52883 POINT(29.361731633963014 -95.48701372404349) bank52883 +52884 POINT(30.423996518319512 -95.1805793910151) bank52884 +52885 POINT(29.53923424136866 -95.13027293573137) bank52885 +52886 POINT(30.731349968956852 -95.43975478859822) bank52886 +52887 POINT(29.82005030571989 -94.91498723020882) bank52887 +52888 POINT(29.667571521213134 -96.35129174921494) bank52888 +52889 POINT(30.61562137435165 -96.30948421615881) bank52889 +52890 POINT(29.047068781693422 -94.85802804095495) bank52890 +52891 POINT(30.588784243835295 -95.90962033029038) bank52891 +52892 POINT(29.731519908288472 -96.34381637392194) bank52892 +52893 POINT(30.67343120390932 -95.61805507859073) bank52893 +52894 POINT(29.15841249725698 -94.61076379412013) bank52894 +52895 POINT(30.0701666746766 -94.49847149351734) bank52895 +52896 POINT(30.628990082118296 -95.62580590085491) bank52896 +52897 POINT(29.613564467687016 -95.46255406175224) bank52897 +52898 POINT(29.35641783924422 -95.24346925493465) bank52898 +52899 POINT(29.816147682317396 -96.36314992113275) bank52899 +52900 POINT(29.695548855806933 -95.14188391414865) bank52900 +52901 POINT(29.154801223488942 -95.71219416615473) bank52901 +52902 POINT(30.556371970074558 -95.59543906136341) bank52902 +52903 POINT(30.29253719114978 -95.2115918153455) bank52903 +52904 POINT(29.374425083572227 -95.56123427419277) bank52904 +52905 POINT(30.462185109559204 -94.68953236324263) bank52905 +52906 POINT(29.052953444208708 -95.44544093267619) bank52906 +52907 POINT(30.05156555858692 -95.66403132632674) bank52907 +52908 POINT(29.255590419599624 -95.45399144755173) bank52908 +52909 POINT(30.448894880034906 -95.99702486642154) bank52909 +52910 POINT(29.07429846656997 -94.47691871708567) bank52910 +52911 POINT(29.58465216824651 -95.87865242490541) bank52911 +52912 POINT(29.32108633450057 -95.30512239065159) bank52912 +52913 POINT(29.325010393207492 -95.51075834591563) bank52913 +52914 POINT(29.671766585147132 -96.31707064976982) bank52914 +52915 POINT(29.124370538961646 -96.1905146492718) bank52915 +52916 POINT(28.826199955454967 -95.44916375699411) bank52916 +52917 POINT(28.983327343187412 -94.91529499370789) bank52917 +52918 POINT(28.87152769500066 -95.65448900622465) bank52918 +52919 POINT(30.61363696442157 -96.27337411148349) bank52919 +52920 POINT(30.27334233140646 -95.4072154902282) bank52920 +52921 POINT(30.04969539132811 -94.63092798818951) bank52921 +52922 POINT(29.592529778716404 -95.65707937870482) bank52922 +52923 POINT(30.276387581129097 -95.99746579661604) bank52923 +52924 POINT(30.6348935567867 -95.46800666403746) bank52924 +52925 POINT(28.938367736408917 -94.96117526854427) bank52925 +52926 POINT(29.88576964657285 -96.03154605216851) bank52926 +52927 POINT(29.74185090363054 -95.2059162195097) bank52927 +52928 POINT(30.610228043987373 -95.74052467724715) bank52928 +52929 POINT(29.176024374801166 -96.14536961498202) bank52929 +52930 POINT(28.777467352149117 -95.95808270571217) bank52930 +52931 POINT(30.481706278104017 -95.32390757844331) bank52931 +52932 POINT(30.41280504858737 -96.05537671558805) bank52932 +52933 POINT(29.855386273163933 -94.88405266745943) bank52933 +52934 POINT(28.905746543433093 -94.82095687759676) bank52934 +52935 POINT(29.885897408492266 -96.11475977663152) bank52935 +52936 POINT(30.1918627429969 -94.5514437655054) bank52936 +52937 POINT(29.934806038543496 -95.24478530200078) bank52937 +52938 POINT(30.308760902577056 -94.69339493968084) bank52938 +52939 POINT(28.83358949090964 -95.36534518384326) bank52939 +52940 POINT(29.157202464687668 -94.88217939464451) bank52940 +52941 POINT(28.902185980068012 -94.59699579034437) bank52941 +52942 POINT(29.648378687365334 -96.14977899896012) bank52942 +52943 POINT(29.598817673010775 -94.68009660523754) bank52943 +52944 POINT(29.053916157520717 -94.77025617311004) bank52944 +52945 POINT(28.946806449701228 -96.0389984121028) bank52945 +52946 POINT(28.94960696924099 -96.3606646367744) bank52946 +52947 POINT(29.362549704727396 -95.4229047889582) bank52947 +52948 POINT(28.869596267793963 -95.41197558500077) bank52948 +52949 POINT(29.817966372328566 -94.41396714948716) bank52949 +52950 POINT(30.201581595220603 -95.5786070407047) bank52950 +52951 POINT(29.182908313971453 -95.60433540546272) bank52951 +52952 POINT(28.968973499352895 -94.50057916325933) bank52952 +52953 POINT(30.139332842197057 -96.19508286700604) bank52953 +52954 POINT(30.103810403526985 -95.77762828295734) bank52954 +52955 POINT(30.19538544558074 -95.24402427371575) bank52955 +52956 POINT(28.891406941715797 -96.04338614275959) bank52956 +52957 POINT(29.385083878324817 -94.85414916937418) bank52957 +52958 POINT(28.92518045842589 -95.6466139237829) bank52958 +52959 POINT(30.371918206965912 -95.87425451566502) bank52959 +52960 POINT(30.519618294452215 -94.39620686147667) bank52960 +52961 POINT(30.236981436897224 -95.09691182584208) bank52961 +52962 POINT(28.83466863507444 -95.30607830169566) bank52962 +52963 POINT(29.380175127613715 -94.90448241164694) bank52963 +52964 POINT(30.6544212426945 -94.82614602680488) bank52964 +52965 POINT(29.227734498324853 -95.2926193974953) bank52965 +52966 POINT(29.073751106447045 -95.74098985629873) bank52966 +52967 POINT(29.788052173522573 -94.68502464782166) bank52967 +52968 POINT(29.744883847608467 -94.6722250249429) bank52968 +52969 POINT(30.117172847273405 -95.39361633383932) bank52969 +52970 POINT(30.41542401879688 -95.86332788366701) bank52970 +52971 POINT(30.268889912473036 -95.50426602819422) bank52971 +52972 POINT(29.73976530963662 -95.71291888128435) bank52972 +52973 POINT(29.89937435339961 -95.59462321403768) bank52973 +52974 POINT(29.62798866369781 -96.23308699687647) bank52974 +52975 POINT(28.84185623982275 -96.07372269570315) bank52975 +52976 POINT(29.757691799795133 -94.91604759065166) bank52976 +52977 POINT(29.168106399993803 -94.702453500978) bank52977 +52978 POINT(29.675528327487086 -94.897157792466) bank52978 +52979 POINT(29.660179698781743 -95.8594101720596) bank52979 +52980 POINT(28.77493896516197 -95.52695422120655) bank52980 +52981 POINT(29.762040967964666 -95.59178442268136) bank52981 +52982 POINT(29.953797212332375 -95.12296614662687) bank52982 +52983 POINT(28.95975729635396 -96.27579812769977) bank52983 +52984 POINT(30.40016645936611 -94.48399758023325) bank52984 +52985 POINT(28.85460084510338 -95.35763131647678) bank52985 +52986 POINT(29.88313994704834 -95.62679335623442) bank52986 +52987 POINT(30.20627851241444 -95.30237154677644) bank52987 +52988 POINT(28.828146016740817 -95.5310183163346) bank52988 +52989 POINT(28.82594773765479 -94.49388303017493) bank52989 +52990 POINT(29.314397714564205 -94.58954514701385) bank52990 +52991 POINT(30.75920646446365 -95.32392111975432) bank52991 +52992 POINT(29.610650119428183 -95.32232516063794) bank52992 +52993 POINT(30.306995368558763 -95.40697973974112) bank52993 +52994 POINT(30.490056606483044 -96.04427614136114) bank52994 +52995 POINT(29.447034656513576 -95.0793108975526) bank52995 +52996 POINT(29.860497174789142 -95.2489878745251) bank52996 +52997 POINT(30.63541868479393 -94.73956365983857) bank52997 +52998 POINT(29.568592954585785 -96.27526372352119) bank52998 +52999 POINT(29.93119910442925 -95.3030408945539) bank52999 +53000 POINT(30.548950684766613 -94.66411273561015) bank53000 +53001 POINT(30.131760442263015 -95.07875469670874) bank53001 +53002 POINT(29.537621197160387 -94.65038647783058) bank53002 +53003 POINT(30.57146758632718 -96.23221374055765) bank53003 +53004 POINT(30.141505611751686 -95.48492027595337) bank53004 +53005 POINT(30.099577504769343 -95.42731229669693) bank53005 +53006 POINT(29.757673579507212 -95.41903956714928) bank53006 +53007 POINT(30.43652188188106 -95.35350536878971) bank53007 +53008 POINT(30.137757269189215 -94.63240934135155) bank53008 +53009 POINT(30.756456860107935 -95.16524980675936) bank53009 +53010 POINT(29.928879036174926 -94.38323283589675) bank53010 +53011 POINT(29.498542090234878 -94.37494169899658) bank53011 +53012 POINT(30.419558310248703 -95.32179384131122) bank53012 +53013 POINT(30.135289995149577 -95.93258276267369) bank53013 +53014 POINT(30.051163899384786 -95.36459806175027) bank53014 +53015 POINT(28.919963670236726 -96.2105606920445) bank53015 +53016 POINT(29.496495712536035 -94.78966129856428) bank53016 +53017 POINT(29.64298881550536 -95.90264094866745) bank53017 +53018 POINT(29.08591160155356 -94.60524882283619) bank53018 +53019 POINT(29.55751090784466 -95.08366478524195) bank53019 +53020 POINT(29.8279728983792 -95.06668012407539) bank53020 +53021 POINT(29.73687646855115 -94.79102840748368) bank53021 +53022 POINT(30.427065714989514 -94.45197398793435) bank53022 +53023 POINT(28.9031123956475 -95.09909057875045) bank53023 +53024 POINT(29.09786672294154 -94.73754411899883) bank53024 +53025 POINT(29.97257458572782 -95.60609744222167) bank53025 +53026 POINT(30.109919639436878 -94.58884384209941) bank53026 +53027 POINT(30.15289012940062 -95.02623016964898) bank53027 +53028 POINT(29.649803133647914 -94.39181174459394) bank53028 +53029 POINT(29.30204661026807 -96.23291591755424) bank53029 +53030 POINT(29.651534270563783 -96.22942803045292) bank53030 +53031 POINT(30.50950659903244 -95.33408058320643) bank53031 +53032 POINT(29.763742379543313 -94.64135154299441) bank53032 +53033 POINT(30.27031294931463 -96.36467788228912) bank53033 +53034 POINT(30.3430368180004 -94.608990320354) bank53034 +53035 POINT(29.986027284882546 -95.51091248407982) bank53035 +53036 POINT(29.937758855312552 -94.7183816424379) bank53036 +53037 POINT(28.90541565993778 -95.06644297468479) bank53037 +53038 POINT(29.394449457954266 -94.55141575164977) bank53038 +53039 POINT(30.61377179120186 -94.57743014433964) bank53039 +53040 POINT(29.511743266684633 -96.31265079001795) bank53040 +53041 POINT(29.40432399785156 -94.70203154561655) bank53041 +53042 POINT(29.226328064478952 -95.044960703804) bank53042 +53043 POINT(29.46531843219214 -95.31679664640643) bank53043 +53044 POINT(29.75536667641266 -96.2184693421993) bank53044 +53045 POINT(30.539652113740264 -95.35514055064186) bank53045 +53046 POINT(30.292320147414426 -96.30620417415362) bank53046 +53047 POINT(30.715508323366212 -94.88880930299403) bank53047 +53048 POINT(30.71045608193109 -95.57922651062171) bank53048 +53049 POINT(30.29692756777919 -96.11067225644034) bank53049 +53050 POINT(29.49466980265572 -94.46211361749387) bank53050 +53051 POINT(28.859159829488 -95.31569390218228) bank53051 +53052 POINT(30.17534884193329 -95.05041681117834) bank53052 +53053 POINT(29.788938469936575 -94.49667739653141) bank53053 +53054 POINT(30.44132002438879 -96.2974621347386) bank53054 +53055 POINT(30.481311608903212 -95.27517501628505) bank53055 +53056 POINT(30.29447213675381 -95.32518620077121) bank53056 +53057 POINT(29.514021660340525 -96.24590708442888) bank53057 +53058 POINT(30.173783199595746 -95.54677190692803) bank53058 +53059 POINT(29.11776313187137 -94.45949505625511) bank53059 +53060 POINT(30.26254010579945 -95.36470221511743) bank53060 +53061 POINT(29.193857579233224 -95.37289390757718) bank53061 +53062 POINT(30.27896295363519 -95.3513286183) bank53062 +53063 POINT(29.60400398038686 -95.71742529728928) bank53063 +53064 POINT(28.85104861765253 -94.57084948997189) bank53064 +53065 POINT(29.273808254996634 -95.05499498927237) bank53065 +53066 POINT(29.695575401277583 -96.16677475737588) bank53066 +53067 POINT(30.723346237073407 -95.22067588335432) bank53067 +53068 POINT(29.084619903758757 -95.3305156034955) bank53068 +53069 POINT(30.00915955314163 -94.91432463416682) bank53069 +53070 POINT(28.86727388004258 -95.05858715481335) bank53070 +53071 POINT(29.77784216937899 -95.9518907534645) bank53071 +53072 POINT(30.467887346335758 -95.41066092016774) bank53072 +53073 POINT(29.939136797866972 -95.7330929801453) bank53073 +53074 POINT(30.137666253330885 -95.54506125756384) bank53074 +53075 POINT(29.134522082777753 -95.54850186637431) bank53075 +53076 POINT(29.872912843911905 -94.85927388288916) bank53076 +53077 POINT(29.483795184747258 -95.61027241767714) bank53077 +53078 POINT(29.095619224522746 -94.60935130542849) bank53078 +53079 POINT(30.156442799960942 -95.34837227313744) bank53079 +53080 POINT(29.87516576063057 -95.04736074331421) bank53080 +53081 POINT(30.61927243674974 -94.61847714160868) bank53081 +53082 POINT(29.222229496728662 -95.24183252190225) bank53082 +53083 POINT(29.88167208350061 -94.53790388827976) bank53083 +53084 POINT(29.070216159199944 -95.05394197407183) bank53084 +53085 POINT(29.15072446678084 -95.27548973603004) bank53085 +53086 POINT(29.25361839914127 -95.3943897990125) bank53086 +53087 POINT(28.911525244331685 -94.42522655466644) bank53087 +53088 POINT(30.335299194789286 -95.67679361447516) bank53088 +53089 POINT(29.241671846265916 -96.26344601754602) bank53089 +53090 POINT(29.871250581380465 -94.74477910173485) bank53090 +53091 POINT(29.51606842459065 -95.65824336058236) bank53091 +53092 POINT(29.924538507500486 -94.85869876291491) bank53092 +53093 POINT(28.950300873647045 -94.89749700539733) bank53093 +53094 POINT(29.979432332885445 -94.54946305303525) bank53094 +53095 POINT(29.712075669088076 -95.75495792561624) bank53095 +53096 POINT(29.709368209164357 -95.2783428141522) bank53096 +53097 POINT(30.197435765729306 -96.09179769049462) bank53097 +53098 POINT(29.86232710617514 -95.2237343246551) bank53098 +53099 POINT(29.55819164128826 -94.63096742202863) bank53099 +53100 POINT(29.850544237325483 -95.43413222047269) bank53100 +53101 POINT(29.192356109740718 -94.94591949091237) bank53101 +53102 POINT(30.631791118029344 -95.20538754260969) bank53102 +53103 POINT(29.369598290573457 -96.15791707866808) bank53103 +53104 POINT(29.6550659603615 -96.11433815107965) bank53104 +53105 POINT(29.167535709788357 -94.9663073311818) bank53105 +53106 POINT(30.383074588669007 -96.31305213052106) bank53106 +53107 POINT(30.68819469714916 -96.34032786905848) bank53107 +53108 POINT(30.158399204792538 -95.81529785688653) bank53108 +53109 POINT(29.52492376307091 -96.18840746545926) bank53109 +53110 POINT(29.903320115559563 -96.32634863551525) bank53110 +53111 POINT(29.24370986521784 -95.17659901048171) bank53111 +53112 POINT(30.026061655180147 -94.52176557711196) bank53112 +53113 POINT(28.850667132669102 -95.85816929828603) bank53113 +53114 POINT(30.396383205994216 -96.07188293410495) bank53114 +53115 POINT(29.80081912438689 -95.1726033195249) bank53115 +53116 POINT(29.45353463881428 -95.0916703463418) bank53116 +53117 POINT(29.061852189668805 -95.10915076948861) bank53117 +53118 POINT(30.624795484393374 -94.92318532409554) bank53118 +53119 POINT(29.548453878898773 -94.79620542627352) bank53119 +53120 POINT(29.18066701851059 -96.04926725484036) bank53120 +53121 POINT(29.847309372989557 -94.49082130020521) bank53121 +53122 POINT(30.692853252486152 -95.69606076816488) bank53122 +53123 POINT(30.580453588062095 -96.18268440623808) bank53123 +53124 POINT(29.81855675475833 -94.4136200137811) bank53124 +53125 POINT(30.208852442205767 -96.01624613611244) bank53125 +53126 POINT(30.55561604579211 -95.81942617322676) bank53126 +53127 POINT(29.425607531136905 -94.55628709940055) bank53127 +53128 POINT(30.715885577873127 -94.37279169546059) bank53128 +53129 POINT(29.971529729404423 -94.6794894315332) bank53129 +53130 POINT(29.094288723419044 -95.2103600669291) bank53130 +53131 POINT(29.65464761153408 -96.23261088010845) bank53131 +53132 POINT(29.050031444491022 -95.00484983255669) bank53132 +53133 POINT(29.267229252018012 -95.36942740839172) bank53133 +53134 POINT(30.062402377908754 -94.5435132524927) bank53134 +53135 POINT(29.729616771583306 -95.55399737715592) bank53135 +53136 POINT(30.2149159717247 -95.45260167387625) bank53136 +53137 POINT(29.049495253558813 -95.06996472265493) bank53137 +53138 POINT(29.939486748967628 -94.73731136952708) bank53138 +53139 POINT(29.662075395951252 -95.72318094243934) bank53139 +53140 POINT(29.77670581099999 -95.50340477490877) bank53140 +53141 POINT(30.177930855869327 -94.88427949235697) bank53141 +53142 POINT(29.22866089322207 -95.80320079331858) bank53142 +53143 POINT(29.97342788753619 -95.09845101449883) bank53143 +53144 POINT(30.14270975123758 -95.35837410660562) bank53144 +53145 POINT(28.894364893798663 -95.74678624896285) bank53145 +53146 POINT(28.849485785702203 -94.45872879207887) bank53146 +53147 POINT(28.798051006616486 -95.81494146995253) bank53147 +53148 POINT(29.121987365273544 -95.0675811546197) bank53148 +53149 POINT(28.871776358490266 -95.53441135806662) bank53149 +53150 POINT(30.727379493185364 -95.75710199694463) bank53150 +53151 POINT(29.3331636484638 -94.52341456790387) bank53151 +53152 POINT(30.44230601147223 -94.84188532861344) bank53152 +53153 POINT(29.11308699218603 -95.89648282404593) bank53153 +53154 POINT(28.886581905406963 -94.95640397796875) bank53154 +53155 POINT(29.657090955628597 -94.4845704268037) bank53155 +53156 POINT(29.286912777773942 -94.98697905836087) bank53156 +53157 POINT(30.33870655446199 -94.58195133272535) bank53157 +53158 POINT(29.48869277047643 -95.14907689245422) bank53158 +53159 POINT(30.378088021259263 -95.40455983080136) bank53159 +53160 POINT(30.6571586335804 -96.027921486061) bank53160 +53161 POINT(29.82348026600781 -96.13790869423133) bank53161 +53162 POINT(29.268996950425237 -95.95907293862335) bank53162 +53163 POINT(29.98750179658769 -94.719120769007) bank53163 +53164 POINT(29.804812181522394 -95.41618530556048) bank53164 +53165 POINT(29.843325987076163 -95.55921922369305) bank53165 +53166 POINT(30.347237264738773 -95.50303841009945) bank53166 +53167 POINT(29.47747829943132 -95.85617087030283) bank53167 +53168 POINT(30.063812575270198 -94.66565163574394) bank53168 +53169 POINT(30.664485344176693 -95.83715656443424) bank53169 +53170 POINT(28.874891786267824 -96.0989127375842) bank53170 +53171 POINT(28.794402836426542 -95.22770647880549) bank53171 +53172 POINT(29.10383355795302 -95.59518350219395) bank53172 +53173 POINT(29.433100876702973 -95.09995273666155) bank53173 +53174 POINT(29.04936113245301 -95.8050037383492) bank53174 +53175 POINT(30.585216572998775 -95.3373551869029) bank53175 +53176 POINT(28.818323420953597 -94.90668558927767) bank53176 +53177 POINT(29.923508577991427 -95.79504999286236) bank53177 +53178 POINT(29.33672739066607 -95.44329902277511) bank53178 +53179 POINT(29.38728728751873 -95.67920673702565) bank53179 +53180 POINT(29.003004325033782 -95.4206377396481) bank53180 +53181 POINT(29.00675135413069 -95.09853658464472) bank53181 +53182 POINT(29.26552622914246 -94.40222831832409) bank53182 +53183 POINT(29.660800250719106 -96.1135027865247) bank53183 +53184 POINT(29.920936006300828 -96.12935599604302) bank53184 +53185 POINT(30.543887804289177 -94.77552210747177) bank53185 +53186 POINT(29.699212577433116 -94.59329087695228) bank53186 +53187 POINT(29.150345702859397 -94.63759968055078) bank53187 +53188 POINT(30.38181049054296 -94.52513126543126) bank53188 +53189 POINT(29.854922443621156 -95.51708599158717) bank53189 +53190 POINT(28.95477062258547 -96.34743063169782) bank53190 +53191 POINT(29.582893638105823 -95.5638004974482) bank53191 +53192 POINT(30.138591395465944 -94.42099464891113) bank53192 +53193 POINT(29.89954848056331 -95.11170440333532) bank53193 +53194 POINT(30.12863018689017 -96.08581435126578) bank53194 +53195 POINT(28.791365451971547 -95.6650703199464) bank53195 +53196 POINT(30.164152643654464 -95.4730318945995) bank53196 +53197 POINT(29.24366910915408 -94.47332363745386) bank53197 +53198 POINT(30.342040210557645 -95.17138321328679) bank53198 +53199 POINT(29.610494095171173 -95.45652261027007) bank53199 +53200 POINT(28.997345617996665 -94.75962166017398) bank53200 +53201 POINT(30.738011484944316 -95.51893743452185) bank53201 +53202 POINT(28.83219572970313 -94.56628121760068) bank53202 +53203 POINT(30.468694856935002 -95.90089732422328) bank53203 +53204 POINT(29.116708486567447 -94.85958894439796) bank53204 +53205 POINT(29.35247410205 -96.09458215887055) bank53205 +53206 POINT(28.83591971627429 -95.29891447244093) bank53206 +53207 POINT(28.87369573209488 -94.52137694124626) bank53207 +53208 POINT(29.834988583306764 -95.78545392490996) bank53208 +53209 POINT(30.542190642351482 -95.97815817541847) bank53209 +53210 POINT(28.839342683207963 -96.23315546598948) bank53210 +53211 POINT(29.050192868189455 -95.40115025053984) bank53211 +53212 POINT(29.911387383284296 -96.29373814388097) bank53212 +53213 POINT(29.02704483833307 -95.04570201013446) bank53213 +53214 POINT(30.045230062335353 -95.49489846334015) bank53214 +53215 POINT(28.928538596663966 -95.75615668643057) bank53215 +53216 POINT(29.192662347562763 -96.32842805549195) bank53216 +53217 POINT(30.526257233071668 -95.11734724054728) bank53217 +53218 POINT(28.902243823448543 -96.01942186795185) bank53218 +53219 POINT(30.00244256460455 -96.03158987336234) bank53219 +53220 POINT(30.71390292905095 -94.97682367239837) bank53220 +53221 POINT(29.98136449206301 -94.6368415359945) bank53221 +53222 POINT(30.21914500105123 -94.75386331020599) bank53222 +53223 POINT(30.35118443314903 -95.48880666275151) bank53223 +53224 POINT(29.056772858874517 -94.52226789799066) bank53224 +53225 POINT(28.877645223077334 -94.37650422242908) bank53225 +53226 POINT(30.602519406151412 -94.91583331381626) bank53226 +53227 POINT(29.805110149748895 -96.12196301041482) bank53227 +53228 POINT(30.319728696435625 -95.22313820793413) bank53228 +53229 POINT(30.54549259025166 -94.7260682506711) bank53229 +53230 POINT(30.122918785842483 -95.33431158063546) bank53230 +53231 POINT(29.968701187976052 -95.99035456141259) bank53231 +53232 POINT(29.70343051054526 -94.45821046970077) bank53232 +53233 POINT(28.835851499767962 -96.1304554820494) bank53233 +53234 POINT(29.08392166315187 -96.15792290663238) bank53234 +53235 POINT(30.262267056273885 -95.28726556793511) bank53235 +53236 POINT(29.33532797804559 -94.62987951606652) bank53236 +53237 POINT(29.010471690218324 -95.95185118038766) bank53237 +53238 POINT(29.282045056508956 -95.48870971518981) bank53238 +53239 POINT(30.67720537470743 -94.46677799876724) bank53239 +53240 POINT(29.455138184010078 -95.29234012105066) bank53240 +53241 POINT(29.775922359050714 -94.83989214114503) bank53241 +53242 POINT(28.96103861772394 -95.1102583237672) bank53242 +53243 POINT(29.267641653086198 -96.09783269723937) bank53243 +53244 POINT(30.750017717338267 -94.89570303915256) bank53244 +53245 POINT(29.095629106494464 -94.62555352275771) bank53245 +53246 POINT(30.618581087786453 -94.50147035761175) bank53246 +53247 POINT(29.277604346939786 -95.05533609887443) bank53247 +53248 POINT(30.40304536652573 -95.42726559648726) bank53248 +53249 POINT(29.815797280746583 -94.54255319738397) bank53249 +53250 POINT(29.21058157962144 -95.53527795068266) bank53250 +53251 POINT(29.3155253504925 -95.94920778839777) bank53251 +53252 POINT(29.37134691352009 -96.25200924022933) bank53252 +53253 POINT(30.31270094922339 -96.11172845204251) bank53253 +53254 POINT(30.350491276620357 -94.91776844178784) bank53254 +53255 POINT(29.642850478685812 -95.6738615427365) bank53255 +53256 POINT(29.654845568183248 -96.00444194084969) bank53256 +53257 POINT(30.161011130313682 -95.78685896897176) bank53257 +53258 POINT(30.336835371002053 -96.01965570442593) bank53258 +53259 POINT(30.649876950498193 -95.00626484781371) bank53259 +53260 POINT(29.741684546049175 -95.48021514464118) bank53260 +53261 POINT(30.22827321530502 -95.73572620741544) bank53261 +53262 POINT(29.14791593992685 -95.62104382633366) bank53262 +53263 POINT(30.562600901407066 -95.5623354970469) bank53263 +53264 POINT(28.813249048164668 -94.99566686001873) bank53264 +53265 POINT(29.711301451735782 -95.00235116294573) bank53265 +53266 POINT(30.547300698723056 -95.89439891942827) bank53266 +53267 POINT(30.596035139461975 -95.41511095241532) bank53267 +53268 POINT(30.395726259446214 -95.51279826578872) bank53268 +53269 POINT(30.393388078418887 -94.69752669163725) bank53269 +53270 POINT(29.57760668313915 -95.57805151953907) bank53270 +53271 POINT(29.83840940988537 -94.55526888349512) bank53271 +53272 POINT(30.75897532894743 -95.79515572049294) bank53272 +53273 POINT(29.59398189998189 -95.49183445376427) bank53273 +53274 POINT(29.412696645837784 -95.00095962855467) bank53274 +53275 POINT(29.609757178422658 -94.40796762521025) bank53275 +53276 POINT(29.308812729548443 -94.55619167890511) bank53276 +53277 POINT(30.14069160463784 -94.91649292160184) bank53277 +53278 POINT(29.33317483549258 -94.64940118150058) bank53278 +53279 POINT(30.656307485421628 -95.29938614818629) bank53279 +53280 POINT(29.548938576702422 -96.16711579542354) bank53280 +53281 POINT(29.761645919057237 -95.97669825509941) bank53281 +53282 POINT(30.347619521653353 -94.80488085920159) bank53282 +53283 POINT(29.997131314477123 -95.17454248684977) bank53283 +53284 POINT(29.180345265477047 -94.5968379230422) bank53284 +53285 POINT(29.17785919358936 -94.92903224625529) bank53285 +53286 POINT(29.254138610219787 -94.45672311792416) bank53286 +53287 POINT(30.582121104288774 -96.20757707479211) bank53287 +53288 POINT(30.16281709988548 -95.53758294514581) bank53288 +53289 POINT(29.480888618959135 -94.54455107918142) bank53289 +53290 POINT(29.758974985256646 -94.91329186901014) bank53290 +53291 POINT(29.413070766893103 -95.77764426339161) bank53291 +53292 POINT(29.09518917925554 -94.4637553889534) bank53292 +53293 POINT(30.7066580725098 -95.29062627471582) bank53293 +53294 POINT(29.150496641844256 -95.59479541031536) bank53294 +53295 POINT(28.957174076929654 -95.17667077158733) bank53295 +53296 POINT(29.89784690117774 -95.05190422110304) bank53296 +53297 POINT(30.222971951764222 -94.42176019900678) bank53297 +53298 POINT(28.940002464670787 -96.34847469332753) bank53298 +53299 POINT(29.359634553007723 -95.26993765639301) bank53299 +53300 POINT(30.432881890998292 -95.49334036127085) bank53300 +53301 POINT(29.282473443629378 -94.89708393878527) bank53301 +53302 POINT(29.819934673331485 -95.3436779164934) bank53302 +53303 POINT(29.50402702519939 -95.95652875143624) bank53303 +53304 POINT(30.630957951332324 -94.93086656207164) bank53304 +53305 POINT(30.508649289453835 -94.7821075694474) bank53305 +53306 POINT(29.117241826788792 -95.42582825978171) bank53306 +53307 POINT(28.98057930062982 -96.22265958242039) bank53307 +53308 POINT(30.53675878053074 -96.14263308652423) bank53308 +53309 POINT(30.369708976415463 -94.98436287030925) bank53309 +53310 POINT(28.87616757301548 -95.78497733754256) bank53310 +53311 POINT(28.905861553990093 -95.46369949746898) bank53311 +53312 POINT(30.719548317020955 -95.29015721940557) bank53312 +53313 POINT(29.459126876119495 -95.01506497685641) bank53313 +53314 POINT(29.55572041019031 -94.88574419274762) bank53314 +53315 POINT(29.134845944967992 -94.41338673410861) bank53315 +53316 POINT(30.613490291381684 -94.86682537071567) bank53316 +53317 POINT(29.842063299788368 -96.32312468557089) bank53317 +53318 POINT(28.899027411176817 -95.92499068859526) bank53318 +53319 POINT(30.441020591496596 -94.51445283072891) bank53319 +53320 POINT(30.679724860491614 -96.08655096420978) bank53320 +53321 POINT(30.55968188005293 -95.68523581289685) bank53321 +53322 POINT(28.96891563829949 -95.51306221602906) bank53322 +53323 POINT(29.168302922727346 -95.43586681978525) bank53323 +53324 POINT(30.728240404851277 -95.84403812071078) bank53324 +53325 POINT(29.804306952821925 -94.82079751682471) bank53325 +53326 POINT(28.832274796190852 -96.15627319081065) bank53326 +53327 POINT(29.260425392734135 -95.47640741559681) bank53327 +53328 POINT(29.44595245247507 -95.91361036203817) bank53328 +53329 POINT(29.75356253950155 -96.34409419369433) bank53329 +53330 POINT(30.107435776489382 -94.87463759108958) bank53330 +53331 POINT(29.11959713130221 -95.54954275506022) bank53331 +53332 POINT(29.159672041293803 -95.77664881059563) bank53332 +53333 POINT(29.112213041592955 -95.83804549307914) bank53333 +53334 POINT(28.78465188566664 -95.14773682380684) bank53334 +53335 POINT(30.113996425984393 -95.12341038581002) bank53335 +53336 POINT(30.747558296554686 -95.82640356698803) bank53336 +53337 POINT(30.248730316594326 -95.5945995955576) bank53337 +53338 POINT(30.10909992936271 -94.99717872865887) bank53338 +53339 POINT(28.920004076326208 -95.56576769411944) bank53339 +53340 POINT(29.79811592933075 -95.74024237292157) bank53340 +53341 POINT(30.56165826308935 -95.40897314651806) bank53341 +53342 POINT(29.83549245214 -95.8761585746906) bank53342 +53343 POINT(29.642379433599977 -96.28154028814613) bank53343 +53344 POINT(30.235644590379778 -95.73027345802171) bank53344 +53345 POINT(29.001878097399064 -94.57165349253015) bank53345 +53346 POINT(30.25395901112403 -96.25710961630881) bank53346 +53347 POINT(30.308561173983392 -95.53534971380846) bank53347 +53348 POINT(29.35304320881135 -95.46274259631868) bank53348 +53349 POINT(30.37077922031144 -95.47484116883102) bank53349 +53350 POINT(29.050315266105926 -95.34149856991431) bank53350 +53351 POINT(30.095009531036123 -95.3812884666141) bank53351 +53352 POINT(30.333294115842744 -96.13610096898226) bank53352 +53353 POINT(30.05021404920685 -95.81492822580725) bank53353 +53354 POINT(29.439302976194348 -95.5259213985311) bank53354 +53355 POINT(29.40674772565468 -94.63475736574784) bank53355 +53356 POINT(29.598531984643973 -95.90763007436325) bank53356 +53357 POINT(30.714490474056735 -95.53936991399935) bank53357 +53358 POINT(28.876020105217982 -95.53007372606228) bank53358 +53359 POINT(30.682440129729358 -95.40867129737346) bank53359 +53360 POINT(30.00411645089119 -96.30511180091295) bank53360 +53361 POINT(28.93154014478031 -95.88974358447277) bank53361 +53362 POINT(29.966022122049544 -94.78083959941759) bank53362 +53363 POINT(30.60322232643632 -95.76982184086457) bank53363 +53364 POINT(30.079976108574733 -94.87709675088098) bank53364 +53365 POINT(29.22516443787312 -95.43616810068542) bank53365 +53366 POINT(28.810632806219736 -96.28672252539816) bank53366 +53367 POINT(29.53592341055125 -95.67033223078039) bank53367 +53368 POINT(29.3345091561041 -96.1642321037252) bank53368 +53369 POINT(28.954613400373308 -95.83623553911568) bank53369 +53370 POINT(29.123556890263796 -95.03358374569345) bank53370 +53371 POINT(29.076065325062228 -95.13725755399251) bank53371 +53372 POINT(29.689387396394977 -95.30810504312176) bank53372 +53373 POINT(30.418938014151404 -95.59211518563085) bank53373 +53374 POINT(29.063111106370147 -94.42263134636539) bank53374 +53375 POINT(29.796959950248294 -96.00220483246379) bank53375 +53376 POINT(29.80589512323238 -94.46634178729022) bank53376 +53377 POINT(29.823729422544556 -95.40084603384977) bank53377 +53378 POINT(29.861968602828515 -95.66362184749495) bank53378 +53379 POINT(29.81987125532588 -95.56272979560237) bank53379 +53380 POINT(30.491434497801293 -96.25950328642031) bank53380 +53381 POINT(29.432270533658976 -96.16743106595408) bank53381 +53382 POINT(30.251097921544662 -94.69652295082936) bank53382 +53383 POINT(28.865212138129632 -95.02344943126569) bank53383 +53384 POINT(29.60898102224654 -94.98402666937874) bank53384 +53385 POINT(29.795158912460753 -96.18846722556003) bank53385 +53386 POINT(30.466946768981884 -94.5044001049751) bank53386 +53387 POINT(29.383475067287126 -95.44935896806763) bank53387 +53388 POINT(30.632420121476837 -94.49143352932242) bank53388 +53389 POINT(29.3988872824965 -95.2123884075758) bank53389 +53390 POINT(29.251585508770123 -95.25322754754632) bank53390 +53391 POINT(30.604332527738922 -94.56856266060707) bank53391 +53392 POINT(29.49421223757789 -95.27693791647938) bank53392 +53393 POINT(29.461159773147916 -94.77458754474992) bank53393 +53394 POINT(29.541933920626207 -95.58941707838218) bank53394 +53395 POINT(29.801465354322108 -95.23664850855829) bank53395 +53396 POINT(30.759950242933147 -95.82855338446296) bank53396 +53397 POINT(29.13548731933689 -96.23935523046477) bank53397 +53398 POINT(29.872688863008033 -95.978859900133) bank53398 +53399 POINT(30.68141031386705 -96.0637606885777) bank53399 +53400 POINT(30.204182922035454 -96.04985195565325) bank53400 +53401 POINT(28.859670552644026 -95.98075914888759) bank53401 +53402 POINT(28.803261571938926 -95.31337611921235) bank53402 +53403 POINT(29.83006648425567 -94.80651029315992) bank53403 +53404 POINT(29.090725480409187 -95.48564975687303) bank53404 +53405 POINT(29.908246810632765 -95.16782201297411) bank53405 +53406 POINT(29.444047823504505 -95.94726419023289) bank53406 +53407 POINT(29.481268559583395 -94.5406943209655) bank53407 +53408 POINT(28.90003871460829 -94.40344577158803) bank53408 +53409 POINT(29.50074059831942 -95.6282378180521) bank53409 +53410 POINT(29.67090486207595 -95.44720913589937) bank53410 +53411 POINT(29.828631842032564 -96.20321751220486) bank53411 +53412 POINT(28.845260133468287 -95.03649201785475) bank53412 +53413 POINT(30.133399323344456 -94.84621630415205) bank53413 +53414 POINT(29.448488509138734 -95.17318580394439) bank53414 +53415 POINT(29.57515063534001 -95.09043907210119) bank53415 +53416 POINT(28.822050051684947 -94.59520937274958) bank53416 +53417 POINT(29.661293124797922 -95.05400286731715) bank53417 +53418 POINT(29.021932629985276 -95.9338887216316) bank53418 +53419 POINT(29.166859775856476 -94.47620122152647) bank53419 +53420 POINT(29.676468169444256 -95.2013499333392) bank53420 +53421 POINT(29.102565307478113 -96.234300362622) bank53421 +53422 POINT(29.604836640710133 -95.35168627808888) bank53422 +53423 POINT(30.25287518492229 -95.36433529959571) bank53423 +53424 POINT(29.931825100794285 -96.1645231370414) bank53424 +53425 POINT(30.531873964364138 -94.98915564782628) bank53425 +53426 POINT(29.38097583392331 -94.47825828026545) bank53426 +53427 POINT(30.52106125973591 -95.56190912673942) bank53427 +53428 POINT(29.834906970505013 -94.42362313167254) bank53428 +53429 POINT(30.076603646372693 -96.13796319598877) bank53429 +53430 POINT(29.595503483783038 -94.89794893334056) bank53430 +53431 POINT(30.169247844661882 -95.15315769259094) bank53431 +53432 POINT(29.84207651054619 -96.14772257572355) bank53432 +53433 POINT(30.313732411214126 -96.1822037378036) bank53433 +53434 POINT(30.351051756813337 -96.19503944567552) bank53434 +53435 POINT(30.272925881916052 -95.64068069824856) bank53435 +53436 POINT(28.781391921559973 -95.1049702369014) bank53436 +53437 POINT(30.32786620559726 -95.17948436712103) bank53437 +53438 POINT(29.92451315147845 -95.35448480894689) bank53438 +53439 POINT(29.841773618546405 -94.45132459224487) bank53439 +53440 POINT(29.80793265833378 -96.10303632166992) bank53440 +53441 POINT(29.707793325140834 -96.00902516244913) bank53441 +53442 POINT(30.28858704607369 -95.8103742490703) bank53442 +53443 POINT(29.92291038694035 -95.29085722487518) bank53443 +53444 POINT(30.337329289603815 -95.03722731767287) bank53444 +53445 POINT(30.148388170940386 -94.78599342812987) bank53445 +53446 POINT(30.04503740783623 -96.33527714780485) bank53446 +53447 POINT(29.438134978673684 -95.20645766280244) bank53447 +53448 POINT(30.558379253125995 -95.10100338822785) bank53448 +53449 POINT(29.8439464213218 -95.80290291810113) bank53449 +53450 POINT(30.29470574417371 -94.72792355676567) bank53450 +53451 POINT(30.508612569854975 -94.46495426516637) bank53451 +53452 POINT(29.410599145290107 -94.87927633500287) bank53452 +53453 POINT(29.641204192823427 -94.91411013408525) bank53453 +53454 POINT(29.02202756426681 -96.03325837704799) bank53454 +53455 POINT(30.56001721079326 -95.35653815509647) bank53455 +53456 POINT(29.727363052348398 -94.79703282625574) bank53456 +53457 POINT(30.739199418266193 -95.81890743776506) bank53457 +53458 POINT(30.09595125679533 -95.82010329123574) bank53458 +53459 POINT(29.195320869472216 -94.44266220687815) bank53459 +53460 POINT(30.544702021105852 -94.94096571948505) bank53460 +53461 POINT(29.537181079487958 -94.47628595233135) bank53461 +53462 POINT(29.69690914964981 -96.24222565056466) bank53462 +53463 POINT(29.65093627185363 -96.2410582296591) bank53463 +53464 POINT(30.582040459923466 -95.93723203979323) bank53464 +53465 POINT(30.529359776934864 -95.45863640205165) bank53465 +53466 POINT(30.402273338943697 -94.52737652827284) bank53466 +53467 POINT(29.100352487961782 -95.39257717659672) bank53467 +53468 POINT(30.11741832577352 -95.96291150965203) bank53468 +53469 POINT(29.92713037337756 -95.18452804400117) bank53469 +53470 POINT(29.182787676418624 -94.57937503383782) bank53470 +53471 POINT(30.573687878666842 -95.8720777503018) bank53471 +53472 POINT(30.587753270915282 -95.24152824539615) bank53472 +53473 POINT(30.317674552424094 -95.92578900743979) bank53473 +53474 POINT(29.61157461158938 -94.97785728557177) bank53474 +53475 POINT(29.96024025441015 -96.15534816599826) bank53475 +53476 POINT(28.963271506082464 -95.00157321568044) bank53476 +53477 POINT(28.852061505555366 -94.52477995184275) bank53477 +53478 POINT(30.166173574835344 -95.949848645662) bank53478 +53479 POINT(29.59436658039121 -95.09052893058112) bank53479 +53480 POINT(30.36206406878742 -96.16821351545465) bank53480 +53481 POINT(29.247133690374454 -94.61969345364685) bank53481 +53482 POINT(30.258919758833148 -95.88568672576463) bank53482 +53483 POINT(29.05828517229833 -94.43529233877993) bank53483 +53484 POINT(30.130030450409848 -95.87172309496867) bank53484 +53485 POINT(30.748288486717605 -94.64675290133384) bank53485 +53486 POINT(30.48680523936996 -96.12228932016636) bank53486 +53487 POINT(29.008329578668473 -95.07293039130799) bank53487 +53488 POINT(29.70105801117064 -95.86781121341883) bank53488 +53489 POINT(29.953812760002048 -94.41941099272404) bank53489 +53490 POINT(30.608177594677354 -95.94675255204639) bank53490 +53491 POINT(29.02969997096539 -96.23252695881914) bank53491 +53492 POINT(30.170981892673204 -96.1332755458553) bank53492 +53493 POINT(28.99784585230838 -96.25473905358263) bank53493 +53494 POINT(29.553144770663717 -96.35475130309038) bank53494 +53495 POINT(29.249183127289637 -95.43219764155819) bank53495 +53496 POINT(29.008244848277702 -95.19273474419262) bank53496 +53497 POINT(28.945131808189522 -96.08785333236193) bank53497 +53498 POINT(29.87984982097643 -95.42767173021865) bank53498 +53499 POINT(28.94986417120295 -94.98782553088907) bank53499 +53500 POINT(28.88677481215109 -94.55927227941486) bank53500 +53501 POINT(30.435372889189765 -94.67459362461616) bank53501 +53502 POINT(29.90044158284418 -95.48946169870999) bank53502 +53503 POINT(29.742884175456812 -96.12175039019883) bank53503 +53504 POINT(29.047591036304183 -94.66650567251992) bank53504 +53505 POINT(29.063668858347075 -96.18067260860826) bank53505 +53506 POINT(30.189579170009814 -95.20063509306635) bank53506 +53507 POINT(30.58989998443742 -96.31558541954679) bank53507 +53508 POINT(30.379592203446304 -94.37257685928095) bank53508 +53509 POINT(29.681179084866713 -95.01670345433504) bank53509 +53510 POINT(28.85983997869748 -95.46956351883885) bank53510 +53511 POINT(29.96728737992432 -94.67461165908848) bank53511 +53512 POINT(28.863482882927848 -95.09487037533872) bank53512 +53513 POINT(29.74388301364773 -94.72140615122986) bank53513 +53514 POINT(29.786439966889677 -95.3769202021325) bank53514 +53515 POINT(29.47930579589313 -95.34019294295963) bank53515 +53516 POINT(29.949915062731243 -96.22344778335358) bank53516 +53517 POINT(29.28651814704713 -95.06817136707454) bank53517 +53518 POINT(29.904359314659626 -95.22105941784814) bank53518 +53519 POINT(30.722508985506693 -95.3322131134769) bank53519 +53520 POINT(30.273978818325048 -95.29211270238655) bank53520 +53521 POINT(30.609753792419763 -96.09282186984339) bank53521 +53522 POINT(29.394309236542234 -95.31505505209309) bank53522 +53523 POINT(28.93918665952132 -95.66211112279771) bank53523 +53524 POINT(29.822900838400468 -94.6488748986493) bank53524 +53525 POINT(30.48079634476032 -95.61611661762825) bank53525 +53526 POINT(29.45649263036083 -94.76843574250506) bank53526 +53527 POINT(28.95816300472848 -95.80320896721464) bank53527 +53528 POINT(28.765899221582995 -95.02128182526404) bank53528 +53529 POINT(30.17300084374542 -94.38347965624355) bank53529 +53530 POINT(30.425381390976163 -95.19526052684363) bank53530 +53531 POINT(28.984549655025145 -95.29473937706346) bank53531 +53532 POINT(30.00111603162844 -94.72006412691178) bank53532 +53533 POINT(29.63130279337459 -94.64604380172538) bank53533 +53534 POINT(29.097814934538597 -95.3220929704449) bank53534 +53535 POINT(30.65116186494932 -95.85659961078596) bank53535 +53536 POINT(29.505198512954728 -96.21199664143867) bank53536 +53537 POINT(29.09143589791305 -95.76263652126924) bank53537 +53538 POINT(30.49056489580993 -94.38722588210445) bank53538 +53539 POINT(30.10725413531403 -95.08018152026521) bank53539 +53540 POINT(29.85453884720698 -94.45862937463973) bank53540 +53541 POINT(30.113722783860933 -95.99170985830865) bank53541 +53542 POINT(29.10973659171158 -94.5485274402284) bank53542 +53543 POINT(30.486550755291 -94.43913215896913) bank53543 +53544 POINT(29.004351120840145 -95.7934963099893) bank53544 +53545 POINT(28.841971886657745 -95.02567221413587) bank53545 +53546 POINT(30.06494548347512 -95.24227247528552) bank53546 +53547 POINT(30.657800741978146 -94.8756615444472) bank53547 +53548 POINT(30.183018780476306 -95.84631543710789) bank53548 +53549 POINT(29.208719907582022 -96.08626641695038) bank53549 +53550 POINT(30.470460488508888 -96.30484494800355) bank53550 +53551 POINT(30.49390102063499 -95.41116565222443) bank53551 +53552 POINT(28.869216395513348 -96.22432644306862) bank53552 +53553 POINT(29.73030903881422 -95.27585263476135) bank53553 +53554 POINT(30.36789331305815 -95.08591531077208) bank53554 +53555 POINT(30.574733610403825 -94.98481291547157) bank53555 +53556 POINT(30.62622134804768 -95.63907303760693) bank53556 +53557 POINT(29.851055106621114 -95.83204162994369) bank53557 +53558 POINT(30.04233398654479 -95.945687765934) bank53558 +53559 POINT(29.404840079702186 -95.49971287984387) bank53559 +53560 POINT(28.7895984308614 -95.22293468848338) bank53560 +53561 POINT(29.732424982969146 -94.39597307621827) bank53561 +53562 POINT(30.05106786281435 -94.9634911522037) bank53562 +53563 POINT(30.37504138898763 -95.55905090473678) bank53563 +53564 POINT(28.914132356961638 -95.9899265344964) bank53564 +53565 POINT(30.0932758405731 -94.92641648150706) bank53565 +53566 POINT(28.83524439972112 -95.41872983790604) bank53566 +53567 POINT(29.435374010901093 -95.54013082066041) bank53567 +53568 POINT(29.059953591969954 -95.4749946742484) bank53568 +53569 POINT(30.724715914359248 -95.57699678168613) bank53569 +53570 POINT(29.159604805060116 -95.28103122189835) bank53570 +53571 POINT(29.069647929401235 -96.02409591719078) bank53571 +53572 POINT(30.199522897879678 -95.12035391925973) bank53572 +53573 POINT(28.981707327480105 -94.69929017365264) bank53573 +53574 POINT(28.807402649461025 -95.74651847218233) bank53574 +53575 POINT(29.211174379330846 -94.86889953139492) bank53575 +53576 POINT(28.849814610664705 -95.10509946733632) bank53576 +53577 POINT(29.33157293350709 -94.92568920641173) bank53577 +53578 POINT(30.19626727293975 -95.97544512325176) bank53578 +53579 POINT(29.4713537394908 -95.57960613708048) bank53579 +53580 POINT(29.39613080232494 -95.67812690709366) bank53580 +53581 POINT(30.00529152963911 -95.30226885801035) bank53581 +53582 POINT(29.930698539211118 -96.0435716800734) bank53582 +53583 POINT(30.679666757189015 -95.57188735054605) bank53583 +53584 POINT(30.62520782903668 -95.11231752604286) bank53584 +53585 POINT(28.988078870941823 -94.52368100684093) bank53585 +53586 POINT(29.923262668401744 -95.49803141450761) bank53586 +53587 POINT(28.794445207957523 -94.47595693084396) bank53587 +53588 POINT(29.48231875469724 -95.54021345721746) bank53588 +53589 POINT(29.488017105574485 -95.93964709160525) bank53589 +53590 POINT(29.51900574794379 -95.31623009813256) bank53590 +53591 POINT(29.382537516811073 -96.34296281750296) bank53591 +53592 POINT(28.94963246125359 -95.17294206997907) bank53592 +53593 POINT(28.824826372780095 -95.27370407535305) bank53593 +53594 POINT(29.687898822140898 -95.68737338892583) bank53594 +53595 POINT(30.383077361588594 -95.70138489760018) bank53595 +53596 POINT(30.63279993068896 -94.45302414422832) bank53596 +53597 POINT(30.333495752465954 -94.52572214209034) bank53597 +53598 POINT(29.807510957170354 -95.37267403048918) bank53598 +53599 POINT(29.909646046331112 -95.06712643363447) bank53599 +53600 POINT(29.59437661846615 -94.3904761236811) bank53600 +53601 POINT(30.450765328534256 -95.58854485588039) bank53601 +53602 POINT(30.319693325893414 -95.20676495666608) bank53602 +53603 POINT(30.624046028683114 -96.11458825457363) bank53603 +53604 POINT(29.860409414267938 -95.20949183856735) bank53604 +53605 POINT(29.68185468235117 -95.46488539340353) bank53605 +53606 POINT(30.680643059318854 -95.58565009166391) bank53606 +53607 POINT(29.12755101705262 -94.84680525278905) bank53607 +53608 POINT(30.540233063080244 -96.34078275823323) bank53608 +53609 POINT(30.159127395422452 -94.69878640742701) bank53609 +53610 POINT(29.472848627787126 -96.02918670453755) bank53610 +53611 POINT(29.643047274654602 -95.84092550612434) bank53611 +53612 POINT(29.594745285617904 -96.20518744021825) bank53612 +53613 POINT(30.110351285718966 -95.87482073053457) bank53613 +53614 POINT(29.330913823862296 -94.87103549690299) bank53614 +53615 POINT(29.670179803441165 -95.74822963087284) bank53615 +53616 POINT(29.99317827774208 -95.8418923418798) bank53616 +53617 POINT(29.570565046771712 -94.60952458861908) bank53617 +53618 POINT(29.355957951693792 -94.89985845550036) bank53618 +53619 POINT(29.764664728301778 -95.47841837439482) bank53619 +53620 POINT(29.676763069034855 -94.66956759496611) bank53620 +53621 POINT(29.662937467459376 -95.67502538763887) bank53621 +53622 POINT(29.05409766625707 -96.21387706148171) bank53622 +53623 POINT(29.549370008071055 -94.77406455700316) bank53623 +53624 POINT(30.12404367182419 -95.937120601008) bank53624 +53625 POINT(29.149917779170217 -96.26608414910898) bank53625 +53626 POINT(28.907269761512516 -95.09909017285183) bank53626 +53627 POINT(28.795651250104214 -94.503656024272) bank53627 +53628 POINT(30.341067005730206 -96.1703526787318) bank53628 +53629 POINT(30.472067560306144 -94.49632652268703) bank53629 +53630 POINT(30.69963042553129 -94.5532462259475) bank53630 +53631 POINT(29.28501098930873 -95.76093072421884) bank53631 +53632 POINT(29.98149247938464 -94.82279284696014) bank53632 +53633 POINT(30.705242767144203 -95.787357896939) bank53633 +53634 POINT(29.152845095844796 -94.53949461185977) bank53634 +53635 POINT(30.040032522999468 -96.15001338529254) bank53635 +53636 POINT(30.15318044866398 -95.83169756708457) bank53636 +53637 POINT(29.533979004917253 -95.09465177435789) bank53637 +53638 POINT(29.475603069023848 -94.97034497878735) bank53638 +53639 POINT(30.592217026311108 -95.16110130774383) bank53639 +53640 POINT(30.3226511462369 -94.42241734671455) bank53640 +53641 POINT(28.87672897503818 -95.93416547659986) bank53641 +53642 POINT(30.200307297183276 -94.8005605859226) bank53642 +53643 POINT(29.772054428815828 -95.57554363537065) bank53643 +53644 POINT(30.461895603995778 -95.83198184770137) bank53644 +53645 POINT(30.7289801021562 -95.34083267339334) bank53645 +53646 POINT(30.14224813581089 -95.94892995915541) bank53646 +53647 POINT(29.254794265037322 -94.60656017985116) bank53647 +53648 POINT(30.6392620021846 -96.05966010920497) bank53648 +53649 POINT(28.785102398401452 -94.41622723026481) bank53649 +53650 POINT(28.94432892396245 -95.59243212919054) bank53650 +53651 POINT(29.536544268859853 -95.85384350959262) bank53651 +53652 POINT(29.99819059193937 -94.67462312137837) bank53652 +53653 POINT(30.425634004610465 -95.81850057607949) bank53653 +53654 POINT(30.607126373451102 -96.16467134685963) bank53654 +53655 POINT(30.665794267167957 -95.28092364853686) bank53655 +53656 POINT(28.825665527116822 -94.65990186289297) bank53656 +53657 POINT(29.27869707559781 -95.93553578555955) bank53657 +53658 POINT(29.042243976802144 -95.9351542381163) bank53658 +53659 POINT(29.4894989855781 -94.71279539582056) bank53659 +53660 POINT(29.40893299986792 -94.79737716333929) bank53660 +53661 POINT(30.16365656557451 -94.86543907326404) bank53661 +53662 POINT(29.805717041631517 -95.15500270610357) bank53662 +53663 POINT(29.770777535486218 -95.97746378864915) bank53663 +53664 POINT(29.394490214421893 -94.7752660987263) bank53664 +53665 POINT(29.595044219870267 -95.44899680457951) bank53665 +53666 POINT(30.597233142390667 -96.13049264373002) bank53666 +53667 POINT(29.742124728719162 -95.70277425204401) bank53667 +53668 POINT(30.310473399705003 -94.6054889073109) bank53668 +53669 POINT(29.006786454373394 -95.1079104398057) bank53669 +53670 POINT(30.3560657694824 -94.62440576784427) bank53670 +53671 POINT(30.598022962338856 -96.11148320096669) bank53671 +53672 POINT(29.841045616356382 -95.93540598724049) bank53672 +53673 POINT(30.35253129523305 -95.5936699498337) bank53673 +53674 POINT(29.04080582437021 -95.92178961650588) bank53674 +53675 POINT(29.724592014440876 -95.7387322348715) bank53675 +53676 POINT(30.12376027261971 -95.98404305853661) bank53676 +53677 POINT(29.65660821408311 -94.97799509924418) bank53677 +53678 POINT(30.371014370005458 -95.49566799183393) bank53678 +53679 POINT(29.57615445495023 -94.56505334678691) bank53679 +53680 POINT(29.243749026116888 -95.10996563020872) bank53680 +53681 POINT(30.03210098678958 -96.09697586293636) bank53681 +53682 POINT(30.088578706944176 -95.57866242219728) bank53682 +53683 POINT(28.848600341544802 -94.63289651778496) bank53683 +53684 POINT(28.912226458595075 -94.73420392961707) bank53684 +53685 POINT(30.741613912555394 -95.18829818049397) bank53685 +53686 POINT(30.522162663551118 -94.84308930699468) bank53686 +53687 POINT(30.725025560804042 -94.53696088313474) bank53687 +53688 POINT(28.939486100572108 -94.66378202615637) bank53688 +53689 POINT(29.43823576400294 -95.6851013712876) bank53689 +53690 POINT(28.92651613955007 -95.59222442218628) bank53690 +53691 POINT(30.36609781225266 -94.70261327440402) bank53691 +53692 POINT(30.40758909536373 -95.39083690116968) bank53692 +53693 POINT(29.699703107722726 -95.7648128574969) bank53693 +53694 POINT(29.37005792325183 -94.64148103445072) bank53694 +53695 POINT(30.444065367969145 -95.75648138618762) bank53695 +53696 POINT(30.243385282589237 -96.29486886963255) bank53696 +53697 POINT(29.38932782535638 -94.41789223536733) bank53697 +53698 POINT(30.547630995842212 -94.63204198068458) bank53698 +53699 POINT(28.84237084120323 -96.22120955937471) bank53699 +53700 POINT(29.258708654418303 -95.20792340830171) bank53700 +53701 POINT(28.985675364172852 -95.56958760543478) bank53701 +53702 POINT(29.14122407711583 -94.93794502034764) bank53702 +53703 POINT(30.150934150992825 -94.58079211449102) bank53703 +53704 POINT(29.449166471853776 -94.87340721669229) bank53704 +53705 POINT(29.86280760178766 -94.43542813995751) bank53705 +53706 POINT(30.26507802618582 -95.24693399473885) bank53706 +53707 POINT(30.38352595457044 -95.35499902549124) bank53707 +53708 POINT(29.864116901299397 -95.94704130225534) bank53708 +53709 POINT(29.701337975014255 -95.24308226836581) bank53709 +53710 POINT(29.4700971039953 -95.76315772011527) bank53710 +53711 POINT(30.013776845785777 -94.76690213764451) bank53711 +53712 POINT(30.03886492317414 -94.82304635688368) bank53712 +53713 POINT(30.70629594962254 -95.33943233224814) bank53713 +53714 POINT(29.85104706485508 -96.15600306540108) bank53714 +53715 POINT(30.33753866349058 -95.85978905610153) bank53715 +53716 POINT(29.274302679532997 -94.92891800805027) bank53716 +53717 POINT(29.994771282438442 -95.17795388836981) bank53717 +53718 POINT(29.78762419237264 -94.5576020380697) bank53718 +53719 POINT(29.856744731330796 -96.36054204110276) bank53719 +53720 POINT(29.324365557965823 -94.71409204929557) bank53720 +53721 POINT(30.75163682004257 -95.4053023581248) bank53721 +53722 POINT(30.400090661180467 -95.18898107225506) bank53722 +53723 POINT(30.372536655447497 -96.15807744824299) bank53723 +53724 POINT(30.16141553859126 -95.87202443874719) bank53724 +53725 POINT(29.870771473902312 -94.55209091906163) bank53725 +53726 POINT(30.361337768608635 -94.94390862607383) bank53726 +53727 POINT(30.404705098748256 -95.68668402505533) bank53727 +53728 POINT(29.923360308775948 -95.15807570125371) bank53728 +53729 POINT(29.10886649721867 -95.3925925172018) bank53729 +53730 POINT(29.159074345309598 -96.2996010004619) bank53730 +53731 POINT(29.012561557108555 -95.69809855122793) bank53731 +53732 POINT(30.64597361782871 -95.30032882620253) bank53732 +53733 POINT(29.611315899103772 -95.63105667061839) bank53733 +53734 POINT(30.454829742677056 -95.70225061330736) bank53734 +53735 POINT(29.970734157994904 -95.29424238285213) bank53735 +53736 POINT(28.894155878093933 -95.4041550375282) bank53736 +53737 POINT(29.027329953548037 -95.16932504119022) bank53737 +53738 POINT(30.507872075173925 -96.21832600851766) bank53738 +53739 POINT(29.609573276329947 -95.5636631351486) bank53739 +53740 POINT(30.697856483565523 -95.12824721985318) bank53740 +53741 POINT(29.486720009486092 -96.05124616661763) bank53741 +53742 POINT(30.31610666839752 -95.2990686046132) bank53742 +53743 POINT(29.936962040559333 -94.8961772511287) bank53743 +53744 POINT(30.33799327263952 -95.30146511123688) bank53744 +53745 POINT(28.938063947235957 -96.28842097096177) bank53745 +53746 POINT(29.95149293862198 -96.01376952872123) bank53746 +53747 POINT(30.07533930359653 -95.99006124872405) bank53747 +53748 POINT(30.2201941385653 -96.21515531171578) bank53748 +53749 POINT(30.245204443277387 -95.3597946696347) bank53749 +53750 POINT(30.51941268481496 -95.03426455617208) bank53750 +53751 POINT(29.82965075330066 -94.89390741354087) bank53751 +53752 POINT(29.17053918364508 -95.30020574053646) bank53752 +53753 POINT(30.631929464755064 -96.1741877451665) bank53753 +53754 POINT(29.937084715416844 -95.61114720400542) bank53754 +53755 POINT(30.585499739328878 -94.56536401676937) bank53755 +53756 POINT(29.624156958374247 -94.57934699556735) bank53756 +53757 POINT(30.19631585106304 -95.62981712494465) bank53757 +53758 POINT(30.37102936307608 -94.66216940435115) bank53758 +53759 POINT(30.2302097459598 -96.12229894263392) bank53759 +53760 POINT(29.730696027740205 -94.79632515215624) bank53760 +53761 POINT(30.12383505934716 -95.94040455613136) bank53761 +53762 POINT(28.816024634824984 -95.51735565492599) bank53762 +53763 POINT(29.505246593985348 -95.11944913761961) bank53763 +53764 POINT(30.247182763001717 -95.91907168956068) bank53764 +53765 POINT(29.157778947898933 -95.29657487477004) bank53765 +53766 POINT(29.1711320124186 -94.85634731817723) bank53766 +53767 POINT(30.26817331213916 -95.3284008221095) bank53767 +53768 POINT(30.159348794997225 -95.6811781698745) bank53768 +53769 POINT(29.52555355017832 -95.5170668767372) bank53769 +53770 POINT(29.808271798170757 -94.53685664530336) bank53770 +53771 POINT(30.54696893199057 -94.91109689863741) bank53771 +53772 POINT(28.826231499781557 -94.52460626672539) bank53772 +53773 POINT(28.928769302827032 -95.17890590409009) bank53773 +53774 POINT(29.331910951897637 -95.21641963385257) bank53774 +53775 POINT(29.47113142067642 -94.7188590785334) bank53775 +53776 POINT(29.39213806640959 -95.84312866654287) bank53776 +53777 POINT(30.634023270521773 -94.61753338740479) bank53777 +53778 POINT(30.620126833236007 -94.69546036192513) bank53778 +53779 POINT(29.7418100177636 -96.1123045781518) bank53779 +53780 POINT(29.888669040214204 -94.60541033948522) bank53780 +53781 POINT(29.58515933869697 -95.5170382183581) bank53781 +53782 POINT(29.577629188228737 -94.60175434821144) bank53782 +53783 POINT(29.110313540697394 -95.86517421860223) bank53783 +53784 POINT(29.451315105512283 -95.8805880263029) bank53784 +53785 POINT(28.895505062717092 -95.41132069861737) bank53785 +53786 POINT(28.92266658440978 -95.06451149267302) bank53786 +53787 POINT(30.56101572755149 -95.93716294927894) bank53787 +53788 POINT(30.670465673970412 -95.46676630934321) bank53788 +53789 POINT(30.33586419761462 -95.08833548185486) bank53789 +53790 POINT(29.89478303338527 -94.38360011256407) bank53790 +53791 POINT(29.241855134138454 -94.96575306006369) bank53791 +53792 POINT(30.03380208127701 -95.02437685170719) bank53792 +53793 POINT(29.704752158375644 -96.08176756531165) bank53793 +53794 POINT(30.052937133280103 -94.49034358564262) bank53794 +53795 POINT(29.556177026481457 -94.63099832759534) bank53795 +53796 POINT(30.537564536373164 -94.94596924885315) bank53796 +53797 POINT(29.125621822940968 -94.7657179595683) bank53797 +53798 POINT(29.970480032049757 -94.53336814657105) bank53798 +53799 POINT(30.1583044771001 -95.92643190619368) bank53799 +53800 POINT(29.124594744505583 -95.96518739475368) bank53800 +53801 POINT(30.35932761522064 -95.33079506335389) bank53801 +53802 POINT(29.29707422278745 -95.36397126873854) bank53802 +53803 POINT(29.959505948094456 -95.33776401337464) bank53803 +53804 POINT(30.53412519537656 -96.21090789867688) bank53804 +53805 POINT(29.336235176193313 -94.98205954282267) bank53805 +53806 POINT(30.312923528014142 -94.80850824336413) bank53806 +53807 POINT(30.292201568685137 -96.29647369605793) bank53807 +53808 POINT(30.758416946043383 -95.18705012559337) bank53808 +53809 POINT(30.546697632731835 -96.18997920665696) bank53809 +53810 POINT(29.319573302717348 -95.69286740105085) bank53810 +53811 POINT(29.70883548802275 -94.93397401363491) bank53811 +53812 POINT(28.950468259404435 -94.79874495025109) bank53812 +53813 POINT(30.686221827198462 -94.75118504242603) bank53813 +53814 POINT(29.272456834761027 -95.17810757554815) bank53814 +53815 POINT(29.82670336236723 -95.5040545690124) bank53815 +53816 POINT(29.282116401196333 -94.76448597970516) bank53816 +53817 POINT(29.28666895152612 -94.47479521557761) bank53817 +53818 POINT(29.269127134640303 -94.87149643458685) bank53818 +53819 POINT(30.15943548782519 -96.35746138269951) bank53819 +53820 POINT(29.072899189534844 -95.20919321601095) bank53820 +53821 POINT(30.621939222222508 -94.89219865581859) bank53821 +53822 POINT(29.540714177259805 -94.77555292569964) bank53822 +53823 POINT(29.187966951570598 -95.89913876061757) bank53823 +53824 POINT(30.327604564669052 -95.19634477786192) bank53824 +53825 POINT(29.484470849792267 -94.94972629092956) bank53825 +53826 POINT(29.721286409282758 -95.93693992713825) bank53826 +53827 POINT(28.839201814926323 -95.14456444641064) bank53827 +53828 POINT(29.796784188343665 -94.85274457280799) bank53828 +53829 POINT(29.190902627735973 -95.83738139966368) bank53829 +53830 POINT(30.718697454888392 -95.12034359549695) bank53830 +53831 POINT(29.71285093074837 -95.90359455467785) bank53831 +53832 POINT(29.88615288181064 -95.12541790843358) bank53832 +53833 POINT(28.81978030002834 -94.54280134372112) bank53833 +53834 POINT(28.992637091407065 -95.23316315852007) bank53834 +53835 POINT(29.899139722917443 -96.09306480669926) bank53835 +53836 POINT(30.21345726152849 -95.27349296558836) bank53836 +53837 POINT(28.95803415650809 -94.9396238143399) bank53837 +53838 POINT(30.316126708999924 -96.16740655872725) bank53838 +53839 POINT(30.4283599511689 -96.2245901769956) bank53839 +53840 POINT(30.46486389528798 -94.59343244325522) bank53840 +53841 POINT(28.844191625982727 -95.47854043093335) bank53841 +53842 POINT(30.240390981598676 -95.3675408641068) bank53842 +53843 POINT(29.9534747323232 -94.83422065770962) bank53843 +53844 POINT(28.809457003451 -94.64396682952862) bank53844 +53845 POINT(29.689661932333472 -95.22518597472951) bank53845 +53846 POINT(30.097639244801105 -96.3168126688995) bank53846 +53847 POINT(30.025849386918587 -95.37616183705836) bank53847 +53848 POINT(30.042148171095924 -94.54283712780628) bank53848 +53849 POINT(29.33484141800648 -95.22641862303908) bank53849 +53850 POINT(30.165598602019873 -95.53441265029275) bank53850 +53851 POINT(29.02507396696638 -94.57902086657664) bank53851 +53852 POINT(29.998165863955506 -95.91034844593646) bank53852 +53853 POINT(29.54413363612364 -95.81313206890178) bank53853 +53854 POINT(29.26848251643899 -94.74044070682919) bank53854 +53855 POINT(29.675571755919517 -95.48381595530738) bank53855 +53856 POINT(30.27272700593315 -95.50121486678994) bank53856 +53857 POINT(29.688959501208483 -94.95147937725483) bank53857 +53858 POINT(29.905276531832534 -96.26744752306624) bank53858 +53859 POINT(30.69610509209099 -95.6282279877911) bank53859 +53860 POINT(29.657032182676822 -95.63940323240783) bank53860 +53861 POINT(30.67165534007053 -95.4331714502921) bank53861 +53862 POINT(28.969075393659207 -96.01913978281542) bank53862 +53863 POINT(30.184781997273774 -95.86162106898912) bank53863 +53864 POINT(29.36174256779658 -95.12615324614724) bank53864 +53865 POINT(29.445874815559755 -95.35641708080504) bank53865 +53866 POINT(28.839244502030944 -94.89257889703575) bank53866 +53867 POINT(29.004465020417044 -94.97215485702345) bank53867 +53868 POINT(28.861230951415006 -95.40410612653949) bank53868 +53869 POINT(29.234100776935822 -96.2645364516877) bank53869 +53870 POINT(30.19591968646261 -94.46120012011606) bank53870 +53871 POINT(29.873687364730714 -94.9106741541227) bank53871 +53872 POINT(29.18924346459163 -96.04490176058755) bank53872 +53873 POINT(30.51556442475047 -95.08582945977875) bank53873 +53874 POINT(30.317444326243383 -95.84391866491093) bank53874 +53875 POINT(30.757452531056252 -94.53373490760397) bank53875 +53876 POINT(30.749099451273473 -95.02076462527862) bank53876 +53877 POINT(30.344651936962176 -94.861099982475) bank53877 +53878 POINT(29.825263858357605 -94.8096016968357) bank53878 +53879 POINT(30.43225296827207 -95.10081694429567) bank53879 +53880 POINT(29.799489249115357 -94.45050947391285) bank53880 +53881 POINT(28.77984112364202 -95.916929450894) bank53881 +53882 POINT(29.07622340499976 -95.41285048626018) bank53882 +53883 POINT(30.00505626691416 -96.27276558099396) bank53883 +53884 POINT(30.33333178774706 -95.96836891380785) bank53884 +53885 POINT(30.26929049035728 -95.80964232143194) bank53885 +53886 POINT(29.135980586722855 -95.63124106879953) bank53886 +53887 POINT(29.492012904352364 -94.42041080997113) bank53887 +53888 POINT(30.06174198392936 -95.7705263670851) bank53888 +53889 POINT(29.059658596682194 -95.16593072162526) bank53889 +53890 POINT(28.831762723735004 -95.37801133280969) bank53890 +53891 POINT(29.71791079359862 -95.83093563701398) bank53891 +53892 POINT(29.262324197424952 -95.4340646257032) bank53892 +53893 POINT(29.205985577791512 -95.11954679871663) bank53893 +53894 POINT(30.0765836812335 -95.65678725866564) bank53894 +53895 POINT(29.159319449102075 -94.63411237420738) bank53895 +53896 POINT(30.355511321174337 -96.06122715616952) bank53896 +53897 POINT(29.000450469787637 -95.30502939711263) bank53897 +53898 POINT(30.31303834772897 -95.31099957135409) bank53898 +53899 POINT(30.081226954441604 -95.36372736847964) bank53899 +53900 POINT(30.26272853274624 -95.6409453687303) bank53900 +53901 POINT(29.48410854480968 -96.15218566461215) bank53901 +53902 POINT(29.48147243075671 -95.00247944321461) bank53902 +53903 POINT(29.324781148704627 -95.51980489865308) bank53903 +53904 POINT(30.667686885007278 -95.86134437991612) bank53904 +53905 POINT(29.68608448504065 -96.25379049890115) bank53905 +53906 POINT(29.150416163857923 -95.95794381155285) bank53906 +53907 POINT(29.261927979608515 -96.29795360337108) bank53907 +53908 POINT(28.907129544150408 -94.83390564075471) bank53908 +53909 POINT(29.514199486794993 -95.36811935461154) bank53909 +53910 POINT(29.03059688692422 -96.04665387187785) bank53910 +53911 POINT(29.118201785921013 -94.56382843903772) bank53911 +53912 POINT(29.625736854959804 -95.12392205013887) bank53912 +53913 POINT(29.263795232134477 -95.33630110030941) bank53913 +53914 POINT(29.409815470772045 -94.94560188939226) bank53914 +53915 POINT(30.244321377728408 -96.26810240234921) bank53915 +53916 POINT(28.925437715509116 -96.23125958447723) bank53916 +53917 POINT(30.44054895913889 -96.17738071412076) bank53917 +53918 POINT(29.189340562736433 -95.54531264665125) bank53918 +53919 POINT(28.851129526575487 -95.63359123527891) bank53919 +53920 POINT(28.942868979262904 -94.9199447546188) bank53920 +53921 POINT(30.244014391817032 -96.19465177014902) bank53921 +53922 POINT(29.355147325438555 -94.54120583578641) bank53922 +53923 POINT(29.17952372567554 -95.656374690124) bank53923 +53924 POINT(29.553718537800012 -95.62198185812197) bank53924 +53925 POINT(30.25778233457606 -95.5418780912859) bank53925 +53926 POINT(29.334100564502858 -96.365191119886) bank53926 +53927 POINT(29.299359291424707 -95.8888277289743) bank53927 +53928 POINT(28.9143187344266 -94.80713228205303) bank53928 +53929 POINT(30.66628245938798 -94.59141477382609) bank53929 +53930 POINT(30.242233184574143 -95.19039973792208) bank53930 +53931 POINT(29.05606557452876 -94.76767980473664) bank53931 +53932 POINT(30.407753865800668 -95.091103353801) bank53932 +53933 POINT(29.162302428024972 -95.46353210254901) bank53933 +53934 POINT(29.035813482543613 -94.68969184475668) bank53934 +53935 POINT(30.737008659808694 -95.00451206400344) bank53935 +53936 POINT(29.82326179125225 -95.93415450787175) bank53936 +53937 POINT(29.135844316272095 -96.35973966366973) bank53937 +53938 POINT(30.255185672534434 -95.03341364163026) bank53938 +53939 POINT(29.44898333084453 -95.47905783995988) bank53939 +53940 POINT(29.631369778689496 -94.38841080069085) bank53940 +53941 POINT(30.245995466109818 -96.24567134792117) bank53941 +53942 POINT(30.053322905137456 -95.05411501459203) bank53942 +53943 POINT(28.982898554100192 -95.73524144683256) bank53943 +53944 POINT(29.995018048003445 -94.99525373479894) bank53944 +53945 POINT(29.643809116158792 -96.05897183392287) bank53945 +53946 POINT(30.72100808751413 -96.21640551109479) bank53946 +53947 POINT(30.119459904640465 -94.73020107281317) bank53947 +53948 POINT(28.980396143191964 -95.37817762763166) bank53948 +53949 POINT(30.696808780293523 -95.35211805630617) bank53949 +53950 POINT(29.43124163355097 -95.8421470536415) bank53950 +53951 POINT(30.73341180627678 -95.60975622055666) bank53951 +53952 POINT(30.49274852903594 -95.67345217105823) bank53952 +53953 POINT(30.218277179518644 -95.4888177180523) bank53953 +53954 POINT(28.79992312083067 -94.70644889280194) bank53954 +53955 POINT(29.996589846247787 -96.13380451314491) bank53955 +53956 POINT(30.5279809365128 -95.84269258761154) bank53956 +53957 POINT(29.985004923321306 -95.47448571828562) bank53957 +53958 POINT(29.950440751115785 -96.2377880258697) bank53958 +53959 POINT(29.130714307902455 -95.36575310384472) bank53959 +53960 POINT(30.152882110979473 -94.66271012061452) bank53960 +53961 POINT(29.702629231060236 -95.61299365994242) bank53961 +53962 POINT(30.284542384160282 -94.92874246915441) bank53962 +53963 POINT(30.414503595181667 -95.2877413368283) bank53963 +53964 POINT(29.41243410285122 -94.74615138653013) bank53964 +53965 POINT(28.78654135828925 -95.75692213319009) bank53965 +53966 POINT(30.6012923569513 -95.58648139246581) bank53966 +53967 POINT(28.866530145016153 -94.50664530942458) bank53967 +53968 POINT(30.13643588843695 -95.32346488106798) bank53968 +53969 POINT(30.560188153363143 -94.5172783484052) bank53969 +53970 POINT(30.732823664568812 -94.67830420710195) bank53970 +53971 POINT(29.78130186336076 -95.41589214029257) bank53971 +53972 POINT(29.214141304594236 -96.13442717468752) bank53972 +53973 POINT(29.376027940912774 -94.92947629654122) bank53973 +53974 POINT(29.237716371086965 -94.99434068980047) bank53974 +53975 POINT(29.063917918504956 -94.83733960011666) bank53975 +53976 POINT(28.761277395054123 -94.6639278784786) bank53976 +53977 POINT(29.616219165141704 -95.19599653497971) bank53977 +53978 POINT(29.981444436828323 -95.23757156850078) bank53978 +53979 POINT(29.59478557690753 -96.35863106890754) bank53979 +53980 POINT(29.93222078942727 -94.53606632724659) bank53980 +53981 POINT(30.40957463576964 -94.52967115007685) bank53981 +53982 POINT(30.734900021813736 -95.35702651830834) bank53982 +53983 POINT(30.663392540108013 -95.39580658738174) bank53983 +53984 POINT(29.729237935436675 -94.72664944705117) bank53984 +53985 POINT(29.862539223757157 -96.06714609505713) bank53985 +53986 POINT(30.309805678749086 -96.06519972504023) bank53986 +53987 POINT(30.739900048052114 -94.64608205199848) bank53987 +53988 POINT(29.46819669709909 -94.41830558165114) bank53988 +53989 POINT(30.734631324487545 -96.21472014775212) bank53989 +53990 POINT(30.63200543194364 -95.69468246511391) bank53990 +53991 POINT(30.34587688926403 -96.2449723448861) bank53991 +53992 POINT(29.91964194831789 -95.64254590777492) bank53992 +53993 POINT(30.701033922188724 -94.65973801276084) bank53993 +53994 POINT(29.839513605659093 -94.90155413843623) bank53994 +53995 POINT(29.615704267926265 -95.82709384313762) bank53995 +53996 POINT(30.474119954545554 -95.46299706750135) bank53996 +53997 POINT(28.88083216268742 -95.33386287304975) bank53997 +53998 POINT(30.245552951739594 -95.78951222961052) bank53998 +53999 POINT(30.64624322973056 -94.83486741116118) bank53999 +54000 POINT(30.300006984307977 -96.07898537716977) bank54000 +54001 POINT(29.258858307191062 -94.48212518735286) bank54001 +54002 POINT(30.121116282097017 -95.82146396485338) bank54002 +54003 POINT(30.680374244358095 -96.0112978891798) bank54003 +54004 POINT(29.396029750431065 -94.58421560037424) bank54004 +54005 POINT(28.891778890242453 -94.87361001957251) bank54005 +54006 POINT(30.462410941542746 -96.1922506744108) bank54006 +54007 POINT(30.166789821179474 -94.4696191540292) bank54007 +54008 POINT(29.104379764201912 -95.23197891018818) bank54008 +54009 POINT(30.66962798866943 -96.19583332585182) bank54009 +54010 POINT(28.7854373452519 -95.23101223297954) bank54010 +54011 POINT(29.41056259342376 -94.61251604556436) bank54011 +54012 POINT(30.12426451154059 -95.89935488041593) bank54012 +54013 POINT(29.076191663446256 -95.04259253256566) bank54013 +54014 POINT(29.9683988359474 -96.21543818071746) bank54014 +54015 POINT(29.350573942353748 -96.19726683077438) bank54015 +54016 POINT(30.064376388490608 -95.81384424206806) bank54016 +54017 POINT(30.347088382224957 -95.44069173042374) bank54017 +54018 POINT(29.265027089725358 -95.40397354063306) bank54018 +54019 POINT(30.759055283029653 -95.85671726799185) bank54019 +54020 POINT(28.804773836310336 -95.66879937305751) bank54020 +54021 POINT(30.546004964213505 -94.60576380063394) bank54021 +54022 POINT(30.193946348235578 -94.63000043903047) bank54022 +54023 POINT(28.873104104526366 -95.55278562850748) bank54023 +54024 POINT(29.853974132211736 -95.03108256877766) bank54024 +54025 POINT(30.355775452419905 -95.87623708614807) bank54025 +54026 POINT(30.04088974142865 -96.31480827222775) bank54026 +54027 POINT(30.298833832147956 -95.95656465034587) bank54027 +54028 POINT(29.477305177423084 -95.51655168302877) bank54028 +54029 POINT(29.71204380542086 -95.82006495764529) bank54029 +54030 POINT(30.090366450287053 -95.24799640022592) bank54030 +54031 POINT(29.655463588929578 -95.37183579776251) bank54031 +54032 POINT(30.006798707010603 -95.24034422429571) bank54032 +54033 POINT(29.020681844069873 -95.834126534017) bank54033 +54034 POINT(28.84113326919123 -95.65096695513053) bank54034 +54035 POINT(28.97486782177001 -95.89066147523708) bank54035 +54036 POINT(29.883042762436347 -94.53326480219795) bank54036 +54037 POINT(30.193880350787836 -95.15523109288391) bank54037 +54038 POINT(30.35519093975387 -95.21195244689011) bank54038 +54039 POINT(29.42478513980203 -95.6873218950274) bank54039 +54040 POINT(30.233530521490064 -96.23400684403023) bank54040 +54041 POINT(30.210978020708154 -94.53881156829983) bank54041 +54042 POINT(30.40251709688246 -95.63866298688455) bank54042 +54043 POINT(29.52648622675884 -95.01254056291006) bank54043 +54044 POINT(29.60849826398767 -94.37990544515107) bank54044 +54045 POINT(29.159217042191706 -94.9489493480624) bank54045 +54046 POINT(29.111603324223363 -95.52185824183637) bank54046 +54047 POINT(29.558295174372464 -94.51736220198065) bank54047 +54048 POINT(30.565669521204246 -96.12493883255885) bank54048 +54049 POINT(30.72056740188012 -95.19055211972663) bank54049 +54050 POINT(29.846026606421724 -95.13411266964292) bank54050 +54051 POINT(29.717621425581374 -96.31047564705955) bank54051 +54052 POINT(30.37706543764744 -95.30768881735166) bank54052 +54053 POINT(30.123906539621206 -96.02223610077738) bank54053 +54054 POINT(29.409989152288812 -95.39417202542103) bank54054 +54055 POINT(30.676656393215104 -95.4299978487337) bank54055 +54056 POINT(28.921500196215124 -95.79986952846002) bank54056 +54057 POINT(29.67631669019872 -96.10758043160462) bank54057 +54058 POINT(30.03936786495084 -95.6814388100398) bank54058 +54059 POINT(29.515250302168347 -94.88319576407072) bank54059 +54060 POINT(30.26961814692309 -94.90458560124125) bank54060 +54061 POINT(28.803305104902016 -95.84494175624613) bank54061 +54062 POINT(29.027026619334517 -95.13509747920467) bank54062 +54063 POINT(30.007718970445467 -96.25594142950214) bank54063 +54064 POINT(28.882124370984148 -94.65901661912383) bank54064 +54065 POINT(29.47480587452899 -95.27833976677454) bank54065 +54066 POINT(29.997521005516834 -95.42606723653921) bank54066 +54067 POINT(28.771108827169282 -95.75408500448385) bank54067 +54068 POINT(29.18292877160426 -95.40000470738444) bank54068 +54069 POINT(29.436249529764364 -96.01643804097641) bank54069 +54070 POINT(30.56484605382768 -94.50517958361837) bank54070 +54071 POINT(30.161433856675455 -95.4484114429751) bank54071 +54072 POINT(29.288780542783712 -95.51050241012119) bank54072 +54073 POINT(30.671358457628198 -96.01492428352279) bank54073 +54074 POINT(30.693224112624975 -95.91885863355174) bank54074 +54075 POINT(29.316432781223966 -96.20654546669154) bank54075 +54076 POINT(28.923002292350443 -95.30270117943464) bank54076 +54077 POINT(29.640275453131466 -96.25471945992373) bank54077 +54078 POINT(29.161797263155293 -96.18563690237472) bank54078 +54079 POINT(28.82951410507777 -94.5538443466842) bank54079 +54080 POINT(30.624203627289898 -94.72926036921521) bank54080 +54081 POINT(29.034515158254397 -94.380715795846) bank54081 +54082 POINT(29.5474974318451 -95.32864262085246) bank54082 +54083 POINT(30.506946423556343 -95.32372678944262) bank54083 +54084 POINT(29.02257942117488 -96.31510478690112) bank54084 +54085 POINT(29.333411129044936 -95.38966143767065) bank54085 +54086 POINT(29.878784362269467 -96.33672921975165) bank54086 +54087 POINT(30.60412791904068 -95.45039655550886) bank54087 +54088 POINT(30.0525880029263 -96.07975921184483) bank54088 +54089 POINT(29.651820249399996 -94.64065626848432) bank54089 +54090 POINT(29.047634908376043 -95.59328431780655) bank54090 +54091 POINT(29.102844179476943 -95.82212836299794) bank54091 +54092 POINT(29.944754894824314 -96.36064916630383) bank54092 +54093 POINT(29.87990950293007 -94.3867981099475) bank54093 +54094 POINT(28.872801440631125 -94.60181745466961) bank54094 +54095 POINT(30.26505087303597 -94.48103346791164) bank54095 +54096 POINT(29.35362162613273 -95.9501078856977) bank54096 +54097 POINT(29.66797038376352 -94.56428064751769) bank54097 +54098 POINT(30.16517586910413 -95.17430091653995) bank54098 +54099 POINT(30.206384552287943 -96.10888213274276) bank54099 +54100 POINT(29.359677264278513 -94.72440857665504) bank54100 +54101 POINT(29.568074874003518 -94.69984075130205) bank54101 +54102 POINT(29.827513902602504 -96.01483729546443) bank54102 +54103 POINT(29.010149017938843 -95.54824076403618) bank54103 +54104 POINT(28.920280720154942 -95.39880301245731) bank54104 +54105 POINT(29.233166300008307 -95.64684386699089) bank54105 +54106 POINT(29.951875118233527 -95.43790499325041) bank54106 +54107 POINT(30.72052235637138 -95.39460228066055) bank54107 +54108 POINT(30.066267293966924 -94.75806670167765) bank54108 +54109 POINT(30.61516329092417 -94.938828522794) bank54109 +54110 POINT(29.331652841834195 -95.70038545720344) bank54110 +54111 POINT(29.31614580702117 -95.88615169626831) bank54111 +54112 POINT(30.303617648873185 -95.33140977994225) bank54112 +54113 POINT(29.549976630912635 -95.91337595971028) bank54113 +54114 POINT(29.35146791752373 -94.9884125436468) bank54114 +54115 POINT(29.479349788295476 -96.03238949595939) bank54115 +54116 POINT(29.870643825951227 -95.34712969104915) bank54116 +54117 POINT(29.240402048383725 -95.8765186234147) bank54117 +54118 POINT(29.548630818948016 -96.16235395399904) bank54118 +54119 POINT(29.23136037654783 -96.13137984696311) bank54119 +54120 POINT(30.604501139054555 -95.12755094393644) bank54120 +54121 POINT(29.589525442329716 -94.62930298315311) bank54121 +54122 POINT(30.026236279130444 -95.38342926108896) bank54122 +54123 POINT(29.177112478091388 -95.70012523812863) bank54123 +54124 POINT(30.585459377581888 -94.39421752106776) bank54124 +54125 POINT(29.044875066472823 -95.99906012925152) bank54125 +54126 POINT(28.948666590838343 -95.92522826767215) bank54126 +54127 POINT(29.139737469696176 -94.77727053106099) bank54127 +54128 POINT(28.894265411768 -94.75612822103349) bank54128 +54129 POINT(29.808458022589377 -95.4465383270552) bank54129 +54130 POINT(30.455151904828284 -96.34182200695864) bank54130 +54131 POINT(29.733881851670482 -95.2384750371493) bank54131 +54132 POINT(30.262312175102352 -95.18529653007495) bank54132 +54133 POINT(29.75784951590648 -95.45403279696205) bank54133 +54134 POINT(30.503215368209393 -95.98327310608656) bank54134 +54135 POINT(29.263044953418397 -95.08599720162395) bank54135 +54136 POINT(30.164153265220108 -95.23728106892077) bank54136 +54137 POINT(29.714556349165054 -95.94037909299388) bank54137 +54138 POINT(29.326196407876107 -94.3972395411065) bank54138 +54139 POINT(30.419093124573354 -95.25872038068883) bank54139 +54140 POINT(29.97301285317464 -96.11111053515417) bank54140 +54141 POINT(30.034318512177876 -95.66092108409585) bank54141 +54142 POINT(29.835269249664044 -95.8717728327804) bank54142 +54143 POINT(29.908187408073942 -94.8361716177246) bank54143 +54144 POINT(29.179737828936243 -94.54737422311479) bank54144 +54145 POINT(30.52893624264431 -94.46858973905148) bank54145 +54146 POINT(29.86725133641637 -94.72083537410617) bank54146 +54147 POINT(29.740677921067793 -96.19155993796261) bank54147 +54148 POINT(30.68864244650036 -96.35977716801797) bank54148 +54149 POINT(28.97762141649179 -95.62807879579871) bank54149 +54150 POINT(29.978001393825664 -95.29925909103264) bank54150 +54151 POINT(29.523159111644546 -95.05297570166255) bank54151 +54152 POINT(30.05862825828509 -95.43420017666385) bank54152 +54153 POINT(30.728813824701685 -95.3054658859924) bank54153 +54154 POINT(29.83166670313323 -95.09282592906074) bank54154 +54155 POINT(29.25354431028176 -95.3573951189669) bank54155 +54156 POINT(29.965222384068998 -95.4212364323641) bank54156 +54157 POINT(30.723917207948702 -95.9248888964965) bank54157 +54158 POINT(29.319081508471555 -95.68261589890733) bank54158 +54159 POINT(30.082661317482565 -94.60615540745495) bank54159 +54160 POINT(28.862837708678256 -95.41866166824016) bank54160 +54161 POINT(29.140589392815038 -94.47525620242983) bank54161 +54162 POINT(29.16815841017553 -95.86567074579527) bank54162 +54163 POINT(30.73454481767532 -94.95964236641247) bank54163 +54164 POINT(29.10762867112325 -95.6504340197734) bank54164 +54165 POINT(29.79039682152123 -95.83406007309742) bank54165 +54166 POINT(28.85649374731322 -94.80270721843101) bank54166 +54167 POINT(29.768165591839168 -95.8253950320975) bank54167 +54168 POINT(30.371620821010374 -94.4765230350407) bank54168 +54169 POINT(30.4934910005666 -95.51492744428111) bank54169 +54170 POINT(30.248442105969453 -95.89065704879576) bank54170 +54171 POINT(30.64119748780488 -94.59885425818665) bank54171 +54172 POINT(29.768453137736135 -95.65311060537837) bank54172 +54173 POINT(30.46533565858998 -94.51922475081474) bank54173 +54174 POINT(30.109596655568104 -95.66086438422415) bank54174 +54175 POINT(29.642420026072568 -94.65897683703551) bank54175 +54176 POINT(29.798046711441533 -95.04421475783136) bank54176 +54177 POINT(29.833519627230686 -94.49618958584868) bank54177 +54178 POINT(30.505551250822993 -95.75859826173769) bank54178 +54179 POINT(30.038625391695263 -94.37190676463464) bank54179 +54180 POINT(30.734923242759212 -95.49984340015621) bank54180 +54181 POINT(30.305116065433154 -95.77119804814134) bank54181 +54182 POINT(29.07238268073603 -94.77458176376676) bank54182 +54183 POINT(29.627824545426755 -96.05552280149857) bank54183 +54184 POINT(30.099896816155496 -94.82842262994413) bank54184 +54185 POINT(29.22118567939526 -94.56876573607555) bank54185 +54186 POINT(30.21810556866676 -95.26865698084747) bank54186 +54187 POINT(29.249590509513308 -95.77763353291789) bank54187 +54188 POINT(29.802913502231696 -94.49348397259074) bank54188 +54189 POINT(29.856763740117827 -96.00434466558808) bank54189 +54190 POINT(29.750394963248613 -94.62617271407036) bank54190 +54191 POINT(29.336899392924305 -95.68984271621856) bank54191 +54192 POINT(29.8645571387129 -95.4662586852452) bank54192 +54193 POINT(29.889148193884836 -94.75803790794951) bank54193 +54194 POINT(28.944246009859192 -96.36127463220225) bank54194 +54195 POINT(29.192348457682982 -96.27004657428232) bank54195 +54196 POINT(28.86899248989535 -95.89899821013336) bank54196 +54197 POINT(29.168925505009817 -96.20906867534259) bank54197 +54198 POINT(29.404385126073496 -95.91778931320205) bank54198 +54199 POINT(30.217803919298035 -94.70548763554666) bank54199 +54200 POINT(30.30889791119672 -95.09463606748655) bank54200 +54201 POINT(29.33264330296758 -95.87401724960498) bank54201 +54202 POINT(29.615247318057556 -94.55554959684905) bank54202 +54203 POINT(30.64754369903536 -96.3128215849244) bank54203 +54204 POINT(28.926110750963872 -95.43028998506902) bank54204 +54205 POINT(30.58938434846534 -95.49519092039718) bank54205 +54206 POINT(30.077751639365946 -95.29891018559357) bank54206 +54207 POINT(30.24154991193848 -96.13126565938236) bank54207 +54208 POINT(30.393772665546916 -96.19234374655095) bank54208 +54209 POINT(30.172748473173606 -95.65490667535829) bank54209 +54210 POINT(30.640067516023144 -96.23214639819933) bank54210 +54211 POINT(29.03049185143036 -95.11559166965867) bank54211 +54212 POINT(28.876740025968036 -95.1328063651153) bank54212 +54213 POINT(28.90030001157704 -94.91305446575896) bank54213 +54214 POINT(29.306674977176172 -96.06985727550848) bank54214 +54215 POINT(30.173104748577096 -95.95875338488409) bank54215 +54216 POINT(29.866100381159853 -95.83217527106954) bank54216 +54217 POINT(30.432930360562555 -95.69660063102623) bank54217 +54218 POINT(30.74460647344523 -95.80201829078182) bank54218 +54219 POINT(29.080029523387527 -95.91450353626351) bank54219 +54220 POINT(29.04094238976525 -95.84432390421193) bank54220 +54221 POINT(30.724128045392934 -94.38721603582194) bank54221 +54222 POINT(28.772472363425166 -94.92879077956165) bank54222 +54223 POINT(29.958970814674068 -95.54119318137053) bank54223 +54224 POINT(30.682289277135126 -95.2026369818259) bank54224 +54225 POINT(29.469768899989568 -94.3722266441957) bank54225 +54226 POINT(29.967919198845255 -94.51654429775321) bank54226 +54227 POINT(29.066752966506336 -95.65670577968389) bank54227 +54228 POINT(29.488322039349903 -95.63632232421833) bank54228 +54229 POINT(28.79755530270139 -94.76326714805329) bank54229 +54230 POINT(29.35450893298772 -95.36650119838164) bank54230 +54231 POINT(30.005682473855874 -96.15307442550143) bank54231 +54232 POINT(29.820034006145505 -95.84610343765344) bank54232 +54233 POINT(29.906693175205383 -95.88515777465744) bank54233 +54234 POINT(29.127327976546944 -94.58637457351507) bank54234 +54235 POINT(29.873793167869447 -95.65261199463801) bank54235 +54236 POINT(29.057833398095617 -94.92789035768315) bank54236 +54237 POINT(30.622409982814077 -95.0588604671395) bank54237 +54238 POINT(29.09987138008131 -95.2899307858915) bank54238 +54239 POINT(30.46474951476147 -94.95741978161045) bank54239 +54240 POINT(28.811948636351445 -96.13601739914817) bank54240 +54241 POINT(29.53836048373696 -95.94502784902089) bank54241 +54242 POINT(29.451237080591053 -94.46873716905291) bank54242 +54243 POINT(30.580301011970846 -96.10058756144869) bank54243 +54244 POINT(29.226864080314513 -94.60814941096349) bank54244 +54245 POINT(29.613362161507368 -94.40401435216103) bank54245 +54246 POINT(30.243865696805074 -95.24719750109311) bank54246 +54247 POINT(30.519466200072905 -94.88096249629164) bank54247 +54248 POINT(29.3335964062356 -94.54871762236523) bank54248 +54249 POINT(29.31268061285141 -95.57624210719553) bank54249 +54250 POINT(29.827587066218964 -94.70328441225273) bank54250 +54251 POINT(30.726757293433018 -94.48373525273394) bank54251 +54252 POINT(30.633714895589378 -95.79308536502086) bank54252 +54253 POINT(29.995715838169108 -95.3172217310691) bank54253 +54254 POINT(29.767292953201796 -94.63184094697658) bank54254 +54255 POINT(30.018319089022793 -94.48985730304936) bank54255 +54256 POINT(29.122837757174707 -95.77957588648452) bank54256 +54257 POINT(29.88928896805654 -95.47950900499119) bank54257 +54258 POINT(29.66920111131398 -95.32574691201677) bank54258 +54259 POINT(29.89781479526317 -95.62735798112844) bank54259 +54260 POINT(30.24154076785218 -95.08963866302712) bank54260 +54261 POINT(30.71678948785658 -95.06530586915679) bank54261 +54262 POINT(29.896313493783612 -95.91899365880421) bank54262 +54263 POINT(30.727450019676112 -95.8006454463659) bank54263 +54264 POINT(29.816628773548015 -94.91253094175337) bank54264 +54265 POINT(30.490821511554966 -96.25659841223636) bank54265 +54266 POINT(29.761404718144153 -95.67357826203849) bank54266 +54267 POINT(30.361675071540628 -95.04256673880549) bank54267 +54268 POINT(30.255966969032208 -96.17982409174222) bank54268 +54269 POINT(28.997550593826556 -94.93055721536619) bank54269 +54270 POINT(30.03728850209016 -94.56452408355976) bank54270 +54271 POINT(30.749559849086094 -96.30862454461891) bank54271 +54272 POINT(29.637928497961166 -94.51427304576674) bank54272 +54273 POINT(29.799723869560633 -95.58237948800625) bank54273 +54274 POINT(30.46871955262182 -95.59400081411889) bank54274 +54275 POINT(29.996317826589785 -95.05520422281113) bank54275 +54276 POINT(30.027340199443383 -94.6879178957813) bank54276 +54277 POINT(29.978971169007032 -94.73742738002285) bank54277 +54278 POINT(29.964643344654714 -94.44138649378374) bank54278 +54279 POINT(29.74055021873719 -94.76833437402675) bank54279 +54280 POINT(30.677679596423957 -95.87600035226605) bank54280 +54281 POINT(29.66949853403794 -95.84977901609398) bank54281 +54282 POINT(29.892993199470904 -94.72390855500419) bank54282 +54283 POINT(28.81219072331087 -96.30845078819932) bank54283 +54284 POINT(29.36656880775365 -94.67208861335244) bank54284 +54285 POINT(30.71398817696123 -95.9795493965049) bank54285 +54286 POINT(29.72256232786165 -95.54351160751332) bank54286 +54287 POINT(28.77437358714194 -96.07937220399484) bank54287 +54288 POINT(30.539003436599625 -94.82977025858955) bank54288 +54289 POINT(29.900477575976897 -95.20404274493461) bank54289 +54290 POINT(29.719776627533363 -96.36117560245057) bank54290 +54291 POINT(29.142302907729903 -95.16365299667187) bank54291 +54292 POINT(29.825214843452105 -94.63430558865441) bank54292 +54293 POINT(29.268116340812796 -95.7727589829882) bank54293 +54294 POINT(29.4021774561162 -95.0715358755915) bank54294 +54295 POINT(29.60138901651507 -96.3356389149959) bank54295 +54296 POINT(29.41495686232932 -95.5562097499718) bank54296 +54297 POINT(28.95216328616793 -95.24798032021903) bank54297 +54298 POINT(30.174015458838927 -95.63421675042379) bank54298 +54299 POINT(29.473780711256886 -94.3903604821705) bank54299 +54300 POINT(29.04066902112509 -94.88127932478164) bank54300 +54301 POINT(29.32337178924182 -95.40226750078018) bank54301 +54302 POINT(29.885653887134232 -95.6788239513021) bank54302 +54303 POINT(30.601373170301944 -95.84413828572612) bank54303 +54304 POINT(30.463635019829752 -96.08340519930567) bank54304 +54305 POINT(29.583470296591557 -95.97024167225693) bank54305 +54306 POINT(29.00666150727914 -94.87108394869895) bank54306 +54307 POINT(28.881512076888136 -95.7780388211462) bank54307 +54308 POINT(30.394424790364056 -96.10567460891149) bank54308 +54309 POINT(30.751090584524718 -94.87208341338162) bank54309 +54310 POINT(29.68769995892867 -94.92123879341702) bank54310 +54311 POINT(29.69296848963691 -96.0319493602937) bank54311 +54312 POINT(30.01303387243439 -96.15526726807735) bank54312 +54313 POINT(29.46458002907354 -94.50582356127298) bank54313 +54314 POINT(29.0434308191809 -96.31363573832691) bank54314 +54315 POINT(28.774059485775247 -96.11733011078964) bank54315 +54316 POINT(30.32890051340397 -94.95266456160462) bank54316 +54317 POINT(29.931929383015692 -94.74574754672787) bank54317 +54318 POINT(29.72850134531374 -95.00260316763301) bank54318 +54319 POINT(29.9870935332366 -95.47238615902677) bank54319 +54320 POINT(30.10968029887689 -94.42834981650793) bank54320 +54321 POINT(28.967841692227093 -96.11694145251762) bank54321 +54322 POINT(29.507623941621976 -95.20626629515165) bank54322 +54323 POINT(29.137534090766135 -96.06985419773764) bank54323 +54324 POINT(30.258370432515694 -94.9896711859068) bank54324 +54325 POINT(29.075165191597588 -95.2601227169967) bank54325 +54326 POINT(29.280416840275496 -96.01548183689034) bank54326 +54327 POINT(29.252386338985023 -95.46003332313448) bank54327 +54328 POINT(29.903963752870798 -95.2863292427004) bank54328 +54329 POINT(29.311383568509683 -96.0246974124224) bank54329 +54330 POINT(29.52537455495725 -94.67547642216556) bank54330 +54331 POINT(30.49947365601862 -96.25364890810441) bank54331 +54332 POINT(29.039491346534383 -95.86457699635608) bank54332 +54333 POINT(30.39651994627668 -95.90005509566721) bank54333 +54334 POINT(29.536206893699514 -96.16578705014759) bank54334 +54335 POINT(29.342211048034702 -95.92874116004201) bank54335 +54336 POINT(29.08304097801112 -96.30592227102636) bank54336 +54337 POINT(30.003464846616644 -95.51030682218689) bank54337 +54338 POINT(29.54094196776481 -95.89499675332671) bank54338 +54339 POINT(29.72045169384934 -94.6644663077601) bank54339 +54340 POINT(29.315943764285887 -94.54659182623223) bank54340 +54341 POINT(30.13276513996138 -95.44198748467647) bank54341 +54342 POINT(29.23725560600106 -96.1535792659427) bank54342 +54343 POINT(29.28977315115081 -95.54737591125419) bank54343 +54344 POINT(28.916901008042068 -94.5536438636706) bank54344 +54345 POINT(30.626233516372718 -95.63354825694464) bank54345 +54346 POINT(29.731705947851935 -95.6479934532225) bank54346 +54347 POINT(28.97283388401098 -96.05565762141048) bank54347 +54348 POINT(29.31085404736047 -95.92000061930798) bank54348 +54349 POINT(30.425265600828816 -96.07418004851093) bank54349 +54350 POINT(29.542927009505924 -96.12669464560706) bank54350 +54351 POINT(28.846153792383348 -94.99124159923589) bank54351 +54352 POINT(29.343956829720945 -95.4888760986207) bank54352 +54353 POINT(29.23800058290567 -95.60549350040847) bank54353 +54354 POINT(29.257514749812806 -94.42660506302119) bank54354 +54355 POINT(30.756073751831472 -95.12570021082746) bank54355 +54356 POINT(29.199375543372852 -94.95391975501956) bank54356 +54357 POINT(30.63578668475101 -96.02791700619103) bank54357 +54358 POINT(30.670887868013487 -95.95830674562102) bank54358 +54359 POINT(29.13689152433765 -94.57564874745111) bank54359 +54360 POINT(29.5901100013314 -95.84580987297355) bank54360 +54361 POINT(30.435245827412498 -94.96545238612258) bank54361 +54362 POINT(30.054587463013927 -95.50966403950271) bank54362 +54363 POINT(30.403589250548844 -95.31568549675178) bank54363 +54364 POINT(30.585805168368186 -95.82800637896563) bank54364 +54365 POINT(30.269112132516756 -94.48839378720466) bank54365 +54366 POINT(30.080823181495784 -94.54945846079956) bank54366 +54367 POINT(29.07556425332174 -95.76851193127288) bank54367 +54368 POINT(30.283324962900558 -95.83305232006268) bank54368 +54369 POINT(30.468686823982647 -94.55811677723533) bank54369 +54370 POINT(29.49781563267316 -95.87781176513846) bank54370 +54371 POINT(28.876496717528628 -96.00122881899959) bank54371 +54372 POINT(30.449175523907567 -94.91553366507065) bank54372 +54373 POINT(29.112434744147446 -96.01401570979132) bank54373 +54374 POINT(30.73048451352177 -96.29808797701638) bank54374 +54375 POINT(29.5819012765946 -96.26366611559052) bank54375 +54376 POINT(30.200366137361765 -96.02486695723039) bank54376 +54377 POINT(30.217318733866637 -95.69704720800307) bank54377 +54378 POINT(30.46224342452705 -95.56197214034174) bank54378 +54379 POINT(29.668604668804747 -95.96685336515316) bank54379 +54380 POINT(29.521836999206048 -94.92840117596114) bank54380 +54381 POINT(30.58534103311187 -95.51922599415106) bank54381 +54382 POINT(29.0299011562988 -94.80627554467948) bank54382 +54383 POINT(29.75657574373076 -94.48128186533307) bank54383 +54384 POINT(30.056690785933426 -95.77766838985735) bank54384 +54385 POINT(29.602087547818744 -94.75368467650999) bank54385 +54386 POINT(29.343364089226053 -95.32059754300205) bank54386 +54387 POINT(29.27922315389003 -95.66928816629608) bank54387 +54388 POINT(30.507215879505523 -95.1575141793587) bank54388 +54389 POINT(30.66555259567555 -94.90683403248174) bank54389 +54390 POINT(29.363549246854994 -94.55974681966481) bank54390 +54391 POINT(29.71616195369194 -95.75684636307322) bank54391 +54392 POINT(29.26611973512833 -95.68341916805515) bank54392 +54393 POINT(29.224579893063233 -96.09741829716035) bank54393 +54394 POINT(29.239815598709693 -94.49208391442303) bank54394 +54395 POINT(29.802379744371734 -94.94454414868724) bank54395 +54396 POINT(30.20720274613585 -94.67667883386953) bank54396 +54397 POINT(29.958221489518127 -96.22326595469772) bank54397 +54398 POINT(29.322490614149082 -95.82754613074532) bank54398 +54399 POINT(29.559430959846576 -95.0991514231283) bank54399 +54400 POINT(29.870695029154444 -95.93089642255345) bank54400 +54401 POINT(30.63747153285052 -95.68657367983997) bank54401 +54402 POINT(29.340163904583726 -94.48539272454512) bank54402 +54403 POINT(29.585785447097088 -95.72143358574725) bank54403 +54404 POINT(30.538616377874124 -95.06791201330526) bank54404 +54405 POINT(30.163941865977552 -95.92818918896525) bank54405 +54406 POINT(29.149342474839504 -96.31262171330354) bank54406 +54407 POINT(29.935440773153328 -95.68180521470646) bank54407 +54408 POINT(30.52482916709538 -95.92655294044069) bank54408 +54409 POINT(29.308296301519782 -96.0781460610051) bank54409 +54410 POINT(30.522091401205444 -94.80146185842932) bank54410 +54411 POINT(29.81410277694633 -95.1575085233821) bank54411 +54412 POINT(29.190645117642553 -95.12127684802815) bank54412 +54413 POINT(29.80872726104749 -94.44537199961749) bank54413 +54414 POINT(28.763625971704617 -95.2302685730942) bank54414 +54415 POINT(30.338946748503634 -96.30261480820273) bank54415 +54416 POINT(29.21850827741455 -95.65525806676962) bank54416 +54417 POINT(29.53028389998228 -94.49321874944243) bank54417 +54418 POINT(29.87019873214866 -95.64225299484538) bank54418 +54419 POINT(29.606529232361545 -94.68486562213769) bank54419 +54420 POINT(29.84128412437727 -95.09506317245543) bank54420 +54421 POINT(28.90560409403025 -95.78105072409727) bank54421 +54422 POINT(30.64517897478071 -95.0829951013797) bank54422 +54423 POINT(29.198017744649572 -96.0868147336601) bank54423 +54424 POINT(30.58082938185924 -95.36198160078146) bank54424 +54425 POINT(29.26903551945368 -95.36535003155814) bank54425 +54426 POINT(30.677260234730763 -95.99294062494336) bank54426 +54427 POINT(28.87115472755777 -95.42433178851194) bank54427 +54428 POINT(28.817585885864382 -96.00484301655231) bank54428 +54429 POINT(30.27619674262167 -95.10190495764238) bank54429 +54430 POINT(29.31059234319381 -95.38102004079376) bank54430 +54431 POINT(30.744974551103244 -95.38943669149194) bank54431 +54432 POINT(28.884811693587064 -95.90146010628332) bank54432 +54433 POINT(30.75692673842005 -95.17517288953871) bank54433 +54434 POINT(30.572218226382333 -94.83712611725977) bank54434 +54435 POINT(29.691815036622803 -95.07760673215195) bank54435 +54436 POINT(29.481153946219766 -94.54256950705725) bank54436 +54437 POINT(30.671479399929325 -94.76647786811903) bank54437 +54438 POINT(29.17006985958542 -95.35977939868214) bank54438 +54439 POINT(30.7266835556371 -95.22301850332173) bank54439 +54440 POINT(29.26443272220655 -96.28418698212715) bank54440 +54441 POINT(29.670018904290643 -94.4249705870613) bank54441 +54442 POINT(30.475429990112122 -94.84078955995498) bank54442 +54443 POINT(29.43584844479419 -94.82148645440824) bank54443 +54444 POINT(30.608052471196075 -95.3582979983153) bank54444 +54445 POINT(29.126839609216656 -95.5532281485227) bank54445 +54446 POINT(29.96473489805512 -94.73952034267468) bank54446 +54447 POINT(29.246299987522157 -94.70890076322863) bank54447 +54448 POINT(29.394555371761452 -95.28482834036001) bank54448 +54449 POINT(29.642123821965455 -95.05395369665051) bank54449 +54450 POINT(30.541014337851863 -94.49624531151044) bank54450 +54451 POINT(28.837607183791565 -95.14340378230358) bank54451 +54452 POINT(30.13959617097887 -94.40906435671299) bank54452 +54453 POINT(30.68881953342445 -95.32696294403056) bank54453 +54454 POINT(29.52187734791394 -94.56433063607531) bank54454 +54455 POINT(30.224261855072108 -95.93898673269494) bank54455 +54456 POINT(29.03209398036357 -94.77468234867328) bank54456 +54457 POINT(29.7676980658356 -96.23845373907685) bank54457 +54458 POINT(29.350850598737505 -95.04165947823935) bank54458 +54459 POINT(29.837478648136436 -96.18731277627589) bank54459 +54460 POINT(29.116118138583186 -94.5527893976541) bank54460 +54461 POINT(30.60875248911643 -95.32298141263762) bank54461 +54462 POINT(29.249651498941397 -94.72035171328747) bank54462 +54463 POINT(30.52943760021078 -95.69442737118362) bank54463 +54464 POINT(30.11887485373088 -94.79030222624328) bank54464 +54465 POINT(29.68685073372323 -96.2044560506715) bank54465 +54466 POINT(28.954808486842804 -94.55660609763285) bank54466 +54467 POINT(29.39785178860654 -94.70319069860125) bank54467 +54468 POINT(29.28069661353465 -95.18758481678206) bank54468 +54469 POINT(29.07727621853236 -95.95658761585635) bank54469 +54470 POINT(29.583734359859584 -94.49015097490015) bank54470 +54471 POINT(29.879125639925615 -96.18297100624966) bank54471 +54472 POINT(30.221475426626125 -95.53312157440307) bank54472 +54473 POINT(30.41934287728644 -94.57425764007697) bank54473 +54474 POINT(29.427098180650137 -95.9619611623185) bank54474 +54475 POINT(29.812987197646862 -95.7011621330814) bank54475 +54476 POINT(30.554568374357768 -95.43647108575182) bank54476 +54477 POINT(30.64857417906739 -94.49922399556841) bank54477 +54478 POINT(30.432101646687364 -94.72855891365771) bank54478 +54479 POINT(29.95798510488837 -95.1809297141422) bank54479 +54480 POINT(30.561326796832393 -94.48954822544388) bank54480 +54481 POINT(30.31483238946937 -96.1497276304366) bank54481 +54482 POINT(29.300762983576544 -94.69744122593302) bank54482 +54483 POINT(30.51692831496605 -96.30861138254457) bank54483 +54484 POINT(30.109911545190982 -96.32852256202898) bank54484 +54485 POINT(29.343963070898788 -96.27519832530284) bank54485 +54486 POINT(29.27468444126244 -95.8046527439898) bank54486 +54487 POINT(30.265234515680753 -96.32889140080827) bank54487 +54488 POINT(30.353015911638543 -95.02275586282708) bank54488 +54489 POINT(28.99795743380299 -94.41710966384282) bank54489 +54490 POINT(30.191746473456245 -96.15863313462279) bank54490 +54491 POINT(29.071311760129625 -95.48550106426428) bank54491 +54492 POINT(29.573177555648275 -94.86578185911736) bank54492 +54493 POINT(29.530777091057885 -94.68394818331285) bank54493 +54494 POINT(30.535982410328717 -96.32420085623406) bank54494 +54495 POINT(30.554328877188162 -96.16881501862565) bank54495 +54496 POINT(28.983094982102905 -95.64333148480488) bank54496 +54497 POINT(29.725079527562155 -95.43022817819006) bank54497 +54498 POINT(29.07502039363139 -95.7778647016845) bank54498 +54499 POINT(30.19697200848028 -95.41128026576129) bank54499 +54500 POINT(29.31281824695658 -95.71504812898938) bank54500 +54501 POINT(29.620607394859274 -94.51423522050969) bank54501 +54502 POINT(29.835929973603136 -94.56460317987381) bank54502 +54503 POINT(29.401794112018283 -95.38091047763388) bank54503 +54504 POINT(29.04247354375169 -95.87774121155434) bank54504 +54505 POINT(28.907917404635633 -95.46659453427287) bank54505 +54506 POINT(29.66164811341966 -95.64060641483782) bank54506 +54507 POINT(28.98934375264034 -95.53205479851746) bank54507 +54508 POINT(30.720310088876076 -95.40817576109463) bank54508 +54509 POINT(29.49866068695524 -96.15514181682296) bank54509 +54510 POINT(29.604328401848687 -95.74850447200211) bank54510 +54511 POINT(28.83711865437121 -94.53649593438621) bank54511 +54512 POINT(30.104280813146953 -96.16515113398448) bank54512 +54513 POINT(30.21743624861662 -96.2263784286913) bank54513 +54514 POINT(29.13775718948785 -94.91176029925539) bank54514 +54515 POINT(30.506549282909706 -95.7102872821711) bank54515 +54516 POINT(30.28230866959042 -95.48596607238474) bank54516 +54517 POINT(30.48886401765346 -95.67398991189894) bank54517 +54518 POINT(30.25225846049857 -94.4115538051966) bank54518 +54519 POINT(30.238223966851383 -95.68517252266601) bank54519 +54520 POINT(30.16938253211424 -95.6353282355231) bank54520 +54521 POINT(29.518398010880023 -95.14259928894658) bank54521 +54522 POINT(30.522159794514778 -95.89790995814694) bank54522 +54523 POINT(30.65799473084243 -94.7324909866881) bank54523 +54524 POINT(30.509377540201957 -96.16056244782467) bank54524 +54525 POINT(30.194061166902543 -94.81426059320404) bank54525 +54526 POINT(29.88938811866691 -94.54130295732179) bank54526 +54527 POINT(29.882461097408314 -96.09342318666114) bank54527 +54528 POINT(30.42498242324402 -95.931239827778) bank54528 +54529 POINT(30.233039693293982 -95.7047584250378) bank54529 +54530 POINT(29.279123983152253 -95.30140800703248) bank54530 +54531 POINT(29.452706224285432 -94.59005821327729) bank54531 +54532 POINT(30.33865839073617 -94.84239502339022) bank54532 +54533 POINT(29.987427219845703 -94.74800067397436) bank54533 +54534 POINT(29.49422046878449 -96.07522495357931) bank54534 +54535 POINT(30.6712805421928 -95.6902468294487) bank54535 +54536 POINT(29.570522437703374 -95.71649107189639) bank54536 +54537 POINT(29.08714575273172 -95.47156772030208) bank54537 +54538 POINT(29.18004141574773 -94.46337633906988) bank54538 +54539 POINT(30.51524976963786 -95.02696180171866) bank54539 +54540 POINT(30.690740559197494 -95.85365272962672) bank54540 +54541 POINT(30.50982233991756 -94.84684073204754) bank54541 +54542 POINT(28.989759318085554 -95.42301431614283) bank54542 +54543 POINT(29.334417049701297 -95.49878380078637) bank54543 +54544 POINT(28.93548962705108 -95.2885395241884) bank54544 +54545 POINT(29.70767984930093 -95.20562813896741) bank54545 +54546 POINT(29.440638999107456 -94.52576587800576) bank54546 +54547 POINT(29.747105186845957 -95.50345658438158) bank54547 +54548 POINT(29.423099853264752 -95.8169334205563) bank54548 +54549 POINT(29.296336786013573 -94.83816409104705) bank54549 +54550 POINT(29.450296423959394 -95.72139718016855) bank54550 +54551 POINT(29.52087485755496 -95.7581358756772) bank54551 +54552 POINT(29.659241862304988 -95.90896149329899) bank54552 +54553 POINT(30.688967237229548 -96.2822909560051) bank54553 +54554 POINT(30.2029808303519 -94.91523424738102) bank54554 +54555 POINT(29.72375432829528 -95.5410053233999) bank54555 +54556 POINT(30.249319645130207 -94.42583778249268) bank54556 +54557 POINT(30.486070970211287 -94.70309328766251) bank54557 +54558 POINT(29.943558163386378 -96.11193007244884) bank54558 +54559 POINT(30.656948465800127 -94.40260956245233) bank54559 +54560 POINT(29.653549975544244 -95.93135588605853) bank54560 +54561 POINT(29.133353844572262 -95.26837474382711) bank54561 +54562 POINT(29.530571327806907 -96.21503134197647) bank54562 +54563 POINT(29.92383241324327 -95.47251458731192) bank54563 +54564 POINT(30.454303839608677 -96.15666488191668) bank54564 +54565 POINT(29.4839645443265 -96.14165652638059) bank54565 +54566 POINT(30.344198146009667 -95.12124486166908) bank54566 +54567 POINT(29.992619077681304 -96.07870744322338) bank54567 +54568 POINT(29.74513924154123 -95.86250944129515) bank54568 +54569 POINT(28.89699461999497 -96.29155646549651) bank54569 +54570 POINT(29.461659455657305 -96.23586731091494) bank54570 +54571 POINT(30.63430454093403 -95.51710944343102) bank54571 +54572 POINT(29.118401604004777 -95.25344074968491) bank54572 +54573 POINT(29.603823524788602 -95.65298648173948) bank54573 +54574 POINT(29.109373223644873 -94.6064484601724) bank54574 +54575 POINT(30.18873653795185 -94.62119450343022) bank54575 +54576 POINT(30.284804360201488 -95.2287854100781) bank54576 +54577 POINT(29.048335124825712 -94.59262542916636) bank54577 +54578 POINT(29.79313142273014 -96.29354200533523) bank54578 +54579 POINT(28.921282024706574 -95.68132778313786) bank54579 +54580 POINT(28.918018031536192 -96.07526724149443) bank54580 +54581 POINT(29.76251120629543 -95.84138554092496) bank54581 +54582 POINT(29.68732187370854 -95.18384475771045) bank54582 +54583 POINT(29.64729030928263 -96.1718738332692) bank54583 +54584 POINT(30.010559341487046 -95.5619335081151) bank54584 +54585 POINT(29.70675069655431 -94.61531943371195) bank54585 +54586 POINT(30.06199485154176 -95.602636027016) bank54586 +54587 POINT(29.54502533101594 -96.31958115522562) bank54587 +54588 POINT(30.30977135520132 -96.24831469881201) bank54588 +54589 POINT(30.37410475178119 -95.74459153119426) bank54589 +54590 POINT(30.5147240931686 -94.84963388338672) bank54590 +54591 POINT(28.938960202578784 -94.55925226340592) bank54591 +54592 POINT(30.13334024903664 -96.02929022768106) bank54592 +54593 POINT(29.901845454636636 -95.79216136308204) bank54593 +54594 POINT(29.91210787531847 -94.75332221379314) bank54594 +54595 POINT(29.507854517384658 -96.26858194514608) bank54595 +54596 POINT(30.607865508059092 -94.928875826025) bank54596 +54597 POINT(29.384033054846633 -94.97731297791647) bank54597 +54598 POINT(28.9511339009039 -95.54077167931061) bank54598 +54599 POINT(29.19458972577271 -95.49820146433406) bank54599 +54600 POINT(30.066358396131818 -96.1064710191008) bank54600 +54601 POINT(29.09376254174322 -95.78652316286345) bank54601 +54602 POINT(29.410513380483422 -94.46518568843311) bank54602 +54603 POINT(30.266444505814786 -94.54742113701624) bank54603 +54604 POINT(29.00357791030699 -96.22121386264833) bank54604 +54605 POINT(29.021840219166975 -95.84671599707775) bank54605 +54606 POINT(30.085733053699062 -94.62319677512659) bank54606 +54607 POINT(30.1485374820445 -95.33127497685636) bank54607 +54608 POINT(30.586272305032523 -95.47270843562865) bank54608 +54609 POINT(30.232077093454265 -94.84518864650171) bank54609 +54610 POINT(29.46705191257284 -94.90718583481119) bank54610 +54611 POINT(29.815302997413053 -96.04115452402665) bank54611 +54612 POINT(30.56531833160706 -96.06001618723634) bank54612 +54613 POINT(29.79737829017592 -96.16235123335657) bank54613 +54614 POINT(30.090148163320617 -95.46859010710357) bank54614 +54615 POINT(29.358443736856643 -96.03343658749849) bank54615 +54616 POINT(29.458580122364044 -96.13545557641326) bank54616 +54617 POINT(29.6348465839605 -94.86701383378026) bank54617 +54618 POINT(29.403724865879216 -94.75580548245809) bank54618 +54619 POINT(30.309102535712736 -95.8426184295254) bank54619 +54620 POINT(29.231339820409012 -95.61872712436154) bank54620 +54621 POINT(30.522616766587436 -95.86803478172448) bank54621 +54622 POINT(29.530801888365833 -96.13346954740696) bank54622 +54623 POINT(30.70197248953426 -95.64361878182696) bank54623 +54624 POINT(30.366220023717826 -94.57154651184022) bank54624 +54625 POINT(29.185491612559908 -95.3257475502317) bank54625 +54626 POINT(29.311301320451474 -95.0682829205004) bank54626 +54627 POINT(29.0772970742798 -94.4396274333681) bank54627 +54628 POINT(30.734678889466117 -94.51712671689262) bank54628 +54629 POINT(30.202483951483952 -95.28457866663345) bank54629 +54630 POINT(29.31760323464153 -95.58553850811947) bank54630 +54631 POINT(30.74702269673941 -94.95617569344928) bank54631 +54632 POINT(29.562590697305712 -95.51624885571101) bank54632 +54633 POINT(29.419562199031482 -96.0583012402195) bank54633 +54634 POINT(29.979445544324804 -95.01031618070863) bank54634 +54635 POINT(30.65963868406948 -95.01305378950433) bank54635 +54636 POINT(29.82116195386877 -95.4372561720857) bank54636 +54637 POINT(29.05920088953902 -96.0155231468413) bank54637 +54638 POINT(29.342295474660894 -95.80605548239471) bank54638 +54639 POINT(29.52142643364448 -94.50952483290675) bank54639 +54640 POINT(29.253802546434606 -95.21599584911829) bank54640 +54641 POINT(29.237039195187425 -94.39172028479635) bank54641 +54642 POINT(28.80609285750031 -94.87275156936256) bank54642 +54643 POINT(30.35959454794223 -96.21868865768609) bank54643 +54644 POINT(29.129685070447938 -96.21825104079737) bank54644 +54645 POINT(29.90086370899647 -96.11767521728186) bank54645 +54646 POINT(28.78900155659593 -95.24746195232305) bank54646 +54647 POINT(29.950070720848647 -95.79656396945435) bank54647 +54648 POINT(30.694658227650347 -96.13342788412848) bank54648 +54649 POINT(29.39828428113423 -94.54869149727327) bank54649 +54650 POINT(29.13939555398376 -96.31843138128858) bank54650 +54651 POINT(29.28254115609362 -96.18666832034909) bank54651 +54652 POINT(30.527537386084436 -95.41767525068286) bank54652 +54653 POINT(28.913105065820265 -95.53886928933463) bank54653 +54654 POINT(30.673286455515957 -96.1014687970122) bank54654 +54655 POINT(30.746713459929964 -96.28588009731591) bank54655 +54656 POINT(29.74980833954913 -95.58895072373316) bank54656 +54657 POINT(29.83814656960207 -95.95389911917773) bank54657 +54658 POINT(28.767169196040278 -94.8222891728802) bank54658 +54659 POINT(29.78675792930864 -94.90841398033332) bank54659 +54660 POINT(29.03015771711792 -95.25749053752668) bank54660 +54661 POINT(29.754324382114493 -95.55169264910198) bank54661 +54662 POINT(29.790695558204284 -96.19737027147009) bank54662 +54663 POINT(29.608429129086424 -96.24724915885852) bank54663 +54664 POINT(29.560496660916517 -95.32774476961548) bank54664 +54665 POINT(30.47518613836975 -95.81348687507457) bank54665 +54666 POINT(29.124915998057492 -95.29277062560094) bank54666 +54667 POINT(30.334580066190885 -95.27417350830531) bank54667 +54668 POINT(29.89929416608465 -96.24160798630105) bank54668 +54669 POINT(29.798330234720446 -94.79034592878311) bank54669 +54670 POINT(29.95786887383978 -95.11193532941647) bank54670 +54671 POINT(29.256056021098644 -95.20050138247288) bank54671 +54672 POINT(29.73563447411631 -95.64911221798302) bank54672 +54673 POINT(30.37834483553828 -95.83133072774987) bank54673 +54674 POINT(28.82065132155494 -95.59891129540443) bank54674 +54675 POINT(29.035057379567686 -94.57164379250362) bank54675 +54676 POINT(29.744742338134184 -95.78717373882647) bank54676 +54677 POINT(30.095199045531718 -94.39104190260342) bank54677 +54678 POINT(30.436442246097947 -95.29943732801868) bank54678 +54679 POINT(30.698538250964802 -95.33930654388885) bank54679 +54680 POINT(29.879379688589385 -95.61639831304653) bank54680 +54681 POINT(29.204827891061115 -95.10710588489292) bank54681 +54682 POINT(29.578818662688306 -94.46966017897981) bank54682 +54683 POINT(29.083547467831885 -95.38793061979182) bank54683 +54684 POINT(28.916360037401454 -95.85632845025557) bank54684 +54685 POINT(30.372271804180418 -95.66633899153774) bank54685 +54686 POINT(29.451832723569627 -96.16143397090462) bank54686 +54687 POINT(30.328158673178972 -95.72775441221658) bank54687 +54688 POINT(30.343338422067667 -96.07389219418343) bank54688 +54689 POINT(29.6204773424309 -94.66039909508106) bank54689 +54690 POINT(30.498869443782834 -94.76178575945774) bank54690 +54691 POINT(30.176677308423656 -94.4847159960284) bank54691 +54692 POINT(28.96619458405223 -95.16594701326878) bank54692 +54693 POINT(30.223377313689202 -94.56165078779294) bank54693 +54694 POINT(30.715744458562572 -95.40972477349443) bank54694 +54695 POINT(29.322554284817723 -94.51297419051038) bank54695 +54696 POINT(30.73366672099833 -95.88905552581657) bank54696 +54697 POINT(29.550561440248963 -95.46796334276692) bank54697 +54698 POINT(30.580273368310298 -95.34281938919446) bank54698 +54699 POINT(29.825289511651974 -95.05140682626181) bank54699 +54700 POINT(29.75847442508858 -94.78959248052097) bank54700 +54701 POINT(30.68707272035124 -95.15272442176706) bank54701 +54702 POINT(30.20569233886628 -94.79671296073207) bank54702 +54703 POINT(29.435466729668118 -95.25691008015085) bank54703 +54704 POINT(30.685152172479672 -95.69077546007729) bank54704 +54705 POINT(29.86132224558521 -94.56824634420164) bank54705 +54706 POINT(29.729571904679077 -95.83279821912703) bank54706 +54707 POINT(29.074301041643196 -94.5839840860868) bank54707 +54708 POINT(30.16427379882832 -95.57589834329696) bank54708 +54709 POINT(28.793618903608515 -95.19191274637612) bank54709 +54710 POINT(29.87750528476892 -95.25818454647373) bank54710 +54711 POINT(29.759022852100475 -95.90003570930648) bank54711 +54712 POINT(29.432249420013644 -95.42956240820648) bank54712 +54713 POINT(30.519260102340414 -96.24445022751442) bank54713 +54714 POINT(29.48297388032237 -95.19181613154407) bank54714 +54715 POINT(28.970716654870895 -94.62375418960066) bank54715 +54716 POINT(30.339957664291163 -94.85360664580465) bank54716 +54717 POINT(29.727715582109784 -96.1494527398238) bank54717 +54718 POINT(29.1214452223865 -96.32301811105386) bank54718 +54719 POINT(29.02285760304629 -94.51736047505746) bank54719 +54720 POINT(29.014049627171293 -94.54383692125897) bank54720 +54721 POINT(30.39959385429874 -95.30003867508701) bank54721 +54722 POINT(28.91728891840822 -95.93383560984009) bank54722 +54723 POINT(28.927553041292665 -95.03266721152427) bank54723 +54724 POINT(30.028323979686327 -96.00406112813283) bank54724 +54725 POINT(29.633849126750224 -94.98782216679605) bank54725 +54726 POINT(29.98882538939996 -95.30324077679244) bank54726 +54727 POINT(30.469727634299513 -95.48921245294522) bank54727 +54728 POINT(28.951931027289703 -96.14737715797355) bank54728 +54729 POINT(29.508112082475023 -96.36975374328941) bank54729 +54730 POINT(29.682639442816715 -95.55981166635702) bank54730 +54731 POINT(29.152667375762054 -95.07035973229638) bank54731 +54732 POINT(29.704782840650267 -95.91822023141667) bank54732 +54733 POINT(29.526017281399444 -96.21113831618351) bank54733 +54734 POINT(30.33301119077278 -94.81010133520392) bank54734 +54735 POINT(28.917732468505047 -96.04604125668345) bank54735 +54736 POINT(30.68109669142609 -95.42349610565911) bank54736 +54737 POINT(29.42061484280494 -95.75495396546542) bank54737 +54738 POINT(29.506857448170685 -94.95150943668558) bank54738 +54739 POINT(30.75205423305484 -95.84227816145032) bank54739 +54740 POINT(30.236541675994552 -95.7228841734921) bank54740 +54741 POINT(30.00887956938554 -94.74986562311916) bank54741 +54742 POINT(30.11276798026069 -95.80112947274648) bank54742 +54743 POINT(29.104005660194 -95.14996862203554) bank54743 +54744 POINT(28.89709014242365 -95.46729367242745) bank54744 +54745 POINT(30.694917406013378 -95.57041384536197) bank54745 +54746 POINT(29.032763778097163 -94.61540645223042) bank54746 +54747 POINT(28.91964150913442 -96.17498534602603) bank54747 +54748 POINT(30.291085231702308 -94.4971021497311) bank54748 +54749 POINT(28.851243813768242 -95.79528405420078) bank54749 +54750 POINT(30.330719862311568 -95.57092299864003) bank54750 +54751 POINT(29.11748788139177 -95.42275523243727) bank54751 +54752 POINT(30.530853447758524 -96.10744453972255) bank54752 +54753 POINT(30.331128987836873 -95.50622578773657) bank54753 +54754 POINT(28.956537754162746 -95.87556739586982) bank54754 +54755 POINT(30.716390709029465 -94.50962843197193) bank54755 +54756 POINT(30.487602585541538 -95.09639443620222) bank54756 +54757 POINT(29.192468411532456 -96.15586094175241) bank54757 +54758 POINT(29.75221004395217 -95.07271765974791) bank54758 +54759 POINT(30.003332716499024 -94.37996814305103) bank54759 +54760 POINT(29.7169334404384 -95.67083142699688) bank54760 +54761 POINT(30.08603808528501 -94.60408006522245) bank54761 +54762 POINT(30.150244029602487 -94.71642125435503) bank54762 +54763 POINT(29.049869606507873 -95.64580803507441) bank54763 +54764 POINT(30.03039666816258 -96.08365806887899) bank54764 +54765 POINT(29.401802871558207 -95.34634407960209) bank54765 +54766 POINT(29.776732441573756 -95.65452201222706) bank54766 +54767 POINT(30.191884714931867 -94.88848298969732) bank54767 +54768 POINT(29.206118115996418 -95.7888004947865) bank54768 +54769 POINT(29.45426363601651 -96.21091719038515) bank54769 +54770 POINT(30.339061164065527 -95.97876727650232) bank54770 +54771 POINT(29.599461519518336 -95.2307721755972) bank54771 +54772 POINT(30.486649162038983 -95.73759203715626) bank54772 +54773 POINT(29.958821576084855 -95.1466666176826) bank54773 +54774 POINT(29.961981072316714 -95.61115561855091) bank54774 +54775 POINT(30.218038750617016 -94.91052462290466) bank54775 +54776 POINT(30.02470239251852 -95.7851480687626) bank54776 +54777 POINT(30.336759643582813 -94.75769479246613) bank54777 +54778 POINT(30.258712412533598 -95.46521340435893) bank54778 +54779 POINT(28.826955021221906 -94.61536355180311) bank54779 +54780 POINT(30.66602829913543 -95.44121621184304) bank54780 +54781 POINT(29.007791564467123 -96.299426930257) bank54781 +54782 POINT(29.183902263548685 -95.4521277331861) bank54782 +54783 POINT(29.722686964467023 -96.36460603574479) bank54783 +54784 POINT(28.974735572385434 -96.3560061201445) bank54784 +54785 POINT(29.93386388321628 -94.69987530837271) bank54785 +54786 POINT(30.743801868988182 -95.25268710710847) bank54786 +54787 POINT(29.12360325304245 -96.23390905919729) bank54787 +54788 POINT(30.183200345768874 -95.98898448768396) bank54788 +54789 POINT(29.24774548591421 -95.35704273854084) bank54789 +54790 POINT(29.383323081840036 -95.59123535946004) bank54790 +54791 POINT(30.197923497023762 -95.10448358864681) bank54791 +54792 POINT(30.6425538140284 -95.32278550462489) bank54792 +54793 POINT(28.81415915350104 -94.52627114551449) bank54793 +54794 POINT(28.82222113286687 -95.37882865344513) bank54794 +54795 POINT(29.105450182641952 -96.36275600696696) bank54795 +54796 POINT(30.300010712220285 -95.8215907030221) bank54796 +54797 POINT(28.96194159780349 -94.78160081817305) bank54797 +54798 POINT(29.716702307221873 -95.90922316779006) bank54798 +54799 POINT(29.655544696378 -95.61795642700403) bank54799 +54800 POINT(29.110803545684085 -94.61332664727043) bank54800 +54801 POINT(29.474325664296522 -94.6428567936436) bank54801 +54802 POINT(29.758214215541944 -94.37338565648345) bank54802 +54803 POINT(30.511483821108097 -96.12863965125024) bank54803 +54804 POINT(30.35085725740316 -95.71565553729579) bank54804 +54805 POINT(29.59625914880038 -95.0305798700081) bank54805 +54806 POINT(29.802470525580567 -95.87514316110439) bank54806 +54807 POINT(30.541362857041207 -95.96176907817157) bank54807 +54808 POINT(30.595849036109378 -94.52818982172775) bank54808 +54809 POINT(29.15207742862883 -95.48894205894042) bank54809 +54810 POINT(29.502944958192533 -96.02178807040025) bank54810 +54811 POINT(29.56375466687682 -94.38586836869972) bank54811 +54812 POINT(29.26427866724502 -94.90228554617711) bank54812 +54813 POINT(28.851836044017258 -95.77465328893966) bank54813 +54814 POINT(29.037017055457508 -95.19814649849785) bank54814 +54815 POINT(29.32446367058561 -95.68935512564913) bank54815 +54816 POINT(30.622006383399157 -95.74259633951048) bank54816 +54817 POINT(30.4970070875963 -94.38980334729547) bank54817 +54818 POINT(29.89858110133892 -94.88614633415361) bank54818 +54819 POINT(29.07464344689306 -95.878900316371) bank54819 +54820 POINT(28.90889909340585 -95.7828423464377) bank54820 +54821 POINT(30.00770571748693 -95.16186585980032) bank54821 +54822 POINT(30.21203946510955 -95.1288697156534) bank54822 +54823 POINT(30.126793427429206 -94.85720806665397) bank54823 +54824 POINT(29.232558455138577 -94.75542051792475) bank54824 +54825 POINT(29.223681453622724 -94.79403987249052) bank54825 +54826 POINT(29.769083812064693 -94.97426946985034) bank54826 +54827 POINT(30.632100732521646 -94.59111054530233) bank54827 +54828 POINT(30.647756237847823 -96.23993377976642) bank54828 +54829 POINT(29.669862865985614 -95.12021563598468) bank54829 +54830 POINT(30.438620186826796 -94.84568333593492) bank54830 +54831 POINT(30.582500661270345 -95.79425232932775) bank54831 +54832 POINT(30.576296417679885 -96.16782026274655) bank54832 +54833 POINT(29.63877793521216 -95.21837299498466) bank54833 +54834 POINT(29.812067386729233 -94.59990069268792) bank54834 +54835 POINT(30.75672923598378 -95.77368001036452) bank54835 +54836 POINT(28.81553446481005 -94.6855685350531) bank54836 +54837 POINT(28.930098905480676 -95.46238003595553) bank54837 +54838 POINT(28.85042169530464 -95.16272493405806) bank54838 +54839 POINT(29.101350230604304 -94.78431918962201) bank54839 +54840 POINT(29.311403180814594 -94.87212878450218) bank54840 +54841 POINT(30.219740417047465 -95.89501127939273) bank54841 +54842 POINT(30.760009572372713 -95.13654727939485) bank54842 +54843 POINT(29.74200391553051 -95.6287125093784) bank54843 +54844 POINT(30.195964279861737 -94.61312521462683) bank54844 +54845 POINT(29.609242689884496 -95.27392664730708) bank54845 +54846 POINT(30.13311989232028 -95.2628373715297) bank54846 +54847 POINT(30.456658987366442 -95.92065823950783) bank54847 +54848 POINT(30.623674291046136 -94.53948124899027) bank54848 +54849 POINT(29.455016497595565 -94.58656304179205) bank54849 +54850 POINT(30.11067155231996 -96.05863227188924) bank54850 +54851 POINT(30.05679279764774 -95.59825005434745) bank54851 +54852 POINT(29.30632042678319 -95.47570702976387) bank54852 +54853 POINT(30.223081611411864 -95.8316844170741) bank54853 +54854 POINT(28.946496090355588 -96.31133317972359) bank54854 +54855 POINT(30.689066871008105 -94.39627729742313) bank54855 +54856 POINT(30.30389912813424 -95.94029278311662) bank54856 +54857 POINT(29.69086353609908 -96.10813235467197) bank54857 +54858 POINT(28.868976221213515 -95.54852801748522) bank54858 +54859 POINT(29.11287230953878 -96.19037935829165) bank54859 +54860 POINT(29.293796295075808 -94.48391153998361) bank54860 +54861 POINT(29.05806225259268 -95.15213316122608) bank54861 +54862 POINT(30.688429386237434 -95.68480827277283) bank54862 +54863 POINT(29.032628560212594 -96.09041564035796) bank54863 +54864 POINT(28.902868939397507 -95.15938431138227) bank54864 +54865 POINT(29.003458345931065 -95.47229964149979) bank54865 +54866 POINT(29.05044184064501 -94.49701271709415) bank54866 +54867 POINT(30.36863404175296 -96.0674051320285) bank54867 +54868 POINT(29.773134256107365 -95.86001341108883) bank54868 +54869 POINT(30.728752289377635 -96.13192422025165) bank54869 +54870 POINT(29.808633788405846 -96.30735816852184) bank54870 +54871 POINT(29.623655288575478 -95.41649609901621) bank54871 +54872 POINT(30.309875045999323 -95.05492485395246) bank54872 +54873 POINT(28.871476525725882 -95.67778039714887) bank54873 +54874 POINT(29.925858075791893 -96.36516232249481) bank54874 +54875 POINT(29.16744729773324 -96.09792243878965) bank54875 +54876 POINT(29.205676600192408 -95.9894442058837) bank54876 +54877 POINT(30.31666739560021 -95.60119884840107) bank54877 +54878 POINT(29.94415656112812 -95.85915944636722) bank54878 +54879 POINT(29.402530383146946 -95.16100611440503) bank54879 +54880 POINT(30.72066472320447 -94.38396737353693) bank54880 +54881 POINT(29.741610604889058 -94.98115673960682) bank54881 +54882 POINT(29.68549601045447 -94.93344048606464) bank54882 +54883 POINT(29.060225720315636 -95.75725694553768) bank54883 +54884 POINT(29.268525084862677 -95.54799884314342) bank54884 +54885 POINT(29.0083236001637 -94.78160640069625) bank54885 +54886 POINT(29.205889168639803 -95.52129050473047) bank54886 +54887 POINT(30.415588886770372 -95.62744759377053) bank54887 +54888 POINT(29.743825701919356 -95.0035362914824) bank54888 +54889 POINT(29.08219704234424 -95.05600139672315) bank54889 +54890 POINT(30.26454474481534 -94.87368881953796) bank54890 +54891 POINT(30.326372543183044 -94.92997848317671) bank54891 +54892 POINT(29.011259043357562 -94.82503295696843) bank54892 +54893 POINT(29.177062266882626 -94.64705587740902) bank54893 +54894 POINT(30.622772592317556 -96.2741297201074) bank54894 +54895 POINT(30.00279792255323 -95.55169843748718) bank54895 +54896 POINT(28.85473779479256 -95.28039692690632) bank54896 +54897 POINT(29.485042139473908 -95.33066072475695) bank54897 +54898 POINT(29.847474343418362 -94.67757114513489) bank54898 +54899 POINT(29.146182445811704 -95.09878228584208) bank54899 +54900 POINT(29.527309430609524 -95.94475713842692) bank54900 +54901 POINT(28.961340724833285 -95.90529749830573) bank54901 +54902 POINT(29.409537055082325 -95.41626931545358) bank54902 +54903 POINT(30.412433215697263 -95.06526584508075) bank54903 +54904 POINT(29.934786091162955 -94.88908286248704) bank54904 +54905 POINT(30.411421247219337 -95.46544118542184) bank54905 +54906 POINT(30.760091317875812 -95.11953222393471) bank54906 +54907 POINT(30.719297805722533 -95.73344041222813) bank54907 +54908 POINT(30.258731419235886 -96.04718215749502) bank54908 +54909 POINT(30.299317411137938 -95.8221420247423) bank54909 +54910 POINT(29.769102215026304 -95.57377396487907) bank54910 +54911 POINT(29.590395226674257 -94.49447816034065) bank54911 +54912 POINT(30.661374329778134 -94.68287951529899) bank54912 +54913 POINT(29.937966278848386 -95.56210038918437) bank54913 +54914 POINT(30.225932387775078 -95.88141279903017) bank54914 +54915 POINT(30.70358743313965 -96.08446310856912) bank54915 +54916 POINT(30.42528373441005 -94.45387629002386) bank54916 +54917 POINT(30.15014403326788 -94.46064356354742) bank54917 +54918 POINT(29.56109592970667 -96.35327567062497) bank54918 +54919 POINT(30.693896097839158 -94.39493685065828) bank54919 +54920 POINT(29.42599022738048 -94.89622864186268) bank54920 +54921 POINT(30.716536243583914 -95.43755245471091) bank54921 +54922 POINT(30.36123114205636 -95.5241752241298) bank54922 +54923 POINT(29.426582708529136 -94.74186415268089) bank54923 +54924 POINT(29.896170412693905 -95.7028953084105) bank54924 +54925 POINT(29.41405510223152 -95.37219963882646) bank54925 +54926 POINT(30.281529913426194 -95.22230194267489) bank54926 +54927 POINT(29.356063948541333 -95.1531739112527) bank54927 +54928 POINT(30.37311971700173 -95.36823860233184) bank54928 +54929 POINT(29.907393833951815 -95.14281467762197) bank54929 +54930 POINT(29.28522305447205 -95.90578342885385) bank54930 +54931 POINT(29.31814287590897 -95.5327337202751) bank54931 +54932 POINT(30.32105392357901 -96.36790347536947) bank54932 +54933 POINT(29.022339178324284 -95.24565331356496) bank54933 +54934 POINT(30.495042192338428 -95.3924891156332) bank54934 +54935 POINT(29.38933922041358 -95.22664442130846) bank54935 +54936 POINT(29.29125320438892 -95.61746966804735) bank54936 +54937 POINT(30.535360992118157 -95.88015005886466) bank54937 +54938 POINT(30.465268622249408 -95.96801096280241) bank54938 +54939 POINT(29.080096992476268 -95.6184675243773) bank54939 +54940 POINT(29.302936473087353 -96.155929812211) bank54940 +54941 POINT(30.05245256337759 -94.98334404363523) bank54941 +54942 POINT(29.217134278937717 -94.91134859661473) bank54942 +54943 POINT(29.6644364871453 -94.9668030711181) bank54943 +54944 POINT(30.571428745122066 -95.71617614962977) bank54944 +54945 POINT(30.142608747357453 -94.8233619118066) bank54945 +54946 POINT(29.073237369211775 -94.53360082620028) bank54946 +54947 POINT(30.15485659972376 -94.59459685542069) bank54947 +54948 POINT(29.630692775131198 -95.2820857405199) bank54948 +54949 POINT(29.67647955060263 -94.93905316061048) bank54949 +54950 POINT(30.366252070325103 -95.40131064315304) bank54950 +54951 POINT(30.221364417413586 -96.11582024886567) bank54951 +54952 POINT(30.24979537887714 -95.53804629464965) bank54952 +54953 POINT(29.29032361223401 -96.09728522476112) bank54953 +54954 POINT(29.011097472906187 -95.28573332785692) bank54954 +54955 POINT(28.831173723497784 -95.4335373339306) bank54955 +54956 POINT(30.747584867467193 -96.1217209006619) bank54956 +54957 POINT(30.5770780867789 -95.1188143184673) bank54957 +54958 POINT(29.31276568077424 -96.27604357881742) bank54958 +54959 POINT(29.512288211099744 -96.22458807341) bank54959 +54960 POINT(29.88750536233749 -95.77529687782686) bank54960 +54961 POINT(29.63349087849922 -94.79912182209301) bank54961 +54962 POINT(30.10324029263271 -95.41440570074703) bank54962 +54963 POINT(29.245304503785576 -95.06071710561746) bank54963 +54964 POINT(29.57439027373358 -94.40561934645268) bank54964 +54965 POINT(29.124796874345602 -95.40682853929582) bank54965 +54966 POINT(29.229709870898166 -95.59661448855933) bank54966 +54967 POINT(29.779950823722547 -94.89730583077929) bank54967 +54968 POINT(29.98862866589604 -94.74290070566387) bank54968 +54969 POINT(30.113311074225535 -95.73832573976314) bank54969 +54970 POINT(29.89606191250428 -95.88441643787836) bank54970 +54971 POINT(30.69343692245283 -96.1298782161386) bank54971 +54972 POINT(29.978808210413295 -95.6711015597064) bank54972 +54973 POINT(29.446136999006534 -95.56097205250579) bank54973 +54974 POINT(29.019635675426496 -96.15348819685757) bank54974 +54975 POINT(28.859568111468008 -95.1343243921608) bank54975 +54976 POINT(29.53974412914775 -94.58023211148857) bank54976 +54977 POINT(29.46699964057214 -94.96842239176509) bank54977 +54978 POINT(29.368998218139485 -95.66803418278887) bank54978 +54979 POINT(30.33034581332675 -95.24972343012116) bank54979 +54980 POINT(29.020322773043286 -95.49051049202254) bank54980 +54981 POINT(30.039223123234247 -94.51245699404501) bank54981 +54982 POINT(30.047346659633693 -95.93948035661826) bank54982 +54983 POINT(30.191520088452876 -95.5367008081836) bank54983 +54984 POINT(30.077977888552013 -96.11002383983426) bank54984 +54985 POINT(29.33888894070181 -96.154696167131) bank54985 +54986 POINT(30.282014325828225 -95.99753998172724) bank54986 +54987 POINT(29.098427092522282 -95.76015822638108) bank54987 +54988 POINT(29.241963631677365 -95.3700637372775) bank54988 +54989 POINT(30.380750715726187 -95.16315453300217) bank54989 +54990 POINT(29.35200937437341 -96.27747336214325) bank54990 +54991 POINT(30.394284772772703 -94.39255090968594) bank54991 +54992 POINT(28.77534507267023 -95.45142335103306) bank54992 +54993 POINT(29.087055822026645 -94.60290801796947) bank54993 +54994 POINT(28.815509917835765 -94.56827516349425) bank54994 +54995 POINT(28.87458238303781 -96.31191983117847) bank54995 +54996 POINT(30.363324426146526 -96.35576036501773) bank54996 +54997 POINT(29.02781373768839 -95.05553378159951) bank54997 +54998 POINT(29.150160375821233 -95.4809918774578) bank54998 +54999 POINT(30.643779029169686 -96.17834090964716) bank54999 +55000 POINT(30.519925281496533 -95.24909800887168) bank55000 +55001 POINT(29.195476336033682 -94.4551635796796) bank55001 +55002 POINT(30.526990810727856 -96.31439359026841) bank55002 +55003 POINT(29.14727580397232 -95.62899668308316) bank55003 +55004 POINT(30.128645261280198 -95.30928368600676) bank55004 +55005 POINT(29.80151071331894 -94.4603915945811) bank55005 +55006 POINT(29.94211029767452 -95.75925755402555) bank55006 +55007 POINT(30.547584189827408 -96.31097576092864) bank55007 +55008 POINT(30.63362477243765 -96.07767488192268) bank55008 +55009 POINT(30.64311094192876 -95.41817928937144) bank55009 +55010 POINT(29.167036938254483 -94.61296572756491) bank55010 +55011 POINT(30.59836051064656 -95.98784084189687) bank55011 +55012 POINT(29.995293705604308 -94.81054476281655) bank55012 +55013 POINT(30.65838499756934 -95.12723341177733) bank55013 +55014 POINT(29.57351440350195 -94.72799394009907) bank55014 +55015 POINT(30.469298933669013 -95.80578403664752) bank55015 +55016 POINT(30.072471022288468 -95.69157074816877) bank55016 +55017 POINT(30.726888618811213 -94.85276439849264) bank55017 +55018 POINT(30.496213199299206 -96.1600166252113) bank55018 +55019 POINT(30.512317199117604 -94.65879427514253) bank55019 +55020 POINT(30.65182361007127 -95.68419543846174) bank55020 +55021 POINT(30.426370932566822 -95.10925599406458) bank55021 +55022 POINT(29.99579516965144 -95.98369416743202) bank55022 +55023 POINT(29.07139437706477 -96.00625141924458) bank55023 +55024 POINT(29.582135282202646 -94.78347433584298) bank55024 +55025 POINT(30.62247929407574 -94.50024999314346) bank55025 +55026 POINT(30.2712283761082 -94.51903622391926) bank55026 +55027 POINT(29.29790236203294 -94.70471101740992) bank55027 +55028 POINT(29.694659410506443 -94.49192393531877) bank55028 +55029 POINT(30.211414562334486 -96.31006474380837) bank55029 +55030 POINT(29.99005387680534 -96.18952122684372) bank55030 +55031 POINT(29.27932846227573 -94.61446857639667) bank55031 +55032 POINT(29.58918572255167 -96.1991334093438) bank55032 +55033 POINT(29.656819315497515 -96.2543831146649) bank55033 +55034 POINT(29.802285762237215 -95.93333588189468) bank55034 +55035 POINT(30.28319186410751 -94.99044927921982) bank55035 +55036 POINT(29.392777992604778 -94.69835194445712) bank55036 +55037 POINT(29.171522442280946 -95.23278694487885) bank55037 +55038 POINT(29.155783048523553 -94.37690996475924) bank55038 +55039 POINT(29.90753116252521 -94.92707746461477) bank55039 +55040 POINT(30.388749077244405 -94.54586031185671) bank55040 +55041 POINT(29.123366771169284 -95.23468720843232) bank55041 +55042 POINT(29.411183849998874 -94.7549436073521) bank55042 +55043 POINT(29.828335425242347 -95.65287148581773) bank55043 +55044 POINT(28.891693299549964 -95.8811298250797) bank55044 +55045 POINT(30.251741638599057 -96.23010813880221) bank55045 +55046 POINT(29.180197074887257 -95.78907602669929) bank55046 +55047 POINT(30.425863701165284 -94.48860893314225) bank55047 +55048 POINT(29.778737338968156 -96.05461563259782) bank55048 +55049 POINT(29.63987255320521 -95.08769623708832) bank55049 +55050 POINT(29.50891116796328 -95.12970884386796) bank55050 +55051 POINT(29.547111003350285 -94.56831205354311) bank55051 +55052 POINT(30.45637354659913 -94.7275972234813) bank55052 +55053 POINT(30.246258553927493 -94.98265494322145) bank55053 +55054 POINT(30.122175640450266 -94.69537270562815) bank55054 +55055 POINT(29.947690532314233 -96.20145633639929) bank55055 +55056 POINT(30.730821648792706 -94.62302576015286) bank55056 +55057 POINT(30.4111335917606 -95.66521718839351) bank55057 +55058 POINT(30.32297439639927 -95.8344367169253) bank55058 +55059 POINT(30.331157241325457 -95.27565450951845) bank55059 +55060 POINT(29.34728580849555 -94.93272972042591) bank55060 +55061 POINT(28.775584499779406 -94.57518407661632) bank55061 +55062 POINT(30.333737520766512 -94.82231557239261) bank55062 +55063 POINT(30.39387016171993 -94.46824659877386) bank55063 +55064 POINT(29.21581741928642 -95.36227548567088) bank55064 +55065 POINT(30.270909983096455 -95.46970936651921) bank55065 +55066 POINT(29.11410494367928 -94.82737066931266) bank55066 +55067 POINT(30.06364289318314 -95.85534420348638) bank55067 +55068 POINT(30.497532786672153 -94.88556418858889) bank55068 +55069 POINT(30.24194558247943 -94.50452041353485) bank55069 +55070 POINT(29.845027452761567 -94.63238929104519) bank55070 +55071 POINT(29.13080319759771 -96.18712614966664) bank55071 +55072 POINT(29.629862673886013 -94.4897686547247) bank55072 +55073 POINT(28.787026888969713 -94.5522737773634) bank55073 +55074 POINT(29.3712653574348 -95.06985536750427) bank55074 +55075 POINT(29.817621306009627 -95.07833748048172) bank55075 +55076 POINT(29.444467658742074 -96.00176365448256) bank55076 +55077 POINT(29.90835661327551 -96.33940165595149) bank55077 +55078 POINT(30.45309913504779 -95.55259475558705) bank55078 +55079 POINT(28.771410889176856 -95.82824741866152) bank55079 +55080 POINT(29.69798061231094 -94.37009405488914) bank55080 +55081 POINT(29.510120757166973 -96.26828823332646) bank55081 +55082 POINT(28.919846924329676 -94.43608621596104) bank55082 +55083 POINT(28.762171986252223 -96.1322030299712) bank55083 +55084 POINT(30.755843207285842 -94.37102378480978) bank55084 +55085 POINT(29.43347999477535 -96.30063569032265) bank55085 +55086 POINT(28.95820369364896 -95.792170852466) bank55086 +55087 POINT(28.98125574973272 -94.4375511279532) bank55087 +55088 POINT(29.114539358354776 -95.26938542395618) bank55088 +55089 POINT(29.825870114897253 -94.72555636309646) bank55089 +55090 POINT(29.35768714271266 -96.33525462313496) bank55090 +55091 POINT(29.05197772390378 -95.42757031756474) bank55091 +55092 POINT(29.200052494875724 -96.13258088714241) bank55092 +55093 POINT(29.528035360693313 -95.02388995948152) bank55093 +55094 POINT(30.72171519270724 -95.48559074801192) bank55094 +55095 POINT(29.5264086497354 -95.26409567983323) bank55095 +55096 POINT(29.17210866951515 -95.13351825700359) bank55096 +55097 POINT(30.624057616789898 -95.04293750468983) bank55097 +55098 POINT(29.922823084538535 -94.98201018213204) bank55098 +55099 POINT(29.30424344122461 -94.71261115309034) bank55099 +55100 POINT(28.772495917121393 -95.27906922395361) bank55100 +55101 POINT(29.315894825834576 -95.01709330196061) bank55101 +55102 POINT(29.785314116610103 -95.90247807740842) bank55102 +55103 POINT(29.118021182910546 -94.81047602631195) bank55103 +55104 POINT(29.70817346744603 -95.65835296951207) bank55104 +55105 POINT(30.08607837075344 -94.61386707651752) bank55105 +55106 POINT(29.695637638199205 -95.49480996459044) bank55106 +55107 POINT(29.796753677091395 -95.09999854744241) bank55107 +55108 POINT(30.57719190401837 -95.90531454301089) bank55108 +55109 POINT(29.416676167949696 -95.13779084087969) bank55109 +55110 POINT(28.82064360672457 -94.59888113525216) bank55110 +55111 POINT(30.041509379096823 -95.7698024961798) bank55111 +55112 POINT(30.447567026423666 -94.90763453521927) bank55112 +55113 POINT(30.45483791882791 -94.49812498514873) bank55113 +55114 POINT(30.24093081645233 -95.42965554673567) bank55114 +55115 POINT(29.945682308101595 -96.03460662475109) bank55115 +55116 POINT(30.27757161070525 -96.0354062814692) bank55116 +55117 POINT(29.854322540529868 -96.18961077378111) bank55117 +55118 POINT(30.14760274687404 -94.47943739908918) bank55118 +55119 POINT(30.053236547053622 -95.18266752828451) bank55119 +55120 POINT(29.731673193791007 -94.57693946879908) bank55120 +55121 POINT(28.766852157733698 -95.31218467100175) bank55121 +55122 POINT(28.9880430694369 -95.86280049981555) bank55122 +55123 POINT(29.499478605603326 -96.10178847719378) bank55123 +55124 POINT(28.808654566519422 -96.14413923175229) bank55124 +55125 POINT(29.732902718870534 -96.05417765310528) bank55125 +55126 POINT(28.97527265013262 -96.21784193101571) bank55126 +55127 POINT(30.124905401844863 -94.42546368729884) bank55127 +55128 POINT(29.44635426394044 -96.31208880861763) bank55128 +55129 POINT(29.753948785862345 -95.02285150080277) bank55129 +55130 POINT(29.750351094913462 -96.3418903541685) bank55130 +55131 POINT(29.9280646318338 -95.82606723713207) bank55131 +55132 POINT(29.20070681248138 -94.54842794942998) bank55132 +55133 POINT(30.12002951642267 -95.33148693084871) bank55133 +55134 POINT(30.514511314034014 -95.05823149066977) bank55134 +55135 POINT(29.748173826913195 -95.17483074402774) bank55135 +55136 POINT(28.85480655060362 -95.99395716264266) bank55136 +55137 POINT(29.758778887392552 -96.20516540776357) bank55137 +55138 POINT(29.821079241608874 -96.0569855639214) bank55138 +55139 POINT(28.82022494958977 -94.85031760768294) bank55139 +55140 POINT(29.402829523725856 -94.76034557749145) bank55140 +55141 POINT(29.728817091075708 -96.30934483923689) bank55141 +55142 POINT(29.526004049358974 -96.24293537289657) bank55142 +55143 POINT(30.579384036631108 -95.08725966170726) bank55143 +55144 POINT(30.02684789042773 -96.34088374541315) bank55144 +55145 POINT(28.833478987988634 -95.46255222914677) bank55145 +55146 POINT(30.268513835779718 -94.6540222646415) bank55146 +55147 POINT(30.57245392932245 -95.40102456002057) bank55147 +55148 POINT(30.000889567682638 -95.72003905229624) bank55148 +55149 POINT(29.99204484948732 -95.02113324074234) bank55149 +55150 POINT(29.8610098057894 -96.32850568748441) bank55150 +55151 POINT(30.75548622084044 -95.29843856298945) bank55151 +55152 POINT(29.529847912361003 -96.03229257579694) bank55152 +55153 POINT(29.387420763192743 -95.4067334969266) bank55153 +55154 POINT(30.319301981277754 -94.75454012306324) bank55154 +55155 POINT(29.631278228980086 -95.78341899738555) bank55155 +55156 POINT(29.95616782070372 -95.53376267695255) bank55156 +55157 POINT(29.194891264832783 -94.61109903515042) bank55157 +55158 POINT(30.07695509734063 -95.7728632034766) bank55158 +55159 POINT(29.392018873246897 -96.20337660320664) bank55159 +55160 POINT(29.188670036495946 -94.6787085815241) bank55160 +55161 POINT(28.8663093875387 -95.86970916824262) bank55161 +55162 POINT(29.236067093365786 -95.12514221044472) bank55162 +55163 POINT(28.94239398709896 -95.47760916422202) bank55163 +55164 POINT(29.72632894915283 -95.21627330280654) bank55164 +55165 POINT(29.494178138235416 -95.8596864493868) bank55165 +55166 POINT(30.217607987691302 -96.28569895086035) bank55166 +55167 POINT(30.10546051068568 -95.01796298808796) bank55167 +55168 POINT(29.470920005826514 -94.76045711310907) bank55168 +55169 POINT(30.24327872058033 -95.93479467974349) bank55169 +55170 POINT(29.919199481099756 -94.82697597101048) bank55170 +55171 POINT(29.901739384736157 -94.40450979753949) bank55171 +55172 POINT(29.432143754530028 -95.69089764908601) bank55172 +55173 POINT(29.88051935189464 -95.96097456358503) bank55173 +55174 POINT(29.396872994771943 -94.44779653656123) bank55174 +55175 POINT(29.681318594423402 -96.35237585479108) bank55175 +55176 POINT(29.43507887418429 -96.05398766440575) bank55176 +55177 POINT(29.696815593085276 -95.7184068199552) bank55177 +55178 POINT(30.674572654406486 -94.57899452140786) bank55178 +55179 POINT(29.26350860049272 -96.28371177492122) bank55179 +55180 POINT(28.91785012299807 -95.17003157299152) bank55180 +55181 POINT(29.657810889527624 -95.69366615465475) bank55181 +55182 POINT(29.543948087697157 -94.70066020849283) bank55182 +55183 POINT(29.001183135475124 -95.3701772194286) bank55183 +55184 POINT(29.58023038654218 -94.7215369022083) bank55184 +55185 POINT(30.03584932544007 -94.82023179958672) bank55185 +55186 POINT(29.089336518162796 -94.39794675537617) bank55186 +55187 POINT(29.772479334525027 -96.31448100994709) bank55187 +55188 POINT(28.949232667390714 -95.76707644982733) bank55188 +55189 POINT(29.352293718204233 -95.89266249058053) bank55189 +55190 POINT(29.845978959624162 -95.44343101638157) bank55190 +55191 POINT(29.419966577804747 -94.89722356526416) bank55191 +55192 POINT(29.35137582612334 -95.94232938865113) bank55192 +55193 POINT(30.562462442020937 -95.60186215736977) bank55193 +55194 POINT(29.368143307214662 -94.80519351596055) bank55194 +55195 POINT(30.060389714354173 -95.11550466570766) bank55195 +55196 POINT(29.040206764297366 -95.11234526837107) bank55196 +55197 POINT(30.30322591438934 -95.12974000311934) bank55197 +55198 POINT(30.1771280762335 -95.9277226261859) bank55198 +55199 POINT(30.192209718528307 -94.44328947602588) bank55199 +55200 POINT(28.96405725593011 -94.78812608658279) bank55200 +55201 POINT(29.784008088570026 -96.00505316864488) bank55201 +55202 POINT(29.801164102727803 -96.14866758498891) bank55202 +55203 POINT(30.679630299192322 -95.06237198464792) bank55203 +55204 POINT(28.90813795984186 -96.18900012934185) bank55204 +55205 POINT(29.8858113124588 -95.25971089628699) bank55205 +55206 POINT(30.75919187125745 -94.91878839450628) bank55206 +55207 POINT(29.969284537621753 -95.09642369112782) bank55207 +55208 POINT(29.848873988577527 -94.7378042498569) bank55208 +55209 POINT(29.610366145047937 -94.44524057261958) bank55209 +55210 POINT(29.80942797647508 -94.79283872881805) bank55210 +55211 POINT(29.00668956773108 -94.77200413411566) bank55211 +55212 POINT(28.771832532827677 -94.91716938119373) bank55212 +55213 POINT(29.03819462045986 -94.95155306375496) bank55213 +55214 POINT(29.108466268263196 -94.4560662058634) bank55214 +55215 POINT(30.622486753591208 -95.42529255669378) bank55215 +55216 POINT(29.405560744443804 -95.23902434023645) bank55216 +55217 POINT(30.690519443268826 -96.17825415908867) bank55217 +55218 POINT(29.45384690810422 -95.7861389584872) bank55218 +55219 POINT(29.347293452016782 -95.70752141187502) bank55219 +55220 POINT(29.17049527314965 -95.21449945089849) bank55220 +55221 POINT(28.95928351749125 -95.36824498169591) bank55221 +55222 POINT(29.339656514406638 -94.93371233891956) bank55222 +55223 POINT(29.402731272704223 -94.84995772138802) bank55223 +55224 POINT(29.971968320978107 -95.53073254965538) bank55224 +55225 POINT(30.515311011126354 -95.08514989950547) bank55225 +55226 POINT(29.926527402257644 -95.22812878418792) bank55226 +55227 POINT(29.24089429593551 -95.6615519207229) bank55227 +55228 POINT(29.621165518333047 -95.41576906816032) bank55228 +55229 POINT(28.806377402699102 -94.46893176370017) bank55229 +55230 POINT(30.723704043311574 -95.15335803824485) bank55230 +55231 POINT(29.86266625974728 -94.78493771254344) bank55231 +55232 POINT(30.11649520695894 -94.9797435296906) bank55232 +55233 POINT(30.702647597576433 -94.50203218635943) bank55233 +55234 POINT(29.47019428511823 -95.95678638901042) bank55234 +55235 POINT(30.069730153395117 -95.54903370429649) bank55235 +55236 POINT(28.88222949622118 -96.1747736871822) bank55236 +55237 POINT(29.656715324362256 -95.6583246040303) bank55237 +55238 POINT(29.5314919805511 -95.14049275352721) bank55238 +55239 POINT(29.016827232448804 -95.99751573795928) bank55239 +55240 POINT(30.60629323732099 -94.50622440840776) bank55240 +55241 POINT(28.91636087152261 -95.557985465263) bank55241 +55242 POINT(30.157840610514086 -95.90957772796307) bank55242 +55243 POINT(29.027202546193244 -94.42554934518033) bank55243 +55244 POINT(29.128689492492494 -95.64917543081602) bank55244 +55245 POINT(29.97642524032797 -94.54504283687906) bank55245 +55246 POINT(29.413747903468778 -94.39672139715775) bank55246 +55247 POINT(28.914742215358913 -95.80898882051702) bank55247 +55248 POINT(30.11440069696548 -96.32481916680469) bank55248 +55249 POINT(30.213631766662402 -95.069660421721) bank55249 +55250 POINT(29.720954046970363 -94.4944196716893) bank55250 +55251 POINT(29.263577388061368 -95.36767160164078) bank55251 +55252 POINT(29.761993248049578 -95.9322027796037) bank55252 +55253 POINT(30.455710872265584 -94.5202583237285) bank55253 +55254 POINT(30.479965210487375 -95.11589749981525) bank55254 +55255 POINT(29.05051501548179 -95.19104734034389) bank55255 +55256 POINT(29.618291630778455 -95.05026676472933) bank55256 +55257 POINT(28.813203555559795 -94.80066366601332) bank55257 +55258 POINT(29.139653700593243 -94.79148457935067) bank55258 +55259 POINT(28.956918568330245 -95.80405658112917) bank55259 +55260 POINT(29.01463310961157 -94.8964759407625) bank55260 +55261 POINT(28.776166452932728 -95.90384061911656) bank55261 +55262 POINT(30.528532264533766 -94.77240675117663) bank55262 +55263 POINT(29.3122173989988 -95.8186732710437) bank55263 +55264 POINT(28.852618483709495 -96.35389432396266) bank55264 +55265 POINT(29.49248632530188 -94.7017769787638) bank55265 +55266 POINT(29.687388329548 -95.42650283721586) bank55266 +55267 POINT(30.004058935077605 -95.17416577942384) bank55267 +55268 POINT(30.190728888197793 -95.77954128297203) bank55268 +55269 POINT(30.067175368736965 -96.14976148082376) bank55269 +55270 POINT(29.27198448653793 -94.79483658034519) bank55270 +55271 POINT(29.282483809567143 -95.86784902631354) bank55271 +55272 POINT(29.72333678792461 -94.62585732045474) bank55272 +55273 POINT(29.34296461242285 -95.10504698653256) bank55273 +55274 POINT(29.77880042771254 -96.36954830333434) bank55274 +55275 POINT(30.6568038990903 -94.91579191712427) bank55275 +55276 POINT(30.494080266343552 -95.22458206072626) bank55276 +55277 POINT(30.039086890146123 -94.50105926512995) bank55277 +55278 POINT(29.607486796674223 -95.95388119982604) bank55278 +55279 POINT(30.270063021410035 -95.80217879916613) bank55279 +55280 POINT(30.532784495654962 -94.65123470057345) bank55280 +55281 POINT(30.233185066802232 -95.56293532291822) bank55281 +55282 POINT(28.805850067075554 -94.69072842248248) bank55282 +55283 POINT(30.33342864404395 -94.56547955254266) bank55283 +55284 POINT(30.27182741557898 -95.46090859441351) bank55284 +55285 POINT(30.658992670993975 -95.52324083006131) bank55285 +55286 POINT(29.95831879594061 -95.16262519814461) bank55286 +55287 POINT(30.0640297513406 -96.2190799553436) bank55287 +55288 POINT(28.890831569394596 -95.40250466697118) bank55288 +55289 POINT(30.714206587004966 -95.16663756017816) bank55289 +55290 POINT(29.04809041966122 -94.74030684142406) bank55290 +55291 POINT(30.31442724585384 -96.0325897800581) bank55291 +55292 POINT(30.183701153723934 -95.25542069601029) bank55292 +55293 POINT(30.439128399012873 -95.65378979903642) bank55293 +55294 POINT(30.611097265133512 -95.10276297625245) bank55294 +55295 POINT(29.165448728936923 -94.65843782096971) bank55295 +55296 POINT(29.875094093293757 -96.0980400844345) bank55296 +55297 POINT(30.08428959622714 -94.93970638577055) bank55297 +55298 POINT(30.503344838231882 -94.58219052683894) bank55298 +55299 POINT(29.521946439136066 -95.06752815431935) bank55299 +55300 POINT(29.950623347730726 -94.88450252726912) bank55300 +55301 POINT(30.00007976949391 -94.44893586752666) bank55301 +55302 POINT(29.2243526352796 -95.97363728701507) bank55302 +55303 POINT(29.40980003500247 -95.42650851172498) bank55303 +55304 POINT(30.163864654889075 -96.02987924820488) bank55304 +55305 POINT(29.98738283359725 -94.7187979270359) bank55305 +55306 POINT(29.026316955726568 -95.95666141254416) bank55306 +55307 POINT(29.95554545433667 -95.01110368451825) bank55307 +55308 POINT(30.277789744196962 -94.91239035178704) bank55308 +55309 POINT(30.287576246933206 -95.7268144063901) bank55309 +55310 POINT(30.403316410664722 -94.45061394227648) bank55310 +55311 POINT(30.080459065787903 -95.05276524507526) bank55311 +55312 POINT(30.09594989882478 -94.47563664547488) bank55312 +55313 POINT(29.65516679227733 -94.40271396150305) bank55313 +55314 POINT(29.11667098555123 -95.9551350838437) bank55314 +55315 POINT(30.313181354451967 -95.29467547997267) bank55315 +55316 POINT(28.79649266399804 -96.23964334663016) bank55316 +55317 POINT(28.929528392349226 -95.56504122009753) bank55317 +55318 POINT(29.65626383849421 -96.16925996752131) bank55318 +55319 POINT(29.48007776217928 -94.70696575479897) bank55319 +55320 POINT(29.452815652898547 -95.44411304274539) bank55320 +55321 POINT(30.75345123140455 -94.86395828119983) bank55321 +55322 POINT(30.287447391035666 -94.47384733963656) bank55322 +55323 POINT(29.496309008036718 -96.11347455964754) bank55323 +55324 POINT(28.87050202740692 -95.7772956444761) bank55324 +55325 POINT(30.117862538551396 -96.0736726803933) bank55325 +55326 POINT(29.51807535157328 -95.61484840541685) bank55326 +55327 POINT(29.3123667522928 -94.63745707147766) bank55327 +55328 POINT(29.58462639677156 -94.83119778414776) bank55328 +55329 POINT(28.83602301501502 -96.09159646884443) bank55329 +55330 POINT(30.547244061337306 -94.49672085032914) bank55330 +55331 POINT(29.730607799011104 -95.16396811552215) bank55331 +55332 POINT(30.587334605851417 -96.04881913031554) bank55332 +55333 POINT(28.79328192904368 -96.02598854401381) bank55333 +55334 POINT(30.49118502526446 -95.92986971646147) bank55334 +55335 POINT(30.215014106923437 -95.24892603712397) bank55335 +55336 POINT(30.28140480567145 -95.90307624921334) bank55336 +55337 POINT(30.155432670786563 -94.84426100882114) bank55337 +55338 POINT(30.677722205939315 -94.82955201010051) bank55338 +55339 POINT(28.95236249960687 -96.03664209066186) bank55339 +55340 POINT(30.26510024411203 -95.4544184603516) bank55340 +55341 POINT(28.84947538388212 -95.20798151029163) bank55341 +55342 POINT(29.317146827480336 -95.83263543720857) bank55342 +55343 POINT(29.04910390380571 -95.30060326523436) bank55343 +55344 POINT(29.25329608241247 -96.11126255446487) bank55344 +55345 POINT(28.908258011454926 -95.26366154206238) bank55345 +55346 POINT(30.30602541418388 -95.51379478763387) bank55346 +55347 POINT(28.809771088314466 -96.22988654301193) bank55347 +55348 POINT(28.81559501076358 -96.05483063495736) bank55348 +55349 POINT(30.220697210251295 -94.67429500647145) bank55349 +55350 POINT(29.077046442753037 -95.64966491358231) bank55350 +55351 POINT(28.784767370432927 -95.45794825430737) bank55351 +55352 POINT(30.4166364196166 -94.48779597047596) bank55352 +55353 POINT(29.743967538161666 -94.41362152691964) bank55353 +55354 POINT(28.837845506225797 -95.87441544597637) bank55354 +55355 POINT(30.501607272423847 -95.79030488325552) bank55355 +55356 POINT(29.54850607001579 -95.76323739402225) bank55356 +55357 POINT(30.510504146237288 -96.36571945219085) bank55357 +55358 POINT(29.13331447922554 -96.24641764173634) bank55358 +55359 POINT(30.732456660063054 -94.68401723731924) bank55359 +55360 POINT(29.93755921308631 -95.72567693542038) bank55360 +55361 POINT(30.3388285999813 -94.43338717677194) bank55361 +55362 POINT(30.542872197652123 -95.45235202959593) bank55362 +55363 POINT(30.1360784923882 -94.49265917122364) bank55363 +55364 POINT(28.83941480076299 -95.18368655941843) bank55364 +55365 POINT(29.1654274997919 -94.71236567946713) bank55365 +55366 POINT(30.142820510574545 -95.19193935376435) bank55366 +55367 POINT(29.45067579002444 -96.21983095359398) bank55367 +55368 POINT(29.986022565859706 -94.6541306855668) bank55368 +55369 POINT(29.105638633748885 -94.86124620016982) bank55369 +55370 POINT(30.22364265527103 -96.29401098196955) bank55370 +55371 POINT(28.943183025663338 -94.57878293488795) bank55371 +55372 POINT(29.79291716233734 -96.07884608704487) bank55372 +55373 POINT(30.439502353369058 -95.59396146934947) bank55373 +55374 POINT(30.31021505849208 -94.51159448826382) bank55374 +55375 POINT(29.786093989353056 -94.96935708712321) bank55375 +55376 POINT(29.73388649153108 -95.43137094147691) bank55376 +55377 POINT(29.97350128766786 -95.89236678699326) bank55377 +55378 POINT(28.97609605911166 -95.88226259660186) bank55378 +55379 POINT(29.87746190741124 -94.9991218956829) bank55379 +55380 POINT(30.453090158207083 -94.63142120738728) bank55380 +55381 POINT(30.436205225933975 -94.73420221829132) bank55381 +55382 POINT(29.569766104575542 -95.5148864158934) bank55382 +55383 POINT(30.18276932766265 -94.63652880116949) bank55383 +55384 POINT(28.837608417603573 -94.38343611366679) bank55384 +55385 POINT(30.3519201232046 -95.89091274451528) bank55385 +55386 POINT(29.881978625463635 -95.22705280968289) bank55386 +55387 POINT(28.868185163805084 -94.86409198112383) bank55387 +55388 POINT(30.607206565549937 -94.58805792542731) bank55388 +55389 POINT(28.937435433526904 -94.56541265898981) bank55389 +55390 POINT(28.771264125262103 -95.58257441994184) bank55390 +55391 POINT(30.17935516208731 -95.05987773728573) bank55391 +55392 POINT(30.031079839036845 -94.39800500649017) bank55392 +55393 POINT(29.047036803557774 -95.22532870604782) bank55393 +55394 POINT(28.905624227973203 -94.82710060721233) bank55394 +55395 POINT(29.566495730973053 -95.69388443100314) bank55395 +55396 POINT(29.746943562843324 -96.1210215104731) bank55396 +55397 POINT(30.268132911799626 -95.12346892440205) bank55397 +55398 POINT(30.096514504304587 -96.22514745824847) bank55398 +55399 POINT(30.44362180591706 -94.9352682077862) bank55399 +55400 POINT(29.546046144231294 -95.39884544447327) bank55400 +55401 POINT(30.759097760751104 -95.50735520171801) bank55401 +55402 POINT(29.882456606188985 -94.92227904100919) bank55402 +55403 POINT(30.436618204985137 -95.93490299225857) bank55403 +55404 POINT(30.273741268937233 -96.20596310489005) bank55404 +55405 POINT(29.179613766635168 -95.41262922354743) bank55405 +55406 POINT(29.91801652572891 -96.02074434294876) bank55406 +55407 POINT(30.325081529572085 -96.36965068459929) bank55407 +55408 POINT(29.83324689423235 -94.78001528552541) bank55408 +55409 POINT(29.926692119831483 -94.66651414980713) bank55409 +55410 POINT(29.942861265782696 -95.91005303267207) bank55410 +55411 POINT(28.95429651404359 -95.22210083985925) bank55411 +55412 POINT(29.372919334581688 -96.2312329748736) bank55412 +55413 POINT(29.78907320713315 -95.32471066645331) bank55413 +55414 POINT(29.570812583767392 -95.9937621796824) bank55414 +55415 POINT(30.239001866583294 -96.07466719347755) bank55415 +55416 POINT(30.522855813619106 -94.38971180537078) bank55416 +55417 POINT(29.677035852250246 -95.29841861314527) bank55417 +55418 POINT(28.981756450959328 -95.7905419167053) bank55418 +55419 POINT(30.71443733788911 -95.21580573381745) bank55419 +55420 POINT(28.78540737632483 -95.45136304196483) bank55420 +55421 POINT(28.87287838510608 -95.05222463180404) bank55421 +55422 POINT(29.252421557806066 -96.17487217622454) bank55422 +55423 POINT(29.224111373774303 -95.22584547721435) bank55423 +55424 POINT(29.765913795244977 -96.18901014143663) bank55424 +55425 POINT(28.78539140977845 -94.80422930132627) bank55425 +55426 POINT(29.235695237185578 -94.49251679047366) bank55426 +55427 POINT(29.152252426628255 -95.41492868415533) bank55427 +55428 POINT(29.912952355938934 -95.00478218495577) bank55428 +55429 POINT(29.51445188826176 -95.72297003759569) bank55429 +55430 POINT(29.011008332081182 -95.06240588029856) bank55430 +55431 POINT(29.733574899198153 -96.34650732306187) bank55431 +55432 POINT(29.996571943135805 -95.95931725256442) bank55432 +55433 POINT(30.11140225430712 -94.74223984145323) bank55433 +55434 POINT(29.464942292372243 -94.45470605822995) bank55434 +55435 POINT(29.667964375808932 -95.93437572352802) bank55435 +55436 POINT(30.007043259547537 -94.48719814742456) bank55436 +55437 POINT(30.530652674445047 -95.32366321992119) bank55437 +55438 POINT(29.74534651275665 -96.06499133824862) bank55438 +55439 POINT(29.57623541554516 -94.87444043606844) bank55439 +55440 POINT(28.77713262404361 -96.35733538834175) bank55440 +55441 POINT(28.82910863794413 -94.74501114298917) bank55441 +55442 POINT(29.474547309019215 -94.8771959602455) bank55442 +55443 POINT(29.973491255280344 -95.65925824202755) bank55443 +55444 POINT(29.793998878079023 -96.07936124458973) bank55444 +55445 POINT(30.56831858927902 -95.95611384079537) bank55445 +55446 POINT(28.973426082283464 -96.3568942430032) bank55446 +55447 POINT(30.606586148775165 -95.15447905198451) bank55447 +55448 POINT(30.026069992018858 -95.62553443373798) bank55448 +55449 POINT(28.963037231203913 -94.3986824050789) bank55449 +55450 POINT(30.41234676954051 -96.1235057312012) bank55450 +55451 POINT(28.84019464040453 -94.40711922734147) bank55451 +55452 POINT(30.752378383320824 -95.08094081778403) bank55452 +55453 POINT(28.990276645580533 -94.56372327103199) bank55453 +55454 POINT(30.423615713005734 -95.48651351627642) bank55454 +55455 POINT(30.691361114486547 -95.57006942086016) bank55455 +55456 POINT(29.972851317593364 -94.78150839892763) bank55456 +55457 POINT(30.195644703133954 -95.88874878414009) bank55457 +55458 POINT(29.88251865263574 -94.76688017493795) bank55458 +55459 POINT(30.73415708861312 -94.7795418864258) bank55459 +55460 POINT(28.94332491836984 -95.04743292767351) bank55460 +55461 POINT(29.613242731021597 -95.35358961046128) bank55461 +55462 POINT(29.32378137140363 -95.92450727280463) bank55462 +55463 POINT(30.641016727839332 -95.35807585678822) bank55463 +55464 POINT(30.160212766826646 -94.70395013992216) bank55464 +55465 POINT(29.387825809211083 -95.5501085283347) bank55465 +55466 POINT(29.85392847422191 -94.65326736974932) bank55466 +55467 POINT(30.1209929141762 -95.1017296114938) bank55467 +55468 POINT(29.71125199171071 -94.95673786984835) bank55468 +55469 POINT(30.5274836192029 -95.17133135403303) bank55469 +55470 POINT(30.35063043249311 -95.9808163394406) bank55470 +55471 POINT(30.641560531330605 -94.88305599559675) bank55471 +55472 POINT(30.574301787157633 -94.86532069032283) bank55472 +55473 POINT(30.096986875956414 -94.79523782992976) bank55473 +55474 POINT(29.65768936432875 -96.2600544178611) bank55474 +55475 POINT(30.130119347290872 -96.35266324701439) bank55475 +55476 POINT(29.197284206373787 -96.32991317398016) bank55476 +55477 POINT(29.274493115473465 -95.11108806934233) bank55477 +55478 POINT(30.320946413006347 -94.44169521142184) bank55478 +55479 POINT(29.35653177191396 -95.44624416689243) bank55479 +55480 POINT(29.113639194886183 -94.45404685037992) bank55480 +55481 POINT(29.742399468232808 -95.65176986259505) bank55481 +55482 POINT(30.618532270067575 -94.7414536587209) bank55482 +55483 POINT(30.494314283377832 -95.63539284527795) bank55483 +55484 POINT(29.509091881488015 -94.82660738783592) bank55484 +55485 POINT(29.90481055574962 -94.78102653893961) bank55485 +55486 POINT(29.780939025409666 -94.89867108982797) bank55486 +55487 POINT(30.036810680818608 -95.38590500047845) bank55487 +55488 POINT(30.058956870369883 -95.7266474599571) bank55488 +55489 POINT(30.335414109982864 -94.60747142250115) bank55489 +55490 POINT(29.702880079887485 -94.37066831269402) bank55490 +55491 POINT(29.289069714173035 -96.13378559331723) bank55491 +55492 POINT(29.257681314981856 -94.61008939402208) bank55492 +55493 POINT(29.245537453865413 -95.82369632951523) bank55493 +55494 POINT(29.75497839126743 -95.55382598412297) bank55494 +55495 POINT(30.298759531902174 -95.4044598949542) bank55495 +55496 POINT(28.994930415477086 -95.7472185279992) bank55496 +55497 POINT(29.05570242093876 -95.08417166992251) bank55497 +55498 POINT(29.91878796546483 -95.17097565108845) bank55498 +55499 POINT(30.569518957050242 -95.93922135892191) bank55499 +55500 POINT(29.604697599359465 -96.23927235712834) bank55500 +55501 POINT(29.900182930022837 -94.58879455010579) bank55501 +55502 POINT(29.655151350062788 -95.86196655302213) bank55502 +55503 POINT(30.695053878502577 -94.81411152124879) bank55503 +55504 POINT(30.574866262711232 -95.12655214570951) bank55504 +55505 POINT(29.43886997873398 -94.55191874012627) bank55505 +55506 POINT(29.488028425573525 -95.14860457613356) bank55506 +55507 POINT(29.23889479849358 -95.30246321874345) bank55507 +55508 POINT(29.29690200948394 -94.5059377228392) bank55508 +55509 POINT(29.14448478517942 -96.30656593833585) bank55509 +55510 POINT(29.1695569909887 -94.47418270649781) bank55510 +55511 POINT(29.29600724786087 -94.53474712800806) bank55511 +55512 POINT(29.24154728162876 -95.82708064380576) bank55512 +55513 POINT(30.447052838827293 -94.42233512766467) bank55513 +55514 POINT(29.119061704401147 -95.25314432880974) bank55514 +55515 POINT(29.878106707558494 -95.34921486341855) bank55515 +55516 POINT(29.130652066822698 -95.9428439674478) bank55516 +55517 POINT(29.643677406106786 -94.43288030493876) bank55517 +55518 POINT(28.801823987032762 -94.75495131778139) bank55518 +55519 POINT(29.49435404628338 -95.44166152882046) bank55519 +55520 POINT(30.081833480454296 -94.56593342190882) bank55520 +55521 POINT(30.651991293934742 -95.30557121717212) bank55521 +55522 POINT(29.395626463942108 -96.26760743721422) bank55522 +55523 POINT(29.330210268031408 -94.89223693913519) bank55523 +55524 POINT(29.130663849189567 -94.64584359998689) bank55524 +55525 POINT(30.030536541894982 -95.9960276861342) bank55525 +55526 POINT(29.121446634188157 -94.5957545835984) bank55526 +55527 POINT(28.97855142866138 -94.71279915532122) bank55527 +55528 POINT(29.71736494963566 -95.68509606606172) bank55528 +55529 POINT(29.389599827506924 -95.91175020077598) bank55529 +55530 POINT(30.162831134716015 -94.56435870310328) bank55530 +55531 POINT(29.678120430153815 -95.2295758945595) bank55531 +55532 POINT(28.96954041507763 -96.20449199722167) bank55532 +55533 POINT(29.798150075627 -96.32818450597476) bank55533 +55534 POINT(29.37880800442839 -95.04031098096894) bank55534 +55535 POINT(28.923477446850868 -95.9159218681683) bank55535 +55536 POINT(29.18213071689312 -95.99107621045675) bank55536 +55537 POINT(29.457068200966084 -94.95616941359847) bank55537 +55538 POINT(29.75160400648617 -95.59818477692673) bank55538 +55539 POINT(28.897090654619358 -95.92701807141337) bank55539 +55540 POINT(28.868976785510313 -95.71139713971813) bank55540 +55541 POINT(30.33212368847848 -95.34654055646712) bank55541 +55542 POINT(30.757201164899858 -96.19363387454847) bank55542 +55543 POINT(28.95466551664958 -94.93671864781761) bank55543 +55544 POINT(29.054589488755298 -95.8912805636074) bank55544 +55545 POINT(29.317422869302824 -95.48882891002118) bank55545 +55546 POINT(29.618936007709372 -95.33738605936436) bank55546 +55547 POINT(30.117425725124487 -95.46274366982432) bank55547 +55548 POINT(30.327630185274113 -96.23547620265566) bank55548 +55549 POINT(30.267182118400974 -95.88619569104945) bank55549 +55550 POINT(30.639029474673432 -94.66893875251701) bank55550 +55551 POINT(29.142988975578387 -94.99649340516784) bank55551 +55552 POINT(29.6337172098972 -94.84363092634999) bank55552 +55553 POINT(29.13228469660777 -96.22821317371604) bank55553 +55554 POINT(29.59044948814372 -96.32657004704835) bank55554 +55555 POINT(29.071479686382034 -96.22202721439271) bank55555 +55556 POINT(29.42429742216602 -94.82687797411884) bank55556 +55557 POINT(29.14212828270977 -94.72804370368536) bank55557 +55558 POINT(30.209101169600928 -95.21742636402134) bank55558 +55559 POINT(30.29252681078739 -95.01540824172459) bank55559 +55560 POINT(28.85078667264245 -94.94582498396474) bank55560 +55561 POINT(29.47979690395057 -95.38443854279463) bank55561 +55562 POINT(30.535802749750047 -94.6358084066033) bank55562 +55563 POINT(28.774707846447626 -95.90880636090458) bank55563 +55564 POINT(30.581556370552764 -95.01666640992237) bank55564 +55565 POINT(29.32595964632884 -95.78485688092404) bank55565 +55566 POINT(28.761376779553366 -95.74805415555635) bank55566 +55567 POINT(29.542061028816057 -95.70701433449247) bank55567 +55568 POINT(30.183066141521095 -96.02359132173233) bank55568 +55569 POINT(29.6604021182087 -94.72936494952873) bank55569 +55570 POINT(29.218439579734298 -94.48228949199333) bank55570 +55571 POINT(30.519574474627063 -94.37959940339152) bank55571 +55572 POINT(29.904146636389882 -95.54840825188292) bank55572 +55573 POINT(29.501583307006115 -95.77187333653593) bank55573 +55574 POINT(29.31004279648121 -94.81246703155743) bank55574 +55575 POINT(29.016244196342242 -95.85098405386866) bank55575 +55576 POINT(30.098206445074542 -94.99113476324213) bank55576 +55577 POINT(28.852956651559012 -96.2304467685838) bank55577 +55578 POINT(29.366405399068725 -96.13585373613155) bank55578 +55579 POINT(29.82994586800174 -95.8389826577597) bank55579 +55580 POINT(29.41502821153394 -95.18318694271292) bank55580 +55581 POINT(28.961887615781865 -95.94944416229859) bank55581 +55582 POINT(30.289693876224433 -94.50889331939062) bank55582 +55583 POINT(29.202056375225165 -95.75394825003798) bank55583 +55584 POINT(29.92002828027906 -96.31878454367873) bank55584 +55585 POINT(29.008160135099484 -95.63702332429315) bank55585 +55586 POINT(30.59068589386336 -95.35716310519804) bank55586 +55587 POINT(29.047992067342193 -96.01134938009811) bank55587 +55588 POINT(30.713921274258276 -95.7563806861817) bank55588 +55589 POINT(29.23090939507128 -95.40592936329108) bank55589 +55590 POINT(30.0875037528009 -94.76180499027896) bank55590 +55591 POINT(30.176935267288663 -95.5881648849549) bank55591 +55592 POINT(30.1678343853918 -94.5060107230352) bank55592 +55593 POINT(29.418988353381668 -96.24455658351228) bank55593 +55594 POINT(30.011073604075122 -95.01927289825548) bank55594 +55595 POINT(30.70035497600771 -94.79677019163317) bank55595 +55596 POINT(29.516197434486617 -94.99391714744642) bank55596 +55597 POINT(29.388452248628063 -95.10217434574751) bank55597 +55598 POINT(30.129423775441822 -95.96015026413612) bank55598 +55599 POINT(29.602790059689752 -95.73455208024808) bank55599 +55600 POINT(28.79770193623459 -94.49405354323923) bank55600 +55601 POINT(30.583579624846024 -95.61938992848397) bank55601 +55602 POINT(29.30686582077348 -95.7995879159652) bank55602 +55603 POINT(30.36550388705859 -95.47193855378879) bank55603 +55604 POINT(29.38360240153853 -94.67265883818736) bank55604 +55605 POINT(30.320024913358672 -95.12613395388627) bank55605 +55606 POINT(29.58848058783883 -95.63030501431581) bank55606 +55607 POINT(29.393521460976594 -94.77569260196383) bank55607 +55608 POINT(29.806204573044607 -95.68818674816403) bank55608 +55609 POINT(30.291350506056503 -95.5503424582818) bank55609 +55610 POINT(29.85714850293876 -95.33371022468495) bank55610 +55611 POINT(30.07823411099784 -95.4341644462426) bank55611 +55612 POINT(29.57520163407599 -94.90761695119491) bank55612 +55613 POINT(28.842058236181654 -94.83663792718016) bank55613 +55614 POINT(30.04776198229093 -95.65714148628388) bank55614 +55615 POINT(29.945255425447364 -94.4165862852198) bank55615 +55616 POINT(29.252092600810155 -95.2518622026149) bank55616 +55617 POINT(30.2534354088336 -95.26142979936304) bank55617 +55618 POINT(29.585393246821592 -95.01291670903414) bank55618 +55619 POINT(29.36115492463579 -96.2855606049929) bank55619 +55620 POINT(28.849422310172482 -95.19268873095864) bank55620 +55621 POINT(30.559295890891406 -95.44009508515839) bank55621 +55622 POINT(30.756607465355973 -94.46172967580506) bank55622 +55623 POINT(30.544511444612098 -96.16281080130646) bank55623 +55624 POINT(29.01503216070221 -95.01880698932084) bank55624 +55625 POINT(29.307761092255735 -96.22361449361546) bank55625 +55626 POINT(30.010554844927857 -95.18296072094137) bank55626 +55627 POINT(29.212588095096674 -95.71082619251573) bank55627 +55628 POINT(28.972567227877168 -94.4004638155708) bank55628 +55629 POINT(29.22721750946181 -95.43877521666931) bank55629 +55630 POINT(28.979658075524625 -95.89551226702783) bank55630 +55631 POINT(30.175814215936562 -94.43400079296573) bank55631 +55632 POINT(30.011014388964423 -94.58396868961877) bank55632 +55633 POINT(29.324750792789867 -96.06612662867578) bank55633 +55634 POINT(29.673997544763058 -96.12153572284568) bank55634 +55635 POINT(30.272003926540297 -96.00368369543291) bank55635 +55636 POINT(29.723290069824152 -96.07775877834091) bank55636 +55637 POINT(29.35592938881837 -95.36964719880898) bank55637 +55638 POINT(29.203566325655906 -96.14486646658555) bank55638 +55639 POINT(29.070979639226856 -95.36890026270841) bank55639 +55640 POINT(30.75892146772344 -94.72960796627159) bank55640 +55641 POINT(29.784146087773426 -96.0761187360974) bank55641 +55642 POINT(30.305125870246624 -94.88695764011952) bank55642 +55643 POINT(29.807861032415445 -96.35156257887914) bank55643 +55644 POINT(29.69409920539458 -95.3813889108437) bank55644 +55645 POINT(29.117444982671785 -95.93110815746986) bank55645 +55646 POINT(29.431054108822813 -96.29498635358576) bank55646 +55647 POINT(30.019177622967895 -95.95027502429164) bank55647 +55648 POINT(30.540971850862164 -95.31107389456217) bank55648 +55649 POINT(28.902032615611823 -96.15632547916591) bank55649 +55650 POINT(29.410452348023508 -95.97165346740506) bank55650 +55651 POINT(29.564227895438272 -94.73754787250846) bank55651 +55652 POINT(28.787553046754557 -95.14414285505377) bank55652 +55653 POINT(29.534664267530683 -94.5828210990211) bank55653 +55654 POINT(29.145470420266776 -94.46419417925509) bank55654 +55655 POINT(29.717823356148894 -94.74029896166161) bank55655 +55656 POINT(29.960659077499145 -94.85571545627991) bank55656 +55657 POINT(30.760272221757045 -95.3102022988962) bank55657 +55658 POINT(30.59010780577232 -95.05478057335816) bank55658 +55659 POINT(30.19522944130927 -94.96885929084199) bank55659 +55660 POINT(29.03859020900868 -95.40387348925736) bank55660 +55661 POINT(30.351449304693 -94.44812211746618) bank55661 +55662 POINT(29.21790556649939 -96.00814844687378) bank55662 +55663 POINT(30.505177872652716 -95.43109551988562) bank55663 +55664 POINT(30.112106527022032 -94.61880971070092) bank55664 +55665 POINT(28.893498750217248 -94.85331859755979) bank55665 +55666 POINT(29.300609274310624 -95.40998269520902) bank55666 +55667 POINT(30.689990797372506 -94.92050164316336) bank55667 +55668 POINT(29.496276086819826 -95.34511698237654) bank55668 +55669 POINT(29.081638756349683 -95.24811807283943) bank55669 +55670 POINT(29.611389517199978 -95.54764467690599) bank55670 +55671 POINT(29.72706746482195 -94.61610380591009) bank55671 +55672 POINT(28.83435157468597 -94.84388101372885) bank55672 +55673 POINT(28.869127146806523 -94.49531164018248) bank55673 +55674 POINT(29.330987997618756 -96.00223290161401) bank55674 +55675 POINT(28.808687812888884 -94.46020098286125) bank55675 +55676 POINT(29.031159151805824 -95.92276584448305) bank55676 +55677 POINT(29.03861980542648 -95.05775126059311) bank55677 +55678 POINT(29.565327725033814 -95.4882246626952) bank55678 +55679 POINT(30.321162132975193 -95.37084477669083) bank55679 +55680 POINT(29.833296591350248 -95.49736982901406) bank55680 +55681 POINT(29.56108635667598 -94.57192198616423) bank55681 +55682 POINT(28.94962731129175 -95.2166126931843) bank55682 +55683 POINT(30.429759348493032 -94.59858853705015) bank55683 +55684 POINT(29.545049442771514 -94.70255420386174) bank55684 +55685 POINT(30.462733283647783 -95.85966980304073) bank55685 +55686 POINT(30.39138317121276 -94.58951127142642) bank55686 +55687 POINT(29.810482689689067 -95.72428489861916) bank55687 +55688 POINT(28.789997975394925 -94.8103810784437) bank55688 +55689 POINT(30.09657639639707 -94.62410262467489) bank55689 +55690 POINT(30.21612396538421 -95.12280850894571) bank55690 +55691 POINT(29.210359779779232 -94.70166445743384) bank55691 +55692 POINT(29.04087652482325 -95.46068479675783) bank55692 +55693 POINT(30.389852761733128 -96.29334548350647) bank55693 +55694 POINT(29.327834011834998 -94.64155233323439) bank55694 +55695 POINT(29.918035900905185 -95.8082938584231) bank55695 +55696 POINT(30.309942530363436 -96.15645894376561) bank55696 +55697 POINT(30.325175573475576 -95.83166430722335) bank55697 +55698 POINT(28.86371196015668 -95.52810506384776) bank55698 +55699 POINT(29.222357879230202 -95.26949522186736) bank55699 +55700 POINT(29.447196454627377 -95.9051310303507) bank55700 +55701 POINT(29.868465578976025 -95.28845161246494) bank55701 +55702 POINT(30.492761233261383 -95.31223308515003) bank55702 +55703 POINT(30.59501961980573 -95.56543874400921) bank55703 +55704 POINT(29.85243190354504 -96.26855428420004) bank55704 +55705 POINT(29.658290146174725 -95.20444097349784) bank55705 +55706 POINT(29.74834654999482 -95.86435090920206) bank55706 +55707 POINT(29.765369936807428 -95.63296214577615) bank55707 +55708 POINT(29.838097332253668 -95.72202357246482) bank55708 +55709 POINT(29.995492069133192 -96.1591820139984) bank55709 +55710 POINT(28.9389190874948 -95.66477875869936) bank55710 +55711 POINT(30.707166487543837 -95.77382722339772) bank55711 +55712 POINT(29.320792657176433 -95.76914043074568) bank55712 +55713 POINT(28.806029230180087 -96.01546247844264) bank55713 +55714 POINT(29.63835603391797 -95.3606004806924) bank55714 +55715 POINT(29.45474600384492 -94.49108697621361) bank55715 +55716 POINT(29.69950414837006 -94.52389248008079) bank55716 +55717 POINT(29.306121337030653 -95.629309516381) bank55717 +55718 POINT(29.184003553949307 -96.36842925208295) bank55718 +55719 POINT(30.56247330536443 -95.13277254887889) bank55719 +55720 POINT(30.29152523647204 -94.6010643708868) bank55720 +55721 POINT(29.5660491718933 -95.42638159668569) bank55721 +55722 POINT(29.416677746485547 -94.93560096942772) bank55722 +55723 POINT(29.605165621513194 -95.39616171276148) bank55723 +55724 POINT(30.276703375314792 -95.1625353538017) bank55724 +55725 POINT(30.155201165491146 -95.15639935565224) bank55725 +55726 POINT(28.899153531389913 -94.64181698429302) bank55726 +55727 POINT(30.051653072850556 -94.42693606893891) bank55727 +55728 POINT(30.35425626169188 -95.68105839833179) bank55728 +55729 POINT(29.56862947412941 -96.35349522743714) bank55729 +55730 POINT(28.81319404485933 -95.2534849191552) bank55730 +55731 POINT(29.921030997667273 -95.22584553963858) bank55731 +55732 POINT(30.66378246975792 -94.82998755239154) bank55732 +55733 POINT(29.50043154341921 -94.49160242505194) bank55733 +55734 POINT(29.7997102505339 -95.80533969681169) bank55734 +55735 POINT(28.901265093359108 -94.69251965810012) bank55735 +55736 POINT(29.31171535676454 -96.23708477277113) bank55736 +55737 POINT(29.8212250901613 -95.73390407113159) bank55737 +55738 POINT(30.478933315370533 -96.18112559430634) bank55738 +55739 POINT(29.605025361056654 -96.21221158426927) bank55739 +55740 POINT(30.665442689878365 -95.06511047754285) bank55740 +55741 POINT(30.242857306423836 -95.45351923588096) bank55741 +55742 POINT(29.00657601958341 -95.82862062272923) bank55742 +55743 POINT(28.873576499178867 -95.55069954544993) bank55743 +55744 POINT(28.82915001261707 -94.92143140843508) bank55744 +55745 POINT(30.071279839057823 -96.00052192380375) bank55745 +55746 POINT(30.56315586764435 -95.13730721830589) bank55746 +55747 POINT(30.41393495833383 -94.87823165372663) bank55747 +55748 POINT(30.641486269726215 -95.23985593684444) bank55748 +55749 POINT(29.286012814720195 -96.2476127454921) bank55749 +55750 POINT(29.255538975523567 -94.78542991640509) bank55750 +55751 POINT(30.016058166879716 -96.33625530010221) bank55751 +55752 POINT(30.206795520778343 -95.62417864242245) bank55752 +55753 POINT(28.952166680514374 -94.74113552278932) bank55753 +55754 POINT(29.83135859103809 -95.86171741409062) bank55754 +55755 POINT(29.004907505764237 -95.95198081360488) bank55755 +55756 POINT(30.57755659286112 -95.86262889132256) bank55756 +55757 POINT(30.25725013905712 -96.03823583647558) bank55757 +55758 POINT(29.886874302002028 -96.0986887000464) bank55758 +55759 POINT(29.73742535531318 -94.66109703251357) bank55759 +55760 POINT(30.181627191815025 -94.92032205602534) bank55760 +55761 POINT(28.878304689743732 -95.67960373464018) bank55761 +55762 POINT(30.06524550954213 -95.97607062086006) bank55762 +55763 POINT(30.15963874625419 -95.76276756334924) bank55763 +55764 POINT(29.89734765412825 -94.56276664269912) bank55764 +55765 POINT(29.701119309175528 -95.21791458526076) bank55765 +55766 POINT(29.603153858405673 -95.73969803763057) bank55766 +55767 POINT(29.25543188306236 -95.6259839637405) bank55767 +55768 POINT(29.040871922950807 -96.26046971520225) bank55768 +55769 POINT(29.070346358654042 -95.32427696221981) bank55769 +55770 POINT(30.31740835027347 -95.76226604833543) bank55770 +55771 POINT(28.999375111714993 -94.481630345945) bank55771 +55772 POINT(30.72655196054951 -96.1496245303093) bank55772 +55773 POINT(30.398693907494657 -94.93822368863238) bank55773 +55774 POINT(30.1007933234515 -95.76075440613081) bank55774 +55775 POINT(29.645690250338912 -95.76072833524759) bank55775 +55776 POINT(28.90761399251884 -95.79794399471344) bank55776 +55777 POINT(29.591173075663278 -95.50252163833915) bank55777 +55778 POINT(30.210404471236345 -96.36507718856011) bank55778 +55779 POINT(29.777987521602505 -95.03362558275175) bank55779 +55780 POINT(29.983511646621682 -96.2103984718284) bank55780 +55781 POINT(29.620125764595937 -95.89254842122558) bank55781 +55782 POINT(29.980548026060177 -95.06623139193867) bank55782 +55783 POINT(29.33757293205027 -96.27411960557963) bank55783 +55784 POINT(30.131453572146494 -95.95034735198713) bank55784 +55785 POINT(30.65097377284064 -95.73911702885528) bank55785 +55786 POINT(28.840589609631788 -95.30463503589117) bank55786 +55787 POINT(29.83309645380888 -95.25166457398323) bank55787 +55788 POINT(29.10064511759512 -95.09924484023988) bank55788 +55789 POINT(30.68418816755315 -94.80181646138259) bank55789 +55790 POINT(29.16747964971344 -96.19821817086658) bank55790 +55791 POINT(29.56444019188271 -95.3652589348775) bank55791 +55792 POINT(30.263764770321934 -95.44208208719331) bank55792 +55793 POINT(30.073652920377462 -95.52293488634564) bank55793 +55794 POINT(30.686029991479963 -94.85719106920746) bank55794 +55795 POINT(30.145289550726574 -95.5777292172635) bank55795 +55796 POINT(29.49512012932828 -95.7850757943067) bank55796 +55797 POINT(29.55102164848075 -95.58040389274115) bank55797 +55798 POINT(29.096727312287626 -95.91679713637592) bank55798 +55799 POINT(30.7501268276701 -95.48621703553583) bank55799 +55800 POINT(28.944474447779573 -94.83809815149668) bank55800 +55801 POINT(28.88075609559527 -96.28068784700751) bank55801 +55802 POINT(29.40216462538447 -94.70505009454928) bank55802 +55803 POINT(29.58551320161722 -95.44308157358239) bank55803 +55804 POINT(30.514250615225475 -96.33602534682953) bank55804 +55805 POINT(29.00941201738945 -95.22268816832458) bank55805 +55806 POINT(30.380339959166616 -94.43625138333245) bank55806 +55807 POINT(29.351023898113123 -96.04037298440835) bank55807 +55808 POINT(29.824719442345756 -96.21303711660765) bank55808 +55809 POINT(30.13243544353028 -94.44532459670499) bank55809 +55810 POINT(30.137243728989468 -94.67787474901324) bank55810 +55811 POINT(28.994071679104607 -95.55102276339571) bank55811 +55812 POINT(29.434257886884062 -96.06529165433476) bank55812 +55813 POINT(29.25004071893998 -95.9757374913021) bank55813 +55814 POINT(29.0370024964174 -95.35350436717351) bank55814 +55815 POINT(29.766952242512726 -94.94938198962001) bank55815 +55816 POINT(29.971786412320835 -96.2478289792867) bank55816 +55817 POINT(30.470189567077504 -95.11120875225053) bank55817 +55818 POINT(29.117794155734273 -96.36520554692623) bank55818 +55819 POINT(28.984865676603057 -95.35470225603937) bank55819 +55820 POINT(29.840180477109946 -94.70210658116632) bank55820 +55821 POINT(29.56420239118833 -95.41742685389404) bank55821 +55822 POINT(30.063938558827353 -94.99167678827777) bank55822 +55823 POINT(29.259638754966204 -95.85412298623487) bank55823 +55824 POINT(29.291126498194505 -95.13928572931282) bank55824 +55825 POINT(29.047629405186335 -94.46219322291618) bank55825 +55826 POINT(30.44468753266373 -95.93751391217965) bank55826 +55827 POINT(30.40911116160301 -95.58011797544629) bank55827 +55828 POINT(30.084809257207866 -96.06398846815853) bank55828 +55829 POINT(30.031040556200384 -94.43140989585879) bank55829 +55830 POINT(28.785509279382246 -94.70503207492435) bank55830 +55831 POINT(29.293663772133748 -95.3540613704425) bank55831 +55832 POINT(29.0132133769788 -95.8447893070008) bank55832 +55833 POINT(29.87654399237449 -95.09856161840304) bank55833 +55834 POINT(29.93539867594345 -94.95222698449163) bank55834 +55835 POINT(30.2122950839563 -94.61904538273421) bank55835 +55836 POINT(29.420346231356667 -95.97218759114773) bank55836 +55837 POINT(29.891602653662027 -96.24821376550017) bank55837 +55838 POINT(30.510043186099207 -95.46996165230767) bank55838 +55839 POINT(30.755129416707312 -96.32790382842869) bank55839 +55840 POINT(29.8621493765902 -94.92571188736919) bank55840 +55841 POINT(30.635654625161095 -94.6774245702336) bank55841 +55842 POINT(30.72508784520616 -95.59225217303234) bank55842 +55843 POINT(29.955171582155263 -94.817062844706) bank55843 +55844 POINT(28.8836829749886 -95.84627699671255) bank55844 +55845 POINT(30.7368379853617 -95.94385664918507) bank55845 +55846 POINT(30.61770582906051 -94.79204766770886) bank55846 +55847 POINT(29.060183608243342 -94.65868394698511) bank55847 +55848 POINT(30.725816944673973 -96.13458290994203) bank55848 +55849 POINT(30.073586471929172 -95.80247414094701) bank55849 +55850 POINT(29.268618263430785 -95.93321151062123) bank55850 +55851 POINT(28.888194355832727 -94.4040999847658) bank55851 +55852 POINT(29.5650060610245 -95.41894106463683) bank55852 +55853 POINT(30.457280832335908 -96.34522450471783) bank55853 +55854 POINT(29.856626817358755 -96.1100378393135) bank55854 +55855 POINT(29.87479091683941 -95.01166733011699) bank55855 +55856 POINT(29.68081813637374 -95.9142265994814) bank55856 +55857 POINT(29.83598054481859 -95.48866942856974) bank55857 +55858 POINT(30.623189782782205 -94.41832354711578) bank55858 +55859 POINT(30.290769892863704 -94.46122793989672) bank55859 +55860 POINT(28.835796780895798 -95.70628853106369) bank55860 +55861 POINT(30.246638143408852 -96.22275445403142) bank55861 +55862 POINT(28.763707092991286 -94.80390310808204) bank55862 +55863 POINT(29.740712359354433 -95.7618793092926) bank55863 +55864 POINT(29.111274856397166 -96.17787526020653) bank55864 +55865 POINT(29.999373403996685 -94.95366569992474) bank55865 +55866 POINT(30.507127684500716 -94.71248424072105) bank55866 +55867 POINT(29.528946967205787 -94.80541369668667) bank55867 +55868 POINT(28.87745315356208 -95.61290251803864) bank55868 +55869 POINT(30.19767850208735 -95.75623846851701) bank55869 +55870 POINT(30.716445961950782 -94.38644815441023) bank55870 +55871 POINT(30.196844517769875 -95.06454813810032) bank55871 +55872 POINT(30.62455973107584 -94.99018212643777) bank55872 +55873 POINT(28.942701844200577 -94.4597246744742) bank55873 +55874 POINT(30.308080253820062 -96.2211566235121) bank55874 +55875 POINT(29.809975868246102 -95.09850537707804) bank55875 +55876 POINT(29.543579210818162 -95.29550938363926) bank55876 +55877 POINT(30.547954227177332 -95.05260530254237) bank55877 +55878 POINT(29.717108343183536 -94.6919250934186) bank55878 +55879 POINT(29.28575434249772 -95.11234590046146) bank55879 +55880 POINT(29.992276620742224 -96.36436219104824) bank55880 +55881 POINT(30.443065796033054 -95.27966173050477) bank55881 +55882 POINT(29.169484640269033 -95.61718475157063) bank55882 +55883 POINT(29.157865379387157 -95.9210194473991) bank55883 +55884 POINT(28.949743220291996 -95.24442384197769) bank55884 +55885 POINT(30.16610574881527 -95.28736738069328) bank55885 +55886 POINT(29.151605236457904 -94.5192751525305) bank55886 +55887 POINT(29.85342012101031 -95.97534571455884) bank55887 +55888 POINT(30.09441812082865 -94.37230539528379) bank55888 +55889 POINT(29.102856201673134 -96.2438808866408) bank55889 +55890 POINT(30.402986437960053 -94.45865834558836) bank55890 +55891 POINT(29.339793067287285 -94.64624841554178) bank55891 +55892 POINT(29.073369527150668 -94.53319724975901) bank55892 +55893 POINT(30.10390073268799 -95.78456571691183) bank55893 +55894 POINT(30.48836399250698 -95.53787889022631) bank55894 +55895 POINT(29.750268560520787 -94.8745924122183) bank55895 +55896 POINT(30.075941334755946 -95.4263416173807) bank55896 +55897 POINT(29.026143869434524 -95.80213773557432) bank55897 +55898 POINT(29.00283938010553 -94.70561641638935) bank55898 +55899 POINT(29.305676570616868 -96.18683586859974) bank55899 +55900 POINT(29.959618613064674 -94.47392143912168) bank55900 +55901 POINT(29.531677857763096 -95.62179761834886) bank55901 +55902 POINT(30.115700843542836 -95.00582504551853) bank55902 +55903 POINT(30.568735539566248 -96.26670392140147) bank55903 +55904 POINT(30.656973974153537 -95.67182769121477) bank55904 +55905 POINT(30.6134182789814 -94.81125812540692) bank55905 +55906 POINT(28.945066143715977 -95.57407495233753) bank55906 +55907 POINT(29.375825175191057 -95.54206126096751) bank55907 +55908 POINT(30.20667494229342 -96.30190612297388) bank55908 +55909 POINT(29.395706225067823 -94.94295268604627) bank55909 +55910 POINT(30.34438811972603 -95.00304844297695) bank55910 +55911 POINT(30.67453078561108 -96.2156906079845) bank55911 +55912 POINT(29.259872298351873 -94.955541003332) bank55912 +55913 POINT(30.399309803661417 -94.60971314605675) bank55913 +55914 POINT(30.225059663401595 -95.46469823733567) bank55914 +55915 POINT(29.554751597302253 -95.94091313985304) bank55915 +55916 POINT(30.69591505743262 -95.90233414392138) bank55916 +55917 POINT(29.562073650856767 -96.3380872372435) bank55917 +55918 POINT(29.941627070351956 -94.52140923897701) bank55918 +55919 POINT(29.75555078834631 -95.17399665260224) bank55919 +55920 POINT(28.81568117698705 -95.78889006516052) bank55920 +55921 POINT(28.790728581821757 -95.42396203214614) bank55921 +55922 POINT(30.60330368638525 -96.30870787018966) bank55922 +55923 POINT(30.704317208023227 -96.10474660284352) bank55923 +55924 POINT(30.34570072604645 -96.33105487772889) bank55924 +55925 POINT(29.559138200615532 -95.18091049875606) bank55925 +55926 POINT(29.413696733248585 -95.03361308659434) bank55926 +55927 POINT(29.139674251165225 -94.92740250309691) bank55927 +55928 POINT(30.408650087618234 -95.47191945194429) bank55928 +55929 POINT(30.189812915533302 -94.62677397345273) bank55929 +55930 POINT(28.974584865905154 -94.512536318616) bank55930 +55931 POINT(30.287593369663046 -95.8741918882199) bank55931 +55932 POINT(29.939037004685012 -95.4995803489781) bank55932 +55933 POINT(28.888921842858252 -95.50510136882717) bank55933 +55934 POINT(30.073505512644335 -95.29840763548758) bank55934 +55935 POINT(29.705772880685405 -96.30901488565141) bank55935 +55936 POINT(29.29325154085519 -94.4087478013185) bank55936 +55937 POINT(30.589055101432987 -95.67321767814468) bank55937 +55938 POINT(29.324862470026794 -94.82895881248244) bank55938 +55939 POINT(29.619685738339363 -95.40976793306557) bank55939 +55940 POINT(30.145433875117376 -96.05313772094897) bank55940 +55941 POINT(28.943404866617787 -95.55123465144013) bank55941 +55942 POINT(29.365038039028022 -94.98452983134607) bank55942 +55943 POINT(29.857307047703397 -94.4849665423705) bank55943 +55944 POINT(30.53008678300232 -95.89355622471182) bank55944 +55945 POINT(29.41850074232454 -96.16576489564575) bank55945 +55946 POINT(30.062157575232543 -95.34825909100736) bank55946 +55947 POINT(29.498290970865316 -94.92610062388358) bank55947 +55948 POINT(29.727092193484243 -96.00564737866752) bank55948 +55949 POINT(29.450009106681446 -95.74975148619288) bank55949 +55950 POINT(29.70940065978958 -95.50222971310896) bank55950 +55951 POINT(29.892557530150356 -95.66062302263951) bank55951 +55952 POINT(29.156231244571856 -96.15617742374033) bank55952 +55953 POINT(29.530131513142248 -96.1895989763939) bank55953 +55954 POINT(29.045280001038268 -94.47010904824623) bank55954 +55955 POINT(30.482932502742283 -95.6024506648321) bank55955 +55956 POINT(29.964564795782955 -94.90543958991172) bank55956 +55957 POINT(29.646034588311686 -95.90289573279968) bank55957 +55958 POINT(29.08318784916655 -94.92030489091522) bank55958 +55959 POINT(29.465867159039746 -95.32408200686953) bank55959 +55960 POINT(30.512781280442645 -94.73993256356577) bank55960 +55961 POINT(29.264188159167198 -95.99042762612375) bank55961 +55962 POINT(30.09056031144102 -95.70426764055924) bank55962 +55963 POINT(30.271596934721916 -95.05256046454943) bank55963 +55964 POINT(29.160163433240058 -96.26846333622154) bank55964 +55965 POINT(29.618273114501022 -95.91212025241859) bank55965 +55966 POINT(30.265999538318894 -96.36827544531141) bank55966 +55967 POINT(29.419666264381334 -96.26763348248723) bank55967 +55968 POINT(29.507649208880913 -95.92365983478064) bank55968 +55969 POINT(28.9441419962318 -95.52450425820437) bank55969 +55970 POINT(29.374774930893494 -95.95271632043745) bank55970 +55971 POINT(30.714854782479268 -94.61166499773272) bank55971 +55972 POINT(30.40508123225234 -95.7850477493945) bank55972 +55973 POINT(29.868170189529952 -95.6442483047212) bank55973 +55974 POINT(28.853196793117686 -95.21796135484706) bank55974 +55975 POINT(29.59396220755651 -94.53439353355125) bank55975 +55976 POINT(29.256026392473 -94.72295368933383) bank55976 +55977 POINT(29.93851269873458 -94.6322162386026) bank55977 +55978 POINT(29.718442833908558 -94.64084895773422) bank55978 +55979 POINT(29.06968557437955 -94.5496232394299) bank55979 +55980 POINT(29.08554817909554 -95.28782108776511) bank55980 +55981 POINT(29.494582780761696 -95.72729604477124) bank55981 +55982 POINT(28.89089393207046 -95.0627598856127) bank55982 +55983 POINT(29.759102556213307 -94.62603481236088) bank55983 +55984 POINT(28.8874854413585 -95.85268912700364) bank55984 +55985 POINT(30.533165515170953 -95.94682745879308) bank55985 +55986 POINT(29.704452217218353 -94.80018911398179) bank55986 +55987 POINT(29.96034777287764 -96.22698511044383) bank55987 +55988 POINT(29.750870705341022 -94.76750798176138) bank55988 +55989 POINT(30.75902836723162 -96.13380484086586) bank55989 +55990 POINT(29.68642974463384 -95.79775012901612) bank55990 +55991 POINT(30.561872454857326 -95.71058704378362) bank55991 +55992 POINT(30.236335825177566 -95.62684280015098) bank55992 +55993 POINT(28.93837416629789 -95.48833829119441) bank55993 +55994 POINT(30.399795412330768 -95.17749710031454) bank55994 +55995 POINT(29.97477486646858 -94.84112724126031) bank55995 +55996 POINT(30.453075616283904 -95.75183007623713) bank55996 +55997 POINT(30.09895966655909 -96.2366164281244) bank55997 +55998 POINT(30.60908268362504 -96.31312649524769) bank55998 +55999 POINT(30.661490732473535 -94.69924461370724) bank55999 +56000 POINT(29.58526303814246 -95.59880793428786) bank56000 +56001 POINT(29.21469875693893 -95.90307507731845) bank56001 +56002 POINT(28.976528297177936 -95.72744048787943) bank56002 +56003 POINT(29.59083746413963 -94.92702124978582) bank56003 +56004 POINT(29.826950843311177 -95.14006203514643) bank56004 +56005 POINT(28.845953515678925 -94.75735169310406) bank56005 +56006 POINT(29.88332629517553 -96.17644366104977) bank56006 +56007 POINT(29.931096011236356 -94.80409818290877) bank56007 +56008 POINT(29.986898667186747 -95.17095305274967) bank56008 +56009 POINT(29.321557053825025 -95.60822345261683) bank56009 +56010 POINT(30.470336868997197 -95.81616725064416) bank56010 +56011 POINT(29.971671649744057 -95.08568867024444) bank56011 +56012 POINT(29.049061366301835 -96.03433503704503) bank56012 +56013 POINT(30.31327299794221 -95.82273232226812) bank56013 +56014 POINT(29.272296741645498 -96.05652086138207) bank56014 +56015 POINT(29.876028199304997 -94.72117109717881) bank56015 +56016 POINT(29.552574787211054 -95.952488929807) bank56016 +56017 POINT(30.567406141657912 -95.40870894752466) bank56017 +56018 POINT(29.922866328934322 -94.86142217320993) bank56018 +56019 POINT(29.380861103711794 -95.40638014210428) bank56019 +56020 POINT(29.82795750356853 -95.42129661698772) bank56020 +56021 POINT(30.372301295170892 -94.81508577639737) bank56021 +56022 POINT(30.558590818964433 -94.97512541433146) bank56022 +56023 POINT(30.157878246540538 -95.36420194564045) bank56023 +56024 POINT(29.365618128791002 -95.92664564753429) bank56024 +56025 POINT(30.262605316045033 -96.276452224058) bank56025 +56026 POINT(29.479114205770628 -95.58554766318106) bank56026 +56027 POINT(30.104474995202963 -96.36828401942425) bank56027 +56028 POINT(30.114444058397304 -95.10804616761347) bank56028 +56029 POINT(30.262840598218844 -94.98959408060979) bank56029 +56030 POINT(29.81349250717985 -94.40825184738975) bank56030 +56031 POINT(30.50431855721017 -94.52660430602666) bank56031 +56032 POINT(30.537473929540308 -95.41667820415091) bank56032 +56033 POINT(28.77006211183084 -95.65843433587806) bank56033 +56034 POINT(29.439972001180553 -94.73118260896496) bank56034 +56035 POINT(28.77173575008451 -95.56455834439599) bank56035 +56036 POINT(30.375848226472343 -96.14472956586742) bank56036 +56037 POINT(29.344343932787517 -95.70185781095255) bank56037 +56038 POINT(30.36786971460807 -94.85605057691896) bank56038 +56039 POINT(30.002347403985002 -95.11694514878802) bank56039 +56040 POINT(29.777791009555667 -95.6073081111821) bank56040 +56041 POINT(30.330175103632765 -95.78285243962546) bank56041 +56042 POINT(29.184302959986706 -95.42540797950215) bank56042 +56043 POINT(30.479970495041883 -95.48456282206077) bank56043 +56044 POINT(29.177677468476446 -95.54441719000916) bank56044 +56045 POINT(29.148954546821987 -95.84654046120679) bank56045 +56046 POINT(29.09368512157906 -96.18334447176098) bank56046 +56047 POINT(30.04917388252499 -95.05786638224073) bank56047 +56048 POINT(29.815829954669546 -95.36589911260442) bank56048 +56049 POINT(30.030225268202432 -94.70120076459091) bank56049 +56050 POINT(29.713744644910342 -94.59267269746023) bank56050 +56051 POINT(29.906436845368297 -96.1700387850357) bank56051 +56052 POINT(30.459377839940604 -95.86049798210213) bank56052 +56053 POINT(30.124574952489958 -94.70054357974404) bank56053 +56054 POINT(29.201110969843857 -94.93160669139357) bank56054 +56055 POINT(30.47424538308663 -95.50057587394444) bank56055 +56056 POINT(30.41292153562276 -96.07673475429881) bank56056 +56057 POINT(29.16731371125442 -95.50317681562657) bank56057 +56058 POINT(30.726028999995467 -95.03562963923088) bank56058 +56059 POINT(30.305943059974595 -95.7219013291299) bank56059 +56060 POINT(28.874274483522218 -96.29409729433056) bank56060 +56061 POINT(30.170106662164937 -96.17347315469473) bank56061 +56062 POINT(30.016451471383522 -96.13644610273528) bank56062 +56063 POINT(30.240665738192437 -94.39188005264032) bank56063 +56064 POINT(29.880808711694193 -95.75090077743387) bank56064 +56065 POINT(29.30810826161706 -94.95138241990458) bank56065 +56066 POINT(28.94979506920394 -96.21531926954229) bank56066 +56067 POINT(29.848915997895475 -94.92573875098809) bank56067 +56068 POINT(29.686404857660165 -94.42246568390364) bank56068 +56069 POINT(29.067047812704764 -95.01525076369312) bank56069 +56070 POINT(29.962954998754345 -94.46998737300835) bank56070 +56071 POINT(29.083716782077275 -94.91646710286399) bank56071 +56072 POINT(29.462355774640336 -96.19632015131113) bank56072 +56073 POINT(30.524060019648978 -96.15180006327581) bank56073 +56074 POINT(30.572024662280153 -94.80285820236058) bank56074 +56075 POINT(30.100861615543316 -94.94931250168449) bank56075 +56076 POINT(30.1601846805267 -95.71304400139083) bank56076 +56077 POINT(29.421931869500447 -95.68207311148818) bank56077 +56078 POINT(29.115498277422994 -94.7132779757626) bank56078 +56079 POINT(29.73361918654334 -94.5724564872867) bank56079 +56080 POINT(30.090583409375345 -95.67927104101406) bank56080 +56081 POINT(29.47511329427092 -96.2612096431577) bank56081 +56082 POINT(28.935463479318898 -96.10197104930026) bank56082 +56083 POINT(29.746945037005457 -96.0930901326054) bank56083 +56084 POINT(30.638793993377824 -95.84657280846021) bank56084 +56085 POINT(30.06271988606833 -96.08859204403873) bank56085 +56086 POINT(30.495488276487123 -94.77356327500192) bank56086 +56087 POINT(29.923557439921748 -95.54457018354786) bank56087 +56088 POINT(29.134392523896867 -94.66660807194285) bank56088 +56089 POINT(29.454340947102686 -96.09185186477713) bank56089 +56090 POINT(30.608647998754122 -95.36760465553722) bank56090 +56091 POINT(29.526047520909003 -96.27297420968173) bank56091 +56092 POINT(30.118136093462738 -95.4021578814231) bank56092 +56093 POINT(29.71678293068148 -95.08558305357126) bank56093 +56094 POINT(29.55034535393153 -95.43901959866331) bank56094 +56095 POINT(28.83796366192929 -96.03807700306325) bank56095 +56096 POINT(29.271860324978565 -95.17054983207582) bank56096 +56097 POINT(30.454468354359303 -95.26058278587928) bank56097 +56098 POINT(29.976482739728006 -94.46913482807828) bank56098 +56099 POINT(30.079645826704784 -95.84262625513155) bank56099 +56100 POINT(29.288721201768915 -95.6025348003253) bank56100 +56101 POINT(29.60781765951112 -94.92953463497237) bank56101 +56102 POINT(29.888700310665907 -94.52883475628036) bank56102 +56103 POINT(29.94305075840385 -95.7936394048549) bank56103 +56104 POINT(29.201134521152284 -95.89830386443717) bank56104 +56105 POINT(29.564976244086782 -95.17291604937246) bank56105 +56106 POINT(29.368688071838356 -95.4597163949971) bank56106 +56107 POINT(29.6696108969218 -94.49325130730442) bank56107 +56108 POINT(29.73192069917656 -95.1615959724532) bank56108 +56109 POINT(29.854293994957562 -95.9699192443667) bank56109 +56110 POINT(30.549352195544536 -94.64111040681756) bank56110 +56111 POINT(29.523749611556113 -94.94903838156137) bank56111 +56112 POINT(29.904047371342248 -94.95406496039436) bank56112 +56113 POINT(30.51311133594055 -96.29505988948063) bank56113 +56114 POINT(29.096946633009914 -95.1020075995484) bank56114 +56115 POINT(28.9980569089958 -95.85081686144467) bank56115 +56116 POINT(29.9529221169012 -96.3671760005832) bank56116 +56117 POINT(29.883661048966907 -95.92639255300463) bank56117 +56118 POINT(29.6640972548012 -96.29851796532196) bank56118 +56119 POINT(30.06738490608555 -94.43592423382985) bank56119 +56120 POINT(29.547528109142725 -94.71244545519914) bank56120 +56121 POINT(30.72586604020017 -94.4412423287566) bank56121 +56122 POINT(29.714316826009316 -95.89490491197569) bank56122 +56123 POINT(29.461457360464447 -94.54599175681264) bank56123 +56124 POINT(28.810295814909015 -94.55156759563015) bank56124 +56125 POINT(28.81334325527334 -94.38371331609908) bank56125 +56126 POINT(29.940104964317882 -95.55540727833205) bank56126 +56127 POINT(30.53534695921381 -95.33904587723951) bank56127 +56128 POINT(30.52515843458591 -94.79044240884471) bank56128 +56129 POINT(29.970511084529086 -96.14081216478938) bank56129 +56130 POINT(30.319290091786254 -94.4584849187522) bank56130 +56131 POINT(30.24320462092869 -94.96725187095055) bank56131 +56132 POINT(29.548152207390757 -95.18299991098968) bank56132 +56133 POINT(29.39198592625947 -94.37125798859492) bank56133 +56134 POINT(28.861957549017102 -94.61400907559707) bank56134 +56135 POINT(30.422654607963082 -95.65319060141337) bank56135 +56136 POINT(28.83551258307971 -96.02971037701869) bank56136 +56137 POINT(30.711187009836685 -96.10964071393187) bank56137 +56138 POINT(29.92916663438661 -95.00303251398567) bank56138 +56139 POINT(30.1527461861385 -95.86794479845923) bank56139 +56140 POINT(29.143856687151384 -95.36971797607822) bank56140 +56141 POINT(29.120513880132894 -94.74043766265838) bank56141 +56142 POINT(29.55155531845716 -94.73588505825776) bank56142 +56143 POINT(29.235682174927017 -95.33023564439625) bank56143 +56144 POINT(28.949696543437426 -94.71117463625382) bank56144 +56145 POINT(29.61667285007396 -94.67915901123699) bank56145 +56146 POINT(29.66911090715139 -95.21606610835819) bank56146 +56147 POINT(30.600903484470134 -94.3958050930121) bank56147 +56148 POINT(30.174838404244493 -95.3651052269787) bank56148 +56149 POINT(29.001012882227116 -94.94506355037741) bank56149 +56150 POINT(30.41276685714994 -96.19140501609414) bank56150 +56151 POINT(29.15258425270859 -95.92129295672655) bank56151 +56152 POINT(29.138318240785456 -95.86593187863096) bank56152 +56153 POINT(30.090490514835317 -95.03211900059468) bank56153 +56154 POINT(30.664118980133054 -95.43917131454539) bank56154 +56155 POINT(30.13054620467476 -95.79983061754496) bank56155 +56156 POINT(29.965861256872532 -95.6943016399114) bank56156 +56157 POINT(30.334818908099756 -96.07916996056746) bank56157 +56158 POINT(29.85441327111753 -95.12106279157942) bank56158 +56159 POINT(29.124958999048914 -95.6197030525102) bank56159 +56160 POINT(29.208863395319263 -94.40247578390371) bank56160 +56161 POINT(29.787176934353376 -95.96010513494511) bank56161 +56162 POINT(29.885449473920254 -95.28484660107448) bank56162 +56163 POINT(29.70266329347953 -94.51462056302914) bank56163 +56164 POINT(29.335477818802346 -94.55938953012712) bank56164 +56165 POINT(30.17856424910289 -95.71348958092112) bank56165 +56166 POINT(29.4129099263603 -95.79977118461434) bank56166 +56167 POINT(30.48802123054059 -94.81567362271704) bank56167 +56168 POINT(29.781769205403535 -96.12773151581163) bank56168 +56169 POINT(29.402277971213156 -95.07209479688365) bank56169 +56170 POINT(29.551983677697486 -96.1192158620915) bank56170 +56171 POINT(30.53107377780996 -94.9228990727651) bank56171 +56172 POINT(29.303150174189735 -96.33424762738241) bank56172 +56173 POINT(30.140762456897153 -96.33343234860011) bank56173 +56174 POINT(30.69884770393772 -95.32642043669641) bank56174 +56175 POINT(30.27375466101018 -95.65179918266088) bank56175 +56176 POINT(30.07738941873219 -95.91304914468695) bank56176 +56177 POINT(29.72383985735169 -95.11214872594803) bank56177 +56178 POINT(29.250394545410213 -94.64461810699738) bank56178 +56179 POINT(29.976039786008187 -96.05805080422886) bank56179 +56180 POINT(29.940788537860797 -95.76204430325696) bank56180 +56181 POINT(30.1108146044688 -96.29614673388583) bank56181 +56182 POINT(30.741419306233507 -94.44996943663948) bank56182 +56183 POINT(30.245409971961973 -95.68518915943122) bank56183 +56184 POINT(29.629880530309052 -95.0663064574047) bank56184 +56185 POINT(29.436490643902903 -95.40502661668819) bank56185 +56186 POINT(29.009450276403832 -94.96099536778533) bank56186 +56187 POINT(30.24078455582198 -95.34076871858728) bank56187 +56188 POINT(29.908250616750934 -94.96863004603968) bank56188 +56189 POINT(30.751341344724132 -95.36407376368183) bank56189 +56190 POINT(30.704006182985385 -95.431774157767) bank56190 +56191 POINT(30.189380640482604 -95.23975229682374) bank56191 +56192 POINT(30.04174659051927 -96.24818617856387) bank56192 +56193 POINT(29.75608414218913 -96.15225067255093) bank56193 +56194 POINT(30.21643403436994 -95.30215572436563) bank56194 +56195 POINT(30.32116880375007 -96.29890258014004) bank56195 +56196 POINT(28.85590405497904 -94.90020621938743) bank56196 +56197 POINT(29.132415612339646 -95.67698094536058) bank56197 +56198 POINT(29.750227154483472 -94.8811701246977) bank56198 +56199 POINT(30.582395919482295 -95.49314519367697) bank56199 +56200 POINT(28.80028422190006 -95.49299818119606) bank56200 +56201 POINT(29.931278095557886 -95.31566829934592) bank56201 +56202 POINT(29.234943990670185 -96.1818063434796) bank56202 +56203 POINT(30.019043580355635 -96.20813079725949) bank56203 +56204 POINT(30.40561161201638 -95.42703472717669) bank56204 +56205 POINT(29.61438855652649 -94.52114037796431) bank56205 +56206 POINT(29.950067149120038 -96.13090487681241) bank56206 +56207 POINT(29.876637605604177 -94.65998492523602) bank56207 +56208 POINT(29.95600688883846 -94.71916196221376) bank56208 +56209 POINT(30.18823393322675 -95.54744989878388) bank56209 +56210 POINT(29.84246919867005 -94.66091196567966) bank56210 +56211 POINT(29.949533615635914 -96.25448836124376) bank56211 +56212 POINT(29.85948105319381 -95.93272341868942) bank56212 +56213 POINT(30.244168880295636 -96.17728548413936) bank56213 +56214 POINT(30.485707825771872 -95.98719038573687) bank56214 +56215 POINT(29.772236526576613 -95.53924456767916) bank56215 +56216 POINT(29.868810398124367 -95.25158417268999) bank56216 +56217 POINT(30.486728837233898 -95.47536966741526) bank56217 +56218 POINT(29.36019211592526 -95.89281771627485) bank56218 +56219 POINT(28.799967238190515 -95.47138399795723) bank56219 +56220 POINT(29.22160998021074 -94.43909512024392) bank56220 +56221 POINT(29.91403347826105 -95.53399632554594) bank56221 +56222 POINT(29.861782276078806 -96.26652566128945) bank56222 +56223 POINT(29.947376632107122 -95.19862345426223) bank56223 +56224 POINT(29.050861562944156 -94.86636703338189) bank56224 +56225 POINT(29.235689772847813 -95.0211475414048) bank56225 +56226 POINT(30.490596231882044 -95.59599159121838) bank56226 +56227 POINT(29.082907481157996 -95.39569964723843) bank56227 +56228 POINT(28.90154484858439 -95.54381036432807) bank56228 +56229 POINT(29.85278282391772 -94.98623654600291) bank56229 +56230 POINT(30.271523808426494 -94.6456158583346) bank56230 +56231 POINT(30.008682655185066 -95.3008568146628) bank56231 +56232 POINT(29.222987580428452 -94.74364309372865) bank56232 +56233 POINT(29.166226451605667 -95.81544685313898) bank56233 +56234 POINT(29.511452476202436 -95.98023969608738) bank56234 +56235 POINT(29.69498691992627 -95.84605167305503) bank56235 +56236 POINT(29.71584911402913 -95.63718212176123) bank56236 +56237 POINT(28.810261659982256 -95.21838633944802) bank56237 +56238 POINT(29.181242963023173 -96.2051715738671) bank56238 +56239 POINT(30.123852345895756 -95.14629111748644) bank56239 +56240 POINT(29.903894413107558 -95.36653497303298) bank56240 +56241 POINT(29.149367653020697 -95.7460452117559) bank56241 +56242 POINT(29.925846616610638 -95.61037697547276) bank56242 +56243 POINT(29.222313165641737 -94.966988697097) bank56243 +56244 POINT(29.209110219358468 -96.3444888655701) bank56244 +56245 POINT(30.534637084123112 -96.26047285147136) bank56245 +56246 POINT(29.208425952690405 -94.69554287745677) bank56246 +56247 POINT(30.16891082165075 -94.60004117616707) bank56247 +56248 POINT(29.569088657751784 -94.5256822693912) bank56248 +56249 POINT(29.31842754666838 -95.465121230481) bank56249 +56250 POINT(30.272009229357707 -96.09967808326134) bank56250 +56251 POINT(29.291586632582753 -94.87789170306647) bank56251 +56252 POINT(30.573043010646987 -95.0511475059272) bank56252 +56253 POINT(28.893554890430313 -94.5817199238032) bank56253 +56254 POINT(29.574210190523697 -96.1096157054101) bank56254 +56255 POINT(29.138079872380263 -95.26982778930414) bank56255 +56256 POINT(28.795550893092987 -96.36844502081901) bank56256 +56257 POINT(29.093712202187206 -94.52759598167928) bank56257 +56258 POINT(30.13574182263311 -95.01565457886224) bank56258 +56259 POINT(28.878289409137807 -95.39746863895374) bank56259 +56260 POINT(30.474706497376765 -95.48472506376064) bank56260 +56261 POINT(29.34457662007218 -96.17246434958962) bank56261 +56262 POINT(28.782367056368216 -95.74136446890289) bank56262 +56263 POINT(28.825302701263492 -95.84663405300584) bank56263 +56264 POINT(30.743147954709478 -94.9071817158809) bank56264 +56265 POINT(29.874931243540193 -95.58806223746619) bank56265 +56266 POINT(29.07007306442383 -96.24930908397204) bank56266 +56267 POINT(29.753805338009855 -95.09395101978384) bank56267 +56268 POINT(30.4284588674284 -94.46640510711975) bank56268 +56269 POINT(28.851485548379905 -96.22399708731783) bank56269 +56270 POINT(29.44149571292991 -95.83590144326213) bank56270 +56271 POINT(29.892203690335702 -94.60867742562107) bank56271 +56272 POINT(29.8638862632046 -95.8715470270192) bank56272 +56273 POINT(29.82265687362302 -95.4418204626658) bank56273 +56274 POINT(30.74344964175913 -95.55988246928474) bank56274 +56275 POINT(30.328302407786897 -95.34382060995158) bank56275 +56276 POINT(30.125464295999823 -94.72948400972112) bank56276 +56277 POINT(29.51907215477147 -94.74968783175258) bank56277 +56278 POINT(29.482333231048596 -95.71531607532545) bank56278 +56279 POINT(30.201643225516403 -96.23395652768957) bank56279 +56280 POINT(30.117891863538652 -95.14485987624545) bank56280 +56281 POINT(29.15166749432448 -95.04542825350782) bank56281 +56282 POINT(30.102458385732767 -95.63330660878066) bank56282 +56283 POINT(30.52214940180251 -96.14486456677339) bank56283 +56284 POINT(30.090516848268294 -94.50092777059183) bank56284 +56285 POINT(30.063271016931598 -96.1545699646093) bank56285 +56286 POINT(30.69219657514592 -94.8123067637873) bank56286 +56287 POINT(28.87584838662196 -95.96669290209357) bank56287 +56288 POINT(30.672923682418343 -96.065234759514) bank56288 +56289 POINT(29.577993072387812 -95.54255403983278) bank56289 +56290 POINT(29.240847186921613 -94.89905123538202) bank56290 +56291 POINT(29.998539305701563 -95.28938335665461) bank56291 +56292 POINT(28.873570756394066 -95.03900639327328) bank56292 +56293 POINT(29.899482175060644 -95.83800554868351) bank56293 +56294 POINT(29.04671265029569 -94.49570030428626) bank56294 +56295 POINT(28.77865230619876 -94.54880801457982) bank56295 +56296 POINT(30.248937075156366 -95.4493027536409) bank56296 +56297 POINT(29.806880040850658 -94.68049441499657) bank56297 +56298 POINT(29.32403094799974 -96.17392872479282) bank56298 +56299 POINT(29.634385914791277 -94.74697583520005) bank56299 +56300 POINT(29.062132406827956 -95.68211491841065) bank56300 +56301 POINT(28.867089042969514 -94.95495753721013) bank56301 +56302 POINT(30.331754549055965 -96.3505818952766) bank56302 +56303 POINT(30.56890111712972 -95.95536672613345) bank56303 +56304 POINT(29.78030033549976 -95.15505798323703) bank56304 +56305 POINT(29.850337451999916 -95.25457306710764) bank56305 +56306 POINT(30.278029956655928 -95.6393408916935) bank56306 +56307 POINT(29.89632244447371 -96.19785557607061) bank56307 +56308 POINT(29.056293559562146 -94.63148339804621) bank56308 +56309 POINT(29.506119143598394 -94.56252152617273) bank56309 +56310 POINT(29.93062528603196 -96.05503337425614) bank56310 +56311 POINT(29.188789410805395 -94.66132495462853) bank56311 +56312 POINT(29.896152868962297 -94.74313122298851) bank56312 +56313 POINT(29.840210261370327 -95.1681901243419) bank56313 +56314 POINT(30.521700241591837 -95.3607530593492) bank56314 +56315 POINT(29.783727958020563 -95.72503513756243) bank56315 +56316 POINT(28.970189780944352 -94.64419436864222) bank56316 +56317 POINT(29.66912145393981 -95.55194790415685) bank56317 +56318 POINT(29.379569620858497 -95.57610022006878) bank56318 +56319 POINT(30.381845416482165 -95.9403933070123) bank56319 +56320 POINT(28.86549992053883 -96.15361184647524) bank56320 +56321 POINT(30.732384240601547 -94.64874688616034) bank56321 +56322 POINT(29.544837152500335 -94.59393866886309) bank56322 +56323 POINT(29.280212287499175 -95.79948840021521) bank56323 +56324 POINT(30.447783394542622 -95.46140509674926) bank56324 +56325 POINT(30.236295373882484 -95.5179253937547) bank56325 +56326 POINT(30.355071621144205 -95.4383506813837) bank56326 +56327 POINT(29.45589166716275 -94.69916856269928) bank56327 +56328 POINT(29.82355337470272 -95.82044369424713) bank56328 +56329 POINT(28.78134746851703 -94.52866406535993) bank56329 +56330 POINT(30.327914098343285 -94.43907382968018) bank56330 +56331 POINT(30.40298812444845 -95.60711627412397) bank56331 +56332 POINT(30.43051496139792 -95.00045137463218) bank56332 +56333 POINT(30.127687697345063 -96.09309309996152) bank56333 +56334 POINT(30.361090925622108 -94.7002852776689) bank56334 +56335 POINT(29.753285195294474 -95.27164251297216) bank56335 +56336 POINT(29.927353293282515 -95.52229000116795) bank56336 +56337 POINT(28.933735875087244 -94.96814747215163) bank56337 +56338 POINT(30.35117235260471 -95.19444583285372) bank56338 +56339 POINT(30.324876435670085 -96.130300596862) bank56339 +56340 POINT(30.301331793974235 -95.03839605041708) bank56340 +56341 POINT(30.16871179337268 -95.34195816399235) bank56341 +56342 POINT(30.34554308250763 -95.81289594701725) bank56342 +56343 POINT(30.603141821260092 -94.55304740234777) bank56343 +56344 POINT(30.321579954566257 -96.03933315386047) bank56344 +56345 POINT(29.103046491146813 -94.6677846035128) bank56345 +56346 POINT(30.464201483040494 -94.4365296040864) bank56346 +56347 POINT(30.413131832236786 -94.99511238263266) bank56347 +56348 POINT(30.308157072161865 -94.43167994978019) bank56348 +56349 POINT(30.70038208873202 -95.86636414997317) bank56349 +56350 POINT(30.16165149681303 -94.4148370868922) bank56350 +56351 POINT(29.00484909785758 -95.24825905903377) bank56351 +56352 POINT(29.450563241154054 -96.14687246298391) bank56352 +56353 POINT(30.334326029742922 -94.93615086272843) bank56353 +56354 POINT(30.57599006017414 -96.2654735657518) bank56354 +56355 POINT(30.733409494870024 -95.95476404078399) bank56355 +56356 POINT(28.984569770781647 -94.64782078626736) bank56356 +56357 POINT(29.306541804313266 -96.25478319571133) bank56357 +56358 POINT(30.395757180030035 -94.89752265094704) bank56358 +56359 POINT(29.158577576960116 -94.43043624358867) bank56359 +56360 POINT(29.315308789764877 -96.2544495813838) bank56360 +56361 POINT(29.58638608366303 -96.17704956386483) bank56361 +56362 POINT(30.06217925932545 -95.83148647383456) bank56362 +56363 POINT(30.048563405041573 -95.43666377377475) bank56363 +56364 POINT(29.4054239592966 -94.7037224701168) bank56364 +56365 POINT(29.579453058117235 -96.22294579655728) bank56365 +56366 POINT(29.864852028708647 -95.45370808222624) bank56366 +56367 POINT(29.006997431040677 -95.11807602984882) bank56367 +56368 POINT(29.787677829165595 -94.63975612080769) bank56368 +56369 POINT(29.704437246319188 -96.08549411993722) bank56369 +56370 POINT(30.223428803422546 -96.06555607848262) bank56370 +56371 POINT(29.18672149933569 -96.32193062123885) bank56371 +56372 POINT(29.861504854275744 -94.49949425140078) bank56372 +56373 POINT(30.11835488180053 -96.11736472040315) bank56373 +56374 POINT(29.69016644038695 -95.7445604806292) bank56374 +56375 POINT(29.98750194110727 -96.24780724990748) bank56375 +56376 POINT(30.132779349343632 -96.15948301266526) bank56376 +56377 POINT(30.399294639473244 -95.78059595680034) bank56377 +56378 POINT(29.371474962289053 -94.99306129794314) bank56378 +56379 POINT(29.534336305477172 -95.28363377221729) bank56379 +56380 POINT(30.040132606268784 -94.64272378820421) bank56380 +56381 POINT(28.937112231493508 -95.73943652303495) bank56381 +56382 POINT(29.82305783872784 -94.9997340310957) bank56382 +56383 POINT(29.789314689997994 -94.72020444343251) bank56383 +56384 POINT(29.352684124189697 -96.09797362713718) bank56384 +56385 POINT(30.15801288008353 -96.3337081828997) bank56385 +56386 POINT(28.917289316440794 -96.35112531224615) bank56386 +56387 POINT(30.3328963019068 -95.5003350279129) bank56387 +56388 POINT(29.057657159112242 -95.40607307872939) bank56388 +56389 POINT(29.55618926123904 -96.21081192825424) bank56389 +56390 POINT(30.321874199578865 -96.09418157192805) bank56390 +56391 POINT(29.377753216383418 -95.10123312643292) bank56391 +56392 POINT(30.22407669132477 -95.58711690530635) bank56392 +56393 POINT(29.30737692201982 -95.78438221763892) bank56393 +56394 POINT(29.351941514081176 -95.61558048227326) bank56394 +56395 POINT(29.864804354905388 -95.68619219463338) bank56395 +56396 POINT(30.003726138322552 -94.51593758300032) bank56396 +56397 POINT(30.32102306296112 -94.80699375632722) bank56397 +56398 POINT(30.52922601031233 -95.71550287707949) bank56398 +56399 POINT(30.2666803683638 -96.31330348414667) bank56399 +56400 POINT(29.609127352199234 -94.54143116140699) bank56400 +56401 POINT(29.408941400049223 -95.99132305297158) bank56401 +56402 POINT(30.730330450588458 -94.8296727619311) bank56402 +56403 POINT(29.259464327863927 -94.60137060352142) bank56403 +56404 POINT(29.966127697895164 -94.68356139590952) bank56404 +56405 POINT(30.323795142748853 -96.05216271700971) bank56405 +56406 POINT(29.74383334634658 -94.71353373114867) bank56406 +56407 POINT(29.49346865013129 -95.25377502381006) bank56407 +56408 POINT(30.214552593038604 -96.13334087280603) bank56408 +56409 POINT(30.42394866528891 -94.63410221556938) bank56409 +56410 POINT(30.434374644951575 -94.743767997804) bank56410 +56411 POINT(28.936319374827743 -95.02875409154097) bank56411 +56412 POINT(28.830146611572328 -94.80949341626807) bank56412 +56413 POINT(28.94915554343324 -95.11022174293183) bank56413 +56414 POINT(30.0735743186654 -94.82599934321453) bank56414 +56415 POINT(28.889025075472052 -94.7452318819908) bank56415 +56416 POINT(30.706565397688877 -94.54018917333697) bank56416 +56417 POINT(29.422767207533727 -95.60860957580202) bank56417 +56418 POINT(30.023958372295613 -94.71739214276633) bank56418 +56419 POINT(29.344865509293967 -95.52368826365479) bank56419 +56420 POINT(30.225827101654748 -94.74337221904734) bank56420 +56421 POINT(30.30628849947851 -94.8382852486339) bank56421 +56422 POINT(29.341751737117942 -96.16721822423656) bank56422 +56423 POINT(29.667806833133163 -95.60958337320888) bank56423 +56424 POINT(29.286878743380523 -96.00108589524184) bank56424 +56425 POINT(30.24770603039245 -95.84093721414581) bank56425 +56426 POINT(28.864618532596946 -96.32542073000192) bank56426 +56427 POINT(30.619259230268593 -95.19911748991815) bank56427 +56428 POINT(29.560531578158095 -94.6827738343676) bank56428 +56429 POINT(29.768996284320057 -95.66244861502078) bank56429 +56430 POINT(30.74091541839045 -95.51945435436465) bank56430 +56431 POINT(30.468171623738076 -95.6156623087765) bank56431 +56432 POINT(30.201176226995006 -96.19943832405619) bank56432 +56433 POINT(29.988735320302972 -96.22076693438758) bank56433 +56434 POINT(29.92092943509964 -95.51419224904524) bank56434 +56435 POINT(29.531303759064173 -96.11862182416633) bank56435 +56436 POINT(30.577938722923808 -95.68306694991604) bank56436 +56437 POINT(30.044359762952663 -95.68445974014385) bank56437 +56438 POINT(29.98045333322017 -95.25885651334086) bank56438 +56439 POINT(30.687872524641183 -95.487378947375) bank56439 +56440 POINT(30.09253213546882 -95.56725712466795) bank56440 +56441 POINT(30.503750752432115 -94.42784921795902) bank56441 +56442 POINT(28.95736765548408 -96.31431755069583) bank56442 +56443 POINT(29.397463481965662 -95.10471373112264) bank56443 +56444 POINT(29.762644603819016 -96.10972590559226) bank56444 +56445 POINT(29.464367584624988 -95.51554978204318) bank56445 +56446 POINT(29.194474883283647 -94.85239068799257) bank56446 +56447 POINT(29.871274573131767 -94.59384767095156) bank56447 +56448 POINT(29.925712317889754 -95.45397164165782) bank56448 +56449 POINT(30.743189206062528 -94.85038552651122) bank56449 +56450 POINT(30.60726515803965 -94.4588830216881) bank56450 +56451 POINT(29.81906797260551 -95.56948252709385) bank56451 +56452 POINT(29.780723833983544 -95.09821337260678) bank56452 +56453 POINT(29.94299088763486 -94.72785656757941) bank56453 +56454 POINT(30.28749134604115 -95.79850498897824) bank56454 +56455 POINT(29.573014179576777 -96.35434603255614) bank56455 +56456 POINT(30.062405834678398 -94.92131680906775) bank56456 +56457 POINT(29.667462248199126 -96.05355563132163) bank56457 +56458 POINT(30.164420639021262 -95.86808584386675) bank56458 +56459 POINT(29.45127118228974 -95.18736130063884) bank56459 +56460 POINT(29.73841882685203 -95.86168801698662) bank56460 +56461 POINT(30.483243866022175 -96.31416153316577) bank56461 +56462 POINT(29.611575769201146 -94.89695792478304) bank56462 +56463 POINT(29.895253254303807 -94.65014870305855) bank56463 +56464 POINT(28.92098907611495 -94.52117674690307) bank56464 +56465 POINT(30.444007414704167 -95.84469838808994) bank56465 +56466 POINT(30.51448856281021 -95.79449310520663) bank56466 +56467 POINT(29.15313907039663 -95.34469409635847) bank56467 +56468 POINT(29.869539946429224 -95.1246601065978) bank56468 +56469 POINT(30.724193843056003 -94.58130099834871) bank56469 +56470 POINT(28.967889127669615 -94.83116092217796) bank56470 +56471 POINT(29.05993782217993 -95.95535562069587) bank56471 +56472 POINT(29.11595812025253 -96.3406204841957) bank56472 +56473 POINT(29.79122164515495 -95.3953757808688) bank56473 +56474 POINT(30.48220329898019 -95.95581394009945) bank56474 +56475 POINT(29.994348350576338 -95.59913695588985) bank56475 +56476 POINT(30.00666022881761 -95.00436539785481) bank56476 +56477 POINT(29.849572957923975 -95.35266338687222) bank56477 +56478 POINT(29.36637419750001 -95.70557719342912) bank56478 +56479 POINT(30.521160988890976 -94.73486900500586) bank56479 +56480 POINT(29.33459448126243 -96.28688849524832) bank56480 +56481 POINT(30.436341076041327 -95.40243013274817) bank56481 +56482 POINT(29.118818515673528 -95.0623499528534) bank56482 +56483 POINT(28.770836118275255 -95.86698261335205) bank56483 +56484 POINT(29.52307484436852 -95.56103523588669) bank56484 +56485 POINT(29.12624967867546 -95.51720856607092) bank56485 +56486 POINT(29.595803636774175 -94.98968678141595) bank56486 +56487 POINT(29.690380621372118 -95.2961093897847) bank56487 +56488 POINT(30.177362964101807 -96.15865250024936) bank56488 +56489 POINT(29.874128110416397 -96.2091925607291) bank56489 +56490 POINT(29.256654831367765 -95.4573041288095) bank56490 +56491 POINT(29.806579681327 -95.22011974853868) bank56491 +56492 POINT(30.226251104825536 -95.44987514732513) bank56492 +56493 POINT(30.47365608072877 -94.90366405851815) bank56493 +56494 POINT(28.787406058240375 -96.15632533881369) bank56494 +56495 POINT(29.37643838551738 -95.04780151212634) bank56495 +56496 POINT(30.42533958519054 -95.46289032147656) bank56496 +56497 POINT(30.278909766336266 -96.27072108298617) bank56497 +56498 POINT(30.206870860380814 -96.35436961037676) bank56498 +56499 POINT(30.467592145129753 -94.89672494826365) bank56499 +56500 POINT(29.61713882631659 -95.69881962954338) bank56500 +56501 POINT(29.28145285710089 -94.90545229830464) bank56501 +56502 POINT(29.827976932050383 -95.85020599129902) bank56502 +56503 POINT(28.900770634774542 -95.07965554352916) bank56503 +56504 POINT(29.05970652444315 -94.70429111080834) bank56504 +56505 POINT(29.059500607302937 -95.83579050806878) bank56505 +56506 POINT(30.332555533265328 -94.78525356466368) bank56506 +56507 POINT(30.29126232230078 -95.90734035092403) bank56507 +56508 POINT(29.86829809154159 -95.34179669650905) bank56508 +56509 POINT(29.518492668439023 -96.09423226009065) bank56509 +56510 POINT(28.920034443221994 -94.81720673077315) bank56510 +56511 POINT(30.17154947382561 -95.45840093283054) bank56511 +56512 POINT(30.245916994218465 -94.91297237553046) bank56512 +56513 POINT(29.174009229340623 -96.30340255443305) bank56513 +56514 POINT(29.055057137437668 -95.16246281095535) bank56514 +56515 POINT(28.806972089483427 -95.818323657174) bank56515 +56516 POINT(29.71775678134451 -96.01058314544032) bank56516 +56517 POINT(29.997227294656355 -94.65321310848678) bank56517 +56518 POINT(30.279653907167578 -95.4220142914062) bank56518 +56519 POINT(30.356460487111097 -95.44522083760593) bank56519 +56520 POINT(30.149545021552942 -94.62441515985964) bank56520 +56521 POINT(30.726473663343526 -96.32686082807736) bank56521 +56522 POINT(28.769078702562357 -95.80795000033359) bank56522 +56523 POINT(29.86440278133171 -95.6648585778463) bank56523 +56524 POINT(30.749667820186207 -96.11705881058302) bank56524 +56525 POINT(29.878738151222123 -96.24019878947016) bank56525 +56526 POINT(30.67689265094236 -95.33482251882567) bank56526 +56527 POINT(30.158019796561785 -95.1501857069456) bank56527 +56528 POINT(29.852569554228946 -95.70230117351682) bank56528 +56529 POINT(30.64084444836038 -96.15444756157) bank56529 +56530 POINT(30.139147814835077 -95.19170758535031) bank56530 +56531 POINT(29.829837034095213 -95.00112793492232) bank56531 +56532 POINT(29.445541678448173 -96.32110643446026) bank56532 +56533 POINT(29.29367376799818 -94.96935377692003) bank56533 +56534 POINT(30.566272628543683 -96.04531584159554) bank56534 +56535 POINT(29.05047278694536 -95.35123772823412) bank56535 +56536 POINT(30.262302667847635 -94.97640948967747) bank56536 +56537 POINT(30.436922298859255 -96.22303392698245) bank56537 +56538 POINT(30.66926407070859 -95.85992575481959) bank56538 +56539 POINT(29.741840597189334 -96.20084405783628) bank56539 +56540 POINT(30.20216622891973 -95.30615944674292) bank56540 +56541 POINT(30.473033582805954 -95.35967701118788) bank56541 +56542 POINT(30.386231308056914 -95.75230043213048) bank56542 +56543 POINT(29.836580316209876 -94.41032546367393) bank56543 +56544 POINT(30.74640466107012 -94.49392870528379) bank56544 +56545 POINT(30.065137333492903 -95.30633605059764) bank56545 +56546 POINT(29.395780875899337 -94.44432718867637) bank56546 +56547 POINT(29.968725917981644 -94.98801229318619) bank56547 +56548 POINT(30.405861925682657 -94.37488390933214) bank56548 +56549 POINT(29.44735742295983 -94.42163079223303) bank56549 +56550 POINT(30.249330961525793 -95.82089215612493) bank56550 +56551 POINT(30.493233369273792 -95.41265074509944) bank56551 +56552 POINT(29.46047140709336 -95.89917387040434) bank56552 +56553 POINT(29.798176261517234 -95.93843617740164) bank56553 +56554 POINT(29.834526096995006 -94.50288093146322) bank56554 +56555 POINT(29.943600366044677 -95.33214580267244) bank56555 +56556 POINT(28.808804107109264 -94.68547567905775) bank56556 +56557 POINT(29.26184135841894 -94.74826558652808) bank56557 +56558 POINT(30.412878017968804 -95.16047598205238) bank56558 +56559 POINT(29.451824362686853 -95.42509739897389) bank56559 +56560 POINT(29.011478776091703 -94.45336738111948) bank56560 +56561 POINT(30.713106694839748 -95.58040822590829) bank56561 +56562 POINT(29.67858286556676 -95.79148121981919) bank56562 +56563 POINT(30.49243607690337 -94.8730690233414) bank56563 +56564 POINT(29.791933258414904 -95.8189030967561) bank56564 +56565 POINT(29.68737081631481 -96.15916980420378) bank56565 +56566 POINT(29.413608070138054 -94.62777064960031) bank56566 +56567 POINT(29.124557001330803 -95.81023059500362) bank56567 +56568 POINT(30.6829397354523 -95.12492242235841) bank56568 +56569 POINT(29.965992505886287 -95.83288758893654) bank56569 +56570 POINT(29.416128611616195 -96.2215061132892) bank56570 +56571 POINT(29.297211265054163 -94.53932060985343) bank56571 +56572 POINT(28.779945649765708 -96.26211453316344) bank56572 +56573 POINT(29.834990041602584 -96.21568303245893) bank56573 +56574 POINT(29.0104350492124 -95.68336971409236) bank56574 +56575 POINT(30.01328403150754 -94.57724788343761) bank56575 +56576 POINT(30.58721464984049 -95.72385416033316) bank56576 +56577 POINT(30.567977539552498 -94.46489763003825) bank56577 +56578 POINT(30.675619679639127 -95.89671858298539) bank56578 +56579 POINT(29.820035697528454 -94.6031718327717) bank56579 +56580 POINT(30.640315975840554 -96.2849791130317) bank56580 +56581 POINT(29.803210140364712 -96.02435796151772) bank56581 +56582 POINT(29.07269222579654 -95.585297749077) bank56582 +56583 POINT(29.06496522904726 -95.94154365847781) bank56583 +56584 POINT(30.651155376754748 -95.26118887298239) bank56584 +56585 POINT(29.022866573730063 -95.69756483558442) bank56585 +56586 POINT(30.052807969315083 -94.70412826062628) bank56586 +56587 POINT(29.111754212296244 -94.52327070264553) bank56587 +56588 POINT(28.915599029821095 -95.76317345681719) bank56588 +56589 POINT(29.928712913569715 -94.92020924638204) bank56589 +56590 POINT(29.71401231882361 -95.62311011505433) bank56590 +56591 POINT(29.61686647865185 -96.12912250027199) bank56591 +56592 POINT(29.73781324302089 -95.87656972186535) bank56592 +56593 POINT(30.477463329509373 -94.87517110267582) bank56593 +56594 POINT(29.437497184256287 -96.28671520079877) bank56594 +56595 POINT(28.930807873878727 -94.7861187769662) bank56595 +56596 POINT(30.680650528743584 -96.29699490599766) bank56596 +56597 POINT(29.375836577480936 -95.22135596786478) bank56597 +56598 POINT(28.891647305405673 -94.92254705801525) bank56598 +56599 POINT(30.42174133960355 -94.60594437428294) bank56599 +56600 POINT(30.376006028296437 -96.20785385593346) bank56600 +56601 POINT(30.680654718276458 -95.98594310876419) bank56601 +56602 POINT(29.004303576344793 -96.28931729043573) bank56602 +56603 POINT(30.177655682507286 -95.4103330819762) bank56603 +56604 POINT(28.998801799342793 -95.08168524056185) bank56604 +56605 POINT(30.64650960678676 -94.75654968302419) bank56605 +56606 POINT(29.58795584807772 -94.97857506790915) bank56606 +56607 POINT(29.760859716629895 -94.37969253527098) bank56607 +56608 POINT(28.849308343961933 -95.9359773072931) bank56608 +56609 POINT(30.276481411763317 -94.54750747432593) bank56609 +56610 POINT(30.49990008550391 -94.62907205836625) bank56610 +56611 POINT(30.729157633500733 -95.18993829610048) bank56611 +56612 POINT(29.968605489774056 -96.12742084605424) bank56612 +56613 POINT(29.49552010123689 -95.78354738809318) bank56613 +56614 POINT(29.408280886436994 -95.27840543387904) bank56614 +56615 POINT(29.239823993467365 -94.96196252457807) bank56615 +56616 POINT(30.71797322776393 -95.23061568391607) bank56616 +56617 POINT(29.473885420899773 -94.78902206302367) bank56617 +56618 POINT(29.281033972611986 -96.11555572678688) bank56618 +56619 POINT(29.961039234569277 -94.450029336379) bank56619 +56620 POINT(28.815364537317066 -95.45188437516505) bank56620 +56621 POINT(30.684790964965117 -94.85819115723268) bank56621 +56622 POINT(29.70437479271679 -95.59840144606733) bank56622 +56623 POINT(30.2889644187752 -94.41597043660592) bank56623 +56624 POINT(28.93769647957595 -95.20922360960353) bank56624 +56625 POINT(30.601780421630924 -95.92176904614433) bank56625 +56626 POINT(30.140665372621356 -94.51710550891714) bank56626 +56627 POINT(29.941482218231933 -96.28471522393231) bank56627 +56628 POINT(30.71746049404349 -95.68992977769211) bank56628 +56629 POINT(29.041757022659677 -96.07229706161128) bank56629 +56630 POINT(28.981467804472302 -96.01828307180268) bank56630 +56631 POINT(29.76901019942145 -95.71186760933459) bank56631 +56632 POINT(29.002397963452612 -94.72601346226617) bank56632 +56633 POINT(29.27582675149554 -94.97866274537454) bank56633 +56634 POINT(29.927592223912267 -94.96012149069864) bank56634 +56635 POINT(29.69910928550219 -94.47958527781627) bank56635 +56636 POINT(30.14863176014665 -95.76173591806132) bank56636 +56637 POINT(30.58901225264904 -95.49750942462155) bank56637 +56638 POINT(29.788898141295157 -96.10555057170707) bank56638 +56639 POINT(29.168260618239984 -95.80906774742509) bank56639 +56640 POINT(30.417211857011548 -95.19875160833308) bank56640 +56641 POINT(29.808215184228285 -95.18194352897783) bank56641 +56642 POINT(29.034115784575196 -94.39379761204654) bank56642 +56643 POINT(30.65584376385231 -94.71288477651842) bank56643 +56644 POINT(30.101414825995715 -95.22207044615378) bank56644 +56645 POINT(30.514854060130634 -95.15464948009276) bank56645 +56646 POINT(30.692536718609574 -94.84063784085846) bank56646 +56647 POINT(30.510392420036034 -94.48791179054912) bank56647 +56648 POINT(30.743412106244186 -94.87781483204112) bank56648 +56649 POINT(30.444670418117088 -95.73801905391697) bank56649 +56650 POINT(30.339975473331492 -96.15195585148054) bank56650 +56651 POINT(30.269431832986623 -95.1353357435527) bank56651 +56652 POINT(29.085734352746318 -94.65251975006171) bank56652 +56653 POINT(29.35084875684558 -94.51312665743451) bank56653 +56654 POINT(29.008930851844084 -96.24004635783236) bank56654 +56655 POINT(29.25101873015679 -94.85547028289416) bank56655 +56656 POINT(29.789863184523675 -95.78791095077328) bank56656 +56657 POINT(29.051700275987034 -96.03140413772107) bank56657 +56658 POINT(29.815304871261255 -95.75755687284105) bank56658 +56659 POINT(29.44484246987103 -94.65574532355159) bank56659 +56660 POINT(30.719144603458307 -95.36474499759629) bank56660 +56661 POINT(29.486066685457228 -95.82755239158546) bank56661 +56662 POINT(29.107195475375285 -96.34338739925867) bank56662 +56663 POINT(29.154352492561824 -95.50480468696716) bank56663 +56664 POINT(30.405108398630894 -96.10718643151995) bank56664 +56665 POINT(28.822403708838348 -94.58670944195279) bank56665 +56666 POINT(29.43630458543625 -96.36871296273864) bank56666 +56667 POINT(29.512681423239183 -94.91978420472655) bank56667 +56668 POINT(29.200536270124765 -95.3416482574419) bank56668 +56669 POINT(29.38805051074087 -94.59452471861646) bank56669 +56670 POINT(29.348996704208098 -96.17109244252724) bank56670 +56671 POINT(29.403329802518236 -94.8546947492302) bank56671 +56672 POINT(29.706037984620245 -94.50013958755746) bank56672 +56673 POINT(29.607440277314026 -96.05197775241884) bank56673 +56674 POINT(29.24477025828973 -94.45695433495807) bank56674 +56675 POINT(29.894094615651156 -95.45925090028668) bank56675 +56676 POINT(30.359192315302366 -96.3504800619771) bank56676 +56677 POINT(30.096916744776856 -94.67594482836711) bank56677 +56678 POINT(29.726625156085777 -95.57666211018196) bank56678 +56679 POINT(30.61121853059325 -94.98426103930987) bank56679 +56680 POINT(29.88393776463036 -95.50030998186209) bank56680 +56681 POINT(30.51175207479498 -95.10118020037194) bank56681 +56682 POINT(29.036724305739867 -94.98039574396614) bank56682 +56683 POINT(29.949352865133992 -96.0241672331612) bank56683 +56684 POINT(29.30475279816209 -94.99143323582565) bank56684 +56685 POINT(30.583233110759615 -96.3061004241632) bank56685 +56686 POINT(29.565027063084422 -95.13697103368774) bank56686 +56687 POINT(30.34936487700966 -94.92235604492048) bank56687 +56688 POINT(29.81194354469509 -95.11007198263606) bank56688 +56689 POINT(28.919287508154895 -94.7367841231454) bank56689 +56690 POINT(28.821875296920773 -95.53002336547887) bank56690 +56691 POINT(30.350617944367862 -94.84747799841918) bank56691 +56692 POINT(30.063787504284075 -95.16195723684324) bank56692 +56693 POINT(30.363647599057167 -94.55538893231268) bank56693 +56694 POINT(28.912497822000155 -95.17935281412315) bank56694 +56695 POINT(28.774507852301273 -95.15865042646072) bank56695 +56696 POINT(29.148867033479448 -96.16767213218948) bank56696 +56697 POINT(29.97472006678252 -95.17864159908511) bank56697 +56698 POINT(29.85226833431123 -95.87044275079032) bank56698 +56699 POINT(29.474426167499644 -95.5964434097107) bank56699 +56700 POINT(30.06148528923747 -95.6604885222787) bank56700 +56701 POINT(30.133323502410025 -95.5550944191943) bank56701 +56702 POINT(29.694076253538878 -96.1050681206159) bank56702 +56703 POINT(28.926767546341235 -96.18072819821093) bank56703 +56704 POINT(30.021707080590996 -96.22855103431066) bank56704 +56705 POINT(29.933756273241574 -94.89049418687352) bank56705 +56706 POINT(30.635118751777686 -95.97371826111463) bank56706 +56707 POINT(29.996085186567278 -94.52848673604855) bank56707 +56708 POINT(30.56557414605375 -95.0049617870568) bank56708 +56709 POINT(28.876592204601707 -95.41916883744668) bank56709 +56710 POINT(30.40987642042381 -95.78579199602996) bank56710 +56711 POINT(30.21231074961976 -95.88126708330604) bank56711 +56712 POINT(30.093740004700884 -95.0273593008524) bank56712 +56713 POINT(29.296421591520748 -95.95741063887792) bank56713 +56714 POINT(30.53354685311385 -95.73603365698182) bank56714 +56715 POINT(29.58395754246435 -94.78708806132889) bank56715 +56716 POINT(30.306285777483115 -94.74890637100806) bank56716 +56717 POINT(30.185364726544368 -96.12403707491482) bank56717 +56718 POINT(30.370532613257563 -94.73282740542334) bank56718 +56719 POINT(29.719063066976407 -94.60801088228492) bank56719 +56720 POINT(28.944821441357618 -94.40735138660771) bank56720 +56721 POINT(29.172727391063493 -95.49393913523217) bank56721 +56722 POINT(29.07044763182163 -95.04384138728553) bank56722 +56723 POINT(30.46535272865156 -94.55682023899269) bank56723 +56724 POINT(30.503105842205926 -95.88301771430685) bank56724 +56725 POINT(29.40937785784489 -96.10904838521735) bank56725 +56726 POINT(29.302341425497744 -95.97991481907353) bank56726 +56727 POINT(30.29453764656184 -95.80492320477735) bank56727 +56728 POINT(30.177329444818668 -94.52418179829114) bank56728 +56729 POINT(29.397539918404977 -94.84462759809858) bank56729 +56730 POINT(29.473707285660378 -95.02314427461765) bank56730 +56731 POINT(29.38095271518606 -95.00428088169137) bank56731 +56732 POINT(30.625315280253425 -95.07503307076325) bank56732 +56733 POINT(29.913625332064125 -94.38755755819408) bank56733 +56734 POINT(30.25858865825604 -95.00867000771586) bank56734 +56735 POINT(28.925568415887458 -96.12007265960487) bank56735 +56736 POINT(29.086424667114223 -95.21481122650557) bank56736 +56737 POINT(28.83255949935041 -95.53146260790669) bank56737 +56738 POINT(29.594261595374338 -94.52287392529868) bank56738 +56739 POINT(29.38664187595383 -95.01332447703254) bank56739 +56740 POINT(30.088824938225144 -94.94184255812013) bank56740 +56741 POINT(29.384631600603146 -95.57932936929946) bank56741 +56742 POINT(28.787262723960012 -95.36753632489173) bank56742 +56743 POINT(29.648139455670343 -95.01539009587412) bank56743 +56744 POINT(28.85392480021442 -95.83095039060451) bank56744 +56745 POINT(30.189888366413406 -95.04199680703779) bank56745 +56746 POINT(29.242197142463375 -95.07575542547497) bank56746 +56747 POINT(29.014436651692392 -96.0673573908273) bank56747 +56748 POINT(29.624133716099138 -95.70157573418965) bank56748 +56749 POINT(29.731399073403903 -94.92601345392451) bank56749 +56750 POINT(29.24628015098368 -95.79179469581608) bank56750 +56751 POINT(30.114848791184862 -95.50811350203226) bank56751 +56752 POINT(29.8024573293849 -95.12536475629544) bank56752 +56753 POINT(29.31088871915448 -95.48648242963917) bank56753 +56754 POINT(29.980893957964344 -95.41528953151037) bank56754 +56755 POINT(29.2524366450254 -95.25859326233044) bank56755 +56756 POINT(30.475210245969702 -95.61599424782557) bank56756 +56757 POINT(29.85375478985952 -96.16082023364572) bank56757 +56758 POINT(30.38101362705786 -94.69118248352099) bank56758 +56759 POINT(30.481753617260065 -95.55224846062875) bank56759 +56760 POINT(29.43551892934797 -96.10327759551922) bank56760 +56761 POINT(29.57010283321682 -95.0576260992284) bank56761 +56762 POINT(28.969554838796423 -96.30123114711195) bank56762 +56763 POINT(29.334452390548833 -95.72064991725217) bank56763 +56764 POINT(29.725184225246533 -94.5591083317865) bank56764 +56765 POINT(29.761960980427666 -94.91761290463465) bank56765 +56766 POINT(29.920138421025445 -95.31917813186598) bank56766 +56767 POINT(29.240159306663053 -94.77516579547002) bank56767 +56768 POINT(28.805608650459025 -94.84224797465022) bank56768 +56769 POINT(28.842228595557373 -94.68271902273065) bank56769 +56770 POINT(29.33559081118875 -94.67473503162718) bank56770 +56771 POINT(30.634330028710494 -95.95976537456202) bank56771 +56772 POINT(30.353857451652956 -96.3171863812073) bank56772 +56773 POINT(28.899019370205597 -94.47364480280595) bank56773 +56774 POINT(30.569776962061795 -95.24011070897639) bank56774 +56775 POINT(30.277253905804276 -95.91545972801374) bank56775 +56776 POINT(29.413288189691844 -95.14562869249116) bank56776 +56777 POINT(30.279937097440957 -95.61903343767409) bank56777 +56778 POINT(29.16362785684645 -94.75731889038505) bank56778 +56779 POINT(30.45575700255271 -94.82938790523389) bank56779 +56780 POINT(29.96147496257338 -94.82434848194872) bank56780 +56781 POINT(29.619038394210282 -95.70918070579272) bank56781 +56782 POINT(29.57065827388155 -95.13256762798548) bank56782 +56783 POINT(29.75152373455567 -96.05295569561113) bank56783 +56784 POINT(30.46878223932616 -94.67773559222317) bank56784 +56785 POINT(28.80324474249297 -95.83608876651603) bank56785 +56786 POINT(29.22472399143044 -95.74189827290321) bank56786 +56787 POINT(30.185337062628953 -94.38090872701662) bank56787 +56788 POINT(29.164108648930096 -94.62369890306897) bank56788 +56789 POINT(29.597914954294655 -95.21496137585119) bank56789 +56790 POINT(28.918132725029217 -94.94122450375713) bank56790 +56791 POINT(29.894658321527842 -95.3183186481318) bank56791 +56792 POINT(29.15997399089152 -96.16940718321824) bank56792 +56793 POINT(30.046842557237248 -94.39991402517812) bank56793 +56794 POINT(29.48183032941043 -95.4673999889827) bank56794 +56795 POINT(29.269358972799747 -96.26081886618816) bank56795 +56796 POINT(30.569154500498133 -95.72403492083514) bank56796 +56797 POINT(29.241106668704568 -95.41843745665275) bank56797 +56798 POINT(30.09193256015704 -95.49742170921837) bank56798 +56799 POINT(29.853633121161284 -94.41808036336838) bank56799 +56800 POINT(29.80494760886111 -95.40069431669761) bank56800 +56801 POINT(30.71570320420441 -95.73170727802531) bank56801 +56802 POINT(30.54176849176714 -95.30422291300316) bank56802 +56803 POINT(29.305616564479383 -94.4614793363601) bank56803 +56804 POINT(29.36599971886718 -96.20870462985498) bank56804 +56805 POINT(29.912086557313195 -95.57620670750485) bank56805 +56806 POINT(30.62241515135273 -95.94875359135084) bank56806 +56807 POINT(30.369893066125183 -94.5612681937692) bank56807 +56808 POINT(30.521245317535246 -94.93290173578451) bank56808 +56809 POINT(29.40840769386979 -94.91591880759229) bank56809 +56810 POINT(29.95758954120767 -96.21873076644061) bank56810 +56811 POINT(28.9145488769955 -95.58754139590326) bank56811 +56812 POINT(29.52684239057602 -94.38502428008768) bank56812 +56813 POINT(29.100784542681456 -95.00291772182139) bank56813 +56814 POINT(30.160646635959967 -94.88169260175663) bank56814 +56815 POINT(30.389321450434995 -96.22927944013895) bank56815 +56816 POINT(28.87917951172135 -94.58098679743654) bank56816 +56817 POINT(30.69210615693585 -95.98909507959901) bank56817 +56818 POINT(30.750119272931382 -94.68724192748722) bank56818 +56819 POINT(29.548979383157114 -94.42290897785499) bank56819 +56820 POINT(30.679935101542302 -94.67669261884951) bank56820 +56821 POINT(29.87212734722792 -94.6463110910315) bank56821 +56822 POINT(30.66896220476589 -94.75935778076787) bank56822 +56823 POINT(29.434273850915414 -96.17738079910065) bank56823 +56824 POINT(30.219147591712307 -95.59087488185708) bank56824 +56825 POINT(29.492085227145353 -95.28107956603951) bank56825 +56826 POINT(29.928819784919867 -95.58190190815984) bank56826 +56827 POINT(29.334452284514327 -95.13544575580674) bank56827 +56828 POINT(29.346143461918146 -95.43532096829658) bank56828 +56829 POINT(29.375229768158654 -95.30124246422724) bank56829 +56830 POINT(28.825930088693 -94.53026834371686) bank56830 +56831 POINT(29.013829369035086 -94.65375861716838) bank56831 +56832 POINT(30.57573717853997 -95.51320560037588) bank56832 +56833 POINT(30.21117701290367 -95.74303610798937) bank56833 +56834 POINT(29.394518148727865 -96.1860201067848) bank56834 +56835 POINT(29.11113463294835 -95.94312060102061) bank56835 +56836 POINT(29.412547760941283 -94.53785713941171) bank56836 +56837 POINT(29.196778023572968 -94.97771010049581) bank56837 +56838 POINT(29.297066526743055 -94.56756862988036) bank56838 +56839 POINT(30.09224779617779 -94.55399085976785) bank56839 +56840 POINT(29.10958808124517 -94.85956532291297) bank56840 +56841 POINT(29.918165233387196 -95.53499190068996) bank56841 +56842 POINT(28.945297747550548 -94.71219980573196) bank56842 +56843 POINT(29.151211723375113 -94.89687031722563) bank56843 +56844 POINT(30.35894182719151 -96.23718414988976) bank56844 +56845 POINT(29.91100420749036 -95.83876811404919) bank56845 +56846 POINT(29.129127734373863 -95.38688863174069) bank56846 +56847 POINT(29.01862236259268 -96.11215494182478) bank56847 +56848 POINT(29.036546005567164 -95.7858397579771) bank56848 +56849 POINT(29.471317059205624 -95.36707837552939) bank56849 +56850 POINT(30.34539984361977 -96.08174476541669) bank56850 +56851 POINT(30.49979807206993 -95.3477075339869) bank56851 +56852 POINT(30.444843665777032 -95.85885567992534) bank56852 +56853 POINT(28.857200267510944 -95.11225308701695) bank56853 +56854 POINT(29.729004358635034 -95.5955359690678) bank56854 +56855 POINT(29.39275178787405 -95.73370191624208) bank56855 +56856 POINT(30.481461987817124 -94.78926556568231) bank56856 +56857 POINT(28.812626714053376 -95.16498945974624) bank56857 +56858 POINT(30.694261305356868 -96.19730424252087) bank56858 +56859 POINT(29.34087577201008 -94.89776511844276) bank56859 +56860 POINT(30.288548152687895 -95.39842220266688) bank56860 +56861 POINT(30.01059695903151 -95.98572751747513) bank56861 +56862 POINT(29.326439486981062 -94.37064379566363) bank56862 +56863 POINT(30.440547126580864 -95.60084135900158) bank56863 +56864 POINT(28.883624343009938 -95.68698233976983) bank56864 +56865 POINT(29.85674838498494 -96.28100989935857) bank56865 +56866 POINT(30.71210433500256 -95.74761686530282) bank56866 +56867 POINT(30.14373291081332 -95.44580359737724) bank56867 +56868 POINT(29.92524614361769 -95.68058897890415) bank56868 +56869 POINT(29.558258263693165 -94.9711591893616) bank56869 +56870 POINT(30.032579327085536 -96.36068104247671) bank56870 +56871 POINT(29.764068981275397 -95.68249942609933) bank56871 +56872 POINT(30.05081633165068 -95.44247237243219) bank56872 +56873 POINT(29.893404755845435 -95.03828960618752) bank56873 +56874 POINT(30.297182829333494 -94.99785493381856) bank56874 +56875 POINT(29.038791849019326 -95.19907797794157) bank56875 +56876 POINT(30.047456417259326 -95.38436676356642) bank56876 +56877 POINT(30.65668031521893 -95.77717088868343) bank56877 +56878 POINT(30.109627989621277 -95.70647002424455) bank56878 +56879 POINT(30.334823481836658 -94.66710360480442) bank56879 +56880 POINT(29.682884449793896 -94.60854546779144) bank56880 +56881 POINT(29.441113725859203 -95.70269932942388) bank56881 +56882 POINT(29.055753131614257 -94.44648612753143) bank56882 +56883 POINT(29.104668683974193 -95.59316257192039) bank56883 +56884 POINT(29.3343330465067 -94.7338979548163) bank56884 +56885 POINT(29.992920125748604 -96.3214358415979) bank56885 +56886 POINT(30.163539495103766 -94.97613469020914) bank56886 +56887 POINT(30.217207610697805 -96.36858575328777) bank56887 +56888 POINT(29.7428250556292 -94.52205336751585) bank56888 +56889 POINT(29.419882476715173 -95.11750129644652) bank56889 +56890 POINT(29.87376850337486 -94.49816527258875) bank56890 +56891 POINT(29.63781034614082 -94.61745569487071) bank56891 +56892 POINT(30.169254358724945 -94.84972590294478) bank56892 +56893 POINT(29.464770231990606 -94.68019417250734) bank56893 +56894 POINT(30.578352448582734 -95.77520347363945) bank56894 +56895 POINT(28.944676783475074 -94.46226086259603) bank56895 +56896 POINT(29.497573291718773 -95.8823319143096) bank56896 +56897 POINT(29.50955743449173 -95.92873332957922) bank56897 +56898 POINT(28.973802187173803 -96.2920183571437) bank56898 +56899 POINT(28.969291527490462 -94.65360196772224) bank56899 +56900 POINT(29.94254483683795 -94.92591374840812) bank56900 +56901 POINT(30.407715505913 -96.04528016891582) bank56901 +56902 POINT(29.517021488210354 -96.34269091075939) bank56902 +56903 POINT(29.425081710835904 -94.58057542345185) bank56903 +56904 POINT(30.138676424888658 -95.68484218973337) bank56904 +56905 POINT(29.883166945263206 -94.39354255460529) bank56905 +56906 POINT(30.667410786285195 -95.48395271357688) bank56906 +56907 POINT(29.48503530991406 -95.75604649253303) bank56907 +56908 POINT(28.80085642085425 -94.81106036673798) bank56908 +56909 POINT(28.914042454367646 -95.98300294516994) bank56909 +56910 POINT(30.686934673680437 -94.69052099546374) bank56910 +56911 POINT(29.657990330353762 -96.12625268726575) bank56911 +56912 POINT(30.647114456958498 -95.45279857288047) bank56912 +56913 POINT(29.527492655112372 -95.79254435011842) bank56913 +56914 POINT(29.53741310528873 -95.36068176572405) bank56914 +56915 POINT(29.914432896677155 -96.28298056973831) bank56915 +56916 POINT(30.017562401664033 -96.24648186907778) bank56916 +56917 POINT(29.158259313575932 -96.05362019070172) bank56917 +56918 POINT(29.417874579827654 -95.47215289462991) bank56918 +56919 POINT(30.37419082635705 -94.84703920936305) bank56919 +56920 POINT(30.114163344639238 -95.23530257751817) bank56920 +56921 POINT(29.49275998703172 -95.44972890939403) bank56921 +56922 POINT(30.322108630232073 -95.07074692446638) bank56922 +56923 POINT(29.949814582961025 -94.84197556286449) bank56923 +56924 POINT(30.430920685052598 -94.40554847400182) bank56924 +56925 POINT(29.67772291650451 -94.87449006018781) bank56925 +56926 POINT(29.291326835852317 -94.46748696711919) bank56926 +56927 POINT(29.711808410109914 -94.8518650218552) bank56927 +56928 POINT(30.535170614158453 -95.8470170450574) bank56928 +56929 POINT(29.13283631403382 -94.81879485855353) bank56929 +56930 POINT(30.21618964748352 -94.55478098565388) bank56930 +56931 POINT(28.838476347432934 -95.36609071553131) bank56931 +56932 POINT(29.423849069187618 -96.11945966920213) bank56932 +56933 POINT(29.70828381082912 -94.73751247336963) bank56933 +56934 POINT(28.80201224613432 -95.99750683781804) bank56934 +56935 POINT(29.325941248822247 -94.59604374461105) bank56935 +56936 POINT(29.80613149632393 -94.90160325839446) bank56936 +56937 POINT(29.289726339508757 -94.41828341610521) bank56937 +56938 POINT(29.343209961392635 -94.44601402665859) bank56938 +56939 POINT(29.590237428310104 -96.30713926589812) bank56939 +56940 POINT(30.602519789101905 -95.56039118108332) bank56940 +56941 POINT(28.78541904600378 -94.5314091366861) bank56941 +56942 POINT(29.543021553691766 -95.73567142764838) bank56942 +56943 POINT(29.285755487143458 -95.82233781635222) bank56943 +56944 POINT(30.54018287074056 -94.471551539047) bank56944 +56945 POINT(29.779301688994046 -95.37791960541146) bank56945 +56946 POINT(30.00235418762036 -94.55418018596437) bank56946 +56947 POINT(29.738599596451394 -95.66357288423171) bank56947 +56948 POINT(30.758049991823704 -95.87502768188466) bank56948 +56949 POINT(29.955409741847454 -95.92056958501664) bank56949 +56950 POINT(30.749685717667468 -96.20831551751806) bank56950 +56951 POINT(28.837842613251773 -94.96487582271371) bank56951 +56952 POINT(29.480701541300366 -94.46916451914757) bank56952 +56953 POINT(29.042303557019917 -95.77196285165563) bank56953 +56954 POINT(29.166531546941012 -95.95744770851887) bank56954 +56955 POINT(30.07138611056508 -95.85338952285542) bank56955 +56956 POINT(30.54571861265733 -95.84641902724874) bank56956 +56957 POINT(29.33780170406275 -96.15179878082778) bank56957 +56958 POINT(29.117155125211436 -95.54683410330466) bank56958 +56959 POINT(29.656250012638374 -95.2913141549676) bank56959 +56960 POINT(29.530880761528046 -94.91519224118451) bank56960 +56961 POINT(30.73251283372157 -95.14010628834647) bank56961 +56962 POINT(29.361600217071818 -94.68300607229514) bank56962 +56963 POINT(30.569805852943663 -94.87965816115685) bank56963 +56964 POINT(30.05103150183994 -94.44435501110908) bank56964 +56965 POINT(28.884265495948807 -95.58884598741393) bank56965 +56966 POINT(29.786180538893458 -94.76477523028666) bank56966 +56967 POINT(30.549444244509058 -95.57447418980331) bank56967 +56968 POINT(28.962833485819452 -95.43172100774275) bank56968 +56969 POINT(30.618841283864626 -94.95122682564433) bank56969 +56970 POINT(28.99108066082909 -94.45758982725394) bank56970 +56971 POINT(30.516395814706275 -94.87634400491068) bank56971 +56972 POINT(30.07629750246723 -95.75553485953071) bank56972 +56973 POINT(29.81311726805686 -94.98400579533576) bank56973 +56974 POINT(29.634272751727462 -95.9292316045445) bank56974 +56975 POINT(30.410383756839963 -95.78098482182102) bank56975 +56976 POINT(29.542669832724478 -95.45954390653264) bank56976 +56977 POINT(30.091653683332584 -95.57813257869905) bank56977 +56978 POINT(30.094896785353857 -95.5387225082904) bank56978 +56979 POINT(29.83186472849683 -96.25051424681793) bank56979 +56980 POINT(29.10525613681871 -95.61041494211149) bank56980 +56981 POINT(30.498757300833095 -95.98407190150529) bank56981 +56982 POINT(29.383442306713587 -95.15044830676688) bank56982 +56983 POINT(29.100212924904152 -95.68989093675104) bank56983 +56984 POINT(30.08093989087255 -94.6581371747153) bank56984 +56985 POINT(28.905272091047834 -95.25147045746893) bank56985 +56986 POINT(29.293976352248908 -94.65207348411707) bank56986 +56987 POINT(30.687587417892583 -96.0452796725121) bank56987 +56988 POINT(28.81554905864663 -95.31326276694723) bank56988 +56989 POINT(29.655452646133522 -96.06859374115903) bank56989 +56990 POINT(30.52312231768957 -95.10672539114242) bank56990 +56991 POINT(30.241852921105657 -95.69948303050914) bank56991 +56992 POINT(29.85434417413064 -95.03213862978619) bank56992 +56993 POINT(30.40003852756203 -96.22718181707906) bank56993 +56994 POINT(30.512465085209566 -95.92781977868331) bank56994 +56995 POINT(29.47339617880305 -96.03040928391603) bank56995 +56996 POINT(28.869044500206048 -95.97156508987983) bank56996 +56997 POINT(30.682426970050294 -95.59343497793758) bank56997 +56998 POINT(29.826630212549226 -95.57709686151003) bank56998 +56999 POINT(28.91280458483012 -94.41449633970714) bank56999 +57000 POINT(29.046429069366624 -96.0272906072001) bank57000 +57001 POINT(29.80540076950779 -94.57822152072762) bank57001 +57002 POINT(29.187367654202713 -95.68695607869321) bank57002 +57003 POINT(30.150156403775206 -95.44472443325854) bank57003 +57004 POINT(29.33727122140385 -96.29300288960457) bank57004 +57005 POINT(29.69817560104305 -95.43627472728846) bank57005 +57006 POINT(29.840505301517126 -95.97825609016341) bank57006 +57007 POINT(29.860258893509027 -95.77878797749754) bank57007 +57008 POINT(28.79872190481914 -94.50256881530012) bank57008 +57009 POINT(29.469351064687757 -94.80895218003579) bank57009 +57010 POINT(29.245321924040002 -94.74211959427909) bank57010 +57011 POINT(29.455727768427266 -96.281705949396) bank57011 +57012 POINT(29.70363310714996 -95.66664336368837) bank57012 +57013 POINT(29.62246214179861 -94.76152563892877) bank57013 +57014 POINT(29.995779470809566 -96.36494799184656) bank57014 +57015 POINT(30.696850920512748 -94.87799423546534) bank57015 +57016 POINT(30.16529812856545 -95.34877179597402) bank57016 +57017 POINT(29.084666132955334 -95.28181375811113) bank57017 +57018 POINT(29.588780016383218 -96.27350925705441) bank57018 +57019 POINT(29.612312717905738 -94.86975981950751) bank57019 +57020 POINT(30.490236889408607 -95.78156490941724) bank57020 +57021 POINT(29.968288192927123 -95.51764313263872) bank57021 +57022 POINT(29.85186746612781 -94.93197313085588) bank57022 +57023 POINT(29.23518711804354 -96.34613541324843) bank57023 +57024 POINT(30.744364492934892 -94.71221144244969) bank57024 +57025 POINT(29.191448312534995 -96.13577918456618) bank57025 +57026 POINT(29.86747708865194 -96.15834222604023) bank57026 +57027 POINT(29.257731307115925 -94.9953751796893) bank57027 +57028 POINT(29.394416893838137 -96.14175255953046) bank57028 +57029 POINT(29.327627522186322 -94.97091190075953) bank57029 +57030 POINT(28.792357576306806 -95.15586460933478) bank57030 +57031 POINT(29.50455617862607 -94.95642314740641) bank57031 +57032 POINT(28.854576885790326 -94.67913641363748) bank57032 +57033 POINT(30.333118715559753 -96.06215222018844) bank57033 +57034 POINT(30.70834920835206 -94.48195214553616) bank57034 +57035 POINT(30.104607269698313 -94.86156924044161) bank57035 +57036 POINT(29.595539300962947 -94.66197481027521) bank57036 +57037 POINT(29.868839260098483 -96.07780823421164) bank57037 +57038 POINT(30.31129272933958 -94.88296561155192) bank57038 +57039 POINT(29.371965453234132 -94.92756586313843) bank57039 +57040 POINT(28.957445374465195 -96.14410951448752) bank57040 +57041 POINT(30.236545308283425 -96.21148558914197) bank57041 +57042 POINT(29.502620608628675 -95.18107852412287) bank57042 +57043 POINT(29.523747764135962 -94.74096519589) bank57043 +57044 POINT(28.9446580648597 -95.68786654477641) bank57044 +57045 POINT(29.130262280751012 -94.6661975539484) bank57045 +57046 POINT(30.7350285366303 -95.89546454593697) bank57046 +57047 POINT(29.810687716934403 -94.7676978629195) bank57047 +57048 POINT(28.82605825176418 -95.61795109190322) bank57048 +57049 POINT(29.68124929705835 -95.98778762628157) bank57049 +57050 POINT(30.649461469348832 -95.38489716694583) bank57050 +57051 POINT(29.614414108323935 -95.40065523290124) bank57051 +57052 POINT(30.671626288533712 -95.96613079673482) bank57052 +57053 POINT(30.00901553722697 -94.40181826976651) bank57053 +57054 POINT(30.251946913930777 -94.70237573199984) bank57054 +57055 POINT(29.96080012649868 -95.96436490807713) bank57055 +57056 POINT(30.129334833742195 -95.99490664086632) bank57056 +57057 POINT(30.33547689019618 -95.19409134673664) bank57057 +57058 POINT(28.947059428377372 -94.67839093997141) bank57058 +57059 POINT(29.8189131177984 -95.86335739410406) bank57059 +57060 POINT(29.68791329360611 -94.56066601464535) bank57060 +57061 POINT(30.446409195686588 -95.77865831503905) bank57061 +57062 POINT(29.38575082595102 -95.1160244695813) bank57062 +57063 POINT(29.45639995841297 -95.82006304868838) bank57063 +57064 POINT(29.040999973183645 -95.21070252426316) bank57064 +57065 POINT(29.385181685070783 -94.81179411845194) bank57065 +57066 POINT(29.71584327326628 -96.07778739262127) bank57066 +57067 POINT(30.584910175426387 -95.38244250119969) bank57067 +57068 POINT(29.223561272178117 -95.41030120455447) bank57068 +57069 POINT(29.305294844890618 -95.83831349524091) bank57069 +57070 POINT(30.549347838627963 -94.71489319211243) bank57070 +57071 POINT(30.419777161547245 -96.18369123405617) bank57071 +57072 POINT(29.43586804460723 -96.32824500242702) bank57072 +57073 POINT(30.052530284632176 -94.59639150546116) bank57073 +57074 POINT(30.724236475810162 -95.24015119411764) bank57074 +57075 POINT(30.27095364902553 -96.35156640335038) bank57075 +57076 POINT(28.902720338933786 -94.8668914367732) bank57076 +57077 POINT(30.43401094572626 -94.80697236716817) bank57077 +57078 POINT(30.056995384911396 -95.9124292405781) bank57078 +57079 POINT(29.811151661073378 -95.79365437295746) bank57079 +57080 POINT(29.975100176897207 -95.37927540844495) bank57080 +57081 POINT(28.964486294077677 -95.22847959441924) bank57081 +57082 POINT(29.454441903219397 -95.08248854699875) bank57082 +57083 POINT(30.545840177351273 -95.32397017168901) bank57083 +57084 POINT(29.14256893947493 -94.96043125233567) bank57084 +57085 POINT(28.897298058192096 -96.28311942556246) bank57085 +57086 POINT(29.047303947612964 -96.19015727092979) bank57086 +57087 POINT(28.99305029410612 -95.08057486284255) bank57087 +57088 POINT(29.492133731173126 -95.25133879022265) bank57088 +57089 POINT(28.927087544184197 -95.9301154129275) bank57089 +57090 POINT(30.759164357837836 -94.52292909687144) bank57090 +57091 POINT(29.932244415839897 -96.36881377275968) bank57091 +57092 POINT(29.03043108458741 -96.14618240181818) bank57092 +57093 POINT(30.494847791189542 -95.82575389142403) bank57093 +57094 POINT(30.63271565514454 -96.06688729139135) bank57094 +57095 POINT(29.710153219233085 -95.22800795806414) bank57095 +57096 POINT(29.572342009723624 -94.56280416773602) bank57096 +57097 POINT(29.357992547650422 -96.24184120869701) bank57097 +57098 POINT(29.621679345703722 -95.50916127972165) bank57098 +57099 POINT(29.001416519224183 -94.42117228100133) bank57099 +57100 POINT(29.7803182136731 -94.855254094946) bank57100 +57101 POINT(29.563760966069143 -95.06306716300584) bank57101 +57102 POINT(30.498313978753437 -96.21869907424595) bank57102 +57103 POINT(30.427826628268424 -94.64301718614172) bank57103 +57104 POINT(29.265527637253076 -94.48679035656025) bank57104 +57105 POINT(30.643000644071993 -96.25056488165353) bank57105 +57106 POINT(29.19924711531237 -94.82795936598872) bank57106 +57107 POINT(29.266299062665357 -94.73331859383643) bank57107 +57108 POINT(30.378476563238294 -95.35298485278601) bank57108 +57109 POINT(29.73450501573328 -96.12791211752611) bank57109 +57110 POINT(30.240766416222538 -94.86346047238601) bank57110 +57111 POINT(29.22288830422484 -95.87086992850443) bank57111 +57112 POINT(30.321169853677105 -96.268116423367) bank57112 +57113 POINT(30.072715436983923 -94.83124610078903) bank57113 +57114 POINT(30.38210528523821 -96.30568710035418) bank57114 +57115 POINT(29.834567459679086 -95.30247030668532) bank57115 +57116 POINT(30.536062272650987 -96.3343782584537) bank57116 +57117 POINT(30.45615693763508 -95.62437587958779) bank57117 +57118 POINT(29.36042916379339 -95.06921498542759) bank57118 +57119 POINT(30.653217998480248 -94.74806826813652) bank57119 +57120 POINT(30.510831889181592 -95.43562840130427) bank57120 +57121 POINT(29.66870754291032 -96.18608108749379) bank57121 +57122 POINT(30.750998954448338 -95.0439523995122) bank57122 +57123 POINT(30.0706142933259 -94.48518803902172) bank57123 +57124 POINT(29.778424105851705 -95.84520586154068) bank57124 +57125 POINT(30.669958778944885 -95.4146067790622) bank57125 +57126 POINT(29.012355991203854 -95.55093572160703) bank57126 +57127 POINT(30.472919613673916 -94.50350804529377) bank57127 +57128 POINT(29.12517080135654 -95.92189393836698) bank57128 +57129 POINT(29.345809450476075 -94.99251937715718) bank57129 +57130 POINT(28.792524461901714 -94.98071133456246) bank57130 +57131 POINT(30.26949004521353 -95.68995663119114) bank57131 +57132 POINT(30.736597190218927 -94.51878373411954) bank57132 +57133 POINT(30.27646773336926 -95.21816769004401) bank57133 +57134 POINT(30.37774258225835 -96.17388817933073) bank57134 +57135 POINT(29.91863435673641 -95.32098891614913) bank57135 +57136 POINT(30.47675024607223 -96.22732376287078) bank57136 +57137 POINT(28.871574114453004 -94.77641728916723) bank57137 +57138 POINT(29.579021239535304 -95.56409053816533) bank57138 +57139 POINT(30.32339841325937 -96.06179741226987) bank57139 +57140 POINT(29.113926705471446 -95.04150671003445) bank57140 +57141 POINT(30.1369541561352 -95.40418012422556) bank57141 +57142 POINT(30.618818871390708 -94.47069772357615) bank57142 +57143 POINT(29.9896417685515 -94.68049424213734) bank57143 +57144 POINT(29.448401066892952 -95.83744056073417) bank57144 +57145 POINT(30.50139409303639 -94.62933933864103) bank57145 +57146 POINT(28.805965876420835 -95.7017544463214) bank57146 +57147 POINT(29.2632992567176 -95.93145686547562) bank57147 +57148 POINT(28.798372673869373 -95.81779624462135) bank57148 +57149 POINT(30.534439045258868 -96.19235431469096) bank57149 +57150 POINT(29.01490461596415 -95.63102738446115) bank57150 +57151 POINT(28.95252829346138 -95.0209109657617) bank57151 +57152 POINT(29.667814857840774 -94.38016016462207) bank57152 +57153 POINT(29.20100144567905 -95.54795487518724) bank57153 +57154 POINT(30.53931131061062 -94.89275547971461) bank57154 +57155 POINT(30.660368490084448 -94.92996777287915) bank57155 +57156 POINT(30.61814123918434 -94.51611786264407) bank57156 +57157 POINT(29.961842488529317 -95.06488557920689) bank57157 +57158 POINT(29.20789651500812 -94.85062668556219) bank57158 +57159 POINT(29.607818250864298 -94.48105103235173) bank57159 +57160 POINT(30.43253834044056 -95.00849377204543) bank57160 +57161 POINT(30.59127913303744 -94.78965223608486) bank57161 +57162 POINT(29.906108992386926 -95.05525781333503) bank57162 +57163 POINT(30.229784687743326 -96.14247236132577) bank57163 +57164 POINT(29.842105689636224 -94.69002513118129) bank57164 +57165 POINT(30.673504072006434 -96.08858104344827) bank57165 +57166 POINT(29.41921918554763 -94.9830176908381) bank57166 +57167 POINT(29.281756624298026 -95.57488456527317) bank57167 +57168 POINT(29.874643997895575 -95.61299365993975) bank57168 +57169 POINT(29.598871001006593 -94.99286436845873) bank57169 +57170 POINT(29.97400675882313 -94.92866744179562) bank57170 +57171 POINT(30.651351546027215 -95.85888286042884) bank57171 +57172 POINT(29.772401149673406 -95.19881414236869) bank57172 +57173 POINT(30.53623153265464 -96.00511718845748) bank57173 +57174 POINT(30.020367813808846 -96.14076299003763) bank57174 +57175 POINT(29.222125566326156 -94.73091736357213) bank57175 +57176 POINT(29.18801679716313 -95.76441971263269) bank57176 +57177 POINT(30.327888577722426 -96.23409976671428) bank57177 +57178 POINT(29.500003842790303 -96.34953488090295) bank57178 +57179 POINT(29.569495517507836 -95.38142402845577) bank57179 +57180 POINT(30.519624210583785 -94.93683125611149) bank57180 +57181 POINT(29.024403268717656 -94.97500729539298) bank57181 +57182 POINT(30.298907869580916 -96.01582417604571) bank57182 +57183 POINT(29.73166747615703 -94.78685338325234) bank57183 +57184 POINT(29.665678996821097 -96.34517979516437) bank57184 +57185 POINT(29.015299901873128 -94.47553515059742) bank57185 +57186 POINT(30.725738569706376 -96.03844785607883) bank57186 +57187 POINT(29.441787396211257 -95.21872446396799) bank57187 +57188 POINT(29.906125711051907 -95.80979657759367) bank57188 +57189 POINT(30.085956184679755 -96.17546566465234) bank57189 +57190 POINT(30.02685595173917 -95.54033309932346) bank57190 +57191 POINT(29.983097367982715 -95.94189548415383) bank57191 +57192 POINT(29.542650742636827 -95.93863909862988) bank57192 +57193 POINT(30.393700052255845 -94.89609030152751) bank57193 +57194 POINT(29.22690098818536 -95.84571154111413) bank57194 +57195 POINT(30.18722375929531 -94.87516269956996) bank57195 +57196 POINT(30.62545459768545 -95.52998188244766) bank57196 +57197 POINT(29.367327665805494 -95.21670612450086) bank57197 +57198 POINT(29.144852741042953 -96.28943098460388) bank57198 +57199 POINT(29.28310168592193 -94.82451377553538) bank57199 +57200 POINT(30.742669501917124 -94.89094278965237) bank57200 +57201 POINT(29.574994542675235 -94.40368999796948) bank57201 +57202 POINT(30.391537764503965 -95.33558647238469) bank57202 +57203 POINT(30.583451508201136 -95.8840554510682) bank57203 +57204 POINT(29.353395687641548 -96.09780740056581) bank57204 +57205 POINT(29.36747871051067 -94.40889456879474) bank57205 +57206 POINT(28.810692256326938 -94.46680043176363) bank57206 +57207 POINT(29.087047317650534 -95.10252216253909) bank57207 +57208 POINT(30.142102902123128 -95.48392594290704) bank57208 +57209 POINT(30.655482537393908 -96.35187507987095) bank57209 +57210 POINT(29.187612043265688 -95.2297052340542) bank57210 +57211 POINT(30.083477287587403 -94.7907850437688) bank57211 +57212 POINT(29.45464956411145 -95.30993073123659) bank57212 +57213 POINT(29.593276222760572 -95.63335885008534) bank57213 +57214 POINT(30.219398222654597 -94.85751097518825) bank57214 +57215 POINT(29.44089619688095 -94.69130554430414) bank57215 +57216 POINT(30.362322918194824 -94.99811011531816) bank57216 +57217 POINT(29.31907380179153 -95.0623181637511) bank57217 +57218 POINT(29.8148736288207 -96.14222915230839) bank57218 +57219 POINT(30.153732489082298 -94.46877878465288) bank57219 +57220 POINT(29.87492335943753 -96.05582127982154) bank57220 +57221 POINT(29.893374411716565 -95.04335790431199) bank57221 +57222 POINT(28.88035359012202 -94.66410896859459) bank57222 +57223 POINT(28.78398625833035 -94.56541652892543) bank57223 +57224 POINT(30.541403324204914 -94.9939241445045) bank57224 +57225 POINT(29.648363193383272 -95.59075506912168) bank57225 +57226 POINT(30.105322006597167 -95.12327232857234) bank57226 +57227 POINT(29.23843699127236 -95.92016552066647) bank57227 +57228 POINT(29.872392283071928 -96.04412783075045) bank57228 +57229 POINT(30.45787764284231 -95.23180331980151) bank57229 +57230 POINT(30.097598981781672 -94.56920337973627) bank57230 +57231 POINT(30.22242367596577 -95.08423253519862) bank57231 +57232 POINT(29.262846749979225 -95.60697993534474) bank57232 +57233 POINT(29.413602606091715 -95.76948511534101) bank57233 +57234 POINT(30.26073790520114 -94.43594745479258) bank57234 +57235 POINT(28.848874610394766 -94.40649585200383) bank57235 +57236 POINT(30.70669550067826 -95.46997309903134) bank57236 +57237 POINT(30.31247790218619 -95.48183031425347) bank57237 +57238 POINT(30.154785952157514 -96.13798958209549) bank57238 +57239 POINT(30.56981777324315 -95.93732887034744) bank57239 +57240 POINT(30.158953443915472 -94.54849606888688) bank57240 +57241 POINT(29.86493548428231 -94.9911690057583) bank57241 +57242 POINT(30.08114522100459 -96.20986000431532) bank57242 +57243 POINT(30.65332920375795 -95.79516726361456) bank57243 +57244 POINT(29.924429210395903 -94.94252810623898) bank57244 +57245 POINT(28.84397071470699 -95.37996159868246) bank57245 +57246 POINT(30.50415208436331 -96.29002478092426) bank57246 +57247 POINT(28.80320226445817 -96.22822042579098) bank57247 +57248 POINT(29.504779016275386 -95.73495637839366) bank57248 +57249 POINT(29.0311531248916 -95.16789130595832) bank57249 +57250 POINT(29.757205028725625 -95.46657300086737) bank57250 +57251 POINT(29.832282130861056 -95.66419803183166) bank57251 +57252 POINT(29.646852763100686 -96.07738671948901) bank57252 +57253 POINT(29.999093571556603 -95.33447427103319) bank57253 +57254 POINT(30.47278147712291 -94.7504911007122) bank57254 +57255 POINT(29.91322644969393 -95.67459621618147) bank57255 +57256 POINT(29.692368737638493 -94.50364918225232) bank57256 +57257 POINT(30.334803088928044 -94.7435777646774) bank57257 +57258 POINT(29.745242034095835 -94.7777571619103) bank57258 +57259 POINT(30.744074746137 -95.68581980324757) bank57259 +57260 POINT(30.35722713609399 -94.59248836890723) bank57260 +57261 POINT(30.294398502403457 -94.81181403012587) bank57261 +57262 POINT(29.85954622803266 -96.2048379390914) bank57262 +57263 POINT(29.569326808074628 -94.60747529352729) bank57263 +57264 POINT(29.46775338822194 -95.81165650071546) bank57264 +57265 POINT(29.720747824008804 -95.48058738631167) bank57265 +57266 POINT(29.19735938164549 -94.67978786162178) bank57266 +57267 POINT(30.608849362700774 -95.05920576235506) bank57267 +57268 POINT(29.240133769624983 -95.95325640741235) bank57268 +57269 POINT(29.981177072286325 -95.25575143680511) bank57269 +57270 POINT(30.36857813556706 -94.9917552724456) bank57270 +57271 POINT(29.180747082344794 -94.73268759237297) bank57271 +57272 POINT(30.106987689461004 -95.33116887155313) bank57272 +57273 POINT(28.915133024720863 -94.66886052222881) bank57273 +57274 POINT(29.478969634294614 -96.18756475688956) bank57274 +57275 POINT(30.449774667657074 -96.03071473227865) bank57275 +57276 POINT(30.738633746821854 -94.89455309533993) bank57276 +57277 POINT(30.030867148637785 -94.96413355822902) bank57277 +57278 POINT(30.039751619272167 -95.25116935362664) bank57278 +57279 POINT(28.941396539145643 -95.13050306337007) bank57279 +57280 POINT(29.39970757473869 -96.33236100056014) bank57280 +57281 POINT(29.771387107534856 -96.32460983782342) bank57281 +57282 POINT(29.654155155202503 -94.92023261940524) bank57282 +57283 POINT(28.966064035098967 -95.34171209141122) bank57283 +57284 POINT(29.588310705429294 -95.06006491281778) bank57284 +57285 POINT(28.898814557484176 -96.33788950187994) bank57285 +57286 POINT(29.29931848631287 -95.8826644483936) bank57286 +57287 POINT(29.95206608195075 -95.90743319616305) bank57287 +57288 POINT(29.03285630584651 -95.28576663414546) bank57288 +57289 POINT(30.371984713814502 -95.62694041066656) bank57289 +57290 POINT(30.363011069695844 -96.1681717723931) bank57290 +57291 POINT(30.741150775964897 -95.1978107336657) bank57291 +57292 POINT(30.39827136611373 -95.87271027752244) bank57292 +57293 POINT(28.883963998622665 -94.4839595959325) bank57293 +57294 POINT(30.231101012988447 -95.5151488777304) bank57294 +57295 POINT(30.57955344721166 -95.83295940134249) bank57295 +57296 POINT(29.30226752000018 -96.00961843957143) bank57296 +57297 POINT(30.64238007282013 -95.01243577251978) bank57297 +57298 POINT(29.48494183320251 -95.96983500186761) bank57298 +57299 POINT(29.001529133737993 -95.0855379586695) bank57299 +57300 POINT(30.157605374622708 -96.30365355065702) bank57300 +57301 POINT(29.33711839640011 -95.59447185968261) bank57301 +57302 POINT(30.06971112099418 -94.83827998348517) bank57302 +57303 POINT(30.396481910818352 -95.33969803613186) bank57303 +57304 POINT(30.701695253139608 -96.00136785352592) bank57304 +57305 POINT(29.165818915996873 -95.73325274226902) bank57305 +57306 POINT(29.111703067046978 -94.97885517854247) bank57306 +57307 POINT(30.655992860792 -95.9978437085949) bank57307 +57308 POINT(28.847215808619598 -94.71196883309717) bank57308 +57309 POINT(30.41172012965427 -95.19195837985853) bank57309 +57310 POINT(28.86979614080156 -95.58572315130927) bank57310 +57311 POINT(29.590239968133364 -96.30637669307248) bank57311 +57312 POINT(30.03819167321129 -95.81404927204856) bank57312 +57313 POINT(28.97624489036777 -95.29526874796217) bank57313 +57314 POINT(29.216458014606115 -96.20565170642791) bank57314 +57315 POINT(29.44969721462553 -95.36967247947835) bank57315 +57316 POINT(29.675901100937175 -94.73229953646839) bank57316 +57317 POINT(30.029358808936536 -94.87456734909719) bank57317 +57318 POINT(29.47879757198907 -95.05183090254634) bank57318 +57319 POINT(29.132135368294673 -95.95865008270775) bank57319 +57320 POINT(30.139336044230134 -95.30966258483926) bank57320 +57321 POINT(29.88057844530944 -96.18489761643622) bank57321 +57322 POINT(29.415680404406523 -95.28862614272442) bank57322 +57323 POINT(29.42281104792077 -95.06140584093978) bank57323 +57324 POINT(30.30048942932878 -95.46105002591291) bank57324 +57325 POINT(29.836305098540826 -95.4660577144363) bank57325 +57326 POINT(30.53309693903384 -94.39225655952471) bank57326 +57327 POINT(28.894211942526894 -94.7947165137159) bank57327 +57328 POINT(28.838633433335886 -96.01741813972087) bank57328 +57329 POINT(29.10149499505981 -94.50573911974519) bank57329 +57330 POINT(29.485914566681124 -94.51031413653419) bank57330 +57331 POINT(29.701894993165087 -95.23331015363645) bank57331 +57332 POINT(30.10653022158656 -95.93033608536855) bank57332 +57333 POINT(30.709218875523916 -95.36125574161446) bank57333 +57334 POINT(30.145879276946406 -94.46942530299165) bank57334 +57335 POINT(30.194552348687676 -96.26028587178972) bank57335 +57336 POINT(30.21115934389446 -95.52193883962447) bank57336 +57337 POINT(29.737958465825724 -95.20485197814043) bank57337 +57338 POINT(29.738374159491805 -96.19736063178836) bank57338 +57339 POINT(28.949826828870318 -95.52175945353498) bank57339 +57340 POINT(29.579910166553443 -95.42495332406457) bank57340 +57341 POINT(30.348032710994637 -96.32658105338251) bank57341 +57342 POINT(29.897660936489558 -94.96081865990544) bank57342 +57343 POINT(29.74878506651014 -95.38343931268328) bank57343 +57344 POINT(30.184749359446574 -95.06134377615622) bank57344 +57345 POINT(30.378821157991528 -95.53122579335286) bank57345 +57346 POINT(30.042406873863527 -94.82759609910019) bank57346 +57347 POINT(30.271618946334772 -94.38298589811515) bank57347 +57348 POINT(28.99845405031415 -96.33814878466745) bank57348 +57349 POINT(28.810350043373735 -95.83465163423706) bank57349 +57350 POINT(30.652850566476193 -94.63847408596351) bank57350 +57351 POINT(30.744777305458463 -95.84070316796536) bank57351 +57352 POINT(29.5385649109576 -95.66807410850508) bank57352 +57353 POINT(30.56685477624469 -94.86254241925268) bank57353 +57354 POINT(29.896706641002023 -96.26053634631369) bank57354 +57355 POINT(29.864045912793266 -95.85921311653657) bank57355 +57356 POINT(29.52102193035641 -94.57198785210066) bank57356 +57357 POINT(29.3380281993058 -94.88081324951762) bank57357 +57358 POINT(30.266406450911077 -96.26279011055746) bank57358 +57359 POINT(29.735892256023693 -95.4042621225583) bank57359 +57360 POINT(29.7371512932277 -95.45659914021307) bank57360 +57361 POINT(30.663203157959316 -95.45145073443916) bank57361 +57362 POINT(29.697656419029997 -94.857373738198) bank57362 +57363 POINT(30.02461364901606 -95.0652742071312) bank57363 +57364 POINT(30.59098331705712 -95.34132392596678) bank57364 +57365 POINT(29.889275171441675 -95.12900199716272) bank57365 +57366 POINT(28.76844140464174 -95.17229810767621) bank57366 +57367 POINT(28.992009467265408 -94.54877587555694) bank57367 +57368 POINT(30.700604656460865 -94.57961372075918) bank57368 +57369 POINT(29.637734944735048 -95.87111080779985) bank57369 +57370 POINT(29.64030399271503 -94.60262263882493) bank57370 +57371 POINT(30.29491902473121 -96.26728395781107) bank57371 +57372 POINT(28.874437740185382 -95.29622786133396) bank57372 +57373 POINT(29.916657438915227 -95.28865157782725) bank57373 +57374 POINT(29.005125202022224 -94.44416501202637) bank57374 +57375 POINT(28.911057946166547 -95.49650012686241) bank57375 +57376 POINT(28.845452481953334 -94.98789891572639) bank57376 +57377 POINT(30.45414644307214 -94.86163453386816) bank57377 +57378 POINT(29.664158237064875 -96.18313756964864) bank57378 +57379 POINT(29.973142993142943 -95.21678946897082) bank57379 +57380 POINT(28.874965851604205 -95.00593358479966) bank57380 +57381 POINT(28.928476269510973 -96.02924814467875) bank57381 +57382 POINT(29.31985996928039 -95.8891004958214) bank57382 +57383 POINT(28.78512788862422 -96.30375789011158) bank57383 +57384 POINT(29.859870877727218 -96.29143011093161) bank57384 +57385 POINT(29.654789178369697 -96.36799949970053) bank57385 +57386 POINT(30.03566996732886 -94.892291277896) bank57386 +57387 POINT(29.051487076227723 -94.60398170775201) bank57387 +57388 POINT(29.256627888432547 -95.6797919225785) bank57388 +57389 POINT(29.60635642126386 -94.74750998072952) bank57389 +57390 POINT(30.1781596225812 -96.01806662197359) bank57390 +57391 POINT(29.89925311678461 -94.91185832990891) bank57391 +57392 POINT(30.078652385465084 -96.30995510551831) bank57392 +57393 POINT(30.394101645332928 -94.46687705294356) bank57393 +57394 POINT(30.228268234750686 -95.11147533665228) bank57394 +57395 POINT(29.223191235185094 -94.8145519356445) bank57395 +57396 POINT(30.376401376368463 -95.11055571626439) bank57396 +57397 POINT(29.762894324260866 -95.49107258130427) bank57397 +57398 POINT(29.63783107796281 -94.40787260493667) bank57398 +57399 POINT(29.98877118446917 -94.88987415889912) bank57399 +57400 POINT(29.27998033061487 -96.17705222064474) bank57400 +57401 POINT(29.681767844340072 -94.39930769539694) bank57401 +57402 POINT(29.82351408991655 -95.91041349568572) bank57402 +57403 POINT(30.379424958476072 -95.26280971006821) bank57403 +57404 POINT(29.90966074419312 -95.65458671826313) bank57404 +57405 POINT(30.72090742554507 -96.16721886245045) bank57405 +57406 POINT(29.770377983237076 -94.6392950581408) bank57406 +57407 POINT(29.2531472182529 -95.83086029815013) bank57407 +57408 POINT(28.939355179573944 -94.93867136715765) bank57408 +57409 POINT(29.949066738732288 -94.69681054101962) bank57409 +57410 POINT(30.00167419201209 -95.98599939687581) bank57410 +57411 POINT(30.12117792682329 -95.50498776785493) bank57411 +57412 POINT(29.351476147872887 -96.20691383180161) bank57412 +57413 POINT(29.555194140004474 -94.71383881298502) bank57413 +57414 POINT(28.921013940029763 -95.86667460736948) bank57414 +57415 POINT(29.476545676605536 -95.12236882502411) bank57415 +57416 POINT(29.044119383070377 -95.83014932366785) bank57416 +57417 POINT(30.740875879960765 -94.69225291631214) bank57417 +57418 POINT(29.607196785075118 -96.30455112537082) bank57418 +57419 POINT(29.296431726306704 -94.37873379126195) bank57419 +57420 POINT(28.86936165298622 -94.4319709195842) bank57420 +57421 POINT(30.06121684256993 -95.22740177799679) bank57421 +57422 POINT(29.87700660909231 -96.18322464534359) bank57422 +57423 POINT(30.24130027276848 -95.61009922046884) bank57423 +57424 POINT(30.051368931867675 -94.69626979218773) bank57424 +57425 POINT(30.369498654366815 -95.05779188057001) bank57425 +57426 POINT(29.64272070874741 -96.23132293272292) bank57426 +57427 POINT(30.32153378836326 -95.8738075628977) bank57427 +57428 POINT(29.35001902847618 -95.33521798911447) bank57428 +57429 POINT(29.378403956031036 -94.497808470964) bank57429 +57430 POINT(28.80936155050646 -94.45681230810156) bank57430 +57431 POINT(29.900741955331718 -95.94826881002582) bank57431 +57432 POINT(29.775797587279715 -94.97155363086108) bank57432 +57433 POINT(30.55284684312459 -94.83095864473539) bank57433 +57434 POINT(29.70241301252582 -95.44765343354659) bank57434 +57435 POINT(30.120840499430813 -94.55170891555542) bank57435 +57436 POINT(28.89595665797674 -95.94814068626604) bank57436 +57437 POINT(28.95081328988558 -95.90772458919264) bank57437 +57438 POINT(30.091124343841546 -94.9246566522379) bank57438 +57439 POINT(30.519762660810766 -94.40063476070921) bank57439 +57440 POINT(29.336410256455896 -95.17855455746587) bank57440 +57441 POINT(29.151810886311555 -95.48991794315758) bank57441 +57442 POINT(29.826128530385468 -95.31559940154175) bank57442 +57443 POINT(29.500218656126325 -95.44933013897757) bank57443 +57444 POINT(29.219989361521833 -94.51424020565904) bank57444 +57445 POINT(30.26608540108247 -96.02853536117598) bank57445 +57446 POINT(29.704979504830792 -96.09204279269917) bank57446 +57447 POINT(29.418391219644416 -95.80407977659713) bank57447 +57448 POINT(30.702029647623142 -95.10638336860599) bank57448 +57449 POINT(29.69479892907871 -94.65090231748783) bank57449 +57450 POINT(29.478705467329974 -94.38339319366625) bank57450 +57451 POINT(30.601143795244504 -96.31768907484708) bank57451 +57452 POINT(29.94953071273785 -95.43368713436332) bank57452 +57453 POINT(30.72228259673194 -95.8313529617932) bank57453 +57454 POINT(30.368034950074854 -96.22520007851222) bank57454 +57455 POINT(29.330460398522053 -95.99562772202597) bank57455 +57456 POINT(29.528692518281755 -95.01817805609673) bank57456 +57457 POINT(29.36339059612263 -95.38625263476064) bank57457 +57458 POINT(30.54155179762956 -96.1694208745248) bank57458 +57459 POINT(29.753062565020624 -95.68638508099444) bank57459 +57460 POINT(29.442051542856564 -96.17544820542615) bank57460 +57461 POINT(29.22560738877 -95.13346823234109) bank57461 +57462 POINT(30.496713799306846 -95.25795841657859) bank57462 +57463 POINT(29.514426414247392 -95.13765054000052) bank57463 +57464 POINT(29.118106530550815 -95.61770876432196) bank57464 +57465 POINT(30.44302830897311 -94.84917605820101) bank57465 +57466 POINT(29.32184740145444 -95.05902605662125) bank57466 +57467 POINT(29.32285845614151 -94.4426019741427) bank57467 +57468 POINT(28.813020995569836 -95.59553558660545) bank57468 +57469 POINT(30.435587133907863 -95.882590128365) bank57469 +57470 POINT(29.759655193991797 -95.53622211894677) bank57470 +57471 POINT(29.483369736209564 -95.97709620593865) bank57471 +57472 POINT(30.558234361138396 -96.02766122106748) bank57472 +57473 POINT(29.15222253569043 -94.72150280130373) bank57473 +57474 POINT(29.618169545151442 -95.85659926904898) bank57474 +57475 POINT(29.973024492291234 -94.6470678128514) bank57475 +57476 POINT(29.872783217611925 -96.35976699349285) bank57476 +57477 POINT(30.136545695955867 -95.81011850280021) bank57477 +57478 POINT(29.926088044734374 -95.53272684375244) bank57478 +57479 POINT(28.932968496784657 -94.89260830736832) bank57479 +57480 POINT(30.32139573298619 -96.01068602787596) bank57480 +57481 POINT(30.41240311172555 -95.97892085620192) bank57481 +57482 POINT(30.47751751673354 -94.57300053033829) bank57482 +57483 POINT(30.17290830357631 -94.43284572373156) bank57483 +57484 POINT(28.879650703137546 -95.46183707843744) bank57484 +57485 POINT(29.8988486997202 -94.87143201304643) bank57485 +57486 POINT(30.158640218458455 -95.55714949188611) bank57486 +57487 POINT(29.397778548160883 -95.91526524519561) bank57487 +57488 POINT(28.814616552349708 -94.90204306922504) bank57488 +57489 POINT(29.919484146950655 -95.68013408096196) bank57489 +57490 POINT(30.601912356191228 -95.11453600109832) bank57490 +57491 POINT(28.97142880864767 -96.25939940481264) bank57491 +57492 POINT(29.85232704903419 -94.39087125652739) bank57492 +57493 POINT(29.202570580103753 -94.93335008687257) bank57493 +57494 POINT(30.45889184660737 -94.5377249036076) bank57494 +57495 POINT(28.801517762848942 -95.1786742087113) bank57495 +57496 POINT(29.17716759849378 -96.32319231114653) bank57496 +57497 POINT(29.81673352109365 -94.40478858078549) bank57497 +57498 POINT(28.97485106577416 -95.53743665521785) bank57498 +57499 POINT(30.188974487593622 -95.93027111530057) bank57499 +57500 POINT(29.5932305617931 -96.07702993981019) bank57500 +57501 POINT(28.99235948040598 -94.9517897658626) bank57501 +57502 POINT(30.44391639289182 -96.14308456797374) bank57502 +57503 POINT(30.32139085157661 -95.57582421503119) bank57503 +57504 POINT(29.238839490299444 -95.47523278745965) bank57504 +57505 POINT(29.658608886458474 -94.40200584258794) bank57505 +57506 POINT(29.342406773099547 -94.66404501889714) bank57506 +57507 POINT(30.17658333220219 -95.0235014720643) bank57507 +57508 POINT(30.37525401649379 -94.42301854525434) bank57508 +57509 POINT(30.01416612698289 -95.16165220009412) bank57509 +57510 POINT(30.588085301273043 -95.72238505567604) bank57510 +57511 POINT(30.288351491289358 -94.95867233226983) bank57511 +57512 POINT(30.065428081024965 -95.912460537851) bank57512 +57513 POINT(29.805448524387106 -94.41515695336784) bank57513 +57514 POINT(29.063026973606046 -94.4571187301477) bank57514 +57515 POINT(30.411603946619646 -95.02998344580465) bank57515 +57516 POINT(29.274235352784927 -94.65802252553067) bank57516 +57517 POINT(29.956190962769572 -94.9345938326891) bank57517 +57518 POINT(30.43525794290201 -95.39091185497716) bank57518 +57519 POINT(29.397375628007243 -95.10242733403703) bank57519 +57520 POINT(30.33741027992732 -96.24105149430001) bank57520 +57521 POINT(30.588968524060775 -94.63789349839266) bank57521 +57522 POINT(29.803407114393206 -95.16205590660229) bank57522 +57523 POINT(28.802747162126053 -94.57122040541533) bank57523 +57524 POINT(28.89341952287878 -96.08145027163438) bank57524 +57525 POINT(30.458610484190466 -94.99915636664487) bank57525 +57526 POINT(30.26235055116376 -95.01427899980943) bank57526 +57527 POINT(28.87498159749201 -94.47776416149692) bank57527 +57528 POINT(29.487054912559067 -95.46172506988985) bank57528 +57529 POINT(29.482775577959277 -95.23113139780659) bank57529 +57530 POINT(29.72371919093382 -96.09819615418778) bank57530 +57531 POINT(30.716958130751774 -95.20504021113528) bank57531 +57532 POINT(29.011240013624906 -95.90170956162683) bank57532 +57533 POINT(29.647943421269552 -94.98697507181215) bank57533 +57534 POINT(29.457136709883898 -94.81790432268521) bank57534 +57535 POINT(29.42760580234854 -96.06778316220365) bank57535 +57536 POINT(28.783818269432025 -95.70334398765264) bank57536 +57537 POINT(30.25415904102936 -94.846105818184) bank57537 +57538 POINT(30.05627985150639 -94.81136144915324) bank57538 +57539 POINT(29.03049422488297 -94.37800259358403) bank57539 +57540 POINT(28.80669972458089 -94.82782165398484) bank57540 +57541 POINT(29.215077162686065 -95.01529677579363) bank57541 +57542 POINT(28.85051834988289 -95.5781467230383) bank57542 +57543 POINT(30.26410698452103 -95.75393724597336) bank57543 +57544 POINT(29.53294363356073 -95.68434885669588) bank57544 +57545 POINT(28.834937954884175 -94.82219991276814) bank57545 +57546 POINT(30.16675339108912 -96.19758854835545) bank57546 +57547 POINT(29.713480839384292 -96.06821905605312) bank57547 +57548 POINT(29.561975288146524 -96.05926044181034) bank57548 +57549 POINT(30.437075693626014 -94.9906551040695) bank57549 +57550 POINT(30.17922761557209 -95.95410027053978) bank57550 +57551 POINT(29.86668339861661 -95.2943059158074) bank57551 +57552 POINT(28.891102353955596 -95.73837979157095) bank57552 +57553 POINT(28.77895572088434 -96.3682463410275) bank57553 +57554 POINT(30.422696020106397 -95.45047497129623) bank57554 +57555 POINT(29.129717740804036 -94.8327290392977) bank57555 +57556 POINT(29.89341598129605 -95.37017489449651) bank57556 +57557 POINT(29.469978268148374 -95.1119421262769) bank57557 +57558 POINT(29.631557157487094 -94.58688865083184) bank57558 +57559 POINT(30.032760248494817 -95.93265784779771) bank57559 +57560 POINT(30.392544165325212 -94.82247794433682) bank57560 +57561 POINT(29.32065039512958 -95.12177492411834) bank57561 +57562 POINT(28.790816513959836 -95.59161080008913) bank57562 +57563 POINT(30.65349297372273 -96.3604857344665) bank57563 +57564 POINT(29.750409249532424 -94.64748391250329) bank57564 +57565 POINT(30.098254780927448 -95.78701315940526) bank57565 +57566 POINT(30.482355906053492 -94.821947350218) bank57566 +57567 POINT(29.83843011726402 -94.99269302217078) bank57567 +57568 POINT(29.614292968135853 -96.00681027776056) bank57568 +57569 POINT(29.19008498951491 -95.31470699995945) bank57569 +57570 POINT(29.047099297370657 -94.816890491103) bank57570 +57571 POINT(29.11108547334646 -96.20021101666697) bank57571 +57572 POINT(28.792031413239084 -94.75423831251435) bank57572 +57573 POINT(29.17497330045064 -95.28305153804726) bank57573 +57574 POINT(29.345553158334443 -95.31188583015906) bank57574 +57575 POINT(29.595211199060937 -94.8836104688895) bank57575 +57576 POINT(29.49251435721172 -95.01022099585012) bank57576 +57577 POINT(30.619866297813747 -95.04899521415096) bank57577 +57578 POINT(29.57645880366507 -95.48649792302051) bank57578 +57579 POINT(30.597723833411646 -94.72612864975758) bank57579 +57580 POINT(29.537510943976077 -95.17017631672182) bank57580 +57581 POINT(29.608715808392112 -95.27079802246863) bank57581 +57582 POINT(28.988448431583418 -94.72926683051594) bank57582 +57583 POINT(29.34610242807913 -94.90503182198752) bank57583 +57584 POINT(29.274110269819285 -95.65688358528412) bank57584 +57585 POINT(29.996548859683394 -95.989091491942) bank57585 +57586 POINT(29.96106405457682 -95.15539075510151) bank57586 +57587 POINT(29.329509267636237 -95.86248987244318) bank57587 +57588 POINT(30.61366542906791 -94.99558872982975) bank57588 +57589 POINT(29.953747697559265 -95.04477378342872) bank57589 +57590 POINT(29.460229337682243 -94.69159582750989) bank57590 +57591 POINT(30.68240559208312 -96.31325743304498) bank57591 +57592 POINT(29.986983280197094 -94.96846767448893) bank57592 +57593 POINT(29.333514850685503 -95.43543153068163) bank57593 +57594 POINT(30.753024130355804 -94.85422619967031) bank57594 +57595 POINT(29.82956667186848 -94.98972011979306) bank57595 +57596 POINT(28.79752234622251 -94.6376533545604) bank57596 +57597 POINT(29.990576782704782 -95.22838180495503) bank57597 +57598 POINT(29.990831183363028 -94.74127502037659) bank57598 +57599 POINT(29.39633399361059 -96.30349173866024) bank57599 +57600 POINT(28.809788181887303 -95.7197514207464) bank57600 +57601 POINT(29.399854134588296 -96.33603883219837) bank57601 +57602 POINT(28.89853443520308 -95.43899108406657) bank57602 +57603 POINT(28.991030699662176 -96.26160310993849) bank57603 +57604 POINT(30.497127068234352 -95.77072456658304) bank57604 +57605 POINT(28.99171036658943 -95.06181251423848) bank57605 +57606 POINT(30.42265481182735 -96.19332118123168) bank57606 +57607 POINT(29.35057066668461 -94.42121271652191) bank57607 +57608 POINT(29.23187628139411 -94.60619388986987) bank57608 +57609 POINT(30.6449168133731 -94.8309216727793) bank57609 +57610 POINT(30.729115038894506 -96.35618754769757) bank57610 +57611 POINT(29.191229103973022 -94.78459863804683) bank57611 +57612 POINT(29.096641364259 -95.17825949814198) bank57612 +57613 POINT(29.53890488255781 -96.07882812448119) bank57613 +57614 POINT(30.54907984354702 -95.17938080349381) bank57614 +57615 POINT(29.22612603154817 -96.23483365665695) bank57615 +57616 POINT(30.441323898700816 -94.963056339198) bank57616 +57617 POINT(29.386739086036545 -94.37656027199044) bank57617 +57618 POINT(30.52456154624057 -96.01069207850264) bank57618 +57619 POINT(29.70276116993974 -95.50405316732733) bank57619 +57620 POINT(30.144480118811742 -95.19179889350438) bank57620 +57621 POINT(29.02356280220047 -94.53992849620244) bank57621 +57622 POINT(29.008681363203486 -95.9996885482898) bank57622 +57623 POINT(28.846478874310407 -95.81646820462484) bank57623 +57624 POINT(29.274004155948848 -95.79535589734175) bank57624 +57625 POINT(30.33798377701378 -95.71183866264661) bank57625 +57626 POINT(28.99391091785847 -94.58037847053119) bank57626 +57627 POINT(29.241428975559568 -96.19089409882267) bank57627 +57628 POINT(28.834783709128015 -94.90830915359642) bank57628 +57629 POINT(30.073090042895938 -94.42664772242183) bank57629 +57630 POINT(30.489916634403283 -94.38960724255972) bank57630 +57631 POINT(29.015469393251372 -94.81810047514806) bank57631 +57632 POINT(29.61788509200449 -95.58122915126005) bank57632 +57633 POINT(29.539256857577513 -95.21915262938894) bank57633 +57634 POINT(28.79335773789108 -94.70986730911348) bank57634 +57635 POINT(30.04971472016686 -94.66809585120974) bank57635 +57636 POINT(30.4627228057813 -96.28207932181869) bank57636 +57637 POINT(29.140452678579404 -94.7001344951111) bank57637 +57638 POINT(29.55821932758978 -95.53684623700508) bank57638 +57639 POINT(30.604715373324108 -94.38692686618761) bank57639 +57640 POINT(30.168741138956612 -95.81856859731097) bank57640 +57641 POINT(29.691962271959305 -95.87150430053063) bank57641 +57642 POINT(30.72884674193645 -96.31837314811332) bank57642 +57643 POINT(30.202010132894856 -95.77650814324474) bank57643 +57644 POINT(30.694782922064846 -94.66819058392076) bank57644 +57645 POINT(29.78168327891093 -95.53524730011551) bank57645 +57646 POINT(30.16193086726609 -95.49176231135823) bank57646 +57647 POINT(29.39654630607535 -95.73945190402908) bank57647 +57648 POINT(30.750655186184392 -95.17697665238272) bank57648 +57649 POINT(29.38845678452352 -94.96354255998882) bank57649 +57650 POINT(29.640753682165688 -95.73659836664699) bank57650 +57651 POINT(29.64505553997904 -95.79012728909643) bank57651 +57652 POINT(28.858900146879797 -94.7895236475762) bank57652 +57653 POINT(29.104783396100153 -94.89265808185905) bank57653 +57654 POINT(29.534823946682824 -96.07411448141967) bank57654 +57655 POINT(29.65996501198801 -94.53900114519116) bank57655 +57656 POINT(29.474851509356803 -95.59578191458748) bank57656 +57657 POINT(29.838028064567673 -95.59176180207403) bank57657 +57658 POINT(29.618078711411773 -95.50516736548082) bank57658 +57659 POINT(30.64436877457168 -95.27841193492894) bank57659 +57660 POINT(30.7572011251836 -94.79988989199022) bank57660 +57661 POINT(29.753803956375847 -94.56075507449195) bank57661 +57662 POINT(30.444800209185594 -96.1648143035816) bank57662 +57663 POINT(29.961467534448964 -95.67688447431641) bank57663 +57664 POINT(30.074275108559522 -96.0444327843553) bank57664 +57665 POINT(30.49387180318882 -96.29436479511467) bank57665 +57666 POINT(29.46849589263281 -95.9595102480988) bank57666 +57667 POINT(30.752691759670398 -95.04540476036715) bank57667 +57668 POINT(28.91436224432434 -96.06618985612262) bank57668 +57669 POINT(30.218178129907287 -94.53218384296255) bank57669 +57670 POINT(29.63234159610434 -95.11884245642372) bank57670 +57671 POINT(29.705948332322095 -95.31370352770918) bank57671 +57672 POINT(30.501439251428835 -95.36176715458288) bank57672 +57673 POINT(29.582406906611354 -95.89765650333962) bank57673 +57674 POINT(29.928656745742366 -95.05679410141603) bank57674 +57675 POINT(29.563966541203637 -95.69349856838285) bank57675 +57676 POINT(29.68962651823901 -94.64489373513653) bank57676 +57677 POINT(30.27292096948917 -95.9875633266009) bank57677 +57678 POINT(29.795774520882905 -94.5912107864182) bank57678 +57679 POINT(30.633773505997972 -95.10300373604497) bank57679 +57680 POINT(29.15288146538158 -95.07296975721115) bank57680 +57681 POINT(30.321751073065304 -95.76466220020052) bank57681 +57682 POINT(29.686044962570595 -95.54428012283933) bank57682 +57683 POINT(30.22030855368058 -94.40477312051516) bank57683 +57684 POINT(29.007424503386453 -95.16745503790011) bank57684 +57685 POINT(29.188769574036407 -95.2170252042665) bank57685 +57686 POINT(29.678614515641733 -95.32886143132163) bank57686 +57687 POINT(30.37594759980177 -95.48251199781954) bank57687 +57688 POINT(29.446726732296863 -96.3440845275912) bank57688 +57689 POINT(29.578397486561617 -94.91652867171267) bank57689 +57690 POINT(30.45298197433974 -96.12939753692689) bank57690 +57691 POINT(29.16482295280554 -94.42868642944454) bank57691 +57692 POINT(28.78606826588983 -94.81632953074278) bank57692 +57693 POINT(29.94802243871581 -95.59360670837802) bank57693 +57694 POINT(29.975934958237964 -95.5449388619403) bank57694 +57695 POINT(30.39013919096987 -96.14623909895042) bank57695 +57696 POINT(29.045375959593095 -94.7453508793824) bank57696 +57697 POINT(29.176617142516026 -95.08100777681985) bank57697 +57698 POINT(30.585509698363207 -95.60925546644374) bank57698 +57699 POINT(29.100033514821263 -95.23479561684688) bank57699 +57700 POINT(28.932654099132748 -96.06151553598802) bank57700 +57701 POINT(29.178463169512188 -94.42627669738481) bank57701 +57702 POINT(30.133433130946294 -94.46210983046238) bank57702 +57703 POINT(29.19827250181027 -95.93431854941004) bank57703 +57704 POINT(30.755411445328026 -95.43503253936692) bank57704 +57705 POINT(29.460478884220585 -94.55341173264196) bank57705 +57706 POINT(30.44206909281885 -95.57036494861472) bank57706 +57707 POINT(29.312993260202628 -94.48038724726099) bank57707 +57708 POINT(29.422932389248842 -96.00887197701847) bank57708 +57709 POINT(29.715799687676718 -94.52783001363456) bank57709 +57710 POINT(29.605387458028332 -95.72491667593427) bank57710 +57711 POINT(29.81061498527579 -94.41705183683405) bank57711 +57712 POINT(29.38904123415088 -95.56913147116805) bank57712 +57713 POINT(30.20488742295943 -94.44031369216384) bank57713 +57714 POINT(30.711135238426493 -95.362435843496) bank57714 +57715 POINT(30.18603868977025 -94.77146607433697) bank57715 +57716 POINT(29.75331835885327 -95.82972411780517) bank57716 +57717 POINT(30.383984638831002 -95.26123701271462) bank57717 +57718 POINT(29.98536000283443 -94.42713057733847) bank57718 +57719 POINT(29.63500293485518 -94.88671077485019) bank57719 +57720 POINT(29.40130971581018 -96.0917726418262) bank57720 +57721 POINT(29.44558146544782 -94.46522332335746) bank57721 +57722 POINT(29.75330947195591 -94.83230380456656) bank57722 +57723 POINT(29.45142841440118 -95.53920444645811) bank57723 +57724 POINT(29.12958941459032 -96.32034601636943) bank57724 +57725 POINT(28.825315529762666 -96.23633765206795) bank57725 +57726 POINT(30.201011046571534 -95.71542263267457) bank57726 +57727 POINT(29.932102194124194 -95.28619534297214) bank57727 +57728 POINT(30.038076206791406 -95.5975693968734) bank57728 +57729 POINT(29.83316169275756 -95.87536117783878) bank57729 +57730 POINT(30.18709361108509 -95.54678542954683) bank57730 +57731 POINT(29.02222902970391 -95.08471623600465) bank57731 +57732 POINT(29.88705488735966 -94.99728215885621) bank57732 +57733 POINT(30.578285904877532 -94.68588116322087) bank57733 +57734 POINT(29.543869529868594 -94.69213540583527) bank57734 +57735 POINT(30.516163880850232 -94.61968979311817) bank57735 +57736 POINT(30.656386406022126 -95.4123388887105) bank57736 +57737 POINT(29.708572823751467 -94.89132939824425) bank57737 +57738 POINT(30.3270632448044 -94.95889383806202) bank57738 +57739 POINT(29.241536522333007 -94.74103264980646) bank57739 +57740 POINT(29.919801603028464 -94.93395518567227) bank57740 +57741 POINT(30.659610914443768 -94.69629109411896) bank57741 +57742 POINT(28.7698742561324 -96.07421706574736) bank57742 +57743 POINT(30.07336830679956 -94.54840716481428) bank57743 +57744 POINT(28.963805853180705 -95.18541154984422) bank57744 +57745 POINT(30.512031748274623 -95.01269501366913) bank57745 +57746 POINT(29.593768342511215 -96.12961244276039) bank57746 +57747 POINT(29.394138441778182 -95.92342424084386) bank57747 +57748 POINT(30.63841493312566 -96.02140664638566) bank57748 +57749 POINT(30.599479584947012 -94.88936273106991) bank57749 +57750 POINT(30.175434611709484 -95.817274429547) bank57750 +57751 POINT(30.24823710629626 -95.19521682003462) bank57751 +57752 POINT(29.20234850598728 -96.11709798875667) bank57752 +57753 POINT(30.016546546078175 -95.39341764744509) bank57753 +57754 POINT(30.2052993411551 -95.34550553904576) bank57754 +57755 POINT(29.896285922216006 -94.65370716233768) bank57755 +57756 POINT(28.8934807901685 -94.79910942145959) bank57756 +57757 POINT(29.735071667894626 -95.70845932563358) bank57757 +57758 POINT(30.03970409950615 -95.41546877269295) bank57758 +57759 POINT(29.097000981713734 -94.40465089063538) bank57759 +57760 POINT(30.079942900768565 -95.17243155504862) bank57760 +57761 POINT(28.845264120694182 -95.43823457220496) bank57761 +57762 POINT(29.308287065219996 -94.45193598407529) bank57762 +57763 POINT(28.834365665186358 -94.38363097188154) bank57763 +57764 POINT(28.787497987112367 -96.00581524982077) bank57764 +57765 POINT(29.58126221070081 -95.72314476120664) bank57765 +57766 POINT(30.191672936499423 -94.83840392953037) bank57766 +57767 POINT(30.630754660005344 -94.705176459535) bank57767 +57768 POINT(29.99137369282475 -95.56774984471753) bank57768 +57769 POINT(30.73998847858924 -95.41327618436279) bank57769 +57770 POINT(28.944636404664227 -95.49490782109885) bank57770 +57771 POINT(29.243966475683944 -95.25134425672606) bank57771 +57772 POINT(30.180081080826692 -95.68427753569374) bank57772 +57773 POINT(29.39224295991262 -95.62292958500291) bank57773 +57774 POINT(29.31051293135798 -96.17455128843274) bank57774 +57775 POINT(29.900141675487976 -94.55272370063695) bank57775 +57776 POINT(29.6625081271977 -95.80578999688714) bank57776 +57777 POINT(30.496392302465086 -94.95878226700987) bank57777 +57778 POINT(29.65150457666109 -94.66945922102731) bank57778 +57779 POINT(29.64735679170572 -95.07121294025534) bank57779 +57780 POINT(28.92717380245545 -94.75919564211704) bank57780 +57781 POINT(30.41121446256304 -95.64629739346954) bank57781 +57782 POINT(29.113573704094254 -94.48584330443589) bank57782 +57783 POINT(30.027774571898817 -94.48349417946496) bank57783 +57784 POINT(29.802442747616105 -94.83255765760147) bank57784 +57785 POINT(29.926834549434577 -95.72356104480406) bank57785 +57786 POINT(29.17877786958109 -95.34133626762147) bank57786 +57787 POINT(30.046321261861983 -94.60968900508954) bank57787 +57788 POINT(29.45245072956364 -95.6870185100343) bank57788 +57789 POINT(30.41658974048038 -96.10753368652101) bank57789 +57790 POINT(29.782865684971554 -95.61522668479817) bank57790 +57791 POINT(30.491919614425747 -94.50864602151712) bank57791 +57792 POINT(29.481443979222114 -96.32807651076926) bank57792 +57793 POINT(30.47275491896484 -96.3400001634) bank57793 +57794 POINT(29.239635428937117 -95.39055315243034) bank57794 +57795 POINT(29.932179043933395 -95.66338157774858) bank57795 +57796 POINT(29.35674117913934 -94.61542872374748) bank57796 +57797 POINT(29.29094284086126 -94.82415421608005) bank57797 +57798 POINT(30.47796038229664 -95.97354470526041) bank57798 +57799 POINT(29.864176727461036 -95.51631560848168) bank57799 +57800 POINT(30.46801667257654 -96.21850831657098) bank57800 +57801 POINT(29.146923436209416 -95.22774799712705) bank57801 +57802 POINT(29.0287022094491 -95.741516379133) bank57802 +57803 POINT(29.263086628807095 -95.72059480195158) bank57803 +57804 POINT(30.23357086612443 -95.97050893907237) bank57804 +57805 POINT(29.85814266410989 -94.67657948461294) bank57805 +57806 POINT(29.530394512026525 -94.807124129678) bank57806 +57807 POINT(29.89407534226045 -94.8888698771281) bank57807 +57808 POINT(28.99559507074874 -96.17697875150989) bank57808 +57809 POINT(29.6064765751246 -96.10151765772765) bank57809 +57810 POINT(29.052604694282973 -95.13344574634183) bank57810 +57811 POINT(29.275103676426422 -95.14876448904347) bank57811 +57812 POINT(29.712199437816025 -94.74511183076146) bank57812 +57813 POINT(29.001691070140843 -95.3560365106132) bank57813 +57814 POINT(29.956710203464787 -95.5358714082344) bank57814 +57815 POINT(30.111777676724465 -95.93428139367468) bank57815 +57816 POINT(28.86928206445467 -95.42282186742906) bank57816 +57817 POINT(28.92585002321012 -96.13580882894333) bank57817 +57818 POINT(29.974841525229134 -95.66630330313835) bank57818 +57819 POINT(30.717890121021018 -96.2450463431355) bank57819 +57820 POINT(28.96738197308819 -94.68579401046634) bank57820 +57821 POINT(28.802987708618144 -94.8155846795714) bank57821 +57822 POINT(30.160967195929707 -95.32925420041109) bank57822 +57823 POINT(29.211126130372048 -95.02418385827667) bank57823 +57824 POINT(29.884635796113304 -95.95861237984242) bank57824 +57825 POINT(30.560793894802842 -95.09714319213519) bank57825 +57826 POINT(30.621492061293033 -96.2973300969318) bank57826 +57827 POINT(29.52713138894662 -94.62981289970793) bank57827 +57828 POINT(29.93960423360037 -95.30735556649063) bank57828 +57829 POINT(29.527577970825295 -95.67685781500646) bank57829 +57830 POINT(28.91857734990689 -95.09502418807381) bank57830 +57831 POINT(29.12428644500004 -96.16431719246421) bank57831 +57832 POINT(29.623907773428513 -96.1073990258522) bank57832 +57833 POINT(29.213513980876744 -94.7872526463107) bank57833 +57834 POINT(29.312073230290135 -95.82128632576324) bank57834 +57835 POINT(30.10340457139413 -95.09146579533433) bank57835 +57836 POINT(29.78575567338645 -95.27845923066948) bank57836 +57837 POINT(29.086005766960234 -94.96670805303133) bank57837 +57838 POINT(29.98848119273962 -96.05116984482018) bank57838 +57839 POINT(29.112855714622306 -95.03550437282219) bank57839 +57840 POINT(29.51111270162414 -95.25257711160081) bank57840 +57841 POINT(29.02310561117328 -96.18653135536236) bank57841 +57842 POINT(29.138652306621648 -95.57504096307679) bank57842 +57843 POINT(30.573130734852192 -95.33043793620371) bank57843 +57844 POINT(30.486098850973605 -94.79245448973403) bank57844 +57845 POINT(29.414727095529347 -95.56653419527584) bank57845 +57846 POINT(30.563029963241846 -95.72213325752604) bank57846 +57847 POINT(29.467479887815518 -95.83514994181219) bank57847 +57848 POINT(30.512198075076974 -95.90912345624986) bank57848 +57849 POINT(30.10701123931309 -95.45517709371491) bank57849 +57850 POINT(30.661692414735754 -94.64020567863908) bank57850 +57851 POINT(30.197987470227616 -95.37811295542981) bank57851 +57852 POINT(29.825081713975106 -94.6843023417007) bank57852 +57853 POINT(30.66792144670837 -95.17272714411217) bank57853 +57854 POINT(28.850335899856084 -95.93548235603588) bank57854 +57855 POINT(28.94890971171116 -94.40810974079591) bank57855 +57856 POINT(28.960659528350245 -94.42906940724843) bank57856 +57857 POINT(30.155856965950985 -94.69564279758087) bank57857 +57858 POINT(30.62075312601041 -95.94287480756238) bank57858 +57859 POINT(29.09266536174578 -96.00169556827801) bank57859 +57860 POINT(30.276007199879974 -96.32503604046101) bank57860 +57861 POINT(29.403249521879093 -95.68862087868214) bank57861 +57862 POINT(30.199041313072744 -94.89323524150612) bank57862 +57863 POINT(29.46987825807524 -96.12431569921803) bank57863 +57864 POINT(29.525777915606554 -96.26870948395323) bank57864 +57865 POINT(28.900747165977666 -95.38745065512904) bank57865 +57866 POINT(30.366660609697167 -95.22837985805624) bank57866 +57867 POINT(29.786256620020737 -96.08752913182471) bank57867 +57868 POINT(28.97790195228796 -95.72790625521343) bank57868 +57869 POINT(30.019369455208874 -95.3744825227446) bank57869 +57870 POINT(29.317423247016162 -95.84505113840417) bank57870 +57871 POINT(29.270778543846752 -95.77224261423808) bank57871 +57872 POINT(30.32181045586354 -95.89274381220172) bank57872 +57873 POINT(30.705219265714703 -94.79936561863961) bank57873 +57874 POINT(29.104050911260536 -94.4323883715539) bank57874 +57875 POINT(29.07409229985456 -95.08275941080036) bank57875 +57876 POINT(30.50588902289667 -95.0207484964529) bank57876 +57877 POINT(30.307700636569965 -95.42805387722963) bank57877 +57878 POINT(29.682460900126227 -95.83352287730064) bank57878 +57879 POINT(29.04517001518417 -95.65131532153258) bank57879 +57880 POINT(29.81991051733014 -96.1384062722945) bank57880 +57881 POINT(29.430286871511107 -94.59578444564416) bank57881 +57882 POINT(30.163865633693835 -96.18887407106882) bank57882 +57883 POINT(29.04508494113885 -95.11787927608941) bank57883 +57884 POINT(30.352019790211948 -95.61863386537377) bank57884 +57885 POINT(30.04732608993496 -95.97143301167134) bank57885 +57886 POINT(30.490341433427062 -95.87057284023669) bank57886 +57887 POINT(30.30688068602817 -96.06985362314323) bank57887 +57888 POINT(30.39881807749655 -94.90286770375329) bank57888 +57889 POINT(29.98637255289181 -95.06583619974772) bank57889 +57890 POINT(29.475507139645494 -95.14582438936502) bank57890 +57891 POINT(29.56796288707683 -95.13702631738093) bank57891 +57892 POINT(28.821853166546955 -95.53345370781938) bank57892 +57893 POINT(29.939699864662444 -96.33995392938782) bank57893 +57894 POINT(30.690643862009697 -95.76369432981467) bank57894 +57895 POINT(29.472368233347154 -95.32384192780343) bank57895 +57896 POINT(30.335299443831378 -96.08287425756328) bank57896 +57897 POINT(30.14942714652376 -96.29275537568842) bank57897 +57898 POINT(30.223067753667053 -94.63476036783076) bank57898 +57899 POINT(29.862334874285704 -94.64880781109582) bank57899 +57900 POINT(28.881791803664765 -95.33307416971756) bank57900 +57901 POINT(28.800931880556586 -94.58649128639813) bank57901 +57902 POINT(30.097919158956795 -94.42183662830833) bank57902 +57903 POINT(30.02567189500381 -94.86174093717558) bank57903 +57904 POINT(28.813551014012354 -95.69118223166197) bank57904 +57905 POINT(29.621747616496897 -96.22797246607335) bank57905 +57906 POINT(29.777857318998464 -95.59562130635747) bank57906 +57907 POINT(30.352810959834542 -95.68180168024134) bank57907 +57908 POINT(30.439377183632917 -95.66325819752574) bank57908 +57909 POINT(30.535779690905727 -95.72201094763196) bank57909 +57910 POINT(29.182584607099646 -94.79292860763037) bank57910 +57911 POINT(29.238310575531912 -94.72622191953012) bank57911 +57912 POINT(29.39300083361207 -96.30502394460397) bank57912 +57913 POINT(29.480842285047455 -95.05283680813315) bank57913 +57914 POINT(30.114460051053655 -95.86503891681735) bank57914 +57915 POINT(29.620522668322014 -95.96747202367449) bank57915 +57916 POINT(29.473989520106862 -95.96660490089619) bank57916 +57917 POINT(29.637746215402164 -95.99241954390523) bank57917 +57918 POINT(29.588840339080622 -95.71970595573515) bank57918 +57919 POINT(30.319777052152855 -96.03757213261801) bank57919 +57920 POINT(30.10976319760312 -94.93581161431848) bank57920 +57921 POINT(30.200841384233023 -96.00450606751683) bank57921 +57922 POINT(29.10515789682278 -95.89858726956821) bank57922 +57923 POINT(30.033373357847502 -95.37734182905986) bank57923 +57924 POINT(29.660604441360043 -94.9137153896142) bank57924 +57925 POINT(29.212113354617443 -95.45775682774601) bank57925 +57926 POINT(29.92761101917563 -95.23492117558081) bank57926 +57927 POINT(30.747565941480584 -96.31328382601758) bank57927 +57928 POINT(29.74968824489319 -95.67947221161351) bank57928 +57929 POINT(28.95038056399453 -96.27225017464208) bank57929 +57930 POINT(28.80035104370593 -94.40000164820023) bank57930 +57931 POINT(29.290269231657135 -95.85836801381505) bank57931 +57932 POINT(30.18498079714694 -96.11173104297657) bank57932 +57933 POINT(29.43305247921933 -94.42561905339295) bank57933 +57934 POINT(30.028495922270846 -95.67604646621577) bank57934 +57935 POINT(29.53926530257512 -94.84826435760436) bank57935 +57936 POINT(29.958063821331194 -94.74783305525501) bank57936 +57937 POINT(29.41084425238793 -94.71559108284599) bank57937 +57938 POINT(30.68205719335743 -94.7617672831202) bank57938 +57939 POINT(29.413066086742255 -96.1534178072537) bank57939 +57940 POINT(30.137598096336575 -94.9763830510376) bank57940 +57941 POINT(30.50574284968939 -95.09650556886544) bank57941 +57942 POINT(30.252420674640927 -96.0309576559835) bank57942 +57943 POINT(29.69018932979563 -95.42062195082529) bank57943 +57944 POINT(30.747809993540336 -95.24430120113078) bank57944 +57945 POINT(29.958929062116653 -95.53374462955804) bank57945 +57946 POINT(28.945930095751116 -94.39784454598784) bank57946 +57947 POINT(29.36171994424087 -95.03648167144343) bank57947 +57948 POINT(29.679243558587242 -95.28620477424093) bank57948 +57949 POINT(29.166829601476508 -96.21523474324226) bank57949 +57950 POINT(28.988272586338233 -95.09025414175423) bank57950 +57951 POINT(30.219970674673547 -95.90680069141764) bank57951 +57952 POINT(29.977267768366815 -94.66756254716816) bank57952 +57953 POINT(30.624616738562107 -95.27360253648668) bank57953 +57954 POINT(30.29043361580899 -95.49554050553674) bank57954 +57955 POINT(29.377807357501766 -95.2698602373927) bank57955 +57956 POINT(29.338532007901247 -95.33963756146045) bank57956 +57957 POINT(29.581836210526365 -94.86674341943359) bank57957 +57958 POINT(30.63492858729358 -95.94682773813308) bank57958 +57959 POINT(30.10924713247401 -94.90917721097249) bank57959 +57960 POINT(30.631038390951584 -94.64482795496038) bank57960 +57961 POINT(30.708988799210225 -94.45639613550709) bank57961 +57962 POINT(29.294002337912744 -94.37857909572267) bank57962 +57963 POINT(29.866968620851818 -94.69131249725682) bank57963 +57964 POINT(28.811719953035368 -95.40985330366024) bank57964 +57965 POINT(29.759100795748626 -95.74685786930374) bank57965 +57966 POINT(29.86683547056532 -94.53384398224934) bank57966 +57967 POINT(30.08761080503685 -95.93023776749416) bank57967 +57968 POINT(29.5456613665628 -96.14668132678483) bank57968 +57969 POINT(29.02675718055037 -94.73793726187262) bank57969 +57970 POINT(28.926636776663024 -94.5900619547329) bank57970 +57971 POINT(29.406076553546313 -95.88683060636808) bank57971 +57972 POINT(30.37124393472224 -95.09602904249338) bank57972 +57973 POINT(30.199991293801894 -95.03367508189245) bank57973 +57974 POINT(30.391247304926875 -94.61933732430694) bank57974 +57975 POINT(29.522084642536857 -95.92857362918372) bank57975 +57976 POINT(30.660270221199788 -94.78779048872275) bank57976 +57977 POINT(29.539444425431345 -96.2204079974725) bank57977 +57978 POINT(29.200968431079566 -94.40399300906049) bank57978 +57979 POINT(29.79541298670574 -95.32480290712415) bank57979 +57980 POINT(28.85492109650309 -95.39600307858134) bank57980 +57981 POINT(29.108289745184997 -94.61597708325704) bank57981 +57982 POINT(28.99193019811588 -94.49548529291765) bank57982 +57983 POINT(29.080988302881046 -94.49578773742411) bank57983 +57984 POINT(29.989651564554922 -95.70745553572715) bank57984 +57985 POINT(29.194704686761728 -96.15652282070756) bank57985 +57986 POINT(30.729559204998843 -95.28700215048184) bank57986 +57987 POINT(29.594813670807007 -95.95004935606593) bank57987 +57988 POINT(29.73701117966472 -95.55120180085017) bank57988 +57989 POINT(29.855634795199368 -94.97668905411358) bank57989 +57990 POINT(29.50131981491318 -96.2413316155124) bank57990 +57991 POINT(28.945176057801856 -95.49540449138925) bank57991 +57992 POINT(30.744848267674303 -95.47570882969993) bank57992 +57993 POINT(30.632414570268264 -94.50875501053679) bank57993 +57994 POINT(29.391111136892402 -94.85895111677459) bank57994 +57995 POINT(29.80669219697088 -95.84138099114492) bank57995 +57996 POINT(29.273096798609938 -95.57764043064626) bank57996 +57997 POINT(29.224299524531084 -95.74540125596067) bank57997 +57998 POINT(29.62028254039205 -95.53556160559066) bank57998 +57999 POINT(30.244424010631104 -95.6782275246337) bank57999 +58000 POINT(30.272079607633128 -94.76656654053768) bank58000 +58001 POINT(29.268208404171776 -95.78356050312017) bank58001 +58002 POINT(29.114927611386992 -94.64054672926794) bank58002 +58003 POINT(30.718638072194597 -96.00738449958891) bank58003 +58004 POINT(28.947987615844774 -95.69907258755501) bank58004 +58005 POINT(30.61543700120396 -96.29141438361886) bank58005 +58006 POINT(30.429620402503232 -95.15390727299435) bank58006 +58007 POINT(29.018030661497377 -94.92008769663323) bank58007 +58008 POINT(30.066348974675464 -95.61874323874645) bank58008 +58009 POINT(29.594622406274695 -96.30861381565295) bank58009 +58010 POINT(29.513736983519035 -95.93206061198376) bank58010 +58011 POINT(30.479776946248336 -96.14345727520853) bank58011 +58012 POINT(28.82987396309034 -95.58549672671369) bank58012 +58013 POINT(30.39365267253405 -94.68679827666628) bank58013 +58014 POINT(30.27799091706821 -94.3973801813054) bank58014 +58015 POINT(29.482541342233514 -94.8272838813786) bank58015 +58016 POINT(29.150978157201365 -94.78452065366145) bank58016 +58017 POINT(29.79242295273541 -94.9109989958841) bank58017 +58018 POINT(30.713483774647624 -96.25031855225467) bank58018 +58019 POINT(28.786734167895734 -95.84192866566093) bank58019 +58020 POINT(30.753078600708033 -96.25223793487923) bank58020 +58021 POINT(29.949122258320322 -94.91358401023722) bank58021 +58022 POINT(30.184211258300643 -95.99109114227167) bank58022 +58023 POINT(29.603453121744977 -94.65226283561417) bank58023 +58024 POINT(30.243071452939517 -95.04757407245509) bank58024 +58025 POINT(29.162534600239557 -95.14746126921835) bank58025 +58026 POINT(28.82152170622246 -96.06921136841466) bank58026 +58027 POINT(29.127139294825795 -95.50317505620733) bank58027 +58028 POINT(29.606202751825016 -94.38381779516962) bank58028 +58029 POINT(29.209723171234295 -96.16412817448035) bank58029 +58030 POINT(30.281037603310917 -94.54227788144762) bank58030 +58031 POINT(30.27192045730049 -94.55015296683047) bank58031 +58032 POINT(29.08065415773552 -94.99554832173656) bank58032 +58033 POINT(29.11486104593156 -95.13732188816105) bank58033 +58034 POINT(29.692383246911092 -94.70009917204897) bank58034 +58035 POINT(28.916850625063557 -95.54973965017035) bank58035 +58036 POINT(29.05579036215295 -96.33117391514558) bank58036 +58037 POINT(29.587840267582113 -95.00264471164759) bank58037 +58038 POINT(30.254946580596666 -96.22103693285825) bank58038 +58039 POINT(29.09867684933795 -94.39418817882827) bank58039 +58040 POINT(29.821646274829597 -96.13741588149007) bank58040 +58041 POINT(29.28492099564051 -95.8090757854583) bank58041 +58042 POINT(29.394144873151113 -95.16231849110385) bank58042 +58043 POINT(29.36117478527155 -94.40002243592994) bank58043 +58044 POINT(29.096064494986052 -95.99605669160593) bank58044 +58045 POINT(29.532453038649628 -95.23502609577679) bank58045 +58046 POINT(30.48688390260108 -96.12692231134544) bank58046 +58047 POINT(30.420488657629456 -96.11980290350157) bank58047 +58048 POINT(30.212518817921936 -95.82148671711654) bank58048 +58049 POINT(30.67815093569012 -96.32228413922901) bank58049 +58050 POINT(29.711788900529402 -94.96396668901512) bank58050 +58051 POINT(30.247163434338457 -96.23324443201001) bank58051 +58052 POINT(30.023903832619542 -96.06619980760011) bank58052 +58053 POINT(28.769441087885525 -96.20925470238525) bank58053 +58054 POINT(30.397272926565137 -95.75587653290394) bank58054 +58055 POINT(28.850995833583312 -95.71806171539812) bank58055 +58056 POINT(30.724797304034148 -95.30619422388995) bank58056 +58057 POINT(29.56209826108148 -94.69094034240652) bank58057 +58058 POINT(30.653659304142028 -95.6547341488796) bank58058 +58059 POINT(29.249228812279032 -95.59598640532285) bank58059 +58060 POINT(29.50436981597548 -94.37256844490003) bank58060 +58061 POINT(30.575517469367032 -95.01263201292026) bank58061 +58062 POINT(30.59438145085556 -95.97361365719365) bank58062 +58063 POINT(29.831293524432546 -95.39547243244114) bank58063 +58064 POINT(29.10234164025731 -95.56520300946833) bank58064 +58065 POINT(30.23272461108681 -95.98176203029566) bank58065 +58066 POINT(29.00082661745219 -95.70159064731251) bank58066 +58067 POINT(30.356004140905053 -95.11800686240944) bank58067 +58068 POINT(29.255942620367282 -94.74897725165535) bank58068 +58069 POINT(30.16845542394958 -95.60488031667965) bank58069 +58070 POINT(30.36254050005255 -94.37079662540773) bank58070 +58071 POINT(30.472318149391672 -95.46479282773223) bank58071 +58072 POINT(29.400526382341237 -95.53753836189948) bank58072 +58073 POINT(29.318647554036975 -96.07830375137567) bank58073 +58074 POINT(30.47912917369475 -95.01116554718219) bank58074 +58075 POINT(30.678012918152767 -95.18491323524532) bank58075 +58076 POINT(30.677751335115033 -95.16514823657175) bank58076 +58077 POINT(29.084733154964407 -94.59129690154741) bank58077 +58078 POINT(29.5107861565661 -96.32631100632355) bank58078 +58079 POINT(29.687425708692945 -94.6137950977732) bank58079 +58080 POINT(29.648248950320717 -94.5476998929169) bank58080 +58081 POINT(29.43643595245893 -94.89461702795906) bank58081 +58082 POINT(30.453417725106736 -94.71808380544425) bank58082 +58083 POINT(29.330673745667184 -95.32632006824434) bank58083 +58084 POINT(30.57840547994028 -95.30473844448174) bank58084 +58085 POINT(29.844890000050842 -95.54268158604177) bank58085 +58086 POINT(29.48468606222803 -94.8670815985945) bank58086 +58087 POINT(30.222811196782637 -95.91286995105392) bank58087 +58088 POINT(30.026175449698933 -96.25994088969439) bank58088 +58089 POINT(30.22533231774694 -96.36324795057322) bank58089 +58090 POINT(30.03216599116425 -94.55184559795055) bank58090 +58091 POINT(30.703733837370855 -95.43265957952013) bank58091 +58092 POINT(30.203755487413417 -95.19820756839054) bank58092 +58093 POINT(29.455566032912277 -94.83892252800588) bank58093 +58094 POINT(29.22677516100926 -94.75106332988173) bank58094 +58095 POINT(29.902277326502468 -94.74221339741658) bank58095 +58096 POINT(29.700905114896873 -94.90741991547459) bank58096 +58097 POINT(29.228889925217715 -94.93451216110192) bank58097 +58098 POINT(30.073599995053993 -94.5135929388631) bank58098 +58099 POINT(30.024515704261187 -95.0631684634345) bank58099 +58100 POINT(30.132464007919726 -94.83193735840767) bank58100 +58101 POINT(30.13744227795665 -95.95298964869338) bank58101 +58102 POINT(29.59206146628554 -95.49999446113125) bank58102 +58103 POINT(29.529085277379828 -95.40836670449751) bank58103 +58104 POINT(29.15475500674673 -94.47033234858249) bank58104 +58105 POINT(29.195723320583575 -94.41325392485727) bank58105 +58106 POINT(29.522492523794508 -95.61465204936206) bank58106 +58107 POINT(30.700344616351604 -95.01497246632697) bank58107 +58108 POINT(30.22177011641207 -95.21544603055013) bank58108 +58109 POINT(29.75406284109573 -94.93813828512785) bank58109 +58110 POINT(29.364068190097086 -95.50810124642754) bank58110 +58111 POINT(30.42740042850365 -94.75373216072103) bank58111 +58112 POINT(29.101608147239507 -94.52077513244832) bank58112 +58113 POINT(29.105311464366046 -96.10861727431616) bank58113 +58114 POINT(30.397385504518123 -96.20695501883218) bank58114 +58115 POINT(29.684187150691972 -95.81378960946952) bank58115 +58116 POINT(29.88099823247859 -96.12520880580199) bank58116 +58117 POINT(30.755757424428186 -95.17851061753373) bank58117 +58118 POINT(29.492191330682306 -95.70819469810748) bank58118 +58119 POINT(29.29862598900084 -94.51855090758289) bank58119 +58120 POINT(30.103610433690765 -95.95758190293792) bank58120 +58121 POINT(29.479165945531765 -94.53276755147) bank58121 +58122 POINT(30.697395770557367 -96.32128071987707) bank58122 +58123 POINT(30.113218922009334 -95.95680762726442) bank58123 +58124 POINT(28.982413102582317 -95.48080412761985) bank58124 +58125 POINT(30.580536233816414 -95.25509677964897) bank58125 +58126 POINT(29.226444583756674 -95.91842549119994) bank58126 +58127 POINT(30.23048088830245 -96.0046283437482) bank58127 +58128 POINT(29.89277669030865 -95.06913162168824) bank58128 +58129 POINT(30.608467601821026 -95.34844046758684) bank58129 +58130 POINT(29.340624574137788 -96.2183726143937) bank58130 +58131 POINT(30.588167342346303 -95.24528162665823) bank58131 +58132 POINT(30.702255272714837 -95.9557595141967) bank58132 +58133 POINT(30.63801130586517 -95.77365182767076) bank58133 +58134 POINT(29.050247574922263 -95.6880473250994) bank58134 +58135 POINT(29.201561715316313 -95.73704409404334) bank58135 +58136 POINT(30.375468589174996 -95.76871063647867) bank58136 +58137 POINT(29.366002481065756 -96.10981806392623) bank58137 +58138 POINT(28.897565721328938 -94.94765586348628) bank58138 +58139 POINT(29.862555737637756 -96.08014629616846) bank58139 +58140 POINT(30.455981189213432 -96.16737001855319) bank58140 +58141 POINT(29.883865797461088 -94.56229151034984) bank58141 +58142 POINT(30.483385548610812 -95.32440011484375) bank58142 +58143 POINT(28.872633752555075 -95.96007917141202) bank58143 +58144 POINT(29.020167206729543 -95.4229659024287) bank58144 +58145 POINT(30.4705629405634 -96.05250740191428) bank58145 +58146 POINT(30.142023894557006 -95.04046385393985) bank58146 +58147 POINT(29.273277294571727 -94.83141799785055) bank58147 +58148 POINT(29.173453632061744 -96.36329150781212) bank58148 +58149 POINT(29.168116314760216 -94.92074088934756) bank58149 +58150 POINT(30.691261654987883 -95.84617846039008) bank58150 +58151 POINT(29.20199927976944 -94.71416565572697) bank58151 +58152 POINT(29.505347157400216 -94.71918531336169) bank58152 +58153 POINT(28.779822980891712 -96.05368652035783) bank58153 +58154 POINT(30.410397614894507 -96.24923517689945) bank58154 +58155 POINT(30.09878794489737 -96.19792645026531) bank58155 +58156 POINT(29.725731898252082 -95.117932552501) bank58156 +58157 POINT(29.796754019642865 -95.55136661040083) bank58157 +58158 POINT(29.500425503377237 -95.89604336990912) bank58158 +58159 POINT(30.291919884334373 -96.3145424914317) bank58159 +58160 POINT(30.645141501733562 -95.22517855496397) bank58160 +58161 POINT(30.749078274310662 -94.83378991394223) bank58161 +58162 POINT(29.016540901155437 -96.18980439162772) bank58162 +58163 POINT(29.176608692632477 -96.03753634353494) bank58163 +58164 POINT(29.702848503361295 -94.91403565391313) bank58164 +58165 POINT(28.825127939711752 -95.96269398997953) bank58165 +58166 POINT(30.67786600009914 -94.3981757197056) bank58166 +58167 POINT(29.05919327178106 -94.54112969990454) bank58167 +58168 POINT(30.60549590435871 -95.29921735443804) bank58168 +58169 POINT(30.633199309710747 -96.01698938159086) bank58169 +58170 POINT(29.83551614045662 -95.06771395697291) bank58170 +58171 POINT(29.004638392721052 -94.45599335216217) bank58171 +58172 POINT(29.484206658039405 -96.01353410552146) bank58172 +58173 POINT(30.641383658244223 -95.03166185328264) bank58173 +58174 POINT(30.084589255244467 -96.1147273069016) bank58174 +58175 POINT(28.77837058013797 -96.27191854169327) bank58175 +58176 POINT(29.240145808626842 -95.55286365925485) bank58176 +58177 POINT(30.10709723120655 -94.41824031945315) bank58177 +58178 POINT(30.117210054125884 -95.19108459214256) bank58178 +58179 POINT(29.853364861691524 -94.8142427751741) bank58179 +58180 POINT(29.903157914122882 -94.61884976112403) bank58180 +58181 POINT(29.721293548098725 -95.93924107757182) bank58181 +58182 POINT(28.987873805925236 -95.27000907459907) bank58182 +58183 POINT(28.893335535807747 -96.13165561933718) bank58183 +58184 POINT(29.111028665046288 -94.79759198080703) bank58184 +58185 POINT(30.37662002729339 -96.22959234249447) bank58185 +58186 POINT(29.733761261535996 -94.49472858151746) bank58186 +58187 POINT(30.3672028691928 -95.54558959827921) bank58187 +58188 POINT(29.68359312246762 -95.18445421003315) bank58188 +58189 POINT(30.65177647198268 -95.60528210148843) bank58189 +58190 POINT(29.364467539579106 -95.0862728651784) bank58190 +58191 POINT(29.811498914608393 -94.74582158247438) bank58191 +58192 POINT(29.848563736420992 -94.96124224449602) bank58192 +58193 POINT(29.368814969288778 -95.08109543411166) bank58193 +58194 POINT(29.154047862940033 -95.96001343997905) bank58194 +58195 POINT(28.78551700212887 -95.56078775341884) bank58195 +58196 POINT(29.925415228541944 -95.94519134137374) bank58196 +58197 POINT(29.980802590433388 -96.12178739728132) bank58197 +58198 POINT(29.97666594278693 -95.95525122294784) bank58198 +58199 POINT(29.57325748116065 -95.35372700190231) bank58199 +58200 POINT(30.027162217042143 -95.24712184946542) bank58200 +58201 POINT(29.777764969468315 -95.43799513412955) bank58201 +58202 POINT(30.429160045309978 -94.73376143696092) bank58202 +58203 POINT(30.38924140549878 -95.8120536452103) bank58203 +58204 POINT(29.681208430926816 -95.10972326911325) bank58204 +58205 POINT(30.482503216958236 -95.45507755005791) bank58205 +58206 POINT(29.06210245283871 -95.94573954909966) bank58206 +58207 POINT(30.253594058133444 -96.08525502904138) bank58207 +58208 POINT(29.697710053875806 -94.71387899896307) bank58208 +58209 POINT(30.335838920763653 -96.26534309109043) bank58209 +58210 POINT(29.61710512434627 -94.98203234422012) bank58210 +58211 POINT(30.182101706703662 -95.4338048846164) bank58211 +58212 POINT(29.729906555257873 -96.01255461367303) bank58212 +58213 POINT(30.55242267913057 -94.68196445352898) bank58213 +58214 POINT(30.1432985156546 -96.1181726444488) bank58214 +58215 POINT(28.968454403230293 -95.20531873156598) bank58215 +58216 POINT(30.157016387535396 -95.91069535897944) bank58216 +58217 POINT(29.676734274092215 -94.9938907403224) bank58217 +58218 POINT(30.461751105702483 -95.85343860466814) bank58218 +58219 POINT(30.431774110926497 -95.46345072579017) bank58219 +58220 POINT(30.698860635232432 -95.83079773926174) bank58220 +58221 POINT(29.200026506356682 -95.40899046968836) bank58221 +58222 POINT(30.16396367609194 -95.8852088936017) bank58222 +58223 POINT(29.151251809833937 -96.31652596344243) bank58223 +58224 POINT(30.284638310546068 -96.01733339009853) bank58224 +58225 POINT(29.889262607076645 -94.39555925927338) bank58225 +58226 POINT(28.77659421187745 -95.70527013575055) bank58226 +58227 POINT(30.107080455476282 -94.3925343148514) bank58227 +58228 POINT(29.60086623153618 -95.30793494063668) bank58228 +58229 POINT(30.643875559875 -96.04496409435117) bank58229 +58230 POINT(30.04456363879726 -94.59761627148761) bank58230 +58231 POINT(29.46939537017892 -94.77436826912702) bank58231 +58232 POINT(29.0020322873898 -96.31652243833109) bank58232 +58233 POINT(29.38906944107307 -94.60379969211537) bank58233 +58234 POINT(29.559220095809913 -96.29976263315048) bank58234 +58235 POINT(30.3895846925063 -96.21004337895471) bank58235 +58236 POINT(29.93843624258355 -94.69776969548357) bank58236 +58237 POINT(29.60668365398166 -95.66858245356461) bank58237 +58238 POINT(30.400049434974285 -94.65852169522596) bank58238 +58239 POINT(29.500759507162996 -95.77551634240868) bank58239 +58240 POINT(30.50476868884522 -94.99547834469695) bank58240 +58241 POINT(29.33614799830588 -94.86690125274966) bank58241 +58242 POINT(30.57570832331471 -95.20521787289394) bank58242 +58243 POINT(29.660695380323457 -95.816631604701) bank58243 +58244 POINT(29.102667536803033 -94.87883221992273) bank58244 +58245 POINT(29.7032225920588 -96.35899948081044) bank58245 +58246 POINT(30.421207140873403 -95.40695513204821) bank58246 +58247 POINT(28.819717396021176 -94.47928714487789) bank58247 +58248 POINT(28.94138577948147 -95.65726719928107) bank58248 +58249 POINT(28.99515331993149 -95.15293613012909) bank58249 +58250 POINT(28.92878043487127 -95.93657769803883) bank58250 +58251 POINT(29.60866994957206 -95.41816392989779) bank58251 +58252 POINT(30.64212315193435 -95.81989439166385) bank58252 +58253 POINT(29.24468594366729 -95.27513633731593) bank58253 +58254 POINT(29.829295119135228 -95.48779292986728) bank58254 +58255 POINT(29.637511268863197 -94.89646285295792) bank58255 +58256 POINT(29.03394028904679 -94.40213156319575) bank58256 +58257 POINT(30.75607829799518 -96.11578634865468) bank58257 +58258 POINT(28.881701532710885 -94.76287960267655) bank58258 +58259 POINT(29.411509637622938 -94.63095499508943) bank58259 +58260 POINT(29.12292039603037 -95.33681085110999) bank58260 +58261 POINT(29.149542015149912 -94.75828232512067) bank58261 +58262 POINT(30.429664231662937 -95.47684525784078) bank58262 +58263 POINT(29.725382731796213 -94.97704226226656) bank58263 +58264 POINT(30.34027270713628 -95.43910687445363) bank58264 +58265 POINT(30.064281097422622 -95.92002705059251) bank58265 +58266 POINT(29.788556974226722 -95.2186060388701) bank58266 +58267 POINT(29.063167345775945 -95.58014618614392) bank58267 +58268 POINT(30.66835326413683 -95.87522346121642) bank58268 +58269 POINT(30.337048130382346 -94.7102105946104) bank58269 +58270 POINT(30.716251453488447 -95.19430252062291) bank58270 +58271 POINT(30.29391978271073 -94.51332159613337) bank58271 +58272 POINT(29.188896752997223 -95.73544203252975) bank58272 +58273 POINT(30.22319295653666 -96.20420210596717) bank58273 +58274 POINT(29.67112235733288 -95.8134517978377) bank58274 +58275 POINT(29.042578381064036 -95.71537100869827) bank58275 +58276 POINT(29.201855906244482 -94.3941563011686) bank58276 +58277 POINT(28.933867656912955 -95.92443266016988) bank58277 +58278 POINT(29.402664775114243 -96.02996075733532) bank58278 +58279 POINT(29.943188013425235 -96.00323821035747) bank58279 +58280 POINT(30.394597471309726 -95.76674222859072) bank58280 +58281 POINT(30.51369562860754 -94.91335445390237) bank58281 +58282 POINT(29.933544538528473 -95.46126223554107) bank58282 +58283 POINT(29.592348093122354 -95.11096757586583) bank58283 +58284 POINT(29.704043919193214 -95.07049126380701) bank58284 +58285 POINT(29.36660864948507 -95.92714006782266) bank58285 +58286 POINT(30.71488706568912 -94.74735189941569) bank58286 +58287 POINT(29.479070378799936 -94.88023980993196) bank58287 +58288 POINT(28.8628845451667 -96.237135098175) bank58288 +58289 POINT(30.175989518993656 -94.97712088742996) bank58289 +58290 POINT(30.566593518205273 -95.21377052810467) bank58290 +58291 POINT(29.285516079508223 -94.64768555045174) bank58291 +58292 POINT(29.525609004565617 -95.564768849014) bank58292 +58293 POINT(29.49476825290611 -95.6212054783399) bank58293 +58294 POINT(29.51937643138506 -95.69403187217642) bank58294 +58295 POINT(28.972966414751248 -95.06314664044271) bank58295 +58296 POINT(30.502877764725746 -95.759239749988) bank58296 +58297 POINT(30.75117957700881 -95.23848820079716) bank58297 +58298 POINT(29.813227786650792 -95.28908646855481) bank58298 +58299 POINT(29.814353723814122 -95.59287146445031) bank58299 +58300 POINT(29.582134109887317 -94.8901769966861) bank58300 +58301 POINT(30.050981938729432 -94.40618316822768) bank58301 +58302 POINT(29.467664483389182 -95.76333056922292) bank58302 +58303 POINT(28.877226145419904 -96.09173099795278) bank58303 +58304 POINT(29.59743794953758 -95.72185045191465) bank58304 +58305 POINT(29.330889006281797 -95.79444168355022) bank58305 +58306 POINT(29.070440409110684 -95.58385204075267) bank58306 +58307 POINT(30.088642361335108 -95.96601238326254) bank58307 +58308 POINT(28.83920349769696 -95.88544700407421) bank58308 +58309 POINT(30.63585832074003 -94.92765014705542) bank58309 +58310 POINT(29.42966138181079 -95.32534584710116) bank58310 +58311 POINT(29.560408849885818 -95.18960351492616) bank58311 +58312 POINT(30.589347082444668 -94.99837261737284) bank58312 +58313 POINT(29.814236749696697 -95.62260083099638) bank58313 +58314 POINT(29.771606746956717 -95.69296326599692) bank58314 +58315 POINT(29.248186804200557 -95.23843363775877) bank58315 +58316 POINT(29.053673985320838 -94.8587111276943) bank58316 +58317 POINT(30.22984399389706 -94.93088346490791) bank58317 +58318 POINT(30.123527560754845 -95.67279202604637) bank58318 +58319 POINT(29.690199746278395 -96.30049688370201) bank58319 +58320 POINT(28.945561647214067 -95.02450767314963) bank58320 +58321 POINT(28.793900945575793 -95.96313597160798) bank58321 +58322 POINT(29.69725641750376 -94.55229928952737) bank58322 +58323 POINT(29.622442227876995 -96.16148733511466) bank58323 +58324 POINT(30.56208463711649 -95.83008511629698) bank58324 +58325 POINT(29.725122154625375 -95.40011105674986) bank58325 +58326 POINT(29.00789509658208 -96.3521052710391) bank58326 +58327 POINT(29.46200921563306 -96.21471265264137) bank58327 +58328 POINT(29.115229843171214 -95.7532303876597) bank58328 +58329 POINT(29.084862265514406 -94.43182181914811) bank58329 +58330 POINT(30.31391392318094 -94.71997736409377) bank58330 +58331 POINT(29.061856624461463 -95.58222581844504) bank58331 +58332 POINT(29.960095488392312 -94.81298086743445) bank58332 +58333 POINT(30.63175288892821 -95.82231452597493) bank58333 +58334 POINT(29.41483042461494 -96.18584245332464) bank58334 +58335 POINT(30.06223857511841 -95.69311600495364) bank58335 +58336 POINT(29.39558517470135 -95.61900828675188) bank58336 +58337 POINT(30.668265869696246 -95.242983690045) bank58337 +58338 POINT(30.70802947429407 -95.8199881389007) bank58338 +58339 POINT(29.766147901714604 -96.16903355397051) bank58339 +58340 POINT(28.999679201459706 -94.48683894543329) bank58340 +58341 POINT(30.20644871797195 -94.5381469140501) bank58341 +58342 POINT(29.084054640125657 -94.55688422628324) bank58342 +58343 POINT(29.363203998787476 -95.3887644214634) bank58343 +58344 POINT(30.013731698365927 -94.7268984986159) bank58344 +58345 POINT(28.892252046601392 -95.26285222128091) bank58345 +58346 POINT(29.200048257522543 -95.81158579970547) bank58346 +58347 POINT(29.720806662868956 -94.60766092890874) bank58347 +58348 POINT(30.712437987899623 -95.59272195523741) bank58348 +58349 POINT(29.852036101972498 -96.08953832783483) bank58349 +58350 POINT(30.472120072429124 -94.44916770200133) bank58350 +58351 POINT(28.880098740703385 -95.74591296756076) bank58351 +58352 POINT(30.21590851797714 -95.66718012199387) bank58352 +58353 POINT(30.094983852949902 -95.96754207093377) bank58353 +58354 POINT(29.39579434758244 -95.7627515016718) bank58354 +58355 POINT(29.7792765394736 -95.97745364642475) bank58355 +58356 POINT(29.08164670482951 -96.01149023634218) bank58356 +58357 POINT(29.011740419962134 -94.58670794394858) bank58357 +58358 POINT(29.670855140597368 -95.95403419603338) bank58358 +58359 POINT(30.35862794817482 -95.81657838410183) bank58359 +58360 POINT(29.994713255473233 -95.095120759996) bank58360 +58361 POINT(30.457061059338496 -94.92611312487702) bank58361 +58362 POINT(29.33694188309725 -94.81294222632948) bank58362 +58363 POINT(30.173999109000967 -95.19909165291415) bank58363 +58364 POINT(29.185541980362213 -95.43411244315139) bank58364 +58365 POINT(29.843513173513237 -94.86902663389962) bank58365 +58366 POINT(29.841222168788583 -94.56383764392946) bank58366 +58367 POINT(29.176562742060966 -94.58993826817542) bank58367 +58368 POINT(30.375181825802628 -95.67519149704243) bank58368 +58369 POINT(28.99638619969849 -94.6802638976279) bank58369 +58370 POINT(30.525059516715082 -95.8735242619947) bank58370 +58371 POINT(30.49031024819133 -96.18636469154738) bank58371 +58372 POINT(30.162035282440677 -96.08431146716181) bank58372 +58373 POINT(29.489912916213754 -95.24049344533526) bank58373 +58374 POINT(30.352132167668362 -94.43308377190579) bank58374 +58375 POINT(28.78120262533068 -95.61331066447895) bank58375 +58376 POINT(30.757627198553315 -94.95263379853441) bank58376 +58377 POINT(29.38169441150605 -96.0862622246315) bank58377 +58378 POINT(28.90437603685585 -95.50393927651923) bank58378 +58379 POINT(29.046284181314274 -94.62509916369008) bank58379 +58380 POINT(29.556190003008652 -95.25611706264398) bank58380 +58381 POINT(29.96500304976327 -95.47747950894289) bank58381 +58382 POINT(29.294793425821698 -94.72142329315365) bank58382 +58383 POINT(29.295313446224842 -94.94758326311127) bank58383 +58384 POINT(30.687641616532563 -96.28036359371234) bank58384 +58385 POINT(29.424815967120097 -95.8968332479164) bank58385 +58386 POINT(30.002181539400844 -95.04635746292176) bank58386 +58387 POINT(28.88459149338667 -95.97290490491422) bank58387 +58388 POINT(29.25593351843938 -94.52327372190038) bank58388 +58389 POINT(29.858380059129182 -94.61449071700994) bank58389 +58390 POINT(29.96867933325735 -95.560015680299) bank58390 +58391 POINT(29.548673748328383 -95.30442307174812) bank58391 +58392 POINT(30.47270573050631 -95.31379111812477) bank58392 +58393 POINT(30.74139213968861 -94.62797647621328) bank58393 +58394 POINT(29.052596627933614 -95.69881382030587) bank58394 +58395 POINT(29.439838934272174 -95.02769467521951) bank58395 +58396 POINT(29.179659887840184 -95.4151205670402) bank58396 +58397 POINT(29.223764505883363 -95.53542316676926) bank58397 +58398 POINT(28.96715919388745 -95.93820651192927) bank58398 +58399 POINT(29.629133245641558 -95.20439128864936) bank58399 +58400 POINT(30.130647009227573 -95.75252770652034) bank58400 +58401 POINT(29.532461284254097 -94.45590252317417) bank58401 +58402 POINT(29.049905927515187 -94.87482460114894) bank58402 +58403 POINT(29.89892087683816 -95.86650369288326) bank58403 +58404 POINT(29.142183588884016 -94.83176756647022) bank58404 +58405 POINT(29.75579038709355 -95.74344938622659) bank58405 +58406 POINT(30.633249364283916 -94.66506571606013) bank58406 +58407 POINT(30.68948293185436 -95.88141297740599) bank58407 +58408 POINT(30.256277035613454 -94.56639115737276) bank58408 +58409 POINT(30.295541160381326 -95.41016712225394) bank58409 +58410 POINT(29.567212102460694 -95.73136251431244) bank58410 +58411 POINT(30.425077788578484 -95.66835095854894) bank58411 +58412 POINT(29.087261864700587 -96.00236309640178) bank58412 +58413 POINT(30.026040379877124 -94.60487491018142) bank58413 +58414 POINT(30.493543781153484 -96.0497575671742) bank58414 +58415 POINT(29.04318005302714 -95.18492285531678) bank58415 +58416 POINT(28.88213189841048 -94.75349277533614) bank58416 +58417 POINT(29.745320447805938 -95.04298787954443) bank58417 +58418 POINT(30.149431231218692 -94.50011049339537) bank58418 +58419 POINT(29.717322440197798 -96.21285843316878) bank58419 +58420 POINT(30.68484600840979 -95.48761581714872) bank58420 +58421 POINT(30.1515923347554 -94.9751702993951) bank58421 +58422 POINT(29.06562412087623 -95.44672067558213) bank58422 +58423 POINT(30.129229091234325 -95.56811435707179) bank58423 +58424 POINT(29.78821855796293 -95.22142021815229) bank58424 +58425 POINT(30.622275749449354 -94.81667360733815) bank58425 +58426 POINT(28.905459883363996 -96.28751621904358) bank58426 +58427 POINT(28.95409296215281 -95.93515442657807) bank58427 +58428 POINT(30.61763983032281 -95.44969042990073) bank58428 +58429 POINT(30.255555701922557 -95.12295410788374) bank58429 +58430 POINT(28.836235070824614 -94.91813647780441) bank58430 +58431 POINT(29.352933479542845 -95.96949805543656) bank58431 +58432 POINT(29.85629728614063 -94.81685388774477) bank58432 +58433 POINT(28.806812415078628 -95.75392012434693) bank58433 +58434 POINT(29.579425823248695 -94.84544955281481) bank58434 +58435 POINT(30.611040395587068 -96.03910939513351) bank58435 +58436 POINT(29.71900437427108 -96.36607867911972) bank58436 +58437 POINT(30.002877337673464 -94.9041754052686) bank58437 +58438 POINT(28.76259116337462 -95.00063389870495) bank58438 +58439 POINT(29.15073322667088 -95.19797282749572) bank58439 +58440 POINT(29.436754467203457 -95.95340084711707) bank58440 +58441 POINT(29.179196107048313 -94.92418060677632) bank58441 +58442 POINT(28.864364975898628 -94.74804149994873) bank58442 +58443 POINT(29.251833322724124 -94.52631839991733) bank58443 +58444 POINT(29.469456755693468 -94.5332008779007) bank58444 +58445 POINT(30.63516750920795 -94.53527511089428) bank58445 +58446 POINT(28.884121968475885 -94.80472816413447) bank58446 +58447 POINT(29.536624428576424 -95.4564827618549) bank58447 +58448 POINT(29.719886275379142 -96.19686889227958) bank58448 +58449 POINT(30.29373991198029 -94.83167108688296) bank58449 +58450 POINT(29.987570972632597 -95.32519911994727) bank58450 +58451 POINT(29.042135727127942 -95.34638358401818) bank58451 +58452 POINT(30.263559959083853 -95.93469424229869) bank58452 +58453 POINT(29.112168283595025 -94.67416263415844) bank58453 +58454 POINT(30.63847151357272 -95.99170073326121) bank58454 +58455 POINT(28.981150047990695 -96.13966428761493) bank58455 +58456 POINT(29.7181264078294 -96.05222423904979) bank58456 +58457 POINT(30.418205899420872 -94.8524081398235) bank58457 +58458 POINT(29.291499563205154 -95.21895723486051) bank58458 +58459 POINT(30.48976542895174 -95.82378499615666) bank58459 +58460 POINT(29.406626237603753 -94.61991448781251) bank58460 +58461 POINT(29.982498679418406 -94.9078345264356) bank58461 +58462 POINT(29.980187550071797 -95.6064077117095) bank58462 +58463 POINT(29.591491505011536 -96.10383102695019) bank58463 +58464 POINT(29.708122998935234 -94.448682744563) bank58464 +58465 POINT(29.177912575596586 -95.7767476728108) bank58465 +58466 POINT(29.422350903462757 -95.573089966022) bank58466 +58467 POINT(29.181110415988474 -95.4564731426173) bank58467 +58468 POINT(29.333903729681523 -94.57780759154068) bank58468 +58469 POINT(30.448579133969567 -95.21997576982528) bank58469 +58470 POINT(28.89207612272078 -95.76158199721822) bank58470 +58471 POINT(29.823883616384812 -95.72671051674722) bank58471 +58472 POINT(29.182129770329706 -95.87940315726964) bank58472 +58473 POINT(30.00319777155948 -96.23016425845755) bank58473 +58474 POINT(29.086376638478757 -96.03798815396001) bank58474 +58475 POINT(30.06900180851015 -94.80459460337482) bank58475 +58476 POINT(28.843549145127493 -96.25436256461462) bank58476 +58477 POINT(30.212190062076125 -95.78647180717016) bank58477 +58478 POINT(29.309054142899047 -95.99986165820054) bank58478 +58479 POINT(29.21531858749251 -95.12923344274274) bank58479 +58480 POINT(30.641553980088243 -94.3823853502559) bank58480 +58481 POINT(29.14826586120096 -94.92187249405637) bank58481 +58482 POINT(28.97979938261499 -95.01186094465876) bank58482 +58483 POINT(28.956435891044258 -95.0951189309722) bank58483 +58484 POINT(29.977166506226762 -95.78695099974519) bank58484 +58485 POINT(30.566351358338313 -95.57400447534424) bank58485 +58486 POINT(30.35352474090036 -95.55216285164057) bank58486 +58487 POINT(29.20850417413363 -94.6538610393099) bank58487 +58488 POINT(28.940808571657836 -95.96197718739216) bank58488 +58489 POINT(30.56623924494162 -96.09023863596694) bank58489 +58490 POINT(30.405965981582916 -95.63513215275536) bank58490 +58491 POINT(30.447167786872058 -95.31951424679177) bank58491 +58492 POINT(30.262214329107515 -95.55363159971961) bank58492 +58493 POINT(29.736407849823237 -95.22565904746963) bank58493 +58494 POINT(28.99216493549663 -95.87510631504233) bank58494 +58495 POINT(30.417147959480314 -95.9281701381036) bank58495 +58496 POINT(29.889873294831776 -94.80101429441332) bank58496 +58497 POINT(28.819180249225237 -95.26018776219907) bank58497 +58498 POINT(29.73120893090048 -95.82079820962528) bank58498 +58499 POINT(29.21634986567516 -95.24630335160943) bank58499 +58500 POINT(30.689876318557456 -96.27847179035321) bank58500 +58501 POINT(30.664693385040728 -96.03580890257234) bank58501 +58502 POINT(29.92802777999884 -95.76581918931562) bank58502 +58503 POINT(29.519386053513095 -94.4230738043462) bank58503 +58504 POINT(30.421465884193594 -95.42566890429018) bank58504 +58505 POINT(30.27164863549562 -96.24915996201631) bank58505 +58506 POINT(29.580967026013546 -96.35476068105196) bank58506 +58507 POINT(29.157609320996436 -95.46928017095554) bank58507 +58508 POINT(29.78448412073711 -94.72947361988612) bank58508 +58509 POINT(29.21860214335434 -94.87510966962017) bank58509 +58510 POINT(29.286510812733862 -96.13658497687805) bank58510 +58511 POINT(30.444514908856867 -95.0701029646994) bank58511 +58512 POINT(30.562887761613773 -94.64845309624988) bank58512 +58513 POINT(30.550244133959136 -95.25745964198637) bank58513 +58514 POINT(30.561710452240916 -95.22242633894302) bank58514 +58515 POINT(30.364351765683637 -94.82407067242433) bank58515 +58516 POINT(29.065831731360273 -95.59558825586205) bank58516 +58517 POINT(30.295384030172364 -95.96135846575282) bank58517 +58518 POINT(30.052494587553905 -94.37505948096403) bank58518 +58519 POINT(30.14117623219261 -94.47635559952208) bank58519 +58520 POINT(30.201497404105982 -94.66690162014768) bank58520 +58521 POINT(30.604584363214524 -94.40384622235405) bank58521 +58522 POINT(29.613547621031994 -95.41705211312302) bank58522 +58523 POINT(28.862081108011242 -94.86222120724726) bank58523 +58524 POINT(29.515635122524806 -94.8420506748751) bank58524 +58525 POINT(29.915256408313585 -96.18759094496976) bank58525 +58526 POINT(30.099826793216728 -95.61568564568087) bank58526 +58527 POINT(30.151419789463585 -95.75321304118573) bank58527 +58528 POINT(30.42414818114593 -94.98947108906364) bank58528 +58529 POINT(29.987280829445332 -96.07763700301756) bank58529 +58530 POINT(28.968661776473528 -95.22610625589832) bank58530 +58531 POINT(29.939025839875956 -94.93762981219874) bank58531 +58532 POINT(30.285202827779553 -95.52484129957566) bank58532 +58533 POINT(30.546684515958432 -95.31104438902817) bank58533 +58534 POINT(29.918145463456042 -94.58066634063773) bank58534 +58535 POINT(30.17458692267624 -95.65983526446206) bank58535 +58536 POINT(30.524955586508693 -95.21902068735623) bank58536 +58537 POINT(30.614422995488205 -96.21307690672201) bank58537 +58538 POINT(30.003931332867563 -96.17714203263054) bank58538 +58539 POINT(30.031505752678353 -96.02901773372335) bank58539 +58540 POINT(29.83311467537226 -94.47115842102656) bank58540 +58541 POINT(30.256844744230143 -94.70122797877889) bank58541 +58542 POINT(29.949964566355746 -94.8085555252234) bank58542 +58543 POINT(29.12643103189428 -94.47411254604832) bank58543 +58544 POINT(30.035827109221383 -95.71815884413458) bank58544 +58545 POINT(30.21347015610551 -94.97302663011592) bank58545 +58546 POINT(29.88557053949222 -96.18205287484466) bank58546 +58547 POINT(29.45247309590277 -95.16768941210293) bank58547 +58548 POINT(30.314735848427137 -95.005708901473) bank58548 +58549 POINT(29.885620872001198 -96.22386487034358) bank58549 +58550 POINT(29.937967949295512 -96.21010544599709) bank58550 +58551 POINT(29.914705193327123 -95.72775488831627) bank58551 +58552 POINT(30.33734120666824 -95.74203330371536) bank58552 +58553 POINT(29.762324512333358 -95.07959546091055) bank58553 +58554 POINT(28.916780364296752 -94.39137529920049) bank58554 +58555 POINT(28.994827540599626 -96.1832308195971) bank58555 +58556 POINT(30.73052338693274 -95.86676982330006) bank58556 +58557 POINT(29.780918123828833 -94.72253230112884) bank58557 +58558 POINT(29.852305786271856 -96.24139224169026) bank58558 +58559 POINT(29.281496522564794 -94.98306241713352) bank58559 +58560 POINT(29.17766584641075 -95.86708355377252) bank58560 +58561 POINT(29.000528777918067 -95.9877064457252) bank58561 +58562 POINT(29.816998194357357 -94.55752139221441) bank58562 +58563 POINT(30.0018451969554 -95.06059924853804) bank58563 +58564 POINT(29.689752415026696 -96.21766345123761) bank58564 +58565 POINT(30.493214986237756 -95.6839083965979) bank58565 +58566 POINT(29.50706408367793 -94.9969277119805) bank58566 +58567 POINT(30.109547957526882 -94.75054008581697) bank58567 +58568 POINT(28.81855797626264 -96.35304417781734) bank58568 +58569 POINT(29.48796641107942 -94.71312225405265) bank58569 +58570 POINT(29.500081901945784 -96.2836431515834) bank58570 +58571 POINT(29.370075259421967 -95.54591622896176) bank58571 +58572 POINT(30.008161995281757 -94.7000458218428) bank58572 +58573 POINT(29.526462898952953 -96.30490879074766) bank58573 +58574 POINT(29.82309633325756 -94.97002463480824) bank58574 +58575 POINT(30.352192244937342 -95.01228127186587) bank58575 +58576 POINT(30.160044843281867 -94.89655241754875) bank58576 +58577 POINT(30.664475092591804 -94.54137814343481) bank58577 +58578 POINT(29.728992255870487 -95.19883361399995) bank58578 +58579 POINT(29.251257815706044 -94.99457846436671) bank58579 +58580 POINT(28.862794782327555 -95.3842370288766) bank58580 +58581 POINT(29.17656750995682 -95.61248295376531) bank58581 +58582 POINT(29.266305223611905 -94.75385231583239) bank58582 +58583 POINT(30.169157571973013 -95.57847369289355) bank58583 +58584 POINT(30.42869016667539 -94.4537225961977) bank58584 +58585 POINT(29.70382357165783 -94.51927865724386) bank58585 +58586 POINT(30.030446890719478 -95.57138910610253) bank58586 +58587 POINT(29.09861945764568 -96.23535731808487) bank58587 +58588 POINT(29.971574162279232 -96.09013294490647) bank58588 +58589 POINT(29.337964374056142 -94.53466653696668) bank58589 +58590 POINT(29.190446914015837 -94.78818078462716) bank58590 +58591 POINT(30.63774505984694 -96.21087960859111) bank58591 +58592 POINT(30.400427247024183 -96.18204228686426) bank58592 +58593 POINT(30.60577885114478 -95.65125695417157) bank58593 +58594 POINT(29.22423005030585 -96.01606997386155) bank58594 +58595 POINT(29.583011393374154 -95.26510948728571) bank58595 +58596 POINT(28.98994487283065 -95.1277629169171) bank58596 +58597 POINT(30.41931259722746 -95.6588164016705) bank58597 +58598 POINT(29.50719071608131 -95.14622347334863) bank58598 +58599 POINT(30.300459928668953 -96.21675486415751) bank58599 +58600 POINT(30.67013536871956 -95.58400461479806) bank58600 +58601 POINT(28.95948515183811 -95.117463758041) bank58601 +58602 POINT(30.68417773150786 -94.73694302065363) bank58602 +58603 POINT(30.037831355853534 -94.45124408429301) bank58603 +58604 POINT(30.586531585337728 -95.03429228951852) bank58604 +58605 POINT(30.245650234857568 -96.33351976098578) bank58605 +58606 POINT(28.789973328441175 -95.99010496918918) bank58606 +58607 POINT(29.734759652607792 -94.73711119238742) bank58607 +58608 POINT(29.163205629788578 -96.26174871454903) bank58608 +58609 POINT(29.968798238783656 -96.02193165586357) bank58609 +58610 POINT(30.041445939311195 -95.12665632376455) bank58610 +58611 POINT(30.70370184205929 -96.06970504212558) bank58611 +58612 POINT(28.760979234189588 -95.72848869570653) bank58612 +58613 POINT(29.3063899335349 -94.58180124806313) bank58613 +58614 POINT(30.40505059608216 -94.6265861889865) bank58614 +58615 POINT(29.889527116733245 -96.1925637051551) bank58615 +58616 POINT(29.03931926115208 -95.29116453961767) bank58616 +58617 POINT(29.638206623875234 -95.07443460923032) bank58617 +58618 POINT(30.289224403036858 -96.2258930507947) bank58618 +58619 POINT(30.22262243627255 -95.67236489345403) bank58619 +58620 POINT(29.14605946474054 -94.38969223470069) bank58620 +58621 POINT(30.186060794294544 -96.09868816386243) bank58621 +58622 POINT(29.743293114665665 -96.23755393123932) bank58622 +58623 POINT(28.942166061103197 -95.55991746976635) bank58623 +58624 POINT(29.78353843044226 -96.1803122830733) bank58624 +58625 POINT(30.430838765221765 -94.91464141263474) bank58625 +58626 POINT(30.346084124816517 -96.24794848496593) bank58626 +58627 POINT(28.942985388293994 -95.2082315770669) bank58627 +58628 POINT(28.89569476302411 -95.59042457593249) bank58628 +58629 POINT(29.034951996227 -96.17978717248721) bank58629 +58630 POINT(30.090815893118315 -95.04508007461617) bank58630 +58631 POINT(29.38556649687802 -94.54111596969538) bank58631 +58632 POINT(29.374664058833314 -95.67220382463695) bank58632 +58633 POINT(29.42255929910299 -95.4028895573607) bank58633 +58634 POINT(28.896845524821174 -94.8276334701623) bank58634 +58635 POINT(29.04230094943762 -95.54053044561405) bank58635 +58636 POINT(30.665131937692546 -95.93585070147877) bank58636 +58637 POINT(30.526375781552 -95.23282178167861) bank58637 +58638 POINT(28.937341966125448 -95.96188870999936) bank58638 +58639 POINT(30.531401115044638 -94.60151051503303) bank58639 +58640 POINT(28.871323401701538 -94.44992859944446) bank58640 +58641 POINT(29.843134538208783 -95.38927079937626) bank58641 +58642 POINT(29.060615970851316 -96.25000013184027) bank58642 +58643 POINT(30.257527721985582 -95.68639293252424) bank58643 +58644 POINT(28.907253243796774 -95.84503420833835) bank58644 +58645 POINT(29.111173026982026 -94.64931320465942) bank58645 +58646 POINT(29.57712597524224 -94.86338157412372) bank58646 +58647 POINT(29.55468363007913 -95.19113028946194) bank58647 +58648 POINT(30.025855620437152 -94.41754807162887) bank58648 +58649 POINT(29.403523715061105 -95.1737222950525) bank58649 +58650 POINT(29.41118319263766 -95.26154906106044) bank58650 +58651 POINT(29.968911827756656 -96.25128314146241) bank58651 +58652 POINT(29.68143265404815 -96.27443298318494) bank58652 +58653 POINT(29.793490204735903 -95.42952305242642) bank58653 +58654 POINT(29.8029189615109 -95.55714716119805) bank58654 +58655 POINT(29.147129835232974 -94.39858865762942) bank58655 +58656 POINT(29.895819970897687 -95.69162643816856) bank58656 +58657 POINT(30.55464741407477 -95.19733902335886) bank58657 +58658 POINT(29.75605959743078 -95.0187714958069) bank58658 +58659 POINT(29.91234085807827 -94.40964553141201) bank58659 +58660 POINT(28.83851810052111 -95.91210661808515) bank58660 +58661 POINT(30.02785365839939 -95.34759670670091) bank58661 +58662 POINT(30.575090320682182 -94.43162964355902) bank58662 +58663 POINT(29.59708923814122 -94.63948119860369) bank58663 +58664 POINT(30.369009084298686 -95.69422319875254) bank58664 +58665 POINT(29.232553251865756 -95.32065061558245) bank58665 +58666 POINT(30.405829055110342 -95.27195610718904) bank58666 +58667 POINT(29.966504742114765 -95.85913514555739) bank58667 +58668 POINT(30.56089262984601 -94.89031079572302) bank58668 +58669 POINT(28.95129482250416 -96.11367828953091) bank58669 +58670 POINT(30.668161429840087 -94.65239634576838) bank58670 +58671 POINT(29.485166679139578 -95.47362549829224) bank58671 +58672 POINT(30.555282182904513 -96.17546056070859) bank58672 +58673 POINT(29.78760415312489 -94.89958505377429) bank58673 +58674 POINT(30.026902164303134 -95.88419164832115) bank58674 +58675 POINT(29.76937026657905 -95.03234258931884) bank58675 +58676 POINT(28.918037361357253 -95.41849037733003) bank58676 +58677 POINT(29.2639506628754 -96.35035826769804) bank58677 +58678 POINT(29.962194514551438 -95.64463559675607) bank58678 +58679 POINT(29.573272097533764 -94.86264014415896) bank58679 +58680 POINT(29.597947879346957 -95.31658992288627) bank58680 +58681 POINT(30.30124211221171 -94.77713868462149) bank58681 +58682 POINT(29.27792268278538 -94.48063582937097) bank58682 +58683 POINT(29.055368494653106 -94.78593093963953) bank58683 +58684 POINT(30.280339426193507 -94.69413869556718) bank58684 +58685 POINT(29.304615256926688 -94.44364397252336) bank58685 +58686 POINT(29.735740021039792 -94.58029160119926) bank58686 +58687 POINT(29.74553623244695 -94.66032264541649) bank58687 +58688 POINT(29.147813828248655 -95.49147953098065) bank58688 +58689 POINT(28.783248807669896 -95.20075630885543) bank58689 +58690 POINT(30.430828803416343 -94.72517488957672) bank58690 +58691 POINT(29.54249947675832 -96.26496248901543) bank58691 +58692 POINT(29.48382747779585 -95.90594848153542) bank58692 +58693 POINT(30.007781105365066 -95.04173768189354) bank58693 +58694 POINT(29.98934609142376 -95.20671114337088) bank58694 +58695 POINT(29.98450489916164 -96.36402712849316) bank58695 +58696 POINT(30.166122036925973 -96.02960142589893) bank58696 +58697 POINT(29.699194631576713 -95.19398336399564) bank58697 +58698 POINT(29.090870728592087 -94.47736643179779) bank58698 +58699 POINT(29.306078728706343 -95.97315588260673) bank58699 +58700 POINT(29.82197277429125 -94.83964804917308) bank58700 +58701 POINT(29.7800505860163 -96.27179013040228) bank58701 +58702 POINT(30.480152024227912 -95.43687196062443) bank58702 +58703 POINT(30.410765170198946 -95.57670305985425) bank58703 +58704 POINT(30.009237514880756 -96.0769107655183) bank58704 +58705 POINT(29.32877804552684 -95.84375461612915) bank58705 +58706 POINT(29.895659502820983 -94.66997219160035) bank58706 +58707 POINT(30.133631175017843 -95.9017439964552) bank58707 +58708 POINT(28.873334748266966 -95.88969182464434) bank58708 +58709 POINT(29.662023883775955 -95.11596034647567) bank58709 +58710 POINT(30.224420806545847 -96.26279315791754) bank58710 +58711 POINT(30.484482050899494 -95.20610680695076) bank58711 +58712 POINT(30.36264377013747 -95.00918795875616) bank58712 +58713 POINT(29.189470966262395 -95.8698856542055) bank58713 +58714 POINT(30.59787893250413 -94.57569749625615) bank58714 +58715 POINT(29.86285819834464 -95.26273524417292) bank58715 +58716 POINT(29.216083423777988 -95.8232141239874) bank58716 +58717 POINT(29.486130958755822 -95.68195041030377) bank58717 +58718 POINT(30.549700497933188 -94.72932815955122) bank58718 +58719 POINT(29.584812975500743 -95.2304158183963) bank58719 +58720 POINT(29.58649032973779 -94.802782649549) bank58720 +58721 POINT(28.81617926327686 -96.02655739331901) bank58721 +58722 POINT(29.6863818065015 -95.19047569539184) bank58722 +58723 POINT(29.109182490594982 -95.56894928000938) bank58723 +58724 POINT(29.079229660737223 -95.17346650693392) bank58724 +58725 POINT(28.883732064591 -96.00038456303902) bank58725 +58726 POINT(30.421772549286857 -94.75532571351216) bank58726 +58727 POINT(30.08970510549572 -95.6158039241687) bank58727 +58728 POINT(28.87962823029467 -95.39794603247366) bank58728 +58729 POINT(30.748998339563634 -94.41878165843805) bank58729 +58730 POINT(29.41336906981101 -96.17001924741983) bank58730 +58731 POINT(30.408063986257062 -95.1445102745569) bank58731 +58732 POINT(29.57175677237561 -95.0656164235208) bank58732 +58733 POINT(29.9177019901135 -96.28599446550722) bank58733 +58734 POINT(30.390793307113483 -95.38075245439667) bank58734 +58735 POINT(29.910908430908876 -94.42795972200307) bank58735 +58736 POINT(29.301534513954397 -96.22042533874614) bank58736 +58737 POINT(30.201383161819987 -95.31622124170688) bank58737 +58738 POINT(30.234187100979085 -95.3258608274176) bank58738 +58739 POINT(29.8802169915525 -95.62334732707191) bank58739 +58740 POINT(30.684151465573702 -95.14653851725708) bank58740 +58741 POINT(29.735715063882196 -96.14182121322287) bank58741 +58742 POINT(29.10418852489996 -94.51247938247015) bank58742 +58743 POINT(30.05507525124911 -94.64177796682242) bank58743 +58744 POINT(30.690144438886414 -94.4680311013903) bank58744 +58745 POINT(29.913090367415446 -94.94573136903556) bank58745 +58746 POINT(28.825088769229477 -95.07306349343469) bank58746 +58747 POINT(29.16973939882766 -94.38475005250773) bank58747 +58748 POINT(29.939126727498333 -96.26007178392466) bank58748 +58749 POINT(28.88524602908047 -94.9317145681106) bank58749 +58750 POINT(30.245047656455952 -96.19895417053496) bank58750 +58751 POINT(29.939257132000254 -94.424898625129) bank58751 +58752 POINT(30.50233251326275 -95.2927063279176) bank58752 +58753 POINT(30.659198626637995 -96.23107313697278) bank58753 +58754 POINT(29.561954676403417 -94.66944945492128) bank58754 +58755 POINT(29.398222548891944 -94.95464464361933) bank58755 +58756 POINT(30.371768863734864 -95.8297373157402) bank58756 +58757 POINT(30.669747231771616 -96.0961177749044) bank58757 +58758 POINT(29.495714277705204 -94.44186673647593) bank58758 +58759 POINT(29.820498814532538 -95.40825471418513) bank58759 +58760 POINT(30.238242415848674 -95.42195913890701) bank58760 +58761 POINT(30.48734945734073 -94.90547230517896) bank58761 +58762 POINT(30.59277260835049 -95.79504577707037) bank58762 +58763 POINT(28.7929537769982 -94.60289274685098) bank58763 +58764 POINT(28.7853466109265 -96.00711991620882) bank58764 +58765 POINT(28.782387867316 -94.51320252877814) bank58765 +58766 POINT(30.136191934632215 -94.51301975239896) bank58766 +58767 POINT(29.51657980438547 -94.6289650505708) bank58767 +58768 POINT(29.22026465489085 -95.61160076232099) bank58768 +58769 POINT(29.786815758061312 -94.51299955964191) bank58769 +58770 POINT(29.919744218856138 -96.20828933150715) bank58770 +58771 POINT(28.994002267793277 -96.0224271846916) bank58771 +58772 POINT(29.056279574572837 -95.2441226417054) bank58772 +58773 POINT(30.53623299099423 -95.22888986649208) bank58773 +58774 POINT(29.685025191028842 -95.49431673368493) bank58774 +58775 POINT(29.618415221607826 -95.68156694132477) bank58775 +58776 POINT(30.127791994751703 -95.71580885313756) bank58776 +58777 POINT(29.676549743204617 -95.64574352518771) bank58777 +58778 POINT(30.383351476287633 -95.34979203455062) bank58778 +58779 POINT(30.3525005131131 -94.60553724250731) bank58779 +58780 POINT(30.627518716908654 -96.3132095346823) bank58780 +58781 POINT(30.26313427584874 -95.13604597208676) bank58781 +58782 POINT(28.803812533861564 -95.53286344493586) bank58782 +58783 POINT(30.745690970823276 -96.28802154446483) bank58783 +58784 POINT(29.820724314807833 -95.87660959044072) bank58784 +58785 POINT(30.08787101308228 -95.79767966446265) bank58785 +58786 POINT(28.962462522042088 -94.60768539013549) bank58786 +58787 POINT(29.979774206802986 -94.98024120010798) bank58787 +58788 POINT(30.286466920004088 -96.01697855782254) bank58788 +58789 POINT(29.567282026808194 -94.68410607276913) bank58789 +58790 POINT(30.315490005026234 -95.76844527948786) bank58790 +58791 POINT(28.78562152637939 -95.18094165431276) bank58791 +58792 POINT(29.962906467539792 -95.04187367554655) bank58792 +58793 POINT(30.422414775826457 -95.24915756704594) bank58793 +58794 POINT(29.573305883311193 -95.23045187899723) bank58794 +58795 POINT(29.453203043804553 -94.78971374510803) bank58795 +58796 POINT(30.679302378626982 -95.37850400468301) bank58796 +58797 POINT(30.22764023726183 -96.34811232742354) bank58797 +58798 POINT(30.26857444853664 -95.57302936636572) bank58798 +58799 POINT(29.949385011987754 -95.08123712249413) bank58799 +58800 POINT(29.611093975448135 -96.01300182480402) bank58800 +58801 POINT(29.262845809852912 -94.99131907388045) bank58801 +58802 POINT(28.9846394228687 -95.24042701477947) bank58802 +58803 POINT(29.97190447131465 -95.41546985055824) bank58803 +58804 POINT(29.352280389108323 -95.72125054088902) bank58804 +58805 POINT(28.91626317533222 -96.24267049191697) bank58805 +58806 POINT(29.57329591505589 -96.27479639781843) bank58806 +58807 POINT(29.927953767725775 -95.3020084034404) bank58807 +58808 POINT(30.6748740118442 -94.47965885580727) bank58808 +58809 POINT(29.677095111778748 -95.7613465690154) bank58809 +58810 POINT(29.68612864747862 -95.5420777109447) bank58810 +58811 POINT(30.411028353625344 -95.53692087111683) bank58811 +58812 POINT(28.829967913416127 -94.82870626337855) bank58812 +58813 POINT(29.41636766831831 -95.71119104484069) bank58813 +58814 POINT(30.19373006450758 -95.61595406901338) bank58814 +58815 POINT(30.720237056915842 -94.96648140931597) bank58815 +58816 POINT(30.421380921851846 -95.6412698154156) bank58816 +58817 POINT(29.869317588416084 -94.50836008034992) bank58817 +58818 POINT(30.523903856669264 -94.79485633145184) bank58818 +58819 POINT(30.70255727700437 -96.08204879215953) bank58819 +58820 POINT(29.799513783787596 -96.08944547007269) bank58820 +58821 POINT(30.691253897851123 -95.68054290005304) bank58821 +58822 POINT(29.14250531012251 -94.71148031041434) bank58822 +58823 POINT(30.162121140008466 -94.51071735935297) bank58823 +58824 POINT(29.766529168695655 -96.20234360289601) bank58824 +58825 POINT(29.666337797578365 -94.95806127720162) bank58825 +58826 POINT(29.52503234597811 -95.78509522762357) bank58826 +58827 POINT(30.100184137921325 -94.4735660895206) bank58827 +58828 POINT(29.216862292090504 -95.51373087635247) bank58828 +58829 POINT(29.13988842748178 -94.92888507681788) bank58829 +58830 POINT(29.963253119358246 -94.69877398727235) bank58830 +58831 POINT(30.13972746651572 -96.13016646794237) bank58831 +58832 POINT(29.22765426535495 -95.08852002745388) bank58832 +58833 POINT(28.93079433418405 -94.3912434888387) bank58833 +58834 POINT(28.839565092810375 -95.69489635946772) bank58834 +58835 POINT(29.734210379863097 -96.33107338206987) bank58835 +58836 POINT(29.967472753816846 -94.82225068201852) bank58836 +58837 POINT(30.260785029252986 -96.07673321691449) bank58837 +58838 POINT(28.81726721295248 -96.16704212422694) bank58838 +58839 POINT(29.711999902744292 -95.24279668240926) bank58839 +58840 POINT(30.52918840773514 -95.92710350193072) bank58840 +58841 POINT(29.434009594678518 -95.00896148055051) bank58841 +58842 POINT(30.435818930817344 -95.10065327080036) bank58842 +58843 POINT(29.7768912695387 -94.95187094620843) bank58843 +58844 POINT(30.15182517422699 -94.9706272614636) bank58844 +58845 POINT(30.657162469867043 -95.30941438613567) bank58845 +58846 POINT(29.067583127086994 -94.74113359555177) bank58846 +58847 POINT(29.38968579383201 -94.69764838369052) bank58847 +58848 POINT(29.9563559354893 -95.85068661151801) bank58848 +58849 POINT(29.374832773797653 -94.72715849223738) bank58849 +58850 POINT(30.746990501722415 -95.17411150133661) bank58850 +58851 POINT(29.90037019586317 -95.5101876493516) bank58851 +58852 POINT(28.8164565329398 -94.56224114440029) bank58852 +58853 POINT(29.39371317674354 -95.86417979734482) bank58853 +58854 POINT(28.81943071739762 -96.35058012958328) bank58854 +58855 POINT(30.716367939919706 -95.33151765040331) bank58855 +58856 POINT(30.113458473852045 -95.54364727228361) bank58856 +58857 POINT(30.302522043599406 -94.75192121054825) bank58857 +58858 POINT(29.799728063057337 -95.33736805326238) bank58858 +58859 POINT(30.75607519988739 -95.16433111581802) bank58859 +58860 POINT(28.933533826473635 -94.38553690512079) bank58860 +58861 POINT(29.34424536628361 -95.25012555727328) bank58861 +58862 POINT(29.143728877141047 -94.92849011018572) bank58862 +58863 POINT(29.212233210488375 -96.23993126041802) bank58863 +58864 POINT(29.192776875763506 -94.47118436359834) bank58864 +58865 POINT(29.463721494203142 -96.27240804792659) bank58865 +58866 POINT(29.619822594785056 -96.2007144035844) bank58866 +58867 POINT(29.0738419548104 -95.6522932280504) bank58867 +58868 POINT(30.610087563708028 -95.18705943312042) bank58868 +58869 POINT(29.424626120882134 -95.45198901622018) bank58869 +58870 POINT(30.07111475335246 -96.36367795193618) bank58870 +58871 POINT(30.18291013735571 -94.8523127724019) bank58871 +58872 POINT(29.032079147725277 -95.15359940304697) bank58872 +58873 POINT(29.966892263575645 -94.69128680102355) bank58873 +58874 POINT(30.248758675387784 -95.57417754126483) bank58874 +58875 POINT(29.3823128395151 -95.66013436369347) bank58875 +58876 POINT(28.902468218213777 -95.97583008190787) bank58876 +58877 POINT(29.88324954054405 -94.70998778380645) bank58877 +58878 POINT(29.727799997006557 -95.48207349320288) bank58878 +58879 POINT(30.405667213128005 -95.46376502324661) bank58879 +58880 POINT(29.76521823555788 -96.1494761052548) bank58880 +58881 POINT(29.298732160339316 -95.26642732153192) bank58881 +58882 POINT(29.13330037901613 -95.78305060339055) bank58882 +58883 POINT(30.269507458336854 -95.7637388995107) bank58883 +58884 POINT(30.565207816270608 -96.3541163287343) bank58884 +58885 POINT(29.42571091093114 -96.03764820579835) bank58885 +58886 POINT(29.573721399118156 -95.46325442961059) bank58886 +58887 POINT(29.1210318693683 -95.6127429603448) bank58887 +58888 POINT(30.48051364419388 -95.14668651636904) bank58888 +58889 POINT(29.46866410498431 -94.40714327797907) bank58889 +58890 POINT(29.95110459989129 -95.37533301485384) bank58890 +58891 POINT(29.421035992157268 -94.42370682210077) bank58891 +58892 POINT(28.799251882621835 -94.53973622570211) bank58892 +58893 POINT(29.691372397484066 -96.10977803036164) bank58893 +58894 POINT(29.927366704298745 -94.3858490783657) bank58894 +58895 POINT(29.78003717138353 -95.32445757300232) bank58895 +58896 POINT(28.843902143955333 -95.200263954919) bank58896 +58897 POINT(30.376997513556567 -95.68066116917451) bank58897 +58898 POINT(29.732812878820454 -94.54588945613932) bank58898 +58899 POINT(29.060674676226622 -95.02335137269138) bank58899 +58900 POINT(29.36527366065115 -94.51840323740299) bank58900 +58901 POINT(30.01271894216405 -95.53386530151066) bank58901 +58902 POINT(30.002951129565904 -96.11621939591208) bank58902 +58903 POINT(28.92783786159969 -95.1838809494999) bank58903 +58904 POINT(29.36848501373178 -95.01351292827741) bank58904 +58905 POINT(30.518856185232952 -95.8769965656127) bank58905 +58906 POINT(28.856533684376487 -96.20061695791753) bank58906 +58907 POINT(29.903319042822158 -96.1615560453497) bank58907 +58908 POINT(29.677884110474327 -95.65085713935643) bank58908 +58909 POINT(29.19390329299861 -94.70127661433762) bank58909 +58910 POINT(30.536360913692562 -96.13224149365679) bank58910 +58911 POINT(29.844360539064667 -95.77082556234602) bank58911 +58912 POINT(30.754268211447453 -95.50497289818023) bank58912 +58913 POINT(30.10818733958309 -95.87971764196239) bank58913 +58914 POINT(29.941948557139753 -95.94556651585013) bank58914 +58915 POINT(29.663154810329655 -95.36817560262357) bank58915 +58916 POINT(29.381703070702414 -95.85657579564786) bank58916 +58917 POINT(30.037644403619858 -95.79161718892595) bank58917 +58918 POINT(30.591630153941516 -95.29012530485656) bank58918 +58919 POINT(30.41669809882896 -94.47969961661346) bank58919 +58920 POINT(30.653950532769823 -95.0031521106175) bank58920 +58921 POINT(30.49051051430689 -95.23228214095326) bank58921 +58922 POINT(28.966861843669083 -94.9350400118702) bank58922 +58923 POINT(30.43539127050761 -94.54227427003492) bank58923 +58924 POINT(29.02609453529283 -95.84028004362094) bank58924 +58925 POINT(28.89214563767397 -95.65283753167341) bank58925 +58926 POINT(28.842228166606805 -96.25940480864948) bank58926 +58927 POINT(30.121608723556825 -95.04015983469311) bank58927 +58928 POINT(29.82172773242776 -95.83783609167425) bank58928 +58929 POINT(29.73375034419382 -96.0987175608341) bank58929 +58930 POINT(30.513281553219848 -95.91721004821238) bank58930 +58931 POINT(28.979629167666225 -95.09820905477311) bank58931 +58932 POINT(30.669512254376176 -94.88113647822357) bank58932 +58933 POINT(29.620680299888846 -95.92043463764922) bank58933 +58934 POINT(30.51002740286891 -96.1485530810323) bank58934 +58935 POINT(30.531578511008128 -95.47895014683566) bank58935 +58936 POINT(30.522266931107556 -95.35996938571347) bank58936 +58937 POINT(29.29230391189806 -94.89911571353294) bank58937 +58938 POINT(29.556775032938607 -95.32985639208898) bank58938 +58939 POINT(30.346682863220064 -95.89509269989536) bank58939 +58940 POINT(30.01704707787364 -94.70409770763952) bank58940 +58941 POINT(29.642966880670134 -95.90693954827735) bank58941 +58942 POINT(29.094609860437462 -94.72528706596964) bank58942 +58943 POINT(29.454393166507145 -95.85114131568565) bank58943 +58944 POINT(29.088287276744655 -94.53023433551307) bank58944 +58945 POINT(29.43235054940007 -95.20803120309961) bank58945 +58946 POINT(29.06818406524544 -96.06223960536425) bank58946 +58947 POINT(29.384220861093706 -95.13354248167462) bank58947 +58948 POINT(28.92040496287085 -95.8258293531936) bank58948 +58949 POINT(29.186976573634144 -95.876164836469) bank58949 +58950 POINT(30.274216951426716 -95.68098272556396) bank58950 +58951 POINT(29.349417126881832 -94.80912618431657) bank58951 +58952 POINT(30.089976123786467 -94.81640404615139) bank58952 +58953 POINT(30.5993696026639 -95.66553518038991) bank58953 +58954 POINT(30.033587741740174 -95.320777330235) bank58954 +58955 POINT(29.746146779815003 -95.1781071735791) bank58955 +58956 POINT(29.232336331331553 -94.50675084795614) bank58956 +58957 POINT(29.120933499610796 -94.8991460152144) bank58957 +58958 POINT(29.475720674146604 -95.448217457565) bank58958 +58959 POINT(28.92023428976085 -95.65696247342913) bank58959 +58960 POINT(29.152346085296376 -95.15904924912067) bank58960 +58961 POINT(29.204696331419875 -96.19466493983579) bank58961 +58962 POINT(29.96286012291397 -95.90765543460309) bank58962 +58963 POINT(28.945392516204148 -95.55476931968161) bank58963 +58964 POINT(29.59739884813469 -95.49814383399305) bank58964 +58965 POINT(29.78123931093824 -95.49901035532335) bank58965 +58966 POINT(30.238809540891147 -96.3344610824024) bank58966 +58967 POINT(30.02313250181671 -95.072551660718) bank58967 +58968 POINT(30.11305077286377 -95.77408189411572) bank58968 +58969 POINT(29.485552407962157 -94.73994892369454) bank58969 +58970 POINT(28.906744600406746 -95.4682850131267) bank58970 +58971 POINT(30.1783528812169 -94.93284900047436) bank58971 +58972 POINT(28.969407249185437 -94.75337901008427) bank58972 +58973 POINT(29.421420976703978 -94.97844584362716) bank58973 +58974 POINT(29.02397461743139 -94.98027559055214) bank58974 +58975 POINT(29.567515259174 -94.74557412685208) bank58975 +58976 POINT(30.358905416378743 -94.79217914393611) bank58976 +58977 POINT(28.782827800284036 -95.9430889032171) bank58977 +58978 POINT(30.60107612622881 -95.02968218239839) bank58978 +58979 POINT(30.491416872695993 -96.34828616584655) bank58979 +58980 POINT(29.56381289312098 -95.12491209520185) bank58980 +58981 POINT(30.107157595182688 -94.3698615084752) bank58981 +58982 POINT(30.46334858712669 -95.46811203375852) bank58982 +58983 POINT(29.139610883783043 -95.16840799012294) bank58983 +58984 POINT(29.098177796219844 -95.42934188951563) bank58984 +58985 POINT(30.578691983472 -94.45596911339241) bank58985 +58986 POINT(30.184363598107232 -94.7600216397587) bank58986 +58987 POINT(30.725753577206397 -94.53760058000512) bank58987 +58988 POINT(30.390601506136967 -95.86647290883926) bank58988 +58989 POINT(30.151184364593078 -95.38370052072875) bank58989 +58990 POINT(30.438315464975336 -94.91556853832004) bank58990 +58991 POINT(29.061096558848565 -95.43665851697185) bank58991 +58992 POINT(29.247656999743093 -94.5070798076422) bank58992 +58993 POINT(28.784559651401352 -95.55339342721655) bank58993 +58994 POINT(28.783519353032236 -95.15860196172997) bank58994 +58995 POINT(30.16589664988124 -96.07163093602611) bank58995 +58996 POINT(29.84535231900367 -95.15698991635902) bank58996 +58997 POINT(28.97538648819514 -95.30223054946026) bank58997 +58998 POINT(29.752587976194274 -94.81732842856643) bank58998 +58999 POINT(30.722667993767633 -95.95800025464301) bank58999 +59000 POINT(30.635918184093658 -95.26236603280694) bank59000 +59001 POINT(30.625353550765578 -95.27102911509708) bank59001 +59002 POINT(29.002713056054827 -94.66968216736521) bank59002 +59003 POINT(30.16302957689656 -96.20597513568963) bank59003 +59004 POINT(30.225839554716213 -95.70842529284303) bank59004 +59005 POINT(30.20974701451607 -95.69831145446825) bank59005 +59006 POINT(28.954552222509438 -96.25979849972155) bank59006 +59007 POINT(29.657356806080447 -96.30563864458335) bank59007 +59008 POINT(29.042314923449513 -96.30739025757937) bank59008 +59009 POINT(30.51944460077826 -96.07701924242579) bank59009 +59010 POINT(29.66296951309445 -95.79347106561863) bank59010 +59011 POINT(30.557094138351868 -94.5291092554748) bank59011 +59012 POINT(30.278317918276308 -96.1117049190872) bank59012 +59013 POINT(28.97185592892284 -95.07145988737895) bank59013 +59014 POINT(30.540362808299268 -94.41185460789674) bank59014 +59015 POINT(29.199127485142785 -95.5286088669544) bank59015 +59016 POINT(29.543765429632025 -95.09620745911452) bank59016 +59017 POINT(28.82372671866508 -95.23752148312494) bank59017 +59018 POINT(29.68456670620267 -96.20829536788496) bank59018 +59019 POINT(29.306824227895113 -95.35129029790389) bank59019 +59020 POINT(30.169570473291166 -94.83410414175437) bank59020 +59021 POINT(29.812194285872913 -94.5245012465658) bank59021 +59022 POINT(29.313111562718106 -94.74420362257429) bank59022 +59023 POINT(28.841749684639666 -94.9966770346584) bank59023 +59024 POINT(29.941096329653902 -95.01678647058607) bank59024 +59025 POINT(29.385816122967064 -95.38427681903694) bank59025 +59026 POINT(29.047234406598662 -95.3662183362729) bank59026 +59027 POINT(28.800508268999785 -94.47697595403234) bank59027 +59028 POINT(29.84115962288508 -95.23553986134652) bank59028 +59029 POINT(29.76518263092079 -95.77062789443475) bank59029 +59030 POINT(29.121171130939665 -95.81760713997136) bank59030 +59031 POINT(29.22663839917829 -96.07640291204196) bank59031 +59032 POINT(28.8713526162208 -94.64052502202836) bank59032 +59033 POINT(28.940938652943256 -96.18869733236212) bank59033 +59034 POINT(29.94816151818486 -95.58780949520524) bank59034 +59035 POINT(30.07871145737186 -94.39229284789215) bank59035 +59036 POINT(30.24241666090489 -94.70069222952867) bank59036 +59037 POINT(30.277460906275984 -96.26930179417698) bank59037 +59038 POINT(29.293916845494277 -94.71740274707426) bank59038 +59039 POINT(29.013559414877115 -95.83734761013876) bank59039 +59040 POINT(28.766275569929682 -96.1071844792424) bank59040 +59041 POINT(29.078240040118295 -94.54023772285693) bank59041 +59042 POINT(29.13912726149124 -96.06779797923906) bank59042 +59043 POINT(29.75681705995749 -95.68404758101802) bank59043 +59044 POINT(29.816216998272147 -95.07785890433661) bank59044 +59045 POINT(29.540684788357126 -96.09587326730794) bank59045 +59046 POINT(29.62422779385738 -95.57487511326617) bank59046 +59047 POINT(30.261442058282917 -96.22611215889577) bank59047 +59048 POINT(30.332402408389903 -94.96141757546783) bank59048 +59049 POINT(28.9287394121116 -94.73533094450445) bank59049 +59050 POINT(28.93338121455012 -95.4334073039809) bank59050 +59051 POINT(29.7472897349039 -94.73016893801287) bank59051 +59052 POINT(29.504556251673495 -95.42457317933275) bank59052 +59053 POINT(29.460037691935142 -95.4523961067469) bank59053 +59054 POINT(30.640384697123324 -95.08808547464022) bank59054 +59055 POINT(29.732886958006663 -96.07049844878561) bank59055 +59056 POINT(30.662069551578284 -95.13741900549176) bank59056 +59057 POINT(30.357414242140486 -94.40237943693035) bank59057 +59058 POINT(30.611174503080846 -94.41460688228797) bank59058 +59059 POINT(29.697257472947648 -96.05638244643806) bank59059 +59060 POINT(29.727586773660413 -95.12677810930938) bank59060 +59061 POINT(29.513874918771513 -94.55902808467845) bank59061 +59062 POINT(29.56534189830251 -95.88885394460773) bank59062 +59063 POINT(29.00799013544524 -94.7111809332253) bank59063 +59064 POINT(29.433197252732295 -94.4043565942945) bank59064 +59065 POINT(29.800580145719362 -96.07776232092914) bank59065 +59066 POINT(29.296312178408076 -95.62554872803491) bank59066 +59067 POINT(30.600040185701527 -96.09187686002461) bank59067 +59068 POINT(30.446424499138903 -95.69985561074243) bank59068 +59069 POINT(30.62364893917695 -95.50027734927824) bank59069 +59070 POINT(29.211959443227755 -95.4491489166439) bank59070 +59071 POINT(28.81874944232441 -94.92845579933407) bank59071 +59072 POINT(30.264381127973536 -95.17013451592113) bank59072 +59073 POINT(29.29256002271574 -94.79228792554306) bank59073 +59074 POINT(30.40726072821557 -95.245009307085) bank59074 +59075 POINT(29.787399737322964 -96.14180708552382) bank59075 +59076 POINT(28.893564934981253 -95.17657286744976) bank59076 +59077 POINT(28.987369261713773 -95.65243277844391) bank59077 +59078 POINT(29.70195137710102 -94.6101246666428) bank59078 +59079 POINT(30.158921866330274 -95.95311100323904) bank59079 +59080 POINT(28.944566409312728 -94.60999384898236) bank59080 +59081 POINT(29.777592303535577 -94.99645661886854) bank59081 +59082 POINT(28.88162259918992 -96.36851424259224) bank59082 +59083 POINT(28.926635357449605 -94.44011195860598) bank59083 +59084 POINT(30.38431495356519 -94.37808743250253) bank59084 +59085 POINT(29.78391505603014 -95.75088084300137) bank59085 +59086 POINT(30.717060786975814 -95.00752884726455) bank59086 +59087 POINT(29.93397365014381 -96.21810561427792) bank59087 +59088 POINT(30.044140832380542 -94.94410290765747) bank59088 +59089 POINT(30.375667095754512 -94.89326904562878) bank59089 +59090 POINT(30.405735801429323 -96.12436102330567) bank59090 +59091 POINT(30.48181582369866 -95.71693621553958) bank59091 +59092 POINT(30.273749283482353 -95.15620477662267) bank59092 +59093 POINT(29.600983879740262 -95.65564115288484) bank59093 +59094 POINT(30.097606448425527 -95.52731661706724) bank59094 +59095 POINT(30.758357280003125 -95.47412634816924) bank59095 +59096 POINT(29.297051077790005 -96.25441754110565) bank59096 +59097 POINT(29.698060958317097 -94.92873939265954) bank59097 +59098 POINT(28.940303494062665 -94.84006051136608) bank59098 +59099 POINT(29.45062389000323 -94.99255130295417) bank59099 +59100 POINT(29.816322808207783 -94.934608778658) bank59100 +59101 POINT(29.612110312823994 -95.63864332716558) bank59101 +59102 POINT(29.91090366664854 -94.8036843729251) bank59102 +59103 POINT(30.26263144032724 -95.47428585607948) bank59103 +59104 POINT(30.020181459661725 -95.83580173919053) bank59104 +59105 POINT(30.23221092885499 -95.44241692984852) bank59105 +59106 POINT(29.70214125093447 -94.86819212515988) bank59106 +59107 POINT(29.254740049174504 -95.74702310174177) bank59107 +59108 POINT(29.164844882533718 -95.05707408925305) bank59108 +59109 POINT(30.11959875254649 -95.46044788324545) bank59109 +59110 POINT(30.128728611399293 -95.66665925628585) bank59110 +59111 POINT(30.420561364587364 -95.14822634643141) bank59111 +59112 POINT(29.276305857014886 -95.02271570175223) bank59112 +59113 POINT(28.903070248301912 -94.93839509847555) bank59113 +59114 POINT(28.868802073968343 -94.72388988341338) bank59114 +59115 POINT(30.23361682345095 -95.61141697826349) bank59115 +59116 POINT(29.266724378984858 -94.49153144121293) bank59116 +59117 POINT(29.589686164960824 -94.86791163003618) bank59117 +59118 POINT(30.521593369687565 -95.03768506213112) bank59118 +59119 POINT(28.830128760015064 -94.75448820935598) bank59119 +59120 POINT(29.526621810239202 -96.01977183142137) bank59120 +59121 POINT(29.117588682637763 -96.34591831148774) bank59121 +59122 POINT(30.631028838604013 -94.86750601676006) bank59122 +59123 POINT(29.27927287174836 -96.10485120261518) bank59123 +59124 POINT(29.301028904288703 -96.0414213069668) bank59124 +59125 POINT(30.570269720533553 -94.42346801815191) bank59125 +59126 POINT(30.082052709902204 -95.78741480405314) bank59126 +59127 POINT(30.032277328364188 -95.19043883135605) bank59127 +59128 POINT(30.642508648732637 -95.83977890017604) bank59128 +59129 POINT(29.582546589672596 -95.60071106911683) bank59129 +59130 POINT(30.289976377305763 -94.50515839717254) bank59130 +59131 POINT(29.65576218702364 -95.2947323150708) bank59131 +59132 POINT(30.230460921165534 -94.90622821218933) bank59132 +59133 POINT(30.52471754879215 -95.71709244783348) bank59133 +59134 POINT(30.004992404243932 -95.49206334456566) bank59134 +59135 POINT(29.974556569569273 -95.62059428650751) bank59135 +59136 POINT(28.939084325301543 -95.57773560064528) bank59136 +59137 POINT(30.561253283855986 -95.15155050094425) bank59137 +59138 POINT(29.361262261175263 -94.87344319766285) bank59138 +59139 POINT(30.717365267409793 -95.66284603494498) bank59139 +59140 POINT(29.33147575927957 -95.1819964478593) bank59140 +59141 POINT(29.223428099098562 -95.60185621396091) bank59141 +59142 POINT(30.657949255720855 -95.67069631762203) bank59142 +59143 POINT(29.714370720184345 -95.7220928243622) bank59143 +59144 POINT(30.470507912625358 -94.6370316277847) bank59144 +59145 POINT(28.87688565187882 -96.04298276999886) bank59145 +59146 POINT(29.563601856406642 -95.16572427094832) bank59146 +59147 POINT(29.935390247494123 -94.89871311416253) bank59147 +59148 POINT(29.452929778802687 -95.7437886833175) bank59148 +59149 POINT(28.838579835890556 -94.8609958346086) bank59149 +59150 POINT(28.815189236956744 -95.21764500409085) bank59150 +59151 POINT(29.771454258393703 -96.36218744454276) bank59151 +59152 POINT(30.067577875763508 -95.04867105877757) bank59152 +59153 POINT(30.67220180833509 -95.20756364808013) bank59153 +59154 POINT(29.603902366251216 -96.29137048179432) bank59154 +59155 POINT(28.991436472760952 -95.46439048749525) bank59155 +59156 POINT(28.821513528306486 -94.87768943200916) bank59156 +59157 POINT(29.533151054045458 -95.36790959044) bank59157 +59158 POINT(29.5559839867236 -94.88172597302193) bank59158 +59159 POINT(29.341509541030675 -94.55578827049038) bank59159 +59160 POINT(30.556500892117946 -94.71529493399522) bank59160 +59161 POINT(29.04265946786829 -96.07610319212284) bank59161 +59162 POINT(29.615640444462443 -96.29479427519101) bank59162 +59163 POINT(29.65603156904416 -95.11152396417485) bank59163 +59164 POINT(30.42367921315796 -95.28612609964888) bank59164 +59165 POINT(29.85360629321284 -95.28728861351814) bank59165 +59166 POINT(28.85762268268379 -96.32349220207811) bank59166 +59167 POINT(29.79963227312203 -95.64002759605589) bank59167 +59168 POINT(29.989113562457245 -94.3828309042326) bank59168 +59169 POINT(30.443478228644434 -95.95927163465497) bank59169 +59170 POINT(30.67270269486512 -96.21745527392659) bank59170 +59171 POINT(30.421029758357548 -94.4744691366907) bank59171 +59172 POINT(28.803618295296168 -95.0695445320867) bank59172 +59173 POINT(29.51078333893764 -95.84632460924354) bank59173 +59174 POINT(28.915454424376655 -95.59979939926674) bank59174 +59175 POINT(30.7340291959323 -95.38698137362273) bank59175 +59176 POINT(29.712146055557405 -94.87667543537039) bank59176 +59177 POINT(29.752497902669706 -96.2539547037883) bank59177 +59178 POINT(28.987654848946956 -95.85628795416355) bank59178 +59179 POINT(30.134973679702355 -96.08334982127846) bank59179 +59180 POINT(29.53647705719709 -95.39546639248124) bank59180 +59181 POINT(29.95725026869146 -95.75043225679464) bank59181 +59182 POINT(29.40498626986327 -95.99846335382264) bank59182 +59183 POINT(29.997780413015082 -94.88556034536087) bank59183 +59184 POINT(29.37967490627765 -95.05720610067374) bank59184 +59185 POINT(28.883125325861506 -95.43099848500775) bank59185 +59186 POINT(30.36753227753065 -95.05963310708167) bank59186 +59187 POINT(30.471785219569483 -96.21521556529903) bank59187 +59188 POINT(30.53440245809681 -95.94612764662463) bank59188 +59189 POINT(30.712908362949285 -94.59253792098897) bank59189 +59190 POINT(29.856283478594946 -95.96283244798212) bank59190 +59191 POINT(29.043285502586535 -94.85719504865162) bank59191 +59192 POINT(30.683179987074546 -95.83320426820188) bank59192 +59193 POINT(30.337911971473694 -95.34792155759025) bank59193 +59194 POINT(30.719162251877854 -94.60743720054) bank59194 +59195 POINT(30.658661895927654 -95.28203409647914) bank59195 +59196 POINT(30.280289642112038 -95.90604626674737) bank59196 +59197 POINT(30.224518556196273 -96.17820510597802) bank59197 +59198 POINT(30.66355248201854 -95.33770759951362) bank59198 +59199 POINT(30.490007501260955 -95.35090363380608) bank59199 +59200 POINT(29.215973446529993 -95.1305073906867) bank59200 +59201 POINT(29.964642082898425 -95.03363937786524) bank59201 +59202 POINT(30.622378187571094 -94.97954281402787) bank59202 +59203 POINT(29.56386668665654 -95.4327425294976) bank59203 +59204 POINT(29.81142843759936 -96.28172221433991) bank59204 +59205 POINT(30.32736717333058 -96.10940584347948) bank59205 +59206 POINT(30.558773393148396 -95.37500090550061) bank59206 +59207 POINT(30.29712094200951 -94.37159129692843) bank59207 +59208 POINT(28.819418637069898 -95.98657953237638) bank59208 +59209 POINT(30.37614372602687 -96.32671138157946) bank59209 +59210 POINT(30.06551064754113 -95.26940065463928) bank59210 +59211 POINT(29.380702754187574 -95.05266893051717) bank59211 +59212 POINT(29.303257720778596 -96.11507139884422) bank59212 +59213 POINT(28.828038901575457 -96.2270730812811) bank59213 +59214 POINT(28.91803827863814 -95.43923593043426) bank59214 +59215 POINT(29.981896478364053 -94.4739022052836) bank59215 +59216 POINT(29.28348476968032 -94.64608721149429) bank59216 +59217 POINT(29.923990708001416 -95.07255842367559) bank59217 +59218 POINT(29.670455635528604 -95.82313250175478) bank59218 +59219 POINT(29.143833529132618 -95.04215245044145) bank59219 +59220 POINT(29.000697700502684 -94.7585900284428) bank59220 +59221 POINT(30.67876028493335 -95.63432049234092) bank59221 +59222 POINT(29.92122523039657 -95.4968948708869) bank59222 +59223 POINT(29.478073996725072 -94.3955740552429) bank59223 +59224 POINT(29.56645763052993 -94.89697120562592) bank59224 +59225 POINT(28.89186169027391 -95.02734193800036) bank59225 +59226 POINT(30.34402156594303 -95.80112409940605) bank59226 +59227 POINT(29.129019399466454 -94.60643792914442) bank59227 +59228 POINT(28.989833888495166 -94.70816587388371) bank59228 +59229 POINT(29.785781792039298 -95.80490297365563) bank59229 +59230 POINT(29.899483096191382 -95.20013485113004) bank59230 +59231 POINT(28.97221171068515 -94.37753812174698) bank59231 +59232 POINT(30.62339608857711 -95.22745333183764) bank59232 +59233 POINT(30.08382632310635 -95.70004451497026) bank59233 +59234 POINT(29.780178578217416 -94.8917659835731) bank59234 +59235 POINT(30.61697275534694 -95.82798612292339) bank59235 +59236 POINT(29.132341671880088 -96.20928088068098) bank59236 +59237 POINT(30.114891965313824 -95.12858189786955) bank59237 +59238 POINT(30.43613212022899 -96.22553144938294) bank59238 +59239 POINT(29.195358101108894 -94.8781737748327) bank59239 +59240 POINT(30.724691176991833 -95.02771197793574) bank59240 +59241 POINT(30.646340026465783 -95.64278361982578) bank59241 +59242 POINT(29.083588647013322 -95.19303225411565) bank59242 +59243 POINT(29.70846736376885 -94.73021023761339) bank59243 +59244 POINT(29.620449092574926 -94.59426604646357) bank59244 +59245 POINT(30.27584930181908 -95.67226117306357) bank59245 +59246 POINT(29.63846716433331 -95.04487558472702) bank59246 +59247 POINT(29.457999939152444 -96.2248236873831) bank59247 +59248 POINT(28.829199277631915 -95.97447200181331) bank59248 +59249 POINT(29.055608266963752 -95.7499929222209) bank59249 +59250 POINT(29.401222224298014 -95.71901267957027) bank59250 +59251 POINT(30.11642589644072 -94.44390114858672) bank59251 +59252 POINT(30.132613285010457 -96.2457119165468) bank59252 +59253 POINT(29.94666475191357 -95.73701697358612) bank59253 +59254 POINT(28.930209815943744 -96.20966949068783) bank59254 +59255 POINT(29.663124607223544 -95.83295540927924) bank59255 +59256 POINT(30.549431061090214 -96.2809575961687) bank59256 +59257 POINT(28.87715901419677 -94.86345631746417) bank59257 +59258 POINT(29.302673701106357 -94.77009282836805) bank59258 +59259 POINT(30.569982158375776 -95.03696872934451) bank59259 +59260 POINT(29.080489845107547 -95.46955029772052) bank59260 +59261 POINT(30.328309285086934 -94.51516723253646) bank59261 +59262 POINT(30.033273974934943 -94.4843990857378) bank59262 +59263 POINT(29.481030493329325 -96.34483940445394) bank59263 +59264 POINT(28.915279597481458 -96.25941774656445) bank59264 +59265 POINT(29.341926275992037 -95.63254707134085) bank59265 +59266 POINT(28.974263923769083 -96.05549353273808) bank59266 +59267 POINT(30.0127796397099 -94.50511976866038) bank59267 +59268 POINT(29.185669150288668 -95.10639460374388) bank59268 +59269 POINT(29.83800031071721 -95.52896688565681) bank59269 +59270 POINT(30.269040345915997 -94.71906162289386) bank59270 +59271 POINT(29.092637698356334 -94.40638180257058) bank59271 +59272 POINT(30.340188401645502 -95.52440984099482) bank59272 +59273 POINT(30.700528593931242 -95.55179793237542) bank59273 +59274 POINT(29.034611259271312 -96.12979535128686) bank59274 +59275 POINT(30.15145468804049 -95.82600181616037) bank59275 +59276 POINT(30.54648603188339 -95.79282220243913) bank59276 +59277 POINT(28.995783757514744 -96.34152180655255) bank59277 +59278 POINT(29.978094368419825 -95.28226222788496) bank59278 +59279 POINT(29.5623293399468 -96.19209732012064) bank59279 +59280 POINT(29.69692879914284 -95.93081778927883) bank59280 +59281 POINT(30.604523507313026 -94.76719527787714) bank59281 +59282 POINT(30.16844169299445 -95.19341383341589) bank59282 +59283 POINT(30.13724639420506 -96.21211396590866) bank59283 +59284 POINT(29.41778539539588 -96.07947135899802) bank59284 +59285 POINT(28.934242088415502 -95.96069356618005) bank59285 +59286 POINT(29.70507484108643 -95.08493074296472) bank59286 +59287 POINT(30.01036579710913 -94.91934340794695) bank59287 +59288 POINT(30.043855319024914 -95.15129390536235) bank59288 +59289 POINT(29.475780660830583 -94.38814983492283) bank59289 +59290 POINT(30.159679432809266 -95.63938497462873) bank59290 +59291 POINT(29.803530472408386 -95.82913839946092) bank59291 +59292 POINT(30.06187469944698 -94.73521905976415) bank59292 +59293 POINT(29.075861918129192 -95.73721947791422) bank59293 +59294 POINT(30.061034833229225 -95.8632793669635) bank59294 +59295 POINT(30.744218230455093 -95.4912522563941) bank59295 +59296 POINT(30.693160606321705 -95.9632638709564) bank59296 +59297 POINT(30.718562019910564 -95.7714651776531) bank59297 +59298 POINT(30.470499707640194 -94.55200861980512) bank59298 +59299 POINT(30.316549375481294 -95.92466528079198) bank59299 +59300 POINT(29.962878743334304 -95.4770275344713) bank59300 +59301 POINT(29.052356006509175 -96.29829711396894) bank59301 +59302 POINT(28.915703032569862 -95.83227894728614) bank59302 +59303 POINT(30.663104919272374 -95.44763209193242) bank59303 +59304 POINT(29.107987067982968 -94.39497624252395) bank59304 +59305 POINT(28.992728007325763 -94.58310167567383) bank59305 +59306 POINT(30.03598841901255 -96.20058758735067) bank59306 +59307 POINT(28.78460263045416 -95.85289586588398) bank59307 +59308 POINT(29.433050146270993 -96.21399309120002) bank59308 +59309 POINT(30.253423709656683 -95.08960424767699) bank59309 +59310 POINT(30.389547311017 -95.24522467162603) bank59310 +59311 POINT(29.883240690102898 -95.03598628481296) bank59311 +59312 POINT(29.51428006902619 -94.91983446509967) bank59312 +59313 POINT(30.348794718119947 -95.03919367269755) bank59313 +59314 POINT(29.04622965118303 -94.76964041504455) bank59314 +59315 POINT(29.657437559989184 -95.96955961074937) bank59315 +59316 POINT(30.215547119002796 -95.86486661816927) bank59316 +59317 POINT(30.335941903251058 -95.31085853298954) bank59317 +59318 POINT(29.580958175178342 -95.28446552708185) bank59318 +59319 POINT(30.503684950035648 -95.03218502858242) bank59319 +59320 POINT(28.851129221299487 -94.58065081658309) bank59320 +59321 POINT(29.4449863188984 -94.60823006732677) bank59321 +59322 POINT(29.235554679709676 -95.01004483019906) bank59322 +59323 POINT(29.007577203893238 -95.19753402387113) bank59323 +59324 POINT(30.02572663995755 -96.36419433339367) bank59324 +59325 POINT(29.551933017430205 -95.57352075324236) bank59325 +59326 POINT(29.167889326176237 -96.03224097404849) bank59326 +59327 POINT(29.697951171145107 -95.69434101517987) bank59327 +59328 POINT(29.30474001116621 -96.30192037850921) bank59328 +59329 POINT(29.577763327407133 -95.95936486051502) bank59329 +59330 POINT(30.19658331420763 -95.39588620892843) bank59330 +59331 POINT(29.810645066718553 -95.40204003220651) bank59331 +59332 POINT(30.397977996496703 -95.4098310760127) bank59332 +59333 POINT(28.93317058262967 -95.4384939587632) bank59333 +59334 POINT(29.4964199643002 -95.96345826604319) bank59334 +59335 POINT(30.640604866504578 -95.57502240528164) bank59335 +59336 POINT(30.0561261092761 -96.0116151420479) bank59336 +59337 POINT(29.105514257896544 -94.96856309283471) bank59337 +59338 POINT(29.49114605381474 -94.77209784952149) bank59338 +59339 POINT(29.498116729397182 -96.14335552575412) bank59339 +59340 POINT(30.392080943046807 -94.99864106333328) bank59340 +59341 POINT(30.56277991635806 -95.5573792294846) bank59341 +59342 POINT(29.990310252443052 -96.23779171937652) bank59342 +59343 POINT(30.6444911243685 -95.8945040371091) bank59343 +59344 POINT(28.926354699503353 -94.47506525658021) bank59344 +59345 POINT(30.18869647461334 -96.10473539021388) bank59345 +59346 POINT(29.010744698758593 -95.50750315366206) bank59346 +59347 POINT(29.68275225027458 -96.224327309407) bank59347 +59348 POINT(29.51342895035948 -94.91077065337097) bank59348 +59349 POINT(29.93998848873377 -95.8644504730718) bank59349 +59350 POINT(30.729094176020986 -95.43932676554807) bank59350 +59351 POINT(30.30726394772781 -94.8153252848297) bank59351 +59352 POINT(29.465103408858752 -95.57500427160758) bank59352 +59353 POINT(30.02115924591408 -96.25682343139707) bank59353 +59354 POINT(29.065527269995993 -96.21659494083923) bank59354 +59355 POINT(29.56375802683434 -94.8955174083673) bank59355 +59356 POINT(29.431119716838065 -94.9132748131492) bank59356 +59357 POINT(29.427739952142506 -94.8201294573752) bank59357 +59358 POINT(29.564081824895855 -96.36126911114786) bank59358 +59359 POINT(30.350452266180937 -95.82194037967841) bank59359 +59360 POINT(30.18794823272814 -95.02206438196643) bank59360 +59361 POINT(29.970246032605928 -94.80721337618543) bank59361 +59362 POINT(29.08750157322815 -96.34156710807822) bank59362 +59363 POINT(28.79600088842717 -94.78229924350462) bank59363 +59364 POINT(30.457981697880935 -95.40110258992283) bank59364 +59365 POINT(28.840536218280036 -95.50766704879439) bank59365 +59366 POINT(29.05266016047888 -96.23262620497933) bank59366 +59367 POINT(28.902221555871225 -95.14144874697347) bank59367 +59368 POINT(30.429169219050518 -94.5184534480211) bank59368 +59369 POINT(28.897295316288933 -95.24701265567691) bank59369 +59370 POINT(28.86607748501987 -95.38714566050962) bank59370 +59371 POINT(30.55505610660898 -95.96768384164615) bank59371 +59372 POINT(29.001856465286274 -95.56844679480987) bank59372 +59373 POINT(29.647113727631844 -94.71402044774126) bank59373 +59374 POINT(29.173618736815914 -96.27123756044756) bank59374 +59375 POINT(29.521799116191303 -96.02048113919071) bank59375 +59376 POINT(29.513643634674995 -95.29399369198889) bank59376 +59377 POINT(28.919617575171355 -95.160403290027) bank59377 +59378 POINT(29.061080374914887 -96.01835238669446) bank59378 +59379 POINT(29.032506456171745 -94.46736833102713) bank59379 +59380 POINT(30.186591661433198 -96.24129381457966) bank59380 +59381 POINT(29.732006130394222 -95.11712456428631) bank59381 +59382 POINT(30.361648449010904 -95.1466792407417) bank59382 +59383 POINT(29.89449314031592 -95.08310511298738) bank59383 +59384 POINT(29.997338650088256 -95.86716932484264) bank59384 +59385 POINT(30.021537102561485 -94.40725762812916) bank59385 +59386 POINT(29.66402419298429 -95.28789833386926) bank59386 +59387 POINT(29.54371579756562 -95.76718569277851) bank59387 +59388 POINT(28.865572209434742 -96.07739622706845) bank59388 +59389 POINT(30.3427489578935 -96.25104461698696) bank59389 +59390 POINT(30.27557704475992 -95.05191867123747) bank59390 +59391 POINT(29.06314548554158 -95.89634549972389) bank59391 +59392 POINT(29.53661567376907 -95.14648305708427) bank59392 +59393 POINT(30.235951768964014 -94.50386267687455) bank59393 +59394 POINT(30.555785075037946 -94.91332721335849) bank59394 +59395 POINT(30.008369091681242 -94.42238283869432) bank59395 +59396 POINT(30.362797587327897 -96.32699857782893) bank59396 +59397 POINT(28.833371754819805 -96.14477494244667) bank59397 +59398 POINT(30.553205281590202 -96.29178730223303) bank59398 +59399 POINT(30.414277372809124 -96.22224183839054) bank59399 +59400 POINT(29.01254242136141 -95.80540727049676) bank59400 +59401 POINT(29.449657787245076 -95.66324193295348) bank59401 +59402 POINT(29.933547890937454 -95.07963152105945) bank59402 +59403 POINT(30.661398557412866 -96.05025376232624) bank59403 +59404 POINT(29.839282687758164 -94.4462140207124) bank59404 +59405 POINT(29.100666651547346 -94.64467744381955) bank59405 +59406 POINT(29.16693438944411 -96.17258572912533) bank59406 +59407 POINT(29.466383661880695 -95.07230329754276) bank59407 +59408 POINT(29.68692366168805 -94.93081122806161) bank59408 +59409 POINT(28.859795535634525 -95.68674741785456) bank59409 +59410 POINT(29.07562605921941 -95.07786109536656) bank59410 +59411 POINT(29.186695092544973 -95.71068336667588) bank59411 +59412 POINT(28.781137591702475 -95.63227713570598) bank59412 +59413 POINT(30.09622985411788 -95.47056617749313) bank59413 +59414 POINT(29.885526002277835 -94.75315534799988) bank59414 +59415 POINT(30.072234828544108 -95.69792994680336) bank59415 +59416 POINT(29.669629498297766 -94.72834986659583) bank59416 +59417 POINT(29.426035212893456 -95.26618558953717) bank59417 +59418 POINT(29.707065716509415 -95.8319564789777) bank59418 +59419 POINT(30.08215752397805 -94.77378373115572) bank59419 +59420 POINT(28.84154898695556 -95.88458573083606) bank59420 +59421 POINT(29.685183965398878 -94.82780667231219) bank59421 +59422 POINT(30.52968588721682 -95.84801435995605) bank59422 +59423 POINT(29.2126300846134 -94.48475535233257) bank59423 +59424 POINT(30.48212094929879 -96.09897749002279) bank59424 +59425 POINT(30.091751893958676 -94.9299088188411) bank59425 +59426 POINT(30.707489100283922 -95.58983343533077) bank59426 +59427 POINT(29.488931187701116 -94.87451644372494) bank59427 +59428 POINT(29.636131610048384 -94.80070696577383) bank59428 +59429 POINT(30.197908511092905 -94.5437916064163) bank59429 +59430 POINT(29.732188348014596 -95.9638071788574) bank59430 +59431 POINT(30.090068688758386 -94.48577894371661) bank59431 +59432 POINT(29.57929182925323 -95.73268941878464) bank59432 +59433 POINT(28.904075956600337 -95.22332762181304) bank59433 +59434 POINT(30.333622505724836 -95.29208292312688) bank59434 +59435 POINT(29.559231939040792 -94.39755076548545) bank59435 +59436 POINT(30.084152479208672 -94.56691737308277) bank59436 +59437 POINT(29.182786135645056 -95.38477675837522) bank59437 +59438 POINT(29.620105802798797 -94.51088609316534) bank59438 +59439 POINT(29.13215573824881 -94.41840670633277) bank59439 +59440 POINT(30.665916048891518 -94.96047170375147) bank59440 +59441 POINT(29.02129574965025 -95.96906691400083) bank59441 +59442 POINT(30.215615823173295 -95.28929681734974) bank59442 +59443 POINT(29.51020391840994 -95.98754278163814) bank59443 +59444 POINT(29.636610350297264 -96.32887487342651) bank59444 +59445 POINT(29.30567901848026 -96.0084067135901) bank59445 +59446 POINT(30.39132092072784 -96.29245845996266) bank59446 +59447 POINT(29.511955208566285 -94.4556835779349) bank59447 +59448 POINT(29.416551515786164 -95.36135980604851) bank59448 +59449 POINT(28.866239390009493 -96.30763013464994) bank59449 +59450 POINT(29.33488987001168 -96.13361514689977) bank59450 +59451 POINT(30.54430906513336 -95.36231086138069) bank59451 +59452 POINT(30.295095086724952 -95.77924855459239) bank59452 +59453 POINT(29.464573680739328 -95.36736428697508) bank59453 +59454 POINT(30.005002790763488 -95.34601681622796) bank59454 +59455 POINT(28.998177033790114 -96.05873743719562) bank59455 +59456 POINT(30.480828861411215 -96.01670058795601) bank59456 +59457 POINT(29.637963434885993 -94.78191102215378) bank59457 +59458 POINT(28.877222786586866 -94.98215308471008) bank59458 +59459 POINT(30.577833600220682 -94.86479995765286) bank59459 +59460 POINT(30.698495355598993 -95.24074393513672) bank59460 +59461 POINT(29.62309535613209 -95.29204238847545) bank59461 +59462 POINT(30.513132679689456 -94.38597974979166) bank59462 +59463 POINT(29.445132486300935 -96.1783509178539) bank59463 +59464 POINT(30.557408967278427 -95.69364731512259) bank59464 +59465 POINT(29.7728208475828 -94.48513412791249) bank59465 +59466 POINT(29.548293883943828 -95.78726925412859) bank59466 +59467 POINT(30.165205522222763 -94.91486692407209) bank59467 +59468 POINT(30.655176915870527 -96.31377500206858) bank59468 +59469 POINT(30.655006194782075 -94.79777869773946) bank59469 +59470 POINT(29.7250307044691 -95.02521127213191) bank59470 +59471 POINT(29.520449093597236 -96.16906308249716) bank59471 +59472 POINT(30.35160429722657 -96.01652404801588) bank59472 +59473 POINT(29.16520506270522 -96.03599139100083) bank59473 +59474 POINT(29.129968502345978 -96.1613466280445) bank59474 +59475 POINT(30.72512287432244 -96.15517117380566) bank59475 +59476 POINT(28.85944604136929 -96.05919668817961) bank59476 +59477 POINT(30.036158819108106 -95.31164289512552) bank59477 +59478 POINT(30.642424089429912 -95.757434061405) bank59478 +59479 POINT(28.83246167147691 -95.5872590887439) bank59479 +59480 POINT(30.39596156380231 -96.31798495884956) bank59480 +59481 POINT(29.08382967780254 -96.34491265549872) bank59481 +59482 POINT(30.09992918565228 -95.27806157331587) bank59482 +59483 POINT(28.928143163032754 -95.89189247166556) bank59483 +59484 POINT(29.27143805523715 -95.23270904137817) bank59484 +59485 POINT(30.332121361164834 -95.45130007111491) bank59485 +59486 POINT(30.33261582015131 -95.32646437916864) bank59486 +59487 POINT(30.470071529055446 -94.69348405861368) bank59487 +59488 POINT(30.71376597058043 -96.1719058500834) bank59488 +59489 POINT(29.227881892945568 -95.85397225552485) bank59489 +59490 POINT(29.389497992496153 -95.33507766363411) bank59490 +59491 POINT(30.69024812628402 -95.8392929956507) bank59491 +59492 POINT(28.860195090216422 -95.0762631594932) bank59492 +59493 POINT(30.22092529164983 -94.52523752464452) bank59493 +59494 POINT(29.273106225183568 -94.58364712790849) bank59494 +59495 POINT(30.072222926289125 -94.75766574063731) bank59495 +59496 POINT(29.674824400141528 -94.70460450439913) bank59496 +59497 POINT(29.438431133293477 -96.2645443506589) bank59497 +59498 POINT(30.189504478237037 -94.97221597699607) bank59498 +59499 POINT(29.926426561710414 -95.15155576947915) bank59499 +59500 POINT(28.95009017624836 -95.39597273822186) bank59500 +59501 POINT(30.21916450667236 -95.74797563947752) bank59501 +59502 POINT(28.76392809147185 -96.34794332682992) bank59502 +59503 POINT(30.598209888000618 -96.03398183199187) bank59503 +59504 POINT(29.11370903744189 -94.87212363413495) bank59504 +59505 POINT(30.045960297388554 -95.86967235356079) bank59505 +59506 POINT(29.25680704078575 -94.65473083244808) bank59506 +59507 POINT(29.341967139188576 -94.67779837046695) bank59507 +59508 POINT(29.867401458031686 -94.91746086671203) bank59508 +59509 POINT(28.846340400972593 -96.33783869603528) bank59509 +59510 POINT(29.841814394020275 -96.31981643697284) bank59510 +59511 POINT(29.95599871343736 -95.34393956213991) bank59511 +59512 POINT(29.266367880723653 -95.33799664198787) bank59512 +59513 POINT(29.660332026221212 -95.81418230045954) bank59513 +59514 POINT(29.230538016491742 -96.1741980556153) bank59514 +59515 POINT(30.56792632399492 -94.59708765007488) bank59515 +59516 POINT(28.812634571452477 -95.83883853439642) bank59516 +59517 POINT(29.694468214357347 -95.39008189432461) bank59517 +59518 POINT(29.338061731074944 -95.08409761485586) bank59518 +59519 POINT(29.477446331028133 -96.29660390430199) bank59519 +59520 POINT(29.879592650215756 -94.79291901474727) bank59520 +59521 POINT(29.680948159907135 -95.24528618360858) bank59521 +59522 POINT(29.411113718546833 -95.14039745936148) bank59522 +59523 POINT(29.99782850494201 -96.23269185917638) bank59523 +59524 POINT(29.62612546706478 -94.3722895745659) bank59524 +59525 POINT(30.00283017235017 -95.96035775272637) bank59525 +59526 POINT(30.206549610471672 -94.71528744067044) bank59526 +59527 POINT(30.44639126633896 -95.68057503044923) bank59527 +59528 POINT(29.017151592044755 -94.89368462453045) bank59528 +59529 POINT(30.168663872493124 -95.86902075782344) bank59529 +59530 POINT(30.0630832036962 -96.18154939015386) bank59530 +59531 POINT(30.089124725930848 -94.75122705706407) bank59531 +59532 POINT(30.718543529142124 -94.62862116719778) bank59532 +59533 POINT(29.90777306535739 -96.36397435595602) bank59533 +59534 POINT(30.279768650573594 -95.88204682023486) bank59534 +59535 POINT(30.217654287629713 -94.649460898102) bank59535 +59536 POINT(29.406822220492977 -95.8649225370051) bank59536 +59537 POINT(28.812919651114775 -95.85073479794494) bank59537 +59538 POINT(29.709581724579035 -94.73946401172901) bank59538 +59539 POINT(28.940127911292695 -95.51448222679382) bank59539 +59540 POINT(28.9959935378426 -94.87437673168829) bank59540 +59541 POINT(29.935076152596576 -95.78368140348445) bank59541 +59542 POINT(29.515636978775795 -96.00544701957548) bank59542 +59543 POINT(29.859215501496344 -95.10190697974963) bank59543 +59544 POINT(30.072448121430575 -95.58279941112248) bank59544 +59545 POINT(29.6845490879325 -96.34328206234864) bank59545 +59546 POINT(29.676991040222088 -94.8093440604847) bank59546 +59547 POINT(29.48822875658488 -95.0278356236895) bank59547 +59548 POINT(29.260444016374 -95.17443396923153) bank59548 +59549 POINT(29.2664825064808 -94.93279869689226) bank59549 +59550 POINT(28.799931791869398 -95.29481974957575) bank59550 +59551 POINT(29.703960045907916 -95.52030878616344) bank59551 +59552 POINT(29.809184837088367 -95.8919041185462) bank59552 +59553 POINT(30.498901857285343 -95.40158833487949) bank59553 +59554 POINT(30.605952422087014 -96.12340821886515) bank59554 +59555 POINT(29.546578361194552 -95.43392758538432) bank59555 +59556 POINT(29.541014675210263 -94.75571353047276) bank59556 +59557 POINT(30.211307869396975 -96.16239471948808) bank59557 +59558 POINT(29.99563581418997 -96.24495257392334) bank59558 +59559 POINT(28.826097164973536 -95.28508117426429) bank59559 +59560 POINT(30.565238709678216 -96.07971846873619) bank59560 +59561 POINT(30.40513445532049 -94.94901120137929) bank59561 +59562 POINT(29.81667434046113 -94.38569946615233) bank59562 +59563 POINT(30.67965494401685 -94.53136956828558) bank59563 +59564 POINT(28.949785886671535 -96.29127238125952) bank59564 +59565 POINT(30.202207260690155 -95.82308210439041) bank59565 +59566 POINT(29.34685399037856 -95.26855062228273) bank59566 +59567 POINT(29.774409004151323 -95.926565828195) bank59567 +59568 POINT(30.24055662626368 -94.9421362176743) bank59568 +59569 POINT(30.1615147376266 -94.93685115907999) bank59569 +59570 POINT(29.911185551222744 -95.23771000880674) bank59570 +59571 POINT(29.389919958718433 -94.46078709184759) bank59571 +59572 POINT(29.476795283365014 -95.22333132487606) bank59572 +59573 POINT(29.56000777728197 -95.63809234280946) bank59573 +59574 POINT(30.531581041668808 -96.05593713215247) bank59574 +59575 POINT(29.79656355438736 -96.2691841596498) bank59575 +59576 POINT(30.442707516342498 -95.85021488937056) bank59576 +59577 POINT(30.55594934328216 -94.37788366239201) bank59577 +59578 POINT(29.56877864657412 -95.1958680138376) bank59578 +59579 POINT(29.41809148516817 -95.15055340945882) bank59579 +59580 POINT(30.739325209334552 -95.7466726231969) bank59580 +59581 POINT(29.666215992468956 -96.0815236436894) bank59581 +59582 POINT(30.24183446315864 -95.51104347618036) bank59582 +59583 POINT(29.40874823473569 -94.78979453171229) bank59583 +59584 POINT(29.308759812287438 -94.49322244477789) bank59584 +59585 POINT(28.781583266082702 -95.08651767441597) bank59585 +59586 POINT(30.156977950364784 -94.8014529518427) bank59586 +59587 POINT(28.969175634364074 -96.19497088913023) bank59587 +59588 POINT(30.72054104643815 -95.32968214420498) bank59588 +59589 POINT(28.810619944042394 -94.77986771007795) bank59589 +59590 POINT(30.648498349983115 -94.99724327360757) bank59590 +59591 POINT(28.813626485676547 -96.12237007609414) bank59591 +59592 POINT(28.88007330203165 -95.7747427957317) bank59592 +59593 POINT(29.373264602114382 -95.85271588131545) bank59593 +59594 POINT(30.710553713812953 -95.87987735331089) bank59594 +59595 POINT(30.442145305327664 -94.95135247469088) bank59595 +59596 POINT(30.07259773319389 -95.5024342088117) bank59596 +59597 POINT(29.240661836586 -95.49666494575037) bank59597 +59598 POINT(29.257515100272933 -96.04716425661006) bank59598 +59599 POINT(30.283979017190404 -94.65553341130155) bank59599 +59600 POINT(29.868533244234843 -95.83326774240078) bank59600 +59601 POINT(30.539795840238476 -95.36976785880888) bank59601 +59602 POINT(29.965119513101634 -96.04209510292839) bank59602 +59603 POINT(30.305764424271022 -94.80196357282942) bank59603 +59604 POINT(28.83900876600388 -94.75250546529163) bank59604 +59605 POINT(29.69054187851711 -95.25770415604626) bank59605 +59606 POINT(30.09657887787669 -95.21694432025588) bank59606 +59607 POINT(29.356246744481165 -96.05521005368364) bank59607 +59608 POINT(30.45901570287169 -96.18100578150523) bank59608 +59609 POINT(29.25516216800022 -95.17682680562032) bank59609 +59610 POINT(30.628903748991416 -94.87897290036366) bank59610 +59611 POINT(30.144063632272733 -95.78617672200339) bank59611 +59612 POINT(29.894769681773038 -96.18961389075886) bank59612 +59613 POINT(29.5435746845515 -96.09177021476808) bank59613 +59614 POINT(29.428301912958002 -95.68749459827733) bank59614 +59615 POINT(30.150802922367575 -95.6461934429318) bank59615 +59616 POINT(29.132779585698525 -94.79576453018186) bank59616 +59617 POINT(28.810109943411355 -94.72953250011054) bank59617 +59618 POINT(28.782360756165964 -95.99451564909383) bank59618 +59619 POINT(29.411228618289815 -95.2844965647565) bank59619 +59620 POINT(29.776700778402063 -95.22331794683832) bank59620 +59621 POINT(29.071890258448605 -94.83918204287166) bank59621 +59622 POINT(28.96572401503098 -95.13794499152205) bank59622 +59623 POINT(28.850008529337632 -95.13970981357497) bank59623 +59624 POINT(30.00876497210028 -95.75242396786079) bank59624 +59625 POINT(29.711924619175267 -96.31240112768526) bank59625 +59626 POINT(29.437693686754283 -94.91694625821727) bank59626 +59627 POINT(30.742624895694117 -94.58552830147961) bank59627 +59628 POINT(30.614975029376197 -95.46072323266863) bank59628 +59629 POINT(29.22990693905113 -95.20303228806767) bank59629 +59630 POINT(30.502851548055574 -94.46906471315243) bank59630 +59631 POINT(30.139340728584127 -95.21794706936555) bank59631 +59632 POINT(29.08436380181647 -95.27280939849432) bank59632 +59633 POINT(29.490271295938438 -95.91405948820551) bank59633 +59634 POINT(29.65098820884766 -96.34754107230725) bank59634 +59635 POINT(29.794057595014344 -95.02754615785794) bank59635 +59636 POINT(30.288741620249628 -94.93812361798511) bank59636 +59637 POINT(29.716456227419478 -95.09539828760798) bank59637 +59638 POINT(29.772520616083234 -95.88354903359809) bank59638 +59639 POINT(30.0479795740001 -95.99055964130909) bank59639 +59640 POINT(29.80186586399285 -96.27105572092334) bank59640 +59641 POINT(29.2460951758623 -94.43616175448287) bank59641 +59642 POINT(29.19668413554088 -96.08516906989065) bank59642 +59643 POINT(29.656295525332514 -94.56234963181642) bank59643 +59644 POINT(29.295329021676878 -95.35207155923376) bank59644 +59645 POINT(29.21109964122396 -95.49038613503708) bank59645 +59646 POINT(28.968682774155013 -94.6877816304299) bank59646 +59647 POINT(30.322282483178213 -94.38093303839203) bank59647 +59648 POINT(28.978848795030892 -94.4999735639973) bank59648 +59649 POINT(29.444750797813814 -95.62849055137175) bank59649 +59650 POINT(28.82321219245921 -94.50934933974577) bank59650 +59651 POINT(29.18974284198478 -95.11744162778807) bank59651 +59652 POINT(29.225494800751214 -95.94244686295166) bank59652 +59653 POINT(29.955200751469043 -94.43088245587539) bank59653 +59654 POINT(29.931075291325545 -95.67941317345112) bank59654 +59655 POINT(29.58145112703156 -95.33806293131003) bank59655 +59656 POINT(30.41765153361149 -96.05542941524179) bank59656 +59657 POINT(29.08383481307031 -95.94092791335527) bank59657 +59658 POINT(29.992348060424792 -95.04319114117233) bank59658 +59659 POINT(29.78795478945664 -94.72383604940326) bank59659 +59660 POINT(30.611611647852506 -96.07795106599032) bank59660 +59661 POINT(30.046514655113018 -95.28451726092948) bank59661 +59662 POINT(29.305362066517354 -95.10495716570492) bank59662 +59663 POINT(30.426778792153645 -96.3596201405859) bank59663 +59664 POINT(29.501875937917532 -96.17671877316916) bank59664 +59665 POINT(30.28594686070755 -95.10079435933086) bank59665 +59666 POINT(29.684119350248316 -94.56979703210003) bank59666 +59667 POINT(29.99765095386075 -94.86066419082228) bank59667 +59668 POINT(29.529408172100503 -94.67004559351369) bank59668 +59669 POINT(30.66101085733544 -94.87526806770836) bank59669 +59670 POINT(30.456372338528478 -96.19498655452522) bank59670 +59671 POINT(29.378014794382302 -96.36433325838871) bank59671 +59672 POINT(29.692983563425173 -95.63798705072843) bank59672 +59673 POINT(29.54698417898015 -96.15474113202555) bank59673 +59674 POINT(30.183664870047096 -95.12286629011643) bank59674 +59675 POINT(29.38375222746721 -96.19110919817706) bank59675 +59676 POINT(29.958942527855417 -94.72794381846391) bank59676 +59677 POINT(30.416335879773143 -95.13430924127594) bank59677 +59678 POINT(29.775625231454033 -95.90762644232039) bank59678 +59679 POINT(30.667275613762587 -96.24685410387607) bank59679 +59680 POINT(29.1645732816839 -95.20530612679609) bank59680 +59681 POINT(29.743581304611897 -94.59881463144191) bank59681 +59682 POINT(29.84894170129096 -94.66592907191112) bank59682 +59683 POINT(29.748681520854912 -95.97030049767486) bank59683 +59684 POINT(28.838734639798993 -94.49878691589258) bank59684 +59685 POINT(29.30411198182359 -96.12037007481466) bank59685 +59686 POINT(29.29529520718022 -95.77065982930176) bank59686 +59687 POINT(30.213074858361647 -96.20427546872956) bank59687 +59688 POINT(29.935060118493695 -94.6530997286694) bank59688 +59689 POINT(29.278828039681553 -94.65137127262467) bank59689 +59690 POINT(29.034435368850886 -95.3656034976938) bank59690 +59691 POINT(30.364966645607307 -96.2819888048294) bank59691 +59692 POINT(29.35899953065402 -95.97565576208193) bank59692 +59693 POINT(30.051404727642883 -96.19705848078692) bank59693 +59694 POINT(30.3042954812595 -95.53473436312292) bank59694 +59695 POINT(29.0019304926542 -96.06498689404108) bank59695 +59696 POINT(28.8428710055426 -95.28515819266207) bank59696 +59697 POINT(30.705145584792355 -96.17862943289299) bank59697 +59698 POINT(29.834674743426778 -95.88739216093305) bank59698 +59699 POINT(28.914138534507128 -96.24609071100924) bank59699 +59700 POINT(29.64120333936199 -95.82969466935283) bank59700 +59701 POINT(29.342286317354784 -94.48453390382447) bank59701 +59702 POINT(30.72444276757266 -94.60831835342341) bank59702 +59703 POINT(29.285045753121057 -95.36127401178268) bank59703 +59704 POINT(29.048322695200994 -94.89252200931409) bank59704 +59705 POINT(29.852135306418525 -95.14291663750626) bank59705 +59706 POINT(30.061511419219045 -95.07635124325708) bank59706 +59707 POINT(29.420361937536395 -95.76061290625847) bank59707 +59708 POINT(29.38215373771731 -96.02648468481019) bank59708 +59709 POINT(30.21129761046404 -95.06017047073136) bank59709 +59710 POINT(29.88809829698599 -96.0881012611448) bank59710 +59711 POINT(28.81973138569741 -95.64168333201266) bank59711 +59712 POINT(29.91261784149374 -95.27344011450263) bank59712 +59713 POINT(29.711958268610317 -96.06464540019434) bank59713 +59714 POINT(30.007842404008198 -95.95607643613697) bank59714 +59715 POINT(30.493804901088588 -95.4032357855897) bank59715 +59716 POINT(30.70678957823737 -94.87440306562236) bank59716 +59717 POINT(30.084114932729417 -95.54097894660295) bank59717 +59718 POINT(28.975525307975737 -95.97519153254414) bank59718 +59719 POINT(29.944799565393673 -95.62880892999961) bank59719 +59720 POINT(29.463059718119847 -94.89800058149069) bank59720 +59721 POINT(30.112338995081164 -95.82583785339716) bank59721 +59722 POINT(29.72869740321734 -94.72044609553495) bank59722 +59723 POINT(30.078756203467183 -95.63422834985329) bank59723 +59724 POINT(30.308318555464208 -95.81404364303542) bank59724 +59725 POINT(29.242019500097257 -94.68101541914591) bank59725 +59726 POINT(29.717792798908217 -96.05359901270634) bank59726 +59727 POINT(29.720781514516375 -96.06367967967283) bank59727 +59728 POINT(30.43089876185963 -95.46071525008679) bank59728 +59729 POINT(29.90044073209839 -96.02119926790506) bank59729 +59730 POINT(29.584138946526323 -95.74177235438637) bank59730 +59731 POINT(30.11334399676461 -95.25207315043414) bank59731 +59732 POINT(30.589853795649258 -94.56242841995163) bank59732 +59733 POINT(29.714838229449846 -94.47982913344507) bank59733 +59734 POINT(30.695568835357847 -94.94660675416827) bank59734 +59735 POINT(30.000904425360872 -95.96178693725356) bank59735 +59736 POINT(30.38147825138105 -95.1133266862109) bank59736 +59737 POINT(30.107449633634015 -96.047752891882) bank59737 +59738 POINT(30.40437141937037 -94.74202902848688) bank59738 +59739 POINT(28.965275669877233 -95.43478998883771) bank59739 +59740 POINT(29.599806331959165 -94.6790090998696) bank59740 +59741 POINT(29.646832037509682 -95.56097886112055) bank59741 +59742 POINT(29.69717855812221 -95.64586267185317) bank59742 +59743 POINT(29.449730597538498 -94.4420592368142) bank59743 +59744 POINT(30.136772958079586 -96.28219177583034) bank59744 +59745 POINT(30.58527058085205 -95.89553002377231) bank59745 +59746 POINT(28.793242138035023 -96.29990796300675) bank59746 +59747 POINT(30.412314913744996 -95.05470560801484) bank59747 +59748 POINT(30.329542076069522 -95.87819261690419) bank59748 +59749 POINT(30.106038809145396 -95.93018675372647) bank59749 +59750 POINT(30.367984685558902 -95.87493698643605) bank59750 +59751 POINT(28.928382441771504 -95.4755831776907) bank59751 +59752 POINT(28.85938453876191 -94.62149453879526) bank59752 +59753 POINT(29.588666698941918 -94.7816765597698) bank59753 +59754 POINT(30.568670960709618 -95.98028351117281) bank59754 +59755 POINT(29.589780197187917 -96.00004867843637) bank59755 +59756 POINT(28.977963155071862 -95.66872700966381) bank59756 +59757 POINT(29.847026931753234 -94.46557529343507) bank59757 +59758 POINT(30.4699560960197 -95.05398716049929) bank59758 +59759 POINT(28.94696254545205 -95.78072051592534) bank59759 +59760 POINT(29.3390710148376 -95.4816473788092) bank59760 +59761 POINT(29.35270366501233 -94.43381991391358) bank59761 +59762 POINT(30.682356753160605 -95.04507237314952) bank59762 +59763 POINT(29.677304729415805 -94.42905775617609) bank59763 +59764 POINT(30.63194917724823 -94.88500421466821) bank59764 +59765 POINT(29.981902010456935 -95.82878467468981) bank59765 +59766 POINT(30.493017542622646 -95.00127239879802) bank59766 +59767 POINT(29.00746098817376 -96.11494656693321) bank59767 +59768 POINT(30.584801033527537 -96.18452640973645) bank59768 +59769 POINT(29.11943713064916 -94.67884166264909) bank59769 +59770 POINT(28.86897747703993 -94.94286489394224) bank59770 +59771 POINT(29.825547991500613 -96.24092619037742) bank59771 +59772 POINT(29.014820638322593 -95.94500594262678) bank59772 +59773 POINT(30.469952363451522 -95.40188780568062) bank59773 +59774 POINT(28.82003881975183 -95.32224004707976) bank59774 +59775 POINT(29.188094524613472 -94.84467143574932) bank59775 +59776 POINT(30.55676702283095 -94.87599642138339) bank59776 +59777 POINT(29.29841499609281 -95.48754945314771) bank59777 +59778 POINT(28.88945723479987 -94.38508943729815) bank59778 +59779 POINT(30.151921251193063 -96.36839778845606) bank59779 +59780 POINT(29.403219144179353 -94.7697645519535) bank59780 +59781 POINT(30.26257983238482 -96.21666983025187) bank59781 +59782 POINT(28.98444958700226 -95.15975088618445) bank59782 +59783 POINT(29.648777441523528 -96.01375528903056) bank59783 +59784 POINT(29.218836369271756 -95.55272321925068) bank59784 +59785 POINT(30.684453156393246 -96.3208517842832) bank59785 +59786 POINT(29.428009499423815 -95.91726017107878) bank59786 +59787 POINT(28.974616264540742 -95.66609465950752) bank59787 +59788 POINT(30.114936176163976 -94.99805889677177) bank59788 +59789 POINT(29.476371605783555 -95.89851781376868) bank59789 +59790 POINT(30.610765546060712 -95.06743953791987) bank59790 +59791 POINT(30.756321539944878 -94.79704788280849) bank59791 +59792 POINT(28.95390864010803 -94.83318519958537) bank59792 +59793 POINT(30.15924177352906 -96.122782351755) bank59793 +59794 POINT(30.589445268762645 -95.98852839070256) bank59794 +59795 POINT(30.629982097518894 -95.1553949195156) bank59795 +59796 POINT(30.67751156060559 -95.30511885227833) bank59796 +59797 POINT(29.15056171000671 -95.16550860968577) bank59797 +59798 POINT(30.43424672191834 -95.29125414099349) bank59798 +59799 POINT(30.54341276851247 -94.7927840978347) bank59799 +59800 POINT(29.922895653819033 -94.58274624417085) bank59800 +59801 POINT(29.898420072164896 -94.46953725848117) bank59801 +59802 POINT(30.341054979381585 -95.29604263349466) bank59802 +59803 POINT(29.71605788580608 -94.95286352363607) bank59803 +59804 POINT(30.100350178152663 -95.73752631328615) bank59804 +59805 POINT(30.350378012632444 -95.54011872266487) bank59805 +59806 POINT(30.003313170512662 -95.39030973628675) bank59806 +59807 POINT(28.833951195826604 -94.37974217594781) bank59807 +59808 POINT(29.341695041692642 -95.78853112998618) bank59808 +59809 POINT(30.019446824087822 -96.14235790530292) bank59809 +59810 POINT(28.797317336271433 -95.07103443850625) bank59810 +59811 POINT(29.552797249637074 -94.88530457143244) bank59811 +59812 POINT(29.07803679327264 -95.23216177739701) bank59812 +59813 POINT(30.674660539522634 -95.02093300722224) bank59813 +59814 POINT(29.55574804201963 -94.43571160800145) bank59814 +59815 POINT(30.291770981851087 -95.97026708250145) bank59815 +59816 POINT(30.116732659447187 -96.1091029276308) bank59816 +59817 POINT(29.23323970818222 -95.34502683970028) bank59817 +59818 POINT(29.72187399822616 -95.06665792218591) bank59818 +59819 POINT(29.161723477396148 -94.52413276950938) bank59819 +59820 POINT(30.343606053760787 -95.08120108770176) bank59820 +59821 POINT(29.69642955818787 -94.72412584538131) bank59821 +59822 POINT(29.711166215852312 -94.54211616265636) bank59822 +59823 POINT(28.86084164631483 -94.47478040142508) bank59823 +59824 POINT(30.430439830650805 -94.44650959370081) bank59824 +59825 POINT(30.23568863435713 -95.00204464510796) bank59825 +59826 POINT(30.12028744397661 -94.72985078485785) bank59826 +59827 POINT(29.468683844474047 -95.94737123154958) bank59827 +59828 POINT(29.476337952152427 -94.84987953401404) bank59828 +59829 POINT(30.411691274205904 -95.31532761875033) bank59829 +59830 POINT(29.140518775946333 -95.63791175627773) bank59830 +59831 POINT(30.615812325245702 -95.42025228716739) bank59831 +59832 POINT(29.004459270363746 -95.7164674110876) bank59832 +59833 POINT(30.20061428379971 -94.67076429609348) bank59833 +59834 POINT(29.289211098527765 -95.2963362804906) bank59834 +59835 POINT(29.010076666133887 -94.65285013761478) bank59835 +59836 POINT(30.48559021530785 -95.40917551546661) bank59836 +59837 POINT(29.618563165469745 -96.24635442048543) bank59837 +59838 POINT(30.66011095953282 -94.38512756320011) bank59838 +59839 POINT(29.704110708787788 -95.26829263255905) bank59839 +59840 POINT(30.07059367320033 -94.92857661010702) bank59840 +59841 POINT(28.94121915218007 -95.8701173463054) bank59841 +59842 POINT(29.44197237708333 -95.54422713187617) bank59842 +59843 POINT(29.267363156664914 -96.20514640429235) bank59843 +59844 POINT(29.456624008144818 -94.76609759005157) bank59844 +59845 POINT(30.23669583658673 -94.51199765724884) bank59845 +59846 POINT(30.615704409856413 -95.08444557999557) bank59846 +59847 POINT(29.387435210532917 -96.0778677483344) bank59847 +59848 POINT(30.145242063571573 -94.54844600859691) bank59848 +59849 POINT(30.271076965245555 -95.58118997145044) bank59849 +59850 POINT(30.190986792982713 -96.14510701377021) bank59850 +59851 POINT(29.896718986510894 -94.6589444121043) bank59851 +59852 POINT(30.15710898087053 -95.15832608843145) bank59852 +59853 POINT(28.905152597776613 -94.63441345601176) bank59853 +59854 POINT(29.470899783983448 -95.03730015303239) bank59854 +59855 POINT(30.6101087357008 -95.69308429684247) bank59855 +59856 POINT(30.153203103325232 -96.1285656547767) bank59856 +59857 POINT(30.13837040693269 -96.2788734742411) bank59857 +59858 POINT(30.01046867229725 -95.24995423305266) bank59858 +59859 POINT(29.155670292221053 -95.07802905720231) bank59859 +59860 POINT(29.87269613912028 -96.14905861378386) bank59860 +59861 POINT(29.957901025404993 -95.96113128100691) bank59861 +59862 POINT(29.979376713526587 -95.4509081186562) bank59862 +59863 POINT(29.329382951307768 -94.70390156656934) bank59863 +59864 POINT(29.518900121181524 -94.6021353811121) bank59864 +59865 POINT(30.705227275312147 -95.85189957845589) bank59865 +59866 POINT(29.504789327921895 -94.4426271830211) bank59866 +59867 POINT(30.0958812476774 -95.67004676022665) bank59867 +59868 POINT(30.022043241005022 -94.59433728203308) bank59868 +59869 POINT(29.43535011703663 -95.79580206556662) bank59869 +59870 POINT(30.068023225747268 -94.94079162103972) bank59870 +59871 POINT(29.94772338392373 -94.45714732553378) bank59871 +59872 POINT(30.343899258968456 -95.42667017326758) bank59872 +59873 POINT(29.333565760135674 -95.70558336820532) bank59873 +59874 POINT(29.7039701589459 -94.88634163261821) bank59874 +59875 POINT(29.222470907485622 -94.58239124891756) bank59875 +59876 POINT(28.915133611092134 -94.7712280049783) bank59876 +59877 POINT(30.327250364752693 -96.19913138857923) bank59877 +59878 POINT(30.15380774297597 -95.62179592592233) bank59878 +59879 POINT(29.400326250560685 -94.37597837713564) bank59879 +59880 POINT(29.513550445606977 -95.08405241549623) bank59880 +59881 POINT(29.932376958696626 -95.98718197290711) bank59881 +59882 POINT(30.26909552782322 -95.20065487824316) bank59882 +59883 POINT(30.48810537363215 -95.16276792015329) bank59883 +59884 POINT(30.11456461644657 -95.27101659943462) bank59884 +59885 POINT(30.556267652271014 -95.19259544627911) bank59885 +59886 POINT(30.727004672456026 -95.17853744983806) bank59886 +59887 POINT(30.43969966846505 -95.19983291609907) bank59887 +59888 POINT(29.276854323957288 -95.024573659478) bank59888 +59889 POINT(29.870306503004112 -95.63721781338145) bank59889 +59890 POINT(29.750536820174908 -95.36205469524043) bank59890 +59891 POINT(29.030162811277727 -95.73041926160909) bank59891 +59892 POINT(28.940041560766645 -94.58950620894015) bank59892 +59893 POINT(29.099479966298418 -96.30765627596145) bank59893 +59894 POINT(29.43958053298254 -96.22620410038635) bank59894 +59895 POINT(29.829937910351518 -95.86688211221048) bank59895 +59896 POINT(29.553838534102944 -96.23614458383996) bank59896 +59897 POINT(30.18961465263306 -95.73777974905123) bank59897 +59898 POINT(30.642271899490073 -95.8954906503632) bank59898 +59899 POINT(29.389444424773036 -95.56505620844452) bank59899 +59900 POINT(30.131747146363285 -94.75318537107138) bank59900 +59901 POINT(29.861879121170535 -96.21731700727103) bank59901 +59902 POINT(30.311672188712947 -96.31234614321573) bank59902 +59903 POINT(29.602960574202335 -94.84224210761025) bank59903 +59904 POINT(29.685835724784923 -95.26356028818012) bank59904 +59905 POINT(29.027762996389725 -96.23765596268157) bank59905 +59906 POINT(29.999817484491214 -94.77054969712414) bank59906 +59907 POINT(29.482557115288465 -94.66557146068625) bank59907 +59908 POINT(29.798381348601165 -95.12884372276409) bank59908 +59909 POINT(30.335978841209645 -96.24921611333683) bank59909 +59910 POINT(29.033120352556974 -95.97440246944642) bank59910 +59911 POINT(29.61786189147945 -94.46059741590307) bank59911 +59912 POINT(29.579178702332154 -95.26350688849621) bank59912 +59913 POINT(28.88740918200541 -95.38279749894639) bank59913 +59914 POINT(30.5843557074633 -94.8894914609379) bank59914 +59915 POINT(30.66835792087949 -94.78882569045416) bank59915 +59916 POINT(29.59724429089597 -95.18829250129163) bank59916 +59917 POINT(30.727621681552716 -95.03386686703455) bank59917 +59918 POINT(28.995452491701748 -96.35262139526229) bank59918 +59919 POINT(30.620052305319163 -95.75272232360814) bank59919 +59920 POINT(29.33232098140056 -96.30071043954571) bank59920 +59921 POINT(29.926789132194564 -94.6474153373561) bank59921 +59922 POINT(29.20829479222816 -94.91935830621863) bank59922 +59923 POINT(28.98684410518426 -95.72949838296147) bank59923 +59924 POINT(29.431962532086448 -95.32596427638737) bank59924 +59925 POINT(29.049808045796336 -94.67230391374126) bank59925 +59926 POINT(29.638168813148784 -96.17441223365168) bank59926 +59927 POINT(29.79225863359418 -94.64901979041963) bank59927 +59928 POINT(29.195938092183454 -95.26273478805045) bank59928 +59929 POINT(29.087506979311232 -96.1889286524911) bank59929 +59930 POINT(29.558481956052734 -94.43758074897472) bank59930 +59931 POINT(28.765495677303868 -96.09295788710082) bank59931 +59932 POINT(30.25033357631803 -95.70346654685875) bank59932 +59933 POINT(30.403407212937505 -95.85440040545049) bank59933 +59934 POINT(30.381517475206063 -94.41380224512862) bank59934 +59935 POINT(29.471283704476434 -95.87592015452215) bank59935 +59936 POINT(28.801779958795404 -94.47962406548557) bank59936 +59937 POINT(29.209820553101938 -95.73774018074946) bank59937 +59938 POINT(30.180265885846424 -95.44446660241358) bank59938 +59939 POINT(29.665626822141466 -94.7047858641039) bank59939 +59940 POINT(29.133501456604435 -96.2823844962998) bank59940 +59941 POINT(29.640827303134152 -94.7612029814229) bank59941 +59942 POINT(29.34178739497586 -94.57762166110257) bank59942 +59943 POINT(30.75007817279454 -95.46262382127323) bank59943 +59944 POINT(28.98631717610846 -94.84789761482702) bank59944 +59945 POINT(30.577648287485225 -95.59464074216953) bank59945 +59946 POINT(29.88027662036443 -96.28901574806244) bank59946 +59947 POINT(29.705912774552793 -95.77579439863416) bank59947 +59948 POINT(29.113616195955352 -94.5519635031743) bank59948 +59949 POINT(30.158859553434162 -95.63770895365535) bank59949 +59950 POINT(29.243104003340033 -95.6837082050351) bank59950 +59951 POINT(30.14355726185386 -94.43770900307821) bank59951 +59952 POINT(28.991194897305228 -94.45352790035273) bank59952 +59953 POINT(30.605354676471677 -96.32872057255797) bank59953 +59954 POINT(29.434770552262197 -95.08995905564579) bank59954 +59955 POINT(30.115236810612352 -95.45012419319974) bank59955 +59956 POINT(28.918128357608243 -94.85558127277324) bank59956 +59957 POINT(29.634109647765825 -94.46317195527983) bank59957 +59958 POINT(28.792577108234582 -95.8747645000117) bank59958 +59959 POINT(29.119420109698112 -94.93377059798235) bank59959 +59960 POINT(29.271052317384058 -96.19184940818825) bank59960 +59961 POINT(28.93975354890315 -95.90317223649441) bank59961 +59962 POINT(30.02124866050454 -95.43968189456413) bank59962 +59963 POINT(30.67096947980191 -94.63756964359658) bank59963 +59964 POINT(30.11286734420282 -95.33112157413893) bank59964 +59965 POINT(30.419091529127666 -95.80649833239609) bank59965 +59966 POINT(30.585182582172806 -94.52658609528521) bank59966 +59967 POINT(30.45871421883035 -96.07361609720387) bank59967 +59968 POINT(29.01076588555039 -95.30639905700052) bank59968 +59969 POINT(30.036894141050674 -95.569598597041) bank59969 +59970 POINT(30.148307421880244 -94.59802858935097) bank59970 +59971 POINT(29.195797710792444 -96.26288577863029) bank59971 +59972 POINT(29.530622298305584 -94.8708887129813) bank59972 +59973 POINT(30.16246802812175 -94.3867408518366) bank59973 +59974 POINT(29.512806541315445 -95.15673145314487) bank59974 +59975 POINT(29.258071211960832 -94.52387457104513) bank59975 +59976 POINT(28.98222078798713 -94.49245556443611) bank59976 +59977 POINT(29.216436352435206 -95.66095253639538) bank59977 +59978 POINT(30.296528497521635 -94.64357067034285) bank59978 +59979 POINT(29.72504073228454 -95.01652380699544) bank59979 +59980 POINT(30.105795756683204 -95.67824135408952) bank59980 +59981 POINT(30.72728254276433 -96.28757699753801) bank59981 +59982 POINT(29.73179708463786 -95.92354551237914) bank59982 +59983 POINT(30.269211639096465 -94.83599263296426) bank59983 +59984 POINT(29.78021775573089 -95.15805053569655) bank59984 +59985 POINT(30.45702616503889 -94.66153496086699) bank59985 +59986 POINT(30.300588377649312 -94.95928413236406) bank59986 +59987 POINT(29.984548711485616 -94.71664031332008) bank59987 +59988 POINT(30.4509494731694 -95.9154137020437) bank59988 +59989 POINT(29.93716963806494 -94.64228892131334) bank59989 +59990 POINT(30.111198487157097 -94.80579062760896) bank59990 +59991 POINT(30.555152370554477 -94.38003580861039) bank59991 +59992 POINT(30.283163211764098 -96.06814263483432) bank59992 +59993 POINT(30.126892968869967 -95.35406601372249) bank59993 +59994 POINT(30.608315098503216 -95.13227827362465) bank59994 +59995 POINT(29.61165812598305 -95.5561025823289) bank59995 +59996 POINT(29.944871598558027 -94.81931925078251) bank59996 +59997 POINT(30.756619557228806 -95.60371924939986) bank59997 +59998 POINT(30.411396267919425 -96.02132548142426) bank59998 +59999 POINT(30.330417958096792 -94.39851181041513) bank59999 +60000 POINT(29.53716723564692 -95.71988814498555) bank60000 +60001 POINT(28.77619607752921 -95.62575057020197) bank60001 +60002 POINT(30.5420165954378 -94.46420036211858) bank60002 +60003 POINT(29.942007034240227 -95.79895508389417) bank60003 +60004 POINT(29.115690202055017 -94.88011905279893) bank60004 +60005 POINT(28.98112563963666 -95.34091353270726) bank60005 +60006 POINT(30.112336091156724 -94.61021477490694) bank60006 +60007 POINT(29.58929043181186 -96.11950575853923) bank60007 +60008 POINT(29.46731308174953 -95.5313323120295) bank60008 +60009 POINT(29.262325099623396 -94.65887909185597) bank60009 +60010 POINT(29.861584100122222 -96.07517961798813) bank60010 +60011 POINT(30.15907537625334 -95.66860461135181) bank60011 +60012 POINT(29.049667927251136 -95.82483852577563) bank60012 +60013 POINT(30.178334884828203 -95.7232007095385) bank60013 +60014 POINT(29.824676210339987 -95.3471192994435) bank60014 +60015 POINT(30.042999375498532 -94.89237890669916) bank60015 +60016 POINT(28.948524667521355 -95.16017968440752) bank60016 +60017 POINT(29.481006135297267 -95.08167107366442) bank60017 +60018 POINT(29.46158062014073 -95.37651330216922) bank60018 +60019 POINT(29.470123078670486 -95.93820084577989) bank60019 +60020 POINT(29.201056393907997 -96.24339936913326) bank60020 +60021 POINT(29.73867841093873 -94.59938080356777) bank60021 +60022 POINT(29.945917659470975 -94.86016903905285) bank60022 +60023 POINT(30.1495932331983 -94.54985210607283) bank60023 +60024 POINT(30.42807596602049 -94.99896444229958) bank60024 +60025 POINT(30.50295651349829 -95.93365910053672) bank60025 +60026 POINT(30.68571577474554 -95.70766581278339) bank60026 +60027 POINT(30.258932895387282 -95.67043288440853) bank60027 +60028 POINT(29.679047380830962 -94.756742333002) bank60028 +60029 POINT(29.540612496911077 -96.04373756263807) bank60029 +60030 POINT(29.890476179338343 -95.6935194028445) bank60030 +60031 POINT(29.857580658341785 -95.98785434002535) bank60031 +60032 POINT(30.72705803194233 -96.28726533694203) bank60032 +60033 POINT(30.54985137734867 -96.23222106991966) bank60033 +60034 POINT(28.821235956426783 -94.72664769951795) bank60034 +60035 POINT(29.336392663750125 -96.02159148787989) bank60035 +60036 POINT(29.379567939291366 -96.26683734793721) bank60036 +60037 POINT(29.976571779494765 -96.24056038000485) bank60037 +60038 POINT(30.306479119441974 -94.90773044835792) bank60038 +60039 POINT(30.58044239129883 -94.53073729725443) bank60039 +60040 POINT(30.11065606170105 -95.65176922879353) bank60040 +60041 POINT(30.61374555312378 -95.26944752949603) bank60041 +60042 POINT(30.11389480376435 -94.62059040623383) bank60042 +60043 POINT(29.001296537037863 -94.51905816622107) bank60043 +60044 POINT(29.139701000024015 -96.29997736351274) bank60044 +60045 POINT(29.76999038154754 -94.48343901974751) bank60045 +60046 POINT(29.217771054674696 -95.4546207190015) bank60046 +60047 POINT(28.833158957851676 -96.22264482690147) bank60047 +60048 POINT(30.619177181547528 -94.42416686649237) bank60048 +60049 POINT(30.1361380312326 -96.09413370445165) bank60049 +60050 POINT(29.403688237489188 -95.3042714171703) bank60050 +60051 POINT(30.018150068362765 -94.4936769472502) bank60051 +60052 POINT(29.493077805423418 -95.34529702151818) bank60052 +60053 POINT(30.100960637111143 -95.55086493531083) bank60053 +60054 POINT(30.330674022007234 -96.28561250156969) bank60054 +60055 POINT(30.18432843525211 -95.84602454225336) bank60055 +60056 POINT(29.651193371030622 -94.53881846455677) bank60056 +60057 POINT(30.728823586235496 -94.9381751207294) bank60057 +60058 POINT(30.65148763637556 -95.31790525573673) bank60058 +60059 POINT(30.61265098257366 -95.53038337637643) bank60059 +60060 POINT(30.332045506818865 -95.25562476808071) bank60060 +60061 POINT(28.976781478209762 -94.49873353692625) bank60061 +60062 POINT(29.462391286324383 -96.15633559631888) bank60062 +60063 POINT(29.583065471806812 -96.20690953983679) bank60063 +60064 POINT(29.146762456271293 -95.37926589794824) bank60064 +60065 POINT(29.734933763509993 -94.96610444386522) bank60065 +60066 POINT(30.290424820225525 -95.99182964573366) bank60066 +60067 POINT(29.17505040007911 -95.56140959996378) bank60067 +60068 POINT(29.405618065140114 -95.16177241984238) bank60068 +60069 POINT(29.144427137951823 -94.93228833791369) bank60069 +60070 POINT(28.903037916973204 -95.23889202669453) bank60070 +60071 POINT(29.259283486586902 -94.72263988232052) bank60071 +60072 POINT(29.229474910422745 -94.72796704406902) bank60072 +60073 POINT(29.67928947788732 -94.72092454833319) bank60073 +60074 POINT(28.855573791274164 -96.02729316031146) bank60074 +60075 POINT(29.6810130569988 -94.839020833836) bank60075 +60076 POINT(29.107474801081867 -94.70756057275591) bank60076 +60077 POINT(30.296770467218444 -94.92496412094512) bank60077 +60078 POINT(30.254336249851857 -95.49841368751926) bank60078 +60079 POINT(29.179187632284055 -95.65005970727967) bank60079 +60080 POINT(30.712050984894574 -95.99521691317946) bank60080 +60081 POINT(30.595963644199077 -96.3395080992558) bank60081 +60082 POINT(29.081744096478953 -94.80462923086058) bank60082 +60083 POINT(30.502508286663314 -94.73037065638073) bank60083 +60084 POINT(28.97137269408379 -96.23417465941056) bank60084 +60085 POINT(30.389581123456306 -95.59583089051388) bank60085 +60086 POINT(30.211102237513263 -95.85400128577238) bank60086 +60087 POINT(29.50443463255291 -96.14008319919463) bank60087 +60088 POINT(29.692346236338484 -95.67573797861438) bank60088 +60089 POINT(29.721058365337235 -96.24638100944237) bank60089 +60090 POINT(30.394185968563384 -95.92225582081524) bank60090 +60091 POINT(29.278450221372044 -95.06167345208922) bank60091 +60092 POINT(29.369555365034596 -95.86663865911868) bank60092 +60093 POINT(29.132071439793684 -96.33057817848872) bank60093 +60094 POINT(29.701159025632265 -96.03208505283497) bank60094 +60095 POINT(29.174508143819736 -94.92447181618132) bank60095 +60096 POINT(30.58464358791435 -95.00143937924916) bank60096 +60097 POINT(29.37782704668976 -95.19010089492716) bank60097 +60098 POINT(30.085937737318527 -96.36887022725281) bank60098 +60099 POINT(28.821919272464473 -95.6371946117319) bank60099 +60100 POINT(30.092821155456317 -94.96727143049807) bank60100 +60101 POINT(28.87043772683208 -94.76952424667968) bank60101 +60102 POINT(29.46315616507055 -94.73028421201431) bank60102 +60103 POINT(30.71894748162381 -94.89787533768249) bank60103 +60104 POINT(28.986931632220994 -94.45864602694618) bank60104 +60105 POINT(28.83149736167474 -96.25830165475514) bank60105 +60106 POINT(29.231942960933466 -94.753028113611) bank60106 +60107 POINT(29.44531422741483 -94.77041899190607) bank60107 +60108 POINT(30.53731208273331 -95.72055702282847) bank60108 +60109 POINT(30.202096488946 -95.09684625217217) bank60109 +60110 POINT(30.232540700850443 -96.00955226291602) bank60110 +60111 POINT(29.005077167402092 -94.50047735840594) bank60111 +60112 POINT(29.702021738971517 -96.22087892626425) bank60112 +60113 POINT(29.873096110494306 -96.29789382659251) bank60113 +60114 POINT(29.188779693057565 -94.62230743563346) bank60114 +60115 POINT(29.221468852230103 -94.66176920119396) bank60115 +60116 POINT(29.799887328063047 -94.85142255522872) bank60116 +60117 POINT(30.0267802133977 -96.27619209544775) bank60117 +60118 POINT(30.189560167686768 -95.71763214346986) bank60118 +60119 POINT(29.264747261695998 -95.95618174669234) bank60119 +60120 POINT(29.764245910555882 -94.64594934327506) bank60120 +60121 POINT(30.40827232781127 -94.38568758335605) bank60121 +60122 POINT(29.78050077637521 -95.10388731930476) bank60122 +60123 POINT(29.284574725446205 -96.0734082497695) bank60123 +60124 POINT(30.49300261290007 -96.2543304149585) bank60124 +60125 POINT(30.566255785915164 -96.01288907261487) bank60125 +60126 POINT(29.028674093305668 -96.22744507763932) bank60126 +60127 POINT(29.36437054638296 -95.04543700237062) bank60127 +60128 POINT(29.91352228887388 -94.64278261388924) bank60128 +60129 POINT(29.298734727584975 -94.81825192580007) bank60129 +60130 POINT(30.07798308931892 -95.94531017834314) bank60130 +60131 POINT(30.343446780178724 -95.08473681501916) bank60131 +60132 POINT(28.962537911592907 -95.57213028475255) bank60132 +60133 POINT(30.399975021180907 -95.99500342897079) bank60133 +60134 POINT(29.843495919842614 -94.71916336811778) bank60134 +60135 POINT(29.412795878425587 -94.70405247121616) bank60135 +60136 POINT(30.602914554230786 -95.53590081083313) bank60136 +60137 POINT(29.895119733555745 -95.16989286521016) bank60137 +60138 POINT(29.563538796425714 -95.76891770577262) bank60138 +60139 POINT(29.839014008940392 -95.66174795784003) bank60139 +60140 POINT(29.524277356193018 -94.79837305771734) bank60140 +60141 POINT(30.02164576880973 -94.46897494856876) bank60141 +60142 POINT(28.850173862333243 -95.74431840877585) bank60142 +60143 POINT(29.62889394280577 -95.95743401317462) bank60143 +60144 POINT(30.363996830128016 -95.96462133344026) bank60144 +60145 POINT(29.36265873783126 -94.62216722567365) bank60145 +60146 POINT(29.753204187477117 -95.92958961859325) bank60146 +60147 POINT(29.958873535733545 -94.86657043410496) bank60147 +60148 POINT(29.678842615941363 -94.67515691185802) bank60148 +60149 POINT(29.031239313286516 -94.56692960343105) bank60149 +60150 POINT(29.65028708752623 -95.31882204856146) bank60150 +60151 POINT(30.05572964532011 -95.51319086595322) bank60151 +60152 POINT(28.972135071921215 -95.92775137162684) bank60152 +60153 POINT(30.16977222866192 -94.67572836639813) bank60153 +60154 POINT(29.523200384902605 -94.78993596955519) bank60154 +60155 POINT(29.296828798589704 -95.54470156770536) bank60155 +60156 POINT(30.281788068540227 -95.71298012522371) bank60156 +60157 POINT(29.70920309662332 -95.0287758984884) bank60157 +60158 POINT(29.76028352560683 -95.07710949967628) bank60158 +60159 POINT(30.44337736496715 -96.16297473025001) bank60159 +60160 POINT(29.2371991376163 -95.50299259225206) bank60160 +60161 POINT(29.01532774396472 -95.42681180982662) bank60161 +60162 POINT(30.331855651190587 -95.81737904465768) bank60162 +60163 POINT(29.85379549186112 -95.52732165573842) bank60163 +60164 POINT(29.69163534369581 -96.20586608355951) bank60164 +60165 POINT(29.101850630723742 -95.433381138954) bank60165 +60166 POINT(28.955134696068903 -95.93932436923218) bank60166 +60167 POINT(29.636581418982388 -94.50219577719676) bank60167 +60168 POINT(29.240274039243012 -94.48034019930321) bank60168 +60169 POINT(30.376518663211275 -94.40167859727167) bank60169 +60170 POINT(29.66776062078662 -94.99035836346734) bank60170 +60171 POINT(30.13772701911372 -96.22585085502662) bank60171 +60172 POINT(29.59917881039431 -94.6047704341532) bank60172 +60173 POINT(30.45294904010749 -95.83903007922875) bank60173 +60174 POINT(30.694667053728484 -95.6202591087287) bank60174 +60175 POINT(30.258096738295908 -95.93843417602454) bank60175 +60176 POINT(30.089756335602594 -94.75589535835567) bank60176 +60177 POINT(30.111277868412124 -96.05682550551654) bank60177 +60178 POINT(28.88736727983718 -94.47447111131936) bank60178 +60179 POINT(30.5044087178213 -94.84002290507338) bank60179 +60180 POINT(28.970576816752523 -95.64707741839504) bank60180 +60181 POINT(30.582828416421595 -95.84707194540361) bank60181 +60182 POINT(30.0767637607453 -94.47209574311503) bank60182 +60183 POINT(30.704525474225534 -95.01956912683491) bank60183 +60184 POINT(29.289320666908893 -95.75883255322947) bank60184 +60185 POINT(29.5216612973413 -95.0264621239519) bank60185 +60186 POINT(30.44370187240409 -94.99816940603709) bank60186 +60187 POINT(29.291279321616415 -94.40377765651726) bank60187 +60188 POINT(29.2062319179812 -94.37623305826871) bank60188 +60189 POINT(29.40274878663737 -95.97685783206947) bank60189 +60190 POINT(30.142659708433648 -95.72324290347231) bank60190 +60191 POINT(30.68796769492634 -94.79219218016566) bank60191 +60192 POINT(29.044986876011247 -95.57027560683164) bank60192 +60193 POINT(30.363546808774792 -95.21262480659964) bank60193 +60194 POINT(30.27237920532468 -95.64129630735353) bank60194 +60195 POINT(29.386441149893955 -94.3925822430547) bank60195 +60196 POINT(30.637573260596636 -94.3804028134311) bank60196 +60197 POINT(30.629829071812097 -96.29761572662832) bank60197 +60198 POINT(29.647689068949465 -95.00169408918588) bank60198 +60199 POINT(30.046329718668435 -95.03004611553665) bank60199 +60200 POINT(29.295402232803045 -95.5313520139521) bank60200 +60201 POINT(30.42220314359876 -95.20224026858969) bank60201 +60202 POINT(30.336544087450015 -95.08112289824963) bank60202 +60203 POINT(30.055078848841926 -96.08695437089291) bank60203 +60204 POINT(29.441202681322096 -95.02987510866443) bank60204 +60205 POINT(29.89178312386501 -95.69169626265526) bank60205 +60206 POINT(29.73594284665159 -96.07868955654523) bank60206 +60207 POINT(29.472285061233944 -94.77413847451463) bank60207 +60208 POINT(30.3953304303925 -95.11203651065816) bank60208 +60209 POINT(29.343756824855344 -95.4504060751498) bank60209 +60210 POINT(30.57465923614066 -95.2887425507418) bank60210 +60211 POINT(30.282678367145525 -94.55339421374192) bank60211 +60212 POINT(30.366931383215768 -95.64409121998669) bank60212 +60213 POINT(29.633459092474215 -94.78361034942019) bank60213 +60214 POINT(29.575656325889096 -94.63900092138773) bank60214 +60215 POINT(29.224174122247867 -94.65658146366957) bank60215 +60216 POINT(29.794523805997745 -95.42640433108906) bank60216 +60217 POINT(30.157634799657508 -95.84918743317638) bank60217 +60218 POINT(29.481095894839488 -96.10774479508886) bank60218 +60219 POINT(29.721353690531355 -95.78207937107852) bank60219 +60220 POINT(29.458517476912377 -96.24913547941941) bank60220 +60221 POINT(30.693035412143843 -95.70822151137641) bank60221 +60222 POINT(30.65217709384096 -96.29605528883678) bank60222 +60223 POINT(29.652267198071154 -95.97679257408087) bank60223 +60224 POINT(30.017334660834184 -94.84233232651242) bank60224 +60225 POINT(29.612626339628626 -94.42560217894103) bank60225 +60226 POINT(29.153609790874615 -94.86972557050984) bank60226 +60227 POINT(29.21698027308978 -95.40133632516707) bank60227 +60228 POINT(30.222676593008817 -95.6454992171658) bank60228 +60229 POINT(29.94977320203555 -95.5479073215714) bank60229 +60230 POINT(28.84947983488791 -94.91286831563876) bank60230 +60231 POINT(29.107952400773193 -95.1144594056993) bank60231 +60232 POINT(29.680859182348005 -96.19000689873285) bank60232 +60233 POINT(29.99687393151421 -95.0548076743783) bank60233 +60234 POINT(29.640596904304246 -95.22684919555287) bank60234 +60235 POINT(29.981697096730056 -94.40174160205042) bank60235 +60236 POINT(29.16096029719725 -95.9710686836956) bank60236 +60237 POINT(29.472599487336947 -94.37799128505482) bank60237 +60238 POINT(29.988085767952413 -95.41680456523001) bank60238 +60239 POINT(29.7214666054731 -94.77633814362623) bank60239 +60240 POINT(30.608315924495916 -94.55254878888847) bank60240 +60241 POINT(28.835698653145588 -96.20758153426726) bank60241 +60242 POINT(30.504176268483278 -94.60179506326423) bank60242 +60243 POINT(30.530222285159603 -94.90615077374244) bank60243 +60244 POINT(29.54551664239949 -94.71010126235065) bank60244 +60245 POINT(30.398647753352975 -94.56189586268614) bank60245 +60246 POINT(29.265489965724363 -94.64191551652266) bank60246 +60247 POINT(29.96965127411183 -96.1781891416116) bank60247 +60248 POINT(30.530893710741513 -95.24825095369366) bank60248 +60249 POINT(30.45608603203565 -95.07413568052151) bank60249 +60250 POINT(30.64313939276004 -94.72478179799775) bank60250 +60251 POINT(29.82447171770949 -95.32308328343485) bank60251 +60252 POINT(29.717833244304387 -95.2640737851882) bank60252 +60253 POINT(28.866251461041184 -95.75359498542893) bank60253 +60254 POINT(29.385040747885462 -95.06901114414268) bank60254 +60255 POINT(29.692699899615576 -95.35409332446899) bank60255 +60256 POINT(29.858752489110984 -95.50683921200367) bank60256 +60257 POINT(30.363148451985086 -95.33276154697066) bank60257 +60258 POINT(29.02765611919534 -94.56431602593896) bank60258 +60259 POINT(28.779299137395217 -95.17720076848076) bank60259 +60260 POINT(29.218665536318788 -95.36366070785563) bank60260 +60261 POINT(28.983877407449835 -95.87626653564213) bank60261 +60262 POINT(28.850375246111604 -95.7902933442312) bank60262 +60263 POINT(30.741169338215876 -94.61073419517044) bank60263 +60264 POINT(30.58508164685422 -94.96553988497877) bank60264 +60265 POINT(30.27406264670168 -94.51924702032922) bank60265 +60266 POINT(30.14486120864884 -96.00747387849569) bank60266 +60267 POINT(29.146794104015264 -96.16699875319507) bank60267 +60268 POINT(30.72491676596781 -95.48326953217673) bank60268 +60269 POINT(29.182092867546007 -94.37361436518749) bank60269 +60270 POINT(29.720304528995186 -95.50327184691182) bank60270 +60271 POINT(29.385755524659487 -95.40595604641803) bank60271 +60272 POINT(29.791504339277555 -94.42971298538973) bank60272 +60273 POINT(29.251025149033467 -95.04490410006919) bank60273 +60274 POINT(29.168307830463633 -96.30223349410855) bank60274 +60275 POINT(29.61869656950304 -94.83715128572322) bank60275 +60276 POINT(29.106566814483877 -96.00092432180915) bank60276 +60277 POINT(30.6579291929452 -95.56682265291842) bank60277 +60278 POINT(30.364853606318576 -94.97979734377736) bank60278 +60279 POINT(29.30084695843733 -94.7887886481336) bank60279 +60280 POINT(29.469846299704013 -95.73802799233772) bank60280 +60281 POINT(30.385226184036316 -95.91187662989219) bank60281 +60282 POINT(29.263723571754998 -95.91433861422176) bank60282 +60283 POINT(30.19557924278699 -96.20113631919301) bank60283 +60284 POINT(29.61955022582849 -95.6663368102871) bank60284 +60285 POINT(30.179932550686193 -94.43462547865812) bank60285 +60286 POINT(29.05363266123023 -94.58852512969169) bank60286 +60287 POINT(30.360722978889815 -96.11602783816974) bank60287 +60288 POINT(30.32301877049945 -95.05127107871022) bank60288 +60289 POINT(29.938811717767205 -94.75845792014967) bank60289 +60290 POINT(30.733719902140017 -95.8933522299981) bank60290 +60291 POINT(30.261382121495657 -96.12727568339406) bank60291 +60292 POINT(28.978240820188415 -96.04344728792269) bank60292 +60293 POINT(28.889733591863966 -95.90231888142458) bank60293 +60294 POINT(29.817059800973233 -96.10068616842668) bank60294 +60295 POINT(30.390499464011253 -94.6247665260765) bank60295 +60296 POINT(30.715075391093993 -95.26272822226228) bank60296 +60297 POINT(29.224180952866384 -95.83770911986726) bank60297 +60298 POINT(30.22669173208744 -95.9424800062276) bank60298 +60299 POINT(30.448663185730815 -96.28364192778538) bank60299 +60300 POINT(29.226325424310417 -94.62425652290108) bank60300 +60301 POINT(30.096398711817017 -95.83864273778366) bank60301 +60302 POINT(28.86813687752936 -94.54603138105892) bank60302 +60303 POINT(29.431971053498177 -95.64333799827864) bank60303 +60304 POINT(30.423034670634067 -95.8624945031791) bank60304 +60305 POINT(30.67822282423016 -95.96099916837375) bank60305 +60306 POINT(30.73250337576255 -95.74134267082553) bank60306 +60307 POINT(30.060626690294903 -94.56785221290393) bank60307 +60308 POINT(30.59905845518256 -96.18742635238151) bank60308 +60309 POINT(30.69372894977359 -94.82468556434713) bank60309 +60310 POINT(29.45430127322039 -94.44221942156284) bank60310 +60311 POINT(29.142747831641756 -94.50466176547101) bank60311 +60312 POINT(30.364649787268743 -96.2136074612819) bank60312 +60313 POINT(29.3520883187321 -95.32060499289894) bank60313 +60314 POINT(30.3621004806834 -96.12169349562485) bank60314 +60315 POINT(29.799321162256568 -96.13832566357236) bank60315 +60316 POINT(30.127568665335414 -96.07200107078624) bank60316 +60317 POINT(30.257646569004397 -95.53762991506056) bank60317 +60318 POINT(29.709216236007723 -95.18912860501743) bank60318 +60319 POINT(30.64289356006225 -95.88457433766881) bank60319 +60320 POINT(30.297487021171847 -94.8284577457053) bank60320 +60321 POINT(29.2580531334223 -94.98878594720162) bank60321 +60322 POINT(29.030025938692347 -95.25294361175669) bank60322 +60323 POINT(29.746332127896448 -95.25931029172226) bank60323 +60324 POINT(29.94300279202082 -94.56771585827349) bank60324 +60325 POINT(29.72602901435223 -95.2825488823255) bank60325 +60326 POINT(29.925307812398362 -96.15040430483837) bank60326 +60327 POINT(28.94435778335271 -95.93692075714404) bank60327 +60328 POINT(30.67622929567934 -94.53395508020341) bank60328 +60329 POINT(29.69446286269075 -95.57602016572598) bank60329 +60330 POINT(29.4404885959967 -95.41635595842126) bank60330 +60331 POINT(29.267053697421783 -96.34057758849242) bank60331 +60332 POINT(29.793956497484114 -94.7920577306356) bank60332 +60333 POINT(29.97276475497985 -94.60780616167754) bank60333 +60334 POINT(28.947314016885432 -94.45125629174585) bank60334 +60335 POINT(30.020850458869088 -95.3790323547576) bank60335 +60336 POINT(30.50066959348774 -96.07760461457246) bank60336 +60337 POINT(28.902696928070373 -95.02987154344002) bank60337 +60338 POINT(29.38315006691389 -95.3123189948414) bank60338 +60339 POINT(30.733652211684998 -94.62696906447114) bank60339 +60340 POINT(30.42273938539369 -94.81343985842707) bank60340 +60341 POINT(28.975412454162694 -95.24628945433608) bank60341 +60342 POINT(30.170656106692693 -94.626119098799) bank60342 +60343 POINT(30.412616973580906 -95.83257224311764) bank60343 +60344 POINT(30.13140176760438 -95.49419202241128) bank60344 +60345 POINT(29.846253615251836 -94.81252921366882) bank60345 +60346 POINT(28.781979723429682 -95.02377518480931) bank60346 +60347 POINT(29.863404384605357 -95.54371533007922) bank60347 +60348 POINT(29.518561041076364 -95.29169538830335) bank60348 +60349 POINT(30.188813036938498 -95.92777023079654) bank60349 +60350 POINT(30.138987592356198 -95.363508944435) bank60350 +60351 POINT(29.251844513961903 -95.95849232736843) bank60351 +60352 POINT(28.826762000905454 -94.56393539013352) bank60352 +60353 POINT(29.518240818163065 -95.12906781054672) bank60353 +60354 POINT(29.779999420773173 -96.25736514112506) bank60354 +60355 POINT(30.468848813875805 -95.388939836452) bank60355 +60356 POINT(30.75256602682181 -95.09821493780797) bank60356 +60357 POINT(29.22649993283827 -96.30454781384049) bank60357 +60358 POINT(30.406019927021184 -96.08978201033496) bank60358 +60359 POINT(29.124337010228427 -96.30754925385628) bank60359 +60360 POINT(29.33921470752889 -94.65510203533225) bank60360 +60361 POINT(29.668930086245478 -95.07027745204118) bank60361 +60362 POINT(29.816983671839235 -94.5864038282022) bank60362 +60363 POINT(30.267637732995667 -95.89673994399722) bank60363 +60364 POINT(28.85678173235861 -95.40843539127609) bank60364 +60365 POINT(29.04401743058274 -95.96399173311062) bank60365 +60366 POINT(29.939678695201188 -94.60802180973258) bank60366 +60367 POINT(29.806379894894388 -94.9530912197595) bank60367 +60368 POINT(30.59168414712618 -95.05801874607) bank60368 +60369 POINT(28.905833793747693 -94.48335495392443) bank60369 +60370 POINT(30.14937166369747 -95.1249524180891) bank60370 +60371 POINT(30.406927661566225 -95.61019089922024) bank60371 +60372 POINT(30.006627183045595 -95.25765792308961) bank60372 +60373 POINT(30.500184279694043 -95.60812659490088) bank60373 +60374 POINT(30.71931822837122 -95.98604365875853) bank60374 +60375 POINT(30.753861859019246 -95.35001065225427) bank60375 +60376 POINT(29.006109972134098 -94.51044396503218) bank60376 +60377 POINT(29.665517215998946 -94.62918804835185) bank60377 +60378 POINT(30.032159329595316 -94.63676200371894) bank60378 +60379 POINT(29.537599075192286 -95.38229459360582) bank60379 +60380 POINT(30.735093118957945 -94.84506500654518) bank60380 +60381 POINT(29.429167993135753 -94.54579812452086) bank60381 +60382 POINT(29.023902660380728 -94.92092722686363) bank60382 +60383 POINT(30.342912517500785 -95.30599798655882) bank60383 +60384 POINT(30.18397570765634 -94.94140083345165) bank60384 +60385 POINT(28.76727965993763 -94.64693429381462) bank60385 +60386 POINT(29.81955157314048 -95.56606697262468) bank60386 +60387 POINT(28.997649227445052 -95.01439139726928) bank60387 +60388 POINT(29.98549477934545 -95.77726204690772) bank60388 +60389 POINT(29.154830079914227 -95.4032500645097) bank60389 +60390 POINT(30.326236319603662 -94.56036372240199) bank60390 +60391 POINT(30.58940563887131 -96.00155492527554) bank60391 +60392 POINT(30.13602817534115 -95.85811497188173) bank60392 +60393 POINT(29.816396259644385 -96.07187207649194) bank60393 +60394 POINT(29.062661803287448 -96.31172004600523) bank60394 +60395 POINT(30.54459359446254 -94.86490716541151) bank60395 +60396 POINT(29.080975216210526 -95.97942423489287) bank60396 +60397 POINT(30.19433526820769 -96.08607430703448) bank60397 +60398 POINT(29.490889136936918 -95.33785307357707) bank60398 +60399 POINT(30.3591344255968 -95.00515165668776) bank60399 +60400 POINT(28.867721278476335 -95.46199393854532) bank60400 +60401 POINT(30.500944578081345 -94.99756825122242) bank60401 +60402 POINT(29.565451439556313 -96.0632578929328) bank60402 +60403 POINT(29.411163584250442 -96.22578321001373) bank60403 +60404 POINT(29.55354637060299 -95.50810721507153) bank60404 +60405 POINT(30.383582315817748 -94.74815019070331) bank60405 +60406 POINT(30.405184021242547 -96.10810310871324) bank60406 +60407 POINT(30.28763453708059 -95.59350351695095) bank60407 +60408 POINT(28.78492853998844 -94.8776684662279) bank60408 +60409 POINT(28.83044268865193 -95.32827164585836) bank60409 +60410 POINT(29.77252474359461 -95.64274909385105) bank60410 +60411 POINT(30.598679128015377 -94.70730708137236) bank60411 +60412 POINT(29.49276741562104 -95.62319466871486) bank60412 +60413 POINT(30.27369800682459 -95.86915820367122) bank60413 +60414 POINT(29.59650276733498 -95.53315893146508) bank60414 +60415 POINT(30.463143656092832 -94.40188838021619) bank60415 +60416 POINT(30.73142051467653 -95.14572952992093) bank60416 +60417 POINT(28.81536289800372 -96.11961843842303) bank60417 +60418 POINT(30.412190518123495 -96.15155835652446) bank60418 +60419 POINT(30.64010090254694 -96.1295737406243) bank60419 +60420 POINT(29.539425173464558 -95.85855368101066) bank60420 +60421 POINT(29.506573156012124 -95.44330733063993) bank60421 +60422 POINT(30.059481421918086 -96.13517838300741) bank60422 +60423 POINT(29.935897195289225 -95.25147722062069) bank60423 +60424 POINT(29.55782893812353 -95.173212909836) bank60424 +60425 POINT(29.282883374280114 -95.26704421139075) bank60425 +60426 POINT(29.95296969387704 -95.98728057371669) bank60426 +60427 POINT(29.348833015946767 -94.96293388341434) bank60427 +60428 POINT(29.960849563106983 -95.13576983433828) bank60428 +60429 POINT(30.61340753814286 -95.30403931536043) bank60429 +60430 POINT(29.09344630193759 -95.71295751008978) bank60430 +60431 POINT(29.46143534653 -95.95464356618163) bank60431 +60432 POINT(30.10191851856993 -95.14185580905969) bank60432 +60433 POINT(29.999909062961336 -96.27293716029779) bank60433 +60434 POINT(30.180273155962485 -94.37109178232089) bank60434 +60435 POINT(29.33509790083486 -95.04480023754547) bank60435 +60436 POINT(29.3695759255239 -95.9085713736174) bank60436 +60437 POINT(29.319105642130086 -95.18418029594885) bank60437 +60438 POINT(29.55352041908609 -95.11285112768022) bank60438 +60439 POINT(30.0165386269447 -94.94030224786097) bank60439 +60440 POINT(30.336423766256583 -96.1278144035963) bank60440 +60441 POINT(30.52131734156154 -94.60777295183085) bank60441 +60442 POINT(29.662291428571397 -94.7820428256556) bank60442 +60443 POINT(28.860891918268504 -96.3442186308726) bank60443 +60444 POINT(30.626430951140183 -94.5856412777087) bank60444 +60445 POINT(30.5855930179728 -96.07460033844941) bank60445 +60446 POINT(29.97015401531744 -95.76896722515625) bank60446 +60447 POINT(29.112732919700804 -95.47955083728425) bank60447 +60448 POINT(30.722864692334255 -95.4441234173452) bank60448 +60449 POINT(29.15624353657082 -95.45251604467997) bank60449 +60450 POINT(29.639481939428798 -96.12270664405078) bank60450 +60451 POINT(30.624943175270072 -95.05830552127523) bank60451 +60452 POINT(30.332794315028956 -94.74760067132843) bank60452 +60453 POINT(29.21090448869979 -95.17851028915351) bank60453 +60454 POINT(30.721396298808838 -95.59515281660356) bank60454 +60455 POINT(29.210445013520008 -94.58045115029626) bank60455 +60456 POINT(29.272864012890167 -94.55755943876035) bank60456 +60457 POINT(29.49353714940849 -95.61405297993583) bank60457 +60458 POINT(28.76072134207163 -94.92279439680158) bank60458 +60459 POINT(30.617145825255555 -95.61928995925915) bank60459 +60460 POINT(30.349651631664976 -95.41124381559527) bank60460 +60461 POINT(30.21661657372517 -95.76219365338034) bank60461 +60462 POINT(29.770412848272578 -96.36159828148517) bank60462 +60463 POINT(30.739236389864846 -95.29142252777038) bank60463 +60464 POINT(30.426191755702373 -94.4101188445021) bank60464 +60465 POINT(29.59292631762408 -95.70325747553821) bank60465 +60466 POINT(29.894499154189138 -96.19595410376112) bank60466 +60467 POINT(30.573013188708913 -95.56986793489875) bank60467 +60468 POINT(29.06389552530739 -95.10761154205414) bank60468 +60469 POINT(29.28516177019139 -95.74750331552973) bank60469 +60470 POINT(30.570752608575713 -95.15198395193569) bank60470 +60471 POINT(30.111477034776456 -96.05495366649157) bank60471 +60472 POINT(30.435445210782426 -94.4017349231013) bank60472 +60473 POINT(29.80369039398208 -94.77164297610152) bank60473 +60474 POINT(29.778186139520713 -95.3316117010948) bank60474 +60475 POINT(29.05744790042077 -95.78518472660633) bank60475 +60476 POINT(30.094168010257047 -94.99003101177702) bank60476 +60477 POINT(30.518379828016986 -95.64050203551993) bank60477 +60478 POINT(29.93724630898894 -95.69603567303078) bank60478 +60479 POINT(29.31905992585204 -95.10762567499154) bank60479 +60480 POINT(29.875963610125883 -94.91963261086083) bank60480 +60481 POINT(29.939269681492664 -96.25775069590946) bank60481 +60482 POINT(28.777771720064102 -96.22792939365723) bank60482 +60483 POINT(30.1062404065478 -95.1204241639632) bank60483 +60484 POINT(30.58859036043379 -95.23383081845034) bank60484 +60485 POINT(29.173181830885305 -95.21724257253338) bank60485 +60486 POINT(29.453466734110506 -94.80734799606567) bank60486 +60487 POINT(30.38524921733809 -94.44327493481076) bank60487 +60488 POINT(29.27955594505209 -94.60699195354516) bank60488 +60489 POINT(30.533382692322917 -95.33236727953485) bank60489 +60490 POINT(28.794360687420465 -95.1216874413985) bank60490 +60491 POINT(29.927945457462442 -95.88574610629692) bank60491 +60492 POINT(29.59806292987122 -94.55467922306497) bank60492 +60493 POINT(29.530614177896407 -95.4885848033776) bank60493 +60494 POINT(30.345824053948935 -94.6424027310871) bank60494 +60495 POINT(30.128622131781942 -94.91631182715003) bank60495 +60496 POINT(30.101981105207166 -96.00303415804292) bank60496 +60497 POINT(29.276447593105452 -94.4372380217808) bank60497 +60498 POINT(29.411890109914108 -94.60983843798714) bank60498 +60499 POINT(29.78591188852283 -95.66204574685581) bank60499 +60500 POINT(28.813116842385654 -95.36171854514122) bank60500 +60501 POINT(29.08574386719061 -95.0301289570664) bank60501 +60502 POINT(29.92390431930911 -94.72868897832372) bank60502 +60503 POINT(30.651286592703674 -96.36187415718884) bank60503 +60504 POINT(28.87232679883306 -94.63059904106706) bank60504 +60505 POINT(28.90462244533005 -95.44604454289748) bank60505 +60506 POINT(29.868964354209222 -95.49789420617327) bank60506 +60507 POINT(29.061976093623617 -94.58839303640119) bank60507 +60508 POINT(28.89801972731044 -96.26204141285643) bank60508 +60509 POINT(30.536908152430758 -95.76435441267807) bank60509 +60510 POINT(29.996560272223785 -95.93205303156375) bank60510 +60511 POINT(28.90889352183316 -96.14579830895505) bank60511 +60512 POINT(29.33354800118324 -95.8604746409537) bank60512 +60513 POINT(30.11504423374442 -96.11278996362829) bank60513 +60514 POINT(29.460142247574673 -95.4042077439312) bank60514 +60515 POINT(30.75118415482892 -95.43869962640089) bank60515 +60516 POINT(29.83234021922041 -95.73299307045768) bank60516 +60517 POINT(29.307729950322955 -94.94736590439494) bank60517 +60518 POINT(29.3925619303173 -94.40878973234041) bank60518 +60519 POINT(29.66765695487399 -94.99251773924085) bank60519 +60520 POINT(30.46504309406572 -94.45613011058614) bank60520 +60521 POINT(29.249498900735368 -95.54739124737932) bank60521 +60522 POINT(29.166025597811533 -94.75582958648731) bank60522 +60523 POINT(29.75022489632509 -95.17688333929361) bank60523 +60524 POINT(30.277351972415527 -95.2869932320977) bank60524 +60525 POINT(30.522212090243027 -95.28359486742752) bank60525 +60526 POINT(29.095976413046614 -94.97396187730713) bank60526 +60527 POINT(29.383641917430246 -94.41232315191297) bank60527 +60528 POINT(30.327949261469076 -95.8516550445311) bank60528 +60529 POINT(29.52010202169936 -96.17036920011228) bank60529 +60530 POINT(28.798888738777386 -96.15087496551241) bank60530 +60531 POINT(29.24823089395623 -95.91705467130593) bank60531 +60532 POINT(28.882514582939418 -95.36861013777371) bank60532 +60533 POINT(30.158926635867136 -94.81981729290767) bank60533 +60534 POINT(28.98238737045409 -95.86256812284857) bank60534 +60535 POINT(29.418703941754757 -95.52482275118385) bank60535 +60536 POINT(29.845609821791882 -96.17486287383409) bank60536 +60537 POINT(29.97677719237706 -94.4915665596882) bank60537 +60538 POINT(30.66248755416044 -95.89005963078597) bank60538 +60539 POINT(28.88320548124059 -95.90347525228808) bank60539 +60540 POINT(28.970597969320924 -95.29187622127846) bank60540 +60541 POINT(28.94597651181054 -96.26198593769097) bank60541 +60542 POINT(28.91247290801794 -96.36543511996094) bank60542 +60543 POINT(29.400533828551886 -95.41767653825333) bank60543 +60544 POINT(28.89996434471164 -94.7462341098378) bank60544 +60545 POINT(29.39788086679092 -96.00606596456656) bank60545 +60546 POINT(29.809999953601885 -94.70870690897354) bank60546 +60547 POINT(29.208523172619802 -94.89662886363274) bank60547 +60548 POINT(29.785695220733295 -95.22542649872044) bank60548 +60549 POINT(30.66173344527268 -94.66706660983606) bank60549 +60550 POINT(28.802701920561365 -94.87722143477447) bank60550 +60551 POINT(30.12229560391826 -96.24731491300042) bank60551 +60552 POINT(29.481582585011413 -94.62804823091216) bank60552 +60553 POINT(30.510917699747324 -94.41697580466597) bank60553 +60554 POINT(30.155540812536792 -95.12334445746617) bank60554 +60555 POINT(29.55494890468412 -95.90027023176641) bank60555 +60556 POINT(30.544866580021484 -95.23403292683659) bank60556 +60557 POINT(29.77285030696172 -96.17199949617036) bank60557 +60558 POINT(30.262881358331157 -95.63770779572806) bank60558 +60559 POINT(29.24333270715053 -95.49222233019435) bank60559 +60560 POINT(29.959970263699514 -95.73202956202392) bank60560 +60561 POINT(30.61285058065639 -94.57713962246596) bank60561 +60562 POINT(29.952045607083534 -94.74904797383103) bank60562 +60563 POINT(29.924501183167923 -96.30095924836756) bank60563 +60564 POINT(29.300134822400366 -94.39773931425253) bank60564 +60565 POINT(29.487124079918885 -94.39295666938315) bank60565 +60566 POINT(29.905340565223607 -95.91971734047301) bank60566 +60567 POINT(29.178540943677973 -95.19890573136908) bank60567 +60568 POINT(28.82705659020531 -94.70833056111128) bank60568 +60569 POINT(29.56130722610984 -95.78011745557264) bank60569 +60570 POINT(28.883106762800452 -95.72098227041002) bank60570 +60571 POINT(29.062984348501267 -95.28371067086539) bank60571 +60572 POINT(28.96827425606352 -95.09461823653936) bank60572 +60573 POINT(30.552583850027087 -95.33401006855131) bank60573 +60574 POINT(29.767572973035424 -95.46895062641599) bank60574 +60575 POINT(29.762740488844397 -96.12068557943164) bank60575 +60576 POINT(29.75990393410919 -95.84358374712355) bank60576 +60577 POINT(30.11190709121912 -95.88487548327562) bank60577 +60578 POINT(28.944690536621305 -95.86928136132957) bank60578 +60579 POINT(29.336743225646014 -96.31358613045276) bank60579 +60580 POINT(29.817881385829672 -95.98857186459175) bank60580 +60581 POINT(30.569843448675275 -95.09143811431403) bank60581 +60582 POINT(30.61829485409792 -95.09695247165216) bank60582 +60583 POINT(29.20823617589531 -96.00997951051066) bank60583 +60584 POINT(29.430443525899705 -94.55688880938149) bank60584 +60585 POINT(30.417774831160727 -95.11792506982415) bank60585 +60586 POINT(29.46964210110638 -96.00130460356293) bank60586 +60587 POINT(29.186658186462562 -94.71393536225936) bank60587 +60588 POINT(30.75469401100232 -96.35225592449335) bank60588 +60589 POINT(29.28567116619223 -95.79494344350825) bank60589 +60590 POINT(29.472111487206693 -94.8225763414643) bank60590 +60591 POINT(29.316544846632027 -94.56569589404366) bank60591 +60592 POINT(30.13688635600041 -95.98300786608178) bank60592 +60593 POINT(30.078168298908082 -94.8084320686572) bank60593 +60594 POINT(30.66042906911484 -95.91555712071096) bank60594 +60595 POINT(30.512427737665767 -95.38849945091272) bank60595 +60596 POINT(30.55285656328103 -94.59795868435016) bank60596 +60597 POINT(29.979811912938143 -94.4951033785369) bank60597 +60598 POINT(30.74618214835676 -96.2981195784718) bank60598 +60599 POINT(29.429865640415453 -96.07971357717263) bank60599 +60600 POINT(30.047731497494013 -94.50568439460365) bank60600 +60601 POINT(28.972641271390465 -95.61584540524377) bank60601 +60602 POINT(29.564073817412925 -94.38052401399176) bank60602 +60603 POINT(30.679591697370032 -96.17919989456156) bank60603 +60604 POINT(29.391318329992547 -95.7205912271121) bank60604 +60605 POINT(30.240971260507973 -95.06704714949062) bank60605 +60606 POINT(30.039528914013744 -95.69427343897877) bank60606 +60607 POINT(29.401214104502287 -95.68865760833283) bank60607 +60608 POINT(29.24370895755612 -94.7093110284948) bank60608 +60609 POINT(29.12980496132877 -94.98521027630747) bank60609 +60610 POINT(30.143908742805195 -96.29602166876234) bank60610 +60611 POINT(28.787471793752882 -95.89207454060069) bank60611 +60612 POINT(30.6494171278113 -94.50817742311864) bank60612 +60613 POINT(30.60790043382467 -94.54255961409449) bank60613 +60614 POINT(29.122198629177014 -95.43431268278256) bank60614 +60615 POINT(30.68034177858021 -95.49621769612448) bank60615 +60616 POINT(29.662011596204785 -95.72765117967498) bank60616 +60617 POINT(30.463982743229 -94.67322801253862) bank60617 +60618 POINT(30.29798762237003 -95.58458880258496) bank60618 +60619 POINT(30.25211347131177 -96.32584607955181) bank60619 +60620 POINT(29.354770271529702 -95.08555414669816) bank60620 +60621 POINT(29.864815021743578 -95.78638204761897) bank60621 +60622 POINT(29.65614103747088 -95.37550301385754) bank60622 +60623 POINT(30.34089603858651 -94.83236431585868) bank60623 +60624 POINT(30.366127528910273 -94.77749254120786) bank60624 +60625 POINT(29.629047561056588 -94.99497844729603) bank60625 +60626 POINT(29.824145607239416 -96.3473831389325) bank60626 +60627 POINT(30.034284840488148 -95.2381891662265) bank60627 +60628 POINT(30.618040545387093 -95.66337064552422) bank60628 +60629 POINT(29.407653827226607 -94.59355715383352) bank60629 +60630 POINT(29.631170178509475 -94.42902865262852) bank60630 +60631 POINT(29.391688552097992 -95.31407123581265) bank60631 +60632 POINT(30.728206838387948 -95.12033908135336) bank60632 +60633 POINT(29.812939744060504 -95.24834902753427) bank60633 +60634 POINT(29.39094849358115 -95.33501977932411) bank60634 +60635 POINT(29.59357102163076 -94.57677545560894) bank60635 +60636 POINT(29.997200285233355 -95.62453977471483) bank60636 +60637 POINT(30.528967441677732 -95.5116326434895) bank60637 +60638 POINT(30.495399217734143 -94.87405184027368) bank60638 +60639 POINT(29.705324768576855 -94.60226074370817) bank60639 +60640 POINT(29.25091979287195 -95.73946706656075) bank60640 +60641 POINT(29.65710071038657 -95.30235864094146) bank60641 +60642 POINT(28.91886596416515 -96.1056184900331) bank60642 +60643 POINT(29.787590153224183 -94.96526663118503) bank60643 +60644 POINT(29.72454592719227 -94.47383802848906) bank60644 +60645 POINT(30.696760147504992 -96.35543241167818) bank60645 +60646 POINT(29.016998543981078 -95.8085409941136) bank60646 +60647 POINT(29.648704385704377 -94.63226992004874) bank60647 +60648 POINT(30.44186425458484 -96.32446169953813) bank60648 +60649 POINT(29.961182674769802 -95.55402359850406) bank60649 +60650 POINT(29.058337079520093 -95.63909672569193) bank60650 +60651 POINT(29.302553492032942 -96.2492087321647) bank60651 +60652 POINT(29.909552649345994 -94.74871257708332) bank60652 +60653 POINT(30.4973832778443 -96.03020574169433) bank60653 +60654 POINT(29.412984808412354 -95.06409827467736) bank60654 +60655 POINT(29.678113312535665 -95.32449974508775) bank60655 +60656 POINT(29.676322329607174 -95.11758451129255) bank60656 +60657 POINT(28.887356515687465 -95.09680467468378) bank60657 +60658 POINT(28.99687514598363 -94.71401196665438) bank60658 +60659 POINT(29.698688857718956 -95.63747942587263) bank60659 +60660 POINT(28.976573554060955 -95.38917828136925) bank60660 +60661 POINT(29.00787392423456 -95.64529430838488) bank60661 +60662 POINT(29.994807314269867 -95.82189703163856) bank60662 +60663 POINT(29.152655933375584 -96.24349502945186) bank60663 +60664 POINT(28.870547238047735 -95.83689301321556) bank60664 +60665 POINT(29.83391342133605 -95.56644263004242) bank60665 +60666 POINT(29.340435628576632 -95.11414731842885) bank60666 +60667 POINT(30.729509016421442 -95.69008612645696) bank60667 +60668 POINT(30.34234892396481 -95.75265463870241) bank60668 +60669 POINT(30.723983963580586 -94.89827118782159) bank60669 +60670 POINT(28.84469318020818 -94.942699114302) bank60670 +60671 POINT(29.056693666185094 -96.00844578416925) bank60671 +60672 POINT(28.994511821119396 -95.33214507533403) bank60672 +60673 POINT(29.17473823250494 -95.56524283631252) bank60673 +60674 POINT(29.542297666110688 -95.93289825366463) bank60674 +60675 POINT(30.752333799066943 -96.2149014335825) bank60675 +60676 POINT(30.52587706323378 -94.90982424075284) bank60676 +60677 POINT(30.57196225393454 -95.37237903890464) bank60677 +60678 POINT(28.90932517340995 -96.28145304775178) bank60678 +60679 POINT(30.587140114039855 -96.00613280630492) bank60679 +60680 POINT(29.313708800637468 -95.98661973793608) bank60680 +60681 POINT(30.032319989247746 -94.83259233447856) bank60681 +60682 POINT(29.29188596565037 -95.47016654232597) bank60682 +60683 POINT(29.866232976998894 -94.74682246168557) bank60683 +60684 POINT(28.841355596162813 -95.8240230105067) bank60684 +60685 POINT(30.385250632796975 -95.16655714251853) bank60685 +60686 POINT(30.49864493912446 -95.80006643476409) bank60686 +60687 POINT(29.90852087817234 -94.5729012007689) bank60687 +60688 POINT(29.138238582156262 -94.67935258945595) bank60688 +60689 POINT(29.5580354459453 -95.39526265861852) bank60689 +60690 POINT(30.524909648582746 -95.9384334918779) bank60690 +60691 POINT(30.450145925488016 -94.74802471059618) bank60691 +60692 POINT(29.104494474916635 -95.79924296605556) bank60692 +60693 POINT(30.675663576194236 -94.52617618267382) bank60693 +60694 POINT(30.699312354731823 -95.55690872540336) bank60694 +60695 POINT(28.844102397231612 -95.81465630709019) bank60695 +60696 POINT(30.419329371701313 -95.93321093140835) bank60696 +60697 POINT(28.992140742999766 -94.77840731513764) bank60697 +60698 POINT(29.67465120651142 -95.7154835450666) bank60698 +60699 POINT(30.46700255448498 -95.92537220503539) bank60699 +60700 POINT(29.316798291677213 -96.32212277206062) bank60700 +60701 POINT(29.167956224362715 -95.48804056650775) bank60701 +60702 POINT(30.307334762407763 -94.5466776898503) bank60702 +60703 POINT(30.40942994737573 -95.17508245113719) bank60703 +60704 POINT(29.70828809682221 -96.17874726238821) bank60704 +60705 POINT(30.639812009241396 -95.90707188445293) bank60705 +60706 POINT(29.993710254546645 -94.4088651238649) bank60706 +60707 POINT(30.55811187079942 -94.7053471144896) bank60707 +60708 POINT(30.081397471382452 -94.76318708158928) bank60708 +60709 POINT(29.33783446116542 -96.01474635816318) bank60709 +60710 POINT(29.775127501233793 -96.00000750224218) bank60710 +60711 POINT(28.780541731613408 -95.52794060064772) bank60711 +60712 POINT(30.202570390116467 -95.67706270765055) bank60712 +60713 POINT(30.065795903760957 -94.56340644192993) bank60713 +60714 POINT(29.59643870871902 -95.67739555358473) bank60714 +60715 POINT(30.254433343873405 -95.54964120397433) bank60715 +60716 POINT(29.866628614688395 -94.92853048018814) bank60716 +60717 POINT(30.07405114430335 -95.84318448892908) bank60717 +60718 POINT(30.488693223051705 -94.46385139956816) bank60718 +60719 POINT(29.524563527734667 -95.15671849503373) bank60719 +60720 POINT(30.334285497483776 -95.56854705049079) bank60720 +60721 POINT(29.694354379495365 -95.23655334291458) bank60721 +60722 POINT(29.72400351539004 -95.69595657488304) bank60722 +60723 POINT(29.074163959851212 -95.50702287772992) bank60723 +60724 POINT(29.13342044649376 -95.52191704587814) bank60724 +60725 POINT(29.908805870186864 -95.84759152473056) bank60725 +60726 POINT(30.198728155972745 -95.15818317505517) bank60726 +60727 POINT(28.99266229798474 -94.94373406616066) bank60727 +60728 POINT(29.975296818930545 -95.97813414732991) bank60728 +60729 POINT(29.705201238047948 -95.06870328112728) bank60729 +60730 POINT(29.74309500805482 -94.69977375940564) bank60730 +60731 POINT(28.979516682740414 -94.70486847615507) bank60731 +60732 POINT(28.81146179070942 -95.7015118897503) bank60732 +60733 POINT(29.349505261576663 -96.31832168682018) bank60733 +60734 POINT(29.36844771516515 -94.62772495885832) bank60734 +60735 POINT(30.478527765631256 -95.14496710490003) bank60735 +60736 POINT(29.35940389318813 -95.61906797405442) bank60736 +60737 POINT(28.830491684002883 -96.10242529373146) bank60737 +60738 POINT(30.750104557680935 -94.94621323574312) bank60738 +60739 POINT(30.14599398355856 -95.29328155298793) bank60739 +60740 POINT(29.88804755664906 -95.53949873221882) bank60740 +60741 POINT(29.45876780773912 -94.54474437849007) bank60741 +60742 POINT(30.528518925873463 -94.76029665811707) bank60742 +60743 POINT(30.315326237290602 -95.39205124694143) bank60743 +60744 POINT(28.85340513706794 -94.85388026911473) bank60744 +60745 POINT(29.955239602066214 -94.65998896698393) bank60745 +60746 POINT(30.392303464102447 -95.57824877598134) bank60746 +60747 POINT(29.017240513079813 -95.21457437570449) bank60747 +60748 POINT(29.106742560674864 -94.96170943020752) bank60748 +60749 POINT(30.687357810097424 -96.2723826947322) bank60749 +60750 POINT(29.39454087168165 -94.63499779062191) bank60750 +60751 POINT(28.828988399992777 -94.66877604262457) bank60751 +60752 POINT(30.019486192261706 -94.93539486957333) bank60752 +60753 POINT(30.647534891448394 -95.31081152422993) bank60753 +60754 POINT(29.66695735566941 -96.12298356342131) bank60754 +60755 POINT(30.61140211546513 -96.26426796798339) bank60755 +60756 POINT(28.813153526458997 -95.52042156345136) bank60756 +60757 POINT(29.35820996995736 -95.97933101283444) bank60757 +60758 POINT(29.726955438950924 -96.35083919257636) bank60758 +60759 POINT(30.492768908471795 -95.65217467419679) bank60759 +60760 POINT(29.76573673175875 -96.12502991408664) bank60760 +60761 POINT(29.838040459167853 -95.4370914080956) bank60761 +60762 POINT(29.342157456702996 -95.51618268368568) bank60762 +60763 POINT(29.43654760912381 -95.60585742435217) bank60763 +60764 POINT(28.983137337460033 -95.65090371102849) bank60764 +60765 POINT(29.070893698879924 -94.6602167225843) bank60765 +60766 POINT(29.85809385525967 -95.97264956340572) bank60766 +60767 POINT(30.709096992312386 -94.94744925048711) bank60767 +60768 POINT(30.297408609269315 -95.51711919342058) bank60768 +60769 POINT(30.55423757459264 -96.1438077886872) bank60769 +60770 POINT(30.021914609201065 -94.76221417551753) bank60770 +60771 POINT(28.99865300411984 -94.73020375686222) bank60771 +60772 POINT(29.729958878279476 -95.77288844847045) bank60772 +60773 POINT(29.65112070750662 -95.48038573012899) bank60773 +60774 POINT(29.493016742924986 -95.85884090003054) bank60774 +60775 POINT(29.676808509840367 -95.7990978061708) bank60775 +60776 POINT(29.83898254548849 -95.37005350179133) bank60776 +60777 POINT(30.459509142763117 -94.86554443527623) bank60777 +60778 POINT(28.92246286793422 -95.22292311592788) bank60778 +60779 POINT(29.320174183733503 -94.42640787859119) bank60779 +60780 POINT(30.20568759074697 -96.04838398625441) bank60780 +60781 POINT(29.248469099146444 -94.80622595282388) bank60781 +60782 POINT(29.365749224889043 -95.85260793961375) bank60782 +60783 POINT(29.921415897784698 -95.93122596299405) bank60783 +60784 POINT(29.164404077336304 -94.50071320071568) bank60784 +60785 POINT(29.164577140249214 -94.61190422800034) bank60785 +60786 POINT(29.270144507112782 -95.45521263980555) bank60786 +60787 POINT(30.432861643436944 -96.23188922213664) bank60787 +60788 POINT(29.064680887339136 -95.33174696322804) bank60788 +60789 POINT(29.95596233299409 -96.30949046014511) bank60789 +60790 POINT(29.08686169140569 -95.28417427159975) bank60790 +60791 POINT(30.38031796494567 -94.80806639296495) bank60791 +60792 POINT(30.140321519232298 -95.790831030922) bank60792 +60793 POINT(28.906663263717636 -94.58298196596863) bank60793 +60794 POINT(28.996720172261547 -94.79811979086114) bank60794 +60795 POINT(29.580634074231643 -95.46857180409417) bank60795 +60796 POINT(30.261690092873554 -96.06977834360727) bank60796 +60797 POINT(30.33559510964521 -95.86856401608297) bank60797 +60798 POINT(29.781804617310847 -95.48666084864632) bank60798 +60799 POINT(30.169955045702608 -96.08242348045556) bank60799 +60800 POINT(29.10989937125591 -94.78398211374235) bank60800 +60801 POINT(29.013752151199096 -96.01783339135653) bank60801 +60802 POINT(28.834814871775727 -95.51779773795805) bank60802 +60803 POINT(28.961843621330445 -95.67298532102556) bank60803 +60804 POINT(29.054611648420412 -96.26280857164994) bank60804 +60805 POINT(29.41642956265846 -95.68215427885899) bank60805 +60806 POINT(30.21842025363398 -95.9260496965265) bank60806 +60807 POINT(30.375792884784644 -94.41387568021761) bank60807 +60808 POINT(28.934642441586032 -95.98951442047479) bank60808 +60809 POINT(29.254819218999838 -95.28954403554415) bank60809 +60810 POINT(30.306210065165512 -95.92337919036189) bank60810 +60811 POINT(30.55632015141429 -96.22582029496859) bank60811 +60812 POINT(28.7773017966824 -96.23798821108855) bank60812 +60813 POINT(30.00321721450999 -95.43193649962427) bank60813 +60814 POINT(29.044499838098744 -95.44898769492006) bank60814 +60815 POINT(30.43684401203225 -95.17009086502966) bank60815 +60816 POINT(30.519329061239205 -94.5612881286262) bank60816 +60817 POINT(30.130088721953822 -96.23878117314347) bank60817 +60818 POINT(29.80226783063806 -95.03922983935249) bank60818 +60819 POINT(29.25552203297087 -95.17769341092533) bank60819 +60820 POINT(29.538720428533768 -95.5017612489245) bank60820 +60821 POINT(29.056778905739478 -94.76939731853143) bank60821 +60822 POINT(28.7665758243553 -95.29460158479708) bank60822 +60823 POINT(29.419996862784494 -95.75671465689453) bank60823 +60824 POINT(29.803614647310273 -94.73574854383776) bank60824 +60825 POINT(28.844948142030468 -96.21441273499786) bank60825 +60826 POINT(28.791968120508177 -95.06003183403233) bank60826 +60827 POINT(29.953297948875154 -95.25505504267957) bank60827 +60828 POINT(30.3738301014026 -95.89445166281986) bank60828 +60829 POINT(30.470807138099584 -95.39066111484055) bank60829 +60830 POINT(30.38479129208169 -95.09522381325894) bank60830 +60831 POINT(29.011997014929182 -95.56694584245433) bank60831 +60832 POINT(28.867638308183185 -94.59235387156322) bank60832 +60833 POINT(30.386693501879666 -95.38140149205917) bank60833 +60834 POINT(29.46936355943288 -95.85267976003465) bank60834 +60835 POINT(30.5093749387445 -94.71796025899394) bank60835 +60836 POINT(30.606902390635423 -95.2653891364371) bank60836 +60837 POINT(30.523933945116195 -95.32829364394894) bank60837 +60838 POINT(29.478945032112495 -94.37056611531517) bank60838 +60839 POINT(29.354891997091553 -94.61655261794746) bank60839 +60840 POINT(29.89422630462458 -96.29328759352757) bank60840 +60841 POINT(30.557380105476682 -94.88124775305387) bank60841 +60842 POINT(30.130223394041227 -95.37222902366518) bank60842 +60843 POINT(29.439974274937832 -95.56519818311139) bank60843 +60844 POINT(29.372237912992443 -95.24681408764975) bank60844 +60845 POINT(29.922419817062618 -95.69680547386295) bank60845 +60846 POINT(28.790133355316442 -95.37848050550154) bank60846 +60847 POINT(30.478844709916224 -94.92566498436311) bank60847 +60848 POINT(30.028447806177194 -95.5168294249048) bank60848 +60849 POINT(30.28388238081617 -95.95920766379135) bank60849 +60850 POINT(28.923948059854776 -96.2358095826874) bank60850 +60851 POINT(29.754224442053385 -94.60321828086197) bank60851 +60852 POINT(30.388072413038685 -94.97753947287653) bank60852 +60853 POINT(29.378689883214633 -96.30322927017133) bank60853 +60854 POINT(30.621207302269216 -94.56529556836631) bank60854 +60855 POINT(29.902709234833154 -95.69804692933224) bank60855 +60856 POINT(29.243029846200074 -94.57772477303956) bank60856 +60857 POINT(29.649160190592724 -96.04146799021929) bank60857 +60858 POINT(30.36271142074459 -94.8310033096033) bank60858 +60859 POINT(29.938847993851414 -95.9338960510869) bank60859 +60860 POINT(29.022009711407176 -96.14193938578188) bank60860 +60861 POINT(29.93295921472986 -94.75871353668649) bank60861 +60862 POINT(29.665867045193174 -95.12636474248752) bank60862 +60863 POINT(30.23778437019919 -95.20450819933393) bank60863 +60864 POINT(29.915256546035447 -94.74341046910989) bank60864 +60865 POINT(30.14996400726766 -95.93425435381985) bank60865 +60866 POINT(28.76110536254516 -95.75626761409757) bank60866 +60867 POINT(30.75368071300815 -95.64308302409086) bank60867 +60868 POINT(30.10089968844541 -94.83899669915519) bank60868 +60869 POINT(30.289645992832572 -96.21521554039043) bank60869 +60870 POINT(30.14251955439783 -95.71643506110084) bank60870 +60871 POINT(30.18715823547704 -94.73857260495801) bank60871 +60872 POINT(30.71244162557152 -95.71483949443129) bank60872 +60873 POINT(30.065785968164295 -95.65986706324301) bank60873 +60874 POINT(29.517435931451878 -96.06982823530262) bank60874 +60875 POINT(29.28111640384825 -95.53686795236433) bank60875 +60876 POINT(30.37126812089179 -96.1611719107958) bank60876 +60877 POINT(28.947816369613236 -95.81848835106109) bank60877 +60878 POINT(29.868698010550727 -95.15734825248778) bank60878 +60879 POINT(30.12827149849722 -94.86710952130505) bank60879 +60880 POINT(29.7508039943674 -94.857100221525) bank60880 +60881 POINT(30.37758389006304 -94.73115094326057) bank60881 +60882 POINT(29.335936445638502 -95.26841562248985) bank60882 +60883 POINT(30.295170640094955 -96.25830146670683) bank60883 +60884 POINT(29.078573817086646 -96.30739776207226) bank60884 +60885 POINT(30.018150835820684 -95.75682925108484) bank60885 +60886 POINT(29.846242138775025 -95.63161629865846) bank60886 +60887 POINT(29.16243833866953 -95.1183463688731) bank60887 +60888 POINT(29.42354887835841 -96.07535760574271) bank60888 +60889 POINT(29.507076052635213 -94.86327646995191) bank60889 +60890 POINT(29.143250840818588 -95.04291833444988) bank60890 +60891 POINT(30.499556517669816 -95.33390464934939) bank60891 +60892 POINT(30.441034575601684 -95.06946963502925) bank60892 +60893 POINT(30.296840900470244 -95.19426401714132) bank60893 +60894 POINT(30.333602294266406 -94.9972637994072) bank60894 +60895 POINT(29.54467613256309 -95.8560385786624) bank60895 +60896 POINT(28.855991805498217 -95.6026524903579) bank60896 +60897 POINT(28.91409315658395 -95.7224181015562) bank60897 +60898 POINT(30.28934079702798 -94.75587709104641) bank60898 +60899 POINT(30.081284557794888 -95.42491918793493) bank60899 +60900 POINT(30.347664662923236 -95.2343797591975) bank60900 +60901 POINT(30.2524222981051 -94.54318238484286) bank60901 +60902 POINT(29.018794490774386 -95.27771119235078) bank60902 +60903 POINT(28.84923180923691 -95.69394187636382) bank60903 +60904 POINT(30.136172530271047 -94.68187681721184) bank60904 +60905 POINT(30.06064759316827 -95.5141440471214) bank60905 +60906 POINT(29.195611832934404 -94.57551503204957) bank60906 +60907 POINT(29.275263367072764 -94.99032129155778) bank60907 +60908 POINT(29.42266375833679 -96.08025679936432) bank60908 +60909 POINT(30.5212352784275 -94.89316063422548) bank60909 +60910 POINT(29.490714824737033 -95.71622449452241) bank60910 +60911 POINT(29.5626964010496 -94.77244159188264) bank60911 +60912 POINT(29.62087795737149 -95.82082604594545) bank60912 +60913 POINT(30.26686249814223 -95.50332170580724) bank60913 +60914 POINT(29.07298751607232 -95.29142405530843) bank60914 +60915 POINT(30.393072419845556 -95.07040931581192) bank60915 +60916 POINT(29.388021249308853 -95.25788841070366) bank60916 +60917 POINT(29.665400802797684 -96.16955250946323) bank60917 +60918 POINT(29.56347988512626 -94.75935184904364) bank60918 +60919 POINT(29.365874162847355 -94.9485744094892) bank60919 +60920 POINT(30.654782258209316 -94.53591116987623) bank60920 +60921 POINT(28.88080862012529 -95.40741381309516) bank60921 +60922 POINT(30.229903128057664 -95.49397058535648) bank60922 +60923 POINT(30.618655176888794 -95.86281968257512) bank60923 +60924 POINT(30.532914416482704 -94.64373073066062) bank60924 +60925 POINT(29.76702310676299 -96.03456981857532) bank60925 +60926 POINT(30.06231116741953 -94.44512133115904) bank60926 +60927 POINT(29.690472296647872 -94.66811399409941) bank60927 +60928 POINT(29.43490724708466 -94.61145106969006) bank60928 +60929 POINT(30.548748871198683 -95.27605035395611) bank60929 +60930 POINT(30.28389656693789 -94.41626512055343) bank60930 +60931 POINT(30.39159053124392 -95.28643508116414) bank60931 +60932 POINT(29.92144211474513 -94.7912369172703) bank60932 +60933 POINT(29.971045916355337 -94.68255143316966) bank60933 +60934 POINT(30.32084299855598 -94.65633038120941) bank60934 +60935 POINT(29.878282120042652 -94.85499311816251) bank60935 +60936 POINT(28.932864392158397 -94.65665063936824) bank60936 +60937 POINT(29.4049095856223 -94.65618785286918) bank60937 +60938 POINT(30.16107709439755 -95.01237339005617) bank60938 +60939 POINT(30.748291220593874 -94.54323532462037) bank60939 +60940 POINT(29.454699056599466 -94.80065653898252) bank60940 +60941 POINT(30.687358301190848 -95.32737370104218) bank60941 +60942 POINT(29.168118975768632 -95.43721448951484) bank60942 +60943 POINT(30.402020296196426 -94.41313122689459) bank60943 +60944 POINT(29.288907813970386 -95.9579451975817) bank60944 +60945 POINT(29.88976190978886 -95.38360696617917) bank60945 +60946 POINT(30.3765574207263 -94.91833840994776) bank60946 +60947 POINT(29.196218583918544 -95.87416263830417) bank60947 +60948 POINT(29.07712115563976 -94.50748266120566) bank60948 +60949 POINT(29.518037245311934 -96.36941070174258) bank60949 +60950 POINT(29.94372349503591 -95.70167073411525) bank60950 +60951 POINT(30.222946623922695 -95.53253900264767) bank60951 +60952 POINT(30.16532313275264 -95.68247217531643) bank60952 +60953 POINT(30.338421958458166 -94.53704139378232) bank60953 +60954 POINT(30.045578574596814 -96.02522816952033) bank60954 +60955 POINT(29.174764913518324 -95.95105507639157) bank60955 +60956 POINT(30.696669090037055 -95.84914738726874) bank60956 +60957 POINT(28.913584073467224 -94.50155888806447) bank60957 +60958 POINT(29.79788460414996 -94.86681840219568) bank60958 +60959 POINT(29.212463564067804 -94.44556002149285) bank60959 +60960 POINT(29.90853596030023 -94.82959472093765) bank60960 +60961 POINT(29.643953479911936 -95.19704417661634) bank60961 +60962 POINT(29.520107534555 -95.38482693461856) bank60962 +60963 POINT(30.716396674565505 -94.53015324105473) bank60963 +60964 POINT(28.93543212084957 -96.05977974610656) bank60964 +60965 POINT(29.342068503265676 -94.44136554780455) bank60965 +60966 POINT(30.74828006087787 -95.47775752588412) bank60966 +60967 POINT(28.796378699891708 -96.16022344119385) bank60967 +60968 POINT(30.010039529241435 -95.8653579799057) bank60968 +60969 POINT(29.902179943439975 -95.03955393160054) bank60969 +60970 POINT(29.256315982049294 -95.76907654011106) bank60970 +60971 POINT(29.099595181277024 -95.97022361307586) bank60971 +60972 POINT(30.373522548463527 -94.3746453661564) bank60972 +60973 POINT(29.409877462071165 -95.25549387789614) bank60973 +60974 POINT(30.173464542050475 -95.15989586506183) bank60974 +60975 POINT(28.90791666089894 -95.98573466963362) bank60975 +60976 POINT(30.50191438065931 -94.54206811645663) bank60976 +60977 POINT(29.850710160306367 -95.80775767647688) bank60977 +60978 POINT(30.339670623879574 -95.69489656018041) bank60978 +60979 POINT(30.475591645304075 -95.07993920986002) bank60979 +60980 POINT(29.307957093903077 -94.94063472011494) bank60980 +60981 POINT(28.864581497661554 -96.16993117444726) bank60981 +60982 POINT(29.774339717399855 -95.56312866842096) bank60982 +60983 POINT(29.380871170527413 -94.931656751959) bank60983 +60984 POINT(30.360799246038546 -95.37518802245125) bank60984 +60985 POINT(29.109500067685822 -96.10063494818445) bank60985 +60986 POINT(30.622310136719456 -94.6861498428923) bank60986 +60987 POINT(30.66295322063108 -95.51601373838177) bank60987 +60988 POINT(29.459983144697016 -94.92225284402984) bank60988 +60989 POINT(28.82404054373614 -95.61175312550989) bank60989 +60990 POINT(30.12736036739173 -94.46218849262019) bank60990 +60991 POINT(29.61580008814734 -94.59039275061072) bank60991 +60992 POINT(29.817730267294582 -94.94027388145652) bank60992 +60993 POINT(30.418295579908904 -95.12127939545651) bank60993 +60994 POINT(30.647072458858794 -95.04790697347912) bank60994 +60995 POINT(29.809896084566205 -94.77874594327545) bank60995 +60996 POINT(29.31737239680755 -95.61503475806462) bank60996 +60997 POINT(29.483190563482932 -95.27234231577691) bank60997 +60998 POINT(28.85127372028171 -95.62911037326691) bank60998 +60999 POINT(29.027717407373036 -95.70399523293091) bank60999 +61000 POINT(28.954611540300643 -94.48754606696556) bank61000 +61001 POINT(29.181251162937684 -96.20582897627142) bank61001 +61002 POINT(30.556715777347026 -94.53913097508728) bank61002 +61003 POINT(30.66148056808986 -96.19020860556299) bank61003 +61004 POINT(28.88682660973319 -95.64775407998673) bank61004 +61005 POINT(28.999266321975156 -96.02840787802596) bank61005 +61006 POINT(29.65419630082883 -95.62960348376652) bank61006 +61007 POINT(29.7766287079639 -95.79206771704503) bank61007 +61008 POINT(29.346138020879057 -95.54176505847151) bank61008 +61009 POINT(29.82359852096513 -94.77169673756052) bank61009 +61010 POINT(28.831162872914966 -95.46135074815729) bank61010 +61011 POINT(29.574730765553902 -95.38918915364937) bank61011 +61012 POINT(28.905166430018646 -95.75832270578042) bank61012 +61013 POINT(30.052010226158767 -94.42343296527449) bank61013 +61014 POINT(30.726920248432624 -95.05801521924026) bank61014 +61015 POINT(29.441804785489193 -95.27416868766079) bank61015 +61016 POINT(29.094617265459718 -94.57020794515948) bank61016 +61017 POINT(29.626050575736528 -95.16832150619003) bank61017 +61018 POINT(29.818311934895366 -95.40644614309356) bank61018 +61019 POINT(30.138183063886643 -95.06696765341628) bank61019 +61020 POINT(30.13203494094389 -94.87702038514675) bank61020 +61021 POINT(28.993796881496603 -95.4791447122616) bank61021 +61022 POINT(30.320477990961322 -94.53428827777579) bank61022 +61023 POINT(29.40831019169776 -94.88031111242063) bank61023 +61024 POINT(28.779991849978973 -94.67353098901614) bank61024 +61025 POINT(29.097532836607893 -95.96532212587368) bank61025 +61026 POINT(29.21640566962912 -94.50113905190716) bank61026 +61027 POINT(29.683683482104826 -95.34755047792652) bank61027 +61028 POINT(29.424575305117727 -94.9865217184869) bank61028 +61029 POINT(30.2028114370474 -95.62660309935366) bank61029 +61030 POINT(29.032621801270032 -94.74066834877931) bank61030 +61031 POINT(30.64789731047053 -95.13367347713472) bank61031 +61032 POINT(30.6599483842158 -95.13664077226872) bank61032 +61033 POINT(29.441530767439644 -94.52645206697514) bank61033 +61034 POINT(30.46695584969701 -96.04208629133291) bank61034 +61035 POINT(29.49660289634197 -95.09647047898761) bank61035 +61036 POINT(29.16278593305038 -94.38868441509632) bank61036 +61037 POINT(30.574576539018196 -95.11622871851532) bank61037 +61038 POINT(28.77774691951773 -95.6548478346701) bank61038 +61039 POINT(30.423798495540595 -96.14248071198998) bank61039 +61040 POINT(29.957447192607834 -95.07321800073808) bank61040 +61041 POINT(30.058184402949095 -95.87483013265009) bank61041 +61042 POINT(30.19199847995928 -95.71230928630789) bank61042 +61043 POINT(30.1242128000897 -95.5872064916867) bank61043 +61044 POINT(28.788675724814006 -96.09815306235063) bank61044 +61045 POINT(29.9753456926146 -94.85834160323304) bank61045 +61046 POINT(29.941805608291688 -95.2647623737464) bank61046 +61047 POINT(28.93317944732386 -95.91861451863006) bank61047 +61048 POINT(29.03347614842311 -95.40822740150367) bank61048 +61049 POINT(29.547428347733216 -94.9417813553062) bank61049 +61050 POINT(30.307462734898486 -94.57185965477635) bank61050 +61051 POINT(29.931341403842747 -95.31861134928424) bank61051 +61052 POINT(29.88549679448345 -94.84926246471277) bank61052 +61053 POINT(28.77045382921098 -94.83276394836689) bank61053 +61054 POINT(29.83945529085831 -95.14456536555733) bank61054 +61055 POINT(30.67506832444421 -95.70900149270634) bank61055 +61056 POINT(29.519153943888856 -94.95923535117282) bank61056 +61057 POINT(29.869435111264053 -95.20264408470739) bank61057 +61058 POINT(29.176667630244378 -96.15585455711536) bank61058 +61059 POINT(29.525410411790705 -96.19446340068887) bank61059 +61060 POINT(29.966063589961568 -95.7295939577246) bank61060 +61061 POINT(29.511376804233368 -96.31915006908994) bank61061 +61062 POINT(29.575933072902338 -95.38521429632937) bank61062 +61063 POINT(30.42072123736511 -95.81569432710418) bank61063 +61064 POINT(29.318633627356558 -95.77751566987791) bank61064 +61065 POINT(28.95583912915854 -95.63376215121164) bank61065 +61066 POINT(29.393189205472744 -96.33600864378626) bank61066 +61067 POINT(30.222087197304198 -95.93160690327376) bank61067 +61068 POINT(29.81864518597614 -96.27054389923988) bank61068 +61069 POINT(30.316123274915206 -94.83141796720551) bank61069 +61070 POINT(30.39830982144251 -96.32614100152679) bank61070 +61071 POINT(30.68592449015368 -94.84988899941494) bank61071 +61072 POINT(29.518272080994816 -94.78622712816752) bank61072 +61073 POINT(29.785679322773575 -94.76488506109436) bank61073 +61074 POINT(29.191629419477437 -96.06648261060822) bank61074 +61075 POINT(29.880484856826456 -95.96364478288999) bank61075 +61076 POINT(30.06234370522085 -95.4036517043112) bank61076 +61077 POINT(30.734793323657964 -94.8626279768438) bank61077 +61078 POINT(29.81004346378351 -96.35329987819831) bank61078 +61079 POINT(29.216414569145297 -94.5360397188839) bank61079 +61080 POINT(30.489898715138747 -95.86648922333089) bank61080 +61081 POINT(30.603985305402517 -95.61220636169152) bank61081 +61082 POINT(29.16759213842014 -95.21439628553577) bank61082 +61083 POINT(28.80236730422019 -95.72433691033734) bank61083 +61084 POINT(30.27759217687796 -96.26632373384524) bank61084 +61085 POINT(30.68320538688013 -94.91909957145108) bank61085 +61086 POINT(30.008842674701345 -94.87652597415341) bank61086 +61087 POINT(29.04530584643418 -96.00431812351877) bank61087 +61088 POINT(30.316980380914906 -96.32156686607269) bank61088 +61089 POINT(28.847910092607847 -95.15689668546723) bank61089 +61090 POINT(30.52130952376725 -95.89862018468189) bank61090 +61091 POINT(29.157998345523556 -94.48083583199859) bank61091 +61092 POINT(30.633723930594943 -95.31194126087678) bank61092 +61093 POINT(28.900690495519996 -95.32904697009468) bank61093 +61094 POINT(29.47980068485721 -94.38165050061109) bank61094 +61095 POINT(30.130691679606308 -95.45130620523648) bank61095 +61096 POINT(28.99283159042697 -94.65696483560323) bank61096 +61097 POINT(30.53558965559542 -96.26146908340206) bank61097 +61098 POINT(30.708172215828817 -94.79740067264513) bank61098 +61099 POINT(29.702279395943886 -95.58022616289871) bank61099 +61100 POINT(30.60871674575244 -95.47411178754146) bank61100 +61101 POINT(29.11596981018024 -96.12189090870285) bank61101 +61102 POINT(29.89396879125603 -95.66147751101722) bank61102 +61103 POINT(30.341985551617444 -96.1574516713038) bank61103 +61104 POINT(30.04964547484012 -94.4437349783422) bank61104 +61105 POINT(30.027568340800663 -95.6081087519492) bank61105 +61106 POINT(29.79550601739952 -95.77937351092253) bank61106 +61107 POINT(29.294044122307643 -95.33791285764752) bank61107 +61108 POINT(29.89503616891492 -94.93454222358046) bank61108 +61109 POINT(29.625855426976145 -95.20573058000228) bank61109 +61110 POINT(29.11853507712864 -95.40205010269423) bank61110 +61111 POINT(30.3417765252558 -94.4750353505865) bank61111 +61112 POINT(30.58398849029614 -94.98983482990081) bank61112 +61113 POINT(30.66202384111394 -96.3664815534046) bank61113 +61114 POINT(30.185127930799016 -94.92127878782256) bank61114 +61115 POINT(30.640112530353665 -95.59930977540495) bank61115 +61116 POINT(29.168824921607243 -95.7135858340215) bank61116 +61117 POINT(29.167275340769702 -95.92150701533693) bank61117 +61118 POINT(30.136593883660982 -95.74881464801989) bank61118 +61119 POINT(30.288709650707187 -94.49895982146232) bank61119 +61120 POINT(30.133581617816724 -94.76839551900616) bank61120 +61121 POINT(30.241820083891866 -96.14834409744846) bank61121 +61122 POINT(30.3297736897305 -95.7207022212087) bank61122 +61123 POINT(30.433070064711305 -95.070374545258) bank61123 +61124 POINT(29.5234633102923 -94.88003401249097) bank61124 +61125 POINT(29.03135296693383 -95.3790883989684) bank61125 +61126 POINT(30.08647440474564 -94.76119379827207) bank61126 +61127 POINT(29.340855848759272 -95.52295360190575) bank61127 +61128 POINT(29.98233496578333 -95.84129886455044) bank61128 +61129 POINT(29.915092942284026 -96.26987478224461) bank61129 +61130 POINT(30.72112411613396 -95.92265653070268) bank61130 +61131 POINT(30.137007946811682 -95.98739032660467) bank61131 +61132 POINT(30.569374152786448 -94.61943186249617) bank61132 +61133 POINT(30.532191016757395 -95.12591479184677) bank61133 +61134 POINT(29.10860068331872 -94.85220970337566) bank61134 +61135 POINT(29.42380945422126 -94.53472055395144) bank61135 +61136 POINT(30.472135765046975 -95.68391768880205) bank61136 +61137 POINT(30.185163441983104 -95.90208178682462) bank61137 +61138 POINT(28.806121023493706 -96.36308249581569) bank61138 +61139 POINT(30.116582847149406 -94.39460654372243) bank61139 +61140 POINT(29.235382982451537 -96.24446284760147) bank61140 +61141 POINT(30.481089600385722 -95.85905213692615) bank61141 +61142 POINT(29.440925573492017 -94.83807834812143) bank61142 +61143 POINT(29.978891182803807 -95.47824886315273) bank61143 +61144 POINT(30.48599757339169 -95.05691805883235) bank61144 +61145 POINT(29.47738499929529 -96.04106437064448) bank61145 +61146 POINT(28.99568099927097 -95.04892431470772) bank61146 +61147 POINT(30.214222054833144 -94.96682424409403) bank61147 +61148 POINT(30.409034147669004 -95.59181669500023) bank61148 +61149 POINT(29.629704350503584 -95.08639779681576) bank61149 +61150 POINT(29.454826602774986 -94.76444764675601) bank61150 +61151 POINT(29.996577495016272 -94.84647654732777) bank61151 +61152 POINT(29.833998332349385 -95.51213392321371) bank61152 +61153 POINT(30.022763369640707 -95.03631700336948) bank61153 +61154 POINT(29.40131702203176 -94.45880285469669) bank61154 +61155 POINT(28.8958502090737 -94.6389579543713) bank61155 +61156 POINT(29.148262012096104 -95.10500226049601) bank61156 +61157 POINT(28.86464542649183 -94.4283342462189) bank61157 +61158 POINT(30.226472650021723 -95.58305516882369) bank61158 +61159 POINT(29.32120769541637 -95.9178203166911) bank61159 +61160 POINT(28.7649851022181 -96.29157021769073) bank61160 +61161 POINT(28.839027860446464 -95.38604358272958) bank61161 +61162 POINT(29.51182579752357 -95.19278976549883) bank61162 +61163 POINT(30.527491785475704 -95.27926059032562) bank61163 +61164 POINT(29.332771382142244 -94.75277312782814) bank61164 +61165 POINT(30.58830661664191 -95.1836848534164) bank61165 +61166 POINT(29.038476627237046 -96.04070169403695) bank61166 +61167 POINT(30.17300808908683 -94.49125885426334) bank61167 +61168 POINT(28.76781486962562 -94.68680220074549) bank61168 +61169 POINT(30.341236833081734 -94.73240200797314) bank61169 +61170 POINT(30.42546792275241 -95.9556243260812) bank61170 +61171 POINT(29.029474729250822 -95.56416384901449) bank61171 +61172 POINT(30.576954066435256 -96.06470746372693) bank61172 +61173 POINT(30.356553960131414 -96.05402606142032) bank61173 +61174 POINT(30.185038416566147 -95.02111048815617) bank61174 +61175 POINT(29.8289307865706 -95.05412575941811) bank61175 +61176 POINT(30.250912885610497 -95.11786528383305) bank61176 +61177 POINT(30.084914676259345 -94.50421269782906) bank61177 +61178 POINT(29.991205152405847 -95.3973105763457) bank61178 +61179 POINT(29.965242131722864 -96.33854554188457) bank61179 +61180 POINT(29.646670372613915 -95.35774472183456) bank61180 +61181 POINT(30.68933435828353 -94.81073742181839) bank61181 +61182 POINT(28.80560185152976 -95.58014946589117) bank61182 +61183 POINT(28.819039808214487 -94.76915083736004) bank61183 +61184 POINT(29.01601374102234 -94.3837024508348) bank61184 +61185 POINT(29.19349952272473 -95.19221370468013) bank61185 +61186 POINT(29.34844991950253 -95.59285844868) bank61186 +61187 POINT(28.81910089390057 -95.36217721979885) bank61187 +61188 POINT(29.095706661733228 -96.22383227778583) bank61188 +61189 POINT(29.46278131078936 -96.09881057253881) bank61189 +61190 POINT(30.203230945074242 -94.78338241279064) bank61190 +61191 POINT(29.930650654398267 -95.86941940513852) bank61191 +61192 POINT(28.938110257746402 -96.31763601510765) bank61192 +61193 POINT(29.54195687639026 -95.84062444166555) bank61193 +61194 POINT(29.126143955935056 -95.77923155660014) bank61194 +61195 POINT(29.106506149372137 -94.76644898812079) bank61195 +61196 POINT(30.562452445662757 -94.50203631173638) bank61196 +61197 POINT(30.405435046099775 -95.5907366038938) bank61197 +61198 POINT(29.549813873264174 -96.1887564873313) bank61198 +61199 POINT(29.734234523423197 -95.32524642624493) bank61199 +61200 POINT(28.82827864534331 -95.20582186990394) bank61200 +61201 POINT(29.261274009251142 -95.47589500295953) bank61201 +61202 POINT(29.832672134500104 -94.81691171145522) bank61202 +61203 POINT(30.23824403056051 -95.14266084413981) bank61203 +61204 POINT(30.47029138948324 -94.42953923811675) bank61204 +61205 POINT(30.54791266988588 -94.80003164367717) bank61205 +61206 POINT(30.22135012778154 -96.10802643913173) bank61206 +61207 POINT(29.076026588851438 -96.01058982303705) bank61207 +61208 POINT(30.50754828351791 -94.74305430690946) bank61208 +61209 POINT(30.542431208294555 -95.8143993169887) bank61209 +61210 POINT(29.42841719509304 -96.05670651846076) bank61210 +61211 POINT(30.58890909312472 -95.93596285347985) bank61211 +61212 POINT(28.868075562868118 -94.57835615376544) bank61212 +61213 POINT(29.960488916361044 -95.89688106876383) bank61213 +61214 POINT(29.732622130100253 -94.55109524661631) bank61214 +61215 POINT(29.80273271127116 -94.40897516533434) bank61215 +61216 POINT(28.96563570612132 -95.79518037453347) bank61216 +61217 POINT(29.805719783437386 -94.46649740292965) bank61217 +61218 POINT(29.016889938287832 -94.92930853566887) bank61218 +61219 POINT(30.420894895492765 -95.4735760977502) bank61219 +61220 POINT(29.57075553459659 -95.68223506083896) bank61220 +61221 POINT(30.56125088797997 -96.11233674151707) bank61221 +61222 POINT(30.19449227247878 -94.51668285725955) bank61222 +61223 POINT(29.974150406964817 -95.00151534748458) bank61223 +61224 POINT(29.524243411685163 -94.91348892531092) bank61224 +61225 POINT(29.12493749565629 -96.3027115011767) bank61225 +61226 POINT(30.144661593216124 -94.95458755515018) bank61226 +61227 POINT(29.58997327533726 -95.75150907275645) bank61227 +61228 POINT(29.16457851356428 -96.29573844030011) bank61228 +61229 POINT(30.290487402951243 -95.97071076829621) bank61229 +61230 POINT(30.449965445492346 -95.29238355138725) bank61230 +61231 POINT(29.908255855299355 -94.49769588866089) bank61231 +61232 POINT(30.687025228033157 -96.19534955386116) bank61232 +61233 POINT(29.540577842349872 -95.91032167414444) bank61233 +61234 POINT(28.893036721090713 -94.43734528524831) bank61234 +61235 POINT(30.62228231183926 -94.84599441047186) bank61235 +61236 POINT(30.028616543514158 -95.62430407257698) bank61236 +61237 POINT(28.976971562766742 -96.16561946918277) bank61237 +61238 POINT(29.651647120073847 -95.23233071748562) bank61238 +61239 POINT(29.418329323652312 -94.92175506860379) bank61239 +61240 POINT(28.77442601360981 -94.96417670803744) bank61240 +61241 POINT(29.54150668824421 -95.32587862271751) bank61241 +61242 POINT(28.850350513978086 -94.4448500458334) bank61242 +61243 POINT(29.773683864952115 -95.19561721212187) bank61243 +61244 POINT(29.174467617603884 -95.41132364583028) bank61244 +61245 POINT(29.976670831019767 -95.33622237268781) bank61245 +61246 POINT(29.166027348522185 -94.5677229345609) bank61246 +61247 POINT(29.06046653994047 -95.52153279514152) bank61247 +61248 POINT(29.13662277128095 -96.05883141085293) bank61248 +61249 POINT(28.821663917140704 -94.92207777081974) bank61249 +61250 POINT(29.947529091616182 -94.6287416746308) bank61250 +61251 POINT(29.285565008644678 -95.70603326378139) bank61251 +61252 POINT(30.41342338251522 -94.98043949200257) bank61252 +61253 POINT(29.28604610090532 -95.70143044542523) bank61253 +61254 POINT(30.376702877823938 -96.08159162302223) bank61254 +61255 POINT(29.080986797434296 -94.7670462866809) bank61255 +61256 POINT(30.271639154189156 -94.4049878717178) bank61256 +61257 POINT(29.14046265841368 -96.3182554101512) bank61257 +61258 POINT(30.727738585714967 -95.69076639080119) bank61258 +61259 POINT(29.122188838928025 -96.06751906653672) bank61259 +61260 POINT(29.704743570555436 -95.07334888778092) bank61260 +61261 POINT(30.466648830739363 -95.4738320343308) bank61261 +61262 POINT(29.52376574138659 -95.87217398059506) bank61262 +61263 POINT(29.93403278923246 -95.3856752471899) bank61263 +61264 POINT(29.790725459076192 -96.08850896504143) bank61264 +61265 POINT(29.626582654393527 -96.35876136141557) bank61265 +61266 POINT(30.521549631108098 -96.1639706631197) bank61266 +61267 POINT(29.350792995751277 -96.07121623945642) bank61267 +61268 POINT(30.549405678834486 -95.46073927288234) bank61268 +61269 POINT(29.603676334773482 -95.62387483516973) bank61269 +61270 POINT(29.31345324362944 -95.08893140127357) bank61270 +61271 POINT(29.02996201164061 -94.3813494416568) bank61271 +61272 POINT(30.519829609725754 -95.83210243218069) bank61272 +61273 POINT(30.74820726676386 -94.72380916212164) bank61273 +61274 POINT(29.93344836907102 -95.41816645057534) bank61274 +61275 POINT(30.696950296405298 -96.05466919557847) bank61275 +61276 POINT(28.78354170885817 -96.34756273065014) bank61276 +61277 POINT(30.364898222988643 -95.8974467683812) bank61277 +61278 POINT(30.5768637345966 -95.74056203825303) bank61278 +61279 POINT(30.057255767079326 -94.65076227684506) bank61279 +61280 POINT(30.734755509827323 -95.30611272975129) bank61280 +61281 POINT(28.96208495150898 -95.75870306457999) bank61281 +61282 POINT(30.020623857024702 -96.16276776072996) bank61282 +61283 POINT(29.410195016333304 -94.4742528837655) bank61283 +61284 POINT(29.472694276270257 -94.81124722918253) bank61284 +61285 POINT(29.40287817150365 -95.67355869254878) bank61285 +61286 POINT(28.881053870384346 -95.14769627087347) bank61286 +61287 POINT(30.456547932939877 -94.57301825639874) bank61287 +61288 POINT(30.19041114319396 -95.41528237459593) bank61288 +61289 POINT(30.110079998270468 -94.74576064970734) bank61289 +61290 POINT(30.757551900933258 -94.39607591491786) bank61290 +61291 POINT(30.071055483158673 -94.96190774786086) bank61291 +61292 POINT(30.12970400699802 -95.74516272262588) bank61292 +61293 POINT(30.301731780565376 -95.25419564226618) bank61293 +61294 POINT(30.629810181818307 -95.5311765296153) bank61294 +61295 POINT(28.91345026114816 -95.4917370881566) bank61295 +61296 POINT(30.161048117895113 -95.69810475766673) bank61296 +61297 POINT(29.811657376311686 -94.68691380365331) bank61297 +61298 POINT(28.811537816168368 -95.1922911410345) bank61298 +61299 POINT(29.725080111628966 -95.95399708931562) bank61299 +61300 POINT(29.734759884236876 -94.57081993334265) bank61300 +61301 POINT(29.41027556943799 -96.16485306594905) bank61301 +61302 POINT(30.50293292406739 -94.47031222572537) bank61302 +61303 POINT(29.035469578484758 -95.50513882492655) bank61303 +61304 POINT(30.348887006664455 -94.81969057763439) bank61304 +61305 POINT(30.60400923154614 -94.39329005949605) bank61305 +61306 POINT(29.25383532798931 -94.43979934964956) bank61306 +61307 POINT(29.919520417566186 -96.2416336807931) bank61307 +61308 POINT(28.817916828833294 -95.80095532873491) bank61308 +61309 POINT(29.923550194760942 -95.52348886238849) bank61309 +61310 POINT(30.02238000506024 -94.56104999023985) bank61310 +61311 POINT(29.852218533370262 -95.51341641828905) bank61311 +61312 POINT(29.986091935244342 -95.50393941693234) bank61312 +61313 POINT(29.283716268682415 -94.61111377114104) bank61313 +61314 POINT(29.760180857142878 -95.93419823710721) bank61314 +61315 POINT(29.200657549507717 -96.27596953822905) bank61315 +61316 POINT(28.828538048618793 -95.55325334772144) bank61316 +61317 POINT(29.351309141455665 -95.08513026423482) bank61317 +61318 POINT(29.393667514045248 -95.16494091484722) bank61318 +61319 POINT(29.956183733643336 -96.0433839632548) bank61319 +61320 POINT(30.6851795180647 -94.86131663401012) bank61320 +61321 POINT(29.604843557123175 -94.44116585755314) bank61321 +61322 POINT(30.71212069680129 -95.45338683642639) bank61322 +61323 POINT(30.13763024800314 -96.28711322237584) bank61323 +61324 POINT(29.598199831109508 -95.91518334070086) bank61324 +61325 POINT(29.780715032956373 -94.96706351093111) bank61325 +61326 POINT(29.992192867941426 -95.74720419617123) bank61326 +61327 POINT(30.47638333234898 -94.98131690457882) bank61327 +61328 POINT(28.95167451915142 -95.57299641495906) bank61328 +61329 POINT(29.803593065728577 -95.01599597927843) bank61329 +61330 POINT(29.172925915535124 -94.92629392233768) bank61330 +61331 POINT(29.98494776814236 -94.54593527212943) bank61331 +61332 POINT(29.032967415429617 -95.8837455360178) bank61332 +61333 POINT(30.11643990867685 -96.12725969045641) bank61333 +61334 POINT(28.876258694979395 -95.41305055190516) bank61334 +61335 POINT(29.66406138992518 -94.41363438868237) bank61335 +61336 POINT(30.01381705227255 -96.2265484580275) bank61336 +61337 POINT(28.948438876973665 -95.61632482925086) bank61337 +61338 POINT(30.003112279931162 -96.1221753929091) bank61338 +61339 POINT(28.90828173560656 -95.15231383406514) bank61339 +61340 POINT(29.40841350109284 -95.58836675997856) bank61340 +61341 POINT(29.846560588959726 -95.70739827201876) bank61341 +61342 POINT(30.010437846094256 -94.97613042717475) bank61342 +61343 POINT(30.20810789928023 -94.63093350138782) bank61343 +61344 POINT(28.899665030579385 -95.98101809675937) bank61344 +61345 POINT(29.569172450874653 -95.92129684738956) bank61345 +61346 POINT(29.102701817625167 -94.82108488673701) bank61346 +61347 POINT(29.79325499895083 -96.09145246880492) bank61347 +61348 POINT(30.634040366701615 -94.41552765956891) bank61348 +61349 POINT(29.044294330307018 -95.60449216859655) bank61349 +61350 POINT(29.900048003859823 -95.08719237927004) bank61350 +61351 POINT(30.01197232506575 -94.57895443248101) bank61351 +61352 POINT(29.076780359432092 -95.86852824580079) bank61352 +61353 POINT(28.79803375405595 -94.92313541464245) bank61353 +61354 POINT(30.012736764829857 -94.75969365642317) bank61354 +61355 POINT(30.08457975555523 -94.59165675231067) bank61355 +61356 POINT(29.61437789996978 -95.75715230870996) bank61356 +61357 POINT(29.103713126720802 -95.06193591683501) bank61357 +61358 POINT(30.3226048310447 -96.13202154408418) bank61358 +61359 POINT(30.108082497412997 -94.37322671353445) bank61359 +61360 POINT(29.506697721415208 -95.67211153493746) bank61360 +61361 POINT(29.58696142699727 -96.23655670368046) bank61361 +61362 POINT(30.27560670121709 -94.7383200263173) bank61362 +61363 POINT(29.83093846056855 -94.48516889577863) bank61363 +61364 POINT(29.944058234402387 -94.54823701638057) bank61364 +61365 POINT(30.259554043648123 -95.09765689479488) bank61365 +61366 POINT(29.073250059307387 -95.06435604683797) bank61366 +61367 POINT(30.514485128980525 -95.99045738217329) bank61367 +61368 POINT(30.179190539108475 -95.09081031973975) bank61368 +61369 POINT(29.024573349313577 -95.54656042519692) bank61369 +61370 POINT(30.69521498937179 -95.86124515491802) bank61370 +61371 POINT(30.369032396061026 -94.77900835775675) bank61371 +61372 POINT(29.47604608018802 -94.93699547150923) bank61372 +61373 POINT(30.27598500234243 -94.57475718033251) bank61373 +61374 POINT(30.101097689420726 -95.45586925537036) bank61374 +61375 POINT(29.53331742286627 -96.27583904069729) bank61375 +61376 POINT(29.563242897908744 -95.45881392036152) bank61376 +61377 POINT(30.732017504057684 -95.61994148144069) bank61377 +61378 POINT(30.52323886380569 -94.96983451165765) bank61378 +61379 POINT(29.28548117299117 -94.57172938331702) bank61379 +61380 POINT(28.83625311814806 -94.87124372148176) bank61380 +61381 POINT(29.447717346030963 -95.99296865859897) bank61381 +61382 POINT(29.235107677085622 -95.25033430218986) bank61382 +61383 POINT(30.60097204606422 -96.2224729120729) bank61383 +61384 POINT(30.551108682336167 -95.71511883471409) bank61384 +61385 POINT(30.27111082671152 -95.95177059876586) bank61385 +61386 POINT(29.200511429736316 -96.27225997072811) bank61386 +61387 POINT(29.181262580814103 -95.67546388614512) bank61387 +61388 POINT(29.26850466807117 -95.88712819586965) bank61388 +61389 POINT(29.173897687859142 -95.06351530680453) bank61389 +61390 POINT(30.110496168664778 -94.41605698543324) bank61390 +61391 POINT(30.65388033870316 -94.78275866476075) bank61391 +61392 POINT(29.218966967609536 -96.17384356197546) bank61392 +61393 POINT(28.922848908915537 -95.81515942041837) bank61393 +61394 POINT(30.64007319690189 -96.04680876992575) bank61394 +61395 POINT(29.74333369970057 -95.82639852972153) bank61395 +61396 POINT(28.9599603145315 -95.41753776623072) bank61396 +61397 POINT(30.126647945613016 -95.27563233495673) bank61397 +61398 POINT(29.768511740928684 -95.82358636668732) bank61398 +61399 POINT(29.112324995920872 -94.76382276457127) bank61399 +61400 POINT(28.814056297340386 -96.14822865886568) bank61400 +61401 POINT(29.490314764729458 -96.1284981457244) bank61401 +61402 POINT(29.17764652560246 -95.51481177593584) bank61402 +61403 POINT(30.450546937464622 -95.82936196708748) bank61403 +61404 POINT(30.739368478286565 -94.91611737641912) bank61404 +61405 POINT(29.662336670837323 -94.7361327576757) bank61405 +61406 POINT(28.807091522316675 -95.97058855962686) bank61406 +61407 POINT(30.64333682501076 -95.5366256832479) bank61407 +61408 POINT(30.50808654625841 -95.10975122231015) bank61408 +61409 POINT(28.87138179537761 -95.47956456329096) bank61409 +61410 POINT(29.078205316393486 -95.30297702797552) bank61410 +61411 POINT(29.067577213788944 -96.14218902198057) bank61411 +61412 POINT(28.862727567807898 -95.54903199406688) bank61412 +61413 POINT(29.068805337218464 -96.22996402429922) bank61413 +61414 POINT(30.173543347417684 -94.5156614374256) bank61414 +61415 POINT(30.731976336497805 -95.35958738229739) bank61415 +61416 POINT(30.5275941056697 -94.95447282326239) bank61416 +61417 POINT(29.904133709522473 -95.42673126153701) bank61417 +61418 POINT(28.88516522024548 -96.04965164195471) bank61418 +61419 POINT(29.834585216720885 -95.96679864249552) bank61419 +61420 POINT(29.862197767996555 -95.49512886749636) bank61420 +61421 POINT(30.219925467539333 -94.74046338652408) bank61421 +61422 POINT(30.652132799315215 -96.03404536174986) bank61422 +61423 POINT(30.744855356015695 -95.4064905561709) bank61423 +61424 POINT(30.288610833913324 -94.8324574402564) bank61424 +61425 POINT(30.21739975966026 -94.64754296490108) bank61425 +61426 POINT(29.299135995535053 -95.27706672415046) bank61426 +61427 POINT(30.53677147952004 -95.64682252255238) bank61427 +61428 POINT(30.39940817235091 -94.41139803179493) bank61428 +61429 POINT(30.72884352911452 -95.52764776656326) bank61429 +61430 POINT(29.54554105687139 -94.41859844603523) bank61430 +61431 POINT(29.925786358305015 -95.12066772620763) bank61431 +61432 POINT(30.61480000829221 -95.98291330911097) bank61432 +61433 POINT(29.933138084725496 -96.25612152376591) bank61433 +61434 POINT(30.311922732580438 -96.01939887375985) bank61434 +61435 POINT(29.880299658769662 -95.5583122868829) bank61435 +61436 POINT(29.325873940318047 -95.28164385213852) bank61436 +61437 POINT(30.628743477852517 -94.85323384130834) bank61437 +61438 POINT(30.491639149203678 -95.13787601154283) bank61438 +61439 POINT(29.390122245622894 -94.43269988485842) bank61439 +61440 POINT(29.023165141577906 -95.86128333229328) bank61440 +61441 POINT(29.43009113056458 -94.528589237163) bank61441 +61442 POINT(29.155085614422827 -95.40941628375353) bank61442 +61443 POINT(29.95456819744681 -95.00554166973079) bank61443 +61444 POINT(30.745621922333466 -96.03376168168462) bank61444 +61445 POINT(29.625197095748057 -95.23701983508447) bank61445 +61446 POINT(29.598116536669153 -95.89362844721636) bank61446 +61447 POINT(29.547521465046994 -94.43492187425763) bank61447 +61448 POINT(29.81934487612755 -95.38890215975499) bank61448 +61449 POINT(29.141550614221696 -95.51165071289954) bank61449 +61450 POINT(30.02465561018369 -95.20479127266542) bank61450 +61451 POINT(29.1618921504139 -94.56519109621769) bank61451 +61452 POINT(30.466958258487438 -94.48666469446512) bank61452 +61453 POINT(28.999646781012878 -95.06738109070449) bank61453 +61454 POINT(29.664880077227092 -95.17925807221204) bank61454 +61455 POINT(30.668439186022066 -94.38583949853215) bank61455 +61456 POINT(29.930837244283598 -95.87155363407584) bank61456 +61457 POINT(29.052240547580052 -95.17633748828774) bank61457 +61458 POINT(30.31180386351921 -94.8185047271723) bank61458 +61459 POINT(29.31338416949473 -95.25883976388603) bank61459 +61460 POINT(28.994351320596966 -95.65105283941243) bank61460 +61461 POINT(30.30723310103102 -95.56691627774651) bank61461 +61462 POINT(29.872933629892685 -95.63895861165813) bank61462 +61463 POINT(30.438918360889993 -94.51703805447423) bank61463 +61464 POINT(30.248108906033156 -95.45153626751389) bank61464 +61465 POINT(29.703719615631975 -96.05816757668838) bank61465 +61466 POINT(30.364279806686167 -95.63593073666594) bank61466 +61467 POINT(29.7739209105051 -95.38793112097517) bank61467 +61468 POINT(30.383832078619275 -95.90202936107487) bank61468 +61469 POINT(30.60894433644495 -94.97779268163404) bank61469 +61470 POINT(29.16578545280371 -95.22030245649333) bank61470 +61471 POINT(30.36409926511307 -95.14487680490394) bank61471 +61472 POINT(30.736138056702707 -95.1215440896682) bank61472 +61473 POINT(29.09201925522377 -94.55170320524934) bank61473 +61474 POINT(30.50377646867866 -94.60702368873037) bank61474 +61475 POINT(29.80279207356971 -95.29133617665184) bank61475 +61476 POINT(30.202132305572505 -95.64639030152377) bank61476 +61477 POINT(29.58761146045454 -94.56433925192763) bank61477 +61478 POINT(30.341381955575198 -95.11056790941011) bank61478 +61479 POINT(29.464193775172625 -94.92239927516957) bank61479 +61480 POINT(30.128842036790893 -96.2290434010831) bank61480 +61481 POINT(28.85412768113077 -95.08772593260069) bank61481 +61482 POINT(29.223836606347103 -94.78292248801198) bank61482 +61483 POINT(29.829973052949057 -95.49546979479378) bank61483 +61484 POINT(29.580591680958463 -94.78954471623656) bank61484 +61485 POINT(29.39859544551769 -94.38322398460849) bank61485 +61486 POINT(29.424016952625074 -95.29222429433047) bank61486 +61487 POINT(29.682923841574944 -95.86135879779061) bank61487 +61488 POINT(30.740755333380726 -95.69127741781055) bank61488 +61489 POINT(29.090006291573726 -96.09137739156029) bank61489 +61490 POINT(30.447080844470282 -94.5323092505925) bank61490 +61491 POINT(28.765384820300014 -94.37050204323833) bank61491 +61492 POINT(29.188789798079366 -96.16004295845835) bank61492 +61493 POINT(28.910014751032413 -95.19077065256336) bank61493 +61494 POINT(29.31096246032968 -95.240045985274) bank61494 +61495 POINT(29.922378301626917 -95.1502523913051) bank61495 +61496 POINT(29.83057436603933 -95.06715263954322) bank61496 +61497 POINT(29.252041121672114 -94.77602249734664) bank61497 +61498 POINT(28.980490517449176 -94.78770494170665) bank61498 +61499 POINT(29.35892014048957 -95.88906196952816) bank61499 +61500 POINT(29.423955458789735 -94.91441574583483) bank61500 +61501 POINT(30.56646183465067 -96.07789936972017) bank61501 +61502 POINT(28.987957102255553 -94.71696955487083) bank61502 +61503 POINT(29.97527759613535 -94.82888635872303) bank61503 +61504 POINT(30.285150400165207 -96.11877106892337) bank61504 +61505 POINT(29.25591005688005 -94.48153685253956) bank61505 +61506 POINT(29.59885935533409 -95.50354582353063) bank61506 +61507 POINT(28.778606563636515 -96.04301951477842) bank61507 +61508 POINT(29.35463543792795 -95.56688383872094) bank61508 +61509 POINT(28.86731489291082 -95.81970284999431) bank61509 +61510 POINT(28.89212562423479 -95.85763246573921) bank61510 +61511 POINT(29.52169761898338 -96.15917802143858) bank61511 +61512 POINT(29.09791144473673 -95.72642745540442) bank61512 +61513 POINT(28.94791888465304 -95.86029428188668) bank61513 +61514 POINT(30.201777445903705 -95.68381957310291) bank61514 +61515 POINT(30.12698146135749 -95.30064684401027) bank61515 +61516 POINT(29.599626576568216 -94.42146839645173) bank61516 +61517 POINT(30.04388199509554 -95.55826492435342) bank61517 +61518 POINT(29.127916994390716 -95.0242080447787) bank61518 +61519 POINT(30.39626863021292 -94.9627504116348) bank61519 +61520 POINT(28.958941492248442 -95.60554635233494) bank61520 +61521 POINT(29.827992242136034 -94.4130967857334) bank61521 +61522 POINT(29.11810065915853 -94.77556485085628) bank61522 +61523 POINT(30.204359660806222 -95.06345141436549) bank61523 +61524 POINT(29.942082675997888 -95.04259585746092) bank61524 +61525 POINT(29.24846564057917 -96.1476632617266) bank61525 +61526 POINT(28.76566368370626 -95.44528944626938) bank61526 +61527 POINT(29.42398787993393 -95.28296241949036) bank61527 +61528 POINT(29.81555278683613 -95.18482515152341) bank61528 +61529 POINT(29.16528493678157 -95.92651819867625) bank61529 +61530 POINT(29.191616531132386 -96.20552729088789) bank61530 +61531 POINT(29.855516877458186 -96.11826787192913) bank61531 +61532 POINT(28.908184714067573 -94.55959751201621) bank61532 +61533 POINT(30.63775842296795 -96.29882323877153) bank61533 +61534 POINT(30.203238003819386 -96.1128244069227) bank61534 +61535 POINT(30.620254749139182 -96.00032346293992) bank61535 +61536 POINT(29.10717335524842 -96.10205167150419) bank61536 +61537 POINT(28.798368579695502 -95.69197083477546) bank61537 +61538 POINT(30.32423862582376 -94.38815634330872) bank61538 +61539 POINT(30.360946797705324 -96.17984864571925) bank61539 +61540 POINT(28.81359611367279 -96.0803367873145) bank61540 +61541 POINT(28.927668574591326 -95.61073313377086) bank61541 +61542 POINT(30.34752220764824 -94.7203148926552) bank61542 +61543 POINT(28.80261875814382 -95.90408934553122) bank61543 +61544 POINT(30.174100893222757 -96.25320975764629) bank61544 +61545 POINT(29.77329490755206 -94.5708588398113) bank61545 +61546 POINT(29.75668310422379 -94.69082599255307) bank61546 +61547 POINT(30.393737455219284 -95.19580720887684) bank61547 +61548 POINT(29.216314690486676 -94.8776374856015) bank61548 +61549 POINT(30.53381074182528 -96.27242116681782) bank61549 +61550 POINT(29.595533725068666 -95.42711669081807) bank61550 +61551 POINT(28.80937498983676 -95.820921122171) bank61551 +61552 POINT(29.36577120887577 -95.1814808190156) bank61552 +61553 POINT(29.76527948263849 -94.63749576479017) bank61553 +61554 POINT(30.672565827640454 -94.77495101484645) bank61554 +61555 POINT(29.556254690089364 -95.31416805441143) bank61555 +61556 POINT(30.453732914563403 -94.60206462611276) bank61556 +61557 POINT(28.899007846153964 -95.12435763044513) bank61557 +61558 POINT(30.546925750619394 -94.99308473320508) bank61558 +61559 POINT(30.061507007994454 -94.52181635430482) bank61559 +61560 POINT(30.201185085329517 -96.28515234995312) bank61560 +61561 POINT(28.968478031293245 -95.81381261178167) bank61561 +61562 POINT(29.759863399101146 -95.0282534413017) bank61562 +61563 POINT(30.7581870436255 -95.08597366745309) bank61563 +61564 POINT(30.222974611106867 -94.5128070821548) bank61564 +61565 POINT(30.416191041071205 -94.64824725464693) bank61565 +61566 POINT(30.514964023717223 -95.79918135628503) bank61566 +61567 POINT(30.076223545555244 -95.40500769313864) bank61567 +61568 POINT(28.96319668482646 -94.98943524940924) bank61568 +61569 POINT(29.310395969019016 -95.26278982438427) bank61569 +61570 POINT(30.69266917195614 -95.44763235762971) bank61570 +61571 POINT(29.47881759243189 -96.13461486062823) bank61571 +61572 POINT(29.754356615195658 -95.66863851056037) bank61572 +61573 POINT(29.894742593665555 -95.11531783575458) bank61573 +61574 POINT(28.955032865496825 -94.8278962913052) bank61574 +61575 POINT(30.34758041173203 -95.75318976676849) bank61575 +61576 POINT(30.553805263881014 -95.08932264941832) bank61576 +61577 POINT(29.705105929361107 -95.01074673988239) bank61577 +61578 POINT(30.537969493914463 -96.23388893554038) bank61578 +61579 POINT(29.509702705879597 -96.21319670252853) bank61579 +61580 POINT(30.50016391493803 -95.35781175702473) bank61580 +61581 POINT(29.977297692903033 -94.76403937894031) bank61581 +61582 POINT(29.702288962787748 -95.89779337334234) bank61582 +61583 POINT(29.388317882992176 -95.43825005286973) bank61583 +61584 POINT(29.860961052726527 -95.34919204445691) bank61584 +61585 POINT(30.73707537637888 -95.9334579673483) bank61585 +61586 POINT(30.390736529376753 -95.79824048628771) bank61586 +61587 POINT(29.22233044427781 -96.08195501827824) bank61587 +61588 POINT(30.219227072254288 -95.84897328313356) bank61588 +61589 POINT(29.04207641639251 -94.42868298913983) bank61589 +61590 POINT(30.47183122415542 -96.0665889545007) bank61590 +61591 POINT(29.276601501368944 -95.67349458174446) bank61591 +61592 POINT(30.391594844039314 -94.49077457658095) bank61592 +61593 POINT(29.959790572907853 -94.684250265221) bank61593 +61594 POINT(30.36835979736758 -96.28727707414609) bank61594 +61595 POINT(29.816445778731705 -94.74692601230333) bank61595 +61596 POINT(29.48379142349482 -94.57801722484756) bank61596 +61597 POINT(28.89793043861784 -95.46999711769506) bank61597 +61598 POINT(29.678332683223385 -94.66565713137328) bank61598 +61599 POINT(30.546690272141785 -94.84715206589935) bank61599 +61600 POINT(29.600780495939475 -96.23384845508262) bank61600 +61601 POINT(30.112191592580814 -94.57432428676522) bank61601 +61602 POINT(30.507090354454515 -94.42140501507703) bank61602 +61603 POINT(30.074182515238075 -96.11715067494676) bank61603 +61604 POINT(29.134136048993895 -94.93670436118359) bank61604 +61605 POINT(30.293249190451732 -95.06986057962304) bank61605 +61606 POINT(30.016005234756935 -94.62012847322738) bank61606 +61607 POINT(29.557024677813047 -94.95438992977702) bank61607 +61608 POINT(29.87964781253503 -95.71783308076651) bank61608 +61609 POINT(29.222898563381538 -94.90321542456542) bank61609 +61610 POINT(30.5393245204919 -94.92582226492107) bank61610 +61611 POINT(29.180698449824572 -95.1249360823233) bank61611 +61612 POINT(28.970988542956086 -95.33634534281751) bank61612 +61613 POINT(29.384841112399386 -95.8108935761331) bank61613 +61614 POINT(29.430573621582756 -94.97447250233984) bank61614 +61615 POINT(29.62355903545484 -95.46084536751003) bank61615 +61616 POINT(30.283914729964145 -95.10081785338022) bank61616 +61617 POINT(30.260389314196303 -95.57221217838047) bank61617 +61618 POINT(30.580937076043917 -96.26886295936389) bank61618 +61619 POINT(28.993158636068546 -95.554604057886) bank61619 +61620 POINT(29.239254384753306 -94.86125824446968) bank61620 +61621 POINT(29.199183674328218 -96.06281444338414) bank61621 +61622 POINT(29.22742417180482 -95.68435024967741) bank61622 +61623 POINT(29.601578739500486 -95.50549052185279) bank61623 +61624 POINT(28.95029139940606 -95.02382847864233) bank61624 +61625 POINT(30.748906567878443 -96.10731970999728) bank61625 +61626 POINT(28.91980841392101 -95.1746951415606) bank61626 +61627 POINT(30.093787591053868 -94.47609558595683) bank61627 +61628 POINT(29.840743728201797 -94.37362994935823) bank61628 +61629 POINT(29.942744963274425 -96.0936404554322) bank61629 +61630 POINT(29.403572186406244 -95.24464857234236) bank61630 +61631 POINT(29.395987633800132 -95.2987277723552) bank61631 +61632 POINT(28.939454993744786 -95.78363681626222) bank61632 +61633 POINT(29.07456455694456 -96.23008608574122) bank61633 +61634 POINT(29.856680693286666 -94.88179865386176) bank61634 +61635 POINT(28.888358516518196 -95.10219018024756) bank61635 +61636 POINT(30.169735235368396 -94.40727522638655) bank61636 +61637 POINT(29.59238489416052 -96.04853046055575) bank61637 +61638 POINT(28.878579261889215 -96.22215766204427) bank61638 +61639 POINT(30.369733765485567 -94.75099726258705) bank61639 +61640 POINT(30.56345033472774 -95.01390266866109) bank61640 +61641 POINT(29.577280267299102 -96.18927587717207) bank61641 +61642 POINT(29.322594919166985 -95.52210730674264) bank61642 +61643 POINT(30.36127195945238 -96.1327453754507) bank61643 +61644 POINT(30.21353596736304 -95.62307912319065) bank61644 +61645 POINT(30.185766522717884 -94.42059760886356) bank61645 +61646 POINT(29.42384381574945 -95.1045529117571) bank61646 +61647 POINT(30.31781219250568 -94.43894482059868) bank61647 +61648 POINT(30.617781240321655 -94.63012651849544) bank61648 +61649 POINT(30.10184181860078 -94.94902609372559) bank61649 +61650 POINT(30.11567598427796 -95.93028557407398) bank61650 +61651 POINT(29.33426478766977 -95.08083459033308) bank61651 +61652 POINT(29.764626620227805 -96.21515693829858) bank61652 +61653 POINT(29.34104748136053 -95.4029581461572) bank61653 +61654 POINT(30.71763666047142 -94.43961707318566) bank61654 +61655 POINT(30.69033066984505 -96.052412069797) bank61655 +61656 POINT(29.01297933391739 -95.23805934746468) bank61656 +61657 POINT(29.45504327967021 -96.33060191953028) bank61657 +61658 POINT(30.066553925975896 -94.60773217506075) bank61658 +61659 POINT(28.82054126142081 -96.28785323659356) bank61659 +61660 POINT(29.285359282089516 -94.44691580813046) bank61660 +61661 POINT(30.389948648494784 -96.03779663139973) bank61661 +61662 POINT(29.388080756112686 -94.6076940605755) bank61662 +61663 POINT(28.832999539092466 -96.06091786761694) bank61663 +61664 POINT(29.953235604478948 -95.1128135474106) bank61664 +61665 POINT(29.185719690103447 -94.92700935131919) bank61665 +61666 POINT(29.189523245744965 -95.16805533076395) bank61666 +61667 POINT(30.36022162888036 -95.23028092141507) bank61667 +61668 POINT(30.257563329746056 -95.84598147580877) bank61668 +61669 POINT(29.84978274891944 -95.44092742958745) bank61669 +61670 POINT(29.256986361256164 -94.48569189826223) bank61670 +61671 POINT(29.37558053341133 -96.3557807434432) bank61671 +61672 POINT(30.12215844035624 -95.76776793005799) bank61672 +61673 POINT(29.422206347492303 -95.17922824153955) bank61673 +61674 POINT(29.23004827018846 -95.2757790194145) bank61674 +61675 POINT(29.09439668923593 -96.11784980704196) bank61675 +61676 POINT(30.52763862752194 -95.52272033180215) bank61676 +61677 POINT(30.183671311452336 -94.56218343027838) bank61677 +61678 POINT(28.940055835079903 -96.0650465732341) bank61678 +61679 POINT(30.415277944534637 -95.92094377723532) bank61679 +61680 POINT(30.549656854718773 -95.18966384955282) bank61680 +61681 POINT(29.728385714802815 -96.21593343697931) bank61681 +61682 POINT(30.696978754059096 -95.5794495885357) bank61682 +61683 POINT(29.622289323544006 -96.35811471368089) bank61683 +61684 POINT(29.94279356292376 -95.07966595261898) bank61684 +61685 POINT(29.575262510833696 -95.209037153319) bank61685 +61686 POINT(29.336001518833708 -94.85369407511385) bank61686 +61687 POINT(29.51590392223641 -94.53313108947327) bank61687 +61688 POINT(30.38537837681815 -95.16217309657162) bank61688 +61689 POINT(29.669275250604446 -95.34289464489304) bank61689 +61690 POINT(30.70812998085369 -94.9727168328468) bank61690 +61691 POINT(29.09774490739054 -95.30811559976515) bank61691 +61692 POINT(28.990897132508024 -95.98339746732478) bank61692 +61693 POINT(30.723448192336207 -96.00332453283389) bank61693 +61694 POINT(29.89910993758279 -95.35499424089632) bank61694 +61695 POINT(29.72566376962864 -94.46532472911674) bank61695 +61696 POINT(29.886152172209908 -95.93556680806135) bank61696 +61697 POINT(30.457577780883152 -96.07101864178195) bank61697 +61698 POINT(30.142666808495616 -95.60569554133923) bank61698 +61699 POINT(29.867241672718905 -95.56378142313211) bank61699 +61700 POINT(29.068064273755148 -94.90709458823298) bank61700 +61701 POINT(28.942309609000496 -96.25331951104305) bank61701 +61702 POINT(28.933868792441462 -94.45320730519543) bank61702 +61703 POINT(30.00324660875761 -95.89811029435167) bank61703 +61704 POINT(30.275290577756373 -95.9840964118589) bank61704 +61705 POINT(30.625016061271964 -95.09286364036313) bank61705 +61706 POINT(30.074060095816197 -95.29374807741745) bank61706 +61707 POINT(29.29480588693913 -96.24519324530397) bank61707 +61708 POINT(29.379905342761138 -96.29629888118068) bank61708 +61709 POINT(28.873232356759022 -94.5254214939352) bank61709 +61710 POINT(29.960907135390997 -94.77535569626939) bank61710 +61711 POINT(30.21019845059748 -95.75532719540291) bank61711 +61712 POINT(30.435048210821517 -95.5498558209874) bank61712 +61713 POINT(29.949839211656624 -95.34682370728854) bank61713 +61714 POINT(28.794246634535774 -95.8823799042237) bank61714 +61715 POINT(29.32978454125599 -95.3380473183829) bank61715 +61716 POINT(29.998423329699516 -95.97602910162608) bank61716 +61717 POINT(29.11014811749289 -94.59728406168047) bank61717 +61718 POINT(30.375005959416807 -96.22975159387066) bank61718 +61719 POINT(30.076118067301593 -95.97732788286334) bank61719 +61720 POINT(29.39786754661324 -94.40278691913394) bank61720 +61721 POINT(30.164571079317806 -94.65997763126815) bank61721 +61722 POINT(28.90190258406825 -95.12266406079016) bank61722 +61723 POINT(29.19972316879092 -95.7483054695713) bank61723 +61724 POINT(30.20767135757394 -95.27761150008303) bank61724 +61725 POINT(30.368370581737057 -95.2275877731204) bank61725 +61726 POINT(29.0588559148224 -96.28912502859393) bank61726 +61727 POINT(29.200793327885442 -95.20389701068366) bank61727 +61728 POINT(30.70540315479094 -94.6690025355439) bank61728 +61729 POINT(29.71910863432006 -94.74174480975317) bank61729 +61730 POINT(30.689887288488396 -96.28494945707715) bank61730 +61731 POINT(29.247398655303233 -94.79997919539385) bank61731 +61732 POINT(29.47017245398976 -96.1087103579923) bank61732 +61733 POINT(29.65312588568798 -95.4139967027627) bank61733 +61734 POINT(29.284854431400483 -95.48272440532408) bank61734 +61735 POINT(30.20869890523693 -96.29538558765397) bank61735 +61736 POINT(28.79811837414421 -95.49704099592365) bank61736 +61737 POINT(29.791535229823303 -94.9422569560811) bank61737 +61738 POINT(29.710802082499985 -94.42646636541603) bank61738 +61739 POINT(29.091608490522532 -95.22615008711051) bank61739 +61740 POINT(29.890993505498905 -96.13942644466971) bank61740 +61741 POINT(30.727250949116275 -95.98095292631626) bank61741 +61742 POINT(30.2259968171013 -95.29845505958879) bank61742 +61743 POINT(29.948088094264868 -95.24787744345836) bank61743 +61744 POINT(29.126892072408296 -95.53942779726543) bank61744 +61745 POINT(29.29095052896585 -96.18882152979211) bank61745 +61746 POINT(30.315224193730558 -95.1406509178721) bank61746 +61747 POINT(29.495083532965257 -95.1748769415139) bank61747 +61748 POINT(28.980722581352794 -94.97438591512689) bank61748 +61749 POINT(28.784061468571974 -96.01442599997281) bank61749 +61750 POINT(30.486936962588988 -95.46819614312732) bank61750 +61751 POINT(30.015774442083178 -95.94884192938937) bank61751 +61752 POINT(29.41514595981852 -95.11472068889597) bank61752 +61753 POINT(29.185033151743347 -95.6351898203588) bank61753 +61754 POINT(29.25538351934908 -95.47751345383136) bank61754 +61755 POINT(30.143305422281244 -96.05461715075401) bank61755 +61756 POINT(29.61186867627285 -94.45931860161333) bank61756 +61757 POINT(28.979764427872944 -95.76144040828575) bank61757 +61758 POINT(30.011267853056392 -94.79864855519041) bank61758 +61759 POINT(30.323118478731875 -96.1426927549501) bank61759 +61760 POINT(29.882128937907055 -95.29779587994194) bank61760 +61761 POINT(30.597587486628644 -95.69521309637177) bank61761 +61762 POINT(29.62680030078875 -96.09187994295006) bank61762 +61763 POINT(29.29108551015969 -95.29771613173456) bank61763 +61764 POINT(28.9342719678676 -94.65905335223016) bank61764 +61765 POINT(29.918883764538837 -95.41446112013584) bank61765 +61766 POINT(29.172637299207366 -94.3978188426419) bank61766 +61767 POINT(28.82137234821506 -94.49402718265668) bank61767 +61768 POINT(30.070992881915064 -96.17129676860179) bank61768 +61769 POINT(28.96267727514556 -96.22162097928518) bank61769 +61770 POINT(30.039371304334004 -95.59171546232727) bank61770 +61771 POINT(28.870493541481327 -96.04244855699444) bank61771 +61772 POINT(29.349204388021302 -96.1114302652984) bank61772 +61773 POINT(30.370223202245175 -94.96961178928991) bank61773 +61774 POINT(29.79084996021618 -94.37259528457561) bank61774 +61775 POINT(30.5649413718108 -95.419709882206) bank61775 +61776 POINT(29.831048530179142 -94.73969029183488) bank61776 +61777 POINT(29.542391064797766 -94.49035342171479) bank61777 +61778 POINT(29.78517191818407 -94.92025179493598) bank61778 +61779 POINT(29.02914628310341 -95.41307522749766) bank61779 +61780 POINT(29.19893175452484 -95.51645833280722) bank61780 +61781 POINT(29.82942674928585 -94.49138276503942) bank61781 +61782 POINT(30.46822448297979 -96.33449737896868) bank61782 +61783 POINT(29.631478854199067 -95.31519744800326) bank61783 +61784 POINT(30.04789215421715 -95.8841388680485) bank61784 +61785 POINT(30.234455379031314 -94.44244131079157) bank61785 +61786 POINT(30.71511653329528 -95.19020476705684) bank61786 +61787 POINT(29.94654673666244 -95.67644025575947) bank61787 +61788 POINT(30.6074472330139 -94.41416904365617) bank61788 +61789 POINT(29.813517627947537 -95.37350127793914) bank61789 +61790 POINT(30.42786856869478 -95.33362914867232) bank61790 +61791 POINT(29.845960562699517 -96.09882750992834) bank61791 +61792 POINT(30.433595444130017 -96.28847180830705) bank61792 +61793 POINT(30.623886496275755 -95.68094095326414) bank61793 +61794 POINT(30.648171791334878 -94.83578414483698) bank61794 +61795 POINT(30.452723714366815 -94.80700094970797) bank61795 +61796 POINT(29.486842002177717 -95.42576985164004) bank61796 +61797 POINT(29.003119913851727 -96.18342554153394) bank61797 +61798 POINT(29.96737789445673 -94.63973226471262) bank61798 +61799 POINT(29.843805181830977 -95.1240217591865) bank61799 +61800 POINT(30.741783808724346 -96.30058971092654) bank61800 +61801 POINT(30.756597491872146 -96.08843133926118) bank61801 +61802 POINT(29.317616638047106 -96.14672210151356) bank61802 +61803 POINT(29.966419504077827 -96.21996403161273) bank61803 +61804 POINT(29.254661068456254 -95.64963370459859) bank61804 +61805 POINT(29.410149061955863 -94.67326010685667) bank61805 +61806 POINT(29.676999149269765 -95.43762137475083) bank61806 +61807 POINT(30.684895633344695 -95.60316003170823) bank61807 +61808 POINT(29.786619239354227 -95.6504149270102) bank61808 +61809 POINT(30.35489697091927 -94.65095039310894) bank61809 +61810 POINT(28.94747201075714 -94.6078260687938) bank61810 +61811 POINT(29.592984080200303 -95.63290164238362) bank61811 +61812 POINT(30.1976046036711 -94.58617116478129) bank61812 +61813 POINT(30.034344714257244 -96.28229231517392) bank61813 +61814 POINT(28.893516942834903 -95.70335435341858) bank61814 +61815 POINT(28.808103527891905 -94.66146318585628) bank61815 +61816 POINT(30.3114516788389 -94.76610953288512) bank61816 +61817 POINT(30.0844450379772 -95.88223108097657) bank61817 +61818 POINT(29.28325726624933 -95.62742523280396) bank61818 +61819 POINT(29.58228121139131 -96.08779255695053) bank61819 +61820 POINT(30.15487520744861 -94.72886801675953) bank61820 +61821 POINT(30.134372777522927 -95.89188062991249) bank61821 +61822 POINT(30.43126449003068 -94.52101068649003) bank61822 +61823 POINT(29.810318686085193 -95.10076358633782) bank61823 +61824 POINT(29.30260341746772 -95.0058512840843) bank61824 +61825 POINT(28.805904635861577 -94.69493912378078) bank61825 +61826 POINT(29.64051589299328 -94.97073371662852) bank61826 +61827 POINT(29.111353718207667 -95.14258019118915) bank61827 +61828 POINT(29.289725222421016 -95.76340565309206) bank61828 +61829 POINT(29.755420456187053 -96.07512767765051) bank61829 +61830 POINT(30.543241974741996 -96.31130405579503) bank61830 +61831 POINT(29.40949865883827 -94.60040252419) bank61831 +61832 POINT(29.860450208389594 -95.57140211086502) bank61832 +61833 POINT(30.37443346852717 -95.41520908876959) bank61833 +61834 POINT(29.430277260208683 -94.45369983131812) bank61834 +61835 POINT(30.619093188291423 -95.2100328664672) bank61835 +61836 POINT(30.1096984506492 -95.66039376224552) bank61836 +61837 POINT(29.25872310621889 -96.16642967442722) bank61837 +61838 POINT(29.363716174544706 -94.4044245042819) bank61838 +61839 POINT(29.00398895292219 -94.71175040258981) bank61839 +61840 POINT(29.914106559390707 -96.2682261246874) bank61840 +61841 POINT(28.95552657918483 -94.48709189319254) bank61841 +61842 POINT(30.616603606929637 -94.39928856544616) bank61842 +61843 POINT(29.2652288423631 -94.68432894084499) bank61843 +61844 POINT(29.57051346941025 -96.29924365159077) bank61844 +61845 POINT(29.44820421501768 -96.02781427822282) bank61845 +61846 POINT(29.78755976979995 -96.32397276239787) bank61846 +61847 POINT(29.040638434352214 -94.4784754607626) bank61847 +61848 POINT(29.373127150129417 -94.95970135987876) bank61848 +61849 POINT(29.166357556243128 -94.46949610213456) bank61849 +61850 POINT(28.942774042001513 -95.43360044174581) bank61850 +61851 POINT(29.34182189902385 -95.01236650386367) bank61851 +61852 POINT(29.930318137997677 -95.63308870891625) bank61852 +61853 POINT(28.843724511524343 -95.081167381126) bank61853 +61854 POINT(30.279995270222177 -96.25035027283953) bank61854 +61855 POINT(29.30415098743408 -96.02682767170218) bank61855 +61856 POINT(29.837846058290353 -95.07028505452107) bank61856 +61857 POINT(28.94593054899741 -95.6817055219833) bank61857 +61858 POINT(28.777590093883028 -95.93802919624372) bank61858 +61859 POINT(29.437381562641324 -94.94433468198474) bank61859 +61860 POINT(29.39886403904509 -96.02427136208988) bank61860 +61861 POINT(29.813373532247287 -96.04556789075762) bank61861 +61862 POINT(30.49848431410466 -95.66420474028622) bank61862 +61863 POINT(30.22881824766562 -94.81853532574428) bank61863 +61864 POINT(30.251505726135726 -96.13624657596881) bank61864 +61865 POINT(30.667804910129345 -96.03641721410219) bank61865 +61866 POINT(29.347989051414313 -94.41192204441694) bank61866 +61867 POINT(28.79129179325028 -95.86055965345228) bank61867 +61868 POINT(29.514641579621678 -96.3638861664461) bank61868 +61869 POINT(29.32769972486068 -95.85721816243833) bank61869 +61870 POINT(30.331937060564012 -95.85618751476058) bank61870 +61871 POINT(29.742821893460086 -95.8538342676674) bank61871 +61872 POINT(29.122350461496424 -94.56428249943036) bank61872 +61873 POINT(30.753313936220263 -95.81844914768496) bank61873 +61874 POINT(29.08580892115901 -94.60704647541398) bank61874 +61875 POINT(29.740803739661064 -95.7318664982992) bank61875 +61876 POINT(30.414009161307934 -94.51272594734908) bank61876 +61877 POINT(29.215007276052173 -95.75445592402312) bank61877 +61878 POINT(30.66287301909793 -95.83771160880474) bank61878 +61879 POINT(30.31230827704397 -94.74360887173513) bank61879 +61880 POINT(30.452692274369024 -95.00002355260357) bank61880 +61881 POINT(28.97437564052561 -96.2129742525284) bank61881 +61882 POINT(29.227187547334697 -94.58178942220944) bank61882 +61883 POINT(29.990138613012014 -95.77324674898469) bank61883 +61884 POINT(29.550049659924355 -95.4314743465524) bank61884 +61885 POINT(30.4324349811081 -95.22872507617538) bank61885 +61886 POINT(30.604469320495323 -94.83596953807945) bank61886 +61887 POINT(30.030872322396537 -96.32127938935596) bank61887 +61888 POINT(28.767570619745015 -96.09607905509723) bank61888 +61889 POINT(30.69173439247496 -95.4857388192109) bank61889 +61890 POINT(30.017275229286465 -96.32978189531092) bank61890 +61891 POINT(29.11399175586465 -96.33251076189899) bank61891 +61892 POINT(29.898651859851505 -95.2210297940666) bank61892 +61893 POINT(29.316472120916668 -96.26346446132233) bank61893 +61894 POINT(29.734297224122407 -96.25304382703372) bank61894 +61895 POINT(29.565785081528333 -95.8209078370224) bank61895 +61896 POINT(30.565430745622763 -94.80749490641848) bank61896 +61897 POINT(29.33455061893453 -94.48454610459251) bank61897 +61898 POINT(30.18180084597912 -94.41208473308251) bank61898 +61899 POINT(29.948654011176984 -94.85659800794303) bank61899 +61900 POINT(29.339687276515406 -95.49450609680342) bank61900 +61901 POINT(30.750453184532788 -95.01314029666187) bank61901 +61902 POINT(29.09254847747322 -95.77352451962237) bank61902 +61903 POINT(30.583113601993976 -94.47702528083668) bank61903 +61904 POINT(30.466731589456067 -96.10112689148966) bank61904 +61905 POINT(30.34730014261665 -94.65820218739634) bank61905 +61906 POINT(30.696482433958174 -96.09558479656472) bank61906 +61907 POINT(29.950147098068708 -94.74141050382889) bank61907 +61908 POINT(30.07858906775666 -95.9465042985146) bank61908 +61909 POINT(29.886535321983818 -95.37051060781398) bank61909 +61910 POINT(29.13561342464948 -94.448257408423) bank61910 +61911 POINT(30.266805018520916 -95.18123300990963) bank61911 +61912 POINT(30.50160764821259 -94.42487939215012) bank61912 +61913 POINT(29.888274083255777 -96.14520830655435) bank61913 +61914 POINT(30.480320103756675 -95.07153002496997) bank61914 +61915 POINT(30.402854293512934 -95.53107280122882) bank61915 +61916 POINT(29.9120121125533 -94.48677219867963) bank61916 +61917 POINT(29.17799548464325 -94.45056557611059) bank61917 +61918 POINT(28.963934687898814 -94.99963706378252) bank61918 +61919 POINT(28.999600352872026 -95.5907718869093) bank61919 +61920 POINT(30.53771298350988 -95.31177612983947) bank61920 +61921 POINT(30.707817613807403 -95.75367313416592) bank61921 +61922 POINT(29.810051025592266 -96.16162911454106) bank61922 +61923 POINT(30.61717084705028 -94.62939238491751) bank61923 +61924 POINT(28.782959149820588 -94.99720901313005) bank61924 +61925 POINT(28.928594070897837 -94.55727398283312) bank61925 +61926 POINT(29.310984513650006 -95.9387134006763) bank61926 +61927 POINT(29.948941641474704 -95.3323956241417) bank61927 +61928 POINT(30.292501106090526 -96.15919802978496) bank61928 +61929 POINT(29.297523184663962 -96.07826566871823) bank61929 +61930 POINT(29.812036922720658 -95.23463511295347) bank61930 +61931 POINT(29.71601728476033 -95.01484692535627) bank61931 +61932 POINT(30.08411377224403 -94.82200744626752) bank61932 +61933 POINT(29.3627333481527 -96.16595587942881) bank61933 +61934 POINT(30.5854628738239 -95.46177768829281) bank61934 +61935 POINT(28.880480321719137 -94.76793532181777) bank61935 +61936 POINT(29.50258636461739 -96.23434583487304) bank61936 +61937 POINT(29.693644337393383 -96.03098989071957) bank61937 +61938 POINT(30.330340345266908 -94.95754692921396) bank61938 +61939 POINT(30.56577209193109 -95.43055848002568) bank61939 +61940 POINT(30.10000698599512 -95.90575677117862) bank61940 +61941 POINT(29.10814453395143 -94.37310931562189) bank61941 +61942 POINT(30.699802500889305 -95.5546310982518) bank61942 +61943 POINT(29.019928478498578 -95.326148068097) bank61943 +61944 POINT(29.906560767933115 -96.15113853212424) bank61944 +61945 POINT(30.6258878547304 -96.05721013354622) bank61945 +61946 POINT(29.609682872509456 -95.03210761470741) bank61946 +61947 POINT(29.399682927690698 -95.46939365049022) bank61947 +61948 POINT(30.126635127131802 -95.86763782871517) bank61948 +61949 POINT(30.527214510297885 -94.78996467145718) bank61949 +61950 POINT(29.78051125403081 -95.4452476509504) bank61950 +61951 POINT(29.775928345450968 -95.77874390078753) bank61951 +61952 POINT(30.74800803312457 -94.6540649138775) bank61952 +61953 POINT(30.398941073726995 -94.76433198266643) bank61953 +61954 POINT(29.67063623192745 -94.79984403273211) bank61954 +61955 POINT(30.419425783894035 -95.53970148109242) bank61955 +61956 POINT(30.730616997598748 -95.52703780402709) bank61956 +61957 POINT(29.667716259337542 -95.73326761664578) bank61957 +61958 POINT(30.32968802347604 -96.16067124915746) bank61958 +61959 POINT(29.97612883320036 -95.32778269575242) bank61959 +61960 POINT(30.738302491137908 -94.45348987871004) bank61960 +61961 POINT(28.88773924415917 -94.56408671641692) bank61961 +61962 POINT(29.43524751131215 -94.63734737501542) bank61962 +61963 POINT(29.278014184467043 -95.80504573654937) bank61963 +61964 POINT(30.535364445899436 -94.77990697887513) bank61964 +61965 POINT(30.08721906829244 -95.54289731491609) bank61965 +61966 POINT(29.791832408364595 -96.06974319249628) bank61966 +61967 POINT(30.109999657979678 -96.16044157120176) bank61967 +61968 POINT(29.858137779198227 -96.21286404358545) bank61968 +61969 POINT(29.881613275383767 -94.73210363773471) bank61969 +61970 POINT(30.419877507856835 -95.42340097267738) bank61970 +61971 POINT(28.857895231792746 -96.03185605550973) bank61971 +61972 POINT(29.393144135739792 -95.67067731614272) bank61972 +61973 POINT(29.97286950214886 -95.05837143225948) bank61973 +61974 POINT(29.830884045189485 -96.29799855629508) bank61974 +61975 POINT(28.952680287936946 -95.64361711485566) bank61975 +61976 POINT(29.19367652232536 -95.50329490768996) bank61976 +61977 POINT(28.867955693362934 -94.74003976982135) bank61977 +61978 POINT(28.791265512395217 -95.30267750457936) bank61978 +61979 POINT(30.606266359266176 -95.68449364727608) bank61979 +61980 POINT(30.23073616883656 -95.58668876400412) bank61980 +61981 POINT(29.222316661721045 -94.65971435816444) bank61981 +61982 POINT(28.779865309170397 -95.45569892833797) bank61982 +61983 POINT(30.029737514513844 -96.00360279499964) bank61983 +61984 POINT(30.08312299867028 -96.00053464741957) bank61984 +61985 POINT(29.545658694022716 -95.67918824422699) bank61985 +61986 POINT(29.620810428539038 -94.4519136464197) bank61986 +61987 POINT(30.15703606997251 -95.73397344376365) bank61987 +61988 POINT(29.086649619237917 -94.55292825228334) bank61988 +61989 POINT(29.320930715258413 -95.66179538358989) bank61989 +61990 POINT(29.605376687601535 -94.81205434493438) bank61990 +61991 POINT(30.31333251497953 -95.63346161324608) bank61991 +61992 POINT(30.348463710747946 -96.23348625533045) bank61992 +61993 POINT(30.00522613337792 -94.53066824202656) bank61993 +61994 POINT(30.25457412330595 -95.28696213313987) bank61994 +61995 POINT(29.7557609194484 -95.30811801326772) bank61995 +61996 POINT(30.150262778535193 -95.48197585379476) bank61996 +61997 POINT(29.433348668474956 -95.15287944621596) bank61997 +61998 POINT(29.37819616200452 -95.3996991169759) bank61998 +61999 POINT(30.176287131632606 -95.20966020891721) bank61999 +62000 POINT(30.548996046934132 -94.37026306674518) bank62000 +62001 POINT(28.847756687230586 -94.64012972300307) bank62001 +62002 POINT(30.061061484735802 -95.49915621387031) bank62002 +62003 POINT(29.380461346615085 -94.94167107235586) bank62003 +62004 POINT(29.70924985302933 -95.4517176179023) bank62004 +62005 POINT(29.63981353153718 -95.84377285270982) bank62005 +62006 POINT(29.161022228067345 -95.8583118911311) bank62006 +62007 POINT(28.897632790215578 -94.47826985272692) bank62007 +62008 POINT(30.71098815580313 -95.31043080681938) bank62008 +62009 POINT(30.18998702386708 -96.29947153405719) bank62009 +62010 POINT(30.627229124092977 -95.01285520485197) bank62010 +62011 POINT(29.957798924824488 -95.48304337909767) bank62011 +62012 POINT(30.13184737294374 -96.31312370986682) bank62012 +62013 POINT(30.289565952989378 -95.55204901816815) bank62013 +62014 POINT(28.84950498690434 -96.17062561595931) bank62014 +62015 POINT(29.271177117384674 -94.64878959983564) bank62015 +62016 POINT(29.01099153613424 -95.61738203289723) bank62016 +62017 POINT(30.55598053671017 -95.55553213609568) bank62017 +62018 POINT(29.33904171166221 -95.30579489471022) bank62018 +62019 POINT(30.641388381632684 -95.25534385835743) bank62019 +62020 POINT(28.909516054155915 -96.27577267633126) bank62020 +62021 POINT(28.937738452443384 -94.39246061799491) bank62021 +62022 POINT(30.312305575031207 -95.53081187465449) bank62022 +62023 POINT(30.37984811930779 -95.26074088068424) bank62023 +62024 POINT(30.524483192158694 -95.35633390476363) bank62024 +62025 POINT(30.676210874248213 -95.61976790411569) bank62025 +62026 POINT(29.32324270035727 -95.71678786478508) bank62026 +62027 POINT(29.460556308077287 -96.31559724916451) bank62027 +62028 POINT(29.901540746890024 -94.39362654242639) bank62028 +62029 POINT(30.2421873461391 -96.21422140345685) bank62029 +62030 POINT(29.699423200493744 -95.57255526057342) bank62030 +62031 POINT(29.146001896677166 -96.31554732886494) bank62031 +62032 POINT(29.744335415601608 -95.21167058748011) bank62032 +62033 POINT(30.46964529529311 -94.91330799520475) bank62033 +62034 POINT(30.2752239595128 -95.56872423222191) bank62034 +62035 POINT(28.903106302133246 -94.90433334617694) bank62035 +62036 POINT(29.824145349277188 -95.04738814624365) bank62036 +62037 POINT(30.538658877733305 -95.43193826248094) bank62037 +62038 POINT(29.6070198469125 -94.72251577705242) bank62038 +62039 POINT(29.83980719828607 -95.13848952230234) bank62039 +62040 POINT(30.656131060754944 -94.91614021507856) bank62040 +62041 POINT(30.176619688243786 -94.4368322112055) bank62041 +62042 POINT(30.085205722536642 -94.58594407411002) bank62042 +62043 POINT(29.130999004246746 -95.35649534601725) bank62043 +62044 POINT(29.951274047662192 -94.52960819063442) bank62044 +62045 POINT(28.845102855660443 -95.66759737932318) bank62045 +62046 POINT(28.89545575970676 -95.17138904863394) bank62046 +62047 POINT(30.483681171785243 -95.16222499033502) bank62047 +62048 POINT(29.30637142302257 -94.46732958413627) bank62048 +62049 POINT(29.194471358129192 -94.54442274361281) bank62049 +62050 POINT(29.03516637630832 -95.84881747847352) bank62050 +62051 POINT(30.281139539488542 -96.05810235474145) bank62051 +62052 POINT(29.083281070538895 -95.91305294234171) bank62052 +62053 POINT(29.9764352466607 -95.58201526647272) bank62053 +62054 POINT(30.46866146570419 -95.80394556040198) bank62054 +62055 POINT(29.05089461822776 -96.26597346587067) bank62055 +62056 POINT(29.852516524351216 -95.66534914867277) bank62056 +62057 POINT(29.07531419279968 -95.2228210769159) bank62057 +62058 POINT(29.872395383509005 -96.13207869186479) bank62058 +62059 POINT(29.46872811499104 -94.53710943236588) bank62059 +62060 POINT(29.38450309403952 -96.251465756203) bank62060 +62061 POINT(29.232160531817346 -95.45590752594384) bank62061 +62062 POINT(30.198923300926793 -94.6507151569765) bank62062 +62063 POINT(30.271934971279318 -96.31713998164467) bank62063 +62064 POINT(30.426638101774945 -94.71131949795505) bank62064 +62065 POINT(30.571548248599626 -95.32545433393925) bank62065 +62066 POINT(30.327857285302436 -94.94474525355605) bank62066 +62067 POINT(30.07720328237349 -96.33886432435597) bank62067 +62068 POINT(30.74305013926849 -94.6075805672552) bank62068 +62069 POINT(29.243119648786923 -95.2165019736807) bank62069 +62070 POINT(29.78219074604941 -95.0329923873248) bank62070 +62071 POINT(29.164491506472583 -95.50484912165783) bank62071 +62072 POINT(30.478611463013564 -95.65461866124203) bank62072 +62073 POINT(30.32234901876882 -95.84622251902378) bank62073 +62074 POINT(29.191603525936944 -95.69011182115113) bank62074 +62075 POINT(29.15139524170539 -95.61188388276244) bank62075 +62076 POINT(28.993960879502144 -95.35610353808578) bank62076 +62077 POINT(30.40987874636912 -95.19651314809776) bank62077 +62078 POINT(30.101171301697825 -95.27458210910827) bank62078 +62079 POINT(30.52584566426354 -96.15171443141804) bank62079 +62080 POINT(30.51259173893311 -94.61685526064744) bank62080 +62081 POINT(28.975379169381846 -95.47388549779632) bank62081 +62082 POINT(30.58421504960479 -94.63337207438795) bank62082 +62083 POINT(29.47636819227705 -96.1592633709919) bank62083 +62084 POINT(29.90219802124001 -95.24444237774426) bank62084 +62085 POINT(30.106655405587276 -94.4183830272839) bank62085 +62086 POINT(30.0085765320024 -96.36796721406117) bank62086 +62087 POINT(28.989792424220617 -94.64669207480185) bank62087 +62088 POINT(30.567123002152574 -95.10740742578298) bank62088 +62089 POINT(30.392182911557036 -95.68634673774547) bank62089 +62090 POINT(29.52922322136415 -96.00692502536073) bank62090 +62091 POINT(29.58235148467509 -94.99614541031828) bank62091 +62092 POINT(30.57814039067263 -95.86634143269255) bank62092 +62093 POINT(30.64359516761436 -95.16203445495407) bank62093 +62094 POINT(29.823043833284604 -96.18611529428676) bank62094 +62095 POINT(30.442010657946643 -94.82962031406181) bank62095 +62096 POINT(29.125963198143836 -96.22997045833371) bank62096 +62097 POINT(29.261939951532234 -94.75072999535118) bank62097 +62098 POINT(28.958075596519784 -94.72322225304865) bank62098 +62099 POINT(28.78761869165017 -94.42257202337791) bank62099 +62100 POINT(29.069036290130793 -96.03293300111987) bank62100 +62101 POINT(28.984161486344 -94.71197050427921) bank62101 +62102 POINT(29.45278859755171 -95.58974679643285) bank62102 +62103 POINT(28.938848041254957 -94.4688823293386) bank62103 +62104 POINT(29.1697998997519 -95.67012711437188) bank62104 +62105 POINT(29.886155572469107 -96.03388083601698) bank62105 +62106 POINT(30.08019817292408 -95.07745243441916) bank62106 +62107 POINT(29.307180548483363 -95.51973573576065) bank62107 +62108 POINT(28.942741004838982 -95.82137831399115) bank62108 +62109 POINT(29.965386363100396 -96.000898234813) bank62109 +62110 POINT(28.823780840113585 -95.9245898827036) bank62110 +62111 POINT(29.343441628087092 -94.58885789586127) bank62111 +62112 POINT(29.187940167438672 -95.09004968223188) bank62112 +62113 POINT(29.82564780540288 -96.20674577154843) bank62113 +62114 POINT(29.741764886391973 -95.01114176031383) bank62114 +62115 POINT(29.420220734398775 -95.55583175195103) bank62115 +62116 POINT(29.02517336348558 -95.13944090763106) bank62116 +62117 POINT(29.001170375943584 -94.94157702637473) bank62117 +62118 POINT(29.2944070474285 -95.62772129868159) bank62118 +62119 POINT(29.40723754019551 -96.14332540950647) bank62119 +62120 POINT(29.496813629386146 -96.02948950084485) bank62120 +62121 POINT(28.82912767313738 -94.71395698664107) bank62121 +62122 POINT(28.815111461184003 -94.42844106187847) bank62122 +62123 POINT(29.18589618220111 -95.8555810256439) bank62123 +62124 POINT(29.826195763770862 -94.6738171977238) bank62124 +62125 POINT(29.545052632804715 -96.32006391815247) bank62125 +62126 POINT(29.768543945590423 -94.9204372201583) bank62126 +62127 POINT(28.921105535331286 -95.28745499040626) bank62127 +62128 POINT(29.094320058529405 -94.6884781532233) bank62128 +62129 POINT(29.755060422845723 -95.06447277597292) bank62129 +62130 POINT(28.7909994577793 -94.77357004734283) bank62130 +62131 POINT(30.71871256358333 -96.10227143441345) bank62131 +62132 POINT(29.25913029730758 -96.19419293986363) bank62132 +62133 POINT(30.428817823227938 -96.30682426812828) bank62133 +62134 POINT(29.80369660598522 -95.29231359865216) bank62134 +62135 POINT(28.88789347364949 -94.88560386232012) bank62135 +62136 POINT(29.095200987708264 -95.9694097937281) bank62136 +62137 POINT(30.338245252680295 -95.80436305437787) bank62137 +62138 POINT(30.216347089928526 -95.64229246235324) bank62138 +62139 POINT(29.533316964596523 -94.88440402953367) bank62139 +62140 POINT(29.7372042439388 -96.2136396603472) bank62140 +62141 POINT(30.09137352662821 -94.82252157430345) bank62141 +62142 POINT(28.810687038531025 -95.5259516862547) bank62142 +62143 POINT(30.753990573535305 -94.45763610288765) bank62143 +62144 POINT(29.490279913306995 -96.14611178383565) bank62144 +62145 POINT(29.97801995373748 -94.60281691469712) bank62145 +62146 POINT(30.63221780249277 -95.03499216513629) bank62146 +62147 POINT(29.48326704853286 -94.91444207626255) bank62147 +62148 POINT(30.457502011787025 -96.27213728540329) bank62148 +62149 POINT(29.61931320434861 -95.83799315006205) bank62149 +62150 POINT(29.497520230769318 -94.71852099995138) bank62150 +62151 POINT(29.12191591709105 -94.5823679500735) bank62151 +62152 POINT(29.633846668255277 -94.66740027912293) bank62152 +62153 POINT(30.09872306128951 -95.02665879466501) bank62153 +62154 POINT(29.051113277130156 -95.06073697208839) bank62154 +62155 POINT(30.30763457211236 -94.74576653195746) bank62155 +62156 POINT(29.1031947498411 -95.50921293366484) bank62156 +62157 POINT(29.295949057643394 -95.81023583852277) bank62157 +62158 POINT(29.28076031779212 -94.61243713006463) bank62158 +62159 POINT(30.5115151693944 -95.87574419942176) bank62159 +62160 POINT(30.531374873913123 -94.75714731251269) bank62160 +62161 POINT(29.934719601202378 -95.76575185796018) bank62161 +62162 POINT(30.708836791909004 -96.3673835047208) bank62162 +62163 POINT(30.549636151906494 -95.8867755741172) bank62163 +62164 POINT(28.876143177231654 -94.75487070067035) bank62164 +62165 POINT(30.436782619302722 -94.7376887543012) bank62165 +62166 POINT(30.752550112206247 -95.36897688247431) bank62166 +62167 POINT(29.140702741504544 -96.07197838363957) bank62167 +62168 POINT(29.33243912921015 -96.32742839926374) bank62168 +62169 POINT(28.871961348401626 -95.80609148927994) bank62169 +62170 POINT(29.069352234753662 -96.09195978933819) bank62170 +62171 POINT(29.54701060392565 -95.37899375768089) bank62171 +62172 POINT(29.789148707105504 -95.78560915127014) bank62172 +62173 POINT(29.394171934516017 -95.08883766673833) bank62173 +62174 POINT(29.87773466225225 -95.480809049015) bank62174 +62175 POINT(28.824297266339098 -94.46215651509145) bank62175 +62176 POINT(29.910399606870737 -94.93070196608625) bank62176 +62177 POINT(29.854660682331748 -96.21993718595147) bank62177 +62178 POINT(28.914430883291548 -95.37662641326648) bank62178 +62179 POINT(30.484578981253726 -94.67955954947257) bank62179 +62180 POINT(28.865494337061662 -95.09866644607158) bank62180 +62181 POINT(30.494844300751165 -96.2362302592031) bank62181 +62182 POINT(29.807175334349584 -95.94943342885422) bank62182 +62183 POINT(29.102311005669222 -94.94511652101768) bank62183 +62184 POINT(28.799955719931383 -95.35897835907431) bank62184 +62185 POINT(30.3056929987064 -96.18687748159154) bank62185 +62186 POINT(30.1987035053233 -95.86798529547349) bank62186 +62187 POINT(28.907005481156034 -95.99757299489846) bank62187 +62188 POINT(29.218733761236262 -94.6744594370623) bank62188 +62189 POINT(30.48550970197333 -95.34492906084893) bank62189 +62190 POINT(29.63220112024182 -95.90121577705791) bank62190 +62191 POINT(30.25795940908505 -95.92376716865566) bank62191 +62192 POINT(29.065667374908095 -95.24406964530273) bank62192 +62193 POINT(28.795504108364803 -95.9212436918298) bank62193 +62194 POINT(29.775708012581347 -95.45616083975968) bank62194 +62195 POINT(29.762453550390518 -95.77081892109867) bank62195 +62196 POINT(30.02626245701576 -94.94141021376926) bank62196 +62197 POINT(30.32018580985706 -96.00825512732419) bank62197 +62198 POINT(30.427449075359096 -95.74299789584059) bank62198 +62199 POINT(30.187889332930165 -94.75745562170967) bank62199 +62200 POINT(30.30971305007038 -95.5791733627823) bank62200 +62201 POINT(29.736619254203223 -94.48731630587186) bank62201 +62202 POINT(30.11307037805488 -95.98958340072592) bank62202 +62203 POINT(29.56281492745734 -96.05893117260979) bank62203 +62204 POINT(29.30355306178409 -94.99054838599457) bank62204 +62205 POINT(29.749554511346485 -95.60311505505972) bank62205 +62206 POINT(29.160528672982625 -95.40935821507055) bank62206 +62207 POINT(29.60313349580368 -95.42377563408319) bank62207 +62208 POINT(29.548572482888236 -95.37836762124644) bank62208 +62209 POINT(29.789664405475257 -96.33525398297314) bank62209 +62210 POINT(30.16340356438946 -95.12141142165154) bank62210 +62211 POINT(29.487759978016253 -95.35912596599674) bank62211 +62212 POINT(30.590643949982834 -94.93498528006891) bank62212 +62213 POINT(30.40280602839466 -95.45702333364429) bank62213 +62214 POINT(30.48390039341101 -95.35371863726897) bank62214 +62215 POINT(30.460338287354517 -96.05958158619637) bank62215 +62216 POINT(29.925729332784446 -94.98232757192449) bank62216 +62217 POINT(30.632668502845295 -94.47352336910909) bank62217 +62218 POINT(29.782908770727303 -96.00137477378733) bank62218 +62219 POINT(30.73661328379788 -95.37496334753988) bank62219 +62220 POINT(29.732444288576037 -95.89477387270817) bank62220 +62221 POINT(29.675905206077836 -94.74710773215399) bank62221 +62222 POINT(29.833330949046353 -94.73072046972644) bank62222 +62223 POINT(30.59226331887249 -95.47388389215307) bank62223 +62224 POINT(29.603299564028305 -96.31616933460165) bank62224 +62225 POINT(29.015827190384655 -95.22570306660432) bank62225 +62226 POINT(30.509778322606294 -96.18812506606712) bank62226 +62227 POINT(28.84399169475261 -96.03524062914009) bank62227 +62228 POINT(30.422908453344437 -94.87699259537784) bank62228 +62229 POINT(30.249762654134155 -96.14662240553179) bank62229 +62230 POINT(30.307225975163615 -95.71058000566917) bank62230 +62231 POINT(29.467130802210423 -95.51643748078007) bank62231 +62232 POINT(29.408881433119074 -94.3928851988465) bank62232 +62233 POINT(29.245668975935114 -94.53363058523463) bank62233 +62234 POINT(29.35618288404866 -94.4915339667521) bank62234 +62235 POINT(29.969600113967083 -94.4553954969569) bank62235 +62236 POINT(30.248169702445402 -95.02183850696224) bank62236 +62237 POINT(30.431710172498033 -95.60248967636099) bank62237 +62238 POINT(30.20094451003253 -94.6552296497367) bank62238 +62239 POINT(29.41136070264775 -95.2328635165222) bank62239 +62240 POINT(29.061854615171804 -95.98234984064665) bank62240 +62241 POINT(30.026487633561562 -94.43175534114273) bank62241 +62242 POINT(28.849540720479418 -94.62153703356287) bank62242 +62243 POINT(29.34015922079384 -95.64680942282303) bank62243 +62244 POINT(29.37849385513699 -95.82586106938594) bank62244 +62245 POINT(28.764192028188738 -96.30421455320798) bank62245 +62246 POINT(28.838229305121587 -95.35184455195238) bank62246 +62247 POINT(28.858512766949666 -96.36646126398432) bank62247 +62248 POINT(29.637246451795704 -94.56964648593792) bank62248 +62249 POINT(28.762120175936335 -95.05182015863907) bank62249 +62250 POINT(30.323851881727705 -95.08995081397677) bank62250 +62251 POINT(30.069118200523242 -95.21106334519817) bank62251 +62252 POINT(30.09637949325785 -94.49423673795313) bank62252 +62253 POINT(29.922871501314802 -95.70446391541489) bank62253 +62254 POINT(29.308582962172878 -94.51126226419261) bank62254 +62255 POINT(29.109873070135553 -95.79786732433497) bank62255 +62256 POINT(29.942790112753375 -95.08354674797016) bank62256 +62257 POINT(29.163243202548284 -96.21240705573021) bank62257 +62258 POINT(30.277027531015396 -96.02510932374248) bank62258 +62259 POINT(30.108296616700432 -94.84976905946543) bank62259 +62260 POINT(29.25711070749702 -94.39147755869033) bank62260 +62261 POINT(29.362247439612343 -94.94071761073239) bank62261 +62262 POINT(30.209295293182763 -96.05385468002196) bank62262 +62263 POINT(30.483429909558936 -96.04933160700763) bank62263 +62264 POINT(28.899040417946658 -95.02984349036859) bank62264 +62265 POINT(30.051458649386994 -96.03207118747429) bank62265 +62266 POINT(30.224349078881623 -95.56443640769741) bank62266 +62267 POINT(28.893420383667085 -95.51162115202418) bank62267 +62268 POINT(29.30976003646408 -95.90368818359819) bank62268 +62269 POINT(29.617283211871023 -94.67406643029703) bank62269 +62270 POINT(29.31686583454049 -96.20939918948014) bank62270 +62271 POINT(30.484200907228903 -94.79020398223592) bank62271 +62272 POINT(29.843977893592207 -95.19678222690212) bank62272 +62273 POINT(29.735186316261967 -95.22329865741465) bank62273 +62274 POINT(29.8575593900461 -95.57010384844023) bank62274 +62275 POINT(29.22989240908515 -95.93035314383164) bank62275 +62276 POINT(30.6017767955451 -94.57423921676101) bank62276 +62277 POINT(29.193529636836978 -94.78887775437694) bank62277 +62278 POINT(30.30485985466467 -95.47982630467828) bank62278 +62279 POINT(30.257343177313334 -94.78310762658127) bank62279 +62280 POINT(30.35543584215087 -95.56597466114148) bank62280 +62281 POINT(30.1488040527903 -94.74380630487053) bank62281 +62282 POINT(30.122114296309086 -96.21367716098527) bank62282 +62283 POINT(30.49371811090931 -95.24993578393203) bank62283 +62284 POINT(29.598455216603167 -95.63853396415088) bank62284 +62285 POINT(30.222503191130045 -95.66296617317361) bank62285 +62286 POINT(30.298356476035252 -95.19311089361317) bank62286 +62287 POINT(30.412398020481792 -95.51558866076577) bank62287 +62288 POINT(29.78373051649441 -95.20328134813947) bank62288 +62289 POINT(29.293996379057308 -94.46361936264057) bank62289 +62290 POINT(30.658320133104496 -94.41351680728451) bank62290 +62291 POINT(28.934486959667836 -94.5579833165117) bank62291 +62292 POINT(29.929415807961444 -95.39980560323016) bank62292 +62293 POINT(29.051521180247498 -95.80439217912964) bank62293 +62294 POINT(30.617632206118262 -94.47580369008624) bank62294 +62295 POINT(30.51358295049737 -96.11751383621406) bank62295 +62296 POINT(29.398991228823075 -96.3481354245849) bank62296 +62297 POINT(29.914337423379322 -95.21118534735204) bank62297 +62298 POINT(30.340823106234517 -95.41466889955186) bank62298 +62299 POINT(30.20362417468633 -95.57402982567167) bank62299 +62300 POINT(28.78787753993374 -95.93601071436412) bank62300 +62301 POINT(30.036545642395104 -94.73260453278898) bank62301 +62302 POINT(29.25189681998372 -95.44565022113233) bank62302 +62303 POINT(29.305691345733486 -95.28124859970697) bank62303 +62304 POINT(29.07347612990444 -94.53314467989884) bank62304 +62305 POINT(30.709733592738786 -96.01953939961099) bank62305 +62306 POINT(28.93967652256686 -96.04200529390896) bank62306 +62307 POINT(29.95669635599202 -96.24892362748771) bank62307 +62308 POINT(29.884148940969148 -94.85684372308856) bank62308 +62309 POINT(29.6002648126958 -94.39138380833016) bank62309 +62310 POINT(29.95556451323209 -94.76892549740226) bank62310 +62311 POINT(29.93302071728735 -94.90102893888648) bank62311 +62312 POINT(30.293100763031017 -95.39575675563516) bank62312 +62313 POINT(30.118300345545553 -95.29450843932644) bank62313 +62314 POINT(30.289031241221828 -94.87299827829669) bank62314 +62315 POINT(30.37757331116443 -95.8856360871546) bank62315 +62316 POINT(29.944748736818507 -94.4371566273339) bank62316 +62317 POINT(29.786175296487418 -94.75537898569468) bank62317 +62318 POINT(29.969369592453994 -94.60216060563752) bank62318 +62319 POINT(30.440824268343718 -96.32157342654351) bank62319 +62320 POINT(29.201137799937015 -96.10915323795933) bank62320 +62321 POINT(30.082637260095435 -95.62237000457068) bank62321 +62322 POINT(28.928729542354066 -95.5201732599702) bank62322 +62323 POINT(29.621797562897992 -94.41010807664358) bank62323 +62324 POINT(29.4988434285988 -94.4030973802208) bank62324 +62325 POINT(28.92173747116901 -95.50311885393234) bank62325 +62326 POINT(29.076799575300896 -96.15562365222924) bank62326 +62327 POINT(28.953976021148215 -96.18465709212441) bank62327 +62328 POINT(30.070853351608097 -96.20007596063672) bank62328 +62329 POINT(28.86575031788367 -94.63869154569021) bank62329 +62330 POINT(29.938017173705646 -94.79007513709874) bank62330 +62331 POINT(29.013872449891263 -95.02314164995336) bank62331 +62332 POINT(30.560782317347993 -95.83415653845276) bank62332 +62333 POINT(28.80490374143319 -95.93744836431043) bank62333 +62334 POINT(30.225851231668262 -95.17677234068243) bank62334 +62335 POINT(30.65510857030062 -95.93370673035434) bank62335 +62336 POINT(30.001230885499403 -94.91336756257292) bank62336 +62337 POINT(28.76697526116088 -95.20909944732391) bank62337 +62338 POINT(30.644738859738435 -95.39773278271862) bank62338 +62339 POINT(30.58492741970977 -94.4098373353319) bank62339 +62340 POINT(30.540342504530983 -95.0044814865593) bank62340 +62341 POINT(30.10805198965626 -94.8851768629577) bank62341 +62342 POINT(28.961676637371884 -94.69376290922328) bank62342 +62343 POINT(28.882355059931392 -95.60806809815865) bank62343 +62344 POINT(29.142713933715996 -96.16587242956568) bank62344 +62345 POINT(29.064666675413555 -95.74436963922874) bank62345 +62346 POINT(29.623546439865553 -94.99681025646618) bank62346 +62347 POINT(30.38395823465131 -95.54742837857447) bank62347 +62348 POINT(29.624935802214903 -95.62317139350338) bank62348 +62349 POINT(28.79747681170598 -95.8345424979631) bank62349 +62350 POINT(28.86042108765212 -95.60108467312) bank62350 +62351 POINT(30.723395886917675 -94.7901851376064) bank62351 +62352 POINT(28.91919871989576 -94.53501049337598) bank62352 +62353 POINT(30.66169536075205 -95.8519370574818) bank62353 +62354 POINT(30.665888441073744 -95.74031402742439) bank62354 +62355 POINT(28.8141285321999 -95.9764202135143) bank62355 +62356 POINT(28.975785229233082 -96.15246990620555) bank62356 +62357 POINT(28.883708946904097 -95.78505839700179) bank62357 +62358 POINT(29.988422662193503 -94.48060287852657) bank62358 +62359 POINT(28.918442022555386 -96.24721789296895) bank62359 +62360 POINT(30.426942151075313 -94.39139463932419) bank62360 +62361 POINT(29.49512145099607 -95.00013000060808) bank62361 +62362 POINT(29.273193320601756 -94.61113116019973) bank62362 +62363 POINT(29.795371952618893 -96.07501242626856) bank62363 +62364 POINT(29.770254610535837 -94.70153306941765) bank62364 +62365 POINT(30.32367593941475 -94.84387704958989) bank62365 +62366 POINT(30.13387010765523 -94.71376447051425) bank62366 +62367 POINT(28.93070144440139 -96.36371110518748) bank62367 +62368 POINT(29.65261831909863 -95.04723264647548) bank62368 +62369 POINT(28.91400597045522 -95.80861615580324) bank62369 +62370 POINT(29.166777985183025 -94.70405162233914) bank62370 +62371 POINT(30.363341381906515 -95.57666981414363) bank62371 +62372 POINT(29.26346638870137 -95.70354108223691) bank62372 +62373 POINT(29.164771538129635 -94.57689498810724) bank62373 +62374 POINT(30.559300395856116 -94.42448699399367) bank62374 +62375 POINT(29.432622352936885 -95.25406303209682) bank62375 +62376 POINT(29.183342680903756 -95.86054768337897) bank62376 +62377 POINT(30.657698600144535 -96.35721999183231) bank62377 +62378 POINT(29.627377624913283 -95.19222716328902) bank62378 +62379 POINT(30.71340138783922 -94.58267805771187) bank62379 +62380 POINT(29.075722330595354 -96.36144031870228) bank62380 +62381 POINT(29.21042129656068 -95.18634928537816) bank62381 +62382 POINT(30.27444726194215 -95.70865813199461) bank62382 +62383 POINT(30.05142733784087 -96.11302448668991) bank62383 +62384 POINT(29.790982808937045 -95.34360240330919) bank62384 +62385 POINT(29.729766462142962 -95.57017325342801) bank62385 +62386 POINT(28.811663059281123 -94.901473237267) bank62386 +62387 POINT(29.50558496155934 -94.5355953464649) bank62387 +62388 POINT(29.63599534028665 -95.44270574491989) bank62388 +62389 POINT(28.84420223151593 -95.9372409164221) bank62389 +62390 POINT(28.901651746822353 -94.96638154911358) bank62390 +62391 POINT(28.963099794262376 -96.09399510155171) bank62391 +62392 POINT(30.213998695030774 -95.4288317495769) bank62392 +62393 POINT(29.934934106172655 -96.24902252641277) bank62393 +62394 POINT(30.37365915253441 -95.92801228581325) bank62394 +62395 POINT(29.256023336036684 -95.89993295845797) bank62395 +62396 POINT(29.724287534987262 -94.8792671771295) bank62396 +62397 POINT(29.606205558046096 -94.98219435252001) bank62397 +62398 POINT(30.33434529320055 -95.8961317644195) bank62398 +62399 POINT(29.424619262943235 -95.79587571517322) bank62399 +62400 POINT(30.413841374142425 -95.268767411925) bank62400 +62401 POINT(29.584611886414603 -94.91765776317463) bank62401 +62402 POINT(29.003416154669146 -95.40455092552725) bank62402 +62403 POINT(29.829326903829877 -95.23922503704762) bank62403 +62404 POINT(29.90167011835073 -96.36302436337519) bank62404 +62405 POINT(29.55962224924612 -95.209219659398) bank62405 +62406 POINT(30.322909413320875 -94.81963372221223) bank62406 +62407 POINT(29.00247986163427 -94.68523159763394) bank62407 +62408 POINT(30.601864869698073 -95.93596279492148) bank62408 +62409 POINT(30.214475648209135 -95.77753410288082) bank62409 +62410 POINT(29.829706307921676 -94.44633522958631) bank62410 +62411 POINT(29.571752873048265 -94.59823844532026) bank62411 +62412 POINT(29.172936343130523 -94.45551291296925) bank62412 +62413 POINT(29.423468865990422 -95.4475673518761) bank62413 +62414 POINT(29.022904714969705 -96.05496178140852) bank62414 +62415 POINT(30.473618929817764 -96.36186423249971) bank62415 +62416 POINT(29.81431779981799 -96.04182898431932) bank62416 +62417 POINT(29.575290197948316 -96.34166433008511) bank62417 +62418 POINT(29.551372152510787 -95.98310085269634) bank62418 +62419 POINT(30.076222557470683 -94.57432668293403) bank62419 +62420 POINT(30.55346494919298 -95.90565161450377) bank62420 +62421 POINT(30.300173538558695 -96.23206588868273) bank62421 +62422 POINT(28.896724268978506 -96.35156108278746) bank62422 +62423 POINT(30.327044995552846 -94.67694941444057) bank62423 +62424 POINT(29.57227318151097 -95.1217258307316) bank62424 +62425 POINT(30.234338335935625 -95.94111441512896) bank62425 +62426 POINT(30.044697981819894 -94.68251504891872) bank62426 +62427 POINT(30.06446134044043 -95.00425387974677) bank62427 +62428 POINT(29.706387474328515 -96.16577923775512) bank62428 +62429 POINT(29.54724586824254 -95.3546572347501) bank62429 +62430 POINT(29.360505179926268 -96.11065623473154) bank62430 +62431 POINT(30.644695961131195 -95.59888446195872) bank62431 +62432 POINT(29.261022031916873 -95.98155513629862) bank62432 +62433 POINT(29.41024661676246 -95.63411604198805) bank62433 +62434 POINT(30.720953812086186 -94.75226513712083) bank62434 +62435 POINT(29.348516417066172 -94.56196630893595) bank62435 +62436 POINT(29.00672106769551 -95.1647255991817) bank62436 +62437 POINT(29.072103311838102 -94.96002218524497) bank62437 +62438 POINT(29.643897173622086 -95.10897552036056) bank62438 +62439 POINT(29.667342889443308 -94.71794140296882) bank62439 +62440 POINT(29.038555223646025 -95.77847556012067) bank62440 +62441 POINT(28.886687053082987 -94.4860179679263) bank62441 +62442 POINT(29.329218936716284 -95.6322137608748) bank62442 +62443 POINT(30.497711264599165 -94.80447208908947) bank62443 +62444 POINT(30.633019110994763 -95.99106275702175) bank62444 +62445 POINT(30.69054205612649 -95.5435474124542) bank62445 +62446 POINT(29.6216118700943 -96.05420560300671) bank62446 +62447 POINT(29.893415809413984 -95.97859751008637) bank62447 +62448 POINT(30.339823925916583 -95.3815324092007) bank62448 +62449 POINT(30.00010904270395 -95.39508642423031) bank62449 +62450 POINT(30.111414391346624 -94.52041415942166) bank62450 +62451 POINT(29.13940511684105 -94.54571278450649) bank62451 +62452 POINT(29.435127866931303 -95.75585539915951) bank62452 +62453 POINT(30.32524403702503 -95.34835130683507) bank62453 +62454 POINT(28.863708825777735 -95.38816154812383) bank62454 +62455 POINT(30.150508560816395 -94.56458186807872) bank62455 +62456 POINT(29.158119469084212 -96.03888691856957) bank62456 +62457 POINT(29.716883017737967 -94.44621043874128) bank62457 +62458 POINT(30.64574802414752 -94.40575255564211) bank62458 +62459 POINT(30.63290321489783 -95.2705486977009) bank62459 +62460 POINT(30.11150249348732 -94.95989903160233) bank62460 +62461 POINT(29.043425303157075 -94.43921644495325) bank62461 +62462 POINT(29.863586614529282 -94.95635947186094) bank62462 +62463 POINT(30.165561151456977 -95.00168672546673) bank62463 +62464 POINT(30.17778021651578 -96.27424782591511) bank62464 +62465 POINT(30.26876183674111 -95.55537511560689) bank62465 +62466 POINT(30.705742408411883 -95.52988005164076) bank62466 +62467 POINT(30.345031996516 -95.95690202083728) bank62467 +62468 POINT(30.022527511216403 -95.51253936413835) bank62468 +62469 POINT(30.47918277398839 -95.3402661799386) bank62469 +62470 POINT(28.78571209660067 -94.53718872144044) bank62470 +62471 POINT(30.301962376657375 -94.41375157499873) bank62471 +62472 POINT(30.55799968436552 -95.7875280798097) bank62472 +62473 POINT(29.614444310574306 -95.63947706180788) bank62473 +62474 POINT(30.020612667226448 -95.9886673152146) bank62474 +62475 POINT(29.153887245826855 -95.24448905805737) bank62475 +62476 POINT(30.59546164995104 -95.42045947855301) bank62476 +62477 POINT(30.20797690067944 -95.02600821858125) bank62477 +62478 POINT(29.667659652668963 -95.76704299863547) bank62478 +62479 POINT(30.480948284286317 -94.74326855631244) bank62479 +62480 POINT(29.19800067248792 -94.55665307906808) bank62480 +62481 POINT(29.751182758605044 -94.51846139115729) bank62481 +62482 POINT(29.857932053702545 -94.40373342833217) bank62482 +62483 POINT(29.771144108146903 -94.74981793933473) bank62483 +62484 POINT(30.308902631571573 -94.62522599576636) bank62484 +62485 POINT(30.532620577356322 -94.9947792875467) bank62485 +62486 POINT(29.558072468289666 -96.27731326426029) bank62486 +62487 POINT(30.437212675273084 -96.02735540099737) bank62487 +62488 POINT(30.26166131025052 -94.7890921006299) bank62488 +62489 POINT(30.693431281221212 -96.367488990246) bank62489 +62490 POINT(29.021720743620023 -94.45289005347001) bank62490 +62491 POINT(30.264152380112574 -94.93583917955125) bank62491 +62492 POINT(29.5002521708854 -95.66813113361943) bank62492 +62493 POINT(29.911823851706103 -94.45959364260992) bank62493 +62494 POINT(29.131411475974822 -96.1451403088368) bank62494 +62495 POINT(28.77628289586077 -94.39562501193828) bank62495 +62496 POINT(28.768484625464623 -95.54475563738183) bank62496 +62497 POINT(28.995037965790768 -95.54030875348184) bank62497 +62498 POINT(30.73415347815426 -95.3835450188229) bank62498 +62499 POINT(29.751188533509005 -95.15982143206736) bank62499 +62500 POINT(29.113623293405173 -94.46794108433599) bank62500 +62501 POINT(29.013855975711962 -95.10101766760712) bank62501 +62502 POINT(30.09740047702614 -95.68189262758841) bank62502 +62503 POINT(30.532206981299172 -95.12212254227242) bank62503 +62504 POINT(30.32872892524578 -95.2840915222061) bank62504 +62505 POINT(30.34638206542432 -96.13729415727735) bank62505 +62506 POINT(30.529772708027398 -95.88515661016642) bank62506 +62507 POINT(29.187521102682926 -95.84348564905687) bank62507 +62508 POINT(29.672698026642475 -95.37609314113803) bank62508 +62509 POINT(29.538165294119825 -95.72723676106105) bank62509 +62510 POINT(30.49018255287169 -95.87501445127461) bank62510 +62511 POINT(29.30325844180131 -96.11969248661057) bank62511 +62512 POINT(29.255315896760454 -94.84477573389765) bank62512 +62513 POINT(30.58603734963952 -95.74311199779956) bank62513 +62514 POINT(29.21816035373811 -94.53949252733793) bank62514 +62515 POINT(30.33301522943087 -95.71240571744173) bank62515 +62516 POINT(29.364681229254348 -95.81316319944297) bank62516 +62517 POINT(29.97797738704568 -95.80789209765904) bank62517 +62518 POINT(28.821967687550654 -94.55616436267674) bank62518 +62519 POINT(29.13288512538592 -96.25853741474475) bank62519 +62520 POINT(29.54349311496005 -96.26762454655926) bank62520 +62521 POINT(29.53765239694341 -95.40258793144416) bank62521 +62522 POINT(28.85312752363094 -95.63643879200905) bank62522 +62523 POINT(29.421486474741254 -95.64483366919092) bank62523 +62524 POINT(29.83845153069928 -94.68181565913304) bank62524 +62525 POINT(29.962905154305062 -94.90244074186454) bank62525 +62526 POINT(28.902172471267573 -95.34066405503377) bank62526 +62527 POINT(30.49112126699954 -95.18457224842322) bank62527 +62528 POINT(29.98755319590644 -94.9060362729471) bank62528 +62529 POINT(30.331047239631403 -96.27038074773151) bank62529 +62530 POINT(28.982158594950437 -94.8550890090931) bank62530 +62531 POINT(29.193460679807465 -96.22481322231013) bank62531 +62532 POINT(28.79696502743585 -94.41507907545781) bank62532 +62533 POINT(30.159688112311066 -96.26554740839899) bank62533 +62534 POINT(29.09680015590879 -94.77427656269582) bank62534 +62535 POINT(29.724428973921313 -95.83542873085386) bank62535 +62536 POINT(30.65190266689061 -94.95840879351672) bank62536 +62537 POINT(29.560014487843205 -95.19786491113884) bank62537 +62538 POINT(29.823309369052843 -94.39964572949718) bank62538 +62539 POINT(28.877375167431815 -94.6227378505905) bank62539 +62540 POINT(30.701651471311 -96.20379873954938) bank62540 +62541 POINT(29.5560796894052 -94.48879609913465) bank62541 +62542 POINT(29.361842446144525 -94.44728781561501) bank62542 +62543 POINT(30.013375925554335 -94.84822298942345) bank62543 +62544 POINT(30.59662014420416 -95.99892890419818) bank62544 +62545 POINT(30.62666466840248 -94.78382045089722) bank62545 +62546 POINT(29.14346704330505 -95.3138049058427) bank62546 +62547 POINT(30.044071286826764 -95.69134702313671) bank62547 +62548 POINT(29.867368098331994 -96.24532643653598) bank62548 +62549 POINT(29.43711054752124 -94.64327151364012) bank62549 +62550 POINT(29.238565696575424 -95.57877637488468) bank62550 +62551 POINT(30.516971424513244 -95.66110623958005) bank62551 +62552 POINT(30.58796258282024 -95.63775242728775) bank62552 +62553 POINT(30.18951912068441 -94.42614691680096) bank62553 +62554 POINT(29.85197402065133 -96.28682725105948) bank62554 +62555 POINT(30.228431605047827 -94.57988041594207) bank62555 +62556 POINT(29.03566449474871 -94.77264799997162) bank62556 +62557 POINT(29.59498284939695 -94.49590040146086) bank62557 +62558 POINT(30.71693809124342 -96.31470301332332) bank62558 +62559 POINT(28.892845501813852 -94.77862298053302) bank62559 +62560 POINT(30.756274508561923 -94.5340336989996) bank62560 +62561 POINT(30.05047442628365 -95.02177416664759) bank62561 +62562 POINT(30.347382143863445 -95.33922614339423) bank62562 +62563 POINT(29.2586972856024 -95.03027636857558) bank62563 +62564 POINT(29.792516931080794 -95.1985749578109) bank62564 +62565 POINT(29.967394344446095 -96.19720471765413) bank62565 +62566 POINT(30.506917936420688 -94.42570902285428) bank62566 +62567 POINT(30.482212134037866 -95.91613251235249) bank62567 +62568 POINT(29.29208269512842 -95.66657121195247) bank62568 +62569 POINT(30.63913640691358 -95.17770745103103) bank62569 +62570 POINT(29.896746930589085 -95.36160541586398) bank62570 +62571 POINT(29.74813984807906 -95.15519771803568) bank62571 +62572 POINT(29.857624872074037 -95.21758710909332) bank62572 +62573 POINT(30.461902049684515 -95.88318019356244) bank62573 +62574 POINT(29.166656826941757 -94.43949413255243) bank62574 +62575 POINT(29.999202642802985 -95.7469076996583) bank62575 +62576 POINT(28.87791802203197 -95.75971594159705) bank62576 +62577 POINT(30.335883319312114 -95.83994006500869) bank62577 +62578 POINT(28.969792116681383 -94.71595669562468) bank62578 +62579 POINT(29.381752436530242 -95.84656717760664) bank62579 +62580 POINT(30.744637133774425 -94.37918669015183) bank62580 +62581 POINT(29.618917870398892 -95.94665687616168) bank62581 +62582 POINT(29.8744168298833 -96.12133070099192) bank62582 +62583 POINT(29.04002340910416 -95.04145671022407) bank62583 +62584 POINT(30.188615084223013 -95.5677609155537) bank62584 +62585 POINT(29.304729679912516 -94.72637469997343) bank62585 +62586 POINT(29.28300451389982 -95.0492826000057) bank62586 +62587 POINT(29.87968374626057 -95.86228147446955) bank62587 +62588 POINT(30.169363996926883 -96.23755384944343) bank62588 +62589 POINT(28.994400164417083 -94.54250762385738) bank62589 +62590 POINT(29.078676173960297 -94.48077363191801) bank62590 +62591 POINT(29.27480485025483 -95.94457252978668) bank62591 +62592 POINT(28.959515225146106 -95.68922647265774) bank62592 +62593 POINT(30.57421506805646 -94.84420906781146) bank62593 +62594 POINT(29.273437980903065 -96.34493627908748) bank62594 +62595 POINT(30.620419034662365 -94.57895539024786) bank62595 +62596 POINT(29.675422030757478 -95.50816786995716) bank62596 +62597 POINT(29.82462916586396 -95.20097227576558) bank62597 +62598 POINT(29.458068954100845 -95.89398340154754) bank62598 +62599 POINT(29.216925396315162 -95.41817297191069) bank62599 +62600 POINT(29.33159354801646 -95.431527972555) bank62600 +62601 POINT(30.424393501218194 -95.54933013614571) bank62601 +62602 POINT(28.98443048507279 -95.9654224373403) bank62602 +62603 POINT(30.669165377301873 -94.53463860674653) bank62603 +62604 POINT(29.81439770078533 -94.3721067823534) bank62604 +62605 POINT(28.955481258076766 -95.08172386400432) bank62605 +62606 POINT(30.61923878268459 -95.03370017459555) bank62606 +62607 POINT(30.57258895097839 -95.93068254950582) bank62607 +62608 POINT(28.796441395012682 -95.70846477542399) bank62608 +62609 POINT(29.51436520662456 -95.24786431261312) bank62609 +62610 POINT(30.199894963930703 -95.95151196238739) bank62610 +62611 POINT(29.683700876758827 -96.36536509530035) bank62611 +62612 POINT(29.26123949215042 -95.9308928121481) bank62612 +62613 POINT(29.48706241381264 -95.89869457278895) bank62613 +62614 POINT(28.87050066743979 -94.79250291034329) bank62614 +62615 POINT(30.260355178107705 -94.62888161682974) bank62615 +62616 POINT(29.273146226601117 -94.55193846620841) bank62616 +62617 POINT(30.117308949114932 -94.62299685505312) bank62617 +62618 POINT(30.342779242020026 -95.4195476351327) bank62618 +62619 POINT(29.1769698731338 -95.97466110290374) bank62619 +62620 POINT(30.01816090125735 -96.02597997715498) bank62620 +62621 POINT(29.78036102190659 -94.61640261150353) bank62621 +62622 POINT(29.86080728718109 -95.08221922775961) bank62622 +62623 POINT(29.633001411279032 -94.94900208517751) bank62623 +62624 POINT(30.44079094649346 -94.49146665714659) bank62624 +62625 POINT(30.735871381536878 -96.04303502720512) bank62625 +62626 POINT(29.680718875478348 -96.3311130862314) bank62626 +62627 POINT(29.51932701423028 -94.98267021481789) bank62627 +62628 POINT(29.418870461818063 -96.27952914484719) bank62628 +62629 POINT(29.30166508297769 -96.31870184087845) bank62629 +62630 POINT(30.490807204709135 -95.25202914256516) bank62630 +62631 POINT(30.448537213850788 -96.2750515750871) bank62631 +62632 POINT(30.320331128347107 -95.39072452106318) bank62632 +62633 POINT(29.37335828853664 -96.1819092451447) bank62633 +62634 POINT(29.08756097647956 -95.21647524143926) bank62634 +62635 POINT(29.48651237191135 -95.24068572475821) bank62635 +62636 POINT(29.782756385921 -95.2981913713516) bank62636 +62637 POINT(29.528122461411993 -94.91853907518768) bank62637 +62638 POINT(30.12697079885089 -94.71786084858607) bank62638 +62639 POINT(29.64721653418248 -96.23321283681119) bank62639 +62640 POINT(30.133813920758172 -95.52295480149172) bank62640 +62641 POINT(30.655548533639227 -94.85493784971298) bank62641 +62642 POINT(29.385763130186923 -94.49888652384266) bank62642 +62643 POINT(30.078177431757936 -95.8256445230284) bank62643 +62644 POINT(29.121494903339784 -94.4527723237684) bank62644 +62645 POINT(29.436821531367514 -95.42418915809087) bank62645 +62646 POINT(30.051516216190993 -94.9410241170012) bank62646 +62647 POINT(29.31895046168807 -95.88984133927346) bank62647 +62648 POINT(28.87903672313292 -95.030721833241) bank62648 +62649 POINT(29.38004036114761 -94.51262018360302) bank62649 +62650 POINT(30.371774066145328 -96.15050603141697) bank62650 +62651 POINT(30.45671635727091 -94.60786092711547) bank62651 +62652 POINT(29.049915769753643 -94.500259837038) bank62652 +62653 POINT(28.881771682541938 -94.99715349713833) bank62653 +62654 POINT(30.36690330678807 -94.81621113425345) bank62654 +62655 POINT(29.559077589710782 -95.24192947521576) bank62655 +62656 POINT(29.970527027672524 -95.76314938444546) bank62656 +62657 POINT(28.914978794606476 -95.67758502078986) bank62657 +62658 POINT(29.036559171118252 -94.96488033263877) bank62658 +62659 POINT(30.412748893915442 -95.32709198615713) bank62659 +62660 POINT(30.744658356025194 -95.87178902093765) bank62660 +62661 POINT(28.859453031325344 -94.8557196684095) bank62661 +62662 POINT(30.140330696889208 -95.81379509865651) bank62662 +62663 POINT(29.823458778057375 -94.68578561681254) bank62663 +62664 POINT(29.626046533839197 -94.65508899672733) bank62664 +62665 POINT(29.51964538750834 -95.82082281893565) bank62665 +62666 POINT(30.721289487650843 -94.38094395504693) bank62666 +62667 POINT(29.26176357336782 -94.92182131274862) bank62667 +62668 POINT(30.084294638024197 -96.21987706554656) bank62668 +62669 POINT(29.418459395970004 -96.04341517170263) bank62669 +62670 POINT(29.854139799851563 -94.85482040562162) bank62670 +62671 POINT(28.956420424529966 -95.81630509845617) bank62671 +62672 POINT(29.051086986403572 -95.38135337169805) bank62672 +62673 POINT(29.013612221044788 -95.6154912768111) bank62673 +62674 POINT(29.27171035160971 -94.73269799313059) bank62674 +62675 POINT(30.240737219574434 -95.55394412604556) bank62675 +62676 POINT(29.584330015480536 -94.83695318527852) bank62676 +62677 POINT(29.11878064224608 -95.1338132055856) bank62677 +62678 POINT(30.126437019820642 -94.71428189964564) bank62678 +62679 POINT(29.146650112532498 -94.49243535282821) bank62679 +62680 POINT(28.969398182342943 -94.62522265874651) bank62680 +62681 POINT(29.41008039863936 -95.60448933489091) bank62681 +62682 POINT(29.539526011728242 -95.15295233296172) bank62682 +62683 POINT(29.51600257033045 -95.65958158951616) bank62683 +62684 POINT(28.916843978512947 -94.57248309761994) bank62684 +62685 POINT(29.29055937098255 -94.58481106765086) bank62685 +62686 POINT(30.43356436236803 -96.27401675661497) bank62686 +62687 POINT(30.5540700185714 -95.48253030931036) bank62687 +62688 POINT(30.244158382988505 -95.33569111595146) bank62688 +62689 POINT(29.88573827041191 -94.68061617349456) bank62689 +62690 POINT(29.065471745180528 -95.26271900504155) bank62690 +62691 POINT(29.80962661638305 -95.25764052283247) bank62691 +62692 POINT(29.768161909945256 -95.9709985086418) bank62692 +62693 POINT(29.87377040550624 -95.17073699029834) bank62693 +62694 POINT(30.111230060555354 -94.39657543334238) bank62694 +62695 POINT(30.432159296820156 -95.2538604347804) bank62695 +62696 POINT(29.36529931703464 -94.62450377038749) bank62696 +62697 POINT(30.394629372487614 -95.97132969502891) bank62697 +62698 POINT(29.03539101275643 -95.25989458777137) bank62698 +62699 POINT(30.02246289422156 -95.47078923033075) bank62699 +62700 POINT(30.062376145047327 -95.97850270338247) bank62700 +62701 POINT(30.367599591567355 -95.97145638119552) bank62701 +62702 POINT(29.68196081238959 -94.78139683850333) bank62702 +62703 POINT(28.97895835997858 -95.06243972703466) bank62703 +62704 POINT(29.666470517141857 -96.21500496787661) bank62704 +62705 POINT(30.274339208808737 -95.75966590741555) bank62705 +62706 POINT(29.671999038738328 -94.92059980452399) bank62706 +62707 POINT(29.618211498958154 -95.84272815189664) bank62707 +62708 POINT(30.469437532123667 -95.67113349388526) bank62708 +62709 POINT(30.711454346732072 -96.01502060753472) bank62709 +62710 POINT(29.870562970730912 -96.14167820535353) bank62710 +62711 POINT(29.919196059397883 -95.17700640303576) bank62711 +62712 POINT(30.169190844098168 -95.08249494479257) bank62712 +62713 POINT(29.547610454995997 -95.1581399899352) bank62713 +62714 POINT(28.97054700604204 -95.99390216315226) bank62714 +62715 POINT(29.67106019209649 -96.00052001920609) bank62715 +62716 POINT(30.11269803031599 -95.7219137772752) bank62716 +62717 POINT(30.709791464715124 -95.56286035460579) bank62717 +62718 POINT(29.72326507577346 -96.22185844609234) bank62718 +62719 POINT(29.909687048406234 -95.6658349875016) bank62719 +62720 POINT(28.99961499575334 -94.99346975384724) bank62720 +62721 POINT(29.0255294489429 -96.26900683321624) bank62721 +62722 POINT(29.67993162319365 -96.00546944232742) bank62722 +62723 POINT(29.55882673166175 -95.48193583463578) bank62723 +62724 POINT(29.512975199120685 -95.60883698095732) bank62724 +62725 POINT(29.88025162139101 -95.87890292195921) bank62725 +62726 POINT(29.371396841912812 -95.82311755369507) bank62726 +62727 POINT(28.961774776744157 -96.00690905469705) bank62727 +62728 POINT(28.914005750846506 -94.71766743693526) bank62728 +62729 POINT(29.796937484838114 -96.28250922474007) bank62729 +62730 POINT(30.19110015830045 -94.92763472511413) bank62730 +62731 POINT(30.374852928484092 -96.1240394813062) bank62731 +62732 POINT(30.51727316367682 -95.07718629009965) bank62732 +62733 POINT(29.365988117174442 -96.34646218528711) bank62733 +62734 POINT(28.89015239075605 -95.06490042190295) bank62734 +62735 POINT(30.019435803213064 -94.69247304592466) bank62735 +62736 POINT(29.542135033998527 -95.26500740957562) bank62736 +62737 POINT(30.424521999853113 -95.26068743035334) bank62737 +62738 POINT(29.857812372136966 -94.60619056699987) bank62738 +62739 POINT(28.941884930450033 -94.96494146370799) bank62739 +62740 POINT(30.078905820120006 -95.34351775044718) bank62740 +62741 POINT(30.707428621683864 -95.34132982498011) bank62741 +62742 POINT(29.065524422719655 -96.1980055580503) bank62742 +62743 POINT(28.98726431694068 -94.44379962905128) bank62743 +62744 POINT(30.105151106090776 -95.46537106022677) bank62744 +62745 POINT(30.194867941728496 -95.36208370095994) bank62745 +62746 POINT(29.392421405037364 -95.10084853115535) bank62746 +62747 POINT(28.91959832228444 -95.75523064665435) bank62747 +62748 POINT(29.735573036977417 -95.51610232094568) bank62748 +62749 POINT(29.810730472949473 -94.97800492319574) bank62749 +62750 POINT(30.70334753718525 -94.39182708363595) bank62750 +62751 POINT(29.13067323077278 -95.0649178648911) bank62751 +62752 POINT(28.962629421202752 -95.9217507732532) bank62752 +62753 POINT(29.140353554948796 -94.73844419291069) bank62753 +62754 POINT(28.91634885081403 -94.40653789567195) bank62754 +62755 POINT(30.11096760980432 -94.43566700676162) bank62755 +62756 POINT(29.933109533989672 -96.05280977950459) bank62756 +62757 POINT(29.282543270781122 -95.3073416245155) bank62757 +62758 POINT(29.871894947951855 -95.02307601489977) bank62758 +62759 POINT(30.635531096967796 -95.28491790227291) bank62759 +62760 POINT(29.472616230138076 -96.27503845614646) bank62760 +62761 POINT(28.884312308912566 -94.7743404995068) bank62761 +62762 POINT(30.623211715582205 -95.33961719675) bank62762 +62763 POINT(28.804087095745786 -96.20813523934157) bank62763 +62764 POINT(30.492815281098007 -94.43026226774282) bank62764 +62765 POINT(30.23669608937436 -95.88372287074199) bank62765 +62766 POINT(30.6365400511826 -95.89434091135581) bank62766 +62767 POINT(28.82168056116143 -94.6718860672708) bank62767 +62768 POINT(29.38957367812667 -94.69107506473432) bank62768 +62769 POINT(29.525067900356323 -95.71327338356889) bank62769 +62770 POINT(29.701350094080734 -96.29121889178013) bank62770 +62771 POINT(30.4555716081327 -94.60314705879301) bank62771 +62772 POINT(28.949539565343322 -94.68550661475035) bank62772 +62773 POINT(30.12533377532681 -95.771461049534) bank62773 +62774 POINT(30.028873682330318 -96.02247573292546) bank62774 +62775 POINT(29.433917900373352 -94.62347338699796) bank62775 +62776 POINT(29.094236940438822 -94.58030855342805) bank62776 +62777 POINT(30.21773114148658 -95.22314454066104) bank62777 +62778 POINT(29.411662483856528 -95.6868802999344) bank62778 +62779 POINT(30.217766554796327 -94.94997021239226) bank62779 +62780 POINT(29.737292823472085 -95.78993839560071) bank62780 +62781 POINT(29.30539859220325 -95.43685321380642) bank62781 +62782 POINT(28.916961465399968 -96.28479611868589) bank62782 +62783 POINT(29.372370033623753 -96.21491629824288) bank62783 +62784 POINT(30.402097299496667 -96.0315209900064) bank62784 +62785 POINT(29.711746815346306 -96.25750700675341) bank62785 +62786 POINT(30.653281237697072 -95.32818764489735) bank62786 +62787 POINT(29.251350450716416 -94.68047053788617) bank62787 +62788 POINT(29.21399302288096 -94.99671661191478) bank62788 +62789 POINT(29.941470990990588 -94.42334452121871) bank62789 +62790 POINT(30.479557141095416 -95.43706786716005) bank62790 +62791 POINT(29.471061223869928 -94.69033090923055) bank62791 +62792 POINT(29.733437214589 -94.38656414919363) bank62792 +62793 POINT(29.43783377366715 -96.27579603713113) bank62793 +62794 POINT(29.473906790761973 -94.82743303379348) bank62794 +62795 POINT(29.352223496358075 -96.23622061933617) bank62795 +62796 POINT(29.696855654307907 -96.19193289286693) bank62796 +62797 POINT(29.767509227130663 -96.19818906284861) bank62797 +62798 POINT(29.84597221759594 -96.29045629980631) bank62798 +62799 POINT(30.519458369247744 -96.28509645986045) bank62799 +62800 POINT(30.259763813666538 -96.15906144089708) bank62800 +62801 POINT(30.666385825013826 -95.65238351273757) bank62801 +62802 POINT(30.284966045220976 -95.99490585084173) bank62802 +62803 POINT(28.834666697473512 -96.0687807363144) bank62803 +62804 POINT(28.850074863772683 -95.80825812855173) bank62804 +62805 POINT(29.284659696250742 -94.37314016763146) bank62805 +62806 POINT(29.938588438299846 -95.46873566240501) bank62806 +62807 POINT(30.390841885858244 -95.45088794973894) bank62807 +62808 POINT(30.61894684968376 -96.31650870617393) bank62808 +62809 POINT(30.543540099275212 -96.3491187923194) bank62809 +62810 POINT(30.595144460983608 -94.63306952073705) bank62810 +62811 POINT(30.19720798106826 -95.17273189150522) bank62811 +62812 POINT(29.561708528197812 -94.53361822125142) bank62812 +62813 POINT(30.698423827602706 -95.9183414034468) bank62813 +62814 POINT(29.380769501962313 -95.78369748620307) bank62814 +62815 POINT(29.021647196726505 -94.64204495258761) bank62815 +62816 POINT(30.433376514480152 -96.06957753933608) bank62816 +62817 POINT(30.20681034936824 -95.39859381806356) bank62817 +62818 POINT(29.760782319142862 -95.32701593343776) bank62818 +62819 POINT(30.103943579843367 -94.55431902648769) bank62819 +62820 POINT(29.208114190804988 -95.34883405536334) bank62820 +62821 POINT(30.567768190357473 -96.13915662753293) bank62821 +62822 POINT(30.004305419895612 -95.35896884401112) bank62822 +62823 POINT(30.107824520650492 -95.3123714723361) bank62823 +62824 POINT(29.46988696986176 -94.6621823520637) bank62824 +62825 POINT(30.161727516326263 -95.75575360749724) bank62825 +62826 POINT(29.11215931512986 -94.77517345615362) bank62826 +62827 POINT(30.665015296929635 -94.51811819065806) bank62827 +62828 POINT(30.10525345510418 -95.16059137063901) bank62828 +62829 POINT(29.759862748631747 -95.43072319907468) bank62829 +62830 POINT(30.073938889998907 -96.26053223009923) bank62830 +62831 POINT(29.92195377856909 -95.8261479460151) bank62831 +62832 POINT(30.727939982405996 -95.99276354465363) bank62832 +62833 POINT(29.18675832602873 -95.64829258427021) bank62833 +62834 POINT(30.04399673625124 -96.09686583853052) bank62834 +62835 POINT(29.814617631638114 -95.61707281067255) bank62835 +62836 POINT(30.717453348139916 -95.52112155285248) bank62836 +62837 POINT(30.339734287927786 -95.32588540778164) bank62837 +62838 POINT(29.929792876809948 -96.09727136118332) bank62838 +62839 POINT(30.656288395698237 -94.44945845120698) bank62839 +62840 POINT(29.99062964054044 -95.38835158381445) bank62840 +62841 POINT(30.42057011416799 -94.90137775618973) bank62841 +62842 POINT(30.163697697150557 -94.9724619573186) bank62842 +62843 POINT(29.608382890931193 -94.95743297175632) bank62843 +62844 POINT(29.60362761098104 -94.76857015065997) bank62844 +62845 POINT(30.312476747854305 -96.21570729973931) bank62845 +62846 POINT(30.413365157759525 -95.41798273972198) bank62846 +62847 POINT(28.76315042906407 -96.04786152061334) bank62847 +62848 POINT(29.155811500251506 -95.08093685692053) bank62848 +62849 POINT(29.750951226401895 -95.16873889461367) bank62849 +62850 POINT(29.21915409190948 -94.64928926542211) bank62850 +62851 POINT(30.067254309502545 -95.00162239365541) bank62851 +62852 POINT(30.29449679163221 -95.08623136008529) bank62852 +62853 POINT(29.56511542769889 -94.39809700808274) bank62853 +62854 POINT(29.49932633832386 -95.23609261545808) bank62854 +62855 POINT(28.93638738738767 -94.64508642063142) bank62855 +62856 POINT(28.949560506431073 -95.41014069777881) bank62856 +62857 POINT(30.473020700595356 -95.99807530955023) bank62857 +62858 POINT(29.575628324014403 -96.19102581869272) bank62858 +62859 POINT(30.709082383167605 -95.35481883892149) bank62859 +62860 POINT(30.336485610922516 -95.69294888754153) bank62860 +62861 POINT(29.871131504249735 -95.67798804835581) bank62861 +62862 POINT(30.562434361022568 -95.00202529392324) bank62862 +62863 POINT(30.495479575670647 -95.03862336149882) bank62863 +62864 POINT(28.984185069350843 -94.43161536432045) bank62864 +62865 POINT(29.58496632974968 -96.34877207042543) bank62865 +62866 POINT(29.93260010702512 -94.71390901333885) bank62866 +62867 POINT(30.4839291264014 -94.39951870789325) bank62867 +62868 POINT(30.374703533428626 -96.00835134766659) bank62868 +62869 POINT(30.63639438211762 -95.47909697135722) bank62869 +62870 POINT(29.80254709167364 -94.5888448676927) bank62870 +62871 POINT(30.48910793097145 -95.37382999566132) bank62871 +62872 POINT(29.788971175130218 -94.8290705480957) bank62872 +62873 POINT(30.513982837206807 -94.5384698809951) bank62873 +62874 POINT(29.35619702037513 -95.96846184886809) bank62874 +62875 POINT(30.57052477222727 -95.84523598657645) bank62875 +62876 POINT(29.740198668960407 -94.48529114434385) bank62876 +62877 POINT(30.00981759435342 -96.18808161273103) bank62877 +62878 POINT(29.485643294420857 -95.13589789087855) bank62878 +62879 POINT(30.29025491224355 -95.69744598530421) bank62879 +62880 POINT(30.62460913311287 -94.57849468385277) bank62880 +62881 POINT(29.95468204063931 -95.17370900074933) bank62881 +62882 POINT(30.3324469177751 -94.70212609132489) bank62882 +62883 POINT(28.924609261021086 -95.61177522572973) bank62883 +62884 POINT(30.205991048195504 -95.57349304313229) bank62884 +62885 POINT(30.049008218527025 -95.46263585065108) bank62885 +62886 POINT(30.551560331996154 -95.36286605442385) bank62886 +62887 POINT(29.37055116012175 -94.41459346053142) bank62887 +62888 POINT(29.752147643930524 -96.22629781327892) bank62888 +62889 POINT(29.682931798733804 -95.3429748556627) bank62889 +62890 POINT(29.002533073820285 -94.40155865411242) bank62890 +62891 POINT(28.966092566375206 -95.80373566560124) bank62891 +62892 POINT(29.769168502252814 -95.90813899735886) bank62892 +62893 POINT(29.85787459481666 -95.20806456902622) bank62893 +62894 POINT(30.50220232481306 -94.420504561083) bank62894 +62895 POINT(30.74689256051278 -96.02570341596322) bank62895 +62896 POINT(30.37459013384804 -94.48771086300047) bank62896 +62897 POINT(29.48925291752596 -94.64941669030414) bank62897 +62898 POINT(30.491597541350533 -96.30114883127399) bank62898 +62899 POINT(29.712329129991815 -96.10133235734024) bank62899 +62900 POINT(30.691761212246895 -95.66218107234515) bank62900 +62901 POINT(30.478198484837918 -96.17763547474586) bank62901 +62902 POINT(30.449572804625006 -95.99618333237385) bank62902 +62903 POINT(30.030384553837933 -95.67920806312912) bank62903 +62904 POINT(30.643242076631633 -95.16851915308904) bank62904 +62905 POINT(28.941660773521942 -94.78842293532728) bank62905 +62906 POINT(29.55184728331986 -95.41504526954206) bank62906 +62907 POINT(29.074464477380097 -95.57686481225947) bank62907 +62908 POINT(30.044416639924158 -96.23647161666042) bank62908 +62909 POINT(29.42496598351401 -96.34074350792021) bank62909 +62910 POINT(28.91530601515764 -95.34288456725244) bank62910 +62911 POINT(30.45740938305723 -94.38777491486351) bank62911 +62912 POINT(30.154238455332337 -95.90713759515805) bank62912 +62913 POINT(29.633715158610205 -94.83570787213078) bank62913 +62914 POINT(29.720836091763992 -96.31375635620982) bank62914 +62915 POINT(29.237290428812223 -95.0804555857305) bank62915 +62916 POINT(30.393050237381672 -94.48807311588965) bank62916 +62917 POINT(29.696829708159637 -94.41056245016499) bank62917 +62918 POINT(30.200040096298636 -95.12193842903376) bank62918 +62919 POINT(29.18288245084393 -95.33627176046015) bank62919 +62920 POINT(29.101668020541275 -94.86984937578009) bank62920 +62921 POINT(30.27202464221487 -95.64032381777282) bank62921 +62922 POINT(29.426785724836854 -96.33268613496784) bank62922 +62923 POINT(29.240760860766425 -95.29773757175167) bank62923 +62924 POINT(29.341297688087547 -96.20763057735456) bank62924 +62925 POINT(30.532717093944797 -95.10650736999798) bank62925 +62926 POINT(29.27749058098321 -95.16264510146941) bank62926 +62927 POINT(30.66762180715387 -96.28948617102223) bank62927 +62928 POINT(28.90304163158467 -95.37218438821965) bank62928 +62929 POINT(30.08092035922698 -94.94077329765564) bank62929 +62930 POINT(29.595750861176604 -95.66597102879962) bank62930 +62931 POINT(30.376807166284735 -95.84158110765254) bank62931 +62932 POINT(30.675672774125612 -96.32066650777244) bank62932 +62933 POINT(29.414049964724324 -96.13121475600975) bank62933 +62934 POINT(30.565707116433067 -96.06499860047549) bank62934 +62935 POINT(29.28493851015797 -94.91988423556381) bank62935 +62936 POINT(30.1306218526524 -94.72864452565322) bank62936 +62937 POINT(29.821139632921142 -94.52755176315416) bank62937 +62938 POINT(29.37525883179212 -95.50954084535653) bank62938 +62939 POINT(30.10639812038252 -95.67534645450476) bank62939 +62940 POINT(28.8403387177827 -96.01081391120029) bank62940 +62941 POINT(29.269875927358044 -94.5161533436195) bank62941 +62942 POINT(29.7518527655196 -95.28359012761415) bank62942 +62943 POINT(29.428491042689135 -94.79349336383326) bank62943 +62944 POINT(29.658356821153504 -95.65013483945495) bank62944 +62945 POINT(29.92211109757946 -95.34079080370077) bank62945 +62946 POINT(29.06622238730912 -95.50939026144391) bank62946 +62947 POINT(28.83804912883933 -95.95928126213204) bank62947 +62948 POINT(28.935313340198558 -95.57067966540359) bank62948 +62949 POINT(28.946342031853057 -95.04782169131418) bank62949 +62950 POINT(29.408693955842004 -95.84323322586482) bank62950 +62951 POINT(30.221742344615702 -95.70582550887997) bank62951 +62952 POINT(29.80330806891292 -94.50630759508125) bank62952 +62953 POINT(30.372394490529608 -96.09610531220633) bank62953 +62954 POINT(30.162932112504546 -94.58010334968945) bank62954 +62955 POINT(29.834600348176984 -95.86743502005642) bank62955 +62956 POINT(29.594309055950525 -95.18644451390087) bank62956 +62957 POINT(29.18971842596843 -95.06662955236243) bank62957 +62958 POINT(29.078198709626 -94.38590517393436) bank62958 +62959 POINT(30.670766922072122 -95.32495298657945) bank62959 +62960 POINT(29.299372511738756 -95.15644887233228) bank62960 +62961 POINT(29.765885500857596 -95.52863736015586) bank62961 +62962 POINT(28.929038101941064 -95.69911928229536) bank62962 +62963 POINT(30.139255669223775 -96.27173614928203) bank62963 +62964 POINT(29.39196275751611 -95.09334016397881) bank62964 +62965 POINT(30.08902895191551 -94.8987482767121) bank62965 +62966 POINT(29.052610279612992 -95.0373006620956) bank62966 +62967 POINT(29.6448191418933 -95.6774954047244) bank62967 +62968 POINT(28.835604860026486 -96.00012492196981) bank62968 +62969 POINT(30.690577182352605 -94.56051142934368) bank62969 +62970 POINT(29.968495772249334 -94.92979874284924) bank62970 +62971 POINT(29.730078428525214 -96.1062748594316) bank62971 +62972 POINT(28.920508931965777 -94.91558996097119) bank62972 +62973 POINT(29.163874261626866 -95.59072006955246) bank62973 +62974 POINT(30.604065070290417 -96.16240105774943) bank62974 +62975 POINT(30.558108999175488 -94.64185948364457) bank62975 +62976 POINT(30.138475767013873 -94.73043743392316) bank62976 +62977 POINT(29.728568218223337 -95.12389323788162) bank62977 +62978 POINT(30.463855394415845 -94.79714084079225) bank62978 +62979 POINT(29.475880135793552 -94.85048318433448) bank62979 +62980 POINT(30.75191734887584 -94.90543535925126) bank62980 +62981 POINT(30.724482155881486 -95.49638130940131) bank62981 +62982 POINT(30.417577392329353 -95.5892154114271) bank62982 +62983 POINT(30.61357502830419 -95.98931469312694) bank62983 +62984 POINT(29.260122864673573 -94.56703893741184) bank62984 +62985 POINT(30.579866768720624 -95.04988193644434) bank62985 +62986 POINT(29.229273681823642 -95.02334822623608) bank62986 +62987 POINT(30.208556620397662 -96.24519090099491) bank62987 +62988 POINT(29.21450480553998 -94.95278761229534) bank62988 +62989 POINT(29.06221367183619 -96.36847000214898) bank62989 +62990 POINT(29.137114788339186 -96.36656078185882) bank62990 +62991 POINT(29.230180080074472 -95.69582377192776) bank62991 +62992 POINT(29.899302974677383 -94.73534960732121) bank62992 +62993 POINT(30.627830848592122 -94.61615365854611) bank62993 +62994 POINT(29.0411223080161 -95.33311646066468) bank62994 +62995 POINT(29.783947563571203 -94.77431769324383) bank62995 +62996 POINT(29.03697954815061 -96.14823297108889) bank62996 +62997 POINT(28.943971391614397 -95.94210992373276) bank62997 +62998 POINT(29.885008441788546 -94.84885845422924) bank62998 +62999 POINT(28.821089909832306 -95.09475096090914) bank62999 +63000 POINT(29.68055519225081 -95.29372375497093) bank63000 +63001 POINT(30.530600423412064 -95.19808163115573) bank63001 +63002 POINT(30.740236953002263 -95.28480346807238) bank63002 +63003 POINT(29.4190807509502 -95.39332308373356) bank63003 +63004 POINT(30.497277434539466 -94.66894467657399) bank63004 +63005 POINT(30.612719775263063 -95.95656060634207) bank63005 +63006 POINT(30.279538743798266 -96.14844857101949) bank63006 +63007 POINT(28.83579647354551 -95.36958191255212) bank63007 +63008 POINT(30.66651938195057 -94.55331027896693) bank63008 +63009 POINT(28.954684620232406 -96.1206219785979) bank63009 +63010 POINT(28.78875598075557 -94.69474407536491) bank63010 +63011 POINT(29.08195921801446 -94.87837657987691) bank63011 +63012 POINT(28.94343021124829 -95.44487611159636) bank63012 +63013 POINT(30.614814580831457 -94.50009837356981) bank63013 +63014 POINT(29.42687194556799 -95.83181839749454) bank63014 +63015 POINT(29.7273168431664 -95.2266797435111) bank63015 +63016 POINT(29.131155442214066 -94.47704129489264) bank63016 +63017 POINT(30.462678412152066 -94.38523799126213) bank63017 +63018 POINT(28.979965774892214 -96.23657932299922) bank63018 +63019 POINT(30.330851225475104 -95.04092174401627) bank63019 +63020 POINT(29.753391153127552 -96.11926499620472) bank63020 +63021 POINT(30.490582570405525 -95.50720717902828) bank63021 +63022 POINT(30.25948510083736 -94.43337193894791) bank63022 +63023 POINT(28.86208575335877 -95.16376520552696) bank63023 +63024 POINT(29.655094608936263 -95.75448471675568) bank63024 +63025 POINT(29.691093025554117 -94.44761033106097) bank63025 +63026 POINT(28.856256939257797 -94.73064444750462) bank63026 +63027 POINT(30.521508659750562 -95.89840698003594) bank63027 +63028 POINT(29.17863078618417 -95.03094058320391) bank63028 +63029 POINT(29.29047964335665 -95.8411115008643) bank63029 +63030 POINT(30.444520736820504 -94.91248873866932) bank63030 +63031 POINT(29.502016226718165 -95.44328222668975) bank63031 +63032 POINT(29.398590893268803 -95.81772839439428) bank63032 +63033 POINT(30.75713591935299 -95.69842411816009) bank63033 +63034 POINT(29.48920087782377 -95.36932319192731) bank63034 +63035 POINT(30.72572825478117 -95.82131125684404) bank63035 +63036 POINT(29.847400874371633 -94.47712050082005) bank63036 +63037 POINT(30.012298078695743 -94.49471048323548) bank63037 +63038 POINT(29.666017489690315 -96.04718231683461) bank63038 +63039 POINT(30.469302294479398 -94.65767364557149) bank63039 +63040 POINT(30.43601369026627 -96.17155071648698) bank63040 +63041 POINT(30.252024740208565 -94.78776489292046) bank63041 +63042 POINT(29.202939882181415 -96.12553098758286) bank63042 +63043 POINT(29.392554821787897 -95.53802719789971) bank63043 +63044 POINT(30.102620937910594 -94.85324074910874) bank63044 +63045 POINT(29.528675946803986 -96.23194609552193) bank63045 +63046 POINT(28.8622029087671 -95.31443466463324) bank63046 +63047 POINT(30.121908449818974 -95.93476856564827) bank63047 +63048 POINT(28.871723390102687 -94.647188549352) bank63048 +63049 POINT(29.242429503128246 -95.79829498520422) bank63049 +63050 POINT(28.947371328238862 -96.03099128420858) bank63050 +63051 POINT(30.66502634652101 -95.29263987211382) bank63051 +63052 POINT(29.539199280389525 -96.02314867877926) bank63052 +63053 POINT(29.549161966272727 -94.73927342684237) bank63053 +63054 POINT(28.985896268604712 -95.16261435843232) bank63054 +63055 POINT(29.026343614593205 -96.01183357108354) bank63055 +63056 POINT(30.46989285169808 -94.44225514978761) bank63056 +63057 POINT(30.610086999395715 -95.42810559921348) bank63057 +63058 POINT(29.380536903228403 -95.59370772094671) bank63058 +63059 POINT(29.36138842914863 -95.9675193221797) bank63059 +63060 POINT(30.18445549608136 -94.51931002004413) bank63060 +63061 POINT(28.7719720405797 -95.83003609950813) bank63061 +63062 POINT(29.310863506607014 -94.88121260193053) bank63062 +63063 POINT(30.091921496546895 -96.19735302108367) bank63063 +63064 POINT(30.311006009615394 -95.02032508758786) bank63064 +63065 POINT(28.79724520744945 -95.2486737170626) bank63065 +63066 POINT(29.429775891163473 -95.73238914919514) bank63066 +63067 POINT(29.286691802920153 -95.85997932463908) bank63067 +63068 POINT(30.372393909408043 -95.7530952389386) bank63068 +63069 POINT(30.021605055653684 -95.54764069379418) bank63069 +63070 POINT(29.46340462791055 -94.97386459787826) bank63070 +63071 POINT(30.474827551063562 -94.77890354115712) bank63071 +63072 POINT(29.74636068388367 -95.01283130385896) bank63072 +63073 POINT(28.976507590844715 -95.69401318978264) bank63073 +63074 POINT(30.180080205304535 -95.9542528988094) bank63074 +63075 POINT(29.36028075918773 -96.31550141630802) bank63075 +63076 POINT(29.594115277866223 -94.37503144735801) bank63076 +63077 POINT(29.11875647773885 -94.89845033705025) bank63077 +63078 POINT(30.147712532923418 -95.62805655296572) bank63078 +63079 POINT(30.293273186666372 -94.40803343506596) bank63079 +63080 POINT(30.681057866532456 -95.45349214140006) bank63080 +63081 POINT(29.731057043714138 -94.76549126467832) bank63081 +63082 POINT(29.87431466465781 -95.11335741268724) bank63082 +63083 POINT(28.851826256717445 -95.54354451427373) bank63083 +63084 POINT(29.52313293645363 -96.14503929653003) bank63084 +63085 POINT(30.308155126881108 -95.13382026456686) bank63085 +63086 POINT(29.76932014167402 -96.20929426897288) bank63086 +63087 POINT(30.45574931073367 -96.21260128174848) bank63087 +63088 POINT(29.90221144306596 -95.60096694228174) bank63088 +63089 POINT(30.366327983254386 -95.2043813995661) bank63089 +63090 POINT(30.567944323056526 -96.24179091359387) bank63090 +63091 POINT(28.989168398303818 -96.26630474222745) bank63091 +63092 POINT(30.706179408017153 -95.1988983420133) bank63092 +63093 POINT(30.584368300998534 -94.8535176585406) bank63093 +63094 POINT(30.085283830483306 -94.51313743597242) bank63094 +63095 POINT(29.1478042158653 -96.33964792507192) bank63095 +63096 POINT(29.948067262764148 -95.8484359632464) bank63096 +63097 POINT(30.582823930092346 -94.9145086197842) bank63097 +63098 POINT(28.924907029850168 -95.29486236211604) bank63098 +63099 POINT(29.7452636167289 -95.91614822762294) bank63099 +63100 POINT(30.284342629194203 -96.06214097839633) bank63100 +63101 POINT(29.676122399189506 -94.44633812323457) bank63101 +63102 POINT(29.665364569223573 -95.01761653085607) bank63102 +63103 POINT(29.465973313015358 -95.19185925980858) bank63103 +63104 POINT(29.767964932211804 -95.6361955564808) bank63104 +63105 POINT(30.47676868928811 -95.22394630575612) bank63105 +63106 POINT(30.464833509359114 -95.54520982425251) bank63106 +63107 POINT(30.69676105185043 -95.8116167449759) bank63107 +63108 POINT(30.126774108371528 -95.39130659224449) bank63108 +63109 POINT(30.180356183825328 -95.50555879336828) bank63109 +63110 POINT(28.83049853994474 -95.99583929793023) bank63110 +63111 POINT(30.552996148826654 -94.78577121579397) bank63111 +63112 POINT(29.070215721273268 -95.73931019240511) bank63112 +63113 POINT(30.52566983608795 -94.70822025935868) bank63113 +63114 POINT(30.637029272905608 -95.16629327903115) bank63114 +63115 POINT(29.60602026700533 -95.68754346187997) bank63115 +63116 POINT(29.026825335693726 -94.75391310348368) bank63116 +63117 POINT(28.91950710412435 -94.59418039843445) bank63117 +63118 POINT(29.253717034571046 -95.01049109200753) bank63118 +63119 POINT(29.663604212578775 -96.04499466401177) bank63119 +63120 POINT(29.490623280993542 -94.89047272917306) bank63120 +63121 POINT(29.21726111385895 -95.28599009237794) bank63121 +63122 POINT(29.011868450254976 -96.30876424362852) bank63122 +63123 POINT(30.257815652147183 -94.57509102678863) bank63123 +63124 POINT(29.680981759165114 -96.03885097621973) bank63124 +63125 POINT(29.46046841127748 -95.81192247665403) bank63125 +63126 POINT(29.07553021817302 -95.62272566988189) bank63126 +63127 POINT(30.370444341441168 -96.07244711058978) bank63127 +63128 POINT(30.507097559390665 -95.48102595392133) bank63128 +63129 POINT(30.676448893977316 -95.2594663685522) bank63129 +63130 POINT(30.673555239423816 -95.15537549687043) bank63130 +63131 POINT(29.463088153618035 -95.80933417904772) bank63131 +63132 POINT(30.453701287218983 -96.09393439749648) bank63132 +63133 POINT(30.146524608542016 -94.48187639803375) bank63133 +63134 POINT(28.779509782652053 -96.01398976907815) bank63134 +63135 POINT(29.49390811807073 -94.90457110917619) bank63135 +63136 POINT(29.37766096738901 -95.4197116648534) bank63136 +63137 POINT(28.816094479149985 -94.66912505126624) bank63137 +63138 POINT(29.585621274007114 -94.53579717836578) bank63138 +63139 POINT(30.295386037260506 -96.01958645731534) bank63139 +63140 POINT(30.073566530557038 -95.96294004792591) bank63140 +63141 POINT(28.794418523203078 -94.96972511492058) bank63141 +63142 POINT(30.277478704397506 -95.66822218768063) bank63142 +63143 POINT(29.33134737380513 -94.59200404526399) bank63143 +63144 POINT(28.977110483298457 -94.37026496517815) bank63144 +63145 POINT(30.413506894341808 -95.18600214076697) bank63145 +63146 POINT(29.192611452798896 -96.02058637638223) bank63146 +63147 POINT(29.083105239121803 -94.61370409005023) bank63147 +63148 POINT(29.610274878374344 -95.96181942083759) bank63148 +63149 POINT(30.338608261198335 -96.11123614125117) bank63149 +63150 POINT(29.656423311694926 -96.05047155918093) bank63150 +63151 POINT(29.84757671031937 -95.23976650534586) bank63151 +63152 POINT(30.610823365119526 -94.5343029504106) bank63152 +63153 POINT(30.62435948958721 -95.58419592036188) bank63153 +63154 POINT(29.451426717147463 -95.00176036277782) bank63154 +63155 POINT(29.051789491887 -95.60488140308858) bank63155 +63156 POINT(29.024679195488027 -94.9266920724452) bank63156 +63157 POINT(30.520210072161206 -96.26527457822384) bank63157 +63158 POINT(29.47901918419102 -94.54916834990348) bank63158 +63159 POINT(28.767029784936717 -96.198694630233) bank63159 +63160 POINT(29.23096085461194 -95.76383270852244) bank63160 +63161 POINT(30.37822802858686 -94.906622235882) bank63161 +63162 POINT(28.938297130214742 -96.30415243177602) bank63162 +63163 POINT(29.290127488874397 -95.2424946867426) bank63163 +63164 POINT(30.63451624493412 -96.14495407013538) bank63164 +63165 POINT(29.88756197893733 -96.00487288670102) bank63165 +63166 POINT(29.281001888589564 -96.04938014049014) bank63166 +63167 POINT(28.880749905552346 -94.54170033324422) bank63167 +63168 POINT(29.385610907806917 -95.57272223432737) bank63168 +63169 POINT(29.675397810244508 -95.14678774361835) bank63169 +63170 POINT(29.013137747438158 -94.4739944263508) bank63170 +63171 POINT(29.7497508960986 -96.25723775027761) bank63171 +63172 POINT(29.578139216676238 -96.17522302102978) bank63172 +63173 POINT(29.247802601779647 -94.90271348856145) bank63173 +63174 POINT(29.51789189776711 -96.18824570971017) bank63174 +63175 POINT(29.664890780977412 -94.88518843054256) bank63175 +63176 POINT(29.90920535350186 -95.66895765196155) bank63176 +63177 POINT(30.314709432514874 -95.86324452584469) bank63177 +63178 POINT(29.092183584987566 -95.35437120277298) bank63178 +63179 POINT(29.1119454823068 -94.44303060997554) bank63179 +63180 POINT(28.886600665017422 -96.365529000066) bank63180 +63181 POINT(29.838021773498706 -96.1757799023763) bank63181 +63182 POINT(29.508848248777774 -94.61780376916231) bank63182 +63183 POINT(30.41558275044435 -96.3588890159914) bank63183 +63184 POINT(29.595011983829682 -94.89792758368195) bank63184 +63185 POINT(28.89435293625006 -96.18767162007438) bank63185 +63186 POINT(29.44826781478048 -96.35094314773667) bank63186 +63187 POINT(29.311350528196396 -96.36755196504382) bank63187 +63188 POINT(30.724897242822756 -95.62141527906513) bank63188 +63189 POINT(29.031562034596615 -94.65114351472893) bank63189 +63190 POINT(29.579539913144664 -96.35789431848504) bank63190 +63191 POINT(29.72782677332201 -95.18073879968236) bank63191 +63192 POINT(29.00166254901472 -95.41522388996599) bank63192 +63193 POINT(29.953402228568738 -95.86528117587) bank63193 +63194 POINT(30.576339973745814 -94.9751635014614) bank63194 +63195 POINT(28.859061019183294 -94.71569340837023) bank63195 +63196 POINT(29.27647378534536 -94.92148228238605) bank63196 +63197 POINT(28.8614842128274 -96.30620877489827) bank63197 +63198 POINT(30.757555140794377 -94.92621155675488) bank63198 +63199 POINT(29.637938343037906 -95.76892373081377) bank63199 +63200 POINT(30.642458988928965 -96.22794271848556) bank63200 +63201 POINT(29.903952154412785 -95.81503323268883) bank63201 +63202 POINT(30.70984394262844 -94.80029252386299) bank63202 +63203 POINT(29.996570391977574 -96.14308508343662) bank63203 +63204 POINT(30.392907893627896 -94.65845842121506) bank63204 +63205 POINT(30.400325477124433 -94.87146995408861) bank63205 +63206 POINT(30.27381881910027 -95.29352673975497) bank63206 +63207 POINT(29.07556259730664 -94.58152622348152) bank63207 +63208 POINT(28.88003554781964 -95.95586271797609) bank63208 +63209 POINT(29.240911829647413 -95.630253977996) bank63209 +63210 POINT(29.893204940248346 -96.16968949648721) bank63210 +63211 POINT(29.10774475336777 -95.89478074465677) bank63211 +63212 POINT(29.95849798441777 -95.76538625666261) bank63212 +63213 POINT(28.86781996386607 -95.7903382646803) bank63213 +63214 POINT(30.308070318492412 -95.15024487221184) bank63214 +63215 POINT(30.572777631247195 -95.12418071773953) bank63215 +63216 POINT(29.463273886979987 -95.68991174380784) bank63216 +63217 POINT(29.091151517501036 -95.20958431500911) bank63217 +63218 POINT(29.56335159077925 -96.23345347523258) bank63218 +63219 POINT(29.96366230689438 -96.3471568712703) bank63219 +63220 POINT(29.071438157494782 -96.01198092538344) bank63220 +63221 POINT(29.466847061997516 -94.98419117669212) bank63221 +63222 POINT(29.804813820094434 -94.74609987157068) bank63222 +63223 POINT(30.077278643205176 -94.88042725795516) bank63223 +63224 POINT(28.789318425182714 -94.9200502101129) bank63224 +63225 POINT(30.426414472452866 -95.1284963621637) bank63225 +63226 POINT(30.410747109462346 -95.6687328509008) bank63226 +63227 POINT(30.510933059354656 -94.39697356153285) bank63227 +63228 POINT(29.838220814775703 -95.37827868430729) bank63228 +63229 POINT(30.75932588877759 -94.86487590995897) bank63229 +63230 POINT(30.377336895416576 -94.53918771667817) bank63230 +63231 POINT(30.291764076990376 -95.83256303669928) bank63231 +63232 POINT(28.99325401537677 -95.0296388171517) bank63232 +63233 POINT(29.538208923501557 -94.5598833117549) bank63233 +63234 POINT(29.40105264833388 -95.84759277539327) bank63234 +63235 POINT(29.80183050986482 -94.7679306834368) bank63235 +63236 POINT(29.063689638798234 -95.12882324145711) bank63236 +63237 POINT(29.038921680657793 -94.9501634384515) bank63237 +63238 POINT(29.254465346248814 -95.13810092757394) bank63238 +63239 POINT(28.99859447921786 -94.663200297747) bank63239 +63240 POINT(29.270363992452236 -94.5256219489782) bank63240 +63241 POINT(29.313719873532065 -94.72151770724604) bank63241 +63242 POINT(29.28789646571451 -95.06073048798473) bank63242 +63243 POINT(29.250547507088108 -95.62141656210913) bank63243 +63244 POINT(28.83357080981887 -94.48399068184236) bank63244 +63245 POINT(30.202851869281076 -95.59078035865116) bank63245 +63246 POINT(29.829052577443683 -95.95335245408137) bank63246 +63247 POINT(30.421526905632206 -95.87869674619954) bank63247 +63248 POINT(29.599753281788125 -94.79135693775793) bank63248 +63249 POINT(30.74537724559147 -94.40372146172432) bank63249 +63250 POINT(30.052320976905467 -95.96786775895698) bank63250 +63251 POINT(29.1407835399437 -96.01883910037907) bank63251 +63252 POINT(30.186238130742897 -94.88244772425466) bank63252 +63253 POINT(29.00100422312481 -94.45103191901737) bank63253 +63254 POINT(30.236295006446568 -95.29525238981583) bank63254 +63255 POINT(29.45278487249919 -95.74571942152355) bank63255 +63256 POINT(30.011270794017925 -95.5047505479746) bank63256 +63257 POINT(29.803444626949148 -96.03743476137491) bank63257 +63258 POINT(30.612096036297267 -94.96161718680129) bank63258 +63259 POINT(29.31371884153048 -94.68111071891933) bank63259 +63260 POINT(30.021105215469635 -94.526984004149) bank63260 +63261 POINT(29.27181412989341 -95.52101630826463) bank63261 +63262 POINT(29.39220009980786 -95.47688591211617) bank63262 +63263 POINT(30.69901092132822 -94.61826424359435) bank63263 +63264 POINT(29.55313822656859 -95.89698028632769) bank63264 +63265 POINT(29.04914547764781 -94.4241305266905) bank63265 +63266 POINT(30.36690339770576 -94.58657405066164) bank63266 +63267 POINT(29.129523768365726 -95.04435162211641) bank63267 +63268 POINT(28.9081446415543 -95.34935854584629) bank63268 +63269 POINT(29.9165328637285 -95.89756266943351) bank63269 +63270 POINT(29.336034023621064 -96.14305740688116) bank63270 +63271 POINT(29.80715497098601 -94.70051706295625) bank63271 +63272 POINT(30.28271313671387 -95.61116166229458) bank63272 +63273 POINT(29.400615062056207 -94.58934306331673) bank63273 +63274 POINT(29.295591983926144 -94.53828791834631) bank63274 +63275 POINT(28.91533058514366 -95.66992480978712) bank63275 +63276 POINT(29.12821842988101 -94.42317371699419) bank63276 +63277 POINT(29.31197995717239 -94.9622496688708) bank63277 +63278 POINT(29.70650027199421 -95.98546071401753) bank63278 +63279 POINT(29.19404580545253 -94.96043281673492) bank63279 +63280 POINT(29.340414595789664 -96.28509448896241) bank63280 +63281 POINT(30.061166936827767 -95.57250295222015) bank63281 +63282 POINT(29.618944260828428 -94.42084539248967) bank63282 +63283 POINT(30.364563991874665 -94.52979273872272) bank63283 +63284 POINT(30.572495059642225 -96.15416763764499) bank63284 +63285 POINT(28.95342604450249 -95.64523696790414) bank63285 +63286 POINT(30.453950337424946 -94.92200518030461) bank63286 +63287 POINT(29.763198029943435 -96.33153010311636) bank63287 +63288 POINT(30.626966600869874 -95.2818697239707) bank63288 +63289 POINT(30.14794081969108 -94.57953491458925) bank63289 +63290 POINT(30.488463095513012 -95.11207967859391) bank63290 +63291 POINT(30.197557133082523 -94.48514585727014) bank63291 +63292 POINT(30.55369052864003 -95.31849827642097) bank63292 +63293 POINT(29.450863575207343 -96.25107832041111) bank63293 +63294 POINT(29.947748803209468 -94.55791251248829) bank63294 +63295 POINT(30.190640987577645 -95.30169248104484) bank63295 +63296 POINT(30.37697115928544 -95.91412027054697) bank63296 +63297 POINT(28.806284325926036 -94.93916806663653) bank63297 +63298 POINT(29.4602825306805 -94.49747284062104) bank63298 +63299 POINT(29.54663960581908 -94.98583102406187) bank63299 +63300 POINT(29.916593128207957 -95.11796649363032) bank63300 +63301 POINT(29.02929994370143 -96.35248098085745) bank63301 +63302 POINT(30.02307826309391 -95.90128437445229) bank63302 +63303 POINT(29.89154497368774 -96.0175074137005) bank63303 +63304 POINT(29.724854842105117 -96.07233640179216) bank63304 +63305 POINT(29.910621873642754 -94.64912031790057) bank63305 +63306 POINT(29.745028819596033 -94.49789924536455) bank63306 +63307 POINT(29.15672387951839 -95.37369999586869) bank63307 +63308 POINT(30.033155470785356 -94.85751628967851) bank63308 +63309 POINT(30.54676733158086 -94.80363470339067) bank63309 +63310 POINT(30.424194598615227 -95.61291142079735) bank63310 +63311 POINT(28.935563123792413 -94.50682619263065) bank63311 +63312 POINT(29.721584711965892 -94.72706449977527) bank63312 +63313 POINT(30.57605902316627 -95.27572929707502) bank63313 +63314 POINT(30.60901055211733 -94.67336294494207) bank63314 +63315 POINT(30.16328404253265 -94.75124576564701) bank63315 +63316 POINT(28.89878376120718 -96.10144330119715) bank63316 +63317 POINT(28.828626068995955 -94.68850987170427) bank63317 +63318 POINT(29.31868276268718 -94.6436528556156) bank63318 +63319 POINT(29.850321383741488 -94.83650210778903) bank63319 +63320 POINT(29.582554820137474 -96.22800274571104) bank63320 +63321 POINT(30.627182439385788 -95.09501694327439) bank63321 +63322 POINT(30.166629324004337 -95.09234240981094) bank63322 +63323 POINT(30.012525212509036 -96.13912235470593) bank63323 +63324 POINT(29.235431804238917 -95.91455173638116) bank63324 +63325 POINT(30.640436367300474 -96.34923621643242) bank63325 +63326 POINT(29.418805125826506 -96.26612053067969) bank63326 +63327 POINT(29.04326530358234 -95.74240092712388) bank63327 +63328 POINT(30.455148566860874 -95.94124899869931) bank63328 +63329 POINT(29.630571373069493 -95.7712827910623) bank63329 +63330 POINT(29.10591305881912 -95.95069514745258) bank63330 +63331 POINT(29.93790162144551 -95.88413546531149) bank63331 +63332 POINT(29.11659137105147 -95.02469676146515) bank63332 +63333 POINT(29.528898982477052 -94.80820827017361) bank63333 +63334 POINT(30.26163840000755 -95.95909911823574) bank63334 +63335 POINT(29.463550167510956 -95.37321871635862) bank63335 +63336 POINT(30.300853420964987 -95.25096110812663) bank63336 +63337 POINT(30.20924647479315 -96.17053472793091) bank63337 +63338 POINT(30.68066033431531 -96.16091904405317) bank63338 +63339 POINT(30.38666366461923 -94.74870803269245) bank63339 +63340 POINT(30.576188731866875 -95.68263042364197) bank63340 +63341 POINT(29.526970279916526 -96.12865953192876) bank63341 +63342 POINT(29.01472258570152 -96.23585144751566) bank63342 +63343 POINT(28.912282322837093 -96.3450455330796) bank63343 +63344 POINT(29.827503419915594 -94.42407858558941) bank63344 +63345 POINT(28.78422741213671 -94.58664167308821) bank63345 +63346 POINT(28.81191754052703 -95.25553499227705) bank63346 +63347 POINT(30.336310219791173 -94.55582192332099) bank63347 +63348 POINT(29.05633595105788 -95.77755533328441) bank63348 +63349 POINT(30.32376140958459 -95.4809939016958) bank63349 +63350 POINT(30.229614774041167 -94.8589144666956) bank63350 +63351 POINT(29.409790030143522 -94.77341491819357) bank63351 +63352 POINT(29.839768683141134 -95.27884414016586) bank63352 +63353 POINT(30.173909021167745 -96.10239542041998) bank63353 +63354 POINT(29.651542770995032 -95.84579249864655) bank63354 +63355 POINT(28.982695123003126 -95.63906641488843) bank63355 +63356 POINT(30.73085629336443 -95.47430548584713) bank63356 +63357 POINT(29.668467545273764 -94.68045652748556) bank63357 +63358 POINT(30.4187913506769 -95.90735720936365) bank63358 +63359 POINT(28.77906716248249 -95.37944263793104) bank63359 +63360 POINT(29.665728522060515 -94.49484982538114) bank63360 +63361 POINT(30.423542649050137 -94.78759519771648) bank63361 +63362 POINT(28.764430811414947 -94.6654523728363) bank63362 +63363 POINT(29.49875290027144 -96.01923773492713) bank63363 +63364 POINT(29.740450035164148 -95.28530219991771) bank63364 +63365 POINT(29.3534206209057 -94.8867818768712) bank63365 +63366 POINT(30.591526792895532 -96.04293836117536) bank63366 +63367 POINT(29.502129168353374 -95.43199678501526) bank63367 +63368 POINT(30.603696169196645 -95.48789624445917) bank63368 +63369 POINT(29.393670507609855 -94.85124548425661) bank63369 +63370 POINT(29.95312386730601 -95.62712134355952) bank63370 +63371 POINT(29.703335538703033 -95.653161868662) bank63371 +63372 POINT(29.508259461011072 -94.64901138867614) bank63372 +63373 POINT(30.112224048369693 -94.91069560501333) bank63373 +63374 POINT(29.824477881969013 -95.26270670733486) bank63374 +63375 POINT(28.949099469760355 -95.28545765306164) bank63375 +63376 POINT(30.398654542981202 -95.85736285021471) bank63376 +63377 POINT(30.033779017056652 -96.02082444378733) bank63377 +63378 POINT(29.93976016597571 -95.41537242039254) bank63378 +63379 POINT(28.857160193444155 -95.26681604732472) bank63379 +63380 POINT(29.598787337116153 -95.05774908599338) bank63380 +63381 POINT(30.54788773530013 -95.86197053642121) bank63381 +63382 POINT(30.503368112816656 -95.70507947974413) bank63382 +63383 POINT(29.49760734303568 -96.36670040973343) bank63383 +63384 POINT(30.41788114187713 -94.89178679961105) bank63384 +63385 POINT(30.49643921715667 -94.86976496295075) bank63385 +63386 POINT(29.43266890368783 -95.76212203063545) bank63386 +63387 POINT(28.986632247516347 -96.18916973407124) bank63387 +63388 POINT(30.567080250864773 -96.31014672832977) bank63388 +63389 POINT(29.093470068963732 -94.37857835200462) bank63389 +63390 POINT(30.289500069609446 -95.65199145505049) bank63390 +63391 POINT(29.655624939239875 -95.944391191855) bank63391 +63392 POINT(30.17644698792136 -95.74534431574507) bank63392 +63393 POINT(29.06242454191618 -95.59715566353101) bank63393 +63394 POINT(29.563072732004997 -94.88033065181959) bank63394 +63395 POINT(29.745139276315122 -95.34810557718662) bank63395 +63396 POINT(29.947393355982044 -94.89056575773905) bank63396 +63397 POINT(29.584794339031767 -95.66250021592953) bank63397 +63398 POINT(29.73782512620853 -94.93075982982448) bank63398 +63399 POINT(29.590422972408952 -95.22243167129952) bank63399 +63400 POINT(29.597502748429648 -94.93545221983914) bank63400 +63401 POINT(29.08246343444747 -94.84126404781217) bank63401 +63402 POINT(30.11110568191225 -95.20310864994916) bank63402 +63403 POINT(29.970237812610957 -95.74556161475338) bank63403 +63404 POINT(28.952897362126983 -95.89730277007492) bank63404 +63405 POINT(29.048129892718208 -95.17707597992944) bank63405 +63406 POINT(29.819307812372536 -95.92466031529997) bank63406 +63407 POINT(30.49551482212568 -94.41273726420813) bank63407 +63408 POINT(29.264741202015085 -96.14069534775277) bank63408 +63409 POINT(30.054316490407754 -95.8086338481084) bank63409 +63410 POINT(30.497319956422572 -96.19662438504379) bank63410 +63411 POINT(29.009951235753967 -95.4190570387736) bank63411 +63412 POINT(30.26550811261414 -95.77886671706362) bank63412 +63413 POINT(30.509700694149153 -95.57259806436494) bank63413 +63414 POINT(30.329248715135062 -95.45336172596062) bank63414 +63415 POINT(30.734034236070467 -95.87523983148161) bank63415 +63416 POINT(30.394505407621345 -95.04948765825661) bank63416 +63417 POINT(29.799905356217447 -95.06511064599115) bank63417 +63418 POINT(30.099105163100525 -95.85883059907925) bank63418 +63419 POINT(29.560423117122 -95.88745856429641) bank63419 +63420 POINT(29.26817727927482 -94.9505947433166) bank63420 +63421 POINT(29.486518331267778 -95.82933467143093) bank63421 +63422 POINT(30.464156218299486 -94.79752432722387) bank63422 +63423 POINT(29.038130253509944 -96.09987828411228) bank63423 +63424 POINT(29.148144845653817 -94.60036513083075) bank63424 +63425 POINT(30.2521533540386 -96.28322849495234) bank63425 +63426 POINT(29.193130552815806 -95.94461493116759) bank63426 +63427 POINT(29.55817298022399 -96.05415139050582) bank63427 +63428 POINT(30.393840616094494 -95.18821126151285) bank63428 +63429 POINT(29.16992571511015 -95.80049278974374) bank63429 +63430 POINT(29.878185442321836 -94.99122065391815) bank63430 +63431 POINT(29.953753165157803 -95.3029349518731) bank63431 +63432 POINT(29.678929990009586 -94.48101842176338) bank63432 +63433 POINT(28.960432825858447 -94.68185071947735) bank63433 +63434 POINT(29.24930565037447 -94.82491722816444) bank63434 +63435 POINT(30.45905010729584 -96.2712887657542) bank63435 +63436 POINT(30.532412771476267 -94.57389372934824) bank63436 +63437 POINT(30.676296896155787 -95.15915584562443) bank63437 +63438 POINT(28.813977191689116 -95.87694861768495) bank63438 +63439 POINT(29.88890369770888 -95.93788701734134) bank63439 +63440 POINT(28.986734617850345 -95.25758395417732) bank63440 +63441 POINT(29.637829549463078 -95.84695920541992) bank63441 +63442 POINT(29.48137158807266 -95.8172496250827) bank63442 +63443 POINT(30.585320834632242 -96.13355398073678) bank63443 +63444 POINT(28.83216336003645 -95.90401448731912) bank63444 +63445 POINT(29.481831304462467 -96.2579479490079) bank63445 +63446 POINT(29.647247864521315 -95.04533266788565) bank63446 +63447 POINT(29.901569055074724 -96.20559509046166) bank63447 +63448 POINT(30.679412415660238 -95.92221761703048) bank63448 +63449 POINT(29.201877292501788 -94.63699932288164) bank63449 +63450 POINT(29.308001920769904 -94.77317139924479) bank63450 +63451 POINT(30.666096327495588 -95.57205955006003) bank63451 +63452 POINT(29.278648628639356 -94.66008993683309) bank63452 +63453 POINT(30.157662938019662 -94.71748209836643) bank63453 +63454 POINT(29.32296842879778 -94.86723134062902) bank63454 +63455 POINT(30.247871969495293 -96.18520040957956) bank63455 +63456 POINT(30.758841113083086 -95.66055143351332) bank63456 +63457 POINT(29.24847675145523 -94.88917963401312) bank63457 +63458 POINT(29.41987886221754 -94.67541346428567) bank63458 +63459 POINT(29.13225791503741 -95.7851033638158) bank63459 +63460 POINT(29.418991996293897 -94.63678986933496) bank63460 +63461 POINT(29.00988745389664 -96.22085024049865) bank63461 +63462 POINT(29.271173823447366 -94.39834658611056) bank63462 +63463 POINT(29.16710339418623 -94.89640797702083) bank63463 +63464 POINT(29.829852412667474 -94.52223382746224) bank63464 +63465 POINT(29.329867316789198 -95.14673348654406) bank63465 +63466 POINT(29.12852426344616 -95.08270391120288) bank63466 +63467 POINT(29.402204371683094 -94.39837406434735) bank63467 +63468 POINT(28.954357214344093 -94.66256688664141) bank63468 +63469 POINT(28.930282462449394 -94.83633465972471) bank63469 +63470 POINT(29.492342457117037 -95.93032021268651) bank63470 +63471 POINT(30.061618087709913 -95.8300097474847) bank63471 +63472 POINT(29.38129121661137 -94.48064970581761) bank63472 +63473 POINT(30.070030289181606 -95.3391711811025) bank63473 +63474 POINT(28.784268771259043 -95.53912257839545) bank63474 +63475 POINT(30.105280185740128 -95.44207253746532) bank63475 +63476 POINT(28.963876858158915 -94.6527018283764) bank63476 +63477 POINT(29.640880306165243 -94.40699569747999) bank63477 +63478 POINT(29.480386670899986 -95.26526169816545) bank63478 +63479 POINT(30.40972572525496 -96.12541178363851) bank63479 +63480 POINT(29.122952749352912 -94.88731087981623) bank63480 +63481 POINT(29.34945031432179 -94.5849699894837) bank63481 +63482 POINT(30.70123361801672 -96.21073915490447) bank63482 +63483 POINT(29.453065907454935 -95.07635534383967) bank63483 +63484 POINT(29.645285271012774 -94.6913720052469) bank63484 +63485 POINT(30.5386559269096 -96.25535902837849) bank63485 +63486 POINT(29.769169683862536 -94.55250765787417) bank63486 +63487 POINT(30.281297583876952 -94.8497889685273) bank63487 +63488 POINT(29.374179599032793 -95.5520217571945) bank63488 +63489 POINT(29.800082125235686 -96.1841522816872) bank63489 +63490 POINT(30.622974560196184 -95.76734526619717) bank63490 +63491 POINT(29.41608534549864 -94.9539963938611) bank63491 +63492 POINT(30.283574869046777 -95.46161962312003) bank63492 +63493 POINT(28.867115799589268 -95.48297923889997) bank63493 +63494 POINT(30.08939535012001 -95.97421999428171) bank63494 +63495 POINT(29.2626485254262 -95.71298988127813) bank63495 +63496 POINT(29.629406697399478 -96.01283369709347) bank63496 +63497 POINT(30.28708988246471 -94.764381711154) bank63497 +63498 POINT(29.052341630846303 -95.01827610009221) bank63498 +63499 POINT(30.51372242060788 -96.2071999970212) bank63499 +63500 POINT(30.46487549993379 -95.00237542812407) bank63500 +63501 POINT(30.71362161312994 -94.69810908795938) bank63501 +63502 POINT(29.619512855670127 -95.22985057129159) bank63502 +63503 POINT(28.964768068461318 -95.6447550059768) bank63503 +63504 POINT(29.888517933923488 -94.85339425846298) bank63504 +63505 POINT(28.78046760605448 -94.57142237983379) bank63505 +63506 POINT(29.195953616498322 -94.67185133991025) bank63506 +63507 POINT(30.755274787897555 -94.39004419801859) bank63507 +63508 POINT(30.020531771689186 -95.94912596749701) bank63508 +63509 POINT(30.405932045027505 -94.71588407562601) bank63509 +63510 POINT(28.994106773680805 -95.99682884013818) bank63510 +63511 POINT(30.228627069636286 -94.79723920691889) bank63511 +63512 POINT(29.044696204260244 -94.75970037210978) bank63512 +63513 POINT(30.30878591638429 -95.25865308799516) bank63513 +63514 POINT(28.968788637779113 -94.76337470070753) bank63514 +63515 POINT(29.337275455081237 -95.95610230224345) bank63515 +63516 POINT(30.15954489605966 -95.77729152577142) bank63516 +63517 POINT(30.382287159631105 -94.87259764120688) bank63517 +63518 POINT(30.70817405076682 -95.0058036146493) bank63518 +63519 POINT(30.42471375244654 -95.49906789467701) bank63519 +63520 POINT(29.111966989330107 -95.83481603450012) bank63520 +63521 POINT(30.481159647973378 -96.17863369610177) bank63521 +63522 POINT(30.220975799056173 -94.44453299480854) bank63522 +63523 POINT(30.464210852496137 -94.52132017081865) bank63523 +63524 POINT(28.820004335522018 -94.93310581017546) bank63524 +63525 POINT(28.787022209862958 -94.80073456334559) bank63525 +63526 POINT(30.291444262313387 -96.10770721280775) bank63526 +63527 POINT(29.930452887295143 -95.22011918577456) bank63527 +63528 POINT(28.907943492188952 -96.10021647692699) bank63528 +63529 POINT(29.069667802341865 -95.02106478832914) bank63529 +63530 POINT(29.643317882412184 -95.19101819836966) bank63530 +63531 POINT(30.73780772455986 -94.75510042485642) bank63531 +63532 POINT(28.7757874690009 -96.03330947598037) bank63532 +63533 POINT(30.4677016690177 -94.85810428343166) bank63533 +63534 POINT(30.540133024800515 -94.70590339902789) bank63534 +63535 POINT(29.337754611062532 -95.60873300195614) bank63535 +63536 POINT(28.993461110644002 -96.2105421405734) bank63536 +63537 POINT(30.566162772985965 -95.71705304507387) bank63537 +63538 POINT(30.45875450728062 -96.05832279996703) bank63538 +63539 POINT(30.5337111064648 -94.99625395516203) bank63539 +63540 POINT(30.422100315014397 -94.47118917850968) bank63540 +63541 POINT(30.43295458305371 -96.01582379743358) bank63541 +63542 POINT(30.40330433713462 -96.05563097329247) bank63542 +63543 POINT(30.120337448666437 -96.02925484802587) bank63543 +63544 POINT(30.139758158197377 -94.89773748983085) bank63544 +63545 POINT(29.49658823523387 -95.83793018499965) bank63545 +63546 POINT(30.552854599354603 -96.23197172178203) bank63546 +63547 POINT(30.21279681618456 -95.97734600749015) bank63547 +63548 POINT(29.063730342506997 -95.68799645275226) bank63548 +63549 POINT(30.298081891998898 -95.75859374471399) bank63549 +63550 POINT(29.927425457762894 -96.35733532794045) bank63550 +63551 POINT(28.78626051987872 -95.69839403716047) bank63551 +63552 POINT(29.457880704303825 -94.85856169527955) bank63552 +63553 POINT(30.053716637550167 -95.82358867192353) bank63553 +63554 POINT(30.118163637047324 -96.35639288291148) bank63554 +63555 POINT(30.226152185705384 -95.48210675727802) bank63555 +63556 POINT(29.738663361409685 -95.4193925670983) bank63556 +63557 POINT(30.44374381625016 -94.87625247802671) bank63557 +63558 POINT(30.75472047963033 -96.16380910673762) bank63558 +63559 POINT(29.373761932699885 -95.56123359568704) bank63559 +63560 POINT(30.57488993675333 -96.33912808358797) bank63560 +63561 POINT(29.11320274967176 -95.65847679887574) bank63561 +63562 POINT(30.48288709152951 -95.84011212492118) bank63562 +63563 POINT(30.38113499297544 -94.930831727038) bank63563 +63564 POINT(28.82489948222555 -94.84890259415131) bank63564 +63565 POINT(29.366495663338654 -94.80176831463041) bank63565 +63566 POINT(30.131271146834234 -95.32730166134297) bank63566 +63567 POINT(29.021742491824874 -96.11574576377126) bank63567 +63568 POINT(29.36859758902495 -95.72740245984224) bank63568 +63569 POINT(30.110451806360338 -95.50984265288541) bank63569 +63570 POINT(29.320051031606763 -94.87189277212856) bank63570 +63571 POINT(29.221700471516716 -95.40761440452957) bank63571 +63572 POINT(30.029205597530716 -94.92659494936545) bank63572 +63573 POINT(29.486265812513636 -95.65120750458453) bank63573 +63574 POINT(29.00960332203756 -95.03884161373699) bank63574 +63575 POINT(29.511607896951922 -95.4182224766163) bank63575 +63576 POINT(30.569543911959336 -95.24765126905007) bank63576 +63577 POINT(29.30616289547288 -95.65943248449916) bank63577 +63578 POINT(29.439677923603636 -95.4622571420623) bank63578 +63579 POINT(30.647400716565077 -95.88961541522971) bank63579 +63580 POINT(30.052292341262604 -95.70741615481882) bank63580 +63581 POINT(30.092667656353886 -96.08064470862178) bank63581 +63582 POINT(29.595766077122533 -94.70613367738063) bank63582 +63583 POINT(29.60286751479171 -96.11877701807423) bank63583 +63584 POINT(28.798664485962114 -95.96807864502232) bank63584 +63585 POINT(30.491454329294164 -95.38844414029298) bank63585 +63586 POINT(30.445917992253715 -94.80903730603569) bank63586 +63587 POINT(30.260869007186187 -94.7886695705766) bank63587 +63588 POINT(29.568871056501443 -95.19034001563537) bank63588 +63589 POINT(29.31904692671565 -95.22917309529666) bank63589 +63590 POINT(28.865537969784754 -96.2278115442722) bank63590 +63591 POINT(29.48337921340164 -95.11235314908436) bank63591 +63592 POINT(30.49412361814147 -95.33166393126604) bank63592 +63593 POINT(30.62055544329573 -94.6434909946078) bank63593 +63594 POINT(30.361953432208473 -95.78761284317186) bank63594 +63595 POINT(29.763795380732745 -94.8491308165292) bank63595 +63596 POINT(30.1102183161031 -96.23517799745375) bank63596 +63597 POINT(29.99667002895806 -95.40041254853617) bank63597 +63598 POINT(29.451334647958436 -96.13107895948905) bank63598 +63599 POINT(30.614042987336397 -95.79676008136624) bank63599 +63600 POINT(30.066392543683882 -95.54445167807728) bank63600 +63601 POINT(29.649997471244795 -95.96999526252631) bank63601 +63602 POINT(29.52616257945475 -94.6416745379626) bank63602 +63603 POINT(29.248819649421925 -95.51825020252375) bank63603 +63604 POINT(29.061256157354986 -95.54482744963188) bank63604 +63605 POINT(28.847501487594123 -95.55732819681477) bank63605 +63606 POINT(30.00436843599011 -94.71609969161355) bank63606 +63607 POINT(29.9778386818375 -95.7911230858676) bank63607 +63608 POINT(29.533501727170112 -94.92581042377033) bank63608 +63609 POINT(28.9012660707378 -95.38388584476606) bank63609 +63610 POINT(29.362063770069998 -94.81148752898493) bank63610 +63611 POINT(28.978510350723464 -94.66463184016493) bank63611 +63612 POINT(29.59347678757151 -95.71368982152228) bank63612 +63613 POINT(28.91927434202193 -94.68580630150502) bank63613 +63614 POINT(29.39424634203832 -94.37816305487999) bank63614 +63615 POINT(29.98859890390098 -96.29222727981457) bank63615 +63616 POINT(30.54941256642467 -94.9570918548842) bank63616 +63617 POINT(30.58768294445423 -95.14101075285812) bank63617 +63618 POINT(29.88402066054631 -94.91879465077236) bank63618 +63619 POINT(30.137695654792598 -96.01901339431696) bank63619 +63620 POINT(28.76927953139868 -95.06340652467496) bank63620 +63621 POINT(29.359468422560383 -94.46899125487741) bank63621 +63622 POINT(30.039953173095117 -96.12609913865766) bank63622 +63623 POINT(30.238747340547903 -96.29486276233125) bank63623 +63624 POINT(29.75712970184907 -94.85475702536253) bank63624 +63625 POINT(29.383835873584093 -94.94628514651708) bank63625 +63626 POINT(30.20826112780052 -94.87016090698114) bank63626 +63627 POINT(29.485811381325085 -94.55106712010151) bank63627 +63628 POINT(29.71282635470628 -96.20449570402302) bank63628 +63629 POINT(29.327140968489804 -94.7787639306816) bank63629 +63630 POINT(29.096005171340035 -96.2537889883031) bank63630 +63631 POINT(30.756900334979928 -96.04331378586187) bank63631 +63632 POINT(29.01372670182682 -94.64610610588088) bank63632 +63633 POINT(29.21583670411014 -95.15715930188694) bank63633 +63634 POINT(30.086145486277392 -95.0352605724973) bank63634 +63635 POINT(29.41873381618817 -96.0299493348309) bank63635 +63636 POINT(29.70410210725651 -95.7196137287464) bank63636 +63637 POINT(29.981653877415965 -94.55455138446166) bank63637 +63638 POINT(29.906992590851928 -94.72655780898278) bank63638 +63639 POINT(30.076623172143393 -95.69255342024793) bank63639 +63640 POINT(30.46368670742571 -95.81624814400574) bank63640 +63641 POINT(30.702664866817358 -95.09267135396163) bank63641 +63642 POINT(28.891754412405874 -95.99218062975761) bank63642 +63643 POINT(29.341360647708978 -96.2012276122435) bank63643 +63644 POINT(30.648058755983815 -94.50351655232276) bank63644 +63645 POINT(28.932811853449792 -95.36784721843635) bank63645 +63646 POINT(30.56408300940381 -95.32797935288032) bank63646 +63647 POINT(29.250282554652454 -95.50876938131138) bank63647 +63648 POINT(30.25271923685226 -95.22317065140976) bank63648 +63649 POINT(30.52805437693682 -95.59973620871821) bank63649 +63650 POINT(30.322360470498012 -96.19676381900584) bank63650 +63651 POINT(28.868550474548126 -95.81288684620161) bank63651 +63652 POINT(29.383855879096885 -95.8962485564384) bank63652 +63653 POINT(29.58436141438171 -96.3227023751879) bank63653 +63654 POINT(29.424946882593588 -95.73644501326527) bank63654 +63655 POINT(30.53362795853628 -96.27444245284946) bank63655 +63656 POINT(29.073958035704337 -96.36973497510135) bank63656 +63657 POINT(29.85262221629788 -94.37381920055302) bank63657 +63658 POINT(29.608474806695423 -95.95939357793274) bank63658 +63659 POINT(29.865767002300565 -95.18956690550819) bank63659 +63660 POINT(29.81426495226339 -95.77393594018695) bank63660 +63661 POINT(30.096119087121025 -94.96229157238095) bank63661 +63662 POINT(29.00536127375451 -94.37271349878421) bank63662 +63663 POINT(29.639543189492233 -95.89619646196218) bank63663 +63664 POINT(29.37705116491865 -95.81531655525718) bank63664 +63665 POINT(30.434641599598063 -95.5094520368224) bank63665 +63666 POINT(29.18880018048269 -95.95161538157862) bank63666 +63667 POINT(30.495630673307065 -96.36762694905703) bank63667 +63668 POINT(29.57344620062035 -94.71674146744918) bank63668 +63669 POINT(30.30009063859826 -95.17478914385454) bank63669 +63670 POINT(30.154359126961374 -95.38440863540556) bank63670 +63671 POINT(29.776064654487406 -96.32283568025206) bank63671 +63672 POINT(30.559817214273988 -95.63418121289101) bank63672 +63673 POINT(29.552345983163793 -95.28342302869245) bank63673 +63674 POINT(30.281163422761335 -94.89096357157159) bank63674 +63675 POINT(29.93131595194622 -94.41638067645594) bank63675 +63676 POINT(30.63047154599317 -94.75584883074357) bank63676 +63677 POINT(30.424322942046466 -96.1286624049481) bank63677 +63678 POINT(29.068650864173645 -95.84689185853273) bank63678 +63679 POINT(29.79615538786717 -94.70980448374192) bank63679 +63680 POINT(29.857889036179724 -96.2089761341726) bank63680 +63681 POINT(29.34890619313162 -95.68877831356853) bank63681 +63682 POINT(30.718587956936133 -95.44100929357744) bank63682 +63683 POINT(29.605957031749877 -94.68994730253516) bank63683 +63684 POINT(30.398165350729727 -96.03588099862871) bank63684 +63685 POINT(29.709346529942888 -95.26761942787373) bank63685 +63686 POINT(30.261480808229216 -96.09953639974799) bank63686 +63687 POINT(29.67468789065896 -95.79454643885246) bank63687 +63688 POINT(29.708589765384566 -94.77795985516121) bank63688 +63689 POINT(28.958170323252283 -95.61632122617826) bank63689 +63690 POINT(29.502217336264852 -95.90775199037635) bank63690 +63691 POINT(29.570794002540207 -94.56796039220058) bank63691 +63692 POINT(29.823180077232514 -95.0888808588057) bank63692 +63693 POINT(29.234607506984432 -95.3185211619557) bank63693 +63694 POINT(29.042116640492672 -94.78744616304913) bank63694 +63695 POINT(29.90426132953656 -96.21587692800733) bank63695 +63696 POINT(30.416702749504072 -95.93706796568138) bank63696 +63697 POINT(29.15233928048609 -95.14364634174055) bank63697 +63698 POINT(30.19412829479489 -96.18711337086867) bank63698 +63699 POINT(29.435582727770303 -95.82929146391292) bank63699 +63700 POINT(29.736901999007628 -96.27504386442689) bank63700 +63701 POINT(30.338417691799137 -94.97243936582365) bank63701 +63702 POINT(30.2776736827325 -95.40677053093977) bank63702 +63703 POINT(29.721636067027205 -96.22272993641188) bank63703 +63704 POINT(30.316425768490298 -94.85276239396553) bank63704 +63705 POINT(29.611578397256267 -95.29384882623513) bank63705 +63706 POINT(29.61829132378795 -95.41051754336188) bank63706 +63707 POINT(29.703673654319324 -95.50783804329691) bank63707 +63708 POINT(30.139622386740733 -95.72159120024106) bank63708 +63709 POINT(29.880402047989136 -95.62302734767057) bank63709 +63710 POINT(28.76358941811877 -96.12529493890825) bank63710 +63711 POINT(29.64358678622045 -96.13616484226405) bank63711 +63712 POINT(30.212311699669055 -96.17380799045871) bank63712 +63713 POINT(30.61746041739632 -96.28170913150299) bank63713 +63714 POINT(29.91653253879916 -95.36231182474005) bank63714 +63715 POINT(30.166938270436642 -95.7672921375352) bank63715 +63716 POINT(29.48502928616248 -96.26381877353394) bank63716 +63717 POINT(30.71608587057989 -95.9255023990761) bank63717 +63718 POINT(29.953083589482837 -94.89975216490664) bank63718 +63719 POINT(29.723078619332657 -95.97619201496292) bank63719 +63720 POINT(29.54774882445849 -95.20521590116746) bank63720 +63721 POINT(30.66694558862721 -95.4558145721614) bank63721 +63722 POINT(30.345241062792393 -94.83931764030557) bank63722 +63723 POINT(28.95420203095895 -94.52655435940525) bank63723 +63724 POINT(30.006741986171342 -95.52756632815053) bank63724 +63725 POINT(30.2812469113957 -95.70497973062247) bank63725 +63726 POINT(30.114623582876057 -95.05159942616523) bank63726 +63727 POINT(28.86319888673304 -95.4217809715676) bank63727 +63728 POINT(29.57737567748617 -95.99706333683487) bank63728 +63729 POINT(29.719335640137846 -95.24371965449097) bank63729 +63730 POINT(30.160517970589005 -95.91381048092873) bank63730 +63731 POINT(29.02329632290924 -95.67580532450775) bank63731 +63732 POINT(29.41371512008518 -96.2230478684947) bank63732 +63733 POINT(29.75302754851276 -94.81969969745403) bank63733 +63734 POINT(30.731406680169382 -95.18386220818797) bank63734 +63735 POINT(30.244175806989897 -96.01690333745263) bank63735 +63736 POINT(29.26315349392464 -96.15678052576689) bank63736 +63737 POINT(30.141491033711485 -95.0790970348323) bank63737 +63738 POINT(28.883487837195407 -94.877785749434) bank63738 +63739 POINT(29.625430181347728 -94.4885753930796) bank63739 +63740 POINT(29.33520723214682 -96.35656682537763) bank63740 +63741 POINT(30.399939706566798 -94.80516827463106) bank63741 +63742 POINT(29.899575921607003 -95.03050902287056) bank63742 +63743 POINT(29.621137495465252 -94.70773641885377) bank63743 +63744 POINT(30.51509065870005 -96.18538658313275) bank63744 +63745 POINT(29.463006072990233 -94.90096693789275) bank63745 +63746 POINT(29.70220663105926 -96.09952222114308) bank63746 +63747 POINT(29.26365539108477 -94.78635648934454) bank63747 +63748 POINT(29.489954015984097 -94.3762875130493) bank63748 +63749 POINT(29.633835427264664 -96.28262651462255) bank63749 +63750 POINT(28.955586746117913 -95.94471728644092) bank63750 +63751 POINT(29.69680935690667 -95.7414633024287) bank63751 +63752 POINT(29.97960115715539 -95.07783436312396) bank63752 +63753 POINT(29.57393361383693 -96.3611228447042) bank63753 +63754 POINT(30.155808687842654 -96.22153002940775) bank63754 +63755 POINT(29.37781180923838 -96.29029823003123) bank63755 +63756 POINT(30.59801617970767 -95.18882414887666) bank63756 +63757 POINT(30.170507358920638 -94.93390964683886) bank63757 +63758 POINT(30.749586542940833 -95.74386227067728) bank63758 +63759 POINT(30.162288807746517 -94.69458323634578) bank63759 +63760 POINT(28.866382355170426 -96.17435054087585) bank63760 +63761 POINT(30.192130956362128 -95.17516926969002) bank63761 +63762 POINT(29.89033896550041 -94.50342057341317) bank63762 +63763 POINT(29.81135834365383 -95.30151297428442) bank63763 +63764 POINT(29.05088759926703 -96.19534137271235) bank63764 +63765 POINT(29.510091187235957 -94.81318185512386) bank63765 +63766 POINT(29.659875496303734 -95.64112818890254) bank63766 +63767 POINT(29.060585751979104 -95.712706152301) bank63767 +63768 POINT(29.73600740608128 -95.9650646292184) bank63768 +63769 POINT(30.17220367180851 -95.68658591093396) bank63769 +63770 POINT(29.77155145246275 -95.53096098473888) bank63770 +63771 POINT(28.907964506575667 -94.45886172531931) bank63771 +63772 POINT(30.051883201867238 -94.49604404831324) bank63772 +63773 POINT(28.81187460871959 -95.74334501157017) bank63773 +63774 POINT(28.775457467094864 -95.30309938106336) bank63774 +63775 POINT(29.7789302167784 -95.63292170102035) bank63775 +63776 POINT(29.27233740708186 -95.11388947225142) bank63776 +63777 POINT(29.737934633694987 -95.25113488217434) bank63777 +63778 POINT(30.173763622037857 -95.00498546704462) bank63778 +63779 POINT(29.34666049136285 -94.96298238993539) bank63779 +63780 POINT(29.544988321302704 -96.14644761272069) bank63780 +63781 POINT(30.29674530649998 -94.60378513171233) bank63781 +63782 POINT(29.52914963430386 -94.66197232277142) bank63782 +63783 POINT(29.480730434649566 -96.10304038042996) bank63783 +63784 POINT(29.010656807109907 -95.63595126298351) bank63784 +63785 POINT(30.18938179425884 -94.51127742712544) bank63785 +63786 POINT(28.980986076932133 -95.18769276334501) bank63786 +63787 POINT(29.4137489374028 -95.62691339577607) bank63787 +63788 POINT(30.755888269825338 -94.99756203494209) bank63788 +63789 POINT(29.90328266459328 -95.83424531453012) bank63789 +63790 POINT(30.6915172006789 -94.4183082980336) bank63790 +63791 POINT(29.754298658994824 -94.44964162876762) bank63791 +63792 POINT(30.507632754669903 -96.19504085848209) bank63792 +63793 POINT(28.77721617751713 -95.8680305174035) bank63793 +63794 POINT(29.525038755846623 -95.41083884131734) bank63794 +63795 POINT(29.924253628605243 -95.84431992171663) bank63795 +63796 POINT(30.23967353058806 -94.59856799337206) bank63796 +63797 POINT(29.19855086450045 -94.59206694359743) bank63797 +63798 POINT(29.940880379072585 -96.34411240522371) bank63798 +63799 POINT(29.45763691556711 -95.1797146583926) bank63799 +63800 POINT(29.15434524553608 -95.82966246234075) bank63800 +63801 POINT(28.92108901270811 -95.08762478403237) bank63801 +63802 POINT(29.06373260137772 -95.94470349784906) bank63802 +63803 POINT(30.67787650911197 -95.95445348384845) bank63803 +63804 POINT(29.50225579118962 -95.10921891702891) bank63804 +63805 POINT(29.91437575983831 -96.05476819922687) bank63805 +63806 POINT(30.42455941508016 -94.54206255623627) bank63806 +63807 POINT(29.942592450847467 -95.63914936252738) bank63807 +63808 POINT(30.281992824422005 -94.4119630125867) bank63808 +63809 POINT(28.86102608819125 -95.16054891928643) bank63809 +63810 POINT(29.316803229059822 -94.46952059308838) bank63810 +63811 POINT(29.84604777089278 -94.60696826357176) bank63811 +63812 POINT(30.540438812455704 -96.08109950580737) bank63812 +63813 POINT(30.575645185548584 -94.40349274044044) bank63813 +63814 POINT(30.69451141507359 -95.23664145104497) bank63814 +63815 POINT(30.19653337590843 -94.41360582105402) bank63815 +63816 POINT(29.262275155607735 -94.57828619186628) bank63816 +63817 POINT(28.98085614826891 -94.60264541743214) bank63817 +63818 POINT(29.201743216707257 -94.96984044396018) bank63818 +63819 POINT(30.370757688704295 -96.2989973410626) bank63819 +63820 POINT(30.132297545913367 -95.95543140510203) bank63820 +63821 POINT(30.130206278976175 -94.52125516895819) bank63821 +63822 POINT(29.38674817390325 -96.25233932658922) bank63822 +63823 POINT(29.6804224597208 -95.25939208051233) bank63823 +63824 POINT(28.93201170342257 -94.4809954393243) bank63824 +63825 POINT(30.706083493037394 -95.23366116363673) bank63825 +63826 POINT(28.901156793826306 -95.86557136867968) bank63826 +63827 POINT(30.59228276271386 -96.00103267242737) bank63827 +63828 POINT(29.8797268403429 -96.10708208232755) bank63828 +63829 POINT(30.11625246897268 -94.95428385617622) bank63829 +63830 POINT(29.464718286640267 -94.8391833350003) bank63830 +63831 POINT(30.669155522657135 -96.11846749660837) bank63831 +63832 POINT(30.722615160481507 -94.61444975146577) bank63832 +63833 POINT(30.150173371458184 -95.83693118426407) bank63833 +63834 POINT(29.94979330750908 -94.81288493927245) bank63834 +63835 POINT(29.178727473453588 -95.05342397142873) bank63835 +63836 POINT(29.256510192656254 -95.21664803007779) bank63836 +63837 POINT(30.136106366827 -96.09740689804327) bank63837 +63838 POINT(30.630078775106714 -95.87884916911207) bank63838 +63839 POINT(30.43029360929286 -95.91587514013713) bank63839 +63840 POINT(30.361237271967 -94.50067464803881) bank63840 +63841 POINT(29.268790162829134 -95.48045875075552) bank63841 +63842 POINT(29.12061632891433 -96.06707882356756) bank63842 +63843 POINT(28.901401671063716 -95.89395492520869) bank63843 +63844 POINT(29.843887059800174 -96.35777013020936) bank63844 +63845 POINT(29.150128403328132 -95.68279485891617) bank63845 +63846 POINT(28.981079050900185 -96.14601868891249) bank63846 +63847 POINT(30.10235726347828 -96.08185971501305) bank63847 +63848 POINT(29.0694519536597 -96.06387127700458) bank63848 +63849 POINT(30.296292366925027 -95.03732654076981) bank63849 +63850 POINT(29.917752290753047 -96.02387502153395) bank63850 +63851 POINT(29.744643497788438 -95.36810534946164) bank63851 +63852 POINT(29.034731375370786 -95.05582070485563) bank63852 +63853 POINT(29.043422245839537 -94.54281169955732) bank63853 +63854 POINT(30.38114562575108 -95.12373732427349) bank63854 +63855 POINT(30.096990991603626 -95.49381487900585) bank63855 +63856 POINT(30.56426495733853 -95.24025167196524) bank63856 +63857 POINT(30.532395321449687 -94.93815323805612) bank63857 +63858 POINT(30.153577785024172 -95.01541319332837) bank63858 +63859 POINT(29.28400980104976 -94.44201631021879) bank63859 +63860 POINT(29.125890762863925 -95.84912965720936) bank63860 +63861 POINT(28.99524570325584 -94.98249543287798) bank63861 +63862 POINT(29.454004190451215 -95.72826908344562) bank63862 +63863 POINT(29.929262539672383 -95.6484669829864) bank63863 +63864 POINT(29.876190765632966 -95.87374417616185) bank63864 +63865 POINT(29.5221291064049 -95.80354091364669) bank63865 +63866 POINT(28.988200343151938 -94.87885275080743) bank63866 +63867 POINT(30.190141424552372 -95.29969520290223) bank63867 +63868 POINT(30.039025946009545 -95.54454596243218) bank63868 +63869 POINT(29.472205799259548 -95.64793389865672) bank63869 +63870 POINT(29.66974689903446 -96.1845391098251) bank63870 +63871 POINT(28.87107608791365 -95.82879912308721) bank63871 +63872 POINT(30.69173150268871 -95.12316548810796) bank63872 +63873 POINT(29.081370675881452 -95.79670092439869) bank63873 +63874 POINT(30.103620949875932 -96.3385091274734) bank63874 +63875 POINT(30.567115611564272 -94.81871700446835) bank63875 +63876 POINT(29.333645774043077 -95.49714121862164) bank63876 +63877 POINT(29.95455965984614 -95.33511226948508) bank63877 +63878 POINT(30.474358136233406 -95.99100611022824) bank63878 +63879 POINT(30.503446868903364 -95.33039616577207) bank63879 +63880 POINT(30.41251450141166 -95.53929690105817) bank63880 +63881 POINT(29.495367353874098 -95.82325120563843) bank63881 +63882 POINT(29.26594741694537 -96.21024974725952) bank63882 +63883 POINT(29.832287797283545 -95.83481550068518) bank63883 +63884 POINT(29.15371626992725 -95.98753284351528) bank63884 +63885 POINT(30.613277171231488 -95.66486439729968) bank63885 +63886 POINT(30.662381166536637 -95.57051081818565) bank63886 +63887 POINT(30.274004558606855 -94.39805612025891) bank63887 +63888 POINT(29.369992880604347 -96.07808563412554) bank63888 +63889 POINT(28.885843054610397 -96.09922991694533) bank63889 +63890 POINT(29.06765213328234 -94.62234351777136) bank63890 +63891 POINT(28.8848886025547 -96.30100517278915) bank63891 +63892 POINT(29.238876226074925 -96.06387672689611) bank63892 +63893 POINT(28.861586430586833 -95.42834816710406) bank63893 +63894 POINT(29.38037410916622 -95.0739982173566) bank63894 +63895 POINT(29.12787983143942 -95.67965781536896) bank63895 +63896 POINT(30.39874867568755 -96.26943921402854) bank63896 +63897 POINT(28.95593913156721 -95.09327684004936) bank63897 +63898 POINT(30.68987195269277 -96.35088502766446) bank63898 +63899 POINT(29.143120321432207 -96.28569203698356) bank63899 +63900 POINT(30.63332978770214 -94.615866749226) bank63900 +63901 POINT(30.10212431523825 -95.02099240583976) bank63901 +63902 POINT(29.481148730265993 -95.37996351036958) bank63902 +63903 POINT(30.475349761788394 -96.07341858860818) bank63903 +63904 POINT(30.33676408373103 -95.84070274327154) bank63904 +63905 POINT(30.270526158184172 -96.24551662514205) bank63905 +63906 POINT(29.21886139453358 -94.84925497982705) bank63906 +63907 POINT(29.6800152315237 -94.62868848463604) bank63907 +63908 POINT(29.91187446132827 -96.36889031290357) bank63908 +63909 POINT(29.174195495168213 -95.47500744367673) bank63909 +63910 POINT(28.979504408598157 -94.37921499643093) bank63910 +63911 POINT(29.67577425018619 -95.59140632100294) bank63911 +63912 POINT(30.426225945782356 -95.12344141687687) bank63912 +63913 POINT(30.23038777888106 -95.51096153370814) bank63913 +63914 POINT(30.62952143861686 -95.71352489375971) bank63914 +63915 POINT(30.63439929366671 -95.67657554666378) bank63915 +63916 POINT(29.975726807528225 -94.83267289350078) bank63916 +63917 POINT(30.47750801084222 -95.11843915025317) bank63917 +63918 POINT(29.09050445173656 -95.66271492739071) bank63918 +63919 POINT(30.581242396987463 -95.83569970512316) bank63919 +63920 POINT(29.262196527227083 -94.73039741654499) bank63920 +63921 POINT(29.081375172209615 -94.64738484458121) bank63921 +63922 POINT(29.69722661874416 -95.62367218378753) bank63922 +63923 POINT(29.9105722384619 -95.9662357847779) bank63923 +63924 POINT(30.301371942462012 -96.29299414012985) bank63924 +63925 POINT(30.605032323311544 -95.06808888081417) bank63925 +63926 POINT(29.573519587621004 -94.92189065938149) bank63926 +63927 POINT(29.77482184722969 -96.2374959828843) bank63927 +63928 POINT(29.645754718232418 -96.20152391732611) bank63928 +63929 POINT(29.716593960038683 -94.57697668974669) bank63929 +63930 POINT(30.48007197354867 -95.54749100189046) bank63930 +63931 POINT(29.99669493005014 -96.33062995800773) bank63931 +63932 POINT(29.69962041165177 -95.92833255720318) bank63932 +63933 POINT(28.92706151439346 -96.29144056129752) bank63933 +63934 POINT(29.64338629804964 -95.30235525519606) bank63934 +63935 POINT(30.313186889287987 -95.17642646496255) bank63935 +63936 POINT(29.890037297740246 -94.43048134920605) bank63936 +63937 POINT(30.333495982996926 -95.75758156965577) bank63937 +63938 POINT(29.78474767673146 -96.14786026963523) bank63938 +63939 POINT(29.95093395260928 -94.62257081078579) bank63939 +63940 POINT(29.832339279479893 -95.85805322323156) bank63940 +63941 POINT(29.567324225710237 -95.76669973321893) bank63941 +63942 POINT(29.94143837406136 -96.19552191380413) bank63942 +63943 POINT(29.872147671248374 -95.4031163471615) bank63943 +63944 POINT(29.122293300580466 -95.5192512816958) bank63944 +63945 POINT(30.659109527118403 -95.97186129654806) bank63945 +63946 POINT(28.782417768665574 -95.14851642402934) bank63946 +63947 POINT(29.43745094673653 -95.52757626727457) bank63947 +63948 POINT(30.75227966203958 -95.71992757246433) bank63948 +63949 POINT(29.151947486816315 -94.84540062951864) bank63949 +63950 POINT(29.712815227627978 -95.28787783075995) bank63950 +63951 POINT(29.784447681692235 -95.6031210716636) bank63951 +63952 POINT(29.466106915230007 -95.20856438648025) bank63952 +63953 POINT(29.21172736040693 -94.86020237640824) bank63953 +63954 POINT(29.849491945151563 -96.28666735968692) bank63954 +63955 POINT(29.724573306708088 -95.85836570926423) bank63955 +63956 POINT(29.03975150315509 -94.5116203074483) bank63956 +63957 POINT(29.77261208503145 -94.79397348311393) bank63957 +63958 POINT(30.598283175477057 -96.17507396682096) bank63958 +63959 POINT(29.41464417467039 -94.62243675628093) bank63959 +63960 POINT(29.779673608723463 -95.23848848980613) bank63960 +63961 POINT(30.083292681952464 -95.72024507512157) bank63961 +63962 POINT(29.6913578558307 -95.58195772694569) bank63962 +63963 POINT(29.803120554336758 -95.99159895628937) bank63963 +63964 POINT(30.17967874467976 -96.27311872808825) bank63964 +63965 POINT(29.97551934481921 -94.67907057104445) bank63965 +63966 POINT(30.479951668847093 -94.53990037075437) bank63966 +63967 POINT(29.489572365683642 -95.65592238642935) bank63967 +63968 POINT(30.35199074954061 -94.51944882050796) bank63968 +63969 POINT(30.176909083819705 -96.14104010990513) bank63969 +63970 POINT(30.35403157945881 -95.00643236579626) bank63970 +63971 POINT(30.143433840406285 -94.47938269723282) bank63971 +63972 POINT(28.956563437112255 -95.4757250098073) bank63972 +63973 POINT(30.147663827504473 -94.45611901680965) bank63973 +63974 POINT(29.948343693338924 -94.37707928683093) bank63974 +63975 POINT(30.27724508977302 -95.68229855760981) bank63975 +63976 POINT(28.76400590977162 -96.32997435007069) bank63976 +63977 POINT(30.627823667672594 -95.87656945820596) bank63977 +63978 POINT(29.752605162506327 -94.54091397702533) bank63978 +63979 POINT(30.117806986308267 -95.79282730537781) bank63979 +63980 POINT(29.176710640533244 -94.99700800237989) bank63980 +63981 POINT(30.66321465104403 -94.64703499660995) bank63981 +63982 POINT(29.687543943405608 -95.16224193303663) bank63982 +63983 POINT(30.267885133662073 -96.14895444812468) bank63983 +63984 POINT(29.096469793320505 -94.6906537644506) bank63984 +63985 POINT(29.572804391961625 -95.32415308699808) bank63985 +63986 POINT(30.43693908683383 -94.63577270583127) bank63986 +63987 POINT(30.069872520536904 -96.17177433226036) bank63987 +63988 POINT(30.37347845179402 -95.02259673364043) bank63988 +63989 POINT(29.020339261417348 -95.86077517007247) bank63989 +63990 POINT(29.601074909543716 -94.55573455186638) bank63990 +63991 POINT(28.980422921340327 -95.39399472294171) bank63991 +63992 POINT(29.2048165883922 -95.26839342419075) bank63992 +63993 POINT(30.75448889766516 -96.00085856805046) bank63993 +63994 POINT(30.41314756661501 -95.55217670239409) bank63994 +63995 POINT(29.626622252513826 -95.84000438315874) bank63995 +63996 POINT(30.106179392791574 -94.59346108725184) bank63996 +63997 POINT(29.23816032769236 -95.92210952739832) bank63997 +63998 POINT(28.818964506933778 -94.55572704735398) bank63998 +63999 POINT(28.775399291150237 -95.85076018435336) bank63999 +64000 POINT(30.358800973008467 -94.40746123514104) bank64000 +64001 POINT(30.717262880127866 -94.39771855995794) bank64001 +64002 POINT(30.72285676316655 -95.25356329335678) bank64002 +64003 POINT(29.24020453742975 -94.45797619175626) bank64003 +64004 POINT(30.331673230545743 -95.60782602941478) bank64004 +64005 POINT(29.026630524078197 -95.08272844027759) bank64005 +64006 POINT(30.70070412549291 -95.76337211203102) bank64006 +64007 POINT(29.68990680477404 -95.86001221091071) bank64007 +64008 POINT(29.462182636280797 -96.10412029418589) bank64008 +64009 POINT(29.006475554900288 -94.82051578086829) bank64009 +64010 POINT(30.638526447575416 -96.04914420981548) bank64010 +64011 POINT(30.46143565038023 -96.14747886005293) bank64011 +64012 POINT(29.300567887080113 -95.62660111243909) bank64012 +64013 POINT(30.08649581632009 -95.99273905284606) bank64013 +64014 POINT(29.983500246039462 -94.85937247208643) bank64014 +64015 POINT(28.85347912017989 -94.86497425257267) bank64015 +64016 POINT(30.260409985104808 -95.43980567703092) bank64016 +64017 POINT(29.617696127990676 -94.82934818100696) bank64017 +64018 POINT(28.818086637269158 -96.32314791428465) bank64018 +64019 POINT(30.225949147943776 -95.11975672244449) bank64019 +64020 POINT(29.184393804822967 -94.87616186866671) bank64020 +64021 POINT(30.60973996992071 -94.5581240415465) bank64021 +64022 POINT(29.182950912969414 -95.53388373888002) bank64022 +64023 POINT(30.59412781869247 -96.26279396593584) bank64023 +64024 POINT(30.51487822456191 -95.00525148983199) bank64024 +64025 POINT(30.073417471912574 -95.96003300850634) bank64025 +64026 POINT(29.83880653524767 -96.07481250410575) bank64026 +64027 POINT(30.06212632151456 -95.49261550388817) bank64027 +64028 POINT(30.183666407244118 -96.09840907543997) bank64028 +64029 POINT(28.958067530755763 -94.52459401593721) bank64029 +64030 POINT(28.79331322494744 -94.51735477328018) bank64030 +64031 POINT(28.914643705446927 -94.80800682161751) bank64031 +64032 POINT(30.483528046251575 -94.46488685100596) bank64032 +64033 POINT(29.070084634919098 -95.83276517942046) bank64033 +64034 POINT(29.90966416751599 -96.01337716306) bank64034 +64035 POINT(29.813855336545835 -95.38572450216033) bank64035 +64036 POINT(29.38536544040136 -96.0473301482317) bank64036 +64037 POINT(30.4366050826212 -95.17577395974428) bank64037 +64038 POINT(28.982503323863643 -95.96417103772221) bank64038 +64039 POINT(30.48757599874599 -95.42510872068009) bank64039 +64040 POINT(29.89118163870931 -96.00990799390814) bank64040 +64041 POINT(30.459509877637032 -96.1128037582838) bank64041 +64042 POINT(30.559093367900015 -95.26253017361675) bank64042 +64043 POINT(29.864038867707357 -95.34996909445248) bank64043 +64044 POINT(29.833890429669975 -95.80120090767774) bank64044 +64045 POINT(29.58125805594402 -95.52699874102878) bank64045 +64046 POINT(30.20778798563617 -96.03520889889887) bank64046 +64047 POINT(29.202449953370177 -95.93705571759688) bank64047 +64048 POINT(30.31094803936529 -94.96749341824078) bank64048 +64049 POINT(28.782027625342607 -94.73897117029608) bank64049 +64050 POINT(29.069495718035633 -95.72641088962928) bank64050 +64051 POINT(29.70006277548757 -95.78632905544701) bank64051 +64052 POINT(28.96263777945393 -94.44810235124824) bank64052 +64053 POINT(29.930169949970153 -95.10114716293276) bank64053 +64054 POINT(29.169528477909477 -94.6603922388419) bank64054 +64055 POINT(29.276343858805337 -94.72838939666985) bank64055 +64056 POINT(28.84405921058032 -95.07106019257868) bank64056 +64057 POINT(29.73446672264802 -95.30032047984824) bank64057 +64058 POINT(30.543471292343465 -94.65910398505368) bank64058 +64059 POINT(29.173596990129152 -95.32593789073125) bank64059 +64060 POINT(30.65373450058365 -96.02001714362534) bank64060 +64061 POINT(29.16761247880749 -95.34728297310629) bank64061 +64062 POINT(29.4175288281745 -94.84695668415166) bank64062 +64063 POINT(29.005590076346763 -95.80433896750007) bank64063 +64064 POINT(30.188972479120054 -96.12777275670832) bank64064 +64065 POINT(29.632059471851196 -95.55953627935928) bank64065 +64066 POINT(30.50705620023967 -95.49846143117227) bank64066 +64067 POINT(30.024870685797175 -95.57052339676697) bank64067 +64068 POINT(29.814032078881883 -94.46048414811344) bank64068 +64069 POINT(29.5263972435124 -95.60146195886232) bank64069 +64070 POINT(28.846051002860108 -95.27515908093238) bank64070 +64071 POINT(29.921619333381866 -95.87892929688775) bank64071 +64072 POINT(30.587693033779367 -94.3857665103013) bank64072 +64073 POINT(28.819604263933893 -95.43264072561874) bank64073 +64074 POINT(29.88362921331751 -94.77435739204834) bank64074 +64075 POINT(29.779292641474097 -95.51708457126085) bank64075 +64076 POINT(28.94193927922869 -95.03924420329983) bank64076 +64077 POINT(30.6738452884151 -94.44911566539561) bank64077 +64078 POINT(29.56206810433624 -95.25096102634747) bank64078 +64079 POINT(29.784667393554678 -96.12851225260702) bank64079 +64080 POINT(30.10827653806893 -95.33088886536014) bank64080 +64081 POINT(29.78174824042722 -95.54181037543319) bank64081 +64082 POINT(29.83484270299791 -94.43848180284158) bank64082 +64083 POINT(29.693062841085275 -94.82619654823344) bank64083 +64084 POINT(30.653278728554962 -94.7452374157675) bank64084 +64085 POINT(29.768841350041434 -94.95746049647131) bank64085 +64086 POINT(30.001812507831765 -95.6518011023286) bank64086 +64087 POINT(29.75275019851617 -96.09929258637781) bank64087 +64088 POINT(30.641818819726534 -95.68666606473946) bank64088 +64089 POINT(28.930903299897572 -94.53337195961112) bank64089 +64090 POINT(29.497530053108758 -94.78752911323174) bank64090 +64091 POINT(28.97876457402163 -95.09004067378484) bank64091 +64092 POINT(29.48843930118892 -94.63152691915825) bank64092 +64093 POINT(28.912708614199808 -94.67430155156069) bank64093 +64094 POINT(30.67292705775389 -96.33134212569618) bank64094 +64095 POINT(30.254427912165017 -94.90178065569323) bank64095 +64096 POINT(28.97216457524673 -94.59113123834027) bank64096 +64097 POINT(29.219198762889707 -95.12054189448982) bank64097 +64098 POINT(30.700779678137017 -96.12842498329151) bank64098 +64099 POINT(30.398055519159076 -95.9485152037566) bank64099 +64100 POINT(30.24712754221425 -94.63051531365691) bank64100 +64101 POINT(30.410020605550248 -96.29120863839917) bank64101 +64102 POINT(29.279560680740158 -95.40759766297421) bank64102 +64103 POINT(28.955157232039607 -94.38842022235313) bank64103 +64104 POINT(30.68728553967283 -94.97932704704789) bank64104 +64105 POINT(29.87195526536999 -94.81308546051825) bank64105 +64106 POINT(29.454143796777192 -96.29689625866058) bank64106 +64107 POINT(30.42787060679696 -95.49862507605346) bank64107 +64108 POINT(28.908329947266864 -95.11151579534221) bank64108 +64109 POINT(29.173767575007215 -95.796118598172) bank64109 +64110 POINT(29.20304285761534 -95.67418916017532) bank64110 +64111 POINT(28.93122553231446 -96.08832150183173) bank64111 +64112 POINT(30.088275306539007 -94.80673950695808) bank64112 +64113 POINT(30.585665920269822 -96.29903746589139) bank64113 +64114 POINT(28.844313768882486 -94.91831161647953) bank64114 +64115 POINT(29.89137148250194 -95.3712496084624) bank64115 +64116 POINT(29.669740346679752 -95.6006960713236) bank64116 +64117 POINT(29.298889273583885 -95.07653687110293) bank64117 +64118 POINT(30.655608813388593 -96.16053288000951) bank64118 +64119 POINT(30.520971440619913 -95.23394090337126) bank64119 +64120 POINT(29.709900045380074 -95.74304288972914) bank64120 +64121 POINT(30.61969793946994 -94.6524934694806) bank64121 +64122 POINT(29.96563663882144 -94.84630522092304) bank64122 +64123 POINT(30.642753906221326 -94.42927206211128) bank64123 +64124 POINT(28.79685838532343 -95.39499778989274) bank64124 +64125 POINT(29.5666261995984 -94.89494723573563) bank64125 +64126 POINT(30.186868104633138 -94.6559068497516) bank64126 +64127 POINT(30.511086904156244 -95.29918149635591) bank64127 +64128 POINT(29.362820764870545 -96.25127764047238) bank64128 +64129 POINT(29.0312455351484 -96.33692452202335) bank64129 +64130 POINT(30.065974453908826 -96.26439427537962) bank64130 +64131 POINT(29.95927682958111 -96.03288379612792) bank64131 +64132 POINT(28.7947153256705 -94.77655408046915) bank64132 +64133 POINT(28.97330112236593 -94.55449695448816) bank64133 +64134 POINT(29.611214841623543 -94.83766770140373) bank64134 +64135 POINT(29.720007395980726 -95.93924567649488) bank64135 +64136 POINT(30.014045342747252 -94.70226246532087) bank64136 +64137 POINT(28.79170817844788 -96.19822004115875) bank64137 +64138 POINT(30.71749080208452 -95.73701647325193) bank64138 +64139 POINT(29.984777324461167 -95.89479371627773) bank64139 +64140 POINT(29.591495492213774 -95.98848307332871) bank64140 +64141 POINT(29.562660904441753 -96.30174930044211) bank64141 +64142 POINT(30.614985603467034 -95.32176959234525) bank64142 +64143 POINT(29.65247245613294 -96.15729130656516) bank64143 +64144 POINT(29.869969787540036 -95.33930708558407) bank64144 +64145 POINT(28.991483505406197 -95.67607828823677) bank64145 +64146 POINT(29.229271593677982 -95.5289075431995) bank64146 +64147 POINT(28.974038809133848 -94.60195009958673) bank64147 +64148 POINT(30.393410852882305 -95.6807070288706) bank64148 +64149 POINT(29.22451296036104 -95.52582908227922) bank64149 +64150 POINT(28.7968808013804 -95.01753026586364) bank64150 +64151 POINT(29.79651103492159 -96.09168695088158) bank64151 +64152 POINT(29.64610232922355 -95.92273937060743) bank64152 +64153 POINT(29.774275987439175 -94.4723005201952) bank64153 +64154 POINT(30.4057323065924 -94.51044233530659) bank64154 +64155 POINT(29.771652521128928 -96.19119566689066) bank64155 +64156 POINT(29.046765578752687 -95.15829345608859) bank64156 +64157 POINT(29.371395271450307 -94.8532154251416) bank64157 +64158 POINT(29.56780350054961 -95.89279080289383) bank64158 +64159 POINT(30.10867665728666 -96.10697532633304) bank64159 +64160 POINT(29.89693403087804 -95.81594119715615) bank64160 +64161 POINT(30.536294935805355 -95.16429459734513) bank64161 +64162 POINT(30.340135805331478 -95.23539536650841) bank64162 +64163 POINT(29.251715554506664 -94.65236775884843) bank64163 +64164 POINT(28.91873614535053 -94.77614277490721) bank64164 +64165 POINT(29.368394570631217 -95.03936649572965) bank64165 +64166 POINT(30.553140040837174 -96.21710621408268) bank64166 +64167 POINT(29.707925165817574 -96.29194769883341) bank64167 +64168 POINT(29.35809268365568 -94.42310673970493) bank64168 +64169 POINT(28.767610095210784 -95.50028674007602) bank64169 +64170 POINT(29.963096048035137 -95.33219514196587) bank64170 +64171 POINT(30.280343277543473 -95.68079545479924) bank64171 +64172 POINT(30.281884560977915 -95.34595656132267) bank64172 +64173 POINT(29.30571549566637 -95.35836221381932) bank64173 +64174 POINT(29.861356576499432 -95.99001582704751) bank64174 +64175 POINT(28.97980834433657 -94.37262424288383) bank64175 +64176 POINT(30.19665469350749 -96.32934182113225) bank64176 +64177 POINT(28.833836197652886 -94.5139571220822) bank64177 +64178 POINT(30.156405935889147 -95.0102109565114) bank64178 +64179 POINT(29.311935375908977 -96.30320227005943) bank64179 +64180 POINT(29.23691567308966 -94.4145563100501) bank64180 +64181 POINT(30.38312929832569 -94.95250407475822) bank64181 +64182 POINT(28.989477537770895 -94.93677145100291) bank64182 +64183 POINT(28.84865087267973 -95.9821978572154) bank64183 +64184 POINT(29.66973745851254 -95.10265153477738) bank64184 +64185 POINT(29.829307667907457 -95.1464086050763) bank64185 +64186 POINT(30.024455990884864 -95.60500597290955) bank64186 +64187 POINT(28.93025876116924 -94.78653608195232) bank64187 +64188 POINT(29.208772813511985 -95.28624786304107) bank64188 +64189 POINT(29.009735518408093 -95.56266175332561) bank64189 +64190 POINT(29.817321548156077 -96.24237448257095) bank64190 +64191 POINT(29.932073096809212 -96.3533796296865) bank64191 +64192 POINT(29.946654771925306 -95.36519524675182) bank64192 +64193 POINT(29.21163336312279 -95.00123899792503) bank64193 +64194 POINT(29.603255986484378 -94.8824358527588) bank64194 +64195 POINT(30.19337473242496 -94.64260892444298) bank64195 +64196 POINT(30.33847146595608 -96.31318997669332) bank64196 +64197 POINT(30.489903028677187 -95.47827802656107) bank64197 +64198 POINT(29.253573349782137 -95.56025965225705) bank64198 +64199 POINT(28.909339389829963 -95.27576576989316) bank64199 +64200 POINT(29.107713491351152 -96.34274020445096) bank64200 +64201 POINT(29.580681493571646 -96.04560614061532) bank64201 +64202 POINT(30.435052071311024 -94.47631753250887) bank64202 +64203 POINT(30.531177511331027 -94.81352294413975) bank64203 +64204 POINT(29.75157853962351 -96.01076276559091) bank64204 +64205 POINT(29.574146380060995 -94.65333531146803) bank64205 +64206 POINT(28.992056593865584 -95.99972105151103) bank64206 +64207 POINT(29.89115253822537 -95.1096382669001) bank64207 +64208 POINT(29.11963058266885 -94.76342895969337) bank64208 +64209 POINT(29.44221524391581 -95.7245380820537) bank64209 +64210 POINT(28.9325849433809 -95.74766798353703) bank64210 +64211 POINT(30.226242073180654 -94.49314895933989) bank64211 +64212 POINT(29.558482371936066 -95.33069702388971) bank64212 +64213 POINT(28.887003215558867 -95.68014955188569) bank64213 +64214 POINT(29.477750317811527 -95.94372212436043) bank64214 +64215 POINT(30.38436417878966 -94.79713333748225) bank64215 +64216 POINT(29.295083035406755 -95.75832038100728) bank64216 +64217 POINT(29.06531054488631 -96.0929024431845) bank64217 +64218 POINT(29.199921685216932 -96.16707268416063) bank64218 +64219 POINT(30.01183457227282 -95.69910928492912) bank64219 +64220 POINT(29.128505233183706 -95.52114084706912) bank64220 +64221 POINT(29.991514139795907 -94.57288552990974) bank64221 +64222 POINT(29.12401785744353 -94.99098897127598) bank64222 +64223 POINT(30.202755654240715 -94.42711626227035) bank64223 +64224 POINT(30.501026836900593 -95.50980181924594) bank64224 +64225 POINT(29.448096512366 -95.33959264654625) bank64225 +64226 POINT(29.370583964971733 -95.66378504550282) bank64226 +64227 POINT(30.31107871607645 -95.32957178148015) bank64227 +64228 POINT(29.7421157595469 -94.52815839255834) bank64228 +64229 POINT(30.004645630110243 -94.39477556866301) bank64229 +64230 POINT(29.161348643775348 -95.47466808999542) bank64230 +64231 POINT(30.383171472752235 -94.71011473273553) bank64231 +64232 POINT(29.309812850757762 -95.00397555141024) bank64232 +64233 POINT(29.257286497918443 -96.16591115003304) bank64233 +64234 POINT(30.57435439300388 -95.55079481401636) bank64234 +64235 POINT(30.024766633281036 -94.52954228889173) bank64235 +64236 POINT(29.140648412646886 -96.09881352666721) bank64236 +64237 POINT(30.04366296274129 -94.71564754024084) bank64237 +64238 POINT(29.504528431193684 -95.81352235163526) bank64238 +64239 POINT(29.105281287587978 -95.84502279468482) bank64239 +64240 POINT(29.996921316040275 -95.45951966918969) bank64240 +64241 POINT(30.36067643001455 -94.43769410940729) bank64241 +64242 POINT(30.723039884006397 -95.19007172046825) bank64242 +64243 POINT(29.293174054711344 -95.65032695849787) bank64243 +64244 POINT(30.69896140670854 -95.18149726133339) bank64244 +64245 POINT(30.460972929763397 -96.00216618252064) bank64245 +64246 POINT(29.26625399861018 -96.01313705280084) bank64246 +64247 POINT(29.51456984565081 -94.95227888058025) bank64247 +64248 POINT(30.378353269142156 -94.85775089751594) bank64248 +64249 POINT(29.081807525709756 -96.02760750209755) bank64249 +64250 POINT(30.67137961839257 -95.06218069422063) bank64250 +64251 POINT(30.737381364482115 -95.41550656042118) bank64251 +64252 POINT(30.61995252890959 -95.12053270882053) bank64252 +64253 POINT(30.274732157426808 -96.03450742807435) bank64253 +64254 POINT(30.485549295673266 -95.77784991639724) bank64254 +64255 POINT(29.337784615786596 -95.99693494432971) bank64255 +64256 POINT(29.05035797904187 -95.73515296470663) bank64256 +64257 POINT(30.374824738644126 -95.39686101255323) bank64257 +64258 POINT(29.175564301929153 -94.56087925683104) bank64258 +64259 POINT(29.637809975258627 -95.77532250647423) bank64259 +64260 POINT(30.20118776460932 -95.84967252246243) bank64260 +64261 POINT(30.520358368283457 -94.65260392969994) bank64261 +64262 POINT(30.681497777480722 -95.7749440862536) bank64262 +64263 POINT(29.80167889206222 -94.76377659899552) bank64263 +64264 POINT(30.088210338836635 -95.44199118089698) bank64264 +64265 POINT(30.200067111707344 -95.42247414634667) bank64265 +64266 POINT(29.338411204171607 -95.33903161913392) bank64266 +64267 POINT(30.227065480059867 -95.58811620067586) bank64267 +64268 POINT(29.988101655985695 -94.44707365278734) bank64268 +64269 POINT(30.589005821616585 -95.06426741402043) bank64269 +64270 POINT(29.630048741872464 -95.39251013636397) bank64270 +64271 POINT(29.491318556084227 -94.67931783946818) bank64271 +64272 POINT(29.790512888124518 -94.42052275497602) bank64272 +64273 POINT(30.37426726598498 -94.69684621263129) bank64273 +64274 POINT(30.08022406613418 -94.45491648751361) bank64274 +64275 POINT(30.600942465116866 -95.90004929856066) bank64275 +64276 POINT(29.052179846871727 -95.64013533151359) bank64276 +64277 POINT(30.602873203671436 -95.05785710384026) bank64277 +64278 POINT(29.827261730029427 -95.07949085684425) bank64278 +64279 POINT(29.833097562856164 -95.133149288014) bank64279 +64280 POINT(28.87893704348001 -96.12382035556656) bank64280 +64281 POINT(30.527380429775995 -95.9117251325398) bank64281 +64282 POINT(29.73854834862133 -95.6130096534942) bank64282 +64283 POINT(29.13298003280807 -95.94594278395833) bank64283 +64284 POINT(29.866821067991097 -94.79713792397443) bank64284 +64285 POINT(29.40351277067133 -96.01835743879494) bank64285 +64286 POINT(29.396103959198175 -96.15918772855613) bank64286 +64287 POINT(28.830953087125966 -96.23037186710857) bank64287 +64288 POINT(30.26100918892167 -94.83678543183083) bank64288 +64289 POINT(29.096173000738254 -94.62612869491224) bank64289 +64290 POINT(29.317545515642433 -95.40804206832031) bank64290 +64291 POINT(29.239896628275883 -96.33058179983621) bank64291 +64292 POINT(29.479708365772325 -95.10291585852656) bank64292 +64293 POINT(30.418292222039803 -96.26136185588683) bank64293 +64294 POINT(29.19684300018146 -95.35107467232159) bank64294 +64295 POINT(29.86443454144996 -95.32124325080497) bank64295 +64296 POINT(29.023280003555268 -95.64140903804565) bank64296 +64297 POINT(30.400884425984813 -95.2348818613633) bank64297 +64298 POINT(29.306242704486795 -95.60232209941864) bank64298 +64299 POINT(28.8103226243398 -96.0438583538376) bank64299 +64300 POINT(30.092636644743617 -95.97224670211902) bank64300 +64301 POINT(30.331852327285645 -96.3186355931758) bank64301 +64302 POINT(29.892220828517463 -95.1779381654466) bank64302 +64303 POINT(28.83815916100585 -96.05604975288415) bank64303 +64304 POINT(29.834350384527827 -95.03818116309526) bank64304 +64305 POINT(30.57006066591274 -96.24538475815568) bank64305 +64306 POINT(29.680533512128125 -95.98101273986062) bank64306 +64307 POINT(29.14482119119944 -96.36575996209804) bank64307 +64308 POINT(29.60606737044228 -95.76296260144674) bank64308 +64309 POINT(30.186859350793515 -96.0127573935729) bank64309 +64310 POINT(29.31541595703779 -96.22927623629721) bank64310 +64311 POINT(30.650665260989367 -94.58578882005986) bank64311 +64312 POINT(29.151274297631637 -96.02446358228694) bank64312 +64313 POINT(30.180476134035537 -95.40839761953669) bank64313 +64314 POINT(29.642779548477446 -95.88009285693009) bank64314 +64315 POINT(29.49123817326503 -95.21678488575867) bank64315 +64316 POINT(28.833390709129915 -95.6862183203455) bank64316 +64317 POINT(28.92846825540298 -94.71465724593972) bank64317 +64318 POINT(29.503293581797866 -94.4681139291577) bank64318 +64319 POINT(29.398105825846542 -94.87753464758148) bank64319 +64320 POINT(29.41145539186671 -95.70185993643862) bank64320 +64321 POINT(29.867550330960203 -96.20382726938526) bank64321 +64322 POINT(30.05312593541334 -94.5177353613257) bank64322 +64323 POINT(30.491052586663322 -95.90246928712534) bank64323 +64324 POINT(30.25266318651615 -94.75565625517807) bank64324 +64325 POINT(30.225914836771935 -95.42059686454209) bank64325 +64326 POINT(28.889408328713824 -95.17248907016683) bank64326 +64327 POINT(28.91933415913496 -94.93890699649687) bank64327 +64328 POINT(28.922409521418754 -95.26803249339065) bank64328 +64329 POINT(28.981316351668646 -94.8860114851139) bank64329 +64330 POINT(30.432004643685435 -96.14874549913351) bank64330 +64331 POINT(30.71909501255494 -95.9257925127614) bank64331 +64332 POINT(29.3593492183228 -94.68032830808372) bank64332 +64333 POINT(30.196350134934953 -94.88033031280476) bank64333 +64334 POINT(30.07213578617409 -95.73818829048616) bank64334 +64335 POINT(30.465014773580087 -96.00111892520249) bank64335 +64336 POINT(30.67576994026226 -96.04417504752439) bank64336 +64337 POINT(30.740457307901583 -96.26685014640746) bank64337 +64338 POINT(29.64988667013738 -96.30445837688262) bank64338 +64339 POINT(28.926548962379613 -95.2944128684103) bank64339 +64340 POINT(29.539348390910632 -96.07028175210435) bank64340 +64341 POINT(29.205138696411296 -95.32305310051866) bank64341 +64342 POINT(29.004559982014886 -94.84770138037618) bank64342 +64343 POINT(29.844656473205966 -95.66820085080406) bank64343 +64344 POINT(29.02446027893794 -94.99035122715972) bank64344 +64345 POINT(29.295132516767566 -95.6301235305716) bank64345 +64346 POINT(29.36029655481332 -96.24987927922179) bank64346 +64347 POINT(29.60612905788015 -94.79253480171728) bank64347 +64348 POINT(30.631722355870505 -95.89104540315046) bank64348 +64349 POINT(29.27788271633865 -95.67453056190871) bank64349 +64350 POINT(29.13918670506414 -96.12683837966182) bank64350 +64351 POINT(30.660179255063177 -95.74609352636301) bank64351 +64352 POINT(29.900326909872355 -95.39015310920468) bank64352 +64353 POINT(29.751004637445195 -95.5323261735627) bank64353 +64354 POINT(29.206622628689377 -94.70665771434535) bank64354 +64355 POINT(30.23598210523436 -95.48867542741715) bank64355 +64356 POINT(30.556325198971003 -95.77531613757617) bank64356 +64357 POINT(30.65864674146304 -95.40929200484226) bank64357 +64358 POINT(30.692662696306755 -95.91910208850075) bank64358 +64359 POINT(29.325532536486236 -95.32790073301008) bank64359 +64360 POINT(30.10680401550047 -95.06692309553807) bank64360 +64361 POINT(30.500822222463775 -94.62732710275627) bank64361 +64362 POINT(29.192417171762123 -94.38652646653797) bank64362 +64363 POINT(30.14794356218432 -95.20788640549303) bank64363 +64364 POINT(30.126065005246897 -94.88894324237768) bank64364 +64365 POINT(29.698586924889522 -95.60382662213856) bank64365 +64366 POINT(28.84638979144277 -94.49014737592928) bank64366 +64367 POINT(29.25845227526054 -95.35506262441204) bank64367 +64368 POINT(29.578385588954518 -95.95353266941626) bank64368 +64369 POINT(29.306357109225505 -94.38195778985417) bank64369 +64370 POINT(30.379758987551018 -95.8511857203786) bank64370 +64371 POINT(30.300293258484214 -95.52362392175264) bank64371 +64372 POINT(29.307013382951396 -94.60248892485514) bank64372 +64373 POINT(29.053550353343752 -94.45126529321992) bank64373 +64374 POINT(30.679872592003008 -94.4276961521923) bank64374 +64375 POINT(29.073112635938575 -94.72570949002717) bank64375 +64376 POINT(28.768708568984767 -94.59449207736218) bank64376 +64377 POINT(30.52990344552468 -96.20080008060006) bank64377 +64378 POINT(30.06115331679829 -95.14464040268234) bank64378 +64379 POINT(29.51846608044952 -94.53891324045233) bank64379 +64380 POINT(29.312487505726914 -95.88816970849408) bank64380 +64381 POINT(28.988331230276852 -96.00936187567346) bank64381 +64382 POINT(29.05684020336424 -95.43085616812395) bank64382 +64383 POINT(29.533233560811013 -95.28339099724838) bank64383 +64384 POINT(29.61141089220725 -96.17613152667258) bank64384 +64385 POINT(30.349941219405807 -94.51723663754518) bank64385 +64386 POINT(28.925065669800397 -95.0890793844723) bank64386 +64387 POINT(29.807860192334516 -95.11515083296719) bank64387 +64388 POINT(28.801657521614064 -95.44805729904624) bank64388 +64389 POINT(28.815406626354136 -94.89739866841526) bank64389 +64390 POINT(29.812929804326963 -95.92105423471386) bank64390 +64391 POINT(29.89522000512187 -95.65891534357944) bank64391 +64392 POINT(29.89059021153117 -95.40061037058406) bank64392 +64393 POINT(29.121286248709282 -95.98419662647369) bank64393 +64394 POINT(29.654809366724994 -95.09091097014587) bank64394 +64395 POINT(29.5413084671069 -95.21988612937763) bank64395 +64396 POINT(28.862378954660617 -94.71615357365039) bank64396 +64397 POINT(29.287545466733917 -96.29700261853287) bank64397 +64398 POINT(30.545098108968357 -94.67166583597702) bank64398 +64399 POINT(29.233373396404136 -94.61289939721593) bank64399 +64400 POINT(30.720087032499347 -95.77559542792893) bank64400 +64401 POINT(30.401487944022616 -95.70812205340465) bank64401 +64402 POINT(28.766979020556306 -94.72481327262355) bank64402 +64403 POINT(29.85541725574845 -94.89415347507446) bank64403 +64404 POINT(30.513389145014052 -95.11427692730742) bank64404 +64405 POINT(29.64131298063857 -94.62555687975426) bank64405 +64406 POINT(30.627148588947165 -94.87991626916076) bank64406 +64407 POINT(29.294613167881266 -95.62650621608742) bank64407 +64408 POINT(30.184779307273402 -96.12072115344279) bank64408 +64409 POINT(28.98817562915753 -94.49865523247814) bank64409 +64410 POINT(30.223094277601785 -95.34874163905552) bank64410 +64411 POINT(28.843473364977662 -94.54377979783723) bank64411 +64412 POINT(30.035347893486364 -94.67274014632153) bank64412 +64413 POINT(30.713200578988804 -95.42219118874239) bank64413 +64414 POINT(29.24807319449133 -95.84387332005556) bank64414 +64415 POINT(30.137255556383185 -96.24888285885378) bank64415 +64416 POINT(28.98862669555498 -95.95557552437602) bank64416 +64417 POINT(30.66045050423177 -95.61761092011474) bank64417 +64418 POINT(29.305732382740633 -94.70053224104818) bank64418 +64419 POINT(29.299483844380386 -96.15730801911269) bank64419 +64420 POINT(30.67619829369861 -95.34235699252517) bank64420 +64421 POINT(30.46615813029701 -96.30013664697445) bank64421 +64422 POINT(30.236294784027113 -95.52369413295435) bank64422 +64423 POINT(29.994237264844667 -95.94418233167985) bank64423 +64424 POINT(29.981548175392902 -94.65523871300682) bank64424 +64425 POINT(29.53955096256014 -96.09798400130197) bank64425 +64426 POINT(30.396670367377013 -95.60377651831615) bank64426 +64427 POINT(30.474093778859814 -95.76934249462087) bank64427 +64428 POINT(29.50313959361391 -95.5403148007461) bank64428 +64429 POINT(29.977825933932305 -96.01203897624673) bank64429 +64430 POINT(30.565970913956708 -94.37270871877577) bank64430 +64431 POINT(28.950309926001797 -95.0463035524372) bank64431 +64432 POINT(29.855753956107332 -95.89405148449998) bank64432 +64433 POINT(30.252875197472868 -95.54921258958528) bank64433 +64434 POINT(29.990222345158145 -95.04123341277376) bank64434 +64435 POINT(30.239614649495177 -96.13533020949149) bank64435 +64436 POINT(29.059055204934257 -96.34099999004445) bank64436 +64437 POINT(29.891220345995727 -95.74600897528065) bank64437 +64438 POINT(29.17069724147145 -95.49004270001996) bank64438 +64439 POINT(28.9021594872661 -94.76438598718917) bank64439 +64440 POINT(29.592445377579434 -95.50207433903556) bank64440 +64441 POINT(29.845224901284958 -95.82076644196158) bank64441 +64442 POINT(29.127974178518915 -96.09654977369541) bank64442 +64443 POINT(29.53813193837681 -95.78475374282321) bank64443 +64444 POINT(29.848723448624398 -94.3868125674694) bank64444 +64445 POINT(30.004929076631342 -95.71312542312518) bank64445 +64446 POINT(29.761153129874973 -94.37341238083667) bank64446 +64447 POINT(29.49145000950083 -96.20965752298603) bank64447 +64448 POINT(29.78744452069926 -95.41291355194747) bank64448 +64449 POINT(29.882366630826862 -95.40466545643095) bank64449 +64450 POINT(30.343656824580094 -95.79955977284872) bank64450 +64451 POINT(29.151425594231487 -94.78420436850035) bank64451 +64452 POINT(29.36804817116437 -94.59951413743899) bank64452 +64453 POINT(29.47664205613593 -95.33915485495078) bank64453 +64454 POINT(28.89575246672504 -96.30466378318063) bank64454 +64455 POINT(29.44685310459449 -95.8437243179135) bank64455 +64456 POINT(28.98737790507626 -94.66740801458111) bank64456 +64457 POINT(29.22039630506203 -95.01020707554781) bank64457 +64458 POINT(30.56812697921133 -94.78241180534049) bank64458 +64459 POINT(29.92262857835669 -95.9017995326643) bank64459 +64460 POINT(30.661179000846534 -94.61096737352695) bank64460 +64461 POINT(30.582234131369546 -95.87644391758495) bank64461 +64462 POINT(29.109575666342078 -95.74228672729993) bank64462 +64463 POINT(29.998114989688982 -95.98204548902196) bank64463 +64464 POINT(30.4428693104054 -95.83434883498072) bank64464 +64465 POINT(28.83395584820093 -95.92374044850403) bank64465 +64466 POINT(30.255093999279246 -94.55398186434232) bank64466 +64467 POINT(29.541785941697576 -94.96782864650388) bank64467 +64468 POINT(29.731929282960778 -94.77543664269544) bank64468 +64469 POINT(29.554617489641675 -95.00621883772136) bank64469 +64470 POINT(29.859066374185883 -95.88365305997331) bank64470 +64471 POINT(29.33800558991532 -94.51904406919684) bank64471 +64472 POINT(30.48468728059084 -96.34845584888443) bank64472 +64473 POINT(29.789312217987913 -95.60673448365688) bank64473 +64474 POINT(28.960552927743304 -95.02511517650576) bank64474 +64475 POINT(30.32723926062575 -95.26743713997976) bank64475 +64476 POINT(30.37936049080007 -94.90096028259772) bank64476 +64477 POINT(29.770554315803565 -94.54211054020756) bank64477 +64478 POINT(30.498762122263223 -94.4528906285465) bank64478 +64479 POINT(29.54184014795804 -94.63299513163024) bank64479 +64480 POINT(30.732040905454365 -95.50524706085945) bank64480 +64481 POINT(29.664502094648903 -94.81598755526646) bank64481 +64482 POINT(30.47104318532838 -94.81780665999541) bank64482 +64483 POINT(30.749385419348336 -95.01122625498347) bank64483 +64484 POINT(30.5041912063395 -95.03366963699689) bank64484 +64485 POINT(29.21188066198172 -94.41165165910336) bank64485 +64486 POINT(30.22690947073064 -94.96616733794505) bank64486 +64487 POINT(30.350238596118906 -95.34624730358333) bank64487 +64488 POINT(29.473100942285136 -95.22531269127691) bank64488 +64489 POINT(28.828718023139047 -95.11080068984711) bank64489 +64490 POINT(30.085208206079074 -95.27468724952409) bank64490 +64491 POINT(29.539174183367173 -94.44000188538912) bank64491 +64492 POINT(30.386355290729682 -95.20552863039293) bank64492 +64493 POINT(29.61729502256108 -94.89955465560324) bank64493 +64494 POINT(29.323742914581736 -94.79624378929272) bank64494 +64495 POINT(30.223008732678423 -95.25001267340599) bank64495 +64496 POINT(28.944275949675305 -95.07344392133473) bank64496 +64497 POINT(29.48350252649657 -94.68383400485948) bank64497 +64498 POINT(29.709300823174743 -96.25463986882326) bank64498 +64499 POINT(30.19824669906286 -95.7859370453182) bank64499 +64500 POINT(29.881091420004438 -95.17326993066058) bank64500 +64501 POINT(30.091220626530703 -94.38684926334837) bank64501 +64502 POINT(29.141988809078008 -94.47093257385227) bank64502 +64503 POINT(29.35489971516002 -95.07925368629887) bank64503 +64504 POINT(30.373545749202446 -95.38345820827462) bank64504 +64505 POINT(30.144080926832284 -95.28343551260183) bank64505 +64506 POINT(30.518544700713676 -95.8062713592528) bank64506 +64507 POINT(30.110090409746785 -95.22271341086207) bank64507 +64508 POINT(29.54052509281226 -94.87717903052359) bank64508 +64509 POINT(29.303614219435833 -94.81276974375088) bank64509 +64510 POINT(28.93999845552165 -95.18930703797562) bank64510 +64511 POINT(29.972179467324317 -94.716058349405) bank64511 +64512 POINT(29.89729030347546 -95.24479354588843) bank64512 +64513 POINT(29.133673292533327 -94.75758881075329) bank64513 +64514 POINT(28.82005641702896 -95.8105293337258) bank64514 +64515 POINT(28.806216925081088 -94.76452129995909) bank64515 +64516 POINT(29.424385691889146 -95.22587388631672) bank64516 +64517 POINT(29.713862981983027 -95.18600632685265) bank64517 +64518 POINT(29.198667971236873 -95.8577287098506) bank64518 +64519 POINT(29.487721107351266 -95.6750768769444) bank64519 +64520 POINT(28.906930548327523 -95.89445934098876) bank64520 +64521 POINT(29.35333196443282 -96.26985495450067) bank64521 +64522 POINT(30.066596801878724 -95.80281708117317) bank64522 +64523 POINT(29.192211345676633 -95.25803996937084) bank64523 +64524 POINT(29.883571509083513 -96.00533291218052) bank64524 +64525 POINT(28.991745009291343 -95.9840956143411) bank64525 +64526 POINT(28.940904837383215 -96.10679512872095) bank64526 +64527 POINT(29.783708585461174 -94.78732971129351) bank64527 +64528 POINT(29.280629854502227 -95.73673128299829) bank64528 +64529 POINT(29.39289895468124 -95.98052226343597) bank64529 +64530 POINT(29.30370248353632 -94.6165390056066) bank64530 +64531 POINT(30.141528442723246 -95.60029446590461) bank64531 +64532 POINT(29.935790347821467 -94.64531112069284) bank64532 +64533 POINT(30.099733255701683 -95.9855830348245) bank64533 +64534 POINT(29.299386230756923 -95.59290050081437) bank64534 +64535 POINT(30.112569737241373 -95.58707699316457) bank64535 +64536 POINT(29.143699851352974 -95.03752667721514) bank64536 +64537 POINT(29.073230949040873 -94.60943278313344) bank64537 +64538 POINT(30.18547361066384 -96.1846926364354) bank64538 +64539 POINT(30.251592555001256 -95.96647898732446) bank64539 +64540 POINT(30.133722853701755 -94.91195138431856) bank64540 +64541 POINT(29.93796060649439 -94.83666888185948) bank64541 +64542 POINT(29.34014671159793 -95.49269890775479) bank64542 +64543 POINT(29.450849263664516 -95.73513140839903) bank64543 +64544 POINT(29.871725741455244 -94.8043964844219) bank64544 +64545 POINT(29.53762018904948 -94.84866264156088) bank64545 +64546 POINT(29.862746464000608 -94.42364466618073) bank64546 +64547 POINT(29.323761125349392 -95.63875083406516) bank64547 +64548 POINT(29.264478707897933 -95.86699354368595) bank64548 +64549 POINT(29.558933160024843 -95.84175629188651) bank64549 +64550 POINT(28.78220403988138 -96.02337895546624) bank64550 +64551 POINT(29.23752883947899 -94.54088366958136) bank64551 +64552 POINT(29.520292112805066 -94.59208402025895) bank64552 +64553 POINT(29.365032283704025 -95.38761922084329) bank64553 +64554 POINT(28.85884040958314 -96.28219566629795) bank64554 +64555 POINT(30.321028222890213 -96.1494674850527) bank64555 +64556 POINT(30.249733291393795 -95.35271531027394) bank64556 +64557 POINT(29.179610483930386 -95.50356121065073) bank64557 +64558 POINT(30.0948143973366 -96.26351042388629) bank64558 +64559 POINT(29.615149392184318 -95.29125816073807) bank64559 +64560 POINT(29.218688602747445 -96.1653329191736) bank64560 +64561 POINT(29.827454757454568 -95.1184332754663) bank64561 +64562 POINT(30.27075082722274 -95.22272994128602) bank64562 +64563 POINT(30.650519674792267 -95.69674305600499) bank64563 +64564 POINT(30.235761429592745 -95.33852218513779) bank64564 +64565 POINT(29.4830949365086 -94.52666249982266) bank64565 +64566 POINT(29.3920770996018 -95.71102528083853) bank64566 +64567 POINT(29.843573396982222 -94.51265981287209) bank64567 +64568 POINT(29.10286664382407 -94.4697870787875) bank64568 +64569 POINT(29.999475625333627 -95.94286589930688) bank64569 +64570 POINT(28.93240458067391 -94.91315205178766) bank64570 +64571 POINT(28.90488947713286 -94.80395209147711) bank64571 +64572 POINT(29.659488439141864 -94.94534243815846) bank64572 +64573 POINT(29.395805982406255 -96.22565152449324) bank64573 +64574 POINT(29.111657436185645 -95.60979450553732) bank64574 +64575 POINT(28.98243274904935 -95.30626150839049) bank64575 +64576 POINT(28.824381685174416 -94.72201135732341) bank64576 +64577 POINT(29.03746298691267 -94.48822574477884) bank64577 +64578 POINT(29.89538106524048 -95.46053956851935) bank64578 +64579 POINT(30.070626648858475 -96.03409720569432) bank64579 +64580 POINT(29.242454270897042 -94.44499731200342) bank64580 +64581 POINT(30.495420176529993 -96.35906609598473) bank64581 +64582 POINT(30.412582593757865 -95.52345330123319) bank64582 +64583 POINT(30.01973654657694 -94.3852729221185) bank64583 +64584 POINT(30.659328853654575 -94.77101142660237) bank64584 +64585 POINT(28.80889196819774 -95.51333213612905) bank64585 +64586 POINT(29.196032928677784 -94.8915502865458) bank64586 +64587 POINT(29.09372386376584 -95.17236892017263) bank64587 +64588 POINT(29.525062070344397 -95.2355311058313) bank64588 +64589 POINT(29.781428002851293 -95.12885890068151) bank64589 +64590 POINT(28.85464240237306 -95.16589287841461) bank64590 +64591 POINT(28.8803928076997 -95.30607176345511) bank64591 +64592 POINT(29.656782999272632 -94.70493631022568) bank64592 +64593 POINT(30.401257639783363 -94.70125687375258) bank64593 +64594 POINT(28.80560569967032 -95.16385089074055) bank64594 +64595 POINT(28.874894926016978 -95.46631406217753) bank64595 +64596 POINT(30.38033613866464 -95.69863847904853) bank64596 +64597 POINT(28.88552473953947 -95.65331453716031) bank64597 +64598 POINT(30.561084677452435 -95.5289431179726) bank64598 +64599 POINT(28.907383825488004 -95.66813649934099) bank64599 +64600 POINT(29.76652091793411 -95.39599706491532) bank64600 +64601 POINT(28.89913655995734 -94.67949735490343) bank64601 +64602 POINT(29.827461839634868 -96.25217110233466) bank64602 +64603 POINT(29.643282424313522 -94.98461443453648) bank64603 +64604 POINT(29.938246891520148 -96.26260299508742) bank64604 +64605 POINT(29.664022131288927 -95.71038496650316) bank64605 +64606 POINT(29.17364692891511 -94.9231672975883) bank64606 +64607 POINT(29.499308275013323 -94.85539584010907) bank64607 +64608 POINT(29.5348083710917 -95.67867059512801) bank64608 +64609 POINT(30.312473397782526 -94.68789718818678) bank64609 +64610 POINT(30.68515937994323 -95.97065310743892) bank64610 +64611 POINT(30.711029671326955 -95.51418242372921) bank64611 +64612 POINT(28.975413054809636 -95.51776761776938) bank64612 +64613 POINT(30.324565162144975 -95.99452275620466) bank64613 +64614 POINT(29.9588790608066 -96.3424412891384) bank64614 +64615 POINT(28.768795279837146 -94.539482325448) bank64615 +64616 POINT(30.298061601484953 -95.70217011595138) bank64616 +64617 POINT(30.000939249885466 -95.91161431841749) bank64617 +64618 POINT(28.786020092794903 -94.39452323921901) bank64618 +64619 POINT(30.47251830443991 -96.0646109526493) bank64619 +64620 POINT(30.469716468287483 -94.78217226993456) bank64620 +64621 POINT(28.84046965953863 -94.49328193805903) bank64621 +64622 POINT(30.733234171465526 -96.16935161229856) bank64622 +64623 POINT(29.4969264016788 -96.10006838598676) bank64623 +64624 POINT(28.780088256197445 -94.91039425446252) bank64624 +64625 POINT(29.55802719135404 -95.1682931022369) bank64625 +64626 POINT(29.084671241656896 -94.49865875530296) bank64626 +64627 POINT(29.234548499790044 -95.62219781201607) bank64627 +64628 POINT(29.78058471522992 -94.43443823595565) bank64628 +64629 POINT(29.494585813902695 -95.52088842921324) bank64629 +64630 POINT(30.744079486317265 -94.90937543529418) bank64630 +64631 POINT(29.503030239006026 -95.41410896291663) bank64631 +64632 POINT(29.306759603070137 -95.30823982732011) bank64632 +64633 POINT(28.99786537699077 -94.7098763824772) bank64633 +64634 POINT(30.372028495342512 -95.39098170498215) bank64634 +64635 POINT(29.87917591300032 -96.16792477923163) bank64635 +64636 POINT(30.259195529274855 -95.59782324416213) bank64636 +64637 POINT(29.601045505197366 -95.88412179940082) bank64637 +64638 POINT(30.24892507599355 -94.79474467562736) bank64638 +64639 POINT(29.378232341346827 -96.10529162828863) bank64639 +64640 POINT(29.974151515884373 -95.61020694519227) bank64640 +64641 POINT(28.96016614325864 -95.39477239301938) bank64641 +64642 POINT(28.949746601668917 -95.47621260649635) bank64642 +64643 POINT(29.651507712749652 -94.43905075762488) bank64643 +64644 POINT(29.910225024764436 -95.54026461568239) bank64644 +64645 POINT(30.510674227100917 -95.38259463887597) bank64645 +64646 POINT(30.18568467309725 -95.5111726417597) bank64646 +64647 POINT(30.292286904048492 -94.60969734104793) bank64647 +64648 POINT(29.740621837249964 -95.77771367088909) bank64648 +64649 POINT(29.871770261748754 -94.7442921116107) bank64649 +64650 POINT(28.940068388369887 -96.34160836191577) bank64650 +64651 POINT(29.4618950575336 -95.9354706184603) bank64651 +64652 POINT(30.447346331253627 -95.07318537497433) bank64652 +64653 POINT(28.9524068030241 -96.18465423915288) bank64653 +64654 POINT(30.130319571524616 -95.59053941068709) bank64654 +64655 POINT(29.883682396078285 -94.74314771799848) bank64655 +64656 POINT(30.395997709587352 -95.39970812620639) bank64656 +64657 POINT(28.815849168102844 -95.6601584034102) bank64657 +64658 POINT(30.393551281897253 -95.55062118146569) bank64658 +64659 POINT(29.162535317968516 -95.5928376515077) bank64659 +64660 POINT(30.401608911122995 -96.15618334997747) bank64660 +64661 POINT(29.33166536258783 -94.5171888448282) bank64661 +64662 POINT(29.221747898351172 -96.21716228098173) bank64662 +64663 POINT(30.583037651341428 -95.14060939144214) bank64663 +64664 POINT(29.328784458844176 -94.57683506047496) bank64664 +64665 POINT(30.199173702370604 -94.70112459308147) bank64665 +64666 POINT(29.816379134215286 -95.86558276349545) bank64666 +64667 POINT(29.044485268751643 -94.77600523785209) bank64667 +64668 POINT(30.628499524147006 -96.35713093711666) bank64668 +64669 POINT(30.677151704636668 -96.28896398829204) bank64669 +64670 POINT(29.973460033973 -95.63544313933794) bank64670 +64671 POINT(29.330080993212242 -95.65703847947104) bank64671 +64672 POINT(29.611363161745917 -94.5713204808742) bank64672 +64673 POINT(28.96655688416855 -94.98670293343905) bank64673 +64674 POINT(29.05545412650689 -94.59877906075963) bank64674 +64675 POINT(30.101759022243726 -95.28391068703037) bank64675 +64676 POINT(30.54465790151277 -94.67729911816957) bank64676 +64677 POINT(29.059155949059598 -95.45142052657975) bank64677 +64678 POINT(29.25186430041687 -94.42008033712243) bank64678 +64679 POINT(29.53922822406411 -95.30189612461226) bank64679 +64680 POINT(29.123457052797985 -94.39707967132269) bank64680 +64681 POINT(30.049322206233903 -94.75980657993249) bank64681 +64682 POINT(30.12978962238018 -94.7043165307773) bank64682 +64683 POINT(30.30134645888099 -95.15041669918621) bank64683 +64684 POINT(29.42306859777811 -96.16011278868731) bank64684 +64685 POINT(29.979030543558235 -96.14853788809128) bank64685 +64686 POINT(29.358939668454845 -96.29206318184154) bank64686 +64687 POINT(30.240951049704833 -94.8363146161486) bank64687 +64688 POINT(28.94778139617652 -96.23085167197456) bank64688 +64689 POINT(28.985316780268292 -96.28956143507597) bank64689 +64690 POINT(30.712126240919833 -94.73470800017488) bank64690 +64691 POINT(29.960312692775698 -94.87567787065706) bank64691 +64692 POINT(30.069567941981738 -95.91722961703525) bank64692 +64693 POINT(30.659664627718097 -94.84664096829972) bank64693 +64694 POINT(29.88639124640936 -94.79031855240737) bank64694 +64695 POINT(29.301208123295154 -95.13877176571843) bank64695 +64696 POINT(30.518439931091436 -94.98941365922381) bank64696 +64697 POINT(30.23949062947641 -94.69024357663766) bank64697 +64698 POINT(29.389383324846268 -95.21292425049164) bank64698 +64699 POINT(29.3238611952859 -94.50577044244686) bank64699 +64700 POINT(29.254938097953303 -95.3891672796561) bank64700 +64701 POINT(30.051884553046108 -94.80167428995286) bank64701 +64702 POINT(28.96864994506042 -96.33317549118976) bank64702 +64703 POINT(29.435389572816984 -95.50015675521718) bank64703 +64704 POINT(30.189926746173963 -95.82879070993555) bank64704 +64705 POINT(29.61191822281029 -95.75555928443103) bank64705 +64706 POINT(29.224206300938185 -95.56026576334258) bank64706 +64707 POINT(29.36326432558811 -95.69389186497175) bank64707 +64708 POINT(30.26538098371822 -95.65274716518311) bank64708 +64709 POINT(29.331244307908207 -95.95412427338607) bank64709 +64710 POINT(30.137603523892025 -95.10731954612332) bank64710 +64711 POINT(29.727350666203698 -94.95470461347183) bank64711 +64712 POINT(30.665380510856604 -95.37758922628011) bank64712 +64713 POINT(30.406077178747392 -94.56996602875859) bank64713 +64714 POINT(29.858509273531745 -96.12785118592011) bank64714 +64715 POINT(30.379715950777964 -95.60906052244015) bank64715 +64716 POINT(29.58148336117622 -96.30735469182828) bank64716 +64717 POINT(29.642449274299924 -95.13138542829559) bank64717 +64718 POINT(29.102435036939408 -95.18575572696041) bank64718 +64719 POINT(29.00816182097448 -95.44641099607433) bank64719 +64720 POINT(29.24416011500303 -94.84047153042185) bank64720 +64721 POINT(30.004535699376397 -94.37974556628181) bank64721 +64722 POINT(29.811457895600267 -95.33049077247456) bank64722 +64723 POINT(28.99105640173208 -95.94980977822661) bank64723 +64724 POINT(28.966743003693495 -94.94861349088762) bank64724 +64725 POINT(29.60791121920606 -96.28777604212907) bank64725 +64726 POINT(29.786352196228133 -95.95774837196008) bank64726 +64727 POINT(29.226870526704268 -95.17926427881712) bank64727 +64728 POINT(29.197510369594212 -95.78990769529618) bank64728 +64729 POINT(29.756152216544237 -95.65749926194265) bank64729 +64730 POINT(30.533645400795827 -95.58374381924361) bank64730 +64731 POINT(29.763941629946448 -95.20086076734374) bank64731 +64732 POINT(30.685434676018865 -96.28076927647669) bank64732 +64733 POINT(28.97525935907814 -96.03978683093) bank64733 +64734 POINT(29.85633593926386 -95.9826722751886) bank64734 +64735 POINT(28.78613227715376 -96.1821447487592) bank64735 +64736 POINT(29.421155322888925 -94.66106718198914) bank64736 +64737 POINT(28.905477787087744 -94.98456735760331) bank64737 +64738 POINT(29.241712321037134 -94.42703383956658) bank64738 +64739 POINT(30.739054853947273 -95.23517770071618) bank64739 +64740 POINT(30.163184116766327 -95.53038146611287) bank64740 +64741 POINT(29.419096609165027 -96.06494468651724) bank64741 +64742 POINT(29.338181492938407 -95.65427138508218) bank64742 +64743 POINT(30.483669987337212 -95.53454262222515) bank64743 +64744 POINT(30.51752426912887 -95.931352002782) bank64744 +64745 POINT(29.224552008849663 -95.76631628808391) bank64745 +64746 POINT(30.72743186565317 -94.63765798594925) bank64746 +64747 POINT(29.04939622650465 -95.22215306906503) bank64747 +64748 POINT(29.685863039387804 -95.36154783907477) bank64748 +64749 POINT(30.32372826811398 -95.53644882001129) bank64749 +64750 POINT(30.33045203229153 -95.49578914011948) bank64750 +64751 POINT(30.222590320456707 -95.71328999955206) bank64751 +64752 POINT(29.23057329837643 -95.14262970965947) bank64752 +64753 POINT(29.513910737704826 -96.12661469305156) bank64753 +64754 POINT(29.216087626174954 -94.96980691203713) bank64754 +64755 POINT(29.689708948992116 -95.43411563667537) bank64755 +64756 POINT(29.866870716415377 -95.39329951897892) bank64756 +64757 POINT(29.470219766490796 -95.84827089721257) bank64757 +64758 POINT(29.03715909517782 -95.7249406053114) bank64758 +64759 POINT(30.649125097394442 -95.18965078773094) bank64759 +64760 POINT(29.29263036638673 -95.63902337051539) bank64760 +64761 POINT(29.319851726364462 -94.57468400558756) bank64761 +64762 POINT(29.8815233423326 -94.99900748649448) bank64762 +64763 POINT(30.000772399748815 -95.54229204173696) bank64763 +64764 POINT(30.62164481992258 -96.15365265315249) bank64764 +64765 POINT(28.985310400964938 -94.73481069569954) bank64765 +64766 POINT(29.693743654801168 -95.6106277846272) bank64766 +64767 POINT(29.651805751110857 -94.58357819164786) bank64767 +64768 POINT(29.911373048296753 -95.91893195340941) bank64768 +64769 POINT(30.03288299590205 -95.16106130422854) bank64769 +64770 POINT(29.206041675246105 -95.0693787147044) bank64770 +64771 POINT(29.978254550011837 -96.07503766255611) bank64771 +64772 POINT(29.264871615338304 -95.82832329975362) bank64772 +64773 POINT(30.459295166009646 -95.2752618571247) bank64773 +64774 POINT(30.11055763699017 -95.91890577400935) bank64774 +64775 POINT(30.68446209995963 -94.65172610068132) bank64775 +64776 POINT(29.17449507098411 -96.27196560673092) bank64776 +64777 POINT(29.955211883832906 -94.75867323883189) bank64777 +64778 POINT(29.918016880710496 -95.9980244583199) bank64778 +64779 POINT(30.590167467758466 -95.56840257010965) bank64779 +64780 POINT(29.89712808781957 -95.84801661199327) bank64780 +64781 POINT(29.782032306042908 -95.78202254500711) bank64781 +64782 POINT(30.145427389870747 -94.59968181335338) bank64782 +64783 POINT(29.43812670384952 -94.95607284024456) bank64783 +64784 POINT(29.7695329393271 -94.60267431259325) bank64784 +64785 POINT(29.958678227600483 -95.26316857126129) bank64785 +64786 POINT(30.189153635320487 -96.33855558128064) bank64786 +64787 POINT(29.287850357102425 -96.35678931483412) bank64787 +64788 POINT(29.222813486374733 -96.32561820551459) bank64788 +64789 POINT(30.757838990872344 -94.45962086231086) bank64789 +64790 POINT(29.85080350205638 -95.1078667342359) bank64790 +64791 POINT(29.68365968137673 -95.142319874942) bank64791 +64792 POINT(30.071917892929974 -95.96213619833759) bank64792 +64793 POINT(29.016411318633242 -95.36089642251055) bank64793 +64794 POINT(29.94269563545226 -95.07450564458902) bank64794 +64795 POINT(28.84562804397387 -95.86525476907433) bank64795 +64796 POINT(29.88457202356991 -95.30184854541953) bank64796 +64797 POINT(29.707117215453938 -94.61666324473092) bank64797 +64798 POINT(28.848347416096793 -95.68417322305311) bank64798 +64799 POINT(29.266340255941703 -95.94490186049357) bank64799 +64800 POINT(29.384133268674898 -94.94516148583651) bank64800 +64801 POINT(30.47339045985597 -95.30816502036606) bank64801 +64802 POINT(30.102685454819465 -95.8166955128513) bank64802 +64803 POINT(29.70568623453768 -95.22015057496785) bank64803 +64804 POINT(29.67358211512304 -96.1912808466883) bank64804 +64805 POINT(30.67512790222429 -95.03540614087399) bank64805 +64806 POINT(30.564613558034097 -95.22914787394113) bank64806 +64807 POINT(29.007922669660893 -96.19773873097199) bank64807 +64808 POINT(30.403072849253363 -95.12647617847125) bank64808 +64809 POINT(28.970284921193247 -95.25548062163071) bank64809 +64810 POINT(30.0601151998151 -94.94436837016592) bank64810 +64811 POINT(30.13153824371273 -95.11383319328344) bank64811 +64812 POINT(30.553227871024877 -96.22353739568162) bank64812 +64813 POINT(30.003938248443983 -95.99519815208984) bank64813 +64814 POINT(28.91587977097782 -95.08631488594581) bank64814 +64815 POINT(28.867276302429786 -94.37534737017135) bank64815 +64816 POINT(28.8161669220852 -96.00259616418346) bank64816 +64817 POINT(30.407988643735965 -95.71624430195604) bank64817 +64818 POINT(30.681924509854852 -95.17210621232248) bank64818 +64819 POINT(29.627558180732446 -95.29344566851621) bank64819 +64820 POINT(28.876466154742985 -96.22245582593803) bank64820 +64821 POINT(30.206487287769892 -95.94551913500538) bank64821 +64822 POINT(30.690593883624654 -95.17687653525918) bank64822 +64823 POINT(30.665501557800035 -95.37686018916452) bank64823 +64824 POINT(30.016822458509125 -95.86261260800273) bank64824 +64825 POINT(29.241527569582402 -95.28534101994394) bank64825 +64826 POINT(29.019715677943864 -96.35374838417636) bank64826 +64827 POINT(29.3220293749956 -94.7911923215373) bank64827 +64828 POINT(29.452642963418832 -96.29786762146475) bank64828 +64829 POINT(29.908583322308086 -94.58199792302518) bank64829 +64830 POINT(29.200220632776276 -95.89860008932749) bank64830 +64831 POINT(29.49314265330737 -95.62260187285702) bank64831 +64832 POINT(30.02960144464251 -96.11638346093913) bank64832 +64833 POINT(30.131646370211637 -95.66889505122113) bank64833 +64834 POINT(28.984004838017963 -95.3052483350292) bank64834 +64835 POINT(30.73483677849762 -95.66978946169499) bank64835 +64836 POINT(30.424321535333707 -95.33204449573883) bank64836 +64837 POINT(30.101294769578832 -94.78246713153361) bank64837 +64838 POINT(29.264050199793793 -94.82162259792307) bank64838 +64839 POINT(28.90313901267984 -94.42113005276303) bank64839 +64840 POINT(30.41750436558651 -96.06672371714156) bank64840 +64841 POINT(29.054801756114212 -94.51014422741902) bank64841 +64842 POINT(29.29247469654329 -95.00154386975282) bank64842 +64843 POINT(30.553537570158856 -95.3438862768896) bank64843 +64844 POINT(28.805983971704077 -95.71640715483821) bank64844 +64845 POINT(30.54786904195661 -95.84106616807662) bank64845 +64846 POINT(29.509972869709344 -95.77930029579706) bank64846 +64847 POINT(29.76872780008593 -95.44879849963618) bank64847 +64848 POINT(29.296337991238204 -95.07105154782894) bank64848 +64849 POINT(29.69333524151247 -95.33520949672149) bank64849 +64850 POINT(30.142411347313462 -96.17193396288718) bank64850 +64851 POINT(29.958943205383445 -95.31447151692882) bank64851 +64852 POINT(29.09498228356668 -95.9727769203754) bank64852 +64853 POINT(30.518066804484665 -95.79882270699244) bank64853 +64854 POINT(29.00264823557724 -95.5771740015903) bank64854 +64855 POINT(29.506268413758633 -94.60866421628124) bank64855 +64856 POINT(29.57124089148354 -94.81451331008432) bank64856 +64857 POINT(30.011998396278745 -95.82384824644654) bank64857 +64858 POINT(30.383768632690966 -95.46485281947199) bank64858 +64859 POINT(30.126591529509767 -94.63694283841635) bank64859 +64860 POINT(29.83449528366271 -94.3839158571478) bank64860 +64861 POINT(30.631255204208202 -96.36791503638338) bank64861 +64862 POINT(29.802042239450056 -95.01303089206647) bank64862 +64863 POINT(30.12140358961008 -94.93347661851392) bank64863 +64864 POINT(28.927011663475287 -96.23344689093761) bank64864 +64865 POINT(29.880704827760447 -95.01910893630927) bank64865 +64866 POINT(28.969258948837584 -95.13537566086954) bank64866 +64867 POINT(28.894769974421138 -95.93268976721691) bank64867 +64868 POINT(30.200305361389038 -95.48472986646286) bank64868 +64869 POINT(29.71549184377351 -94.57156665339123) bank64869 +64870 POINT(28.879490839624918 -96.36020838041912) bank64870 +64871 POINT(29.346169988932353 -95.10854274743855) bank64871 +64872 POINT(30.083256269823885 -96.10869802813514) bank64872 +64873 POINT(30.24297529716239 -95.20441824946163) bank64873 +64874 POINT(29.714870655674233 -95.21731478324361) bank64874 +64875 POINT(30.62333424848253 -95.26965484843686) bank64875 +64876 POINT(30.05662731037831 -96.10451846754682) bank64876 +64877 POINT(29.372505604669815 -94.93584266048235) bank64877 +64878 POINT(30.362043824987826 -96.26594042829842) bank64878 +64879 POINT(30.661619950961754 -94.41513192208966) bank64879 +64880 POINT(30.74608750701781 -96.06749926369434) bank64880 +64881 POINT(28.78636924515033 -95.40484524020611) bank64881 +64882 POINT(30.45279023346601 -94.96919547291566) bank64882 +64883 POINT(30.641626556762617 -96.0036265729564) bank64883 +64884 POINT(29.615792128046042 -95.04446653159494) bank64884 +64885 POINT(29.327815926333482 -95.55153721875759) bank64885 +64886 POINT(29.744917530176217 -95.25848568388231) bank64886 +64887 POINT(30.01016704482811 -95.29503117524223) bank64887 +64888 POINT(29.523241492734186 -94.57994829487252) bank64888 +64889 POINT(29.895212141589134 -95.21647219579577) bank64889 +64890 POINT(30.357966921020456 -94.79883383239812) bank64890 +64891 POINT(30.197927495701595 -94.76334772759027) bank64891 +64892 POINT(30.01105157960909 -95.31687224118754) bank64892 +64893 POINT(30.63260507352957 -94.69374414575105) bank64893 +64894 POINT(30.594801018900366 -96.25802157014195) bank64894 +64895 POINT(29.00571482808136 -95.1613474883488) bank64895 +64896 POINT(29.312693626605512 -95.89682029846438) bank64896 +64897 POINT(30.59916770920129 -94.88341356534478) bank64897 +64898 POINT(29.59630862726635 -94.5745146976971) bank64898 +64899 POINT(29.27002595978991 -96.29549670420036) bank64899 +64900 POINT(30.56378535194679 -94.87898514155965) bank64900 +64901 POINT(30.72706411613356 -94.84495286421263) bank64901 +64902 POINT(29.877189114810346 -95.26101571820843) bank64902 +64903 POINT(29.031016176991905 -95.85525738972886) bank64903 +64904 POINT(30.146463729726975 -95.16869382939446) bank64904 +64905 POINT(29.772461135974634 -95.89221660955167) bank64905 +64906 POINT(30.302866105093994 -95.75307666473205) bank64906 +64907 POINT(29.633704747684458 -96.14811837524682) bank64907 +64908 POINT(28.973259085126113 -94.7011602810556) bank64908 +64909 POINT(30.21966538903053 -95.76165914430983) bank64909 +64910 POINT(30.49199529606753 -96.22752609428848) bank64910 +64911 POINT(30.609545947261456 -95.5876347662225) bank64911 +64912 POINT(29.073178894034008 -94.53823806785037) bank64912 +64913 POINT(28.80668630936204 -95.28876817280297) bank64913 +64914 POINT(30.4666033529287 -95.78887612775246) bank64914 +64915 POINT(29.801173397426133 -95.58915930359237) bank64915 +64916 POINT(29.92452167164547 -94.50840539040983) bank64916 +64917 POINT(30.127253809295897 -95.77685224861493) bank64917 +64918 POINT(30.346980982033585 -94.86437078275648) bank64918 +64919 POINT(29.488076349562007 -95.40919415237752) bank64919 +64920 POINT(28.7841620879931 -95.22847591385371) bank64920 +64921 POINT(30.580308383497613 -94.75143771167018) bank64921 +64922 POINT(29.620609907396638 -94.51612439775333) bank64922 +64923 POINT(30.518394848305864 -95.71485929305824) bank64923 +64924 POINT(30.52428554574408 -95.95966785695649) bank64924 +64925 POINT(29.173147180267083 -95.24554756995317) bank64925 +64926 POINT(30.189860151090297 -95.69997174526866) bank64926 +64927 POINT(29.636967018865935 -95.80343577489978) bank64927 +64928 POINT(29.64723718419509 -95.52663133118648) bank64928 +64929 POINT(30.52942845044198 -95.8702835101527) bank64929 +64930 POINT(29.137589059112962 -96.0556699462927) bank64930 +64931 POINT(28.900810961288016 -95.16260960222829) bank64931 +64932 POINT(30.605494439231812 -94.6134959202689) bank64932 +64933 POINT(29.562558814342772 -95.18352638073632) bank64933 +64934 POINT(29.083052293756616 -96.0320839937423) bank64934 +64935 POINT(29.513243251341393 -95.86790361336337) bank64935 +64936 POINT(30.012233565651506 -94.56704642456447) bank64936 +64937 POINT(29.650378816333667 -95.78211539560388) bank64937 +64938 POINT(29.06879268600123 -94.59159172165961) bank64938 +64939 POINT(29.34545931171177 -95.16912589916788) bank64939 +64940 POINT(29.88378936857267 -96.3228221218008) bank64940 +64941 POINT(29.931398761772513 -95.21457680390955) bank64941 +64942 POINT(29.706036035280434 -96.28182095406171) bank64942 +64943 POINT(29.44404170739943 -96.22731850362453) bank64943 +64944 POINT(30.456426290668904 -95.51431392717129) bank64944 +64945 POINT(30.538258268343935 -95.69312191796357) bank64945 +64946 POINT(29.254573937839883 -94.82702874425891) bank64946 +64947 POINT(30.01364239092553 -94.57509626010369) bank64947 +64948 POINT(29.244669993208745 -96.13422299159265) bank64948 +64949 POINT(28.774850602570538 -95.2654128364777) bank64949 +64950 POINT(29.310174990504926 -96.1944752332432) bank64950 +64951 POINT(29.578456512070314 -94.39311677605902) bank64951 +64952 POINT(29.28145060060153 -95.24441626551108) bank64952 +64953 POINT(28.76157078940603 -96.36729244003745) bank64953 +64954 POINT(29.544111739577506 -94.9900478349504) bank64954 +64955 POINT(30.117609468006826 -96.30861744397991) bank64955 +64956 POINT(30.074733464511684 -94.69324392771082) bank64956 +64957 POINT(30.35663946463826 -95.51849434713195) bank64957 +64958 POINT(29.18081363713028 -94.83028091572282) bank64958 +64959 POINT(29.066982453614443 -95.55945326997627) bank64959 +64960 POINT(30.117509587417366 -95.39301581917965) bank64960 +64961 POINT(29.687570556181626 -95.3262764232713) bank64961 +64962 POINT(30.546436610243756 -94.92553591538848) bank64962 +64963 POINT(29.679208227473318 -94.86258688968454) bank64963 +64964 POINT(30.73330315349846 -95.90580848643353) bank64964 +64965 POINT(30.689008739437405 -94.55873258888914) bank64965 +64966 POINT(29.450777423973186 -95.73543435173247) bank64966 +64967 POINT(30.50832311693834 -95.15377116384693) bank64967 +64968 POINT(29.751424393600082 -95.57868651680305) bank64968 +64969 POINT(28.966429411933444 -94.53028968146792) bank64969 +64970 POINT(29.122442769928806 -94.54890129736033) bank64970 +64971 POINT(28.98524114702337 -95.75517673943932) bank64971 +64972 POINT(30.03558558766617 -95.03312748804036) bank64972 +64973 POINT(30.551510288595956 -96.3445065142707) bank64973 +64974 POINT(30.597096208990646 -94.71268866638513) bank64974 +64975 POINT(29.69921412792234 -96.10650909903423) bank64975 +64976 POINT(29.745355588871583 -95.67605847485493) bank64976 +64977 POINT(30.202178969064104 -95.81430846869965) bank64977 +64978 POINT(29.283588439638965 -94.70353823358163) bank64978 +64979 POINT(29.550256910281288 -95.56158642559664) bank64979 +64980 POINT(29.09079733600569 -95.82369242849403) bank64980 +64981 POINT(28.761958063003934 -95.62258933394976) bank64981 +64982 POINT(30.394206894360686 -94.59282844819971) bank64982 +64983 POINT(30.27809998132126 -96.30934911271503) bank64983 +64984 POINT(30.351046690517126 -95.34327436143228) bank64984 +64985 POINT(29.291003728622034 -94.92826963891491) bank64985 +64986 POINT(30.325929463306537 -96.19784096533834) bank64986 +64987 POINT(29.572327807053714 -95.21118874598105) bank64987 +64988 POINT(30.009287996859182 -95.16559143259184) bank64988 +64989 POINT(29.249144559838538 -96.13413929241109) bank64989 +64990 POINT(30.626445871231414 -95.4932330785639) bank64990 +64991 POINT(28.889591586210827 -96.03804478850944) bank64991 +64992 POINT(30.383286545161223 -94.61771005367864) bank64992 +64993 POINT(30.37531435744418 -94.55568665846751) bank64993 +64994 POINT(30.38183795494816 -95.2284885179196) bank64994 +64995 POINT(29.360027807848596 -94.47255562054319) bank64995 +64996 POINT(30.29690016926564 -95.68079806047334) bank64996 +64997 POINT(28.95424899099515 -95.11837447762647) bank64997 +64998 POINT(29.907504148184422 -96.18502686667836) bank64998 +64999 POINT(29.951697631365512 -95.80395412522952) bank64999 +65000 POINT(28.82012938187775 -95.9247257487522) bank65000 +65001 POINT(29.07484373785881 -96.08408342882646) bank65001 +65002 POINT(29.842519981019976 -95.83171221787953) bank65002 +65003 POINT(30.344480058597846 -94.51200699780179) bank65003 +65004 POINT(30.401681252895273 -95.96343547423416) bank65004 +65005 POINT(30.00193903131772 -95.70690720609409) bank65005 +65006 POINT(30.140369056505776 -96.31219621330949) bank65006 +65007 POINT(29.040822006162603 -95.74506614663092) bank65007 +65008 POINT(29.18628983158994 -95.25099102810745) bank65008 +65009 POINT(30.560104968801134 -95.5819669126835) bank65009 +65010 POINT(29.676986801466796 -95.77924482861864) bank65010 +65011 POINT(29.590033776069312 -94.98240586157176) bank65011 +65012 POINT(29.921302107837164 -95.2180424897575) bank65012 +65013 POINT(29.13891819991364 -94.84848503490802) bank65013 +65014 POINT(30.679727857742325 -95.21996234192505) bank65014 +65015 POINT(30.669265036815958 -94.87243697103216) bank65015 +65016 POINT(30.61143968251132 -95.02956483477503) bank65016 +65017 POINT(29.929299459104854 -94.72046898457337) bank65017 +65018 POINT(29.849754223610397 -94.87513250091521) bank65018 +65019 POINT(30.257944562742907 -95.14560893436098) bank65019 +65020 POINT(28.77609480909768 -94.79123933359799) bank65020 +65021 POINT(28.998429404755154 -95.16449932183903) bank65021 +65022 POINT(29.819978689450032 -95.23876562926394) bank65022 +65023 POINT(29.502774611726924 -95.60861936035263) bank65023 +65024 POINT(28.847929682954934 -96.16991441527941) bank65024 +65025 POINT(30.723961545963604 -95.5770978433472) bank65025 +65026 POINT(30.33321589542478 -94.9062872752632) bank65026 +65027 POINT(28.808887046963 -94.67765025845348) bank65027 +65028 POINT(29.491438058829857 -95.76168355102836) bank65028 +65029 POINT(30.660996270569864 -94.64572402635669) bank65029 +65030 POINT(29.487228154066734 -96.08068929663948) bank65030 +65031 POINT(29.963185676582288 -96.07891407209978) bank65031 +65032 POINT(30.669233142652665 -95.89297033128148) bank65032 +65033 POINT(29.20279610173734 -95.89155255947914) bank65033 +65034 POINT(29.516779929210852 -96.25869492438409) bank65034 +65035 POINT(29.99941076283358 -95.60473105362487) bank65035 +65036 POINT(30.511694365428472 -96.2108239108021) bank65036 +65037 POINT(29.349700017059877 -95.745929710861) bank65037 +65038 POINT(28.962546344461938 -95.6307108021528) bank65038 +65039 POINT(30.351245399786617 -94.62878481083492) bank65039 +65040 POINT(30.692703120117866 -95.1374822703121) bank65040 +65041 POINT(30.042841088258612 -96.14298417971892) bank65041 +65042 POINT(29.76734474230402 -94.42759138139013) bank65042 +65043 POINT(30.502355725027517 -94.84078557195313) bank65043 +65044 POINT(29.361899887269526 -95.19304227331247) bank65044 +65045 POINT(28.79488967732709 -94.45034372758468) bank65045 +65046 POINT(29.637290674794347 -94.41279825352358) bank65046 +65047 POINT(28.869957715214326 -95.90664997406134) bank65047 +65048 POINT(29.348867120149162 -94.68023498389336) bank65048 +65049 POINT(29.236661412877854 -94.43040955867994) bank65049 +65050 POINT(29.805109964878056 -96.33277569653562) bank65050 +65051 POINT(30.63317529067183 -95.94585760600454) bank65051 +65052 POINT(28.85185140476123 -94.41757973960318) bank65052 +65053 POINT(29.35400025704476 -94.46785914889485) bank65053 +65054 POINT(29.75706150909847 -94.855722136357) bank65054 +65055 POINT(30.02348128344622 -94.54077543617856) bank65055 +65056 POINT(30.533215165557177 -95.01134401286848) bank65056 +65057 POINT(29.064621550128415 -95.94335108702751) bank65057 +65058 POINT(29.858523767324126 -96.20943593707501) bank65058 +65059 POINT(30.741312261228593 -95.0137874123285) bank65059 +65060 POINT(30.55423916642608 -94.49972105103257) bank65060 +65061 POINT(29.9562959382449 -95.40138542472852) bank65061 +65062 POINT(30.608953962509048 -95.69658955527305) bank65062 +65063 POINT(29.574610222043592 -94.8167420055729) bank65063 +65064 POINT(30.020127517702242 -94.91337961610213) bank65064 +65065 POINT(29.109215187314557 -95.72506447458467) bank65065 +65066 POINT(29.115292689966033 -94.73122661513334) bank65066 +65067 POINT(30.731764037017875 -95.17134466270929) bank65067 +65068 POINT(30.184253513985023 -94.99338798179822) bank65068 +65069 POINT(30.087042346215352 -96.2146301121172) bank65069 +65070 POINT(30.731448545436244 -95.81650052182049) bank65070 +65071 POINT(30.260605668782638 -95.19546414113357) bank65071 +65072 POINT(30.263519772087427 -95.72637272787053) bank65072 +65073 POINT(29.408231584961943 -95.79206064802875) bank65073 +65074 POINT(29.17878278504093 -95.89442012173213) bank65074 +65075 POINT(29.931352934023415 -95.75015199372093) bank65075 +65076 POINT(29.536931724229706 -95.10457768880302) bank65076 +65077 POINT(30.706515338772434 -95.93037388333062) bank65077 +65078 POINT(29.657220050860246 -94.40528333951376) bank65078 +65079 POINT(30.277022057730285 -95.32172972892702) bank65079 +65080 POINT(30.53032129199611 -94.92432709335777) bank65080 +65081 POINT(29.44934565685115 -94.70697708155247) bank65081 +65082 POINT(29.7598267362441 -94.42261977021874) bank65082 +65083 POINT(29.584901330310682 -96.25374786441665) bank65083 +65084 POINT(29.06709545494254 -95.81724589001182) bank65084 +65085 POINT(29.227604186314736 -96.16887906181324) bank65085 +65086 POINT(29.347454195509453 -96.00731963571845) bank65086 +65087 POINT(29.921329272354566 -96.35058514610638) bank65087 +65088 POINT(29.712004340282665 -94.57340374048682) bank65088 +65089 POINT(30.513191505797955 -94.6297994458516) bank65089 +65090 POINT(29.324817451534308 -94.39464828360218) bank65090 +65091 POINT(29.485576350992424 -95.68623442628562) bank65091 +65092 POINT(29.016300466147918 -95.21869877975354) bank65092 +65093 POINT(29.197817124971877 -96.06200409190247) bank65093 +65094 POINT(29.501848235170804 -96.19527091203626) bank65094 +65095 POINT(30.169012425099467 -96.06109509083558) bank65095 +65096 POINT(28.989486676452213 -96.04491414489898) bank65096 +65097 POINT(30.09869212686522 -96.32024040916893) bank65097 +65098 POINT(28.884246318240212 -96.19882410063961) bank65098 +65099 POINT(29.84163069449075 -96.17288548534223) bank65099 +65100 POINT(29.254242694109376 -95.81027397781546) bank65100 +65101 POINT(30.29807226330448 -95.87107464016856) bank65101 +65102 POINT(29.152473831986313 -94.82164299998307) bank65102 +65103 POINT(29.867959762653644 -94.8611746460051) bank65103 +65104 POINT(30.094344089793903 -94.91744396468617) bank65104 +65105 POINT(30.412803003705246 -95.04100371982966) bank65105 +65106 POINT(28.855496133898356 -94.87904747661695) bank65106 +65107 POINT(30.218314364753972 -95.81446253991973) bank65107 +65108 POINT(30.436832526337735 -95.75007108900542) bank65108 +65109 POINT(30.04011894649915 -94.68671346457096) bank65109 +65110 POINT(30.284005311904956 -94.58444534394557) bank65110 +65111 POINT(29.497995505278123 -95.37944482874391) bank65111 +65112 POINT(29.44828509945757 -94.62217275846588) bank65112 +65113 POINT(29.55324773023306 -95.05392492635015) bank65113 +65114 POINT(29.019371622892834 -96.00454524546745) bank65114 +65115 POINT(29.310072836203755 -95.44299312050842) bank65115 +65116 POINT(30.731973164393125 -96.2713439845191) bank65116 +65117 POINT(29.526261686668764 -94.76966172005436) bank65117 +65118 POINT(30.59135675180047 -95.72249826236504) bank65118 +65119 POINT(29.720435432502004 -96.01829385795834) bank65119 +65120 POINT(29.817612620504374 -95.8479801784859) bank65120 +65121 POINT(29.047693985608987 -96.14881248345903) bank65121 +65122 POINT(30.677202897221687 -94.99609187728342) bank65122 +65123 POINT(29.134983389187127 -94.44265767251267) bank65123 +65124 POINT(29.237519155601287 -95.31169857600484) bank65124 +65125 POINT(30.57930977366519 -94.60242679730072) bank65125 +65126 POINT(30.434918372894057 -95.92346513144413) bank65126 +65127 POINT(29.28131265827321 -96.36361309999828) bank65127 +65128 POINT(30.512242045926474 -95.36906254111962) bank65128 +65129 POINT(30.378751440692515 -95.69393895874435) bank65129 +65130 POINT(29.204122249439024 -95.27066407953647) bank65130 +65131 POINT(29.647034832028023 -94.85644452632765) bank65131 +65132 POINT(29.721022623542197 -95.56981456004145) bank65132 +65133 POINT(29.11915912757079 -95.55618187840122) bank65133 +65134 POINT(28.96777624937503 -95.75814682593139) bank65134 +65135 POINT(28.986545197171548 -94.88412685183957) bank65135 +65136 POINT(29.434547870634233 -95.53904816601768) bank65136 +65137 POINT(30.327600898720288 -95.60365100833783) bank65137 +65138 POINT(28.83362628113978 -94.81365024622414) bank65138 +65139 POINT(30.473452292326467 -95.83043373275433) bank65139 +65140 POINT(29.85693155354681 -95.5594315956054) bank65140 +65141 POINT(30.409676494377383 -95.44522617542803) bank65141 +65142 POINT(29.160745043309486 -95.73478415485067) bank65142 +65143 POINT(29.530678103806657 -95.67870680087974) bank65143 +65144 POINT(30.64592946009799 -96.15555705809385) bank65144 +65145 POINT(29.688716105422607 -94.61342488561023) bank65145 +65146 POINT(29.00004843549452 -94.60967293098618) bank65146 +65147 POINT(30.588478683858472 -94.6818685947756) bank65147 +65148 POINT(29.55178677296518 -95.38048539441017) bank65148 +65149 POINT(29.220916010243116 -95.00568424249592) bank65149 +65150 POINT(29.98507267917323 -95.27094294900394) bank65150 +65151 POINT(29.985009733756044 -96.14844734775347) bank65151 +65152 POINT(29.933778812605613 -95.1126470196178) bank65152 +65153 POINT(30.69253846519681 -95.75714967041304) bank65153 +65154 POINT(30.647867409296616 -94.97154930296304) bank65154 +65155 POINT(30.752882711307315 -95.11275094525934) bank65155 +65156 POINT(30.102107740767067 -94.74764960574382) bank65156 +65157 POINT(30.662525085590058 -94.95925990779583) bank65157 +65158 POINT(30.409797741780135 -94.37471357946457) bank65158 +65159 POINT(30.435392212406978 -95.50059726524228) bank65159 +65160 POINT(29.93126813228411 -95.14368547123486) bank65160 +65161 POINT(28.992053543890133 -96.10920085651706) bank65161 +65162 POINT(29.32692032303711 -95.69060768879207) bank65162 +65163 POINT(30.596921909197324 -95.2121320384933) bank65163 +65164 POINT(29.73166190379068 -95.45383303230211) bank65164 +65165 POINT(30.261418760361124 -95.75400590347625) bank65165 +65166 POINT(28.78226821771748 -96.3498033856397) bank65166 +65167 POINT(29.13215910142583 -94.43934794842092) bank65167 +65168 POINT(29.477084964712922 -94.95305215224732) bank65168 +65169 POINT(29.075485323655286 -95.75501373415901) bank65169 +65170 POINT(30.106679152198858 -95.36356655738102) bank65170 +65171 POINT(29.71293541960901 -94.48810489222286) bank65171 +65172 POINT(29.687961239058513 -94.89157820541888) bank65172 +65173 POINT(30.721884722262676 -96.28635492488195) bank65173 +65174 POINT(29.929148772508512 -94.95393556530868) bank65174 +65175 POINT(29.404451965117048 -94.37987691487417) bank65175 +65176 POINT(29.604122650221306 -94.62686006655545) bank65176 +65177 POINT(29.530540097139635 -94.77772942117657) bank65177 +65178 POINT(30.60627903293913 -94.62542708988671) bank65178 +65179 POINT(30.202152935536052 -95.85120329490303) bank65179 +65180 POINT(29.098032936900506 -94.46665762787086) bank65180 +65181 POINT(30.255190578865694 -95.48554088614995) bank65181 +65182 POINT(29.82969270249738 -95.73632644584154) bank65182 +65183 POINT(29.612777877687904 -94.49884749796509) bank65183 +65184 POINT(29.583571822219046 -94.50120144114516) bank65184 +65185 POINT(29.40055504042721 -96.2632519061872) bank65185 +65186 POINT(29.10628261904447 -96.11796020648129) bank65186 +65187 POINT(30.634264228110258 -94.5445429167818) bank65187 +65188 POINT(29.683040889626565 -95.79782437143153) bank65188 +65189 POINT(29.76707375954433 -96.02711726392083) bank65189 +65190 POINT(29.88617134447059 -94.53271629908883) bank65190 +65191 POINT(30.737370346501525 -95.98141891351571) bank65191 +65192 POINT(29.082080867405644 -95.46627494800772) bank65192 +65193 POINT(30.54328334742989 -94.38359112842464) bank65193 +65194 POINT(29.344174642295606 -94.67361961003218) bank65194 +65195 POINT(29.74349716181986 -95.98752334745079) bank65195 +65196 POINT(30.272790604533704 -94.7623429356908) bank65196 +65197 POINT(29.1398751049982 -95.47177471971597) bank65197 +65198 POINT(28.918320700084195 -95.09095668067785) bank65198 +65199 POINT(29.868483117367724 -94.80088883558857) bank65199 +65200 POINT(29.143453647221566 -95.30721180768651) bank65200 +65201 POINT(29.194936956808505 -94.90084926878973) bank65201 +65202 POINT(29.020141585700056 -94.94274373739445) bank65202 +65203 POINT(28.944334594658606 -94.58396672466459) bank65203 +65204 POINT(29.20737381621112 -96.19832569509299) bank65204 +65205 POINT(29.98637866981624 -95.95417576765078) bank65205 +65206 POINT(29.51099575139478 -96.31432417423201) bank65206 +65207 POINT(29.827489064654962 -96.35049836646658) bank65207 +65208 POINT(29.707178102553573 -95.41637845942309) bank65208 +65209 POINT(30.243132005974676 -94.9784245868079) bank65209 +65210 POINT(29.29546303801202 -95.43107906834614) bank65210 +65211 POINT(30.37699915027992 -95.56608910069123) bank65211 +65212 POINT(30.186097319326073 -95.57467711218528) bank65212 +65213 POINT(30.438546692197328 -95.23770572179451) bank65213 +65214 POINT(29.6769919028826 -95.73689807471388) bank65214 +65215 POINT(29.50149772042715 -95.05389015284958) bank65215 +65216 POINT(29.60206978559883 -95.8458012087332) bank65216 +65217 POINT(30.451659337889687 -95.20637030705576) bank65217 +65218 POINT(29.448246481883988 -95.56617169268014) bank65218 +65219 POINT(29.383959836962347 -94.60820308790804) bank65219 +65220 POINT(29.821032133169748 -95.50830230248512) bank65220 +65221 POINT(30.245006388482558 -95.70898729970713) bank65221 +65222 POINT(30.470961662090122 -94.95300658586625) bank65222 +65223 POINT(29.621071516103278 -94.450141263468) bank65223 +65224 POINT(29.538324346484778 -96.10811261217155) bank65224 +65225 POINT(29.30494665859363 -94.52890210173551) bank65225 +65226 POINT(30.32468868287539 -95.51899157128794) bank65226 +65227 POINT(30.444034485246064 -94.66032015239868) bank65227 +65228 POINT(29.319348553019672 -94.60110688897954) bank65228 +65229 POINT(30.009279242047718 -95.59628677869188) bank65229 +65230 POINT(28.98392704365612 -95.42465448755813) bank65230 +65231 POINT(30.066776420834756 -96.33157425535504) bank65231 +65232 POINT(29.16553475481696 -95.45654610144021) bank65232 +65233 POINT(30.39729021287574 -95.40999825909222) bank65233 +65234 POINT(29.207924359456875 -94.99111487762057) bank65234 +65235 POINT(30.114225777567746 -95.20453160159192) bank65235 +65236 POINT(29.798423948292346 -94.38814613757624) bank65236 +65237 POINT(29.444079149083358 -94.95125504063557) bank65237 +65238 POINT(29.158001217905607 -95.5691595125489) bank65238 +65239 POINT(30.75890270723986 -94.62312714225317) bank65239 +65240 POINT(30.36356683684789 -95.95984263513228) bank65240 +65241 POINT(29.15188431800598 -94.83254945727424) bank65241 +65242 POINT(30.37261949160508 -95.44445409438367) bank65242 +65243 POINT(29.538181828172046 -94.47996580449303) bank65243 +65244 POINT(29.788674946354604 -96.10110617862135) bank65244 +65245 POINT(29.694831331289922 -94.73425819063394) bank65245 +65246 POINT(30.115068068235953 -95.26641482270334) bank65246 +65247 POINT(30.271067284660965 -95.4161612377184) bank65247 +65248 POINT(30.36157988223009 -94.91913787371146) bank65248 +65249 POINT(29.025939579362362 -95.8493941513062) bank65249 +65250 POINT(29.196810986499187 -95.37985748460888) bank65250 +65251 POINT(30.228538565178628 -94.86763002878801) bank65251 +65252 POINT(29.09705537117267 -96.23840840914359) bank65252 +65253 POINT(30.464991684583634 -95.68024326849131) bank65253 +65254 POINT(29.163327316915108 -94.88109549700717) bank65254 +65255 POINT(29.741166547306257 -95.1251741671993) bank65255 +65256 POINT(29.091929440228007 -95.48968763563562) bank65256 +65257 POINT(30.57561641810701 -95.32764618221907) bank65257 +65258 POINT(30.33970809094974 -95.92067867418392) bank65258 +65259 POINT(29.9304620061967 -95.88370099414189) bank65259 +65260 POINT(30.097048142855645 -95.42776222401022) bank65260 +65261 POINT(29.95870631989401 -95.74345181775257) bank65261 +65262 POINT(30.47637715550024 -95.31735205879843) bank65262 +65263 POINT(28.80816284798389 -95.80486267513245) bank65263 +65264 POINT(29.67811816464134 -94.44254309048733) bank65264 +65265 POINT(29.227724352872727 -94.52417516041828) bank65265 +65266 POINT(29.148841635623317 -95.11686722449787) bank65266 +65267 POINT(30.704935457738046 -96.3462043700059) bank65267 +65268 POINT(30.642812784316767 -94.80616730171043) bank65268 +65269 POINT(30.1251019391945 -95.14387020376194) bank65269 +65270 POINT(30.042462727358718 -95.57834288729063) bank65270 +65271 POINT(29.837862444619212 -95.96210406318428) bank65271 +65272 POINT(30.586081263699157 -95.80044358519079) bank65272 +65273 POINT(29.864096880051427 -95.27423570182441) bank65273 +65274 POINT(29.414606835021758 -95.21822534899229) bank65274 +65275 POINT(28.919305694968195 -96.16780616730338) bank65275 +65276 POINT(30.119819187977217 -96.05698129460065) bank65276 +65277 POINT(30.69909576150794 -95.10293958767977) bank65277 +65278 POINT(29.257646798989626 -96.25489493807817) bank65278 +65279 POINT(29.930645923359105 -95.41245761587847) bank65279 +65280 POINT(28.842643505810777 -95.54490952680351) bank65280 +65281 POINT(29.307547870426514 -96.00423650715058) bank65281 +65282 POINT(30.537850241528552 -96.22486379714384) bank65282 +65283 POINT(29.816515685201477 -95.24945538749829) bank65283 +65284 POINT(29.276809305144877 -95.4287247176775) bank65284 +65285 POINT(29.047256501275232 -95.15694522643699) bank65285 +65286 POINT(29.126485069157916 -95.09848504621242) bank65286 +65287 POINT(29.046041367336485 -94.73381322134965) bank65287 +65288 POINT(29.192615466709253 -95.10474965552747) bank65288 +65289 POINT(29.230804343563882 -94.7879340874125) bank65289 +65290 POINT(29.501605932807795 -95.00696775916992) bank65290 +65291 POINT(29.062192720361985 -95.82564483097867) bank65291 +65292 POINT(29.159557448870988 -95.07006851167918) bank65292 +65293 POINT(30.49268430606265 -94.43319144717873) bank65293 +65294 POINT(30.5554766910948 -94.60998422814308) bank65294 +65295 POINT(30.244346389549328 -95.31108052728943) bank65295 +65296 POINT(30.05595702548971 -95.20846939203577) bank65296 +65297 POINT(29.216941517906697 -95.0832771196597) bank65297 +65298 POINT(30.64097803542312 -95.46625725152596) bank65298 +65299 POINT(29.357225614456084 -94.5742608814314) bank65299 +65300 POINT(28.952252954737208 -95.61520768012844) bank65300 +65301 POINT(29.313797735812972 -96.20537416477505) bank65301 +65302 POINT(29.691864208222533 -94.9706078343681) bank65302 +65303 POINT(30.654910872682613 -96.3012469929735) bank65303 +65304 POINT(28.81935684406886 -95.62944189970358) bank65304 +65305 POINT(29.476092478675348 -94.92695995593006) bank65305 +65306 POINT(29.34499128237453 -95.17048347785493) bank65306 +65307 POINT(29.670205647630823 -95.13649132676416) bank65307 +65308 POINT(30.530415904464245 -94.46328265979477) bank65308 +65309 POINT(30.410713495559623 -95.91731065515363) bank65309 +65310 POINT(30.494981750353915 -95.486471890467) bank65310 +65311 POINT(30.35745314163577 -94.68107610481476) bank65311 +65312 POINT(29.239740516650933 -94.90574145480578) bank65312 +65313 POINT(29.235090764850444 -96.15820565440538) bank65313 +65314 POINT(29.96356116466122 -96.0974003270856) bank65314 +65315 POINT(30.27020444972335 -95.08484436697722) bank65315 +65316 POINT(28.909799438713893 -94.58538475518519) bank65316 +65317 POINT(29.441688359595286 -95.63016054031604) bank65317 +65318 POINT(29.37676962318636 -95.74340367904571) bank65318 +65319 POINT(29.676413109387187 -95.893628449924) bank65319 +65320 POINT(30.484306378756276 -96.05831211999916) bank65320 +65321 POINT(29.3332760667547 -95.3839624104559) bank65321 +65322 POINT(29.667472055236836 -94.57843379742484) bank65322 +65323 POINT(29.67200547523158 -95.45000038142898) bank65323 +65324 POINT(29.44455046821213 -95.40516865011874) bank65324 +65325 POINT(29.772113607741073 -95.20831375570806) bank65325 +65326 POINT(29.108573876608318 -95.44619989314653) bank65326 +65327 POINT(29.152587543786552 -96.22095804683359) bank65327 +65328 POINT(29.819669658894966 -95.69563424318109) bank65328 +65329 POINT(30.28655821817347 -94.40605372114996) bank65329 +65330 POINT(29.483772113193123 -95.00627158397556) bank65330 +65331 POINT(29.54082427344434 -95.46689006136327) bank65331 +65332 POINT(29.667346588945463 -95.3305357713812) bank65332 +65333 POINT(29.59088190796544 -94.73403386744965) bank65333 +65334 POINT(30.251934772979517 -94.77485614841832) bank65334 +65335 POINT(30.04296892793512 -95.9416909152862) bank65335 +65336 POINT(30.408357536850843 -94.84021675628861) bank65336 +65337 POINT(29.098901287158824 -96.1643836119716) bank65337 +65338 POINT(29.795940735826406 -96.14398203204497) bank65338 +65339 POINT(29.229217097098065 -94.99875526988372) bank65339 +65340 POINT(29.330009297115936 -95.03765618285061) bank65340 +65341 POINT(30.255410998240976 -95.2377202471532) bank65341 +65342 POINT(28.875049589374647 -94.72983380487453) bank65342 +65343 POINT(30.19682177164847 -95.81844228698114) bank65343 +65344 POINT(30.54028571006981 -95.68412350982555) bank65344 +65345 POINT(29.28472344651013 -95.43037819739915) bank65345 +65346 POINT(30.632252840612278 -96.20268518362373) bank65346 +65347 POINT(30.025990642321897 -94.84514120103472) bank65347 +65348 POINT(30.603341585892103 -95.80015694029112) bank65348 +65349 POINT(30.518369832563366 -94.6835264989534) bank65349 +65350 POINT(29.03454502280427 -94.51374352193389) bank65350 +65351 POINT(29.360046383201254 -96.08983414647165) bank65351 +65352 POINT(30.160683124253673 -95.11465302036622) bank65352 +65353 POINT(29.528293011810753 -94.76966745668894) bank65353 +65354 POINT(30.387448451526684 -95.35556769876982) bank65354 +65355 POINT(30.15595729394345 -95.70074805283258) bank65355 +65356 POINT(28.785691763828158 -94.85590025490997) bank65356 +65357 POINT(29.72043964782154 -95.71425654043131) bank65357 +65358 POINT(30.44752285031414 -95.46100020746066) bank65358 +65359 POINT(29.147389698604005 -95.24017057053369) bank65359 +65360 POINT(30.3830147230008 -96.12495350971271) bank65360 +65361 POINT(29.16444409878757 -95.74606297626401) bank65361 +65362 POINT(29.995906892965408 -96.29961902034138) bank65362 +65363 POINT(29.200426845777613 -95.35350120243737) bank65363 +65364 POINT(29.778767398013965 -94.931960353448) bank65364 +65365 POINT(29.66934691903363 -95.65065562763463) bank65365 +65366 POINT(29.042152850656965 -95.2455559739415) bank65366 +65367 POINT(29.078420966404014 -95.41110852293441) bank65367 +65368 POINT(30.002540115668065 -94.77252252862303) bank65368 +65369 POINT(30.292879596447527 -94.78062023224754) bank65369 +65370 POINT(29.169770036430396 -94.4313038084742) bank65370 +65371 POINT(29.45084818560216 -94.41969163234221) bank65371 +65372 POINT(29.83870999484878 -96.25858154531728) bank65372 +65373 POINT(29.607322974567452 -95.83687312786955) bank65373 +65374 POINT(29.437318182668335 -95.08441387816286) bank65374 +65375 POINT(29.290950856957597 -95.37501896264344) bank65375 +65376 POINT(29.549669855350896 -95.33123898893481) bank65376 +65377 POINT(30.185185416142296 -95.8548521797122) bank65377 +65378 POINT(29.44798649557413 -95.77768422042537) bank65378 +65379 POINT(29.901265267121946 -95.52738383015891) bank65379 +65380 POINT(30.269939790306385 -95.31387589578264) bank65380 +65381 POINT(30.349784931998833 -94.72980910420335) bank65381 +65382 POINT(29.120022477762884 -95.61503816724749) bank65382 +65383 POINT(28.891885971211416 -94.44934545924679) bank65383 +65384 POINT(29.444200953476873 -96.32123272053522) bank65384 +65385 POINT(30.64160592356348 -95.0765561598378) bank65385 +65386 POINT(29.960115897922442 -94.6284723431586) bank65386 +65387 POINT(29.461440640702108 -95.45632247079189) bank65387 +65388 POINT(28.877771404830497 -95.58168766978302) bank65388 +65389 POINT(30.520915797871435 -95.4611570521264) bank65389 +65390 POINT(29.88306009140395 -95.47851125193272) bank65390 +65391 POINT(30.576782920715814 -94.54559382663712) bank65391 +65392 POINT(29.709075228777536 -95.54765650267967) bank65392 +65393 POINT(30.311726459813457 -95.32672869522631) bank65393 +65394 POINT(29.93771307499578 -94.76882147247628) bank65394 +65395 POINT(29.490896717327267 -96.29520813629233) bank65395 +65396 POINT(29.6031152274328 -94.4964080738703) bank65396 +65397 POINT(30.569695799559916 -94.69992515064595) bank65397 +65398 POINT(30.18111367432126 -94.45284454651437) bank65398 +65399 POINT(28.870509436477455 -94.95391199512927) bank65399 +65400 POINT(29.122606119211085 -94.86396827403289) bank65400 +65401 POINT(30.29981397233509 -95.90378174953622) bank65401 +65402 POINT(30.633768495784178 -94.50874754141904) bank65402 +65403 POINT(28.872550543777308 -95.42324327792299) bank65403 +65404 POINT(29.774487143487544 -94.37127740172846) bank65404 +65405 POINT(30.56461263566482 -96.14680983847931) bank65405 +65406 POINT(30.5217208884934 -94.41922344870707) bank65406 +65407 POINT(28.958995841339807 -95.97805748360379) bank65407 +65408 POINT(30.195673173417156 -94.71944867820368) bank65408 +65409 POINT(30.207282089776708 -95.58668964170273) bank65409 +65410 POINT(29.42740560729816 -95.94126870975663) bank65410 +65411 POINT(28.860517427265822 -95.97899405801324) bank65411 +65412 POINT(28.957359281682233 -95.00550472276699) bank65412 +65413 POINT(29.624500782347404 -95.12124852183143) bank65413 +65414 POINT(29.939640849804874 -94.54028825116806) bank65414 +65415 POINT(29.749477877382407 -96.04898063981092) bank65415 +65416 POINT(29.246298820191917 -96.32423535471614) bank65416 +65417 POINT(28.856033265860326 -94.97980661105584) bank65417 +65418 POINT(28.897461836828263 -96.13300806274239) bank65418 +65419 POINT(29.606471990835526 -94.66300333053235) bank65419 +65420 POINT(29.11993055701051 -95.69962083968899) bank65420 +65421 POINT(30.085146838452342 -94.39922749379767) bank65421 +65422 POINT(30.284522880277542 -95.07017593270795) bank65422 +65423 POINT(29.209268675830774 -96.14296777084425) bank65423 +65424 POINT(29.84958176478497 -96.13547836876896) bank65424 +65425 POINT(29.595315405576446 -96.24036885699147) bank65425 +65426 POINT(29.58829181253537 -94.71684127958386) bank65426 +65427 POINT(28.954055393433812 -95.78918211419477) bank65427 +65428 POINT(28.847447755290574 -94.46261187232216) bank65428 +65429 POINT(29.431002029855716 -94.66595685796727) bank65429 +65430 POINT(29.525305704744788 -95.53503858689719) bank65430 +65431 POINT(29.63586045649636 -95.26625718964064) bank65431 +65432 POINT(30.45541785741149 -94.67475429437734) bank65432 +65433 POINT(29.475792604225372 -95.74062217352368) bank65433 +65434 POINT(29.323789259651946 -95.48221894698176) bank65434 +65435 POINT(29.483974993511012 -96.20438293904628) bank65435 +65436 POINT(28.967322569233332 -95.5743307632263) bank65436 +65437 POINT(29.680404754903293 -96.16253855523179) bank65437 +65438 POINT(28.799147403448558 -95.77276345584944) bank65438 +65439 POINT(28.942401644520896 -96.22829361799133) bank65439 +65440 POINT(30.42635260567632 -94.87229472306805) bank65440 +65441 POINT(29.84119868368499 -95.58645518166307) bank65441 +65442 POINT(29.866704658101405 -96.20372758818823) bank65442 +65443 POINT(28.850099471814193 -96.30231551339823) bank65443 +65444 POINT(30.338286732807394 -95.97652387791169) bank65444 +65445 POINT(30.15523979239908 -94.9619480845602) bank65445 +65446 POINT(28.97291938783526 -95.51586067529747) bank65446 +65447 POINT(29.880638040804335 -94.57620506601802) bank65447 +65448 POINT(30.048665156453524 -95.91981012508893) bank65448 +65449 POINT(29.7022822586825 -95.72157968951122) bank65449 +65450 POINT(29.228743742158525 -96.14727120416721) bank65450 +65451 POINT(28.899941080946768 -96.01727115861378) bank65451 +65452 POINT(29.80935508158893 -94.88497472286207) bank65452 +65453 POINT(29.931250404066503 -95.18173661685285) bank65453 +65454 POINT(29.472325430467638 -96.2255976811946) bank65454 +65455 POINT(29.254446587280338 -95.58435812726418) bank65455 +65456 POINT(29.69018815714962 -95.70660539280944) bank65456 +65457 POINT(28.876543673201148 -95.73256734186964) bank65457 +65458 POINT(29.331125961545368 -94.72218158097346) bank65458 +65459 POINT(30.53975792297836 -96.16242747875279) bank65459 +65460 POINT(30.158601507065313 -94.42378482701957) bank65460 +65461 POINT(29.789333029914864 -95.70788974544303) bank65461 +65462 POINT(29.273594197544018 -95.9015121342514) bank65462 +65463 POINT(30.737957254928354 -95.49828060032158) bank65463 +65464 POINT(29.22273866175622 -94.4248814300569) bank65464 +65465 POINT(30.373042864244503 -94.93967733333182) bank65465 +65466 POINT(30.744128992381405 -95.99021055574175) bank65466 +65467 POINT(29.877742948285793 -95.83768929614762) bank65467 +65468 POINT(29.32724981145628 -95.8232985708146) bank65468 +65469 POINT(29.140253296815736 -96.19037937395078) bank65469 +65470 POINT(28.805862279475193 -94.95513692357065) bank65470 +65471 POINT(30.16541918632659 -95.08102911203667) bank65471 +65472 POINT(28.77972091032481 -94.98585893712992) bank65472 +65473 POINT(29.581683885176066 -96.0299694696015) bank65473 +65474 POINT(29.518243589963582 -95.08941064627872) bank65474 +65475 POINT(30.359879346636546 -96.00749663772501) bank65475 +65476 POINT(30.045268527400488 -96.20463786558787) bank65476 +65477 POINT(30.53355130668529 -94.37443896748101) bank65477 +65478 POINT(29.790465502825533 -95.99906617815026) bank65478 +65479 POINT(29.707535476524342 -95.59875922740167) bank65479 +65480 POINT(29.111738183652935 -95.93528787553932) bank65480 +65481 POINT(30.09221307632937 -96.11396729077633) bank65481 +65482 POINT(29.366698960983058 -95.96748668769023) bank65482 +65483 POINT(29.97517244503661 -94.87746603433173) bank65483 +65484 POINT(29.88146909150673 -95.34633023554031) bank65484 +65485 POINT(30.066495667120186 -94.97140377062561) bank65485 +65486 POINT(29.686949071343875 -95.5353675572966) bank65486 +65487 POINT(30.156319375322518 -95.26810874524172) bank65487 +65488 POINT(30.52316085411996 -94.8726345211809) bank65488 +65489 POINT(30.100859367849026 -94.74400395356568) bank65489 +65490 POINT(29.51559057026418 -95.18695333428417) bank65490 +65491 POINT(30.142692487384508 -94.51283255934337) bank65491 +65492 POINT(30.198457905741034 -94.75811098246251) bank65492 +65493 POINT(29.208743010659052 -95.31927195862396) bank65493 +65494 POINT(29.102008577455592 -95.50939631061725) bank65494 +65495 POINT(29.41235639818765 -94.54899651192709) bank65495 +65496 POINT(30.646446511981058 -94.43735615346867) bank65496 +65497 POINT(28.910465606249833 -95.54838934208367) bank65497 +65498 POINT(30.460209006651738 -95.3155334299002) bank65498 +65499 POINT(29.634743660861854 -95.22155880992763) bank65499 +65500 POINT(30.042512545110593 -95.44851822706029) bank65500 +65501 POINT(30.491166170322167 -94.63224399830608) bank65501 +65502 POINT(30.10842421051376 -95.61078275144703) bank65502 +65503 POINT(29.4872582991715 -95.8426238407408) bank65503 +65504 POINT(29.613690176191742 -96.03162282733732) bank65504 +65505 POINT(29.923359442119406 -96.33999396500225) bank65505 +65506 POINT(29.20527684697175 -96.33331083996669) bank65506 +65507 POINT(30.317540284930853 -94.47346243905704) bank65507 +65508 POINT(29.45905898011505 -95.24905946649501) bank65508 +65509 POINT(29.990504465145104 -94.61393271384529) bank65509 +65510 POINT(29.43363930607445 -96.2175671999164) bank65510 +65511 POINT(29.33974060901176 -95.41587512638442) bank65511 +65512 POINT(29.220530144219076 -94.8464896921819) bank65512 +65513 POINT(29.732799741586202 -94.63437561861087) bank65513 +65514 POINT(29.85883198981408 -94.59993850151439) bank65514 +65515 POINT(30.13559384276286 -94.8509697079612) bank65515 +65516 POINT(29.741371978140023 -95.23175522473353) bank65516 +65517 POINT(28.877520967972934 -94.47052692623174) bank65517 +65518 POINT(29.06370668957856 -94.48412974697865) bank65518 +65519 POINT(30.462877610150453 -94.59706726936743) bank65519 +65520 POINT(28.987866685179753 -95.8876813158684) bank65520 +65521 POINT(30.085773201620782 -95.76329106524753) bank65521 +65522 POINT(29.666889669264695 -95.51742211093308) bank65522 +65523 POINT(28.91220405418522 -95.51694948684639) bank65523 +65524 POINT(30.409051926804263 -95.36152370467705) bank65524 +65525 POINT(29.725874159583935 -95.7071089053138) bank65525 +65526 POINT(29.395828145407812 -95.36769810024714) bank65526 +65527 POINT(30.59970172495779 -96.1630562593906) bank65527 +65528 POINT(30.682127187247797 -95.13364017590254) bank65528 +65529 POINT(30.24346706934387 -94.37693783435878) bank65529 +65530 POINT(29.078302216502305 -95.59394249683771) bank65530 +65531 POINT(29.24496992076645 -96.27912315110648) bank65531 +65532 POINT(29.470195086751964 -95.40825526202612) bank65532 +65533 POINT(30.37632573197717 -95.7814900997666) bank65533 +65534 POINT(29.85264887319794 -94.97268021249509) bank65534 +65535 POINT(30.13469464004381 -96.1280537855946) bank65535 +65536 POINT(28.96017617647597 -94.57864522348608) bank65536 +65537 POINT(29.460561696616658 -94.39578764075131) bank65537 +65538 POINT(30.18221225197864 -96.27606971574563) bank65538 +65539 POINT(29.84060337416633 -95.38392965871003) bank65539 +65540 POINT(29.122694959401485 -94.8863325388753) bank65540 +65541 POINT(29.39574537724346 -94.68666502014781) bank65541 +65542 POINT(30.631236556134084 -95.83621924194878) bank65542 +65543 POINT(30.33927784876977 -95.43761239650853) bank65543 +65544 POINT(30.204856559231995 -94.65927137850454) bank65544 +65545 POINT(29.289115880538848 -94.5781429671646) bank65545 +65546 POINT(30.520825991725754 -95.51021464464154) bank65546 +65547 POINT(29.856923567943735 -94.69941604685988) bank65547 +65548 POINT(28.88961702118054 -95.530324737004) bank65548 +65549 POINT(29.693667901841675 -95.8744988797129) bank65549 +65550 POINT(28.97219836427932 -94.60363341207403) bank65550 +65551 POINT(29.846765350674204 -95.70514026311264) bank65551 +65552 POINT(28.816221323640463 -94.98154236482011) bank65552 +65553 POINT(29.971700811438318 -95.32691133447884) bank65553 +65554 POINT(29.50631144228216 -96.17814510237964) bank65554 +65555 POINT(30.51782509181468 -94.806680591205) bank65555 +65556 POINT(30.458872932096362 -95.80948165183032) bank65556 +65557 POINT(30.31575639230504 -95.11028234440329) bank65557 +65558 POINT(29.121824122301213 -95.62610867048944) bank65558 +65559 POINT(29.835320932752378 -95.82190145022798) bank65559 +65560 POINT(29.0903838146597 -95.01083944578593) bank65560 +65561 POINT(29.615093956876827 -94.89892884014478) bank65561 +65562 POINT(29.345637646176495 -96.1550501683016) bank65562 +65563 POINT(30.72530780598541 -95.7759201190262) bank65563 +65564 POINT(28.886791488542983 -95.81933266580525) bank65564 +65565 POINT(29.57131750069366 -95.57892648733782) bank65565 +65566 POINT(30.54391416077174 -94.90315334087951) bank65566 +65567 POINT(29.77312745413252 -94.47811133414199) bank65567 +65568 POINT(30.35584047961504 -95.56838738040221) bank65568 +65569 POINT(28.805596640448236 -94.42765176644234) bank65569 +65570 POINT(30.61880844415023 -96.0352598924758) bank65570 +65571 POINT(30.715179152192707 -95.82094135640897) bank65571 +65572 POINT(29.684187445218683 -95.75766575333174) bank65572 +65573 POINT(29.521940543418353 -95.88027724117835) bank65573 +65574 POINT(30.151151280279247 -95.38033934075926) bank65574 +65575 POINT(30.52855022117884 -95.23477321933349) bank65575 +65576 POINT(29.718637593971103 -94.69683915420701) bank65576 +65577 POINT(28.905202226467328 -95.04586989399915) bank65577 +65578 POINT(29.77557850274257 -94.57871091788806) bank65578 +65579 POINT(28.762533678020358 -95.06179608430615) bank65579 +65580 POINT(29.050186667044017 -95.87488535483627) bank65580 +65581 POINT(30.126781322661515 -95.88393945859217) bank65581 +65582 POINT(30.635603661044026 -94.9854190201261) bank65582 +65583 POINT(30.247596370016254 -95.87682487699014) bank65583 +65584 POINT(28.834785907979168 -95.64282081109201) bank65584 +65585 POINT(30.057160114471827 -96.12734303555644) bank65585 +65586 POINT(29.227873766115156 -96.04098481917674) bank65586 +65587 POINT(29.432418025931337 -94.5821289694029) bank65587 +65588 POINT(30.026689441818867 -96.36602266683886) bank65588 +65589 POINT(29.012069763975838 -95.46812588698289) bank65589 +65590 POINT(30.22196703152019 -95.61020933160185) bank65590 +65591 POINT(30.254539098632247 -96.15745483282154) bank65591 +65592 POINT(30.14584480146801 -95.00001860336718) bank65592 +65593 POINT(29.850225643084663 -95.56652574810813) bank65593 +65594 POINT(30.210806183606376 -95.43067962090886) bank65594 +65595 POINT(29.10321711610888 -94.8026943981707) bank65595 +65596 POINT(30.755368919996478 -95.26235516720226) bank65596 +65597 POINT(28.91338499012972 -95.47728523744387) bank65597 +65598 POINT(30.45801923707227 -94.78552235860548) bank65598 +65599 POINT(30.408544953632443 -95.34919437811277) bank65599 +65600 POINT(29.132467065718274 -95.47555843990777) bank65600 +65601 POINT(30.240010081085025 -95.40232739839053) bank65601 +65602 POINT(29.16305357134996 -95.80849703479807) bank65602 +65603 POINT(30.681793426080894 -94.64507418556472) bank65603 +65604 POINT(28.97756645360823 -94.84137479166905) bank65604 +65605 POINT(30.568625153174228 -95.60517677664485) bank65605 +65606 POINT(30.48862272343008 -94.8770463976882) bank65606 +65607 POINT(30.28726075720655 -95.67638922027594) bank65607 +65608 POINT(29.00768087729248 -94.98163055820105) bank65608 +65609 POINT(29.619831679019114 -94.97077319873021) bank65609 +65610 POINT(29.166679920179966 -96.30197642264915) bank65610 +65611 POINT(29.56244476276087 -96.25086307002874) bank65611 +65612 POINT(30.310354576266942 -96.36110859415179) bank65612 +65613 POINT(30.370832190220128 -94.84709284903204) bank65613 +65614 POINT(29.85357262745077 -94.97088196148604) bank65614 +65615 POINT(29.77564062323215 -95.22276827558369) bank65615 +65616 POINT(30.374202827281056 -95.63822414236297) bank65616 +65617 POINT(29.929467840386344 -96.27046873728335) bank65617 +65618 POINT(28.96166087122168 -96.0007105346609) bank65618 +65619 POINT(30.49616480294682 -95.75055963249851) bank65619 +65620 POINT(29.845688841568062 -96.25527033459812) bank65620 +65621 POINT(29.030392030594466 -94.80972896169175) bank65621 +65622 POINT(28.8819407852401 -94.62194081014276) bank65622 +65623 POINT(30.46737391334338 -94.9085785510847) bank65623 +65624 POINT(30.07452358124778 -96.02724295150236) bank65624 +65625 POINT(29.490580178684535 -95.37735024210369) bank65625 +65626 POINT(29.833356274624208 -94.8411671745637) bank65626 +65627 POINT(29.956204858281428 -95.6688969537754) bank65627 +65628 POINT(29.627047815718853 -94.43803595524514) bank65628 +65629 POINT(30.461357928532497 -94.6531778743214) bank65629 +65630 POINT(29.511719133639215 -94.80752767232984) bank65630 +65631 POINT(29.117510786481116 -94.97177899080079) bank65631 +65632 POINT(29.365774783510844 -95.19325370392265) bank65632 +65633 POINT(29.99689049011635 -96.36595972012226) bank65633 +65634 POINT(30.418297319121784 -94.81378959683818) bank65634 +65635 POINT(29.56351295352978 -94.73577570210347) bank65635 +65636 POINT(30.591790894628545 -96.08450981967071) bank65636 +65637 POINT(29.170086015226513 -94.73343329223908) bank65637 +65638 POINT(30.45299068384572 -96.0841264479459) bank65638 +65639 POINT(30.398812060027215 -95.59188739129861) bank65639 +65640 POINT(30.27537373343122 -95.47082626397543) bank65640 +65641 POINT(29.469416081069287 -95.98746792858887) bank65641 +65642 POINT(28.938928402647385 -96.20608290760966) bank65642 +65643 POINT(29.64684463595206 -96.31357943377644) bank65643 +65644 POINT(29.488670873275794 -94.95720880470148) bank65644 +65645 POINT(29.380437512086953 -95.80428220041428) bank65645 +65646 POINT(29.87614634169168 -95.24744926065864) bank65646 +65647 POINT(30.606783451718115 -95.25149486483551) bank65647 +65648 POINT(30.11406334268653 -95.98372748509834) bank65648 +65649 POINT(29.79593588397924 -96.14941989696528) bank65649 +65650 POINT(30.0091159299677 -94.45980881538584) bank65650 +65651 POINT(29.907980278674266 -95.57810715093518) bank65651 +65652 POINT(30.447712893317078 -94.9750708560455) bank65652 +65653 POINT(29.321039021287845 -94.70563391047324) bank65653 +65654 POINT(30.107165898974053 -96.11100071670923) bank65654 +65655 POINT(30.057047598395553 -96.15077704705705) bank65655 +65656 POINT(30.47891050394008 -95.52052262710592) bank65656 +65657 POINT(30.09880125472627 -95.11652421943832) bank65657 +65658 POINT(30.349756282805807 -94.68191953740397) bank65658 +65659 POINT(29.682021857833224 -95.44353304004883) bank65659 +65660 POINT(30.727671781087462 -96.05128642566217) bank65660 +65661 POINT(30.063516737007575 -96.1478788436074) bank65661 +65662 POINT(29.661712654851787 -95.44248248657503) bank65662 +65663 POINT(30.355930519010982 -94.67900279308907) bank65663 +65664 POINT(30.439211218387257 -94.5601290280594) bank65664 +65665 POINT(29.30006663761078 -96.06898069001896) bank65665 +65666 POINT(30.473548138856444 -94.48083482558712) bank65666 +65667 POINT(30.52751065599802 -94.52162482187536) bank65667 +65668 POINT(30.11210788587449 -95.12273916825117) bank65668 +65669 POINT(30.446119621459957 -96.16653455893403) bank65669 +65670 POINT(29.722837558173037 -95.64720088926069) bank65670 +65671 POINT(28.96277744049029 -94.4626683191365) bank65671 +65672 POINT(30.555656412934837 -95.22159407537794) bank65672 +65673 POINT(30.573406092302783 -95.36486120279127) bank65673 +65674 POINT(30.618502560078078 -95.74700658809049) bank65674 +65675 POINT(29.825313376554018 -94.7584533651441) bank65675 +65676 POINT(29.80704724826078 -96.30149630022986) bank65676 +65677 POINT(30.293566308836173 -95.25133904832289) bank65677 +65678 POINT(30.478474089536963 -95.65689937207335) bank65678 +65679 POINT(29.359623195056834 -95.69288871005136) bank65679 +65680 POINT(29.17872238641995 -95.97287550514756) bank65680 +65681 POINT(29.047691263191577 -95.19896262109602) bank65681 +65682 POINT(29.305416780657584 -94.40340412392815) bank65682 +65683 POINT(29.28073004997277 -95.07414954243208) bank65683 +65684 POINT(30.19228686589141 -95.85458280388795) bank65684 +65685 POINT(29.565407585109273 -95.07926904370925) bank65685 +65686 POINT(29.902847404562042 -96.24223751372423) bank65686 +65687 POINT(30.307676767113968 -95.60629792119173) bank65687 +65688 POINT(30.08366234576411 -94.89439833006186) bank65688 +65689 POINT(28.9859250055449 -95.12423259294263) bank65689 +65690 POINT(29.266433022248115 -96.18706011057222) bank65690 +65691 POINT(30.1218141468759 -95.37032420493186) bank65691 +65692 POINT(28.812553147695635 -95.75168854864886) bank65692 +65693 POINT(30.064368011310094 -96.03503775747404) bank65693 +65694 POINT(30.474600517122862 -94.58073784630378) bank65694 +65695 POINT(29.68579624796935 -95.91676843922806) bank65695 +65696 POINT(29.334887891410702 -95.52680950128884) bank65696 +65697 POINT(30.677152038030137 -95.35034258353281) bank65697 +65698 POINT(29.344853069469497 -96.13021552355917) bank65698 +65699 POINT(30.65414598359311 -95.721900319065) bank65699 +65700 POINT(29.79442049241497 -96.10222708992907) bank65700 +65701 POINT(30.563498255842546 -96.07829070795634) bank65701 +65702 POINT(29.75901651276415 -95.50086022955054) bank65702 +65703 POINT(30.58266842095252 -96.33399434539653) bank65703 +65704 POINT(29.916859278685696 -95.5534704309444) bank65704 +65705 POINT(28.772917782385985 -96.18183048596812) bank65705 +65706 POINT(29.350913589267886 -95.86751160011012) bank65706 +65707 POINT(30.360408572494002 -96.19384149698715) bank65707 +65708 POINT(30.03381946216517 -95.04565719145143) bank65708 +65709 POINT(28.807521515453367 -94.80692520696006) bank65709 +65710 POINT(29.534166224500144 -96.18090937406755) bank65710 +65711 POINT(29.584330306609584 -94.90325644952786) bank65711 +65712 POINT(29.33605846812049 -94.59356059671043) bank65712 +65713 POINT(30.139077080352966 -96.17611667784058) bank65713 +65714 POINT(30.753176076847826 -95.9844697360528) bank65714 +65715 POINT(29.25171268699579 -94.844243808846) bank65715 +65716 POINT(30.503145099267314 -94.67365033259986) bank65716 +65717 POINT(30.02466691958547 -94.8479780526967) bank65717 +65718 POINT(30.58180893878016 -95.20131788202085) bank65718 +65719 POINT(30.427582451713082 -95.86493666803311) bank65719 +65720 POINT(29.623439802898744 -96.17264030897672) bank65720 +65721 POINT(29.29464426069866 -96.12523259976307) bank65721 +65722 POINT(30.168214207327015 -95.98412163885013) bank65722 +65723 POINT(28.96792325890638 -95.07841895140334) bank65723 +65724 POINT(30.342083906700413 -95.78069294765946) bank65724 +65725 POINT(29.36346541240895 -96.28019070143355) bank65725 +65726 POINT(28.79377307989722 -94.47814013408208) bank65726 +65727 POINT(30.48925579769422 -95.9245917596219) bank65727 +65728 POINT(29.509249974364103 -94.8512587034968) bank65728 +65729 POINT(29.499279895232522 -95.15516721034243) bank65729 +65730 POINT(30.199049094666652 -94.93860725213538) bank65730 +65731 POINT(30.06121518013911 -94.41879300908832) bank65731 +65732 POINT(29.35486255203172 -94.61774667050304) bank65732 +65733 POINT(30.657129829043292 -95.65285663053643) bank65733 +65734 POINT(30.566523833029024 -94.6380167814303) bank65734 +65735 POINT(30.687771726963543 -95.13427785103072) bank65735 +65736 POINT(28.902039006486707 -95.64973535837193) bank65736 +65737 POINT(28.78044632579229 -94.99729608327421) bank65737 +65738 POINT(29.084357070944048 -94.39424159278414) bank65738 +65739 POINT(29.238195207269484 -95.79868571445166) bank65739 +65740 POINT(29.524680177650744 -95.37201067392598) bank65740 +65741 POINT(30.66902157820872 -94.73946903090896) bank65741 +65742 POINT(29.543121770395715 -96.22821818523198) bank65742 +65743 POINT(29.632410790396197 -95.97675994880134) bank65743 +65744 POINT(29.35739960367646 -95.77747267211457) bank65744 +65745 POINT(30.063095570935186 -94.6229490664174) bank65745 +65746 POINT(30.250428544867034 -96.1508316520296) bank65746 +65747 POINT(30.07167022908572 -95.03259798225181) bank65747 +65748 POINT(29.06801187826413 -96.05606286299594) bank65748 +65749 POINT(29.88946838508707 -95.64802574259102) bank65749 +65750 POINT(28.814133982535015 -94.53052063755145) bank65750 +65751 POINT(30.586864403795985 -96.35926829333921) bank65751 +65752 POINT(28.851936946003114 -95.60905791340797) bank65752 +65753 POINT(29.263438530334597 -94.86059465873225) bank65753 +65754 POINT(29.505538743357924 -95.85340083673428) bank65754 +65755 POINT(28.841793704012538 -95.0014775552098) bank65755 +65756 POINT(29.007298130599764 -94.87515031998116) bank65756 +65757 POINT(30.549685072546563 -95.57156416210074) bank65757 +65758 POINT(29.488183373372443 -94.95364104880866) bank65758 +65759 POINT(29.341342998748885 -95.70686333690433) bank65759 +65760 POINT(30.26147269978715 -95.17813500394733) bank65760 +65761 POINT(29.033792465272064 -95.83519283829216) bank65761 +65762 POINT(30.122469825359726 -95.5056387629728) bank65762 +65763 POINT(28.852617850356648 -94.64528459439984) bank65763 +65764 POINT(30.711350117443576 -94.80248781755347) bank65764 +65765 POINT(30.254617662461463 -95.31443963478819) bank65765 +65766 POINT(29.55753579670553 -95.96976092117428) bank65766 +65767 POINT(30.348333263060866 -95.30270452895715) bank65767 +65768 POINT(29.178686904485943 -94.54247055703561) bank65768 +65769 POINT(29.70586030983416 -95.14161146092452) bank65769 +65770 POINT(30.035958005888382 -94.39477302349033) bank65770 +65771 POINT(30.08726817603166 -95.03663063571624) bank65771 +65772 POINT(29.006390530124943 -94.55722939507261) bank65772 +65773 POINT(29.46328506867446 -95.54121067571813) bank65773 +65774 POINT(30.150125258362337 -94.79150774462089) bank65774 +65775 POINT(29.196415250408638 -94.85390173489802) bank65775 +65776 POINT(30.307715648145813 -95.16770388760216) bank65776 +65777 POINT(29.32373041363173 -94.80966082534657) bank65777 +65778 POINT(30.41638473556728 -94.59277140138813) bank65778 +65779 POINT(29.97709924762956 -94.83706759534955) bank65779 +65780 POINT(30.465139007913198 -95.09626518721667) bank65780 +65781 POINT(30.477052539172032 -94.93008751055278) bank65781 +65782 POINT(30.139138835206126 -94.51984385217305) bank65782 +65783 POINT(29.715984943431696 -95.08832723109387) bank65783 +65784 POINT(29.62958066869264 -94.93160264794024) bank65784 +65785 POINT(29.980767432802377 -94.46163620160038) bank65785 +65786 POINT(28.97017528199775 -95.31384977008179) bank65786 +65787 POINT(29.00286216241648 -94.52677057771342) bank65787 +65788 POINT(30.471270523251707 -96.31589745595845) bank65788 +65789 POINT(29.937559734692947 -94.6982661504814) bank65789 +65790 POINT(30.291340700930313 -96.29186452928852) bank65790 +65791 POINT(28.968348111976635 -95.77806138847122) bank65791 +65792 POINT(29.972976152984703 -94.47378774457319) bank65792 +65793 POINT(28.940959595944914 -95.13073387315492) bank65793 +65794 POINT(29.91720225086008 -94.68577166563618) bank65794 +65795 POINT(29.261754957343257 -96.16082777102665) bank65795 +65796 POINT(30.555627682058418 -94.40543668146968) bank65796 +65797 POINT(29.0765336765177 -95.47152344015356) bank65797 +65798 POINT(28.91599415820907 -94.68243387327375) bank65798 +65799 POINT(29.159592719571176 -95.8876283761764) bank65799 +65800 POINT(28.934361728732565 -94.93816836288717) bank65800 +65801 POINT(28.8736886705315 -95.89252526460284) bank65801 +65802 POINT(29.412969913517227 -94.41524023085532) bank65802 +65803 POINT(29.80555844927247 -94.46866711774537) bank65803 +65804 POINT(29.59710678167276 -95.04020849073133) bank65804 +65805 POINT(29.520079133581497 -96.17333327216541) bank65805 +65806 POINT(30.079967308658347 -94.67083316203112) bank65806 +65807 POINT(30.725373480434783 -94.51240528952992) bank65807 +65808 POINT(28.865396709338132 -95.37921531547536) bank65808 +65809 POINT(30.16253630956037 -95.80928766749639) bank65809 +65810 POINT(30.75772732766165 -94.77848611325967) bank65810 +65811 POINT(30.1724308908769 -95.76555171898306) bank65811 +65812 POINT(29.958524440363526 -95.66787124182737) bank65812 +65813 POINT(28.846054519174707 -94.88742351550098) bank65813 +65814 POINT(29.215358322188504 -94.41003581919334) bank65814 +65815 POINT(28.979388486933285 -94.46198644060998) bank65815 +65816 POINT(29.712920957481504 -96.16552815565899) bank65816 +65817 POINT(29.161466261674335 -96.15652478795728) bank65817 +65818 POINT(30.57469170775414 -96.27386388363023) bank65818 +65819 POINT(30.72617015534008 -96.36161092502826) bank65819 +65820 POINT(29.54873735333043 -95.53070703568066) bank65820 +65821 POINT(30.604367720356848 -95.2682261332356) bank65821 +65822 POINT(30.004075464926807 -94.56672059133514) bank65822 +65823 POINT(29.264648493129098 -96.32002735533625) bank65823 +65824 POINT(30.497704839574155 -95.06740763543702) bank65824 +65825 POINT(29.4857455436277 -95.6770362728169) bank65825 +65826 POINT(29.44349863068842 -96.16789371246419) bank65826 +65827 POINT(28.890201227866125 -96.04730027433592) bank65827 +65828 POINT(30.535613898663893 -95.26232455484354) bank65828 +65829 POINT(30.01511131137235 -95.68438728851666) bank65829 +65830 POINT(29.915376223039942 -95.534384007799) bank65830 +65831 POINT(29.491091016879516 -94.6044261266873) bank65831 +65832 POINT(30.623476205199655 -96.16045959913917) bank65832 +65833 POINT(29.136195322142214 -95.19434355876284) bank65833 +65834 POINT(28.822418939255495 -95.94269118230038) bank65834 +65835 POINT(28.821122271535458 -96.11495054744967) bank65835 +65836 POINT(28.99396320348762 -94.75819361291298) bank65836 +65837 POINT(29.789993147087984 -96.15026864288025) bank65837 +65838 POINT(29.196913413168573 -95.43091175080042) bank65838 +65839 POINT(30.508126542711114 -94.51297370891865) bank65839 +65840 POINT(29.08906799010716 -94.3945583478359) bank65840 +65841 POINT(30.554861507002858 -95.76498702669684) bank65841 +65842 POINT(29.021316363409877 -95.84289730975306) bank65842 +65843 POINT(29.88396102933728 -94.66569618676897) bank65843 +65844 POINT(29.620492232423132 -94.86177603000077) bank65844 +65845 POINT(29.091114723850257 -95.47327965401257) bank65845 +65846 POINT(30.60339373112961 -95.89283085683263) bank65846 +65847 POINT(30.372027942081004 -95.5546766988651) bank65847 +65848 POINT(29.713351924096955 -94.5916734429957) bank65848 +65849 POINT(29.451689035864128 -94.4186225057504) bank65849 +65850 POINT(30.54519540960855 -96.04328777285359) bank65850 +65851 POINT(29.811284220021683 -96.16178107580313) bank65851 +65852 POINT(28.846738907050057 -94.42519303333198) bank65852 +65853 POINT(29.036070634611633 -95.74275425584189) bank65853 +65854 POINT(30.671995643450632 -95.80578957932855) bank65854 +65855 POINT(29.87580583773911 -95.32656011528493) bank65855 +65856 POINT(30.704195016014975 -94.93782049116663) bank65856 +65857 POINT(29.493087872368946 -94.96735797851635) bank65857 +65858 POINT(29.600429805400545 -96.01006126820499) bank65858 +65859 POINT(29.993716987786428 -95.59922672644791) bank65859 +65860 POINT(30.535010819980826 -94.61640777977695) bank65860 +65861 POINT(30.31638358549681 -95.37913621981593) bank65861 +65862 POINT(29.968251097407073 -95.99524380664158) bank65862 +65863 POINT(29.364997272835936 -95.9486411433771) bank65863 +65864 POINT(29.798609103775707 -95.77787196895315) bank65864 +65865 POINT(29.47337058422288 -95.6011999715339) bank65865 +65866 POINT(30.62478165440731 -96.09670331763209) bank65866 +65867 POINT(29.86868968252936 -96.21552256030594) bank65867 +65868 POINT(30.719804301940645 -94.6043716426183) bank65868 +65869 POINT(29.68462349475169 -95.24299329044611) bank65869 +65870 POINT(28.809450898704547 -95.45916637841623) bank65870 +65871 POINT(30.023196970546078 -95.08719904169317) bank65871 +65872 POINT(30.024788741290664 -95.20872494177075) bank65872 +65873 POINT(29.84097751060134 -94.80575408889236) bank65873 +65874 POINT(29.692679309996766 -95.94978454960604) bank65874 +65875 POINT(28.7745230324914 -95.94354334332967) bank65875 +65876 POINT(29.917069686531015 -94.42255934525906) bank65876 +65877 POINT(29.376848319287934 -94.94602535172015) bank65877 +65878 POINT(30.5208657751305 -95.80419021101072) bank65878 +65879 POINT(29.67349435382151 -94.75987682161484) bank65879 +65880 POINT(30.398474375351174 -95.23310735595301) bank65880 +65881 POINT(29.275635712119815 -94.85626955713566) bank65881 +65882 POINT(29.83440436294281 -95.97791507271465) bank65882 +65883 POINT(29.70742795890045 -95.16721317921434) bank65883 +65884 POINT(29.23518050041667 -96.23424112681619) bank65884 +65885 POINT(29.890601215902738 -95.98334507744572) bank65885 +65886 POINT(28.933289999354372 -96.1743035814247) bank65886 +65887 POINT(30.719856530323817 -96.11123761582424) bank65887 +65888 POINT(29.77472299530681 -94.52635331651764) bank65888 +65889 POINT(29.307097978430562 -94.73499143478224) bank65889 +65890 POINT(30.198244973772493 -94.67962382960042) bank65890 +65891 POINT(30.42354826372305 -96.25363950408642) bank65891 +65892 POINT(29.604885270383797 -96.25617407819233) bank65892 +65893 POINT(29.06425842264085 -95.54735459909172) bank65893 +65894 POINT(29.785680915224802 -94.68934826710696) bank65894 +65895 POINT(28.903448726348717 -94.92227302288636) bank65895 +65896 POINT(30.388129570120782 -96.30937363635977) bank65896 +65897 POINT(30.67366833904678 -94.79294482070418) bank65897 +65898 POINT(30.426320770977412 -95.92552935588499) bank65898 +65899 POINT(29.22304023383633 -96.28252424843372) bank65899 +65900 POINT(30.430406348404798 -96.3018544524513) bank65900 +65901 POINT(30.728172863814223 -95.32002160615593) bank65901 +65902 POINT(30.740286716430813 -94.91823877131289) bank65902 +65903 POINT(30.221569463729814 -96.00586510250517) bank65903 +65904 POINT(29.12315893822629 -95.35473478187428) bank65904 +65905 POINT(29.433860946675516 -95.84360426502482) bank65905 +65906 POINT(28.783981668770572 -95.68406824598101) bank65906 +65907 POINT(28.790770168165462 -95.08916438907202) bank65907 +65908 POINT(30.72093357886007 -95.98852296257736) bank65908 +65909 POINT(29.813739515986803 -94.45562618411971) bank65909 +65910 POINT(29.3644076047897 -94.93784714051529) bank65910 +65911 POINT(30.349467521547034 -94.48225462142355) bank65911 +65912 POINT(30.720182566714644 -94.6027490164786) bank65912 +65913 POINT(29.875885392709034 -95.25642458635515) bank65913 +65914 POINT(28.8108828043342 -94.85408021029296) bank65914 +65915 POINT(30.05473801717908 -94.6286955839742) bank65915 +65916 POINT(29.672089093947903 -94.42703593312157) bank65916 +65917 POINT(30.63205997837563 -96.10086537264462) bank65917 +65918 POINT(30.489035511152043 -95.05211089551969) bank65918 +65919 POINT(29.15657684072617 -94.94135013840008) bank65919 +65920 POINT(30.759845283841866 -94.84493100871268) bank65920 +65921 POINT(29.713768526796088 -95.30705338703758) bank65921 +65922 POINT(29.232420881252768 -96.2731947973935) bank65922 +65923 POINT(29.648699127718675 -96.36044015187402) bank65923 +65924 POINT(30.633080912225804 -95.26038627338049) bank65924 +65925 POINT(29.53718091282147 -95.59980847957007) bank65925 +65926 POINT(29.491255222682717 -95.71935084412718) bank65926 +65927 POINT(29.429590344395585 -95.21832011838976) bank65927 +65928 POINT(29.708432528507476 -95.87165805869276) bank65928 +65929 POINT(29.670159500596675 -95.40745588573532) bank65929 +65930 POINT(29.959204658741076 -94.87617420921326) bank65930 +65931 POINT(29.647097718935626 -95.5963335164562) bank65931 +65932 POINT(29.58482732678985 -95.40359844165185) bank65932 +65933 POINT(29.718152039441247 -94.62789553166424) bank65933 +65934 POINT(30.72647881769119 -95.71867584385974) bank65934 +65935 POINT(28.82343234282508 -94.73527084850333) bank65935 +65936 POINT(29.702806199036104 -95.55550696774881) bank65936 +65937 POINT(28.83262104417322 -94.65267079639172) bank65937 +65938 POINT(29.436918995574892 -95.06862337384138) bank65938 +65939 POINT(30.345841838388168 -95.12617993675822) bank65939 +65940 POINT(30.21357479397352 -96.2798282251849) bank65940 +65941 POINT(29.96747185889186 -94.5929753123493) bank65941 +65942 POINT(30.602668601873603 -96.17087886924739) bank65942 +65943 POINT(30.193503872262188 -94.80312692544854) bank65943 +65944 POINT(28.952207034993542 -94.98646958860174) bank65944 +65945 POINT(30.07398864002236 -94.74821213325107) bank65945 +65946 POINT(30.211810884389017 -96.20939927369344) bank65946 +65947 POINT(28.77774519299639 -95.60412968555474) bank65947 +65948 POINT(29.595767919802004 -96.00231440131064) bank65948 +65949 POINT(29.58370756333864 -94.94122909113678) bank65949 +65950 POINT(30.24864058400784 -95.47038309171306) bank65950 +65951 POINT(29.40961239711707 -95.8203830593768) bank65951 +65952 POINT(30.498473292010043 -95.82737383882925) bank65952 +65953 POINT(29.979729159122016 -96.33748619828539) bank65953 +65954 POINT(29.639639139949722 -95.37592421693516) bank65954 +65955 POINT(29.367878611671983 -94.56760220988754) bank65955 +65956 POINT(29.885263278152962 -95.05253919938731) bank65956 +65957 POINT(28.799394066772255 -95.44571283781423) bank65957 +65958 POINT(30.4249396586318 -95.4293318630041) bank65958 +65959 POINT(29.441293704382424 -94.91968769465484) bank65959 +65960 POINT(30.246302296439197 -95.90083857005833) bank65960 +65961 POINT(30.705676302809888 -94.94190535510806) bank65961 +65962 POINT(29.80473613384785 -94.76960330234186) bank65962 +65963 POINT(29.579371239122445 -95.09076634294209) bank65963 +65964 POINT(29.70440758204715 -95.36090996245574) bank65964 +65965 POINT(30.609410784375267 -95.55800040799193) bank65965 +65966 POINT(30.368774792640075 -95.75020055977954) bank65966 +65967 POINT(30.62461511188788 -95.89439363026095) bank65967 +65968 POINT(28.980323242302042 -95.42009184675808) bank65968 +65969 POINT(30.668510976554796 -95.57908287659437) bank65969 +65970 POINT(29.951623545826763 -95.12507293849714) bank65970 +65971 POINT(28.82723061434155 -96.0155065689778) bank65971 +65972 POINT(28.975028440258672 -95.20120091802502) bank65972 +65973 POINT(30.340233279890462 -95.71270134379297) bank65973 +65974 POINT(29.705805214003835 -96.05980240721091) bank65974 +65975 POINT(29.115954076615605 -94.6630346370294) bank65975 +65976 POINT(29.5217282753008 -94.56750943539423) bank65976 +65977 POINT(30.747040268813493 -95.69275453419523) bank65977 +65978 POINT(30.003474443877224 -96.03313170155062) bank65978 +65979 POINT(30.597575446517936 -96.06305136204313) bank65979 +65980 POINT(28.965828714045884 -95.09621499893031) bank65980 +65981 POINT(29.607435189353218 -96.33356801816267) bank65981 +65982 POINT(29.056075338860918 -95.29202862613822) bank65982 +65983 POINT(29.647512682216338 -94.8454376378708) bank65983 +65984 POINT(29.610730887241274 -95.91835809092424) bank65984 +65985 POINT(29.993613972177627 -95.45263816359068) bank65985 +65986 POINT(29.050366512390724 -95.8034814745581) bank65986 +65987 POINT(29.62435078737008 -95.89208270100771) bank65987 +65988 POINT(29.94040787047256 -95.56489356870992) bank65988 +65989 POINT(28.80959910980565 -96.3129271212275) bank65989 +65990 POINT(29.61067466002769 -94.83200761847364) bank65990 +65991 POINT(29.51715222445862 -95.24975103198763) bank65991 +65992 POINT(30.073491272092273 -94.57142839968668) bank65992 +65993 POINT(29.08142122836082 -95.7705007933297) bank65993 +65994 POINT(30.545953329667917 -94.39084065826853) bank65994 +65995 POINT(29.596995573603255 -96.21680397813186) bank65995 +65996 POINT(28.784859837601793 -95.97243250697284) bank65996 +65997 POINT(30.46122272875491 -95.19635951403632) bank65997 +65998 POINT(30.361237140524807 -95.64650266066094) bank65998 +65999 POINT(29.69983822694615 -96.25738360080267) bank65999 +66000 POINT(29.936513114314042 -96.33687052947671) bank66000 +66001 POINT(29.376906633228693 -95.78578972191681) bank66001 +66002 POINT(29.594955528394614 -95.54601410965893) bank66002 +66003 POINT(29.61708132924015 -96.15184084096938) bank66003 +66004 POINT(29.07003321587286 -95.59949142289514) bank66004 +66005 POINT(29.567529455162447 -96.29000795808344) bank66005 +66006 POINT(29.4604255236904 -95.5474198307652) bank66006 +66007 POINT(29.585374464710462 -95.19318373949541) bank66007 +66008 POINT(29.59686979236741 -96.32890063874828) bank66008 +66009 POINT(29.03561071028189 -94.73039797805954) bank66009 +66010 POINT(29.482170182833027 -96.35943133112299) bank66010 +66011 POINT(30.19719667567192 -95.22626632233714) bank66011 +66012 POINT(30.630014284710818 -95.21691183624134) bank66012 +66013 POINT(29.362797138716815 -95.31917317080895) bank66013 +66014 POINT(29.291579764736017 -96.09153166880121) bank66014 +66015 POINT(28.982516436025787 -96.16337150938055) bank66015 +66016 POINT(29.466040541112818 -95.1111924635463) bank66016 +66017 POINT(30.301471178828734 -95.63904699805617) bank66017 +66018 POINT(29.827364469283182 -95.37938592142245) bank66018 +66019 POINT(30.08412270920265 -96.28766223945843) bank66019 +66020 POINT(30.13167723742141 -95.56281831424752) bank66020 +66021 POINT(29.946011037899392 -95.14991692295222) bank66021 +66022 POINT(29.942212375437173 -95.14646817481548) bank66022 +66023 POINT(29.700177697004793 -95.36673418096181) bank66023 +66024 POINT(29.030592594580277 -95.69841380952946) bank66024 +66025 POINT(28.8808709962408 -95.73938217815572) bank66025 +66026 POINT(29.571510373559967 -95.38942160317417) bank66026 +66027 POINT(28.913567652280737 -95.1525574811665) bank66027 +66028 POINT(29.72313438218792 -94.70015727362501) bank66028 +66029 POINT(30.243626289026118 -95.06075221989902) bank66029 +66030 POINT(30.672451050593885 -94.66709982266275) bank66030 +66031 POINT(30.365574280177665 -96.0650630402303) bank66031 +66032 POINT(30.155518218077905 -94.60034980637901) bank66032 +66033 POINT(29.15099330671482 -96.34049921103728) bank66033 +66034 POINT(29.242964894516042 -95.51389839231952) bank66034 +66035 POINT(28.78974443787728 -95.31751578694852) bank66035 +66036 POINT(30.129200097226164 -95.82701936770275) bank66036 +66037 POINT(28.892019276738605 -96.33519111928487) bank66037 +66038 POINT(29.471325753740178 -95.45840504385292) bank66038 +66039 POINT(28.8309183048312 -96.12437296018503) bank66039 +66040 POINT(30.357719618670007 -95.57872498526461) bank66040 +66041 POINT(29.662205599050417 -95.22638312172747) bank66041 +66042 POINT(30.714407117660393 -95.42488679313132) bank66042 +66043 POINT(29.598968931301613 -96.01342110681442) bank66043 +66044 POINT(29.73090501410679 -95.79666978527254) bank66044 +66045 POINT(30.534017889338163 -94.82844976904926) bank66045 +66046 POINT(29.76493300304797 -96.0023666290476) bank66046 +66047 POINT(30.567203521856158 -95.04227964497763) bank66047 +66048 POINT(29.090493812107557 -95.24707613252696) bank66048 +66049 POINT(29.532080244414935 -95.73304049642587) bank66049 +66050 POINT(30.405485404957375 -94.86900761976975) bank66050 +66051 POINT(29.33087631926566 -95.72212953841083) bank66051 +66052 POINT(29.965596083083458 -94.61553829621683) bank66052 +66053 POINT(30.499096748600795 -94.52914925917158) bank66053 +66054 POINT(29.273009019641663 -94.92959734045021) bank66054 +66055 POINT(29.561679689084613 -94.64014088369157) bank66055 +66056 POINT(29.306502217995543 -94.85427300024293) bank66056 +66057 POINT(29.479249017293785 -95.9749668649816) bank66057 +66058 POINT(30.547148969000776 -95.75359945750265) bank66058 +66059 POINT(30.131659331194616 -95.06571085351375) bank66059 +66060 POINT(30.195589340685345 -95.77071634029366) bank66060 +66061 POINT(29.570212023111825 -95.02987716365499) bank66061 +66062 POINT(29.740766495691414 -95.56784876362975) bank66062 +66063 POINT(30.2343510243373 -96.15334571769253) bank66063 +66064 POINT(30.051098496458167 -96.039612144642) bank66064 +66065 POINT(29.466605529373872 -95.07766267704834) bank66065 +66066 POINT(30.02043355910049 -96.09717760228264) bank66066 +66067 POINT(29.75396113672763 -95.66097935190093) bank66067 +66068 POINT(30.084964189434533 -94.69962699167512) bank66068 +66069 POINT(30.38250109563996 -96.31215349943983) bank66069 +66070 POINT(29.909604945121767 -95.07396310274675) bank66070 +66071 POINT(29.39255399428201 -96.16124245047111) bank66071 +66072 POINT(28.81965098739181 -94.9234872575635) bank66072 +66073 POINT(29.687302511118773 -95.88201640492458) bank66073 +66074 POINT(28.803453816032654 -95.24264321922456) bank66074 +66075 POINT(29.630154074910926 -94.61533135836909) bank66075 +66076 POINT(30.76020201792437 -95.81218414242907) bank66076 +66077 POINT(29.309087033015594 -96.2028304145848) bank66077 +66078 POINT(28.912665381892218 -95.07119623688438) bank66078 +66079 POINT(29.773157843367727 -94.4464330323301) bank66079 +66080 POINT(29.952761246108846 -94.41720376717578) bank66080 +66081 POINT(29.15411626201751 -96.25359048902696) bank66081 +66082 POINT(29.096596292695953 -95.3530585407852) bank66082 +66083 POINT(29.526404255615216 -94.57069176035529) bank66083 +66084 POINT(29.475641626245658 -96.00942751364846) bank66084 +66085 POINT(30.669414563744056 -94.86318597119084) bank66085 +66086 POINT(30.750853259446806 -94.68761951054816) bank66086 +66087 POINT(30.244145538468846 -95.97952478511611) bank66087 +66088 POINT(30.42159969079779 -94.59645700061151) bank66088 +66089 POINT(30.335516383676403 -96.03945067829892) bank66089 +66090 POINT(29.689285542441844 -96.34849415460619) bank66090 +66091 POINT(29.092188765226304 -96.32055679298522) bank66091 +66092 POINT(29.85233488179994 -95.47649217818224) bank66092 +66093 POINT(30.456853322554096 -95.2123077598332) bank66093 +66094 POINT(29.23414466239146 -95.559747815914) bank66094 +66095 POINT(30.14773890346549 -96.29272401481431) bank66095 +66096 POINT(29.402922360234655 -95.85783909715632) bank66096 +66097 POINT(30.491454519045995 -95.1306043290253) bank66097 +66098 POINT(29.653727696375128 -95.6374301159175) bank66098 +66099 POINT(30.672095408302752 -94.77974089205432) bank66099 +66100 POINT(30.462245858407584 -96.10882349079102) bank66100 +66101 POINT(29.939889747307575 -94.73589402486921) bank66101 +66102 POINT(29.138658328311788 -95.03889009239124) bank66102 +66103 POINT(29.748478304944253 -95.98379881511062) bank66103 +66104 POINT(30.339251245331283 -95.43439125838873) bank66104 +66105 POINT(30.087255000601854 -95.96644255767538) bank66105 +66106 POINT(29.708974438080393 -96.17977718061975) bank66106 +66107 POINT(30.227883818226868 -94.85422785350558) bank66107 +66108 POINT(30.70174804313118 -96.1134271733227) bank66108 +66109 POINT(29.603795748561094 -96.13979409777772) bank66109 +66110 POINT(29.82256078198658 -94.62225730522665) bank66110 +66111 POINT(29.53022913082402 -94.98734136739222) bank66111 +66112 POINT(29.115249609936217 -95.30542612100058) bank66112 +66113 POINT(30.00924078061421 -95.25661714159321) bank66113 +66114 POINT(29.398515430370274 -95.62543652806693) bank66114 +66115 POINT(30.489690622706565 -94.95693936135693) bank66115 +66116 POINT(30.634460277419144 -96.05276238701713) bank66116 +66117 POINT(30.35642619560406 -94.7749711803552) bank66117 +66118 POINT(28.807911489641366 -96.07872864748416) bank66118 +66119 POINT(29.189077291544024 -95.03990059840793) bank66119 +66120 POINT(29.968992088379636 -94.85750190478514) bank66120 +66121 POINT(30.705046613639087 -95.12256764041798) bank66121 +66122 POINT(29.434050427852657 -95.45669521144364) bank66122 +66123 POINT(29.021815329294025 -95.03548676088192) bank66123 +66124 POINT(29.409339687508805 -95.05345258755968) bank66124 +66125 POINT(28.964115942548155 -94.61655479954555) bank66125 +66126 POINT(28.805522530802566 -95.64325606606073) bank66126 +66127 POINT(29.83040546914519 -96.19281682142484) bank66127 +66128 POINT(30.740001954064564 -96.23087429339547) bank66128 +66129 POINT(29.58559606527103 -95.4921820329745) bank66129 +66130 POINT(30.143164545810436 -96.20712638647409) bank66130 +66131 POINT(30.138001331690496 -95.47335623452682) bank66131 +66132 POINT(30.69880373055214 -96.07178497038605) bank66132 +66133 POINT(29.82337325132859 -94.41373887506029) bank66133 +66134 POINT(29.009650031342968 -94.88446049784376) bank66134 +66135 POINT(29.125898928575094 -94.62987999403724) bank66135 +66136 POINT(29.30043527852132 -95.03544624107418) bank66136 +66137 POINT(29.358011644511727 -95.91273510729516) bank66137 +66138 POINT(30.552454508395588 -96.3660508405101) bank66138 +66139 POINT(30.220141206230174 -95.43612267526261) bank66139 +66140 POINT(30.183282129483953 -94.42851425093586) bank66140 +66141 POINT(29.732171270946733 -95.82667206638348) bank66141 +66142 POINT(30.54500652388387 -95.14444940292515) bank66142 +66143 POINT(30.257555861050562 -96.33335635236189) bank66143 +66144 POINT(29.163804814741756 -94.41362144255815) bank66144 +66145 POINT(30.24932719277404 -94.88703595736322) bank66145 +66146 POINT(29.358052245539138 -95.20092591908218) bank66146 +66147 POINT(28.869022941869517 -95.4411657423735) bank66147 +66148 POINT(30.34674375067512 -95.97524620863167) bank66148 +66149 POINT(30.31978925377057 -95.22865655738167) bank66149 +66150 POINT(30.373282373895783 -94.4578449653649) bank66150 +66151 POINT(30.70583211033437 -95.59658427080943) bank66151 +66152 POINT(29.186927482162083 -95.89139021242687) bank66152 +66153 POINT(30.116943211280294 -95.30393467592224) bank66153 +66154 POINT(29.952726286062003 -95.79563046242518) bank66154 +66155 POINT(29.227488552864756 -95.64388129102943) bank66155 +66156 POINT(30.468297404122623 -94.8010838298514) bank66156 +66157 POINT(30.731383624914518 -96.2515740089977) bank66157 +66158 POINT(29.209971757624018 -94.37177596256043) bank66158 +66159 POINT(30.071831982682767 -95.62309931613476) bank66159 +66160 POINT(29.60317108501993 -95.96543662437959) bank66160 +66161 POINT(29.76270208176954 -95.15039658565684) bank66161 +66162 POINT(29.660640248963478 -94.69593806840939) bank66162 +66163 POINT(29.72101934643229 -95.94618293743648) bank66163 +66164 POINT(29.370343707039712 -95.22720097135343) bank66164 +66165 POINT(30.448538658204463 -94.5264918870124) bank66165 +66166 POINT(29.387819717352244 -95.7613987035386) bank66166 +66167 POINT(30.508159763783805 -94.41985436545254) bank66167 +66168 POINT(30.433161754878206 -95.1673994629448) bank66168 +66169 POINT(29.402384206707673 -95.70871308513668) bank66169 +66170 POINT(30.447279961541252 -96.1682185411519) bank66170 +66171 POINT(29.203812407154757 -95.56314252046629) bank66171 +66172 POINT(30.055154084250095 -94.82100561357741) bank66172 +66173 POINT(30.48989899292154 -94.45972188828976) bank66173 +66174 POINT(30.13641176072177 -95.78909852466565) bank66174 +66175 POINT(30.32671880609639 -96.36094502515904) bank66175 +66176 POINT(30.220963013296117 -95.02170794986876) bank66176 +66177 POINT(29.526712036120347 -96.11970138693906) bank66177 +66178 POINT(30.593951718439737 -95.00215866101723) bank66178 +66179 POINT(28.827430393093536 -96.05415426902593) bank66179 +66180 POINT(28.84571112125841 -94.70946754785174) bank66180 +66181 POINT(30.54951047079174 -94.51001215240451) bank66181 +66182 POINT(30.234423649500332 -96.15697313511663) bank66182 +66183 POINT(29.434868373899295 -94.39362942883314) bank66183 +66184 POINT(30.248038890791115 -96.33818677279163) bank66184 +66185 POINT(30.0012940756219 -95.33252031350135) bank66185 +66186 POINT(30.13036676382548 -94.54506264580539) bank66186 +66187 POINT(30.64833674700694 -94.6907293483049) bank66187 +66188 POINT(29.511004359441 -95.64343658857557) bank66188 +66189 POINT(30.203544042050005 -95.69337628166373) bank66189 +66190 POINT(29.255708355147707 -94.41520685516791) bank66190 +66191 POINT(28.834739642501685 -95.95384866532802) bank66191 +66192 POINT(30.277906697420374 -95.780369590517) bank66192 +66193 POINT(29.665186924827154 -95.31113119366398) bank66193 +66194 POINT(29.859616863567442 -95.03361110347731) bank66194 +66195 POINT(30.298707380466624 -95.66485753101985) bank66195 +66196 POINT(29.27643880673588 -95.33632080659478) bank66196 +66197 POINT(29.44652987000989 -94.38790462095955) bank66197 +66198 POINT(29.476613900806576 -95.76638644907554) bank66198 +66199 POINT(29.79989295980255 -95.1468938745781) bank66199 +66200 POINT(30.457282587372543 -94.43924958631415) bank66200 +66201 POINT(30.183997534568867 -95.34541180948081) bank66201 +66202 POINT(29.219442724240018 -95.64333350027032) bank66202 +66203 POINT(29.23060712266934 -94.55861916118617) bank66203 +66204 POINT(29.981149883906006 -95.67310148723148) bank66204 +66205 POINT(29.521920110038934 -95.84368699043833) bank66205 +66206 POINT(29.62293920212376 -95.12753154133897) bank66206 +66207 POINT(29.488061035633674 -95.44618829730614) bank66207 +66208 POINT(29.731707749710434 -96.27619173189976) bank66208 +66209 POINT(30.059587772774538 -95.33315707680636) bank66209 +66210 POINT(29.7623437519437 -94.70110603401761) bank66210 +66211 POINT(30.255281566661807 -94.54930425258596) bank66211 +66212 POINT(29.008104781230855 -96.09994845244266) bank66212 +66213 POINT(29.58465647492405 -95.22455304761046) bank66213 +66214 POINT(30.740943715901217 -94.67398004326468) bank66214 +66215 POINT(30.29709855835943 -95.04451820869178) bank66215 +66216 POINT(29.741380132507835 -95.10213779971203) bank66216 +66217 POINT(30.464229889742345 -94.6739181497791) bank66217 +66218 POINT(30.646677256112685 -95.76046116197699) bank66218 +66219 POINT(30.55392424899312 -94.80999607786829) bank66219 +66220 POINT(30.223752839041765 -96.05650627965834) bank66220 +66221 POINT(30.05678294050781 -95.40819946940321) bank66221 +66222 POINT(29.581529439610126 -96.32363948553345) bank66222 +66223 POINT(29.105531634406567 -94.69463836031397) bank66223 +66224 POINT(30.0182973804679 -94.59617442521883) bank66224 +66225 POINT(29.81869333243245 -94.61904111334901) bank66225 +66226 POINT(29.523097893060775 -95.59291831930565) bank66226 +66227 POINT(28.927456121564873 -94.70709157830595) bank66227 +66228 POINT(30.650459448653752 -95.65392636169759) bank66228 +66229 POINT(29.10986153864761 -94.54938817730556) bank66229 +66230 POINT(29.957955983539414 -95.28114682021996) bank66230 +66231 POINT(28.971084942016333 -94.83341319401812) bank66231 +66232 POINT(30.659089937729057 -95.67587228331321) bank66232 +66233 POINT(28.975009544776427 -94.47569214308199) bank66233 +66234 POINT(29.560865151282076 -95.77132332528305) bank66234 +66235 POINT(29.825234281829857 -94.57263247492094) bank66235 +66236 POINT(28.81458686939437 -94.91216272815666) bank66236 +66237 POINT(28.763458816257025 -95.5256669756588) bank66237 +66238 POINT(30.218946265958245 -96.33150696115192) bank66238 +66239 POINT(30.22200645714563 -95.3661332032482) bank66239 +66240 POINT(29.69893755087492 -96.16852499598284) bank66240 +66241 POINT(30.281589861263885 -94.78276159813299) bank66241 +66242 POINT(30.35648173854033 -95.81192817215853) bank66242 +66243 POINT(29.724866867414526 -94.77429283819427) bank66243 +66244 POINT(28.837492757421916 -96.33170148733282) bank66244 +66245 POINT(29.920342497188713 -95.3377986160798) bank66245 +66246 POINT(29.2493619942921 -95.78345102544408) bank66246 +66247 POINT(30.614981451477522 -94.86543792724143) bank66247 +66248 POINT(30.55990379515308 -94.3745178684996) bank66248 +66249 POINT(29.638449389220128 -94.5511715630034) bank66249 +66250 POINT(30.162024949780413 -96.32472076092655) bank66250 +66251 POINT(30.293497721454376 -94.45517653541079) bank66251 +66252 POINT(29.453128590632044 -95.27143841480259) bank66252 +66253 POINT(29.208133205043794 -94.97364761403496) bank66253 +66254 POINT(29.51041209114826 -95.4275033274356) bank66254 +66255 POINT(29.40732812304657 -95.54764112356554) bank66255 +66256 POINT(30.170349595742454 -96.15908245035845) bank66256 +66257 POINT(30.14426568657448 -94.93023214411348) bank66257 +66258 POINT(29.581765160957804 -95.25874694624527) bank66258 +66259 POINT(28.855843860809788 -94.88787698485586) bank66259 +66260 POINT(29.292836418070515 -94.87408086814969) bank66260 +66261 POINT(30.58355946722785 -95.20343442156509) bank66261 +66262 POINT(30.469403106396484 -96.06068057822684) bank66262 +66263 POINT(30.517890781301226 -94.689068527502) bank66263 +66264 POINT(30.31962360371276 -94.56327215913214) bank66264 +66265 POINT(29.000933087586983 -96.0353101687673) bank66265 +66266 POINT(29.24216114732026 -95.07781405843974) bank66266 +66267 POINT(30.677767574138862 -94.58946956314007) bank66267 +66268 POINT(29.17631445886613 -96.14972442170195) bank66268 +66269 POINT(29.320982707876095 -96.31071947591302) bank66269 +66270 POINT(29.446026269610346 -94.86436773550449) bank66270 +66271 POINT(30.12635039481728 -96.00774593558666) bank66271 +66272 POINT(28.93474860422881 -95.94099405119576) bank66272 +66273 POINT(29.32588043296687 -95.87510224927219) bank66273 +66274 POINT(29.785984277624213 -95.2255441037612) bank66274 +66275 POINT(29.365253628886133 -96.23398731250792) bank66275 +66276 POINT(30.56035222745547 -95.76319344678546) bank66276 +66277 POINT(30.537079706944454 -95.00101774962697) bank66277 +66278 POINT(30.462067235092704 -96.36024958925107) bank66278 +66279 POINT(29.195555546755603 -95.85565430400516) bank66279 +66280 POINT(28.841303681828315 -95.20504531212516) bank66280 +66281 POINT(29.102316636200985 -95.99962993755496) bank66281 +66282 POINT(29.448036206605483 -95.43521545439111) bank66282 +66283 POINT(29.823864065297506 -95.62991134692626) bank66283 +66284 POINT(29.574174399298336 -94.73615788572914) bank66284 +66285 POINT(29.01003982414423 -94.55905321467932) bank66285 +66286 POINT(29.79100053927592 -94.63377513743124) bank66286 +66287 POINT(30.23328951242992 -95.14416304526897) bank66287 +66288 POINT(30.440821380328217 -96.06707284133655) bank66288 +66289 POINT(28.938477542761994 -96.17369187450998) bank66289 +66290 POINT(30.452782355210044 -95.96504013868831) bank66290 +66291 POINT(29.937866620627556 -96.08034865140624) bank66291 +66292 POINT(30.594207354152093 -94.38209910649294) bank66292 +66293 POINT(30.373717105088403 -94.46111017517178) bank66293 +66294 POINT(30.096124686333397 -96.2293100213738) bank66294 +66295 POINT(30.426111940567566 -95.09535038455223) bank66295 +66296 POINT(29.576058811500587 -95.0682356865382) bank66296 +66297 POINT(30.066058261526745 -94.37105655882542) bank66297 +66298 POINT(30.55740843339087 -94.88508299991997) bank66298 +66299 POINT(29.474635831380276 -95.59307645410787) bank66299 +66300 POINT(29.714672562593126 -95.41314874143457) bank66300 +66301 POINT(30.579634832046185 -96.26729969729989) bank66301 +66302 POINT(29.0834559778106 -95.05875314603377) bank66302 +66303 POINT(29.088736923131574 -96.21238241501041) bank66303 +66304 POINT(30.287714709720206 -94.53525626241876) bank66304 +66305 POINT(29.889127610100342 -94.80444052849542) bank66305 +66306 POINT(30.200882713893197 -95.51178596760298) bank66306 +66307 POINT(30.020293787006416 -95.14260768841218) bank66307 +66308 POINT(29.99859277829295 -95.90713738184857) bank66308 +66309 POINT(30.519241949902444 -96.30485810860822) bank66309 +66310 POINT(30.183626983068276 -95.28698189305643) bank66310 +66311 POINT(28.8891120666976 -94.91432091789474) bank66311 +66312 POINT(29.014464817296883 -94.92647386028956) bank66312 +66313 POINT(29.545104373030025 -95.50231583615394) bank66313 +66314 POINT(28.967348923729723 -95.62499008180467) bank66314 +66315 POINT(29.433160010864718 -95.58927483557878) bank66315 +66316 POINT(29.694514770824625 -95.85580919644735) bank66316 +66317 POINT(29.218782023148457 -95.33803688425505) bank66317 +66318 POINT(29.33045582581996 -95.51480322561315) bank66318 +66319 POINT(29.304167912704365 -95.45360152026954) bank66319 +66320 POINT(28.995134383211578 -94.93979660907752) bank66320 +66321 POINT(30.553899068008043 -95.68688087095617) bank66321 +66322 POINT(29.78882761870653 -95.86609095453834) bank66322 +66323 POINT(30.518898568180933 -94.4798674712974) bank66323 +66324 POINT(29.369052784923095 -96.20458057328592) bank66324 +66325 POINT(29.786704849818655 -96.13064373380156) bank66325 +66326 POINT(29.936984991937038 -94.48332458503224) bank66326 +66327 POINT(30.684096585568113 -95.70804881652286) bank66327 +66328 POINT(30.535673206990726 -96.31639255262264) bank66328 +66329 POINT(30.469654317225974 -94.91396221030004) bank66329 +66330 POINT(29.816673887051955 -94.58705610243946) bank66330 +66331 POINT(30.184425408433192 -94.42486734509819) bank66331 +66332 POINT(30.17509975594427 -95.40396955203354) bank66332 +66333 POINT(28.992534214973844 -94.47077251003981) bank66333 +66334 POINT(30.493642816767217 -95.53572701087195) bank66334 +66335 POINT(28.883010169466402 -95.71496962081446) bank66335 +66336 POINT(30.34686593488606 -96.22337801341631) bank66336 +66337 POINT(29.78135975681809 -95.68457479115396) bank66337 +66338 POINT(30.456062033217574 -96.34281706394711) bank66338 +66339 POINT(29.2393744323824 -94.7011972527617) bank66339 +66340 POINT(29.759787789804125 -96.29998021857003) bank66340 +66341 POINT(30.359721317270182 -96.06513432896492) bank66341 +66342 POINT(29.18584440217807 -95.17984968359502) bank66342 +66343 POINT(29.05560677028289 -95.30162893332574) bank66343 +66344 POINT(30.13372538312462 -94.89508590462881) bank66344 +66345 POINT(29.480825298917825 -95.80936472685883) bank66345 +66346 POINT(29.081875461883502 -95.54993067493139) bank66346 +66347 POINT(29.47822992105603 -95.68409877847652) bank66347 +66348 POINT(29.87429831820074 -94.45291212050962) bank66348 +66349 POINT(30.542027489078844 -94.39365608510884) bank66349 +66350 POINT(30.285413739012384 -95.217984255381) bank66350 +66351 POINT(30.09484100757937 -94.42093095501284) bank66351 +66352 POINT(30.61990561016954 -94.37634841915737) bank66352 +66353 POINT(29.546197623924975 -95.40227459978006) bank66353 +66354 POINT(28.862588321980596 -94.6618361031174) bank66354 +66355 POINT(30.47772738850043 -94.70428590281402) bank66355 +66356 POINT(28.905122712656677 -94.98701914174625) bank66356 +66357 POINT(30.486809338702706 -95.27342730625318) bank66357 +66358 POINT(29.804272491775713 -95.89360214672894) bank66358 +66359 POINT(29.84426290273606 -94.88920654027484) bank66359 +66360 POINT(29.817841715070085 -95.45136397198738) bank66360 +66361 POINT(28.791050305902555 -95.15363088307642) bank66361 +66362 POINT(29.43191311554285 -94.41886988157891) bank66362 +66363 POINT(30.215490002534622 -96.0980820489559) bank66363 +66364 POINT(29.255939257731104 -94.38415316991427) bank66364 +66365 POINT(30.44722746916824 -96.1312264628605) bank66365 +66366 POINT(29.055050664629437 -95.88385315040104) bank66366 +66367 POINT(30.533770569443753 -96.16706922147219) bank66367 +66368 POINT(28.763406164238017 -94.84808837665696) bank66368 +66369 POINT(28.90366988903662 -94.67870020717525) bank66369 +66370 POINT(30.326626852507918 -96.12723156428709) bank66370 +66371 POINT(29.958423843878883 -95.099847681784) bank66371 +66372 POINT(29.150246585768773 -94.54156691270147) bank66372 +66373 POINT(29.90014265963613 -95.73920553569656) bank66373 +66374 POINT(29.705081076108925 -95.69434045620295) bank66374 +66375 POINT(29.821131738369562 -95.69779957712569) bank66375 +66376 POINT(29.277171582789276 -95.69631606679377) bank66376 +66377 POINT(30.311717667322803 -94.78035744990699) bank66377 +66378 POINT(29.80270696877654 -95.526145435938) bank66378 +66379 POINT(30.715296110581065 -95.63291917014226) bank66379 +66380 POINT(29.148085592267655 -96.20452560184454) bank66380 +66381 POINT(30.728849368744047 -94.67044085206221) bank66381 +66382 POINT(30.114616900539673 -96.20557982614613) bank66382 +66383 POINT(30.709611149907364 -96.13568626193084) bank66383 +66384 POINT(29.111957291635118 -94.52199111588789) bank66384 +66385 POINT(30.170449870513607 -95.1603953111632) bank66385 +66386 POINT(30.63144159574724 -95.85370518979924) bank66386 +66387 POINT(29.823235103501503 -95.08585921501674) bank66387 +66388 POINT(29.904590314420645 -94.57205599719272) bank66388 +66389 POINT(29.67785206768845 -96.21991327559023) bank66389 +66390 POINT(29.415785872927195 -96.32747179811504) bank66390 +66391 POINT(29.053028233842156 -96.10001491461168) bank66391 +66392 POINT(30.576360385642488 -95.03584317633693) bank66392 +66393 POINT(29.85012546300254 -94.87267774480122) bank66393 +66394 POINT(29.890278235775366 -94.49817693492523) bank66394 +66395 POINT(30.218319572007026 -94.73160814458664) bank66395 +66396 POINT(28.94478063948971 -95.29929864307057) bank66396 +66397 POINT(29.256329686023754 -96.04871651077634) bank66397 +66398 POINT(29.644080511162365 -96.07989887376311) bank66398 +66399 POINT(29.56785230726574 -94.42490262163957) bank66399 +66400 POINT(30.400617558298634 -94.57926035304817) bank66400 +66401 POINT(30.59799475582313 -95.51748155553467) bank66401 +66402 POINT(29.454867262552636 -95.3996631516878) bank66402 +66403 POINT(29.746597556222866 -96.31485689214496) bank66403 +66404 POINT(29.087031681739482 -95.0936865125078) bank66404 +66405 POINT(30.35891015835018 -94.75357800334837) bank66405 +66406 POINT(30.219377417036235 -95.5400786543056) bank66406 +66407 POINT(29.56349819229624 -96.20822009714028) bank66407 +66408 POINT(30.131649127336043 -94.64981221201089) bank66408 +66409 POINT(28.826538291446496 -94.9139032111945) bank66409 +66410 POINT(29.38392328948876 -95.44297412148515) bank66410 +66411 POINT(29.53408730248849 -95.54392629336485) bank66411 +66412 POINT(30.32381795512142 -95.07030626896928) bank66412 +66413 POINT(29.3774049783787 -95.31956527687892) bank66413 +66414 POINT(29.07058112245581 -94.92123236370881) bank66414 +66415 POINT(30.4733601515533 -95.12018484308922) bank66415 +66416 POINT(30.229064494935663 -95.36661503758214) bank66416 +66417 POINT(29.395848312490514 -96.2559209117427) bank66417 +66418 POINT(30.549467744502987 -94.64258539543792) bank66418 +66419 POINT(28.918926092914173 -95.78331747014185) bank66419 +66420 POINT(29.804169824479427 -95.72450963630502) bank66420 +66421 POINT(29.762805265945296 -94.97220312522073) bank66421 +66422 POINT(28.823734690036606 -94.56249560481677) bank66422 +66423 POINT(30.07269113598496 -95.70289900072282) bank66423 +66424 POINT(30.72387887571225 -96.04909394435481) bank66424 +66425 POINT(30.084811808809256 -95.02921015907614) bank66425 +66426 POINT(29.927521087237444 -94.5587628897816) bank66426 +66427 POINT(29.822125716283125 -95.28258839420693) bank66427 +66428 POINT(30.583127304158094 -96.2188841835687) bank66428 +66429 POINT(29.277066650727924 -95.69326426248973) bank66429 +66430 POINT(28.871011368734496 -95.168833037627) bank66430 +66431 POINT(30.076386651889724 -94.62402711633709) bank66431 +66432 POINT(29.52831789918069 -96.06077391451832) bank66432 +66433 POINT(29.379325512938777 -95.10435063821441) bank66433 +66434 POINT(30.687995342929707 -96.0017710425389) bank66434 +66435 POINT(30.134672875939398 -95.49896716656875) bank66435 +66436 POINT(29.003552614007855 -95.89023127086092) bank66436 +66437 POINT(29.357307711318796 -94.87056268080619) bank66437 +66438 POINT(29.413215228924425 -95.28170293447299) bank66438 +66439 POINT(29.054717062487697 -95.18688677620737) bank66439 +66440 POINT(30.734309830029144 -95.92461376770385) bank66440 +66441 POINT(29.205627788316736 -95.94077371648964) bank66441 +66442 POINT(29.03943141134964 -94.88086056203362) bank66442 +66443 POINT(29.568579922109567 -95.62109727387904) bank66443 +66444 POINT(30.52121831788409 -95.16715386157287) bank66444 +66445 POINT(29.73516604015526 -94.85920552696517) bank66445 +66446 POINT(30.165750844440517 -94.46479331698181) bank66446 +66447 POINT(30.156461423591768 -94.98779477200235) bank66447 +66448 POINT(30.735637628247222 -95.44873002566209) bank66448 +66449 POINT(29.475736920273654 -96.3094767550194) bank66449 +66450 POINT(30.163671432639234 -95.67528696647899) bank66450 +66451 POINT(29.230898425704645 -94.55191617518479) bank66451 +66452 POINT(28.98010941404945 -95.0767856667942) bank66452 +66453 POINT(29.641988762390852 -95.3180270152943) bank66453 +66454 POINT(28.81599062441384 -95.56923923329575) bank66454 +66455 POINT(29.10818875158946 -96.11094809211126) bank66455 +66456 POINT(28.933820463635634 -96.05000358224642) bank66456 +66457 POINT(30.100430050365283 -95.05317934877168) bank66457 +66458 POINT(30.130899666827215 -94.63684792730733) bank66458 +66459 POINT(29.629221385324943 -96.28743783743185) bank66459 +66460 POINT(28.990358688683987 -95.509171724338) bank66460 +66461 POINT(30.565098121336806 -96.1705994051318) bank66461 +66462 POINT(29.136857601062594 -95.65556280449499) bank66462 +66463 POINT(29.247706703817453 -95.31784701923642) bank66463 +66464 POINT(28.969695332089987 -94.70042849037482) bank66464 +66465 POINT(29.434583279935417 -94.38724595057332) bank66465 +66466 POINT(29.366886285175607 -95.64173761258716) bank66466 +66467 POINT(29.773381411021692 -95.675921409401) bank66467 +66468 POINT(28.916757295742947 -94.46591540756813) bank66468 +66469 POINT(30.30657351713921 -95.76317299371503) bank66469 +66470 POINT(29.79398153018657 -95.8814716659329) bank66470 +66471 POINT(28.89258513483569 -95.55786974906495) bank66471 +66472 POINT(28.991885148010063 -94.69675759077309) bank66472 +66473 POINT(29.98933040947849 -95.92683182859123) bank66473 +66474 POINT(29.43720666761376 -95.5062781388709) bank66474 +66475 POINT(29.23369748043256 -94.40664354311689) bank66475 +66476 POINT(30.23305572024095 -95.98539655340059) bank66476 +66477 POINT(29.149980122007765 -94.82024715832841) bank66477 +66478 POINT(30.453095456707914 -96.34609817950385) bank66478 +66479 POINT(30.723917635647133 -95.67831723779624) bank66479 +66480 POINT(30.15519624394931 -95.30038778341381) bank66480 +66481 POINT(30.538840890926977 -95.05091997885312) bank66481 +66482 POINT(30.506073319107333 -95.64816349325451) bank66482 +66483 POINT(30.63596465840754 -95.19524026589197) bank66483 +66484 POINT(29.945946242714523 -95.3893944969179) bank66484 +66485 POINT(28.937989171394964 -95.01499180160481) bank66485 +66486 POINT(29.3967807782209 -96.04579049614435) bank66486 +66487 POINT(30.28861166860413 -95.38977644074573) bank66487 +66488 POINT(30.470264866616528 -94.95231513383034) bank66488 +66489 POINT(29.881841565827408 -95.07332843555992) bank66489 +66490 POINT(28.977189549013776 -95.25521360672204) bank66490 +66491 POINT(30.605189504570223 -95.69963610518582) bank66491 +66492 POINT(29.67188876738447 -95.80838856028176) bank66492 +66493 POINT(29.56665633507929 -95.9276536764088) bank66493 +66494 POINT(30.2111874238747 -95.61274617276828) bank66494 +66495 POINT(30.192087520844016 -95.82404248881835) bank66495 +66496 POINT(29.236863536850983 -95.70064015896465) bank66496 +66497 POINT(30.578293207438836 -95.78970516981693) bank66497 +66498 POINT(29.3518019219184 -95.67884912792128) bank66498 +66499 POINT(29.0403021701204 -94.6821501329557) bank66499 +66500 POINT(30.288873644179148 -95.81680121279952) bank66500 +66501 POINT(30.386963819734493 -94.70246164597026) bank66501 +66502 POINT(30.62921848499645 -95.46992277593046) bank66502 +66503 POINT(29.339898832270784 -95.73770238993049) bank66503 +66504 POINT(29.82117459272164 -95.09631056430726) bank66504 +66505 POINT(29.61866433502349 -96.10401874375778) bank66505 +66506 POINT(29.500906758723048 -94.75976755910696) bank66506 +66507 POINT(29.189696143566145 -94.4422969594176) bank66507 +66508 POINT(29.42663527931842 -94.8678433666632) bank66508 +66509 POINT(30.677472651333026 -95.99628207843503) bank66509 +66510 POINT(30.470804072194362 -96.09985400184048) bank66510 +66511 POINT(29.883338773953902 -95.94706309287031) bank66511 +66512 POINT(29.87640770328696 -95.05570869034301) bank66512 +66513 POINT(30.228804547821053 -96.00279621772879) bank66513 +66514 POINT(29.295151443420014 -96.24194317290066) bank66514 +66515 POINT(30.487252627604793 -95.69322551117637) bank66515 +66516 POINT(29.27699787162284 -94.95688050953757) bank66516 +66517 POINT(29.743495201311745 -96.05902743153737) bank66517 +66518 POINT(30.062130512136573 -94.98093501784908) bank66518 +66519 POINT(29.38109087215387 -95.4413026251897) bank66519 +66520 POINT(30.62440514601425 -95.425392227582) bank66520 +66521 POINT(30.618704259413935 -94.44544710170908) bank66521 +66522 POINT(30.277587984465058 -94.38131847021896) bank66522 +66523 POINT(30.68218504419776 -94.54908112110316) bank66523 +66524 POINT(28.893073552266753 -94.79443438292725) bank66524 +66525 POINT(30.400233166550695 -94.82708964187896) bank66525 +66526 POINT(29.14404715603094 -96.25002804966861) bank66526 +66527 POINT(29.06078699672639 -95.27143290815955) bank66527 +66528 POINT(28.872282547121983 -96.2237841816932) bank66528 +66529 POINT(29.525489007749893 -95.67915697211214) bank66529 +66530 POINT(29.50422185976188 -96.22619802185912) bank66530 +66531 POINT(29.304579535586978 -95.86700488420126) bank66531 +66532 POINT(29.393079663141446 -95.83511855497825) bank66532 +66533 POINT(30.684144245598894 -96.1531439858936) bank66533 +66534 POINT(29.349447030001105 -95.8162826491843) bank66534 +66535 POINT(29.24348503277139 -95.71269659324271) bank66535 +66536 POINT(29.346167415132026 -94.46575060332127) bank66536 +66537 POINT(30.051829853691757 -94.7020749280749) bank66537 +66538 POINT(29.74848327783669 -96.25063803092037) bank66538 +66539 POINT(29.14632290123808 -96.18638094806495) bank66539 +66540 POINT(29.86028333258798 -95.21852578819188) bank66540 +66541 POINT(30.299841500861913 -95.75355919512134) bank66541 +66542 POINT(30.136185553440065 -96.05088358831887) bank66542 +66543 POINT(28.84439067159525 -94.57169669631188) bank66543 +66544 POINT(29.744616968059066 -95.13846600449892) bank66544 +66545 POINT(28.806849395312398 -95.31024871967685) bank66545 +66546 POINT(28.98199735424448 -96.24916475636701) bank66546 +66547 POINT(30.407147122854692 -95.8175499479888) bank66547 +66548 POINT(29.4429000304879 -94.71971773950789) bank66548 +66549 POINT(30.549194184858806 -94.46488437245772) bank66549 +66550 POINT(29.67919668575221 -96.12603201961788) bank66550 +66551 POINT(30.29344385840159 -96.13140259631207) bank66551 +66552 POINT(28.768849677994 -95.11844158741583) bank66552 +66553 POINT(30.307441903191922 -95.0111845094974) bank66553 +66554 POINT(29.4581566360275 -95.53166263853778) bank66554 +66555 POINT(30.21329743943265 -95.80139362805808) bank66555 +66556 POINT(29.236740957761096 -96.33509300955917) bank66556 +66557 POINT(29.152467669098375 -95.11310433877352) bank66557 +66558 POINT(29.97908383729151 -94.39272689711295) bank66558 +66559 POINT(30.118005594173187 -95.3671493832356) bank66559 +66560 POINT(30.051708664396173 -95.7268593564417) bank66560 +66561 POINT(29.106449958865444 -94.70320336694184) bank66561 +66562 POINT(29.790185088345165 -96.27908578616628) bank66562 +66563 POINT(29.89167222052877 -94.42703613533823) bank66563 +66564 POINT(29.681964219337154 -95.13426765379175) bank66564 +66565 POINT(30.67331670223127 -94.45906227975442) bank66565 +66566 POINT(30.5688553899009 -94.61896112547994) bank66566 +66567 POINT(30.451191490514468 -95.46209004262474) bank66567 +66568 POINT(30.603342935624053 -94.64242839057974) bank66568 +66569 POINT(30.08107137485092 -95.96836768569727) bank66569 +66570 POINT(29.361424982662072 -95.87163813870835) bank66570 +66571 POINT(29.73080928416637 -95.10926738398993) bank66571 +66572 POINT(29.463756542403836 -96.16267620300533) bank66572 +66573 POINT(29.87699625356551 -96.10733723714816) bank66573 +66574 POINT(29.84683065759854 -95.76028199642148) bank66574 +66575 POINT(30.254322406577568 -95.63663324032322) bank66575 +66576 POINT(30.2180939583753 -95.45291730968329) bank66576 +66577 POINT(28.891875858311984 -95.82768729189394) bank66577 +66578 POINT(29.35246129522786 -95.07784424208278) bank66578 +66579 POINT(30.51846931490597 -95.72795502985345) bank66579 +66580 POINT(29.192434417648254 -96.1482357261412) bank66580 +66581 POINT(30.169728341682283 -95.98885897342807) bank66581 +66582 POINT(30.466856771204785 -95.02181382345738) bank66582 +66583 POINT(29.566308399514465 -95.9350521018513) bank66583 +66584 POINT(30.701377209225907 -96.14443328162832) bank66584 +66585 POINT(29.070560165880636 -96.30955783842464) bank66585 +66586 POINT(28.796723999955077 -96.08661639183376) bank66586 +66587 POINT(29.805629177115225 -95.43369781476807) bank66587 +66588 POINT(30.636988848394704 -96.09067634584109) bank66588 +66589 POINT(30.403318115713592 -96.3644337953349) bank66589 +66590 POINT(29.594539043016653 -94.56868327233406) bank66590 +66591 POINT(28.823029359632418 -94.43339193906783) bank66591 +66592 POINT(30.30081959648134 -95.14401758108693) bank66592 +66593 POINT(30.639719362918804 -95.47308452185953) bank66593 +66594 POINT(30.616547416443016 -95.89819191639644) bank66594 +66595 POINT(29.98073493927454 -96.28831928250966) bank66595 +66596 POINT(29.245379117814256 -95.14126630923907) bank66596 +66597 POINT(30.66180030789672 -95.4872867377186) bank66597 +66598 POINT(29.916684332246234 -95.87763984580262) bank66598 +66599 POINT(28.877878683731847 -96.33313379901901) bank66599 +66600 POINT(30.102232809718263 -96.25929428524434) bank66600 +66601 POINT(29.485886610496113 -96.03125040166996) bank66601 +66602 POINT(29.42274104498472 -94.91953723486233) bank66602 +66603 POINT(30.395239691759027 -95.45426322893488) bank66603 +66604 POINT(29.074482962990974 -95.3691052153512) bank66604 +66605 POINT(30.325917706766795 -96.1564603725735) bank66605 +66606 POINT(28.778937337539087 -96.29257821479482) bank66606 +66607 POINT(30.276763570764682 -96.08572893155294) bank66607 +66608 POINT(30.05195430176413 -95.57497655237171) bank66608 +66609 POINT(29.700449705551353 -96.29723528251964) bank66609 +66610 POINT(29.644090817241533 -95.91320201556965) bank66610 +66611 POINT(29.16380863634293 -95.44643165050702) bank66611 +66612 POINT(30.098553271926733 -96.02736380722482) bank66612 +66613 POINT(28.929014979209253 -95.99231095777782) bank66613 +66614 POINT(29.076894027417147 -95.62306140325782) bank66614 +66615 POINT(30.219355932257425 -95.85115811604524) bank66615 +66616 POINT(30.023905922280875 -95.78077889599236) bank66616 +66617 POINT(29.64550039682142 -95.64509149883955) bank66617 +66618 POINT(30.557839326396543 -94.39198712701904) bank66618 +66619 POINT(30.433269805726514 -96.26778807205223) bank66619 +66620 POINT(30.388985308078183 -94.84235189918813) bank66620 +66621 POINT(30.30486224668789 -95.95488472681907) bank66621 +66622 POINT(29.249445635107378 -95.37393017060728) bank66622 +66623 POINT(29.879335768194515 -94.89547981113392) bank66623 +66624 POINT(29.017752349342327 -94.66856986596086) bank66624 +66625 POINT(30.435559507180344 -96.0873556718617) bank66625 +66626 POINT(29.955295345778303 -96.2793294493985) bank66626 +66627 POINT(30.255802017631503 -94.40423715801802) bank66627 +66628 POINT(29.836766581335095 -94.66598640163484) bank66628 +66629 POINT(29.593502609603963 -95.46969646183334) bank66629 +66630 POINT(29.435624389244428 -94.54873358450821) bank66630 +66631 POINT(30.103508698758375 -95.36515749392797) bank66631 +66632 POINT(28.952950227437093 -94.56046149087115) bank66632 +66633 POINT(29.880695924832494 -96.06587594734516) bank66633 +66634 POINT(28.847768391643175 -95.17017192669525) bank66634 +66635 POINT(29.32027567003899 -95.37623870297762) bank66635 +66636 POINT(30.171249360119607 -96.2729886813571) bank66636 +66637 POINT(29.641237161204728 -94.6545665931938) bank66637 +66638 POINT(30.156213806896314 -95.26013117187786) bank66638 +66639 POINT(30.479541002074633 -95.02142771094091) bank66639 +66640 POINT(30.654050043784668 -94.78561724518674) bank66640 +66641 POINT(30.311913631728626 -94.57084491958815) bank66641 +66642 POINT(30.41679455716463 -96.28146044431486) bank66642 +66643 POINT(29.67847093723335 -95.44350045336081) bank66643 +66644 POINT(28.771664033065015 -96.35308242340942) bank66644 +66645 POINT(30.53917788442702 -96.00451963238275) bank66645 +66646 POINT(29.885841534364708 -95.34075009978756) bank66646 +66647 POINT(29.116531197016478 -94.6633101107676) bank66647 +66648 POINT(30.497037877481215 -96.08063746336703) bank66648 +66649 POINT(29.611296750789677 -95.65015818529426) bank66649 +66650 POINT(28.771860570363067 -95.65567443748867) bank66650 +66651 POINT(29.407518330428665 -94.5307629080004) bank66651 +66652 POINT(29.107471424016204 -96.04889994352891) bank66652 +66653 POINT(29.096957228069066 -95.3411356396584) bank66653 +66654 POINT(29.659664223252516 -95.89584353561135) bank66654 +66655 POINT(29.676411652245758 -95.87309526525749) bank66655 +66656 POINT(29.089793751007164 -95.52856653506613) bank66656 +66657 POINT(30.475837590636758 -95.00440148952252) bank66657 +66658 POINT(28.785780277112522 -95.95683172129299) bank66658 +66659 POINT(29.77089526205606 -95.49800397587842) bank66659 +66660 POINT(29.10828560071435 -95.43328547309535) bank66660 +66661 POINT(30.29951027165821 -95.81729085629901) bank66661 +66662 POINT(28.89297292056227 -95.1491253003419) bank66662 +66663 POINT(29.93343954094147 -95.7821103142019) bank66663 +66664 POINT(29.750979989618923 -95.11988979790976) bank66664 +66665 POINT(30.327646341174177 -94.39492360045747) bank66665 +66666 POINT(30.50051692141781 -95.67843878899367) bank66666 +66667 POINT(30.38461360782595 -96.35578365619925) bank66667 +66668 POINT(29.506401485334504 -94.6134544484151) bank66668 +66669 POINT(28.93875987910992 -95.40129372416146) bank66669 +66670 POINT(30.41860908124089 -94.58231058048639) bank66670 +66671 POINT(30.122419719184588 -94.97680766048794) bank66671 +66672 POINT(29.529848033146852 -95.9395973322758) bank66672 +66673 POINT(30.731312411443902 -95.90907118297723) bank66673 +66674 POINT(30.0591230685143 -95.83611327593181) bank66674 +66675 POINT(28.868647507861482 -95.83213445600067) bank66675 +66676 POINT(30.142092754007706 -94.53235218860968) bank66676 +66677 POINT(29.214689896169414 -95.68968929408268) bank66677 +66678 POINT(29.240805267866445 -94.81365161408233) bank66678 +66679 POINT(29.068831021715194 -96.33970406114265) bank66679 +66680 POINT(40.72729467548141 -74.03680498681032) bank66680 +66681 POINT(41.2846015023659 -74.53313333989244) bank66681 +66682 POINT(40.2058849036842 -74.88849204996023) bank66682 +66683 POINT(41.61392919968612 -73.23047932798077) bank66683 +66684 POINT(40.62917085765427 -73.04419298778492) bank66684 +66685 POINT(40.67219213124521 -74.90073462838636) bank66685 +66686 POINT(41.368606467979276 -74.95256141792683) bank66686 +66687 POINT(39.77837193378219 -74.47917558630168) bank66687 +66688 POINT(40.232195914941656 -74.3337197810911) bank66688 +66689 POINT(40.49150388126443 -73.09960618639262) bank66689 +66690 POINT(40.65706267722217 -73.92443267561734) bank66690 +66691 POINT(41.41882456747093 -74.4609864437783) bank66691 +66692 POINT(41.25531496541492 -74.6000307684521) bank66692 +66693 POINT(41.019658168282724 -73.20959220749754) bank66693 +66694 POINT(40.209561513725156 -74.59488224401072) bank66694 +66695 POINT(40.404150004241444 -73.17042705736998) bank66695 +66696 POINT(40.115410719101234 -74.94364051066574) bank66696 +66697 POINT(40.39883182580009 -74.17169156679492) bank66697 +66698 POINT(40.79444648834414 -73.29391597471316) bank66698 +66699 POINT(40.6370271999289 -74.44167746624368) bank66699 +66700 POINT(40.955940489391786 -73.47886746246866) bank66700 +66701 POINT(40.03442279083154 -74.0952448590834) bank66701 +66702 POINT(41.32030271234697 -74.94044839040353) bank66702 +66703 POINT(40.470523990829896 -73.06200835002731) bank66703 +66704 POINT(41.421018262976716 -74.88976146542115) bank66704 +66705 POINT(41.161708734721174 -75.00173881310165) bank66705 +66706 POINT(40.11248282365803 -73.28522549640087) bank66706 +66707 POINT(40.17330470336199 -73.7148065314229) bank66707 +66708 POINT(39.75581392795097 -74.33143285288281) bank66708 +66709 POINT(40.99724199754919 -73.36934501942899) bank66709 +66710 POINT(40.05318995925466 -74.54570214173974) bank66710 +66711 POINT(40.52581044244416 -73.42315381499277) bank66711 +66712 POINT(39.71554185174983 -73.7537060288826) bank66712 +66713 POINT(40.89178547885592 -73.8055625867039) bank66713 +66714 POINT(40.98453362158073 -74.01548991646231) bank66714 +66715 POINT(41.117438578687505 -73.42861115669668) bank66715 +66716 POINT(41.20183836348869 -74.02097257213232) bank66716 +66717 POINT(40.13271647952091 -73.92768116866382) bank66717 +66718 POINT(41.45288403947539 -74.81187946059593) bank66718 +66719 POINT(41.05178657571784 -73.13224950556922) bank66719 +66720 POINT(40.32517640006657 -73.34499066096843) bank66720 +66721 POINT(41.14845808605416 -73.23181312321658) bank66721 +66722 POINT(40.24214366836843 -74.73194142907595) bank66722 +66723 POINT(40.89679618476074 -74.2627967992853) bank66723 +66724 POINT(41.105060631935196 -73.37890711073345) bank66724 +66725 POINT(40.27646663886434 -74.58569976552565) bank66725 +66726 POINT(41.23422788419779 -73.17865000565254) bank66726 +66727 POINT(40.38823908457728 -73.53337859835355) bank66727 +66728 POINT(41.02158755631987 -74.8522143011231) bank66728 +66729 POINT(40.330945694465676 -74.59467857942319) bank66729 +66730 POINT(40.124561759388946 -73.85432246144943) bank66730 +66731 POINT(39.7919260951566 -74.23105768168422) bank66731 +66732 POINT(40.744846646562735 -74.23245333695249) bank66732 +66733 POINT(41.29869311617253 -74.9346850669715) bank66733 +66734 POINT(41.19347587887792 -73.35642709658272) bank66734 +66735 POINT(40.83502594196337 -73.67668587762901) bank66735 +66736 POINT(40.32930283541889 -74.51630959820385) bank66736 +66737 POINT(40.75447514525484 -74.33046597533787) bank66737 +66738 POINT(40.741476836233595 -74.39214678732951) bank66738 +66739 POINT(39.93520053581352 -74.23393809813537) bank66739 +66740 POINT(41.59278600172001 -74.25415746817613) bank66740 +66741 POINT(40.757162016285406 -73.65696991808204) bank66741 +66742 POINT(40.97085221302958 -74.09176860502208) bank66742 +66743 POINT(41.391296101221855 -73.38489357327896) bank66743 +66744 POINT(40.92697791049997 -73.9256675072797) bank66744 +66745 POINT(41.67833427135284 -74.90392159536441) bank66745 +66746 POINT(40.633787488358 -74.75398086922924) bank66746 +66747 POINT(41.47713624493885 -73.22026645645822) bank66747 +66748 POINT(39.725624373986484 -74.00392642377189) bank66748 +66749 POINT(41.62113052016071 -73.93927424512059) bank66749 +66750 POINT(40.02523055822837 -73.55396124747935) bank66750 +66751 POINT(40.32426899930072 -73.41786153414157) bank66751 +66752 POINT(41.48533240575677 -74.20102600372846) bank66752 +66753 POINT(41.12169709589352 -73.91801690279046) bank66753 +66754 POINT(41.620116177774115 -73.2129332614463) bank66754 +66755 POINT(40.551478381068904 -73.65702625310415) bank66755 +66756 POINT(39.91233898186711 -74.80968494118848) bank66756 +66757 POINT(40.197560699588934 -73.94407160253459) bank66757 +66758 POINT(40.89700319075499 -74.34041827466616) bank66758 +66759 POINT(40.791505052707606 -73.38756797691791) bank66759 +66760 POINT(40.08822551281053 -73.12368823952328) bank66760 +66761 POINT(40.71365897630021 -73.17327668313807) bank66761 +66762 POINT(40.43724829977732 -74.81304866802974) bank66762 +66763 POINT(40.192535295832634 -74.36760269344094) bank66763 +66764 POINT(41.06923104239423 -74.15359964658298) bank66764 +66765 POINT(40.611137516491944 -74.302724673714) bank66765 +66766 POINT(41.64307809532244 -74.95551551908555) bank66766 +66767 POINT(39.75648425836895 -74.80870396390263) bank66767 +66768 POINT(41.650883687042274 -73.37494445185952) bank66768 +66769 POINT(40.31168570957935 -74.76490725596277) bank66769 +66770 POINT(40.871899667234246 -74.39562775403935) bank66770 +66771 POINT(40.901688778788355 -74.32152209153182) bank66771 +66772 POINT(40.76185137428228 -73.63690638495616) bank66772 +66773 POINT(41.250120135456 -73.50601199750504) bank66773 +66774 POINT(41.655261946135944 -73.15994382043482) bank66774 +66775 POINT(41.458921499709774 -73.03090929428927) bank66775 +66776 POINT(41.03175339359126 -74.93695532593412) bank66776 +66777 POINT(39.88419101101771 -73.11538355513497) bank66777 +66778 POINT(40.52346873584452 -74.51296042291524) bank66778 +66779 POINT(41.46571889802512 -74.46863314735666) bank66779 +66780 POINT(40.51033127422587 -73.89258160289384) bank66780 +66781 POINT(39.90763593166836 -73.81577457988888) bank66781 +66782 POINT(41.46214409075436 -74.49441466772268) bank66782 +66783 POINT(40.32451309629493 -74.35716248278888) bank66783 +66784 POINT(39.71897343599099 -73.64088545038862) bank66784 +66785 POINT(40.87444774114488 -74.6865760707207) bank66785 +66786 POINT(39.83901584118975 -73.96755211263654) bank66786 +66787 POINT(40.316638479651566 -73.01979509778153) bank66787 +66788 POINT(40.53917287946128 -73.30866975805016) bank66788 +66789 POINT(41.4919655249641 -73.80147794034025) bank66789 +66790 POINT(40.16546511924278 -73.38108009500078) bank66790 +66791 POINT(41.40514790762148 -74.59925281473235) bank66791 +66792 POINT(40.45590708370908 -73.23901082346975) bank66792 +66793 POINT(39.74035733242369 -74.3422931929963) bank66793 +66794 POINT(40.157570608321166 -73.8481205775849) bank66794 +66795 POINT(40.6207006239334 -74.82914894238532) bank66795 +66796 POINT(40.13407674517741 -73.29961779573024) bank66796 +66797 POINT(40.69355531893344 -74.43695328862802) bank66797 +66798 POINT(41.27632006009413 -73.20151100210342) bank66798 +66799 POINT(41.65167863485563 -73.9771684319889) bank66799 +66800 POINT(41.440018528503096 -74.0173297896823) bank66800 +66801 POINT(40.887417991158514 -74.45754933400701) bank66801 +66802 POINT(40.64548516300396 -74.8792374325004) bank66802 +66803 POINT(40.76405744314614 -73.65591718731419) bank66803 +66804 POINT(39.94702918604584 -74.98186106715335) bank66804 +66805 POINT(40.49000813247211 -74.1723265252214) bank66805 +66806 POINT(40.09688831727198 -73.21794211909732) bank66806 +66807 POINT(39.76924503421586 -74.14914831838405) bank66807 +66808 POINT(40.23327144047712 -73.47887840154775) bank66808 +66809 POINT(40.39750220089533 -73.07290016692036) bank66809 +66810 POINT(40.72442061196141 -74.54383843815349) bank66810 +66811 POINT(40.944614180660366 -73.48535535298181) bank66811 +66812 POINT(41.21929124372939 -74.54432842529202) bank66812 +66813 POINT(40.982721990467354 -73.69782897780865) bank66813 +66814 POINT(40.48240513730923 -73.00857796346635) bank66814 +66815 POINT(40.015455118984235 -73.40820123546689) bank66815 +66816 POINT(40.211414582889816 -73.1723268223919) bank66816 +66817 POINT(41.29315889567885 -74.87064875759039) bank66817 +66818 POINT(40.93273668337591 -73.94687848821282) bank66818 +66819 POINT(40.404280208743955 -73.14771620583518) bank66819 +66820 POINT(40.301116888650334 -74.80727873871565) bank66820 +66821 POINT(39.95445318691898 -74.70087397335317) bank66821 +66822 POINT(41.41276604482103 -73.30599321605659) bank66822 +66823 POINT(40.6942024793976 -73.01902175058042) bank66823 +66824 POINT(39.95683423241389 -73.25144880058944) bank66824 +66825 POINT(39.726202919495734 -73.76571049181308) bank66825 +66826 POINT(40.56605338392462 -74.4452411085851) bank66826 +66827 POINT(40.28939808227174 -73.5305342142704) bank66827 +66828 POINT(41.541727369889315 -73.5284384531102) bank66828 +66829 POINT(41.555107770069654 -73.34805301148428) bank66829 +66830 POINT(40.78216050836592 -74.71718999976105) bank66830 +66831 POINT(41.69575910478541 -74.96817871180613) bank66831 +66832 POINT(40.0254793589849 -74.42799956760784) bank66832 +66833 POINT(39.75455436567218 -74.45924412368515) bank66833 +66834 POINT(40.36421009421633 -74.64561002541966) bank66834 +66835 POINT(40.843898115704974 -74.48436265220758) bank66835 +66836 POINT(40.403266575366324 -73.94837549234524) bank66836 +66837 POINT(41.17875101360799 -74.04547704888151) bank66837 +66838 POINT(40.07077882607394 -73.74616429259538) bank66838 +66839 POINT(41.402055464939465 -74.22403049018871) bank66839 +66840 POINT(39.82148439666695 -74.09750489791338) bank66840 +66841 POINT(40.38118983840068 -73.7873899003716) bank66841 +66842 POINT(41.098732059710215 -74.53378601032428) bank66842 +66843 POINT(41.32582761889323 -73.11484008915819) bank66843 +66844 POINT(40.535930383013145 -73.57302700952147) bank66844 +66845 POINT(40.42212054449551 -73.89519389407147) bank66845 +66846 POINT(39.92039368402982 -74.31993414040893) bank66846 +66847 POINT(39.86575382665117 -74.35106314125778) bank66847 +66848 POINT(39.99990909416602 -73.56167276099802) bank66848 +66849 POINT(41.658363279986425 -73.7837008162425) bank66849 +66850 POINT(41.2805488313617 -73.94104657169295) bank66850 +66851 POINT(41.35391335848267 -73.20380469077685) bank66851 +66852 POINT(40.52147253163714 -73.0701863113105) bank66852 +66853 POINT(41.400009037734705 -74.43656493803353) bank66853 +66854 POINT(41.263004128909174 -73.50386603252576) bank66854 +66855 POINT(40.83186739555558 -74.0015023673781) bank66855 +66856 POINT(41.59208674531364 -73.54001257580384) bank66856 +66857 POINT(39.74955959784717 -74.11293283799488) bank66857 +66858 POINT(40.36215636628715 -73.16945698571767) bank66858 +66859 POINT(41.517656770050024 -73.0354778090014) bank66859 +66860 POINT(40.0059378834801 -74.69830111821081) bank66860 +66861 POINT(40.07660367866915 -74.88836464035799) bank66861 +66862 POINT(40.530690281974714 -74.79805589268206) bank66862 +66863 POINT(40.53747634731008 -74.51938003841725) bank66863 +66864 POINT(40.3533094940413 -73.27714125158705) bank66864 +66865 POINT(41.224017400823676 -73.18986074184518) bank66865 +66866 POINT(39.74483232038674 -74.46842031862806) bank66866 +66867 POINT(41.05605850699195 -73.2781919330245) bank66867 +66868 POINT(40.3346668245314 -73.60091104031686) bank66868 +66869 POINT(40.852882468090925 -73.67544214073357) bank66869 +66870 POINT(41.32685973359251 -73.35353809895352) bank66870 +66871 POINT(41.628945313201605 -73.11032956679801) bank66871 +66872 POINT(40.70936516297787 -73.07461502397514) bank66872 +66873 POINT(40.68871363498926 -74.22277261990428) bank66873 +66874 POINT(40.47597810576326 -73.47892948339839) bank66874 +66875 POINT(40.47312325804649 -73.35500170909927) bank66875 +66876 POINT(41.428375688162625 -73.84972677094885) bank66876 +66877 POINT(41.560035552947475 -74.89828852261009) bank66877 +66878 POINT(40.69745662055692 -74.11170246988097) bank66878 +66879 POINT(40.29072831477899 -73.42422246189224) bank66879 +66880 POINT(39.99404665866361 -74.00703576082658) bank66880 +66881 POINT(40.374168871024025 -74.6734409365496) bank66881 +66882 POINT(41.68983859034747 -73.14404634289481) bank66882 +66883 POINT(40.9561104735654 -73.79178944470364) bank66883 +66884 POINT(40.33202084312229 -73.61750370267222) bank66884 +66885 POINT(41.2343780295402 -74.78142956411864) bank66885 +66886 POINT(41.535914614593075 -73.1196456958557) bank66886 +66887 POINT(39.84884645747531 -74.1499256710771) bank66887 +66888 POINT(41.2048352677386 -74.58930171254832) bank66888 +66889 POINT(40.578151439367055 -73.79622827952521) bank66889 +66890 POINT(41.32524964259282 -73.61326643249429) bank66890 +66891 POINT(41.249674423062416 -73.8024558861842) bank66891 +66892 POINT(39.7161156652129 -73.84353109131669) bank66892 +66893 POINT(41.53433126487476 -73.55737784483495) bank66893 +66894 POINT(40.6474596466494 -73.25867016958036) bank66894 +66895 POINT(39.834314434344336 -74.82680509510654) bank66895 +66896 POINT(39.81469872971667 -73.07206557864995) bank66896 +66897 POINT(41.69789178463004 -74.20702534126704) bank66897 +66898 POINT(40.834244972797514 -74.52151266505486) bank66898 +66899 POINT(39.88794471368419 -73.0863162844087) bank66899 +66900 POINT(41.2547555728717 -73.20814984356883) bank66900 +66901 POINT(39.8102324969993 -74.82009219132512) bank66901 +66902 POINT(40.913564205844395 -74.09505481494665) bank66902 +66903 POINT(40.49930613903576 -74.72852886767336) bank66903 +66904 POINT(40.721761359262665 -74.37515345075626) bank66904 +66905 POINT(40.70565657948817 -73.1058598915512) bank66905 +66906 POINT(40.27635244239262 -74.11043665441443) bank66906 +66907 POINT(41.25262712650532 -74.49611026241458) bank66907 +66908 POINT(41.159252105375515 -73.12218824116319) bank66908 +66909 POINT(41.35521422668844 -73.73515607043359) bank66909 +66910 POINT(40.47561325761114 -74.49339316425969) bank66910 +66911 POINT(40.30427214231539 -73.72743790308078) bank66911 +66912 POINT(41.638397055737585 -73.52842758137498) bank66912 +66913 POINT(41.160924510626515 -74.29714688466122) bank66913 +66914 POINT(40.979852116041556 -74.71416553302117) bank66914 +66915 POINT(40.91372818622346 -74.80337567222494) bank66915 +66916 POINT(40.52528171173013 -74.65780155762003) bank66916 +66917 POINT(41.14815462671676 -73.47145854516035) bank66917 +66918 POINT(41.55686622628385 -74.0509181085279) bank66918 +66919 POINT(40.838621603732065 -74.97288979365617) bank66919 +66920 POINT(41.581459197484584 -73.72838590653595) bank66920 +66921 POINT(39.95239021054955 -74.84830271224274) bank66921 +66922 POINT(40.153760063658154 -73.72023292590715) bank66922 +66923 POINT(39.800694162999264 -74.11903557662444) bank66923 +66924 POINT(40.71313899402311 -74.6852785794352) bank66924 +66925 POINT(41.397151194978875 -73.16920431716167) bank66925 +66926 POINT(40.31337949272914 -73.53018315832482) bank66926 +66927 POINT(41.565408818439884 -73.4158970787961) bank66927 +66928 POINT(40.52118612430589 -74.05792476846128) bank66928 +66929 POINT(39.825106004308886 -75.00514495583596) bank66929 +66930 POINT(41.658126963566424 -73.03610995659605) bank66930 +66931 POINT(40.53322878310402 -73.76178567257547) bank66931 +66932 POINT(41.16735369053911 -74.75050520965941) bank66932 +66933 POINT(40.25729429076453 -74.84480749454723) bank66933 +66934 POINT(40.25968312513315 -73.46923020696349) bank66934 +66935 POINT(39.926930405360146 -73.4827273377135) bank66935 +66936 POINT(40.84883158493761 -74.82520898234122) bank66936 +66937 POINT(40.585014782396605 -73.1441719731489) bank66937 +66938 POINT(40.87661895588659 -73.8780521133985) bank66938 +66939 POINT(40.12036261254241 -74.23894912328491) bank66939 +66940 POINT(40.29950332912203 -73.25819494020783) bank66940 +66941 POINT(39.94629164566016 -74.85338138284992) bank66941 +66942 POINT(41.54746390159355 -73.30287357703828) bank66942 +66943 POINT(40.23224434875228 -74.77139828666849) bank66943 +66944 POINT(39.93739207230372 -74.12199517752093) bank66944 +66945 POINT(40.67921228495375 -74.70626355891207) bank66945 +66946 POINT(41.26000592541326 -73.74544820681808) bank66946 +66947 POINT(40.981443335116694 -74.92946030659587) bank66947 +66948 POINT(40.32511246657138 -74.7014372718443) bank66948 +66949 POINT(39.79859932742155 -74.01679915262903) bank66949 +66950 POINT(40.24428528306579 -73.9264807973655) bank66950 +66951 POINT(39.71763574537845 -74.17979463984003) bank66951 +66952 POINT(39.81810388627813 -74.33855763694538) bank66952 +66953 POINT(40.672482461104174 -73.64231437439854) bank66953 +66954 POINT(39.92368875124402 -74.81714686522076) bank66954 +66955 POINT(41.002468836769374 -73.28258326770295) bank66955 +66956 POINT(40.62714407111211 -73.2595849103831) bank66956 +66957 POINT(41.54576082823036 -74.88435407872674) bank66957 +66958 POINT(40.76108662938455 -73.40392288388598) bank66958 +66959 POINT(40.622476357292534 -73.72660290733018) bank66959 +66960 POINT(40.47653483470716 -73.39397623585293) bank66960 +66961 POINT(40.51051119516892 -73.04955045347103) bank66961 +66962 POINT(39.95395350593385 -74.83536982641999) bank66962 +66963 POINT(40.52206552955314 -73.88405640728189) bank66963 +66964 POINT(40.290155040629855 -73.49972085118335) bank66964 +66965 POINT(40.46078216911586 -73.74655501372824) bank66965 +66966 POINT(40.67320221141246 -73.26390191440395) bank66966 +66967 POINT(40.15598342028682 -74.54983528688525) bank66967 +66968 POINT(39.841398274557484 -74.40845161814917) bank66968 +66969 POINT(40.2314615733735 -74.50483317567434) bank66969 +66970 POINT(41.07205072959618 -73.36258080841733) bank66970 +66971 POINT(39.85949350074118 -73.17233261904639) bank66971 +66972 POINT(40.129694189395835 -74.54829475834988) bank66972 +66973 POINT(40.00898310390136 -74.01396238908066) bank66973 +66974 POINT(41.09197523813838 -73.75478967810912) bank66974 +66975 POINT(40.525563881184915 -73.28303660906569) bank66975 +66976 POINT(40.53778020642967 -74.93170279753275) bank66976 +66977 POINT(39.745071941112315 -74.04501786666982) bank66977 +66978 POINT(40.345342107442626 -73.5289463389422) bank66978 +66979 POINT(39.868535312521175 -73.83437703891312) bank66979 +66980 POINT(40.83100613587459 -74.95336143534075) bank66980 +66981 POINT(41.0003845980582 -74.29813507240047) bank66981 +66982 POINT(39.93249145662135 -73.01925820380038) bank66982 +66983 POINT(41.45466836699937 -74.15037879300735) bank66983 +66984 POINT(39.8882256273049 -74.29070821033056) bank66984 +66985 POINT(39.95503002179914 -73.16420574791434) bank66985 +66986 POINT(40.51524544762264 -74.61057188805118) bank66986 +66987 POINT(41.12116274092359 -73.71995211586768) bank66987 +66988 POINT(40.194112761406615 -74.7941304312568) bank66988 +66989 POINT(40.82425183938304 -73.12465168451669) bank66989 +66990 POINT(39.951891610816595 -73.38039652033692) bank66990 +66991 POINT(41.085896354344655 -74.17753080565853) bank66991 +66992 POINT(40.66274925720051 -73.74788771116121) bank66992 +66993 POINT(41.45995768709483 -74.58950632206566) bank66993 +66994 POINT(40.83799234443994 -74.93051021526756) bank66994 +66995 POINT(40.17894212242723 -74.83671720212297) bank66995 +66996 POINT(40.44526156393079 -74.47634761778416) bank66996 +66997 POINT(40.22055715346178 -74.69666909306797) bank66997 +66998 POINT(40.76223653593938 -73.0693048470204) bank66998 +66999 POINT(40.321124630642814 -74.20875009059579) bank66999 +67000 POINT(40.89438880208009 -74.56054912729066) bank67000 +67001 POINT(41.55292163887565 -74.97392305629913) bank67001 +67002 POINT(40.75255774114412 -73.66783757364922) bank67002 +67003 POINT(40.8625976973043 -73.09402227300866) bank67003 +67004 POINT(39.80595108308656 -73.8455460180862) bank67004 +67005 POINT(40.634064020201 -74.91014337756775) bank67005 +67006 POINT(40.814697237725305 -74.37529924991172) bank67006 +67007 POINT(39.85737272844342 -74.97967032134902) bank67007 +67008 POINT(40.56467324520665 -73.1758431748997) bank67008 +67009 POINT(39.89981794215186 -74.1920867055084) bank67009 +67010 POINT(41.50006289929592 -73.61557420104417) bank67010 +67011 POINT(39.836484788899305 -74.57178347756533) bank67011 +67012 POINT(40.03163944146787 -74.88282197872718) bank67012 +67013 POINT(41.266590855090634 -74.8855164950933) bank67013 +67014 POINT(40.583069684766585 -74.64820567444826) bank67014 +67015 POINT(40.67416997451496 -74.39800098571197) bank67015 +67016 POINT(39.71572034060914 -73.99829467947254) bank67016 +67017 POINT(41.117546183987315 -73.81849126785563) bank67017 +67018 POINT(40.78127910293445 -74.5654523015719) bank67018 +67019 POINT(40.32576111624545 -73.71142122426711) bank67019 +67020 POINT(39.99531825038502 -74.54627352060531) bank67020 +67021 POINT(40.7209331911161 -74.51101721175108) bank67021 +67022 POINT(40.84355082798919 -74.27783848000999) bank67022 +67023 POINT(40.9294279097989 -74.96301906695838) bank67023 +67024 POINT(40.05530925930124 -74.88500835110284) bank67024 +67025 POINT(40.53169999176415 -73.2029697091028) bank67025 +67026 POINT(40.3400044414319 -73.53885315745345) bank67026 +67027 POINT(40.11540548299761 -73.22498053576886) bank67027 +67028 POINT(40.14951419930759 -73.88966220613604) bank67028 +67029 POINT(41.599311001567834 -73.07667640213407) bank67029 +67030 POINT(40.88081709149304 -73.13784832481335) bank67030 +67031 POINT(40.52416839418199 -74.93849601040252) bank67031 +67032 POINT(41.16443822091837 -73.71367950564884) bank67032 +67033 POINT(41.689254186775266 -73.67825330978961) bank67033 +67034 POINT(41.014451038013476 -74.33814946359496) bank67034 +67035 POINT(40.50793295603139 -73.30665510551991) bank67035 +67036 POINT(40.88949524157112 -73.22493678291097) bank67036 +67037 POINT(41.61766833660753 -74.89290224742398) bank67037 +67038 POINT(41.29061637171131 -74.7325749974592) bank67038 +67039 POINT(39.77862911065923 -74.09619924709668) bank67039 +67040 POINT(41.40441296423879 -74.98465643283772) bank67040 +67041 POINT(40.44181202982146 -74.79722875152376) bank67041 +67042 POINT(40.31209034285345 -74.85288069863489) bank67042 +67043 POINT(40.51196499882992 -73.98560835142322) bank67043 +67044 POINT(40.712641592744966 -74.0014345971682) bank67044 +67045 POINT(40.94364316194796 -73.72663517255685) bank67045 +67046 POINT(40.240139314865665 -73.35778097197733) bank67046 +67047 POINT(40.388429936720186 -73.61023092246278) bank67047 +67048 POINT(40.04552444156352 -73.36757239370542) bank67048 +67049 POINT(41.37388638264527 -74.42011677275453) bank67049 +67050 POINT(41.43606349165572 -73.39451391700634) bank67050 +67051 POINT(40.016164268659544 -73.29802011743278) bank67051 +67052 POINT(41.05285959570266 -73.56270889265788) bank67052 +67053 POINT(40.92780704217376 -74.24732876073661) bank67053 +67054 POINT(40.304994569201774 -74.78223164157153) bank67054 +67055 POINT(41.56785531500807 -74.00098699732534) bank67055 +67056 POINT(40.4562577949612 -74.24488313852355) bank67056 +67057 POINT(40.73496922841341 -74.57262499785341) bank67057 +67058 POINT(40.39831215086642 -73.02747743379959) bank67058 +67059 POINT(40.92077146870001 -74.75709933388761) bank67059 +67060 POINT(41.23684444931281 -74.41278387910516) bank67060 +67061 POINT(41.19824182963978 -73.29437530107602) bank67061 +67062 POINT(40.29852770589315 -74.21881883213123) bank67062 +67063 POINT(41.43220899947537 -73.14044321102034) bank67063 +67064 POINT(41.69801229999743 -74.35344193332928) bank67064 +67065 POINT(41.034262179143866 -74.82070532240257) bank67065 +67066 POINT(40.30590383728404 -73.18049583028177) bank67066 +67067 POINT(41.186105544568285 -73.07888937108676) bank67067 +67068 POINT(41.32229105324081 -73.32029530771526) bank67068 +67069 POINT(41.52895885905724 -74.63614950494984) bank67069 +67070 POINT(40.115994152843975 -74.26590952737119) bank67070 +67071 POINT(39.89685863299772 -73.42773908162472) bank67071 +67072 POINT(40.561726226373885 -73.92060794880904) bank67072 +67073 POINT(40.15657480589171 -73.10061774944974) bank67073 +67074 POINT(41.50874121021976 -74.42347699055705) bank67074 +67075 POINT(41.63244598840463 -73.45753000307334) bank67075 +67076 POINT(40.69026174120422 -73.32516671917845) bank67076 +67077 POINT(41.05251350071818 -74.12571189160853) bank67077 +67078 POINT(40.46894812263766 -74.98638495086935) bank67078 +67079 POINT(41.58631449271104 -74.31757876024298) bank67079 +67080 POINT(41.29459516256566 -74.0130040657658) bank67080 +67081 POINT(41.53451471368827 -74.73846866718355) bank67081 +67082 POINT(41.376876121319576 -73.16071615710429) bank67082 +67083 POINT(40.271839984897106 -74.16943500214171) bank67083 +67084 POINT(40.172375771222825 -73.97554969015852) bank67084 +67085 POINT(40.9120761699514 -74.89686519167549) bank67085 +67086 POINT(40.33219032892101 -73.73212400089659) bank67086 +67087 POINT(40.7241018463191 -74.10787596615843) bank67087 +67088 POINT(41.37113763518264 -74.7319980185785) bank67088 +67089 POINT(41.29829274107748 -73.74456421962839) bank67089 +67090 POINT(41.38017725396794 -74.40214793920235) bank67090 +67091 POINT(40.74777343317104 -74.65970028961934) bank67091 +67092 POINT(39.78165414937466 -73.86599794929278) bank67092 +67093 POINT(41.234960080082104 -73.62551119592695) bank67093 +67094 POINT(41.496790123094456 -74.74015918905427) bank67094 +67095 POINT(41.25786325997784 -73.83008461023452) bank67095 +67096 POINT(40.638989472980384 -74.34343800865516) bank67096 +67097 POINT(40.57874008758959 -73.18564598150716) bank67097 +67098 POINT(39.93789162764744 -74.04190273115024) bank67098 +67099 POINT(39.726142660798544 -73.84577986933586) bank67099 +67100 POINT(40.35788500232458 -73.95732302144334) bank67100 +67101 POINT(40.54118397742637 -74.77346345062436) bank67101 +67102 POINT(40.845106172188885 -73.38867076699545) bank67102 +67103 POINT(40.74350933599373 -74.4407120920618) bank67103 +67104 POINT(41.53883941552854 -73.72207347326238) bank67104 +67105 POINT(41.04601889376665 -74.38184133950674) bank67105 +67106 POINT(39.75618037277677 -74.84472353845541) bank67106 +67107 POINT(39.73175717178522 -73.5390865422284) bank67107 +67108 POINT(40.22227127249793 -73.27291545535599) bank67108 +67109 POINT(39.80158729075236 -73.98008436100118) bank67109 +67110 POINT(41.09613328640012 -73.90659403861581) bank67110 +67111 POINT(40.50455839312707 -73.85314124060223) bank67111 +67112 POINT(41.01158331070448 -73.43320659581342) bank67112 +67113 POINT(41.508114533935526 -73.99414877659183) bank67113 +67114 POINT(40.41698483721548 -74.27116708342464) bank67114 +67115 POINT(40.76186659463646 -74.25987573933718) bank67115 +67116 POINT(41.076149923274954 -73.80383095126301) bank67116 +67117 POINT(39.90499600514344 -73.60814674748212) bank67117 +67118 POINT(41.293130816346796 -74.13439190395604) bank67118 +67119 POINT(41.20963778584022 -73.72306803403264) bank67119 +67120 POINT(40.90008226528307 -74.17038877685245) bank67120 +67121 POINT(40.5689463291002 -74.79491644635361) bank67121 +67122 POINT(41.55733775243698 -74.14356712321842) bank67122 +67123 POINT(41.01814746399821 -73.79130192107665) bank67123 +67124 POINT(41.138670733631955 -74.35316695934134) bank67124 +67125 POINT(40.589851684671906 -74.37782996584872) bank67125 +67126 POINT(40.786759579469866 -73.34526063187033) bank67126 +67127 POINT(40.93479580710948 -74.11422724030703) bank67127 +67128 POINT(40.38685997565135 -74.52842625608433) bank67128 +67129 POINT(41.38205311425217 -74.83815119748394) bank67129 +67130 POINT(40.66601693971193 -74.36801868482988) bank67130 +67131 POINT(40.78656356759681 -73.57464153020501) bank67131 +67132 POINT(41.59135711645469 -74.41832850657737) bank67132 +67133 POINT(39.728605443837786 -73.32681838471814) bank67133 +67134 POINT(40.49352043951424 -73.00890074724494) bank67134 +67135 POINT(41.33038839299467 -73.5489052806413) bank67135 +67136 POINT(41.11596776633198 -74.11997182467064) bank67136 +67137 POINT(41.533773493139385 -73.20535219533166) bank67137 +67138 POINT(40.91272701635077 -74.78347272392615) bank67138 +67139 POINT(41.441442250754406 -74.39728916147915) bank67139 +67140 POINT(40.23637910030928 -73.67961429375767) bank67140 +67141 POINT(40.21539976805175 -74.73741661048494) bank67141 +67142 POINT(40.18025739196904 -73.95095917467675) bank67142 +67143 POINT(40.18074520468123 -73.23011604270297) bank67143 +67144 POINT(41.66480149722477 -73.64941555901582) bank67144 +67145 POINT(41.258553967809995 -73.83803769565311) bank67145 +67146 POINT(39.77635838998648 -74.45604401767353) bank67146 +67147 POINT(41.400979022410425 -74.35999721289755) bank67147 +67148 POINT(39.79974167356961 -74.81953098687282) bank67148 +67149 POINT(40.980383661085796 -74.68489840684363) bank67149 +67150 POINT(40.21179469029446 -73.416712732795) bank67150 +67151 POINT(40.31967519758433 -73.08575232025784) bank67151 +67152 POINT(41.691058050496544 -73.6038694334117) bank67152 +67153 POINT(40.749526824625946 -74.87902588544611) bank67153 +67154 POINT(40.94414371722475 -74.90367601521667) bank67154 +67155 POINT(41.60999728662699 -73.97894658800638) bank67155 +67156 POINT(41.60578830773708 -74.31083970310915) bank67156 +67157 POINT(40.0095806529934 -73.70005425107476) bank67157 +67158 POINT(40.32227200763043 -73.84318985395086) bank67158 +67159 POINT(40.77060367772089 -74.47271297222183) bank67159 +67160 POINT(39.73921674214273 -74.1763936629754) bank67160 +67161 POINT(40.24228489561253 -73.95192627759532) bank67161 +67162 POINT(40.59618848766981 -73.70537289044802) bank67162 +67163 POINT(40.44656212414891 -74.89230231134819) bank67163 +67164 POINT(41.49121559089681 -74.7211048304524) bank67164 +67165 POINT(40.31882962539638 -73.9830038421352) bank67165 +67166 POINT(41.00775723487669 -74.11929322912466) bank67166 +67167 POINT(40.70793791557112 -73.5377002243225) bank67167 +67168 POINT(41.03572556121011 -74.55699732283124) bank67168 +67169 POINT(39.94824755578 -74.03916038844066) bank67169 +67170 POINT(41.54724415662239 -73.09647417632661) bank67170 +67171 POINT(40.170696717801924 -73.56083124516006) bank67171 +67172 POINT(40.670522514767306 -74.75046253292413) bank67172 +67173 POINT(41.15514096429889 -73.14884286158927) bank67173 +67174 POINT(40.21108035633772 -74.9320871169954) bank67174 +67175 POINT(40.09254852373336 -73.13910746225439) bank67175 +67176 POINT(40.349869874217895 -73.09567943586538) bank67176 +67177 POINT(40.71813316209818 -74.13117739255424) bank67177 +67178 POINT(40.2562503671934 -73.93219758878644) bank67178 +67179 POINT(40.31171636876176 -74.56160928074546) bank67179 +67180 POINT(40.40500091689079 -73.83271758130647) bank67180 +67181 POINT(40.05979302821271 -73.23101617261669) bank67181 +67182 POINT(40.253102494736616 -74.91987732363053) bank67182 +67183 POINT(41.05385300914142 -73.46388653177942) bank67183 +67184 POINT(41.18951040757065 -74.77531525953171) bank67184 +67185 POINT(40.881855447066314 -74.57204961590075) bank67185 +67186 POINT(41.259362003712425 -73.57722323899732) bank67186 +67187 POINT(40.43111045893131 -74.72343034829385) bank67187 +67188 POINT(40.26296002803749 -74.0649583776126) bank67188 +67189 POINT(39.91210880286033 -75.00517742821913) bank67189 +67190 POINT(40.98479774589167 -74.41174906198874) bank67190 +67191 POINT(40.641707902820094 -73.57671013384483) bank67191 +67192 POINT(40.58105963052356 -73.63472468674281) bank67192 +67193 POINT(41.10325808293329 -73.15145511669692) bank67193 +67194 POINT(41.51919842662049 -73.9810966787777) bank67194 +67195 POINT(40.462029215238985 -74.66545495789074) bank67195 +67196 POINT(41.489908171615205 -73.96176633474391) bank67196 +67197 POINT(40.055486963234976 -74.52901638196103) bank67197 +67198 POINT(40.66139226930726 -74.25135427715935) bank67198 +67199 POINT(40.90633276635571 -73.37214737114866) bank67199 +67200 POINT(41.26425638783197 -74.37293492328446) bank67200 +67201 POINT(40.96452806231205 -73.51297128993698) bank67201 +67202 POINT(41.568896987556144 -74.0809564628305) bank67202 +67203 POINT(40.66398410421511 -74.17012088572493) bank67203 +67204 POINT(41.32162620539226 -74.15055852631077) bank67204 +67205 POINT(41.17809828500338 -74.31615712006365) bank67205 +67206 POINT(40.86528511311587 -73.63369022231116) bank67206 +67207 POINT(40.010194608129076 -73.07830109186125) bank67207 +67208 POINT(41.12975879233744 -74.82673085839589) bank67208 +67209 POINT(40.9482443066 -73.9425257796635) bank67209 +67210 POINT(41.387509100684504 -73.54535333682936) bank67210 +67211 POINT(40.61629242953355 -74.6656710433351) bank67211 +67212 POINT(41.52794804641447 -74.05677056628069) bank67212 +67213 POINT(40.763686357542305 -74.35314911435479) bank67213 +67214 POINT(40.566052275242164 -74.66522590103423) bank67214 +67215 POINT(39.99631942923534 -74.17030245674754) bank67215 +67216 POINT(39.93658287488679 -73.26365315660311) bank67216 +67217 POINT(40.663269624110164 -73.57825806750387) bank67217 +67218 POINT(41.53394584862087 -74.93531110847542) bank67218 +67219 POINT(40.23617536103505 -74.69251109944051) bank67219 +67220 POINT(41.43749309378413 -73.03563420019071) bank67220 +67221 POINT(39.780182010322086 -74.1617423067123) bank67221 +67222 POINT(41.41063701622547 -74.47460707070258) bank67222 +67223 POINT(41.40289454947406 -74.28465451914673) bank67223 +67224 POINT(40.3011535739757 -74.4097823671976) bank67224 +67225 POINT(40.9388594933769 -74.36983799436923) bank67225 +67226 POINT(40.41237241175062 -73.6878051320105) bank67226 +67227 POINT(40.075997556498265 -74.60428334334284) bank67227 +67228 POINT(41.15997015773942 -73.96046118972097) bank67228 +67229 POINT(40.58213255571869 -74.6033831270782) bank67229 +67230 POINT(40.21249075211564 -74.34664322790978) bank67230 +67231 POINT(40.184198241437514 -74.26100829044287) bank67231 +67232 POINT(41.36387837689482 -73.99197464142152) bank67232 +67233 POINT(41.401432266332776 -73.204025703365) bank67233 +67234 POINT(40.655536986050556 -73.68597525126016) bank67234 +67235 POINT(41.47258066271953 -73.61607459122197) bank67235 +67236 POINT(40.505187175096815 -73.59219334388342) bank67236 +67237 POINT(40.27526235791202 -74.56594643578175) bank67237 +67238 POINT(41.538736696813594 -73.14622781625793) bank67238 +67239 POINT(39.93008943827749 -74.31717508635448) bank67239 +67240 POINT(40.997387973838435 -73.25238156765387) bank67240 +67241 POINT(39.93624569922443 -74.77902548313783) bank67241 +67242 POINT(40.02180554098351 -74.13651352493206) bank67242 +67243 POINT(41.48519220911229 -74.21883015788885) bank67243 +67244 POINT(40.81060582288398 -74.46486015022451) bank67244 +67245 POINT(41.34744871790857 -74.97771942519617) bank67245 +67246 POINT(41.51392700016816 -73.33492057801875) bank67246 +67247 POINT(40.04270042867623 -74.82423068325465) bank67247 +67248 POINT(40.80465500626196 -73.92304296015386) bank67248 +67249 POINT(41.02378993865717 -73.10383755868142) bank67249 +67250 POINT(41.46985998993727 -73.70156567060364) bank67250 +67251 POINT(41.3189044228268 -74.99913131764744) bank67251 +67252 POINT(40.40214688379387 -74.63871019297643) bank67252 +67253 POINT(41.03255916303007 -73.09498566188527) bank67253 +67254 POINT(40.62551103114782 -73.7251659540842) bank67254 +67255 POINT(41.24647816307281 -73.44292603044218) bank67255 +67256 POINT(40.241737512369795 -73.02470408369562) bank67256 +67257 POINT(40.631154838209405 -74.29038721639643) bank67257 +67258 POINT(40.640946515040845 -73.04844344444449) bank67258 +67259 POINT(40.704451982240315 -74.71801342883629) bank67259 +67260 POINT(40.55665268043722 -73.5480586512465) bank67260 +67261 POINT(40.68399753111872 -73.2065856234096) bank67261 +67262 POINT(40.97995742588262 -73.24091751264304) bank67262 +67263 POINT(40.944572711496974 -73.03065667563224) bank67263 +67264 POINT(40.64699202421833 -73.36513679553966) bank67264 +67265 POINT(41.53125810861915 -73.81008967166372) bank67265 +67266 POINT(41.661470048352 -73.59483458995668) bank67266 +67267 POINT(41.17446615977858 -73.08340155025536) bank67267 +67268 POINT(40.45199902893247 -74.66841811614702) bank67268 +67269 POINT(39.8874174287471 -73.68221772136019) bank67269 +67270 POINT(41.41877257660102 -73.56580533566209) bank67270 +67271 POINT(41.04006921344599 -74.87114383391908) bank67271 +67272 POINT(40.33685993206097 -74.43172260176797) bank67272 +67273 POINT(41.58107043451726 -73.73902642580295) bank67273 +67274 POINT(41.40389069934072 -74.47087635800773) bank67274 +67275 POINT(40.28667969820843 -73.80259812447797) bank67275 +67276 POINT(40.351013870428396 -74.26337470694173) bank67276 +67277 POINT(39.750968227560904 -74.16508971010107) bank67277 +67278 POINT(40.14601726672057 -73.0757681576998) bank67278 +67279 POINT(40.381084683987154 -74.89687893897498) bank67279 +67280 POINT(40.7816443205243 -74.18527252075756) bank67280 +67281 POINT(41.083662976941206 -74.74733893540237) bank67281 +67282 POINT(40.3456376815571 -74.01344510472435) bank67282 +67283 POINT(40.69182238001763 -74.87297028241318) bank67283 +67284 POINT(40.40748620688769 -73.5544697773974) bank67284 +67285 POINT(41.27865878180845 -73.94030514120284) bank67285 +67286 POINT(39.84214490851192 -73.59531714088457) bank67286 +67287 POINT(39.75418735256842 -74.00565235286415) bank67287 +67288 POINT(40.347919974115044 -74.08132500743474) bank67288 +67289 POINT(41.26452167210835 -73.1940453766219) bank67289 +67290 POINT(41.10687079545258 -74.08465259811561) bank67290 +67291 POINT(41.06907396614486 -73.60912222154238) bank67291 +67292 POINT(41.275885650885584 -73.97943885126361) bank67292 +67293 POINT(41.425918181383786 -74.03834469014575) bank67293 +67294 POINT(41.5342241130361 -74.4944823974537) bank67294 +67295 POINT(41.471873295795284 -73.53870587968424) bank67295 +67296 POINT(41.05498916807307 -74.97584526048733) bank67296 +67297 POINT(41.56582346115188 -74.71595849524665) bank67297 +67298 POINT(40.97654574628033 -74.43369766048369) bank67298 +67299 POINT(41.2100235654763 -73.30793858339561) bank67299 +67300 POINT(40.901117918201734 -73.91273255472755) bank67300 +67301 POINT(40.70720732869767 -73.05335898844355) bank67301 +67302 POINT(41.174363475192855 -74.14858410894496) bank67302 +67303 POINT(39.98366474917024 -74.78466970921947) bank67303 +67304 POINT(41.69457289346271 -74.62697097427865) bank67304 +67305 POINT(40.85968149583099 -74.82775747229817) bank67305 +67306 POINT(41.53451862535432 -74.38926712825958) bank67306 +67307 POINT(41.65462265533255 -73.38749225363232) bank67307 +67308 POINT(40.56568373116447 -74.66011475224582) bank67308 +67309 POINT(41.35241281227742 -73.55550794663714) bank67309 +67310 POINT(41.28801395747639 -73.12119010530266) bank67310 +67311 POINT(41.554420818664894 -74.65429336912997) bank67311 +67312 POINT(39.88592237717626 -73.59843086920179) bank67312 +67313 POINT(40.57977359702001 -74.2715318109266) bank67313 +67314 POINT(41.33397979648318 -73.45261405882721) bank67314 +67315 POINT(41.36004661642633 -74.27583249929756) bank67315 +67316 POINT(39.97746109287912 -73.14377279603482) bank67316 +67317 POINT(40.866338329290066 -74.40425823793879) bank67317 +67318 POINT(40.584685912424945 -73.25769529324388) bank67318 +67319 POINT(39.929879141025836 -74.41788572243317) bank67319 +67320 POINT(40.66973402753802 -74.79863069817509) bank67320 +67321 POINT(41.54436687171675 -73.4813806281326) bank67321 +67322 POINT(41.60542128912079 -74.97306095157826) bank67322 +67323 POINT(41.54618814570063 -74.00897275602256) bank67323 +67324 POINT(40.52144147465225 -73.08385216428596) bank67324 +67325 POINT(40.12095368030223 -74.30434695112397) bank67325 +67326 POINT(41.2501818099595 -74.4110657414542) bank67326 +67327 POINT(40.822237831804635 -74.38888196750554) bank67327 +67328 POINT(39.736540523899045 -73.05840483515256) bank67328 +67329 POINT(40.68697886685333 -73.43069856101256) bank67329 +67330 POINT(39.88187192968528 -73.26077140072503) bank67330 +67331 POINT(39.842452864858785 -74.6571222276243) bank67331 +67332 POINT(40.00616661297091 -73.08325002318604) bank67332 +67333 POINT(40.05921567160235 -73.46514030259566) bank67333 +67334 POINT(40.19777984552579 -74.81287233376021) bank67334 +67335 POINT(40.77695864973165 -73.982663726988) bank67335 +67336 POINT(40.6831391909618 -74.45641586402492) bank67336 +67337 POINT(41.10221336497085 -73.35863013351299) bank67337 +67338 POINT(40.83142106356096 -73.53965360408225) bank67338 +67339 POINT(40.95233187647726 -74.84867242658632) bank67339 +67340 POINT(40.79930396813042 -74.32070351970972) bank67340 +67341 POINT(40.43387348439369 -74.84876290021438) bank67341 +67342 POINT(41.298493445939386 -74.13516923927915) bank67342 +67343 POINT(39.91571506077212 -74.13156099772235) bank67343 +67344 POINT(41.05561979475871 -74.55342950374303) bank67344 +67345 POINT(39.806643194288334 -73.25605536558948) bank67345 +67346 POINT(40.59005551841951 -74.14275343977357) bank67346 +67347 POINT(40.75802798830692 -74.17934291165277) bank67347 +67348 POINT(41.260407393342085 -73.55954239386) bank67348 +67349 POINT(40.02382081830136 -73.43973995532576) bank67349 +67350 POINT(41.09210087488913 -73.71243691869249) bank67350 +67351 POINT(40.48986753154374 -73.25597304740786) bank67351 +67352 POINT(41.412120396204166 -73.64880547310969) bank67352 +67353 POINT(39.94956200266626 -73.74021372876585) bank67353 +67354 POINT(41.15658001610434 -73.92975471708296) bank67354 +67355 POINT(41.00348457359161 -74.54603760526547) bank67355 +67356 POINT(41.020219392334624 -73.60003248077763) bank67356 +67357 POINT(41.43559468226891 -73.45734657359506) bank67357 +67358 POINT(40.15691132856888 -74.76183579637086) bank67358 +67359 POINT(40.97092227832896 -74.41487223176692) bank67359 +67360 POINT(41.07275014701489 -73.58671608765563) bank67360 +67361 POINT(40.99451911964749 -74.41186191369928) bank67361 +67362 POINT(41.48443756980513 -74.29793827975769) bank67362 +67363 POINT(41.48511618118303 -74.26672790034249) bank67363 +67364 POINT(39.97597232627749 -73.11917347577138) bank67364 +67365 POINT(39.77192394515415 -73.01328782654093) bank67365 +67366 POINT(41.51703362537015 -74.84357870069194) bank67366 +67367 POINT(40.42517130258016 -74.34516407422367) bank67367 +67368 POINT(40.16773081585737 -74.72353197178086) bank67368 +67369 POINT(40.642412718317814 -73.30262693268237) bank67369 +67370 POINT(41.17910194770089 -73.32237179738505) bank67370 +67371 POINT(40.29380341409118 -73.35906432477755) bank67371 +67372 POINT(41.01589488786859 -73.86392826361659) bank67372 +67373 POINT(40.209995564417476 -73.40633796254527) bank67373 +67374 POINT(41.5981384517764 -73.57981469405553) bank67374 +67375 POINT(40.9246250135311 -74.75935531735395) bank67375 +67376 POINT(41.34381029449679 -74.23034552477635) bank67376 +67377 POINT(40.4806412031146 -73.02215708331725) bank67377 +67378 POINT(41.05582688485867 -74.43219906254848) bank67378 +67379 POINT(40.477235568249306 -73.75942926901483) bank67379 +67380 POINT(40.7306705304153 -74.9346671708392) bank67380 +67381 POINT(41.51558406866896 -73.75759926363625) bank67381 +67382 POINT(41.49703436755649 -74.5613638171613) bank67382 +67383 POINT(40.81114741354376 -74.80162172476012) bank67383 +67384 POINT(41.63283823917425 -74.80614990553639) bank67384 +67385 POINT(40.165925122601486 -73.89796739531187) bank67385 +67386 POINT(39.87773978109255 -73.27780180396144) bank67386 +67387 POINT(41.58366226172885 -74.25151773688533) bank67387 +67388 POINT(40.96215086644548 -73.29792482676953) bank67388 +67389 POINT(39.84379411862876 -73.39176337977858) bank67389 +67390 POINT(40.214406859887646 -73.36286115140655) bank67390 +67391 POINT(41.44253222262583 -74.04758817289903) bank67391 +67392 POINT(40.124411994282674 -73.18546485245153) bank67392 +67393 POINT(40.706991946778025 -73.10682534633082) bank67393 +67394 POINT(41.540837773960085 -74.07289460601079) bank67394 +67395 POINT(40.91195057405403 -74.46018386960927) bank67395 +67396 POINT(41.16330321215931 -73.03084824091086) bank67396 +67397 POINT(39.89546311337782 -74.94212877081888) bank67397 +67398 POINT(40.73162588084823 -74.12739410043831) bank67398 +67399 POINT(40.7780491199707 -74.78642399114067) bank67399 +67400 POINT(41.236355085969066 -74.92246811521036) bank67400 +67401 POINT(40.562643811258624 -74.3650855695707) bank67401 +67402 POINT(41.15538642761117 -74.77520423895679) bank67402 +67403 POINT(40.35031636764343 -74.38770613464841) bank67403 +67404 POINT(40.480426038254514 -74.61689181756773) bank67404 +67405 POINT(40.649022920130164 -73.45362273213216) bank67405 +67406 POINT(40.018401804723595 -74.600064594583) bank67406 +67407 POINT(40.73538471192092 -73.95403929458806) bank67407 +67408 POINT(39.715251094257326 -73.76726824783148) bank67408 +67409 POINT(40.74949350597105 -73.49458926488698) bank67409 +67410 POINT(40.60353688589263 -74.76940605314437) bank67410 +67411 POINT(41.183027175553164 -74.47180301262378) bank67411 +67412 POINT(40.85694484023331 -74.90459917276718) bank67412 +67413 POINT(39.92520347497641 -73.85933897651248) bank67413 +67414 POINT(40.94227912268156 -74.55727292506438) bank67414 +67415 POINT(40.89846820041841 -74.8400909933719) bank67415 +67416 POINT(41.335350974374705 -74.90494658356498) bank67416 +67417 POINT(41.042874912376014 -73.14147732718772) bank67417 +67418 POINT(40.327847473103944 -74.08300463808261) bank67418 +67419 POINT(40.18126835693251 -74.32439199447408) bank67419 +67420 POINT(40.87807561115126 -74.2427857175648) bank67420 +67421 POINT(40.81959809151263 -73.42342150948659) bank67421 +67422 POINT(40.09448125066288 -73.53739132303797) bank67422 +67423 POINT(40.61871078334517 -74.87973927090522) bank67423 +67424 POINT(40.69089883983054 -74.58233427370295) bank67424 +67425 POINT(40.88522006915877 -73.37976350690253) bank67425 +67426 POINT(41.555532891436485 -73.42009541681638) bank67426 +67427 POINT(41.52958795323505 -74.14765870782117) bank67427 +67428 POINT(41.20140653286226 -73.71403725642851) bank67428 +67429 POINT(41.43782126159852 -73.37798583216487) bank67429 +67430 POINT(40.493744458415264 -74.94486358189634) bank67430 +67431 POINT(40.41369729424229 -74.79158346058598) bank67431 +67432 POINT(40.42070931336329 -73.9746624356084) bank67432 +67433 POINT(39.75063224611832 -74.27253406558003) bank67433 +67434 POINT(41.429250336341084 -74.57093259995862) bank67434 +67435 POINT(39.76889376357522 -74.67166670902462) bank67435 +67436 POINT(41.05467198437058 -74.5751124822888) bank67436 +67437 POINT(40.36362460341039 -73.9899779370555) bank67437 +67438 POINT(41.259778154848924 -73.40207835530067) bank67438 +67439 POINT(40.49117368070083 -73.48442111610494) bank67439 +67440 POINT(39.751980150107755 -74.27026219558441) bank67440 +67441 POINT(39.81236919447584 -73.30161396579689) bank67441 +67442 POINT(41.609912237288356 -74.86135439208935) bank67442 +67443 POINT(40.46867936481587 -73.70825844632718) bank67443 +67444 POINT(40.04604252692829 -73.64156879842702) bank67444 +67445 POINT(40.665180789559166 -73.24666755871272) bank67445 +67446 POINT(39.90879562544484 -74.91719888398315) bank67446 +67447 POINT(40.389063293639964 -73.90133460337913) bank67447 +67448 POINT(40.74081727955682 -73.18545396299027) bank67448 +67449 POINT(40.324653151627444 -74.56310241503138) bank67449 +67450 POINT(40.543030299169956 -73.45957475098415) bank67450 +67451 POINT(39.75157658179766 -73.00790622746838) bank67451 +67452 POINT(40.59546359251215 -74.96436368088095) bank67452 +67453 POINT(41.71013327789626 -74.92482705721015) bank67453 +67454 POINT(41.27316685079679 -73.96747310917522) bank67454 +67455 POINT(41.60204147972478 -73.75115413466747) bank67455 +67456 POINT(41.20635929837181 -74.95488254915954) bank67456 +67457 POINT(41.01354643594209 -73.06538815690217) bank67457 +67458 POINT(40.93456410840133 -74.78626418833603) bank67458 +67459 POINT(41.33382283204417 -74.33005070980569) bank67459 +67460 POINT(41.15093791383969 -73.32969161691999) bank67460 +67461 POINT(39.919487979666265 -74.44987448674516) bank67461 +67462 POINT(41.493773318345866 -74.17717300277975) bank67462 +67463 POINT(41.184963134580386 -73.31739809033522) bank67463 +67464 POINT(40.61041702011112 -73.85976456315355) bank67464 +67465 POINT(40.44205214738335 -74.37146055716099) bank67465 +67466 POINT(40.59305755240487 -73.42413397043849) bank67466 +67467 POINT(40.467979245936895 -73.40256771249408) bank67467 +67468 POINT(41.35301166098725 -73.46472390167624) bank67468 +67469 POINT(39.900866465368786 -73.1729678293828) bank67469 +67470 POINT(39.89247122812017 -73.4034606484906) bank67470 +67471 POINT(41.0141973088615 -73.16345073192008) bank67471 +67472 POINT(40.4941692474959 -73.33776542069205) bank67472 +67473 POINT(41.098361307114686 -74.20661208162495) bank67473 +67474 POINT(40.06081836454771 -73.02851483889471) bank67474 +67475 POINT(41.230344657641844 -74.6514036929284) bank67475 +67476 POINT(40.585583064024384 -74.46525998200315) bank67476 +67477 POINT(41.46473547349468 -73.8331835201342) bank67477 +67478 POINT(40.58967767459371 -73.50389394747319) bank67478 +67479 POINT(41.56473599884445 -74.61695527628565) bank67479 +67480 POINT(40.375474921027646 -74.41492878521832) bank67480 +67481 POINT(41.10971580252504 -74.28108846642732) bank67481 +67482 POINT(40.40825510857114 -74.77570513587717) bank67482 +67483 POINT(40.79836236792377 -73.03647540884724) bank67483 +67484 POINT(40.5823179612176 -74.88751156424556) bank67484 +67485 POINT(40.928943655504845 -73.11778795169101) bank67485 +67486 POINT(40.57278260989034 -74.43300537183994) bank67486 +67487 POINT(40.61039380098048 -73.34911831234282) bank67487 +67488 POINT(40.62779601545282 -73.20315295384488) bank67488 +67489 POINT(39.92310436639772 -73.6210125529549) bank67489 +67490 POINT(40.45564909661103 -74.6953819200755) bank67490 +67491 POINT(41.660895989788244 -73.41306528322202) bank67491 +67492 POINT(40.30626949291873 -74.57401284567806) bank67492 +67493 POINT(40.418526037382925 -73.09350317173092) bank67493 +67494 POINT(39.78923885355739 -73.74764372370548) bank67494 +67495 POINT(39.886186698566675 -74.72689024942589) bank67495 +67496 POINT(40.90353258276234 -73.75200180552156) bank67496 +67497 POINT(41.64548632418467 -74.4668667725198) bank67497 +67498 POINT(41.073222259127334 -74.95538787774258) bank67498 +67499 POINT(41.04397050617328 -74.32566143004918) bank67499 +67500 POINT(41.019327950123945 -74.73119109323187) bank67500 +67501 POINT(41.53123813724625 -74.39279506070602) bank67501 +67502 POINT(40.20405786493831 -73.6823220144902) bank67502 +67503 POINT(39.845203138093694 -74.22279023554933) bank67503 +67504 POINT(41.08538094065443 -73.55994318607526) bank67504 +67505 POINT(40.13774500332655 -74.60497706344043) bank67505 +67506 POINT(41.51343159550165 -73.95659397688564) bank67506 +67507 POINT(41.21402836085644 -74.09395710415525) bank67507 +67508 POINT(40.42258187260865 -74.93646827814943) bank67508 +67509 POINT(39.941190451829634 -73.70397283741042) bank67509 +67510 POINT(40.90068163369218 -74.84158725403576) bank67510 +67511 POINT(40.89305487467168 -74.19082007238553) bank67511 +67512 POINT(40.606855381547895 -74.57534038816856) bank67512 +67513 POINT(40.331121256540975 -73.8847191905965) bank67513 +67514 POINT(41.35143825509504 -74.30315972269652) bank67514 +67515 POINT(41.45043595539392 -74.98201034334855) bank67515 +67516 POINT(40.14192025205928 -73.46871568162672) bank67516 +67517 POINT(40.23418445655077 -73.65053721461273) bank67517 +67518 POINT(40.08100094905374 -73.3307831913611) bank67518 +67519 POINT(40.92177566775009 -73.55685045480301) bank67519 +67520 POINT(40.18383799691904 -74.96971474989367) bank67520 +67521 POINT(39.80836489055294 -73.69113308673965) bank67521 +67522 POINT(40.511014986263454 -74.35418138071144) bank67522 +67523 POINT(41.436291179146 -74.8067200758514) bank67523 +67524 POINT(40.36818713583112 -73.07795492615683) bank67524 +67525 POINT(41.41740664294699 -74.1497391381706) bank67525 +67526 POINT(41.23595727762658 -73.26239542417133) bank67526 +67527 POINT(41.078516159748695 -74.23812923777928) bank67527 +67528 POINT(40.45835146153767 -73.44380987847384) bank67528 +67529 POINT(40.36027921750873 -73.97144630190851) bank67529 +67530 POINT(40.21018546312114 -73.34387979297999) bank67530 +67531 POINT(40.55444719660631 -74.04066765112569) bank67531 +67532 POINT(40.582070734957966 -73.13719238947411) bank67532 +67533 POINT(40.26178123668707 -74.61305231333085) bank67533 +67534 POINT(40.93364247130002 -73.97982866861197) bank67534 +67535 POINT(40.19225414595935 -73.31950387294563) bank67535 +67536 POINT(39.74234122015767 -73.42842433725455) bank67536 +67537 POINT(40.456765054487406 -73.48048711848404) bank67537 +67538 POINT(40.97099250819787 -73.82113038306454) bank67538 +67539 POINT(41.43097195593133 -73.52445284671256) bank67539 +67540 POINT(41.32488249197675 -73.3033617910479) bank67540 +67541 POINT(40.942718802234204 -73.52072875583202) bank67541 +67542 POINT(40.807035958127734 -74.93383730647199) bank67542 +67543 POINT(39.794755137327954 -73.3905615720494) bank67543 +67544 POINT(41.199518260604435 -74.59543097650345) bank67544 +67545 POINT(41.35226305709503 -73.13852937631562) bank67545 +67546 POINT(41.16709746746599 -74.13060678993457) bank67546 +67547 POINT(41.19213206538201 -74.54189960968684) bank67547 +67548 POINT(40.903735029145594 -74.22060332379941) bank67548 +67549 POINT(41.03575706672173 -74.58287092945602) bank67549 +67550 POINT(40.85515611146287 -74.96275838762769) bank67550 +67551 POINT(40.598691346509774 -73.76936966923182) bank67551 +67552 POINT(41.63989091969433 -74.68405982248659) bank67552 +67553 POINT(40.962567752924414 -73.29753769838781) bank67553 +67554 POINT(40.457495491138715 -73.30640203980161) bank67554 +67555 POINT(40.58173554335734 -73.35764973146154) bank67555 +67556 POINT(40.96509996509551 -73.47566427564527) bank67556 +67557 POINT(39.71942920992979 -74.97049382884336) bank67557 +67558 POINT(40.93285780040284 -73.68701077448188) bank67558 +67559 POINT(41.63618776589508 -74.1941698690276) bank67559 +67560 POINT(40.14976689181778 -73.20756585303046) bank67560 +67561 POINT(40.51303194065986 -73.1063619289519) bank67561 +67562 POINT(41.358660511157574 -74.33991195702119) bank67562 +67563 POINT(40.6132982700795 -74.77440837174498) bank67563 +67564 POINT(40.384832627591706 -73.48224307559096) bank67564 +67565 POINT(40.17338623714875 -73.48697658393976) bank67565 +67566 POINT(40.56527396676017 -73.13285652647028) bank67566 +67567 POINT(40.26410520319157 -74.19984195380964) bank67567 +67568 POINT(40.62565773484208 -74.87009304358837) bank67568 +67569 POINT(40.90355172614353 -73.56699689411447) bank67569 +67570 POINT(39.84242611947602 -74.51011483832158) bank67570 +67571 POINT(41.337824847158565 -74.98115348124317) bank67571 +67572 POINT(40.44203948708232 -74.72017803342068) bank67572 +67573 POINT(41.42577984535176 -74.11256510952772) bank67573 +67574 POINT(41.262890008979184 -74.43901660964724) bank67574 +67575 POINT(40.174815046721356 -74.69268411428617) bank67575 +67576 POINT(40.763920347328465 -73.46682221057982) bank67576 +67577 POINT(39.75178150407216 -73.67493276834388) bank67577 +67578 POINT(41.577623097890644 -73.9032352999518) bank67578 +67579 POINT(40.85081034074823 -74.21950087789239) bank67579 +67580 POINT(40.9967931204627 -73.39890148788854) bank67580 +67581 POINT(40.87350456630965 -73.58814714218508) bank67581 +67582 POINT(40.46192971442204 -73.8964642756725) bank67582 +67583 POINT(39.836115333718176 -73.51957989870756) bank67583 +67584 POINT(40.912277722692295 -74.01228592365744) bank67584 +67585 POINT(40.493679433283546 -73.01089346821465) bank67585 +67586 POINT(41.596071873938826 -74.51733683617242) bank67586 +67587 POINT(40.67080895843752 -73.20743727165032) bank67587 +67588 POINT(40.63576743280095 -73.51560369644363) bank67588 +67589 POINT(41.35974007914467 -74.78301782910549) bank67589 +67590 POINT(40.86074814611624 -74.88382779387935) bank67590 +67591 POINT(40.566509304178005 -74.78873122837689) bank67591 +67592 POINT(40.744565854896315 -73.84572402437821) bank67592 +67593 POINT(41.62474548141524 -74.72286114661854) bank67593 +67594 POINT(40.32855138237715 -74.8702761662797) bank67594 +67595 POINT(40.23783785262721 -73.82965325111107) bank67595 +67596 POINT(40.97227786992236 -73.65850645987692) bank67596 +67597 POINT(41.40677276561828 -73.45068287434381) bank67597 +67598 POINT(40.48761740384585 -74.7358510101289) bank67598 +67599 POINT(40.351183681119075 -74.8177490911173) bank67599 +67600 POINT(40.10622639764592 -73.368897537443) bank67600 +67601 POINT(41.01077352963406 -74.22273806797365) bank67601 +67602 POINT(41.510608744758144 -74.25940723975215) bank67602 +67603 POINT(41.238706427324736 -74.45901237052936) bank67603 +67604 POINT(40.643475471613364 -74.30778813133233) bank67604 +67605 POINT(41.135929358688195 -73.47714786755336) bank67605 +67606 POINT(40.008014713773385 -73.81035581350596) bank67606 +67607 POINT(41.15044405412161 -74.13054005341563) bank67607 +67608 POINT(41.06068828982135 -73.55404671175505) bank67608 +67609 POINT(41.703795224795115 -73.17433628274966) bank67609 +67610 POINT(40.97880772322842 -74.4296148336171) bank67610 +67611 POINT(41.70775639155745 -74.8441107107598) bank67611 +67612 POINT(41.35424967753072 -74.21666949587448) bank67612 +67613 POINT(39.86260340395098 -73.3078894921258) bank67613 +67614 POINT(39.8981117452835 -74.89967672505087) bank67614 +67615 POINT(40.31616419950075 -73.9348706458996) bank67615 +67616 POINT(41.56233590687483 -74.08532977203151) bank67616 +67617 POINT(40.92271169438552 -73.54289176320833) bank67617 +67618 POINT(41.31916608583381 -73.2810391555079) bank67618 +67619 POINT(39.8078215961373 -74.9901342214753) bank67619 +67620 POINT(40.19108651040788 -73.17411527729634) bank67620 +67621 POINT(39.828825101862954 -73.69171568507404) bank67621 +67622 POINT(40.53877015555208 -74.69208332462716) bank67622 +67623 POINT(40.696306338315026 -74.57846924258352) bank67623 +67624 POINT(40.87675931565616 -74.94925827397924) bank67624 +67625 POINT(41.500343626497596 -73.22409504008772) bank67625 +67626 POINT(41.69036682396785 -74.456462321936) bank67626 +67627 POINT(40.8424814089987 -73.96186567251657) bank67627 +67628 POINT(41.08926516420631 -73.8733137312222) bank67628 +67629 POINT(41.27944452139464 -73.77994294812575) bank67629 +67630 POINT(41.50890993493295 -73.36290212535533) bank67630 +67631 POINT(40.997964882540465 -73.73514414124315) bank67631 +67632 POINT(41.36924143121145 -74.30258083410091) bank67632 +67633 POINT(39.88312375455199 -73.99102328836959) bank67633 +67634 POINT(40.959827381377885 -73.925539002079) bank67634 +67635 POINT(40.21785215621844 -74.53332728468807) bank67635 +67636 POINT(40.325551722359286 -74.41094794614811) bank67636 +67637 POINT(40.51030555202715 -74.13301105766504) bank67637 +67638 POINT(40.258712500009324 -74.87562992670122) bank67638 +67639 POINT(41.48374858064902 -73.56046681129644) bank67639 +67640 POINT(41.413531671318744 -74.62295527590673) bank67640 +67641 POINT(41.4507425159583 -74.21608452993304) bank67641 +67642 POINT(39.94426933265189 -73.91423679494524) bank67642 +67643 POINT(40.26803432162597 -73.92619655100285) bank67643 +67644 POINT(40.22729004066383 -74.89549424106556) bank67644 +67645 POINT(41.58192126807645 -73.44079709606848) bank67645 +67646 POINT(39.96798616066907 -73.9931144069518) bank67646 +67647 POINT(41.48263716878958 -73.49696330676984) bank67647 +67648 POINT(40.56743731137435 -74.09624466947658) bank67648 +67649 POINT(39.85309688605208 -73.39199192854181) bank67649 +67650 POINT(41.13693171272746 -73.13525976411088) bank67650 +67651 POINT(39.93311983404729 -74.5140593256217) bank67651 +67652 POINT(40.25114685825944 -73.81432449849653) bank67652 +67653 POINT(41.64344937192989 -74.73936616159168) bank67653 +67654 POINT(40.78427574290919 -73.57378006110272) bank67654 +67655 POINT(40.0824164611635 -74.36791559183062) bank67655 +67656 POINT(41.340247099397125 -73.99499159494938) bank67656 +67657 POINT(40.152393735495025 -74.57396884295858) bank67657 +67658 POINT(40.688546151310355 -73.26367797597837) bank67658 +67659 POINT(41.26789642302734 -73.05377314186157) bank67659 +67660 POINT(41.05856938661616 -74.73370489511865) bank67660 +67661 POINT(39.728127230541844 -73.61737590559932) bank67661 +67662 POINT(40.89808196884398 -74.67581679735125) bank67662 +67663 POINT(40.468587099010776 -73.06715013214297) bank67663 +67664 POINT(40.84527366298534 -73.764795126091) bank67664 +67665 POINT(40.110644812631755 -73.48767200999342) bank67665 +67666 POINT(41.34590134665244 -73.16005606930995) bank67666 +67667 POINT(40.957875596073976 -73.78602384080686) bank67667 +67668 POINT(40.39853605592504 -73.55610620230878) bank67668 +67669 POINT(39.84845171548727 -73.72991887572762) bank67669 +67670 POINT(41.015814741347306 -73.71105318186913) bank67670 +67671 POINT(39.96689452504061 -73.64285174890256) bank67671 +67672 POINT(40.49962324416556 -74.250708457271) bank67672 +67673 POINT(39.853499897161 -74.12551924723827) bank67673 +67674 POINT(41.27127889748887 -74.09511574553834) bank67674 +67675 POINT(41.34296106475177 -73.87345526052908) bank67675 +67676 POINT(39.809632266955724 -74.26126272580274) bank67676 +67677 POINT(40.61553333333137 -75.00442898911507) bank67677 +67678 POINT(39.93639781381946 -73.1567745646195) bank67678 +67679 POINT(39.89642996036903 -74.43330657116293) bank67679 +67680 POINT(40.44789680293109 -74.01893767988041) bank67680 +67681 POINT(40.32533253977591 -73.17828437151246) bank67681 +67682 POINT(41.23342267874871 -73.3723238453657) bank67682 +67683 POINT(40.87637285282615 -74.39700661884554) bank67683 +67684 POINT(39.78213244276577 -73.26043861323444) bank67684 +67685 POINT(41.62296913698735 -73.16288576630372) bank67685 +67686 POINT(41.547568373853245 -74.52439202303002) bank67686 +67687 POINT(40.90837665470461 -73.51801068267537) bank67687 +67688 POINT(40.11975963180346 -74.97169178856919) bank67688 +67689 POINT(40.48788889907488 -74.81256493870038) bank67689 +67690 POINT(40.649488626002395 -73.97570405127293) bank67690 +67691 POINT(41.49761580532014 -74.39785055311299) bank67691 +67692 POINT(40.96932245318024 -74.3884514009601) bank67692 +67693 POINT(41.14290281927695 -73.23607667993214) bank67693 +67694 POINT(40.90629047045452 -74.70047440599095) bank67694 +67695 POINT(40.102222733815935 -74.79332166586262) bank67695 +67696 POINT(40.84339177489044 -73.20191079916545) bank67696 +67697 POINT(41.2388788423326 -74.61565831596754) bank67697 +67698 POINT(40.25003023507392 -73.39015083158597) bank67698 +67699 POINT(41.651430736983116 -74.16540685278748) bank67699 +67700 POINT(40.00074156954481 -74.95332417188492) bank67700 +67701 POINT(39.724279325462376 -74.74591562246646) bank67701 +67702 POINT(40.803843306279774 -74.91622598350108) bank67702 +67703 POINT(40.04868914428335 -74.91107026510865) bank67703 +67704 POINT(41.57102053896374 -73.80112574721952) bank67704 +67705 POINT(40.86948586057917 -74.21263121695797) bank67705 +67706 POINT(39.737183133032694 -74.33027242529121) bank67706 +67707 POINT(41.51831466426332 -73.69484907724247) bank67707 +67708 POINT(41.64575080898722 -74.86290250686584) bank67708 +67709 POINT(40.10794944551924 -73.97718396574136) bank67709 +67710 POINT(41.17886837148566 -74.74491711409839) bank67710 +67711 POINT(39.9026992562402 -74.64486209171154) bank67711 +67712 POINT(41.21752587932557 -73.550959877436) bank67712 +67713 POINT(41.48058163510339 -74.48834619729458) bank67713 +67714 POINT(39.86545912949189 -74.17907253212888) bank67714 +67715 POINT(41.27172284326118 -74.15001075569455) bank67715 +67716 POINT(41.04592862843745 -74.86196354703753) bank67716 +67717 POINT(40.90299641725282 -74.32560299454859) bank67717 +67718 POINT(41.00254973938611 -74.02874461854684) bank67718 +67719 POINT(41.18297268168428 -74.47466296573567) bank67719 +67720 POINT(39.86296257608392 -73.87819592549799) bank67720 +67721 POINT(39.75878026135932 -73.2649521519723) bank67721 +67722 POINT(41.51174658078017 -73.43730021427619) bank67722 +67723 POINT(40.9136320664081 -74.42932564828737) bank67723 +67724 POINT(41.056830463286495 -74.05582507464139) bank67724 +67725 POINT(40.02829506866886 -74.76185606012801) bank67725 +67726 POINT(39.87083916901209 -74.97163272700799) bank67726 +67727 POINT(40.411557409878306 -73.30953938025442) bank67727 +67728 POINT(41.64464392151895 -74.86080090405257) bank67728 +67729 POINT(40.89225554517132 -73.72039428068567) bank67729 +67730 POINT(40.95967520613058 -73.28013783886867) bank67730 +67731 POINT(40.22332044091769 -73.15589203949766) bank67731 +67732 POINT(40.189329893931955 -73.92474285082096) bank67732 +67733 POINT(40.63902491720252 -74.46055004275448) bank67733 +67734 POINT(41.110992504557686 -73.41498316078616) bank67734 +67735 POINT(40.58906112471459 -73.45245408944152) bank67735 +67736 POINT(40.39283140803452 -74.83265471598664) bank67736 +67737 POINT(40.81863099519473 -74.61622754580924) bank67737 +67738 POINT(40.054282725840366 -74.45464693308992) bank67738 +67739 POINT(40.770537073596635 -74.80776559402436) bank67739 +67740 POINT(40.42061727929433 -74.46720285186606) bank67740 +67741 POINT(39.74543649931633 -73.55884326174963) bank67741 +67742 POINT(39.99188002873741 -73.26569379828786) bank67742 +67743 POINT(40.443513458233355 -74.72249344636928) bank67743 +67744 POINT(41.49681875991963 -73.81286527635818) bank67744 +67745 POINT(41.4421215783479 -73.56671056947658) bank67745 +67746 POINT(39.71908660409922 -74.05132982789397) bank67746 +67747 POINT(40.607053502850405 -74.96703226032261) bank67747 +67748 POINT(40.16883026789861 -73.87986468649518) bank67748 +67749 POINT(41.522931442026774 -73.07259643805855) bank67749 +67750 POINT(40.4599058154553 -74.62688482717682) bank67750 +67751 POINT(40.696147894737145 -74.08281719065405) bank67751 +67752 POINT(40.29437609949956 -73.29897362181674) bank67752 +67753 POINT(40.9091728708229 -74.67582958001735) bank67753 +67754 POINT(39.89947401370277 -73.07831873155126) bank67754 +67755 POINT(40.65804132667972 -74.32622325402741) bank67755 +67756 POINT(39.73404525036718 -73.49971518635087) bank67756 +67757 POINT(41.705194597181645 -73.77027660624225) bank67757 +67758 POINT(41.196465728834234 -74.79125303070315) bank67758 +67759 POINT(41.08572331158029 -73.70278429802313) bank67759 +67760 POINT(40.247528697674774 -74.4843967954611) bank67760 +67761 POINT(41.29042762668781 -74.83881092469431) bank67761 +67762 POINT(40.960052882308574 -73.158328891115) bank67762 +67763 POINT(41.57551609168861 -74.44398817561931) bank67763 +67764 POINT(39.94185754751284 -73.03724339794152) bank67764 +67765 POINT(39.84659190021621 -74.9539202357941) bank67765 +67766 POINT(41.40723673234009 -73.49883795742296) bank67766 +67767 POINT(40.040426949610115 -74.29651876201707) bank67767 +67768 POINT(40.49969125394202 -74.2765167866014) bank67768 +67769 POINT(41.6480414629669 -74.28788457623232) bank67769 +67770 POINT(41.09190979766545 -73.52646629223767) bank67770 +67771 POINT(40.194054338498866 -74.38321838909737) bank67771 +67772 POINT(40.26143341751971 -74.9105510688588) bank67772 +67773 POINT(41.5125007679446 -74.7342717773896) bank67773 +67774 POINT(40.92268852450896 -73.85815793290445) bank67774 +67775 POINT(41.36772974165744 -74.63906747272512) bank67775 +67776 POINT(41.18664357718359 -73.90929585295659) bank67776 +67777 POINT(41.3951133256934 -74.51902857134884) bank67777 +67778 POINT(41.25882769901935 -74.27898225127427) bank67778 +67779 POINT(40.32333362781933 -73.61198016327256) bank67779 +67780 POINT(39.94834910050426 -73.19191940716814) bank67780 +67781 POINT(40.378019854568315 -73.83390250785752) bank67781 +67782 POINT(40.49809074713972 -73.62020537685495) bank67782 +67783 POINT(41.433017505242795 -73.53886494138142) bank67783 +67784 POINT(39.9422089642044 -74.33946760520104) bank67784 +67785 POINT(40.184954658424274 -74.95700686619492) bank67785 +67786 POINT(40.834871530989545 -73.40665120148797) bank67786 +67787 POINT(41.644713342718234 -74.22429779931167) bank67787 +67788 POINT(39.91304675626772 -74.60494074672084) bank67788 +67789 POINT(40.683684605489724 -74.39683501755226) bank67789 +67790 POINT(40.301205300811866 -74.17849003019533) bank67790 +67791 POINT(40.8935565458308 -74.57719221606955) bank67791 +67792 POINT(41.631049245557115 -74.29680243694993) bank67792 +67793 POINT(40.0860065840033 -74.77873242249184) bank67793 +67794 POINT(41.287778443837674 -73.25015725395326) bank67794 +67795 POINT(41.68757806367413 -73.47477203853653) bank67795 +67796 POINT(40.043346808279985 -74.74220972122238) bank67796 +67797 POINT(41.50953548274732 -74.90121416829284) bank67797 +67798 POINT(40.932937215720024 -73.09707538720357) bank67798 +67799 POINT(41.05208939144115 -73.23372601096513) bank67799 +67800 POINT(40.428189433848324 -73.73765183110812) bank67800 +67801 POINT(41.279259408498156 -74.09142168904036) bank67801 +67802 POINT(41.48298914693045 -73.87801825792792) bank67802 +67803 POINT(40.52661104591083 -73.86791801342862) bank67803 +67804 POINT(40.537121415088706 -74.25770302156562) bank67804 +67805 POINT(40.63635239050544 -73.3556757573796) bank67805 +67806 POINT(40.06411565001568 -73.94185323143383) bank67806 +67807 POINT(40.69720146486949 -74.39138972748725) bank67807 +67808 POINT(40.601575542944424 -73.51021415839591) bank67808 +67809 POINT(40.13113854544877 -73.08172460319902) bank67809 +67810 POINT(40.22269629289015 -73.25195645660041) bank67810 +67811 POINT(41.240902019610886 -73.26185709510935) bank67811 +67812 POINT(41.168329006990426 -73.2863044882305) bank67812 +67813 POINT(41.27886791390527 -74.30591348089001) bank67813 +67814 POINT(41.15684221825527 -73.12362405941448) bank67814 +67815 POINT(41.163072395596586 -74.41436330445717) bank67815 +67816 POINT(41.20543113639621 -73.8306037578179) bank67816 +67817 POINT(39.81821753994844 -74.84316477640405) bank67817 +67818 POINT(39.99999593649917 -73.75645488162719) bank67818 +67819 POINT(41.5745117942335 -74.9593465741077) bank67819 +67820 POINT(40.615048552683675 -74.71598796802256) bank67820 +67821 POINT(40.993369991327526 -74.6381519580392) bank67821 +67822 POINT(40.51538522684428 -73.86267191272022) bank67822 +67823 POINT(40.55621230747136 -74.24215793140975) bank67823 +67824 POINT(41.634201889223114 -74.04757825676703) bank67824 +67825 POINT(41.55262766095118 -73.42755689403958) bank67825 +67826 POINT(40.16942097928191 -74.20473824368933) bank67826 +67827 POINT(40.852647990465535 -73.94469272377715) bank67827 +67828 POINT(40.15211346097091 -73.517133690071) bank67828 +67829 POINT(39.842589485914175 -73.15745393113724) bank67829 +67830 POINT(40.36630887862322 -74.03280896230616) bank67830 +67831 POINT(41.28182051661486 -74.82531137660354) bank67831 +67832 POINT(39.719058710364756 -73.94339051689624) bank67832 +67833 POINT(41.66336373220395 -73.42224297775117) bank67833 +67834 POINT(41.437562796254795 -73.32604363081079) bank67834 +67835 POINT(40.1532104455218 -74.9144792729831) bank67835 +67836 POINT(41.08155808312234 -74.02501793225422) bank67836 +67837 POINT(41.31449113346228 -73.12330737917053) bank67837 +67838 POINT(41.654701181367166 -73.62994934734245) bank67838 +67839 POINT(41.680553490077855 -73.24460619939843) bank67839 +67840 POINT(40.050600596505504 -73.73319835991941) bank67840 +67841 POINT(41.33097417575922 -73.69110610062152) bank67841 +67842 POINT(41.1585175985282 -74.4428709780874) bank67842 +67843 POINT(40.06457670834795 -73.22288718381326) bank67843 +67844 POINT(40.00979605538518 -74.46232200414671) bank67844 +67845 POINT(41.210864256087405 -74.54844231031082) bank67845 +67846 POINT(41.607830559805336 -73.22281094412546) bank67846 +67847 POINT(41.67598656131547 -74.2164107310611) bank67847 +67848 POINT(41.111879402733095 -74.5810160233329) bank67848 +67849 POINT(39.87486289456438 -74.27725202275884) bank67849 +67850 POINT(41.661305411228525 -73.97104916447665) bank67850 +67851 POINT(41.70997407572243 -74.27080733130293) bank67851 +67852 POINT(41.43363569867791 -74.71488785741786) bank67852 +67853 POINT(40.28455973024737 -73.09372709322024) bank67853 +67854 POINT(40.910417908879985 -73.61326366893536) bank67854 +67855 POINT(41.23849056406264 -73.21282721555973) bank67855 +67856 POINT(41.35283562122716 -74.13485304959022) bank67856 +67857 POINT(39.86299639522064 -73.8642086769845) bank67857 +67858 POINT(40.54305132625932 -74.16296978792846) bank67858 +67859 POINT(41.48380457695049 -73.15939011462105) bank67859 +67860 POINT(40.75948960093922 -74.70096057805236) bank67860 +67861 POINT(40.05842519886428 -74.44709289136607) bank67861 +67862 POINT(40.55715023548009 -73.83382701691103) bank67862 +67863 POINT(40.676853294714164 -74.75400701307795) bank67863 +67864 POINT(39.92688976020008 -74.50917869569444) bank67864 +67865 POINT(40.21676389649932 -74.22749562601358) bank67865 +67866 POINT(41.528879772340694 -74.8002469442184) bank67866 +67867 POINT(41.61790967224421 -74.92094856698502) bank67867 +67868 POINT(40.45152840681479 -73.7963179885927) bank67868 +67869 POINT(41.01387263505834 -74.16096587047967) bank67869 +67870 POINT(39.90836140765943 -74.65556398793096) bank67870 +67871 POINT(41.573753009305676 -74.3081207414577) bank67871 +67872 POINT(41.14136987941687 -74.81262372861408) bank67872 +67873 POINT(40.512343310708275 -74.11873127939005) bank67873 +67874 POINT(40.17065264667307 -74.68892446238361) bank67874 +67875 POINT(39.854709685761456 -73.32770118243128) bank67875 +67876 POINT(40.03641961187787 -73.19531600143581) bank67876 +67877 POINT(40.13466212035033 -73.03038246594248) bank67877 +67878 POINT(40.26572587608263 -73.54223471894892) bank67878 +67879 POINT(40.27108918030492 -74.445260925486) bank67879 +67880 POINT(41.35780139802955 -74.47712947397949) bank67880 +67881 POINT(39.9172235972734 -73.12263565190143) bank67881 +67882 POINT(41.42196085684588 -74.65603223977503) bank67882 +67883 POINT(41.302018596819174 -73.11410487971523) bank67883 +67884 POINT(39.76270735327239 -74.25499821375448) bank67884 +67885 POINT(41.18101553303408 -74.15236668565485) bank67885 +67886 POINT(40.28266300802294 -73.58660369499411) bank67886 +67887 POINT(40.751723746493056 -73.76091436490636) bank67887 +67888 POINT(41.57169661093883 -73.34806647711707) bank67888 +67889 POINT(40.374679014200225 -73.50992021489182) bank67889 +67890 POINT(40.366750937759264 -73.92774219844559) bank67890 +67891 POINT(41.59835783134404 -74.1317584037797) bank67891 +67892 POINT(40.24163802032579 -74.5005445761428) bank67892 +67893 POINT(40.49065522404198 -74.38468548368465) bank67893 +67894 POINT(39.86270007538492 -73.97347896016564) bank67894 +67895 POINT(40.848415396794394 -74.42806150095836) bank67895 +67896 POINT(40.81078906530301 -73.32739920738409) bank67896 +67897 POINT(41.1200795851832 -74.70468276606778) bank67897 +67898 POINT(40.758765156400884 -74.47752609553264) bank67898 +67899 POINT(41.271258352072024 -74.80919719884774) bank67899 +67900 POINT(39.90032446049167 -73.4249590587286) bank67900 +67901 POINT(40.61026527349498 -73.35953384223824) bank67901 +67902 POINT(40.34449821140525 -74.89668847020161) bank67902 +67903 POINT(41.0708768731849 -74.45510492662132) bank67903 +67904 POINT(40.164185097847465 -73.10111010810526) bank67904 +67905 POINT(40.80483979497591 -73.44507692207833) bank67905 +67906 POINT(40.82236160018982 -73.30349978567904) bank67906 +67907 POINT(40.7197886678733 -74.89922770931024) bank67907 +67908 POINT(40.29278387447937 -74.46034945072873) bank67908 +67909 POINT(40.20234882382844 -73.59554958864908) bank67909 +67910 POINT(39.84728116605896 -74.3638419443137) bank67910 +67911 POINT(41.14723343164971 -74.76504127363874) bank67911 +67912 POINT(41.39032268381505 -73.29437595282221) bank67912 +67913 POINT(41.193939269508114 -73.50324686393319) bank67913 +67914 POINT(40.922783468163104 -73.03331894742047) bank67914 +67915 POINT(40.99871856796957 -73.24776657971464) bank67915 +67916 POINT(40.04744281926596 -74.52760878778811) bank67916 +67917 POINT(41.57891557755336 -73.63536785831504) bank67917 +67918 POINT(40.780339473410564 -74.32036663062753) bank67918 +67919 POINT(40.327818508944425 -73.82154654045345) bank67919 +67920 POINT(39.756688396456525 -74.3641624382136) bank67920 +67921 POINT(40.05361715148586 -74.32245094869258) bank67921 +67922 POINT(40.09566311713974 -74.72616359235852) bank67922 +67923 POINT(40.26851518270753 -73.52971349954169) bank67923 +67924 POINT(41.1887634732226 -73.01174797818108) bank67924 +67925 POINT(41.69522670552151 -74.15635301204323) bank67925 +67926 POINT(40.01476517146396 -73.41559228742231) bank67926 +67927 POINT(40.644609440490996 -74.1875723021521) bank67927 +67928 POINT(40.38217864921583 -74.99835878687583) bank67928 +67929 POINT(40.84426777295586 -74.24841935044066) bank67929 +67930 POINT(40.49376763714547 -73.30973351628836) bank67930 +67931 POINT(40.58691883907949 -73.73259328995812) bank67931 +67932 POINT(41.04488709388261 -74.62557350343337) bank67932 +67933 POINT(40.293432989339514 -73.56682126354065) bank67933 +67934 POINT(40.783994123163886 -74.63849209748587) bank67934 +67935 POINT(39.872624924787374 -74.19552487771202) bank67935 +67936 POINT(41.27294393578578 -73.19123363587909) bank67936 +67937 POINT(40.591463140018675 -74.27283916428243) bank67937 +67938 POINT(41.18557635245301 -73.84145114214225) bank67938 +67939 POINT(41.02673893311182 -74.23491919068051) bank67939 +67940 POINT(40.51099750633662 -73.62991162804968) bank67940 +67941 POINT(41.29122999874202 -74.23459663823144) bank67941 +67942 POINT(41.04438678585095 -73.0296118360936) bank67942 +67943 POINT(40.50634896598952 -73.67038107610206) bank67943 +67944 POINT(41.59864992204296 -74.84542931217261) bank67944 +67945 POINT(40.69310003589724 -73.8012107041314) bank67945 +67946 POINT(40.43383186165727 -74.29760938141764) bank67946 +67947 POINT(40.8511890254234 -74.37447614112027) bank67947 +67948 POINT(40.75228763112347 -74.5145400415086) bank67948 +67949 POINT(40.45712338863104 -73.8454635173312) bank67949 +67950 POINT(40.86560447885553 -73.22547821918415) bank67950 +67951 POINT(40.572980894652694 -73.06461286276381) bank67951 +67952 POINT(41.643259738090954 -74.34346963587585) bank67952 +67953 POINT(41.091824714219975 -73.85646017644216) bank67953 +67954 POINT(40.81293924338791 -73.00883315330277) bank67954 +67955 POINT(40.55976078908189 -73.57955677881947) bank67955 +67956 POINT(41.022573064875374 -74.09678095451254) bank67956 +67957 POINT(41.3843378969862 -73.95170235507696) bank67957 +67958 POINT(40.55409016244394 -73.2226434419042) bank67958 +67959 POINT(40.20007638240615 -74.83982952799491) bank67959 +67960 POINT(41.17338523578359 -73.78288604289325) bank67960 +67961 POINT(39.73213087070103 -73.68603987325307) bank67961 +67962 POINT(40.056644686980135 -73.90033761550134) bank67962 +67963 POINT(41.37233424345962 -74.98473914291336) bank67963 +67964 POINT(40.79983253094164 -73.79153683681905) bank67964 +67965 POINT(40.5578645603806 -73.0221961493976) bank67965 +67966 POINT(41.610014165071085 -74.79025245459474) bank67966 +67967 POINT(40.15195877866079 -74.30084093894277) bank67967 +67968 POINT(40.49529844485704 -74.6490955112689) bank67968 +67969 POINT(40.56568999291293 -73.75759650475646) bank67969 +67970 POINT(39.997894649902165 -73.79965299160784) bank67970 +67971 POINT(40.683670772189686 -74.49132879915084) bank67971 +67972 POINT(40.64282660068214 -73.27320363588925) bank67972 +67973 POINT(39.94433929112856 -73.11351579521117) bank67973 +67974 POINT(40.017444714658225 -73.3694818889229) bank67974 +67975 POINT(41.2547003472676 -74.78733816361739) bank67975 +67976 POINT(41.64585131019325 -74.3653151108493) bank67976 +67977 POINT(40.23167209451399 -74.50146006982271) bank67977 +67978 POINT(40.81050347642468 -74.58780630288585) bank67978 +67979 POINT(40.647419176853006 -73.36817369864917) bank67979 +67980 POINT(41.33518786586227 -74.57278428582592) bank67980 +67981 POINT(40.54866411311275 -73.6188148835463) bank67981 +67982 POINT(41.225113056788416 -73.59203507269713) bank67982 +67983 POINT(41.70689560664565 -73.92715838467608) bank67983 +67984 POINT(40.13626230481296 -73.74011148272318) bank67984 +67985 POINT(40.002447569820006 -74.3094257790222) bank67985 +67986 POINT(41.50329185282448 -73.27746562664788) bank67986 +67987 POINT(39.78450978895953 -73.18483673325036) bank67987 +67988 POINT(40.2781318259769 -73.30843550260408) bank67988 +67989 POINT(40.6855042974176 -74.20843940511085) bank67989 +67990 POINT(39.82732601310126 -73.54568241392627) bank67990 +67991 POINT(41.632686672470165 -73.10333504731989) bank67991 +67992 POINT(40.22339309248165 -73.1204959170492) bank67992 +67993 POINT(40.592430569396434 -73.1821775569447) bank67993 +67994 POINT(39.90034835284142 -74.59509918356943) bank67994 +67995 POINT(39.97309606767141 -74.50545786631777) bank67995 +67996 POINT(41.221204600795865 -73.85167696613269) bank67996 +67997 POINT(40.134481561250766 -74.19033554103677) bank67997 +67998 POINT(41.48291570828477 -74.37247326068473) bank67998 +67999 POINT(40.672405531267536 -74.54655579086142) bank67999 +68000 POINT(39.77685609623658 -74.26424707681701) bank68000 +68001 POINT(40.86657022984512 -73.10780315211451) bank68001 +68002 POINT(40.26217704311659 -73.1949754255308) bank68002 +68003 POINT(41.193916034732496 -73.17209302228724) bank68003 +68004 POINT(40.49774489904664 -74.57899069385829) bank68004 +68005 POINT(41.618237686357304 -73.76770858344533) bank68005 +68006 POINT(41.28868715246568 -74.82885544561529) bank68006 +68007 POINT(39.77634223269758 -73.62470563500675) bank68007 +68008 POINT(40.61676132818866 -73.30945340059786) bank68008 +68009 POINT(40.9220097214653 -74.98874376317484) bank68009 +68010 POINT(40.039416634890394 -74.99826278821956) bank68010 +68011 POINT(41.06418737885437 -74.09108020071412) bank68011 +68012 POINT(41.25384511078089 -74.38661719864452) bank68012 +68013 POINT(40.507418151004416 -73.88643394394317) bank68013 +68014 POINT(40.099702982136776 -73.63747535257366) bank68014 +68015 POINT(39.87261753669786 -74.03466632558599) bank68015 +68016 POINT(41.44376783075235 -73.73939137272397) bank68016 +68017 POINT(41.445657797343905 -74.64255027010336) bank68017 +68018 POINT(41.06733522180628 -73.99491777156616) bank68018 +68019 POINT(40.061533125905065 -73.7242791657164) bank68019 +68020 POINT(39.78159113363999 -74.42060779271564) bank68020 +68021 POINT(40.61107588894734 -73.09717074270894) bank68021 +68022 POINT(40.904092442239474 -74.29585947753351) bank68022 +68023 POINT(41.07358182428482 -73.26477668765317) bank68023 +68024 POINT(41.031653038374344 -74.08983631693766) bank68024 +68025 POINT(41.18697483060608 -74.10385663120854) bank68025 +68026 POINT(41.046008839491016 -73.47836466096683) bank68026 +68027 POINT(40.4137683565652 -73.18769227983466) bank68027 +68028 POINT(40.15760258834908 -74.03104607759704) bank68028 +68029 POINT(40.059426236924935 -74.36384322767032) bank68029 +68030 POINT(39.96436691382425 -74.47209542782464) bank68030 +68031 POINT(39.996464210309526 -74.28913676079391) bank68031 +68032 POINT(40.814318740992505 -74.32593576036959) bank68032 +68033 POINT(41.381556711495996 -73.97440128524192) bank68033 +68034 POINT(39.930351701448714 -74.99101246954888) bank68034 +68035 POINT(40.8482651920181 -74.98968421413994) bank68035 +68036 POINT(41.182467428795235 -74.7662722088838) bank68036 +68037 POINT(40.116145278431034 -74.0785518491362) bank68037 +68038 POINT(41.47410382444966 -73.68177818191324) bank68038 +68039 POINT(40.67964529628212 -73.41990837250492) bank68039 +68040 POINT(40.22465235922647 -74.37381204677868) bank68040 +68041 POINT(40.8248289658153 -73.22828251150706) bank68041 +68042 POINT(41.35126751946102 -74.20178638311468) bank68042 +68043 POINT(40.33885898835586 -74.11315918998783) bank68043 +68044 POINT(40.83094000718505 -73.90086856275585) bank68044 +68045 POINT(39.75940837768476 -74.44189780486786) bank68045 +68046 POINT(41.28822997695908 -74.97516319861393) bank68046 +68047 POINT(41.51674454172063 -73.85332351150427) bank68047 +68048 POINT(40.14729854356884 -73.91993327612907) bank68048 +68049 POINT(40.865185240744104 -73.63097827936492) bank68049 +68050 POINT(40.329296402510465 -73.40893712344634) bank68050 +68051 POINT(41.13770261203403 -74.77235147508867) bank68051 +68052 POINT(40.04770171087635 -74.34609547694315) bank68052 +68053 POINT(41.40886671156833 -74.08340789645044) bank68053 +68054 POINT(39.83380559306005 -73.59097934646121) bank68054 +68055 POINT(41.609368357236555 -74.28113292075567) bank68055 +68056 POINT(40.86262276444947 -73.29662764066431) bank68056 +68057 POINT(40.421983865303915 -74.2639310998435) bank68057 +68058 POINT(40.36572789986981 -74.90711397484455) bank68058 +68059 POINT(39.83438968529459 -73.01121213650192) bank68059 +68060 POINT(40.48111341744992 -73.43809943517405) bank68060 +68061 POINT(39.91567782508495 -73.98123562050982) bank68061 +68062 POINT(41.18043595217871 -73.12577246716135) bank68062 +68063 POINT(40.31571888286778 -74.51881925231295) bank68063 +68064 POINT(41.359354672938295 -73.18301861983049) bank68064 +68065 POINT(41.614147811042216 -74.15499370920865) bank68065 +68066 POINT(40.351001911487316 -74.9061956436147) bank68066 +68067 POINT(40.98136089653055 -73.40161348024097) bank68067 +68068 POINT(39.80307828730079 -74.89278549302148) bank68068 +68069 POINT(40.273164237510585 -74.16786115833712) bank68069 +68070 POINT(40.67144599139376 -73.37404171347059) bank68070 +68071 POINT(40.44203967511155 -74.79779194838372) bank68071 +68072 POINT(40.268597150704835 -74.0589135183225) bank68072 +68073 POINT(40.3505340113676 -73.56499793082376) bank68073 +68074 POINT(40.33665112718362 -73.03065789385606) bank68074 +68075 POINT(40.3886142928149 -74.78591032761867) bank68075 +68076 POINT(40.635902898370695 -73.40553257593069) bank68076 +68077 POINT(39.81747054066626 -74.2832967963252) bank68077 +68078 POINT(40.96869380592286 -74.88883874193074) bank68078 +68079 POINT(41.47037896807065 -74.78085266345623) bank68079 +68080 POINT(41.59675706947174 -74.51051051471454) bank68080 +68081 POINT(39.87353583125046 -74.5254542573748) bank68081 +68082 POINT(39.97552611734712 -73.58396764368868) bank68082 +68083 POINT(39.88156470249443 -74.68545581988043) bank68083 +68084 POINT(41.444516569935566 -73.31072835868888) bank68084 +68085 POINT(40.83046486810872 -73.5372929345998) bank68085 +68086 POINT(41.01717509724367 -74.2062207374382) bank68086 +68087 POINT(39.970711667768455 -74.56500044459808) bank68087 +68088 POINT(40.07717720133788 -73.33327148298761) bank68088 +68089 POINT(40.93859744376848 -73.12503482631803) bank68089 +68090 POINT(41.02693143910013 -73.35644964056137) bank68090 +68091 POINT(40.796554759699006 -74.81619200114483) bank68091 +68092 POINT(40.792103060296476 -74.17634506816214) bank68092 +68093 POINT(41.35872798975314 -74.67832271954097) bank68093 +68094 POINT(39.920805390653484 -74.63512445363483) bank68094 +68095 POINT(41.22918320142217 -74.29890740814254) bank68095 +68096 POINT(40.240679548379674 -73.44768164502904) bank68096 +68097 POINT(40.50018556335453 -74.20543875834655) bank68097 +68098 POINT(40.77606947731481 -74.21751460240614) bank68098 +68099 POINT(41.595898117453345 -75.00388436720883) bank68099 +68100 POINT(40.373361740981856 -74.2701096297124) bank68100 +68101 POINT(41.370405734158595 -73.69596173905472) bank68101 +68102 POINT(41.512946070394825 -74.2809470082198) bank68102 +68103 POINT(41.380795237889814 -74.96950768860106) bank68103 +68104 POINT(40.06059188403074 -74.52560934037136) bank68104 +68105 POINT(39.83566827700105 -73.48983241285816) bank68105 +68106 POINT(39.88705851346977 -74.77390560958945) bank68106 +68107 POINT(40.25677496890706 -74.39473903324725) bank68107 +68108 POINT(40.5000641629558 -74.39112520220695) bank68108 +68109 POINT(40.17017084885761 -74.46919963644562) bank68109 +68110 POINT(40.92494885570504 -73.82789505177642) bank68110 +68111 POINT(41.3334354571065 -74.9857749784545) bank68111 +68112 POINT(40.43812543196316 -73.14509040269247) bank68112 +68113 POINT(41.53203687152249 -73.10176654077938) bank68113 +68114 POINT(39.80348130619416 -73.301773368709) bank68114 +68115 POINT(40.294329801740474 -73.52290563372097) bank68115 +68116 POINT(39.88019000462138 -73.56129461576406) bank68116 +68117 POINT(40.08245770424872 -74.76503373031339) bank68117 +68118 POINT(40.40486326816838 -74.49734073899569) bank68118 +68119 POINT(41.48407015500654 -74.63176272683165) bank68119 +68120 POINT(40.621653942367246 -73.25358864664051) bank68120 +68121 POINT(40.875868340453586 -73.34034536832716) bank68121 +68122 POINT(40.284974754827154 -74.89950907124216) bank68122 +68123 POINT(41.291353006858316 -73.11014827209685) bank68123 +68124 POINT(40.067748186266954 -74.30778710361874) bank68124 +68125 POINT(40.69933763319982 -74.88969734334034) bank68125 +68126 POINT(40.008869076993676 -74.49017127657032) bank68126 +68127 POINT(40.03225752696522 -73.28027396503587) bank68127 +68128 POINT(41.518162432919915 -74.07769375293924) bank68128 +68129 POINT(40.55617220722055 -74.86474796300642) bank68129 +68130 POINT(41.68923049780155 -73.69824570232157) bank68130 +68131 POINT(40.70346756350281 -73.57109293681711) bank68131 +68132 POINT(41.33109866560242 -73.77171366775235) bank68132 +68133 POINT(41.29603048118087 -73.1925745612262) bank68133 +68134 POINT(41.536354879445376 -73.49133025615552) bank68134 +68135 POINT(40.43315368274665 -73.99463622821308) bank68135 +68136 POINT(40.85572888998536 -73.52169806845382) bank68136 +68137 POINT(40.7573754089145 -74.1745504652942) bank68137 +68138 POINT(40.242987841693775 -73.15265234279893) bank68138 +68139 POINT(40.56260821393513 -73.4473805465315) bank68139 +68140 POINT(41.447526961248684 -74.8749512897078) bank68140 +68141 POINT(39.826497731164366 -74.18744128502502) bank68141 +68142 POINT(39.88001240113793 -73.84638755048492) bank68142 +68143 POINT(39.715683942845786 -74.75440491855277) bank68143 +68144 POINT(41.02950076849123 -74.72583196345722) bank68144 +68145 POINT(40.32834295096422 -74.28756464575602) bank68145 +68146 POINT(41.47190258185564 -73.56462072551228) bank68146 +68147 POINT(40.801660272632475 -74.76834654125682) bank68147 +68148 POINT(41.55977979965147 -74.72717292129981) bank68148 +68149 POINT(40.50305112841263 -74.16123144484028) bank68149 +68150 POINT(41.12887804157649 -74.19204941338593) bank68150 +68151 POINT(40.17042646312745 -74.19269801399422) bank68151 +68152 POINT(41.30125018491871 -73.81746346819365) bank68152 +68153 POINT(41.01864209385636 -73.38452469288315) bank68153 +68154 POINT(40.518798072545856 -73.72598086485908) bank68154 +68155 POINT(40.81731347872297 -74.76899791032321) bank68155 +68156 POINT(41.111186090388635 -74.60590346441738) bank68156 +68157 POINT(41.52511576589671 -74.99843737969782) bank68157 +68158 POINT(41.08500144804603 -73.17038276587989) bank68158 +68159 POINT(40.04981492114637 -73.23724210245638) bank68159 +68160 POINT(40.558048105905776 -74.46756084961474) bank68160 +68161 POINT(40.683487787281685 -74.42224227287278) bank68161 +68162 POINT(40.68886663348165 -73.80152121809817) bank68162 +68163 POINT(41.519768869126736 -74.01558690812536) bank68163 +68164 POINT(39.960762740159716 -73.01302872214343) bank68164 +68165 POINT(41.08742836879176 -73.74695579142688) bank68165 +68166 POINT(41.13673358997168 -74.22740634057587) bank68166 +68167 POINT(40.772481822252026 -73.34072914117752) bank68167 +68168 POINT(40.394636912003186 -74.26387833701064) bank68168 +68169 POINT(41.396946080965755 -74.08532466366914) bank68169 +68170 POINT(41.61321999705174 -74.85920832976836) bank68170 +68171 POINT(41.09066239105879 -74.5527222297279) bank68171 +68172 POINT(40.36855204159162 -73.10463632959885) bank68172 +68173 POINT(40.60163469714939 -74.87373683062161) bank68173 +68174 POINT(41.49048149164075 -73.85019664875465) bank68174 +68175 POINT(41.493833983987734 -73.07794782396738) bank68175 +68176 POINT(41.0953205008476 -73.46041653865099) bank68176 +68177 POINT(41.20225861566132 -73.10995031398005) bank68177 +68178 POINT(40.8013285038833 -73.07225481237934) bank68178 +68179 POINT(40.20445902411181 -73.33828742942524) bank68179 +68180 POINT(41.10791847706339 -73.82383090667187) bank68180 +68181 POINT(39.93669737378404 -74.634526453189) bank68181 +68182 POINT(40.08874014756676 -74.02373448720566) bank68182 +68183 POINT(40.8475288714829 -74.77726648876616) bank68183 +68184 POINT(41.51655445196354 -73.69858115770776) bank68184 +68185 POINT(39.99614410998742 -73.52714692817894) bank68185 +68186 POINT(40.55484293227335 -74.3455585994949) bank68186 +68187 POINT(41.25968536540368 -74.9013008889405) bank68187 +68188 POINT(40.44827223620801 -74.7270957454696) bank68188 +68189 POINT(40.71596918948293 -74.4874176931596) bank68189 +68190 POINT(40.92972364066577 -74.75216967719413) bank68190 +68191 POINT(41.56249922220898 -73.42128677813254) bank68191 +68192 POINT(40.77282271666159 -73.64657425687072) bank68192 +68193 POINT(40.36300683275602 -73.27981468805129) bank68193 +68194 POINT(39.80724649590401 -74.89557799672994) bank68194 +68195 POINT(40.830660369101686 -74.7560814904659) bank68195 +68196 POINT(40.696366196576435 -74.48152991589286) bank68196 +68197 POINT(40.33291397086703 -74.03917889235096) bank68197 +68198 POINT(40.79427547424506 -73.4972117400034) bank68198 +68199 POINT(40.10290121515953 -73.26327467994231) bank68199 +68200 POINT(40.967335671538066 -74.1710719283084) bank68200 +68201 POINT(41.23483227973378 -73.36959000139825) bank68201 +68202 POINT(40.90742246880135 -74.31178983975437) bank68202 +68203 POINT(40.38237149074892 -73.97180366990244) bank68203 +68204 POINT(40.12004090464067 -74.12536460295473) bank68204 +68205 POINT(40.20236211681369 -73.22202074267504) bank68205 +68206 POINT(39.86931132908361 -73.15292895652442) bank68206 +68207 POINT(39.78673560428653 -73.63978999070706) bank68207 +68208 POINT(41.41376425061867 -74.44717397382232) bank68208 +68209 POINT(40.05737808741663 -73.0718497961431) bank68209 +68210 POINT(40.33028566690562 -73.32296872505852) bank68210 +68211 POINT(40.37557018374775 -73.89176921817588) bank68211 +68212 POINT(40.64283604417332 -73.27221642652809) bank68212 +68213 POINT(40.20886456081748 -73.32214859586385) bank68213 +68214 POINT(40.75113724841491 -74.80368939328645) bank68214 +68215 POINT(41.480650751621816 -74.35093528136446) bank68215 +68216 POINT(41.50917875694689 -73.6333054773471) bank68216 +68217 POINT(41.53010354005051 -73.48779532827152) bank68217 +68218 POINT(40.82663568292363 -73.03163870222947) bank68218 +68219 POINT(39.76919148421046 -74.66660151011642) bank68219 +68220 POINT(41.17737050373767 -74.27716192360724) bank68220 +68221 POINT(40.8482548046856 -73.42050266879336) bank68221 +68222 POINT(40.8247690238216 -74.58792328542093) bank68222 +68223 POINT(40.58033650734461 -74.79407564528694) bank68223 +68224 POINT(41.42645336483426 -74.21437562239686) bank68224 +68225 POINT(40.82429740341175 -74.89399991510497) bank68225 +68226 POINT(39.91984667322288 -74.46505067648681) bank68226 +68227 POINT(40.811997480429035 -74.94332919260822) bank68227 +68228 POINT(41.36362483216575 -73.44736453357332) bank68228 +68229 POINT(41.28261490980931 -73.06590624842545) bank68229 +68230 POINT(41.17691590996383 -73.60037048261232) bank68230 +68231 POINT(40.898494569951765 -74.36018427991625) bank68231 +68232 POINT(41.12084138894654 -73.19274946428227) bank68232 +68233 POINT(40.85796508879101 -73.2887035057189) bank68233 +68234 POINT(41.56935971020552 -73.22297176510881) bank68234 +68235 POINT(40.86816831576685 -74.70838163815269) bank68235 +68236 POINT(41.52720612953983 -74.1362263139017) bank68236 +68237 POINT(39.90146603223753 -73.16169485647346) bank68237 +68238 POINT(41.43895755395501 -74.02193143041833) bank68238 +68239 POINT(41.43430861895407 -73.72334925104381) bank68239 +68240 POINT(39.779745031785104 -74.9640892298736) bank68240 +68241 POINT(39.837372530683 -73.6114316216395) bank68241 +68242 POINT(39.83194498430224 -73.67664118767075) bank68242 +68243 POINT(40.699079620063884 -74.76067945343317) bank68243 +68244 POINT(40.4208117728958 -73.06545362068333) bank68244 +68245 POINT(41.47209579247332 -73.94973098473758) bank68245 +68246 POINT(41.067639238514936 -74.97517277061932) bank68246 +68247 POINT(40.8232629362536 -73.46224199134085) bank68247 +68248 POINT(39.80495909027728 -74.44480967004046) bank68248 +68249 POINT(40.130607735248795 -74.05753876734704) bank68249 +68250 POINT(41.45590759057089 -73.81477883220923) bank68250 +68251 POINT(40.58068387017023 -73.55138384629065) bank68251 +68252 POINT(40.86004908726863 -74.63483058000065) bank68252 +68253 POINT(39.975090839985505 -74.58952797199366) bank68253 +68254 POINT(40.02232187024285 -73.21549540472975) bank68254 +68255 POINT(40.26948319956476 -74.4100274493449) bank68255 +68256 POINT(41.570842410583374 -74.10806385063775) bank68256 +68257 POINT(41.63644006774079 -74.12244159472971) bank68257 +68258 POINT(40.64736041287406 -74.72879565751902) bank68258 +68259 POINT(41.67821996475255 -74.99676699450468) bank68259 +68260 POINT(41.67866664559043 -74.17887110143123) bank68260 +68261 POINT(40.18683711955391 -73.2336129105068) bank68261 +68262 POINT(41.46402015993102 -74.72292959362818) bank68262 +68263 POINT(39.75021392521036 -74.35181122143678) bank68263 +68264 POINT(39.91313675765586 -73.34688834016315) bank68264 +68265 POINT(40.787658109153746 -73.8455975424395) bank68265 +68266 POINT(41.034056909017984 -74.79880879124063) bank68266 +68267 POINT(41.31686938107024 -74.71615016770419) bank68267 +68268 POINT(41.19187826356777 -73.23667174005104) bank68268 +68269 POINT(40.1785915260428 -73.96189157603445) bank68269 +68270 POINT(40.91056721338832 -73.43986287513324) bank68270 +68271 POINT(41.622504694400554 -74.50625483904875) bank68271 +68272 POINT(39.75696910245815 -73.95457963988288) bank68272 +68273 POINT(40.55681809528356 -73.90607329495076) bank68273 +68274 POINT(40.65577282032112 -74.96785962317654) bank68274 +68275 POINT(41.31489970505443 -74.25349825804324) bank68275 +68276 POINT(41.24764236118385 -73.78623940554343) bank68276 +68277 POINT(40.45551419435728 -74.05828197896638) bank68277 +68278 POINT(41.60261244231749 -74.17654869220083) bank68278 +68279 POINT(40.88304049343855 -74.03111729688507) bank68279 +68280 POINT(41.59136494456395 -73.59434321460776) bank68280 +68281 POINT(40.00876901938645 -74.96433999713805) bank68281 +68282 POINT(40.6074806504056 -73.84365960873461) bank68282 +68283 POINT(41.70013309334633 -73.267505450677) bank68283 +68284 POINT(40.657916743458614 -73.93652401469231) bank68284 +68285 POINT(40.767725629984035 -73.70550937207388) bank68285 +68286 POINT(40.425776918719414 -74.6898307093124) bank68286 +68287 POINT(40.25004535900654 -74.43942786297103) bank68287 +68288 POINT(40.96020582229308 -74.9610825898112) bank68288 +68289 POINT(39.93230990889175 -73.7395049082345) bank68289 +68290 POINT(40.753263709095734 -74.03352451669139) bank68290 +68291 POINT(41.495845654844665 -73.32886252817796) bank68291 +68292 POINT(41.0409499248466 -74.0582136569962) bank68292 +68293 POINT(40.61721598408365 -73.2907144902148) bank68293 +68294 POINT(40.90081727847025 -73.10062136936149) bank68294 +68295 POINT(40.39913181517531 -74.14684341020175) bank68295 +68296 POINT(39.71825104939824 -74.68087403038976) bank68296 +68297 POINT(41.34812435674392 -74.28971805841341) bank68297 +68298 POINT(41.312689904623596 -73.19420114296379) bank68298 +68299 POINT(41.49062859656863 -74.44149726179174) bank68299 +68300 POINT(41.575877309053155 -73.70754925153165) bank68300 +68301 POINT(40.999020825883655 -74.19981664365869) bank68301 +68302 POINT(41.63414658702611 -74.05755821530771) bank68302 +68303 POINT(40.64141988358151 -73.63574863662352) bank68303 +68304 POINT(40.70883525301861 -74.33105228638915) bank68304 +68305 POINT(41.403584979453655 -73.65195346038519) bank68305 +68306 POINT(40.52461581736608 -73.6042575199001) bank68306 +68307 POINT(41.438175496410985 -74.65944206832006) bank68307 +68308 POINT(41.68091084068725 -73.85960160784016) bank68308 +68309 POINT(39.88917483903299 -73.35237994376521) bank68309 +68310 POINT(40.99633634048023 -73.87652771389463) bank68310 +68311 POINT(40.07396214883044 -73.6446414554191) bank68311 +68312 POINT(40.61770520347092 -74.9537706136049) bank68312 +68313 POINT(40.00468257000486 -73.50529299232817) bank68313 +68314 POINT(39.77745210236955 -74.83112313243973) bank68314 +68315 POINT(40.51775646252466 -73.36890274881655) bank68315 +68316 POINT(41.45515541836402 -74.2669322007245) bank68316 +68317 POINT(41.21490778127876 -74.28256211938884) bank68317 +68318 POINT(41.11504170199612 -73.38382720977172) bank68318 +68319 POINT(41.65200739827816 -74.02338706576457) bank68319 +68320 POINT(41.004292532275294 -73.51800780549499) bank68320 +68321 POINT(40.1873284005835 -73.33257462450628) bank68321 +68322 POINT(41.2865386982717 -74.76856611615575) bank68322 +68323 POINT(40.17160218779658 -74.30553007399251) bank68323 +68324 POINT(40.91177936206185 -74.13783808628484) bank68324 +68325 POINT(40.47656573939239 -73.47284910990705) bank68325 +68326 POINT(41.280134408949394 -73.23834357872022) bank68326 +68327 POINT(40.36485848516301 -73.20316068004118) bank68327 +68328 POINT(41.611397047985115 -73.1645235774704) bank68328 +68329 POINT(41.0837633500256 -73.26657016734791) bank68329 +68330 POINT(40.683169231569615 -73.33026035513902) bank68330 +68331 POINT(40.728520016614475 -73.40357717355599) bank68331 +68332 POINT(40.980633284871544 -73.20706278453373) bank68332 +68333 POINT(39.91028785860875 -74.02962295066399) bank68333 +68334 POINT(40.35121669785532 -73.65574177402279) bank68334 +68335 POINT(40.247345871179114 -73.59597311239794) bank68335 +68336 POINT(40.45401746592632 -73.79807021014923) bank68336 +68337 POINT(40.55319629620116 -73.4809905838553) bank68337 +68338 POINT(40.48706265188343 -74.4828000260596) bank68338 +68339 POINT(41.09030194731511 -73.46763200004989) bank68339 +68340 POINT(41.35875742427206 -74.27142761250852) bank68340 +68341 POINT(39.986985371305664 -73.98859968279714) bank68341 +68342 POINT(40.2336077023268 -73.54826689485371) bank68342 +68343 POINT(40.03292404095493 -74.1638859807043) bank68343 +68344 POINT(41.70487162451991 -73.93671267466205) bank68344 +68345 POINT(41.56422205214017 -73.10075679382659) bank68345 +68346 POINT(40.239509280591555 -74.57243474657278) bank68346 +68347 POINT(40.61527449347903 -73.6799229280676) bank68347 +68348 POINT(40.58490554974793 -74.55842017618228) bank68348 +68349 POINT(40.886757864026116 -74.42588247390984) bank68349 +68350 POINT(40.36603354483551 -73.0354042616484) bank68350 +68351 POINT(41.252090852130635 -73.38079720442589) bank68351 +68352 POINT(40.36729004625218 -73.93312336719919) bank68352 +68353 POINT(41.54715746882982 -74.67920274246596) bank68353 +68354 POINT(41.10654452014485 -74.55140248389556) bank68354 +68355 POINT(39.73910248575954 -73.35276874066416) bank68355 +68356 POINT(40.41245937597654 -74.25471871101695) bank68356 +68357 POINT(41.47430973858233 -73.71436966278469) bank68357 +68358 POINT(40.13754803494971 -74.39382953709791) bank68358 +68359 POINT(40.501406256921 -74.27207905508268) bank68359 +68360 POINT(41.64712089680969 -74.1124567920913) bank68360 +68361 POINT(39.904056499479914 -74.88878930957007) bank68361 +68362 POINT(40.15948272575457 -74.49010894977242) bank68362 +68363 POINT(41.37965075817983 -73.35005543202266) bank68363 +68364 POINT(40.23851737792513 -74.31076535373127) bank68364 +68365 POINT(40.9684106250261 -74.56716721038262) bank68365 +68366 POINT(41.57101576657373 -73.05808015979883) bank68366 +68367 POINT(40.54090750935049 -74.13166629479036) bank68367 +68368 POINT(40.69972379053542 -73.74712275424058) bank68368 +68369 POINT(40.318514703754175 -73.2252456437813) bank68369 +68370 POINT(40.79675122433321 -73.24736925049218) bank68370 +68371 POINT(41.61545234738165 -74.89770989099927) bank68371 +68372 POINT(39.93274856557102 -74.97156366790638) bank68372 +68373 POINT(41.670623052376676 -74.62513528932982) bank68373 +68374 POINT(40.90142899416628 -73.30166900261962) bank68374 +68375 POINT(39.845818958359814 -74.95503360904638) bank68375 +68376 POINT(41.29758114678187 -74.13248827838152) bank68376 +68377 POINT(40.5926927568107 -73.69322879272985) bank68377 +68378 POINT(41.24391571713652 -74.04293048030759) bank68378 +68379 POINT(41.102820293271876 -73.30784305067624) bank68379 +68380 POINT(41.228042038314115 -74.13319595358809) bank68380 +68381 POINT(40.75700937696102 -73.98459735152225) bank68381 +68382 POINT(40.32244960788221 -73.97351789367279) bank68382 +68383 POINT(40.35671491194367 -74.67236604775125) bank68383 +68384 POINT(41.508451664623806 -74.02730585512161) bank68384 +68385 POINT(41.22740150018663 -73.22954400825081) bank68385 +68386 POINT(40.450724880028055 -73.50958864099033) bank68386 +68387 POINT(40.88727675107663 -74.48885339662624) bank68387 +68388 POINT(40.17268022595862 -73.27695044368629) bank68388 +68389 POINT(41.70309459451211 -74.56475254107566) bank68389 +68390 POINT(41.25626787653172 -74.67195340301859) bank68390 +68391 POINT(41.52769641450946 -74.9401795648003) bank68391 +68392 POINT(40.45178986660898 -74.75568158366717) bank68392 +68393 POINT(40.50136732086165 -73.91060809220976) bank68393 +68394 POINT(40.74730442194243 -73.11591775671589) bank68394 +68395 POINT(40.08451002375673 -73.4932974710094) bank68395 +68396 POINT(40.189923181599454 -74.58949161818938) bank68396 +68397 POINT(40.04732404619236 -73.32672353843724) bank68397 +68398 POINT(41.38115045699242 -73.07253528948638) bank68398 +68399 POINT(39.853452463844505 -74.09264020133385) bank68399 +68400 POINT(40.29825111096063 -73.29898453900636) bank68400 +68401 POINT(41.05333159858644 -74.3931117947529) bank68401 +68402 POINT(39.89803434876932 -74.23939421680436) bank68402 +68403 POINT(40.88046794802864 -74.47468195214542) bank68403 +68404 POINT(40.83849898253439 -74.83543574226293) bank68404 +68405 POINT(40.73603750768429 -73.50757299572716) bank68405 +68406 POINT(41.359033062100416 -73.05884423338986) bank68406 +68407 POINT(41.402395173876364 -73.66359725263906) bank68407 +68408 POINT(39.76495173223797 -73.89473165703951) bank68408 +68409 POINT(39.80693708219021 -73.16079369703965) bank68409 +68410 POINT(41.35199405552116 -74.84458530947266) bank68410 +68411 POINT(39.99014132474626 -73.2929655044674) bank68411 +68412 POINT(40.44615500956073 -74.09440884991132) bank68412 +68413 POINT(39.77346044549095 -74.46325407509062) bank68413 +68414 POINT(41.24302884114342 -74.96786224077584) bank68414 +68415 POINT(41.35325134199545 -74.45854182614372) bank68415 +68416 POINT(40.46712719843603 -74.72619956549715) bank68416 +68417 POINT(40.41223192430051 -74.17023627985282) bank68417 +68418 POINT(41.4167067840993 -74.62623938026061) bank68418 +68419 POINT(39.98358964505408 -73.18542398109521) bank68419 +68420 POINT(40.02695359694732 -74.20152144352964) bank68420 +68421 POINT(40.81707249084445 -74.23387435534258) bank68421 +68422 POINT(40.20142827091597 -74.51548232887235) bank68422 +68423 POINT(40.9545288569291 -74.09546636489456) bank68423 +68424 POINT(40.221802378180655 -73.3548754765224) bank68424 +68425 POINT(40.43999441515508 -74.36253299849525) bank68425 +68426 POINT(40.86709809135655 -74.2495360294151) bank68426 +68427 POINT(40.083490929699 -73.25155506727378) bank68427 +68428 POINT(40.09240068961709 -73.0834445568972) bank68428 +68429 POINT(41.183018030802856 -74.04781496788604) bank68429 +68430 POINT(40.861406513156346 -74.76555012573907) bank68430 +68431 POINT(40.84821333961767 -73.21940259750252) bank68431 +68432 POINT(41.404988159491865 -73.61113480433201) bank68432 +68433 POINT(40.65138444323802 -73.40961057272919) bank68433 +68434 POINT(40.8381607077438 -74.30817704133494) bank68434 +68435 POINT(40.004882880271 -73.70938535537272) bank68435 +68436 POINT(41.02038101174468 -73.84782373414936) bank68436 +68437 POINT(40.870708454630936 -74.81830237162308) bank68437 +68438 POINT(41.13202688057918 -74.79200494141622) bank68438 +68439 POINT(41.49587801333201 -74.44211725066909) bank68439 +68440 POINT(41.26027764884658 -74.45597400706285) bank68440 +68441 POINT(41.41982467654331 -74.75514839578157) bank68441 +68442 POINT(40.02929966674742 -73.56876797502001) bank68442 +68443 POINT(41.31387807760589 -73.72961952107114) bank68443 +68444 POINT(40.498790579702316 -73.1731270410271) bank68444 +68445 POINT(40.21359560344902 -73.50676137452643) bank68445 +68446 POINT(39.78794271925195 -74.25248469204092) bank68446 +68447 POINT(41.365258638304454 -73.58937250241024) bank68447 +68448 POINT(40.06070204334749 -74.83862549977334) bank68448 +68449 POINT(40.745167829730185 -73.76659696629338) bank68449 +68450 POINT(40.9956436846459 -74.92459623932788) bank68450 +68451 POINT(39.778299061037664 -74.78224128736497) bank68451 +68452 POINT(41.67994294137561 -74.905095184445) bank68452 +68453 POINT(40.35512346422402 -74.50607370992172) bank68453 +68454 POINT(40.446667724319106 -74.20829641291347) bank68454 +68455 POINT(39.95762245281384 -73.7443027750613) bank68455 +68456 POINT(41.571467420434736 -73.90350049413075) bank68456 +68457 POINT(41.11858878035574 -74.04111621434288) bank68457 +68458 POINT(40.63071552156732 -73.72074183790566) bank68458 +68459 POINT(39.72928439058252 -73.39848101658228) bank68459 +68460 POINT(39.7559863482119 -73.97522919735769) bank68460 +68461 POINT(39.93980046977875 -74.48998619227369) bank68461 +68462 POINT(41.649571106881126 -73.78626719253441) bank68462 +68463 POINT(41.62415109288648 -74.01551000068447) bank68463 +68464 POINT(40.2210606881959 -73.41298798319103) bank68464 +68465 POINT(40.36253282708954 -73.51834281995836) bank68465 +68466 POINT(41.63143039685223 -74.18449013221446) bank68466 +68467 POINT(39.91761059573912 -73.4634851494043) bank68467 +68468 POINT(40.85024037276916 -73.79028705930475) bank68468 +68469 POINT(39.95501332306307 -73.3009961618811) bank68469 +68470 POINT(41.285316876184055 -73.80387471688942) bank68470 +68471 POINT(41.46482405324491 -74.0867988288827) bank68471 +68472 POINT(41.1242021204617 -74.95804890583427) bank68472 +68473 POINT(40.1173698448779 -74.08586078473277) bank68473 +68474 POINT(39.90811546360807 -74.23939918085468) bank68474 +68475 POINT(41.58988442224778 -74.86565039393284) bank68475 +68476 POINT(40.72837718943169 -73.51308105528085) bank68476 +68477 POINT(40.749962133212264 -73.90030157415775) bank68477 +68478 POINT(39.942371696234474 -73.57842845724953) bank68478 +68479 POINT(39.78996490631704 -73.01601584598532) bank68479 +68480 POINT(41.17750663897286 -73.70506369238562) bank68480 +68481 POINT(41.643614810994634 -74.18528638565424) bank68481 +68482 POINT(41.27724544421212 -74.62963730159245) bank68482 +68483 POINT(41.484248952410525 -73.82588194209035) bank68483 +68484 POINT(41.383771037752794 -73.37525310331739) bank68484 +68485 POINT(41.436032518039006 -73.18810114581588) bank68485 +68486 POINT(40.26268849043067 -73.31649939866429) bank68486 +68487 POINT(41.122539833798186 -74.23638271707964) bank68487 +68488 POINT(41.02534255613251 -73.22578524928723) bank68488 +68489 POINT(40.52638216218779 -73.33211300773809) bank68489 +68490 POINT(40.056390060890735 -73.5546366913728) bank68490 +68491 POINT(39.935514803573106 -73.42703374398478) bank68491 +68492 POINT(40.483217504102065 -74.3535343090276) bank68492 +68493 POINT(41.235922390668755 -73.11101611256372) bank68493 +68494 POINT(40.18683578635069 -74.42021641643666) bank68494 +68495 POINT(39.95533576533014 -73.56300391064873) bank68495 +68496 POINT(40.34332637561981 -73.78972161499907) bank68496 +68497 POINT(41.63348124038897 -73.84746207259036) bank68497 +68498 POINT(41.61392768567107 -73.72387462584287) bank68498 +68499 POINT(39.8741356084078 -73.51721926815263) bank68499 +68500 POINT(40.31424864909642 -74.46813928928115) bank68500 +68501 POINT(41.4188583882035 -74.28061217591375) bank68501 +68502 POINT(40.202332233252 -74.74882859088213) bank68502 +68503 POINT(40.28494744346204 -73.32148542341409) bank68503 +68504 POINT(40.54954484750975 -74.0671481080392) bank68504 +68505 POINT(41.40645002152747 -73.15352430511881) bank68505 +68506 POINT(39.89876366135994 -74.45299644758214) bank68506 +68507 POINT(40.56008186238804 -74.067490357656) bank68507 +68508 POINT(40.44962048189052 -73.1399067670669) bank68508 +68509 POINT(41.120881387650186 -74.35131424599311) bank68509 +68510 POINT(40.238139563785886 -73.83738349040908) bank68510 +68511 POINT(40.14612821096451 -74.73658187734443) bank68511 +68512 POINT(40.619349417026946 -73.01315693220432) bank68512 +68513 POINT(41.67534608478015 -74.18010878622776) bank68513 +68514 POINT(41.09680396005569 -75.00483694299213) bank68514 +68515 POINT(41.4264851425162 -74.88315116034403) bank68515 +68516 POINT(41.39246346436026 -73.87285000186635) bank68516 +68517 POINT(41.12666003067772 -73.63837757458634) bank68517 +68518 POINT(41.596718057773245 -74.19334991841605) bank68518 +68519 POINT(39.99120855443664 -73.57939452627024) bank68519 +68520 POINT(40.0838324804906 -74.89997125501155) bank68520 +68521 POINT(41.464685542987546 -73.32222350737388) bank68521 +68522 POINT(41.682416941461135 -73.10204209542574) bank68522 +68523 POINT(40.38337041691611 -74.68312733672136) bank68523 +68524 POINT(40.854735877772505 -74.3436193943385) bank68524 +68525 POINT(40.611124698078235 -74.61986620442609) bank68525 +68526 POINT(40.13003135877071 -74.19581006674244) bank68526 +68527 POINT(40.81652480851767 -74.40481056302616) bank68527 +68528 POINT(39.72309943256638 -73.6471019705176) bank68528 +68529 POINT(40.43780401966627 -73.76191335477097) bank68529 +68530 POINT(40.19502848519752 -74.93176471268764) bank68530 +68531 POINT(41.050026528148884 -73.42575183583277) bank68531 +68532 POINT(39.737765637736075 -73.10733511940238) bank68532 +68533 POINT(40.70914967181146 -73.27304368867786) bank68533 +68534 POINT(40.77155872069834 -74.82285762769413) bank68534 +68535 POINT(41.30701503374481 -73.7456053402432) bank68535 +68536 POINT(41.67325829823014 -73.70898541265761) bank68536 +68537 POINT(41.31999145279625 -73.46653491221088) bank68537 +68538 POINT(41.41203073277236 -73.79549340884179) bank68538 +68539 POINT(40.779653605044516 -73.444430249744) bank68539 +68540 POINT(40.74650472500209 -73.93019082989937) bank68540 +68541 POINT(40.019918566671045 -73.78955557332527) bank68541 +68542 POINT(40.32375488719168 -73.79457004392768) bank68542 +68543 POINT(40.68058353648991 -73.43166730660514) bank68543 +68544 POINT(41.596925757922484 -74.38004940301295) bank68544 +68545 POINT(41.336507685801976 -74.83536558725147) bank68545 +68546 POINT(40.54892309012017 -73.08747736739977) bank68546 +68547 POINT(41.18976310304545 -74.09509179635698) bank68547 +68548 POINT(39.821373557751045 -73.07486752614186) bank68548 +68549 POINT(41.24797690689437 -73.46347407594226) bank68549 +68550 POINT(41.6686010390589 -73.38861243207695) bank68550 +68551 POINT(41.347831756646215 -74.61535520486471) bank68551 +68552 POINT(41.08682417356388 -74.67647450574991) bank68552 +68553 POINT(40.103837279369046 -73.73168854669987) bank68553 +68554 POINT(41.44168809332132 -74.89437496947842) bank68554 +68555 POINT(41.46846719686462 -74.57665234991912) bank68555 +68556 POINT(39.939192994879626 -73.62446610461565) bank68556 +68557 POINT(41.25527393666573 -73.22562234828182) bank68557 +68558 POINT(41.501024091342934 -74.13002628901116) bank68558 +68559 POINT(40.7923278205382 -73.64123067881538) bank68559 +68560 POINT(41.11965394126013 -74.03801157507824) bank68560 +68561 POINT(39.85549017791206 -74.01840403557904) bank68561 +68562 POINT(40.40186068845554 -74.83140206014677) bank68562 +68563 POINT(40.9813682321423 -73.24361980145841) bank68563 +68564 POINT(40.49638997679118 -74.32421010503943) bank68564 +68565 POINT(40.12510278767186 -74.76898570977394) bank68565 +68566 POINT(40.5986007770576 -74.56635719364282) bank68566 +68567 POINT(39.96692411510713 -73.31758121913902) bank68567 +68568 POINT(41.429267087933404 -74.11266997365192) bank68568 +68569 POINT(40.33158040745535 -73.03666491852391) bank68569 +68570 POINT(40.545871525679246 -74.79905308220239) bank68570 +68571 POINT(41.08835042167378 -73.58574023603822) bank68571 +68572 POINT(41.70631110800305 -74.58476776042015) bank68572 +68573 POINT(41.256746637866414 -73.28494151593947) bank68573 +68574 POINT(40.219669207300655 -74.65926935814994) bank68574 +68575 POINT(40.27248989788936 -74.0356539501729) bank68575 +68576 POINT(40.563637104343 -74.99206993724303) bank68576 +68577 POINT(40.8303472726091 -73.77727283041534) bank68577 +68578 POINT(40.64400540352899 -74.09556323096314) bank68578 +68579 POINT(40.5234240397931 -74.98610731935786) bank68579 +68580 POINT(41.1919295373796 -73.73565858110344) bank68580 +68581 POINT(40.4533488079863 -74.95947378379722) bank68581 +68582 POINT(41.40314220349651 -74.81286141175289) bank68582 +68583 POINT(39.767299444124895 -74.47777561255256) bank68583 +68584 POINT(40.812381932730396 -74.32839429825107) bank68584 +68585 POINT(39.73349758234311 -73.65888936758674) bank68585 +68586 POINT(40.28781102284454 -74.5317237067516) bank68586 +68587 POINT(41.64815697740165 -73.58915987616659) bank68587 +68588 POINT(41.19821524619295 -74.2816833075737) bank68588 +68589 POINT(40.749252508094386 -73.30416817742136) bank68589 +68590 POINT(40.79502855524087 -74.99713198244878) bank68590 +68591 POINT(41.401929537711425 -74.16064258581822) bank68591 +68592 POINT(39.78080913915351 -73.93868896356247) bank68592 +68593 POINT(41.30428540290565 -74.2427671393135) bank68593 +68594 POINT(41.443190070130385 -73.60113050217566) bank68594 +68595 POINT(41.14236180826214 -73.29592206914255) bank68595 +68596 POINT(41.395635592057175 -73.40149498521356) bank68596 +68597 POINT(41.27652379851828 -74.67437450817403) bank68597 +68598 POINT(41.37690142055034 -74.18511254851067) bank68598 +68599 POINT(39.81334337270017 -73.97554563433185) bank68599 +68600 POINT(40.44259135851769 -73.94744012166862) bank68600 +68601 POINT(39.72898855026719 -73.10685051558387) bank68601 +68602 POINT(40.457504783997635 -73.20487649644836) bank68602 +68603 POINT(39.76271368190156 -73.35465297462143) bank68603 +68604 POINT(40.36223429627352 -73.58898521197409) bank68604 +68605 POINT(41.16555165981126 -74.45432039512492) bank68605 +68606 POINT(41.00729038394776 -73.35172419874829) bank68606 +68607 POINT(41.37072763721085 -74.68253367881884) bank68607 +68608 POINT(40.763349861370614 -74.33356186421676) bank68608 +68609 POINT(40.63764744538983 -74.2473760658632) bank68609 +68610 POINT(41.69421521924286 -74.41631848449035) bank68610 +68611 POINT(41.45744764159291 -74.41420880956348) bank68611 +68612 POINT(41.443226670845135 -73.90319573748364) bank68612 +68613 POINT(41.264071885853596 -73.77409527716067) bank68613 +68614 POINT(39.9438556793544 -74.71590010146782) bank68614 +68615 POINT(41.22704832265663 -74.77499041276886) bank68615 +68616 POINT(40.11755760427881 -73.0353303806393) bank68616 +68617 POINT(39.72294766919819 -74.13443166250231) bank68617 +68618 POINT(41.326667329169304 -74.35398491525208) bank68618 +68619 POINT(41.58112343977597 -74.56226146029985) bank68619 +68620 POINT(41.55554825080641 -74.57902257131943) bank68620 +68621 POINT(40.41888344667116 -74.66392668863054) bank68621 +68622 POINT(40.88480040949752 -74.43867737168) bank68622 +68623 POINT(41.5285038935833 -73.07492980944198) bank68623 +68624 POINT(40.72529353155396 -74.22431502627951) bank68624 +68625 POINT(40.01926555662279 -75.00465257338605) bank68625 +68626 POINT(40.965957385922344 -74.88161400901336) bank68626 +68627 POINT(40.9322288798299 -73.70882810995256) bank68627 +68628 POINT(41.28497439328426 -73.32013999886958) bank68628 +68629 POINT(41.24241172283456 -74.67514368189059) bank68629 +68630 POINT(39.82201450518007 -74.92406630963116) bank68630 +68631 POINT(41.59212608106677 -74.45062653886083) bank68631 +68632 POINT(40.08075859846877 -73.55731223719067) bank68632 +68633 POINT(40.22899235370979 -74.73632717572896) bank68633 +68634 POINT(40.73839557509517 -73.93393057604922) bank68634 +68635 POINT(40.910792824244766 -74.43989068895675) bank68635 +68636 POINT(41.210360594149755 -74.96093897409148) bank68636 +68637 POINT(41.14959135735947 -73.08671372592465) bank68637 +68638 POINT(40.34951344836363 -73.29772421992249) bank68638 +68639 POINT(40.77867813607935 -73.21614752752562) bank68639 +68640 POINT(41.21080709664674 -74.25485211606718) bank68640 +68641 POINT(40.82098854448492 -73.5344939547891) bank68641 +68642 POINT(40.8887859076369 -73.55172799206547) bank68642 +68643 POINT(40.77619769393482 -73.09024298227614) bank68643 +68644 POINT(40.50779458454562 -74.79868389439082) bank68644 +68645 POINT(41.54475931471126 -73.20948760482462) bank68645 +68646 POINT(40.45224130897891 -73.01836054897035) bank68646 +68647 POINT(41.61575999299309 -73.78629139762864) bank68647 +68648 POINT(40.254422349705514 -73.73711203332486) bank68648 +68649 POINT(40.82640550851943 -74.67567180300279) bank68649 +68650 POINT(40.57694306275625 -73.27622271462923) bank68650 +68651 POINT(39.728625111895106 -73.94377127243658) bank68651 +68652 POINT(40.863502326412025 -74.06692421700369) bank68652 +68653 POINT(41.64463218758205 -74.0466211262869) bank68653 +68654 POINT(41.10320067753121 -74.65501605734713) bank68654 +68655 POINT(41.58142901764293 -74.63866380750864) bank68655 +68656 POINT(40.69380337057326 -73.69203104143484) bank68656 +68657 POINT(41.43615509358874 -74.94996474636794) bank68657 +68658 POINT(40.76382048181309 -73.53330940553474) bank68658 +68659 POINT(41.29265808707342 -73.5462302988406) bank68659 +68660 POINT(41.305150181866914 -73.68749884211165) bank68660 +68661 POINT(40.101294946665085 -73.02602402541515) bank68661 +68662 POINT(40.084172779073775 -74.53665930325924) bank68662 +68663 POINT(41.54708259521099 -73.01216809471711) bank68663 +68664 POINT(41.21958381959803 -73.56447302697202) bank68664 +68665 POINT(39.984055258774035 -73.72633395880752) bank68665 +68666 POINT(41.072718392827056 -74.67690114843406) bank68666 +68667 POINT(41.45436137676031 -73.50223907531957) bank68667 +68668 POINT(40.937011030219224 -74.24104309339783) bank68668 +68669 POINT(40.28111672296754 -74.63944024424814) bank68669 +68670 POINT(40.92702570323335 -74.05252743887227) bank68670 +68671 POINT(40.597970221596235 -74.4699477987011) bank68671 +68672 POINT(40.19289744238571 -74.51770276016241) bank68672 +68673 POINT(41.10848270706998 -73.88588319571981) bank68673 +68674 POINT(41.12866644018096 -73.75100676052057) bank68674 +68675 POINT(39.80508349800264 -73.40573732032657) bank68675 +68676 POINT(40.104360741316796 -73.73554264423207) bank68676 +68677 POINT(40.41686151676466 -73.4091053541433) bank68677 +68678 POINT(40.96553464175997 -74.00961360809443) bank68678 +68679 POINT(40.857903799546406 -74.26539799956323) bank68679 +68680 POINT(40.443826464282886 -74.23146514301624) bank68680 +68681 POINT(40.67771568041088 -74.09345119939273) bank68681 +68682 POINT(41.42760210308367 -73.44380497974862) bank68682 +68683 POINT(40.72567631477301 -73.22974238984754) bank68683 +68684 POINT(40.87236244318429 -73.99978319647505) bank68684 +68685 POINT(40.531265087177566 -74.043792649713) bank68685 +68686 POINT(40.61741876301761 -74.6585771038851) bank68686 +68687 POINT(39.948266862929124 -73.1941823717062) bank68687 +68688 POINT(40.68225879774273 -73.71506576994044) bank68688 +68689 POINT(41.31483609973303 -73.69562183726947) bank68689 +68690 POINT(40.91520666740183 -74.59868471447594) bank68690 +68691 POINT(41.64689634592536 -74.83345695813804) bank68691 +68692 POINT(40.32546873755319 -73.77682551311472) bank68692 +68693 POINT(40.52370611098698 -73.96182874374738) bank68693 +68694 POINT(40.62188639866948 -73.94361927594221) bank68694 +68695 POINT(41.2225885904551 -73.69584017295693) bank68695 +68696 POINT(41.10495876833334 -73.90697849670389) bank68696 +68697 POINT(41.23550007076825 -74.21256730103153) bank68697 +68698 POINT(40.630960594488165 -74.27003923144228) bank68698 +68699 POINT(40.65638984633873 -73.19978601788554) bank68699 +68700 POINT(40.989585497594625 -73.40589082496672) bank68700 +68701 POINT(40.524463260770275 -74.05253330355869) bank68701 +68702 POINT(41.009652256793146 -73.37839653961547) bank68702 +68703 POINT(40.22366815689348 -74.3539983083373) bank68703 +68704 POINT(41.182739597981275 -73.05930612378673) bank68704 +68705 POINT(41.1569955615651 -74.75182106930627) bank68705 +68706 POINT(41.4886916164351 -74.58244233729882) bank68706 +68707 POINT(41.532529799679594 -73.16982623891133) bank68707 +68708 POINT(40.76952692651018 -73.83678590874607) bank68708 +68709 POINT(40.96473746556092 -73.34948087490429) bank68709 +68710 POINT(41.55125890674564 -74.33459753846626) bank68710 +68711 POINT(40.71640588904567 -73.40754203108392) bank68711 +68712 POINT(40.70956772384285 -74.11183579353454) bank68712 +68713 POINT(41.080612886582074 -74.53721358991935) bank68713 +68714 POINT(40.1898687687599 -74.8686504905161) bank68714 +68715 POINT(41.00965552208643 -73.03239789234482) bank68715 +68716 POINT(41.42251558769037 -73.36708675157459) bank68716 +68717 POINT(41.58158985749436 -73.38985327273012) bank68717 +68718 POINT(41.536362222083746 -73.59253490445171) bank68718 +68719 POINT(41.348663504102056 -73.94365332945044) bank68719 +68720 POINT(41.02488831603391 -74.72901674708908) bank68720 +68721 POINT(41.03667717516124 -73.18079799778519) bank68721 +68722 POINT(41.220370206750346 -73.86651220485321) bank68722 +68723 POINT(40.291062237558485 -74.07892696974778) bank68723 +68724 POINT(40.92699604280256 -74.19911432499478) bank68724 +68725 POINT(41.236254405521336 -73.35622118500895) bank68725 +68726 POINT(40.23617303366672 -73.20556854540602) bank68726 +68727 POINT(40.28356121262197 -74.14323695597452) bank68727 +68728 POINT(40.59415140797241 -73.33127851473445) bank68728 +68729 POINT(40.55017612905497 -74.75923855285993) bank68729 +68730 POINT(40.17091972075039 -74.91606860253988) bank68730 +68731 POINT(40.118445552588994 -74.85379247051904) bank68731 +68732 POINT(41.16168460545103 -74.35794622620595) bank68732 +68733 POINT(41.44207434248609 -73.78597629393886) bank68733 +68734 POINT(40.90525372747253 -73.48477850511665) bank68734 +68735 POINT(40.62696260636406 -74.23610987006958) bank68735 +68736 POINT(40.21971315051599 -74.8412151357286) bank68736 +68737 POINT(41.170648409448624 -74.88443991849827) bank68737 +68738 POINT(40.133376066740816 -74.48413654871355) bank68738 +68739 POINT(40.1131618244862 -74.0015976243649) bank68739 +68740 POINT(41.35262489717481 -74.8247020898649) bank68740 +68741 POINT(40.74204449297426 -74.17795869394608) bank68741 +68742 POINT(41.64280397454334 -73.70695536065826) bank68742 +68743 POINT(41.68758069300116 -74.86498501045916) bank68743 +68744 POINT(40.41893669731075 -73.57663414238654) bank68744 +68745 POINT(40.44821907454736 -74.2066939190637) bank68745 +68746 POINT(40.099044704714274 -73.67141912287207) bank68746 +68747 POINT(40.23000079140648 -74.7920116690204) bank68747 +68748 POINT(41.2830164034484 -73.09739661812623) bank68748 +68749 POINT(40.28692208707802 -73.9255250889017) bank68749 +68750 POINT(39.87394142506316 -73.10800625008426) bank68750 +68751 POINT(40.281864566911814 -73.79362526039826) bank68751 +68752 POINT(41.50453491720565 -74.25677960040193) bank68752 +68753 POINT(40.269515211050056 -73.55475343163549) bank68753 +68754 POINT(40.054613406666945 -73.58991301865468) bank68754 +68755 POINT(40.75245115629982 -74.96932529867408) bank68755 +68756 POINT(40.4422169829653 -74.21559875015997) bank68756 +68757 POINT(41.545512193562246 -74.73254177892485) bank68757 +68758 POINT(41.416653989106926 -73.12700079979635) bank68758 +68759 POINT(40.962420851941516 -73.65575782167998) bank68759 +68760 POINT(39.910429544975074 -74.98656914507859) bank68760 +68761 POINT(40.48658640991157 -73.40198747478496) bank68761 +68762 POINT(41.22390709720251 -73.5572849036379) bank68762 +68763 POINT(41.11114030854727 -74.33805654090499) bank68763 +68764 POINT(41.70244716395799 -73.38807931561033) bank68764 +68765 POINT(40.493340580918485 -73.51816264520095) bank68765 +68766 POINT(40.14206751545747 -73.75275613228624) bank68766 +68767 POINT(41.31197473132025 -73.43119218479553) bank68767 +68768 POINT(41.31280102302129 -74.12495136325211) bank68768 +68769 POINT(41.62148843575719 -74.92646087545894) bank68769 +68770 POINT(40.80756276607384 -73.03472733988822) bank68770 +68771 POINT(41.339908974020496 -73.1587925868108) bank68771 +68772 POINT(41.59611316885915 -74.79127097544283) bank68772 +68773 POINT(41.57312441660402 -74.87067547638321) bank68773 +68774 POINT(40.2203700058588 -74.44415628080083) bank68774 +68775 POINT(40.486457894966 -74.32761536468982) bank68775 +68776 POINT(40.74536520146399 -73.17684923332521) bank68776 +68777 POINT(40.6947231122536 -74.63066851835585) bank68777 +68778 POINT(40.74033165123466 -74.41125072487787) bank68778 +68779 POINT(40.28954076300886 -73.30509694559336) bank68779 +68780 POINT(41.66092365516783 -73.69952344472418) bank68780 +68781 POINT(39.9493712777138 -74.24617046644232) bank68781 +68782 POINT(40.70170839942557 -73.84201477800296) bank68782 +68783 POINT(41.612611685971174 -74.57993899923957) bank68783 +68784 POINT(40.09442895459584 -73.49335600917405) bank68784 +68785 POINT(41.19991998543864 -74.64785021313675) bank68785 +68786 POINT(40.166822779978226 -74.35005458203813) bank68786 +68787 POINT(41.68899696305934 -74.73564740515185) bank68787 +68788 POINT(41.235878738149154 -73.25682734051222) bank68788 +68789 POINT(41.64958472284445 -74.34425440383775) bank68789 +68790 POINT(41.3498664521427 -73.59246671211243) bank68790 +68791 POINT(39.96969972171858 -74.49232688437345) bank68791 +68792 POINT(40.13964303889167 -74.91556699095551) bank68792 +68793 POINT(39.86431981973139 -74.01720811221094) bank68793 +68794 POINT(41.119888785149804 -74.79714725248826) bank68794 +68795 POINT(40.59776887411833 -73.82029580395502) bank68795 +68796 POINT(39.92699180683184 -73.77070751257004) bank68796 +68797 POINT(40.887066195932924 -73.93710225137995) bank68797 +68798 POINT(41.589015127635484 -74.68808534769293) bank68798 +68799 POINT(40.64945088591016 -73.11556584801251) bank68799 +68800 POINT(41.50380676133321 -74.03655219384457) bank68800 +68801 POINT(41.294664293393225 -73.67452819801107) bank68801 +68802 POINT(40.37412793688949 -74.64424376505883) bank68802 +68803 POINT(40.017172161414464 -73.51617088385636) bank68803 +68804 POINT(40.754814806108406 -73.83149843970787) bank68804 +68805 POINT(40.328616856365365 -73.28955278681077) bank68805 +68806 POINT(40.720113982372204 -74.39865163728305) bank68806 +68807 POINT(40.87092544669566 -74.99636750359348) bank68807 +68808 POINT(40.11976446554851 -74.44690010655624) bank68808 +68809 POINT(41.23982779798718 -73.96479129298167) bank68809 +68810 POINT(40.53501045417933 -74.95601720902187) bank68810 +68811 POINT(41.15420636323454 -73.54423944581241) bank68811 +68812 POINT(41.19673015534843 -73.009746502615) bank68812 +68813 POINT(40.6332506343019 -74.64550355936665) bank68813 +68814 POINT(41.1080044379322 -73.5461777109333) bank68814 +68815 POINT(40.67890186856226 -73.27500907039088) bank68815 +68816 POINT(41.05228886011043 -74.46976481680508) bank68816 +68817 POINT(39.80522101115141 -73.85204380152798) bank68817 +68818 POINT(40.27520973221569 -73.05455252993336) bank68818 +68819 POINT(41.095292547574715 -73.73970559153338) bank68819 +68820 POINT(41.41371382099642 -74.01600758846766) bank68820 +68821 POINT(41.256365899406084 -74.78841997836619) bank68821 +68822 POINT(41.55454586145101 -73.04898480458563) bank68822 +68823 POINT(40.61978695120209 -73.37945300948844) bank68823 +68824 POINT(39.71589296054594 -73.23208887666607) bank68824 +68825 POINT(40.35507624414046 -74.29776050998001) bank68825 +68826 POINT(40.31181866993059 -73.35529090385646) bank68826 +68827 POINT(41.31178936992657 -74.34440653704715) bank68827 +68828 POINT(40.67025626429145 -73.9685390300341) bank68828 +68829 POINT(39.795236069389965 -74.31127193879763) bank68829 +68830 POINT(41.38826271206436 -74.37744900211815) bank68830 +68831 POINT(41.1402187022001 -73.54732825274905) bank68831 +68832 POINT(40.6416641497573 -73.38150315172516) bank68832 +68833 POINT(39.74191191176739 -74.33604026562486) bank68833 +68834 POINT(41.663619807707704 -74.93694933551295) bank68834 +68835 POINT(41.237214736314776 -73.3053852842917) bank68835 +68836 POINT(41.534367174332274 -74.03541154466488) bank68836 +68837 POINT(40.53652797356495 -74.50495252172077) bank68837 +68838 POINT(40.02961352960849 -73.04043298355647) bank68838 +68839 POINT(41.0688197666415 -73.38767473483924) bank68839 +68840 POINT(40.82365662513889 -73.16973816009978) bank68840 +68841 POINT(40.6835042921528 -74.42242094326915) bank68841 +68842 POINT(41.183896928123275 -74.9822517726414) bank68842 +68843 POINT(40.98377907376676 -73.3064135725286) bank68843 +68844 POINT(41.09975419740104 -73.10017609281434) bank68844 +68845 POINT(41.547158543936646 -73.82413520880985) bank68845 +68846 POINT(40.48781306355947 -73.8632830374058) bank68846 +68847 POINT(41.01361269398792 -74.84667670297142) bank68847 +68848 POINT(40.83791621200098 -74.8921077252589) bank68848 +68849 POINT(39.78850029609432 -74.61921720600306) bank68849 +68850 POINT(41.679838684443034 -74.1277928154676) bank68850 +68851 POINT(40.11976451875761 -73.9344861184746) bank68851 +68852 POINT(41.510753361865454 -73.86793067435192) bank68852 +68853 POINT(40.03027129995378 -73.9813447842305) bank68853 +68854 POINT(40.59355472432437 -74.76494426252093) bank68854 +68855 POINT(40.65102296285855 -73.66282149967671) bank68855 +68856 POINT(40.84691272146842 -74.60295068191529) bank68856 +68857 POINT(40.389666502785836 -73.50825026746215) bank68857 +68858 POINT(39.78861674884327 -73.61500101747313) bank68858 +68859 POINT(41.342421711430276 -73.99155823679722) bank68859 +68860 POINT(40.975101993937756 -74.1019528525211) bank68860 +68861 POINT(41.473308503266296 -73.96510038333616) bank68861 +68862 POINT(39.80641422605095 -73.26625335794691) bank68862 +68863 POINT(41.67169081227394 -74.74865752523206) bank68863 +68864 POINT(40.25428505367139 -73.53701767790326) bank68864 +68865 POINT(40.182238591523195 -74.64137945870289) bank68865 +68866 POINT(40.13358711188771 -74.75885908629142) bank68866 +68867 POINT(40.56501053240499 -74.45159987577185) bank68867 +68868 POINT(41.024019130128316 -73.60893705017538) bank68868 +68869 POINT(40.31137556761318 -74.12359313766119) bank68869 +68870 POINT(41.551860958626904 -74.82891901598745) bank68870 +68871 POINT(39.72743429223379 -74.16571710547942) bank68871 +68872 POINT(40.84843837336136 -74.41137978865272) bank68872 +68873 POINT(41.48704088149733 -73.91178856167605) bank68873 +68874 POINT(40.81442640742066 -73.35218131702277) bank68874 +68875 POINT(41.005569763735515 -74.2140484715331) bank68875 +68876 POINT(40.866815340824935 -74.15679025392927) bank68876 +68877 POINT(40.66207646606175 -73.66240536827254) bank68877 +68878 POINT(39.979187578206115 -74.22700593358734) bank68878 +68879 POINT(40.22764850616842 -74.44961941528034) bank68879 +68880 POINT(40.947202991575175 -73.51106579572684) bank68880 +68881 POINT(40.7594157757696 -73.39315852105736) bank68881 +68882 POINT(41.020870961975795 -74.16504871984071) bank68882 +68883 POINT(40.58639633140872 -73.0503676554353) bank68883 +68884 POINT(41.647991240506784 -74.67711528470412) bank68884 +68885 POINT(40.72090156319485 -73.64932769716803) bank68885 +68886 POINT(40.03337513746991 -73.40675233214772) bank68886 +68887 POINT(40.050663493755366 -73.55014727343848) bank68887 +68888 POINT(40.43537834061675 -74.92525564659573) bank68888 +68889 POINT(40.59242137090953 -74.95261497636916) bank68889 +68890 POINT(41.65897828255462 -74.28920973412401) bank68890 +68891 POINT(39.71800769018964 -73.4277309952201) bank68891 +68892 POINT(40.04456693676605 -74.95502877339027) bank68892 +68893 POINT(41.5176021185786 -73.79995106179405) bank68893 +68894 POINT(40.66904872693364 -73.60176447105819) bank68894 +68895 POINT(41.35076368457682 -73.27172715201085) bank68895 +68896 POINT(39.81928084115544 -74.15353326621471) bank68896 +68897 POINT(40.30588496210759 -73.47093776481142) bank68897 +68898 POINT(40.372969903534475 -73.60881833791719) bank68898 +68899 POINT(39.97426289850929 -74.85671901220486) bank68899 +68900 POINT(40.79140442189358 -74.76318710691041) bank68900 +68901 POINT(40.71858800967239 -73.32966529633127) bank68901 +68902 POINT(40.052724157500975 -74.25275254076863) bank68902 +68903 POINT(41.098931298678835 -73.08568646944849) bank68903 +68904 POINT(40.85866799021003 -74.36492848893928) bank68904 +68905 POINT(40.59712911856547 -73.20318410015565) bank68905 +68906 POINT(40.69563103611246 -74.12754643583611) bank68906 +68907 POINT(40.94605745953518 -74.61667652274605) bank68907 +68908 POINT(41.38665917264348 -74.0100319413218) bank68908 +68909 POINT(40.83497985204059 -73.26281915317648) bank68909 +68910 POINT(39.741624740203044 -74.28165757924587) bank68910 +68911 POINT(40.61922376049601 -74.18868312564085) bank68911 +68912 POINT(40.39856370025606 -73.10025004086792) bank68912 +68913 POINT(40.12829987299321 -73.94045051685804) bank68913 +68914 POINT(40.23980393685186 -73.38307103495941) bank68914 +68915 POINT(40.08327153131413 -74.49715502100122) bank68915 +68916 POINT(40.10361264375134 -74.7826624435141) bank68916 +68917 POINT(40.444238614824215 -74.54751250934005) bank68917 +68918 POINT(40.609181697752035 -73.3544860787746) bank68918 +68919 POINT(39.74171992381243 -74.72454311116222) bank68919 +68920 POINT(39.916650142766684 -73.13065937153318) bank68920 +68921 POINT(39.937335064424644 -73.92637887263243) bank68921 +68922 POINT(41.56969608026465 -73.31336950144014) bank68922 +68923 POINT(40.655710149468455 -73.54333897664239) bank68923 +68924 POINT(41.39934754143767 -74.79658906990392) bank68924 +68925 POINT(41.42455344884021 -74.75915040900057) bank68925 +68926 POINT(40.89062920682289 -74.57655896967005) bank68926 +68927 POINT(39.81363277415506 -74.20016372630849) bank68927 +68928 POINT(40.05294118954611 -74.62641405715046) bank68928 +68929 POINT(40.87857583178566 -74.11047114574826) bank68929 +68930 POINT(40.47101854125742 -74.76550962988657) bank68930 +68931 POINT(40.89064424004261 -73.95574553044479) bank68931 +68932 POINT(40.65775886086353 -73.31359675751042) bank68932 +68933 POINT(41.54266268864727 -73.7127078113034) bank68933 +68934 POINT(40.88287854905137 -73.03263506296645) bank68934 +68935 POINT(41.39834265231357 -73.15673026941133) bank68935 +68936 POINT(41.41827929145735 -73.68612811070136) bank68936 +68937 POINT(40.72231282561638 -74.30272086474226) bank68937 +68938 POINT(39.922305979926286 -74.11661854634751) bank68938 +68939 POINT(40.425376364902945 -74.61765090110036) bank68939 +68940 POINT(41.09088682210867 -74.05826113336356) bank68940 +68941 POINT(40.20928487846402 -74.771525676937) bank68941 +68942 POINT(39.86626577660392 -74.34554921969742) bank68942 +68943 POINT(40.138406223306255 -73.52554417644217) bank68943 +68944 POINT(41.19770305003526 -73.07247183360478) bank68944 +68945 POINT(41.010406169583625 -73.1051588061931) bank68945 +68946 POINT(40.2025318888969 -73.96123443643876) bank68946 +68947 POINT(41.51960945625207 -74.83830585427162) bank68947 +68948 POINT(41.22757708847427 -74.42548360747737) bank68948 +68949 POINT(41.00302974610727 -74.74941369117025) bank68949 +68950 POINT(41.4595176310439 -73.09033086124593) bank68950 +68951 POINT(40.86293322778703 -73.98142278817625) bank68951 +68952 POINT(40.985078571563896 -74.88830912242678) bank68952 +68953 POINT(40.61952912817023 -74.7600266807128) bank68953 +68954 POINT(40.213033648431505 -74.5321980531906) bank68954 +68955 POINT(41.65365581656639 -74.88207275268434) bank68955 +68956 POINT(41.251337220450786 -74.31625739685639) bank68956 +68957 POINT(40.29567447215093 -74.04824872144846) bank68957 +68958 POINT(40.36253066789824 -73.95304246526662) bank68958 +68959 POINT(39.893827628930524 -73.04015055925927) bank68959 +68960 POINT(40.3175779128055 -74.48244684586228) bank68960 +68961 POINT(40.0154560627725 -74.39049103903682) bank68961 +68962 POINT(39.73263845965668 -74.06593174333646) bank68962 +68963 POINT(40.39773044264652 -74.61343695615152) bank68963 +68964 POINT(40.377807078408104 -74.0869431701901) bank68964 +68965 POINT(39.91005993516885 -73.99342895038232) bank68965 +68966 POINT(39.92004914728475 -73.9173208456666) bank68966 +68967 POINT(40.8402890821746 -74.59446306518544) bank68967 +68968 POINT(40.507713469723534 -73.72751201268828) bank68968 +68969 POINT(41.3368413035243 -74.7165473513187) bank68969 +68970 POINT(40.24016077314049 -73.76227117499671) bank68970 +68971 POINT(41.374594937429706 -74.69737202262283) bank68971 +68972 POINT(40.00160687040982 -73.11301085374521) bank68972 +68973 POINT(41.06702126480965 -73.38713989002116) bank68973 +68974 POINT(41.706658077792255 -74.03381386212826) bank68974 +68975 POINT(39.88104273153556 -73.32512516029159) bank68975 +68976 POINT(40.02309059911984 -73.89744257380558) bank68976 +68977 POINT(40.873161417366795 -73.24918863209884) bank68977 +68978 POINT(41.01562484655165 -74.57804390663132) bank68978 +68979 POINT(40.5172660512435 -74.28973772558084) bank68979 +68980 POINT(40.683178805280036 -74.94932776772276) bank68980 +68981 POINT(40.289812889967436 -74.050769269079) bank68981 +68982 POINT(40.73963758551767 -74.48689996420437) bank68982 +68983 POINT(39.91995793237874 -74.26981355650926) bank68983 +68984 POINT(40.963708254817064 -74.09580810401756) bank68984 +68985 POINT(40.904244109652 -74.73924961317829) bank68985 +68986 POINT(40.25473664194777 -73.7112642062958) bank68986 +68987 POINT(40.095273485753715 -73.97627836467458) bank68987 +68988 POINT(40.39381931127843 -74.60667053449204) bank68988 +68989 POINT(40.253410634294525 -73.98443533949323) bank68989 +68990 POINT(41.15065556971714 -73.50964053900617) bank68990 +68991 POINT(40.750211386044334 -73.68038425146831) bank68991 +68992 POINT(40.86825912830196 -74.86351986564053) bank68992 +68993 POINT(40.55734001137548 -74.4010741689159) bank68993 +68994 POINT(40.55647159930765 -73.51756002125327) bank68994 +68995 POINT(39.90460091461262 -74.2291789226877) bank68995 +68996 POINT(40.73589694925635 -73.21986778762304) bank68996 +68997 POINT(40.97938858396363 -73.6771757268875) bank68997 +68998 POINT(41.46950012650486 -73.50203781292835) bank68998 +68999 POINT(40.015655602588716 -73.9863937937968) bank68999 +69000 POINT(41.19271681911595 -73.12492324619811) bank69000 +69001 POINT(41.331166759293424 -73.22761632917529) bank69001 +69002 POINT(40.19360213139597 -73.31312567524758) bank69002 +69003 POINT(39.843466376752524 -73.85724570410478) bank69003 +69004 POINT(41.166871560146106 -73.02724155625314) bank69004 +69005 POINT(41.204030196918666 -73.607726926207) bank69005 +69006 POINT(41.59197558540603 -73.70686871440166) bank69006 +69007 POINT(41.068417059427354 -74.0579378153303) bank69007 +69008 POINT(40.8881090180814 -74.23836745072552) bank69008 +69009 POINT(40.902600752369 -73.97335313977818) bank69009 +69010 POINT(41.04942411671156 -73.68698694226514) bank69010 +69011 POINT(40.276094637951324 -73.2850460613899) bank69011 +69012 POINT(40.93547347017364 -73.27076885572997) bank69012 +69013 POINT(40.167556441858885 -73.55591504477886) bank69013 +69014 POINT(39.88093317371812 -74.33748051209918) bank69014 +69015 POINT(39.732925084972685 -74.14312949571834) bank69015 +69016 POINT(40.902505533046124 -73.75062635517173) bank69016 +69017 POINT(40.38951419963827 -73.74150989800061) bank69017 +69018 POINT(40.62280756327971 -74.54511341528698) bank69018 +69019 POINT(39.91247543493049 -74.25038339750338) bank69019 +69020 POINT(39.7775227336567 -73.71013304285381) bank69020 +69021 POINT(39.914985017368046 -73.08456897382213) bank69021 +69022 POINT(39.8524519262026 -74.01066947571617) bank69022 +69023 POINT(39.951558103775696 -73.33761460294755) bank69023 +69024 POINT(40.481108597256906 -73.29876263904383) bank69024 +69025 POINT(39.76802470055335 -73.57832607554852) bank69025 +69026 POINT(40.95682532710319 -74.49253134655368) bank69026 +69027 POINT(41.008746926996054 -73.84106616082961) bank69027 +69028 POINT(41.41980306622949 -74.52697406575328) bank69028 +69029 POINT(40.5417002064383 -73.32651061761615) bank69029 +69030 POINT(41.331360221875016 -73.3300737823702) bank69030 +69031 POINT(39.84141832063997 -74.47273988414054) bank69031 +69032 POINT(40.49480267160667 -73.75997551508956) bank69032 +69033 POINT(40.83336538590452 -74.2548161933597) bank69033 +69034 POINT(41.00290920528513 -74.16050735274803) bank69034 +69035 POINT(40.666944837429 -73.69049470304161) bank69035 +69036 POINT(40.31194353163398 -74.14695408623969) bank69036 +69037 POINT(40.08653219170163 -73.33704151462535) bank69037 +69038 POINT(40.26103478563093 -74.77369299808701) bank69038 +69039 POINT(40.2751735428618 -73.75968315473096) bank69039 +69040 POINT(40.54392167791634 -73.71400056899576) bank69040 +69041 POINT(41.33636340668813 -73.54416527722395) bank69041 +69042 POINT(40.63432941818428 -74.34859504493653) bank69042 +69043 POINT(39.85149688562518 -74.901069401796) bank69043 +69044 POINT(40.63562612202912 -73.45319165319904) bank69044 +69045 POINT(39.85670285705187 -74.45113028519383) bank69045 +69046 POINT(41.22598919339715 -74.75850199230834) bank69046 +69047 POINT(40.98504649430235 -73.62988371101868) bank69047 +69048 POINT(41.26956333793732 -73.21583415740601) bank69048 +69049 POINT(40.217833913805485 -74.05230234547045) bank69049 +69050 POINT(41.34490465687316 -73.8175264437655) bank69050 +69051 POINT(41.07539129188277 -74.23705068092971) bank69051 +69052 POINT(40.57261650432746 -73.85124350933552) bank69052 +69053 POINT(40.60452804270253 -74.55951966806211) bank69053 +69054 POINT(41.4755621139659 -73.68607592753813) bank69054 +69055 POINT(41.367725509348986 -73.57689832803952) bank69055 +69056 POINT(39.76162047572193 -74.52739183785755) bank69056 +69057 POINT(40.83040382731257 -73.35473871696759) bank69057 +69058 POINT(40.657726956184796 -74.51043542527349) bank69058 +69059 POINT(41.67951799840271 -74.2594273262925) bank69059 +69060 POINT(40.014773062301764 -74.76002740824322) bank69060 +69061 POINT(40.30861874048183 -73.25597842574939) bank69061 +69062 POINT(41.404990565139045 -73.84082645810643) bank69062 +69063 POINT(41.10099662540676 -73.631837190348) bank69063 +69064 POINT(40.23002160683721 -74.92749467755921) bank69064 +69065 POINT(40.23495297729046 -73.76795556020696) bank69065 +69066 POINT(40.363958885028744 -74.48409130993694) bank69066 +69067 POINT(40.05427354083404 -73.69057392857937) bank69067 +69068 POINT(40.456664863407326 -74.05509232830981) bank69068 +69069 POINT(40.91224302338834 -73.52208637212063) bank69069 +69070 POINT(39.865339964317656 -74.84666528996166) bank69070 +69071 POINT(41.58227936867102 -74.78949354502717) bank69071 +69072 POINT(40.78986398204831 -74.08433463235505) bank69072 +69073 POINT(40.3553199783611 -74.59240470552561) bank69073 +69074 POINT(39.826320916557634 -73.30116950853518) bank69074 +69075 POINT(41.151751286209674 -74.9591946824291) bank69075 +69076 POINT(40.87904532342804 -73.4777065238752) bank69076 +69077 POINT(39.82762556117299 -74.57715948270953) bank69077 +69078 POINT(41.64143572603235 -73.25720307057959) bank69078 +69079 POINT(40.305025716911004 -73.49342038792184) bank69079 +69080 POINT(40.78903393050619 -73.83993455278345) bank69080 +69081 POINT(40.03273022859816 -74.62543405252853) bank69081 +69082 POINT(40.38072128783927 -73.30960234868053) bank69082 +69083 POINT(40.361108486813656 -73.76348783008413) bank69083 +69084 POINT(41.40320034066489 -73.67117132214304) bank69084 +69085 POINT(40.56421029127138 -74.35500284605097) bank69085 +69086 POINT(41.327674940988906 -73.89444859414277) bank69086 +69087 POINT(40.85258056711352 -74.02948255317861) bank69087 +69088 POINT(41.026392016270655 -74.2445752344035) bank69088 +69089 POINT(41.412478785215434 -74.51477325291609) bank69089 +69090 POINT(41.39502182412929 -73.1851056321165) bank69090 +69091 POINT(41.104087029194446 -73.67450325898855) bank69091 +69092 POINT(40.14540125271745 -74.19864718762092) bank69092 +69093 POINT(41.30164521855527 -74.14359894553195) bank69093 +69094 POINT(40.073090953210816 -73.7704278519954) bank69094 +69095 POINT(41.52905296689039 -73.9121080260852) bank69095 +69096 POINT(41.12648034497371 -74.50844135060909) bank69096 +69097 POINT(40.05560904247718 -74.4175078291464) bank69097 +69098 POINT(41.4983539011848 -74.16269684690845) bank69098 +69099 POINT(40.659380647772814 -73.53205371079031) bank69099 +69100 POINT(39.97736122514865 -74.47374883438387) bank69100 +69101 POINT(39.83906162363938 -74.76112745762035) bank69101 +69102 POINT(41.587944926409584 -73.74310569148345) bank69102 +69103 POINT(40.09980159413817 -74.99101196134323) bank69103 +69104 POINT(41.14129720963348 -73.29507315859443) bank69104 +69105 POINT(41.54439479062726 -73.89838385581875) bank69105 +69106 POINT(40.933711666602086 -74.74862167789173) bank69106 +69107 POINT(40.07637503464624 -74.90650456366238) bank69107 +69108 POINT(40.23812972511351 -74.0012528834868) bank69108 +69109 POINT(41.34472815387402 -73.96947117594848) bank69109 +69110 POINT(40.86056360096295 -73.12579318107382) bank69110 +69111 POINT(41.03324435360895 -73.23531897020366) bank69111 +69112 POINT(41.70243614025382 -73.15836050005524) bank69112 +69113 POINT(41.11178091829746 -74.46134378787487) bank69113 +69114 POINT(41.447467436431005 -74.30882735948079) bank69114 +69115 POINT(39.82123211086305 -74.33239718194183) bank69115 +69116 POINT(40.95685762853269 -74.76451805148484) bank69116 +69117 POINT(41.65887833160275 -73.69253225171636) bank69117 +69118 POINT(41.38626413763615 -73.90339320087283) bank69118 +69119 POINT(41.57842144709132 -74.74554168477623) bank69119 +69120 POINT(41.056484332024006 -73.5735125290812) bank69120 +69121 POINT(40.9047631790549 -74.00186907648407) bank69121 +69122 POINT(41.24351746797632 -74.84458945581748) bank69122 +69123 POINT(41.38344661456134 -74.64036620408893) bank69123 +69124 POINT(40.9997796695358 -74.06035151409172) bank69124 +69125 POINT(41.12884112750327 -74.79511949355278) bank69125 +69126 POINT(41.55407973309786 -73.4972565211875) bank69126 +69127 POINT(39.885043090594635 -73.38018971991406) bank69127 +69128 POINT(41.070526673596596 -74.66875111935745) bank69128 +69129 POINT(40.24934928233366 -73.20048798756372) bank69129 +69130 POINT(40.47583540826157 -74.86785620008273) bank69130 +69131 POINT(40.439127872364026 -74.58556352924818) bank69131 +69132 POINT(40.7912264064555 -74.08654990881652) bank69132 +69133 POINT(40.183540271578465 -74.11418412434804) bank69133 +69134 POINT(40.36657320149111 -74.64091997336874) bank69134 +69135 POINT(40.51037560254942 -74.50454702372808) bank69135 +69136 POINT(40.62123567168425 -74.70644321059615) bank69136 +69137 POINT(41.67927808225045 -74.85777411585363) bank69137 +69138 POINT(41.1140937012154 -73.3351841921627) bank69138 +69139 POINT(40.694328379159494 -73.22689879191682) bank69139 +69140 POINT(40.39021964872696 -73.98514629572898) bank69140 +69141 POINT(39.71353403895843 -74.1673535768498) bank69141 +69142 POINT(40.198644830235644 -73.76044918341294) bank69142 +69143 POINT(39.980668609079686 -74.06287659284875) bank69143 +69144 POINT(40.223895054614964 -73.71622612194541) bank69144 +69145 POINT(40.50292364241031 -73.7214374147608) bank69145 +69146 POINT(41.67853018239662 -73.3844576971614) bank69146 +69147 POINT(40.387868591491504 -73.91181202955578) bank69147 +69148 POINT(40.67187602983664 -74.39775411105897) bank69148 +69149 POINT(40.38047401657792 -74.03793957879445) bank69149 +69150 POINT(39.7889962004034 -74.70141476800364) bank69150 +69151 POINT(39.92558402997749 -74.95782233857958) bank69151 +69152 POINT(41.64050042751136 -74.8138288308373) bank69152 +69153 POINT(40.45206695301131 -74.19811328903607) bank69153 +69154 POINT(41.47277146326226 -73.08536184719425) bank69154 +69155 POINT(40.91187001313855 -73.65643181983107) bank69155 +69156 POINT(40.98501506336007 -73.18656788608152) bank69156 +69157 POINT(40.761191987515005 -73.27471614397015) bank69157 +69158 POINT(40.202603350056364 -73.14626560144656) bank69158 +69159 POINT(41.40712826017759 -74.31733545602259) bank69159 +69160 POINT(39.79298579779736 -73.5671484870434) bank69160 +69161 POINT(40.159461124061444 -73.02309754559316) bank69161 +69162 POINT(40.11384262480883 -74.56221685819665) bank69162 +69163 POINT(40.74533870087813 -74.43147007873952) bank69163 +69164 POINT(40.516733698376 -73.74894228400083) bank69164 +69165 POINT(40.110476718165806 -74.59949495491196) bank69165 +69166 POINT(39.90386622771121 -73.11075368731869) bank69166 +69167 POINT(39.750047487561844 -74.01307468048982) bank69167 +69168 POINT(41.13529671277826 -73.23499109669666) bank69168 +69169 POINT(40.01516534256037 -74.0215879909429) bank69169 +69170 POINT(40.35644239201307 -74.37825040806665) bank69170 +69171 POINT(41.358012914510475 -74.12363685055426) bank69171 +69172 POINT(41.43731695198656 -73.37542666441868) bank69172 +69173 POINT(39.72045639839254 -73.6478576287372) bank69173 +69174 POINT(41.54850742637524 -73.32898008847974) bank69174 +69175 POINT(41.64124631634634 -74.90355214930224) bank69175 +69176 POINT(41.36588567364332 -74.27294635936299) bank69176 +69177 POINT(41.5865921813788 -74.7618889778199) bank69177 +69178 POINT(40.49944825497958 -74.09547054949115) bank69178 +69179 POINT(40.77715431471776 -74.9429381488871) bank69179 +69180 POINT(41.40909431760329 -74.98631057671322) bank69180 +69181 POINT(41.05482671784637 -74.88950037287312) bank69181 +69182 POINT(39.89853456938521 -74.9838740320852) bank69182 +69183 POINT(41.151594251760315 -74.23610046017956) bank69183 +69184 POINT(41.26483480431676 -74.85162852525909) bank69184 +69185 POINT(39.74802237624644 -74.76970029414939) bank69185 +69186 POINT(40.53614754098247 -73.57323382575484) bank69186 +69187 POINT(39.838113085663075 -73.51469420511576) bank69187 +69188 POINT(41.129504420597 -74.77867954315957) bank69188 +69189 POINT(40.217090284303104 -74.86552479291669) bank69189 +69190 POINT(41.19207899933525 -74.36318784857983) bank69190 +69191 POINT(39.87981280605317 -74.83838189804484) bank69191 +69192 POINT(40.281545729750356 -73.79497783832808) bank69192 +69193 POINT(40.406639469961995 -74.82232657013765) bank69193 +69194 POINT(41.66986055010476 -73.28885714532845) bank69194 +69195 POINT(39.73039333428031 -74.16341003660627) bank69195 +69196 POINT(40.003917026176396 -73.4631542020752) bank69196 +69197 POINT(40.80428872057449 -73.65026441372311) bank69197 +69198 POINT(41.662894982773516 -73.35008093205457) bank69198 +69199 POINT(41.41356406766817 -73.77201013996367) bank69199 +69200 POINT(40.61782930397682 -74.40523359053383) bank69200 +69201 POINT(39.947946563019684 -73.7252681046552) bank69201 +69202 POINT(41.37763683782157 -73.15720926314803) bank69202 +69203 POINT(41.141276637411465 -74.68279793412603) bank69203 +69204 POINT(40.979480801690066 -74.6477801575444) bank69204 +69205 POINT(41.40525320991857 -74.38927370881143) bank69205 +69206 POINT(40.57593522269425 -73.38107227180888) bank69206 +69207 POINT(40.14014535376351 -74.40934956035335) bank69207 +69208 POINT(40.35735853434665 -74.51036340570836) bank69208 +69209 POINT(39.936600041975716 -73.93136293970612) bank69209 +69210 POINT(40.947475753231785 -73.2697970539555) bank69210 +69211 POINT(41.60373482353477 -74.94405689544752) bank69211 +69212 POINT(40.86730203401489 -73.6202642785516) bank69212 +69213 POINT(40.1336734860292 -74.34281436278049) bank69213 +69214 POINT(39.81942396650045 -73.99848391736944) bank69214 +69215 POINT(40.3509842156952 -73.15974445100288) bank69215 +69216 POINT(41.60264356733427 -74.74925885316914) bank69216 +69217 POINT(41.30264776830625 -74.4607338490373) bank69217 +69218 POINT(40.22817587067768 -74.82813509281517) bank69218 +69219 POINT(40.43118048713033 -73.69749452520477) bank69219 +69220 POINT(39.921455633670135 -74.95839918728652) bank69220 +69221 POINT(41.70959077647769 -74.31153005708332) bank69221 +69222 POINT(40.015417030344395 -74.4177244367117) bank69222 +69223 POINT(40.12371908415029 -73.88396980328031) bank69223 +69224 POINT(40.74194140109579 -74.85242232245226) bank69224 +69225 POINT(40.33776116907842 -74.18050213684072) bank69225 +69226 POINT(41.48533375653047 -73.27303801458112) bank69226 +69227 POINT(39.79713991075471 -73.50996382474816) bank69227 +69228 POINT(40.81042697760105 -74.34116281637505) bank69228 +69229 POINT(41.53583992562914 -74.24253244851776) bank69229 +69230 POINT(40.31374049107001 -73.58671250021469) bank69230 +69231 POINT(41.012469311098656 -74.55732020137216) bank69231 +69232 POINT(40.447801346332334 -73.0716408512303) bank69232 +69233 POINT(41.14999604894972 -74.83736617621229) bank69233 +69234 POINT(40.891424393924 -74.83264116635335) bank69234 +69235 POINT(40.48995776034426 -73.48245080537855) bank69235 +69236 POINT(39.782080300749506 -74.58706563719916) bank69236 +69237 POINT(40.11219512951215 -73.30654569448919) bank69237 +69238 POINT(40.274597108582206 -74.81918247962314) bank69238 +69239 POINT(40.405351762154226 -74.23124844944415) bank69239 +69240 POINT(40.99122945432757 -74.50834786576068) bank69240 +69241 POINT(41.53381281552495 -74.62494728548796) bank69241 +69242 POINT(39.73288816218388 -74.82712010100748) bank69242 +69243 POINT(41.36025127776244 -73.8478188790277) bank69243 +69244 POINT(41.357697252391176 -74.89149081282316) bank69244 +69245 POINT(40.93826425007908 -73.27437525708721) bank69245 +69246 POINT(41.221672542319986 -74.04014582813808) bank69246 +69247 POINT(41.60606564714987 -73.33665945729722) bank69247 +69248 POINT(40.54039928832375 -73.76665415429837) bank69248 +69249 POINT(41.03107324204229 -74.73506932324594) bank69249 +69250 POINT(41.68132285637016 -73.41329709041845) bank69250 +69251 POINT(41.39357646709744 -74.3223388974525) bank69251 +69252 POINT(41.29959720757852 -73.2707657893895) bank69252 +69253 POINT(39.8911732421634 -74.96805542806648) bank69253 +69254 POINT(41.08788042598772 -74.44436710545136) bank69254 +69255 POINT(40.533490635288196 -73.07757117635646) bank69255 +69256 POINT(41.19089205993174 -74.6242884496771) bank69256 +69257 POINT(39.85489514186863 -74.26567523951371) bank69257 +69258 POINT(40.16700851395783 -73.95268239351017) bank69258 +69259 POINT(40.41840250228534 -73.05776925689989) bank69259 +69260 POINT(41.68673694255526 -73.18368072316599) bank69260 +69261 POINT(41.06487849009313 -73.60172195000443) bank69261 +69262 POINT(40.51747860583252 -74.2042588312138) bank69262 +69263 POINT(40.41818176943683 -74.64674374973737) bank69263 +69264 POINT(41.02546171023996 -73.93949888082477) bank69264 +69265 POINT(41.59307273416296 -74.37546258854428) bank69265 +69266 POINT(41.09421111260358 -73.12574180347129) bank69266 +69267 POINT(40.946772942902875 -73.41814413026337) bank69267 +69268 POINT(39.88168708270735 -74.42400035700778) bank69268 +69269 POINT(40.685476652995824 -74.56234957200618) bank69269 +69270 POINT(40.82350092118017 -73.32935371858382) bank69270 +69271 POINT(41.126375435206725 -73.45803078538331) bank69271 +69272 POINT(41.01548506051851 -74.13272295007269) bank69272 +69273 POINT(41.097280145331226 -73.52534900202522) bank69273 +69274 POINT(41.56501224142998 -74.8172611850713) bank69274 +69275 POINT(39.746230602510906 -73.70170765243867) bank69275 +69276 POINT(41.19973020510311 -74.67513792989763) bank69276 +69277 POINT(40.53225409570125 -74.97362793129652) bank69277 +69278 POINT(39.731579336474844 -74.67495455456012) bank69278 +69279 POINT(41.48775457501942 -74.55456212771507) bank69279 +69280 POINT(40.29477164968051 -73.51426994294663) bank69280 +69281 POINT(41.69959788174963 -74.19963978767768) bank69281 +69282 POINT(41.39015444711795 -73.79010586531649) bank69282 +69283 POINT(40.62224670288781 -73.55549165646892) bank69283 +69284 POINT(41.33739224863248 -74.15450964333569) bank69284 +69285 POINT(39.961836636379665 -73.34879610996664) bank69285 +69286 POINT(39.95767016456483 -73.41692744090398) bank69286 +69287 POINT(41.30129272000993 -73.45172933121196) bank69287 +69288 POINT(40.67153050243846 -74.65148292844444) bank69288 +69289 POINT(40.61291967190935 -73.23893521715367) bank69289 +69290 POINT(41.31171852428055 -73.49076163035288) bank69290 +69291 POINT(40.68032266945838 -74.55695714331891) bank69291 +69292 POINT(41.34960377407421 -74.94714571961508) bank69292 +69293 POINT(40.868944419305926 -74.29699832259178) bank69293 +69294 POINT(40.776227715727074 -73.97483018646967) bank69294 +69295 POINT(40.04031128262355 -73.00694764756786) bank69295 +69296 POINT(40.57590701278151 -74.39838688625552) bank69296 +69297 POINT(40.89210777264944 -74.11230036610843) bank69297 +69298 POINT(39.987908432233425 -74.81529531479572) bank69298 +69299 POINT(40.087661358422714 -73.59513597558619) bank69299 +69300 POINT(41.37266422290352 -73.16346897328896) bank69300 +69301 POINT(41.62867822293567 -73.02597489433532) bank69301 +69302 POINT(40.01126653822639 -73.32405944037296) bank69302 +69303 POINT(41.36548005420608 -74.62776629058739) bank69303 +69304 POINT(40.798660516649285 -74.94592317689461) bank69304 +69305 POINT(41.16300954129813 -74.51100507117849) bank69305 +69306 POINT(40.205948715924194 -73.8636523299104) bank69306 +69307 POINT(41.23582883524618 -73.4365386068311) bank69307 +69308 POINT(39.774713461069624 -75.00115321734127) bank69308 +69309 POINT(39.98703951108796 -73.76635171114573) bank69309 +69310 POINT(40.957223273681265 -73.54283218232598) bank69310 +69311 POINT(40.484194726108484 -73.29383163034181) bank69311 +69312 POINT(41.60173744426615 -73.30324021779047) bank69312 +69313 POINT(40.888413332763456 -74.22177000715942) bank69313 +69314 POINT(40.018089140023555 -73.90672541608042) bank69314 +69315 POINT(41.20594352121906 -74.12791779124845) bank69315 +69316 POINT(39.788771970892654 -73.16831701542544) bank69316 +69317 POINT(39.826335102332614 -73.14516202679027) bank69317 +69318 POINT(41.30489102060422 -74.52857457807764) bank69318 +69319 POINT(39.72450808760017 -74.97061400096045) bank69319 +69320 POINT(39.87753286253298 -73.56455709254273) bank69320 +69321 POINT(41.58552049183099 -74.16792960446412) bank69321 +69322 POINT(40.25672606197784 -74.91800464085587) bank69322 +69323 POINT(40.21766220531467 -74.3596974755111) bank69323 +69324 POINT(41.0532521857484 -73.91302084625909) bank69324 +69325 POINT(40.4652678666111 -74.95298283742666) bank69325 +69326 POINT(40.201377516948405 -73.2607291158213) bank69326 +69327 POINT(40.78626516855461 -73.85764662848361) bank69327 +69328 POINT(41.34008485422771 -73.51169098030773) bank69328 +69329 POINT(40.85607960756516 -73.74265120196745) bank69329 +69330 POINT(40.59407881653732 -73.79463465691315) bank69330 +69331 POINT(40.172062716165165 -74.55925491522518) bank69331 +69332 POINT(39.94438117076794 -73.10756061785071) bank69332 +69333 POINT(41.07106308175689 -74.85246650632259) bank69333 +69334 POINT(41.25318527216478 -73.71876273595835) bank69334 +69335 POINT(40.90009865676639 -74.42146466546728) bank69335 +69336 POINT(40.16248103616339 -73.21251329451286) bank69336 +69337 POINT(40.72840930216679 -74.10769163494109) bank69337 +69338 POINT(40.70452734685393 -73.60117186165125) bank69338 +69339 POINT(40.22421722602694 -73.668239575584) bank69339 +69340 POINT(40.30089519330934 -73.85639934958131) bank69340 +69341 POINT(41.16296328089314 -73.94730283833427) bank69341 +69342 POINT(41.488965607508334 -73.50826699612989) bank69342 +69343 POINT(40.230122596715994 -73.14207770378741) bank69343 +69344 POINT(41.524340953250935 -73.23188450336492) bank69344 +69345 POINT(39.84464870299481 -73.6937479468541) bank69345 +69346 POINT(41.45567006465932 -74.37885743385326) bank69346 +69347 POINT(40.56682158853515 -73.82541216226818) bank69347 +69348 POINT(40.559243808719245 -74.30306180527553) bank69348 +69349 POINT(41.42053999399474 -73.95813326319421) bank69349 +69350 POINT(41.126544362582536 -74.99165327928345) bank69350 +69351 POINT(40.44445158595462 -73.74662001681953) bank69351 +69352 POINT(40.04822455501645 -74.66869038217774) bank69352 +69353 POINT(41.52418563551434 -73.08458444264) bank69353 +69354 POINT(40.050641746528406 -74.9766678147161) bank69354 +69355 POINT(41.36748532022 -73.64776646778844) bank69355 +69356 POINT(39.88403551250306 -73.97131819839596) bank69356 +69357 POINT(41.42877990004607 -73.30703396791078) bank69357 +69358 POINT(41.52391368256287 -74.30840434911377) bank69358 +69359 POINT(39.99626010635664 -73.1310175753816) bank69359 +69360 POINT(41.05440395063375 -73.25502542176879) bank69360 +69361 POINT(41.068348273583034 -73.62153688344434) bank69361 +69362 POINT(39.90171898569499 -74.3541216069014) bank69362 +69363 POINT(40.833949910074715 -73.45919066934458) bank69363 +69364 POINT(40.434283337151825 -73.17481029780411) bank69364 +69365 POINT(40.72906640325288 -74.23002640969126) bank69365 +69366 POINT(40.5702254783907 -73.08542966643402) bank69366 +69367 POINT(40.96422063293418 -74.81570269422967) bank69367 +69368 POINT(40.85200516486062 -73.27905608678635) bank69368 +69369 POINT(40.13753794720879 -74.26250082875382) bank69369 +69370 POINT(40.355508543777596 -73.16065311524052) bank69370 +69371 POINT(39.86273301041257 -73.28543624437776) bank69371 +69372 POINT(40.79227591004347 -73.0315829129471) bank69372 +69373 POINT(39.980035463091674 -74.31823316699555) bank69373 +69374 POINT(40.90047033604891 -74.23176764245139) bank69374 +69375 POINT(39.901212482185144 -73.65345950130829) bank69375 +69376 POINT(39.855938893511116 -73.24659376751679) bank69376 +69377 POINT(39.893685998910016 -74.85120057131988) bank69377 +69378 POINT(40.53433657682055 -73.53233400668017) bank69378 +69379 POINT(40.4009212014339 -73.74342096569953) bank69379 +69380 POINT(39.72049750761691 -73.50480726849912) bank69380 +69381 POINT(41.26708938302113 -73.64446716078363) bank69381 +69382 POINT(40.61268671688229 -73.66459706739403) bank69382 +69383 POINT(40.40714621533874 -74.39245102115251) bank69383 +69384 POINT(41.09018201563523 -74.88775512792753) bank69384 +69385 POINT(41.52289394428135 -74.55772990303082) bank69385 +69386 POINT(41.59042733735043 -74.61823964745862) bank69386 +69387 POINT(40.31671975322413 -74.60927909733749) bank69387 +69388 POINT(40.68059712455346 -73.3741491962254) bank69388 +69389 POINT(40.691022304558416 -74.0174457760879) bank69389 +69390 POINT(40.1086012049651 -74.4923884468837) bank69390 +69391 POINT(39.7768204662895 -74.64077119609502) bank69391 +69392 POINT(41.173316498374156 -74.77288382396905) bank69392 +69393 POINT(40.08469301109733 -73.56497000659527) bank69393 +69394 POINT(41.58744801214572 -74.17068410579081) bank69394 +69395 POINT(41.349259369076094 -74.60951631132185) bank69395 +69396 POINT(40.431175491442794 -73.12143743002245) bank69396 +69397 POINT(40.990652707146076 -74.25994352289675) bank69397 +69398 POINT(41.34867528512167 -73.20543735864503) bank69398 +69399 POINT(39.83059081982894 -73.79978028372837) bank69399 +69400 POINT(40.47575779618285 -73.69638420845381) bank69400 +69401 POINT(40.594944338304465 -74.35043962118395) bank69401 +69402 POINT(40.97466772076728 -73.5258809016323) bank69402 +69403 POINT(40.638185502234926 -74.39699260170266) bank69403 +69404 POINT(40.824100052105976 -73.89772389842828) bank69404 +69405 POINT(41.39690693612367 -74.53587936809569) bank69405 +69406 POINT(41.49239064613142 -73.60168266198788) bank69406 +69407 POINT(41.07824835148757 -74.379326825474) bank69407 +69408 POINT(40.69206099280819 -73.03936734337283) bank69408 +69409 POINT(40.75628951256955 -74.89454908336153) bank69409 +69410 POINT(41.15873627764415 -74.54051364784463) bank69410 +69411 POINT(39.924895568434145 -74.09968827655321) bank69411 +69412 POINT(40.03901673019899 -74.71976173107998) bank69412 +69413 POINT(40.64441517891356 -73.99698204894773) bank69413 +69414 POINT(41.64804875651009 -74.22356612665494) bank69414 +69415 POINT(41.03585747955947 -74.10144904949291) bank69415 +69416 POINT(41.60184354129789 -73.0764444204595) bank69416 +69417 POINT(41.47944812788282 -73.38607027068016) bank69417 +69418 POINT(40.98573556703631 -74.92120113931617) bank69418 +69419 POINT(40.67269224682763 -74.12469964885643) bank69419 +69420 POINT(39.86402791446671 -74.93557471845037) bank69420 +69421 POINT(41.21945787202312 -74.39001070155985) bank69421 +69422 POINT(41.423577590209305 -73.82034231737026) bank69422 +69423 POINT(40.7006252915984 -74.42597651682794) bank69423 +69424 POINT(40.58665281133219 -74.81673951494248) bank69424 +69425 POINT(40.68187663129949 -73.0092041033955) bank69425 +69426 POINT(40.722959275305406 -73.95724213305012) bank69426 +69427 POINT(40.74150155884013 -74.9818456732497) bank69427 +69428 POINT(40.296298651446946 -73.80549107738146) bank69428 +69429 POINT(40.88828898967765 -73.88513504378798) bank69429 +69430 POINT(41.242521463570604 -73.1023506880124) bank69430 +69431 POINT(41.10074066764636 -73.40757881083513) bank69431 +69432 POINT(41.3432336942576 -73.39909467737284) bank69432 +69433 POINT(40.2433717144652 -73.1513047401082) bank69433 +69434 POINT(41.47617375478564 -73.45802020566465) bank69434 +69435 POINT(40.60661221113903 -74.207952554722) bank69435 +69436 POINT(40.3421684914356 -73.78076976257236) bank69436 +69437 POINT(41.461296407091695 -73.86758971194105) bank69437 +69438 POINT(40.60049334152722 -73.25660305089556) bank69438 +69439 POINT(41.08895286384506 -74.21946042880401) bank69439 +69440 POINT(39.95979501842777 -73.38473880007002) bank69440 +69441 POINT(39.911555720854054 -73.09817818611184) bank69441 +69442 POINT(39.994538325311936 -74.81958101892307) bank69442 +69443 POINT(41.46883490737355 -74.09829714620959) bank69443 +69444 POINT(41.08269540815788 -73.17716805153981) bank69444 +69445 POINT(40.05615039644693 -74.34235495169271) bank69445 +69446 POINT(41.026732773648135 -74.09303510271116) bank69446 +69447 POINT(39.88678417724253 -73.56123983890723) bank69447 +69448 POINT(41.553849380331386 -74.76221984498648) bank69448 +69449 POINT(41.12829453259697 -74.50104519638023) bank69449 +69450 POINT(40.81471960593186 -74.45777013039512) bank69450 +69451 POINT(41.111247151786216 -74.18809793959133) bank69451 +69452 POINT(41.281613980038244 -74.74336373815967) bank69452 +69453 POINT(41.18577741482674 -73.1974037891609) bank69453 +69454 POINT(39.98667941026141 -74.65134122028174) bank69454 +69455 POINT(40.017208978259156 -73.89136484619716) bank69455 +69456 POINT(39.845203913819056 -73.52274220641574) bank69456 +69457 POINT(41.0607212668616 -74.54476012206169) bank69457 +69458 POINT(40.210899122969714 -73.81562386368356) bank69458 +69459 POINT(40.609280521170284 -74.95842365636014) bank69459 +69460 POINT(40.06510151295268 -73.50921348975227) bank69460 +69461 POINT(39.776257942451196 -73.28659108577746) bank69461 +69462 POINT(40.91508343709776 -73.2943678568108) bank69462 +69463 POINT(40.55749234915635 -74.70514967085796) bank69463 +69464 POINT(39.73694100637069 -73.41473120187777) bank69464 +69465 POINT(41.119161819320645 -73.23702146178591) bank69465 +69466 POINT(41.33168334485771 -74.64237301514694) bank69466 +69467 POINT(41.309350490925375 -73.58944397342873) bank69467 +69468 POINT(39.781592331876006 -74.44005230305491) bank69468 +69469 POINT(40.046427422549556 -73.22030048630266) bank69469 +69470 POINT(39.72695723733163 -73.72107911755447) bank69470 +69471 POINT(41.02452670029308 -73.96915934683143) bank69471 +69472 POINT(41.43597198133557 -73.72394011791853) bank69472 +69473 POINT(40.7606470762842 -73.846921649268) bank69473 +69474 POINT(41.04868006700793 -73.97591420000029) bank69474 +69475 POINT(40.53237704593885 -73.07118980001226) bank69475 +69476 POINT(41.486250674559955 -74.58097725602646) bank69476 +69477 POINT(40.8662289594709 -73.94932517613685) bank69477 +69478 POINT(40.52610875593581 -73.3589755244153) bank69478 +69479 POINT(41.60755854502313 -74.09252396331179) bank69479 +69480 POINT(40.68520526154249 -73.3277035770805) bank69480 +69481 POINT(40.17320922637565 -73.82766241910788) bank69481 +69482 POINT(41.3882145753515 -74.69773863649401) bank69482 +69483 POINT(41.0937905799219 -74.54649148720111) bank69483 +69484 POINT(41.64855899191632 -74.8877229961684) bank69484 +69485 POINT(40.62834743910146 -73.55489438078666) bank69485 +69486 POINT(40.51486822317633 -74.45423342966384) bank69486 +69487 POINT(40.22729506826425 -73.41498833019087) bank69487 +69488 POINT(40.36493543696823 -74.75850245488876) bank69488 +69489 POINT(41.28750216689269 -73.29896934866711) bank69489 +69490 POINT(41.545594016976814 -74.60718234220508) bank69490 +69491 POINT(40.661183653335094 -74.96119010995538) bank69491 +69492 POINT(41.38631759159608 -73.43463923432711) bank69492 +69493 POINT(41.421217316713275 -74.76938248914792) bank69493 +69494 POINT(40.310185727225615 -73.51501910469109) bank69494 +69495 POINT(40.75022715816007 -73.10915539050015) bank69495 +69496 POINT(40.768937206281805 -73.08444643582955) bank69496 +69497 POINT(40.82738436091061 -74.21815153365323) bank69497 +69498 POINT(41.46297949135493 -73.20629787244769) bank69498 +69499 POINT(40.80245404788247 -74.68196120881356) bank69499 +69500 POINT(41.041631491036036 -74.74845084749533) bank69500 +69501 POINT(41.434423387316556 -74.19160721502669) bank69501 +69502 POINT(41.7026392710748 -73.52471561128449) bank69502 +69503 POINT(39.99196182802901 -73.45975565323276) bank69503 +69504 POINT(40.714275788105375 -74.7020322040781) bank69504 +69505 POINT(40.275815108057294 -74.42471083110215) bank69505 +69506 POINT(40.24633406861652 -74.57708567559642) bank69506 +69507 POINT(40.365667248260806 -74.0747250240518) bank69507 +69508 POINT(40.93203693290741 -74.10698114986413) bank69508 +69509 POINT(41.61531572731347 -74.05760946152496) bank69509 +69510 POINT(41.626826586763514 -73.52339169159231) bank69510 +69511 POINT(39.90803839059194 -74.00871106319988) bank69511 +69512 POINT(40.551065227786054 -74.17810378198092) bank69512 +69513 POINT(39.960697353943395 -74.00840717573557) bank69513 +69514 POINT(41.09515142962473 -74.08351861876513) bank69514 +69515 POINT(40.28552690938631 -74.28104134474967) bank69515 +69516 POINT(41.21184815034363 -74.56550590250896) bank69516 +69517 POINT(40.484287108248886 -73.64839464381373) bank69517 +69518 POINT(40.12307715638616 -74.24014952887431) bank69518 +69519 POINT(40.242009770384286 -73.82036615161455) bank69519 +69520 POINT(41.20135432358587 -73.2956853920712) bank69520 +69521 POINT(40.65370101124149 -73.01776691757061) bank69521 +69522 POINT(40.681406972177285 -74.45019337738889) bank69522 +69523 POINT(40.00346967899758 -74.59775321826234) bank69523 +69524 POINT(41.04323727908589 -74.17220128075253) bank69524 +69525 POINT(40.117136555346974 -73.68154834995904) bank69525 +69526 POINT(39.79282141658676 -74.71796353840377) bank69526 +69527 POINT(41.25833536922379 -74.47052811579948) bank69527 +69528 POINT(41.043926495931686 -74.00931068746755) bank69528 +69529 POINT(39.80752039636949 -73.12975420172697) bank69529 +69530 POINT(41.216405634444484 -74.68407391605899) bank69530 +69531 POINT(40.99587486983443 -74.26870620509396) bank69531 +69532 POINT(41.03304959199552 -73.8965750468375) bank69532 +69533 POINT(41.68024806846512 -74.1906806988639) bank69533 +69534 POINT(41.16894601303739 -74.8808421215666) bank69534 +69535 POINT(39.99135750715697 -73.93214296446953) bank69535 +69536 POINT(40.01787074216307 -73.22242906361288) bank69536 +69537 POINT(40.781623918231105 -73.84504856630393) bank69537 +69538 POINT(39.90082111198422 -73.26562991636784) bank69538 +69539 POINT(40.148153098832516 -73.23823484018925) bank69539 +69540 POINT(40.14612951467196 -74.31121299219484) bank69540 +69541 POINT(40.11730117859105 -74.7953240803565) bank69541 +69542 POINT(40.07882708588733 -73.45224322969862) bank69542 +69543 POINT(40.49972428708734 -74.5090698486906) bank69543 +69544 POINT(39.89243318574304 -73.76629097002697) bank69544 +69545 POINT(41.57366094675187 -74.32858728709058) bank69545 +69546 POINT(40.86376951353476 -73.38347005135498) bank69546 +69547 POINT(39.808517673370766 -74.68825089939467) bank69547 +69548 POINT(40.99360894596898 -74.54971876455035) bank69548 +69549 POINT(41.528815378341996 -74.07399789929198) bank69549 +69550 POINT(40.42855313939386 -74.70160372388646) bank69550 +69551 POINT(40.8241044944431 -74.68194627444089) bank69551 +69552 POINT(40.110469945693296 -74.92066341743028) bank69552 +69553 POINT(40.28342766775338 -74.20378164785355) bank69553 +69554 POINT(39.84027907738614 -73.06215201708291) bank69554 +69555 POINT(40.88834790062994 -73.45080545023504) bank69555 +69556 POINT(41.099264877446 -73.7843950884785) bank69556 +69557 POINT(40.18805624191195 -74.62402078931345) bank69557 +69558 POINT(41.39495784640776 -74.09697708965577) bank69558 +69559 POINT(40.00346170289369 -74.99870208257563) bank69559 +69560 POINT(40.37445916873877 -74.0135496035855) bank69560 +69561 POINT(40.897418853914985 -74.01231506753005) bank69561 +69562 POINT(41.092718717850246 -74.26425826976912) bank69562 +69563 POINT(41.29381636588422 -74.43274183993343) bank69563 +69564 POINT(39.86074043580614 -73.3740130771504) bank69564 +69565 POINT(40.1899600004752 -73.73799346494687) bank69565 +69566 POINT(39.99893004704932 -73.48216512448687) bank69566 +69567 POINT(40.192311151146384 -74.35988443660487) bank69567 +69568 POINT(39.90180565049763 -73.41087109180874) bank69568 +69569 POINT(39.815705959824214 -74.39903595969682) bank69569 +69570 POINT(41.640751986368656 -74.89646427648144) bank69570 +69571 POINT(41.27055906145831 -74.50799747354318) bank69571 +69572 POINT(41.50931748267555 -73.04125283520524) bank69572 +69573 POINT(40.051406469556184 -74.61368850643184) bank69573 +69574 POINT(40.858283003230014 -74.21957582840675) bank69574 +69575 POINT(40.50475844293146 -73.29902217683265) bank69575 +69576 POINT(40.576887885791805 -74.23288189894663) bank69576 +69577 POINT(40.82771833907492 -73.63907197553583) bank69577 +69578 POINT(40.63375328993523 -74.73903544878038) bank69578 +69579 POINT(41.551613863082586 -73.39101433356898) bank69579 +69580 POINT(39.986914188561606 -73.15928318287484) bank69580 +69581 POINT(41.54162986804184 -74.49738671150992) bank69581 +69582 POINT(41.53196113279136 -73.43719345950093) bank69582 +69583 POINT(40.343590724955064 -74.0681949005202) bank69583 +69584 POINT(41.15802539499317 -74.65146381115319) bank69584 +69585 POINT(40.81199089139173 -73.13479531107279) bank69585 +69586 POINT(41.353231367769716 -73.0495710010408) bank69586 +69587 POINT(40.55410894036821 -73.59630405944382) bank69587 +69588 POINT(40.750137322101274 -74.36711260110712) bank69588 +69589 POINT(40.96707777559365 -73.56602404704893) bank69589 +69590 POINT(39.82735115759133 -74.6193768942129) bank69590 +69591 POINT(41.10232589148106 -73.48608808563279) bank69591 +69592 POINT(40.56517450014674 -73.58925319716748) bank69592 +69593 POINT(40.50282238475024 -73.54147645805739) bank69593 +69594 POINT(39.769242533418904 -73.10345936054745) bank69594 +69595 POINT(41.12831372953576 -73.64344772295834) bank69595 +69596 POINT(41.62916934838947 -74.74795399209549) bank69596 +69597 POINT(40.71594669857104 -73.12789115888896) bank69597 +69598 POINT(41.210890847973516 -74.45636875289219) bank69598 +69599 POINT(41.254296872105726 -73.10482434554025) bank69599 +69600 POINT(40.27635748804113 -74.80519013887567) bank69600 +69601 POINT(40.4354395358063 -73.89950580471324) bank69601 +69602 POINT(40.27490762355069 -73.33873358424306) bank69602 +69603 POINT(40.47150789252984 -73.66835216920045) bank69603 +69604 POINT(39.74933901187213 -73.79559916709725) bank69604 +69605 POINT(40.51895780941275 -73.65506192867961) bank69605 +69606 POINT(41.09539402941327 -74.26870615132877) bank69606 +69607 POINT(40.406788253614046 -73.5495555261881) bank69607 +69608 POINT(40.71234585942366 -73.8009287036983) bank69608 +69609 POINT(40.941443572174585 -73.48688965581037) bank69609 +69610 POINT(41.11218044952859 -74.39291623142188) bank69610 +69611 POINT(40.494512498435824 -73.83593619316927) bank69611 +69612 POINT(41.270581907970666 -74.87814672710367) bank69612 +69613 POINT(40.81156636921615 -74.13038942774324) bank69613 +69614 POINT(40.07514133238865 -74.25841502218995) bank69614 +69615 POINT(40.99387538398005 -73.42131962907645) bank69615 +69616 POINT(41.62038556585919 -73.32600863998681) bank69616 +69617 POINT(41.63625254383161 -73.49324955883297) bank69617 +69618 POINT(40.47167832881675 -74.52691196005337) bank69618 +69619 POINT(40.46824631833886 -74.61124801876873) bank69619 +69620 POINT(40.107440319979794 -73.58110808226492) bank69620 +69621 POINT(41.2943234597488 -73.01725539224117) bank69621 +69622 POINT(40.56133453073516 -74.8332684090867) bank69622 +69623 POINT(41.140074294403966 -74.30981001705834) bank69623 +69624 POINT(41.37191350154642 -74.02113592158172) bank69624 +69625 POINT(41.64073711870601 -73.96059554373348) bank69625 +69626 POINT(40.0910614132613 -73.90145659619499) bank69626 +69627 POINT(41.29304712756718 -73.39351469051347) bank69627 +69628 POINT(40.02164018151845 -73.89809083430208) bank69628 +69629 POINT(39.88107441363798 -73.06042111494665) bank69629 +69630 POINT(40.61138349556148 -73.42506428663327) bank69630 +69631 POINT(41.355947472825335 -73.45895943399536) bank69631 +69632 POINT(39.949386395093676 -74.11498727333596) bank69632 +69633 POINT(40.265077216285434 -74.4945979115909) bank69633 +69634 POINT(41.13052627568014 -73.59292080014399) bank69634 +69635 POINT(41.23956171528448 -74.70859237129605) bank69635 +69636 POINT(40.43913466437728 -74.21066256710333) bank69636 +69637 POINT(40.306243507403735 -74.36519914842962) bank69637 +69638 POINT(40.93525829862329 -74.87884017471633) bank69638 +69639 POINT(39.998454444593584 -74.12900708480976) bank69639 +69640 POINT(40.486309478171684 -73.68930761338078) bank69640 +69641 POINT(41.683950079629845 -74.1587834908087) bank69641 +69642 POINT(40.24621270967174 -73.76850599150748) bank69642 +69643 POINT(41.24468945690889 -74.26071432749723) bank69643 +69644 POINT(40.16822155364977 -74.47744035127965) bank69644 +69645 POINT(39.81823359020267 -74.68586963338909) bank69645 +69646 POINT(40.98181883227881 -74.82431606191601) bank69646 +69647 POINT(41.082688698251935 -74.31211261237387) bank69647 +69648 POINT(40.03942284962635 -73.73053331039978) bank69648 +69649 POINT(40.425827942350985 -73.15987562768012) bank69649 +69650 POINT(40.80684219495694 -73.70956755554198) bank69650 +69651 POINT(40.94279424618436 -74.97134141736821) bank69651 +69652 POINT(40.937512383620664 -73.87897171633048) bank69652 +69653 POINT(40.74428635357827 -74.19423237834262) bank69653 +69654 POINT(41.51490414256011 -74.27375272584868) bank69654 +69655 POINT(40.17803899571834 -74.1344940061458) bank69655 +69656 POINT(40.95052719109779 -73.9381728218794) bank69656 +69657 POINT(39.96855597708516 -73.95444483471361) bank69657 +69658 POINT(40.35393466423 -73.96112334132505) bank69658 +69659 POINT(41.32457697978389 -73.19420471303886) bank69659 +69660 POINT(40.96938110805845 -74.4264333774763) bank69660 +69661 POINT(40.92125565474872 -73.51686695332721) bank69661 +69662 POINT(40.1258459147274 -73.39490648405844) bank69662 +69663 POINT(40.39201494640545 -74.58928244395354) bank69663 +69664 POINT(39.825045276086406 -74.16281119684857) bank69664 +69665 POINT(40.032285699665685 -73.16906507403257) bank69665 +69666 POINT(40.72055753824135 -73.13970324483181) bank69666 +69667 POINT(40.97601770260574 -73.5137362026) bank69667 +69668 POINT(40.41171106108779 -74.11565320052333) bank69668 +69669 POINT(39.79220860412166 -73.58304111790116) bank69669 +69670 POINT(41.605787079855105 -73.59389596895029) bank69670 +69671 POINT(41.42586736605413 -73.11995838671871) bank69671 +69672 POINT(40.07797803827606 -73.14393255615704) bank69672 +69673 POINT(41.13428101056999 -74.81042366106857) bank69673 +69674 POINT(41.66404933602464 -74.0590067085595) bank69674 +69675 POINT(40.67230016298083 -74.2894508113555) bank69675 +69676 POINT(40.135175504801225 -74.21466316145097) bank69676 +69677 POINT(39.81205410118698 -74.76158556872986) bank69677 +69678 POINT(41.507696510861315 -73.39618843107449) bank69678 +69679 POINT(41.1577129276465 -74.06444437846754) bank69679 +69680 POINT(39.933494693750646 -73.1036486050064) bank69680 +69681 POINT(40.51140049224262 -73.66153389332503) bank69681 +69682 POINT(40.16164592108876 -73.9111070762779) bank69682 +69683 POINT(41.2218571803374 -74.94417779850669) bank69683 +69684 POINT(40.12132049297661 -73.3306727529378) bank69684 +69685 POINT(39.98852374816335 -73.49750179795832) bank69685 +69686 POINT(41.11971398066735 -73.75372241889544) bank69686 +69687 POINT(41.315337790502824 -73.34043486737983) bank69687 +69688 POINT(40.59319456200348 -74.04480334492759) bank69688 +69689 POINT(40.55424890757955 -74.67471038499642) bank69689 +69690 POINT(40.286944958846526 -74.11593100884423) bank69690 +69691 POINT(40.76442515301107 -73.71362973226283) bank69691 +69692 POINT(40.24142440047774 -74.10518968646042) bank69692 +69693 POINT(41.38139819061026 -73.99129712989851) bank69693 +69694 POINT(40.02803990965922 -74.9608137445695) bank69694 +69695 POINT(39.78134533374549 -73.63253881051894) bank69695 +69696 POINT(41.17399909148495 -74.66469113767751) bank69696 +69697 POINT(40.34796550949965 -73.57961076971267) bank69697 +69698 POINT(41.30937597409776 -74.45294762472363) bank69698 +69699 POINT(41.51687822348207 -74.98998885030304) bank69699 +69700 POINT(40.75372426753546 -73.44045963248062) bank69700 +69701 POINT(40.82481817477841 -73.84128091215668) bank69701 +69702 POINT(41.04766062008697 -74.28143931739173) bank69702 +69703 POINT(41.23770296109491 -73.35547404606133) bank69703 +69704 POINT(41.39897575876421 -74.06792772943356) bank69704 +69705 POINT(41.09719300025766 -73.71149098669122) bank69705 +69706 POINT(41.05830599148567 -74.6425412360455) bank69706 +69707 POINT(39.94436998906155 -74.3992285714969) bank69707 +69708 POINT(41.422973454988785 -73.45432852698367) bank69708 +69709 POINT(41.59553348623897 -73.70991824562489) bank69709 +69710 POINT(41.677786855930165 -73.28650304208524) bank69710 +69711 POINT(40.89291756025406 -73.97067494641227) bank69711 +69712 POINT(40.00409141626641 -74.86762078844698) bank69712 +69713 POINT(40.232875085912745 -73.77575584654718) bank69713 +69714 POINT(40.54203438009916 -74.4334597774478) bank69714 +69715 POINT(40.463596499738784 -73.15007190501429) bank69715 +69716 POINT(39.80356533792161 -73.70164743331155) bank69716 +69717 POINT(41.51803471342401 -73.22869358705906) bank69717 +69718 POINT(41.379009172815074 -74.49421494331791) bank69718 +69719 POINT(39.811581310828934 -73.95589675093767) bank69719 +69720 POINT(40.80398585727364 -74.28730818547545) bank69720 +69721 POINT(40.897243894067074 -74.97411370385915) bank69721 +69722 POINT(40.44038867879893 -73.33445184963901) bank69722 +69723 POINT(40.64834767919679 -73.29126530478926) bank69723 +69724 POINT(40.097467666457526 -73.89657684225376) bank69724 +69725 POINT(40.721003909341704 -74.04501338564893) bank69725 +69726 POINT(41.01859116472554 -73.1247427793249) bank69726 +69727 POINT(39.868127494054654 -74.65083242615272) bank69727 +69728 POINT(41.18081271466531 -74.63095872822008) bank69728 +69729 POINT(40.096752951241385 -74.93265679284895) bank69729 +69730 POINT(40.888812507857146 -73.15250519866034) bank69730 +69731 POINT(40.332944278976264 -73.71486093309343) bank69731 +69732 POINT(40.30512814946064 -74.13554989289942) bank69732 +69733 POINT(40.462550609267986 -73.16715541609616) bank69733 +69734 POINT(40.99993055886308 -73.69097185170105) bank69734 +69735 POINT(41.67781119759037 -73.45049164297336) bank69735 +69736 POINT(41.233226030223626 -73.92540998035874) bank69736 +69737 POINT(41.307684382412816 -73.09981460665348) bank69737 +69738 POINT(40.912601869822 -73.41270021581968) bank69738 +69739 POINT(39.755086269228066 -73.61301901315066) bank69739 +69740 POINT(39.81156576159318 -74.06055208955561) bank69740 +69741 POINT(41.433764442109165 -73.88600219518273) bank69741 +69742 POINT(40.254803957668585 -74.16116459988065) bank69742 +69743 POINT(40.49480684411906 -74.25720249507675) bank69743 +69744 POINT(41.096987637112306 -73.78296677344925) bank69744 +69745 POINT(39.756653060241355 -73.80033405378536) bank69745 +69746 POINT(40.68420123017509 -73.72278409916393) bank69746 +69747 POINT(40.59183100304684 -74.60279540151569) bank69747 +69748 POINT(41.660082167231096 -74.9055927717842) bank69748 +69749 POINT(40.11858321488747 -73.42855797088461) bank69749 +69750 POINT(39.935953796273786 -74.96031404286995) bank69750 +69751 POINT(40.29619184308057 -73.54347108048759) bank69751 +69752 POINT(39.9912074486905 -73.75947816689641) bank69752 +69753 POINT(41.178003385350166 -74.5537128015925) bank69753 +69754 POINT(41.68568113101033 -74.61122487895061) bank69754 +69755 POINT(40.697446378419045 -73.11640982119802) bank69755 +69756 POINT(41.13314680147868 -74.66955467599993) bank69756 +69757 POINT(41.46651497615196 -74.82627676476318) bank69757 +69758 POINT(41.021393356608904 -74.60459871262681) bank69758 +69759 POINT(41.49012227048659 -73.8549303086288) bank69759 +69760 POINT(41.0989418187617 -74.66989347864369) bank69760 +69761 POINT(41.2452058854581 -74.57382550292294) bank69761 +69762 POINT(39.718141339422175 -74.6789917302304) bank69762 +69763 POINT(41.441891203658315 -73.89617790952353) bank69763 +69764 POINT(40.06923957786754 -74.45452544392981) bank69764 +69765 POINT(39.72901117443331 -74.19000550414022) bank69765 +69766 POINT(40.03924785629394 -73.47808365926055) bank69766 +69767 POINT(40.76952282755601 -74.30111683127372) bank69767 +69768 POINT(41.5183472915357 -73.12379400531316) bank69768 +69769 POINT(40.63414812232829 -74.75934902216638) bank69769 +69770 POINT(41.363039017406656 -73.66334700792243) bank69770 +69771 POINT(40.15058409057853 -73.29416625982063) bank69771 +69772 POINT(41.16849517383308 -73.61447639408512) bank69772 +69773 POINT(41.158085105921096 -73.72315919508422) bank69773 +69774 POINT(39.787999016516146 -73.33876971602054) bank69774 +69775 POINT(41.05482269514371 -73.9210080176285) bank69775 +69776 POINT(41.048302059473755 -73.03759219794034) bank69776 +69777 POINT(40.9746355499139 -74.59367269333028) bank69777 +69778 POINT(40.030953821403024 -73.17582512829071) bank69778 +69779 POINT(41.44591707015838 -74.06618697232237) bank69779 +69780 POINT(41.52744789043203 -74.86437682738344) bank69780 +69781 POINT(40.75412671005526 -74.90568044968698) bank69781 +69782 POINT(39.74815970372958 -73.45195248297979) bank69782 +69783 POINT(40.39949670464735 -73.36126729601612) bank69783 +69784 POINT(41.01673953926204 -73.37619460791355) bank69784 +69785 POINT(40.689060652169026 -73.35408934725352) bank69785 +69786 POINT(41.131846813288135 -74.41531514994597) bank69786 +69787 POINT(40.00970149127907 -73.2098188512666) bank69787 +69788 POINT(41.12446534190839 -73.30616588054859) bank69788 +69789 POINT(40.01465553956052 -74.80002026816415) bank69789 +69790 POINT(41.20753771376043 -74.76457235738332) bank69790 +69791 POINT(40.003008427919184 -74.20303862418906) bank69791 +69792 POINT(41.05166863747622 -74.75882423101993) bank69792 +69793 POINT(40.62932150065343 -73.92248332307935) bank69793 +69794 POINT(40.83224057804519 -73.29239232951623) bank69794 +69795 POINT(41.01824019199278 -74.76245483320245) bank69795 +69796 POINT(40.53809203602564 -73.11187483783337) bank69796 +69797 POINT(40.09038539764757 -74.80753267619076) bank69797 +69798 POINT(40.19856848433603 -73.08211999566996) bank69798 +69799 POINT(40.463543254845405 -74.64340058917625) bank69799 +69800 POINT(39.98732558646854 -73.62999994170582) bank69800 +69801 POINT(40.2328801207769 -74.94006842454709) bank69801 +69802 POINT(39.82097458943482 -74.72122057641448) bank69802 +69803 POINT(41.00528859545381 -74.20484030595881) bank69803 +69804 POINT(41.54045849331162 -74.77707841923299) bank69804 +69805 POINT(41.323057065327895 -73.20009745539576) bank69805 +69806 POINT(40.66989474732297 -74.04580092416084) bank69806 +69807 POINT(40.99979964040944 -73.64191596651249) bank69807 +69808 POINT(41.32640282860525 -73.47786732552694) bank69808 +69809 POINT(40.674458910793724 -74.9259653123122) bank69809 +69810 POINT(39.892044849986554 -74.10840956524363) bank69810 +69811 POINT(40.360437486302175 -73.65803532635235) bank69811 +69812 POINT(40.40451067415282 -74.4075149854905) bank69812 +69813 POINT(39.938620809806125 -74.65394183556009) bank69813 +69814 POINT(40.13370550269453 -73.94681216620982) bank69814 +69815 POINT(40.38335246540169 -74.87565870004849) bank69815 +69816 POINT(40.44157964698851 -73.34987528807207) bank69816 +69817 POINT(41.01868632467577 -74.38272174466456) bank69817 +69818 POINT(40.25842831328153 -73.37236975862204) bank69818 +69819 POINT(41.12827268199734 -74.53436357141766) bank69819 +69820 POINT(41.341719880716305 -73.29801350007483) bank69820 +69821 POINT(40.83806403443525 -74.84270080011592) bank69821 +69822 POINT(41.183311814892285 -74.45540318140323) bank69822 +69823 POINT(41.60027689983894 -74.49123506953477) bank69823 +69824 POINT(41.47526772201326 -73.30988391758098) bank69824 +69825 POINT(41.47525311444897 -73.53608895419273) bank69825 +69826 POINT(39.848543895713014 -73.0126241920049) bank69826 +69827 POINT(40.4311161960794 -74.7099136700529) bank69827 +69828 POINT(40.341947886366896 -73.50458247926723) bank69828 +69829 POINT(40.95460168431219 -74.32957755399192) bank69829 +69830 POINT(40.52076562691697 -74.31612463178524) bank69830 +69831 POINT(40.62568858370804 -73.8394150753478) bank69831 +69832 POINT(41.25800228607163 -74.0230033377193) bank69832 +69833 POINT(40.57034904664006 -73.53062259852688) bank69833 +69834 POINT(41.00916593123047 -73.52089487726187) bank69834 +69835 POINT(41.13088846324551 -73.34367896021908) bank69835 +69836 POINT(39.777062789029486 -74.05649129774473) bank69836 +69837 POINT(39.86374489743299 -74.98379838153743) bank69837 +69838 POINT(40.72472609686242 -73.29543352903492) bank69838 +69839 POINT(40.275576181878556 -74.43724888739176) bank69839 +69840 POINT(40.14111525053703 -74.18474446754085) bank69840 +69841 POINT(41.38519897206681 -74.2134618130155) bank69841 +69842 POINT(41.33855705524705 -74.24721141056149) bank69842 +69843 POINT(41.342211245387624 -74.90345938678277) bank69843 +69844 POINT(40.53291004700026 -73.6309074822025) bank69844 +69845 POINT(40.324157810530124 -73.67446711932139) bank69845 +69846 POINT(40.58953697358451 -74.97698740303173) bank69846 +69847 POINT(41.20682124688662 -74.83328727379883) bank69847 +69848 POINT(40.021377152598774 -74.07386963225458) bank69848 +69849 POINT(41.4550255868183 -73.68148081452755) bank69849 +69850 POINT(41.665728818621616 -74.74483928534026) bank69850 +69851 POINT(40.73374976419636 -74.6020497131427) bank69851 +69852 POINT(40.31714560290069 -74.6376506402066) bank69852 +69853 POINT(40.67921847914138 -74.90541439640965) bank69853 +69854 POINT(40.15536829897394 -73.53473566376356) bank69854 +69855 POINT(41.05558922913739 -73.70095967325177) bank69855 +69856 POINT(39.732993445041444 -73.99386650764657) bank69856 +69857 POINT(39.76003388884348 -73.5665643480541) bank69857 +69858 POINT(39.75648286088409 -73.25243608005731) bank69858 +69859 POINT(41.10609602124936 -73.65299699024847) bank69859 +69860 POINT(40.300700468789046 -74.3724437360966) bank69860 +69861 POINT(41.59505465084114 -74.1901732376412) bank69861 +69862 POINT(40.38736705805081 -74.91191701306609) bank69862 +69863 POINT(41.458457051996064 -73.71328905197008) bank69863 +69864 POINT(41.163131845076094 -74.05088797718903) bank69864 +69865 POINT(40.22407048020156 -73.10036256702979) bank69865 +69866 POINT(40.814927095092195 -74.71412469913034) bank69866 +69867 POINT(41.03905743580123 -73.64298823300422) bank69867 +69868 POINT(41.31485814490977 -73.70987165991455) bank69868 +69869 POINT(41.535775057755835 -74.63627572515183) bank69869 +69870 POINT(40.33238373518447 -73.63083271131268) bank69870 +69871 POINT(40.755229443368044 -73.54091978124082) bank69871 +69872 POINT(41.57449497076209 -73.59523348859575) bank69872 +69873 POINT(41.1608600158421 -74.02452561362185) bank69873 +69874 POINT(40.899052810999464 -73.81912538656891) bank69874 +69875 POINT(40.72033657559534 -73.96859081081533) bank69875 +69876 POINT(40.522679103383126 -74.84890970811355) bank69876 +69877 POINT(40.67661491628216 -73.79378540142324) bank69877 +69878 POINT(40.87558402303718 -74.87204164937515) bank69878 +69879 POINT(41.61757030591648 -74.69213857814067) bank69879 +69880 POINT(40.07343887913973 -73.1812041892086) bank69880 +69881 POINT(40.29578880959674 -74.89118419182932) bank69881 +69882 POINT(41.601494447106674 -74.47451090844905) bank69882 +69883 POINT(41.638126489886204 -74.65374512788641) bank69883 +69884 POINT(39.96120577419614 -74.73897078990922) bank69884 +69885 POINT(39.90028323811647 -74.22952268079214) bank69885 +69886 POINT(40.672295503531664 -73.55744768769374) bank69886 +69887 POINT(39.81746986573439 -73.84574482614534) bank69887 +69888 POINT(40.65956484546075 -73.71503557644488) bank69888 +69889 POINT(41.61260497313317 -74.16984907067476) bank69889 +69890 POINT(40.41247074202872 -74.17102558427034) bank69890 +69891 POINT(39.71503752801807 -73.16403901319492) bank69891 +69892 POINT(40.407109107269285 -73.04572415309875) bank69892 +69893 POINT(39.77966601218222 -74.64088731298165) bank69893 +69894 POINT(41.01555554234787 -73.6827697495883) bank69894 +69895 POINT(40.86695968498492 -75.0051472350482) bank69895 +69896 POINT(41.46737983263155 -74.89019704687587) bank69896 +69897 POINT(40.6896998267786 -74.21595480607901) bank69897 +69898 POINT(40.537958248364106 -74.58416824034438) bank69898 +69899 POINT(40.76870505211652 -73.2325463121845) bank69899 +69900 POINT(40.50794496333588 -73.32267692750678) bank69900 +69901 POINT(41.444384351772605 -74.30648432008653) bank69901 +69902 POINT(40.86078149127485 -74.01848877281455) bank69902 +69903 POINT(41.65978895285549 -74.75885311366191) bank69903 +69904 POINT(40.207496020194895 -73.88754609933531) bank69904 +69905 POINT(39.76252885021563 -73.96819648888341) bank69905 +69906 POINT(40.904193012198036 -74.75628342985561) bank69906 +69907 POINT(40.26723153691385 -74.38057217267043) bank69907 +69908 POINT(41.34421928339983 -74.38745076176897) bank69908 +69909 POINT(39.86071464544479 -74.87411274949862) bank69909 +69910 POINT(40.92907610078676 -74.78808006399564) bank69910 +69911 POINT(41.44606079513621 -74.86779201979984) bank69911 +69912 POINT(39.817804692273896 -73.88778486428936) bank69912 +69913 POINT(41.15135958419247 -74.45024469436822) bank69913 +69914 POINT(41.531857749088886 -74.10123895571202) bank69914 +69915 POINT(40.23806548200425 -74.95688407824) bank69915 +69916 POINT(40.5667487028478 -74.01316717049751) bank69916 +69917 POINT(41.61203072523868 -74.29561483021288) bank69917 +69918 POINT(39.945541609493915 -73.73313376304888) bank69918 +69919 POINT(41.22022250384138 -74.80392016678053) bank69919 +69920 POINT(41.62865085206884 -74.36875266815196) bank69920 +69921 POINT(41.06195337554208 -74.19699352610425) bank69921 +69922 POINT(40.41330967935645 -74.41376121753127) bank69922 +69923 POINT(41.47303151047134 -73.13006109405023) bank69923 +69924 POINT(40.07952143216238 -74.89273873661243) bank69924 +69925 POINT(39.79267080936956 -73.85729206520071) bank69925 +69926 POINT(41.63107155350904 -74.07119504835329) bank69926 +69927 POINT(41.239576818261376 -74.3697076593596) bank69927 +69928 POINT(40.38758033357002 -74.18216801603405) bank69928 +69929 POINT(40.16760803149682 -74.30605313334905) bank69929 +69930 POINT(40.83283653675065 -74.45049161611384) bank69930 +69931 POINT(41.186239625355455 -73.82329215030863) bank69931 +69932 POINT(39.92254167099916 -73.69518867620923) bank69932 +69933 POINT(40.93285248899053 -74.96912112072262) bank69933 +69934 POINT(39.765638372318534 -73.17272279939131) bank69934 +69935 POINT(41.40686552676356 -73.87380176701022) bank69935 +69936 POINT(40.13026173670852 -73.88222913507282) bank69936 +69937 POINT(40.72944176439078 -74.22668659166403) bank69937 +69938 POINT(41.59786924127796 -73.47029817998762) bank69938 +69939 POINT(40.45729209434994 -73.24960030708877) bank69939 +69940 POINT(40.24182191875543 -73.96065269949291) bank69940 +69941 POINT(40.93839179704308 -74.98387116342057) bank69941 +69942 POINT(40.0375072277075 -74.9589315055546) bank69942 +69943 POINT(41.03426926428763 -74.70111582327154) bank69943 +69944 POINT(40.23155187513949 -74.50145775105523) bank69944 +69945 POINT(41.26352157913465 -73.37061901061338) bank69945 +69946 POINT(40.69917948905236 -74.86875167842894) bank69946 +69947 POINT(41.325232802457236 -74.24795508531977) bank69947 +69948 POINT(41.49385952723413 -74.11419483451006) bank69948 +69949 POINT(40.899392586704785 -74.14209641083814) bank69949 +69950 POINT(40.7303506543387 -74.10629775366745) bank69950 +69951 POINT(41.57660575024185 -73.90400245772483) bank69951 +69952 POINT(41.70909258486057 -73.59376828856341) bank69952 +69953 POINT(41.316264753688316 -73.1464848611787) bank69953 +69954 POINT(41.17547197500885 -73.79000087093812) bank69954 +69955 POINT(41.23412232905551 -74.36341740550989) bank69955 +69956 POINT(41.679967727721646 -73.6475715035229) bank69956 +69957 POINT(41.684386691376844 -73.70559293811495) bank69957 +69958 POINT(39.798704528546224 -73.91071112149785) bank69958 +69959 POINT(41.03454527554519 -74.52269258958815) bank69959 +69960 POINT(40.05526542199179 -73.36714222822464) bank69960 +69961 POINT(40.65797783140259 -74.86560964485997) bank69961 +69962 POINT(40.5622043184073 -73.5515423768493) bank69962 +69963 POINT(41.376501450379486 -73.40586864001202) bank69963 +69964 POINT(40.285733398923085 -74.87990335716893) bank69964 +69965 POINT(41.37613940862922 -74.32077696108048) bank69965 +69966 POINT(40.37956505903244 -74.02144471456761) bank69966 +69967 POINT(41.35124218869891 -73.50342536407892) bank69967 +69968 POINT(41.662600313983866 -74.60147958073638) bank69968 +69969 POINT(41.0309404098467 -73.30366285772863) bank69969 +69970 POINT(40.49834765648628 -73.46858335500905) bank69970 +69971 POINT(41.081094905921724 -74.45205840013239) bank69971 +69972 POINT(40.29495913419509 -74.58528864380592) bank69972 +69973 POINT(40.355567076817444 -74.02461721481671) bank69973 +69974 POINT(40.506419063778296 -73.16923528834332) bank69974 +69975 POINT(41.447952235744005 -74.74435321827896) bank69975 +69976 POINT(40.61358213659304 -73.81825254237047) bank69976 +69977 POINT(39.8966842095152 -74.63290443101607) bank69977 +69978 POINT(41.31711151628198 -74.7187049187317) bank69978 +69979 POINT(40.88240213373201 -74.69191065257755) bank69979 +69980 POINT(40.35876676949527 -74.00235852767675) bank69980 +69981 POINT(41.29401049719982 -73.73764390522267) bank69981 +69982 POINT(39.87937513777817 -74.23932572936165) bank69982 +69983 POINT(40.604067028974875 -73.03752834346639) bank69983 +69984 POINT(39.99688407051402 -74.33435680010625) bank69984 +69985 POINT(40.0112284477269 -74.71071652019393) bank69985 +69986 POINT(40.1028612436028 -73.18542177044348) bank69986 +69987 POINT(40.250721249744 -74.65876076397703) bank69987 +69988 POINT(41.58869771373916 -73.30942316963298) bank69988 +69989 POINT(40.629991142643725 -74.82933242703002) bank69989 +69990 POINT(41.21198335436362 -73.53788216137704) bank69990 +69991 POINT(40.0882833813217 -73.4691153368106) bank69991 +69992 POINT(40.29127818593112 -74.96814262472955) bank69992 +69993 POINT(40.69803814800381 -73.25998301295178) bank69993 +69994 POINT(40.91155237507949 -74.51320675979612) bank69994 +69995 POINT(40.21615385130156 -74.49559906109769) bank69995 +69996 POINT(40.427250114674244 -73.13997999032016) bank69996 +69997 POINT(39.92059953051843 -74.3290667710064) bank69997 +69998 POINT(41.29821153897662 -74.91201142043687) bank69998 +69999 POINT(40.355823538580474 -74.14724890313568) bank69999 +70000 POINT(40.06042889452665 -73.32430492650256) bank70000 +70001 POINT(41.6857162789589 -74.03305514781847) bank70001 +70002 POINT(40.39058194444594 -74.4479072974968) bank70002 +70003 POINT(40.97976865053436 -73.36086559993187) bank70003 +70004 POINT(41.323654022618065 -73.06670644550852) bank70004 +70005 POINT(41.07551870803154 -73.07829405320216) bank70005 +70006 POINT(41.676877357997505 -73.64449674442993) bank70006 +70007 POINT(41.13559106851741 -74.17686409487051) bank70007 +70008 POINT(39.90538780657566 -74.14230676811178) bank70008 +70009 POINT(39.811954077737475 -73.28919139270093) bank70009 +70010 POINT(40.44202211643012 -74.14396116554236) bank70010 +70011 POINT(41.134880182092274 -74.0561772725632) bank70011 +70012 POINT(41.66914755781242 -74.88889047120126) bank70012 +70013 POINT(40.15469458761824 -74.14838898054232) bank70013 +70014 POINT(41.25838355976112 -74.93954403391416) bank70014 +70015 POINT(41.40149190200919 -74.00443664886133) bank70015 +70016 POINT(40.47781888523438 -73.94491808848416) bank70016 +70017 POINT(40.93527218080732 -73.90799625002876) bank70017 +70018 POINT(40.31035613630498 -73.56448095911028) bank70018 +70019 POINT(40.924059697542944 -74.2254664556022) bank70019 +70020 POINT(39.90995731182114 -74.92720935493907) bank70020 +70021 POINT(40.3667110303021 -74.73897026056083) bank70021 +70022 POINT(40.777365768132384 -73.93794191095257) bank70022 +70023 POINT(40.456837086525944 -73.67920375717515) bank70023 +70024 POINT(41.498525840290995 -73.56078357445506) bank70024 +70025 POINT(40.1741544416503 -75.00112960422445) bank70025 +70026 POINT(40.04804141082238 -73.50614078238195) bank70026 +70027 POINT(41.55781197881071 -74.70293548013068) bank70027 +70028 POINT(40.65065540303195 -74.86881244799973) bank70028 +70029 POINT(40.96723936475799 -74.70652542253795) bank70029 +70030 POINT(41.22295571409266 -73.27997879910079) bank70030 +70031 POINT(40.78137764164212 -73.57184458193271) bank70031 +70032 POINT(39.951796536568736 -73.44763439210939) bank70032 +70033 POINT(40.241592853618734 -74.3184748626207) bank70033 +70034 POINT(41.59529800972162 -73.95758900160556) bank70034 +70035 POINT(39.908891033211205 -74.27962303547207) bank70035 +70036 POINT(41.22917185240874 -73.55200123454459) bank70036 +70037 POINT(40.88999235614096 -74.24243805820095) bank70037 +70038 POINT(40.503457091423165 -73.55212236654363) bank70038 +70039 POINT(39.751725102552804 -73.38028037342907) bank70039 +70040 POINT(40.92685102419309 -74.48275298979395) bank70040 +70041 POINT(40.82336918761209 -74.94125326828875) bank70041 +70042 POINT(41.628799167861516 -73.55907058872425) bank70042 +70043 POINT(41.34638908549458 -73.45024257936844) bank70043 +70044 POINT(41.59091162732652 -73.47247567770117) bank70044 +70045 POINT(41.62357139769776 -74.43837442432078) bank70045 +70046 POINT(40.07237117913367 -74.64859282430295) bank70046 +70047 POINT(40.76243637100291 -74.3939795656823) bank70047 +70048 POINT(41.41789327834956 -74.05725133068123) bank70048 +70049 POINT(41.292206507208505 -73.23599393239319) bank70049 +70050 POINT(40.17715822712647 -74.05691627098405) bank70050 +70051 POINT(41.2099660063749 -74.06074616268796) bank70051 +70052 POINT(41.68530838777365 -74.26084515015899) bank70052 +70053 POINT(40.235917807829196 -73.4849400050115) bank70053 +70054 POINT(40.866008477684105 -74.37401994110097) bank70054 +70055 POINT(40.57513485959767 -74.41889171409609) bank70055 +70056 POINT(40.507891830100284 -74.49946355479173) bank70056 +70057 POINT(39.81902290340396 -73.89741958713614) bank70057 +70058 POINT(39.74932350027086 -73.8092205652599) bank70058 +70059 POINT(41.28887076644187 -73.42538081786779) bank70059 +70060 POINT(40.13129812393313 -73.46257011222548) bank70060 +70061 POINT(39.856375868223715 -73.46348619509338) bank70061 +70062 POINT(41.46578492867203 -74.20508629589361) bank70062 +70063 POINT(40.369024509447975 -73.62871252190112) bank70063 +70064 POINT(41.529123216040155 -73.7703117976476) bank70064 +70065 POINT(41.54152435515067 -74.49586258887996) bank70065 +70066 POINT(41.63654355599257 -73.0459857116928) bank70066 +70067 POINT(40.992260122009355 -74.7813189725185) bank70067 +70068 POINT(40.31524269007589 -74.6461014467649) bank70068 +70069 POINT(40.63707790702682 -74.70119198345104) bank70069 +70070 POINT(40.175334027270495 -74.29092678526833) bank70070 +70071 POINT(40.19752006903178 -74.6442314564731) bank70071 +70072 POINT(41.23228157471209 -74.0281040648062) bank70072 +70073 POINT(39.99279449747002 -73.96652274040692) bank70073 +70074 POINT(39.920958791321766 -73.51582444301144) bank70074 +70075 POINT(40.75760808485197 -74.15103578797392) bank70075 +70076 POINT(41.55884904192137 -73.83837073208134) bank70076 +70077 POINT(41.05584665384197 -74.12924597324852) bank70077 +70078 POINT(40.24449371105127 -74.64773734588482) bank70078 +70079 POINT(40.864005177382154 -74.41077455161079) bank70079 +70080 POINT(39.92646665376225 -73.84735907681431) bank70080 +70081 POINT(40.36383880862114 -73.6944432840625) bank70081 +70082 POINT(40.86788888639236 -73.11477071564556) bank70082 +70083 POINT(40.71919742731272 -74.85009264257715) bank70083 +70084 POINT(40.93868413846394 -73.74189950343626) bank70084 +70085 POINT(40.5491822342244 -73.1128481429506) bank70085 +70086 POINT(40.993678471501866 -74.36052384011998) bank70086 +70087 POINT(40.806073103436276 -73.5553129190558) bank70087 +70088 POINT(40.689464308525515 -73.34785411872993) bank70088 +70089 POINT(39.94900370455302 -74.40038564008887) bank70089 +70090 POINT(41.16785713829946 -73.73639862880782) bank70090 +70091 POINT(39.7636328727581 -74.5243358449816) bank70091 +70092 POINT(39.82118044245979 -73.89338794882659) bank70092 +70093 POINT(41.352470691003205 -73.88592675625223) bank70093 +70094 POINT(39.95314778915716 -74.20932972234995) bank70094 +70095 POINT(40.65540959280598 -74.8011030776356) bank70095 +70096 POINT(40.81685939345789 -73.01676680763725) bank70096 +70097 POINT(41.333821166542364 -73.28088714604503) bank70097 +70098 POINT(41.05296921800288 -74.23110206040687) bank70098 +70099 POINT(40.54001890052151 -73.84604434809124) bank70099 +70100 POINT(40.0350918463186 -74.80610955647118) bank70100 +70101 POINT(40.395661209404196 -73.34866394017216) bank70101 +70102 POINT(40.021114190489094 -74.91493014481506) bank70102 +70103 POINT(41.327733529487595 -73.14080474261736) bank70103 +70104 POINT(41.18392514194594 -73.51830463558358) bank70104 +70105 POINT(41.16712513269657 -74.92977475074771) bank70105 +70106 POINT(40.429644615351805 -73.958412508604) bank70106 +70107 POINT(40.80908190617772 -73.44352421359604) bank70107 +70108 POINT(39.9505210912595 -74.63275284213125) bank70108 +70109 POINT(40.43864831527606 -74.6881683288275) bank70109 +70110 POINT(41.6896319843242 -73.07058180478418) bank70110 +70111 POINT(40.26344452602186 -73.73116818335818) bank70111 +70112 POINT(40.42563443964587 -74.68206647723024) bank70112 +70113 POINT(41.45501638118437 -74.13701335287666) bank70113 +70114 POINT(40.73779317176711 -73.41262910524604) bank70114 +70115 POINT(41.45007678567824 -74.38348237783019) bank70115 +70116 POINT(40.60443287882818 -73.09444165019318) bank70116 +70117 POINT(41.588285315236206 -74.99456004460794) bank70117 +70118 POINT(40.66922206702953 -73.8768479394941) bank70118 +70119 POINT(40.61551584049544 -74.85413008829367) bank70119 +70120 POINT(41.64416672739396 -74.67242510747967) bank70120 +70121 POINT(41.63346835964526 -73.61447449580722) bank70121 +70122 POINT(40.260719919328366 -73.6266640739452) bank70122 +70123 POINT(41.45883176294181 -74.83010935621627) bank70123 +70124 POINT(40.204775784306385 -74.83002957446806) bank70124 +70125 POINT(40.692318525044065 -74.38510819266476) bank70125 +70126 POINT(41.693203020123974 -73.84074525106627) bank70126 +70127 POINT(39.90154997608371 -73.33348808142708) bank70127 +70128 POINT(41.61959642650661 -73.66128656174686) bank70128 +70129 POINT(40.3253854201404 -74.24882411042068) bank70129 +70130 POINT(39.94432366661121 -74.62246231770436) bank70130 +70131 POINT(40.48832547751865 -74.1858181220204) bank70131 +70132 POINT(41.17008693057548 -74.94558835235755) bank70132 +70133 POINT(41.063877577436735 -73.4673120848466) bank70133 +70134 POINT(41.554566747470616 -74.91432091736587) bank70134 +70135 POINT(40.58817281540215 -73.78730774294685) bank70135 +70136 POINT(41.07755061080551 -73.83557041519568) bank70136 +70137 POINT(40.3472084938182 -73.32845903793475) bank70137 +70138 POINT(41.37576072833612 -74.2084752677182) bank70138 +70139 POINT(41.25195974554279 -74.57089441606635) bank70139 +70140 POINT(41.203597226719715 -74.91606644767378) bank70140 +70141 POINT(41.62899033443571 -74.80391619694188) bank70141 +70142 POINT(40.724003498661006 -73.81508154690695) bank70142 +70143 POINT(40.075204337244514 -73.70377624736707) bank70143 +70144 POINT(41.2218997709515 -73.68696293686757) bank70144 +70145 POINT(39.9500308154751 -73.05558795379265) bank70145 +70146 POINT(41.08257269707613 -73.20937133007882) bank70146 +70147 POINT(40.27769509704066 -73.58947961493206) bank70147 +70148 POINT(40.23749876953434 -74.98419273134314) bank70148 +70149 POINT(41.283864149546794 -73.68242704792941) bank70149 +70150 POINT(41.53591290756064 -73.32810555389692) bank70150 +70151 POINT(41.424057377944656 -74.240214572639) bank70151 +70152 POINT(40.50649999052173 -73.74474328678048) bank70152 +70153 POINT(40.993185950448094 -73.6427116674847) bank70153 +70154 POINT(40.89552411474993 -74.19850062930651) bank70154 +70155 POINT(41.59148271208793 -73.1858408455274) bank70155 +70156 POINT(40.59423966899494 -73.40425907840317) bank70156 +70157 POINT(40.3611798544637 -74.40801090639677) bank70157 +70158 POINT(40.72784366429684 -74.10221968485165) bank70158 +70159 POINT(40.73090756528594 -74.03076831764227) bank70159 +70160 POINT(41.69657159392464 -73.728054151676) bank70160 +70161 POINT(40.95112541038439 -74.4231083944224) bank70161 +70162 POINT(41.46628027584534 -74.03528043786278) bank70162 +70163 POINT(40.10827321085161 -73.5713916077695) bank70163 +70164 POINT(40.79294404247968 -73.68713494934492) bank70164 +70165 POINT(40.07663705113268 -74.76552140547032) bank70165 +70166 POINT(41.392939678269336 -74.41148920567349) bank70166 +70167 POINT(39.972584367336175 -74.16491125527662) bank70167 +70168 POINT(40.00000953892257 -73.14448983796899) bank70168 +70169 POINT(41.64380387185106 -74.69597634068968) bank70169 +70170 POINT(39.907976131957085 -73.8588630929681) bank70170 +70171 POINT(39.80222554519881 -75.00567744883762) bank70171 +70172 POINT(41.41980173691265 -74.7319377479591) bank70172 +70173 POINT(40.791096808508556 -73.46802462288173) bank70173 +70174 POINT(39.816242462015936 -73.84189969932913) bank70174 +70175 POINT(40.88966304188694 -74.62134206194953) bank70175 +70176 POINT(40.182900528287945 -73.64639468421629) bank70176 +70177 POINT(40.520409067675665 -74.43713654495899) bank70177 +70178 POINT(41.685100274512145 -74.88347408451571) bank70178 +70179 POINT(40.60711660762589 -73.1263243614541) bank70179 +70180 POINT(40.51495205641272 -73.48290254442031) bank70180 +70181 POINT(41.66981288935707 -74.62381682120215) bank70181 +70182 POINT(40.10410273018602 -73.43376442988567) bank70182 +70183 POINT(40.6631175072526 -74.50135083058784) bank70183 +70184 POINT(41.083787386820504 -74.73629767034922) bank70184 +70185 POINT(39.76391145678744 -74.48374674522579) bank70185 +70186 POINT(41.38048474082774 -73.23986974731899) bank70186 +70187 POINT(41.11146449763008 -74.39016331187189) bank70187 +70188 POINT(39.8299583692552 -73.97076509984804) bank70188 +70189 POINT(39.734172048628984 -73.32353380995144) bank70189 +70190 POINT(40.67589444760574 -73.23417070316589) bank70190 +70191 POINT(40.62210284163073 -74.6911847068096) bank70191 +70192 POINT(40.408264799007085 -73.60301105672225) bank70192 +70193 POINT(40.06416605888984 -74.04108512920753) bank70193 +70194 POINT(41.23467952162498 -73.36230781704197) bank70194 +70195 POINT(40.58515451611166 -74.18840386295588) bank70195 +70196 POINT(40.191198732435076 -74.40019243852666) bank70196 +70197 POINT(41.43732567473713 -74.22632591025643) bank70197 +70198 POINT(41.66724101978888 -74.24556604632342) bank70198 +70199 POINT(41.09531743936332 -74.47837341714606) bank70199 +70200 POINT(40.13595375730378 -73.21602427280847) bank70200 +70201 POINT(39.95697354742629 -74.68423875462626) bank70201 +70202 POINT(39.85285686966573 -74.11804664998358) bank70202 +70203 POINT(40.63004589698071 -74.76599762377973) bank70203 +70204 POINT(41.632419672041785 -73.77792573098058) bank70204 +70205 POINT(41.5681321391089 -74.85959602013955) bank70205 +70206 POINT(41.02336768230124 -73.42306584198279) bank70206 +70207 POINT(40.995152697082055 -74.91632600544783) bank70207 +70208 POINT(41.56927139463795 -74.33559105814004) bank70208 +70209 POINT(39.94276722016819 -74.25118229304869) bank70209 +70210 POINT(41.68200168444748 -73.00921578231134) bank70210 +70211 POINT(40.50558527350893 -73.37206792825666) bank70211 +70212 POINT(41.57588120781689 -73.41868029606887) bank70212 +70213 POINT(40.132293838255734 -73.93366806918358) bank70213 +70214 POINT(41.072548893401134 -73.173613131131) bank70214 +70215 POINT(40.37041572856767 -73.64872444256443) bank70215 +70216 POINT(41.457970690760085 -74.91848416848858) bank70216 +70217 POINT(40.158844586952235 -74.64755209122097) bank70217 +70218 POINT(40.3422886362427 -73.34809615819115) bank70218 +70219 POINT(41.32253704943894 -73.95161261139621) bank70219 +70220 POINT(40.78411587719739 -73.92430296303746) bank70220 +70221 POINT(41.10209920617791 -73.49476527779774) bank70221 +70222 POINT(41.22348888839185 -73.83737106937352) bank70222 +70223 POINT(41.02493372240995 -74.25666366884217) bank70223 +70224 POINT(41.01934633052212 -73.05431436890684) bank70224 +70225 POINT(40.08260278037156 -73.9129230227783) bank70225 +70226 POINT(40.93396891965625 -73.87224091199008) bank70226 +70227 POINT(40.00314928921265 -73.57294201567865) bank70227 +70228 POINT(41.472538007023466 -73.73676830697472) bank70228 +70229 POINT(40.83322046233266 -74.30430791264891) bank70229 +70230 POINT(40.33690094537273 -74.45987185119093) bank70230 +70231 POINT(41.17684530394688 -74.23229033119465) bank70231 +70232 POINT(41.31506818407789 -74.39923022869044) bank70232 +70233 POINT(39.78112382013874 -73.75125073719937) bank70233 +70234 POINT(41.00106001645512 -73.25623028838703) bank70234 +70235 POINT(40.91284155446494 -73.04774086585117) bank70235 +70236 POINT(39.717719588420934 -74.00895667157452) bank70236 +70237 POINT(40.26083924510762 -73.32478579274235) bank70237 +70238 POINT(40.31849046271756 -74.82770018292426) bank70238 +70239 POINT(40.46243923639347 -74.5568672438768) bank70239 +70240 POINT(41.15142274950112 -73.30317015340931) bank70240 +70241 POINT(40.67687376453912 -74.13472854370556) bank70241 +70242 POINT(41.318036201489676 -74.27768996313004) bank70242 +70243 POINT(41.61042103323501 -74.37055633210116) bank70243 +70244 POINT(40.55931827124989 -74.79128581746393) bank70244 +70245 POINT(39.87725057181186 -73.96923748580734) bank70245 +70246 POINT(41.030717232221164 -74.48126787011533) bank70246 +70247 POINT(40.0190650509366 -74.93913052958713) bank70247 +70248 POINT(40.87428280661068 -73.43965884614659) bank70248 +70249 POINT(41.101226925853645 -73.52537807146047) bank70249 +70250 POINT(40.7547116137103 -74.2317158025017) bank70250 +70251 POINT(40.61742259281347 -74.26060339536141) bank70251 +70252 POINT(40.567747169260024 -74.45972425213057) bank70252 +70253 POINT(40.47255323576015 -73.50142323757642) bank70253 +70254 POINT(40.66055178647407 -73.41105084990113) bank70254 +70255 POINT(40.375715906299384 -74.87289082279358) bank70255 +70256 POINT(40.21325946728787 -74.53607377235915) bank70256 +70257 POINT(40.691878200312765 -74.91079611491745) bank70257 +70258 POINT(39.91881831105892 -73.16371046232403) bank70258 +70259 POINT(41.281843726312 -74.8462637690167) bank70259 +70260 POINT(40.5194468926568 -73.93367874579269) bank70260 +70261 POINT(41.13975911948141 -74.36999463230885) bank70261 +70262 POINT(40.74175228037635 -74.2977998828029) bank70262 +70263 POINT(40.552785409620434 -74.39718720801952) bank70263 +70264 POINT(41.53809345819043 -74.64657546003616) bank70264 +70265 POINT(41.60696508571203 -73.65646767508431) bank70265 +70266 POINT(41.34139840562678 -74.09755366359192) bank70266 +70267 POINT(40.60103980448671 -74.37460786690046) bank70267 +70268 POINT(39.824871071211724 -74.81203036798738) bank70268 +70269 POINT(41.607041561382495 -73.00781424285982) bank70269 +70270 POINT(41.03725730539628 -74.99211767824333) bank70270 +70271 POINT(39.886916487834576 -74.35672481896334) bank70271 +70272 POINT(41.60259898655933 -74.8199842536742) bank70272 +70273 POINT(41.42841018883889 -74.95002258143403) bank70273 +70274 POINT(40.261808985134905 -73.87516489247925) bank70274 +70275 POINT(41.42238766080608 -74.34031707783149) bank70275 +70276 POINT(40.01803533170067 -73.58843011486802) bank70276 +70277 POINT(41.49355798513887 -73.0681631850664) bank70277 +70278 POINT(40.45729369489622 -74.81070450841247) bank70278 +70279 POINT(40.15253667027504 -74.10307008494726) bank70279 +70280 POINT(39.75823881175935 -73.52996024114255) bank70280 +70281 POINT(40.56353373199712 -74.45341197528903) bank70281 +70282 POINT(40.09658208820007 -74.81741653629295) bank70282 +70283 POINT(40.350653402163694 -73.71768863135016) bank70283 +70284 POINT(41.527078669174124 -74.37195622529698) bank70284 +70285 POINT(39.735913828145364 -73.17293800650738) bank70285 +70286 POINT(40.41454341709738 -73.54175939259169) bank70286 +70287 POINT(41.56302879224184 -73.54698389624414) bank70287 +70288 POINT(41.37126034265792 -73.80407228409106) bank70288 +70289 POINT(41.574720663832224 -73.38554786802015) bank70289 +70290 POINT(40.032837588957776 -73.60706841064426) bank70290 +70291 POINT(40.72531833298283 -73.63103613942903) bank70291 +70292 POINT(41.561854628465866 -73.33441220211749) bank70292 +70293 POINT(40.9197693925998 -74.95989844490006) bank70293 +70294 POINT(41.32589422263229 -74.87361164317592) bank70294 +70295 POINT(40.164362826435514 -74.930306135639) bank70295 +70296 POINT(40.594971361455194 -74.86790891581211) bank70296 +70297 POINT(39.96270731215058 -74.5671473480838) bank70297 +70298 POINT(40.68398276360085 -73.91046337067324) bank70298 +70299 POINT(41.5740546502776 -74.45084703374452) bank70299 +70300 POINT(41.28024514090634 -73.66903033334403) bank70300 +70301 POINT(41.34367793612149 -74.67269257858831) bank70301 +70302 POINT(39.80377802361376 -74.60845671438082) bank70302 +70303 POINT(40.47907546678029 -74.47596898945528) bank70303 +70304 POINT(39.832757404551714 -73.77710581597447) bank70304 +70305 POINT(40.889350526831215 -73.23845603527) bank70305 +70306 POINT(40.15470211122096 -74.29809467329333) bank70306 +70307 POINT(39.921095219610976 -73.33662050052627) bank70307 +70308 POINT(41.17081948835773 -73.74330782495734) bank70308 +70309 POINT(40.51741919983275 -73.68783468187652) bank70309 +70310 POINT(39.917068728903836 -73.03088111014003) bank70310 +70311 POINT(40.42022737794755 -74.30587339705295) bank70311 +70312 POINT(40.010388062750955 -73.17239051801191) bank70312 +70313 POINT(40.34262218558512 -74.45126613354961) bank70313 +70314 POINT(41.49162432513573 -74.61618369445782) bank70314 +70315 POINT(40.382648396786685 -74.42709848895211) bank70315 +70316 POINT(41.032316170571896 -74.59331902769382) bank70316 +70317 POINT(40.59733543287588 -73.09161757130258) bank70317 +70318 POINT(40.241016564903724 -74.3598367436394) bank70318 +70319 POINT(39.92231132605111 -74.19716514125643) bank70319 +70320 POINT(41.45117631568319 -73.93993814486254) bank70320 +70321 POINT(39.80835456883046 -73.46736892428912) bank70321 +70322 POINT(41.47116461864532 -74.82995234723553) bank70322 +70323 POINT(39.826688628250345 -73.54311051496342) bank70323 +70324 POINT(40.696154023755426 -73.45492872306166) bank70324 +70325 POINT(40.58706105538594 -74.43694675490654) bank70325 +70326 POINT(40.682803618051274 -73.05644362778803) bank70326 +70327 POINT(39.80207274082156 -74.64994460632393) bank70327 +70328 POINT(41.022913368571274 -73.50925452173149) bank70328 +70329 POINT(40.42041878753793 -74.45459916731625) bank70329 +70330 POINT(39.72047121865597 -74.39284653493577) bank70330 +70331 POINT(41.28077837892828 -74.21923332430467) bank70331 +70332 POINT(40.327323042297486 -74.33624858239169) bank70332 +70333 POINT(40.430180265446424 -74.2414837386711) bank70333 +70334 POINT(40.10574849542566 -74.72592127721583) bank70334 +70335 POINT(41.11854617542118 -73.41330600079361) bank70335 +70336 POINT(41.57957792668727 -73.10652206620615) bank70336 +70337 POINT(40.501723311069156 -74.1410398427256) bank70337 +70338 POINT(41.22800757537264 -74.29003551334569) bank70338 +70339 POINT(40.26964535319858 -73.69812428247563) bank70339 +70340 POINT(39.85806808296382 -74.05176353795638) bank70340 +70341 POINT(40.5054120536323 -74.36344895187527) bank70341 +70342 POINT(41.061643977489354 -73.31252641927082) bank70342 +70343 POINT(40.7332915351284 -73.9546650639808) bank70343 +70344 POINT(40.05848750008759 -73.84416424205499) bank70344 +70345 POINT(39.78374581051736 -73.27615916120973) bank70345 +70346 POINT(40.97435187642942 -74.37001462161625) bank70346 +70347 POINT(40.802956333976866 -73.5934796576049) bank70347 +70348 POINT(40.60716046984391 -73.30139625047059) bank70348 +70349 POINT(40.38802778302024 -74.17360318741137) bank70349 +70350 POINT(41.5254781068153 -74.5327260584046) bank70350 +70351 POINT(40.66423941510859 -74.12128667319698) bank70351 +70352 POINT(39.921173591531804 -74.98695800139377) bank70352 +70353 POINT(39.806523031875685 -74.66464892368633) bank70353 +70354 POINT(39.905336692861376 -74.11175572076718) bank70354 +70355 POINT(41.06656384062125 -73.0834446468838) bank70355 +70356 POINT(40.70002672622918 -73.83328814808189) bank70356 +70357 POINT(40.43859237366412 -73.56343899414705) bank70357 +70358 POINT(40.33532582332324 -74.23137246448303) bank70358 +70359 POINT(41.36012627104484 -73.84168471838154) bank70359 +70360 POINT(39.82729788719279 -74.43466503496506) bank70360 +70361 POINT(40.81922188953863 -73.65318960961727) bank70361 +70362 POINT(39.979044130497385 -74.74688836162365) bank70362 +70363 POINT(40.669532850053386 -74.02957799938825) bank70363 +70364 POINT(39.767067013664004 -74.00201862310163) bank70364 +70365 POINT(41.3349929079478 -74.17094094600755) bank70365 +70366 POINT(40.37885987293148 -74.39593231471319) bank70366 +70367 POINT(40.551044722450165 -73.74110351569544) bank70367 +70368 POINT(40.428693031725224 -74.54591449483205) bank70368 +70369 POINT(41.41390805622645 -74.24796089277535) bank70369 +70370 POINT(39.954110477348465 -74.39501387644279) bank70370 +70371 POINT(41.509626319407616 -73.19484557579376) bank70371 +70372 POINT(41.70170533151743 -74.54746093897987) bank70372 +70373 POINT(40.66013358504726 -73.85189617259928) bank70373 +70374 POINT(39.85968615763286 -73.3454976383792) bank70374 +70375 POINT(40.897944103853455 -74.30979292973883) bank70375 +70376 POINT(41.52115434990137 -73.44253911194443) bank70376 +70377 POINT(40.47892391692157 -73.72758348467207) bank70377 +70378 POINT(40.60539772938316 -73.25010093381202) bank70378 +70379 POINT(40.037708638865794 -73.05667322165375) bank70379 +70380 POINT(40.93398613780205 -73.30908252408051) bank70380 +70381 POINT(40.97954566182041 -73.22612441017309) bank70381 +70382 POINT(40.34658980990065 -74.10253787716832) bank70382 +70383 POINT(41.473658756500484 -74.42290240110808) bank70383 +70384 POINT(40.15030143270146 -73.56151380889541) bank70384 +70385 POINT(41.48390886660131 -74.36305648470696) bank70385 +70386 POINT(41.455106042578066 -74.07000670400045) bank70386 +70387 POINT(40.69168983258809 -74.20745690453998) bank70387 +70388 POINT(40.00951165906857 -73.8550306425416) bank70388 +70389 POINT(40.07169568574024 -73.93657491564711) bank70389 +70390 POINT(40.56306461951288 -73.55626866657376) bank70390 +70391 POINT(39.82626158316322 -74.80925054690337) bank70391 +70392 POINT(41.13608777873604 -74.96424899557603) bank70392 +70393 POINT(40.58921810919725 -73.22309684262147) bank70393 +70394 POINT(40.07598278574673 -74.54957971579573) bank70394 +70395 POINT(40.514229823223324 -74.55172596465478) bank70395 +70396 POINT(40.872183204553345 -74.25866855081405) bank70396 +70397 POINT(40.518210869901196 -73.40221253404503) bank70397 +70398 POINT(40.591820453618155 -74.08838409018477) bank70398 +70399 POINT(40.231618071117964 -73.61050520645827) bank70399 +70400 POINT(41.427990942554445 -73.99503032131237) bank70400 +70401 POINT(41.63540384911884 -74.85006142388086) bank70401 +70402 POINT(41.3079626448726 -73.80983132517785) bank70402 +70403 POINT(41.18900169242733 -74.83685153907368) bank70403 +70404 POINT(41.22119930932467 -73.33645888821034) bank70404 +70405 POINT(40.92958803817498 -74.09419400217429) bank70405 +70406 POINT(40.29843949683939 -73.61562140761042) bank70406 +70407 POINT(40.827867706238166 -73.57141001430132) bank70407 +70408 POINT(41.3411697540565 -74.03430625857379) bank70408 +70409 POINT(39.716000354355444 -73.36462099462571) bank70409 +70410 POINT(40.97014429712954 -73.86287840453599) bank70410 +70411 POINT(40.810791307348715 -73.39997799959518) bank70411 +70412 POINT(40.34983134036071 -73.63438014611144) bank70412 +70413 POINT(39.96782743945484 -74.34452140823852) bank70413 +70414 POINT(40.72599351571415 -73.58176542049381) bank70414 +70415 POINT(40.787943803257704 -74.9419715180637) bank70415 +70416 POINT(40.6463040154214 -74.86992857088704) bank70416 +70417 POINT(40.09727529524811 -73.73154713125099) bank70417 +70418 POINT(40.848288239717974 -73.88758813583051) bank70418 +70419 POINT(40.2350509386955 -73.42513791127787) bank70419 +70420 POINT(39.7136037265988 -73.03512398819576) bank70420 +70421 POINT(40.09162183992268 -74.68305548055217) bank70421 +70422 POINT(41.54254363847042 -73.22109965531799) bank70422 +70423 POINT(40.437744844083824 -74.6415065395262) bank70423 +70424 POINT(40.41246648149205 -73.75249154121433) bank70424 +70425 POINT(40.05759821137653 -73.22391033104174) bank70425 +70426 POINT(41.26690153123568 -74.6117351611781) bank70426 +70427 POINT(40.78004094993379 -73.75462359140936) bank70427 +70428 POINT(41.422189244092 -73.15580065938117) bank70428 +70429 POINT(39.75422171879175 -73.9234371320708) bank70429 +70430 POINT(40.46962789594749 -74.72257729224494) bank70430 +70431 POINT(40.984564114995024 -73.54714847357505) bank70431 +70432 POINT(41.273465385201824 -73.84024513609549) bank70432 +70433 POINT(40.35985180113824 -73.27704915099191) bank70433 +70434 POINT(41.13172115475571 -74.80714480135464) bank70434 +70435 POINT(39.75078129876403 -73.8600085557267) bank70435 +70436 POINT(41.27710235611077 -74.58640318547128) bank70436 +70437 POINT(40.192173461817774 -74.01246249529267) bank70437 +70438 POINT(41.01770271299168 -74.28975374489873) bank70438 +70439 POINT(40.67637957435424 -74.51227377815428) bank70439 +70440 POINT(41.35667144629088 -74.51836410069703) bank70440 +70441 POINT(40.10130222419498 -73.2142827884883) bank70441 +70442 POINT(40.37684079191903 -73.07288820233777) bank70442 +70443 POINT(39.89658300768877 -73.01577585117889) bank70443 +70444 POINT(40.7879834196835 -74.19147094296848) bank70444 +70445 POINT(40.35981669275306 -73.49888731146734) bank70445 +70446 POINT(39.87293910458402 -73.18715372515848) bank70446 +70447 POINT(41.642016295094905 -74.92585445017318) bank70447 +70448 POINT(41.58602547816441 -73.24689073066715) bank70448 +70449 POINT(39.77173408441297 -73.55899514456223) bank70449 +70450 POINT(39.79810154793907 -74.48377843022078) bank70450 +70451 POINT(39.767326204195356 -74.38503095975922) bank70451 +70452 POINT(40.921944043048 -73.0845236791493) bank70452 +70453 POINT(40.310126767317264 -74.78396489847388) bank70453 +70454 POINT(40.1430496492822 -73.05878917492758) bank70454 +70455 POINT(39.96682227526534 -74.735507110356) bank70455 +70456 POINT(40.219746505310134 -74.06411377391305) bank70456 +70457 POINT(39.88780365554363 -73.97205077417227) bank70457 +70458 POINT(41.37708774014154 -73.3434057834492) bank70458 +70459 POINT(39.75108426058207 -74.0251366531823) bank70459 +70460 POINT(40.24335048691256 -73.4679514101677) bank70460 +70461 POINT(39.79501911338562 -73.08781878509001) bank70461 +70462 POINT(41.1844635612268 -74.98837442464307) bank70462 +70463 POINT(40.18335683717109 -74.27659779967225) bank70463 +70464 POINT(41.09811357142531 -73.8261881044942) bank70464 +70465 POINT(40.1576874806566 -74.32247414259683) bank70465 +70466 POINT(41.600063825213354 -73.80268198748165) bank70466 +70467 POINT(41.16220158409041 -74.9463666140069) bank70467 +70468 POINT(41.146348844877664 -73.36842073142611) bank70468 +70469 POINT(40.083972775556816 -73.06586938257078) bank70469 +70470 POINT(40.752052779037385 -73.09030077818491) bank70470 +70471 POINT(40.11253446498751 -74.985189464488) bank70471 +70472 POINT(39.78720487967825 -73.44864842199132) bank70472 +70473 POINT(40.38500773284886 -73.11988615927369) bank70473 +70474 POINT(41.15864168571257 -73.07084870978825) bank70474 +70475 POINT(41.48357951330298 -73.93839685871859) bank70475 +70476 POINT(40.15635434184066 -73.71515733173597) bank70476 +70477 POINT(41.10863827439539 -73.42261242346038) bank70477 +70478 POINT(41.549374462148116 -73.16539616259513) bank70478 +70479 POINT(41.16943871621507 -74.62440404125829) bank70479 +70480 POINT(41.69861635444204 -73.0314986290401) bank70480 +70481 POINT(40.19554729463893 -73.58142035052448) bank70481 +70482 POINT(40.13031452345057 -73.51020584348343) bank70482 +70483 POINT(41.61639417511902 -73.09529022910401) bank70483 +70484 POINT(41.66825846998568 -74.11836013244414) bank70484 +70485 POINT(39.82737965122748 -74.71671733380018) bank70485 +70486 POINT(40.170591837334115 -73.98648348793007) bank70486 +70487 POINT(40.54286581266675 -73.80630500283712) bank70487 +70488 POINT(40.359201157553365 -74.093566832143) bank70488 +70489 POINT(41.31309723365007 -73.21362115187458) bank70489 +70490 POINT(40.83559019986879 -74.78281285137491) bank70490 +70491 POINT(40.782408381838444 -74.81235668803706) bank70491 +70492 POINT(41.589066423664136 -73.32771947709394) bank70492 +70493 POINT(41.67826243518081 -73.58893906723453) bank70493 +70494 POINT(40.04856780279233 -73.28305475026171) bank70494 +70495 POINT(39.99675266685983 -73.74170875395677) bank70495 +70496 POINT(41.18946629421105 -74.7903154474852) bank70496 +70497 POINT(41.631305383898415 -73.046943317269) bank70497 +70498 POINT(41.46052205726736 -73.80317380136972) bank70498 +70499 POINT(40.91945404310905 -74.45443219798759) bank70499 +70500 POINT(39.72169666462067 -73.25059078927998) bank70500 +70501 POINT(39.83037396538779 -74.0807368198293) bank70501 +70502 POINT(40.32379623262344 -73.52450413958535) bank70502 +70503 POINT(41.70385112603741 -73.68013647134097) bank70503 +70504 POINT(40.871928769651525 -74.01096544578978) bank70504 +70505 POINT(41.3593516468069 -74.64588917213803) bank70505 +70506 POINT(40.32352351377977 -73.9780580697628) bank70506 +70507 POINT(41.04869945310554 -74.72971416243945) bank70507 +70508 POINT(40.77968964864124 -74.38749838421961) bank70508 +70509 POINT(39.78584149761341 -73.7693675059147) bank70509 +70510 POINT(40.23545093542021 -74.7795528962685) bank70510 +70511 POINT(41.614095609191914 -74.1936891014197) bank70511 +70512 POINT(40.9352478093243 -74.34066705324445) bank70512 +70513 POINT(39.83368661854029 -74.69846578669392) bank70513 +70514 POINT(39.750444875817244 -74.35129102262182) bank70514 +70515 POINT(40.4454764853511 -74.99907546607008) bank70515 +70516 POINT(40.99452912877646 -74.4371632119301) bank70516 +70517 POINT(40.02677995213998 -73.41231600363972) bank70517 +70518 POINT(40.320628403664415 -74.53418080518078) bank70518 +70519 POINT(40.10395353309302 -73.404025833358) bank70519 +70520 POINT(39.96450727768429 -73.65744356474715) bank70520 +70521 POINT(39.83435699792946 -74.24168821450401) bank70521 +70522 POINT(40.73164398052612 -73.26862445725273) bank70522 +70523 POINT(40.54905489150934 -74.17721528712464) bank70523 +70524 POINT(40.71090459120525 -74.05323460732644) bank70524 +70525 POINT(39.99249659592178 -74.4877207003023) bank70525 +70526 POINT(41.52658291316093 -74.45321892598247) bank70526 +70527 POINT(40.30282697168554 -73.76366150191922) bank70527 +70528 POINT(40.13291455630028 -73.04395205203778) bank70528 +70529 POINT(40.86896018336655 -74.80185697654406) bank70529 +70530 POINT(41.230680186944625 -74.24373892630969) bank70530 +70531 POINT(40.21599654109705 -74.41782950170511) bank70531 +70532 POINT(41.10039790097728 -73.74878088036448) bank70532 +70533 POINT(41.53383440110821 -73.57641938167625) bank70533 +70534 POINT(41.63840793205645 -73.26202858998423) bank70534 +70535 POINT(41.39183294179724 -73.06626025648386) bank70535 +70536 POINT(40.81778807717659 -73.2768606515908) bank70536 +70537 POINT(39.89993640060085 -73.43076645586034) bank70537 +70538 POINT(41.04994747540816 -74.89491328888107) bank70538 +70539 POINT(40.68141253023914 -73.77523175628123) bank70539 +70540 POINT(39.86809671147043 -74.08981579264763) bank70540 +70541 POINT(40.82580223141111 -73.536017797008) bank70541 +70542 POINT(41.13764104092319 -74.05158798250814) bank70542 +70543 POINT(41.4732220076195 -74.05762228943593) bank70543 +70544 POINT(39.7207223747459 -73.51987017675557) bank70544 +70545 POINT(41.367077515447924 -74.60497201475809) bank70545 +70546 POINT(40.255343827349485 -74.56885944905065) bank70546 +70547 POINT(40.95771241380992 -73.02062123366791) bank70547 +70548 POINT(41.10335620924156 -73.8287333736374) bank70548 +70549 POINT(40.33795718794839 -74.80515684653851) bank70549 +70550 POINT(40.820412705382 -74.91740538192127) bank70550 +70551 POINT(41.56727744997109 -74.2055149217824) bank70551 +70552 POINT(41.28546573396864 -74.44969050148204) bank70552 +70553 POINT(41.54402636158091 -73.38610656101719) bank70553 +70554 POINT(41.35330198158132 -73.89480803281144) bank70554 +70555 POINT(41.66411660226641 -73.48430861876649) bank70555 +70556 POINT(40.28026642259113 -73.56388032866786) bank70556 +70557 POINT(41.53680681047772 -73.91689204188808) bank70557 +70558 POINT(41.04519069179356 -73.71374428240433) bank70558 +70559 POINT(41.57583819601163 -73.19246696577059) bank70559 +70560 POINT(40.98984856689643 -73.32686181678874) bank70560 +70561 POINT(41.39754927906975 -74.37261794033245) bank70561 +70562 POINT(41.582217788032516 -74.75025265131612) bank70562 +70563 POINT(41.65479329657244 -74.42733108971882) bank70563 +70564 POINT(40.04797110435774 -73.19274703136533) bank70564 +70565 POINT(41.04481467879868 -73.73522170126614) bank70565 +70566 POINT(40.48994716880676 -73.88830575844412) bank70566 +70567 POINT(40.9193017589885 -73.23066742057955) bank70567 +70568 POINT(39.95705298521867 -74.92463759413212) bank70568 +70569 POINT(40.970561044296886 -73.56078446228145) bank70569 +70570 POINT(40.661927163047 -73.11309624600916) bank70570 +70571 POINT(41.6189317544815 -73.41762453437457) bank70571 +70572 POINT(41.524363704572835 -73.52707556582595) bank70572 +70573 POINT(41.615097279524974 -74.56093395711123) bank70573 +70574 POINT(39.920853474545616 -73.80992557603771) bank70574 +70575 POINT(40.52081105621991 -74.52807782705715) bank70575 +70576 POINT(41.27090726855243 -73.77407333327022) bank70576 +70577 POINT(40.53344327175847 -74.02551149698576) bank70577 +70578 POINT(41.41210817149561 -74.29401317174097) bank70578 +70579 POINT(40.06180214399863 -75.00256786731416) bank70579 +70580 POINT(40.26929852970153 -74.58961859666071) bank70580 +70581 POINT(40.15853022318425 -73.35551914724302) bank70581 +70582 POINT(41.60816541440447 -74.44330846138756) bank70582 +70583 POINT(39.793604696676276 -74.47797846996747) bank70583 +70584 POINT(41.680000753909084 -74.64205688453397) bank70584 +70585 POINT(40.3957822802641 -74.44208209755438) bank70585 +70586 POINT(39.79874998277159 -73.66546496630355) bank70586 +70587 POINT(41.69122050868968 -74.28697196089061) bank70587 +70588 POINT(40.08766278391681 -73.15712972805173) bank70588 +70589 POINT(40.264962892574175 -74.9041167293082) bank70589 +70590 POINT(41.395386676345495 -73.80851159133867) bank70590 +70591 POINT(41.44476087377535 -74.88296588902901) bank70591 +70592 POINT(40.84418707734656 -73.04786075429391) bank70592 +70593 POINT(40.04518886307073 -73.664507021326) bank70593 +70594 POINT(40.43420992451273 -74.80347871224996) bank70594 +70595 POINT(40.44108744750499 -73.78851194730292) bank70595 +70596 POINT(41.23854863636932 -74.67987104372963) bank70596 +70597 POINT(40.592833716926584 -73.97976728039019) bank70597 +70598 POINT(41.421749364210804 -73.49621418793218) bank70598 +70599 POINT(39.97809544545008 -74.13901222188592) bank70599 +70600 POINT(41.60198963279312 -74.42251833613093) bank70600 +70601 POINT(41.03144744500186 -73.89406629863156) bank70601 +70602 POINT(40.72814757069228 -73.61921163246924) bank70602 +70603 POINT(40.79317591418732 -74.50765284947683) bank70603 +70604 POINT(40.447956877600085 -74.83136500507644) bank70604 +70605 POINT(41.52420401595807 -74.34184059799162) bank70605 +70606 POINT(40.8667580702652 -74.99930790740842) bank70606 +70607 POINT(40.95272369107647 -74.23196732220842) bank70607 +70608 POINT(41.45820840435952 -74.42295692495678) bank70608 +70609 POINT(39.94579119838021 -74.83396481400383) bank70609 +70610 POINT(40.05650240891297 -74.94703089742943) bank70610 +70611 POINT(40.25931374988603 -73.9125459334256) bank70611 +70612 POINT(41.1206311514847 -73.97473448809356) bank70612 +70613 POINT(41.34794122831796 -74.07490677898366) bank70613 +70614 POINT(41.17630294522654 -73.89125224180125) bank70614 +70615 POINT(39.8285317073593 -73.57913299524304) bank70615 +70616 POINT(40.500398126349666 -73.78951370620845) bank70616 +70617 POINT(41.447222324284205 -73.67604078280954) bank70617 +70618 POINT(39.73539006215776 -74.96476020147199) bank70618 +70619 POINT(40.21802736342786 -73.4646529990406) bank70619 +70620 POINT(41.6263653187441 -74.0968629664495) bank70620 +70621 POINT(41.080920811392616 -74.48303604101746) bank70621 +70622 POINT(40.29966709752271 -74.01426160140349) bank70622 +70623 POINT(41.148304223141714 -73.44133176622508) bank70623 +70624 POINT(39.98786505423575 -74.51262650431462) bank70624 +70625 POINT(40.60977901909025 -74.4073474017458) bank70625 +70626 POINT(40.10001510158057 -73.09555564450402) bank70626 +70627 POINT(40.77297860871091 -74.97163217152045) bank70627 +70628 POINT(41.53544718270616 -74.08314030133567) bank70628 +70629 POINT(41.574418471506974 -74.98248673708646) bank70629 +70630 POINT(41.264324877207315 -74.73334154106146) bank70630 +70631 POINT(41.260580377241986 -73.48101743567575) bank70631 +70632 POINT(41.07913671798034 -74.96541917483778) bank70632 +70633 POINT(41.05581336563058 -73.31101449523631) bank70633 +70634 POINT(40.91463908535503 -74.98850417199844) bank70634 +70635 POINT(41.078636691031896 -73.86175864101796) bank70635 +70636 POINT(40.00835756312096 -73.58961359185005) bank70636 +70637 POINT(41.46977356374672 -74.67345283348271) bank70637 +70638 POINT(39.81536060625566 -73.29511184392528) bank70638 +70639 POINT(39.7512495669347 -73.09743915113715) bank70639 +70640 POINT(41.030226642197206 -73.11667405658088) bank70640 +70641 POINT(40.05479608900969 -74.02888810588915) bank70641 +70642 POINT(40.338340460577065 -74.79854882378328) bank70642 +70643 POINT(40.171385057867425 -74.54260890596913) bank70643 +70644 POINT(40.88820736233425 -74.81017144240587) bank70644 +70645 POINT(40.14465685297379 -74.66937684046721) bank70645 +70646 POINT(39.73033274640803 -74.9404624728937) bank70646 +70647 POINT(40.34344188005606 -74.75563345383733) bank70647 +70648 POINT(41.641189321699656 -74.0137314052251) bank70648 +70649 POINT(40.37336958821213 -73.73907903013247) bank70649 +70650 POINT(40.88808007551979 -74.10118443112721) bank70650 +70651 POINT(41.01192131624118 -74.97095740366991) bank70651 +70652 POINT(40.50923376797416 -73.50988837357939) bank70652 +70653 POINT(41.330342160286285 -74.19455375266199) bank70653 +70654 POINT(41.59006821705128 -74.2118964286142) bank70654 +70655 POINT(41.497624420064334 -73.20274255334262) bank70655 +70656 POINT(40.5882930188592 -74.94977818726171) bank70656 +70657 POINT(40.166151863790766 -74.69204418258452) bank70657 +70658 POINT(40.27881090398378 -73.36187819178127) bank70658 +70659 POINT(40.68237249672957 -73.67130748769576) bank70659 +70660 POINT(39.986037047363375 -74.71596071380654) bank70660 +70661 POINT(40.71963635633887 -74.03756488718092) bank70661 +70662 POINT(40.160641416781026 -73.80755323447539) bank70662 +70663 POINT(41.54040223041268 -73.05746231647464) bank70663 +70664 POINT(41.18895982521232 -73.00895130301085) bank70664 +70665 POINT(40.35465669898054 -74.85877049527875) bank70665 +70666 POINT(40.257809387323746 -73.98311290173449) bank70666 +70667 POINT(40.23034299183505 -73.44713440920673) bank70667 +70668 POINT(40.84312890739932 -74.42277379801988) bank70668 +70669 POINT(39.71750815112813 -73.90435646582378) bank70669 +70670 POINT(41.34909444632944 -73.53927344725767) bank70670 +70671 POINT(39.95829174899133 -74.90995031315421) bank70671 +70672 POINT(39.77782946331596 -74.8240340933905) bank70672 +70673 POINT(39.99049014089679 -73.45223310381428) bank70673 +70674 POINT(41.0343037541613 -73.30447384125578) bank70674 +70675 POINT(40.12401566212766 -74.01261637283454) bank70675 +70676 POINT(39.87258482063415 -73.4786030121488) bank70676 +70677 POINT(41.08691389886449 -74.15735517876541) bank70677 +70678 POINT(41.01170741183341 -74.7474411201477) bank70678 +70679 POINT(40.9773580302748 -73.10676491495262) bank70679 +70680 POINT(41.1963835688422 -74.77912472978214) bank70680 +70681 POINT(40.819743506631056 -74.14735827210671) bank70681 +70682 POINT(41.41342301683075 -74.37094649436933) bank70682 +70683 POINT(40.817556124518504 -74.19483544611015) bank70683 +70684 POINT(40.982892754410535 -74.7236704562314) bank70684 +70685 POINT(40.605979904930486 -74.85139036681055) bank70685 +70686 POINT(41.260322462941936 -73.97593894264276) bank70686 +70687 POINT(40.36244026121209 -74.68680343541719) bank70687 +70688 POINT(39.75793873564268 -74.99562774724477) bank70688 +70689 POINT(40.020791530377195 -74.87017367429371) bank70689 +70690 POINT(39.836821746841245 -74.66835542696573) bank70690 +70691 POINT(40.152842325208226 -74.39326140698131) bank70691 +70692 POINT(40.515194809803 -73.11902712958721) bank70692 +70693 POINT(40.14470139634717 -74.68146705020263) bank70693 +70694 POINT(41.24165080150772 -73.9939945067469) bank70694 +70695 POINT(41.66367806036713 -73.98604560104958) bank70695 +70696 POINT(39.85082996199483 -74.19858495804179) bank70696 +70697 POINT(39.810602453878595 -74.71304235794585) bank70697 +70698 POINT(40.83840731642202 -73.01451836861321) bank70698 +70699 POINT(41.634681545061525 -73.72250866073388) bank70699 +70700 POINT(41.37305717484924 -74.40930413799217) bank70700 +70701 POINT(40.832080915912385 -74.4099509671916) bank70701 +70702 POINT(41.017751049617836 -74.38505274355278) bank70702 +70703 POINT(40.54594086616078 -74.1595069838369) bank70703 +70704 POINT(40.340613966669956 -73.2867620357787) bank70704 +70705 POINT(41.182640716310075 -74.27579702096654) bank70705 +70706 POINT(40.1095081211951 -74.37385172818469) bank70706 +70707 POINT(40.07541158436015 -74.75164091662344) bank70707 +70708 POINT(40.50464739469443 -74.14160838017668) bank70708 +70709 POINT(40.95861225489421 -73.01667279554023) bank70709 +70710 POINT(41.04791229054387 -74.75734836400136) bank70710 +70711 POINT(40.81989927355344 -74.0846849206011) bank70711 +70712 POINT(40.35751698153412 -74.93898312378568) bank70712 +70713 POINT(41.077698181295254 -73.82062255130322) bank70713 +70714 POINT(39.804089342429634 -74.0617746990242) bank70714 +70715 POINT(40.25109200884454 -74.27123988320182) bank70715 +70716 POINT(40.324657057117946 -73.20928767590837) bank70716 +70717 POINT(40.05546197029339 -73.0405487661606) bank70717 +70718 POINT(40.272457368266174 -73.73866816648777) bank70718 +70719 POINT(39.8764999375109 -73.35689456694274) bank70719 +70720 POINT(41.7078310253937 -73.33496008487312) bank70720 +70721 POINT(40.740848201935584 -73.73825738529402) bank70721 +70722 POINT(39.845233306768826 -74.45807601052896) bank70722 +70723 POINT(41.27877458244312 -74.387218489044) bank70723 +70724 POINT(39.81362333661795 -74.56270623856707) bank70724 +70725 POINT(41.03394437131021 -74.66091852975205) bank70725 +70726 POINT(41.50930944454542 -73.02753689945881) bank70726 +70727 POINT(40.799475027826944 -74.01947613909452) bank70727 +70728 POINT(40.081537707814974 -74.35280665188007) bank70728 +70729 POINT(41.380586388451476 -73.20620293799813) bank70729 +70730 POINT(40.562271181739824 -74.31123025494044) bank70730 +70731 POINT(41.34766333446646 -73.24414228792854) bank70731 +70732 POINT(39.810552902375264 -74.7968696024541) bank70732 +70733 POINT(40.30866986394966 -73.71319254590115) bank70733 +70734 POINT(39.88693142763061 -74.74521690397721) bank70734 +70735 POINT(41.51556217627405 -74.50263538729395) bank70735 +70736 POINT(40.18301377477076 -74.80679960228822) bank70736 +70737 POINT(41.080624896473005 -74.37224221008546) bank70737 +70738 POINT(40.7018233844685 -74.01572922252913) bank70738 +70739 POINT(40.95688186355279 -74.82253410523215) bank70739 +70740 POINT(40.03246758576181 -73.32465952780657) bank70740 +70741 POINT(40.495178261729826 -74.18374151448394) bank70741 +70742 POINT(41.7122488838394 -74.7886618847391) bank70742 +70743 POINT(41.416906833611385 -74.98386679541038) bank70743 +70744 POINT(41.438257304801134 -73.67948244044682) bank70744 +70745 POINT(40.758665920233184 -73.92691005765572) bank70745 +70746 POINT(40.13659356217847 -73.13190037658109) bank70746 +70747 POINT(41.4844967842435 -74.52572550410956) bank70747 +70748 POINT(40.44398872097372 -74.25320700730921) bank70748 +70749 POINT(41.436000956089785 -73.49915955462856) bank70749 +70750 POINT(39.92373648307149 -73.81320044296255) bank70750 +70751 POINT(40.98518843973965 -74.61478832956513) bank70751 +70752 POINT(41.09829965815352 -74.83346304856875) bank70752 +70753 POINT(41.36386203240946 -74.13024468805862) bank70753 +70754 POINT(41.09304716535967 -74.46537260569895) bank70754 +70755 POINT(39.743658518760185 -74.59535521052041) bank70755 +70756 POINT(41.704672754702784 -73.90446725273746) bank70756 +70757 POINT(40.79542930885492 -74.3257365829321) bank70757 +70758 POINT(41.08631873159943 -74.43649925198095) bank70758 +70759 POINT(39.870132246143235 -73.11200928082928) bank70759 +70760 POINT(40.60178059273163 -73.12656000898308) bank70760 +70761 POINT(40.66899471437935 -73.12321083182435) bank70761 +70762 POINT(40.99403536260967 -74.4156488480787) bank70762 +70763 POINT(40.046591735165926 -74.61981059103604) bank70763 +70764 POINT(41.10371388277979 -73.39275459288676) bank70764 +70765 POINT(40.45991990634063 -74.87998218370157) bank70765 +70766 POINT(41.482583273203055 -74.94175170214221) bank70766 +70767 POINT(40.36655844383189 -74.84642239391476) bank70767 +70768 POINT(40.97509024882342 -73.17199125971739) bank70768 +70769 POINT(40.33952728723201 -74.01051979935457) bank70769 +70770 POINT(39.91602960850616 -73.0882955222612) bank70770 +70771 POINT(40.413712623097695 -73.90295823974652) bank70771 +70772 POINT(40.86957297076893 -74.01814187600179) bank70772 +70773 POINT(40.015274796360394 -74.291426497517) bank70773 +70774 POINT(41.08177425736383 -73.84808543625178) bank70774 +70775 POINT(41.52242620533216 -74.44495918199678) bank70775 +70776 POINT(40.65632247007397 -73.67433402965666) bank70776 +70777 POINT(40.10118175538878 -74.56649017045551) bank70777 +70778 POINT(41.464830830434174 -73.79675436665644) bank70778 +70779 POINT(41.59049624271006 -73.04206403703941) bank70779 +70780 POINT(40.53758520089768 -74.55601150577132) bank70780 +70781 POINT(41.27488552339452 -73.72084107339435) bank70781 +70782 POINT(40.05748393061491 -73.95905750015484) bank70782 +70783 POINT(40.58521957339712 -73.36081971892378) bank70783 +70784 POINT(40.25607583926925 -73.90927322122178) bank70784 +70785 POINT(41.451810749156316 -73.88275351859166) bank70785 +70786 POINT(41.68856982204352 -73.20334884952689) bank70786 +70787 POINT(41.09862005320392 -74.20392517724208) bank70787 +70788 POINT(40.85760090797949 -74.41578999959174) bank70788 +70789 POINT(40.5617503844049 -73.49601750914687) bank70789 +70790 POINT(40.0664830183226 -74.19197958917425) bank70790 +70791 POINT(40.788715565379945 -73.6503068230599) bank70791 +70792 POINT(40.95591919638715 -74.86824812460057) bank70792 +70793 POINT(41.23850383645606 -74.82707832140038) bank70793 +70794 POINT(40.28756629795793 -74.68730279528238) bank70794 +70795 POINT(39.935011779502325 -74.11588825463207) bank70795 +70796 POINT(41.056941331413675 -73.3956393615417) bank70796 +70797 POINT(41.03596093424802 -74.95290901711063) bank70797 +70798 POINT(40.064186222480245 -73.24213120399098) bank70798 +70799 POINT(39.71811270106387 -74.92096986910437) bank70799 +70800 POINT(41.27181253979791 -73.46652012086187) bank70800 +70801 POINT(40.713445085487564 -74.84759328420311) bank70801 +70802 POINT(41.405315242113254 -74.4538081760512) bank70802 +70803 POINT(40.86557704393446 -74.88757725469065) bank70803 +70804 POINT(40.76096887165256 -73.99652751839044) bank70804 +70805 POINT(40.60684205066583 -74.74520828478961) bank70805 +70806 POINT(40.415848240876606 -74.11157664863653) bank70806 +70807 POINT(39.97845011829073 -74.38063994742186) bank70807 +70808 POINT(40.1852172218809 -73.88938748703158) bank70808 +70809 POINT(40.7263658145356 -74.53534558276857) bank70809 +70810 POINT(40.85236598088628 -73.54776972813293) bank70810 +70811 POINT(41.57255323497268 -73.93628256653442) bank70811 +70812 POINT(40.30917293682781 -73.22039803715943) bank70812 +70813 POINT(40.30881002326666 -74.04118080267341) bank70813 +70814 POINT(40.114476228354754 -73.83202771569484) bank70814 +70815 POINT(40.83387095594865 -74.66345316558053) bank70815 +70816 POINT(40.625768282242156 -74.88180495161652) bank70816 +70817 POINT(39.84032939222912 -74.99947231048635) bank70817 +70818 POINT(41.5914331881821 -74.52970465524727) bank70818 +70819 POINT(39.87192640182038 -73.76010204929918) bank70819 +70820 POINT(41.68519299115461 -73.97681323564943) bank70820 +70821 POINT(40.113357997840936 -74.47585354915012) bank70821 +70822 POINT(41.66807748916073 -73.10371941345471) bank70822 +70823 POINT(40.6444593227976 -74.23293258629944) bank70823 +70824 POINT(39.91941883670099 -73.44789783069817) bank70824 +70825 POINT(41.5692486614645 -73.18519880863967) bank70825 +70826 POINT(40.54846134026729 -74.07026883021322) bank70826 +70827 POINT(40.573562384269735 -73.40749335520259) bank70827 +70828 POINT(39.798477034656166 -73.26345514001413) bank70828 +70829 POINT(40.07452421707463 -74.58498396004705) bank70829 +70830 POINT(40.68700977264809 -73.87554550833654) bank70830 +70831 POINT(40.0641719566956 -74.79207484391374) bank70831 +70832 POINT(41.477806701239125 -74.74639971399101) bank70832 +70833 POINT(41.689081403740204 -74.83291158102284) bank70833 +70834 POINT(41.36136624747486 -73.61899038788282) bank70834 +70835 POINT(40.18649971364213 -74.01195284608148) bank70835 +70836 POINT(41.55853207585132 -73.50874504345353) bank70836 +70837 POINT(40.05533843211021 -74.3691655338365) bank70837 +70838 POINT(41.43010141981093 -74.37198420625694) bank70838 +70839 POINT(41.332052886309384 -73.2563572848619) bank70839 +70840 POINT(40.24555117935735 -74.83764120534026) bank70840 +70841 POINT(40.104103158613476 -73.57238258553024) bank70841 +70842 POINT(41.012949410820255 -74.41665616117879) bank70842 +70843 POINT(41.13550839388744 -74.09713083511711) bank70843 +70844 POINT(40.69193608730721 -74.25449279630857) bank70844 +70845 POINT(39.92772735597627 -73.02189679181613) bank70845 +70846 POINT(40.92292057975908 -74.69963127452061) bank70846 +70847 POINT(41.61267709174602 -73.32811251048598) bank70847 +70848 POINT(40.139849600107894 -73.95381084905661) bank70848 +70849 POINT(41.30248307551261 -73.83463023605687) bank70849 +70850 POINT(39.892562785887286 -73.67291967422422) bank70850 +70851 POINT(41.017059372331836 -74.58886309391576) bank70851 +70852 POINT(40.30328781162363 -74.14469281585035) bank70852 +70853 POINT(40.88469509128254 -73.2077902488147) bank70853 +70854 POINT(39.958558282781084 -74.73594915425504) bank70854 +70855 POINT(40.78438516171895 -73.89837886275514) bank70855 +70856 POINT(39.80791946041632 -74.57635816878451) bank70856 +70857 POINT(40.60168245873747 -73.19625811227421) bank70857 +70858 POINT(41.59412830048698 -74.66763521875349) bank70858 +70859 POINT(41.51345586580843 -74.22833184884236) bank70859 +70860 POINT(41.374621678706106 -74.51892149642732) bank70860 +70861 POINT(41.60849210912702 -73.76501035795035) bank70861 +70862 POINT(41.018377178693875 -74.65899028708336) bank70862 +70863 POINT(41.64708473737668 -73.24700961659939) bank70863 +70864 POINT(40.81373599222992 -74.1478208716255) bank70864 +70865 POINT(39.778584370633475 -73.87753705160145) bank70865 +70866 POINT(40.236117015402115 -73.01185945961167) bank70866 +70867 POINT(41.50374222803458 -74.8997841294599) bank70867 +70868 POINT(40.29268755750859 -74.4048269118439) bank70868 +70869 POINT(40.92004019677644 -74.07475691462035) bank70869 +70870 POINT(40.419908426576605 -74.3224221180853) bank70870 +70871 POINT(39.96998316583987 -74.0315623833948) bank70871 +70872 POINT(40.79421170877591 -73.15639246373706) bank70872 +70873 POINT(40.27704922847096 -74.09902963429715) bank70873 +70874 POINT(41.38417020196163 -73.29431796127622) bank70874 +70875 POINT(40.08968894946301 -74.05407838957622) bank70875 +70876 POINT(40.15655031557108 -73.3742617842522) bank70876 +70877 POINT(40.00435815762479 -74.3677658606061) bank70877 +70878 POINT(40.33890866941534 -74.08124039543102) bank70878 +70879 POINT(40.07030715566837 -74.69155436992992) bank70879 +70880 POINT(40.17920336719852 -73.26247986510153) bank70880 +70881 POINT(41.40757428283724 -73.73060767213029) bank70881 +70882 POINT(40.50025347721479 -73.71645821629065) bank70882 +70883 POINT(40.88827562808069 -73.29935436090119) bank70883 +70884 POINT(41.3216873304202 -73.77940934186468) bank70884 +70885 POINT(40.258535684687786 -74.79989594887262) bank70885 +70886 POINT(41.16936561994459 -74.22136669922918) bank70886 +70887 POINT(40.367860858653216 -74.48119115901244) bank70887 +70888 POINT(41.680601241494806 -74.96543088829964) bank70888 +70889 POINT(41.37785127902216 -74.08617027333801) bank70889 +70890 POINT(41.381814835105345 -73.96369495453361) bank70890 +70891 POINT(40.421986518477425 -74.84852427479895) bank70891 +70892 POINT(40.42270775611711 -73.6333380526602) bank70892 +70893 POINT(41.30795121893086 -74.9692461274474) bank70893 +70894 POINT(41.16258622079536 -73.01863264197577) bank70894 +70895 POINT(41.365342340659794 -73.2208976011895) bank70895 +70896 POINT(40.854985062960615 -74.08152069206078) bank70896 +70897 POINT(41.46541888262891 -73.69931526282885) bank70897 +70898 POINT(40.429148926605656 -74.30775344444893) bank70898 +70899 POINT(40.3960332868234 -73.65170043737018) bank70899 +70900 POINT(41.33891249246476 -73.68181890095337) bank70900 +70901 POINT(40.61643452700292 -74.38990408284263) bank70901 +70902 POINT(40.3865455116322 -74.46498910977208) bank70902 +70903 POINT(41.6338417117218 -74.50243094794446) bank70903 +70904 POINT(41.156370895498945 -74.60874180701474) bank70904 +70905 POINT(40.87312484088685 -73.94606954222249) bank70905 +70906 POINT(41.07018913118055 -74.58481679849632) bank70906 +70907 POINT(40.613415275141605 -73.55515723434954) bank70907 +70908 POINT(39.908615315793156 -74.05378115643127) bank70908 +70909 POINT(41.414864491873246 -73.37911654415966) bank70909 +70910 POINT(40.309778119146515 -73.91762753619567) bank70910 +70911 POINT(41.27672112591324 -73.171713807807) bank70911 +70912 POINT(41.289598647594275 -74.25251935604398) bank70912 +70913 POINT(40.36018230274682 -74.97473840838957) bank70913 +70914 POINT(40.421231081494206 -73.35572229843433) bank70914 +70915 POINT(40.480999302181026 -73.6852162426915) bank70915 +70916 POINT(41.19323091019499 -74.08356563996915) bank70916 +70917 POINT(41.4444384859273 -74.66335036551804) bank70917 +70918 POINT(41.340591007680146 -73.65861398732427) bank70918 +70919 POINT(40.30473085269882 -74.81568300292655) bank70919 +70920 POINT(41.667125339705535 -73.95494296923513) bank70920 +70921 POINT(40.09503562063959 -74.07073814752722) bank70921 +70922 POINT(41.29484460506716 -73.45459772268347) bank70922 +70923 POINT(40.38401563829332 -74.28380688515779) bank70923 +70924 POINT(40.509594002016854 -74.01155085140668) bank70924 +70925 POINT(40.61124249736465 -73.65247602180412) bank70925 +70926 POINT(40.097672184038224 -73.91378730988797) bank70926 +70927 POINT(40.398303542905246 -74.28525143511575) bank70927 +70928 POINT(40.310112729850246 -74.7282277447726) bank70928 +70929 POINT(41.15834709119701 -74.11976363218542) bank70929 +70930 POINT(39.97410928876074 -73.42001431901049) bank70930 +70931 POINT(40.4287509629911 -73.50484397482144) bank70931 +70932 POINT(40.10319823601574 -74.2459383823786) bank70932 +70933 POINT(40.92252462781258 -74.47151780720299) bank70933 +70934 POINT(39.81910863341645 -73.71339953710864) bank70934 +70935 POINT(39.98129552615009 -74.67150743749875) bank70935 +70936 POINT(39.74021058038141 -73.32246131006711) bank70936 +70937 POINT(41.153439329491704 -74.19743216496735) bank70937 +70938 POINT(41.64789313722726 -73.99539101483414) bank70938 +70939 POINT(41.3742694363079 -74.54779790836004) bank70939 +70940 POINT(40.86694113898792 -73.06903550434406) bank70940 +70941 POINT(41.015880231804914 -74.86439395480983) bank70941 +70942 POINT(40.08088933362771 -74.78106029884158) bank70942 +70943 POINT(40.31008129339732 -73.80383260340435) bank70943 +70944 POINT(41.208904451859524 -73.89030638545404) bank70944 +70945 POINT(40.49780942035305 -74.01006823088639) bank70945 +70946 POINT(40.63643937696842 -74.33040076157768) bank70946 +70947 POINT(40.21673387366019 -73.83159843475563) bank70947 +70948 POINT(39.96107207680753 -74.57795169818736) bank70948 +70949 POINT(41.66206364947629 -74.27618272716111) bank70949 +70950 POINT(40.924989808588045 -73.42830693518198) bank70950 +70951 POINT(41.52860704442839 -73.31219785816533) bank70951 +70952 POINT(41.67031302810334 -73.92732462256872) bank70952 +70953 POINT(41.22246694885545 -74.28881334616187) bank70953 +70954 POINT(41.6349053146212 -74.83114836070698) bank70954 +70955 POINT(41.25664867507567 -74.98715412642669) bank70955 +70956 POINT(41.077226392969926 -74.43796476245697) bank70956 +70957 POINT(40.52581350341972 -73.65284041435036) bank70957 +70958 POINT(41.39205127673343 -73.12151882794069) bank70958 +70959 POINT(40.441320671221085 -73.47081353675189) bank70959 +70960 POINT(40.345047779712964 -74.7758623824133) bank70960 +70961 POINT(40.26243474472536 -74.41833387934794) bank70961 +70962 POINT(41.319719705722925 -74.66213602591618) bank70962 +70963 POINT(41.259734820581876 -73.94393868970484) bank70963 +70964 POINT(40.46788578922222 -73.11260352449258) bank70964 +70965 POINT(41.547861515868675 -74.41505749628094) bank70965 +70966 POINT(40.850409329518236 -73.12266976642012) bank70966 +70967 POINT(41.70612919616776 -73.44656934704622) bank70967 +70968 POINT(39.84161777939154 -74.00497783522033) bank70968 +70969 POINT(40.616428314360796 -73.33028034808126) bank70969 +70970 POINT(40.92872871844404 -73.42548245536625) bank70970 +70971 POINT(40.72604702321183 -73.5861968794291) bank70971 +70972 POINT(41.12612636094498 -74.55929735951192) bank70972 +70973 POINT(39.78062832011647 -74.4627154509827) bank70973 +70974 POINT(40.004226522095244 -74.18548660446736) bank70974 +70975 POINT(39.7912743047534 -74.64873220810723) bank70975 +70976 POINT(39.85156370496274 -74.39063270362037) bank70976 +70977 POINT(39.9738091887502 -73.54578532601815) bank70977 +70978 POINT(40.17239942165095 -74.15617315737705) bank70978 +70979 POINT(41.27299765581958 -73.7388029754498) bank70979 +70980 POINT(39.885690524599454 -73.72127599085407) bank70980 +70981 POINT(41.64672643522122 -74.3042840173688) bank70981 +70982 POINT(41.069439992690015 -74.83164115284285) bank70982 +70983 POINT(40.51148780626605 -73.39269272236132) bank70983 +70984 POINT(40.21563827920192 -73.40718129602377) bank70984 +70985 POINT(41.35485502325545 -74.43081117319448) bank70985 +70986 POINT(40.30418223608785 -73.23549655564487) bank70986 +70987 POINT(40.33680892544189 -74.01674384347139) bank70987 +70988 POINT(41.04954944554668 -73.56079733792797) bank70988 +70989 POINT(41.11993226663703 -73.22055867686736) bank70989 +70990 POINT(40.54447226039055 -74.41368289341435) bank70990 +70991 POINT(40.80260095407462 -73.25250172408046) bank70991 +70992 POINT(40.99204943553833 -74.5643329217675) bank70992 +70993 POINT(40.93472707739081 -74.4930442359505) bank70993 +70994 POINT(41.077399568769984 -73.82056871009227) bank70994 +70995 POINT(41.53845592207202 -74.85511881775288) bank70995 +70996 POINT(40.95539567691453 -73.5307230798288) bank70996 +70997 POINT(41.28486726637621 -74.11010455597044) bank70997 +70998 POINT(41.54990908069519 -74.40730837602739) bank70998 +70999 POINT(41.664588110706276 -73.9518860572692) bank70999 +71000 POINT(39.86442314318621 -74.12179067576751) bank71000 +71001 POINT(41.69605779250699 -74.1913641303499) bank71001 +71002 POINT(41.13599829091543 -73.52905837604047) bank71002 +71003 POINT(41.1206776857748 -74.83848029826814) bank71003 +71004 POINT(40.04316802064075 -74.34064236119565) bank71004 +71005 POINT(41.06652491670459 -73.173185285835) bank71005 +71006 POINT(40.84678558228058 -73.08898513165315) bank71006 +71007 POINT(41.55093609023249 -73.65772150408257) bank71007 +71008 POINT(41.63023662189232 -73.04725352340836) bank71008 +71009 POINT(39.79345686604289 -74.43140410720665) bank71009 +71010 POINT(40.54675842537614 -73.43787402895384) bank71010 +71011 POINT(40.548099304430025 -73.70484634316132) bank71011 +71012 POINT(41.207985138888155 -74.70059275861554) bank71012 +71013 POINT(41.43028816810909 -74.06653927783117) bank71013 +71014 POINT(41.69687780422257 -74.34204319860069) bank71014 +71015 POINT(40.494792972052274 -73.48264890116154) bank71015 +71016 POINT(39.76519520326929 -73.02508098478769) bank71016 +71017 POINT(40.00080676059912 -73.61431553191176) bank71017 +71018 POINT(41.17688654820527 -73.67097661534262) bank71018 +71019 POINT(40.516297419131845 -74.50226771249088) bank71019 +71020 POINT(40.91139978656465 -74.22232603707353) bank71020 +71021 POINT(40.041293652135224 -73.14560537465775) bank71021 +71022 POINT(41.511407886386216 -74.04030716271883) bank71022 +71023 POINT(40.92064821846431 -74.96734804652665) bank71023 +71024 POINT(40.20893905383085 -74.46467073219696) bank71024 +71025 POINT(40.06733392976679 -73.01905182117042) bank71025 +71026 POINT(40.52083802508308 -73.90533392974919) bank71026 +71027 POINT(40.02724157438834 -73.3391307567153) bank71027 +71028 POINT(40.42656374946832 -73.430997918903) bank71028 +71029 POINT(41.207667123542016 -74.38061615901883) bank71029 +71030 POINT(40.18917730571252 -73.00793763040515) bank71030 +71031 POINT(41.124412714219865 -73.60610557588096) bank71031 +71032 POINT(40.888042446139345 -74.70959180784341) bank71032 +71033 POINT(41.13950864010356 -73.97182230333924) bank71033 +71034 POINT(41.391742205770655 -73.37482461857029) bank71034 +71035 POINT(41.66505047777024 -74.75224999059445) bank71035 +71036 POINT(40.11345907825734 -74.81234502584753) bank71036 +71037 POINT(39.76622648925813 -74.77886923229349) bank71037 +71038 POINT(41.48851573562631 -74.61522330462132) bank71038 +71039 POINT(41.430428956633804 -74.66888400343196) bank71039 +71040 POINT(40.182943165403636 -73.94942388243585) bank71040 +71041 POINT(40.68610820600403 -74.97792762509701) bank71041 +71042 POINT(39.909762907695175 -74.81816379542772) bank71042 +71043 POINT(40.62805616055217 -73.11573668147011) bank71043 +71044 POINT(39.9533277831886 -74.05748825114271) bank71044 +71045 POINT(41.54326891446083 -74.42287307813254) bank71045 +71046 POINT(40.07315840878295 -73.06068356915142) bank71046 +71047 POINT(40.73530209573426 -73.58342912975257) bank71047 +71048 POINT(40.061785441345 -74.59357114813331) bank71048 +71049 POINT(40.58160823307621 -73.56257941547914) bank71049 +71050 POINT(41.69605835701219 -74.63023952392301) bank71050 +71051 POINT(40.63070130886384 -73.92694078101596) bank71051 +71052 POINT(40.32529824702937 -74.72299983242539) bank71052 +71053 POINT(39.8849153471012 -74.16651007661032) bank71053 +71054 POINT(41.60652613056199 -74.81095015679362) bank71054 +71055 POINT(40.775858310924306 -74.8447707144662) bank71055 +71056 POINT(41.389740703278974 -74.69435364869275) bank71056 +71057 POINT(40.565463619278695 -73.2383236603969) bank71057 +71058 POINT(39.959838546628895 -73.51996550548951) bank71058 +71059 POINT(40.451280906704895 -74.38370845923063) bank71059 +71060 POINT(40.2129127973366 -74.33573941497562) bank71060 +71061 POINT(40.69448126221224 -74.9229431800457) bank71061 +71062 POINT(40.03968774187919 -74.91422114200046) bank71062 +71063 POINT(40.30036813429716 -73.25719719710152) bank71063 +71064 POINT(41.54751369443678 -74.46805247536057) bank71064 +71065 POINT(40.31687342908283 -74.10520893031648) bank71065 +71066 POINT(39.90710959760645 -74.1893597634825) bank71066 +71067 POINT(41.546781455357134 -74.03003199049715) bank71067 +71068 POINT(40.2017308852022 -73.21051510659869) bank71068 +71069 POINT(41.353760998979716 -73.79335904590995) bank71069 +71070 POINT(40.85088796559715 -73.89176105565303) bank71070 +71071 POINT(40.52542904047515 -73.88805809572636) bank71071 +71072 POINT(40.34874342645058 -74.04529586078527) bank71072 +71073 POINT(41.68492981163292 -73.09398408332999) bank71073 +71074 POINT(40.717719820353686 -74.98236191980281) bank71074 +71075 POINT(40.19132417513532 -73.49031182939791) bank71075 +71076 POINT(41.10880898088447 -73.8330002276004) bank71076 +71077 POINT(40.18688568551237 -73.97215585974723) bank71077 +71078 POINT(39.83865245178372 -74.60691519814876) bank71078 +71079 POINT(40.34446243091902 -75.00402771088315) bank71079 +71080 POINT(40.57659892653352 -73.04787706618994) bank71080 +71081 POINT(40.06899195611682 -74.78291488269905) bank71081 +71082 POINT(40.77743436717779 -74.39674893078333) bank71082 +71083 POINT(41.335051878176955 -74.26146417736486) bank71083 +71084 POINT(40.33073718942883 -73.9843773391341) bank71084 +71085 POINT(40.138397204865896 -74.84727387647351) bank71085 +71086 POINT(39.798857861861876 -73.79833341507248) bank71086 +71087 POINT(40.938345438394656 -74.55459292419279) bank71087 +71088 POINT(40.44033282722293 -73.88715386350384) bank71088 +71089 POINT(41.012276413179386 -73.6109411339919) bank71089 +71090 POINT(40.33322221180505 -73.09214763548707) bank71090 +71091 POINT(41.527917185106205 -74.02127952065902) bank71091 +71092 POINT(40.974984744582216 -74.79185425303086) bank71092 +71093 POINT(41.04368173551177 -73.96605796663938) bank71093 +71094 POINT(41.44034922770414 -74.52620775304192) bank71094 +71095 POINT(40.814093986288405 -73.94740663871202) bank71095 +71096 POINT(40.031628034012826 -74.99466672399392) bank71096 +71097 POINT(40.66512451519029 -73.79827509660416) bank71097 +71098 POINT(41.240542792543934 -73.65616252810443) bank71098 +71099 POINT(41.618742788078364 -73.45671552386906) bank71099 +71100 POINT(39.780469426175095 -73.09992926278984) bank71100 +71101 POINT(41.364618439777004 -73.20059489904708) bank71101 +71102 POINT(40.335968853865126 -73.10577704058178) bank71102 +71103 POINT(40.406370554356315 -73.2945704017599) bank71103 +71104 POINT(40.12110077142864 -74.13208375920608) bank71104 +71105 POINT(40.89524381175052 -73.56427780917366) bank71105 +71106 POINT(40.31129139965855 -74.73087985231355) bank71106 +71107 POINT(39.812700830188156 -73.61426050359795) bank71107 +71108 POINT(41.444554519947964 -74.68226569061939) bank71108 +71109 POINT(41.31467820148546 -74.87670419377332) bank71109 +71110 POINT(40.25055632391179 -74.78532988765883) bank71110 +71111 POINT(40.16869117997732 -73.60216589701207) bank71111 +71112 POINT(40.638399376889446 -73.36993079530262) bank71112 +71113 POINT(39.7771362724916 -73.31015230295746) bank71113 +71114 POINT(39.96797600647373 -74.15178672923578) bank71114 +71115 POINT(41.04982973605791 -73.58011689098646) bank71115 +71116 POINT(40.318637367375395 -73.55419698865664) bank71116 +71117 POINT(40.325636409868046 -74.73499630901465) bank71117 +71118 POINT(40.93881494509037 -74.39569390724955) bank71118 +71119 POINT(41.04546684579397 -73.90627623097109) bank71119 +71120 POINT(40.21696632901949 -73.07836639496679) bank71120 +71121 POINT(40.726989081232915 -74.16657805189327) bank71121 +71122 POINT(41.60519443915037 -73.76266604058682) bank71122 +71123 POINT(40.15431696668349 -74.25812627739965) bank71123 +71124 POINT(41.64300459289636 -73.4953347187401) bank71124 +71125 POINT(40.4511916540411 -74.91155341709008) bank71125 +71126 POINT(40.821376723621626 -73.45494500564462) bank71126 +71127 POINT(41.500397521304315 -74.04870395692349) bank71127 +71128 POINT(39.92412330921752 -73.24143273575764) bank71128 +71129 POINT(41.10621761199429 -74.77106375049955) bank71129 +71130 POINT(40.34503722326875 -74.64644293453586) bank71130 +71131 POINT(41.371471031878635 -73.95903639203712) bank71131 +71132 POINT(40.348478078226904 -73.91452682612028) bank71132 +71133 POINT(40.25162318565595 -74.83021465544697) bank71133 +71134 POINT(39.7977765125445 -73.28761520384334) bank71134 +71135 POINT(41.21136887937461 -73.28190788747665) bank71135 +71136 POINT(41.09309785448774 -73.71721684290478) bank71136 +71137 POINT(40.56905007487738 -74.5780587949291) bank71137 +71138 POINT(39.74551715586103 -74.33468037315457) bank71138 +71139 POINT(39.77463772833702 -73.44715454882176) bank71139 +71140 POINT(41.048761491239944 -74.17764028171753) bank71140 +71141 POINT(41.3906794718869 -73.59230217744766) bank71141 +71142 POINT(40.36347808296801 -74.86383978604019) bank71142 +71143 POINT(40.02465772316901 -73.78710048554125) bank71143 +71144 POINT(41.07967905417245 -73.44216463130228) bank71144 +71145 POINT(40.03185068577771 -73.34835777403273) bank71145 +71146 POINT(39.840974955482146 -74.94072751717106) bank71146 +71147 POINT(41.134950677143294 -73.86185856506513) bank71147 +71148 POINT(39.9306640419748 -73.93597731310689) bank71148 +71149 POINT(40.93197345275401 -74.45013083917806) bank71149 +71150 POINT(39.879622638226124 -73.61282191066569) bank71150 +71151 POINT(40.46590589551774 -73.06912884631754) bank71151 +71152 POINT(40.20354768335077 -73.87909263322337) bank71152 +71153 POINT(41.56693647176017 -73.31107314969225) bank71153 +71154 POINT(41.20737104232889 -74.27158411776743) bank71154 +71155 POINT(40.66745381256544 -73.16310586970688) bank71155 +71156 POINT(40.8935368615104 -73.09633908105077) bank71156 +71157 POINT(40.67934523416488 -73.4767343437687) bank71157 +71158 POINT(40.25109389003072 -73.01601559670428) bank71158 +71159 POINT(40.45435289010809 -73.45108416316735) bank71159 +71160 POINT(40.84541837063205 -73.28630765372779) bank71160 +71161 POINT(39.76124469893293 -73.18646181759868) bank71161 +71162 POINT(40.44351880352122 -74.98845333690693) bank71162 +71163 POINT(39.96415251523647 -74.8053074621121) bank71163 +71164 POINT(39.800606218653826 -73.88935104840269) bank71164 +71165 POINT(41.63656617127379 -73.95462673673806) bank71165 +71166 POINT(40.927268720286925 -74.0659479747024) bank71166 +71167 POINT(41.606397256574795 -74.40680956689263) bank71167 +71168 POINT(40.35338696341842 -73.42270416609026) bank71168 +71169 POINT(41.674902830290904 -74.59591784163291) bank71169 +71170 POINT(40.25539483389634 -74.07974611934281) bank71170 +71171 POINT(39.98404601503925 -74.59475855234348) bank71171 +71172 POINT(40.20056200980724 -73.94514188879025) bank71172 +71173 POINT(41.22910338139338 -74.81423037218195) bank71173 +71174 POINT(41.645153720089255 -74.0046584139522) bank71174 +71175 POINT(41.352694906569525 -73.5541404770423) bank71175 +71176 POINT(41.12479130348194 -73.46101864793498) bank71176 +71177 POINT(39.96687215935954 -74.95295890642792) bank71177 +71178 POINT(40.50039242683575 -73.33162902839601) bank71178 +71179 POINT(39.98636896643696 -73.25594159847975) bank71179 +71180 POINT(40.9760611737394 -73.82111570432482) bank71180 +71181 POINT(40.41874577725852 -74.63141496263474) bank71181 +71182 POINT(41.581343655092255 -74.87642267280789) bank71182 +71183 POINT(40.14031910038181 -73.61888014350096) bank71183 +71184 POINT(39.79424967922684 -73.54304736748891) bank71184 +71185 POINT(40.414282654279404 -74.98517823541368) bank71185 +71186 POINT(40.91660288376028 -73.96157251735927) bank71186 +71187 POINT(40.60286620860814 -73.40613593216779) bank71187 +71188 POINT(40.732728266765 -73.14316114119822) bank71188 +71189 POINT(41.37789638318676 -74.0602373136956) bank71189 +71190 POINT(40.322328283260745 -73.78962764492846) bank71190 +71191 POINT(40.033832835705894 -74.59365467616736) bank71191 +71192 POINT(41.66927195010823 -74.84726112094164) bank71192 +71193 POINT(41.63844683967296 -74.1875205484746) bank71193 +71194 POINT(41.43019808896369 -74.33778548151281) bank71194 +71195 POINT(41.695242385028614 -73.50436183502818) bank71195 +71196 POINT(41.58790594266758 -74.15808663449072) bank71196 +71197 POINT(41.07155434555501 -73.57048899225595) bank71197 +71198 POINT(41.51614117326001 -74.45048307000287) bank71198 +71199 POINT(39.9544165107077 -73.46952619092612) bank71199 +71200 POINT(41.61519816592993 -73.80910143186298) bank71200 +71201 POINT(40.97890405770187 -73.77387900286497) bank71201 +71202 POINT(40.05839304613928 -73.40895050485723) bank71202 +71203 POINT(40.242971344828526 -73.07081869185534) bank71203 +71204 POINT(40.48038536857419 -73.97238563460229) bank71204 +71205 POINT(40.67572220600799 -73.43929131474978) bank71205 +71206 POINT(40.353743856260834 -74.91914468234347) bank71206 +71207 POINT(41.40568079020948 -74.74239948998724) bank71207 +71208 POINT(39.97760373404892 -73.19752717060737) bank71208 +71209 POINT(40.001902126834175 -73.2134085683426) bank71209 +71210 POINT(39.7765138888649 -74.26567351799511) bank71210 +71211 POINT(41.6786543493306 -73.8310869742379) bank71211 +71212 POINT(41.40306552090962 -73.42042222021328) bank71212 +71213 POINT(40.770598314463115 -73.34697158718107) bank71213 +71214 POINT(40.95602106094892 -73.59291705781683) bank71214 +71215 POINT(40.93273727718461 -74.32910115099239) bank71215 +71216 POINT(39.77355251825174 -73.9020362764912) bank71216 +71217 POINT(40.78594598314945 -73.45267235322112) bank71217 +71218 POINT(40.517484728862634 -74.77957613866518) bank71218 +71219 POINT(41.093075506401576 -73.09732520078965) bank71219 +71220 POINT(40.535657574221766 -74.06617168766353) bank71220 +71221 POINT(40.061397844914495 -74.36251516513845) bank71221 +71222 POINT(41.69614149059392 -73.74540586497929) bank71222 +71223 POINT(41.643999839951924 -74.29068176718434) bank71223 +71224 POINT(41.36315325198186 -74.30598391549492) bank71224 +71225 POINT(40.860210917183814 -74.54090774928966) bank71225 +71226 POINT(40.01100535533669 -74.7131409902133) bank71226 +71227 POINT(39.9918061272596 -74.91101572797795) bank71227 +71228 POINT(40.04886021973235 -73.28509581684759) bank71228 +71229 POINT(41.04392139197714 -74.51667795148728) bank71229 +71230 POINT(41.02214593923612 -73.92417232676651) bank71230 +71231 POINT(41.654285251904 -73.544245427552) bank71231 +71232 POINT(40.88488835037553 -74.58232719311366) bank71232 +71233 POINT(41.5048184940132 -73.12202222799925) bank71233 +71234 POINT(40.548870503461735 -73.13446719988642) bank71234 +71235 POINT(40.19793145277031 -73.40369309128081) bank71235 +71236 POINT(41.05639973083897 -74.1681190046057) bank71236 +71237 POINT(40.05954215736398 -74.88426312967385) bank71237 +71238 POINT(41.683907778962784 -74.56025553801334) bank71238 +71239 POINT(41.60864224030597 -73.84421447647169) bank71239 +71240 POINT(39.753506451976 -73.78303609576052) bank71240 +71241 POINT(40.9905061027583 -73.94621781638833) bank71241 +71242 POINT(41.32633106004035 -73.76643501391041) bank71242 +71243 POINT(40.79348720873674 -74.09786208012886) bank71243 +71244 POINT(39.73150235980287 -73.77861760914553) bank71244 +71245 POINT(40.660612448759466 -74.91800167817247) bank71245 +71246 POINT(41.20524199569006 -74.18781060420811) bank71246 +71247 POINT(41.14239216652296 -73.54661463535946) bank71247 +71248 POINT(40.68336504826034 -74.5514395833967) bank71248 +71249 POINT(41.22298641680466 -73.70915376685528) bank71249 +71250 POINT(40.617061365244616 -75.00064175997882) bank71250 +71251 POINT(41.65445435830799 -73.82461818916983) bank71251 +71252 POINT(41.34331309105394 -73.74168862874409) bank71252 +71253 POINT(41.64249928064755 -74.26892261406806) bank71253 +71254 POINT(40.48582660162049 -73.28239810932757) bank71254 +71255 POINT(41.1314302844737 -73.8994627256135) bank71255 +71256 POINT(40.80478662177305 -73.51367244628901) bank71256 +71257 POINT(40.13314623674479 -74.84820942003745) bank71257 +71258 POINT(39.94934672153409 -74.7478485250352) bank71258 +71259 POINT(41.528038632568 -73.31593124467494) bank71259 +71260 POINT(40.54801349625298 -73.88742861918436) bank71260 +71261 POINT(40.19143480907504 -73.7911448677894) bank71261 +71262 POINT(41.48261131453862 -74.04243467271965) bank71262 +71263 POINT(41.40400518374849 -74.16168963664953) bank71263 +71264 POINT(40.03715832382237 -73.03184727855371) bank71264 +71265 POINT(39.98057396375273 -74.12348114410325) bank71265 +71266 POINT(41.5344139926096 -73.99484816576827) bank71266 +71267 POINT(41.17526827428601 -73.92834822127342) bank71267 +71268 POINT(39.856189869343176 -74.84133007790088) bank71268 +71269 POINT(40.49317403406811 -74.95301541554998) bank71269 +71270 POINT(41.18949656199716 -73.34848382989973) bank71270 +71271 POINT(41.30973983285638 -74.29311387418285) bank71271 +71272 POINT(40.98527839225923 -73.56411097714461) bank71272 +71273 POINT(41.01267579514885 -74.02606106610938) bank71273 +71274 POINT(39.789206128630795 -73.80467015423586) bank71274 +71275 POINT(41.497850900556756 -73.58389107636879) bank71275 +71276 POINT(40.252069452254425 -74.19345295998886) bank71276 +71277 POINT(40.93329629325374 -74.88533587940175) bank71277 +71278 POINT(40.581898468975766 -74.21567652930675) bank71278 +71279 POINT(40.48882454322893 -73.68909314286397) bank71279 +71280 POINT(41.12902553638217 -74.39130626056821) bank71280 +71281 POINT(41.57481817499446 -73.87826722353589) bank71281 +71282 POINT(40.65854952448858 -74.82090828592878) bank71282 +71283 POINT(41.15380039991888 -74.05303212602335) bank71283 +71284 POINT(41.689720792692164 -73.11663461017876) bank71284 +71285 POINT(40.12905548568597 -74.23707260241176) bank71285 +71286 POINT(39.7312782470994 -74.15033782547759) bank71286 +71287 POINT(41.04553184683096 -74.67580253290907) bank71287 +71288 POINT(41.660643434394906 -74.06607109588282) bank71288 +71289 POINT(40.91010595961201 -74.30915939263453) bank71289 +71290 POINT(40.314509426613306 -73.81373245102374) bank71290 +71291 POINT(40.99888889442207 -73.34061174215988) bank71291 +71292 POINT(40.54736588768931 -73.11956327980495) bank71292 +71293 POINT(40.51990396365676 -73.47232906248921) bank71293 +71294 POINT(39.973694565455794 -74.13265072758426) bank71294 +71295 POINT(40.70760519507714 -74.03882576726838) bank71295 +71296 POINT(40.64527095278987 -73.72882511798502) bank71296 +71297 POINT(40.191031487682615 -73.04363379220045) bank71297 +71298 POINT(40.01601790275571 -74.70428495619528) bank71298 +71299 POINT(41.52210456304369 -74.79378915116999) bank71299 +71300 POINT(40.243855775267 -73.44203859711239) bank71300 +71301 POINT(41.00479237762678 -73.90688596639536) bank71301 +71302 POINT(40.79558745932952 -74.8766286578516) bank71302 +71303 POINT(41.446705112752454 -74.47734324785444) bank71303 +71304 POINT(39.86862296608191 -74.77273699891926) bank71304 +71305 POINT(41.28856371878549 -74.34050017704809) bank71305 +71306 POINT(39.953218711861815 -74.22135702754089) bank71306 +71307 POINT(40.49974116512171 -74.54347306510208) bank71307 +71308 POINT(40.44361798730941 -74.70835527344238) bank71308 +71309 POINT(41.28103017630503 -74.01662614996094) bank71309 +71310 POINT(39.92681480535941 -74.2374206452055) bank71310 +71311 POINT(39.9512869010111 -73.39942041682771) bank71311 +71312 POINT(40.7068734431738 -74.89297790920025) bank71312 +71313 POINT(41.587022378179924 -74.3469450649285) bank71313 +71314 POINT(41.69467251168981 -73.3046922591854) bank71314 +71315 POINT(40.778870430120755 -73.55162027379966) bank71315 +71316 POINT(40.30771690408664 -73.52557717245415) bank71316 +71317 POINT(39.77339314335716 -73.6723204280251) bank71317 +71318 POINT(40.58009980382939 -73.66936870055413) bank71318 +71319 POINT(40.98821128230525 -74.45038110586033) bank71319 +71320 POINT(41.09916521071698 -74.54449470263182) bank71320 +71321 POINT(40.5084895469847 -73.2120947060719) bank71321 +71322 POINT(41.40255187286831 -73.0922364155513) bank71322 +71323 POINT(39.73037919605288 -73.1906793964763) bank71323 +71324 POINT(40.00503602536373 -73.98755525130844) bank71324 +71325 POINT(40.02130689788495 -74.0526233100025) bank71325 +71326 POINT(40.29575145276473 -73.22404333854233) bank71326 +71327 POINT(39.91506775153891 -73.40425717146697) bank71327 +71328 POINT(40.2955557083561 -73.0365657945463) bank71328 +71329 POINT(39.81561759977256 -73.65144485232861) bank71329 +71330 POINT(40.59984846905913 -74.00674373571235) bank71330 +71331 POINT(40.35118502603828 -74.53786635432472) bank71331 +71332 POINT(40.70040963401169 -74.68168381512137) bank71332 +71333 POINT(39.94354624074338 -73.35213926794954) bank71333 +71334 POINT(41.50605450078682 -74.67104017872505) bank71334 +71335 POINT(40.105332676909256 -73.29579607185391) bank71335 +71336 POINT(39.9424609173822 -74.47139403570543) bank71336 +71337 POINT(39.90685582626688 -73.62193659969117) bank71337 +71338 POINT(41.23338404059789 -74.00718250689042) bank71338 +71339 POINT(39.89592291934891 -73.77758203295936) bank71339 +71340 POINT(40.94729630131266 -73.59583896498599) bank71340 +71341 POINT(40.585130486763305 -74.80207670033559) bank71341 +71342 POINT(40.84962269347022 -74.41351908305303) bank71342 +71343 POINT(40.146535623246564 -74.33052309863776) bank71343 +71344 POINT(40.23305695758141 -74.31191159396445) bank71344 +71345 POINT(41.226813518893735 -73.15191889736465) bank71345 +71346 POINT(41.139399937019526 -73.24441733150726) bank71346 +71347 POINT(40.86576327806204 -73.1219942696285) bank71347 +71348 POINT(41.65233209857478 -73.39586014997052) bank71348 +71349 POINT(41.680222696220284 -73.85276412996497) bank71349 +71350 POINT(41.316802852661034 -73.07648198032359) bank71350 +71351 POINT(40.74777598396088 -73.26908791683249) bank71351 +71352 POINT(40.00235162193989 -74.21244215011451) bank71352 +71353 POINT(40.45737388754874 -74.60498818265111) bank71353 +71354 POINT(40.91632329786539 -73.05050785890872) bank71354 +71355 POINT(41.65561515971387 -73.11477031934788) bank71355 +71356 POINT(40.317058286633134 -73.21149674982489) bank71356 +71357 POINT(40.45718972507671 -74.27047748539373) bank71357 +71358 POINT(41.54120797814872 -73.06779621279811) bank71358 +71359 POINT(41.68913686717858 -73.04836097268279) bank71359 +71360 POINT(41.37556902968467 -73.42309180944264) bank71360 +71361 POINT(40.98534152005303 -73.24540908244192) bank71361 +71362 POINT(40.5484371286946 -74.40689943304939) bank71362 +71363 POINT(40.805603926780485 -74.31896998661792) bank71363 +71364 POINT(41.09739405530597 -73.16504041341187) bank71364 +71365 POINT(39.88874835442645 -74.02295539421412) bank71365 +71366 POINT(40.047791639411074 -74.2847058501278) bank71366 +71367 POINT(40.87438989256458 -73.36850277168091) bank71367 +71368 POINT(39.92686109777435 -73.07896008509935) bank71368 +71369 POINT(40.637861240493315 -74.58836087866227) bank71369 +71370 POINT(39.93447604127881 -74.57511899985181) bank71370 +71371 POINT(40.79002512100102 -74.51732752285852) bank71371 +71372 POINT(40.6097069926397 -73.90036811685448) bank71372 +71373 POINT(40.43234818828573 -73.62185602282916) bank71373 +71374 POINT(40.72337593999152 -73.08326995515094) bank71374 +71375 POINT(39.8234193905467 -73.55852028111916) bank71375 +71376 POINT(40.143716721508326 -73.78767036171921) bank71376 +71377 POINT(40.4730648793062 -73.54758291315588) bank71377 +71378 POINT(40.625589684747595 -74.59394256865225) bank71378 +71379 POINT(40.26307214717966 -74.16341406922227) bank71379 +71380 POINT(40.75861274222808 -74.09952111419315) bank71380 +71381 POINT(40.290892874712284 -73.87625989561127) bank71381 +71382 POINT(40.205274818712304 -74.4514158766024) bank71382 +71383 POINT(40.85172332643306 -74.13640455031563) bank71383 +71384 POINT(40.27406009014927 -73.80873730446156) bank71384 +71385 POINT(40.09651758844231 -74.571294182053) bank71385 +71386 POINT(40.39911288809168 -73.36155109624202) bank71386 +71387 POINT(41.69437717891011 -74.00473242107537) bank71387 +71388 POINT(41.508219527765846 -74.10066948000096) bank71388 +71389 POINT(41.40675101485577 -74.26308223629941) bank71389 +71390 POINT(40.60540142413384 -74.49122756112202) bank71390 +71391 POINT(40.176226700061655 -74.5242676530584) bank71391 +71392 POINT(41.08702231746072 -74.61000406973939) bank71392 +71393 POINT(41.322126461440796 -73.64309060478556) bank71393 +71394 POINT(41.15943253603123 -73.91815594614926) bank71394 +71395 POINT(40.0263096495887 -74.57634427475986) bank71395 +71396 POINT(41.61767000509788 -73.33151392121476) bank71396 +71397 POINT(39.8994016507851 -73.21224313447033) bank71397 +71398 POINT(41.008662231988524 -74.57510112372766) bank71398 +71399 POINT(39.88135701242351 -74.86647344641312) bank71399 +71400 POINT(40.812514449081775 -73.38865254863094) bank71400 +71401 POINT(41.707613382866064 -73.9481366363225) bank71401 +71402 POINT(40.925308942484726 -74.09484189037869) bank71402 +71403 POINT(40.204067227531205 -73.04218098571386) bank71403 +71404 POINT(40.81486535630844 -74.53477239674964) bank71404 +71405 POINT(41.1125974132862 -73.55287552807224) bank71405 +71406 POINT(40.48047163600772 -73.5712613585631) bank71406 +71407 POINT(40.38210519686372 -73.23979224480385) bank71407 +71408 POINT(40.49950500281612 -74.78326146701764) bank71408 +71409 POINT(39.95919075234765 -74.892643386872) bank71409 +71410 POINT(40.338212799422415 -73.17657966824083) bank71410 +71411 POINT(41.05488577927139 -73.16814794682594) bank71411 +71412 POINT(40.86050965647067 -73.15237055317091) bank71412 +71413 POINT(40.29761109843169 -73.6216417102815) bank71413 +71414 POINT(40.06702924594689 -74.80915332648277) bank71414 +71415 POINT(41.38329572935274 -74.8460048025441) bank71415 +71416 POINT(40.41832449686561 -73.42818527668285) bank71416 +71417 POINT(41.412308412463716 -74.25663283252217) bank71417 +71418 POINT(41.55189243110601 -74.71583238613714) bank71418 +71419 POINT(40.57250816952384 -73.13845401099617) bank71419 +71420 POINT(39.93086101992386 -73.95648931810933) bank71420 +71421 POINT(40.157637969693624 -74.61165256183067) bank71421 +71422 POINT(40.398915104658215 -74.42393028695528) bank71422 +71423 POINT(40.172621531299285 -74.89396953037533) bank71423 +71424 POINT(40.42735330412606 -74.69391258282099) bank71424 +71425 POINT(40.544928513259606 -73.48273450427557) bank71425 +71426 POINT(40.50567394489176 -73.70771722830378) bank71426 +71427 POINT(40.04850844466152 -74.6234209954147) bank71427 +71428 POINT(40.433150503734744 -73.43999212410111) bank71428 +71429 POINT(41.300404009400566 -74.34808218141438) bank71429 +71430 POINT(39.818037528350146 -73.72310620440159) bank71430 +71431 POINT(41.53662081327351 -74.44960821056601) bank71431 +71432 POINT(40.9192093635302 -74.76910048627482) bank71432 +71433 POINT(39.912776308859854 -73.53087666750478) bank71433 +71434 POINT(39.99805265233843 -74.80696820466785) bank71434 +71435 POINT(40.142090874290666 -74.83091106289233) bank71435 +71436 POINT(40.489037602362416 -74.3327773658288) bank71436 +71437 POINT(40.94675888332846 -74.05707104535455) bank71437 +71438 POINT(40.54518787718046 -74.83984557597363) bank71438 +71439 POINT(41.183174441840116 -73.63052112973419) bank71439 +71440 POINT(40.89194217739399 -74.49024248846345) bank71440 +71441 POINT(40.296894532461906 -74.03340660371633) bank71441 +71442 POINT(40.55362619573812 -73.09600788029128) bank71442 +71443 POINT(40.09349996945568 -74.71860314163274) bank71443 +71444 POINT(40.08329539437434 -73.13460445138402) bank71444 +71445 POINT(41.530263628838185 -73.59827127475396) bank71445 +71446 POINT(41.288377712691144 -73.03143212071456) bank71446 +71447 POINT(41.52951016304956 -74.37728088336456) bank71447 +71448 POINT(41.29679635574734 -74.51996247153177) bank71448 +71449 POINT(40.52584909379168 -73.29287552575865) bank71449 +71450 POINT(39.91157130550973 -74.27877941096962) bank71450 +71451 POINT(40.77187118482538 -74.46043279114723) bank71451 +71452 POINT(41.55473459634584 -73.30472766891182) bank71452 +71453 POINT(41.28721319079539 -73.28892975145205) bank71453 +71454 POINT(40.747452141655984 -73.82400498266152) bank71454 +71455 POINT(41.236592633976606 -73.46924915989712) bank71455 +71456 POINT(40.827680580088625 -74.22910629627745) bank71456 +71457 POINT(40.32947360850272 -74.28919245583208) bank71457 +71458 POINT(41.30165183800723 -73.53513191856959) bank71458 +71459 POINT(40.356750197561645 -73.87228746591722) bank71459 +71460 POINT(41.2213576594998 -74.08941890057457) bank71460 +71461 POINT(39.89740152090678 -74.50947951186218) bank71461 +71462 POINT(41.24178836840658 -74.49067105207043) bank71462 +71463 POINT(40.93437711257792 -73.32116834007212) bank71463 +71464 POINT(41.19754954577068 -74.9678847610778) bank71464 +71465 POINT(40.46162072077946 -74.15751720156764) bank71465 +71466 POINT(41.26726409008304 -73.53372453436884) bank71466 +71467 POINT(40.071739055610514 -73.16289080662379) bank71467 +71468 POINT(41.23061121212727 -73.70481059079829) bank71468 +71469 POINT(40.6394832414988 -73.7875972934563) bank71469 +71470 POINT(41.32272279094215 -74.22128831173893) bank71470 +71471 POINT(40.73519961091518 -74.9781430830408) bank71471 +71472 POINT(40.46618341249922 -73.71876835297117) bank71472 +71473 POINT(39.72724744496854 -73.24456208269555) bank71473 +71474 POINT(40.08282340825897 -73.43175919071568) bank71474 +71475 POINT(40.75826360091974 -73.28859840683477) bank71475 +71476 POINT(41.098404790864016 -74.23703706038546) bank71476 +71477 POINT(40.1643323888428 -73.47402964985993) bank71477 +71478 POINT(41.35676636052254 -74.53976793407517) bank71478 +71479 POINT(41.58900737544912 -73.53199991303252) bank71479 +71480 POINT(40.36519101841538 -73.6494632767268) bank71480 +71481 POINT(40.281696235145475 -73.12943381051747) bank71481 +71482 POINT(41.58423342665519 -74.71024070265402) bank71482 +71483 POINT(39.76880840475461 -73.16090832237063) bank71483 +71484 POINT(41.4649560758474 -74.00655724484136) bank71484 +71485 POINT(40.40911687311889 -74.50949822521468) bank71485 +71486 POINT(40.90639439365547 -74.86241666287972) bank71486 +71487 POINT(40.299744048536105 -73.43207759569147) bank71487 +71488 POINT(39.86368902381194 -73.91908638457532) bank71488 +71489 POINT(40.33211353466981 -73.86187255611254) bank71489 +71490 POINT(40.797534889098635 -74.18409017798952) bank71490 +71491 POINT(39.799543806504474 -73.67921460181981) bank71491 +71492 POINT(40.25337536087122 -74.36141299511587) bank71492 +71493 POINT(40.315655678955125 -74.23905199414432) bank71493 +71494 POINT(40.99058831413263 -73.97116676606947) bank71494 +71495 POINT(41.22656421696424 -73.03599435507175) bank71495 +71496 POINT(41.31609312156042 -73.37800379998102) bank71496 +71497 POINT(39.874784003853065 -73.52333575589842) bank71497 +71498 POINT(40.75005748952319 -74.35223439957882) bank71498 +71499 POINT(40.95471527661533 -74.62934355934226) bank71499 +71500 POINT(39.94516899004582 -73.24824469671222) bank71500 +71501 POINT(40.58512284765549 -73.3198392741962) bank71501 +71502 POINT(39.73181500874338 -73.67974412879066) bank71502 +71503 POINT(39.949198711031464 -73.451705566685) bank71503 +71504 POINT(40.84776497959363 -73.21035950856964) bank71504 +71505 POINT(41.697858935730295 -73.1346101120284) bank71505 +71506 POINT(39.92542862683256 -73.5988460638255) bank71506 +71507 POINT(40.55163774284492 -74.10708297464572) bank71507 +71508 POINT(41.51315920062292 -73.95888784529298) bank71508 +71509 POINT(40.07795072319401 -74.52263680782173) bank71509 +71510 POINT(41.17701986526648 -73.87780382188934) bank71510 +71511 POINT(41.0744802865746 -73.34245107882296) bank71511 +71512 POINT(41.59279642922542 -74.65293662839625) bank71512 +71513 POINT(40.081406623215976 -74.81180660011694) bank71513 +71514 POINT(40.4685407008065 -74.66718548210414) bank71514 +71515 POINT(40.96669267949883 -73.92062118613522) bank71515 +71516 POINT(40.829955461414116 -73.85328604060167) bank71516 +71517 POINT(41.43700024200565 -74.02288767409763) bank71517 +71518 POINT(41.50484161595756 -74.05785973041831) bank71518 +71519 POINT(40.240957873678795 -73.51759264905883) bank71519 +71520 POINT(40.19744677105331 -73.32386470692754) bank71520 +71521 POINT(40.18799814074277 -74.9722929102539) bank71521 +71522 POINT(40.02018472730457 -74.46198170821555) bank71522 +71523 POINT(40.51628342072611 -73.38495148354765) bank71523 +71524 POINT(40.37652884549951 -74.7221509431835) bank71524 +71525 POINT(40.54944007956798 -73.48427118596872) bank71525 +71526 POINT(40.80915518463188 -73.64567019783335) bank71526 +71527 POINT(40.06084950443933 -73.33312092188024) bank71527 +71528 POINT(40.458662554706294 -73.25676337647418) bank71528 +71529 POINT(39.79993430436283 -73.19510992169837) bank71529 +71530 POINT(40.58080940392287 -73.84165393976684) bank71530 +71531 POINT(39.82216212173137 -74.48410196300702) bank71531 +71532 POINT(41.15168997818508 -73.49969401935888) bank71532 +71533 POINT(41.14104726387641 -73.16465836258686) bank71533 +71534 POINT(40.97679386092774 -73.87164234772237) bank71534 +71535 POINT(40.570620574750365 -74.73947747207325) bank71535 +71536 POINT(41.017348767911756 -73.5838391115166) bank71536 +71537 POINT(40.29582455588258 -74.48664390519659) bank71537 +71538 POINT(41.49512019267453 -74.85823755334037) bank71538 +71539 POINT(40.785532415919334 -73.36241285734353) bank71539 +71540 POINT(40.84779504563666 -74.72984883782718) bank71540 +71541 POINT(40.17067099483959 -73.57961860036313) bank71541 +71542 POINT(41.35869365475802 -74.20256971742667) bank71542 +71543 POINT(40.20684347331726 -74.5530555692347) bank71543 +71544 POINT(39.85648566180233 -74.33705843693386) bank71544 +71545 POINT(40.08354656441036 -73.12379641786656) bank71545 +71546 POINT(40.73478882638683 -74.96507596083401) bank71546 +71547 POINT(41.37208215168338 -74.94025968227336) bank71547 +71548 POINT(41.27669685859838 -74.99713225817612) bank71548 +71549 POINT(40.14294877800435 -73.46586390223138) bank71549 +71550 POINT(41.21730206934312 -74.52926598752454) bank71550 +71551 POINT(41.3392551311078 -73.10474208403498) bank71551 +71552 POINT(40.69870129884705 -74.7524531895617) bank71552 +71553 POINT(41.02153267795124 -73.34926314079863) bank71553 +71554 POINT(40.006809501748194 -73.1374893573738) bank71554 +71555 POINT(40.31942082521364 -73.40595911834617) bank71555 +71556 POINT(39.866061778115814 -74.76301919116304) bank71556 +71557 POINT(41.48737056380488 -73.09261390568437) bank71557 +71558 POINT(41.68408162682521 -73.40394702954018) bank71558 +71559 POINT(41.03008795999845 -73.30995967005191) bank71559 +71560 POINT(40.57556323971865 -74.4778714080401) bank71560 +71561 POINT(40.17061117265483 -74.23700562739319) bank71561 +71562 POINT(41.213225018544776 -74.32174378262165) bank71562 +71563 POINT(39.76548038751522 -74.21583206029464) bank71563 +71564 POINT(41.25168968876788 -73.84007972801778) bank71564 +71565 POINT(41.70609028916703 -73.2886768883266) bank71565 +71566 POINT(41.59365696407851 -74.2352008327716) bank71566 +71567 POINT(40.06351863658896 -74.62082648648719) bank71567 +71568 POINT(41.18182092591873 -74.83418190465369) bank71568 +71569 POINT(41.27320561417874 -73.29739039398487) bank71569 +71570 POINT(39.81508186070145 -73.5159500666378) bank71570 +71571 POINT(41.27100847647069 -74.10076333806998) bank71571 +71572 POINT(41.125685346405845 -74.89953931713596) bank71572 +71573 POINT(41.197347695761096 -73.13596189890069) bank71573 +71574 POINT(41.076798361125334 -73.99709233346245) bank71574 +71575 POINT(41.41661305477924 -74.75724722974846) bank71575 +71576 POINT(40.103457596987454 -73.94945243256895) bank71576 +71577 POINT(40.93753521022085 -73.18573439147673) bank71577 +71578 POINT(41.10579370657253 -73.11833547400006) bank71578 +71579 POINT(41.052274228463794 -74.29190747469639) bank71579 +71580 POINT(40.83053513345604 -73.8125779540218) bank71580 +71581 POINT(40.91430356366488 -74.63269297028627) bank71581 +71582 POINT(40.532027438355435 -73.43947653249555) bank71582 +71583 POINT(39.88887280771499 -73.95851948792946) bank71583 +71584 POINT(40.028765636746265 -73.00839603404165) bank71584 +71585 POINT(40.1476446534109 -73.75712043124402) bank71585 +71586 POINT(40.845902490139395 -73.57327859474064) bank71586 +71587 POINT(40.976896030001576 -74.21084077059886) bank71587 +71588 POINT(41.08095219757659 -73.31318122098328) bank71588 +71589 POINT(41.45980658968371 -73.79428118178447) bank71589 +71590 POINT(39.78642054085199 -74.37407508357983) bank71590 +71591 POINT(41.565853128107086 -74.26740806279003) bank71591 +71592 POINT(40.11705844723746 -73.49055281006532) bank71592 +71593 POINT(41.16438266941606 -73.36479796609514) bank71593 +71594 POINT(39.7536485754129 -74.14813153322228) bank71594 +71595 POINT(40.944089194306265 -73.86039170416059) bank71595 +71596 POINT(40.60264027480036 -73.70752036412433) bank71596 +71597 POINT(39.93230835578657 -73.42606081664404) bank71597 +71598 POINT(40.033620024983755 -74.434165950291) bank71598 +71599 POINT(40.61927015110079 -73.4416438936042) bank71599 +71600 POINT(41.178740384940504 -74.51469115614441) bank71600 +71601 POINT(41.07856419274519 -74.08254082341892) bank71601 +71602 POINT(40.80122418969369 -74.86504392686143) bank71602 +71603 POINT(40.688435598149724 -74.05574738428206) bank71603 +71604 POINT(41.32322822842444 -73.63026404427492) bank71604 +71605 POINT(40.96410002083775 -74.59279389734618) bank71605 +71606 POINT(40.86963808617283 -73.02484098544056) bank71606 +71607 POINT(40.57516149822049 -74.27252051933336) bank71607 +71608 POINT(40.6782617874357 -73.90036701547209) bank71608 +71609 POINT(41.399478682343826 -73.87169832041799) bank71609 +71610 POINT(40.858849803356904 -73.4259804104782) bank71610 +71611 POINT(40.71229533855605 -74.76524476856872) bank71611 +71612 POINT(40.947666969792955 -74.93175953545668) bank71612 +71613 POINT(40.54206793405559 -74.60753256011996) bank71613 +71614 POINT(40.957303843987404 -74.27224394260288) bank71614 +71615 POINT(40.253356183003376 -73.42984660568793) bank71615 +71616 POINT(39.807544114699446 -73.38652808315905) bank71616 +71617 POINT(39.77744307093922 -74.819272309863) bank71617 +71618 POINT(41.68027017053242 -73.86182297360968) bank71618 +71619 POINT(40.66958206903139 -73.92403697789439) bank71619 +71620 POINT(41.05349316180093 -73.76622873417807) bank71620 +71621 POINT(40.248621906612556 -73.42385894336032) bank71621 +71622 POINT(40.935498473149636 -74.76616444254883) bank71622 +71623 POINT(40.87040564036214 -73.42611547616431) bank71623 +71624 POINT(40.043739657483194 -73.42923349749186) bank71624 +71625 POINT(40.2877559223992 -74.70822777709289) bank71625 +71626 POINT(40.47583782422804 -73.30151599043569) bank71626 +71627 POINT(40.08421683959947 -74.09115076272946) bank71627 +71628 POINT(40.56775252433519 -73.11945245015093) bank71628 +71629 POINT(39.84578254075831 -74.20432099156952) bank71629 +71630 POINT(41.65814431599513 -74.59513260661238) bank71630 +71631 POINT(41.617344160737936 -74.52802828762658) bank71631 +71632 POINT(41.187665791922946 -73.18803523159377) bank71632 +71633 POINT(40.34894279632373 -74.6465313878026) bank71633 +71634 POINT(41.39674096877386 -74.67325633264913) bank71634 +71635 POINT(39.9351980370849 -74.18297134303677) bank71635 +71636 POINT(41.45939190826914 -73.5723079946434) bank71636 +71637 POINT(40.88619564492237 -74.57656981288983) bank71637 +71638 POINT(41.61746320138459 -74.41161796474384) bank71638 +71639 POINT(41.156980595315524 -73.75294106774115) bank71639 +71640 POINT(40.70499229746 -73.44314088940457) bank71640 +71641 POINT(40.50336896150824 -75.00257900637328) bank71641 +71642 POINT(40.47650698613077 -74.75792673508799) bank71642 +71643 POINT(40.07320237059972 -74.11543475871296) bank71643 +71644 POINT(41.20611078032582 -73.94281406083012) bank71644 +71645 POINT(39.858435495367516 -74.13809025445347) bank71645 +71646 POINT(41.22504877862862 -73.40959646913403) bank71646 +71647 POINT(41.40276264779485 -73.5501757363645) bank71647 +71648 POINT(40.325345289314704 -73.76440578362038) bank71648 +71649 POINT(39.799585883400866 -73.4660161697964) bank71649 +71650 POINT(41.372880686662214 -73.5836137210392) bank71650 +71651 POINT(41.37149277961677 -73.1888399128693) bank71651 +71652 POINT(39.99747494368402 -74.39191417731068) bank71652 +71653 POINT(40.06291789507741 -73.06759712391631) bank71653 +71654 POINT(41.230059425071566 -73.39455533305548) bank71654 +71655 POINT(41.26145704634982 -73.33884624383336) bank71655 +71656 POINT(40.4426023005285 -74.14096782379806) bank71656 +71657 POINT(39.88258303907261 -74.283885876665) bank71657 +71658 POINT(41.539801422122665 -73.27986061896296) bank71658 +71659 POINT(41.09923911425601 -74.0062635183402) bank71659 +71660 POINT(41.27759990103969 -74.13356955959178) bank71660 +71661 POINT(41.077373746995455 -74.97678339616705) bank71661 +71662 POINT(41.671043517008734 -73.1661743257019) bank71662 +71663 POINT(41.280828912058766 -73.42440770547923) bank71663 +71664 POINT(39.75428334424743 -74.6150969443798) bank71664 +71665 POINT(41.08347860309946 -73.51392919516653) bank71665 +71666 POINT(39.97002592498123 -73.96727286662423) bank71666 +71667 POINT(41.52871290106907 -74.37095825870472) bank71667 +71668 POINT(39.83266357071287 -74.68754597265973) bank71668 +71669 POINT(39.85799361082971 -74.95016764867728) bank71669 +71670 POINT(40.50542179157154 -74.1066107357807) bank71670 +71671 POINT(40.72623520778932 -74.62903214015114) bank71671 +71672 POINT(41.378005826442376 -74.03198055890192) bank71672 +71673 POINT(41.4250512788177 -74.20089152133576) bank71673 +71674 POINT(40.53722273791627 -74.50598766003047) bank71674 +71675 POINT(39.87889472789431 -73.97481243277392) bank71675 +71676 POINT(40.66943483775704 -74.35688627342654) bank71676 +71677 POINT(40.308508521056986 -73.56109101025173) bank71677 +71678 POINT(40.46958430707495 -74.87842466380394) bank71678 +71679 POINT(40.648382715159 -75.00241935402086) bank71679 +71680 POINT(40.14611913781015 -73.761165468287) bank71680 +71681 POINT(40.052654228464775 -73.75346635029315) bank71681 +71682 POINT(40.72988404287181 -73.69780368900487) bank71682 +71683 POINT(40.54390572687349 -74.4689226422005) bank71683 +71684 POINT(40.6629681226525 -74.04858398393193) bank71684 +71685 POINT(40.9639695092725 -74.54374560737672) bank71685 +71686 POINT(41.010864910367346 -74.31435013484922) bank71686 +71687 POINT(40.82532451584402 -74.66669536121549) bank71687 +71688 POINT(40.28578469211615 -74.09989518434693) bank71688 +71689 POINT(40.15269211949903 -74.19208739518209) bank71689 +71690 POINT(40.36736683328993 -73.65504777410074) bank71690 +71691 POINT(40.462293314560696 -74.35836588867122) bank71691 +71692 POINT(41.27887966300383 -73.74191471982239) bank71692 +71693 POINT(40.92898970755498 -74.80534420649425) bank71693 +71694 POINT(40.14711904872904 -73.6456375526603) bank71694 +71695 POINT(41.672512183034506 -73.11015559946345) bank71695 +71696 POINT(41.35530721182271 -73.32965115360179) bank71696 +71697 POINT(39.83663776436293 -74.31796996109745) bank71697 +71698 POINT(41.23181959185451 -73.93437398814402) bank71698 +71699 POINT(40.25496306356568 -74.67685523687786) bank71699 +71700 POINT(40.235260478977736 -73.20504550999249) bank71700 +71701 POINT(40.99997245667657 -74.935525525559) bank71701 +71702 POINT(41.627732417363845 -74.88191601840877) bank71702 +71703 POINT(41.64985244605656 -73.18895337006093) bank71703 +71704 POINT(40.69339170401641 -73.81105908983277) bank71704 +71705 POINT(40.46801302088219 -73.3272574293025) bank71705 +71706 POINT(40.487115649010164 -74.61694614252573) bank71706 +71707 POINT(40.2363975199822 -73.73237830426558) bank71707 +71708 POINT(41.582275323841756 -73.43888192052647) bank71708 +71709 POINT(40.78729141628717 -74.31644177885471) bank71709 +71710 POINT(40.996107589834345 -74.68712052879084) bank71710 +71711 POINT(39.89365694572727 -74.33269974951124) bank71711 +71712 POINT(39.71706843570316 -73.46724123137754) bank71712 +71713 POINT(40.81489173712668 -73.15428575882535) bank71713 +71714 POINT(40.39563418468049 -74.36737111697632) bank71714 +71715 POINT(40.36028913491226 -73.2537957150964) bank71715 +71716 POINT(41.50659659514755 -73.17013482965078) bank71716 +71717 POINT(41.19887380977459 -73.65936455688403) bank71717 +71718 POINT(41.47169999429394 -73.68787066691377) bank71718 +71719 POINT(41.6013049611357 -73.02380276674153) bank71719 +71720 POINT(40.80346541120782 -74.22317236371346) bank71720 +71721 POINT(41.39817433661104 -74.8650654990862) bank71721 +71722 POINT(39.91948192412433 -73.16307366469246) bank71722 +71723 POINT(41.34599079807638 -74.73007766525178) bank71723 +71724 POINT(40.083309341027004 -73.1185410596351) bank71724 +71725 POINT(40.37345894191205 -73.5902814451921) bank71725 +71726 POINT(40.595207503119084 -73.23753758839783) bank71726 +71727 POINT(41.04532699139485 -74.4584561935785) bank71727 +71728 POINT(41.38352820708938 -73.46923991320699) bank71728 +71729 POINT(41.3506899310526 -73.45399145539933) bank71729 +71730 POINT(40.36229631503509 -74.11216008781916) bank71730 +71731 POINT(40.62549800054512 -73.90004034334065) bank71731 +71732 POINT(40.65272183026868 -73.21907468018723) bank71732 +71733 POINT(41.586866461349224 -75.00593547216829) bank71733 +71734 POINT(40.161909646412994 -74.67729227997941) bank71734 +71735 POINT(41.02314892611291 -73.16283127734772) bank71735 +71736 POINT(41.44955783209798 -73.59701916420887) bank71736 +71737 POINT(40.69094397781295 -73.37141049685522) bank71737 +71738 POINT(41.56539018514722 -73.31517780430988) bank71738 +71739 POINT(40.50614651160372 -73.55413224561624) bank71739 +71740 POINT(41.60891663290859 -73.78903408857808) bank71740 +71741 POINT(40.79524471031364 -73.08523371746922) bank71741 +71742 POINT(40.59171148951777 -73.92843880013282) bank71742 +71743 POINT(41.60027362579076 -74.71905509849245) bank71743 +71744 POINT(40.70350452112897 -73.54337684510558) bank71744 +71745 POINT(40.40980068492371 -74.69710037377021) bank71745 +71746 POINT(39.860582959094934 -74.4317140554762) bank71746 +71747 POINT(41.21622382518753 -73.9207464432684) bank71747 +71748 POINT(41.05442784227071 -74.01418617805324) bank71748 +71749 POINT(41.23581342925851 -73.17263675236035) bank71749 +71750 POINT(40.7199339582742 -74.55908736951585) bank71750 +71751 POINT(39.75195268975746 -74.89110899487972) bank71751 +71752 POINT(40.385115384278095 -74.17613096278248) bank71752 +71753 POINT(40.49170424031448 -73.35067262701219) bank71753 +71754 POINT(40.629839638852026 -73.56102052892646) bank71754 +71755 POINT(40.48445505364579 -73.68914319561519) bank71755 +71756 POINT(41.00307915089593 -74.85122102260377) bank71756 +71757 POINT(40.843840664943286 -74.12623035773365) bank71757 +71758 POINT(40.524000419473246 -73.95175637794925) bank71758 +71759 POINT(41.50105304227718 -73.12222498112318) bank71759 +71760 POINT(40.761160339566786 -74.92481800754618) bank71760 +71761 POINT(41.458566053526596 -74.59503531786943) bank71761 +71762 POINT(40.825117304958646 -74.73406076385125) bank71762 +71763 POINT(41.59435051062607 -73.7921898580495) bank71763 +71764 POINT(41.33434661025504 -74.05543977534914) bank71764 +71765 POINT(41.12398154048313 -73.60669686023662) bank71765 +71766 POINT(40.98053957899976 -74.6447252624173) bank71766 +71767 POINT(41.099558641457 -74.87841668292036) bank71767 +71768 POINT(40.40296203920297 -73.2569698429863) bank71768 +71769 POINT(39.933075588109645 -74.52614503429454) bank71769 +71770 POINT(41.45725210687641 -73.43136959432292) bank71770 +71771 POINT(40.973237481765395 -73.89977721142515) bank71771 +71772 POINT(40.898258046586825 -73.50158025090884) bank71772 +71773 POINT(41.382265289471206 -73.18423229705606) bank71773 +71774 POINT(41.607931459205524 -73.99748606508132) bank71774 +71775 POINT(40.13126334588808 -74.13146306527204) bank71775 +71776 POINT(39.870305884836405 -73.35743427698604) bank71776 +71777 POINT(39.872597634515884 -74.29221747079077) bank71777 +71778 POINT(40.32340857009425 -73.59315057158874) bank71778 +71779 POINT(40.012147905156404 -73.08496163052033) bank71779 +71780 POINT(40.524443913936416 -74.88688027683185) bank71780 +71781 POINT(40.23367219044608 -73.45123902986697) bank71781 +71782 POINT(40.15121798201738 -73.65594950866692) bank71782 +71783 POINT(39.75002254450913 -73.01520960186329) bank71783 +71784 POINT(40.34888492975313 -73.69758296818313) bank71784 +71785 POINT(41.30872223133259 -73.18357098693869) bank71785 +71786 POINT(40.00341228618077 -74.29853368599444) bank71786 +71787 POINT(40.475861399001914 -74.21464323046406) bank71787 +71788 POINT(40.62388481904511 -73.35115182440872) bank71788 +71789 POINT(40.51425007120353 -73.54243101471481) bank71789 +71790 POINT(40.2039415266821 -73.51609564488491) bank71790 +71791 POINT(41.4173389878098 -74.3877238019014) bank71791 +71792 POINT(39.99140284510916 -74.11413716059428) bank71792 +71793 POINT(40.956473692968 -74.89571661980764) bank71793 +71794 POINT(41.34027677194116 -73.71125639988732) bank71794 +71795 POINT(40.64936486857893 -74.4788667127395) bank71795 +71796 POINT(41.435335778674244 -74.30539198707909) bank71796 +71797 POINT(40.05435704655744 -73.20865917111666) bank71797 +71798 POINT(41.57444137078916 -74.60643825054215) bank71798 +71799 POINT(41.19567461361277 -73.43412523132427) bank71799 +71800 POINT(40.4732934656377 -73.05458797879255) bank71800 +71801 POINT(40.549086681593636 -74.37183753799258) bank71801 +71802 POINT(40.976748161869 -73.92873442563808) bank71802 +71803 POINT(39.80651417902701 -73.43023951510716) bank71803 +71804 POINT(40.139527611011104 -73.24920975673201) bank71804 +71805 POINT(40.49001161999596 -74.27942635147411) bank71805 +71806 POINT(40.960695888710106 -73.2123592582626) bank71806 +71807 POINT(40.22974849863597 -73.71448579737874) bank71807 +71808 POINT(40.55505265955745 -73.6926322193384) bank71808 +71809 POINT(40.62684194942933 -73.1631621268892) bank71809 +71810 POINT(41.43096635515546 -74.16329078183927) bank71810 +71811 POINT(39.90072166775492 -74.53055590768811) bank71811 +71812 POINT(41.13370025211164 -73.85210823604922) bank71812 +71813 POINT(41.3136608387464 -73.07059631523006) bank71813 +71814 POINT(41.11305586306543 -73.68181702427808) bank71814 +71815 POINT(41.57023568715018 -73.84988740855589) bank71815 +71816 POINT(40.64258285716913 -74.78477911127251) bank71816 +71817 POINT(40.38127942507445 -74.4196852633016) bank71817 +71818 POINT(41.21862866626951 -73.16203066704004) bank71818 +71819 POINT(40.72133351803864 -74.62409349900588) bank71819 +71820 POINT(39.899396340016985 -73.0606631631989) bank71820 +71821 POINT(41.31851862939974 -74.84775163059955) bank71821 +71822 POINT(41.67646774839832 -73.81179111665381) bank71822 +71823 POINT(40.760318702024 -74.61269265721936) bank71823 +71824 POINT(40.5791060316139 -74.83436395121626) bank71824 +71825 POINT(41.34659107665064 -73.13845414830308) bank71825 +71826 POINT(40.0261008341917 -73.71714693971052) bank71826 +71827 POINT(40.72153703231899 -73.61319869419748) bank71827 +71828 POINT(40.2951629232261 -73.17108000259992) bank71828 +71829 POINT(40.494002248604616 -73.72739459149415) bank71829 +71830 POINT(39.931013462829284 -74.16540103360214) bank71830 +71831 POINT(41.5598773895199 -73.99060116786549) bank71831 +71832 POINT(41.29079474935129 -74.23425167111046) bank71832 +71833 POINT(41.55236976002378 -73.90508795424142) bank71833 +71834 POINT(39.82441737957302 -73.65553949696077) bank71834 +71835 POINT(41.1290803797833 -74.8356327977707) bank71835 +71836 POINT(41.00995985353285 -73.47891057219752) bank71836 +71837 POINT(40.349058270677745 -73.37248752371805) bank71837 +71838 POINT(41.323754528132575 -73.53658815043252) bank71838 +71839 POINT(40.50342961928869 -74.72509378417139) bank71839 +71840 POINT(39.7592605146253 -73.06485330910152) bank71840 +71841 POINT(40.816088196591956 -73.35742142634767) bank71841 +71842 POINT(39.993668006114866 -73.8350605845132) bank71842 +71843 POINT(40.56099683602219 -74.61555798077237) bank71843 +71844 POINT(41.58055509787905 -74.58765880235616) bank71844 +71845 POINT(40.49179780048567 -74.36203455484457) bank71845 +71846 POINT(40.29689724408156 -73.36586346986417) bank71846 +71847 POINT(39.8195440480649 -73.93716716673676) bank71847 +71848 POINT(40.421902011377824 -73.2812312394356) bank71848 +71849 POINT(40.57225244621347 -73.27872560556773) bank71849 +71850 POINT(40.96373451296309 -73.67722194380781) bank71850 +71851 POINT(41.21817634338209 -73.95333231384198) bank71851 +71852 POINT(39.72483206883827 -74.96629348086391) bank71852 +71853 POINT(39.825303207108405 -73.05434715592482) bank71853 +71854 POINT(40.39560335566193 -73.04909892486533) bank71854 +71855 POINT(40.878264082909325 -73.2050498715336) bank71855 +71856 POINT(40.02396704144868 -74.04227233331079) bank71856 +71857 POINT(39.97003370485828 -73.01746752944786) bank71857 +71858 POINT(40.71769951091889 -74.3604458768932) bank71858 +71859 POINT(40.04265997682882 -74.57792372995566) bank71859 +71860 POINT(40.946205398715264 -73.24589910764772) bank71860 +71861 POINT(41.145337072266344 -74.23214608600271) bank71861 +71862 POINT(40.01819702389981 -73.38022395276288) bank71862 +71863 POINT(41.50136410462876 -73.56714853144416) bank71863 +71864 POINT(41.570347202047564 -74.99167111216038) bank71864 +71865 POINT(40.252969763938275 -73.76650542745423) bank71865 +71866 POINT(41.40856499683632 -73.47401775730033) bank71866 +71867 POINT(40.08796709809337 -73.81090887830746) bank71867 +71868 POINT(41.41464048114654 -74.5204111187509) bank71868 +71869 POINT(40.121441535767694 -74.02820140441244) bank71869 +71870 POINT(40.05704675974999 -73.77912912290267) bank71870 +71871 POINT(41.46570160152628 -74.76993902654755) bank71871 +71872 POINT(39.957167306108836 -74.51863484638349) bank71872 +71873 POINT(41.27900055704476 -74.33154801902081) bank71873 +71874 POINT(40.05312512474153 -73.72272600874699) bank71874 +71875 POINT(40.11739061115904 -73.64145426163496) bank71875 +71876 POINT(40.168385273567736 -74.7945511041707) bank71876 +71877 POINT(39.71603619883645 -74.35713421141678) bank71877 +71878 POINT(39.78171138420766 -74.44354846774341) bank71878 +71879 POINT(39.79165428394833 -74.04291071429043) bank71879 +71880 POINT(39.891772322470295 -74.25274451919792) bank71880 +71881 POINT(41.2772820666675 -73.5542206434953) bank71881 +71882 POINT(40.72987459686494 -73.74681945428792) bank71882 +71883 POINT(41.67743469648658 -73.65579634059087) bank71883 +71884 POINT(40.28152294884285 -74.8357775585741) bank71884 +71885 POINT(41.61396600453228 -74.3888748969068) bank71885 +71886 POINT(40.27221168025967 -73.0124123078107) bank71886 +71887 POINT(41.44328214933786 -74.73598516752459) bank71887 +71888 POINT(41.49680475899224 -73.77914397174557) bank71888 +71889 POINT(40.94039827663662 -74.4049710273936) bank71889 +71890 POINT(39.88146153029101 -74.40173365826679) bank71890 +71891 POINT(40.56426844715107 -73.97034073364316) bank71891 +71892 POINT(41.14930151141677 -73.35299180595703) bank71892 +71893 POINT(40.29414000697958 -73.01633637917831) bank71893 +71894 POINT(40.88800143653194 -74.95188462400947) bank71894 +71895 POINT(40.165464428223146 -73.86766318512235) bank71895 +71896 POINT(40.871337079898495 -73.51771383010467) bank71896 +71897 POINT(41.130257180951105 -74.10357807831355) bank71897 +71898 POINT(40.95488569648323 -73.32898036446062) bank71898 +71899 POINT(41.65180878480751 -74.72449848458447) bank71899 +71900 POINT(40.53327480753941 -74.28347271691888) bank71900 +71901 POINT(40.991567517294456 -74.19545475680451) bank71901 +71902 POINT(41.07529391696831 -73.2874409784978) bank71902 +71903 POINT(40.18154566693004 -73.21332201875423) bank71903 +71904 POINT(40.322762693309244 -73.61728683904964) bank71904 +71905 POINT(40.309940349924204 -73.36003227579424) bank71905 +71906 POINT(40.87875747981037 -74.08585038726193) bank71906 +71907 POINT(40.47561878507752 -74.50627362611937) bank71907 +71908 POINT(40.62398612619574 -74.11983118208758) bank71908 +71909 POINT(40.70931404027209 -73.34221246429338) bank71909 +71910 POINT(40.66834947393233 -74.1338536562944) bank71910 +71911 POINT(40.154084387983005 -73.20215488052372) bank71911 +71912 POINT(40.613702624342146 -73.33978484164233) bank71912 +71913 POINT(40.583001006177824 -74.70072371884721) bank71913 +71914 POINT(40.29191916493727 -73.80006176011393) bank71914 +71915 POINT(40.73007165754817 -74.6190882598324) bank71915 +71916 POINT(41.22191002994673 -74.31256464954092) bank71916 +71917 POINT(41.52025395562345 -73.39356706158244) bank71917 +71918 POINT(40.89378372787685 -73.41226835447668) bank71918 +71919 POINT(40.09561245123724 -73.04465004677975) bank71919 +71920 POINT(40.72707780035711 -74.32503706072883) bank71920 +71921 POINT(41.64141122758796 -74.68937950862015) bank71921 +71922 POINT(40.44594209031024 -73.55614994407298) bank71922 +71923 POINT(39.91832022167374 -74.39647109137682) bank71923 +71924 POINT(40.613572440111085 -74.57137301281013) bank71924 +71925 POINT(41.68883879308189 -74.04943262920499) bank71925 +71926 POINT(39.994573177639154 -74.44756337623576) bank71926 +71927 POINT(40.96534824199639 -73.24946188445614) bank71927 +71928 POINT(40.583843616232365 -73.69211202490028) bank71928 +71929 POINT(41.61867051806926 -74.7112282617313) bank71929 +71930 POINT(41.11854923460275 -74.01788512056486) bank71930 +71931 POINT(40.774013394542415 -73.63710852323824) bank71931 +71932 POINT(41.47700692995707 -74.50320144994915) bank71932 +71933 POINT(40.47649929595761 -74.17710526630218) bank71933 +71934 POINT(40.25043881131968 -75.00252765299535) bank71934 +71935 POINT(40.67747728862242 -74.22717272183378) bank71935 +71936 POINT(40.297475483043506 -73.7662991559671) bank71936 +71937 POINT(40.22005478517458 -74.19183383744544) bank71937 +71938 POINT(41.679937277119144 -74.03626466706197) bank71938 +71939 POINT(41.36319159814192 -74.93624813600523) bank71939 +71940 POINT(41.07879615821168 -73.08720967262752) bank71940 +71941 POINT(41.39712903966715 -74.4549481677541) bank71941 +71942 POINT(39.77309680507698 -73.39397573025768) bank71942 +71943 POINT(39.89005825973312 -73.0905182122683) bank71943 +71944 POINT(41.0086235537781 -73.48295897462606) bank71944 +71945 POINT(41.40193406908262 -74.56790272946314) bank71945 +71946 POINT(40.35732067288127 -74.01890091641042) bank71946 +71947 POINT(40.299540449739 -73.79192316525206) bank71947 +71948 POINT(39.89128283246955 -74.75582063249304) bank71948 +71949 POINT(41.117756131273005 -74.70138943291076) bank71949 +71950 POINT(40.844250104488715 -73.45673949741382) bank71950 +71951 POINT(39.878068198040424 -74.405232312833) bank71951 +71952 POINT(40.921383220237736 -74.29052249878731) bank71952 +71953 POINT(40.22911799842392 -73.85620203058502) bank71953 +71954 POINT(41.41963603000114 -74.41709955511529) bank71954 +71955 POINT(40.010343532009415 -74.4116901664687) bank71955 +71956 POINT(41.58466652534158 -73.01624387009117) bank71956 +71957 POINT(40.14370756250356 -73.69297152161728) bank71957 +71958 POINT(41.66267362568079 -73.62560022925658) bank71958 +71959 POINT(40.047765360921325 -74.28103869681685) bank71959 +71960 POINT(41.06394380573869 -73.70564227168721) bank71960 +71961 POINT(41.235995674083235 -74.26247784961494) bank71961 +71962 POINT(41.142563956210594 -73.4830312792938) bank71962 +71963 POINT(41.685562438744874 -73.7529246526432) bank71963 +71964 POINT(40.51893038584911 -74.57407182924848) bank71964 +71965 POINT(40.56700395333632 -73.06404882504353) bank71965 +71966 POINT(40.619750240550815 -74.81879373511234) bank71966 +71967 POINT(40.96525136501501 -74.94284861897414) bank71967 +71968 POINT(41.36486603310179 -73.70595927321934) bank71968 +71969 POINT(41.64359605959081 -74.6710821989888) bank71969 +71970 POINT(41.073747179663876 -73.68813648231415) bank71970 +71971 POINT(41.635153262333624 -73.31937433695778) bank71971 +71972 POINT(39.74168983485245 -73.6912650536471) bank71972 +71973 POINT(41.16839940527203 -74.75288776769406) bank71973 +71974 POINT(41.44175038707306 -73.3965758795146) bank71974 +71975 POINT(41.207473841918905 -74.45757777128037) bank71975 +71976 POINT(40.09955812363745 -74.1795530822726) bank71976 +71977 POINT(41.7126574478051 -73.56201503616275) bank71977 +71978 POINT(40.24364147990898 -73.93354206223644) bank71978 +71979 POINT(41.26253068265306 -73.6378077854566) bank71979 +71980 POINT(41.52877099840818 -74.30025642282914) bank71980 +71981 POINT(39.78469604677289 -73.90142805941703) bank71981 +71982 POINT(41.04153196511706 -73.0857609126611) bank71982 +71983 POINT(41.24529089440999 -73.49783122900303) bank71983 +71984 POINT(39.76855791753955 -74.6911458119124) bank71984 +71985 POINT(40.75866705656825 -74.92614955649212) bank71985 +71986 POINT(41.534573128158826 -73.55899443793676) bank71986 +71987 POINT(40.87969964461607 -74.248176274723) bank71987 +71988 POINT(39.815274247283845 -74.90557043358696) bank71988 +71989 POINT(39.99834464538362 -73.14457419165608) bank71989 +71990 POINT(40.244472014149736 -74.37334668542762) bank71990 +71991 POINT(41.3006200381859 -74.3521499482255) bank71991 +71992 POINT(41.54627274957844 -73.04754927246675) bank71992 +71993 POINT(41.45480997600168 -73.72079662419524) bank71993 +71994 POINT(39.88241776959934 -73.70484144223) bank71994 +71995 POINT(41.544519697569676 -73.15429864091443) bank71995 +71996 POINT(40.79682675612868 -74.01466382484769) bank71996 +71997 POINT(41.44150265846933 -74.87299550007192) bank71997 +71998 POINT(41.4559116419794 -74.21905638182488) bank71998 +71999 POINT(40.73786633751198 -74.19204476004903) bank71999 +72000 POINT(40.806286830940444 -73.86023619508421) bank72000 +72001 POINT(41.20305426397889 -73.95430671229953) bank72001 +72002 POINT(41.64678442390495 -74.68823318560132) bank72002 +72003 POINT(39.87052203991313 -74.40322492782715) bank72003 +72004 POINT(39.831374770285365 -73.47431961522072) bank72004 +72005 POINT(40.176912601821556 -73.80123785764773) bank72005 +72006 POINT(40.12521848539336 -73.48796369125405) bank72006 +72007 POINT(40.683402028298225 -74.15162961438999) bank72007 +72008 POINT(41.18829510875133 -74.19253795016222) bank72008 +72009 POINT(40.67397312900128 -74.80494887742316) bank72009 +72010 POINT(40.7930196105809 -73.97669498179042) bank72010 +72011 POINT(41.32327922930714 -74.10341534326092) bank72011 +72012 POINT(40.98382462068374 -73.19115103280762) bank72012 +72013 POINT(40.042471555711764 -74.2738760217958) bank72013 +72014 POINT(39.81980893632096 -74.30159811311782) bank72014 +72015 POINT(39.93508381008466 -73.22050367474623) bank72015 +72016 POINT(41.46518045811627 -73.53821013002002) bank72016 +72017 POINT(39.72033958698837 -73.41017196023998) bank72017 +72018 POINT(40.5935156108386 -73.12626253084701) bank72018 +72019 POINT(39.99281082128839 -74.40770087937794) bank72019 +72020 POINT(40.68041706215227 -73.8574217315549) bank72020 +72021 POINT(39.774614674498004 -73.79093975590287) bank72021 +72022 POINT(41.609775102031655 -73.69750109991398) bank72022 +72023 POINT(40.59711520620538 -73.96797739747382) bank72023 +72024 POINT(40.376465333014956 -74.12299498287796) bank72024 +72025 POINT(41.30179910851368 -74.56867377075382) bank72025 +72026 POINT(40.89659886987577 -73.57550496229707) bank72026 +72027 POINT(41.2513281849669 -74.4124300955374) bank72027 +72028 POINT(41.040400879004316 -73.61995860859597) bank72028 +72029 POINT(40.51353496450658 -73.9669783380663) bank72029 +72030 POINT(40.004867295701025 -73.75177012702274) bank72030 +72031 POINT(41.00216320449699 -73.3209485785907) bank72031 +72032 POINT(41.58709281325816 -73.98574173365266) bank72032 +72033 POINT(39.8600534643451 -73.61349403738699) bank72033 +72034 POINT(40.884483021097424 -73.84982765389248) bank72034 +72035 POINT(40.91269761996275 -74.6432556538932) bank72035 +72036 POINT(40.65855760346249 -73.75230514066078) bank72036 +72037 POINT(39.88974950178737 -74.82269925261394) bank72037 +72038 POINT(40.70442957872261 -73.29412260449892) bank72038 +72039 POINT(41.467929693011456 -74.0114201845446) bank72039 +72040 POINT(40.40634059485149 -73.82859966929844) bank72040 +72041 POINT(40.607043096735715 -74.7135371463383) bank72041 +72042 POINT(41.15695071097534 -74.24028566692805) bank72042 +72043 POINT(40.42380723333362 -73.27417731397689) bank72043 +72044 POINT(40.103416301298 -73.42601284551984) bank72044 +72045 POINT(40.99823939306547 -73.17987256912936) bank72045 +72046 POINT(40.9244259538744 -73.03235485684014) bank72046 +72047 POINT(40.36806801959418 -73.88484571582565) bank72047 +72048 POINT(41.08642568115418 -74.5606522281695) bank72048 +72049 POINT(40.15324624736185 -74.27344778451415) bank72049 +72050 POINT(40.993589375039754 -73.12074378797618) bank72050 +72051 POINT(40.23613066982657 -74.56076673448071) bank72051 +72052 POINT(40.9139887343928 -73.45577852665483) bank72052 +72053 POINT(41.22499122312729 -73.72214308569725) bank72053 +72054 POINT(39.891441329844284 -73.86145649557733) bank72054 +72055 POINT(39.775205260736236 -74.25171022169441) bank72055 +72056 POINT(41.3281892075504 -73.9906872960501) bank72056 +72057 POINT(40.39493619714988 -74.59391076187805) bank72057 +72058 POINT(40.02024142638616 -74.68202632899902) bank72058 +72059 POINT(41.114175704633446 -73.28081752360984) bank72059 +72060 POINT(39.908454904879314 -74.78964500162766) bank72060 +72061 POINT(40.09413449923481 -73.55439775582235) bank72061 +72062 POINT(39.92298916793683 -73.72006755900749) bank72062 +72063 POINT(41.65543226344859 -73.73278776370474) bank72063 +72064 POINT(40.812591895025214 -73.86165396605173) bank72064 +72065 POINT(40.18882338423049 -74.12827510429821) bank72065 +72066 POINT(40.08003594788028 -73.30893571893202) bank72066 +72067 POINT(40.42417446224196 -73.3508642349278) bank72067 +72068 POINT(40.556059433772226 -73.6984286586105) bank72068 +72069 POINT(41.5487832361863 -74.34420704118841) bank72069 +72070 POINT(41.41076093241837 -74.01536506421185) bank72070 +72071 POINT(40.23305209366162 -74.6162472240017) bank72071 +72072 POINT(41.127417295170815 -73.99753059252335) bank72072 +72073 POINT(41.098976351299676 -74.02878365457457) bank72073 +72074 POINT(39.77821489114835 -73.9369617977547) bank72074 +72075 POINT(40.340468222655 -73.88332771783287) bank72075 +72076 POINT(41.365175635897465 -73.32531207457035) bank72076 +72077 POINT(40.714581133825625 -73.1396394714432) bank72077 +72078 POINT(40.51425111995813 -74.22911871074366) bank72078 +72079 POINT(40.40107235592737 -73.5449932002899) bank72079 +72080 POINT(39.74653730718799 -73.95718869638185) bank72080 +72081 POINT(40.17212642436542 -74.734993496227) bank72081 +72082 POINT(41.07410146941101 -73.90119896177617) bank72082 +72083 POINT(41.34767264716427 -74.60320664634543) bank72083 +72084 POINT(41.434561794134666 -73.52882069961346) bank72084 +72085 POINT(40.059952857490266 -74.5530882071076) bank72085 +72086 POINT(41.42708826173987 -73.45079922661941) bank72086 +72087 POINT(39.842584511925985 -73.95335096302884) bank72087 +72088 POINT(41.122684216836106 -74.97742508428159) bank72088 +72089 POINT(40.375830858301306 -73.06380941330097) bank72089 +72090 POINT(40.88408479394873 -73.29465588575849) bank72090 +72091 POINT(41.676829369207255 -74.80487392935662) bank72091 +72092 POINT(41.66136633717187 -74.08229014464399) bank72092 +72093 POINT(41.51429557126662 -74.90323953685227) bank72093 +72094 POINT(40.43668488689361 -74.0061597341249) bank72094 +72095 POINT(40.345709918005355 -73.8149550329567) bank72095 +72096 POINT(40.52728565020873 -74.17801124850756) bank72096 +72097 POINT(40.176752318843555 -73.01274217666418) bank72097 +72098 POINT(40.008420526200034 -74.59345014089816) bank72098 +72099 POINT(41.61092896070444 -73.95019086313029) bank72099 +72100 POINT(40.61229334361445 -73.69104992702542) bank72100 +72101 POINT(41.32025813882452 -73.40103774287732) bank72101 +72102 POINT(40.34335963740834 -73.08956174844944) bank72102 +72103 POINT(41.01826898702802 -73.89614036314553) bank72103 +72104 POINT(41.39578364929824 -74.44201234314609) bank72104 +72105 POINT(40.753526896459945 -73.06157556198038) bank72105 +72106 POINT(40.436324261647016 -73.01147231772961) bank72106 +72107 POINT(41.709657377652015 -74.82467771420933) bank72107 +72108 POINT(41.38683905566019 -74.7653235962532) bank72108 +72109 POINT(40.54666705677367 -73.74018785518369) bank72109 +72110 POINT(41.26952759069196 -74.93419247896705) bank72110 +72111 POINT(40.29258284756407 -73.26469860880134) bank72111 +72112 POINT(40.03684870255897 -74.72269122373474) bank72112 +72113 POINT(41.406184376484106 -74.19478720851464) bank72113 +72114 POINT(40.12202438258167 -74.6956765027767) bank72114 +72115 POINT(40.408507303207145 -74.27649560101881) bank72115 +72116 POINT(39.893820506024404 -74.44538832882652) bank72116 +72117 POINT(41.1050756466385 -74.64714748031334) bank72117 +72118 POINT(39.796224306156766 -73.03017284316805) bank72118 +72119 POINT(40.0383403819784 -74.5475640863231) bank72119 +72120 POINT(41.45118963678092 -74.57457505817055) bank72120 +72121 POINT(39.80092078212494 -74.61795281927773) bank72121 +72122 POINT(40.476211745892556 -74.27929959367665) bank72122 +72123 POINT(41.43386720070293 -74.80108061141199) bank72123 +72124 POINT(41.35056963739364 -74.83647334294771) bank72124 +72125 POINT(40.81243597050065 -74.57201966597371) bank72125 +72126 POINT(40.70197197175828 -74.722822727557) bank72126 +72127 POINT(40.25608830293767 -74.33906351316571) bank72127 +72128 POINT(41.6888386467308 -73.7262969793894) bank72128 +72129 POINT(40.425378221241324 -73.5730732449187) bank72129 +72130 POINT(40.64457517691468 -74.44016022425195) bank72130 +72131 POINT(41.00498372237015 -74.86351287838414) bank72131 +72132 POINT(40.25230434526623 -73.09101965708065) bank72132 +72133 POINT(41.14485227042958 -73.36548015330054) bank72133 +72134 POINT(40.95005530217633 -74.51595584650514) bank72134 +72135 POINT(39.87130110513225 -73.6310058435427) bank72135 +72136 POINT(40.080395143861686 -73.10507226362792) bank72136 +72137 POINT(40.8158657393655 -74.29910622982224) bank72137 +72138 POINT(40.68766428792105 -73.8394160040923) bank72138 +72139 POINT(40.753957926459606 -74.5721488661606) bank72139 +72140 POINT(39.981437296417404 -74.01834262178546) bank72140 +72141 POINT(41.191779007667606 -74.13363302625325) bank72141 +72142 POINT(41.588431120534636 -74.01624171768009) bank72142 +72143 POINT(41.15174284703472 -74.07038188156415) bank72143 +72144 POINT(40.51714630972717 -73.25047261241768) bank72144 +72145 POINT(41.330825034225626 -73.20046560706058) bank72145 +72146 POINT(39.82888906114995 -73.49281790649307) bank72146 +72147 POINT(40.64826100722496 -73.97666588184066) bank72147 +72148 POINT(40.65715715948854 -73.27169309181083) bank72148 +72149 POINT(41.32643835073078 -73.79502359494668) bank72149 +72150 POINT(41.58551664564439 -73.0417685104412) bank72150 +72151 POINT(41.690954349575975 -73.97286444139692) bank72151 +72152 POINT(39.99187984969096 -73.2555500758461) bank72152 +72153 POINT(40.195414983866456 -74.96395656005737) bank72153 +72154 POINT(40.77514993262723 -74.35617536444998) bank72154 +72155 POINT(40.269264185572396 -73.45275272927802) bank72155 +72156 POINT(41.04886163988877 -73.31944834213617) bank72156 +72157 POINT(40.88334927568375 -74.87621327713757) bank72157 +72158 POINT(41.27582588014193 -74.87217798026657) bank72158 +72159 POINT(40.75894136055261 -74.76585493984035) bank72159 +72160 POINT(40.894425257309294 -73.23051633472514) bank72160 +72161 POINT(40.11207186477186 -73.71516448537275) bank72161 +72162 POINT(41.267511317780006 -73.01562096874376) bank72162 +72163 POINT(40.791400529518974 -73.57733654311491) bank72163 +72164 POINT(41.27122637026457 -73.63964562874627) bank72164 +72165 POINT(41.68594129401747 -73.29069732699874) bank72165 +72166 POINT(40.10049188897594 -73.89569920511235) bank72166 +72167 POINT(41.64559285831429 -73.70411101543343) bank72167 +72168 POINT(41.13652641307919 -73.96461704728267) bank72168 +72169 POINT(40.15159130399742 -73.09788436611466) bank72169 +72170 POINT(40.94567958087679 -74.02300308074732) bank72170 +72171 POINT(40.512604975322716 -73.8981256715235) bank72171 +72172 POINT(41.10825076381992 -73.63569751895) bank72172 +72173 POINT(39.868983922712445 -74.35157382612852) bank72173 +72174 POINT(40.11909602673545 -74.50878528168828) bank72174 +72175 POINT(41.25395411841487 -74.98038411748769) bank72175 +72176 POINT(40.97236260800574 -73.73474785309533) bank72176 +72177 POINT(40.76427472876787 -74.65311029719129) bank72177 +72178 POINT(39.96663605340933 -74.10516428367241) bank72178 +72179 POINT(40.29886496592797 -74.52024425955082) bank72179 +72180 POINT(40.63516786864196 -73.33372362896515) bank72180 +72181 POINT(40.72860376219808 -74.70607544117428) bank72181 +72182 POINT(40.513247951895906 -73.31182015552017) bank72182 +72183 POINT(40.75082502419705 -74.60446738310861) bank72183 +72184 POINT(39.977823681665 -74.85988673127042) bank72184 +72185 POINT(41.274729341788515 -73.68504513432072) bank72185 +72186 POINT(39.830848228448176 -73.55644425458115) bank72186 +72187 POINT(41.38228229070264 -73.62644406033607) bank72187 +72188 POINT(40.55914199824427 -73.10897295151518) bank72188 +72189 POINT(40.752097302427515 -74.70106315821984) bank72189 +72190 POINT(41.37296659647045 -74.47106249912348) bank72190 +72191 POINT(39.91979032277763 -73.51363317000695) bank72191 +72192 POINT(40.047455404622596 -73.25201677076252) bank72192 +72193 POINT(41.480659757720524 -73.80831230885654) bank72193 +72194 POINT(39.99314970161094 -73.94638784324053) bank72194 +72195 POINT(40.10746430850739 -74.93100443912118) bank72195 +72196 POINT(40.71356906041387 -74.36962056022931) bank72196 +72197 POINT(41.67840414825642 -74.98858462735953) bank72197 +72198 POINT(41.61300194287315 -74.99378258329085) bank72198 +72199 POINT(40.58953652672775 -73.09359765833447) bank72199 +72200 POINT(39.974392576866656 -74.21499542537808) bank72200 +72201 POINT(40.637524995596785 -73.40610990117898) bank72201 +72202 POINT(40.83559752282688 -73.49906739989973) bank72202 +72203 POINT(40.7597857131839 -73.26926329947604) bank72203 +72204 POINT(39.99179227081526 -73.53132856652101) bank72204 +72205 POINT(41.57736371403434 -73.32968276246338) bank72205 +72206 POINT(39.74777667026433 -73.45280019716392) bank72206 +72207 POINT(40.591602508089835 -73.22127545433806) bank72207 +72208 POINT(41.39730079413257 -73.9110107269604) bank72208 +72209 POINT(39.84152107131294 -73.05957181814175) bank72209 +72210 POINT(40.43039453666037 -74.59770387917825) bank72210 +72211 POINT(41.70495303821832 -73.72541207851204) bank72211 +72212 POINT(40.27989366598852 -73.4482283994925) bank72212 +72213 POINT(40.496659701737435 -73.03040679548218) bank72213 +72214 POINT(40.181530065929984 -74.44794939843912) bank72214 +72215 POINT(40.23573543736171 -73.26185000212745) bank72215 +72216 POINT(41.419144695641386 -74.35599325594347) bank72216 +72217 POINT(41.263310673343554 -73.75174833037451) bank72217 +72218 POINT(40.51806663152735 -74.6440795863392) bank72218 +72219 POINT(41.48184089873349 -74.49545242101892) bank72219 +72220 POINT(40.64660518037689 -73.76155455620825) bank72220 +72221 POINT(41.07040667300501 -73.38016241114997) bank72221 +72222 POINT(41.63104430648167 -73.85188605006606) bank72222 +72223 POINT(39.7621628026483 -74.47193321929734) bank72223 +72224 POINT(40.64856086153359 -74.76279848350924) bank72224 +72225 POINT(40.906096081406275 -75.00375495031898) bank72225 +72226 POINT(40.34564152891679 -74.36252085082391) bank72226 +72227 POINT(41.27610499884693 -74.41364491758972) bank72227 +72228 POINT(41.484263865620704 -74.87704511325263) bank72228 +72229 POINT(40.15652399921533 -74.91060459120897) bank72229 +72230 POINT(41.01683616575744 -73.161594129212) bank72230 +72231 POINT(40.029102452877986 -73.15408744300159) bank72231 +72232 POINT(39.79781098964886 -73.71182549418647) bank72232 +72233 POINT(41.12597732240357 -74.36650114863194) bank72233 +72234 POINT(41.187256222245445 -73.71909995298593) bank72234 +72235 POINT(41.08351203813887 -74.04338253423262) bank72235 +72236 POINT(41.68226240623583 -73.6821163925948) bank72236 +72237 POINT(40.007672448836196 -73.53104124759255) bank72237 +72238 POINT(40.91600947385141 -74.45988590372582) bank72238 +72239 POINT(41.287196611800496 -74.55852904107651) bank72239 +72240 POINT(41.46694452692893 -74.32364707881835) bank72240 +72241 POINT(40.28571833264845 -74.21981311910011) bank72241 +72242 POINT(40.7144315786742 -73.56084700158264) bank72242 +72243 POINT(41.60227636719918 -73.01572919097546) bank72243 +72244 POINT(40.95934291562087 -74.92254630044647) bank72244 +72245 POINT(40.09411915247852 -73.89382183552584) bank72245 +72246 POINT(41.101186375101726 -74.53528303616933) bank72246 +72247 POINT(40.07790585824829 -73.51706209545229) bank72247 +72248 POINT(39.92336971380944 -73.62099459695595) bank72248 +72249 POINT(40.05404939894783 -74.38201369889248) bank72249 +72250 POINT(41.22076539935346 -74.78471089563186) bank72250 +72251 POINT(41.708861396632045 -73.9991630091157) bank72251 +72252 POINT(39.98638580726243 -74.78837063780298) bank72252 +72253 POINT(40.68808105140213 -74.7314853455302) bank72253 +72254 POINT(41.104290024608524 -74.10383569909752) bank72254 +72255 POINT(41.08151773388556 -74.97045692633868) bank72255 +72256 POINT(40.76104077718137 -74.2122294815976) bank72256 +72257 POINT(41.327534505027685 -73.88863595645637) bank72257 +72258 POINT(40.26891949328245 -73.08975110990215) bank72258 +72259 POINT(40.02229714704143 -74.83785880682095) bank72259 +72260 POINT(41.63154746781247 -73.56101713638786) bank72260 +72261 POINT(40.21200235514732 -73.68832484786466) bank72261 +72262 POINT(40.884907359845855 -73.62924483380198) bank72262 +72263 POINT(40.405344743286676 -74.11698096537123) bank72263 +72264 POINT(40.66435900962241 -73.73059141877495) bank72264 +72265 POINT(40.15064558105087 -74.78414387622344) bank72265 +72266 POINT(40.49815696439609 -74.72459055243903) bank72266 +72267 POINT(40.17430669762985 -74.51036465237627) bank72267 +72268 POINT(41.266284259689705 -73.59052202212368) bank72268 +72269 POINT(41.00885958560556 -73.93107081188873) bank72269 +72270 POINT(40.84265329155135 -73.21858356826837) bank72270 +72271 POINT(39.75241846967845 -74.67651522052887) bank72271 +72272 POINT(39.75347368611155 -73.90758248268456) bank72272 +72273 POINT(41.658031753966696 -73.80876478243184) bank72273 +72274 POINT(41.06306472168275 -73.24409336842066) bank72274 +72275 POINT(40.03089567638013 -73.0488458612162) bank72275 +72276 POINT(40.19710171876513 -74.72436609661072) bank72276 +72277 POINT(41.50680758772826 -73.31781337762023) bank72277 +72278 POINT(41.59540376018742 -74.28044844348798) bank72278 +72279 POINT(39.95912166070871 -73.15004419297102) bank72279 +72280 POINT(39.94472908530352 -74.44939882589146) bank72280 +72281 POINT(40.87477721552682 -73.42892555611093) bank72281 +72282 POINT(41.62944835643676 -74.91632528995552) bank72282 +72283 POINT(40.182580927077815 -74.73595839883745) bank72283 +72284 POINT(41.70717451468675 -74.48180638658883) bank72284 +72285 POINT(40.63675093633695 -74.43648022513192) bank72285 +72286 POINT(41.3471046710395 -73.98825426707879) bank72286 +72287 POINT(40.76749408825042 -73.60477347642288) bank72287 +72288 POINT(40.54073250821749 -73.19669431277555) bank72288 +72289 POINT(40.33499215755602 -74.48485879118756) bank72289 +72290 POINT(40.51820061922677 -74.04668368892561) bank72290 +72291 POINT(40.34377471965949 -74.08098042515768) bank72291 +72292 POINT(40.88980491178412 -73.23585882387083) bank72292 +72293 POINT(40.75661273365712 -75.00100987224361) bank72293 +72294 POINT(41.57631747095701 -74.45407167620853) bank72294 +72295 POINT(40.843718776780335 -74.83458253318038) bank72295 +72296 POINT(41.46129547007903 -73.18520763587239) bank72296 +72297 POINT(41.37536712405693 -74.31047469577717) bank72297 +72298 POINT(40.41957293038866 -73.13065126556786) bank72298 +72299 POINT(40.29224229927994 -74.87414786420335) bank72299 +72300 POINT(41.70799123716704 -74.82233619359714) bank72300 +72301 POINT(40.04633977760547 -74.47864090743558) bank72301 +72302 POINT(40.92320165939219 -73.14592746618627) bank72302 +72303 POINT(41.329616096182484 -73.44127172436447) bank72303 +72304 POINT(39.98855595578742 -74.23317443076071) bank72304 +72305 POINT(40.532889540500555 -74.79611103679765) bank72305 +72306 POINT(40.340582734768894 -73.80377359926862) bank72306 +72307 POINT(41.579969100363996 -73.40162744691416) bank72307 +72308 POINT(40.3697274967634 -73.11987918619813) bank72308 +72309 POINT(40.50323245570993 -73.15531612877291) bank72309 +72310 POINT(40.522555914525746 -74.0818056136666) bank72310 +72311 POINT(40.269863238943586 -73.0371950669828) bank72311 +72312 POINT(41.60424664231787 -73.73149622321718) bank72312 +72313 POINT(41.16474132236716 -73.88504303136807) bank72313 +72314 POINT(41.25312050484158 -74.2940723353537) bank72314 +72315 POINT(39.73980545377328 -73.5129938097479) bank72315 +72316 POINT(40.251110402193646 -73.54684837768497) bank72316 +72317 POINT(39.85894597241783 -74.65883194594066) bank72317 +72318 POINT(40.629714464782126 -74.17006905025173) bank72318 +72319 POINT(40.35472060493573 -74.76036823655582) bank72319 +72320 POINT(40.07644038886636 -73.24425049144212) bank72320 +72321 POINT(40.67390363802738 -73.07859771442388) bank72321 +72322 POINT(41.01156005360631 -73.36482252656953) bank72322 +72323 POINT(41.45559644601768 -73.41081798285765) bank72323 +72324 POINT(39.97123616060522 -73.54782446440848) bank72324 +72325 POINT(40.4854112803021 -74.32338044037228) bank72325 +72326 POINT(40.97849090620096 -74.63660956007296) bank72326 +72327 POINT(41.14002780873017 -74.44564268870114) bank72327 +72328 POINT(41.13324057951489 -73.42014717387613) bank72328 +72329 POINT(41.058910739452116 -73.87371309731945) bank72329 +72330 POINT(40.83756464166966 -73.85917325227085) bank72330 +72331 POINT(40.935200359467935 -74.50087755150157) bank72331 +72332 POINT(39.9604522786652 -74.48644588204861) bank72332 +72333 POINT(41.25966695979153 -74.17187277184851) bank72333 +72334 POINT(41.54929800546609 -74.34660017863492) bank72334 +72335 POINT(40.68391200825933 -74.2234733678703) bank72335 +72336 POINT(39.74037749583111 -74.69353493802782) bank72336 +72337 POINT(40.12795504045868 -74.030827483987) bank72337 +72338 POINT(40.64559174050056 -73.03237803441546) bank72338 +72339 POINT(40.59760535593644 -74.52843479936635) bank72339 +72340 POINT(40.748143025386206 -74.26277426106071) bank72340 +72341 POINT(40.75003937635042 -74.65360081751241) bank72341 +72342 POINT(41.0724830770849 -74.64121522758998) bank72342 +72343 POINT(40.151928060167 -73.42464698834199) bank72343 +72344 POINT(40.80511123146248 -74.20706432143331) bank72344 +72345 POINT(40.31299637713175 -73.68353699387879) bank72345 +72346 POINT(39.72754510661634 -73.6917939655297) bank72346 +72347 POINT(41.29168017637345 -73.88373720162328) bank72347 +72348 POINT(40.54832561327656 -73.52869308594273) bank72348 +72349 POINT(41.137053747140705 -73.6261182088901) bank72349 +72350 POINT(41.49933470812792 -74.96471305903972) bank72350 +72351 POINT(39.88513627075533 -73.71809102861599) bank72351 +72352 POINT(39.78985004818876 -73.89381316616434) bank72352 +72353 POINT(40.0366903119729 -74.54777786091901) bank72353 +72354 POINT(40.26387757663782 -73.79332005311431) bank72354 +72355 POINT(40.80519636748674 -73.68203711572276) bank72355 +72356 POINT(40.78504610210863 -73.65756359851336) bank72356 +72357 POINT(40.365462830346964 -73.19129322020521) bank72357 +72358 POINT(40.507884199698545 -74.87485117744924) bank72358 +72359 POINT(41.56715357903033 -73.56491302426153) bank72359 +72360 POINT(40.3840163126561 -73.86770136066305) bank72360 +72361 POINT(40.802553259578545 -73.61094523225167) bank72361 +72362 POINT(40.81730886277667 -73.09727895615714) bank72362 +72363 POINT(40.33205635216702 -74.04630541558858) bank72363 +72364 POINT(40.39752416900333 -73.87274718173327) bank72364 +72365 POINT(40.28065559164529 -74.63087991700218) bank72365 +72366 POINT(40.334644556479084 -73.75838006954964) bank72366 +72367 POINT(41.581432630376604 -73.91187551406306) bank72367 +72368 POINT(40.660492488705 -73.70291069123066) bank72368 +72369 POINT(40.42578750810968 -73.74544511428124) bank72369 +72370 POINT(40.58988744030366 -73.18123639176964) bank72370 +72371 POINT(40.20619613930031 -74.99612156908) bank72371 +72372 POINT(41.566807107541045 -73.25827315002707) bank72372 +72373 POINT(40.6828286988624 -74.37063472997204) bank72373 +72374 POINT(40.228929393813985 -73.19407557198859) bank72374 +72375 POINT(40.68525804071198 -74.03090013793432) bank72375 +72376 POINT(41.27106460483099 -74.94891518541361) bank72376 +72377 POINT(41.48002937252468 -73.72330208263129) bank72377 +72378 POINT(41.68413677266004 -74.96819615789295) bank72378 +72379 POINT(39.83615547124064 -74.90121837373754) bank72379 +72380 POINT(41.57610754978099 -74.1302935232481) bank72380 +72381 POINT(40.67350332009202 -74.4456106612701) bank72381 +72382 POINT(40.80572534681296 -73.90834466940565) bank72382 +72383 POINT(41.03628911135026 -74.93487933482473) bank72383 +72384 POINT(40.18037630331253 -74.9706929383348) bank72384 +72385 POINT(41.62134387176855 -74.00750373032892) bank72385 +72386 POINT(40.88116302863511 -73.91827698134313) bank72386 +72387 POINT(39.93246169664603 -74.16948923466305) bank72387 +72388 POINT(41.68614395579933 -73.79540751461435) bank72388 +72389 POINT(41.23632663014189 -73.66684238129432) bank72389 +72390 POINT(41.26807872425176 -74.5315372781146) bank72390 +72391 POINT(39.83329340612235 -73.66967731596519) bank72391 +72392 POINT(40.04759569035906 -74.8457406297391) bank72392 +72393 POINT(41.59094077348753 -73.35346863896173) bank72393 +72394 POINT(39.876823209200005 -74.49959381220222) bank72394 +72395 POINT(40.384400494461445 -74.45075780466166) bank72395 +72396 POINT(39.811676826954994 -74.41939977393254) bank72396 +72397 POINT(41.43709867340312 -74.19965216802338) bank72397 +72398 POINT(41.49782078384007 -74.49510129102184) bank72398 +72399 POINT(41.24302392500767 -73.97642399324025) bank72399 +72400 POINT(39.73377532628536 -74.30221068570974) bank72400 +72401 POINT(40.004882634323636 -74.02795498699784) bank72401 +72402 POINT(41.37543904569224 -74.34743430289625) bank72402 +72403 POINT(41.544333909099436 -74.38521425721686) bank72403 +72404 POINT(40.503927093554495 -74.4455419331108) bank72404 +72405 POINT(39.85065113876986 -74.83194643156739) bank72405 +72406 POINT(41.07395292017699 -73.21606873712007) bank72406 +72407 POINT(41.27443711882867 -73.41362402954725) bank72407 +72408 POINT(40.93660180840966 -73.86639489951821) bank72408 +72409 POINT(40.22481069979765 -74.22270058691032) bank72409 +72410 POINT(39.73543050347685 -73.68917846918828) bank72410 +72411 POINT(40.40043279679999 -74.86734466908868) bank72411 +72412 POINT(39.90931273366565 -73.09811361724923) bank72412 +72413 POINT(41.593895475209756 -73.5088946160093) bank72413 +72414 POINT(41.06774368834062 -74.17101279087724) bank72414 +72415 POINT(41.68815028944612 -73.04976995526171) bank72415 +72416 POINT(39.92355583878793 -74.84475908066767) bank72416 +72417 POINT(41.46137104530837 -74.94620328498186) bank72417 +72418 POINT(40.08928317660199 -74.26058827652801) bank72418 +72419 POINT(40.60743301807048 -73.09608269256519) bank72419 +72420 POINT(41.25836385081832 -73.41104572137348) bank72420 +72421 POINT(41.07626452673364 -74.98945045882293) bank72421 +72422 POINT(41.34108114962974 -74.65961343528484) bank72422 +72423 POINT(39.99199663645431 -74.72091772183333) bank72423 +72424 POINT(40.43286544205792 -73.29022324063824) bank72424 +72425 POINT(41.03568835561274 -73.92041461179956) bank72425 +72426 POINT(39.87179423779275 -73.73133997765832) bank72426 +72427 POINT(39.75301855456301 -73.18102559065363) bank72427 +72428 POINT(41.52849883664448 -74.66056625772579) bank72428 +72429 POINT(40.43788347315439 -73.3261725724931) bank72429 +72430 POINT(39.8131841633423 -73.99969751662051) bank72430 +72431 POINT(41.588447703490104 -73.5903691573247) bank72431 +72432 POINT(39.719160087945994 -73.40273098695343) bank72432 +72433 POINT(39.883474510512976 -73.30519014379539) bank72433 +72434 POINT(41.699836941273574 -74.22019512010253) bank72434 +72435 POINT(41.38065266616313 -74.91112313340362) bank72435 +72436 POINT(41.26720644954565 -74.08875941118265) bank72436 +72437 POINT(40.82377749795824 -73.61395450884187) bank72437 +72438 POINT(41.35053993910612 -74.49614435507515) bank72438 +72439 POINT(40.33485657506847 -74.05446996101907) bank72439 +72440 POINT(40.24832181190254 -73.34673267323626) bank72440 +72441 POINT(40.82093692521602 -73.59170173987165) bank72441 +72442 POINT(39.822938594011376 -73.58574645525017) bank72442 +72443 POINT(41.28559286457447 -74.09151851154733) bank72443 +72444 POINT(40.556673866233886 -74.37458458079016) bank72444 +72445 POINT(40.02044562012215 -74.3260595802628) bank72445 +72446 POINT(39.865655116967396 -74.92293264773012) bank72446 +72447 POINT(39.92046662342472 -73.75137623501104) bank72447 +72448 POINT(41.66013359527064 -73.93270903330952) bank72448 +72449 POINT(40.484371493399344 -74.42689375064637) bank72449 +72450 POINT(39.826308657179766 -73.5099946758365) bank72450 +72451 POINT(40.070455517224595 -73.73953662401372) bank72451 +72452 POINT(39.82259459163915 -73.82662655286576) bank72452 +72453 POINT(41.35757640285608 -73.3953375123586) bank72453 +72454 POINT(41.34879203001187 -74.7435364351131) bank72454 +72455 POINT(40.39540676190037 -74.23916025532897) bank72455 +72456 POINT(41.04885140686289 -74.94087469726553) bank72456 +72457 POINT(41.2608094707151 -74.60555025251944) bank72457 +72458 POINT(39.93505907004854 -73.47500800952221) bank72458 +72459 POINT(41.26903326216224 -73.5795189727462) bank72459 +72460 POINT(41.39880667999799 -74.19414181941215) bank72460 +72461 POINT(41.473482702327125 -74.45652841986605) bank72461 +72462 POINT(39.827451664269084 -73.34040748831531) bank72462 +72463 POINT(40.935793112745415 -74.36274597930156) bank72463 +72464 POINT(40.290086899550516 -74.89941573979789) bank72464 +72465 POINT(40.88350926525411 -74.75930196419732) bank72465 +72466 POINT(40.0623950354423 -73.46866893324436) bank72466 +72467 POINT(41.655527289487274 -74.63651047677682) bank72467 +72468 POINT(40.03382357817711 -74.79703869658726) bank72468 +72469 POINT(40.53558720290586 -73.07094656416824) bank72469 +72470 POINT(40.058265337348345 -73.5739784153305) bank72470 +72471 POINT(39.811614217100264 -73.33582187863091) bank72471 +72472 POINT(40.644964109196046 -73.79268222498486) bank72472 +72473 POINT(39.7865301987649 -73.66367215978802) bank72473 +72474 POINT(40.87730342927897 -73.46566438421655) bank72474 +72475 POINT(41.20596387626035 -74.20064789976134) bank72475 +72476 POINT(40.42130267662411 -74.1861542273856) bank72476 +72477 POINT(41.03572621145358 -74.69110655399739) bank72477 +72478 POINT(41.5672527512787 -73.33002966295243) bank72478 +72479 POINT(40.38056938269465 -74.76673515682882) bank72479 +72480 POINT(41.02907848221775 -73.78837573695672) bank72480 +72481 POINT(41.15203188185462 -74.04584698719492) bank72481 +72482 POINT(40.16726674224792 -74.61404162912181) bank72482 +72483 POINT(40.601266303620356 -74.43060964115669) bank72483 +72484 POINT(40.55854033651193 -74.26158042902996) bank72484 +72485 POINT(41.06329545101418 -74.27141118120662) bank72485 +72486 POINT(39.93574230957365 -73.73965204516301) bank72486 +72487 POINT(41.56145700008127 -74.71953107680295) bank72487 +72488 POINT(41.39235519339343 -74.85927162502362) bank72488 +72489 POINT(40.77845540611464 -74.50707830577943) bank72489 +72490 POINT(40.52748882961572 -74.081281410462) bank72490 +72491 POINT(41.534332287049125 -73.29371146204647) bank72491 +72492 POINT(39.96923340856159 -74.64544412749024) bank72492 +72493 POINT(40.993497694248816 -75.00198224990295) bank72493 +72494 POINT(39.97543783430218 -74.49325739921738) bank72494 +72495 POINT(41.57102791527002 -74.72738969561637) bank72495 +72496 POINT(39.826972656436205 -73.62328385091998) bank72496 +72497 POINT(40.862003529774945 -74.27599563117028) bank72497 +72498 POINT(40.365074039194134 -74.43311974841407) bank72498 +72499 POINT(39.98069777911336 -73.1463772862654) bank72499 +72500 POINT(40.08870160063047 -73.47037558597722) bank72500 +72501 POINT(40.44262828980079 -74.80222664771284) bank72501 +72502 POINT(41.35359353629203 -74.51611222192474) bank72502 +72503 POINT(39.89359565830652 -74.99971003805946) bank72503 +72504 POINT(41.69444140534615 -74.99693290271445) bank72504 +72505 POINT(40.400758312410936 -73.1366615715402) bank72505 +72506 POINT(39.896681187203335 -74.77129497505001) bank72506 +72507 POINT(41.22891317807528 -73.56819994728396) bank72507 +72508 POINT(41.31273661292599 -74.11488910540798) bank72508 +72509 POINT(41.33219059360874 -73.7906304729212) bank72509 +72510 POINT(41.03763234638598 -74.22904983064555) bank72510 +72511 POINT(41.69870572792117 -74.5846559618275) bank72511 +72512 POINT(40.29403270664206 -74.55877972401463) bank72512 +72513 POINT(41.57979426102053 -73.03739369661551) bank72513 +72514 POINT(41.42848695884994 -73.29935828004339) bank72514 +72515 POINT(41.38393169980262 -74.34483889977076) bank72515 +72516 POINT(39.72339454765242 -74.49032540969397) bank72516 +72517 POINT(40.72781190892121 -73.8156910591615) bank72517 +72518 POINT(40.39126085186194 -73.58648643630524) bank72518 +72519 POINT(41.05672443958588 -74.10654458524819) bank72519 +72520 POINT(40.247566710724044 -73.08957065592563) bank72520 +72521 POINT(40.597351314438775 -73.85997886426586) bank72521 +72522 POINT(41.47273479840815 -74.74168124058026) bank72522 +72523 POINT(40.40455444807289 -74.82104942302021) bank72523 +72524 POINT(39.73012359169714 -73.60539998523826) bank72524 +72525 POINT(39.87817137465341 -74.39867481935919) bank72525 +72526 POINT(40.10506347965886 -74.97157623994605) bank72526 +72527 POINT(41.37459590162157 -74.69766224934736) bank72527 +72528 POINT(40.27965874771673 -73.29318329738975) bank72528 +72529 POINT(40.205607643681816 -74.93661321435927) bank72529 +72530 POINT(41.14214965637098 -74.00213654545954) bank72530 +72531 POINT(41.577802268714976 -73.58313189081002) bank72531 +72532 POINT(40.86083872706965 -73.13020660627264) bank72532 +72533 POINT(41.43974088435211 -74.02083197983039) bank72533 +72534 POINT(40.58950960658032 -73.69939865716827) bank72534 +72535 POINT(40.751427431321794 -73.80556606939784) bank72535 +72536 POINT(41.615238821197934 -74.33145904350572) bank72536 +72537 POINT(41.092318920123525 -74.26961946276519) bank72537 +72538 POINT(41.415512002203975 -74.42307265487017) bank72538 +72539 POINT(40.72858168998412 -74.61686459703655) bank72539 +72540 POINT(40.4588495542955 -74.33905273711702) bank72540 +72541 POINT(40.00785271096916 -73.58477649794766) bank72541 +72542 POINT(40.271162313436975 -73.77406262285247) bank72542 +72543 POINT(40.684501394395525 -73.86234842732242) bank72543 +72544 POINT(41.58975130945298 -74.61507227466926) bank72544 +72545 POINT(40.081356049373206 -74.860185152529) bank72545 +72546 POINT(40.41137056754587 -73.38598902186865) bank72546 +72547 POINT(41.45797577994975 -74.82082666354424) bank72547 +72548 POINT(40.03502305553983 -73.4155024212933) bank72548 +72549 POINT(40.90598582682943 -74.17379834587186) bank72549 +72550 POINT(41.259965247902116 -74.56206671632322) bank72550 +72551 POINT(39.92676193713654 -73.01333793835234) bank72551 +72552 POINT(40.38567150359495 -74.22631072392723) bank72552 +72553 POINT(40.03375676456193 -73.63245554442241) bank72553 +72554 POINT(41.091336928994565 -74.95063204609232) bank72554 +72555 POINT(39.77958022144002 -74.91073521707155) bank72555 +72556 POINT(40.50710685099037 -73.139521242927) bank72556 +72557 POINT(40.41289014248504 -73.93797335323984) bank72557 +72558 POINT(40.41591554213415 -73.89321593303589) bank72558 +72559 POINT(41.51929713529211 -74.55145054132899) bank72559 +72560 POINT(41.545325167523394 -73.28020115702725) bank72560 +72561 POINT(39.728143273721635 -74.95958936736294) bank72561 +72562 POINT(39.86015122517647 -73.68440389659462) bank72562 +72563 POINT(40.49847358332615 -73.04514982950623) bank72563 +72564 POINT(40.459601664519894 -74.80718664719726) bank72564 +72565 POINT(40.49602753630587 -73.40468025473201) bank72565 +72566 POINT(41.67623634968064 -73.17182116222881) bank72566 +72567 POINT(40.86799964482266 -73.41551449639411) bank72567 +72568 POINT(41.446987312347794 -73.36962821318517) bank72568 +72569 POINT(39.91837838137725 -74.15724877973568) bank72569 +72570 POINT(40.35028831540897 -74.41984142760181) bank72570 +72571 POINT(41.29441085470658 -73.32661073776532) bank72571 +72572 POINT(41.00298405638036 -73.116912612152) bank72572 +72573 POINT(40.16384140404032 -73.5987367786808) bank72573 +72574 POINT(41.396276746422586 -74.97968196173198) bank72574 +72575 POINT(41.49258143909234 -74.26619907644486) bank72575 +72576 POINT(39.99637453863615 -74.46953189296202) bank72576 +72577 POINT(41.445685945239795 -73.98227081806448) bank72577 +72578 POINT(40.77981288299659 -73.36511935167484) bank72578 +72579 POINT(40.72145463279803 -74.64407818295405) bank72579 +72580 POINT(40.9773199075491 -74.04746136146238) bank72580 +72581 POINT(41.40631255642553 -74.76390965907902) bank72581 +72582 POINT(40.778606625034236 -74.70608649621911) bank72582 +72583 POINT(40.30668739206081 -73.10043841978796) bank72583 +72584 POINT(40.20096460095381 -73.76224522635138) bank72584 +72585 POINT(40.263872042177766 -74.98405218736377) bank72585 +72586 POINT(39.85484733383636 -73.85911600052388) bank72586 +72587 POINT(39.99502085006662 -74.34702162962957) bank72587 +72588 POINT(41.36802792182208 -74.31329582337735) bank72588 +72589 POINT(41.051709913436845 -74.33624742416632) bank72589 +72590 POINT(40.6958555371194 -73.9600712999964) bank72590 +72591 POINT(41.69772927808123 -73.25464705751693) bank72591 +72592 POINT(40.23940987715009 -73.40345659976569) bank72592 +72593 POINT(40.13640691826064 -74.78614244423943) bank72593 +72594 POINT(40.906779149948804 -73.35951843183156) bank72594 +72595 POINT(41.03598214941382 -74.72133695541262) bank72595 +72596 POINT(40.63000085869261 -73.22534708942683) bank72596 +72597 POINT(41.68387257559121 -74.25540455289399) bank72597 +72598 POINT(39.9325316876928 -73.30104783474911) bank72598 +72599 POINT(41.35166780422572 -74.47742160754144) bank72599 +72600 POINT(40.84998400766179 -74.28289233022582) bank72600 +72601 POINT(41.081455104527514 -73.1751060865216) bank72601 +72602 POINT(39.815705836795885 -73.82241842529238) bank72602 +72603 POINT(39.719520301625224 -74.60997535944757) bank72603 +72604 POINT(40.753288576105334 -74.31614346060964) bank72604 +72605 POINT(40.71549282770051 -74.26495091854072) bank72605 +72606 POINT(41.266492827788085 -73.373886321232) bank72606 +72607 POINT(41.31978230851618 -73.64740216120931) bank72607 +72608 POINT(41.00149670044753 -73.78465222979324) bank72608 +72609 POINT(41.09887110518011 -73.21294733210111) bank72609 +72610 POINT(41.510705339565206 -74.16006193309185) bank72610 +72611 POINT(39.89475242494845 -74.85645185247526) bank72611 +72612 POINT(40.65300520809417 -74.5699601850039) bank72612 +72613 POINT(40.75245540411728 -74.84987040906161) bank72613 +72614 POINT(40.745648926184145 -74.03680132043095) bank72614 +72615 POINT(40.71510552712931 -73.80479325316003) bank72615 +72616 POINT(40.58107951264388 -74.21129403672063) bank72616 +72617 POINT(40.99798255905208 -74.36156664457039) bank72617 +72618 POINT(40.08572160597316 -74.7262957003949) bank72618 +72619 POINT(41.65945163673141 -74.45964595467115) bank72619 +72620 POINT(41.38383495636373 -73.96505234558339) bank72620 +72621 POINT(39.821746738863474 -73.36994062437256) bank72621 +72622 POINT(40.759423829123676 -74.11935823584638) bank72622 +72623 POINT(40.058958349242296 -73.62442367075478) bank72623 +72624 POINT(41.42837947900017 -73.55653117297263) bank72624 +72625 POINT(39.97971075627145 -74.94686115747031) bank72625 +72626 POINT(40.41374572757403 -73.80808321101598) bank72626 +72627 POINT(40.43787678344979 -74.14289804088843) bank72627 +72628 POINT(41.53832048034349 -73.84182277460475) bank72628 +72629 POINT(40.53919703537847 -73.69826818394827) bank72629 +72630 POINT(39.80594672724435 -73.05518121277943) bank72630 +72631 POINT(40.072995014251696 -74.11425697380652) bank72631 +72632 POINT(39.91173578166386 -74.20033538234222) bank72632 +72633 POINT(40.428646008585154 -74.08763371239476) bank72633 +72634 POINT(40.32890910470402 -74.67968232374393) bank72634 +72635 POINT(40.639375980748376 -73.61352056359608) bank72635 +72636 POINT(40.20774574879322 -73.92324535590453) bank72636 +72637 POINT(40.325058422315614 -73.84667248404052) bank72637 +72638 POINT(40.34646495027444 -74.47478297092309) bank72638 +72639 POINT(41.704699398527836 -74.42521848099454) bank72639 +72640 POINT(39.94619769988299 -74.4347390195022) bank72640 +72641 POINT(41.16335568675118 -74.89620259386555) bank72641 +72642 POINT(40.87111872560101 -74.98040618997562) bank72642 +72643 POINT(40.65939510722762 -73.77708966434423) bank72643 +72644 POINT(41.25279249809034 -73.60150420923128) bank72644 +72645 POINT(40.108165069878964 -73.24811910531454) bank72645 +72646 POINT(41.12761480789801 -74.22961503697404) bank72646 +72647 POINT(40.85521677147436 -74.6417368301935) bank72647 +72648 POINT(41.30628486800784 -73.24199068706085) bank72648 +72649 POINT(41.075769850206015 -73.17463497469754) bank72649 +72650 POINT(41.59102283664139 -73.67637807321331) bank72650 +72651 POINT(39.96600517200993 -73.83800255593225) bank72651 +72652 POINT(40.59215137005178 -73.56192686786558) bank72652 +72653 POINT(40.46088189369936 -74.0753477547239) bank72653 +72654 POINT(41.38375075818404 -74.56610911266361) bank72654 +72655 POINT(41.24618835274894 -73.67159816286444) bank72655 +72656 POINT(41.22630169996999 -73.76963907848133) bank72656 +72657 POINT(41.14497146122203 -74.74832299437728) bank72657 +72658 POINT(41.216573821124825 -74.4159812593486) bank72658 +72659 POINT(39.82591407383538 -74.32222460334839) bank72659 +72660 POINT(41.66738540023186 -73.36235780190927) bank72660 +72661 POINT(40.85876876496559 -74.53980423484637) bank72661 +72662 POINT(40.21680193024317 -73.89218412242245) bank72662 +72663 POINT(41.58261178357738 -73.23920067700138) bank72663 +72664 POINT(40.40586334673371 -73.64992286116922) bank72664 +72665 POINT(41.12718344697426 -73.80401366433628) bank72665 +72666 POINT(39.7728218858333 -73.77015651622156) bank72666 +72667 POINT(40.94273133520638 -73.59896472672087) bank72667 +72668 POINT(41.215031448199476 -73.44317372847728) bank72668 +72669 POINT(39.98794452435804 -73.60370485940112) bank72669 +72670 POINT(40.60791566535738 -73.33324938922343) bank72670 +72671 POINT(40.224450126410865 -73.91036994851584) bank72671 +72672 POINT(40.44154401384285 -73.66887838675464) bank72672 +72673 POINT(41.05635975928051 -73.44430828749982) bank72673 +72674 POINT(40.6937861846372 -73.9934157450802) bank72674 +72675 POINT(40.37224884806459 -73.03388381171833) bank72675 +72676 POINT(41.08589374435342 -74.99950013551795) bank72676 +72677 POINT(41.639245185829694 -73.10329098847782) bank72677 +72678 POINT(40.73627828219695 -73.89653570599266) bank72678 +72679 POINT(40.29379857919739 -74.86564550561444) bank72679 +72680 POINT(39.77457125812889 -74.32144781710174) bank72680 +72681 POINT(40.128343198237864 -73.50544750872689) bank72681 +72682 POINT(40.448950369229166 -73.64150144279334) bank72682 +72683 POINT(41.57631695364943 -74.80817087175299) bank72683 +72684 POINT(41.29010059087211 -74.9552299161793) bank72684 +72685 POINT(41.18760917919891 -74.96659122267643) bank72685 +72686 POINT(41.62667969435685 -73.78227772064025) bank72686 +72687 POINT(40.36896184054 -73.94746249909736) bank72687 +72688 POINT(41.482322815146325 -74.53803762018913) bank72688 +72689 POINT(40.484987452426026 -73.53970669078534) bank72689 +72690 POINT(40.02066283416119 -74.32545567153142) bank72690 +72691 POINT(41.252961520697305 -74.78199494803324) bank72691 +72692 POINT(40.287425263225835 -74.67421488378582) bank72692 +72693 POINT(39.998422276380474 -73.45446382589863) bank72693 +72694 POINT(41.52593229083924 -73.33801509784074) bank72694 +72695 POINT(40.79567567689474 -74.80008827191219) bank72695 +72696 POINT(40.742326308442564 -73.6012778095609) bank72696 +72697 POINT(41.464509643135884 -73.05004666631139) bank72697 +72698 POINT(40.73178854874439 -74.7422168831685) bank72698 +72699 POINT(40.349251767695954 -74.43749011134051) bank72699 +72700 POINT(40.075716393282 -73.0900738228811) bank72700 +72701 POINT(40.852888500835704 -73.74365184403469) bank72701 +72702 POINT(40.864500954984415 -74.96722435748944) bank72702 +72703 POINT(40.136849905140885 -73.11477014079978) bank72703 +72704 POINT(40.8751145664276 -73.10181690874) bank72704 +72705 POINT(40.0855174650766 -73.3117974477369) bank72705 +72706 POINT(41.670932270353255 -74.23073856679679) bank72706 +72707 POINT(40.93174590973474 -73.31460243967949) bank72707 +72708 POINT(39.80168880470684 -73.6662860928367) bank72708 +72709 POINT(41.50745260904435 -74.96314851491276) bank72709 +72710 POINT(40.78478206556406 -74.56850763524776) bank72710 +72711 POINT(40.10823469556432 -74.03090538224548) bank72711 +72712 POINT(40.17149567683383 -73.94770108657856) bank72712 +72713 POINT(40.202166594173235 -74.00929246548125) bank72713 +72714 POINT(40.832049290500514 -73.96113440887605) bank72714 +72715 POINT(39.92568693948636 -74.15086205930828) bank72715 +72716 POINT(40.2299346980429 -73.46346607880079) bank72716 +72717 POINT(41.05971988049762 -74.39545400775985) bank72717 +72718 POINT(41.55338455938766 -73.78484351865181) bank72718 +72719 POINT(39.866327380654056 -73.8578112635516) bank72719 +72720 POINT(40.723700388058106 -73.35353665947247) bank72720 +72721 POINT(39.835591309188146 -73.46120071045823) bank72721 +72722 POINT(40.043865950628025 -73.72266624142496) bank72722 +72723 POINT(41.01878283958416 -74.00807526018187) bank72723 +72724 POINT(39.718944336344656 -73.83711163820806) bank72724 +72725 POINT(39.872849398848885 -73.1166286197269) bank72725 +72726 POINT(40.52780560719202 -74.49683701103163) bank72726 +72727 POINT(40.7839005907665 -74.83941967558792) bank72727 +72728 POINT(41.25071485588814 -73.30107222840952) bank72728 +72729 POINT(40.25176543197367 -74.82997030621281) bank72729 +72730 POINT(40.72016768773912 -73.98954209908551) bank72730 +72731 POINT(40.616229302542386 -73.26008168456038) bank72731 +72732 POINT(39.75227475853703 -74.83892789954669) bank72732 +72733 POINT(39.78696935354605 -73.97539694876734) bank72733 +72734 POINT(39.92814627383968 -73.7607154249204) bank72734 +72735 POINT(40.69046575370583 -74.34302090550204) bank72735 +72736 POINT(40.5092648903222 -74.86619258437103) bank72736 +72737 POINT(41.711211930430274 -73.33504128343466) bank72737 +72738 POINT(40.02617829193416 -73.22337198482238) bank72738 +72739 POINT(39.868101059312735 -73.72671702471257) bank72739 +72740 POINT(40.99685962783364 -74.52714936270482) bank72740 +72741 POINT(40.07992334679685 -74.58915057609809) bank72741 +72742 POINT(41.14852126753947 -74.03335395078403) bank72742 +72743 POINT(39.984014462616464 -73.37363614757805) bank72743 +72744 POINT(41.43313041818284 -73.5743176464402) bank72744 +72745 POINT(40.907015995874 -74.82251414165862) bank72745 +72746 POINT(41.3905243289141 -73.91785127360774) bank72746 +72747 POINT(40.96505887931002 -74.35807765987698) bank72747 +72748 POINT(41.29350817382947 -73.36369537067297) bank72748 +72749 POINT(40.46071113967092 -73.46685792768572) bank72749 +72750 POINT(41.35576749749421 -74.34957976720281) bank72750 +72751 POINT(40.986221768720014 -73.80956929325365) bank72751 +72752 POINT(40.05274485659148 -74.18614640880811) bank72752 +72753 POINT(39.99681153314147 -73.63089951548451) bank72753 +72754 POINT(41.65357647174731 -73.8315521533312) bank72754 +72755 POINT(40.97499941498137 -73.1415198911033) bank72755 +72756 POINT(40.05841665404224 -73.4784647601481) bank72756 +72757 POINT(40.86585112165609 -74.93455171377165) bank72757 +72758 POINT(39.80943543547676 -74.81623482537643) bank72758 +72759 POINT(40.99097614390763 -74.06792744555175) bank72759 +72760 POINT(39.853826508498344 -74.56294175298186) bank72760 +72761 POINT(39.725286632680636 -74.55695131721967) bank72761 +72762 POINT(40.161253692065515 -73.7694998126053) bank72762 +72763 POINT(39.91644643001014 -73.4786189657669) bank72763 +72764 POINT(40.92252187981217 -74.67903351380303) bank72764 +72765 POINT(41.71168422654295 -74.58690204559791) bank72765 +72766 POINT(40.927145300023874 -73.26738311512233) bank72766 +72767 POINT(41.52778719194353 -74.96407134670036) bank72767 +72768 POINT(40.94444876320654 -74.0520378556111) bank72768 +72769 POINT(41.38120714607971 -73.57562295738003) bank72769 +72770 POINT(40.37070729117331 -73.29854629565921) bank72770 +72771 POINT(41.359340876457125 -74.74541031519061) bank72771 +72772 POINT(41.441962271306714 -73.32536157466055) bank72772 +72773 POINT(40.861422070948116 -74.7386856701996) bank72773 +72774 POINT(41.41467666319686 -73.32492142225665) bank72774 +72775 POINT(41.5468495766391 -73.77174747585704) bank72775 +72776 POINT(40.49933335002131 -74.7149094514527) bank72776 +72777 POINT(41.101702810653485 -74.45529511563292) bank72777 +72778 POINT(40.53177765035831 -74.73361837439506) bank72778 +72779 POINT(41.596356987896286 -74.02443851703056) bank72779 +72780 POINT(41.153087319128794 -73.2400182325277) bank72780 +72781 POINT(40.521698004307055 -74.86291216480818) bank72781 +72782 POINT(41.06608350124322 -74.15610545409636) bank72782 +72783 POINT(40.28841637963285 -74.33314181763916) bank72783 +72784 POINT(41.37350649994937 -74.98061902467694) bank72784 +72785 POINT(41.096142456842216 -73.96903255357554) bank72785 +72786 POINT(40.270076439490545 -73.38944858104789) bank72786 +72787 POINT(40.97527200066834 -73.21273039373739) bank72787 +72788 POINT(41.44188689036258 -74.41680050048714) bank72788 +72789 POINT(41.16257789173536 -74.37220764767648) bank72789 +72790 POINT(40.15839328669771 -73.01566402141123) bank72790 +72791 POINT(40.578759811183524 -73.92844726929786) bank72791 +72792 POINT(40.820693601662626 -74.10889408659256) bank72792 +72793 POINT(40.29441583844959 -74.7472814110573) bank72793 +72794 POINT(41.61357611144826 -74.27283468514571) bank72794 +72795 POINT(41.44247030565647 -73.80453663994437) bank72795 +72796 POINT(40.0537046421869 -74.2286320162041) bank72796 +72797 POINT(39.803427425224704 -74.60085444820638) bank72797 +72798 POINT(40.41456906544425 -73.76715478825405) bank72798 +72799 POINT(41.04300471757842 -74.86385588760497) bank72799 +72800 POINT(41.047807762782845 -74.82769111921426) bank72800 +72801 POINT(40.03684298345022 -73.66639778970008) bank72801 +72802 POINT(40.10678040191841 -73.40569503506202) bank72802 +72803 POINT(41.5932220108614 -73.38783366603793) bank72803 +72804 POINT(40.139761044079584 -73.01470032432378) bank72804 +72805 POINT(41.31925161277863 -73.34830056578556) bank72805 +72806 POINT(40.854787620690615 -74.07969864797045) bank72806 +72807 POINT(39.80257393356908 -73.20481052330781) bank72807 +72808 POINT(39.83621489919921 -73.7334357736324) bank72808 +72809 POINT(41.658618216194675 -74.63610940433014) bank72809 +72810 POINT(40.36245374674202 -73.20895558696756) bank72810 +72811 POINT(41.00263146369984 -74.65897731867345) bank72811 +72812 POINT(39.74606283288788 -73.36658208833741) bank72812 +72813 POINT(40.82161268129898 -74.49911339863668) bank72813 +72814 POINT(40.89585832529326 -74.39374831761177) bank72814 +72815 POINT(41.01622370882701 -73.5164325858873) bank72815 +72816 POINT(41.4250316242447 -74.32711196714452) bank72816 +72817 POINT(41.14847715787496 -74.60851546245252) bank72817 +72818 POINT(40.40626947031385 -73.4015323216768) bank72818 +72819 POINT(41.06278592047678 -73.46410026225016) bank72819 +72820 POINT(40.42162296383712 -74.35877510906253) bank72820 +72821 POINT(41.28959552957523 -74.52756439291086) bank72821 +72822 POINT(40.13348788679184 -73.39709132024333) bank72822 +72823 POINT(40.787703168708994 -73.04920199068165) bank72823 +72824 POINT(40.31897331440571 -73.68376340577736) bank72824 +72825 POINT(41.16005239603062 -73.65076709778249) bank72825 +72826 POINT(41.05679529474265 -74.50447724590599) bank72826 +72827 POINT(41.34400325529211 -73.22475543356424) bank72827 +72828 POINT(41.08409236858736 -74.80689208288788) bank72828 +72829 POINT(39.82411154964752 -73.76542422364889) bank72829 +72830 POINT(40.03497431062496 -74.69197896519843) bank72830 +72831 POINT(40.82109661473359 -74.2658898672443) bank72831 +72832 POINT(40.46685261950809 -74.94652218484991) bank72832 +72833 POINT(41.521247630338046 -73.83641288759203) bank72833 +72834 POINT(40.92194098575052 -74.29731357611213) bank72834 +72835 POINT(40.625635469068385 -74.31650375057859) bank72835 +72836 POINT(41.43365248221687 -74.51038123092466) bank72836 +72837 POINT(40.35862133114061 -74.74526560110513) bank72837 +72838 POINT(41.54008586483617 -74.43524889474156) bank72838 +72839 POINT(39.808487441913506 -73.31564439607253) bank72839 +72840 POINT(41.099688031382435 -74.0665276704412) bank72840 +72841 POINT(40.563766928167205 -74.19695171412954) bank72841 +72842 POINT(41.343987359344354 -74.02209694119303) bank72842 +72843 POINT(40.420167429565794 -73.6659855492409) bank72843 +72844 POINT(39.984537015266106 -73.37180795910173) bank72844 +72845 POINT(40.62269769038514 -73.62146356852637) bank72845 +72846 POINT(41.279644294877045 -73.06916720201856) bank72846 +72847 POINT(40.66068544924788 -74.97599717001738) bank72847 +72848 POINT(39.72797723265383 -74.90162312121893) bank72848 +72849 POINT(41.21766709158105 -74.12129975740386) bank72849 +72850 POINT(40.29176414743916 -73.68341597574857) bank72850 +72851 POINT(40.167087769735446 -73.57303824602643) bank72851 +72852 POINT(40.29362429475529 -74.34039914369698) bank72852 +72853 POINT(40.009136936291455 -74.13462715287363) bank72853 +72854 POINT(40.44960510618479 -74.03715556845295) bank72854 +72855 POINT(41.2340549919026 -73.67093740373376) bank72855 +72856 POINT(41.68492824183892 -74.46006888635331) bank72856 +72857 POINT(40.84024229644089 -74.50383061040557) bank72857 +72858 POINT(40.03389873274964 -73.32314759254918) bank72858 +72859 POINT(41.175069145645125 -74.70908168788394) bank72859 +72860 POINT(40.85113350325614 -74.87394419216206) bank72860 +72861 POINT(41.34070272173711 -73.25595181088138) bank72861 +72862 POINT(41.60967166580954 -73.62983526890206) bank72862 +72863 POINT(40.17305293830291 -73.71094372216618) bank72863 +72864 POINT(41.08143955672142 -74.640197883841) bank72864 +72865 POINT(40.837630084377416 -74.04421908118393) bank72865 +72866 POINT(41.571924283153116 -74.30352393116114) bank72866 +72867 POINT(40.663614260445726 -74.19919547990526) bank72867 +72868 POINT(41.325986999331164 -73.5703473569693) bank72868 +72869 POINT(40.31268825099437 -74.78474484484109) bank72869 +72870 POINT(40.12399756078576 -73.59345666986897) bank72870 +72871 POINT(40.37722455849681 -73.2084643048114) bank72871 +72872 POINT(40.322190781025796 -73.74454944867507) bank72872 +72873 POINT(41.37419472200107 -74.48438013915238) bank72873 +72874 POINT(40.087074096652174 -73.65667519254383) bank72874 +72875 POINT(40.173775624547844 -73.01264228695781) bank72875 +72876 POINT(41.358018465789016 -73.19551070207281) bank72876 +72877 POINT(40.243281905153566 -73.33540363907107) bank72877 +72878 POINT(40.82042555521192 -73.61820272165167) bank72878 +72879 POINT(39.7856228095228 -73.41318769304289) bank72879 +72880 POINT(40.430914604270846 -74.35923687847378) bank72880 +72881 POINT(40.74511721961312 -73.77998137425726) bank72881 +72882 POINT(40.22476103198601 -74.8805358118563) bank72882 +72883 POINT(40.4485536877322 -73.22554854886026) bank72883 +72884 POINT(40.140560768253515 -73.48071007153354) bank72884 +72885 POINT(41.71076008202728 -74.65355950281818) bank72885 +72886 POINT(40.8962093636893 -74.3180994792141) bank72886 +72887 POINT(40.56339094041223 -73.83740545079502) bank72887 +72888 POINT(41.42535294788766 -74.72792789747503) bank72888 +72889 POINT(41.568811655964964 -73.35923912312636) bank72889 +72890 POINT(41.089914279180675 -74.20713943347485) bank72890 +72891 POINT(40.71683959577198 -74.90509466023131) bank72891 +72892 POINT(40.55428986186636 -73.86153233934039) bank72892 +72893 POINT(40.74952841135185 -74.63917301679415) bank72893 +72894 POINT(41.49077209434925 -73.97723863533633) bank72894 +72895 POINT(40.824029727239555 -73.71922083444335) bank72895 +72896 POINT(41.223449403068635 -73.66400098178178) bank72896 +72897 POINT(39.97382245246587 -74.27248694690489) bank72897 +72898 POINT(41.10534180586442 -74.9677297559433) bank72898 +72899 POINT(41.179643845865876 -74.5971104493369) bank72899 +72900 POINT(40.24971660598534 -73.12664654702425) bank72900 +72901 POINT(40.80248674841539 -74.24276644121055) bank72901 +72902 POINT(40.09538442970707 -73.26063845417532) bank72902 +72903 POINT(41.51431299814217 -73.95960596076795) bank72903 +72904 POINT(40.15533946420579 -74.10879851370584) bank72904 +72905 POINT(40.60709594610025 -73.046721304511) bank72905 +72906 POINT(41.53417583185033 -74.41665482361364) bank72906 +72907 POINT(41.566245024592455 -73.72690097288296) bank72907 +72908 POINT(39.917133305224006 -73.38530755899555) bank72908 +72909 POINT(40.003199781444025 -74.08645908006231) bank72909 +72910 POINT(41.0031229104394 -74.27081863666106) bank72910 +72911 POINT(41.63722281190851 -73.79450712532488) bank72911 +72912 POINT(40.83252385983124 -73.09779186690054) bank72912 +72913 POINT(40.59789092416527 -73.38349125789654) bank72913 +72914 POINT(41.031198596269306 -73.88590562342219) bank72914 +72915 POINT(41.654294254446505 -73.26580517772608) bank72915 +72916 POINT(41.53863231559991 -73.09331924719702) bank72916 +72917 POINT(39.796060896067125 -74.71285632947249) bank72917 +72918 POINT(39.82249139447698 -74.71824541329772) bank72918 +72919 POINT(40.32747599623975 -73.99549716010186) bank72919 +72920 POINT(41.278366541428156 -74.67767582974372) bank72920 +72921 POINT(40.068367780618516 -74.16853915658248) bank72921 +72922 POINT(40.690422837275925 -73.59738337910126) bank72922 +72923 POINT(41.34187059288215 -74.29787900114896) bank72923 +72924 POINT(40.772143612067566 -74.1717931190077) bank72924 +72925 POINT(39.740858930186384 -74.79455948359333) bank72925 +72926 POINT(40.290518794777356 -74.86934918780189) bank72926 +72927 POINT(40.6347662185238 -74.96252780240992) bank72927 +72928 POINT(40.354300392143756 -74.33083515728734) bank72928 +72929 POINT(39.82413421451772 -73.97049969077361) bank72929 +72930 POINT(40.31952208309471 -74.35118070878083) bank72930 +72931 POINT(40.91189246097791 -74.61855514514264) bank72931 +72932 POINT(40.31016626021732 -74.21611055641743) bank72932 +72933 POINT(39.82709025862739 -73.6418034167443) bank72933 +72934 POINT(40.69980307377558 -73.45273752863262) bank72934 +72935 POINT(41.57708875634218 -73.54187382902191) bank72935 +72936 POINT(41.33969780511399 -74.6245126603247) bank72936 +72937 POINT(41.10550511519068 -73.16360584631842) bank72937 +72938 POINT(40.74412928585068 -74.0899569973912) bank72938 +72939 POINT(40.48497802479955 -74.3750637662221) bank72939 +72940 POINT(40.66919857919894 -73.04296205526767) bank72940 +72941 POINT(41.60974751932417 -73.43046741728156) bank72941 +72942 POINT(40.37658068297054 -74.2763750226825) bank72942 +72943 POINT(40.891575137079144 -73.37706962860688) bank72943 +72944 POINT(40.45838274711452 -73.37466454937329) bank72944 +72945 POINT(40.59824340115809 -74.03279531797074) bank72945 +72946 POINT(40.12614870654763 -73.20789969835572) bank72946 +72947 POINT(40.18996072706608 -73.71498036967724) bank72947 +72948 POINT(41.61225810132662 -74.83061454753303) bank72948 +72949 POINT(41.620690273501836 -74.60186682958036) bank72949 +72950 POINT(41.657948793953544 -74.87341736279164) bank72950 +72951 POINT(39.870643518777065 -73.9413180044475) bank72951 +72952 POINT(41.184514202935745 -73.0795249994221) bank72952 +72953 POINT(39.91252543470298 -73.53925226675393) bank72953 +72954 POINT(41.394423252875754 -74.22745396295952) bank72954 +72955 POINT(40.192040809750374 -74.19828072488922) bank72955 +72956 POINT(40.90947099958878 -73.29902238991882) bank72956 +72957 POINT(41.08091247688894 -74.04848855905695) bank72957 +72958 POINT(40.61513459246045 -74.17455942725489) bank72958 +72959 POINT(39.749262490859515 -73.59814462113273) bank72959 +72960 POINT(41.47600641859067 -74.39713706979795) bank72960 +72961 POINT(41.13987438379205 -73.62474226293682) bank72961 +72962 POINT(40.6837832940306 -73.64966923786287) bank72962 +72963 POINT(41.64999770865921 -73.67396444925437) bank72963 +72964 POINT(41.389663317492 -74.33806029119309) bank72964 +72965 POINT(40.14920202926107 -73.59643951078685) bank72965 +72966 POINT(41.57187821902705 -73.63703823284777) bank72966 +72967 POINT(40.40275892886415 -74.53522162014877) bank72967 +72968 POINT(41.245901592575095 -74.86725879531171) bank72968 +72969 POINT(41.160865144271575 -73.28489807256109) bank72969 +72970 POINT(41.16102157455959 -74.44223232121112) bank72970 +72971 POINT(40.0807162525069 -74.7762672241718) bank72971 +72972 POINT(40.53402572944755 -73.418035720538) bank72972 +72973 POINT(41.36673941473264 -73.19450838746225) bank72973 +72974 POINT(40.66842187666304 -73.89996717189128) bank72974 +72975 POINT(40.993771030180845 -74.50594583814362) bank72975 +72976 POINT(40.82230797393037 -74.89769662180942) bank72976 +72977 POINT(40.64273104229354 -73.85178955861315) bank72977 +72978 POINT(39.916505077009774 -74.82707382345362) bank72978 +72979 POINT(39.75956368432905 -73.70221680791617) bank72979 +72980 POINT(40.64694479882362 -74.85895273766742) bank72980 +72981 POINT(40.35409136737312 -74.10268647784761) bank72981 +72982 POINT(40.663666629839256 -73.45472522320729) bank72982 +72983 POINT(39.8851086760521 -73.94827789964796) bank72983 +72984 POINT(41.31145698650362 -73.45373699940798) bank72984 +72985 POINT(41.458901930462716 -74.88822707901562) bank72985 +72986 POINT(39.893067876828546 -74.38128720800754) bank72986 +72987 POINT(41.025112244214796 -74.21183920553894) bank72987 +72988 POINT(40.82605117093478 -74.82428729293903) bank72988 +72989 POINT(41.595085691311496 -73.50052107086134) bank72989 +72990 POINT(40.93773465568256 -74.56095329530314) bank72990 +72991 POINT(41.19641299019723 -73.74565043911885) bank72991 +72992 POINT(40.81504287420451 -74.4267324203197) bank72992 +72993 POINT(41.37951988564109 -73.46152706052155) bank72993 +72994 POINT(41.66016416241239 -73.438514474691) bank72994 +72995 POINT(41.44935560490369 -74.75191587105566) bank72995 +72996 POINT(40.10683716229154 -74.40744779735753) bank72996 +72997 POINT(40.45378081954595 -74.90608948594573) bank72997 +72998 POINT(40.686644503511296 -74.77278398498267) bank72998 +72999 POINT(40.91779940764378 -73.07583928046581) bank72999 +73000 POINT(41.370031100302086 -74.21746738481515) bank73000 +73001 POINT(40.99441122303849 -73.13871710828441) bank73001 +73002 POINT(40.83854585447353 -73.74020714857635) bank73002 +73003 POINT(41.44118127821708 -74.65962112529365) bank73003 +73004 POINT(41.352953307789065 -74.47965092898434) bank73004 +73005 POINT(40.38236605373487 -74.06022684852476) bank73005 +73006 POINT(40.515195102245194 -73.88752561263854) bank73006 +73007 POINT(40.27072949803786 -73.37559867571052) bank73007 +73008 POINT(41.030610944860854 -73.57272570788251) bank73008 +73009 POINT(41.54773950455953 -73.13374853385767) bank73009 +73010 POINT(39.95568190745295 -73.7461716142676) bank73010 +73011 POINT(40.54399693851296 -73.53957540230172) bank73011 +73012 POINT(40.147318564466026 -73.54309877823218) bank73012 +73013 POINT(40.16283602908436 -74.07181024533374) bank73013 +73014 POINT(41.31789816016898 -74.60987721428893) bank73014 +73015 POINT(40.500148695424684 -73.73946493975416) bank73015 +73016 POINT(41.71062608205461 -74.58963952365777) bank73016 +73017 POINT(41.63833078679111 -74.51907189361724) bank73017 +73018 POINT(41.59303103995492 -74.36769995318058) bank73018 +73019 POINT(40.0191098451478 -74.64143510031751) bank73019 +73020 POINT(40.217174519579 -73.06344230777485) bank73020 +73021 POINT(39.90552530616618 -73.50883795220173) bank73021 +73022 POINT(40.105267409313385 -74.63663261745296) bank73022 +73023 POINT(39.97074892745622 -74.21131392245977) bank73023 +73024 POINT(39.7167670239407 -73.33269018128769) bank73024 +73025 POINT(41.18382910584959 -74.8348322519916) bank73025 +73026 POINT(41.34791458797177 -74.7263424005289) bank73026 +73027 POINT(41.0168556278129 -73.32439576064614) bank73027 +73028 POINT(40.837386400756756 -74.32585597023429) bank73028 +73029 POINT(40.74134878616553 -73.35301973636172) bank73029 +73030 POINT(41.123894695811885 -73.3285156671788) bank73030 +73031 POINT(41.32170714632025 -74.85189380080445) bank73031 +73032 POINT(40.817680804513174 -73.68044819853043) bank73032 +73033 POINT(40.702492413429596 -74.95714106097545) bank73033 +73034 POINT(41.66266688636253 -73.78352635143722) bank73034 +73035 POINT(40.20314477541688 -74.2163590094742) bank73035 +73036 POINT(40.64644674684993 -74.89974171657713) bank73036 +73037 POINT(40.55424908397969 -74.2678562232879) bank73037 +73038 POINT(40.118028950910045 -74.54369344870793) bank73038 +73039 POINT(40.498982117210176 -73.04140678823431) bank73039 +73040 POINT(40.42109992620469 -73.24510036229547) bank73040 +73041 POINT(40.649246749393726 -73.00695824494277) bank73041 +73042 POINT(40.54728183396106 -73.01615599989587) bank73042 +73043 POINT(41.45298407749757 -74.8615312329758) bank73043 +73044 POINT(41.66810822460351 -74.32299437586583) bank73044 +73045 POINT(40.71832449171881 -74.49586879200986) bank73045 +73046 POINT(39.85890410242799 -73.8611274450329) bank73046 +73047 POINT(41.11455520956368 -73.4740083892535) bank73047 +73048 POINT(41.23989245183355 -73.99410063465868) bank73048 +73049 POINT(40.127644940736786 -74.87083970034661) bank73049 +73050 POINT(39.75195425066816 -74.58957534582981) bank73050 +73051 POINT(41.44061267383756 -74.79878824846976) bank73051 +73052 POINT(41.38519114670427 -73.44399009792335) bank73052 +73053 POINT(41.402280552713044 -74.34075050102531) bank73053 +73054 POINT(41.34366331602188 -73.81626772570613) bank73054 +73055 POINT(40.48288464664495 -73.70074531620111) bank73055 +73056 POINT(41.17446825848568 -74.08300559276222) bank73056 +73057 POINT(40.627288121126206 -74.34987353153087) bank73057 +73058 POINT(41.44227699420474 -74.31365629627967) bank73058 +73059 POINT(41.051227740250084 -73.97152893206582) bank73059 +73060 POINT(39.957897301709835 -74.21001941575108) bank73060 +73061 POINT(40.86650273813294 -73.4425982382338) bank73061 +73062 POINT(40.94428539523203 -73.91466955345653) bank73062 +73063 POINT(40.45924717246083 -73.63384162004498) bank73063 +73064 POINT(40.25686002609091 -73.32096099723448) bank73064 +73065 POINT(40.8965290550525 -73.79064003893505) bank73065 +73066 POINT(41.38560280073799 -73.17103678645293) bank73066 +73067 POINT(39.95232040322255 -73.70868589081968) bank73067 +73068 POINT(39.83560223565458 -73.01207482760694) bank73068 +73069 POINT(40.44680434487038 -73.47499509072206) bank73069 +73070 POINT(39.784931967780196 -73.38640617995426) bank73070 +73071 POINT(40.211582189075855 -74.96012613142959) bank73071 +73072 POINT(40.55617396614424 -74.5650204836889) bank73072 +73073 POINT(39.9964501734502 -74.57427805430734) bank73073 +73074 POINT(41.470307146990606 -73.54808069157524) bank73074 +73075 POINT(41.60176522370437 -73.00862122602204) bank73075 +73076 POINT(41.2449624904378 -74.62373029172463) bank73076 +73077 POINT(40.239208685822994 -74.43295761644023) bank73077 +73078 POINT(39.90821859937417 -73.83004797006103) bank73078 +73079 POINT(41.35163361653007 -73.18810029402516) bank73079 +73080 POINT(41.176717064239725 -74.09748119662703) bank73080 +73081 POINT(41.678542365595945 -73.27340532165236) bank73081 +73082 POINT(40.139831408049034 -74.88795317808327) bank73082 +73083 POINT(41.11130847874509 -73.83768473464464) bank73083 +73084 POINT(40.80483770869201 -74.30823419189004) bank73084 +73085 POINT(40.849788670758436 -73.71493619226472) bank73085 +73086 POINT(41.19374415625914 -74.1770215407838) bank73086 +73087 POINT(39.893430355463224 -74.46212211278859) bank73087 +73088 POINT(41.055878055743754 -73.07976435049346) bank73088 +73089 POINT(41.36332435301098 -73.14300610630072) bank73089 +73090 POINT(39.7527455297883 -73.22968573015535) bank73090 +73091 POINT(41.122111237876126 -74.45335488428367) bank73091 +73092 POINT(40.709982754404734 -73.54346004731082) bank73092 +73093 POINT(41.53980005091993 -73.78571043978711) bank73093 +73094 POINT(39.72746058619825 -74.99249830472937) bank73094 +73095 POINT(41.52118601566808 -73.67904018257484) bank73095 +73096 POINT(39.78829845561996 -73.7210844481397) bank73096 +73097 POINT(40.12797647885036 -74.45171370246216) bank73097 +73098 POINT(40.37087205074401 -73.73732548304076) bank73098 +73099 POINT(41.25539243996634 -73.39435801415958) bank73099 +73100 POINT(40.29646072316504 -74.46007261887065) bank73100 +73101 POINT(40.532870859128494 -73.87459658555481) bank73101 +73102 POINT(40.89519318467645 -73.81865165171641) bank73102 +73103 POINT(41.36980909063429 -74.9799346529776) bank73103 +73104 POINT(40.85004505421102 -74.06814417654405) bank73104 +73105 POINT(40.52603069916931 -73.56245053741051) bank73105 +73106 POINT(39.81474374753755 -74.59723901213385) bank73106 +73107 POINT(41.00263851211172 -74.59164202140025) bank73107 +73108 POINT(41.28559811960628 -73.37239946120604) bank73108 +73109 POINT(40.22025887179768 -74.68670253102839) bank73109 +73110 POINT(40.8072288471196 -73.4307492879154) bank73110 +73111 POINT(40.483992272796236 -73.74523035071972) bank73111 +73112 POINT(40.71137930035985 -74.28193666488751) bank73112 +73113 POINT(41.272198119508715 -74.7065183967873) bank73113 +73114 POINT(41.02070346780946 -73.5892605083986) bank73114 +73115 POINT(40.60334349725523 -73.18218949007426) bank73115 +73116 POINT(39.88620354786495 -73.73025502351767) bank73116 +73117 POINT(41.47487018912635 -74.94045207419599) bank73117 +73118 POINT(40.265652771030595 -73.72209444216915) bank73118 +73119 POINT(39.819895894377964 -73.67487303740513) bank73119 +73120 POINT(40.13479864183728 -73.37433244221539) bank73120 +73121 POINT(40.14836007968828 -73.81862331014182) bank73121 +73122 POINT(41.11718533902243 -73.56625344815089) bank73122 +73123 POINT(41.366338051489485 -74.26299825098634) bank73123 +73124 POINT(41.3417612291735 -74.56838321316911) bank73124 +73125 POINT(39.87394408480747 -73.88451937313056) bank73125 +73126 POINT(39.822567974839735 -73.67081542411422) bank73126 +73127 POINT(40.45878997274582 -74.1390799862017) bank73127 +73128 POINT(39.793084741004435 -73.0391726845588) bank73128 +73129 POINT(41.36644549798216 -73.20683788587608) bank73129 +73130 POINT(41.15786730089361 -73.52553799844254) bank73130 +73131 POINT(40.17720295546949 -73.84559959158399) bank73131 +73132 POINT(41.045102220939725 -73.60666191480279) bank73132 +73133 POINT(39.747279871358515 -73.79211907934534) bank73133 +73134 POINT(40.34459515554977 -73.48969397909009) bank73134 +73135 POINT(41.13495258288299 -73.23061080920763) bank73135 +73136 POINT(41.19314946999396 -74.43136011608333) bank73136 +73137 POINT(40.784726863836134 -73.06455213840705) bank73137 +73138 POINT(41.31426317563887 -73.4867089902078) bank73138 +73139 POINT(41.09253193898574 -74.03709798636675) bank73139 +73140 POINT(41.5918022625551 -73.43870067621384) bank73140 +73141 POINT(40.35410795057383 -73.01052641486386) bank73141 +73142 POINT(40.24869838702822 -73.05595455951867) bank73142 +73143 POINT(41.18971061335412 -74.83663422836483) bank73143 +73144 POINT(40.31476867360388 -74.03224301789697) bank73144 +73145 POINT(40.86899643322221 -73.95856093102748) bank73145 +73146 POINT(40.56176269776896 -73.41457213492414) bank73146 +73147 POINT(40.958854219022655 -73.16326193538798) bank73147 +73148 POINT(40.93268284973131 -73.13958609528227) bank73148 +73149 POINT(40.96807181232827 -73.0715594446577) bank73149 +73150 POINT(40.839104187856854 -74.9717654927768) bank73150 +73151 POINT(40.49705848450944 -74.53927262367756) bank73151 +73152 POINT(41.05903199667832 -74.84891663655084) bank73152 +73153 POINT(41.367986269790315 -73.35490651610137) bank73153 +73154 POINT(41.374790269560464 -74.06128626345534) bank73154 +73155 POINT(40.007221264936774 -74.81041027087092) bank73155 +73156 POINT(41.130277754606105 -73.95748798972608) bank73156 +73157 POINT(40.86439706101035 -74.64620817587084) bank73157 +73158 POINT(40.95635568159104 -74.29546951814704) bank73158 +73159 POINT(40.62702883655341 -73.41985652808737) bank73159 +73160 POINT(39.99128977911585 -73.09364851660226) bank73160 +73161 POINT(41.55817449940404 -73.67544598700896) bank73161 +73162 POINT(40.09933881245045 -74.40680362987868) bank73162 +73163 POINT(40.813631218329306 -73.4704944120621) bank73163 +73164 POINT(40.678481221855044 -73.06663323272248) bank73164 +73165 POINT(41.48923852345977 -73.34911316890533) bank73165 +73166 POINT(40.78846458133113 -73.3846789125597) bank73166 +73167 POINT(41.22930308000546 -73.62341494131809) bank73167 +73168 POINT(40.17901238126118 -73.56164542876856) bank73168 +73169 POINT(39.720274298347206 -73.65343704969546) bank73169 +73170 POINT(40.866461590031655 -73.36942845102706) bank73170 +73171 POINT(40.39195949093515 -73.72959589765004) bank73171 +73172 POINT(40.21786258037044 -74.32710189933599) bank73172 +73173 POINT(41.39142397497736 -74.37933694571805) bank73173 +73174 POINT(41.53642589062064 -73.90426482864889) bank73174 +73175 POINT(39.983825050285 -73.10240797956965) bank73175 +73176 POINT(39.72244830371674 -74.08153701572665) bank73176 +73177 POINT(40.43151777926987 -73.65908045522258) bank73177 +73178 POINT(41.5638785297974 -74.99173081545459) bank73178 +73179 POINT(41.197455715013405 -73.93163393284685) bank73179 +73180 POINT(39.82194338843858 -74.17587546028506) bank73180 +73181 POINT(39.8663963417474 -73.10370647269953) bank73181 +73182 POINT(40.977171502785744 -74.31777033174762) bank73182 +73183 POINT(40.58150987712572 -74.07628489457069) bank73183 +73184 POINT(41.38990293192373 -74.41727113869372) bank73184 +73185 POINT(40.13542014039489 -73.94232195099752) bank73185 +73186 POINT(40.77868883046734 -73.51635818528014) bank73186 +73187 POINT(41.55402688743578 -74.86378141369474) bank73187 +73188 POINT(40.43654463098752 -74.57224401197314) bank73188 +73189 POINT(39.90332716154266 -74.86178405756459) bank73189 +73190 POINT(41.68961715202753 -73.13121671005177) bank73190 +73191 POINT(40.29860659571875 -73.81533334807068) bank73191 +73192 POINT(41.146412641315194 -74.3919156371296) bank73192 +73193 POINT(41.253005650010834 -74.88878405354329) bank73193 +73194 POINT(41.61552175319885 -74.750115528716) bank73194 +73195 POINT(41.08173012964241 -75.0043442913478) bank73195 +73196 POINT(40.54751283254256 -73.49054208533003) bank73196 +73197 POINT(41.029325240986026 -74.23849836871725) bank73197 +73198 POINT(40.52695968203686 -74.77041310692981) bank73198 +73199 POINT(40.120401977483745 -74.72004730076358) bank73199 +73200 POINT(39.838754189577216 -73.21756492682466) bank73200 +73201 POINT(40.829190369139496 -74.15465039725596) bank73201 +73202 POINT(39.71756972020519 -73.59446463490401) bank73202 +73203 POINT(41.34795562119979 -73.23129211511483) bank73203 +73204 POINT(40.915205111061 -74.95129604773746) bank73204 +73205 POINT(41.38786860546471 -73.47434845469652) bank73205 +73206 POINT(40.246313052979076 -74.10742668519711) bank73206 +73207 POINT(39.777657516795436 -73.15768076516189) bank73207 +73208 POINT(40.813903439688595 -74.93187001640095) bank73208 +73209 POINT(40.05729197885802 -73.88999637427112) bank73209 +73210 POINT(41.610867101546056 -74.04951887090596) bank73210 +73211 POINT(40.3429680526135 -74.97025331701683) bank73211 +73212 POINT(40.08833821225156 -74.31426831176715) bank73212 +73213 POINT(39.71590012373148 -73.47386056306951) bank73213 +73214 POINT(40.9499845551564 -74.88178902753042) bank73214 +73215 POINT(40.62085755277028 -74.18576297270361) bank73215 +73216 POINT(40.17072807771596 -74.8448605486893) bank73216 +73217 POINT(40.3346636176993 -73.1970169335486) bank73217 +73218 POINT(40.91019259593769 -74.10413618230024) bank73218 +73219 POINT(40.46697172297937 -74.6356145635992) bank73219 +73220 POINT(40.682655066364276 -74.8767296623621) bank73220 +73221 POINT(40.67872240125498 -74.46991864800539) bank73221 +73222 POINT(41.52120571296237 -74.20304691960794) bank73222 +73223 POINT(40.390654842710454 -73.61716049635889) bank73223 +73224 POINT(40.14574959819709 -73.34171195704226) bank73224 +73225 POINT(41.65512938427072 -74.78504789998367) bank73225 +73226 POINT(39.96358672116598 -73.05885238346875) bank73226 +73227 POINT(40.835083303726705 -74.31131495297736) bank73227 +73228 POINT(41.652249074015565 -74.96428740798292) bank73228 +73229 POINT(40.22819168760774 -73.93722426498218) bank73229 +73230 POINT(39.72611286671992 -73.57023252292069) bank73230 +73231 POINT(39.72997749660711 -74.06696961157562) bank73231 +73232 POINT(40.46870541804399 -73.87730834527731) bank73232 +73233 POINT(40.27082963063522 -73.29265689344318) bank73233 +73234 POINT(41.69484057367237 -73.01493269692173) bank73234 +73235 POINT(40.444312913211654 -73.85460832797429) bank73235 +73236 POINT(41.425661458344244 -73.23923887247058) bank73236 +73237 POINT(39.82309810773643 -73.47234764635004) bank73237 +73238 POINT(40.51042045433271 -74.17779219302544) bank73238 +73239 POINT(40.236803893600566 -74.16978477634268) bank73239 +73240 POINT(40.08722345373087 -73.02879197778233) bank73240 +73241 POINT(39.85095970978269 -74.72212403970867) bank73241 +73242 POINT(40.19153072554189 -74.05799734456407) bank73242 +73243 POINT(40.90363784766854 -74.29526815393405) bank73243 +73244 POINT(40.16907294535156 -73.67541886880907) bank73244 +73245 POINT(41.18083044943684 -74.05220843427244) bank73245 +73246 POINT(41.193539119471204 -73.17291951804008) bank73246 +73247 POINT(41.1863858010541 -73.77048467569351) bank73247 +73248 POINT(41.159033256031826 -73.12736492852534) bank73248 +73249 POINT(40.70495657401895 -73.33077666791355) bank73249 +73250 POINT(39.74703226718562 -74.22100454823426) bank73250 +73251 POINT(40.115861445378094 -74.7094801705934) bank73251 +73252 POINT(40.8241133981412 -74.07331633524123) bank73252 +73253 POINT(40.09628102789826 -74.72767538640782) bank73253 +73254 POINT(41.46286262625012 -74.75348551301931) bank73254 +73255 POINT(41.09842271887212 -73.66018416596334) bank73255 +73256 POINT(41.692871746041604 -74.07596044615484) bank73256 +73257 POINT(40.60018701005177 -74.9502150228172) bank73257 +73258 POINT(40.577572371575044 -73.29847088194349) bank73258 +73259 POINT(41.306372907429115 -73.44099984145582) bank73259 +73260 POINT(41.227454218285295 -73.03494446632718) bank73260 +73261 POINT(40.922049281999875 -74.74381341539073) bank73261 +73262 POINT(40.31279217534326 -74.69813637481393) bank73262 +73263 POINT(40.746749753199744 -73.24281078816134) bank73263 +73264 POINT(40.93065216796167 -73.2931985930127) bank73264 +73265 POINT(41.30794640135945 -74.20032347473568) bank73265 +73266 POINT(40.52599694454327 -73.0529652342016) bank73266 +73267 POINT(39.99825973206483 -73.79169118169443) bank73267 +73268 POINT(40.3953761033759 -74.21737036419135) bank73268 +73269 POINT(41.339855180006225 -74.89156885001144) bank73269 +73270 POINT(40.485497364722555 -73.75925831599379) bank73270 +73271 POINT(41.3918137553276 -73.64282759542841) bank73271 +73272 POINT(39.957670097444506 -73.43370838679768) bank73272 +73273 POINT(41.69191735762326 -74.54759308504448) bank73273 +73274 POINT(40.037400867688525 -73.36076991087133) bank73274 +73275 POINT(41.27069466652702 -73.56020847337913) bank73275 +73276 POINT(41.44504653961091 -74.287321696093) bank73276 +73277 POINT(40.3509839266874 -74.13032172131413) bank73277 +73278 POINT(41.33020600205826 -74.78639466181433) bank73278 +73279 POINT(39.821095810313345 -74.76147392706115) bank73279 +73280 POINT(41.07456204733962 -73.37954137200836) bank73280 +73281 POINT(41.63464751104487 -73.3900750302979) bank73281 +73282 POINT(40.67502881129108 -74.38992393341057) bank73282 +73283 POINT(40.269350210690135 -74.94322118434218) bank73283 +73284 POINT(40.208893782510934 -74.82396430001553) bank73284 +73285 POINT(40.41014897781941 -74.23674971094545) bank73285 +73286 POINT(40.07475862888657 -74.24812007389859) bank73286 +73287 POINT(40.80106966698763 -73.55359800892839) bank73287 +73288 POINT(41.26230982562215 -73.52085540006982) bank73288 +73289 POINT(41.22189903791841 -73.6914305742432) bank73289 +73290 POINT(41.32707241095724 -73.1441294084862) bank73290 +73291 POINT(41.002056502024985 -74.86174754146658) bank73291 +73292 POINT(41.13140862078528 -74.6591983905667) bank73292 +73293 POINT(41.62713615803493 -73.71927580554477) bank73293 +73294 POINT(41.37473274460545 -73.15361451229624) bank73294 +73295 POINT(39.98879717357882 -74.13462741065027) bank73295 +73296 POINT(41.60173510962678 -74.27678532668712) bank73296 +73297 POINT(40.94685001166226 -74.97165318350878) bank73297 +73298 POINT(40.2402685506534 -74.54128585066007) bank73298 +73299 POINT(40.06895326228186 -74.7142774693452) bank73299 +73300 POINT(40.684769169292544 -73.41036675295726) bank73300 +73301 POINT(41.039175710987635 -73.84135108587124) bank73301 +73302 POINT(41.439779989743556 -73.80996766449059) bank73302 +73303 POINT(39.883986772891966 -73.44645687865129) bank73303 +73304 POINT(40.68717293300747 -74.70392451625717) bank73304 +73305 POINT(41.04792250331503 -74.15205889863093) bank73305 +73306 POINT(40.80634162884316 -73.70090339462493) bank73306 +73307 POINT(41.41351801475658 -74.21227179001838) bank73307 +73308 POINT(41.11450773870034 -74.80683828480358) bank73308 +73309 POINT(40.721520678752874 -74.69373201223173) bank73309 +73310 POINT(40.59912777285521 -73.02320010785591) bank73310 +73311 POINT(41.29864982390774 -74.32609921390204) bank73311 +73312 POINT(40.100682219284636 -73.55537545588129) bank73312 +73313 POINT(41.18410388109782 -74.50850238202997) bank73313 +73314 POINT(41.59445652613546 -74.6462089922639) bank73314 +73315 POINT(40.84637397313948 -74.17778354462942) bank73315 +73316 POINT(41.15686039567494 -73.41130039399467) bank73316 +73317 POINT(39.93185857867198 -74.20892175854942) bank73317 +73318 POINT(40.3161401998458 -74.89477553538886) bank73318 +73319 POINT(40.58379206812784 -74.9188755648729) bank73319 +73320 POINT(41.26447043302485 -73.2501358745895) bank73320 +73321 POINT(41.53065619553529 -73.09066018327127) bank73321 +73322 POINT(39.860088144353256 -74.93831563237303) bank73322 +73323 POINT(41.173203468269165 -74.32599374084165) bank73323 +73324 POINT(41.44186779385126 -74.74164371508994) bank73324 +73325 POINT(41.17236644841178 -73.81068234229028) bank73325 +73326 POINT(40.94488681672039 -73.53599744099127) bank73326 +73327 POINT(40.64544631101348 -74.94978564992459) bank73327 +73328 POINT(39.81406136007261 -73.0353061141329) bank73328 +73329 POINT(41.37073939957459 -74.44387631822462) bank73329 +73330 POINT(41.32575140308212 -74.43655882404045) bank73330 +73331 POINT(41.49574730618944 -74.97494632611277) bank73331 +73332 POINT(40.22454013681957 -73.30594621774462) bank73332 +73333 POINT(40.54547562314862 -73.89606463943005) bank73333 +73334 POINT(40.58528042982106 -74.15157489749267) bank73334 +73335 POINT(41.44613509722053 -74.96702527836351) bank73335 +73336 POINT(40.63296499905073 -73.53484071251701) bank73336 +73337 POINT(39.73475487275347 -73.75641656881088) bank73337 +73338 POINT(39.91908943127629 -73.51844433565671) bank73338 +73339 POINT(40.32921703385748 -74.74862130858247) bank73339 +73340 POINT(41.19690906594177 -74.49440207389192) bank73340 +73341 POINT(41.52547294832664 -74.452653359419) bank73341 +73342 POINT(41.33432112449632 -73.95363939542123) bank73342 +73343 POINT(41.491870574317204 -74.65396831018525) bank73343 +73344 POINT(41.487057169290125 -73.65568127244508) bank73344 +73345 POINT(39.89745930742346 -73.01631009404815) bank73345 +73346 POINT(40.143060023009895 -73.94358867882278) bank73346 +73347 POINT(41.64264536554593 -74.66407963502846) bank73347 +73348 POINT(40.42720286843708 -73.50836855862802) bank73348 +73349 POINT(39.955611081509176 -73.06844263275472) bank73349 +73350 POINT(40.05117733190985 -73.97006037050112) bank73350 +73351 POINT(40.85889583310705 -74.95748934659116) bank73351 +73352 POINT(40.565147946335316 -74.41724394852304) bank73352 +73353 POINT(40.74041908501095 -73.62968317336102) bank73353 +73354 POINT(41.66336906998775 -74.57315546245486) bank73354 +73355 POINT(41.54798548925254 -73.56929299159987) bank73355 +73356 POINT(40.57213045781578 -73.27802398949241) bank73356 +73357 POINT(40.45280174834524 -73.5931506732938) bank73357 +73358 POINT(40.158732763948784 -73.89612576995091) bank73358 +73359 POINT(41.103430352061025 -73.72619500216861) bank73359 +73360 POINT(40.975383721476405 -73.88932170172608) bank73360 +73361 POINT(40.374787407945085 -74.58693881889624) bank73361 +73362 POINT(41.07277849318041 -74.45595617906595) bank73362 +73363 POINT(40.27436144633254 -73.73036533701573) bank73363 +73364 POINT(40.85410802203684 -74.31706753024899) bank73364 +73365 POINT(40.00260952277314 -73.19003472343768) bank73365 +73366 POINT(41.68136249975284 -73.18128186857737) bank73366 +73367 POINT(40.65739115991239 -74.38644367709155) bank73367 +73368 POINT(40.09890989296873 -74.43200870644527) bank73368 +73369 POINT(40.186531146873605 -74.85260378951185) bank73369 +73370 POINT(40.448966108410275 -73.61608812093496) bank73370 +73371 POINT(41.09967875619446 -74.41249862002621) bank73371 +73372 POINT(40.88129949580224 -73.81083209480641) bank73372 +73373 POINT(41.2249569358597 -74.2070567980673) bank73373 +73374 POINT(40.70627009499693 -73.8086628742421) bank73374 +73375 POINT(41.259234582737555 -74.86484394906626) bank73375 +73376 POINT(41.24150033090882 -73.73977578344989) bank73376 +73377 POINT(41.366741646181545 -74.73866942844812) bank73377 +73378 POINT(40.917019673257734 -73.54688201126045) bank73378 +73379 POINT(39.83846892973143 -73.40938870318317) bank73379 +73380 POINT(39.721414152900735 -74.79917922332997) bank73380 +73381 POINT(40.64321959011019 -73.84306687888514) bank73381 +73382 POINT(41.054726726326216 -74.45727213961408) bank73382 +73383 POINT(40.51701238101319 -73.28321686386192) bank73383 +73384 POINT(40.21539703059211 -73.4414830904804) bank73384 +73385 POINT(39.91035549376775 -73.71791464556833) bank73385 +73386 POINT(39.99639107470955 -73.0508064784491) bank73386 +73387 POINT(40.342891994695876 -74.15787195729538) bank73387 +73388 POINT(40.30449918381837 -73.17830194114033) bank73388 +73389 POINT(41.1593165087094 -74.74197081954655) bank73389 +73390 POINT(40.33056909343214 -74.37747906748844) bank73390 +73391 POINT(41.55905121018303 -74.61791698072321) bank73391 +73392 POINT(39.94140145486803 -74.93535358765958) bank73392 +73393 POINT(40.78836076556188 -74.67692926378572) bank73393 +73394 POINT(39.95949343438406 -73.51137104856684) bank73394 +73395 POINT(41.18831510684705 -74.44547666525136) bank73395 +73396 POINT(40.581773078710846 -74.0312705704744) bank73396 +73397 POINT(40.78362152152741 -74.02495165852383) bank73397 +73398 POINT(40.77296405643416 -73.53724437031327) bank73398 +73399 POINT(40.92214541730151 -73.34199426122694) bank73399 +73400 POINT(40.15857967961273 -73.31410860706576) bank73400 +73401 POINT(41.09316038946091 -73.39350068809526) bank73401 +73402 POINT(40.72337101397452 -73.66178622871571) bank73402 +73403 POINT(41.43377842161762 -73.5581828698173) bank73403 +73404 POINT(39.92097677029963 -74.37440121746725) bank73404 +73405 POINT(41.134405757815614 -74.37919022764598) bank73405 +73406 POINT(41.44731535671811 -74.91275256629197) bank73406 +73407 POINT(41.655757025999044 -73.26251400356509) bank73407 +73408 POINT(41.144285933578985 -74.2030125675629) bank73408 +73409 POINT(40.31946698642273 -74.94500394570423) bank73409 +73410 POINT(41.54522450598453 -73.82978416070868) bank73410 +73411 POINT(40.80509335897128 -74.04632519900059) bank73411 +73412 POINT(40.17664096043044 -73.71128989995434) bank73412 +73413 POINT(41.27742488232859 -73.07769346488348) bank73413 +73414 POINT(40.13964958675091 -74.19356877112439) bank73414 +73415 POINT(41.050171758498784 -74.7065438395347) bank73415 +73416 POINT(40.67179507563622 -73.99918042262512) bank73416 +73417 POINT(41.19104876682195 -73.92223000569591) bank73417 +73418 POINT(40.28249115734428 -73.82331107950354) bank73418 +73419 POINT(41.11151886212253 -74.74312597410298) bank73419 +73420 POINT(41.320054730035245 -74.62680250394828) bank73420 +73421 POINT(39.87313379932411 -74.89609082631762) bank73421 +73422 POINT(41.68933855529499 -74.21456627325867) bank73422 +73423 POINT(40.117388089469976 -74.35326840040236) bank73423 +73424 POINT(40.20431813941193 -73.32906817284086) bank73424 +73425 POINT(40.87095982538247 -73.53615240764817) bank73425 +73426 POINT(40.0038953080755 -74.29184676626816) bank73426 +73427 POINT(41.374925299748384 -74.39368869440996) bank73427 +73428 POINT(40.25024837508579 -73.07238140810891) bank73428 +73429 POINT(40.28044507548054 -74.85256870650689) bank73429 +73430 POINT(41.21000224021201 -74.85310327970872) bank73430 +73431 POINT(41.28056519883618 -74.10607945465769) bank73431 +73432 POINT(40.41199949515799 -73.66903230776278) bank73432 +73433 POINT(40.92501587706391 -73.16343254227635) bank73433 +73434 POINT(40.242589199569466 -74.86195173307281) bank73434 +73435 POINT(39.77883467809036 -74.58349258151767) bank73435 +73436 POINT(41.542570982049625 -74.85959969156117) bank73436 +73437 POINT(40.759840283915636 -73.71398797485578) bank73437 +73438 POINT(40.42329388498303 -73.74485414769524) bank73438 +73439 POINT(40.97756079835783 -74.99118762437703) bank73439 +73440 POINT(40.607806319093676 -73.8300753211371) bank73440 +73441 POINT(40.01072635938453 -73.72515580716681) bank73441 +73442 POINT(39.81931168757689 -74.82293961035013) bank73442 +73443 POINT(41.15342146550916 -74.5574072011114) bank73443 +73444 POINT(40.068836111733866 -73.40019635148082) bank73444 +73445 POINT(39.801905099693066 -73.69713472395956) bank73445 +73446 POINT(40.88275269395217 -74.31970181561213) bank73446 +73447 POINT(40.55589681736781 -73.63339416225837) bank73447 +73448 POINT(41.38612648236087 -73.85746768508784) bank73448 +73449 POINT(40.20004470748855 -74.6702667696166) bank73449 +73450 POINT(41.44075492168293 -74.95721956064362) bank73450 +73451 POINT(41.551809629148345 -74.1973019211167) bank73451 +73452 POINT(41.44548636609415 -74.46340453151097) bank73452 +73453 POINT(41.48333141700199 -74.99708212317142) bank73453 +73454 POINT(40.463145686558185 -73.60808995037075) bank73454 +73455 POINT(40.27498637346012 -73.48069302288826) bank73455 +73456 POINT(40.50680279254443 -73.56130176258134) bank73456 +73457 POINT(41.54097001785094 -73.45599075767689) bank73457 +73458 POINT(39.906880927121065 -73.60524634566174) bank73458 +73459 POINT(40.363438971834206 -73.66686604047672) bank73459 +73460 POINT(41.418485345434036 -73.3742452215895) bank73460 +73461 POINT(40.82409183505707 -74.09669177865254) bank73461 +73462 POINT(40.84897190455146 -73.23493669190191) bank73462 +73463 POINT(40.56489506625923 -73.79358683002692) bank73463 +73464 POINT(39.92409983204764 -74.06126029442235) bank73464 +73465 POINT(39.84254005279231 -73.3222047770986) bank73465 +73466 POINT(40.4930690276843 -73.66123536691414) bank73466 +73467 POINT(39.82883807627541 -73.3279048998915) bank73467 +73468 POINT(40.51849663213067 -74.7455224439863) bank73468 +73469 POINT(41.015114068503536 -74.32921249380092) bank73469 +73470 POINT(41.082373430168325 -73.8162533992362) bank73470 +73471 POINT(41.023327272560266 -74.9693179806529) bank73471 +73472 POINT(39.90065156796393 -73.13779066806654) bank73472 +73473 POINT(40.773661499468346 -74.4249629977659) bank73473 +73474 POINT(40.55385408486832 -74.31725546602965) bank73474 +73475 POINT(40.99407268320712 -74.29565369619675) bank73475 +73476 POINT(41.40238219138645 -74.51767657470272) bank73476 +73477 POINT(41.54577290057881 -73.65050097653709) bank73477 +73478 POINT(40.63620575374447 -74.68548046560886) bank73478 +73479 POINT(40.29716742921856 -73.11546024341153) bank73479 +73480 POINT(41.328114921520616 -73.50885191547196) bank73480 +73481 POINT(39.94879718879267 -73.49793823824159) bank73481 +73482 POINT(41.18532353791294 -73.87060451213367) bank73482 +73483 POINT(40.65912590712124 -73.37186851865116) bank73483 +73484 POINT(41.25417931979662 -74.16931961918338) bank73484 +73485 POINT(40.133755571315255 -73.47155080731125) bank73485 +73486 POINT(41.4488367011411 -73.16295394426709) bank73486 +73487 POINT(40.77195948758715 -73.79774124065308) bank73487 +73488 POINT(41.00506332641501 -74.43662493673045) bank73488 +73489 POINT(40.84777876868758 -73.60891462731469) bank73489 +73490 POINT(41.46436587141703 -73.10932111764491) bank73490 +73491 POINT(40.024302570919964 -73.4761057412817) bank73491 +73492 POINT(40.22066542340444 -73.12752652938669) bank73492 +73493 POINT(41.245982793986784 -73.84940296459074) bank73493 +73494 POINT(39.8595407909755 -73.635274414348) bank73494 +73495 POINT(39.763595009960184 -74.44397323839253) bank73495 +73496 POINT(40.133583630591644 -73.52783240448092) bank73496 +73497 POINT(41.127642062910766 -73.60883868854286) bank73497 +73498 POINT(40.3015148024101 -73.18795840509567) bank73498 +73499 POINT(40.56469178577977 -74.45405115169622) bank73499 +73500 POINT(40.0642934207393 -74.2727584876655) bank73500 +73501 POINT(40.13364878181467 -73.75757283860472) bank73501 +73502 POINT(39.97728406260801 -74.11875765902728) bank73502 +73503 POINT(40.51302515034311 -73.80054537092741) bank73503 +73504 POINT(40.00085656003747 -73.30424243698837) bank73504 +73505 POINT(40.1035260691436 -74.62197986693897) bank73505 +73506 POINT(40.426358031071544 -73.28562105274945) bank73506 +73507 POINT(41.65295508644772 -74.86679389820887) bank73507 +73508 POINT(41.66850219571417 -73.60988128047276) bank73508 +73509 POINT(40.12711423026039 -73.35588549732984) bank73509 +73510 POINT(40.06838346340998 -74.7191101752319) bank73510 +73511 POINT(41.44062213683702 -73.21991314294482) bank73511 +73512 POINT(39.72261282746816 -73.53601518085252) bank73512 +73513 POINT(41.225921607995986 -74.24171401902544) bank73513 +73514 POINT(39.91389779512352 -73.01103177132495) bank73514 +73515 POINT(40.689941985229396 -74.6376750559393) bank73515 +73516 POINT(41.67598934084335 -73.13474676606573) bank73516 +73517 POINT(41.6626146802124 -73.62362330305768) bank73517 +73518 POINT(41.32590799233519 -73.03784610296033) bank73518 +73519 POINT(40.95592826154879 -73.32838984739384) bank73519 +73520 POINT(39.86294010125725 -74.46314741898124) bank73520 +73521 POINT(40.15018276574139 -73.99595170510078) bank73521 +73522 POINT(39.911323404764325 -73.05263993796474) bank73522 +73523 POINT(39.821197580467256 -74.48933095940357) bank73523 +73524 POINT(41.26568912467788 -73.91626385923892) bank73524 +73525 POINT(40.06029747766396 -74.88426622037069) bank73525 +73526 POINT(40.56362160813004 -74.59247624024562) bank73526 +73527 POINT(41.33272905545787 -74.07376323298153) bank73527 +73528 POINT(41.41635581313476 -73.65228936847294) bank73528 +73529 POINT(41.45280815317314 -74.54402904661976) bank73529 +73530 POINT(39.9735507374398 -73.07870149218546) bank73530 +73531 POINT(39.956044316355836 -73.7187268431043) bank73531 +73532 POINT(40.63589108430785 -74.64440940677773) bank73532 +73533 POINT(39.85844605134841 -74.04208494166565) bank73533 +73534 POINT(39.99873730968603 -74.02677732288551) bank73534 +73535 POINT(40.707323295405914 -74.6212906610254) bank73535 +73536 POINT(40.16099379143305 -73.81419819073811) bank73536 +73537 POINT(41.46324375173328 -73.92701255112144) bank73537 +73538 POINT(41.2941201327539 -73.06638008266869) bank73538 +73539 POINT(40.4208867653008 -73.3586583850594) bank73539 +73540 POINT(41.19034004504911 -73.65579498865064) bank73540 +73541 POINT(40.60395388734274 -74.07698865257692) bank73541 +73542 POINT(39.85794780637117 -73.03512633168143) bank73542 +73543 POINT(39.81777750596704 -73.3474878544488) bank73543 +73544 POINT(41.29903707199561 -74.0097046055743) bank73544 +73545 POINT(40.364972724232494 -73.20912906322553) bank73545 +73546 POINT(40.9379468571257 -73.78690165897432) bank73546 +73547 POINT(41.102737585535124 -73.1304864239018) bank73547 +73548 POINT(40.32474550685043 -73.3544955231766) bank73548 +73549 POINT(41.5868592261968 -74.27548471631654) bank73549 +73550 POINT(39.99354060923308 -73.2372773168557) bank73550 +73551 POINT(41.3846347990807 -73.94128390862699) bank73551 +73552 POINT(40.230310577306334 -74.67987085038064) bank73552 +73553 POINT(39.786395401622 -73.00690038332193) bank73553 +73554 POINT(41.598128723733936 -74.89605533667839) bank73554 +73555 POINT(40.323582564178324 -74.91649973343787) bank73555 +73556 POINT(40.938515263978424 -73.80037899645735) bank73556 +73557 POINT(41.25710967372371 -74.17883758556039) bank73557 +73558 POINT(41.52786164160109 -74.76906627155687) bank73558 +73559 POINT(40.25879006162287 -73.51558173444752) bank73559 +73560 POINT(40.085656796166305 -74.84665485614737) bank73560 +73561 POINT(39.97985436325826 -73.39387389258029) bank73561 +73562 POINT(41.37670166448073 -73.69367511326305) bank73562 +73563 POINT(41.4181577552762 -74.3725866367742) bank73563 +73564 POINT(41.070970111370954 -73.86720458791605) bank73564 +73565 POINT(41.28612228999724 -74.91552682658492) bank73565 +73566 POINT(40.781155803956125 -74.05473888716179) bank73566 +73567 POINT(41.3066636127048 -73.93635845836641) bank73567 +73568 POINT(41.28451642964573 -74.215942930641) bank73568 +73569 POINT(40.3392665140007 -73.74648986180348) bank73569 +73570 POINT(39.79679956916155 -73.16688166703565) bank73570 +73571 POINT(40.26180031952708 -73.14588997925512) bank73571 +73572 POINT(41.19851103367066 -73.13241478307643) bank73572 +73573 POINT(40.6425108931607 -73.7603632092713) bank73573 +73574 POINT(41.16463693987829 -73.2603151125828) bank73574 +73575 POINT(41.619564795365214 -73.90177742488166) bank73575 +73576 POINT(40.27208910657907 -73.81094978632393) bank73576 +73577 POINT(40.034551190836154 -74.33388884470651) bank73577 +73578 POINT(40.14745926614758 -74.09246600945536) bank73578 +73579 POINT(40.44489283784051 -74.74438843897634) bank73579 +73580 POINT(39.7354436169653 -74.3593019832792) bank73580 +73581 POINT(40.891525938603834 -73.64646364956178) bank73581 +73582 POINT(41.40159576471116 -73.69377979588113) bank73582 +73583 POINT(41.265491296413835 -74.71526244680064) bank73583 +73584 POINT(41.31645733693869 -73.3759290972621) bank73584 +73585 POINT(39.882986447588 -73.06305042092379) bank73585 +73586 POINT(40.342389423388965 -73.75167369338445) bank73586 +73587 POINT(41.34908105445583 -73.83512095688197) bank73587 +73588 POINT(40.99694298275013 -74.14010739514453) bank73588 +73589 POINT(41.48450034045999 -73.46729739953247) bank73589 +73590 POINT(41.507381725638574 -73.72548608402275) bank73590 +73591 POINT(40.244957241838904 -73.07604247828402) bank73591 +73592 POINT(41.08318746832684 -73.85916036216052) bank73592 +73593 POINT(40.70628407937757 -73.78283573191484) bank73593 +73594 POINT(39.81467046270581 -74.35023837190487) bank73594 +73595 POINT(41.08819495780568 -73.28882482679131) bank73595 +73596 POINT(40.078209400873064 -74.81412627177076) bank73596 +73597 POINT(40.336302160657716 -73.94367960045243) bank73597 +73598 POINT(40.37154059256143 -74.42315027935858) bank73598 +73599 POINT(39.95029968019358 -74.8949522275818) bank73599 +73600 POINT(40.409998385063204 -74.41203895978701) bank73600 +73601 POINT(41.546625133224495 -74.99423277609121) bank73601 +73602 POINT(40.99312981495607 -74.2813270857458) bank73602 +73603 POINT(41.122766278216154 -73.6972563869984) bank73603 +73604 POINT(40.94353495161543 -73.94924380407178) bank73604 +73605 POINT(40.455778209292454 -74.15668855325139) bank73605 +73606 POINT(41.124411619803666 -73.71862705300612) bank73606 +73607 POINT(40.38733415223981 -73.31816432633678) bank73607 +73608 POINT(40.354820081414616 -74.3506006293999) bank73608 +73609 POINT(40.40721729065543 -74.83717131769512) bank73609 +73610 POINT(41.354123986906316 -73.03066567702221) bank73610 +73611 POINT(41.23742572402054 -74.32468532249962) bank73611 +73612 POINT(41.19813500762433 -73.40196276500397) bank73612 +73613 POINT(40.166872764367895 -73.959677541788) bank73613 +73614 POINT(40.11258711327962 -73.88360230048994) bank73614 +73615 POINT(41.16242193859559 -74.39174917193125) bank73615 +73616 POINT(41.01185150444412 -74.18657953653408) bank73616 +73617 POINT(39.742433308578676 -73.8835732193533) bank73617 +73618 POINT(40.85302680870252 -73.75552762160892) bank73618 +73619 POINT(40.55699898746354 -73.22012661598573) bank73619 +73620 POINT(41.13468735585538 -74.58470139931036) bank73620 +73621 POINT(40.68401578878912 -73.18928423439952) bank73621 +73622 POINT(40.58561530861314 -73.03466240603267) bank73622 +73623 POINT(40.95051998552838 -73.4085988340484) bank73623 +73624 POINT(41.07096072049025 -73.26483637468468) bank73624 +73625 POINT(40.94680680380016 -74.13940429555672) bank73625 +73626 POINT(39.909820656581786 -74.09378491543401) bank73626 +73627 POINT(39.89663448869827 -73.9216315634682) bank73627 +73628 POINT(39.901992801747724 -74.55389300795574) bank73628 +73629 POINT(40.8535784951826 -73.26823513163238) bank73629 +73630 POINT(40.432584626690215 -74.37553428398142) bank73630 +73631 POINT(40.16433587078657 -73.54229210427859) bank73631 +73632 POINT(41.68732256468036 -73.63579441277116) bank73632 +73633 POINT(39.763626537883454 -74.90441171786352) bank73633 +73634 POINT(41.22332809223297 -74.9480010222236) bank73634 +73635 POINT(39.80319101052581 -73.40052924700883) bank73635 +73636 POINT(40.76388270796524 -73.06246401355794) bank73636 +73637 POINT(40.79729769567138 -73.4029243974534) bank73637 +73638 POINT(39.745775471898156 -73.09337154276568) bank73638 +73639 POINT(40.55151987153211 -73.81427911331862) bank73639 +73640 POINT(39.81860209744407 -73.33496282657117) bank73640 +73641 POINT(40.20237749423175 -73.33445939374725) bank73641 +73642 POINT(40.12766474440449 -73.96224726842537) bank73642 +73643 POINT(41.288334368350725 -73.04744874045045) bank73643 +73644 POINT(40.99975461232431 -73.87392605594773) bank73644 +73645 POINT(40.052482307158506 -73.71440567524671) bank73645 +73646 POINT(40.57594368744894 -73.18992929313721) bank73646 +73647 POINT(41.289303800569165 -73.66912167637666) bank73647 +73648 POINT(41.06255869648987 -74.7317987836418) bank73648 +73649 POINT(41.14709752414681 -73.75893179534476) bank73649 +73650 POINT(41.35867351711938 -73.83701863101345) bank73650 +73651 POINT(39.75857093589811 -73.9957551125117) bank73651 +73652 POINT(40.49087858649536 -73.52765013940765) bank73652 +73653 POINT(41.451508869742476 -74.10976946861146) bank73653 +73654 POINT(39.74919239362812 -74.03349197348693) bank73654 +73655 POINT(39.796185175202076 -74.10460957947834) bank73655 +73656 POINT(41.23256999656821 -73.73006842925744) bank73656 +73657 POINT(40.543986395270935 -73.25189726656906) bank73657 +73658 POINT(41.2683635056392 -74.11229555232143) bank73658 +73659 POINT(40.90516139190978 -74.74682006661422) bank73659 +73660 POINT(40.03559356503481 -73.49870923096165) bank73660 +73661 POINT(40.07950836630965 -74.31749286051796) bank73661 +73662 POINT(40.099549198329925 -73.05965162485352) bank73662 +73663 POINT(40.71013572283521 -74.36082375093541) bank73663 +73664 POINT(40.83091962674556 -73.08444646846831) bank73664 +73665 POINT(40.22384534822281 -73.45588295061675) bank73665 +73666 POINT(40.707124438142266 -73.47039555871117) bank73666 +73667 POINT(40.79257869417172 -74.62777295030949) bank73667 +73668 POINT(40.990404707381394 -73.47395864798028) bank73668 +73669 POINT(39.990690139138074 -74.28748389501098) bank73669 +73670 POINT(40.96679198113637 -74.10588289178968) bank73670 +73671 POINT(41.65605977534499 -74.1377799825032) bank73671 +73672 POINT(40.805151455393954 -73.0082304949393) bank73672 +73673 POINT(39.92941467637044 -73.6657436448381) bank73673 +73674 POINT(40.98264800595007 -73.54334170966945) bank73674 +73675 POINT(39.94629516580893 -73.27507365179173) bank73675 +73676 POINT(39.726336821676945 -73.75584166939025) bank73676 +73677 POINT(39.99143551804645 -74.54309149765476) bank73677 +73678 POINT(41.27655422332905 -73.70377507084311) bank73678 +73679 POINT(41.615435150002604 -74.76932722857694) bank73679 +73680 POINT(41.56360498843789 -74.16358443839148) bank73680 +73681 POINT(40.91221269775007 -73.70857874667948) bank73681 +73682 POINT(41.40267283448068 -73.07787847096266) bank73682 +73683 POINT(40.30379532664395 -73.60114463105978) bank73683 +73684 POINT(41.158543974870454 -73.99496017650931) bank73684 +73685 POINT(40.91329804376704 -73.43805655402215) bank73685 +73686 POINT(39.766008628422476 -73.29551884030707) bank73686 +73687 POINT(40.01299366508116 -73.93862989513097) bank73687 +73688 POINT(40.21365786757913 -73.75868212820676) bank73688 +73689 POINT(40.54524840298459 -74.59296161133888) bank73689 +73690 POINT(40.79231846174417 -73.17168884149366) bank73690 +73691 POINT(41.383749583990685 -74.94901657517164) bank73691 +73692 POINT(41.095829328004186 -74.50876023091728) bank73692 +73693 POINT(41.211219696019505 -74.57449699431966) bank73693 +73694 POINT(41.180508486004705 -74.43831717544181) bank73694 +73695 POINT(40.73424000963163 -74.8287227504505) bank73695 +73696 POINT(41.069551305788565 -74.8876244846513) bank73696 +73697 POINT(40.759733041583814 -74.93906318433037) bank73697 +73698 POINT(40.72724523064508 -73.35222989912116) bank73698 +73699 POINT(40.50657400968995 -74.02118782543344) bank73699 +73700 POINT(41.06352501800268 -74.73238387677522) bank73700 +73701 POINT(40.25687574200377 -75.00337150167744) bank73701 +73702 POINT(40.81407665301 -73.66515078594672) bank73702 +73703 POINT(40.527983748394355 -73.40816262818255) bank73703 +73704 POINT(40.96076665355066 -74.22187263341486) bank73704 +73705 POINT(40.525370286053025 -74.99441594764157) bank73705 +73706 POINT(41.08195509058836 -74.68303166027329) bank73706 +73707 POINT(39.80648272431562 -73.38297315670049) bank73707 +73708 POINT(41.25726847622551 -74.28468842108325) bank73708 +73709 POINT(40.350637553056565 -73.5998535386168) bank73709 +73710 POINT(40.61518329627193 -74.72291624292674) bank73710 +73711 POINT(40.45315583325401 -74.82095292359637) bank73711 +73712 POINT(39.99643495884553 -74.36400718280788) bank73712 +73713 POINT(41.220702868875165 -73.3620264105575) bank73713 +73714 POINT(41.03477104577561 -73.4194185681841) bank73714 +73715 POINT(40.93095331922318 -74.81311948021018) bank73715 +73716 POINT(40.71539079055914 -74.77285706952206) bank73716 +73717 POINT(40.24910403786401 -73.85007664726275) bank73717 +73718 POINT(40.22161227297257 -74.83517625728241) bank73718 +73719 POINT(40.225749696149 -73.15916463203305) bank73719 +73720 POINT(40.04531228213977 -73.70157903332029) bank73720 +73721 POINT(41.578135496755195 -74.18825579571795) bank73721 +73722 POINT(41.44212669417116 -73.97218592926966) bank73722 +73723 POINT(40.12498645015124 -73.48055077311895) bank73723 +73724 POINT(40.55397755733561 -73.04411437214546) bank73724 +73725 POINT(39.74430613954284 -73.81432935738374) bank73725 +73726 POINT(40.78995994891412 -74.28113941218909) bank73726 +73727 POINT(40.828362490821476 -73.50049653755329) bank73727 +73728 POINT(39.93285559441045 -73.13078933674991) bank73728 +73729 POINT(41.4237086919704 -74.47165947506475) bank73729 +73730 POINT(41.674879374407865 -74.3142013827958) bank73730 +73731 POINT(41.569080124955306 -74.14734276966055) bank73731 +73732 POINT(39.910185652476486 -74.14845581275627) bank73732 +73733 POINT(40.72138383386139 -73.68935851087394) bank73733 +73734 POINT(41.374523120816434 -74.48173212192955) bank73734 +73735 POINT(40.56454362230197 -73.40902964153408) bank73735 +73736 POINT(41.61719562807158 -74.13357785133368) bank73736 +73737 POINT(40.282295877354706 -74.3267251900935) bank73737 +73738 POINT(40.57483303792609 -73.36231226112464) bank73738 +73739 POINT(40.30604055923098 -74.81701277020512) bank73739 +73740 POINT(39.85035297493371 -73.97444482666174) bank73740 +73741 POINT(41.082374468454326 -73.65007255568314) bank73741 +73742 POINT(40.831866161000825 -74.31690527434746) bank73742 +73743 POINT(40.30342572423463 -74.51533158932355) bank73743 +73744 POINT(40.75283911369896 -74.76751765830812) bank73744 +73745 POINT(40.31343686084337 -74.70610028795272) bank73745 +73746 POINT(39.892032559064354 -74.16847388798367) bank73746 +73747 POINT(40.69980434969856 -73.62996051888318) bank73747 +73748 POINT(41.37007183515085 -73.03865343413312) bank73748 +73749 POINT(40.52684441376391 -74.92127236606385) bank73749 +73750 POINT(41.27477603293432 -74.30376658013317) bank73750 +73751 POINT(40.18131184342442 -74.00454076685436) bank73751 +73752 POINT(41.40717979348799 -73.84510831996049) bank73752 +73753 POINT(41.33917975782655 -74.0631375593767) bank73753 +73754 POINT(39.885082052352196 -74.89586578924701) bank73754 +73755 POINT(41.50991915637799 -74.94832317685758) bank73755 +73756 POINT(41.27759237351631 -73.72697898512293) bank73756 +73757 POINT(41.145262524684476 -73.53985371873733) bank73757 +73758 POINT(39.82020235802542 -73.56352823801781) bank73758 +73759 POINT(41.1260113228258 -73.87399233529644) bank73759 +73760 POINT(40.847620250151756 -74.05997529244601) bank73760 +73761 POINT(41.55658857390589 -73.2152171091351) bank73761 +73762 POINT(40.20241160696991 -73.3454183737554) bank73762 +73763 POINT(41.46450524656212 -74.84558545146629) bank73763 +73764 POINT(39.77466647072708 -74.87005456782528) bank73764 +73765 POINT(41.02905890703436 -73.70505209595973) bank73765 +73766 POINT(40.256709105803026 -74.44038442179821) bank73766 +73767 POINT(41.28408991737924 -73.94152787950208) bank73767 +73768 POINT(40.63075602137836 -73.05042632783903) bank73768 +73769 POINT(41.001283133936866 -73.69708185776116) bank73769 +73770 POINT(39.805401983233764 -74.63349201017567) bank73770 +73771 POINT(40.90030839137803 -74.99550525817827) bank73771 +73772 POINT(40.27798533376146 -73.43867436243845) bank73772 +73773 POINT(40.950132145267865 -73.93095231028701) bank73773 +73774 POINT(39.714745722487166 -74.60996452867624) bank73774 +73775 POINT(40.150776787515056 -73.49521290717261) bank73775 +73776 POINT(40.044651004342086 -73.58238516839207) bank73776 +73777 POINT(41.45535111260176 -73.24566306311665) bank73777 +73778 POINT(41.45398715950547 -73.72751022405762) bank73778 +73779 POINT(39.972922327722756 -73.37873782325532) bank73779 +73780 POINT(40.442331474152006 -74.3619239335779) bank73780 +73781 POINT(40.28770100528949 -73.87026984483882) bank73781 +73782 POINT(40.94231046117656 -74.19729208932525) bank73782 +73783 POINT(40.659901818739186 -73.45306840345441) bank73783 +73784 POINT(40.61714455161269 -74.6831979812959) bank73784 +73785 POINT(40.98185005777578 -73.06885730104173) bank73785 +73786 POINT(40.46704109204585 -74.09990597121477) bank73786 +73787 POINT(41.133499099578735 -74.16928346076126) bank73787 +73788 POINT(39.755420201137866 -74.70900074241611) bank73788 +73789 POINT(41.180170184104334 -73.34183864699126) bank73789 +73790 POINT(41.64517583818193 -74.7225935292531) bank73790 +73791 POINT(41.54578358533409 -73.15592112777176) bank73791 +73792 POINT(40.829338524077656 -74.85559702661342) bank73792 +73793 POINT(40.93272882781837 -74.5148285452789) bank73793 +73794 POINT(41.034037144865465 -74.10723685162206) bank73794 +73795 POINT(40.53029767324963 -74.90718862601078) bank73795 +73796 POINT(39.95340417001652 -74.88789656772884) bank73796 +73797 POINT(40.94537274921861 -74.05285688387102) bank73797 +73798 POINT(40.78834558041982 -73.55268310558354) bank73798 +73799 POINT(40.510084535137274 -73.20264439463737) bank73799 +73800 POINT(39.84978440171109 -73.56781396349093) bank73800 +73801 POINT(39.93073626491075 -74.6440431505398) bank73801 +73802 POINT(40.127960501120235 -73.35840885307) bank73802 +73803 POINT(40.315791330928576 -74.94225746844172) bank73803 +73804 POINT(41.374279034779924 -74.30711388175712) bank73804 +73805 POINT(40.682227881272055 -73.33397752352573) bank73805 +73806 POINT(41.58094242763383 -73.45649521167547) bank73806 +73807 POINT(41.22600958080464 -73.1424113965226) bank73807 +73808 POINT(41.618442776372206 -73.02675997246686) bank73808 +73809 POINT(40.41861085042342 -74.29955888410703) bank73809 +73810 POINT(41.5700820523087 -73.81810149241973) bank73810 +73811 POINT(40.774662130318426 -74.72761050989502) bank73811 +73812 POINT(41.659898998189995 -74.42825135042808) bank73812 +73813 POINT(41.23592940719665 -74.99126491772898) bank73813 +73814 POINT(40.57054300860658 -73.43648738414926) bank73814 +73815 POINT(40.40165515377847 -74.35382160462586) bank73815 +73816 POINT(40.89504306980788 -74.61654369584767) bank73816 +73817 POINT(40.369239555695195 -74.33195474938074) bank73817 +73818 POINT(41.02651927681177 -74.22818149911065) bank73818 +73819 POINT(41.67094743726912 -74.02300364002744) bank73819 +73820 POINT(41.52931152560741 -74.36575552685602) bank73820 +73821 POINT(40.27149501663238 -74.47140140745537) bank73821 +73822 POINT(41.01057495077608 -74.51015255259439) bank73822 +73823 POINT(40.357514292498024 -74.58188254010443) bank73823 +73824 POINT(40.66951231163597 -74.9189401141715) bank73824 +73825 POINT(40.929014395400216 -74.68412776991315) bank73825 +73826 POINT(41.470128099796106 -73.8606811152385) bank73826 +73827 POINT(39.932854735906275 -74.50505182540243) bank73827 +73828 POINT(40.71437394592513 -74.65858030497347) bank73828 +73829 POINT(40.951396254173986 -73.85277264740337) bank73829 +73830 POINT(40.79220063603025 -74.58695597197472) bank73830 +73831 POINT(40.20030339531118 -74.516851266476) bank73831 +73832 POINT(40.90543214495504 -73.55640977957805) bank73832 +73833 POINT(41.081140437880585 -74.41481817297462) bank73833 +73834 POINT(40.15295802954924 -73.011471915277) bank73834 +73835 POINT(39.839745522031244 -74.33288369338217) bank73835 +73836 POINT(40.830044538324096 -74.37347212134591) bank73836 +73837 POINT(41.286739496067455 -74.7560440226466) bank73837 +73838 POINT(41.528231270300424 -73.9397924657936) bank73838 +73839 POINT(40.833857593041174 -73.13932635518353) bank73839 +73840 POINT(40.861401016375666 -73.2752287273999) bank73840 +73841 POINT(40.69576396721716 -74.04633384047995) bank73841 +73842 POINT(41.12266944627696 -74.53743181674791) bank73842 +73843 POINT(40.93485115616731 -73.78950540478789) bank73843 +73844 POINT(40.09094074739373 -74.70029663538217) bank73844 +73845 POINT(40.41749174079907 -74.4489999804344) bank73845 +73846 POINT(41.51098636619348 -73.25799237884951) bank73846 +73847 POINT(40.458481912559094 -73.67132553466423) bank73847 +73848 POINT(40.68085129552301 -73.99124854094164) bank73848 +73849 POINT(40.77557746903972 -74.19617149934072) bank73849 +73850 POINT(41.02806160467858 -73.11477382073144) bank73850 +73851 POINT(40.99575305943066 -74.67648363378657) bank73851 +73852 POINT(40.60111563803742 -74.17221846478903) bank73852 +73853 POINT(40.85208708787177 -74.6255212611835) bank73853 +73854 POINT(40.96565469614339 -73.09104447584627) bank73854 +73855 POINT(40.722330373712175 -74.85444633772849) bank73855 +73856 POINT(40.73504971416864 -74.65834850298845) bank73856 +73857 POINT(40.75814710106323 -74.33593012529153) bank73857 +73858 POINT(40.39020154278207 -74.77076290924784) bank73858 +73859 POINT(41.104953559864164 -73.51247945734896) bank73859 +73860 POINT(40.44584808864729 -73.74753870702999) bank73860 +73861 POINT(41.64748888514073 -74.5505063781042) bank73861 +73862 POINT(40.47905169030798 -73.88807653571678) bank73862 +73863 POINT(40.36462353528884 -73.28084144850743) bank73863 +73864 POINT(40.88589672127984 -74.44100401035931) bank73864 +73865 POINT(40.088310647222535 -73.35039406754677) bank73865 +73866 POINT(41.09656282894792 -73.5365518373994) bank73866 +73867 POINT(40.21848669331572 -73.91424178090845) bank73867 +73868 POINT(40.68510666186921 -73.81597843319979) bank73868 +73869 POINT(39.87155822464603 -73.75994668285577) bank73869 +73870 POINT(41.51707008855371 -73.23451578993465) bank73870 +73871 POINT(41.234519806505055 -74.76449170638334) bank73871 +73872 POINT(40.7635107455503 -73.75679513163792) bank73872 +73873 POINT(41.03664244975185 -73.93002252189405) bank73873 +73874 POINT(41.69543195586511 -74.30534993883886) bank73874 +73875 POINT(40.15368153679632 -73.27555965208256) bank73875 +73876 POINT(41.60861008136564 -73.93717468156092) bank73876 +73877 POINT(41.24929871069504 -74.20797022581169) bank73877 +73878 POINT(40.33043045404013 -74.13092643513555) bank73878 +73879 POINT(40.465143795475164 -74.53522673027616) bank73879 +73880 POINT(39.931075706602215 -73.21154733532256) bank73880 +73881 POINT(40.20761338224605 -74.11160503955801) bank73881 +73882 POINT(40.80775468193681 -73.08985811639525) bank73882 +73883 POINT(40.91719864892246 -73.19862895507178) bank73883 +73884 POINT(41.680157438637515 -74.18570869056217) bank73884 +73885 POINT(39.94629580852968 -73.97381301105258) bank73885 +73886 POINT(41.01265127180276 -74.79738360289988) bank73886 +73887 POINT(40.59697232448263 -74.03983816682195) bank73887 +73888 POINT(41.533470559083064 -74.26310032165935) bank73888 +73889 POINT(41.60975860855514 -74.71433189057824) bank73889 +73890 POINT(41.71255553951712 -74.93372651573478) bank73890 +73891 POINT(40.543527266671646 -73.5178095306303) bank73891 +73892 POINT(40.19502284453384 -74.95677092902984) bank73892 +73893 POINT(40.96023413584428 -73.57770477901549) bank73893 +73894 POINT(40.2522964192211 -74.62578528661633) bank73894 +73895 POINT(40.33743531717109 -74.62050935333801) bank73895 +73896 POINT(41.46142919166801 -74.66854979737067) bank73896 +73897 POINT(39.94860675321533 -74.22928912665584) bank73897 +73898 POINT(41.414423626533306 -73.39831817761657) bank73898 +73899 POINT(40.0626224792735 -74.03724903190646) bank73899 +73900 POINT(40.01281311714937 -73.71190304259079) bank73900 +73901 POINT(40.01071553780775 -74.05315292534975) bank73901 +73902 POINT(39.919914099521506 -73.81915908049453) bank73902 +73903 POINT(40.797993433669724 -73.8252827927821) bank73903 +73904 POINT(40.47180298062293 -73.6550403916926) bank73904 +73905 POINT(40.70530537586828 -73.57184200547627) bank73905 +73906 POINT(40.610528829020886 -74.14212386902636) bank73906 +73907 POINT(40.030149035953734 -74.3263851608578) bank73907 +73908 POINT(39.87613444177812 -73.2908414982943) bank73908 +73909 POINT(39.74972179242039 -74.43542325665898) bank73909 +73910 POINT(41.54156705451317 -73.65625499588616) bank73910 +73911 POINT(40.35818417096668 -73.34512638224287) bank73911 +73912 POINT(41.64143207361865 -74.82059693924288) bank73912 +73913 POINT(41.57324743090526 -74.62534342307426) bank73913 +73914 POINT(39.820084841894534 -74.95091117526542) bank73914 +73915 POINT(41.12089200444957 -73.57193337696496) bank73915 +73916 POINT(40.76955259114054 -74.05667527892473) bank73916 +73917 POINT(40.7609962026369 -74.00237022032375) bank73917 +73918 POINT(41.51120390648909 -73.61015411236741) bank73918 +73919 POINT(41.36656106506603 -74.17507196894621) bank73919 +73920 POINT(40.90468634499148 -74.99856024272995) bank73920 +73921 POINT(39.873772267189786 -74.78735389979104) bank73921 +73922 POINT(40.56346980177468 -74.87379171139602) bank73922 +73923 POINT(40.03732063941177 -74.35689823039505) bank73923 +73924 POINT(41.531220552839926 -74.73670006560528) bank73924 +73925 POINT(40.35381889841571 -73.90505527170106) bank73925 +73926 POINT(40.387606050017986 -73.5269306998048) bank73926 +73927 POINT(40.94498622927286 -74.10930002285701) bank73927 +73928 POINT(40.8365401706697 -73.32838559734782) bank73928 +73929 POINT(39.785077316479345 -74.51137991104825) bank73929 +73930 POINT(40.873149166318804 -74.89539547644927) bank73930 +73931 POINT(39.84461666502564 -74.30137040861885) bank73931 +73932 POINT(41.38346670003673 -73.97723334113495) bank73932 +73933 POINT(39.889232319710736 -73.85820642992974) bank73933 +73934 POINT(40.92174691836187 -74.0658248994348) bank73934 +73935 POINT(41.021749889548644 -73.47067817478553) bank73935 +73936 POINT(40.209763354084 -74.74971245523982) bank73936 +73937 POINT(40.41811639289127 -74.04415735748147) bank73937 +73938 POINT(41.07091275229585 -73.43563273733734) bank73938 +73939 POINT(40.61256137971138 -74.05075249618008) bank73939 +73940 POINT(41.10662703436139 -73.43948254632494) bank73940 +73941 POINT(40.16053318171999 -74.55671782435897) bank73941 +73942 POINT(41.39877426484449 -73.10531189677762) bank73942 +73943 POINT(41.564309914799246 -73.86718309508707) bank73943 +73944 POINT(41.48881265785452 -74.66317778294226) bank73944 +73945 POINT(41.18521899009037 -74.64921475462826) bank73945 +73946 POINT(40.32909157437144 -73.89535947764928) bank73946 +73947 POINT(39.852078059166985 -74.91148722019574) bank73947 +73948 POINT(40.82671796642271 -74.14070554358226) bank73948 +73949 POINT(40.07900354156209 -73.3566426784172) bank73949 +73950 POINT(40.48451864597731 -74.58448522673305) bank73950 +73951 POINT(39.73378916435786 -74.0505697361737) bank73951 +73952 POINT(40.73756857002855 -73.83834960123349) bank73952 +73953 POINT(40.40748890480008 -73.92280141471686) bank73953 +73954 POINT(39.93254667935709 -74.54130864315992) bank73954 +73955 POINT(41.264184584222505 -73.3220644783412) bank73955 +73956 POINT(41.22107824053562 -74.05415761451847) bank73956 +73957 POINT(40.760170641439835 -73.03690348642552) bank73957 +73958 POINT(40.48128769165513 -73.0383909147812) bank73958 +73959 POINT(40.81467133767601 -74.28679246594363) bank73959 +73960 POINT(40.09133493817752 -74.46237265965445) bank73960 +73961 POINT(40.77892210439709 -74.30255035984761) bank73961 +73962 POINT(41.47174548715293 -74.19346698308772) bank73962 +73963 POINT(40.97006111359268 -74.87261172935672) bank73963 +73964 POINT(40.846796244855106 -74.8954668769001) bank73964 +73965 POINT(40.37722085864389 -73.47802951932891) bank73965 +73966 POINT(41.438391594935354 -73.15209666775222) bank73966 +73967 POINT(41.59376925275399 -74.119598232164) bank73967 +73968 POINT(40.39979146809618 -73.90189410600998) bank73968 +73969 POINT(40.42098410431316 -73.27845681141845) bank73969 +73970 POINT(40.20335245326521 -73.39513228322944) bank73970 +73971 POINT(40.95019104311977 -74.5422527537246) bank73971 +73972 POINT(40.288622968706775 -74.10027738522619) bank73972 +73973 POINT(39.77922212497875 -74.25327753154762) bank73973 +73974 POINT(39.899598682244445 -74.42250536409202) bank73974 +73975 POINT(41.48007038617503 -74.01458309024336) bank73975 +73976 POINT(40.35416269459975 -74.63530050138966) bank73976 +73977 POINT(40.18634499807317 -73.01550292422905) bank73977 +73978 POINT(41.631145689141576 -73.98478667902003) bank73978 +73979 POINT(41.68960181014167 -74.72242530715123) bank73979 +73980 POINT(40.073812476952966 -74.33170157695004) bank73980 +73981 POINT(41.1493812871143 -73.46208448754707) bank73981 +73982 POINT(40.31645380963081 -73.92902922613881) bank73982 +73983 POINT(41.490672813761435 -74.68071543414084) bank73983 +73984 POINT(40.52981645728319 -73.70584551291263) bank73984 +73985 POINT(40.308254332617786 -73.03671011399234) bank73985 +73986 POINT(40.121456864986534 -73.86110367850185) bank73986 +73987 POINT(41.41474348109112 -73.64322783804744) bank73987 +73988 POINT(39.79458649900658 -74.10122653618897) bank73988 +73989 POINT(40.416331939671565 -74.66844360799277) bank73989 +73990 POINT(41.389835990739826 -74.42260598673596) bank73990 +73991 POINT(40.99532217087666 -73.5394942556653) bank73991 +73992 POINT(41.21539635569002 -74.00008261058154) bank73992 +73993 POINT(40.92959112081396 -74.73016453960514) bank73993 +73994 POINT(41.08063596691148 -74.04246415157381) bank73994 +73995 POINT(39.960702444943834 -74.07408398578784) bank73995 +73996 POINT(41.71161233814866 -74.57245446177866) bank73996 +73997 POINT(39.85006105882146 -74.67238830104725) bank73997 +73998 POINT(40.29775672113367 -74.02976965935412) bank73998 +73999 POINT(40.57343278268156 -74.36608511925112) bank73999 +74000 POINT(40.69583207902344 -74.07930696010693) bank74000 +74001 POINT(41.559191088150314 -74.27340645619607) bank74001 +74002 POINT(40.594536284809486 -73.38711167211538) bank74002 +74003 POINT(41.14109210775177 -74.79858589807507) bank74003 +74004 POINT(39.902998946492865 -74.36653987662484) bank74004 +74005 POINT(40.35127260163226 -74.017688038986) bank74005 +74006 POINT(40.90999493237938 -73.85279239585694) bank74006 +74007 POINT(40.14722031402576 -73.73190553637689) bank74007 +74008 POINT(40.239496134104186 -73.89101592055297) bank74008 +74009 POINT(40.892311853665504 -73.38145473202147) bank74009 +74010 POINT(40.913965663089094 -74.92441145416733) bank74010 +74011 POINT(41.38452999256017 -73.06404527987296) bank74011 +74012 POINT(39.986695995697 -74.03097954574311) bank74012 +74013 POINT(41.16428462511874 -73.24983412105293) bank74013 +74014 POINT(40.68276210175452 -73.84221191096856) bank74014 +74015 POINT(41.62807269664265 -74.98257155233478) bank74015 +74016 POINT(41.21206558647857 -73.36505350280345) bank74016 +74017 POINT(41.38485410516952 -73.83946875336572) bank74017 +74018 POINT(41.675496897163015 -73.70852366504597) bank74018 +74019 POINT(40.860029727749726 -74.15946445542367) bank74019 +74020 POINT(39.77684315986849 -73.40066376291716) bank74020 +74021 POINT(39.85755799943447 -73.15443592062255) bank74021 +74022 POINT(41.31026781732807 -73.17391227489384) bank74022 +74023 POINT(39.99856915345198 -73.04281568320968) bank74023 +74024 POINT(41.463771203953875 -73.35375615185411) bank74024 +74025 POINT(40.545014503841514 -73.35745610000919) bank74025 +74026 POINT(41.031186335115315 -74.13460022136864) bank74026 +74027 POINT(40.540980616650735 -74.4450752887047) bank74027 +74028 POINT(41.148689433460945 -73.86635337038696) bank74028 +74029 POINT(40.90496780854144 -74.59565635053133) bank74029 +74030 POINT(40.11398542746389 -73.46942853438034) bank74030 +74031 POINT(40.961609649020005 -73.88122901461907) bank74031 +74032 POINT(39.74369623381156 -73.35471697915179) bank74032 +74033 POINT(40.644043414075355 -74.1937021618146) bank74033 +74034 POINT(39.86200111382213 -73.68481183545941) bank74034 +74035 POINT(40.407827488346214 -74.10727162616999) bank74035 +74036 POINT(41.1303590382262 -74.8954613278429) bank74036 +74037 POINT(39.78737542438656 -73.96517919600481) bank74037 +74038 POINT(40.64915948959538 -74.26716873808208) bank74038 +74039 POINT(41.37952819621229 -73.3747712963957) bank74039 +74040 POINT(41.22609950047364 -73.45575924808406) bank74040 +74041 POINT(41.04171575578982 -73.16405920471401) bank74041 +74042 POINT(41.342636401324356 -74.96198404260956) bank74042 +74043 POINT(39.721710445382676 -73.22630087664689) bank74043 +74044 POINT(40.9915916128922 -73.82557281369186) bank74044 +74045 POINT(40.47507276899975 -74.29367276645827) bank74045 +74046 POINT(39.921649960820865 -73.22840157965575) bank74046 +74047 POINT(41.36420700646986 -74.18722984440527) bank74047 +74048 POINT(40.292202107872306 -74.86819623158492) bank74048 +74049 POINT(40.148699138114935 -74.73296016498458) bank74049 +74050 POINT(40.04866355489044 -74.66305747943743) bank74050 +74051 POINT(40.85538266719882 -73.95190759557933) bank74051 +74052 POINT(41.62968252124284 -73.5341347906114) bank74052 +74053 POINT(40.78014375268495 -73.37205597945757) bank74053 +74054 POINT(40.71733886522429 -73.19647410010883) bank74054 +74055 POINT(41.17912047932994 -73.77391981603122) bank74055 +74056 POINT(41.496487580582524 -73.15781262731434) bank74056 +74057 POINT(40.51643331878871 -74.45553149336222) bank74057 +74058 POINT(41.42741105408449 -74.05362685590512) bank74058 +74059 POINT(40.01503536496337 -74.86450860138548) bank74059 +74060 POINT(41.05904442323157 -74.19269963380495) bank74060 +74061 POINT(40.82382645456459 -74.03446352238741) bank74061 +74062 POINT(40.22907443178968 -74.36444326002146) bank74062 +74063 POINT(40.29723125955346 -74.87887151307932) bank74063 +74064 POINT(40.661061346298595 -73.0772163928648) bank74064 +74065 POINT(41.05340209600504 -73.57976507680188) bank74065 +74066 POINT(41.62976008588311 -73.70516649730162) bank74066 +74067 POINT(41.07541631507469 -74.50769549032718) bank74067 +74068 POINT(40.43005885807299 -74.00921686060263) bank74068 +74069 POINT(40.129584708644224 -74.24515873654491) bank74069 +74070 POINT(41.03648193548024 -73.26869557149242) bank74070 +74071 POINT(41.40830140272967 -74.26469079684858) bank74071 +74072 POINT(40.51052792075661 -74.2499509041077) bank74072 +74073 POINT(40.59509777726264 -73.76772309042958) bank74073 +74074 POINT(40.50162380083522 -74.89642951626018) bank74074 +74075 POINT(40.114049625682064 -74.45712484762518) bank74075 +74076 POINT(40.991670057990575 -73.27223683336634) bank74076 +74077 POINT(40.35683802946889 -73.86220958477783) bank74077 +74078 POINT(41.205118204808876 -74.19059467257485) bank74078 +74079 POINT(40.85410363944079 -73.51704451068605) bank74079 +74080 POINT(41.26113039622423 -73.5757451117404) bank74080 +74081 POINT(41.345790579647094 -73.01131039444584) bank74081 +74082 POINT(41.308697247580746 -73.05660794170956) bank74082 +74083 POINT(40.65364900581979 -74.43940979111387) bank74083 +74084 POINT(40.05055518302429 -74.78223374070889) bank74084 +74085 POINT(40.77590850259487 -74.20599962822413) bank74085 +74086 POINT(41.21359552733588 -74.37983320642795) bank74086 +74087 POINT(40.10311038851353 -74.52230334415616) bank74087 +74088 POINT(41.055559718354985 -74.8162501239821) bank74088 +74089 POINT(40.57694604678622 -73.67667492294404) bank74089 +74090 POINT(39.77934765907379 -73.65814388354936) bank74090 +74091 POINT(41.33407885446177 -74.7058754071888) bank74091 +74092 POINT(41.57151965076075 -74.91111672392871) bank74092 +74093 POINT(41.629798978284455 -74.91919234241021) bank74093 +74094 POINT(40.33300477503345 -74.22795438638788) bank74094 +74095 POINT(40.880664814389036 -74.85433895739229) bank74095 +74096 POINT(41.698543587358714 -73.56460066156272) bank74096 +74097 POINT(40.71239276373476 -73.56707473716477) bank74097 +74098 POINT(40.475438527213356 -73.60990629116704) bank74098 +74099 POINT(40.56643589049822 -74.51222635810115) bank74099 +74100 POINT(40.91208727506577 -73.29113663249562) bank74100 +74101 POINT(39.751655486751936 -74.89358395059519) bank74101 +74102 POINT(40.04699460060694 -74.18517470964791) bank74102 +74103 POINT(40.268031959370404 -74.30700229901018) bank74103 +74104 POINT(40.535069672721704 -73.92244679285103) bank74104 +74105 POINT(39.839589764982826 -74.082822963633) bank74105 +74106 POINT(40.905268223062535 -74.06651610173793) bank74106 +74107 POINT(40.27974016478768 -73.61510272607578) bank74107 +74108 POINT(40.36618771665853 -73.30977919094792) bank74108 +74109 POINT(40.315458733538456 -73.83171241697188) bank74109 +74110 POINT(41.285563056368794 -73.52075526513232) bank74110 +74111 POINT(40.94852395262417 -73.62804416216805) bank74111 +74112 POINT(40.150475631903 -74.25885719907991) bank74112 +74113 POINT(40.98782207166583 -74.70568111411046) bank74113 +74114 POINT(41.10417511582567 -73.05696586311676) bank74114 +74115 POINT(41.706098171686186 -74.80163396226631) bank74115 +74116 POINT(40.326630919513974 -73.50289410927724) bank74116 +74117 POINT(40.31483388050148 -73.39454039879527) bank74117 +74118 POINT(40.10911019425886 -73.29379201221883) bank74118 +74119 POINT(40.649415605175236 -74.1444566006408) bank74119 +74120 POINT(40.58480396709014 -73.39172834523366) bank74120 +74121 POINT(39.94635191298145 -73.83640790439279) bank74121 +74122 POINT(40.17342933726651 -73.78131473918187) bank74122 +74123 POINT(40.44852419797178 -74.139749310367) bank74123 +74124 POINT(39.726810923785 -74.2544616536599) bank74124 +74125 POINT(41.18806617222477 -74.23205916956226) bank74125 +74126 POINT(40.43948758998489 -73.1920922637508) bank74126 +74127 POINT(41.54369181096724 -73.56426678993006) bank74127 +74128 POINT(41.567085826597825 -74.66544481236696) bank74128 +74129 POINT(40.560478523266234 -74.75580210578174) bank74129 +74130 POINT(40.155353800461704 -73.63270931374667) bank74130 +74131 POINT(40.426603565583726 -73.75961718236601) bank74131 +74132 POINT(40.74845225576934 -73.01749314949214) bank74132 +74133 POINT(40.37328064814812 -74.64036718811184) bank74133 +74134 POINT(40.15175593377469 -74.00904920427661) bank74134 +74135 POINT(40.212683947676936 -74.16092926176817) bank74135 +74136 POINT(39.74749316366107 -74.64927498062531) bank74136 +74137 POINT(39.82800341708556 -73.42620040633693) bank74137 +74138 POINT(40.61000120760345 -74.24452692910211) bank74138 +74139 POINT(41.018675344755984 -74.37821079039146) bank74139 +74140 POINT(41.69294452506362 -73.04855703304605) bank74140 +74141 POINT(40.571827626237805 -73.00735461329118) bank74141 +74142 POINT(39.76570469752515 -73.49376078242788) bank74142 +74143 POINT(40.94037141390584 -74.05839550097389) bank74143 +74144 POINT(39.960882627496055 -74.24075728260898) bank74144 +74145 POINT(39.77184039909074 -74.94300005021675) bank74145 +74146 POINT(41.35963000544565 -73.71523904256317) bank74146 +74147 POINT(40.785085137684064 -73.78113585291327) bank74147 +74148 POINT(40.81792576910117 -73.26993904025666) bank74148 +74149 POINT(39.84763114017551 -73.89859001511223) bank74149 +74150 POINT(40.28359030055141 -74.03904684330622) bank74150 +74151 POINT(41.592867851967156 -74.80637098123306) bank74151 +74152 POINT(41.709536341117534 -74.79606853383187) bank74152 +74153 POINT(40.06567246742922 -74.27185278490359) bank74153 +74154 POINT(40.28456656984006 -74.34677953627494) bank74154 +74155 POINT(41.56283563136769 -74.99615182519254) bank74155 +74156 POINT(40.60631763959454 -74.83669720034139) bank74156 +74157 POINT(39.947705812529364 -73.01335224574345) bank74157 +74158 POINT(40.147300517019076 -73.71205649195241) bank74158 +74159 POINT(40.24815341887114 -74.28714876209315) bank74159 +74160 POINT(40.466153834703135 -73.3230216027748) bank74160 +74161 POINT(40.6801813066578 -73.42211704675744) bank74161 +74162 POINT(39.76818014074622 -74.31440575773809) bank74162 +74163 POINT(40.36223188273593 -73.97517768756123) bank74163 +74164 POINT(40.54557331164993 -74.46103113281514) bank74164 +74165 POINT(40.72815398812992 -73.89021576979763) bank74165 +74166 POINT(40.22885873553727 -74.97721096306178) bank74166 +74167 POINT(40.62223905047914 -74.31888612007793) bank74167 +74168 POINT(41.410703572117654 -73.60840716787719) bank74168 +74169 POINT(41.533635186972944 -74.22066217866916) bank74169 +74170 POINT(40.614320161618664 -73.54543611839381) bank74170 +74171 POINT(40.703999668982966 -73.28555520967531) bank74171 +74172 POINT(40.86161872338147 -73.62338647499541) bank74172 +74173 POINT(40.48373013094523 -74.46511527506982) bank74173 +74174 POINT(41.38763035621082 -74.39586807456753) bank74174 +74175 POINT(41.51398808758323 -74.36061195086906) bank74175 +74176 POINT(40.73617862567842 -74.82298497504469) bank74176 +74177 POINT(40.085520133113356 -74.33671170037468) bank74177 +74178 POINT(41.60990442939017 -73.30704675748541) bank74178 +74179 POINT(40.18436681765412 -73.42515750388473) bank74179 +74180 POINT(40.476828594579864 -74.47586135102983) bank74180 +74181 POINT(40.610856232791704 -74.7658451995725) bank74181 +74182 POINT(41.669328716265774 -74.35366302958423) bank74182 +74183 POINT(40.14557336374858 -74.27641924866188) bank74183 +74184 POINT(39.84384937744438 -73.45094591981407) bank74184 +74185 POINT(40.38137290464618 -74.79341940362106) bank74185 +74186 POINT(41.007101558911565 -74.23843431582827) bank74186 +74187 POINT(41.2996691794402 -73.35415297265006) bank74187 +74188 POINT(41.40249105196574 -73.81236866954075) bank74188 +74189 POINT(40.95148723986616 -73.30247202726991) bank74189 +74190 POINT(40.6341215116236 -74.2859475332594) bank74190 +74191 POINT(40.498515958385084 -73.96740790648391) bank74191 +74192 POINT(39.96740130513027 -73.26318120077428) bank74192 +74193 POINT(40.268357623557485 -73.86035513789933) bank74193 +74194 POINT(41.19005734764819 -73.09964453191485) bank74194 +74195 POINT(40.85033615935731 -74.47070645607774) bank74195 +74196 POINT(40.4379686632504 -73.50523601720958) bank74196 +74197 POINT(41.33227194745276 -73.42351793778816) bank74197 +74198 POINT(39.92304890719554 -74.62945949320648) bank74198 +74199 POINT(41.64692898399757 -73.68802498639931) bank74199 +74200 POINT(40.99318826337919 -73.5762303310058) bank74200 +74201 POINT(40.67740209690099 -74.32368781190989) bank74201 +74202 POINT(40.12676997792626 -74.20335391191949) bank74202 +74203 POINT(40.61462303841954 -74.01780391090162) bank74203 +74204 POINT(41.025479151336626 -73.9353231537498) bank74204 +74205 POINT(40.97839557135882 -73.41495643738227) bank74205 +74206 POINT(39.95173837415844 -73.15649823338183) bank74206 +74207 POINT(41.46919459810491 -74.4080305893783) bank74207 +74208 POINT(41.4373702750342 -75.00442707755063) bank74208 +74209 POINT(39.92089837489877 -74.56154688262298) bank74209 +74210 POINT(41.2539368826162 -73.86857524538225) bank74210 +74211 POINT(39.81885026625644 -73.40456735675723) bank74211 +74212 POINT(39.84761557110547 -74.6451390467081) bank74212 +74213 POINT(41.230926764607844 -73.21920786807304) bank74213 +74214 POINT(40.60133573703804 -73.53702161881728) bank74214 +74215 POINT(41.69741902699015 -73.35810095292287) bank74215 +74216 POINT(39.77202893291651 -73.43870299886098) bank74216 +74217 POINT(40.0794402828171 -74.75919304458256) bank74217 +74218 POINT(40.727168270098 -74.17707142510896) bank74218 +74219 POINT(40.38006977106015 -74.4534442218734) bank74219 +74220 POINT(40.14734975674767 -73.21864115591195) bank74220 +74221 POINT(39.93434794281046 -73.21603418454758) bank74221 +74222 POINT(40.229276979566855 -73.27159683361202) bank74222 +74223 POINT(40.60564663322756 -73.84989334113864) bank74223 +74224 POINT(39.811295752398955 -73.23925390372686) bank74224 +74225 POINT(39.908560137780896 -73.3263718747086) bank74225 +74226 POINT(40.07840598429515 -73.22453710754344) bank74226 +74227 POINT(41.574094444075854 -74.81538587353828) bank74227 +74228 POINT(41.20516709063461 -73.39811598363204) bank74228 +74229 POINT(40.11303835308799 -74.02746839161942) bank74229 +74230 POINT(39.86177661758838 -73.37452955771468) bank74230 +74231 POINT(39.75718060485348 -74.4681371952348) bank74231 +74232 POINT(40.71531090723983 -74.13052174230337) bank74232 +74233 POINT(40.88258489318705 -73.87739073558208) bank74233 +74234 POINT(41.57655143083203 -74.33371433604367) bank74234 +74235 POINT(41.47232400991394 -73.43621145505578) bank74235 +74236 POINT(40.587992971095055 -73.15715521687405) bank74236 +74237 POINT(41.14918778273925 -73.41643394117084) bank74237 +74238 POINT(40.413940997201685 -74.1889849101105) bank74238 +74239 POINT(41.41168678072206 -73.92583732928551) bank74239 +74240 POINT(40.58239185007347 -74.71854691804643) bank74240 +74241 POINT(40.89255844056923 -74.15562765438585) bank74241 +74242 POINT(41.38507059203092 -73.11464891050241) bank74242 +74243 POINT(41.567188555910505 -74.23266626759444) bank74243 +74244 POINT(40.48390832058886 -73.07432098197457) bank74244 +74245 POINT(39.96220983353405 -74.17752119920212) bank74245 +74246 POINT(41.24938521997586 -74.5624620938293) bank74246 +74247 POINT(40.81361957303183 -73.17236515112614) bank74247 +74248 POINT(39.89928680679431 -74.17332711763332) bank74248 +74249 POINT(41.41688087690096 -74.50567547812622) bank74249 +74250 POINT(39.96917351829718 -73.2258330500375) bank74250 +74251 POINT(40.564448559717206 -73.19901099084578) bank74251 +74252 POINT(40.75058475243727 -73.22100584447952) bank74252 +74253 POINT(40.089041611280415 -73.98638021527704) bank74253 +74254 POINT(40.445851760341945 -73.92677852316383) bank74254 +74255 POINT(40.92077431875519 -73.79292883869161) bank74255 +74256 POINT(40.166704646615656 -74.64002811143841) bank74256 +74257 POINT(39.80353057547496 -73.80359248729356) bank74257 +74258 POINT(41.6639982752167 -73.5288840138222) bank74258 +74259 POINT(40.46405963155189 -73.7885805571521) bank74259 +74260 POINT(41.278154756059 -73.93319051391052) bank74260 +74261 POINT(41.18882387861776 -74.2220388347519) bank74261 +74262 POINT(40.44257659641834 -74.75367275886875) bank74262 +74263 POINT(40.167262804686125 -74.92938284057719) bank74263 +74264 POINT(40.22543887979276 -74.3107390237826) bank74264 +74265 POINT(41.68873010771142 -73.5886506410378) bank74265 +74266 POINT(41.3292520681182 -74.07085683099366) bank74266 +74267 POINT(39.89291618900546 -74.62737644246437) bank74267 +74268 POINT(40.3115978722599 -74.92474468334969) bank74268 +74269 POINT(41.18654902435745 -73.29939279312981) bank74269 +74270 POINT(39.737025953919776 -73.96255565747101) bank74270 +74271 POINT(40.36494012195973 -74.88915568923949) bank74271 +74272 POINT(40.34839368235236 -74.45799997217829) bank74272 +74273 POINT(41.238938476351336 -73.20657416828735) bank74273 +74274 POINT(40.45024111654534 -73.35385773184201) bank74274 +74275 POINT(40.97755838934929 -73.83548672191542) bank74275 +74276 POINT(41.68173147040789 -73.731372123476) bank74276 +74277 POINT(40.785410058803855 -74.4708124461338) bank74277 +74278 POINT(40.3639371862034 -73.1800813055362) bank74278 +74279 POINT(41.0004577294704 -74.44189933313196) bank74279 +74280 POINT(41.689985989343405 -74.94807519490864) bank74280 +74281 POINT(40.01454513298373 -74.432289901918) bank74281 +74282 POINT(41.317100342111644 -74.38124511927953) bank74282 +74283 POINT(40.297095739832514 -73.36506787215919) bank74283 +74284 POINT(40.43088055811453 -73.93321577718861) bank74284 +74285 POINT(41.61284854680229 -73.0170245666867) bank74285 +74286 POINT(39.85417770162221 -74.04764401599671) bank74286 +74287 POINT(40.60565481606746 -74.82881528634077) bank74287 +74288 POINT(41.50456208880141 -73.71505915495179) bank74288 +74289 POINT(40.20672573464559 -73.28649328678344) bank74289 +74290 POINT(40.8858482430459 -73.46237396864984) bank74290 +74291 POINT(41.28338911698254 -74.85538791125444) bank74291 +74292 POINT(41.61357377535567 -74.16392100400279) bank74292 +74293 POINT(40.2684858584357 -74.58674762392326) bank74293 +74294 POINT(40.43379377777923 -74.13401887716452) bank74294 +74295 POINT(40.65437462606766 -74.87095921852038) bank74295 +74296 POINT(40.7964914258164 -74.1418983129607) bank74296 +74297 POINT(41.63866176730924 -74.87133209058192) bank74297 +74298 POINT(40.33259920332101 -73.37235735876864) bank74298 +74299 POINT(41.04708023757171 -73.31202095305422) bank74299 +74300 POINT(40.65612849539965 -73.40362601932635) bank74300 +74301 POINT(40.14720264966649 -73.28721140234666) bank74301 +74302 POINT(40.677022816622554 -73.37988124189964) bank74302 +74303 POINT(40.31422643781458 -73.60339712911905) bank74303 +74304 POINT(40.519021586869364 -74.89557551514534) bank74304 +74305 POINT(41.22487369528115 -74.36501443292767) bank74305 +74306 POINT(41.04814983445067 -73.89333283680223) bank74306 +74307 POINT(41.21548177250872 -74.92003334084889) bank74307 +74308 POINT(40.36576995562311 -73.02538078142895) bank74308 +74309 POINT(40.16218026845211 -74.05219624241923) bank74309 +74310 POINT(41.635912212924204 -74.6993549721213) bank74310 +74311 POINT(40.15954254687255 -73.65852055120124) bank74311 +74312 POINT(40.68113979079991 -73.82491929761) bank74312 +74313 POINT(39.91183310040883 -73.92557972663553) bank74313 +74314 POINT(39.78408743963658 -74.78128975016324) bank74314 +74315 POINT(40.920137282918965 -74.31630541143885) bank74315 +74316 POINT(40.93362677143907 -74.14297364738637) bank74316 +74317 POINT(41.34458280911915 -73.66576111630525) bank74317 +74318 POINT(40.46731266945536 -74.75379094416893) bank74318 +74319 POINT(41.28796139295593 -74.1237840915811) bank74319 +74320 POINT(41.08540805949362 -74.38148875031312) bank74320 +74321 POINT(41.56483933931329 -74.716708844517) bank74321 +74322 POINT(41.25950704150016 -73.48356806877469) bank74322 +74323 POINT(40.03101101195781 -73.21636025682248) bank74323 +74324 POINT(41.013030908862106 -73.45242701923408) bank74324 +74325 POINT(40.15835924987088 -74.35575120167786) bank74325 +74326 POINT(41.67560672171165 -74.81754943275101) bank74326 +74327 POINT(40.19372832969591 -73.74524687603875) bank74327 +74328 POINT(40.2406622215786 -74.94480516215518) bank74328 +74329 POINT(40.36832891372868 -74.84442015269742) bank74329 +74330 POINT(41.292154780360086 -73.95054197370706) bank74330 +74331 POINT(40.13623463407136 -74.48880201004346) bank74331 +74332 POINT(39.89570574682784 -73.66051819285849) bank74332 +74333 POINT(40.16005688115761 -74.26633949852769) bank74333 +74334 POINT(40.59426184821637 -73.28707185347731) bank74334 +74335 POINT(40.02909326613373 -73.02459319289412) bank74335 +74336 POINT(41.46565629838141 -73.22305724972706) bank74336 +74337 POINT(39.76448079486019 -74.8373530026936) bank74337 +74338 POINT(40.17812742874619 -74.65554807611517) bank74338 +74339 POINT(41.70324333223906 -73.49229677458521) bank74339 +74340 POINT(41.15986966586277 -73.59053265633978) bank74340 +74341 POINT(40.62536721507454 -73.22154676028399) bank74341 +74342 POINT(40.257595884588454 -73.09819164330906) bank74342 +74343 POINT(40.05008150636915 -74.72100411888805) bank74343 +74344 POINT(40.44730946116579 -73.41386508912497) bank74344 +74345 POINT(39.93021883573741 -73.31884982040917) bank74345 +74346 POINT(40.07925698345737 -73.4523512330468) bank74346 +74347 POINT(41.55431278104057 -73.44402638165481) bank74347 +74348 POINT(39.798619817264154 -73.3053961753346) bank74348 +74349 POINT(40.615023316462675 -73.18618863666423) bank74349 +74350 POINT(40.05681757508974 -74.1096548594247) bank74350 +74351 POINT(41.683043375805966 -73.59167184251767) bank74351 +74352 POINT(39.875350123668476 -73.70561841351716) bank74352 +74353 POINT(41.451757286502726 -73.54104523881443) bank74353 +74354 POINT(41.57095109705844 -74.23608607329565) bank74354 +74355 POINT(40.7172937008001 -74.04381028775072) bank74355 +74356 POINT(40.97832873573522 -73.92980985754285) bank74356 +74357 POINT(39.82536993372885 -74.37719076020072) bank74357 +74358 POINT(39.88657868606125 -74.12640766207804) bank74358 +74359 POINT(40.829579371518804 -74.9607539711554) bank74359 +74360 POINT(41.209932817061905 -74.0238028904715) bank74360 +74361 POINT(40.50262045580825 -74.28249036877433) bank74361 +74362 POINT(40.91987119583786 -73.79063736896381) bank74362 +74363 POINT(41.01074633726493 -73.02654709878692) bank74363 +74364 POINT(39.83652504403286 -73.17954408127757) bank74364 +74365 POINT(41.08781909226872 -74.66576049855242) bank74365 +74366 POINT(40.55943887018323 -73.77642910248913) bank74366 +74367 POINT(41.29556627787531 -73.83904755419566) bank74367 +74368 POINT(40.43597528568632 -74.40113281406681) bank74368 +74369 POINT(41.24659926220579 -73.12110514225307) bank74369 +74370 POINT(40.44836263901666 -73.04072875371158) bank74370 +74371 POINT(41.3423205993827 -74.33674088740347) bank74371 +74372 POINT(41.15031343700758 -74.6886476023862) bank74372 +74373 POINT(41.618934112546555 -73.63232687484611) bank74373 +74374 POINT(40.264652848296244 -73.78308705709598) bank74374 +74375 POINT(39.78606420033234 -73.68905806736606) bank74375 +74376 POINT(40.19939090144705 -74.84274746777297) bank74376 +74377 POINT(40.923252412367844 -74.99200639625955) bank74377 +74378 POINT(40.03734555292555 -73.81191415418276) bank74378 +74379 POINT(40.3425615194903 -73.95150451637303) bank74379 +74380 POINT(40.73739736965936 -73.53482569944677) bank74380 +74381 POINT(40.33898798014103 -73.7857439904048) bank74381 +74382 POINT(40.98324776330449 -73.28119668655262) bank74382 +74383 POINT(40.613272404108905 -73.026711844736) bank74383 +74384 POINT(39.79505773136558 -73.33376942580934) bank74384 +74385 POINT(40.55706699621671 -74.00399838641987) bank74385 +74386 POINT(40.50123102060469 -74.88792121465326) bank74386 +74387 POINT(40.39989643038344 -74.39282891969401) bank74387 +74388 POINT(41.47050867524237 -74.42831653419891) bank74388 +74389 POINT(40.853367730990065 -73.37312880676997) bank74389 +74390 POINT(39.83105953401808 -73.88255462474096) bank74390 +74391 POINT(40.69345544805769 -73.05846984045779) bank74391 +74392 POINT(41.356982738128686 -73.69945070590981) bank74392 +74393 POINT(40.74569535588916 -73.15476118099775) bank74393 +74394 POINT(41.47341358190654 -74.1217853103964) bank74394 +74395 POINT(40.19932152078164 -73.29297119046993) bank74395 +74396 POINT(40.77473118437397 -74.79766943263823) bank74396 +74397 POINT(40.933755911849616 -73.80481565555937) bank74397 +74398 POINT(40.70786445545509 -73.95298446379084) bank74398 +74399 POINT(41.086191828571835 -74.93081004953231) bank74399 +74400 POINT(39.87449320164288 -73.95273951546586) bank74400 +74401 POINT(41.402407689795425 -73.97789751825063) bank74401 +74402 POINT(41.61749861367233 -73.64988239911867) bank74402 +74403 POINT(40.445472574780105 -74.81135148145704) bank74403 +74404 POINT(40.20619742320277 -73.72365776128366) bank74404 +74405 POINT(39.746099781033756 -74.58314541606502) bank74405 +74406 POINT(39.92861436036804 -73.82912376590278) bank74406 +74407 POINT(41.05027355550886 -73.20312703126422) bank74407 +74408 POINT(40.20419399057358 -74.39987513205642) bank74408 +74409 POINT(41.07048900900652 -73.45823333659312) bank74409 +74410 POINT(40.120878503285006 -73.433647668896) bank74410 +74411 POINT(40.39163845453609 -74.6382546906849) bank74411 +74412 POINT(41.516297440114045 -73.63019200842488) bank74412 +74413 POINT(40.45380563892965 -73.88249134708629) bank74413 +74414 POINT(41.52406079008422 -73.73077681378881) bank74414 +74415 POINT(40.7806370137667 -74.4163803057837) bank74415 +74416 POINT(40.6357262730081 -73.97937201202781) bank74416 +74417 POINT(40.90468954770023 -74.49563261980468) bank74417 +74418 POINT(40.51857677156155 -74.93792817697936) bank74418 +74419 POINT(40.45241284868937 -74.65628058872488) bank74419 +74420 POINT(40.95640986934884 -73.88660516858255) bank74420 +74421 POINT(40.24948597253936 -74.78583020281906) bank74421 +74422 POINT(39.85252043882325 -73.29783388324262) bank74422 +74423 POINT(40.97318283053141 -73.95426780232462) bank74423 +74424 POINT(40.77251128047398 -73.89487475971829) bank74424 +74425 POINT(41.53511486902116 -74.40790984743694) bank74425 +74426 POINT(40.915223413795296 -73.32367883385369) bank74426 +74427 POINT(41.1611426934584 -73.69909866558307) bank74427 +74428 POINT(40.54600549833164 -74.23747702151832) bank74428 +74429 POINT(40.80687201467951 -73.73317057440727) bank74429 +74430 POINT(41.1619108452424 -73.5259652956571) bank74430 +74431 POINT(40.23055369210483 -73.67709562895736) bank74431 +74432 POINT(41.49612012346587 -74.36408974634035) bank74432 +74433 POINT(41.55156136355971 -73.88353510772927) bank74433 +74434 POINT(40.27859329091745 -73.03784617487173) bank74434 +74435 POINT(39.84449700732889 -74.44983073698451) bank74435 +74436 POINT(41.32493765151955 -74.96337414922925) bank74436 +74437 POINT(41.55708868748106 -74.61804336356019) bank74437 +74438 POINT(40.58806138963163 -73.18688928175587) bank74438 +74439 POINT(41.344682468044 -74.44142855776914) bank74439 +74440 POINT(41.35026989891224 -73.47345043611645) bank74440 +74441 POINT(41.10498380666284 -73.32121024215552) bank74441 +74442 POINT(39.86719944920971 -73.11506857289213) bank74442 +74443 POINT(41.35382955587554 -74.04655133004134) bank74443 +74444 POINT(41.59893031166555 -73.27611090601997) bank74444 +74445 POINT(40.57200243931195 -74.03216076557324) bank74445 +74446 POINT(39.78251921227983 -74.1657935239616) bank74446 +74447 POINT(39.75793359216256 -74.09493227168358) bank74447 +74448 POINT(40.059393586815055 -73.8679807155766) bank74448 +74449 POINT(40.05840989821535 -73.54325172352675) bank74449 +74450 POINT(41.56919213912619 -73.73598864452936) bank74450 +74451 POINT(41.6382617407195 -74.75643911391921) bank74451 +74452 POINT(40.28410481979502 -74.26577963743317) bank74452 +74453 POINT(40.70795818823874 -74.76107169196067) bank74453 +74454 POINT(40.29177115591406 -74.07905048673312) bank74454 +74455 POINT(41.67419363059506 -74.0477677339262) bank74455 +74456 POINT(41.2676637928708 -74.2404390784417) bank74456 +74457 POINT(40.662482483954484 -73.3964644478486) bank74457 +74458 POINT(41.01751488772713 -73.56397742611563) bank74458 +74459 POINT(40.815532960792545 -74.63168114882342) bank74459 +74460 POINT(40.265360660633846 -74.02085839087087) bank74460 +74461 POINT(41.534857092261085 -73.0780709318033) bank74461 +74462 POINT(40.66759679371042 -73.21686224255372) bank74462 +74463 POINT(40.263252541479325 -73.5305982255537) bank74463 +74464 POINT(41.57361179088787 -74.54758044050074) bank74464 +74465 POINT(41.64932582636201 -73.08778870593243) bank74465 +74466 POINT(41.05495308999372 -73.71432682464481) bank74466 +74467 POINT(41.272097685473746 -73.0823448886643) bank74467 +74468 POINT(39.72478428881058 -73.1705980923947) bank74468 +74469 POINT(41.32297268474253 -73.49220679212706) bank74469 +74470 POINT(40.95783137333934 -74.70841926875354) bank74470 +74471 POINT(41.4124370099112 -73.36333987993285) bank74471 +74472 POINT(40.967681459167444 -74.12562073485267) bank74472 +74473 POINT(40.53994495371647 -74.12948810784458) bank74473 +74474 POINT(41.285968437783886 -74.02908932985139) bank74474 +74475 POINT(40.756123592507464 -74.1093402179251) bank74475 +74476 POINT(41.538794096399684 -74.34659428394127) bank74476 +74477 POINT(40.07496331742375 -73.69057618248976) bank74477 +74478 POINT(40.604209056305315 -74.83129362463251) bank74478 +74479 POINT(40.584160195980736 -73.42734930061023) bank74479 +74480 POINT(41.4766193365215 -74.25874798413128) bank74480 +74481 POINT(40.57186754267029 -73.6776556746278) bank74481 +74482 POINT(41.24116184142002 -73.02297049606295) bank74482 +74483 POINT(39.85755992982878 -73.12034933482757) bank74483 +74484 POINT(41.42233485081188 -73.57501499267381) bank74484 +74485 POINT(40.46504988949856 -73.53518122886501) bank74485 +74486 POINT(40.61810934430596 -74.09279890951268) bank74486 +74487 POINT(40.58801952284134 -73.39294785423492) bank74487 +74488 POINT(40.683756052792845 -74.31561940696467) bank74488 +74489 POINT(41.531038776393466 -73.49126344514515) bank74489 +74490 POINT(39.74928259747404 -73.28892799900598) bank74490 +74491 POINT(40.59077869825811 -74.94831887486187) bank74491 +74492 POINT(39.99992616862594 -73.32614895835385) bank74492 +74493 POINT(40.58018104975807 -73.15727036071453) bank74493 +74494 POINT(41.353190465871236 -74.33069948859703) bank74494 +74495 POINT(41.09306052725039 -74.97778872620783) bank74495 +74496 POINT(40.947291436129596 -74.33788497227974) bank74496 +74497 POINT(41.29577724690186 -74.82138226142442) bank74497 +74498 POINT(40.31844825600409 -73.39501895402806) bank74498 +74499 POINT(40.101696454642706 -73.32780728292666) bank74499 +74500 POINT(40.324031884510475 -74.01433092253565) bank74500 +74501 POINT(40.828996535895996 -73.12466867701808) bank74501 +74502 POINT(41.23086219436063 -73.24713862959844) bank74502 +74503 POINT(40.14957470898438 -74.66068592437428) bank74503 +74504 POINT(39.82818048676188 -74.9001514540494) bank74504 +74505 POINT(41.266339846499136 -74.82113128226194) bank74505 +74506 POINT(40.48562378393422 -73.34408020196992) bank74506 +74507 POINT(41.21145722120636 -73.46019932121321) bank74507 +74508 POINT(41.380990019939425 -73.20752034687372) bank74508 +74509 POINT(40.275149353301174 -74.85755590635296) bank74509 +74510 POINT(40.4154477047949 -74.81178386865786) bank74510 +74511 POINT(39.90415577451857 -74.9751199937457) bank74511 +74512 POINT(41.11204202557733 -74.7547598255015) bank74512 +74513 POINT(40.495049550400914 -74.54934413558992) bank74513 +74514 POINT(40.67644866987875 -74.83371069493133) bank74514 +74515 POINT(41.36810410205736 -74.47104283185234) bank74515 +74516 POINT(41.20362677639094 -73.77076034214471) bank74516 +74517 POINT(40.41632743566111 -73.31228445083195) bank74517 +74518 POINT(40.320569213937 -73.91419554452756) bank74518 +74519 POINT(40.03213026621195 -73.22252897443809) bank74519 +74520 POINT(41.61249112532555 -74.27491166773171) bank74520 +74521 POINT(41.55759223840925 -73.11349382987758) bank74521 +74522 POINT(39.958815787460914 -73.36057398998365) bank74522 +74523 POINT(40.73800821969036 -74.7180125264192) bank74523 +74524 POINT(40.65092044038972 -74.3387860759653) bank74524 +74525 POINT(40.06340668435868 -74.50598984701618) bank74525 +74526 POINT(40.534930070313536 -74.30848475831792) bank74526 +74527 POINT(40.2109717393129 -74.53029215648492) bank74527 +74528 POINT(41.19264506384497 -74.29151367797296) bank74528 +74529 POINT(40.839112767380776 -73.49510128098248) bank74529 +74530 POINT(41.496920457244286 -73.03218125733449) bank74530 +74531 POINT(41.153659067227416 -74.5675177072922) bank74531 +74532 POINT(41.624807768084956 -74.25951505436805) bank74532 +74533 POINT(39.83821039753732 -74.53129717852748) bank74533 +74534 POINT(40.834364530840666 -73.60438107298484) bank74534 +74535 POINT(40.895073569399244 -73.71953730868785) bank74535 +74536 POINT(40.22573938058687 -74.56326203671789) bank74536 +74537 POINT(40.110821389264274 -73.2366056270863) bank74537 +74538 POINT(41.29196939311363 -73.35444461091683) bank74538 +74539 POINT(40.85604520796916 -73.93296447514749) bank74539 +74540 POINT(40.21225500801316 -73.82099161892381) bank74540 +74541 POINT(40.6246224797752 -74.8829340527612) bank74541 +74542 POINT(40.290213955763406 -74.20821223391467) bank74542 +74543 POINT(40.144610678074 -73.69801070081813) bank74543 +74544 POINT(41.31442912598108 -73.02979540598031) bank74544 +74545 POINT(41.31511902653995 -74.68369641095146) bank74545 +74546 POINT(40.5376832597326 -74.75030240810695) bank74546 +74547 POINT(39.73020952062209 -73.28334796849629) bank74547 +74548 POINT(40.33979779652362 -74.83582353252308) bank74548 +74549 POINT(40.383254254522036 -74.68015393910507) bank74549 +74550 POINT(41.63770514049856 -74.99908593273507) bank74550 +74551 POINT(41.56682044449034 -74.51854494560033) bank74551 +74552 POINT(40.51933802389944 -73.3169860468356) bank74552 +74553 POINT(39.987669058097715 -74.32966173383166) bank74553 +74554 POINT(40.13429686214423 -74.17014443569929) bank74554 +74555 POINT(40.8511972397773 -74.88177398026836) bank74555 +74556 POINT(40.426923630533665 -74.09057149312575) bank74556 +74557 POINT(41.42330086377422 -73.97470970338775) bank74557 +74558 POINT(40.24255680230438 -74.92008150985441) bank74558 +74559 POINT(41.545389011046446 -74.15615662929775) bank74559 +74560 POINT(41.29662694722084 -74.44615731398778) bank74560 +74561 POINT(40.05477843372989 -74.97802961101208) bank74561 +74562 POINT(39.97903890069545 -73.6404994026962) bank74562 +74563 POINT(40.79012724554874 -73.86124657096242) bank74563 +74564 POINT(40.34143619487173 -73.47171564316757) bank74564 +74565 POINT(40.253204184822586 -73.20354619212593) bank74565 +74566 POINT(41.16002777024708 -74.98965376652296) bank74566 +74567 POINT(40.617155310003476 -73.04298205016778) bank74567 +74568 POINT(40.566500052132746 -74.41195025845984) bank74568 +74569 POINT(41.05584595766771 -73.01829096484283) bank74569 +74570 POINT(40.556394553630845 -73.93313912671235) bank74570 +74571 POINT(41.00545206660692 -74.22073928670537) bank74571 +74572 POINT(40.6980050577134 -74.18773374876564) bank74572 +74573 POINT(41.126149480436595 -74.0950328912908) bank74573 +74574 POINT(41.2195280101108 -74.51070902125556) bank74574 +74575 POINT(40.11223643352806 -74.34664962944981) bank74575 +74576 POINT(40.13190718919956 -74.05206960743119) bank74576 +74577 POINT(40.51899562492246 -74.22423116510436) bank74577 +74578 POINT(39.88804903951479 -74.37274003346657) bank74578 +74579 POINT(40.581265517017705 -74.67688223047217) bank74579 +74580 POINT(40.00150393640998 -74.58867701782118) bank74580 +74581 POINT(41.29824356584058 -73.2574065190535) bank74581 +74582 POINT(40.69338968385132 -74.71064963739272) bank74582 +74583 POINT(40.71698113468119 -73.21752872515493) bank74583 +74584 POINT(39.877558042614154 -73.38262438285177) bank74584 +74585 POINT(40.32548479977576 -74.4582765748008) bank74585 +74586 POINT(40.41692758518258 -73.6593158057805) bank74586 +74587 POINT(41.329032768504845 -74.67847574099767) bank74587 +74588 POINT(39.917823480071874 -74.71160837105953) bank74588 +74589 POINT(40.80426940402427 -73.68107451030278) bank74589 +74590 POINT(40.53899071349765 -74.54474294641717) bank74590 +74591 POINT(41.36029496161207 -73.46086401799658) bank74591 +74592 POINT(39.99399841951813 -73.49900584643889) bank74592 +74593 POINT(40.92429945397568 -73.39308651208823) bank74593 +74594 POINT(41.489067281253504 -73.33384824532274) bank74594 +74595 POINT(40.30755180634791 -73.79116312099666) bank74595 +74596 POINT(40.20488466755584 -74.13150138665976) bank74596 +74597 POINT(41.22569921813861 -73.26715533225497) bank74597 +74598 POINT(41.65545082293297 -74.90403666043971) bank74598 +74599 POINT(41.61985551885153 -73.94143718871737) bank74599 +74600 POINT(41.52551422888801 -73.26727788977726) bank74600 +74601 POINT(40.435141988475856 -73.31549599262131) bank74601 +74602 POINT(41.65299178649734 -73.2571583487169) bank74602 +74603 POINT(40.01210164941589 -73.07478087834703) bank74603 +74604 POINT(41.44122629821227 -74.72150020595375) bank74604 +74605 POINT(41.02532757157594 -74.25066824058717) bank74605 +74606 POINT(41.47515987589376 -74.62884410573406) bank74606 +74607 POINT(40.66912233366567 -74.46123878142427) bank74607 +74608 POINT(40.14093144157968 -74.25147001868885) bank74608 +74609 POINT(39.7950233468138 -74.22963865214241) bank74609 +74610 POINT(40.17051253191838 -73.48887782387641) bank74610 +74611 POINT(40.50026334771772 -74.45417951263198) bank74611 +74612 POINT(41.124950546161315 -73.79186382941334) bank74612 +74613 POINT(41.53078032478806 -74.08947666381063) bank74613 +74614 POINT(41.30771323086453 -74.64025222553124) bank74614 +74615 POINT(40.98720262258942 -74.07362956776697) bank74615 +74616 POINT(40.48055342963958 -73.9107317563944) bank74616 +74617 POINT(40.783733715483564 -74.75589332163942) bank74617 +74618 POINT(40.90330335680829 -73.60769677391896) bank74618 +74619 POINT(40.96517424581934 -73.61916108069538) bank74619 +74620 POINT(40.17953657334884 -73.4030972775757) bank74620 +74621 POINT(41.538606763976965 -73.46609993334295) bank74621 +74622 POINT(39.78398810307387 -74.48128258656774) bank74622 +74623 POINT(39.799025185647196 -74.8275564937122) bank74623 +74624 POINT(40.18009924877886 -73.32122763273532) bank74624 +74625 POINT(40.97310896397205 -74.45342981776976) bank74625 +74626 POINT(41.57605375621306 -74.83865643395474) bank74626 +74627 POINT(41.53855167938632 -74.17136554403469) bank74627 +74628 POINT(41.31152770844899 -73.03258590206379) bank74628 +74629 POINT(39.79651039141758 -74.6315518547125) bank74629 +74630 POINT(39.813825578588784 -74.9591933328637) bank74630 +74631 POINT(40.30740690051987 -73.17303300014486) bank74631 +74632 POINT(40.06184248382404 -74.0718071848364) bank74632 +74633 POINT(41.1575049352146 -73.45414225870203) bank74633 +74634 POINT(40.61518296297266 -73.29839030123308) bank74634 +74635 POINT(39.8995720201447 -74.98070565480268) bank74635 +74636 POINT(41.30852213680824 -73.16327086835092) bank74636 +74637 POINT(40.97242478204086 -74.64857744521554) bank74637 +74638 POINT(39.89776813901952 -73.17807203867217) bank74638 +74639 POINT(39.940670524055015 -74.52882952263121) bank74639 +74640 POINT(40.41520554985584 -74.8868124562043) bank74640 +74641 POINT(41.58495993965347 -74.8114141242943) bank74641 +74642 POINT(41.22881375432594 -73.44231442987198) bank74642 +74643 POINT(40.92937005651729 -73.5341410758511) bank74643 +74644 POINT(40.67749188937123 -73.65856845078446) bank74644 +74645 POINT(40.22306670397307 -74.04251771773059) bank74645 +74646 POINT(41.11706985530481 -74.17068987388362) bank74646 +74647 POINT(40.193077011278056 -74.11325534223482) bank74647 +74648 POINT(40.995909673213745 -74.22200741600706) bank74648 +74649 POINT(40.208893535109354 -74.15507652360728) bank74649 +74650 POINT(40.29829664375753 -73.13528679954) bank74650 +74651 POINT(40.51155747861719 -74.82414158689767) bank74651 +74652 POINT(41.48713561289612 -74.67207075288977) bank74652 +74653 POINT(40.339287729635224 -73.07737582186618) bank74653 +74654 POINT(40.23675368411396 -73.85149017736984) bank74654 +74655 POINT(39.72771398237617 -73.54860119572282) bank74655 +74656 POINT(40.733851142906985 -74.0743257036176) bank74656 +74657 POINT(40.87787818505214 -74.50273154258079) bank74657 +74658 POINT(39.97371941708076 -73.34325867766773) bank74658 +74659 POINT(40.36699997573416 -74.09451483670215) bank74659 +74660 POINT(39.8029936139444 -74.49185710938906) bank74660 +74661 POINT(41.54305780923608 -73.43728420383422) bank74661 +74662 POINT(41.038184024631924 -73.95592094505189) bank74662 +74663 POINT(39.95440634795862 -74.88732666950715) bank74663 +74664 POINT(41.500765236819504 -73.26601476054009) bank74664 +74665 POINT(41.64347777056398 -74.35361772792798) bank74665 +74666 POINT(40.39613651279729 -74.85345903858439) bank74666 +74667 POINT(40.455904098132066 -74.10525943833662) bank74667 +74668 POINT(40.12234313142058 -74.72994228992458) bank74668 +74669 POINT(40.70719268654368 -74.89452978741447) bank74669 +74670 POINT(40.87646435059883 -74.3367905979809) bank74670 +74671 POINT(41.31106630141908 -74.46316851784597) bank74671 +74672 POINT(41.13038601035124 -74.33719163559887) bank74672 +74673 POINT(40.96557654167943 -74.51206538010061) bank74673 +74674 POINT(41.39088864800556 -73.59290486887724) bank74674 +74675 POINT(40.86255140215647 -73.42078122087355) bank74675 +74676 POINT(39.72097261700441 -73.84663759527083) bank74676 +74677 POINT(40.07138316841083 -74.70093551062443) bank74677 +74678 POINT(40.60191375928927 -73.62867774139562) bank74678 +74679 POINT(39.96439465199754 -73.4259810669442) bank74679 +74680 POINT(41.11963307975469 -74.79541872962868) bank74680 +74681 POINT(41.2976049338092 -73.04403790714001) bank74681 +74682 POINT(41.54923129488175 -74.84304766340199) bank74682 +74683 POINT(40.82254684416258 -74.65018888114248) bank74683 +74684 POINT(39.93293823791072 -73.36213985345329) bank74684 +74685 POINT(41.606283875696384 -74.18210246849559) bank74685 +74686 POINT(41.477930846851685 -74.73137486162177) bank74686 +74687 POINT(40.72284405061935 -74.59973844998441) bank74687 +74688 POINT(41.012439036785715 -73.6231186650984) bank74688 +74689 POINT(40.84199681340352 -74.58649126637286) bank74689 +74690 POINT(40.36252109887566 -73.68661748665033) bank74690 +74691 POINT(40.51920380724626 -74.27501201955604) bank74691 +74692 POINT(39.83866144297244 -74.26665730580784) bank74692 +74693 POINT(41.555036662925886 -73.44156230936585) bank74693 +74694 POINT(40.66473673504397 -73.27184067969894) bank74694 +74695 POINT(40.963386532019534 -74.0191999091251) bank74695 +74696 POINT(41.37882522944108 -74.87741082687009) bank74696 +74697 POINT(41.37012111220399 -74.434023059478) bank74697 +74698 POINT(41.21824061666704 -73.28973130973708) bank74698 +74699 POINT(41.449408998426826 -74.0531732848527) bank74699 +74700 POINT(40.37295592816305 -73.93853606665164) bank74700 +74701 POINT(40.86786319866354 -73.6112826697131) bank74701 +74702 POINT(40.291720285791406 -73.92377689452655) bank74702 +74703 POINT(40.47319734810271 -73.63188888483994) bank74703 +74704 POINT(41.34500313350382 -73.63610448834542) bank74704 +74705 POINT(41.69562695128758 -74.62008345868175) bank74705 +74706 POINT(41.52566999322519 -74.50866227646935) bank74706 +74707 POINT(41.51794568991343 -73.56451771361107) bank74707 +74708 POINT(41.11060049451215 -74.76963007976377) bank74708 +74709 POINT(40.38763246647369 -73.37750262615786) bank74709 +74710 POINT(40.4879177627198 -73.85467182439378) bank74710 +74711 POINT(40.629209300284764 -74.6846957406328) bank74711 +74712 POINT(40.19758420294764 -73.5304012022985) bank74712 +74713 POINT(40.0433386818745 -74.9657280959572) bank74713 +74714 POINT(40.45570667902744 -74.42827338636437) bank74714 +74715 POINT(40.19161489205742 -73.03028322563698) bank74715 +74716 POINT(41.52853327739041 -74.18111649560949) bank74716 +74717 POINT(41.61546224824747 -73.63506154009275) bank74717 +74718 POINT(40.11348881786945 -74.20505012585485) bank74718 +74719 POINT(39.75573865340715 -74.69718349637316) bank74719 +74720 POINT(40.51196335331528 -73.95978863556185) bank74720 +74721 POINT(39.78824330952487 -73.14630904774788) bank74721 +74722 POINT(40.92700916459495 -74.05362876425994) bank74722 +74723 POINT(40.98078789424749 -73.42297125812608) bank74723 +74724 POINT(41.63090089464369 -73.35937934668777) bank74724 +74725 POINT(39.91482665740919 -73.06044896825333) bank74725 +74726 POINT(41.69685396492063 -73.89601501504599) bank74726 +74727 POINT(40.88149978100262 -74.25286529602778) bank74727 +74728 POINT(40.189260658634105 -74.00020759852735) bank74728 +74729 POINT(40.47502315889255 -73.98195355176102) bank74729 +74730 POINT(39.79381554250678 -74.94930415374374) bank74730 +74731 POINT(41.231038111108155 -73.09109623688259) bank74731 +74732 POINT(40.50846008974725 -73.18320712768474) bank74732 +74733 POINT(40.56310872806671 -73.49221714603395) bank74733 +74734 POINT(40.25450556387261 -74.54872514571697) bank74734 +74735 POINT(40.709240482276336 -73.48044973259763) bank74735 +74736 POINT(41.226617070305814 -73.72737044955143) bank74736 +74737 POINT(40.52495638150815 -74.90090765784498) bank74737 +74738 POINT(40.52079520229998 -74.89763460422584) bank74738 +74739 POINT(41.51767420241421 -73.90601369366381) bank74739 +74740 POINT(41.50559534437517 -73.2883629553254) bank74740 +74741 POINT(41.274206590593415 -73.25276511537014) bank74741 +74742 POINT(41.07754466381344 -73.44596905159938) bank74742 +74743 POINT(41.51266406012422 -73.35639811739273) bank74743 +74744 POINT(39.778438546399386 -74.01355655038914) bank74744 +74745 POINT(39.862547874874764 -73.204413841697) bank74745 +74746 POINT(41.58471317584973 -74.64250624630144) bank74746 +74747 POINT(39.753283657262315 -73.12119494676449) bank74747 +74748 POINT(40.565046302203214 -73.30839878843915) bank74748 +74749 POINT(40.37032091501704 -74.18615126030794) bank74749 +74750 POINT(40.77765594190259 -74.031986393441) bank74750 +74751 POINT(39.86975880445265 -73.12882557000219) bank74751 +74752 POINT(40.71901432968118 -73.49707633071732) bank74752 +74753 POINT(40.39485181274992 -73.43361908014434) bank74753 +74754 POINT(40.50224537213799 -73.12050282592035) bank74754 +74755 POINT(40.474561955926994 -74.49257507862049) bank74755 +74756 POINT(40.02632655911276 -74.87815616191199) bank74756 +74757 POINT(40.09379251633826 -74.27097225758841) bank74757 +74758 POINT(40.029242670172636 -73.94471476870926) bank74758 +74759 POINT(40.589587320413564 -74.25997622198727) bank74759 +74760 POINT(40.642330180428395 -74.65790630014469) bank74760 +74761 POINT(39.84913684675623 -73.26782559961438) bank74761 +74762 POINT(40.881199466989464 -74.73205121724295) bank74762 +74763 POINT(40.985048916794554 -73.32311287059169) bank74763 +74764 POINT(40.99043199323104 -73.41635905656177) bank74764 +74765 POINT(40.630792572103715 -74.63952314117638) bank74765 +74766 POINT(40.74974439016303 -73.10630662692292) bank74766 +74767 POINT(39.90578106046984 -74.82570193488004) bank74767 +74768 POINT(39.75658894752153 -74.16672428164567) bank74768 +74769 POINT(40.434783610246626 -73.79241875288054) bank74769 +74770 POINT(41.54407499652565 -73.37709155095656) bank74770 +74771 POINT(40.71840939356284 -74.14777632823227) bank74771 +74772 POINT(41.25976363370305 -73.3107415668754) bank74772 +74773 POINT(40.186511546360705 -74.12298797694382) bank74773 +74774 POINT(39.761441429268054 -74.25434229097743) bank74774 +74775 POINT(41.03579072871125 -73.27692858445465) bank74775 +74776 POINT(41.06640438263746 -73.94622060511217) bank74776 +74777 POINT(41.54111510007816 -73.61794026202199) bank74777 +74778 POINT(41.22311403087138 -74.03152507901828) bank74778 +74779 POINT(41.26352503087387 -73.39432814133804) bank74779 +74780 POINT(40.355992556442416 -74.55504617509347) bank74780 +74781 POINT(41.31034904860945 -73.76489490780074) bank74781 +74782 POINT(40.09296661216602 -73.01926501538799) bank74782 +74783 POINT(40.027497760789146 -73.90843335655758) bank74783 +74784 POINT(41.52567494732554 -74.42548800760146) bank74784 +74785 POINT(40.69002617479473 -74.18655238868567) bank74785 +74786 POINT(40.56409894743631 -74.06216856284249) bank74786 +74787 POINT(41.366622650372335 -73.42874285890873) bank74787 +74788 POINT(41.18525257081711 -73.88585650729924) bank74788 +74789 POINT(41.140040024084705 -74.93067782640335) bank74789 +74790 POINT(41.16095204453248 -74.70999747395376) bank74790 +74791 POINT(40.684063065649404 -73.15711105963231) bank74791 +74792 POINT(41.25588136219075 -73.57025313451072) bank74792 +74793 POINT(40.59254468572793 -74.72521033493162) bank74793 +74794 POINT(39.80602766520036 -74.65955490912212) bank74794 +74795 POINT(41.31137404835346 -73.4338367025511) bank74795 +74796 POINT(40.09891558586279 -73.04131625909892) bank74796 +74797 POINT(41.056235858950124 -74.55238125988704) bank74797 +74798 POINT(40.123696698993854 -74.00940951087402) bank74798 +74799 POINT(39.86140200843287 -74.8007696287594) bank74799 +74800 POINT(40.208808048357575 -74.17936453486845) bank74800 +74801 POINT(40.72965191026384 -74.03425361770672) bank74801 +74802 POINT(40.5838702147572 -73.50506329157666) bank74802 +74803 POINT(41.22813028953401 -74.33524249569747) bank74803 +74804 POINT(40.26135838153484 -73.06008250205024) bank74804 +74805 POINT(41.6536104934797 -74.3648664301501) bank74805 +74806 POINT(40.06376323789388 -73.25577988990845) bank74806 +74807 POINT(40.24032266027175 -74.54723237632714) bank74807 +74808 POINT(40.81982023218754 -74.79855083381955) bank74808 +74809 POINT(41.21456096139779 -73.59188763415031) bank74809 +74810 POINT(39.88934217272895 -73.27813033705304) bank74810 +74811 POINT(41.10254426495901 -73.16991236958043) bank74811 +74812 POINT(40.13287720622748 -74.71019410370283) bank74812 +74813 POINT(41.08143085065917 -73.94450267328102) bank74813 +74814 POINT(41.51470255325648 -74.0619034919545) bank74814 +74815 POINT(40.55095734327182 -74.49397143347153) bank74815 +74816 POINT(41.134281398648554 -73.9973395124278) bank74816 +74817 POINT(40.19662142042942 -74.33848756327367) bank74817 +74818 POINT(41.053279509450945 -73.317661321648) bank74818 +74819 POINT(39.9028584723515 -74.56709750609011) bank74819 +74820 POINT(40.751171833575334 -74.62466953085297) bank74820 +74821 POINT(41.17214260543493 -74.2941684853827) bank74821 +74822 POINT(41.11960655339808 -73.83117069118971) bank74822 +74823 POINT(40.063463337056156 -73.58208308941121) bank74823 +74824 POINT(41.672717413527266 -73.29107361983009) bank74824 +74825 POINT(40.17437362279861 -73.9960624533343) bank74825 +74826 POINT(40.221213441503764 -74.8027511686722) bank74826 +74827 POINT(41.36343627653113 -74.73676548217077) bank74827 +74828 POINT(40.47262659366733 -74.70958137426689) bank74828 +74829 POINT(41.1660087400815 -74.88559767885704) bank74829 +74830 POINT(41.51392264139156 -73.1846628454315) bank74830 +74831 POINT(39.848274067723054 -74.44652747894062) bank74831 +74832 POINT(40.433099938560055 -74.52261329778723) bank74832 +74833 POINT(40.71763247357337 -73.36316826184586) bank74833 +74834 POINT(40.307137043947726 -73.96411099003689) bank74834 +74835 POINT(40.24347949676823 -73.09860802441138) bank74835 +74836 POINT(40.884511148880655 -74.17181326801885) bank74836 +74837 POINT(40.37553654552651 -74.61567248200186) bank74837 +74838 POINT(40.350529070315254 -74.77443807573633) bank74838 +74839 POINT(41.65470602286559 -74.76919252998873) bank74839 +74840 POINT(40.16455282077793 -73.91679918044217) bank74840 +74841 POINT(40.35449829956954 -73.37539892013324) bank74841 +74842 POINT(40.02078973413967 -74.78828570838434) bank74842 +74843 POINT(41.31892978765466 -74.98190280173937) bank74843 +74844 POINT(39.884017930874755 -74.14139601739808) bank74844 +74845 POINT(41.12333248065593 -73.47326145293344) bank74845 +74846 POINT(40.76993905874744 -73.18488717102079) bank74846 +74847 POINT(40.78921061506089 -73.76355747855615) bank74847 +74848 POINT(41.673461495409676 -74.64123647499075) bank74848 +74849 POINT(40.08009380364475 -74.12415737813643) bank74849 +74850 POINT(40.716931764804734 -73.90344651549442) bank74850 +74851 POINT(39.97687317152206 -74.90636261578877) bank74851 +74852 POINT(40.86321408410918 -73.03136379045915) bank74852 +74853 POINT(40.769302820654616 -73.12133636662398) bank74853 +74854 POINT(40.17218379137129 -73.20205677977832) bank74854 +74855 POINT(40.81561711721248 -73.29185404122636) bank74855 +74856 POINT(41.083123949649824 -74.32798675006211) bank74856 +74857 POINT(41.58331108054858 -74.61813275229568) bank74857 +74858 POINT(40.69700859645063 -73.76597893321909) bank74858 +74859 POINT(41.55684935381731 -73.24588171691542) bank74859 +74860 POINT(41.70330306717584 -73.80888768758696) bank74860 +74861 POINT(40.673979049855255 -73.56977921713236) bank74861 +74862 POINT(41.215419810434184 -73.08235708351408) bank74862 +74863 POINT(41.02935259315215 -74.89509448192241) bank74863 +74864 POINT(41.42279681444956 -73.11417850010866) bank74864 +74865 POINT(41.15282729391404 -73.58720379686882) bank74865 +74866 POINT(39.926708253646396 -74.21964898199363) bank74866 +74867 POINT(41.04589076775194 -73.45214441790344) bank74867 +74868 POINT(40.70661954703563 -74.76804655000537) bank74868 +74869 POINT(41.32261318917179 -74.43842694252373) bank74869 +74870 POINT(40.55660071090942 -73.44653833040175) bank74870 +74871 POINT(40.05487512225888 -74.04870826346112) bank74871 +74872 POINT(39.77519944130153 -74.38606928922992) bank74872 +74873 POINT(40.48063933864103 -73.93615356864962) bank74873 +74874 POINT(41.2364305145969 -74.20535559690255) bank74874 +74875 POINT(40.72005524040207 -74.20599142500726) bank74875 +74876 POINT(40.32240536778222 -73.2513705812085) bank74876 +74877 POINT(41.08310564415658 -73.54310770706627) bank74877 +74878 POINT(41.337334440197765 -73.43525771130147) bank74878 +74879 POINT(40.30239928031255 -73.40387258489808) bank74879 +74880 POINT(41.224824064943576 -74.6048269204141) bank74880 +74881 POINT(40.74615546086889 -74.44425993014566) bank74881 +74882 POINT(40.19238495511143 -73.42227028912563) bank74882 +74883 POINT(41.66419052026048 -73.00914446897247) bank74883 +74884 POINT(41.2069860627008 -74.30608658442267) bank74884 +74885 POINT(41.70506673082353 -74.76397780203799) bank74885 +74886 POINT(41.08515461272204 -73.83481054355173) bank74886 +74887 POINT(41.62340837246679 -73.11137456393135) bank74887 +74888 POINT(41.379211509010126 -74.51250112439348) bank74888 +74889 POINT(41.31673171169005 -73.0235518793281) bank74889 +74890 POINT(39.975642483163654 -73.84851391455138) bank74890 +74891 POINT(40.50253997447548 -73.70614452043543) bank74891 +74892 POINT(41.59047206515344 -74.97671934554289) bank74892 +74893 POINT(41.19734045973292 -73.31790859090692) bank74893 +74894 POINT(41.24086606086975 -74.71498285985354) bank74894 +74895 POINT(41.21197600596113 -74.76092981444623) bank74895 +74896 POINT(40.973447565596175 -73.02414306213285) bank74896 +74897 POINT(40.243204729380764 -73.06144426810236) bank74897 +74898 POINT(39.92160683478612 -74.87183493457702) bank74898 +74899 POINT(41.27597176147688 -74.2804335056136) bank74899 +74900 POINT(40.156795684150346 -74.70718486296975) bank74900 +74901 POINT(41.23270533963662 -73.89227625785216) bank74901 +74902 POINT(40.97769383243414 -74.22725520705373) bank74902 +74903 POINT(40.82445832359133 -73.8226034897389) bank74903 +74904 POINT(39.905792498769706 -73.8788351311709) bank74904 +74905 POINT(41.32262749881913 -74.35676372147941) bank74905 +74906 POINT(41.18442768619493 -73.54598183239486) bank74906 +74907 POINT(40.11220911205923 -73.72954218735184) bank74907 +74908 POINT(41.40035362816307 -73.28088977236033) bank74908 +74909 POINT(41.60245534004099 -73.3894489600301) bank74909 +74910 POINT(40.64658118903626 -74.39534912066439) bank74910 +74911 POINT(40.20293032958229 -74.797187416732) bank74911 +74912 POINT(40.42858334731253 -73.61021327746664) bank74912 +74913 POINT(41.55496612809258 -73.1366726633122) bank74913 +74914 POINT(40.39616177060349 -73.134773932979) bank74914 +74915 POINT(41.254131186140945 -74.53455937667624) bank74915 +74916 POINT(40.25582370246579 -74.14476849083243) bank74916 +74917 POINT(40.634737479994044 -74.6072738576183) bank74917 +74918 POINT(40.71534503919799 -74.53159089196373) bank74918 +74919 POINT(41.33314363084787 -74.83277841854205) bank74919 +74920 POINT(41.37370941304644 -74.68599093006658) bank74920 +74921 POINT(39.756318679128306 -74.65906332059956) bank74921 +74922 POINT(40.09837323533724 -73.3767643918152) bank74922 +74923 POINT(39.8743234700664 -74.28365107023433) bank74923 +74924 POINT(40.60021473487911 -74.60962255821187) bank74924 +74925 POINT(39.84669494957081 -74.91737503870213) bank74925 +74926 POINT(41.59374969980853 -73.95321344388628) bank74926 +74927 POINT(40.77156330108787 -73.72517389486073) bank74927 +74928 POINT(41.2115789310952 -73.22413781617136) bank74928 +74929 POINT(41.45613304851371 -74.94689061685948) bank74929 +74930 POINT(41.54086140047661 -73.54245873410581) bank74930 +74931 POINT(40.23754723740551 -73.75518050803342) bank74931 +74932 POINT(39.76988252684149 -74.72289462313756) bank74932 +74933 POINT(40.66632739086475 -74.21296916339585) bank74933 +74934 POINT(41.38286543843233 -74.72400216263318) bank74934 +74935 POINT(41.09002490380143 -74.39993531658112) bank74935 +74936 POINT(41.11044967794149 -74.52300639862757) bank74936 +74937 POINT(41.1330208320463 -73.98370325998529) bank74937 +74938 POINT(40.79778392804901 -74.56641538726977) bank74938 +74939 POINT(40.490960445019326 -73.56853000013719) bank74939 +74940 POINT(39.99113498424489 -73.57300640968363) bank74940 +74941 POINT(40.541809524441554 -74.90767407639271) bank74941 +74942 POINT(40.39226605290529 -73.22318616478641) bank74942 +74943 POINT(39.73738775643705 -73.74077460933523) bank74943 +74944 POINT(39.71800544764273 -73.33201794243855) bank74944 +74945 POINT(41.21470398835794 -73.33612429287957) bank74945 +74946 POINT(39.994086320574844 -73.58608983138934) bank74946 +74947 POINT(40.88478909789139 -73.80289233049533) bank74947 +74948 POINT(41.187932758275046 -73.00696149412018) bank74948 +74949 POINT(40.581942390178426 -73.60966079409394) bank74949 +74950 POINT(40.836775695498034 -74.29466949580824) bank74950 +74951 POINT(41.437226638266374 -73.43596449413116) bank74951 +74952 POINT(41.18374876350308 -73.48356181127353) bank74952 +74953 POINT(40.90011851514444 -74.97341017286347) bank74953 +74954 POINT(41.242740030676096 -74.377869628701) bank74954 +74955 POINT(41.120313005779124 -73.46623953820684) bank74955 +74956 POINT(41.051739091136405 -74.23339050666115) bank74956 +74957 POINT(39.87395674673739 -74.66084741712105) bank74957 +74958 POINT(41.6301183499251 -73.60603105146694) bank74958 +74959 POINT(41.547410370098135 -74.90464341488996) bank74959 +74960 POINT(40.70018721170848 -73.46981696930574) bank74960 +74961 POINT(40.236382363434444 -74.85340708241931) bank74961 +74962 POINT(39.85312726983866 -74.43288495574326) bank74962 +74963 POINT(40.35778804990395 -74.61654068277849) bank74963 +74964 POINT(41.45190301178704 -73.4686776609223) bank74964 +74965 POINT(40.06606238533347 -74.72665710365627) bank74965 +74966 POINT(40.47923332067908 -73.68737865961147) bank74966 +74967 POINT(41.27690123937233 -73.57244906854125) bank74967 +74968 POINT(41.54213382170652 -74.22597235136959) bank74968 +74969 POINT(40.456414416895946 -74.36138343960847) bank74969 +74970 POINT(41.138029553573354 -74.50451404074973) bank74970 +74971 POINT(40.99972276667601 -74.11820462882618) bank74971 +74972 POINT(40.73862848469107 -74.18280175149471) bank74972 +74973 POINT(40.65771872159486 -73.3659514176144) bank74973 +74974 POINT(40.21985630166583 -74.45949087479651) bank74974 +74975 POINT(40.210961455290615 -74.02164239464447) bank74975 +74976 POINT(41.167317346001255 -74.4921150993745) bank74976 +74977 POINT(40.35457667339362 -73.07949293190384) bank74977 +74978 POINT(40.87106090846858 -73.03373982286068) bank74978 +74979 POINT(40.414688790487816 -74.91148213903584) bank74979 +74980 POINT(41.011227835949256 -74.71782704917413) bank74980 +74981 POINT(40.020462052445566 -74.96986515236944) bank74981 +74982 POINT(41.3430055762083 -74.02024029913709) bank74982 +74983 POINT(40.49500679768891 -74.41716530968888) bank74983 +74984 POINT(40.60239439871368 -74.8289613413811) bank74984 +74985 POINT(41.638870785114314 -74.79883899244068) bank74985 +74986 POINT(40.45810000099656 -74.32398421324419) bank74986 +74987 POINT(41.16456523561351 -73.3003957167025) bank74987 +74988 POINT(40.652603399954785 -74.55276612726126) bank74988 +74989 POINT(41.54017981189832 -74.52468346334416) bank74989 +74990 POINT(41.03668972265632 -74.81936144503229) bank74990 +74991 POINT(40.27858553272975 -74.49238018896457) bank74991 +74992 POINT(41.20343248419981 -73.17140588376591) bank74992 +74993 POINT(41.486862477115054 -73.91495531749649) bank74993 +74994 POINT(40.73409052148648 -74.87700819019969) bank74994 +74995 POINT(41.546201810391594 -74.51453578074432) bank74995 +74996 POINT(41.006020445263026 -73.84150247584851) bank74996 +74997 POINT(41.70234644028271 -73.70315048019022) bank74997 +74998 POINT(39.752752869625624 -73.88650705534816) bank74998 +74999 POINT(41.37841703349121 -73.29157817742423) bank74999 +75000 POINT(40.118827599684096 -73.96365977194945) bank75000 +75001 POINT(39.74063595520328 -73.91225024689012) bank75001 +75002 POINT(40.1222642719924 -74.00261389421752) bank75002 +75003 POINT(41.291268509925075 -74.95584915724284) bank75003 +75004 POINT(39.717965759794076 -73.25200019324551) bank75004 +75005 POINT(41.456734238740545 -74.09573154041746) bank75005 +75006 POINT(41.19567602269654 -73.66647704221612) bank75006 +75007 POINT(40.40172053783657 -73.4583262999575) bank75007 +75008 POINT(41.47165962410538 -73.84336676971877) bank75008 +75009 POINT(40.718694080193636 -73.17966827497182) bank75009 +75010 POINT(40.89234312693503 -73.0513348732481) bank75010 +75011 POINT(40.4719284942198 -74.91812125343435) bank75011 +75012 POINT(40.2860253631155 -73.96872155500752) bank75012 +75013 POINT(39.99590583072109 -73.3692127803623) bank75013 +75014 POINT(40.85896209967057 -73.338178337669) bank75014 +75015 POINT(41.213984871749524 -73.14765990925582) bank75015 +75016 POINT(41.62383334951329 -73.3130191062758) bank75016 +75017 POINT(40.586528847153744 -74.50535336437434) bank75017 +75018 POINT(41.39233689740067 -74.43985556435413) bank75018 +75019 POINT(41.382210830861034 -74.54653531043073) bank75019 +75020 POINT(40.99651932765579 -73.93528878249715) bank75020 +75021 POINT(40.78602703830399 -73.46160300674904) bank75021 +75022 POINT(39.764045923446304 -73.26963067500652) bank75022 +75023 POINT(40.29277370150086 -73.68818009590409) bank75023 +75024 POINT(39.79863073004691 -73.74943310091868) bank75024 +75025 POINT(41.305367767320966 -74.17249667655153) bank75025 +75026 POINT(39.849622357300476 -73.1908859647295) bank75026 +75027 POINT(40.70452053059519 -74.95786426945475) bank75027 +75028 POINT(40.530703426835 -74.19355638840993) bank75028 +75029 POINT(39.977291711498104 -73.42879751196963) bank75029 +75030 POINT(41.40364375310111 -73.89609890296586) bank75030 +75031 POINT(40.45021129684296 -73.80600262862737) bank75031 +75032 POINT(40.99843835134345 -74.2734934061642) bank75032 +75033 POINT(41.345579819465286 -73.25865019081589) bank75033 +75034 POINT(40.27419684882473 -74.69603864047961) bank75034 +75035 POINT(41.293249809886156 -73.7139217415978) bank75035 +75036 POINT(41.24938389853636 -74.89988937885481) bank75036 +75037 POINT(40.726049917971864 -74.60038014782555) bank75037 +75038 POINT(40.64701179819318 -74.45794239624378) bank75038 +75039 POINT(41.63634661458518 -74.77242902708409) bank75039 +75040 POINT(39.94903332143618 -74.693042073597) bank75040 +75041 POINT(40.9012913617693 -74.9280150353009) bank75041 +75042 POINT(39.84146244094757 -74.955231457525) bank75042 +75043 POINT(41.059074905444305 -74.1794447870591) bank75043 +75044 POINT(40.262136724911876 -74.58925902622184) bank75044 +75045 POINT(40.32444296638406 -73.47142531714663) bank75045 +75046 POINT(41.71167152382202 -73.5083390561025) bank75046 +75047 POINT(40.00201057466063 -73.13229474260763) bank75047 +75048 POINT(41.29355582212694 -74.19388969101558) bank75048 +75049 POINT(40.70964804964708 -73.41999497189427) bank75049 +75050 POINT(41.38569636938818 -73.41409404696125) bank75050 +75051 POINT(40.850079161943505 -74.52764576185183) bank75051 +75052 POINT(40.591452764165375 -74.5774381160193) bank75052 +75053 POINT(40.91091408999422 -74.3452619901489) bank75053 +75054 POINT(40.796419318265336 -73.06968466167791) bank75054 +75055 POINT(41.36525200910809 -74.10308236496248) bank75055 +75056 POINT(40.60857567021233 -73.75003627062304) bank75056 +75057 POINT(41.31019667742559 -74.78010993859783) bank75057 +75058 POINT(40.025499394175945 -73.45841429709859) bank75058 +75059 POINT(40.03565679881649 -73.35282118399503) bank75059 +75060 POINT(40.295260674889015 -74.18085398501707) bank75060 +75061 POINT(40.62683605840427 -73.31594625126324) bank75061 +75062 POINT(40.67447094274337 -73.29099193655371) bank75062 +75063 POINT(41.65373312165938 -73.99305003142983) bank75063 +75064 POINT(40.459181671861586 -74.93323305113647) bank75064 +75065 POINT(40.02692464304577 -74.53241926682885) bank75065 +75066 POINT(40.13782884615998 -74.50633627671877) bank75066 +75067 POINT(40.38692044345846 -74.11018384923582) bank75067 +75068 POINT(40.26791565931092 -73.3502037559908) bank75068 +75069 POINT(40.327529566693556 -74.6036440047147) bank75069 +75070 POINT(40.99252401300444 -74.62649973467299) bank75070 +75071 POINT(39.783572802437554 -73.51046117713398) bank75071 +75072 POINT(40.215870366550305 -74.00275672862254) bank75072 +75073 POINT(41.554428814892 -73.09102043380679) bank75073 +75074 POINT(40.21440623925599 -74.18326468668398) bank75074 +75075 POINT(40.412760405993254 -74.4802001354198) bank75075 +75076 POINT(40.80036765240145 -74.31707846763261) bank75076 +75077 POINT(40.44406013516503 -73.32711185019002) bank75077 +75078 POINT(40.509196029279884 -73.1605821599588) bank75078 +75079 POINT(40.5048543496473 -73.1303626572832) bank75079 +75080 POINT(41.164516525095344 -74.980221901495) bank75080 +75081 POINT(40.14905939007639 -73.59176105203868) bank75081 +75082 POINT(41.488885290644184 -74.4434163214461) bank75082 +75083 POINT(41.47821575389856 -74.9426845377242) bank75083 +75084 POINT(40.28617709460497 -74.96394757865338) bank75084 +75085 POINT(40.41700532422683 -73.1285770687154) bank75085 +75086 POINT(41.34831313469903 -74.54485208282094) bank75086 +75087 POINT(40.48463203111992 -74.34724445622045) bank75087 +75088 POINT(40.218633327476184 -74.00174494989453) bank75088 +75089 POINT(40.458012229398115 -73.39424026976981) bank75089 +75090 POINT(40.028458602577196 -73.62486595793881) bank75090 +75091 POINT(40.50858884375521 -74.7804637207046) bank75091 +75092 POINT(41.49175283032952 -74.8059985325709) bank75092 +75093 POINT(40.51666635549543 -73.85317111305928) bank75093 +75094 POINT(41.18884971318681 -74.25215001191273) bank75094 +75095 POINT(40.41585760280144 -74.22019643607175) bank75095 +75096 POINT(40.60135921325251 -74.40943047447068) bank75096 +75097 POINT(41.3754481018973 -73.17504917987553) bank75097 +75098 POINT(40.04012363306611 -73.71451201782865) bank75098 +75099 POINT(41.489071803483164 -73.71292308089173) bank75099 +75100 POINT(41.40485653290641 -74.02256691283397) bank75100 +75101 POINT(40.10487759567521 -74.79249166778231) bank75101 +75102 POINT(40.84590909252161 -73.05982971316814) bank75102 +75103 POINT(41.63058336951306 -73.71074724856243) bank75103 +75104 POINT(41.03132996887916 -74.83206032032902) bank75104 +75105 POINT(40.87427711598237 -73.22581306090751) bank75105 +75106 POINT(41.50253046665246 -74.48364563499558) bank75106 +75107 POINT(40.64114504837993 -73.23135865969031) bank75107 +75108 POINT(39.952390255892006 -74.91950016095348) bank75108 +75109 POINT(40.98238876271809 -73.49463570585576) bank75109 +75110 POINT(40.78943945889186 -73.8482734662715) bank75110 +75111 POINT(41.43125854848652 -73.07593658792156) bank75111 +75112 POINT(41.25300818456268 -74.44969452347954) bank75112 +75113 POINT(41.367826923985206 -73.69100264433385) bank75113 +75114 POINT(41.710831810491044 -73.67645998629177) bank75114 +75115 POINT(39.862216164247734 -73.13514426174241) bank75115 +75116 POINT(40.15563686950432 -74.05687569898343) bank75116 +75117 POINT(39.89855139293748 -73.93138760401494) bank75117 +75118 POINT(39.94065747649914 -73.381143571107) bank75118 +75119 POINT(40.858101282064325 -73.49570785508826) bank75119 +75120 POINT(40.51555583558062 -74.01449959789724) bank75120 +75121 POINT(39.88008798066917 -73.88807462036073) bank75121 +75122 POINT(41.320254418932 -73.07365131688813) bank75122 +75123 POINT(40.458798983837774 -74.45813444585659) bank75123 +75124 POINT(41.575547502737614 -74.36774714932628) bank75124 +75125 POINT(41.18009746887696 -73.16079629334841) bank75125 +75126 POINT(41.26931408064016 -74.46069950758991) bank75126 +75127 POINT(41.232528728005796 -74.42634079851472) bank75127 +75128 POINT(41.0657317454486 -73.61178526931022) bank75128 +75129 POINT(40.69581346091894 -73.51881682771973) bank75129 +75130 POINT(40.243217797117154 -74.65078575102075) bank75130 +75131 POINT(41.35777215358698 -74.46622810940308) bank75131 +75132 POINT(40.17121639871532 -73.67155867944423) bank75132 +75133 POINT(41.47155984316286 -74.18486916760664) bank75133 +75134 POINT(41.17500990836158 -74.62434684080313) bank75134 +75135 POINT(41.25062615575798 -74.36473054229974) bank75135 +75136 POINT(41.054885175414995 -73.99930535773967) bank75136 +75137 POINT(41.54590499042763 -74.27167080474234) bank75137 +75138 POINT(39.979395566371146 -74.85430053677246) bank75138 +75139 POINT(39.89053120371212 -74.29980028533748) bank75139 +75140 POINT(40.3630036673921 -74.19140827112415) bank75140 +75141 POINT(41.39783989241982 -74.77054146880832) bank75141 +75142 POINT(40.93628518664368 -74.56357880720567) bank75142 +75143 POINT(40.17191575313404 -74.66318656942018) bank75143 +75144 POINT(41.27514970610275 -73.64450191567119) bank75144 +75145 POINT(40.7660097863353 -74.79446770635575) bank75145 +75146 POINT(41.39237444565372 -73.67461143779802) bank75146 +75147 POINT(41.582732433201606 -73.25148969587094) bank75147 +75148 POINT(39.7834382858092 -74.42258823474891) bank75148 +75149 POINT(40.964266723532496 -73.88237321707135) bank75149 +75150 POINT(41.42559281784021 -73.86243940295837) bank75150 +75151 POINT(40.17723854417964 -74.8653288855202) bank75151 +75152 POINT(39.90785748532649 -73.9672577436749) bank75152 +75153 POINT(41.42964952246201 -73.93635598585806) bank75153 +75154 POINT(40.75074576389113 -73.57963513171693) bank75154 +75155 POINT(40.79182396565997 -73.72680562938203) bank75155 +75156 POINT(40.80911325261536 -74.47322366898905) bank75156 +75157 POINT(40.04001569593612 -73.74086049281016) bank75157 +75158 POINT(40.62891643813157 -73.40637387169434) bank75158 +75159 POINT(40.79924549488339 -73.36657469385038) bank75159 +75160 POINT(41.58874253221989 -74.13539934330514) bank75160 +75161 POINT(40.34256020699207 -74.48353059215894) bank75161 +75162 POINT(41.49302321748297 -73.08611399242982) bank75162 +75163 POINT(41.492648260019294 -74.1682729737156) bank75163 +75164 POINT(41.15321832795808 -74.37259478267715) bank75164 +75165 POINT(40.38759922279362 -73.55726732161781) bank75165 +75166 POINT(40.118110214376664 -73.41924376915152) bank75166 +75167 POINT(41.04625551440757 -73.49960997081703) bank75167 +75168 POINT(41.56643614578225 -74.18341231428745) bank75168 +75169 POINT(41.20760719223651 -73.44327085457894) bank75169 +75170 POINT(41.700035318908895 -74.30449390050445) bank75170 +75171 POINT(41.69104003538111 -73.26696886814514) bank75171 +75172 POINT(39.83933731047311 -73.8855916278588) bank75172 +75173 POINT(40.890978651096304 -74.36166768410094) bank75173 +75174 POINT(39.75952279485634 -74.81294327319313) bank75174 +75175 POINT(39.864472441734065 -74.88513900422515) bank75175 +75176 POINT(41.26881206554069 -74.35869466549028) bank75176 +75177 POINT(40.99585912687038 -73.44682423430815) bank75177 +75178 POINT(40.08706715475986 -74.57369489832448) bank75178 +75179 POINT(40.39967148871981 -74.72027796875099) bank75179 +75180 POINT(41.090893460609 -73.50619909161965) bank75180 +75181 POINT(41.29082536587029 -74.10994440973903) bank75181 +75182 POINT(41.67012772805704 -73.62016464250084) bank75182 +75183 POINT(41.128525121628655 -73.86982233225604) bank75183 +75184 POINT(40.18457266310721 -74.14477127534195) bank75184 +75185 POINT(40.282296810678154 -74.83074162258791) bank75185 +75186 POINT(41.46372212001064 -73.42987050579907) bank75186 +75187 POINT(40.537731847701195 -74.96138320369198) bank75187 +75188 POINT(41.277984677643836 -73.09910406935391) bank75188 +75189 POINT(40.11955057000773 -74.4821389199892) bank75189 +75190 POINT(41.21018291048738 -74.4399051372952) bank75190 +75191 POINT(40.806158553834784 -73.59794312099886) bank75191 +75192 POINT(40.596311163936925 -73.38100720709487) bank75192 +75193 POINT(41.297772531020016 -74.62719396162518) bank75193 +75194 POINT(40.76878342532116 -74.62702128346241) bank75194 +75195 POINT(39.833916954392826 -73.33274980705862) bank75195 +75196 POINT(40.03161160821465 -73.7460194265856) bank75196 +75197 POINT(40.39092395853418 -73.19597131150368) bank75197 +75198 POINT(40.64549640820008 -73.01107574453947) bank75198 +75199 POINT(40.54795984366308 -74.75157000813765) bank75199 +75200 POINT(41.58665206227714 -73.47759887563721) bank75200 +75201 POINT(39.89858184786562 -74.97236152756854) bank75201 +75202 POINT(41.297554962974665 -74.05098923927856) bank75202 +75203 POINT(39.76457850801137 -73.80683284802575) bank75203 +75204 POINT(40.820146272683054 -73.14590753192277) bank75204 +75205 POINT(40.15313843987864 -73.68550275811253) bank75205 +75206 POINT(40.21035039902438 -73.29878995252514) bank75206 +75207 POINT(40.504184073572596 -74.91740856120617) bank75207 +75208 POINT(40.07212626004527 -74.31926923495298) bank75208 +75209 POINT(40.71237102957375 -73.25547706165807) bank75209 +75210 POINT(41.138607659889594 -73.13876158276116) bank75210 +75211 POINT(41.420361086895674 -74.09583554563454) bank75211 +75212 POINT(40.10435981848955 -73.84414896476741) bank75212 +75213 POINT(40.039542381230085 -74.01184015525608) bank75213 +75214 POINT(39.955744189358605 -73.15839507994953) bank75214 +75215 POINT(40.364177190057795 -74.91919345633535) bank75215 +75216 POINT(39.9586222541949 -73.3538509792514) bank75216 +75217 POINT(41.56303841069146 -73.79911227208565) bank75217 +75218 POINT(40.76538153990665 -73.06828043095739) bank75218 +75219 POINT(40.96516045035897 -74.02047634770157) bank75219 +75220 POINT(40.10224301358679 -73.84066462683191) bank75220 +75221 POINT(41.011038115504014 -73.36606780642157) bank75221 +75222 POINT(40.0208558558169 -74.8035158985877) bank75222 +75223 POINT(41.37586160093522 -73.18945656622036) bank75223 +75224 POINT(41.5109245504187 -74.33316092536346) bank75224 +75225 POINT(41.18757188181272 -74.44412023006048) bank75225 +75226 POINT(40.80924853047944 -74.63679719447194) bank75226 +75227 POINT(40.79334675413641 -73.07702425858344) bank75227 +75228 POINT(40.15594153863759 -73.11356808633703) bank75228 +75229 POINT(40.759412250765074 -74.69398204163957) bank75229 +75230 POINT(41.34435594255819 -74.11002184186447) bank75230 +75231 POINT(41.60895473467249 -73.81746550859768) bank75231 +75232 POINT(41.30886519263779 -73.01842947120483) bank75232 +75233 POINT(41.37215709933196 -73.15036217416024) bank75233 +75234 POINT(40.65942630244288 -74.17703469663951) bank75234 +75235 POINT(40.143590695068575 -73.19259549083907) bank75235 +75236 POINT(41.14264623199552 -74.80093133276264) bank75236 +75237 POINT(41.51966229201155 -74.44739856109916) bank75237 +75238 POINT(41.66308791410343 -74.8080578023063) bank75238 +75239 POINT(41.617277399844745 -74.57714276859264) bank75239 +75240 POINT(40.69738604726498 -73.49205165001904) bank75240 +75241 POINT(41.712582104817926 -74.48826565150381) bank75241 +75242 POINT(40.986325419612236 -73.20182504412155) bank75242 +75243 POINT(39.72342272978583 -73.25402869431676) bank75243 +75244 POINT(40.98680133281887 -74.61553941046259) bank75244 +75245 POINT(39.844947697958155 -73.81854859971354) bank75245 +75246 POINT(40.30040642651291 -74.5742983172213) bank75246 +75247 POINT(39.982390620797766 -74.441932775755) bank75247 +75248 POINT(39.773779813728076 -74.85348400137129) bank75248 +75249 POINT(40.945240192604295 -74.47421082854501) bank75249 +75250 POINT(40.07490300588946 -74.67307648798104) bank75250 +75251 POINT(40.13818341010361 -74.02469001164567) bank75251 +75252 POINT(39.99672744207958 -73.17401502067688) bank75252 +75253 POINT(40.399038254645674 -73.33417568406374) bank75253 +75254 POINT(41.163657582167836 -73.74787094068436) bank75254 +75255 POINT(39.96247643425032 -74.98041430046435) bank75255 +75256 POINT(40.80990604723688 -73.91305480740739) bank75256 +75257 POINT(41.55299882967094 -74.86016698186253) bank75257 +75258 POINT(40.284664799167324 -73.22254830908591) bank75258 +75259 POINT(41.37799373271231 -73.41041110462844) bank75259 +75260 POINT(40.30326426087655 -73.57661893141454) bank75260 +75261 POINT(40.116269281347485 -73.14341833159453) bank75261 +75262 POINT(40.10889650445224 -74.74235167580501) bank75262 +75263 POINT(41.396812514427815 -74.74058396529074) bank75263 +75264 POINT(41.20007269710541 -73.5484491395257) bank75264 +75265 POINT(40.182267086017866 -73.33954078682206) bank75265 +75266 POINT(40.400768047107796 -74.59084274717365) bank75266 +75267 POINT(41.605115840545594 -73.21682642994179) bank75267 +75268 POINT(40.621988613388424 -73.4429353424903) bank75268 +75269 POINT(41.37982616049684 -73.48778847746944) bank75269 +75270 POINT(41.58544771921102 -74.04143537080638) bank75270 +75271 POINT(40.198561272105856 -74.63850980095741) bank75271 +75272 POINT(40.7311464847213 -73.08020991721112) bank75272 +75273 POINT(41.65020550715575 -73.89342213413694) bank75273 +75274 POINT(39.77564142240971 -73.33450732331188) bank75274 +75275 POINT(41.43170470797701 -73.74937955202572) bank75275 +75276 POINT(41.02404053490777 -74.65964837468445) bank75276 +75277 POINT(40.69066163407645 -74.7774224206301) bank75277 +75278 POINT(40.0927311214129 -73.78203164911301) bank75278 +75279 POINT(39.713365159819155 -74.66568915245095) bank75279 +75280 POINT(41.582106903182115 -74.07152435514517) bank75280 +75281 POINT(39.994620020235 -73.9717666827486) bank75281 +75282 POINT(41.20511073257854 -74.11165157216782) bank75282 +75283 POINT(39.834494047718984 -73.89611757702008) bank75283 +75284 POINT(41.64489671288482 -73.17670268888274) bank75284 +75285 POINT(40.50415535006172 -73.34200928662712) bank75285 +75286 POINT(39.76453882651187 -74.78415960250081) bank75286 +75287 POINT(40.98925144705215 -73.66815457700174) bank75287 +75288 POINT(40.56599997010812 -74.67790817947753) bank75288 +75289 POINT(40.203148208442165 -74.94733021909404) bank75289 +75290 POINT(40.29836130476303 -73.87639578150207) bank75290 +75291 POINT(41.164346777979766 -73.64760382682316) bank75291 +75292 POINT(41.22689924090099 -74.91701278287758) bank75292 +75293 POINT(41.05596177155293 -73.77883041535841) bank75293 +75294 POINT(40.74480996614586 -73.3115652445978) bank75294 +75295 POINT(40.99395497449775 -73.39086768126637) bank75295 +75296 POINT(41.457751848151396 -74.39979603240164) bank75296 +75297 POINT(41.050278975524215 -73.43432938731537) bank75297 +75298 POINT(40.47654528235107 -73.8979409464838) bank75298 +75299 POINT(41.712552927760456 -73.64962483248519) bank75299 +75300 POINT(41.353269654246155 -74.40923811483287) bank75300 +75301 POINT(40.99541894635936 -74.6384672110024) bank75301 +75302 POINT(41.23876316486387 -73.36671300517897) bank75302 +75303 POINT(40.370435901486246 -73.8330977329458) bank75303 +75304 POINT(40.900310380111264 -73.84600477391494) bank75304 +75305 POINT(40.87367134374181 -73.55867392133862) bank75305 +75306 POINT(39.7759044614995 -74.07196513195288) bank75306 +75307 POINT(39.866431130858416 -73.76831609316228) bank75307 +75308 POINT(40.296642265461834 -74.15389454584393) bank75308 +75309 POINT(41.52645729909332 -74.07740165889369) bank75309 +75310 POINT(40.81532816561777 -74.05112296455358) bank75310 +75311 POINT(39.96414707491228 -73.02426317214632) bank75311 +75312 POINT(40.204683903767425 -74.56042495187896) bank75312 +75313 POINT(41.65847717388329 -73.34814992885205) bank75313 +75314 POINT(40.78969092880502 -73.62499506274833) bank75314 +75315 POINT(41.141966392457874 -73.73642074867873) bank75315 +75316 POINT(40.95816843511776 -73.20693386531084) bank75316 +75317 POINT(41.45333935180785 -74.72481794156552) bank75317 +75318 POINT(40.32836706919936 -73.0412994029099) bank75318 +75319 POINT(40.698529939629 -74.5080981723689) bank75319 +75320 POINT(41.59833740669097 -73.649746273552) bank75320 +75321 POINT(39.95421652724987 -73.77279943585309) bank75321 +75322 POINT(41.47709385250045 -73.93814848531562) bank75322 +75323 POINT(40.017476175017954 -74.81568063379027) bank75323 +75324 POINT(41.56352570317163 -73.90323585421159) bank75324 +75325 POINT(40.32278586768899 -74.51251633047701) bank75325 +75326 POINT(40.25465375748715 -73.27759247081018) bank75326 +75327 POINT(40.94391028039394 -73.18235868678235) bank75327 +75328 POINT(40.30914620510509 -73.80448501445241) bank75328 +75329 POINT(41.27063335282475 -74.22054465282577) bank75329 +75330 POINT(41.11022309460406 -73.08308626416982) bank75330 +75331 POINT(40.70439482324252 -74.55013730641514) bank75331 +75332 POINT(40.53039787358843 -74.91639885881257) bank75332 +75333 POINT(40.105349275562865 -74.46499056370585) bank75333 +75334 POINT(40.87787346969986 -74.87890451526651) bank75334 +75335 POINT(40.651473802872474 -74.94830846070937) bank75335 +75336 POINT(40.71888231928712 -74.47698485518201) bank75336 +75337 POINT(40.75512575957015 -73.55301414448095) bank75337 +75338 POINT(40.060051098511416 -73.70709073846672) bank75338 +75339 POINT(40.487646600516555 -73.34413493167074) bank75339 +75340 POINT(40.6869911183818 -73.20488684692953) bank75340 +75341 POINT(41.40187973283351 -74.81769295311896) bank75341 +75342 POINT(39.97708442818378 -73.77236779789183) bank75342 +75343 POINT(40.92696985008407 -74.35354858062067) bank75343 +75344 POINT(41.5417934808244 -74.89368209246526) bank75344 +75345 POINT(40.88333856574997 -74.17090885001619) bank75345 +75346 POINT(41.34871372443649 -73.71267673708999) bank75346 +75347 POINT(41.28973175178968 -74.71337484923852) bank75347 +75348 POINT(39.79181118259243 -74.76409079210362) bank75348 +75349 POINT(40.064549113016916 -73.15611000442391) bank75349 +75350 POINT(40.941985161363036 -74.30569594652881) bank75350 +75351 POINT(39.78666413134522 -74.2279388742219) bank75351 +75352 POINT(40.01159729579568 -74.89510519349285) bank75352 +75353 POINT(39.75813522303661 -73.06036609816692) bank75353 +75354 POINT(40.36332287546593 -74.711551046801) bank75354 +75355 POINT(39.78218760313575 -74.84475624470835) bank75355 +75356 POINT(39.81056012841772 -74.39825004023687) bank75356 +75357 POINT(41.278218327760975 -74.05583237616185) bank75357 +75358 POINT(40.68728087945682 -73.23676798311541) bank75358 +75359 POINT(40.07203410199698 -73.07313099840512) bank75359 +75360 POINT(40.16342457110025 -74.96132933213377) bank75360 +75361 POINT(41.14191201292417 -74.89896978579372) bank75361 +75362 POINT(41.294447475753636 -73.50157413382928) bank75362 +75363 POINT(40.61248454861418 -74.11543575232233) bank75363 +75364 POINT(41.16024445243851 -74.26945637756675) bank75364 +75365 POINT(41.08540902618746 -74.24646356368814) bank75365 +75366 POINT(41.326100362387386 -74.16395044616422) bank75366 +75367 POINT(40.2657028874803 -74.79574492301765) bank75367 +75368 POINT(40.97238437471945 -74.50601563097975) bank75368 +75369 POINT(41.55917874370102 -73.21685955456161) bank75369 +75370 POINT(40.43177256922138 -74.55213488559514) bank75370 +75371 POINT(41.398662117193936 -74.5646300606333) bank75371 +75372 POINT(40.30961906900112 -74.77356336869806) bank75372 +75373 POINT(41.24617548210412 -74.07386732771768) bank75373 +75374 POINT(40.26908326750689 -73.07248729680445) bank75374 +75375 POINT(41.16802597722185 -74.23854130852276) bank75375 +75376 POINT(40.992037355012535 -74.04876206730685) bank75376 +75377 POINT(40.99142847278861 -74.48102883059008) bank75377 +75378 POINT(40.312822425379494 -74.90844938031198) bank75378 +75379 POINT(41.406306256288794 -73.68040909979584) bank75379 +75380 POINT(40.84867183708171 -74.98941216478097) bank75380 +75381 POINT(41.696289899670006 -73.19110138672369) bank75381 +75382 POINT(39.723480440294736 -74.23699625452385) bank75382 +75383 POINT(40.20187755548685 -73.30783296612326) bank75383 +75384 POINT(40.918568664734295 -73.74406616481681) bank75384 +75385 POINT(41.38482242276532 -73.24536901630536) bank75385 +75386 POINT(40.949235314451634 -73.69576906660318) bank75386 +75387 POINT(40.81279768566367 -73.07275593437389) bank75387 +75388 POINT(39.92894597171115 -74.88352365224893) bank75388 +75389 POINT(40.118055177384726 -74.58776306438682) bank75389 +75390 POINT(40.16308341151875 -73.91071917332354) bank75390 +75391 POINT(40.53876226301825 -73.44788663497518) bank75391 +75392 POINT(39.89334191784739 -74.46950676459495) bank75392 +75393 POINT(41.48272098613218 -73.66343819071994) bank75393 +75394 POINT(40.63842401793727 -73.88994915309225) bank75394 +75395 POINT(40.26575794217071 -74.56622859693918) bank75395 +75396 POINT(41.30596383278174 -74.84145198985357) bank75396 +75397 POINT(40.04992056369569 -73.7885111106024) bank75397 +75398 POINT(41.10370962735796 -74.82253800264542) bank75398 +75399 POINT(40.996779108328845 -73.59294927349477) bank75399 +75400 POINT(40.57636719130186 -73.38518593955939) bank75400 +75401 POINT(40.92036065920772 -73.76822448951744) bank75401 +75402 POINT(39.743729134977215 -74.78944317074918) bank75402 +75403 POINT(40.70297972565822 -73.8007704954745) bank75403 +75404 POINT(41.13638145276958 -74.80326485152892) bank75404 +75405 POINT(40.439933837349685 -73.0222697881544) bank75405 +75406 POINT(41.4107617591872 -73.79960938567908) bank75406 +75407 POINT(40.48974927260567 -74.05977920117942) bank75407 +75408 POINT(41.6039558278874 -73.72673934402319) bank75408 +75409 POINT(41.33604020474112 -74.969056912238) bank75409 +75410 POINT(41.01520497833345 -73.56425068590727) bank75410 +75411 POINT(40.25426015636518 -73.5810271569977) bank75411 +75412 POINT(40.05655127413237 -74.99657271390107) bank75412 +75413 POINT(40.62823716010697 -74.98463215178732) bank75413 +75414 POINT(41.355547292060706 -73.01927863884681) bank75414 +75415 POINT(40.73554389229299 -73.5948229142872) bank75415 +75416 POINT(40.636214573584176 -74.37090774402792) bank75416 +75417 POINT(41.02020919390145 -73.57894737398857) bank75417 +75418 POINT(40.1846774570287 -73.48345821629175) bank75418 +75419 POINT(41.20788341252747 -73.55193350229166) bank75419 +75420 POINT(40.352652935966844 -74.11165858285791) bank75420 +75421 POINT(40.78536027497431 -74.42612421605381) bank75421 +75422 POINT(39.757121785628584 -74.73359980637933) bank75422 +75423 POINT(41.65802777961346 -73.92502542502268) bank75423 +75424 POINT(40.48848515162582 -74.83079493907017) bank75424 +75425 POINT(39.91575091084575 -74.97894073905631) bank75425 +75426 POINT(41.65183740990591 -73.90817769237516) bank75426 +75427 POINT(40.03174024967969 -73.91280826125013) bank75427 +75428 POINT(41.617787816962675 -73.47268100517107) bank75428 +75429 POINT(40.416834926782656 -73.2388592846016) bank75429 +75430 POINT(41.345766282377284 -74.12325388072563) bank75430 +75431 POINT(41.383558208865225 -73.53053156736095) bank75431 +75432 POINT(39.99686919070658 -74.52408651616952) bank75432 +75433 POINT(41.45187046404838 -73.4456271127029) bank75433 +75434 POINT(41.17703303466408 -74.51235429336083) bank75434 +75435 POINT(40.71879996176948 -74.686795536101) bank75435 +75436 POINT(41.661183000286805 -73.58959220455799) bank75436 +75437 POINT(40.853422386193714 -73.57952794911229) bank75437 +75438 POINT(39.82226884600583 -74.34557322322254) bank75438 +75439 POINT(41.00756259979249 -73.34373381987466) bank75439 +75440 POINT(40.0851282058924 -73.80180516584429) bank75440 +75441 POINT(39.90105192013946 -73.63314878731255) bank75441 +75442 POINT(41.320735942123115 -73.98147756138326) bank75442 +75443 POINT(40.83355482272889 -74.13144322190186) bank75443 +75444 POINT(41.0663367706799 -73.60443001228019) bank75444 +75445 POINT(40.21642644070431 -74.30588369881666) bank75445 +75446 POINT(40.100213132705775 -75.00031504715794) bank75446 +75447 POINT(40.44912948715982 -74.13863500488955) bank75447 +75448 POINT(41.41556752179587 -74.34647837971096) bank75448 +75449 POINT(41.15692411231185 -74.87829739790489) bank75449 +75450 POINT(41.23119819230545 -73.87304662977934) bank75450 +75451 POINT(41.03464089121766 -74.01699314664629) bank75451 +75452 POINT(41.60990388114587 -73.17208863451702) bank75452 +75453 POINT(41.350985830734835 -74.76995329999184) bank75453 +75454 POINT(40.457141275295314 -74.1997362052173) bank75454 +75455 POINT(40.60888695130558 -74.7005694214833) bank75455 +75456 POINT(39.83463784167825 -73.38755265843558) bank75456 +75457 POINT(40.79646267354566 -74.01029977925795) bank75457 +75458 POINT(40.77358976514713 -73.10265483343956) bank75458 +75459 POINT(41.25870708315394 -74.6260758819974) bank75459 +75460 POINT(41.19671305615144 -74.2141594190342) bank75460 +75461 POINT(41.61416236796149 -74.47259354703355) bank75461 +75462 POINT(40.19418373617612 -73.56008230284446) bank75462 +75463 POINT(40.68646220856949 -74.28244590701858) bank75463 +75464 POINT(40.24561142337336 -74.22740483682041) bank75464 +75465 POINT(40.18396117947503 -74.86153959094663) bank75465 +75466 POINT(41.379370605002215 -74.20517093216012) bank75466 +75467 POINT(39.75489153787789 -74.55116962273682) bank75467 +75468 POINT(40.67766760545141 -73.97071289672455) bank75468 +75469 POINT(41.13920259589405 -73.91329152186957) bank75469 +75470 POINT(40.59909702413984 -73.2649863139659) bank75470 +75471 POINT(40.68499474793947 -74.50549668620882) bank75471 +75472 POINT(40.198025976497576 -74.0540698697927) bank75472 +75473 POINT(40.735567404102866 -74.49575346468367) bank75473 +75474 POINT(40.92533232665179 -73.17843255565293) bank75474 +75475 POINT(40.294991754035415 -73.39824700667876) bank75475 +75476 POINT(40.89521403063334 -73.55097385962229) bank75476 +75477 POINT(39.73947292288341 -74.40377588823269) bank75477 +75478 POINT(40.56644314670785 -74.14452779033205) bank75478 +75479 POINT(40.97306317902778 -73.11534973105515) bank75479 +75480 POINT(41.33658013060509 -74.81976754826283) bank75480 +75481 POINT(41.49296370227943 -74.80410011353592) bank75481 +75482 POINT(39.76079993402921 -74.14438228021227) bank75482 +75483 POINT(41.03795594106782 -73.56149793057142) bank75483 +75484 POINT(40.33189426532003 -74.58684002694001) bank75484 +75485 POINT(40.25794363966563 -73.17940988959457) bank75485 +75486 POINT(39.93550933423315 -74.91076820281788) bank75486 +75487 POINT(41.672812699775136 -74.54190845117174) bank75487 +75488 POINT(40.143044666751784 -73.76911882085918) bank75488 +75489 POINT(41.433791412125174 -74.55886262660928) bank75489 +75490 POINT(41.304157209995076 -73.86599698886764) bank75490 +75491 POINT(39.94028349228932 -73.5996532349894) bank75491 +75492 POINT(40.94547006239278 -74.77134861088194) bank75492 +75493 POINT(40.8367630057831 -73.07187167734881) bank75493 +75494 POINT(41.257916672802914 -74.4044428997364) bank75494 +75495 POINT(40.3029001744979 -74.42397138397301) bank75495 +75496 POINT(40.608104870268356 -73.6083801208135) bank75496 +75497 POINT(41.71272296039238 -74.67057054174876) bank75497 +75498 POINT(41.33557700333011 -74.0270838054722) bank75498 +75499 POINT(41.085246304823336 -74.2484894172231) bank75499 +75500 POINT(41.14837499796568 -73.5269280286098) bank75500 +75501 POINT(41.384367211097775 -73.55582402769089) bank75501 +75502 POINT(41.71089618168169 -73.47900712516484) bank75502 +75503 POINT(41.42582591500713 -73.24782419801441) bank75503 +75504 POINT(41.345875014867566 -74.97457050749819) bank75504 +75505 POINT(41.71157828614606 -74.4738674404089) bank75505 +75506 POINT(41.4200584251394 -73.63212305744992) bank75506 +75507 POINT(40.66763141422966 -73.64872759102674) bank75507 +75508 POINT(40.997858952999586 -73.48738071000865) bank75508 +75509 POINT(40.40604125470801 -73.26705690800057) bank75509 +75510 POINT(41.055793442229735 -74.77689889830518) bank75510 +75511 POINT(41.335263713061266 -74.87450034278892) bank75511 +75512 POINT(40.38827903105523 -73.0147783397809) bank75512 +75513 POINT(40.79615972643219 -73.92174930780301) bank75513 +75514 POINT(40.15031649485863 -74.28959182131617) bank75514 +75515 POINT(40.160158169338324 -73.16984255593341) bank75515 +75516 POINT(40.48401203224671 -73.94132199325978) bank75516 +75517 POINT(41.0291815276917 -73.92610885200791) bank75517 +75518 POINT(40.95300311912663 -74.59434445212487) bank75518 +75519 POINT(41.00240964555971 -74.61457727160895) bank75519 +75520 POINT(41.07236483858063 -74.60137809426656) bank75520 +75521 POINT(40.11979304777959 -74.50082320427998) bank75521 +75522 POINT(41.707809102614625 -74.17751975049322) bank75522 +75523 POINT(39.872112761747914 -73.18491522670877) bank75523 +75524 POINT(41.69331576777746 -73.23578967145579) bank75524 +75525 POINT(40.01529398456388 -74.02225201763139) bank75525 +75526 POINT(41.09382053138277 -74.03539580567349) bank75526 +75527 POINT(40.84635586804607 -74.88902221971631) bank75527 +75528 POINT(40.23106997743267 -73.12340414555632) bank75528 +75529 POINT(40.79767549199453 -73.09396707203925) bank75529 +75530 POINT(40.818507454747945 -73.09637189292829) bank75530 +75531 POINT(40.1536414669189 -73.06093003665093) bank75531 +75532 POINT(40.75792877850697 -74.57371890023472) bank75532 +75533 POINT(39.81490374788263 -74.1355243271125) bank75533 +75534 POINT(41.24692027308806 -74.88259398539073) bank75534 +75535 POINT(40.144406228403504 -74.07746591721998) bank75535 +75536 POINT(40.89995225535379 -73.09882923076202) bank75536 +75537 POINT(39.99833928286773 -73.6840755837366) bank75537 +75538 POINT(40.09130436529031 -74.17524825063572) bank75538 +75539 POINT(40.67229074844996 -73.73162161907128) bank75539 +75540 POINT(40.382166163676395 -74.25922654313521) bank75540 +75541 POINT(41.250067768931174 -73.23652703235852) bank75541 +75542 POINT(40.742960053816255 -73.63005668188833) bank75542 +75543 POINT(41.149890906793935 -73.76412871588217) bank75543 +75544 POINT(40.38017241040928 -74.15540150242664) bank75544 +75545 POINT(39.89592476879707 -73.36322328789633) bank75545 +75546 POINT(40.625241552680116 -74.66380793816533) bank75546 +75547 POINT(40.396709397149394 -73.61341519764588) bank75547 +75548 POINT(41.07240418806303 -74.44460714441594) bank75548 +75549 POINT(40.37789675049029 -74.4684727607353) bank75549 +75550 POINT(40.47173110885004 -73.60030665489093) bank75550 +75551 POINT(39.953534545758345 -74.48625669681188) bank75551 +75552 POINT(41.315047017332596 -74.61135164769435) bank75552 +75553 POINT(40.42917207141675 -74.52578969008076) bank75553 +75554 POINT(40.98660399610826 -73.64216710625558) bank75554 +75555 POINT(41.023870119942586 -74.61449283539768) bank75555 +75556 POINT(41.61113018980641 -73.28563510265252) bank75556 +75557 POINT(41.13902571747043 -73.69839615564828) bank75557 +75558 POINT(41.5530083318295 -74.96964648724413) bank75558 +75559 POINT(40.42168300055221 -73.64840375186391) bank75559 +75560 POINT(40.955331851519986 -73.4533088302694) bank75560 +75561 POINT(41.04320941838874 -73.72020799792823) bank75561 +75562 POINT(39.89146956839767 -74.71541127103337) bank75562 +75563 POINT(41.57576905732829 -75.00381026865945) bank75563 +75564 POINT(39.87937877328895 -73.43384319736074) bank75564 +75565 POINT(40.39158290987746 -74.72749301509532) bank75565 +75566 POINT(41.14322972679301 -73.44113991802406) bank75566 +75567 POINT(39.74916323698457 -74.70062041050258) bank75567 +75568 POINT(40.778343964405565 -73.60628145734772) bank75568 +75569 POINT(40.586021594253054 -74.44000327776585) bank75569 +75570 POINT(41.7028286485423 -73.5412963347431) bank75570 +75571 POINT(40.19322304269144 -73.50930599840589) bank75571 +75572 POINT(40.91219878463312 -73.30379935588059) bank75572 +75573 POINT(40.295909408199904 -74.3392449387644) bank75573 +75574 POINT(40.21718243421845 -73.17946217972614) bank75574 +75575 POINT(39.94080856674549 -73.19010864422344) bank75575 +75576 POINT(40.701095710433165 -74.871885588939) bank75576 +75577 POINT(40.38311943440568 -73.85097680572672) bank75577 +75578 POINT(41.20124937805084 -73.58385904078656) bank75578 +75579 POINT(40.88681599933495 -74.78462334534225) bank75579 +75580 POINT(40.60089870450752 -73.04024906679453) bank75580 +75581 POINT(40.78700109086086 -74.47924560441142) bank75581 +75582 POINT(40.49664408115122 -74.58007526105344) bank75582 +75583 POINT(41.66222871374637 -73.95545290111441) bank75583 +75584 POINT(41.09537617518231 -73.52527350744946) bank75584 +75585 POINT(40.25573917141607 -73.03277077989134) bank75585 +75586 POINT(40.13418084508454 -73.78182942302759) bank75586 +75587 POINT(40.980318865689725 -73.15598975602191) bank75587 +75588 POINT(41.44397815775513 -74.68986416074546) bank75588 +75589 POINT(40.54220394055337 -74.87872262690938) bank75589 +75590 POINT(41.249931415164596 -73.1980613079808) bank75590 +75591 POINT(39.994956235031225 -73.58861019716447) bank75591 +75592 POINT(41.30608409960752 -73.19103155461863) bank75592 +75593 POINT(41.19292308065837 -74.7543161249474) bank75593 +75594 POINT(40.321694204551626 -74.98887428285822) bank75594 +75595 POINT(41.18237534319529 -73.28415837591072) bank75595 +75596 POINT(41.18215615310039 -74.8686386654118) bank75596 +75597 POINT(41.55175877627694 -73.65672602862601) bank75597 +75598 POINT(40.33997600658507 -74.33830934477992) bank75598 +75599 POINT(41.58075927669626 -74.46376757659387) bank75599 +75600 POINT(40.339612202288954 -74.15759680044916) bank75600 +75601 POINT(41.34222260784023 -74.48663161276808) bank75601 +75602 POINT(41.552305543566185 -73.73933114661958) bank75602 +75603 POINT(40.91427126810578 -73.60039782183156) bank75603 +75604 POINT(40.57481007398772 -74.83523515429425) bank75604 +75605 POINT(40.1902085468711 -74.14516155266091) bank75605 +75606 POINT(39.73698863782793 -73.10213825995463) bank75606 +75607 POINT(40.4356707194169 -73.30036194698987) bank75607 +75608 POINT(41.605650315573826 -74.60434923224494) bank75608 +75609 POINT(40.920540526256566 -73.39377441512237) bank75609 +75610 POINT(39.95477258943488 -74.31722755019112) bank75610 +75611 POINT(39.7927015911118 -73.52586369103182) bank75611 +75612 POINT(41.45632214871874 -73.24966941887463) bank75612 +75613 POINT(40.39582921306716 -73.78062403477686) bank75613 +75614 POINT(40.555223760667275 -73.15597155959955) bank75614 +75615 POINT(40.424425319244314 -73.5833051750431) bank75615 +75616 POINT(39.91411061386013 -73.917102503359) bank75616 +75617 POINT(40.90750238816962 -73.86239386350678) bank75617 +75618 POINT(40.100295833540734 -73.16058797642768) bank75618 +75619 POINT(41.26724366086827 -73.59405296739182) bank75619 +75620 POINT(39.757352119464876 -74.38949552885063) bank75620 +75621 POINT(41.02006793031088 -74.61148046177823) bank75621 +75622 POINT(40.017006025131124 -74.14233225937319) bank75622 +75623 POINT(40.475172569324286 -74.10456119087002) bank75623 +75624 POINT(41.19153568655366 -74.9146320137362) bank75624 +75625 POINT(41.64817380139299 -73.06326376484911) bank75625 +75626 POINT(40.09089816560608 -73.5450864093453) bank75626 +75627 POINT(41.2430996896195 -74.82348872785218) bank75627 +75628 POINT(40.29720810661641 -73.71716528214884) bank75628 +75629 POINT(40.179154268887324 -73.43718688386824) bank75629 +75630 POINT(41.495026783875794 -74.12146848218431) bank75630 +75631 POINT(39.9733120889313 -74.85968216244028) bank75631 +75632 POINT(40.178164727723356 -74.26843912286473) bank75632 +75633 POINT(39.834985380466144 -74.87292000360331) bank75633 +75634 POINT(40.263975375980564 -73.95299810923837) bank75634 +75635 POINT(41.24609501261113 -74.97690348679996) bank75635 +75636 POINT(40.94170580431431 -73.05289083309646) bank75636 +75637 POINT(41.37735979088163 -73.26725773052301) bank75637 +75638 POINT(41.39228232436291 -73.46152880364657) bank75638 +75639 POINT(39.80612226794679 -73.42311243808302) bank75639 +75640 POINT(40.93930640306864 -74.51921533618707) bank75640 +75641 POINT(41.52758530688231 -74.04117804739953) bank75641 +75642 POINT(41.557411084015186 -74.26649299114014) bank75642 +75643 POINT(40.740624779982625 -74.98437947176872) bank75643 +75644 POINT(41.25999377885571 -74.70615042864728) bank75644 +75645 POINT(41.599967676024306 -73.03271519075732) bank75645 +75646 POINT(40.09623846449367 -73.75313571265107) bank75646 +75647 POINT(41.15358114012017 -73.19645141985332) bank75647 +75648 POINT(40.99367043213238 -73.24454557746934) bank75648 +75649 POINT(41.12318599373352 -74.98709219171784) bank75649 +75650 POINT(40.83246249789186 -73.1264161636396) bank75650 +75651 POINT(40.40891466710868 -74.30712430757231) bank75651 +75652 POINT(40.850290989171825 -74.56796029168981) bank75652 +75653 POINT(40.22411888714766 -74.70347784068662) bank75653 +75654 POINT(39.812971456089414 -74.29721718781585) bank75654 +75655 POINT(40.79981172615669 -74.27267401489523) bank75655 +75656 POINT(40.54235264695862 -74.07078454075142) bank75656 +75657 POINT(39.82520736618613 -74.66857321067499) bank75657 +75658 POINT(40.27969721658718 -74.33431466587886) bank75658 +75659 POINT(40.36119962146253 -73.01407780324334) bank75659 +75660 POINT(40.266344186183616 -74.69681592046048) bank75660 +75661 POINT(40.176051775362495 -73.31042059215956) bank75661 +75662 POINT(40.04726741757061 -74.1906706565501) bank75662 +75663 POINT(41.591574445948126 -73.68918492690752) bank75663 +75664 POINT(41.304336168818764 -73.32024862209776) bank75664 +75665 POINT(40.53899757706233 -73.89768647988568) bank75665 +75666 POINT(39.72044887232531 -74.29215785861888) bank75666 +75667 POINT(40.75508449609867 -74.8950765139366) bank75667 +75668 POINT(40.74616129330611 -74.45845704049525) bank75668 +75669 POINT(40.020689593557506 -74.79118149269551) bank75669 +75670 POINT(41.16609862570768 -73.10701275695158) bank75670 +75671 POINT(40.13107827533173 -74.2807340303032) bank75671 +75672 POINT(41.17543851508738 -74.43751903064518) bank75672 +75673 POINT(40.41805523956587 -73.0232030882483) bank75673 +75674 POINT(40.10689451561713 -73.26029665110876) bank75674 +75675 POINT(40.74334347748757 -74.20963538449708) bank75675 +75676 POINT(41.45637232072951 -74.87242201926318) bank75676 +75677 POINT(40.09053418607405 -74.24550641925045) bank75677 +75678 POINT(39.71839571931933 -74.37980486772793) bank75678 +75679 POINT(41.089083131108694 -73.7115780558592) bank75679 +75680 POINT(41.11050528903538 -73.78323035887406) bank75680 +75681 POINT(41.02185574793585 -74.74758958561483) bank75681 +75682 POINT(40.72432614829119 -74.79379134779781) bank75682 +75683 POINT(41.64600520200817 -73.42533045762268) bank75683 +75684 POINT(40.279048286921125 -74.80511919572218) bank75684 +75685 POINT(40.7891212999933 -74.6540767246082) bank75685 +75686 POINT(39.819164142963096 -73.37705277142263) bank75686 +75687 POINT(40.93250706021698 -74.50543183064461) bank75687 +75688 POINT(41.26592394603712 -73.22818751037701) bank75688 +75689 POINT(41.521238968890046 -74.475046996549) bank75689 +75690 POINT(40.20861414042165 -73.89177815344262) bank75690 +75691 POINT(41.26883736845941 -74.42755820564517) bank75691 +75692 POINT(41.500217566146496 -73.75120691413466) bank75692 +75693 POINT(40.59651060710814 -74.61993477955629) bank75693 +75694 POINT(40.223287959425015 -74.66486559090252) bank75694 +75695 POINT(40.77886238362142 -74.03288894551771) bank75695 +75696 POINT(41.12841374132491 -74.9404132632607) bank75696 +75697 POINT(41.22692508427536 -73.52821222417185) bank75697 +75698 POINT(41.27782819317723 -73.13257823230526) bank75698 +75699 POINT(40.32830496290237 -73.02945266549082) bank75699 +75700 POINT(40.14098903605614 -74.86597074213496) bank75700 +75701 POINT(41.35756247163513 -73.95816173585517) bank75701 +75702 POINT(39.92250572695217 -74.0190344115452) bank75702 +75703 POINT(41.03172808158642 -74.50642038455514) bank75703 +75704 POINT(40.89737417982113 -74.54843867434016) bank75704 +75705 POINT(40.21559790033148 -74.7111637054073) bank75705 +75706 POINT(40.06889279177259 -73.44698879698296) bank75706 +75707 POINT(41.46890552393057 -74.58399957679093) bank75707 +75708 POINT(41.54983397153222 -74.1416327919108) bank75708 +75709 POINT(41.32052551382687 -74.26446217623621) bank75709 +75710 POINT(40.74981942448286 -74.10414537909622) bank75710 +75711 POINT(41.32151733055187 -74.34519307047645) bank75711 +75712 POINT(39.9152244903454 -73.8137154529174) bank75712 +75713 POINT(41.57661236832663 -73.8153612674458) bank75713 +75714 POINT(41.32997379743561 -74.97659328058457) bank75714 +75715 POINT(40.6081787523418 -73.74075509626131) bank75715 +75716 POINT(39.73078482363124 -74.49437068934603) bank75716 +75717 POINT(39.96092326840656 -75.0057958132819) bank75717 +75718 POINT(40.85100590779778 -74.16682963697178) bank75718 +75719 POINT(41.13687000004988 -74.03055722682585) bank75719 +75720 POINT(41.382990266631154 -73.61369950075346) bank75720 +75721 POINT(41.52165241735679 -73.94489656030379) bank75721 +75722 POINT(41.293241249117685 -73.37580134264964) bank75722 +75723 POINT(41.60107536202544 -74.32079898861345) bank75723 +75724 POINT(40.06852843061425 -74.32867005924341) bank75724 +75725 POINT(41.11930247935157 -74.8933783005636) bank75725 +75726 POINT(40.39520972962041 -74.24974359556545) bank75726 +75727 POINT(41.180834445954524 -74.71335157498206) bank75727 +75728 POINT(40.01329770971231 -74.91957958505455) bank75728 +75729 POINT(40.389950703067655 -74.25597355907699) bank75729 +75730 POINT(39.74877811261037 -74.2360461382989) bank75730 +75731 POINT(41.464745773484225 -74.55760455063336) bank75731 +75732 POINT(39.974103517053095 -74.96304458905776) bank75732 +75733 POINT(39.96945581840758 -73.74360923578601) bank75733 +75734 POINT(40.852477262823335 -74.61208953134353) bank75734 +75735 POINT(40.82537703126463 -74.52979267237664) bank75735 +75736 POINT(40.605101585549235 -73.7496507086163) bank75736 +75737 POINT(39.987509116284805 -73.11518757981537) bank75737 +75738 POINT(41.226596623064715 -74.16652405452739) bank75738 +75739 POINT(40.05948927409001 -74.84107569240764) bank75739 +75740 POINT(40.45281827558704 -73.25490287838964) bank75740 +75741 POINT(39.93895479198397 -73.2475073749891) bank75741 +75742 POINT(41.01498930680072 -73.70291451848409) bank75742 +75743 POINT(41.139757615114036 -73.19461342619212) bank75743 +75744 POINT(40.83978266680071 -74.60000584057407) bank75744 +75745 POINT(41.05953979175144 -73.90202440468589) bank75745 +75746 POINT(41.195196620528016 -74.9353166737321) bank75746 +75747 POINT(40.948253683221196 -74.83738873205695) bank75747 +75748 POINT(40.42388427854416 -73.66345048154781) bank75748 +75749 POINT(40.2335683840718 -74.35002505688146) bank75749 +75750 POINT(39.89658887233216 -73.6919899591589) bank75750 +75751 POINT(41.60437376968374 -74.75009027320877) bank75751 +75752 POINT(41.57648472593895 -74.48151437921933) bank75752 +75753 POINT(39.857310749896456 -74.37096066857264) bank75753 +75754 POINT(41.28539992268209 -74.46319990319463) bank75754 +75755 POINT(39.91733749202974 -73.00769895546125) bank75755 +75756 POINT(40.78882541600102 -74.26703746115354) bank75756 +75757 POINT(40.2790668012075 -73.2673130077905) bank75757 +75758 POINT(40.69030836559846 -73.05401468970248) bank75758 +75759 POINT(39.989124950031545 -74.18827177107494) bank75759 +75760 POINT(40.57164769444193 -74.73450803944931) bank75760 +75761 POINT(40.53048926342511 -74.4141617001346) bank75761 +75762 POINT(41.148007594521296 -73.6118544930529) bank75762 +75763 POINT(39.85140859070585 -73.16103332617507) bank75763 +75764 POINT(41.40673331085772 -74.39885076092506) bank75764 +75765 POINT(39.973685142514256 -74.41114114349352) bank75765 +75766 POINT(40.682698115719475 -74.35815722053516) bank75766 +75767 POINT(40.47278973177128 -73.27463491011244) bank75767 +75768 POINT(40.37512711459704 -73.22975517440075) bank75768 +75769 POINT(41.40821959837485 -74.31156771575334) bank75769 +75770 POINT(40.511936959947384 -73.71412554661794) bank75770 +75771 POINT(39.811647174267534 -73.39531210535705) bank75771 +75772 POINT(41.642876467999415 -74.04346358943262) bank75772 +75773 POINT(40.96705630066578 -74.84364756067075) bank75773 +75774 POINT(40.02692340362175 -73.70510075058172) bank75774 +75775 POINT(40.626536638880616 -73.07928085316647) bank75775 +75776 POINT(39.84806781588231 -74.56086197212367) bank75776 +75777 POINT(41.49267276294028 -73.44505651215744) bank75777 +75778 POINT(40.985102276809315 -73.25405011185555) bank75778 +75779 POINT(40.17862570938777 -73.82210296426149) bank75779 +75780 POINT(41.15690749253922 -74.44654756625128) bank75780 +75781 POINT(41.675305991700135 -74.93327382220187) bank75781 +75782 POINT(41.307995871222246 -73.25333074521956) bank75782 +75783 POINT(41.64642428539476 -74.15293614570557) bank75783 +75784 POINT(40.62914728501756 -73.64678403457205) bank75784 +75785 POINT(40.35269902021975 -74.32321136029049) bank75785 +75786 POINT(40.64163425183123 -74.08632113211462) bank75786 +75787 POINT(40.337055900950936 -74.98950626129498) bank75787 +75788 POINT(40.02318805146573 -74.37471693510503) bank75788 +75789 POINT(39.84112517300045 -73.0119753293392) bank75789 +75790 POINT(40.237257240794825 -73.0860477797748) bank75790 +75791 POINT(40.44123735790691 -74.95232361500537) bank75791 +75792 POINT(39.733330072215544 -74.6235336786214) bank75792 +75793 POINT(41.011106403415326 -74.89994995696298) bank75793 +75794 POINT(39.766947137854324 -73.16107983279228) bank75794 +75795 POINT(40.101739958458715 -74.30550353501232) bank75795 +75796 POINT(41.430438652658616 -73.5880905044381) bank75796 +75797 POINT(40.02202490553169 -73.88626185503016) bank75797 +75798 POINT(40.909042108837426 -74.57006677846354) bank75798 +75799 POINT(41.228650823446145 -73.05957473251364) bank75799 +75800 POINT(39.84371787839571 -73.94295470137347) bank75800 +75801 POINT(41.49160299162608 -74.73725224004741) bank75801 +75802 POINT(40.58095842629663 -73.63950816460253) bank75802 +75803 POINT(41.18348672146077 -73.07902365017652) bank75803 +75804 POINT(41.23940546127896 -74.61510625079151) bank75804 +75805 POINT(41.56887136906551 -74.03552750979988) bank75805 +75806 POINT(40.506694731724565 -74.39429085187143) bank75806 +75807 POINT(40.54071103847082 -73.01916653056294) bank75807 +75808 POINT(40.23371360241437 -74.07753762137997) bank75808 +75809 POINT(40.729877134469184 -74.1876209235668) bank75809 +75810 POINT(41.293412414817595 -74.78387049453995) bank75810 +75811 POINT(40.823675183520166 -74.68133015328274) bank75811 +75812 POINT(41.35383246430823 -73.7996467653066) bank75812 +75813 POINT(40.23220144815728 -74.01855596413975) bank75813 +75814 POINT(40.96589521333378 -74.90293718123102) bank75814 +75815 POINT(41.06010154946835 -73.5178568649073) bank75815 +75816 POINT(40.94183993231407 -74.98791305049323) bank75816 +75817 POINT(41.165631821172 -74.96469195334126) bank75817 +75818 POINT(40.01054828367295 -73.24993050986872) bank75818 +75819 POINT(40.78967656418066 -73.93057032376186) bank75819 +75820 POINT(41.33777395290769 -73.3881943166445) bank75820 +75821 POINT(41.271707326451285 -73.32494187756295) bank75821 +75822 POINT(41.663024863569795 -73.20363698367) bank75822 +75823 POINT(40.222231509580546 -73.66905207751826) bank75823 +75824 POINT(41.04623060314961 -74.70143070967588) bank75824 +75825 POINT(40.81674449794041 -74.06173303433796) bank75825 +75826 POINT(40.75100287814333 -74.57287070558388) bank75826 +75827 POINT(41.448992099338064 -74.06816021869058) bank75827 +75828 POINT(41.06404775204124 -73.03951931992349) bank75828 +75829 POINT(40.67322125124205 -74.9890123182941) bank75829 +75830 POINT(39.75487204211332 -74.90279068442871) bank75830 +75831 POINT(41.00160384801666 -74.48438470495788) bank75831 +75832 POINT(40.32870752941733 -73.98765479901655) bank75832 +75833 POINT(41.5405317311191 -73.10306792990382) bank75833 +75834 POINT(41.08485046491185 -73.54256357905552) bank75834 +75835 POINT(40.66854241706493 -73.99485040089462) bank75835 +75836 POINT(41.220530455613485 -74.52748521849993) bank75836 +75837 POINT(40.72356115418989 -74.21372413665082) bank75837 +75838 POINT(40.6329837472264 -73.12770800065867) bank75838 +75839 POINT(40.48400725354294 -74.90925824110431) bank75839 +75840 POINT(41.16827800484906 -74.26070996534276) bank75840 +75841 POINT(40.03180859944716 -73.66862340634503) bank75841 +75842 POINT(40.68920616288835 -73.83451684543788) bank75842 +75843 POINT(40.8264321748221 -74.7586314880563) bank75843 +75844 POINT(41.47060570449658 -73.01868810940827) bank75844 +75845 POINT(41.11043160770144 -74.43254709858424) bank75845 +75846 POINT(41.24669799416934 -73.69102498299412) bank75846 +75847 POINT(39.96801514283679 -73.62595660923881) bank75847 +75848 POINT(39.92812963528196 -73.74028289841935) bank75848 +75849 POINT(40.51924917929175 -73.17835138305585) bank75849 +75850 POINT(40.09073568980557 -73.28117781820045) bank75850 +75851 POINT(41.03940864945634 -74.75727106549368) bank75851 +75852 POINT(41.09669184804917 -73.03216131339771) bank75852 +75853 POINT(41.25203201860556 -73.74689562226158) bank75853 +75854 POINT(41.30597233900693 -73.48100688527202) bank75854 +75855 POINT(40.668247672470685 -73.6071764763475) bank75855 +75856 POINT(41.67399761675809 -73.94294479870267) bank75856 +75857 POINT(40.50843172118611 -74.70367245691865) bank75857 +75858 POINT(40.13594012408729 -74.81888006520585) bank75858 +75859 POINT(39.74904511655924 -73.60678833298554) bank75859 +75860 POINT(40.431621190427755 -73.07946390848785) bank75860 +75861 POINT(41.11300022551494 -74.63801469792968) bank75861 +75862 POINT(41.0213337596432 -74.96268549216528) bank75862 +75863 POINT(40.01813135911583 -73.08291064365764) bank75863 +75864 POINT(41.35802155323124 -73.21712732090583) bank75864 +75865 POINT(40.22503370880692 -74.41630876730365) bank75865 +75866 POINT(41.70216091177841 -74.40669547812675) bank75866 +75867 POINT(40.73328395888227 -74.51718918987417) bank75867 +75868 POINT(40.766195736442796 -74.83485169098854) bank75868 +75869 POINT(41.12802825266638 -73.74560793635054) bank75869 +75870 POINT(40.545687971542314 -74.56942203453069) bank75870 +75871 POINT(39.71979155017281 -74.917476717222) bank75871 +75872 POINT(41.12044734065706 -73.43301678166853) bank75872 +75873 POINT(41.51780177054432 -73.15243866783527) bank75873 +75874 POINT(41.55167033516565 -74.97264666506197) bank75874 +75875 POINT(40.503432931801285 -73.68043445061389) bank75875 +75876 POINT(41.56465370662302 -74.31332104450703) bank75876 +75877 POINT(41.71145913785148 -74.36835813740466) bank75877 +75878 POINT(40.841131326573915 -73.55398395174886) bank75878 +75879 POINT(41.61731943759945 -74.13350552523227) bank75879 +75880 POINT(40.94163834203405 -73.65201575299189) bank75880 +75881 POINT(41.6831421709671 -74.21800046008954) bank75881 +75882 POINT(40.23062593056007 -74.33950069499106) bank75882 +75883 POINT(40.2689478849562 -73.5119862764438) bank75883 +75884 POINT(40.01650313039524 -73.2348077789676) bank75884 +75885 POINT(40.61694970894526 -73.66549124547117) bank75885 +75886 POINT(40.130658016747745 -74.07304568293246) bank75886 +75887 POINT(39.880475882717924 -73.83948636447212) bank75887 +75888 POINT(41.685972862214605 -74.49365995106457) bank75888 +75889 POINT(40.94064430877172 -74.98722858381116) bank75889 +75890 POINT(40.397862448920606 -74.31978430805638) bank75890 +75891 POINT(39.935280869430095 -73.6035770438073) bank75891 +75892 POINT(40.84083945351332 -74.92163694394921) bank75892 +75893 POINT(41.615174272613714 -74.4089094797385) bank75893 +75894 POINT(40.306108991326084 -74.92470968472333) bank75894 +75895 POINT(40.9832965091977 -73.23033715695335) bank75895 +75896 POINT(41.29479899648611 -74.00383317689807) bank75896 +75897 POINT(41.701383129975035 -73.68265532208807) bank75897 +75898 POINT(41.35550647632795 -73.99452920697068) bank75898 +75899 POINT(40.85984088114145 -74.1311039986306) bank75899 +75900 POINT(41.3094100088925 -74.47373581092236) bank75900 +75901 POINT(41.40956802899542 -73.20231189485115) bank75901 +75902 POINT(40.47351499487603 -73.22030482632265) bank75902 +75903 POINT(40.6712198589395 -74.91527844372978) bank75903 +75904 POINT(40.52411652144299 -73.55653833636816) bank75904 +75905 POINT(40.40183638133144 -73.691591212291) bank75905 +75906 POINT(39.89922925780896 -73.092312726868) bank75906 +75907 POINT(40.739317280887946 -74.72814802354404) bank75907 +75908 POINT(40.173888739768124 -73.84880005740807) bank75908 +75909 POINT(41.08905009071647 -73.9659351643969) bank75909 +75910 POINT(40.46819629893404 -73.06283784734084) bank75910 +75911 POINT(39.738415573666224 -74.10147223004556) bank75911 +75912 POINT(40.14405001124093 -74.14890230102012) bank75912 +75913 POINT(41.15547359031716 -73.12818775081269) bank75913 +75914 POINT(40.10498848631468 -73.80978855172839) bank75914 +75915 POINT(41.61018515011017 -73.41806508375635) bank75915 +75916 POINT(39.92391907212618 -73.29262604812652) bank75916 +75917 POINT(41.179781543201116 -74.15166001661038) bank75917 +75918 POINT(41.69326946774678 -74.41842118378919) bank75918 +75919 POINT(40.242507341667554 -73.0559461194184) bank75919 +75920 POINT(40.88182318785202 -73.82512876574951) bank75920 +75921 POINT(41.247318479681546 -74.95478736524535) bank75921 +75922 POINT(39.84136045744164 -73.7244442394801) bank75922 +75923 POINT(40.60349751708059 -73.9055312899174) bank75923 +75924 POINT(40.29755379876331 -73.53846764623479) bank75924 +75925 POINT(40.53155798629273 -74.45839974121215) bank75925 +75926 POINT(40.105440806470355 -74.68052839715249) bank75926 +75927 POINT(40.64957459302682 -73.10617858617314) bank75927 +75928 POINT(39.721864777101345 -74.16166087544988) bank75928 +75929 POINT(39.778045941817645 -73.87843204573865) bank75929 +75930 POINT(41.32973753029781 -74.55491597501391) bank75930 +75931 POINT(41.67765250480452 -74.17754766245368) bank75931 +75932 POINT(39.93543821564165 -73.44484372989082) bank75932 +75933 POINT(40.080810746743914 -73.89073741230311) bank75933 +75934 POINT(41.355283501596595 -73.79783153008614) bank75934 +75935 POINT(39.90956570784958 -73.89603126649816) bank75935 +75936 POINT(40.79606563757211 -73.93510244290003) bank75936 +75937 POINT(41.6580401719153 -74.09340961852432) bank75937 +75938 POINT(40.376121282882636 -73.30373287773428) bank75938 +75939 POINT(40.65712957419825 -74.21092194848273) bank75939 +75940 POINT(41.383745833685325 -73.29463276381443) bank75940 +75941 POINT(40.187409957806075 -73.69555337480483) bank75941 +75942 POINT(40.97699055142578 -74.04529809131344) bank75942 +75943 POINT(40.20675816329334 -74.36661596300851) bank75943 +75944 POINT(40.22184638370355 -74.41601644649445) bank75944 +75945 POINT(41.04169855555469 -73.80600868776747) bank75945 +75946 POINT(40.360976692864156 -74.82606444260256) bank75946 +75947 POINT(41.19465900785779 -74.16244496437204) bank75947 +75948 POINT(40.877486809500816 -74.26910675298981) bank75948 +75949 POINT(40.09081118793584 -73.29288552758715) bank75949 +75950 POINT(41.46476082129742 -73.27557910123068) bank75950 +75951 POINT(41.432189003315585 -73.0800668268456) bank75951 +75952 POINT(39.93769421175934 -74.92930595074915) bank75952 +75953 POINT(41.22386798753345 -74.45713703365065) bank75953 +75954 POINT(39.71626950878008 -74.84095477216034) bank75954 +75955 POINT(41.678626944232526 -74.63726644549465) bank75955 +75956 POINT(40.4626952556558 -74.83769017525144) bank75956 +75957 POINT(39.9689930311073 -73.35935730260472) bank75957 +75958 POINT(40.65672322203438 -73.79695036870791) bank75958 +75959 POINT(39.77435898724086 -73.11101176618465) bank75959 +75960 POINT(39.78940348195404 -73.85703880801833) bank75960 +75961 POINT(40.96812476707891 -74.32441417180421) bank75961 +75962 POINT(40.77767536409327 -73.97788307725851) bank75962 +75963 POINT(40.17958807580818 -74.0655865941378) bank75963 +75964 POINT(40.5867142299351 -73.75873905746379) bank75964 +75965 POINT(40.45124480608319 -74.57113463550262) bank75965 +75966 POINT(41.44719012306209 -74.76597454090407) bank75966 +75967 POINT(40.39866945833767 -73.67703822343478) bank75967 +75968 POINT(41.1035363441334 -73.27898004216533) bank75968 +75969 POINT(41.64244989436939 -74.65418584910566) bank75969 +75970 POINT(40.83762534507519 -73.40794463859538) bank75970 +75971 POINT(40.948139576612 -74.55015810882425) bank75971 +75972 POINT(39.87401763038382 -73.27347511881464) bank75972 +75973 POINT(40.664133969721604 -73.07511245633485) bank75973 +75974 POINT(40.79064357753723 -74.58789311254041) bank75974 +75975 POINT(39.71532241991651 -73.00692041535233) bank75975 +75976 POINT(41.077829603827055 -74.86137142517029) bank75976 +75977 POINT(41.5204948768909 -73.11147030916854) bank75977 +75978 POINT(40.06284087178021 -73.1561281391539) bank75978 +75979 POINT(41.216713911012505 -74.63568177338682) bank75979 +75980 POINT(41.237374767873575 -74.96062104398057) bank75980 +75981 POINT(39.962620176034434 -73.37697155455776) bank75981 +75982 POINT(41.167934155351126 -74.02544522579069) bank75982 +75983 POINT(41.345571420276265 -73.84007520296011) bank75983 +75984 POINT(40.52594847775844 -74.00921452960225) bank75984 +75985 POINT(39.945599621928224 -73.38966296052673) bank75985 +75986 POINT(40.687285202619584 -73.36225773048938) bank75986 +75987 POINT(40.096745193645795 -73.65886642907383) bank75987 +75988 POINT(41.116238508303724 -74.10734636068786) bank75988 +75989 POINT(39.99335312702664 -73.25988904186308) bank75989 +75990 POINT(41.41322717463801 -73.11835525509704) bank75990 +75991 POINT(39.892733232351524 -74.17740764664887) bank75991 +75992 POINT(41.25593709914841 -73.59121084075875) bank75992 +75993 POINT(40.85565852907172 -73.98658476058313) bank75993 +75994 POINT(40.181878107408316 -73.26438554246418) bank75994 +75995 POINT(40.46346362332891 -73.48435353207422) bank75995 +75996 POINT(40.91746244109096 -74.81100035338261) bank75996 +75997 POINT(40.55500441161715 -74.10604732023617) bank75997 +75998 POINT(40.995634156424906 -73.28105233550089) bank75998 +75999 POINT(41.53663367356499 -73.93244232722004) bank75999 +76000 POINT(40.03619761787967 -74.83365836123185) bank76000 +76001 POINT(40.2184733510695 -73.89430461718463) bank76001 +76002 POINT(41.48987679846312 -73.43339701414205) bank76002 +76003 POINT(40.606555716031195 -74.65377736669907) bank76003 +76004 POINT(39.979704059005186 -73.79194978802495) bank76004 +76005 POINT(41.20431604794994 -73.38136204597052) bank76005 +76006 POINT(39.84665730295359 -73.50499315900942) bank76006 +76007 POINT(41.358512496917804 -74.1882924750692) bank76007 +76008 POINT(41.1256959283305 -74.48281489750012) bank76008 +76009 POINT(40.94148128735315 -73.28464790477064) bank76009 +76010 POINT(40.648854427531006 -73.46999645508791) bank76010 +76011 POINT(40.253908732517694 -73.40329736118248) bank76011 +76012 POINT(40.58433416883552 -73.18934871042308) bank76012 +76013 POINT(40.171318476989065 -73.44932667478923) bank76013 +76014 POINT(41.3077141560905 -73.12964168725868) bank76014 +76015 POINT(41.11544896694761 -73.6474277110461) bank76015 +76016 POINT(39.73570461948905 -73.88189678977426) bank76016 +76017 POINT(40.25348348211253 -73.06792534332399) bank76017 +76018 POINT(40.308920821188664 -74.13747540434588) bank76018 +76019 POINT(41.01417225288308 -73.65276150094417) bank76019 +76020 POINT(40.29142514530881 -74.81798451564053) bank76020 +76021 POINT(39.95436343729132 -73.6038143087644) bank76021 +76022 POINT(40.04533427810968 -73.01491994775076) bank76022 +76023 POINT(40.06014131662159 -74.28528802290914) bank76023 +76024 POINT(40.99816975492879 -73.19360933895744) bank76024 +76025 POINT(40.54730781930415 -73.23360195300357) bank76025 +76026 POINT(39.74214361103529 -74.51970891983912) bank76026 +76027 POINT(41.352549658741246 -74.88262563424593) bank76027 +76028 POINT(40.71896142144613 -73.41084790429156) bank76028 +76029 POINT(41.55895326570378 -73.10474027948253) bank76029 +76030 POINT(40.71410928893954 -73.2195498371184) bank76030 +76031 POINT(40.13856746145806 -74.53805509781513) bank76031 +76032 POINT(41.06385425325009 -73.1444943010053) bank76032 +76033 POINT(41.647538576934096 -73.6120740626516) bank76033 +76034 POINT(41.43889224324295 -74.09982228323777) bank76034 +76035 POINT(40.94988310304574 -74.67339304547994) bank76035 +76036 POINT(40.21294040981597 -74.57147808471117) bank76036 +76037 POINT(40.094996548143904 -73.3796844746836) bank76037 +76038 POINT(41.61705359800292 -74.18884005820227) bank76038 +76039 POINT(41.14271753187859 -73.10483024565869) bank76039 +76040 POINT(40.959978666460835 -74.55885402259979) bank76040 +76041 POINT(40.69662657177321 -74.89998973753887) bank76041 +76042 POINT(40.315376130691384 -74.78009881728617) bank76042 +76043 POINT(41.54246232585952 -74.26040512164245) bank76043 +76044 POINT(41.035845949205736 -73.20695477695185) bank76044 +76045 POINT(39.95562760853378 -73.07219528737123) bank76045 +76046 POINT(40.02912082970287 -73.05121438971487) bank76046 +76047 POINT(40.018587439857825 -74.42698336816771) bank76047 +76048 POINT(40.826928383311824 -73.8058944175804) bank76048 +76049 POINT(40.679651926824626 -74.98483597513557) bank76049 +76050 POINT(41.63520148926357 -73.38742744222644) bank76050 +76051 POINT(39.89880702623006 -73.86201149467941) bank76051 +76052 POINT(41.46061855628406 -74.69935259562942) bank76052 +76053 POINT(40.579270646778596 -74.35573512705128) bank76053 +76054 POINT(40.895805229110685 -73.90843205213076) bank76054 +76055 POINT(40.84502757280856 -73.76158361506081) bank76055 +76056 POINT(41.38563972786528 -74.81117656784228) bank76056 +76057 POINT(40.98023296631765 -73.08029337242914) bank76057 +76058 POINT(39.74557837863528 -73.80906017769915) bank76058 +76059 POINT(39.94595117557623 -74.93437340283208) bank76059 +76060 POINT(40.40272670844157 -73.47800832287643) bank76060 +76061 POINT(41.582545716791735 -74.37419122326) bank76061 +76062 POINT(39.90360505459929 -74.62335765002707) bank76062 +76063 POINT(40.209611679200606 -73.54885273541976) bank76063 +76064 POINT(41.45979993169913 -74.21853874426196) bank76064 +76065 POINT(39.92360441863236 -74.69032170998479) bank76065 +76066 POINT(39.78692826104043 -74.80803197576003) bank76066 +76067 POINT(39.982901777152875 -74.54489697781923) bank76067 +76068 POINT(40.76569674699305 -73.70544321536282) bank76068 +76069 POINT(41.27995657728183 -74.47513268889817) bank76069 +76070 POINT(40.23004434214349 -74.20858284704865) bank76070 +76071 POINT(39.71370963038623 -73.85312459373515) bank76071 +76072 POINT(39.9628711066124 -74.78971755039689) bank76072 +76073 POINT(40.71326328650206 -74.23993767128148) bank76073 +76074 POINT(40.020723966818416 -74.15968945369609) bank76074 +76075 POINT(40.61540442271193 -73.2396734667499) bank76075 +76076 POINT(39.94857730366502 -74.49854433918037) bank76076 +76077 POINT(40.43308737207105 -73.21501747622943) bank76077 +76078 POINT(41.706124612544826 -73.08198437274407) bank76078 +76079 POINT(40.78386209359309 -74.38380496033223) bank76079 +76080 POINT(40.36889193276006 -74.09460137355163) bank76080 +76081 POINT(40.00511607755478 -73.43333887151519) bank76081 +76082 POINT(41.58681298724098 -73.91404876722342) bank76082 +76083 POINT(40.146385053534495 -74.32856213578371) bank76083 +76084 POINT(39.84803772036951 -74.80437872689062) bank76084 +76085 POINT(41.08056049248724 -73.05826303740966) bank76085 +76086 POINT(40.598700251655295 -74.09280588493817) bank76086 +76087 POINT(40.11880747334445 -74.09189448410439) bank76087 +76088 POINT(40.216501342278555 -73.19199112117339) bank76088 +76089 POINT(41.04134272117128 -73.12715805372468) bank76089 +76090 POINT(40.09732299312755 -74.29675197989856) bank76090 +76091 POINT(40.126617599353615 -73.13438191148657) bank76091 +76092 POINT(40.44653179666672 -73.30241672968202) bank76092 +76093 POINT(41.34265112224879 -73.9037555498622) bank76093 +76094 POINT(39.76339668041131 -73.86646217180443) bank76094 +76095 POINT(40.194669930526615 -73.91248894183967) bank76095 +76096 POINT(39.837560141094094 -73.95472497667275) bank76096 +76097 POINT(41.70702273323407 -74.34800478336413) bank76097 +76098 POINT(40.6943193428628 -74.7485125131554) bank76098 +76099 POINT(40.93875790755004 -73.43072087281668) bank76099 +76100 POINT(40.619217399223935 -74.88200002172107) bank76100 +76101 POINT(41.68337476876751 -74.09463084496937) bank76101 +76102 POINT(40.21351685276759 -73.64701885471551) bank76102 +76103 POINT(40.610004582854046 -73.70162828604944) bank76103 +76104 POINT(41.563918085005795 -73.06741547141493) bank76104 +76105 POINT(41.370947139052475 -74.62609681652854) bank76105 +76106 POINT(40.709435319067076 -74.29798619939312) bank76106 +76107 POINT(39.8524177798859 -73.2366735003846) bank76107 +76108 POINT(41.219115891080826 -74.47147500003736) bank76108 +76109 POINT(41.41157352366671 -74.54317438139292) bank76109 +76110 POINT(41.386336666425166 -73.82676413145666) bank76110 +76111 POINT(40.2094711559922 -73.53915423011338) bank76111 +76112 POINT(40.36095105733953 -73.16090718814571) bank76112 +76113 POINT(41.187251781102205 -74.37411122169055) bank76113 +76114 POINT(41.16037064323726 -74.03563282331599) bank76114 +76115 POINT(41.09105164651038 -73.17080558570564) bank76115 +76116 POINT(41.165535014383096 -74.34231400458356) bank76116 +76117 POINT(40.54129879819825 -73.47225498517908) bank76117 +76118 POINT(40.93358374515903 -73.21675070152243) bank76118 +76119 POINT(41.26618528997038 -74.31169804851565) bank76119 +76120 POINT(39.736992548353264 -74.03864894444428) bank76120 +76121 POINT(40.092959194251634 -73.18436304444354) bank76121 +76122 POINT(40.65645709928856 -73.76476210558336) bank76122 +76123 POINT(41.54011357814348 -73.62456015285352) bank76123 +76124 POINT(40.863097161550975 -73.90945718203561) bank76124 +76125 POINT(39.749222733942894 -74.20123939330725) bank76125 +76126 POINT(41.22136394242646 -74.47485889612125) bank76126 +76127 POINT(41.594324035631985 -74.38472177300923) bank76127 +76128 POINT(40.94673683381423 -74.92358259605504) bank76128 +76129 POINT(41.05168534709427 -74.28118063657865) bank76129 +76130 POINT(41.13667852393327 -74.1077584993922) bank76130 +76131 POINT(40.193217307455676 -73.29692863240506) bank76131 +76132 POINT(41.66933254186292 -74.96628175562005) bank76132 +76133 POINT(41.391757793073246 -74.66183700670794) bank76133 +76134 POINT(40.36684674309059 -74.03522679395388) bank76134 +76135 POINT(41.098947480163524 -73.9650006421663) bank76135 +76136 POINT(41.63248229553976 -74.37074711792636) bank76136 +76137 POINT(40.487442369922704 -73.11707457585662) bank76137 +76138 POINT(40.0515876707747 -74.27111304071246) bank76138 +76139 POINT(41.69395198140018 -74.86683270287317) bank76139 +76140 POINT(40.8608365586968 -74.92182797692902) bank76140 +76141 POINT(40.69141525635415 -74.39603296403267) bank76141 +76142 POINT(40.31965509624814 -74.04162760106772) bank76142 +76143 POINT(40.27834670464218 -74.20103533134518) bank76143 +76144 POINT(39.80317379635249 -73.55979675059749) bank76144 +76145 POINT(40.895559057859494 -74.14574818810335) bank76145 +76146 POINT(40.85646109483348 -73.24335358299608) bank76146 +76147 POINT(41.19406142676756 -73.16333252174313) bank76147 +76148 POINT(39.86432064018458 -73.68097807477943) bank76148 +76149 POINT(41.395102270107614 -73.85120409164747) bank76149 +76150 POINT(40.160645937737236 -73.9089513948219) bank76150 +76151 POINT(40.16590774634238 -74.19415036990105) bank76151 +76152 POINT(40.43206490862112 -74.55461684950636) bank76152 +76153 POINT(40.56421665378591 -74.57485980776539) bank76153 +76154 POINT(40.45256400115606 -74.64535033658787) bank76154 +76155 POINT(41.446150426949686 -73.10617694258094) bank76155 +76156 POINT(40.316890296809895 -74.51274040631249) bank76156 +76157 POINT(41.39992483707315 -74.04635943663091) bank76157 +76158 POINT(39.72531977290244 -74.63670494977617) bank76158 +76159 POINT(40.16603514319285 -73.55207610558878) bank76159 +76160 POINT(40.104228244944125 -73.4267443977035) bank76160 +76161 POINT(39.861938165115376 -74.83619583950471) bank76161 +76162 POINT(41.27382568276122 -73.89272169898737) bank76162 +76163 POINT(39.925111813560335 -74.51585918786076) bank76163 +76164 POINT(40.8162565476228 -74.0275854878421) bank76164 +76165 POINT(40.84040884298333 -74.19640014168563) bank76165 +76166 POINT(40.26267403273912 -74.32116227620206) bank76166 +76167 POINT(41.014847353361766 -74.37267056761196) bank76167 +76168 POINT(41.484675550008454 -74.09897396130285) bank76168 +76169 POINT(41.191160434950035 -73.88135697524122) bank76169 +76170 POINT(39.8562940758929 -74.43954126428584) bank76170 +76171 POINT(41.08250752707171 -74.36187072669306) bank76171 +76172 POINT(41.70812033619883 -74.06431030294519) bank76172 +76173 POINT(40.510692069630956 -74.5964006355336) bank76173 +76174 POINT(40.97893411774619 -73.19127042991879) bank76174 +76175 POINT(39.81324741030346 -73.11781408000695) bank76175 +76176 POINT(40.80349493098189 -74.88492279094147) bank76176 +76177 POINT(41.40823062582641 -74.26085800756732) bank76177 +76178 POINT(40.6511248954696 -73.52894760828627) bank76178 +76179 POINT(41.07096399207974 -73.23912929902275) bank76179 +76180 POINT(41.577023716904314 -73.58588645818283) bank76180 +76181 POINT(40.448122011776235 -73.12838311095264) bank76181 +76182 POINT(41.628055271147495 -73.72347465478718) bank76182 +76183 POINT(40.41125779944851 -73.07028463044095) bank76183 +76184 POINT(40.865664692051304 -73.7641364922133) bank76184 +76185 POINT(41.18142491266197 -74.00116675087179) bank76185 +76186 POINT(40.110979588931826 -73.44152673240518) bank76186 +76187 POINT(39.929086492741725 -73.11938536200437) bank76187 +76188 POINT(41.472005246612326 -74.16281103650408) bank76188 +76189 POINT(41.652403875266714 -73.28703479902337) bank76189 +76190 POINT(40.08276887093668 -74.28134816931566) bank76190 +76191 POINT(41.66489554578936 -74.25128303639302) bank76191 +76192 POINT(40.32907217047746 -74.75211680509689) bank76192 +76193 POINT(41.52677080619512 -74.12748103992308) bank76193 +76194 POINT(39.912762329394596 -73.17244171190538) bank76194 +76195 POINT(41.15771862711325 -74.28626851093777) bank76195 +76196 POINT(40.73820755199785 -73.24675817360064) bank76196 +76197 POINT(41.605477344155716 -73.35276752091188) bank76197 +76198 POINT(41.33287502726511 -73.75590487144821) bank76198 +76199 POINT(40.00762522861998 -73.74146047558185) bank76199 +76200 POINT(40.79266153872236 -74.6807940755031) bank76200 +76201 POINT(40.100716503250226 -74.504273481261) bank76201 +76202 POINT(40.87883143994839 -73.01852855145387) bank76202 +76203 POINT(40.5921842354861 -74.96716017341805) bank76203 +76204 POINT(40.99588185118742 -74.3477292455193) bank76204 +76205 POINT(40.209827083046655 -73.25855606795764) bank76205 +76206 POINT(40.24564601553418 -73.78644761520732) bank76206 +76207 POINT(41.48485563874613 -73.85974454496049) bank76207 +76208 POINT(41.298238794678625 -73.28744879816591) bank76208 +76209 POINT(41.332227949847876 -73.6792017089474) bank76209 +76210 POINT(40.677780217109564 -74.5728924382247) bank76210 +76211 POINT(41.20578518732633 -73.64331523401628) bank76211 +76212 POINT(39.767085673402946 -74.78430828716688) bank76212 +76213 POINT(40.775800376234805 -73.26868583038264) bank76213 +76214 POINT(40.528421316881534 -74.88350792394289) bank76214 +76215 POINT(39.951992099079206 -73.99215921550996) bank76215 +76216 POINT(41.00153100327652 -74.28041260905277) bank76216 +76217 POINT(40.70082856035817 -73.70214068033845) bank76217 +76218 POINT(41.66940981058062 -74.5952752753657) bank76218 +76219 POINT(39.991853546294436 -74.50945552376461) bank76219 +76220 POINT(40.50739340888462 -74.25814118129492) bank76220 +76221 POINT(40.15130100583311 -73.33645032754237) bank76221 +76222 POINT(40.45434559846636 -73.65744439365696) bank76222 +76223 POINT(40.37841740093873 -74.91716730712213) bank76223 +76224 POINT(40.15093532588897 -73.7675616240133) bank76224 +76225 POINT(41.42636413932026 -73.51499035170468) bank76225 +76226 POINT(40.257253198037766 -73.78330601536409) bank76226 +76227 POINT(40.95970273081356 -73.16184541648772) bank76227 +76228 POINT(41.42958064243374 -73.50675461040271) bank76228 +76229 POINT(40.99809039927809 -73.70632765063561) bank76229 +76230 POINT(40.23897033195312 -74.04775144572353) bank76230 +76231 POINT(41.370075848480695 -73.73764195155172) bank76231 +76232 POINT(40.12420085069943 -73.15311738477551) bank76232 +76233 POINT(40.93348033657399 -74.86069472937577) bank76233 +76234 POINT(41.04810711921245 -74.01644946537982) bank76234 +76235 POINT(39.857058263515306 -74.27319641101049) bank76235 +76236 POINT(40.74435957841532 -74.36326907309642) bank76236 +76237 POINT(40.06842444580948 -74.23239243971807) bank76237 +76238 POINT(40.574162854874935 -73.09818694745077) bank76238 +76239 POINT(41.26242436813948 -74.68558087597246) bank76239 +76240 POINT(41.14164440463345 -74.25654120927308) bank76240 +76241 POINT(41.014183751003266 -73.69075916373515) bank76241 +76242 POINT(41.01281178165946 -74.28259181599664) bank76242 +76243 POINT(41.24214889335203 -74.31080096296778) bank76243 +76244 POINT(40.57023683193467 -73.22324371411494) bank76244 +76245 POINT(41.000663604450544 -73.36835651753736) bank76245 +76246 POINT(41.26745215155601 -73.75823765393112) bank76246 +76247 POINT(40.524857831916854 -74.01549792550371) bank76247 +76248 POINT(40.96751492997807 -74.67389797756873) bank76248 +76249 POINT(41.13800916875843 -74.53527424558358) bank76249 +76250 POINT(41.140545953979576 -74.47387162189949) bank76250 +76251 POINT(41.47233008477576 -74.78958888091636) bank76251 +76252 POINT(40.63693537702503 -74.82841027736414) bank76252 +76253 POINT(41.63584679187064 -74.7274342520143) bank76253 +76254 POINT(41.255832265358066 -73.77394022282725) bank76254 +76255 POINT(39.957458107704745 -74.92358960686869) bank76255 +76256 POINT(40.39852818634109 -74.31363761535698) bank76256 +76257 POINT(40.69995362961136 -74.32980600080548) bank76257 +76258 POINT(41.36524821621029 -74.07768919430974) bank76258 +76259 POINT(39.756641140911846 -74.87762660640955) bank76259 +76260 POINT(40.16322277886685 -73.84825917411152) bank76260 +76261 POINT(39.92629997971797 -73.17469675473896) bank76261 +76262 POINT(39.810187275592384 -74.02547836455379) bank76262 +76263 POINT(41.53561149895057 -73.30703597355098) bank76263 +76264 POINT(41.073389731304445 -73.511333305074) bank76264 +76265 POINT(41.66462277421275 -73.95031976418568) bank76265 +76266 POINT(40.33045804389693 -73.32631757674238) bank76266 +76267 POINT(40.91141773293836 -74.13730543091776) bank76267 +76268 POINT(41.1699973640341 -74.67297699355292) bank76268 +76269 POINT(41.10746624334297 -73.3343782423688) bank76269 +76270 POINT(41.652277348679426 -73.8296180053789) bank76270 +76271 POINT(41.006739744355144 -74.6843397564074) bank76271 +76272 POINT(39.87682736126982 -73.16150435730475) bank76272 +76273 POINT(40.09375434212318 -74.87850812276407) bank76273 +76274 POINT(41.30954170345657 -74.15171935957267) bank76274 +76275 POINT(39.82382097433174 -73.7375010945142) bank76275 +76276 POINT(40.57273364261226 -73.71115655287399) bank76276 +76277 POINT(41.50365970667946 -73.62988588750383) bank76277 +76278 POINT(40.714138197649554 -74.84524886494967) bank76278 +76279 POINT(41.53219161910606 -73.90930207241615) bank76279 +76280 POINT(40.58575267451059 -73.00661016677256) bank76280 +76281 POINT(40.173668773584 -73.59649008128683) bank76281 +76282 POINT(39.75668199570341 -74.76716256070623) bank76282 +76283 POINT(40.05558853308202 -73.24861940047332) bank76283 +76284 POINT(41.391131604608645 -73.87265134106642) bank76284 +76285 POINT(41.18033450778735 -73.53684395865129) bank76285 +76286 POINT(39.864555945568526 -73.75841723328124) bank76286 +76287 POINT(41.06858262418696 -74.2286372770136) bank76287 +76288 POINT(40.33534018027884 -74.5767367349934) bank76288 +76289 POINT(40.02604769822871 -74.08414554408652) bank76289 +76290 POINT(40.80571675221753 -73.30729970286352) bank76290 +76291 POINT(40.38220095737163 -74.71282262709408) bank76291 +76292 POINT(39.72367278642856 -74.336683315731) bank76292 +76293 POINT(41.57207681552316 -74.58804900897053) bank76293 +76294 POINT(40.481858290910445 -73.87024195804013) bank76294 +76295 POINT(40.018189055397535 -74.66675143601677) bank76295 +76296 POINT(40.821463251221445 -73.24330731182216) bank76296 +76297 POINT(40.24142496470778 -74.74798727333422) bank76297 +76298 POINT(40.39173369340125 -73.42745105381094) bank76298 +76299 POINT(40.80742850976929 -73.2370374673466) bank76299 +76300 POINT(40.39761480792453 -73.20125422610501) bank76300 +76301 POINT(40.00281675860349 -74.80492796924153) bank76301 +76302 POINT(39.76877796257298 -74.9199371930212) bank76302 +76303 POINT(40.04664348529933 -73.12012247204967) bank76303 +76304 POINT(41.297536213840786 -74.81598578397002) bank76304 +76305 POINT(41.29892019659441 -73.47332972111201) bank76305 +76306 POINT(40.726207790161844 -74.09496137045087) bank76306 +76307 POINT(40.36676399208905 -74.47860858592358) bank76307 +76308 POINT(40.46684562520023 -73.04980518695601) bank76308 +76309 POINT(41.457738400177064 -73.46035535644576) bank76309 +76310 POINT(39.98179510226879 -74.24355201933437) bank76310 +76311 POINT(40.40548740260343 -73.1383846240125) bank76311 +76312 POINT(41.02266814486918 -74.39619948536283) bank76312 +76313 POINT(40.2533897988297 -74.69459814759314) bank76313 +76314 POINT(41.126859620647565 -73.72983250896084) bank76314 +76315 POINT(41.09475411796106 -73.80728012449254) bank76315 +76316 POINT(40.27121751631083 -73.54469826302311) bank76316 +76317 POINT(39.73498620602827 -74.75024258578769) bank76317 +76318 POINT(40.0623134158341 -74.82297381200621) bank76318 +76319 POINT(41.16548366224087 -73.40248897983415) bank76319 +76320 POINT(41.6830669344774 -74.29699141725109) bank76320 +76321 POINT(40.4022540142488 -73.23140277839805) bank76321 +76322 POINT(40.41679238352074 -74.90537485226223) bank76322 +76323 POINT(41.09016678352294 -74.72063163373858) bank76323 +76324 POINT(41.24629232027562 -74.94974168843852) bank76324 +76325 POINT(41.330147256101014 -73.28590280760014) bank76325 +76326 POINT(40.83880425955618 -73.95450534963129) bank76326 +76327 POINT(39.866595975073174 -74.87916956660875) bank76327 +76328 POINT(40.144035805669354 -74.9928969621074) bank76328 +76329 POINT(40.699169475864416 -74.30649429710705) bank76329 +76330 POINT(41.63188285605317 -74.2494090099033) bank76330 +76331 POINT(40.6661311441884 -74.21855886175435) bank76331 +76332 POINT(40.92962425502823 -73.77552260059662) bank76332 +76333 POINT(40.96268936346534 -74.18896892847779) bank76333 +76334 POINT(40.86497170905419 -74.45220745817863) bank76334 +76335 POINT(40.2475807422323 -73.18395900298839) bank76335 +76336 POINT(39.9161923968971 -74.64600938403228) bank76336 +76337 POINT(40.87751220640069 -73.53141976530576) bank76337 +76338 POINT(40.134169762177926 -73.44540457183074) bank76338 +76339 POINT(40.23826639603472 -74.34024258894648) bank76339 +76340 POINT(41.57041616912421 -73.10015605698754) bank76340 +76341 POINT(41.17937161769882 -73.22278559004884) bank76341 +76342 POINT(41.48834701173403 -73.83436842889579) bank76342 +76343 POINT(40.030754009751185 -73.94073109517501) bank76343 +76344 POINT(40.019863439018266 -74.58349118808304) bank76344 +76345 POINT(40.05061328927913 -74.35746559623583) bank76345 +76346 POINT(40.131167001262504 -73.82013541001672) bank76346 +76347 POINT(40.74155037946767 -74.65708948718901) bank76347 +76348 POINT(41.240507763338805 -73.98235280025604) bank76348 +76349 POINT(41.40171743486849 -73.79857766606648) bank76349 +76350 POINT(40.10305549101259 -74.14194253943678) bank76350 +76351 POINT(40.60308562633769 -74.95554622735781) bank76351 +76352 POINT(39.9863303794655 -73.74505842956498) bank76352 +76353 POINT(40.62453642138276 -73.72863740067088) bank76353 +76354 POINT(41.37474750714425 -74.90393917665752) bank76354 +76355 POINT(40.13273821640245 -74.36553362550436) bank76355 +76356 POINT(41.21358849556578 -73.57116883752552) bank76356 +76357 POINT(41.70516304151345 -73.0656593315187) bank76357 +76358 POINT(40.34305325588325 -74.31661421701978) bank76358 +76359 POINT(40.87294276151303 -74.79377294579068) bank76359 +76360 POINT(39.72551215503798 -73.50189193590377) bank76360 +76361 POINT(41.18958652859554 -74.83478863172039) bank76361 +76362 POINT(40.5374102603187 -73.1042726855404) bank76362 +76363 POINT(41.53348141865884 -74.69080729099748) bank76363 +76364 POINT(41.33089282405212 -73.23624853747769) bank76364 +76365 POINT(40.92235843280583 -74.55683720394667) bank76365 +76366 POINT(40.96362282730102 -73.67637024085008) bank76366 +76367 POINT(39.75597615468343 -74.10818671650625) bank76367 +76368 POINT(40.52361055003523 -73.05771084926444) bank76368 +76369 POINT(40.322709997477226 -73.85917881036063) bank76369 +76370 POINT(39.83790720332368 -75.00179661758239) bank76370 +76371 POINT(41.20589256170583 -73.92247994573138) bank76371 +76372 POINT(40.36868373326065 -73.26392506548947) bank76372 +76373 POINT(40.967069689181244 -73.2683638745197) bank76373 +76374 POINT(40.80621774918166 -74.86562662834363) bank76374 +76375 POINT(40.410903200848814 -74.30315403905412) bank76375 +76376 POINT(39.750092770588836 -73.89386730810044) bank76376 +76377 POINT(41.59637385944146 -73.37329857143662) bank76377 +76378 POINT(41.27915950063039 -74.02624567384825) bank76378 +76379 POINT(41.614840321786716 -74.63379450670212) bank76379 +76380 POINT(40.885061451934256 -73.93954414272693) bank76380 +76381 POINT(40.09704540269674 -73.49923378168903) bank76381 +76382 POINT(40.0012159699081 -74.11215603488003) bank76382 +76383 POINT(40.55950277229243 -74.37499671398541) bank76383 +76384 POINT(40.47804363822426 -73.46708142296823) bank76384 +76385 POINT(40.352895311563145 -73.7294914678844) bank76385 +76386 POINT(39.86389679090488 -74.65428043270735) bank76386 +76387 POINT(39.922359061296405 -74.8293793243489) bank76387 +76388 POINT(41.127205518893874 -74.84473050957283) bank76388 +76389 POINT(40.02348290850429 -74.0106508884935) bank76389 +76390 POINT(41.244208333684035 -73.89325995117332) bank76390 +76391 POINT(41.1204717915288 -74.33307912095947) bank76391 +76392 POINT(41.50817796206011 -74.05219531553048) bank76392 +76393 POINT(39.91926151281129 -74.97607499112328) bank76393 +76394 POINT(40.22633121341174 -74.71409672902566) bank76394 +76395 POINT(39.91587856718747 -73.07580849334254) bank76395 +76396 POINT(41.39276828232732 -73.80722674706011) bank76396 +76397 POINT(40.57287503682728 -73.24587493011741) bank76397 +76398 POINT(39.72757944606746 -73.97907105027944) bank76398 +76399 POINT(41.51862524420359 -73.9552034290112) bank76399 +76400 POINT(41.06533928521481 -74.05127738842239) bank76400 +76401 POINT(40.5646584123145 -74.65451387396519) bank76401 +76402 POINT(40.65741520725068 -73.45330038489425) bank76402 +76403 POINT(39.95758460575299 -74.51527970610195) bank76403 +76404 POINT(41.410114641649585 -73.52851603458615) bank76404 +76405 POINT(41.524328941177025 -74.99396304805919) bank76405 +76406 POINT(40.09290939201354 -73.14642163912475) bank76406 +76407 POINT(39.93546303098242 -74.21919501465773) bank76407 +76408 POINT(40.24614030919343 -74.38520909319038) bank76408 +76409 POINT(41.63346254904545 -74.48522182562895) bank76409 +76410 POINT(40.09508221137096 -73.07984887767063) bank76410 +76411 POINT(41.00989043414856 -73.29076926387472) bank76411 +76412 POINT(40.46452594550633 -73.71249256918635) bank76412 +76413 POINT(41.27237007216829 -74.87389581043206) bank76413 +76414 POINT(40.90441883838737 -74.44629268667371) bank76414 +76415 POINT(40.34166885184993 -73.37550723223599) bank76415 +76416 POINT(41.2344469591533 -73.94079747951551) bank76416 +76417 POINT(40.83874182314166 -74.21694704497318) bank76417 +76418 POINT(40.808422491419876 -74.18651411126442) bank76418 +76419 POINT(40.70329284152997 -74.49176354123905) bank76419 +76420 POINT(40.130446546203686 -73.7242174725788) bank76420 +76421 POINT(40.700378241341795 -74.72176331958948) bank76421 +76422 POINT(41.313648853547676 -74.400440876102) bank76422 +76423 POINT(41.0535240737589 -73.78962416793767) bank76423 +76424 POINT(41.32396041593982 -73.99503059537464) bank76424 +76425 POINT(40.41016654835525 -73.70720325838268) bank76425 +76426 POINT(41.32778869545768 -74.55625813887602) bank76426 +76427 POINT(41.12980344315748 -73.15652170023574) bank76427 +76428 POINT(40.34230589237347 -73.79075572606098) bank76428 +76429 POINT(40.19949275872132 -74.82747688385356) bank76429 +76430 POINT(40.16654669849332 -73.39620220860199) bank76430 +76431 POINT(40.34946763876294 -74.59163562024706) bank76431 +76432 POINT(41.566506433579654 -73.21614323880381) bank76432 +76433 POINT(41.06475186133695 -73.04399900591261) bank76433 +76434 POINT(40.11005388839535 -74.12603172283788) bank76434 +76435 POINT(41.678940816117326 -74.52883885886601) bank76435 +76436 POINT(39.914075538672535 -74.54227763921898) bank76436 +76437 POINT(40.62283078577699 -73.50204420183928) bank76437 +76438 POINT(40.78629731035509 -74.55233519105211) bank76438 +76439 POINT(40.247522774150944 -73.18703533856896) bank76439 +76440 POINT(40.24008936700652 -73.90145092347387) bank76440 +76441 POINT(40.403431268948374 -74.18279501676655) bank76441 +76442 POINT(40.684729441108914 -73.87784513973588) bank76442 +76443 POINT(40.860251059617156 -73.67864557352851) bank76443 +76444 POINT(40.92805866201401 -73.19392068746087) bank76444 +76445 POINT(39.82734755746896 -74.69590593884176) bank76445 +76446 POINT(39.98245933080013 -74.03483298895966) bank76446 +76447 POINT(39.938195018599416 -74.11126481177818) bank76447 +76448 POINT(39.96390156414446 -73.30131572799513) bank76448 +76449 POINT(39.97540521485004 -73.49110072041536) bank76449 +76450 POINT(39.95772036553916 -73.56589652254176) bank76450 +76451 POINT(40.97187379383183 -73.22339337281113) bank76451 +76452 POINT(40.34383120605389 -74.53511632490789) bank76452 +76453 POINT(39.993977445326294 -74.98265782080935) bank76453 +76454 POINT(39.74615283981118 -73.57812408884969) bank76454 +76455 POINT(39.87151373944827 -74.01107558384678) bank76455 +76456 POINT(40.15906336947596 -73.18615317032176) bank76456 +76457 POINT(39.88828301845814 -74.64572994117769) bank76457 +76458 POINT(39.7471865019986 -74.78118418576084) bank76458 +76459 POINT(41.21729193438101 -73.74622947841065) bank76459 +76460 POINT(40.31862220704238 -74.88537115651464) bank76460 +76461 POINT(41.70623599835761 -74.95823943321037) bank76461 +76462 POINT(40.33883458390066 -73.02367044444581) bank76462 +76463 POINT(40.89849096449849 -73.17188053398637) bank76463 +76464 POINT(40.11888488403313 -74.82626205956547) bank76464 +76465 POINT(41.66065528337461 -74.1306057781071) bank76465 +76466 POINT(40.86993438001935 -74.06537098181339) bank76466 +76467 POINT(41.70640269514725 -73.9700400650616) bank76467 +76468 POINT(40.12157119469898 -73.75495187711385) bank76468 +76469 POINT(39.92262247821625 -73.80848717965378) bank76469 +76470 POINT(40.5180958806424 -73.42221694857274) bank76470 +76471 POINT(39.96925380792502 -74.33225704403978) bank76471 +76472 POINT(40.50982116814938 -73.57945650786095) bank76472 +76473 POINT(41.13165288786059 -73.32044359469225) bank76473 +76474 POINT(40.85171998720858 -74.5128143049373) bank76474 +76475 POINT(40.79974663516324 -74.45642805399592) bank76475 +76476 POINT(41.52664879163978 -74.86848945271316) bank76476 +76477 POINT(40.5450706824541 -73.32609954983259) bank76477 +76478 POINT(40.74814195030303 -73.15205584813658) bank76478 +76479 POINT(41.46057736431231 -74.32320740244727) bank76479 +76480 POINT(41.652429120966396 -74.0812817515591) bank76480 +76481 POINT(40.26869348313432 -74.94797292695759) bank76481 +76482 POINT(40.38065765519421 -73.04299425441204) bank76482 +76483 POINT(40.557087752233834 -74.91608834741723) bank76483 +76484 POINT(40.39130172842261 -74.69573937963929) bank76484 +76485 POINT(40.48393386250084 -73.64763288306496) bank76485 +76486 POINT(40.25628449336806 -73.33966662392713) bank76486 +76487 POINT(41.531892128006035 -74.73148588298548) bank76487 +76488 POINT(40.383555217269624 -74.11113018697985) bank76488 +76489 POINT(40.87200499524867 -74.71502744933115) bank76489 +76490 POINT(41.35127528258266 -73.9656725453674) bank76490 +76491 POINT(40.5300770929482 -74.17470410110518) bank76491 +76492 POINT(40.460451377078215 -73.01558014453893) bank76492 +76493 POINT(40.60741755728779 -74.48766499000413) bank76493 +76494 POINT(40.176564349724785 -74.7390576544993) bank76494 +76495 POINT(41.655086040872526 -74.02304063374316) bank76495 +76496 POINT(41.64447252204311 -74.15198355457439) bank76496 +76497 POINT(41.15904565177414 -73.3476135151864) bank76497 +76498 POINT(41.695425951355666 -74.5551795625344) bank76498 +76499 POINT(40.61474456386568 -74.41635335301054) bank76499 +76500 POINT(40.799461002657004 -74.66994848197804) bank76500 +76501 POINT(39.74971980233531 -73.1263384570107) bank76501 +76502 POINT(40.599661260277834 -74.72479068393316) bank76502 +76503 POINT(40.35460748864321 -73.12370058039244) bank76503 +76504 POINT(39.71583506352097 -74.06853965828137) bank76504 +76505 POINT(41.342664803850695 -73.93189891033487) bank76505 +76506 POINT(39.97907711880987 -73.57604693873262) bank76506 +76507 POINT(41.62267392998876 -74.85086215886007) bank76507 +76508 POINT(41.61746466993117 -74.29798513110515) bank76508 +76509 POINT(41.27957594762109 -74.97823008285727) bank76509 +76510 POINT(41.1040714678674 -74.79660878227286) bank76510 +76511 POINT(40.86804518114546 -73.3269914062382) bank76511 +76512 POINT(41.06799304390151 -74.13162582965136) bank76512 +76513 POINT(40.579523778064924 -73.83128839651523) bank76513 +76514 POINT(40.02105923842399 -74.32908117183275) bank76514 +76515 POINT(41.38700080936032 -74.91330457945124) bank76515 +76516 POINT(39.75158618300217 -73.29580089811175) bank76516 +76517 POINT(39.96093691448326 -73.75780730865085) bank76517 +76518 POINT(41.46152547698234 -73.94651322268975) bank76518 +76519 POINT(40.764565517276644 -73.09453358415402) bank76519 +76520 POINT(39.84716248669824 -74.21210360298322) bank76520 +76521 POINT(40.36846059995183 -73.66871596342291) bank76521 +76522 POINT(41.63840381148513 -74.39384066482351) bank76522 +76523 POINT(40.684787766178694 -74.19261751856413) bank76523 +76524 POINT(40.174408139070444 -73.92921448559468) bank76524 +76525 POINT(39.72031094464396 -73.38895240316347) bank76525 +76526 POINT(40.166968975052434 -73.68756083266595) bank76526 +76527 POINT(40.081497284023456 -74.09165951027707) bank76527 +76528 POINT(41.15404664309974 -74.15647043113127) bank76528 +76529 POINT(40.4476763895861 -73.07743064479746) bank76529 +76530 POINT(40.959019868864445 -74.13060568920194) bank76530 +76531 POINT(40.4747038625491 -74.21725335564452) bank76531 +76532 POINT(40.97622620532081 -73.88521478377085) bank76532 +76533 POINT(40.453201499161196 -73.56326910846337) bank76533 +76534 POINT(41.36520365937442 -74.56024414077994) bank76534 +76535 POINT(41.183092808707435 -73.80226721695111) bank76535 +76536 POINT(40.2672573727974 -73.31298075058186) bank76536 +76537 POINT(41.27004846036233 -74.65554208792021) bank76537 +76538 POINT(41.054410499458996 -73.46307391654491) bank76538 +76539 POINT(41.125004938727145 -74.04126044087391) bank76539 +76540 POINT(40.83101852499521 -74.33325884795963) bank76540 +76541 POINT(41.67682077677554 -74.75747912129995) bank76541 +76542 POINT(40.934410369766155 -74.72095721520493) bank76542 +76543 POINT(41.53794438950846 -74.30943685989662) bank76543 +76544 POINT(40.51479353743885 -74.6482635231251) bank76544 +76545 POINT(40.52153570716558 -73.34440085306252) bank76545 +76546 POINT(40.94798315903108 -73.58828827630354) bank76546 +76547 POINT(39.80182240313353 -73.92167948740573) bank76547 +76548 POINT(40.94688339404969 -74.38690044760537) bank76548 +76549 POINT(41.34541486311697 -74.78436138711375) bank76549 +76550 POINT(40.86277099825902 -74.19687325045423) bank76550 +76551 POINT(40.119994497839485 -74.31786034163723) bank76551 +76552 POINT(40.528898520229134 -74.65651319663694) bank76552 +76553 POINT(40.420923918783735 -73.49313767025913) bank76553 +76554 POINT(40.08868876455444 -73.93763035161516) bank76554 +76555 POINT(40.006844772881244 -74.00294750789283) bank76555 +76556 POINT(41.23109325445552 -73.60459299522029) bank76556 +76557 POINT(41.7053438095215 -73.14854303198558) bank76557 +76558 POINT(40.79429994598404 -74.68776900100357) bank76558 +76559 POINT(41.54465219428924 -74.46998184019652) bank76559 +76560 POINT(40.277234554562696 -73.62223099247346) bank76560 +76561 POINT(40.777474354519214 -74.44745598297519) bank76561 +76562 POINT(39.782761561988615 -74.72246808290573) bank76562 +76563 POINT(39.93798194040025 -74.06051188436564) bank76563 +76564 POINT(41.46501333416956 -74.31727201068811) bank76564 +76565 POINT(40.48786588426314 -73.93486742874407) bank76565 +76566 POINT(40.83317784875463 -73.59682578506884) bank76566 +76567 POINT(41.51424018873969 -74.42990275096606) bank76567 +76568 POINT(40.49612736395454 -74.67503215787424) bank76568 +76569 POINT(40.56902418789257 -73.88631691130826) bank76569 +76570 POINT(40.89528341523145 -74.30166635560185) bank76570 +76571 POINT(39.89485970529957 -73.2181604816396) bank76571 +76572 POINT(40.40268739871364 -73.08262042565272) bank76572 +76573 POINT(40.04330808796905 -73.10178230328395) bank76573 +76574 POINT(40.662504454978 -74.0898260106774) bank76574 +76575 POINT(39.959256820198576 -74.14522757796395) bank76575 +76576 POINT(40.112850227090036 -73.85374066537142) bank76576 +76577 POINT(40.50643849139527 -73.33148070879423) bank76577 +76578 POINT(41.210548695755904 -74.82239257001858) bank76578 +76579 POINT(40.874515316100705 -73.32238640197511) bank76579 +76580 POINT(40.49609835231892 -73.09651467543475) bank76580 +76581 POINT(39.814362901018754 -74.69920687824965) bank76581 +76582 POINT(39.805437284581366 -73.63368835294408) bank76582 +76583 POINT(41.59172982307886 -74.86178710666795) bank76583 +76584 POINT(40.873923721516384 -73.28182256518268) bank76584 +76585 POINT(41.23999669986985 -74.65765560518544) bank76585 +76586 POINT(40.43697692935893 -73.09988669778348) bank76586 +76587 POINT(40.194326917453395 -74.37915315950752) bank76587 +76588 POINT(39.871420650036924 -74.67425062001162) bank76588 +76589 POINT(40.02163672964249 -73.47409783093333) bank76589 +76590 POINT(40.5284633121212 -74.90940760858463) bank76590 +76591 POINT(41.035898343549995 -73.73399893505055) bank76591 +76592 POINT(41.59870495664149 -74.47893166631029) bank76592 +76593 POINT(41.031446683904626 -73.52602785715969) bank76593 +76594 POINT(39.94437261446062 -73.86838335673373) bank76594 +76595 POINT(40.65053107540686 -74.0884686985346) bank76595 +76596 POINT(41.04335626252953 -73.40799379517634) bank76596 +76597 POINT(41.58455160919644 -73.90601090390255) bank76597 +76598 POINT(41.281588827111364 -73.79258561692399) bank76598 +76599 POINT(40.14014346644245 -73.81845124905391) bank76599 +76600 POINT(40.83332080576361 -73.09530499306067) bank76600 +76601 POINT(41.557950831759335 -73.42874940906064) bank76601 +76602 POINT(40.97182971980667 -73.64515414529906) bank76602 +76603 POINT(41.522477920558686 -74.71807041381469) bank76603 +76604 POINT(40.19991467939741 -74.40227102737771) bank76604 +76605 POINT(40.28780352384756 -73.28584444693861) bank76605 +76606 POINT(40.59457633692216 -74.00210024510717) bank76606 +76607 POINT(40.01582488649911 -74.88089106402734) bank76607 +76608 POINT(39.99856258243756 -73.23927681961551) bank76608 +76609 POINT(41.19620763115301 -74.39500917317085) bank76609 +76610 POINT(40.77145476548474 -74.18111387640401) bank76610 +76611 POINT(39.76256589557626 -74.15924884563367) bank76611 +76612 POINT(40.47222558166844 -74.71667163077211) bank76612 +76613 POINT(41.326054450728535 -74.9474055196845) bank76613 +76614 POINT(39.87089521916539 -73.39503171889628) bank76614 +76615 POINT(41.626005414984185 -73.64961947048106) bank76615 +76616 POINT(40.93202527611064 -73.8356891190134) bank76616 +76617 POINT(40.94889401788068 -74.79928402271337) bank76617 +76618 POINT(41.25506592101441 -73.73756198524781) bank76618 +76619 POINT(40.111328194347045 -74.63988848042645) bank76619 +76620 POINT(40.308541020910305 -74.08921070030867) bank76620 +76621 POINT(41.52854430473589 -73.45916816446113) bank76621 +76622 POINT(41.04451712032854 -74.16999903390845) bank76622 +76623 POINT(40.58101531073015 -74.69901109934058) bank76623 +76624 POINT(41.09231231658994 -73.4117919106339) bank76624 +76625 POINT(39.77404060250896 -74.19478391146272) bank76625 +76626 POINT(39.81023238324636 -73.04496086683724) bank76626 +76627 POINT(39.762357090268296 -74.06527787017835) bank76627 +76628 POINT(40.34138477419899 -73.548357574137) bank76628 +76629 POINT(41.104011678493016 -74.10530244581649) bank76629 +76630 POINT(39.77297725634997 -73.45185621658212) bank76630 +76631 POINT(41.676094797646584 -74.09883043653701) bank76631 +76632 POINT(40.42182491042644 -74.19002389682544) bank76632 +76633 POINT(40.7594841585142 -74.8517564815926) bank76633 +76634 POINT(40.733158011478025 -74.30369797187333) bank76634 +76635 POINT(40.9409688744662 -73.72614117336133) bank76635 +76636 POINT(39.73084048356812 -73.8274498933543) bank76636 +76637 POINT(40.75029110493871 -73.37570533115469) bank76637 +76638 POINT(39.76482973589332 -74.06891385329061) bank76638 +76639 POINT(41.32891194834368 -73.23627118054789) bank76639 +76640 POINT(41.439418294765424 -73.43273265577622) bank76640 +76641 POINT(40.51294818455526 -73.90205243276905) bank76641 +76642 POINT(41.62668528076132 -73.61166851230674) bank76642 +76643 POINT(40.27478903566322 -74.00239735878819) bank76643 +76644 POINT(41.46982225002535 -73.5558769136992) bank76644 +76645 POINT(39.736532408819386 -73.13677076802412) bank76645 +76646 POINT(39.9935204292969 -74.22405230574644) bank76646 +76647 POINT(40.391723551690234 -73.52289857169039) bank76647 +76648 POINT(40.38781073017544 -74.09838604895326) bank76648 +76649 POINT(41.08835691493686 -73.21167059370653) bank76649 +76650 POINT(39.79091811340634 -74.34609508052068) bank76650 +76651 POINT(39.991239865224955 -73.02144223823454) bank76651 +76652 POINT(40.6511040219169 -73.33908243449727) bank76652 +76653 POINT(40.354097141538176 -73.38497299133263) bank76653 +76654 POINT(40.727418490279824 -73.37287849903207) bank76654 +76655 POINT(41.02577458120662 -74.19863769364913) bank76655 +76656 POINT(41.09078522289259 -74.44750763431568) bank76656 +76657 POINT(41.6443970063397 -74.73486801518146) bank76657 +76658 POINT(41.235816274770514 -73.98130985605131) bank76658 +76659 POINT(40.44066573681214 -74.46201122938466) bank76659 +76660 POINT(41.65735393885062 -73.94381454675165) bank76660 +76661 POINT(41.147347416068925 -74.00655846296121) bank76661 +76662 POINT(41.30007555658598 -73.35250474472471) bank76662 +76663 POINT(41.40680232901967 -73.28365706239103) bank76663 +76664 POINT(41.179373939101616 -73.17036281699855) bank76664 +76665 POINT(41.182568554283925 -73.132138402823) bank76665 +76666 POINT(41.02351001344286 -73.46467557352302) bank76666 +76667 POINT(40.048255836557175 -73.14844472399488) bank76667 +76668 POINT(41.15201393282573 -74.0466433774901) bank76668 +76669 POINT(41.425020724565634 -73.80151161515258) bank76669 +76670 POINT(41.52178148411075 -74.70534869687633) bank76670 +76671 POINT(41.42118086780353 -74.80269837445675) bank76671 +76672 POINT(41.102604569703004 -73.20956309709955) bank76672 +76673 POINT(40.26019775087459 -73.40803429097375) bank76673 +76674 POINT(40.831648944959106 -74.22153773394147) bank76674 +76675 POINT(40.76570148609403 -73.12903881525064) bank76675 +76676 POINT(40.594293692982454 -74.74396211410966) bank76676 +76677 POINT(41.33467777842499 -73.54767948696396) bank76677 +76678 POINT(40.04638636277633 -74.83538721910686) bank76678 +76679 POINT(41.39147694637145 -73.5481169479182) bank76679 +76680 POINT(40.33633540561276 -74.95658233033824) bank76680 +76681 POINT(41.61234060819806 -74.74712016840405) bank76681 +76682 POINT(41.43302154950737 -73.32927707280993) bank76682 +76683 POINT(41.12323059253908 -74.30639818495627) bank76683 +76684 POINT(41.354177182376276 -74.42848336352566) bank76684 +76685 POINT(40.366768210759055 -73.22324668146153) bank76685 +76686 POINT(41.31838149417038 -73.58055718677126) bank76686 +76687 POINT(40.50567391525334 -73.92477424958655) bank76687 +76688 POINT(40.12620330332038 -74.53175622822175) bank76688 +76689 POINT(39.75925733362233 -73.35077415372261) bank76689 +76690 POINT(41.39972852775131 -73.32312199424915) bank76690 +76691 POINT(41.571784297092286 -74.91937782797875) bank76691 +76692 POINT(40.824066449838256 -73.43311950465387) bank76692 +76693 POINT(40.19297837323053 -73.81144883810272) bank76693 +76694 POINT(41.428814634730436 -73.73837250942148) bank76694 +76695 POINT(40.09172215370937 -74.37057914916298) bank76695 +76696 POINT(40.02525752591902 -73.97179864757872) bank76696 +76697 POINT(40.818111681796495 -74.14232132809836) bank76697 +76698 POINT(40.6758427176004 -73.84415757500497) bank76698 +76699 POINT(40.576866840327185 -73.91882164828212) bank76699 +76700 POINT(40.70524348118806 -74.71268141899732) bank76700 +76701 POINT(41.361604196169075 -73.09064626568284) bank76701 +76702 POINT(40.510667771701435 -74.55447518091691) bank76702 +76703 POINT(40.93271801852297 -73.39400310262087) bank76703 +76704 POINT(40.50922262660739 -74.31599956111874) bank76704 +76705 POINT(40.32620778216228 -73.6104812706911) bank76705 +76706 POINT(41.05543739976563 -74.3915027213525) bank76706 +76707 POINT(40.35753624673836 -74.99463332064799) bank76707 +76708 POINT(40.38148841552084 -74.68880322217157) bank76708 +76709 POINT(40.47474862699988 -74.29294535562754) bank76709 +76710 POINT(40.82723114279659 -74.1678363910498) bank76710 +76711 POINT(40.44629636852258 -74.14109625545814) bank76711 +76712 POINT(40.10505971952106 -74.10236310963619) bank76712 +76713 POINT(41.28871359667129 -73.51091448280302) bank76713 +76714 POINT(39.9483267365099 -73.27838689796296) bank76714 +76715 POINT(40.278071877289086 -74.01000565734067) bank76715 +76716 POINT(39.834306381300074 -73.26138364530618) bank76716 +76717 POINT(40.997529473007724 -74.6727094356634) bank76717 +76718 POINT(41.228737327695896 -74.94725636689448) bank76718 +76719 POINT(41.08898304316283 -73.1445499301121) bank76719 +76720 POINT(41.23510340445749 -74.3024769195834) bank76720 +76721 POINT(39.82498093839491 -73.47946332221255) bank76721 +76722 POINT(41.471443230469916 -73.55433906701369) bank76722 +76723 POINT(39.87523768992983 -73.14829573428172) bank76723 +76724 POINT(40.971573936149376 -74.17662042171455) bank76724 +76725 POINT(39.95827473504516 -74.9306100605437) bank76725 +76726 POINT(40.03264496506985 -74.99706284275845) bank76726 +76727 POINT(40.97941920040283 -73.93543449260589) bank76727 +76728 POINT(39.89780268645006 -73.87006396283954) bank76728 +76729 POINT(40.78836291926459 -74.62646084752906) bank76729 +76730 POINT(39.94742321189532 -74.49747381515478) bank76730 +76731 POINT(41.2187882252926 -73.73064305186215) bank76731 +76732 POINT(40.52614146576058 -73.47012197574948) bank76732 +76733 POINT(40.086046631403136 -74.33264264001988) bank76733 +76734 POINT(39.77839580888952 -73.74719625203205) bank76734 +76735 POINT(40.44653982015406 -74.31562628415476) bank76735 +76736 POINT(41.435704648748455 -73.73082997836114) bank76736 +76737 POINT(40.63647897217268 -74.16216307406533) bank76737 +76738 POINT(41.09092218193115 -74.33170133429681) bank76738 +76739 POINT(40.35094042581081 -73.04260195182287) bank76739 +76740 POINT(40.660077391406915 -74.50265557284465) bank76740 +76741 POINT(40.88231233641292 -74.25996938998478) bank76741 +76742 POINT(40.14773120061162 -74.22881521344246) bank76742 +76743 POINT(40.75762363649357 -74.07324750770387) bank76743 +76744 POINT(41.47528655799298 -74.84263952665488) bank76744 +76745 POINT(41.241577118762464 -73.67301765493688) bank76745 +76746 POINT(40.93965014629698 -73.50930744418882) bank76746 +76747 POINT(40.6948691323841 -73.513444568206) bank76747 +76748 POINT(40.22683110162104 -73.85016891824517) bank76748 +76749 POINT(40.85015998186628 -74.65781116114596) bank76749 +76750 POINT(41.488246395428256 -73.77557993483727) bank76750 +76751 POINT(41.269060856262506 -74.91043473659398) bank76751 +76752 POINT(40.559565749644676 -73.178334599738) bank76752 +76753 POINT(41.50638676400659 -74.66848339995626) bank76753 +76754 POINT(39.81082821497606 -74.67228294104751) bank76754 +76755 POINT(40.81520439394708 -74.07944358030788) bank76755 +76756 POINT(40.268518187683306 -74.0423531428178) bank76756 +76757 POINT(40.247144127141105 -74.8844951159118) bank76757 +76758 POINT(41.04234943004183 -74.709678912671) bank76758 +76759 POINT(41.03568632251259 -73.44860402114878) bank76759 +76760 POINT(40.44356661307535 -73.35400712186082) bank76760 +76761 POINT(40.30349711640482 -73.7669392092161) bank76761 +76762 POINT(39.77934908597032 -73.47533994855071) bank76762 +76763 POINT(41.56059443246514 -74.00270936132546) bank76763 +76764 POINT(39.75213371521854 -74.30945816626927) bank76764 +76765 POINT(41.491794097604455 -74.82813809261646) bank76765 +76766 POINT(40.94194568721567 -73.36059533620853) bank76766 +76767 POINT(40.20718793516312 -73.8318354473262) bank76767 +76768 POINT(41.18980058407057 -74.49038490742842) bank76768 +76769 POINT(41.02775148635008 -74.87078323672308) bank76769 +76770 POINT(40.92088879310082 -74.96423874701482) bank76770 +76771 POINT(40.14782581645888 -73.3724837861094) bank76771 +76772 POINT(40.27369129428237 -73.4657796749781) bank76772 +76773 POINT(40.528907141871315 -74.67774604273325) bank76773 +76774 POINT(40.77988043675393 -74.14451371027991) bank76774 +76775 POINT(40.45078281370369 -74.35641565939113) bank76775 +76776 POINT(41.04938780900783 -73.65415355985013) bank76776 +76777 POINT(40.902697411133126 -74.91337958645036) bank76777 +76778 POINT(40.39507154551106 -73.52596071798814) bank76778 +76779 POINT(40.341027421518085 -73.12636401889216) bank76779 +76780 POINT(40.52829679538272 -73.15979052705522) bank76780 +76781 POINT(40.96200037522312 -73.72115303322138) bank76781 +76782 POINT(40.013472626218785 -74.9015997781819) bank76782 +76783 POINT(39.821666793016064 -73.83127853079282) bank76783 +76784 POINT(40.88196333037614 -73.83017860695301) bank76784 +76785 POINT(41.64638390337887 -73.05600793912475) bank76785 +76786 POINT(40.86525211605589 -73.48784660260024) bank76786 +76787 POINT(40.28940728456576 -74.34691591098408) bank76787 +76788 POINT(41.480493481354486 -73.81804808829199) bank76788 +76789 POINT(40.78874085050876 -74.36092669237203) bank76789 +76790 POINT(40.32342283688903 -74.20865606480145) bank76790 +76791 POINT(40.254395249353095 -74.86055108421668) bank76791 +76792 POINT(40.02989913672965 -74.53402435133505) bank76792 +76793 POINT(40.57374239690454 -73.4940247004818) bank76793 +76794 POINT(40.05515221397937 -73.06713978293864) bank76794 +76795 POINT(41.517559842152 -74.76113119315455) bank76795 +76796 POINT(39.86426665982446 -73.71504347169476) bank76796 +76797 POINT(41.19588886476238 -74.14355886934554) bank76797 +76798 POINT(40.872916413765 -73.01722693188874) bank76798 +76799 POINT(41.094636464748035 -74.7211174315404) bank76799 +76800 POINT(40.10653134986599 -73.1813778277413) bank76800 +76801 POINT(41.56982468756759 -74.3799466057029) bank76801 +76802 POINT(40.44420833480493 -73.81905643563357) bank76802 +76803 POINT(41.641009493768244 -74.91643308985407) bank76803 +76804 POINT(41.30303494436621 -74.74257376924953) bank76804 +76805 POINT(40.660580180538005 -73.44663555983188) bank76805 +76806 POINT(40.20520451183148 -73.2415593863101) bank76806 +76807 POINT(39.847834433050515 -74.38307752734325) bank76807 +76808 POINT(39.786275594354976 -74.67545263894692) bank76808 +76809 POINT(41.435877562158666 -74.23057958652826) bank76809 +76810 POINT(41.46340750976053 -74.43454761762017) bank76810 +76811 POINT(40.18198426816253 -73.10213347647455) bank76811 +76812 POINT(41.52070448941209 -73.9639031425433) bank76812 +76813 POINT(40.365615272699976 -74.68294671735453) bank76813 +76814 POINT(41.15743135896027 -74.3399457560599) bank76814 +76815 POINT(41.35827227857769 -74.32428921929765) bank76815 +76816 POINT(41.01694938210544 -74.65918566583811) bank76816 +76817 POINT(40.837773429091975 -74.80724925613812) bank76817 +76818 POINT(41.20822210854995 -74.71589133141264) bank76818 +76819 POINT(41.68209019553168 -73.33343503276188) bank76819 +76820 POINT(40.378597152690645 -73.5101034613037) bank76820 +76821 POINT(40.44949651587518 -74.52068404866544) bank76821 +76822 POINT(40.85937929595966 -74.0169556734433) bank76822 +76823 POINT(40.28520779555277 -74.47473012048461) bank76823 +76824 POINT(40.326826594392784 -73.62882152922771) bank76824 +76825 POINT(40.68916704131425 -73.87377850191662) bank76825 +76826 POINT(39.77453094631366 -74.1897014956682) bank76826 +76827 POINT(40.033791959711856 -74.51153080018014) bank76827 +76828 POINT(40.06358050528848 -73.89123545902446) bank76828 +76829 POINT(41.15179334979009 -73.93266130326138) bank76829 +76830 POINT(40.42480962799052 -74.29102156931005) bank76830 +76831 POINT(40.40190942478837 -73.09376796233903) bank76831 +76832 POINT(39.960012190816784 -73.72571220032944) bank76832 +76833 POINT(40.124139413590235 -74.12189348091417) bank76833 +76834 POINT(41.03734397162145 -73.10973396310852) bank76834 +76835 POINT(40.40044789317441 -73.58084076742625) bank76835 +76836 POINT(40.98151498559816 -73.14118583431792) bank76836 +76837 POINT(40.43449434989026 -73.57703632961508) bank76837 +76838 POINT(41.665523286359814 -73.03171783399343) bank76838 +76839 POINT(39.90126562905691 -74.15148117210337) bank76839 +76840 POINT(40.60115940894696 -73.90093324538458) bank76840 +76841 POINT(41.0680642235564 -74.54250761781668) bank76841 +76842 POINT(41.46340959617404 -73.3536472762674) bank76842 +76843 POINT(40.035976390186015 -73.49106543362419) bank76843 +76844 POINT(39.9045301744584 -74.61783696214093) bank76844 +76845 POINT(39.9739482548029 -74.27667036671076) bank76845 +76846 POINT(40.08773850074231 -74.7314290868163) bank76846 +76847 POINT(41.54506191908326 -74.05730095492328) bank76847 +76848 POINT(40.7363782573585 -73.95525886135707) bank76848 +76849 POINT(40.896179713889154 -74.1178581775628) bank76849 +76850 POINT(40.48984813933506 -74.64122207899047) bank76850 +76851 POINT(41.686128353697086 -74.82883407205249) bank76851 +76852 POINT(40.50804169637979 -74.63263187852527) bank76852 +76853 POINT(40.80921341350751 -74.81617713015196) bank76853 +76854 POINT(40.28647046238769 -74.13233061988204) bank76854 +76855 POINT(41.31978429837653 -74.05742227158422) bank76855 +76856 POINT(41.324880180223964 -73.69066249201333) bank76856 +76857 POINT(40.727840573877984 -74.86456349886433) bank76857 +76858 POINT(39.84515091013569 -73.21652606169164) bank76858 +76859 POINT(39.92658348037716 -73.1590641192116) bank76859 +76860 POINT(41.56434466988267 -73.05396781036504) bank76860 +76861 POINT(41.236832037614306 -73.56937748555143) bank76861 +76862 POINT(39.872960956797165 -73.88209944461205) bank76862 +76863 POINT(41.592934158639714 -73.37230452099745) bank76863 +76864 POINT(40.157180269813914 -74.2183391290944) bank76864 +76865 POINT(41.41489117053304 -74.57302399873002) bank76865 +76866 POINT(41.35481278637304 -74.86204928482697) bank76866 +76867 POINT(40.94482696009625 -74.89938102451835) bank76867 +76868 POINT(40.65033502098699 -73.14817412982322) bank76868 +76869 POINT(41.46284943350075 -73.50066499607331) bank76869 +76870 POINT(39.90843709736663 -74.05161917306778) bank76870 +76871 POINT(40.488747488154225 -74.9700484379725) bank76871 +76872 POINT(40.52652876553223 -74.25065400047842) bank76872 +76873 POINT(41.605914372679635 -74.17083384004562) bank76873 +76874 POINT(39.89959339549017 -74.72854103429037) bank76874 +76875 POINT(40.94848535608615 -74.79532086625065) bank76875 +76876 POINT(39.97571663938187 -74.78271066792483) bank76876 +76877 POINT(40.94780052264235 -74.6704360131946) bank76877 +76878 POINT(40.99842978141982 -74.94910653170174) bank76878 +76879 POINT(39.97381881574229 -74.98888488376444) bank76879 +76880 POINT(41.10090011534679 -74.55679698855191) bank76880 +76881 POINT(41.426504712000536 -73.70637330019457) bank76881 +76882 POINT(41.40914681208775 -73.5848349514758) bank76882 +76883 POINT(40.1810298482354 -73.6067260414195) bank76883 +76884 POINT(40.31132375772243 -73.09706430981701) bank76884 +76885 POINT(41.07790517032632 -74.78389400594558) bank76885 +76886 POINT(39.9734765087741 -73.62778287368943) bank76886 +76887 POINT(40.682049133973464 -73.10404066244907) bank76887 +76888 POINT(40.08026002200007 -73.11857450888049) bank76888 +76889 POINT(40.19578456323839 -73.20815318034757) bank76889 +76890 POINT(41.034288840587735 -74.49233180936331) bank76890 +76891 POINT(40.763878171299396 -74.96272815076865) bank76891 +76892 POINT(39.9249845625496 -74.15465565995787) bank76892 +76893 POINT(40.068860334322665 -73.62072637325721) bank76893 +76894 POINT(40.46660934410735 -74.99414104349478) bank76894 +76895 POINT(40.412042597838024 -74.57547749422162) bank76895 +76896 POINT(39.74623546350752 -74.04075907382793) bank76896 +76897 POINT(41.52285484058385 -74.41263385081855) bank76897 +76898 POINT(40.90432518375262 -73.76785468685578) bank76898 +76899 POINT(41.67253799337627 -73.26588991643166) bank76899 +76900 POINT(40.163682356368156 -73.22539011353712) bank76900 +76901 POINT(41.21102974932814 -74.97486086165581) bank76901 +76902 POINT(40.768993037809125 -74.27410407655884) bank76902 +76903 POINT(41.18336687267068 -74.84063366454771) bank76903 +76904 POINT(41.51846608734727 -73.60122787743468) bank76904 +76905 POINT(39.82598173117615 -73.10611953046104) bank76905 +76906 POINT(41.370006425179426 -73.2646021648474) bank76906 +76907 POINT(40.81636158184127 -74.70453598499586) bank76907 +76908 POINT(40.287053521686666 -74.34925173223573) bank76908 +76909 POINT(41.70575035175344 -73.41662334071493) bank76909 +76910 POINT(41.687451114568645 -73.86094143684545) bank76910 +76911 POINT(41.302778346095565 -73.4926886582931) bank76911 +76912 POINT(40.72817287844662 -73.99077332793703) bank76912 +76913 POINT(41.272901666872734 -74.97673579737632) bank76913 +76914 POINT(40.51945045582635 -74.21150023568507) bank76914 +76915 POINT(40.2613831907091 -74.87827142446366) bank76915 +76916 POINT(40.05323495093239 -73.68999117342535) bank76916 +76917 POINT(40.46271518100589 -74.07618398060562) bank76917 +76918 POINT(40.76147218051143 -74.60280532173783) bank76918 +76919 POINT(40.10296965042826 -74.79639999642886) bank76919 +76920 POINT(40.167851542333466 -74.45830552038692) bank76920 +76921 POINT(41.202055126373374 -73.35088159832777) bank76921 +76922 POINT(41.11646906424576 -74.68750097931444) bank76922 +76923 POINT(41.231936325427085 -74.18219551211743) bank76923 +76924 POINT(40.75206908059576 -73.61355095912626) bank76924 +76925 POINT(40.4734524280899 -73.52886120269164) bank76925 +76926 POINT(41.19312027487043 -73.9098546230136) bank76926 +76927 POINT(40.183545225318575 -73.57474829006811) bank76927 +76928 POINT(41.27335633292076 -73.87029353257941) bank76928 +76929 POINT(41.06112947226426 -73.26904023444476) bank76929 +76930 POINT(41.63585830758968 -73.67841659417995) bank76930 +76931 POINT(41.421943886728236 -74.92843942735786) bank76931 +76932 POINT(41.26740929485114 -74.99551353391641) bank76932 +76933 POINT(40.668041012860876 -73.49872746266352) bank76933 +76934 POINT(40.77728710773404 -73.42321851764414) bank76934 +76935 POINT(41.21346333274181 -73.73275683102152) bank76935 +76936 POINT(40.83574729772102 -74.66879250052703) bank76936 +76937 POINT(41.27750295705585 -74.30599369880628) bank76937 +76938 POINT(41.54797244734158 -74.14496184690047) bank76938 +76939 POINT(40.44060282802727 -73.15756683287292) bank76939 +76940 POINT(41.33523895940181 -74.26574738312935) bank76940 +76941 POINT(41.57678203256769 -74.19572642113393) bank76941 +76942 POINT(40.28784655519614 -73.05859990337433) bank76942 +76943 POINT(41.24618532196053 -74.02845407026085) bank76943 +76944 POINT(39.91286779195367 -73.68126487573443) bank76944 +76945 POINT(40.801600646222504 -74.16777719448864) bank76945 +76946 POINT(40.13588209418924 -74.7458460220307) bank76946 +76947 POINT(40.85432781809284 -73.26064148547621) bank76947 +76948 POINT(41.27741844150782 -74.00411199708894) bank76948 +76949 POINT(40.67044630027378 -74.12475624078681) bank76949 +76950 POINT(40.09531743039065 -73.5485060140302) bank76950 +76951 POINT(40.657532502027365 -73.12271802397989) bank76951 +76952 POINT(40.740620895707224 -74.7646472465454) bank76952 +76953 POINT(39.73721002227659 -73.90077724053577) bank76953 +76954 POINT(39.9110474773375 -74.9947790146345) bank76954 +76955 POINT(39.96302526500531 -73.68111823866315) bank76955 +76956 POINT(40.704204651036065 -74.44197591963376) bank76956 +76957 POINT(40.87877989783363 -73.68784627713109) bank76957 +76958 POINT(39.98706593285322 -73.45312503445147) bank76958 +76959 POINT(40.83929109574973 -73.29746802554608) bank76959 +76960 POINT(40.653444550857124 -74.27891905346523) bank76960 +76961 POINT(41.04547735955923 -74.67015819359337) bank76961 +76962 POINT(41.22217869325163 -74.46573518025764) bank76962 +76963 POINT(41.310591076736415 -74.89970459090172) bank76963 +76964 POINT(41.33091265618869 -73.78567558627059) bank76964 +76965 POINT(40.659033871586544 -74.11414895263161) bank76965 +76966 POINT(40.47710559843108 -73.78718136413761) bank76966 +76967 POINT(41.213091136700726 -73.2243905357482) bank76967 +76968 POINT(39.96501550600421 -73.21051737558767) bank76968 +76969 POINT(40.13963239964988 -73.47267501032192) bank76969 +76970 POINT(39.902989041152615 -73.85362678282624) bank76970 +76971 POINT(41.33021336982563 -73.46584704918779) bank76971 +76972 POINT(40.4126946930058 -74.49262468988296) bank76972 +76973 POINT(41.10776640939124 -73.9586965061279) bank76973 +76974 POINT(40.72862364544194 -74.75325270953697) bank76974 +76975 POINT(41.361558673815495 -73.14753623257255) bank76975 +76976 POINT(40.369859299446304 -74.88290620529601) bank76976 +76977 POINT(39.986318501495425 -73.69607815885915) bank76977 +76978 POINT(40.64602851918736 -74.95441952727944) bank76978 +76979 POINT(41.34061155835673 -73.05476257071365) bank76979 +76980 POINT(40.83034620807457 -74.20104846234888) bank76980 +76981 POINT(41.17403195640137 -73.89318537165187) bank76981 +76982 POINT(40.720956467413245 -74.24577920082218) bank76982 +76983 POINT(40.94182612201863 -73.38130345480356) bank76983 +76984 POINT(41.08749388374907 -73.03022085889509) bank76984 +76985 POINT(41.05480973797309 -73.51855155581453) bank76985 +76986 POINT(40.00651490650148 -74.89421502720322) bank76986 +76987 POINT(41.49233411556657 -73.13939162029338) bank76987 +76988 POINT(40.84332300430665 -74.0202991511059) bank76988 +76989 POINT(40.73025587906388 -74.13187049826769) bank76989 +76990 POINT(40.46051974571518 -73.63636111183277) bank76990 +76991 POINT(40.958032321421044 -73.92303450869045) bank76991 +76992 POINT(41.43145037821486 -73.68339987510475) bank76992 +76993 POINT(40.54449312469385 -74.92145929468559) bank76993 +76994 POINT(40.94125934317058 -73.07245011313934) bank76994 +76995 POINT(40.96766143856825 -74.3490113584633) bank76995 +76996 POINT(41.139730405746086 -73.81836122113067) bank76996 +76997 POINT(41.05616077419062 -74.65680711249848) bank76997 +76998 POINT(40.85442955209478 -73.02624181454343) bank76998 +76999 POINT(40.68466310313941 -74.49308329898388) bank76999 +77000 POINT(40.52851248990641 -74.3978506732046) bank77000 +77001 POINT(39.983027950188216 -73.30065968639704) bank77001 +77002 POINT(40.6866155093833 -73.72290880496762) bank77002 +77003 POINT(40.2449930993432 -74.82634992967006) bank77003 +77004 POINT(40.69753313524997 -73.82756444087507) bank77004 +77005 POINT(40.073721590865034 -73.09701105230482) bank77005 +77006 POINT(40.079298035310636 -74.88960503882586) bank77006 +77007 POINT(41.62058273180029 -74.53281841653806) bank77007 +77008 POINT(41.09196771434754 -73.13569056734173) bank77008 +77009 POINT(40.48760324424123 -74.08060116325663) bank77009 +77010 POINT(40.8491491087125 -73.22790715786931) bank77010 +77011 POINT(40.9193569248844 -73.13985343471015) bank77011 +77012 POINT(41.49490790320007 -74.05906783928714) bank77012 +77013 POINT(41.329254102346134 -74.87615647958702) bank77013 +77014 POINT(41.53118902903355 -73.17876932596249) bank77014 +77015 POINT(40.28034110706601 -74.25078524832759) bank77015 +77016 POINT(41.4949284592185 -73.6148938678138) bank77016 +77017 POINT(39.984853598535096 -74.947497592293) bank77017 +77018 POINT(39.87030237122192 -73.58604134835744) bank77018 +77019 POINT(41.33824324343 -73.26372329610201) bank77019 +77020 POINT(41.690831497694354 -74.37957221874979) bank77020 +77021 POINT(41.31269055990981 -74.88859652779294) bank77021 +77022 POINT(41.410935948218885 -73.02599131313112) bank77022 +77023 POINT(40.25141474087844 -74.24999434864137) bank77023 +77024 POINT(41.08671233012481 -74.13541934991895) bank77024 +77025 POINT(41.113208160636866 -74.04865064935913) bank77025 +77026 POINT(39.80771181672035 -74.62951390010679) bank77026 +77027 POINT(40.21800632753549 -73.81914083758024) bank77027 +77028 POINT(41.47617304099988 -74.36439456178701) bank77028 +77029 POINT(40.70784036505648 -73.33797227245306) bank77029 +77030 POINT(40.94794114058454 -74.4478862547892) bank77030 +77031 POINT(40.430764626821144 -74.95625835257599) bank77031 +77032 POINT(41.1907013451187 -73.60212954146353) bank77032 +77033 POINT(40.51305553512754 -74.65437946638224) bank77033 +77034 POINT(40.92739639399508 -74.28152991614074) bank77034 +77035 POINT(40.64358753697515 -74.06635806476505) bank77035 +77036 POINT(40.22075829511895 -73.41991132842223) bank77036 +77037 POINT(41.28583717285552 -74.71339677196649) bank77037 +77038 POINT(40.68857923354323 -73.68198499193103) bank77038 +77039 POINT(40.46178550435698 -73.25601548771486) bank77039 +77040 POINT(41.17982490610666 -74.36257682770835) bank77040 +77041 POINT(39.79084912107165 -74.03468066220222) bank77041 +77042 POINT(40.79407737399415 -74.33085501910412) bank77042 +77043 POINT(41.15834173559636 -74.18664697583918) bank77043 +77044 POINT(41.01019093488265 -74.95406737702758) bank77044 +77045 POINT(40.245934227719715 -73.3450218828774) bank77045 +77046 POINT(40.019854292703016 -73.77833214249736) bank77046 +77047 POINT(40.40411926532005 -74.78389553614295) bank77047 +77048 POINT(39.8599582077729 -73.42039374778619) bank77048 +77049 POINT(39.77910622627676 -74.06443850625459) bank77049 +77050 POINT(40.92794656413078 -73.2443462797958) bank77050 +77051 POINT(39.94470690713235 -74.91717767556173) bank77051 +77052 POINT(41.397689209328036 -73.5653826845959) bank77052 +77053 POINT(41.15418407088868 -74.34485760617397) bank77053 +77054 POINT(41.513683162920955 -73.50274896778656) bank77054 +77055 POINT(39.97786246048472 -73.62547370289771) bank77055 +77056 POINT(41.53140586987682 -73.70972652685724) bank77056 +77057 POINT(40.14572979777112 -74.78367022808382) bank77057 +77058 POINT(41.11509207332873 -73.27133133810999) bank77058 +77059 POINT(39.74331455497029 -74.55191420442324) bank77059 +77060 POINT(39.80031530735645 -74.13248996264373) bank77060 +77061 POINT(41.08813188624972 -74.45876941543037) bank77061 +77062 POINT(39.963539847014665 -73.2650724390264) bank77062 +77063 POINT(41.44589192162202 -74.78745068320013) bank77063 +77064 POINT(40.904069738420766 -74.9775273090644) bank77064 +77065 POINT(40.69946025656972 -73.01034113734718) bank77065 +77066 POINT(40.39165240740755 -74.56419977302883) bank77066 +77067 POINT(40.3571330649468 -73.72121093496223) bank77067 +77068 POINT(40.090463779475485 -74.9580679899639) bank77068 +77069 POINT(41.585372637777006 -73.25588471272977) bank77069 +77070 POINT(40.86587602127986 -74.73447269173622) bank77070 +77071 POINT(41.24809201519242 -74.64722574410773) bank77071 +77072 POINT(41.646528869451345 -74.00025875542391) bank77072 +77073 POINT(40.05815743333651 -73.72398681512979) bank77073 +77074 POINT(40.01771829839296 -73.37238606550159) bank77074 +77075 POINT(39.79175484939398 -74.41517596150264) bank77075 +77076 POINT(41.355814480506474 -74.88133331160917) bank77076 +77077 POINT(40.59280522234182 -73.65054180876467) bank77077 +77078 POINT(40.57461640791028 -74.41890331896666) bank77078 +77079 POINT(41.6881427555781 -73.10359830187755) bank77079 +77080 POINT(41.350101086249985 -73.43644098982443) bank77080 +77081 POINT(40.481510191975495 -73.25843027122121) bank77081 +77082 POINT(41.607982626400876 -74.60695872876553) bank77082 +77083 POINT(41.16367066337164 -74.31572763528173) bank77083 +77084 POINT(40.88204399029169 -73.1595279483815) bank77084 +77085 POINT(40.70819968461569 -74.52980634289544) bank77085 +77086 POINT(41.69125414135444 -74.5484440619907) bank77086 +77087 POINT(41.336797185262604 -73.42979382914504) bank77087 +77088 POINT(41.186118792513696 -73.21774680594649) bank77088 +77089 POINT(39.88619073743391 -74.84908002659145) bank77089 +77090 POINT(41.47652685682194 -73.10353713093187) bank77090 +77091 POINT(41.115826684575225 -74.6809633229886) bank77091 +77092 POINT(40.5898162720142 -73.70047722306234) bank77092 +77093 POINT(40.27467311626328 -73.77183553619864) bank77093 +77094 POINT(41.196118448572896 -74.31839869995937) bank77094 +77095 POINT(40.2160506954142 -73.35843170372647) bank77095 +77096 POINT(40.924280032047164 -73.44042935914702) bank77096 +77097 POINT(40.64317357640513 -74.4340883956526) bank77097 +77098 POINT(40.51833555574425 -73.4956719118554) bank77098 +77099 POINT(39.99577357608022 -74.64292455554141) bank77099 +77100 POINT(41.62761001775188 -73.39782815770238) bank77100 +77101 POINT(41.44751550206705 -73.85795854071742) bank77101 +77102 POINT(39.91387930320277 -74.80662453301571) bank77102 +77103 POINT(40.368004705824724 -73.64943721619584) bank77103 +77104 POINT(40.25664675697624 -74.0090145660756) bank77104 +77105 POINT(41.090922469971346 -73.14517101484716) bank77105 +77106 POINT(40.98918310603012 -74.30861471146001) bank77106 +77107 POINT(40.88106612430839 -73.45711943667575) bank77107 +77108 POINT(41.56028094314134 -74.51304918607485) bank77108 +77109 POINT(39.81900012342186 -73.2963873424942) bank77109 +77110 POINT(41.32367112781397 -74.54277155993401) bank77110 +77111 POINT(40.97742418858865 -74.83318248231022) bank77111 +77112 POINT(41.39394438921979 -73.89411489723368) bank77112 +77113 POINT(40.27782504580449 -73.88572931356713) bank77113 +77114 POINT(40.231743015827725 -74.6142572098239) bank77114 +77115 POINT(41.07152353200153 -73.22308033767746) bank77115 +77116 POINT(40.72289999956019 -74.00900427824766) bank77116 +77117 POINT(40.65268352686303 -74.60253776910075) bank77117 +77118 POINT(41.27068815149185 -74.51206997069035) bank77118 +77119 POINT(40.275024464111894 -74.85961159631944) bank77119 +77120 POINT(40.46693605400683 -74.15798630473512) bank77120 +77121 POINT(40.06090094938418 -74.54470040205396) bank77121 +77122 POINT(41.485109624882256 -73.12240412030933) bank77122 +77123 POINT(40.269717843884116 -73.81768588926646) bank77123 +77124 POINT(41.46665527627014 -73.94502854183312) bank77124 +77125 POINT(41.096538261355605 -74.27583292195156) bank77125 +77126 POINT(40.81497190201271 -74.06968454861416) bank77126 +77127 POINT(40.766153685604976 -73.49007354684349) bank77127 +77128 POINT(40.12404849898674 -74.0661128848715) bank77128 +77129 POINT(39.76557629139921 -74.89801994271141) bank77129 +77130 POINT(40.65010902827962 -73.09148776230505) bank77130 +77131 POINT(40.49972466575745 -73.07789616037904) bank77131 +77132 POINT(41.67802523740013 -74.56918679473304) bank77132 +77133 POINT(40.300252553872134 -73.18581199007005) bank77133 +77134 POINT(41.092367784772684 -73.59395790294937) bank77134 +77135 POINT(40.60701477268809 -73.5132987195588) bank77135 +77136 POINT(39.90824293090988 -73.04635808406186) bank77136 +77137 POINT(40.05839542084256 -74.32033610103228) bank77137 +77138 POINT(41.49509569442597 -74.78115743985205) bank77138 +77139 POINT(40.119411297164 -74.87304991888762) bank77139 +77140 POINT(40.508669896048225 -73.86078862269243) bank77140 +77141 POINT(41.46793807190904 -74.19668125647406) bank77141 +77142 POINT(41.67260368493679 -73.40950584503719) bank77142 +77143 POINT(40.56069451001453 -74.21676043122308) bank77143 +77144 POINT(40.9749074488252 -73.43769108158885) bank77144 +77145 POINT(40.59574644515965 -74.32505022003184) bank77145 +77146 POINT(41.17050012260773 -74.07882553459758) bank77146 +77147 POINT(40.973548374534786 -73.54098944831445) bank77147 +77148 POINT(39.909376025855586 -73.11620899087661) bank77148 +77149 POINT(41.32699835740515 -74.64328517120185) bank77149 +77150 POINT(40.90669862910882 -74.44603030483758) bank77150 +77151 POINT(41.18709750628594 -73.34253681109521) bank77151 +77152 POINT(39.76384180545184 -73.8480593418444) bank77152 +77153 POINT(39.96217836559048 -74.6090964195879) bank77153 +77154 POINT(41.55012935261721 -74.23788085829032) bank77154 +77155 POINT(41.091869225636536 -74.80926698145265) bank77155 +77156 POINT(41.1471960384276 -73.70098141576418) bank77156 +77157 POINT(40.87078240144304 -73.50481495861028) bank77157 +77158 POINT(40.927142827406854 -73.79910922966299) bank77158 +77159 POINT(40.53774913269924 -73.83107042439188) bank77159 +77160 POINT(41.16150063169029 -73.64765578183693) bank77160 +77161 POINT(41.695714750093586 -73.39884758511633) bank77161 +77162 POINT(40.66029804603712 -74.63836393532237) bank77162 +77163 POINT(40.617832097724 -73.98151247984244) bank77163 +77164 POINT(40.8084863323342 -73.46628641285331) bank77164 +77165 POINT(40.432352922792404 -73.29737032980853) bank77165 +77166 POINT(40.76107826842652 -74.65845005420557) bank77166 +77167 POINT(40.99476248439447 -73.74795296080316) bank77167 +77168 POINT(41.66140075355755 -73.97549534502494) bank77168 +77169 POINT(40.46837061703605 -73.0524285776116) bank77169 +77170 POINT(40.16014939035624 -73.8852494861408) bank77170 +77171 POINT(40.62457650597543 -73.70877026672052) bank77171 +77172 POINT(40.60678096951226 -74.43207415513987) bank77172 +77173 POINT(41.48615582919346 -73.2914818270906) bank77173 +77174 POINT(41.12162794934042 -73.02335190872536) bank77174 +77175 POINT(40.866140770992864 -74.67329027866235) bank77175 +77176 POINT(41.287306978842246 -74.14999497418486) bank77176 +77177 POINT(40.59328670580973 -73.28172577506497) bank77177 +77178 POINT(40.320077806437865 -73.58907472039873) bank77178 +77179 POINT(40.25782850950403 -74.6383551295265) bank77179 +77180 POINT(41.241458840235325 -73.91060752143638) bank77180 +77181 POINT(41.71118542922154 -73.59229444126537) bank77181 +77182 POINT(40.650450963669826 -74.49070884681701) bank77182 +77183 POINT(40.97688676173399 -73.64278158641503) bank77183 +77184 POINT(39.9723656942483 -74.62639460170189) bank77184 +77185 POINT(40.62209320785215 -74.42024016560694) bank77185 +77186 POINT(39.96989483010165 -74.69919602865679) bank77186 +77187 POINT(41.64299510302072 -73.07844342057132) bank77187 +77188 POINT(40.03551010942573 -74.30889468901152) bank77188 +77189 POINT(39.831490065161 -74.86545097770168) bank77189 +77190 POINT(41.304389439124336 -73.52644136744767) bank77190 +77191 POINT(40.281804597923255 -73.84335920613896) bank77191 +77192 POINT(41.53186902521012 -74.15588493520923) bank77192 +77193 POINT(41.677900574018324 -73.8293134170547) bank77193 +77194 POINT(41.26860872844471 -73.25133005478523) bank77194 +77195 POINT(39.83706625247607 -73.56881008061191) bank77195 +77196 POINT(41.21139471754884 -74.92876234427897) bank77196 +77197 POINT(41.38897883455897 -73.32456004878338) bank77197 +77198 POINT(41.0209587096938 -74.77282198769372) bank77198 +77199 POINT(41.635188591300846 -73.24851370701441) bank77199 +77200 POINT(41.69347565540317 -73.34417394354904) bank77200 +77201 POINT(40.42073940297551 -73.26174878658787) bank77201 +77202 POINT(40.435369219792605 -74.02996092951027) bank77202 +77203 POINT(40.14022349950464 -73.66942345956471) bank77203 +77204 POINT(40.37279355462908 -73.93888493365615) bank77204 +77205 POINT(40.87652944053367 -73.22477233990429) bank77205 +77206 POINT(40.265366946701214 -74.61687424398117) bank77206 +77207 POINT(40.742036931350896 -73.06198937520084) bank77207 +77208 POINT(40.02260816700167 -74.90414365008775) bank77208 +77209 POINT(40.85362562521637 -74.72063740080792) bank77209 +77210 POINT(40.88031037914273 -74.60946083353397) bank77210 +77211 POINT(40.548525348934035 -73.51495356871158) bank77211 +77212 POINT(40.06325917983026 -74.03927940021016) bank77212 +77213 POINT(40.158098490907705 -74.71340497167454) bank77213 +77214 POINT(41.06117198223823 -74.69530345938139) bank77214 +77215 POINT(40.521944780164 -73.8344753828683) bank77215 +77216 POINT(40.021465269227356 -74.80378136001428) bank77216 +77217 POINT(41.35267775244218 -73.35105801394124) bank77217 +77218 POINT(41.581927222940735 -74.44916049511625) bank77218 +77219 POINT(39.73258635990989 -73.20290514321172) bank77219 +77220 POINT(40.69553898689946 -73.17684157754803) bank77220 +77221 POINT(40.00482182105579 -73.25749286472214) bank77221 +77222 POINT(41.080680273329506 -74.64770878215252) bank77222 +77223 POINT(40.675430206316285 -73.17316799035346) bank77223 +77224 POINT(40.19277724270559 -74.84633174980158) bank77224 +77225 POINT(40.96627665767278 -74.2473991361056) bank77225 +77226 POINT(41.20435708476652 -74.02308897519026) bank77226 +77227 POINT(40.89623629150914 -73.7297749688779) bank77227 +77228 POINT(41.66512977331444 -73.02334973791014) bank77228 +77229 POINT(40.70106083862071 -73.18102498179198) bank77229 +77230 POINT(40.9436669578378 -73.08770601696642) bank77230 +77231 POINT(40.43321232241552 -73.8296039443169) bank77231 +77232 POINT(39.95172113149866 -74.05397266144726) bank77232 +77233 POINT(41.624667813326965 -74.94046184884138) bank77233 +77234 POINT(41.24521616530015 -73.41407635658571) bank77234 +77235 POINT(41.48467987772707 -73.38853398522369) bank77235 +77236 POINT(39.893767361673966 -74.7436394013025) bank77236 +77237 POINT(40.07435655700899 -74.44610406428643) bank77237 +77238 POINT(40.70962121697733 -74.17921122791019) bank77238 +77239 POINT(40.307543555058025 -73.40847454815983) bank77239 +77240 POINT(41.56186559913766 -73.24393525820024) bank77240 +77241 POINT(40.699125938526024 -74.00677308775323) bank77241 +77242 POINT(39.99043153759819 -74.7165152075767) bank77242 +77243 POINT(40.53883930078049 -73.30635105290617) bank77243 +77244 POINT(39.94399134831647 -74.76228976521598) bank77244 +77245 POINT(41.46510573362207 -73.97108659584208) bank77245 +77246 POINT(39.951585736726464 -73.93702384373313) bank77246 +77247 POINT(41.20732568394806 -74.7959037921981) bank77247 +77248 POINT(40.96291143856941 -74.33628539757558) bank77248 +77249 POINT(40.10275515313717 -74.31507581872047) bank77249 +77250 POINT(41.15249966605925 -74.90160599742758) bank77250 +77251 POINT(41.46196837258824 -74.7519272608104) bank77251 +77252 POINT(39.7939386328522 -74.45654072881435) bank77252 +77253 POINT(39.85716705341471 -73.25416421701198) bank77253 +77254 POINT(40.49407674454406 -74.44410814204406) bank77254 +77255 POINT(41.08461621337203 -74.4987197901165) bank77255 +77256 POINT(40.15423388325369 -73.76081400746568) bank77256 +77257 POINT(39.766470125566535 -74.80439605590979) bank77257 +77258 POINT(40.978156697585966 -73.7458626191069) bank77258 +77259 POINT(40.70179614114224 -74.32612831455988) bank77259 +77260 POINT(40.910568384503726 -74.33902780422223) bank77260 +77261 POINT(39.87813385352493 -74.53258283428758) bank77261 +77262 POINT(41.707182089982055 -73.3771257198802) bank77262 +77263 POINT(40.78474535875584 -73.47640431867234) bank77263 +77264 POINT(41.36713654172556 -73.27367269209972) bank77264 +77265 POINT(41.52187168450083 -74.32113679033725) bank77265 +77266 POINT(40.400237534907085 -74.12703468836234) bank77266 +77267 POINT(40.98024811800496 -74.74027718012276) bank77267 +77268 POINT(40.076509577306474 -74.83389324389933) bank77268 +77269 POINT(40.065945009508624 -74.99104016384567) bank77269 +77270 POINT(41.35565926233253 -73.05955174992195) bank77270 +77271 POINT(40.42373375156804 -74.41162735579661) bank77271 +77272 POINT(40.49437606490711 -73.75343568526182) bank77272 +77273 POINT(41.33528875792508 -74.88780047897453) bank77273 +77274 POINT(40.08637463883547 -74.46457047524227) bank77274 +77275 POINT(40.72791836690501 -74.56820079956394) bank77275 +77276 POINT(39.977076967136455 -73.2611365498981) bank77276 +77277 POINT(41.422178429815936 -73.7172805163596) bank77277 +77278 POINT(40.60134028643286 -73.4557276427278) bank77278 +77279 POINT(40.643043170859436 -74.31317800111817) bank77279 +77280 POINT(40.436831368141625 -73.53241748690681) bank77280 +77281 POINT(41.444054647078715 -73.9651522425718) bank77281 +77282 POINT(39.753299054915324 -74.44950182435498) bank77282 +77283 POINT(41.20329491481146 -74.31535181394618) bank77283 +77284 POINT(40.32105814804533 -73.5474825567961) bank77284 +77285 POINT(41.22761925511297 -73.91252811351991) bank77285 +77286 POINT(40.27761786568862 -73.13292546848268) bank77286 +77287 POINT(41.280589907690526 -74.67100168962172) bank77287 +77288 POINT(39.89205753197828 -74.35476437912688) bank77288 +77289 POINT(40.77577197220086 -73.52870053371547) bank77289 +77290 POINT(41.6879262383171 -74.04184807845859) bank77290 +77291 POINT(41.26149913009641 -73.57814353663858) bank77291 +77292 POINT(41.148680470937336 -74.39750858790991) bank77292 +77293 POINT(41.57054993534549 -73.82881948770225) bank77293 +77294 POINT(40.89693243001579 -73.68647509998239) bank77294 +77295 POINT(39.72251030168821 -74.00568310699471) bank77295 +77296 POINT(41.45654211060484 -74.8306428322062) bank77296 +77297 POINT(41.696610733154415 -73.72258785619644) bank77297 +77298 POINT(41.35811569481794 -73.31886698861707) bank77298 +77299 POINT(39.77847962434065 -74.81693422472236) bank77299 +77300 POINT(39.72600877011697 -73.46901707209229) bank77300 +77301 POINT(41.57679038462716 -74.93718132127287) bank77301 +77302 POINT(41.48408929868695 -73.42018550800695) bank77302 +77303 POINT(40.83851137437751 -74.49808359808506) bank77303 +77304 POINT(39.884894639580445 -73.19485067338223) bank77304 +77305 POINT(41.3949509179783 -73.22429213424536) bank77305 +77306 POINT(40.0296762656451 -74.08306086504258) bank77306 +77307 POINT(41.32512416160681 -73.64650701807328) bank77307 +77308 POINT(41.608289746196895 -74.74200116064773) bank77308 +77309 POINT(39.95770194392586 -74.38241178455515) bank77309 +77310 POINT(40.62612309191285 -74.56246130451261) bank77310 +77311 POINT(40.4043068996192 -74.67220576885698) bank77311 +77312 POINT(41.29657338027482 -74.91468254015025) bank77312 +77313 POINT(40.618625846038135 -73.26201328968693) bank77313 +77314 POINT(41.17264366245015 -73.20798463064067) bank77314 +77315 POINT(41.029525096792234 -74.1252836388494) bank77315 +77316 POINT(41.058756357544056 -74.34940112606192) bank77316 +77317 POINT(41.20239352705877 -74.01285214653778) bank77317 +77318 POINT(40.029934659773936 -73.88418049780309) bank77318 +77319 POINT(40.263791492740225 -73.50614297440134) bank77319 +77320 POINT(40.63818684024306 -73.42718183667334) bank77320 +77321 POINT(41.07054403241342 -74.58288408588874) bank77321 +77322 POINT(40.47965490434205 -74.52503123222243) bank77322 +77323 POINT(41.603922713462595 -74.11528220291001) bank77323 +77324 POINT(40.76146464193257 -73.12794516635826) bank77324 +77325 POINT(41.48173460148044 -73.60996390470585) bank77325 +77326 POINT(40.330045109779384 -73.2262775775417) bank77326 +77327 POINT(41.595987750383955 -73.7271277632172) bank77327 +77328 POINT(40.52732193058874 -74.74600006664257) bank77328 +77329 POINT(40.92196821078088 -74.13722487609319) bank77329 +77330 POINT(41.592047778192935 -74.34928978184462) bank77330 +77331 POINT(40.56165355666846 -73.62145360803858) bank77331 +77332 POINT(41.136767915972634 -74.66564952202482) bank77332 +77333 POINT(40.91672934539622 -74.55964210024354) bank77333 +77334 POINT(41.146652960129906 -73.94903566431893) bank77334 +77335 POINT(39.7489968634007 -73.7482646318716) bank77335 +77336 POINT(40.75156670279734 -73.35108674950929) bank77336 +77337 POINT(40.040248137644454 -73.87255831626683) bank77337 +77338 POINT(41.58186638111662 -73.15239966126154) bank77338 +77339 POINT(40.25363516495237 -74.50065426211938) bank77339 +77340 POINT(40.85668253501936 -73.53669096762492) bank77340 +77341 POINT(40.545848118570106 -73.65608689965319) bank77341 +77342 POINT(40.34852162612039 -73.25629759946592) bank77342 +77343 POINT(40.17934558255123 -73.78435886426122) bank77343 +77344 POINT(41.58853003987451 -74.19973081739056) bank77344 +77345 POINT(40.87822887950861 -74.24426410684762) bank77345 +77346 POINT(41.05064249362182 -73.1486211047779) bank77346 +77347 POINT(41.66293171668665 -73.91757451094917) bank77347 +77348 POINT(41.6700923102267 -73.01305831189585) bank77348 +77349 POINT(40.926236130068574 -73.36665477175447) bank77349 +77350 POINT(39.887835072664835 -73.86091210575813) bank77350 +77351 POINT(41.56886432896053 -74.32863862227184) bank77351 +77352 POINT(40.76988285096811 -74.13663997271374) bank77352 +77353 POINT(40.99387308239438 -73.86540119889501) bank77353 +77354 POINT(41.357720708360574 -74.58255035785378) bank77354 +77355 POINT(39.76475709876174 -73.77255092771976) bank77355 +77356 POINT(41.41462375975426 -74.25396260018003) bank77356 +77357 POINT(41.42196544647184 -74.41319325141474) bank77357 +77358 POINT(41.641818572473696 -74.73223460938206) bank77358 +77359 POINT(41.1780022350553 -74.46258686158887) bank77359 +77360 POINT(41.68900154166261 -74.89231821200194) bank77360 +77361 POINT(41.083888827682344 -74.33764223343901) bank77361 +77362 POINT(41.39678814494204 -74.86536584485606) bank77362 +77363 POINT(40.541857296671836 -73.88833889449191) bank77363 +77364 POINT(40.057735008763814 -73.48155122550783) bank77364 +77365 POINT(41.372369049577266 -74.85712422514342) bank77365 +77366 POINT(41.025329277876565 -74.98150495404614) bank77366 +77367 POINT(41.26207447871632 -74.25758216055362) bank77367 +77368 POINT(40.186539736275115 -74.65927976577196) bank77368 +77369 POINT(40.118197067595865 -74.50497232153569) bank77369 +77370 POINT(40.79105411219132 -74.67279336148253) bank77370 +77371 POINT(40.66743208070048 -74.0195076781581) bank77371 +77372 POINT(39.87447495677571 -73.05638733978721) bank77372 +77373 POINT(39.989224769208334 -73.61442940652964) bank77373 +77374 POINT(40.74075741379116 -73.8870137500279) bank77374 +77375 POINT(41.497956559537855 -74.12222050992497) bank77375 +77376 POINT(41.40653492323659 -74.32934020487679) bank77376 +77377 POINT(40.18876743511741 -74.99800394849595) bank77377 +77378 POINT(41.49963841112801 -73.48991491622246) bank77378 +77379 POINT(40.10881083770326 -73.76787130546914) bank77379 +77380 POINT(41.34094874194295 -73.6239077640023) bank77380 +77381 POINT(39.96937255656765 -74.35706008067551) bank77381 +77382 POINT(39.83710924714555 -73.88407679642187) bank77382 +77383 POINT(41.29777813212805 -73.1823541942332) bank77383 +77384 POINT(39.965250804356025 -74.61245861893741) bank77384 +77385 POINT(41.357828277895116 -74.47183529956845) bank77385 +77386 POINT(40.174848483012696 -74.82091685229489) bank77386 +77387 POINT(40.590407237468455 -73.79021102121858) bank77387 +77388 POINT(40.9129000705814 -73.05071111167287) bank77388 +77389 POINT(39.81514002798984 -74.40166543032406) bank77389 +77390 POINT(39.89279866975519 -74.3881029640618) bank77390 +77391 POINT(40.59367010142846 -75.00148578795394) bank77391 +77392 POINT(40.441252846085014 -74.36600341073775) bank77392 +77393 POINT(40.792367630381584 -73.29253186624311) bank77393 +77394 POINT(40.663096049126864 -73.91573631253375) bank77394 +77395 POINT(41.359775421951 -73.24974291487665) bank77395 +77396 POINT(40.90117512167391 -74.41043030829) bank77396 +77397 POINT(41.04163098736687 -73.66763717562036) bank77397 +77398 POINT(41.24954888297437 -73.3602083077296) bank77398 +77399 POINT(40.15408780940125 -73.64874732559552) bank77399 +77400 POINT(40.63281432562901 -74.6334636067293) bank77400 +77401 POINT(41.64613910408036 -74.41066136729523) bank77401 +77402 POINT(40.866271781878616 -74.69257677346717) bank77402 +77403 POINT(39.967315138461416 -74.17075328488669) bank77403 +77404 POINT(40.974744315077956 -74.73581283638859) bank77404 +77405 POINT(41.44285221495635 -74.22984502957112) bank77405 +77406 POINT(40.66523876708314 -73.18135068211146) bank77406 +77407 POINT(40.07658516348363 -74.4917713875897) bank77407 +77408 POINT(40.66035944719324 -73.14669044013309) bank77408 +77409 POINT(41.59791288305307 -74.3974600029902) bank77409 +77410 POINT(39.86759314025204 -73.98625493614644) bank77410 +77411 POINT(41.21937762357331 -74.33578478093408) bank77411 +77412 POINT(41.05579726450938 -74.81726735347331) bank77412 +77413 POINT(41.037560071127416 -73.1650650454898) bank77413 +77414 POINT(40.46855816760698 -73.9027346437744) bank77414 +77415 POINT(41.27056540799463 -73.62913450393006) bank77415 +77416 POINT(39.98993876137331 -73.04554699173056) bank77416 +77417 POINT(39.72317683643215 -73.4424857768414) bank77417 +77418 POINT(40.38127782185898 -74.66419601320399) bank77418 +77419 POINT(41.389618317826404 -73.95772530769922) bank77419 +77420 POINT(40.137499630022425 -74.84817851950119) bank77420 +77421 POINT(41.66738621216008 -73.21893089631477) bank77421 +77422 POINT(40.283516789266066 -73.0078968471362) bank77422 +77423 POINT(40.449923160055455 -73.95872664330592) bank77423 +77424 POINT(41.30761906300773 -73.8558685107601) bank77424 +77425 POINT(40.00783048587133 -74.68023891551479) bank77425 +77426 POINT(40.53519392959393 -73.86691608815396) bank77426 +77427 POINT(41.21942443665882 -73.63083084571042) bank77427 +77428 POINT(40.44339093698273 -74.54544220483078) bank77428 +77429 POINT(41.37050078978168 -73.15919983389499) bank77429 +77430 POINT(40.344367231249734 -73.34690482535878) bank77430 +77431 POINT(40.009754091253214 -73.11163346366882) bank77431 +77432 POINT(41.170148311376714 -74.31141687218135) bank77432 +77433 POINT(39.76337100538256 -74.41226442759586) bank77433 +77434 POINT(40.57541653192157 -74.70765325675019) bank77434 +77435 POINT(41.03479515484056 -73.88563161851876) bank77435 +77436 POINT(40.63179403203173 -73.75741852526728) bank77436 +77437 POINT(39.83477381306594 -74.52241347048772) bank77437 +77438 POINT(40.20642655322986 -73.92113393892389) bank77438 +77439 POINT(41.033537259450085 -74.82910415526442) bank77439 +77440 POINT(41.02420866138754 -73.65387956240487) bank77440 +77441 POINT(40.540953266669526 -73.98412548591246) bank77441 +77442 POINT(40.0984299065821 -74.77274595266805) bank77442 +77443 POINT(39.98407157853582 -73.23359973790481) bank77443 +77444 POINT(41.586141929585175 -74.6300357746854) bank77444 +77445 POINT(40.94303871654425 -73.15218755034667) bank77445 +77446 POINT(40.891315449412666 -73.97085384734598) bank77446 +77447 POINT(39.71579650440092 -74.1307371935518) bank77447 +77448 POINT(40.27222915549066 -74.14224461476827) bank77448 +77449 POINT(40.2301843064021 -73.38060709143426) bank77449 +77450 POINT(40.54011520049434 -73.10035221881651) bank77450 +77451 POINT(41.25724713640667 -74.66382295847782) bank77451 +77452 POINT(40.2370146681547 -73.23611301826745) bank77452 +77453 POINT(39.72507589504422 -74.81638089611047) bank77453 +77454 POINT(40.021837911337855 -74.35628389387661) bank77454 +77455 POINT(41.212854264548724 -73.09680150006804) bank77455 +77456 POINT(40.286760790792236 -73.24725002890422) bank77456 +77457 POINT(41.18934688542665 -73.2372673320671) bank77457 +77458 POINT(39.81393383456038 -73.65132958459218) bank77458 +77459 POINT(39.768169599616506 -73.45696961584936) bank77459 +77460 POINT(41.47036689946813 -73.13337369624541) bank77460 +77461 POINT(41.6857392434782 -73.91905711194035) bank77461 +77462 POINT(39.90292475895576 -74.99564179948942) bank77462 +77463 POINT(41.24950008666567 -74.3213029666059) bank77463 +77464 POINT(40.97707095962542 -74.67434955735902) bank77464 +77465 POINT(40.36350171582054 -73.80951892475765) bank77465 +77466 POINT(41.42269158227044 -73.83661444201663) bank77466 +77467 POINT(40.76804777942428 -74.83837612617589) bank77467 +77468 POINT(39.93999231527128 -74.46354340100164) bank77468 +77469 POINT(40.552724074624756 -73.90367623037834) bank77469 +77470 POINT(41.44701939018863 -74.16635779490962) bank77470 +77471 POINT(40.18480232093079 -73.82254914040531) bank77471 +77472 POINT(41.56265414957032 -74.69519034275594) bank77472 +77473 POINT(40.39866292320445 -74.2398195302072) bank77473 +77474 POINT(41.25095460444493 -73.14524176738333) bank77474 +77475 POINT(40.44689172026008 -73.35523130122738) bank77475 +77476 POINT(41.31508679841508 -73.03870721627439) bank77476 +77477 POINT(41.08875560387258 -73.63373713195595) bank77477 +77478 POINT(41.68766428134702 -73.28191573271151) bank77478 +77479 POINT(40.970143193786846 -73.1857703059485) bank77479 +77480 POINT(41.6802059874402 -73.54307730459814) bank77480 +77481 POINT(39.79645959599798 -73.31495987213587) bank77481 +77482 POINT(40.00454608038235 -74.8764644779358) bank77482 +77483 POINT(40.16250312442481 -73.0501326076308) bank77483 +77484 POINT(41.17865758404378 -74.78102528804344) bank77484 +77485 POINT(40.15950860403685 -73.19076399732026) bank77485 +77486 POINT(40.11812931371213 -74.61326566268437) bank77486 +77487 POINT(41.35750390235113 -74.4477744319798) bank77487 +77488 POINT(40.30479920409175 -73.31352795296486) bank77488 +77489 POINT(39.8851698555141 -73.98668195980127) bank77489 +77490 POINT(40.23263290893964 -73.33234164611775) bank77490 +77491 POINT(41.57711188233325 -73.66899302985483) bank77491 +77492 POINT(40.15092655675651 -73.12706752664097) bank77492 +77493 POINT(40.20836704370535 -73.55282660728146) bank77493 +77494 POINT(39.87116695215601 -73.30226925340578) bank77494 +77495 POINT(40.05831466191529 -73.83200569182492) bank77495 +77496 POINT(40.648839850541336 -74.00830931226423) bank77496 +77497 POINT(40.37268473376691 -73.35914990920304) bank77497 +77498 POINT(40.79170815949618 -74.21100616573203) bank77498 +77499 POINT(40.469918311517965 -74.74119054488855) bank77499 +77500 POINT(41.40262733793732 -74.10464933682941) bank77500 +77501 POINT(41.163767054224685 -73.68697020723039) bank77501 +77502 POINT(39.990602885751485 -74.64705520982372) bank77502 +77503 POINT(40.45569706534966 -74.72096014851144) bank77503 +77504 POINT(40.129560756049955 -73.07181975023447) bank77504 +77505 POINT(39.95989496082429 -73.2767928863577) bank77505 +77506 POINT(40.749429216538225 -73.11935774181774) bank77506 +77507 POINT(40.11379026452808 -73.69602975829056) bank77507 +77508 POINT(40.96213342700008 -73.27029708846831) bank77508 +77509 POINT(41.48863678876817 -73.5468780272818) bank77509 +77510 POINT(41.156111570917666 -73.03696491276165) bank77510 +77511 POINT(39.80021291653716 -73.61216795192104) bank77511 +77512 POINT(40.713089259353595 -74.01331799258719) bank77512 +77513 POINT(39.73333615285385 -74.76328236534629) bank77513 +77514 POINT(41.40777114133523 -73.01305649568687) bank77514 +77515 POINT(40.12056536406919 -73.9774647112721) bank77515 +77516 POINT(39.91759949848071 -73.64366098291958) bank77516 +77517 POINT(40.0338765602147 -73.93405138138476) bank77517 +77518 POINT(41.03085871285202 -73.60919286473107) bank77518 +77519 POINT(41.60953414556869 -73.2321244236794) bank77519 +77520 POINT(41.27033274562429 -73.22244493401811) bank77520 +77521 POINT(40.335549406522105 -73.04969452447011) bank77521 +77522 POINT(41.15543903957871 -73.28581695798042) bank77522 +77523 POINT(40.08189501409412 -73.86706573967463) bank77523 +77524 POINT(41.397276906629806 -74.54233861883371) bank77524 +77525 POINT(41.47328523840491 -74.211017133093) bank77525 +77526 POINT(39.875218187158914 -74.78918516701772) bank77526 +77527 POINT(40.90846151091795 -74.19991002515124) bank77527 +77528 POINT(40.94778831647252 -73.53371282656585) bank77528 +77529 POINT(41.55189333864605 -73.89512691153705) bank77529 +77530 POINT(40.59688921525625 -74.20648435691203) bank77530 +77531 POINT(40.736446360518556 -73.52474959522416) bank77531 +77532 POINT(40.22754305042572 -73.49082326208372) bank77532 +77533 POINT(39.740529112273315 -74.97657653879544) bank77533 +77534 POINT(40.90281337162706 -73.03761698425512) bank77534 +77535 POINT(39.850543201901885 -73.45384563528778) bank77535 +77536 POINT(41.50253455648197 -74.1147576669764) bank77536 +77537 POINT(41.265202783583845 -73.3002649424767) bank77537 +77538 POINT(40.557425430465706 -74.36421633834155) bank77538 +77539 POINT(40.21534061459311 -74.38413294998793) bank77539 +77540 POINT(40.91906967886073 -74.04344119714212) bank77540 +77541 POINT(40.30677324215852 -73.28990769791639) bank77541 +77542 POINT(41.53318354446808 -73.0646851670928) bank77542 +77543 POINT(40.88845046753068 -73.34782943591614) bank77543 +77544 POINT(40.072992978118535 -73.99084924716385) bank77544 +77545 POINT(41.581205408199246 -74.1621630572275) bank77545 +77546 POINT(40.292240160757274 -73.27778780750195) bank77546 +77547 POINT(40.46275618757159 -73.06158856431112) bank77547 +77548 POINT(39.94688366243037 -74.68364087726589) bank77548 +77549 POINT(40.45622289623621 -73.41394919453504) bank77549 +77550 POINT(41.306674682990135 -74.67147522054319) bank77550 +77551 POINT(39.78722074176841 -74.05126806587526) bank77551 +77552 POINT(40.90264011101344 -74.00824442785095) bank77552 +77553 POINT(41.381630944719134 -74.44001135602583) bank77553 +77554 POINT(39.721545985132906 -73.92039875889074) bank77554 +77555 POINT(41.13710147330383 -73.0371875369) bank77555 +77556 POINT(39.80805968311646 -74.5046297606336) bank77556 +77557 POINT(41.095471281249516 -74.89364340929103) bank77557 +77558 POINT(40.04984616903357 -73.92845285800692) bank77558 +77559 POINT(40.537257092162214 -73.69402001220323) bank77559 +77560 POINT(41.26712712269488 -73.69969996313834) bank77560 +77561 POINT(40.603730284318765 -73.25004970070681) bank77561 +77562 POINT(41.446290354189415 -74.44849532044935) bank77562 +77563 POINT(41.24885711003741 -74.7101976671914) bank77563 +77564 POINT(40.508059168996404 -73.49758147659414) bank77564 +77565 POINT(41.275958130368906 -73.53964705561097) bank77565 +77566 POINT(40.96313355267314 -73.86731684263822) bank77566 +77567 POINT(40.0285763329154 -74.62061028467932) bank77567 +77568 POINT(40.602508364073124 -73.37786450122485) bank77568 +77569 POINT(40.12532747173135 -73.16958520295243) bank77569 +77570 POINT(40.638971667774896 -74.26362778688711) bank77570 +77571 POINT(41.41861137788838 -74.87707410348011) bank77571 +77572 POINT(40.48872230141675 -73.42583009705852) bank77572 +77573 POINT(39.87762185871061 -74.23810468765096) bank77573 +77574 POINT(39.739241769166725 -74.4497545141977) bank77574 +77575 POINT(41.64413751220634 -74.13818419448388) bank77575 +77576 POINT(41.28815437127664 -74.59485792200961) bank77576 +77577 POINT(40.111653213379725 -74.19686619491799) bank77577 +77578 POINT(40.43845179228095 -74.41677824948971) bank77578 +77579 POINT(40.62600480643424 -73.58044861004225) bank77579 +77580 POINT(41.65706845707167 -73.75470461392676) bank77580 +77581 POINT(40.64439232889142 -74.35433846522186) bank77581 +77582 POINT(39.8343286023994 -73.93652170528424) bank77582 +77583 POINT(41.407940990254374 -73.83440404642681) bank77583 +77584 POINT(39.98516696135764 -74.35571044001864) bank77584 +77585 POINT(41.41515675975585 -73.64023697080323) bank77585 +77586 POINT(41.1802407275956 -74.67603451273347) bank77586 +77587 POINT(41.178372649421085 -73.51177030083228) bank77587 +77588 POINT(40.54416162071759 -73.35927990252992) bank77588 +77589 POINT(40.89734351737121 -74.32348313679593) bank77589 +77590 POINT(40.05133252730232 -74.20313810657497) bank77590 +77591 POINT(40.57168607134303 -73.76152302321307) bank77591 +77592 POINT(40.555572245449525 -73.93071487279337) bank77592 +77593 POINT(40.01401500994553 -73.03334420230573) bank77593 +77594 POINT(41.49963614275113 -73.19033936713407) bank77594 +77595 POINT(40.829530018508294 -73.8567373982405) bank77595 +77596 POINT(41.42063592557085 -73.93918611731249) bank77596 +77597 POINT(41.07800008279545 -74.57439100827517) bank77597 +77598 POINT(40.05227781627166 -74.33167139020719) bank77598 +77599 POINT(40.128588130744575 -73.3388070613908) bank77599 +77600 POINT(40.33055666632367 -73.7583763398686) bank77600 +77601 POINT(41.20283740716138 -74.43508152494047) bank77601 +77602 POINT(40.55030262170219 -73.67677798417407) bank77602 +77603 POINT(41.05549280062215 -74.01053136382865) bank77603 +77604 POINT(40.63333531972799 -74.38611627056775) bank77604 +77605 POINT(41.323164464609306 -73.34842111893379) bank77605 +77606 POINT(40.036153512462164 -73.2240020081623) bank77606 +77607 POINT(40.44765522024881 -74.56672258584378) bank77607 +77608 POINT(41.70266839708019 -74.27514341656378) bank77608 +77609 POINT(40.91051881844622 -73.6135981258211) bank77609 +77610 POINT(40.03008864582573 -73.10760848266017) bank77610 +77611 POINT(39.96873100772624 -74.81497140191028) bank77611 +77612 POINT(40.60679673376002 -74.02831981708586) bank77612 +77613 POINT(40.06104489375813 -74.85515114049028) bank77613 +77614 POINT(41.66430578984965 -74.66816390595189) bank77614 +77615 POINT(40.21718119317934 -74.79322909614265) bank77615 +77616 POINT(40.67895035699101 -74.71765490897884) bank77616 +77617 POINT(40.9690058072124 -74.45219088743795) bank77617 +77618 POINT(39.830518085563014 -74.20568022758393) bank77618 +77619 POINT(39.733017302874536 -74.73147973917473) bank77619 +77620 POINT(40.74141809687183 -74.07155727827148) bank77620 +77621 POINT(41.56725171161979 -73.87758861239766) bank77621 +77622 POINT(40.44027085860044 -74.00635433426044) bank77622 +77623 POINT(41.57199698413127 -73.13055987663724) bank77623 +77624 POINT(41.46377209397605 -73.73863111225509) bank77624 +77625 POINT(40.98555771745072 -74.30967868973023) bank77625 +77626 POINT(39.93008405897887 -73.8936089799123) bank77626 +77627 POINT(40.88436867616571 -73.59404738151997) bank77627 +77628 POINT(39.77722779142383 -73.58370351422239) bank77628 +77629 POINT(41.43861673173984 -73.15421315015854) bank77629 +77630 POINT(40.397350694113825 -73.5533380250448) bank77630 +77631 POINT(41.567521173299944 -74.40528987224087) bank77631 +77632 POINT(40.110521572018065 -74.21610176027433) bank77632 +77633 POINT(40.99951069708308 -73.53535321489936) bank77633 +77634 POINT(41.030361482438785 -73.40443024980364) bank77634 +77635 POINT(39.83193560148026 -74.0468335161513) bank77635 +77636 POINT(40.082790211718134 -73.03972492321002) bank77636 +77637 POINT(40.92280672169005 -74.48387025891775) bank77637 +77638 POINT(40.95314135699803 -74.76997313787417) bank77638 +77639 POINT(41.339328049873146 -73.35413753805523) bank77639 +77640 POINT(41.069101081993374 -74.41277414206571) bank77640 +77641 POINT(40.420232487874515 -73.21148874086668) bank77641 +77642 POINT(41.04768232990269 -74.85824793765411) bank77642 +77643 POINT(41.44467234365186 -74.89139387203149) bank77643 +77644 POINT(40.330596392274046 -74.50358991645598) bank77644 +77645 POINT(41.480053387008944 -74.79010188766222) bank77645 +77646 POINT(39.85040565796466 -74.1829806556601) bank77646 +77647 POINT(41.18654794553412 -74.7974435720022) bank77647 +77648 POINT(41.04239410932996 -73.11060607499033) bank77648 +77649 POINT(39.812179978583885 -74.97257375217276) bank77649 +77650 POINT(40.60733261653885 -74.21806555729393) bank77650 +77651 POINT(40.29539448303753 -73.69633190596616) bank77651 +77652 POINT(40.11472228156427 -74.73910634169951) bank77652 +77653 POINT(39.928947710532555 -74.31431302392413) bank77653 +77654 POINT(41.365312823767894 -73.69177530322447) bank77654 +77655 POINT(41.43076662148559 -74.22631049499789) bank77655 +77656 POINT(40.33622911859504 -74.60487225166571) bank77656 +77657 POINT(40.09474884796583 -73.59201485599033) bank77657 +77658 POINT(40.48260424840191 -74.51479058759698) bank77658 +77659 POINT(40.382614587275846 -73.30236049258416) bank77659 +77660 POINT(40.21524268371714 -73.64205897441943) bank77660 +77661 POINT(40.03155802631689 -73.89194841291065) bank77661 +77662 POINT(40.48230432496205 -73.10514425710856) bank77662 +77663 POINT(41.62641827571002 -73.75327309564561) bank77663 +77664 POINT(40.72449738951373 -74.87107287432922) bank77664 +77665 POINT(39.926520325433536 -74.6281017197436) bank77665 +77666 POINT(41.64246456143602 -74.63935292500986) bank77666 +77667 POINT(40.975824090893376 -74.4382199148421) bank77667 +77668 POINT(41.26742250954632 -73.31906799441033) bank77668 +77669 POINT(41.38426571350238 -74.27214689059963) bank77669 +77670 POINT(39.9061751150579 -74.94618410608714) bank77670 +77671 POINT(40.14696163087089 -73.78752466760484) bank77671 +77672 POINT(41.69916098008718 -73.94100418263172) bank77672 +77673 POINT(41.3996925842972 -74.63461966768789) bank77673 +77674 POINT(40.945329638590685 -74.51657630161748) bank77674 +77675 POINT(40.48563205417216 -74.04745753392393) bank77675 +77676 POINT(40.55596325334412 -74.19654075869613) bank77676 +77677 POINT(40.9572673920767 -73.47814378871874) bank77677 +77678 POINT(41.52924867248672 -73.81098855793196) bank77678 +77679 POINT(41.090733874593674 -74.18454547662772) bank77679 +77680 POINT(39.90639581199879 -73.30214499690138) bank77680 +77681 POINT(41.432950623300286 -73.86901070373638) bank77681 +77682 POINT(40.082054234493974 -74.79793272795878) bank77682 +77683 POINT(40.57500759487187 -73.20693417882721) bank77683 +77684 POINT(40.503257534972356 -74.58056589489406) bank77684 +77685 POINT(40.94357831149935 -73.95956719207486) bank77685 +77686 POINT(41.53326233725415 -73.02079320697538) bank77686 +77687 POINT(41.37542492346603 -73.64975245041752) bank77687 +77688 POINT(41.63466130424128 -74.01777160734278) bank77688 +77689 POINT(41.28765812312647 -74.75466373742937) bank77689 +77690 POINT(39.93828195451243 -73.69478317764222) bank77690 +77691 POINT(41.10887503436676 -74.20332035055884) bank77691 +77692 POINT(40.73858447455171 -74.16966032188644) bank77692 +77693 POINT(41.468300185666926 -74.97562065555702) bank77693 +77694 POINT(41.30335788953049 -74.46716383772936) bank77694 +77695 POINT(41.165173321469425 -74.78748950628267) bank77695 +77696 POINT(40.818086099662956 -74.70307948582634) bank77696 +77697 POINT(39.83940359786675 -74.20499221142244) bank77697 +77698 POINT(40.54100428717551 -73.49973546835868) bank77698 +77699 POINT(41.388962036934196 -74.86628864442308) bank77699 +77700 POINT(40.453069928442794 -73.92225481100648) bank77700 +77701 POINT(40.20913714050008 -74.15208479067918) bank77701 +77702 POINT(41.501059349926145 -74.62994223938755) bank77702 +77703 POINT(40.277606058591054 -73.41625409205136) bank77703 +77704 POINT(40.946786367987066 -74.21914355650722) bank77704 +77705 POINT(40.70508025624263 -73.59632176393578) bank77705 +77706 POINT(41.08199702578892 -73.59453124157844) bank77706 +77707 POINT(39.9879028054913 -74.80060859900449) bank77707 +77708 POINT(40.54521505967947 -74.37831784313859) bank77708 +77709 POINT(39.78299432435973 -73.12980789632739) bank77709 +77710 POINT(41.62391025763688 -74.8934495893087) bank77710 +77711 POINT(40.618468668293914 -73.986539491229) bank77711 +77712 POINT(40.78474077086054 -73.15184202232084) bank77712 +77713 POINT(41.57462522705189 -73.84245025526573) bank77713 +77714 POINT(40.99883199074582 -73.43345450712468) bank77714 +77715 POINT(40.64106445264624 -73.43246327311516) bank77715 +77716 POINT(40.190942880391304 -74.54059476733265) bank77716 +77717 POINT(40.51367389721077 -74.64870818364925) bank77717 +77718 POINT(40.44633740329059 -74.37400691932203) bank77718 +77719 POINT(40.97555099394148 -74.56769601416042) bank77719 +77720 POINT(40.62436820040917 -74.27106291041991) bank77720 +77721 POINT(40.686191093661115 -74.75093419216482) bank77721 +77722 POINT(41.40556803103345 -73.32060125896113) bank77722 +77723 POINT(40.609968270289926 -74.87539447697634) bank77723 +77724 POINT(41.547171242407835 -73.49963644076406) bank77724 +77725 POINT(41.36572144555749 -73.56141582895218) bank77725 +77726 POINT(41.632325755226475 -73.35365315587161) bank77726 +77727 POINT(41.08871407626619 -73.93200520948643) bank77727 +77728 POINT(41.17002214202468 -74.22039868254937) bank77728 +77729 POINT(40.83894262112133 -73.70625568774996) bank77729 +77730 POINT(40.979219640007635 -73.1665540777285) bank77730 +77731 POINT(41.44789746938899 -73.57273485748192) bank77731 +77732 POINT(41.015247477520354 -74.31388857519616) bank77732 +77733 POINT(40.11193422115568 -74.2591757210238) bank77733 +77734 POINT(39.76864361966931 -73.30386231572446) bank77734 +77735 POINT(40.84271620296658 -74.19417097361541) bank77735 +77736 POINT(39.95844738401259 -73.5775922883535) bank77736 +77737 POINT(40.440671575433335 -73.67890256809864) bank77737 +77738 POINT(40.296165298913266 -73.6314428412069) bank77738 +77739 POINT(41.118357775384695 -74.54269209282911) bank77739 +77740 POINT(41.2996987500049 -73.79860731444141) bank77740 +77741 POINT(40.33243765773077 -73.46660860000601) bank77741 +77742 POINT(40.6438655823995 -74.68291576164881) bank77742 +77743 POINT(39.9458122184323 -73.95873086875457) bank77743 +77744 POINT(41.335642504598574 -73.26144526558792) bank77744 +77745 POINT(40.7990286689834 -74.7104491353792) bank77745 +77746 POINT(40.84394256335372 -74.6991987200728) bank77746 +77747 POINT(40.166736830345705 -73.40259951123411) bank77747 +77748 POINT(40.05080163750807 -74.01975606630897) bank77748 +77749 POINT(39.9272149721634 -74.54696745676027) bank77749 +77750 POINT(40.749849073854165 -74.66541386398514) bank77750 +77751 POINT(39.731058851676764 -74.46873326698851) bank77751 +77752 POINT(40.023337186829416 -74.0325330555626) bank77752 +77753 POINT(40.25460722616502 -74.58627720405299) bank77753 +77754 POINT(41.58514318287137 -74.29817681905355) bank77754 +77755 POINT(40.95574306566405 -73.09922773400831) bank77755 +77756 POINT(40.223167484906284 -74.25984944613553) bank77756 +77757 POINT(40.961509829529035 -73.48339653118312) bank77757 +77758 POINT(41.494202245866994 -74.03707478712553) bank77758 +77759 POINT(41.67077147005937 -74.33064356266013) bank77759 +77760 POINT(39.713436248080804 -74.82377946095095) bank77760 +77761 POINT(41.016792056176286 -74.51861787824078) bank77761 +77762 POINT(40.97578060258872 -73.94925919537633) bank77762 +77763 POINT(40.576479265632116 -74.81083043370225) bank77763 +77764 POINT(41.33345223231277 -73.54325170239692) bank77764 +77765 POINT(40.56794873388667 -74.25373205406238) bank77765 +77766 POINT(41.04089766261195 -74.62432392915183) bank77766 +77767 POINT(41.10165123468227 -74.71460708410355) bank77767 +77768 POINT(41.23713428784726 -74.13211497811831) bank77768 +77769 POINT(39.97028354807188 -74.15328875095756) bank77769 +77770 POINT(41.47098529012829 -74.35847057328759) bank77770 +77771 POINT(41.63614893836611 -74.32666041120945) bank77771 +77772 POINT(40.61611978669343 -73.81900160809258) bank77772 +77773 POINT(40.513971600742494 -74.90832543482242) bank77773 +77774 POINT(41.56032913212253 -74.02900336297644) bank77774 +77775 POINT(41.43339445649993 -74.17626662973183) bank77775 +77776 POINT(40.31537690169706 -73.23291512799368) bank77776 +77777 POINT(40.30559274230328 -74.39649308469015) bank77777 +77778 POINT(39.74542682828226 -73.7815634115785) bank77778 +77779 POINT(40.28536396850702 -73.9514106607578) bank77779 +77780 POINT(40.41808757852842 -74.92383582740752) bank77780 +77781 POINT(41.55168407343943 -74.78318882160205) bank77781 +77782 POINT(41.49662109436641 -73.82723650427803) bank77782 +77783 POINT(39.95411622645209 -74.28725437961258) bank77783 +77784 POINT(40.972324173557254 -74.61287820860515) bank77784 +77785 POINT(40.55472232615623 -73.9855355948004) bank77785 +77786 POINT(39.713684864592096 -73.22634458831183) bank77786 +77787 POINT(40.41224393244582 -73.6708640254162) bank77787 +77788 POINT(40.044386217686615 -74.939138304363) bank77788 +77789 POINT(41.64505608718626 -74.27872699934451) bank77789 +77790 POINT(40.59170333220605 -73.60825330790745) bank77790 +77791 POINT(41.53987418952651 -73.65762363421436) bank77791 +77792 POINT(40.49374002529374 -74.88307267882924) bank77792 +77793 POINT(40.731844260313636 -73.95107671045194) bank77793 +77794 POINT(39.79538697364703 -73.56302154067724) bank77794 +77795 POINT(40.87653150599392 -74.58996970613708) bank77795 +77796 POINT(41.15002294043834 -73.55168418464041) bank77796 +77797 POINT(40.85619300203271 -73.4985256944434) bank77797 +77798 POINT(40.66642202374939 -74.71429227347947) bank77798 +77799 POINT(39.94170632934569 -74.13180150574526) bank77799 +77800 POINT(41.708600159293844 -74.46405835034173) bank77800 +77801 POINT(41.17427255403933 -73.43324901892021) bank77801 +77802 POINT(41.108657722584624 -73.07049584740827) bank77802 +77803 POINT(40.639170298613514 -73.48970945834307) bank77803 +77804 POINT(40.01958309236824 -74.71591904119765) bank77804 +77805 POINT(41.397580246150824 -73.74391707086947) bank77805 +77806 POINT(41.533345627153246 -74.7256291049545) bank77806 +77807 POINT(39.8133824535078 -73.24554852032794) bank77807 +77808 POINT(41.37925742896368 -73.35117324277974) bank77808 +77809 POINT(40.026072431388386 -73.35528200333917) bank77809 +77810 POINT(40.83493559327679 -73.02495949200326) bank77810 +77811 POINT(40.74860702077695 -73.50879541535224) bank77811 +77812 POINT(40.09918867399202 -73.32777039060937) bank77812 +77813 POINT(40.03613927957614 -73.90937898757574) bank77813 +77814 POINT(40.872849736184804 -73.11487539032467) bank77814 +77815 POINT(40.76291717821009 -74.26360356510929) bank77815 +77816 POINT(41.422197661460714 -74.21997405080155) bank77816 +77817 POINT(40.00014615639449 -74.25632327969468) bank77817 +77818 POINT(41.38580636286868 -74.80037359863888) bank77818 +77819 POINT(40.414937186689734 -74.32548238545412) bank77819 +77820 POINT(40.718726758467234 -73.81386593660382) bank77820 +77821 POINT(39.85288808484441 -74.71761599046488) bank77821 +77822 POINT(40.298302159999174 -73.15650339596084) bank77822 +77823 POINT(41.62154846269037 -73.88682837976741) bank77823 +77824 POINT(41.34587633803013 -73.30510376365602) bank77824 +77825 POINT(40.090600403424844 -73.92342075264364) bank77825 +77826 POINT(41.200567522488825 -73.98266368168517) bank77826 +77827 POINT(40.69419203669465 -74.88670766391658) bank77827 +77828 POINT(40.3045821130066 -73.39515530358496) bank77828 +77829 POINT(40.07946846803714 -73.52526586601168) bank77829 +77830 POINT(40.13328252671088 -74.79207828221321) bank77830 +77831 POINT(40.8265011320345 -73.44668399769483) bank77831 +77832 POINT(40.92971001176233 -73.62466434345767) bank77832 +77833 POINT(40.22209487283384 -73.5095553399783) bank77833 +77834 POINT(40.228655188964616 -73.57945352929754) bank77834 +77835 POINT(40.261666927323226 -73.31866249295555) bank77835 +77836 POINT(41.43719992940633 -74.70689407403951) bank77836 +77837 POINT(40.363314041933265 -74.13006605877244) bank77837 +77838 POINT(40.83600426440761 -73.3974906690007) bank77838 +77839 POINT(41.25008829534597 -74.47324709533642) bank77839 +77840 POINT(40.3016918431423 -73.53869484498891) bank77840 +77841 POINT(40.52276059385902 -73.16656852979126) bank77841 +77842 POINT(41.13048777897956 -74.57403412137694) bank77842 +77843 POINT(40.81979354404491 -74.10314829236292) bank77843 +77844 POINT(39.90199540855877 -74.50358343795783) bank77844 +77845 POINT(40.2203714156657 -73.65806708775727) bank77845 +77846 POINT(40.695655924320484 -73.8864694273436) bank77846 +77847 POINT(39.97095701629538 -73.89559942836269) bank77847 +77848 POINT(41.27210076200996 -74.13813691912524) bank77848 +77849 POINT(40.10272193576753 -74.00316653653462) bank77849 +77850 POINT(40.73302987643729 -73.7280516451269) bank77850 +77851 POINT(39.94582301123119 -73.5546417354289) bank77851 +77852 POINT(40.674748207512096 -74.44500449241028) bank77852 +77853 POINT(40.99358272464925 -74.36463350067929) bank77853 +77854 POINT(41.470110286248115 -73.22451274330516) bank77854 +77855 POINT(41.416617359378726 -73.41298378704633) bank77855 +77856 POINT(41.0881726958767 -73.28412754828261) bank77856 +77857 POINT(40.308813334351484 -73.37374990213125) bank77857 +77858 POINT(40.05990588677055 -73.89443784576959) bank77858 +77859 POINT(40.105669396067285 -74.98109919998655) bank77859 +77860 POINT(39.757326649170714 -74.51718201177968) bank77860 +77861 POINT(41.373328439010216 -74.08634040299772) bank77861 +77862 POINT(40.68390077855205 -74.56784019758655) bank77862 +77863 POINT(41.5196798483599 -73.29576587764095) bank77863 +77864 POINT(41.255371700525686 -73.2074802077457) bank77864 +77865 POINT(40.33651589410013 -73.94877387052823) bank77865 +77866 POINT(40.61831514777106 -73.52093260921914) bank77866 +77867 POINT(41.02173806841244 -73.49647242920595) bank77867 +77868 POINT(40.469782532248345 -74.81617628276547) bank77868 +77869 POINT(41.19066595271349 -74.98718908139762) bank77869 +77870 POINT(40.28930253155366 -74.6423468362343) bank77870 +77871 POINT(40.433958662152826 -73.04532557256302) bank77871 +77872 POINT(41.28574892899084 -74.4583421374816) bank77872 +77873 POINT(40.82765412262894 -74.623512859157) bank77873 +77874 POINT(40.38433196303839 -74.6132491265032) bank77874 +77875 POINT(39.88079047774403 -73.5041530400962) bank77875 +77876 POINT(40.4105774195666 -74.73665370831621) bank77876 +77877 POINT(40.52671458892843 -74.861665614915) bank77877 +77878 POINT(40.87421676306629 -73.68739171297032) bank77878 +77879 POINT(40.56269173000085 -74.16673539749047) bank77879 +77880 POINT(41.40255392918647 -74.91675334009474) bank77880 +77881 POINT(39.95636613836561 -74.321179772132) bank77881 +77882 POINT(41.593111888484266 -73.97878475679113) bank77882 +77883 POINT(41.19960674194011 -73.7919051745938) bank77883 +77884 POINT(40.19096986284748 -74.43432629707306) bank77884 +77885 POINT(39.728586457580356 -73.62446618201986) bank77885 +77886 POINT(41.13608107277267 -74.41986051120118) bank77886 +77887 POINT(40.88277578743836 -73.62207864571478) bank77887 +77888 POINT(40.399421211603396 -73.2082067734858) bank77888 +77889 POINT(40.615801215314335 -73.81967124756842) bank77889 +77890 POINT(40.748514437107815 -73.754880345184) bank77890 +77891 POINT(41.39881109828202 -74.60856281539361) bank77891 +77892 POINT(40.626426934139516 -74.56072333858386) bank77892 +77893 POINT(40.02837024930795 -73.35790888072451) bank77893 +77894 POINT(40.16713430373781 -73.89938437630454) bank77894 +77895 POINT(41.54036867856793 -74.6151704869605) bank77895 +77896 POINT(40.160741035222 -73.23952886003252) bank77896 +77897 POINT(40.959243639883006 -73.36438971899264) bank77897 +77898 POINT(40.91459423069393 -74.59348817556993) bank77898 +77899 POINT(39.99147200319055 -74.16171717914176) bank77899 +77900 POINT(39.904189619884264 -73.94471750861251) bank77900 +77901 POINT(41.41105190837377 -74.6403179573485) bank77901 +77902 POINT(40.88471428245805 -73.48928260655296) bank77902 +77903 POINT(39.85968998866803 -73.5772288048012) bank77903 +77904 POINT(40.74633842629851 -73.05141496899573) bank77904 +77905 POINT(39.98688200943117 -73.01416628537936) bank77905 +77906 POINT(40.15925541410581 -73.44659517400339) bank77906 +77907 POINT(39.97951854387546 -73.72767124343648) bank77907 +77908 POINT(41.247362279447294 -73.5521023156748) bank77908 +77909 POINT(41.71124128367968 -73.74867634157782) bank77909 +77910 POINT(39.934812863669286 -73.96969775301997) bank77910 +77911 POINT(40.69229848807533 -73.14414213566519) bank77911 +77912 POINT(41.627465529038865 -73.12857025384086) bank77912 +77913 POINT(40.672091531979405 -74.25308978325369) bank77913 +77914 POINT(40.61563135727296 -74.49473139875737) bank77914 +77915 POINT(41.14163391023652 -73.27409829846516) bank77915 +77916 POINT(40.545935284982036 -73.30754593945598) bank77916 +77917 POINT(40.48408725419377 -74.40425888798488) bank77917 +77918 POINT(39.891327404242375 -73.43976779395925) bank77918 +77919 POINT(41.28907784932534 -74.99186108647727) bank77919 +77920 POINT(40.841842364372404 -73.17626070168998) bank77920 +77921 POINT(40.17628027232574 -73.06128897251534) bank77921 +77922 POINT(39.90280690386374 -74.39245761545239) bank77922 +77923 POINT(39.83779685752047 -73.36035676840348) bank77923 +77924 POINT(41.70130591322213 -73.9253087296502) bank77924 +77925 POINT(41.193342629515726 -73.82139291302218) bank77925 +77926 POINT(40.66679072814724 -74.53305171234726) bank77926 +77927 POINT(41.19618332267934 -74.34967888320561) bank77927 +77928 POINT(41.35180831833982 -73.57858072218916) bank77928 +77929 POINT(40.85712460558778 -73.88270815136143) bank77929 +77930 POINT(41.64093105618358 -73.5995691889036) bank77930 +77931 POINT(40.6421408677426 -74.56297286518759) bank77931 +77932 POINT(41.20526889121923 -74.66207431202828) bank77932 +77933 POINT(40.15920436855217 -73.82210774293858) bank77933 +77934 POINT(40.27027941077078 -74.57376675105826) bank77934 +77935 POINT(41.65543928770785 -73.57533824064791) bank77935 +77936 POINT(40.440695590858375 -74.79514391466131) bank77936 +77937 POINT(40.88441627006139 -73.28836059291773) bank77937 +77938 POINT(40.20866203846964 -73.38176181956965) bank77938 +77939 POINT(40.92878350055009 -73.83649805497909) bank77939 +77940 POINT(40.46234130481599 -73.4293004518356) bank77940 +77941 POINT(40.24243227534234 -73.79911469605821) bank77941 +77942 POINT(39.790343419044625 -73.67212535448292) bank77942 +77943 POINT(41.205129710302636 -74.97960523407762) bank77943 +77944 POINT(40.90809610476026 -73.35651899973193) bank77944 +77945 POINT(40.09074803146153 -73.20901041446601) bank77945 +77946 POINT(41.426525392548335 -74.61746909364768) bank77946 +77947 POINT(41.6636692507038 -74.02926498836767) bank77947 +77948 POINT(40.78571235499356 -74.25493216941481) bank77948 +77949 POINT(40.816375605463264 -73.76385527406558) bank77949 +77950 POINT(40.07115611175441 -73.47472268870018) bank77950 +77951 POINT(41.6001882042839 -74.98633789226892) bank77951 +77952 POINT(40.798755882454195 -73.40846507498931) bank77952 +77953 POINT(40.03769370386492 -73.2419780901914) bank77953 +77954 POINT(40.20877128907482 -74.04215046027541) bank77954 +77955 POINT(39.7318058027293 -73.62154994428965) bank77955 +77956 POINT(41.12725149691869 -73.20849378535854) bank77956 +77957 POINT(40.88261497017455 -74.58212521094657) bank77957 +77958 POINT(40.10418189234611 -73.78472149464402) bank77958 +77959 POINT(41.197625021605084 -73.74189078334247) bank77959 +77960 POINT(40.963022235008616 -74.28066350227458) bank77960 +77961 POINT(40.84423448664004 -73.14704734630236) bank77961 +77962 POINT(41.278552873314815 -73.62589340136462) bank77962 +77963 POINT(39.92860371897996 -73.18382083687155) bank77963 +77964 POINT(40.38875088475016 -73.07010218844454) bank77964 +77965 POINT(40.321408497160064 -73.03330183225816) bank77965 +77966 POINT(41.2721605969777 -74.07929433802838) bank77966 +77967 POINT(40.36447710603306 -74.52378265725834) bank77967 +77968 POINT(41.20126482979525 -73.64270279486122) bank77968 +77969 POINT(39.77144317277829 -74.76281492653959) bank77969 +77970 POINT(39.793483354806035 -73.71794110484204) bank77970 +77971 POINT(40.23438362919636 -74.00124425996744) bank77971 +77972 POINT(40.78305356387731 -73.24734885620188) bank77972 +77973 POINT(41.0957745943742 -74.01498540583772) bank77973 +77974 POINT(39.99855203512 -74.76252944118076) bank77974 +77975 POINT(41.679261321584555 -74.26144550951103) bank77975 +77976 POINT(41.26888044158268 -74.45122337399337) bank77976 +77977 POINT(41.46598923474552 -74.20795529331104) bank77977 +77978 POINT(41.710554075282005 -74.05258207850356) bank77978 +77979 POINT(41.28617574539025 -73.53143696441784) bank77979 +77980 POINT(40.947859560955834 -74.25810366468964) bank77980 +77981 POINT(40.33125956855582 -74.34738615333958) bank77981 +77982 POINT(39.83847599543376 -74.93223327526478) bank77982 +77983 POINT(40.4678315580225 -73.22432367733886) bank77983 +77984 POINT(41.038235978668816 -74.05073962628983) bank77984 +77985 POINT(41.485846765784714 -74.33882049370763) bank77985 +77986 POINT(41.43693662877269 -74.15314325387635) bank77986 +77987 POINT(39.81434701735525 -73.87565052679965) bank77987 +77988 POINT(41.4473376064865 -74.20198877713821) bank77988 +77989 POINT(40.106012110829724 -73.6403092408295) bank77989 +77990 POINT(40.813785339506396 -73.52209157797796) bank77990 +77991 POINT(40.30329321843545 -74.84087542081478) bank77991 +77992 POINT(40.91225718660933 -73.13742133754812) bank77992 +77993 POINT(40.060933667260116 -73.48337273482343) bank77993 +77994 POINT(40.78523090479956 -73.7418629206694) bank77994 +77995 POINT(41.279297662478086 -73.65044992645218) bank77995 +77996 POINT(41.548382792976064 -73.37439406225164) bank77996 +77997 POINT(40.608268035242865 -73.71317843766778) bank77997 +77998 POINT(39.84897824623684 -74.76967299493481) bank77998 +77999 POINT(40.654969429713574 -74.04189323103925) bank77999 +78000 POINT(40.29844100122776 -73.38588376796787) bank78000 +78001 POINT(40.979063804825486 -73.08375956595593) bank78001 +78002 POINT(41.4497987228554 -74.12570063778878) bank78002 +78003 POINT(40.28474445914903 -74.35895074391499) bank78003 +78004 POINT(40.34823883913173 -74.08552785702507) bank78004 +78005 POINT(39.90736109311971 -73.66606201734136) bank78005 +78006 POINT(40.43507165528009 -74.47349014739835) bank78006 +78007 POINT(41.58698317760411 -74.74734408853213) bank78007 +78008 POINT(39.80126752613118 -74.44780801625926) bank78008 +78009 POINT(41.65134655164593 -73.14438409080212) bank78009 +78010 POINT(40.983915856605265 -73.44980461089533) bank78010 +78011 POINT(40.10547470209219 -73.66745366312526) bank78011 +78012 POINT(41.577884530294455 -74.18034189195616) bank78012 +78013 POINT(41.31279276069613 -74.59389819565132) bank78013 +78014 POINT(41.37824942877233 -73.21927390925522) bank78014 +78015 POINT(40.00432971489325 -73.56385223492818) bank78015 +78016 POINT(40.36403180699219 -74.09701623544055) bank78016 +78017 POINT(40.75570622585851 -73.75685365963682) bank78017 +78018 POINT(40.13506187241388 -73.90277209913545) bank78018 +78019 POINT(40.672754861360175 -73.19062612870732) bank78019 +78020 POINT(40.36998958471603 -73.6858110159582) bank78020 +78021 POINT(40.17124083788304 -73.32650979611911) bank78021 +78022 POINT(40.51851484316112 -73.69281848819644) bank78022 +78023 POINT(41.45875986667693 -73.83815173215646) bank78023 +78024 POINT(40.46350611670578 -74.33292051977823) bank78024 +78025 POINT(40.12619109310371 -74.33662710756295) bank78025 +78026 POINT(39.893669399783086 -74.826119080976) bank78026 +78027 POINT(41.44625968012925 -74.39070872748962) bank78027 +78028 POINT(39.830647803956964 -73.9021028002117) bank78028 +78029 POINT(39.8916376447573 -73.05669934577763) bank78029 +78030 POINT(41.083381171542165 -74.21082217898976) bank78030 +78031 POINT(40.45766911454575 -73.86607614473213) bank78031 +78032 POINT(40.373992661187806 -73.38503723397598) bank78032 +78033 POINT(41.49358041006329 -74.5820585043392) bank78033 +78034 POINT(40.70843809272161 -74.02223167432751) bank78034 +78035 POINT(40.53553985788681 -74.5704587579888) bank78035 +78036 POINT(41.50143724369014 -74.89385490730663) bank78036 +78037 POINT(41.00856721697245 -74.12695032001047) bank78037 +78038 POINT(40.17441500396184 -73.74789588717073) bank78038 +78039 POINT(40.25590494545094 -74.97614455971869) bank78039 +78040 POINT(41.18050079618743 -73.85028949463214) bank78040 +78041 POINT(40.309520583057605 -73.59043410517101) bank78041 +78042 POINT(41.14488366421223 -74.80539522238055) bank78042 +78043 POINT(40.917750628329024 -74.67369670326966) bank78043 +78044 POINT(40.751134202629615 -74.43506115048302) bank78044 +78045 POINT(40.38416081035512 -73.6832515286651) bank78045 +78046 POINT(40.889245852129875 -73.32065452598394) bank78046 +78047 POINT(41.22700086650201 -73.41996094057795) bank78047 +78048 POINT(39.77371290199739 -73.049721157632) bank78048 +78049 POINT(40.15381031929472 -73.39429877017942) bank78049 +78050 POINT(39.73682103393642 -73.20696391999397) bank78050 +78051 POINT(41.703399254786035 -73.0444804049673) bank78051 +78052 POINT(40.1865724260724 -74.42686862295146) bank78052 +78053 POINT(40.6001484161554 -74.47022587180352) bank78053 +78054 POINT(39.80798602907571 -73.42530950242946) bank78054 +78055 POINT(40.317067814943286 -73.60888850156114) bank78055 +78056 POINT(39.99508645011026 -74.18955024849392) bank78056 +78057 POINT(41.5941598211795 -73.44509655804679) bank78057 +78058 POINT(40.617769359381654 -74.4954492053915) bank78058 +78059 POINT(41.018802023680195 -73.13857878638963) bank78059 +78060 POINT(40.36750579148058 -73.56761836089048) bank78060 +78061 POINT(40.321892723319344 -74.37865456478356) bank78061 +78062 POINT(40.10534500672667 -74.08609062897095) bank78062 +78063 POINT(41.194025478881635 -74.64745249313343) bank78063 +78064 POINT(40.47212960566468 -74.88480203672871) bank78064 +78065 POINT(40.139377444807096 -74.2029186810019) bank78065 +78066 POINT(40.926659268576564 -73.54918761136243) bank78066 +78067 POINT(41.200292398276844 -74.2383351252104) bank78067 +78068 POINT(40.02445605124524 -74.78266001731336) bank78068 +78069 POINT(41.1874502219137 -74.80771978497893) bank78069 +78070 POINT(39.71716078423421 -74.34731899226585) bank78070 +78071 POINT(40.831726529093764 -73.84717752519494) bank78071 +78072 POINT(41.65800729691507 -74.92916142815963) bank78072 +78073 POINT(40.5351439922424 -74.39916032831033) bank78073 +78074 POINT(41.113462232688086 -73.59581756134905) bank78074 +78075 POINT(41.414892226209844 -73.9849584597073) bank78075 +78076 POINT(40.57931425298786 -74.94011976647356) bank78076 +78077 POINT(41.597458663304 -74.62252989310863) bank78077 +78078 POINT(40.49267347060145 -73.26028379146874) bank78078 +78079 POINT(40.45494792634502 -73.58626388058086) bank78079 +78080 POINT(41.66467361092448 -74.39740541908837) bank78080 +78081 POINT(40.988865678353996 -73.33291631835341) bank78081 +78082 POINT(41.266329442401194 -73.26464770454531) bank78082 +78083 POINT(41.43066283491977 -73.51477121953099) bank78083 +78084 POINT(40.337101888310016 -73.9436414108303) bank78084 +78085 POINT(41.37467905432567 -73.02118619681139) bank78085 +78086 POINT(40.89845132015553 -73.87792271299976) bank78086 +78087 POINT(41.196392642250444 -74.67865780204934) bank78087 +78088 POINT(40.03701371770872 -74.64592033869327) bank78088 +78089 POINT(41.6582093970119 -73.37511161325806) bank78089 +78090 POINT(39.87895929546538 -74.95019597855192) bank78090 +78091 POINT(40.27458119988665 -73.39761389275728) bank78091 +78092 POINT(41.32937523815693 -73.24989439335957) bank78092 +78093 POINT(41.15093156080894 -74.79203316801245) bank78093 +78094 POINT(40.93715882526565 -74.52243593141938) bank78094 +78095 POINT(40.630114221354994 -73.39985114440432) bank78095 +78096 POINT(39.9721080600441 -74.14615606044167) bank78096 +78097 POINT(40.86069783460031 -73.19331647203147) bank78097 +78098 POINT(40.00592148820648 -73.3606468906039) bank78098 +78099 POINT(40.7573907843569 -73.73772550809738) bank78099 +78100 POINT(40.73113087843731 -73.09099696040056) bank78100 +78101 POINT(41.59769890027779 -74.04222177242701) bank78101 +78102 POINT(40.79421808507658 -74.4341573217091) bank78102 +78103 POINT(40.95959722138097 -74.34733818044698) bank78103 +78104 POINT(41.475640064883144 -73.7441218478526) bank78104 +78105 POINT(40.17834453652494 -73.5244159046446) bank78105 +78106 POINT(40.62433131842805 -73.18233514645914) bank78106 +78107 POINT(40.190362463595925 -73.63694123963) bank78107 +78108 POINT(40.63837080348827 -74.44975672413759) bank78108 +78109 POINT(39.9142433963933 -74.5980315468657) bank78109 +78110 POINT(40.12900487457724 -74.60761216757767) bank78110 +78111 POINT(41.602764829899606 -73.33041772370683) bank78111 +78112 POINT(40.086533279619694 -73.90528918005971) bank78112 +78113 POINT(40.929950695687786 -73.42684790806902) bank78113 +78114 POINT(41.631281300713326 -74.51599997602221) bank78114 +78115 POINT(39.80630394082196 -73.69665670544241) bank78115 +78116 POINT(40.9120580433231 -73.591573100974) bank78116 +78117 POINT(41.28886057208542 -73.72555712257837) bank78117 +78118 POINT(40.05121938140268 -74.65638015504786) bank78118 +78119 POINT(41.353885495955176 -73.84928388012918) bank78119 +78120 POINT(39.82683191193071 -74.76608356940716) bank78120 +78121 POINT(40.74690849696951 -74.2749541474631) bank78121 +78122 POINT(40.58371796748365 -73.39647219165086) bank78122 +78123 POINT(41.27677047517996 -74.08495426616732) bank78123 +78124 POINT(40.83409872975984 -74.67072810813416) bank78124 +78125 POINT(40.87908758747732 -74.04258118779073) bank78125 +78126 POINT(40.05252115304491 -74.6050223043231) bank78126 +78127 POINT(40.18572790264836 -74.88106343600613) bank78127 +78128 POINT(41.51689837300284 -73.89381571408423) bank78128 +78129 POINT(40.78955254998904 -74.07057383784539) bank78129 +78130 POINT(40.07220311632657 -74.97005826528459) bank78130 +78131 POINT(40.85599901090129 -74.04351746383917) bank78131 +78132 POINT(40.936003054838224 -73.48635027604901) bank78132 +78133 POINT(40.443541167024385 -74.99469540701647) bank78133 +78134 POINT(40.326465655159865 -74.25375636570503) bank78134 +78135 POINT(41.17277777749873 -73.11226373613468) bank78135 +78136 POINT(41.252350528195 -74.77334837057333) bank78136 +78137 POINT(41.573248992027374 -74.10542913362762) bank78137 +78138 POINT(40.46870519561997 -74.53529194850341) bank78138 +78139 POINT(39.9254756172427 -73.67234528643269) bank78139 +78140 POINT(40.14383354041012 -74.95701690579274) bank78140 +78141 POINT(40.2995669587805 -73.56552139026789) bank78141 +78142 POINT(41.04980208515208 -74.60045626249807) bank78142 +78143 POINT(41.17871501923643 -74.4878576797799) bank78143 +78144 POINT(39.835022915942545 -73.39959238996862) bank78144 +78145 POINT(41.33629309017848 -74.37391880350711) bank78145 +78146 POINT(40.19351646002519 -73.10097654451842) bank78146 +78147 POINT(40.23158390643559 -74.80202573633757) bank78147 +78148 POINT(41.246260321965856 -73.6136607229653) bank78148 +78149 POINT(39.91006509246358 -74.45758754308612) bank78149 +78150 POINT(40.197919206369264 -73.76730542594532) bank78150 +78151 POINT(41.65706978346374 -73.38888227833718) bank78151 +78152 POINT(39.83694673266891 -74.72873093405661) bank78152 +78153 POINT(40.43333883699963 -73.89716843512348) bank78153 +78154 POINT(40.21536677518441 -73.09412442639746) bank78154 +78155 POINT(41.61734547536622 -73.7749856951113) bank78155 +78156 POINT(41.4388692419989 -74.85718127994494) bank78156 +78157 POINT(41.03671418483186 -73.70426868952363) bank78157 +78158 POINT(40.95263992973514 -74.77611146436486) bank78158 +78159 POINT(41.06382574835272 -73.99742100707182) bank78159 +78160 POINT(41.184349959303034 -73.4967481454752) bank78160 +78161 POINT(41.242721839769374 -73.4990498934529) bank78161 +78162 POINT(41.46611876940452 -73.43694385544819) bank78162 +78163 POINT(40.67658514024254 -74.83973996249925) bank78163 +78164 POINT(40.03292524848075 -74.59766029555824) bank78164 +78165 POINT(39.750122182132095 -74.05695046965808) bank78165 +78166 POINT(40.77695977155966 -75.00026708572165) bank78166 +78167 POINT(41.704322890857384 -73.79729886450986) bank78167 +78168 POINT(40.35172158300702 -74.2293312063908) bank78168 +78169 POINT(39.82224499177875 -73.90108264488258) bank78169 +78170 POINT(41.255324551440324 -74.27589086697886) bank78170 +78171 POINT(40.62884366586756 -73.31824149619761) bank78171 +78172 POINT(40.07757132715371 -73.35921528507112) bank78172 +78173 POINT(40.11051435448542 -73.86248765001699) bank78173 +78174 POINT(41.28328799899885 -74.27711384798083) bank78174 +78175 POINT(41.22460573845956 -74.47693009629693) bank78175 +78176 POINT(40.43138041006684 -73.53262888515772) bank78176 +78177 POINT(40.940613999641684 -73.75080054190634) bank78177 +78178 POINT(41.10094414605262 -74.8582638383364) bank78178 +78179 POINT(39.925227920107716 -73.07360932316341) bank78179 +78180 POINT(41.190991308133675 -73.17527295137738) bank78180 +78181 POINT(41.64473642498653 -73.264969271496) bank78181 +78182 POINT(41.234918064896135 -74.10486612378045) bank78182 +78183 POINT(40.99625554386782 -74.26794925131726) bank78183 +78184 POINT(40.452645985852975 -73.07489120652068) bank78184 +78185 POINT(40.86930292682287 -74.40654275385019) bank78185 +78186 POINT(40.70416764151827 -73.70752998523854) bank78186 +78187 POINT(39.71962881711691 -74.59470265272296) bank78187 +78188 POINT(40.29321427040424 -74.78017675699182) bank78188 +78189 POINT(40.27011870123026 -73.90304800753941) bank78189 +78190 POINT(40.16967372094629 -73.1299655442263) bank78190 +78191 POINT(41.321179464607475 -73.28190423304513) bank78191 +78192 POINT(41.28405082391014 -73.65199277522852) bank78192 +78193 POINT(40.31839896059355 -73.2881207819872) bank78193 +78194 POINT(40.67026148218591 -74.25275479907734) bank78194 +78195 POINT(40.89956876382858 -73.39961738160807) bank78195 +78196 POINT(41.548145588603134 -73.58135097642706) bank78196 +78197 POINT(40.323049571232175 -73.19285369983763) bank78197 +78198 POINT(40.117901709307695 -74.68351432430447) bank78198 +78199 POINT(39.83146769690535 -74.3175889785314) bank78199 +78200 POINT(41.22702476496168 -74.47082636756836) bank78200 +78201 POINT(41.617661713544244 -73.12769871529505) bank78201 +78202 POINT(41.13931631646163 -74.15626657558198) bank78202 +78203 POINT(39.73005233595292 -74.18631799802974) bank78203 +78204 POINT(40.11315775958979 -74.74040284853318) bank78204 +78205 POINT(40.25286317694894 -73.48239258022356) bank78205 +78206 POINT(40.45160863216058 -74.01251574379297) bank78206 +78207 POINT(40.57193040227591 -73.02135652754211) bank78207 +78208 POINT(41.484715452288604 -74.14027658832637) bank78208 +78209 POINT(41.199830648460306 -74.88161533138961) bank78209 +78210 POINT(40.888060618452464 -74.57351990820786) bank78210 +78211 POINT(40.513846981820755 -73.8553274738597) bank78211 +78212 POINT(40.1572199623628 -73.97811469508675) bank78212 +78213 POINT(40.60642657036044 -73.27716645138328) bank78213 +78214 POINT(41.45314733893293 -74.39795616698491) bank78214 +78215 POINT(40.856304589595226 -74.58620062992682) bank78215 +78216 POINT(40.14320004005137 -74.8751434369679) bank78216 +78217 POINT(40.38532420915299 -74.01668056131295) bank78217 +78218 POINT(40.05947197898747 -74.49066253363908) bank78218 +78219 POINT(39.97402014666116 -73.53257209296133) bank78219 +78220 POINT(41.07508479885384 -73.26491667054776) bank78220 +78221 POINT(41.16814932226327 -74.09317699788342) bank78221 +78222 POINT(41.27011703362014 -74.10443554167267) bank78222 +78223 POINT(41.70293262125189 -74.7867747738708) bank78223 +78224 POINT(40.309265708647345 -74.14855717332249) bank78224 +78225 POINT(41.06222408421138 -73.50539229420221) bank78225 +78226 POINT(40.87186798071584 -74.62094544774688) bank78226 +78227 POINT(40.89705905426653 -73.55327627358446) bank78227 +78228 POINT(39.994435805511195 -74.70816183870889) bank78228 +78229 POINT(40.07434305548968 -74.8934261116223) bank78229 +78230 POINT(41.63572596398444 -73.84359879241502) bank78230 +78231 POINT(41.410999022953334 -73.4589191778623) bank78231 +78232 POINT(41.57382352158435 -73.96640788563104) bank78232 +78233 POINT(41.56268517865698 -74.44175036332314) bank78233 +78234 POINT(41.23242400643246 -74.42865394021418) bank78234 +78235 POINT(40.81215968795204 -73.67067746330696) bank78235 +78236 POINT(41.47237681471446 -74.01184936459164) bank78236 +78237 POINT(40.16190040303459 -74.71875008016444) bank78237 +78238 POINT(40.6475864118084 -74.34686767640288) bank78238 +78239 POINT(41.645638381006584 -73.9483413102347) bank78239 +78240 POINT(41.148497824931326 -73.85198427301842) bank78240 +78241 POINT(41.6733954505905 -73.3538519893773) bank78241 +78242 POINT(40.30388973759714 -73.02318484806364) bank78242 +78243 POINT(40.438120181606955 -73.44834170916333) bank78243 +78244 POINT(40.641978383702046 -74.31300680383971) bank78244 +78245 POINT(40.766019398738344 -74.54259530536798) bank78245 +78246 POINT(39.85427232391213 -74.48622477859043) bank78246 +78247 POINT(40.58639213158277 -74.88750285339599) bank78247 +78248 POINT(41.45100243731724 -73.06629712536778) bank78248 +78249 POINT(40.20190276205746 -73.22701641490329) bank78249 +78250 POINT(40.49973712058115 -73.45266560998247) bank78250 +78251 POINT(40.67263686559375 -74.7669496933992) bank78251 +78252 POINT(41.37097506454261 -73.6926172011745) bank78252 +78253 POINT(41.64971666759781 -74.4015344167468) bank78253 +78254 POINT(39.8463470706885 -74.49649206870146) bank78254 +78255 POINT(39.98894808192221 -73.1561873456166) bank78255 +78256 POINT(41.27973884604412 -73.32699156107262) bank78256 +78257 POINT(39.911545905751495 -73.36169863645934) bank78257 +78258 POINT(41.51994543780663 -74.85333164932293) bank78258 +78259 POINT(40.50065658567275 -74.79463454315906) bank78259 +78260 POINT(39.9883998719025 -73.19918259857636) bank78260 +78261 POINT(39.97589598027753 -73.81052357191325) bank78261 +78262 POINT(39.78707157057054 -74.21151718307848) bank78262 +78263 POINT(40.26026004364919 -74.99186261071) bank78263 +78264 POINT(39.784438928927976 -74.44838562290185) bank78264 +78265 POINT(39.83970024544071 -73.13979404878214) bank78265 +78266 POINT(39.741565774127494 -74.61424190556856) bank78266 +78267 POINT(40.42331005217155 -73.73239230614428) bank78267 +78268 POINT(41.639807762863676 -74.25265567339953) bank78268 +78269 POINT(40.71349001390512 -74.46566912342028) bank78269 +78270 POINT(41.614654591877304 -74.7686249983725) bank78270 +78271 POINT(40.47473247912765 -74.65909393007412) bank78271 +78272 POINT(40.27951908813371 -73.27845276116888) bank78272 +78273 POINT(41.704585506447856 -74.03458218645916) bank78273 +78274 POINT(40.61202483201009 -74.65038524454722) bank78274 +78275 POINT(40.080386740216134 -73.50499150131641) bank78275 +78276 POINT(39.8875634024007 -73.61783600108896) bank78276 +78277 POINT(39.82026898343353 -74.99802640380776) bank78277 +78278 POINT(41.38337803547369 -73.40458927217855) bank78278 +78279 POINT(41.09744516280621 -74.16201273365907) bank78279 +78280 POINT(40.0900176499095 -73.59333810463684) bank78280 +78281 POINT(40.62599901401126 -74.91562689214747) bank78281 +78282 POINT(40.85219014888783 -73.78695449759786) bank78282 +78283 POINT(40.542310785771924 -73.29198988835095) bank78283 +78284 POINT(41.06754584487812 -74.47311296349653) bank78284 +78285 POINT(41.44421515554186 -73.07348838463899) bank78285 +78286 POINT(41.01207435138165 -73.28317054532316) bank78286 +78287 POINT(41.44554051286962 -74.48771008338252) bank78287 +78288 POINT(40.65841964813197 -74.17074015391768) bank78288 +78289 POINT(39.73437809223398 -74.40301301732907) bank78289 +78290 POINT(40.133285549523904 -74.81449539825114) bank78290 +78291 POINT(40.185549729147574 -74.7422975501926) bank78291 +78292 POINT(40.177508243058114 -74.1792789771421) bank78292 +78293 POINT(41.043911497177035 -74.18886370580968) bank78293 +78294 POINT(39.74021046006815 -73.18554023497401) bank78294 +78295 POINT(40.66236464662757 -73.24400715834267) bank78295 +78296 POINT(40.09600674183338 -73.369568760164) bank78296 +78297 POINT(41.25982783153201 -74.00076510551688) bank78297 +78298 POINT(41.09726094827684 -73.69529785422452) bank78298 +78299 POINT(40.83279964911888 -73.89243541043453) bank78299 +78300 POINT(41.65615651592474 -74.29546011386822) bank78300 +78301 POINT(40.574368276744806 -74.2683713097473) bank78301 +78302 POINT(40.02608476130736 -73.39640947198373) bank78302 +78303 POINT(40.018976312812384 -74.05691931465695) bank78303 +78304 POINT(41.42779692222757 -73.9592528562629) bank78304 +78305 POINT(41.55753974373692 -74.31114168120357) bank78305 +78306 POINT(40.4006472856991 -74.65888635397636) bank78306 +78307 POINT(40.2169322772956 -74.54544032064432) bank78307 +78308 POINT(40.77169856379296 -74.51165001910093) bank78308 +78309 POINT(39.8482459000986 -74.96794765432193) bank78309 +78310 POINT(41.06286840312516 -73.70851973272606) bank78310 +78311 POINT(40.95533876530583 -74.24339410665931) bank78311 +78312 POINT(40.47021084855806 -74.67190217754586) bank78312 +78313 POINT(40.740977294907715 -74.51355711932923) bank78313 +78314 POINT(39.98416748481197 -73.72109508042608) bank78314 +78315 POINT(40.09011717269131 -74.95834866519476) bank78315 +78316 POINT(41.651116068920054 -74.13363135427693) bank78316 +78317 POINT(40.46082673789887 -74.23004778707782) bank78317 +78318 POINT(40.715399802923436 -74.69645913071513) bank78318 +78319 POINT(41.321431240234475 -74.51495861102494) bank78319 +78320 POINT(41.6225040927915 -73.19467445763365) bank78320 +78321 POINT(40.046666577099096 -73.94329263000141) bank78321 +78322 POINT(39.82470888517981 -74.35075180859958) bank78322 +78323 POINT(40.58365752319757 -74.43050425493125) bank78323 +78324 POINT(41.69128209377443 -74.82206794199385) bank78324 +78325 POINT(41.35979021563929 -73.44786453072828) bank78325 +78326 POINT(41.079886830769865 -73.84113363163266) bank78326 +78327 POINT(40.95548906306854 -74.45547423956539) bank78327 +78328 POINT(40.92788653175286 -74.71801033887382) bank78328 +78329 POINT(39.964414213375484 -73.55433164751447) bank78329 +78330 POINT(40.30472737045965 -74.76594454272652) bank78330 +78331 POINT(40.613572207725994 -74.20699342983583) bank78331 +78332 POINT(39.90994872124858 -74.58325905705514) bank78332 +78333 POINT(40.567242548807066 -74.95137769644181) bank78333 +78334 POINT(40.220288755621844 -73.28711947351337) bank78334 +78335 POINT(40.426697159977124 -74.9380128030094) bank78335 +78336 POINT(40.159590973740244 -73.14918009895746) bank78336 +78337 POINT(41.45010954499394 -73.502728551362) bank78337 +78338 POINT(40.695470529947514 -74.3701257288313) bank78338 +78339 POINT(41.39920514127692 -74.11280271119733) bank78339 +78340 POINT(41.641814161302285 -73.50422659522432) bank78340 +78341 POINT(40.05688836109101 -74.7686428771819) bank78341 +78342 POINT(40.44223905819817 -74.91593019086915) bank78342 +78343 POINT(41.60823884991814 -74.15682157805222) bank78343 +78344 POINT(41.07779506199766 -73.42558203560915) bank78344 +78345 POINT(40.750811454684666 -73.76456772297963) bank78345 +78346 POINT(40.2512490688712 -73.8679722554684) bank78346 +78347 POINT(40.06074391519328 -74.05363897047408) bank78347 +78348 POINT(40.02407518015912 -73.37058824835992) bank78348 +78349 POINT(40.18733349389548 -74.64386366407196) bank78349 +78350 POINT(40.985369428231465 -73.82915918701528) bank78350 +78351 POINT(39.843684406010546 -73.40221403314227) bank78351 +78352 POINT(41.20429344910868 -73.14771087219003) bank78352 +78353 POINT(40.56374901417954 -73.01868643996141) bank78353 +78354 POINT(39.89715646936438 -74.02764913965427) bank78354 +78355 POINT(40.07378858918809 -73.17284762760582) bank78355 +78356 POINT(41.68285041598735 -74.35823155836307) bank78356 +78357 POINT(40.870916554119404 -74.27280806710024) bank78357 +78358 POINT(40.12784698709013 -74.61879918915638) bank78358 +78359 POINT(40.9068969174913 -73.73499410798064) bank78359 +78360 POINT(41.65947194449021 -74.03579132213187) bank78360 +78361 POINT(41.34331216464042 -74.01709212115684) bank78361 +78362 POINT(41.21394730375636 -73.20817512460091) bank78362 +78363 POINT(40.4488845061471 -74.38595245600165) bank78363 +78364 POINT(40.36090882674537 -73.51645538739426) bank78364 +78365 POINT(40.73729165243741 -74.73941540613849) bank78365 +78366 POINT(40.07376296522397 -73.10623341679691) bank78366 +78367 POINT(40.882238664233796 -73.69707543075232) bank78367 +78368 POINT(41.01184768465713 -74.01999164368716) bank78368 +78369 POINT(40.6643432051892 -73.44337379483207) bank78369 +78370 POINT(40.4099411978992 -74.20550525848172) bank78370 +78371 POINT(40.0688625839226 -74.05471678873667) bank78371 +78372 POINT(41.39926924048596 -74.1785518190199) bank78372 +78373 POINT(40.71132542562612 -74.87731634206692) bank78373 +78374 POINT(40.49076532786858 -74.04881300037869) bank78374 +78375 POINT(41.042449018459976 -74.28567613563305) bank78375 +78376 POINT(41.62072648670928 -73.39775991186481) bank78376 +78377 POINT(40.65169791112027 -73.0378001749844) bank78377 +78378 POINT(39.749262012353434 -73.56104184534651) bank78378 +78379 POINT(40.44938943256268 -74.17384498426429) bank78379 +78380 POINT(41.55137910020004 -74.59203283487942) bank78380 +78381 POINT(40.91849949760117 -74.74519071748269) bank78381 +78382 POINT(40.59023838387067 -73.6127378947712) bank78382 +78383 POINT(41.370061423077786 -74.07452849807761) bank78383 +78384 POINT(41.678104481526695 -73.68049897967344) bank78384 +78385 POINT(40.59415687009567 -74.8778417586325) bank78385 +78386 POINT(41.65416543254852 -73.0808502359789) bank78386 +78387 POINT(41.109416717393515 -74.73633335970699) bank78387 +78388 POINT(40.22996999353862 -74.48521556406672) bank78388 +78389 POINT(41.70017307476986 -74.96781891961118) bank78389 +78390 POINT(40.60542428351469 -74.90215988998533) bank78390 +78391 POINT(40.48477834475234 -73.60906857484902) bank78391 +78392 POINT(40.93702824133587 -74.3632880231009) bank78392 +78393 POINT(39.9722222602824 -74.29549006676473) bank78393 +78394 POINT(40.30406300094824 -73.96668743449389) bank78394 +78395 POINT(40.355879506343086 -74.5186199205158) bank78395 +78396 POINT(40.15272037398254 -74.76933021721834) bank78396 +78397 POINT(41.588271447598146 -74.61074607726775) bank78397 +78398 POINT(40.02498999375302 -74.62051220491932) bank78398 +78399 POINT(40.30609189385713 -73.14066694403537) bank78399 +78400 POINT(39.76580376284481 -74.3759720608157) bank78400 +78401 POINT(41.347238552001365 -74.22305220330385) bank78401 +78402 POINT(39.94331144348854 -74.10358477360398) bank78402 +78403 POINT(41.07124865512586 -74.58551067282205) bank78403 +78404 POINT(40.21812491185396 -73.72720591987884) bank78404 +78405 POINT(41.41743421925805 -73.1656599721255) bank78405 +78406 POINT(40.46201995034324 -74.41461490140215) bank78406 +78407 POINT(40.53385756476948 -74.67491753630034) bank78407 +78408 POINT(40.608418936365005 -74.76662700657951) bank78408 +78409 POINT(41.41521344176148 -74.82943414701492) bank78409 +78410 POINT(41.575195236155345 -73.3532506797326) bank78410 +78411 POINT(41.26239566307248 -74.66501945807765) bank78411 +78412 POINT(41.67381926165228 -73.95035433271254) bank78412 +78413 POINT(40.05643478638028 -74.334857506455) bank78413 +78414 POINT(40.80321149698812 -74.64730290843515) bank78414 +78415 POINT(40.122812757043796 -74.13569767709772) bank78415 +78416 POINT(40.26667599032684 -73.91444657569657) bank78416 +78417 POINT(40.01931909890872 -74.39475910192702) bank78417 +78418 POINT(40.96696390203741 -73.48322470654345) bank78418 +78419 POINT(40.99205717392625 -74.2165490256159) bank78419 +78420 POINT(39.91592645386302 -74.69172159482798) bank78420 +78421 POINT(41.554885822666776 -73.37031293196597) bank78421 +78422 POINT(39.834864676161864 -74.8718959667464) bank78422 +78423 POINT(41.090700909594425 -73.53067227795626) bank78423 +78424 POINT(40.59455857093716 -74.73571105409727) bank78424 +78425 POINT(41.39540491725127 -73.93683101957082) bank78425 +78426 POINT(40.84212122549205 -74.76078218463522) bank78426 +78427 POINT(40.1026990673206 -74.33044367072856) bank78427 +78428 POINT(40.42656115529738 -74.13201679443752) bank78428 +78429 POINT(39.74893445321625 -73.18340097787612) bank78429 +78430 POINT(41.32663599940138 -73.89378949018541) bank78430 +78431 POINT(40.784372684692045 -73.43847912811755) bank78431 +78432 POINT(41.111796731505684 -73.89551637109886) bank78432 +78433 POINT(41.35078922328054 -74.11129988016349) bank78433 +78434 POINT(41.052760441338876 -74.16262585105875) bank78434 +78435 POINT(41.60153274944873 -74.05135954431303) bank78435 +78436 POINT(40.859516642066964 -73.6120325948529) bank78436 +78437 POINT(40.241387592110115 -74.56736601669323) bank78437 +78438 POINT(41.327247691781075 -74.24892392522233) bank78438 +78439 POINT(39.82223355690912 -73.31653884148764) bank78439 +78440 POINT(41.69339140959373 -73.60935639980273) bank78440 +78441 POINT(41.41761262743734 -73.74971297794843) bank78441 +78442 POINT(41.442659080091076 -73.9896566127781) bank78442 +78443 POINT(41.12947655650463 -74.83349242954326) bank78443 +78444 POINT(39.75206942597677 -74.25719829205941) bank78444 +78445 POINT(40.82691154399832 -74.53925616119959) bank78445 +78446 POINT(40.466969246192974 -74.159023514322) bank78446 +78447 POINT(39.716133467314144 -74.67879570020386) bank78447 +78448 POINT(40.10506609598751 -73.39897392991246) bank78448 +78449 POINT(40.484070386508066 -74.94340037047635) bank78449 +78450 POINT(41.57294840279382 -74.73579130025408) bank78450 +78451 POINT(41.23913602229981 -74.78241379064141) bank78451 +78452 POINT(41.23845438637515 -73.84786290042034) bank78452 +78453 POINT(40.22397444417638 -73.24156488259044) bank78453 +78454 POINT(39.940998737922726 -73.96065382100461) bank78454 +78455 POINT(40.420279525552225 -74.6810545220943) bank78455 +78456 POINT(39.78018290515535 -74.03984763505856) bank78456 +78457 POINT(40.38388677490467 -73.77840075640863) bank78457 +78458 POINT(39.981451116851986 -73.18334775679101) bank78458 +78459 POINT(40.906491361104266 -73.09363039194199) bank78459 +78460 POINT(41.316704489228954 -74.0213098692083) bank78460 +78461 POINT(41.194292651821996 -74.72236063697784) bank78461 +78462 POINT(41.4655044071896 -73.30667408655825) bank78462 +78463 POINT(40.74056051692451 -73.25898695861514) bank78463 +78464 POINT(41.32326925606294 -74.48417402190654) bank78464 +78465 POINT(40.35718541697341 -74.06647969132744) bank78465 +78466 POINT(40.67400730408109 -73.37563487020365) bank78466 +78467 POINT(40.32543423142022 -73.48109334766266) bank78467 +78468 POINT(40.062349296875276 -74.00238532915782) bank78468 +78469 POINT(40.874348860730585 -73.21930004477855) bank78469 +78470 POINT(41.345713200216736 -73.70419842849462) bank78470 +78471 POINT(40.94118692814372 -74.82767101986268) bank78471 +78472 POINT(41.70860140263826 -74.32914170384595) bank78472 +78473 POINT(40.18478800665551 -73.96548255992637) bank78473 +78474 POINT(41.396227888483565 -73.5908063883968) bank78474 +78475 POINT(40.2718323199764 -73.42680274647266) bank78475 +78476 POINT(41.12992621689894 -73.66033389211498) bank78476 +78477 POINT(40.004750643222295 -73.9477413442168) bank78477 +78478 POINT(40.86094253891167 -74.66289953724527) bank78478 +78479 POINT(41.07630022517412 -73.91534227185466) bank78479 +78480 POINT(40.95121605383972 -74.51654913184352) bank78480 +78481 POINT(40.171381749996414 -73.39008788916242) bank78481 +78482 POINT(40.69285214111133 -73.20197660157058) bank78482 +78483 POINT(40.67302862138743 -74.91381105863522) bank78483 +78484 POINT(41.517147700291765 -74.65126403243275) bank78484 +78485 POINT(39.93759319475172 -73.61943795686713) bank78485 +78486 POINT(41.66221481962296 -74.90705350154916) bank78486 +78487 POINT(40.79109895892716 -74.34990031284961) bank78487 +78488 POINT(40.917861134301596 -73.0587517850506) bank78488 +78489 POINT(40.179027511223744 -73.68378441303608) bank78489 +78490 POINT(40.12243485452604 -73.15616097175726) bank78490 +78491 POINT(41.68891470148811 -73.43230026408823) bank78491 +78492 POINT(40.49367953171917 -73.83410642513735) bank78492 +78493 POINT(41.0024693178266 -73.97474481748523) bank78493 +78494 POINT(41.66173131630472 -74.22288181471795) bank78494 +78495 POINT(40.95181558992128 -73.8487102634687) bank78495 +78496 POINT(40.08351657955256 -74.27784712100463) bank78496 +78497 POINT(40.17417161275196 -74.65643980138762) bank78497 +78498 POINT(41.164493092482026 -73.54781394566433) bank78498 +78499 POINT(41.44528542961668 -73.89765508594719) bank78499 +78500 POINT(39.928416337003924 -73.8777395342651) bank78500 +78501 POINT(40.40674730809329 -74.8526262520787) bank78501 +78502 POINT(41.38617824789612 -73.34508979086303) bank78502 +78503 POINT(40.62224890277221 -74.3060368849741) bank78503 +78504 POINT(41.43013274901966 -73.89550159622644) bank78504 +78505 POINT(41.42915380590255 -73.17180124961547) bank78505 +78506 POINT(40.54413393917606 -73.44843306857302) bank78506 +78507 POINT(40.42138274225236 -73.4843748738276) bank78507 +78508 POINT(40.38253507977592 -74.37253717353073) bank78508 +78509 POINT(40.97452123319235 -74.57212079317708) bank78509 +78510 POINT(41.25190172803543 -74.87605899751242) bank78510 +78511 POINT(40.81074606204189 -74.62554928349216) bank78511 +78512 POINT(41.366994590620244 -73.6285225536317) bank78512 +78513 POINT(41.55637415399507 -74.0379196062343) bank78513 +78514 POINT(40.33326114906496 -73.63373856730657) bank78514 +78515 POINT(41.30234918323155 -74.16645066549376) bank78515 +78516 POINT(39.95045634672216 -74.28397042577627) bank78516 +78517 POINT(40.56466461547924 -73.3149038598083) bank78517 +78518 POINT(40.40815013068464 -74.24332871882793) bank78518 +78519 POINT(40.26231014531698 -73.18421236691364) bank78519 +78520 POINT(40.35790632491456 -73.74047218880467) bank78520 +78521 POINT(41.49364224724785 -73.9968942890367) bank78521 +78522 POINT(40.61556850815857 -73.90760173830314) bank78522 +78523 POINT(41.70320429699396 -73.78955122396586) bank78523 +78524 POINT(40.301197971897594 -73.33282276680502) bank78524 +78525 POINT(41.146386000599556 -73.96051911817585) bank78525 +78526 POINT(41.549984915519325 -74.16858288694027) bank78526 +78527 POINT(41.57256945188459 -73.41117062263254) bank78527 +78528 POINT(41.54326261738454 -74.73279827603815) bank78528 +78529 POINT(40.60278630785755 -73.60689948031991) bank78529 +78530 POINT(41.64550675102663 -73.78575220322546) bank78530 +78531 POINT(41.063571212147814 -74.58340691154713) bank78531 +78532 POINT(39.997311000375355 -73.14394361860316) bank78532 +78533 POINT(40.866726340202845 -73.69975026407627) bank78533 +78534 POINT(40.74388601394155 -74.39011010986496) bank78534 +78535 POINT(40.819376609603125 -74.28241538234069) bank78535 +78536 POINT(40.305732152888964 -73.62882597679074) bank78536 +78537 POINT(39.98258589380349 -73.69380157111132) bank78537 +78538 POINT(41.47476053692268 -73.46212381732614) bank78538 +78539 POINT(40.79508854709156 -73.06047568429155) bank78539 +78540 POINT(39.95094790963938 -73.30949416611277) bank78540 +78541 POINT(40.87757638797596 -73.96114057911984) bank78541 +78542 POINT(40.13980454029772 -74.66846410429591) bank78542 +78543 POINT(40.013372609264884 -74.27620039410269) bank78543 +78544 POINT(40.07503713805096 -73.80680154111383) bank78544 +78545 POINT(40.03257653222854 -74.50433109088466) bank78545 +78546 POINT(41.65062964907747 -73.5888543642793) bank78546 +78547 POINT(39.89202880450439 -74.04222240206863) bank78547 +78548 POINT(40.402624233007344 -74.49564521667696) bank78548 +78549 POINT(40.68751658218346 -73.83747963269363) bank78549 +78550 POINT(40.328811587824724 -74.40207724472847) bank78550 +78551 POINT(40.1382649906742 -74.67990781336326) bank78551 +78552 POINT(41.391519702625985 -74.99090098669802) bank78552 +78553 POINT(40.752715336657964 -74.23574295566955) bank78553 +78554 POINT(39.748611291841925 -73.84351304206224) bank78554 +78555 POINT(41.32090891035993 -73.34841223959195) bank78555 +78556 POINT(41.69631376280703 -73.40342229776988) bank78556 +78557 POINT(40.32510001547703 -73.23444529307291) bank78557 +78558 POINT(41.382162814219 -73.99354666613883) bank78558 +78559 POINT(39.87214506715181 -73.32095950971525) bank78559 +78560 POINT(40.79654365912953 -73.7405824805113) bank78560 +78561 POINT(40.49348518810384 -73.23739410000677) bank78561 +78562 POINT(40.448701516990674 -74.896110523412) bank78562 +78563 POINT(40.88066782210096 -73.0168779778839) bank78563 +78564 POINT(40.324077724399125 -74.28634975032364) bank78564 +78565 POINT(41.486241239127196 -73.79327787604944) bank78565 +78566 POINT(40.86201406864005 -74.99845134841692) bank78566 +78567 POINT(40.320164104076724 -73.46771937401749) bank78567 +78568 POINT(40.01848349088843 -73.57075343115262) bank78568 +78569 POINT(41.16103949528493 -74.28830357283488) bank78569 +78570 POINT(40.92398745719837 -74.98202106252057) bank78570 +78571 POINT(41.59646583252822 -73.13847152459452) bank78571 +78572 POINT(40.231688656380584 -74.3889179651136) bank78572 +78573 POINT(39.93811018105332 -74.99175785035672) bank78573 +78574 POINT(40.91195903694285 -74.62921694497099) bank78574 +78575 POINT(40.365594894797574 -73.41282091368063) bank78575 +78576 POINT(40.80504452274122 -73.47270020781065) bank78576 +78577 POINT(41.0617967934539 -74.16902312695032) bank78577 +78578 POINT(39.86577480573022 -73.45336772620907) bank78578 +78579 POINT(40.075875782859256 -74.70206463870251) bank78579 +78580 POINT(40.30298313049298 -73.06221836518178) bank78580 +78581 POINT(40.29709348294147 -74.43078242753394) bank78581 +78582 POINT(40.32734978277272 -74.83910335240915) bank78582 +78583 POINT(40.18235642422084 -73.14692227593567) bank78583 +78584 POINT(41.2905418381164 -74.3224730807829) bank78584 +78585 POINT(41.51869624579084 -74.59000084550091) bank78585 +78586 POINT(41.21751549463736 -74.01391662208486) bank78586 +78587 POINT(41.05936375709201 -74.15210137545728) bank78587 +78588 POINT(41.38154940520884 -74.94000670653807) bank78588 +78589 POINT(40.953507224874954 -74.24425320421813) bank78589 +78590 POINT(40.68727715991228 -74.51720091086602) bank78590 +78591 POINT(40.23928415062398 -74.62777253907376) bank78591 +78592 POINT(40.47483012409051 -73.44507103885915) bank78592 +78593 POINT(39.93151260250814 -73.04699022116795) bank78593 +78594 POINT(40.88067278420312 -74.21022282308301) bank78594 +78595 POINT(39.97392978441033 -74.35578371052601) bank78595 +78596 POINT(40.074827755742234 -73.25150230257391) bank78596 +78597 POINT(40.84591105595713 -74.79350428149431) bank78597 +78598 POINT(40.02317169560636 -73.34920242159) bank78598 +78599 POINT(40.379620709964755 -73.33965808625933) bank78599 +78600 POINT(40.19127568516625 -73.26256981407035) bank78600 +78601 POINT(40.34050516523096 -73.70737914064958) bank78601 +78602 POINT(40.711449917765215 -74.58778499352813) bank78602 +78603 POINT(41.270341891730325 -73.19593653168951) bank78603 +78604 POINT(40.83048160844361 -74.37740553111807) bank78604 +78605 POINT(41.29674098931778 -74.7268930646418) bank78605 +78606 POINT(41.56271166742184 -73.18480461914771) bank78606 +78607 POINT(40.535443632086405 -74.12948692812226) bank78607 +78608 POINT(41.54531621424761 -73.18598315624693) bank78608 +78609 POINT(40.69899228888954 -74.80050453229565) bank78609 +78610 POINT(40.18765478822365 -73.42257203419487) bank78610 +78611 POINT(41.30165198229848 -73.24963833040724) bank78611 +78612 POINT(40.02103708710927 -74.5946132029712) bank78612 +78613 POINT(41.18693221887659 -74.6872699988517) bank78613 +78614 POINT(41.4923034711074 -74.58715412264173) bank78614 +78615 POINT(41.428968863581076 -74.5913904904119) bank78615 +78616 POINT(40.25250314222889 -74.43566126017421) bank78616 +78617 POINT(39.79247332367072 -73.30706991248638) bank78617 +78618 POINT(40.272812559096906 -74.96849041322815) bank78618 +78619 POINT(40.336702341104235 -73.01568279985666) bank78619 +78620 POINT(40.782982496558255 -74.55795528665854) bank78620 +78621 POINT(40.94089343877341 -73.30111805281548) bank78621 +78622 POINT(39.90484516604898 -74.56560659620214) bank78622 +78623 POINT(40.25771477113462 -74.54777016387087) bank78623 +78624 POINT(41.40129095731852 -74.61246668147253) bank78624 +78625 POINT(41.68347107091418 -74.41258968957781) bank78625 +78626 POINT(41.63089191125304 -74.52325047712159) bank78626 +78627 POINT(41.47209904408418 -74.97145495850063) bank78627 +78628 POINT(39.71673277443487 -73.06187654762951) bank78628 +78629 POINT(39.991654809684434 -73.52721220538878) bank78629 +78630 POINT(39.882726759776126 -74.67153359927148) bank78630 +78631 POINT(41.27833933895402 -74.88070616514304) bank78631 +78632 POINT(39.781948059047124 -73.63911186619762) bank78632 +78633 POINT(41.68637154616073 -73.46349181454684) bank78633 +78634 POINT(41.26516107009746 -74.34356860981144) bank78634 +78635 POINT(41.31401425098011 -73.69589822120305) bank78635 +78636 POINT(39.93310395699259 -73.31428283746776) bank78636 +78637 POINT(41.29256976533946 -73.83417553318797) bank78637 +78638 POINT(40.32578456767002 -74.97194851005476) bank78638 +78639 POINT(39.919251782071 -73.90308566329438) bank78639 +78640 POINT(40.651438132153274 -74.76331226140337) bank78640 +78641 POINT(40.54923286879963 -74.46436670476399) bank78641 +78642 POINT(40.464701636481166 -74.425511295544) bank78642 +78643 POINT(40.51488266497003 -74.48442283613952) bank78643 +78644 POINT(40.296094086954454 -73.393814857531) bank78644 +78645 POINT(41.593403156982866 -74.38588956010094) bank78645 +78646 POINT(41.47189454531545 -73.25650360881167) bank78646 +78647 POINT(41.665935291774375 -74.88981865080329) bank78647 +78648 POINT(39.80179597496666 -73.73573627257046) bank78648 +78649 POINT(40.85445663415741 -73.47146091789577) bank78649 +78650 POINT(39.87421567062285 -73.18487190189084) bank78650 +78651 POINT(40.41675307593051 -73.34393479321368) bank78651 +78652 POINT(40.79027383718342 -73.88992658266132) bank78652 +78653 POINT(41.52389626038585 -73.8202367016895) bank78653 +78654 POINT(39.90240547549986 -74.65092039587381) bank78654 +78655 POINT(39.75421733719626 -74.36353461031705) bank78655 +78656 POINT(41.55164444150201 -73.19308299963521) bank78656 +78657 POINT(40.06171066177159 -74.30596039834926) bank78657 +78658 POINT(41.07793217566148 -74.40391446668895) bank78658 +78659 POINT(39.90197367519108 -74.56930653342641) bank78659 +78660 POINT(40.87219414056186 -73.76710424874487) bank78660 +78661 POINT(41.25161623438548 -74.14389076957441) bank78661 +78662 POINT(39.982140161214296 -74.65215996406506) bank78662 +78663 POINT(39.89439713707078 -74.27644092622731) bank78663 +78664 POINT(41.393668585914064 -74.1876701643873) bank78664 +78665 POINT(40.44439569076515 -73.92994617510031) bank78665 +78666 POINT(40.70446604455732 -73.31759004066981) bank78666 +78667 POINT(40.18703547271733 -73.18092951408127) bank78667 +78668 POINT(40.61639995739091 -74.48314570537946) bank78668 +78669 POINT(41.07268807450036 -73.74988839377612) bank78669 +78670 POINT(39.979653533550874 -73.9646059368485) bank78670 +78671 POINT(41.30386616411066 -73.44882451324933) bank78671 +78672 POINT(41.17256735921694 -74.35853703502816) bank78672 +78673 POINT(41.16918544841202 -73.37215193389149) bank78673 +78674 POINT(41.662581106416816 -74.39310303322584) bank78674 +78675 POINT(41.32368818037156 -73.08764978952338) bank78675 +78676 POINT(41.59461170048119 -74.25250887783004) bank78676 +78677 POINT(41.246654981962045 -73.0835083416294) bank78677 +78678 POINT(41.3667077708953 -73.33200335635769) bank78678 +78679 POINT(40.165945554167756 -73.82433082314145) bank78679 +78680 POINT(40.53184281483731 -73.0632675960048) bank78680 +78681 POINT(41.70841795608171 -74.8128049996708) bank78681 +78682 POINT(40.46690430879597 -73.51270710191709) bank78682 +78683 POINT(39.79394863049409 -74.69881550559147) bank78683 +78684 POINT(40.08564950546221 -73.25394543799688) bank78684 +78685 POINT(39.982907790650735 -74.04389892347122) bank78685 +78686 POINT(40.75822950423546 -74.18356571593154) bank78686 +78687 POINT(41.43674180638054 -74.43750241035981) bank78687 +78688 POINT(41.419210329375645 -74.16807899968612) bank78688 +78689 POINT(40.991355421579584 -73.7782535278164) bank78689 +78690 POINT(40.610828440395785 -73.55153760801018) bank78690 +78691 POINT(40.35872099873053 -73.84260241178872) bank78691 +78692 POINT(41.38176344023088 -73.88986784576687) bank78692 +78693 POINT(40.08205657198472 -74.98402996888437) bank78693 +78694 POINT(41.29922774283569 -74.37168655458551) bank78694 +78695 POINT(39.9629744623359 -73.49423908446907) bank78695 +78696 POINT(39.795457824004394 -74.11970415014324) bank78696 +78697 POINT(41.04372539568252 -73.09702341239782) bank78697 +78698 POINT(40.14642622174931 -73.76468980791161) bank78698 +78699 POINT(40.260155450271185 -74.38860537119427) bank78699 +78700 POINT(39.781686700542366 -73.7483477669665) bank78700 +78701 POINT(40.21841540345828 -74.07698900133558) bank78701 +78702 POINT(41.497420541912234 -73.89983069544967) bank78702 +78703 POINT(40.0344204551961 -74.04537367832071) bank78703 +78704 POINT(40.00990917730116 -74.992696177416) bank78704 +78705 POINT(40.82659251410082 -73.06175869949585) bank78705 +78706 POINT(41.05327224158794 -74.61978342998125) bank78706 +78707 POINT(39.7538364697824 -74.41437905161817) bank78707 +78708 POINT(41.135559928974864 -74.41687438959967) bank78708 +78709 POINT(40.07298953900257 -73.04072204883768) bank78709 +78710 POINT(40.029952889689376 -74.1379702398713) bank78710 +78711 POINT(41.689680572954316 -74.39439952561804) bank78711 +78712 POINT(41.298801116446654 -73.69479379490524) bank78712 +78713 POINT(40.7919723619555 -73.21673044686472) bank78713 +78714 POINT(40.44514920418236 -74.65390963229734) bank78714 +78715 POINT(40.05620359206048 -74.15259018679482) bank78715 +78716 POINT(41.34611642061495 -74.24820568603789) bank78716 +78717 POINT(41.27902066919644 -73.77262639845628) bank78717 +78718 POINT(40.793079145300396 -74.5754220477747) bank78718 +78719 POINT(40.03693162734378 -74.94988620874079) bank78719 +78720 POINT(40.064209354044664 -73.69777448977942) bank78720 +78721 POINT(41.139422736771834 -73.14762159988695) bank78721 +78722 POINT(40.49633626445291 -73.88228261895563) bank78722 +78723 POINT(41.605639565866745 -74.98515256786783) bank78723 +78724 POINT(41.557718238479836 -74.4244734791533) bank78724 +78725 POINT(40.16430676766027 -74.7813555119357) bank78725 +78726 POINT(40.356328137833486 -73.09502541609032) bank78726 +78727 POINT(41.31984258444809 -74.54628441048288) bank78727 +78728 POINT(40.043126740683405 -74.2990978127412) bank78728 +78729 POINT(40.44348157618691 -74.07117311183606) bank78729 +78730 POINT(41.43673774960927 -74.0192652882707) bank78730 +78731 POINT(40.81352135003221 -74.10345037718027) bank78731 +78732 POINT(40.28547360422915 -73.4547009423765) bank78732 +78733 POINT(40.23710227888781 -74.55495816476227) bank78733 +78734 POINT(40.94094585889174 -73.44246334465593) bank78734 +78735 POINT(39.811710463811245 -74.34915741214282) bank78735 +78736 POINT(41.20124157325943 -73.72988210324203) bank78736 +78737 POINT(40.51324474536522 -74.60087222447078) bank78737 +78738 POINT(41.49758063893424 -73.18527612950754) bank78738 +78739 POINT(40.355392509788196 -74.69815151836669) bank78739 +78740 POINT(40.76296166945277 -74.31328869711687) bank78740 +78741 POINT(40.13235832093301 -74.03827098284978) bank78741 +78742 POINT(40.95778638424713 -73.88028474930805) bank78742 +78743 POINT(39.852338019719014 -74.5368319332161) bank78743 +78744 POINT(41.28533830545943 -74.74758348835394) bank78744 +78745 POINT(41.39220449507821 -74.69649441206265) bank78745 +78746 POINT(40.489134370183464 -73.49050920265125) bank78746 +78747 POINT(39.860581303415216 -74.19683870363443) bank78747 +78748 POINT(41.03533306675291 -73.17025619578082) bank78748 +78749 POINT(40.42156539803718 -74.37271558933901) bank78749 +78750 POINT(41.113504870496485 -74.87959772054943) bank78750 +78751 POINT(41.68075740314081 -74.44557913397743) bank78751 +78752 POINT(40.461741800951884 -73.42668655636392) bank78752 +78753 POINT(40.31497846623457 -73.63031402344971) bank78753 +78754 POINT(39.75410753868375 -74.19308481317758) bank78754 +78755 POINT(40.336894274969325 -74.69844411372016) bank78755 +78756 POINT(40.8571379158613 -74.00071875896329) bank78756 +78757 POINT(41.43258606033802 -74.91052220564958) bank78757 +78758 POINT(41.459121372184626 -74.0756951049149) bank78758 +78759 POINT(40.96126858762641 -73.97098395050378) bank78759 +78760 POINT(40.782485451355974 -73.29117337660875) bank78760 +78761 POINT(40.601625699034656 -73.36310666474778) bank78761 +78762 POINT(40.985599921639405 -74.23900669679757) bank78762 +78763 POINT(40.80637524853447 -74.45029357426966) bank78763 +78764 POINT(40.34699020105417 -74.9753488638201) bank78764 +78765 POINT(41.20860694493048 -73.09617840880342) bank78765 +78766 POINT(40.68409490614424 -74.02356771625453) bank78766 +78767 POINT(41.42018035542538 -74.48563894670811) bank78767 +78768 POINT(40.54593221193234 -73.01780877923571) bank78768 +78769 POINT(39.76837649864891 -73.63781768665837) bank78769 +78770 POINT(40.460522694971466 -74.62260822858501) bank78770 +78771 POINT(41.08436305724107 -73.66817707888661) bank78771 +78772 POINT(40.28845284766681 -74.69153198883636) bank78772 +78773 POINT(39.859049256519896 -74.90034722357441) bank78773 +78774 POINT(40.46079919416489 -74.93441368942456) bank78774 +78775 POINT(40.12495651778272 -74.19008781789789) bank78775 +78776 POINT(40.86957234407691 -73.68341505785725) bank78776 +78777 POINT(40.38667324729544 -73.4739481885338) bank78777 +78778 POINT(39.9487818741955 -74.8471297095944) bank78778 +78779 POINT(41.64469654682201 -73.3649982363082) bank78779 +78780 POINT(41.39468525889416 -74.68786182500651) bank78780 +78781 POINT(41.469355091707186 -73.16722236123123) bank78781 +78782 POINT(40.53532375551588 -73.84278577337045) bank78782 +78783 POINT(40.45352656384482 -73.93161483829962) bank78783 +78784 POINT(40.87331254674839 -73.23735210693538) bank78784 +78785 POINT(41.1294773625876 -73.03839089206743) bank78785 +78786 POINT(41.080146979817485 -73.89444839314508) bank78786 +78787 POINT(41.021453744170564 -74.44229236804212) bank78787 +78788 POINT(41.61792552994185 -74.58804637070595) bank78788 +78789 POINT(41.0340106726405 -73.71435353066694) bank78789 +78790 POINT(41.6801986206151 -74.62899555657359) bank78790 +78791 POINT(39.95894606803785 -74.87138336868746) bank78791 +78792 POINT(41.21550244877097 -74.36526452587567) bank78792 +78793 POINT(39.78072803038487 -73.55955547868174) bank78793 +78794 POINT(40.621425199964115 -74.16704336407776) bank78794 +78795 POINT(39.78180727215427 -74.93145096814715) bank78795 +78796 POINT(39.88182691152767 -74.03625591690607) bank78796 +78797 POINT(41.24738653974208 -73.65730140136598) bank78797 +78798 POINT(40.54064900085609 -74.18254308741615) bank78798 +78799 POINT(41.56678169598597 -74.03027445489178) bank78799 +78800 POINT(40.891744046424556 -74.64791848317164) bank78800 +78801 POINT(39.7550996645202 -74.71103377122112) bank78801 +78802 POINT(41.035942388593945 -74.12057574141991) bank78802 +78803 POINT(41.28937257985582 -74.90767439538692) bank78803 +78804 POINT(41.29721860771163 -73.72809057719971) bank78804 +78805 POINT(41.306419275782005 -73.95845635895911) bank78805 +78806 POINT(40.63065804378938 -73.2366848869821) bank78806 +78807 POINT(41.34218001036271 -73.48888330934362) bank78807 +78808 POINT(41.18095509354539 -73.00762237325593) bank78808 +78809 POINT(40.90146503178917 -74.74821783067028) bank78809 +78810 POINT(41.691169416690606 -74.7599688615334) bank78810 +78811 POINT(40.77921706025186 -74.03378332381047) bank78811 +78812 POINT(40.551675089859174 -73.08115631249309) bank78812 +78813 POINT(40.16314511727565 -74.19435146829909) bank78813 +78814 POINT(40.074491325893206 -74.4136723758887) bank78814 +78815 POINT(40.777077012462456 -73.75489115680479) bank78815 +78816 POINT(40.70944116128135 -73.39131507491098) bank78816 +78817 POINT(39.84539424684291 -74.58557502008307) bank78817 +78818 POINT(40.99099845977048 -73.70974520548164) bank78818 +78819 POINT(41.25905718477517 -73.20190807572173) bank78819 +78820 POINT(39.88503544129143 -74.37747633554659) bank78820 +78821 POINT(40.27122412257838 -73.05966051644383) bank78821 +78822 POINT(40.836225132203815 -73.45214943280438) bank78822 +78823 POINT(41.60651722276372 -74.12494030935278) bank78823 +78824 POINT(41.3589908736389 -73.70884392423643) bank78824 +78825 POINT(40.80830774661618 -74.9525028138924) bank78825 +78826 POINT(40.087707222791046 -73.1571756719348) bank78826 +78827 POINT(41.15226987921679 -73.27584960146423) bank78827 +78828 POINT(39.89288439409971 -74.62721217534134) bank78828 +78829 POINT(40.28983309478992 -74.29977159265758) bank78829 +78830 POINT(40.79929598660464 -74.01621307452751) bank78830 +78831 POINT(39.9251291187942 -74.5922490205609) bank78831 +78832 POINT(39.94500348573083 -74.87622375457146) bank78832 +78833 POINT(41.1969641779847 -74.84979484322297) bank78833 +78834 POINT(40.91526634682174 -74.57917830491367) bank78834 +78835 POINT(40.65712424128233 -73.27370333839639) bank78835 +78836 POINT(39.82666853214757 -73.4180808908655) bank78836 +78837 POINT(40.18533915270659 -74.87780005931974) bank78837 +78838 POINT(40.43248824295903 -74.0562679261683) bank78838 +78839 POINT(40.67809819736739 -74.14679996160525) bank78839 +78840 POINT(40.13020858789557 -74.08327983624432) bank78840 +78841 POINT(40.2998365453679 -73.815961922189) bank78841 +78842 POINT(40.0533243336439 -73.26429910695764) bank78842 +78843 POINT(41.32046308518063 -74.6557976167776) bank78843 +78844 POINT(41.026918608303696 -73.0487841540595) bank78844 +78845 POINT(40.648193224712244 -73.11849948680897) bank78845 +78846 POINT(40.42394579475474 -74.16896175489711) bank78846 +78847 POINT(39.836700625967 -73.59927037870305) bank78847 +78848 POINT(41.15221883749419 -74.52381135062146) bank78848 +78849 POINT(40.96768539838034 -73.27769375797642) bank78849 +78850 POINT(41.19931093153086 -74.52432876767763) bank78850 +78851 POINT(40.08062406057269 -74.84396698017713) bank78851 +78852 POINT(41.17482533585731 -73.35066504230686) bank78852 +78853 POINT(40.32961265165764 -73.77133679197827) bank78853 +78854 POINT(41.228575933268765 -74.08296255646336) bank78854 +78855 POINT(41.661025074451196 -74.2744134379352) bank78855 +78856 POINT(40.49667619092393 -73.76905863482595) bank78856 +78857 POINT(40.511492178431176 -74.18710784016531) bank78857 +78858 POINT(41.45783903034874 -73.30245418186045) bank78858 +78859 POINT(40.210363077603276 -74.1863700241259) bank78859 +78860 POINT(40.97807693437375 -73.26663570793598) bank78860 +78861 POINT(40.37947974951271 -73.4611970145578) bank78861 +78862 POINT(39.82785896495407 -74.97392827352654) bank78862 +78863 POINT(40.3765018484971 -73.24775917379964) bank78863 +78864 POINT(39.97258330234866 -73.45566389901683) bank78864 +78865 POINT(40.8364444051985 -73.62676033273874) bank78865 +78866 POINT(40.04024926970734 -73.8900638028818) bank78866 +78867 POINT(40.19249450683039 -73.57076297155221) bank78867 +78868 POINT(39.82711937774684 -74.89927535946272) bank78868 +78869 POINT(40.604651325735084 -73.89918755698027) bank78869 +78870 POINT(40.52905228081018 -73.17465031793746) bank78870 +78871 POINT(41.33899488132658 -73.37944557746127) bank78871 +78872 POINT(41.651943668016585 -74.41077140248676) bank78872 +78873 POINT(40.832812138849306 -73.6363726347388) bank78873 +78874 POINT(40.33549990018621 -73.42525805397456) bank78874 +78875 POINT(40.91622066550975 -74.7419421914317) bank78875 +78876 POINT(41.70859460288554 -73.1778724134574) bank78876 +78877 POINT(41.66176873462974 -74.00706110481453) bank78877 +78878 POINT(40.04858238876156 -73.54551245352035) bank78878 +78879 POINT(40.43376116469426 -73.33944312417769) bank78879 +78880 POINT(41.38771593299775 -73.2547407610641) bank78880 +78881 POINT(41.67300557688236 -74.90338297823911) bank78881 +78882 POINT(41.1580116574591 -73.32396522592563) bank78882 +78883 POINT(41.642899998615064 -73.9791533479999) bank78883 +78884 POINT(40.92778017836714 -73.75867655973342) bank78884 +78885 POINT(40.402879314786404 -73.6695865356341) bank78885 +78886 POINT(40.564297765489194 -73.88655110857734) bank78886 +78887 POINT(41.41224690603416 -74.57514946823483) bank78887 +78888 POINT(40.61763612985113 -74.04295964107217) bank78888 +78889 POINT(40.530956962538276 -73.7848428655573) bank78889 +78890 POINT(39.74855012635125 -73.87340595679184) bank78890 +78891 POINT(39.95768552296208 -73.03276330248647) bank78891 +78892 POINT(40.51074738810641 -73.19709632228052) bank78892 +78893 POINT(40.8643413211652 -73.4387732008726) bank78893 +78894 POINT(41.30134857628845 -73.39110786781977) bank78894 +78895 POINT(40.001807418070804 -73.45661469402336) bank78895 +78896 POINT(40.64365903506822 -73.36350085177233) bank78896 +78897 POINT(41.59955645470617 -74.52556213268892) bank78897 +78898 POINT(40.52121252549428 -74.52190514809683) bank78898 +78899 POINT(41.22628744409243 -74.29292972334942) bank78899 +78900 POINT(41.42009091392611 -74.10513305008983) bank78900 +78901 POINT(40.22090135784645 -74.90486836729434) bank78901 +78902 POINT(41.51864013788044 -74.5237035458897) bank78902 +78903 POINT(40.64176686476984 -73.87162596313775) bank78903 +78904 POINT(41.42530686234064 -73.01848208732045) bank78904 +78905 POINT(41.47653545325103 -73.72610967259553) bank78905 +78906 POINT(40.88825794792215 -73.08361839226569) bank78906 +78907 POINT(41.50729723393877 -74.88022386401991) bank78907 +78908 POINT(40.63212383333694 -74.4831033392034) bank78908 +78909 POINT(41.29697409546196 -74.19022062881241) bank78909 +78910 POINT(41.45429532312032 -74.94259337458082) bank78910 +78911 POINT(41.67807465556097 -73.31957158067644) bank78911 +78912 POINT(41.06241890243811 -73.34621279039736) bank78912 +78913 POINT(40.501019057161635 -74.93143330865556) bank78913 +78914 POINT(40.48994905113513 -73.36209318590294) bank78914 +78915 POINT(40.458221070871424 -74.69408604397762) bank78915 +78916 POINT(41.35795116323148 -74.12231020982125) bank78916 +78917 POINT(40.98771538472153 -73.88199080774844) bank78917 +78918 POINT(40.12694031435206 -74.84589362590785) bank78918 +78919 POINT(40.46906131774298 -74.78575011060848) bank78919 +78920 POINT(41.50034360993273 -73.56969227360676) bank78920 +78921 POINT(41.229252656197474 -73.08766400905515) bank78921 +78922 POINT(40.28124399202619 -74.69860492039385) bank78922 +78923 POINT(40.21302201857752 -74.03006141287452) bank78923 +78924 POINT(41.42767603695145 -74.88214698437315) bank78924 +78925 POINT(40.25639670291582 -74.8091019220261) bank78925 +78926 POINT(41.57256355284742 -74.6315299440547) bank78926 +78927 POINT(40.67191166854072 -73.3035347472471) bank78927 +78928 POINT(41.27699154529818 -73.64581436936967) bank78928 +78929 POINT(41.46312657130902 -73.28826172943322) bank78929 +78930 POINT(40.385614026192705 -73.718183927047) bank78930 +78931 POINT(40.02816259391613 -74.55281078787033) bank78931 +78932 POINT(40.42672512225853 -73.66345356341314) bank78932 +78933 POINT(40.12717343429796 -74.12967607496451) bank78933 +78934 POINT(40.90317494835372 -74.00116431684503) bank78934 +78935 POINT(41.08378315410469 -74.66668803752727) bank78935 +78936 POINT(39.71333249090796 -73.71156247719357) bank78936 +78937 POINT(40.71505910262381 -74.74284338642708) bank78937 +78938 POINT(40.1630761393844 -74.03299160386466) bank78938 +78939 POINT(40.48783216567749 -73.56676319183535) bank78939 +78940 POINT(40.845958575280875 -73.73616535309885) bank78940 +78941 POINT(41.05072567356074 -73.33276270909128) bank78941 +78942 POINT(40.90676407556913 -74.49421933595218) bank78942 +78943 POINT(40.68560176199045 -73.30028981634412) bank78943 +78944 POINT(41.22431798958012 -73.18089599126493) bank78944 +78945 POINT(40.15649242588976 -73.6877239987022) bank78945 +78946 POINT(40.80693456047282 -74.4900300696524) bank78946 +78947 POINT(40.042697509057284 -73.97381750541669) bank78947 +78948 POINT(40.54862459881604 -73.28746189858686) bank78948 +78949 POINT(41.13480384304289 -73.54549397631455) bank78949 +78950 POINT(41.14612828790415 -74.13107651675965) bank78950 +78951 POINT(40.447110038002926 -73.18153496544136) bank78951 +78952 POINT(41.49540584182051 -74.96193563294224) bank78952 +78953 POINT(41.3916522657629 -74.39561267665394) bank78953 +78954 POINT(40.95243241931509 -73.26307122681077) bank78954 +78955 POINT(41.164594399729886 -74.88817672699143) bank78955 +78956 POINT(41.088199062719625 -74.49291172592604) bank78956 +78957 POINT(41.526099335144295 -74.22579581313262) bank78957 +78958 POINT(40.52001826899059 -74.2461293187945) bank78958 +78959 POINT(41.28343495158093 -73.44767443396518) bank78959 +78960 POINT(41.27278959088127 -73.14509529735523) bank78960 +78961 POINT(40.16675653737239 -73.61697868738824) bank78961 +78962 POINT(41.03426534139173 -74.77660057644219) bank78962 +78963 POINT(40.833520263065246 -73.89914155937382) bank78963 +78964 POINT(41.434687805151576 -73.52505975702934) bank78964 +78965 POINT(40.963304153968586 -74.28789968939677) bank78965 +78966 POINT(41.16995563086514 -73.89090029689127) bank78966 +78967 POINT(39.75966286927424 -73.13194646813405) bank78967 +78968 POINT(41.518326885602924 -74.79784558564137) bank78968 +78969 POINT(40.83416521337246 -73.45324040673579) bank78969 +78970 POINT(39.76680670286416 -73.12626796809431) bank78970 +78971 POINT(40.58107173205443 -74.37300279232575) bank78971 +78972 POINT(40.74660722900908 -73.11914728136624) bank78972 +78973 POINT(39.87608792613014 -74.08681566324566) bank78973 +78974 POINT(41.368516025085526 -73.89251079532622) bank78974 +78975 POINT(40.476110947677064 -73.62075331290123) bank78975 +78976 POINT(40.033058417408334 -74.56716417176285) bank78976 +78977 POINT(40.063217336281504 -74.92967025675227) bank78977 +78978 POINT(39.79018360523477 -73.3044166551873) bank78978 +78979 POINT(40.00239632263811 -73.09047358618922) bank78979 +78980 POINT(41.19518395691165 -73.24698441143609) bank78980 +78981 POINT(40.215178628896325 -74.33169119583835) bank78981 +78982 POINT(40.73307146150421 -73.45284298309153) bank78982 +78983 POINT(41.56288986104649 -73.21026398733127) bank78983 +78984 POINT(39.9988667411536 -73.53198308849831) bank78984 +78985 POINT(41.521007930806 -74.2693842526831) bank78985 +78986 POINT(40.11338885394317 -73.56091650896181) bank78986 +78987 POINT(41.12636471148227 -74.71996227770858) bank78987 +78988 POINT(41.417345926924455 -73.42267344955987) bank78988 +78989 POINT(40.04591138205236 -73.10348374568545) bank78989 +78990 POINT(40.90643912725555 -73.67670175012229) bank78990 +78991 POINT(40.445995258687056 -74.25327115552821) bank78991 +78992 POINT(40.231221362355896 -74.70415331190367) bank78992 +78993 POINT(39.968636137552934 -74.36946292594297) bank78993 +78994 POINT(40.22611751061759 -73.55562782974047) bank78994 +78995 POINT(40.243429458757404 -74.16093819957908) bank78995 +78996 POINT(40.04218546258214 -74.8846529489155) bank78996 +78997 POINT(40.733100670006344 -74.75346042583463) bank78997 +78998 POINT(40.59864317012282 -74.67431131693651) bank78998 +78999 POINT(40.713615035106194 -73.71869297015057) bank78999 +79000 POINT(41.35934945706417 -73.98936700749334) bank79000 +79001 POINT(40.28279134879825 -74.76715043159625) bank79001 +79002 POINT(40.72196012680323 -74.34866775805047) bank79002 +79003 POINT(41.332006062299165 -73.71408508009885) bank79003 +79004 POINT(41.41703907665564 -73.55867479677563) bank79004 +79005 POINT(40.46671396244287 -73.718651439859) bank79005 +79006 POINT(39.73433392592449 -73.54091095364396) bank79006 +79007 POINT(40.33465207670745 -74.53473626428678) bank79007 +79008 POINT(39.903251108181415 -74.4095863447829) bank79008 +79009 POINT(40.4680389966275 -73.86377217839834) bank79009 +79010 POINT(40.19065420465784 -74.15293697684179) bank79010 +79011 POINT(40.95281725814179 -74.03134304581624) bank79011 +79012 POINT(41.65653450674759 -74.58233038977158) bank79012 +79013 POINT(40.98442736017595 -73.54548080978697) bank79013 +79014 POINT(41.128207236497836 -74.78118908309462) bank79014 +79015 POINT(40.39535875841498 -73.31809003479901) bank79015 +79016 POINT(40.280047054162544 -73.98506827036775) bank79016 +79017 POINT(41.2029915190199 -73.53785252020933) bank79017 +79018 POINT(40.46382026153191 -74.96089927656347) bank79018 +79019 POINT(40.04393408016412 -74.22244721272168) bank79019 +79020 POINT(39.99168916601975 -74.07414457527926) bank79020 +79021 POINT(39.88788496849423 -74.7083045601163) bank79021 +79022 POINT(40.185820471886224 -73.75605689546386) bank79022 +79023 POINT(39.790394692476205 -74.23196342443713) bank79023 +79024 POINT(41.005506446138874 -74.10104926733308) bank79024 +79025 POINT(41.69613306670697 -73.90914021741334) bank79025 +79026 POINT(41.61822282749268 -73.6009910659716) bank79026 +79027 POINT(39.84779052661817 -73.09321666844042) bank79027 +79028 POINT(40.998506838171075 -74.23471981976827) bank79028 +79029 POINT(41.29052495597254 -73.5031931380016) bank79029 +79030 POINT(41.419613449035786 -74.43182601961333) bank79030 +79031 POINT(40.7925088795107 -74.92671015969792) bank79031 +79032 POINT(41.28469351205891 -74.25985054583374) bank79032 +79033 POINT(40.10203422631864 -74.53125573641306) bank79033 +79034 POINT(40.800882831768604 -73.01901164697244) bank79034 +79035 POINT(40.45743633821961 -74.50078632668631) bank79035 +79036 POINT(40.08492270950725 -74.10804507107072) bank79036 +79037 POINT(39.749591778192574 -74.70073543326224) bank79037 +79038 POINT(39.89113745904225 -73.79385382921114) bank79038 +79039 POINT(41.11604270416093 -74.19411743533811) bank79039 +79040 POINT(39.75335125415796 -73.6085869879562) bank79040 +79041 POINT(40.32216294620036 -73.40289515691863) bank79041 +79042 POINT(40.536512624850324 -73.82598009964326) bank79042 +79043 POINT(41.44646984506793 -74.06888800011907) bank79043 +79044 POINT(39.86305953127686 -73.93509578627265) bank79044 +79045 POINT(41.26575009470351 -73.0086552067365) bank79045 +79046 POINT(40.567195391657464 -74.4900951943637) bank79046 +79047 POINT(40.180442331420565 -73.85864702267163) bank79047 +79048 POINT(40.53213560116289 -74.00686283043068) bank79048 +79049 POINT(41.09978218022842 -73.73508074529012) bank79049 +79050 POINT(40.356723733058566 -74.72652346299832) bank79050 +79051 POINT(40.13617553050542 -74.39134421717092) bank79051 +79052 POINT(41.109804700214475 -73.43105745727021) bank79052 +79053 POINT(40.352080046043376 -74.28638010581987) bank79053 +79054 POINT(40.51164427912126 -73.68549676787714) bank79054 +79055 POINT(41.32630303455388 -74.73645152374351) bank79055 +79056 POINT(40.568274630332134 -74.480711380718) bank79056 +79057 POINT(41.242794731615476 -74.05795757725521) bank79057 +79058 POINT(39.812351956229925 -74.52915658521403) bank79058 +79059 POINT(40.16115913088938 -73.12991173932119) bank79059 +79060 POINT(40.84839918824542 -73.81589769712055) bank79060 +79061 POINT(41.162026793516716 -74.98269030871498) bank79061 +79062 POINT(40.16658316842139 -73.39928149204599) bank79062 +79063 POINT(41.65408848214464 -73.83040694904453) bank79063 +79064 POINT(41.08466953153555 -74.93299052656828) bank79064 +79065 POINT(41.231555471769134 -73.97350404708615) bank79065 +79066 POINT(41.48037497288468 -74.2990673810955) bank79066 +79067 POINT(39.91363176137674 -74.43762328898548) bank79067 +79068 POINT(40.33380173915879 -73.47355633479327) bank79068 +79069 POINT(41.03596747025485 -74.04936031598824) bank79069 +79070 POINT(40.02554390749765 -73.48799398516908) bank79070 +79071 POINT(40.41916576468539 -74.22105796455976) bank79071 +79072 POINT(41.12135591394354 -73.18800588948454) bank79072 +79073 POINT(41.5714941392062 -73.4514217715255) bank79073 +79074 POINT(40.59452072400205 -73.09950284538563) bank79074 +79075 POINT(40.366257651057246 -74.36735336375581) bank79075 +79076 POINT(40.85761373074015 -74.19464921108255) bank79076 +79077 POINT(41.48357856294173 -74.48916207437705) bank79077 +79078 POINT(41.173626982482965 -73.55468261886591) bank79078 +79079 POINT(41.51310145999413 -73.4745651450297) bank79079 +79080 POINT(40.80210538025877 -73.63193431541089) bank79080 +79081 POINT(40.730401731442825 -74.32369057461831) bank79081 +79082 POINT(41.32962993414929 -74.4695844228266) bank79082 +79083 POINT(40.78517904677481 -74.88000540072203) bank79083 +79084 POINT(40.00110457631584 -74.36387810746504) bank79084 +79085 POINT(39.88002240315721 -73.52423354718127) bank79085 +79086 POINT(41.52707059132757 -73.87450678798861) bank79086 +79087 POINT(40.69346519689622 -73.55896910840042) bank79087 +79088 POINT(40.20387350924099 -74.39714385455301) bank79088 +79089 POINT(40.614347923756426 -73.87882089229751) bank79089 +79090 POINT(41.247516923194915 -74.60216125996286) bank79090 +79091 POINT(41.19360009208349 -73.81982655504184) bank79091 +79092 POINT(40.82570879757432 -73.6587033880151) bank79092 +79093 POINT(40.97540847374305 -73.79284776482353) bank79093 +79094 POINT(40.393603716428636 -74.28668631810797) bank79094 +79095 POINT(40.48942842735307 -73.46430556693439) bank79095 +79096 POINT(41.48594828102713 -74.30099237454924) bank79096 +79097 POINT(40.49825079909241 -74.42049620415654) bank79097 +79098 POINT(41.66127290130411 -74.77967322818357) bank79098 +79099 POINT(40.94054482473067 -74.69122213693859) bank79099 +79100 POINT(41.375773485531695 -74.69673251987803) bank79100 +79101 POINT(40.92823659369669 -74.54635688307597) bank79101 +79102 POINT(39.719701557921596 -73.28095693700072) bank79102 +79103 POINT(41.54287628318956 -73.12432970401646) bank79103 +79104 POINT(40.0841131131978 -74.54637592376321) bank79104 +79105 POINT(41.336635512857484 -74.50182622580827) bank79105 +79106 POINT(41.276533944812094 -73.09594971131914) bank79106 +79107 POINT(40.92705875546963 -73.98975064544373) bank79107 +79108 POINT(40.595949808951346 -73.2844013071288) bank79108 +79109 POINT(41.45263113567097 -74.16135118828471) bank79109 +79110 POINT(41.07074456892385 -73.28539635947551) bank79110 +79111 POINT(39.89859879533071 -74.11827648804298) bank79111 +79112 POINT(41.04109573573717 -74.57470055268078) bank79112 +79113 POINT(41.67594980330088 -74.38155773993734) bank79113 +79114 POINT(40.3415023335716 -73.0739452448245) bank79114 +79115 POINT(39.77463270782824 -73.1200372567966) bank79115 +79116 POINT(39.774241655534695 -73.39531192799818) bank79116 +79117 POINT(40.51443446663292 -74.43400260592361) bank79117 +79118 POINT(40.47567803297588 -73.12694018513719) bank79118 +79119 POINT(41.16748586823908 -74.33745490292792) bank79119 +79120 POINT(40.43403075027536 -73.49697817889435) bank79120 +79121 POINT(41.27849549257712 -73.40467032587854) bank79121 +79122 POINT(39.97470738327393 -74.76090078899398) bank79122 +79123 POINT(40.299249989802064 -73.19625573594853) bank79123 +79124 POINT(40.627814363473696 -73.23677290478119) bank79124 +79125 POINT(40.30550649164173 -74.79245975327679) bank79125 +79126 POINT(41.11114437396893 -74.92019666676393) bank79126 +79127 POINT(41.581386801697 -74.13862817225295) bank79127 +79128 POINT(41.25341269344505 -74.43363840403777) bank79128 +79129 POINT(40.20843830436109 -73.6995682977546) bank79129 +79130 POINT(40.66335958473934 -74.30505583419601) bank79130 +79131 POINT(41.508775699836924 -73.6761774399829) bank79131 +79132 POINT(40.63491947486195 -73.98751715275533) bank79132 +79133 POINT(39.911350473578274 -73.58717427113503) bank79133 +79134 POINT(40.51857545119483 -73.9630685123834) bank79134 +79135 POINT(39.81719348629204 -74.5971049326432) bank79135 +79136 POINT(40.067529498330394 -74.53410998200359) bank79136 +79137 POINT(41.51530386239029 -73.2884990951809) bank79137 +79138 POINT(41.504245398350285 -73.06319020003538) bank79138 +79139 POINT(40.674108309967785 -73.2450767322915) bank79139 +79140 POINT(40.978761011874596 -74.27596528970844) bank79140 +79141 POINT(41.023123999730586 -73.28980580966548) bank79141 +79142 POINT(40.00504407698831 -73.14470904267937) bank79142 +79143 POINT(41.21675285415248 -74.67224209641928) bank79143 +79144 POINT(41.33388060119313 -74.46424923356072) bank79144 +79145 POINT(41.29720282149097 -73.54143565412801) bank79145 +79146 POINT(39.767627697639846 -73.55993322603864) bank79146 +79147 POINT(40.17397540950124 -74.23756336899667) bank79147 +79148 POINT(41.07982881190057 -73.90815035078859) bank79148 +79149 POINT(40.816813534346075 -74.9031448241017) bank79149 +79150 POINT(41.29763728974672 -73.83113952061815) bank79150 +79151 POINT(39.78402455423464 -73.94613818371454) bank79151 +79152 POINT(40.07952410686545 -74.70851150845024) bank79152 +79153 POINT(40.68233846888767 -74.71496797567669) bank79153 +79154 POINT(41.19340479349044 -73.18573102791974) bank79154 +79155 POINT(40.45413241313961 -74.19515012552584) bank79155 +79156 POINT(41.51232171059942 -73.87690777610969) bank79156 +79157 POINT(40.619115150903866 -73.82771032241929) bank79157 +79158 POINT(40.23021987874255 -74.42337949217607) bank79158 +79159 POINT(40.479703823640016 -73.12835225125623) bank79159 +79160 POINT(41.31285415818337 -73.54768691642192) bank79160 +79161 POINT(40.32811442703737 -74.93251299783975) bank79161 +79162 POINT(40.680571369848515 -73.4381876403469) bank79162 +79163 POINT(41.38203427770667 -73.14145057149845) bank79163 +79164 POINT(39.77058194670907 -74.38468431140181) bank79164 +79165 POINT(40.533450824957896 -73.24241513610941) bank79165 +79166 POINT(41.511298543811336 -73.78952821755317) bank79166 +79167 POINT(40.27990798208256 -74.75854711502801) bank79167 +79168 POINT(41.665090308029875 -73.21961810980521) bank79168 +79169 POINT(41.6092766649458 -74.35875337765027) bank79169 +79170 POINT(40.638385350254715 -73.54917345947256) bank79170 +79171 POINT(40.19452854770649 -73.37233082453939) bank79171 +79172 POINT(39.77578050566167 -73.15124509144886) bank79172 +79173 POINT(41.33770617779337 -74.8744271816914) bank79173 +79174 POINT(41.632093573934426 -73.59604946124807) bank79174 +79175 POINT(41.05618975019353 -73.09633106971216) bank79175 +79176 POINT(39.859125435687595 -74.33388977620297) bank79176 +79177 POINT(40.60485998992435 -73.8462077855238) bank79177 +79178 POINT(40.062070063966985 -73.13258784647061) bank79178 +79179 POINT(39.996745297138595 -74.90005746740127) bank79179 +79180 POINT(39.85361001749197 -73.47997746367054) bank79180 +79181 POINT(40.82137360175686 -73.61748294357078) bank79181 +79182 POINT(40.26823075053755 -74.43963446156482) bank79182 +79183 POINT(41.563240151944804 -74.54056724030241) bank79183 +79184 POINT(39.85673457426404 -74.32554663503706) bank79184 +79185 POINT(40.924285365650846 -74.58206340837118) bank79185 +79186 POINT(40.963140639594364 -73.1087658161618) bank79186 +79187 POINT(40.819210658866005 -73.62663406625445) bank79187 +79188 POINT(41.675090104212515 -74.50245348662155) bank79188 +79189 POINT(40.16739583760053 -73.15885655074898) bank79189 +79190 POINT(39.77154022273894 -73.51582293265932) bank79190 +79191 POINT(40.51696904928979 -74.29902511885251) bank79191 +79192 POINT(39.800529730059466 -73.40451823356027) bank79192 +79193 POINT(41.457708786356335 -73.87311334777115) bank79193 +79194 POINT(39.919960224968094 -74.58133950099524) bank79194 +79195 POINT(40.6905622432408 -73.63727031298956) bank79195 +79196 POINT(41.18140411647055 -74.15049548931101) bank79196 +79197 POINT(39.83699604740447 -73.4024176491716) bank79197 +79198 POINT(41.26357935317091 -74.36609151400515) bank79198 +79199 POINT(41.2758832643168 -74.3774127936488) bank79199 +79200 POINT(40.964058023347256 -74.87708901445689) bank79200 +79201 POINT(41.180107645867814 -74.30122066692348) bank79201 +79202 POINT(40.806915914908814 -74.62081664903492) bank79202 +79203 POINT(40.98901288333224 -73.2343310945619) bank79203 +79204 POINT(40.7200129967708 -73.49372518708347) bank79204 +79205 POINT(40.569506536588854 -74.47370438783388) bank79205 +79206 POINT(41.34978087727903 -74.01128658582226) bank79206 +79207 POINT(39.9760593841863 -73.62427942693064) bank79207 +79208 POINT(41.05739765460037 -73.37048115909151) bank79208 +79209 POINT(40.52889844871727 -74.59184829193833) bank79209 +79210 POINT(39.81951597353461 -74.5422316180405) bank79210 +79211 POINT(39.73162015142518 -74.02831408834884) bank79211 +79212 POINT(40.64714753800233 -73.53201853485244) bank79212 +79213 POINT(41.31056002686508 -74.42706536199378) bank79213 +79214 POINT(40.19685857216968 -73.91975103310045) bank79214 +79215 POINT(39.888643394245605 -74.62499593251101) bank79215 +79216 POINT(40.87590926153343 -74.2938926345288) bank79216 +79217 POINT(40.84483625854874 -73.4652866730316) bank79217 +79218 POINT(41.37667157355632 -74.58347860752713) bank79218 +79219 POINT(40.342898196465974 -74.97243057646038) bank79219 +79220 POINT(41.475405236462116 -74.18826872221523) bank79220 +79221 POINT(41.50121206492671 -73.7581528774268) bank79221 +79222 POINT(40.40311766744905 -74.67085012062812) bank79222 +79223 POINT(39.76670255119153 -73.85492499647687) bank79223 +79224 POINT(41.691981279059924 -73.09010190971497) bank79224 +79225 POINT(40.37135246406155 -73.57985406280802) bank79225 +79226 POINT(40.63332120992078 -73.70839670738721) bank79226 +79227 POINT(39.8286171481763 -73.70933926814952) bank79227 +79228 POINT(40.89205906312219 -74.33455995639784) bank79228 +79229 POINT(40.02622789412414 -74.56792936537684) bank79229 +79230 POINT(41.30977178657154 -74.7632711591235) bank79230 +79231 POINT(39.72389942523042 -74.77914069272619) bank79231 +79232 POINT(40.095208897843754 -73.07691706332083) bank79232 +79233 POINT(39.72011317765479 -73.04809622555621) bank79233 +79234 POINT(40.24091984523133 -74.48783929924811) bank79234 +79235 POINT(40.824598425012546 -74.30991191245961) bank79235 +79236 POINT(40.916812294681556 -73.67302658771085) bank79236 +79237 POINT(40.45382448804027 -74.93786924596175) bank79237 +79238 POINT(40.84269764056621 -74.20346318150695) bank79238 +79239 POINT(40.39550898501277 -73.52122078350766) bank79239 +79240 POINT(40.79880856189678 -74.32101600158245) bank79240 +79241 POINT(40.28031042097145 -74.84138753513308) bank79241 +79242 POINT(41.327045926220414 -74.96798619257511) bank79242 +79243 POINT(40.30820887693299 -73.22502599617246) bank79243 +79244 POINT(40.55883763362287 -74.20334452188396) bank79244 +79245 POINT(40.12278990810472 -73.19276880661009) bank79245 +79246 POINT(40.658549914827375 -73.21174424063797) bank79246 +79247 POINT(41.17080538237858 -74.43016649925006) bank79247 +79248 POINT(39.94116939814738 -74.83276637326892) bank79248 +79249 POINT(40.337391896932516 -73.78675139933807) bank79249 +79250 POINT(40.13608935243978 -74.04843964767853) bank79250 +79251 POINT(40.06448661734427 -74.64005080886257) bank79251 +79252 POINT(40.89673792878751 -74.92246007462221) bank79252 +79253 POINT(39.96984132618889 -73.98227385784973) bank79253 +79254 POINT(41.36730071244088 -74.96869350342728) bank79254 +79255 POINT(41.70237564078859 -73.53542945644546) bank79255 +79256 POINT(41.402436960140754 -74.79982115609495) bank79256 +79257 POINT(39.99586119192332 -73.86896243098346) bank79257 +79258 POINT(39.777255262867946 -73.95192352848922) bank79258 +79259 POINT(41.443853525264515 -73.73379857367803) bank79259 +79260 POINT(40.16057969424868 -73.2717202389868) bank79260 +79261 POINT(40.016887399893285 -74.05283786216054) bank79261 +79262 POINT(40.37982953915028 -74.61795860239421) bank79262 +79263 POINT(39.99917838475167 -74.7172937144499) bank79263 +79264 POINT(39.871418340051186 -74.83684836270373) bank79264 +79265 POINT(40.33146745930115 -73.22349208220221) bank79265 +79266 POINT(40.34025931100739 -73.15730220597214) bank79266 +79267 POINT(40.27086470994582 -73.10732644988076) bank79267 +79268 POINT(41.192906531691875 -74.43501858053952) bank79268 +79269 POINT(39.78162527718458 -74.24419269154329) bank79269 +79270 POINT(40.300778826332056 -73.0080234299902) bank79270 +79271 POINT(41.58307361129929 -74.76930584406007) bank79271 +79272 POINT(39.802051708972314 -74.15215934542194) bank79272 +79273 POINT(41.373951539597236 -73.9706463778244) bank79273 +79274 POINT(39.80099685417657 -74.78699381056728) bank79274 +79275 POINT(40.442541614649315 -74.73523126622526) bank79275 +79276 POINT(40.39575699776086 -73.30162255580278) bank79276 +79277 POINT(39.73495673404475 -74.79099939684559) bank79277 +79278 POINT(40.8979307863621 -74.82260684673315) bank79278 +79279 POINT(41.63187568640482 -73.972338280742) bank79279 +79280 POINT(40.77877673479459 -73.28170374744124) bank79280 +79281 POINT(41.44459053062064 -74.20164143576872) bank79281 +79282 POINT(41.169844856933196 -74.17414592324252) bank79282 +79283 POINT(40.86680864033939 -73.51942649204715) bank79283 +79284 POINT(40.54205906344032 -74.80163170234036) bank79284 +79285 POINT(40.11339837203191 -73.83716348763973) bank79285 +79286 POINT(41.39396504901916 -74.75870552275869) bank79286 +79287 POINT(40.122423490403726 -74.50529676481545) bank79287 +79288 POINT(41.645726475156415 -73.19045744541218) bank79288 +79289 POINT(40.87841287735328 -73.62928163584243) bank79289 +79290 POINT(40.252334740921484 -74.33053071092567) bank79290 +79291 POINT(40.318888371440785 -73.81983889324663) bank79291 +79292 POINT(40.368500938984724 -73.31807768593517) bank79292 +79293 POINT(40.73007646564 -74.97651686440004) bank79293 +79294 POINT(40.786265687804644 -74.8169142905331) bank79294 +79295 POINT(40.90290439903215 -74.46275107778682) bank79295 +79296 POINT(39.821297647820295 -74.74436438635598) bank79296 +79297 POINT(41.339834283318545 -74.14907693150253) bank79297 +79298 POINT(40.8517905876758 -73.3394492968371) bank79298 +79299 POINT(39.77827966639477 -74.6808071942715) bank79299 +79300 POINT(40.702299872530425 -74.5724918915309) bank79300 +79301 POINT(40.62764407097001 -74.93546002180302) bank79301 +79302 POINT(41.24924705672326 -73.74701818150866) bank79302 +79303 POINT(39.79970031735968 -73.27377475262863) bank79303 +79304 POINT(40.61763344000893 -74.07856243212197) bank79304 +79305 POINT(39.99017162155072 -73.27491184030994) bank79305 +79306 POINT(40.45514061176705 -74.97662081464097) bank79306 +79307 POINT(40.83429174677808 -74.75076641541403) bank79307 +79308 POINT(41.05484562337136 -73.27407668655671) bank79308 +79309 POINT(40.48510011168556 -74.9916243413141) bank79309 +79310 POINT(40.82563678105794 -74.93051919448386) bank79310 +79311 POINT(41.10640536354104 -74.29659462873018) bank79311 +79312 POINT(41.19799812506368 -73.69526007472632) bank79312 +79313 POINT(40.014402170700464 -73.67914486479995) bank79313 +79314 POINT(41.16385135322555 -74.34148436715854) bank79314 +79315 POINT(41.603159894771174 -74.79965673140035) bank79315 +79316 POINT(40.501606512738405 -74.44919063878419) bank79316 +79317 POINT(39.89702898226653 -74.52890132869197) bank79317 +79318 POINT(40.12889479817894 -74.3431292685643) bank79318 +79319 POINT(40.15718394857454 -73.36138095112344) bank79319 +79320 POINT(40.39857898408327 -73.51229470240733) bank79320 +79321 POINT(40.670036559417085 -74.37231076987686) bank79321 +79322 POINT(40.861435557910106 -73.06000416286628) bank79322 +79323 POINT(41.19152116708785 -74.72562535171977) bank79323 +79324 POINT(39.76724822401125 -74.97283407588658) bank79324 +79325 POINT(41.53668820143602 -74.84048175014404) bank79325 +79326 POINT(39.87665551151745 -73.40818521167199) bank79326 +79327 POINT(41.38578833434842 -74.53069585833228) bank79327 +79328 POINT(40.195389265007265 -74.44850667558585) bank79328 +79329 POINT(41.11593026597443 -73.04494585335026) bank79329 +79330 POINT(40.518248506387245 -73.85374974990727) bank79330 +79331 POINT(39.869392299761785 -73.87774773972298) bank79331 +79332 POINT(40.93601490666546 -73.37275594447749) bank79332 +79333 POINT(41.22917471725062 -74.88321774125895) bank79333 +79334 POINT(40.67871900169085 -73.76896055868838) bank79334 +79335 POINT(41.355023249485086 -74.14744516250128) bank79335 +79336 POINT(41.562176243443545 -73.71120747286352) bank79336 +79337 POINT(40.796992126092974 -73.66402092897253) bank79337 +79338 POINT(41.499998358513764 -74.12962123120856) bank79338 +79339 POINT(41.19072754911068 -74.90967503231136) bank79339 +79340 POINT(39.797629117672365 -73.1886294560828) bank79340 +79341 POINT(40.39335494708725 -73.22308626214166) bank79341 +79342 POINT(39.73315564394322 -74.53057304381375) bank79342 +79343 POINT(40.981510997624994 -74.6008021705502) bank79343 +79344 POINT(40.17365464640054 -73.2085281415542) bank79344 +79345 POINT(41.19003476778591 -74.67873559173309) bank79345 +79346 POINT(40.8797543574709 -73.96491472026588) bank79346 +79347 POINT(41.4243991103468 -74.60843014858153) bank79347 +79348 POINT(40.83761391097137 -74.54014376923945) bank79348 +79349 POINT(41.27955668356185 -74.56493507501295) bank79349 +79350 POINT(40.99540546706546 -74.00315394094832) bank79350 +79351 POINT(41.279336711836066 -74.79520848956523) bank79351 +79352 POINT(41.461692598999754 -74.6878023707057) bank79352 +79353 POINT(41.2910726339238 -74.7844054052342) bank79353 +79354 POINT(41.697696186108836 -74.41988670180751) bank79354 +79355 POINT(40.210435034577806 -73.08895479767669) bank79355 +79356 POINT(41.437753258828565 -74.55018938560562) bank79356 +79357 POINT(41.14943868390573 -73.2863056232313) bank79357 +79358 POINT(41.1287706125362 -74.27170958283504) bank79358 +79359 POINT(40.46486942258677 -74.55918489893756) bank79359 +79360 POINT(39.85311110495973 -73.33733633049764) bank79360 +79361 POINT(39.81054577637711 -73.12881633670357) bank79361 +79362 POINT(41.29999107492688 -74.18064514627493) bank79362 +79363 POINT(41.66295002393993 -74.0184789467215) bank79363 +79364 POINT(41.06704189037622 -74.6396616789803) bank79364 +79365 POINT(40.051332652506616 -73.93250120580662) bank79365 +79366 POINT(39.90228928066899 -73.46548555846101) bank79366 +79367 POINT(40.87674815136374 -73.7725034778213) bank79367 +79368 POINT(41.619437640808464 -73.90132202135018) bank79368 +79369 POINT(39.784151504313364 -74.06513702797916) bank79369 +79370 POINT(41.17950182918583 -73.74087853156693) bank79370 +79371 POINT(41.384385856790864 -73.69750101204058) bank79371 +79372 POINT(40.78002028600917 -74.33054560656558) bank79372 +79373 POINT(40.76938196302964 -74.9514803692632) bank79373 +79374 POINT(41.686190222786834 -74.19303313217476) bank79374 +79375 POINT(40.60405910353498 -73.66021107069938) bank79375 +79376 POINT(41.16847963045397 -73.4666010254104) bank79376 +79377 POINT(41.26726643142873 -73.91484204376573) bank79377 +79378 POINT(41.68221659732796 -73.55612880688477) bank79378 +79379 POINT(41.04144600493497 -73.51851969293602) bank79379 +79380 POINT(41.434715817844825 -73.77915605343921) bank79380 +79381 POINT(41.22129634576208 -74.95445753399756) bank79381 +79382 POINT(41.355616357763985 -74.30542910201133) bank79382 +79383 POINT(40.40758960113789 -74.99295197971595) bank79383 +79384 POINT(41.1322788441409 -73.28831518678398) bank79384 +79385 POINT(41.435670293440445 -74.84578106227973) bank79385 +79386 POINT(41.1311152487151 -73.5778243293234) bank79386 +79387 POINT(39.76978709523876 -73.88857118304217) bank79387 +79388 POINT(40.74621270023337 -74.77665691095191) bank79388 +79389 POINT(40.233724533009266 -73.48689425542345) bank79389 +79390 POINT(40.87110651240046 -74.97731020299375) bank79390 +79391 POINT(41.525379663127296 -73.92949179660586) bank79391 +79392 POINT(41.43889188238806 -73.72430895563599) bank79392 +79393 POINT(41.5340574884702 -73.1410786890184) bank79393 +79394 POINT(41.32009093399352 -74.91120362341996) bank79394 +79395 POINT(39.76993737407837 -73.76991018894644) bank79395 +79396 POINT(41.44677849288288 -74.63153103504678) bank79396 +79397 POINT(40.68791419127648 -74.24585531142945) bank79397 +79398 POINT(40.503498069892615 -73.83903440918667) bank79398 +79399 POINT(41.3277957186905 -73.34151112182687) bank79399 +79400 POINT(40.64229541151631 -74.95039157134194) bank79400 +79401 POINT(40.12162954434298 -73.75595366919771) bank79401 +79402 POINT(40.02163015747359 -73.49912860476888) bank79402 +79403 POINT(40.27389641123348 -74.45320291133584) bank79403 +79404 POINT(41.57090500269129 -73.38959297638957) bank79404 +79405 POINT(40.35884788078971 -74.32627515050865) bank79405 +79406 POINT(41.626588490876514 -74.79518367966556) bank79406 +79407 POINT(41.144546730192914 -74.84279810937832) bank79407 +79408 POINT(41.239217279726226 -74.78025068173375) bank79408 +79409 POINT(41.28298956847894 -74.72069383958788) bank79409 +79410 POINT(41.2524714470312 -74.15387525091435) bank79410 +79411 POINT(41.173691236130736 -74.53250928930701) bank79411 +79412 POINT(40.099617050308254 -74.53808317371602) bank79412 +79413 POINT(40.15235464127067 -73.55755654443941) bank79413 +79414 POINT(39.777624098895174 -74.30398936325004) bank79414 +79415 POINT(40.50685424715969 -73.77212564935576) bank79415 +79416 POINT(41.06384255149403 -74.65869651545677) bank79416 +79417 POINT(40.89058766536778 -73.67769253877483) bank79417 +79418 POINT(40.55262020894986 -74.81466238096506) bank79418 +79419 POINT(40.518630007455606 -74.55130302648696) bank79419 +79420 POINT(40.229558090506934 -73.0816012541502) bank79420 +79421 POINT(40.04312818846935 -74.03237908732606) bank79421 +79422 POINT(41.44647421342883 -74.89742926550994) bank79422 +79423 POINT(41.12114098141508 -73.70853416334015) bank79423 +79424 POINT(40.788616460262936 -73.06954047717882) bank79424 +79425 POINT(40.565513245281934 -73.34155013656536) bank79425 +79426 POINT(41.65831471979587 -74.56131602875998) bank79426 +79427 POINT(41.41303324853178 -73.44926440113093) bank79427 +79428 POINT(41.11820957896522 -73.5245786982336) bank79428 +79429 POINT(41.35925554767336 -73.37172886495102) bank79429 +79430 POINT(40.58837434902772 -73.41248679905684) bank79430 +79431 POINT(40.364592065165404 -73.06299481952065) bank79431 +79432 POINT(40.90074981134974 -73.13580313257968) bank79432 +79433 POINT(39.942979386056564 -73.17678412889374) bank79433 +79434 POINT(41.47662903224529 -73.81813461975018) bank79434 +79435 POINT(39.72568125431341 -74.61970785815325) bank79435 +79436 POINT(40.88330640864522 -74.43047854612848) bank79436 +79437 POINT(40.03576918816518 -74.75300746513908) bank79437 +79438 POINT(41.35888765056701 -74.97221267209288) bank79438 +79439 POINT(41.1078231204053 -74.02115847679725) bank79439 +79440 POINT(40.90843271646947 -74.75145006283961) bank79440 +79441 POINT(40.92579644890405 -74.01491739803107) bank79441 +79442 POINT(40.08418511270683 -73.07514874569891) bank79442 +79443 POINT(39.79477968137969 -74.16827243643792) bank79443 +79444 POINT(41.34357880513149 -73.97080548322921) bank79444 +79445 POINT(40.35728322301744 -74.15128607247303) bank79445 +79446 POINT(41.120309964224845 -73.5919935238037) bank79446 +79447 POINT(41.60887412294538 -73.88118645876791) bank79447 +79448 POINT(40.65345280496325 -73.7620181717374) bank79448 +79449 POINT(41.37348975403541 -73.05348320269542) bank79449 +79450 POINT(41.66085721152507 -73.67320225141954) bank79450 +79451 POINT(39.74733551188942 -74.32073978001266) bank79451 +79452 POINT(40.513087696209666 -74.92307133534659) bank79452 +79453 POINT(40.933161315641264 -74.58452191759643) bank79453 +79454 POINT(40.18356548878612 -73.1697441951195) bank79454 +79455 POINT(40.174531356221614 -73.12441283413462) bank79455 +79456 POINT(40.47482600655861 -73.5877603828848) bank79456 +79457 POINT(39.850838213046075 -74.14977808435229) bank79457 +79458 POINT(39.83760376272142 -74.65338861347982) bank79458 +79459 POINT(39.83882274351963 -74.46080949933985) bank79459 +79460 POINT(39.77263536521529 -73.8959081278431) bank79460 +79461 POINT(40.15895527110054 -73.43980530131563) bank79461 +79462 POINT(41.70213045736165 -73.6426472553991) bank79462 +79463 POINT(39.8771343514139 -74.11838287803157) bank79463 +79464 POINT(40.061383466005545 -73.12096025520633) bank79464 +79465 POINT(40.68962214181497 -73.29504921264173) bank79465 +79466 POINT(40.33230200959032 -74.9868785991512) bank79466 +79467 POINT(40.2864320634048 -73.25326370553202) bank79467 +79468 POINT(39.72838047224663 -73.96510084992555) bank79468 +79469 POINT(41.60952487866461 -74.21407497545444) bank79469 +79470 POINT(41.51963832525483 -73.62836125817418) bank79470 +79471 POINT(40.557661517391395 -73.63571650740475) bank79471 +79472 POINT(41.43782980036978 -74.00557727426266) bank79472 +79473 POINT(41.591716787544655 -74.54721882764882) bank79473 +79474 POINT(40.654311183870036 -74.27630454632545) bank79474 +79475 POINT(40.136447181845654 -73.45934247902444) bank79475 +79476 POINT(40.620590770893585 -74.53000586270215) bank79476 +79477 POINT(40.59779553564428 -74.02118596700646) bank79477 +79478 POINT(41.18075146684588 -74.35428417729435) bank79478 +79479 POINT(41.18590853602046 -73.64488110580142) bank79479 +79480 POINT(40.35690395642853 -73.93669621583949) bank79480 +79481 POINT(40.08304594954208 -73.23871002746127) bank79481 +79482 POINT(41.35566984984345 -74.7217126722462) bank79482 +79483 POINT(41.63456235509088 -74.25091493549067) bank79483 +79484 POINT(41.12378415525779 -73.70654234687971) bank79484 +79485 POINT(39.84317333900662 -73.9632789594091) bank79485 +79486 POINT(40.246249012124906 -73.26308736119162) bank79486 +79487 POINT(41.25203073622449 -74.05228672189514) bank79487 +79488 POINT(41.468484426474916 -74.72171060985876) bank79488 +79489 POINT(39.86680467448044 -74.22027270112856) bank79489 +79490 POINT(39.84069340685813 -74.39767254227769) bank79490 +79491 POINT(41.69153493396279 -74.37978299191653) bank79491 +79492 POINT(40.028990224922524 -74.24782798922939) bank79492 +79493 POINT(41.245200126906084 -73.24865497621239) bank79493 +79494 POINT(40.83197950705223 -73.96262759627884) bank79494 +79495 POINT(41.5080059007476 -74.22340879461956) bank79495 +79496 POINT(39.926823521630645 -74.7732382260196) bank79496 +79497 POINT(39.79421731041274 -74.68188449672449) bank79497 +79498 POINT(40.27332688575344 -73.03692398970294) bank79498 +79499 POINT(41.611906541627526 -74.70664074339302) bank79499 +79500 POINT(41.3102876959285 -74.20840616381483) bank79500 +79501 POINT(40.02777238200847 -74.63792522995169) bank79501 +79502 POINT(40.13747773735372 -73.92852741728699) bank79502 +79503 POINT(40.41107016262102 -74.6518167755959) bank79503 +79504 POINT(41.142592286102364 -74.62325050992082) bank79504 +79505 POINT(40.78227474364246 -73.77094376325974) bank79505 +79506 POINT(41.49584289990131 -74.31054653217632) bank79506 +79507 POINT(39.72938393812868 -73.89359725600065) bank79507 +79508 POINT(41.29543192570703 -74.85214127849872) bank79508 +79509 POINT(41.54057155192987 -74.84858752903995) bank79509 +79510 POINT(39.74181341452095 -73.552059112777) bank79510 +79511 POINT(39.811226845123 -74.53446313900588) bank79511 +79512 POINT(40.43669991191398 -74.58818433943566) bank79512 +79513 POINT(41.301292805732075 -74.83248973003917) bank79513 +79514 POINT(41.038956225107896 -74.68593499249221) bank79514 +79515 POINT(39.84931261997099 -73.88526256168808) bank79515 +79516 POINT(39.88195340037609 -74.9522241174476) bank79516 +79517 POINT(41.3129871018029 -73.87278158823617) bank79517 +79518 POINT(39.716191981716115 -74.8365819738409) bank79518 +79519 POINT(40.67400392998724 -74.37500939167884) bank79519 +79520 POINT(40.63005838339769 -74.62737118702942) bank79520 +79521 POINT(41.01441454038352 -74.59875262394307) bank79521 +79522 POINT(41.62042451079088 -73.91740614500507) bank79522 +79523 POINT(40.52001430967891 -74.10623676327806) bank79523 +79524 POINT(41.12478343741805 -74.5464480738303) bank79524 +79525 POINT(41.31814057483505 -74.53722502622144) bank79525 +79526 POINT(40.15814089247577 -73.49533528283406) bank79526 +79527 POINT(40.75706504319165 -73.36352886196153) bank79527 +79528 POINT(41.20073305896887 -74.42835014715219) bank79528 +79529 POINT(40.71371557531599 -74.73675319036245) bank79529 +79530 POINT(41.16734771338729 -74.76621546333466) bank79530 +79531 POINT(40.95829003137531 -74.04655053464472) bank79531 +79532 POINT(40.7410661851154 -74.76144623053408) bank79532 +79533 POINT(40.61714313746554 -74.24604471729573) bank79533 +79534 POINT(40.751027150072666 -73.46041932376994) bank79534 +79535 POINT(41.618400457192195 -73.86811424708304) bank79535 +79536 POINT(40.72869217897161 -74.95680097381602) bank79536 +79537 POINT(41.43227887057501 -73.86195095794339) bank79537 +79538 POINT(41.46071807285813 -74.96735135688162) bank79538 +79539 POINT(40.376783516169624 -73.17582928102834) bank79539 +79540 POINT(41.012480756857194 -73.16156584289934) bank79540 +79541 POINT(41.609158986174464 -74.43998647483154) bank79541 +79542 POINT(41.41979567451148 -73.20569285731442) bank79542 +79543 POINT(40.075669534369446 -73.8230441771159) bank79543 +79544 POINT(39.999880796418005 -74.84930709551838) bank79544 +79545 POINT(40.16027604229377 -73.11193025148478) bank79545 +79546 POINT(40.79291099813843 -73.51118826554081) bank79546 +79547 POINT(39.751415096892195 -74.22530611014231) bank79547 +79548 POINT(41.36255262399937 -74.72829230943394) bank79548 +79549 POINT(41.34733728852258 -74.53912404475662) bank79549 +79550 POINT(41.48247286847056 -73.15942040474849) bank79550 +79551 POINT(40.45210631376324 -74.52920125738014) bank79551 +79552 POINT(41.64422458863679 -73.36878317460616) bank79552 +79553 POINT(39.770287892598944 -73.33422358882633) bank79553 +79554 POINT(40.5667238791358 -74.72978425776812) bank79554 +79555 POINT(41.108990430710506 -74.69512566528812) bank79555 +79556 POINT(40.49670065577089 -74.50214258454753) bank79556 +79557 POINT(41.20119431490643 -74.49774524362999) bank79557 +79558 POINT(39.98408247235125 -74.6666415806366) bank79558 +79559 POINT(40.007421680579874 -74.83884136329384) bank79559 +79560 POINT(41.58296501065124 -73.00649970967882) bank79560 +79561 POINT(40.605462283268324 -74.66127952306117) bank79561 +79562 POINT(40.96827033514096 -74.22143511351153) bank79562 +79563 POINT(40.798519549876104 -74.71593744114995) bank79563 +79564 POINT(41.63004480017186 -74.19336655316286) bank79564 +79565 POINT(41.63349745739836 -73.59845820432129) bank79565 +79566 POINT(41.220964802402975 -74.13391001706452) bank79566 +79567 POINT(41.7078019890523 -74.20748368597549) bank79567 +79568 POINT(40.51553042997622 -73.62643983550595) bank79568 +79569 POINT(40.456397044621475 -74.89880381293452) bank79569 +79570 POINT(40.9739486761748 -74.88921967845255) bank79570 +79571 POINT(40.98000548322932 -74.0204964393139) bank79571 +79572 POINT(41.62152541924154 -73.74731093739992) bank79572 +79573 POINT(41.102278604839064 -74.04907129340478) bank79573 +79574 POINT(41.65818798826953 -73.09779801007308) bank79574 +79575 POINT(40.14146459790262 -73.52452850435772) bank79575 +79576 POINT(39.89791413601226 -73.3424940340543) bank79576 +79577 POINT(41.48281200149909 -74.277680799666) bank79577 +79578 POINT(40.49476967901854 -73.65987721945419) bank79578 +79579 POINT(40.25505894422687 -73.59855512657433) bank79579 +79580 POINT(40.363159369084165 -73.08354097843483) bank79580 +79581 POINT(39.800522134905776 -74.49732699464002) bank79581 +79582 POINT(39.74828215497316 -74.23591693220607) bank79582 +79583 POINT(41.22913000628146 -73.44088911519323) bank79583 +79584 POINT(41.0844515481505 -74.4584046191164) bank79584 +79585 POINT(40.38622388939211 -73.40031484175567) bank79585 +79586 POINT(41.01275928290323 -74.50645329733858) bank79586 +79587 POINT(39.862798637705126 -73.41805532434928) bank79587 +79588 POINT(39.77234557847215 -73.5176989292007) bank79588 +79589 POINT(40.23587676670864 -73.20250216306488) bank79589 +79590 POINT(39.91924096440367 -73.87717208823815) bank79590 +79591 POINT(40.291203963395404 -73.8954823481028) bank79591 +79592 POINT(40.40487767347206 -73.3547903036823) bank79592 +79593 POINT(39.98707991710292 -73.21105436763673) bank79593 +79594 POINT(41.648253414998415 -74.50878911512027) bank79594 +79595 POINT(41.512720105070464 -73.62027019520774) bank79595 +79596 POINT(40.15605670292867 -74.25184959111282) bank79596 +79597 POINT(40.11852347578988 -74.39749140412532) bank79597 +79598 POINT(39.8656425067419 -74.22720256196797) bank79598 +79599 POINT(40.26019569898202 -73.91199421236328) bank79599 +79600 POINT(41.074210527952395 -74.72454145538545) bank79600 +79601 POINT(41.52513116900801 -74.16977520853517) bank79601 +79602 POINT(40.9371897115171 -74.21728330752585) bank79602 +79603 POINT(40.60227763634592 -73.6345578715516) bank79603 +79604 POINT(39.94094439910135 -74.7278076419278) bank79604 +79605 POINT(41.253276813140495 -73.91334529228158) bank79605 +79606 POINT(40.63772501373763 -73.2789284616078) bank79606 +79607 POINT(40.331208919505315 -74.0184742031693) bank79607 +79608 POINT(40.54313093144559 -73.04256060508322) bank79608 +79609 POINT(41.53256813664583 -74.25677507932042) bank79609 +79610 POINT(40.0945848016204 -74.29978393278168) bank79610 +79611 POINT(41.36686203552916 -73.0709482510243) bank79611 +79612 POINT(40.504221305453825 -74.82109451346291) bank79612 +79613 POINT(40.916370412941546 -74.81402792054341) bank79613 +79614 POINT(39.765741448662524 -73.77260647222757) bank79614 +79615 POINT(41.52144958786004 -74.5346504028048) bank79615 +79616 POINT(40.71682154013459 -74.02079081011232) bank79616 +79617 POINT(40.95678872864695 -74.24009131146201) bank79617 +79618 POINT(40.79330455732559 -73.67063227669948) bank79618 +79619 POINT(40.686729769734214 -73.61340179401218) bank79619 +79620 POINT(41.650567647332764 -74.62102617004899) bank79620 +79621 POINT(41.64630655021685 -74.70111039368297) bank79621 +79622 POINT(40.64127876306821 -73.51277352422235) bank79622 +79623 POINT(39.724784337300186 -74.12693229494265) bank79623 +79624 POINT(41.23110749345135 -74.96411677891393) bank79624 +79625 POINT(40.0443214703901 -73.65284501894764) bank79625 +79626 POINT(41.33423621199469 -73.90820039227988) bank79626 +79627 POINT(40.41310988794931 -74.72712601967216) bank79627 +79628 POINT(41.61074138252974 -73.53320100292933) bank79628 +79629 POINT(41.47734071446721 -74.67454195913267) bank79629 +79630 POINT(41.45680258823444 -73.0586023557652) bank79630 +79631 POINT(39.95555482428931 -73.41530236737837) bank79631 +79632 POINT(40.30307107747054 -74.68674455682635) bank79632 +79633 POINT(39.95963269265678 -74.33174642467496) bank79633 +79634 POINT(40.35846659124516 -74.54256755019884) bank79634 +79635 POINT(41.43967549422102 -73.99479904673098) bank79635 +79636 POINT(40.5464580323611 -74.05309587030888) bank79636 +79637 POINT(41.294477801742154 -73.48064563001401) bank79637 +79638 POINT(40.30440729497209 -73.16239489873448) bank79638 +79639 POINT(40.82110865739756 -74.7875864812042) bank79639 +79640 POINT(40.996682414548154 -74.74256011336932) bank79640 +79641 POINT(41.2354988802642 -73.38858428404845) bank79641 +79642 POINT(40.73624301509961 -73.26251366579338) bank79642 +79643 POINT(40.90232111368559 -73.29948386398242) bank79643 +79644 POINT(40.592082055525765 -73.55068513400917) bank79644 +79645 POINT(41.6049595303085 -74.73815970380741) bank79645 +79646 POINT(39.763422640203196 -73.80894366426872) bank79646 +79647 POINT(41.329217515911296 -74.30543680629646) bank79647 +79648 POINT(41.032773418034125 -73.69722929321816) bank79648 +79649 POINT(41.46115997292932 -73.45860921620672) bank79649 +79650 POINT(40.946648980968085 -74.79758904712064) bank79650 +79651 POINT(41.63151341214438 -73.29024962157669) bank79651 +79652 POINT(40.10564521186669 -74.32050854389392) bank79652 +79653 POINT(40.304363581351005 -73.24844295113971) bank79653 +79654 POINT(41.654240156831385 -73.6269271114634) bank79654 +79655 POINT(39.949329266619316 -73.41580964186257) bank79655 +79656 POINT(39.73664744877172 -73.2839270722225) bank79656 +79657 POINT(41.274268322794825 -74.48402136736429) bank79657 +79658 POINT(40.53091584723266 -74.5744417538524) bank79658 +79659 POINT(40.66396168769165 -74.32465442018169) bank79659 +79660 POINT(39.935868995989495 -74.96414279911328) bank79660 +79661 POINT(40.45418615386905 -74.87339483736302) bank79661 +79662 POINT(40.86846398192015 -74.6098518250247) bank79662 +79663 POINT(40.453273831636416 -74.94731063839563) bank79663 +79664 POINT(41.369937886161786 -74.59795835438985) bank79664 +79665 POINT(40.71211906311673 -74.15666294744874) bank79665 +79666 POINT(40.55768285220167 -74.81760216439335) bank79666 +79667 POINT(39.8073980794625 -74.00763624623649) bank79667 +79668 POINT(41.015974194840815 -73.48988734310716) bank79668 +79669 POINT(39.81609703012057 -73.87675017393094) bank79669 +79670 POINT(40.02309420318332 -74.44535420363472) bank79670 +79671 POINT(40.455116652803746 -74.79171798511068) bank79671 +79672 POINT(40.415463735483 -74.48287304103826) bank79672 +79673 POINT(40.07123933711666 -74.86279141770618) bank79673 +79674 POINT(39.801654982637935 -73.08455678747715) bank79674 +79675 POINT(40.54024739016843 -74.49492200603544) bank79675 +79676 POINT(40.013482682018044 -73.01278202910358) bank79676 +79677 POINT(39.74131475905272 -74.59474005651794) bank79677 +79678 POINT(40.460868638219246 -74.07376990799872) bank79678 +79679 POINT(40.783106962172035 -74.46596713767464) bank79679 +79680 POINT(40.48185531528405 -74.7640465095401) bank79680 +79681 POINT(41.267644450641676 -74.83982708788353) bank79681 +79682 POINT(40.35170041444816 -74.63453071039292) bank79682 +79683 POINT(40.81674363762056 -73.3612405990719) bank79683 +79684 POINT(41.35744641776163 -73.25345025806295) bank79684 +79685 POINT(40.7122403699699 -73.45572097810275) bank79685 +79686 POINT(40.24445750891256 -73.52304142665596) bank79686 +79687 POINT(39.74891369277489 -74.56265728411871) bank79687 +79688 POINT(39.737750282377654 -73.4060507405328) bank79688 +79689 POINT(41.52407470106399 -74.32713238453303) bank79689 +79690 POINT(40.400756183513415 -74.09045771800669) bank79690 +79691 POINT(40.65325806725549 -73.32777443389885) bank79691 +79692 POINT(40.86217691393062 -74.29254601344599) bank79692 +79693 POINT(40.36431147294709 -74.18564595235556) bank79693 +79694 POINT(41.162407399818996 -73.66065121501862) bank79694 +79695 POINT(41.63526751100267 -74.36484298538926) bank79695 +79696 POINT(40.88036214180529 -74.82948758198951) bank79696 +79697 POINT(41.119058982895304 -73.84591225772733) bank79697 +79698 POINT(39.86417013693675 -73.53600711258258) bank79698 +79699 POINT(40.52396631434059 -74.67540140210203) bank79699 +79700 POINT(40.3938039415747 -74.90329296715348) bank79700 +79701 POINT(41.33832061301707 -73.17165659672476) bank79701 +79702 POINT(40.85785513300611 -73.5826854208967) bank79702 +79703 POINT(39.77752575566102 -74.43930067083085) bank79703 +79704 POINT(40.843907640559515 -74.56940657658384) bank79704 +79705 POINT(40.7868132764577 -74.80228299171183) bank79705 +79706 POINT(40.942644588133604 -73.26437780012297) bank79706 +79707 POINT(40.29730768264785 -74.18163370101743) bank79707 +79708 POINT(40.335569956987044 -73.87135968415315) bank79708 +79709 POINT(41.40474083989987 -74.70741525849303) bank79709 +79710 POINT(41.354484428471174 -74.17734952813356) bank79710 +79711 POINT(41.37289617013911 -73.14086199667135) bank79711 +79712 POINT(41.56038885343026 -73.65754870824513) bank79712 +79713 POINT(41.686984957523386 -74.62570632288119) bank79713 +79714 POINT(40.44614020751306 -73.35075209368902) bank79714 +79715 POINT(41.17204691581119 -73.44149097023615) bank79715 +79716 POINT(39.969609637141225 -73.6739158852884) bank79716 +79717 POINT(41.319464490087476 -74.50946261072586) bank79717 +79718 POINT(41.437711485009366 -74.56709765552264) bank79718 +79719 POINT(41.25212040596565 -73.9009312960242) bank79719 +79720 POINT(41.24645166181844 -73.95929815164173) bank79720 +79721 POINT(39.95484303547914 -73.02334577774985) bank79721 +79722 POINT(39.78412106185498 -74.61336753741591) bank79722 +79723 POINT(40.089036278307304 -74.54735691419843) bank79723 +79724 POINT(40.6521375604772 -73.58112384664818) bank79724 +79725 POINT(40.72045859264378 -73.43315776119688) bank79725 +79726 POINT(41.456889368160844 -74.34291317939051) bank79726 +79727 POINT(39.77146949206493 -73.91583470951676) bank79727 +79728 POINT(40.50070087075586 -74.28460161108711) bank79728 +79729 POINT(41.402434156160766 -73.88587964716916) bank79729 +79730 POINT(40.15317533827394 -73.01432273851132) bank79730 +79731 POINT(41.49883419777744 -73.97694191692213) bank79731 +79732 POINT(41.23943808059839 -74.16439638900954) bank79732 +79733 POINT(41.499750389962 -74.1822881978299) bank79733 +79734 POINT(40.75300234542062 -74.68288189845768) bank79734 +79735 POINT(40.584024203674545 -74.13637408726566) bank79735 +79736 POINT(40.061203954246054 -74.36954504227722) bank79736 +79737 POINT(41.37122452732539 -73.57296854646253) bank79737 +79738 POINT(39.880147894230696 -73.68617862265782) bank79738 +79739 POINT(40.22075032597198 -74.69169159703382) bank79739 +79740 POINT(41.066300081340884 -74.32777465772736) bank79740 +79741 POINT(40.21719599084141 -73.9650060619627) bank79741 +79742 POINT(40.97131962611972 -73.01960151743044) bank79742 +79743 POINT(40.3055269772944 -73.80303265263505) bank79743 +79744 POINT(41.694815358585785 -73.0510040537967) bank79744 +79745 POINT(41.71092631860841 -73.87702797732868) bank79745 +79746 POINT(41.1289943268517 -74.86395082362034) bank79746 +79747 POINT(40.472283024919335 -73.90761525234447) bank79747 +79748 POINT(40.30451282411907 -74.97050935730584) bank79748 +79749 POINT(40.04318199844641 -74.22274048548759) bank79749 +79750 POINT(41.09791549207557 -74.98844176877842) bank79750 +79751 POINT(40.30648799795001 -73.60034631151926) bank79751 +79752 POINT(40.85785952384492 -73.41037254047068) bank79752 +79753 POINT(40.236172057202005 -74.10630086661) bank79753 +79754 POINT(40.256802745707986 -74.33305262337014) bank79754 +79755 POINT(40.79948601466719 -74.76481144311418) bank79755 +79756 POINT(40.0761332085868 -74.76654314042113) bank79756 +79757 POINT(40.77160198962397 -74.69677859149988) bank79757 +79758 POINT(41.15189780634733 -73.18866645869049) bank79758 +79759 POINT(40.82895160121343 -74.40459645068553) bank79759 +79760 POINT(40.04063332507767 -73.89828459860175) bank79760 +79761 POINT(41.212567932120066 -74.35294760500398) bank79761 +79762 POINT(41.60234456739332 -74.96349472487178) bank79762 +79763 POINT(41.51162363133976 -73.97860652123163) bank79763 +79764 POINT(41.4790745132481 -73.4868038529323) bank79764 +79765 POINT(41.389241037286155 -73.33707070078331) bank79765 +79766 POINT(41.666661115140904 -73.21443811313843) bank79766 +79767 POINT(39.99027853294583 -73.51126134174487) bank79767 +79768 POINT(40.27754250614823 -73.41613538529555) bank79768 +79769 POINT(41.18713931293479 -73.13599126057314) bank79769 +79770 POINT(40.83637735990578 -73.70622450389675) bank79770 +79771 POINT(39.71561847170169 -74.54933618306742) bank79771 +79772 POINT(41.11086031654059 -73.27229933173527) bank79772 +79773 POINT(40.49778155446538 -74.99683675819144) bank79773 +79774 POINT(39.97745081764378 -73.28797500364449) bank79774 +79775 POINT(40.20764237559152 -74.51936982082684) bank79775 +79776 POINT(41.66156744278342 -74.97347082492276) bank79776 +79777 POINT(40.27593279598357 -74.51349074908339) bank79777 +79778 POINT(41.46344675344756 -74.56704944363025) bank79778 +79779 POINT(40.1120379704636 -73.31323998632686) bank79779 +79780 POINT(40.47952230810143 -73.2625881058535) bank79780 +79781 POINT(40.2571499178839 -73.89142359728882) bank79781 +79782 POINT(41.581122506320725 -73.05739243319988) bank79782 +79783 POINT(40.67489513369304 -74.01058947660044) bank79783 +79784 POINT(41.07088425991268 -73.64677621461252) bank79784 +79785 POINT(39.974727996743205 -73.94226091060429) bank79785 +79786 POINT(39.89420379109907 -73.26420291920549) bank79786 +79787 POINT(40.42414340677473 -74.16000674864512) bank79787 +79788 POINT(40.64974227499992 -73.75640829803599) bank79788 +79789 POINT(39.832133149646495 -74.78167523473925) bank79789 +79790 POINT(40.93548158278024 -73.05475123236948) bank79790 +79791 POINT(40.27534355659747 -73.25043896761301) bank79791 +79792 POINT(41.50068930959196 -74.82180404590524) bank79792 +79793 POINT(40.58756981713748 -74.10499618024409) bank79793 +79794 POINT(40.31177493533435 -73.78884053883196) bank79794 +79795 POINT(40.20144903481805 -73.32279189653599) bank79795 +79796 POINT(40.36151176961147 -74.18150011083553) bank79796 +79797 POINT(40.192884670108455 -73.1549076565244) bank79797 +79798 POINT(41.047111180572244 -74.21414274546981) bank79798 +79799 POINT(41.216433525190965 -74.13653651390584) bank79799 +79800 POINT(41.4140225780419 -73.93176880190126) bank79800 +79801 POINT(41.38748247960445 -73.76237693169773) bank79801 +79802 POINT(41.10696552194357 -74.81595244230967) bank79802 +79803 POINT(40.515848468423755 -73.38794649642817) bank79803 +79804 POINT(41.21813733051277 -73.61752994072754) bank79804 +79805 POINT(41.538565619606736 -73.79753836012905) bank79805 +79806 POINT(40.45394734377374 -73.3439808864599) bank79806 +79807 POINT(40.98262901830141 -74.92756736605095) bank79807 +79808 POINT(41.030838559868585 -74.69940032507846) bank79808 +79809 POINT(41.21137837391849 -73.41613589108668) bank79809 +79810 POINT(40.45559972941749 -74.33040104305813) bank79810 +79811 POINT(40.72647329820047 -73.99901498624364) bank79811 +79812 POINT(40.79349712256445 -73.22155709734399) bank79812 +79813 POINT(40.82026643390879 -73.82848206309927) bank79813 +79814 POINT(41.13917185806858 -74.60280797408885) bank79814 +79815 POINT(41.323119539298204 -74.1831305400342) bank79815 +79816 POINT(40.00745994979563 -73.02904175764255) bank79816 +79817 POINT(40.90759951916194 -73.1059294280855) bank79817 +79818 POINT(41.522508771263745 -74.44711099099675) bank79818 +79819 POINT(41.25187270420106 -73.81897326274651) bank79819 +79820 POINT(41.484782485596995 -73.71118423277112) bank79820 +79821 POINT(41.00947591462733 -74.29240533074099) bank79821 +79822 POINT(40.97269777305073 -74.28443673978522) bank79822 +79823 POINT(40.33154607480575 -74.97721764972583) bank79823 +79824 POINT(41.57343823444092 -73.71404823958736) bank79824 +79825 POINT(40.69608665467006 -74.95635074394049) bank79825 +79826 POINT(40.33495728928097 -74.51293001936301) bank79826 +79827 POINT(41.08378992063499 -73.09734480755068) bank79827 +79828 POINT(41.424092451809365 -73.83246215077875) bank79828 +79829 POINT(40.44657050768942 -74.57635123380891) bank79829 +79830 POINT(40.21330458169896 -73.23110383098613) bank79830 +79831 POINT(41.22808028806989 -74.56761627134215) bank79831 +79832 POINT(40.16001563013975 -74.70066293722263) bank79832 +79833 POINT(39.88725354563741 -73.82848240943575) bank79833 +79834 POINT(41.09345617376935 -74.75987903824586) bank79834 +79835 POINT(41.024441269896975 -73.42107424517717) bank79835 +79836 POINT(40.77937425351202 -73.47464752371408) bank79836 +79837 POINT(40.47234262453076 -73.06652744095648) bank79837 +79838 POINT(41.28013232725492 -74.05053824195262) bank79838 +79839 POINT(41.50004425386049 -74.56844548186263) bank79839 +79840 POINT(41.240385492226636 -73.96750107863885) bank79840 +79841 POINT(41.60455893335873 -73.54118870508943) bank79841 +79842 POINT(40.12905542412888 -73.60586391761994) bank79842 +79843 POINT(41.623125831319925 -73.49916241959772) bank79843 +79844 POINT(39.719410015476996 -74.7352823960999) bank79844 +79845 POINT(41.42325530342629 -73.43592881590206) bank79845 +79846 POINT(40.17355761292006 -74.9429723323858) bank79846 +79847 POINT(40.950442113949535 -74.6023819292638) bank79847 +79848 POINT(40.02595325377196 -74.50026396233976) bank79848 +79849 POINT(40.50757598757674 -73.9450483273551) bank79849 +79850 POINT(40.17185369364055 -74.53864943674249) bank79850 +79851 POINT(40.99964241487886 -73.47317254191155) bank79851 +79852 POINT(41.350812224671145 -73.91027898022926) bank79852 +79853 POINT(41.03603343471881 -73.09369022028439) bank79853 +79854 POINT(40.097657828280354 -74.15568875572755) bank79854 +79855 POINT(40.23098752339975 -73.42114020006856) bank79855 +79856 POINT(40.3312647323961 -74.57443436731413) bank79856 +79857 POINT(40.95717652627761 -74.45673453479324) bank79857 +79858 POINT(40.88133209910022 -74.27711841572044) bank79858 +79859 POINT(40.90250627601373 -73.40068202113534) bank79859 +79860 POINT(41.50737765001454 -73.63270968900383) bank79860 +79861 POINT(41.276548118455395 -74.48075120911739) bank79861 +79862 POINT(39.79314811577831 -73.90142417519803) bank79862 +79863 POINT(40.53612260817723 -73.24749320691815) bank79863 +79864 POINT(40.505318917383875 -73.78658782632724) bank79864 +79865 POINT(40.828615571543466 -74.80581381987335) bank79865 +79866 POINT(40.91893309195975 -73.38320747950416) bank79866 +79867 POINT(40.43456042183479 -74.77331765935287) bank79867 +79868 POINT(41.526102078715866 -74.8087231308814) bank79868 +79869 POINT(39.89556575275377 -74.3069587105267) bank79869 +79870 POINT(40.96411164577603 -73.89532634010376) bank79870 +79871 POINT(40.252435017906684 -73.56852527491944) bank79871 +79872 POINT(41.68481038110248 -73.25547557590684) bank79872 +79873 POINT(39.86798584577792 -74.10887017560758) bank79873 +79874 POINT(41.39381976067738 -73.05875357241266) bank79874 +79875 POINT(41.08744707233192 -74.04715564782438) bank79875 +79876 POINT(39.8714631923884 -74.91460839713345) bank79876 +79877 POINT(40.70109959052874 -74.9070218026697) bank79877 +79878 POINT(39.79656585050661 -74.81367871565713) bank79878 +79879 POINT(41.415474912638075 -73.37250882391513) bank79879 +79880 POINT(40.57395539025464 -73.60452396785001) bank79880 +79881 POINT(40.730391664055524 -73.84761915209872) bank79881 +79882 POINT(41.45527777028358 -74.84217490161507) bank79882 +79883 POINT(39.81456668712096 -73.94749789171561) bank79883 +79884 POINT(39.812196337295774 -73.94608817734797) bank79884 +79885 POINT(40.58831465432985 -74.03086398987348) bank79885 +79886 POINT(40.439880709642445 -74.2677779497015) bank79886 +79887 POINT(41.52521187305611 -74.44799402723143) bank79887 +79888 POINT(41.26436290943733 -73.52919464075197) bank79888 +79889 POINT(39.82452604605764 -74.14953957887597) bank79889 +79890 POINT(40.76215979269785 -73.8631329581689) bank79890 +79891 POINT(41.140883784476884 -74.41101541411071) bank79891 +79892 POINT(41.375969284333436 -73.32562239892845) bank79892 +79893 POINT(40.04682848095044 -73.63920276325398) bank79893 +79894 POINT(40.08896691602199 -73.99503035544569) bank79894 +79895 POINT(40.67999285867273 -73.05038990385556) bank79895 +79896 POINT(40.79904621997192 -73.12061977118618) bank79896 +79897 POINT(40.738767269494076 -74.27743047873571) bank79897 +79898 POINT(40.88129777518302 -74.15065175481224) bank79898 +79899 POINT(40.50252840355767 -73.17295357653543) bank79899 +79900 POINT(39.8659802804196 -74.00513139509575) bank79900 +79901 POINT(41.397374177519715 -73.45988800269575) bank79901 +79902 POINT(40.73526149232809 -74.64253499119145) bank79902 +79903 POINT(40.110448961498044 -73.98169747191018) bank79903 +79904 POINT(41.147299644839755 -73.93094319733513) bank79904 +79905 POINT(41.09109687146535 -73.05888040159772) bank79905 +79906 POINT(41.333786791688134 -74.32834117674781) bank79906 +79907 POINT(40.21284362652284 -74.68083454656956) bank79907 +79908 POINT(40.611703532203 -74.7804450424786) bank79908 +79909 POINT(40.19655311621622 -74.41521854249052) bank79909 +79910 POINT(40.90406121576596 -73.61920962396383) bank79910 +79911 POINT(41.43536707931593 -74.64467975882482) bank79911 +79912 POINT(40.26178830873595 -73.19445896888074) bank79912 +79913 POINT(40.19903875142242 -73.73965764376449) bank79913 +79914 POINT(40.75193991453789 -74.49906137396388) bank79914 +79915 POINT(40.184363802082025 -73.84530415064148) bank79915 +79916 POINT(40.77969400538483 -74.27677259392594) bank79916 +79917 POINT(39.75509078768495 -73.32659993292941) bank79917 +79918 POINT(41.681026478676486 -74.31507957069542) bank79918 +79919 POINT(40.2637775979979 -74.82970835922782) bank79919 +79920 POINT(39.89492184583436 -74.527083620886) bank79920 +79921 POINT(41.06654058780058 -74.37248990136159) bank79921 +79922 POINT(41.28509959886351 -74.13086694964318) bank79922 +79923 POINT(39.73758456700424 -74.12253525395498) bank79923 +79924 POINT(41.53223814082965 -74.76693285513554) bank79924 +79925 POINT(41.6133825646209 -73.38493167003314) bank79925 +79926 POINT(40.05048107364709 -74.36298070725637) bank79926 +79927 POINT(41.110480592149266 -73.30835399891909) bank79927 +79928 POINT(40.58771767323177 -74.41929928451539) bank79928 +79929 POINT(40.0006967392291 -74.55753471765374) bank79929 +79930 POINT(40.98429174260848 -74.31364314267944) bank79930 +79931 POINT(40.30640648909241 -74.978012181619) bank79931 +79932 POINT(40.67219899167913 -74.80954567547656) bank79932 +79933 POINT(41.51955882927331 -73.986094494742) bank79933 +79934 POINT(40.59091825158509 -74.27833385492926) bank79934 +79935 POINT(41.118848602005734 -73.06666510431776) bank79935 +79936 POINT(40.7084251706068 -73.28308469389802) bank79936 +79937 POINT(39.806769840945606 -73.02621858220226) bank79937 +79938 POINT(41.675861932609315 -74.24777072232501) bank79938 +79939 POINT(41.154750486242946 -74.8581598117491) bank79939 +79940 POINT(39.79889114085145 -73.57075787704238) bank79940 +79941 POINT(41.02120509062102 -74.94047748381304) bank79941 +79942 POINT(40.82506533828956 -74.75213829510957) bank79942 +79943 POINT(39.906489124651024 -73.70138240637733) bank79943 +79944 POINT(41.70707120090524 -74.4720643766978) bank79944 +79945 POINT(41.1753893155626 -74.43304673220123) bank79945 +79946 POINT(41.578775776841724 -73.44306245098443) bank79946 +79947 POINT(40.71668792442777 -74.12392141442216) bank79947 +79948 POINT(40.25904271393244 -73.45091361143432) bank79948 +79949 POINT(41.406998066499035 -73.58233443761964) bank79949 +79950 POINT(41.466624819402 -73.21797957874033) bank79950 +79951 POINT(40.13556692774834 -74.2918992515731) bank79951 +79952 POINT(41.44100988709464 -74.41661592028198) bank79952 +79953 POINT(40.02755930671854 -74.83591687988559) bank79953 +79954 POINT(41.332502974962274 -74.30797547495756) bank79954 +79955 POINT(41.22169252544516 -74.98765000229582) bank79955 +79956 POINT(41.63118395929846 -73.63159895701929) bank79956 +79957 POINT(40.36784760979015 -73.19222988490374) bank79957 +79958 POINT(40.73344306967797 -74.34641794194407) bank79958 +79959 POINT(41.381096880501076 -73.23085758535035) bank79959 +79960 POINT(41.26153942815256 -74.82185739839925) bank79960 +79961 POINT(40.95870500309704 -74.27279810931348) bank79961 +79962 POINT(40.270241050441136 -74.36209791532013) bank79962 +79963 POINT(41.32678838950607 -74.5668754925679) bank79963 +79964 POINT(39.841995220856944 -73.74356206776974) bank79964 +79965 POINT(41.2343557030844 -74.96614937480771) bank79965 +79966 POINT(40.03674161227943 -73.50358557686067) bank79966 +79967 POINT(39.93705510886817 -74.13502497399496) bank79967 +79968 POINT(41.4983729298982 -73.07666305096319) bank79968 +79969 POINT(39.89656528077017 -74.94190231284674) bank79969 +79970 POINT(41.6047676140172 -73.67523991846164) bank79970 +79971 POINT(40.25518190984391 -74.03624353098657) bank79971 +79972 POINT(41.100378593501105 -74.55444317152848) bank79972 +79973 POINT(40.9555919307002 -74.50446829709774) bank79973 +79974 POINT(40.09290344511986 -74.2300755617195) bank79974 +79975 POINT(40.384235451604745 -74.19209417060485) bank79975 +79976 POINT(40.14092031720569 -74.24808805200249) bank79976 +79977 POINT(39.99504688996666 -74.05139137456794) bank79977 +79978 POINT(40.745261894035124 -74.42017458139995) bank79978 +79979 POINT(41.010346846740866 -73.03369963862622) bank79979 +79980 POINT(41.62031190069608 -74.80667443177585) bank79980 +79981 POINT(41.27834398854068 -74.21022995161792) bank79981 +79982 POINT(39.83523263684017 -73.39807552781785) bank79982 +79983 POINT(41.38873992155601 -73.09346862459435) bank79983 +79984 POINT(40.10223593711132 -73.661586758069) bank79984 +79985 POINT(39.8581980376531 -74.66474249965296) bank79985 +79986 POINT(39.957594103371385 -73.05421896418832) bank79986 +79987 POINT(40.772042819346275 -73.07145789323818) bank79987 +79988 POINT(41.34938872390988 -73.44902422591909) bank79988 +79989 POINT(40.391287828155754 -73.25560882269835) bank79989 +79990 POINT(41.27870604264726 -73.8785411794236) bank79990 +79991 POINT(39.98687610775512 -74.14844605361631) bank79991 +79992 POINT(41.62994639480696 -73.94729607302209) bank79992 +79993 POINT(39.94166243967567 -74.9280013912414) bank79993 +79994 POINT(40.85112222266022 -74.22857852151718) bank79994 +79995 POINT(41.25801346500953 -74.19946515336848) bank79995 +79996 POINT(39.75219897427085 -73.1819592316393) bank79996 +79997 POINT(40.22992707786099 -73.33106656797474) bank79997 +79998 POINT(41.35978168844078 -73.12135176209394) bank79998 +79999 POINT(40.67109237059915 -73.53572595869886) bank79999 +80000 POINT(39.826792915161064 -74.78172306709655) bank80000 +80001 POINT(40.8074835569827 -73.15310376782072) bank80001 +80002 POINT(39.90120984533742 -74.85359067952594) bank80002 +80003 POINT(41.4181758931399 -73.3093945850417) bank80003 +80004 POINT(39.720636011733504 -74.36392631763856) bank80004 +80005 POINT(40.17625450329713 -74.998381188855) bank80005 +80006 POINT(39.81307091586889 -73.65479233203477) bank80006 +80007 POINT(41.26503059963019 -73.17429176673042) bank80007 +80008 POINT(40.55431275649186 -74.83113761863444) bank80008 +80009 POINT(40.7578230138666 -74.98961253750613) bank80009 +80010 POINT(40.22099080706648 -74.81404935381924) bank80010 +80011 POINT(39.80530224666064 -73.4296348226564) bank80011 +80012 POINT(41.0012010452781 -74.30864953221104) bank80012 +80013 POINT(40.776044474149764 -74.99467898325953) bank80013 +80014 POINT(39.851947078391554 -74.05199779549103) bank80014 +80015 POINT(39.75310515414259 -74.58200674779789) bank80015 +80016 POINT(41.53761117245977 -73.74872610192178) bank80016 +80017 POINT(41.34820940708082 -73.30728424503012) bank80017 +80018 POINT(39.76080792880539 -74.32615475337438) bank80018 +80019 POINT(41.709103263194756 -73.07689723989479) bank80019 +80020 POINT(41.36902245907424 -73.59068152335081) bank80020 +80021 POINT(40.1738015193869 -73.87594800051379) bank80021 +80022 POINT(41.09951205316981 -74.42961125143624) bank80022 +80023 POINT(40.81380241479354 -73.49057857571005) bank80023 +80024 POINT(40.11088343148933 -73.97500144756854) bank80024 +80025 POINT(39.91815941647811 -74.48243577998635) bank80025 +80026 POINT(40.15681169287861 -73.88369740045925) bank80026 +80027 POINT(40.4583312295044 -74.97799310975344) bank80027 +80028 POINT(40.78449399302103 -74.53637321030389) bank80028 +80029 POINT(39.922515686273734 -73.55446785538263) bank80029 +80030 POINT(40.47144302079265 -74.16542222923609) bank80030 +80031 POINT(40.75041767987233 -73.13764050871042) bank80031 +80032 POINT(41.26785940393404 -73.98404019628208) bank80032 +80033 POINT(40.46502199813383 -73.84081121447359) bank80033 +80034 POINT(41.40212158370443 -74.70488197033329) bank80034 +80035 POINT(40.24446702848834 -73.48830715340317) bank80035 +80036 POINT(41.015472475259024 -74.75580592851105) bank80036 +80037 POINT(41.045084388738076 -73.25663308616471) bank80037 +80038 POINT(40.533099148402975 -73.63622310233326) bank80038 +80039 POINT(40.12018653583571 -74.84794751211977) bank80039 +80040 POINT(39.805239380739984 -74.34075022015594) bank80040 +80041 POINT(41.656319286944615 -73.87425745432249) bank80041 +80042 POINT(41.399917214603086 -73.77365246881706) bank80042 +80043 POINT(40.38151680567428 -73.05412520768436) bank80043 +80044 POINT(41.197523050582944 -74.46037136733418) bank80044 +80045 POINT(40.99723808950458 -73.57373715280491) bank80045 +80046 POINT(40.52163789146268 -73.8683705692899) bank80046 +80047 POINT(39.872834462435556 -74.58162813747734) bank80047 +80048 POINT(40.528310128769256 -74.58710661406958) bank80048 +80049 POINT(40.677022426952206 -74.41350077132732) bank80049 +80050 POINT(40.7580591999953 -73.10503041676687) bank80050 +80051 POINT(40.11831164705649 -74.2642663317232) bank80051 +80052 POINT(41.24503574824776 -74.01678104596354) bank80052 +80053 POINT(39.956753901284735 -74.72536623401157) bank80053 +80054 POINT(40.6599501817932 -73.70649188456528) bank80054 +80055 POINT(41.23193650018857 -74.71834258043016) bank80055 +80056 POINT(40.655834370990306 -74.60317791426141) bank80056 +80057 POINT(40.33202715617187 -73.36615629633323) bank80057 +80058 POINT(40.077438833056576 -74.28776909531855) bank80058 +80059 POINT(40.81104456060097 -73.62769333753737) bank80059 +80060 POINT(41.03324732235952 -74.13693169792295) bank80060 +80061 POINT(40.12285810729454 -73.5704932353184) bank80061 +80062 POINT(41.26793968485572 -74.36684811818871) bank80062 +80063 POINT(41.109433494372155 -73.88966551102048) bank80063 +80064 POINT(40.49585301145437 -73.39041521094059) bank80064 +80065 POINT(40.7179370913121 -73.00902870129546) bank80065 +80066 POINT(40.46582836913924 -74.20265067882823) bank80066 +80067 POINT(40.844673152387465 -74.04562283521916) bank80067 +80068 POINT(39.95960960074172 -73.38891829431374) bank80068 +80069 POINT(40.95484607462107 -73.18641193471089) bank80069 +80070 POINT(41.55121246458838 -73.90713335756028) bank80070 +80071 POINT(39.89413588722526 -73.30746193711607) bank80071 +80072 POINT(41.336568801685296 -73.40028534584548) bank80072 +80073 POINT(41.21094752494775 -74.23619377074304) bank80073 +80074 POINT(41.23899789218656 -74.82183335122312) bank80074 +80075 POINT(41.431393076401555 -73.4804019594476) bank80075 +80076 POINT(40.81974923846524 -73.82310777760107) bank80076 +80077 POINT(41.558473937066985 -73.93049802064003) bank80077 +80078 POINT(41.63809036085755 -74.5025445825508) bank80078 +80079 POINT(40.724621452446186 -73.8473980438426) bank80079 +80080 POINT(40.583574720287544 -74.20828008738454) bank80080 +80081 POINT(39.729129302164395 -73.49808359563333) bank80081 +80082 POINT(41.15580111106027 -73.14695417323318) bank80082 +80083 POINT(41.277636069325226 -73.28813970766453) bank80083 +80084 POINT(40.048531769982226 -73.42648920168348) bank80084 +80085 POINT(40.33838658418986 -74.05618328057999) bank80085 +80086 POINT(39.92203008662307 -73.7645847057136) bank80086 +80087 POINT(41.06493902861909 -73.28795790965641) bank80087 +80088 POINT(41.26346580333758 -74.89465911531155) bank80088 +80089 POINT(40.21794001937383 -73.72680836781234) bank80089 +80090 POINT(40.672818908921094 -74.88177247907392) bank80090 +80091 POINT(41.240364773395626 -73.16109605496484) bank80091 +80092 POINT(40.5384847522011 -74.08217556999344) bank80092 +80093 POINT(41.46274786138939 -73.0645538998748) bank80093 +80094 POINT(40.34799938013984 -73.52861246763673) bank80094 +80095 POINT(41.10547348776577 -73.7894693892395) bank80095 +80096 POINT(40.295895121427904 -74.66376890497477) bank80096 +80097 POINT(40.513889436861845 -74.21236689457058) bank80097 +80098 POINT(40.07881501556662 -73.50686813228336) bank80098 +80099 POINT(39.9719607030291 -74.23105889409256) bank80099 +80100 POINT(40.210760001407735 -73.80167369870179) bank80100 +80101 POINT(41.5521140331822 -74.87935520273003) bank80101 +80102 POINT(40.20312552122975 -74.84236775355456) bank80102 +80103 POINT(40.17295158597486 -73.30873439171623) bank80103 +80104 POINT(40.3143662825798 -73.58494069210612) bank80104 +80105 POINT(39.805132889343035 -74.51410337636013) bank80105 +80106 POINT(39.93716620115295 -74.84165314255661) bank80106 +80107 POINT(40.159946891876686 -74.36780856373584) bank80107 +80108 POINT(39.86761751670026 -74.62520309130106) bank80108 +80109 POINT(39.932052002383294 -74.51734708633278) bank80109 +80110 POINT(40.67808527500034 -73.81437919946663) bank80110 +80111 POINT(39.77665725626885 -74.81813301773228) bank80111 +80112 POINT(40.685830310744485 -73.26761118848864) bank80112 +80113 POINT(40.40396468743287 -74.26478865047228) bank80113 +80114 POINT(40.84114926456554 -73.9826597805741) bank80114 +80115 POINT(39.81464048745094 -74.46666548812065) bank80115 +80116 POINT(40.52301841099665 -73.77384827214621) bank80116 +80117 POINT(40.00725092151271 -74.8561155822329) bank80117 +80118 POINT(40.57097640616296 -74.77230047441826) bank80118 +80119 POINT(40.69257967293255 -73.53100666097188) bank80119 +80120 POINT(41.32239405841843 -73.45839480692773) bank80120 +80121 POINT(40.77153123415047 -74.24409879721479) bank80121 +80122 POINT(41.157130065515965 -73.49212074304317) bank80122 +80123 POINT(40.98303263676453 -73.17919229132701) bank80123 +80124 POINT(39.80969983728958 -73.45418991714212) bank80124 +80125 POINT(40.53823949892753 -73.73883136951399) bank80125 +80126 POINT(40.23551873430037 -73.01208773911956) bank80126 +80127 POINT(41.59855738570498 -73.68743733735839) bank80127 +80128 POINT(40.01903893855599 -74.18196592268939) bank80128 +80129 POINT(40.247857376673664 -73.80473617658491) bank80129 +80130 POINT(40.71238202593048 -73.89849166049241) bank80130 +80131 POINT(41.407223615597715 -73.6056734031077) bank80131 +80132 POINT(41.33045789106936 -74.71277218761428) bank80132 +80133 POINT(40.217696163160745 -73.99294524491917) bank80133 +80134 POINT(40.1173494604142 -73.31996097147349) bank80134 +80135 POINT(41.35171037918677 -73.5130408104421) bank80135 +80136 POINT(41.65502592657771 -73.47561396961397) bank80136 +80137 POINT(40.9859149327151 -74.04210128145058) bank80137 +80138 POINT(40.59795675147577 -74.22828176882348) bank80138 +80139 POINT(41.032485452125854 -74.8959831265598) bank80139 +80140 POINT(39.73393022110457 -73.8383838976774) bank80140 +80141 POINT(41.026120670180724 -73.98945232272307) bank80141 +80142 POINT(40.98898971889329 -74.99285224497699) bank80142 +80143 POINT(41.37115396964799 -74.01382262839137) bank80143 +80144 POINT(41.37838567908153 -74.85276311105432) bank80144 +80145 POINT(40.93057412584257 -73.1048939148012) bank80145 +80146 POINT(41.4108354892953 -74.23394375142463) bank80146 +80147 POINT(40.148556864854406 -73.8928384323136) bank80147 +80148 POINT(40.506098600357404 -73.46635957512242) bank80148 +80149 POINT(40.57541299287921 -74.81751566605259) bank80149 +80150 POINT(40.853123047920036 -73.64305850782313) bank80150 +80151 POINT(41.57801656670994 -73.42987251869359) bank80151 +80152 POINT(40.399553990940426 -74.24058977392723) bank80152 +80153 POINT(41.554415719423375 -73.85760466411696) bank80153 +80154 POINT(40.55598742171292 -73.27367433730818) bank80154 +80155 POINT(41.2573386471677 -74.11742316478285) bank80155 +80156 POINT(40.93080819941364 -73.46405664055696) bank80156 +80157 POINT(41.568035695419205 -73.11697729617238) bank80157 +80158 POINT(40.75640428511175 -73.29526464237102) bank80158 +80159 POINT(40.05057510117571 -73.44240160174365) bank80159 +80160 POINT(40.14299479204793 -73.13309280884798) bank80160 +80161 POINT(40.04994557880942 -73.48231835421811) bank80161 +80162 POINT(41.672729906780106 -74.02977693215186) bank80162 +80163 POINT(39.79553787469467 -73.54037492694623) bank80163 +80164 POINT(40.24178963330621 -74.83946679048873) bank80164 +80165 POINT(40.827115528586965 -74.91959864543252) bank80165 +80166 POINT(40.931991196538476 -73.30685910386636) bank80166 +80167 POINT(41.545754582744685 -73.77206243677034) bank80167 +80168 POINT(40.891973745690876 -74.41224063554428) bank80168 +80169 POINT(40.06634609532635 -73.48223034548298) bank80169 +80170 POINT(41.085813411767184 -73.73680019329639) bank80170 +80171 POINT(40.06811008240576 -74.99385990754712) bank80171 +80172 POINT(41.250098111700595 -73.83289792532764) bank80172 +80173 POINT(41.07894409526148 -73.1010380839649) bank80173 +80174 POINT(39.90416370204046 -74.992862589074) bank80174 +80175 POINT(41.15413964509321 -73.15236809675132) bank80175 +80176 POINT(41.15298229751559 -74.80911399578098) bank80176 +80177 POINT(39.95123446196096 -74.34493187899042) bank80177 +80178 POINT(40.51505992996815 -74.08794620498583) bank80178 +80179 POINT(39.810174747358865 -73.72613555278221) bank80179 +80180 POINT(40.78236705916964 -74.39298687052967) bank80180 +80181 POINT(40.85589966314158 -74.59124708859555) bank80181 +80182 POINT(41.196953628209435 -74.28939127625566) bank80182 +80183 POINT(40.92953958326579 -73.27844038888243) bank80183 +80184 POINT(39.73166926963217 -73.34917861709889) bank80184 +80185 POINT(40.7798133313813 -74.38095695530237) bank80185 +80186 POINT(40.688122885054845 -73.05974869244275) bank80186 +80187 POINT(39.855922985551324 -74.88353983210314) bank80187 +80188 POINT(40.95908658933267 -73.01862678485125) bank80188 +80189 POINT(40.623968375434 -73.86303569298637) bank80189 +80190 POINT(41.58181802983229 -73.51582386536198) bank80190 +80191 POINT(41.54685544825518 -74.46699540508754) bank80191 +80192 POINT(40.057262245595844 -74.52669102302511) bank80192 +80193 POINT(40.87629681551168 -73.30702442568108) bank80193 +80194 POINT(40.5034792046593 -74.3294439566182) bank80194 +80195 POINT(40.36454654484587 -74.12875096283642) bank80195 +80196 POINT(40.06334823222669 -73.67899903897161) bank80196 +80197 POINT(39.89365470717457 -74.34157386100209) bank80197 +80198 POINT(41.69731152303498 -73.05487964614393) bank80198 +80199 POINT(40.19504555870245 -74.76574461252865) bank80199 +80200 POINT(39.9623929230037 -74.54627511055756) bank80200 +80201 POINT(41.02718555153353 -73.06978208007065) bank80201 +80202 POINT(40.57024301269866 -73.45452959632462) bank80202 +80203 POINT(40.259261905290344 -74.20632478153318) bank80203 +80204 POINT(41.521170617721396 -73.2191602706003) bank80204 +80205 POINT(40.17450522389917 -74.11086449185386) bank80205 +80206 POINT(40.41728005084043 -73.23358643941556) bank80206 +80207 POINT(39.92815255306414 -73.44712335787749) bank80207 +80208 POINT(40.13619131691226 -74.91536858497858) bank80208 +80209 POINT(39.97203529266697 -74.12062899854286) bank80209 +80210 POINT(39.95787725622835 -74.48456364901173) bank80210 +80211 POINT(39.922550938354085 -74.01897489367005) bank80211 +80212 POINT(40.76627749846242 -73.19987649353732) bank80212 +80213 POINT(40.49597963145211 -74.95680922798066) bank80213 +80214 POINT(40.78857558981136 -74.48046825881737) bank80214 +80215 POINT(41.677056577078616 -74.73610106420789) bank80215 +80216 POINT(41.172815787040456 -73.51712190865761) bank80216 +80217 POINT(41.085235249302976 -73.5378938461737) bank80217 +80218 POINT(40.50801214113013 -73.53342236977461) bank80218 +80219 POINT(39.75368455264888 -73.03515960207197) bank80219 +80220 POINT(40.11753764352144 -74.02055901893694) bank80220 +80221 POINT(39.89045673080174 -73.34811846208837) bank80221 +80222 POINT(41.20671979393373 -74.85525980463031) bank80222 +80223 POINT(40.15896082873432 -73.3162386949009) bank80223 +80224 POINT(39.78090120478246 -73.19792306976572) bank80224 +80225 POINT(40.645773512514545 -73.16222557835373) bank80225 +80226 POINT(41.196278015828454 -73.09825609992133) bank80226 +80227 POINT(40.364981441593834 -74.36821418712131) bank80227 +80228 POINT(40.4801003243678 -74.33325772408321) bank80228 +80229 POINT(41.1261892409719 -73.8032600060783) bank80229 +80230 POINT(40.85357473669765 -73.09981223740895) bank80230 +80231 POINT(40.73641176215309 -73.9657805811028) bank80231 +80232 POINT(40.02813217977556 -74.76700358362592) bank80232 +80233 POINT(41.51525036707982 -73.73466338968917) bank80233 +80234 POINT(41.67044010902637 -73.63822474633557) bank80234 +80235 POINT(41.58302604876863 -74.62539620337462) bank80235 +80236 POINT(41.076864840815595 -73.4717151420822) bank80236 +80237 POINT(41.25486126141166 -74.15681059146178) bank80237 +80238 POINT(40.949689081936015 -74.23436163948843) bank80238 +80239 POINT(41.54395417955206 -73.67083709401216) bank80239 +80240 POINT(41.43802341299556 -73.5961924604191) bank80240 +80241 POINT(40.97000104014853 -74.70520246121923) bank80241 +80242 POINT(40.30717326283281 -73.26356105580001) bank80242 +80243 POINT(40.88333457284709 -74.7425730573228) bank80243 +80244 POINT(40.79291118815893 -74.05683949606458) bank80244 +80245 POINT(40.45727988174234 -74.80606066926427) bank80245 +80246 POINT(40.230842013940844 -73.18043079390446) bank80246 +80247 POINT(40.23691612805715 -73.22020945377075) bank80247 +80248 POINT(41.34910490428027 -74.98806609308869) bank80248 +80249 POINT(39.94069517654831 -73.78015769950437) bank80249 +80250 POINT(40.331136621561164 -74.5081690868432) bank80250 +80251 POINT(40.45112708103535 -73.6871320405418) bank80251 +80252 POINT(40.77001955821749 -74.6791194332985) bank80252 +80253 POINT(39.820131461953714 -74.29589916546328) bank80253 +80254 POINT(39.94586272349218 -74.31767128154128) bank80254 +80255 POINT(40.746357699923074 -74.70185065521153) bank80255 +80256 POINT(40.19694894898914 -74.41997067432898) bank80256 +80257 POINT(40.107385529144 -73.4852026150979) bank80257 +80258 POINT(39.71644721119302 -73.36831912033665) bank80258 +80259 POINT(39.80934929046744 -73.75396535403252) bank80259 +80260 POINT(40.83929617284193 -73.38980153703938) bank80260 +80261 POINT(41.06871849673665 -74.29385380627453) bank80261 +80262 POINT(41.69572824000366 -73.97923684844966) bank80262 +80263 POINT(41.29234702772236 -73.87457123611443) bank80263 +80264 POINT(40.05055413805101 -73.68937602352301) bank80264 +80265 POINT(40.64059961749939 -73.52181713390596) bank80265 +80266 POINT(41.568627163058814 -73.06085271404817) bank80266 +80267 POINT(40.66227684924312 -73.07414518694655) bank80267 +80268 POINT(40.803751272831406 -73.58947276901691) bank80268 +80269 POINT(40.476551506611194 -73.37935973186089) bank80269 +80270 POINT(41.01053684784753 -73.01312077089645) bank80270 +80271 POINT(41.624902746343146 -74.42869915247975) bank80271 +80272 POINT(39.752852421083276 -73.55789120501414) bank80272 +80273 POINT(40.835163645904096 -74.63866406209817) bank80273 +80274 POINT(39.71382304780987 -74.88544481586896) bank80274 +80275 POINT(39.960179630490174 -74.88235471107703) bank80275 +80276 POINT(41.222541633141475 -73.5916439720647) bank80276 +80277 POINT(41.36138938065145 -74.52490217791613) bank80277 +80278 POINT(39.87378760844245 -73.41612711657032) bank80278 +80279 POINT(40.519782422187376 -73.76607502346577) bank80279 +80280 POINT(41.65651810957629 -74.01887537823632) bank80280 +80281 POINT(40.51073036296912 -74.53809932325413) bank80281 +80282 POINT(39.81528582704181 -73.26852032226333) bank80282 +80283 POINT(39.840438591446784 -74.73694812552755) bank80283 +80284 POINT(40.80523818026401 -74.21460310949364) bank80284 +80285 POINT(39.95942803274338 -74.58614617063668) bank80285 +80286 POINT(41.09168771777102 -73.65466845893111) bank80286 +80287 POINT(40.45974719504809 -74.55351959897106) bank80287 +80288 POINT(40.23291641498867 -74.81803228107881) bank80288 +80289 POINT(41.05331365162035 -73.77500374490509) bank80289 +80290 POINT(41.00790987310654 -73.11633099134598) bank80290 +80291 POINT(40.49233283091028 -74.07209614477559) bank80291 +80292 POINT(40.55721301867278 -74.17434571078519) bank80292 +80293 POINT(39.8141801157703 -73.73470869086287) bank80293 +80294 POINT(41.14937248507083 -74.21216926950733) bank80294 +80295 POINT(40.117270792344456 -73.02156629652414) bank80295 +80296 POINT(40.10296398365256 -74.64002885434623) bank80296 +80297 POINT(39.97787774207453 -74.11731694288717) bank80297 +80298 POINT(40.961466604336614 -73.91427988203861) bank80298 +80299 POINT(40.788402692619265 -73.98326757938295) bank80299 +80300 POINT(40.90208353303943 -74.38291679243989) bank80300 +80301 POINT(40.85862833010471 -74.81861963664053) bank80301 +80302 POINT(40.472526367594625 -74.8530523634556) bank80302 +80303 POINT(41.51152116773503 -74.82478441673943) bank80303 +80304 POINT(40.73945504797767 -73.50219161765604) bank80304 +80305 POINT(41.584193266602085 -74.97765037517877) bank80305 +80306 POINT(39.861040049310304 -74.47947640336812) bank80306 +80307 POINT(39.74276360274532 -74.06708099364978) bank80307 +80308 POINT(40.11929923038287 -74.07699502803203) bank80308 +80309 POINT(40.86722255674147 -73.82018780860936) bank80309 +80310 POINT(41.40039143128333 -74.7502252103748) bank80310 +80311 POINT(41.12386855264062 -73.91975137372893) bank80311 +80312 POINT(40.78776060605715 -74.35462219841236) bank80312 +80313 POINT(40.72749202860914 -73.37661390767063) bank80313 +80314 POINT(41.65373126343542 -73.93028076868451) bank80314 +80315 POINT(41.519111456248915 -73.1864080831307) bank80315 +80316 POINT(40.173887524929455 -74.43279769387314) bank80316 +80317 POINT(39.98412945865527 -73.62945574705901) bank80317 +80318 POINT(40.12400445172452 -73.73139152844676) bank80318 +80319 POINT(40.11935576146242 -73.97529527470742) bank80319 +80320 POINT(39.95436083881036 -74.8611583481774) bank80320 +80321 POINT(39.97387053241252 -73.85116609941144) bank80321 +80322 POINT(41.23250043386537 -73.74974454080679) bank80322 +80323 POINT(40.9455844816055 -74.39327618906262) bank80323 +80324 POINT(41.07233416596144 -74.88829941654963) bank80324 +80325 POINT(40.98057861143692 -74.88036526185113) bank80325 +80326 POINT(40.29298774233998 -75.0053038700825) bank80326 +80327 POINT(41.55474487583554 -73.22788261548033) bank80327 +80328 POINT(41.11971854978305 -74.50118911895221) bank80328 +80329 POINT(41.006252619595784 -73.75340274530122) bank80329 +80330 POINT(39.90981345907875 -74.17401629572664) bank80330 +80331 POINT(41.22094211069798 -74.6307774613378) bank80331 +80332 POINT(39.72929750355046 -74.18401211260267) bank80332 +80333 POINT(41.06971869922215 -74.44671162183711) bank80333 +80334 POINT(40.222396866884026 -73.3763537789461) bank80334 +80335 POINT(40.49985096599717 -73.52533279786932) bank80335 +80336 POINT(41.4139232049761 -73.28244550208169) bank80336 +80337 POINT(40.62089617964957 -73.19971063117808) bank80337 +80338 POINT(40.33896090945345 -74.36914308241847) bank80338 +80339 POINT(41.542935689556906 -74.43365302806394) bank80339 +80340 POINT(40.397801731606265 -74.2496197729528) bank80340 +80341 POINT(41.115767254648176 -74.66751689917564) bank80341 +80342 POINT(41.26202626966849 -73.0828399757982) bank80342 +80343 POINT(40.79999489400107 -74.28367721947676) bank80343 +80344 POINT(40.28454326488471 -74.6513247125589) bank80344 +80345 POINT(41.41278426584455 -74.25796395085555) bank80345 +80346 POINT(41.37970944987668 -73.07753312772762) bank80346 +80347 POINT(40.98491258457779 -73.49724716719194) bank80347 +80348 POINT(41.42810382213217 -73.93507617886898) bank80348 +80349 POINT(40.56666363897431 -73.97966023727325) bank80349 +80350 POINT(41.267185241633356 -73.51261106032894) bank80350 +80351 POINT(41.50797645315015 -74.93616582055458) bank80351 +80352 POINT(40.63315372806106 -73.1949774477118) bank80352 +80353 POINT(40.02652302057217 -73.47728196289918) bank80353 +80354 POINT(40.254769692109164 -73.10649830767613) bank80354 +80355 POINT(40.02498320300253 -74.94959044362028) bank80355 +80356 POINT(40.38541848651569 -74.31586137756081) bank80356 +80357 POINT(40.18426989519661 -74.27085762773476) bank80357 +80358 POINT(40.87334624433781 -73.25355357296571) bank80358 +80359 POINT(41.271031117742304 -74.3973874288408) bank80359 +80360 POINT(41.44200460288196 -73.58604208704774) bank80360 +80361 POINT(39.73193849092764 -73.7693317509599) bank80361 +80362 POINT(40.930021449440545 -74.59837448558903) bank80362 +80363 POINT(41.70536608787125 -73.74075404312804) bank80363 +80364 POINT(41.70426896412317 -73.674715619207) bank80364 +80365 POINT(41.281617485922254 -74.9755718083272) bank80365 +80366 POINT(40.289153797731394 -73.43579449180629) bank80366 +80367 POINT(41.27369023839041 -74.18530841763878) bank80367 +80368 POINT(41.63140660254156 -74.84077699672594) bank80368 +80369 POINT(40.129975705182886 -73.78813871578409) bank80369 +80370 POINT(40.92865860483531 -73.95025440406121) bank80370 +80371 POINT(40.93166391520589 -73.95200061885879) bank80371 +80372 POINT(41.33912468171319 -74.48102606488291) bank80372 +80373 POINT(41.34351455921234 -74.73985276334983) bank80373 +80374 POINT(40.49798700636174 -74.60657868976737) bank80374 +80375 POINT(41.48113180656347 -74.50413947955667) bank80375 +80376 POINT(41.711685539615694 -73.9157958607348) bank80376 +80377 POINT(40.17951012821253 -73.3066439408733) bank80377 +80378 POINT(40.74452884870285 -74.12109097417213) bank80378 +80379 POINT(41.495866150594395 -74.82470142619887) bank80379 +80380 POINT(40.62883117388752 -73.2220095114291) bank80380 +80381 POINT(41.451478915471206 -73.41626850372666) bank80381 +80382 POINT(39.972528850186876 -73.45870685947936) bank80382 +80383 POINT(40.76142355258684 -73.96350083946088) bank80383 +80384 POINT(40.24820923402639 -74.29828305426128) bank80384 +80385 POINT(41.672527570989395 -74.87314355536782) bank80385 +80386 POINT(39.97381307235384 -74.53401163526145) bank80386 +80387 POINT(41.65008134856332 -74.50629962995671) bank80387 +80388 POINT(41.09513016796102 -74.48827013711931) bank80388 +80389 POINT(40.585613722266864 -73.53081770842041) bank80389 +80390 POINT(40.882777240568835 -74.75940860623527) bank80390 +80391 POINT(40.40898460544483 -74.66446976496863) bank80391 +80392 POINT(40.963253334036665 -74.14686287782865) bank80392 +80393 POINT(40.3112478387842 -74.07906126089777) bank80393 +80394 POINT(40.77492297654081 -73.60430170230777) bank80394 +80395 POINT(40.21207869722343 -74.66693940834695) bank80395 +80396 POINT(40.39512253871876 -74.37137227277383) bank80396 +80397 POINT(41.25800754400978 -74.18196053490341) bank80397 +80398 POINT(40.89487224951071 -73.86073530280476) bank80398 +80399 POINT(40.42305504342233 -73.88851660626018) bank80399 +80400 POINT(40.80632068117063 -74.95004141514416) bank80400 +80401 POINT(41.57859596952474 -73.97659754052133) bank80401 +80402 POINT(39.85910034479492 -73.56558907362823) bank80402 +80403 POINT(41.46922095286568 -73.17745807606707) bank80403 +80404 POINT(40.071600189692106 -73.25110856933699) bank80404 +80405 POINT(41.51913138399694 -74.24035512489493) bank80405 +80406 POINT(40.95238804374503 -73.80575357902129) bank80406 +80407 POINT(41.25049320192696 -74.85231951278573) bank80407 +80408 POINT(39.80935464549265 -74.18914638860926) bank80408 +80409 POINT(40.258726745675546 -73.6899069318018) bank80409 +80410 POINT(40.81837235594371 -74.89242054714225) bank80410 +80411 POINT(41.31364287985754 -74.87320516969132) bank80411 +80412 POINT(40.08886979470383 -73.19901645301559) bank80412 +80413 POINT(41.06626482884105 -74.88177338567002) bank80413 +80414 POINT(41.650383575509686 -73.31244910897276) bank80414 +80415 POINT(41.082293590428904 -73.4918375587333) bank80415 +80416 POINT(40.15668739494531 -73.78721847001114) bank80416 +80417 POINT(41.69572166639583 -74.45812220376682) bank80417 +80418 POINT(40.976162723799035 -73.71772888932632) bank80418 +80419 POINT(40.44510638564096 -74.34608633079061) bank80419 +80420 POINT(40.32743381346566 -74.47801252374349) bank80420 +80421 POINT(41.265774030157175 -73.34419515627334) bank80421 +80422 POINT(41.4875727907837 -73.00843188068004) bank80422 +80423 POINT(40.87796612448942 -73.21170523415027) bank80423 +80424 POINT(40.585003486270125 -74.37757104354012) bank80424 +80425 POINT(40.283459459520046 -74.86331980043624) bank80425 +80426 POINT(41.61249248386045 -74.36991244344034) bank80426 +80427 POINT(40.696035877720554 -74.3074447602761) bank80427 +80428 POINT(39.82964035314128 -74.57630644209428) bank80428 +80429 POINT(40.44273746598849 -74.09947958212767) bank80429 +80430 POINT(40.38205220050863 -73.78151209869185) bank80430 +80431 POINT(40.05324012626327 -73.19933128039375) bank80431 +80432 POINT(40.64739550355958 -73.94693923850615) bank80432 +80433 POINT(39.786352911642915 -73.83090775176568) bank80433 +80434 POINT(40.28972976251476 -73.01073784209888) bank80434 +80435 POINT(40.19768751769761 -73.39591329097641) bank80435 +80436 POINT(40.52683105919387 -73.93597116355284) bank80436 +80437 POINT(40.10067949961708 -74.02904580118722) bank80437 +80438 POINT(41.228222232182944 -74.97895624958537) bank80438 +80439 POINT(41.15506301218296 -73.40491769684519) bank80439 +80440 POINT(41.144596049936574 -73.22618040632779) bank80440 +80441 POINT(41.63559377432202 -73.37971366912753) bank80441 +80442 POINT(41.38429068837599 -73.88421457481455) bank80442 +80443 POINT(41.4971682325073 -73.17344721538943) bank80443 +80444 POINT(41.606004706302464 -74.70589332001828) bank80444 +80445 POINT(40.50621848728824 -73.57403297850898) bank80445 +80446 POINT(40.20158281617061 -74.81360700943954) bank80446 +80447 POINT(40.91159848137233 -73.5502981351911) bank80447 +80448 POINT(40.46003650641299 -74.26310075113301) bank80448 +80449 POINT(40.34679249163234 -73.90802287040661) bank80449 +80450 POINT(40.28420432258975 -74.79325711988446) bank80450 +80451 POINT(41.67611433758258 -73.01591147672158) bank80451 +80452 POINT(40.27604781331473 -74.86530832413263) bank80452 +80453 POINT(39.95336657962366 -73.57477621911008) bank80453 +80454 POINT(41.33862502355751 -74.97006090756523) bank80454 +80455 POINT(41.03997609902553 -74.5373311952836) bank80455 +80456 POINT(41.149614985034184 -74.25692765240399) bank80456 +80457 POINT(41.16808038347588 -74.59394730615936) bank80457 +80458 POINT(39.9070468249579 -73.31232899911353) bank80458 +80459 POINT(40.852801993636184 -73.66441900288415) bank80459 +80460 POINT(41.4033105690556 -74.68685694589412) bank80460 +80461 POINT(41.540180975375804 -74.91477090914213) bank80461 +80462 POINT(41.68770417200925 -74.88792307729494) bank80462 +80463 POINT(41.5893226351204 -73.5358176278901) bank80463 +80464 POINT(39.8966715733292 -73.52157108445103) bank80464 +80465 POINT(41.12421111009864 -73.77624990108323) bank80465 +80466 POINT(40.09967305907156 -74.6462405243407) bank80466 +80467 POINT(41.349035668238834 -73.14857036170531) bank80467 +80468 POINT(40.98375246662518 -73.41226816152188) bank80468 +80469 POINT(41.62907645600086 -74.13749329819697) bank80469 +80470 POINT(40.76364992028581 -74.16386210234857) bank80470 +80471 POINT(39.941883660665304 -74.70747461591279) bank80471 +80472 POINT(40.37511646904048 -74.38814702421834) bank80472 +80473 POINT(41.04819280351795 -74.80970198129182) bank80473 +80474 POINT(41.240105926480304 -74.06243046623852) bank80474 +80475 POINT(40.97693936780764 -73.16014312422905) bank80475 +80476 POINT(39.901258681535964 -73.24538616888225) bank80476 +80477 POINT(39.9191127691124 -74.69757364819891) bank80477 +80478 POINT(39.96660623071857 -74.95011081404338) bank80478 +80479 POINT(40.28795594075938 -74.65397315652855) bank80479 +80480 POINT(41.24256749038262 -73.06321527663161) bank80480 +80481 POINT(41.21668145271877 -74.37693685785138) bank80481 +80482 POINT(39.943346291308075 -74.78202002538742) bank80482 +80483 POINT(41.49942429648044 -73.65132404128778) bank80483 +80484 POINT(40.57381015478962 -74.16719644133798) bank80484 +80485 POINT(41.54519288373167 -73.06072974492893) bank80485 +80486 POINT(39.744538307241676 -74.38224852323467) bank80486 +80487 POINT(41.003978934992716 -73.70764233981033) bank80487 +80488 POINT(40.717931364171704 -73.20273470972275) bank80488 +80489 POINT(41.1432178033891 -73.43269031328296) bank80489 +80490 POINT(40.529155655884296 -74.21190959270446) bank80490 +80491 POINT(41.696131666530114 -74.52691539666012) bank80491 +80492 POINT(41.692127619880694 -74.09924656478807) bank80492 +80493 POINT(40.97273638490728 -74.85891544253822) bank80493 +80494 POINT(40.70308290027827 -73.10961164843684) bank80494 +80495 POINT(40.234406870285646 -73.1530057816821) bank80495 +80496 POINT(40.246070147606915 -73.91319675137535) bank80496 +80497 POINT(40.466522304252344 -74.96032210354431) bank80497 +80498 POINT(40.260517163688085 -73.54613234828682) bank80498 +80499 POINT(40.30345500625423 -73.32901022011838) bank80499 +80500 POINT(40.21423584436927 -74.67028487985385) bank80500 +80501 POINT(41.32091324587624 -74.45376309153026) bank80501 +80502 POINT(39.86698654743808 -74.47219995454518) bank80502 +80503 POINT(39.73633403995928 -74.0421192034266) bank80503 +80504 POINT(39.847234498908065 -74.8088691762811) bank80504 +80505 POINT(40.86450716657241 -73.97693357023641) bank80505 +80506 POINT(39.98529858875655 -74.19507317280879) bank80506 +80507 POINT(41.14207360397296 -74.89021761088483) bank80507 +80508 POINT(41.29821928078185 -74.15125400920707) bank80508 +80509 POINT(41.17783727788047 -74.75597459704817) bank80509 +80510 POINT(40.4186308358959 -73.25317402460699) bank80510 +80511 POINT(41.17065150947914 -73.08209371077677) bank80511 +80512 POINT(41.53093166900806 -73.68597894681697) bank80512 +80513 POINT(39.87806774203521 -74.98756334076451) bank80513 +80514 POINT(41.264937649401844 -73.53671871603598) bank80514 +80515 POINT(40.678907723166226 -74.3968208053424) bank80515 +80516 POINT(40.241726100394295 -73.4723391489952) bank80516 +80517 POINT(41.36937280924164 -73.51463433946823) bank80517 +80518 POINT(41.02222630925221 -73.17268129818312) bank80518 +80519 POINT(41.55054580090607 -73.040508866128) bank80519 +80520 POINT(40.424011580985464 -73.42913812615765) bank80520 +80521 POINT(41.54322417926793 -74.8032919818615) bank80521 +80522 POINT(40.39803058966196 -74.39595671138261) bank80522 +80523 POINT(41.40264432442809 -73.83141225805721) bank80523 +80524 POINT(40.32493999491025 -74.34089792144256) bank80524 +80525 POINT(40.29427666998574 -74.20011134535464) bank80525 +80526 POINT(40.460707790239034 -73.95172805398836) bank80526 +80527 POINT(40.00907028747715 -73.9114167329271) bank80527 +80528 POINT(41.17841366496759 -73.5799072036663) bank80528 +80529 POINT(40.452854662098055 -74.41678541800519) bank80529 +80530 POINT(41.613254410290075 -73.78746940119935) bank80530 +80531 POINT(40.01020473451443 -73.14782160096243) bank80531 +80532 POINT(40.853993717110015 -74.25297380327306) bank80532 +80533 POINT(41.569433065893115 -73.57135072080895) bank80533 +80534 POINT(40.51433182629687 -74.00749162270937) bank80534 +80535 POINT(39.950307440809325 -73.98734693112895) bank80535 +80536 POINT(41.170191281824216 -73.94881503814133) bank80536 +80537 POINT(41.14795801059907 -73.49310628620843) bank80537 +80538 POINT(40.89683705161084 -74.89420232750182) bank80538 +80539 POINT(41.63961902359431 -73.3549230057486) bank80539 +80540 POINT(39.799668618195504 -73.10389467560843) bank80540 +80541 POINT(40.47047878991237 -74.43098927656457) bank80541 +80542 POINT(40.60045823502499 -73.77614123576721) bank80542 +80543 POINT(41.182171559213046 -74.7985525704415) bank80543 +80544 POINT(40.16965206018468 -74.61748293090547) bank80544 +80545 POINT(40.60019180804309 -74.91122940584275) bank80545 +80546 POINT(41.567559551388335 -73.27564573639442) bank80546 +80547 POINT(39.94621070373588 -73.5763955529583) bank80547 +80548 POINT(39.79042075113851 -74.52359009188977) bank80548 +80549 POINT(40.156164215387804 -74.51502164186672) bank80549 +80550 POINT(40.73908758413143 -74.73394141333142) bank80550 +80551 POINT(40.60738856436139 -73.86734251027731) bank80551 +80552 POINT(39.81099105938278 -73.0949444843956) bank80552 +80553 POINT(41.57845847446584 -74.40090879936452) bank80553 +80554 POINT(40.736929405895786 -74.29373765600681) bank80554 +80555 POINT(40.54904148782482 -73.56422181560038) bank80555 +80556 POINT(41.54908577675386 -73.01006060217296) bank80556 +80557 POINT(40.94410748460727 -74.43063282044616) bank80557 +80558 POINT(39.89885432707117 -73.6477521532463) bank80558 +80559 POINT(41.630334471260305 -73.21095479328818) bank80559 +80560 POINT(40.20245387238029 -74.34538699958772) bank80560 +80561 POINT(40.90480654133196 -73.02940822967088) bank80561 +80562 POINT(41.3322473927168 -73.54421680470121) bank80562 +80563 POINT(39.89750228515042 -73.39131390164523) bank80563 +80564 POINT(41.52327370458521 -74.2025324152901) bank80564 +80565 POINT(39.85255915013099 -73.68766301997003) bank80565 +80566 POINT(40.725969455645824 -74.18843796297126) bank80566 +80567 POINT(41.26686282669708 -73.80302023627722) bank80567 +80568 POINT(40.74750880533753 -73.812875658175) bank80568 +80569 POINT(41.27460981219275 -73.036658076602) bank80569 +80570 POINT(40.216681708434336 -74.3219040708606) bank80570 +80571 POINT(40.27340968967636 -73.82500072042443) bank80571 +80572 POINT(41.13624034282233 -73.7721546096828) bank80572 +80573 POINT(41.57784464594126 -74.21354302297368) bank80573 +80574 POINT(39.96770101449121 -73.84908183258425) bank80574 +80575 POINT(41.41622698423069 -74.45789945303176) bank80575 +80576 POINT(40.86999651786364 -74.69623666071361) bank80576 +80577 POINT(40.391525885521474 -74.4168586755103) bank80577 +80578 POINT(39.80517980816359 -74.12710518712996) bank80578 +80579 POINT(40.9146708167169 -73.70288662306548) bank80579 +80580 POINT(41.37332616055573 -73.98816817347684) bank80580 +80581 POINT(41.05879743067284 -73.15825621919763) bank80581 +80582 POINT(40.115792247100806 -73.99460239422862) bank80582 +80583 POINT(41.27293201740331 -74.32432303080806) bank80583 +80584 POINT(41.32798072323817 -73.62706084062013) bank80584 +80585 POINT(40.82494511925777 -74.81600586104679) bank80585 +80586 POINT(40.984686688494385 -73.68469765162295) bank80586 +80587 POINT(40.34716081837916 -74.23844799117423) bank80587 +80588 POINT(40.58345212036222 -74.66136459297815) bank80588 +80589 POINT(41.25517831560885 -74.31107724410218) bank80589 +80590 POINT(40.47208682471691 -74.45586871541025) bank80590 +80591 POINT(41.10522561602398 -73.490031707954) bank80591 +80592 POINT(40.21647418819958 -73.8537878934435) bank80592 +80593 POINT(41.192164279517705 -74.80705550158751) bank80593 +80594 POINT(40.53523255803582 -73.37847529434833) bank80594 +80595 POINT(40.814838089279974 -73.13034737074031) bank80595 +80596 POINT(41.52476720574895 -74.29168256391971) bank80596 +80597 POINT(40.18442227388097 -73.01549957453972) bank80597 +80598 POINT(39.9133817211204 -73.18363648574622) bank80598 +80599 POINT(39.86090243496221 -73.03328587708266) bank80599 +80600 POINT(41.577741148386735 -73.23876030976743) bank80600 +80601 POINT(39.78801634197404 -74.87339791668441) bank80601 +80602 POINT(40.7492853588499 -75.00278083989444) bank80602 +80603 POINT(40.88866428489718 -73.60858114461584) bank80603 +80604 POINT(41.2988911439598 -73.12190297041381) bank80604 +80605 POINT(39.91614042771656 -73.12427215360081) bank80605 +80606 POINT(41.66478833480115 -73.7179917261065) bank80606 +80607 POINT(40.91751046134521 -73.09932759360086) bank80607 +80608 POINT(40.95776686732386 -73.27616678579987) bank80608 +80609 POINT(41.585844917513306 -74.32891754831532) bank80609 +80610 POINT(41.63629062833706 -73.97777530598101) bank80610 +80611 POINT(40.84351573486382 -74.3312238499842) bank80611 +80612 POINT(40.46675462604946 -73.33879994643654) bank80612 +80613 POINT(40.045439226549554 -74.27325209419016) bank80613 +80614 POINT(41.65963585012524 -73.23499667627732) bank80614 +80615 POINT(40.18816215051282 -74.26932468916407) bank80615 +80616 POINT(41.21540508323628 -73.89856533540743) bank80616 +80617 POINT(41.60319534617951 -74.67480575636381) bank80617 +80618 POINT(41.55824355145356 -74.27894174982718) bank80618 +80619 POINT(40.07100784210639 -74.44215267877188) bank80619 +80620 POINT(41.445557157799065 -74.19787749945382) bank80620 +80621 POINT(40.51500903895767 -73.0107096306732) bank80621 +80622 POINT(41.53591833016681 -73.36427896333723) bank80622 +80623 POINT(41.040746422352136 -73.49588321333769) bank80623 +80624 POINT(39.92386659746281 -73.71290967457598) bank80624 +80625 POINT(41.14477337415821 -74.04699494458502) bank80625 +80626 POINT(39.83823581651154 -74.76831402417936) bank80626 +80627 POINT(41.33900865176453 -74.23856036897858) bank80627 +80628 POINT(41.45460766471102 -74.44616833601914) bank80628 +80629 POINT(41.39604652555403 -73.44988785292509) bank80629 +80630 POINT(41.11110613790233 -74.25410808146474) bank80630 +80631 POINT(40.86382400425816 -73.14051156084044) bank80631 +80632 POINT(40.600206857722995 -73.24934719213088) bank80632 +80633 POINT(41.09974884782189 -73.32448242124961) bank80633 +80634 POINT(39.950227974426795 -73.4365203910281) bank80634 +80635 POINT(41.63494298020778 -74.3416962970622) bank80635 +80636 POINT(40.4383585147431 -73.56097265063654) bank80636 +80637 POINT(40.89052230311133 -73.63821501199668) bank80637 +80638 POINT(39.92060033782812 -74.17835162031041) bank80638 +80639 POINT(40.192667232097705 -74.42764493230692) bank80639 +80640 POINT(40.19067129181352 -73.12817235979792) bank80640 +80641 POINT(41.66814093101147 -73.95104502157369) bank80641 +80642 POINT(40.92877766156956 -73.93888859586565) bank80642 +80643 POINT(40.876986531206946 -73.99006599760799) bank80643 +80644 POINT(41.05781386563641 -73.68574747202648) bank80644 +80645 POINT(41.555214433835054 -74.2228773594363) bank80645 +80646 POINT(40.85962186840234 -74.57406834037103) bank80646 +80647 POINT(40.2082615712051 -74.9643735832115) bank80647 +80648 POINT(41.70999079383632 -74.81010029818349) bank80648 +80649 POINT(40.846389308925694 -73.64653920303128) bank80649 +80650 POINT(41.522299242852036 -73.87077681243791) bank80650 +80651 POINT(41.12331441963592 -74.0501310864287) bank80651 +80652 POINT(39.94302918302606 -73.793688601596) bank80652 +80653 POINT(40.66866646651016 -73.92906859054095) bank80653 +80654 POINT(40.591151686884004 -74.07261629987376) bank80654 +80655 POINT(40.035002689911146 -74.82962565921595) bank80655 +80656 POINT(40.94184686941605 -73.22472412014612) bank80656 +80657 POINT(41.49066838110622 -74.4077522955465) bank80657 +80658 POINT(40.196158352005774 -73.52564187738568) bank80658 +80659 POINT(41.12445759658327 -74.65620970001505) bank80659 +80660 POINT(40.240200738510055 -73.08798288523136) bank80660 +80661 POINT(39.86572153659762 -74.54378088877813) bank80661 +80662 POINT(39.839405150946774 -73.87368782024407) bank80662 +80663 POINT(41.70558354632991 -74.46178505193014) bank80663 +80664 POINT(40.550649795304714 -73.18816702474767) bank80664 +80665 POINT(40.90173541061035 -73.86490394169235) bank80665 +80666 POINT(40.21698025959936 -73.20002810624896) bank80666 +80667 POINT(40.043831174953404 -73.78914304998382) bank80667 +80668 POINT(41.43394181329031 -74.04805409700143) bank80668 +80669 POINT(40.42297685744132 -74.45266639872855) bank80669 +80670 POINT(39.856232771946964 -73.51192104940996) bank80670 +80671 POINT(40.116066043486136 -73.32368097531082) bank80671 +80672 POINT(40.78736242033754 -73.52214247609749) bank80672 +80673 POINT(40.91290705625173 -74.72032326035341) bank80673 +80674 POINT(39.72407598109955 -74.48278160690595) bank80674 +80675 POINT(40.701637412413326 -74.63635001749361) bank80675 +80676 POINT(40.777498599492354 -74.92711697541408) bank80676 +80677 POINT(41.45704215238847 -73.97704093222417) bank80677 +80678 POINT(40.3291978100344 -74.99479585418268) bank80678 +80679 POINT(40.048851413135466 -73.80943831871411) bank80679 +80680 POINT(40.32146816608112 -74.55349491539467) bank80680 +80681 POINT(39.89658240241877 -74.13653243849313) bank80681 +80682 POINT(41.61262228064627 -74.19987391047133) bank80682 +80683 POINT(40.43036892466718 -74.38943308155145) bank80683 +80684 POINT(41.67795189680492 -73.87016160405062) bank80684 +80685 POINT(40.982147530759384 -73.87503917712449) bank80685 +80686 POINT(40.34697368103872 -74.80152417916925) bank80686 +80687 POINT(40.07075644009775 -73.4003126178812) bank80687 +80688 POINT(40.84852124678666 -73.38798341109366) bank80688 +80689 POINT(40.01287822806242 -73.69982774844087) bank80689 +80690 POINT(41.32338829615202 -74.00865341409813) bank80690 +80691 POINT(40.813572908320836 -73.21704982654715) bank80691 +80692 POINT(41.077010134091445 -74.51188687682115) bank80692 +80693 POINT(40.53989788297002 -74.5747002949218) bank80693 +80694 POINT(40.76079493690067 -74.34229345837154) bank80694 +80695 POINT(40.1217443247094 -74.3084476801209) bank80695 +80696 POINT(40.604227297548704 -74.89398348101159) bank80696 +80697 POINT(40.4556348258281 -73.02038012706994) bank80697 +80698 POINT(40.14051660351177 -74.70438394620471) bank80698 +80699 POINT(41.65640447260729 -74.90632934196803) bank80699 +80700 POINT(40.462143456826695 -73.54440328551162) bank80700 +80701 POINT(40.38299734905359 -73.43615679571685) bank80701 +80702 POINT(39.86724309981435 -74.30584659879975) bank80702 +80703 POINT(41.69640007290102 -74.83508068269536) bank80703 +80704 POINT(39.72821100061241 -73.35310299689323) bank80704 +80705 POINT(40.006338817896335 -73.95036104538208) bank80705 +80706 POINT(40.8130125023871 -73.72708101521992) bank80706 +80707 POINT(41.062873646023796 -74.11582568028359) bank80707 +80708 POINT(39.897029348158036 -74.91791472387095) bank80708 +80709 POINT(40.72360842563589 -74.79621589252494) bank80709 +80710 POINT(40.14911037071625 -74.75110569130283) bank80710 +80711 POINT(41.3776818752662 -73.60484642292042) bank80711 +80712 POINT(40.53140311045303 -74.77796434790882) bank80712 +80713 POINT(41.60135556458673 -73.22331260530578) bank80713 +80714 POINT(41.04174612534855 -74.30299726012396) bank80714 +80715 POINT(40.018614961766886 -73.31032807968597) bank80715 +80716 POINT(40.90020214833077 -73.72614614917394) bank80716 +80717 POINT(40.506944879896785 -74.38945033695147) bank80717 +80718 POINT(40.17349385474995 -74.63566015093217) bank80718 +80719 POINT(40.03831412815061 -73.0480317017849) bank80719 +80720 POINT(41.361599806659854 -73.7314529870626) bank80720 +80721 POINT(41.11192182297539 -73.10616369150196) bank80721 +80722 POINT(41.25548793951884 -73.72171496204497) bank80722 +80723 POINT(40.83763230625864 -73.54920591433405) bank80723 +80724 POINT(41.19405509506395 -73.88507338086491) bank80724 +80725 POINT(40.99878081533452 -74.74277216625845) bank80725 +80726 POINT(41.64940782446724 -73.68137271604017) bank80726 +80727 POINT(40.98126634021327 -74.02931981268134) bank80727 +80728 POINT(41.634673391321556 -73.29586985809483) bank80728 +80729 POINT(40.40814663302464 -73.81663901856933) bank80729 +80730 POINT(39.738237403019454 -73.8402225097347) bank80730 +80731 POINT(40.3060113634909 -74.68489939781242) bank80731 +80732 POINT(40.090880774675085 -74.85840537621988) bank80732 +80733 POINT(40.05467935240794 -73.57135852231117) bank80733 +80734 POINT(40.63083744242585 -74.62664932896833) bank80734 +80735 POINT(40.030949432053355 -73.87341129966059) bank80735 +80736 POINT(39.81711912529272 -73.9418920761264) bank80736 +80737 POINT(41.463879272759236 -73.11107841932102) bank80737 +80738 POINT(40.644808178622 -73.10589911348754) bank80738 +80739 POINT(41.45514086198499 -74.27866938893453) bank80739 +80740 POINT(39.91486120052872 -73.99088665876008) bank80740 +80741 POINT(40.17141942982584 -74.67410733709696) bank80741 +80742 POINT(40.886633563066674 -73.49605242482167) bank80742 +80743 POINT(40.413328564858574 -73.7785183874476) bank80743 +80744 POINT(40.09524480392739 -74.86967184551662) bank80744 +80745 POINT(39.957344626564556 -73.67751306605757) bank80745 +80746 POINT(40.449897317060454 -73.74061462388961) bank80746 +80747 POINT(40.918286785792404 -73.10231636445813) bank80747 +80748 POINT(39.96372835834379 -73.99734669542411) bank80748 +80749 POINT(40.568407576586885 -73.27689145501802) bank80749 +80750 POINT(40.04731184131908 -74.30126807129864) bank80750 +80751 POINT(40.32290476472823 -73.79954486151337) bank80751 +80752 POINT(40.800798236652795 -73.09094868639289) bank80752 +80753 POINT(40.36384022129458 -74.70737278709561) bank80753 +80754 POINT(41.391292840949255 -73.56662236674241) bank80754 +80755 POINT(40.07420080507616 -73.6182740159636) bank80755 +80756 POINT(40.5781897441843 -74.84792649369177) bank80756 +80757 POINT(40.55567234362528 -73.45703456729615) bank80757 +80758 POINT(39.80443557274337 -74.8767404248007) bank80758 +80759 POINT(39.86719905836029 -73.90166702631645) bank80759 +80760 POINT(41.31610678917916 -74.99946468991891) bank80760 +80761 POINT(41.054752655253765 -73.42651644638421) bank80761 +80762 POINT(40.18694362653487 -74.54824557010602) bank80762 +80763 POINT(39.89066354645053 -73.07064254643134) bank80763 +80764 POINT(40.106417053133555 -73.09774304106567) bank80764 +80765 POINT(40.731228616698196 -73.05291734123793) bank80765 +80766 POINT(39.95910291934429 -74.55470039201278) bank80766 +80767 POINT(40.73580943821693 -74.8915149499842) bank80767 +80768 POINT(40.101980793482305 -73.30174818996068) bank80768 +80769 POINT(41.08062437367024 -73.79610838318227) bank80769 +80770 POINT(41.126984919263975 -73.65111555603957) bank80770 +80771 POINT(39.98225574210324 -73.83273696550113) bank80771 +80772 POINT(39.91894055340065 -74.42211045320262) bank80772 +80773 POINT(41.43295715543466 -73.07163264962246) bank80773 +80774 POINT(40.68247024455541 -74.877998416557) bank80774 +80775 POINT(41.113278776687444 -73.58941319865815) bank80775 +80776 POINT(40.78236299416844 -73.79494686912776) bank80776 +80777 POINT(39.90889371471751 -73.68247395530297) bank80777 +80778 POINT(40.99820721368288 -74.00150274697609) bank80778 +80779 POINT(41.58490789183697 -74.21532396447854) bank80779 +80780 POINT(40.97258945707336 -74.79762292878473) bank80780 +80781 POINT(39.94511565823 -73.66510818806636) bank80781 +80782 POINT(40.19068228267475 -73.31553953871337) bank80782 +80783 POINT(40.575691393304425 -73.83947009592303) bank80783 +80784 POINT(39.941931194463386 -74.29606396708331) bank80784 +80785 POINT(41.21860398204404 -74.48271242073193) bank80785 +80786 POINT(41.51504526488705 -73.87557481511337) bank80786 +80787 POINT(40.47285165630663 -73.50892899891792) bank80787 +80788 POINT(40.334508133765326 -73.09065230604315) bank80788 +80789 POINT(40.18691529961699 -74.68910231755547) bank80789 +80790 POINT(41.359588919675026 -74.1803207777966) bank80790 +80791 POINT(40.360393576751314 -73.65604294052793) bank80791 +80792 POINT(41.24485125610419 -73.43721246887401) bank80792 +80793 POINT(40.15853440271396 -73.21016827512202) bank80793 +80794 POINT(40.66986639422942 -73.49428778822241) bank80794 +80795 POINT(41.46156481251049 -73.84039477401657) bank80795 +80796 POINT(40.30581913992297 -74.38039947171889) bank80796 +80797 POINT(41.26574716870856 -73.92840068429054) bank80797 +80798 POINT(40.61352747382026 -73.09679016047912) bank80798 +80799 POINT(41.24365901118976 -73.63432163644319) bank80799 +80800 POINT(39.94692578862161 -74.41416711969505) bank80800 +80801 POINT(40.78649883287608 -74.22736938982116) bank80801 +80802 POINT(40.27552704580576 -74.77017461553302) bank80802 +80803 POINT(40.593794384331154 -74.94641877404555) bank80803 +80804 POINT(39.94979973567912 -73.73125677121213) bank80804 +80805 POINT(40.09545542747986 -74.63165626718138) bank80805 +80806 POINT(40.563475581869376 -73.58890123946107) bank80806 +80807 POINT(40.02583115610996 -74.33702390912327) bank80807 +80808 POINT(40.61391302369521 -75.00273737982928) bank80808 +80809 POINT(40.20514289921299 -74.46816870348395) bank80809 +80810 POINT(41.642323299469815 -74.36098541498029) bank80810 +80811 POINT(40.34816151163086 -73.59106459721306) bank80811 +80812 POINT(40.024802966039815 -74.32633337475907) bank80812 +80813 POINT(40.31346149549146 -73.8313477544589) bank80813 +80814 POINT(41.04276158677981 -73.21877189922908) bank80814 +80815 POINT(39.786275176885866 -73.91556383581089) bank80815 +80816 POINT(40.479447914802925 -74.51627713923612) bank80816 +80817 POINT(41.51685780301928 -73.26693092927238) bank80817 +80818 POINT(40.46772120554372 -73.33403779181138) bank80818 +80819 POINT(39.74091288247902 -73.27674277618564) bank80819 +80820 POINT(41.05521498452581 -73.62857438393593) bank80820 +80821 POINT(39.90980799932099 -74.67870663272431) bank80821 +80822 POINT(41.54348840868186 -74.6146993724338) bank80822 +80823 POINT(40.50960005944388 -73.85935502411866) bank80823 +80824 POINT(40.9572867242783 -73.53916361903437) bank80824 +80825 POINT(41.286075558925454 -73.13449130056505) bank80825 +80826 POINT(40.61497133651868 -73.46588605326804) bank80826 +80827 POINT(41.26290685976815 -73.07713348064013) bank80827 +80828 POINT(40.69379889069386 -74.38236272016536) bank80828 +80829 POINT(41.62217987323496 -73.61795371110034) bank80829 +80830 POINT(41.15397262629395 -74.54784479126909) bank80830 +80831 POINT(40.51658917970766 -74.46684180930431) bank80831 +80832 POINT(40.030516508021215 -74.8002610291618) bank80832 +80833 POINT(41.26660054016777 -74.6430514732463) bank80833 +80834 POINT(41.67460472693406 -73.01259725304793) bank80834 +80835 POINT(40.594677820344 -73.39441322506227) bank80835 +80836 POINT(40.386124817054295 -73.63053018921441) bank80836 +80837 POINT(41.68063395274045 -73.91565187016926) bank80837 +80838 POINT(40.30805766872163 -73.11810344771114) bank80838 +80839 POINT(41.56352352611252 -73.61647792581826) bank80839 +80840 POINT(41.1578035633624 -73.96710174414827) bank80840 +80841 POINT(41.29726326146014 -73.2864889504863) bank80841 +80842 POINT(40.193666241790275 -74.11524234265084) bank80842 +80843 POINT(41.19551450983315 -74.92632180537744) bank80843 +80844 POINT(40.913582318043865 -73.97547542880277) bank80844 +80845 POINT(40.41313194085653 -74.1136964545054) bank80845 +80846 POINT(41.525247702848525 -73.29784887317307) bank80846 +80847 POINT(41.663806465047344 -74.81990747626965) bank80847 +80848 POINT(41.36437471022101 -73.15227920456334) bank80848 +80849 POINT(39.97730072239478 -73.47738204622051) bank80849 +80850 POINT(40.5632599270758 -73.50031594041745) bank80850 +80851 POINT(40.404913193783 -74.20396121686949) bank80851 +80852 POINT(41.58801438119862 -73.11301050337046) bank80852 +80853 POINT(40.62703809010036 -74.41904724050586) bank80853 +80854 POINT(39.849781165353534 -73.80635659161645) bank80854 +80855 POINT(40.34984691078584 -74.35782775476518) bank80855 +80856 POINT(41.5007200260035 -74.441710197989) bank80856 +80857 POINT(40.9632425399134 -73.33507950826036) bank80857 +80858 POINT(39.92997801729697 -73.73783841013696) bank80858 +80859 POINT(41.38221842391453 -73.25191006458417) bank80859 +80860 POINT(41.22098659107819 -74.99485996530635) bank80860 +80861 POINT(41.3167119021203 -73.95363717128372) bank80861 +80862 POINT(40.51070519793429 -73.11597623352682) bank80862 +80863 POINT(41.327821101289075 -73.79933415176963) bank80863 +80864 POINT(41.694416692866966 -73.47500357825275) bank80864 +80865 POINT(40.574367153208875 -74.14662361611741) bank80865 +80866 POINT(41.28882338487106 -73.86884310771907) bank80866 +80867 POINT(40.23939556358532 -74.56307001686427) bank80867 +80868 POINT(40.09250123764186 -73.42951219962336) bank80868 +80869 POINT(41.21263117322105 -73.09424891425634) bank80869 +80870 POINT(40.65219383757747 -73.90372116986235) bank80870 +80871 POINT(40.669895981064535 -74.22879265396088) bank80871 +80872 POINT(40.561942837804615 -73.14202080914427) bank80872 +80873 POINT(39.800011673522846 -74.49907816883324) bank80873 +80874 POINT(41.583807642891884 -73.60394480218612) bank80874 +80875 POINT(39.737806969662266 -73.55062546793374) bank80875 +80876 POINT(41.540111715212575 -74.07787524674791) bank80876 +80877 POINT(40.30040816534538 -74.03474296282509) bank80877 +80878 POINT(40.70727009077378 -73.54849573270623) bank80878 +80879 POINT(41.49227340560553 -73.58023000012226) bank80879 +80880 POINT(40.64520095392219 -73.32558094581516) bank80880 +80881 POINT(40.536818707344 -74.09825803246083) bank80881 +80882 POINT(40.63625527955633 -74.44006904410081) bank80882 +80883 POINT(40.14361372418137 -74.93732859116132) bank80883 +80884 POINT(40.492769509884596 -74.4879754054179) bank80884 +80885 POINT(40.91904379152596 -74.71260628096074) bank80885 +80886 POINT(41.25613290606153 -73.55772266146992) bank80886 +80887 POINT(40.06051505091482 -73.85248646209796) bank80887 +80888 POINT(40.75146426312532 -74.181064053115) bank80888 +80889 POINT(40.09622673371089 -74.72095001759158) bank80889 +80890 POINT(41.242241389577295 -73.26226884089961) bank80890 +80891 POINT(41.32994291479816 -74.51999201716386) bank80891 +80892 POINT(39.72451805248076 -73.01586373482426) bank80892 +80893 POINT(41.34198186127351 -74.39537522458151) bank80893 +80894 POINT(40.05601746280275 -73.76405933244858) bank80894 +80895 POINT(40.65564811356673 -73.78232746895802) bank80895 +80896 POINT(40.72081867356 -74.54338923516903) bank80896 +80897 POINT(41.108031535684 -73.17856753404747) bank80897 +80898 POINT(40.255615532599386 -74.6935154122474) bank80898 +80899 POINT(40.74212720423638 -74.11855853398033) bank80899 +80900 POINT(40.19605882474142 -74.85425399388511) bank80900 +80901 POINT(41.197429212280696 -74.93253218766112) bank80901 +80902 POINT(40.4469571336389 -74.32423388113408) bank80902 +80903 POINT(40.27877428877265 -73.72653670108723) bank80903 +80904 POINT(40.26323607128899 -74.2023901984029) bank80904 +80905 POINT(41.08682853121332 -74.35532996780854) bank80905 +80906 POINT(40.3254209802951 -73.78129582279611) bank80906 +80907 POINT(41.278843349195455 -74.21768480733381) bank80907 +80908 POINT(41.13464327565191 -73.58933323292872) bank80908 +80909 POINT(40.69397866524368 -73.44883946504943) bank80909 +80910 POINT(40.10847352890428 -74.71789869278321) bank80910 +80911 POINT(41.35933773112834 -74.03608067112917) bank80911 +80912 POINT(41.438547949921 -73.75434840903456) bank80912 +80913 POINT(41.16293928563377 -74.19804047821611) bank80913 +80914 POINT(39.992591215357315 -73.23756645989644) bank80914 +80915 POINT(40.71207685337272 -74.99856225838901) bank80915 +80916 POINT(41.14851915472969 -74.16918773025706) bank80916 +80917 POINT(40.87585432909315 -73.7830821734326) bank80917 +80918 POINT(41.56062558751843 -73.42584987185896) bank80918 +80919 POINT(40.657029826818984 -74.18723998784114) bank80919 +80920 POINT(41.634688611993106 -74.42002064125198) bank80920 +80921 POINT(41.54045513350642 -73.158382096426) bank80921 +80922 POINT(41.53794599818918 -73.73276130792622) bank80922 +80923 POINT(39.829497166517584 -73.58276827279025) bank80923 +80924 POINT(41.2867825310242 -73.12663345266229) bank80924 +80925 POINT(41.02953884461172 -74.94751837915504) bank80925 +80926 POINT(40.19088274704845 -73.84187670293397) bank80926 +80927 POINT(41.63154744245709 -73.79508009935108) bank80927 +80928 POINT(41.4313132744673 -74.18491996221555) bank80928 +80929 POINT(40.239813806721834 -73.73258076976387) bank80929 +80930 POINT(41.306583924938295 -74.05839817165312) bank80930 +80931 POINT(41.069146540383635 -74.30899616494887) bank80931 +80932 POINT(39.7231096261396 -74.65199564307237) bank80932 +80933 POINT(41.29099901010652 -74.5065017428058) bank80933 +80934 POINT(40.76591728900373 -73.16997140419865) bank80934 +80935 POINT(41.23780871750273 -74.82231166173882) bank80935 +80936 POINT(40.28100764540767 -74.2846199518287) bank80936 +80937 POINT(41.23405446481151 -74.70848578575301) bank80937 +80938 POINT(40.60493407827547 -73.29207304611901) bank80938 +80939 POINT(39.992577155864765 -74.9237951074742) bank80939 +80940 POINT(41.28868449806122 -74.71485702859287) bank80940 +80941 POINT(40.16380423425943 -74.85763192201141) bank80941 +80942 POINT(41.27857346817598 -73.4291663857258) bank80942 +80943 POINT(41.21372531646943 -73.22810194585087) bank80943 +80944 POINT(41.573742381219915 -73.62559552722121) bank80944 +80945 POINT(40.909590979132275 -73.7705690384206) bank80945 +80946 POINT(41.549376034932656 -74.38280297083523) bank80946 +80947 POINT(39.8346086484684 -74.00082021442047) bank80947 +80948 POINT(40.73564003705059 -73.2254732745207) bank80948 +80949 POINT(41.1015059636264 -73.14890072301257) bank80949 +80950 POINT(41.03350917324207 -74.35827810478706) bank80950 +80951 POINT(40.76559421249529 -73.37927708766762) bank80951 +80952 POINT(41.085851806744365 -73.3357297648686) bank80952 +80953 POINT(39.95115348354949 -73.7580767197907) bank80953 +80954 POINT(40.21763337292562 -74.97281683023311) bank80954 +80955 POINT(41.04279052187794 -73.03358278985243) bank80955 +80956 POINT(40.850339630062095 -74.59831674133142) bank80956 +80957 POINT(41.556667081172336 -74.20498624602051) bank80957 +80958 POINT(41.2024559842371 -73.8870162097798) bank80958 +80959 POINT(41.041176001757975 -73.0694097286363) bank80959 +80960 POINT(40.73782938865115 -74.51286063495667) bank80960 +80961 POINT(41.61559273881745 -74.10288378427646) bank80961 +80962 POINT(41.29626664690646 -73.88097805723928) bank80962 +80963 POINT(41.27631202934936 -74.42147959972203) bank80963 +80964 POINT(40.00824047717496 -73.04199549257103) bank80964 +80965 POINT(41.4538720894658 -73.8197746790673) bank80965 +80966 POINT(40.92394113540784 -74.60997423570352) bank80966 +80967 POINT(40.58832740902235 -73.83215683151656) bank80967 +80968 POINT(40.34975934807003 -74.79459516337018) bank80968 +80969 POINT(39.818613043521154 -74.64780523079177) bank80969 +80970 POINT(40.91731677154578 -74.1264761265944) bank80970 +80971 POINT(40.75061431365433 -73.28677205660513) bank80971 +80972 POINT(41.318809826139635 -73.1112416741447) bank80972 +80973 POINT(40.00554366834646 -73.29506332395752) bank80973 +80974 POINT(41.014619069917394 -74.2422247814159) bank80974 +80975 POINT(40.94939980814069 -74.35094371904185) bank80975 +80976 POINT(41.6227728166214 -73.30294952707759) bank80976 +80977 POINT(40.55244293028895 -74.79177969740918) bank80977 +80978 POINT(40.40712171247442 -74.1199209636975) bank80978 +80979 POINT(41.07902947924266 -73.03928050878662) bank80979 +80980 POINT(39.7883648775188 -74.08581978744117) bank80980 +80981 POINT(41.109662099714015 -73.5321540452496) bank80981 +80982 POINT(40.717939099387905 -74.54664788583888) bank80982 +80983 POINT(41.16612528456075 -74.0626706831857) bank80983 +80984 POINT(41.22436385487485 -74.60605175980369) bank80984 +80985 POINT(40.10746943895876 -74.01222000296063) bank80985 +80986 POINT(41.212363752725906 -74.12896264733916) bank80986 +80987 POINT(40.49465146192295 -73.59674652196726) bank80987 +80988 POINT(41.16520504043935 -74.45155945519339) bank80988 +80989 POINT(40.12956292210193 -73.22486409604633) bank80989 +80990 POINT(40.54155113408659 -73.71143176932793) bank80990 +80991 POINT(40.40181254531967 -74.48009392357565) bank80991 +80992 POINT(40.675643027418815 -73.52387242451239) bank80992 +80993 POINT(40.252044552336834 -73.52879624707008) bank80993 +80994 POINT(40.09581862574884 -73.68516682371826) bank80994 +80995 POINT(41.06959508779224 -73.7738412306121) bank80995 +80996 POINT(39.83793397323287 -73.72377486352856) bank80996 +80997 POINT(40.45330861259247 -73.28326550808127) bank80997 +80998 POINT(40.794536478194104 -74.13306134488394) bank80998 +80999 POINT(41.39280184932715 -73.84077944116069) bank80999 +81000 POINT(40.26021143084151 -74.60343955491795) bank81000 +81001 POINT(40.68839595781751 -73.5252083315816) bank81001 +81002 POINT(41.233802630076696 -73.21990529514144) bank81002 +81003 POINT(40.07685553936126 -73.94356467948832) bank81003 +81004 POINT(40.00559717175569 -74.82255313613234) bank81004 +81005 POINT(40.90368068745907 -74.76757983008707) bank81005 +81006 POINT(41.42447349810845 -73.79407486177166) bank81006 +81007 POINT(40.241119772283255 -74.3119120961049) bank81007 +81008 POINT(41.53965120581963 -74.63246697392883) bank81008 +81009 POINT(40.270718465595095 -74.61765211808351) bank81009 +81010 POINT(40.04514603740109 -74.11628134693291) bank81010 +81011 POINT(40.439802779189165 -73.01650394247045) bank81011 +81012 POINT(41.346977511057474 -73.68602652351186) bank81012 +81013 POINT(39.9416017269717 -73.42042229394391) bank81013 +81014 POINT(41.238608364668096 -74.17811798368987) bank81014 +81015 POINT(40.422923821019786 -73.2905531795394) bank81015 +81016 POINT(41.567222390205416 -73.1009011325772) bank81016 +81017 POINT(41.10475716813905 -74.66513937773652) bank81017 +81018 POINT(40.20559811603733 -74.50604501211791) bank81018 +81019 POINT(39.753731188334896 -74.90720272155396) bank81019 +81020 POINT(40.35704628788077 -74.19803217364341) bank81020 +81021 POINT(40.90807916544291 -74.86120559959333) bank81021 +81022 POINT(40.28018015411919 -73.99127424905375) bank81022 +81023 POINT(39.85838809934525 -74.35985563278548) bank81023 +81024 POINT(41.10461931265609 -73.69872049431176) bank81024 +81025 POINT(41.530075905114735 -74.15085717042713) bank81025 +81026 POINT(40.47913758019804 -74.25220108919025) bank81026 +81027 POINT(39.871416661356875 -75.0001634484719) bank81027 +81028 POINT(39.904793096827355 -74.33018125401058) bank81028 +81029 POINT(41.194603983709605 -73.13185631459251) bank81029 +81030 POINT(41.61404733999641 -74.63600144160692) bank81030 +81031 POINT(40.69809121346255 -74.62435474336974) bank81031 +81032 POINT(39.72198261385955 -73.1900901051277) bank81032 +81033 POINT(40.06513507246904 -73.40380873527641) bank81033 +81034 POINT(41.182365618724205 -74.18317541598114) bank81034 +81035 POINT(41.568638879823496 -74.5845696894587) bank81035 +81036 POINT(40.492050821657294 -73.83273488965492) bank81036 +81037 POINT(41.01814290188351 -73.27328119907715) bank81037 +81038 POINT(41.20792765838317 -74.17346485882476) bank81038 +81039 POINT(40.35538997619761 -73.32677775221421) bank81039 +81040 POINT(41.530152496133915 -74.47884190004777) bank81040 +81041 POINT(40.74737070824015 -74.6459722987976) bank81041 +81042 POINT(40.45081977351136 -73.73289666229226) bank81042 +81043 POINT(40.05060499696291 -73.35920562230328) bank81043 +81044 POINT(40.90520552590684 -74.71209730792975) bank81044 +81045 POINT(40.29699001035868 -74.44903848827617) bank81045 +81046 POINT(39.95610094312244 -73.97047954182601) bank81046 +81047 POINT(41.19363639863651 -74.05135987037826) bank81047 +81048 POINT(40.45014071543639 -74.72859450499688) bank81048 +81049 POINT(41.18575388287584 -73.12557746732162) bank81049 +81050 POINT(40.13038109140163 -73.15327962237615) bank81050 +81051 POINT(40.02651031740861 -74.14519869724728) bank81051 +81052 POINT(41.70310994963797 -73.70914679635335) bank81052 +81053 POINT(40.03982454969771 -74.29153322650626) bank81053 +81054 POINT(41.394789299886796 -73.14701796425251) bank81054 +81055 POINT(40.908173580191395 -73.40289793359038) bank81055 +81056 POINT(41.53772813412901 -74.30758429919081) bank81056 +81057 POINT(40.577057043016204 -73.84749112975084) bank81057 +81058 POINT(41.2128395725619 -73.25624668594598) bank81058 +81059 POINT(41.5337503280401 -73.80553276850792) bank81059 +81060 POINT(40.039005429936964 -73.93429278202662) bank81060 +81061 POINT(41.555455565256125 -73.60370360430795) bank81061 +81062 POINT(40.89085465308673 -73.62185600527054) bank81062 +81063 POINT(39.975763030559904 -74.57652764203411) bank81063 +81064 POINT(40.86302889725106 -73.75732135622904) bank81064 +81065 POINT(40.872389096818246 -74.60987500506373) bank81065 +81066 POINT(40.46212750989699 -73.30321279022961) bank81066 +81067 POINT(39.78709618564184 -73.88733901769928) bank81067 +81068 POINT(40.13432225287697 -74.28711024948055) bank81068 +81069 POINT(40.4964615509754 -73.89337932088236) bank81069 +81070 POINT(40.536778219391685 -73.60579987851823) bank81070 +81071 POINT(40.573067206627194 -74.47391138210045) bank81071 +81072 POINT(40.36922943273234 -74.28244251509908) bank81072 +81073 POINT(40.51001722956984 -73.8435447320193) bank81073 +81074 POINT(41.707213251811396 -74.37705206532668) bank81074 +81075 POINT(40.244408532326084 -73.10827587283057) bank81075 +81076 POINT(41.5113625678008 -73.76845554126284) bank81076 +81077 POINT(40.525520778218215 -73.82349253338874) bank81077 +81078 POINT(40.09465196904515 -73.319177094847) bank81078 +81079 POINT(40.44153855226869 -74.40935994630942) bank81079 +81080 POINT(41.41304770587955 -73.92440783218403) bank81080 +81081 POINT(40.53787903578812 -73.29655702963888) bank81081 +81082 POINT(41.433155736081694 -74.25593904642764) bank81082 +81083 POINT(41.36170147049893 -74.15519053072762) bank81083 +81084 POINT(41.228079753368725 -73.9898093554237) bank81084 +81085 POINT(40.96992215198489 -73.05448530356776) bank81085 +81086 POINT(40.62401135295668 -74.65095194371823) bank81086 +81087 POINT(40.78715828705862 -73.33462715234833) bank81087 +81088 POINT(41.519836479517856 -74.2588483499257) bank81088 +81089 POINT(41.175425160365016 -74.82619547013928) bank81089 +81090 POINT(40.77917707649813 -74.94041930993136) bank81090 +81091 POINT(40.34854829014742 -74.89443794437092) bank81091 +81092 POINT(41.03673421122563 -73.35371284929157) bank81092 +81093 POINT(40.04781562241291 -73.06773026258271) bank81093 +81094 POINT(40.62628029687584 -73.7762341629944) bank81094 +81095 POINT(40.55792311435339 -74.79928548303113) bank81095 +81096 POINT(41.30293796631219 -74.35018069851719) bank81096 +81097 POINT(41.49991437146911 -74.88520469917303) bank81097 +81098 POINT(40.813456091386044 -73.8254736629436) bank81098 +81099 POINT(41.205178607944276 -74.78946119011484) bank81099 +81100 POINT(41.62949300242815 -73.6749924109149) bank81100 +81101 POINT(39.80948269509567 -73.71803535755234) bank81101 +81102 POINT(41.31999695166855 -73.81147253257535) bank81102 +81103 POINT(40.14961856087198 -73.55495922405922) bank81103 +81104 POINT(41.1049550444676 -74.08144838363607) bank81104 +81105 POINT(40.50291752580858 -73.87042222793454) bank81105 +81106 POINT(41.26658647490934 -73.26501983349569) bank81106 +81107 POINT(41.12600513632967 -73.95511465790997) bank81107 +81108 POINT(41.31978561591955 -73.70355430043398) bank81108 +81109 POINT(40.63771077429178 -74.42587163906572) bank81109 +81110 POINT(40.76266126726853 -74.55926098627697) bank81110 +81111 POINT(40.18701783966742 -74.9795792101861) bank81111 +81112 POINT(41.0716879859615 -73.24297893141176) bank81112 +81113 POINT(40.65847919287857 -74.98269039753147) bank81113 +81114 POINT(40.8275950026264 -73.36880424092391) bank81114 +81115 POINT(39.8540854285176 -73.86865001064349) bank81115 +81116 POINT(40.71397124871513 -73.07483100037129) bank81116 +81117 POINT(41.17456870951521 -73.53861647662643) bank81117 +81118 POINT(40.47690497367123 -74.36308198332976) bank81118 +81119 POINT(40.840337458886005 -74.4003608196616) bank81119 +81120 POINT(40.79680634056335 -74.31946332134925) bank81120 +81121 POINT(40.629286358027585 -73.11672599112411) bank81121 +81122 POINT(41.19731147626284 -74.51975839905987) bank81122 +81123 POINT(40.17621112425769 -73.2540313475554) bank81123 +81124 POINT(40.102715820504706 -73.0065306608397) bank81124 +81125 POINT(40.76647457017311 -74.38283657428386) bank81125 +81126 POINT(40.57431667531062 -74.20039118337519) bank81126 +81127 POINT(41.626012318993155 -74.36645857545022) bank81127 +81128 POINT(39.92088707367541 -73.79553594387637) bank81128 +81129 POINT(39.95195322828995 -73.54303676997881) bank81129 +81130 POINT(41.062798313677305 -74.17505873820892) bank81130 +81131 POINT(39.88132059376261 -73.68343966029539) bank81131 +81132 POINT(41.51619790384884 -73.28003388306726) bank81132 +81133 POINT(41.06236200243364 -73.34466345859703) bank81133 +81134 POINT(40.479352057529574 -73.6291668589064) bank81134 +81135 POINT(41.14564391589656 -74.4122819132936) bank81135 +81136 POINT(41.348445098220395 -73.66528533663903) bank81136 +81137 POINT(41.18600695061263 -74.6366605449078) bank81137 +81138 POINT(39.90742641130419 -74.38520586186209) bank81138 +81139 POINT(41.25858890071818 -73.87892822707069) bank81139 +81140 POINT(40.76546183318891 -74.33646997841196) bank81140 +81141 POINT(41.32212480676316 -74.730948043232) bank81141 +81142 POINT(40.32539059348785 -73.64582630895961) bank81142 +81143 POINT(40.49165787481083 -73.42995048263474) bank81143 +81144 POINT(40.16001194399113 -73.28649990518647) bank81144 +81145 POINT(40.205561990479175 -73.81352323473779) bank81145 +81146 POINT(40.46797122207674 -74.69385574093572) bank81146 +81147 POINT(40.397605575608864 -73.48803111479502) bank81147 +81148 POINT(40.07339913385873 -74.08781056898236) bank81148 +81149 POINT(41.393035124493345 -73.3576057498896) bank81149 +81150 POINT(41.29321227387682 -73.51490295434553) bank81150 +81151 POINT(41.436105816937484 -74.30193577267555) bank81151 +81152 POINT(40.57364396399833 -73.63518334556221) bank81152 +81153 POINT(40.18838665056426 -74.05644054862097) bank81153 +81154 POINT(41.46258215425184 -74.00764011687568) bank81154 +81155 POINT(41.272412621152846 -73.39306061727648) bank81155 +81156 POINT(40.452093760079215 -74.93204114253564) bank81156 +81157 POINT(39.89341276240257 -73.85359122319616) bank81157 +81158 POINT(40.70232277205471 -74.90370398089266) bank81158 +81159 POINT(40.2006435315837 -74.0784837356895) bank81159 +81160 POINT(40.65358042618734 -73.56485174280232) bank81160 +81161 POINT(41.32998618670627 -73.57122393427481) bank81161 +81162 POINT(40.08070654796784 -74.05665955880036) bank81162 +81163 POINT(39.89937132452293 -73.18671856235025) bank81163 +81164 POINT(41.52434422976446 -73.06176498784272) bank81164 +81165 POINT(39.76064088111187 -73.09723879725364) bank81165 +81166 POINT(40.988014517109995 -74.3397399764182) bank81166 +81167 POINT(40.84856515751108 -74.92288569190151) bank81167 +81168 POINT(40.133740107603714 -73.77074787604087) bank81168 +81169 POINT(39.837706528479615 -74.91454642355706) bank81169 +81170 POINT(40.641745784994455 -73.3169152020256) bank81170 +81171 POINT(40.59746461222433 -73.42326678140725) bank81171 +81172 POINT(40.7364481530822 -73.69171261576062) bank81172 +81173 POINT(41.55889287846321 -74.00379928308651) bank81173 +81174 POINT(40.28457810015267 -74.54920202841161) bank81174 +81175 POINT(41.42491647039344 -74.7444372833305) bank81175 +81176 POINT(40.851777633461744 -74.63569419821383) bank81176 +81177 POINT(40.63475617951632 -74.15143573584187) bank81177 +81178 POINT(39.719781384558246 -74.98611485813976) bank81178 +81179 POINT(41.66169807092907 -73.83179570423019) bank81179 +81180 POINT(41.557356602003644 -73.77165627807791) bank81180 +81181 POINT(40.59708236341184 -73.52082098908761) bank81181 +81182 POINT(40.15743643465623 -73.93173821971622) bank81182 +81183 POINT(41.15798861717151 -74.00218346744076) bank81183 +81184 POINT(41.67537516264661 -73.8029604457324) bank81184 +81185 POINT(41.59164234303578 -73.04864823628307) bank81185 +81186 POINT(40.53417236912877 -73.11615850096867) bank81186 +81187 POINT(40.5884798800911 -74.49715449091282) bank81187 +81188 POINT(41.352258787135995 -73.78933510569212) bank81188 +81189 POINT(39.722096649140276 -73.55194791818953) bank81189 +81190 POINT(40.86950925835533 -73.47761421004218) bank81190 +81191 POINT(41.21036919189115 -73.41810871998166) bank81191 +81192 POINT(40.72782107607462 -74.3388000168518) bank81192 +81193 POINT(40.29138463936897 -73.16746591952115) bank81193 +81194 POINT(41.44803902048944 -74.56448319807097) bank81194 +81195 POINT(41.16028161315768 -74.40119649419904) bank81195 +81196 POINT(40.6191051857259 -73.59282405452959) bank81196 +81197 POINT(41.40574317049695 -73.84976487832203) bank81197 +81198 POINT(41.11171162321727 -73.13357745739249) bank81198 +81199 POINT(39.88924721556915 -73.249817668832) bank81199 +81200 POINT(40.63429441675267 -74.42243682957769) bank81200 +81201 POINT(40.46095469840586 -73.68006888402813) bank81201 +81202 POINT(40.81057767901011 -74.49030289865496) bank81202 +81203 POINT(41.36890077231302 -73.68211098161514) bank81203 +81204 POINT(41.667531539405076 -74.15355429125637) bank81204 +81205 POINT(40.27567394099227 -74.62649906122293) bank81205 +81206 POINT(41.03227660481983 -73.84854579923704) bank81206 +81207 POINT(41.58760909190653 -74.70093582357278) bank81207 +81208 POINT(41.36748204093585 -73.59159701025298) bank81208 +81209 POINT(41.54218128264489 -73.31711583781143) bank81209 +81210 POINT(41.555498060880076 -73.46967365642818) bank81210 +81211 POINT(40.65555785573042 -74.75805257044283) bank81211 +81212 POINT(39.880603067760674 -74.3317318594624) bank81212 +81213 POINT(39.991442775150254 -74.94665338484066) bank81213 +81214 POINT(40.15799857071638 -74.79084427410561) bank81214 +81215 POINT(39.767630450545326 -74.01108404319076) bank81215 +81216 POINT(40.1223935478066 -74.83733011835749) bank81216 +81217 POINT(41.5827074518144 -73.96829088133182) bank81217 +81218 POINT(41.60509828790367 -73.22266023695214) bank81218 +81219 POINT(40.757215195308696 -73.43934792085909) bank81219 +81220 POINT(39.89515132810001 -74.7267334787782) bank81220 +81221 POINT(40.336445065958046 -73.7809287517065) bank81221 +81222 POINT(41.16849488617864 -73.21980018296439) bank81222 +81223 POINT(40.33265443491414 -73.94181328113683) bank81223 +81224 POINT(40.16882905089715 -73.45936734895452) bank81224 +81225 POINT(39.85687618426597 -74.00509674279613) bank81225 +81226 POINT(41.4809314447144 -74.25196841928572) bank81226 +81227 POINT(40.640440004206255 -73.80838530419308) bank81227 +81228 POINT(41.55404177294877 -74.9623116976894) bank81228 +81229 POINT(41.18853144742406 -73.52902265967299) bank81229 +81230 POINT(41.25073418839267 -74.58322531192825) bank81230 +81231 POINT(40.32352591744293 -73.21705161993083) bank81231 +81232 POINT(41.43791529575281 -73.2691431174372) bank81232 +81233 POINT(40.23952960727225 -74.55985446095215) bank81233 +81234 POINT(40.14014398977243 -74.1409008679763) bank81234 +81235 POINT(40.84860116900218 -73.81246609614342) bank81235 +81236 POINT(40.931667844951384 -73.42270723935331) bank81236 +81237 POINT(40.15785873532228 -74.38936198717536) bank81237 +81238 POINT(40.67515759889143 -74.40550444854587) bank81238 +81239 POINT(41.208158332768285 -74.59795180203615) bank81239 +81240 POINT(40.068077232376844 -74.53148759142813) bank81240 +81241 POINT(41.15515238851386 -74.16993995953236) bank81241 +81242 POINT(40.13561396268373 -74.08065533137038) bank81242 +81243 POINT(39.80881663202162 -73.77443147471108) bank81243 +81244 POINT(40.65532785543362 -74.70454576990608) bank81244 +81245 POINT(40.44209791647963 -74.15733747039306) bank81245 +81246 POINT(40.46598906262145 -74.6555183788689) bank81246 +81247 POINT(40.388726701966185 -74.41029258597045) bank81247 +81248 POINT(40.933582137623404 -74.4025220813002) bank81248 +81249 POINT(40.88635158978515 -73.15779975934367) bank81249 +81250 POINT(39.76581728058303 -73.70361888653049) bank81250 +81251 POINT(40.13536908381862 -74.44463043594838) bank81251 +81252 POINT(40.92296054868903 -74.36917636884019) bank81252 +81253 POINT(40.79369665883501 -74.245559144127) bank81253 +81254 POINT(40.27019191754979 -74.76800991598702) bank81254 +81255 POINT(40.902980702130236 -73.3418023412901) bank81255 +81256 POINT(41.07244043772846 -74.56138070750193) bank81256 +81257 POINT(39.99220123961542 -73.97615630769849) bank81257 +81258 POINT(39.93537388356576 -73.03070634107542) bank81258 +81259 POINT(40.891393287412576 -74.13148291291841) bank81259 +81260 POINT(40.01945716683638 -73.29863011956452) bank81260 +81261 POINT(40.93957944847754 -73.11306200916877) bank81261 +81262 POINT(41.345984945769054 -74.2240026184036) bank81262 +81263 POINT(39.89824414031315 -74.41935684904763) bank81263 +81264 POINT(40.946963258011365 -73.8958905228452) bank81264 +81265 POINT(39.80849273837943 -73.53254806143742) bank81265 +81266 POINT(40.593890492845546 -73.6139653599482) bank81266 +81267 POINT(41.12501689939021 -73.55310622451216) bank81267 +81268 POINT(40.40482835166248 -74.77915641764099) bank81268 +81269 POINT(41.335898077101106 -73.97720301527708) bank81269 +81270 POINT(39.92652507823659 -74.85144827143267) bank81270 +81271 POINT(40.23884151901274 -73.43137665301457) bank81271 +81272 POINT(41.61995712715636 -73.38854669080922) bank81272 +81273 POINT(40.033909222627926 -74.47105990129965) bank81273 +81274 POINT(39.861383617122144 -73.79257794155042) bank81274 +81275 POINT(40.332435896891035 -74.70360933433803) bank81275 +81276 POINT(40.42424078241269 -73.08174830322858) bank81276 +81277 POINT(39.75485684578636 -73.28345816389184) bank81277 +81278 POINT(40.77950724581563 -74.06474098667066) bank81278 +81279 POINT(40.42226449795983 -74.36446319828309) bank81279 +81280 POINT(40.21379250454978 -73.65865457477454) bank81280 +81281 POINT(41.559040579089185 -74.0046255389938) bank81281 +81282 POINT(40.25168272766517 -73.88397373797171) bank81282 +81283 POINT(40.88514297512645 -75.00578321717288) bank81283 +81284 POINT(41.05365746541893 -74.37867103069279) bank81284 +81285 POINT(39.79163008393537 -73.78800041961912) bank81285 +81286 POINT(40.245795729818255 -73.24958747500395) bank81286 +81287 POINT(39.81314369272796 -74.83542318329513) bank81287 +81288 POINT(40.09606171343432 -73.32295561745933) bank81288 +81289 POINT(40.613115578267774 -73.47445375984412) bank81289 +81290 POINT(40.762811047945895 -73.87440358654094) bank81290 +81291 POINT(41.46793264804776 -74.65933202418519) bank81291 +81292 POINT(41.446568666170975 -74.82157452527798) bank81292 +81293 POINT(40.58833462697174 -73.49984437430906) bank81293 +81294 POINT(40.37183818795762 -74.55396289146915) bank81294 +81295 POINT(41.12942820306751 -73.79112992217068) bank81295 +81296 POINT(40.58480774028926 -73.65509098748608) bank81296 +81297 POINT(40.89447348289919 -74.81613835801737) bank81297 +81298 POINT(40.33487477100334 -74.76596617281491) bank81298 +81299 POINT(40.76559798416702 -74.44170940046745) bank81299 +81300 POINT(41.36537720511247 -74.67245634296954) bank81300 +81301 POINT(39.7682303545538 -74.88065323723102) bank81301 +81302 POINT(40.67409079167581 -73.55047257838305) bank81302 +81303 POINT(40.49452914072717 -73.44124156255705) bank81303 +81304 POINT(40.49976013929097 -73.96589593667233) bank81304 +81305 POINT(40.65893425122607 -74.6580252379639) bank81305 +81306 POINT(41.66728253508364 -74.71751728113689) bank81306 +81307 POINT(40.235312326513125 -74.48049747600562) bank81307 +81308 POINT(41.03494366109186 -74.22121534831861) bank81308 +81309 POINT(41.32929271745586 -74.53765934934421) bank81309 +81310 POINT(40.00541166130414 -73.28628084602185) bank81310 +81311 POINT(41.471135694311776 -73.88216240273351) bank81311 +81312 POINT(40.410567982725176 -73.7945295863326) bank81312 +81313 POINT(41.64900276800629 -74.82068883949984) bank81313 +81314 POINT(40.668895661359805 -73.8779059204982) bank81314 +81315 POINT(39.84792585561563 -74.11023692872119) bank81315 +81316 POINT(39.86621454100206 -73.4457529598459) bank81316 +81317 POINT(40.77053948509828 -74.29931605803557) bank81317 +81318 POINT(40.048734210500676 -74.05549568948067) bank81318 +81319 POINT(39.76442988453838 -74.9745087017955) bank81319 +81320 POINT(40.34342123663967 -74.41245613256775) bank81320 +81321 POINT(40.41437847975223 -73.43246610458172) bank81321 +81322 POINT(41.66639728995608 -74.98597278480308) bank81322 +81323 POINT(41.640860392506724 -74.2864528852516) bank81323 +81324 POINT(41.47814157202881 -74.74820210070348) bank81324 +81325 POINT(39.989631033307816 -74.11861234656648) bank81325 +81326 POINT(40.09618856725396 -73.20997356809782) bank81326 +81327 POINT(40.77574677795087 -74.17845634381598) bank81327 +81328 POINT(39.889814591044264 -74.809580426101) bank81328 +81329 POINT(40.14533213226911 -73.75131030948715) bank81329 +81330 POINT(39.88893390190997 -74.7452006270263) bank81330 +81331 POINT(39.97308480867391 -73.6583231947324) bank81331 +81332 POINT(40.65057833375339 -73.65856422051004) bank81332 +81333 POINT(40.800018254576486 -74.96931350845949) bank81333 +81334 POINT(41.23617751352944 -74.10734130352144) bank81334 +81335 POINT(41.00335982416685 -73.65133556383316) bank81335 +81336 POINT(40.948201038222464 -74.60486389634868) bank81336 +81337 POINT(40.090713915196964 -74.39436849004231) bank81337 +81338 POINT(41.60188560913304 -73.86052866524747) bank81338 +81339 POINT(40.70641957580081 -73.49525969338102) bank81339 +81340 POINT(39.83980168488007 -74.46386194983214) bank81340 +81341 POINT(40.17911339369851 -73.05406987351655) bank81341 +81342 POINT(41.12600761059869 -74.74215293255928) bank81342 +81343 POINT(40.71692711609158 -73.25793892449798) bank81343 +81344 POINT(40.32583911148356 -74.46086870099187) bank81344 +81345 POINT(41.08028306934368 -73.64813059820727) bank81345 +81346 POINT(40.93866407029172 -74.84520527829436) bank81346 +81347 POINT(40.7605290059965 -74.1588173945593) bank81347 +81348 POINT(39.890722050850655 -73.04591008764163) bank81348 +81349 POINT(40.46182850897333 -73.99286117222822) bank81349 +81350 POINT(40.39556367423182 -74.80818591397778) bank81350 +81351 POINT(41.48303314504609 -74.63889524447384) bank81351 +81352 POINT(41.652033556813315 -73.44250687556303) bank81352 +81353 POINT(41.324098356307324 -74.0007723896313) bank81353 +81354 POINT(41.352293749083366 -73.0419628975654) bank81354 +81355 POINT(40.871741944202164 -74.80965234742459) bank81355 +81356 POINT(41.282974439165066 -74.0814651856914) bank81356 +81357 POINT(39.79983981253476 -73.31150469648254) bank81357 +81358 POINT(41.43472559113402 -74.44024579400106) bank81358 +81359 POINT(40.88019414338299 -73.19613018387003) bank81359 +81360 POINT(41.25950609021581 -74.96036252970556) bank81360 +81361 POINT(40.08787125501691 -74.67712240274838) bank81361 +81362 POINT(40.21829024143471 -73.74771172774136) bank81362 +81363 POINT(40.312702635521426 -73.29496540860036) bank81363 +81364 POINT(39.798479267462305 -74.72075779985377) bank81364 +81365 POINT(40.22319269286453 -73.10677291679579) bank81365 +81366 POINT(41.52006958000636 -74.4318288376328) bank81366 +81367 POINT(40.778551330502175 -74.73528273777805) bank81367 +81368 POINT(40.5602358654496 -74.96426606544996) bank81368 +81369 POINT(40.459734119379576 -73.05757589703221) bank81369 +81370 POINT(41.57447482919862 -74.42578392242461) bank81370 +81371 POINT(39.84047017108035 -73.79207321452546) bank81371 +81372 POINT(41.28309068446087 -74.39391401166125) bank81372 +81373 POINT(41.61278745789517 -74.62776259394224) bank81373 +81374 POINT(41.39158353656096 -73.57433001043661) bank81374 +81375 POINT(41.49301963441031 -74.000002050726) bank81375 +81376 POINT(40.63657609187155 -74.12813462015006) bank81376 +81377 POINT(40.27655981905244 -74.11103226519242) bank81377 +81378 POINT(40.36531285030721 -74.35848988616897) bank81378 +81379 POINT(40.258956771936795 -73.99061957302334) bank81379 +81380 POINT(40.91871120681752 -74.86479829173106) bank81380 +81381 POINT(40.235698198092486 -73.43623319367728) bank81381 +81382 POINT(39.77982776352572 -74.3870997429353) bank81382 +81383 POINT(41.28758489201562 -73.01505473646459) bank81383 +81384 POINT(40.307073243552246 -74.12709288277763) bank81384 +81385 POINT(40.907618930276755 -74.23718073429721) bank81385 +81386 POINT(39.86105676058125 -74.21461911595377) bank81386 +81387 POINT(41.45783069645573 -74.84880917485383) bank81387 +81388 POINT(41.52611859849977 -74.4138675961366) bank81388 +81389 POINT(40.641097876229495 -73.01727852179125) bank81389 +81390 POINT(40.1417372538183 -74.83603967042305) bank81390 +81391 POINT(40.23922388166346 -73.73859079117534) bank81391 +81392 POINT(40.679178691755126 -74.79344860413116) bank81392 +81393 POINT(40.18173192788909 -73.86235974595034) bank81393 +81394 POINT(41.582953399549396 -73.1391624321574) bank81394 +81395 POINT(40.69039828365751 -73.20454575479093) bank81395 +81396 POINT(40.905998595648285 -74.848927528531) bank81396 +81397 POINT(39.95807336921876 -74.14382526422685) bank81397 +81398 POINT(41.53548239373137 -74.01694352183853) bank81398 +81399 POINT(41.58490387036122 -73.08970481467263) bank81399 +81400 POINT(40.33316102389686 -74.45234044354315) bank81400 +81401 POINT(40.74679654302638 -74.66057361081961) bank81401 +81402 POINT(41.17848536891814 -74.50665512467904) bank81402 +81403 POINT(40.191260028232335 -74.44847236976484) bank81403 +81404 POINT(40.28722785068707 -74.02255770398452) bank81404 +81405 POINT(40.75245620511918 -73.23527780512431) bank81405 +81406 POINT(41.42011571113766 -74.56265672234888) bank81406 +81407 POINT(41.425434326117795 -73.34413987437235) bank81407 +81408 POINT(39.99613081049285 -74.06180172438532) bank81408 +81409 POINT(40.10030338787434 -73.54619811072092) bank81409 +81410 POINT(41.175126723941155 -74.34316674110755) bank81410 +81411 POINT(40.464642332174776 -74.93907597748066) bank81411 +81412 POINT(40.45821190791852 -73.69299315330045) bank81412 +81413 POINT(41.514484144091824 -74.78061284385919) bank81413 +81414 POINT(41.49224789789343 -74.31212974129528) bank81414 +81415 POINT(41.44053968387231 -73.84369489362207) bank81415 +81416 POINT(40.58369382114718 -73.47792516051113) bank81416 +81417 POINT(41.424146860586355 -73.10510587037723) bank81417 +81418 POINT(40.042864560507326 -73.14633719123697) bank81418 +81419 POINT(40.102043558802414 -74.76923305119402) bank81419 +81420 POINT(41.068490769830724 -74.44132147958607) bank81420 +81421 POINT(40.578991197126534 -73.22179213006962) bank81421 +81422 POINT(40.46605532048534 -73.99227699879818) bank81422 +81423 POINT(41.363565136903475 -73.10449843629596) bank81423 +81424 POINT(40.03936138073176 -73.21751952933228) bank81424 +81425 POINT(40.23444161959042 -74.26135303655542) bank81425 +81426 POINT(40.44787614696344 -74.4092522450836) bank81426 +81427 POINT(40.95145623419032 -74.41687907896974) bank81427 +81428 POINT(41.17508091047745 -74.44769502357839) bank81428 +81429 POINT(41.167944228209734 -74.03682163989191) bank81429 +81430 POINT(39.98032786164326 -73.38229327915482) bank81430 +81431 POINT(41.67856261134397 -73.39803737601505) bank81431 +81432 POINT(41.43190381223744 -74.42041029724801) bank81432 +81433 POINT(41.4016325950812 -74.59284730227823) bank81433 +81434 POINT(41.50727308247085 -74.67709661153422) bank81434 +81435 POINT(41.11072348653179 -73.32022853961914) bank81435 +81436 POINT(40.80708514674632 -74.32487996765661) bank81436 +81437 POINT(41.490342343391845 -73.19295208916381) bank81437 +81438 POINT(40.375904590624444 -74.57776483643525) bank81438 +81439 POINT(39.95043004130447 -74.82556213042727) bank81439 +81440 POINT(39.83271998862404 -74.97887264026716) bank81440 +81441 POINT(39.861420386071565 -73.91181666787563) bank81441 +81442 POINT(41.36744560329736 -74.7212187503234) bank81442 +81443 POINT(40.02891587399036 -74.53297755101286) bank81443 +81444 POINT(41.32395039216042 -74.74110554777747) bank81444 +81445 POINT(41.68664218606133 -74.22566960161014) bank81445 +81446 POINT(40.75073604347827 -74.58466577016333) bank81446 +81447 POINT(40.09543809219311 -74.64629314589423) bank81447 +81448 POINT(39.8777584196284 -74.3767765288471) bank81448 +81449 POINT(39.75623835956537 -73.55638684117805) bank81449 +81450 POINT(40.670210025935596 -73.76465112737334) bank81450 +81451 POINT(41.52207759386302 -74.00285085136092) bank81451 +81452 POINT(41.648936922886996 -73.11951745215941) bank81452 +81453 POINT(41.510789017480086 -74.43029733807144) bank81453 +81454 POINT(39.886613589357424 -73.27341199627905) bank81454 +81455 POINT(40.54276665339263 -73.2899235987115) bank81455 +81456 POINT(41.32475017886306 -73.66611566913318) bank81456 +81457 POINT(41.644986715725494 -74.15428244485324) bank81457 +81458 POINT(40.39324521406011 -73.76944178556465) bank81458 +81459 POINT(41.54882252825653 -73.53051995108758) bank81459 +81460 POINT(40.49232335788019 -73.30169879839532) bank81460 +81461 POINT(40.094935613594394 -74.21291487559428) bank81461 +81462 POINT(40.97104645218903 -74.20658512412832) bank81462 +81463 POINT(41.250289882856414 -74.79850886194684) bank81463 +81464 POINT(40.53250373742651 -74.2262815177601) bank81464 +81465 POINT(41.20223194263235 -73.37798662912007) bank81465 +81466 POINT(40.056488519048564 -74.9960221698576) bank81466 +81467 POINT(40.10289178008625 -74.18327366530207) bank81467 +81468 POINT(40.0459518653221 -73.82379080821904) bank81468 +81469 POINT(41.300875900251604 -74.0361686559568) bank81469 +81470 POINT(40.0380546744733 -73.28548478759447) bank81470 +81471 POINT(40.93119437701987 -74.11924562846424) bank81471 +81472 POINT(41.208630611588454 -73.05784147824987) bank81472 +81473 POINT(41.1334380543489 -74.44720384895918) bank81473 +81474 POINT(40.38780358027441 -74.69524316866496) bank81474 +81475 POINT(41.15812030706774 -73.82014081362742) bank81475 +81476 POINT(41.19733203431493 -74.48759927231286) bank81476 +81477 POINT(41.564206870606114 -74.24928433471678) bank81477 +81478 POINT(39.81137240191692 -74.59530451241388) bank81478 +81479 POINT(40.689545033926194 -74.47246253678256) bank81479 +81480 POINT(39.91555882291679 -73.2464189426391) bank81480 +81481 POINT(41.081607418873816 -74.50522507639518) bank81481 +81482 POINT(40.21464404324722 -73.19359141681447) bank81482 +81483 POINT(40.98877433581307 -73.30740792579932) bank81483 +81484 POINT(40.599974095915215 -74.56934475669316) bank81484 +81485 POINT(40.75183726998892 -73.95019749150269) bank81485 +81486 POINT(41.137410948726114 -73.51281452161679) bank81486 +81487 POINT(40.06950928982875 -73.28542954977615) bank81487 +81488 POINT(40.07025267393725 -73.71859156805787) bank81488 +81489 POINT(40.08683683301839 -74.74764662764417) bank81489 +81490 POINT(41.34163596371965 -73.51682873913501) bank81490 +81491 POINT(39.964968918037535 -73.79697123633659) bank81491 +81492 POINT(40.285942138346364 -73.96699512556413) bank81492 +81493 POINT(39.91073005777398 -73.66517262964682) bank81493 +81494 POINT(39.815626533147515 -74.59022359959994) bank81494 +81495 POINT(39.81180485892004 -74.17168879547728) bank81495 +81496 POINT(41.48920247473784 -74.40421438571781) bank81496 +81497 POINT(40.25836476804155 -74.12069788855258) bank81497 +81498 POINT(40.289218020264876 -74.6025132308374) bank81498 +81499 POINT(41.2577445816911 -73.0323349050467) bank81499 +81500 POINT(40.91872333631039 -74.64602395100032) bank81500 +81501 POINT(41.65696006959862 -74.59178683566242) bank81501 +81502 POINT(40.790246921548864 -73.9677508446395) bank81502 +81503 POINT(40.523318271738624 -73.90686104172858) bank81503 +81504 POINT(41.2431666638734 -73.99714448289748) bank81504 +81505 POINT(40.020886229570515 -73.37182389041669) bank81505 +81506 POINT(41.206342021793056 -73.22534264955064) bank81506 +81507 POINT(39.75619513222344 -74.54433891802104) bank81507 +81508 POINT(40.344308688872005 -74.92264206404658) bank81508 +81509 POINT(41.30012529209442 -73.81121929208958) bank81509 +81510 POINT(40.46425976757632 -73.58255940051279) bank81510 +81511 POINT(40.09243347039451 -74.74418445271183) bank81511 +81512 POINT(39.99526854334422 -74.8526999389869) bank81512 +81513 POINT(41.538190189309056 -74.88357614654862) bank81513 +81514 POINT(40.86082336864554 -73.28487071519189) bank81514 +81515 POINT(40.27837944805926 -74.07365727729518) bank81515 +81516 POINT(41.58150151725196 -74.80310704247304) bank81516 +81517 POINT(41.5785514521862 -74.87970412240587) bank81517 +81518 POINT(40.3707562845853 -74.76522413989417) bank81518 +81519 POINT(39.876579644463675 -73.7569017734214) bank81519 +81520 POINT(39.91623775882307 -73.11488635238362) bank81520 +81521 POINT(41.033543521019396 -73.28796979584291) bank81521 +81522 POINT(41.659084772127784 -73.01115645798201) bank81522 +81523 POINT(41.71065508668915 -74.66497239433637) bank81523 +81524 POINT(40.9515240100879 -74.37012911049541) bank81524 +81525 POINT(40.938331375128286 -74.86894600447395) bank81525 +81526 POINT(41.478494592474505 -74.4650577078882) bank81526 +81527 POINT(41.28808958552023 -73.49239527806112) bank81527 +81528 POINT(40.2810393195565 -74.53165237528451) bank81528 +81529 POINT(41.249309009320264 -74.77028176528098) bank81529 +81530 POINT(41.22957385982127 -74.31530069117548) bank81530 +81531 POINT(39.88237189957645 -73.27093173934874) bank81531 +81532 POINT(40.357546612218755 -74.57763075422605) bank81532 +81533 POINT(40.83044642482247 -74.8053110799867) bank81533 +81534 POINT(40.377869429119954 -73.43511122133067) bank81534 +81535 POINT(40.13287005261682 -73.29686008514679) bank81535 +81536 POINT(40.57593170567151 -74.19802503057456) bank81536 +81537 POINT(41.625978882506864 -73.646098686789) bank81537 +81538 POINT(39.73910927674782 -74.49937460809001) bank81538 +81539 POINT(41.58892665728316 -73.48399131256136) bank81539 +81540 POINT(40.47320657894826 -73.25178300109864) bank81540 +81541 POINT(39.936041178467164 -74.12552268274816) bank81541 +81542 POINT(40.24700401097566 -73.87326475527023) bank81542 +81543 POINT(41.44587545872369 -74.93837407735982) bank81543 +81544 POINT(39.80739631521307 -73.26517192472895) bank81544 +81545 POINT(41.46429164136871 -74.66829286392748) bank81545 +81546 POINT(39.75471288255235 -73.93248797252492) bank81546 +81547 POINT(41.251038496137255 -73.5851494684449) bank81547 +81548 POINT(40.71272400231848 -74.5583083335583) bank81548 +81549 POINT(39.90612343213163 -74.81638320064901) bank81549 +81550 POINT(40.854547371999566 -73.47087765207807) bank81550 +81551 POINT(41.68767733031831 -74.99939582530634) bank81551 +81552 POINT(39.981629402379205 -73.34960910534501) bank81552 +81553 POINT(40.91251210645426 -73.50029222465344) bank81553 +81554 POINT(40.25136801622696 -74.42107187490863) bank81554 +81555 POINT(40.02530359252516 -74.25407870023199) bank81555 +81556 POINT(39.80927108527278 -73.38818247086199) bank81556 +81557 POINT(41.690900099692186 -73.55357218873397) bank81557 +81558 POINT(41.43452046529383 -74.64549454959581) bank81558 +81559 POINT(40.29259760470941 -73.53036415251174) bank81559 +81560 POINT(39.91186563918086 -73.04172576913851) bank81560 +81561 POINT(40.635770795715736 -74.74081413349056) bank81561 +81562 POINT(41.20242653105312 -74.23883881340957) bank81562 +81563 POINT(39.83614858721044 -73.1978884097413) bank81563 +81564 POINT(40.16106466869342 -74.32747275950965) bank81564 +81565 POINT(40.10731819585118 -73.29297361347301) bank81565 +81566 POINT(40.656377468477594 -74.89021254705418) bank81566 +81567 POINT(40.052377458364525 -74.30614949218308) bank81567 +81568 POINT(40.98656539570126 -73.06040670125479) bank81568 +81569 POINT(40.386482453446796 -74.30725412540133) bank81569 +81570 POINT(41.66784905344696 -75.0045017735234) bank81570 +81571 POINT(39.75486086409176 -73.32787492286855) bank81571 +81572 POINT(39.983032546646875 -73.4652266224066) bank81572 +81573 POINT(41.51123054799522 -74.35143896232158) bank81573 +81574 POINT(41.70202968951896 -73.12084457438834) bank81574 +81575 POINT(41.530988999334724 -74.17613394434035) bank81575 +81576 POINT(41.59936232747075 -73.57321336641886) bank81576 +81577 POINT(41.431518427132474 -74.8455109057672) bank81577 +81578 POINT(40.099062735063285 -73.15163669691039) bank81578 +81579 POINT(40.71337903657474 -74.23129844253262) bank81579 +81580 POINT(41.63093030545489 -74.41334423795433) bank81580 +81581 POINT(41.27769900458049 -74.35866365610315) bank81581 +81582 POINT(40.47054435209301 -73.70855007274629) bank81582 +81583 POINT(39.97135758462677 -74.59849554849326) bank81583 +81584 POINT(40.9691217396615 -73.4503651850136) bank81584 +81585 POINT(40.51093586833156 -73.01676486890052) bank81585 +81586 POINT(41.22365748113401 -74.5221250854053) bank81586 +81587 POINT(40.48004343758166 -74.85238735783176) bank81587 +81588 POINT(40.80175099987626 -73.01531217407853) bank81588 +81589 POINT(40.2046609136888 -73.16675895519803) bank81589 +81590 POINT(40.68129699590883 -74.2324350831457) bank81590 +81591 POINT(41.24393862986392 -73.12278244942) bank81591 +81592 POINT(40.18509162323811 -74.15934814466628) bank81592 +81593 POINT(40.65288467759826 -73.76317539546923) bank81593 +81594 POINT(41.52011485523516 -74.91344353519798) bank81594 +81595 POINT(40.06537666396321 -73.6733639731218) bank81595 +81596 POINT(39.89503440202398 -74.23482595381252) bank81596 +81597 POINT(40.97524910962459 -73.32128274338032) bank81597 +81598 POINT(40.08855784797832 -73.91486294679517) bank81598 +81599 POINT(41.60644689144063 -74.37970302023335) bank81599 +81600 POINT(40.56363678812325 -73.62469955108287) bank81600 +81601 POINT(40.522443320259995 -73.28037229008405) bank81601 +81602 POINT(41.38683512960255 -73.76490326131535) bank81602 +81603 POINT(39.73661462259509 -73.19054858659145) bank81603 +81604 POINT(40.41644660596605 -74.23276400687901) bank81604 +81605 POINT(41.31651568898369 -73.67804977971974) bank81605 +81606 POINT(39.83903457419319 -73.22602624347175) bank81606 +81607 POINT(39.88920656514951 -74.19405341020963) bank81607 +81608 POINT(41.68228167555694 -74.24856937271826) bank81608 +81609 POINT(41.56830957934414 -74.35517083222524) bank81609 +81610 POINT(40.43680004353807 -73.69784265558533) bank81610 +81611 POINT(41.4869865092084 -74.88287867485472) bank81611 +81612 POINT(41.00065683207873 -74.28645346078312) bank81612 +81613 POINT(41.37566694598414 -73.2907185876352) bank81613 +81614 POINT(40.909207714803564 -74.63385440268969) bank81614 +81615 POINT(40.71408462905023 -73.69075011102446) bank81615 +81616 POINT(41.21383891415368 -74.04042624178848) bank81616 +81617 POINT(41.697535894366844 -74.32759646055283) bank81617 +81618 POINT(41.22731826856185 -74.94609772325036) bank81618 +81619 POINT(40.01067291168742 -74.47167527219929) bank81619 +81620 POINT(39.93769937612753 -74.5993719944432) bank81620 +81621 POINT(40.27862932073349 -74.17317494298034) bank81621 +81622 POINT(41.689787382485484 -73.19700559310368) bank81622 +81623 POINT(41.24273724582836 -73.56378840440786) bank81623 +81624 POINT(39.995528334403346 -74.88143363466996) bank81624 +81625 POINT(40.43226526887173 -74.06642812149788) bank81625 +81626 POINT(41.342274814878735 -74.82945191052555) bank81626 +81627 POINT(40.71812358682979 -73.91310642855258) bank81627 +81628 POINT(40.29319367434329 -74.25354515517235) bank81628 +81629 POINT(40.45519833531177 -73.70455710637853) bank81629 +81630 POINT(40.5577393723393 -73.06367474877403) bank81630 +81631 POINT(40.123295651296566 -73.32940986790793) bank81631 +81632 POINT(41.31903441491556 -74.82349150251792) bank81632 +81633 POINT(41.1699412770647 -73.87115622043454) bank81633 +81634 POINT(41.11893044891033 -73.57140829857022) bank81634 +81635 POINT(40.32136646347226 -73.58545631281633) bank81635 +81636 POINT(41.17309437357907 -73.94469791235139) bank81636 +81637 POINT(41.00210920965686 -73.26232763543547) bank81637 +81638 POINT(39.902646979587516 -74.82392662570008) bank81638 +81639 POINT(40.27845458139126 -73.92504388001166) bank81639 +81640 POINT(40.63149455050801 -73.75999593428637) bank81640 +81641 POINT(40.564226211745655 -74.36620129539729) bank81641 +81642 POINT(40.62392737456547 -73.40239688242413) bank81642 +81643 POINT(40.67171337948399 -73.12434228743851) bank81643 +81644 POINT(41.5998245925239 -74.0714452869974) bank81644 +81645 POINT(40.59206751447307 -74.02848946119208) bank81645 +81646 POINT(39.97648998402382 -73.39023121753313) bank81646 +81647 POINT(40.78917463938119 -74.88896130477917) bank81647 +81648 POINT(39.725450245382255 -73.87944729472112) bank81648 +81649 POINT(40.22152121382676 -73.40435901028691) bank81649 +81650 POINT(40.82007316900873 -73.61378372245106) bank81650 +81651 POINT(39.92520856456736 -73.1681088037135) bank81651 +81652 POINT(39.7242713649799 -74.53996988793713) bank81652 +81653 POINT(41.471064728717714 -74.70798997118769) bank81653 +81654 POINT(40.49109514796634 -74.75274291022073) bank81654 +81655 POINT(41.57066738286667 -74.82226744982351) bank81655 +81656 POINT(41.34148779931649 -73.1347119413334) bank81656 +81657 POINT(39.95575164499838 -74.43900013407948) bank81657 +81658 POINT(40.629979585463786 -74.35686873728827) bank81658 +81659 POINT(40.76034065773273 -74.29275286656019) bank81659 +81660 POINT(40.46869413633402 -74.97274928286481) bank81660 +81661 POINT(41.59434776180367 -74.8038903139665) bank81661 +81662 POINT(41.394323041293816 -74.90514084641056) bank81662 +81663 POINT(41.570906755922465 -74.2626897400641) bank81663 +81664 POINT(40.833218242401905 -74.92357579306383) bank81664 +81665 POINT(39.88331645257623 -74.55815618235566) bank81665 +81666 POINT(40.8405907450813 -74.47239865112937) bank81666 +81667 POINT(40.436623602521315 -73.12289862532587) bank81667 +81668 POINT(40.92003755376479 -73.86965582937023) bank81668 +81669 POINT(40.23899619550504 -74.1700855142996) bank81669 +81670 POINT(40.20972157347043 -73.7286141506438) bank81670 +81671 POINT(40.01399889528061 -74.5803438109085) bank81671 +81672 POINT(39.78616498351214 -74.71315940308774) bank81672 +81673 POINT(40.980838935927466 -73.04137284542244) bank81673 +81674 POINT(40.78881744707738 -73.45106217043005) bank81674 +81675 POINT(40.00363523305574 -73.48847705023191) bank81675 +81676 POINT(41.62208009922653 -74.7715002101453) bank81676 +81677 POINT(40.2422321925105 -74.73326311329035) bank81677 +81678 POINT(40.688527636440305 -73.07833094202707) bank81678 +81679 POINT(40.06042350312017 -73.28173772977671) bank81679 +81680 POINT(40.933170553992085 -73.05101094536427) bank81680 +81681 POINT(40.46456533184218 -74.19235399074361) bank81681 +81682 POINT(40.30289207310793 -73.35859897025045) bank81682 +81683 POINT(40.638906034895804 -74.51797389537415) bank81683 +81684 POINT(41.25653181827692 -73.75310043255499) bank81684 +81685 POINT(40.510537093556984 -73.31806906939941) bank81685 +81686 POINT(40.14488538392683 -73.29049619274578) bank81686 +81687 POINT(41.4203945021112 -73.5126952125286) bank81687 +81688 POINT(41.68099213335994 -74.26400225225925) bank81688 +81689 POINT(41.506408334423746 -74.24177051552287) bank81689 +81690 POINT(40.5873986608392 -73.90163984217529) bank81690 +81691 POINT(40.96770564429672 -73.89561154847951) bank81691 +81692 POINT(40.120228682429044 -73.01337842971974) bank81692 +81693 POINT(40.11238873380666 -74.55988844260507) bank81693 +81694 POINT(41.45778758595801 -74.33353748546085) bank81694 +81695 POINT(40.42760325966186 -74.33523071133453) bank81695 +81696 POINT(40.17150778191533 -73.65128274164175) bank81696 +81697 POINT(41.56855146767699 -73.63141265164586) bank81697 +81698 POINT(40.775886195308445 -74.20812308876074) bank81698 +81699 POINT(39.91582727433804 -73.71539970011986) bank81699 +81700 POINT(41.025036035261756 -73.12701962882306) bank81700 +81701 POINT(40.05694181898871 -74.20696246160234) bank81701 +81702 POINT(40.197407790592706 -74.98039836121835) bank81702 +81703 POINT(40.982615918434036 -74.7220758350524) bank81703 +81704 POINT(39.99724033976151 -74.82702123517214) bank81704 +81705 POINT(41.439330905878975 -74.87433702973125) bank81705 +81706 POINT(40.29001458017435 -74.3687217951115) bank81706 +81707 POINT(40.26422481265499 -74.3625864563615) bank81707 +81708 POINT(39.915696260968474 -73.01081288015706) bank81708 +81709 POINT(41.38360448416067 -73.12275451874069) bank81709 +81710 POINT(40.018787785815576 -74.41362714349917) bank81710 +81711 POINT(41.439530392435756 -73.60975065991379) bank81711 +81712 POINT(40.54407179998039 -73.79426177350592) bank81712 +81713 POINT(41.33428907250194 -73.15268410833522) bank81713 +81714 POINT(41.629741549079085 -74.48186070803742) bank81714 +81715 POINT(39.82166371263402 -74.74379709631408) bank81715 +81716 POINT(40.29892037627104 -73.13998350088174) bank81716 +81717 POINT(41.39460760323678 -74.42712352987252) bank81717 +81718 POINT(40.27908129843047 -74.79865986723675) bank81718 +81719 POINT(41.17335520280787 -74.3715646226718) bank81719 +81720 POINT(40.39000338221148 -73.52304454649982) bank81720 +81721 POINT(41.28844860800152 -74.10202600694993) bank81721 +81722 POINT(40.12562898821849 -73.50604454182093) bank81722 +81723 POINT(41.442988073823436 -73.1304731065233) bank81723 +81724 POINT(41.19094422758223 -73.00941315502713) bank81724 +81725 POINT(41.55198243235904 -73.0963672373745) bank81725 +81726 POINT(41.158700953023185 -73.38763038589586) bank81726 +81727 POINT(40.82946312619857 -74.48328398986786) bank81727 +81728 POINT(40.774167037944856 -73.73644983281577) bank81728 +81729 POINT(39.845564964519944 -73.16997195161332) bank81729 +81730 POINT(41.43493920949454 -73.29705435225459) bank81730 +81731 POINT(41.621517914992396 -74.1674962762949) bank81731 +81732 POINT(40.334724729253125 -73.74954308633669) bank81732 +81733 POINT(41.090326617254966 -74.74995817424994) bank81733 +81734 POINT(40.29372620110953 -74.01551792337368) bank81734 +81735 POINT(40.739912520325575 -73.12191900497083) bank81735 +81736 POINT(40.924989907341455 -74.85470579715071) bank81736 +81737 POINT(39.99061487731499 -74.72324587073575) bank81737 +81738 POINT(39.819858934242895 -73.62981650442964) bank81738 +81739 POINT(40.12470077421441 -73.02868416585703) bank81739 +81740 POINT(40.98773742831173 -73.06036305420857) bank81740 +81741 POINT(40.58083030268393 -74.6918287029003) bank81741 +81742 POINT(40.100253058727795 -73.28455416924287) bank81742 +81743 POINT(41.620806598026384 -73.95667862350457) bank81743 +81744 POINT(39.86362673396292 -73.55228713612587) bank81744 +81745 POINT(40.895608704526204 -74.97792432489706) bank81745 +81746 POINT(39.924304081224435 -73.27685633370196) bank81746 +81747 POINT(40.012778891472514 -73.30446343258093) bank81747 +81748 POINT(39.99021005099397 -73.79509903885445) bank81748 +81749 POINT(40.93510087720192 -74.86056865855358) bank81749 +81750 POINT(41.43037198388899 -73.43008243185234) bank81750 +81751 POINT(40.521545777982226 -73.50221004962829) bank81751 +81752 POINT(40.81423006556043 -73.38285113620502) bank81752 +81753 POINT(40.649634669994015 -74.0044701504926) bank81753 +81754 POINT(41.651481799691226 -73.68079248455444) bank81754 +81755 POINT(40.277721674383486 -74.32053495261133) bank81755 +81756 POINT(39.950742609990144 -74.91781131476965) bank81756 +81757 POINT(40.58243485920061 -74.41067234008298) bank81757 +81758 POINT(40.573058233981236 -73.37441175667273) bank81758 +81759 POINT(40.6603846202171 -73.37204556937333) bank81759 +81760 POINT(39.74838260199235 -73.4769661750277) bank81760 +81761 POINT(40.386565852853316 -73.83115262794941) bank81761 +81762 POINT(40.85435699555895 -73.60339921465024) bank81762 +81763 POINT(41.367977759546086 -74.16713266762028) bank81763 +81764 POINT(41.395713314290994 -74.49734801406115) bank81764 +81765 POINT(39.764256430306524 -74.41787140063397) bank81765 +81766 POINT(40.01340152971074 -73.44787116425391) bank81766 +81767 POINT(40.8149285182497 -73.81247911754701) bank81767 +81768 POINT(41.110909600023035 -73.76714050965295) bank81768 +81769 POINT(41.62448769289982 -74.33637433346098) bank81769 +81770 POINT(40.111865948055595 -74.60399472994374) bank81770 +81771 POINT(40.25566527290273 -73.2467220377253) bank81771 +81772 POINT(41.10227060125035 -73.4922089313942) bank81772 +81773 POINT(41.11454543014645 -74.15101975220455) bank81773 +81774 POINT(41.10328472036588 -74.61874829835799) bank81774 +81775 POINT(40.92826261454661 -73.71474344679643) bank81775 +81776 POINT(40.984586560340524 -73.1788056060763) bank81776 +81777 POINT(41.32513802141565 -73.97759147345097) bank81777 +81778 POINT(40.8094854451782 -73.49086450602344) bank81778 +81779 POINT(41.66048703358226 -74.35513767006019) bank81779 +81780 POINT(39.81130047202665 -73.1232191957066) bank81780 +81781 POINT(41.068411264109976 -74.65462609382195) bank81781 +81782 POINT(40.93551772368592 -74.25417210035052) bank81782 +81783 POINT(40.07618754492675 -73.67479091979907) bank81783 +81784 POINT(40.86865925524502 -73.06321024566785) bank81784 +81785 POINT(40.0239055575314 -74.304600643707) bank81785 +81786 POINT(40.06409324693315 -74.56812648776149) bank81786 +81787 POINT(41.437085709116836 -74.41598057386966) bank81787 +81788 POINT(40.254851618138964 -73.26792342370751) bank81788 +81789 POINT(40.80133638996916 -74.05237168404557) bank81789 +81790 POINT(41.58273685461071 -74.07367779361937) bank81790 +81791 POINT(41.6104779500953 -74.97011191559865) bank81791 +81792 POINT(41.209769610251605 -73.4217537760125) bank81792 +81793 POINT(40.98428852872395 -73.53440416711562) bank81793 +81794 POINT(41.29584604202503 -74.28143354022798) bank81794 +81795 POINT(40.76837979566543 -74.67705798421824) bank81795 +81796 POINT(40.02056632346783 -74.55595952103712) bank81796 +81797 POINT(41.2396087235982 -73.55805790225934) bank81797 +81798 POINT(41.17751338084887 -73.4541675606191) bank81798 +81799 POINT(40.02127376471624 -74.92458534171323) bank81799 +81800 POINT(41.155215841077826 -73.09315406303162) bank81800 +81801 POINT(40.296596742848706 -73.62045495266426) bank81801 +81802 POINT(40.73704323697195 -73.74679471661254) bank81802 +81803 POINT(40.81189497510986 -74.59866824002285) bank81803 +81804 POINT(41.54532665568604 -74.76007507278598) bank81804 +81805 POINT(40.61855695896924 -73.93210460669479) bank81805 +81806 POINT(40.549409295633644 -73.70414994894084) bank81806 +81807 POINT(41.42878485298831 -73.52606420829345) bank81807 +81808 POINT(41.10701749726585 -73.36036215057023) bank81808 +81809 POINT(40.86451315943894 -74.77381493705187) bank81809 +81810 POINT(40.48268247759835 -73.54183581591604) bank81810 +81811 POINT(41.66022209583303 -73.31679036968019) bank81811 +81812 POINT(40.71956226224058 -74.63035841822257) bank81812 +81813 POINT(39.82989250916856 -74.62128423985025) bank81813 +81814 POINT(40.539485614811454 -74.25940824600511) bank81814 +81815 POINT(41.5816454005821 -74.96208616086076) bank81815 +81816 POINT(40.540517616841335 -73.19226842595668) bank81816 +81817 POINT(40.53158901640285 -74.72220003084965) bank81817 +81818 POINT(41.546878278369775 -73.51285820251866) bank81818 +81819 POINT(41.01599251848081 -73.87355771503088) bank81819 +81820 POINT(40.671863997011435 -73.52140588159921) bank81820 +81821 POINT(39.784172931549584 -74.96616912120295) bank81821 +81822 POINT(41.15774946241363 -74.04607098515552) bank81822 +81823 POINT(41.33209259735665 -73.07436541959598) bank81823 +81824 POINT(41.326828592212216 -73.51285607593188) bank81824 +81825 POINT(40.08005653001909 -73.47791763062389) bank81825 +81826 POINT(39.86741658825242 -73.99964144061941) bank81826 +81827 POINT(40.714269173059925 -74.01908328814837) bank81827 +81828 POINT(41.55205868601561 -73.27370360964797) bank81828 +81829 POINT(40.56923994394518 -73.15910180221331) bank81829 +81830 POINT(40.93013225974525 -73.73315388738311) bank81830 +81831 POINT(41.68502572420539 -74.00928369446302) bank81831 +81832 POINT(41.66188643472069 -74.903361341231) bank81832 +81833 POINT(40.848535383201764 -74.76743799541381) bank81833 +81834 POINT(40.296037564853926 -74.69701552492104) bank81834 +81835 POINT(41.01202393296765 -74.09505094708894) bank81835 +81836 POINT(40.999216528021854 -73.33661876869267) bank81836 +81837 POINT(41.29320605248507 -73.3956072098619) bank81837 +81838 POINT(40.00603560185444 -73.51438096558442) bank81838 +81839 POINT(41.124097193011536 -74.63102382247206) bank81839 +81840 POINT(40.6898849056533 -74.77941555945257) bank81840 +81841 POINT(40.5000326369751 -74.24963949833281) bank81841 +81842 POINT(41.58647851711552 -73.78676173156275) bank81842 +81843 POINT(39.98410081710424 -73.9562923437066) bank81843 +81844 POINT(41.2689830872415 -73.85896361261554) bank81844 +81845 POINT(41.56413112352428 -73.07426152376642) bank81845 +81846 POINT(40.24583544299523 -74.62390892045013) bank81846 +81847 POINT(41.35093826569371 -73.04295702530999) bank81847 +81848 POINT(41.366215644930826 -73.13074242473894) bank81848 +81849 POINT(41.07734195081916 -74.23968839778246) bank81849 +81850 POINT(41.46228881925425 -73.74564191117034) bank81850 +81851 POINT(40.52501353189107 -74.61441160900742) bank81851 +81852 POINT(40.627684707656904 -74.19766646056526) bank81852 +81853 POINT(40.411188567252836 -73.24568221725052) bank81853 +81854 POINT(40.75182405112698 -73.55100580168835) bank81854 +81855 POINT(41.496059601634855 -74.24374931134533) bank81855 +81856 POINT(39.78494200334158 -73.57041646858663) bank81856 +81857 POINT(40.750347232589334 -74.47133588997963) bank81857 +81858 POINT(39.991556982030666 -74.06476119641032) bank81858 +81859 POINT(40.751025559314286 -73.41220245052307) bank81859 +81860 POINT(41.63629191181494 -73.44926852995965) bank81860 +81861 POINT(41.458371789292165 -74.37901545068092) bank81861 +81862 POINT(41.52351222348696 -73.5605203735769) bank81862 +81863 POINT(40.09432884499123 -74.89834577365946) bank81863 +81864 POINT(41.425456428794845 -73.79747276734643) bank81864 +81865 POINT(40.47938809765385 -73.70291913663206) bank81865 +81866 POINT(40.103968204480715 -74.69581270654122) bank81866 +81867 POINT(40.966038638850996 -74.76451856381333) bank81867 +81868 POINT(40.1771537055194 -73.49092641659357) bank81868 +81869 POINT(41.1854395579876 -73.84637403456733) bank81869 +81870 POINT(41.63298815182144 -74.08507700546629) bank81870 +81871 POINT(40.03999479886261 -74.91726097823499) bank81871 +81872 POINT(41.40863771315763 -74.90946300841759) bank81872 +81873 POINT(41.535296118393106 -74.43952754436029) bank81873 +81874 POINT(39.79435333050694 -73.86369814974087) bank81874 +81875 POINT(40.85724575678218 -73.43107320707102) bank81875 +81876 POINT(40.21504820749752 -73.1508148102145) bank81876 +81877 POINT(40.79599070455896 -74.11458848638308) bank81877 +81878 POINT(40.40950869094098 -73.58504155596985) bank81878 +81879 POINT(39.73203902179807 -73.30228393300013) bank81879 +81880 POINT(39.96589714276764 -73.81166897824758) bank81880 +81881 POINT(40.246750895080936 -73.05915202239764) bank81881 +81882 POINT(41.010725939134346 -73.85470310078324) bank81882 +81883 POINT(41.679697241929304 -74.06283186655435) bank81883 +81884 POINT(40.56659545920249 -74.0074373055413) bank81884 +81885 POINT(39.88826121507484 -73.52076566235873) bank81885 +81886 POINT(40.01126849222171 -74.57440417523527) bank81886 +81887 POINT(40.64088373257159 -73.91859650055578) bank81887 +81888 POINT(41.31015512583543 -73.30027654735176) bank81888 +81889 POINT(41.04536635800615 -73.19206687563081) bank81889 +81890 POINT(40.548448811943445 -73.43037805058451) bank81890 +81891 POINT(41.555631149948155 -73.21788486345407) bank81891 +81892 POINT(40.11548920055505 -74.93622651647017) bank81892 +81893 POINT(40.285952184196276 -74.01462556030886) bank81893 +81894 POINT(41.17732364764775 -74.27690137951845) bank81894 +81895 POINT(40.93006012221461 -73.26877854970064) bank81895 +81896 POINT(40.47996338211167 -73.48938971374434) bank81896 +81897 POINT(39.91185579614494 -73.89231971250844) bank81897 +81898 POINT(39.73945227374927 -74.86907909104322) bank81898 +81899 POINT(40.36981498770232 -73.8485624672134) bank81899 +81900 POINT(41.04282895868796 -73.33849096074688) bank81900 +81901 POINT(40.34583680452768 -73.50771819932467) bank81901 +81902 POINT(40.684891174193375 -73.33971283393093) bank81902 +81903 POINT(40.74754646793004 -74.86750660973824) bank81903 +81904 POINT(41.24731436775069 -73.16797361531286) bank81904 +81905 POINT(40.14009389890447 -73.86273813724753) bank81905 +81906 POINT(41.03833800370709 -73.35574194490009) bank81906 +81907 POINT(41.60504919084185 -73.6737233865542) bank81907 +81908 POINT(41.27309688438729 -73.00834625567721) bank81908 +81909 POINT(40.07697386630771 -73.90692786280844) bank81909 +81910 POINT(41.11030741636367 -73.6660348454345) bank81910 +81911 POINT(40.880788269791154 -73.3214707849329) bank81911 +81912 POINT(40.34343288428877 -74.35830785823133) bank81912 +81913 POINT(40.13172612290225 -74.42968804993457) bank81913 +81914 POINT(41.34628230620005 -74.04626771489654) bank81914 +81915 POINT(40.2799845052931 -74.19850877496802) bank81915 +81916 POINT(40.63733122583885 -74.65708726203815) bank81916 +81917 POINT(40.5212310553584 -74.59151700097603) bank81917 +81918 POINT(39.87779161320098 -74.54445950881178) bank81918 +81919 POINT(41.26143939977549 -74.61767298579049) bank81919 +81920 POINT(41.56540166007641 -74.1509803869958) bank81920 +81921 POINT(40.47444027929507 -74.65213319001631) bank81921 +81922 POINT(40.13740765993731 -73.46410146157352) bank81922 +81923 POINT(41.394510358353656 -73.541929053488) bank81923 +81924 POINT(39.946420425675804 -74.21149954763091) bank81924 +81925 POINT(41.23462033719037 -73.0221756989397) bank81925 +81926 POINT(39.8469333211587 -73.13894731185624) bank81926 +81927 POINT(40.57198023215557 -74.57518793431592) bank81927 +81928 POINT(40.06858117666397 -74.35111360540662) bank81928 +81929 POINT(41.36666661689398 -74.11032418650264) bank81929 +81930 POINT(40.99371117976471 -74.20687631886344) bank81930 +81931 POINT(39.872733670752694 -74.44842600663371) bank81931 +81932 POINT(40.383283941499556 -73.0617460899497) bank81932 +81933 POINT(41.261638822335115 -74.54340837100361) bank81933 +81934 POINT(40.74147592017863 -74.97619693654772) bank81934 +81935 POINT(40.17818212419742 -74.18636186928346) bank81935 +81936 POINT(39.80613362637499 -74.89270892345199) bank81936 +81937 POINT(41.088806337375416 -73.68294824794455) bank81937 +81938 POINT(39.902762306941 -74.61659502788973) bank81938 +81939 POINT(39.90822500845922 -73.64491391450301) bank81939 +81940 POINT(40.96377725432748 -73.65341939779772) bank81940 +81941 POINT(40.638512337541535 -74.2149361015462) bank81941 +81942 POINT(40.92105720643153 -73.4190800645592) bank81942 +81943 POINT(41.35417379929976 -74.47719254846233) bank81943 +81944 POINT(41.630969822700365 -74.82865617154258) bank81944 +81945 POINT(40.181469599686395 -73.94109023154195) bank81945 +81946 POINT(39.86034279182398 -74.44649586224965) bank81946 +81947 POINT(40.656026126212986 -73.89025324364147) bank81947 +81948 POINT(41.2084467469736 -73.56360213005661) bank81948 +81949 POINT(41.083020889423146 -74.4450095071809) bank81949 +81950 POINT(39.71888709544909 -73.81437611280457) bank81950 +81951 POINT(40.831312477301054 -73.17350695753319) bank81951 +81952 POINT(41.27644528920282 -73.97883084491204) bank81952 +81953 POINT(41.210464918263206 -74.85106454287579) bank81953 +81954 POINT(40.756461472063016 -73.59787215315099) bank81954 +81955 POINT(40.07769837901902 -73.24571818574587) bank81955 +81956 POINT(40.92537006064856 -73.61338534764629) bank81956 +81957 POINT(41.267625027643405 -73.53738827528416) bank81957 +81958 POINT(40.72227312376571 -73.12085737212381) bank81958 +81959 POINT(40.13181551918228 -74.54299825154149) bank81959 +81960 POINT(40.61063910557294 -73.54328625743797) bank81960 +81961 POINT(41.10489941706132 -74.51517385057207) bank81961 +81962 POINT(39.75955298890519 -73.23668044749304) bank81962 +81963 POINT(41.084659202320175 -74.35180512952533) bank81963 +81964 POINT(40.997980394859496 -74.36041015882445) bank81964 +81965 POINT(41.3917289189901 -74.9980362942184) bank81965 +81966 POINT(40.77594490294945 -73.12111623461385) bank81966 +81967 POINT(41.012457961255535 -73.10463716790574) bank81967 +81968 POINT(40.62777285746336 -74.97920748578632) bank81968 +81969 POINT(41.16333868937268 -74.18170786932528) bank81969 +81970 POINT(41.00208445646687 -74.60437921060333) bank81970 +81971 POINT(41.599017414622224 -74.54586021466422) bank81971 +81972 POINT(41.49911940373873 -74.54759428796464) bank81972 +81973 POINT(41.060589911223516 -73.5032943076954) bank81973 +81974 POINT(41.21168116389964 -73.55132725284963) bank81974 +81975 POINT(40.13391942460569 -73.9244461009777) bank81975 +81976 POINT(39.78904444962803 -74.06219190091146) bank81976 +81977 POINT(39.984497938502265 -73.12496333922178) bank81977 +81978 POINT(41.50600719186032 -74.62625291893117) bank81978 +81979 POINT(40.839502464540864 -74.63976917753308) bank81979 +81980 POINT(41.453389266499435 -74.59084160122829) bank81980 +81981 POINT(39.72303438767593 -74.54329629283856) bank81981 +81982 POINT(40.948522122692 -74.47699473125182) bank81982 +81983 POINT(41.365112007338666 -74.28746233870415) bank81983 +81984 POINT(40.25348468961057 -73.73925393724127) bank81984 +81985 POINT(41.56660740030505 -74.89272223274791) bank81985 +81986 POINT(41.27350576161721 -73.26505998115417) bank81986 +81987 POINT(41.237986061037375 -73.5872558536726) bank81987 +81988 POINT(41.42490967861931 -74.52935258179265) bank81988 +81989 POINT(39.765368261888625 -73.87227511864477) bank81989 +81990 POINT(39.801687964144 -73.24681323534222) bank81990 +81991 POINT(41.31170760602256 -74.83243784339757) bank81991 +81992 POINT(41.278282298486474 -73.99869426924299) bank81992 +81993 POINT(41.259366817710266 -74.79125576838678) bank81993 +81994 POINT(39.848762066911966 -74.76226504915265) bank81994 +81995 POINT(41.14327323845346 -74.46757546705072) bank81995 +81996 POINT(39.92852302271712 -73.51689424791599) bank81996 +81997 POINT(39.75981888308863 -73.18766567391256) bank81997 +81998 POINT(40.576000804003954 -74.89949810750674) bank81998 +81999 POINT(40.48889249879951 -74.06774924639342) bank81999 +82000 POINT(40.78058306022811 -74.85965117684802) bank82000 +82001 POINT(41.30006057265432 -73.41377528411063) bank82001 +82002 POINT(39.86321219607495 -73.26753753731468) bank82002 +82003 POINT(40.940381227632926 -73.49331117979543) bank82003 +82004 POINT(39.749172837275204 -74.1759625186482) bank82004 +82005 POINT(40.19960958221345 -74.59448935872992) bank82005 +82006 POINT(41.26140527244489 -74.40063094601241) bank82006 +82007 POINT(41.0113486223014 -73.5624434662938) bank82007 +82008 POINT(41.3824439860451 -74.98020470160263) bank82008 +82009 POINT(40.286178672501386 -73.61180907588019) bank82009 +82010 POINT(41.428116323770375 -73.64084275894314) bank82010 +82011 POINT(41.3479641540478 -74.63845215828671) bank82011 +82012 POINT(39.922838046968856 -73.82794373659029) bank82012 +82013 POINT(40.92589069939361 -74.47421865479275) bank82013 +82014 POINT(41.00362586956347 -73.9562186493512) bank82014 +82015 POINT(41.336558631065074 -73.61689595608058) bank82015 +82016 POINT(41.31125964406562 -74.55088348669435) bank82016 +82017 POINT(40.16799668553059 -73.02676320910605) bank82017 +82018 POINT(40.40252161791604 -74.99137111034544) bank82018 +82019 POINT(40.65366730842309 -73.62243151691798) bank82019 +82020 POINT(40.62427741454521 -74.76255437077495) bank82020 +82021 POINT(39.89472732447949 -73.0149496463131) bank82021 +82022 POINT(40.89354694514557 -74.69524664990765) bank82022 +82023 POINT(41.22555259899048 -73.98392761848898) bank82023 +82024 POINT(40.0500885164058 -73.25941749883508) bank82024 +82025 POINT(41.032842438593946 -74.9444580469154) bank82025 +82026 POINT(41.356530782350546 -73.61642317457267) bank82026 +82027 POINT(41.58645034675083 -74.26552318779235) bank82027 +82028 POINT(39.920263018837225 -73.76992638393179) bank82028 +82029 POINT(41.148225762143845 -73.53112162740739) bank82029 +82030 POINT(40.24397973714512 -73.4252554340993) bank82030 +82031 POINT(40.218667380383586 -74.54464767976408) bank82031 +82032 POINT(40.15926745680984 -74.56058555794932) bank82032 +82033 POINT(40.742539771121024 -74.82114558455632) bank82033 +82034 POINT(40.554854254274915 -74.96992600495139) bank82034 +82035 POINT(40.29350442707389 -74.49307906698576) bank82035 +82036 POINT(39.86916050260383 -74.40916970405085) bank82036 +82037 POINT(41.153881221368046 -74.92143433352255) bank82037 +82038 POINT(39.947504717167504 -73.29347024820606) bank82038 +82039 POINT(40.00352005318786 -73.23513226131288) bank82039 +82040 POINT(40.24012947015086 -73.72533164449975) bank82040 +82041 POINT(40.289390219188576 -74.17759624241266) bank82041 +82042 POINT(41.47531357904017 -73.77094306791639) bank82042 +82043 POINT(40.47733609065233 -73.85766989136161) bank82043 +82044 POINT(40.27751392831185 -74.84674864074964) bank82044 +82045 POINT(40.13394939753604 -74.87422230720574) bank82045 +82046 POINT(40.9602622160371 -73.79413043408094) bank82046 +82047 POINT(39.9472231172351 -74.61612251448688) bank82047 +82048 POINT(40.109945626938575 -74.41763179391147) bank82048 +82049 POINT(41.35278665329431 -74.51803052283925) bank82049 +82050 POINT(40.05523703070102 -73.51868287628474) bank82050 +82051 POINT(39.74986612151743 -74.2887406165699) bank82051 +82052 POINT(41.265946869794625 -73.1322363812275) bank82052 +82053 POINT(40.55002034345502 -74.80674621441133) bank82053 +82054 POINT(39.72045892233973 -73.98347388128796) bank82054 +82055 POINT(40.7947079805073 -74.75023165983113) bank82055 +82056 POINT(40.213060163737694 -74.88658494318082) bank82056 +82057 POINT(41.346687040820605 -74.97136772126346) bank82057 +82058 POINT(41.63157646817956 -73.5684687428031) bank82058 +82059 POINT(41.18253022373163 -73.72593911757632) bank82059 +82060 POINT(40.779390636161644 -73.85522565983223) bank82060 +82061 POINT(41.011873562131974 -74.50612417436015) bank82061 +82062 POINT(40.20631296202054 -74.22063063355395) bank82062 +82063 POINT(40.906944814557704 -73.6859195891061) bank82063 +82064 POINT(41.05989861200675 -74.24902497467095) bank82064 +82065 POINT(39.931977833193635 -74.9939375046155) bank82065 +82066 POINT(40.73271313478445 -74.54545837081068) bank82066 +82067 POINT(41.30386508442781 -74.0930829223453) bank82067 +82068 POINT(40.76256519806321 -73.01537352099757) bank82068 +82069 POINT(40.588414895766235 -74.71218767277585) bank82069 +82070 POINT(41.35539475489129 -74.18830604090061) bank82070 +82071 POINT(40.00618906878461 -74.88357193414691) bank82071 +82072 POINT(40.120229789169045 -74.11047717805872) bank82072 +82073 POINT(41.63148796890065 -73.33369258726377) bank82073 +82074 POINT(41.30183132937001 -74.67562425488539) bank82074 +82075 POINT(40.57414282869064 -73.92354730866849) bank82075 +82076 POINT(39.930573957274426 -74.95513620924021) bank82076 +82077 POINT(41.29700358171865 -73.80091858603744) bank82077 +82078 POINT(40.69974802487567 -73.68284866617078) bank82078 +82079 POINT(39.861319036994196 -74.70163089147484) bank82079 +82080 POINT(40.923348066126024 -74.88337046627667) bank82080 +82081 POINT(39.846579473606575 -73.25386071855824) bank82081 +82082 POINT(40.323184319304254 -74.79309976554488) bank82082 +82083 POINT(40.24026530555099 -74.93337921708856) bank82083 +82084 POINT(40.10373696978148 -73.16808523999764) bank82084 +82085 POINT(39.89743827879094 -73.4586207145834) bank82085 +82086 POINT(41.570724628273375 -74.27127958487769) bank82086 +82087 POINT(40.336822067510944 -74.8027969835813) bank82087 +82088 POINT(40.825260281077455 -74.38614503829913) bank82088 +82089 POINT(40.676539142464684 -74.93645232109806) bank82089 +82090 POINT(40.15717661956631 -73.58433124151024) bank82090 +82091 POINT(40.03575164326132 -73.32424858046615) bank82091 +82092 POINT(41.65571230408222 -74.00710391329245) bank82092 +82093 POINT(41.51404588224085 -73.24665697980663) bank82093 +82094 POINT(41.65639345712756 -73.71349189565966) bank82094 +82095 POINT(40.50839792247781 -73.1146055899587) bank82095 +82096 POINT(40.29546438851615 -74.33975941710497) bank82096 +82097 POINT(41.081871257272844 -73.37958724960755) bank82097 +82098 POINT(39.991060215750664 -74.16681469878283) bank82098 +82099 POINT(41.11458582496343 -73.11414362628875) bank82099 +82100 POINT(40.57498925350882 -73.32567675726936) bank82100 +82101 POINT(40.420704447193195 -74.03337815467403) bank82101 +82102 POINT(41.37201551734972 -73.67410827598832) bank82102 +82103 POINT(40.52952479025368 -74.34246165593443) bank82103 +82104 POINT(41.42349201154124 -73.06192158662493) bank82104 +82105 POINT(41.051603012967234 -74.38168591406907) bank82105 +82106 POINT(40.71700723461545 -73.4177621976027) bank82106 +82107 POINT(40.537248364129404 -73.57476561086236) bank82107 +82108 POINT(41.15045002341259 -73.76476917398642) bank82108 +82109 POINT(41.3551236813009 -74.98472356924105) bank82109 +82110 POINT(40.910646546352496 -73.68062124547598) bank82110 +82111 POINT(40.760957241145974 -74.293961830826) bank82111 +82112 POINT(41.03110983307497 -74.71018447476548) bank82112 +82113 POINT(39.82924516216377 -73.82080201731279) bank82113 +82114 POINT(40.11830689305404 -73.01456464263187) bank82114 +82115 POINT(40.89469421116741 -74.66799545066775) bank82115 +82116 POINT(41.14810830187734 -74.00358755283347) bank82116 +82117 POINT(39.9427183676865 -73.80171709156777) bank82117 +82118 POINT(40.10830564770699 -73.87542670474717) bank82118 +82119 POINT(40.97924798496538 -73.5236522004891) bank82119 +82120 POINT(40.52622082635916 -74.03326256387233) bank82120 +82121 POINT(41.06738363968724 -73.76488465861999) bank82121 +82122 POINT(39.80655466201413 -73.14975268912359) bank82122 +82123 POINT(41.538233440627934 -74.1865115249516) bank82123 +82124 POINT(40.23874941690611 -74.37664951623051) bank82124 +82125 POINT(41.022482595347725 -74.4396255849952) bank82125 +82126 POINT(39.858796053590126 -73.91740514934622) bank82126 +82127 POINT(41.59554178009739 -73.06684684123198) bank82127 +82128 POINT(40.366426237136345 -74.32632380706217) bank82128 +82129 POINT(40.50459893239303 -74.50048593999082) bank82129 +82130 POINT(41.4738544196823 -74.23022035253025) bank82130 +82131 POINT(39.78586554921859 -74.74207194142814) bank82131 +82132 POINT(40.635982533327756 -74.32172906839502) bank82132 +82133 POINT(40.72539758060138 -73.42944155596666) bank82133 +82134 POINT(41.236426973237535 -73.32336945387658) bank82134 +82135 POINT(39.913322441942256 -74.21459346910022) bank82135 +82136 POINT(40.93586734210662 -73.30356499299955) bank82136 +82137 POINT(41.60204735884315 -73.94969692168087) bank82137 +82138 POINT(39.83455720925857 -74.3396612726284) bank82138 +82139 POINT(40.50048962470165 -73.81834873821391) bank82139 +82140 POINT(40.17577164934345 -73.62159428986425) bank82140 +82141 POINT(41.08162849502362 -73.84498523450918) bank82141 +82142 POINT(40.410118511874124 -73.28557326247822) bank82142 +82143 POINT(39.89661326935823 -74.03257149685341) bank82143 +82144 POINT(40.39959675375338 -74.07292631075545) bank82144 +82145 POINT(41.22053596289512 -74.33254616560325) bank82145 +82146 POINT(40.0516037831752 -73.87064737984605) bank82146 +82147 POINT(39.84038517289531 -74.2573283969731) bank82147 +82148 POINT(41.0242835590141 -73.4911799038124) bank82148 +82149 POINT(40.887072051831424 -75.00234767033385) bank82149 +82150 POINT(39.88492665411648 -74.30469223064397) bank82150 +82151 POINT(40.52135128160651 -74.18887570218692) bank82151 +82152 POINT(41.151879539597495 -73.86234744783204) bank82152 +82153 POINT(41.685938084995854 -74.9580041653373) bank82153 +82154 POINT(41.4816400749146 -73.17317106050807) bank82154 +82155 POINT(41.53621758431773 -74.69042897452009) bank82155 +82156 POINT(40.16852487714647 -74.01624363766018) bank82156 +82157 POINT(39.885011111633524 -74.66901169666644) bank82157 +82158 POINT(40.71113687072863 -73.24017097091365) bank82158 +82159 POINT(40.217099883788045 -73.67656133282526) bank82159 +82160 POINT(40.634215170447696 -73.04646943154222) bank82160 +82161 POINT(40.45596725797227 -73.45546738228143) bank82161 +82162 POINT(39.79827695083671 -74.85782123742689) bank82162 +82163 POINT(40.09136832755329 -73.54765160875654) bank82163 +82164 POINT(41.49515581479777 -74.08034391253376) bank82164 +82165 POINT(40.80050148829296 -73.85159684327563) bank82165 +82166 POINT(41.3563000127806 -73.27874958083852) bank82166 +82167 POINT(41.63863485239207 -73.52447086599886) bank82167 +82168 POINT(40.421114936348474 -74.90976708781105) bank82168 +82169 POINT(40.42207417822981 -73.49160455189566) bank82169 +82170 POINT(40.425241961699456 -73.7272587067034) bank82170 +82171 POINT(40.72764106349352 -73.50176208910335) bank82171 +82172 POINT(39.78897965885583 -73.51373024002386) bank82172 +82173 POINT(40.68555323363372 -74.50206525190858) bank82173 +82174 POINT(39.965467647848364 -74.30754230684516) bank82174 +82175 POINT(41.08165007373112 -73.1540609995638) bank82175 +82176 POINT(40.77992541366526 -73.08420741767563) bank82176 +82177 POINT(40.5695982283185 -74.39676258533319) bank82177 +82178 POINT(40.9685830766675 -73.75598476893123) bank82178 +82179 POINT(41.49145774351811 -73.90308273212696) bank82179 +82180 POINT(40.91273875975638 -74.56298739226344) bank82180 +82181 POINT(40.1689085874291 -74.49899008469983) bank82181 +82182 POINT(40.51288032348945 -74.2539298391773) bank82182 +82183 POINT(40.41401958757815 -73.26312465708881) bank82183 +82184 POINT(40.157591840840475 -73.2218413938235) bank82184 +82185 POINT(41.51050702790316 -73.80821409115859) bank82185 +82186 POINT(40.1076618034166 -73.53874157493303) bank82186 +82187 POINT(40.22963974478328 -74.39821536356651) bank82187 +82188 POINT(40.34114764286882 -73.79964800622757) bank82188 +82189 POINT(40.32961421735179 -74.23173915142348) bank82189 +82190 POINT(40.23716286588664 -74.21312210023527) bank82190 +82191 POINT(40.48293090717174 -73.57364547043828) bank82191 +82192 POINT(40.537866656107475 -74.51522293025572) bank82192 +82193 POINT(40.28787691323617 -74.82854639845647) bank82193 +82194 POINT(40.77146264682736 -73.30606244694799) bank82194 +82195 POINT(41.228287164026334 -73.28246716232175) bank82195 +82196 POINT(41.44478049819972 -74.3960045248681) bank82196 +82197 POINT(41.30261386825358 -74.66309726024254) bank82197 +82198 POINT(41.11542443991162 -73.30947087734226) bank82198 +82199 POINT(41.314805111372735 -74.5276422989536) bank82199 +82200 POINT(40.67402350096868 -74.45454386693577) bank82200 +82201 POINT(40.31215488958346 -73.94169373881523) bank82201 +82202 POINT(40.59762165104028 -75.00334024514392) bank82202 +82203 POINT(39.926187858226356 -73.87872166674615) bank82203 +82204 POINT(40.547222718422425 -73.95212866298134) bank82204 +82205 POINT(41.04976387349889 -73.40031798567895) bank82205 +82206 POINT(40.17657596804294 -73.3554270195841) bank82206 +82207 POINT(39.99009085883736 -74.2018455126141) bank82207 +82208 POINT(41.52966888522208 -73.07168457190528) bank82208 +82209 POINT(40.03721100274559 -74.03842044181476) bank82209 +82210 POINT(40.06132273563154 -74.48733686116832) bank82210 +82211 POINT(40.385920494267 -73.25803184026556) bank82211 +82212 POINT(41.636442283181516 -74.26786014484074) bank82212 +82213 POINT(40.72315726358663 -73.84024182765151) bank82213 +82214 POINT(41.56154375870113 -74.59659420394124) bank82214 +82215 POINT(41.687210445442304 -73.12984531487035) bank82215 +82216 POINT(39.946368243871454 -74.88916101100357) bank82216 +82217 POINT(39.80337893080984 -73.31960668348934) bank82217 +82218 POINT(40.0837494217861 -74.09135623577383) bank82218 +82219 POINT(40.94683577659749 -74.30809054691855) bank82219 +82220 POINT(40.97287795348448 -74.58598284973691) bank82220 +82221 POINT(39.97806246755434 -74.58171610847872) bank82221 +82222 POINT(41.27382872512236 -73.50139614772056) bank82222 +82223 POINT(39.77001640594481 -74.3840970122846) bank82223 +82224 POINT(39.85633508650968 -74.48197736970371) bank82224 +82225 POINT(41.583903496036115 -74.73137495873841) bank82225 +82226 POINT(39.91851780550306 -74.49961916579397) bank82226 +82227 POINT(40.16389006324087 -73.73968712410144) bank82227 +82228 POINT(41.12500901614264 -74.2076044453511) bank82228 +82229 POINT(40.39557980320191 -74.23263656091083) bank82229 +82230 POINT(41.055038825971025 -73.62031375127941) bank82230 +82231 POINT(40.93119261015217 -73.94171994966204) bank82231 +82232 POINT(41.38384025010546 -73.69272462615399) bank82232 +82233 POINT(40.163258351570484 -73.10888420527918) bank82233 +82234 POINT(40.0180159503276 -74.2979597624099) bank82234 +82235 POINT(39.79223910802499 -73.70453126329166) bank82235 +82236 POINT(39.95724470222907 -74.1620063866769) bank82236 +82237 POINT(41.37302348627367 -73.26258018593184) bank82237 +82238 POINT(40.86274341133899 -74.5594470523261) bank82238 +82239 POINT(41.55581725632886 -74.11678793591618) bank82239 +82240 POINT(40.51206965704357 -74.03598576690096) bank82240 +82241 POINT(40.017005772037166 -74.2627707982124) bank82241 +82242 POINT(40.46042226078277 -73.80759188128793) bank82242 +82243 POINT(40.041025975017554 -73.8251152317917) bank82243 +82244 POINT(40.964318933821396 -73.1383239162784) bank82244 +82245 POINT(41.44470551755941 -74.86713094873969) bank82245 +82246 POINT(40.10223724737798 -73.51974353466412) bank82246 +82247 POINT(41.64077375013539 -74.98513258953707) bank82247 +82248 POINT(40.734314501440274 -73.30590417713445) bank82248 +82249 POINT(39.90640931063356 -74.14457626440955) bank82249 +82250 POINT(40.540813655968925 -73.99056949527206) bank82250 +82251 POINT(41.298720058507065 -73.97188581974045) bank82251 +82252 POINT(41.589636717124215 -74.45593949973065) bank82252 +82253 POINT(40.75329791573405 -74.01662367143953) bank82253 +82254 POINT(39.965410092550684 -74.34290453545717) bank82254 +82255 POINT(40.28027339855892 -73.5314839714529) bank82255 +82256 POINT(41.12209242345969 -73.93660814784619) bank82256 +82257 POINT(41.29742030130947 -74.9665752451079) bank82257 +82258 POINT(41.34370297610509 -74.73216139992812) bank82258 +82259 POINT(41.15649343188109 -74.32162275889075) bank82259 +82260 POINT(40.38077896469958 -73.65948767245516) bank82260 +82261 POINT(40.60169641597303 -73.22199824587571) bank82261 +82262 POINT(39.798698298451484 -73.30122657002215) bank82262 +82263 POINT(41.44988695476284 -74.75111632523742) bank82263 +82264 POINT(41.35340560131505 -73.50565907472226) bank82264 +82265 POINT(39.908903880172204 -73.45180146023249) bank82265 +82266 POINT(40.63029243499349 -74.44830475555284) bank82266 +82267 POINT(40.009362845531776 -73.05726137436625) bank82267 +82268 POINT(41.43172216662664 -73.32942324067497) bank82268 +82269 POINT(41.04851635999675 -73.94810014994738) bank82269 +82270 POINT(40.517213978113304 -73.32352353223017) bank82270 +82271 POINT(41.29572132144222 -74.33102474694411) bank82271 +82272 POINT(41.22904681886463 -74.53321555276693) bank82272 +82273 POINT(41.68251367140646 -74.27117523297358) bank82273 +82274 POINT(40.92606353626002 -73.39568921166848) bank82274 +82275 POINT(40.40237756975548 -74.28778356459917) bank82275 +82276 POINT(40.848087552065486 -73.35370615022975) bank82276 +82277 POINT(39.79491138078986 -73.94422631110562) bank82277 +82278 POINT(41.485336609362776 -74.00127626216548) bank82278 +82279 POINT(41.54017782156145 -73.2243751252882) bank82279 +82280 POINT(40.0465024458333 -74.31976981321147) bank82280 +82281 POINT(40.575862820492006 -73.74889769531201) bank82281 +82282 POINT(40.29397772219494 -73.99191831221871) bank82282 +82283 POINT(40.550391421623864 -74.41204153253653) bank82283 +82284 POINT(40.355026375679564 -73.54683016641381) bank82284 +82285 POINT(41.51950771553242 -73.32921959588325) bank82285 +82286 POINT(39.94975075500621 -74.54619470216318) bank82286 +82287 POINT(40.23946949520911 -74.18841238143281) bank82287 +82288 POINT(41.37406354798055 -73.49439455428225) bank82288 +82289 POINT(41.385764361361936 -73.21244662286033) bank82289 +82290 POINT(40.335476340790834 -73.63286130134028) bank82290 +82291 POINT(40.968643223750306 -73.19314810147299) bank82291 +82292 POINT(40.743671793905264 -74.16337205117192) bank82292 +82293 POINT(40.195937488979254 -74.2276211938737) bank82293 +82294 POINT(41.04636079460589 -74.54269317825757) bank82294 +82295 POINT(40.81027401226169 -73.72537653015496) bank82295 +82296 POINT(39.880454916423254 -73.68803483661645) bank82296 +82297 POINT(40.807357272447334 -74.70960773190176) bank82297 +82298 POINT(40.9182612536612 -73.80252891003406) bank82298 +82299 POINT(40.14787260964396 -74.4909389473906) bank82299 +82300 POINT(40.870336142479495 -73.13208205150471) bank82300 +82301 POINT(39.73520783334823 -73.30806420048326) bank82301 +82302 POINT(41.191268861045984 -73.6025000151162) bank82302 +82303 POINT(39.82449144949555 -74.02602262177655) bank82303 +82304 POINT(40.59752960434536 -74.79876249368233) bank82304 +82305 POINT(40.11382239702078 -74.76799169767136) bank82305 +82306 POINT(40.994211837918236 -74.96928017086688) bank82306 +82307 POINT(40.59634519613764 -73.11141751667563) bank82307 +82308 POINT(40.76132349213517 -74.71114801815992) bank82308 +82309 POINT(40.89143888333672 -74.54752899715889) bank82309 +82310 POINT(39.869345617719375 -73.72674377367315) bank82310 +82311 POINT(41.218863698709455 -74.77790878675034) bank82311 +82312 POINT(40.21109472086633 -74.551764438391) bank82312 +82313 POINT(40.81240144730191 -73.73070500014046) bank82313 +82314 POINT(40.06852606954684 -73.05708340946227) bank82314 +82315 POINT(39.98345229896766 -74.41364674594017) bank82315 +82316 POINT(41.37426072789026 -74.05844794263834) bank82316 +82317 POINT(41.32161487243525 -74.56829559736488) bank82317 +82318 POINT(41.44424350351267 -73.9880298297187) bank82318 +82319 POINT(40.91491546209786 -74.084918461542) bank82319 +82320 POINT(41.24716226937639 -73.87417444081427) bank82320 +82321 POINT(40.89431168338754 -74.19464885538284) bank82321 +82322 POINT(41.17437764049529 -74.96498991345378) bank82322 +82323 POINT(41.103188073900846 -74.0248104573024) bank82323 +82324 POINT(41.067873100116095 -74.46365278204074) bank82324 +82325 POINT(39.89666290112594 -73.52948315740399) bank82325 +82326 POINT(40.86843172449307 -73.47795764775267) bank82326 +82327 POINT(40.54173770810739 -74.60854341834263) bank82327 +82328 POINT(40.92670660165191 -73.12727507312151) bank82328 +82329 POINT(39.90254419529859 -74.14753495960097) bank82329 +82330 POINT(41.62208611266154 -74.57940172231359) bank82330 +82331 POINT(40.876262258661676 -73.72311733515103) bank82331 +82332 POINT(40.52964287135127 -73.1980495631063) bank82332 +82333 POINT(41.44230028881268 -73.02978580635228) bank82333 +82334 POINT(41.61189501371524 -74.45336798989534) bank82334 +82335 POINT(40.03688655927961 -73.0953306828411) bank82335 +82336 POINT(39.962043306731545 -73.96885501647161) bank82336 +82337 POINT(40.349600285754796 -73.20234690124371) bank82337 +82338 POINT(40.12309471646943 -73.48438627173356) bank82338 +82339 POINT(39.9135327989887 -73.28435675997137) bank82339 +82340 POINT(40.409263049340225 -75.0031052297781) bank82340 +82341 POINT(40.66446044958218 -74.6761024711437) bank82341 +82342 POINT(39.82345395459229 -73.38077318913116) bank82342 +82343 POINT(40.89460921480942 -73.37189605276053) bank82343 +82344 POINT(41.04342653404733 -73.41541836573056) bank82344 +82345 POINT(41.3731939998183 -74.38020391549055) bank82345 +82346 POINT(41.2434375799431 -74.96341491254672) bank82346 +82347 POINT(40.928801058067386 -73.62921111084597) bank82347 +82348 POINT(40.548164022082794 -73.72169279147563) bank82348 +82349 POINT(41.558923027477206 -74.43818132333807) bank82349 +82350 POINT(39.741788991427384 -74.7262140758477) bank82350 +82351 POINT(39.999136520422525 -74.39434780956087) bank82351 +82352 POINT(39.93716536084483 -74.04692708412088) bank82352 +82353 POINT(41.65861829487992 -73.91713092233518) bank82353 +82354 POINT(40.71011441055354 -73.77497220300367) bank82354 +82355 POINT(39.73970027830184 -73.1045002645608) bank82355 +82356 POINT(40.620782315813685 -73.26902659841616) bank82356 +82357 POINT(41.65768231963073 -74.0301011374526) bank82357 +82358 POINT(40.216838340619255 -74.89652516164402) bank82358 +82359 POINT(40.504664481807424 -74.60917252887657) bank82359 +82360 POINT(41.02183958237252 -73.63496667432082) bank82360 +82361 POINT(39.935744306181434 -74.91775923508305) bank82361 +82362 POINT(40.68973754943005 -74.01530502459173) bank82362 +82363 POINT(41.62842029632036 -73.40159558566695) bank82363 +82364 POINT(39.77364529402058 -74.60090198129762) bank82364 +82365 POINT(40.60026276618367 -73.52723089773566) bank82365 +82366 POINT(41.17258909142706 -74.40469538854899) bank82366 +82367 POINT(40.427605501450856 -73.05257741605189) bank82367 +82368 POINT(40.13902128122379 -73.78797990083339) bank82368 +82369 POINT(41.52468569436567 -73.36273611528522) bank82369 +82370 POINT(40.43803479497632 -73.03702711656284) bank82370 +82371 POINT(40.31759837058002 -74.83665784541253) bank82371 +82372 POINT(41.46243950543037 -73.10031491169293) bank82372 +82373 POINT(41.33267042353661 -74.92572802648479) bank82373 +82374 POINT(41.01446443199759 -73.37185069745999) bank82374 +82375 POINT(39.95279149097895 -74.19246134167373) bank82375 +82376 POINT(41.0142678479 -74.96782526011103) bank82376 +82377 POINT(41.142652162862916 -74.35173359825104) bank82377 +82378 POINT(40.30761951071108 -74.03631381577966) bank82378 +82379 POINT(41.60524501796107 -74.724017565421) bank82379 +82380 POINT(41.68758790174749 -73.76031390102648) bank82380 +82381 POINT(40.942756819329745 -74.28801338272365) bank82381 +82382 POINT(40.74354169547609 -74.62543781165898) bank82382 +82383 POINT(40.53670845157225 -74.70384795284608) bank82383 +82384 POINT(41.553489294843544 -73.59318529043898) bank82384 +82385 POINT(40.35193310202318 -73.05731609742058) bank82385 +82386 POINT(41.12045280224083 -73.55871721131908) bank82386 +82387 POINT(40.92278798551408 -73.19223645909139) bank82387 +82388 POINT(40.81229822106483 -73.07933381720724) bank82388 +82389 POINT(39.96751004383242 -74.80974322800066) bank82389 +82390 POINT(40.631857168460314 -74.84429961856426) bank82390 +82391 POINT(39.89538654012899 -74.80697800573505) bank82391 +82392 POINT(41.4465508006469 -74.70819327025383) bank82392 +82393 POINT(40.819431473690614 -73.58074683113192) bank82393 +82394 POINT(39.91500123514263 -73.13520858392519) bank82394 +82395 POINT(41.58819800627685 -74.30788568544453) bank82395 +82396 POINT(40.80086077375342 -73.30810495023793) bank82396 +82397 POINT(40.14727854475555 -74.85262994646995) bank82397 +82398 POINT(40.35995465296281 -73.42240468695162) bank82398 +82399 POINT(40.60383639180643 -74.78179751346657) bank82399 +82400 POINT(41.22201771330288 -73.90530073407484) bank82400 +82401 POINT(41.26830031024681 -73.10960548436081) bank82401 +82402 POINT(40.494497449913844 -73.70951723972664) bank82402 +82403 POINT(40.00459083061408 -73.050916616435) bank82403 +82404 POINT(40.178348440625456 -74.35831550918616) bank82404 +82405 POINT(41.004978693921146 -73.93472646693616) bank82405 +82406 POINT(40.12000736394806 -73.2997464227832) bank82406 +82407 POINT(40.34553597924981 -74.31000003794075) bank82407 +82408 POINT(40.401619565910394 -74.91212484743505) bank82408 +82409 POINT(40.02186757237092 -73.53745228478482) bank82409 +82410 POINT(40.958192720996784 -74.6007670336072) bank82410 +82411 POINT(40.03072238488848 -74.8477527431135) bank82411 +82412 POINT(40.117670094509755 -73.78129397787355) bank82412 +82413 POINT(40.925730354893524 -73.37783550857976) bank82413 +82414 POINT(40.464347639431026 -73.10418421776778) bank82414 +82415 POINT(41.47165109808997 -73.4950488352155) bank82415 +82416 POINT(40.42677181782527 -73.3349893290458) bank82416 +82417 POINT(41.17117156086163 -73.56838466936921) bank82417 +82418 POINT(40.86890927730416 -73.57803260612346) bank82418 +82419 POINT(40.191548437526826 -74.75020723749157) bank82419 +82420 POINT(40.95311452931714 -74.5222206263908) bank82420 +82421 POINT(40.09660680852852 -73.26375876068245) bank82421 +82422 POINT(40.64180664870574 -73.92082748359856) bank82422 +82423 POINT(40.04584271106848 -74.57548159162943) bank82423 +82424 POINT(40.926182973401886 -73.46097609449261) bank82424 +82425 POINT(40.568854383678904 -73.90639540869235) bank82425 +82426 POINT(40.61332729540641 -73.6546228830386) bank82426 +82427 POINT(41.218741695245804 -74.56469594710656) bank82427 +82428 POINT(41.22198151151407 -73.97955976419091) bank82428 +82429 POINT(40.99447618433424 -73.37340743449897) bank82429 +82430 POINT(39.95893984473349 -74.3088266295706) bank82430 +82431 POINT(41.612305055933696 -74.05185716532522) bank82431 +82432 POINT(40.52133559092102 -74.3835348621646) bank82432 +82433 POINT(39.99814856437909 -74.20090771830903) bank82433 +82434 POINT(41.35757539420409 -74.63732190084552) bank82434 +82435 POINT(41.38577377588218 -74.1991660588349) bank82435 +82436 POINT(41.16325480170523 -73.97738139183366) bank82436 +82437 POINT(41.51597966535045 -74.45499423435295) bank82437 +82438 POINT(40.83462825545841 -74.85178014390496) bank82438 +82439 POINT(39.90627913932266 -74.46450838169203) bank82439 +82440 POINT(40.294838689420345 -74.38291287459307) bank82440 +82441 POINT(40.620583260217295 -75.00207134206246) bank82441 +82442 POINT(41.539571196870114 -74.8965308991831) bank82442 +82443 POINT(40.314179357607124 -73.46624312596381) bank82443 +82444 POINT(40.203167140922886 -73.33002045907521) bank82444 +82445 POINT(39.898971318031535 -73.25867917818942) bank82445 +82446 POINT(40.49113567777205 -73.04305908526622) bank82446 +82447 POINT(40.024250107644974 -73.13853534256994) bank82447 +82448 POINT(41.349261144416566 -74.70271490028435) bank82448 +82449 POINT(40.226434170883216 -74.18720448887973) bank82449 +82450 POINT(41.41167879467587 -74.67940407975863) bank82450 +82451 POINT(40.49281705070589 -73.83059122847494) bank82451 +82452 POINT(40.47323139234322 -74.54870232135801) bank82452 +82453 POINT(40.260870405744996 -74.6173939650484) bank82453 +82454 POINT(40.91706983695267 -74.00629749232516) bank82454 +82455 POINT(39.81831808950227 -73.81743544647809) bank82455 +82456 POINT(41.43660212418083 -75.00195899920587) bank82456 +82457 POINT(40.83570747736163 -74.10284224309524) bank82457 +82458 POINT(40.21675427265527 -73.1163248542057) bank82458 +82459 POINT(40.034510027882874 -73.33040944031957) bank82459 +82460 POINT(39.92101561747849 -74.5642766656976) bank82460 +82461 POINT(40.03902885251197 -73.69788587406421) bank82461 +82462 POINT(40.13483636355965 -73.84361850453274) bank82462 +82463 POINT(40.671385519987126 -74.79324521359668) bank82463 +82464 POINT(41.078506896166424 -74.40492995344468) bank82464 +82465 POINT(39.76628889543466 -73.26372105904947) bank82465 +82466 POINT(41.481411813451736 -74.62161441175014) bank82466 +82467 POINT(40.7095438565026 -74.20790571592518) bank82467 +82468 POINT(40.35603097058235 -73.94776320002609) bank82468 +82469 POINT(40.68189999554554 -74.10424402352727) bank82469 +82470 POINT(40.44179429736005 -73.93670901454982) bank82470 +82471 POINT(40.16388977275522 -73.89463845218198) bank82471 +82472 POINT(40.87213997699075 -73.48589989196401) bank82472 +82473 POINT(40.215160632016456 -73.1275855135116) bank82473 +82474 POINT(40.33800906243723 -74.01803708332433) bank82474 +82475 POINT(41.576969071121866 -73.597556440446) bank82475 +82476 POINT(39.720037950026104 -73.47619381089144) bank82476 +82477 POINT(40.1979677626587 -73.20998727259736) bank82477 +82478 POINT(40.37481431055699 -73.65003790786203) bank82478 +82479 POINT(39.78785917315041 -74.14096402757265) bank82479 +82480 POINT(40.62893171026108 -73.40277903100467) bank82480 +82481 POINT(41.48100119126694 -73.88796465938799) bank82481 +82482 POINT(41.541694092566246 -73.98441554813984) bank82482 +82483 POINT(40.48818078951002 -74.85715833119897) bank82483 +82484 POINT(40.215166456697574 -74.13993917199471) bank82484 +82485 POINT(40.833872405261445 -73.49573357909658) bank82485 +82486 POINT(39.79246831523578 -73.57734318121284) bank82486 +82487 POINT(41.1980566605454 -74.34488376543064) bank82487 +82488 POINT(41.07555763661975 -73.58833328397134) bank82488 +82489 POINT(41.35523837739632 -74.50200100183108) bank82489 +82490 POINT(39.79891030398533 -73.71521697007672) bank82490 +82491 POINT(41.030212198820585 -73.64238186403108) bank82491 +82492 POINT(40.17471697598389 -73.4117121885299) bank82492 +82493 POINT(40.54016117746852 -74.34251957779944) bank82493 +82494 POINT(39.76617280441361 -74.7356012577387) bank82494 +82495 POINT(41.16045446922306 -73.972888822553) bank82495 +82496 POINT(40.90779149364732 -73.15594829365293) bank82496 +82497 POINT(40.59787499034566 -74.56315544830206) bank82497 +82498 POINT(39.81240698575949 -74.23012353718092) bank82498 +82499 POINT(41.23185962231451 -74.52401603992766) bank82499 +82500 POINT(40.037311684015194 -73.61254840829756) bank82500 +82501 POINT(40.26975138141451 -74.71086643562015) bank82501 +82502 POINT(40.10934622163659 -74.68979323004143) bank82502 +82503 POINT(40.803692320236486 -74.2542270343743) bank82503 +82504 POINT(39.77986773089787 -74.74142058840809) bank82504 +82505 POINT(40.83787841526667 -73.85016021659114) bank82505 +82506 POINT(39.883314513085146 -73.71343396146668) bank82506 +82507 POINT(40.156743673389215 -73.11178215384037) bank82507 +82508 POINT(40.599685633961705 -74.12584268537718) bank82508 +82509 POINT(40.871796492373534 -73.66129508389267) bank82509 +82510 POINT(40.77157538728349 -73.57845171140737) bank82510 +82511 POINT(41.07559202391437 -73.41357465664566) bank82511 +82512 POINT(40.968746512556855 -74.56305147889103) bank82512 +82513 POINT(40.52440725620577 -74.2838514775172) bank82513 +82514 POINT(40.97055529220342 -73.0707128565204) bank82514 +82515 POINT(41.292999694200596 -74.97228691554238) bank82515 +82516 POINT(41.40451071427466 -74.42207615510426) bank82516 +82517 POINT(41.57425170125327 -73.28796938965415) bank82517 +82518 POINT(40.322708934964055 -73.38256248168807) bank82518 +82519 POINT(40.10912352011093 -73.06911985609541) bank82519 +82520 POINT(41.00102326016337 -74.98012961986112) bank82520 +82521 POINT(40.90442031986585 -73.31200401090683) bank82521 +82522 POINT(39.849049905390814 -73.53078747700701) bank82522 +82523 POINT(40.97272597218706 -73.55708248568206) bank82523 +82524 POINT(39.829222694985816 -74.93667989811107) bank82524 +82525 POINT(40.808227392897464 -73.33879939826774) bank82525 +82526 POINT(40.00119947256207 -73.14905693945836) bank82526 +82527 POINT(40.02421301337259 -73.79305814879423) bank82527 +82528 POINT(41.68211763421889 -73.40648929247261) bank82528 +82529 POINT(39.90872595602596 -74.8291062276803) bank82529 +82530 POINT(41.26242695109013 -73.31257038872397) bank82530 +82531 POINT(39.895398189844975 -74.41131541535093) bank82531 +82532 POINT(40.18461823431565 -73.5289405410326) bank82532 +82533 POINT(41.52270755600987 -74.30072490462973) bank82533 +82534 POINT(40.14412053144269 -74.43881064283276) bank82534 +82535 POINT(41.30893688061798 -73.90077972183006) bank82535 +82536 POINT(40.33714747804116 -73.96085441076147) bank82536 +82537 POINT(39.919382330216806 -73.58787845063708) bank82537 +82538 POINT(41.097767593857846 -73.57644126789914) bank82538 +82539 POINT(40.25278597510805 -74.89432870364323) bank82539 +82540 POINT(40.83590308697334 -74.4694827101735) bank82540 +82541 POINT(40.62540082966652 -73.115823218114) bank82541 +82542 POINT(41.03645988892741 -73.11322589036531) bank82542 +82543 POINT(39.82745326349166 -73.14461821126847) bank82543 +82544 POINT(41.589527470680586 -73.17269945027341) bank82544 +82545 POINT(41.647670454701675 -74.81007582626411) bank82545 +82546 POINT(41.67508133218676 -73.83274465224606) bank82546 +82547 POINT(41.42097793353584 -73.05296101988745) bank82547 +82548 POINT(40.565695171247995 -74.52389297397681) bank82548 +82549 POINT(40.731582373046976 -73.33132429052728) bank82549 +82550 POINT(41.468237810270914 -73.44029028423375) bank82550 +82551 POINT(40.994776817804194 -73.95926913646095) bank82551 +82552 POINT(41.2779419867277 -74.27081808916272) bank82552 +82553 POINT(40.29477300317257 -74.43446067894224) bank82553 +82554 POINT(41.68353126743173 -74.81310174683128) bank82554 +82555 POINT(41.63973598027268 -73.66805481977721) bank82555 +82556 POINT(41.27069667141317 -73.24120450257492) bank82556 +82557 POINT(41.43853389001943 -74.07126155112798) bank82557 +82558 POINT(41.4250277876825 -74.99409763219671) bank82558 +82559 POINT(40.996898407773564 -73.12181054116901) bank82559 +82560 POINT(41.166347168037596 -74.38780993505648) bank82560 +82561 POINT(39.75201928172316 -73.65393096341607) bank82561 +82562 POINT(41.39109176743497 -73.48181856264331) bank82562 +82563 POINT(41.240650935291654 -73.58226045042296) bank82563 +82564 POINT(41.35319966934123 -73.05743325110883) bank82564 +82565 POINT(41.63114032384098 -73.80874957321912) bank82565 +82566 POINT(39.99976413555031 -73.15581448777262) bank82566 +82567 POINT(40.67054067301414 -73.2275001324549) bank82567 +82568 POINT(40.99604227592941 -74.42558681611165) bank82568 +82569 POINT(41.62675733933835 -74.16001563865379) bank82569 +82570 POINT(40.81638736219351 -74.39941915571232) bank82570 +82571 POINT(40.90839900771002 -74.85542309177558) bank82571 +82572 POINT(40.32450023327708 -74.3872009455406) bank82572 +82573 POINT(41.699837835662166 -74.92096571205903) bank82573 +82574 POINT(40.00564522620623 -73.21694357977704) bank82574 +82575 POINT(41.12203618053454 -74.64667575808318) bank82575 +82576 POINT(39.91266880909633 -74.23729905484366) bank82576 +82577 POINT(40.17819169311068 -74.648875036248) bank82577 +82578 POINT(40.323531837695455 -73.00991477003338) bank82578 +82579 POINT(40.10269528384697 -74.86593122879457) bank82579 +82580 POINT(40.50727551463058 -74.46696247503922) bank82580 +82581 POINT(40.77443180903679 -73.13688946923521) bank82581 +82582 POINT(40.52287953388884 -73.58529209662943) bank82582 +82583 POINT(40.626352309883934 -74.81619622968512) bank82583 +82584 POINT(40.31364325662828 -74.02930132569801) bank82584 +82585 POINT(41.515328240310474 -74.82409269226521) bank82585 +82586 POINT(40.68462910773545 -73.94613477280673) bank82586 +82587 POINT(39.99053863670514 -73.96887732132869) bank82587 +82588 POINT(40.48954891483132 -73.05215120559326) bank82588 +82589 POINT(40.951726102617585 -73.79965256307617) bank82589 +82590 POINT(41.51629982365607 -74.443277771675) bank82590 +82591 POINT(39.92113787016051 -74.99791685094964) bank82591 +82592 POINT(40.70269415881554 -73.65210840799107) bank82592 +82593 POINT(41.58115881556334 -74.53280784803911) bank82593 +82594 POINT(40.790780339881145 -73.4029223739729) bank82594 +82595 POINT(41.62228954140898 -73.71662065593048) bank82595 +82596 POINT(41.69924301074943 -73.06214125161335) bank82596 +82597 POINT(40.40581925906318 -73.69508223186315) bank82597 +82598 POINT(40.93161483981772 -73.68377468915976) bank82598 +82599 POINT(40.949031568739045 -73.10523807642006) bank82599 +82600 POINT(40.79121968227441 -74.94888856466305) bank82600 +82601 POINT(40.79453418896798 -73.7534774495628) bank82601 +82602 POINT(40.21518705540656 -74.46762091716694) bank82602 +82603 POINT(40.859048842967276 -74.77614878053278) bank82603 +82604 POINT(40.08191090032689 -73.22722394071616) bank82604 +82605 POINT(39.85651495852414 -74.52484453636514) bank82605 +82606 POINT(40.434734871343736 -74.76763078396203) bank82606 +82607 POINT(41.30773627662622 -73.72481172442914) bank82607 +82608 POINT(39.88217177088967 -74.86557036795367) bank82608 +82609 POINT(40.3751249029538 -74.07389683310268) bank82609 +82610 POINT(40.55870418639171 -73.59530946785567) bank82610 +82611 POINT(40.0549810451299 -73.17006659080432) bank82611 +82612 POINT(41.219594314995014 -73.56751994723979) bank82612 +82613 POINT(41.52988442448361 -74.93495456580273) bank82613 +82614 POINT(41.400886841190875 -74.95294399804413) bank82614 +82615 POINT(40.35899703404508 -73.29379309523783) bank82615 +82616 POINT(41.40795109891706 -74.3233664229592) bank82616 +82617 POINT(41.08954497787465 -73.4091057550691) bank82617 +82618 POINT(41.27292822551688 -73.63571467009044) bank82618 +82619 POINT(41.42651372442598 -74.03490899563846) bank82619 +82620 POINT(41.23167455318977 -73.42928097933957) bank82620 +82621 POINT(41.441052796114704 -73.66043668514546) bank82621 +82622 POINT(40.69340843294701 -73.34410141872944) bank82622 +82623 POINT(39.74841107143573 -73.03494011906157) bank82623 +82624 POINT(40.87298439251892 -73.66873892086585) bank82624 +82625 POINT(40.5026512835904 -73.78691099506227) bank82625 +82626 POINT(39.98196627485286 -74.77223535004651) bank82626 +82627 POINT(41.2767403510216 -73.18245054953087) bank82627 +82628 POINT(39.81593466780234 -73.66408637613517) bank82628 +82629 POINT(40.281864690720575 -73.92164740915314) bank82629 +82630 POINT(41.071841778105494 -73.65325104890074) bank82630 +82631 POINT(41.49035739190783 -74.09085138949884) bank82631 +82632 POINT(40.90006974413085 -73.73264819541318) bank82632 +82633 POINT(39.96184036154081 -73.10859879050957) bank82633 +82634 POINT(40.939222617108065 -73.45775728212053) bank82634 +82635 POINT(40.042026484024305 -73.42695838313962) bank82635 +82636 POINT(40.24523394150701 -73.04042607196742) bank82636 +82637 POINT(39.96562160339688 -74.86058945590662) bank82637 +82638 POINT(40.389652311494544 -73.77607215729628) bank82638 +82639 POINT(40.07930902989833 -74.39013160177149) bank82639 +82640 POINT(40.31054446614975 -74.01448820947846) bank82640 +82641 POINT(41.70754902170629 -73.12532942971758) bank82641 +82642 POINT(40.662998131181155 -73.74019226001607) bank82642 +82643 POINT(40.80597158097511 -73.25725302048205) bank82643 +82644 POINT(40.08584030182237 -73.4972936683257) bank82644 +82645 POINT(41.610988197971885 -74.52359508207412) bank82645 +82646 POINT(41.48858455470825 -74.48380323485353) bank82646 +82647 POINT(40.89156967219702 -74.36960007795886) bank82647 +82648 POINT(41.399768857856785 -74.50349822753016) bank82648 +82649 POINT(41.20151688998794 -73.4211117302261) bank82649 +82650 POINT(40.08666037772399 -73.93669402422255) bank82650 +82651 POINT(41.31295774420062 -74.13808760089763) bank82651 +82652 POINT(41.15036719453333 -73.7556367478494) bank82652 +82653 POINT(39.75903197027026 -73.1562088732707) bank82653 +82654 POINT(40.6841440789382 -73.9276352068709) bank82654 +82655 POINT(40.88128949671634 -74.38056058812008) bank82655 +82656 POINT(40.52479900436038 -73.91315954719717) bank82656 +82657 POINT(40.490004327890354 -74.3616497874312) bank82657 +82658 POINT(41.64624202888426 -73.80615896258097) bank82658 +82659 POINT(41.49385186863574 -74.29652006846759) bank82659 +82660 POINT(41.67895987280282 -73.69262509555288) bank82660 +82661 POINT(40.00396376200965 -74.43625686325082) bank82661 +82662 POINT(40.6612388785729 -74.90141873767948) bank82662 +82663 POINT(40.67028161883994 -74.67266183060148) bank82663 +82664 POINT(39.95359593914017 -74.97264955651933) bank82664 +82665 POINT(40.958369567395785 -73.84989808839497) bank82665 +82666 POINT(41.241289497808 -74.87568657690427) bank82666 +82667 POINT(41.13784415533188 -73.96361008427459) bank82667 +82668 POINT(41.05516233344836 -73.40999990922263) bank82668 +82669 POINT(41.6144759904866 -73.53669472450393) bank82669 +82670 POINT(41.68703696117462 -74.69541351129267) bank82670 +82671 POINT(41.3777603803217 -74.93193941966577) bank82671 +82672 POINT(40.037256075996645 -74.69760709296962) bank82672 +82673 POINT(40.76881842638412 -74.12890460789545) bank82673 +82674 POINT(41.38772426899521 -74.82740510061934) bank82674 +82675 POINT(41.685357834411924 -74.85676099882875) bank82675 +82676 POINT(41.48746379038658 -73.73894554963573) bank82676 +82677 POINT(40.1783826933762 -73.68384809546957) bank82677 +82678 POINT(40.23130126433475 -73.49385931906775) bank82678 +82679 POINT(40.41925394242061 -73.8881150518377) bank82679 +82680 POINT(40.09998557468256 -74.01453671095682) bank82680 +82681 POINT(40.74131825346982 -73.48825045443004) bank82681 +82682 POINT(40.73241834036066 -73.54867567096368) bank82682 +82683 POINT(39.7192076868958 -74.4623534012017) bank82683 +82684 POINT(41.13278666885775 -74.12237243678491) bank82684 +82685 POINT(41.08997908031588 -74.16437813267328) bank82685 +82686 POINT(41.534015250367105 -74.03827573984901) bank82686 +82687 POINT(40.68907238063167 -74.50719365064711) bank82687 +82688 POINT(41.07434681488437 -73.41373004341169) bank82688 +82689 POINT(40.18414695715662 -73.03251978490694) bank82689 +82690 POINT(39.879531614206016 -74.18068646003427) bank82690 +82691 POINT(40.35822686690679 -73.4250937561441) bank82691 +82692 POINT(40.21421086390452 -73.93320313516298) bank82692 +82693 POINT(40.65407386464258 -73.88171163223537) bank82693 +82694 POINT(40.800287861984124 -74.40627576150227) bank82694 +82695 POINT(40.07264161671133 -73.30100943933633) bank82695 +82696 POINT(40.757731207911384 -74.32392190424554) bank82696 +82697 POINT(40.47160868022962 -73.95699665533137) bank82697 +82698 POINT(40.89354472817285 -73.58339834518468) bank82698 +82699 POINT(41.235881016941576 -73.74628518635231) bank82699 +82700 POINT(41.04972524989243 -73.87243751254879) bank82700 +82701 POINT(40.74817124619493 -73.33850261800163) bank82701 +82702 POINT(39.93533930745436 -73.17132751985541) bank82702 +82703 POINT(40.279421442427065 -73.61836002190987) bank82703 +82704 POINT(41.11437479467914 -73.53997419642087) bank82704 +82705 POINT(41.53826254109062 -74.23901226261768) bank82705 +82706 POINT(40.44129990266802 -74.75732547520774) bank82706 +82707 POINT(40.201773375948314 -74.11892690187486) bank82707 +82708 POINT(39.91589095776148 -74.94853541074897) bank82708 +82709 POINT(40.737308056366665 -74.21831488734091) bank82709 +82710 POINT(39.805387713487306 -73.73364001051095) bank82710 +82711 POINT(41.67370729943753 -74.38018415883325) bank82711 +82712 POINT(41.166552094059256 -73.74716828993125) bank82712 +82713 POINT(41.37096369317567 -74.70885380218733) bank82713 +82714 POINT(40.76842116709123 -74.38645846363637) bank82714 +82715 POINT(40.46042823437869 -74.79359765828816) bank82715 +82716 POINT(41.125396696479825 -73.42518750103505) bank82716 +82717 POINT(41.25083164505868 -73.78514722217956) bank82717 +82718 POINT(40.92357855006719 -73.059711427503) bank82718 +82719 POINT(41.3724226958063 -74.75988760108056) bank82719 +82720 POINT(39.841561070218276 -73.59867519899005) bank82720 +82721 POINT(40.3702425938366 -74.65939905506296) bank82721 +82722 POINT(40.115448037255895 -73.27443649893857) bank82722 +82723 POINT(39.880836701709704 -74.42635113979202) bank82723 +82724 POINT(40.389523713738846 -74.5671625527397) bank82724 +82725 POINT(41.25810246022528 -73.70402747431162) bank82725 +82726 POINT(41.67363494565307 -73.1648673967267) bank82726 +82727 POINT(40.093161967265004 -73.64764170934221) bank82727 +82728 POINT(39.75106558728423 -74.46568687899168) bank82728 +82729 POINT(40.05229406926268 -74.43600609008472) bank82729 +82730 POINT(40.43554732915575 -74.01028892520014) bank82730 +82731 POINT(41.35741411052443 -73.9138611739999) bank82731 +82732 POINT(40.79768504856567 -73.36663272554411) bank82732 +82733 POINT(40.08615543850897 -73.94856589629617) bank82733 +82734 POINT(40.02440406161418 -74.22792013752284) bank82734 +82735 POINT(39.7474941646926 -73.28190559619343) bank82735 +82736 POINT(40.83323925976786 -74.17003260773416) bank82736 +82737 POINT(41.08703965788319 -73.68539843696826) bank82737 +82738 POINT(41.38605324508094 -73.677409109711) bank82738 +82739 POINT(40.41659114658732 -74.73785244389886) bank82739 +82740 POINT(39.792881377170374 -74.09277091313524) bank82740 +82741 POINT(41.2654932178602 -73.25082489809252) bank82741 +82742 POINT(41.08600440869928 -73.06243921216091) bank82742 +82743 POINT(41.27271432690764 -73.23188283911702) bank82743 +82744 POINT(40.01896688189467 -74.01155647930612) bank82744 +82745 POINT(39.85981246752843 -73.21249901785842) bank82745 +82746 POINT(41.48842207650719 -74.78508163057914) bank82746 +82747 POINT(40.43534154825545 -74.79183596945957) bank82747 +82748 POINT(40.04932605526782 -74.60130710565852) bank82748 +82749 POINT(40.523367969367584 -73.77264144423991) bank82749 +82750 POINT(40.43317812649556 -74.7005862679699) bank82750 +82751 POINT(40.69565648421259 -74.48114039837903) bank82751 +82752 POINT(40.36563945527943 -73.20782300584136) bank82752 +82753 POINT(40.363211129043805 -74.08076500824247) bank82753 +82754 POINT(41.07726818157008 -74.6269809598857) bank82754 +82755 POINT(40.592640164352375 -73.13730993514963) bank82755 +82756 POINT(40.32565666336695 -74.35406927599459) bank82756 +82757 POINT(40.794547204350415 -73.5806284391275) bank82757 +82758 POINT(40.648659857545056 -73.7872530971276) bank82758 +82759 POINT(39.86786499343102 -73.98699308412834) bank82759 +82760 POINT(40.59095403446805 -74.80317330106641) bank82760 +82761 POINT(41.45984694197532 -74.02090565106116) bank82761 +82762 POINT(41.56132290857015 -74.63231835570048) bank82762 +82763 POINT(41.19225044456823 -74.25546367631911) bank82763 +82764 POINT(41.018150493556895 -73.47669621784756) bank82764 +82765 POINT(41.26826778670644 -74.49161253266004) bank82765 +82766 POINT(41.02036049094207 -73.42724540689892) bank82766 +82767 POINT(40.7182878256083 -73.49287500785846) bank82767 +82768 POINT(40.95522779490575 -74.62538722328621) bank82768 +82769 POINT(41.43293047911727 -74.05133556989732) bank82769 +82770 POINT(40.45885283152412 -74.15161410179142) bank82770 +82771 POINT(41.70988021213456 -73.83062380286943) bank82771 +82772 POINT(40.754431356658245 -74.73330113673929) bank82772 +82773 POINT(41.357089149878135 -74.17802432077275) bank82773 +82774 POINT(39.91563274128442 -74.38863121978201) bank82774 +82775 POINT(40.97906152155052 -74.83983366071648) bank82775 +82776 POINT(41.56948085225781 -73.98082230681986) bank82776 +82777 POINT(41.45601080793204 -73.06842528616092) bank82777 +82778 POINT(39.92406363278907 -73.79314700199097) bank82778 +82779 POINT(40.76947880164644 -73.0924486608107) bank82779 +82780 POINT(41.214775596210586 -74.72130942278316) bank82780 +82781 POINT(40.16436278652425 -74.5151932137431) bank82781 +82782 POINT(40.88833911468996 -73.76476431610669) bank82782 +82783 POINT(40.34110480560307 -73.689079206832) bank82783 +82784 POINT(41.50307580481058 -74.43270840582986) bank82784 +82785 POINT(40.754380877258896 -73.33949390922184) bank82785 +82786 POINT(39.7611045770769 -73.34507141145198) bank82786 +82787 POINT(39.992626588067836 -73.87609751906773) bank82787 +82788 POINT(40.47226204009872 -74.02712534880625) bank82788 +82789 POINT(40.45399295557721 -73.10223712945611) bank82789 +82790 POINT(40.12859900573501 -73.53140924560613) bank82790 +82791 POINT(39.71658279000242 -73.867818989411) bank82791 +82792 POINT(39.74818144679588 -73.07203558344882) bank82792 +82793 POINT(39.993128819782804 -74.27451491431243) bank82793 +82794 POINT(39.89222425879868 -73.42562280267428) bank82794 +82795 POINT(41.337596169753695 -73.51773642617374) bank82795 +82796 POINT(41.57778443604835 -73.97589133198484) bank82796 +82797 POINT(40.279426207007965 -73.93788690402188) bank82797 +82798 POINT(40.174306047409225 -73.6563504004233) bank82798 +82799 POINT(41.30416917488887 -74.75544915465265) bank82799 +82800 POINT(39.85728186515683 -73.71457043798954) bank82800 +82801 POINT(41.00388711803466 -73.94245272163411) bank82801 +82802 POINT(41.274339566988026 -73.99253891363848) bank82802 +82803 POINT(40.205067430211436 -73.44087314326826) bank82803 +82804 POINT(41.56447203456388 -74.67810428667131) bank82804 +82805 POINT(40.3978274453427 -74.87972265137688) bank82805 +82806 POINT(41.29431948722909 -73.36175881448224) bank82806 +82807 POINT(41.675792126369366 -74.30497665485684) bank82807 +82808 POINT(40.3916577608843 -73.84673647769597) bank82808 +82809 POINT(41.59232818908414 -73.93186736471215) bank82809 +82810 POINT(39.98126210944072 -73.80120626826394) bank82810 +82811 POINT(40.365867830533354 -73.86093203603929) bank82811 +82812 POINT(40.73534415125598 -74.4255634778659) bank82812 +82813 POINT(40.89061696571893 -74.53835681343836) bank82813 +82814 POINT(41.02261691694784 -73.01802265929237) bank82814 +82815 POINT(40.02265073573227 -74.6169827642685) bank82815 +82816 POINT(40.963622383038654 -74.04369733234176) bank82816 +82817 POINT(40.71940796081644 -74.86509416339821) bank82817 +82818 POINT(40.20450588663082 -74.32403303688018) bank82818 +82819 POINT(40.9965665430288 -73.8537219435626) bank82819 +82820 POINT(40.121718134648916 -74.79020642355746) bank82820 +82821 POINT(41.52747858620019 -73.99014640368259) bank82821 +82822 POINT(40.73941839198011 -73.07049366932759) bank82822 +82823 POINT(39.8680427119131 -73.95235984631188) bank82823 +82824 POINT(41.28577865373727 -73.77615692490137) bank82824 +82825 POINT(40.88730267730073 -74.411244359476) bank82825 +82826 POINT(41.23588430372621 -73.8683321647106) bank82826 +82827 POINT(41.100852344121954 -73.12827764464463) bank82827 +82828 POINT(39.903409468868816 -74.69973841054193) bank82828 +82829 POINT(41.36764888715989 -73.59967577080688) bank82829 +82830 POINT(40.4566896202078 -73.91639980187101) bank82830 +82831 POINT(41.30236201586751 -74.06429789364068) bank82831 +82832 POINT(39.77880697960425 -74.76981991505566) bank82832 +82833 POINT(41.23806292250904 -73.54792665879921) bank82833 +82834 POINT(39.91426759002525 -73.51064976702467) bank82834 +82835 POINT(40.24283666688647 -73.50901549558299) bank82835 +82836 POINT(40.094184903943436 -74.06389019538754) bank82836 +82837 POINT(40.81415368941451 -74.6851924816977) bank82837 +82838 POINT(39.84699697157155 -73.93662155826993) bank82838 +82839 POINT(39.7778741476508 -73.99267899584812) bank82839 +82840 POINT(40.053969887115045 -74.6400562987848) bank82840 +82841 POINT(41.64296667208511 -74.1003078014385) bank82841 +82842 POINT(40.5067521420197 -73.92596970593105) bank82842 +82843 POINT(41.67261780633506 -73.24417447558768) bank82843 +82844 POINT(40.125646993646626 -74.85104564383568) bank82844 +82845 POINT(41.483037182085056 -73.04398848708027) bank82845 +82846 POINT(40.12051841037371 -73.10769242713991) bank82846 +82847 POINT(41.307556204214954 -74.49795796922352) bank82847 +82848 POINT(40.69237843056786 -73.29793274766907) bank82848 +82849 POINT(41.353795754844164 -74.76974730175365) bank82849 +82850 POINT(40.10557688510044 -74.72022913047702) bank82850 +82851 POINT(40.317383749649004 -74.9774520444675) bank82851 +82852 POINT(41.709265167590864 -73.71741998758124) bank82852 +82853 POINT(40.255857592214475 -73.7227489966112) bank82853 +82854 POINT(40.285117157758606 -73.93745612719447) bank82854 +82855 POINT(40.66807086303532 -74.84250440561192) bank82855 +82856 POINT(40.78293651024186 -73.0068178956766) bank82856 +82857 POINT(40.02843217197158 -73.28539489775106) bank82857 +82858 POINT(40.36382323813158 -73.8112054492754) bank82858 +82859 POINT(40.4232669611352 -74.60673827928211) bank82859 +82860 POINT(40.287664435090875 -74.49786072156009) bank82860 +82861 POINT(41.537488141421285 -73.52632653184163) bank82861 +82862 POINT(40.76079248772173 -73.55239665537714) bank82862 +82863 POINT(41.422284160970875 -74.05521770463466) bank82863 +82864 POINT(40.015102083607104 -73.70830032783968) bank82864 +82865 POINT(40.92734366982286 -73.80123688031433) bank82865 +82866 POINT(41.179582186505016 -74.79368063370568) bank82866 +82867 POINT(41.60044518121922 -73.16805093035703) bank82867 +82868 POINT(40.150492081872095 -73.9131007493699) bank82868 +82869 POINT(40.54724917080706 -74.15263408899982) bank82869 +82870 POINT(40.15253478855335 -74.56819174521759) bank82870 +82871 POINT(39.77270379976297 -73.88480458478664) bank82871 +82872 POINT(40.45554290852719 -74.60506946698351) bank82872 +82873 POINT(40.72823149306503 -74.96884304184798) bank82873 +82874 POINT(40.30125911239581 -73.1083697455241) bank82874 +82875 POINT(40.42702133499108 -74.63725915540837) bank82875 +82876 POINT(40.95836869162917 -73.90922303742701) bank82876 +82877 POINT(39.76334369625431 -74.8927240787516) bank82877 +82878 POINT(39.79113818794263 -74.10440514647786) bank82878 +82879 POINT(40.37123237561788 -73.40936005515253) bank82879 +82880 POINT(40.757814681420776 -74.26714528724952) bank82880 +82881 POINT(40.0703270412732 -74.22786549150297) bank82881 +82882 POINT(41.12268991291655 -73.12056563301473) bank82882 +82883 POINT(39.89254754273374 -74.93863479712543) bank82883 +82884 POINT(39.73350578924194 -73.30824378304169) bank82884 +82885 POINT(41.05170477908043 -73.46942199288571) bank82885 +82886 POINT(40.318579345884714 -73.84371767810713) bank82886 +82887 POINT(39.8117786842505 -73.95784805377896) bank82887 +82888 POINT(40.30121753770395 -73.55793464441153) bank82888 +82889 POINT(40.068565223598775 -74.50156618167772) bank82889 +82890 POINT(40.894478283080595 -74.81295512840252) bank82890 +82891 POINT(40.05203259701652 -73.44832097170462) bank82891 +82892 POINT(40.25873360168846 -74.20805548765753) bank82892 +82893 POINT(39.86796313418234 -73.46919503517935) bank82893 +82894 POINT(40.549578239128394 -73.18193605104898) bank82894 +82895 POINT(39.924241200051675 -74.26470001845347) bank82895 +82896 POINT(41.30417882497687 -74.8384023795248) bank82896 +82897 POINT(41.2656639097967 -74.3044963893211) bank82897 +82898 POINT(40.1482922456013 -73.570362207724) bank82898 +82899 POINT(41.21461634109282 -73.62167637374571) bank82899 +82900 POINT(40.068888862682826 -74.05066861986201) bank82900 +82901 POINT(40.47762815368001 -73.09541856631616) bank82901 +82902 POINT(41.58865375132776 -73.19623445761472) bank82902 +82903 POINT(40.90591468268585 -73.3661936408047) bank82903 +82904 POINT(41.60909898295537 -74.49880112295361) bank82904 +82905 POINT(40.93284797299116 -73.5986878634074) bank82905 +82906 POINT(40.67550380396939 -74.57733357629404) bank82906 +82907 POINT(39.910899662266004 -73.14582287085194) bank82907 +82908 POINT(40.70660906703981 -74.42788107500662) bank82908 +82909 POINT(39.93951004318223 -73.79355839729611) bank82909 +82910 POINT(41.02473364087031 -73.04774963472656) bank82910 +82911 POINT(39.94533872530501 -73.69929997821369) bank82911 +82912 POINT(41.60712663799378 -73.4136693817585) bank82912 +82913 POINT(40.21169088944872 -73.62529054887294) bank82913 +82914 POINT(40.938072863626275 -73.96489949732451) bank82914 +82915 POINT(40.17879819420616 -73.04292425988938) bank82915 +82916 POINT(40.1814844105632 -73.5035358379735) bank82916 +82917 POINT(41.65749150199875 -74.63768447738053) bank82917 +82918 POINT(41.371451136788636 -74.64505970149561) bank82918 +82919 POINT(40.70729712525501 -74.67198811415578) bank82919 +82920 POINT(41.39418400208375 -73.2898584145029) bank82920 +82921 POINT(41.13325005946818 -73.10315057668215) bank82921 +82922 POINT(40.026396393487445 -73.35837697520074) bank82922 +82923 POINT(41.281533940902236 -74.17672038447085) bank82923 +82924 POINT(40.75055455409176 -73.07691309847388) bank82924 +82925 POINT(41.38744541929002 -74.49603999681028) bank82925 +82926 POINT(41.2565215689712 -74.91358842214035) bank82926 +82927 POINT(39.982730853170956 -73.79731697321674) bank82927 +82928 POINT(41.67466685872678 -73.66960858675866) bank82928 +82929 POINT(40.98369606318342 -74.76559760684837) bank82929 +82930 POINT(41.295340557906705 -74.43846855046111) bank82930 +82931 POINT(41.66140771799898 -74.96492362623053) bank82931 +82932 POINT(41.03886275406721 -74.63160022627176) bank82932 +82933 POINT(40.00288996058842 -74.2855559711781) bank82933 +82934 POINT(40.84666336034379 -73.07263402542249) bank82934 +82935 POINT(40.68462393670904 -73.94272866057705) bank82935 +82936 POINT(40.23862735461877 -73.75471895777804) bank82936 +82937 POINT(41.05995756059486 -74.56440398988899) bank82937 +82938 POINT(40.896179890854334 -73.10482970442538) bank82938 +82939 POINT(40.023495744186356 -74.56250354688414) bank82939 +82940 POINT(40.6318992822554 -73.09462660456408) bank82940 +82941 POINT(40.072658416947064 -73.59945211889013) bank82941 +82942 POINT(41.005154316148115 -74.60226859862084) bank82942 +82943 POINT(40.91793362854079 -73.03165826276557) bank82943 +82944 POINT(40.22601146482113 -74.75735740284034) bank82944 +82945 POINT(40.91873608011167 -74.17912996279483) bank82945 +82946 POINT(41.067815718460416 -73.06701945884456) bank82946 +82947 POINT(40.358096129937046 -74.1304395420662) bank82947 +82948 POINT(41.21268745035697 -74.66714184859092) bank82948 +82949 POINT(41.30215693646058 -74.7503852300519) bank82949 +82950 POINT(40.04189724635175 -74.2240109171139) bank82950 +82951 POINT(40.68036160708986 -73.19156093661289) bank82951 +82952 POINT(40.45183596198356 -73.19322349751678) bank82952 +82953 POINT(40.00136713318986 -74.80596070102384) bank82953 +82954 POINT(41.53527796350505 -74.23404794618378) bank82954 +82955 POINT(40.50198494280787 -74.48069905593442) bank82955 +82956 POINT(41.65675230516716 -73.65607378411048) bank82956 +82957 POINT(41.301686753177606 -73.41914815285824) bank82957 +82958 POINT(40.061771179728304 -74.27125408054384) bank82958 +82959 POINT(39.966382469082454 -73.37354447095667) bank82959 +82960 POINT(40.233147210286646 -74.39111652611287) bank82960 +82961 POINT(40.29829917092489 -74.54620214975833) bank82961 +82962 POINT(40.32229025802526 -74.76624949949938) bank82962 +82963 POINT(41.192354681226725 -73.79784016151228) bank82963 +82964 POINT(40.031939864414625 -73.85880302973196) bank82964 +82965 POINT(41.35612287202297 -74.19180567419438) bank82965 +82966 POINT(40.23293202328542 -73.31395874595188) bank82966 +82967 POINT(41.29846594313993 -73.42719135847531) bank82967 +82968 POINT(41.17121240898292 -73.2883359255034) bank82968 +82969 POINT(40.138913665463626 -74.32438667303559) bank82969 +82970 POINT(40.32946368406956 -74.37681829378604) bank82970 +82971 POINT(41.36352037944356 -73.64306817615378) bank82971 +82972 POINT(41.29932171972787 -73.16014871550323) bank82972 +82973 POINT(40.91625957852653 -73.02918358138618) bank82973 +82974 POINT(40.707665739207336 -73.23449754349113) bank82974 +82975 POINT(41.09294458960888 -73.59934512884081) bank82975 +82976 POINT(40.62778560153309 -74.37580480751647) bank82976 +82977 POINT(41.56691964868709 -73.83721480923597) bank82977 +82978 POINT(41.10898775936312 -74.75880894294784) bank82978 +82979 POINT(39.732525219131496 -74.31217215461425) bank82979 +82980 POINT(40.79985310666428 -73.60730796127496) bank82980 +82981 POINT(40.345759985388 -73.07193120812187) bank82981 +82982 POINT(40.686982039670575 -74.37847547800524) bank82982 +82983 POINT(40.67130843977619 -73.54804841549925) bank82983 +82984 POINT(41.39659632009457 -74.7828392118938) bank82984 +82985 POINT(41.19880659382497 -74.99906402993335) bank82985 +82986 POINT(40.44161751892614 -73.12305663695257) bank82986 +82987 POINT(41.25675984142033 -73.91191384673837) bank82987 +82988 POINT(39.86390896226449 -73.78629031551964) bank82988 +82989 POINT(41.43878539771188 -74.39439296901489) bank82989 +82990 POINT(41.365031282796274 -73.7200647330655) bank82990 +82991 POINT(41.3063370450344 -74.89777764116201) bank82991 +82992 POINT(40.198308569457936 -73.68401184940744) bank82992 +82993 POINT(41.38083331855094 -74.24404974932374) bank82993 +82994 POINT(39.79602883652513 -73.01831477631261) bank82994 +82995 POINT(40.80108901652331 -73.65687633949848) bank82995 +82996 POINT(40.12299395053958 -74.47989406347833) bank82996 +82997 POINT(41.37672015937526 -73.55967381389992) bank82997 +82998 POINT(39.71923066589642 -74.75305246346254) bank82998 +82999 POINT(40.63456826163489 -74.99782410226076) bank82999 +83000 POINT(41.02368371846191 -74.38687210279531) bank83000 +83001 POINT(40.115953863791965 -73.09261844385485) bank83001 +83002 POINT(40.967608078327366 -74.12734589495378) bank83002 +83003 POINT(41.23327368094512 -73.43012711580513) bank83003 +83004 POINT(39.79855943561999 -74.26482494062114) bank83004 +83005 POINT(40.36930482436771 -73.91077124259269) bank83005 +83006 POINT(40.07883966173763 -74.25164561310534) bank83006 +83007 POINT(40.1395182042579 -74.23151552490879) bank83007 +83008 POINT(40.631127534418916 -74.01733870559167) bank83008 +83009 POINT(40.483163078447205 -73.17626557571242) bank83009 +83010 POINT(40.0787249274121 -74.2826493056579) bank83010 +83011 POINT(39.79183475762182 -74.85811376427691) bank83011 +83012 POINT(40.542495616821896 -74.30869030653878) bank83012 +83013 POINT(40.78999270448928 -73.79529964748858) bank83013 +83014 POINT(39.73548981547675 -73.659739893566) bank83014 +83015 POINT(40.289096737196246 -74.6314157940473) bank83015 +83016 POINT(40.519355340769515 -74.24429882795907) bank83016 +83017 POINT(40.18084681908265 -74.60996525559503) bank83017 +83018 POINT(41.170667732081355 -74.5985044506776) bank83018 +83019 POINT(39.80692694204435 -74.40282460933567) bank83019 +83020 POINT(40.02664008083829 -74.86587528411026) bank83020 +83021 POINT(41.34530603135289 -73.49586868156118) bank83021 +83022 POINT(41.61403321238035 -74.70023380538888) bank83022 +83023 POINT(41.001190100453115 -73.16205868267092) bank83023 +83024 POINT(40.837334437578896 -74.29865142011899) bank83024 +83025 POINT(40.63419939195827 -74.70169787116158) bank83025 +83026 POINT(41.66198820314811 -74.80994362307577) bank83026 +83027 POINT(41.2835237470258 -74.42748091547092) bank83027 +83028 POINT(41.58646655708088 -73.60670100014109) bank83028 +83029 POINT(41.20926455761427 -73.83387183485482) bank83029 +83030 POINT(40.088096074832585 -73.26357331444494) bank83030 +83031 POINT(41.29229023470238 -73.50837354411779) bank83031 +83032 POINT(39.888322327039134 -74.76436391075744) bank83032 +83033 POINT(39.88404621860126 -74.9695685743939) bank83033 +83034 POINT(40.281273830993314 -73.17776771464217) bank83034 +83035 POINT(40.3225950915175 -74.2492074160144) bank83035 +83036 POINT(41.26708046735334 -74.02632829103169) bank83036 +83037 POINT(41.20868510524023 -73.21790598768506) bank83037 +83038 POINT(41.53151210006105 -73.59541695294567) bank83038 +83039 POINT(39.75669814283463 -74.40604134399996) bank83039 +83040 POINT(41.28102907438058 -73.82149568264417) bank83040 +83041 POINT(41.56914003200091 -73.70102071527508) bank83041 +83042 POINT(40.41653936955092 -73.09286530966598) bank83042 +83043 POINT(39.85598319632019 -73.80092805149357) bank83043 +83044 POINT(40.23486359637967 -73.44287446002866) bank83044 +83045 POINT(41.06313631929821 -73.12809221695898) bank83045 +83046 POINT(41.50524209538261 -74.6490348085112) bank83046 +83047 POINT(40.303430017657305 -74.37374814859724) bank83047 +83048 POINT(41.68894880667252 -74.66345619761238) bank83048 +83049 POINT(40.08507535484374 -74.64863852443337) bank83049 +83050 POINT(41.011413648082126 -73.96628691599122) bank83050 +83051 POINT(41.58361739991378 -74.51555146852373) bank83051 +83052 POINT(41.212584337488416 -73.79027156291318) bank83052 +83053 POINT(40.991590425359185 -74.91682495021979) bank83053 +83054 POINT(39.722111902782196 -73.09730335949669) bank83054 +83055 POINT(40.85722696770388 -74.88157489251248) bank83055 +83056 POINT(39.96951678966593 -73.16420120047043) bank83056 +83057 POINT(41.127638758494655 -73.87872105905794) bank83057 +83058 POINT(41.2288136468747 -74.42625847970763) bank83058 +83059 POINT(40.67567669363193 -73.8073082204575) bank83059 +83060 POINT(41.053131640881475 -74.90781794654293) bank83060 +83061 POINT(40.15253363191606 -74.7037877698531) bank83061 +83062 POINT(40.36778507914178 -73.21304403661061) bank83062 +83063 POINT(41.487820731080994 -74.54148401714502) bank83063 +83064 POINT(41.6361391222006 -73.96654709038786) bank83064 +83065 POINT(41.48497173717389 -74.4945226201523) bank83065 +83066 POINT(41.21072429436831 -74.2908443007317) bank83066 +83067 POINT(41.26710147391813 -74.3590245856134) bank83067 +83068 POINT(40.463015850752726 -74.75633031878408) bank83068 +83069 POINT(41.40876438882648 -74.16713156852624) bank83069 +83070 POINT(40.899874862050204 -73.44237204270785) bank83070 +83071 POINT(40.272390503048804 -75.00099368598835) bank83071 +83072 POINT(41.42806465146321 -73.24333408191676) bank83072 +83073 POINT(41.42539979864515 -74.29317171226818) bank83073 +83074 POINT(41.70028394373876 -74.4041917010958) bank83074 +83075 POINT(41.21457424596032 -73.38990953420698) bank83075 +83076 POINT(39.90864379232564 -74.5115075065225) bank83076 +83077 POINT(40.76728731383204 -73.56885968768617) bank83077 +83078 POINT(40.8341875933831 -73.50943914483017) bank83078 +83079 POINT(40.78170740451195 -73.86762134020813) bank83079 +83080 POINT(41.018187756767624 -74.57217996623528) bank83080 +83081 POINT(41.18188333707518 -73.20621140717795) bank83081 +83082 POINT(40.35008717995589 -74.59621031517106) bank83082 +83083 POINT(40.37294537220472 -74.80149925390742) bank83083 +83084 POINT(41.671220937412535 -73.67611089683204) bank83084 +83085 POINT(40.960293033102054 -74.18615265480709) bank83085 +83086 POINT(39.71963014444627 -74.42695340480898) bank83086 +83087 POINT(39.79426749719071 -74.93859755252498) bank83087 +83088 POINT(41.32672220766646 -74.13070200694722) bank83088 +83089 POINT(41.01613933626899 -74.18537869075311) bank83089 +83090 POINT(40.393618945842306 -74.97923899647705) bank83090 +83091 POINT(40.71600767255424 -74.32447015042368) bank83091 +83092 POINT(41.25594285692467 -74.33644752427128) bank83092 +83093 POINT(41.4874678452144 -73.12027706169394) bank83093 +83094 POINT(41.65786071242215 -74.71454571705632) bank83094 +83095 POINT(40.472202944008195 -73.30846244931591) bank83095 +83096 POINT(40.42895169199954 -73.3573974789852) bank83096 +83097 POINT(41.08186261252315 -73.26681661671006) bank83097 +83098 POINT(41.51273952503125 -73.48379353297223) bank83098 +83099 POINT(40.36757439687545 -73.17928637279407) bank83099 +83100 POINT(39.728137669525736 -74.26577064106799) bank83100 +83101 POINT(40.263858264937895 -73.58488488776857) bank83101 +83102 POINT(39.779336840959665 -74.95545458446138) bank83102 +83103 POINT(39.84543072555938 -74.05923931422508) bank83103 +83104 POINT(41.19893166848405 -74.36746542187278) bank83104 +83105 POINT(41.51090505206106 -74.54006826767034) bank83105 +83106 POINT(41.52454820981957 -74.54826081149474) bank83106 +83107 POINT(41.00861248803251 -73.29843685907244) bank83107 +83108 POINT(41.6660656581584 -74.03594585965132) bank83108 +83109 POINT(41.109541469764046 -74.91992995813058) bank83109 +83110 POINT(41.34999288553333 -73.93531617465341) bank83110 +83111 POINT(40.95882507806099 -73.26956323906121) bank83111 +83112 POINT(40.75728933870228 -74.70106774410719) bank83112 +83113 POINT(40.2846139295698 -74.10458474593412) bank83113 +83114 POINT(40.46984310187925 -74.45116785637765) bank83114 +83115 POINT(41.63089533706342 -73.65964240671423) bank83115 +83116 POINT(40.38963081317211 -73.73576668771919) bank83116 +83117 POINT(40.895479043841924 -74.40955295480079) bank83117 +83118 POINT(40.170740801481614 -74.96625826421376) bank83118 +83119 POINT(41.28961870461391 -73.66174711168568) bank83119 +83120 POINT(40.163903953559256 -73.11319291307636) bank83120 +83121 POINT(40.55839165194122 -74.51363069368651) bank83121 +83122 POINT(39.76545247577234 -73.555361425254) bank83122 +83123 POINT(40.471823858543786 -74.34169822713369) bank83123 +83124 POINT(41.45480582057009 -74.6827970996209) bank83124 +83125 POINT(41.07902489378039 -73.2014239563627) bank83125 +83126 POINT(40.69002424070872 -74.32712807313216) bank83126 +83127 POINT(39.95154055106078 -74.39346462230328) bank83127 +83128 POINT(40.86782896536855 -73.80045535304042) bank83128 +83129 POINT(39.91765338442729 -74.33015117245364) bank83129 +83130 POINT(41.028503259154355 -74.15565924354938) bank83130 +83131 POINT(39.807638769671286 -73.87906291885945) bank83131 +83132 POINT(40.073144624080875 -73.91255620284406) bank83132 +83133 POINT(39.93802749202205 -73.03396144314107) bank83133 +83134 POINT(40.58292418330922 -73.40927440850294) bank83134 +83135 POINT(40.6466204908912 -74.50319268321823) bank83135 +83136 POINT(40.86825428033966 -74.71396707360458) bank83136 +83137 POINT(40.24091695092236 -73.52557256475504) bank83137 +83138 POINT(40.09258862178361 -74.13102679841128) bank83138 +83139 POINT(41.41368878591784 -74.73148196348586) bank83139 +83140 POINT(41.475416995260524 -73.20860996210065) bank83140 +83141 POINT(39.949874834164575 -73.18404095290657) bank83141 +83142 POINT(39.92415239329609 -74.9033548280526) bank83142 +83143 POINT(41.32520793579542 -74.03958961697535) bank83143 +83144 POINT(41.26933417072572 -74.60101882286781) bank83144 +83145 POINT(40.1342735226252 -73.67199107823983) bank83145 +83146 POINT(40.69506415426064 -74.87316049180289) bank83146 +83147 POINT(40.45740001920912 -74.97175704927075) bank83147 +83148 POINT(41.17953014894307 -74.55655805643855) bank83148 +83149 POINT(39.95082326895672 -73.24759328975608) bank83149 +83150 POINT(39.80566271774914 -73.62180835618419) bank83150 +83151 POINT(40.59279763947309 -73.26233152376506) bank83151 +83152 POINT(39.99182243814822 -74.43093841947218) bank83152 +83153 POINT(40.69477344957125 -74.79291633417068) bank83153 +83154 POINT(41.28315144554821 -74.17558632254237) bank83154 +83155 POINT(39.89233312936796 -74.29079326382622) bank83155 +83156 POINT(40.68902176888625 -74.32331941863464) bank83156 +83157 POINT(41.432690514963056 -73.45978031190799) bank83157 +83158 POINT(40.78062560510345 -74.11318375285809) bank83158 +83159 POINT(40.74867164479861 -74.9760185742295) bank83159 +83160 POINT(40.24938164312947 -74.32199411846756) bank83160 +83161 POINT(41.55914182282663 -73.64059543623632) bank83161 +83162 POINT(41.436762647639256 -73.21501331232179) bank83162 +83163 POINT(41.52693873388969 -73.15578248757753) bank83163 +83164 POINT(40.549422078711366 -74.25462269734093) bank83164 +83165 POINT(40.0178523731422 -74.12772133747382) bank83165 +83166 POINT(39.77146293194095 -74.87741097861463) bank83166 +83167 POINT(39.94343257077853 -74.27026241238153) bank83167 +83168 POINT(41.4058148374553 -74.9337920818552) bank83168 +83169 POINT(40.73330546511284 -73.84214890757124) bank83169 +83170 POINT(40.96212301762351 -74.90597163494432) bank83170 +83171 POINT(39.89046752689362 -73.95105669356823) bank83171 +83172 POINT(41.67157527148781 -74.65230790518903) bank83172 +83173 POINT(40.039702151423235 -73.8163692334272) bank83173 +83174 POINT(41.53531509003379 -74.63955381867626) bank83174 +83175 POINT(40.64024153161899 -73.72754131639728) bank83175 +83176 POINT(41.09965952397786 -73.1737480087441) bank83176 +83177 POINT(40.419418769762196 -73.46320567803033) bank83177 +83178 POINT(41.06492778424038 -74.95796027658798) bank83178 +83179 POINT(40.08166981374276 -73.2625538488274) bank83179 +83180 POINT(41.687504425221135 -73.47448194728644) bank83180 +83181 POINT(40.71594877329265 -73.38158847896693) bank83181 +83182 POINT(40.6092076119545 -74.123016847974) bank83182 +83183 POINT(41.56156054551668 -73.89262645355758) bank83183 +83184 POINT(40.51404387631838 -74.76995291258663) bank83184 +83185 POINT(40.03458174614151 -74.4352652010395) bank83185 +83186 POINT(40.401888932069234 -74.61631434572021) bank83186 +83187 POINT(40.006700443644554 -73.34975816399431) bank83187 +83188 POINT(40.105318886646984 -73.19962186738653) bank83188 +83189 POINT(39.748330725705806 -74.3052617112715) bank83189 +83190 POINT(41.420614040408566 -73.42745476751472) bank83190 +83191 POINT(40.737110229790176 -73.16589960387678) bank83191 +83192 POINT(41.00855072394548 -73.86571147780936) bank83192 +83193 POINT(40.06023414890076 -73.68311062419842) bank83193 +83194 POINT(41.135198614278764 -74.57757926956945) bank83194 +83195 POINT(40.394000599399156 -74.0426601514142) bank83195 +83196 POINT(39.984983011838715 -73.67220265188328) bank83196 +83197 POINT(40.362106148884514 -74.28094267571588) bank83197 +83198 POINT(41.00577579917321 -73.6621017440381) bank83198 +83199 POINT(40.273564660109884 -74.33734205775059) bank83199 +83200 POINT(40.90358223620866 -74.4636957079334) bank83200 +83201 POINT(40.071372497306825 -74.73918503804674) bank83201 +83202 POINT(40.13880862224125 -73.32596487237886) bank83202 +83203 POINT(41.504408132082524 -73.73613653163616) bank83203 +83204 POINT(41.50143804497854 -74.74807338473093) bank83204 +83205 POINT(41.21449261425854 -73.15538827181472) bank83205 +83206 POINT(41.461553364768584 -73.59583771855576) bank83206 +83207 POINT(41.25626402779195 -74.55109677286843) bank83207 +83208 POINT(41.654093906915115 -73.06440659462072) bank83208 +83209 POINT(40.87263939412338 -74.71460192160727) bank83209 +83210 POINT(40.40915601592459 -73.97146012060533) bank83210 +83211 POINT(41.41696092330523 -73.34988255366179) bank83211 +83212 POINT(39.933055323904696 -73.75687256095038) bank83212 +83213 POINT(41.366636788094624 -73.32917123310595) bank83213 +83214 POINT(40.857206584993065 -73.92808848842957) bank83214 +83215 POINT(40.04253279520928 -73.8623675814998) bank83215 +83216 POINT(41.53341920628932 -73.1098356332568) bank83216 +83217 POINT(40.64151414989121 -73.17969317880407) bank83217 +83218 POINT(41.69384907903717 -74.34150640929107) bank83218 +83219 POINT(40.81998076581327 -73.1669549701763) bank83219 +83220 POINT(40.57113347941504 -73.51562480019575) bank83220 +83221 POINT(40.664629153913495 -74.31196315327196) bank83221 +83222 POINT(41.19480785984194 -73.0694730626693) bank83222 +83223 POINT(41.409035986828926 -73.44315612594751) bank83223 +83224 POINT(39.90656451030252 -73.0263704822925) bank83224 +83225 POINT(39.75229608252042 -73.71411748670776) bank83225 +83226 POINT(41.29280233564195 -73.09722781665513) bank83226 +83227 POINT(40.197175641477656 -73.62561163419802) bank83227 +83228 POINT(41.67733050698955 -74.41981296006146) bank83228 +83229 POINT(41.19168251184588 -74.67434169788855) bank83229 +83230 POINT(39.97557891447531 -74.04895010315408) bank83230 +83231 POINT(40.58110338011654 -74.59168579039529) bank83231 +83232 POINT(40.32350698720379 -74.06424146534756) bank83232 +83233 POINT(39.73815640536883 -74.86256168715653) bank83233 +83234 POINT(41.12536742098216 -74.45595611851525) bank83234 +83235 POINT(40.50418391628775 -74.32666988671596) bank83235 +83236 POINT(39.9393373263429 -73.91644751292934) bank83236 +83237 POINT(40.72402003570414 -73.43605524750097) bank83237 +83238 POINT(40.802483784346244 -74.76423640105232) bank83238 +83239 POINT(40.21122617291526 -74.9385796256556) bank83239 +83240 POINT(40.77296563443977 -73.07323587262411) bank83240 +83241 POINT(40.86214793953531 -73.64634926478654) bank83241 +83242 POINT(40.17320429819705 -74.80540680823529) bank83242 +83243 POINT(39.93795355559666 -73.77573892258691) bank83243 +83244 POINT(41.0537053726912 -74.36423007285204) bank83244 +83245 POINT(40.03630759333152 -73.50913073034367) bank83245 +83246 POINT(40.71757603784652 -73.36944737185446) bank83246 +83247 POINT(40.76349070125029 -74.06552761473127) bank83247 +83248 POINT(41.37621918563704 -73.95553942234835) bank83248 +83249 POINT(40.65666010258899 -74.42610115044263) bank83249 +83250 POINT(40.62567942787236 -73.90372681127022) bank83250 +83251 POINT(39.8125423520904 -73.63495879576983) bank83251 +83252 POINT(40.7663071594154 -73.08554190052818) bank83252 +83253 POINT(41.017295296502844 -74.02975919051559) bank83253 +83254 POINT(40.85650639866271 -74.13713106024106) bank83254 +83255 POINT(39.76626248952011 -73.42088301871114) bank83255 +83256 POINT(40.36688650979051 -74.24369933065931) bank83256 +83257 POINT(40.40812664690255 -73.12210366164037) bank83257 +83258 POINT(40.65491178478686 -73.29880946463705) bank83258 +83259 POINT(40.00012013408922 -74.8254656221847) bank83259 +83260 POINT(41.06740318179362 -74.87718155456378) bank83260 +83261 POINT(40.08501983080477 -73.81895227937284) bank83261 +83262 POINT(41.579582177515164 -74.94856462251761) bank83262 +83263 POINT(41.3597448789603 -73.33557475043331) bank83263 +83264 POINT(41.25932263463822 -74.18044801517657) bank83264 +83265 POINT(41.332570507419774 -73.64601391212335) bank83265 +83266 POINT(41.34264281105753 -74.47127009483883) bank83266 +83267 POINT(39.98394359928421 -74.63202527227617) bank83267 +83268 POINT(40.09208392940637 -74.1690137543221) bank83268 +83269 POINT(40.85828461829468 -73.57840348469051) bank83269 +83270 POINT(41.35743653921594 -73.31008342598462) bank83270 +83271 POINT(40.314192332699605 -74.4682555881301) bank83271 +83272 POINT(40.975581164875 -74.71819855371457) bank83272 +83273 POINT(40.06992471292101 -73.65475256739714) bank83273 +83274 POINT(41.56611449006879 -73.04994117021472) bank83274 +83275 POINT(40.11969879451875 -73.82809225666684) bank83275 +83276 POINT(40.121619329568595 -74.08670961679974) bank83276 +83277 POINT(40.62274124308246 -74.15393872060015) bank83277 +83278 POINT(41.5130762226644 -73.48836029559025) bank83278 +83279 POINT(39.96807742508527 -74.62459682720926) bank83279 +83280 POINT(40.98876168239305 -74.99099374235401) bank83280 +83281 POINT(41.507670692301716 -74.20603699901896) bank83281 +83282 POINT(40.50413960323846 -73.53189203908418) bank83282 +83283 POINT(39.82673102527121 -73.12520143988743) bank83283 +83284 POINT(40.88587751690725 -73.96347782329138) bank83284 +83285 POINT(41.03414094795281 -74.88353499608317) bank83285 +83286 POINT(40.85566363291788 -73.03413930959191) bank83286 +83287 POINT(40.505156353781636 -73.74914941635899) bank83287 +83288 POINT(40.37837872664641 -74.11984394802474) bank83288 +83289 POINT(41.1824482137976 -73.26262940751766) bank83289 +83290 POINT(40.94146861029569 -74.40766793886125) bank83290 +83291 POINT(40.146103627910975 -73.85131766434236) bank83291 +83292 POINT(40.613996817339995 -73.21083494865552) bank83292 +83293 POINT(41.05716377864969 -74.29329245427658) bank83293 +83294 POINT(40.42719433909737 -74.55735739582512) bank83294 +83295 POINT(40.418841519448186 -73.36747334482676) bank83295 +83296 POINT(41.4768709069026 -74.08880568289855) bank83296 +83297 POINT(39.739799434767015 -73.55725486559543) bank83297 +83298 POINT(39.925059018683605 -74.25182223317718) bank83298 +83299 POINT(40.19154593982038 -73.43806741541874) bank83299 +83300 POINT(41.46994411356583 -74.82574487594655) bank83300 +83301 POINT(41.53321108780763 -73.35584670527312) bank83301 +83302 POINT(40.82661564977639 -74.45320975976207) bank83302 +83303 POINT(41.438893991701164 -73.72878903663805) bank83303 +83304 POINT(41.28586326999289 -74.59115347914434) bank83304 +83305 POINT(40.62320465969421 -73.78815210571325) bank83305 +83306 POINT(39.96702878215432 -74.85610529331733) bank83306 +83307 POINT(40.6612569731871 -75.00389354963248) bank83307 +83308 POINT(40.46529980060055 -73.62542434234867) bank83308 +83309 POINT(41.248103450499684 -74.81962816356528) bank83309 +83310 POINT(41.40746130014802 -73.31670201466429) bank83310 +83311 POINT(41.31742771509151 -73.62849107167068) bank83311 +83312 POINT(39.949131001812454 -74.05567254671594) bank83312 +83313 POINT(40.398733017186004 -74.06470608450137) bank83313 +83314 POINT(40.87000179565961 -74.67265619129996) bank83314 +83315 POINT(41.70857464152207 -74.06592419325153) bank83315 +83316 POINT(40.435368833871905 -74.80556477484626) bank83316 +83317 POINT(41.66852614065847 -73.06583635604568) bank83317 +83318 POINT(41.60495609691191 -73.08494588937248) bank83318 +83319 POINT(40.08749223507557 -74.41504431507234) bank83319 +83320 POINT(40.24103885710418 -74.25844006260895) bank83320 +83321 POINT(40.10398952229169 -74.9366557897121) bank83321 +83322 POINT(40.854915785632485 -74.34046905211517) bank83322 +83323 POINT(40.443946500761726 -74.45498573757864) bank83323 +83324 POINT(41.02171493541751 -73.66902606068932) bank83324 +83325 POINT(41.3718109625485 -73.83995611796395) bank83325 +83326 POINT(39.80606319913609 -74.10777375221258) bank83326 +83327 POINT(41.474139220221346 -74.6960834534173) bank83327 +83328 POINT(39.89688913589374 -73.6233106101541) bank83328 +83329 POINT(40.75862215620728 -74.59344332436565) bank83329 +83330 POINT(40.47384313547117 -74.76587708437302) bank83330 +83331 POINT(41.49757537742962 -73.57169117621329) bank83331 +83332 POINT(40.82098975002842 -73.23046588780605) bank83332 +83333 POINT(41.67485580359035 -74.69749333194096) bank83333 +83334 POINT(40.771806484032844 -73.1462112799661) bank83334 +83335 POINT(40.509460413783856 -73.2671048339407) bank83335 +83336 POINT(40.250137250709216 -74.34027576291288) bank83336 +83337 POINT(40.77915471234205 -74.50165206221673) bank83337 +83338 POINT(40.48870030607812 -74.83880391992905) bank83338 +83339 POINT(40.53018999077282 -73.65032368975658) bank83339 +83340 POINT(41.41650227359805 -74.89222696504001) bank83340 +83341 POINT(40.082173469609565 -73.99349523836169) bank83341 +83342 POINT(40.581279080471774 -73.19137472353937) bank83342 +83343 POINT(39.925753527733285 -73.49790681136146) bank83343 +83344 POINT(40.11775978680506 -73.60793646626894) bank83344 +83345 POINT(41.06199481562719 -73.88668592654638) bank83345 +83346 POINT(41.43610773527021 -74.30084272116892) bank83346 +83347 POINT(39.75685333203838 -74.92711776733269) bank83347 +83348 POINT(40.35719815610346 -73.47084451704795) bank83348 +83349 POINT(40.21142847800154 -73.23469891378237) bank83349 +83350 POINT(41.31568607397982 -74.31672722256049) bank83350 +83351 POINT(40.324702334283124 -73.92792961633491) bank83351 +83352 POINT(41.47593112409605 -74.51037605849021) bank83352 +83353 POINT(41.620261931391205 -74.95061296677869) bank83353 +83354 POINT(40.49765306575796 -74.05956563280786) bank83354 +83355 POINT(39.78998802548548 -73.47397918533596) bank83355 +83356 POINT(41.4103989250358 -73.33058313967882) bank83356 +83357 POINT(40.122606385014954 -74.25734761336754) bank83357 +83358 POINT(40.341319299063116 -73.15437762885244) bank83358 +83359 POINT(40.839786517854975 -73.43492915398812) bank83359 +83360 POINT(39.82485050556553 -73.66534344476031) bank83360 +83361 POINT(41.46177303247637 -74.54168302827716) bank83361 +83362 POINT(41.44720197558985 -73.70345207960821) bank83362 +83363 POINT(39.960980837643454 -74.71647595561338) bank83363 +83364 POINT(41.55445710206428 -74.00645740366942) bank83364 +83365 POINT(40.424277705534124 -73.45861005913547) bank83365 +83366 POINT(40.84588262753508 -74.49043641795821) bank83366 +83367 POINT(40.55455052870769 -73.86154682866449) bank83367 +83368 POINT(41.47412693196197 -74.75500032078001) bank83368 +83369 POINT(41.267003560898395 -74.50060078078103) bank83369 +83370 POINT(40.361181416326794 -73.11493459578642) bank83370 +83371 POINT(40.744231155879625 -73.41115001391108) bank83371 +83372 POINT(39.82151408231994 -73.90894474188039) bank83372 +83373 POINT(39.72183801086267 -73.66998285930308) bank83373 +83374 POINT(40.081489999441786 -73.95609849996312) bank83374 +83375 POINT(40.75422546029068 -74.94488119508048) bank83375 +83376 POINT(41.333927224139394 -73.43852575651245) bank83376 +83377 POINT(40.759377363304246 -74.90376124871037) bank83377 +83378 POINT(41.63253301807037 -73.96779354243031) bank83378 +83379 POINT(41.197017878286616 -74.54933910703335) bank83379 +83380 POINT(40.83592155540653 -74.657351695995) bank83380 +83381 POINT(41.578500279435055 -74.87084802995766) bank83381 +83382 POINT(40.57440985616941 -74.09882237353796) bank83382 +83383 POINT(41.27394660907645 -73.51022370628155) bank83383 +83384 POINT(39.9899903949249 -73.62787193778756) bank83384 +83385 POINT(40.77488915812597 -73.03193376450332) bank83385 +83386 POINT(41.19738285129167 -73.9127087475101) bank83386 +83387 POINT(41.560323857678505 -74.9922888647445) bank83387 +83388 POINT(39.78778887434207 -73.72018027098221) bank83388 +83389 POINT(40.129517583280375 -73.44777484001278) bank83389 +83390 POINT(39.80656106964929 -74.60087829928312) bank83390 +83391 POINT(40.25311637980388 -73.51469407569489) bank83391 +83392 POINT(40.93569341413899 -73.59436506558268) bank83392 +83393 POINT(41.13892114799979 -74.17055098823322) bank83393 +83394 POINT(40.110804615146186 -74.15313911640928) bank83394 +83395 POINT(41.49907180646469 -74.33426518047003) bank83395 +83396 POINT(40.35683139335713 -73.38816302924856) bank83396 +83397 POINT(41.685233712606454 -74.40575321047034) bank83397 +83398 POINT(41.33937970649706 -74.32091989818181) bank83398 +83399 POINT(39.95403533655101 -73.80979944291032) bank83399 +83400 POINT(41.14941833130042 -74.18351261173976) bank83400 +83401 POINT(41.71030304905461 -74.1873771848208) bank83401 +83402 POINT(40.43597494833811 -73.0791813613373) bank83402 +83403 POINT(41.188594750156696 -74.896356616907) bank83403 +83404 POINT(40.70099916690084 -74.6720956119252) bank83404 +83405 POINT(41.149254455937324 -74.21462547239459) bank83405 +83406 POINT(39.78843811753523 -74.09903588073327) bank83406 +83407 POINT(41.17271811346558 -73.94913634414131) bank83407 +83408 POINT(41.45017318471505 -74.93631241913907) bank83408 +83409 POINT(39.99851205177896 -73.4836122121311) bank83409 +83410 POINT(40.3625545865586 -73.1232513167746) bank83410 +83411 POINT(40.427733595771684 -73.68657165921718) bank83411 +83412 POINT(40.84609044255361 -74.81806455157647) bank83412 +83413 POINT(40.04100043968726 -73.56800184716977) bank83413 +83414 POINT(41.69560143890078 -73.69369220757511) bank83414 +83415 POINT(41.145446644072805 -74.33929587258963) bank83415 +83416 POINT(40.85273449836044 -73.6976375876454) bank83416 +83417 POINT(41.07114006504037 -74.17849941893988) bank83417 +83418 POINT(41.57196329619357 -73.7861410198448) bank83418 +83419 POINT(40.05296554871992 -74.46616904824023) bank83419 +83420 POINT(40.8718403194474 -73.10257397302185) bank83420 +83421 POINT(40.78395933057511 -74.71603578200269) bank83421 +83422 POINT(39.95150902332893 -74.56776565466123) bank83422 +83423 POINT(40.323339018244276 -73.57377680404903) bank83423 +83424 POINT(40.65657450091906 -74.64629426764775) bank83424 +83425 POINT(39.790669420369674 -74.32109721575895) bank83425 +83426 POINT(40.62949174175275 -73.8208018092207) bank83426 +83427 POINT(41.033560062771315 -73.83232798365526) bank83427 +83428 POINT(39.884791768022225 -73.97848473034672) bank83428 +83429 POINT(40.58269448544986 -74.27017942071242) bank83429 +83430 POINT(41.14337988586718 -74.43782802519053) bank83430 +83431 POINT(40.78316894746298 -74.45992950259644) bank83431 +83432 POINT(40.84445815816152 -73.58836471684155) bank83432 +83433 POINT(40.24257042275408 -73.242597607969) bank83433 +83434 POINT(39.84918806643715 -73.47962680661412) bank83434 +83435 POINT(40.781581838933214 -73.26379666845536) bank83435 +83436 POINT(40.20842835113021 -74.31226878346399) bank83436 +83437 POINT(41.06838612344689 -74.17086060001127) bank83437 +83438 POINT(39.978259014177965 -73.15312127481242) bank83438 +83439 POINT(41.3592998949361 -73.77494775779789) bank83439 +83440 POINT(41.47437936594654 -73.42093535202618) bank83440 +83441 POINT(40.280084572075836 -73.35238404148124) bank83441 +83442 POINT(40.49927556996825 -73.94010520564117) bank83442 +83443 POINT(40.688095193821894 -74.20837020407) bank83443 +83444 POINT(40.829569873546866 -73.22876772534377) bank83444 +83445 POINT(40.25515736048424 -73.88009448771234) bank83445 +83446 POINT(41.35466179745394 -74.88066927722036) bank83446 +83447 POINT(41.014479628372726 -73.15085991293466) bank83447 +83448 POINT(41.156808188082046 -73.13449090687047) bank83448 +83449 POINT(41.55595250828633 -74.19236972450925) bank83449 +83450 POINT(39.98858372954543 -73.85461793976562) bank83450 +83451 POINT(40.47633960047992 -73.73321845416339) bank83451 +83452 POINT(40.23118838456726 -74.99166788271197) bank83452 +83453 POINT(40.22261490332284 -73.50903318261753) bank83453 +83454 POINT(41.16086479055113 -74.15793490374706) bank83454 +83455 POINT(40.159032385906585 -74.7064707630691) bank83455 +83456 POINT(41.28891490626858 -74.08209679678558) bank83456 +83457 POINT(40.38645127764791 -73.11427985606473) bank83457 +83458 POINT(41.595422205961505 -73.06906804385063) bank83458 +83459 POINT(40.96227132808258 -74.27332565299119) bank83459 +83460 POINT(39.75110899918544 -73.20005768935172) bank83460 +83461 POINT(40.765411151086845 -73.78507055492626) bank83461 +83462 POINT(40.01758021088714 -73.18848430164377) bank83462 +83463 POINT(40.07830089158891 -73.7049893429156) bank83463 +83464 POINT(40.03881684108777 -74.06888767000775) bank83464 +83465 POINT(41.49701643979648 -73.54541824592589) bank83465 +83466 POINT(40.408337866138034 -74.08366999326151) bank83466 +83467 POINT(39.85032235794227 -74.47948327508138) bank83467 +83468 POINT(41.427907496534516 -73.05151099588795) bank83468 +83469 POINT(41.00562170262061 -73.46305215255984) bank83469 +83470 POINT(41.69495193911708 -74.53108334456493) bank83470 +83471 POINT(40.14652660837616 -74.71514464351199) bank83471 +83472 POINT(40.5811468303212 -74.8086638042599) bank83472 +83473 POINT(40.79103641753175 -73.24129927810743) bank83473 +83474 POINT(40.87648491789943 -73.2505027116886) bank83474 +83475 POINT(40.120358337783756 -73.057990294038) bank83475 +83476 POINT(41.68696488118888 -74.43683617179808) bank83476 +83477 POINT(39.95933919629735 -74.44634359609199) bank83477 +83478 POINT(40.7879296410063 -73.19901540763924) bank83478 +83479 POINT(40.02499098262671 -73.49014663069892) bank83479 +83480 POINT(41.40142223782663 -73.77080162667127) bank83480 +83481 POINT(41.5962119054274 -74.97868879342718) bank83481 +83482 POINT(41.32106129350581 -73.42839140187806) bank83482 +83483 POINT(39.95695226103734 -73.42205929517499) bank83483 +83484 POINT(41.620568818009914 -73.8006771032085) bank83484 +83485 POINT(39.776392301804314 -73.17998081008751) bank83485 +83486 POINT(40.19007225484106 -74.27919447131387) bank83486 +83487 POINT(40.24221416858953 -74.66864162367484) bank83487 +83488 POINT(39.77513259741057 -74.11325332295394) bank83488 +83489 POINT(40.72542207773342 -73.74286511029908) bank83489 +83490 POINT(39.87967053416211 -74.3446458713747) bank83490 +83491 POINT(41.692063531522564 -73.07985653556851) bank83491 +83492 POINT(41.16079076839384 -73.83073671403163) bank83492 +83493 POINT(41.11619135742481 -73.82708656078142) bank83493 +83494 POINT(41.265597807465625 -73.67891148211311) bank83494 +83495 POINT(41.22671911747248 -74.9429087083972) bank83495 +83496 POINT(41.44560783353652 -74.30524687084997) bank83496 +83497 POINT(40.10180891209481 -74.97092295674065) bank83497 +83498 POINT(41.47742597080801 -74.54140664298303) bank83498 +83499 POINT(39.85510185315129 -74.61518035030552) bank83499 +83500 POINT(39.81631874854644 -73.28454270642567) bank83500 +83501 POINT(41.16157726830378 -73.73710232073446) bank83501 +83502 POINT(40.680609374924956 -74.78024863309426) bank83502 +83503 POINT(40.0679667751932 -73.6942487205237) bank83503 +83504 POINT(41.06448729171602 -73.68756423676338) bank83504 +83505 POINT(40.679397682566034 -74.62574695466786) bank83505 +83506 POINT(40.40484002065645 -74.24807414581836) bank83506 +83507 POINT(41.25891958868332 -73.61902847000181) bank83507 +83508 POINT(40.65962827532967 -74.83002214665784) bank83508 +83509 POINT(40.15657249699611 -74.87741353391823) bank83509 +83510 POINT(41.4003135856997 -74.53756975342664) bank83510 +83511 POINT(39.722958176860544 -73.04466669390663) bank83511 +83512 POINT(41.70596577306645 -73.41055471712964) bank83512 +83513 POINT(41.65006947607561 -73.2074284393369) bank83513 +83514 POINT(39.917046561553285 -74.81602476621701) bank83514 +83515 POINT(41.00793488074226 -73.26469486059321) bank83515 +83516 POINT(40.95720239169554 -74.2377904722326) bank83516 +83517 POINT(40.642502012089544 -73.37602763563096) bank83517 +83518 POINT(39.877462073188596 -74.57966803892755) bank83518 +83519 POINT(39.783266285295895 -74.00587886487149) bank83519 +83520 POINT(40.22943849389816 -74.91230819108227) bank83520 +83521 POINT(41.23408587748144 -74.62251438434475) bank83521 +83522 POINT(39.9463474867181 -74.43742581303684) bank83522 +83523 POINT(41.101410299087284 -74.54325477083992) bank83523 +83524 POINT(39.775431976135245 -73.43170544911402) bank83524 +83525 POINT(40.92284065165297 -74.95107890112857) bank83525 +83526 POINT(41.301810079292096 -73.92540673813318) bank83526 +83527 POINT(40.35544190363115 -73.92120282480096) bank83527 +83528 POINT(41.116206838070646 -73.73666107709477) bank83528 +83529 POINT(39.78848649399352 -74.32436910294852) bank83529 +83530 POINT(41.4016337654938 -74.1961936273987) bank83530 +83531 POINT(40.754106142489185 -74.03586856206644) bank83531 +83532 POINT(40.39543441197946 -73.05695489110471) bank83532 +83533 POINT(40.2689802826494 -73.86823137115158) bank83533 +83534 POINT(39.97598418219086 -73.88399187657026) bank83534 +83535 POINT(41.507545058619804 -74.8235581222471) bank83535 +83536 POINT(39.852264509398985 -73.62236027853936) bank83536 +83537 POINT(40.18698968207723 -73.83652640793773) bank83537 +83538 POINT(40.25191336755659 -73.41385111259609) bank83538 +83539 POINT(41.30484013537275 -73.98332010983685) bank83539 +83540 POINT(39.94462376178258 -74.69626642105686) bank83540 +83541 POINT(39.952727660247575 -74.88780121492081) bank83541 +83542 POINT(41.588646080550895 -74.34804495956699) bank83542 +83543 POINT(40.860326126163976 -73.1464177392641) bank83543 +83544 POINT(41.4317348662144 -73.76919416712174) bank83544 +83545 POINT(39.744536330238766 -74.67422450494688) bank83545 +83546 POINT(40.86600271143886 -74.40499346341507) bank83546 +83547 POINT(39.82126912455045 -73.66320356414766) bank83547 +83548 POINT(41.58880663498469 -73.64360649419412) bank83548 +83549 POINT(40.033479272791894 -73.62592807579205) bank83549 +83550 POINT(41.271256845828006 -74.7519125653771) bank83550 +83551 POINT(40.287090959204974 -74.30569991806915) bank83551 +83552 POINT(40.94381093528301 -74.06783375868683) bank83552 +83553 POINT(40.46017998520136 -73.97055830994604) bank83553 +83554 POINT(40.16723233596809 -73.03634914742702) bank83554 +83555 POINT(39.777751662122036 -74.04688249682039) bank83555 +83556 POINT(40.41811372312419 -74.59020420355414) bank83556 +83557 POINT(39.970724872667326 -74.18112471317147) bank83557 +83558 POINT(40.56679772622016 -74.60251532790748) bank83558 +83559 POINT(40.90219634296858 -74.13815257677476) bank83559 +83560 POINT(41.271667031567986 -73.44810923782151) bank83560 +83561 POINT(40.624110443928465 -73.31961951103145) bank83561 +83562 POINT(41.440670632001684 -73.36948991219495) bank83562 +83563 POINT(39.96887627702887 -74.68115897207491) bank83563 +83564 POINT(41.13365619453448 -74.95376675398045) bank83564 +83565 POINT(41.164218008066555 -73.75892690236327) bank83565 +83566 POINT(41.52903101201318 -74.6645871500892) bank83566 +83567 POINT(40.0427630558648 -74.48127842320045) bank83567 +83568 POINT(40.36535592434282 -73.06013522590607) bank83568 +83569 POINT(41.64182305833325 -73.54479712373265) bank83569 +83570 POINT(40.185432908517576 -74.7913234503528) bank83570 +83571 POINT(40.439056650230995 -74.4863830184389) bank83571 +83572 POINT(41.10877739823313 -73.60227412723016) bank83572 +83573 POINT(40.45724153996477 -74.58512809703059) bank83573 +83574 POINT(40.99255030278487 -73.12921578402144) bank83574 +83575 POINT(40.5851559909718 -74.70605546522616) bank83575 +83576 POINT(41.08470677856172 -74.78230263532129) bank83576 +83577 POINT(41.071399736723755 -73.46391652489537) bank83577 +83578 POINT(41.505598270675975 -74.18924499276844) bank83578 +83579 POINT(40.521107935549736 -73.24317229952268) bank83579 +83580 POINT(40.164359205071406 -74.25027504997551) bank83580 +83581 POINT(41.691613966801825 -73.9888222076804) bank83581 +83582 POINT(41.58389599801552 -74.42350254285306) bank83582 +83583 POINT(41.70045375726289 -73.3772453775097) bank83583 +83584 POINT(40.781826242878104 -73.63650977726091) bank83584 +83585 POINT(40.50790746363768 -73.09769737727157) bank83585 +83586 POINT(39.842053013025875 -74.8496411840876) bank83586 +83587 POINT(40.58277623917008 -73.41148810450095) bank83587 +83588 POINT(39.84469906612262 -73.9866724494503) bank83588 +83589 POINT(40.099358660576705 -73.60192788811743) bank83589 +83590 POINT(41.324826809913766 -73.68101053845203) bank83590 +83591 POINT(39.94073329556246 -73.3645764822379) bank83591 +83592 POINT(39.786965925526836 -73.66459407328927) bank83592 +83593 POINT(40.37059434212239 -74.42518799609579) bank83593 +83594 POINT(40.23776904312015 -74.67553897287253) bank83594 +83595 POINT(40.582618556470905 -74.05011488890162) bank83595 +83596 POINT(41.10100964926434 -73.87395821154699) bank83596 +83597 POINT(40.182100831419405 -73.96730285900257) bank83597 +83598 POINT(40.31606733196917 -74.345470516427) bank83598 +83599 POINT(40.072275760567805 -73.540551139975) bank83599 +83600 POINT(39.7132280818814 -74.33065051790558) bank83600 +83601 POINT(40.849176344707274 -73.83350753791514) bank83601 +83602 POINT(40.55935766206768 -74.76196989142873) bank83602 +83603 POINT(41.12146383752586 -73.8067982528397) bank83603 +83604 POINT(40.77847468727787 -73.1041109232183) bank83604 +83605 POINT(40.79544230507146 -73.07318253511013) bank83605 +83606 POINT(41.653252750541476 -74.84545974344934) bank83606 +83607 POINT(40.30711737562253 -74.65721649893722) bank83607 +83608 POINT(39.92673725422185 -74.63082404425883) bank83608 +83609 POINT(41.239099494143034 -74.39045846900882) bank83609 +83610 POINT(41.481911812182716 -73.40226267235987) bank83610 +83611 POINT(41.68778478733085 -74.17344239094155) bank83611 +83612 POINT(40.52996556488989 -73.94631523921966) bank83612 +83613 POINT(40.979777658482135 -73.52804544697112) bank83613 +83614 POINT(41.337902850620694 -73.3514549523464) bank83614 +83615 POINT(40.93391703537789 -73.40532115561864) bank83615 +83616 POINT(40.450710206015934 -73.57634312837399) bank83616 +83617 POINT(39.93889984490215 -74.980569385368) bank83617 +83618 POINT(40.319475819681784 -73.87718469433435) bank83618 +83619 POINT(40.271879601716236 -73.77313192412373) bank83619 +83620 POINT(40.37460550061948 -74.35918187626115) bank83620 +83621 POINT(39.81020257627323 -74.56999446219746) bank83621 +83622 POINT(40.460398597146465 -73.49920495715753) bank83622 +83623 POINT(39.86264908066983 -74.86856291973736) bank83623 +83624 POINT(41.54609544457395 -74.11698579635065) bank83624 +83625 POINT(40.3161423343344 -73.39721189959349) bank83625 +83626 POINT(41.14231411173372 -74.50290252881273) bank83626 +83627 POINT(40.26765961740008 -73.51163839712576) bank83627 +83628 POINT(41.361462284385475 -74.00234061827697) bank83628 +83629 POINT(41.3364054742377 -74.13147321564158) bank83629 +83630 POINT(40.33894377111867 -73.61772029478378) bank83630 +83631 POINT(40.4872177245422 -74.89728121818743) bank83631 +83632 POINT(41.67421507821912 -74.3624220527104) bank83632 +83633 POINT(40.58550462237532 -73.46867949636739) bank83633 +83634 POINT(41.56011913964703 -73.23261903927457) bank83634 +83635 POINT(41.57947656877023 -73.57418621350541) bank83635 +83636 POINT(40.72268808155051 -74.9082728026501) bank83636 +83637 POINT(41.19287125905556 -74.45131733350601) bank83637 +83638 POINT(40.70575191781594 -74.27662174489194) bank83638 +83639 POINT(40.51145772643635 -73.97875572827748) bank83639 +83640 POINT(40.26906518275841 -73.1578542656263) bank83640 +83641 POINT(41.254610697361926 -74.29768635979872) bank83641 +83642 POINT(40.72125248620999 -74.88364203044758) bank83642 +83643 POINT(41.33095832883383 -73.74744918550927) bank83643 +83644 POINT(40.860226682615945 -73.14939666668151) bank83644 +83645 POINT(40.725907022396335 -73.33898210557906) bank83645 +83646 POINT(41.517339415773066 -73.02406190617532) bank83646 +83647 POINT(40.04561972917143 -74.2694260447303) bank83647 +83648 POINT(40.846877389815674 -73.92557435617773) bank83648 +83649 POINT(40.40915832777935 -73.02199912922306) bank83649 +83650 POINT(41.40041282981253 -73.08341482491139) bank83650 +83651 POINT(41.25774336324996 -74.76316802024357) bank83651 +83652 POINT(40.38923556202939 -73.3447273788078) bank83652 +83653 POINT(41.52381559195578 -73.17925228442313) bank83653 +83654 POINT(41.22250833075855 -73.9657153138037) bank83654 +83655 POINT(40.61014646535633 -74.4036463852068) bank83655 +83656 POINT(41.61562541027589 -74.49268455542703) bank83656 +83657 POINT(41.255463095618104 -74.98089366318229) bank83657 +83658 POINT(39.75648520908556 -74.37472068627433) bank83658 +83659 POINT(40.79071383773707 -74.81985840702785) bank83659 +83660 POINT(41.63459416802968 -74.40363609217768) bank83660 +83661 POINT(39.76787180940833 -74.97371855280274) bank83661 +83662 POINT(40.59236376581558 -73.63257613281833) bank83662 +83663 POINT(39.79455044743041 -73.18486037848484) bank83663 +83664 POINT(39.758594853464274 -73.42109868371466) bank83664 +83665 POINT(39.78518290466247 -74.06968551767501) bank83665 +83666 POINT(40.00737785062975 -73.51634257130189) bank83666 +83667 POINT(41.01170598037306 -73.85567754369727) bank83667 +83668 POINT(41.37723127370146 -74.75522425564172) bank83668 +83669 POINT(40.100119027463315 -73.72004093354568) bank83669 +83670 POINT(40.377952409880265 -74.93788752674058) bank83670 +83671 POINT(40.2737893250687 -74.87516431103097) bank83671 +83672 POINT(39.863802010790636 -73.52876796894786) bank83672 +83673 POINT(40.22125724110016 -74.30587376932789) bank83673 +83674 POINT(41.60798793099007 -73.42602876462672) bank83674 +83675 POINT(40.861462414729154 -74.11513209216524) bank83675 +83676 POINT(40.48434730365946 -74.42405578300081) bank83676 +83677 POINT(41.07676020015714 -74.69571799585064) bank83677 +83678 POINT(41.59277659262597 -73.91160059529771) bank83678 +83679 POINT(39.80862484907315 -74.86649777815401) bank83679 +83680 POINT(39.8329716672926 -74.03247570875935) bank83680 +83681 POINT(40.1074837864583 -74.35639697167464) bank83681 +83682 POINT(40.39427794842977 -74.1399666989577) bank83682 +83683 POINT(40.5256214881935 -74.34641636275892) bank83683 +83684 POINT(41.13803871739541 -74.53566579685464) bank83684 +83685 POINT(41.59751479080872 -74.50247187319887) bank83685 +83686 POINT(41.41473830759103 -73.96043828053375) bank83686 +83687 POINT(39.74679843645065 -74.16431899383826) bank83687 +83688 POINT(41.38050207702072 -74.67590283071404) bank83688 +83689 POINT(40.426186611656995 -74.22995362448268) bank83689 +83690 POINT(41.60432593688658 -74.9999481627608) bank83690 +83691 POINT(40.528523594523854 -74.30844874945632) bank83691 +83692 POINT(40.58913875088278 -74.95609118901635) bank83692 +83693 POINT(39.80356054353388 -74.95672374026343) bank83693 +83694 POINT(41.173590984708134 -73.56968515007418) bank83694 +83695 POINT(40.43540057852944 -74.1006163005071) bank83695 +83696 POINT(40.67435198577543 -74.66776674899278) bank83696 +83697 POINT(40.892719589929854 -73.83234754208621) bank83697 +83698 POINT(40.885233444837155 -74.99078944340715) bank83698 +83699 POINT(40.19286828164724 -73.2204567434653) bank83699 +83700 POINT(40.110237984526684 -74.28124767956542) bank83700 +83701 POINT(40.49741896015052 -74.89089220654856) bank83701 +83702 POINT(41.3181954277094 -73.62643089907532) bank83702 +83703 POINT(41.373382388122586 -74.36138721003339) bank83703 +83704 POINT(40.43869583533285 -73.57681341525928) bank83704 +83705 POINT(40.56879502381334 -74.34900492427799) bank83705 +83706 POINT(39.785375344845725 -74.29749689396469) bank83706 +83707 POINT(40.6301541824059 -74.16787977383566) bank83707 +83708 POINT(40.26742861672864 -73.17557220814828) bank83708 +83709 POINT(41.23236102020192 -73.87254235181386) bank83709 +83710 POINT(39.71384963588831 -74.7391501056719) bank83710 +83711 POINT(39.735939600323896 -73.50052448848821) bank83711 +83712 POINT(40.68743738300433 -74.28459223941007) bank83712 +83713 POINT(40.37250182295477 -73.64383727005882) bank83713 +83714 POINT(39.88780687091575 -74.2479867585468) bank83714 +83715 POINT(40.31749526964352 -73.592998733728) bank83715 +83716 POINT(41.00052767893785 -74.21111562007579) bank83716 +83717 POINT(40.249457571018915 -74.48776316103738) bank83717 +83718 POINT(40.414649873671976 -74.17181623586484) bank83718 +83719 POINT(41.12011982392056 -73.1602871715632) bank83719 +83720 POINT(41.65961079128429 -73.94653132252213) bank83720 +83721 POINT(41.49945897763273 -73.16680090725143) bank83721 +83722 POINT(41.59647738131378 -74.3028748923923) bank83722 +83723 POINT(40.14156644368342 -74.81350929650242) bank83723 +83724 POINT(41.23190535302026 -73.9056094542518) bank83724 +83725 POINT(40.11971788701568 -73.54195371415001) bank83725 +83726 POINT(39.91630795893808 -74.79375653451811) bank83726 +83727 POINT(41.670142486185156 -73.3947839543034) bank83727 +83728 POINT(40.764182240569326 -74.238300911949) bank83728 +83729 POINT(41.51502283650011 -73.84009465183692) bank83729 +83730 POINT(40.85490950091571 -73.37165166158954) bank83730 +83731 POINT(40.95284882035816 -73.20952515599429) bank83731 +83732 POINT(40.66374105720073 -73.83697196603828) bank83732 +83733 POINT(41.146819897282434 -73.97322695139691) bank83733 +83734 POINT(40.21459674441613 -74.30528908994152) bank83734 +83735 POINT(40.1810317674241 -74.1856422207963) bank83735 +83736 POINT(40.7705943910118 -74.27102596231497) bank83736 +83737 POINT(39.80564943349425 -73.33700347319335) bank83737 +83738 POINT(41.454856003536285 -73.33139948658443) bank83738 +83739 POINT(40.76644688277262 -73.27618125187338) bank83739 +83740 POINT(40.448116724609484 -73.06613639433436) bank83740 +83741 POINT(41.29486640953353 -73.06820868673317) bank83741 +83742 POINT(41.53748439354621 -74.0393304764844) bank83742 +83743 POINT(40.616588539733954 -73.45032750616018) bank83743 +83744 POINT(40.34568489839144 -74.60655749346827) bank83744 +83745 POINT(41.60484011132397 -74.91963951815141) bank83745 +83746 POINT(40.512705746845164 -74.53631511095944) bank83746 +83747 POINT(40.53279917006755 -74.47588697382322) bank83747 +83748 POINT(41.184578547616866 -74.55398019531081) bank83748 +83749 POINT(41.02383775079273 -74.5486312452434) bank83749 +83750 POINT(40.78791822413632 -74.65582005651966) bank83750 +83751 POINT(41.54309761191614 -74.96530393779452) bank83751 +83752 POINT(39.73175207075698 -74.64031297126307) bank83752 +83753 POINT(41.71097972207621 -74.59461689153183) bank83753 +83754 POINT(41.21618869623542 -74.99516159525749) bank83754 +83755 POINT(41.08878672670485 -74.11461040465768) bank83755 +83756 POINT(40.274558335994676 -73.9824634823289) bank83756 +83757 POINT(40.84540846841769 -74.89225482945506) bank83757 +83758 POINT(39.76817976725992 -73.80953898190187) bank83758 +83759 POINT(40.76604740868841 -74.74972066822832) bank83759 +83760 POINT(41.429597708456754 -74.03109035541796) bank83760 +83761 POINT(40.34209633388383 -74.13770672284693) bank83761 +83762 POINT(40.91010838650093 -74.0588500321299) bank83762 +83763 POINT(40.425765476415826 -73.60404279154776) bank83763 +83764 POINT(41.30095631841736 -74.20377103564229) bank83764 +83765 POINT(40.278289269863286 -74.92244576423926) bank83765 +83766 POINT(41.608028524800694 -74.81823368966045) bank83766 +83767 POINT(41.039106103207075 -73.5106451707619) bank83767 +83768 POINT(40.13270570533112 -74.4546936959806) bank83768 +83769 POINT(40.94831851424648 -74.55863163880673) bank83769 +83770 POINT(39.84775873944686 -74.23452596105169) bank83770 +83771 POINT(40.957247092086334 -74.19375957115044) bank83771 +83772 POINT(40.590638294920026 -74.60060196386893) bank83772 +83773 POINT(40.94718557746091 -73.9899610682846) bank83773 +83774 POINT(40.80081113509925 -74.76058780811732) bank83774 +83775 POINT(41.322297412559884 -73.39778863755375) bank83775 +83776 POINT(40.3114087420514 -74.13177541540766) bank83776 +83777 POINT(40.117864445148385 -73.24634375431704) bank83777 +83778 POINT(41.61500558310437 -74.13041116475678) bank83778 +83779 POINT(40.32032121515507 -74.7635664281657) bank83779 +83780 POINT(41.231757197633094 -73.33426891279873) bank83780 +83781 POINT(39.884430730570465 -74.1244926981389) bank83781 +83782 POINT(41.22094356285628 -74.34580859749553) bank83782 +83783 POINT(39.78209377675047 -74.60576052708132) bank83783 +83784 POINT(41.43139235808234 -73.34650077815859) bank83784 +83785 POINT(39.884777034457 -73.74374341100518) bank83785 +83786 POINT(39.947656344685136 -73.0325433068611) bank83786 +83787 POINT(41.34614098936056 -73.29151458602917) bank83787 +83788 POINT(40.65969363282149 -73.29537068586893) bank83788 +83789 POINT(41.000371740087566 -74.15642413948578) bank83789 +83790 POINT(39.78729319990986 -73.35470124680327) bank83790 +83791 POINT(40.40145314171892 -73.60137819490134) bank83791 +83792 POINT(40.02582681216642 -73.72860546297966) bank83792 +83793 POINT(41.313635649016206 -73.47035215689151) bank83793 +83794 POINT(41.282128462604696 -74.69717090579309) bank83794 +83795 POINT(39.78010659210129 -74.04159011988372) bank83795 +83796 POINT(40.5166237393575 -73.18541031639433) bank83796 +83797 POINT(41.44050471610393 -73.85663527934754) bank83797 +83798 POINT(41.481687106342136 -73.23381481105275) bank83798 +83799 POINT(40.400946357273064 -73.08083028270323) bank83799 +83800 POINT(41.10262449737226 -73.9720629744204) bank83800 +83801 POINT(40.23587696962506 -74.2566748459653) bank83801 +83802 POINT(40.36482579758721 -74.6912674864682) bank83802 +83803 POINT(39.90606956992769 -73.63082027836968) bank83803 +83804 POINT(39.71994180338344 -73.0152517496883) bank83804 +83805 POINT(41.21358138423806 -74.50637949183223) bank83805 +83806 POINT(41.021288350662566 -74.94499216357288) bank83806 +83807 POINT(41.696176483121114 -73.57239083772598) bank83807 +83808 POINT(39.98673655313796 -74.94951335632051) bank83808 +83809 POINT(40.85167784911919 -73.79689992427639) bank83809 +83810 POINT(40.05263082477935 -74.60919228878721) bank83810 +83811 POINT(40.50790761953424 -74.07443867154535) bank83811 +83812 POINT(39.933725978388956 -74.01958610104688) bank83812 +83813 POINT(40.18357527561101 -74.18700764027952) bank83813 +83814 POINT(40.131469939647324 -74.30443343083623) bank83814 +83815 POINT(39.88717175664614 -74.80057446705636) bank83815 +83816 POINT(40.71146424614431 -74.2643588675687) bank83816 +83817 POINT(41.027361430770085 -74.40029982902817) bank83817 +83818 POINT(41.03425557518506 -74.71255284465857) bank83818 +83819 POINT(41.64452698866363 -73.17008572018328) bank83819 +83820 POINT(40.1580574010397 -74.16371532848156) bank83820 +83821 POINT(41.20125724248695 -73.65637617044594) bank83821 +83822 POINT(40.49922655647332 -74.07492477240028) bank83822 +83823 POINT(40.606635124590255 -74.45161352875843) bank83823 +83824 POINT(39.945155116397636 -73.53242493650734) bank83824 +83825 POINT(40.38079739392689 -74.80565821233282) bank83825 +83826 POINT(40.56923372658967 -73.6876688484252) bank83826 +83827 POINT(41.01304768329022 -73.07780850861595) bank83827 +83828 POINT(41.51420996636239 -73.14469440371592) bank83828 +83829 POINT(40.587028241552844 -74.78893267751346) bank83829 +83830 POINT(41.21280689114541 -74.7007431388822) bank83830 +83831 POINT(41.53260471480534 -74.81604567812568) bank83831 +83832 POINT(40.90721961677493 -73.35733756282603) bank83832 +83833 POINT(41.047410135330736 -74.06591980568858) bank83833 +83834 POINT(40.12093473850808 -74.29529158270002) bank83834 +83835 POINT(40.797408886634614 -73.4556597437141) bank83835 +83836 POINT(40.46934186428709 -73.47931834397616) bank83836 +83837 POINT(41.64039961324922 -74.5794676144088) bank83837 +83838 POINT(41.6358922396046 -74.61867936485719) bank83838 +83839 POINT(40.728279729660436 -73.45593236415283) bank83839 +83840 POINT(40.11253201559474 -74.62915898717414) bank83840 +83841 POINT(40.59307389172787 -73.212494927454) bank83841 +83842 POINT(40.26531560174563 -73.46159495105086) bank83842 +83843 POINT(40.67329677362026 -73.23388671764747) bank83843 +83844 POINT(40.323896338071485 -73.9950504311731) bank83844 +83845 POINT(41.673843456646495 -74.42179029045839) bank83845 +83846 POINT(40.279013382525115 -74.79101784894031) bank83846 +83847 POINT(41.44138091355792 -74.49958248368637) bank83847 +83848 POINT(40.77513590623103 -73.98441185147752) bank83848 +83849 POINT(41.561831037193706 -74.74781287671249) bank83849 +83850 POINT(41.46596534254172 -74.66961872105072) bank83850 +83851 POINT(41.14068610284567 -74.63104569515073) bank83851 +83852 POINT(40.05061395736999 -74.16280025150553) bank83852 +83853 POINT(41.057691989596925 -73.0400769973884) bank83853 +83854 POINT(41.01409493769802 -74.43326800897597) bank83854 +83855 POINT(41.408018269201676 -74.22685036178484) bank83855 +83856 POINT(41.363581627667415 -74.63119446833777) bank83856 +83857 POINT(41.15029695812616 -73.54914433364162) bank83857 +83858 POINT(40.661845304951655 -74.62685338902325) bank83858 +83859 POINT(40.56987081636779 -74.36965841289994) bank83859 +83860 POINT(41.49206598518237 -73.59513389594791) bank83860 +83861 POINT(40.201759955608765 -73.10467820775692) bank83861 +83862 POINT(40.55696181692918 -73.4523796335458) bank83862 +83863 POINT(40.35015316680117 -73.77873959644054) bank83863 +83864 POINT(41.57158595410589 -73.4421507663424) bank83864 +83865 POINT(40.46002307961742 -73.78022082890429) bank83865 +83866 POINT(40.419456950680015 -73.61264759188333) bank83866 +83867 POINT(40.10535468935851 -74.48405007044173) bank83867 +83868 POINT(40.894362560722236 -74.29193330808076) bank83868 +83869 POINT(39.91689873577811 -74.13840776762936) bank83869 +83870 POINT(39.922059188662274 -74.6474063412245) bank83870 +83871 POINT(41.264007836561966 -74.36670662375876) bank83871 +83872 POINT(40.20253750328832 -73.37977945762037) bank83872 +83873 POINT(41.22267796050128 -74.55349691080492) bank83873 +83874 POINT(40.67095394579294 -73.5535019240426) bank83874 +83875 POINT(40.103568648587604 -74.07205768768456) bank83875 +83876 POINT(40.475596100673 -73.79609216393449) bank83876 +83877 POINT(40.38948775052207 -73.06304668463855) bank83877 +83878 POINT(40.972361530681745 -73.24717772294849) bank83878 +83879 POINT(39.826635269776496 -73.41212851886851) bank83879 +83880 POINT(40.362364412952594 -74.34838401899704) bank83880 +83881 POINT(41.0346358497793 -74.7620451777321) bank83881 +83882 POINT(40.260210659926074 -73.35310487120735) bank83882 +83883 POINT(40.33938436808576 -74.36573928494046) bank83883 +83884 POINT(40.39766005746487 -74.6843963106269) bank83884 +83885 POINT(40.05034562914574 -73.50733990366336) bank83885 +83886 POINT(40.94667021319051 -74.60728227290352) bank83886 +83887 POINT(41.70629795135347 -74.01283795315598) bank83887 +83888 POINT(40.96636603798107 -74.171380443809) bank83888 +83889 POINT(40.29086676522563 -74.18155246252095) bank83889 +83890 POINT(40.73272505964843 -74.11416046623097) bank83890 +83891 POINT(39.719920535011525 -74.97361596856453) bank83891 +83892 POINT(39.79372368111523 -73.43222939858207) bank83892 +83893 POINT(41.271371241091515 -74.55961868963537) bank83893 +83894 POINT(40.31343706022767 -74.99124481703373) bank83894 +83895 POINT(41.05019492371998 -73.14857945193926) bank83895 +83896 POINT(40.915003617553175 -74.89437289968042) bank83896 +83897 POINT(41.17447492946352 -74.55289985988726) bank83897 +83898 POINT(41.65304824168676 -74.48578120719564) bank83898 +83899 POINT(40.963431311057846 -74.27016665466992) bank83899 +83900 POINT(40.080782929048176 -74.19269846438766) bank83900 +83901 POINT(41.35311158312922 -74.01524875306711) bank83901 +83902 POINT(41.36506422686754 -74.57126557455938) bank83902 +83903 POINT(40.11753704365351 -73.60862781524844) bank83903 +83904 POINT(41.52075509053577 -74.80945255419783) bank83904 +83905 POINT(41.16551064947511 -73.38305335974786) bank83905 +83906 POINT(39.76723340189583 -74.40179706342775) bank83906 +83907 POINT(40.783945140869385 -73.89138583223861) bank83907 +83908 POINT(39.76273770151099 -73.48084112653291) bank83908 +83909 POINT(41.474744339802584 -73.80484484609148) bank83909 +83910 POINT(41.32828643844624 -74.97242696566111) bank83910 +83911 POINT(39.807470800652524 -73.16127503827813) bank83911 +83912 POINT(41.579080619994414 -74.54418250967139) bank83912 +83913 POINT(41.372990919352745 -73.91779325961942) bank83913 +83914 POINT(39.877355690645274 -73.54744682448963) bank83914 +83915 POINT(41.23387456717703 -73.6569332005359) bank83915 +83916 POINT(40.129979952172505 -73.73799030264493) bank83916 +83917 POINT(40.353943650457126 -74.35299477880406) bank83917 +83918 POINT(40.17835641241115 -73.21439846230865) bank83918 +83919 POINT(40.45441176542594 -73.31156146589409) bank83919 +83920 POINT(40.97798501994107 -73.79897230412406) bank83920 +83921 POINT(41.03290252835136 -73.2097365840226) bank83921 +83922 POINT(40.295890724275885 -74.43524168357177) bank83922 +83923 POINT(40.09282729840362 -74.08644629053667) bank83923 +83924 POINT(40.371276608348204 -74.10406053255979) bank83924 +83925 POINT(40.53115429650854 -73.00943683555015) bank83925 +83926 POINT(41.40451798867018 -74.85192009573369) bank83926 +83927 POINT(40.22946685546605 -73.12339415631014) bank83927 +83928 POINT(40.6163668097379 -73.9747927335974) bank83928 +83929 POINT(40.12196244667706 -74.51902046331564) bank83929 +83930 POINT(40.711269244672096 -73.73003268142591) bank83930 +83931 POINT(39.7611447005592 -73.67939354050557) bank83931 +83932 POINT(40.567239001497875 -74.5190144521677) bank83932 +83933 POINT(40.22055937745214 -74.98432776500312) bank83933 +83934 POINT(41.02510099678409 -74.06744131389235) bank83934 +83935 POINT(40.64325688655176 -73.64359526781618) bank83935 +83936 POINT(40.87845785415965 -73.85851583376896) bank83936 +83937 POINT(41.67485580730002 -74.91785483303974) bank83937 +83938 POINT(41.62430803149958 -74.14208950192592) bank83938 +83939 POINT(40.96252383070499 -74.61216721383639) bank83939 +83940 POINT(40.1628592562148 -74.34452663661149) bank83940 +83941 POINT(41.07638534251664 -73.95953518407461) bank83941 +83942 POINT(41.330056935117746 -74.93094552534697) bank83942 +83943 POINT(39.948589253440296 -74.03389620950864) bank83943 +83944 POINT(41.36796581496947 -74.97346617427829) bank83944 +83945 POINT(40.457635689257565 -74.35750413847795) bank83945 +83946 POINT(39.72732410701383 -74.0717961604605) bank83946 +83947 POINT(39.7751700794909 -73.18815497723315) bank83947 +83948 POINT(40.31993843335767 -73.43139883865948) bank83948 +83949 POINT(40.83003754208849 -74.05042489102145) bank83949 +83950 POINT(40.12503736651311 -74.03342955978428) bank83950 +83951 POINT(39.767346711492664 -73.04249832282927) bank83951 +83952 POINT(41.03974444043594 -74.67853706642977) bank83952 +83953 POINT(40.69019184931292 -74.93300072802164) bank83953 +83954 POINT(40.04125549664746 -73.14399688017538) bank83954 +83955 POINT(41.70851456471884 -73.15632024138792) bank83955 +83956 POINT(40.26319083687082 -74.40160661407232) bank83956 +83957 POINT(41.19462598027206 -74.68033246338818) bank83957 +83958 POINT(40.716250941240986 -74.4978837278779) bank83958 +83959 POINT(40.81832376445665 -74.79834379672543) bank83959 +83960 POINT(41.619047863256995 -73.75986058735859) bank83960 +83961 POINT(40.015538838013555 -73.50407103643437) bank83961 +83962 POINT(41.38075126391561 -73.9610791107238) bank83962 +83963 POINT(41.19706771462302 -74.68786615195606) bank83963 +83964 POINT(40.415343357336575 -74.4411947842951) bank83964 +83965 POINT(39.86299779663329 -73.32499956309277) bank83965 +83966 POINT(40.71725911606968 -73.25744524465391) bank83966 +83967 POINT(40.546251122079575 -74.85611688905558) bank83967 +83968 POINT(40.548868385446795 -73.25566400947362) bank83968 +83969 POINT(39.77759262676297 -73.62988493491814) bank83969 +83970 POINT(39.75234543883336 -73.47611004953143) bank83970 +83971 POINT(40.12605679243648 -74.90716118597395) bank83971 +83972 POINT(39.90676660030893 -73.73443481700264) bank83972 +83973 POINT(41.101324577103625 -74.76019108931048) bank83973 +83974 POINT(39.999030601587556 -73.61606501707533) bank83974 +83975 POINT(40.08778439409359 -74.53327477065116) bank83975 +83976 POINT(40.47814667223362 -74.57350821500012) bank83976 +83977 POINT(41.06772925726402 -74.11523369648943) bank83977 +83978 POINT(40.37211158341175 -74.77501355806399) bank83978 +83979 POINT(40.935486013070864 -74.62762101990552) bank83979 +83980 POINT(40.88771954267442 -73.3201254749946) bank83980 +83981 POINT(39.99590726872189 -73.37358836481377) bank83981 +83982 POINT(40.114683263259764 -73.86427412990349) bank83982 +83983 POINT(39.78344963456315 -73.83717263828541) bank83983 +83984 POINT(40.000360674393576 -73.41648022941165) bank83984 +83985 POINT(41.417263396418036 -73.50096754361256) bank83985 +83986 POINT(39.98143453329686 -74.5019245518563) bank83986 +83987 POINT(40.47199953980537 -74.17238837579995) bank83987 +83988 POINT(40.83324155373854 -73.56328091103227) bank83988 +83989 POINT(40.12063476563384 -73.97421910363815) bank83989 +83990 POINT(40.38592380651406 -73.61281958391417) bank83990 +83991 POINT(41.35948933469509 -73.10585504135017) bank83991 +83992 POINT(40.9310041532669 -73.60470257047776) bank83992 +83993 POINT(39.745797293168415 -73.20486368866291) bank83993 +83994 POINT(41.375464084959546 -73.66989190751518) bank83994 +83995 POINT(40.225690329948314 -73.57921283367799) bank83995 +83996 POINT(40.79482358766481 -74.85709304683007) bank83996 +83997 POINT(40.459075035518175 -74.8368666543076) bank83997 +83998 POINT(40.43774349531543 -73.35137472708085) bank83998 +83999 POINT(41.20424183848908 -74.72298530956189) bank83999 +84000 POINT(41.29701257083094 -73.77332821794467) bank84000 +84001 POINT(40.27864845490204 -74.04749593593225) bank84001 +84002 POINT(41.14965800378206 -74.0525489008041) bank84002 +84003 POINT(41.61686933149938 -73.8480019195159) bank84003 +84004 POINT(41.067892703266494 -74.83502350286722) bank84004 +84005 POINT(41.0835958359393 -73.29269675156212) bank84005 +84006 POINT(40.69067318190738 -73.28094449948111) bank84006 +84007 POINT(39.78085176157896 -74.91467932092945) bank84007 +84008 POINT(40.35008929124384 -73.61674362724322) bank84008 +84009 POINT(41.64487437936609 -74.87824165127184) bank84009 +84010 POINT(41.498869227261466 -73.61174107703545) bank84010 +84011 POINT(40.848457588573936 -74.60274355428828) bank84011 +84012 POINT(41.18184909179084 -73.36002733948408) bank84012 +84013 POINT(41.017010062303484 -73.97081253071379) bank84013 +84014 POINT(41.18747121313446 -74.03118828754769) bank84014 +84015 POINT(41.44694592795803 -73.41677788641641) bank84015 +84016 POINT(40.545017232541404 -73.73394811405056) bank84016 +84017 POINT(41.62638507753091 -74.60440453391551) bank84017 +84018 POINT(40.7739972479662 -74.71893698395662) bank84018 +84019 POINT(40.00660556148655 -74.66966833678273) bank84019 +84020 POINT(41.434799705473445 -74.4106305007101) bank84020 +84021 POINT(40.080408357690914 -74.4231284207101) bank84021 +84022 POINT(40.31518216371608 -74.73234558992691) bank84022 +84023 POINT(39.98350049562135 -74.01808905017029) bank84023 +84024 POINT(41.700103703658776 -73.5484088446718) bank84024 +84025 POINT(41.1718439852994 -74.09234829637074) bank84025 +84026 POINT(40.05200133571664 -73.86495951537222) bank84026 +84027 POINT(41.598705699005464 -74.5444985927948) bank84027 +84028 POINT(39.72430129550369 -73.33353539210147) bank84028 +84029 POINT(41.63753871995775 -73.07396045176024) bank84029 +84030 POINT(40.10237211855088 -74.63436736456897) bank84030 +84031 POINT(40.971033296232456 -73.88775205985417) bank84031 +84032 POINT(41.07427622426133 -73.77886393958177) bank84032 +84033 POINT(41.56336063526143 -74.8758296483325) bank84033 +84034 POINT(39.86156692632469 -73.93826210024991) bank84034 +84035 POINT(39.891966955058 -74.6560127476995) bank84035 +84036 POINT(40.406225380036 -73.65533125292417) bank84036 +84037 POINT(40.80236788201441 -74.37785370695696) bank84037 +84038 POINT(41.65335315880353 -73.44551710050602) bank84038 +84039 POINT(41.54584264920429 -73.36347403009032) bank84039 +84040 POINT(41.31796003431178 -74.79961937396246) bank84040 +84041 POINT(40.83697383546853 -75.00180505199572) bank84041 +84042 POINT(39.8087893075448 -73.27621801960251) bank84042 +84043 POINT(40.39493573061059 -73.0876271830565) bank84043 +84044 POINT(41.33323292865716 -74.25992713551854) bank84044 +84045 POINT(41.37938109811924 -73.12768632857764) bank84045 +84046 POINT(40.67138000599248 -73.07520826425748) bank84046 +84047 POINT(40.006270908391656 -73.45430299574933) bank84047 +84048 POINT(40.84907623350038 -74.94475482933265) bank84048 +84049 POINT(41.361054053388536 -73.45445872001108) bank84049 +84050 POINT(41.25811199570289 -74.75979996374804) bank84050 +84051 POINT(40.25832922780998 -73.5947093963969) bank84051 +84052 POINT(39.81866059101953 -74.20086166073543) bank84052 +84053 POINT(41.22140101065414 -73.85609301478034) bank84053 +84054 POINT(41.51797976469315 -73.5798455323285) bank84054 +84055 POINT(40.72530658998238 -74.87530988315957) bank84055 +84056 POINT(41.3368433601094 -73.03339300756556) bank84056 +84057 POINT(39.984028367608616 -74.78460797367744) bank84057 +84058 POINT(40.1072770010945 -73.64814720884725) bank84058 +84059 POINT(40.25884430510142 -73.9492732044939) bank84059 +84060 POINT(39.91648434270309 -73.0919203789706) bank84060 +84061 POINT(41.576818155915326 -73.58383400354549) bank84061 +84062 POINT(41.65274580117463 -74.4647990527626) bank84062 +84063 POINT(41.615228447766995 -73.7146383437377) bank84063 +84064 POINT(40.03145749672514 -73.06308992410885) bank84064 +84065 POINT(39.7992235884348 -73.1704334031442) bank84065 +84066 POINT(41.33628793429388 -74.51547796493698) bank84066 +84067 POINT(40.23974754149696 -73.34021063106478) bank84067 +84068 POINT(40.576603876796554 -74.00733100621312) bank84068 +84069 POINT(40.950502019590566 -74.98658783897962) bank84069 +84070 POINT(40.214126463266005 -73.64833188791746) bank84070 +84071 POINT(40.96698690389436 -73.17583756892783) bank84071 +84072 POINT(40.304814075701294 -74.8095013104144) bank84072 +84073 POINT(41.19935856357907 -73.36964194187269) bank84073 +84074 POINT(41.22012284200386 -74.85837451957924) bank84074 +84075 POINT(40.95396199227702 -73.75426730787642) bank84075 +84076 POINT(41.05067272937317 -74.88692305130346) bank84076 +84077 POINT(40.16185027907681 -74.50647098253233) bank84077 +84078 POINT(40.49306465882671 -73.9499758120921) bank84078 +84079 POINT(41.28457367986968 -74.62410364733302) bank84079 +84080 POINT(40.95362778843641 -73.59324255563554) bank84080 +84081 POINT(41.36514161391532 -74.74175566027817) bank84081 +84082 POINT(40.40344586572138 -73.68544521986837) bank84082 +84083 POINT(40.1623208369581 -74.47590145843866) bank84083 +84084 POINT(41.30135192421731 -73.06506742985427) bank84084 +84085 POINT(40.825280311658496 -74.73239408473789) bank84085 +84086 POINT(41.290751017232616 -74.68250489658587) bank84086 +84087 POINT(40.47286604095436 -74.16346372583858) bank84087 +84088 POINT(41.12650552155308 -73.7820820086851) bank84088 +84089 POINT(41.5109176723923 -74.07501020651796) bank84089 +84090 POINT(39.85284144038409 -74.44848491330931) bank84090 +84091 POINT(40.39860251931598 -74.79998100767519) bank84091 +84092 POINT(40.351821569403654 -73.87240064639273) bank84092 +84093 POINT(39.93572131045108 -74.52986865792118) bank84093 +84094 POINT(40.127888609283204 -73.39817309821872) bank84094 +84095 POINT(41.61719111361492 -73.47385323588618) bank84095 +84096 POINT(39.974003855558166 -74.10662306970286) bank84096 +84097 POINT(40.01458470592202 -74.35328284886828) bank84097 +84098 POINT(40.47211165346974 -74.56317520290584) bank84098 +84099 POINT(40.373147894229355 -74.32749739541629) bank84099 +84100 POINT(41.27160855180908 -73.18147396131675) bank84100 +84101 POINT(39.79210535049354 -73.68151905977257) bank84101 +84102 POINT(40.98123725782754 -73.7278430989724) bank84102 +84103 POINT(41.5966349217629 -73.14517257760852) bank84103 +84104 POINT(41.56498387315944 -73.26207300275573) bank84104 +84105 POINT(41.231933609535766 -74.86481036136067) bank84105 +84106 POINT(40.186957169006455 -73.35680390617284) bank84106 +84107 POINT(40.100462472356355 -73.78459329397131) bank84107 +84108 POINT(40.50593449054194 -74.97452284740967) bank84108 +84109 POINT(40.404885857906464 -74.89615370812115) bank84109 +84110 POINT(41.49558914420699 -73.88667407307034) bank84110 +84111 POINT(40.52852018708782 -74.64290651681307) bank84111 +84112 POINT(40.10969593012579 -73.72410145701477) bank84112 +84113 POINT(39.832082481693654 -73.46891821562086) bank84113 +84114 POINT(40.52054276815025 -74.34621022299612) bank84114 +84115 POINT(40.17420236206198 -74.57424313666748) bank84115 +84116 POINT(39.96593732062214 -73.05478579754917) bank84116 +84117 POINT(40.42155120175255 -73.0506385367121) bank84117 +84118 POINT(39.98536886616269 -73.55466292643797) bank84118 +84119 POINT(40.67452674254688 -73.21888876192088) bank84119 +84120 POINT(39.80674368014952 -74.94801350539471) bank84120 +84121 POINT(40.72169549543555 -74.08696246779331) bank84121 +84122 POINT(41.69378110219898 -73.81822596104563) bank84122 +84123 POINT(40.46897863846786 -74.89169634753085) bank84123 +84124 POINT(39.83922052568135 -73.35938604120896) bank84124 +84125 POINT(40.421441087042325 -73.98932531719124) bank84125 +84126 POINT(41.70578397983872 -74.50155816069602) bank84126 +84127 POINT(41.65581873971722 -74.43885538792331) bank84127 +84128 POINT(41.2603557112688 -74.70658700677538) bank84128 +84129 POINT(41.491642655809315 -73.90879493556507) bank84129 +84130 POINT(41.54961139420971 -73.61513388013698) bank84130 +84131 POINT(40.79333855345085 -73.9154386836597) bank84131 +84132 POINT(40.23769486632128 -74.4387766274453) bank84132 +84133 POINT(40.407910696987116 -74.64012557552236) bank84133 +84134 POINT(41.18670552497526 -74.3967912273667) bank84134 +84135 POINT(41.29738548926991 -73.21553080699127) bank84135 +84136 POINT(39.85797238210187 -73.70101296471144) bank84136 +84137 POINT(40.303207036076195 -73.49450090075239) bank84137 +84138 POINT(41.6384444657383 -74.15279730577483) bank84138 +84139 POINT(40.572905662032376 -74.68513799049471) bank84139 +84140 POINT(40.65030661541728 -74.46243397087184) bank84140 +84141 POINT(40.83367878380264 -74.86277920091693) bank84141 +84142 POINT(40.08886086698155 -74.1702717648844) bank84142 +84143 POINT(40.92618891262309 -74.78395828629904) bank84143 +84144 POINT(41.17166475022336 -73.81162703803648) bank84144 +84145 POINT(41.08774111456677 -74.01789205694688) bank84145 +84146 POINT(41.655725708025024 -74.84930446402028) bank84146 +84147 POINT(39.79762731251867 -74.46935349934185) bank84147 +84148 POINT(40.668924876967594 -74.40046002921547) bank84148 +84149 POINT(41.14349394498929 -73.54518739190623) bank84149 +84150 POINT(41.33801458236178 -74.21294635458861) bank84150 +84151 POINT(40.91071704700571 -73.43885772228742) bank84151 +84152 POINT(41.08067412370537 -74.04537783483309) bank84152 +84153 POINT(40.33796809559446 -74.38887497681108) bank84153 +84154 POINT(41.446590769562185 -73.787385719201) bank84154 +84155 POINT(40.014346777806274 -74.02225918073928) bank84155 +84156 POINT(40.16503687114354 -73.57522841601775) bank84156 +84157 POINT(41.10674134937079 -74.48402450889832) bank84157 +84158 POINT(39.7600090726336 -73.02333782757675) bank84158 +84159 POINT(39.97453790811983 -74.23503908354608) bank84159 +84160 POINT(41.58155839537803 -73.89775078934974) bank84160 +84161 POINT(40.434102231893014 -74.03461325571324) bank84161 +84162 POINT(39.99321889827667 -73.52874725471747) bank84162 +84163 POINT(40.28471982463353 -74.19227972349356) bank84163 +84164 POINT(39.96518976995521 -74.80698071885371) bank84164 +84165 POINT(40.771245495641395 -73.12795550741805) bank84165 +84166 POINT(39.759820512458326 -73.7614432747947) bank84166 +84167 POINT(41.48467253052262 -74.96506228844589) bank84167 +84168 POINT(41.23695712585656 -73.00830423765817) bank84168 +84169 POINT(40.283971312618576 -73.59071215846474) bank84169 +84170 POINT(40.32924363902305 -73.62670563714823) bank84170 +84171 POINT(40.527145304917504 -74.40757051572601) bank84171 +84172 POINT(40.80315771310269 -74.62323152431136) bank84172 +84173 POINT(41.022323662040236 -74.70349296620876) bank84173 +84174 POINT(41.68908306903322 -74.53955691635096) bank84174 +84175 POINT(40.86237405771242 -73.64926102842013) bank84175 +84176 POINT(40.13884976604766 -73.88711646843977) bank84176 +84177 POINT(40.01273288105208 -73.32368439554914) bank84177 +84178 POINT(41.27519595202673 -73.27755147704272) bank84178 +84179 POINT(41.4568903727441 -73.54669625006461) bank84179 +84180 POINT(41.42215812966439 -73.79393893558951) bank84180 +84181 POINT(40.689008669811905 -74.61198342499843) bank84181 +84182 POINT(40.66992160708835 -74.10788781530975) bank84182 +84183 POINT(41.38449161747263 -73.39342905045368) bank84183 +84184 POINT(39.94825367567695 -73.40873095406606) bank84184 +84185 POINT(41.69423678082031 -74.24306131073477) bank84185 +84186 POINT(41.415693925610235 -73.24229057161409) bank84186 +84187 POINT(40.27334997098805 -73.3104771749337) bank84187 +84188 POINT(39.8740377745019 -74.84266459039006) bank84188 +84189 POINT(40.67148002827128 -73.47994089115745) bank84189 +84190 POINT(40.46465278034975 -73.54024010604257) bank84190 +84191 POINT(40.91394344277806 -73.55088688017064) bank84191 +84192 POINT(41.17135504886066 -73.20889459026824) bank84192 +84193 POINT(40.10361090708983 -73.8359774202551) bank84193 +84194 POINT(41.335867016070814 -73.31542432316921) bank84194 +84195 POINT(39.848435808563536 -73.43048589697317) bank84195 +84196 POINT(41.11070783715681 -74.13311999856157) bank84196 +84197 POINT(41.57958251422782 -73.30319904894658) bank84197 +84198 POINT(41.3510488393935 -73.12476875714299) bank84198 +84199 POINT(40.16532706939084 -73.99849408935773) bank84199 +84200 POINT(40.34210004393755 -73.47739771393026) bank84200 +84201 POINT(41.09241754419552 -73.79889296561683) bank84201 +84202 POINT(39.901767315728996 -74.33665619076832) bank84202 +84203 POINT(40.048285507691254 -74.13714848812432) bank84203 +84204 POINT(39.789273240098396 -73.99618637614745) bank84204 +84205 POINT(41.03147930653747 -73.93712547543774) bank84205 +84206 POINT(40.90558982099456 -73.65767465144594) bank84206 +84207 POINT(41.16116681472828 -74.19067742542765) bank84207 +84208 POINT(41.56264437654027 -73.90614277460944) bank84208 +84209 POINT(39.92245025624524 -73.86178735111716) bank84209 +84210 POINT(40.540993228761955 -73.53915147124795) bank84210 +84211 POINT(40.01581215176463 -73.77278037223212) bank84211 +84212 POINT(41.48837520936133 -74.78045573048564) bank84212 +84213 POINT(39.9449195430042 -74.97745156894013) bank84213 +84214 POINT(39.96157856330038 -73.32305942725435) bank84214 +84215 POINT(40.19322783615709 -73.44435680696323) bank84215 +84216 POINT(40.58603092970049 -74.00787982453454) bank84216 +84217 POINT(41.27547262902591 -74.22245808831943) bank84217 +84218 POINT(40.206730850863515 -74.36577833596401) bank84218 +84219 POINT(41.51539150780518 -73.12998420233166) bank84219 +84220 POINT(41.58113429823942 -74.3126870969963) bank84220 +84221 POINT(39.83383769770527 -74.82899158133624) bank84221 +84222 POINT(40.81205246673209 -73.57309650747968) bank84222 +84223 POINT(40.420390724295046 -73.57621335771775) bank84223 +84224 POINT(40.15120883071521 -74.1187845829287) bank84224 +84225 POINT(39.883883868984185 -74.08686402202983) bank84225 +84226 POINT(40.27677623050066 -73.12623433561511) bank84226 +84227 POINT(40.27237752818327 -74.56817792995591) bank84227 +84228 POINT(40.241171499366814 -74.20428796222782) bank84228 +84229 POINT(40.749110921743586 -73.38548748019504) bank84229 +84230 POINT(40.86034138099012 -74.93306995702017) bank84230 +84231 POINT(40.02401663884048 -73.33515952498115) bank84231 +84232 POINT(40.1451414308509 -74.52700893434107) bank84232 +84233 POINT(41.48622065938997 -74.6444896728091) bank84233 +84234 POINT(40.51170318081938 -74.58124355995294) bank84234 +84235 POINT(41.52521983440299 -74.21984914778928) bank84235 +84236 POINT(41.22651167751259 -73.10384048912802) bank84236 +84237 POINT(41.703534774384124 -74.80693854781723) bank84237 +84238 POINT(39.95678733978248 -73.96101947329221) bank84238 +84239 POINT(41.18098518393241 -73.06228810930045) bank84239 +84240 POINT(41.475460697049265 -73.01844863507816) bank84240 +84241 POINT(41.3339796607546 -74.94821484746267) bank84241 +84242 POINT(40.70782583504121 -74.66470149589667) bank84242 +84243 POINT(41.237950656726774 -74.14925387575333) bank84243 +84244 POINT(40.28424803347795 -73.25051144623271) bank84244 +84245 POINT(40.743533148188774 -74.25238298706658) bank84245 +84246 POINT(40.87365987218148 -74.75349083377672) bank84246 +84247 POINT(41.453349151412446 -73.27406990250158) bank84247 +84248 POINT(40.67134847817459 -73.98857230523419) bank84248 +84249 POINT(41.03294676205783 -74.32523486606296) bank84249 +84250 POINT(41.29891798117655 -74.3466424910131) bank84250 +84251 POINT(41.50884646337206 -74.40169796922726) bank84251 +84252 POINT(40.79172943048891 -74.59727686924559) bank84252 +84253 POINT(40.76294059144226 -73.32673168926321) bank84253 +84254 POINT(40.024845966324555 -74.90253740467993) bank84254 +84255 POINT(40.23252711873738 -74.05323878835262) bank84255 +84256 POINT(40.159034810692724 -74.45205053060171) bank84256 +84257 POINT(40.56677779985891 -74.51813043953084) bank84257 +84258 POINT(40.90901317401957 -73.52947161164194) bank84258 +84259 POINT(40.46464769755396 -74.31312014603262) bank84259 +84260 POINT(41.47182640342385 -73.52677351211332) bank84260 +84261 POINT(41.02781720932396 -73.31579190839838) bank84261 +84262 POINT(40.14151259426221 -73.44611856270734) bank84262 +84263 POINT(41.20420159212247 -73.6544131968473) bank84263 +84264 POINT(39.80769681967739 -74.50240246803924) bank84264 +84265 POINT(41.17441169488561 -74.48593024867426) bank84265 +84266 POINT(40.61308287442212 -73.73201779816935) bank84266 +84267 POINT(41.652645316075635 -73.68536697501582) bank84267 +84268 POINT(40.36850947155993 -73.05608147309917) bank84268 +84269 POINT(40.81508129874837 -74.99734212473378) bank84269 +84270 POINT(40.313944388041726 -74.33859844999272) bank84270 +84271 POINT(40.45524932220958 -74.98259177082586) bank84271 +84272 POINT(40.01049123594882 -73.81528732457986) bank84272 +84273 POINT(41.4628192966184 -74.88202224856168) bank84273 +84274 POINT(39.7307304212957 -73.80848974493364) bank84274 +84275 POINT(40.54905140915253 -74.62292140982855) bank84275 +84276 POINT(41.33802005108563 -74.82705091696802) bank84276 +84277 POINT(41.51043086152483 -74.84665232841127) bank84277 +84278 POINT(40.232744327850135 -74.27892764262452) bank84278 +84279 POINT(41.297625440111226 -74.40327000626026) bank84279 +84280 POINT(39.73076157696064 -74.3447743217682) bank84280 +84281 POINT(41.449997364025634 -74.60591095652866) bank84281 +84282 POINT(41.36145756091555 -73.07546649763655) bank84282 +84283 POINT(40.52047449174859 -73.7956208507137) bank84283 +84284 POINT(41.70920088538998 -74.89920448284154) bank84284 +84285 POINT(40.46327849402454 -74.0564938243179) bank84285 +84286 POINT(40.77265249605238 -74.24791248900321) bank84286 +84287 POINT(39.9242978389626 -74.54166416885079) bank84287 +84288 POINT(40.582178182167176 -75.00441721033336) bank84288 +84289 POINT(40.76074949230062 -73.01583648332796) bank84289 +84290 POINT(41.61367649596907 -73.14777047598936) bank84290 +84291 POINT(41.051344370579 -74.07766245237046) bank84291 +84292 POINT(39.83768718728289 -73.64849699979271) bank84292 +84293 POINT(41.549105428002555 -74.09648681733592) bank84293 +84294 POINT(40.953187686929446 -73.22042148303484) bank84294 +84295 POINT(39.988561911135626 -73.86230834682026) bank84295 +84296 POINT(41.01416765942208 -73.913728958174) bank84296 +84297 POINT(40.2912135560205 -73.74980420334838) bank84297 +84298 POINT(40.60443340506597 -74.39562543942094) bank84298 +84299 POINT(39.999627894031356 -74.15864349531326) bank84299 +84300 POINT(41.06771729719076 -74.3818862392732) bank84300 +84301 POINT(40.21034523584565 -74.64501243907281) bank84301 +84302 POINT(40.3393359169226 -74.96457353428656) bank84302 +84303 POINT(40.88306542978294 -74.19666968071533) bank84303 +84304 POINT(41.10913829993205 -73.75711354722648) bank84304 +84305 POINT(41.45007424780648 -73.61688820497955) bank84305 +84306 POINT(40.20792809343666 -74.63603372125361) bank84306 +84307 POINT(39.71954800987692 -73.20531435388577) bank84307 +84308 POINT(40.83399514351779 -74.93063945067915) bank84308 +84309 POINT(41.41016458844337 -74.16564813039007) bank84309 +84310 POINT(40.36518899860814 -74.72968785672522) bank84310 +84311 POINT(41.15052437458814 -73.1029427697047) bank84311 +84312 POINT(40.252032575644776 -74.05045441564444) bank84312 +84313 POINT(41.02677831212202 -74.71635613193139) bank84313 +84314 POINT(41.6585206295402 -74.90662707118004) bank84314 +84315 POINT(40.101419648794305 -74.94019900239773) bank84315 +84316 POINT(40.25701734944422 -74.74748903370244) bank84316 +84317 POINT(39.988197043692885 -73.76097006054817) bank84317 +84318 POINT(40.593126284298165 -74.87923205459418) bank84318 +84319 POINT(40.74443856845935 -74.12644311978868) bank84319 +84320 POINT(40.841628576932735 -74.05505744395265) bank84320 +84321 POINT(40.12145035093145 -73.08964708409228) bank84321 +84322 POINT(40.996386346225286 -73.3689227466285) bank84322 +84323 POINT(39.837908497587684 -73.5839144964388) bank84323 +84324 POINT(40.187481610739475 -73.28992922077124) bank84324 +84325 POINT(41.205947245299804 -74.18271973635336) bank84325 +84326 POINT(41.1217099515553 -74.29062653119499) bank84326 +84327 POINT(40.835607882085554 -73.5262996873799) bank84327 +84328 POINT(40.79098471482176 -74.30064999149582) bank84328 +84329 POINT(41.48590593965162 -73.83627741370229) bank84329 +84330 POINT(41.212604998440476 -73.54225723250157) bank84330 +84331 POINT(41.60923448637548 -73.72829731224873) bank84331 +84332 POINT(40.4219406051985 -74.99998010182946) bank84332 +84333 POINT(40.01851229605387 -73.31048482223586) bank84333 +84334 POINT(40.97533062391162 -74.2229442619948) bank84334 +84335 POINT(41.149191812693736 -73.85564103159501) bank84335 +84336 POINT(40.29294683567638 -73.17091660963092) bank84336 +84337 POINT(39.89145735167931 -73.44034706227292) bank84337 +84338 POINT(41.692553836900906 -73.52821282425145) bank84338 +84339 POINT(41.05465331332557 -74.66908754103514) bank84339 +84340 POINT(40.15722999459847 -74.13455496821285) bank84340 +84341 POINT(41.09004774887668 -74.56886226644878) bank84341 +84342 POINT(40.56130750549192 -73.50535440628771) bank84342 +84343 POINT(40.4393907805109 -74.98983559822955) bank84343 +84344 POINT(41.65384328182406 -73.1372448178353) bank84344 +84345 POINT(39.74566520305747 -74.78994150365435) bank84345 +84346 POINT(40.037221429469305 -74.56037694355508) bank84346 +84347 POINT(41.3034628486095 -73.08891562451711) bank84347 +84348 POINT(41.61827775284806 -74.58421435050963) bank84348 +84349 POINT(41.607521513853165 -74.39656471349295) bank84349 +84350 POINT(41.526391498942424 -73.23463273350364) bank84350 +84351 POINT(40.44687014265849 -73.45798153887273) bank84351 +84352 POINT(40.07081617282951 -74.6725450687219) bank84352 +84353 POINT(39.805005407497106 -74.35896134637062) bank84353 +84354 POINT(40.98612394040158 -73.53610957054376) bank84354 +84355 POINT(40.19244003363135 -74.8881551874396) bank84355 +84356 POINT(39.93884777638225 -73.24549086732607) bank84356 +84357 POINT(41.012926708412955 -74.21865251410173) bank84357 +84358 POINT(41.37539121971025 -73.0775827977308) bank84358 +84359 POINT(41.190726550166666 -73.78297575714966) bank84359 +84360 POINT(41.37290785302952 -74.27810233964472) bank84360 +84361 POINT(41.08875654957831 -73.2902034539734) bank84361 +84362 POINT(41.05184483242756 -74.17236573690018) bank84362 +84363 POINT(39.76034684504486 -73.40690607983487) bank84363 +84364 POINT(40.90547531090531 -73.91654114592906) bank84364 +84365 POINT(40.59695620621964 -73.77837616098661) bank84365 +84366 POINT(40.22986275835708 -74.82801893085804) bank84366 +84367 POINT(40.95908014004717 -74.845104689899) bank84367 +84368 POINT(41.422065328116 -73.0076949157158) bank84368 +84369 POINT(41.40904639372789 -74.0162767970724) bank84369 +84370 POINT(40.525538046563 -73.52739289629774) bank84370 +84371 POINT(39.78794657779851 -74.37692118582963) bank84371 +84372 POINT(40.91409749766005 -74.07790324908305) bank84372 +84373 POINT(41.069711414815714 -74.30848360307387) bank84373 +84374 POINT(40.65515639544775 -74.4535199304033) bank84374 +84375 POINT(41.50271479935693 -74.72703340146529) bank84375 +84376 POINT(39.91714595680862 -73.80951837772294) bank84376 +84377 POINT(41.1957243728048 -74.72792192870327) bank84377 +84378 POINT(39.75401103350954 -73.83371784946343) bank84378 +84379 POINT(39.83613146503129 -73.97746402456335) bank84379 +84380 POINT(41.057185216305925 -73.46544841565265) bank84380 +84381 POINT(40.45232405393063 -73.91044365103578) bank84381 +84382 POINT(40.40910811719512 -74.89624381410279) bank84382 +84383 POINT(40.679316907638025 -73.11630679466884) bank84383 +84384 POINT(39.764863971349506 -74.00487726749114) bank84384 +84385 POINT(40.988805991309796 -74.9737940674156) bank84385 +84386 POINT(40.60529256115431 -73.38807361960488) bank84386 +84387 POINT(39.94471526017399 -74.79488335061868) bank84387 +84388 POINT(40.80677544425393 -74.56672054328199) bank84388 +84389 POINT(40.65362033459108 -73.66676344769216) bank84389 +84390 POINT(40.32001461223344 -74.59812908628186) bank84390 +84391 POINT(40.012215295673585 -73.54045769962849) bank84391 +84392 POINT(41.70831701582705 -73.35851110131745) bank84392 +84393 POINT(40.55068722533191 -74.85732570590918) bank84393 +84394 POINT(40.09825841938798 -73.62150121031958) bank84394 +84395 POINT(41.662162956057124 -73.80885473180564) bank84395 +84396 POINT(40.73435965837529 -74.38699330162711) bank84396 +84397 POINT(41.414485016511904 -74.79873444647488) bank84397 +84398 POINT(41.610605849389536 -74.75660780272626) bank84398 +84399 POINT(41.52440186427428 -73.41503526080565) bank84399 +84400 POINT(41.23243581844482 -73.09018072066102) bank84400 +84401 POINT(41.32389381915721 -74.91905817392185) bank84401 +84402 POINT(40.100035781577375 -73.35529209713687) bank84402 +84403 POINT(40.77151173345274 -73.3813588648343) bank84403 +84404 POINT(41.034906393570644 -73.93312146327162) bank84404 +84405 POINT(41.068209335921566 -73.49049454606848) bank84405 +84406 POINT(40.60225250872126 -73.83960121485968) bank84406 +84407 POINT(40.96994435358908 -74.10423746347254) bank84407 +84408 POINT(39.84623907782759 -73.8795302459852) bank84408 +84409 POINT(41.00495116509921 -73.017193664918) bank84409 +84410 POINT(40.399458661060365 -73.05750310947738) bank84410 +84411 POINT(41.28766326121548 -74.33107143521772) bank84411 +84412 POINT(40.736594979019415 -74.64368120124233) bank84412 +84413 POINT(41.060478941901515 -73.37769199458667) bank84413 +84414 POINT(39.94669474837528 -74.295287290138) bank84414 +84415 POINT(40.41812110509767 -73.80784176894693) bank84415 +84416 POINT(40.97556287402502 -73.33397609604476) bank84416 +84417 POINT(40.09056424327857 -74.69164333708842) bank84417 +84418 POINT(40.789502635851875 -73.77327454728551) bank84418 +84419 POINT(39.785837759321346 -73.05396214556592) bank84419 +84420 POINT(40.5496102027721 -74.85176124165457) bank84420 +84421 POINT(41.606905086259765 -74.9249926133827) bank84421 +84422 POINT(41.17812558473661 -73.86001054561613) bank84422 +84423 POINT(40.073238602225395 -73.33457386250195) bank84423 +84424 POINT(41.49343470817377 -74.44677152077122) bank84424 +84425 POINT(41.290121040504836 -74.81631729745995) bank84425 +84426 POINT(40.98020396821144 -73.19306879470483) bank84426 +84427 POINT(40.19109359336798 -73.64217828880572) bank84427 +84428 POINT(41.41729002222893 -73.77448119227589) bank84428 +84429 POINT(39.96056187916654 -74.80545306462423) bank84429 +84430 POINT(41.62321674470606 -73.84798000027948) bank84430 +84431 POINT(39.81869641539719 -74.67115225489461) bank84431 +84432 POINT(40.9939603705482 -74.86434915823806) bank84432 +84433 POINT(39.80263680225614 -74.96873996443463) bank84433 +84434 POINT(40.195926633427625 -73.12912924856522) bank84434 +84435 POINT(39.76756521345023 -74.0885649368303) bank84435 +84436 POINT(41.44868394987307 -74.19179426450421) bank84436 +84437 POINT(40.63274618259445 -74.65381759798598) bank84437 +84438 POINT(40.25530198713875 -74.8450831059003) bank84438 +84439 POINT(40.62171017356749 -74.97406858460825) bank84439 +84440 POINT(41.701476001088636 -74.41581218983339) bank84440 +84441 POINT(40.61985569009601 -73.9498997092974) bank84441 +84442 POINT(40.516833899222796 -73.99039245692266) bank84442 +84443 POINT(41.56251013124207 -74.28344974489045) bank84443 +84444 POINT(41.439128847970636 -73.92192801487508) bank84444 +84445 POINT(40.35132949091522 -74.80423593981982) bank84445 +84446 POINT(41.59792441008648 -73.53837791755318) bank84446 +84447 POINT(40.63310687284633 -73.78913202304962) bank84447 +84448 POINT(39.78886680713978 -74.16291831582758) bank84448 +84449 POINT(40.891270503018774 -74.06568073299) bank84449 +84450 POINT(40.17145969874609 -73.07435491137865) bank84450 +84451 POINT(40.49160743953951 -74.40086012597877) bank84451 +84452 POINT(41.44860175913154 -74.56360609739174) bank84452 +84453 POINT(40.43922403428376 -73.33661141406505) bank84453 +84454 POINT(40.094073005603946 -73.32889218984337) bank84454 +84455 POINT(40.80078103563591 -74.13853342935235) bank84455 +84456 POINT(41.05860764447896 -73.1418456892809) bank84456 +84457 POINT(40.29164875084849 -73.29586408886074) bank84457 +84458 POINT(41.6791772156071 -73.86557470688565) bank84458 +84459 POINT(40.10403918786181 -74.04738200149131) bank84459 +84460 POINT(39.93000680929996 -73.45156801381923) bank84460 +84461 POINT(40.912505853126476 -74.81759740912497) bank84461 +84462 POINT(40.73062313369841 -73.6809121185894) bank84462 +84463 POINT(40.37557226652538 -74.99316982924448) bank84463 +84464 POINT(40.14381356031377 -73.9890209821244) bank84464 +84465 POINT(40.501198608500374 -73.36518743133311) bank84465 +84466 POINT(40.14435871166441 -74.33499744530978) bank84466 +84467 POINT(40.22193687655922 -73.69917026553802) bank84467 +84468 POINT(40.00512909296047 -74.0549748546546) bank84468 +84469 POINT(39.847994610325564 -74.34623009320975) bank84469 +84470 POINT(40.959396389320744 -74.70270792473652) bank84470 +84471 POINT(41.44570371336939 -74.57738293516029) bank84471 +84472 POINT(40.258376266856196 -74.0683494951397) bank84472 +84473 POINT(41.5496456171927 -74.03265863880932) bank84473 +84474 POINT(41.32856436184391 -73.75115026608087) bank84474 +84475 POINT(39.839613980831686 -73.27686405779599) bank84475 +84476 POINT(40.841735089558476 -73.34877472642157) bank84476 +84477 POINT(41.433057633135924 -74.48674749046644) bank84477 +84478 POINT(41.246049989174885 -74.83336474722671) bank84478 +84479 POINT(40.120018479569254 -73.01809756111284) bank84479 +84480 POINT(39.871646359615156 -73.9130726900607) bank84480 +84481 POINT(41.209475741404965 -74.2734215838153) bank84481 +84482 POINT(41.07581783930786 -73.19953166241093) bank84482 +84483 POINT(40.12265709917289 -74.86944275424077) bank84483 +84484 POINT(40.039535303595095 -73.44484229682652) bank84484 +84485 POINT(41.10308764757353 -73.7963162820253) bank84485 +84486 POINT(41.01230472466668 -74.05252775749577) bank84486 +84487 POINT(40.5612957908997 -74.68454696221832) bank84487 +84488 POINT(40.170299820307655 -73.0742162329698) bank84488 +84489 POINT(41.24781218860396 -73.93831715139163) bank84489 +84490 POINT(40.18556610405162 -74.1182097678281) bank84490 +84491 POINT(40.28854796277783 -73.82196353253974) bank84491 +84492 POINT(41.011984111047 -74.20419366482263) bank84492 +84493 POINT(39.9466315859464 -74.48174392560908) bank84493 +84494 POINT(40.46301834232628 -73.83148621201754) bank84494 +84495 POINT(40.99333590151626 -74.0953155502776) bank84495 +84496 POINT(41.173892722974344 -73.32674735115735) bank84496 +84497 POINT(39.879783595865014 -73.57095086672224) bank84497 +84498 POINT(40.88251843813588 -74.86837964572341) bank84498 +84499 POINT(41.48323519885123 -73.65038464236989) bank84499 +84500 POINT(41.07473515942309 -74.37880446353363) bank84500 +84501 POINT(40.72807646482938 -73.73604124665427) bank84501 +84502 POINT(41.190572659016226 -73.70853337236709) bank84502 +84503 POINT(40.73370508701244 -73.79172408097641) bank84503 +84504 POINT(41.646410572494105 -73.75360170124559) bank84504 +84505 POINT(39.94803810260234 -73.12444170792546) bank84505 +84506 POINT(40.577872050277584 -74.57236036522359) bank84506 +84507 POINT(40.554309964999 -73.87598797172949) bank84507 +84508 POINT(41.080786234390544 -74.44896643646224) bank84508 +84509 POINT(41.356462332093116 -73.34344337769215) bank84509 +84510 POINT(39.93103266362782 -73.36777451151168) bank84510 +84511 POINT(41.041020510350215 -73.43215030303506) bank84511 +84512 POINT(40.65047090158057 -73.87331449471655) bank84512 +84513 POINT(41.39823227276326 -74.62963891591414) bank84513 +84514 POINT(39.817717975378955 -74.43832226746704) bank84514 +84515 POINT(40.27977216368138 -74.7579299982542) bank84515 +84516 POINT(41.69096740610013 -73.07355057558804) bank84516 +84517 POINT(40.38783071056266 -74.04154143231986) bank84517 +84518 POINT(41.01517508773881 -73.50871234984345) bank84518 +84519 POINT(41.70073955680464 -73.31211319260488) bank84519 +84520 POINT(40.58745627191487 -73.98511324376604) bank84520 +84521 POINT(41.5438382910292 -73.27553148174368) bank84521 +84522 POINT(40.551490190218104 -73.86383574702903) bank84522 +84523 POINT(40.36811239258544 -73.14419613251937) bank84523 +84524 POINT(41.67980284799492 -74.66731821565266) bank84524 +84525 POINT(39.89270136177939 -74.8779366687108) bank84525 +84526 POINT(41.534352309556404 -73.51038617792544) bank84526 +84527 POINT(40.93344532014135 -74.26564861721143) bank84527 +84528 POINT(39.863851037381906 -73.56349019384209) bank84528 +84529 POINT(40.75744669809945 -73.1354564356625) bank84529 +84530 POINT(40.99702672232951 -73.2883575804909) bank84530 +84531 POINT(40.72621664329239 -74.49340637367594) bank84531 +84532 POINT(39.73784292989099 -73.75807739978488) bank84532 +84533 POINT(39.83794769882994 -74.36623670239555) bank84533 +84534 POINT(40.617472670399025 -73.62364941268208) bank84534 +84535 POINT(40.271390649112845 -73.99972445101645) bank84535 +84536 POINT(41.24386751338097 -73.20923539764705) bank84536 +84537 POINT(40.480728432170686 -73.83509131579136) bank84537 +84538 POINT(41.650937750796835 -74.02414239793008) bank84538 +84539 POINT(41.50706732512107 -73.3488291415993) bank84539 +84540 POINT(41.097665327884 -73.1564850318294) bank84540 +84541 POINT(40.43338807823695 -73.4109966198457) bank84541 +84542 POINT(40.69447818032323 -74.99074754540935) bank84542 +84543 POINT(40.4762161877441 -74.43432794424311) bank84543 +84544 POINT(41.05842698705936 -73.13130692448607) bank84544 +84545 POINT(40.41817531244998 -74.93494958964165) bank84545 +84546 POINT(41.52287170064261 -73.18526809135273) bank84546 +84547 POINT(39.905222082229116 -74.8231587822876) bank84547 +84548 POINT(40.49746296915164 -74.05342949055326) bank84548 +84549 POINT(41.30932723242432 -74.10744366843132) bank84549 +84550 POINT(41.099560778573974 -73.60483219908711) bank84550 +84551 POINT(40.076246100330444 -74.73335319244184) bank84551 +84552 POINT(41.10259386197685 -74.40149813818768) bank84552 +84553 POINT(40.55688346065387 -74.64318047197872) bank84553 +84554 POINT(40.6880570914512 -73.72385415709698) bank84554 +84555 POINT(40.43648773219786 -73.31924304416503) bank84555 +84556 POINT(40.602689274828585 -73.23968561719904) bank84556 +84557 POINT(40.1651151373125 -74.74574714675197) bank84557 +84558 POINT(40.81395939570426 -73.41879878726104) bank84558 +84559 POINT(41.28505628375689 -74.88035802015501) bank84559 +84560 POINT(40.0875776205632 -73.18359448467321) bank84560 +84561 POINT(40.34350360246052 -73.4636912862486) bank84561 +84562 POINT(40.346106692505366 -73.84372561325986) bank84562 +84563 POINT(40.81486331207411 -73.43898485771788) bank84563 +84564 POINT(39.9162315504226 -74.92599080724499) bank84564 +84565 POINT(41.66290757225513 -74.7581704356225) bank84565 +84566 POINT(40.01435241529073 -74.54059932008545) bank84566 +84567 POINT(40.80394442772434 -74.12336947052695) bank84567 +84568 POINT(40.422689361049905 -74.52401784711112) bank84568 +84569 POINT(39.77262570796292 -73.57046672795482) bank84569 +84570 POINT(41.4907830489615 -74.77450063059757) bank84570 +84571 POINT(41.09852601864333 -74.51733116691524) bank84571 +84572 POINT(40.60283924346287 -73.9155763088699) bank84572 +84573 POINT(39.909731141256515 -74.66533320196751) bank84573 +84574 POINT(41.66151938176701 -73.81602809197227) bank84574 +84575 POINT(41.341495364727905 -73.99607770345936) bank84575 +84576 POINT(41.69611057991609 -73.1867173343573) bank84576 +84577 POINT(40.12828237164788 -74.79896324839152) bank84577 +84578 POINT(41.47406787547539 -74.96540950352109) bank84578 +84579 POINT(41.25937947328579 -74.59816116587493) bank84579 +84580 POINT(41.28315008711745 -74.94642000214243) bank84580 +84581 POINT(40.06429844054179 -74.61886901571165) bank84581 +84582 POINT(39.8171224413136 -74.11791937334253) bank84582 +84583 POINT(39.944277943351885 -73.64724253707438) bank84583 +84584 POINT(39.825344070795985 -74.35333563649436) bank84584 +84585 POINT(39.80072150472925 -74.53902461336087) bank84585 +84586 POINT(40.39729196837855 -73.90529784122464) bank84586 +84587 POINT(39.956361786808785 -73.22572500278643) bank84587 +84588 POINT(40.40022869350948 -73.41952934287583) bank84588 +84589 POINT(41.50591065300326 -74.53345095724488) bank84589 +84590 POINT(41.135217995615456 -74.47820705766975) bank84590 +84591 POINT(39.90253589046088 -73.02208363074853) bank84591 +84592 POINT(41.53553928463532 -74.93819044299062) bank84592 +84593 POINT(40.76154724203297 -74.41099064180024) bank84593 +84594 POINT(40.05612524152304 -74.67451604151324) bank84594 +84595 POINT(40.4998260879711 -73.69646469290379) bank84595 +84596 POINT(39.98719814822841 -74.97433681162755) bank84596 +84597 POINT(41.13011416642458 -74.08704753257012) bank84597 +84598 POINT(41.527810086265646 -74.20141094123133) bank84598 +84599 POINT(39.745021831472464 -74.83798694706346) bank84599 +84600 POINT(40.77199643945249 -73.96488851336613) bank84600 +84601 POINT(41.59275307406504 -74.0869495435419) bank84601 +84602 POINT(41.141035461877316 -73.18221520811962) bank84602 +84603 POINT(40.45103032915388 -73.74765479448034) bank84603 +84604 POINT(41.620576943146204 -73.24262832961985) bank84604 +84605 POINT(40.955170222069064 -74.28743197456888) bank84605 +84606 POINT(40.324506468277136 -73.45482947269994) bank84606 +84607 POINT(40.54426144495569 -73.35594265475453) bank84607 +84608 POINT(41.31561476802684 -74.39605506011839) bank84608 +84609 POINT(39.757180523866374 -74.96297304781254) bank84609 +84610 POINT(40.95122450749188 -74.27379644067584) bank84610 +84611 POINT(41.5429146848326 -73.57322454635678) bank84611 +84612 POINT(40.632622314581795 -73.32791794932179) bank84612 +84613 POINT(40.16565858867446 -73.53175202402998) bank84613 +84614 POINT(40.14883136084073 -74.14560602073281) bank84614 +84615 POINT(41.490138170798986 -73.77270402238887) bank84615 +84616 POINT(40.4482343571248 -74.14600968637733) bank84616 +84617 POINT(40.32604638567253 -74.77629498376129) bank84617 +84618 POINT(41.19619959737961 -74.00730456414188) bank84618 +84619 POINT(40.37903639447057 -73.10450777231264) bank84619 +84620 POINT(40.96514940344944 -74.42638371482032) bank84620 +84621 POINT(40.46570405149731 -73.3003591875362) bank84621 +84622 POINT(40.3954171838949 -74.56733357588335) bank84622 +84623 POINT(40.26316403188221 -73.72785214045601) bank84623 +84624 POINT(40.9177899426488 -74.37413286474478) bank84624 +84625 POINT(40.67378687754665 -74.28859919852694) bank84625 +84626 POINT(41.5123020544438 -74.59644407796229) bank84626 +84627 POINT(41.1096252997975 -73.85971404736526) bank84627 +84628 POINT(40.76011388451609 -74.52534767493762) bank84628 +84629 POINT(39.87554516907996 -73.80080023350159) bank84629 +84630 POINT(41.607848849607365 -74.41922133602515) bank84630 +84631 POINT(40.04039092601643 -73.17953059069568) bank84631 +84632 POINT(40.63604627161428 -74.70850570809141) bank84632 +84633 POINT(40.106177358463654 -74.22609796341365) bank84633 +84634 POINT(40.238380796078566 -74.99498407685498) bank84634 +84635 POINT(41.17085376922398 -73.67055809427632) bank84635 +84636 POINT(40.5646550330942 -74.63292780608265) bank84636 +84637 POINT(41.06121323682307 -73.19412755187241) bank84637 +84638 POINT(41.1812557869627 -73.62180669882504) bank84638 +84639 POINT(40.329989469931505 -74.47189161874046) bank84639 +84640 POINT(41.09692417370049 -73.21711775361982) bank84640 +84641 POINT(41.26744776424016 -74.42506719218534) bank84641 +84642 POINT(40.67638482296782 -73.29057872304418) bank84642 +84643 POINT(41.26953418485099 -74.4185918945964) bank84643 +84644 POINT(40.08247759842266 -73.18379186482014) bank84644 +84645 POINT(39.73106829025671 -74.90972584869297) bank84645 +84646 POINT(40.11210313010201 -74.57369579048282) bank84646 +84647 POINT(40.12988307680667 -74.26991599427804) bank84647 +84648 POINT(41.08269013322112 -74.01560054070895) bank84648 +84649 POINT(40.610650896687865 -74.53553227909039) bank84649 +84650 POINT(40.82788852333788 -73.6948055064071) bank84650 +84651 POINT(40.3940389973845 -74.24144931532392) bank84651 +84652 POINT(40.113212243407276 -74.29493890431769) bank84652 +84653 POINT(40.625168784782325 -74.3845006383025) bank84653 +84654 POINT(41.48923360841273 -74.51677959900576) bank84654 +84655 POINT(40.206181399278336 -74.05420524308063) bank84655 +84656 POINT(40.36152082875895 -73.18328321112601) bank84656 +84657 POINT(41.51229377901793 -74.95060271244192) bank84657 +84658 POINT(40.04892416836984 -73.83143280746566) bank84658 +84659 POINT(40.27004288050494 -74.03638312566372) bank84659 +84660 POINT(41.12050656023959 -73.73349574342299) bank84660 +84661 POINT(41.3325982691834 -74.09813636254795) bank84661 +84662 POINT(40.660809187588825 -74.02574654217686) bank84662 +84663 POINT(40.97374969806748 -73.26257926588457) bank84663 +84664 POINT(41.23451511992381 -73.18462543414704) bank84664 +84665 POINT(40.94130675608312 -74.24156129180454) bank84665 +84666 POINT(39.99905515368771 -73.88104180131106) bank84666 +84667 POINT(40.930392366941504 -74.94070338944158) bank84667 +84668 POINT(40.67058528380157 -74.77067195233342) bank84668 +84669 POINT(39.74292049961365 -74.48810881055057) bank84669 +84670 POINT(40.67558349806208 -74.08086412610295) bank84670 +84671 POINT(41.09870512574871 -74.2391089804068) bank84671 +84672 POINT(41.19039276867544 -74.23962679115037) bank84672 +84673 POINT(40.05274511747399 -73.59699811952675) bank84673 +84674 POINT(40.63995006060312 -73.43080196488175) bank84674 +84675 POINT(40.80998454377046 -74.37951092588301) bank84675 +84676 POINT(41.198761847980414 -73.06862539790744) bank84676 +84677 POINT(41.22766254909499 -74.67395762543195) bank84677 +84678 POINT(40.72793178382864 -73.34928958771056) bank84678 +84679 POINT(39.7453302776563 -74.16124115087858) bank84679 +84680 POINT(39.71849862431887 -73.0843007117964) bank84680 +84681 POINT(40.506808447824014 -73.30339560478563) bank84681 +84682 POINT(40.37944025508105 -74.9472957562561) bank84682 +84683 POINT(41.49771154485837 -73.27058764797425) bank84683 +84684 POINT(41.64362728006309 -73.55322793490127) bank84684 +84685 POINT(41.48218930730684 -74.20047227442568) bank84685 +84686 POINT(41.5641401580317 -73.97306129061705) bank84686 +84687 POINT(40.53942882945689 -74.43554213746012) bank84687 +84688 POINT(40.62723079367687 -73.75737253402653) bank84688 +84689 POINT(39.850421392314765 -73.37967761322749) bank84689 +84690 POINT(41.17819376572857 -74.28179156544513) bank84690 +84691 POINT(41.5285895238397 -74.51188235271198) bank84691 +84692 POINT(41.15733683796297 -73.21334442145319) bank84692 +84693 POINT(41.057845183309176 -73.6903005120989) bank84693 +84694 POINT(40.464770132893804 -73.4335141475344) bank84694 +84695 POINT(41.282522718186414 -73.51110290483646) bank84695 +84696 POINT(41.67623096310408 -74.93796831889293) bank84696 +84697 POINT(41.025783823148956 -74.69818928794034) bank84697 +84698 POINT(40.9244748370026 -74.61093163578582) bank84698 +84699 POINT(41.391475950830504 -73.66473437255962) bank84699 +84700 POINT(40.57584218072455 -73.36721088024251) bank84700 +84701 POINT(40.14981569831659 -73.57943271173187) bank84701 +84702 POINT(39.71820345921753 -74.48574080947289) bank84702 +84703 POINT(41.70632744460571 -73.05703199737312) bank84703 +84704 POINT(40.67749135611963 -74.15480388836147) bank84704 +84705 POINT(41.250344777438315 -74.69628948025986) bank84705 +84706 POINT(41.42920281877892 -74.05134684245392) bank84706 +84707 POINT(40.56647178265312 -74.80699942227415) bank84707 +84708 POINT(41.252557288463564 -73.662875527249) bank84708 +84709 POINT(40.48807877882314 -74.2526924876649) bank84709 +84710 POINT(40.53007910805703 -73.5196430960439) bank84710 +84711 POINT(41.641915014476666 -73.13733981540861) bank84711 +84712 POINT(41.373195867418396 -74.28786753436553) bank84712 +84713 POINT(39.936405645535686 -74.52983543711198) bank84713 +84714 POINT(40.85878511962939 -74.84563565584743) bank84714 +84715 POINT(40.75989163251433 -74.3455768121262) bank84715 +84716 POINT(41.63192336719442 -74.07635585731312) bank84716 +84717 POINT(40.00149083513579 -74.98135452721101) bank84717 +84718 POINT(40.05994231119503 -73.04997065028809) bank84718 +84719 POINT(40.27968609270565 -74.80336973396341) bank84719 +84720 POINT(40.830548141231645 -74.70386515078674) bank84720 +84721 POINT(41.58057052803734 -74.94946764246801) bank84721 +84722 POINT(41.57517539489694 -74.68574845060886) bank84722 +84723 POINT(41.05410998831819 -73.87945077155366) bank84723 +84724 POINT(39.932689885583976 -74.83859749608173) bank84724 +84725 POINT(39.815278160967424 -73.5816444499451) bank84725 +84726 POINT(40.944125862558955 -73.22259514501148) bank84726 +84727 POINT(41.616560376938104 -73.02227334281856) bank84727 +84728 POINT(40.270288008003426 -74.37982961468362) bank84728 +84729 POINT(40.53160723007249 -74.2016281613851) bank84729 +84730 POINT(40.23858284550324 -73.89800708859941) bank84730 +84731 POINT(41.01656609634922 -74.41455384067017) bank84731 +84732 POINT(40.201831962410594 -73.75851719959674) bank84732 +84733 POINT(41.662669452367425 -74.51948177975642) bank84733 +84734 POINT(41.27438537071286 -74.98938231616266) bank84734 +84735 POINT(41.310079605835995 -74.72554338652976) bank84735 +84736 POINT(40.89037873438225 -74.30058211120277) bank84736 +84737 POINT(40.955014662707114 -74.8993874811159) bank84737 +84738 POINT(40.169948766570464 -74.80995538332667) bank84738 +84739 POINT(41.36867108910464 -74.04589830343527) bank84739 +84740 POINT(41.49913553392188 -74.54344099488688) bank84740 +84741 POINT(40.11584963183554 -74.99438887686877) bank84741 +84742 POINT(41.493981429611054 -74.33426024565375) bank84742 +84743 POINT(40.40999836598825 -74.4340377796324) bank84743 +84744 POINT(40.513611669479864 -73.05045013255739) bank84744 +84745 POINT(41.27549322315377 -73.167786138885) bank84745 +84746 POINT(41.63672623213182 -73.73847002519236) bank84746 +84747 POINT(40.98626047941043 -73.1925448334485) bank84747 +84748 POINT(39.71851027589497 -73.47536945401872) bank84748 +84749 POINT(41.53575441234517 -75.00127222777732) bank84749 +84750 POINT(41.07437550493157 -74.26148844324108) bank84750 +84751 POINT(41.62632557813365 -74.61096986191924) bank84751 +84752 POINT(40.462931754257966 -74.40772310243463) bank84752 +84753 POINT(40.637197555303636 -74.33356178928089) bank84753 +84754 POINT(40.90132508584873 -73.24193913593726) bank84754 +84755 POINT(39.74530505809772 -73.75625589975573) bank84755 +84756 POINT(39.87220502932533 -74.20185421899873) bank84756 +84757 POINT(39.850905643494805 -74.6874058306345) bank84757 +84758 POINT(40.829342243322465 -74.04324241300061) bank84758 +84759 POINT(40.69977458801023 -73.508453278266) bank84759 +84760 POINT(39.97748302011458 -73.30714728624964) bank84760 +84761 POINT(40.53635090426711 -73.96926080906792) bank84761 +84762 POINT(40.44690541580026 -73.98961666492008) bank84762 +84763 POINT(39.95954588058005 -73.73060621672991) bank84763 +84764 POINT(39.85551813460246 -73.59231193817095) bank84764 +84765 POINT(40.50079896856277 -74.38489637688254) bank84765 +84766 POINT(41.173780335045116 -73.7354434933014) bank84766 +84767 POINT(41.68994070267869 -74.56841371916575) bank84767 +84768 POINT(40.41011835056845 -74.81802110850839) bank84768 +84769 POINT(40.065950750450746 -74.62447611581388) bank84769 +84770 POINT(39.921234426919746 -74.22270850091301) bank84770 +84771 POINT(40.16943257372463 -74.13681315576234) bank84771 +84772 POINT(41.68667068583604 -74.26965374270864) bank84772 +84773 POINT(40.22848142637436 -74.4919909899997) bank84773 +84774 POINT(39.77571204789145 -73.44259973428758) bank84774 +84775 POINT(41.34462963610966 -73.92069830846158) bank84775 +84776 POINT(40.8763310834461 -73.03299912316028) bank84776 +84777 POINT(40.13916360819931 -74.06263427410556) bank84777 +84778 POINT(40.000541888404406 -74.88711591337795) bank84778 +84779 POINT(40.93213755553653 -73.16290169056613) bank84779 +84780 POINT(40.86405042365959 -74.21378265603309) bank84780 +84781 POINT(40.115583052798776 -73.36048235658369) bank84781 +84782 POINT(40.08507350929973 -74.3733687502346) bank84782 +84783 POINT(41.62841309016025 -74.63731258457913) bank84783 +84784 POINT(39.80194747763535 -74.65812767608514) bank84784 +84785 POINT(40.54138741294097 -74.67661883111107) bank84785 +84786 POINT(41.254030077284966 -74.51530719570381) bank84786 +84787 POINT(39.977046134474435 -73.88329412923119) bank84787 +84788 POINT(39.87179717610687 -73.01402366255985) bank84788 +84789 POINT(41.131633639115165 -73.35850657483512) bank84789 +84790 POINT(39.86399666912082 -74.665093757058) bank84790 +84791 POINT(41.28577845712341 -73.52585709606772) bank84791 +84792 POINT(40.12982019008946 -74.74655704133868) bank84792 +84793 POINT(40.27405847399846 -73.19887651668186) bank84793 +84794 POINT(40.6664749349979 -74.747276721731) bank84794 +84795 POINT(41.36275656153659 -74.72058644484767) bank84795 +84796 POINT(40.36543483968564 -74.03340772864931) bank84796 +84797 POINT(41.28045828521916 -74.67284785698274) bank84797 +84798 POINT(41.014342777005844 -73.23701366274867) bank84798 +84799 POINT(40.54943300928613 -73.40025980565679) bank84799 +84800 POINT(40.09763749405926 -73.46717072675816) bank84800 +84801 POINT(41.15546654189102 -74.67966163757112) bank84801 +84802 POINT(40.557975281621786 -73.3567093825254) bank84802 +84803 POINT(40.29457446942367 -74.91028583666053) bank84803 +84804 POINT(40.112074050679546 -73.54593877463726) bank84804 +84805 POINT(40.657780177760344 -73.31090843264155) bank84805 +84806 POINT(40.38243412869613 -74.8159621058136) bank84806 +84807 POINT(40.41857972567915 -74.23479789018558) bank84807 +84808 POINT(41.086061505695234 -73.50446499883114) bank84808 +84809 POINT(40.48199848715313 -73.07793010065568) bank84809 +84810 POINT(40.45278792947293 -73.07395790590107) bank84810 +84811 POINT(40.67996862938051 -73.18747823155476) bank84811 +84812 POINT(41.57265075291523 -73.23066498023056) bank84812 +84813 POINT(40.94810797007629 -74.81549223560262) bank84813 +84814 POINT(39.802748769134865 -74.97498217071397) bank84814 +84815 POINT(40.48809744088646 -74.67113082248122) bank84815 +84816 POINT(40.38973056407589 -74.44873869631175) bank84816 +84817 POINT(39.906283492403816 -74.24375795043713) bank84817 +84818 POINT(40.10218280368764 -74.01235458741152) bank84818 +84819 POINT(41.55152845232754 -73.66885511092772) bank84819 +84820 POINT(40.6292726610261 -73.46425152463942) bank84820 +84821 POINT(40.862689044550315 -73.85672871820836) bank84821 +84822 POINT(41.444699031259226 -74.09778064262409) bank84822 +84823 POINT(40.036640904157565 -73.26244529171333) bank84823 +84824 POINT(40.26529017326103 -73.51617069198143) bank84824 +84825 POINT(41.58685353506926 -74.77561505010932) bank84825 +84826 POINT(40.58708066180546 -73.72547930160066) bank84826 +84827 POINT(40.876068325626505 -73.4820631393928) bank84827 +84828 POINT(40.6897259915408 -73.30877666545582) bank84828 +84829 POINT(40.20934256911659 -73.93532107785578) bank84829 +84830 POINT(41.216079678493095 -74.74812000390277) bank84830 +84831 POINT(40.833662210019014 -73.32922194808013) bank84831 +84832 POINT(40.63098496304144 -73.59963403751766) bank84832 +84833 POINT(41.71212340156852 -74.45630258386923) bank84833 +84834 POINT(40.104855347999305 -73.6953581984422) bank84834 +84835 POINT(41.3061377631014 -74.82617498605357) bank84835 +84836 POINT(40.16021814481767 -73.85611216224282) bank84836 +84837 POINT(41.31066373589913 -73.85243979377604) bank84837 +84838 POINT(41.3204240084701 -74.58287980066117) bank84838 +84839 POINT(41.28725991943088 -74.42894737042819) bank84839 +84840 POINT(41.544267590135064 -73.7756804598556) bank84840 +84841 POINT(41.294685853157766 -73.91855897947396) bank84841 +84842 POINT(41.61612125753199 -73.45145004049428) bank84842 +84843 POINT(41.445684692505 -73.93992025892902) bank84843 +84844 POINT(40.364403302622804 -74.50898981114759) bank84844 +84845 POINT(40.12407447299809 -74.10572059364259) bank84845 +84846 POINT(41.38696056417165 -73.32780067549444) bank84846 +84847 POINT(41.653412739219725 -74.17239054017794) bank84847 +84848 POINT(41.35897675677281 -74.56345455678938) bank84848 +84849 POINT(41.385818588101884 -74.99007879783602) bank84849 +84850 POINT(40.51494275861997 -74.8082421762788) bank84850 +84851 POINT(39.93363067124766 -73.50459903451599) bank84851 +84852 POINT(40.093593931617626 -74.79371317081812) bank84852 +84853 POINT(41.683920827175136 -74.25658294823322) bank84853 +84854 POINT(40.94091046144033 -73.52495203627328) bank84854 +84855 POINT(40.32642740295702 -73.01134268656399) bank84855 +84856 POINT(41.28589329631548 -74.6861544820149) bank84856 +84857 POINT(40.032083400422074 -73.14711972509939) bank84857 +84858 POINT(40.04749926216747 -73.37717511709907) bank84858 +84859 POINT(39.83704374337685 -74.8083783407986) bank84859 +84860 POINT(40.66092626052267 -73.74130249977284) bank84860 +84861 POINT(40.91712990140134 -74.87198097855756) bank84861 +84862 POINT(40.23210467310656 -73.9866574230352) bank84862 +84863 POINT(41.63867332609104 -73.57218052350514) bank84863 +84864 POINT(39.972327614074615 -74.51465423943523) bank84864 +84865 POINT(40.55846264864788 -73.74846377232417) bank84865 +84866 POINT(40.716468681456305 -73.02552690949321) bank84866 +84867 POINT(40.69823852015083 -73.85169866698402) bank84867 +84868 POINT(41.58324702216263 -73.40809953502256) bank84868 +84869 POINT(41.212103170777844 -73.26498310579211) bank84869 +84870 POINT(39.83912530831482 -74.10612444989273) bank84870 +84871 POINT(40.2449345947182 -74.49889504220907) bank84871 +84872 POINT(40.81187329477734 -73.78542038147631) bank84872 +84873 POINT(40.49063717937082 -74.16256661867979) bank84873 +84874 POINT(39.87399308340578 -74.39855178728946) bank84874 +84875 POINT(41.027623724185005 -73.11980585153046) bank84875 +84876 POINT(41.010346211552616 -73.83809588878907) bank84876 +84877 POINT(40.8792162780867 -74.51372532789335) bank84877 +84878 POINT(40.98940810979579 -73.28030891876057) bank84878 +84879 POINT(41.57883280715701 -74.91019287879686) bank84879 +84880 POINT(40.82448476882123 -73.88824238850015) bank84880 +84881 POINT(40.31809828548789 -74.05491723043333) bank84881 +84882 POINT(40.97127574445273 -73.61929024775976) bank84882 +84883 POINT(40.85640854151398 -74.14119157851907) bank84883 +84884 POINT(40.59966117174371 -74.50568262108466) bank84884 +84885 POINT(41.582673992858396 -74.32625079807679) bank84885 +84886 POINT(40.50646853002921 -73.38896799756063) bank84886 +84887 POINT(40.81883027566682 -74.62386583010938) bank84887 +84888 POINT(40.90912515178708 -73.3369354087366) bank84888 +84889 POINT(39.83435417995822 -74.21303155225623) bank84889 +84890 POINT(40.782185110519166 -73.21330192320562) bank84890 +84891 POINT(41.65513295222468 -73.44782279963803) bank84891 +84892 POINT(40.03792139599292 -74.76054532147029) bank84892 +84893 POINT(41.48041224541807 -73.37514183674108) bank84893 +84894 POINT(39.73314010940572 -73.19077352026507) bank84894 +84895 POINT(39.931817156583186 -74.10288474521528) bank84895 +84896 POINT(40.35220243013797 -73.81159514786968) bank84896 +84897 POINT(41.09138377054217 -73.55539953555069) bank84897 +84898 POINT(40.75955504869046 -74.66802140925672) bank84898 +84899 POINT(41.11254232306564 -73.9726007817537) bank84899 +84900 POINT(40.903539966241254 -74.10027097120431) bank84900 +84901 POINT(40.3656847888321 -74.96789230233595) bank84901 +84902 POINT(40.9934075901239 -73.21505984005927) bank84902 +84903 POINT(41.67668746186731 -73.71650502977276) bank84903 +84904 POINT(41.300174937457264 -73.61586924250868) bank84904 +84905 POINT(41.460682477285594 -73.79265689833008) bank84905 +84906 POINT(40.73752714683875 -73.6014001884867) bank84906 +84907 POINT(41.575581596853254 -73.16507531439542) bank84907 +84908 POINT(41.503497107553684 -74.96360861703407) bank84908 +84909 POINT(40.68559956254179 -74.95137046355369) bank84909 +84910 POINT(39.789185543985404 -73.83718695692052) bank84910 +84911 POINT(41.02771196894106 -74.24175933449371) bank84911 +84912 POINT(41.22069270303993 -73.9404213781531) bank84912 +84913 POINT(41.124157967814156 -73.62659322049389) bank84913 +84914 POINT(41.156697489080955 -73.83635094888) bank84914 +84915 POINT(40.82496933562146 -74.61826242726168) bank84915 +84916 POINT(40.95059832143513 -73.93862380541445) bank84916 +84917 POINT(40.01636666132641 -74.3267893157411) bank84917 +84918 POINT(40.93395072150646 -74.02430290903521) bank84918 +84919 POINT(40.02264303199104 -74.75439532811976) bank84919 +84920 POINT(40.838285394230944 -73.8301923392002) bank84920 +84921 POINT(40.114127041091194 -73.57963424702876) bank84921 +84922 POINT(41.31660242917129 -74.59732980615755) bank84922 +84923 POINT(41.649213607895916 -74.30562097021756) bank84923 +84924 POINT(41.59371275932158 -73.00941742941285) bank84924 +84925 POINT(40.50803287524921 -73.07311829939177) bank84925 +84926 POINT(40.74031895922325 -73.9728209128309) bank84926 +84927 POINT(40.1356336869524 -74.23389592543498) bank84927 +84928 POINT(41.44900218732689 -74.86941720095375) bank84928 +84929 POINT(39.83797695705841 -73.31987084262131) bank84929 +84930 POINT(40.608438955485 -74.16099060723535) bank84930 +84931 POINT(40.79553332979989 -73.01901262671726) bank84931 +84932 POINT(40.10098836478961 -73.136769181056) bank84932 +84933 POINT(40.95501279698334 -74.2590267197593) bank84933 +84934 POINT(41.232782922840315 -73.4240722414844) bank84934 +84935 POINT(41.37809933341888 -73.88943679938515) bank84935 +84936 POINT(40.71072026669034 -73.11869807533614) bank84936 +84937 POINT(41.0859000482939 -74.20608349019655) bank84937 +84938 POINT(41.54704321990709 -73.17049283459333) bank84938 +84939 POINT(41.1724095094813 -74.8884541338713) bank84939 +84940 POINT(41.094166471861875 -73.75260319477121) bank84940 +84941 POINT(41.55567118956066 -73.93209658652412) bank84941 +84942 POINT(40.30231827654861 -74.92124665652788) bank84942 +84943 POINT(41.66254328548128 -73.01667893304435) bank84943 +84944 POINT(40.49462925709484 -73.25342415809915) bank84944 +84945 POINT(39.868698669620386 -74.36616565768234) bank84945 +84946 POINT(39.818913394592805 -74.21477539589395) bank84946 +84947 POINT(40.970787610424864 -73.88754223347817) bank84947 +84948 POINT(40.49178646012103 -74.84998070182417) bank84948 +84949 POINT(41.03063178874928 -73.48723712960057) bank84949 +84950 POINT(41.564806087125106 -73.1735683139627) bank84950 +84951 POINT(40.119129474574684 -74.50473229992552) bank84951 +84952 POINT(39.909320439471756 -74.52906598200029) bank84952 +84953 POINT(39.95475410343878 -74.43805893397783) bank84953 +84954 POINT(40.367159466255885 -73.2382801494491) bank84954 +84955 POINT(40.605451614788215 -75.00519671891983) bank84955 +84956 POINT(40.73048968207885 -73.6128404896542) bank84956 +84957 POINT(39.71697343211159 -73.72111651142089) bank84957 +84958 POINT(41.65228681748399 -73.92271827475503) bank84958 +84959 POINT(40.4511448641876 -74.37949289349469) bank84959 +84960 POINT(41.45806877649533 -74.59544276587789) bank84960 +84961 POINT(40.4433725962405 -74.65702810233529) bank84961 +84962 POINT(40.718961814609216 -73.50717397896987) bank84962 +84963 POINT(40.52960631239541 -73.37234720955348) bank84963 +84964 POINT(40.545578043672904 -74.78646314406241) bank84964 +84965 POINT(40.29059141743499 -74.70114300434363) bank84965 +84966 POINT(41.328977578854484 -73.24397213862444) bank84966 +84967 POINT(40.088049559099545 -74.06925230337431) bank84967 +84968 POINT(40.89715823207511 -73.90683770485657) bank84968 +84969 POINT(40.713246357332004 -73.08065099638104) bank84969 +84970 POINT(40.76656798153187 -74.7886345156177) bank84970 +84971 POINT(40.22967352633475 -73.88017020297485) bank84971 +84972 POINT(40.239439912164514 -74.94320677712972) bank84972 +84973 POINT(39.73836711843705 -74.03693791029112) bank84973 +84974 POINT(39.89921687903448 -74.70188703382212) bank84974 +84975 POINT(40.08370519023528 -73.84120912712) bank84975 +84976 POINT(41.49033111326625 -74.34055308905947) bank84976 +84977 POINT(39.739100900657675 -74.68113543606641) bank84977 +84978 POINT(40.95927176601938 -74.82528538591924) bank84978 +84979 POINT(40.975381063111975 -73.50913977802452) bank84979 +84980 POINT(39.81743105819918 -74.9090537945542) bank84980 +84981 POINT(40.512981322188445 -74.89313972416146) bank84981 +84982 POINT(40.549670798997006 -74.90271085145447) bank84982 +84983 POINT(40.5774259528314 -74.67855244995374) bank84983 +84984 POINT(40.333256965127816 -73.82600381621678) bank84984 +84985 POINT(40.81835169992243 -73.40741820844381) bank84985 +84986 POINT(40.811033643951305 -74.76940111733512) bank84986 +84987 POINT(40.478783915990945 -73.46542304905375) bank84987 +84988 POINT(39.90389863827387 -73.04790995239635) bank84988 +84989 POINT(41.560786514934435 -74.05667019652658) bank84989 +84990 POINT(40.12569827073639 -74.40449340197374) bank84990 +84991 POINT(41.16456657605592 -73.97731841482032) bank84991 +84992 POINT(39.84486001555599 -74.92756881911824) bank84992 +84993 POINT(40.996865048489276 -74.03012968960242) bank84993 +84994 POINT(41.423550217966934 -73.59412233136383) bank84994 +84995 POINT(40.70658923783036 -73.64571925977653) bank84995 +84996 POINT(40.87689161664805 -74.94485977781473) bank84996 +84997 POINT(40.437454293415996 -74.78612724598275) bank84997 +84998 POINT(40.54463906444031 -74.36089569120622) bank84998 +84999 POINT(39.71988474396065 -73.45906960504895) bank84999 +85000 POINT(40.09997716092424 -73.66059101097258) bank85000 +85001 POINT(41.036610742076114 -74.33576356317825) bank85001 +85002 POINT(41.22943336436676 -74.74308701179287) bank85002 +85003 POINT(40.22303048031128 -74.25734698668286) bank85003 +85004 POINT(40.1744141618008 -74.86655795916616) bank85004 +85005 POINT(41.42339978902429 -73.01252129654392) bank85005 +85006 POINT(41.29512755552787 -74.64000958673253) bank85006 +85007 POINT(40.758777612883904 -74.05298441764477) bank85007 +85008 POINT(40.53175446010103 -73.59744275383271) bank85008 +85009 POINT(40.70693403681554 -73.35103559596051) bank85009 +85010 POINT(40.15958505823829 -74.5125698789914) bank85010 +85011 POINT(40.65672670047348 -73.28928648780408) bank85011 +85012 POINT(41.34148327866632 -74.93654991567871) bank85012 +85013 POINT(39.85016426231614 -74.83333789363478) bank85013 +85014 POINT(39.86616825966682 -74.9694930661221) bank85014 +85015 POINT(41.612034950806674 -73.94579682365661) bank85015 +85016 POINT(41.63680157609499 -74.554697322236) bank85016 +85017 POINT(39.85049157482111 -73.55540475413463) bank85017 +85018 POINT(41.293919535409856 -73.52044128457068) bank85018 +85019 POINT(39.922154262715374 -73.09460085290198) bank85019 +85020 POINT(40.372594519315676 -73.53953391956611) bank85020 +85021 POINT(40.79651619486534 -74.48741976410541) bank85021 +85022 POINT(39.75275295265146 -73.81467565006602) bank85022 +85023 POINT(40.215152871545406 -73.57090298480728) bank85023 +85024 POINT(39.72721079851383 -73.35863445277978) bank85024 +85025 POINT(40.13687281701386 -73.38334711511563) bank85025 +85026 POINT(41.676759342932684 -73.981282795165) bank85026 +85027 POINT(41.171291427148404 -74.65137363384606) bank85027 +85028 POINT(40.754155916347756 -73.2794247039174) bank85028 +85029 POINT(40.438977155997605 -73.53905411710808) bank85029 +85030 POINT(39.96918270513354 -74.54591746214899) bank85030 +85031 POINT(40.93070876565184 -74.60991138055743) bank85031 +85032 POINT(41.369153884005534 -73.89564681990414) bank85032 +85033 POINT(40.24373917603813 -73.59411998505726) bank85033 +85034 POINT(41.047945963422706 -74.76622437174319) bank85034 +85035 POINT(40.764241122333395 -74.0013550426525) bank85035 +85036 POINT(41.582746536304995 -74.42935892012947) bank85036 +85037 POINT(40.73143606780782 -74.60427149217564) bank85037 +85038 POINT(40.29602914107951 -74.31828155649913) bank85038 +85039 POINT(40.97151311751762 -73.92024483446903) bank85039 +85040 POINT(41.14497464732862 -74.15764068140321) bank85040 +85041 POINT(39.86366983235979 -74.99986320715934) bank85041 +85042 POINT(41.10898150756226 -74.97282887126794) bank85042 +85043 POINT(40.14248211724147 -73.43354897798963) bank85043 +85044 POINT(41.27927719474472 -74.8112383264835) bank85044 +85045 POINT(41.39927201687933 -74.22481870223564) bank85045 +85046 POINT(40.405447813205654 -73.31427447592007) bank85046 +85047 POINT(41.34328226284833 -74.47561773419595) bank85047 +85048 POINT(39.983456742562154 -74.81305053483145) bank85048 +85049 POINT(39.96840436683086 -73.50122388996502) bank85049 +85050 POINT(41.59824890545442 -74.3198765243035) bank85050 +85051 POINT(40.08621482208224 -73.6655215297885) bank85051 +85052 POINT(39.85314622659855 -74.81339224819315) bank85052 +85053 POINT(39.79315835074502 -73.78577306667724) bank85053 +85054 POINT(39.95273729945326 -73.07551563074644) bank85054 +85055 POINT(39.850824664426604 -73.20920033689636) bank85055 +85056 POINT(40.96190873682981 -74.22631223176076) bank85056 +85057 POINT(41.389748458193296 -73.21364142288785) bank85057 +85058 POINT(41.27254252319876 -73.39018823788936) bank85058 +85059 POINT(40.75747465898847 -74.87773355077275) bank85059 +85060 POINT(41.3887064827714 -74.53124047738072) bank85060 +85061 POINT(40.73303802367807 -73.3999006987162) bank85061 +85062 POINT(41.319314944055826 -73.20991340867712) bank85062 +85063 POINT(41.48690414991477 -73.47332865013158) bank85063 +85064 POINT(41.16926675801132 -73.28094783679671) bank85064 +85065 POINT(40.40091073267544 -73.76945030694085) bank85065 +85066 POINT(40.9787145559975 -74.09762826974138) bank85066 +85067 POINT(40.899621807808956 -73.41736338239787) bank85067 +85068 POINT(41.65172628770157 -73.27283696087243) bank85068 +85069 POINT(40.461753049348005 -74.03748794336605) bank85069 +85070 POINT(40.30541680632213 -74.244493381665) bank85070 +85071 POINT(41.64092223536725 -74.61497934822978) bank85071 +85072 POINT(40.76938507229932 -73.67415231177539) bank85072 +85073 POINT(41.125617360212765 -74.61741039397714) bank85073 +85074 POINT(41.706579796654836 -74.5851490530072) bank85074 +85075 POINT(40.75591755981383 -73.91525310158757) bank85075 +85076 POINT(40.17709686430205 -73.29161475398374) bank85076 +85077 POINT(41.41498147624466 -73.23567175420773) bank85077 +85078 POINT(39.98026079287915 -73.13366208389589) bank85078 +85079 POINT(41.24492281842732 -73.51425364775899) bank85079 +85080 POINT(40.685338617725336 -74.06444160353797) bank85080 +85081 POINT(40.55605324919176 -74.62820475571225) bank85081 +85082 POINT(41.647025769837654 -74.00989381809035) bank85082 +85083 POINT(39.88382255162237 -74.72790770062362) bank85083 +85084 POINT(41.05922674697377 -74.97050719703512) bank85084 +85085 POINT(40.43362577299296 -73.95197044770096) bank85085 +85086 POINT(41.698501651675514 -74.9074226916497) bank85086 +85087 POINT(40.61987167887568 -74.09389114287475) bank85087 +85088 POINT(41.12573616842751 -73.77927319971721) bank85088 +85089 POINT(39.9412616635563 -73.35414203582538) bank85089 +85090 POINT(40.48740115988684 -74.72003235136691) bank85090 +85091 POINT(40.190652595441776 -73.89630609528737) bank85091 +85092 POINT(39.92605475724695 -73.54259712785704) bank85092 +85093 POINT(40.86458096934054 -73.57817180303206) bank85093 +85094 POINT(41.67377128743801 -74.86467388139735) bank85094 +85095 POINT(39.7595349674778 -74.02200621120346) bank85095 +85096 POINT(41.34351920581894 -73.08076268023126) bank85096 +85097 POINT(39.865458900896265 -74.16981104145206) bank85097 +85098 POINT(40.69759282656298 -75.00191200818067) bank85098 +85099 POINT(41.50604276637619 -73.74692222932076) bank85099 +85100 POINT(40.845756842985914 -73.0535240341979) bank85100 +85101 POINT(41.03319390481358 -74.31465713899337) bank85101 +85102 POINT(41.042568998256165 -73.6031439279739) bank85102 +85103 POINT(41.47779952257438 -74.94712765536505) bank85103 +85104 POINT(40.53453685776515 -73.01694098416692) bank85104 +85105 POINT(41.398178349040045 -73.39243032044203) bank85105 +85106 POINT(40.09790626126406 -74.69678136380342) bank85106 +85107 POINT(41.34001172270179 -73.70985994070239) bank85107 +85108 POINT(41.124003093404156 -73.91651959368981) bank85108 +85109 POINT(39.97212707728284 -74.23555338230385) bank85109 +85110 POINT(40.781048163237564 -74.56672089166366) bank85110 +85111 POINT(40.82808306707675 -73.45446087205482) bank85111 +85112 POINT(40.12284264483701 -74.76634638848341) bank85112 +85113 POINT(41.444682457608124 -74.61576495800689) bank85113 +85114 POINT(41.01489254212199 -74.13496295379761) bank85114 +85115 POINT(40.696862810732334 -74.62261653743505) bank85115 +85116 POINT(40.63268772414794 -74.549924977327) bank85116 +85117 POINT(40.10272861628408 -74.02864808272466) bank85117 +85118 POINT(41.039409774600344 -74.33779077484108) bank85118 +85119 POINT(40.35651830328148 -73.72430784607774) bank85119 +85120 POINT(40.84487810847415 -73.57750747177874) bank85120 +85121 POINT(41.68375136664019 -73.4319899302181) bank85121 +85122 POINT(41.21954759994425 -73.23861483160245) bank85122 +85123 POINT(41.57899146813561 -73.31825977889012) bank85123 +85124 POINT(41.37415640460921 -73.36614353572368) bank85124 +85125 POINT(41.558050172116715 -73.81712165815325) bank85125 +85126 POINT(41.17579104099334 -73.67994271057631) bank85126 +85127 POINT(40.218802382046626 -73.09062298284475) bank85127 +85128 POINT(40.54739606760098 -73.31532743795643) bank85128 +85129 POINT(40.79767400372498 -74.31048734996729) bank85129 +85130 POINT(40.6124028083335 -74.4009389356236) bank85130 +85131 POINT(40.086527899552195 -73.73726997598789) bank85131 +85132 POINT(39.88525108937029 -73.95245547305767) bank85132 +85133 POINT(41.25382296191422 -73.437359494012) bank85133 +85134 POINT(41.54117212615904 -74.01862014617886) bank85134 +85135 POINT(40.87395651743432 -74.2969696231571) bank85135 +85136 POINT(40.532473012407266 -74.59136902176279) bank85136 +85137 POINT(40.46817630574528 -74.55817997926313) bank85137 +85138 POINT(41.54472030550831 -73.33094921177675) bank85138 +85139 POINT(40.9914564342725 -74.14020764529224) bank85139 +85140 POINT(40.114340971689344 -73.22176267075733) bank85140 +85141 POINT(40.812594676417426 -74.2667928791966) bank85141 +85142 POINT(40.5476577407644 -74.11611417349566) bank85142 +85143 POINT(40.082420342122894 -73.25995687645087) bank85143 +85144 POINT(39.71603840448094 -74.38790573024647) bank85144 +85145 POINT(40.34643785930145 -74.3303637222704) bank85145 +85146 POINT(40.662817137962286 -73.4601113022787) bank85146 +85147 POINT(41.69954867591475 -73.61413441098489) bank85147 +85148 POINT(39.908952796370485 -73.1875346337249) bank85148 +85149 POINT(40.88928628672684 -74.19897994260576) bank85149 +85150 POINT(41.6919642143253 -74.27109133229216) bank85150 +85151 POINT(40.35149197005738 -73.52166109152591) bank85151 +85152 POINT(40.721423240136744 -74.1339250973226) bank85152 +85153 POINT(40.992458165125335 -73.62858436284309) bank85153 +85154 POINT(40.252072056433214 -73.60028265676686) bank85154 +85155 POINT(40.4331525325446 -74.26474774719068) bank85155 +85156 POINT(41.1648453045575 -74.09837678440088) bank85156 +85157 POINT(39.87113224593234 -74.33737979780194) bank85157 +85158 POINT(40.86693276764325 -73.40264474559845) bank85158 +85159 POINT(39.75231271191417 -73.50236999737065) bank85159 +85160 POINT(40.18445830005736 -74.64728682688073) bank85160 +85161 POINT(40.81521247646507 -74.82679034904751) bank85161 +85162 POINT(41.182386507617494 -73.4543780819507) bank85162 +85163 POINT(41.509438192893974 -74.36798669788354) bank85163 +85164 POINT(39.82527838382186 -73.5079730470942) bank85164 +85165 POINT(41.622152202122265 -74.26866522251346) bank85165 +85166 POINT(40.27896933552042 -74.43129495786805) bank85166 +85167 POINT(41.20212149630541 -74.86341957236843) bank85167 +85168 POINT(41.45396923184976 -74.67002868700361) bank85168 +85169 POINT(41.20293930405165 -73.58703776415001) bank85169 +85170 POINT(40.89057972246397 -73.3956978042623) bank85170 +85171 POINT(41.026213557588136 -74.6283672107424) bank85171 +85172 POINT(40.896898489070296 -73.03876180873921) bank85172 +85173 POINT(40.847857452765794 -74.72137699743531) bank85173 +85174 POINT(41.46108103155562 -74.42802745048088) bank85174 +85175 POINT(39.83848046948487 -73.11200807224401) bank85175 +85176 POINT(41.68453704174708 -74.01294163198305) bank85176 +85177 POINT(40.94256642407012 -73.79566499729762) bank85177 +85178 POINT(41.18720565874482 -74.7425530338316) bank85178 +85179 POINT(40.41808254789211 -73.41692508520214) bank85179 +85180 POINT(40.829886286413746 -74.63680597699418) bank85180 +85181 POINT(39.852714253435124 -74.65801436785634) bank85181 +85182 POINT(39.93976610987531 -74.21595244319) bank85182 +85183 POINT(39.846996037978 -74.22541167996162) bank85183 +85184 POINT(41.23294210540167 -74.24646237642625) bank85184 +85185 POINT(41.57719402232551 -73.37548938254601) bank85185 +85186 POINT(41.28973669384464 -73.25819898661217) bank85186 +85187 POINT(40.756031494392985 -74.62516600387397) bank85187 +85188 POINT(40.23199130132109 -73.36917561186937) bank85188 +85189 POINT(41.51119275526857 -73.35839036537955) bank85189 +85190 POINT(41.2464589081347 -74.22933709134134) bank85190 +85191 POINT(39.90134177118955 -74.55945598120003) bank85191 +85192 POINT(40.90254090459627 -74.15318084015597) bank85192 +85193 POINT(41.25259797935546 -74.96736348444628) bank85193 +85194 POINT(40.743161962653545 -74.05833667193318) bank85194 +85195 POINT(41.288589908670886 -74.1991566379345) bank85195 +85196 POINT(40.351439904701884 -74.27059209999557) bank85196 +85197 POINT(41.53242550526716 -74.81727435526867) bank85197 +85198 POINT(40.65998711717424 -73.48682761992556) bank85198 +85199 POINT(40.003400372802275 -73.67252259171742) bank85199 +85200 POINT(39.71585475369761 -74.51246279302502) bank85200 +85201 POINT(41.28693881949123 -74.74157148627299) bank85201 +85202 POINT(39.80321919029076 -74.90402570988746) bank85202 +85203 POINT(41.28999543895552 -73.4044230974418) bank85203 +85204 POINT(39.94056969286963 -74.74778198541011) bank85204 +85205 POINT(40.20715813911599 -73.99929636413091) bank85205 +85206 POINT(39.7134001943001 -74.0374307335982) bank85206 +85207 POINT(41.062824666362694 -74.26762381440773) bank85207 +85208 POINT(40.36001642852544 -73.17235586130728) bank85208 +85209 POINT(41.300703678162826 -74.97583900236906) bank85209 +85210 POINT(41.59751376648717 -74.3027031740651) bank85210 +85211 POINT(40.704244858846316 -73.97882372314862) bank85211 +85212 POINT(40.97725433248919 -74.60451869519011) bank85212 +85213 POINT(41.085627151533295 -74.5783902709896) bank85213 +85214 POINT(41.62049222890431 -73.23732992063715) bank85214 +85215 POINT(41.69150972827606 -73.59340767465127) bank85215 +85216 POINT(41.04855315743367 -74.45975801321485) bank85216 +85217 POINT(41.03762709410081 -73.75860578689412) bank85217 +85218 POINT(39.72939141005574 -74.24092598189857) bank85218 +85219 POINT(40.86910825339349 -74.91671052989699) bank85219 +85220 POINT(41.531582217027484 -73.26268999560611) bank85220 +85221 POINT(39.72200992503905 -73.12804631457549) bank85221 +85222 POINT(41.26300584363377 -74.4094808811335) bank85222 +85223 POINT(40.704571097390804 -74.25164780542362) bank85223 +85224 POINT(40.11625441101617 -74.84063374321484) bank85224 +85225 POINT(39.87004784380275 -74.866947639041) bank85225 +85226 POINT(41.56451778818105 -73.39035462838817) bank85226 +85227 POINT(39.971990354479175 -73.25264555629037) bank85227 +85228 POINT(41.29761375064879 -73.83620074096868) bank85228 +85229 POINT(41.43298803993236 -74.3971510303909) bank85229 +85230 POINT(41.21035969885055 -74.23885500216336) bank85230 +85231 POINT(40.32629377574127 -73.51461694209505) bank85231 +85232 POINT(41.08825010783134 -73.03002424257551) bank85232 +85233 POINT(40.78326147954566 -73.02482292498144) bank85233 +85234 POINT(40.23262023436668 -74.59146212732384) bank85234 +85235 POINT(40.17956827221207 -74.60823926941639) bank85235 +85236 POINT(40.97084900570728 -74.90704271295682) bank85236 +85237 POINT(40.318043528224386 -74.30136879923722) bank85237 +85238 POINT(41.02449263996515 -73.7607707252129) bank85238 +85239 POINT(40.35573056400204 -74.99460053421848) bank85239 +85240 POINT(41.04955449546328 -73.87869002600702) bank85240 +85241 POINT(40.338231836089285 -74.87322314992711) bank85241 +85242 POINT(41.6725684064731 -74.20885647505014) bank85242 +85243 POINT(41.70364021248377 -73.19013218756834) bank85243 +85244 POINT(41.45906749305369 -73.29449427383149) bank85244 +85245 POINT(41.553399560740125 -74.20730888471205) bank85245 +85246 POINT(41.00334938709036 -73.50742372577976) bank85246 +85247 POINT(41.56657799528179 -73.43806219244217) bank85247 +85248 POINT(41.58074534433383 -73.87643119139204) bank85248 +85249 POINT(41.41374314235969 -73.64677782674798) bank85249 +85250 POINT(40.14753369737282 -73.94907509157655) bank85250 +85251 POINT(39.86854891836779 -73.09814130737244) bank85251 +85252 POINT(41.61087416666318 -73.18172661680322) bank85252 +85253 POINT(40.1753621745803 -73.79312330692403) bank85253 +85254 POINT(41.020612706756495 -75.00095183163195) bank85254 +85255 POINT(40.0053294124873 -73.72288655789077) bank85255 +85256 POINT(41.507293678524185 -74.63745596521429) bank85256 +85257 POINT(41.69571249450337 -74.9854448100228) bank85257 +85258 POINT(39.830417215198885 -74.78952624477397) bank85258 +85259 POINT(40.75766822525704 -74.66992340801461) bank85259 +85260 POINT(41.16731910640702 -73.91641804364936) bank85260 +85261 POINT(40.6495500605772 -74.3618674548288) bank85261 +85262 POINT(40.930373918869805 -74.14999566110525) bank85262 +85263 POINT(39.90782270032756 -74.19835958198858) bank85263 +85264 POINT(40.52962241170278 -74.62512076396618) bank85264 +85265 POINT(41.09695871606028 -73.39247997064413) bank85265 +85266 POINT(40.84680011060606 -73.72378534240161) bank85266 +85267 POINT(40.39465526293911 -73.3326880121518) bank85267 +85268 POINT(41.028612023772126 -74.08369151571199) bank85268 +85269 POINT(41.10082160899879 -74.27320005858614) bank85269 +85270 POINT(41.159590420646815 -73.35265557720962) bank85270 +85271 POINT(40.377496927470474 -74.5631817571288) bank85271 +85272 POINT(40.48119676949122 -74.53462469599576) bank85272 +85273 POINT(40.09673482206876 -74.9637576468181) bank85273 +85274 POINT(40.78178637222883 -74.7881809102754) bank85274 +85275 POINT(40.93168293103007 -74.88019862756896) bank85275 +85276 POINT(40.0953318785656 -74.39132731794756) bank85276 +85277 POINT(40.63642380089418 -74.84312591050886) bank85277 +85278 POINT(39.94628644787519 -74.9363960112866) bank85278 +85279 POINT(39.81910657350631 -74.9133593596779) bank85279 +85280 POINT(40.83629966062554 -73.33420192852871) bank85280 +85281 POINT(39.80676136800315 -73.44611369975465) bank85281 +85282 POINT(41.068422249348366 -74.52675449965565) bank85282 +85283 POINT(41.42331131853467 -73.34167466901499) bank85283 +85284 POINT(41.68143608949607 -73.91651448316141) bank85284 +85285 POINT(39.944887497897824 -74.99950557678159) bank85285 +85286 POINT(40.05835421217468 -74.23666448736056) bank85286 +85287 POINT(40.911980985463956 -74.22535688285315) bank85287 +85288 POINT(39.909036153543965 -74.51631748734874) bank85288 +85289 POINT(41.22049367425974 -73.03026914521983) bank85289 +85290 POINT(41.53768463319813 -73.53938828938684) bank85290 +85291 POINT(40.230832147030256 -73.51130420115841) bank85291 +85292 POINT(41.13622135131122 -74.56292404996174) bank85292 +85293 POINT(40.27566512815577 -73.99444130259322) bank85293 +85294 POINT(40.58869269025007 -74.70426099757198) bank85294 +85295 POINT(41.001314699206674 -73.50028377251915) bank85295 +85296 POINT(39.81978906884151 -73.79117798906853) bank85296 +85297 POINT(41.13610132413693 -74.42775114412395) bank85297 +85298 POINT(40.356979899810845 -74.9442742985397) bank85298 +85299 POINT(41.01321224265547 -73.82427658929922) bank85299 +85300 POINT(40.04400647723581 -73.84200810473165) bank85300 +85301 POINT(40.06229672075916 -73.65284072906114) bank85301 +85302 POINT(41.69126031492966 -74.49576788437366) bank85302 +85303 POINT(40.614297920290674 -74.78942876125826) bank85303 +85304 POINT(40.141768569052765 -73.6188967007301) bank85304 +85305 POINT(40.951759326813765 -73.54177199058046) bank85305 +85306 POINT(40.02280296730445 -73.51689262748225) bank85306 +85307 POINT(40.43222128417166 -74.55425090927447) bank85307 +85308 POINT(40.62834565538267 -74.76297818953293) bank85308 +85309 POINT(41.24903940972034 -74.54847882790867) bank85309 +85310 POINT(41.04719981165987 -74.49251544094471) bank85310 +85311 POINT(41.558807221308044 -73.38293501192366) bank85311 +85312 POINT(41.3653541721823 -74.04614504028034) bank85312 +85313 POINT(40.71461298139961 -74.59650723446897) bank85313 +85314 POINT(40.67818930237338 -74.46556317529662) bank85314 +85315 POINT(41.52644578500637 -74.62323852626234) bank85315 +85316 POINT(40.71470312321777 -74.85466246889413) bank85316 +85317 POINT(41.429252926252836 -74.99166482971367) bank85317 +85318 POINT(41.66540131656839 -74.81067019826219) bank85318 +85319 POINT(40.430537000612276 -74.73836195502261) bank85319 +85320 POINT(41.038189478714486 -74.83869193759372) bank85320 +85321 POINT(40.841556166692584 -74.11228513038819) bank85321 +85322 POINT(41.305639581422206 -74.1362749918372) bank85322 +85323 POINT(39.905572399503406 -73.06247365181633) bank85323 +85324 POINT(41.561856494431396 -74.46529918663894) bank85324 +85325 POINT(41.49595450173142 -74.36381225521615) bank85325 +85326 POINT(41.5633047502605 -73.25419904092534) bank85326 +85327 POINT(40.698511267221996 -74.98292622500354) bank85327 +85328 POINT(40.203398397516246 -74.81994590051474) bank85328 +85329 POINT(40.68767545781365 -73.29539266949408) bank85329 +85330 POINT(40.67909964378693 -74.9066320079603) bank85330 +85331 POINT(41.03098072131414 -73.6367799913019) bank85331 +85332 POINT(40.97645984268338 -73.43411200540945) bank85332 +85333 POINT(41.37161713998353 -74.5602553917812) bank85333 +85334 POINT(39.953405221301345 -73.67999178631621) bank85334 +85335 POINT(41.61611486127358 -73.88298502531215) bank85335 +85336 POINT(40.48737699553708 -74.36657971278815) bank85336 +85337 POINT(40.33771739045319 -73.55729902178658) bank85337 +85338 POINT(39.7579525652303 -73.01147992394878) bank85338 +85339 POINT(41.50457931362082 -74.70447286323353) bank85339 +85340 POINT(40.152546640469915 -73.757959241494) bank85340 +85341 POINT(40.569411008118095 -74.1949386447892) bank85341 +85342 POINT(39.92362772001363 -74.95937136904553) bank85342 +85343 POINT(39.74619763593447 -73.63589486196206) bank85343 +85344 POINT(40.669147019526385 -73.64153018638422) bank85344 +85345 POINT(41.61144926987545 -74.92170971064094) bank85345 +85346 POINT(40.520235115956794 -73.06735955128337) bank85346 +85347 POINT(40.011768899024204 -74.92647919261852) bank85347 +85348 POINT(40.404001919319555 -74.77286197130726) bank85348 +85349 POINT(40.73118021902446 -73.02374766666684) bank85349 +85350 POINT(41.22543562748671 -73.79273535127622) bank85350 +85351 POINT(40.620166370990404 -73.7986552314639) bank85351 +85352 POINT(39.929506627997874 -74.53729957473445) bank85352 +85353 POINT(40.06464566608137 -73.68544966942407) bank85353 +85354 POINT(41.59854507685543 -73.06251224205167) bank85354 +85355 POINT(40.508380808019496 -74.61200780583545) bank85355 +85356 POINT(41.43615143017565 -74.794500019707) bank85356 +85357 POINT(39.87240708008773 -73.21893407674526) bank85357 +85358 POINT(40.51490000397089 -73.4135016524362) bank85358 +85359 POINT(40.71187385579087 -73.88885467749442) bank85359 +85360 POINT(41.324750190928484 -74.58603798717765) bank85360 +85361 POINT(41.060222366759376 -73.50317491060144) bank85361 +85362 POINT(41.42493563239834 -74.21214985751381) bank85362 +85363 POINT(40.286504303396704 -73.69080468830718) bank85363 +85364 POINT(40.195067495680846 -74.02765628421517) bank85364 +85365 POINT(39.80575041698888 -73.05925242440117) bank85365 +85366 POINT(41.68221505524239 -74.41163905898995) bank85366 +85367 POINT(40.15066807181229 -73.82998092477965) bank85367 +85368 POINT(40.60513661096 -74.08130669156951) bank85368 +85369 POINT(41.12432269018375 -73.26863068564651) bank85369 +85370 POINT(40.25940991024159 -73.17146651620743) bank85370 +85371 POINT(39.763842483708665 -73.50849759304961) bank85371 +85372 POINT(41.60296545206932 -74.14750803409277) bank85372 +85373 POINT(40.70437667603059 -73.35000948663766) bank85373 +85374 POINT(41.20697040411602 -74.8916346892616) bank85374 +85375 POINT(41.61582935552532 -74.33053239331757) bank85375 +85376 POINT(40.40770449395788 -74.95582736158443) bank85376 +85377 POINT(40.5128505724602 -74.00603378550406) bank85377 +85378 POINT(39.95056310361774 -74.82609567907734) bank85378 +85379 POINT(39.739042647281515 -74.43037248077792) bank85379 +85380 POINT(40.03802986496368 -73.91917495909131) bank85380 +85381 POINT(40.2670060431887 -73.56320023641781) bank85381 +85382 POINT(40.5664429954894 -74.54573419151906) bank85382 +85383 POINT(40.70067243963505 -73.45982977602097) bank85383 +85384 POINT(41.12378611218691 -74.78559993702588) bank85384 +85385 POINT(40.70271735965018 -74.89767494715532) bank85385 +85386 POINT(39.993855368664775 -74.51720834546857) bank85386 +85387 POINT(40.14587566326302 -74.2357450644558) bank85387 +85388 POINT(41.516387123914775 -73.45049246241544) bank85388 +85389 POINT(41.329763067953735 -73.3707947017181) bank85389 +85390 POINT(40.920795932887586 -74.9582104577974) bank85390 +85391 POINT(41.23842027449009 -74.07625230177706) bank85391 +85392 POINT(41.38670522218929 -73.79581357701818) bank85392 +85393 POINT(40.766180964647475 -74.84669597959034) bank85393 +85394 POINT(40.41180727550506 -73.12132312782006) bank85394 +85395 POINT(40.5461136243423 -74.15966153955392) bank85395 +85396 POINT(41.60935465749205 -74.72548244887302) bank85396 +85397 POINT(41.653138482047225 -74.14034643761744) bank85397 +85398 POINT(41.35256388978589 -74.61576555001271) bank85398 +85399 POINT(40.78271900740998 -73.69025284333043) bank85399 +85400 POINT(41.33411889495848 -74.52182643669487) bank85400 +85401 POINT(41.15276335208152 -74.94747198915928) bank85401 +85402 POINT(39.888773039390806 -74.24622049415134) bank85402 +85403 POINT(41.61204450111225 -73.39888431200652) bank85403 +85404 POINT(40.56266755657287 -73.56460229022721) bank85404 +85405 POINT(40.18868585105656 -73.58987078168211) bank85405 +85406 POINT(40.29079099194284 -73.6827471866995) bank85406 +85407 POINT(41.424383006009215 -74.19050584263148) bank85407 +85408 POINT(39.954008445962984 -74.25906210577135) bank85408 +85409 POINT(41.16091425312677 -74.40994124041883) bank85409 +85410 POINT(40.10442789528763 -73.93911915834657) bank85410 +85411 POINT(41.45269539274509 -73.8420214371524) bank85411 +85412 POINT(40.56145868742675 -74.91265570491838) bank85412 +85413 POINT(40.72481651795852 -74.8602762102203) bank85413 +85414 POINT(40.055150475724375 -74.11463085758733) bank85414 +85415 POINT(40.336477922702485 -74.1010805041015) bank85415 +85416 POINT(40.54916831971343 -74.08307031249812) bank85416 +85417 POINT(41.012075620572936 -73.56522523206715) bank85417 +85418 POINT(39.79212796680415 -73.41069614974882) bank85418 +85419 POINT(40.18407052110254 -73.11448222555808) bank85419 +85420 POINT(39.96081421572869 -73.94237301404357) bank85420 +85421 POINT(40.681850014996904 -73.62338710753761) bank85421 +85422 POINT(40.39441176484485 -74.6576802732676) bank85422 +85423 POINT(40.28016595286269 -73.07507396984182) bank85423 +85424 POINT(40.50533342235104 -73.1677364954386) bank85424 +85425 POINT(40.301657495339306 -73.75976566129528) bank85425 +85426 POINT(41.36011525599049 -74.3948708153543) bank85426 +85427 POINT(40.28063288792535 -74.36293053604672) bank85427 +85428 POINT(40.070693731147806 -73.45767197142123) bank85428 +85429 POINT(40.110633515725205 -73.55881539147565) bank85429 +85430 POINT(40.99189241368542 -74.45092154707841) bank85430 +85431 POINT(40.864571923699096 -74.50952476808429) bank85431 +85432 POINT(40.925226680649274 -73.81541856235803) bank85432 +85433 POINT(40.133084604399116 -73.88478345733228) bank85433 +85434 POINT(39.983206084984786 -73.30014258014066) bank85434 +85435 POINT(41.207481480339084 -73.03306300391624) bank85435 +85436 POINT(41.15511390776441 -73.67606740290896) bank85436 +85437 POINT(39.715623965786214 -74.70485150312605) bank85437 +85438 POINT(41.600590662688134 -73.25525624464433) bank85438 +85439 POINT(40.886752629090566 -74.58626145914019) bank85439 +85440 POINT(39.975184124864164 -74.48390161068579) bank85440 +85441 POINT(41.39132907295352 -74.96570990988766) bank85441 +85442 POINT(41.041406210627926 -73.80612243705997) bank85442 +85443 POINT(39.85010412578296 -73.97890849467635) bank85443 +85444 POINT(40.747385009982665 -74.45279042684392) bank85444 +85445 POINT(40.97239046164296 -73.0548804270751) bank85445 +85446 POINT(40.37566579766199 -73.11347904274508) bank85446 +85447 POINT(39.864709670360746 -73.76489407290383) bank85447 +85448 POINT(40.052115666812746 -73.47467174721541) bank85448 +85449 POINT(41.44438997736408 -73.88002669882486) bank85449 +85450 POINT(41.07419645912181 -74.28431142205301) bank85450 +85451 POINT(39.807058534702016 -73.00689807447172) bank85451 +85452 POINT(39.89654556279558 -73.66574109956176) bank85452 +85453 POINT(41.36882173950086 -73.96284675296071) bank85453 +85454 POINT(41.17589498102676 -74.19699125098784) bank85454 +85455 POINT(41.137588428645536 -73.83683199822859) bank85455 +85456 POINT(40.83293827420594 -74.37876922622277) bank85456 +85457 POINT(41.37787124462964 -73.23116657881492) bank85457 +85458 POINT(41.44759065587249 -74.67549597547013) bank85458 +85459 POINT(41.356752773347864 -74.75346067895823) bank85459 +85460 POINT(41.63278601842897 -73.44057957220717) bank85460 +85461 POINT(41.59507274888717 -74.88366807226059) bank85461 +85462 POINT(41.1551071358757 -74.1009476021711) bank85462 +85463 POINT(41.27415828341237 -74.7493918270525) bank85463 +85464 POINT(41.199091306322764 -74.83424883183523) bank85464 +85465 POINT(40.010076629826195 -73.35238859664578) bank85465 +85466 POINT(40.353311284048736 -73.60564628899746) bank85466 +85467 POINT(40.036337710318946 -73.20547834873807) bank85467 +85468 POINT(39.80426847522001 -74.18919377004939) bank85468 +85469 POINT(39.79060686788421 -74.45675037388366) bank85469 +85470 POINT(41.01754016530296 -74.70790590790172) bank85470 +85471 POINT(39.729180590726884 -74.52054313807324) bank85471 +85472 POINT(40.53828832647778 -73.03997422965317) bank85472 +85473 POINT(40.12815922337209 -73.95133885988918) bank85473 +85474 POINT(40.71896050690731 -74.77800425366348) bank85474 +85475 POINT(40.55399141232195 -74.43663888531776) bank85475 +85476 POINT(40.395826148536194 -73.39796393374749) bank85476 +85477 POINT(41.22493036151671 -74.35515395376538) bank85477 +85478 POINT(41.7016765164041 -74.9329760207932) bank85478 +85479 POINT(41.307834148943115 -73.2273699883053) bank85479 +85480 POINT(40.616939109991186 -73.67888677975373) bank85480 +85481 POINT(40.64044581416555 -74.93924176133058) bank85481 +85482 POINT(41.696378006308706 -74.70365388409084) bank85482 +85483 POINT(39.77986915327607 -73.7179351086798) bank85483 +85484 POINT(41.008275645680705 -73.55234598890775) bank85484 +85485 POINT(40.28765589230465 -74.40411912584854) bank85485 +85486 POINT(41.05010808612958 -73.98108166962093) bank85486 +85487 POINT(39.86540572666702 -74.03069939564953) bank85487 +85488 POINT(40.595999950094495 -73.52517912284432) bank85488 +85489 POINT(40.404404068226945 -73.79498594838742) bank85489 +85490 POINT(41.521899440065894 -73.57237234005765) bank85490 +85491 POINT(40.874287851398414 -73.4748394942607) bank85491 +85492 POINT(41.50184093719569 -74.10798193057573) bank85492 +85493 POINT(41.51660005440165 -74.73676006606757) bank85493 +85494 POINT(39.7824390834069 -73.30158428382418) bank85494 +85495 POINT(41.20174392284656 -73.52746602953981) bank85495 +85496 POINT(40.17566253939946 -74.54277129175127) bank85496 +85497 POINT(40.5297464350852 -73.50520704430804) bank85497 +85498 POINT(41.60748623374182 -74.13398520548013) bank85498 +85499 POINT(39.90047097887918 -74.76769042842734) bank85499 +85500 POINT(40.46166933354114 -73.9859465727899) bank85500 +85501 POINT(40.658146452273336 -73.16895274955134) bank85501 +85502 POINT(41.187993687223724 -73.72559008277831) bank85502 +85503 POINT(41.02925822582346 -74.89118534929926) bank85503 +85504 POINT(41.54674888387416 -73.77304872052125) bank85504 +85505 POINT(40.082641869538854 -73.79348615939024) bank85505 +85506 POINT(40.637942481847105 -73.50521791015113) bank85506 +85507 POINT(41.69722274141393 -73.15979988820793) bank85507 +85508 POINT(40.44352817235706 -73.0344442891126) bank85508 +85509 POINT(41.00689018612615 -73.28972183402134) bank85509 +85510 POINT(41.284924474565706 -74.87951348240114) bank85510 +85511 POINT(40.88612510771084 -74.57017418007733) bank85511 +85512 POINT(40.73752805464701 -74.01967482801935) bank85512 +85513 POINT(39.720755804832706 -74.83354866497265) bank85513 +85514 POINT(41.58151516444759 -74.27293818122544) bank85514 +85515 POINT(40.56269376015184 -73.95496401164314) bank85515 +85516 POINT(41.29922136350998 -74.0169236882173) bank85516 +85517 POINT(40.18835241516286 -74.43654936521203) bank85517 +85518 POINT(40.77766317859545 -74.93604171027245) bank85518 +85519 POINT(41.05135144030886 -73.33396884761667) bank85519 +85520 POINT(40.996444726367294 -73.0619651809458) bank85520 +85521 POINT(40.35883799790786 -74.07646883592042) bank85521 +85522 POINT(39.97573226765275 -73.67938846955276) bank85522 +85523 POINT(41.29198347501204 -74.57861042663511) bank85523 +85524 POINT(39.91967967000949 -74.98859251527476) bank85524 +85525 POINT(40.94307918738522 -73.2701147344244) bank85525 +85526 POINT(39.958277479920454 -73.73364489059958) bank85526 +85527 POINT(41.54029256356292 -74.49275204313898) bank85527 +85528 POINT(41.14518153104886 -73.65212091579625) bank85528 +85529 POINT(39.739876983600126 -73.20070681864597) bank85529 +85530 POINT(40.40717923742587 -73.6210194243133) bank85530 +85531 POINT(41.57362349309928 -73.15184629932169) bank85531 +85532 POINT(41.64879268464169 -73.66143070324723) bank85532 +85533 POINT(40.26295770596763 -74.4230914401748) bank85533 +85534 POINT(39.74944857138009 -74.19293497765774) bank85534 +85535 POINT(40.218070162712664 -73.13925852698269) bank85535 +85536 POINT(40.452632888272305 -74.45980067680301) bank85536 +85537 POINT(40.81206614510427 -74.53284240022779) bank85537 +85538 POINT(41.45542958980172 -74.2298398984696) bank85538 +85539 POINT(40.84001956632121 -73.95274064060155) bank85539 +85540 POINT(39.85858400940253 -74.54700285452239) bank85540 +85541 POINT(39.81156932662721 -74.5357475085553) bank85541 +85542 POINT(40.307734841334685 -73.87197766160743) bank85542 +85543 POINT(40.71622183560463 -74.70393479892553) bank85543 +85544 POINT(40.99766070246088 -74.72847951324107) bank85544 +85545 POINT(39.97364274277614 -73.08118597656721) bank85545 +85546 POINT(41.06997141593037 -74.76851601495149) bank85546 +85547 POINT(41.37425822834879 -73.52461584905451) bank85547 +85548 POINT(40.80121615403429 -73.86702683489595) bank85548 +85549 POINT(40.3176753571079 -74.37042003161807) bank85549 +85550 POINT(41.067017636456846 -73.51474844164272) bank85550 +85551 POINT(40.681151621331296 -73.6748617158963) bank85551 +85552 POINT(41.108238625975865 -73.12408612526725) bank85552 +85553 POINT(40.89921312541042 -73.5986341123346) bank85553 +85554 POINT(41.697987423211956 -73.7590601538911) bank85554 +85555 POINT(40.983798312215534 -73.33723077825582) bank85555 +85556 POINT(40.16160834800417 -74.01197022200112) bank85556 +85557 POINT(40.40107688193976 -74.7298305108034) bank85557 +85558 POINT(40.48096145072059 -73.2580238086808) bank85558 +85559 POINT(40.501249701397356 -73.88319623068702) bank85559 +85560 POINT(41.636377391162526 -73.94328612096582) bank85560 +85561 POINT(40.61686821833476 -73.19825906683087) bank85561 +85562 POINT(40.73259970695016 -74.17084412745155) bank85562 +85563 POINT(41.15679295753722 -74.71067219263139) bank85563 +85564 POINT(40.93605256372377 -74.70931798699739) bank85564 +85565 POINT(41.63937107921105 -73.0913311847011) bank85565 +85566 POINT(40.5742554247676 -73.19427121018089) bank85566 +85567 POINT(40.91224891056574 -73.34186093131389) bank85567 +85568 POINT(40.96339396620813 -73.55034534433285) bank85568 +85569 POINT(41.31822685636849 -74.73398036137036) bank85569 +85570 POINT(41.48367332188917 -74.09270255719682) bank85570 +85571 POINT(41.2310220946679 -74.2681522473166) bank85571 +85572 POINT(41.536365563599965 -74.26045686648239) bank85572 +85573 POINT(41.03616408971142 -74.80005550759543) bank85573 +85574 POINT(41.562531973529566 -74.13100455127812) bank85574 +85575 POINT(40.45661776710303 -74.82458019188451) bank85575 +85576 POINT(40.447792370162176 -73.30423192175398) bank85576 +85577 POINT(40.22055620615309 -74.1388795257164) bank85577 +85578 POINT(40.976487003781294 -74.59931542193344) bank85578 +85579 POINT(40.21414869789531 -73.18515842750952) bank85579 +85580 POINT(40.28271425477122 -73.11787752816664) bank85580 +85581 POINT(40.11580026804582 -74.49437425617312) bank85581 +85582 POINT(40.796152231173195 -74.73411229504102) bank85582 +85583 POINT(40.04133841045022 -74.73393889947354) bank85583 +85584 POINT(41.45638649577212 -74.60621503186633) bank85584 +85585 POINT(40.094236743834145 -74.27819396400325) bank85585 +85586 POINT(41.710326390286916 -74.3524517429972) bank85586 +85587 POINT(41.66999551026333 -73.04263021104943) bank85587 +85588 POINT(40.31770591101994 -74.61037025988655) bank85588 +85589 POINT(40.63447129540408 -73.57013539308072) bank85589 +85590 POINT(41.337596175164904 -74.99210558977722) bank85590 +85591 POINT(41.31587102915315 -74.85226629739473) bank85591 +85592 POINT(40.20168008727262 -73.97393531748725) bank85592 +85593 POINT(40.313343060115656 -73.47146324252174) bank85593 +85594 POINT(41.20721373642483 -73.05046534135145) bank85594 +85595 POINT(41.09835388813152 -73.61815316764338) bank85595 +85596 POINT(40.08116615866652 -73.80521645847806) bank85596 +85597 POINT(40.50733318744562 -73.14511457029226) bank85597 +85598 POINT(41.323448920816844 -75.0055706744691) bank85598 +85599 POINT(41.553294415073566 -74.84630147131985) bank85599 +85600 POINT(39.93909563647827 -74.85833872813082) bank85600 +85601 POINT(41.411178854913565 -74.6378193565967) bank85601 +85602 POINT(39.831562133090166 -73.36935004732659) bank85602 +85603 POINT(39.814276414159025 -74.9548107526367) bank85603 +85604 POINT(41.17293487543385 -73.54037583886996) bank85604 +85605 POINT(40.36483876207561 -73.03825398995058) bank85605 +85606 POINT(39.8449822622902 -74.63180673028417) bank85606 +85607 POINT(41.35196257302262 -73.40688159234628) bank85607 +85608 POINT(39.912337276976444 -74.68846073631158) bank85608 +85609 POINT(40.10119536367476 -74.21274895431326) bank85609 +85610 POINT(40.156293941035585 -73.37950050213584) bank85610 +85611 POINT(41.457011724073084 -74.97671790823343) bank85611 +85612 POINT(41.39933647143107 -73.23821923501842) bank85612 +85613 POINT(40.07664251596064 -73.39757837735932) bank85613 +85614 POINT(41.3765442113596 -73.43114349767498) bank85614 +85615 POINT(39.85691251975236 -74.7209621645766) bank85615 +85616 POINT(41.149316913709924 -74.25750735288666) bank85616 +85617 POINT(41.49304054923982 -74.50971038552775) bank85617 +85618 POINT(40.68557725141474 -74.16335487394812) bank85618 +85619 POINT(40.4714656469206 -73.67460749688213) bank85619 +85620 POINT(40.120883337161885 -73.08647244078013) bank85620 +85621 POINT(40.67993190676218 -73.30104874968919) bank85621 +85622 POINT(41.26375059273766 -74.49278167251583) bank85622 +85623 POINT(41.42008265977108 -74.25490864531936) bank85623 +85624 POINT(40.001972829070915 -74.63406898855544) bank85624 +85625 POINT(39.984968716913684 -74.43058144769755) bank85625 +85626 POINT(41.568247986163456 -74.86277806783048) bank85626 +85627 POINT(41.517918491384435 -74.29345548754395) bank85627 +85628 POINT(40.82313799196681 -74.70354917232703) bank85628 +85629 POINT(39.867120353586124 -73.93812383774672) bank85629 +85630 POINT(40.73743538923608 -73.61124752254591) bank85630 +85631 POINT(41.24025917334028 -74.30213887262498) bank85631 +85632 POINT(40.114477145698494 -74.89196856263992) bank85632 +85633 POINT(41.1491040202697 -74.42610439812152) bank85633 +85634 POINT(41.584168152325624 -74.92738904802661) bank85634 +85635 POINT(41.24316621000857 -73.3667903103631) bank85635 +85636 POINT(40.09138445673054 -74.74890906089125) bank85636 +85637 POINT(41.164063757303616 -74.77337885802322) bank85637 +85638 POINT(40.205953416714294 -74.05171183187426) bank85638 +85639 POINT(41.66980808535426 -73.4871300051261) bank85639 +85640 POINT(40.412104722137606 -74.6191793411924) bank85640 +85641 POINT(40.42322780864978 -74.36428848532016) bank85641 +85642 POINT(41.40746991300378 -74.89502275844794) bank85642 +85643 POINT(41.59093298972505 -74.45392631622477) bank85643 +85644 POINT(40.57786831679473 -74.85671289442456) bank85644 +85645 POINT(40.79922489734634 -73.05622200104266) bank85645 +85646 POINT(40.053828907361705 -74.41778426249809) bank85646 +85647 POINT(41.3199443413404 -73.20032035894378) bank85647 +85648 POINT(41.56318123218897 -73.2923678714047) bank85648 +85649 POINT(40.20014514820527 -73.9104312850186) bank85649 +85650 POINT(39.75357318998001 -73.66710898259043) bank85650 +85651 POINT(40.39047948551014 -73.14792046252984) bank85651 +85652 POINT(40.85989204666139 -73.92882338412343) bank85652 +85653 POINT(41.09102114170968 -73.3337439619699) bank85653 +85654 POINT(40.12781841921655 -73.18541104313043) bank85654 +85655 POINT(40.43487792610638 -73.774643913719) bank85655 +85656 POINT(40.729296498255884 -74.24680180060379) bank85656 +85657 POINT(39.88208168687792 -73.48517666614289) bank85657 +85658 POINT(40.370623141247414 -74.04858962129468) bank85658 +85659 POINT(39.71425776878896 -74.57689463982732) bank85659 +85660 POINT(41.32813579671692 -73.11403289585975) bank85660 +85661 POINT(40.19455231716935 -73.98733906020563) bank85661 +85662 POINT(40.63203269691299 -74.98992728921456) bank85662 +85663 POINT(41.07150424188203 -74.98937095354718) bank85663 +85664 POINT(40.24124749262902 -74.87460707329534) bank85664 +85665 POINT(41.25883170920573 -74.97567253858628) bank85665 +85666 POINT(41.505746418665744 -73.60514025317424) bank85666 +85667 POINT(40.24273635215345 -73.55978730738698) bank85667 +85668 POINT(39.854595520535355 -74.92473996067152) bank85668 +85669 POINT(40.478528625380996 -73.2111570086182) bank85669 +85670 POINT(41.38280047533518 -73.01333427418572) bank85670 +85671 POINT(41.564631201164396 -73.94458605005528) bank85671 +85672 POINT(40.89886285344022 -74.92654354992278) bank85672 +85673 POINT(41.28994561754745 -74.51672191604162) bank85673 +85674 POINT(40.46984400436768 -73.99153647786723) bank85674 +85675 POINT(40.45534593615672 -74.92949212471422) bank85675 +85676 POINT(40.44892947721562 -74.65451082380183) bank85676 +85677 POINT(39.995898717859276 -73.92951696222175) bank85677 +85678 POINT(40.86214756620593 -74.01889184292304) bank85678 +85679 POINT(40.93460640817289 -74.48040463830928) bank85679 +85680 POINT(40.55502486229417 -73.42807341180537) bank85680 +85681 POINT(39.86230497104515 -74.26173719838866) bank85681 +85682 POINT(41.651758340532076 -73.46847084726998) bank85682 +85683 POINT(40.03277913526435 -73.77016740305623) bank85683 +85684 POINT(40.32754534146689 -73.86536827466932) bank85684 +85685 POINT(40.1103162403817 -73.96093392537736) bank85685 +85686 POINT(40.15748848581894 -73.50108282468226) bank85686 +85687 POINT(41.47569939096752 -73.01864552145665) bank85687 +85688 POINT(40.14755613842383 -73.80281037889985) bank85688 +85689 POINT(40.033714803439224 -74.7039462949119) bank85689 +85690 POINT(41.35937130138336 -74.20349957199718) bank85690 +85691 POINT(40.84757783241317 -74.30551115763132) bank85691 +85692 POINT(40.57179328976351 -73.05424811114229) bank85692 +85693 POINT(40.923804570960215 -73.64369175349245) bank85693 +85694 POINT(41.379544097550955 -73.30488726067331) bank85694 +85695 POINT(40.830121452553286 -74.866148340291) bank85695 +85696 POINT(39.83744528009224 -74.05622592882754) bank85696 +85697 POINT(40.99758882815125 -74.74184581305995) bank85697 +85698 POINT(39.91632955956796 -73.86626000199334) bank85698 +85699 POINT(41.654708214313345 -74.04521606661426) bank85699 +85700 POINT(39.969006278460384 -74.09211482014368) bank85700 +85701 POINT(40.73775233537351 -73.21306805879134) bank85701 +85702 POINT(39.99916503288772 -73.34625417059381) bank85702 +85703 POINT(39.85322557078024 -73.57195947640318) bank85703 +85704 POINT(40.59898706150392 -73.42831815287808) bank85704 +85705 POINT(39.9925241505388 -73.31387155025303) bank85705 +85706 POINT(40.55143844282072 -74.78875949273046) bank85706 +85707 POINT(41.13527232896848 -73.41258812720595) bank85707 +85708 POINT(41.087924731085465 -74.82879517964513) bank85708 +85709 POINT(41.00754895944605 -73.36281858395968) bank85709 +85710 POINT(41.61693032572536 -73.35296329512543) bank85710 +85711 POINT(39.98015158332796 -74.81152615282613) bank85711 +85712 POINT(39.97419108891932 -74.71695898099131) bank85712 +85713 POINT(41.169056696018224 -74.94721261734173) bank85713 +85714 POINT(41.19465612211448 -73.10527719616567) bank85714 +85715 POINT(41.6442286762417 -73.46520823498331) bank85715 +85716 POINT(40.246094215517175 -73.81521451251734) bank85716 +85717 POINT(41.579105606900065 -73.54858521944551) bank85717 +85718 POINT(40.89930161542746 -74.56067101535147) bank85718 +85719 POINT(40.67693290283273 -73.86603834820475) bank85719 +85720 POINT(41.13530068819884 -73.4084076322358) bank85720 +85721 POINT(40.52393226140609 -73.4938104982714) bank85721 +85722 POINT(40.45326588036504 -74.03102259634306) bank85722 +85723 POINT(39.86732874743784 -73.5354439556623) bank85723 +85724 POINT(40.26608357201645 -74.45248545405322) bank85724 +85725 POINT(41.69570788240087 -74.18822432321424) bank85725 +85726 POINT(39.83834955201121 -74.85541517554356) bank85726 +85727 POINT(40.880182151819874 -74.27111556050302) bank85727 +85728 POINT(40.38910622130731 -73.06273077918502) bank85728 +85729 POINT(41.631169975977436 -73.68271045356896) bank85729 +85730 POINT(40.4338809484676 -73.74591990626438) bank85730 +85731 POINT(39.93985803197222 -74.549485245413) bank85731 +85732 POINT(40.47277569583744 -73.15277681916672) bank85732 +85733 POINT(40.448951585770246 -73.55443495565146) bank85733 +85734 POINT(40.62516880125695 -73.85884874660277) bank85734 +85735 POINT(41.64810928895761 -73.63943996760133) bank85735 +85736 POINT(41.23834329025747 -73.08039890107557) bank85736 +85737 POINT(40.87217404222713 -74.71474567361533) bank85737 +85738 POINT(40.289807566842036 -73.68950537163437) bank85738 +85739 POINT(40.163110170032006 -74.8838022663712) bank85739 +85740 POINT(41.183527805109506 -74.13483913466702) bank85740 +85741 POINT(39.855171715165966 -74.08910187386398) bank85741 +85742 POINT(41.66498280453809 -74.44641242071111) bank85742 +85743 POINT(40.1712854725623 -73.79136258522558) bank85743 +85744 POINT(40.1815239608035 -74.71827934970958) bank85744 +85745 POINT(40.37912324562197 -74.94375174413) bank85745 +85746 POINT(40.18022111528872 -73.09052842793058) bank85746 +85747 POINT(39.96731751135299 -73.5049006864327) bank85747 +85748 POINT(40.70581033493301 -73.10521442276465) bank85748 +85749 POINT(40.766542230073746 -74.75178172128562) bank85749 +85750 POINT(39.838566243071945 -74.15494927620297) bank85750 +85751 POINT(41.0178478960224 -74.10480715926869) bank85751 +85752 POINT(40.58204133800079 -73.57394193664037) bank85752 +85753 POINT(41.343722427019 -74.55613298966729) bank85753 +85754 POINT(40.99248717053626 -74.07655358711523) bank85754 +85755 POINT(39.92916589847981 -73.2693928169443) bank85755 +85756 POINT(41.37236407617743 -73.35791411459996) bank85756 +85757 POINT(40.500878475635176 -74.58699471566264) bank85757 +85758 POINT(40.42036007300336 -74.41409755973237) bank85758 +85759 POINT(40.013634896362966 -73.41725316307945) bank85759 +85760 POINT(39.925766999808815 -74.89686212820004) bank85760 +85761 POINT(40.478707969895936 -73.19458949159424) bank85761 +85762 POINT(40.193908669817766 -74.31721182224106) bank85762 +85763 POINT(40.15538979515614 -74.40413608037386) bank85763 +85764 POINT(40.4969905525537 -74.0972381863906) bank85764 +85765 POINT(41.077767694238744 -74.35352159288078) bank85765 +85766 POINT(41.65388009986926 -74.0744676817628) bank85766 +85767 POINT(40.27683634005322 -73.22007129788774) bank85767 +85768 POINT(39.99773939098113 -73.04994095571071) bank85768 +85769 POINT(41.49455910670883 -73.87124286242364) bank85769 +85770 POINT(40.5616726016282 -74.13602475656934) bank85770 +85771 POINT(41.6207691355451 -74.71690697441349) bank85771 +85772 POINT(40.11885500230343 -74.74746772818442) bank85772 +85773 POINT(39.966097720518434 -73.32023501251153) bank85773 +85774 POINT(40.34220935275425 -74.65018265793337) bank85774 +85775 POINT(41.08020492829981 -74.49443801989928) bank85775 +85776 POINT(39.811351537226365 -73.64885759722448) bank85776 +85777 POINT(40.48758164228349 -73.60522514991678) bank85777 +85778 POINT(41.484642929743906 -73.06063514875265) bank85778 +85779 POINT(40.316525034996125 -73.1018324132636) bank85779 +85780 POINT(41.53048166428114 -73.76759707969214) bank85780 +85781 POINT(40.07504765580712 -74.92182546499363) bank85781 +85782 POINT(41.3022762455019 -73.94734581470344) bank85782 +85783 POINT(40.69521780717962 -74.93824112950097) bank85783 +85784 POINT(40.551643664129145 -73.82586679680546) bank85784 +85785 POINT(39.88740638899184 -73.47212305749666) bank85785 +85786 POINT(41.4161995519683 -73.99665456935121) bank85786 +85787 POINT(41.05196191340385 -73.82214722713863) bank85787 +85788 POINT(40.19619855063399 -74.18925908708165) bank85788 +85789 POINT(41.314585380839674 -74.33752422307177) bank85789 +85790 POINT(39.86737054230747 -73.94861297697884) bank85790 +85791 POINT(39.829685436913834 -73.32098272398363) bank85791 +85792 POINT(41.28723435650796 -74.5132986942977) bank85792 +85793 POINT(40.7768715143842 -74.68879357115713) bank85793 +85794 POINT(41.50339426323186 -73.72443071267708) bank85794 +85795 POINT(39.77989452258593 -73.63635205857457) bank85795 +85796 POINT(40.76133850699207 -73.84177687739793) bank85796 +85797 POINT(39.87367317274694 -74.93280042701186) bank85797 +85798 POINT(41.03869016599622 -73.20610012026452) bank85798 +85799 POINT(40.43064781366145 -73.42049569718766) bank85799 +85800 POINT(41.020174105244735 -73.32304041917149) bank85800 +85801 POINT(39.96218661390245 -74.91768254171569) bank85801 +85802 POINT(41.202869563982674 -74.27186871787026) bank85802 +85803 POINT(41.54186690058031 -73.12269010844689) bank85803 +85804 POINT(41.59971685337063 -74.96233170347273) bank85804 +85805 POINT(40.814328569032014 -73.37512081913412) bank85805 +85806 POINT(40.10775741784318 -73.95909313712417) bank85806 +85807 POINT(40.550281192727084 -73.13113702797453) bank85807 +85808 POINT(40.56249180749978 -73.03671278040086) bank85808 +85809 POINT(41.29553973036176 -74.09010254747076) bank85809 +85810 POINT(41.387467443053715 -73.44204225169864) bank85810 +85811 POINT(40.1912133682894 -73.63473695652397) bank85811 +85812 POINT(39.87624370284303 -73.16473091319398) bank85812 +85813 POINT(41.65369023930327 -73.28676545335544) bank85813 +85814 POINT(41.45583723188313 -74.0800595927301) bank85814 +85815 POINT(40.90908927858585 -74.62530145112727) bank85815 +85816 POINT(40.60330713579289 -73.06406921386953) bank85816 +85817 POINT(40.20294986758721 -74.5391625706641) bank85817 +85818 POINT(40.15503649487719 -73.9218071510674) bank85818 +85819 POINT(40.56547679378777 -73.94201918314374) bank85819 +85820 POINT(39.8430401877376 -75.0007569720019) bank85820 +85821 POINT(40.03988815802041 -73.20115816227077) bank85821 +85822 POINT(40.401203513821955 -73.69118356995764) bank85822 +85823 POINT(41.662936417894414 -73.7006099192396) bank85823 +85824 POINT(41.32113022414676 -73.35501134215198) bank85824 +85825 POINT(41.47628302493986 -73.43906871018697) bank85825 +85826 POINT(41.396030595362774 -74.17605368590888) bank85826 +85827 POINT(41.43221049233357 -73.75976824381017) bank85827 +85828 POINT(40.77766372506318 -73.36984825890703) bank85828 +85829 POINT(40.473083775192286 -74.15852198608233) bank85829 +85830 POINT(40.78729989021798 -74.77567384532804) bank85830 +85831 POINT(41.08594688560492 -74.20384918224573) bank85831 +85832 POINT(40.903183769898206 -73.57064092078556) bank85832 +85833 POINT(40.53739410203043 -73.03163599740368) bank85833 +85834 POINT(41.701385122364705 -74.70746753278605) bank85834 +85835 POINT(41.29974504637795 -74.0427270491527) bank85835 +85836 POINT(40.6796073405455 -73.56016790078117) bank85836 +85837 POINT(40.45952892287876 -74.54199187132441) bank85837 +85838 POINT(40.5290444893271 -74.40969309812883) bank85838 +85839 POINT(41.06729926519094 -74.51300167585433) bank85839 +85840 POINT(39.94661669259727 -73.10330358936635) bank85840 +85841 POINT(39.92519721217177 -74.8226680639515) bank85841 +85842 POINT(40.35604622480573 -73.96027733078664) bank85842 +85843 POINT(41.086577306369854 -73.64675605446627) bank85843 +85844 POINT(40.920234095571274 -73.06790520690578) bank85844 +85845 POINT(40.78336742139769 -74.14786673087772) bank85845 +85846 POINT(41.016389940595076 -74.7418915530272) bank85846 +85847 POINT(41.220648924605165 -74.28550642440383) bank85847 +85848 POINT(41.216390884745486 -74.11745855021957) bank85848 +85849 POINT(40.00874513734578 -74.49203329293658) bank85849 +85850 POINT(41.70004413273118 -74.0551761939556) bank85850 +85851 POINT(40.17071829550349 -73.45593447564644) bank85851 +85852 POINT(40.934018747745235 -74.67482301556629) bank85852 +85853 POINT(40.591970703900834 -73.7653778770371) bank85853 +85854 POINT(41.56189505273159 -74.55579833070824) bank85854 +85855 POINT(40.19489833366283 -74.78269298335672) bank85855 +85856 POINT(41.132616205373644 -74.45290885268085) bank85856 +85857 POINT(40.036492778328544 -73.37936210638286) bank85857 +85858 POINT(40.26840001340167 -74.4113368370925) bank85858 +85859 POINT(40.88794143472445 -73.79841937335517) bank85859 +85860 POINT(40.64159444669898 -74.50988348347006) bank85860 +85861 POINT(39.95093691590707 -73.77710833940407) bank85861 +85862 POINT(40.35317387843797 -73.22211196252603) bank85862 +85863 POINT(41.122534480651474 -74.6627093664101) bank85863 +85864 POINT(40.461696343984784 -73.35751427069778) bank85864 +85865 POINT(41.6468169365524 -73.92408255426271) bank85865 +85866 POINT(40.1444760429843 -74.16503620944465) bank85866 +85867 POINT(40.18532465346795 -73.98261963771103) bank85867 +85868 POINT(40.81097123941795 -73.23060852664447) bank85868 +85869 POINT(39.9382246554521 -73.78899507641414) bank85869 +85870 POINT(41.35333461602378 -73.37509259992969) bank85870 +85871 POINT(40.835832111886006 -74.71643507478477) bank85871 +85872 POINT(40.53063191452458 -74.57555059514755) bank85872 +85873 POINT(39.93867230187607 -74.1464375897814) bank85873 +85874 POINT(41.06165735580789 -74.62446464354211) bank85874 +85875 POINT(40.85360834128013 -73.3536133998862) bank85875 +85876 POINT(39.756043986099755 -74.97922254732313) bank85876 +85877 POINT(41.607528461238985 -73.40049490046951) bank85877 +85878 POINT(41.351640567856094 -74.79746378452633) bank85878 +85879 POINT(40.43705314898055 -74.7222813055876) bank85879 +85880 POINT(39.852671124790575 -74.06468090654474) bank85880 +85881 POINT(40.598878539878214 -73.8398657906066) bank85881 +85882 POINT(41.218860630447686 -73.16005527363092) bank85882 +85883 POINT(41.52422476088921 -74.77262062927426) bank85883 +85884 POINT(39.777176016669124 -74.59150189541943) bank85884 +85885 POINT(40.738358043206254 -73.71853395150775) bank85885 +85886 POINT(39.82684358572179 -73.12007507928729) bank85886 +85887 POINT(40.95183121912016 -74.67068824273987) bank85887 +85888 POINT(41.474127966453715 -73.23682556331715) bank85888 +85889 POINT(41.57268021487119 -74.34008082045152) bank85889 +85890 POINT(40.53447550293022 -74.36628253467255) bank85890 +85891 POINT(40.33972673940676 -73.59166986776087) bank85891 +85892 POINT(40.621460822105774 -73.49687416274938) bank85892 +85893 POINT(39.71658008882545 -74.56049078668963) bank85893 +85894 POINT(41.64665934691748 -75.00432695326558) bank85894 +85895 POINT(39.761920792907276 -74.33187442135753) bank85895 +85896 POINT(40.05853620392677 -73.36648858780558) bank85896 +85897 POINT(41.40847874508704 -73.16791392327357) bank85897 +85898 POINT(40.71054576608653 -73.1849490722167) bank85898 +85899 POINT(40.79069370601273 -74.17879285230944) bank85899 +85900 POINT(40.820919644091745 -74.82194532944443) bank85900 +85901 POINT(40.597199667362716 -74.69318363081402) bank85901 +85902 POINT(40.24356560379126 -73.68656161002399) bank85902 +85903 POINT(40.22823362452955 -74.32459287526387) bank85903 +85904 POINT(41.7089244221661 -73.20108046617109) bank85904 +85905 POINT(40.42664118539479 -73.93600084706962) bank85905 +85906 POINT(40.146148124323695 -73.51741972334763) bank85906 +85907 POINT(40.8764419698302 -74.06912716119443) bank85907 +85908 POINT(41.08923274631081 -73.48859122250374) bank85908 +85909 POINT(41.54796005853956 -74.02323398949976) bank85909 +85910 POINT(40.210662915544965 -73.92501157108012) bank85910 +85911 POINT(40.31124587241716 -73.11910283242335) bank85911 +85912 POINT(40.373330487647586 -73.46506516602926) bank85912 +85913 POINT(39.98976993268643 -73.98754754956688) bank85913 +85914 POINT(41.07774568869726 -73.84125071379827) bank85914 +85915 POINT(40.33929261054275 -73.01768115435887) bank85915 +85916 POINT(41.34926120514236 -73.96281471797242) bank85916 +85917 POINT(39.95068178579318 -73.70316824892302) bank85917 +85918 POINT(41.61339430896334 -73.23922581593897) bank85918 +85919 POINT(41.01671201737347 -74.56471689326779) bank85919 +85920 POINT(40.66139766410028 -74.02135114239758) bank85920 +85921 POINT(41.38946181257249 -74.90639365382596) bank85921 +85922 POINT(39.76457696085765 -73.60612365096657) bank85922 +85923 POINT(40.740087150114455 -74.68730796211352) bank85923 +85924 POINT(40.18206483244438 -73.72813484043499) bank85924 +85925 POINT(40.22244561081535 -74.24601014318549) bank85925 +85926 POINT(41.40232216621269 -74.62453385669066) bank85926 +85927 POINT(40.2676356015996 -74.3459778397539) bank85927 +85928 POINT(40.227520056614765 -73.98338148610985) bank85928 +85929 POINT(40.23276570050908 -73.66832529117096) bank85929 +85930 POINT(41.15145446811012 -74.48481095027559) bank85930 +85931 POINT(40.03374936027591 -74.61162380999312) bank85931 +85932 POINT(40.092123471443294 -73.40448738534913) bank85932 +85933 POINT(41.48534334818563 -74.62114056098072) bank85933 +85934 POINT(41.643598331566025 -73.62973485725001) bank85934 +85935 POINT(40.52326595727823 -73.65539879897013) bank85935 +85936 POINT(40.479653640231774 -73.86840025835052) bank85936 +85937 POINT(41.03725044345508 -74.09293899685143) bank85937 +85938 POINT(40.27509195523512 -73.8485260597266) bank85938 +85939 POINT(41.536806122763245 -73.29700318225794) bank85939 +85940 POINT(40.64340534467089 -74.89411723204486) bank85940 +85941 POINT(41.43455600604898 -73.44211177494822) bank85941 +85942 POINT(40.48318835814388 -73.7812311400563) bank85942 +85943 POINT(41.17828570388011 -73.49309230727374) bank85943 +85944 POINT(41.37028939181696 -73.5055884447636) bank85944 +85945 POINT(39.89173126675497 -73.02902234238196) bank85945 +85946 POINT(40.394290741173144 -74.15528129986383) bank85946 +85947 POINT(41.1775088449547 -73.41250366397664) bank85947 +85948 POINT(39.99319090300746 -73.87692533352701) bank85948 +85949 POINT(41.260324196133745 -74.6605746517675) bank85949 +85950 POINT(41.46468534465388 -74.26557256151816) bank85950 +85951 POINT(41.439475316786606 -74.17438809197392) bank85951 +85952 POINT(39.93974494596071 -73.67545758242909) bank85952 +85953 POINT(39.89966583087883 -73.52119115627949) bank85953 +85954 POINT(39.89767031576992 -73.42712638422475) bank85954 +85955 POINT(41.03449635344753 -74.98619236171632) bank85955 +85956 POINT(40.33508811271548 -74.06673974605184) bank85956 +85957 POINT(41.37971317664485 -74.99515176850907) bank85957 +85958 POINT(41.07268203865562 -73.89173505571385) bank85958 +85959 POINT(40.10336270870551 -74.2602903401615) bank85959 +85960 POINT(41.572967158685394 -73.17586220268298) bank85960 +85961 POINT(41.41272703017965 -74.1028088494567) bank85961 +85962 POINT(40.8935902886669 -73.47548463899165) bank85962 +85963 POINT(40.28226641560977 -73.22460777678438) bank85963 +85964 POINT(40.071173042727466 -74.56326075746888) bank85964 +85965 POINT(40.340766579959116 -73.5963025644095) bank85965 +85966 POINT(40.42534047679565 -73.938064388247) bank85966 +85967 POINT(40.939367753945724 -73.29767905631724) bank85967 +85968 POINT(40.44779773529616 -73.79253683298143) bank85968 +85969 POINT(40.01196497233561 -73.98973475532577) bank85969 +85970 POINT(40.970302177433695 -74.6715839138818) bank85970 +85971 POINT(39.998107547209415 -73.27638248518953) bank85971 +85972 POINT(41.03525613978407 -73.68958558694283) bank85972 +85973 POINT(41.05453593628553 -73.19942447278807) bank85973 +85974 POINT(40.958176134657236 -73.43352891783762) bank85974 +85975 POINT(41.617050843973914 -74.87560585968406) bank85975 +85976 POINT(40.662080651148116 -73.87140527242953) bank85976 +85977 POINT(40.200779574639014 -74.84111790787826) bank85977 +85978 POINT(41.29396399381933 -74.76493211343968) bank85978 +85979 POINT(40.85978981284079 -74.67014694884747) bank85979 +85980 POINT(41.381613088195984 -74.03523873953898) bank85980 +85981 POINT(40.93444735502778 -74.62974430516162) bank85981 +85982 POINT(40.070268898892046 -73.7027698220813) bank85982 +85983 POINT(41.7060265615918 -74.70242262472863) bank85983 +85984 POINT(40.71989182425598 -74.07593623227409) bank85984 +85985 POINT(40.06194590294095 -73.92920554697506) bank85985 +85986 POINT(40.79912567245048 -74.42676493799276) bank85986 +85987 POINT(41.18654060788099 -73.3097102779458) bank85987 +85988 POINT(40.28010743323365 -74.05748740935046) bank85988 +85989 POINT(40.026852428493264 -73.49966681444108) bank85989 +85990 POINT(41.57250878241709 -73.80606639285547) bank85990 +85991 POINT(41.4396910870934 -74.04402744320679) bank85991 +85992 POINT(40.63408271819596 -74.32726669393747) bank85992 +85993 POINT(40.62348204789129 -74.59412918290971) bank85993 +85994 POINT(40.25129279025908 -73.94771287505769) bank85994 +85995 POINT(41.125042673021326 -73.5711426357005) bank85995 +85996 POINT(41.17809938286733 -74.42899416047491) bank85996 +85997 POINT(41.41272600011567 -73.26180048002293) bank85997 +85998 POINT(40.8439617345444 -74.4537174191013) bank85998 +85999 POINT(41.663286478988816 -74.02741706181308) bank85999 +86000 POINT(39.71812829352253 -74.80760425895097) bank86000 +86001 POINT(40.70272536708832 -73.4907943031964) bank86001 +86002 POINT(40.810032429230745 -73.03267817151423) bank86002 +86003 POINT(40.151573372573964 -74.76588857258731) bank86003 +86004 POINT(41.53428673677438 -74.64944995124948) bank86004 +86005 POINT(41.53929347199507 -74.99023454292814) bank86005 +86006 POINT(40.77446118898856 -74.44966867413827) bank86006 +86007 POINT(40.6507447299844 -73.69252621828363) bank86007 +86008 POINT(40.892538301481395 -74.48774303223388) bank86008 +86009 POINT(41.046042462372526 -73.00653648743696) bank86009 +86010 POINT(40.865108414822686 -74.06489974126573) bank86010 +86011 POINT(39.800530058542364 -73.8352845801484) bank86011 +86012 POINT(40.86090168848468 -73.06483012234382) bank86012 +86013 POINT(40.324123373636 -73.86765653518769) bank86013 +86014 POINT(40.9069709086684 -74.26766271910607) bank86014 +86015 POINT(39.96673386250222 -75.0007934106635) bank86015 +86016 POINT(40.07030436156325 -74.19796360262676) bank86016 +86017 POINT(41.23020565466304 -73.80331234060962) bank86017 +86018 POINT(40.87625198031382 -74.92280670575356) bank86018 +86019 POINT(40.473891859087566 -73.70966516507566) bank86019 +86020 POINT(40.1732270951623 -74.09450658986904) bank86020 +86021 POINT(40.482586457950944 -74.08939771948597) bank86021 +86022 POINT(40.32809207755531 -74.46684483732487) bank86022 +86023 POINT(41.215532230253324 -74.68821499872806) bank86023 +86024 POINT(41.54229293081064 -74.75453070900942) bank86024 +86025 POINT(41.51753675057826 -73.76550665547433) bank86025 +86026 POINT(41.45280518647522 -74.72272559797788) bank86026 +86027 POINT(40.63158194565168 -73.15283754456158) bank86027 +86028 POINT(41.151497774714095 -74.0987306406532) bank86028 +86029 POINT(40.85347410876766 -73.54982052216572) bank86029 +86030 POINT(41.17292657573448 -74.98263677033397) bank86030 +86031 POINT(40.84892107527973 -74.97692657518267) bank86031 +86032 POINT(41.66786606302044 -74.3400227524645) bank86032 +86033 POINT(40.053327966902195 -73.25127839058204) bank86033 +86034 POINT(40.237432266887375 -74.81842325018451) bank86034 +86035 POINT(41.08687998732699 -74.4283613278616) bank86035 +86036 POINT(40.522858491968286 -74.39782646173587) bank86036 +86037 POINT(40.629984610175455 -73.80890591835578) bank86037 +86038 POINT(41.614016309998846 -74.78009364174142) bank86038 +86039 POINT(40.03859714489084 -73.79876751352909) bank86039 +86040 POINT(40.70887563774473 -73.22610155883639) bank86040 +86041 POINT(40.065933543334594 -74.49597356599581) bank86041 +86042 POINT(39.97415525629103 -74.74406288667907) bank86042 +86043 POINT(41.324425226567925 -73.19603554539984) bank86043 +86044 POINT(40.15079007959616 -74.12482780099003) bank86044 +86045 POINT(41.54566228091429 -73.16571587342645) bank86045 +86046 POINT(41.33274090265858 -73.73850677951847) bank86046 +86047 POINT(40.22230043389369 -74.36696673624579) bank86047 +86048 POINT(39.888476219612066 -73.96799280043837) bank86048 +86049 POINT(40.3909778131941 -73.34680034784655) bank86049 +86050 POINT(39.85053347877956 -74.07486836847987) bank86050 +86051 POINT(40.38839699484645 -73.09348198702621) bank86051 +86052 POINT(39.86496541878891 -73.34474434014112) bank86052 +86053 POINT(41.00229959851929 -73.63146750267562) bank86053 +86054 POINT(39.87394875933885 -73.43780138035322) bank86054 +86055 POINT(40.26641434094975 -74.8058944005981) bank86055 +86056 POINT(40.653857697069206 -73.04508239241416) bank86056 +86057 POINT(41.38575382312288 -73.79959848869372) bank86057 +86058 POINT(40.06542339866644 -73.16396525528687) bank86058 +86059 POINT(41.37585630771508 -74.44894979211807) bank86059 +86060 POINT(40.11949212009893 -74.43226384307695) bank86060 +86061 POINT(40.736174090756116 -74.307831153117) bank86061 +86062 POINT(41.54194937643274 -74.61123668603193) bank86062 +86063 POINT(40.18178292513542 -74.73902066111506) bank86063 +86064 POINT(39.840102119726346 -74.27533244445738) bank86064 +86065 POINT(41.31654165242277 -73.671486934766) bank86065 +86066 POINT(41.53696227233131 -74.6899876141639) bank86066 +86067 POINT(39.846378475122215 -73.3940604881475) bank86067 +86068 POINT(41.606592998877595 -73.5111287582712) bank86068 +86069 POINT(41.544916148742686 -73.53720411452662) bank86069 +86070 POINT(40.73452199059518 -73.49328597839292) bank86070 +86071 POINT(41.07390043551244 -73.84940053908878) bank86071 +86072 POINT(40.5894514633575 -73.73737465520225) bank86072 +86073 POINT(40.53038052912831 -73.07262472505285) bank86073 +86074 POINT(40.65260694530986 -73.56970464791263) bank86074 +86075 POINT(40.201325301291014 -74.47813393139238) bank86075 +86076 POINT(41.06836413017689 -74.49706511356929) bank86076 +86077 POINT(41.37905132624987 -73.73587781401268) bank86077 +86078 POINT(40.54145824265784 -73.48345370283893) bank86078 +86079 POINT(40.109332274169326 -74.87929210307362) bank86079 +86080 POINT(40.254461181848896 -73.16533051869921) bank86080 +86081 POINT(41.221394568948774 -73.29675985926147) bank86081 +86082 POINT(41.514339036866176 -73.23687535950174) bank86082 +86083 POINT(40.930108028025934 -73.42732717002222) bank86083 +86084 POINT(40.6134046327417 -74.36776543231181) bank86084 +86085 POINT(41.32413852174776 -74.14432687371186) bank86085 +86086 POINT(40.91348426729378 -74.2190538412398) bank86086 +86087 POINT(40.46118404248309 -73.25222402734923) bank86087 +86088 POINT(40.38971694562054 -73.69260811109099) bank86088 +86089 POINT(41.1901644104983 -73.48610934207305) bank86089 +86090 POINT(41.5473413403249 -74.1887630812321) bank86090 +86091 POINT(39.77993630682918 -74.20313410412146) bank86091 +86092 POINT(40.10551953871146 -73.32218246768097) bank86092 +86093 POINT(41.53183428465029 -74.04494366938964) bank86093 +86094 POINT(40.56047326110099 -74.43368990782851) bank86094 +86095 POINT(39.83904217435893 -74.56289954364617) bank86095 +86096 POINT(40.164078177898105 -73.84412048149558) bank86096 +86097 POINT(41.31347539669502 -74.76774371254893) bank86097 +86098 POINT(41.462759680983005 -73.90229952896942) bank86098 +86099 POINT(41.566976974827185 -75.00474103182597) bank86099 +86100 POINT(40.22390509187409 -74.03510798312114) bank86100 +86101 POINT(40.592197456041994 -74.34171646464321) bank86101 +86102 POINT(40.20890936709026 -73.33279936871092) bank86102 +86103 POINT(41.51108613570504 -73.4624077198732) bank86103 +86104 POINT(39.99066956640347 -74.42476593065517) bank86104 +86105 POINT(39.90869745531604 -73.17063631494915) bank86105 +86106 POINT(41.58836681023913 -73.15043241583489) bank86106 +86107 POINT(40.96616131671214 -73.9862381240856) bank86107 +86108 POINT(40.22936664507343 -73.41715201231715) bank86108 +86109 POINT(41.488321767345084 -74.79331575848906) bank86109 +86110 POINT(40.64128976812783 -74.30718189415843) bank86110 +86111 POINT(41.30485867803694 -73.94652921313175) bank86111 +86112 POINT(40.02400130697261 -73.80639246914319) bank86112 +86113 POINT(40.44147770580173 -75.00143113735106) bank86113 +86114 POINT(39.760220351410105 -74.34315894260186) bank86114 +86115 POINT(41.42044203458724 -74.7277739206776) bank86115 +86116 POINT(41.29836776785524 -73.62551624226289) bank86116 +86117 POINT(40.43344211327876 -73.5413616154647) bank86117 +86118 POINT(41.22417599769418 -74.48899139528687) bank86118 +86119 POINT(40.0172966790574 -74.41573509872161) bank86119 +86120 POINT(41.55749835906588 -74.05287118820523) bank86120 +86121 POINT(40.12856633605515 -73.60159342359894) bank86121 +86122 POINT(40.24689138363145 -73.48521421818965) bank86122 +86123 POINT(41.612330282243 -73.46039485209975) bank86123 +86124 POINT(41.524056346020934 -74.6481158084812) bank86124 +86125 POINT(40.32916668287945 -73.62230230869147) bank86125 +86126 POINT(40.162350888516116 -73.8370473256898) bank86126 +86127 POINT(40.308279186062585 -73.92394241420475) bank86127 +86128 POINT(40.06211239379571 -74.61834475161703) bank86128 +86129 POINT(40.47720767491348 -73.22660840801692) bank86129 +86130 POINT(41.53984574447188 -74.10527763366268) bank86130 +86131 POINT(40.84666063696738 -73.81912971788715) bank86131 +86132 POINT(40.89895028441789 -74.91309299988492) bank86132 +86133 POINT(41.52870490676204 -73.91610826277386) bank86133 +86134 POINT(39.95838190058966 -74.15009605530987) bank86134 +86135 POINT(39.96507115183954 -74.76333788422917) bank86135 +86136 POINT(41.13217377370933 -74.78341704952608) bank86136 +86137 POINT(40.103933126508515 -74.58526295172119) bank86137 +86138 POINT(41.24727455590354 -74.3021856553664) bank86138 +86139 POINT(39.993872495542114 -74.62705330155293) bank86139 +86140 POINT(40.996083488124086 -73.02097687671595) bank86140 +86141 POINT(40.75745119991589 -73.08828305991543) bank86141 +86142 POINT(41.574039810175435 -73.14523538314532) bank86142 +86143 POINT(39.78872964900299 -73.04279062879762) bank86143 +86144 POINT(41.683366964207174 -74.58047650303159) bank86144 +86145 POINT(40.13958289845504 -74.18751580640507) bank86145 +86146 POINT(41.46477772313171 -73.186013899187) bank86146 +86147 POINT(41.24618877441823 -73.41499630680003) bank86147 +86148 POINT(39.91564396537272 -73.73889001794606) bank86148 +86149 POINT(41.28934867286015 -73.82153156000042) bank86149 +86150 POINT(41.51175497856002 -74.9024810911605) bank86150 +86151 POINT(41.178040523488065 -73.25143779683977) bank86151 +86152 POINT(40.06122773397289 -73.21278969053759) bank86152 +86153 POINT(40.54205152423919 -74.98018612521389) bank86153 +86154 POINT(39.823246870183574 -73.0095342137536) bank86154 +86155 POINT(41.43378761903212 -73.34894204870236) bank86155 +86156 POINT(41.14046920006174 -73.5714474178873) bank86156 +86157 POINT(40.34636355636621 -74.5537577391544) bank86157 +86158 POINT(40.703373834800225 -73.06111636034706) bank86158 +86159 POINT(41.132255939851795 -74.57193548055842) bank86159 +86160 POINT(41.21055212187903 -73.3545529271775) bank86160 +86161 POINT(40.235586533370096 -74.0045986658513) bank86161 +86162 POINT(41.18526621225939 -74.85566047637747) bank86162 +86163 POINT(40.21901594087392 -74.08173247094149) bank86163 +86164 POINT(41.200361732989954 -74.84853611111014) bank86164 +86165 POINT(40.12521078052036 -73.38525541707516) bank86165 +86166 POINT(39.71557540743271 -74.58554607142169) bank86166 +86167 POINT(39.74523313816378 -73.6932381375064) bank86167 +86168 POINT(40.750314093010964 -74.15001353357196) bank86168 +86169 POINT(40.44211443536244 -74.62982737566225) bank86169 +86170 POINT(39.87409434183225 -74.12089202793265) bank86170 +86171 POINT(40.877724652342174 -74.92927190912523) bank86171 +86172 POINT(41.55061153901315 -73.87561294735232) bank86172 +86173 POINT(40.01556897604497 -73.53766481303235) bank86173 +86174 POINT(40.70156673618403 -73.68882348864385) bank86174 +86175 POINT(40.26542654751683 -73.10584887203673) bank86175 +86176 POINT(40.114226406191214 -73.25896148303984) bank86176 +86177 POINT(40.6416239506994 -74.06517362009572) bank86177 +86178 POINT(41.29052831354332 -74.6613040138577) bank86178 +86179 POINT(40.07794742140453 -73.73157613515707) bank86179 +86180 POINT(39.917978978903 -73.45446198327377) bank86180 +86181 POINT(40.66082422219742 -73.4647612723203) bank86181 +86182 POINT(40.33991203095708 -74.90335265179041) bank86182 +86183 POINT(40.32506562107387 -73.4119314076429) bank86183 +86184 POINT(39.81744563727707 -74.24379659684674) bank86184 +86185 POINT(40.85521005065072 -73.55590392194276) bank86185 +86186 POINT(40.69654069456208 -74.48855289890388) bank86186 +86187 POINT(40.10295956693788 -73.62730245075346) bank86187 +86188 POINT(41.58513210560533 -73.7430570347119) bank86188 +86189 POINT(40.44067022308113 -73.55277491020449) bank86189 +86190 POINT(40.83803607202694 -73.81370450742594) bank86190 +86191 POINT(41.14781942458547 -73.43097771470322) bank86191 +86192 POINT(39.75569221566346 -74.01551330577148) bank86192 +86193 POINT(40.32737918826325 -73.77215972447145) bank86193 +86194 POINT(40.051888966188336 -74.92442780675628) bank86194 +86195 POINT(40.28429622650828 -73.65822013503677) bank86195 +86196 POINT(39.99626540446626 -74.72278058238976) bank86196 +86197 POINT(41.65851813571151 -73.72633554439696) bank86197 +86198 POINT(41.47975941218457 -73.23080197072758) bank86198 +86199 POINT(40.657717596681884 -74.434645298353) bank86199 +86200 POINT(40.136076871773945 -74.88517432579835) bank86200 +86201 POINT(41.474168531254286 -74.24921464605482) bank86201 +86202 POINT(41.62007645391645 -74.95502046324115) bank86202 +86203 POINT(41.498715736537854 -73.08737761492456) bank86203 +86204 POINT(41.13593544868821 -73.86829101689659) bank86204 +86205 POINT(40.02668415285488 -74.67064164311655) bank86205 +86206 POINT(40.56610484155029 -73.05314234169151) bank86206 +86207 POINT(40.49653778335374 -74.91833613022837) bank86207 +86208 POINT(41.22237383888591 -73.7302566924558) bank86208 +86209 POINT(40.720565026133485 -74.65084664825487) bank86209 +86210 POINT(40.78859621020132 -73.32871779207018) bank86210 +86211 POINT(40.115894156404714 -74.357144499562) bank86211 +86212 POINT(40.44832429407043 -73.40121288955308) bank86212 +86213 POINT(40.03820342654778 -74.0866454631302) bank86213 +86214 POINT(40.212410789704855 -73.7833097307103) bank86214 +86215 POINT(40.07852885394133 -74.55500837772378) bank86215 +86216 POINT(40.51415143112243 -73.37564698005764) bank86216 +86217 POINT(40.652306225202985 -73.8947488253452) bank86217 +86218 POINT(40.761818418285884 -73.15870645614213) bank86218 +86219 POINT(41.2057958381695 -74.28842198688538) bank86219 +86220 POINT(41.2320227514929 -73.71303839460693) bank86220 +86221 POINT(41.234773539440404 -73.72178883205396) bank86221 +86222 POINT(40.70770625726203 -73.51490621442434) bank86222 +86223 POINT(39.852198046854674 -73.8518427865781) bank86223 +86224 POINT(41.50296835519697 -74.39309819272664) bank86224 +86225 POINT(40.37576904115228 -74.11013884490036) bank86225 +86226 POINT(40.04714970390957 -74.57445237662773) bank86226 +86227 POINT(41.487628707891744 -74.87253236289519) bank86227 +86228 POINT(40.629560331817366 -73.33576738838975) bank86228 +86229 POINT(39.90586043615985 -73.87159212876502) bank86229 +86230 POINT(40.37301503270837 -74.06898239752691) bank86230 +86231 POINT(41.69527524554876 -74.5047988363783) bank86231 +86232 POINT(40.218451200240004 -74.04657726667725) bank86232 +86233 POINT(41.03315108818148 -73.03453438742307) bank86233 +86234 POINT(40.94388508793241 -74.47989111910928) bank86234 +86235 POINT(40.26826790580141 -74.00552628588369) bank86235 +86236 POINT(40.52550576139721 -74.12077873853175) bank86236 +86237 POINT(41.243177183746546 -73.15524969177994) bank86237 +86238 POINT(39.899753057821094 -74.65104266431187) bank86238 +86239 POINT(41.46731600332579 -74.44880587728295) bank86239 +86240 POINT(40.688310454564316 -74.494010019857) bank86240 +86241 POINT(40.27592675190768 -74.82444541526286) bank86241 +86242 POINT(39.787270278317095 -74.73441837935763) bank86242 +86243 POINT(41.39817082191122 -73.31125146797973) bank86243 +86244 POINT(40.53478590164507 -73.23694354370345) bank86244 +86245 POINT(41.22708554095565 -73.56928294486094) bank86245 +86246 POINT(39.84297262473414 -74.87389202380585) bank86246 +86247 POINT(40.937367926697554 -73.69926470519995) bank86247 +86248 POINT(41.37264406908273 -74.8284096203235) bank86248 +86249 POINT(40.20942102537029 -73.8869947326864) bank86249 +86250 POINT(40.99130103951833 -73.55599462625186) bank86250 +86251 POINT(40.061827709384275 -73.43069822038262) bank86251 +86252 POINT(39.982692754841096 -74.80759480232312) bank86252 +86253 POINT(41.35668665695 -74.55149812489948) bank86253 +86254 POINT(41.310324173501954 -73.26301399990919) bank86254 +86255 POINT(40.41498333710336 -74.9380213212402) bank86255 +86256 POINT(41.50638081668929 -74.40352048444419) bank86256 +86257 POINT(40.11433182341032 -73.04943275938845) bank86257 +86258 POINT(40.904731457374346 -73.66596257053675) bank86258 +86259 POINT(40.52124828798896 -74.97908853973672) bank86259 +86260 POINT(40.174572860176916 -74.18672737218671) bank86260 +86261 POINT(40.21503821640084 -74.7812538142693) bank86261 +86262 POINT(40.61566563452823 -74.99624139061656) bank86262 +86263 POINT(40.38020444724658 -74.00396179540242) bank86263 +86264 POINT(41.298560051749504 -73.04892205568495) bank86264 +86265 POINT(40.48473774536342 -73.48506122572716) bank86265 +86266 POINT(41.5957226393567 -73.61210020423765) bank86266 +86267 POINT(41.36151702626345 -73.26833098775874) bank86267 +86268 POINT(40.308309458041656 -74.31063679743232) bank86268 +86269 POINT(41.50658231933528 -73.94914128705821) bank86269 +86270 POINT(40.92181998214362 -74.77976864440349) bank86270 +86271 POINT(40.01242994949546 -74.72473604972299) bank86271 +86272 POINT(41.49474649491853 -74.71128335618883) bank86272 +86273 POINT(41.03641283967733 -73.64589796878286) bank86273 +86274 POINT(40.275951475216104 -74.72148473208325) bank86274 +86275 POINT(40.779499007226086 -74.64337025944006) bank86275 +86276 POINT(39.92861803719636 -74.6149193223955) bank86276 +86277 POINT(41.244965281242244 -73.7133867541343) bank86277 +86278 POINT(40.05414120706015 -74.48795960141604) bank86278 +86279 POINT(40.105782002745656 -74.335346576557) bank86279 +86280 POINT(40.28338825031469 -74.21341983255948) bank86280 +86281 POINT(40.41833965296352 -73.8781031605559) bank86281 +86282 POINT(41.43861862760364 -74.1709232535336) bank86282 +86283 POINT(40.34672396805837 -73.64831300474373) bank86283 +86284 POINT(40.83437572009062 -73.19289544372744) bank86284 +86285 POINT(40.77225512227382 -74.06764985372277) bank86285 +86286 POINT(40.12759035351864 -73.7387004970218) bank86286 +86287 POINT(39.814315624851666 -73.7139540717811) bank86287 +86288 POINT(39.82461703743576 -74.88300264335471) bank86288 +86289 POINT(39.74889328355592 -74.80903821760245) bank86289 +86290 POINT(40.96593318709793 -73.82992378162503) bank86290 +86291 POINT(41.160813374977025 -73.80615111363811) bank86291 +86292 POINT(41.160490168167875 -74.33757801828337) bank86292 +86293 POINT(40.63629804338616 -74.820319426407) bank86293 +86294 POINT(40.48949390266556 -73.30165446681158) bank86294 +86295 POINT(41.69564749226315 -74.80216602870445) bank86295 +86296 POINT(41.56319214179159 -74.35273967583467) bank86296 +86297 POINT(40.19469149490777 -74.9070095987683) bank86297 +86298 POINT(41.40507054361809 -74.18594074505528) bank86298 +86299 POINT(40.85653715979732 -74.33818375209867) bank86299 +86300 POINT(40.67039665801155 -74.7669692945769) bank86300 +86301 POINT(39.758613006967316 -74.03824055794553) bank86301 +86302 POINT(40.657699980574144 -74.13652610885173) bank86302 +86303 POINT(39.82291828209126 -73.44039355189366) bank86303 +86304 POINT(41.257031217467464 -74.01562535403032) bank86304 +86305 POINT(41.090518858896075 -74.16408481084079) bank86305 +86306 POINT(40.178763859399545 -74.87977596454067) bank86306 +86307 POINT(41.419813773813225 -74.07801512178057) bank86307 +86308 POINT(40.92037298835497 -73.50427775550439) bank86308 +86309 POINT(40.669970025319486 -73.44267409660621) bank86309 +86310 POINT(41.168793318783536 -74.97850842477715) bank86310 +86311 POINT(40.82125281507957 -74.6456507137451) bank86311 +86312 POINT(40.89031694219106 -73.30152440286277) bank86312 +86313 POINT(41.298885642595415 -74.06087857164776) bank86313 +86314 POINT(41.63918940676758 -74.94527638563663) bank86314 +86315 POINT(40.54710913787669 -73.92473395163243) bank86315 +86316 POINT(41.45051756004192 -74.36627751491193) bank86316 +86317 POINT(41.07024795048292 -74.45996176635893) bank86317 +86318 POINT(39.97265669771986 -73.92538669338684) bank86318 +86319 POINT(41.508105349005774 -73.25357961498423) bank86319 +86320 POINT(40.771598031824084 -73.56482738442364) bank86320 +86321 POINT(41.53164718107593 -74.66614272109055) bank86321 +86322 POINT(41.53454401554341 -73.41006642997952) bank86322 +86323 POINT(40.33259294572638 -74.51296730098967) bank86323 +86324 POINT(39.78881278658896 -73.17087756815508) bank86324 +86325 POINT(40.34989788339508 -74.87932113436199) bank86325 +86326 POINT(40.904384656629645 -74.23637058974401) bank86326 +86327 POINT(39.87105898255404 -74.3746357930787) bank86327 +86328 POINT(40.88892862056368 -73.2984165627428) bank86328 +86329 POINT(39.95000288401448 -73.4434851036349) bank86329 +86330 POINT(39.88886418100283 -73.73212683335555) bank86330 +86331 POINT(40.52423970932358 -73.98393636662882) bank86331 +86332 POINT(40.310601822759544 -74.93345818642113) bank86332 +86333 POINT(41.38751294528471 -74.96140023611184) bank86333 +86334 POINT(41.58843603736161 -74.41990613334244) bank86334 +86335 POINT(40.77211976168199 -74.07962734420957) bank86335 +86336 POINT(40.16930748730037 -74.63705828743234) bank86336 +86337 POINT(40.30016739556052 -73.40579120023405) bank86337 +86338 POINT(40.49206784696986 -73.2382709316767) bank86338 +86339 POINT(40.08616910772561 -74.77596852022334) bank86339 +86340 POINT(41.546616870625066 -74.1983415997448) bank86340 +86341 POINT(41.02586465417241 -74.54442202101798) bank86341 +86342 POINT(39.9266511185287 -73.37790759605525) bank86342 +86343 POINT(40.23203684396704 -74.05860220992507) bank86343 +86344 POINT(40.49585906947676 -73.61729117196505) bank86344 +86345 POINT(40.94035800664785 -73.81532428295466) bank86345 +86346 POINT(40.59227550230297 -73.15393774375536) bank86346 +86347 POINT(40.84606979891063 -73.6794536011891) bank86347 +86348 POINT(41.67609693102626 -73.78068156699965) bank86348 +86349 POINT(39.85833790546153 -73.8651512450665) bank86349 +86350 POINT(40.023743526476814 -73.52353701461954) bank86350 +86351 POINT(41.483202897370205 -73.54571430516118) bank86351 +86352 POINT(41.366254041642144 -73.16433544842259) bank86352 +86353 POINT(40.104188335542325 -74.82670135859998) bank86353 +86354 POINT(40.95504328720803 -73.5414597143913) bank86354 +86355 POINT(39.81813276037346 -74.34245905347618) bank86355 +86356 POINT(41.021068872509986 -73.23439484955787) bank86356 +86357 POINT(40.66777530521383 -73.0325069409728) bank86357 +86358 POINT(40.18790059596822 -74.8036618245693) bank86358 +86359 POINT(40.52906340447666 -73.468638905942) bank86359 +86360 POINT(41.30441547651659 -73.04651039909733) bank86360 +86361 POINT(41.04855882156956 -74.5163001794659) bank86361 +86362 POINT(39.868076744039875 -73.35222831107495) bank86362 +86363 POINT(40.4141850441478 -74.01777141914168) bank86363 +86364 POINT(40.418722100216584 -74.56575622088327) bank86364 +86365 POINT(40.37395500980205 -74.61611027740763) bank86365 +86366 POINT(41.01156858415487 -73.16071593493355) bank86366 +86367 POINT(40.224390629121025 -74.42974549474) bank86367 +86368 POINT(39.73020360191045 -74.10793709859432) bank86368 +86369 POINT(41.179412755394935 -73.7207988965725) bank86369 +86370 POINT(40.4928715484125 -74.31011804394906) bank86370 +86371 POINT(40.94712938855816 -73.65593295048522) bank86371 +86372 POINT(41.62464502125305 -73.50950339610392) bank86372 +86373 POINT(40.40039807540358 -73.10604283157393) bank86373 +86374 POINT(41.47504997095653 -73.6589596589763) bank86374 +86375 POINT(41.641924608392905 -74.7782830758586) bank86375 +86376 POINT(40.709244289633425 -74.25915703100273) bank86376 +86377 POINT(40.05269466225429 -74.04398590858632) bank86377 +86378 POINT(41.00217844973387 -74.838577290341) bank86378 +86379 POINT(41.57388411848249 -74.69010353406809) bank86379 +86380 POINT(40.27961919243109 -74.54086204235492) bank86380 +86381 POINT(40.76768587500538 -74.73434552912937) bank86381 +86382 POINT(40.450237004865684 -73.5059751640132) bank86382 +86383 POINT(39.93699015230643 -73.88128300793319) bank86383 +86384 POINT(41.654506894740734 -74.04416902807112) bank86384 +86385 POINT(40.05372971462727 -74.78559138791046) bank86385 +86386 POINT(40.3861074632135 -74.09747231809388) bank86386 +86387 POINT(41.18501614930254 -73.5632756040819) bank86387 +86388 POINT(41.578970554763664 -73.76185201765735) bank86388 +86389 POINT(41.57177264752024 -74.291128927843) bank86389 +86390 POINT(40.26562847886658 -74.9175629307952) bank86390 +86391 POINT(40.67584545509578 -73.34478395715382) bank86391 +86392 POINT(41.349343583929794 -73.09079544981822) bank86392 +86393 POINT(40.32652095541118 -73.95299839602662) bank86393 +86394 POINT(41.202052780847254 -73.2861031075723) bank86394 +86395 POINT(41.50822580631572 -74.14302764710412) bank86395 +86396 POINT(40.745744665205265 -73.52555046912653) bank86396 +86397 POINT(40.26089877065455 -73.05478877744206) bank86397 +86398 POINT(41.543930487474455 -74.36852400788626) bank86398 +86399 POINT(41.34256885206631 -73.05161711334752) bank86399 +86400 POINT(40.29083247102385 -74.03405587170909) bank86400 +86401 POINT(40.834503194444835 -74.77839890523235) bank86401 +86402 POINT(40.518141122405346 -74.02024050573279) bank86402 +86403 POINT(41.121940212915064 -73.48848894580497) bank86403 +86404 POINT(40.45191324864851 -73.71303604934378) bank86404 +86405 POINT(40.56745076150438 -73.76329280029715) bank86405 +86406 POINT(40.997329086757524 -74.44938830411611) bank86406 +86407 POINT(41.504728624403526 -73.43542770025466) bank86407 +86408 POINT(41.67617008583062 -74.19332581584665) bank86408 +86409 POINT(41.23464358029691 -74.09641154917038) bank86409 +86410 POINT(40.71093163590803 -74.34839288400305) bank86410 +86411 POINT(39.72059693153504 -74.09764083140897) bank86411 +86412 POINT(40.900622712102994 -73.58556107011924) bank86412 +86413 POINT(41.5913899982804 -73.03130306150936) bank86413 +86414 POINT(41.027016319109045 -73.37485760980827) bank86414 +86415 POINT(39.97010353158714 -74.9487200360727) bank86415 +86416 POINT(39.96191873251649 -73.52687552298532) bank86416 +86417 POINT(40.6617273739897 -73.77732615010642) bank86417 +86418 POINT(40.555231087485666 -74.77465433583579) bank86418 +86419 POINT(40.30392870830613 -74.4730177782049) bank86419 +86420 POINT(40.77972205272176 -73.20435964800026) bank86420 +86421 POINT(40.038574597059 -74.80836533050451) bank86421 +86422 POINT(40.43458312649232 -74.6872846926979) bank86422 +86423 POINT(40.93425612217723 -73.8587298778777) bank86423 +86424 POINT(40.87105176329079 -73.28813305458743) bank86424 +86425 POINT(41.28291033512668 -73.1327541474894) bank86425 +86426 POINT(40.60496695961213 -74.50421522862456) bank86426 +86427 POINT(40.77036012822362 -73.69355063829475) bank86427 +86428 POINT(39.906607833485424 -74.84014707463504) bank86428 +86429 POINT(41.55735523780541 -73.7297879591618) bank86429 +86430 POINT(40.75094925719946 -74.9541424006624) bank86430 +86431 POINT(41.39759998406819 -73.06216385308925) bank86431 +86432 POINT(41.45640789893096 -74.97511562840734) bank86432 +86433 POINT(39.78932603704183 -73.23240555453637) bank86433 +86434 POINT(41.03010925929527 -73.50576117855441) bank86434 +86435 POINT(41.64204580770037 -73.06100865526054) bank86435 +86436 POINT(41.599946667083096 -74.31294870172731) bank86436 +86437 POINT(40.13813739243523 -73.02323157122692) bank86437 +86438 POINT(40.29038066190466 -73.12859153151169) bank86438 +86439 POINT(41.06951568540839 -73.95187667136794) bank86439 +86440 POINT(40.07812127413509 -74.33055382926177) bank86440 +86441 POINT(40.078821232346165 -73.16398908887126) bank86441 +86442 POINT(40.70712418006992 -73.9617994290091) bank86442 +86443 POINT(39.84752896630442 -74.06589397762994) bank86443 +86444 POINT(41.62017257450534 -73.25834009277833) bank86444 +86445 POINT(40.92675477997531 -74.0439526236509) bank86445 +86446 POINT(39.924572465282054 -74.37794593453445) bank86446 +86447 POINT(40.71663755492359 -74.31575453194414) bank86447 +86448 POINT(41.0713232311964 -74.5985646267983) bank86448 +86449 POINT(40.95200864954094 -74.75849839852795) bank86449 +86450 POINT(40.33445385897851 -74.48232548764466) bank86450 +86451 POINT(39.996852049471634 -73.9801261513368) bank86451 +86452 POINT(41.57827705933522 -73.23103058502336) bank86452 +86453 POINT(40.29017933618348 -73.29166366896892) bank86453 +86454 POINT(41.6575730274831 -74.81481021773409) bank86454 +86455 POINT(41.51442328591387 -73.50510164067528) bank86455 +86456 POINT(41.241758556713876 -73.86225903030343) bank86456 +86457 POINT(40.79647751343563 -74.25708405665344) bank86457 +86458 POINT(40.52741389620001 -73.49552160975165) bank86458 +86459 POINT(39.89020556284857 -73.87051070875037) bank86459 +86460 POINT(40.15026747905515 -73.14901527007848) bank86460 +86461 POINT(40.743735168804946 -73.6506658426063) bank86461 +86462 POINT(40.922607575838946 -73.21706584423102) bank86462 +86463 POINT(40.33493093798977 -73.36273331726802) bank86463 +86464 POINT(40.37117272059227 -74.03659228770103) bank86464 +86465 POINT(39.80312398856769 -74.2058588989719) bank86465 +86466 POINT(40.12388000292362 -74.62138657694116) bank86466 +86467 POINT(41.31769353584767 -73.19813873798697) bank86467 +86468 POINT(40.81940749484684 -73.41638982806612) bank86468 +86469 POINT(40.3656907409266 -74.04026715936614) bank86469 +86470 POINT(40.144231955530735 -74.64629601444364) bank86470 +86471 POINT(39.86024942839877 -73.40905414859826) bank86471 +86472 POINT(41.02136785629261 -73.89898962797221) bank86472 +86473 POINT(41.667612803040626 -73.60109553281093) bank86473 +86474 POINT(40.953608307914166 -73.02881853465178) bank86474 +86475 POINT(41.29935564966182 -74.4108226624406) bank86475 +86476 POINT(40.75021108346772 -74.18322110788831) bank86476 +86477 POINT(39.99316210544467 -74.70120432824709) bank86477 +86478 POINT(41.08307893429443 -73.77963936151035) bank86478 +86479 POINT(40.99088319387713 -73.51076616717454) bank86479 +86480 POINT(41.64799359702947 -73.9903929949846) bank86480 +86481 POINT(41.28691454585799 -73.3237193693133) bank86481 +86482 POINT(40.75577652200206 -74.07014658114338) bank86482 +86483 POINT(40.2104234228577 -74.75913745187093) bank86483 +86484 POINT(39.98571924761326 -74.12537997826948) bank86484 +86485 POINT(41.33454901337507 -74.54432718258944) bank86485 +86486 POINT(40.05717903985049 -73.32942251411443) bank86486 +86487 POINT(40.53068368963284 -74.38697658239093) bank86487 +86488 POINT(40.34937455489764 -73.7447623531055) bank86488 +86489 POINT(39.93708543071608 -73.32537085425793) bank86489 +86490 POINT(39.797185475817045 -74.44804028747485) bank86490 +86491 POINT(39.96996699209746 -74.98225050608848) bank86491 +86492 POINT(41.22462158457664 -73.93845252639542) bank86492 +86493 POINT(40.48417854336921 -74.80348268148863) bank86493 +86494 POINT(40.18182477153212 -73.65181503952961) bank86494 +86495 POINT(40.062325869735545 -73.5970057631964) bank86495 +86496 POINT(41.70835106810685 -73.57042572027609) bank86496 +86497 POINT(40.200423547876504 -74.03771789779687) bank86497 +86498 POINT(40.94182854398816 -73.415559629476) bank86498 +86499 POINT(40.879684155970665 -74.83904360716863) bank86499 +86500 POINT(41.43775328623821 -74.15253688340849) bank86500 +86501 POINT(41.66075095476207 -74.80112129053929) bank86501 +86502 POINT(40.64618857963272 -73.19207535633828) bank86502 +86503 POINT(41.533344787245746 -73.99977821227907) bank86503 +86504 POINT(41.511438211112086 -74.16125208639299) bank86504 +86505 POINT(41.468733748004176 -74.01106644424176) bank86505 +86506 POINT(39.84117819038672 -73.41990399796465) bank86506 +86507 POINT(40.92279156929504 -74.01063857862101) bank86507 +86508 POINT(40.09208921096215 -74.65948402832962) bank86508 +86509 POINT(40.931705473108615 -73.19836276525149) bank86509 +86510 POINT(41.47612548308014 -73.57945189000006) bank86510 +86511 POINT(40.68666667519955 -74.35044559483939) bank86511 +86512 POINT(41.077392495920996 -73.91142102151248) bank86512 +86513 POINT(40.7442753638567 -74.4849497000355) bank86513 +86514 POINT(41.41888746207157 -74.84811239851621) bank86514 +86515 POINT(41.31390437685861 -74.20789813395771) bank86515 +86516 POINT(41.158575972660415 -73.86918984628282) bank86516 +86517 POINT(39.74826132358287 -74.165772285085) bank86517 +86518 POINT(40.07481903661525 -73.17312310491008) bank86518 +86519 POINT(39.74602732929916 -73.36491443997488) bank86519 +86520 POINT(41.609629073149655 -74.20260600157395) bank86520 +86521 POINT(41.039854986922904 -74.91123137609927) bank86521 +86522 POINT(39.928702227768376 -74.94335397236905) bank86522 +86523 POINT(41.49235613130376 -73.82070913139373) bank86523 +86524 POINT(41.280738735557755 -73.10152233608333) bank86524 +86525 POINT(41.66818874571083 -74.46063081866858) bank86525 +86526 POINT(39.766078344976364 -73.76591901959206) bank86526 +86527 POINT(40.01889518336392 -73.7634123384107) bank86527 +86528 POINT(41.081818588539164 -73.18839249385957) bank86528 +86529 POINT(41.679737118158556 -73.09782175488175) bank86529 +86530 POINT(40.56639344819723 -73.49157911141172) bank86530 +86531 POINT(40.71344189072757 -74.86034865567082) bank86531 +86532 POINT(40.699604801858634 -74.30465137326064) bank86532 +86533 POINT(41.33789015591831 -74.32842232503208) bank86533 +86534 POINT(40.42074902331265 -74.21270891216821) bank86534 +86535 POINT(40.69254591736632 -73.80522350528724) bank86535 +86536 POINT(40.14788481320565 -74.35842227853708) bank86536 +86537 POINT(41.441094503523715 -73.85399960491148) bank86537 +86538 POINT(39.71910971969243 -73.69315435017324) bank86538 +86539 POINT(41.04208538884682 -74.21949729749487) bank86539 +86540 POINT(40.82238879159315 -74.57376904164768) bank86540 +86541 POINT(41.14034717393317 -73.18833012823553) bank86541 +86542 POINT(41.00419273342979 -73.32167989727549) bank86542 +86543 POINT(40.64635711362488 -73.96959040423448) bank86543 +86544 POINT(39.93389347726454 -73.46507291215065) bank86544 +86545 POINT(40.88586177419569 -73.2677160335114) bank86545 +86546 POINT(39.8836297690271 -74.26370025482585) bank86546 +86547 POINT(41.08251650930622 -73.31815780745126) bank86547 +86548 POINT(39.92031336176432 -73.35489962707398) bank86548 +86549 POINT(41.023994475707184 -74.32012803480241) bank86549 +86550 POINT(39.8639147018285 -73.74383171050141) bank86550 +86551 POINT(40.54508915578245 -74.7912223365543) bank86551 +86552 POINT(41.049663511244525 -73.13280838798977) bank86552 +86553 POINT(40.18700168354446 -73.7695290087153) bank86553 +86554 POINT(39.853504721984955 -73.9675132147847) bank86554 +86555 POINT(40.70533155512809 -73.89281621546114) bank86555 +86556 POINT(41.57805234782881 -74.7774223601768) bank86556 +86557 POINT(41.62554555441968 -73.39222882531345) bank86557 +86558 POINT(39.8090841328567 -73.348224881398) bank86558 +86559 POINT(40.65089115623045 -73.38332667035036) bank86559 +86560 POINT(40.75685126755183 -74.38533980136224) bank86560 +86561 POINT(41.234457300961786 -73.10725757317071) bank86561 +86562 POINT(40.87251440887198 -74.57371440930335) bank86562 +86563 POINT(41.36879973160288 -73.5260767914041) bank86563 +86564 POINT(41.236623267275874 -74.3517043160645) bank86564 +86565 POINT(40.49845069498277 -73.60501399514872) bank86565 +86566 POINT(41.091246284738446 -73.16960709804091) bank86566 +86567 POINT(41.208265217537416 -74.80593694435501) bank86567 +86568 POINT(39.904997441281125 -73.96948264248938) bank86568 +86569 POINT(41.5889710665544 -74.14638785401843) bank86569 +86570 POINT(40.06442865945687 -73.5350004166856) bank86570 +86571 POINT(41.60189598801301 -73.41748573943595) bank86571 +86572 POINT(41.51589201004271 -73.10966615264284) bank86572 +86573 POINT(41.51641708171534 -74.65083057066272) bank86573 +86574 POINT(40.30424452183396 -74.04482049486924) bank86574 +86575 POINT(41.13985212337693 -73.33879005746189) bank86575 +86576 POINT(41.055198397193536 -74.78338012919563) bank86576 +86577 POINT(41.633340372872944 -73.04190355589252) bank86577 +86578 POINT(40.730202559087026 -74.466307411092) bank86578 +86579 POINT(41.2976335460847 -73.13870349542924) bank86579 +86580 POINT(40.242034100785496 -73.34563674791387) bank86580 +86581 POINT(39.94597988239538 -73.10399805191133) bank86581 +86582 POINT(40.44474835052458 -74.85128718539512) bank86582 +86583 POINT(39.779601735707224 -73.78954754) bank86583 +86584 POINT(40.96799430128783 -74.21267712314355) bank86584 +86585 POINT(39.71794124423575 -74.29596827998036) bank86585 +86586 POINT(39.904534931872504 -73.24199541398589) bank86586 +86587 POINT(41.29754938504715 -74.2563931145985) bank86587 +86588 POINT(39.95719585593393 -73.99516973896061) bank86588 +86589 POINT(41.2306109421894 -73.160284479717) bank86589 +86590 POINT(41.46704915588801 -73.43822716938035) bank86590 +86591 POINT(40.75883904692824 -74.8323337382905) bank86591 +86592 POINT(40.449307357664914 -73.32386515206818) bank86592 +86593 POINT(41.17308021999595 -73.75160422939808) bank86593 +86594 POINT(40.29251051519203 -73.97048203320688) bank86594 +86595 POINT(39.990247982233896 -74.11899689376415) bank86595 +86596 POINT(41.050190164868454 -74.11327324004652) bank86596 +86597 POINT(39.87278221669481 -74.36872653007066) bank86597 +86598 POINT(40.83791062904361 -74.34253848621245) bank86598 +86599 POINT(41.6441824585123 -73.1480844669496) bank86599 +86600 POINT(40.31937952425738 -74.99318894043022) bank86600 +86601 POINT(40.668378431915 -74.89016598607448) bank86601 +86602 POINT(40.44448125326119 -74.29976817576755) bank86602 +86603 POINT(41.05308314289286 -74.85028828817576) bank86603 +86604 POINT(41.45852655206872 -74.39741055209187) bank86604 +86605 POINT(40.70847366486128 -74.74851279961831) bank86605 +86606 POINT(41.5934998691765 -74.04033496785753) bank86606 +86607 POINT(41.22536463265143 -74.56590630505713) bank86607 +86608 POINT(40.627970030986454 -74.9740362447849) bank86608 +86609 POINT(39.71662659942564 -73.74538209795062) bank86609 +86610 POINT(41.254650997399985 -73.47704240452006) bank86610 +86611 POINT(41.41169118961894 -74.96220448378106) bank86611 +86612 POINT(39.75813510883769 -73.08623264012223) bank86612 +86613 POINT(40.22687285704514 -73.7993806615319) bank86613 +86614 POINT(40.04620195995264 -73.12800257092816) bank86614 +86615 POINT(40.69198163735043 -74.78311263596399) bank86615 +86616 POINT(41.046475096968514 -74.74578566108691) bank86616 +86617 POINT(41.00629526699013 -73.74203109002043) bank86617 +86618 POINT(40.35124696080652 -73.77094755220412) bank86618 +86619 POINT(41.56248101881818 -73.7008270107774) bank86619 +86620 POINT(41.36666069998223 -74.82010021375731) bank86620 +86621 POINT(41.547255270736095 -73.78191565874992) bank86621 +86622 POINT(40.45766622945272 -74.53687551446679) bank86622 +86623 POINT(41.60224575846744 -74.76142599740348) bank86623 +86624 POINT(39.796818546867925 -75.00515298089987) bank86624 +86625 POINT(40.355870156471426 -73.12388664870372) bank86625 +86626 POINT(40.30544079983002 -73.99642940419655) bank86626 +86627 POINT(41.002589024782395 -74.05354448784139) bank86627 +86628 POINT(40.282254352147994 -74.41006248131083) bank86628 +86629 POINT(40.236527693899035 -73.67595220827019) bank86629 +86630 POINT(41.62509416558865 -73.69012344905528) bank86630 +86631 POINT(40.023807749691656 -73.09261184373767) bank86631 +86632 POINT(40.054691544252314 -73.16091328635274) bank86632 +86633 POINT(40.80208886642341 -73.77509174709395) bank86633 +86634 POINT(40.89996750710026 -74.33199552836086) bank86634 +86635 POINT(41.06308795365251 -74.86957757926673) bank86635 +86636 POINT(40.33674037428784 -74.19446107346411) bank86636 +86637 POINT(40.95833809088406 -73.95768751910042) bank86637 +86638 POINT(39.73513146965887 -73.71637742484499) bank86638 +86639 POINT(40.06324408145401 -73.18041684811216) bank86639 +86640 POINT(40.897817826553236 -74.52780435913516) bank86640 +86641 POINT(40.205008859441136 -73.08003183970438) bank86641 +86642 POINT(41.20279878620144 -73.23219270155529) bank86642 +86643 POINT(40.941477426671376 -73.21421132301863) bank86643 +86644 POINT(41.659099060166426 -74.94154415536882) bank86644 +86645 POINT(39.894452779656966 -74.05092365749438) bank86645 +86646 POINT(41.70329031775822 -74.66725765922655) bank86646 +86647 POINT(40.84670961642076 -74.29600374155001) bank86647 +86648 POINT(39.74687506157064 -74.46990870563636) bank86648 +86649 POINT(39.84180697118021 -74.16327931445339) bank86649 +86650 POINT(41.24949952615462 -74.93720562861272) bank86650 +86651 POINT(41.5190003138335 -74.72704830298109) bank86651 +86652 POINT(40.15378337636801 -74.50576782231212) bank86652 +86653 POINT(40.76124077413342 -73.75041235979329) bank86653 +86654 POINT(41.693497393716186 -73.5164341017703) bank86654 +86655 POINT(40.1437328085078 -74.65206842279487) bank86655 +86656 POINT(39.96172813574909 -74.94366206722657) bank86656 +86657 POINT(40.28759841054029 -73.32505205322265) bank86657 +86658 POINT(41.1057603344463 -73.73427903523681) bank86658 +86659 POINT(40.348700990623016 -73.38215845623095) bank86659 +86660 POINT(40.2834597914711 -74.28280735399365) bank86660 +86661 POINT(40.91035936719133 -73.90193196445085) bank86661 +86662 POINT(40.763726896711304 -73.8290868009667) bank86662 +86663 POINT(40.660182918802 -73.31330988514118) bank86663 +86664 POINT(40.86490046291645 -73.77571927918464) bank86664 +86665 POINT(41.65319124833597 -74.4441475927458) bank86665 +86666 POINT(39.901188546421295 -73.46159292158339) bank86666 +86667 POINT(39.72549192198117 -74.24862150215348) bank86667 +86668 POINT(40.39027232488947 -73.38734708260908) bank86668 +86669 POINT(41.55401740953217 -74.7323448985967) bank86669 +86670 POINT(40.66602794697711 -73.94625121557635) bank86670 +86671 POINT(41.122354448818434 -73.03858650613358) bank86671 +86672 POINT(41.635905442868506 -74.0882824542506) bank86672 +86673 POINT(41.125680026835404 -73.82884015217348) bank86673 +86674 POINT(40.35544499068322 -73.808181692027) bank86674 +86675 POINT(40.9712898095352 -74.9022758221663) bank86675 +86676 POINT(41.235692655760246 -73.82284240628125) bank86676 +86677 POINT(41.1909845356152 -74.30345364328784) bank86677 +86678 POINT(40.89059835416208 -74.22228361295831) bank86678 +86679 POINT(41.691543556334636 -73.02320507426768) bank86679 +86680 POINT(41.03165700192695 -74.88503906751369) bank86680 +86681 POINT(41.60333425340531 -73.60307311175723) bank86681 +86682 POINT(40.11162910670634 -73.7797360258288) bank86682 +86683 POINT(40.35946578369572 -74.82594183783802) bank86683 +86684 POINT(40.82874381993967 -73.17516828208963) bank86684 +86685 POINT(40.82634322356602 -74.89823994557834) bank86685 +86686 POINT(40.967025883325526 -74.13625713220208) bank86686 +86687 POINT(41.05054259703618 -73.37056200245796) bank86687 +86688 POINT(41.468320291209864 -74.52023225691872) bank86688 +86689 POINT(41.56666542126601 -74.43470250242865) bank86689 +86690 POINT(40.046695161924596 -73.21984925075915) bank86690 +86691 POINT(39.750152380037626 -73.1204220214952) bank86691 +86692 POINT(40.96973152317515 -74.44704759711172) bank86692 +86693 POINT(41.30149897073406 -73.22991433459232) bank86693 +86694 POINT(40.57564092831448 -73.32413864858482) bank86694 +86695 POINT(39.92984321841752 -73.25983346510556) bank86695 +86696 POINT(41.09459222035607 -74.57892593516232) bank86696 +86697 POINT(41.45475672779475 -74.78334454903596) bank86697 +86698 POINT(41.65622183091952 -74.05055458134832) bank86698 +86699 POINT(41.372297887696334 -74.97166322644081) bank86699 +86700 POINT(40.121171123607624 -74.98960888245635) bank86700 +86701 POINT(40.32684136524736 -73.28803691679495) bank86701 +86702 POINT(40.53886906704584 -74.58445712829754) bank86702 +86703 POINT(40.90026453427245 -74.15278039208938) bank86703 +86704 POINT(41.22479441246014 -73.32790591907252) bank86704 +86705 POINT(41.20032532055001 -73.57218174703549) bank86705 +86706 POINT(40.36686688401377 -73.58391748878167) bank86706 +86707 POINT(40.37236333771992 -74.11631962510675) bank86707 +86708 POINT(41.23495600007996 -73.32533420616521) bank86708 +86709 POINT(40.070582268963946 -74.08295193195536) bank86709 +86710 POINT(40.9971659292826 -73.86095940831983) bank86710 +86711 POINT(40.725765164454266 -73.84255541069223) bank86711 +86712 POINT(40.40918586981027 -74.3726455201928) bank86712 +86713 POINT(39.969728676848874 -73.78069614106838) bank86713 +86714 POINT(40.00339943913028 -74.59906205084613) bank86714 +86715 POINT(40.05613403302354 -73.11761730133908) bank86715 +86716 POINT(40.00803999566889 -73.65091404949207) bank86716 +86717 POINT(39.86652637540958 -73.69952752922615) bank86717 +86718 POINT(41.617758005810565 -73.04688150616467) bank86718 +86719 POINT(40.14698845217983 -73.00994442650996) bank86719 +86720 POINT(40.00070560797874 -73.79466367426093) bank86720 +86721 POINT(41.34641662181386 -73.63950443791235) bank86721 +86722 POINT(41.21338208008376 -74.134703539384) bank86722 +86723 POINT(40.81319226915956 -73.06039945223952) bank86723 +86724 POINT(41.25499217279717 -74.61560237701812) bank86724 +86725 POINT(40.1319261036878 -73.34131531358071) bank86725 +86726 POINT(41.232145498229855 -74.63567275486362) bank86726 +86727 POINT(41.36518618750103 -74.7773708376327) bank86727 +86728 POINT(40.465665952437824 -74.39271733960602) bank86728 +86729 POINT(40.88554198387286 -74.93735584636441) bank86729 +86730 POINT(41.59111013778855 -73.76936500565942) bank86730 +86731 POINT(40.96894197772625 -74.8082685722782) bank86731 +86732 POINT(40.708192076796 -73.12186081737968) bank86732 +86733 POINT(40.11906001304278 -74.71932405067494) bank86733 +86734 POINT(39.81946632928095 -73.1737676177834) bank86734 +86735 POINT(41.10013500137784 -74.40064920441924) bank86735 +86736 POINT(40.71436966771098 -73.48761067653828) bank86736 +86737 POINT(41.20480830242856 -73.25010968036568) bank86737 +86738 POINT(41.30392290468368 -74.34606451651776) bank86738 +86739 POINT(40.84621577995806 -73.30302854680977) bank86739 +86740 POINT(41.13934865086714 -73.26485647049437) bank86740 +86741 POINT(40.303541527274554 -74.6347879427433) bank86741 +86742 POINT(41.541184625608224 -74.53082640944857) bank86742 +86743 POINT(41.112721441206 -73.21980858481759) bank86743 +86744 POINT(40.38942359022543 -73.46365347612894) bank86744 +86745 POINT(39.788140932989194 -74.91924660239296) bank86745 +86746 POINT(41.487713847265596 -74.60822286840025) bank86746 +86747 POINT(39.95472067124917 -74.80411922710702) bank86747 +86748 POINT(41.679149060671826 -74.49434742027131) bank86748 +86749 POINT(40.397652798749434 -74.72544816276269) bank86749 +86750 POINT(41.66801741855107 -74.7098466066124) bank86750 +86751 POINT(40.49309771371876 -74.63796061008934) bank86751 +86752 POINT(39.86210453033779 -73.4537454257881) bank86752 +86753 POINT(40.641931284848795 -74.75231489162438) bank86753 +86754 POINT(40.1055091811328 -74.57943050855876) bank86754 +86755 POINT(40.05387067552032 -73.1766479676356) bank86755 +86756 POINT(40.24326280116181 -73.77828548827763) bank86756 +86757 POINT(41.35449547063179 -73.39421792820843) bank86757 +86758 POINT(41.44727217612746 -73.92300911441944) bank86758 +86759 POINT(41.42851758828095 -73.59552370219666) bank86759 +86760 POINT(41.67211761064342 -74.05125517487298) bank86760 +86761 POINT(40.06049431306221 -73.86957144096114) bank86761 +86762 POINT(40.06186931380663 -73.56317388578637) bank86762 +86763 POINT(40.49328357805845 -74.9350126455016) bank86763 +86764 POINT(39.81296917259683 -73.59403877283644) bank86764 +86765 POINT(41.66691802255953 -74.65805575731731) bank86765 +86766 POINT(40.74318385663506 -73.03647546663818) bank86766 +86767 POINT(40.29400594293478 -73.77016643801043) bank86767 +86768 POINT(40.27544208009208 -74.44836774536257) bank86768 +86769 POINT(40.405243694879886 -74.49471210667647) bank86769 +86770 POINT(41.640351351455536 -74.24411567692431) bank86770 +86771 POINT(39.92041274998137 -73.56047714437577) bank86771 +86772 POINT(39.77006108240082 -73.31005036092827) bank86772 +86773 POINT(40.018406396389274 -73.62640961646865) bank86773 +86774 POINT(40.58594271577276 -74.72299882592152) bank86774 +86775 POINT(40.382021335306206 -74.85653650456781) bank86775 +86776 POINT(40.96377383445265 -73.76075774735529) bank86776 +86777 POINT(40.33894324528083 -73.32521249036667) bank86777 +86778 POINT(41.51061536486908 -74.71967629595014) bank86778 +86779 POINT(41.08313365163862 -73.99817478763829) bank86779 +86780 POINT(41.43740154055126 -74.29862442117721) bank86780 +86781 POINT(39.999317199654016 -73.27284533807172) bank86781 +86782 POINT(40.879394951405125 -74.73947450719085) bank86782 +86783 POINT(41.34783703784557 -74.4105854758968) bank86783 +86784 POINT(41.53860532838182 -74.22307256024177) bank86784 +86785 POINT(41.694347941035176 -73.56676249754324) bank86785 +86786 POINT(40.665894555528844 -73.06222581108915) bank86786 +86787 POINT(41.08175014511701 -74.69813518476519) bank86787 +86788 POINT(41.004791257015384 -73.3882626318073) bank86788 +86789 POINT(40.98061686981028 -73.13741210709142) bank86789 +86790 POINT(39.81072608503587 -73.701701701431) bank86790 +86791 POINT(40.118315575034146 -73.5145362834615) bank86791 +86792 POINT(41.58043702202877 -73.16009679501614) bank86792 +86793 POINT(40.69648041295742 -74.73438646422504) bank86793 +86794 POINT(41.4436123736945 -74.57395058383072) bank86794 +86795 POINT(41.10338209995482 -73.27448218062761) bank86795 +86796 POINT(39.89532887316207 -73.22753300458484) bank86796 +86797 POINT(40.117841931377754 -74.67351381091115) bank86797 +86798 POINT(40.95088137245556 -73.49587413699956) bank86798 +86799 POINT(40.55688897602484 -73.76017697288226) bank86799 +86800 POINT(40.56570235903493 -74.28569940142125) bank86800 +86801 POINT(40.955327420588404 -73.5033850823592) bank86801 +86802 POINT(39.844588821779865 -74.03372550070539) bank86802 +86803 POINT(40.043084980925606 -73.06557220291624) bank86803 +86804 POINT(40.260122997683006 -73.83135754478494) bank86804 +86805 POINT(39.8251475354778 -73.6855219389781) bank86805 +86806 POINT(41.54102746073529 -73.97228077778968) bank86806 +86807 POINT(39.75586945108149 -73.73791287827223) bank86807 +86808 POINT(40.31351505970138 -73.90358452284143) bank86808 +86809 POINT(40.090993150725616 -73.0942654120798) bank86809 +86810 POINT(40.68428661612448 -73.73349551814569) bank86810 +86811 POINT(41.12914121496832 -74.1344509785221) bank86811 +86812 POINT(40.29257413055781 -73.69127304979588) bank86812 +86813 POINT(40.990605269076 -74.81977036851893) bank86813 +86814 POINT(41.455494692079135 -74.33330728608125) bank86814 +86815 POINT(39.90018435494649 -74.18349574051061) bank86815 +86816 POINT(40.45765679400335 -73.66953363664115) bank86816 +86817 POINT(40.06048361531292 -73.95906736999208) bank86817 +86818 POINT(40.08263889399763 -73.96805472426071) bank86818 +86819 POINT(40.84558440158138 -73.40669592654557) bank86819 +86820 POINT(41.25724686269165 -73.02188791347363) bank86820 +86821 POINT(41.036515495127695 -73.24031742522726) bank86821 +86822 POINT(39.98911851412089 -73.01994479167361) bank86822 +86823 POINT(40.7277178375186 -73.1719091868793) bank86823 +86824 POINT(41.44848370066377 -74.9144639913698) bank86824 +86825 POINT(40.23079313845354 -74.19885062600257) bank86825 +86826 POINT(39.90631096813234 -74.2944168221503) bank86826 +86827 POINT(41.677659954891496 -74.82885840050946) bank86827 +86828 POINT(40.484123758944044 -73.91440015356422) bank86828 +86829 POINT(40.83819430018451 -73.67183441425449) bank86829 +86830 POINT(40.49556820735148 -74.58969494580361) bank86830 +86831 POINT(41.11962220253573 -74.52586456999852) bank86831 +86832 POINT(41.1784983782203 -73.85619277448329) bank86832 +86833 POINT(39.72172193004116 -74.4631787320442) bank86833 +86834 POINT(41.44753026315736 -73.11604628148048) bank86834 +86835 POINT(41.329321294278465 -73.37132449305972) bank86835 +86836 POINT(39.967036797089655 -74.87741979004767) bank86836 +86837 POINT(41.37582733304051 -74.56584366423893) bank86837 +86838 POINT(40.90155223951615 -73.18443508128868) bank86838 +86839 POINT(40.520150416331354 -73.18987137185772) bank86839 +86840 POINT(40.36232669211004 -74.4118010114758) bank86840 +86841 POINT(39.760841608557065 -73.50822845618575) bank86841 +86842 POINT(41.244798476765425 -74.20819189667331) bank86842 +86843 POINT(41.176083012995456 -74.230954269541) bank86843 +86844 POINT(40.31946168303145 -73.76406610212481) bank86844 +86845 POINT(40.07435370120516 -74.02040877206805) bank86845 +86846 POINT(41.47042941682567 -73.91184163264391) bank86846 +86847 POINT(40.67701411037211 -74.95009717310924) bank86847 +86848 POINT(41.231063483630564 -74.67082834210926) bank86848 +86849 POINT(41.516845367577886 -73.8907255646918) bank86849 +86850 POINT(41.32574606190846 -73.95469501760277) bank86850 +86851 POINT(40.581427211010784 -73.46126175743875) bank86851 +86852 POINT(41.37923496247089 -73.02105632024517) bank86852 +86853 POINT(39.82029386874796 -74.60926065748566) bank86853 +86854 POINT(40.50603905025998 -73.77831111419168) bank86854 +86855 POINT(39.82150116640864 -73.64823533359468) bank86855 +86856 POINT(41.54759601334769 -73.38661279917454) bank86856 +86857 POINT(41.1446674709564 -74.11772049287188) bank86857 +86858 POINT(40.91386248729785 -74.33287728303911) bank86858 +86859 POINT(41.32463964959396 -74.11270566207442) bank86859 +86860 POINT(39.901065130311295 -73.87047451825764) bank86860 +86861 POINT(39.89557023752427 -73.17016368957138) bank86861 +86862 POINT(40.99313040143562 -73.26050352947253) bank86862 +86863 POINT(40.089947780751075 -74.31502257093199) bank86863 +86864 POINT(41.384318399196395 -74.5156531128145) bank86864 +86865 POINT(41.312102352853245 -73.06410884519993) bank86865 +86866 POINT(40.0135475289242 -74.64631101545734) bank86866 +86867 POINT(41.50336766361849 -74.33730719300085) bank86867 +86868 POINT(40.399087191825195 -74.2591242565629) bank86868 +86869 POINT(40.08613820081524 -74.238345628266) bank86869 +86870 POINT(41.10811779724885 -73.70178478443677) bank86870 +86871 POINT(41.5174316566885 -73.9072686453526) bank86871 +86872 POINT(40.13184432620777 -73.70412758872745) bank86872 +86873 POINT(41.61289533245637 -73.16086837780266) bank86873 +86874 POINT(41.20013443460372 -74.39942217538595) bank86874 +86875 POINT(39.92554007010498 -74.49860107953981) bank86875 +86876 POINT(40.4415379333115 -74.54291373896486) bank86876 +86877 POINT(41.57033657875938 -73.98661309577265) bank86877 +86878 POINT(40.319893065008 -73.14320469533283) bank86878 +86879 POINT(40.00961747877851 -73.5907248901647) bank86879 +86880 POINT(39.84768553808706 -73.13712276212601) bank86880 +86881 POINT(40.6549242843028 -73.51719126303497) bank86881 +86882 POINT(40.53116832295709 -73.37680634371767) bank86882 +86883 POINT(40.4722192300961 -73.54733878785022) bank86883 +86884 POINT(40.06469737760749 -73.98562072375437) bank86884 +86885 POINT(40.23167050783197 -73.89065250620047) bank86885 +86886 POINT(39.91593915520098 -73.21182285397498) bank86886 +86887 POINT(41.12678499736748 -74.04616545559469) bank86887 +86888 POINT(39.95420400462494 -74.95431559540783) bank86888 +86889 POINT(39.78546040225081 -73.97121154940146) bank86889 +86890 POINT(40.0831325124311 -73.88611374177218) bank86890 +86891 POINT(40.42699783619997 -74.9089365100607) bank86891 +86892 POINT(41.18015625188111 -73.54780433637647) bank86892 +86893 POINT(41.66314851928145 -74.84080646907971) bank86893 +86894 POINT(40.13067489570581 -73.75489924618928) bank86894 +86895 POINT(39.759724407614165 -74.50011545158863) bank86895 +86896 POINT(39.738547383545026 -74.72563392076978) bank86896 +86897 POINT(41.357724707131666 -74.49910470495404) bank86897 +86898 POINT(40.00997305041928 -74.3971475206066) bank86898 +86899 POINT(40.89973293512153 -74.24198879001673) bank86899 +86900 POINT(40.114925775708116 -73.50165288995383) bank86900 +86901 POINT(41.15035200573218 -74.2660544295711) bank86901 +86902 POINT(41.225812245164626 -73.30893385547445) bank86902 +86903 POINT(40.668996710163114 -73.48369012189514) bank86903 +86904 POINT(39.979388286270904 -74.64828060888883) bank86904 +86905 POINT(41.180232966610646 -73.7261450266558) bank86905 +86906 POINT(40.235994836429605 -73.65385669853544) bank86906 +86907 POINT(39.998929478847046 -73.8604417319473) bank86907 +86908 POINT(40.035303012625754 -73.14256262527338) bank86908 +86909 POINT(40.37193422498831 -73.2731351310965) bank86909 +86910 POINT(41.39387874268026 -73.22428629231099) bank86910 +86911 POINT(40.044760538679704 -74.88224741114838) bank86911 +86912 POINT(40.11074316582384 -73.52559472371948) bank86912 +86913 POINT(39.84493039283224 -73.2579308561334) bank86913 +86914 POINT(39.94032086147632 -73.421101567198) bank86914 +86915 POINT(39.76213220734906 -74.0603204239225) bank86915 +86916 POINT(40.43212009773352 -74.22917163942407) bank86916 +86917 POINT(41.47334170540702 -73.21154705742488) bank86917 +86918 POINT(41.64194046485363 -73.74053622857517) bank86918 +86919 POINT(40.28668021704997 -74.34365608197375) bank86919 +86920 POINT(40.692455720013776 -74.61103058950846) bank86920 +86921 POINT(41.677545221161544 -73.96643306545424) bank86921 +86922 POINT(39.79286239502167 -73.19089734437645) bank86922 +86923 POINT(41.190735159729826 -73.51022165983959) bank86923 +86924 POINT(40.550107836256075 -74.75569944298017) bank86924 +86925 POINT(39.8216741637013 -73.93162830059407) bank86925 +86926 POINT(40.66481293223254 -73.46631544837436) bank86926 +86927 POINT(41.31366976797769 -73.55852188846714) bank86927 +86928 POINT(40.23476125559302 -73.58498369329598) bank86928 +86929 POINT(40.09088625450252 -73.30708992151006) bank86929 +86930 POINT(40.745325982595844 -73.17792057415755) bank86930 +86931 POINT(41.0490762266212 -73.29553040783796) bank86931 +86932 POINT(41.631440837898786 -73.04037788651739) bank86932 +86933 POINT(40.48541990944877 -74.48750738724327) bank86933 +86934 POINT(41.14137303557215 -73.19094825815756) bank86934 +86935 POINT(39.716838499281195 -73.3142317877794) bank86935 +86936 POINT(41.414786062108284 -74.81765683035533) bank86936 +86937 POINT(41.45165759809111 -74.24958503036318) bank86937 +86938 POINT(40.00519939504876 -73.78983322527867) bank86938 +86939 POINT(41.482364855152625 -73.08224983909405) bank86939 +86940 POINT(40.42719463126079 -73.33234233337667) bank86940 +86941 POINT(41.344564001095435 -73.18796813842522) bank86941 +86942 POINT(39.79898212658056 -74.22798889521954) bank86942 +86943 POINT(40.72206219013683 -74.74616900434867) bank86943 +86944 POINT(40.72740716891332 -73.6618665892663) bank86944 +86945 POINT(40.56499861875529 -74.37593537045221) bank86945 +86946 POINT(40.78600492464224 -74.25253539075659) bank86946 +86947 POINT(39.792990816705654 -74.21335701126958) bank86947 +86948 POINT(41.463444988699 -74.7212509035229) bank86948 +86949 POINT(41.564365267394955 -73.76808364129023) bank86949 +86950 POINT(41.32952607976414 -74.66714193577991) bank86950 +86951 POINT(41.53135111121883 -73.75759751668892) bank86951 +86952 POINT(40.26597968891209 -73.39184670749603) bank86952 +86953 POINT(41.17019523642743 -73.87734280822045) bank86953 +86954 POINT(41.403977634324825 -73.68081109098995) bank86954 +86955 POINT(40.59628489169827 -73.02671809218856) bank86955 +86956 POINT(40.96343034986519 -73.60192887627635) bank86956 +86957 POINT(41.53364359951117 -74.52892508052423) bank86957 +86958 POINT(39.875428151923366 -73.39548245532123) bank86958 +86959 POINT(40.494012424693715 -74.89466086120869) bank86959 +86960 POINT(40.281195913358815 -74.02733940854202) bank86960 +86961 POINT(40.418138563563424 -73.94675765783533) bank86961 +86962 POINT(40.4501122150041 -74.13176573211933) bank86962 +86963 POINT(41.38607073980578 -73.20146442652494) bank86963 +86964 POINT(39.95500073211919 -73.23177342138708) bank86964 +86965 POINT(40.89290662920717 -73.17263704994198) bank86965 +86966 POINT(41.69082634595919 -74.56998258103809) bank86966 +86967 POINT(39.71639343246046 -74.24001746324318) bank86967 +86968 POINT(41.3806850576052 -74.88545980617134) bank86968 +86969 POINT(40.12347828381509 -73.43760441315094) bank86969 +86970 POINT(41.12130543958433 -73.64581684966794) bank86970 +86971 POINT(40.70345847615922 -74.55215729427427) bank86971 +86972 POINT(41.08559473836829 -73.58662760161758) bank86972 +86973 POINT(41.314623454181636 -74.06699885984374) bank86973 +86974 POINT(41.03521531847771 -74.40805412835223) bank86974 +86975 POINT(39.8713008515051 -74.61889397991433) bank86975 +86976 POINT(41.253655535526 -74.65238942404703) bank86976 +86977 POINT(40.45497335657902 -73.35760804226909) bank86977 +86978 POINT(41.64908997091001 -74.10838670848742) bank86978 +86979 POINT(39.745064479725606 -73.21525763200398) bank86979 +86980 POINT(41.12745549591328 -73.82811271485623) bank86980 +86981 POINT(40.49930346029224 -73.22659128030325) bank86981 +86982 POINT(41.56593448344696 -74.23409576363018) bank86982 +86983 POINT(40.941721444231455 -74.5966769032238) bank86983 +86984 POINT(40.23152611481132 -74.68323820997155) bank86984 +86985 POINT(39.858882143392556 -73.400972620579) bank86985 +86986 POINT(41.335483467778126 -74.49604459003375) bank86986 +86987 POINT(40.56246646565064 -73.06125290241604) bank86987 +86988 POINT(40.706105774521355 -74.88670515197295) bank86988 +86989 POINT(40.35610111359945 -74.42390506973223) bank86989 +86990 POINT(41.08154980175528 -73.11787418702693) bank86990 +86991 POINT(40.00321916538656 -73.05957868426493) bank86991 +86992 POINT(41.13956446671712 -74.58770766391162) bank86992 +86993 POINT(40.00165962882515 -73.36831393818466) bank86993 +86994 POINT(40.71475564435913 -73.61710416672011) bank86994 +86995 POINT(39.78470329137158 -74.81829946778373) bank86995 +86996 POINT(41.17982152400996 -74.06164876098107) bank86996 +86997 POINT(41.688179877999026 -73.04413783768749) bank86997 +86998 POINT(40.19577914099565 -74.1974432152969) bank86998 +86999 POINT(40.62247912832081 -74.97751080759161) bank86999 +87000 POINT(41.55670708983404 -73.18101874490685) bank87000 +87001 POINT(40.66684384681132 -74.92996942547016) bank87001 +87002 POINT(40.842072462253384 -74.62654384253513) bank87002 +87003 POINT(40.120366345157834 -73.05694288519449) bank87003 +87004 POINT(40.79845091485715 -74.98362160083275) bank87004 +87005 POINT(40.18435529814959 -74.0683451934064) bank87005 +87006 POINT(39.91314791124846 -74.84498401148016) bank87006 +87007 POINT(39.738374423766615 -73.274861141935) bank87007 +87008 POINT(41.283265935617784 -74.32132260879627) bank87008 +87009 POINT(41.48139513103902 -73.55746663777207) bank87009 +87010 POINT(39.86676264532592 -74.83560235086179) bank87010 +87011 POINT(39.99445851844878 -73.64824894890215) bank87011 +87012 POINT(41.05171368596238 -74.76613294789557) bank87012 +87013 POINT(41.35169290564818 -74.66144917883987) bank87013 +87014 POINT(39.959069444832664 -74.38582395762216) bank87014 +87015 POINT(39.926692602721225 -73.4684798088336) bank87015 +87016 POINT(40.03289003368356 -73.1476056107183) bank87016 +87017 POINT(40.10957980669694 -74.52154826093438) bank87017 +87018 POINT(41.16766334171287 -73.05781405440297) bank87018 +87019 POINT(39.99810526665399 -74.94974761779973) bank87019 +87020 POINT(40.78856152815662 -73.30793339471292) bank87020 +87021 POINT(41.36014448852537 -74.78048677055253) bank87021 +87022 POINT(40.867902899731845 -73.85776059583615) bank87022 +87023 POINT(40.894432105527244 -74.01776477101654) bank87023 +87024 POINT(39.766701425996324 -74.95035406974404) bank87024 +87025 POINT(40.58070904138177 -74.04672234253702) bank87025 +87026 POINT(40.21440891628044 -74.00157994196131) bank87026 +87027 POINT(41.17161217140155 -74.3751800213274) bank87027 +87028 POINT(40.60494521912988 -73.75140950822545) bank87028 +87029 POINT(39.97580191123295 -74.71054850291802) bank87029 +87030 POINT(41.16517696044884 -73.1191599154623) bank87030 +87031 POINT(40.96274035369888 -74.74475503673573) bank87031 +87032 POINT(41.560888206636704 -74.38499266805277) bank87032 +87033 POINT(41.154385199360775 -74.56810085257092) bank87033 +87034 POINT(40.30507745593232 -74.18699830996908) bank87034 +87035 POINT(40.00301172953448 -73.14602396475911) bank87035 +87036 POINT(40.01726174567394 -74.41153299110982) bank87036 +87037 POINT(40.81433797982295 -74.3897415845618) bank87037 +87038 POINT(40.3816218083537 -74.30445149615059) bank87038 +87039 POINT(40.157465557145045 -73.14049609815329) bank87039 +87040 POINT(39.766077634363 -74.04263796725297) bank87040 +87041 POINT(41.58239723280191 -73.69590971877548) bank87041 +87042 POINT(40.733766004283304 -73.82354711343402) bank87042 +87043 POINT(40.15091846722598 -74.15909182397685) bank87043 +87044 POINT(41.25252662171668 -73.13980066220437) bank87044 +87045 POINT(40.75545076687133 -74.86354555680194) bank87045 +87046 POINT(40.26117929354027 -74.18728470973853) bank87046 +87047 POINT(41.01721296767124 -73.26648286444109) bank87047 +87048 POINT(40.76767590711735 -74.76659345773433) bank87048 +87049 POINT(41.67351796003868 -73.69713859095903) bank87049 +87050 POINT(41.519538302765575 -74.71974078794354) bank87050 +87051 POINT(39.971202780571616 -74.18946271805713) bank87051 +87052 POINT(40.63531949907511 -73.0510589956902) bank87052 +87053 POINT(41.1743188923844 -73.29780031692603) bank87053 +87054 POINT(41.17521542510021 -74.05718151513582) bank87054 +87055 POINT(41.26236688536265 -73.5379407524051) bank87055 +87056 POINT(41.46129991574259 -73.80022995830832) bank87056 +87057 POINT(40.984444273441916 -73.85634387286807) bank87057 +87058 POINT(40.58148714437059 -74.6237664817399) bank87058 +87059 POINT(39.9207359549651 -74.77705310411747) bank87059 +87060 POINT(40.27491532912913 -74.599193467815) bank87060 +87061 POINT(40.03851748939453 -74.1438579600409) bank87061 +87062 POINT(41.55936783643026 -73.65248687151191) bank87062 +87063 POINT(39.927652804539214 -74.9273219551403) bank87063 +87064 POINT(40.672378407833214 -73.9951847597205) bank87064 +87065 POINT(41.17925563928126 -74.46615821875636) bank87065 +87066 POINT(41.014449316061196 -73.69171778648766) bank87066 +87067 POINT(40.786564514026324 -73.69909191461409) bank87067 +87068 POINT(40.071107240047894 -73.34615057977135) bank87068 +87069 POINT(40.81436141564653 -74.50984904465422) bank87069 +87070 POINT(41.04122492416112 -74.4261815526269) bank87070 +87071 POINT(40.439622393857086 -73.51682215555105) bank87071 +87072 POINT(41.42454486998186 -74.80722132125842) bank87072 +87073 POINT(40.443490482783936 -73.74427113213554) bank87073 +87074 POINT(39.94803439837922 -74.9270821449677) bank87074 +87075 POINT(41.5401173454866 -74.39465899992665) bank87075 +87076 POINT(41.348011923401344 -74.42086713584989) bank87076 +87077 POINT(39.87506909394468 -73.77141493612433) bank87077 +87078 POINT(41.4598580384447 -74.7282219467031) bank87078 +87079 POINT(41.10209175778121 -73.48943382013633) bank87079 +87080 POINT(40.52805929170336 -75.00348893940486) bank87080 +87081 POINT(40.333872722138345 -73.62118560572384) bank87081 +87082 POINT(40.34771385224031 -74.09114184007107) bank87082 +87083 POINT(40.4992909154774 -74.58801986721465) bank87083 +87084 POINT(41.24053555048062 -74.94182922391583) bank87084 +87085 POINT(39.897341816525625 -73.08161766090272) bank87085 +87086 POINT(40.56052569982317 -73.07965214194273) bank87086 +87087 POINT(40.57101743660708 -74.02308322018186) bank87087 +87088 POINT(40.32816235587207 -73.49978335656661) bank87088 +87089 POINT(41.50505372753454 -74.36120570438227) bank87089 +87090 POINT(40.54857084712815 -73.53591041841618) bank87090 +87091 POINT(41.1682540800573 -74.33321574713592) bank87091 +87092 POINT(39.9321960877016 -73.23457800419766) bank87092 +87093 POINT(41.11373292746283 -74.06715255931279) bank87093 +87094 POINT(39.94250890682114 -73.79453220164774) bank87094 +87095 POINT(41.68236999954019 -74.79658531825682) bank87095 +87096 POINT(40.86860086620859 -74.14314882239997) bank87096 +87097 POINT(39.76836032771348 -73.08650117333848) bank87097 +87098 POINT(40.52121779495766 -73.16067807432333) bank87098 +87099 POINT(40.672148303875595 -74.60098365077084) bank87099 +87100 POINT(41.68015631448231 -73.45791591686121) bank87100 +87101 POINT(40.50791227790346 -73.92380162468865) bank87101 +87102 POINT(40.04572641586316 -73.8377119719111) bank87102 +87103 POINT(40.99915198404004 -73.5892724271744) bank87103 +87104 POINT(39.771174555420316 -73.98562033442487) bank87104 +87105 POINT(41.12495720255856 -73.07880097948127) bank87105 +87106 POINT(41.24390249881421 -74.48222133166284) bank87106 +87107 POINT(40.27000532898453 -74.4860880166731) bank87107 +87108 POINT(39.72713181503994 -73.58741001732878) bank87108 +87109 POINT(41.42165015923917 -73.96838589204692) bank87109 +87110 POINT(40.31536021236127 -74.34624786550474) bank87110 +87111 POINT(40.72658204962002 -74.54469164621486) bank87111 +87112 POINT(40.01351062929098 -73.71935134537341) bank87112 +87113 POINT(39.79578143295896 -73.02045037833909) bank87113 +87114 POINT(41.625709800582136 -74.83649185432961) bank87114 +87115 POINT(40.28416223527741 -73.83398364676911) bank87115 +87116 POINT(41.65435158791408 -73.65882965415216) bank87116 +87117 POINT(39.955219731791644 -73.51143197691765) bank87117 +87118 POINT(40.77231840470116 -73.13098198434373) bank87118 +87119 POINT(41.67538738613951 -74.90726003972438) bank87119 +87120 POINT(40.21546551708106 -74.82189765784759) bank87120 +87121 POINT(40.142997290982976 -74.81109860899953) bank87121 +87122 POINT(40.43058208672685 -73.57768613751514) bank87122 +87123 POINT(40.87806640387977 -74.07179366728454) bank87123 +87124 POINT(41.53640953598558 -74.40619455096027) bank87124 +87125 POINT(39.713363757332 -73.70940343616891) bank87125 +87126 POINT(41.57758166104594 -74.13130915614599) bank87126 +87127 POINT(40.14233698942629 -74.81356218101442) bank87127 +87128 POINT(40.55316197070147 -73.02394267475448) bank87128 +87129 POINT(41.01752037718502 -74.27711673410597) bank87129 +87130 POINT(39.93697919623697 -74.10320939805307) bank87130 +87131 POINT(40.61180902702669 -73.48509721161473) bank87131 +87132 POINT(41.448544829702676 -73.60876170392336) bank87132 +87133 POINT(40.59191154952086 -73.96696470102592) bank87133 +87134 POINT(40.87961328785139 -73.5634106132507) bank87134 +87135 POINT(41.2233951456613 -73.61481497872302) bank87135 +87136 POINT(41.05692454864259 -73.31721465464382) bank87136 +87137 POINT(40.18267948924257 -73.66944016819717) bank87137 +87138 POINT(40.963656283120734 -73.49341513933304) bank87138 +87139 POINT(39.88632683755108 -73.50610804805166) bank87139 +87140 POINT(40.232128193197134 -74.15491617830067) bank87140 +87141 POINT(40.96575012549114 -73.59352301781983) bank87141 +87142 POINT(40.85967984774282 -74.4012900038778) bank87142 +87143 POINT(39.86519332486302 -73.23013872137491) bank87143 +87144 POINT(40.61648321731347 -73.82141835574379) bank87144 +87145 POINT(40.01529686861829 -74.64918852502419) bank87145 +87146 POINT(40.77498898291175 -74.01366145223952) bank87146 +87147 POINT(41.12074148320794 -74.87984678309701) bank87147 +87148 POINT(40.550434432041534 -73.99615730132419) bank87148 +87149 POINT(41.22991225057735 -73.64754332060281) bank87149 +87150 POINT(41.29003290790403 -73.48972876953724) bank87150 +87151 POINT(40.700979917169164 -74.64627267311062) bank87151 +87152 POINT(41.29192777017067 -74.76838181146483) bank87152 +87153 POINT(40.21074926824076 -73.27618463432589) bank87153 +87154 POINT(41.23937893699605 -74.61111924374852) bank87154 +87155 POINT(39.78583059648932 -73.47245236874141) bank87155 +87156 POINT(40.80188037740395 -74.95368385490902) bank87156 +87157 POINT(40.75805685629683 -73.31001520973554) bank87157 +87158 POINT(41.46232198986886 -74.72240828529245) bank87158 +87159 POINT(41.275788131339866 -73.30088636806622) bank87159 +87160 POINT(41.41687847215699 -73.25267595096518) bank87160 +87161 POINT(40.16032815459963 -74.27849904642737) bank87161 +87162 POINT(40.50046177085002 -73.3500927149914) bank87162 +87163 POINT(40.736387408982594 -74.1950588366442) bank87163 +87164 POINT(40.09335765501286 -73.43912175616536) bank87164 +87165 POINT(39.946862294385184 -73.58778490364867) bank87165 +87166 POINT(41.574956047196686 -73.17412338323479) bank87166 +87167 POINT(40.72563590780633 -73.59398868688189) bank87167 +87168 POINT(40.758164716027665 -74.43026160588455) bank87168 +87169 POINT(40.18544687561846 -73.8365037568712) bank87169 +87170 POINT(39.835425893397364 -74.4961213270252) bank87170 +87171 POINT(40.116984592006574 -73.92526220459109) bank87171 +87172 POINT(40.38374333084465 -73.1043086846391) bank87172 +87173 POINT(41.30623679178311 -74.40073445046629) bank87173 +87174 POINT(40.33198518894339 -74.75144920143892) bank87174 +87175 POINT(40.12029555805273 -73.00837834850945) bank87175 +87176 POINT(41.076374272197924 -73.89811831926558) bank87176 +87177 POINT(40.14834497375231 -74.46305101700055) bank87177 +87178 POINT(39.80558355665957 -73.33713728450506) bank87178 +87179 POINT(40.20915309798168 -73.1261187671928) bank87179 +87180 POINT(41.50986877204023 -73.67774748327983) bank87180 +87181 POINT(41.49185407856442 -74.37731778140022) bank87181 +87182 POINT(41.12927496116943 -74.42547151591819) bank87182 +87183 POINT(41.34725675778467 -74.08988835370577) bank87183 +87184 POINT(41.29408792265046 -73.97926841272881) bank87184 +87185 POINT(40.854370469215795 -74.1050833889454) bank87185 +87186 POINT(40.038698151470584 -73.49940372577375) bank87186 +87187 POINT(40.191063625182046 -74.19417376968684) bank87187 +87188 POINT(41.58706902932664 -74.67691282818784) bank87188 +87189 POINT(41.23884692630213 -74.03163777206927) bank87189 +87190 POINT(41.254864620621355 -74.0563263199702) bank87190 +87191 POINT(41.641710080044916 -74.03968906749755) bank87191 +87192 POINT(40.590298958389035 -73.31461080700207) bank87192 +87193 POINT(41.04474255723321 -74.76959899195245) bank87193 +87194 POINT(41.63932786719422 -73.97385625978126) bank87194 +87195 POINT(41.54721373459988 -74.62164821352341) bank87195 +87196 POINT(40.241948774156484 -73.74676806724581) bank87196 +87197 POINT(40.595693114475544 -74.96284883691331) bank87197 +87198 POINT(41.607244243475975 -74.41053995944651) bank87198 +87199 POINT(40.56101931666781 -74.68369440089145) bank87199 +87200 POINT(41.02568850941132 -74.88724998524985) bank87200 +87201 POINT(40.42544076782462 -73.84246813066143) bank87201 +87202 POINT(40.491222619336746 -74.5047479649338) bank87202 +87203 POINT(40.926502862875815 -73.65402703293165) bank87203 +87204 POINT(40.61436073052331 -73.72246868804173) bank87204 +87205 POINT(41.70248896486691 -74.10414351852691) bank87205 +87206 POINT(41.28267736890201 -74.26452961768332) bank87206 +87207 POINT(40.332187339599976 -74.76068518929448) bank87207 +87208 POINT(40.78909122976026 -74.8632199736639) bank87208 +87209 POINT(41.49493726277702 -73.6577495791797) bank87209 +87210 POINT(40.365306749095204 -74.18139886941361) bank87210 +87211 POINT(41.491256136100766 -74.58297047830811) bank87211 +87212 POINT(41.38608093625429 -73.26423316280936) bank87212 +87213 POINT(41.12638188275161 -74.9370241677428) bank87213 +87214 POINT(40.75427435673321 -73.54690899465128) bank87214 +87215 POINT(40.728239862462395 -73.11656910460604) bank87215 +87216 POINT(39.72704887401877 -73.88558377692661) bank87216 +87217 POINT(39.861424972145784 -74.0085780225213) bank87217 +87218 POINT(41.6979181422543 -73.58645216057063) bank87218 +87219 POINT(41.5921793942349 -74.00608742111865) bank87219 +87220 POINT(40.03037823672271 -73.84688779100985) bank87220 +87221 POINT(41.539933172931875 -73.88653861151347) bank87221 +87222 POINT(39.95184397299752 -73.3412004741723) bank87222 +87223 POINT(39.91147559876948 -74.633164340642) bank87223 +87224 POINT(41.310183368405276 -74.11419161727247) bank87224 +87225 POINT(41.591292318021665 -73.52833543466168) bank87225 +87226 POINT(40.36679599146075 -73.79452340657114) bank87226 +87227 POINT(41.07955688060723 -74.2617694385959) bank87227 +87228 POINT(41.21153537473439 -74.54824592077813) bank87228 +87229 POINT(40.68189328278124 -73.08597666127433) bank87229 +87230 POINT(40.86760436193314 -74.07774731110638) bank87230 +87231 POINT(40.432590767793315 -74.5275778271224) bank87231 +87232 POINT(40.93859956702935 -74.9894113513189) bank87232 +87233 POINT(41.28829456692227 -74.43567023993447) bank87233 +87234 POINT(41.63098283341518 -73.37290861256335) bank87234 +87235 POINT(41.54229982007344 -74.94287807650683) bank87235 +87236 POINT(39.91785459950822 -73.28663006416025) bank87236 +87237 POINT(40.33392729669405 -74.3543792592212) bank87237 +87238 POINT(41.377734512948756 -74.9072848634681) bank87238 +87239 POINT(41.103601594000324 -74.25716909098514) bank87239 +87240 POINT(41.398334375451626 -74.77266647423471) bank87240 +87241 POINT(41.16317981892257 -73.6799151144142) bank87241 +87242 POINT(40.839752561215526 -73.3460849003695) bank87242 +87243 POINT(40.1662449238186 -74.97090073935964) bank87243 +87244 POINT(40.30787968354527 -74.28175151113808) bank87244 +87245 POINT(41.30165256559898 -74.50079449959702) bank87245 +87246 POINT(40.62013725638777 -74.42216291255419) bank87246 +87247 POINT(39.783571767248944 -73.16795507958359) bank87247 +87248 POINT(41.43984871937619 -74.29771172756872) bank87248 +87249 POINT(40.15709500467645 -73.49986362909434) bank87249 +87250 POINT(40.68653152764182 -74.54702245489585) bank87250 +87251 POINT(40.081017595065276 -73.48004303405949) bank87251 +87252 POINT(41.59532062269383 -73.79863554543796) bank87252 +87253 POINT(40.21744180248141 -73.43815232201814) bank87253 +87254 POINT(40.249871033142774 -74.4483230620048) bank87254 +87255 POINT(41.34456271234208 -74.49985446115511) bank87255 +87256 POINT(39.942034663997156 -73.83734448108044) bank87256 +87257 POINT(40.18505832480484 -73.96965553415018) bank87257 +87258 POINT(39.99595248247049 -74.50663201059712) bank87258 +87259 POINT(39.953763197451025 -73.87669516572778) bank87259 +87260 POINT(40.57888356943183 -74.34286715432107) bank87260 +87261 POINT(39.82881386758863 -73.54520698559537) bank87261 +87262 POINT(41.2874072489073 -73.71397284118889) bank87262 +87263 POINT(40.09735889287688 -74.0871756601523) bank87263 +87264 POINT(40.917178357141054 -73.32442875870345) bank87264 +87265 POINT(40.761385110798585 -73.47688723579206) bank87265 +87266 POINT(40.88771528975626 -74.26844269674868) bank87266 +87267 POINT(41.18534271808818 -73.25875134166431) bank87267 +87268 POINT(39.845127058096345 -73.5153540122831) bank87268 +87269 POINT(41.036336386004635 -73.31892397105065) bank87269 +87270 POINT(41.11008461360597 -73.35583530958507) bank87270 +87271 POINT(39.79836040778871 -73.31601983215647) bank87271 +87272 POINT(40.06276042198623 -74.69355422958782) bank87272 +87273 POINT(39.733996597978326 -74.1584807191876) bank87273 +87274 POINT(39.72923974800263 -74.51041083417115) bank87274 +87275 POINT(40.99571291047436 -74.61441967671688) bank87275 +87276 POINT(41.2491153363507 -74.82818331020309) bank87276 +87277 POINT(40.97340933465869 -73.10441073994257) bank87277 +87278 POINT(41.087791856407186 -73.29038480279316) bank87278 +87279 POINT(40.45232641749937 -74.78375540997764) bank87279 +87280 POINT(41.64553294384895 -73.53883552145957) bank87280 +87281 POINT(40.76435977587157 -74.95430738732678) bank87281 +87282 POINT(40.28032382781808 -74.9278110366388) bank87282 +87283 POINT(41.36060241147017 -74.8275229474495) bank87283 +87284 POINT(41.35199812098703 -73.36564879581118) bank87284 +87285 POINT(41.287134195573174 -74.74374688884961) bank87285 +87286 POINT(39.910733678723304 -73.25631812435938) bank87286 +87287 POINT(41.6432049724806 -74.6079789943218) bank87287 +87288 POINT(40.507893322993695 -74.26328968262948) bank87288 +87289 POINT(40.0495239459917 -73.93904247168317) bank87289 +87290 POINT(40.125985696065534 -74.23304568672306) bank87290 +87291 POINT(41.06596886361863 -73.6753915355124) bank87291 +87292 POINT(39.9043637611888 -73.57290769236114) bank87292 +87293 POINT(41.3672277077595 -73.93449334046709) bank87293 +87294 POINT(40.55806517594029 -73.73088129463268) bank87294 +87295 POINT(40.43146041563307 -73.58881902834021) bank87295 +87296 POINT(40.29655681758208 -74.95539010948644) bank87296 +87297 POINT(40.09788620405746 -74.16407271438737) bank87297 +87298 POINT(41.46701675839415 -74.73157230664489) bank87298 +87299 POINT(41.46386185280006 -73.35573760033098) bank87299 +87300 POINT(41.303960509095944 -74.65366616262827) bank87300 +87301 POINT(39.80738802708602 -74.77265099111705) bank87301 +87302 POINT(40.778601095327744 -73.9425538058237) bank87302 +87303 POINT(40.04034981010738 -74.05344174747957) bank87303 +87304 POINT(39.95559058113453 -73.38572825285989) bank87304 +87305 POINT(40.52999745007795 -73.43049917564619) bank87305 +87306 POINT(40.71664431225042 -74.9791837263017) bank87306 +87307 POINT(40.14017588762905 -74.85028066404692) bank87307 +87308 POINT(40.568928007443354 -74.1359607812107) bank87308 +87309 POINT(40.78567910067101 -73.58796962305229) bank87309 +87310 POINT(40.73187336544036 -74.4131921783456) bank87310 +87311 POINT(40.673616540476246 -73.21984442992297) bank87311 +87312 POINT(41.04824307828419 -75.00338490885318) bank87312 +87313 POINT(40.803741946281015 -73.195683111112) bank87313 +87314 POINT(40.231185464329734 -73.22683392412027) bank87314 +87315 POINT(40.632585073945044 -74.72178362338587) bank87315 +87316 POINT(41.13216193607839 -73.33153217732885) bank87316 +87317 POINT(39.712848258274455 -74.76915274159113) bank87317 +87318 POINT(40.243185298948276 -73.4437377972465) bank87318 +87319 POINT(40.4446997505887 -73.58575383191523) bank87319 +87320 POINT(41.32124587638172 -74.94419371751405) bank87320 +87321 POINT(40.926488574792344 -73.44309024874451) bank87321 +87322 POINT(40.67174550943544 -74.67139991983055) bank87322 +87323 POINT(40.79288045454588 -73.1816122024939) bank87323 +87324 POINT(41.48407986674752 -74.11824353142418) bank87324 +87325 POINT(40.61081161138907 -73.17442746657748) bank87325 +87326 POINT(41.579813233694445 -74.65605398325452) bank87326 +87327 POINT(40.852540536772494 -73.80478592609123) bank87327 +87328 POINT(40.17654507659219 -73.91165884729894) bank87328 +87329 POINT(40.84803700874242 -74.34061535075293) bank87329 +87330 POINT(40.672560474396725 -74.59500417193179) bank87330 +87331 POINT(41.625427180437974 -73.80488470411453) bank87331 +87332 POINT(41.69258864809145 -73.94520598937406) bank87332 +87333 POINT(39.76945133348557 -74.17163458921706) bank87333 +87334 POINT(39.80893264070865 -73.34059947802936) bank87334 +87335 POINT(40.24431678486498 -73.4509807913297) bank87335 +87336 POINT(40.314731521086756 -74.29033950221613) bank87336 +87337 POINT(40.34551812579244 -74.74002326876246) bank87337 +87338 POINT(40.59491477241358 -74.34853372104759) bank87338 +87339 POINT(39.88549963673165 -73.23310538066671) bank87339 +87340 POINT(39.98328537634472 -73.38013211940564) bank87340 +87341 POINT(40.87991688626301 -73.71732586252428) bank87341 +87342 POINT(40.20714726166075 -73.21043593159673) bank87342 +87343 POINT(41.05205175134572 -74.88323261692167) bank87343 +87344 POINT(40.01134247171445 -73.48327666667453) bank87344 +87345 POINT(41.3930001256152 -74.7434015690375) bank87345 +87346 POINT(40.48075364697582 -74.78852827089467) bank87346 +87347 POINT(40.212942212270335 -73.64043935282446) bank87347 +87348 POINT(39.9798146539776 -73.45247973065959) bank87348 +87349 POINT(40.08793813613139 -73.24303803979642) bank87349 +87350 POINT(41.65315688286485 -74.47756822157056) bank87350 +87351 POINT(40.626839233997764 -73.66008576474104) bank87351 +87352 POINT(40.02986879267718 -74.43845619210872) bank87352 +87353 POINT(41.2251553841443 -74.6050554274544) bank87353 +87354 POINT(41.413943137274416 -74.59583970060758) bank87354 +87355 POINT(40.41319629690394 -73.15731873881866) bank87355 +87356 POINT(40.5770283187666 -74.30446011006747) bank87356 +87357 POINT(39.971139221729004 -74.62931239971081) bank87357 +87358 POINT(41.64528482596508 -74.64125442102898) bank87358 +87359 POINT(40.086528997778096 -74.86230930607924) bank87359 +87360 POINT(41.138014169748466 -74.58873330753663) bank87360 +87361 POINT(40.832542234131736 -73.25110534620384) bank87361 +87362 POINT(41.54058969870176 -74.09397052110866) bank87362 +87363 POINT(40.21626892324442 -74.00367155362495) bank87363 +87364 POINT(41.48874053264572 -74.50728696591551) bank87364 +87365 POINT(40.3532700577581 -73.18600393349668) bank87365 +87366 POINT(41.66654454926653 -74.14824178223192) bank87366 +87367 POINT(40.83775189768869 -74.3141726929709) bank87367 +87368 POINT(40.31203590394773 -73.06186074403203) bank87368 +87369 POINT(39.876031629770914 -74.76827330085858) bank87369 +87370 POINT(40.33192157952473 -73.21588634709138) bank87370 +87371 POINT(39.774377566024725 -73.03414730172982) bank87371 +87372 POINT(39.89065167855789 -74.02236658974707) bank87372 +87373 POINT(40.08021924905577 -74.42966533226902) bank87373 +87374 POINT(41.390109801608176 -74.1898348160895) bank87374 +87375 POINT(40.18234934725678 -73.36785744773309) bank87375 +87376 POINT(41.425560892898055 -73.35233126557185) bank87376 +87377 POINT(40.86767829931621 -73.76875426128383) bank87377 +87378 POINT(40.49545575470258 -74.18521604077907) bank87378 +87379 POINT(41.61222207558238 -73.71900128388299) bank87379 +87380 POINT(40.43095906169396 -73.65970311101198) bank87380 +87381 POINT(41.09171186529975 -74.21961136500153) bank87381 +87382 POINT(40.95529045164517 -73.79972189595856) bank87382 +87383 POINT(40.440834522758394 -73.78722048998202) bank87383 +87384 POINT(39.88271438614327 -74.52398577390092) bank87384 +87385 POINT(40.64730921493834 -73.99372797674116) bank87385 +87386 POINT(40.320665450069946 -73.95612555859356) bank87386 +87387 POINT(40.045748612275794 -74.9041680791769) bank87387 +87388 POINT(40.39116176185586 -74.12620517485068) bank87388 +87389 POINT(41.48307978662531 -73.97940904758534) bank87389 +87390 POINT(40.02946238421181 -73.12405253120254) bank87390 +87391 POINT(39.78989522449059 -74.58265980225984) bank87391 +87392 POINT(41.09411112409441 -74.69983649563073) bank87392 +87393 POINT(41.23782611341948 -74.91358431361058) bank87393 +87394 POINT(41.59677526578418 -73.85256635976091) bank87394 +87395 POINT(39.96985984985379 -73.72603043569455) bank87395 +87396 POINT(40.054776800232304 -73.91821060587095) bank87396 +87397 POINT(40.1359668148491 -73.23728542885672) bank87397 +87398 POINT(41.2313605559255 -73.798035614703) bank87398 +87399 POINT(41.637647623645755 -74.57333272343328) bank87399 +87400 POINT(39.80653561504287 -74.74798779365932) bank87400 +87401 POINT(40.23994528505429 -74.56163565832091) bank87401 +87402 POINT(41.24664162214644 -73.02825703017191) bank87402 +87403 POINT(40.69599507185222 -74.03183239642017) bank87403 +87404 POINT(40.68212314399307 -74.98460138787266) bank87404 +87405 POINT(41.417066424410855 -73.26257294116711) bank87405 +87406 POINT(39.75906097030184 -74.05346090083775) bank87406 +87407 POINT(40.23310368492112 -73.7697633385356) bank87407 +87408 POINT(40.93118071913363 -74.10751063983732) bank87408 +87409 POINT(40.709079922969856 -74.98288973687812) bank87409 +87410 POINT(40.19814693962661 -74.06044397244113) bank87410 +87411 POINT(41.07983541824305 -73.22402726170867) bank87411 +87412 POINT(40.31393788578115 -74.21826114184712) bank87412 +87413 POINT(40.12737774865386 -73.38915848064545) bank87413 +87414 POINT(39.89295413877735 -73.33130818548814) bank87414 +87415 POINT(40.34639777850973 -73.51058294301497) bank87415 +87416 POINT(40.10560848776792 -73.32302791946437) bank87416 +87417 POINT(40.94396373733664 -74.6720528779294) bank87417 +87418 POINT(39.85331009552347 -73.67309962801433) bank87418 +87419 POINT(41.6563289133806 -74.68958983367881) bank87419 +87420 POINT(40.238798701524125 -74.8952395747837) bank87420 +87421 POINT(40.10768833211261 -73.37892374364101) bank87421 +87422 POINT(41.54863643420587 -74.29439612718306) bank87422 +87423 POINT(41.71009973634327 -73.63526285613663) bank87423 +87424 POINT(41.4310913148813 -74.64479913119118) bank87424 +87425 POINT(40.61220862188847 -74.02869355851945) bank87425 +87426 POINT(39.88825061232644 -74.46989575571932) bank87426 +87427 POINT(41.35418829935282 -73.90489140387602) bank87427 +87428 POINT(41.1291784879441 -73.0417738581401) bank87428 +87429 POINT(41.54608364953681 -73.68211159232177) bank87429 +87430 POINT(40.587982241639075 -74.67088358341483) bank87430 +87431 POINT(40.17560665248473 -74.46943320668319) bank87431 +87432 POINT(40.94567421651124 -74.50917169155073) bank87432 +87433 POINT(40.13873272571981 -73.72975467213824) bank87433 +87434 POINT(39.790266232726026 -74.52110612227291) bank87434 +87435 POINT(41.117133137263025 -74.63750990986523) bank87435 +87436 POINT(41.26105670928696 -74.81335298294138) bank87436 +87437 POINT(40.25722177436917 -74.93191866044374) bank87437 +87438 POINT(40.62358759885241 -73.1140558221284) bank87438 +87439 POINT(40.26572374797338 -73.64545360713991) bank87439 +87440 POINT(39.85553193827143 -74.03036331776948) bank87440 +87441 POINT(40.32912667555719 -73.47166128741912) bank87441 +87442 POINT(41.66129187303021 -73.10775025582566) bank87442 +87443 POINT(40.5179112512161 -74.8287740022672) bank87443 +87444 POINT(40.70272055506311 -74.10262130720477) bank87444 +87445 POINT(40.07669541326635 -74.0777147138568) bank87445 +87446 POINT(39.72145493204183 -73.92151354808875) bank87446 +87447 POINT(40.054326759108754 -74.34747538064158) bank87447 +87448 POINT(40.26085622643722 -73.89945545330716) bank87448 +87449 POINT(39.931601107501265 -73.64867883674833) bank87449 +87450 POINT(41.10841878769521 -73.31141830126988) bank87450 +87451 POINT(41.10526262571564 -74.37859895058166) bank87451 +87452 POINT(41.610075808379285 -73.32424315999917) bank87452 +87453 POINT(40.47030199114597 -74.8578100506022) bank87453 +87454 POINT(40.09920027518631 -73.75625942588039) bank87454 +87455 POINT(39.771811056759425 -74.8408849572081) bank87455 +87456 POINT(40.19177246966806 -74.14174368790383) bank87456 +87457 POINT(41.53442676174773 -73.73770681519356) bank87457 +87458 POINT(39.9696054555979 -74.78664833041472) bank87458 +87459 POINT(41.61176413698345 -73.6669917578591) bank87459 +87460 POINT(41.34675816971608 -73.90659882832287) bank87460 +87461 POINT(40.84396696851188 -74.4252053517107) bank87461 +87462 POINT(41.296252448700315 -73.39127365632935) bank87462 +87463 POINT(39.93085484527809 -74.33872227915793) bank87463 +87464 POINT(40.86649664679917 -73.87798082416849) bank87464 +87465 POINT(41.6585478350648 -74.32005479757574) bank87465 +87466 POINT(41.197814026284746 -73.95265156791949) bank87466 +87467 POINT(40.26033831712036 -74.09571063181413) bank87467 +87468 POINT(41.48627938915056 -74.46207758257168) bank87468 +87469 POINT(39.72432748143406 -74.79628142288051) bank87469 +87470 POINT(40.45074352226092 -74.63255899246401) bank87470 +87471 POINT(41.20994406206045 -73.34408264179329) bank87471 +87472 POINT(41.54281693114354 -73.69883195458284) bank87472 +87473 POINT(41.02152884591154 -73.06650928007402) bank87473 +87474 POINT(40.63778982351365 -74.92026376935348) bank87474 +87475 POINT(41.34246357211516 -73.29895235629307) bank87475 +87476 POINT(41.51529187741377 -74.66587144093957) bank87476 +87477 POINT(40.99751581100519 -74.01577646300531) bank87477 +87478 POINT(41.430718495353105 -74.76537283508866) bank87478 +87479 POINT(39.733197187221975 -73.87100033065363) bank87479 +87480 POINT(40.24475615463347 -73.99372929313616) bank87480 +87481 POINT(41.26056708498223 -74.15687822037359) bank87481 +87482 POINT(39.792704409291865 -73.28305492989038) bank87482 +87483 POINT(39.865112702986565 -73.01413867621386) bank87483 +87484 POINT(40.44924940079679 -73.7876768249223) bank87484 +87485 POINT(39.7469499328504 -73.18992228300749) bank87485 +87486 POINT(41.59288159472604 -74.94818737789346) bank87486 +87487 POINT(41.292479861618524 -73.52111499096257) bank87487 +87488 POINT(39.777180075531284 -74.11953206090352) bank87488 +87489 POINT(40.7249668443676 -73.51917928910704) bank87489 +87490 POINT(41.133864254065045 -73.41038763449559) bank87490 +87491 POINT(41.27556603674521 -73.58738585408364) bank87491 +87492 POINT(40.77778025930855 -74.78893700626087) bank87492 +87493 POINT(41.691625610848504 -74.4685328259229) bank87493 +87494 POINT(40.11256942109325 -74.27887406446271) bank87494 +87495 POINT(41.625742160334006 -74.37136474416765) bank87495 +87496 POINT(40.410134477720604 -73.07387486645757) bank87496 +87497 POINT(41.23564718286243 -73.12096037634119) bank87497 +87498 POINT(39.80244320068851 -74.29331503470571) bank87498 +87499 POINT(41.36358061134804 -73.36431509103116) bank87499 +87500 POINT(40.48126450740783 -74.03154531829094) bank87500 +87501 POINT(41.56305916382638 -73.45158207020943) bank87501 +87502 POINT(39.8457792226803 -73.24029348601778) bank87502 +87503 POINT(41.50550490445911 -74.94824310260834) bank87503 +87504 POINT(39.97649480983839 -74.53523563453263) bank87504 +87505 POINT(40.09550860067751 -73.35003909637534) bank87505 +87506 POINT(40.676922675061164 -73.11688348081655) bank87506 +87507 POINT(40.74658214984287 -74.55782028662641) bank87507 +87508 POINT(40.495665426758926 -73.3326047792) bank87508 +87509 POINT(41.55897369487546 -74.12013699996312) bank87509 +87510 POINT(40.432130994339985 -74.74758106635845) bank87510 +87511 POINT(40.87879888032168 -73.68169582632639) bank87511 +87512 POINT(41.103731877834385 -74.00628609113058) bank87512 +87513 POINT(40.51008029968039 -73.13359591933174) bank87513 +87514 POINT(41.01601903634452 -73.37246650266714) bank87514 +87515 POINT(41.29070490513058 -74.74891898286023) bank87515 +87516 POINT(40.57199801491273 -74.0322416428759) bank87516 +87517 POINT(41.26407588675157 -74.98518143349035) bank87517 +87518 POINT(40.2593247327456 -73.79861863380295) bank87518 +87519 POINT(41.32452598532461 -74.51598604351251) bank87519 +87520 POINT(41.57426080417297 -73.55199997454028) bank87520 +87521 POINT(41.147137500064815 -73.98811067306782) bank87521 +87522 POINT(39.82804538003898 -73.16561654790759) bank87522 +87523 POINT(40.5949710683237 -74.92399832790554) bank87523 +87524 POINT(40.19254883270633 -74.69314474833831) bank87524 +87525 POINT(41.35423317529978 -74.12980785501757) bank87525 +87526 POINT(41.24406623658358 -73.43247132831887) bank87526 +87527 POINT(40.787342444754614 -73.06381348440672) bank87527 +87528 POINT(41.127608172711966 -73.09812483458683) bank87528 +87529 POINT(40.92561189086306 -73.12515077278144) bank87529 +87530 POINT(40.355299102149424 -74.3383638822937) bank87530 +87531 POINT(40.95936348820517 -73.50392890736956) bank87531 +87532 POINT(40.19197897823891 -74.30210498208992) bank87532 +87533 POINT(41.236159436497104 -74.76058916978226) bank87533 +87534 POINT(41.69954909034844 -73.0432968650139) bank87534 +87535 POINT(40.37563297187484 -74.1299759183165) bank87535 +87536 POINT(40.613683719185985 -73.11782282464469) bank87536 +87537 POINT(40.160126161807476 -74.59865655346917) bank87537 +87538 POINT(40.38680078124349 -73.35363975071954) bank87538 +87539 POINT(41.1269661239892 -74.66042629946486) bank87539 +87540 POINT(40.90582535112872 -74.2943409461444) bank87540 +87541 POINT(41.31637515189132 -73.73132274609196) bank87541 +87542 POINT(40.862425547998974 -74.49834059289712) bank87542 +87543 POINT(41.46651843578471 -74.01413465730239) bank87543 +87544 POINT(40.39611022943259 -73.65869762631611) bank87544 +87545 POINT(40.554247975858544 -73.6962432203711) bank87545 +87546 POINT(40.50423808563283 -74.65528476066187) bank87546 +87547 POINT(40.892314821777404 -74.44341743751737) bank87547 +87548 POINT(40.47554440471568 -73.72823260559377) bank87548 +87549 POINT(40.056571712414026 -73.80725792329058) bank87549 +87550 POINT(39.98896336224295 -74.54365315693317) bank87550 +87551 POINT(40.99602564159529 -74.83739542337905) bank87551 +87552 POINT(41.45001657181525 -74.259855005747) bank87552 +87553 POINT(41.367139408301625 -73.09835024785954) bank87553 +87554 POINT(41.0115469495959 -74.36787616350693) bank87554 +87555 POINT(41.15177874133926 -73.61754035918678) bank87555 +87556 POINT(40.576361006394215 -74.11208472230791) bank87556 +87557 POINT(40.6291961797834 -74.34529097145943) bank87557 +87558 POINT(40.67728948810663 -74.76404089347577) bank87558 +87559 POINT(40.27582144350073 -73.25212124028617) bank87559 +87560 POINT(41.28792402451376 -74.27435255062872) bank87560 +87561 POINT(40.750151248873195 -74.03679407579764) bank87561 +87562 POINT(41.25288107246122 -73.09581565642615) bank87562 +87563 POINT(41.26664891613001 -74.07678321774911) bank87563 +87564 POINT(41.27039081327453 -74.08735136836907) bank87564 +87565 POINT(41.097031699893854 -73.12626900474538) bank87565 +87566 POINT(40.38773707350424 -73.56823732771448) bank87566 +87567 POINT(40.858217044497984 -73.30186730174974) bank87567 +87568 POINT(40.274702924356866 -74.0117583801454) bank87568 +87569 POINT(41.34956415063191 -74.9054380453267) bank87569 +87570 POINT(40.96429771004903 -74.78809644038189) bank87570 +87571 POINT(40.62154823671705 -74.3888230670617) bank87571 +87572 POINT(40.75775897892251 -73.12240387529489) bank87572 +87573 POINT(40.30348252172166 -74.84369815194526) bank87573 +87574 POINT(41.556785230018235 -73.77338595443736) bank87574 +87575 POINT(40.92879180450714 -73.80473439897686) bank87575 +87576 POINT(41.643997473267675 -74.36648949469956) bank87576 +87577 POINT(41.56627848502389 -74.12378853059944) bank87577 +87578 POINT(40.69488114348823 -74.17207836021781) bank87578 +87579 POINT(41.68988987293279 -73.73145662829322) bank87579 +87580 POINT(40.42560393324051 -73.82234554653022) bank87580 +87581 POINT(41.644720771264694 -74.13738055763524) bank87581 +87582 POINT(39.90330743424527 -73.16120734835363) bank87582 +87583 POINT(40.81512933330059 -73.01841543903879) bank87583 +87584 POINT(40.57657653568092 -73.32364682610482) bank87584 +87585 POINT(40.37303840223985 -74.7844281505225) bank87585 +87586 POINT(40.64942883478006 -73.72394966709179) bank87586 +87587 POINT(39.90388957149398 -73.95105236472892) bank87587 +87588 POINT(40.069064004109535 -73.49616945264938) bank87588 +87589 POINT(41.13300444282292 -74.984079528026) bank87589 +87590 POINT(41.61220339520387 -73.51377814065494) bank87590 +87591 POINT(39.75598415360605 -74.92910730665547) bank87591 +87592 POINT(40.237559222202215 -74.66675623222551) bank87592 +87593 POINT(40.70479655856136 -73.51057807508268) bank87593 +87594 POINT(41.24370200028001 -74.22715220816457) bank87594 +87595 POINT(40.582757936095305 -74.55211041316615) bank87595 +87596 POINT(40.01027346624865 -74.24279822210408) bank87596 +87597 POINT(41.43952263217531 -73.95551725343634) bank87597 +87598 POINT(40.35153561270643 -74.47181374350092) bank87598 +87599 POINT(40.14827125117461 -74.13879523278739) bank87599 +87600 POINT(41.10960571550571 -74.62237298070166) bank87600 +87601 POINT(41.30524506955413 -73.79677976159677) bank87601 +87602 POINT(40.28392502289523 -74.30561763887543) bank87602 +87603 POINT(41.2489144105868 -74.60613240687596) bank87603 +87604 POINT(40.182604695823386 -74.66601816603583) bank87604 +87605 POINT(41.43171499717718 -73.36667987562713) bank87605 +87606 POINT(40.30323686989348 -74.36870108385631) bank87606 +87607 POINT(40.35517058004872 -74.99674228584385) bank87607 +87608 POINT(39.77158113821417 -74.39293031320271) bank87608 +87609 POINT(39.73377264154581 -74.81799466218315) bank87609 +87610 POINT(40.86469351175188 -73.75874368337388) bank87610 +87611 POINT(40.9779295888202 -74.2089146594601) bank87611 +87612 POINT(41.69058873526612 -73.30121648185992) bank87612 +87613 POINT(41.5910698552904 -73.87640166160247) bank87613 +87614 POINT(40.287993417388606 -73.57176287332058) bank87614 +87615 POINT(40.76658483029597 -73.10152184913971) bank87615 +87616 POINT(39.86441369868915 -73.82024921479396) bank87616 +87617 POINT(39.78883509065596 -73.96566982060396) bank87617 +87618 POINT(39.77424490075331 -74.79032928994454) bank87618 +87619 POINT(40.787419674190055 -74.14765716748258) bank87619 +87620 POINT(40.97124628224581 -74.57281580361615) bank87620 +87621 POINT(40.6644423585062 -74.21250967402533) bank87621 +87622 POINT(41.62748217260672 -73.09544980781396) bank87622 +87623 POINT(41.09748631541582 -74.68756511798449) bank87623 +87624 POINT(40.11115988914054 -73.21132079225272) bank87624 +87625 POINT(40.64855047815911 -73.29711471178396) bank87625 +87626 POINT(41.22025193894187 -73.07477689065107) bank87626 +87627 POINT(39.731362662740025 -74.42848763260594) bank87627 +87628 POINT(40.73708037866578 -74.36602118412894) bank87628 +87629 POINT(41.68346351105666 -73.98309831130906) bank87629 +87630 POINT(39.96557297229069 -73.55047151389716) bank87630 +87631 POINT(41.282491812634056 -74.65793963551484) bank87631 +87632 POINT(40.31353224305983 -73.85774957595123) bank87632 +87633 POINT(40.41264663191247 -73.48623294228629) bank87633 +87634 POINT(39.88129540815463 -74.83125008232643) bank87634 +87635 POINT(40.23806490257597 -73.8287715717394) bank87635 +87636 POINT(40.85986283261357 -74.6348009005362) bank87636 +87637 POINT(41.68449870511631 -73.49408093246295) bank87637 +87638 POINT(41.063311726958254 -73.05354743780842) bank87638 +87639 POINT(40.60390578342425 -74.71253364351139) bank87639 +87640 POINT(40.07322235052477 -73.76948048204702) bank87640 +87641 POINT(41.57498482194476 -73.5620624102792) bank87641 +87642 POINT(41.01792139971308 -74.24664680618852) bank87642 +87643 POINT(40.8341677303324 -73.93201805598743) bank87643 +87644 POINT(40.956198155395725 -74.43728741881776) bank87644 +87645 POINT(39.84513617670183 -73.15749384134196) bank87645 +87646 POINT(40.83915768012514 -73.1690380767677) bank87646 +87647 POINT(40.58403805981462 -73.71812387770241) bank87647 +87648 POINT(40.317851840711455 -73.17676440982854) bank87648 +87649 POINT(40.395049502855166 -74.81390854007786) bank87649 +87650 POINT(40.03799007957204 -74.04665859793694) bank87650 +87651 POINT(40.97311979523247 -74.70818918488091) bank87651 +87652 POINT(40.42718477238701 -74.0761801029857) bank87652 +87653 POINT(40.82547600807924 -74.3164396801837) bank87653 +87654 POINT(41.413272119947266 -73.84553781827977) bank87654 +87655 POINT(41.04454708691716 -73.68744755389821) bank87655 +87656 POINT(40.468540230562496 -73.29476003707221) bank87656 +87657 POINT(41.6767340993534 -73.91324003315485) bank87657 +87658 POINT(41.643635742366406 -73.56398288370593) bank87658 +87659 POINT(41.579384536466776 -73.01145450046504) bank87659 +87660 POINT(40.409335126724294 -73.95802816994264) bank87660 +87661 POINT(41.24329852531901 -74.63379367931283) bank87661 +87662 POINT(41.04285671945192 -74.82588795191619) bank87662 +87663 POINT(41.03930117118545 -74.10157518401847) bank87663 +87664 POINT(40.35313898840897 -74.08288292822189) bank87664 +87665 POINT(41.53263743840584 -73.57024575293947) bank87665 +87666 POINT(40.050560061878684 -74.73509644111778) bank87666 +87667 POINT(39.95524836805052 -73.04349510514874) bank87667 +87668 POINT(41.10186019538805 -74.91220810633585) bank87668 +87669 POINT(41.28383930069315 -74.57296154379546) bank87669 +87670 POINT(41.43367449875504 -74.23356224338312) bank87670 +87671 POINT(40.4445001491061 -73.89538790052733) bank87671 +87672 POINT(41.61305654492611 -74.27101064123086) bank87672 +87673 POINT(41.48961267887313 -73.30946267832809) bank87673 +87674 POINT(40.92132485544328 -75.00302648999714) bank87674 +87675 POINT(41.58323771538604 -74.07442769121121) bank87675 +87676 POINT(39.80987276369233 -73.20112506582286) bank87676 +87677 POINT(40.66754796883363 -73.72952908952806) bank87677 +87678 POINT(40.560350658630256 -74.34264945242224) bank87678 +87679 POINT(41.414251878524716 -73.4568632061088) bank87679 +87680 POINT(41.63984492756382 -73.11870965321019) bank87680 +87681 POINT(40.511316457098374 -73.91757205071146) bank87681 +87682 POINT(40.54435841245112 -74.74291723225011) bank87682 +87683 POINT(40.044078960178204 -73.1689721301591) bank87683 +87684 POINT(41.116199086286485 -74.3339509623675) bank87684 +87685 POINT(41.68233143110663 -74.2629643032765) bank87685 +87686 POINT(39.932329861843485 -74.27332250248368) bank87686 +87687 POINT(40.66094135275252 -74.28605553128237) bank87687 +87688 POINT(40.51834023446102 -73.20267472156603) bank87688 +87689 POINT(40.58608973638109 -74.50998049201172) bank87689 +87690 POINT(41.407970027743886 -74.93340827399092) bank87690 +87691 POINT(39.98510896805421 -74.30977507499814) bank87691 +87692 POINT(39.790090888673035 -73.8440041689847) bank87692 +87693 POINT(40.401751709205136 -73.49703853429303) bank87693 +87694 POINT(41.047688671327286 -74.86101035089804) bank87694 +87695 POINT(40.805755404026606 -74.01545301446737) bank87695 +87696 POINT(40.128204185215345 -74.28907539172906) bank87696 +87697 POINT(39.74102423907871 -73.0808991171416) bank87697 +87698 POINT(39.77650449939235 -73.4141801245659) bank87698 +87699 POINT(39.97693741808639 -73.23794192403672) bank87699 +87700 POINT(40.60967592258638 -73.01262104273768) bank87700 +87701 POINT(39.8928077359487 -74.50686782419945) bank87701 +87702 POINT(41.03515783448177 -73.91715471067525) bank87702 +87703 POINT(39.76252928517527 -73.16518423493805) bank87703 +87704 POINT(39.82573290574253 -73.50841195691261) bank87704 +87705 POINT(40.37434811169327 -74.02937289080302) bank87705 +87706 POINT(39.82331085033312 -73.63383437112198) bank87706 +87707 POINT(41.06696239461232 -73.73242714867922) bank87707 +87708 POINT(39.775399003542645 -74.65131402231215) bank87708 +87709 POINT(39.89245296731643 -73.02254856907555) bank87709 +87710 POINT(40.633466707148806 -74.33584805074598) bank87710 +87711 POINT(41.35084961401573 -74.74877012789626) bank87711 +87712 POINT(40.281429024597216 -74.42760622111457) bank87712 +87713 POINT(39.890777568506074 -74.15581502174165) bank87713 +87714 POINT(40.59766616466195 -74.24073119829512) bank87714 +87715 POINT(41.64773743659864 -73.35367141520858) bank87715 +87716 POINT(41.584262859812064 -73.0774218632446) bank87716 +87717 POINT(41.11755468812925 -73.12614404416972) bank87717 +87718 POINT(41.1125984069999 -73.34502164626514) bank87718 +87719 POINT(40.21954646174404 -74.8948599950178) bank87719 +87720 POINT(40.46586684339347 -74.45323891194309) bank87720 +87721 POINT(40.71703872793658 -73.40340913576476) bank87721 +87722 POINT(40.148869662867625 -74.09704974314957) bank87722 +87723 POINT(39.80285536600849 -74.09313397565555) bank87723 +87724 POINT(41.69116848946196 -73.63966248665737) bank87724 +87725 POINT(40.495189308243084 -74.21524084445932) bank87725 +87726 POINT(39.79058004317414 -74.30605901791836) bank87726 +87727 POINT(40.626137335308314 -73.63185485277921) bank87727 +87728 POINT(41.39999996827345 -74.17712976780754) bank87728 +87729 POINT(40.016166376248364 -74.15253447480956) bank87729 +87730 POINT(40.57319053225919 -73.75535002737766) bank87730 +87731 POINT(40.636281898823015 -74.86486087539946) bank87731 +87732 POINT(39.984383829141144 -73.55897077790357) bank87732 +87733 POINT(41.40484113424099 -74.47958562876353) bank87733 +87734 POINT(40.242619519715866 -74.19272016822549) bank87734 +87735 POINT(40.87897375542169 -74.79014698297016) bank87735 +87736 POINT(40.33427197900363 -73.32882743398162) bank87736 +87737 POINT(39.8211285934934 -74.04090806706179) bank87737 +87738 POINT(41.56069587051164 -74.31666250702946) bank87738 +87739 POINT(40.86432866708007 -73.70594976325556) bank87739 +87740 POINT(40.287563377981556 -73.78403975845718) bank87740 +87741 POINT(40.201696245174254 -73.98604946948655) bank87741 +87742 POINT(40.82454725935962 -74.96916506763625) bank87742 +87743 POINT(40.89557732776272 -74.61053322673298) bank87743 +87744 POINT(41.47954119692738 -73.36596043814887) bank87744 +87745 POINT(40.70759587064719 -73.41862967311354) bank87745 +87746 POINT(40.344978613714645 -74.00672275592295) bank87746 +87747 POINT(40.283338568943805 -74.09125775941186) bank87747 +87748 POINT(39.84652101548686 -74.36930303789687) bank87748 +87749 POINT(40.62893041647616 -73.27966360473656) bank87749 +87750 POINT(41.70551772654357 -73.47303768006721) bank87750 +87751 POINT(41.38731074139013 -74.9407569884748) bank87751 +87752 POINT(40.69005549080298 -73.52767444092707) bank87752 +87753 POINT(40.051003691189656 -73.38198000987651) bank87753 +87754 POINT(40.10775895382751 -73.15220291988638) bank87754 +87755 POINT(40.95068843897481 -74.85482497807584) bank87755 +87756 POINT(41.10832240349392 -74.83365186651976) bank87756 +87757 POINT(40.73290290154062 -73.01245153615311) bank87757 +87758 POINT(40.42282371319335 -73.2185307740381) bank87758 +87759 POINT(41.27380387250449 -74.96582625231386) bank87759 +87760 POINT(40.03518711524417 -73.50227264093023) bank87760 +87761 POINT(41.61537663218586 -74.71823585090554) bank87761 +87762 POINT(39.93274293100885 -73.742838223835) bank87762 +87763 POINT(40.99740544576137 -74.72523405497617) bank87763 +87764 POINT(39.986153695259915 -74.06406320501864) bank87764 +87765 POINT(41.07733696364091 -74.90495272930542) bank87765 +87766 POINT(41.083502316040686 -74.80252694815844) bank87766 +87767 POINT(41.274321485857065 -73.0162310212533) bank87767 +87768 POINT(40.17382187150904 -73.53294722027313) bank87768 +87769 POINT(41.515848300764425 -74.19290249555429) bank87769 +87770 POINT(40.32513907048415 -73.59944014879578) bank87770 +87771 POINT(40.25824172695919 -73.71688989748375) bank87771 +87772 POINT(40.86616616894395 -73.19211943811317) bank87772 +87773 POINT(41.41402842590905 -74.50685815614955) bank87773 +87774 POINT(41.48452540540978 -74.28375450994263) bank87774 +87775 POINT(40.064742123990946 -73.4637706804358) bank87775 +87776 POINT(39.7402619987788 -73.91812122970842) bank87776 +87777 POINT(40.56511480816232 -74.05595368331728) bank87777 +87778 POINT(40.00676651821837 -73.33101122510426) bank87778 +87779 POINT(39.98204591116923 -74.34123725615524) bank87779 +87780 POINT(39.86154226501015 -74.31827286764413) bank87780 +87781 POINT(41.26074451863872 -74.68709205728346) bank87781 +87782 POINT(41.615345087935474 -73.98687172944894) bank87782 +87783 POINT(41.46238047094639 -74.58759480669684) bank87783 +87784 POINT(41.44019529763593 -73.9112814396429) bank87784 +87785 POINT(40.28205601260791 -73.9153745149226) bank87785 +87786 POINT(41.53759411976977 -73.31068890142335) bank87786 +87787 POINT(39.72324367770307 -73.17744128838868) bank87787 +87788 POINT(41.488786156299454 -73.78353688996665) bank87788 +87789 POINT(41.189287506857795 -73.5796786899412) bank87789 +87790 POINT(41.43919526846524 -74.51884192209313) bank87790 +87791 POINT(40.072830811195466 -73.15649265674944) bank87791 +87792 POINT(40.914658894909586 -74.23649920032376) bank87792 +87793 POINT(40.69174211471498 -73.35903753167194) bank87793 +87794 POINT(41.5902917547574 -74.40214873099816) bank87794 +87795 POINT(39.87765495186828 -73.89669816679734) bank87795 +87796 POINT(40.39936009959814 -73.11867964730412) bank87796 +87797 POINT(41.17256024173213 -73.82266719430203) bank87797 +87798 POINT(40.31526966275929 -73.49888505577525) bank87798 +87799 POINT(41.25558141908532 -74.77889083494978) bank87799 +87800 POINT(41.45885426356229 -73.9212865419563) bank87800 +87801 POINT(39.99433946874776 -73.62680087879139) bank87801 +87802 POINT(40.38168917571842 -74.84434734229939) bank87802 +87803 POINT(40.53637841830505 -74.68699486962684) bank87803 +87804 POINT(40.12175101504663 -74.46466370974885) bank87804 +87805 POINT(41.50607055813582 -73.1419731852042) bank87805 +87806 POINT(40.0830923873933 -74.07728091336713) bank87806 +87807 POINT(40.016745385920515 -74.93607317655301) bank87807 +87808 POINT(41.101360093189506 -74.96990092715346) bank87808 +87809 POINT(41.471830168841414 -74.85035451057742) bank87809 +87810 POINT(40.55168896344353 -74.85463339833407) bank87810 +87811 POINT(41.12213362726802 -73.9156670301512) bank87811 +87812 POINT(41.01672433841624 -74.78010669524627) bank87812 +87813 POINT(41.26867396842076 -74.75937117898035) bank87813 +87814 POINT(40.98759192387507 -74.30669088144137) bank87814 +87815 POINT(41.66333318744592 -73.08392093448529) bank87815 +87816 POINT(41.49578303819652 -74.29811497554994) bank87816 +87817 POINT(40.79179887758832 -74.00965260417287) bank87817 +87818 POINT(40.36416714472138 -73.58599896973085) bank87818 +87819 POINT(39.840195138473014 -73.25808609569808) bank87819 +87820 POINT(40.1829922132368 -74.2535400446198) bank87820 +87821 POINT(40.309842257174985 -73.37226927133621) bank87821 +87822 POINT(41.26558162149403 -73.81835749749433) bank87822 +87823 POINT(41.449869950550934 -74.08463708694323) bank87823 +87824 POINT(40.40235691673567 -73.06871974132169) bank87824 +87825 POINT(40.44842167789126 -74.39895978442995) bank87825 +87826 POINT(40.28818322517272 -74.39960734571024) bank87826 +87827 POINT(41.677939013228446 -73.63666178587764) bank87827 +87828 POINT(41.29144290890908 -73.95704806910844) bank87828 +87829 POINT(41.663141744600075 -73.95741001777505) bank87829 +87830 POINT(41.25346150285508 -74.2451595531155) bank87830 +87831 POINT(40.540728823797984 -74.5940211802537) bank87831 +87832 POINT(40.463158331516595 -74.36411479878943) bank87832 +87833 POINT(40.76781291942827 -73.02494335203666) bank87833 +87834 POINT(40.94649274506047 -74.49558784023563) bank87834 +87835 POINT(40.373896372316686 -73.36110870156162) bank87835 +87836 POINT(40.12615555865012 -73.04017711268916) bank87836 +87837 POINT(40.14151245831385 -73.57979860038145) bank87837 +87838 POINT(40.7030353508636 -74.57694776839465) bank87838 +87839 POINT(40.61333208565515 -74.06064358193602) bank87839 +87840 POINT(41.70017393119831 -73.20968852203171) bank87840 +87841 POINT(40.26301527915202 -74.02717834769764) bank87841 +87842 POINT(40.01502082813248 -73.32469346684192) bank87842 +87843 POINT(41.48241678806484 -74.2272018445648) bank87843 +87844 POINT(40.034195833054596 -73.75530532330983) bank87844 +87845 POINT(40.21545319191465 -74.65312565013153) bank87845 +87846 POINT(39.85714101690046 -73.59470108166923) bank87846 +87847 POINT(40.241676270721555 -74.02965964808993) bank87847 +87848 POINT(41.580011310824716 -74.716154134297) bank87848 +87849 POINT(40.56793863797267 -73.27032917162006) bank87849 +87850 POINT(39.83170663411062 -73.7901692061824) bank87850 +87851 POINT(41.603491870000106 -74.23748646398896) bank87851 +87852 POINT(40.7708715897129 -73.73613644468126) bank87852 +87853 POINT(40.32342679046906 -74.34246493583893) bank87853 +87854 POINT(40.96568500678854 -73.9228501458108) bank87854 +87855 POINT(41.272209019177154 -73.74528533503914) bank87855 +87856 POINT(41.429473050192925 -73.82447038893514) bank87856 +87857 POINT(41.02596686793558 -73.61751209579805) bank87857 +87858 POINT(39.978164569863495 -73.39775185417736) bank87858 +87859 POINT(39.954254215779486 -73.13180943026669) bank87859 +87860 POINT(40.43600520873638 -73.23808760440677) bank87860 +87861 POINT(40.81354360038136 -74.46373077977782) bank87861 +87862 POINT(40.611338768205286 -74.030325730976) bank87862 +87863 POINT(40.777821659821 -74.61002682792667) bank87863 +87864 POINT(40.53045844296835 -74.26833854480498) bank87864 +87865 POINT(40.61302474806406 -73.26847366866812) bank87865 +87866 POINT(41.62620763093724 -73.63096518369592) bank87866 +87867 POINT(40.53150566111333 -74.94145974166746) bank87867 +87868 POINT(40.35960986810515 -75.00355758295434) bank87868 +87869 POINT(41.05397993126307 -73.33418860019071) bank87869 +87870 POINT(40.51091559415904 -74.89856023076239) bank87870 +87871 POINT(41.438325232950554 -73.32635604405822) bank87871 +87872 POINT(40.667406556538566 -73.39692659486062) bank87872 +87873 POINT(41.28331747248565 -74.40727768379145) bank87873 +87874 POINT(41.372407899476826 -73.23436110674592) bank87874 +87875 POINT(39.72356627404185 -74.89721964196248) bank87875 +87876 POINT(40.34149055895715 -73.83335441567766) bank87876 +87877 POINT(40.11745978124396 -73.1156295954104) bank87877 +87878 POINT(41.570439920955515 -74.03405023049461) bank87878 +87879 POINT(40.43183331420753 -73.12109858025622) bank87879 +87880 POINT(41.170657627501164 -73.62838416494169) bank87880 +87881 POINT(40.467049964092546 -73.82227806297085) bank87881 +87882 POINT(40.39910491492115 -74.55941836937315) bank87882 +87883 POINT(40.15090058392525 -73.6135715107953) bank87883 +87884 POINT(41.182894308414625 -73.3635752970606) bank87884 +87885 POINT(41.20353961014275 -73.05460740095388) bank87885 +87886 POINT(40.90537131589264 -73.74852646360493) bank87886 +87887 POINT(40.45738990934369 -74.18265683638593) bank87887 +87888 POINT(40.20462230004577 -73.34647101702791) bank87888 +87889 POINT(41.701433185405236 -73.10497474318265) bank87889 +87890 POINT(41.48415941392613 -73.53587520211565) bank87890 +87891 POINT(40.57728101921343 -74.6391422225477) bank87891 +87892 POINT(40.49698800447825 -73.9202977413145) bank87892 +87893 POINT(40.19617870734359 -74.57877370649838) bank87893 +87894 POINT(41.283649991293494 -74.26029757244014) bank87894 +87895 POINT(40.494656565899774 -73.34785543507864) bank87895 +87896 POINT(41.08587163064048 -73.42605299074076) bank87896 +87897 POINT(39.73953556776272 -74.20959844505323) bank87897 +87898 POINT(40.57932693012678 -74.32796568456413) bank87898 +87899 POINT(40.15901554942907 -73.64365345362478) bank87899 +87900 POINT(39.868121669900745 -74.26708654666821) bank87900 +87901 POINT(41.46105120866333 -73.90180079738344) bank87901 +87902 POINT(41.12670089013428 -73.3823502215476) bank87902 +87903 POINT(41.64026311728116 -73.11701492575733) bank87903 +87904 POINT(40.769334521650634 -73.94553712501443) bank87904 +87905 POINT(40.91237890887349 -73.19898753073133) bank87905 +87906 POINT(39.790606049172695 -74.76898224747707) bank87906 +87907 POINT(41.3764399450173 -73.14999526687666) bank87907 +87908 POINT(40.74737588364895 -73.5251401212528) bank87908 +87909 POINT(41.15184193695556 -73.07396717256849) bank87909 +87910 POINT(41.615103822299936 -73.33847226734275) bank87910 +87911 POINT(41.31007858748856 -74.02428683441552) bank87911 +87912 POINT(39.89799550577563 -73.91963220116587) bank87912 +87913 POINT(41.11520161425152 -74.7354709672042) bank87913 +87914 POINT(39.73343771310084 -73.42241136339582) bank87914 +87915 POINT(40.96372040361342 -74.61707560141393) bank87915 +87916 POINT(41.3973857978506 -73.33846109582967) bank87916 +87917 POINT(41.181414154210415 -73.15047176412921) bank87917 +87918 POINT(41.62636355903654 -73.83206728324451) bank87918 +87919 POINT(39.999246530457796 -73.72121432082571) bank87919 +87920 POINT(41.65613385274657 -74.84054476272772) bank87920 +87921 POINT(40.030021494927425 -73.47984382764304) bank87921 +87922 POINT(39.84613060486174 -74.7199141317201) bank87922 +87923 POINT(40.76631017051463 -73.7981948691425) bank87923 +87924 POINT(41.69150045802707 -74.23575299594516) bank87924 +87925 POINT(40.084705443093995 -73.43279359882297) bank87925 +87926 POINT(40.85285783266724 -73.74289598436302) bank87926 +87927 POINT(39.83512676632296 -74.49230420740484) bank87927 +87928 POINT(41.24748829061694 -73.14163914783792) bank87928 +87929 POINT(41.38489805970691 -74.86073004838745) bank87929 +87930 POINT(41.60453367473954 -74.906695508736) bank87930 +87931 POINT(40.36442017229131 -74.7893144409789) bank87931 +87932 POINT(40.9280308332333 -73.72093454816388) bank87932 +87933 POINT(40.37191449325035 -74.07598297941429) bank87933 +87934 POINT(40.70790160518734 -74.86699386565967) bank87934 +87935 POINT(41.10840799873016 -74.13670125465254) bank87935 +87936 POINT(40.51398877684243 -74.86740382520793) bank87936 +87937 POINT(39.94888597917819 -73.77233851700744) bank87937 +87938 POINT(41.610459729775805 -73.93060883210076) bank87938 +87939 POINT(39.9506871443037 -73.46807572351827) bank87939 +87940 POINT(41.219820860117395 -74.51917601066357) bank87940 +87941 POINT(41.052702416746456 -73.84976372855284) bank87941 +87942 POINT(40.21952497091687 -73.15996968570744) bank87942 +87943 POINT(41.36927704886264 -73.50227308384488) bank87943 +87944 POINT(41.08802840974962 -74.68107996755747) bank87944 +87945 POINT(40.13635520583757 -74.76685849430456) bank87945 +87946 POINT(39.79081914631891 -73.94640390926209) bank87946 +87947 POINT(40.85541242104521 -74.49751759840777) bank87947 +87948 POINT(39.9623343505292 -74.29904515105864) bank87948 +87949 POINT(40.49125120727981 -74.38953872498175) bank87949 +87950 POINT(40.49139644173439 -74.20139847980023) bank87950 +87951 POINT(41.23996116494743 -73.41541923676489) bank87951 +87952 POINT(40.9381806674794 -73.54953002199665) bank87952 +87953 POINT(41.23142333906658 -74.86509621013207) bank87953 +87954 POINT(40.16904223206552 -73.23020788230356) bank87954 +87955 POINT(41.61094618849891 -73.12300501480019) bank87955 +87956 POINT(39.972537111726 -73.52773599259085) bank87956 +87957 POINT(41.64041107649773 -74.48565568606256) bank87957 +87958 POINT(41.38590868586136 -74.4163924252183) bank87958 +87959 POINT(41.05937633131692 -74.60586057290429) bank87959 +87960 POINT(41.40957494580972 -74.35473622743217) bank87960 +87961 POINT(40.08461835765962 -74.23521637999777) bank87961 +87962 POINT(40.78984916440874 -74.02428770910085) bank87962 +87963 POINT(40.51303937969666 -73.60629291626982) bank87963 +87964 POINT(41.64753872867104 -74.352594021106) bank87964 +87965 POINT(40.27096714977417 -74.71291362962636) bank87965 +87966 POINT(40.18919504767441 -74.32127255108264) bank87966 +87967 POINT(41.00065232081667 -74.44808682801295) bank87967 +87968 POINT(40.837137007176494 -73.16148276112021) bank87968 +87969 POINT(41.39905593092219 -73.37149344055686) bank87969 +87970 POINT(40.35091244244081 -73.89451821747716) bank87970 +87971 POINT(40.919645827012666 -74.31960786652215) bank87971 +87972 POINT(40.61133274353298 -73.71141126040798) bank87972 +87973 POINT(40.20693390168915 -73.23883784139899) bank87973 +87974 POINT(40.75330489109502 -73.34573492369364) bank87974 +87975 POINT(41.66414523784262 -74.75716968969873) bank87975 +87976 POINT(40.27051899159571 -73.74038293328765) bank87976 +87977 POINT(40.80692383928579 -73.13121983208924) bank87977 +87978 POINT(40.73728218476793 -74.81161241286947) bank87978 +87979 POINT(40.083794241687976 -73.7154506577227) bank87979 +87980 POINT(40.111360788485214 -74.9976571497016) bank87980 +87981 POINT(39.933462686540146 -74.33698588127916) bank87981 +87982 POINT(39.783610062521035 -74.47421952739047) bank87982 +87983 POINT(39.91250671618024 -73.84528174483277) bank87983 +87984 POINT(41.158515506003226 -74.96998872327241) bank87984 +87985 POINT(41.39117073904795 -73.73233106855126) bank87985 +87986 POINT(41.27264723576659 -73.46479886028877) bank87986 +87987 POINT(39.85876720256203 -74.91951716100284) bank87987 +87988 POINT(40.742793994235456 -74.6680376350802) bank87988 +87989 POINT(40.22091046129788 -74.76521461344237) bank87989 +87990 POINT(41.654259008332644 -73.16425351420894) bank87990 +87991 POINT(40.91807508849138 -73.23351918021756) bank87991 +87992 POINT(41.54562050205199 -74.40353099863351) bank87992 +87993 POINT(39.8259720776602 -74.36951414757306) bank87993 +87994 POINT(41.18731348693237 -73.26546126363762) bank87994 +87995 POINT(40.01219850535607 -74.46549075728448) bank87995 +87996 POINT(40.01241170408596 -74.11961439794) bank87996 +87997 POINT(40.44862627795691 -74.27035549570537) bank87997 +87998 POINT(41.613743899605005 -74.96895091948204) bank87998 +87999 POINT(40.19166766524342 -75.00196713322323) bank87999 +88000 POINT(40.31239512757562 -73.04743051729352) bank88000 +88001 POINT(40.850588468992775 -73.29929193138713) bank88001 +88002 POINT(40.38774584102097 -73.63144149229166) bank88002 +88003 POINT(41.58240416218112 -73.76336165490905) bank88003 +88004 POINT(40.24091942022189 -73.91697646235865) bank88004 +88005 POINT(41.34216358026305 -74.38656247448246) bank88005 +88006 POINT(40.653218489229445 -73.78841637110618) bank88006 +88007 POINT(40.53018341239939 -73.65536403943744) bank88007 +88008 POINT(40.920551979279466 -73.34699696839677) bank88008 +88009 POINT(40.99528329254074 -73.82055858907766) bank88009 +88010 POINT(41.08147135092166 -73.27813352928742) bank88010 +88011 POINT(41.608383101036175 -74.7440547297516) bank88011 +88012 POINT(40.51828112144459 -74.51205398657942) bank88012 +88013 POINT(40.62695586073389 -74.82160279853309) bank88013 +88014 POINT(40.45573404047287 -73.60404167590433) bank88014 +88015 POINT(40.95707301041884 -73.36271140769634) bank88015 +88016 POINT(41.5462960835495 -74.8955261251341) bank88016 +88017 POINT(41.415193331178415 -74.02810348796517) bank88017 +88018 POINT(41.69711966934026 -73.53263097918918) bank88018 +88019 POINT(40.066315691165194 -74.05723663449434) bank88019 +88020 POINT(40.55011429905585 -74.7056403742344) bank88020 +88021 POINT(41.24288658920489 -73.82537456641367) bank88021 +88022 POINT(41.302793089793894 -74.07789706830991) bank88022 +88023 POINT(40.7063608201687 -74.8855724332627) bank88023 +88024 POINT(40.53801059944985 -74.1991038865142) bank88024 +88025 POINT(41.09676539357784 -73.91075168418375) bank88025 +88026 POINT(40.587821887310206 -74.25042931111844) bank88026 +88027 POINT(41.567038363291395 -74.87228480318616) bank88027 +88028 POINT(40.23202919952749 -73.95392463934961) bank88028 +88029 POINT(39.78914980415718 -73.94654791827153) bank88029 +88030 POINT(39.808598568363294 -73.76696117427367) bank88030 +88031 POINT(41.02102020141248 -73.61459274807834) bank88031 +88032 POINT(41.11435035269791 -74.92030173466) bank88032 +88033 POINT(39.99561562111567 -73.7027411468421) bank88033 +88034 POINT(40.69536470973283 -73.3555806091509) bank88034 +88035 POINT(39.83161316913468 -73.54817996565897) bank88035 +88036 POINT(40.892495203141564 -74.03740167507713) bank88036 +88037 POINT(40.13617395823814 -74.13010416580923) bank88037 +88038 POINT(39.923875600093744 -74.31892470333682) bank88038 +88039 POINT(41.45328543509205 -73.67926490870927) bank88039 +88040 POINT(40.41738332300942 -74.44067406621771) bank88040 +88041 POINT(40.546025240337 -74.66366534672804) bank88041 +88042 POINT(39.95256240884143 -74.48664302406057) bank88042 +88043 POINT(41.449285114116236 -74.61540169383623) bank88043 +88044 POINT(39.812745075643285 -74.2671182403429) bank88044 +88045 POINT(41.44185740225056 -74.52327918100404) bank88045 +88046 POINT(41.13310300598121 -73.08022724222315) bank88046 +88047 POINT(41.27890062471907 -74.08084646422206) bank88047 +88048 POINT(40.78567588827611 -74.38330365502756) bank88048 +88049 POINT(40.584830966791095 -74.4343628271429) bank88049 +88050 POINT(41.199247516140254 -74.33298035686228) bank88050 +88051 POINT(40.43923750185572 -73.36795568050087) bank88051 +88052 POINT(40.1613185028024 -73.93567353482831) bank88052 +88053 POINT(41.26715196943504 -74.96379921579629) bank88053 +88054 POINT(41.44775695702307 -74.88546301636664) bank88054 +88055 POINT(41.276129101245154 -73.34766583536369) bank88055 +88056 POINT(39.741621613319964 -73.13256548558115) bank88056 +88057 POINT(40.96659127035949 -74.41251607970607) bank88057 +88058 POINT(39.90544797118229 -73.22183882368925) bank88058 +88059 POINT(41.29586956276804 -73.88314078117472) bank88059 +88060 POINT(40.316915135220505 -73.1396691897895) bank88060 +88061 POINT(40.43851155454737 -74.45274079122484) bank88061 +88062 POINT(41.24619504422434 -73.47234755368328) bank88062 +88063 POINT(40.137340147907985 -74.8804975883004) bank88063 +88064 POINT(39.86643508660852 -74.83366943606153) bank88064 +88065 POINT(41.49918719250648 -74.3462960291788) bank88065 +88066 POINT(40.310964275718604 -74.0061734002284) bank88066 +88067 POINT(41.19393712628817 -73.67346276183446) bank88067 +88068 POINT(40.936007127088665 -74.40025948846821) bank88068 +88069 POINT(41.37783389099395 -74.11865293111472) bank88069 +88070 POINT(41.29717216876926 -73.59041604949182) bank88070 +88071 POINT(41.292177015311985 -73.67769165225646) bank88071 +88072 POINT(41.248433904269376 -73.50229573922581) bank88072 +88073 POINT(40.730489177812814 -73.24165786741544) bank88073 +88074 POINT(40.02050881977491 -74.57936909966878) bank88074 +88075 POINT(39.88645385315713 -74.54586797358175) bank88075 +88076 POINT(40.49257499299141 -74.97598104694703) bank88076 +88077 POINT(40.422598261888 -73.62730730789897) bank88077 +88078 POINT(40.95276576997588 -74.5564085274766) bank88078 +88079 POINT(40.24377396789507 -74.15409583473182) bank88079 +88080 POINT(41.321594078814826 -74.6528231587922) bank88080 +88081 POINT(41.37266562126781 -74.6036958309742) bank88081 +88082 POINT(39.925753467440835 -73.51139672037652) bank88082 +88083 POINT(41.35735401315577 -73.66423627728774) bank88083 +88084 POINT(39.97874351921373 -74.01590343314507) bank88084 +88085 POINT(41.18805630513826 -73.94116994993882) bank88085 +88086 POINT(40.91324409345594 -73.85017000032865) bank88086 +88087 POINT(40.13138709774733 -74.38423263063117) bank88087 +88088 POINT(39.9708978770496 -74.61140027024423) bank88088 +88089 POINT(41.02930739720048 -74.02613458342228) bank88089 +88090 POINT(41.00373656579515 -74.16073545528128) bank88090 +88091 POINT(41.01565503651836 -73.57980554133002) bank88091 +88092 POINT(40.99962945146536 -74.04063306604868) bank88092 +88093 POINT(40.158681297012656 -74.13013818845083) bank88093 +88094 POINT(40.57945389744634 -73.46690320180241) bank88094 +88095 POINT(39.8959225271166 -73.43397930122737) bank88095 +88096 POINT(41.282586065675936 -73.9109139244621) bank88096 +88097 POINT(40.42008237762855 -74.43314959620989) bank88097 +88098 POINT(40.942560229303915 -73.25537406814972) bank88098 +88099 POINT(41.62865100386326 -74.87527664004355) bank88099 +88100 POINT(41.61910754213875 -74.42580910616462) bank88100 +88101 POINT(40.487739065769816 -74.26458161589586) bank88101 +88102 POINT(40.138873307133245 -73.75041831231948) bank88102 +88103 POINT(40.146156315387486 -73.26375750166751) bank88103 +88104 POINT(40.18980249619986 -74.91442476794428) bank88104 +88105 POINT(40.47573835776361 -74.09990900194397) bank88105 +88106 POINT(41.35255225741421 -74.41433124253368) bank88106 +88107 POINT(40.35593179284442 -74.59285457700042) bank88107 +88108 POINT(39.88992832669604 -74.97451374433831) bank88108 +88109 POINT(41.69474458011264 -73.88041569516167) bank88109 +88110 POINT(39.76253628727152 -73.18654630197801) bank88110 +88111 POINT(41.23007654093539 -74.3664308447274) bank88111 +88112 POINT(39.91400073239322 -73.97071464508582) bank88112 +88113 POINT(41.45897513816542 -73.42239744428231) bank88113 +88114 POINT(40.85066026967879 -74.63857929581837) bank88114 +88115 POINT(40.43395472061829 -74.58048419589666) bank88115 +88116 POINT(40.63024394823712 -74.46284872394034) bank88116 +88117 POINT(40.8347127885975 -74.37273449954016) bank88117 +88118 POINT(41.30810797857164 -74.24940397353805) bank88118 +88119 POINT(40.96641171829521 -74.73692148649629) bank88119 +88120 POINT(41.677495947269136 -74.32208830302582) bank88120 +88121 POINT(39.80205733581835 -73.20449435785767) bank88121 +88122 POINT(40.28238429605924 -73.77120621811737) bank88122 +88123 POINT(40.897958135512205 -74.06870233538595) bank88123 +88124 POINT(41.39246425398832 -74.48032902991147) bank88124 +88125 POINT(40.829978116188784 -74.74606485777515) bank88125 +88126 POINT(39.82116539756261 -74.0183180037737) bank88126 +88127 POINT(39.92845143661733 -74.73368398070717) bank88127 +88128 POINT(41.06458279805579 -74.38762680369064) bank88128 +88129 POINT(41.661004255660245 -74.92672566600774) bank88129 +88130 POINT(40.00820742167259 -74.36172172264938) bank88130 +88131 POINT(40.188801249921184 -73.98589718295803) bank88131 +88132 POINT(40.48019662698743 -73.44394946378608) bank88132 +88133 POINT(40.43902744008667 -74.62142007462666) bank88133 +88134 POINT(41.00759104197395 -73.34899579593849) bank88134 +88135 POINT(40.162593779530646 -74.97976938524535) bank88135 +88136 POINT(40.133393036497544 -74.19883353971652) bank88136 +88137 POINT(40.746522674938696 -74.13021803117763) bank88137 +88138 POINT(40.14861739541106 -73.36964186657681) bank88138 +88139 POINT(41.69370715624917 -73.84838755825324) bank88139 +88140 POINT(41.44435857265775 -74.69305980903371) bank88140 +88141 POINT(40.88338541735673 -73.89227022416732) bank88141 +88142 POINT(41.650060674527566 -73.44999909588218) bank88142 +88143 POINT(41.57974377595951 -74.48210325922946) bank88143 +88144 POINT(40.61826864236023 -73.11058601193702) bank88144 +88145 POINT(39.89649547472928 -74.79908227093033) bank88145 +88146 POINT(41.27601126395816 -74.27447259327876) bank88146 +88147 POINT(41.219326626508504 -74.62275813708413) bank88147 +88148 POINT(41.526671342914774 -74.3736368243333) bank88148 +88149 POINT(40.85013350585931 -74.90993415970316) bank88149 +88150 POINT(40.75933910832192 -74.606137237699) bank88150 +88151 POINT(39.8418614408561 -73.70580971449685) bank88151 +88152 POINT(39.92650422878 -74.24545321987122) bank88152 +88153 POINT(41.180083732964476 -74.21168055959156) bank88153 +88154 POINT(41.1143245745119 -73.06202900153008) bank88154 +88155 POINT(41.22141713523742 -74.9294986829199) bank88155 +88156 POINT(40.11855815217269 -73.28305031752387) bank88156 +88157 POINT(41.61640909963217 -74.22115675821276) bank88157 +88158 POINT(41.242021160646175 -73.61611890115299) bank88158 +88159 POINT(40.19674698721299 -74.9976465886953) bank88159 +88160 POINT(41.611337314436035 -74.51171428679316) bank88160 +88161 POINT(40.939160256185424 -73.71097465898168) bank88161 +88162 POINT(39.88966829966 -74.8991156582039) bank88162 +88163 POINT(39.80039572157692 -73.9868562994643) bank88163 +88164 POINT(41.61999878943345 -74.78740420812723) bank88164 +88165 POINT(39.994400405551175 -74.54824701156551) bank88165 +88166 POINT(40.75946456720979 -73.04209517900357) bank88166 +88167 POINT(41.56549831359467 -73.57041859758002) bank88167 +88168 POINT(40.2289163717581 -74.84297594199964) bank88168 +88169 POINT(40.60920227132287 -73.99316596672692) bank88169 +88170 POINT(40.193645594635626 -73.20952646358901) bank88170 +88171 POINT(40.849357930623825 -74.91999060299236) bank88171 +88172 POINT(40.39062393487871 -74.07739600463867) bank88172 +88173 POINT(41.16776521071019 -74.82462873264868) bank88173 +88174 POINT(40.67458373429945 -73.11852351021116) bank88174 +88175 POINT(39.87305236655084 -73.63756742011586) bank88175 +88176 POINT(41.47460433500549 -73.94508525018466) bank88176 +88177 POINT(40.77407447441351 -74.54096782475477) bank88177 +88178 POINT(40.50281580638273 -74.09139776078833) bank88178 +88179 POINT(41.24099528725838 -74.9490590482762) bank88179 +88180 POINT(41.34194698423895 -74.83359204095046) bank88180 +88181 POINT(40.66974735212214 -73.03821339062124) bank88181 +88182 POINT(40.14441853938066 -73.54074938269126) bank88182 +88183 POINT(40.92694544360837 -74.83760578121571) bank88183 +88184 POINT(40.867770962821055 -74.72697550656189) bank88184 +88185 POINT(39.97414568009018 -73.32889942462968) bank88185 +88186 POINT(39.868372074523464 -73.2461466921064) bank88186 +88187 POINT(41.1946143754773 -73.5510345045478) bank88187 +88188 POINT(40.53020611145667 -73.02068434816675) bank88188 +88189 POINT(40.276936941190726 -74.25111933948676) bank88189 +88190 POINT(40.53267328353461 -74.13526514331164) bank88190 +88191 POINT(41.2917062440689 -74.62363731365326) bank88191 +88192 POINT(39.745196521415544 -74.51787429055221) bank88192 +88193 POINT(40.12558746113942 -73.19886829445028) bank88193 +88194 POINT(39.81176168666199 -74.11033301714882) bank88194 +88195 POINT(41.51185283305058 -74.365219532075) bank88195 +88196 POINT(41.221279731711185 -73.89125827949492) bank88196 +88197 POINT(40.04131787519398 -74.50574184566446) bank88197 +88198 POINT(41.30070431576551 -74.06014892146405) bank88198 +88199 POINT(41.053171061951296 -74.66371424227256) bank88199 +88200 POINT(40.455346651152446 -74.57920605752402) bank88200 +88201 POINT(41.69153237666436 -73.11561741540075) bank88201 +88202 POINT(41.45622361608953 -73.80210181719919) bank88202 +88203 POINT(40.04563867445413 -74.56167163832698) bank88203 +88204 POINT(40.08191448686227 -74.56706356270482) bank88204 +88205 POINT(41.29306911092022 -74.58252088331288) bank88205 +88206 POINT(41.04665075382806 -73.95840178092239) bank88206 +88207 POINT(40.20243612771298 -73.11530496214088) bank88207 +88208 POINT(40.56220603603673 -73.86195203580465) bank88208 +88209 POINT(40.505497588981335 -73.76885911480666) bank88209 +88210 POINT(40.36873884397518 -73.65765840039933) bank88210 +88211 POINT(41.15742924624739 -74.09780961781318) bank88211 +88212 POINT(41.230331088265885 -74.0998766281462) bank88212 +88213 POINT(41.28345355956602 -74.94668227734165) bank88213 +88214 POINT(41.38912775930896 -74.05881447254421) bank88214 +88215 POINT(39.82559378206079 -73.46755181607541) bank88215 +88216 POINT(41.47524329719581 -74.78785974707066) bank88216 +88217 POINT(40.76455868194689 -74.3914032706052) bank88217 +88218 POINT(39.99811222151787 -74.87189422677557) bank88218 +88219 POINT(40.21514907207038 -74.76879651176043) bank88219 +88220 POINT(39.95719818291781 -74.28145147785536) bank88220 +88221 POINT(40.11098643828336 -73.57871912130801) bank88221 +88222 POINT(40.7019679537437 -73.06363359315498) bank88222 +88223 POINT(41.063983215526214 -73.97404811680389) bank88223 +88224 POINT(41.17113127684306 -74.40942396421813) bank88224 +88225 POINT(40.8528519952766 -74.80455688733838) bank88225 +88226 POINT(40.281907969169396 -73.23820985149439) bank88226 +88227 POINT(41.202261346183796 -73.5197619086027) bank88227 +88228 POINT(40.29488080594189 -73.83600700392208) bank88228 +88229 POINT(40.78310723474651 -73.98845677619535) bank88229 +88230 POINT(41.18056106566248 -73.04867863820726) bank88230 +88231 POINT(40.90817653471055 -74.3170899507112) bank88231 +88232 POINT(40.34326323792144 -73.96002838405317) bank88232 +88233 POINT(40.88549168300059 -73.10657046453522) bank88233 +88234 POINT(40.01433617314885 -73.84531297950684) bank88234 +88235 POINT(39.76309629697713 -73.9162545777319) bank88235 +88236 POINT(41.68316403621243 -73.8425265936494) bank88236 +88237 POINT(40.93966113495902 -74.14028966097287) bank88237 +88238 POINT(41.21751630443465 -74.09933551025885) bank88238 +88239 POINT(40.82579199437183 -73.60227132135542) bank88239 +88240 POINT(40.36240625427434 -74.45130044654132) bank88240 +88241 POINT(40.873657325965844 -74.7293196759617) bank88241 +88242 POINT(40.54395617639038 -73.33290201960716) bank88242 +88243 POINT(41.367815238824356 -74.49993398404409) bank88243 +88244 POINT(41.24733370466497 -74.94135761780241) bank88244 +88245 POINT(40.895563519380524 -73.71761127450874) bank88245 +88246 POINT(41.63916680011277 -73.32671167164959) bank88246 +88247 POINT(39.96716194307684 -73.05907265149732) bank88247 +88248 POINT(41.47351330879795 -73.60914986563037) bank88248 +88249 POINT(41.045299408859684 -74.42128017967069) bank88249 +88250 POINT(41.410122780870616 -73.08537260788015) bank88250 +88251 POINT(39.731147771670805 -73.14375921518558) bank88251 +88252 POINT(40.33069843536211 -73.71680226602344) bank88252 +88253 POINT(41.207547030433894 -73.98549489279995) bank88253 +88254 POINT(39.81434176234822 -73.55474724224163) bank88254 +88255 POINT(39.88714167255014 -74.27002130055796) bank88255 +88256 POINT(40.68283423771833 -73.84685001411343) bank88256 +88257 POINT(39.870388111717816 -73.0403582751555) bank88257 +88258 POINT(41.043176466438474 -73.61631359315365) bank88258 +88259 POINT(40.97280337687249 -74.00558314560897) bank88259 +88260 POINT(40.690732318839665 -73.74199108575516) bank88260 +88261 POINT(39.97744981755143 -74.85235900792821) bank88261 +88262 POINT(41.6890552611432 -73.56715195307751) bank88262 +88263 POINT(40.88917021371655 -74.64751478111444) bank88263 +88264 POINT(39.79523935357871 -73.2304141716228) bank88264 +88265 POINT(40.73234293381169 -74.21031832111497) bank88265 +88266 POINT(40.919432526671905 -74.39196083362862) bank88266 +88267 POINT(41.4996586055036 -74.5884711108503) bank88267 +88268 POINT(40.45838420932801 -73.24131623922455) bank88268 +88269 POINT(40.614738459968066 -74.6840780158056) bank88269 +88270 POINT(39.74942581616895 -74.49829501063817) bank88270 +88271 POINT(40.19866298866732 -74.28515017577253) bank88271 +88272 POINT(40.6125665689405 -73.33252619590921) bank88272 +88273 POINT(39.94672415898127 -73.38310691092494) bank88273 +88274 POINT(40.808404638934356 -74.33518098308582) bank88274 +88275 POINT(39.798079811648385 -73.21918022482984) bank88275 +88276 POINT(41.24088984661523 -73.55408373570243) bank88276 +88277 POINT(40.63900922192039 -74.88720668174393) bank88277 +88278 POINT(40.941858428451816 -73.47582956020099) bank88278 +88279 POINT(41.260085517015895 -74.28234802793894) bank88279 +88280 POINT(41.38998388610161 -74.78424050470201) bank88280 +88281 POINT(39.98859183151883 -73.94783286392484) bank88281 +88282 POINT(41.62293402917364 -74.36673656487113) bank88282 +88283 POINT(41.37500501909985 -73.29346934237572) bank88283 +88284 POINT(41.16962543927518 -73.68193048971607) bank88284 +88285 POINT(40.625294598394895 -74.7500216780963) bank88285 +88286 POINT(41.501393942063345 -74.90892293111393) bank88286 +88287 POINT(41.4270171150326 -73.0646435767933) bank88287 +88288 POINT(40.02462153149702 -74.294323929194) bank88288 +88289 POINT(39.727824713287006 -73.06478263896797) bank88289 +88290 POINT(40.267956315219536 -73.74799225852038) bank88290 +88291 POINT(40.9331941371632 -73.75552515989034) bank88291 +88292 POINT(41.50338169203351 -73.09624555318983) bank88292 +88293 POINT(39.94191969758121 -74.10527094102696) bank88293 +88294 POINT(41.03412017626089 -74.57582472129143) bank88294 +88295 POINT(39.77215824289041 -73.87373071136739) bank88295 +88296 POINT(40.76252248788234 -74.23108852008191) bank88296 +88297 POINT(40.60037401513719 -74.52131346592846) bank88297 +88298 POINT(40.3021281142421 -74.03775745340043) bank88298 +88299 POINT(40.44692706175203 -74.10318700631738) bank88299 +88300 POINT(39.8194327150473 -73.86453883169622) bank88300 +88301 POINT(39.95967168926145 -74.71461822354834) bank88301 +88302 POINT(40.374472252925614 -74.62204332835002) bank88302 +88303 POINT(41.076728485911026 -73.41897570298754) bank88303 +88304 POINT(41.213218060309224 -74.51038712927345) bank88304 +88305 POINT(39.79409495667088 -73.99955142136882) bank88305 +88306 POINT(40.814862311459116 -73.61326403691459) bank88306 +88307 POINT(40.48796029126265 -73.69294108448217) bank88307 +88308 POINT(40.83491184761786 -74.77983041442013) bank88308 +88309 POINT(40.665526536974035 -74.34653320652613) bank88309 +88310 POINT(40.785470425567794 -73.77563570358407) bank88310 +88311 POINT(39.842809789484036 -74.34403861067439) bank88311 +88312 POINT(41.178995902905925 -74.39396440667659) bank88312 +88313 POINT(39.813929064851735 -74.39645080891201) bank88313 +88314 POINT(40.10204638955943 -73.60325078036922) bank88314 +88315 POINT(41.69172692893682 -74.69456283344914) bank88315 +88316 POINT(39.92441433704897 -75.00376444262454) bank88316 +88317 POINT(41.015041259840984 -73.66701987629197) bank88317 +88318 POINT(40.172818749903115 -73.67723297282619) bank88318 +88319 POINT(41.65581640628247 -74.0093331275042) bank88319 +88320 POINT(40.640178807584405 -74.29712451090786) bank88320 +88321 POINT(40.231472394253075 -73.02901705147958) bank88321 +88322 POINT(41.46738779414321 -74.95577566641515) bank88322 +88323 POINT(40.73326953075879 -74.04426733543647) bank88323 +88324 POINT(40.40468582440346 -73.51215435768653) bank88324 +88325 POINT(39.74091782196755 -74.11583892239604) bank88325 +88326 POINT(39.80540728118095 -73.36155378274825) bank88326 +88327 POINT(41.148202561974536 -74.2262230732063) bank88327 +88328 POINT(40.98842210360419 -73.71557994465935) bank88328 +88329 POINT(39.7754613400296 -73.6121028967456) bank88329 +88330 POINT(39.91173836108203 -73.44941737928399) bank88330 +88331 POINT(40.91287730412786 -73.31621228298793) bank88331 +88332 POINT(40.250708747638214 -73.79511946013399) bank88332 +88333 POINT(40.78783286151422 -74.19343109306271) bank88333 +88334 POINT(40.98875254182223 -73.37215051205165) bank88334 +88335 POINT(41.654568710818324 -74.01937001351956) bank88335 +88336 POINT(40.20058080337491 -73.78877838955371) bank88336 +88337 POINT(40.18759582878867 -73.87607277078516) bank88337 +88338 POINT(40.36692842890172 -74.6485579858837) bank88338 +88339 POINT(41.303672258494665 -73.2419781035502) bank88339 +88340 POINT(41.53559745935084 -73.81819297141129) bank88340 +88341 POINT(39.88413021905389 -74.47506751062924) bank88341 +88342 POINT(41.410863070224586 -73.55767566066045) bank88342 +88343 POINT(40.59273215807493 -73.51441418388438) bank88343 +88344 POINT(40.20017759456578 -74.90543291644467) bank88344 +88345 POINT(41.62619212713036 -74.1527529921042) bank88345 +88346 POINT(40.29762172680421 -74.79474482876793) bank88346 +88347 POINT(40.17966073158572 -73.86384021320178) bank88347 +88348 POINT(40.91857764708602 -74.87773155313697) bank88348 +88349 POINT(41.481327354475326 -74.73920900204426) bank88349 +88350 POINT(40.063013433697684 -74.17164278720675) bank88350 +88351 POINT(40.07079603478464 -74.96307677048847) bank88351 +88352 POINT(39.953602771291834 -74.64059865544286) bank88352 +88353 POINT(40.32514668130355 -73.31465188574893) bank88353 +88354 POINT(40.3447558282743 -73.99729714764193) bank88354 +88355 POINT(40.38257946807581 -74.99358598793924) bank88355 +88356 POINT(41.29842380146579 -74.95038150173865) bank88356 +88357 POINT(41.07435092633163 -73.1442273111346) bank88357 +88358 POINT(40.95386470659063 -74.03847806912286) bank88358 +88359 POINT(41.1166896021058 -73.24668214211053) bank88359 +88360 POINT(39.91337496598878 -73.30871498077211) bank88360 +88361 POINT(41.2963002415952 -73.1151651789328) bank88361 +88362 POINT(41.634799431008304 -74.35885674870586) bank88362 +88363 POINT(40.11548066476375 -73.80986020889719) bank88363 +88364 POINT(40.689633967531755 -73.7949624322525) bank88364 +88365 POINT(41.17860584648055 -74.7225427101867) bank88365 +88366 POINT(41.669412595745655 -73.4734518298845) bank88366 +88367 POINT(40.6682533941474 -74.99031169755243) bank88367 +88368 POINT(40.07996850163879 -73.29615628704143) bank88368 +88369 POINT(40.482555194356785 -73.04278576182591) bank88369 +88370 POINT(40.9549352772863 -73.80588172576152) bank88370 +88371 POINT(41.198810060974225 -74.0642865289459) bank88371 +88372 POINT(40.18201226734882 -73.30298925021988) bank88372 +88373 POINT(40.71902362210626 -73.07729445864574) bank88373 +88374 POINT(40.88886935512224 -73.9394875309301) bank88374 +88375 POINT(40.41180670379655 -74.15873973102028) bank88375 +88376 POINT(39.8656608090537 -74.0839050176223) bank88376 +88377 POINT(41.440490011829226 -74.39841233550642) bank88377 +88378 POINT(39.824519781227586 -73.61065912497376) bank88378 +88379 POINT(41.07170878071262 -74.33490765496752) bank88379 +88380 POINT(41.59508670103083 -74.00242201706945) bank88380 +88381 POINT(39.91392456938534 -74.55765996742281) bank88381 +88382 POINT(40.479231525430045 -73.0476077566599) bank88382 +88383 POINT(39.841398975273194 -74.94017791968244) bank88383 +88384 POINT(41.43952363165656 -74.11119693183714) bank88384 +88385 POINT(41.16358408255168 -74.98499399207661) bank88385 +88386 POINT(40.08449349013711 -74.28453610044247) bank88386 +88387 POINT(39.80492360779294 -74.9761837161891) bank88387 +88388 POINT(41.15229072517691 -73.67072760988789) bank88388 +88389 POINT(39.8377819575823 -73.3086963623882) bank88389 +88390 POINT(41.58408139468122 -73.04213807289243) bank88390 +88391 POINT(40.97710959100955 -74.77080568189557) bank88391 +88392 POINT(40.88396252758115 -74.07715613500092) bank88392 +88393 POINT(41.45498644755353 -74.46595958602066) bank88393 +88394 POINT(40.3860965173701 -74.89788372645184) bank88394 +88395 POINT(41.67054664651858 -73.89792959945142) bank88395 +88396 POINT(40.733491287194354 -73.19589591974452) bank88396 +88397 POINT(41.64385841357621 -73.7130229745928) bank88397 +88398 POINT(40.78171681318016 -73.16484648938754) bank88398 +88399 POINT(41.54873291481757 -74.11707448707678) bank88399 +88400 POINT(41.26373944925153 -74.30201215247041) bank88400 +88401 POINT(40.53359023082963 -73.33558370075077) bank88401 +88402 POINT(40.345317233051325 -73.27344566621734) bank88402 +88403 POINT(41.36383831076001 -74.29469883852241) bank88403 +88404 POINT(41.03229458829321 -73.1241772555014) bank88404 +88405 POINT(40.121764154972965 -74.6026244319278) bank88405 +88406 POINT(39.79008497882563 -73.17012187256637) bank88406 +88407 POINT(41.194942833211115 -73.58921609516366) bank88407 +88408 POINT(40.286747818989355 -74.79969841771565) bank88408 +88409 POINT(41.15688967718797 -73.15627739361894) bank88409 +88410 POINT(41.251994380075395 -73.22471569275942) bank88410 +88411 POINT(40.78800966220159 -74.74098909298384) bank88411 +88412 POINT(40.46203461920211 -73.33431506911903) bank88412 +88413 POINT(41.082717868041314 -74.5142899660421) bank88413 +88414 POINT(40.83783202630776 -75.00121068149737) bank88414 +88415 POINT(41.073082322103964 -73.8160261635317) bank88415 +88416 POINT(41.0495655770436 -73.49369339763771) bank88416 +88417 POINT(40.299610626634575 -73.72065320474555) bank88417 +88418 POINT(40.39586866036939 -73.89259679477522) bank88418 +88419 POINT(40.202975371521894 -74.30206528051049) bank88419 +88420 POINT(40.83640626689148 -73.01201791606691) bank88420 +88421 POINT(40.11074716231523 -73.91126051001854) bank88421 +88422 POINT(40.52647375877494 -74.9500873377112) bank88422 +88423 POINT(40.53198996396145 -74.80572649540854) bank88423 +88424 POINT(41.53660910797898 -74.18985241083931) bank88424 +88425 POINT(40.318373682181615 -73.42965756655649) bank88425 +88426 POINT(41.63731430012659 -74.18092773151655) bank88426 +88427 POINT(41.50483209286162 -74.80720186131359) bank88427 +88428 POINT(39.97954833908514 -73.43827151802012) bank88428 +88429 POINT(41.151489003556705 -74.87083204326721) bank88429 +88430 POINT(40.1050303579665 -74.6686845648874) bank88430 +88431 POINT(41.697807506231186 -74.30500466453523) bank88431 +88432 POINT(39.76501547991253 -73.20032192141979) bank88432 +88433 POINT(40.3483427002065 -74.07648780987934) bank88433 +88434 POINT(40.51369675911829 -73.03363576783273) bank88434 +88435 POINT(40.828875890024214 -74.7692156595244) bank88435 +88436 POINT(39.88015415541736 -73.5225222709307) bank88436 +88437 POINT(40.34521414041592 -74.36061897730598) bank88437 +88438 POINT(40.96366524133866 -73.14463465129238) bank88438 +88439 POINT(41.623099483868295 -73.1823474101112) bank88439 +88440 POINT(41.28039950098209 -73.59660801605773) bank88440 +88441 POINT(40.280071447549886 -74.55361435573644) bank88441 +88442 POINT(40.71973662674645 -74.55695129778772) bank88442 +88443 POINT(40.047483981980825 -74.76280303653617) bank88443 +88444 POINT(41.63630355245101 -74.38600437608618) bank88444 +88445 POINT(41.228503213240245 -74.76031804599282) bank88445 +88446 POINT(40.75415942561964 -74.23276296062566) bank88446 +88447 POINT(40.45534473253487 -73.16629917656805) bank88447 +88448 POINT(40.095224319271566 -73.03607807437297) bank88448 +88449 POINT(40.82092319249338 -74.57706921546891) bank88449 +88450 POINT(40.28743371050102 -73.7462210896914) bank88450 +88451 POINT(40.19122877804815 -73.39333084787792) bank88451 +88452 POINT(40.13081871559264 -73.1434491489867) bank88452 +88453 POINT(40.891552110604735 -73.9421881100443) bank88453 +88454 POINT(40.38651051967156 -74.53191769442299) bank88454 +88455 POINT(40.727226516734994 -73.71131537694347) bank88455 +88456 POINT(41.151850187006396 -73.60440615973977) bank88456 +88457 POINT(40.993745459495855 -73.02254781701554) bank88457 +88458 POINT(41.19417474255721 -73.28730676397788) bank88458 +88459 POINT(41.01333050064807 -74.23260503656634) bank88459 +88460 POINT(41.02229474340371 -73.49247580385241) bank88460 +88461 POINT(40.5821847781121 -74.2419804865178) bank88461 +88462 POINT(41.192735624708945 -73.59532402386704) bank88462 +88463 POINT(41.652480815375114 -73.35646275942267) bank88463 +88464 POINT(39.72383380629742 -73.80534603572539) bank88464 +88465 POINT(41.19392870852037 -74.8277015334455) bank88465 +88466 POINT(40.72987835598314 -73.54181881709279) bank88466 +88467 POINT(40.58744963727396 -74.72503638211904) bank88467 +88468 POINT(40.63916806918312 -74.80999165662318) bank88468 +88469 POINT(39.86824116243699 -73.69916084428156) bank88469 +88470 POINT(41.50707347036897 -74.20577276313082) bank88470 +88471 POINT(40.78633176932105 -73.75956573246197) bank88471 +88472 POINT(40.919964365912406 -73.64350459234684) bank88472 +88473 POINT(41.65681168505515 -74.85109230872992) bank88473 +88474 POINT(40.359775638462715 -74.948282837963) bank88474 +88475 POINT(40.26380580548894 -73.32936809198841) bank88475 +88476 POINT(41.36607652284791 -74.00226361466882) bank88476 +88477 POINT(41.51824986092432 -74.05068679766218) bank88477 +88478 POINT(40.52149071947316 -73.35570748588532) bank88478 +88479 POINT(41.555677565417454 -73.98896821745662) bank88479 +88480 POINT(39.8583566818901 -73.44170867616683) bank88480 +88481 POINT(41.17424363164241 -74.91600649733748) bank88481 +88482 POINT(39.90858185588084 -74.08963437837808) bank88482 +88483 POINT(40.50050267179311 -73.59270194027104) bank88483 +88484 POINT(40.293330855185985 -73.35204429301515) bank88484 +88485 POINT(40.18855435578864 -74.17025166243998) bank88485 +88486 POINT(40.98113037912029 -73.50808320035374) bank88486 +88487 POINT(39.84765853224272 -74.3528629392284) bank88487 +88488 POINT(41.087149189425 -74.1093664174279) bank88488 +88489 POINT(40.780205041416906 -73.67633067230959) bank88489 +88490 POINT(40.13145752808408 -74.57577890635491) bank88490 +88491 POINT(41.01356475457783 -73.89155127304602) bank88491 +88492 POINT(40.972345123805994 -74.6233759552438) bank88492 +88493 POINT(41.4241167076466 -74.38381603071905) bank88493 +88494 POINT(40.94691277951682 -74.71197854855369) bank88494 +88495 POINT(40.18879908575088 -74.03219473046062) bank88495 +88496 POINT(40.913586141437456 -73.7313137857598) bank88496 +88497 POINT(39.72233236313718 -73.19113829525153) bank88497 +88498 POINT(40.57928069106421 -73.16475868882242) bank88498 +88499 POINT(39.86642810036644 -73.49266539382471) bank88499 +88500 POINT(40.177074674139305 -74.07576238418696) bank88500 +88501 POINT(41.452915325372054 -74.66750088615471) bank88501 +88502 POINT(41.13864620368037 -73.22143324138474) bank88502 +88503 POINT(41.26304544086597 -73.14944539464221) bank88503 +88504 POINT(41.238729070627 -73.24029953498979) bank88504 +88505 POINT(40.70428616829229 -73.58957803316406) bank88505 +88506 POINT(41.441458348455065 -73.50691659648268) bank88506 +88507 POINT(41.487517030713484 -73.74381480460998) bank88507 +88508 POINT(41.685282542757506 -74.86562704800056) bank88508 +88509 POINT(40.21052282948106 -73.44817442764598) bank88509 +88510 POINT(41.61638238151123 -73.40706319388866) bank88510 +88511 POINT(41.063519238195745 -73.20969253355413) bank88511 +88512 POINT(39.930307040395114 -73.1524450577132) bank88512 +88513 POINT(40.280575141727496 -74.32424652677905) bank88513 +88514 POINT(41.08439415189556 -73.18314386494815) bank88514 +88515 POINT(41.04575555027373 -74.16041633907246) bank88515 +88516 POINT(40.884081729185446 -73.29017459576886) bank88516 +88517 POINT(40.68303834689678 -74.27755385646012) bank88517 +88518 POINT(39.89507535369668 -73.64406806882847) bank88518 +88519 POINT(39.96762200069475 -74.62781131431588) bank88519 +88520 POINT(40.58369923849978 -74.83930828515379) bank88520 +88521 POINT(40.3924789304709 -73.20481427893377) bank88521 +88522 POINT(41.511635473313405 -73.79568574745208) bank88522 +88523 POINT(41.48167789809876 -74.67337708573905) bank88523 +88524 POINT(40.48996212931166 -74.39326648428899) bank88524 +88525 POINT(41.34983872704452 -73.58526494482388) bank88525 +88526 POINT(41.13689772345412 -74.56925308947864) bank88526 +88527 POINT(40.1736606924006 -73.40627733888613) bank88527 +88528 POINT(41.26916661120382 -74.1897515373654) bank88528 +88529 POINT(40.88479166504389 -74.31537770312866) bank88529 +88530 POINT(41.28543486315032 -73.16077447954254) bank88530 +88531 POINT(41.62611800017235 -74.65074266211037) bank88531 +88532 POINT(41.239089134937196 -74.4002493763783) bank88532 +88533 POINT(40.16987007250956 -74.7493863207279) bank88533 +88534 POINT(40.81564673129912 -73.03301124651901) bank88534 +88535 POINT(40.890209939176515 -73.76993420702459) bank88535 +88536 POINT(40.60742606046335 -73.97277162622315) bank88536 +88537 POINT(39.99364470135949 -73.69190109854863) bank88537 +88538 POINT(40.026471783325015 -73.62897393889268) bank88538 +88539 POINT(40.871141530689926 -73.16593219300272) bank88539 +88540 POINT(41.1409397051212 -74.33693949633545) bank88540 +88541 POINT(40.312053144835374 -74.0962592911402) bank88541 +88542 POINT(41.19029988860227 -73.87028381482004) bank88542 +88543 POINT(40.511878976046646 -73.06731980264131) bank88543 +88544 POINT(39.925735670199884 -73.28186236917448) bank88544 +88545 POINT(40.972130692033595 -74.4574136309822) bank88545 +88546 POINT(39.91879541828433 -74.46641460517871) bank88546 +88547 POINT(41.38746928464897 -74.72987126850977) bank88547 +88548 POINT(39.74707786234475 -74.94740538023152) bank88548 +88549 POINT(41.66560957334033 -74.4419450811345) bank88549 +88550 POINT(40.526012271947295 -73.9287623156406) bank88550 +88551 POINT(40.18035356756547 -74.44556060676399) bank88551 +88552 POINT(40.135409367591606 -73.3279502076842) bank88552 +88553 POINT(40.287570194894215 -73.80720732212879) bank88553 +88554 POINT(41.539611309048084 -74.49117673388767) bank88554 +88555 POINT(40.633484152509126 -73.24600551376471) bank88555 +88556 POINT(40.828951696991204 -73.90542431978993) bank88556 +88557 POINT(40.105490875614336 -74.31787718237578) bank88557 +88558 POINT(41.631120495793056 -73.48765317698663) bank88558 +88559 POINT(41.07152047246215 -73.3888015341596) bank88559 +88560 POINT(40.29255964775909 -73.06283896252732) bank88560 +88561 POINT(40.20062796439715 -74.83836267689757) bank88561 +88562 POINT(41.05545838257257 -74.62483296544666) bank88562 +88563 POINT(41.56470983109168 -74.97808180714617) bank88563 +88564 POINT(41.235999439968964 -74.48761216994284) bank88564 +88565 POINT(40.58606777971277 -73.02271999518888) bank88565 +88566 POINT(40.0399026708124 -74.15837230279662) bank88566 +88567 POINT(40.69620590577865 -74.61633169892265) bank88567 +88568 POINT(40.33951638883315 -74.85699042892901) bank88568 +88569 POINT(39.90708380815677 -73.89162338637) bank88569 +88570 POINT(40.33726877692463 -74.57533592502416) bank88570 +88571 POINT(40.497616302323586 -73.41384345087509) bank88571 +88572 POINT(39.74659196611897 -73.59566060538913) bank88572 +88573 POINT(40.08830286725617 -73.89286453490992) bank88573 +88574 POINT(40.837920563503666 -73.61156120942373) bank88574 +88575 POINT(40.84168265561235 -73.18333563098719) bank88575 +88576 POINT(40.689972042151425 -74.78990406036291) bank88576 +88577 POINT(41.14725494129416 -73.47453467458942) bank88577 +88578 POINT(41.28160613812238 -74.35773323982303) bank88578 +88579 POINT(39.79161323419298 -74.98544180953382) bank88579 +88580 POINT(39.9292045564426 -74.70927015904215) bank88580 +88581 POINT(40.84801689424442 -74.10195798526216) bank88581 +88582 POINT(41.03195156414225 -73.35879883819361) bank88582 +88583 POINT(40.07383834756868 -74.89054237185958) bank88583 +88584 POINT(40.13396092903219 -73.03781835326437) bank88584 +88585 POINT(41.341336016948155 -74.78322705065717) bank88585 +88586 POINT(40.611965287448605 -74.77502912969437) bank88586 +88587 POINT(39.8326727470137 -74.8361587923349) bank88587 +88588 POINT(40.29041514231494 -73.82541495494611) bank88588 +88589 POINT(40.362008073709355 -73.12855294682159) bank88589 +88590 POINT(40.37159789660606 -73.1011092992143) bank88590 +88591 POINT(40.08326133119463 -74.58761834991273) bank88591 +88592 POINT(40.78845258188381 -73.04232356366927) bank88592 +88593 POINT(41.47400854738158 -73.93443701050748) bank88593 +88594 POINT(40.6872214296881 -74.80822424821409) bank88594 +88595 POINT(40.61772912226298 -73.10126585466426) bank88595 +88596 POINT(40.588452414538175 -74.37010186774927) bank88596 +88597 POINT(40.88633964604369 -74.56015442095548) bank88597 +88598 POINT(40.90742116594654 -74.76176691532523) bank88598 +88599 POINT(40.72158541717306 -73.11238402554461) bank88599 +88600 POINT(40.418474995040484 -74.39981962838277) bank88600 +88601 POINT(39.940830868744314 -74.61538845208973) bank88601 +88602 POINT(41.40463634065341 -74.82477219559203) bank88602 +88603 POINT(41.021902382311055 -73.06942735851042) bank88603 +88604 POINT(40.09771125316915 -74.51965248047242) bank88604 +88605 POINT(40.24993665809957 -73.7933226114371) bank88605 +88606 POINT(40.89869998745996 -73.46636021368832) bank88606 +88607 POINT(40.018219734685985 -74.13123507493296) bank88607 +88608 POINT(40.448417555447286 -73.39877686809842) bank88608 +88609 POINT(41.455501670263814 -73.72923497089924) bank88609 +88610 POINT(41.693365685903196 -74.57861946386339) bank88610 +88611 POINT(39.78680116728098 -73.35179038165387) bank88611 +88612 POINT(41.67309111819315 -73.19525708088192) bank88612 +88613 POINT(40.7314485121812 -74.91093848520006) bank88613 +88614 POINT(41.4669638949465 -74.17590184483534) bank88614 +88615 POINT(41.11852089221447 -74.9990046096586) bank88615 +88616 POINT(40.51050915168237 -73.64983612095962) bank88616 +88617 POINT(40.973528940994136 -73.21327779831182) bank88617 +88618 POINT(39.85011506612557 -73.09636572406092) bank88618 +88619 POINT(40.12066618192998 -74.40118715106134) bank88619 +88620 POINT(39.870936217188536 -73.52644421745644) bank88620 +88621 POINT(41.289545939014815 -73.36809743115425) bank88621 +88622 POINT(40.459322498065205 -73.7410052471881) bank88622 +88623 POINT(40.69705062764638 -74.357740753995) bank88623 +88624 POINT(40.500589527212746 -74.62151447716958) bank88624 +88625 POINT(40.29465536853554 -74.58223392384186) bank88625 +88626 POINT(41.308231776524785 -74.4262792853151) bank88626 +88627 POINT(40.16410868338286 -73.80211278772113) bank88627 +88628 POINT(40.33537156167036 -74.09208064855267) bank88628 +88629 POINT(40.40431152994612 -73.9355655040276) bank88629 +88630 POINT(40.90865410306448 -73.35872920346857) bank88630 +88631 POINT(40.72138248482818 -74.41937525854365) bank88631 +88632 POINT(41.09993446689675 -74.16382278732362) bank88632 +88633 POINT(40.486049318266055 -73.12820678632835) bank88633 +88634 POINT(41.089475441087025 -73.89251311839533) bank88634 +88635 POINT(40.28912356609938 -73.70097446481813) bank88635 +88636 POINT(40.565930911344815 -74.91058256980881) bank88636 +88637 POINT(40.98716017835755 -73.55363551188415) bank88637 +88638 POINT(39.83837546902941 -74.02552047762302) bank88638 +88639 POINT(39.75096167624994 -74.43349567904973) bank88639 +88640 POINT(40.52565444447892 -74.6490150788744) bank88640 +88641 POINT(41.25694202608243 -73.2298313937227) bank88641 +88642 POINT(39.78861549658734 -73.81186665199395) bank88642 +88643 POINT(41.67405943689467 -73.83738931387929) bank88643 +88644 POINT(39.730384610624824 -74.97159853519742) bank88644 +88645 POINT(39.90015635307645 -74.08584672310516) bank88645 +88646 POINT(39.92511268037214 -73.18867744411578) bank88646 +88647 POINT(41.17352257443181 -73.51092110306573) bank88647 +88648 POINT(40.53998633934906 -74.83858874650198) bank88648 +88649 POINT(41.48171856616331 -74.41255971395394) bank88649 +88650 POINT(40.19956583137547 -73.51830804390386) bank88650 +88651 POINT(40.63714104312727 -73.60437331877208) bank88651 +88652 POINT(40.6222177380259 -74.77367023664118) bank88652 +88653 POINT(41.1293385023139 -73.72156518659158) bank88653 +88654 POINT(41.27734679010246 -74.10216998162734) bank88654 +88655 POINT(40.31011705133315 -74.04751865128307) bank88655 +88656 POINT(41.253722588041285 -74.25543888415955) bank88656 +88657 POINT(41.02151548893028 -74.00138972873948) bank88657 +88658 POINT(41.42849944274129 -73.02766801507002) bank88658 +88659 POINT(39.71469580609531 -74.83986506636118) bank88659 +88660 POINT(41.128518177209344 -73.99681250951305) bank88660 +88661 POINT(40.3113780571572 -74.99346895024318) bank88661 +88662 POINT(40.054094718548605 -74.19338169209719) bank88662 +88663 POINT(40.019678166560546 -73.20704076063979) bank88663 +88664 POINT(40.58056875340747 -73.28177322967748) bank88664 +88665 POINT(40.00845870165594 -73.33145283040497) bank88665 +88666 POINT(40.90482764280014 -73.38053616022037) bank88666 +88667 POINT(40.000070053454934 -73.88600148075254) bank88667 +88668 POINT(39.889918096658526 -73.78440261025334) bank88668 +88669 POINT(40.084004964893005 -73.79902724765871) bank88669 +88670 POINT(40.54797217558391 -74.90310161517134) bank88670 +88671 POINT(40.45837976432732 -74.42606780351429) bank88671 +88672 POINT(40.95709565117547 -73.93440271899242) bank88672 +88673 POINT(41.626082111604994 -74.8215034200071) bank88673 +88674 POINT(40.91661837821987 -73.34420975239203) bank88674 +88675 POINT(39.863751460859056 -74.96435341779261) bank88675 +88676 POINT(41.29460760738323 -73.00723269972059) bank88676 +88677 POINT(40.89960221608819 -73.7098612151719) bank88677 +88678 POINT(41.57418337582193 -74.19001580385695) bank88678 +88679 POINT(40.71978924667251 -73.36273788860846) bank88679 +88680 POINT(40.50298586463077 -74.87080141299731) bank88680 +88681 POINT(41.31050182102343 -73.85872176366205) bank88681 +88682 POINT(41.46930350773344 -74.97387340912806) bank88682 +88683 POINT(41.095725310398784 -73.07819988863015) bank88683 +88684 POINT(40.18422990508808 -74.91697309045857) bank88684 +88685 POINT(41.107541083348465 -73.0185437902821) bank88685 +88686 POINT(40.35483435481861 -73.73948089393636) bank88686 +88687 POINT(41.37766799950102 -74.08505662660076) bank88687 +88688 POINT(41.48276283382539 -74.7818895293356) bank88688 +88689 POINT(39.96468179825987 -74.34068531674549) bank88689 +88690 POINT(40.008940225427956 -74.93030432684844) bank88690 +88691 POINT(39.879836063091204 -74.68510306245085) bank88691 +88692 POINT(40.90822143561594 -74.17317419379759) bank88692 +88693 POINT(39.77773899060306 -74.61645239671223) bank88693 +88694 POINT(41.41959329891422 -74.5956734107474) bank88694 +88695 POINT(40.08520729077906 -73.68910084335629) bank88695 +88696 POINT(40.986413746038956 -74.99627946639889) bank88696 +88697 POINT(40.249682199512776 -74.64850283115759) bank88697 +88698 POINT(40.20413867457903 -73.04169365069076) bank88698 +88699 POINT(41.44831597613049 -73.74383375855543) bank88699 +88700 POINT(40.050870857926576 -74.05724885087606) bank88700 +88701 POINT(39.98606384428898 -74.2679693729872) bank88701 +88702 POINT(41.48541600224651 -73.82695948773363) bank88702 +88703 POINT(41.048000340454166 -74.44362060540337) bank88703 +88704 POINT(40.41741446312847 -74.0104133857807) bank88704 +88705 POINT(40.22686455612673 -74.20620435546446) bank88705 +88706 POINT(39.856795901094664 -74.41575510609808) bank88706 +88707 POINT(40.498813997193736 -73.8661942124552) bank88707 +88708 POINT(41.535303698802764 -73.44610377885991) bank88708 +88709 POINT(41.34848222548505 -73.40735677833041) bank88709 +88710 POINT(40.30992129436011 -73.07519212055666) bank88710 +88711 POINT(41.00931985240058 -73.17772095511663) bank88711 +88712 POINT(40.73490127617686 -73.5110733585962) bank88712 +88713 POINT(40.092055306163566 -73.04812540500947) bank88713 +88714 POINT(41.44313842952496 -73.72354744162345) bank88714 +88715 POINT(39.80993565232046 -74.48530352722187) bank88715 +88716 POINT(40.29653399296672 -74.77495158055191) bank88716 +88717 POINT(41.33604178228564 -73.29055722633858) bank88717 +88718 POINT(41.10247934127851 -74.66004972934294) bank88718 +88719 POINT(40.42218107016204 -73.25592801087492) bank88719 +88720 POINT(41.53037210252685 -74.70298939301892) bank88720 +88721 POINT(40.10309623226841 -73.49067195660496) bank88721 +88722 POINT(41.691520957219424 -73.32414574865805) bank88722 +88723 POINT(40.80095390194348 -74.30536451498325) bank88723 +88724 POINT(41.46528582614166 -74.61081792989597) bank88724 +88725 POINT(40.63877209159903 -74.61057111739157) bank88725 +88726 POINT(40.91253098082694 -74.82055091072966) bank88726 +88727 POINT(39.95129935280298 -73.3008016792972) bank88727 +88728 POINT(41.68648817823863 -73.27007392818811) bank88728 +88729 POINT(40.35438325963931 -73.81431013607555) bank88729 +88730 POINT(39.91176157728481 -74.59383197809639) bank88730 +88731 POINT(40.36112208420556 -73.05831672120011) bank88731 +88732 POINT(40.721574432205635 -74.61717272333195) bank88732 +88733 POINT(40.720198363464874 -73.83502113094544) bank88733 +88734 POINT(40.33421981887394 -73.23199757571193) bank88734 +88735 POINT(40.04339081662461 -74.31566452152678) bank88735 +88736 POINT(40.461878288223154 -74.8701698640363) bank88736 +88737 POINT(41.6730508391997 -74.09092925673885) bank88737 +88738 POINT(41.38095389011147 -74.05768986507611) bank88738 +88739 POINT(40.02004303152071 -73.41177947218023) bank88739 +88740 POINT(40.83787331944683 -74.27405579535238) bank88740 +88741 POINT(40.738464267508384 -73.88747197472576) bank88741 +88742 POINT(41.49714534699576 -73.14174498962748) bank88742 +88743 POINT(39.74271495654127 -73.41539000035328) bank88743 +88744 POINT(41.130985622171 -74.44230889783105) bank88744 +88745 POINT(40.494858822053196 -74.44550631171063) bank88745 +88746 POINT(40.376251360259154 -74.53332464179164) bank88746 +88747 POINT(41.27305483163387 -73.20858965777214) bank88747 +88748 POINT(40.63570403145364 -74.65995090969221) bank88748 +88749 POINT(41.15112831002052 -73.97506245638249) bank88749 +88750 POINT(40.936747752195835 -73.85060927064536) bank88750 +88751 POINT(40.024444563866005 -74.11845638997653) bank88751 +88752 POINT(41.55419631513389 -73.45496451121758) bank88752 +88753 POINT(40.890841189200806 -73.53116335345332) bank88753 +88754 POINT(40.83600836640631 -74.76277672345051) bank88754 +88755 POINT(41.47436529475349 -73.75042520993028) bank88755 +88756 POINT(40.01594545614232 -73.42858766299734) bank88756 +88757 POINT(40.38747665068287 -73.96951620910723) bank88757 +88758 POINT(41.12864430499691 -73.2686640931673) bank88758 +88759 POINT(40.932730442844246 -73.13114725988879) bank88759 +88760 POINT(40.82521272611025 -74.10519923627739) bank88760 +88761 POINT(40.619004421516046 -74.65811468834909) bank88761 +88762 POINT(40.94727014156101 -74.58656268606971) bank88762 +88763 POINT(40.890909559862514 -74.3280017864936) bank88763 +88764 POINT(40.76866917821161 -74.47697717570124) bank88764 +88765 POINT(40.394471611798366 -74.39272478324831) bank88765 +88766 POINT(41.13172093776246 -73.4720616570466) bank88766 +88767 POINT(41.14304788824962 -74.08280282720035) bank88767 +88768 POINT(41.583818048893725 -73.81636945701133) bank88768 +88769 POINT(41.55643779772017 -74.55995013831391) bank88769 +88770 POINT(39.9061967693724 -74.75027076797512) bank88770 +88771 POINT(41.57365358558566 -73.33160168272144) bank88771 +88772 POINT(40.32972950868964 -73.1191150971298) bank88772 +88773 POINT(40.842372599814546 -74.16645769710348) bank88773 +88774 POINT(40.64014433436208 -73.71222021534936) bank88774 +88775 POINT(40.11496942053933 -74.61282136263556) bank88775 +88776 POINT(41.62478281728127 -74.49526575896144) bank88776 +88777 POINT(40.93709000493836 -73.1353682091854) bank88777 +88778 POINT(41.62941688234367 -74.53764551475514) bank88778 +88779 POINT(41.42260809054162 -73.48779337632422) bank88779 +88780 POINT(40.83991451324005 -74.48122602046058) bank88780 +88781 POINT(41.385266500518036 -74.43132605346489) bank88781 +88782 POINT(41.13747880519694 -74.41679251898292) bank88782 +88783 POINT(41.553608702229226 -74.28699144258258) bank88783 +88784 POINT(41.63014818204584 -74.81081154940456) bank88784 +88785 POINT(41.224543439949294 -73.40103914496869) bank88785 +88786 POINT(41.08389544932976 -73.88794507425207) bank88786 +88787 POINT(40.7231159534875 -73.93611977696035) bank88787 +88788 POINT(40.63145244417464 -73.421277228517) bank88788 +88789 POINT(41.52796785851425 -74.84706826953104) bank88789 +88790 POINT(40.162524793397424 -74.94014946451415) bank88790 +88791 POINT(40.71274823787778 -74.04571706044489) bank88791 +88792 POINT(40.47281191728128 -73.65105095886679) bank88792 +88793 POINT(41.432742284068404 -74.57129962338078) bank88793 +88794 POINT(40.39830724128555 -73.83267711862203) bank88794 +88795 POINT(40.38866114472471 -73.27117522364449) bank88795 +88796 POINT(40.42213706555456 -73.87335371200236) bank88796 +88797 POINT(41.47250473588657 -74.73063439279727) bank88797 +88798 POINT(39.811976810788885 -73.09443995083474) bank88798 +88799 POINT(40.02568539827278 -74.22396891794355) bank88799 +88800 POINT(41.15572363001542 -73.40795324479681) bank88800 +88801 POINT(40.15008080152596 -73.9650573618005) bank88801 +88802 POINT(41.57797395319321 -74.73924908302897) bank88802 +88803 POINT(40.73534446337601 -73.75794923370411) bank88803 +88804 POINT(41.201492917630645 -74.02070194184466) bank88804 +88805 POINT(41.29077188547643 -73.16394942297316) bank88805 +88806 POINT(40.342195982927464 -74.2824348502295) bank88806 +88807 POINT(40.39583762506927 -73.89179932061415) bank88807 +88808 POINT(40.82164296317399 -73.70534263129407) bank88808 +88809 POINT(40.543972101066885 -73.51833685560418) bank88809 +88810 POINT(40.08736269375519 -74.22388455070364) bank88810 +88811 POINT(39.860003810159256 -74.59657977704067) bank88811 +88812 POINT(41.6194896152848 -74.18244383021327) bank88812 +88813 POINT(41.196589600462964 -74.06116839877603) bank88813 +88814 POINT(41.21669851207239 -74.40625205789557) bank88814 +88815 POINT(40.51348634726184 -74.35610312997476) bank88815 +88816 POINT(41.1128261261514 -74.44619587985511) bank88816 +88817 POINT(40.418604430223404 -74.42336189332461) bank88817 +88818 POINT(39.7947434964011 -74.17349682091073) bank88818 +88819 POINT(41.560235384891655 -73.05285004960517) bank88819 +88820 POINT(40.77105581010203 -74.52892364088336) bank88820 +88821 POINT(40.67842806154367 -73.23246598022287) bank88821 +88822 POINT(41.20791215821721 -74.10719640853108) bank88822 +88823 POINT(41.10967548264707 -73.08039296069535) bank88823 +88824 POINT(41.02634198387841 -73.27921671825192) bank88824 +88825 POINT(40.05968888165701 -73.50610167797753) bank88825 +88826 POINT(40.61118227638247 -73.97883796427155) bank88826 +88827 POINT(40.79294665687586 -73.87288979050508) bank88827 +88828 POINT(40.19413287386715 -73.54234189285224) bank88828 +88829 POINT(40.48453032886701 -73.90293238587405) bank88829 +88830 POINT(41.4654283889437 -74.43096480018193) bank88830 +88831 POINT(41.39591013588126 -73.18942930330833) bank88831 +88832 POINT(40.52428558458757 -73.98556317704988) bank88832 +88833 POINT(40.94542061969508 -74.81105969034374) bank88833 +88834 POINT(40.49218075275975 -73.82323905119085) bank88834 +88835 POINT(41.16958137522359 -73.06563405237874) bank88835 +88836 POINT(40.66678634901221 -74.33834197580838) bank88836 +88837 POINT(40.7037982244829 -74.61827712875316) bank88837 +88838 POINT(40.610385828715486 -73.91025315732708) bank88838 +88839 POINT(40.273270774531746 -73.95687864623156) bank88839 +88840 POINT(41.61260316393342 -74.46699849885313) bank88840 +88841 POINT(39.76118625259221 -74.34999667562366) bank88841 +88842 POINT(41.102983144954315 -74.05225340035541) bank88842 +88843 POINT(40.209570963570414 -73.95995067977807) bank88843 +88844 POINT(40.30219348850491 -73.56214868395656) bank88844 +88845 POINT(39.932249908202245 -73.04165740462777) bank88845 +88846 POINT(40.85098734410405 -73.59409845652227) bank88846 +88847 POINT(40.520814101789135 -74.01504940757233) bank88847 +88848 POINT(40.390215370550735 -73.0446626442347) bank88848 +88849 POINT(40.49384334117244 -73.90081406363177) bank88849 +88850 POINT(40.35596422595444 -73.19589329565869) bank88850 +88851 POINT(41.34832653648973 -73.55548991592082) bank88851 +88852 POINT(40.657375016903046 -74.63011820280926) bank88852 +88853 POINT(40.627132271840196 -74.65924521292501) bank88853 +88854 POINT(40.8042817938805 -73.58551389050243) bank88854 +88855 POINT(40.2523091632915 -74.54804988004858) bank88855 +88856 POINT(41.437295416964986 -75.0047412516543) bank88856 +88857 POINT(41.66670646309895 -74.97734490794188) bank88857 +88858 POINT(40.099267156784485 -73.60985799891041) bank88858 +88859 POINT(40.795538424054136 -73.12011220539071) bank88859 +88860 POINT(40.96844644959525 -74.57034835760771) bank88860 +88861 POINT(41.67421600834169 -73.27424767133) bank88861 +88862 POINT(40.798113446612355 -73.90605206794875) bank88862 +88863 POINT(40.944508110054066 -74.58969722986173) bank88863 +88864 POINT(41.205819924801865 -74.96649535400807) bank88864 +88865 POINT(40.70902376764908 -73.3084466347682) bank88865 +88866 POINT(40.82189694126906 -74.12168957952086) bank88866 +88867 POINT(41.574895166708686 -73.72306783448933) bank88867 +88868 POINT(40.50853846731793 -74.2875192833364) bank88868 +88869 POINT(41.51523572672514 -73.94568977118055) bank88869 +88870 POINT(41.21205622531867 -74.37215306079) bank88870 +88871 POINT(40.85667865952556 -73.94546034011454) bank88871 +88872 POINT(41.652076643003234 -73.45696908971074) bank88872 +88873 POINT(39.78387921072964 -73.29034454364988) bank88873 +88874 POINT(39.83425061333468 -74.46387806484832) bank88874 +88875 POINT(40.5511177701751 -74.08070221249388) bank88875 +88876 POINT(40.27645013971541 -74.3161211385161) bank88876 +88877 POINT(41.08854049475619 -74.73078735517967) bank88877 +88878 POINT(40.93327870501794 -74.64715715937452) bank88878 +88879 POINT(41.02484310768614 -73.77809478485057) bank88879 +88880 POINT(41.06349065871732 -73.73109634760068) bank88880 +88881 POINT(41.207856863079705 -74.87270383687864) bank88881 +88882 POINT(40.03281460607103 -74.7066066323378) bank88882 +88883 POINT(40.22695301174533 -73.46837711260656) bank88883 +88884 POINT(41.38898622671656 -74.71447990486776) bank88884 +88885 POINT(41.611801133299615 -74.84662982521482) bank88885 +88886 POINT(40.792023844955615 -73.55079977075276) bank88886 +88887 POINT(41.62064952263643 -74.50108623817388) bank88887 +88888 POINT(39.96662210505051 -74.8921753700729) bank88888 +88889 POINT(40.85523522830642 -74.32927849625581) bank88889 +88890 POINT(41.42675491287801 -73.61603940932028) bank88890 +88891 POINT(40.087396289857956 -74.96308085112808) bank88891 +88892 POINT(40.44557619528512 -75.00295903574029) bank88892 +88893 POINT(41.11923048315877 -74.55052979290706) bank88893 +88894 POINT(39.81324301508804 -73.70586656562396) bank88894 +88895 POINT(40.48702904588774 -73.24194328434365) bank88895 +88896 POINT(39.750145067151664 -73.24309410426045) bank88896 +88897 POINT(41.19357313511363 -74.73267157823724) bank88897 +88898 POINT(40.19199106909111 -74.58793691889034) bank88898 +88899 POINT(39.72333431711655 -73.34509260759009) bank88899 +88900 POINT(39.91859797885766 -73.03974435406732) bank88900 +88901 POINT(41.349092670671084 -74.17558975353192) bank88901 +88902 POINT(40.59133419378333 -74.50140653987545) bank88902 +88903 POINT(40.84566924521513 -73.03447233992672) bank88903 +88904 POINT(40.693160566734655 -74.9249197679845) bank88904 +88905 POINT(41.63053258434023 -74.206265212766) bank88905 +88906 POINT(41.42908631530057 -73.54901789105224) bank88906 +88907 POINT(39.95423333726885 -74.51427868134414) bank88907 +88908 POINT(39.85283334419843 -74.3081718886732) bank88908 +88909 POINT(40.11928798316582 -74.34992966432213) bank88909 +88910 POINT(40.19854804238315 -73.03205803843153) bank88910 +88911 POINT(41.24907944788872 -74.51037282182878) bank88911 +88912 POINT(40.940191931475525 -73.54100844137298) bank88912 +88913 POINT(40.11234173660211 -73.53469446979179) bank88913 +88914 POINT(39.7651753948399 -74.65646045791885) bank88914 +88915 POINT(39.7549305896816 -73.78486979928408) bank88915 +88916 POINT(40.96440859801502 -74.47637024977065) bank88916 +88917 POINT(41.65843722695455 -74.11657473626228) bank88917 +88918 POINT(41.666761285934804 -73.64334821362858) bank88918 +88919 POINT(41.27702431868525 -74.72295822188929) bank88919 +88920 POINT(41.454870836247075 -75.00085816028691) bank88920 +88921 POINT(40.03160427860136 -73.0619534203216) bank88921 +88922 POINT(41.192931679447064 -74.30242556477387) bank88922 +88923 POINT(41.04611818584763 -73.65602699405811) bank88923 +88924 POINT(40.056536754191704 -73.78421518683076) bank88924 +88925 POINT(40.29657121267844 -74.90508369974256) bank88925 +88926 POINT(40.1531609844599 -74.18309007465747) bank88926 +88927 POINT(41.022371739593936 -73.0723945924906) bank88927 +88928 POINT(41.699786564914746 -74.31768336574424) bank88928 +88929 POINT(40.521935055914135 -74.17568296376996) bank88929 +88930 POINT(40.93192826924545 -73.10404280125653) bank88930 +88931 POINT(40.777205020753435 -73.75078858167215) bank88931 +88932 POINT(41.54049791650858 -74.93580142396637) bank88932 +88933 POINT(40.99110151489362 -73.3774829681415) bank88933 +88934 POINT(40.69643826480392 -73.14949408269419) bank88934 +88935 POINT(41.35741961793218 -73.12847324927515) bank88935 +88936 POINT(40.781493726265005 -74.12900875972146) bank88936 +88937 POINT(39.969094680707535 -74.86563150376264) bank88937 +88938 POINT(40.40330702260136 -74.98716919553577) bank88938 +88939 POINT(40.960550782754815 -74.20460256396092) bank88939 +88940 POINT(41.506913842280355 -74.80382100381857) bank88940 +88941 POINT(41.5338139841824 -73.48625140375962) bank88941 +88942 POINT(40.45086547065228 -74.3438367027897) bank88942 +88943 POINT(41.63507603171741 -73.23116312224741) bank88943 +88944 POINT(40.336320219799745 -73.97924204974875) bank88944 +88945 POINT(41.25403134318345 -74.37789769899582) bank88945 +88946 POINT(41.62346230666681 -73.31508208856525) bank88946 +88947 POINT(40.54190351333044 -73.52746142347992) bank88947 +88948 POINT(40.43439990618125 -73.22008503776206) bank88948 +88949 POINT(41.046798014371824 -73.97826732771459) bank88949 +88950 POINT(41.094908224716676 -73.06176429026831) bank88950 +88951 POINT(41.47585182126087 -73.09991780048504) bank88951 +88952 POINT(39.946774758226695 -73.331624706936) bank88952 +88953 POINT(40.98525507187665 -74.97859030308305) bank88953 +88954 POINT(41.553899182795305 -73.43194610536993) bank88954 +88955 POINT(40.9899858520361 -73.38735490489233) bank88955 +88956 POINT(40.738547473733824 -74.82111974867449) bank88956 +88957 POINT(40.340409015828385 -73.74004594163772) bank88957 +88958 POINT(40.51155032849639 -74.72120812984713) bank88958 +88959 POINT(39.94995307861489 -73.11590869283123) bank88959 +88960 POINT(40.10902319304813 -73.3896282473138) bank88960 +88961 POINT(39.8583784788234 -73.87791220526125) bank88961 +88962 POINT(40.935048691452536 -74.87404073558645) bank88962 +88963 POINT(41.153424093286745 -74.37195753881294) bank88963 +88964 POINT(39.735987179145475 -74.81287477547659) bank88964 +88965 POINT(40.89665515391585 -74.01165262582427) bank88965 +88966 POINT(40.18117899355256 -73.79116308716719) bank88966 +88967 POINT(40.21786927147491 -74.7581077180104) bank88967 +88968 POINT(41.24038003732783 -73.7238517982485) bank88968 +88969 POINT(40.82674319085307 -73.87897534373789) bank88969 +88970 POINT(39.91614994649595 -73.82523466830911) bank88970 +88971 POINT(40.693087064374204 -73.31200867437228) bank88971 +88972 POINT(41.107882028148836 -74.405657939187) bank88972 +88973 POINT(40.897694994060856 -73.96976418323923) bank88973 +88974 POINT(41.23701210056243 -74.54034987746734) bank88974 +88975 POINT(41.14678377361393 -74.15354048991793) bank88975 +88976 POINT(40.910258981259716 -73.4310548756751) bank88976 +88977 POINT(40.92830634654015 -73.53179308344332) bank88977 +88978 POINT(41.18514549890788 -74.27847431285522) bank88978 +88979 POINT(41.498855184215024 -74.63698255732557) bank88979 +88980 POINT(41.3511816017389 -74.13252154243335) bank88980 +88981 POINT(39.79801912544164 -73.44606179075107) bank88981 +88982 POINT(40.5208449291782 -73.45932517683045) bank88982 +88983 POINT(40.51041745973688 -74.67942907783922) bank88983 +88984 POINT(41.35084719783727 -74.79379768463525) bank88984 +88985 POINT(40.807721781287135 -73.71469092507253) bank88985 +88986 POINT(40.362541635327624 -74.44819805671044) bank88986 +88987 POINT(40.914611920762255 -74.90007017514063) bank88987 +88988 POINT(41.40664288445528 -73.97769122897861) bank88988 +88989 POINT(41.24312487999712 -74.45128771052008) bank88989 +88990 POINT(40.62609062432976 -74.06952464835884) bank88990 +88991 POINT(41.27933658692826 -74.88166741951389) bank88991 +88992 POINT(40.691782938646355 -73.33740436395944) bank88992 +88993 POINT(41.19762184628556 -73.05116840116453) bank88993 +88994 POINT(41.13239028905998 -73.23369356193922) bank88994 +88995 POINT(41.01123607787466 -74.04692957602323) bank88995 +88996 POINT(40.18741097649608 -73.05154563350608) bank88996 +88997 POINT(41.322850321405454 -74.31583013573668) bank88997 +88998 POINT(40.158083882974495 -74.80077667190608) bank88998 +88999 POINT(41.611986258100515 -74.14100523504933) bank88999 +89000 POINT(40.66711689111792 -73.39674412627566) bank89000 +89001 POINT(41.50120445310019 -74.88411733585652) bank89001 +89002 POINT(39.83976136489181 -73.0892642972134) bank89002 +89003 POINT(41.642501672685974 -73.88382450081085) bank89003 +89004 POINT(41.497472881079304 -74.35628203381309) bank89004 +89005 POINT(40.76877796024434 -74.12850357639594) bank89005 +89006 POINT(41.08875141741204 -73.04210318214639) bank89006 +89007 POINT(41.34584704426439 -74.77736360607732) bank89007 +89008 POINT(40.28499755156631 -74.21327321489125) bank89008 +89009 POINT(40.92480167277518 -74.40028903733055) bank89009 +89010 POINT(40.9820559870413 -74.81314984285278) bank89010 +89011 POINT(41.06433906820632 -74.79877578312342) bank89011 +89012 POINT(40.747844288011684 -73.17633366535614) bank89012 +89013 POINT(39.98906358884943 -74.00420810321744) bank89013 +89014 POINT(41.68360873402386 -73.30138861374002) bank89014 +89015 POINT(39.981289841979965 -74.63864064824054) bank89015 +89016 POINT(40.61363480248191 -73.35235419639001) bank89016 +89017 POINT(40.01211956061941 -74.11038865467724) bank89017 +89018 POINT(40.04683872455807 -73.01103504506233) bank89018 +89019 POINT(40.49087413740945 -74.6131305220583) bank89019 +89020 POINT(40.556173459170566 -74.91706257202077) bank89020 +89021 POINT(39.91478556708907 -74.9942635747715) bank89021 +89022 POINT(40.89121946876796 -74.01080340224931) bank89022 +89023 POINT(41.50415857526342 -74.49275579775768) bank89023 +89024 POINT(41.53462388614041 -74.06119888402982) bank89024 +89025 POINT(40.587236032060446 -74.18892392248736) bank89025 +89026 POINT(41.21250332358527 -73.08871102507689) bank89026 +89027 POINT(41.26286327284481 -74.78613587682662) bank89027 +89028 POINT(41.34484828321638 -73.03350588526106) bank89028 +89029 POINT(41.03435759398553 -73.26096010236276) bank89029 +89030 POINT(40.50730210207805 -73.6865184188131) bank89030 +89031 POINT(40.84477486650208 -73.15605855806592) bank89031 +89032 POINT(39.88706804602228 -74.79496689146401) bank89032 +89033 POINT(40.43359871692674 -74.3529930719646) bank89033 +89034 POINT(40.22581965608364 -73.40419007904534) bank89034 +89035 POINT(39.75620109431445 -73.77713399823988) bank89035 +89036 POINT(40.978886499594125 -74.11518986529198) bank89036 +89037 POINT(39.76692585925035 -73.69815957226626) bank89037 +89038 POINT(41.69736120454028 -74.30583242907159) bank89038 +89039 POINT(40.09134389075439 -73.69117496099862) bank89039 +89040 POINT(40.201050322924054 -73.55690987083563) bank89040 +89041 POINT(40.82602636273606 -74.34649885817873) bank89041 +89042 POINT(39.82752713924231 -73.20752714698284) bank89042 +89043 POINT(40.046454036184905 -73.72385470952509) bank89043 +89044 POINT(40.47357969122034 -73.91855976197749) bank89044 +89045 POINT(41.08283828355776 -74.65461826570186) bank89045 +89046 POINT(41.44632919766582 -73.60722397217384) bank89046 +89047 POINT(40.20410152697619 -74.96121573832862) bank89047 +89048 POINT(41.570634469069226 -74.74264725836966) bank89048 +89049 POINT(41.42650812157359 -74.3509775052475) bank89049 +89050 POINT(41.19824380960709 -74.55070971206085) bank89050 +89051 POINT(41.43363763322814 -74.85504897958158) bank89051 +89052 POINT(40.0106531534233 -73.46268863589572) bank89052 +89053 POINT(40.404292806076356 -74.27907156684567) bank89053 +89054 POINT(40.763803224784645 -74.36223913064534) bank89054 +89055 POINT(41.057404894866345 -73.94459686810123) bank89055 +89056 POINT(40.42431757105993 -73.49553742518015) bank89056 +89057 POINT(40.00035941841869 -73.57475261634661) bank89057 +89058 POINT(40.76058799447203 -74.46963311936635) bank89058 +89059 POINT(41.57428052493 -74.71762956390317) bank89059 +89060 POINT(41.33988727380031 -74.80621366311381) bank89060 +89061 POINT(40.31085513578772 -74.53248429892412) bank89061 +89062 POINT(40.230030693349505 -74.86224286567337) bank89062 +89063 POINT(40.490085231280204 -73.80344385327079) bank89063 +89064 POINT(39.85962814133858 -73.7857658284395) bank89064 +89065 POINT(41.62916914901167 -74.16078025241359) bank89065 +89066 POINT(41.36999075522935 -73.07229968402443) bank89066 +89067 POINT(41.21112662393271 -73.7323661542494) bank89067 +89068 POINT(40.794348478912866 -74.44055416230383) bank89068 +89069 POINT(40.74562024237962 -73.34730303873874) bank89069 +89070 POINT(41.644746422954285 -74.35591876589673) bank89070 +89071 POINT(40.88556315984809 -73.78877065930324) bank89071 +89072 POINT(41.69677754234494 -73.73568815296301) bank89072 +89073 POINT(40.808852452841016 -73.85171808665466) bank89073 +89074 POINT(39.79454111076212 -74.3068932820307) bank89074 +89075 POINT(41.247294121722916 -73.95515537179347) bank89075 +89076 POINT(40.55638356441763 -73.21426564228899) bank89076 +89077 POINT(39.839994817748035 -73.58784819489148) bank89077 +89078 POINT(40.51985611334129 -73.65597547698728) bank89078 +89079 POINT(40.62828158255458 -74.58740694145848) bank89079 +89080 POINT(41.50481474914691 -74.4680659108968) bank89080 +89081 POINT(40.941356041605225 -74.7583705270364) bank89081 +89082 POINT(40.58185415947072 -73.50119655642887) bank89082 +89083 POINT(41.072052503323754 -73.56731375174886) bank89083 +89084 POINT(41.15352040064882 -74.53562774701825) bank89084 +89085 POINT(41.383284283796755 -75.00065434716083) bank89085 +89086 POINT(39.73906510518158 -73.72001249578754) bank89086 +89087 POINT(40.87377270240124 -73.15564160474115) bank89087 +89088 POINT(40.68040598767967 -73.36817833273014) bank89088 +89089 POINT(41.436736225823815 -74.64867587367864) bank89089 +89090 POINT(40.546255344048184 -73.36492999886832) bank89090 +89091 POINT(40.212648153887145 -74.11221861865238) bank89091 +89092 POINT(40.20693074605914 -74.43841530328746) bank89092 +89093 POINT(39.74210723771711 -73.97147515421871) bank89093 +89094 POINT(40.57420139978328 -73.61324124405859) bank89094 +89095 POINT(39.85879384056404 -74.1883206628771) bank89095 +89096 POINT(40.275205527463484 -74.81828254937635) bank89096 +89097 POINT(40.03223099413413 -73.69010435908399) bank89097 +89098 POINT(41.26310232537814 -74.73789189990113) bank89098 +89099 POINT(40.07756396109324 -74.05336457711978) bank89099 +89100 POINT(41.44210886532838 -74.84901602810058) bank89100 +89101 POINT(40.68099147503146 -74.73616330374668) bank89101 +89102 POINT(41.18775952196721 -74.2081920313395) bank89102 +89103 POINT(40.10979938683684 -74.27021636459871) bank89103 +89104 POINT(41.12206094912735 -73.93760017037911) bank89104 +89105 POINT(39.92823751776677 -73.58915979726982) bank89105 +89106 POINT(40.09168799023289 -74.86056279569397) bank89106 +89107 POINT(41.539797518012705 -74.41704175688963) bank89107 +89108 POINT(40.577177404563244 -73.58377483684907) bank89108 +89109 POINT(41.48080392633404 -74.48271777925648) bank89109 +89110 POINT(41.08649379334863 -74.19879578594829) bank89110 +89111 POINT(41.51046675531185 -74.69749842025179) bank89111 +89112 POINT(41.466607149786284 -73.78663236522476) bank89112 +89113 POINT(40.58161307041233 -73.26301406705845) bank89113 +89114 POINT(40.12665254999411 -74.82591167873748) bank89114 +89115 POINT(40.42039868913962 -73.95176455091101) bank89115 +89116 POINT(41.18125391561323 -74.40490613770389) bank89116 +89117 POINT(40.99929053797816 -73.0243591980582) bank89117 +89118 POINT(41.1407855332443 -74.99106043156512) bank89118 +89119 POINT(41.07822280512024 -74.5221840773299) bank89119 +89120 POINT(39.90595221178359 -74.68660398560681) bank89120 +89121 POINT(40.96580439546153 -74.92843207305458) bank89121 +89122 POINT(39.779192639646475 -74.10215936963564) bank89122 +89123 POINT(40.845756649924425 -73.86415020996576) bank89123 +89124 POINT(39.896806386623325 -73.63868220230508) bank89124 +89125 POINT(40.66550693600863 -74.2189790311771) bank89125 +89126 POINT(41.21791721581747 -74.22366883419896) bank89126 +89127 POINT(41.53945614838423 -73.29922288737062) bank89127 +89128 POINT(40.984494109469395 -73.3844420303406) bank89128 +89129 POINT(41.47048684488656 -74.62966973826381) bank89129 +89130 POINT(41.24397752481365 -73.0218502936333) bank89130 +89131 POINT(40.45023738892231 -73.31614892385159) bank89131 +89132 POINT(40.72318795931717 -73.80598879209224) bank89132 +89133 POINT(40.203073885782935 -73.0075809688753) bank89133 +89134 POINT(39.94571447808793 -74.7073989684677) bank89134 +89135 POINT(40.15442123954127 -74.59651500042669) bank89135 +89136 POINT(39.902324746359035 -73.61569412735128) bank89136 +89137 POINT(41.11073170148859 -74.83708067500504) bank89137 +89138 POINT(41.493974775863215 -73.21029321718584) bank89138 +89139 POINT(41.618413293623554 -74.9672376466299) bank89139 +89140 POINT(41.1605283589223 -73.29894941200071) bank89140 +89141 POINT(39.87851128260739 -73.88165152603376) bank89141 +89142 POINT(40.17073146936059 -74.25971762213439) bank89142 +89143 POINT(39.970249772800436 -73.66330653253708) bank89143 +89144 POINT(41.16049828164251 -73.0091128432008) bank89144 +89145 POINT(40.65731025699147 -73.15703333794139) bank89145 +89146 POINT(39.77420172546145 -73.95890442927326) bank89146 +89147 POINT(41.235694064284914 -73.14746866515212) bank89147 +89148 POINT(40.99066014873696 -73.76054641655134) bank89148 +89149 POINT(40.47498413980975 -73.58344293247778) bank89149 +89150 POINT(40.470773494309356 -74.59468562197725) bank89150 +89151 POINT(40.74506405205163 -74.00362200329224) bank89151 +89152 POINT(41.021834194742965 -74.88765780207058) bank89152 +89153 POINT(41.6482281990373 -74.43539650413449) bank89153 +89154 POINT(41.39302322746466 -74.52447019101851) bank89154 +89155 POINT(40.37304012022518 -74.18150212950984) bank89155 +89156 POINT(40.418815691335176 -74.71787554984908) bank89156 +89157 POINT(41.07972023610368 -74.45211375167685) bank89157 +89158 POINT(41.13518253886793 -73.99845599572559) bank89158 +89159 POINT(41.096151940427376 -73.38484698905896) bank89159 +89160 POINT(40.46124079664861 -74.91465644017609) bank89160 +89161 POINT(40.37742962713125 -74.1485130679671) bank89161 +89162 POINT(40.697973989408034 -73.06098730369251) bank89162 +89163 POINT(40.856598644034804 -73.14577916829097) bank89163 +89164 POINT(40.85158554781565 -73.89505311040672) bank89164 +89165 POINT(40.40780843269758 -73.52295161683494) bank89165 +89166 POINT(40.527460283443055 -74.46348940315539) bank89166 +89167 POINT(40.77917757661872 -73.19174333446698) bank89167 +89168 POINT(41.519798364822925 -73.28014133614354) bank89168 +89169 POINT(41.5851883566135 -74.53676440755677) bank89169 +89170 POINT(40.99647264489849 -73.0287040167954) bank89170 +89171 POINT(41.235395007013665 -73.22447904207769) bank89171 +89172 POINT(40.49757866587983 -73.0811102899878) bank89172 +89173 POINT(41.48264205021362 -74.05897236744241) bank89173 +89174 POINT(40.398449849098256 -74.84980496108524) bank89174 +89175 POINT(39.916334843272885 -74.76776214874855) bank89175 +89176 POINT(41.3467798047059 -73.48310438373498) bank89176 +89177 POINT(41.70742266374174 -74.24970186797479) bank89177 +89178 POINT(40.81084260525668 -73.77522464940179) bank89178 +89179 POINT(41.479196180511664 -73.85542762079741) bank89179 +89180 POINT(41.48247391076607 -74.04250495683574) bank89180 +89181 POINT(40.44994481483588 -74.73261872951886) bank89181 +89182 POINT(40.37550123328177 -74.40514875904611) bank89182 +89183 POINT(40.22517955404211 -73.85314428072907) bank89183 +89184 POINT(40.566436811173546 -74.64423091691151) bank89184 +89185 POINT(40.14105036380271 -74.04842176726723) bank89185 +89186 POINT(40.84550124230066 -73.86393060064732) bank89186 +89187 POINT(41.16363228350294 -74.1859899579673) bank89187 +89188 POINT(40.0117932668066 -74.92949405327674) bank89188 +89189 POINT(41.02756027524544 -73.50831596495179) bank89189 +89190 POINT(40.573047501141666 -73.9258296794829) bank89190 +89191 POINT(39.843855524880865 -73.01920935211535) bank89191 +89192 POINT(41.693820100651614 -73.87843899954134) bank89192 +89193 POINT(41.04815506586859 -73.74256075382918) bank89193 +89194 POINT(41.31780464005552 -74.46823219349886) bank89194 +89195 POINT(40.136664966172894 -73.56752917102827) bank89195 +89196 POINT(39.75127446651673 -74.96698131579424) bank89196 +89197 POINT(40.235984896882094 -74.36671367736602) bank89197 +89198 POINT(40.2763580707407 -73.55909517208455) bank89198 +89199 POINT(39.73263578064 -74.22459263499537) bank89199 +89200 POINT(41.15633719500094 -73.66675783733636) bank89200 +89201 POINT(40.11542061992299 -74.71165992111138) bank89201 +89202 POINT(41.136244895159265 -73.6170276074138) bank89202 +89203 POINT(41.62334156809368 -73.2645627568978) bank89203 +89204 POINT(40.24436427539638 -73.44248918229061) bank89204 +89205 POINT(40.229930947813244 -73.69708515999648) bank89205 +89206 POINT(41.44032693221277 -73.27642325001341) bank89206 +89207 POINT(40.88424690489481 -74.44650552774141) bank89207 +89208 POINT(41.51519208852142 -73.55958273010958) bank89208 +89209 POINT(40.45561712807096 -74.60259074815522) bank89209 +89210 POINT(41.669551070705076 -74.56248027726838) bank89210 +89211 POINT(41.181037239926226 -74.97229148571755) bank89211 +89212 POINT(41.653197884843 -74.35431866141627) bank89212 +89213 POINT(40.136028597534214 -73.15335188071327) bank89213 +89214 POINT(39.79299904460039 -74.37254436176617) bank89214 +89215 POINT(41.12187269484596 -74.02593339450631) bank89215 +89216 POINT(41.10045151596184 -74.55580791950807) bank89216 +89217 POINT(40.1658872547234 -74.34771007083326) bank89217 +89218 POINT(41.1930569661052 -73.2558443595392) bank89218 +89219 POINT(40.946911848713775 -74.89528952472496) bank89219 +89220 POINT(41.07331253584 -74.79178000999453) bank89220 +89221 POINT(40.709057870713536 -74.66434574378742) bank89221 +89222 POINT(41.18395797279005 -74.30510488374432) bank89222 +89223 POINT(40.29165063399101 -74.27883451782232) bank89223 +89224 POINT(41.43311376218261 -73.35709914571385) bank89224 +89225 POINT(40.81136055034713 -74.22675854109873) bank89225 +89226 POINT(39.750639718904054 -74.57497898537922) bank89226 +89227 POINT(41.16081383453696 -73.60317244532118) bank89227 +89228 POINT(40.65961770842474 -73.59822056423533) bank89228 +89229 POINT(40.353637995363684 -74.96697913419132) bank89229 +89230 POINT(40.829066684521685 -74.35903827497128) bank89230 +89231 POINT(40.20942716511941 -74.36534845522237) bank89231 +89232 POINT(40.49216861704316 -73.61127781059774) bank89232 +89233 POINT(41.70561757793251 -74.27450027503833) bank89233 +89234 POINT(39.77317379584222 -73.38293943606078) bank89234 +89235 POINT(41.500378762585136 -74.90125878385277) bank89235 +89236 POINT(40.30506242458501 -74.49879653786651) bank89236 +89237 POINT(40.26467771398534 -73.80089023680709) bank89237 +89238 POINT(39.78369623339614 -74.2967381042014) bank89238 +89239 POINT(39.78889734063653 -73.4384152927637) bank89239 +89240 POINT(41.17755909751026 -73.55881193673316) bank89240 +89241 POINT(41.133569494676514 -73.96884432338048) bank89241 +89242 POINT(41.50400335830422 -74.96017585190909) bank89242 +89243 POINT(41.07635958461062 -74.41136454846213) bank89243 +89244 POINT(41.674312961397284 -73.61028178913443) bank89244 +89245 POINT(41.6131555193729 -74.46487911348389) bank89245 +89246 POINT(39.89032522696501 -74.4101757793151) bank89246 +89247 POINT(40.19867549701601 -74.54803748882755) bank89247 +89248 POINT(39.85292495552126 -74.37023956930372) bank89248 +89249 POINT(41.20092158875477 -75.0052240549417) bank89249 +89250 POINT(41.596348001938864 -73.72269795235091) bank89250 +89251 POINT(41.62322345139227 -74.89761445457505) bank89251 +89252 POINT(40.47626080769569 -74.71498585664632) bank89252 +89253 POINT(41.65050204933827 -74.59798227788306) bank89253 +89254 POINT(41.277424355186 -74.8814176622146) bank89254 +89255 POINT(41.260149749014396 -73.41557126435464) bank89255 +89256 POINT(41.03312456146386 -74.89242015117057) bank89256 +89257 POINT(41.220956089161135 -73.53629495008049) bank89257 +89258 POINT(40.317235012774766 -73.58470282159664) bank89258 +89259 POINT(40.7599290302325 -74.78527997067287) bank89259 +89260 POINT(41.67307597032992 -74.44956420747586) bank89260 +89261 POINT(41.10497055562955 -73.03088664961423) bank89261 +89262 POINT(40.948758814515884 -74.66962409652757) bank89262 +89263 POINT(39.77598685907918 -73.33788416705822) bank89263 +89264 POINT(41.25205862648796 -74.44648583526877) bank89264 +89265 POINT(40.28469356642218 -73.86840416072782) bank89265 +89266 POINT(40.609207276119825 -73.562986363511) bank89266 +89267 POINT(39.98898281511823 -73.6540520169847) bank89267 +89268 POINT(41.1422245387317 -74.12099370143113) bank89268 +89269 POINT(41.15979675521234 -73.94124789784428) bank89269 +89270 POINT(41.403532335964314 -73.99093535870043) bank89270 +89271 POINT(40.76242845442911 -74.0857703520628) bank89271 +89272 POINT(41.48197132406929 -74.26067408228703) bank89272 +89273 POINT(40.69655837478129 -74.77339333758825) bank89273 +89274 POINT(41.41654508269654 -74.4736129124542) bank89274 +89275 POINT(41.678735480651696 -74.94889916937645) bank89275 +89276 POINT(40.54575647695498 -73.23344594626887) bank89276 +89277 POINT(40.101264801994695 -74.24355132909308) bank89277 +89278 POINT(41.06800166714169 -74.74021459460499) bank89278 +89279 POINT(41.583649526836545 -74.09302309664815) bank89279 +89280 POINT(40.89755660739929 -73.97667295097587) bank89280 +89281 POINT(40.71900485754166 -73.49577666192818) bank89281 +89282 POINT(40.97910344250559 -74.41056921631446) bank89282 +89283 POINT(40.63142784876558 -73.83130226787101) bank89283 +89284 POINT(40.67655218886077 -73.69793201198712) bank89284 +89285 POINT(40.20399669628944 -74.55558463129147) bank89285 +89286 POINT(40.26803668926262 -73.36963062716966) bank89286 +89287 POINT(41.229288933201325 -74.51109386349002) bank89287 +89288 POINT(40.36291667760548 -74.19118783335186) bank89288 +89289 POINT(40.958117744418935 -73.96534470295451) bank89289 +89290 POINT(41.12509482190088 -73.35707847726941) bank89290 +89291 POINT(40.64431373044893 -74.74528752977729) bank89291 +89292 POINT(39.99237638910787 -74.68051645260225) bank89292 +89293 POINT(40.54253835120421 -74.14564714974733) bank89293 +89294 POINT(41.56782550221793 -73.725806256025) bank89294 +89295 POINT(40.20874592526875 -74.20080771212616) bank89295 +89296 POINT(41.6868996502621 -73.61971564505484) bank89296 +89297 POINT(40.07582256211542 -74.03209764279346) bank89297 +89298 POINT(40.106352793915235 -74.09819317502692) bank89298 +89299 POINT(40.95460184183965 -74.43561836188637) bank89299 +89300 POINT(40.63873656591683 -73.84953042156705) bank89300 +89301 POINT(40.04064082945475 -73.614833603699) bank89301 +89302 POINT(40.46591206299839 -74.54390445849985) bank89302 +89303 POINT(41.53108616970544 -73.82263619926255) bank89303 +89304 POINT(41.458975127683104 -73.07954964501803) bank89304 +89305 POINT(41.43207064038654 -73.56983184366482) bank89305 +89306 POINT(40.60447768917867 -73.18902987711766) bank89306 +89307 POINT(40.19087695149755 -73.7979607962048) bank89307 +89308 POINT(40.45778699867881 -74.09153455969067) bank89308 +89309 POINT(41.41848089671898 -74.93642354817655) bank89309 +89310 POINT(40.287231960189025 -73.38914869236797) bank89310 +89311 POINT(40.4496767881206 -74.04551325639808) bank89311 +89312 POINT(40.06717315473917 -73.41416442116835) bank89312 +89313 POINT(41.468129652418966 -73.54098690348629) bank89313 +89314 POINT(40.93798799273948 -73.00948165996697) bank89314 +89315 POINT(41.096728465014664 -74.2223741189173) bank89315 +89316 POINT(41.10861858538429 -74.98898545675038) bank89316 +89317 POINT(41.253170461166405 -73.5388768529653) bank89317 +89318 POINT(40.87403742908325 -73.10660004567963) bank89318 +89319 POINT(40.517656240639184 -74.63296763385885) bank89319 +89320 POINT(41.252871250083246 -74.4847692888354) bank89320 +89321 POINT(41.021150833595925 -74.91929087283026) bank89321 +89322 POINT(39.84445308175383 -74.09806737970466) bank89322 +89323 POINT(40.45089574916414 -74.89960997265489) bank89323 +89324 POINT(41.468690035719604 -73.93483260285052) bank89324 +89325 POINT(41.04486476991291 -74.77435493928748) bank89325 +89326 POINT(40.88976675378114 -73.10490665830676) bank89326 +89327 POINT(40.85755721313839 -73.3062261762209) bank89327 +89328 POINT(40.39891819433761 -73.39479691968819) bank89328 +89329 POINT(40.19058673720662 -73.20009293430553) bank89329 +89330 POINT(40.76443016222129 -73.70330829654307) bank89330 +89331 POINT(40.56581081739566 -73.16276167996342) bank89331 +89332 POINT(39.89073061689392 -73.83370452378752) bank89332 +89333 POINT(41.05375245310055 -74.28883270763379) bank89333 +89334 POINT(39.9689378623318 -74.15936031410719) bank89334 +89335 POINT(41.260987809272024 -73.8807603423193) bank89335 +89336 POINT(41.42412256698713 -74.09821008158652) bank89336 +89337 POINT(39.86108173938508 -74.00673972559687) bank89337 +89338 POINT(41.50078851272826 -73.94693652264851) bank89338 +89339 POINT(40.518242934202455 -74.06958856268554) bank89339 +89340 POINT(40.71860391447974 -74.42760758498632) bank89340 +89341 POINT(41.34098228990163 -73.34609882672028) bank89341 +89342 POINT(40.8045180731198 -74.5955657519723) bank89342 +89343 POINT(41.69907285031659 -74.7617396774428) bank89343 +89344 POINT(40.44184079831518 -73.35262313104494) bank89344 +89345 POINT(41.235688806515256 -73.25904375593556) bank89345 +89346 POINT(40.285933995476185 -74.57785019398744) bank89346 +89347 POINT(40.783478357624645 -73.41128432970203) bank89347 +89348 POINT(41.63087409467708 -74.2768300962851) bank89348 +89349 POINT(40.493324578444934 -74.84372682344325) bank89349 +89350 POINT(40.80355735248823 -74.97199022315155) bank89350 +89351 POINT(41.24741069027501 -74.68152760843577) bank89351 +89352 POINT(40.424359606299404 -74.50403909590511) bank89352 +89353 POINT(40.979347110208856 -74.78083263299071) bank89353 +89354 POINT(39.73226283841728 -74.15856852179574) bank89354 +89355 POINT(39.96961743674746 -74.9829591282941) bank89355 +89356 POINT(40.32451193391515 -74.57294971254353) bank89356 +89357 POINT(41.5517425462982 -74.32506508333917) bank89357 +89358 POINT(40.822517962244426 -73.33475943618592) bank89358 +89359 POINT(39.994374468122054 -75.00379254867549) bank89359 +89360 POINT(40.09998026627601 -73.95209412732343) bank89360 +89361 POINT(40.20746329408541 -73.7453134226928) bank89361 +89362 POINT(41.356892122749265 -74.6674262543728) bank89362 +89363 POINT(40.632368295764145 -73.3929824975221) bank89363 +89364 POINT(41.30428234303023 -74.30512419534432) bank89364 +89365 POINT(39.9110310810855 -73.61462207356995) bank89365 +89366 POINT(39.90539099236641 -74.91562249362309) bank89366 +89367 POINT(41.7003057906965 -73.15828837310546) bank89367 +89368 POINT(41.70665079186741 -74.73276540597864) bank89368 +89369 POINT(40.561457593468205 -73.16707588703667) bank89369 +89370 POINT(40.282562285698376 -74.21797227512457) bank89370 +89371 POINT(40.30288552871364 -74.97269035748386) bank89371 +89372 POINT(40.74998954776194 -74.89110420488404) bank89372 +89373 POINT(40.28596694322624 -73.8277239901847) bank89373 +89374 POINT(40.030264546091225 -74.95004536314246) bank89374 +89375 POINT(39.76209763915292 -73.10553542146955) bank89375 +89376 POINT(41.05495619248584 -74.9650634599792) bank89376 +89377 POINT(41.13563069970319 -74.42021704999638) bank89377 +89378 POINT(41.597771983624625 -74.87455153088749) bank89378 +89379 POINT(40.25484100171076 -74.60890682807172) bank89379 +89380 POINT(41.51458076548452 -74.38184046677172) bank89380 +89381 POINT(40.37466669759227 -74.16441360495) bank89381 +89382 POINT(40.31208249187402 -73.02880795828015) bank89382 +89383 POINT(40.26727831598621 -74.96396323010516) bank89383 +89384 POINT(40.23888168598888 -74.9808457650278) bank89384 +89385 POINT(40.70564349545317 -73.98691283862667) bank89385 +89386 POINT(40.18686886805125 -74.85955807949709) bank89386 +89387 POINT(40.035258790430795 -74.46112799214106) bank89387 +89388 POINT(41.407556283595234 -74.40930416369713) bank89388 +89389 POINT(40.9832783199575 -74.94386117779104) bank89389 +89390 POINT(41.40065629936072 -74.68383195495652) bank89390 +89391 POINT(40.97204137954387 -73.89930798116065) bank89391 +89392 POINT(39.94695500683668 -73.01303618804995) bank89392 +89393 POINT(41.33933480683874 -74.24766432729741) bank89393 +89394 POINT(40.97368630682592 -74.36673179079563) bank89394 +89395 POINT(39.895408367905574 -73.31557527367303) bank89395 +89396 POINT(41.619978806477185 -73.07947202353945) bank89396 +89397 POINT(41.12464810577093 -74.66912286959757) bank89397 +89398 POINT(40.24287402697493 -73.79327481885325) bank89398 +89399 POINT(39.88506498980895 -74.37870222483998) bank89399 +89400 POINT(40.31871781657392 -73.63013403669912) bank89400 +89401 POINT(41.44647528692602 -74.76798718656097) bank89401 +89402 POINT(40.89927032111418 -74.15356823420382) bank89402 +89403 POINT(40.722587201703696 -74.04863994935485) bank89403 +89404 POINT(39.99339566226409 -73.16501985751101) bank89404 +89405 POINT(40.29023120349557 -73.29284158791832) bank89405 +89406 POINT(40.67449581925442 -74.32315557124727) bank89406 +89407 POINT(41.674456891224644 -73.86293803905927) bank89407 +89408 POINT(40.744651984637656 -73.53410370850129) bank89408 +89409 POINT(40.040711765245085 -74.0192768776921) bank89409 +89410 POINT(41.261897823780146 -73.77628632042371) bank89410 +89411 POINT(39.97763810996034 -73.38557324153922) bank89411 +89412 POINT(40.27654656761857 -74.44327545058329) bank89412 +89413 POINT(39.985727989737455 -73.74068216576448) bank89413 +89414 POINT(40.52388739037786 -74.53603670615345) bank89414 +89415 POINT(40.68660111474409 -73.73273043841913) bank89415 +89416 POINT(41.58616901906359 -73.94513838500043) bank89416 +89417 POINT(41.58350232953103 -74.92698094996027) bank89417 +89418 POINT(39.75822162559179 -73.79368098099984) bank89418 +89419 POINT(41.31549919873376 -73.29264727954339) bank89419 +89420 POINT(41.36473052625706 -74.69025504395381) bank89420 +89421 POINT(41.441771862288235 -74.50212627248558) bank89421 +89422 POINT(39.90531935325485 -73.38862992277583) bank89422 +89423 POINT(41.30422940092988 -74.4843322426587) bank89423 +89424 POINT(40.57681542599169 -74.71337466243116) bank89424 +89425 POINT(39.83886639693775 -74.97914214538729) bank89425 +89426 POINT(41.392360052531245 -73.47631874411846) bank89426 +89427 POINT(40.49566376022632 -74.25146686423489) bank89427 +89428 POINT(40.46851228474012 -73.02039911854287) bank89428 +89429 POINT(41.683622095630305 -74.85782189043218) bank89429 +89430 POINT(40.09504640008359 -74.98980434271812) bank89430 +89431 POINT(39.86252375912464 -73.72037211918712) bank89431 +89432 POINT(40.78115687645282 -73.39223898338494) bank89432 +89433 POINT(40.860598572028444 -74.96074944533746) bank89433 +89434 POINT(40.288673119260224 -73.4836621658414) bank89434 +89435 POINT(40.94514954985575 -74.7751535633105) bank89435 +89436 POINT(39.81410895248815 -73.5951636375157) bank89436 +89437 POINT(40.8185624159631 -74.6645354998182) bank89437 +89438 POINT(41.07753879964041 -73.81365001493613) bank89438 +89439 POINT(41.4104962253759 -73.11934478967625) bank89439 +89440 POINT(41.13527323208148 -73.48304106321633) bank89440 +89441 POINT(40.55094803331021 -73.79084404226793) bank89441 +89442 POINT(40.582054940983745 -74.62124648680322) bank89442 +89443 POINT(39.90993964190122 -73.51865683982038) bank89443 +89444 POINT(40.22503013999737 -74.15845991981556) bank89444 +89445 POINT(40.53270815767281 -74.50194436672805) bank89445 +89446 POINT(41.22634725462221 -74.2569922455823) bank89446 +89447 POINT(41.172557805669975 -73.19701664204975) bank89447 +89448 POINT(40.522022250673615 -74.33441494918834) bank89448 +89449 POINT(41.31779020476981 -73.65394366500091) bank89449 +89450 POINT(41.362661063577875 -73.32032993159594) bank89450 +89451 POINT(41.33142028897205 -73.05200406334149) bank89451 +89452 POINT(40.99132214471656 -74.19683982730565) bank89452 +89453 POINT(40.77653061450131 -73.11319365678807) bank89453 +89454 POINT(40.62460720519303 -74.60216942463742) bank89454 +89455 POINT(40.98256007009464 -74.52332544987523) bank89455 +89456 POINT(40.069274232140046 -74.61073942843575) bank89456 +89457 POINT(41.410704284385716 -73.8797921958894) bank89457 +89458 POINT(39.80121531787284 -74.18178960722241) bank89458 +89459 POINT(40.21096002337402 -74.9335815563912) bank89459 +89460 POINT(41.44293862637133 -73.95704305873974) bank89460 +89461 POINT(40.02407969260112 -73.25759036613275) bank89461 +89462 POINT(41.047109514193814 -73.82689220693472) bank89462 +89463 POINT(40.80906244479211 -74.80768662538554) bank89463 +89464 POINT(40.59618708045852 -74.1501745494827) bank89464 +89465 POINT(40.1732251792233 -74.22049237253151) bank89465 +89466 POINT(40.10395681014924 -73.7554637330339) bank89466 +89467 POINT(41.23870680809381 -73.63480406756744) bank89467 +89468 POINT(41.51743469642644 -73.89871403766013) bank89468 +89469 POINT(40.58525662269691 -73.11605323075773) bank89469 +89470 POINT(39.79254876488293 -73.49966661632794) bank89470 +89471 POINT(41.55934694974226 -74.83061790235665) bank89471 +89472 POINT(39.98176074438908 -74.19836650308703) bank89472 +89473 POINT(40.574428969113825 -73.59228232637092) bank89473 +89474 POINT(40.11884502813742 -74.13624590319841) bank89474 +89475 POINT(40.1979806017619 -74.22477889048912) bank89475 +89476 POINT(40.43124729982372 -74.60502344767964) bank89476 +89477 POINT(40.440700609354245 -73.42365362814658) bank89477 +89478 POINT(39.83485895098173 -73.51774738583165) bank89478 +89479 POINT(41.01369634405187 -73.40056852516167) bank89479 +89480 POINT(41.409622741347114 -74.66443270741198) bank89480 +89481 POINT(40.36445041437813 -74.45861693233233) bank89481 +89482 POINT(40.18835613330548 -73.99279407932379) bank89482 +89483 POINT(39.96797529492314 -73.10612128182436) bank89483 +89484 POINT(39.818869780745786 -75.0035608195932) bank89484 +89485 POINT(40.710881990909286 -73.86398738849876) bank89485 +89486 POINT(40.79189763193598 -74.5217132610768) bank89486 +89487 POINT(40.29228188022206 -74.31418301641908) bank89487 +89488 POINT(40.91567849529252 -74.21407270570212) bank89488 +89489 POINT(39.97030182875209 -74.09323222264793) bank89489 +89490 POINT(40.22350844052057 -74.38317456774134) bank89490 +89491 POINT(40.609355499711896 -73.32955876116273) bank89491 +89492 POINT(41.51590160144254 -74.19631772104437) bank89492 +89493 POINT(41.286668536920615 -73.09188110644575) bank89493 +89494 POINT(41.65640934116507 -74.60189725241639) bank89494 +89495 POINT(39.861463608854216 -73.93143628400608) bank89495 +89496 POINT(40.89721465553201 -74.53241010195596) bank89496 +89497 POINT(41.63246882477297 -73.34137974251568) bank89497 +89498 POINT(41.048934674174866 -73.03527538853534) bank89498 +89499 POINT(40.35068746230693 -73.632216857269) bank89499 +89500 POINT(40.163131558995154 -74.31484442045598) bank89500 +89501 POINT(40.97781982324729 -74.94810737572138) bank89501 +89502 POINT(39.96370326185965 -73.83918229007371) bank89502 +89503 POINT(41.325610592508866 -74.34476713433868) bank89503 +89504 POINT(40.49132405007661 -73.09532098371186) bank89504 +89505 POINT(39.92546874999913 -73.90076654643244) bank89505 +89506 POINT(40.953686417496606 -73.30613959812975) bank89506 +89507 POINT(40.50814492916659 -73.69202586447508) bank89507 +89508 POINT(41.61996006676353 -73.87059672097017) bank89508 +89509 POINT(40.63767911372478 -73.88743566047886) bank89509 +89510 POINT(40.580736552419154 -73.39260086003445) bank89510 +89511 POINT(40.412394209689204 -73.02776204203961) bank89511 +89512 POINT(40.535338526633865 -73.59408446653876) bank89512 +89513 POINT(40.80281264681044 -74.09289174351436) bank89513 +89514 POINT(40.76108989924248 -74.88900066349474) bank89514 +89515 POINT(41.146732937860804 -74.07149945128619) bank89515 +89516 POINT(41.5789751130483 -73.90645310059301) bank89516 +89517 POINT(40.17720856864921 -74.60241617829425) bank89517 +89518 POINT(40.46568322701088 -74.49853886334682) bank89518 +89519 POINT(41.56730620927547 -74.37824784508564) bank89519 +89520 POINT(40.1405004882168 -73.0375965605905) bank89520 +89521 POINT(40.89633669527554 -74.67412934252003) bank89521 +89522 POINT(41.450015320852245 -74.45930691006976) bank89522 +89523 POINT(41.3543309571207 -73.82674146492457) bank89523 +89524 POINT(40.012465014213234 -74.697652668497) bank89524 +89525 POINT(40.26264196100321 -73.49150964569434) bank89525 +89526 POINT(40.82237859822589 -74.34878015529027) bank89526 +89527 POINT(41.48652312031067 -74.75180156107996) bank89527 +89528 POINT(40.661040511241616 -73.81919855541771) bank89528 +89529 POINT(41.499461431640285 -74.12499802421388) bank89529 +89530 POINT(41.480274632422 -73.64109489743834) bank89530 +89531 POINT(39.8983898685965 -73.43027744719849) bank89531 +89532 POINT(40.326512038582905 -73.8542534989025) bank89532 +89533 POINT(41.28875476533424 -74.83380510187304) bank89533 +89534 POINT(40.26812539699069 -74.01081004657891) bank89534 +89535 POINT(39.795297397663674 -74.01954345945805) bank89535 +89536 POINT(40.91998998354503 -73.81078412425467) bank89536 +89537 POINT(40.59520383498537 -73.03087824073748) bank89537 +89538 POINT(40.46379951078336 -73.99071874124687) bank89538 +89539 POINT(39.737226529076935 -73.56078384949191) bank89539 +89540 POINT(41.248874368516155 -74.99242707551757) bank89540 +89541 POINT(41.14227531118821 -73.14166747806019) bank89541 +89542 POINT(40.57261697871677 -74.33682356493655) bank89542 +89543 POINT(41.138241210189115 -74.70958597980731) bank89543 +89544 POINT(40.04250364909919 -73.57248525187) bank89544 +89545 POINT(40.65205140403811 -73.45571657035647) bank89545 +89546 POINT(41.371847641875874 -74.71122311023517) bank89546 +89547 POINT(40.845871632767874 -73.51387279394383) bank89547 +89548 POINT(39.746684693300566 -74.4400692679118) bank89548 +89549 POINT(41.120753079664325 -73.2421144106787) bank89549 +89550 POINT(41.059510330911664 -73.04993206403105) bank89550 +89551 POINT(41.42448524559458 -74.99106978821602) bank89551 +89552 POINT(40.68076574708704 -74.37985357849124) bank89552 +89553 POINT(40.638312286165466 -73.01606165308549) bank89553 +89554 POINT(40.878768988690595 -73.71652415806636) bank89554 +89555 POINT(41.653426536101385 -74.29624302126828) bank89555 +89556 POINT(40.90692233483101 -73.48966311834016) bank89556 +89557 POINT(40.74768188990624 -73.1522186961358) bank89557 +89558 POINT(40.59383773854186 -74.65536031230036) bank89558 +89559 POINT(40.65353695180763 -73.95430949898042) bank89559 +89560 POINT(39.936028512692204 -74.21332099143378) bank89560 +89561 POINT(41.09897162071397 -73.74877845472184) bank89561 +89562 POINT(40.98147055764351 -74.37234688666453) bank89562 +89563 POINT(41.39682139666774 -73.0708608951265) bank89563 +89564 POINT(41.46122003247496 -73.07621802387752) bank89564 +89565 POINT(40.01902236769196 -74.08187949355127) bank89565 +89566 POINT(39.76041552226232 -74.64153581470131) bank89566 +89567 POINT(40.87789086411101 -73.66239795782973) bank89567 +89568 POINT(40.170269649865915 -74.57822376512328) bank89568 +89569 POINT(41.52149001903333 -74.04460514040983) bank89569 +89570 POINT(40.94778860229881 -74.18480160161357) bank89570 +89571 POINT(41.195013913310405 -73.35206073913182) bank89571 +89572 POINT(40.20184349242051 -74.44813519043336) bank89572 +89573 POINT(40.372291739526204 -73.90567986424338) bank89573 +89574 POINT(40.48292942895983 -73.66512622180718) bank89574 +89575 POINT(40.7299447259002 -74.54373344789349) bank89575 +89576 POINT(41.69893012478017 -74.7126625979677) bank89576 +89577 POINT(41.68780666918986 -73.48216589692788) bank89577 +89578 POINT(40.18232891955733 -73.8748970375453) bank89578 +89579 POINT(41.38835514862374 -73.2351301973821) bank89579 +89580 POINT(40.53370051193939 -73.23731085917012) bank89580 +89581 POINT(40.37485959662324 -74.22393131072133) bank89581 +89582 POINT(40.62393454360919 -73.2484912020663) bank89582 +89583 POINT(39.94554201973125 -73.69843384731136) bank89583 +89584 POINT(40.92842840071909 -73.54972812223029) bank89584 +89585 POINT(40.77110691923867 -73.49001086284943) bank89585 +89586 POINT(41.09790585113802 -74.06079503058197) bank89586 +89587 POINT(40.00343635672857 -74.80328630383349) bank89587 +89588 POINT(40.116016556415346 -73.57641097397097) bank89588 +89589 POINT(41.11540519262668 -74.82465405797625) bank89589 +89590 POINT(40.05222409176327 -74.86608822602759) bank89590 +89591 POINT(40.35751135357965 -74.17397066385048) bank89591 +89592 POINT(40.325092807423324 -73.74696408739686) bank89592 +89593 POINT(40.72867499185624 -73.37234262930349) bank89593 +89594 POINT(40.00401195823622 -73.1337609930514) bank89594 +89595 POINT(40.41302449740226 -74.14460063124547) bank89595 +89596 POINT(40.57280707450226 -74.17750299676982) bank89596 +89597 POINT(40.87971383277821 -73.25161641407934) bank89597 +89598 POINT(40.460628482989854 -73.84205186437715) bank89598 +89599 POINT(40.31155565998879 -74.56915664250592) bank89599 +89600 POINT(40.44443241151108 -74.35872844701899) bank89600 +89601 POINT(41.20340951659648 -74.04073502532134) bank89601 +89602 POINT(41.556934927987264 -74.34660836092357) bank89602 +89603 POINT(39.949626293387134 -74.34927189804446) bank89603 +89604 POINT(40.14118770923804 -73.09001265302365) bank89604 +89605 POINT(40.52139434041918 -74.0166739779417) bank89605 +89606 POINT(41.03258809472891 -73.58222915915715) bank89606 +89607 POINT(41.26311395857998 -74.56278435071363) bank89607 +89608 POINT(40.715964255652935 -73.4458426968905) bank89608 +89609 POINT(40.10960581085374 -73.32704893103406) bank89609 +89610 POINT(41.580375120086714 -74.53498860214773) bank89610 +89611 POINT(41.458102209475385 -74.36503711535836) bank89611 +89612 POINT(40.08701614744814 -73.5979257452006) bank89612 +89613 POINT(41.202395271401855 -73.0180191336955) bank89613 +89614 POINT(40.3836263410924 -73.25852322361372) bank89614 +89615 POINT(41.67261225810719 -73.4572523192846) bank89615 +89616 POINT(41.461501722139985 -74.9734795946423) bank89616 +89617 POINT(40.73991377168195 -73.74166323426867) bank89617 +89618 POINT(39.73347859256226 -74.86184789809192) bank89618 +89619 POINT(40.44511456720871 -74.85437080818919) bank89619 +89620 POINT(41.05002290578949 -73.53840651667366) bank89620 +89621 POINT(41.67536948358902 -73.37754521302487) bank89621 +89622 POINT(39.81828161710163 -74.5907150570396) bank89622 +89623 POINT(40.33742292264369 -73.87729242688245) bank89623 +89624 POINT(41.24881107811827 -73.21450785693835) bank89624 +89625 POINT(40.036230786254094 -73.44239591486249) bank89625 +89626 POINT(40.870409713250865 -74.47825803950451) bank89626 +89627 POINT(40.71957264358535 -73.97111631164145) bank89627 +89628 POINT(40.356720103610876 -73.80033660668317) bank89628 +89629 POINT(41.100781626770775 -73.15683142698254) bank89629 +89630 POINT(40.03835778821121 -73.54497931672181) bank89630 +89631 POINT(41.48383203698263 -73.78997276951792) bank89631 +89632 POINT(41.46220304061752 -73.60918248438624) bank89632 +89633 POINT(40.878451125859826 -73.3322530959956) bank89633 +89634 POINT(41.48975397345253 -73.49856061140603) bank89634 +89635 POINT(41.084711246584064 -73.63968973844605) bank89635 +89636 POINT(40.04392611173688 -73.39600977349347) bank89636 +89637 POINT(41.4896787558462 -74.68451680529178) bank89637 +89638 POINT(40.34316778986793 -74.48335016930315) bank89638 +89639 POINT(40.925174154489014 -74.8018780305334) bank89639 +89640 POINT(40.68169143031243 -74.04896346492252) bank89640 +89641 POINT(40.58683866579269 -74.9319529811539) bank89641 +89642 POINT(40.42514390029045 -73.76819776373549) bank89642 +89643 POINT(40.764276607197324 -73.94689487351032) bank89643 +89644 POINT(39.81233824108283 -74.49428474947668) bank89644 +89645 POINT(39.92837329788512 -74.7660371247888) bank89645 +89646 POINT(41.48087731104631 -74.39999477702476) bank89646 +89647 POINT(40.94091702222151 -74.6040419310953) bank89647 +89648 POINT(41.31030611245746 -73.81059298591683) bank89648 +89649 POINT(41.49028591744489 -73.13990288197772) bank89649 +89650 POINT(39.82988631111624 -73.35635644822229) bank89650 +89651 POINT(40.52122295616767 -74.04030857957807) bank89651 +89652 POINT(41.25146862368423 -73.19001805931711) bank89652 +89653 POINT(40.69503384738719 -74.4238636827447) bank89653 +89654 POINT(39.859546301166624 -74.825949200987) bank89654 +89655 POINT(39.77754043251344 -74.16014474349652) bank89655 +89656 POINT(40.52689884277219 -74.77413766357222) bank89656 +89657 POINT(40.36569912506682 -74.85324525288611) bank89657 +89658 POINT(41.523533212288456 -74.24609347536733) bank89658 +89659 POINT(40.69994445740333 -74.59088101353318) bank89659 +89660 POINT(39.783348033418186 -73.13743161001833) bank89660 +89661 POINT(40.105658607046365 -74.59140528413283) bank89661 +89662 POINT(40.466841399501945 -74.76138222005467) bank89662 +89663 POINT(40.51638722841501 -73.69647338161369) bank89663 +89664 POINT(39.89627499349323 -73.31969664984912) bank89664 +89665 POINT(41.472605026635215 -73.42848418236495) bank89665 +89666 POINT(41.21200009068129 -74.85236749065753) bank89666 +89667 POINT(40.24540447252636 -74.75845757817474) bank89667 +89668 POINT(39.80962355541753 -74.78155672975117) bank89668 +89669 POINT(41.01887071424136 -74.91227339425082) bank89669 +89670 POINT(41.01162255542267 -74.83664105848032) bank89670 +89671 POINT(39.849110439068895 -74.45829747371117) bank89671 +89672 POINT(40.012117200286745 -74.25699585473366) bank89672 +89673 POINT(40.01264942834811 -73.96186068497103) bank89673 +89674 POINT(41.35480234234569 -74.16116024723812) bank89674 +89675 POINT(40.44379601191478 -74.52572932938673) bank89675 +89676 POINT(41.54582743006653 -74.55694092911533) bank89676 +89677 POINT(40.50406716781152 -73.31636133262991) bank89677 +89678 POINT(39.96708736291595 -73.38234118624625) bank89678 +89679 POINT(40.598510514701 -74.50011176022988) bank89679 +89680 POINT(41.641569803323776 -74.70329761612349) bank89680 +89681 POINT(40.52421289705862 -73.71537797630435) bank89681 +89682 POINT(40.48095117714859 -74.94537045508946) bank89682 +89683 POINT(40.00447885771233 -74.0924544997806) bank89683 +89684 POINT(41.283076987995884 -74.03152385384028) bank89684 +89685 POINT(41.195433564388495 -73.8644235181123) bank89685 +89686 POINT(41.03370960755796 -74.35552020602371) bank89686 +89687 POINT(41.54721384761882 -73.32398246496142) bank89687 +89688 POINT(39.79513632818396 -74.0325589435491) bank89688 +89689 POINT(41.366015442256426 -74.67095309693399) bank89689 +89690 POINT(40.90868154833603 -73.92291047078223) bank89690 +89691 POINT(40.67005933774947 -74.1273624813056) bank89691 +89692 POINT(40.5009937806084 -74.29670323340069) bank89692 +89693 POINT(39.774109704592504 -74.54536645287384) bank89693 +89694 POINT(40.07068425444954 -73.32342352983797) bank89694 +89695 POINT(40.257577859935694 -73.47560199421845) bank89695 +89696 POINT(40.923053987718816 -73.3730934447549) bank89696 +89697 POINT(41.38949924766885 -74.65728964581787) bank89697 +89698 POINT(40.89224019530343 -74.76208870825192) bank89698 +89699 POINT(40.393694450210425 -73.39386984803158) bank89699 +89700 POINT(40.0260637026587 -74.01974629787205) bank89700 +89701 POINT(40.9598002361319 -73.98668950780282) bank89701 +89702 POINT(40.670610298877705 -73.11967181258126) bank89702 +89703 POINT(40.056576136784564 -73.7245780194663) bank89703 +89704 POINT(40.84913165541137 -73.40190101126812) bank89704 +89705 POINT(39.90911670248227 -74.82670272285254) bank89705 +89706 POINT(41.218992186615395 -73.21091742931652) bank89706 +89707 POINT(41.41347075706622 -73.76247555085443) bank89707 +89708 POINT(40.964936979878765 -73.60635659469834) bank89708 +89709 POINT(40.214882275851615 -74.69827852065448) bank89709 +89710 POINT(39.80855854148317 -74.19307651242293) bank89710 +89711 POINT(40.397380054674855 -74.43587406399537) bank89711 +89712 POINT(40.71487069433649 -74.1474553823858) bank89712 +89713 POINT(39.98653238671943 -74.71991550357698) bank89713 +89714 POINT(41.23145932343458 -73.92088706299431) bank89714 +89715 POINT(41.32014842573594 -74.08889331276765) bank89715 +89716 POINT(41.46666907999927 -74.5953768077373) bank89716 +89717 POINT(41.611374408354656 -73.78070406191006) bank89717 +89718 POINT(40.57848835581484 -74.09698378453035) bank89718 +89719 POINT(39.98800931963456 -73.40500584658093) bank89719 +89720 POINT(39.830953661917306 -74.94403871410908) bank89720 +89721 POINT(40.61501824713027 -73.42537021920684) bank89721 +89722 POINT(41.41204082004229 -74.32274145617885) bank89722 +89723 POINT(40.23849324270567 -74.96821247059586) bank89723 +89724 POINT(40.614724563087904 -73.74087624475253) bank89724 +89725 POINT(41.71002667866586 -73.8009872391727) bank89725 +89726 POINT(40.94185484764124 -74.11200882398047) bank89726 +89727 POINT(41.13250600915865 -73.76713921247354) bank89727 +89728 POINT(41.24888030203393 -73.7163378065347) bank89728 +89729 POINT(40.64432238425287 -74.42312614931616) bank89729 +89730 POINT(40.91785890243464 -73.63673245901975) bank89730 +89731 POINT(39.78527669324664 -74.42931395278677) bank89731 +89732 POINT(41.1821668676557 -73.18999205017282) bank89732 +89733 POINT(41.03009982860691 -74.81986916864963) bank89733 +89734 POINT(41.18408111082558 -74.44006181838145) bank89734 +89735 POINT(40.30583582254487 -73.1219045328387) bank89735 +89736 POINT(40.65162419727118 -74.53661459785694) bank89736 +89737 POINT(39.78047394665608 -74.91471310700435) bank89737 +89738 POINT(40.54239792393763 -74.47950970616044) bank89738 +89739 POINT(41.70882880533993 -73.01638072950482) bank89739 +89740 POINT(40.84262901337314 -74.59376174995514) bank89740 +89741 POINT(40.76507471564018 -73.33317804612902) bank89741 +89742 POINT(40.38100256985731 -74.50837195346749) bank89742 +89743 POINT(40.9975921593157 -73.70168651327248) bank89743 +89744 POINT(40.91302029872354 -73.3188377251935) bank89744 +89745 POINT(41.31260671635592 -74.31816919474787) bank89745 +89746 POINT(41.62108932704886 -74.58746318980116) bank89746 +89747 POINT(40.376715252703164 -73.12928261192788) bank89747 +89748 POINT(41.43016452781622 -73.19257034745841) bank89748 +89749 POINT(40.691991751274024 -74.92303948875816) bank89749 +89750 POINT(39.940979080803146 -73.80478810120758) bank89750 +89751 POINT(39.90204451057121 -73.35277295832519) bank89751 +89752 POINT(41.01749871648194 -73.59262809695227) bank89752 +89753 POINT(39.78059305819902 -74.62664952661044) bank89753 +89754 POINT(41.19865292517739 -74.84880852206379) bank89754 +89755 POINT(40.72442546125365 -73.33645156917994) bank89755 +89756 POINT(40.579109167915284 -74.39693457669047) bank89756 +89757 POINT(40.90940851568419 -73.60327741262824) bank89757 +89758 POINT(41.302805468158674 -74.20439786894156) bank89758 +89759 POINT(41.66603275193579 -74.22149220242694) bank89759 +89760 POINT(41.62217684682591 -73.93278949241785) bank89760 +89761 POINT(41.26099410977796 -74.31797108533212) bank89761 +89762 POINT(40.446660606605036 -74.69168566446601) bank89762 +89763 POINT(41.31642810864465 -74.75019824439448) bank89763 +89764 POINT(41.459928992912594 -74.41384770922879) bank89764 +89765 POINT(41.26540410283536 -74.1740555869404) bank89765 +89766 POINT(41.29908063354825 -73.2740678738228) bank89766 +89767 POINT(40.043609341768054 -74.55090015899869) bank89767 +89768 POINT(41.61591924051857 -74.11859931735682) bank89768 +89769 POINT(40.9300388164551 -73.52871182311998) bank89769 +89770 POINT(41.462377334474944 -73.56309858832344) bank89770 +89771 POINT(39.726912928772784 -74.70114674005357) bank89771 +89772 POINT(39.769655767598024 -74.05047330308541) bank89772 +89773 POINT(40.738430487939006 -74.18288443208282) bank89773 +89774 POINT(40.49489981367243 -74.89702369226995) bank89774 +89775 POINT(39.87080410057097 -73.64908222005596) bank89775 +89776 POINT(41.07900053427609 -74.7294684531031) bank89776 +89777 POINT(40.89675921795647 -74.2913539216526) bank89777 +89778 POINT(40.46993452447251 -73.70773751903049) bank89778 +89779 POINT(40.25601203448221 -73.89086218357053) bank89779 +89780 POINT(40.203561085167145 -74.84722985899222) bank89780 +89781 POINT(41.162572209760604 -73.0932234235833) bank89781 +89782 POINT(41.67732500403455 -73.97011323136732) bank89782 +89783 POINT(40.85266657213063 -74.42708215254176) bank89783 +89784 POINT(41.25723448594153 -74.86220988553985) bank89784 +89785 POINT(40.91864140436604 -74.08784033781077) bank89785 +89786 POINT(40.10895679249741 -74.62260328926942) bank89786 +89787 POINT(41.03061337189799 -74.21365661959777) bank89787 +89788 POINT(40.36170853445505 -74.0246105651916) bank89788 +89789 POINT(41.51669199757726 -74.46409317920903) bank89789 +89790 POINT(40.48699441536855 -74.02119504192868) bank89790 +89791 POINT(40.839299247212054 -73.95672048645514) bank89791 +89792 POINT(40.667153116129455 -73.77357507469814) bank89792 +89793 POINT(40.73858542793895 -74.38444552758617) bank89793 +89794 POINT(41.55186381638455 -74.19294178407158) bank89794 +89795 POINT(41.35733996650425 -74.92268946854456) bank89795 +89796 POINT(40.5799512168821 -74.15945250614742) bank89796 +89797 POINT(40.48085231982975 -74.71668455662014) bank89797 +89798 POINT(41.170215730718155 -73.97872864881869) bank89798 +89799 POINT(41.00549344090467 -74.74809588951705) bank89799 +89800 POINT(40.270160555002455 -74.58964751289847) bank89800 +89801 POINT(40.96230872949949 -73.86348568602007) bank89801 +89802 POINT(41.09974551810607 -74.64690245294842) bank89802 +89803 POINT(40.43076092195581 -74.14750821942981) bank89803 +89804 POINT(39.71712371534769 -74.69125438543524) bank89804 +89805 POINT(39.993319582332624 -73.13853274503754) bank89805 +89806 POINT(41.17265078685276 -74.1984317175462) bank89806 +89807 POINT(40.87796502142681 -74.43751540267095) bank89807 +89808 POINT(40.05066527932512 -74.29144980979855) bank89808 +89809 POINT(40.78173479743136 -74.19708621551543) bank89809 +89810 POINT(40.26410136107473 -73.8393810650338) bank89810 +89811 POINT(40.24905698475338 -74.97435720676565) bank89811 +89812 POINT(41.68206077752651 -73.85879369795879) bank89812 +89813 POINT(41.61834246244827 -73.27917700420288) bank89813 +89814 POINT(41.70072974819403 -73.64795158855793) bank89814 +89815 POINT(41.02498252186501 -73.50530375424329) bank89815 +89816 POINT(41.007217252763276 -73.02900986663408) bank89816 +89817 POINT(41.26692185241378 -74.90695029558243) bank89817 +89818 POINT(39.75500513794466 -73.09251863215543) bank89818 +89819 POINT(40.99256227730274 -73.46130499040922) bank89819 +89820 POINT(39.8269993288135 -74.66631691821047) bank89820 +89821 POINT(41.24208449232875 -73.83060387162072) bank89821 +89822 POINT(39.83579639857993 -73.51516823852192) bank89822 +89823 POINT(41.55788179542103 -74.26686730942502) bank89823 +89824 POINT(41.29755993972537 -74.84656354109339) bank89824 +89825 POINT(39.73815500841137 -73.0713737307547) bank89825 +89826 POINT(39.74582454349565 -74.1884347065673) bank89826 +89827 POINT(41.09216924148524 -74.01554501532033) bank89827 +89828 POINT(41.022897258018176 -73.75319290833707) bank89828 +89829 POINT(41.297168048226105 -74.57208585976478) bank89829 +89830 POINT(41.61328180497979 -73.15722404564184) bank89830 +89831 POINT(40.631054585582774 -73.4716081959624) bank89831 +89832 POINT(40.74840349953573 -73.7550551184085) bank89832 +89833 POINT(41.23848836293959 -74.16836466934113) bank89833 +89834 POINT(40.91197995530394 -74.0067607106835) bank89834 +89835 POINT(39.92271573927341 -74.44625074761382) bank89835 +89836 POINT(41.337618452113134 -74.67107783790351) bank89836 +89837 POINT(41.444275049615605 -74.30056621436654) bank89837 +89838 POINT(40.344088502052614 -74.15772038116417) bank89838 +89839 POINT(39.770913319199884 -73.79589792948602) bank89839 +89840 POINT(41.1143096825573 -73.6762679425611) bank89840 +89841 POINT(40.04410616323268 -74.46882150353248) bank89841 +89842 POINT(41.44599972021207 -73.54685442594815) bank89842 +89843 POINT(40.9816793888026 -74.31179570713434) bank89843 +89844 POINT(40.569576961713054 -73.4169198689809) bank89844 +89845 POINT(40.16768261217476 -73.10650199278744) bank89845 +89846 POINT(41.153320171431204 -73.33017959732035) bank89846 +89847 POINT(41.02620120851317 -74.46873725458978) bank89847 +89848 POINT(40.57689191503008 -73.71985941553454) bank89848 +89849 POINT(41.33219897826283 -73.98700366748919) bank89849 +89850 POINT(41.062939330689865 -74.83039375343955) bank89850 +89851 POINT(40.56887572369747 -74.75709828829869) bank89851 +89852 POINT(41.240283493105025 -74.61537509420985) bank89852 +89853 POINT(41.170413679181436 -74.590846462228) bank89853 +89854 POINT(41.69385470384891 -74.0975171168638) bank89854 +89855 POINT(40.35260442076701 -74.3559354607239) bank89855 +89856 POINT(39.874437036359716 -73.29910543013803) bank89856 +89857 POINT(40.55184120228235 -74.704684475109) bank89857 +89858 POINT(40.93114256170474 -73.36765185603159) bank89858 +89859 POINT(40.090946696203815 -74.45985279208303) bank89859 +89860 POINT(40.94538026745446 -73.20977587671067) bank89860 +89861 POINT(41.27437145089865 -73.27721637435516) bank89861 +89862 POINT(40.41015726733029 -74.27400206513812) bank89862 +89863 POINT(40.29773664236685 -74.9535964480932) bank89863 +89864 POINT(41.093198623784325 -73.84320924349694) bank89864 +89865 POINT(40.88095704622542 -73.27680390525391) bank89865 +89866 POINT(41.06753357901618 -74.4191410373448) bank89866 +89867 POINT(40.17759931256406 -74.31798691263917) bank89867 +89868 POINT(41.39305811916502 -74.61073470068361) bank89868 +89869 POINT(40.37528566773246 -73.71098794402961) bank89869 +89870 POINT(41.06828485545846 -73.68616923251606) bank89870 +89871 POINT(41.00987364802675 -73.26264869498513) bank89871 +89872 POINT(41.46308641604329 -73.70979435802998) bank89872 +89873 POINT(40.114784296619746 -74.09521183259521) bank89873 +89874 POINT(40.109421254916356 -74.59584017273988) bank89874 +89875 POINT(40.123632383033446 -74.34605146536465) bank89875 +89876 POINT(40.41611298740873 -73.92342909670903) bank89876 +89877 POINT(40.101559907977155 -73.23284780912473) bank89877 +89878 POINT(39.81820067525213 -74.39631528577326) bank89878 +89879 POINT(39.757383467801475 -74.96468394490044) bank89879 +89880 POINT(41.66748838584329 -74.73872959961858) bank89880 +89881 POINT(41.62048122138112 -73.9917657309803) bank89881 +89882 POINT(39.76568178575948 -74.20699578402863) bank89882 +89883 POINT(40.895869680380926 -74.09691993994817) bank89883 +89884 POINT(39.78596696527945 -74.08862644209779) bank89884 +89885 POINT(41.54445066177089 -74.90525622012628) bank89885 +89886 POINT(40.544549060741495 -73.74007340943328) bank89886 +89887 POINT(41.0332352688332 -73.75719726553747) bank89887 +89888 POINT(41.1767837804447 -73.59472007656228) bank89888 +89889 POINT(41.28226023372754 -74.77798282705834) bank89889 +89890 POINT(40.50607602454723 -73.08891410166983) bank89890 +89891 POINT(40.18588095122228 -73.87336359419507) bank89891 +89892 POINT(41.18975748236198 -74.15075458207944) bank89892 +89893 POINT(40.03138336780095 -74.59889318799135) bank89893 +89894 POINT(40.85684211047685 -74.38056385408787) bank89894 +89895 POINT(40.61244990294947 -74.65167337714776) bank89895 +89896 POINT(40.611253125967835 -74.86689642624448) bank89896 +89897 POINT(40.57962945025536 -74.28829639377376) bank89897 +89898 POINT(39.80113916295198 -74.62934949231817) bank89898 +89899 POINT(39.95495564699584 -73.07594251619192) bank89899 +89900 POINT(40.53665634135344 -73.69999630298375) bank89900 +89901 POINT(39.95580491159292 -74.72477311250566) bank89901 +89902 POINT(41.70870088210252 -73.35331223384942) bank89902 +89903 POINT(41.620323155638175 -74.33145817280524) bank89903 +89904 POINT(40.68124514356328 -74.41089827314113) bank89904 +89905 POINT(40.09977845068621 -74.2729378061931) bank89905 +89906 POINT(41.30249030176908 -74.9493293628933) bank89906 +89907 POINT(40.93929089206484 -74.50835116683083) bank89907 +89908 POINT(40.53134492354659 -74.47172438179179) bank89908 +89909 POINT(41.037672380303015 -74.96588741234052) bank89909 +89910 POINT(40.33736921831009 -73.20510491067189) bank89910 +89911 POINT(40.8832754308698 -73.33528957736708) bank89911 +89912 POINT(40.92513724789309 -74.03237957409894) bank89912 +89913 POINT(41.16311819433753 -74.42812965153793) bank89913 +89914 POINT(40.799234509143766 -73.4515951305854) bank89914 +89915 POINT(41.04128579384076 -74.61390731269432) bank89915 +89916 POINT(40.08131892174436 -73.69697065367502) bank89916 +89917 POINT(41.42439955482767 -74.39368746309974) bank89917 +89918 POINT(41.37595194018211 -73.17609552282028) bank89918 +89919 POINT(41.559149390444624 -73.2188672742245) bank89919 +89920 POINT(41.689426745499624 -73.98855027426279) bank89920 +89921 POINT(41.02556742928648 -73.51627769453084) bank89921 +89922 POINT(40.54874218265273 -73.37020235938645) bank89922 +89923 POINT(41.00605880325349 -73.25103534484836) bank89923 +89924 POINT(41.36825528809893 -74.59336960326316) bank89924 +89925 POINT(39.85919389426109 -74.47839685972187) bank89925 +89926 POINT(41.05384116823521 -73.85887980248906) bank89926 +89927 POINT(40.18952708072215 -74.99882047695745) bank89927 +89928 POINT(40.75294580608947 -73.61165586823267) bank89928 +89929 POINT(39.95944376529438 -74.22176277445129) bank89929 +89930 POINT(41.50144087366578 -74.89300046849651) bank89930 +89931 POINT(40.930822108559184 -74.37444090874287) bank89931 +89932 POINT(40.10816631103403 -74.4088858245194) bank89932 +89933 POINT(40.53357583019055 -73.34275434206134) bank89933 +89934 POINT(41.00764007561604 -74.37258392472843) bank89934 +89935 POINT(41.61567409865663 -74.13707739236673) bank89935 +89936 POINT(40.17088710068123 -74.12079925445447) bank89936 +89937 POINT(41.09993855158664 -73.14575322706031) bank89937 +89938 POINT(41.4382797374442 -73.03612819454288) bank89938 +89939 POINT(40.10270629973049 -74.34729113695388) bank89939 +89940 POINT(40.12149795653739 -73.63952448243428) bank89940 +89941 POINT(40.111691029422055 -73.051189052034) bank89941 +89942 POINT(41.08091915807417 -73.17499650968517) bank89942 +89943 POINT(40.418421105667335 -73.46175007549337) bank89943 +89944 POINT(39.86751385892853 -73.30909542124729) bank89944 +89945 POINT(41.014838336461146 -73.24567151501192) bank89945 +89946 POINT(39.74885555367182 -73.88157504721929) bank89946 +89947 POINT(41.624752862399234 -74.98778767213742) bank89947 +89948 POINT(39.918345087880645 -73.5203118590879) bank89948 +89949 POINT(41.56285873953372 -73.29025052278617) bank89949 +89950 POINT(39.76967357825145 -74.09714625802467) bank89950 +89951 POINT(40.62091998105329 -73.63578763888756) bank89951 +89952 POINT(40.3786012378179 -73.01678191064444) bank89952 +89953 POINT(39.913221329890376 -74.13137179097137) bank89953 +89954 POINT(41.51473749450662 -74.79381039467451) bank89954 +89955 POINT(41.269769041303455 -73.52275257892747) bank89955 +89956 POINT(41.24107897926103 -73.0762955947096) bank89956 +89957 POINT(41.07405221466498 -74.44432151114192) bank89957 +89958 POINT(40.05580153734514 -73.04528269782806) bank89958 +89959 POINT(40.62276348889375 -74.44667984454162) bank89959 +89960 POINT(40.4913106727377 -73.59036873889539) bank89960 +89961 POINT(40.13826319036944 -73.8879319044856) bank89961 +89962 POINT(40.664460262079295 -73.37214347208176) bank89962 +89963 POINT(40.18068436940807 -73.33856449243108) bank89963 +89964 POINT(40.82305864375876 -74.6484910913053) bank89964 +89965 POINT(41.20683075526025 -74.57816471584157) bank89965 +89966 POINT(41.67285177418005 -74.9127242439894) bank89966 +89967 POINT(41.0202454248566 -73.01547637634319) bank89967 +89968 POINT(40.02058758426081 -74.677988548249) bank89968 +89969 POINT(41.14909329415238 -74.06312737713701) bank89969 +89970 POINT(39.76510091783235 -74.83900550829973) bank89970 +89971 POINT(41.047756864718345 -73.59476006103269) bank89971 +89972 POINT(41.121458086373856 -74.39062536984544) bank89972 +89973 POINT(40.560517096166315 -74.85434256689048) bank89973 +89974 POINT(40.55509233060072 -73.4287388785833) bank89974 +89975 POINT(41.201597097695704 -74.17935924819376) bank89975 +89976 POINT(40.26277200080142 -74.16106119582214) bank89976 +89977 POINT(41.58323895103228 -74.15946050871594) bank89977 +89978 POINT(40.180030940994776 -74.61894489008718) bank89978 +89979 POINT(40.14867335801534 -73.41471347249973) bank89979 +89980 POINT(41.55745577445659 -74.80063615971963) bank89980 +89981 POINT(39.798619091388 -74.30235199201336) bank89981 +89982 POINT(41.20019892179122 -73.4451607302064) bank89982 +89983 POINT(40.141536035839195 -73.22250371361638) bank89983 +89984 POINT(40.017878869382024 -73.83510578884915) bank89984 +89985 POINT(40.04216389410364 -73.03864721341371) bank89985 +89986 POINT(41.52115543451689 -73.88389618455197) bank89986 +89987 POINT(40.57390299977791 -73.10206211644753) bank89987 +89988 POINT(41.06844527401304 -74.83679070306304) bank89988 +89989 POINT(41.13244984456291 -73.93889705847874) bank89989 +89990 POINT(40.58438976981773 -73.58134097935988) bank89990 +89991 POINT(39.723273448580116 -73.95352425969584) bank89991 +89992 POINT(41.45081183529906 -73.60147422439988) bank89992 +89993 POINT(41.20050569602311 -74.50147913361751) bank89993 +89994 POINT(41.26030033810099 -73.83417486898225) bank89994 +89995 POINT(39.954606425398374 -73.2106781807277) bank89995 +89996 POINT(41.042954719874636 -73.98578355191539) bank89996 +89997 POINT(41.151330170789066 -74.70862502781742) bank89997 +89998 POINT(40.59862624506544 -74.85412955182846) bank89998 +89999 POINT(40.831262054858534 -73.12196636344783) bank89999 +90000 POINT(40.219380858826185 -74.60795825456299) bank90000 +90001 POINT(39.971440049384306 -74.67883309776921) bank90001 +90002 POINT(39.880779137081475 -74.24975293240813) bank90002 +90003 POINT(41.64834730030765 -73.93437360095785) bank90003 +90004 POINT(39.75312611918385 -74.3026280857328) bank90004 +90005 POINT(40.14085566579297 -74.94092569555855) bank90005 +90006 POINT(40.25515241360598 -74.2366963063292) bank90006 +90007 POINT(41.15717329533302 -74.36823159054092) bank90007 +90008 POINT(41.355467728285745 -73.7584376609713) bank90008 +90009 POINT(39.821548174164 -73.12007357685879) bank90009 +90010 POINT(40.865535427902564 -74.51331852804266) bank90010 +90011 POINT(41.2745855515265 -74.4820901421038) bank90011 +90012 POINT(40.40183180298828 -73.44634593521445) bank90012 +90013 POINT(40.49648393092948 -74.11093741287115) bank90013 +90014 POINT(41.11116703604116 -73.2425844818401) bank90014 +90015 POINT(40.261639248102156 -74.98930641651941) bank90015 +90016 POINT(40.446899819577105 -74.25726150596034) bank90016 +90017 POINT(39.76908002447582 -74.79341663891356) bank90017 +90018 POINT(41.18897944668762 -74.55781152473682) bank90018 +90019 POINT(40.7284764307992 -73.42283385792041) bank90019 +90020 POINT(41.32932297469445 -73.37662853383348) bank90020 +90021 POINT(41.119927787851175 -74.88172090919846) bank90021 +90022 POINT(41.0528539047889 -73.64778064375164) bank90022 +90023 POINT(41.32692532568197 -74.67986521591455) bank90023 +90024 POINT(40.74488412598451 -73.45794046154822) bank90024 +90025 POINT(40.887834301160716 -74.41123813119653) bank90025 +90026 POINT(41.29468177842544 -73.71024806520668) bank90026 +90027 POINT(39.72918971517622 -74.35180807230931) bank90027 +90028 POINT(40.515019223716614 -73.03753834133154) bank90028 +90029 POINT(41.375807631948106 -73.26590412130982) bank90029 +90030 POINT(40.52102935231411 -73.44127430842161) bank90030 +90031 POINT(41.05209085234607 -73.89217381329506) bank90031 +90032 POINT(40.68989468166869 -74.82732956622827) bank90032 +90033 POINT(40.35974720953292 -74.50666930632849) bank90033 +90034 POINT(40.91695112437282 -73.50790834432158) bank90034 +90035 POINT(41.27218410599412 -73.93711185669154) bank90035 +90036 POINT(40.925114724764455 -74.91581654637909) bank90036 +90037 POINT(39.72791129519211 -73.9192228782262) bank90037 +90038 POINT(41.54935532466356 -74.95290183938737) bank90038 +90039 POINT(40.94184031071051 -73.7711375429702) bank90039 +90040 POINT(41.58151787749656 -73.4745638852117) bank90040 +90041 POINT(40.66298447450413 -73.75002877809777) bank90041 +90042 POINT(41.698131891640124 -74.90346923092422) bank90042 +90043 POINT(41.31209600940931 -73.13428829312129) bank90043 +90044 POINT(41.16343929878483 -74.96481466986496) bank90044 +90045 POINT(39.92569627366968 -74.91055009260013) bank90045 +90046 POINT(40.98022988101184 -74.92395079632473) bank90046 +90047 POINT(39.808887534579945 -73.74521933086318) bank90047 +90048 POINT(40.51008508474901 -73.33828829749383) bank90048 +90049 POINT(40.11032402296933 -74.90459599994611) bank90049 +90050 POINT(40.31715606418288 -74.22786998110548) bank90050 +90051 POINT(41.61859152623678 -74.77283082995311) bank90051 +90052 POINT(41.28681607284271 -73.17626690195712) bank90052 +90053 POINT(41.48710486434138 -73.49640254314613) bank90053 +90054 POINT(40.1550094961255 -74.6956560393966) bank90054 +90055 POINT(39.86859758077873 -73.84679937848247) bank90055 +90056 POINT(41.20088285956386 -74.14598872405716) bank90056 +90057 POINT(40.119423089655676 -74.40123181230553) bank90057 +90058 POINT(40.31368809176283 -73.3678627218564) bank90058 +90059 POINT(41.452751380973645 -74.10873492461626) bank90059 +90060 POINT(41.55802875421634 -73.35611127619103) bank90060 +90061 POINT(41.556060519081335 -73.16208206117689) bank90061 +90062 POINT(40.68691981980577 -74.14834814301743) bank90062 +90063 POINT(40.153809491421995 -74.18617764662905) bank90063 +90064 POINT(41.02238325963973 -74.64950096546123) bank90064 +90065 POINT(41.69201730512697 -74.47723777996669) bank90065 +90066 POINT(40.67032557495267 -74.4050238579358) bank90066 +90067 POINT(39.90972063443524 -73.2320286205813) bank90067 +90068 POINT(40.69628182618358 -73.20611798064803) bank90068 +90069 POINT(41.08224729386659 -73.2436876551765) bank90069 +90070 POINT(40.05101325477883 -73.67156144911117) bank90070 +90071 POINT(40.53830512524885 -73.14903436915117) bank90071 +90072 POINT(40.15288658024798 -74.68233406610581) bank90072 +90073 POINT(39.998667012274986 -74.8819844985945) bank90073 +90074 POINT(39.99608780760746 -73.1184415336519) bank90074 +90075 POINT(40.73491976282038 -73.35832951821632) bank90075 +90076 POINT(40.10663626124614 -74.54975981061939) bank90076 +90077 POINT(40.13353540699422 -74.89398961769948) bank90077 +90078 POINT(40.11218159618648 -73.96414948195857) bank90078 +90079 POINT(40.03436328809231 -74.07509175484175) bank90079 +90080 POINT(40.96034480753842 -74.49532063932445) bank90080 +90081 POINT(41.18503755134577 -73.57422397985098) bank90081 +90082 POINT(39.92783969782086 -73.89816765712987) bank90082 +90083 POINT(40.563510274282486 -74.49170752996594) bank90083 +90084 POINT(40.85924189396124 -74.49305872013355) bank90084 +90085 POINT(40.14245776537922 -74.78036869539301) bank90085 +90086 POINT(39.84524274328068 -74.4983509336256) bank90086 +90087 POINT(41.55700594734409 -74.22023106372905) bank90087 +90088 POINT(40.57000373317704 -73.96579077166767) bank90088 +90089 POINT(41.12553660643604 -73.50116646877177) bank90089 +90090 POINT(41.566181549188336 -74.78681205438485) bank90090 +90091 POINT(39.872441079301716 -73.42906990062684) bank90091 +90092 POINT(39.9271755366952 -73.03196794631009) bank90092 +90093 POINT(39.99426415782652 -73.30264575378744) bank90093 +90094 POINT(41.27956565134959 -73.63733461262659) bank90094 +90095 POINT(40.49925355013459 -74.06133020800124) bank90095 +90096 POINT(40.17644194906676 -73.22338021505523) bank90096 +90097 POINT(40.81945409958543 -74.13380397326875) bank90097 +90098 POINT(41.56802053600026 -74.82584472043351) bank90098 +90099 POINT(39.827277764576884 -73.8792408693188) bank90099 +90100 POINT(41.19314169352297 -73.09016746550779) bank90100 +90101 POINT(40.1768055307018 -74.05014132812266) bank90101 +90102 POINT(40.41911195807946 -74.97199051708034) bank90102 +90103 POINT(40.70885646196828 -74.03596542233026) bank90103 +90104 POINT(40.95229156966496 -74.18752573210067) bank90104 +90105 POINT(40.33513666896937 -74.3774444052697) bank90105 +90106 POINT(40.26348057894403 -73.56427513455628) bank90106 +90107 POINT(40.22726812419995 -73.35928406659036) bank90107 +90108 POINT(39.881769425717195 -73.13782529554648) bank90108 +90109 POINT(40.418250098720044 -74.7786438618987) bank90109 +90110 POINT(41.24179419870111 -74.86206336139024) bank90110 +90111 POINT(40.382063304329215 -74.82586306652905) bank90111 +90112 POINT(40.470817340560174 -73.16873066120866) bank90112 +90113 POINT(40.52152597635469 -73.45091553116835) bank90113 +90114 POINT(41.62601075403533 -73.89528299278567) bank90114 +90115 POINT(40.57284149783063 -73.68509864134509) bank90115 +90116 POINT(40.126833351852184 -74.20834750101294) bank90116 +90117 POINT(40.4126045680258 -74.5065681873661) bank90117 +90118 POINT(40.649531178996995 -73.37859085932054) bank90118 +90119 POINT(40.013880194312456 -73.86567437490653) bank90119 +90120 POINT(40.773223409036945 -73.73984612645259) bank90120 +90121 POINT(41.280605676773746 -73.98132060269141) bank90121 +90122 POINT(41.27268017944446 -74.75664101040425) bank90122 +90123 POINT(39.77056364441526 -73.44471358099301) bank90123 +90124 POINT(39.71384565071247 -73.4079021645246) bank90124 +90125 POINT(41.70966212839689 -73.46116873822044) bank90125 +90126 POINT(41.62838357356932 -74.31854937678592) bank90126 +90127 POINT(40.73303237120252 -73.93772001698233) bank90127 +90128 POINT(41.676256492579114 -73.02679111719371) bank90128 +90129 POINT(40.918770860371296 -73.13110224560957) bank90129 +90130 POINT(41.59913433195854 -73.21776580707665) bank90130 +90131 POINT(41.344848340017634 -73.75999942972774) bank90131 +90132 POINT(41.45340814691257 -74.94916252313696) bank90132 +90133 POINT(40.81371141394188 -74.17449465736436) bank90133 +90134 POINT(40.7605545770338 -73.85513762861993) bank90134 +90135 POINT(40.15552202971134 -74.32061212415653) bank90135 +90136 POINT(40.359461912068625 -73.7630640772684) bank90136 +90137 POINT(41.15236939327358 -74.36948284425041) bank90137 +90138 POINT(41.09150660316929 -73.92948964715721) bank90138 +90139 POINT(40.222724804955675 -74.83030352941844) bank90139 +90140 POINT(41.12612390177723 -73.59879399711184) bank90140 +90141 POINT(40.63216733760953 -73.9448413159296) bank90141 +90142 POINT(40.27312830565775 -73.92530391830073) bank90142 +90143 POINT(40.85757709607529 -74.39352780231434) bank90143 +90144 POINT(40.63880148903963 -74.10607732229474) bank90144 +90145 POINT(40.291332887074034 -74.07210099879069) bank90145 +90146 POINT(40.22373989955637 -74.53758336965781) bank90146 +90147 POINT(40.655294252119795 -74.47414911257472) bank90147 +90148 POINT(40.50500502702805 -74.91491420323075) bank90148 +90149 POINT(39.752085051622394 -74.87324619817547) bank90149 +90150 POINT(39.90308333916805 -73.91844193976269) bank90150 +90151 POINT(40.33982474454394 -74.06320734694184) bank90151 +90152 POINT(40.706026840857085 -73.1648858110761) bank90152 +90153 POINT(40.95245245999048 -73.28175129092094) bank90153 +90154 POINT(40.211873699484 -73.69965261121904) bank90154 +90155 POINT(40.177627278756255 -73.64240871220782) bank90155 +90156 POINT(41.09970031990698 -75.00311706331661) bank90156 +90157 POINT(39.79698210760932 -74.9912416836967) bank90157 +90158 POINT(40.436139005304994 -73.24504525187042) bank90158 +90159 POINT(39.90299904393335 -73.28640795469502) bank90159 +90160 POINT(40.36223263186781 -73.72021934294376) bank90160 +90161 POINT(40.563970263595124 -73.22190096875715) bank90161 +90162 POINT(41.16733939216388 -74.98256619014293) bank90162 +90163 POINT(41.49526964694739 -73.84963370442411) bank90163 +90164 POINT(39.838020470546255 -73.2235905042969) bank90164 +90165 POINT(41.10120477443639 -74.46989350119307) bank90165 +90166 POINT(41.26648586608208 -73.87350084019442) bank90166 +90167 POINT(41.387121476574734 -74.15934586395348) bank90167 +90168 POINT(40.39891740480575 -74.54135926520325) bank90168 +90169 POINT(40.473190803485416 -73.47060473306965) bank90169 +90170 POINT(40.80904360521292 -73.1454081869983) bank90170 +90171 POINT(39.769181511019724 -74.61959588174894) bank90171 +90172 POINT(39.83211314798838 -73.43995700500201) bank90172 +90173 POINT(41.02753628198963 -74.7610030024102) bank90173 +90174 POINT(40.42447546683025 -74.33537248728165) bank90174 +90175 POINT(41.33945020134234 -73.49687070791754) bank90175 +90176 POINT(41.1682435894994 -73.30127913493342) bank90176 +90177 POINT(40.21846855088464 -73.36156550629539) bank90177 +90178 POINT(40.135250398468656 -74.66305125808964) bank90178 +90179 POINT(39.73331080854393 -74.60547055021044) bank90179 +90180 POINT(40.02351580399247 -74.90382754551217) bank90180 +90181 POINT(40.310981657100974 -73.18397147980004) bank90181 +90182 POINT(40.205337968491776 -73.84928029650062) bank90182 +90183 POINT(40.156450833206826 -74.51618492411617) bank90183 +90184 POINT(41.20944157370497 -73.36983608851567) bank90184 +90185 POINT(40.08934935112969 -74.54576332970419) bank90185 +90186 POINT(39.92949740941005 -74.6042508624172) bank90186 +90187 POINT(40.522674548105456 -74.15346702947492) bank90187 +90188 POINT(41.541975748241846 -74.56660571456885) bank90188 +90189 POINT(41.67581728328847 -74.35460757902783) bank90189 +90190 POINT(40.30172174046749 -74.93049055701397) bank90190 +90191 POINT(40.74345762428751 -74.59837001011245) bank90191 +90192 POINT(41.41647218472305 -73.60442745112283) bank90192 +90193 POINT(40.80553918500249 -73.53712073620761) bank90193 +90194 POINT(39.78784440255517 -74.46356467188657) bank90194 +90195 POINT(40.44843881920019 -73.89147725643477) bank90195 +90196 POINT(40.7726343462434 -73.14094759550069) bank90196 +90197 POINT(41.124435306504964 -74.0720459214998) bank90197 +90198 POINT(41.03161304713386 -74.54351366827899) bank90198 +90199 POINT(40.75348399177665 -73.13811432487955) bank90199 +90200 POINT(40.76185387798516 -74.8492222096711) bank90200 +90201 POINT(41.08137829390776 -73.68746783441563) bank90201 +90202 POINT(40.921289660722245 -73.70616629020046) bank90202 +90203 POINT(40.783264362199645 -74.3217853432193) bank90203 +90204 POINT(39.92360426005672 -73.3914676655761) bank90204 +90205 POINT(40.24161319616615 -73.97864177890538) bank90205 +90206 POINT(40.46469645069403 -74.49558797722716) bank90206 +90207 POINT(40.38100760840474 -74.44825065342908) bank90207 +90208 POINT(40.02044855314951 -73.73090872082074) bank90208 +90209 POINT(40.558656227340855 -73.29032860124032) bank90209 +90210 POINT(40.970489630828816 -74.89640292151228) bank90210 +90211 POINT(40.02894919482853 -74.71806119554698) bank90211 +90212 POINT(40.07158167422683 -74.37515970166409) bank90212 +90213 POINT(40.37733824504738 -73.5195195133801) bank90213 +90214 POINT(40.47515882360106 -74.47948144309385) bank90214 +90215 POINT(40.80043869064019 -73.01230632343153) bank90215 +90216 POINT(40.06768864904506 -74.05147141833493) bank90216 +90217 POINT(40.67411053791078 -74.60700807640988) bank90217 +90218 POINT(40.04881306427195 -73.27635790087449) bank90218 +90219 POINT(40.28799187498518 -73.52093521687108) bank90219 +90220 POINT(40.022265212068916 -73.07294032989616) bank90220 +90221 POINT(40.335691822740934 -74.3812081097091) bank90221 +90222 POINT(40.40781804310531 -74.60668818975319) bank90222 +90223 POINT(40.75401600857706 -73.51940544733344) bank90223 +90224 POINT(39.91602105519109 -74.9459131642194) bank90224 +90225 POINT(40.86331074689737 -74.49023255929133) bank90225 +90226 POINT(40.4122602638349 -73.22686724416576) bank90226 +90227 POINT(39.943579119904804 -74.51673307399898) bank90227 +90228 POINT(40.84485570799494 -73.51773068441456) bank90228 +90229 POINT(39.80942283925405 -74.36877518157374) bank90229 +90230 POINT(40.1647499593671 -73.57953189196431) bank90230 +90231 POINT(41.16176508166107 -74.39228030555293) bank90231 +90232 POINT(39.911604727184766 -74.32535462887662) bank90232 +90233 POINT(39.865283814042506 -73.00601792338786) bank90233 +90234 POINT(41.468184664860814 -75.00302451346263) bank90234 +90235 POINT(41.05950448873835 -73.80603858292857) bank90235 +90236 POINT(40.38296333651114 -73.91659947192605) bank90236 +90237 POINT(41.35807583605216 -73.51373011947842) bank90237 +90238 POINT(40.862794863757166 -74.91088964488198) bank90238 +90239 POINT(41.18365045639232 -74.42644421076517) bank90239 +90240 POINT(40.88989776053084 -74.10092142250294) bank90240 +90241 POINT(41.540980188189835 -74.03318468786517) bank90241 +90242 POINT(39.83056597625454 -74.9398181689995) bank90242 +90243 POINT(41.61638868514122 -73.84439458486133) bank90243 +90244 POINT(40.990388093211955 -73.03201643240747) bank90244 +90245 POINT(39.96500722163857 -73.85532237900148) bank90245 +90246 POINT(40.331592542693194 -74.3158290192654) bank90246 +90247 POINT(39.82308189758963 -73.09460173100469) bank90247 +90248 POINT(39.988787487107686 -74.91057308129048) bank90248 +90249 POINT(39.83148061689281 -74.1159126998738) bank90249 +90250 POINT(41.19846882035425 -74.73242898246163) bank90250 +90251 POINT(41.34244441064448 -74.09643818910295) bank90251 +90252 POINT(41.48733420834825 -73.12814135567525) bank90252 +90253 POINT(39.847868692830545 -74.05062108184606) bank90253 +90254 POINT(41.423352183021066 -74.45180441994398) bank90254 +90255 POINT(40.85589631480911 -73.49251685544122) bank90255 +90256 POINT(41.12804900800794 -73.45332574785057) bank90256 +90257 POINT(40.50192826089646 -73.90361101233846) bank90257 +90258 POINT(40.627394035463816 -73.42611792977823) bank90258 +90259 POINT(40.9639096741819 -74.68319282695174) bank90259 +90260 POINT(40.91037699014074 -73.70083369180112) bank90260 +90261 POINT(40.2565943640937 -73.50926053791066) bank90261 +90262 POINT(40.3374565043495 -73.95171254346816) bank90262 +90263 POINT(40.87337412966047 -74.18294003249999) bank90263 +90264 POINT(40.40825785301612 -73.46353626962029) bank90264 +90265 POINT(41.494374680898126 -73.13804109681175) bank90265 +90266 POINT(40.303829668997764 -73.48865473630315) bank90266 +90267 POINT(40.11385644327322 -73.13253440360711) bank90267 +90268 POINT(40.62714314098186 -73.17548954146699) bank90268 +90269 POINT(39.779269032076044 -73.75517935280548) bank90269 +90270 POINT(41.093674580645015 -74.66170766230091) bank90270 +90271 POINT(41.116257761486374 -73.63170216811736) bank90271 +90272 POINT(39.76639998939309 -74.66576672594017) bank90272 +90273 POINT(39.905152654459066 -73.50539170010796) bank90273 +90274 POINT(41.35395579336522 -73.11951781955787) bank90274 +90275 POINT(39.875514169892504 -73.54069307242865) bank90275 +90276 POINT(39.80021025174641 -74.58553438180785) bank90276 +90277 POINT(40.26428748794898 -74.60794087054339) bank90277 +90278 POINT(41.56897766641141 -73.99814536634152) bank90278 +90279 POINT(40.923570457411955 -73.30001131909044) bank90279 +90280 POINT(40.049287857451645 -73.33203620976434) bank90280 +90281 POINT(40.31563368808593 -73.6274633705931) bank90281 +90282 POINT(41.03178370162285 -73.87079505237116) bank90282 +90283 POINT(41.32227260681017 -73.66096652581663) bank90283 +90284 POINT(41.20037158082657 -74.61109835405955) bank90284 +90285 POINT(41.52925648666025 -73.98254470521998) bank90285 +90286 POINT(41.655731246093644 -74.33009604895439) bank90286 +90287 POINT(40.89703278341028 -73.31583443711187) bank90287 +90288 POINT(40.30149860064016 -73.54535546942091) bank90288 +90289 POINT(41.23117702054722 -73.42440686968742) bank90289 +90290 POINT(39.92204285404712 -73.58898708066158) bank90290 +90291 POINT(41.30181504035598 -74.64202510500589) bank90291 +90292 POINT(41.50738657428463 -74.99410461800879) bank90292 +90293 POINT(41.67143062141187 -74.6938345246251) bank90293 +90294 POINT(41.454543740115305 -74.55355340058141) bank90294 +90295 POINT(40.606297605739584 -73.36993067167332) bank90295 +90296 POINT(39.901191560864305 -74.80784288795378) bank90296 +90297 POINT(40.4452398333929 -73.31646473311271) bank90297 +90298 POINT(40.82574672593877 -74.20483730363308) bank90298 +90299 POINT(40.57486155184119 -73.60182891478959) bank90299 +90300 POINT(40.9379448269148 -74.36074587671027) bank90300 +90301 POINT(41.028962376239164 -74.65979241612126) bank90301 +90302 POINT(40.442455687994276 -74.60598827280793) bank90302 +90303 POINT(41.05838457109161 -74.4382657728014) bank90303 +90304 POINT(41.213068179570975 -74.71834166246737) bank90304 +90305 POINT(40.833606722143394 -73.07433229471424) bank90305 +90306 POINT(41.45689296921115 -74.3740577533113) bank90306 +90307 POINT(40.20551701604597 -73.0489272834714) bank90307 +90308 POINT(41.265959076700284 -74.55601622986401) bank90308 +90309 POINT(40.15127217532747 -74.94117624781401) bank90309 +90310 POINT(40.12304402149092 -74.28393186263726) bank90310 +90311 POINT(40.38735529811137 -73.32834301362959) bank90311 +90312 POINT(39.951272851953625 -74.85610495403184) bank90312 +90313 POINT(41.172070409223274 -74.1015633063854) bank90313 +90314 POINT(40.70559656734557 -73.91947988208939) bank90314 +90315 POINT(40.53843368940857 -73.09508791224364) bank90315 +90316 POINT(40.06125180724951 -74.05225186493884) bank90316 +90317 POINT(40.02940545001161 -74.76005729513567) bank90317 +90318 POINT(40.34407097085277 -74.51810181884828) bank90318 +90319 POINT(41.53462158177333 -74.30368276247698) bank90319 +90320 POINT(41.43471692494848 -74.45408732495058) bank90320 +90321 POINT(39.74063322121263 -74.07418490608521) bank90321 +90322 POINT(41.34202401685096 -74.01141732729276) bank90322 +90323 POINT(40.659001488311745 -73.57579724800877) bank90323 +90324 POINT(41.37757281687545 -74.83241915690947) bank90324 +90325 POINT(40.77624869102709 -73.61982738131394) bank90325 +90326 POINT(39.819137393466434 -74.46553809057393) bank90326 +90327 POINT(40.08623129096124 -73.62629459011173) bank90327 +90328 POINT(41.20965375477971 -73.5000264105461) bank90328 +90329 POINT(40.77284803707358 -73.9704162701295) bank90329 +90330 POINT(41.07863790967178 -74.69136076079673) bank90330 +90331 POINT(41.26442828941904 -73.71041400020479) bank90331 +90332 POINT(40.30198320717736 -73.33051920127825) bank90332 +90333 POINT(40.84017395672132 -73.69197059490864) bank90333 +90334 POINT(40.08223165636119 -73.62568244050797) bank90334 +90335 POINT(39.79901389433895 -73.49808167187945) bank90335 +90336 POINT(41.164206873318655 -73.99701593647639) bank90336 +90337 POINT(39.761317887295256 -74.8922777756019) bank90337 +90338 POINT(41.30234787694807 -73.36920812297761) bank90338 +90339 POINT(40.67549378413059 -74.47162393622254) bank90339 +90340 POINT(40.585734539473656 -74.81896869236664) bank90340 +90341 POINT(41.52210845726249 -74.11668690970222) bank90341 +90342 POINT(40.84961766877947 -73.59399142624967) bank90342 +90343 POINT(40.187537330985705 -74.3875317439896) bank90343 +90344 POINT(40.49753722847879 -73.64217826927711) bank90344 +90345 POINT(40.8458552510559 -74.54127723711989) bank90345 +90346 POINT(41.676233436140535 -73.26912199215816) bank90346 +90347 POINT(41.245406803256266 -74.23873358288064) bank90347 +90348 POINT(39.75988199314296 -73.48693922902211) bank90348 +90349 POINT(40.87628698422379 -73.0705965791819) bank90349 +90350 POINT(40.489570876238155 -73.30178750691626) bank90350 +90351 POINT(41.43004152983641 -73.87001838935107) bank90351 +90352 POINT(39.73304322246137 -73.3466249163513) bank90352 +90353 POINT(40.54174362674518 -73.25420124809513) bank90353 +90354 POINT(40.058646378794016 -73.96131515197152) bank90354 +90355 POINT(41.120172401412326 -74.51163459243864) bank90355 +90356 POINT(40.18406125386906 -73.99299125474471) bank90356 +90357 POINT(41.35663117517327 -73.76066610241665) bank90357 +90358 POINT(41.17486873105049 -74.0629090692216) bank90358 +90359 POINT(40.42029156006491 -73.87438625964491) bank90359 +90360 POINT(40.25131271294648 -73.06224852160277) bank90360 +90361 POINT(41.38904742923261 -74.21921989059734) bank90361 +90362 POINT(40.4151733935715 -73.38947966934674) bank90362 +90363 POINT(41.34592688633259 -74.79979008851379) bank90363 +90364 POINT(40.96438564870246 -73.01992563351689) bank90364 +90365 POINT(40.36832582363642 -73.49172861430837) bank90365 +90366 POINT(41.34315580219665 -74.12856868569564) bank90366 +90367 POINT(41.29995964911903 -73.96155677838408) bank90367 +90368 POINT(40.761578993255306 -74.35648313393764) bank90368 +90369 POINT(40.17771746588662 -73.6890655999398) bank90369 +90370 POINT(39.89624535310837 -74.35640953083102) bank90370 +90371 POINT(40.67309104332294 -73.09440652032293) bank90371 +90372 POINT(41.431236635816234 -74.95839927715822) bank90372 +90373 POINT(41.17402630924612 -74.9460038493998) bank90373 +90374 POINT(40.322265857113855 -73.65178780491144) bank90374 +90375 POINT(39.78839662415323 -73.80545980693643) bank90375 +90376 POINT(41.29281851453839 -73.50019903691357) bank90376 +90377 POINT(40.368135326008726 -74.12698386037806) bank90377 +90378 POINT(41.25120756391991 -73.1684218483049) bank90378 +90379 POINT(40.37180080424541 -73.9194041754828) bank90379 +90380 POINT(40.617489610872056 -73.72454714621186) bank90380 +90381 POINT(39.80053885954237 -73.89146931310708) bank90381 +90382 POINT(40.74014617866965 -73.16662992942794) bank90382 +90383 POINT(40.380930456703126 -73.6984582425998) bank90383 +90384 POINT(40.88519477133695 -74.97672001360807) bank90384 +90385 POINT(39.84241400672368 -74.7401616828775) bank90385 +90386 POINT(41.09504990885595 -73.79925496891323) bank90386 +90387 POINT(41.60804227276292 -73.24325277071462) bank90387 +90388 POINT(40.17664660618412 -74.56099308528667) bank90388 +90389 POINT(40.00065061274865 -73.50137533805608) bank90389 +90390 POINT(40.38546268990327 -73.94792800909248) bank90390 +90391 POINT(41.05723467203944 -73.93276465263007) bank90391 +90392 POINT(41.6568166562494 -73.43819394855265) bank90392 +90393 POINT(40.75840915716658 -73.60906883371307) bank90393 +90394 POINT(40.787818029628156 -74.59264041044518) bank90394 +90395 POINT(39.98674259789706 -74.71558826979405) bank90395 +90396 POINT(39.963878272099905 -73.57985614607406) bank90396 +90397 POINT(40.54219077105553 -73.39132088795053) bank90397 +90398 POINT(39.767840873534126 -73.24794032691524) bank90398 +90399 POINT(39.963491283126736 -74.14230261190099) bank90399 +90400 POINT(40.97874713031504 -74.78262232892637) bank90400 +90401 POINT(41.05907869842219 -73.19682257574968) bank90401 +90402 POINT(40.87602235031144 -74.79285414137767) bank90402 +90403 POINT(40.145553892403534 -74.84610721705599) bank90403 +90404 POINT(41.25237996157016 -74.18292611355741) bank90404 +90405 POINT(41.375043890436494 -74.99530979185006) bank90405 +90406 POINT(41.70426026160831 -74.96617480191016) bank90406 +90407 POINT(40.155752912024326 -73.88749175253642) bank90407 +90408 POINT(39.8596330953194 -73.9189726628438) bank90408 +90409 POINT(40.87735538394804 -74.13838154605119) bank90409 +90410 POINT(40.994326124862226 -74.70565791214413) bank90410 +90411 POINT(40.1652915429779 -73.64636890430127) bank90411 +90412 POINT(40.0325366196467 -74.9788932291518) bank90412 +90413 POINT(40.40515894796566 -73.67229016781505) bank90413 +90414 POINT(40.27766234489141 -74.00216274816508) bank90414 +90415 POINT(39.71631727914063 -73.12072271300669) bank90415 +90416 POINT(40.43242667357819 -73.31666164049156) bank90416 +90417 POINT(40.150466769226995 -74.84255218414322) bank90417 +90418 POINT(39.86730246750179 -74.13054249871223) bank90418 +90419 POINT(39.77143853848485 -73.92623199366207) bank90419 +90420 POINT(39.77526847848313 -73.32309589390505) bank90420 +90421 POINT(40.28312441264809 -73.01503687639308) bank90421 +90422 POINT(39.86430337255921 -74.88824734723255) bank90422 +90423 POINT(39.96763262736455 -73.72479939128388) bank90423 +90424 POINT(40.710589153524886 -74.61105650834048) bank90424 +90425 POINT(41.70513742700264 -73.01384399859529) bank90425 +90426 POINT(40.455702780008664 -74.24007456670381) bank90426 +90427 POINT(40.51131758495751 -74.43917502933559) bank90427 +90428 POINT(40.5372997464515 -73.43240717265347) bank90428 +90429 POINT(39.90293054035072 -74.10974493873088) bank90429 +90430 POINT(41.43719273919206 -74.25287976074326) bank90430 +90431 POINT(40.779799338390546 -73.64185893384605) bank90431 +90432 POINT(41.10214472500101 -74.63212989371019) bank90432 +90433 POINT(40.27040769347771 -74.5892451867307) bank90433 +90434 POINT(39.988464534426676 -73.59500881310431) bank90434 +90435 POINT(41.260081960876015 -74.4015758878087) bank90435 +90436 POINT(40.30991406988873 -74.37151797181984) bank90436 +90437 POINT(41.26257890336976 -73.36437579503874) bank90437 +90438 POINT(40.511320335522946 -73.17917408370964) bank90438 +90439 POINT(41.50015400407757 -74.10079316248151) bank90439 +90440 POINT(41.45534799675917 -73.70153752556962) bank90440 +90441 POINT(41.19502578313913 -74.97483926416352) bank90441 +90442 POINT(40.93570925948963 -74.02872354064313) bank90442 +90443 POINT(40.45458568575353 -74.65353747327387) bank90443 +90444 POINT(40.316464142656834 -74.97803753398391) bank90444 +90445 POINT(40.32312252054361 -74.71502545018123) bank90445 +90446 POINT(40.24067177789895 -74.08255437030266) bank90446 +90447 POINT(41.01200806331502 -74.13176579777918) bank90447 +90448 POINT(41.51686352700795 -73.82691151853281) bank90448 +90449 POINT(41.28265459530879 -74.09582086104317) bank90449 +90450 POINT(41.37324384248408 -73.92479279697201) bank90450 +90451 POINT(40.59739709746523 -74.7908967012858) bank90451 +90452 POINT(41.615766681392756 -73.94053025988985) bank90452 +90453 POINT(41.47845830283536 -73.82496876921452) bank90453 +90454 POINT(40.29412740590306 -73.2926602855017) bank90454 +90455 POINT(40.98206202201774 -73.87168683224515) bank90455 +90456 POINT(40.472638713417865 -74.00157198338205) bank90456 +90457 POINT(41.14952329492758 -73.87822352818311) bank90457 +90458 POINT(41.479263322098525 -74.78005649039558) bank90458 +90459 POINT(41.494311224971355 -74.80169043234292) bank90459 +90460 POINT(40.4292466908561 -73.56101610185411) bank90460 +90461 POINT(40.71841397472552 -74.64496002487081) bank90461 +90462 POINT(41.62836837982135 -74.00997937464172) bank90462 +90463 POINT(41.16664460024097 -74.18716034931681) bank90463 +90464 POINT(40.74022055870439 -73.91108504612401) bank90464 +90465 POINT(40.07109685915585 -73.29444107718051) bank90465 +90466 POINT(39.741619123995676 -73.03918267281887) bank90466 +90467 POINT(40.86939741013873 -73.58967206507869) bank90467 +90468 POINT(40.783324244766284 -73.13304988157797) bank90468 +90469 POINT(40.04799717027766 -74.97536781184421) bank90469 +90470 POINT(40.350205694614004 -74.64407896607591) bank90470 +90471 POINT(41.285247077100735 -74.9198908867889) bank90471 +90472 POINT(40.59655714227102 -73.65939849008016) bank90472 +90473 POINT(40.683153706233256 -74.89170570130356) bank90473 +90474 POINT(40.54133272378454 -73.99663234196373) bank90474 +90475 POINT(41.43565120188769 -73.15694767124629) bank90475 +90476 POINT(39.965821034880356 -74.24262747299309) bank90476 +90477 POINT(40.4848883080127 -74.64429669366965) bank90477 +90478 POINT(40.86828293594239 -73.97637104127395) bank90478 +90479 POINT(39.98544465403696 -73.33667758995925) bank90479 +90480 POINT(40.80279820003318 -73.78646857513532) bank90480 +90481 POINT(40.59535555046151 -74.57441422466397) bank90481 +90482 POINT(40.800198521515284 -73.12233370534393) bank90482 +90483 POINT(40.32161978252526 -75.00510559857734) bank90483 +90484 POINT(41.193013478109215 -74.40941579155569) bank90484 +90485 POINT(40.76579477078676 -73.41857651761332) bank90485 +90486 POINT(40.032222260875855 -74.50779156310874) bank90486 +90487 POINT(40.311240792889286 -73.14777260309491) bank90487 +90488 POINT(41.47747503112371 -73.0890335759501) bank90488 +90489 POINT(40.917098954712294 -74.0427728313035) bank90489 +90490 POINT(40.77142237572202 -73.76907941122796) bank90490 +90491 POINT(40.2252958385805 -74.61351316793585) bank90491 +90492 POINT(39.90391128266224 -74.33743102197167) bank90492 +90493 POINT(40.94450504867556 -73.22697196497859) bank90493 +90494 POINT(40.55534686474428 -74.79578854635874) bank90494 +90495 POINT(40.23920736152477 -73.70657939276168) bank90495 +90496 POINT(39.94788901789028 -73.23462908127996) bank90496 +90497 POINT(40.30944616947609 -74.53417526420259) bank90497 +90498 POINT(41.673689463098924 -73.16719453584966) bank90498 +90499 POINT(40.650201424770614 -73.23578855160417) bank90499 +90500 POINT(41.39329058036566 -73.64904262902033) bank90500 +90501 POINT(40.434416634158154 -73.49120625691353) bank90501 +90502 POINT(41.20850608840801 -73.93068118475551) bank90502 +90503 POINT(41.262049859409935 -74.21304243246169) bank90503 +90504 POINT(41.033364937909944 -74.26984091215634) bank90504 +90505 POINT(41.11584830515728 -73.68701900379575) bank90505 +90506 POINT(39.7144873384293 -73.32438080278504) bank90506 +90507 POINT(40.10954279382844 -73.03346323453833) bank90507 +90508 POINT(39.768002924352075 -73.27949408884169) bank90508 +90509 POINT(40.818350148500535 -73.01074201153583) bank90509 +90510 POINT(39.79690597126975 -74.28594665088394) bank90510 +90511 POINT(41.06606165286254 -74.38123396533177) bank90511 +90512 POINT(41.518739828393 -73.17440734950546) bank90512 +90513 POINT(41.18814744545844 -74.35125689967255) bank90513 +90514 POINT(39.99874495509551 -73.69722475094774) bank90514 +90515 POINT(40.619960725554094 -73.58967886532504) bank90515 +90516 POINT(39.90323617401118 -74.38311657282998) bank90516 +90517 POINT(40.23012006823789 -73.18809099069082) bank90517 +90518 POINT(41.36436902376924 -73.64878049611475) bank90518 +90519 POINT(39.85533366789262 -73.18702333053528) bank90519 +90520 POINT(40.294829671007264 -74.29971807830101) bank90520 +90521 POINT(40.58144328478958 -74.01946415137628) bank90521 +90522 POINT(40.65628171462725 -74.80712528347749) bank90522 +90523 POINT(41.05136790005737 -73.46503099638022) bank90523 +90524 POINT(39.73889467353599 -73.25061642450747) bank90524 +90525 POINT(41.699084666717354 -73.31771681939077) bank90525 +90526 POINT(40.95216388093047 -73.64680254556377) bank90526 +90527 POINT(40.19456302707014 -73.93795583325188) bank90527 +90528 POINT(39.73922049756703 -73.99033956084236) bank90528 +90529 POINT(41.17233718521017 -73.47867126586179) bank90529 +90530 POINT(41.67062001592755 -74.52237925410772) bank90530 +90531 POINT(41.57231590584135 -74.55247787812863) bank90531 +90532 POINT(40.728349760179114 -74.21466027629047) bank90532 +90533 POINT(41.004037751049864 -73.85790545797299) bank90533 +90534 POINT(39.928984031607705 -74.10703434202843) bank90534 +90535 POINT(40.18884272318132 -74.71276675337442) bank90535 +90536 POINT(40.65885650497854 -73.8228473607617) bank90536 +90537 POINT(40.905521189820504 -73.13606585597223) bank90537 +90538 POINT(41.07847231130309 -73.25770466797728) bank90538 +90539 POINT(41.48622121134079 -74.51706364644744) bank90539 +90540 POINT(40.558738683048865 -73.88835560631517) bank90540 +90541 POINT(40.05569111905831 -74.15289141773914) bank90541 +90542 POINT(41.26079047211473 -74.91089768490866) bank90542 +90543 POINT(41.09160761990006 -73.94988614786963) bank90543 +90544 POINT(39.99406974657914 -73.5808653100793) bank90544 +90545 POINT(40.09551522997159 -73.02666512557084) bank90545 +90546 POINT(39.88771395210949 -74.04914927815969) bank90546 +90547 POINT(39.93974183223772 -73.39964119522179) bank90547 +90548 POINT(41.27769015784018 -74.35255923737472) bank90548 +90549 POINT(41.19838900815681 -74.48269475074495) bank90549 +90550 POINT(39.98514139074303 -73.8150413849788) bank90550 +90551 POINT(40.24239180891878 -73.57561738121761) bank90551 +90552 POINT(40.26117512699015 -73.62257286256445) bank90552 +90553 POINT(39.81683040926808 -74.72186995194281) bank90553 +90554 POINT(40.54993913923606 -73.78026461884647) bank90554 +90555 POINT(41.15910115439338 -74.87857728685215) bank90555 +90556 POINT(40.51708153569498 -73.23072497968845) bank90556 +90557 POINT(40.9031766745238 -73.64455607797757) bank90557 +90558 POINT(39.78236286435737 -73.01028627370269) bank90558 +90559 POINT(39.96788940079597 -74.67613157491924) bank90559 +90560 POINT(40.76798900875664 -74.51878905685804) bank90560 +90561 POINT(41.17034549011531 -73.46608131649543) bank90561 +90562 POINT(40.812868943595525 -74.92119745261436) bank90562 +90563 POINT(40.34646489248229 -73.60900188074356) bank90563 +90564 POINT(40.38515470370431 -74.68828764410144) bank90564 +90565 POINT(40.20652727127681 -73.13032901232245) bank90565 +90566 POINT(41.187841047629256 -74.98152989839993) bank90566 +90567 POINT(40.60871576375938 -74.33016517262585) bank90567 +90568 POINT(41.27528233775149 -73.9065128295658) bank90568 +90569 POINT(40.82946612713904 -74.56513889086439) bank90569 +90570 POINT(40.65040092391271 -74.35682430762645) bank90570 +90571 POINT(40.52581692336917 -73.83709353762501) bank90571 +90572 POINT(40.043727353921795 -73.89573033441027) bank90572 +90573 POINT(41.52891287688915 -73.83369685775925) bank90573 +90574 POINT(41.04856364386841 -74.75315774839773) bank90574 +90575 POINT(41.323194510592074 -73.31622066527522) bank90575 +90576 POINT(39.836490249479475 -74.26349068325908) bank90576 +90577 POINT(40.66863931303545 -74.73725341492634) bank90577 +90578 POINT(41.22404428382328 -73.27426712660257) bank90578 +90579 POINT(39.79621551766282 -73.34364888076973) bank90579 +90580 POINT(41.03480275043729 -74.83681551110567) bank90580 +90581 POINT(41.57750852285582 -74.2895353678805) bank90581 +90582 POINT(40.606049700605425 -73.2489598558116) bank90582 +90583 POINT(40.24495470840227 -73.73792779111886) bank90583 +90584 POINT(41.30287845898199 -73.92953111086628) bank90584 +90585 POINT(39.71724603348655 -74.2783543115233) bank90585 +90586 POINT(41.6573858838222 -74.90394443126466) bank90586 +90587 POINT(40.568030569165266 -73.73974185973228) bank90587 +90588 POINT(41.6607087647224 -74.72515998632652) bank90588 +90589 POINT(41.42913307521192 -74.09490053757918) bank90589 +90590 POINT(41.37486549936337 -73.3964703954341) bank90590 +90591 POINT(41.37754190546316 -74.01256427590248) bank90591 +90592 POINT(40.762113661944646 -73.74520248510397) bank90592 +90593 POINT(40.77769389500864 -74.87956755891611) bank90593 +90594 POINT(40.50002161259755 -74.68914215394302) bank90594 +90595 POINT(40.46477403983943 -73.35651309578557) bank90595 +90596 POINT(40.5531705586623 -74.24021842522303) bank90596 +90597 POINT(40.89992045821582 -73.07753136787146) bank90597 +90598 POINT(40.50592999945026 -74.42150906445859) bank90598 +90599 POINT(41.11725589034788 -74.545961941438) bank90599 +90600 POINT(41.01753788145848 -74.98235718020601) bank90600 +90601 POINT(40.17723574977578 -73.1094994953029) bank90601 +90602 POINT(40.117344979242844 -73.02803850701437) bank90602 +90603 POINT(41.15709204969677 -74.44211649365556) bank90603 +90604 POINT(40.479741608881966 -74.01195178914226) bank90604 +90605 POINT(40.003190821349804 -74.95693392530907) bank90605 +90606 POINT(40.69117908950565 -73.15863003851031) bank90606 +90607 POINT(41.299884880154266 -73.69915473370962) bank90607 +90608 POINT(40.39317142934187 -74.04547706919097) bank90608 +90609 POINT(41.651386759510046 -74.10812062273901) bank90609 +90610 POINT(41.620197734067084 -74.00788788456958) bank90610 +90611 POINT(41.14548038385019 -74.21274368120075) bank90611 +90612 POINT(39.895798485876156 -74.6798342313048) bank90612 +90613 POINT(40.50497618740509 -74.54771209983811) bank90613 +90614 POINT(40.936767799170575 -73.37054750604682) bank90614 +90615 POINT(40.33050678182457 -74.77887880561526) bank90615 +90616 POINT(41.12050471391901 -73.27187133072007) bank90616 +90617 POINT(41.607346247635874 -73.96738392337367) bank90617 +90618 POINT(40.83900780633521 -74.50288970285503) bank90618 +90619 POINT(40.355102680975214 -73.94252291282402) bank90619 +90620 POINT(41.45666029469966 -74.75766942832925) bank90620 +90621 POINT(40.234725324975216 -74.01133237711137) bank90621 +90622 POINT(40.37518335833211 -73.45336376693622) bank90622 +90623 POINT(39.776647633482966 -73.64430439278858) bank90623 +90624 POINT(40.36786188865838 -73.53288171947693) bank90624 +90625 POINT(39.92324836199338 -74.53611969106281) bank90625 +90626 POINT(41.614545887023084 -74.02994829445724) bank90626 +90627 POINT(41.68567875823679 -74.38146348528471) bank90627 +90628 POINT(41.404373927624256 -74.14467431404489) bank90628 +90629 POINT(40.1405788072146 -74.05968921675695) bank90629 +90630 POINT(41.67301556075948 -73.09542507372385) bank90630 +90631 POINT(40.58820924785755 -73.10834112519879) bank90631 +90632 POINT(41.67934083298503 -73.41985058446286) bank90632 +90633 POINT(39.810805702982734 -74.07784001135482) bank90633 +90634 POINT(41.119334473856604 -74.72564527756057) bank90634 +90635 POINT(41.48666744571898 -73.85786769922491) bank90635 +90636 POINT(40.33793792613507 -74.1634606160272) bank90636 +90637 POINT(40.28874577977668 -74.38048737513644) bank90637 +90638 POINT(40.97256022686833 -74.43452458055864) bank90638 +90639 POINT(40.680759784903955 -73.04357342397053) bank90639 +90640 POINT(41.59867511734142 -73.62222043947058) bank90640 +90641 POINT(39.834628830078685 -74.85182190866786) bank90641 +90642 POINT(41.632697476933835 -74.76872986187492) bank90642 +90643 POINT(40.199854151405766 -73.22437459522519) bank90643 +90644 POINT(40.771942136615316 -73.8526834793302) bank90644 +90645 POINT(40.29660361196425 -74.49225002131283) bank90645 +90646 POINT(41.29334332508953 -74.45306256981456) bank90646 +90647 POINT(40.668036460936094 -74.65154584264921) bank90647 +90648 POINT(40.12581741212898 -74.14264754907407) bank90648 +90649 POINT(41.25711457173734 -74.35365233299335) bank90649 +90650 POINT(41.443398673116825 -73.42537412294816) bank90650 +90651 POINT(40.52807454717454 -74.73699014583154) bank90651 +90652 POINT(40.11684437265618 -74.0443748751193) bank90652 +90653 POINT(39.827094220017685 -74.21313827177445) bank90653 +90654 POINT(39.82968085048693 -73.70123230169577) bank90654 +90655 POINT(41.581556559578935 -73.10863819514726) bank90655 +90656 POINT(41.49627294721375 -74.00689862605851) bank90656 +90657 POINT(41.46846608800586 -73.31625239358598) bank90657 +90658 POINT(41.70962794418406 -74.18064932299936) bank90658 +90659 POINT(40.87097175064598 -74.79432570385072) bank90659 +90660 POINT(41.277116830904056 -74.63966033648441) bank90660 +90661 POINT(40.146899479765516 -73.09181361962045) bank90661 +90662 POINT(41.6195378752068 -73.08235683206829) bank90662 +90663 POINT(41.21017371334787 -74.23874640888165) bank90663 +90664 POINT(40.455482934950076 -74.5728710042629) bank90664 +90665 POINT(39.81624265759793 -74.06681331685066) bank90665 +90666 POINT(41.54510345555671 -73.42554786736471) bank90666 +90667 POINT(39.73649238896657 -74.78434678223974) bank90667 +90668 POINT(39.90059816775929 -73.22611750806443) bank90668 +90669 POINT(40.74704440933277 -74.86909880426474) bank90669 +90670 POINT(40.089998199728676 -74.30287813442837) bank90670 +90671 POINT(39.724115068584055 -73.05058628091535) bank90671 +90672 POINT(40.940466540982136 -74.58473467621405) bank90672 +90673 POINT(40.89830323932925 -74.53328931642527) bank90673 +90674 POINT(41.272691174385635 -73.71297594327038) bank90674 +90675 POINT(39.93951324555023 -73.73516674554838) bank90675 +90676 POINT(40.584876392301396 -74.74837850455562) bank90676 +90677 POINT(39.98034779360105 -73.11902785770825) bank90677 +90678 POINT(40.510908035539316 -74.74597826683407) bank90678 +90679 POINT(40.674755547276035 -73.71813091136539) bank90679 +90680 POINT(40.96801513984974 -74.96735682201687) bank90680 +90681 POINT(40.337281904019775 -74.7540694944103) bank90681 +90682 POINT(41.47874879113881 -73.21752469897757) bank90682 +90683 POINT(40.748224050345414 -73.73761032246188) bank90683 +90684 POINT(41.458175358107795 -74.06047617751838) bank90684 +90685 POINT(40.80701837263467 -74.6191526816349) bank90685 +90686 POINT(41.2594519085008 -74.00564170849837) bank90686 +90687 POINT(41.096033954646224 -73.12011731329451) bank90687 +90688 POINT(40.51411941022878 -74.50681410113381) bank90688 +90689 POINT(40.52055173366188 -73.00727829794002) bank90689 +90690 POINT(41.60659476643052 -73.76383403969399) bank90690 +90691 POINT(39.947851612828416 -74.17136713528139) bank90691 +90692 POINT(40.38974015122452 -74.67588120616611) bank90692 +90693 POINT(40.17333026740037 -74.16748256237733) bank90693 +90694 POINT(41.6430887032849 -74.49067386022236) bank90694 +90695 POINT(40.496650730805065 -74.8188702536811) bank90695 +90696 POINT(40.477718629391326 -74.48288570014041) bank90696 +90697 POINT(40.65785871543697 -73.55822057905789) bank90697 +90698 POINT(40.64896948435321 -73.30398733154271) bank90698 +90699 POINT(39.73568125790627 -74.76136136782885) bank90699 +90700 POINT(40.918279505845156 -73.899452709667) bank90700 +90701 POINT(41.379305605016896 -73.24717410616816) bank90701 +90702 POINT(39.81680072054179 -73.93340554806136) bank90702 +90703 POINT(39.92879919307408 -73.2746879251473) bank90703 +90704 POINT(40.193278785341995 -74.00019662323567) bank90704 +90705 POINT(41.575627575581706 -74.51186698292102) bank90705 +90706 POINT(40.530519008080404 -74.58964272896627) bank90706 +90707 POINT(41.06645492204835 -73.02719070208336) bank90707 +90708 POINT(41.42154947023247 -74.40237915317859) bank90708 +90709 POINT(41.26088302241677 -74.27393987635065) bank90709 +90710 POINT(40.29220330334854 -74.7701431487497) bank90710 +90711 POINT(41.65791050727641 -73.61571677504709) bank90711 +90712 POINT(40.65693501184176 -74.33257045440746) bank90712 +90713 POINT(40.86655212941091 -74.83471010012143) bank90713 +90714 POINT(41.376931377307315 -73.84292131731358) bank90714 +90715 POINT(40.62924006181076 -74.85653073766231) bank90715 +90716 POINT(40.80922144605838 -73.21665342610845) bank90716 +90717 POINT(41.20801272527483 -74.85704648648212) bank90717 +90718 POINT(41.470090006615 -74.31951003052717) bank90718 +90719 POINT(40.52908579603536 -74.9784017246496) bank90719 +90720 POINT(40.28689246899265 -73.13935062992466) bank90720 +90721 POINT(40.626537367193336 -74.13066455986723) bank90721 +90722 POINT(41.1014104816917 -74.13429024223008) bank90722 +90723 POINT(41.47731875022628 -74.35277936285772) bank90723 +90724 POINT(40.153646259737016 -73.89015251818292) bank90724 +90725 POINT(39.77306303760168 -73.50469370452566) bank90725 +90726 POINT(39.98341978150847 -74.54146137610593) bank90726 +90727 POINT(39.935960251541225 -73.11350528777598) bank90727 +90728 POINT(39.844381493383466 -74.10102868675729) bank90728 +90729 POINT(40.9342309467613 -74.80902565271023) bank90729 +90730 POINT(40.31648929270783 -73.23223775469481) bank90730 +90731 POINT(40.82254750566571 -74.0057027939915) bank90731 +90732 POINT(40.780909482983795 -73.05638299948092) bank90732 +90733 POINT(40.08561969378822 -74.49397906135484) bank90733 +90734 POINT(41.66694203206393 -73.33235312333382) bank90734 +90735 POINT(40.51418476426951 -74.8956142291867) bank90735 +90736 POINT(41.40976744675419 -73.74016362920578) bank90736 +90737 POINT(40.46319740886014 -74.0476533757081) bank90737 +90738 POINT(40.26664731878697 -73.44446301077187) bank90738 +90739 POINT(40.58027504171516 -74.91000326466491) bank90739 +90740 POINT(40.109223756215634 -73.69471984507025) bank90740 +90741 POINT(40.86491328613075 -74.66481400503925) bank90741 +90742 POINT(41.25062702373325 -73.8232821141251) bank90742 +90743 POINT(41.375375395837985 -74.85443692541952) bank90743 +90744 POINT(41.32362796082819 -74.73038565491632) bank90744 +90745 POINT(41.35537712994604 -74.49648573351291) bank90745 +90746 POINT(41.27358246981525 -74.97805672743489) bank90746 +90747 POINT(41.23441130112933 -73.95190590083402) bank90747 +90748 POINT(41.35291821167179 -73.8157890581085) bank90748 +90749 POINT(41.4435082602294 -74.98875857083888) bank90749 +90750 POINT(41.120964127244804 -74.1968230491657) bank90750 +90751 POINT(40.24745070902019 -73.71987010025565) bank90751 +90752 POINT(41.53872098655113 -74.8221863378772) bank90752 +90753 POINT(41.2305029039568 -73.3260127876017) bank90753 +90754 POINT(41.395878911720516 -74.76775278243643) bank90754 +90755 POINT(41.31897539443909 -74.10607828814179) bank90755 +90756 POINT(41.70454926772297 -74.61409082398332) bank90756 +90757 POINT(39.97632254427919 -73.94369906927666) bank90757 +90758 POINT(41.00412616336551 -74.48466009104763) bank90758 +90759 POINT(39.838576753536245 -73.2932753344049) bank90759 +90760 POINT(40.39864123822582 -73.88575353219126) bank90760 +90761 POINT(41.47236336695766 -74.15624249897544) bank90761 +90762 POINT(40.389248896336724 -74.78779268887838) bank90762 +90763 POINT(41.53792456715795 -73.30291015897501) bank90763 +90764 POINT(40.22979140192602 -74.4383901444117) bank90764 +90765 POINT(39.951064631318914 -73.83362221487074) bank90765 +90766 POINT(41.36617636800493 -73.78638626972135) bank90766 +90767 POINT(41.63327279093218 -74.63010091868762) bank90767 +90768 POINT(40.938207006275874 -74.25116532181283) bank90768 +90769 POINT(40.72405166720451 -74.39850534676309) bank90769 +90770 POINT(40.437682240005216 -73.3251490477083) bank90770 +90771 POINT(39.86995664939233 -73.20784532314599) bank90771 +90772 POINT(39.81414323835129 -74.22674380956472) bank90772 +90773 POINT(39.85785365630003 -74.48382403203166) bank90773 +90774 POINT(39.92155080131161 -73.03344003247335) bank90774 +90775 POINT(40.38049446192547 -73.94899222761903) bank90775 +90776 POINT(41.24478206846694 -74.11103644732894) bank90776 +90777 POINT(40.899182473074106 -74.75822226256064) bank90777 +90778 POINT(39.85052315654028 -73.69414184718256) bank90778 +90779 POINT(40.54489365595497 -74.05349667907575) bank90779 +90780 POINT(40.62044183290162 -74.39041715827929) bank90780 +90781 POINT(39.78599809474063 -74.28224835858009) bank90781 +90782 POINT(40.980656657796224 -73.64512229917526) bank90782 +90783 POINT(40.86028848826416 -74.00967963018834) bank90783 +90784 POINT(39.896156238154994 -74.36613990066378) bank90784 +90785 POINT(41.11957573460255 -73.59838528461823) bank90785 +90786 POINT(40.212913826612684 -74.11939097783859) bank90786 +90787 POINT(40.74313823126281 -74.82068079044953) bank90787 +90788 POINT(40.014168566009765 -74.9801436595983) bank90788 +90789 POINT(41.28540812129045 -74.52135762476995) bank90789 +90790 POINT(41.51844733085629 -73.39078160738426) bank90790 +90791 POINT(40.294829051348074 -74.96870123591489) bank90791 +90792 POINT(41.05233852278159 -73.21260321583051) bank90792 +90793 POINT(41.084623383154316 -73.70703396784677) bank90793 +90794 POINT(41.580751642847545 -73.4954845387658) bank90794 +90795 POINT(41.306029507411125 -74.41568845860448) bank90795 +90796 POINT(41.52680121128294 -73.86791729858713) bank90796 +90797 POINT(39.72144616383379 -74.73830991774787) bank90797 +90798 POINT(39.93334771066695 -73.45723524379227) bank90798 +90799 POINT(39.84025495486555 -73.25484759618675) bank90799 +90800 POINT(41.009252426785835 -74.98715670755303) bank90800 +90801 POINT(41.200113755235755 -73.5388788269415) bank90801 +90802 POINT(40.8608544522081 -74.8590537159663) bank90802 +90803 POINT(40.66050515923452 -74.72961295526983) bank90803 +90804 POINT(41.028898492142055 -74.51515933922751) bank90804 +90805 POINT(40.45740303380983 -74.25432890216436) bank90805 +90806 POINT(41.56650387004425 -73.96859826217529) bank90806 +90807 POINT(40.599138071003296 -74.86354923951373) bank90807 +90808 POINT(40.47951549698781 -74.88568307742032) bank90808 +90809 POINT(39.936049320141905 -73.06433590865585) bank90809 +90810 POINT(41.34600663760255 -74.30438514004743) bank90810 +90811 POINT(40.76619141984891 -74.36648355285047) bank90811 +90812 POINT(40.88032315831784 -74.36806403913741) bank90812 +90813 POINT(41.5946971661788 -74.94516028084678) bank90813 +90814 POINT(40.958950055214586 -74.8057580356543) bank90814 +90815 POINT(40.387457612382725 -74.71749783470439) bank90815 +90816 POINT(41.20116879285328 -74.32334212459918) bank90816 +90817 POINT(40.75343895639663 -74.25017786300819) bank90817 +90818 POINT(41.09942482756613 -74.86663230063861) bank90818 +90819 POINT(41.233152961867994 -74.96623054714165) bank90819 +90820 POINT(41.479709221746425 -73.40902572536555) bank90820 +90821 POINT(41.360517121530144 -73.94822404844808) bank90821 +90822 POINT(41.6265869810131 -74.60109941722784) bank90822 +90823 POINT(40.69618316282048 -73.5084095060838) bank90823 +90824 POINT(39.94848545507354 -74.68366323359587) bank90824 +90825 POINT(40.329370309616095 -73.81455068483261) bank90825 +90826 POINT(39.84482962154065 -73.59504681519519) bank90826 +90827 POINT(40.287195777455295 -73.21679758398339) bank90827 +90828 POINT(41.086844019027865 -74.38137075504712) bank90828 +90829 POINT(40.997825304601264 -73.81114064370466) bank90829 +90830 POINT(40.64398685312232 -74.38543434421788) bank90830 +90831 POINT(40.03794699578576 -73.10653798467133) bank90831 +90832 POINT(41.225259192768455 -74.54472175224589) bank90832 +90833 POINT(41.090350872697236 -73.08847456484754) bank90833 +90834 POINT(40.38531636485253 -73.36319637052442) bank90834 +90835 POINT(41.00012540757283 -74.62055283799411) bank90835 +90836 POINT(40.615057089858716 -73.79323215320275) bank90836 +90837 POINT(40.37360614537991 -74.16683097266645) bank90837 +90838 POINT(40.5202852340876 -73.28647601979259) bank90838 +90839 POINT(41.68424740167806 -74.1999592445222) bank90839 +90840 POINT(40.73050670702328 -74.87721321574723) bank90840 +90841 POINT(41.63038584396302 -74.85145044972312) bank90841 +90842 POINT(40.433388986295796 -74.33933409742384) bank90842 +90843 POINT(40.849244178386726 -74.9139577361268) bank90843 +90844 POINT(40.22311908873814 -73.59162706702385) bank90844 +90845 POINT(39.80905083606447 -73.11506481621589) bank90845 +90846 POINT(40.94376630881166 -74.54286879103667) bank90846 +90847 POINT(41.08493065194481 -73.18685129531168) bank90847 +90848 POINT(41.399321554705445 -74.98072032148404) bank90848 +90849 POINT(40.30037325849445 -74.87951694376801) bank90849 +90850 POINT(41.573141750974784 -74.30287498460221) bank90850 +90851 POINT(41.37239095277908 -74.34389558150932) bank90851 +90852 POINT(41.255807471853615 -74.66759750282834) bank90852 +90853 POINT(40.92469164655337 -73.7638011800556) bank90853 +90854 POINT(39.798866048754874 -73.85037205218006) bank90854 +90855 POINT(40.55865581307218 -74.3710651869963) bank90855 +90856 POINT(41.69850840443054 -74.94208663918994) bank90856 +90857 POINT(40.68950353361469 -74.0671492403442) bank90857 +90858 POINT(39.979151337809284 -74.54494760079704) bank90858 +90859 POINT(40.35314604436245 -73.98619701803946) bank90859 +90860 POINT(40.72807221779829 -74.8142353146658) bank90860 +90861 POINT(41.654767953492936 -73.46130845326684) bank90861 +90862 POINT(39.771443442518255 -73.64815342514873) bank90862 +90863 POINT(41.33350216949731 -73.20046939208089) bank90863 +90864 POINT(40.594737987925534 -73.43691675205109) bank90864 +90865 POINT(41.579880149617175 -74.04792641115148) bank90865 +90866 POINT(40.32411373185671 -74.27915613115066) bank90866 +90867 POINT(40.58172653116824 -74.6225875549286) bank90867 +90868 POINT(40.38850954419088 -74.64292271288724) bank90868 +90869 POINT(40.03001648142307 -74.20086883522241) bank90869 +90870 POINT(41.05564579339364 -74.79430867208073) bank90870 +90871 POINT(40.303300194374295 -73.1548197800366) bank90871 +90872 POINT(40.13880956597532 -73.6926415494823) bank90872 +90873 POINT(40.76381801386644 -74.60649510501325) bank90873 +90874 POINT(39.71427958310104 -74.30774226151087) bank90874 +90875 POINT(41.712513275357566 -74.62654381170559) bank90875 +90876 POINT(39.830570011747795 -73.64355920053745) bank90876 +90877 POINT(40.20633128396377 -73.34099510274095) bank90877 +90878 POINT(40.3734601601829 -73.97306106185252) bank90878 +90879 POINT(41.0936148621984 -73.95191514086558) bank90879 +90880 POINT(40.00681652838854 -73.72611069517724) bank90880 +90881 POINT(40.945781762630496 -74.41571075880537) bank90881 +90882 POINT(39.74774090581687 -73.43196895616246) bank90882 +90883 POINT(39.94758133305244 -74.95028637515223) bank90883 +90884 POINT(40.43909425868103 -74.63688281607239) bank90884 +90885 POINT(40.484417509589484 -73.40498773147343) bank90885 +90886 POINT(40.06333848020709 -73.45424469974593) bank90886 +90887 POINT(40.04033862933075 -74.67509659370135) bank90887 +90888 POINT(41.70079097405708 -73.93671583853279) bank90888 +90889 POINT(40.03723827523795 -74.95823943761482) bank90889 +90890 POINT(40.718944984595424 -74.85297693836763) bank90890 +90891 POINT(39.87118904706614 -73.41606929543029) bank90891 +90892 POINT(40.11137925861735 -74.00483008457118) bank90892 +90893 POINT(41.51923674297959 -74.82186382973936) bank90893 +90894 POINT(39.80554604499369 -73.41751328827927) bank90894 +90895 POINT(39.94981234604075 -73.88511918596429) bank90895 +90896 POINT(40.4815040969548 -73.11602900867697) bank90896 +90897 POINT(41.18294990741849 -73.31685633746008) bank90897 +90898 POINT(40.68013781364502 -73.93571356502197) bank90898 +90899 POINT(39.78164464511314 -74.00872548977308) bank90899 +90900 POINT(41.03697238750972 -74.0955135193597) bank90900 +90901 POINT(39.98562504953693 -74.02224218699352) bank90901 +90902 POINT(40.975708104883424 -74.77150086095821) bank90902 +90903 POINT(40.96802150969274 -74.38746386918417) bank90903 +90904 POINT(40.21226217313777 -73.88218644784635) bank90904 +90905 POINT(41.33145876251742 -74.36251525960482) bank90905 +90906 POINT(41.581397583675255 -73.95956420382586) bank90906 +90907 POINT(41.0824605592875 -74.25752092273562) bank90907 +90908 POINT(40.467879242748275 -73.70845128827975) bank90908 +90909 POINT(40.428210495571925 -74.48662892590973) bank90909 +90910 POINT(41.12491711272298 -74.01609766340364) bank90910 +90911 POINT(41.4270067223847 -74.09501090990639) bank90911 +90912 POINT(40.42595756675275 -74.20098334363871) bank90912 +90913 POINT(41.032151182037126 -73.57109426899157) bank90913 +90914 POINT(40.72230525496086 -73.95325196747845) bank90914 +90915 POINT(41.58207626981419 -74.62668936663626) bank90915 +90916 POINT(39.8361449674957 -74.3101058859257) bank90916 +90917 POINT(40.2746456585895 -73.95856707940054) bank90917 +90918 POINT(41.048205770606984 -74.95357694132493) bank90918 +90919 POINT(41.41871314305855 -74.85666883161231) bank90919 +90920 POINT(41.14533455751917 -74.29510088042527) bank90920 +90921 POINT(40.63178234286431 -73.94981870719474) bank90921 +90922 POINT(40.16935890755804 -73.26270571380587) bank90922 +90923 POINT(40.96036266761855 -73.1861534921219) bank90923 +90924 POINT(40.48648717428193 -73.2556854224672) bank90924 +90925 POINT(41.007846522760495 -74.19492012848295) bank90925 +90926 POINT(40.48546989347357 -73.24011375284728) bank90926 +90927 POINT(39.95456176852502 -74.73504698922267) bank90927 +90928 POINT(40.56474279403665 -73.66421518194593) bank90928 +90929 POINT(40.71521419608145 -74.44318392235546) bank90929 +90930 POINT(40.87536212942113 -74.76257112431574) bank90930 +90931 POINT(41.69450460343442 -73.5726929701494) bank90931 +90932 POINT(40.545151705393856 -73.37820042282084) bank90932 +90933 POINT(39.72764374921947 -74.13528625058109) bank90933 +90934 POINT(41.078396071239844 -74.05990553730875) bank90934 +90935 POINT(40.05208277789037 -73.23482403266695) bank90935 +90936 POINT(40.92431615645611 -74.40938582559274) bank90936 +90937 POINT(41.0213711772071 -73.13106536915362) bank90937 +90938 POINT(40.552278565066935 -73.5649069610864) bank90938 +90939 POINT(39.99261211555363 -73.25295159125838) bank90939 +90940 POINT(41.239446679035744 -73.60024655314163) bank90940 +90941 POINT(40.84555676118948 -73.69544640179043) bank90941 +90942 POINT(40.93354425972131 -73.60219206780164) bank90942 +90943 POINT(39.88058734997773 -73.06274787791692) bank90943 +90944 POINT(39.904437663372065 -73.28968926594959) bank90944 +90945 POINT(41.67874053230153 -74.8029988026158) bank90945 +90946 POINT(41.32680572973307 -74.92808382405607) bank90946 +90947 POINT(40.30488301072185 -74.97899299890608) bank90947 +90948 POINT(41.570418373686756 -73.47155452584543) bank90948 +90949 POINT(40.898905208335336 -73.49101922421056) bank90949 +90950 POINT(41.41355036067296 -74.66433856629438) bank90950 +90951 POINT(39.92636253147707 -73.98125621506415) bank90951 +90952 POINT(40.403338283505015 -74.38729347827791) bank90952 +90953 POINT(41.44789035567209 -73.19233408800271) bank90953 +90954 POINT(39.79902551231441 -74.45010785959441) bank90954 +90955 POINT(40.602374569056025 -74.53165344039209) bank90955 +90956 POINT(40.45342335504751 -73.97405197684827) bank90956 +90957 POINT(40.26115177708703 -73.32147192668236) bank90957 +90958 POINT(41.709858464277026 -73.41100303732559) bank90958 +90959 POINT(40.46011286838267 -73.71439981336167) bank90959 +90960 POINT(41.27045077994918 -73.78285826154668) bank90960 +90961 POINT(40.53893982381884 -73.5635344056526) bank90961 +90962 POINT(39.84616141440571 -74.9565790379269) bank90962 +90963 POINT(41.321169450729066 -73.168976395694) bank90963 +90964 POINT(40.67468150507611 -74.0985814925329) bank90964 +90965 POINT(39.78013880694017 -74.57489202632256) bank90965 +90966 POINT(39.98003156416336 -73.21331274304455) bank90966 +90967 POINT(41.45002066212694 -73.64500026269673) bank90967 +90968 POINT(40.748650247068234 -74.99090312758891) bank90968 +90969 POINT(40.951638942736885 -74.77789436866799) bank90969 +90970 POINT(39.807222107115706 -74.15123385053563) bank90970 +90971 POINT(39.94529158420221 -73.62822918013474) bank90971 +90972 POINT(41.53362016699747 -73.23796301125321) bank90972 +90973 POINT(40.46342610139181 -73.12000224888577) bank90973 +90974 POINT(41.04361472805638 -74.4925052092993) bank90974 +90975 POINT(40.33063909170093 -74.91519638542611) bank90975 +90976 POINT(41.149789413639056 -73.8524964679424) bank90976 +90977 POINT(39.89382386505479 -73.60302225848055) bank90977 +90978 POINT(40.38300576357481 -73.28283474305425) bank90978 +90979 POINT(41.708248090295406 -74.8328597401263) bank90979 +90980 POINT(39.956931340407785 -74.0405776178805) bank90980 +90981 POINT(41.54608049018762 -73.38721495728798) bank90981 +90982 POINT(41.098550499310846 -73.37566424787222) bank90982 +90983 POINT(40.186461080168506 -74.86155560056174) bank90983 +90984 POINT(41.6827893069418 -73.81920000432869) bank90984 +90985 POINT(41.2573269542099 -74.95113673498919) bank90985 +90986 POINT(40.81385268333708 -74.0711064107983) bank90986 +90987 POINT(39.82146108568295 -73.45775614381215) bank90987 +90988 POINT(39.903975556819695 -73.75238207400734) bank90988 +90989 POINT(41.711365303317976 -73.54459236609256) bank90989 +90990 POINT(41.0837617348701 -73.67010326176946) bank90990 +90991 POINT(41.07587778161043 -73.79322994094136) bank90991 +90992 POINT(40.83530575582788 -74.53899842320922) bank90992 +90993 POINT(40.780626307071124 -73.00887249938727) bank90993 +90994 POINT(39.98626662525586 -73.82916801478441) bank90994 +90995 POINT(40.64350766185232 -74.26241740172185) bank90995 +90996 POINT(41.05500570734049 -74.50672353962577) bank90996 +90997 POINT(40.6468834927461 -73.47060769474776) bank90997 +90998 POINT(40.93704024754561 -73.41605560145703) bank90998 +90999 POINT(40.460614585028026 -74.59787895426705) bank90999 +91000 POINT(40.46868403965228 -73.57892198559047) bank91000 +91001 POINT(40.19924304555299 -74.66693602005117) bank91001 +91002 POINT(41.01083687814344 -73.82667561470628) bank91002 +91003 POINT(40.13765055899266 -74.62790734028077) bank91003 +91004 POINT(40.61359806585107 -73.0136198071222) bank91004 +91005 POINT(39.90593132870116 -74.67767516648021) bank91005 +91006 POINT(39.72866869766771 -74.15585228808958) bank91006 +91007 POINT(41.197627879589206 -73.82341840271961) bank91007 +91008 POINT(41.49723995432322 -74.5479139950269) bank91008 +91009 POINT(39.994740292317736 -74.83726835184825) bank91009 +91010 POINT(40.15797022603813 -73.78813783602092) bank91010 +91011 POINT(39.840146215748696 -74.90289663686424) bank91011 +91012 POINT(41.209425455691466 -73.52712465969275) bank91012 +91013 POINT(41.3760488452554 -74.94059335257472) bank91013 +91014 POINT(41.58063778648171 -73.4638510352877) bank91014 +91015 POINT(39.93551677628899 -73.08627116539735) bank91015 +91016 POINT(40.140162039249105 -74.32070585109327) bank91016 +91017 POINT(39.87608434680419 -74.11600962915699) bank91017 +91018 POINT(41.49076370280092 -74.74767984363103) bank91018 +91019 POINT(40.98547342586899 -73.65604858837833) bank91019 +91020 POINT(40.26824777483296 -73.09651858154798) bank91020 +91021 POINT(40.19927757364412 -74.75711119513043) bank91021 +91022 POINT(40.26120218194054 -74.28614142875561) bank91022 +91023 POINT(40.14756624552339 -73.81813856019926) bank91023 +91024 POINT(39.997404376793064 -74.80675784057748) bank91024 +91025 POINT(41.60196867962981 -73.02309405347866) bank91025 +91026 POINT(41.39361892193516 -74.23307673729765) bank91026 +91027 POINT(40.679299010353745 -73.14033599863511) bank91027 +91028 POINT(41.65757073110617 -73.29744532974121) bank91028 +91029 POINT(41.22973325882279 -74.63136965612344) bank91029 +91030 POINT(41.33076725466568 -74.59796169168858) bank91030 +91031 POINT(40.086036362350924 -73.1717962721678) bank91031 +91032 POINT(40.96182043756096 -74.55077758924871) bank91032 +91033 POINT(40.91356835260868 -74.42408151475321) bank91033 +91034 POINT(40.72241404946872 -74.78506462450528) bank91034 +91035 POINT(40.92385417301648 -73.3312562355503) bank91035 +91036 POINT(41.586198718877846 -73.43878456324173) bank91036 +91037 POINT(41.35769057840663 -74.8227845315855) bank91037 +91038 POINT(40.05758455786298 -73.65642223835248) bank91038 +91039 POINT(41.081667797022234 -73.67593555916143) bank91039 +91040 POINT(41.10201654026843 -74.97820748681583) bank91040 +91041 POINT(40.877860127134376 -73.29578968771708) bank91041 +91042 POINT(41.077279029678365 -74.61302035134005) bank91042 +91043 POINT(40.85694298939524 -73.6791345560074) bank91043 +91044 POINT(40.6607944259043 -74.58369379415328) bank91044 +91045 POINT(40.55579676828948 -74.06933702351385) bank91045 +91046 POINT(40.9879224308012 -73.36842069901267) bank91046 +91047 POINT(40.651012176448006 -73.93052629670794) bank91047 +91048 POINT(40.18264822711182 -73.38915162532905) bank91048 +91049 POINT(40.71789601665926 -74.09271257316702) bank91049 +91050 POINT(41.459851612635745 -73.49499770747946) bank91050 +91051 POINT(40.86752941923028 -73.35581726037138) bank91051 +91052 POINT(41.69561694997211 -74.84981210573186) bank91052 +91053 POINT(39.75064022394904 -73.69007357860798) bank91053 +91054 POINT(40.57686890299564 -74.78311385129899) bank91054 +91055 POINT(41.56003365907226 -74.06351748921307) bank91055 +91056 POINT(40.284260515324526 -74.70824278269149) bank91056 +91057 POINT(40.62415404209535 -73.48062203819964) bank91057 +91058 POINT(40.74413933199782 -73.409268324183) bank91058 +91059 POINT(41.64388264161966 -74.63089274405088) bank91059 +91060 POINT(40.47481041435278 -73.56518123080745) bank91060 +91061 POINT(40.46004773363847 -73.59261376273317) bank91061 +91062 POINT(41.306714217065576 -73.42647854156465) bank91062 +91063 POINT(40.74087940817439 -73.1187694298835) bank91063 +91064 POINT(40.83454720260467 -73.20018595264824) bank91064 +91065 POINT(40.429352737725836 -73.93077721046856) bank91065 +91066 POINT(40.489203278958975 -74.43803367092313) bank91066 +91067 POINT(39.879148701399515 -74.47135393126035) bank91067 +91068 POINT(39.90241789011588 -73.21767331807166) bank91068 +91069 POINT(40.16760155583399 -74.44990952274) bank91069 +91070 POINT(39.793096633224344 -73.02845080241762) bank91070 +91071 POINT(39.73601690995529 -74.9213808437228) bank91071 +91072 POINT(40.71928103957264 -73.63999211488063) bank91072 +91073 POINT(40.842784632184014 -73.25182892660813) bank91073 +91074 POINT(41.52732721550824 -73.23682309215353) bank91074 +91075 POINT(41.090602763722416 -74.9435990489637) bank91075 +91076 POINT(40.11370734974239 -74.11886208100641) bank91076 +91077 POINT(40.071334149770806 -73.03567152838731) bank91077 +91078 POINT(41.548466714335085 -73.08805690931919) bank91078 +91079 POINT(41.40917955911736 -74.24258829769956) bank91079 +91080 POINT(40.61916178615025 -74.06459532904786) bank91080 +91081 POINT(40.409626562508386 -73.06348500964496) bank91081 +91082 POINT(41.378160954022874 -73.76588817090602) bank91082 +91083 POINT(40.71099783804069 -74.7253234454466) bank91083 +91084 POINT(41.06885463469757 -73.02244958081675) bank91084 +91085 POINT(41.44764796695961 -74.20802616025689) bank91085 +91086 POINT(40.6059292329909 -73.97402626277518) bank91086 +91087 POINT(41.48075556870311 -73.43395456278787) bank91087 +91088 POINT(41.47205623638202 -73.8426346384353) bank91088 +91089 POINT(41.44185775412192 -73.1760615892143) bank91089 +91090 POINT(40.36608269757335 -74.52162372409762) bank91090 +91091 POINT(41.53876284431641 -74.5958602584572) bank91091 +91092 POINT(40.16858865913682 -73.48688150271347) bank91092 +91093 POINT(41.26203001326338 -74.71524543237905) bank91093 +91094 POINT(40.782375545093416 -74.02153685846181) bank91094 +91095 POINT(41.57129178060412 -74.49231349843149) bank91095 +91096 POINT(40.30649203942758 -73.03175508288969) bank91096 +91097 POINT(40.495402339288795 -73.17679177770322) bank91097 +91098 POINT(40.02223791894994 -73.69225336277938) bank91098 +91099 POINT(41.40125645988787 -73.73328445355568) bank91099 +91100 POINT(41.503974835063396 -74.2388870926285) bank91100 +91101 POINT(40.35883460858608 -73.5596033263933) bank91101 +91102 POINT(40.20922056701599 -74.56787002900963) bank91102 +91103 POINT(41.250493036366535 -74.64540493640853) bank91103 +91104 POINT(40.966513610437744 -73.10587135152207) bank91104 +91105 POINT(40.70826861846811 -73.17312843802932) bank91105 +91106 POINT(41.62558604438939 -73.55451598190719) bank91106 +91107 POINT(41.329295743500616 -74.34885769311897) bank91107 +91108 POINT(40.303434335862136 -74.24679174444502) bank91108 +91109 POINT(41.031881837791126 -74.86929179966415) bank91109 +91110 POINT(41.44742773823285 -73.72109088940725) bank91110 +91111 POINT(40.408559610294326 -74.64037527098978) bank91111 +91112 POINT(41.51450451602612 -74.51530134976574) bank91112 +91113 POINT(40.47168719337932 -73.78848136993793) bank91113 +91114 POINT(39.79446034419501 -73.55685830532093) bank91114 +91115 POINT(41.7078776817373 -74.08022453226972) bank91115 +91116 POINT(40.13984847626965 -74.70878856721637) bank91116 +91117 POINT(40.00650622236794 -74.21053318342835) bank91117 +91118 POINT(40.38337336987381 -74.43754504871893) bank91118 +91119 POINT(40.36594312714529 -74.71244245467467) bank91119 +91120 POINT(40.43014776729622 -73.16988410962826) bank91120 +91121 POINT(40.87399452226064 -74.26873871151504) bank91121 +91122 POINT(39.92769588251411 -74.5845523844144) bank91122 +91123 POINT(41.65748371986602 -73.73408895959508) bank91123 +91124 POINT(40.02092024846153 -73.31886415369254) bank91124 +91125 POINT(41.58183165887478 -73.76382600272474) bank91125 +91126 POINT(39.88051216216108 -73.40838092376453) bank91126 +91127 POINT(41.16989798456813 -73.69849821935166) bank91127 +91128 POINT(40.83172498188668 -75.00424327520405) bank91128 +91129 POINT(41.10775293177969 -73.7236178673023) bank91129 +91130 POINT(41.12582480647317 -74.79022760797483) bank91130 +91131 POINT(40.76208154279576 -73.51500841129118) bank91131 +91132 POINT(39.83304094558336 -74.8643967524886) bank91132 +91133 POINT(40.748669756337705 -73.0741263655658) bank91133 +91134 POINT(41.44931891249161 -73.67828448942508) bank91134 +91135 POINT(41.40439639121783 -74.75776369969364) bank91135 +91136 POINT(39.8122956174008 -73.70811899034541) bank91136 +91137 POINT(40.41693072206608 -74.62795189716412) bank91137 +91138 POINT(39.76977884743188 -73.40226465088779) bank91138 +91139 POINT(40.40499575366699 -73.51380792272668) bank91139 +91140 POINT(40.250125364454185 -73.24252946497495) bank91140 +91141 POINT(40.71472320156097 -73.13201605628794) bank91141 +91142 POINT(40.56712993795583 -73.40922775163497) bank91142 +91143 POINT(40.827354672132806 -74.73250163998506) bank91143 +91144 POINT(40.037827628510705 -73.28176758376873) bank91144 +91145 POINT(41.437269974894235 -73.42625823433627) bank91145 +91146 POINT(40.11302172368872 -73.20632466413421) bank91146 +91147 POINT(40.694590482226666 -73.42623983110484) bank91147 +91148 POINT(41.418056272694535 -74.16904128257843) bank91148 +91149 POINT(41.66352062178669 -73.29736820696166) bank91149 +91150 POINT(41.20282207350075 -73.20343257587068) bank91150 +91151 POINT(40.12311795697662 -73.77036293220627) bank91151 +91152 POINT(40.00783711261991 -73.87721453166364) bank91152 +91153 POINT(40.774623453305296 -74.70645795037514) bank91153 +91154 POINT(40.92556212130232 -73.36238575598053) bank91154 +91155 POINT(41.358938129036574 -74.52505679273798) bank91155 +91156 POINT(41.09445358672353 -74.43663801217401) bank91156 +91157 POINT(39.73292658163382 -73.24416586902856) bank91157 +91158 POINT(40.0637555556301 -74.806914490911) bank91158 +91159 POINT(40.034392361112445 -73.21524233808404) bank91159 +91160 POINT(41.09345084064503 -74.90573713380174) bank91160 +91161 POINT(39.8196359872409 -74.59865878057902) bank91161 +91162 POINT(40.381107619052145 -74.89591218616906) bank91162 +91163 POINT(40.277747432482855 -73.95600731702041) bank91163 +91164 POINT(41.70721301218211 -74.73222381643582) bank91164 +91165 POINT(40.957243183250284 -73.40896211298147) bank91165 +91166 POINT(40.10139145060019 -74.59754708025689) bank91166 +91167 POINT(41.346848154376424 -73.68262464997846) bank91167 +91168 POINT(40.947594339625546 -73.06027362452325) bank91168 +91169 POINT(39.836696297912646 -73.69058553888244) bank91169 +91170 POINT(40.11725050746866 -73.35807682030224) bank91170 +91171 POINT(41.6663732901806 -73.65098454812883) bank91171 +91172 POINT(41.16504953180879 -74.48851933989371) bank91172 +91173 POINT(40.21690480072348 -74.62007652166422) bank91173 +91174 POINT(40.410611571679354 -74.89244373256894) bank91174 +91175 POINT(40.07348223120709 -74.85538607525112) bank91175 +91176 POINT(40.864081034373996 -74.62959765985148) bank91176 +91177 POINT(40.0564323717527 -73.80689194095227) bank91177 +91178 POINT(39.756748462950924 -73.63485954742947) bank91178 +91179 POINT(40.475761493346965 -74.91850426826487) bank91179 +91180 POINT(40.744667176408534 -74.33726027203933) bank91180 +91181 POINT(41.10918610707328 -73.87473379460027) bank91181 +91182 POINT(40.17032478134382 -74.12661062184189) bank91182 +91183 POINT(40.664121667969496 -74.51586056795742) bank91183 +91184 POINT(40.72479458344496 -73.96445687886171) bank91184 +91185 POINT(40.90540411366541 -73.21916369226517) bank91185 +91186 POINT(40.503146816945105 -74.41999333163302) bank91186 +91187 POINT(40.09733172647064 -74.66297983526943) bank91187 +91188 POINT(41.448135057828615 -74.2391045833998) bank91188 +91189 POINT(40.91986543891725 -73.02032275685579) bank91189 +91190 POINT(40.22366363961966 -74.66955570385561) bank91190 +91191 POINT(39.84408682299022 -73.85771669153219) bank91191 +91192 POINT(40.78877851823408 -73.70657741036399) bank91192 +91193 POINT(39.85037466798338 -74.70520322858056) bank91193 +91194 POINT(39.75491680132475 -74.17115778466729) bank91194 +91195 POINT(41.13747386466744 -74.41757000686712) bank91195 +91196 POINT(40.91790711347908 -74.4723274167347) bank91196 +91197 POINT(41.644590019780956 -73.17063735458494) bank91197 +91198 POINT(41.255648311293974 -74.40488154939321) bank91198 +91199 POINT(40.49388539445144 -73.6016585105966) bank91199 +91200 POINT(40.49968956222098 -74.50255242500997) bank91200 +91201 POINT(40.9304216633624 -73.2422803135275) bank91201 +91202 POINT(41.37441095060309 -74.70206144166207) bank91202 +91203 POINT(41.703548607427166 -74.43317300289043) bank91203 +91204 POINT(39.76616774881626 -73.18161579882948) bank91204 +91205 POINT(39.83934910054024 -74.84669712288085) bank91205 +91206 POINT(40.98703242237192 -73.3849621884644) bank91206 +91207 POINT(41.64531329164491 -73.42410661342777) bank91207 +91208 POINT(41.537956518858465 -74.35739087476263) bank91208 +91209 POINT(41.67229951452916 -73.88481467088397) bank91209 +91210 POINT(40.79948589413016 -74.0599586727941) bank91210 +91211 POINT(40.71700620731747 -74.32511848027845) bank91211 +91212 POINT(40.25803585119605 -74.63169086430393) bank91212 +91213 POINT(40.312641125415766 -73.80577366481327) bank91213 +91214 POINT(39.80961278383105 -73.26191386812859) bank91214 +91215 POINT(40.94529156708634 -73.09961687335242) bank91215 +91216 POINT(41.137918276112174 -73.63294000419435) bank91216 +91217 POINT(40.15259067555216 -74.6361172775519) bank91217 +91218 POINT(40.329125238029505 -73.25108203751125) bank91218 +91219 POINT(40.750116295330976 -74.01695493624726) bank91219 +91220 POINT(40.8919200436073 -73.15995631188837) bank91220 +91221 POINT(40.896115646784125 -73.78205380028069) bank91221 +91222 POINT(40.4848614159059 -73.50177339161311) bank91222 +91223 POINT(40.08388302488487 -74.43075823194788) bank91223 +91224 POINT(39.96604830464015 -74.18082248615752) bank91224 +91225 POINT(41.23056485704514 -73.80449136473888) bank91225 +91226 POINT(39.821225404821774 -74.10117732142625) bank91226 +91227 POINT(41.665051518011346 -74.13819768307418) bank91227 +91228 POINT(41.177588758449 -74.8533612788287) bank91228 +91229 POINT(40.31161597158458 -74.75441527250469) bank91229 +91230 POINT(41.47863228053879 -74.45566143602218) bank91230 +91231 POINT(40.23290672284776 -73.98896584112417) bank91231 +91232 POINT(39.741678969265486 -73.03269179343809) bank91232 +91233 POINT(41.24265724727387 -74.97034895187085) bank91233 +91234 POINT(40.95334197265552 -73.97653365271128) bank91234 +91235 POINT(40.513176627720796 -74.17125560276124) bank91235 +91236 POINT(40.947679399000364 -74.97142936184572) bank91236 +91237 POINT(41.60533176275098 -74.17904084449549) bank91237 +91238 POINT(41.493291978385955 -74.53261629421864) bank91238 +91239 POINT(41.10419964326702 -73.54052630942157) bank91239 +91240 POINT(41.2089384863727 -73.6168258363221) bank91240 +91241 POINT(40.92506487979958 -74.16460166003716) bank91241 +91242 POINT(41.420992637328 -73.56721814533773) bank91242 +91243 POINT(39.91238685146861 -74.28262097218425) bank91243 +91244 POINT(41.01473198979746 -74.76011528939905) bank91244 +91245 POINT(41.27000382833749 -73.11907335941235) bank91245 +91246 POINT(39.733184140305674 -74.71779280921653) bank91246 +91247 POINT(40.558959039694855 -74.7744160423813) bank91247 +91248 POINT(41.13102748139982 -73.59870602978626) bank91248 +91249 POINT(40.800666280099755 -74.0361555231594) bank91249 +91250 POINT(40.429968787161414 -74.17238759312431) bank91250 +91251 POINT(41.28828272096164 -74.99134312683361) bank91251 +91252 POINT(40.31796146120087 -73.80902477102107) bank91252 +91253 POINT(41.36747013961489 -73.63728761174087) bank91253 +91254 POINT(39.94791922107107 -74.75046480168923) bank91254 +91255 POINT(41.26841718697278 -73.4922540312181) bank91255 +91256 POINT(41.188862488922375 -73.48119533200313) bank91256 +91257 POINT(40.17647986796781 -73.59268929694149) bank91257 +91258 POINT(39.95776665083778 -73.74427698440654) bank91258 +91259 POINT(39.86072586922722 -74.46873413277189) bank91259 +91260 POINT(41.52010862462724 -74.27248492263428) bank91260 +91261 POINT(40.09572901372443 -74.99638034780745) bank91261 +91262 POINT(41.02580166841989 -74.53557852411544) bank91262 +91263 POINT(40.74512801082261 -74.81785845150803) bank91263 +91264 POINT(40.472223185913755 -73.34439422657645) bank91264 +91265 POINT(40.21812339517164 -73.63904949202878) bank91265 +91266 POINT(40.86076063514487 -73.12726441728587) bank91266 +91267 POINT(41.70588893018116 -73.15664571767249) bank91267 +91268 POINT(39.951956906261096 -73.0655651556783) bank91268 +91269 POINT(41.08864028795969 -74.52990878197882) bank91269 +91270 POINT(40.47640782599541 -74.46428397258181) bank91270 +91271 POINT(39.791947282610586 -74.83422416731189) bank91271 +91272 POINT(40.99015090904421 -73.9938472345483) bank91272 +91273 POINT(39.87716797198691 -74.66679227064702) bank91273 +91274 POINT(40.29544084791987 -74.98777095915047) bank91274 +91275 POINT(41.226001720063245 -73.2809242891003) bank91275 +91276 POINT(40.01214304964602 -74.00488185770573) bank91276 +91277 POINT(40.21931439067508 -74.2127831292969) bank91277 +91278 POINT(40.44052160533805 -74.33912748812361) bank91278 +91279 POINT(40.390787298224446 -74.49555713266253) bank91279 +91280 POINT(40.00127885010659 -74.2768247972256) bank91280 +91281 POINT(39.955990152223144 -74.41500319801999) bank91281 +91282 POINT(40.080570426924666 -73.90866434857756) bank91282 +91283 POINT(41.04717750197204 -74.404257725672) bank91283 +91284 POINT(40.22541183079208 -73.92425030400892) bank91284 +91285 POINT(40.222172629783245 -74.26116513613229) bank91285 +91286 POINT(40.027645615116896 -74.36630311375443) bank91286 +91287 POINT(41.34978824727725 -73.45150200814946) bank91287 +91288 POINT(40.0018498431109 -74.19744566130807) bank91288 +91289 POINT(39.84229951718221 -73.74821096718343) bank91289 +91290 POINT(40.52980135928337 -74.43367760664867) bank91290 +91291 POINT(41.16076854199928 -73.1495976154704) bank91291 +91292 POINT(40.93534160410438 -74.06714607944843) bank91292 +91293 POINT(41.61030429645498 -74.20811602791669) bank91293 +91294 POINT(41.16239267733211 -74.47527464713328) bank91294 +91295 POINT(40.771747867474296 -74.26629830642736) bank91295 +91296 POINT(40.06804188782655 -74.75131874931022) bank91296 +91297 POINT(39.78100029735203 -74.77113038202495) bank91297 +91298 POINT(41.26888148154261 -74.6540905612894) bank91298 +91299 POINT(40.853209737816975 -73.74295466567143) bank91299 +91300 POINT(41.27877322866206 -73.13331874842886) bank91300 +91301 POINT(39.86930415105569 -73.96897296870422) bank91301 +91302 POINT(40.881384180798364 -74.89900706124946) bank91302 +91303 POINT(41.259560331944826 -74.7019768672584) bank91303 +91304 POINT(40.302910056493786 -74.07527658110271) bank91304 +91305 POINT(41.072030556641714 -74.47987193805305) bank91305 +91306 POINT(40.04800706386517 -73.0742103853703) bank91306 +91307 POINT(40.59774867350799 -73.70355677654305) bank91307 +91308 POINT(41.39331032649204 -73.50281979181207) bank91308 +91309 POINT(40.58191832659107 -73.41216596347105) bank91309 +91310 POINT(39.84366662212453 -74.49871374286) bank91310 +91311 POINT(40.504952696558064 -74.56507863327654) bank91311 +91312 POINT(40.38562663581892 -73.60514180521865) bank91312 +91313 POINT(39.845142619494766 -74.56101376033529) bank91313 +91314 POINT(41.46920978845339 -74.96417322930775) bank91314 +91315 POINT(40.05094622901182 -73.09912696566661) bank91315 +91316 POINT(40.75251177223556 -74.39598165714031) bank91316 +91317 POINT(41.21962647105619 -73.9904044872743) bank91317 +91318 POINT(41.60186120986111 -74.56281221595962) bank91318 +91319 POINT(41.24158578360191 -74.70902551045354) bank91319 +91320 POINT(41.352023686840695 -74.67404901750436) bank91320 +91321 POINT(40.00291239996748 -73.17491826414927) bank91321 +91322 POINT(40.08150269679667 -74.5691130420584) bank91322 +91323 POINT(40.524826772372265 -73.38367719426881) bank91323 +91324 POINT(40.983931462312974 -73.87197327835977) bank91324 +91325 POINT(40.88999335483755 -74.82534089182144) bank91325 +91326 POINT(40.642146995432626 -74.56251325924337) bank91326 +91327 POINT(40.248177915667306 -73.91044914008435) bank91327 +91328 POINT(40.89783881619457 -73.04794669145747) bank91328 +91329 POINT(41.58845895528707 -73.22511545390806) bank91329 +91330 POINT(40.19316871492546 -74.67018731597186) bank91330 +91331 POINT(41.66016112745101 -74.86169744790006) bank91331 +91332 POINT(41.19928282642901 -73.85954207643701) bank91332 +91333 POINT(41.41558544864027 -73.3830985657861) bank91333 +91334 POINT(40.45937680121714 -74.88296749416124) bank91334 +91335 POINT(41.61292744281259 -74.88709353006794) bank91335 +91336 POINT(39.763845369270165 -74.47684919396112) bank91336 +91337 POINT(41.443842634102964 -74.1962497837711) bank91337 +91338 POINT(40.41901947194038 -73.0766326626708) bank91338 +91339 POINT(41.22783034756869 -73.50094847560727) bank91339 +91340 POINT(40.918005447204905 -73.38574513541238) bank91340 +91341 POINT(40.68573279328181 -73.4427422397881) bank91341 +91342 POINT(40.88651779761207 -74.494876220801) bank91342 +91343 POINT(41.34008496222549 -73.04032023280168) bank91343 +91344 POINT(40.02268663385427 -74.90219737252812) bank91344 +91345 POINT(40.78494146213038 -74.00549173884073) bank91345 +91346 POINT(41.064880692040326 -73.84003157828388) bank91346 +91347 POINT(40.350156057724504 -74.05021756121202) bank91347 +91348 POINT(40.87531211039945 -74.74570435719701) bank91348 +91349 POINT(40.917548624541546 -74.45725509999235) bank91349 +91350 POINT(40.30475418520379 -73.57130825179378) bank91350 +91351 POINT(39.78926905947541 -73.08155776462443) bank91351 +91352 POINT(40.34290803722441 -73.96026105764386) bank91352 +91353 POINT(41.66670000844592 -73.26414359010043) bank91353 +91354 POINT(40.487553130224946 -74.18254851649318) bank91354 +91355 POINT(40.87550416015097 -74.52191131342371) bank91355 +91356 POINT(41.19591845992435 -74.280726534654) bank91356 +91357 POINT(40.414362470609056 -74.04283239212798) bank91357 +91358 POINT(41.09695786623842 -73.29249228590955) bank91358 +91359 POINT(40.81590494435986 -73.80212741893872) bank91359 +91360 POINT(40.381167423616255 -74.6911559160839) bank91360 +91361 POINT(41.231431970923005 -74.22864699106094) bank91361 +91362 POINT(39.972878347225226 -73.0375794717276) bank91362 +91363 POINT(41.18472773152688 -73.53056983747658) bank91363 +91364 POINT(39.737356005055545 -74.05073924761209) bank91364 +91365 POINT(40.891143723719516 -74.55452704316288) bank91365 +91366 POINT(40.945148402556114 -74.19945715194008) bank91366 +91367 POINT(41.4554205409192 -73.23833946546826) bank91367 +91368 POINT(41.549665689143254 -74.8169278674893) bank91368 +91369 POINT(40.80915185834969 -74.81995603373997) bank91369 +91370 POINT(40.993497128905126 -74.46306286487977) bank91370 +91371 POINT(40.38449246200704 -73.12530825412077) bank91371 +91372 POINT(40.12460284521123 -74.47744828035275) bank91372 +91373 POINT(41.66737451793643 -74.03521962643357) bank91373 +91374 POINT(40.53913854544097 -73.29214377385232) bank91374 +91375 POINT(40.633197842710835 -74.02180639974532) bank91375 +91376 POINT(40.56958817716538 -74.85927441266062) bank91376 +91377 POINT(39.71498953230461 -73.1259811160545) bank91377 +91378 POINT(41.242286420675896 -74.18571731540156) bank91378 +91379 POINT(39.770448515925025 -73.55467372101866) bank91379 +91380 POINT(40.31222470893139 -73.10077484213464) bank91380 +91381 POINT(39.95875444526757 -73.87575555933907) bank91381 +91382 POINT(41.240392633466506 -74.34125565754356) bank91382 +91383 POINT(40.305122292560064 -74.35886046157141) bank91383 +91384 POINT(41.256991086483346 -74.5146995118927) bank91384 +91385 POINT(41.185450069983936 -73.90102295222128) bank91385 +91386 POINT(40.01638470905024 -73.06853471147001) bank91386 +91387 POINT(41.35614720896869 -74.19973250713676) bank91387 +91388 POINT(40.96272393644837 -73.77883097582566) bank91388 +91389 POINT(40.82869782821023 -73.13791505098727) bank91389 +91390 POINT(39.80375062722836 -74.94213414843036) bank91390 +91391 POINT(40.56068381038417 -74.95196716949565) bank91391 +91392 POINT(40.270691829047315 -74.01109694383776) bank91392 +91393 POINT(40.441146943809755 -74.37569097388834) bank91393 +91394 POINT(40.226859890485784 -73.09480287775382) bank91394 +91395 POINT(40.622194234369964 -73.91881263713563) bank91395 +91396 POINT(40.415175395543635 -74.30865535440019) bank91396 +91397 POINT(40.87124981905322 -74.88891649899386) bank91397 +91398 POINT(41.51948409556958 -73.77978230064393) bank91398 +91399 POINT(40.564256093008986 -73.22760668765702) bank91399 +91400 POINT(40.99436130240527 -73.34171786893889) bank91400 +91401 POINT(40.85251287244059 -73.54382297178732) bank91401 +91402 POINT(40.90216386182361 -73.17329386169556) bank91402 +91403 POINT(40.29156164718481 -73.99245295908447) bank91403 +91404 POINT(41.41028388737841 -73.36009541849391) bank91404 +91405 POINT(41.56550782257195 -73.38514853593256) bank91405 +91406 POINT(39.71585699199496 -73.34796708969891) bank91406 +91407 POINT(40.15207184710514 -73.74070727354263) bank91407 +91408 POINT(40.606006359418494 -73.63388379459333) bank91408 +91409 POINT(40.50698356403221 -74.11675207571953) bank91409 +91410 POINT(41.19109935321408 -73.33283715303111) bank91410 +91411 POINT(40.9280492507868 -73.30270985746053) bank91411 +91412 POINT(39.72659949309985 -73.59595424834802) bank91412 +91413 POINT(40.65018535072393 -74.44853493645407) bank91413 +91414 POINT(40.171245670228146 -73.91662351374586) bank91414 +91415 POINT(40.1260473825896 -74.03480187256218) bank91415 +91416 POINT(40.994800592475514 -74.25466795894167) bank91416 +91417 POINT(40.26703246737906 -73.48712256403654) bank91417 +91418 POINT(40.25360599703569 -74.66266206140202) bank91418 +91419 POINT(40.36969041064257 -74.42689317770542) bank91419 +91420 POINT(41.6701613969993 -74.87541188205059) bank91420 +91421 POINT(40.215256203896644 -73.2781967474941) bank91421 +91422 POINT(41.1303585837052 -73.01412199171547) bank91422 +91423 POINT(40.49534176256367 -73.84310993680022) bank91423 +91424 POINT(39.73414007756288 -73.86189193766181) bank91424 +91425 POINT(39.82686667002723 -73.08630237258832) bank91425 +91426 POINT(41.37586412243759 -74.7408885479039) bank91426 +91427 POINT(40.887484790439245 -74.16819741288982) bank91427 +91428 POINT(41.62440418668176 -74.01518092006862) bank91428 +91429 POINT(40.981601283312116 -73.12143942412986) bank91429 +91430 POINT(41.19517321742934 -74.66092827367368) bank91430 +91431 POINT(41.20079405033582 -73.47045352407861) bank91431 +91432 POINT(41.50627134839217 -74.00704072804163) bank91432 +91433 POINT(41.551417550984475 -74.845125641391) bank91433 +91434 POINT(40.3049153334273 -73.10875947395776) bank91434 +91435 POINT(40.070868667149696 -73.67848995531773) bank91435 +91436 POINT(40.05526636101505 -74.50621376926182) bank91436 +91437 POINT(41.31862679593938 -73.62261352271105) bank91437 +91438 POINT(40.13488992903084 -74.81727471439686) bank91438 +91439 POINT(39.83353057471712 -74.4247877510915) bank91439 +91440 POINT(41.08501331943311 -73.11639595706711) bank91440 +91441 POINT(39.949142955536146 -74.58847314587369) bank91441 +91442 POINT(41.18859128962459 -74.161187024966) bank91442 +91443 POINT(41.65398369200996 -73.64104711252561) bank91443 +91444 POINT(40.979776406568476 -74.12725790645797) bank91444 +91445 POINT(39.98926387734995 -73.02539326918196) bank91445 +91446 POINT(41.24438729177443 -73.11913303473551) bank91446 +91447 POINT(40.941093435717605 -74.31546146100337) bank91447 +91448 POINT(39.94818073712715 -74.38013277648038) bank91448 +91449 POINT(40.596344277327844 -74.31464295029606) bank91449 +91450 POINT(40.91234705687529 -74.76331351789774) bank91450 +91451 POINT(40.62858581721262 -74.25844443480547) bank91451 +91452 POINT(40.532812556413994 -74.91341822468529) bank91452 +91453 POINT(41.171133645606105 -73.53533899722808) bank91453 +91454 POINT(41.2485768600164 -74.22482063290919) bank91454 +91455 POINT(40.376703971534006 -74.87666110139928) bank91455 +91456 POINT(40.90410943326446 -73.82103899232828) bank91456 +91457 POINT(40.89374132827997 -74.83461044835211) bank91457 +91458 POINT(40.20262387473519 -73.55657611693074) bank91458 +91459 POINT(41.497722371731584 -73.87405392073414) bank91459 +91460 POINT(40.33921346304853 -73.16537483510648) bank91460 +91461 POINT(41.31021186264344 -74.5030917319387) bank91461 +91462 POINT(40.754900905842845 -74.3265630474525) bank91462 +91463 POINT(40.79103648786471 -74.91055425078724) bank91463 +91464 POINT(41.30347197682264 -74.88356070036023) bank91464 +91465 POINT(41.67350962284115 -73.1132189125448) bank91465 +91466 POINT(40.65522316306412 -73.17054215453538) bank91466 +91467 POINT(39.96547730729938 -73.39042481545789) bank91467 +91468 POINT(39.725217653131196 -74.72097388310111) bank91468 +91469 POINT(40.4812594958734 -74.61093596299752) bank91469 +91470 POINT(40.7503752716095 -73.28541846485341) bank91470 +91471 POINT(40.971965231802166 -74.80872058822806) bank91471 +91472 POINT(39.83558914009764 -74.42431763450172) bank91472 +91473 POINT(40.29913823475725 -74.15679905051692) bank91473 +91474 POINT(40.56752987850386 -73.33208453469754) bank91474 +91475 POINT(41.523466758041245 -74.93733397699805) bank91475 +91476 POINT(41.32547078093595 -74.55164077035643) bank91476 +91477 POINT(41.38124278691647 -73.35096836996618) bank91477 +91478 POINT(40.24797671465428 -74.35471502935853) bank91478 +91479 POINT(40.349521588325736 -74.83852374064591) bank91479 +91480 POINT(40.342878483125816 -73.2693280293411) bank91480 +91481 POINT(41.31669298552234 -74.4713150633908) bank91481 +91482 POINT(39.726663097498545 -74.45624659741964) bank91482 +91483 POINT(41.06506181344956 -74.63039686031416) bank91483 +91484 POINT(41.54126646936177 -74.70746427626968) bank91484 +91485 POINT(40.316294096971916 -73.518253021107) bank91485 +91486 POINT(41.23658780186049 -73.2612169527297) bank91486 +91487 POINT(40.13626231893444 -73.99267873415482) bank91487 +91488 POINT(40.46024058602589 -73.80863802070353) bank91488 +91489 POINT(41.21608513049242 -73.43435797230971) bank91489 +91490 POINT(40.190365583221 -73.87741570496901) bank91490 +91491 POINT(41.540816670104746 -74.25013765808559) bank91491 +91492 POINT(40.58188156488774 -74.55666177164895) bank91492 +91493 POINT(40.06964011818011 -74.3037249740586) bank91493 +91494 POINT(41.69197363528575 -74.537941474248) bank91494 +91495 POINT(41.431975765237766 -73.84482353436178) bank91495 +91496 POINT(41.664366877726 -74.99020963145263) bank91496 +91497 POINT(40.02255845237934 -73.84027122814423) bank91497 +91498 POINT(41.20936158052321 -73.14144962315962) bank91498 +91499 POINT(41.600630726747106 -74.32669175061392) bank91499 +91500 POINT(40.69378333185512 -74.00121910852795) bank91500 +91501 POINT(39.877981089556336 -74.24478379273974) bank91501 +91502 POINT(40.66536836018391 -73.36797992028824) bank91502 +91503 POINT(41.176545645467165 -74.9205768445425) bank91503 +91504 POINT(40.16774401043238 -74.44655178371656) bank91504 +91505 POINT(41.059695484959526 -73.37100346824135) bank91505 +91506 POINT(40.13547448471491 -73.96206522728215) bank91506 +91507 POINT(41.236570615905414 -73.22763652906752) bank91507 +91508 POINT(39.73179497224896 -74.97580645827192) bank91508 +91509 POINT(41.521553856805994 -73.33651806537719) bank91509 +91510 POINT(40.32431498090487 -73.74564887070656) bank91510 +91511 POINT(39.81661488897829 -73.83347436708237) bank91511 +91512 POINT(40.812397197311725 -73.67203284448283) bank91512 +91513 POINT(40.66211755095736 -74.90450644413392) bank91513 +91514 POINT(40.61665572468704 -74.0447485250254) bank91514 +91515 POINT(40.29028987251688 -74.2127632401901) bank91515 +91516 POINT(40.90426112648299 -74.34935582775633) bank91516 +91517 POINT(40.85705593453095 -73.96369638012179) bank91517 +91518 POINT(41.495057572517716 -73.18028965466502) bank91518 +91519 POINT(41.18332267608108 -74.66577212463845) bank91519 +91520 POINT(40.70538930727778 -74.84501472035514) bank91520 +91521 POINT(40.84421791660984 -74.3286023967067) bank91521 +91522 POINT(40.20061876391899 -73.26037825918877) bank91522 +91523 POINT(41.157449645102275 -73.23704411180964) bank91523 +91524 POINT(40.54993592214577 -73.92674651506289) bank91524 +91525 POINT(40.80951563032666 -73.72330740108228) bank91525 +91526 POINT(40.88363468932454 -74.39155914112612) bank91526 +91527 POINT(41.68505585080419 -74.1457908989764) bank91527 +91528 POINT(41.227087024989906 -73.58646005463562) bank91528 +91529 POINT(40.62361032873792 -73.88162635167286) bank91529 +91530 POINT(40.19718170643964 -73.52343247382899) bank91530 +91531 POINT(40.683639059150735 -74.53296340242386) bank91531 +91532 POINT(41.71092329107352 -73.28588801974371) bank91532 +91533 POINT(39.7535151053942 -74.40640566399256) bank91533 +91534 POINT(40.50760173900215 -74.15348035072073) bank91534 +91535 POINT(40.275505859901436 -73.5357526022606) bank91535 +91536 POINT(40.71128894141194 -74.66272752600655) bank91536 +91537 POINT(41.480121836556705 -73.27444665850173) bank91537 +91538 POINT(41.04189121184156 -74.26852893636712) bank91538 +91539 POINT(40.68012383311533 -74.88438941229211) bank91539 +91540 POINT(40.108485694582996 -73.50659317603352) bank91540 +91541 POINT(40.78442817008119 -74.05643898448011) bank91541 +91542 POINT(41.49233350676369 -74.69080346541156) bank91542 +91543 POINT(41.58848263342229 -73.05315228304912) bank91543 +91544 POINT(40.4755341153992 -74.88519055059066) bank91544 +91545 POINT(40.806490714987824 -73.06556291544112) bank91545 +91546 POINT(40.156672651910306 -74.57180810114704) bank91546 +91547 POINT(40.32667580573761 -74.08251306690492) bank91547 +91548 POINT(41.01339760660686 -74.17761055901586) bank91548 +91549 POINT(39.98721729812591 -73.83161483264328) bank91549 +91550 POINT(40.27392154423071 -74.38913913685647) bank91550 +91551 POINT(41.093001103021095 -73.9689845949064) bank91551 +91552 POINT(40.35638934017127 -73.82306549435464) bank91552 +91553 POINT(41.03893043977633 -74.28977341835402) bank91553 +91554 POINT(40.812997935526695 -73.3955636099428) bank91554 +91555 POINT(40.00539266876569 -73.48035572480312) bank91555 +91556 POINT(40.985081727197574 -74.72429174135773) bank91556 +91557 POINT(40.34031117665371 -74.9618068396462) bank91557 +91558 POINT(40.574865652227565 -74.16787029643116) bank91558 +91559 POINT(40.50670904790337 -74.90141015159915) bank91559 +91560 POINT(41.44079070095621 -74.07882819815956) bank91560 +91561 POINT(41.312900026212 -74.28900809793326) bank91561 +91562 POINT(40.19111055985375 -73.24846140537086) bank91562 +91563 POINT(40.77788719442785 -74.51545271884089) bank91563 +91564 POINT(39.73571999587309 -74.25855906435498) bank91564 +91565 POINT(40.90287731304777 -74.45499959486622) bank91565 +91566 POINT(41.51146313463151 -73.77539531504311) bank91566 +91567 POINT(40.36838305581972 -73.6306661164619) bank91567 +91568 POINT(40.26906219051158 -73.54162749385542) bank91568 +91569 POINT(41.709133954929506 -73.48506311949707) bank91569 +91570 POINT(40.531624742869035 -73.95033226769561) bank91570 +91571 POINT(40.94821900979623 -73.30748758697011) bank91571 +91572 POINT(41.13759926864544 -73.90385373220097) bank91572 +91573 POINT(40.73147908336176 -73.73685993358072) bank91573 +91574 POINT(41.398102329598146 -73.11228873538802) bank91574 +91575 POINT(40.65917597473747 -74.55088876313648) bank91575 +91576 POINT(41.61110636312671 -73.07636599611598) bank91576 +91577 POINT(41.52486938146892 -74.97055982952506) bank91577 +91578 POINT(41.48865479718274 -74.32280019580227) bank91578 +91579 POINT(41.47803027103692 -73.28502736945964) bank91579 +91580 POINT(40.39202778901513 -74.85161850850223) bank91580 +91581 POINT(41.31885216360569 -73.40776434312448) bank91581 +91582 POINT(41.36978903575185 -73.58653396319004) bank91582 +91583 POINT(41.05994428046464 -74.62444003276676) bank91583 +91584 POINT(40.6706093049285 -74.6431159448079) bank91584 +91585 POINT(41.42523411884839 -74.3879914894758) bank91585 +91586 POINT(40.64663376259139 -73.7072282968735) bank91586 +91587 POINT(40.54767266991217 -73.7631380131806) bank91587 +91588 POINT(40.468125861093185 -73.96352854071222) bank91588 +91589 POINT(40.206600495628265 -74.89850304013662) bank91589 +91590 POINT(41.57227635821063 -73.4440020752582) bank91590 +91591 POINT(41.15921879697424 -74.89732543355105) bank91591 +91592 POINT(40.94141592408637 -74.08085469814347) bank91592 +91593 POINT(40.772608578483144 -74.06161861652011) bank91593 +91594 POINT(40.08016862613004 -73.51205260675101) bank91594 +91595 POINT(41.06451560545119 -73.76388246261477) bank91595 +91596 POINT(41.527705910586285 -73.14488135603487) bank91596 +91597 POINT(39.939618542548764 -73.28380705023702) bank91597 +91598 POINT(39.72423121875584 -74.13226093130845) bank91598 +91599 POINT(40.99306053666843 -74.15114377495101) bank91599 +91600 POINT(40.29100392329388 -73.86279686343575) bank91600 +91601 POINT(41.682406187253044 -73.02401389926513) bank91601 +91602 POINT(41.31936305066751 -73.49736562859465) bank91602 +91603 POINT(41.40482407559212 -73.96726001692193) bank91603 +91604 POINT(41.6359762573968 -73.81126349462868) bank91604 +91605 POINT(40.91249891587786 -73.78188116423581) bank91605 +91606 POINT(39.97728096035628 -74.94936651336371) bank91606 +91607 POINT(40.840627276860204 -74.42061734273591) bank91607 +91608 POINT(41.133032724536434 -74.24826491578224) bank91608 +91609 POINT(41.52463376465663 -73.80768764618084) bank91609 +91610 POINT(41.18190332881295 -73.35739561451281) bank91610 +91611 POINT(40.5702050089227 -73.09035952101394) bank91611 +91612 POINT(41.610287383793555 -73.27490817879456) bank91612 +91613 POINT(39.91994058099881 -74.65880060039147) bank91613 +91614 POINT(40.14224997622803 -74.17541420061826) bank91614 +91615 POINT(40.4637188837418 -73.04292423894563) bank91615 +91616 POINT(40.66406228098799 -74.26173760452625) bank91616 +91617 POINT(40.23802849523573 -73.58065506726749) bank91617 +91618 POINT(40.65683294918811 -74.37196399240278) bank91618 +91619 POINT(41.08312643859437 -73.93332365814575) bank91619 +91620 POINT(40.467675714670875 -74.88951927290171) bank91620 +91621 POINT(39.766006891500744 -73.82903254664271) bank91621 +91622 POINT(40.185518743931866 -73.72599421261451) bank91622 +91623 POINT(39.74325474372052 -74.68345098278199) bank91623 +91624 POINT(41.30839348718958 -74.56617551601502) bank91624 +91625 POINT(39.77685476482888 -73.49462247814303) bank91625 +91626 POINT(40.72773382114685 -73.43125112065925) bank91626 +91627 POINT(40.156287543696074 -74.76345305938194) bank91627 +91628 POINT(40.46089303756162 -73.75691105397922) bank91628 +91629 POINT(39.788745461259495 -74.17311857364474) bank91629 +91630 POINT(40.50246220699114 -73.68340939463309) bank91630 +91631 POINT(40.16207435858807 -73.3169418643844) bank91631 +91632 POINT(41.61176865576049 -73.1216655872817) bank91632 +91633 POINT(40.61465497798793 -73.46586534443264) bank91633 +91634 POINT(40.44002639891963 -73.63697349597084) bank91634 +91635 POINT(41.03833356297203 -73.34796636859735) bank91635 +91636 POINT(40.64772414408294 -74.18956602440032) bank91636 +91637 POINT(40.85475562835769 -74.28239074619465) bank91637 +91638 POINT(40.90471640100188 -73.15828319861195) bank91638 +91639 POINT(40.90481955855382 -74.5583369047522) bank91639 +91640 POINT(40.54261633629172 -74.63480254107567) bank91640 +91641 POINT(41.220221597453346 -74.08174023489939) bank91641 +91642 POINT(39.79146346603731 -73.4329465016475) bank91642 +91643 POINT(41.259555190699615 -74.55395265879316) bank91643 +91644 POINT(41.26708317525235 -73.94885886890086) bank91644 +91645 POINT(40.68270959341735 -73.84496782014399) bank91645 +91646 POINT(40.12458801101174 -73.36489352150068) bank91646 +91647 POINT(39.85946900053192 -74.51625724573057) bank91647 +91648 POINT(41.55524407277565 -74.18664660961501) bank91648 +91649 POINT(40.52950390015199 -73.4052290801777) bank91649 +91650 POINT(40.74959740335275 -74.06725590565644) bank91650 +91651 POINT(40.76063324525566 -74.11119291697737) bank91651 +91652 POINT(41.700386184687744 -73.45544338794593) bank91652 +91653 POINT(41.66472715582097 -73.26811723198153) bank91653 +91654 POINT(39.736207089172375 -74.72798944750814) bank91654 +91655 POINT(41.143655541628064 -73.35192877769146) bank91655 +91656 POINT(41.545363332506014 -73.04989242008665) bank91656 +91657 POINT(41.10746392700443 -74.7545054152505) bank91657 +91658 POINT(41.5217015721084 -73.23737953687431) bank91658 +91659 POINT(40.45089356407375 -74.3447740506801) bank91659 +91660 POINT(41.125030380247196 -73.5048841996175) bank91660 +91661 POINT(41.66814332088363 -74.30451390282555) bank91661 +91662 POINT(40.80739406483346 -73.79834834738124) bank91662 +91663 POINT(41.171396799701945 -74.3568810865549) bank91663 +91664 POINT(41.375391857647806 -73.3829481535632) bank91664 +91665 POINT(39.991069214899056 -74.03492333694744) bank91665 +91666 POINT(40.529516348737026 -74.90284151336766) bank91666 +91667 POINT(40.04480562053364 -73.68347026305155) bank91667 +91668 POINT(40.07646388472075 -74.65236827731385) bank91668 +91669 POINT(41.21635798871479 -74.83077533946147) bank91669 +91670 POINT(40.40898557567618 -74.23655339560375) bank91670 +91671 POINT(40.154939295971566 -74.22251762316783) bank91671 +91672 POINT(40.17565684738774 -73.7926351407182) bank91672 +91673 POINT(40.08178751809497 -73.69166036455911) bank91673 +91674 POINT(41.44757574924252 -73.36262683513723) bank91674 +91675 POINT(40.93147044173989 -73.365028451792) bank91675 +91676 POINT(39.78876805545258 -73.22932950242513) bank91676 +91677 POINT(40.71313609335123 -73.1448741863062) bank91677 +91678 POINT(41.37602163592049 -73.01806406731069) bank91678 +91679 POINT(41.46919077819622 -74.11427160838534) bank91679 +91680 POINT(41.56827840767569 -73.14791386795301) bank91680 +91681 POINT(40.078816350271296 -74.36794681823645) bank91681 +91682 POINT(40.60370931802726 -74.8648041439114) bank91682 +91683 POINT(39.9693233073468 -73.96786293573368) bank91683 +91684 POINT(40.35183409231035 -74.31167766358091) bank91684 +91685 POINT(41.07395969939019 -73.63103797334877) bank91685 +91686 POINT(41.27681324398151 -74.91762673032365) bank91686 +91687 POINT(40.648614770166226 -73.39993784483212) bank91687 +91688 POINT(39.74022480163661 -73.83844922688444) bank91688 +91689 POINT(40.7697814323755 -73.10576459187413) bank91689 +91690 POINT(40.29564281831441 -74.55172044042) bank91690 +91691 POINT(40.93761325609234 -73.84941075538043) bank91691 +91692 POINT(40.04046306941228 -74.04484957144538) bank91692 +91693 POINT(41.677847450455516 -73.56062288164263) bank91693 +91694 POINT(41.279996313430786 -73.42029114795686) bank91694 +91695 POINT(40.55841454098333 -73.06256698753246) bank91695 +91696 POINT(39.96509945439611 -74.89787037244282) bank91696 +91697 POINT(40.33012529759265 -73.41613491566798) bank91697 +91698 POINT(41.275449624432014 -73.37780200390804) bank91698 +91699 POINT(40.44938285705525 -73.29352851272019) bank91699 +91700 POINT(40.16610780786164 -73.67599531778586) bank91700 +91701 POINT(40.41917217398085 -73.14481790756818) bank91701 +91702 POINT(40.09964044396964 -73.436629248712) bank91702 +91703 POINT(40.01223182629348 -74.48648876447344) bank91703 +91704 POINT(40.937559964079476 -73.17293070158735) bank91704 +91705 POINT(40.527716484423955 -74.64816375150096) bank91705 +91706 POINT(41.002747552884706 -74.21613166447231) bank91706 +91707 POINT(41.60002663328838 -74.77232730754777) bank91707 +91708 POINT(39.91484857717546 -73.25419148955986) bank91708 +91709 POINT(41.28187116456368 -74.57892992914749) bank91709 +91710 POINT(40.3298565471299 -73.10359701026091) bank91710 +91711 POINT(39.90819510711423 -73.90662132390298) bank91711 +91712 POINT(40.09117840816799 -74.08235740827257) bank91712 +91713 POINT(40.26762608157974 -74.04361138334997) bank91713 +91714 POINT(41.69519605123797 -73.73226530653716) bank91714 +91715 POINT(41.60849993013094 -74.76452387522646) bank91715 +91716 POINT(41.117941312297326 -73.29779155722727) bank91716 +91717 POINT(41.15454818325823 -74.73448160745475) bank91717 +91718 POINT(40.44734343144836 -74.00612702684175) bank91718 +91719 POINT(40.4090241952353 -74.13297091115027) bank91719 +91720 POINT(40.48769392696164 -73.12010409435445) bank91720 +91721 POINT(40.66394060661509 -74.01265638700892) bank91721 +91722 POINT(41.52125674104436 -73.47043997566246) bank91722 +91723 POINT(40.56568848171738 -73.40926637932) bank91723 +91724 POINT(41.134458468727225 -74.07276497186962) bank91724 +91725 POINT(40.550028409882245 -73.38566713560229) bank91725 +91726 POINT(40.06530689846826 -74.74490311115231) bank91726 +91727 POINT(40.39423046371421 -73.06673919766587) bank91727 +91728 POINT(41.222440755810105 -74.49728424012036) bank91728 +91729 POINT(40.081446370052845 -73.4839646993242) bank91729 +91730 POINT(40.05838705286382 -74.88449975152153) bank91730 +91731 POINT(41.56747261139958 -74.74255874122598) bank91731 +91732 POINT(41.589767237450964 -74.0969522404257) bank91732 +91733 POINT(40.74480107588795 -74.44062542858467) bank91733 +91734 POINT(40.21872650473074 -73.63572976734478) bank91734 +91735 POINT(40.71172008100103 -75.0033030387293) bank91735 +91736 POINT(41.167250910701135 -73.07224971980243) bank91736 +91737 POINT(39.84873842141225 -74.03655324692977) bank91737 +91738 POINT(40.31389424352038 -73.2964270741913) bank91738 +91739 POINT(40.95675973751323 -73.98200961924431) bank91739 +91740 POINT(40.40131359066813 -74.70595999296872) bank91740 +91741 POINT(39.73250791774756 -73.12317365215448) bank91741 +91742 POINT(40.62437812472044 -73.69329179406145) bank91742 +91743 POINT(40.21168341185426 -73.74135045886507) bank91743 +91744 POINT(40.0209042389872 -73.3593242654881) bank91744 +91745 POINT(39.81421902725209 -74.96639529024488) bank91745 +91746 POINT(40.45487419518082 -73.39621253905625) bank91746 +91747 POINT(40.070609358019766 -73.63974938276296) bank91747 +91748 POINT(40.69741368796606 -74.92962775443699) bank91748 +91749 POINT(40.65497455663562 -73.8478515607137) bank91749 +91750 POINT(41.583019677443595 -74.68977240070456) bank91750 +91751 POINT(41.5270571488184 -73.11344299498683) bank91751 +91752 POINT(40.735705857726586 -74.39567413258813) bank91752 +91753 POINT(40.07771022600406 -73.38374816251473) bank91753 +91754 POINT(41.69216922236759 -73.64494591293924) bank91754 +91755 POINT(41.29389026380777 -74.6463287676349) bank91755 +91756 POINT(40.01511675322569 -73.30921155392146) bank91756 +91757 POINT(41.664055652342576 -73.15029259333757) bank91757 +91758 POINT(40.35051514765205 -74.06616210551178) bank91758 +91759 POINT(40.62074068911827 -73.40140206111529) bank91759 +91760 POINT(40.889924617526226 -74.59697622152545) bank91760 +91761 POINT(41.31279350774771 -73.60825533720046) bank91761 +91762 POINT(40.557580791495816 -73.78073699840485) bank91762 +91763 POINT(40.30086825241825 -73.23340085943848) bank91763 +91764 POINT(40.41309077016345 -74.63080424551944) bank91764 +91765 POINT(41.069383368001674 -74.6070706014766) bank91765 +91766 POINT(40.53073165689834 -73.44182797939564) bank91766 +91767 POINT(40.74935157626396 -74.4887969781273) bank91767 +91768 POINT(40.29624184689421 -73.33267414149049) bank91768 +91769 POINT(41.505388332678706 -73.7095502228403) bank91769 +91770 POINT(40.40312177223813 -74.38575585175538) bank91770 +91771 POINT(41.301785912897216 -74.46091076592656) bank91771 +91772 POINT(39.953184242357175 -74.04022899152922) bank91772 +91773 POINT(40.62041171480598 -73.47075657590008) bank91773 +91774 POINT(40.41091496563481 -73.36947177009517) bank91774 +91775 POINT(39.82821542239679 -73.73604401150746) bank91775 +91776 POINT(41.42148946597376 -74.81360127903572) bank91776 +91777 POINT(39.89172362582489 -73.79005518096763) bank91777 +91778 POINT(41.481796771648206 -74.19922558020153) bank91778 +91779 POINT(40.98596458100584 -73.31818142355242) bank91779 +91780 POINT(40.63265036198828 -74.13389039170654) bank91780 +91781 POINT(41.12435730004792 -73.06155111806335) bank91781 +91782 POINT(40.278308892826765 -73.62594783868464) bank91782 +91783 POINT(39.956492679461526 -74.77178679757749) bank91783 +91784 POINT(41.648191382655156 -74.29640331922135) bank91784 +91785 POINT(40.87075816164632 -74.12168911482996) bank91785 +91786 POINT(41.18676633451364 -73.26346437016275) bank91786 +91787 POINT(40.89553358068134 -74.95621317054379) bank91787 +91788 POINT(41.67753881460846 -74.81941704936675) bank91788 +91789 POINT(40.393906232290874 -73.08871486222438) bank91789 +91790 POINT(39.8506636048499 -74.73990205963784) bank91790 +91791 POINT(40.330837360831595 -73.79515657089718) bank91791 +91792 POINT(41.44579056697643 -73.4041652138074) bank91792 +91793 POINT(40.527043011559066 -74.92485583798066) bank91793 +91794 POINT(40.065299302056886 -73.78708813002419) bank91794 +91795 POINT(41.11195961541986 -73.2606024639901) bank91795 +91796 POINT(41.34956795411849 -73.8894254003203) bank91796 +91797 POINT(39.91923675077036 -73.25994497477464) bank91797 +91798 POINT(40.061029006401924 -74.93940171510205) bank91798 +91799 POINT(41.174606489297034 -73.8571021968219) bank91799 +91800 POINT(41.20130069106438 -74.7089660560483) bank91800 +91801 POINT(41.0165416409531 -74.86458591912294) bank91801 +91802 POINT(40.0520171412781 -73.07762764225602) bank91802 +91803 POINT(39.7276193795748 -74.02444625598339) bank91803 +91804 POINT(41.582652885938195 -74.2632327331991) bank91804 +91805 POINT(41.69022810683092 -74.54474554663605) bank91805 +91806 POINT(41.324801382266166 -74.87520464006276) bank91806 +91807 POINT(41.23406399766526 -74.7743741389949) bank91807 +91808 POINT(40.14002762394766 -74.60975763641234) bank91808 +91809 POINT(39.940757181393195 -73.77710623619984) bank91809 +91810 POINT(40.29131628063313 -74.24620279533951) bank91810 +91811 POINT(40.99777420683445 -74.32392016060669) bank91811 +91812 POINT(40.66056309437629 -73.84684838144587) bank91812 +91813 POINT(40.46645465162636 -73.46955653216669) bank91813 +91814 POINT(40.19985564584504 -74.83389920278186) bank91814 +91815 POINT(40.280733971051184 -73.85378152146247) bank91815 +91816 POINT(40.92804283105491 -73.63414506113303) bank91816 +91817 POINT(40.98518561073608 -73.72827576330853) bank91817 +91818 POINT(41.4298986566269 -73.72022821423737) bank91818 +91819 POINT(41.64965848662534 -74.08627849561137) bank91819 +91820 POINT(40.87587214788947 -73.17017888434043) bank91820 +91821 POINT(39.98507813981781 -74.38906565587645) bank91821 +91822 POINT(41.00417013795993 -74.98436861285143) bank91822 +91823 POINT(40.09767714599571 -74.05793011319903) bank91823 +91824 POINT(39.918313007638844 -74.71969199980099) bank91824 +91825 POINT(40.891520415203054 -74.14727242210462) bank91825 +91826 POINT(40.23548967207468 -73.33379118648483) bank91826 +91827 POINT(39.85874696959374 -74.22082227395668) bank91827 +91828 POINT(40.249243516096925 -74.26323732033423) bank91828 +91829 POINT(40.910119184177816 -73.5040109391374) bank91829 +91830 POINT(39.7734633356241 -73.80189890837754) bank91830 +91831 POINT(40.21547732749326 -73.84177794075237) bank91831 +91832 POINT(40.7675395944552 -74.33198062626772) bank91832 +91833 POINT(39.862301250994136 -74.09003688549296) bank91833 +91834 POINT(40.670397737260735 -73.82549608956903) bank91834 +91835 POINT(40.494055379421944 -73.01597860715229) bank91835 +91836 POINT(40.200643789717724 -73.88684290092898) bank91836 +91837 POINT(40.45184188754468 -73.7140118405549) bank91837 +91838 POINT(41.17783369475588 -73.5896482787452) bank91838 +91839 POINT(40.51572337549828 -73.31477313079542) bank91839 +91840 POINT(41.32817001049652 -73.00839116526218) bank91840 +91841 POINT(41.02367232254222 -74.6832298704013) bank91841 +91842 POINT(41.301594086287686 -73.92920280280012) bank91842 +91843 POINT(40.964007522827586 -73.14945167249691) bank91843 +91844 POINT(40.56968676070126 -73.88873200102583) bank91844 +91845 POINT(40.77394053503845 -74.13579769112904) bank91845 +91846 POINT(41.030725565553205 -73.37840719644264) bank91846 +91847 POINT(40.474189292541425 -73.49255615091803) bank91847 +91848 POINT(41.60354750434695 -74.73559355850114) bank91848 +91849 POINT(39.933601377103486 -73.75496984041553) bank91849 +91850 POINT(40.39257518769632 -74.19757595739388) bank91850 +91851 POINT(40.72966945441246 -74.16205923900252) bank91851 +91852 POINT(40.97735605456855 -74.88809754915891) bank91852 +91853 POINT(41.52441069622892 -73.89078267867629) bank91853 +91854 POINT(40.68688679401928 -74.31639347628288) bank91854 +91855 POINT(39.94371839472987 -74.00225718038374) bank91855 +91856 POINT(40.94653188595734 -74.31480190954065) bank91856 +91857 POINT(41.56155636919217 -73.21418335237301) bank91857 +91858 POINT(40.265345031546374 -74.29728020348341) bank91858 +91859 POINT(40.57662838711536 -74.47229410529887) bank91859 +91860 POINT(40.89668173625429 -74.05981080358308) bank91860 +91861 POINT(40.58032910845514 -73.3825455374227) bank91861 +91862 POINT(41.41601350173194 -74.26736093663051) bank91862 +91863 POINT(41.22523683946627 -74.08142570437556) bank91863 +91864 POINT(41.49720921655316 -73.04966317777517) bank91864 +91865 POINT(41.65175301921385 -74.54843527371717) bank91865 +91866 POINT(40.36549545417223 -73.19860611606194) bank91866 +91867 POINT(40.162492453603065 -74.62428535568577) bank91867 +91868 POINT(40.80788806542282 -73.5379843470855) bank91868 +91869 POINT(39.74693275371147 -74.7797087180016) bank91869 +91870 POINT(41.62067349755771 -73.8408303811759) bank91870 +91871 POINT(41.19175138149071 -73.88527366537437) bank91871 +91872 POINT(40.38286426643252 -74.87450874931578) bank91872 +91873 POINT(39.8844327791519 -74.86279758546807) bank91873 +91874 POINT(40.81492941953071 -73.062285562922) bank91874 +91875 POINT(40.88360864789479 -74.3400290561909) bank91875 +91876 POINT(41.64193795325816 -74.20666642817386) bank91876 +91877 POINT(40.49643792140698 -73.94262422316139) bank91877 +91878 POINT(41.532701282645824 -73.69480409333609) bank91878 +91879 POINT(39.911749623407246 -73.9850803797721) bank91879 +91880 POINT(41.51839212712641 -73.16710960031635) bank91880 +91881 POINT(41.245585221551835 -74.84746413669617) bank91881 +91882 POINT(39.737257070345656 -73.47476430562364) bank91882 +91883 POINT(40.16540914742744 -73.0495949410634) bank91883 +91884 POINT(40.11209735935495 -74.94029359004632) bank91884 +91885 POINT(41.40542818084699 -74.2744389823227) bank91885 +91886 POINT(41.30784787113483 -73.14134339769286) bank91886 +91887 POINT(40.065504099830726 -74.75253648646891) bank91887 +91888 POINT(39.71977238006049 -74.03444160649846) bank91888 +91889 POINT(40.95450458515137 -74.74339182644559) bank91889 +91890 POINT(40.04590933386541 -73.28219163377366) bank91890 +91891 POINT(40.565615142717306 -73.94533679808892) bank91891 +91892 POINT(41.25784952768987 -74.32284490356894) bank91892 +91893 POINT(41.65542903785021 -73.69582338512718) bank91893 +91894 POINT(41.505081948687526 -73.85095858589943) bank91894 +91895 POINT(40.55725155636775 -74.25921839988656) bank91895 +91896 POINT(40.01413863720503 -73.16254068487383) bank91896 +91897 POINT(40.876043887506654 -73.77186598607533) bank91897 +91898 POINT(40.252489746515955 -73.96578069997611) bank91898 +91899 POINT(40.82147915087141 -74.94064960811853) bank91899 +91900 POINT(40.63725081071496 -73.7218207032888) bank91900 +91901 POINT(39.828468672935074 -73.81543154408541) bank91901 +91902 POINT(40.28179629850703 -74.56125989500302) bank91902 +91903 POINT(39.92467247926106 -73.24400821170349) bank91903 +91904 POINT(41.6719636140119 -74.46291109978137) bank91904 +91905 POINT(41.31268112194778 -74.92575501414221) bank91905 +91906 POINT(41.243511278472084 -74.11367148680156) bank91906 +91907 POINT(41.087125193985194 -74.33628080666291) bank91907 +91908 POINT(41.124309789407235 -73.98758543566866) bank91908 +91909 POINT(39.716453240680686 -73.63927813512713) bank91909 +91910 POINT(40.04562330623959 -74.60136861867093) bank91910 +91911 POINT(41.08410903717176 -74.807180917925) bank91911 +91912 POINT(41.40916995874521 -74.07566816787468) bank91912 +91913 POINT(41.21195274038973 -74.29890322155696) bank91913 +91914 POINT(40.22287379937077 -73.39283303180576) bank91914 +91915 POINT(39.85119071680584 -74.73510851251675) bank91915 +91916 POINT(39.816463862351064 -74.50501957806301) bank91916 +91917 POINT(39.959928214644385 -74.03221339715061) bank91917 +91918 POINT(40.80647234965295 -73.39007290923831) bank91918 +91919 POINT(39.836062707651465 -74.90331077910116) bank91919 +91920 POINT(39.97271039273761 -73.17044168709826) bank91920 +91921 POINT(40.74762070258515 -73.2140764982378) bank91921 +91922 POINT(41.39714555495394 -73.31947403925571) bank91922 +91923 POINT(40.643814853619006 -73.08586980494691) bank91923 +91924 POINT(40.94186143299666 -74.5005649381681) bank91924 +91925 POINT(39.72421032187736 -74.4207015297166) bank91925 +91926 POINT(40.379832248874884 -73.97266848744356) bank91926 +91927 POINT(40.622606198322664 -74.33032143138634) bank91927 +91928 POINT(40.839264616412684 -73.71592034770792) bank91928 +91929 POINT(39.92931256234237 -73.50896015901628) bank91929 +91930 POINT(40.98843219112451 -74.03369943143157) bank91930 +91931 POINT(41.44278491392345 -73.05031014965232) bank91931 +91932 POINT(40.16147485330989 -74.88628694660147) bank91932 +91933 POINT(41.507308791582204 -74.2174487804026) bank91933 +91934 POINT(41.65445698111669 -74.62748385296491) bank91934 +91935 POINT(39.90154168531359 -73.0425196241297) bank91935 +91936 POINT(40.83406786381901 -74.79724069716977) bank91936 +91937 POINT(40.825764281301566 -74.26857904971777) bank91937 +91938 POINT(41.09471535063747 -73.42090131144761) bank91938 +91939 POINT(41.60436709872523 -74.58883160777785) bank91939 +91940 POINT(40.89842096133184 -73.2677362117958) bank91940 +91941 POINT(40.780480174978486 -73.42867719217206) bank91941 +91942 POINT(41.03154144190922 -73.82906106721424) bank91942 +91943 POINT(40.191934536098906 -74.822877396927) bank91943 +91944 POINT(41.654336534636585 -74.6370015110745) bank91944 +91945 POINT(39.85486074744995 -74.48631130164542) bank91945 +91946 POINT(40.59336966918094 -73.97790404333028) bank91946 +91947 POINT(41.13560275940159 -73.53914664851796) bank91947 +91948 POINT(40.1782879213269 -74.80620068399011) bank91948 +91949 POINT(39.83766786245271 -73.4194168025009) bank91949 +91950 POINT(41.44496108700256 -73.73632310369304) bank91950 +91951 POINT(41.65782592272605 -74.15184291149917) bank91951 +91952 POINT(40.12577685343788 -74.2942946177933) bank91952 +91953 POINT(40.41138287731928 -74.97228100237783) bank91953 +91954 POINT(40.690062700783045 -73.62309925363355) bank91954 +91955 POINT(41.58499587803811 -73.83379256670871) bank91955 +91956 POINT(41.61733054845624 -73.4176759677172) bank91956 +91957 POINT(40.194390054746144 -73.22525136793763) bank91957 +91958 POINT(40.37175552819572 -74.29015585819305) bank91958 +91959 POINT(40.968677858229505 -74.59504490651078) bank91959 +91960 POINT(39.91363441972739 -74.35147918183127) bank91960 +91961 POINT(41.26040083924682 -73.363010140264) bank91961 +91962 POINT(40.27721038508911 -73.9855364997764) bank91962 +91963 POINT(40.90256734045501 -74.6011985606597) bank91963 +91964 POINT(40.612794676769404 -73.64428401691909) bank91964 +91965 POINT(39.885643015051514 -74.76747198796889) bank91965 +91966 POINT(41.55572463396015 -74.07536941461129) bank91966 +91967 POINT(41.44791537334945 -73.83111113220916) bank91967 +91968 POINT(41.55416397367639 -74.72438529741625) bank91968 +91969 POINT(40.344792568127815 -74.86042920834105) bank91969 +91970 POINT(40.9130107685225 -74.74938341815533) bank91970 +91971 POINT(39.71352479599041 -74.23904092879813) bank91971 +91972 POINT(41.4529844888454 -74.10960650091074) bank91972 +91973 POINT(40.0567903877371 -73.21002929700988) bank91973 +91974 POINT(40.74905054828489 -73.60324829569305) bank91974 +91975 POINT(41.10335017567375 -73.47713724885196) bank91975 +91976 POINT(41.296106946343244 -73.31468234309936) bank91976 +91977 POINT(41.47551138461557 -73.2483594236163) bank91977 +91978 POINT(40.3435613115331 -74.21584072875902) bank91978 +91979 POINT(41.383583077317276 -73.94548309595719) bank91979 +91980 POINT(41.48076194136026 -74.97379773208462) bank91980 +91981 POINT(41.52317308979428 -73.62431787021464) bank91981 +91982 POINT(40.65276579604528 -74.10254523940797) bank91982 +91983 POINT(39.87868789033974 -73.61309890416457) bank91983 +91984 POINT(41.15192243181503 -73.5471769045235) bank91984 +91985 POINT(40.205201431780885 -73.28561423611993) bank91985 +91986 POINT(41.28838772390554 -73.44143481490579) bank91986 +91987 POINT(40.22239651033428 -74.80619717259931) bank91987 +91988 POINT(40.0976411765417 -73.76355859433043) bank91988 +91989 POINT(41.48355693410831 -74.34478884546361) bank91989 +91990 POINT(39.82574305485149 -73.30188261597173) bank91990 +91991 POINT(40.47703236867972 -73.30459104206686) bank91991 +91992 POINT(41.23143525273774 -73.23557703669573) bank91992 +91993 POINT(41.3218336266345 -73.77866738046791) bank91993 +91994 POINT(40.734181685177724 -73.40646087610891) bank91994 +91995 POINT(40.97193729293643 -73.21558810445198) bank91995 +91996 POINT(39.914369067018264 -73.22828375946305) bank91996 +91997 POINT(40.368376949454884 -73.32656445329263) bank91997 +91998 POINT(40.80987742850714 -74.03932226882415) bank91998 +91999 POINT(40.30378932364962 -74.44937140934485) bank91999 +92000 POINT(40.778697215300966 -73.64032218507617) bank92000 +92001 POINT(40.96906363059799 -74.21079270062724) bank92001 +92002 POINT(40.502385287306716 -73.74613991061523) bank92002 +92003 POINT(41.038301825583915 -74.493208010946) bank92003 +92004 POINT(40.94883547353561 -73.65466440826422) bank92004 +92005 POINT(40.81437781696868 -74.35365022652097) bank92005 +92006 POINT(40.75571313862375 -74.31684507012784) bank92006 +92007 POINT(39.76570748780174 -73.6157871139526) bank92007 +92008 POINT(40.574397942216926 -73.1760322188392) bank92008 +92009 POINT(41.280093768594966 -74.1214670578485) bank92009 +92010 POINT(39.8763112296648 -74.25699790714461) bank92010 +92011 POINT(41.454957149784384 -73.63080600071066) bank92011 +92012 POINT(41.61764151030892 -73.09097943167075) bank92012 +92013 POINT(41.20109249923871 -73.86118225952558) bank92013 +92014 POINT(41.330022046635094 -74.80115815542975) bank92014 +92015 POINT(41.59221174422916 -73.81504966566395) bank92015 +92016 POINT(39.90959047936742 -73.96130597748402) bank92016 +92017 POINT(40.5400135204797 -74.53953402598978) bank92017 +92018 POINT(40.0257270627368 -74.39662599345992) bank92018 +92019 POINT(41.26443348961248 -74.0886693924838) bank92019 +92020 POINT(40.24836823874196 -74.84497087927157) bank92020 +92021 POINT(40.91652603974498 -74.28733803868928) bank92021 +92022 POINT(40.50403558673537 -74.2701736857073) bank92022 +92023 POINT(39.73633984352316 -74.51529651733598) bank92023 +92024 POINT(40.26993953588533 -74.7695156245045) bank92024 +92025 POINT(40.27658472567707 -73.16439442129052) bank92025 +92026 POINT(40.28070831365766 -74.48735041376446) bank92026 +92027 POINT(41.476387752299786 -73.07056501537102) bank92027 +92028 POINT(41.507766837630854 -74.62102701277962) bank92028 +92029 POINT(40.09610146650586 -73.2684336105199) bank92029 +92030 POINT(40.64985967778404 -73.4475701724991) bank92030 +92031 POINT(40.95103945334357 -73.50888952905656) bank92031 +92032 POINT(40.38245756837036 -74.91259481036168) bank92032 +92033 POINT(40.74708314938915 -73.08125200750504) bank92033 +92034 POINT(39.79252462930127 -73.25926383826874) bank92034 +92035 POINT(41.17080937069467 -73.45806437708764) bank92035 +92036 POINT(40.87190496270127 -74.91119058770863) bank92036 +92037 POINT(41.18625041488513 -74.62167045727087) bank92037 +92038 POINT(41.17813347909949 -73.77115194509753) bank92038 +92039 POINT(40.83064722251122 -74.46587132199714) bank92039 +92040 POINT(41.275927727214864 -74.36684231856854) bank92040 +92041 POINT(39.94950809690795 -73.8244361292519) bank92041 +92042 POINT(39.86160272401724 -73.13169699592517) bank92042 +92043 POINT(40.29914071933787 -73.18747309281775) bank92043 +92044 POINT(40.96624240620885 -73.71377562271809) bank92044 +92045 POINT(39.918652824077874 -73.24573723098064) bank92045 +92046 POINT(40.64029807497143 -73.0161921880824) bank92046 +92047 POINT(40.10936035491237 -73.3630038061821) bank92047 +92048 POINT(41.54937204146638 -73.6234466962877) bank92048 +92049 POINT(40.33597158549976 -74.50340900470964) bank92049 +92050 POINT(40.80455857536015 -73.26013932761536) bank92050 +92051 POINT(41.363043507446406 -73.42317378221043) bank92051 +92052 POINT(40.43418290456914 -73.78598706284434) bank92052 +92053 POINT(41.034179089233106 -73.97596867528779) bank92053 +92054 POINT(40.861765767351805 -73.39286750698838) bank92054 +92055 POINT(41.144468990667356 -74.70869511443696) bank92055 +92056 POINT(41.51280103791004 -74.12010461753697) bank92056 +92057 POINT(40.34747185981157 -74.87262473691115) bank92057 +92058 POINT(41.06401184688913 -74.27139692448182) bank92058 +92059 POINT(39.7556429651319 -74.56019381370903) bank92059 +92060 POINT(41.45047869310861 -74.09674991821467) bank92060 +92061 POINT(39.729268871783106 -74.82364134913517) bank92061 +92062 POINT(41.147948506039214 -73.19371697197623) bank92062 +92063 POINT(39.82744157639707 -74.03078191364027) bank92063 +92064 POINT(39.80149568172871 -74.39356884968137) bank92064 +92065 POINT(40.79242946823435 -73.03766540964709) bank92065 +92066 POINT(41.170167345734825 -73.88016708892019) bank92066 +92067 POINT(40.34981745537686 -73.16205653958134) bank92067 +92068 POINT(41.633841320045164 -74.8770784030872) bank92068 +92069 POINT(40.2593071380511 -73.30915836360819) bank92069 +92070 POINT(41.62625574499193 -74.99575735971267) bank92070 +92071 POINT(41.018528523933554 -74.8694369027779) bank92071 +92072 POINT(40.010840188815365 -73.16329895245335) bank92072 +92073 POINT(40.81879807494976 -74.34022061969087) bank92073 +92074 POINT(40.882034702430744 -74.01539616065033) bank92074 +92075 POINT(40.258511699328736 -74.20783951468624) bank92075 +92076 POINT(40.82480999001916 -73.78157575547435) bank92076 +92077 POINT(40.094920861156695 -74.5927851097865) bank92077 +92078 POINT(39.80676152634754 -74.42352388443118) bank92078 +92079 POINT(40.949129182348486 -73.575671920957) bank92079 +92080 POINT(40.23212112055774 -73.05059553487605) bank92080 +92081 POINT(41.150661935284624 -74.75816797677535) bank92081 +92082 POINT(40.89680015665273 -74.393029387594) bank92082 +92083 POINT(40.776863454916075 -73.25322479679579) bank92083 +92084 POINT(40.17139564236334 -73.13228654547382) bank92084 +92085 POINT(39.78500780749003 -74.24896389452111) bank92085 +92086 POINT(40.701301346880214 -74.07155688428615) bank92086 +92087 POINT(40.64928609132156 -74.8222402847045) bank92087 +92088 POINT(41.172430362275946 -73.5087189515067) bank92088 +92089 POINT(39.99068251854395 -73.60850050846749) bank92089 +92090 POINT(41.04124708459973 -73.8559808581219) bank92090 +92091 POINT(39.778523800133115 -74.2537265158949) bank92091 +92092 POINT(40.08436457627387 -74.5520740975296) bank92092 +92093 POINT(39.95856728766381 -74.08296990250572) bank92093 +92094 POINT(40.95192617815916 -74.24154368199932) bank92094 +92095 POINT(41.50270326571644 -74.5603700581317) bank92095 +92096 POINT(39.857765795161754 -73.0584356332206) bank92096 +92097 POINT(40.18654527563237 -73.04115522989495) bank92097 +92098 POINT(40.34865973275658 -74.72131037035582) bank92098 +92099 POINT(41.59198944674169 -73.95609090303599) bank92099 +92100 POINT(41.14831681392273 -74.9768812717608) bank92100 +92101 POINT(41.66244330679481 -74.43024687899629) bank92101 +92102 POINT(40.792705003221656 -73.16249390383832) bank92102 +92103 POINT(40.018836145046095 -73.16749342769914) bank92103 +92104 POINT(40.10433659850217 -74.64704952246862) bank92104 +92105 POINT(40.98974614775351 -73.37447326032627) bank92105 +92106 POINT(41.06843380078887 -73.80438078701982) bank92106 +92107 POINT(41.68792624133122 -73.05256048848311) bank92107 +92108 POINT(40.89063610675037 -73.29325933063689) bank92108 +92109 POINT(41.507068446197735 -74.65382736202021) bank92109 +92110 POINT(40.49022899873936 -73.09041115567018) bank92110 +92111 POINT(40.598140251553524 -74.31799862068821) bank92111 +92112 POINT(40.008312135982855 -73.959982459523) bank92112 +92113 POINT(39.86761428760338 -73.0191287680837) bank92113 +92114 POINT(39.937077350190656 -73.33664959616436) bank92114 +92115 POINT(40.52672492415296 -73.29513589909685) bank92115 +92116 POINT(41.367353497309374 -73.9126298723742) bank92116 +92117 POINT(40.30475465314071 -74.31571228795684) bank92117 +92118 POINT(39.97567132833684 -73.05855247230964) bank92118 +92119 POINT(40.535832090376374 -73.6723769516506) bank92119 +92120 POINT(39.86192295321724 -73.09254025883854) bank92120 +92121 POINT(41.621801614368884 -74.00334903222358) bank92121 +92122 POINT(40.141177196031265 -73.16185770949149) bank92122 +92123 POINT(40.34940987607037 -73.16409406036212) bank92123 +92124 POINT(41.30946125612497 -74.85012167624977) bank92124 +92125 POINT(40.23920399362658 -74.40461134406756) bank92125 +92126 POINT(41.53137245878882 -73.87819144626124) bank92126 +92127 POINT(39.882918563431105 -73.07350701612529) bank92127 +92128 POINT(41.471226281855 -74.49508667060452) bank92128 +92129 POINT(40.851146675909646 -73.61022772889172) bank92129 +92130 POINT(39.95677901153314 -73.36207401386817) bank92130 +92131 POINT(41.63331278682041 -73.93061668354993) bank92131 +92132 POINT(40.76934385637724 -74.13431743611032) bank92132 +92133 POINT(41.08733533122076 -74.30833924244442) bank92133 +92134 POINT(40.9987653534434 -73.66125493656367) bank92134 +92135 POINT(40.44350529949374 -74.28272616270114) bank92135 +92136 POINT(39.95690866044259 -73.76015372575762) bank92136 +92137 POINT(41.36241310660524 -73.41764771194386) bank92137 +92138 POINT(40.77314965292986 -73.40117563475712) bank92138 +92139 POINT(40.01737729023054 -73.30703078730144) bank92139 +92140 POINT(40.367369103751585 -73.14162698610355) bank92140 +92141 POINT(40.474461871066005 -73.77681958582016) bank92141 +92142 POINT(39.80917169302106 -74.0929930667521) bank92142 +92143 POINT(40.869905651391065 -73.60659763232914) bank92143 +92144 POINT(40.664004001878645 -74.07592532833726) bank92144 +92145 POINT(40.10558907819784 -73.79921181016188) bank92145 +92146 POINT(39.78362468014233 -74.9498063627288) bank92146 +92147 POINT(40.138637630716396 -73.81445154180624) bank92147 +92148 POINT(40.9087434524253 -73.24054469609254) bank92148 +92149 POINT(40.47323090390358 -73.6062240200025) bank92149 +92150 POINT(41.355595162478096 -74.88156571939274) bank92150 +92151 POINT(40.65268608875936 -74.14607896637337) bank92151 +92152 POINT(41.59209285272929 -73.43359799456779) bank92152 +92153 POINT(40.54985339962004 -74.30123348113673) bank92153 +92154 POINT(41.57887425581809 -73.4416034906488) bank92154 +92155 POINT(40.03889935318571 -74.02546473829602) bank92155 +92156 POINT(40.349141634797874 -74.86837449705021) bank92156 +92157 POINT(40.32193719486237 -73.79332526315032) bank92157 +92158 POINT(40.29239208669758 -73.16024943357945) bank92158 +92159 POINT(39.946422095263266 -73.01977989837123) bank92159 +92160 POINT(39.92997395379871 -73.1800840060534) bank92160 +92161 POINT(41.533137529916694 -74.11910473388377) bank92161 +92162 POINT(41.70478372855559 -74.98858226759886) bank92162 +92163 POINT(40.796509901845724 -74.03423780230271) bank92163 +92164 POINT(41.27596660380763 -74.27962880298355) bank92164 +92165 POINT(40.055175862567935 -73.690179901477) bank92165 +92166 POINT(39.82912085790534 -73.27644504556159) bank92166 +92167 POINT(40.398981178064616 -74.28286037422353) bank92167 +92168 POINT(40.09481190188954 -73.3529640851179) bank92168 +92169 POINT(41.579993410075275 -73.59851235116484) bank92169 +92170 POINT(41.324859217591595 -74.15654707461837) bank92170 +92171 POINT(41.47938131133891 -74.74009318628622) bank92171 +92172 POINT(41.39845193673962 -73.50283098410083) bank92172 +92173 POINT(41.418772496653936 -73.08734028263274) bank92173 +92174 POINT(41.16561244573637 -74.15690732640914) bank92174 +92175 POINT(41.61421048119997 -73.18712579624169) bank92175 +92176 POINT(41.369490262347426 -73.58375868546744) bank92176 +92177 POINT(40.944255348093925 -73.11041965473007) bank92177 +92178 POINT(39.90985344816392 -73.28712851200518) bank92178 +92179 POINT(41.496526351772744 -74.86553348395523) bank92179 +92180 POINT(40.77341228468804 -73.63505396447873) bank92180 +92181 POINT(40.814555327365184 -74.07093187053214) bank92181 +92182 POINT(40.291500276623026 -74.73513528375861) bank92182 +92183 POINT(40.964139959077144 -74.48420807736504) bank92183 +92184 POINT(39.73555260592273 -74.57974748937437) bank92184 +92185 POINT(41.45418779675153 -73.52074004522359) bank92185 +92186 POINT(40.91497167409481 -74.669242107687) bank92186 +92187 POINT(40.80950341849781 -74.05459311151436) bank92187 +92188 POINT(41.68442965725238 -73.12468984095501) bank92188 +92189 POINT(40.74616506010167 -75.00234371247292) bank92189 +92190 POINT(40.14701549177491 -74.22623408159174) bank92190 +92191 POINT(41.30742093552661 -74.68016978908042) bank92191 +92192 POINT(40.39587937796996 -73.95732357861422) bank92192 +92193 POINT(39.801122665483554 -74.81352988748655) bank92193 +92194 POINT(41.341525991308416 -73.81393702677211) bank92194 +92195 POINT(41.584963484423646 -73.76086576480202) bank92195 +92196 POINT(41.34281066208528 -74.893068231357) bank92196 +92197 POINT(41.37472972960331 -73.56099880197651) bank92197 +92198 POINT(40.39824739636755 -74.32872853388113) bank92198 +92199 POINT(41.654652659989395 -73.39065528307104) bank92199 +92200 POINT(40.18049266763237 -73.22515791920412) bank92200 +92201 POINT(41.51705468888405 -73.88758706671263) bank92201 +92202 POINT(40.37666330623685 -73.13068531929646) bank92202 +92203 POINT(41.212881521300396 -73.71509067077639) bank92203 +92204 POINT(40.00339793158352 -73.44274978367812) bank92204 +92205 POINT(41.61003944040205 -73.27457487003178) bank92205 +92206 POINT(41.10069643436199 -74.08032761941118) bank92206 +92207 POINT(40.97203518302225 -73.71108734869178) bank92207 +92208 POINT(40.873249368914614 -73.52146218073267) bank92208 +92209 POINT(40.830210953030466 -74.69605264790017) bank92209 +92210 POINT(40.74880265235497 -73.8018268554175) bank92210 +92211 POINT(39.916297303083326 -73.6768226979584) bank92211 +92212 POINT(41.129095174973145 -74.23443266640582) bank92212 +92213 POINT(40.230675417102745 -73.94940846405224) bank92213 +92214 POINT(41.09961231785844 -74.92772392905817) bank92214 +92215 POINT(40.82905812381994 -74.82350156211034) bank92215 +92216 POINT(40.28586199454822 -74.41827520446944) bank92216 +92217 POINT(41.29142621679753 -73.75439153704853) bank92217 +92218 POINT(41.58835745635715 -73.07489999018173) bank92218 +92219 POINT(40.71356467298848 -74.71278456954296) bank92219 +92220 POINT(39.85719707748785 -73.2421121558987) bank92220 +92221 POINT(40.9312128176304 -74.65818150205874) bank92221 +92222 POINT(40.390124315752786 -74.22008988457551) bank92222 +92223 POINT(41.50607925581439 -73.39782990386348) bank92223 +92224 POINT(41.265041183920985 -73.8969339876297) bank92224 +92225 POINT(40.8729722713299 -74.57153313232547) bank92225 +92226 POINT(41.57486588806771 -73.01574534074967) bank92226 +92227 POINT(40.715676081645185 -73.13418743316623) bank92227 +92228 POINT(40.24742759863893 -73.36659066167071) bank92228 +92229 POINT(39.91889709363786 -74.08594821473106) bank92229 +92230 POINT(39.90802435635749 -74.26087559595143) bank92230 +92231 POINT(39.987676136135825 -73.17201318956184) bank92231 +92232 POINT(40.07382261683345 -73.02569445036211) bank92232 +92233 POINT(39.969183160249074 -74.64043623746213) bank92233 +92234 POINT(40.07584649693409 -74.3117675526892) bank92234 +92235 POINT(40.51077478938248 -73.0706657077284) bank92235 +92236 POINT(39.908331153714435 -74.69875545371121) bank92236 +92237 POINT(40.93449617239468 -73.75938868379326) bank92237 +92238 POINT(40.26732297083751 -73.15121838843369) bank92238 +92239 POINT(40.61638629953559 -74.20166415418473) bank92239 +92240 POINT(41.11716613846826 -74.66034859052692) bank92240 +92241 POINT(39.82076604746692 -74.81011402264963) bank92241 +92242 POINT(40.457925085929574 -74.94341964124048) bank92242 +92243 POINT(39.859127003454596 -73.10495849883687) bank92243 +92244 POINT(41.07754695375268 -74.53458254363355) bank92244 +92245 POINT(40.0289175079595 -74.69414334940186) bank92245 +92246 POINT(41.04619635369081 -74.03630327402867) bank92246 +92247 POINT(39.87707561692603 -73.11275323312032) bank92247 +92248 POINT(41.19543966112196 -73.82553967672995) bank92248 +92249 POINT(41.17364406100837 -74.96122066285793) bank92249 +92250 POINT(41.29250552841586 -74.70353001812536) bank92250 +92251 POINT(41.323740330469846 -73.0767512966859) bank92251 +92252 POINT(41.35308655174084 -74.49924750096528) bank92252 +92253 POINT(41.38314866720482 -74.25541067095169) bank92253 +92254 POINT(40.936806538994524 -73.42929404113951) bank92254 +92255 POINT(40.73364208843388 -73.09508792852668) bank92255 +92256 POINT(41.26490943991804 -74.75159314089854) bank92256 +92257 POINT(41.55871142125765 -75.00279625203446) bank92257 +92258 POINT(40.50515301584945 -73.35102237492046) bank92258 +92259 POINT(41.53942701722923 -73.2703223180294) bank92259 +92260 POINT(40.06543879691295 -74.51602763200768) bank92260 +92261 POINT(40.650033300620336 -73.64671820385891) bank92261 +92262 POINT(40.27346680139321 -73.92324967698768) bank92262 +92263 POINT(40.50962145570553 -73.0538106807491) bank92263 +92264 POINT(39.90760646232886 -74.29011268565073) bank92264 +92265 POINT(41.34271055293014 -74.34425273314976) bank92265 +92266 POINT(39.85075147050531 -73.65739285511194) bank92266 +92267 POINT(40.208534674172135 -73.10564025825651) bank92267 +92268 POINT(40.93437730746917 -74.93240917161033) bank92268 +92269 POINT(39.95098398450733 -73.43338704066566) bank92269 +92270 POINT(40.31176420837152 -74.72810504240246) bank92270 +92271 POINT(40.71373486382527 -74.54427183796453) bank92271 +92272 POINT(40.47224958210576 -74.62431884602996) bank92272 +92273 POINT(40.039593428556884 -73.09325577112351) bank92273 +92274 POINT(41.68724122006978 -74.51235537973366) bank92274 +92275 POINT(40.391166969060635 -73.03970335274445) bank92275 +92276 POINT(41.341131965243164 -74.50012456365467) bank92276 +92277 POINT(41.079274124352395 -74.4031383076578) bank92277 +92278 POINT(41.19940742466004 -73.12575786199045) bank92278 +92279 POINT(39.86493974638627 -73.90063543448697) bank92279 +92280 POINT(40.1677870997679 -73.27649417052915) bank92280 +92281 POINT(39.94831454784688 -74.34537914734986) bank92281 +92282 POINT(40.7865707449037 -74.73195989293201) bank92282 +92283 POINT(39.759584836677945 -74.41853492473429) bank92283 +92284 POINT(40.571437718622754 -74.797180098505) bank92284 +92285 POINT(40.282279198240815 -73.76515172936317) bank92285 +92286 POINT(40.37668487901539 -73.73257917649553) bank92286 +92287 POINT(41.13052514265258 -73.85245275725573) bank92287 +92288 POINT(40.98772230187286 -74.31223707977857) bank92288 +92289 POINT(40.611951112439314 -74.91173978108858) bank92289 +92290 POINT(41.62270133880373 -74.45542086247829) bank92290 +92291 POINT(40.878324604871736 -74.08167090088521) bank92291 +92292 POINT(40.0440473376169 -73.19189145210328) bank92292 +92293 POINT(40.82985362977965 -74.33440239792114) bank92293 +92294 POINT(40.893040727852636 -73.67469487566906) bank92294 +92295 POINT(41.506220846332155 -73.50665617218377) bank92295 +92296 POINT(40.110056295555 -74.76043197860369) bank92296 +92297 POINT(40.092024221062985 -73.57200689397429) bank92297 +92298 POINT(41.65579759554027 -73.4553150122083) bank92298 +92299 POINT(40.536838181070415 -73.29122447723613) bank92299 +92300 POINT(41.13601296776888 -73.09056062361117) bank92300 +92301 POINT(41.140105931110845 -73.35820688623078) bank92301 +92302 POINT(40.853621606173895 -74.97383693949061) bank92302 +92303 POINT(41.03452579925449 -73.13640678886415) bank92303 +92304 POINT(41.203461442351426 -74.26327063263781) bank92304 +92305 POINT(40.024784877421745 -74.13595541408021) bank92305 +92306 POINT(40.5491147231247 -73.64805877758339) bank92306 +92307 POINT(40.789731283590775 -73.65922827815147) bank92307 +92308 POINT(40.101989636058406 -74.00320378810447) bank92308 +92309 POINT(41.663313808670765 -73.38416182745772) bank92309 +92310 POINT(41.23603617950783 -74.85487598757695) bank92310 +92311 POINT(40.364626501895444 -73.71514115663597) bank92311 +92312 POINT(41.04867053482048 -73.23249606752302) bank92312 +92313 POINT(40.17074435568115 -73.24359542429953) bank92313 +92314 POINT(40.77342209915332 -73.9105864680168) bank92314 +92315 POINT(40.93564358300733 -74.02816711102761) bank92315 +92316 POINT(39.89139071689484 -74.81114116284726) bank92316 +92317 POINT(40.46001514110128 -73.38461022170833) bank92317 +92318 POINT(40.07273047209043 -74.95438125384644) bank92318 +92319 POINT(40.16236094074784 -73.51545292223298) bank92319 +92320 POINT(41.68931844379773 -74.33963597117523) bank92320 +92321 POINT(40.473581525051486 -74.91655927516794) bank92321 +92322 POINT(41.52508892703815 -73.36737296068894) bank92322 +92323 POINT(41.217489281095666 -74.9388323214588) bank92323 +92324 POINT(40.43697409801862 -73.69404005013921) bank92324 +92325 POINT(41.19457729569341 -74.1981126816622) bank92325 +92326 POINT(41.032899111268534 -73.48380676620927) bank92326 +92327 POINT(41.05801328563063 -74.1321318812544) bank92327 +92328 POINT(40.13063323365401 -73.63504045350815) bank92328 +92329 POINT(41.43817886474608 -74.35604759292848) bank92329 +92330 POINT(41.70766912521943 -74.91736184702184) bank92330 +92331 POINT(41.3161684374852 -74.92563755063773) bank92331 +92332 POINT(40.501001675109755 -73.54654430759342) bank92332 +92333 POINT(40.85266916416517 -73.28475653046563) bank92333 +92334 POINT(40.57079426147544 -74.47452819802547) bank92334 +92335 POINT(39.784046156631256 -74.72416953540753) bank92335 +92336 POINT(39.723144569620146 -73.70598196534215) bank92336 +92337 POINT(41.13750426928108 -73.98335435134122) bank92337 +92338 POINT(41.019631108035675 -74.95957748900568) bank92338 +92339 POINT(39.829784967282436 -74.88012772573956) bank92339 +92340 POINT(40.16953893564311 -74.66054824176067) bank92340 +92341 POINT(41.14559997866335 -73.87829384671974) bank92341 +92342 POINT(40.60134044003482 -74.86750112927575) bank92342 +92343 POINT(41.173957599913095 -73.72637857274027) bank92343 +92344 POINT(39.9022626236582 -73.20496638780307) bank92344 +92345 POINT(41.42828687682828 -74.39199693824439) bank92345 +92346 POINT(40.368463988814526 -73.95028025142975) bank92346 +92347 POINT(41.52123695194887 -74.12122824361568) bank92347 +92348 POINT(40.28371504182194 -74.15401531157434) bank92348 +92349 POINT(40.33388202686096 -73.82520380846923) bank92349 +92350 POINT(40.55768413427308 -73.39161537719) bank92350 +92351 POINT(41.36154579555743 -73.95914641686329) bank92351 +92352 POINT(41.67215329536371 -73.08995194385376) bank92352 +92353 POINT(40.412369152892516 -73.61038880153127) bank92353 +92354 POINT(41.6641228407534 -73.4318202671114) bank92354 +92355 POINT(41.07847454815506 -74.54050979449786) bank92355 +92356 POINT(39.880274025426715 -74.02056307130772) bank92356 +92357 POINT(41.68021010608007 -73.4652491178124) bank92357 +92358 POINT(41.54330126166685 -73.80051111654173) bank92358 +92359 POINT(39.893495403497845 -73.77198002389098) bank92359 +92360 POINT(41.260265530318996 -73.81701734203867) bank92360 +92361 POINT(40.55287697045531 -74.20646273823955) bank92361 +92362 POINT(41.21916486346163 -74.03470581980234) bank92362 +92363 POINT(41.000571306717895 -74.5552803486255) bank92363 +92364 POINT(41.619611849951134 -74.54659351019241) bank92364 +92365 POINT(40.33302916740492 -74.59903987974432) bank92365 +92366 POINT(41.170141332129376 -73.22110771584283) bank92366 +92367 POINT(40.29180385057664 -74.47164119944499) bank92367 +92368 POINT(40.315789146456396 -74.1362139420767) bank92368 +92369 POINT(39.720495418574345 -74.74038996797505) bank92369 +92370 POINT(41.474399656413944 -73.84114008430228) bank92370 +92371 POINT(40.990671793608314 -74.3650971952724) bank92371 +92372 POINT(41.210801502714254 -73.21338734097857) bank92372 +92373 POINT(41.05967916062528 -74.50835868538306) bank92373 +92374 POINT(41.56469537647358 -74.23950621466454) bank92374 +92375 POINT(39.77936267741518 -73.57491503883546) bank92375 +92376 POINT(41.711565493440034 -73.33262128101934) bank92376 +92377 POINT(39.92159914723693 -73.74609415140941) bank92377 +92378 POINT(40.822374747369025 -73.52494299934277) bank92378 +92379 POINT(41.10659435499988 -73.06531910233792) bank92379 +92380 POINT(41.702966837467606 -74.9156191793902) bank92380 +92381 POINT(41.27869398497512 -73.64849030572871) bank92381 +92382 POINT(41.50475859292469 -73.9437399750604) bank92382 +92383 POINT(41.400079511053214 -74.78549168310617) bank92383 +92384 POINT(40.117315786735425 -73.01750728277314) bank92384 +92385 POINT(39.73732528810714 -74.44868161321504) bank92385 +92386 POINT(40.67739941279677 -74.53832106549314) bank92386 +92387 POINT(39.71581109276564 -73.9166090167495) bank92387 +92388 POINT(41.45635973036799 -73.23404248199208) bank92388 +92389 POINT(40.68797817823296 -74.1142183554498) bank92389 +92390 POINT(40.271703237701445 -74.67757323508448) bank92390 +92391 POINT(41.03077925588405 -74.47515181752095) bank92391 +92392 POINT(41.23493006892809 -73.90286365015152) bank92392 +92393 POINT(40.80806659086157 -73.3089950963385) bank92393 +92394 POINT(41.02841640421326 -73.62332676806854) bank92394 +92395 POINT(40.78728319787528 -73.10384500081645) bank92395 +92396 POINT(40.35976793312693 -73.91212158868345) bank92396 +92397 POINT(41.076150925092925 -73.74521561891727) bank92397 +92398 POINT(41.29812960925257 -74.89313589682546) bank92398 +92399 POINT(41.39068030224681 -74.96008720088655) bank92399 +92400 POINT(40.52625255039411 -73.62219161961826) bank92400 +92401 POINT(40.70899465107181 -73.19161474175365) bank92401 +92402 POINT(40.2513760609293 -74.31552128671163) bank92402 +92403 POINT(40.80881117763141 -74.37968745124013) bank92403 +92404 POINT(40.48736365192891 -74.84556576485075) bank92404 +92405 POINT(41.710601908947865 -74.0988293216472) bank92405 +92406 POINT(40.749812102191 -73.78516929473732) bank92406 +92407 POINT(40.32014194119999 -73.74557938836921) bank92407 +92408 POINT(41.38598289878909 -74.67351722256197) bank92408 +92409 POINT(41.09862328488501 -73.71517386073904) bank92409 +92410 POINT(40.46141007887771 -73.29981246899526) bank92410 +92411 POINT(41.670718062621106 -73.45079456928269) bank92411 +92412 POINT(41.475243566890796 -73.52646680224893) bank92412 +92413 POINT(40.84398869185888 -74.22550849516784) bank92413 +92414 POINT(40.70618996662193 -74.65661067540832) bank92414 +92415 POINT(40.161280124754974 -74.09218475014731) bank92415 +92416 POINT(39.89094634855984 -73.04058514594412) bank92416 +92417 POINT(40.87858403291157 -74.20318649603601) bank92417 +92418 POINT(40.23551105746924 -73.28976601424858) bank92418 +92419 POINT(40.73222338565138 -74.00695002022962) bank92419 +92420 POINT(40.83439830659398 -74.18432429181543) bank92420 +92421 POINT(40.42646227470964 -73.87697444827452) bank92421 +92422 POINT(41.07186697185071 -74.7158312639625) bank92422 +92423 POINT(40.163905543083985 -74.34762714610224) bank92423 +92424 POINT(40.17387312760054 -74.3358358241665) bank92424 +92425 POINT(40.92665875023093 -74.53442774373804) bank92425 +92426 POINT(40.36056956393209 -73.50864854887375) bank92426 +92427 POINT(40.33219781731876 -74.58496878888606) bank92427 +92428 POINT(40.771759700657114 -73.27345749869113) bank92428 +92429 POINT(41.12841139376838 -73.65359766367627) bank92429 +92430 POINT(41.519950549292005 -73.43017730401054) bank92430 +92431 POINT(40.14554092624319 -73.95296709310901) bank92431 +92432 POINT(40.61910005571153 -74.25075271653071) bank92432 +92433 POINT(39.74267298257715 -74.60959567788893) bank92433 +92434 POINT(39.94649118543375 -73.06938645970644) bank92434 +92435 POINT(40.413360276301376 -74.54694033773815) bank92435 +92436 POINT(41.1470483328143 -74.95342520070176) bank92436 +92437 POINT(39.817062998751474 -74.22379460524095) bank92437 +92438 POINT(40.84291398974305 -73.5805177616158) bank92438 +92439 POINT(40.93657205597131 -73.54299984500688) bank92439 +92440 POINT(40.8186918267636 -73.74273320265331) bank92440 +92441 POINT(41.32346852604023 -73.30538469942994) bank92441 +92442 POINT(40.31796934874332 -74.303390507896) bank92442 +92443 POINT(41.63144210304879 -74.12584996300349) bank92443 +92444 POINT(41.0452054314776 -73.42156997062989) bank92444 +92445 POINT(40.14633759073727 -74.63637256531777) bank92445 +92446 POINT(40.46894657236116 -74.85531398800202) bank92446 +92447 POINT(40.25871380225688 -73.9521428100057) bank92447 +92448 POINT(39.717100041538096 -74.81505304915245) bank92448 +92449 POINT(40.97212216488557 -73.05546480896818) bank92449 +92450 POINT(41.57961622135884 -74.67815623118634) bank92450 +92451 POINT(41.016561074524674 -73.64509593852839) bank92451 +92452 POINT(40.93634510816088 -74.09889289245102) bank92452 +92453 POINT(41.630488185581456 -74.58204862438097) bank92453 +92454 POINT(41.051926539229854 -74.52288980384903) bank92454 +92455 POINT(40.42643655829533 -74.73046821151374) bank92455 +92456 POINT(40.269795643591436 -73.22354268323141) bank92456 +92457 POINT(40.14386227011998 -73.8236564442131) bank92457 +92458 POINT(40.23634231455343 -74.05476688206554) bank92458 +92459 POINT(39.94864034247034 -74.42435260673463) bank92459 +92460 POINT(40.92793597148231 -74.32221349988271) bank92460 +92461 POINT(40.18958717361636 -74.9202548023224) bank92461 +92462 POINT(41.54305955803551 -74.79280223194586) bank92462 +92463 POINT(40.97904274332247 -74.51214755224422) bank92463 +92464 POINT(40.870686855275295 -73.0359709437496) bank92464 +92465 POINT(40.0686598080023 -73.95390686115591) bank92465 +92466 POINT(39.847579585143684 -74.94296397314187) bank92466 +92467 POINT(41.30396316770986 -73.29470986043172) bank92467 +92468 POINT(41.375240992818895 -73.2822166222291) bank92468 +92469 POINT(41.63099700276887 -74.26687706480601) bank92469 +92470 POINT(40.79369577598616 -73.89418011464163) bank92470 +92471 POINT(40.73929013441913 -74.9059877588176) bank92471 +92472 POINT(40.74443415063122 -74.74161278246746) bank92472 +92473 POINT(39.81762191749076 -74.60751965120727) bank92473 +92474 POINT(40.574864315324035 -73.4208244924946) bank92474 +92475 POINT(40.07364201889625 -74.70037316346777) bank92475 +92476 POINT(41.50898210143003 -73.54024009626782) bank92476 +92477 POINT(41.45742684514682 -74.45334551171065) bank92477 +92478 POINT(40.729255900073625 -73.04850371616628) bank92478 +92479 POINT(41.4297141197571 -74.21360699082206) bank92479 +92480 POINT(40.99456048416745 -74.39525916329922) bank92480 +92481 POINT(41.39600072984801 -73.34893545663179) bank92481 +92482 POINT(41.42494965472382 -73.14994015548358) bank92482 +92483 POINT(40.20746283431967 -73.01659407078371) bank92483 +92484 POINT(40.70122495316257 -73.00856281001977) bank92484 +92485 POINT(41.0150479585308 -74.97093508856832) bank92485 +92486 POINT(39.907990011343976 -74.55234979727528) bank92486 +92487 POINT(41.01349546804465 -73.28134638737741) bank92487 +92488 POINT(41.17363171669737 -73.09588031105707) bank92488 +92489 POINT(40.8952747225086 -73.5009237007018) bank92489 +92490 POINT(41.32807685768845 -74.57253037173282) bank92490 +92491 POINT(40.23338615811924 -74.94390610438174) bank92491 +92492 POINT(40.14613950760162 -73.78176275536038) bank92492 +92493 POINT(40.788635970968286 -73.62281388263642) bank92493 +92494 POINT(40.7893336952128 -74.63128399706116) bank92494 +92495 POINT(39.79191756443086 -73.86293178475233) bank92495 +92496 POINT(41.52917410878843 -74.15536558571986) bank92496 +92497 POINT(39.927709279984775 -73.09876871982598) bank92497 +92498 POINT(41.03395333736945 -73.77864779807496) bank92498 +92499 POINT(39.75881474003894 -73.55176693035476) bank92499 +92500 POINT(40.99969399438017 -73.7403494697525) bank92500 +92501 POINT(40.926356733807324 -73.7108417071154) bank92501 +92502 POINT(40.22937016338182 -74.88078376505268) bank92502 +92503 POINT(40.44453663902886 -73.68984460856865) bank92503 +92504 POINT(41.259226906898206 -73.63243875219032) bank92504 +92505 POINT(40.94823096737291 -74.22313922545251) bank92505 +92506 POINT(40.41659646261791 -74.77505555542616) bank92506 +92507 POINT(41.33980845706221 -73.32862271975908) bank92507 +92508 POINT(40.094275376291066 -74.38443200851901) bank92508 +92509 POINT(41.418510604460614 -74.22153827540346) bank92509 +92510 POINT(41.631525880195476 -74.06491091752574) bank92510 +92511 POINT(40.47267874583212 -74.43961027172354) bank92511 +92512 POINT(40.044288575424524 -74.07940843628626) bank92512 +92513 POINT(41.16178224733362 -74.4774584748914) bank92513 +92514 POINT(40.30205794044281 -74.73080788938456) bank92514 +92515 POINT(41.01518016964245 -74.99688227945677) bank92515 +92516 POINT(41.60493819573632 -73.87892741322894) bank92516 +92517 POINT(41.606377234993765 -73.75685567736328) bank92517 +92518 POINT(40.80700569506427 -74.27526594769458) bank92518 +92519 POINT(40.936287070826296 -73.55362536351154) bank92519 +92520 POINT(40.817222609114026 -74.43321706228309) bank92520 +92521 POINT(40.61039395197663 -73.22057056303085) bank92521 +92522 POINT(40.66830018072764 -73.62763784706836) bank92522 +92523 POINT(40.892688216325034 -73.26839535927544) bank92523 +92524 POINT(41.50802537044355 -73.31077520125993) bank92524 +92525 POINT(40.73621995961498 -74.83483693337415) bank92525 +92526 POINT(40.533517343864034 -74.01619266125469) bank92526 +92527 POINT(40.28585761410216 -73.47683906822506) bank92527 +92528 POINT(40.86629794166437 -74.4089691104927) bank92528 +92529 POINT(39.93605387888032 -73.65509856093682) bank92529 +92530 POINT(41.062850314345965 -74.72196229659343) bank92530 +92531 POINT(41.252784290868775 -74.34272989813175) bank92531 +92532 POINT(41.6918678154652 -74.71126911177483) bank92532 +92533 POINT(39.748935531104564 -73.01676878721042) bank92533 +92534 POINT(40.427196305519686 -73.09598370120096) bank92534 +92535 POINT(39.84599066564908 -74.10249934811502) bank92535 +92536 POINT(40.6941492321674 -73.68771320936428) bank92536 +92537 POINT(40.41342702360697 -74.76790232781595) bank92537 +92538 POINT(40.45630888801317 -74.59861909430856) bank92538 +92539 POINT(40.957250907723534 -73.76320688070517) bank92539 +92540 POINT(40.57934019545503 -74.62028325159861) bank92540 +92541 POINT(39.73412589870284 -73.07198193972394) bank92541 +92542 POINT(41.116780454590995 -73.37174411550498) bank92542 +92543 POINT(41.693523281327046 -73.40652524575768) bank92543 +92544 POINT(40.588472329140615 -73.80853814396568) bank92544 +92545 POINT(39.78413049044952 -74.49561937042027) bank92545 +92546 POINT(39.92563287976329 -74.76316762968135) bank92546 +92547 POINT(40.07012104107226 -73.01044830848438) bank92547 +92548 POINT(40.99183001889671 -73.68627698871374) bank92548 +92549 POINT(41.17324541220717 -73.4384551419938) bank92549 +92550 POINT(40.902105322310064 -73.69121144896282) bank92550 +92551 POINT(41.67038666725186 -74.70744832507654) bank92551 +92552 POINT(40.04632268003019 -73.65231168693873) bank92552 +92553 POINT(41.40534916988834 -74.38268964842129) bank92553 +92554 POINT(40.789111881268006 -73.6997986317117) bank92554 +92555 POINT(40.21252156135308 -74.47906176022737) bank92555 +92556 POINT(41.38999911541029 -73.28553411241224) bank92556 +92557 POINT(40.27116408731308 -73.52559315508879) bank92557 +92558 POINT(41.46419632520376 -74.1316176977626) bank92558 +92559 POINT(40.2840594831611 -73.4258348453888) bank92559 +92560 POINT(41.207750823035944 -73.18334190744878) bank92560 +92561 POINT(40.74834881521809 -73.10255155043458) bank92561 +92562 POINT(41.67402326554051 -73.96610657936208) bank92562 +92563 POINT(40.731532708483876 -73.95487779645242) bank92563 +92564 POINT(41.236245073009115 -73.12096467518161) bank92564 +92565 POINT(41.064487012177736 -73.8466311267552) bank92565 +92566 POINT(40.88835383403239 -73.19876682977379) bank92566 +92567 POINT(41.19639080929944 -73.76964051184629) bank92567 +92568 POINT(41.03642321457192 -74.45647470552717) bank92568 +92569 POINT(40.290689506722536 -74.49475060317866) bank92569 +92570 POINT(40.6331914419063 -73.29567884333851) bank92570 +92571 POINT(41.34246148091579 -73.8752149059484) bank92571 +92572 POINT(41.155142802932986 -73.10587972066035) bank92572 +92573 POINT(39.72320683692608 -73.2969655517204) bank92573 +92574 POINT(41.24708162106913 -73.0217494547246) bank92574 +92575 POINT(40.702122851810906 -73.96826002999563) bank92575 +92576 POINT(41.326108360441516 -73.04792756699757) bank92576 +92577 POINT(39.81907999596774 -74.81898583680498) bank92577 +92578 POINT(40.488575296147246 -73.94254773446481) bank92578 +92579 POINT(41.36705288180996 -73.1869030958805) bank92579 +92580 POINT(39.83773252751316 -74.36564560513467) bank92580 +92581 POINT(40.15950052524921 -73.38131552520403) bank92581 +92582 POINT(40.28478691327504 -73.81015102700513) bank92582 +92583 POINT(39.734921760319594 -73.66125289110018) bank92583 +92584 POINT(40.63247027225916 -73.13749145802245) bank92584 +92585 POINT(40.141816569228524 -74.90610857531907) bank92585 +92586 POINT(39.89943241993041 -74.30734911680159) bank92586 +92587 POINT(41.24168182082962 -73.76008466407609) bank92587 +92588 POINT(40.68066758564631 -74.81971627731538) bank92588 +92589 POINT(40.3446324618212 -74.90886541377937) bank92589 +92590 POINT(40.15160719280117 -73.64649800390949) bank92590 +92591 POINT(41.34853408131404 -73.52751868098423) bank92591 +92592 POINT(41.62550324955293 -74.08528624094986) bank92592 +92593 POINT(39.75332585164948 -74.59144147319789) bank92593 +92594 POINT(41.56199024395933 -73.00878462064429) bank92594 +92595 POINT(40.89724650900619 -74.64709159683444) bank92595 +92596 POINT(40.25383983972164 -74.06774759130653) bank92596 +92597 POINT(39.98016356207351 -73.7209629875292) bank92597 +92598 POINT(41.67727848611998 -73.8731635336843) bank92598 +92599 POINT(40.97620121841266 -74.00943259805221) bank92599 +92600 POINT(39.74635887410518 -74.87398529089933) bank92600 +92601 POINT(40.689840919625404 -74.58834148721259) bank92601 +92602 POINT(39.92869408742661 -73.30347352586877) bank92602 +92603 POINT(40.549911685707656 -73.5047820777875) bank92603 +92604 POINT(41.29858212033848 -73.9862679480607) bank92604 +92605 POINT(40.16397232534869 -73.74264612487238) bank92605 +92606 POINT(40.66995052927469 -73.9711024080278) bank92606 +92607 POINT(39.98405727890682 -73.30796149063016) bank92607 +92608 POINT(39.79538113624657 -74.74035065192916) bank92608 +92609 POINT(39.891782132461316 -74.5942183703743) bank92609 +92610 POINT(39.71444702639825 -74.26898739554426) bank92610 +92611 POINT(40.27587759046328 -74.39543107807098) bank92611 +92612 POINT(41.09352145979921 -73.34207951529316) bank92612 +92613 POINT(41.58629486123231 -73.5264704251167) bank92613 +92614 POINT(41.55414432542367 -73.66101341793593) bank92614 +92615 POINT(41.64733344433874 -74.17516555568908) bank92615 +92616 POINT(41.62011364016172 -73.97943563972997) bank92616 +92617 POINT(41.02483286748834 -74.6062124380114) bank92617 +92618 POINT(40.69052824184333 -74.40405499469038) bank92618 +92619 POINT(40.607662725830586 -73.93840903380631) bank92619 +92620 POINT(39.91136669045318 -74.70251879512088) bank92620 +92621 POINT(41.69667982387792 -73.4778136118382) bank92621 +92622 POINT(40.06350170468949 -73.50258324804706) bank92622 +92623 POINT(40.73196158050937 -74.45700249710407) bank92623 +92624 POINT(41.27504802693726 -73.1997263643728) bank92624 +92625 POINT(41.36597834455178 -75.00538966760762) bank92625 +92626 POINT(41.50056719739501 -73.1379629533525) bank92626 +92627 POINT(39.87171616137022 -73.13681537264323) bank92627 +92628 POINT(39.7225631653443 -73.50251081009071) bank92628 +92629 POINT(40.22182220589843 -73.0491181511997) bank92629 +92630 POINT(40.87918589945379 -74.54965285696716) bank92630 +92631 POINT(41.217751146275134 -73.11527449319719) bank92631 +92632 POINT(41.18671698649952 -73.08736876952894) bank92632 +92633 POINT(40.74171840092537 -74.23008916970657) bank92633 +92634 POINT(40.48266201680566 -73.5703459397379) bank92634 +92635 POINT(41.59700299445896 -74.07053741442971) bank92635 +92636 POINT(39.951372969821165 -74.54585504573706) bank92636 +92637 POINT(39.986561104877644 -74.85290723910592) bank92637 +92638 POINT(41.44887295029076 -73.29328180627381) bank92638 +92639 POINT(40.605884857632525 -74.29821527385818) bank92639 +92640 POINT(41.618838270543066 -73.2805725637115) bank92640 +92641 POINT(41.330045776208394 -74.74919222591978) bank92641 +92642 POINT(41.42863350710417 -73.15650734330764) bank92642 +92643 POINT(40.4694525853492 -73.54750129510074) bank92643 +92644 POINT(41.24056111510603 -73.7907547031236) bank92644 +92645 POINT(40.42253635146658 -73.49822171380349) bank92645 +92646 POINT(41.061876617668645 -73.88884984502022) bank92646 +92647 POINT(41.341276535967914 -74.42799932036044) bank92647 +92648 POINT(41.7117185706082 -74.55729924300812) bank92648 +92649 POINT(40.88708603239087 -74.27713091415964) bank92649 +92650 POINT(39.740517414605165 -73.14727004928396) bank92650 +92651 POINT(40.691949164568896 -73.83964270370961) bank92651 +92652 POINT(40.39994629895133 -74.98643438145999) bank92652 +92653 POINT(39.77587573361565 -74.05034382351894) bank92653 +92654 POINT(40.08008296955917 -74.25305388242249) bank92654 +92655 POINT(41.6375625335639 -73.46426919490142) bank92655 +92656 POINT(40.86220769400121 -73.94673224565273) bank92656 +92657 POINT(40.896865868888455 -74.74570039150052) bank92657 +92658 POINT(41.20510450810026 -73.03751635997968) bank92658 +92659 POINT(41.417744174214185 -73.54439580652881) bank92659 +92660 POINT(41.26142025368014 -73.25911346011723) bank92660 +92661 POINT(41.27290310000434 -73.14408300855432) bank92661 +92662 POINT(40.35032758587776 -73.48652340534905) bank92662 +92663 POINT(40.40067853727124 -74.76177534305582) bank92663 +92664 POINT(41.39665269484298 -74.610032753698) bank92664 +92665 POINT(39.76273293184395 -74.70535944726838) bank92665 +92666 POINT(40.89016940884632 -73.60158482464654) bank92666 +92667 POINT(39.88691021739337 -73.02314661876677) bank92667 +92668 POINT(40.095995097129716 -74.90027374176296) bank92668 +92669 POINT(39.80289310351115 -74.4933596405924) bank92669 +92670 POINT(40.907659549471006 -73.43435838397542) bank92670 +92671 POINT(40.31303060550935 -74.60647256037352) bank92671 +92672 POINT(41.07315204520321 -74.9914833994302) bank92672 +92673 POINT(39.79073359022615 -73.53861829504106) bank92673 +92674 POINT(40.786669700798846 -74.21056522665441) bank92674 +92675 POINT(39.824345477818646 -73.04523740250761) bank92675 +92676 POINT(39.98273265564766 -73.0393127184365) bank92676 +92677 POINT(39.87660322461694 -74.22023081159762) bank92677 +92678 POINT(40.622871768601044 -73.30171717035726) bank92678 +92679 POINT(40.20462094871449 -74.48748751491411) bank92679 +92680 POINT(40.38373739230362 -73.7704791484147) bank92680 +92681 POINT(39.937351902692704 -73.43108306333272) bank92681 +92682 POINT(40.2090561583423 -74.69363511690847) bank92682 +92683 POINT(41.65061855197095 -73.41697428071534) bank92683 +92684 POINT(41.55223439282648 -74.2090418666916) bank92684 +92685 POINT(40.24055952474443 -73.2317372713239) bank92685 +92686 POINT(41.011948293820524 -73.40314182475694) bank92686 +92687 POINT(40.03655868796658 -74.64758340238699) bank92687 +92688 POINT(41.63680122786751 -73.85685355083993) bank92688 +92689 POINT(39.87131984350768 -73.72881268417872) bank92689 +92690 POINT(41.46077307286209 -73.62718459808013) bank92690 +92691 POINT(41.58757570678575 -74.39937499350555) bank92691 +92692 POINT(41.12315511269282 -74.26067295800176) bank92692 +92693 POINT(41.64507755227767 -73.3067266938618) bank92693 +92694 POINT(40.92205272975807 -73.49253613919468) bank92694 +92695 POINT(41.53571619863806 -74.3836931506771) bank92695 +92696 POINT(41.4871497597996 -73.09316162524055) bank92696 +92697 POINT(40.45431180101443 -74.69625787698533) bank92697 +92698 POINT(41.60442345376923 -73.22842643770659) bank92698 +92699 POINT(40.33618963826928 -73.77595056315003) bank92699 +92700 POINT(41.583433508880056 -74.40961786364623) bank92700 +92701 POINT(41.05995600359764 -74.52216307391105) bank92701 +92702 POINT(41.60089181666752 -73.39797658582846) bank92702 +92703 POINT(41.08495380726103 -74.03889500871234) bank92703 +92704 POINT(40.339278792296014 -74.1639549729219) bank92704 +92705 POINT(41.49734555822526 -74.07510671488714) bank92705 +92706 POINT(40.44791339435422 -74.81768361830608) bank92706 +92707 POINT(40.53993714543573 -74.97544098830275) bank92707 +92708 POINT(39.979103842098354 -74.12772584463437) bank92708 +92709 POINT(41.554017331455235 -74.61910359938888) bank92709 +92710 POINT(41.67068925494753 -74.45748118632415) bank92710 +92711 POINT(40.07067047747319 -73.3866760462841) bank92711 +92712 POINT(40.81759598189602 -73.58103009320658) bank92712 +92713 POINT(40.92715573869675 -74.62310554504202) bank92713 +92714 POINT(40.791566645510805 -73.94680257524182) bank92714 +92715 POINT(40.17972740881071 -73.10832017286681) bank92715 +92716 POINT(40.711872465934064 -73.50162273699401) bank92716 +92717 POINT(40.25548838664959 -74.94606492363079) bank92717 +92718 POINT(40.013831197526585 -74.5250701090075) bank92718 +92719 POINT(39.99226893917558 -73.0806941853663) bank92719 +92720 POINT(39.95852225411847 -74.3987952894085) bank92720 +92721 POINT(39.89614244447764 -74.00371395057468) bank92721 +92722 POINT(41.09985275186536 -74.30741606703323) bank92722 +92723 POINT(39.93481411857477 -74.55191928071913) bank92723 +92724 POINT(41.14215234904648 -74.9332044743105) bank92724 +92725 POINT(40.28673255957726 -74.44810802314176) bank92725 +92726 POINT(41.38696081559246 -74.52112149198811) bank92726 +92727 POINT(40.912819845182476 -74.35358735250877) bank92727 +92728 POINT(40.83177674830309 -73.88279840724951) bank92728 +92729 POINT(41.50261074418005 -73.38070688467124) bank92729 +92730 POINT(39.76682172100091 -74.42170942132998) bank92730 +92731 POINT(41.54686719437191 -73.85670264145487) bank92731 +92732 POINT(39.853460391845445 -74.85543185629288) bank92732 +92733 POINT(40.995664932572964 -74.08002462562436) bank92733 +92734 POINT(40.939097890845574 -73.92820431293603) bank92734 +92735 POINT(40.98693856432983 -74.35339588211434) bank92735 +92736 POINT(40.72785327616755 -74.06843238477022) bank92736 +92737 POINT(40.66623082115132 -74.42559033339617) bank92737 +92738 POINT(41.21892532051744 -74.53890036278466) bank92738 +92739 POINT(39.98760964956622 -74.24137143562895) bank92739 +92740 POINT(40.76578335730381 -74.38113250368319) bank92740 +92741 POINT(40.531096199949054 -73.02264818248965) bank92741 +92742 POINT(41.244025704102675 -73.53895674920793) bank92742 +92743 POINT(40.304195054021285 -73.13489509847314) bank92743 +92744 POINT(41.61443501887274 -73.62004908368912) bank92744 +92745 POINT(40.50536238832851 -73.30279386413379) bank92745 +92746 POINT(39.928252649969664 -73.579242789702) bank92746 +92747 POINT(40.95754332385668 -73.36857400316254) bank92747 +92748 POINT(40.6576472229468 -74.95418407408448) bank92748 +92749 POINT(40.76565941607402 -73.41583435127411) bank92749 +92750 POINT(40.983771471676434 -74.59634974993237) bank92750 +92751 POINT(40.454815974306754 -74.94914109945825) bank92751 +92752 POINT(40.78707044879093 -74.61835717289256) bank92752 +92753 POINT(41.23947963340603 -75.00157690919988) bank92753 +92754 POINT(40.57884219443771 -73.86977144037152) bank92754 +92755 POINT(41.502040673919126 -73.26681436035693) bank92755 +92756 POINT(41.168689137379985 -74.91175788694743) bank92756 +92757 POINT(41.282545744997144 -74.93440307545104) bank92757 +92758 POINT(40.03133410238245 -73.99581812217146) bank92758 +92759 POINT(41.65466579315428 -73.78522346503189) bank92759 +92760 POINT(41.26081764831144 -73.56486997176415) bank92760 +92761 POINT(41.31070388546437 -73.27453396348474) bank92761 +92762 POINT(41.054585230537654 -74.22984548931284) bank92762 +92763 POINT(40.92172297632105 -74.2363010471748) bank92763 +92764 POINT(39.74641227466305 -74.37721946821462) bank92764 +92765 POINT(41.37263528200569 -73.75724039141022) bank92765 +92766 POINT(40.36499266126083 -74.86296238863105) bank92766 +92767 POINT(41.68201346728658 -74.99542363999811) bank92767 +92768 POINT(39.71548534358675 -73.20188448345951) bank92768 +92769 POINT(40.253656672620245 -73.96045960383928) bank92769 +92770 POINT(40.93249124773095 -74.63115763578246) bank92770 +92771 POINT(41.27844351782221 -74.21152653876496) bank92771 +92772 POINT(40.584547375610306 -74.82851203554463) bank92772 +92773 POINT(41.604431605944235 -73.61912046639256) bank92773 +92774 POINT(41.29438746166713 -74.51383993993709) bank92774 +92775 POINT(41.27489968197427 -74.4956026032209) bank92775 +92776 POINT(40.45091824582569 -74.70698733893258) bank92776 +92777 POINT(39.7909998364114 -73.895833807996) bank92777 +92778 POINT(40.08251842659057 -73.08211072463006) bank92778 +92779 POINT(41.57165293824862 -74.94675534640693) bank92779 +92780 POINT(41.506053901740245 -74.52717432662001) bank92780 +92781 POINT(40.93762347021923 -73.36402397496354) bank92781 +92782 POINT(41.52864538176017 -73.71316503879652) bank92782 +92783 POINT(40.42964976805748 -74.20490064564694) bank92783 +92784 POINT(40.039892143633736 -73.03755937599149) bank92784 +92785 POINT(40.94335407501817 -74.00007183110428) bank92785 +92786 POINT(40.953274516507875 -73.28405131637949) bank92786 +92787 POINT(40.97211634613973 -74.00783482154371) bank92787 +92788 POINT(41.22392229638256 -73.08726546287976) bank92788 +92789 POINT(40.985408942206455 -73.82005611719848) bank92789 +92790 POINT(40.12676628930775 -73.47097334655606) bank92790 +92791 POINT(41.668547080743764 -73.040462276466) bank92791 +92792 POINT(40.58624037780395 -73.94104809396043) bank92792 +92793 POINT(41.15325259747485 -74.04041811650717) bank92793 +92794 POINT(41.40495627131702 -74.90429961742498) bank92794 +92795 POINT(40.335975988274484 -73.36028347207194) bank92795 +92796 POINT(40.8150935598018 -74.81304214152289) bank92796 +92797 POINT(41.618804105228385 -74.73512288677458) bank92797 +92798 POINT(41.418193987979755 -73.03483262604485) bank92798 +92799 POINT(39.82066856320955 -73.55661768984106) bank92799 +92800 POINT(41.620448426246405 -73.90865502650762) bank92800 +92801 POINT(41.659408268631395 -73.24781688237071) bank92801 +92802 POINT(40.55020848224913 -74.23352182007656) bank92802 +92803 POINT(40.46763744640012 -74.9623282956823) bank92803 +92804 POINT(41.25473530924597 -73.91436124131252) bank92804 +92805 POINT(40.45262875481305 -74.6674752637188) bank92805 +92806 POINT(40.71887077027877 -73.5835154961389) bank92806 +92807 POINT(40.1207111029035 -74.7128938592654) bank92807 +92808 POINT(40.823915389572555 -74.3937349596711) bank92808 +92809 POINT(39.9698353814036 -74.64082224986507) bank92809 +92810 POINT(41.58208573040902 -74.2418119444744) bank92810 +92811 POINT(40.589294307818015 -73.77932755596305) bank92811 +92812 POINT(40.071489236111894 -73.23791269597791) bank92812 +92813 POINT(40.90752972724047 -73.39875054288704) bank92813 +92814 POINT(41.082545877632064 -74.14933800098315) bank92814 +92815 POINT(41.55054638113339 -73.74915187441454) bank92815 +92816 POINT(40.16011012106189 -73.4720327524323) bank92816 +92817 POINT(41.67748681916837 -74.00120522676879) bank92817 +92818 POINT(40.657189481915985 -74.83302766413105) bank92818 +92819 POINT(39.85385337426709 -73.81058722866761) bank92819 +92820 POINT(41.64467013029664 -73.67399495273531) bank92820 +92821 POINT(41.49890982317121 -73.77364906317926) bank92821 +92822 POINT(39.92583531431097 -74.7038604830367) bank92822 +92823 POINT(39.7685569136304 -73.82245595253934) bank92823 +92824 POINT(41.08448400810542 -74.4417095935528) bank92824 +92825 POINT(40.95606696089115 -74.40597415352207) bank92825 +92826 POINT(40.66537142490031 -74.42001914971726) bank92826 +92827 POINT(40.638039585612944 -73.36553118063992) bank92827 +92828 POINT(40.20051203754344 -73.73235347110678) bank92828 +92829 POINT(41.4766813965752 -74.38267245188845) bank92829 +92830 POINT(41.222582109752366 -73.14644370144279) bank92830 +92831 POINT(41.42901234734362 -74.41272815704255) bank92831 +92832 POINT(40.78550262182658 -73.89842406198477) bank92832 +92833 POINT(39.99252948563983 -74.12263107373764) bank92833 +92834 POINT(41.380596775070906 -73.19773986204667) bank92834 +92835 POINT(41.02906375869495 -73.92836828110094) bank92835 +92836 POINT(41.124537333217795 -74.30766239584588) bank92836 +92837 POINT(39.726890633228685 -73.40869216000587) bank92837 +92838 POINT(41.67211766960622 -74.9479897940307) bank92838 +92839 POINT(40.250357244902844 -73.96368725021111) bank92839 +92840 POINT(40.9565436653731 -73.28335362448576) bank92840 +92841 POINT(40.443608156961666 -74.52058561027906) bank92841 +92842 POINT(40.620229170149784 -73.13170901410379) bank92842 +92843 POINT(40.17524559822163 -73.03034254778174) bank92843 +92844 POINT(40.144728597969575 -73.45859913441804) bank92844 +92845 POINT(41.54926180886743 -74.08688785927536) bank92845 +92846 POINT(40.37369871051776 -74.2492066291197) bank92846 +92847 POINT(41.62251522030968 -73.61860323312872) bank92847 +92848 POINT(41.62469150393818 -74.92570727927105) bank92848 +92849 POINT(39.79070932426361 -73.80321025465796) bank92849 +92850 POINT(39.82098237145513 -73.24658128161785) bank92850 +92851 POINT(41.51315627230468 -73.41802539409842) bank92851 +92852 POINT(41.580753752748905 -73.04919035756086) bank92852 +92853 POINT(41.55243046484518 -73.78096842029613) bank92853 +92854 POINT(40.934374537518664 -73.98948837409573) bank92854 +92855 POINT(40.54481992852758 -73.47156353748137) bank92855 +92856 POINT(40.07622474237889 -73.12380739681808) bank92856 +92857 POINT(40.75952371670529 -74.0324753755148) bank92857 +92858 POINT(40.850664660507924 -73.47757291995879) bank92858 +92859 POINT(40.914578571766086 -73.9487105084406) bank92859 +92860 POINT(40.97394709706969 -74.84519755303917) bank92860 +92861 POINT(40.0174126947785 -73.06799998464851) bank92861 +92862 POINT(41.61283602910329 -73.86192562035696) bank92862 +92863 POINT(40.713127235175016 -73.43242110330134) bank92863 +92864 POINT(40.92971805971627 -74.5001034407326) bank92864 +92865 POINT(41.204542200578665 -74.77708131230871) bank92865 +92866 POINT(40.485611763144064 -74.51746298143082) bank92866 +92867 POINT(40.41129774799874 -74.86883932380785) bank92867 +92868 POINT(40.52819295300651 -74.86121873704163) bank92868 +92869 POINT(40.14227276789089 -73.52612975037195) bank92869 +92870 POINT(41.15461690611191 -74.25672152249601) bank92870 +92871 POINT(40.62549641125945 -74.26624299626705) bank92871 +92872 POINT(40.93259746130222 -74.8143703411907) bank92872 +92873 POINT(41.29437785828897 -74.52131654638909) bank92873 +92874 POINT(40.09601255139267 -74.60077892530879) bank92874 +92875 POINT(41.30178598526592 -74.87504476345168) bank92875 +92876 POINT(40.010719262164365 -73.76462912879683) bank92876 +92877 POINT(41.12510305939309 -74.72723580576682) bank92877 +92878 POINT(41.5719783940774 -73.17436650538221) bank92878 +92879 POINT(41.19571786940659 -74.67106841612406) bank92879 +92880 POINT(41.45879006417141 -74.27631480174394) bank92880 +92881 POINT(39.77125393821053 -73.80101237173683) bank92881 +92882 POINT(41.2771517268219 -74.94961198874118) bank92882 +92883 POINT(40.32644285536248 -74.09906101349695) bank92883 +92884 POINT(41.411117527291616 -73.59130330297288) bank92884 +92885 POINT(40.14325516458189 -74.44182367662249) bank92885 +92886 POINT(40.090791586525036 -73.77940967066989) bank92886 +92887 POINT(41.059981907009906 -74.39593937619814) bank92887 +92888 POINT(39.77409560592553 -73.2366111075804) bank92888 +92889 POINT(40.80272092899136 -73.04994147840755) bank92889 +92890 POINT(41.6951122227021 -73.30638614819722) bank92890 +92891 POINT(39.93268201996453 -73.31367180894101) bank92891 +92892 POINT(41.4736578600901 -74.74412777599834) bank92892 +92893 POINT(41.5286498750329 -73.53279748561405) bank92893 +92894 POINT(40.39215299487546 -73.65430867386635) bank92894 +92895 POINT(41.267043006860604 -73.6880546083975) bank92895 +92896 POINT(39.94986613528162 -74.09366943527282) bank92896 +92897 POINT(41.636881028450446 -74.5356545880839) bank92897 +92898 POINT(39.99410967114795 -73.90235181712826) bank92898 +92899 POINT(41.47073759076306 -74.10545961792391) bank92899 +92900 POINT(40.37074230999062 -74.89744839103324) bank92900 +92901 POINT(40.39720306442208 -74.06445698111027) bank92901 +92902 POINT(41.279025716225505 -74.82498970834492) bank92902 +92903 POINT(41.70149158555518 -74.01412215824563) bank92903 +92904 POINT(41.04549926094087 -74.62183118473445) bank92904 +92905 POINT(40.611748990542225 -73.89695162027839) bank92905 +92906 POINT(39.73417357382519 -73.5835246131493) bank92906 +92907 POINT(40.32155268815423 -73.92638737758675) bank92907 +92908 POINT(41.01013462603568 -73.44595933466542) bank92908 +92909 POINT(41.49394334767207 -73.36164490531638) bank92909 +92910 POINT(40.696014368600494 -74.1685259490468) bank92910 +92911 POINT(41.34134790362862 -73.90717921570696) bank92911 +92912 POINT(41.277293833758264 -74.45292175455235) bank92912 +92913 POINT(40.90891892008714 -73.52748947145868) bank92913 +92914 POINT(40.63469776690429 -73.6042008910037) bank92914 +92915 POINT(40.56030218074794 -73.64876959038573) bank92915 +92916 POINT(40.87419372783045 -74.37969531384519) bank92916 +92917 POINT(40.05634277962538 -73.79651843384913) bank92917 +92918 POINT(41.47191352441142 -74.59312584688041) bank92918 +92919 POINT(39.97850491554878 -74.66724610133159) bank92919 +92920 POINT(41.26163061371032 -73.44327273642267) bank92920 +92921 POINT(39.76614501885 -73.03629181887969) bank92921 +92922 POINT(41.56302962255168 -73.09858710239234) bank92922 +92923 POINT(41.09492902124903 -73.76395366398268) bank92923 +92924 POINT(40.888300588033445 -74.94418394612555) bank92924 +92925 POINT(40.61250092996683 -74.63404901614989) bank92925 +92926 POINT(39.7607033496902 -73.57419812301394) bank92926 +92927 POINT(40.961920326808986 -73.94203689608096) bank92927 +92928 POINT(41.52931967991042 -73.96814388503408) bank92928 +92929 POINT(41.36964405406679 -73.76796766825838) bank92929 +92930 POINT(41.680775854519325 -73.45239123840344) bank92930 +92931 POINT(39.93590781015523 -73.44770321294355) bank92931 +92932 POINT(39.82820613162592 -74.61970086908792) bank92932 +92933 POINT(41.29704246114857 -73.36768975315871) bank92933 +92934 POINT(41.66753495252695 -73.2957795387354) bank92934 +92935 POINT(41.007013134900276 -73.02538676135396) bank92935 +92936 POINT(40.9684470277785 -73.23042653161559) bank92936 +92937 POINT(40.915196669908035 -73.41098692116172) bank92937 +92938 POINT(40.343394760957544 -74.8764095085157) bank92938 +92939 POINT(41.30814471981417 -73.54461060448921) bank92939 +92940 POINT(39.95323001256663 -73.62076675188538) bank92940 +92941 POINT(41.698721455181726 -74.95175356256208) bank92941 +92942 POINT(40.63829186424098 -74.62274740664893) bank92942 +92943 POINT(41.0910920668869 -74.69340888576225) bank92943 +92944 POINT(41.59379936526374 -74.05166751078447) bank92944 +92945 POINT(39.86786107743878 -74.13198153090507) bank92945 +92946 POINT(41.67043198155818 -73.37717223600333) bank92946 +92947 POINT(40.57378351936495 -74.25973767336738) bank92947 +92948 POINT(41.003818900908065 -74.64442424807793) bank92948 +92949 POINT(40.28354074135916 -74.72813428133438) bank92949 +92950 POINT(41.66226524169584 -74.27931995682104) bank92950 +92951 POINT(40.48738495407483 -73.58314457171642) bank92951 +92952 POINT(40.37316306250061 -73.35059985525189) bank92952 +92953 POINT(40.13041474017796 -73.48247352055402) bank92953 +92954 POINT(40.64233742692028 -73.10325712915764) bank92954 +92955 POINT(40.63701855060817 -73.98045124743732) bank92955 +92956 POINT(40.86967594210096 -73.42146725840881) bank92956 +92957 POINT(40.80069792033166 -74.49277056879707) bank92957 +92958 POINT(41.22114926819174 -74.12901669780257) bank92958 +92959 POINT(40.43700039226258 -74.06695325598331) bank92959 +92960 POINT(40.623385448778166 -74.41982598972763) bank92960 +92961 POINT(41.000851596555925 -74.01631518902003) bank92961 +92962 POINT(40.75712046279769 -73.16473925484198) bank92962 +92963 POINT(41.678248537859034 -74.43796379617005) bank92963 +92964 POINT(41.690581289332854 -73.92066056561083) bank92964 +92965 POINT(40.8218467418187 -73.3492122576467) bank92965 +92966 POINT(39.90068787504139 -74.6766744505718) bank92966 +92967 POINT(40.022246957736506 -73.91273236297782) bank92967 +92968 POINT(41.096339748469504 -73.5259948301835) bank92968 +92969 POINT(39.88910296764956 -74.37875583133201) bank92969 +92970 POINT(40.197208234365334 -73.10098112303223) bank92970 +92971 POINT(40.50064456325344 -74.04118051427004) bank92971 +92972 POINT(41.07268898637744 -73.57413121891224) bank92972 +92973 POINT(41.061959427786825 -74.7372694573369) bank92973 +92974 POINT(40.497602159139085 -73.51163418377261) bank92974 +92975 POINT(39.807377799790636 -73.13180987970418) bank92975 +92976 POINT(40.657103613506024 -73.35737317281409) bank92976 +92977 POINT(41.29274962939236 -74.17395033950739) bank92977 +92978 POINT(40.10496874372279 -74.88636871639915) bank92978 +92979 POINT(41.540569312025795 -73.37427379600366) bank92979 +92980 POINT(40.65163398052084 -73.5143715529169) bank92980 +92981 POINT(40.510061840423404 -73.67041482592415) bank92981 +92982 POINT(41.68118133455395 -74.21639139399888) bank92982 +92983 POINT(40.241300376762254 -73.42216431383605) bank92983 +92984 POINT(40.08424868522569 -74.5662569145911) bank92984 +92985 POINT(39.972452383131255 -73.7782508446707) bank92985 +92986 POINT(41.03309776764817 -73.90845330688255) bank92986 +92987 POINT(39.92748540492986 -74.08425457128473) bank92987 +92988 POINT(41.42469865873392 -74.75353872646461) bank92988 +92989 POINT(41.27670457200802 -73.91765145592016) bank92989 +92990 POINT(40.91430932903482 -73.06647949260496) bank92990 +92991 POINT(41.69090443701512 -74.56210859249921) bank92991 +92992 POINT(41.22002756672645 -74.49770256646947) bank92992 +92993 POINT(40.865518634463704 -73.3171700571656) bank92993 +92994 POINT(40.0542139390508 -73.97555717480385) bank92994 +92995 POINT(40.10154734921268 -73.25119853917133) bank92995 +92996 POINT(41.65694169709875 -73.92581288382834) bank92996 +92997 POINT(41.59004145887404 -73.56679155011774) bank92997 +92998 POINT(40.174903878053236 -74.12677799721966) bank92998 +92999 POINT(40.754676332517015 -73.56948238394784) bank92999 +93000 POINT(40.23682782569363 -74.47652726381388) bank93000 +93001 POINT(40.2865085699361 -74.45172624495653) bank93001 +93002 POINT(40.23177856847228 -74.81589640501028) bank93002 +93003 POINT(40.026319801377205 -74.55251652078142) bank93003 +93004 POINT(40.13419099769949 -73.38335582919153) bank93004 +93005 POINT(41.60575485660931 -73.40002107354809) bank93005 +93006 POINT(40.89647634562857 -74.35125317706884) bank93006 +93007 POINT(40.385047381801854 -74.52189979429875) bank93007 +93008 POINT(41.16751420022755 -73.97697488910703) bank93008 +93009 POINT(41.36793887252135 -74.31727062090214) bank93009 +93010 POINT(39.91411886870262 -73.37281231357024) bank93010 +93011 POINT(40.895622391901504 -73.57774082340715) bank93011 +93012 POINT(41.404443775828355 -73.49842683168032) bank93012 +93013 POINT(40.83950483394013 -74.90482537447396) bank93013 +93014 POINT(41.168984977486005 -73.78855407993402) bank93014 +93015 POINT(40.209563455015626 -74.61881908107298) bank93015 +93016 POINT(41.674873442311394 -73.29222976254394) bank93016 +93017 POINT(41.65051597484181 -74.2750320354653) bank93017 +93018 POINT(40.50917386014839 -73.14769224157685) bank93018 +93019 POINT(41.210415198732555 -73.90656019618191) bank93019 +93020 POINT(40.96393499446307 -74.64098516406393) bank93020 +93021 POINT(41.68969378126874 -74.07253096865773) bank93021 +93022 POINT(40.070859315859806 -74.29123064443738) bank93022 +93023 POINT(40.88809876784122 -73.82337107283286) bank93023 +93024 POINT(41.63131726972493 -74.6462592884353) bank93024 +93025 POINT(40.4767838809469 -73.91109647860183) bank93025 +93026 POINT(40.47581336549288 -73.64340725147412) bank93026 +93027 POINT(41.30590447704121 -73.16123104773033) bank93027 +93028 POINT(40.94404481908837 -74.43701148447327) bank93028 +93029 POINT(40.563223548974655 -74.52744795991931) bank93029 +93030 POINT(41.24665818658257 -74.35205864512962) bank93030 +93031 POINT(41.14507388433118 -74.8107195244365) bank93031 +93032 POINT(40.54810940536529 -74.06499115114232) bank93032 +93033 POINT(41.25411561037257 -74.11369375234067) bank93033 +93034 POINT(41.24973929126473 -74.22581957350204) bank93034 +93035 POINT(40.11368160885468 -74.1300708880195) bank93035 +93036 POINT(40.13883884692124 -73.8320835122689) bank93036 +93037 POINT(41.383091480208996 -74.118523450479) bank93037 +93038 POINT(40.84749617046193 -74.56009322565878) bank93038 +93039 POINT(40.210598859215295 -74.75161062254469) bank93039 +93040 POINT(40.891572197732486 -74.35520790332515) bank93040 +93041 POINT(41.49849805204439 -74.95562859600177) bank93041 +93042 POINT(41.595512219173436 -74.03634366677683) bank93042 +93043 POINT(41.70276737115119 -73.79791871265584) bank93043 +93044 POINT(41.654268397290934 -73.7012337494682) bank93044 +93045 POINT(40.898632416787045 -73.22896829944126) bank93045 +93046 POINT(40.12542727295966 -74.87862078051232) bank93046 +93047 POINT(41.03140685815075 -74.02585359905342) bank93047 +93048 POINT(41.29447311833187 -73.44670889238874) bank93048 +93049 POINT(40.04625711441539 -74.42197677908503) bank93049 +93050 POINT(40.081036892641826 -73.6046248229603) bank93050 +93051 POINT(41.46717706103049 -74.72593943956176) bank93051 +93052 POINT(41.075226392203845 -73.13638691970102) bank93052 +93053 POINT(40.765803466912814 -73.94579634417062) bank93053 +93054 POINT(40.15913995968364 -74.3613409892535) bank93054 +93055 POINT(40.788565440746126 -73.89705768984241) bank93055 +93056 POINT(40.89408029512147 -74.68356346161653) bank93056 +93057 POINT(40.05099867986864 -73.22254606960459) bank93057 +93058 POINT(40.95009833476171 -74.3384533236998) bank93058 +93059 POINT(41.467977551422756 -73.20822386788755) bank93059 +93060 POINT(40.18666707734859 -73.02972627949298) bank93060 +93061 POINT(40.50672242328758 -74.71711304468252) bank93061 +93062 POINT(41.29475490127196 -74.36459302422737) bank93062 +93063 POINT(41.264128977616465 -74.78094791976633) bank93063 +93064 POINT(40.41860425090919 -74.7175443781361) bank93064 +93065 POINT(41.14853253775957 -74.90416947561313) bank93065 +93066 POINT(40.07077574648653 -73.55199142456748) bank93066 +93067 POINT(41.44120499624335 -74.30218246599951) bank93067 +93068 POINT(41.69484149244564 -73.21490379486553) bank93068 +93069 POINT(39.813097399435705 -73.62961294698376) bank93069 +93070 POINT(40.33271053843779 -74.61215338269892) bank93070 +93071 POINT(41.694647034492746 -74.2803490008831) bank93071 +93072 POINT(40.061708165182225 -73.50163171875431) bank93072 +93073 POINT(41.01007890545747 -74.3410840803753) bank93073 +93074 POINT(41.3122265209833 -73.53873847565806) bank93074 +93075 POINT(40.31712900376161 -73.69519853630074) bank93075 +93076 POINT(40.13261177485447 -73.85222018849916) bank93076 +93077 POINT(41.499261776346806 -74.22879087477428) bank93077 +93078 POINT(39.846469513926664 -73.51647714469401) bank93078 +93079 POINT(40.76512127255676 -74.31037207690837) bank93079 +93080 POINT(41.10181943434919 -74.70530757539093) bank93080 +93081 POINT(40.117197165655035 -74.57760614332688) bank93081 +93082 POINT(40.16418878074358 -73.64032517481448) bank93082 +93083 POINT(41.04691322257777 -73.504619585619) bank93083 +93084 POINT(40.71019535831687 -74.21551121044213) bank93084 +93085 POINT(40.12880531718071 -73.8777797676339) bank93085 +93086 POINT(40.41214292676451 -74.61144380933725) bank93086 +93087 POINT(40.09019363873083 -73.95359400599068) bank93087 +93088 POINT(39.94326489171804 -73.74313666900352) bank93088 +93089 POINT(40.3796974073914 -73.73471680405821) bank93089 +93090 POINT(39.81063184585508 -74.0466990678949) bank93090 +93091 POINT(39.718295873161914 -73.88078788541027) bank93091 +93092 POINT(39.87070137862971 -74.7227073998902) bank93092 +93093 POINT(41.24001137427202 -73.39107223297923) bank93093 +93094 POINT(41.701791404125586 -74.39773610353002) bank93094 +93095 POINT(40.50375632221535 -74.18643294811648) bank93095 +93096 POINT(40.87720209644935 -74.68019099748936) bank93096 +93097 POINT(40.2727166820447 -74.48468905644967) bank93097 +93098 POINT(41.57829649838847 -73.01320219167101) bank93098 +93099 POINT(40.820994332579616 -73.70209039029776) bank93099 +93100 POINT(41.05081106245234 -73.86764030931093) bank93100 +93101 POINT(40.84243684454415 -73.24437674657852) bank93101 +93102 POINT(40.783559392996494 -74.80903140930104) bank93102 +93103 POINT(40.84895174587487 -73.00996878071656) bank93103 +93104 POINT(41.046987784503806 -74.16697728809662) bank93104 +93105 POINT(40.706146831170265 -74.60247095738524) bank93105 +93106 POINT(41.05052215098294 -74.13715171776236) bank93106 +93107 POINT(40.05696399045176 -74.80684638897922) bank93107 +93108 POINT(41.42238580645329 -74.7162428690829) bank93108 +93109 POINT(40.35375034732665 -73.86500184839153) bank93109 +93110 POINT(41.386133240065305 -73.96479240955247) bank93110 +93111 POINT(40.480075037754716 -74.83579505833681) bank93111 +93112 POINT(40.82496110398573 -73.03572617330724) bank93112 +93113 POINT(39.933149838409975 -74.5277484586019) bank93113 +93114 POINT(39.887320445027804 -73.56016498049712) bank93114 +93115 POINT(39.81636530586663 -73.01497607143564) bank93115 +93116 POINT(41.11744221176354 -73.88951010900668) bank93116 +93117 POINT(41.398527699550534 -73.33329639810479) bank93117 +93118 POINT(40.75753603058318 -73.16092187621086) bank93118 +93119 POINT(40.911342848770865 -73.78938649861307) bank93119 +93120 POINT(41.06194076581953 -74.77604826701118) bank93120 +93121 POINT(40.62643220260125 -74.84623223491117) bank93121 +93122 POINT(41.30624428383781 -73.15365994226899) bank93122 +93123 POINT(40.21633145231147 -73.01220062301473) bank93123 +93124 POINT(40.296175252623236 -73.85059336017328) bank93124 +93125 POINT(40.419154237337125 -73.99874802201774) bank93125 +93126 POINT(41.274618254368434 -73.80900098519436) bank93126 +93127 POINT(40.97919261077683 -73.90557246114484) bank93127 +93128 POINT(41.569758996461665 -74.61755340015162) bank93128 +93129 POINT(40.53387367536092 -73.2263030169894) bank93129 +93130 POINT(40.661868242102486 -74.75662568256091) bank93130 +93131 POINT(41.329666984029274 -73.85963632959094) bank93131 +93132 POINT(40.4238334584217 -73.9179919554413) bank93132 +93133 POINT(40.32964948193812 -73.38496361321278) bank93133 +93134 POINT(41.26224089728968 -74.82555840572981) bank93134 +93135 POINT(41.57351829150208 -73.45460606187594) bank93135 +93136 POINT(41.63416462147188 -73.19880534788892) bank93136 +93137 POINT(40.9993755843923 -73.80996180942235) bank93137 +93138 POINT(39.89031765370176 -74.82053265876898) bank93138 +93139 POINT(40.339313757575574 -74.62947789266644) bank93139 +93140 POINT(41.04279681682281 -74.53381543949719) bank93140 +93141 POINT(41.47047986284535 -74.41455494804198) bank93141 +93142 POINT(40.89912812895025 -74.21559445480476) bank93142 +93143 POINT(41.12082783521987 -74.60503550964349) bank93143 +93144 POINT(40.743772717394556 -74.0047780976424) bank93144 +93145 POINT(41.20315871079636 -74.09196936135305) bank93145 +93146 POINT(40.68958808605038 -73.68070337783588) bank93146 +93147 POINT(40.400444147063055 -74.8395135811708) bank93147 +93148 POINT(40.5171414892955 -74.93275068989188) bank93148 +93149 POINT(40.733031687403574 -73.91500750685738) bank93149 +93150 POINT(40.42662032218023 -73.60635734594882) bank93150 +93151 POINT(41.6049452338947 -74.56717275912628) bank93151 +93152 POINT(40.59171638411852 -74.56201438114115) bank93152 +93153 POINT(39.74398953232382 -73.91024651642014) bank93153 +93154 POINT(40.97896490268623 -73.5838883857581) bank93154 +93155 POINT(41.06716644496777 -74.12969280534378) bank93155 +93156 POINT(41.02469805969466 -73.34700773883019) bank93156 +93157 POINT(39.785610476829994 -74.87257832424558) bank93157 +93158 POINT(41.4007863966455 -73.21151485673745) bank93158 +93159 POINT(41.17129628393991 -73.29631446938588) bank93159 +93160 POINT(41.06518135990305 -73.67848852951016) bank93160 +93161 POINT(40.492560895846175 -73.09558039681308) bank93161 +93162 POINT(40.08538362643576 -74.88416735687437) bank93162 +93163 POINT(40.614692726020486 -74.39794058189585) bank93163 +93164 POINT(40.352351051330565 -74.89146589115119) bank93164 +93165 POINT(40.476309944089 -74.76464489808635) bank93165 +93166 POINT(40.41141663147489 -74.49633332807751) bank93166 +93167 POINT(40.036375285002904 -74.44203276083128) bank93167 +93168 POINT(40.22056237357005 -74.1287398629028) bank93168 +93169 POINT(40.05288582411166 -74.01741407372059) bank93169 +93170 POINT(41.66132439390412 -74.40537121540501) bank93170 +93171 POINT(40.153719767971914 -74.61476805784321) bank93171 +93172 POINT(40.67412381012185 -74.80091547792318) bank93172 +93173 POINT(40.21790277833433 -73.4725126593246) bank93173 +93174 POINT(40.127980835520816 -74.53979508264416) bank93174 +93175 POINT(41.386569653905866 -74.71462852164073) bank93175 +93176 POINT(40.40273679615669 -74.78241734127775) bank93176 +93177 POINT(40.52566986815298 -74.7154493081735) bank93177 +93178 POINT(39.96206955712362 -74.67342897827665) bank93178 +93179 POINT(39.86708502617697 -73.87618122996636) bank93179 +93180 POINT(40.7768104640806 -74.38468138039727) bank93180 +93181 POINT(41.688807594020716 -73.67093888170633) bank93181 +93182 POINT(41.233604534645856 -74.25235892862537) bank93182 +93183 POINT(40.07412161210249 -73.675194812562) bank93183 +93184 POINT(40.70943830468127 -74.21155656178946) bank93184 +93185 POINT(40.25192916119244 -73.696057209897) bank93185 +93186 POINT(41.388351325641786 -73.36327405278752) bank93186 +93187 POINT(40.89207567227101 -73.99951448558885) bank93187 +93188 POINT(41.6690266680524 -74.74962563836432) bank93188 +93189 POINT(40.719160288194814 -74.48228737596527) bank93189 +93190 POINT(41.12641171252377 -73.66200338905847) bank93190 +93191 POINT(39.90962363884677 -74.21705538344256) bank93191 +93192 POINT(40.92219551525011 -73.54692544878631) bank93192 +93193 POINT(41.40826967895198 -74.96667219292833) bank93193 +93194 POINT(41.67600381839313 -74.60133921198675) bank93194 +93195 POINT(40.80646194467189 -73.04875119970667) bank93195 +93196 POINT(40.197876574217105 -73.51514074151774) bank93196 +93197 POINT(41.45583587001986 -73.38108944306776) bank93197 +93198 POINT(40.81430529417342 -74.60848717706972) bank93198 +93199 POINT(40.89544876128961 -73.23679623902004) bank93199 +93200 POINT(41.574186418367354 -73.59860039516661) bank93200 +93201 POINT(40.44397337985898 -73.71793733473619) bank93201 +93202 POINT(41.03469927987681 -74.02540935901179) bank93202 +93203 POINT(40.45342519093834 -73.14706655266455) bank93203 +93204 POINT(40.10586145299131 -73.92138099277084) bank93204 +93205 POINT(41.358987647860445 -73.46123882911711) bank93205 +93206 POINT(40.81076320807919 -74.75776922743972) bank93206 +93207 POINT(40.66433056286673 -74.47918648454225) bank93207 +93208 POINT(40.061871302460695 -74.91922247090714) bank93208 +93209 POINT(40.94787543269018 -74.11549752148463) bank93209 +93210 POINT(40.09289872724451 -73.39257550547967) bank93210 +93211 POINT(40.80346064095973 -74.39735748071335) bank93211 +93212 POINT(41.65873477683439 -74.39971998967553) bank93212 +93213 POINT(40.74068889504541 -73.61748353027048) bank93213 +93214 POINT(39.86697165098961 -74.23892190386674) bank93214 +93215 POINT(41.28797714856033 -74.92328284332024) bank93215 +93216 POINT(41.53442462443937 -73.7192331129268) bank93216 +93217 POINT(39.7781611658809 -73.06715229982723) bank93217 +93218 POINT(41.31681293251178 -74.93371608383615) bank93218 +93219 POINT(40.17852716162572 -74.3132659609721) bank93219 +93220 POINT(40.291471935156565 -73.20586476782033) bank93220 +93221 POINT(40.3015320372301 -74.52835481138133) bank93221 +93222 POINT(40.809407535714065 -74.83775666266101) bank93222 +93223 POINT(39.909646468751816 -73.52484253590241) bank93223 +93224 POINT(40.02071103834497 -74.92545637415643) bank93224 +93225 POINT(40.057716255627994 -74.15327606956632) bank93225 +93226 POINT(40.24958463417277 -74.36941033021229) bank93226 +93227 POINT(40.151451730811836 -73.88158038824852) bank93227 +93228 POINT(40.54570214209385 -74.43672060710202) bank93228 +93229 POINT(40.15711987611207 -73.52344167954969) bank93229 +93230 POINT(40.479463653249745 -73.28391545564942) bank93230 +93231 POINT(40.23253853741162 -73.6524490584109) bank93231 +93232 POINT(41.00542647507424 -74.79262483931892) bank93232 +93233 POINT(39.87845016591827 -74.10772187605059) bank93233 +93234 POINT(40.25187903469696 -73.1942561136366) bank93234 +93235 POINT(40.03704325355847 -73.87678673097828) bank93235 +93236 POINT(40.86907595788894 -74.64578868162745) bank93236 +93237 POINT(41.391038588792036 -74.31150511466704) bank93237 +93238 POINT(40.76646821325175 -73.37039683351698) bank93238 +93239 POINT(39.98159875768402 -74.99950813714426) bank93239 +93240 POINT(41.60552580156419 -73.12543434600671) bank93240 +93241 POINT(41.042509591285096 -74.41286484300198) bank93241 +93242 POINT(40.17409259689086 -74.06173142419891) bank93242 +93243 POINT(40.107574667100074 -74.69997868542765) bank93243 +93244 POINT(41.66747429996343 -74.7167593876545) bank93244 +93245 POINT(40.29388393457597 -73.87005982173791) bank93245 +93246 POINT(40.11865597607546 -73.26179318612179) bank93246 +93247 POINT(41.17972438603941 -73.39096942570102) bank93247 +93248 POINT(41.4922792102116 -74.5921383640974) bank93248 +93249 POINT(40.88463152240801 -74.3544651957182) bank93249 +93250 POINT(41.31742751560237 -73.4134238569165) bank93250 +93251 POINT(40.47998468934835 -73.99857626005553) bank93251 +93252 POINT(40.68030262946286 -74.07236390443501) bank93252 +93253 POINT(40.52196105623847 -74.59144666501481) bank93253 +93254 POINT(40.1365401437424 -74.16035104538406) bank93254 +93255 POINT(39.83868609206421 -73.0618043208518) bank93255 +93256 POINT(40.31247474072618 -73.66282125317994) bank93256 +93257 POINT(40.00780443266165 -73.53450103286067) bank93257 +93258 POINT(40.90772521473628 -74.01130138009576) bank93258 +93259 POINT(40.425072980507395 -73.31596684331338) bank93259 +93260 POINT(40.23430611993127 -74.91496014073199) bank93260 +93261 POINT(40.07746181274669 -73.3908461717727) bank93261 +93262 POINT(41.205216205348634 -73.49420252533974) bank93262 +93263 POINT(41.50488220295129 -73.9279064479699) bank93263 +93264 POINT(41.50693999182807 -74.7099519140241) bank93264 +93265 POINT(40.60027437948071 -73.2998262961881) bank93265 +93266 POINT(41.66123731243207 -74.62567743003841) bank93266 +93267 POINT(40.60391601557244 -74.15786480040452) bank93267 +93268 POINT(41.40199054240672 -74.261035079841) bank93268 +93269 POINT(40.12998942631211 -74.93775898966219) bank93269 +93270 POINT(39.71486178132481 -73.45806085248753) bank93270 +93271 POINT(39.788868551222826 -74.02000502247955) bank93271 +93272 POINT(41.03438237571086 -73.09231272081573) bank93272 +93273 POINT(41.249493146275604 -74.05193867195108) bank93273 +93274 POINT(40.729611542476675 -73.77910207503221) bank93274 +93275 POINT(41.003316939915706 -73.91114114392923) bank93275 +93276 POINT(41.135098312898805 -74.29362268409679) bank93276 +93277 POINT(41.113463656478324 -74.74373356314678) bank93277 +93278 POINT(40.25090320250855 -74.02302796599511) bank93278 +93279 POINT(40.64246159654987 -73.1312500697234) bank93279 +93280 POINT(41.55555412471312 -73.20038578568347) bank93280 +93281 POINT(40.37412991541275 -74.04452715712446) bank93281 +93282 POINT(40.51834730468722 -74.43702500763513) bank93282 +93283 POINT(40.412536370423005 -73.29416909164837) bank93283 +93284 POINT(41.39941786492847 -73.10870196083356) bank93284 +93285 POINT(40.47761756847545 -74.29247878711256) bank93285 +93286 POINT(40.383842186223795 -73.84674260830643) bank93286 +93287 POINT(39.971305670604146 -73.11475018846052) bank93287 +93288 POINT(39.90840617041652 -73.76627294553315) bank93288 +93289 POINT(39.78065823429521 -74.40934430832314) bank93289 +93290 POINT(40.091876893636616 -74.15098075495656) bank93290 +93291 POINT(41.16604459581271 -74.6390040111486) bank93291 +93292 POINT(39.80113438122384 -73.07005421440063) bank93292 +93293 POINT(41.06806993473515 -74.49858039819904) bank93293 +93294 POINT(40.460364693134395 -73.79180055611685) bank93294 +93295 POINT(41.300014304084385 -73.50441105150743) bank93295 +93296 POINT(39.89650663436348 -74.50232236024935) bank93296 +93297 POINT(40.81227692560052 -73.49659860101352) bank93297 +93298 POINT(41.13075906249273 -74.85263813057716) bank93298 +93299 POINT(40.38053178052202 -73.81165698684771) bank93299 +93300 POINT(41.06162222735396 -74.77031331904315) bank93300 +93301 POINT(41.06470030707649 -73.68268170192704) bank93301 +93302 POINT(40.77339465053057 -73.11986550152248) bank93302 +93303 POINT(41.15333495852826 -74.95035483159704) bank93303 +93304 POINT(41.09834157959203 -73.5711863452083) bank93304 +93305 POINT(41.233266095210325 -73.68711722236299) bank93305 +93306 POINT(41.14052730378387 -73.03640496121223) bank93306 +93307 POINT(41.05888641250758 -74.03366299000422) bank93307 +93308 POINT(40.72065708226562 -73.85980324615201) bank93308 +93309 POINT(40.20779224068746 -74.56455737648314) bank93309 +93310 POINT(41.21369976731319 -74.76330247928486) bank93310 +93311 POINT(40.199253163648585 -74.82892789734849) bank93311 +93312 POINT(40.63512024282166 -73.15545939282302) bank93312 +93313 POINT(40.92214541091848 -74.17970196224088) bank93313 +93314 POINT(41.54461430530281 -73.21812207642746) bank93314 +93315 POINT(40.93907473922589 -74.65399475512686) bank93315 +93316 POINT(40.8963380965107 -73.55304664690938) bank93316 +93317 POINT(39.79638987726966 -74.19891992383101) bank93317 +93318 POINT(40.17830269716322 -74.19747083302204) bank93318 +93319 POINT(40.7432870979893 -73.19401735932625) bank93319 +93320 POINT(40.96353687806287 -73.22188885985358) bank93320 +93321 POINT(41.02264065586306 -73.06883194256419) bank93321 +93322 POINT(40.831672678047894 -73.42310620139332) bank93322 +93323 POINT(40.715982654442634 -74.87188433599545) bank93323 +93324 POINT(40.170228817808535 -73.4330273023127) bank93324 +93325 POINT(39.748696255718116 -73.25677260285313) bank93325 +93326 POINT(41.62622084587165 -73.92034571422538) bank93326 +93327 POINT(41.71274345791645 -74.35542240696753) bank93327 +93328 POINT(40.88413520590153 -74.12163701322314) bank93328 +93329 POINT(39.987538501341625 -73.56943787292327) bank93329 +93330 POINT(40.70457275527058 -73.1706248640789) bank93330 +93331 POINT(40.501786222721 -73.38671143406177) bank93331 +93332 POINT(39.98932504138735 -73.92840460346962) bank93332 +93333 POINT(40.4768500656841 -74.38034016902067) bank93333 +93334 POINT(40.999130034125386 -74.59095192184513) bank93334 +93335 POINT(41.64531972976164 -74.18754933239397) bank93335 +93336 POINT(40.35200937601342 -73.68130067792899) bank93336 +93337 POINT(39.95365264456608 -73.58847126298757) bank93337 +93338 POINT(41.33803314348241 -73.85874858408324) bank93338 +93339 POINT(40.92045353149326 -73.0214056059527) bank93339 +93340 POINT(41.35223076178252 -73.26396795712152) bank93340 +93341 POINT(40.358652816181745 -73.89490190520048) bank93341 +93342 POINT(40.00904594117357 -73.69344161956283) bank93342 +93343 POINT(41.21102440724415 -73.22978114923721) bank93343 +93344 POINT(40.98159773414406 -73.25103080397855) bank93344 +93345 POINT(41.12814808046224 -74.19950567531744) bank93345 +93346 POINT(40.40978549256497 -74.17984961653796) bank93346 +93347 POINT(41.124991144761815 -73.40718618274742) bank93347 +93348 POINT(40.6092906919291 -73.13177894323138) bank93348 +93349 POINT(40.72991951394382 -73.07722817361002) bank93349 +93350 POINT(41.61094253736225 -73.36108111576613) bank93350 +93351 POINT(40.18824129659801 -74.02052105318033) bank93351 +93352 POINT(39.94430918745207 -74.02136556023804) bank93352 +93353 POINT(40.94724273634917 -74.85376756128893) bank93353 +93354 POINT(40.42845773771829 -74.21910242300234) bank93354 +93355 POINT(40.295010412278515 -73.50622092540965) bank93355 +93356 POINT(40.84322028150473 -74.98736311326944) bank93356 +93357 POINT(41.55883053508726 -74.18173485923552) bank93357 +93358 POINT(40.157965499061405 -73.56573762773048) bank93358 +93359 POINT(41.35701054664534 -73.50656635186219) bank93359 +93360 POINT(40.89303079770788 -73.0830457358245) bank93360 +93361 POINT(40.83356096563721 -74.02510797597226) bank93361 +93362 POINT(41.28095612862725 -73.16054068923764) bank93362 +93363 POINT(40.65000637792875 -74.13105526459364) bank93363 +93364 POINT(40.30366413223494 -73.05854885710065) bank93364 +93365 POINT(40.78943587970471 -73.65049630716405) bank93365 +93366 POINT(41.19607757231511 -73.1252085361015) bank93366 +93367 POINT(40.571426863523854 -73.2279123781363) bank93367 +93368 POINT(41.692534350592915 -74.29333757389972) bank93368 +93369 POINT(41.41908771344403 -73.25130135699557) bank93369 +93370 POINT(40.125671172291554 -73.77711820682826) bank93370 +93371 POINT(40.47604755077652 -73.35973029740161) bank93371 +93372 POINT(40.69252723687211 -73.80718634630489) bank93372 +93373 POINT(39.877517057204095 -74.39607113906608) bank93373 +93374 POINT(40.27127408483046 -74.40605974011928) bank93374 +93375 POINT(40.06107512475383 -74.62624730722459) bank93375 +93376 POINT(39.84055396492453 -73.52641031784532) bank93376 +93377 POINT(41.38105241216156 -74.99276094036652) bank93377 +93378 POINT(40.897062501314714 -73.44148680275231) bank93378 +93379 POINT(40.983926786270516 -74.51976571238782) bank93379 +93380 POINT(41.41335248776202 -74.90822479878751) bank93380 +93381 POINT(41.37125601543872 -74.63785521487905) bank93381 +93382 POINT(39.87167378691675 -73.18653230954037) bank93382 +93383 POINT(40.07606620550882 -74.32662486321384) bank93383 +93384 POINT(41.669619137489526 -74.20350707482774) bank93384 +93385 POINT(41.164898508595 -73.72988873825828) bank93385 +93386 POINT(40.13985080340065 -73.68545254960269) bank93386 +93387 POINT(40.826488354327765 -74.57537840273862) bank93387 +93388 POINT(41.11276147857205 -74.1347399535375) bank93388 +93389 POINT(41.07880481376916 -74.78848612206683) bank93389 +93390 POINT(39.97942025420915 -74.62783767042133) bank93390 +93391 POINT(40.60315448965802 -74.43954034005543) bank93391 +93392 POINT(40.077349262510374 -74.25725139725856) bank93392 +93393 POINT(39.79136834627009 -74.29560007032651) bank93393 +93394 POINT(40.92861338244056 -74.27410490394017) bank93394 +93395 POINT(40.35470216382643 -73.60066644607829) bank93395 +93396 POINT(40.432778720989056 -74.4246140323763) bank93396 +93397 POINT(41.61520921191407 -74.1024159296638) bank93397 +93398 POINT(41.19475692421715 -74.67352354114598) bank93398 +93399 POINT(40.45287058650896 -74.97451841378673) bank93399 +93400 POINT(39.86289497920861 -73.40921661989857) bank93400 +93401 POINT(41.00202266909741 -74.78975368277972) bank93401 +93402 POINT(41.70300354803083 -74.56855272066925) bank93402 +93403 POINT(40.13793390353096 -74.42638575451902) bank93403 +93404 POINT(39.71733648543673 -73.45947484042473) bank93404 +93405 POINT(41.0090416401072 -73.11828490000444) bank93405 +93406 POINT(41.33478631530353 -73.3857918951813) bank93406 +93407 POINT(39.7129333466259 -74.740200742904) bank93407 +93408 POINT(40.90513525938934 -73.88539946274214) bank93408 +93409 POINT(39.95902123467915 -73.12751413705928) bank93409 +93410 POINT(41.52778281321582 -73.0598494280623) bank93410 +93411 POINT(40.84577900132847 -73.64480102144542) bank93411 +93412 POINT(40.50397494447991 -74.74765801172208) bank93412 +93413 POINT(40.35847380995182 -73.93840258188536) bank93413 +93414 POINT(41.45058863949288 -73.97867097641632) bank93414 +93415 POINT(41.5079922787184 -74.60213695791516) bank93415 +93416 POINT(40.77224155734175 -73.75582896275178) bank93416 +93417 POINT(40.823119097459134 -74.69529948920115) bank93417 +93418 POINT(39.84528751215963 -73.99092264806028) bank93418 +93419 POINT(41.318225356260754 -73.85567148509546) bank93419 +93420 POINT(40.66700478003248 -74.91469522121304) bank93420 +93421 POINT(40.83250518591693 -74.37865780348157) bank93421 +93422 POINT(41.623765711572744 -74.84715723578613) bank93422 +93423 POINT(41.2654437923054 -74.35373822834943) bank93423 +93424 POINT(40.08939506192322 -73.174931360174) bank93424 +93425 POINT(40.01546854134625 -74.57720087602198) bank93425 +93426 POINT(40.842402540191365 -73.20647140025557) bank93426 +93427 POINT(39.80341691142741 -73.7665621813261) bank93427 +93428 POINT(41.234250569061764 -74.13327879993552) bank93428 +93429 POINT(40.804008069683924 -73.86645021150004) bank93429 +93430 POINT(40.31860885119755 -73.58684563646786) bank93430 +93431 POINT(40.61136116157191 -73.11258661642508) bank93431 +93432 POINT(40.29914626786485 -73.74054502046654) bank93432 +93433 POINT(40.614433214746434 -73.99565386488173) bank93433 +93434 POINT(40.757974812934386 -74.16247632493979) bank93434 +93435 POINT(41.29087875809963 -74.21362298995004) bank93435 +93436 POINT(39.80416859427416 -73.38728820228266) bank93436 +93437 POINT(41.51863129370143 -74.63848771840678) bank93437 +93438 POINT(41.084809144237 -74.91152019423068) bank93438 +93439 POINT(40.8094889343299 -73.11057360577773) bank93439 +93440 POINT(41.675120128550844 -74.23066651653969) bank93440 +93441 POINT(41.70899322958214 -73.41379611982116) bank93441 +93442 POINT(40.74092087659164 -73.31183167393543) bank93442 +93443 POINT(41.48183680716151 -73.89791108382083) bank93443 +93444 POINT(41.12961599496215 -73.4739779927099) bank93444 +93445 POINT(40.491997143133474 -73.01981319582727) bank93445 +93446 POINT(41.257578468754176 -73.28103867975886) bank93446 +93447 POINT(41.3601518482104 -73.02507361371049) bank93447 +93448 POINT(41.23228367961544 -73.66563165212663) bank93448 +93449 POINT(40.92305711898672 -74.59210702354176) bank93449 +93450 POINT(39.82642720410983 -73.27693363060118) bank93450 +93451 POINT(41.088073796393694 -73.42960018138069) bank93451 +93452 POINT(41.23456157460088 -73.98876320673786) bank93452 +93453 POINT(40.996764254040954 -73.99957032363112) bank93453 +93454 POINT(41.539975302297506 -73.84115977671694) bank93454 +93455 POINT(40.358810944689374 -74.03938529983925) bank93455 +93456 POINT(40.897110394825674 -73.99675084806715) bank93456 +93457 POINT(40.63587105506748 -74.19145515289186) bank93457 +93458 POINT(41.65462110543505 -73.53576154161362) bank93458 +93459 POINT(40.04691369271358 -74.56043746551117) bank93459 +93460 POINT(40.95571148889421 -73.23702315200349) bank93460 +93461 POINT(40.157537394453605 -73.9135754176246) bank93461 +93462 POINT(41.1144641079486 -73.16057909197525) bank93462 +93463 POINT(39.97035590946512 -74.61630182034445) bank93463 +93464 POINT(40.69044445646829 -74.78244263866807) bank93464 +93465 POINT(40.80270431172357 -73.76329264450384) bank93465 +93466 POINT(40.77607271135323 -73.61829466597997) bank93466 +93467 POINT(40.19610578374705 -74.30256319730778) bank93467 +93468 POINT(40.1963337488811 -73.01741021597094) bank93468 +93469 POINT(41.5298978942798 -73.18466541120587) bank93469 +93470 POINT(41.06739541410628 -73.5056266988787) bank93470 +93471 POINT(40.47734042260835 -73.79572820797343) bank93471 +93472 POINT(40.327509271211675 -73.71750215674099) bank93472 +93473 POINT(40.90297154459247 -73.25916776738978) bank93473 +93474 POINT(40.72596373589504 -73.48470472235418) bank93474 +93475 POINT(41.36957740184171 -73.45279313102573) bank93475 +93476 POINT(40.75909697537316 -74.64575305157918) bank93476 +93477 POINT(40.386888220949054 -73.72002709180524) bank93477 +93478 POINT(41.26235154573419 -74.76312861120958) bank93478 +93479 POINT(40.512246241145014 -73.62210896240522) bank93479 +93480 POINT(40.3275881300052 -74.52402010726406) bank93480 +93481 POINT(39.98044644982002 -73.2184967586762) bank93481 +93482 POINT(40.8828880522309 -73.59859168377774) bank93482 +93483 POINT(40.77254769260611 -74.29046922663228) bank93483 +93484 POINT(41.436546001051404 -74.63051092106605) bank93484 +93485 POINT(40.70237899473871 -74.96470634293095) bank93485 +93486 POINT(40.71943911937226 -73.95429962311846) bank93486 +93487 POINT(40.49551710119509 -74.31531446532362) bank93487 +93488 POINT(40.02026733432634 -74.54180173802067) bank93488 +93489 POINT(40.47943978531839 -74.50158324962028) bank93489 +93490 POINT(41.71138432373691 -73.07583505530495) bank93490 +93491 POINT(40.38076513164299 -74.04488354244904) bank93491 +93492 POINT(40.64130696396048 -74.83980807921701) bank93492 +93493 POINT(41.494838953564 -74.87363014675492) bank93493 +93494 POINT(39.771720020710326 -73.96538340054175) bank93494 +93495 POINT(39.99201234754785 -73.09050490406852) bank93495 +93496 POINT(40.4586170903153 -73.02001610545237) bank93496 +93497 POINT(40.46068755250171 -73.4057485178156) bank93497 +93498 POINT(41.480884216998014 -74.9696518848321) bank93498 +93499 POINT(40.5644271548114 -74.13489864297857) bank93499 +93500 POINT(41.30355266809935 -73.4384935673293) bank93500 +93501 POINT(41.56847450045194 -74.95903190974728) bank93501 +93502 POINT(40.76082920483429 -73.03762274381359) bank93502 +93503 POINT(41.23512502751437 -73.44517874625224) bank93503 +93504 POINT(41.42204611476785 -73.34011366645055) bank93504 +93505 POINT(39.86960601117751 -73.79024339789238) bank93505 +93506 POINT(40.001788936494314 -73.67649004695909) bank93506 +93507 POINT(40.07273833166169 -74.12459622346086) bank93507 +93508 POINT(40.2879627282335 -73.95711744488727) bank93508 +93509 POINT(40.214145666253465 -73.88264081304467) bank93509 +93510 POINT(39.8101634984261 -73.87611025720834) bank93510 +93511 POINT(41.177799778929106 -74.1815650791926) bank93511 +93512 POINT(40.92660657677896 -74.27763908671797) bank93512 +93513 POINT(40.102076767576655 -73.73035486801835) bank93513 +93514 POINT(39.73670841992938 -74.94805527326936) bank93514 +93515 POINT(40.986517697359744 -74.19376805028666) bank93515 +93516 POINT(40.83019445291809 -73.88901198623813) bank93516 +93517 POINT(39.79250256813533 -73.17748350772027) bank93517 +93518 POINT(41.128358263438486 -73.96022209209825) bank93518 +93519 POINT(40.57575229129145 -75.0011675041994) bank93519 +93520 POINT(39.75460865578003 -74.64095830330996) bank93520 +93521 POINT(41.32538643522503 -73.89239081485023) bank93521 +93522 POINT(40.94179456391545 -74.46331090907489) bank93522 +93523 POINT(41.65741345600158 -73.62906949830621) bank93523 +93524 POINT(39.86145527892584 -73.14104526101103) bank93524 +93525 POINT(40.46188534822667 -74.1094052323226) bank93525 +93526 POINT(41.58568746999398 -73.88202800138993) bank93526 +93527 POINT(41.35894356355857 -73.23836368416619) bank93527 +93528 POINT(40.27816257698255 -74.89964353434418) bank93528 +93529 POINT(41.15781357135687 -74.6501987540988) bank93529 +93530 POINT(41.0979461138948 -74.2300084341953) bank93530 +93531 POINT(41.579974154356435 -74.23299088232933) bank93531 +93532 POINT(41.37746159545619 -73.83212009611047) bank93532 +93533 POINT(41.11366128419871 -74.01043256567054) bank93533 +93534 POINT(41.07615087561621 -73.86402698756821) bank93534 +93535 POINT(41.44537967911007 -73.3455949524025) bank93535 +93536 POINT(40.46530871031283 -74.18021188397277) bank93536 +93537 POINT(39.87440156880704 -74.00142375632427) bank93537 +93538 POINT(40.83704650819761 -74.90285219012027) bank93538 +93539 POINT(40.34279231942965 -73.48794255499071) bank93539 +93540 POINT(40.385623659439496 -73.370701928233) bank93540 +93541 POINT(40.34662733972173 -74.26795439080168) bank93541 +93542 POINT(40.758115947315915 -73.49948360249721) bank93542 +93543 POINT(41.23865068189245 -74.69507573132701) bank93543 +93544 POINT(40.80060556653269 -74.4980698561518) bank93544 +93545 POINT(41.51985958554569 -73.90209574499927) bank93545 +93546 POINT(39.88782835788602 -74.76628317643728) bank93546 +93547 POINT(39.92614810141562 -73.77958514167774) bank93547 +93548 POINT(40.400592658127785 -74.68112518542377) bank93548 +93549 POINT(41.474659982666225 -73.79238706628969) bank93549 +93550 POINT(40.3935195583442 -73.09605491290185) bank93550 +93551 POINT(41.0173013925744 -74.17665635060432) bank93551 +93552 POINT(41.15918897690363 -73.17119332835391) bank93552 +93553 POINT(39.95424941985082 -73.04158612651898) bank93553 +93554 POINT(40.43126325847681 -73.81786182875433) bank93554 +93555 POINT(41.01570639842461 -74.2816976010947) bank93555 +93556 POINT(40.102213387635096 -74.13605808007604) bank93556 +93557 POINT(40.55816542314221 -74.95291257711129) bank93557 +93558 POINT(41.53627260285541 -73.46370809240695) bank93558 +93559 POINT(41.165905288617346 -74.71176123646627) bank93559 +93560 POINT(40.384102586406605 -74.47774351556086) bank93560 +93561 POINT(40.89717796308422 -73.81554892341686) bank93561 +93562 POINT(40.14838409700023 -73.14395793620574) bank93562 +93563 POINT(40.73478374869887 -73.94797462401527) bank93563 +93564 POINT(39.933530430390554 -73.4720623799643) bank93564 +93565 POINT(41.15981917698045 -74.33038252026488) bank93565 +93566 POINT(40.035973457975075 -74.43068949498694) bank93566 +93567 POINT(41.100782368636004 -74.90802039790232) bank93567 +93568 POINT(40.85945191114114 -73.08559848325173) bank93568 +93569 POINT(40.852178288002136 -73.50959508650898) bank93569 +93570 POINT(41.21102611630559 -73.9185225565187) bank93570 +93571 POINT(40.277352843776775 -73.41084659411736) bank93571 +93572 POINT(40.45843322796549 -74.7941029896588) bank93572 +93573 POINT(39.97728719714457 -73.01847479775466) bank93573 +93574 POINT(40.14975305673934 -74.45796941274646) bank93574 +93575 POINT(40.69504816257126 -73.50307363737066) bank93575 +93576 POINT(40.14621068125638 -73.36501289829755) bank93576 +93577 POINT(40.81291070453415 -73.5629986491426) bank93577 +93578 POINT(40.06053395000545 -73.63303762583952) bank93578 +93579 POINT(40.812734525359545 -74.93569539424298) bank93579 +93580 POINT(41.40317089434189 -74.0105185137587) bank93580 +93581 POINT(40.95922974453799 -73.36649102874304) bank93581 +93582 POINT(39.84549174522095 -74.45088684206273) bank93582 +93583 POINT(40.48856572601517 -73.64680801193052) bank93583 +93584 POINT(40.76708452025537 -74.44244995653233) bank93584 +93585 POINT(39.75700318405429 -74.56730463440242) bank93585 +93586 POINT(40.42751297824233 -74.80630682683555) bank93586 +93587 POINT(40.12342421449057 -73.58849054973389) bank93587 +93588 POINT(41.024821773594454 -73.64765833369601) bank93588 +93589 POINT(40.6049476086022 -74.79353284092333) bank93589 +93590 POINT(41.57534578338912 -73.25419499243259) bank93590 +93591 POINT(41.31826560740618 -73.60938097897748) bank93591 +93592 POINT(39.752685085741305 -74.55727397748171) bank93592 +93593 POINT(39.99072388941081 -73.76461078586131) bank93593 +93594 POINT(40.832579832760196 -74.86033863715286) bank93594 +93595 POINT(40.65769645420116 -74.24412514693213) bank93595 +93596 POINT(40.06259181041338 -74.25855998583684) bank93596 +93597 POINT(40.73340243262373 -73.61168137114167) bank93597 +93598 POINT(40.97248368607835 -73.89806516886992) bank93598 +93599 POINT(40.74284329521599 -74.5622495158163) bank93599 +93600 POINT(41.13733148962738 -73.52733004344455) bank93600 +93601 POINT(40.76407172578208 -73.96949910037934) bank93601 +93602 POINT(40.3402338089863 -73.29698292521248) bank93602 +93603 POINT(41.605764866243376 -73.65571427571639) bank93603 +93604 POINT(40.82740694629533 -74.51882846212995) bank93604 +93605 POINT(40.83846873997782 -73.8015512036143) bank93605 +93606 POINT(40.81725750143105 -74.53123510926623) bank93606 +93607 POINT(40.61475329667873 -74.43287212100165) bank93607 +93608 POINT(40.87070868864989 -74.96730033098657) bank93608 +93609 POINT(40.58463464321527 -73.9131448230163) bank93609 +93610 POINT(40.88301122519367 -74.88799439750208) bank93610 +93611 POINT(39.770642865328306 -74.47461349095407) bank93611 +93612 POINT(41.67828727060464 -73.36608732493812) bank93612 +93613 POINT(40.538512526126915 -73.0691512421185) bank93613 +93614 POINT(41.44907904627793 -74.99222059938005) bank93614 +93615 POINT(41.42057658822597 -74.59412142344357) bank93615 +93616 POINT(40.25744780312351 -73.78429827917121) bank93616 +93617 POINT(39.96949302871599 -74.5519329624945) bank93617 +93618 POINT(40.923299543609204 -73.52209897978835) bank93618 +93619 POINT(41.09828482299687 -74.72493833257371) bank93619 +93620 POINT(40.82532787773596 -74.00439933823226) bank93620 +93621 POINT(41.46607018446415 -74.27166448203793) bank93621 +93622 POINT(41.579537006227035 -73.16210862213853) bank93622 +93623 POINT(41.683613878412544 -74.594108165797) bank93623 +93624 POINT(40.525770292647834 -74.16673300507301) bank93624 +93625 POINT(40.62132578536704 -74.1562190055866) bank93625 +93626 POINT(41.460351944634475 -74.65718004679374) bank93626 +93627 POINT(39.88846079706098 -74.03750201092153) bank93627 +93628 POINT(40.306776570909165 -74.43669376235533) bank93628 +93629 POINT(41.05238439024557 -74.58280473675681) bank93629 +93630 POINT(39.824224440814 -74.24207864785714) bank93630 +93631 POINT(41.35477021202502 -74.1313149125987) bank93631 +93632 POINT(39.960505449802426 -73.80872144861064) bank93632 +93633 POINT(40.37123417590149 -73.22697627446486) bank93633 +93634 POINT(41.256227427296864 -73.78036447057252) bank93634 +93635 POINT(39.90429722463777 -74.20970176568356) bank93635 +93636 POINT(39.78447855736743 -74.31718023958504) bank93636 +93637 POINT(40.211854862803236 -74.48973049540933) bank93637 +93638 POINT(41.33093293142885 -73.03928108497723) bank93638 +93639 POINT(41.18204457218961 -74.2651266591488) bank93639 +93640 POINT(41.498703481709 -73.04731535446545) bank93640 +93641 POINT(40.985023659224986 -74.28583674446102) bank93641 +93642 POINT(39.954343143875015 -73.50982757435251) bank93642 +93643 POINT(39.84122146350552 -74.02740554888176) bank93643 +93644 POINT(40.738823467552876 -74.1318177160837) bank93644 +93645 POINT(40.83418751431111 -73.69530845144281) bank93645 +93646 POINT(41.058990821476776 -73.58037075786065) bank93646 +93647 POINT(39.80435099147195 -74.808611713847) bank93647 +93648 POINT(41.6452876958217 -74.2248229863574) bank93648 +93649 POINT(40.45423028781318 -74.98400866072177) bank93649 +93650 POINT(41.20845419631679 -74.36237467245137) bank93650 +93651 POINT(41.02258091085005 -74.84922926440724) bank93651 +93652 POINT(39.83385663694706 -74.45414861014727) bank93652 +93653 POINT(41.22392223335216 -74.02403480228318) bank93653 +93654 POINT(41.562104600513734 -73.04692244006576) bank93654 +93655 POINT(39.77012265933015 -73.09827749643051) bank93655 +93656 POINT(40.58355806480572 -74.02057225424741) bank93656 +93657 POINT(41.034186189599 -73.1342417052941) bank93657 +93658 POINT(39.84357091344696 -73.17064707605759) bank93658 +93659 POINT(41.54725862227163 -74.57758393972365) bank93659 +93660 POINT(40.18799126701695 -73.61151650350959) bank93660 +93661 POINT(40.911080469281416 -74.42639287709156) bank93661 +93662 POINT(40.27615506874683 -73.60196429347177) bank93662 +93663 POINT(41.58966874874608 -74.9387312173429) bank93663 +93664 POINT(39.91270115784462 -74.28520942384245) bank93664 +93665 POINT(41.52941990170524 -73.76655349061906) bank93665 +93666 POINT(41.70208057007342 -73.08236859609832) bank93666 +93667 POINT(41.12466773995329 -73.91943542966473) bank93667 +93668 POINT(41.582546019625944 -74.20818684659271) bank93668 +93669 POINT(40.89012195111341 -73.58408080169649) bank93669 +93670 POINT(39.99476521701119 -74.02275527326825) bank93670 +93671 POINT(40.27207313485023 -74.46050454536359) bank93671 +93672 POINT(39.77327690537578 -74.3269081897913) bank93672 +93673 POINT(39.90327332362588 -73.50288424456072) bank93673 +93674 POINT(40.44419749954034 -73.21290313066083) bank93674 +93675 POINT(40.97376999533702 -73.54043040519683) bank93675 +93676 POINT(41.519254144054685 -73.03189810789733) bank93676 +93677 POINT(40.015217593936164 -73.06280662199349) bank93677 +93678 POINT(39.960404833268655 -74.88155668071559) bank93678 +93679 POINT(41.079736651064806 -73.62344017120421) bank93679 +93680 POINT(41.27112059785603 -74.79512386671855) bank93680 +93681 POINT(41.684017537959825 -73.04472530922024) bank93681 +93682 POINT(40.486725651899576 -73.153614671329) bank93682 +93683 POINT(40.543133926702865 -74.66225670446715) bank93683 +93684 POINT(40.07605462871778 -73.82447485470747) bank93684 +93685 POINT(40.70732309231665 -73.30707653858798) bank93685 +93686 POINT(41.64776486387766 -74.80406690913604) bank93686 +93687 POINT(41.11916045863627 -74.29229829139805) bank93687 +93688 POINT(39.77555317529915 -73.8414814629716) bank93688 +93689 POINT(39.81046196450881 -73.89341202557956) bank93689 +93690 POINT(40.951887417383766 -74.76666387703456) bank93690 +93691 POINT(40.29395569213314 -74.9661451770266) bank93691 +93692 POINT(40.22393057736129 -73.91469274303556) bank93692 +93693 POINT(41.01650492450725 -73.42518343147444) bank93693 +93694 POINT(41.315308602289704 -74.17600228233377) bank93694 +93695 POINT(41.337037887644 -74.50834186871337) bank93695 +93696 POINT(40.733980837732645 -74.30658126823327) bank93696 +93697 POINT(40.48903079051663 -74.62412350137016) bank93697 +93698 POINT(39.97595625873868 -74.96754078984029) bank93698 +93699 POINT(40.24253292137504 -74.41201265799803) bank93699 +93700 POINT(41.23547146334968 -74.38294517283627) bank93700 +93701 POINT(40.890479500404986 -74.01299249610976) bank93701 +93702 POINT(39.97403771556774 -73.03660706362245) bank93702 +93703 POINT(40.307416556979554 -73.59163037180605) bank93703 +93704 POINT(41.606981742599075 -74.78572433867298) bank93704 +93705 POINT(41.47943830847857 -74.8381979632153) bank93705 +93706 POINT(39.792012794735584 -74.91216843066844) bank93706 +93707 POINT(40.038616244761506 -73.53183465984307) bank93707 +93708 POINT(41.5139412054195 -74.82746303552791) bank93708 +93709 POINT(40.657417046201594 -74.07412725148058) bank93709 +93710 POINT(40.12538665260583 -74.66431378510669) bank93710 +93711 POINT(41.22129114422622 -74.07463605375276) bank93711 +93712 POINT(41.4990726780442 -73.05805908037203) bank93712 +93713 POINT(40.04202085262445 -73.77847684583948) bank93713 +93714 POINT(40.381254303231856 -74.33405377401351) bank93714 +93715 POINT(39.97680131160217 -74.26103709477367) bank93715 +93716 POINT(41.25763741523009 -75.00317392429822) bank93716 +93717 POINT(40.10902947947519 -74.22911294610299) bank93717 +93718 POINT(40.70378383128934 -74.45363096864007) bank93718 +93719 POINT(41.3988207327494 -74.18104298759799) bank93719 +93720 POINT(40.5617913234613 -74.59989856880455) bank93720 +93721 POINT(39.9890531284491 -73.8464919507106) bank93721 +93722 POINT(40.15860102214579 -74.52655224797164) bank93722 +93723 POINT(40.982400493819725 -74.92285187795503) bank93723 +93724 POINT(41.243962895595715 -74.13686234449204) bank93724 +93725 POINT(41.56717199442255 -73.33888901232532) bank93725 +93726 POINT(41.521532658191724 -73.92839548536148) bank93726 +93727 POINT(40.76907651181036 -73.48351466187253) bank93727 +93728 POINT(41.37061773008377 -74.6410258900221) bank93728 +93729 POINT(40.08733148811164 -73.44472986125018) bank93729 +93730 POINT(41.69048900906495 -73.79972105396902) bank93730 +93731 POINT(41.356416404253665 -74.59131832919262) bank93731 +93732 POINT(39.86919455880989 -74.73474648262133) bank93732 +93733 POINT(41.60428825890529 -74.79844743906153) bank93733 +93734 POINT(40.063521721174254 -74.01020842552262) bank93734 +93735 POINT(40.58808142596973 -73.22313477763795) bank93735 +93736 POINT(41.18899465021927 -74.10639738691036) bank93736 +93737 POINT(41.336604002624284 -74.70753718771309) bank93737 +93738 POINT(41.07592381675779 -73.89400174922515) bank93738 +93739 POINT(40.568684933337785 -74.04198480853381) bank93739 +93740 POINT(40.72879309435623 -73.37523212932219) bank93740 +93741 POINT(41.081673173891176 -74.40729969697472) bank93741 +93742 POINT(41.64102596942465 -73.66395085481511) bank93742 +93743 POINT(39.7565342442377 -73.02909078513483) bank93743 +93744 POINT(40.29490628534099 -73.53429244990782) bank93744 +93745 POINT(40.6922382260883 -74.08998150683878) bank93745 +93746 POINT(41.293428891287135 -74.27380690417635) bank93746 +93747 POINT(40.61358866456542 -74.30316910358717) bank93747 +93748 POINT(40.39208952320218 -73.3350252219481) bank93748 +93749 POINT(41.40135484086723 -74.63400098814796) bank93749 +93750 POINT(40.681834942389806 -73.52447094000354) bank93750 +93751 POINT(40.54949290727934 -74.28036826775813) bank93751 +93752 POINT(41.50440718221504 -73.41276232552669) bank93752 +93753 POINT(40.48767680693096 -73.33437833030999) bank93753 +93754 POINT(40.70784803179899 -74.63517639935529) bank93754 +93755 POINT(40.68169150104074 -74.84370002782273) bank93755 +93756 POINT(40.01717175135584 -74.10263588466235) bank93756 +93757 POINT(41.58784376718053 -74.4592460078503) bank93757 +93758 POINT(40.45428113894663 -73.49060712503237) bank93758 +93759 POINT(40.2193674918913 -73.40309497255774) bank93759 +93760 POINT(40.80128856926103 -73.99606858968914) bank93760 +93761 POINT(40.491563565596756 -74.0684866508318) bank93761 +93762 POINT(40.2754014183471 -74.87680823457745) bank93762 +93763 POINT(41.617856491707215 -74.38609873936792) bank93763 +93764 POINT(40.96985911990467 -73.91216285641298) bank93764 +93765 POINT(41.42795240648558 -74.11466752985174) bank93765 +93766 POINT(41.54760478025182 -73.52685441522011) bank93766 +93767 POINT(40.72155587267993 -74.2916300966879) bank93767 +93768 POINT(41.19141573982221 -73.68092028941436) bank93768 +93769 POINT(40.14055522974918 -74.86642989322993) bank93769 +93770 POINT(40.568896848576905 -74.24426538344129) bank93770 +93771 POINT(41.291163240471484 -74.60291193001412) bank93771 +93772 POINT(41.39447016632785 -73.21171877722678) bank93772 +93773 POINT(40.847553814724954 -74.18627103662789) bank93773 +93774 POINT(40.15189413261261 -74.20707708224715) bank93774 +93775 POINT(40.50658975123564 -73.286236644359) bank93775 +93776 POINT(40.395964008985715 -74.01271632961877) bank93776 +93777 POINT(41.40225019739267 -73.78754972457844) bank93777 +93778 POINT(40.46594482285456 -74.66422562673655) bank93778 +93779 POINT(40.01823882716751 -74.43022260600908) bank93779 +93780 POINT(41.70276942413341 -73.58665181779378) bank93780 +93781 POINT(41.40008712358464 -74.50789228934904) bank93781 +93782 POINT(40.908427144205916 -73.43895558171084) bank93782 +93783 POINT(41.239392656049546 -73.71230783525898) bank93783 +93784 POINT(39.89434037108997 -74.41176499755743) bank93784 +93785 POINT(41.09141423510503 -73.44454957841967) bank93785 +93786 POINT(39.83248299017563 -73.51244389508943) bank93786 +93787 POINT(40.891588378864924 -73.95979535413886) bank93787 +93788 POINT(41.50641892498167 -73.86537279153815) bank93788 +93789 POINT(40.14436046288642 -73.41763547750486) bank93789 +93790 POINT(41.53334142811701 -74.72940041230123) bank93790 +93791 POINT(41.178118459578684 -73.17011654618173) bank93791 +93792 POINT(41.36039234911542 -74.06871669242149) bank93792 +93793 POINT(40.63767917420325 -73.3194832996595) bank93793 +93794 POINT(39.953272974797386 -73.70386147093522) bank93794 +93795 POINT(40.790283283324634 -74.3919786604837) bank93795 +93796 POINT(41.35117330617834 -73.68195104343702) bank93796 +93797 POINT(40.85778794331581 -73.40180814248598) bank93797 +93798 POINT(40.480966707804164 -73.52393753334496) bank93798 +93799 POINT(41.24249571855032 -73.29469833265607) bank93799 +93800 POINT(39.86490042192427 -73.82087774828358) bank93800 +93801 POINT(40.14266106565342 -74.7448486290376) bank93801 +93802 POINT(40.844469881184565 -74.09919275822162) bank93802 +93803 POINT(39.986477514016244 -74.32934836496038) bank93803 +93804 POINT(40.9695443873702 -74.11882710595876) bank93804 +93805 POINT(39.894096079953925 -73.66431819923756) bank93805 +93806 POINT(40.50576574749618 -74.48802747349741) bank93806 +93807 POINT(40.243866310820145 -73.85060755773172) bank93807 +93808 POINT(40.447333625245314 -73.24764783633394) bank93808 +93809 POINT(40.43309409163215 -74.95767417928113) bank93809 +93810 POINT(39.9026582900625 -74.32000999668055) bank93810 +93811 POINT(40.22213226713906 -73.84345341192966) bank93811 +93812 POINT(40.10553473049453 -73.08838136993624) bank93812 +93813 POINT(41.40271512121625 -73.64572924318493) bank93813 +93814 POINT(40.664749874843736 -74.27539622822849) bank93814 +93815 POINT(40.93322085813341 -74.17782568869923) bank93815 +93816 POINT(39.727708657438214 -74.74395346690726) bank93816 +93817 POINT(40.25888532678325 -74.8603131962628) bank93817 +93818 POINT(41.15349869733162 -74.3774885407473) bank93818 +93819 POINT(41.59682060181347 -74.35869771793854) bank93819 +93820 POINT(40.921158313344364 -74.44616776424569) bank93820 +93821 POINT(41.411671826229245 -74.24393065468038) bank93821 +93822 POINT(41.43764447151851 -73.23915093294588) bank93822 +93823 POINT(40.613788145703396 -74.07335462630415) bank93823 +93824 POINT(40.91855101767301 -74.98031994800235) bank93824 +93825 POINT(40.61334050888986 -74.56937110215236) bank93825 +93826 POINT(41.50210787553746 -73.6665048479886) bank93826 +93827 POINT(41.13943525196051 -73.75732040517377) bank93827 +93828 POINT(41.69801242676356 -73.7610548711944) bank93828 +93829 POINT(41.26931484783283 -73.62795946760284) bank93829 +93830 POINT(41.160330223120724 -74.7498384074354) bank93830 +93831 POINT(40.36803889512151 -73.67534730533764) bank93831 +93832 POINT(41.07758701611098 -74.74498378593626) bank93832 +93833 POINT(39.718853041736814 -73.28880746054789) bank93833 +93834 POINT(41.364022130647626 -74.98419333903864) bank93834 +93835 POINT(41.17114477883106 -74.19915141145935) bank93835 +93836 POINT(39.9940033291483 -73.24210581163898) bank93836 +93837 POINT(41.61150024015077 -73.34839058577444) bank93837 +93838 POINT(41.33160695483006 -73.55415786036764) bank93838 +93839 POINT(39.98106407717304 -73.49600095464808) bank93839 +93840 POINT(40.419001191783465 -73.69280446171227) bank93840 +93841 POINT(40.104959190064186 -74.35524336397009) bank93841 +93842 POINT(40.33299691639546 -73.23557643771231) bank93842 +93843 POINT(39.91297983620296 -74.93929179646156) bank93843 +93844 POINT(40.22411049125858 -73.71308383678665) bank93844 +93845 POINT(40.94418756869955 -74.35004989045635) bank93845 +93846 POINT(41.526355208468964 -73.38568298727203) bank93846 +93847 POINT(40.25024620312338 -74.07700884899414) bank93847 +93848 POINT(41.45321629962861 -73.91558823932249) bank93848 +93849 POINT(40.652373638025466 -73.61908591436602) bank93849 +93850 POINT(40.75581105135374 -74.17896380883516) bank93850 +93851 POINT(40.86980906212138 -74.31776259913272) bank93851 +93852 POINT(40.323792503921204 -73.85953014129288) bank93852 +93853 POINT(39.870751739083325 -74.77857257110622) bank93853 +93854 POINT(40.02953524646207 -73.08992542407495) bank93854 +93855 POINT(41.52089074319436 -74.49042407605997) bank93855 +93856 POINT(40.26035678296359 -73.28146368700001) bank93856 +93857 POINT(40.607895889837216 -73.24443307681035) bank93857 +93858 POINT(40.53716512557269 -74.19071355832261) bank93858 +93859 POINT(41.1074684788255 -74.00361492641538) bank93859 +93860 POINT(40.82456278223341 -73.03095554127533) bank93860 +93861 POINT(39.93408164835144 -74.43065021927542) bank93861 +93862 POINT(39.75636976675788 -73.39404936692165) bank93862 +93863 POINT(41.45117735421366 -74.1810784454025) bank93863 +93864 POINT(40.44183987046959 -73.4174737842491) bank93864 +93865 POINT(39.77802288286563 -74.53317818262514) bank93865 +93866 POINT(40.79301055055288 -73.38973182072509) bank93866 +93867 POINT(40.99386989462756 -74.1069782135035) bank93867 +93868 POINT(39.95896679435969 -73.86694979967085) bank93868 +93869 POINT(40.90518220410321 -74.60248893207945) bank93869 +93870 POINT(40.798414413527155 -73.65525408471238) bank93870 +93871 POINT(40.44802657520064 -73.05370803276331) bank93871 +93872 POINT(41.28628783108548 -74.24170763681714) bank93872 +93873 POINT(41.01058293500794 -74.5670920155164) bank93873 +93874 POINT(40.497722710423375 -73.53931061197565) bank93874 +93875 POINT(41.52768329556483 -73.9042601869576) bank93875 +93876 POINT(39.87792665493976 -74.52660180591361) bank93876 +93877 POINT(40.40501084822417 -73.46050594934422) bank93877 +93878 POINT(41.66796818921144 -73.70520141392177) bank93878 +93879 POINT(40.42031442625782 -73.67431535899259) bank93879 +93880 POINT(39.96834577220437 -74.94470347272265) bank93880 +93881 POINT(41.02411921437846 -73.74226200976891) bank93881 +93882 POINT(41.43443071523036 -73.50605920830291) bank93882 +93883 POINT(40.85330133166778 -74.67514469368898) bank93883 +93884 POINT(39.963930447292 -74.44499280867414) bank93884 +93885 POINT(39.92731545817437 -74.77915374053558) bank93885 +93886 POINT(39.71714239964928 -73.22471040123348) bank93886 +93887 POINT(40.15135445098239 -73.77734659368218) bank93887 +93888 POINT(41.39053597891034 -74.20764857691886) bank93888 +93889 POINT(41.61832275108526 -74.37193421748637) bank93889 +93890 POINT(41.37890852000327 -74.46468809889446) bank93890 +93891 POINT(40.37708810855248 -73.92313862031303) bank93891 +93892 POINT(40.78116972762956 -73.74261941532619) bank93892 +93893 POINT(40.65967999831527 -73.84095958960343) bank93893 +93894 POINT(40.46700079320828 -73.48427751569167) bank93894 +93895 POINT(41.029756176090565 -74.40113115175805) bank93895 +93896 POINT(41.25679051248674 -73.07215615236709) bank93896 +93897 POINT(39.7230388090621 -74.28579719842385) bank93897 +93898 POINT(41.61003783478734 -74.72261298812661) bank93898 +93899 POINT(40.09870378918474 -74.9962491921892) bank93899 +93900 POINT(40.15505098153115 -74.06539984488748) bank93900 +93901 POINT(41.07483985098221 -73.01988091284844) bank93901 +93902 POINT(39.85044072642744 -73.6145572948875) bank93902 +93903 POINT(40.19390196087087 -74.5909755239404) bank93903 +93904 POINT(39.74011099155332 -73.76596263566358) bank93904 +93905 POINT(40.52796241772292 -73.65765102876107) bank93905 +93906 POINT(40.58042173353704 -73.13484943372451) bank93906 +93907 POINT(40.58483811196812 -73.5712289466866) bank93907 +93908 POINT(40.778791818750186 -73.10901983261006) bank93908 +93909 POINT(39.714512182162956 -74.68965934563008) bank93909 +93910 POINT(41.439317739687155 -73.42736000678414) bank93910 +93911 POINT(40.187590310053444 -73.2370810601999) bank93911 +93912 POINT(40.02497455368377 -74.65642132229213) bank93912 +93913 POINT(40.937744127383844 -74.7174047298779) bank93913 +93914 POINT(40.62933422517377 -73.51138912159628) bank93914 +93915 POINT(40.152510439125656 -73.90725137143652) bank93915 +93916 POINT(41.613374224496326 -73.68968255113948) bank93916 +93917 POINT(40.0761201515609 -73.19388482332377) bank93917 +93918 POINT(40.211134410654665 -73.70961962747705) bank93918 +93919 POINT(39.73462842354811 -74.43853340242303) bank93919 +93920 POINT(40.85775186456599 -74.56603111214197) bank93920 +93921 POINT(41.36333083710391 -74.53036220407928) bank93921 +93922 POINT(41.20697941888158 -73.45092501587622) bank93922 +93923 POINT(39.809353490564185 -73.6951721092614) bank93923 +93924 POINT(40.47960521067596 -73.84538587878235) bank93924 +93925 POINT(41.046287496147876 -73.01303061163763) bank93925 +93926 POINT(41.21555907904696 -73.08903396093183) bank93926 +93927 POINT(41.65167582337846 -74.97958242469666) bank93927 +93928 POINT(40.9742177529626 -73.71512650740364) bank93928 +93929 POINT(40.43132088303305 -74.51880514689756) bank93929 +93930 POINT(41.66148834037213 -74.95269389744449) bank93930 +93931 POINT(41.52411287058894 -74.42074308938561) bank93931 +93932 POINT(40.991112143218466 -74.01433141442705) bank93932 +93933 POINT(41.646220698386365 -73.1703987833336) bank93933 +93934 POINT(39.96185250592007 -73.0108346478207) bank93934 +93935 POINT(40.876652764787174 -73.11233139833352) bank93935 +93936 POINT(41.178109764074186 -73.62904558020298) bank93936 +93937 POINT(41.06932076249151 -74.63072497546261) bank93937 +93938 POINT(41.62740693657627 -74.53641915679675) bank93938 +93939 POINT(39.79035652142863 -73.39004484915931) bank93939 +93940 POINT(40.375097234252074 -73.5878215428237) bank93940 +93941 POINT(39.92714917403826 -73.87380926308731) bank93941 +93942 POINT(41.531190239634384 -74.28779742645598) bank93942 +93943 POINT(40.914155792383056 -74.94943931568449) bank93943 +93944 POINT(41.056185303337664 -73.84414197162873) bank93944 +93945 POINT(39.948227780790816 -74.30972862431389) bank93945 +93946 POINT(41.249350372513945 -74.65394208995248) bank93946 +93947 POINT(39.8198267237613 -74.8915610739118) bank93947 +93948 POINT(41.26963068843676 -73.82390753950874) bank93948 +93949 POINT(41.010092464504986 -74.09976468065824) bank93949 +93950 POINT(39.917257596885285 -74.52349708787948) bank93950 +93951 POINT(40.880101557227206 -74.09528018611701) bank93951 +93952 POINT(41.69726535493594 -74.01033640679915) bank93952 +93953 POINT(39.815028816833895 -73.99723817028777) bank93953 +93954 POINT(40.90742582792002 -74.10278592354572) bank93954 +93955 POINT(40.33109618934508 -73.17084010717038) bank93955 +93956 POINT(40.090038511984616 -73.77860776002521) bank93956 +93957 POINT(40.56681933829831 -73.32893435758848) bank93957 +93958 POINT(40.03372678933933 -73.76310553450114) bank93958 +93959 POINT(40.839703429268056 -74.630738946078) bank93959 +93960 POINT(39.887752088744186 -73.4005705513023) bank93960 +93961 POINT(41.27049854070481 -73.76878569326776) bank93961 +93962 POINT(41.55907559966702 -74.53853868602755) bank93962 +93963 POINT(39.98473535792778 -74.8524237736701) bank93963 +93964 POINT(41.44030530891196 -73.1900962673742) bank93964 +93965 POINT(41.05077512119654 -73.56153465058863) bank93965 +93966 POINT(40.448989410617656 -73.4297308756177) bank93966 +93967 POINT(41.61600319508377 -74.99893305518793) bank93967 +93968 POINT(40.696232200625666 -74.26631046468809) bank93968 +93969 POINT(41.57014306414384 -74.65812033574569) bank93969 +93970 POINT(40.185488547164795 -74.44065017515814) bank93970 +93971 POINT(40.841182885283175 -74.52595806754559) bank93971 +93972 POINT(40.86020678917677 -74.60150275104377) bank93972 +93973 POINT(40.42851055729312 -73.57636813212156) bank93973 +93974 POINT(41.25381838438262 -73.72982341381072) bank93974 +93975 POINT(41.53333941153982 -74.33516512427752) bank93975 +93976 POINT(41.10770878881474 -74.87890642201694) bank93976 +93977 POINT(41.04648467932696 -73.43739686373748) bank93977 +93978 POINT(39.72629818827785 -74.07246241645457) bank93978 +93979 POINT(40.96921266646236 -73.11118054446123) bank93979 +93980 POINT(39.746396078966235 -73.10016055149579) bank93980 +93981 POINT(40.326576897161914 -73.78061192825766) bank93981 +93982 POINT(40.23487175847673 -74.68705161590654) bank93982 +93983 POINT(41.71156641255756 -74.01038905483107) bank93983 +93984 POINT(41.3817110595535 -73.88888534507804) bank93984 +93985 POINT(41.14308655389318 -73.12227111351017) bank93985 +93986 POINT(41.09780604362369 -74.13712168297178) bank93986 +93987 POINT(41.55033884893866 -74.87587340734655) bank93987 +93988 POINT(41.05882820807008 -74.86123698794525) bank93988 +93989 POINT(40.79050476609725 -73.09271477515837) bank93989 +93990 POINT(41.5927282036337 -74.73193155082804) bank93990 +93991 POINT(39.98649336243935 -73.34496000471444) bank93991 +93992 POINT(41.06202700906036 -74.49307875118743) bank93992 +93993 POINT(39.79224636054353 -74.90584765474344) bank93993 +93994 POINT(39.955925800712535 -74.76694235822887) bank93994 +93995 POINT(41.60329683479493 -73.40697999436446) bank93995 +93996 POINT(40.602470767939145 -73.34765735926483) bank93996 +93997 POINT(39.925338935023746 -73.28689106165908) bank93997 +93998 POINT(40.60644614895054 -73.21037956656663) bank93998 +93999 POINT(41.421022696380035 -73.00931916940554) bank93999 +94000 POINT(41.48174273470408 -73.0348199157503) bank94000 +94001 POINT(41.54129953745158 -74.68375959760631) bank94001 +94002 POINT(40.83029103033304 -73.86919774918157) bank94002 +94003 POINT(40.5261982855054 -74.7048360310457) bank94003 +94004 POINT(41.69123579580392 -73.8332881012831) bank94004 +94005 POINT(41.6185225024983 -73.86188620598891) bank94005 +94006 POINT(41.71074476459131 -74.81558206981856) bank94006 +94007 POINT(40.91049575587954 -74.77269625469711) bank94007 +94008 POINT(41.5359242276239 -74.46051178627837) bank94008 +94009 POINT(40.23066970360265 -73.84890491464529) bank94009 +94010 POINT(40.081918969607706 -74.05394201965935) bank94010 +94011 POINT(39.93371486111085 -73.95251030128459) bank94011 +94012 POINT(40.259134152040424 -74.20397489469126) bank94012 +94013 POINT(41.414636559303396 -73.68062815319499) bank94013 +94014 POINT(41.451710763426135 -73.72081012818383) bank94014 +94015 POINT(41.018425839123125 -74.74770657902191) bank94015 +94016 POINT(40.780494331247915 -73.44345108578021) bank94016 +94017 POINT(39.76939317385727 -73.43227817664871) bank94017 +94018 POINT(41.66096498385513 -74.44450182429125) bank94018 +94019 POINT(39.872538682816156 -74.68696117622638) bank94019 +94020 POINT(39.96634444331112 -74.57631604813886) bank94020 +94021 POINT(40.45742252238788 -74.04365386967957) bank94021 +94022 POINT(41.18042026568999 -74.52056946645632) bank94022 +94023 POINT(40.29292097611168 -73.4947089625691) bank94023 +94024 POINT(41.36197419783182 -73.44821000362309) bank94024 +94025 POINT(41.252029951221566 -73.18844774991112) bank94025 +94026 POINT(40.04252640248909 -73.2641608303155) bank94026 +94027 POINT(40.27665994074205 -73.04599998480482) bank94027 +94028 POINT(41.03020015554731 -73.52885251180919) bank94028 +94029 POINT(41.495082844157665 -74.10192870148273) bank94029 +94030 POINT(40.99222852915138 -74.75040290008491) bank94030 +94031 POINT(39.77262312219361 -74.0073576411396) bank94031 +94032 POINT(40.010373504821125 -74.4454890351819) bank94032 +94033 POINT(40.471575278353576 -73.50762480118678) bank94033 +94034 POINT(41.65014655089051 -73.12268311498165) bank94034 +94035 POINT(40.1953548145469 -73.35955219737396) bank94035 +94036 POINT(40.480927054795856 -73.62921036682256) bank94036 +94037 POINT(41.36715388429076 -73.64918857558901) bank94037 +94038 POINT(39.90875338695886 -73.15037258741756) bank94038 +94039 POINT(40.76107078145483 -73.89757244244956) bank94039 +94040 POINT(40.66452126059545 -73.94618020677179) bank94040 +94041 POINT(40.59476121615888 -74.2847127151686) bank94041 +94042 POINT(40.815730925337846 -73.82017841429449) bank94042 +94043 POINT(39.946409152335654 -73.4249393854914) bank94043 +94044 POINT(40.97398660439217 -74.90368509873645) bank94044 +94045 POINT(41.24167137075995 -73.63297026914415) bank94045 +94046 POINT(40.43791967927711 -74.48156572033103) bank94046 +94047 POINT(40.92787642050471 -73.13222135449429) bank94047 +94048 POINT(40.390005315340446 -74.519301438638) bank94048 +94049 POINT(40.43155875809556 -74.2845616456393) bank94049 +94050 POINT(40.99843327939689 -74.10652842793587) bank94050 +94051 POINT(41.41496737141965 -73.76651898590929) bank94051 +94052 POINT(41.66782053151691 -74.48810835471723) bank94052 +94053 POINT(39.73354188692653 -73.90696103147539) bank94053 +94054 POINT(40.34755535702641 -73.19120602857734) bank94054 +94055 POINT(39.96826297586999 -74.35174327530305) bank94055 +94056 POINT(39.7190314855567 -73.15363428987014) bank94056 +94057 POINT(39.73551195534075 -73.510363296856) bank94057 +94058 POINT(41.275159665557574 -73.13056272243878) bank94058 +94059 POINT(41.48639320858194 -73.51816650204232) bank94059 +94060 POINT(41.00122688705737 -73.28086044784646) bank94060 +94061 POINT(40.01495630321086 -73.6145394337638) bank94061 +94062 POINT(40.167147235371395 -73.09514617350867) bank94062 +94063 POINT(40.94040013803514 -74.77893725227908) bank94063 +94064 POINT(40.71116628184438 -73.48652430984097) bank94064 +94065 POINT(41.56558765302091 -73.96164001807378) bank94065 +94066 POINT(41.214449770144455 -73.75970735447618) bank94066 +94067 POINT(40.00836267077156 -73.35127226967461) bank94067 +94068 POINT(41.41080558518818 -73.45277827035896) bank94068 +94069 POINT(41.66343857922292 -74.00208411934766) bank94069 +94070 POINT(41.45940265566539 -73.55504343037344) bank94070 +94071 POINT(39.943866600468716 -74.00563699068384) bank94071 +94072 POINT(41.67659008812025 -73.01507238875261) bank94072 +94073 POINT(40.86708856787295 -74.86528902152261) bank94073 +94074 POINT(40.55726095514159 -73.55375690394472) bank94074 +94075 POINT(40.682396867426554 -74.0612480571211) bank94075 +94076 POINT(39.865346859049865 -73.23818040604394) bank94076 +94077 POINT(40.639462475768084 -74.96695577037438) bank94077 +94078 POINT(40.03474564982705 -73.55262353970066) bank94078 +94079 POINT(39.945421679995015 -73.38684048649104) bank94079 +94080 POINT(41.30348749868993 -74.80811096521832) bank94080 +94081 POINT(40.88786216959762 -73.52291209115468) bank94081 +94082 POINT(40.57180407614129 -74.33569873401804) bank94082 +94083 POINT(40.01279765699779 -74.42358864523744) bank94083 +94084 POINT(40.13852738331025 -73.09870058700773) bank94084 +94085 POINT(41.5210907604941 -73.35342270550912) bank94085 +94086 POINT(40.33324684044551 -74.99955794074818) bank94086 +94087 POINT(40.22919917246659 -73.47652921903857) bank94087 +94088 POINT(40.92060943854443 -74.80576655728464) bank94088 +94089 POINT(41.401507429744505 -74.8423218359557) bank94089 +94090 POINT(41.12269307508818 -73.2514345538604) bank94090 +94091 POINT(40.202402669757944 -74.04121390965746) bank94091 +94092 POINT(40.98925401950125 -74.07078578107163) bank94092 +94093 POINT(41.568408318534104 -74.90292899934427) bank94093 +94094 POINT(40.07035169116614 -74.16853715711672) bank94094 +94095 POINT(39.93653449934332 -73.22713143402393) bank94095 +94096 POINT(39.9177454821575 -73.53119016430654) bank94096 +94097 POINT(41.069013583196956 -73.69140671849648) bank94097 +94098 POINT(41.202405385688394 -73.792102037152) bank94098 +94099 POINT(41.12069102909519 -73.08917141821274) bank94099 +94100 POINT(39.80045245665104 -73.46714352967575) bank94100 +94101 POINT(40.163599528218256 -73.23307809368272) bank94101 +94102 POINT(40.05687505219121 -73.87234523674205) bank94102 +94103 POINT(41.051953286444636 -74.61978360695265) bank94103 +94104 POINT(40.147846280174036 -73.31217398844184) bank94104 +94105 POINT(40.404472989595824 -74.53079848043723) bank94105 +94106 POINT(40.01842552371555 -73.39720270772068) bank94106 +94107 POINT(41.60290542657523 -73.1850997576309) bank94107 +94108 POINT(40.187764702437846 -74.18064839921601) bank94108 +94109 POINT(41.321471113120985 -73.25962820609523) bank94109 +94110 POINT(40.03922942006779 -74.03140876596481) bank94110 +94111 POINT(40.95324009956324 -74.73904763420336) bank94111 +94112 POINT(40.08865617227668 -73.52284778339167) bank94112 +94113 POINT(41.70619745111552 -74.19616505834739) bank94113 +94114 POINT(40.81335974901795 -73.13824186657183) bank94114 +94115 POINT(41.64977024237243 -74.27791669021022) bank94115 +94116 POINT(40.84041692955959 -73.04245628476328) bank94116 +94117 POINT(40.46006333813924 -74.061208963083) bank94117 +94118 POINT(40.27732771986374 -74.6188315493643) bank94118 +94119 POINT(40.32119882313187 -73.20508464059819) bank94119 +94120 POINT(40.0171269980245 -74.26334641543568) bank94120 +94121 POINT(41.083837058325194 -73.59178004085261) bank94121 +94122 POINT(41.38443252081846 -73.89956756685022) bank94122 +94123 POINT(41.209112480997135 -74.21036499659542) bank94123 +94124 POINT(40.66232112007584 -74.04115056209466) bank94124 +94125 POINT(41.13871325353847 -73.76944593765555) bank94125 +94126 POINT(41.274457053734096 -73.0645947656083) bank94126 +94127 POINT(40.85747587589204 -74.2187568670555) bank94127 +94128 POINT(40.58208020499511 -74.97690638020451) bank94128 +94129 POINT(40.49036226905771 -73.40779942027287) bank94129 +94130 POINT(41.56451863125168 -74.61826339128682) bank94130 +94131 POINT(41.346453133136876 -74.23892315454832) bank94131 +94132 POINT(40.72601169909398 -73.95941367922947) bank94132 +94133 POINT(40.13137986345664 -74.42557515163782) bank94133 +94134 POINT(40.4351250602892 -73.87089067260908) bank94134 +94135 POINT(41.2717896730405 -74.8291429905987) bank94135 +94136 POINT(40.93557612538207 -73.18471138192898) bank94136 +94137 POINT(40.749136288934345 -74.51989833052629) bank94137 +94138 POINT(41.16125068391176 -74.8442662727261) bank94138 +94139 POINT(41.21014342935849 -73.48163828334718) bank94139 +94140 POINT(41.23962752170113 -73.76840314824666) bank94140 +94141 POINT(41.53874181097876 -73.48794365076591) bank94141 +94142 POINT(41.36792941092807 -74.8653113408693) bank94142 +94143 POINT(39.86535700866851 -73.56855142396834) bank94143 +94144 POINT(40.1508042240272 -74.61221535249759) bank94144 +94145 POINT(39.93722449868537 -73.9738258646738) bank94145 +94146 POINT(39.94953971818427 -74.49774975853572) bank94146 +94147 POINT(40.33771792314657 -73.43010425741952) bank94147 +94148 POINT(41.43416144919157 -74.69711240766655) bank94148 +94149 POINT(41.218611724186346 -74.44949222185906) bank94149 +94150 POINT(40.41069732569931 -74.9495202894123) bank94150 +94151 POINT(41.227962502678565 -74.15870549578925) bank94151 +94152 POINT(39.77147435746312 -73.20847988513921) bank94152 +94153 POINT(40.89047411556745 -73.28440332578246) bank94153 +94154 POINT(41.5922693120115 -73.74113953590559) bank94154 +94155 POINT(40.35941019309479 -73.44151016777906) bank94155 +94156 POINT(40.149907799492745 -74.2948967624907) bank94156 +94157 POINT(40.03240664137576 -74.44151368767372) bank94157 +94158 POINT(41.475942244271586 -73.48497234283634) bank94158 +94159 POINT(40.67807991416809 -73.0414040653925) bank94159 +94160 POINT(41.65093320524612 -73.09926483132634) bank94160 +94161 POINT(40.457195825304446 -74.76691275644669) bank94161 +94162 POINT(40.66661089252467 -74.63564787300908) bank94162 +94163 POINT(41.625280655056386 -73.7129811698159) bank94163 +94164 POINT(41.182095970018125 -73.91699152134575) bank94164 +94165 POINT(40.35409420957896 -73.57436954908745) bank94165 +94166 POINT(40.24554524799446 -74.95954267139594) bank94166 +94167 POINT(41.24065539594008 -73.873596103972) bank94167 +94168 POINT(39.766226370824704 -73.99125962763421) bank94168 +94169 POINT(41.172304691905076 -74.80569632917816) bank94169 +94170 POINT(41.66111871092323 -74.28610767275637) bank94170 +94171 POINT(39.98760234717807 -74.63779538899836) bank94171 +94172 POINT(39.75668360634268 -73.75376815191606) bank94172 +94173 POINT(40.50417805615417 -73.92204500096354) bank94173 +94174 POINT(40.250128694328794 -73.22657551116464) bank94174 +94175 POINT(39.73083822165674 -74.61675330345369) bank94175 +94176 POINT(40.8883122167458 -73.7428163171464) bank94176 +94177 POINT(41.42118312065156 -73.88152322507854) bank94177 +94178 POINT(39.92506164837008 -74.70572695071397) bank94178 +94179 POINT(40.90468903061571 -73.45834527015111) bank94179 +94180 POINT(40.955302829090996 -74.90305340506524) bank94180 +94181 POINT(40.91219654432174 -74.34047838430422) bank94181 +94182 POINT(39.77317906396096 -74.0671949250247) bank94182 +94183 POINT(41.53929702017531 -73.42891557191227) bank94183 +94184 POINT(41.10209524201421 -74.28901241048733) bank94184 +94185 POINT(40.97150246917486 -74.87948705283648) bank94185 +94186 POINT(41.09145134368962 -73.60912018640306) bank94186 +94187 POINT(39.71704820091674 -73.75161059857201) bank94187 +94188 POINT(41.495429639478346 -73.07311582170962) bank94188 +94189 POINT(41.2913424251664 -73.2643331328621) bank94189 +94190 POINT(40.26998114864309 -74.52938103870757) bank94190 +94191 POINT(40.65411780313037 -74.5783969109088) bank94191 +94192 POINT(41.711239332568475 -73.01971228489981) bank94192 +94193 POINT(41.56915697647762 -73.6915729595029) bank94193 +94194 POINT(40.451561167784405 -74.38812160937734) bank94194 +94195 POINT(41.17988208378647 -74.9080382492663) bank94195 +94196 POINT(41.59859908165409 -74.70497120135278) bank94196 +94197 POINT(40.66323795939214 -73.67294433203838) bank94197 +94198 POINT(41.18683519037849 -74.19062904226092) bank94198 +94199 POINT(40.751214262433066 -73.84142221165763) bank94199 +94200 POINT(41.02708294455558 -74.20968604215885) bank94200 +94201 POINT(40.44477478143956 -73.27183982053461) bank94201 +94202 POINT(41.62072719664693 -74.12059356884119) bank94202 +94203 POINT(40.80283677804278 -73.77824055099303) bank94203 +94204 POINT(41.008189206250506 -73.39791635829873) bank94204 +94205 POINT(40.67038238751102 -73.8518181660434) bank94205 +94206 POINT(39.86488898437604 -73.79998876024604) bank94206 +94207 POINT(40.96265798702198 -74.1144421945913) bank94207 +94208 POINT(40.44565333262658 -74.33745177995065) bank94208 +94209 POINT(39.90831659408951 -74.26181331573146) bank94209 +94210 POINT(41.17137304556612 -74.67160192067436) bank94210 +94211 POINT(40.02014523405707 -73.43771663441431) bank94211 +94212 POINT(40.001131512923145 -73.56936032891514) bank94212 +94213 POINT(40.32371705576778 -74.3166172667988) bank94213 +94214 POINT(41.51190454424887 -74.77417484937574) bank94214 +94215 POINT(40.34123498059767 -73.83647798133981) bank94215 +94216 POINT(40.65634345891216 -73.27881879677192) bank94216 +94217 POINT(40.21757351321936 -74.064318824343) bank94217 +94218 POINT(41.39387418233017 -74.51602384173373) bank94218 +94219 POINT(41.58203809170186 -73.46414180488699) bank94219 +94220 POINT(41.397449314582644 -74.01203132643602) bank94220 +94221 POINT(40.71568950827184 -74.52636048581991) bank94221 +94222 POINT(41.22217672702719 -74.06270369199824) bank94222 +94223 POINT(41.04285323407738 -74.46181972875638) bank94223 +94224 POINT(41.169197589706094 -74.99474028453702) bank94224 +94225 POINT(40.49080898995086 -74.78146947385089) bank94225 +94226 POINT(41.29694884855703 -74.4164150977506) bank94226 +94227 POINT(41.38677893354587 -74.4187249514339) bank94227 +94228 POINT(41.4683981857176 -73.42641902766078) bank94228 +94229 POINT(41.006285918001595 -73.75465677385432) bank94229 +94230 POINT(40.77135600946952 -73.63185454751829) bank94230 +94231 POINT(39.94073071428587 -74.75442735157992) bank94231 +94232 POINT(40.77556455005547 -74.83136491312519) bank94232 +94233 POINT(40.20417716159362 -73.90687065482545) bank94233 +94234 POINT(40.51646357298197 -73.44058194908872) bank94234 +94235 POINT(39.77577320398273 -73.92704428489019) bank94235 +94236 POINT(40.67075190309807 -73.21063515280326) bank94236 +94237 POINT(41.682131742480045 -74.12550067006882) bank94237 +94238 POINT(39.79342372809094 -74.954528054354) bank94238 +94239 POINT(40.90813558971165 -73.67757957522208) bank94239 +94240 POINT(40.96439279071153 -74.18901902215104) bank94240 +94241 POINT(40.34649523303465 -73.46173321604768) bank94241 +94242 POINT(41.52930586756141 -74.55068881232125) bank94242 +94243 POINT(40.27976442845667 -74.54511417061751) bank94243 +94244 POINT(40.8757369655238 -74.24497806067569) bank94244 +94245 POINT(40.375607778772725 -74.42750449379197) bank94245 +94246 POINT(40.97002464815195 -73.64832842827195) bank94246 +94247 POINT(40.97277364134155 -74.26748992727244) bank94247 +94248 POINT(39.915267011014336 -74.76345242770991) bank94248 +94249 POINT(40.62849665161214 -73.90946174765575) bank94249 +94250 POINT(41.399291123334166 -74.67637023952838) bank94250 +94251 POINT(41.001338712021095 -73.5624926709426) bank94251 +94252 POINT(41.444738345759916 -74.70062062217863) bank94252 +94253 POINT(39.87671213444239 -74.90404246362623) bank94253 +94254 POINT(40.473067441898344 -73.26717111143303) bank94254 +94255 POINT(41.07175256101241 -74.48717350170898) bank94255 +94256 POINT(41.2384137256976 -74.70583264818472) bank94256 +94257 POINT(39.72351725046836 -73.92246270144463) bank94257 +94258 POINT(40.53418913883058 -73.32885805859875) bank94258 +94259 POINT(40.55462082791577 -73.96636288478577) bank94259 +94260 POINT(40.2442325509281 -74.32245350620387) bank94260 +94261 POINT(41.70043588554951 -74.3860116374487) bank94261 +94262 POINT(41.10420737798874 -74.6573508680195) bank94262 +94263 POINT(41.45942189098765 -74.82962910085845) bank94263 +94264 POINT(40.437397842448846 -73.12557641297737) bank94264 +94265 POINT(39.86043512827148 -73.46200092804231) bank94265 +94266 POINT(39.978683473794064 -73.4582908681984) bank94266 +94267 POINT(41.13345403844845 -73.58170995419061) bank94267 +94268 POINT(40.35075582482158 -73.82938584815217) bank94268 +94269 POINT(40.0784564698521 -73.88211375802764) bank94269 +94270 POINT(40.520062565346116 -74.2682826850156) bank94270 +94271 POINT(39.85659751458349 -74.5150117624799) bank94271 +94272 POINT(41.223106844896 -74.83453307013922) bank94272 +94273 POINT(41.61424948879073 -73.61369592012664) bank94273 +94274 POINT(39.71754402929452 -74.0587598808379) bank94274 +94275 POINT(40.87630643626261 -74.9881461784802) bank94275 +94276 POINT(41.497001555233005 -73.23948752633386) bank94276 +94277 POINT(40.344294042921234 -73.87109877985901) bank94277 +94278 POINT(40.51504842790248 -74.32691089700535) bank94278 +94279 POINT(40.93676977420904 -73.20906236606577) bank94279 +94280 POINT(40.264452552728244 -74.43311691725363) bank94280 +94281 POINT(39.93121252089998 -73.24421107880931) bank94281 +94282 POINT(40.243100389216345 -73.96256732162108) bank94282 +94283 POINT(41.05444299714054 -74.4020143041752) bank94283 +94284 POINT(39.90868801453428 -73.25579076015772) bank94284 +94285 POINT(41.341147764697745 -73.80736702966738) bank94285 +94286 POINT(40.60131590698518 -74.17099564638494) bank94286 +94287 POINT(41.35355365010695 -74.54973536601264) bank94287 +94288 POINT(40.94887787320552 -73.3817177674802) bank94288 +94289 POINT(41.69250888332313 -74.1407257618806) bank94289 +94290 POINT(39.74350825891935 -74.23641754263572) bank94290 +94291 POINT(40.78165900145895 -73.69122986775258) bank94291 +94292 POINT(40.902584326743884 -74.02994480270665) bank94292 +94293 POINT(40.24862883670098 -74.85994527758909) bank94293 +94294 POINT(41.587377402232356 -73.18399382478948) bank94294 +94295 POINT(40.69937286570627 -73.58226960108317) bank94295 +94296 POINT(41.22437696415792 -74.24968542495863) bank94296 +94297 POINT(39.94070640991882 -74.82982292555184) bank94297 +94298 POINT(41.15111129196349 -74.96167125827728) bank94298 +94299 POINT(40.41133186270552 -74.22273135284561) bank94299 +94300 POINT(39.96272794507535 -74.89111970671514) bank94300 +94301 POINT(41.269141977610246 -73.50872691670803) bank94301 +94302 POINT(40.118796777685674 -74.91259754227725) bank94302 +94303 POINT(40.282616442543755 -73.91259107673457) bank94303 +94304 POINT(41.14427882236986 -74.93104077956704) bank94304 +94305 POINT(39.7654913669275 -73.3074845576014) bank94305 +94306 POINT(40.19996017985674 -74.67868057495801) bank94306 +94307 POINT(41.09600329460872 -73.99905985236946) bank94307 +94308 POINT(39.77691933078623 -73.11937261263326) bank94308 +94309 POINT(41.13660661760943 -74.81870390098811) bank94309 +94310 POINT(40.260391768127505 -73.52047109311604) bank94310 +94311 POINT(40.86157698822417 -73.01484134831435) bank94311 +94312 POINT(41.30373060501771 -74.61957889637492) bank94312 +94313 POINT(40.0178472752271 -74.52847787408191) bank94313 +94314 POINT(39.72401454058675 -74.39366200839348) bank94314 +94315 POINT(40.31758819033779 -73.31469540769774) bank94315 +94316 POINT(41.626839845977194 -73.72756105618605) bank94316 +94317 POINT(41.500625212922245 -73.1756424337741) bank94317 +94318 POINT(40.95072504691064 -74.42023018777124) bank94318 +94319 POINT(40.288093940730725 -73.02399605700316) bank94319 +94320 POINT(40.90508267648971 -73.4460051975467) bank94320 +94321 POINT(40.70216100288162 -74.39997572838811) bank94321 +94322 POINT(39.99639218850514 -74.32455435320239) bank94322 +94323 POINT(40.81368623487015 -73.08098623437343) bank94323 +94324 POINT(39.88185925013273 -74.97476109287769) bank94324 +94325 POINT(39.79802157463097 -74.46876341775959) bank94325 +94326 POINT(41.50547428890636 -74.3133304292185) bank94326 +94327 POINT(40.97624008219131 -73.54159639939304) bank94327 +94328 POINT(41.223243936739436 -74.909112067142) bank94328 +94329 POINT(41.34830866032659 -73.1617497446258) bank94329 +94330 POINT(41.07507575136648 -73.1715628004866) bank94330 +94331 POINT(40.621020573257226 -73.9497948331491) bank94331 +94332 POINT(41.708025876609575 -74.73356552745176) bank94332 +94333 POINT(41.36831877454369 -73.11649188886774) bank94333 +94334 POINT(40.93591437633389 -73.55547823587676) bank94334 +94335 POINT(40.63843365154545 -74.98657765382278) bank94335 +94336 POINT(41.37436688265441 -73.69737860394181) bank94336 +94337 POINT(40.83496048471011 -74.2862948381763) bank94337 +94338 POINT(39.90657464142477 -73.54195467841363) bank94338 +94339 POINT(41.35103044985825 -73.12313401387729) bank94339 +94340 POINT(40.191729331432995 -74.63049186620076) bank94340 +94341 POINT(39.94890590979399 -73.05984318774308) bank94341 +94342 POINT(40.614397938681535 -73.29178375968286) bank94342 +94343 POINT(41.51030010776715 -74.16565110552187) bank94343 +94344 POINT(41.55881669181656 -73.83181129873519) bank94344 +94345 POINT(40.724001620464655 -73.13829339576479) bank94345 +94346 POINT(41.636854399297874 -73.64776897954658) bank94346 +94347 POINT(40.18999104699423 -74.96497571338027) bank94347 +94348 POINT(40.49790604070405 -73.55831996758454) bank94348 +94349 POINT(39.75380698658533 -73.55042554219621) bank94349 +94350 POINT(40.89230788602144 -74.20418516876796) bank94350 +94351 POINT(41.57617210071128 -74.9867084416772) bank94351 +94352 POINT(40.68664482958099 -73.09247626676243) bank94352 +94353 POINT(40.26409562657558 -74.41214403854367) bank94353 +94354 POINT(41.037665920739144 -74.82732363950892) bank94354 +94355 POINT(41.37323417007054 -73.45605291437549) bank94355 +94356 POINT(40.189709583354855 -73.92047585860192) bank94356 +94357 POINT(41.000616126645056 -74.88194999893679) bank94357 +94358 POINT(40.30133472666867 -74.44686994741177) bank94358 +94359 POINT(40.3475844895109 -73.89611211273788) bank94359 +94360 POINT(40.908974498426694 -74.35519050287402) bank94360 +94361 POINT(40.303655024686456 -73.19899656656341) bank94361 +94362 POINT(41.503139575394016 -73.20531490854869) bank94362 +94363 POINT(39.90186236976019 -74.11822489140373) bank94363 +94364 POINT(41.24227661636578 -74.91233849806419) bank94364 +94365 POINT(39.84756239181236 -74.7828412342081) bank94365 +94366 POINT(41.17723999637416 -73.56305460641373) bank94366 +94367 POINT(40.432939263537456 -73.9402312223056) bank94367 +94368 POINT(40.04981502980727 -73.72359337324424) bank94368 +94369 POINT(40.221411446430615 -74.2324179046474) bank94369 +94370 POINT(41.676714550148475 -74.62407534332297) bank94370 +94371 POINT(40.58109829608394 -73.61325731917353) bank94371 +94372 POINT(40.70980162575091 -74.810150334488) bank94372 +94373 POINT(41.148806059519394 -73.36760073561935) bank94373 +94374 POINT(39.7149781627575 -74.91016942988945) bank94374 +94375 POINT(41.21176924429601 -73.94713994796844) bank94375 +94376 POINT(41.24081411022444 -73.96608726302577) bank94376 +94377 POINT(41.22933486182034 -73.72370833446894) bank94377 +94378 POINT(40.29702527762421 -74.3713716217988) bank94378 +94379 POINT(41.71262576532945 -73.83496413440844) bank94379 +94380 POINT(40.01747574130698 -74.66251077225935) bank94380 +94381 POINT(40.87733475713554 -73.576137009954) bank94381 +94382 POINT(40.99210287624256 -74.58431625853667) bank94382 +94383 POINT(40.59103499798185 -74.64472619547851) bank94383 +94384 POINT(41.19697117043805 -73.03443665886371) bank94384 +94385 POINT(41.399567558117845 -74.41331761142183) bank94385 +94386 POINT(40.24132539769253 -74.41868425386816) bank94386 +94387 POINT(41.514823333494846 -73.13253450790425) bank94387 +94388 POINT(40.28824163081875 -73.14940363938356) bank94388 +94389 POINT(40.681110642671456 -74.47288878080286) bank94389 +94390 POINT(40.33547079951553 -73.39172436806604) bank94390 +94391 POINT(40.651786442719434 -74.31830381806469) bank94391 +94392 POINT(40.51007789054412 -73.3362217128241) bank94392 +94393 POINT(41.23705326583996 -74.20595227370735) bank94393 +94394 POINT(40.51570173963107 -74.87738643952095) bank94394 +94395 POINT(41.189181924462645 -73.56028090578376) bank94395 +94396 POINT(40.90433252918656 -73.49450267933285) bank94396 +94397 POINT(40.70722337627089 -73.78841552019097) bank94397 +94398 POINT(39.73924015550031 -74.37107133444333) bank94398 +94399 POINT(40.223366759902156 -73.11676869219316) bank94399 +94400 POINT(40.94850489669897 -74.45190963819671) bank94400 +94401 POINT(40.893368614082235 -74.68133364242634) bank94401 +94402 POINT(41.614245513096705 -74.5770713599706) bank94402 +94403 POINT(39.81615847216582 -74.01165274837264) bank94403 +94404 POINT(40.449256618218016 -73.61961030115062) bank94404 +94405 POINT(41.486847711971016 -73.82319749367208) bank94405 +94406 POINT(41.225623627320175 -73.06988254565826) bank94406 +94407 POINT(41.05235310015424 -74.03367991161298) bank94407 +94408 POINT(39.76535737320803 -74.64702010321881) bank94408 +94409 POINT(41.016867108823796 -73.01417033274171) bank94409 +94410 POINT(40.98050141303102 -73.91723721821654) bank94410 +94411 POINT(39.730990098242124 -74.62956146912538) bank94411 +94412 POINT(41.478759116386094 -74.03175599339319) bank94412 +94413 POINT(41.67583148242516 -74.5292234786006) bank94413 +94414 POINT(40.2781690946183 -73.3957352885625) bank94414 +94415 POINT(40.80748936618291 -73.36796511028416) bank94415 +94416 POINT(40.002261241913416 -73.78391734686429) bank94416 +94417 POINT(40.42484375277273 -73.30890619080307) bank94417 +94418 POINT(41.580680049252614 -74.89569460728984) bank94418 +94419 POINT(39.72085235242659 -73.2408289636372) bank94419 +94420 POINT(40.14951479842018 -73.13607481165474) bank94420 +94421 POINT(40.91758029898354 -74.22297904111106) bank94421 +94422 POINT(40.096769991842145 -73.69842905159898) bank94422 +94423 POINT(39.744084181233475 -74.85535770797831) bank94423 +94424 POINT(40.5118965575966 -74.45328560862217) bank94424 +94425 POINT(41.48770646536693 -73.52534540102823) bank94425 +94426 POINT(39.86254538272656 -74.64636761525752) bank94426 +94427 POINT(40.93633957213125 -73.74220627332099) bank94427 +94428 POINT(40.2500724023983 -74.5534642885004) bank94428 +94429 POINT(40.137420221935876 -74.64730010579807) bank94429 +94430 POINT(39.8008627642708 -74.89421975052525) bank94430 +94431 POINT(40.14068819779129 -73.07747715722142) bank94431 +94432 POINT(40.403757385805896 -74.09866111528704) bank94432 +94433 POINT(40.067543046059065 -74.21000011991322) bank94433 +94434 POINT(40.62826619315128 -73.38716440222683) bank94434 +94435 POINT(39.95132264424903 -74.05273508469246) bank94435 +94436 POINT(40.28366365534247 -73.31582594389647) bank94436 +94437 POINT(40.088910391547245 -74.60722947575516) bank94437 +94438 POINT(40.384800779618274 -73.85978134050598) bank94438 +94439 POINT(41.29427727583784 -74.18599013716212) bank94439 +94440 POINT(40.598479958491254 -74.09066253237863) bank94440 +94441 POINT(40.17566979738933 -73.90546217004379) bank94441 +94442 POINT(40.63198492464344 -73.14061375050085) bank94442 +94443 POINT(40.228148399802905 -73.88313702128715) bank94443 +94444 POINT(40.83720974510781 -73.21834467599851) bank94444 +94445 POINT(41.408600544731605 -74.09626573531406) bank94445 +94446 POINT(41.5036562885924 -74.87806205572552) bank94446 +94447 POINT(40.65447483230221 -74.36159549229677) bank94447 +94448 POINT(40.07390407119294 -73.8666950807979) bank94448 +94449 POINT(40.44015715532529 -74.34313791122752) bank94449 +94450 POINT(40.412710983780435 -74.82976285097885) bank94450 +94451 POINT(41.57365964183359 -74.59820122862062) bank94451 +94452 POINT(40.84051334419378 -74.27793810725724) bank94452 +94453 POINT(41.314154935321596 -74.56511963973867) bank94453 +94454 POINT(39.98989678960244 -74.51494904642263) bank94454 +94455 POINT(40.9160523204254 -74.39725270230178) bank94455 +94456 POINT(39.85965736116837 -73.47518202734604) bank94456 +94457 POINT(41.6910697786019 -74.6089533142856) bank94457 +94458 POINT(41.513002294373734 -73.21177213305985) bank94458 +94459 POINT(39.94004462664094 -74.42705312660765) bank94459 +94460 POINT(40.57639663789033 -74.75817425135895) bank94460 +94461 POINT(40.94595574118977 -74.97673687677555) bank94461 +94462 POINT(40.52533080912051 -74.2172699267551) bank94462 +94463 POINT(39.8833306388597 -74.64464875111184) bank94463 +94464 POINT(40.73011661543754 -74.43665625398765) bank94464 +94465 POINT(41.13631718503224 -74.73828667685831) bank94465 +94466 POINT(40.98794524055489 -73.16009388386615) bank94466 +94467 POINT(41.05018263848419 -74.74559985475327) bank94467 +94468 POINT(40.362095473079385 -74.74270131559643) bank94468 +94469 POINT(40.236324845519356 -74.87585287018345) bank94469 +94470 POINT(41.61366886032225 -74.13309660112911) bank94470 +94471 POINT(41.1444443156883 -73.04855510703189) bank94471 +94472 POINT(40.94495533928293 -74.96060244143868) bank94472 +94473 POINT(41.13961430923168 -73.67287041408609) bank94473 +94474 POINT(40.851288399699015 -73.18604295024332) bank94474 +94475 POINT(39.920861028132066 -73.38018668773583) bank94475 +94476 POINT(39.91605707351806 -73.42031512936906) bank94476 +94477 POINT(40.707357006827785 -74.76702502242145) bank94477 +94478 POINT(41.256231002470635 -74.16125625257787) bank94478 +94479 POINT(40.52715597464156 -74.72255830505095) bank94479 +94480 POINT(40.58155496502193 -74.61002647521013) bank94480 +94481 POINT(39.907175954371766 -74.62079476531238) bank94481 +94482 POINT(40.79838471507723 -73.04352975261415) bank94482 +94483 POINT(41.04506170091505 -73.0785025704919) bank94483 +94484 POINT(40.59629077303388 -74.7430495007794) bank94484 +94485 POINT(40.107379813432104 -74.01524109093867) bank94485 +94486 POINT(40.47920510292615 -74.16657945476398) bank94486 +94487 POINT(40.83053396014388 -74.45092860768199) bank94487 +94488 POINT(39.97164939997737 -74.6189803054833) bank94488 +94489 POINT(41.68825915294604 -73.86749949816597) bank94489 +94490 POINT(39.881828263526586 -73.23889657216527) bank94490 +94491 POINT(41.36424559576421 -73.21797309775681) bank94491 +94492 POINT(41.59846413864528 -73.10812755466138) bank94492 +94493 POINT(40.987441670787646 -73.06808593936502) bank94493 +94494 POINT(41.653273227917694 -73.25851522815225) bank94494 +94495 POINT(40.46887467919927 -73.22112876675445) bank94495 +94496 POINT(40.5832575882669 -73.47785620325476) bank94496 +94497 POINT(40.712935392948324 -74.32344169806323) bank94497 +94498 POINT(40.21111783028004 -74.480409194781) bank94498 +94499 POINT(40.120649603421285 -73.25788848128724) bank94499 +94500 POINT(41.592910212717136 -74.62200836533242) bank94500 +94501 POINT(41.45512817525737 -74.76256365493951) bank94501 +94502 POINT(39.97588369842365 -74.91314663257019) bank94502 +94503 POINT(40.697865392216286 -73.01838470061406) bank94503 +94504 POINT(40.384141238002734 -73.8244343388035) bank94504 +94505 POINT(40.91848972972367 -74.62602295898577) bank94505 +94506 POINT(41.326429966354844 -74.87677898846394) bank94506 +94507 POINT(40.63691006661344 -73.5725687101035) bank94507 +94508 POINT(41.651026362167244 -74.40515850668447) bank94508 +94509 POINT(40.63308839725305 -74.45198972113197) bank94509 +94510 POINT(40.50696813366364 -74.77737114162008) bank94510 +94511 POINT(40.87041540575568 -74.12257279439093) bank94511 +94512 POINT(40.86374188092862 -74.07553333869207) bank94512 +94513 POINT(40.50281945406895 -74.8759720641903) bank94513 +94514 POINT(41.580076556120474 -74.44752121749954) bank94514 +94515 POINT(41.52740667447739 -74.16084177300019) bank94515 +94516 POINT(40.63410299835737 -74.45430315598219) bank94516 +94517 POINT(41.35800716703869 -73.99357344515653) bank94517 +94518 POINT(40.41302501643819 -73.14640439322123) bank94518 +94519 POINT(40.66319486891072 -73.83977896155788) bank94519 +94520 POINT(40.24379569568959 -74.19731708181715) bank94520 +94521 POINT(40.37511880198451 -74.50837791760716) bank94521 +94522 POINT(41.16309792913168 -73.83998807149848) bank94522 +94523 POINT(40.13301454119641 -74.2318001518266) bank94523 +94524 POINT(41.41527551481144 -74.27712841515695) bank94524 +94525 POINT(41.28268402260174 -73.2300220965654) bank94525 +94526 POINT(40.2057984137527 -74.70361623272093) bank94526 +94527 POINT(40.028340591759545 -73.42296457213128) bank94527 +94528 POINT(40.372187103935694 -74.80095557608499) bank94528 +94529 POINT(40.69855775268992 -74.44290158371624) bank94529 +94530 POINT(41.341915718461394 -73.68106363688788) bank94530 +94531 POINT(41.257810120770024 -73.17619643080444) bank94531 +94532 POINT(41.31642359262851 -74.40668407340338) bank94532 +94533 POINT(41.466103266972794 -74.48346824628631) bank94533 +94534 POINT(39.74493189790026 -73.42449017921399) bank94534 +94535 POINT(39.88722774119413 -73.79588359058941) bank94535 +94536 POINT(40.27746310982174 -74.85966918416796) bank94536 +94537 POINT(39.927350539189995 -73.81596788966097) bank94537 +94538 POINT(40.59222947931456 -73.73460671929013) bank94538 +94539 POINT(41.35599308285492 -74.38377577463419) bank94539 +94540 POINT(40.52417214998731 -74.7358761919601) bank94540 +94541 POINT(39.7922841603486 -73.92911749321944) bank94541 +94542 POINT(40.95774075020377 -74.00438372982892) bank94542 +94543 POINT(41.53660645359603 -73.8295493202342) bank94543 +94544 POINT(41.648011001645685 -73.22981807311899) bank94544 +94545 POINT(40.901536350083795 -74.08825841787588) bank94545 +94546 POINT(41.228714554046995 -73.47614666689105) bank94546 +94547 POINT(40.24600672967315 -73.69488628992336) bank94547 +94548 POINT(40.11665560209857 -73.9282169717954) bank94548 +94549 POINT(40.60853108216473 -74.61787243586708) bank94549 +94550 POINT(40.457537448783235 -73.54249191156167) bank94550 +94551 POINT(39.72546072772188 -73.43740721282991) bank94551 +94552 POINT(40.456857715684166 -74.96240373861062) bank94552 +94553 POINT(39.7128071263362 -73.73741892514275) bank94553 +94554 POINT(40.50339566597081 -73.2117575792483) bank94554 +94555 POINT(40.271022884680065 -73.47330944863444) bank94555 +94556 POINT(41.31826601419211 -73.44559977018658) bank94556 +94557 POINT(39.79153847711817 -73.31507931586414) bank94557 +94558 POINT(40.0458024655434 -74.79911379304363) bank94558 +94559 POINT(40.13539935852026 -74.04856461144956) bank94559 +94560 POINT(41.31705715824867 -74.52769935080326) bank94560 +94561 POINT(40.80663089232803 -74.29345055223247) bank94561 +94562 POINT(40.767812638536995 -74.40208909562315) bank94562 +94563 POINT(40.516018290040506 -73.38129454996121) bank94563 +94564 POINT(40.109895515055996 -73.3199657036477) bank94564 +94565 POINT(40.87365364293482 -74.28948511221266) bank94565 +94566 POINT(40.568234339424485 -74.44116423318651) bank94566 +94567 POINT(40.492688617256434 -73.87853310496428) bank94567 +94568 POINT(39.85979401760873 -73.31611404642508) bank94568 +94569 POINT(40.56954670613759 -74.6446654578664) bank94569 +94570 POINT(41.08203215668083 -74.34202512497704) bank94570 +94571 POINT(41.350447785741686 -73.0364816708542) bank94571 +94572 POINT(39.95562824401095 -74.70338226841396) bank94572 +94573 POINT(41.55541069335753 -74.39788413581505) bank94573 +94574 POINT(41.39135430389281 -73.99212025718921) bank94574 +94575 POINT(39.7822326168898 -73.81354772633927) bank94575 +94576 POINT(40.245405061156234 -73.29562775043512) bank94576 +94577 POINT(40.15824856935208 -74.20224192365052) bank94577 +94578 POINT(40.61592472994074 -73.03349883303007) bank94578 +94579 POINT(40.0667423083388 -74.27837019383685) bank94579 +94580 POINT(40.12878107543021 -73.23737812110492) bank94580 +94581 POINT(40.70834384890386 -73.57529150747968) bank94581 +94582 POINT(39.93955236642752 -73.79612141366496) bank94582 +94583 POINT(41.29698744822567 -73.1981126973966) bank94583 +94584 POINT(41.66002792246232 -74.4916848104565) bank94584 +94585 POINT(40.28764917258657 -73.65882129871054) bank94585 +94586 POINT(39.93172641974192 -74.01405458229362) bank94586 +94587 POINT(41.15855275960597 -73.08763861557601) bank94587 +94588 POINT(40.88912483683944 -74.20150289010321) bank94588 +94589 POINT(39.80701124229049 -74.11599158845219) bank94589 +94590 POINT(41.21139252511499 -73.86544854119104) bank94590 +94591 POINT(40.546819719498714 -74.00066627198713) bank94591 +94592 POINT(40.39016343146277 -73.2815964086273) bank94592 +94593 POINT(40.99787761638683 -73.21656846306364) bank94593 +94594 POINT(40.80261816097416 -73.35942332222609) bank94594 +94595 POINT(41.62121750288482 -73.02656724338617) bank94595 +94596 POINT(40.41416221359094 -74.93332007273848) bank94596 +94597 POINT(41.61582377993841 -73.93128756029765) bank94597 +94598 POINT(41.08017768445014 -73.75008725336573) bank94598 +94599 POINT(40.827061936680835 -74.9869427509512) bank94599 +94600 POINT(41.090997241553204 -74.337205628413) bank94600 +94601 POINT(40.407319857116924 -73.9084393884717) bank94601 +94602 POINT(40.40926214816246 -73.09073591576612) bank94602 +94603 POINT(41.235680455890545 -74.9201524989409) bank94603 +94604 POINT(39.797425646128815 -74.79708615160474) bank94604 +94605 POINT(41.14007644921864 -74.49815399105903) bank94605 +94606 POINT(41.39529429238401 -73.31718248623704) bank94606 +94607 POINT(41.20299557862653 -73.36649907418581) bank94607 +94608 POINT(40.09466219718544 -74.17535586088391) bank94608 +94609 POINT(39.98096289925542 -74.38495676049902) bank94609 +94610 POINT(40.60691635756493 -73.48121996464594) bank94610 +94611 POINT(41.62830374278619 -73.0698894119872) bank94611 +94612 POINT(41.03684272567974 -73.86382326754135) bank94612 +94613 POINT(40.95689902257086 -73.97680679994438) bank94613 +94614 POINT(40.936072045823245 -74.40925831845381) bank94614 +94615 POINT(41.39925883015224 -74.67752797687922) bank94615 +94616 POINT(40.69425378424819 -74.92313996527302) bank94616 +94617 POINT(40.82219939463162 -74.88645292399575) bank94617 +94618 POINT(41.079780341181646 -73.3034800467975) bank94618 +94619 POINT(41.27919812626661 -73.82771723012394) bank94619 +94620 POINT(40.2054099493703 -74.03427774277917) bank94620 +94621 POINT(40.386233909193365 -73.01317657072994) bank94621 +94622 POINT(40.89369400633923 -74.51902232730143) bank94622 +94623 POINT(41.09553259596579 -73.66421677011698) bank94623 +94624 POINT(41.58371895492253 -74.34000769999031) bank94624 +94625 POINT(40.87241484892801 -74.21036860378067) bank94625 +94626 POINT(40.001591391543066 -74.1669132034215) bank94626 +94627 POINT(39.78269979422805 -74.99313429017023) bank94627 +94628 POINT(41.63584228530095 -73.19076620044592) bank94628 +94629 POINT(40.31897060628231 -73.19405893293565) bank94629 +94630 POINT(41.43284882589261 -74.75877597011309) bank94630 +94631 POINT(39.7391904335164 -74.27040289230965) bank94631 +94632 POINT(39.840576855251214 -74.82903434625507) bank94632 +94633 POINT(41.4727618359206 -73.78728206350631) bank94633 +94634 POINT(39.740220972258676 -73.75484577557245) bank94634 +94635 POINT(41.50150561165286 -73.63846246323735) bank94635 +94636 POINT(40.72837459063461 -74.1487700104839) bank94636 +94637 POINT(39.79204458679863 -74.02463753028164) bank94637 +94638 POINT(40.23312994707068 -74.42660758101835) bank94638 +94639 POINT(39.77152823143249 -73.97142669521972) bank94639 +94640 POINT(40.50184697702143 -73.06622342491535) bank94640 +94641 POINT(39.735477609107406 -74.20074460423382) bank94641 +94642 POINT(41.038985306332435 -74.86624522858146) bank94642 +94643 POINT(39.78081547603048 -74.89714152243283) bank94643 +94644 POINT(40.55291480784671 -73.00922988575103) bank94644 +94645 POINT(41.00637522691084 -73.5204646556806) bank94645 +94646 POINT(40.66421199263128 -74.32927659992025) bank94646 +94647 POINT(40.964826669605145 -73.4945759868805) bank94647 +94648 POINT(41.65085713109666 -73.69050480901612) bank94648 +94649 POINT(40.2120493987406 -74.52150264269227) bank94649 +94650 POINT(41.398512786912214 -74.2076720855723) bank94650 +94651 POINT(39.94547321946611 -74.39666224840082) bank94651 +94652 POINT(41.29832256130458 -73.9857100394709) bank94652 +94653 POINT(41.37704917009615 -74.608359581373) bank94653 +94654 POINT(40.304861966772485 -73.39478635335946) bank94654 +94655 POINT(40.78813123881332 -73.48203795039618) bank94655 +94656 POINT(39.85438263491538 -73.02013766572883) bank94656 +94657 POINT(39.95294582883862 -74.7658704392248) bank94657 +94658 POINT(41.37415780378963 -73.23814169181821) bank94658 +94659 POINT(40.25276493823125 -73.68745985316885) bank94659 +94660 POINT(39.85265847445973 -73.26519153724954) bank94660 +94661 POINT(41.56527309216044 -74.75506150139157) bank94661 +94662 POINT(41.60503216732143 -74.62746162387049) bank94662 +94663 POINT(41.52055605202129 -73.0271220457834) bank94663 +94664 POINT(41.304013732001984 -73.81845836789259) bank94664 +94665 POINT(40.405383973053766 -73.13657769380417) bank94665 +94666 POINT(40.074385003225515 -73.02011433337832) bank94666 +94667 POINT(40.715828577767276 -74.28171346311218) bank94667 +94668 POINT(40.042606037806536 -73.5799152815351) bank94668 +94669 POINT(40.4745310941043 -73.33393770918642) bank94669 +94670 POINT(40.0065901684783 -74.95210174667278) bank94670 +94671 POINT(40.32494822080684 -74.4549424124867) bank94671 +94672 POINT(40.66560012357606 -74.11369793379237) bank94672 +94673 POINT(40.392574527576 -74.53589749098492) bank94673 +94674 POINT(40.28637105835591 -74.11359731204362) bank94674 +94675 POINT(41.03930775530179 -73.38691270987195) bank94675 +94676 POINT(40.709273815037584 -74.55990377092455) bank94676 +94677 POINT(41.50121428542593 -73.29614223483016) bank94677 +94678 POINT(40.47827343780555 -74.21207574146368) bank94678 +94679 POINT(40.37724583490657 -74.05765007628544) bank94679 +94680 POINT(41.2227668957109 -74.58452553960153) bank94680 +94681 POINT(40.18212492550157 -73.93481069670065) bank94681 +94682 POINT(41.65379043856704 -73.6685569708469) bank94682 +94683 POINT(41.15401063123817 -73.57912673741441) bank94683 +94684 POINT(40.205339702864535 -73.54066163239267) bank94684 +94685 POINT(41.05555571930702 -74.59155699297266) bank94685 +94686 POINT(41.644304159364495 -73.19124527342373) bank94686 +94687 POINT(40.56102979071068 -74.48653375047965) bank94687 +94688 POINT(39.915546548452376 -73.28602408194192) bank94688 +94689 POINT(39.9538721314634 -73.24410620794484) bank94689 +94690 POINT(40.352379727241015 -74.25472892941598) bank94690 +94691 POINT(40.558120695705526 -74.72524103591584) bank94691 +94692 POINT(39.965275173671756 -73.4702163676809) bank94692 +94693 POINT(39.936129063041996 -74.14969791201938) bank94693 +94694 POINT(40.312643916067024 -74.25906827959751) bank94694 +94695 POINT(41.27195526229642 -73.81002026364858) bank94695 +94696 POINT(39.75063117555999 -74.85595332919073) bank94696 +94697 POINT(41.411000138617204 -74.01399465696903) bank94697 +94698 POINT(40.242733441328 -74.4435955831211) bank94698 +94699 POINT(40.67511243071149 -74.7682748027806) bank94699 +94700 POINT(40.207341915534094 -74.51869595687408) bank94700 +94701 POINT(40.85547712545392 -74.59899866278286) bank94701 +94702 POINT(41.40248281998508 -74.36795710550616) bank94702 +94703 POINT(40.6766317708054 -74.06328733071796) bank94703 +94704 POINT(40.89527458456282 -74.0959413542356) bank94704 +94705 POINT(41.32458225208086 -73.93667240833311) bank94705 +94706 POINT(39.9639273271779 -74.14479015989014) bank94706 +94707 POINT(40.16136785129825 -73.07527069565496) bank94707 +94708 POINT(40.87709032575282 -74.80740694606116) bank94708 +94709 POINT(40.054591068811085 -73.73550601061967) bank94709 +94710 POINT(40.275439457907524 -73.68551070444401) bank94710 +94711 POINT(40.93259969429593 -74.20645748889231) bank94711 +94712 POINT(40.741506101831796 -74.13444360607247) bank94712 +94713 POINT(40.50102692206943 -73.56893089962871) bank94713 +94714 POINT(40.91374181293839 -74.7202500129596) bank94714 +94715 POINT(39.98406343541939 -74.98673713544699) bank94715 +94716 POINT(39.83838549765408 -74.18144847887555) bank94716 +94717 POINT(41.03484122116422 -73.42015568487065) bank94717 +94718 POINT(41.63270392455422 -73.80971189323625) bank94718 +94719 POINT(41.633449301070854 -73.60674972466086) bank94719 +94720 POINT(40.698741220588 -73.04080295470109) bank94720 +94721 POINT(39.7807505270762 -73.27454173204768) bank94721 +94722 POINT(40.009117795491 -73.48569442526268) bank94722 +94723 POINT(41.14791730847709 -73.22074489753948) bank94723 +94724 POINT(40.13366254032355 -74.22710193449603) bank94724 +94725 POINT(39.98951355689717 -73.1152072100418) bank94725 +94726 POINT(41.0754108340418 -73.77664263050707) bank94726 +94727 POINT(39.836057342967074 -74.35784195258051) bank94727 +94728 POINT(40.31573875808361 -74.84242248665726) bank94728 +94729 POINT(40.86774255218629 -73.8377940939028) bank94729 +94730 POINT(40.96612775783806 -73.041739322029) bank94730 +94731 POINT(41.584471299674576 -74.90244330620443) bank94731 +94732 POINT(41.60081183547089 -74.45285997216949) bank94732 +94733 POINT(39.9178650525315 -74.04229319869253) bank94733 +94734 POINT(39.94712606607141 -74.42061671910999) bank94734 +94735 POINT(40.337546849463926 -73.57580340641815) bank94735 +94736 POINT(41.44877597703862 -73.34468654308237) bank94736 +94737 POINT(40.29107551534478 -74.4823688629531) bank94737 +94738 POINT(40.14759155008722 -74.564615774852) bank94738 +94739 POINT(40.20083928249528 -74.22316698488231) bank94739 +94740 POINT(39.97691606202386 -73.41995013669107) bank94740 +94741 POINT(41.70145705346785 -73.72236917318132) bank94741 +94742 POINT(41.2677072323842 -73.26857911064698) bank94742 +94743 POINT(40.40003189933161 -74.90704046781407) bank94743 +94744 POINT(40.993303352839874 -74.62449188228898) bank94744 +94745 POINT(40.73944846327855 -74.78058613519333) bank94745 +94746 POINT(40.630068665786844 -73.32891079527775) bank94746 +94747 POINT(41.175166622881065 -74.7827475906198) bank94747 +94748 POINT(40.54384027732078 -73.49327812420722) bank94748 +94749 POINT(40.73723826731962 -73.52626097452898) bank94749 +94750 POINT(39.98672554990549 -74.01943953212577) bank94750 +94751 POINT(39.820453479784064 -73.52198480466909) bank94751 +94752 POINT(39.98076452456657 -74.14809172762405) bank94752 +94753 POINT(41.56232343930011 -74.58310001830202) bank94753 +94754 POINT(40.14064411407896 -73.22253662047046) bank94754 +94755 POINT(41.11961611220056 -73.15404589537938) bank94755 +94756 POINT(41.303764550026166 -74.11713725775503) bank94756 +94757 POINT(41.60158514389134 -74.28945895976115) bank94757 +94758 POINT(40.99352913807207 -73.09864548337092) bank94758 +94759 POINT(40.832190205433164 -73.59547407218172) bank94759 +94760 POINT(41.48575690994562 -74.26883651188766) bank94760 +94761 POINT(40.075440809451905 -73.99798409918242) bank94761 +94762 POINT(40.3029848508122 -74.09955850684615) bank94762 +94763 POINT(41.08825887836283 -73.30335195137981) bank94763 +94764 POINT(39.92757710104213 -73.60993481159906) bank94764 +94765 POINT(40.64048022471458 -74.59683167135722) bank94765 +94766 POINT(39.99229136958441 -74.53752499885671) bank94766 +94767 POINT(41.53753088371114 -73.9060511762711) bank94767 +94768 POINT(40.495322678616205 -73.83114799636743) bank94768 +94769 POINT(41.13586465946955 -74.59451036421713) bank94769 +94770 POINT(40.91210063375945 -74.64599921515003) bank94770 +94771 POINT(40.557511160922424 -73.46612534225041) bank94771 +94772 POINT(41.293696260931334 -73.68255275119132) bank94772 +94773 POINT(39.822568522534674 -74.89243240757591) bank94773 +94774 POINT(41.48414894889898 -73.05118334217964) bank94774 +94775 POINT(41.55729272133598 -74.69667079563753) bank94775 +94776 POINT(40.65932211280457 -74.45176816989685) bank94776 +94777 POINT(40.12603655162262 -73.22352611676806) bank94777 +94778 POINT(39.76290604243043 -73.35358249974942) bank94778 +94779 POINT(41.55252249618936 -74.97224816921693) bank94779 +94780 POINT(40.42793524132112 -73.96353324222991) bank94780 +94781 POINT(41.55830256456177 -73.24459907649283) bank94781 +94782 POINT(40.411142777545614 -74.38474049488742) bank94782 +94783 POINT(40.15552475687675 -73.75409810534663) bank94783 +94784 POINT(39.75316842138617 -73.44783815384613) bank94784 +94785 POINT(41.0173147118007 -74.15684843353648) bank94785 +94786 POINT(40.812982355231085 -73.5228058148356) bank94786 +94787 POINT(40.53507134523605 -73.46719697296537) bank94787 +94788 POINT(40.63322311585825 -74.7893145787053) bank94788 +94789 POINT(41.239280481930265 -74.18149480140598) bank94789 +94790 POINT(41.69349610834811 -73.70428172812053) bank94790 +94791 POINT(41.130689375421234 -73.40169358404759) bank94791 +94792 POINT(40.93637577552358 -74.85447874611445) bank94792 +94793 POINT(39.97751184493662 -74.78460625381467) bank94793 +94794 POINT(40.26785861426329 -74.34925676175612) bank94794 +94795 POINT(39.97033361129657 -73.50867439558604) bank94795 +94796 POINT(40.70419975662706 -73.44131263501828) bank94796 +94797 POINT(41.33194417847123 -73.21079284690407) bank94797 +94798 POINT(41.47353083415504 -74.20905123805046) bank94798 +94799 POINT(41.36758141603297 -74.36610377306297) bank94799 +94800 POINT(40.13668032306406 -73.16116448132385) bank94800 +94801 POINT(40.37365190504137 -74.39491069195864) bank94801 +94802 POINT(40.54053914593071 -73.23221174895019) bank94802 +94803 POINT(40.954176884337464 -73.08276315413536) bank94803 +94804 POINT(41.54403319918423 -73.06835155009544) bank94804 +94805 POINT(41.696319138598106 -74.5589396002922) bank94805 +94806 POINT(40.74364454278454 -74.06818451393161) bank94806 +94807 POINT(41.086055759344625 -73.31931543427561) bank94807 +94808 POINT(40.7187891246123 -74.04576905474262) bank94808 +94809 POINT(41.426471518506226 -73.57234701106165) bank94809 +94810 POINT(40.018382222537056 -73.4761353396493) bank94810 +94811 POINT(40.61364878198986 -73.18477410218867) bank94811 +94812 POINT(41.38206155301626 -73.29198765417432) bank94812 +94813 POINT(40.13232308147816 -74.61438943023806) bank94813 +94814 POINT(39.82693909648622 -73.44039406296048) bank94814 +94815 POINT(41.49211022547543 -74.13389889825352) bank94815 +94816 POINT(41.13280274145343 -74.18998002471277) bank94816 +94817 POINT(40.96260618351355 -73.3087013618655) bank94817 +94818 POINT(41.17092690379 -73.14691611679581) bank94818 +94819 POINT(41.36596177551078 -73.56056711168236) bank94819 +94820 POINT(40.289849751835455 -73.24712414792675) bank94820 +94821 POINT(40.65318338672573 -73.03304700115429) bank94821 +94822 POINT(39.87194975468264 -73.3571794097529) bank94822 +94823 POINT(41.342114265170615 -74.4258039189453) bank94823 +94824 POINT(40.42354812142819 -73.38329718860463) bank94824 +94825 POINT(41.058892132344305 -73.25507480906612) bank94825 +94826 POINT(41.12177614560783 -74.64738605960989) bank94826 +94827 POINT(41.314055438880445 -73.66517256033552) bank94827 +94828 POINT(41.27532641938132 -74.67976615154396) bank94828 +94829 POINT(40.25376923716854 -74.33161422549824) bank94829 +94830 POINT(39.98945499638458 -74.4040734598771) bank94830 +94831 POINT(40.53689387334976 -74.92388829188222) bank94831 +94832 POINT(40.968324288309496 -74.64054355410688) bank94832 +94833 POINT(41.24443676389948 -73.92981551437296) bank94833 +94834 POINT(41.29485354212198 -73.93445563512051) bank94834 +94835 POINT(39.79156247619828 -73.3039222286379) bank94835 +94836 POINT(41.19232937686113 -74.1070459300579) bank94836 +94837 POINT(40.22113445147539 -74.37876589674637) bank94837 +94838 POINT(40.27073989972409 -74.4169835583524) bank94838 +94839 POINT(40.87699411535072 -74.13951291183032) bank94839 +94840 POINT(39.833874207462046 -73.98438993570286) bank94840 +94841 POINT(41.07943182957769 -74.85283302856992) bank94841 +94842 POINT(40.50712562925963 -74.26633881563392) bank94842 +94843 POINT(40.776751330511935 -74.73059351739296) bank94843 +94844 POINT(41.593191981515155 -74.45056108199776) bank94844 +94845 POINT(40.05985393869633 -74.12867937058567) bank94845 +94846 POINT(40.017950501256415 -74.4919184539853) bank94846 +94847 POINT(40.290586013836595 -73.95516561455432) bank94847 +94848 POINT(40.41509885178719 -73.42530884955744) bank94848 +94849 POINT(40.73689601276144 -73.7988161708457) bank94849 +94850 POINT(41.21738359301461 -73.8826616504596) bank94850 +94851 POINT(41.392950532482786 -74.26637052261985) bank94851 +94852 POINT(41.02018480691915 -73.1168931091789) bank94852 +94853 POINT(40.39006965900365 -73.43795621216088) bank94853 +94854 POINT(40.26632583522505 -73.72447203759342) bank94854 +94855 POINT(40.41269327249589 -74.51039602703936) bank94855 +94856 POINT(41.30706592807914 -74.66777891147962) bank94856 +94857 POINT(41.25501900560868 -74.57883352697297) bank94857 +94858 POINT(41.13434492394596 -73.14313946295849) bank94858 +94859 POINT(40.590287091983164 -73.8589800196552) bank94859 +94860 POINT(41.33482408253387 -73.65366521271585) bank94860 +94861 POINT(41.68281282401621 -73.93473229971372) bank94861 +94862 POINT(40.82731803532948 -73.81191198105805) bank94862 +94863 POINT(41.234847837695135 -74.23402811096459) bank94863 +94864 POINT(39.871537182065055 -74.42659724843334) bank94864 +94865 POINT(40.12065806906279 -74.30926374655934) bank94865 +94866 POINT(39.71325237950636 -73.9399959441801) bank94866 +94867 POINT(41.11048853942143 -74.50807754106063) bank94867 +94868 POINT(40.015731499484836 -73.5371420223151) bank94868 +94869 POINT(40.01632695146777 -73.7198334752838) bank94869 +94870 POINT(40.00488418635524 -74.90925444874831) bank94870 +94871 POINT(39.73112723891237 -73.42232235682062) bank94871 +94872 POINT(41.596331437721325 -74.51073896958141) bank94872 +94873 POINT(40.50895349060805 -73.74838886973903) bank94873 +94874 POINT(41.65897634323762 -73.15293180269494) bank94874 +94875 POINT(41.22650183874109 -74.56680804355379) bank94875 +94876 POINT(39.73264109947656 -74.45128062266393) bank94876 +94877 POINT(40.55749669085669 -73.72119155996832) bank94877 +94878 POINT(39.96725578400139 -73.4737178368319) bank94878 +94879 POINT(41.0343844392262 -74.62492060182012) bank94879 +94880 POINT(41.389880897143335 -74.19848776358842) bank94880 +94881 POINT(41.54173511240438 -74.92857523417244) bank94881 +94882 POINT(41.427472010350726 -73.99992703853167) bank94882 +94883 POINT(41.02672767178205 -73.95861061778886) bank94883 +94884 POINT(41.378471316627106 -74.23085179816307) bank94884 +94885 POINT(40.598382156607414 -73.69276205581338) bank94885 +94886 POINT(40.44236835359745 -73.25236593243638) bank94886 +94887 POINT(39.79798019763048 -73.85973538919059) bank94887 +94888 POINT(40.48565954695018 -74.58325050752038) bank94888 +94889 POINT(41.42176463079086 -73.25588734117835) bank94889 +94890 POINT(40.66064256962098 -74.95243047407425) bank94890 +94891 POINT(40.76083413855654 -73.57372385527047) bank94891 +94892 POINT(39.88537723690629 -74.75881365973599) bank94892 +94893 POINT(40.152130260208 -74.76444480204765) bank94893 +94894 POINT(40.95453347156486 -74.02391443654132) bank94894 +94895 POINT(41.4658646941607 -74.11422463603854) bank94895 +94896 POINT(41.6051128430081 -73.86057180949089) bank94896 +94897 POINT(41.12784130003336 -73.2621083371953) bank94897 +94898 POINT(41.05066328301842 -74.94401479705715) bank94898 +94899 POINT(40.23675166612377 -74.17416350509733) bank94899 +94900 POINT(41.71010441752466 -74.33732433979367) bank94900 +94901 POINT(40.52609734975654 -73.5353821654766) bank94901 +94902 POINT(40.42163682297372 -73.57549099599605) bank94902 +94903 POINT(41.50626104653606 -73.69148290206888) bank94903 +94904 POINT(40.08114185323668 -73.70303919816222) bank94904 +94905 POINT(41.651702683076664 -73.68388815752841) bank94905 +94906 POINT(40.81503911120236 -74.79959357228404) bank94906 +94907 POINT(41.113787910547494 -73.54119332592441) bank94907 +94908 POINT(41.704201623306794 -74.83501736817162) bank94908 +94909 POINT(39.75161536628208 -74.80887336947174) bank94909 +94910 POINT(40.377204576249845 -73.18938531902779) bank94910 +94911 POINT(40.22396866603288 -74.64017072373026) bank94911 +94912 POINT(40.385809811616724 -74.41552119352772) bank94912 +94913 POINT(40.002042619948334 -74.18011523520693) bank94913 +94914 POINT(40.40485180911547 -74.09789218890384) bank94914 +94915 POINT(40.096764318692294 -74.90064238609176) bank94915 +94916 POINT(40.34990536084136 -73.01883032002402) bank94916 +94917 POINT(41.064897137025596 -74.93532656965525) bank94917 +94918 POINT(40.10136492264837 -73.5406127966828) bank94918 +94919 POINT(40.67340395885212 -74.22951431184339) bank94919 +94920 POINT(41.05509486466862 -73.17209005886114) bank94920 +94921 POINT(41.60948621482551 -74.9747960775291) bank94921 +94922 POINT(40.649214127038775 -73.34983205855767) bank94922 +94923 POINT(41.25460135171615 -73.41129247101205) bank94923 +94924 POINT(40.34550900726996 -73.58570360917285) bank94924 +94925 POINT(40.98555281949062 -74.1063502270902) bank94925 +94926 POINT(41.62750514716632 -73.6902210694981) bank94926 +94927 POINT(39.97519037235704 -74.50531293937527) bank94927 +94928 POINT(40.596443191583255 -74.25615493492245) bank94928 +94929 POINT(41.25929051756597 -73.72685716128116) bank94929 +94930 POINT(41.44009231713972 -73.87588617451497) bank94930 +94931 POINT(40.546069030793326 -74.2659212245666) bank94931 +94932 POINT(41.46228838305525 -74.07441451824072) bank94932 +94933 POINT(41.4286084702963 -74.58006165233607) bank94933 +94934 POINT(39.79094022764441 -74.58250812407205) bank94934 +94935 POINT(39.99627007987684 -74.71856002375188) bank94935 +94936 POINT(39.87920334376978 -73.82123099311517) bank94936 +94937 POINT(39.92423589567118 -74.01770244498) bank94937 +94938 POINT(41.294045091004556 -73.14732936710982) bank94938 +94939 POINT(41.544217390106006 -73.6129485203193) bank94939 +94940 POINT(41.70354635357063 -73.33967008209002) bank94940 +94941 POINT(41.64169502063652 -73.88476471442) bank94941 +94942 POINT(39.78537390778089 -74.72020856525987) bank94942 +94943 POINT(40.75928398304551 -73.96672464081936) bank94943 +94944 POINT(40.677990566319885 -74.50118722679883) bank94944 +94945 POINT(39.95839130450982 -73.38963909328629) bank94945 +94946 POINT(40.815098498769125 -73.92858662413454) bank94946 +94947 POINT(41.681175600722014 -73.46303917548026) bank94947 +94948 POINT(40.419451628208996 -74.4006807180874) bank94948 +94949 POINT(41.58021260402648 -74.14323578958825) bank94949 +94950 POINT(41.21745857738612 -73.06863325172662) bank94950 +94951 POINT(41.45258913129891 -74.69230004973637) bank94951 +94952 POINT(40.16272260405008 -74.05530335333764) bank94952 +94953 POINT(40.97174546335575 -74.83714466432) bank94953 +94954 POINT(41.68339767136886 -74.50524994679392) bank94954 +94955 POINT(40.02825014868543 -74.42067280378174) bank94955 +94956 POINT(40.033790983918706 -73.03560294746052) bank94956 +94957 POINT(40.39636658718215 -73.94737274039413) bank94957 +94958 POINT(40.240758314313624 -73.01325077628594) bank94958 +94959 POINT(40.618403506568164 -73.2764431095693) bank94959 +94960 POINT(40.06746398471633 -74.25778895404554) bank94960 +94961 POINT(41.554868803648326 -73.84336218981323) bank94961 +94962 POINT(40.66015225962041 -74.66101679736468) bank94962 +94963 POINT(41.57415258069135 -73.88362525254439) bank94963 +94964 POINT(40.13561578403629 -74.63686654998459) bank94964 +94965 POINT(39.8802208570511 -74.10880050438503) bank94965 +94966 POINT(41.44779652155259 -73.7160696757097) bank94966 +94967 POINT(40.06788325720459 -74.69673564535425) bank94967 +94968 POINT(41.48523486952162 -74.50299374392344) bank94968 +94969 POINT(40.33230117185678 -74.8311108722774) bank94969 +94970 POINT(40.320962604892735 -74.68795721051656) bank94970 +94971 POINT(39.80207315894698 -73.49656120375907) bank94971 +94972 POINT(40.89256660000378 -74.71002408678275) bank94972 +94973 POINT(40.17359556414296 -74.42654196924212) bank94973 +94974 POINT(40.198276403396164 -74.65849216724055) bank94974 +94975 POINT(41.29613266537153 -74.13838938388636) bank94975 +94976 POINT(40.12019739698136 -73.99562341866763) bank94976 +94977 POINT(41.497498916661606 -73.30983011449166) bank94977 +94978 POINT(41.21074913749989 -73.72280996813281) bank94978 +94979 POINT(40.58214595229442 -73.73081685981847) bank94979 +94980 POINT(40.91710646330308 -74.28488898677244) bank94980 +94981 POINT(40.72881021354354 -74.63047215599371) bank94981 +94982 POINT(40.397193653829184 -73.20044323381897) bank94982 +94983 POINT(40.24020308077551 -74.57645731580847) bank94983 +94984 POINT(40.32411324539514 -73.29885215247856) bank94984 +94985 POINT(40.888173692173034 -74.61061328243396) bank94985 +94986 POINT(40.365257137169245 -74.39064813085666) bank94986 +94987 POINT(41.4493860838481 -74.41395204908147) bank94987 +94988 POINT(39.8814021704968 -73.29467095807233) bank94988 +94989 POINT(41.21253169703359 -73.9297666421579) bank94989 +94990 POINT(41.06149346501148 -74.51390477989466) bank94990 +94991 POINT(41.204697063241284 -74.18629403271412) bank94991 +94992 POINT(40.11867113332522 -73.90355812985236) bank94992 +94993 POINT(40.79173966160721 -73.51176505730284) bank94993 +94994 POINT(40.88887666127083 -74.16503862861285) bank94994 +94995 POINT(40.195057912665796 -74.55677773966082) bank94995 +94996 POINT(40.707354628371654 -74.79538370952946) bank94996 +94997 POINT(40.77357637017859 -73.396890046914) bank94997 +94998 POINT(39.973967665132456 -74.18291323057628) bank94998 +94999 POINT(41.48042671615803 -73.01434663362143) bank94999 +95000 POINT(39.959189967175334 -74.66220561525141) bank95000 +95001 POINT(40.33257396076202 -73.96293602816415) bank95001 +95002 POINT(40.60540872042915 -73.64068570816293) bank95002 +95003 POINT(41.71194566500644 -73.37831073631422) bank95003 +95004 POINT(41.30949045911309 -74.26580135718571) bank95004 +95005 POINT(40.98178231013114 -74.64733501123315) bank95005 +95006 POINT(41.361013328053126 -73.70408849789031) bank95006 +95007 POINT(40.246617654110786 -74.18515220733036) bank95007 +95008 POINT(41.54258660899132 -74.6972879420717) bank95008 +95009 POINT(41.56845237249722 -74.69374803526479) bank95009 +95010 POINT(41.62564337790822 -74.75886904111924) bank95010 +95011 POINT(39.85542410979239 -74.89237685006212) bank95011 +95012 POINT(40.94533499017082 -73.23762303809268) bank95012 +95013 POINT(41.03905716635088 -74.77731673589936) bank95013 +95014 POINT(40.11025740905177 -74.18465655313672) bank95014 +95015 POINT(40.8666171932667 -74.4135729102752) bank95015 +95016 POINT(39.83798199599913 -73.28307504420253) bank95016 +95017 POINT(39.910066807002266 -73.29911918381679) bank95017 +95018 POINT(40.95350446842627 -74.57406395053447) bank95018 +95019 POINT(40.66149295796681 -74.60137142978968) bank95019 +95020 POINT(40.921217620573046 -74.4752879827361) bank95020 +95021 POINT(39.9053367142571 -74.2423925779792) bank95021 +95022 POINT(40.44757008898092 -74.56150584238844) bank95022 +95023 POINT(39.960127546506115 -73.23233486549569) bank95023 +95024 POINT(41.353823368739015 -73.82657275428899) bank95024 +95025 POINT(41.45247742386583 -74.78050601881297) bank95025 +95026 POINT(39.787311892669884 -73.86941331948691) bank95026 +95027 POINT(41.66577076661233 -74.78277429965671) bank95027 +95028 POINT(40.50216253250475 -73.66711864557968) bank95028 +95029 POINT(40.68224293990535 -74.74082731143697) bank95029 +95030 POINT(41.14367561640245 -74.17458484571506) bank95030 +95031 POINT(39.950949534278166 -74.39279813685124) bank95031 +95032 POINT(40.37286044106017 -74.29614633303476) bank95032 +95033 POINT(40.76577101487352 -74.22702861162931) bank95033 +95034 POINT(40.16372706460617 -73.18359867203738) bank95034 +95035 POINT(41.57956874133702 -74.33091777056922) bank95035 +95036 POINT(41.364266310776955 -73.61787938251501) bank95036 +95037 POINT(40.97445593761768 -73.629031942738) bank95037 +95038 POINT(40.80841614738241 -73.36002735853981) bank95038 +95039 POINT(41.66157364746881 -73.47219175430658) bank95039 +95040 POINT(41.455985297703194 -74.86456483925923) bank95040 +95041 POINT(40.72616906093664 -73.49869031719066) bank95041 +95042 POINT(40.91111603394612 -74.61434565703007) bank95042 +95043 POINT(39.84050285063708 -73.71130842882486) bank95043 +95044 POINT(39.818061995075695 -73.57376819374417) bank95044 +95045 POINT(40.453334580675936 -73.8421668774421) bank95045 +95046 POINT(41.6176210381703 -73.24712156034514) bank95046 +95047 POINT(41.684829413229785 -74.75226407228136) bank95047 +95048 POINT(39.736988088164246 -74.22731223379327) bank95048 +95049 POINT(40.18728273433213 -74.442196390189) bank95049 +95050 POINT(40.48612876144014 -74.03123709406674) bank95050 +95051 POINT(40.130071939048975 -74.66744098511931) bank95051 +95052 POINT(39.83961538321735 -74.45479558125675) bank95052 +95053 POINT(41.452486211298364 -74.02866941844911) bank95053 +95054 POINT(41.59956229269863 -74.34055176551796) bank95054 +95055 POINT(40.087095356550876 -74.18378171016914) bank95055 +95056 POINT(40.031046514397055 -73.52896471888525) bank95056 +95057 POINT(39.92837994187824 -74.33868677784041) bank95057 +95058 POINT(41.32815296030994 -74.68194968683311) bank95058 +95059 POINT(40.64716904538784 -73.37942967705736) bank95059 +95060 POINT(41.34740102488615 -74.18666570213783) bank95060 +95061 POINT(39.75038512562017 -74.66975954524251) bank95061 +95062 POINT(40.52288672050178 -74.53225455325541) bank95062 +95063 POINT(40.786995389489576 -74.18610264166742) bank95063 +95064 POINT(40.63392394063146 -74.85297645692945) bank95064 +95065 POINT(40.89654126768673 -74.35997402826317) bank95065 +95066 POINT(39.98299043999291 -73.52640872944339) bank95066 +95067 POINT(41.45459430279038 -73.72427088903775) bank95067 +95068 POINT(40.83990388768893 -74.52773442623418) bank95068 +95069 POINT(39.802993009723 -74.10997212359928) bank95069 +95070 POINT(39.959330430957706 -73.98570875827625) bank95070 +95071 POINT(41.45034149248289 -74.99705287930882) bank95071 +95072 POINT(40.76398532909378 -74.9197617796476) bank95072 +95073 POINT(41.39428549579776 -74.16203138798109) bank95073 +95074 POINT(40.35642975760172 -74.93420495543857) bank95074 +95075 POINT(40.313840518545376 -73.29278810085746) bank95075 +95076 POINT(39.98180312033639 -73.04818118314279) bank95076 +95077 POINT(40.923608224969286 -74.93474542704179) bank95077 +95078 POINT(41.025469803714685 -74.15424182358167) bank95078 +95079 POINT(39.72043398948521 -73.80869774248305) bank95079 +95080 POINT(41.069286855649416 -74.8702809319442) bank95080 +95081 POINT(39.80619219583085 -73.68953125634768) bank95081 +95082 POINT(40.875719763423376 -74.87588612026089) bank95082 +95083 POINT(41.31914145340519 -74.44999499829197) bank95083 +95084 POINT(40.70814822180873 -73.1337433075399) bank95084 +95085 POINT(41.37773992195034 -73.94469925938047) bank95085 +95086 POINT(40.98194066656179 -74.83030793372647) bank95086 +95087 POINT(40.185118306538875 -73.55998990371539) bank95087 +95088 POINT(40.18820014335087 -73.89083911757065) bank95088 +95089 POINT(41.589124422878704 -74.0704328406601) bank95089 +95090 POINT(40.99466811892324 -73.67536259381242) bank95090 +95091 POINT(39.75483391494613 -74.92041998088692) bank95091 +95092 POINT(41.198263346608925 -73.9949447396588) bank95092 +95093 POINT(40.24212507768991 -73.77215483130826) bank95093 +95094 POINT(39.73223473850405 -74.44580619724726) bank95094 +95095 POINT(41.67051490984341 -74.75881274507127) bank95095 +95096 POINT(40.48901363462423 -73.46115189238748) bank95096 +95097 POINT(40.15706512690963 -74.22139610028418) bank95097 +95098 POINT(40.06397741555194 -74.024750948864) bank95098 +95099 POINT(41.158157356433996 -74.32076142920258) bank95099 +95100 POINT(41.44561072118667 -73.71403045542492) bank95100 +95101 POINT(40.030889375022404 -74.00604339613417) bank95101 +95102 POINT(40.39879015466631 -74.58130862334545) bank95102 +95103 POINT(41.31958557259073 -73.38373417856309) bank95103 +95104 POINT(40.70684719883896 -73.3648792668758) bank95104 +95105 POINT(41.33042772019114 -73.91678284378487) bank95105 +95106 POINT(39.81205693125659 -73.50787598095751) bank95106 +95107 POINT(41.71235151212615 -74.12126920744186) bank95107 +95108 POINT(41.52929328656674 -73.98031571368035) bank95108 +95109 POINT(40.01973312748016 -73.39512046882152) bank95109 +95110 POINT(41.142001653582184 -74.29533363837619) bank95110 +95111 POINT(39.88516180592761 -73.88734834256167) bank95111 +95112 POINT(39.84919196519627 -73.78567861801592) bank95112 +95113 POINT(39.7515116247989 -73.9269562469774) bank95113 +95114 POINT(39.76189179018318 -73.15290414904894) bank95114 +95115 POINT(40.78520956398527 -73.33162970307427) bank95115 +95116 POINT(40.326983801108256 -74.17626205211852) bank95116 +95117 POINT(39.732543586445196 -74.81027193850069) bank95117 +95118 POINT(41.673802530191196 -73.034848341674) bank95118 +95119 POINT(39.9783152391277 -73.41902072538512) bank95119 +95120 POINT(41.10135452592554 -73.40040315924355) bank95120 +95121 POINT(41.54884660023368 -74.43647049100049) bank95121 +95122 POINT(40.15288144777213 -74.75487332554921) bank95122 +95123 POINT(40.82277047129558 -73.11280367797794) bank95123 +95124 POINT(40.17910153247864 -74.54883475166115) bank95124 +95125 POINT(41.36033959236264 -73.879064792962) bank95125 +95126 POINT(39.78632934607952 -73.27463068050109) bank95126 +95127 POINT(40.851848162935916 -74.27203425035128) bank95127 +95128 POINT(41.708799657680935 -74.19564963207739) bank95128 +95129 POINT(41.663560211985875 -74.78721050271878) bank95129 +95130 POINT(40.806126101084615 -73.27548405469088) bank95130 +95131 POINT(40.19661412031552 -73.31710118090152) bank95131 +95132 POINT(40.00441917180914 -74.09092959875818) bank95132 +95133 POINT(41.5617477616272 -74.86265876606181) bank95133 +95134 POINT(40.61658256805399 -73.39549760336405) bank95134 +95135 POINT(41.567678417459874 -74.2511468334467) bank95135 +95136 POINT(40.28929430787257 -74.57610504086969) bank95136 +95137 POINT(40.89624577574523 -73.15615802044303) bank95137 +95138 POINT(40.6165653064188 -74.35471131883683) bank95138 +95139 POINT(39.7850937492919 -73.44321412878803) bank95139 +95140 POINT(40.00512288327969 -74.4291926656494) bank95140 +95141 POINT(40.665731532723605 -73.32949609162401) bank95141 +95142 POINT(41.20153440013594 -74.53035764316283) bank95142 +95143 POINT(41.40630202672334 -73.78425298863685) bank95143 +95144 POINT(40.355500886779126 -73.34083991185372) bank95144 +95145 POINT(40.86155937479562 -74.28030893776753) bank95145 +95146 POINT(41.43127794701653 -73.70390073886283) bank95146 +95147 POINT(40.05192577088212 -73.65541160260237) bank95147 +95148 POINT(40.218131301473626 -74.9441961387534) bank95148 +95149 POINT(41.67525492439976 -74.65356225035998) bank95149 +95150 POINT(41.55748971790725 -73.6175204515245) bank95150 +95151 POINT(40.99400378433776 -74.95593538352173) bank95151 +95152 POINT(39.934999795443204 -73.42666880109516) bank95152 +95153 POINT(40.27730237453882 -74.48410095992034) bank95153 +95154 POINT(40.33115859184305 -73.7454669596417) bank95154 +95155 POINT(40.77854043075882 -73.64785928732896) bank95155 +95156 POINT(40.1645367302592 -74.35646265434256) bank95156 +95157 POINT(40.770849721754814 -73.29210109527004) bank95157 +95158 POINT(40.39713297342752 -74.6956753700506) bank95158 +95159 POINT(40.35312514588282 -74.38260486574069) bank95159 +95160 POINT(41.6308212547729 -74.95847945895777) bank95160 +95161 POINT(40.56839102435916 -74.02550758050849) bank95161 +95162 POINT(40.97278812956772 -73.89571747025774) bank95162 +95163 POINT(40.87344747724993 -73.09755718419615) bank95163 +95164 POINT(40.30704848427956 -73.80791387154073) bank95164 +95165 POINT(41.538628116473376 -73.69648414829211) bank95165 +95166 POINT(40.94058928230464 -73.2113208605563) bank95166 +95167 POINT(40.40043183263123 -73.95129500618538) bank95167 +95168 POINT(40.04401529011795 -73.91765004407291) bank95168 +95169 POINT(40.64115178232087 -73.49856494949593) bank95169 +95170 POINT(41.684104683288275 -74.47898074959136) bank95170 +95171 POINT(41.25394623138783 -74.1047207485694) bank95171 +95172 POINT(40.793130729554335 -74.11392109586485) bank95172 +95173 POINT(40.768377860035656 -73.73461688484858) bank95173 +95174 POINT(40.70468370405861 -74.62960785686104) bank95174 +95175 POINT(40.30025719927086 -74.22411049616753) bank95175 +95176 POINT(40.99558831486621 -74.31970563960918) bank95176 +95177 POINT(40.95941898973669 -73.57990874865193) bank95177 +95178 POINT(40.625587108246805 -73.29570104557749) bank95178 +95179 POINT(40.90979061398088 -74.576263624015) bank95179 +95180 POINT(40.331511705041805 -74.40691327012898) bank95180 +95181 POINT(41.20409273289751 -73.48006251717624) bank95181 +95182 POINT(39.79567909969895 -73.94746007676882) bank95182 +95183 POINT(41.01902267200823 -73.30314748990595) bank95183 +95184 POINT(41.47062109874778 -74.07595480940395) bank95184 +95185 POINT(40.75952360526102 -73.26641622280434) bank95185 +95186 POINT(40.33157572893517 -73.71245805755044) bank95186 +95187 POINT(39.79656729503434 -74.51998772708889) bank95187 +95188 POINT(41.08455526603505 -73.42223534230283) bank95188 +95189 POINT(41.27469674891131 -74.54694011845595) bank95189 +95190 POINT(40.12920673720026 -73.4788732758297) bank95190 +95191 POINT(40.920964606051 -73.54014323910857) bank95191 +95192 POINT(41.541868669428254 -73.8168838601909) bank95192 +95193 POINT(39.98425438311767 -74.91004745609389) bank95193 +95194 POINT(39.95544884762978 -74.12849061875065) bank95194 +95195 POINT(40.33217939531931 -74.75767699386253) bank95195 +95196 POINT(41.44257292049128 -73.42972475735664) bank95196 +95197 POINT(41.088704002201624 -73.28304218908453) bank95197 +95198 POINT(41.291529421592095 -74.6542704172747) bank95198 +95199 POINT(40.11178412414063 -74.4016953339737) bank95199 +95200 POINT(40.355832907082075 -74.63095638423535) bank95200 +95201 POINT(40.478130239760766 -73.04773435227382) bank95201 +95202 POINT(40.86932050074405 -75.00002825963679) bank95202 +95203 POINT(41.07043934035628 -73.68129775556231) bank95203 +95204 POINT(39.9659310846314 -74.9355128530122) bank95204 +95205 POINT(39.87635485815982 -74.6710389944832) bank95205 +95206 POINT(41.47813894100339 -74.95920443689022) bank95206 +95207 POINT(40.45198799171962 -74.97938817869439) bank95207 +95208 POINT(41.43385444585064 -73.86759806289676) bank95208 +95209 POINT(41.25746497098728 -74.61820610132445) bank95209 +95210 POINT(40.49384827316208 -74.94529927733049) bank95210 +95211 POINT(41.32138035306931 -73.90592879175895) bank95211 +95212 POINT(40.64470373534957 -73.31907580543174) bank95212 +95213 POINT(40.49148416705397 -74.09221950906371) bank95213 +95214 POINT(41.532843263308365 -74.87329042227005) bank95214 +95215 POINT(40.83955185910851 -73.9604260408541) bank95215 +95216 POINT(40.09997800057664 -73.11892848595028) bank95216 +95217 POINT(40.60096045555213 -74.93201088788582) bank95217 +95218 POINT(41.393206739414005 -73.67019071341834) bank95218 +95219 POINT(40.99621808515602 -74.88189510199501) bank95219 +95220 POINT(41.195607164886994 -73.43592017355434) bank95220 +95221 POINT(40.55105923381225 -74.67647238940013) bank95221 +95222 POINT(39.97921118696474 -74.36228016015869) bank95222 +95223 POINT(41.44538222948285 -74.88931817805599) bank95223 +95224 POINT(40.7908040611958 -73.85384878553495) bank95224 +95225 POINT(41.08345330595039 -73.26177150219857) bank95225 +95226 POINT(40.44251332683817 -74.0678684201518) bank95226 +95227 POINT(40.82692993036119 -73.24022370435905) bank95227 +95228 POINT(40.71458028916499 -73.06920790348454) bank95228 +95229 POINT(41.49267413229157 -73.05943231140222) bank95229 +95230 POINT(39.73786961126456 -74.17785448342107) bank95230 +95231 POINT(40.84196120925327 -73.90350771443254) bank95231 +95232 POINT(40.08793590639725 -73.65389703104547) bank95232 +95233 POINT(41.56776388158028 -74.54221356239381) bank95233 +95234 POINT(41.06837470642745 -74.29267983417877) bank95234 +95235 POINT(40.53132159082092 -74.21284943402142) bank95235 +95236 POINT(39.73523599392475 -73.56348211596686) bank95236 +95237 POINT(40.90390853692994 -73.7383795565939) bank95237 +95238 POINT(40.96704436476394 -73.8773683998033) bank95238 +95239 POINT(41.49398268889757 -74.85333886203045) bank95239 +95240 POINT(40.94083398677365 -74.80873580090109) bank95240 +95241 POINT(41.65352332085189 -74.38223843779008) bank95241 +95242 POINT(40.430884820357946 -74.25440261199881) bank95242 +95243 POINT(39.7603128653366 -74.05915474924376) bank95243 +95244 POINT(41.073922841488915 -74.65309644385965) bank95244 +95245 POINT(41.425333182104204 -73.10696874156898) bank95245 +95246 POINT(41.26449847987261 -73.30494434461835) bank95246 +95247 POINT(41.64856333759015 -73.50739704305256) bank95247 +95248 POINT(40.42084591478984 -73.69113573397392) bank95248 +95249 POINT(40.630790032230664 -73.11649824911977) bank95249 +95250 POINT(39.713423284984536 -73.86895114842888) bank95250 +95251 POINT(39.992105692120106 -73.69136926930524) bank95251 +95252 POINT(40.246750265350514 -74.13605348122682) bank95252 +95253 POINT(41.410247638617676 -74.62868509751763) bank95253 +95254 POINT(41.103947021108546 -74.17635110256818) bank95254 +95255 POINT(40.336873942943726 -73.5750119580553) bank95255 +95256 POINT(40.047843646053515 -74.34531318559897) bank95256 +95257 POINT(40.24464438305316 -73.27748470916775) bank95257 +95258 POINT(39.956711672246975 -74.96803528988801) bank95258 +95259 POINT(40.16708677723973 -73.73344192894201) bank95259 +95260 POINT(41.3477376889161 -73.43584494996837) bank95260 +95261 POINT(40.10092682537527 -73.75398646168225) bank95261 +95262 POINT(41.357153794895154 -73.41067950797625) bank95262 +95263 POINT(40.6790387058619 -73.9160489647312) bank95263 +95264 POINT(40.81613637426169 -73.86315018086094) bank95264 +95265 POINT(40.08773437183228 -73.64348584679544) bank95265 +95266 POINT(40.131527205461445 -74.06370993515029) bank95266 +95267 POINT(41.42752584727274 -73.46185659670664) bank95267 +95268 POINT(39.9427919832702 -73.8440435081389) bank95268 +95269 POINT(40.80347141888052 -73.98703384366672) bank95269 +95270 POINT(40.85632973171016 -74.19110582078876) bank95270 +95271 POINT(41.446316126645314 -74.27097299056875) bank95271 +95272 POINT(39.71836598231759 -73.72362997822229) bank95272 +95273 POINT(40.035201934133084 -73.75406003835907) bank95273 +95274 POINT(40.33174917006945 -73.02698674800168) bank95274 +95275 POINT(39.75343815328149 -74.0214252161223) bank95275 +95276 POINT(40.8088083914117 -73.83642875336126) bank95276 +95277 POINT(40.313634573425766 -73.81287310183532) bank95277 +95278 POINT(40.668161801988425 -74.06521789246628) bank95278 +95279 POINT(40.17623164080983 -74.45590232696127) bank95279 +95280 POINT(40.14145993286263 -73.84240698479122) bank95280 +95281 POINT(39.94609305255912 -73.62911559856221) bank95281 +95282 POINT(41.2597959754713 -73.60280761364025) bank95282 +95283 POINT(41.142403010938644 -74.83080064615041) bank95283 +95284 POINT(41.089847216765264 -74.57634388081253) bank95284 +95285 POINT(40.022321893253824 -73.40765488976231) bank95285 +95286 POINT(40.125884182260975 -74.82064760049215) bank95286 +95287 POINT(39.72402834009054 -73.80570993769689) bank95287 +95288 POINT(40.3449721001585 -73.92356730938675) bank95288 +95289 POINT(41.59362918607697 -73.59348442243714) bank95289 +95290 POINT(41.656715781570256 -74.54265247027061) bank95290 +95291 POINT(40.5034343525449 -74.035812033699) bank95291 +95292 POINT(40.28029709880931 -74.39388790338849) bank95292 +95293 POINT(40.24981186073556 -74.61992525377885) bank95293 +95294 POINT(41.64177447855569 -74.24147471828647) bank95294 +95295 POINT(40.520642241073396 -74.20387737191658) bank95295 +95296 POINT(41.68190112483054 -73.33637716271258) bank95296 +95297 POINT(41.58206634203099 -73.39879414915904) bank95297 +95298 POINT(40.78384379849797 -74.5213908649641) bank95298 +95299 POINT(41.154587358531884 -73.26224637405228) bank95299 +95300 POINT(39.87292801317785 -73.57669282692507) bank95300 +95301 POINT(40.52439010398226 -74.44393622109811) bank95301 +95302 POINT(40.33706862644354 -74.3255332756273) bank95302 +95303 POINT(41.150835910550015 -73.30841380483847) bank95303 +95304 POINT(40.83719231096351 -74.24441965400867) bank95304 +95305 POINT(40.596790132827984 -73.1906159829546) bank95305 +95306 POINT(40.95130393594011 -73.82885302135267) bank95306 +95307 POINT(41.01269192622721 -73.40863917704183) bank95307 +95308 POINT(40.846574688718604 -74.13270303768667) bank95308 +95309 POINT(41.13032354452494 -73.55813016393145) bank95309 +95310 POINT(41.61822943867604 -73.71159952709257) bank95310 +95311 POINT(41.35363599388948 -73.77267229131816) bank95311 +95312 POINT(40.3860827946868 -73.9218046090181) bank95312 +95313 POINT(41.254004072799745 -74.31332071362407) bank95313 +95314 POINT(41.12096295434306 -74.59148407181604) bank95314 +95315 POINT(40.88147469654366 -73.96788877382942) bank95315 +95316 POINT(39.78379487252721 -74.86992142280386) bank95316 +95317 POINT(41.65287527998245 -73.11707558456536) bank95317 +95318 POINT(39.9442136220076 -74.62899840670282) bank95318 +95319 POINT(39.98725175448502 -73.31173571288149) bank95319 +95320 POINT(41.37003258153142 -73.71337081387627) bank95320 +95321 POINT(40.56184223456142 -73.7925668247388) bank95321 +95322 POINT(39.76425235726766 -73.40510259398191) bank95322 +95323 POINT(40.998399569016584 -74.41386415706879) bank95323 +95324 POINT(39.81351733417197 -73.50814923780536) bank95324 +95325 POINT(41.53442207487805 -73.95076915188025) bank95325 +95326 POINT(41.43567243991937 -73.94431603376833) bank95326 +95327 POINT(40.06091310981465 -73.64303765921483) bank95327 +95328 POINT(40.35907984929198 -74.69751849127107) bank95328 +95329 POINT(40.08296795112027 -74.94706831073407) bank95329 +95330 POINT(40.411342371540165 -73.70454772977774) bank95330 +95331 POINT(41.66732872892595 -73.39046682808286) bank95331 +95332 POINT(39.92526168253535 -73.77855135415749) bank95332 +95333 POINT(40.423273954173 -74.30457840883774) bank95333 +95334 POINT(41.040953836115996 -73.53207782869099) bank95334 +95335 POINT(41.52345537241511 -73.73568515030729) bank95335 +95336 POINT(39.98556391384312 -73.36578143733847) bank95336 +95337 POINT(40.20731499786159 -73.29119017349078) bank95337 +95338 POINT(39.78635280835378 -74.21620453494383) bank95338 +95339 POINT(39.76038412713696 -73.64687827626281) bank95339 +95340 POINT(40.47325084311376 -73.78800906134023) bank95340 +95341 POINT(40.88634452788987 -74.35946936964524) bank95341 +95342 POINT(39.7272535950634 -73.58052904246176) bank95342 +95343 POINT(40.18452694672096 -73.55757516236709) bank95343 +95344 POINT(39.79698172113792 -73.33476831426547) bank95344 +95345 POINT(40.025460592098895 -74.98001194450303) bank95345 +95346 POINT(40.21637282307222 -73.99322324971934) bank95346 +95347 POINT(41.24937289598987 -74.01431600884659) bank95347 +95348 POINT(40.54470699281474 -73.63334073059849) bank95348 +95349 POINT(40.54221317201702 -73.15231638747022) bank95349 +95350 POINT(40.67497555810746 -73.35564097203152) bank95350 +95351 POINT(40.230552382299216 -73.01216801652753) bank95351 +95352 POINT(39.85013074312204 -74.82751158483407) bank95352 +95353 POINT(40.08552227182504 -74.48909983234141) bank95353 +95354 POINT(41.00086173660377 -73.64946674798941) bank95354 +95355 POINT(40.227165890076726 -73.74270821887072) bank95355 +95356 POINT(41.57799468786802 -74.04165636652147) bank95356 +95357 POINT(40.87595004499536 -73.85504479306705) bank95357 +95358 POINT(39.93514575366422 -74.21598495614623) bank95358 +95359 POINT(40.71037698453687 -73.55365540629987) bank95359 +95360 POINT(40.991074335402054 -74.32187119644232) bank95360 +95361 POINT(40.316901894104056 -74.94282299275311) bank95361 +95362 POINT(40.433115508735426 -73.14813186848774) bank95362 +95363 POINT(40.005272803870845 -73.8828678409401) bank95363 +95364 POINT(41.42638190472632 -73.76689030048635) bank95364 +95365 POINT(41.096821109442764 -73.53184647114449) bank95365 +95366 POINT(41.121432963441265 -74.85256538073648) bank95366 +95367 POINT(41.45040878423218 -74.79316334185805) bank95367 +95368 POINT(40.148159956343946 -74.09618786095108) bank95368 +95369 POINT(41.201917046646905 -74.28988005923279) bank95369 +95370 POINT(40.95768862404604 -74.84055700581146) bank95370 +95371 POINT(41.65900328643689 -74.35451717261462) bank95371 +95372 POINT(41.542551277706835 -74.64437583435014) bank95372 +95373 POINT(41.21853903907968 -73.58437479622016) bank95373 +95374 POINT(40.256426759998774 -74.31795864180984) bank95374 +95375 POINT(41.51122293986868 -74.01013183858377) bank95375 +95376 POINT(41.43112949338898 -73.82372797827308) bank95376 +95377 POINT(40.51550627489053 -74.32348347909067) bank95377 +95378 POINT(40.5823852495482 -73.77164310015588) bank95378 +95379 POINT(40.31129791893428 -73.99337308157592) bank95379 +95380 POINT(40.09850675881079 -73.4792298055252) bank95380 +95381 POINT(41.37192483192078 -74.7057179190459) bank95381 +95382 POINT(40.329483467372384 -74.20407192445795) bank95382 +95383 POINT(40.46783125288644 -73.30673747355982) bank95383 +95384 POINT(40.17711949098151 -73.09600644922706) bank95384 +95385 POINT(40.06170917676778 -73.60692924626231) bank95385 +95386 POINT(40.87237200255265 -73.43228042884255) bank95386 +95387 POINT(40.28416645261931 -74.37515972795983) bank95387 +95388 POINT(40.723516970128856 -73.44872647277285) bank95388 +95389 POINT(40.16019476500953 -73.32804055307037) bank95389 +95390 POINT(40.79842015151853 -73.29301651450949) bank95390 +95391 POINT(41.01985088535833 -73.62573860538679) bank95391 +95392 POINT(41.612999395477985 -74.81403917824488) bank95392 +95393 POINT(41.016342774668374 -73.89714261307344) bank95393 +95394 POINT(41.043075903460405 -73.17519592411014) bank95394 +95395 POINT(39.775247702653395 -74.95538908988094) bank95395 +95396 POINT(39.795499328011886 -74.76272520164058) bank95396 +95397 POINT(40.51179613311691 -73.77818361615279) bank95397 +95398 POINT(40.16018751725271 -73.61199746117794) bank95398 +95399 POINT(40.106060598023646 -74.09440142173221) bank95399 +95400 POINT(40.751617891092955 -74.95624311277012) bank95400 +95401 POINT(40.34960648060916 -74.9742537954086) bank95401 +95402 POINT(40.24711298544504 -74.44132157477655) bank95402 +95403 POINT(40.84306576588623 -74.52432593228629) bank95403 +95404 POINT(41.563740112558406 -73.98855038487534) bank95404 +95405 POINT(39.900215786026955 -73.96225415088325) bank95405 +95406 POINT(40.32579527331709 -73.18862650760586) bank95406 +95407 POINT(41.651862074395595 -74.00230391044833) bank95407 +95408 POINT(40.1120777200905 -73.4671261536202) bank95408 +95409 POINT(39.929697611649416 -73.26516241196177) bank95409 +95410 POINT(41.68136081958263 -73.29714327456777) bank95410 +95411 POINT(41.47175818773217 -74.01322667408664) bank95411 +95412 POINT(40.031737359714796 -73.5879035661715) bank95412 +95413 POINT(40.305057221134625 -73.09134725627008) bank95413 +95414 POINT(41.23759762963179 -73.59749556758582) bank95414 +95415 POINT(40.51911974246141 -73.02009834050645) bank95415 +95416 POINT(39.998653008999646 -73.24282852128925) bank95416 +95417 POINT(41.65549033334907 -74.99922536532226) bank95417 +95418 POINT(40.37140044738541 -73.6262960314492) bank95418 +95419 POINT(40.49875335403996 -74.03843222501054) bank95419 +95420 POINT(41.070068185246555 -74.60486639391118) bank95420 +95421 POINT(40.61037440114926 -74.55122763049326) bank95421 +95422 POINT(41.56067338763726 -73.00867672860805) bank95422 +95423 POINT(40.38875694966994 -74.3779257093204) bank95423 +95424 POINT(39.86645964674229 -74.76593488685762) bank95424 +95425 POINT(40.20681428076538 -74.67010578263614) bank95425 +95426 POINT(41.079693818759154 -74.12106924716433) bank95426 +95427 POINT(40.080096136001806 -74.60085966731285) bank95427 +95428 POINT(40.79387125433403 -73.07476258655056) bank95428 +95429 POINT(40.09704789883784 -73.02506831447191) bank95429 +95430 POINT(40.54788690602809 -73.54881121739758) bank95430 +95431 POINT(40.22087614531249 -73.04108600569697) bank95431 +95432 POINT(41.36912552315483 -73.91517489747831) bank95432 +95433 POINT(41.05412200541947 -73.07283219596171) bank95433 +95434 POINT(41.645821344730045 -74.37126241133433) bank95434 +95435 POINT(39.76811930278228 -74.82891322400668) bank95435 +95436 POINT(41.00296822976125 -73.67789899495982) bank95436 +95437 POINT(41.076543675273626 -74.416739957476) bank95437 +95438 POINT(41.30709812990544 -73.70640319338163) bank95438 +95439 POINT(40.69257024650807 -74.43965384687473) bank95439 +95440 POINT(40.524020360502746 -74.30779393738472) bank95440 +95441 POINT(40.3536985454617 -73.72574223801452) bank95441 +95442 POINT(40.40052831759443 -74.64804888914924) bank95442 +95443 POINT(41.42052768387063 -74.94102852395416) bank95443 +95444 POINT(40.19517816311083 -73.98144978281175) bank95444 +95445 POINT(40.27214882334238 -73.53207419676778) bank95445 +95446 POINT(41.52989863608079 -74.8951856543929) bank95446 +95447 POINT(40.69400661723035 -74.98270958720816) bank95447 +95448 POINT(41.691807170133195 -73.98449239589822) bank95448 +95449 POINT(41.57964715050793 -74.55057741728372) bank95449 +95450 POINT(41.567503395022825 -73.69859384500198) bank95450 +95451 POINT(41.11838420823081 -74.93128995537995) bank95451 +95452 POINT(40.873706079131885 -74.73645342871467) bank95452 +95453 POINT(40.51696103105814 -73.70286488325158) bank95453 +95454 POINT(40.48568807929593 -74.71701666492196) bank95454 +95455 POINT(39.999340922660174 -73.65596618905799) bank95455 +95456 POINT(40.56089166118308 -74.75224248150458) bank95456 +95457 POINT(40.965277451057766 -73.17981932627191) bank95457 +95458 POINT(41.09841894710712 -74.4116307842313) bank95458 +95459 POINT(41.36060863652994 -73.84487295024005) bank95459 +95460 POINT(40.97166400686571 -74.6647096555807) bank95460 +95461 POINT(40.390545949947494 -73.01823955957583) bank95461 +95462 POINT(40.860831496048526 -73.4241284477866) bank95462 +95463 POINT(39.89802748169455 -73.89056827972148) bank95463 +95464 POINT(40.65366979596412 -73.03631998251979) bank95464 +95465 POINT(40.88396709660305 -74.81634746107322) bank95465 +95466 POINT(39.855771326318305 -73.36495729198322) bank95466 +95467 POINT(40.03937411690691 -74.76496585577912) bank95467 +95468 POINT(39.841600808447396 -73.42683642383872) bank95468 +95469 POINT(41.60926274644344 -74.54124771213478) bank95469 +95470 POINT(41.087633313387826 -74.44167127504457) bank95470 +95471 POINT(40.38435325180485 -74.58436524363947) bank95471 +95472 POINT(40.372930268484225 -73.44465528738148) bank95472 +95473 POINT(40.18148056306957 -73.92638160155947) bank95473 +95474 POINT(39.81279610668256 -73.53251742610071) bank95474 +95475 POINT(39.84335541991394 -74.01592560211724) bank95475 +95476 POINT(41.446396273097136 -74.06707870744295) bank95476 +95477 POINT(40.233343427189716 -74.79814103409106) bank95477 +95478 POINT(41.41219816659839 -74.7950782960556) bank95478 +95479 POINT(40.161561522416946 -74.39546895000828) bank95479 +95480 POINT(39.75891033719379 -74.62007896430065) bank95480 +95481 POINT(40.40676362412005 -73.5266395992376) bank95481 +95482 POINT(40.22839270108095 -73.88826462066717) bank95482 +95483 POINT(40.756109125106406 -74.1887318788876) bank95483 +95484 POINT(40.07753311577601 -73.61814653381342) bank95484 +95485 POINT(41.66429245616608 -74.84616442064942) bank95485 +95486 POINT(40.81100413325943 -73.6598960818779) bank95486 +95487 POINT(39.932063645512834 -73.04539243471994) bank95487 +95488 POINT(41.58474944347562 -74.6404969945097) bank95488 +95489 POINT(41.207975589975135 -73.23376311225007) bank95489 +95490 POINT(39.71719545827761 -74.7279255022161) bank95490 +95491 POINT(40.53866937371514 -74.61644484755234) bank95491 +95492 POINT(41.26757317506013 -73.4092101973281) bank95492 +95493 POINT(39.89801998147657 -74.53232942721056) bank95493 +95494 POINT(41.09094235654434 -73.21003538075149) bank95494 +95495 POINT(41.57467818814009 -74.90742884395841) bank95495 +95496 POINT(39.96533714773059 -73.6417604865915) bank95496 +95497 POINT(40.90016268265204 -73.5234338635444) bank95497 +95498 POINT(40.055374632013496 -74.63034662495498) bank95498 +95499 POINT(40.776922218292945 -73.64255158601759) bank95499 +95500 POINT(40.16055083488427 -74.7105647483905) bank95500 +95501 POINT(40.259074207588625 -73.85907844619196) bank95501 +95502 POINT(40.09221549958679 -74.40049564066845) bank95502 +95503 POINT(40.989851543634956 -74.13543755233991) bank95503 +95504 POINT(40.50433015294193 -74.41710930863267) bank95504 +95505 POINT(40.127854546540846 -74.83245968357251) bank95505 +95506 POINT(40.8036921582547 -74.80769736931758) bank95506 +95507 POINT(41.08687224898325 -73.93745920637053) bank95507 +95508 POINT(40.60064696872689 -74.83111249491786) bank95508 +95509 POINT(41.44248772594852 -73.73713427508191) bank95509 +95510 POINT(39.743354514791584 -73.21139818059277) bank95510 +95511 POINT(40.85828247657746 -73.87843897114607) bank95511 +95512 POINT(40.68490924027832 -74.75465502686076) bank95512 +95513 POINT(40.97427973060288 -74.63947638212078) bank95513 +95514 POINT(40.15733521167415 -74.42841508112032) bank95514 +95515 POINT(40.14372963381198 -74.00340466074846) bank95515 +95516 POINT(40.08938269045947 -74.03967746182089) bank95516 +95517 POINT(40.13185819605677 -73.01220613255425) bank95517 +95518 POINT(40.57244636249936 -74.80285704364319) bank95518 +95519 POINT(39.75822051532519 -74.00705797245219) bank95519 +95520 POINT(40.98553220555803 -73.0739315062747) bank95520 +95521 POINT(40.36832267124201 -73.78948816228775) bank95521 +95522 POINT(41.540139200703756 -73.88936021919542) bank95522 +95523 POINT(41.32587037400139 -73.15827830899848) bank95523 +95524 POINT(39.81530620850032 -74.9176819346177) bank95524 +95525 POINT(41.40451684634936 -74.4858619085412) bank95525 +95526 POINT(40.02915165426975 -73.41907332231936) bank95526 +95527 POINT(40.00054838861708 -73.49727620216339) bank95527 +95528 POINT(41.23712645309155 -74.35637605337199) bank95528 +95529 POINT(40.55152974901069 -74.33721715810981) bank95529 +95530 POINT(40.277598635141025 -74.48313541102024) bank95530 +95531 POINT(41.446616841347875 -74.97982510246524) bank95531 +95532 POINT(41.19549457659063 -74.49696346890937) bank95532 +95533 POINT(40.1284483855011 -74.67569323010504) bank95533 +95534 POINT(40.53443593538073 -74.8159620218617) bank95534 +95535 POINT(39.7425844708706 -74.22313479545663) bank95535 +95536 POINT(41.42323908901976 -73.67271521365109) bank95536 +95537 POINT(39.86165719709665 -74.89962132514532) bank95537 +95538 POINT(40.03650825174912 -73.36301696924822) bank95538 +95539 POINT(39.72403463512776 -73.60554976285296) bank95539 +95540 POINT(39.72447417725547 -73.66458286011083) bank95540 +95541 POINT(40.90314736723724 -73.84108955654366) bank95541 +95542 POINT(41.58267096466246 -73.33275540429798) bank95542 +95543 POINT(41.124877662852214 -74.2048939820766) bank95543 +95544 POINT(41.29545793974504 -73.3884441664809) bank95544 +95545 POINT(40.756143950807164 -73.32773261369815) bank95545 +95546 POINT(40.60591234929164 -73.26666100395491) bank95546 +95547 POINT(41.3391633084399 -73.68383698152596) bank95547 +95548 POINT(41.507103464994934 -74.48768483812874) bank95548 +95549 POINT(39.86764896651123 -74.28696565678715) bank95549 +95550 POINT(41.2294549293869 -74.99210175239966) bank95550 +95551 POINT(40.31661553276516 -73.26598781477045) bank95551 +95552 POINT(41.53029085592499 -73.14009295502085) bank95552 +95553 POINT(41.06322514592362 -73.49861619006155) bank95553 +95554 POINT(39.71852231084724 -74.88234782876118) bank95554 +95555 POINT(41.489186788044194 -74.62309184808038) bank95555 +95556 POINT(40.16633557111624 -74.04622250645924) bank95556 +95557 POINT(40.77732972592013 -73.95605985340411) bank95557 +95558 POINT(41.3110793082836 -73.28468577849904) bank95558 +95559 POINT(40.684234566755265 -74.46752447213616) bank95559 +95560 POINT(41.46772450788988 -73.80775332360311) bank95560 +95561 POINT(41.00973043469552 -73.46693439503969) bank95561 +95562 POINT(41.11227237980302 -74.38368890677611) bank95562 +95563 POINT(41.37432431964974 -74.82344903479255) bank95563 +95564 POINT(41.33163739641626 -74.14017055124756) bank95564 +95565 POINT(41.05176840756963 -73.27975014987133) bank95565 +95566 POINT(40.81829942262417 -74.52758303359556) bank95566 +95567 POINT(41.458154896572815 -73.14041474406531) bank95567 +95568 POINT(41.502484184047916 -73.98505944844437) bank95568 +95569 POINT(41.3902084682774 -73.51733804225498) bank95569 +95570 POINT(41.42812036084799 -73.25273689770621) bank95570 +95571 POINT(40.91286817024454 -74.1520527470377) bank95571 +95572 POINT(41.70773524070776 -74.67936678438771) bank95572 +95573 POINT(40.07021702409249 -74.73440684573936) bank95573 +95574 POINT(40.05384796973493 -73.44679957275933) bank95574 +95575 POINT(40.64638370088342 -73.44591107038151) bank95575 +95576 POINT(40.04458190984282 -74.00064266844953) bank95576 +95577 POINT(41.5424847829055 -74.45396531058762) bank95577 +95578 POINT(40.69471758995955 -74.86017961140811) bank95578 +95579 POINT(40.88331318505987 -74.98537814211821) bank95579 +95580 POINT(41.51095704453614 -74.2307698042273) bank95580 +95581 POINT(40.905772086810984 -73.95947939866605) bank95581 +95582 POINT(41.503022464592256 -73.69331713533879) bank95582 +95583 POINT(41.26394521052814 -74.64611692946741) bank95583 +95584 POINT(40.583087810623816 -73.2720212299876) bank95584 +95585 POINT(41.06095681256565 -74.63778646582453) bank95585 +95586 POINT(40.264016282735305 -73.74185640074168) bank95586 +95587 POINT(41.18080454441571 -73.76273661472601) bank95587 +95588 POINT(41.4441373554781 -74.91214033043106) bank95588 +95589 POINT(39.88891122162191 -74.2926107698965) bank95589 +95590 POINT(40.20842025911128 -73.26240924501572) bank95590 +95591 POINT(40.80917729090068 -74.81928987264789) bank95591 +95592 POINT(41.48671032530674 -74.9548222807788) bank95592 +95593 POINT(41.53392782648645 -74.66458119690157) bank95593 +95594 POINT(40.759358097373436 -73.28516740240914) bank95594 +95595 POINT(39.838239552215136 -73.66543810247067) bank95595 +95596 POINT(41.230166566587656 -73.01087089221069) bank95596 +95597 POINT(40.46575415957835 -73.85134275925434) bank95597 +95598 POINT(41.24144857573907 -74.01740345312653) bank95598 +95599 POINT(40.09842678686583 -73.19712737967963) bank95599 +95600 POINT(40.584344896285224 -73.25131732671213) bank95600 +95601 POINT(40.624607678222894 -73.26457961847422) bank95601 +95602 POINT(40.6892552762343 -73.5315861298505) bank95602 +95603 POINT(41.472022135965744 -74.03995343655649) bank95603 +95604 POINT(41.53419256487108 -73.69004034234193) bank95604 +95605 POINT(41.238525725848355 -73.02715814040637) bank95605 +95606 POINT(40.03692138407382 -74.69505360174254) bank95606 +95607 POINT(39.773685934136154 -74.76740011377579) bank95607 +95608 POINT(39.73858163237352 -73.65006513404313) bank95608 +95609 POINT(41.12105558335807 -74.52716267168434) bank95609 +95610 POINT(41.05790300354856 -73.99478360980717) bank95610 +95611 POINT(41.61688795787301 -73.74291255722022) bank95611 +95612 POINT(41.277936972099404 -74.76086603326993) bank95612 +95613 POINT(39.95545895698512 -73.78420525087951) bank95613 +95614 POINT(41.026615476600206 -74.24365078414657) bank95614 +95615 POINT(40.030061925338025 -73.93429115490146) bank95615 +95616 POINT(40.02969928937529 -74.02200878569995) bank95616 +95617 POINT(40.66062085049282 -73.4235351883088) bank95617 +95618 POINT(40.47700512868016 -74.79010712876931) bank95618 +95619 POINT(39.72933574295828 -73.69044137499328) bank95619 +95620 POINT(41.66688729335189 -73.65181156012163) bank95620 +95621 POINT(40.24258375280462 -74.92284799929344) bank95621 +95622 POINT(41.37954747156678 -73.1913922079184) bank95622 +95623 POINT(41.615304682587656 -74.25607858738694) bank95623 +95624 POINT(40.689820657382036 -73.40905557440303) bank95624 +95625 POINT(40.56278137066429 -73.95573572097777) bank95625 +95626 POINT(40.84633186526811 -74.21057978978092) bank95626 +95627 POINT(40.815560100560255 -73.19673006059986) bank95627 +95628 POINT(41.02414737599937 -74.69342501496321) bank95628 +95629 POINT(40.47671499675451 -73.29751758815169) bank95629 +95630 POINT(40.741102716466024 -73.41516288481695) bank95630 +95631 POINT(40.11921716317854 -74.09721935092907) bank95631 +95632 POINT(40.20469527803952 -74.24423974731238) bank95632 +95633 POINT(41.46509225381065 -74.51136461906216) bank95633 +95634 POINT(41.46706485783241 -74.69647455925812) bank95634 +95635 POINT(39.81525692978919 -73.27424089889583) bank95635 +95636 POINT(40.35480006595489 -74.02546876827564) bank95636 +95637 POINT(40.99666716981961 -73.44792341931002) bank95637 +95638 POINT(40.91413810395925 -73.9864376082967) bank95638 +95639 POINT(40.74135347579349 -73.98616890541435) bank95639 +95640 POINT(40.38091689485212 -74.43611992623079) bank95640 +95641 POINT(40.96863650563662 -73.40233042947133) bank95641 +95642 POINT(40.28331282700708 -74.55347746683498) bank95642 +95643 POINT(41.36300572295638 -73.18168135596179) bank95643 +95644 POINT(40.04460824428885 -74.70649440880979) bank95644 +95645 POINT(40.219550916236365 -73.90079815309834) bank95645 +95646 POINT(40.24577756663123 -73.28442308266561) bank95646 +95647 POINT(40.27456214197921 -73.77678716241617) bank95647 +95648 POINT(40.04344577733491 -74.72975689405538) bank95648 +95649 POINT(41.08322670200576 -73.60309395909513) bank95649 +95650 POINT(39.8604046517094 -74.0190384168012) bank95650 +95651 POINT(40.10903133121003 -73.47172260227534) bank95651 +95652 POINT(41.56100682661378 -74.04212190587735) bank95652 +95653 POINT(39.94309222557437 -74.81995983808545) bank95653 +95654 POINT(41.3734729469046 -73.04166974390483) bank95654 +95655 POINT(41.43909458502266 -73.08007714696932) bank95655 +95656 POINT(40.34790549512154 -73.8446693244851) bank95656 +95657 POINT(39.81220373006177 -73.80035627746099) bank95657 +95658 POINT(41.222183823059325 -73.65785376357056) bank95658 +95659 POINT(41.65799333551874 -74.49018597884957) bank95659 +95660 POINT(40.817114851014225 -73.49855585457257) bank95660 +95661 POINT(39.84353530583778 -73.47030199457475) bank95661 +95662 POINT(41.21211034950901 -74.13383885462794) bank95662 +95663 POINT(41.199762510472105 -74.15484669814536) bank95663 +95664 POINT(40.19000413538886 -73.81758613479708) bank95664 +95665 POINT(41.46868474282171 -74.8157335703106) bank95665 +95666 POINT(41.30692886139072 -73.63227951656087) bank95666 +95667 POINT(41.14578674550869 -74.09835680209468) bank95667 +95668 POINT(40.81748549255935 -73.90448374762383) bank95668 +95669 POINT(40.405038740639185 -74.44374803587061) bank95669 +95670 POINT(39.987951754003745 -73.64879324904722) bank95670 +95671 POINT(41.49955760191976 -73.05141936591936) bank95671 +95672 POINT(40.06996360706892 -74.68997790994698) bank95672 +95673 POINT(41.32406343342633 -73.80209357083248) bank95673 +95674 POINT(40.097792696121275 -74.01080520820568) bank95674 +95675 POINT(39.77368392836619 -74.72146941910854) bank95675 +95676 POINT(40.20141406613557 -74.54207025385139) bank95676 +95677 POINT(40.49929356782228 -74.23766122589164) bank95677 +95678 POINT(40.627918220493754 -74.13480327207147) bank95678 +95679 POINT(40.75912600941035 -74.47468827652824) bank95679 +95680 POINT(39.930944411119704 -74.64278306347984) bank95680 +95681 POINT(40.198386025481035 -74.4392040749459) bank95681 +95682 POINT(39.85934919368119 -74.39181184882595) bank95682 +95683 POINT(40.16907393159235 -73.11629460384289) bank95683 +95684 POINT(41.24656348509927 -74.40953760305904) bank95684 +95685 POINT(41.360625885432874 -73.27110785974719) bank95685 +95686 POINT(41.687470845431136 -74.42351214723665) bank95686 +95687 POINT(41.48695487443675 -73.02936599375667) bank95687 +95688 POINT(41.11592478887557 -73.30224666672008) bank95688 +95689 POINT(40.27576324482631 -74.65985655381127) bank95689 +95690 POINT(41.39294143686643 -74.38208076979977) bank95690 +95691 POINT(40.42733787804011 -73.45503632994696) bank95691 +95692 POINT(39.85191923013996 -74.47796692262692) bank95692 +95693 POINT(40.22909210824964 -74.94100592865962) bank95693 +95694 POINT(41.696830412917876 -74.66498074103178) bank95694 +95695 POINT(40.56832539286099 -74.2525358428169) bank95695 +95696 POINT(41.541590041222136 -73.90229668748506) bank95696 +95697 POINT(41.23593751246151 -73.89616523566123) bank95697 +95698 POINT(41.419525446494966 -74.26201982860186) bank95698 +95699 POINT(40.58530717762657 -73.92834950465063) bank95699 +95700 POINT(41.23553060045453 -74.3252395633979) bank95700 +95701 POINT(39.79212017173757 -73.2135626130694) bank95701 +95702 POINT(40.77548184307257 -74.60688841066634) bank95702 +95703 POINT(40.17200169987896 -74.75412781329949) bank95703 +95704 POINT(40.795011448018585 -74.1914870321394) bank95704 +95705 POINT(40.676209563629435 -73.53217647619314) bank95705 +95706 POINT(40.531359637907116 -73.47070217997475) bank95706 +95707 POINT(40.354715822277306 -74.36449400609496) bank95707 +95708 POINT(40.21571151904792 -74.0652088411635) bank95708 +95709 POINT(40.656019313215886 -74.20884841810395) bank95709 +95710 POINT(41.63105258142987 -73.02994365369939) bank95710 +95711 POINT(41.451303700905704 -74.48911304021256) bank95711 +95712 POINT(40.412301950841965 -73.7742963740246) bank95712 +95713 POINT(41.594749224652055 -74.81729991958073) bank95713 +95714 POINT(41.51155046465391 -74.88294554302892) bank95714 +95715 POINT(40.90862561504238 -74.17570684243299) bank95715 +95716 POINT(41.00396492503288 -73.4659188914974) bank95716 +95717 POINT(40.053055555006836 -73.65103162869474) bank95717 +95718 POINT(40.81085195046509 -73.20653242756383) bank95718 +95719 POINT(41.021384290179085 -73.96241882934946) bank95719 +95720 POINT(41.26176574259739 -73.8022372288396) bank95720 +95721 POINT(41.198400256100214 -73.87629176687031) bank95721 +95722 POINT(39.96670228208187 -73.14139607319902) bank95722 +95723 POINT(40.152810253141126 -74.90859235840018) bank95723 +95724 POINT(41.19375002982919 -74.86075807318237) bank95724 +95725 POINT(40.20899749895182 -74.77666239534172) bank95725 +95726 POINT(40.90185212247287 -74.27555011374852) bank95726 +95727 POINT(41.35247155004286 -73.51803394240012) bank95727 +95728 POINT(39.911668606753395 -73.49827273187391) bank95728 +95729 POINT(41.70050812019794 -74.35655709125939) bank95729 +95730 POINT(40.976419926671376 -74.91609712812667) bank95730 +95731 POINT(40.542567473344704 -73.1218007576814) bank95731 +95732 POINT(39.76196350580618 -73.80500196402848) bank95732 +95733 POINT(40.13990771378093 -73.38868747295419) bank95733 +95734 POINT(39.90653209692981 -73.08092190044846) bank95734 +95735 POINT(41.14999950506822 -73.96541337753024) bank95735 +95736 POINT(41.44754400658359 -73.68227955839544) bank95736 +95737 POINT(40.677903546042 -74.11702859108728) bank95737 +95738 POINT(40.10376876173232 -74.71085504400723) bank95738 +95739 POINT(41.40833951705173 -74.67085437683244) bank95739 +95740 POINT(39.879867875154986 -73.1547582322824) bank95740 +95741 POINT(40.31524046933886 -74.992297037359) bank95741 +95742 POINT(41.093815708692794 -74.52287366190461) bank95742 +95743 POINT(40.56211915207759 -74.04115584773916) bank95743 +95744 POINT(40.04595188277781 -73.08341344884234) bank95744 +95745 POINT(39.85458417976489 -74.40806929816611) bank95745 +95746 POINT(41.675438326431156 -73.15678772240707) bank95746 +95747 POINT(39.723721241078714 -73.74765847056233) bank95747 +95748 POINT(41.00907007745981 -73.2550651454433) bank95748 +95749 POINT(40.2662475612745 -73.81491854355765) bank95749 +95750 POINT(41.665563316235364 -74.5925888417536) bank95750 +95751 POINT(41.18628802384961 -74.25376415867478) bank95751 +95752 POINT(39.982723748469695 -74.97746673185992) bank95752 +95753 POINT(40.83140782225678 -74.82322144311831) bank95753 +95754 POINT(40.12862082508912 -73.22913233599004) bank95754 +95755 POINT(40.0924244375414 -74.24319799547771) bank95755 +95756 POINT(41.13058428565764 -74.44495891346918) bank95756 +95757 POINT(41.304451341129585 -73.08055477917583) bank95757 +95758 POINT(40.98588918810042 -73.00700421051884) bank95758 +95759 POINT(40.475682933536746 -74.1353854316838) bank95759 +95760 POINT(39.804199549602856 -74.1997140610423) bank95760 +95761 POINT(41.582355097126346 -74.02128374084975) bank95761 +95762 POINT(40.18047920496849 -73.96488578191475) bank95762 +95763 POINT(40.5267136244483 -73.04427874288231) bank95763 +95764 POINT(41.04004002289275 -74.48636147552098) bank95764 +95765 POINT(41.051751020607334 -74.25673135396467) bank95765 +95766 POINT(40.22566474308052 -74.4663977799919) bank95766 +95767 POINT(39.916545716429084 -74.3958968594541) bank95767 +95768 POINT(40.91486441356238 -74.76269632946152) bank95768 +95769 POINT(40.86590480269258 -74.44652017563155) bank95769 +95770 POINT(41.33374781206633 -73.51508770737759) bank95770 +95771 POINT(40.292650401098996 -73.80134404329631) bank95771 +95772 POINT(41.38717884421812 -73.69497154551532) bank95772 +95773 POINT(41.61902381386753 -73.66758240593101) bank95773 +95774 POINT(41.19734303593547 -74.02902509871136) bank95774 +95775 POINT(41.53139781350698 -73.09569361025093) bank95775 +95776 POINT(40.70883835318747 -74.50811109140184) bank95776 +95777 POINT(41.22677898477153 -74.6044133759789) bank95777 +95778 POINT(41.53853167727938 -73.35080819702016) bank95778 +95779 POINT(39.823143017821906 -73.53323389890066) bank95779 +95780 POINT(39.806083843235236 -74.73883535022193) bank95780 +95781 POINT(40.62488607574053 -74.5807337975814) bank95781 +95782 POINT(40.279876050613794 -73.83981260116724) bank95782 +95783 POINT(40.61071262687326 -74.8228492807192) bank95783 +95784 POINT(40.38602646557291 -73.64500616433789) bank95784 +95785 POINT(39.97353077828679 -73.39891228349556) bank95785 +95786 POINT(41.23032597489405 -73.31298400649077) bank95786 +95787 POINT(40.16914062647195 -74.36145684242729) bank95787 +95788 POINT(40.0158835554805 -73.4775676204727) bank95788 +95789 POINT(39.889930794995756 -74.23974822894165) bank95789 +95790 POINT(40.75749128213434 -73.00975746159952) bank95790 +95791 POINT(40.94854733987028 -74.12771212474831) bank95791 +95792 POINT(41.22365913430108 -74.16846476636309) bank95792 +95793 POINT(40.996739319057255 -73.38949955614034) bank95793 +95794 POINT(40.00302576579518 -74.38386266835111) bank95794 +95795 POINT(41.5106565987044 -73.6709292687476) bank95795 +95796 POINT(41.257381709857086 -73.69522697568091) bank95796 +95797 POINT(39.849753900500666 -74.18038948020835) bank95797 +95798 POINT(41.222468077324 -73.78854462725434) bank95798 +95799 POINT(39.77839855343696 -73.85041081664374) bank95799 +95800 POINT(40.9777378981959 -74.87889947953963) bank95800 +95801 POINT(41.543852418005535 -73.3161787668617) bank95801 +95802 POINT(40.46362766914844 -73.54411292845053) bank95802 +95803 POINT(41.416609673856364 -73.20117626749052) bank95803 +95804 POINT(40.78372532515418 -73.43705838711486) bank95804 +95805 POINT(40.91727431980581 -73.05437063652947) bank95805 +95806 POINT(40.406711741160535 -73.34153385640045) bank95806 +95807 POINT(40.132800084916674 -74.84233188066916) bank95807 +95808 POINT(40.62305794787102 -73.21348827570534) bank95808 +95809 POINT(39.813384463567644 -74.4316618303312) bank95809 +95810 POINT(40.631197642165226 -73.20458167332853) bank95810 +95811 POINT(40.889758677723655 -74.33047264987408) bank95811 +95812 POINT(40.202081403268814 -74.60481434009824) bank95812 +95813 POINT(40.892522340085954 -73.38366505880106) bank95813 +95814 POINT(40.536719564729424 -73.14710528620319) bank95814 +95815 POINT(40.872833774540055 -74.51588261291823) bank95815 +95816 POINT(40.509384003413636 -73.59169972574794) bank95816 +95817 POINT(41.604258780848156 -73.44570517190299) bank95817 +95818 POINT(41.01024639485798 -73.67748440975228) bank95818 +95819 POINT(40.32912328592955 -74.87000085423057) bank95819 +95820 POINT(40.61461622826837 -73.3165808898187) bank95820 +95821 POINT(41.592110267009716 -74.32556081082781) bank95821 +95822 POINT(41.66066397769681 -73.82012289306742) bank95822 +95823 POINT(39.78310874555338 -73.19751096134543) bank95823 +95824 POINT(41.34310715387057 -73.63432633689574) bank95824 +95825 POINT(41.21131673968318 -73.48306306109238) bank95825 +95826 POINT(39.98327219543201 -74.4101618009685) bank95826 +95827 POINT(40.68777707574444 -74.67502449581303) bank95827 +95828 POINT(41.516834759482684 -73.43528499701542) bank95828 +95829 POINT(39.88275708530149 -73.4430484010339) bank95829 +95830 POINT(40.0570669451978 -73.40149911806189) bank95830 +95831 POINT(39.74619320493227 -73.20583451223274) bank95831 +95832 POINT(41.427618656628724 -74.09585835322311) bank95832 +95833 POINT(39.92687330849427 -74.51743856959537) bank95833 +95834 POINT(40.81520751983463 -74.9278367270523) bank95834 +95835 POINT(41.4155320475448 -74.51279218743193) bank95835 +95836 POINT(41.04818174827956 -73.0877142322171) bank95836 +95837 POINT(40.005887468039866 -73.45893379290622) bank95837 +95838 POINT(41.2911201214379 -73.10044754574588) bank95838 +95839 POINT(41.06332377995356 -74.60164478833987) bank95839 +95840 POINT(40.76597373918595 -74.80937778817332) bank95840 +95841 POINT(41.61222492344921 -73.78569453599545) bank95841 +95842 POINT(40.25549753671141 -73.69146785596003) bank95842 +95843 POINT(40.310013787149565 -73.39565905011723) bank95843 +95844 POINT(40.9406935906893 -74.87238040009863) bank95844 +95845 POINT(40.66903277790751 -74.50949550925341) bank95845 +95846 POINT(40.34801450894927 -73.18127950228656) bank95846 +95847 POINT(40.69989462263185 -73.49074301659424) bank95847 +95848 POINT(41.30500537567532 -73.94365991398578) bank95848 +95849 POINT(41.170864966506734 -73.21085576677139) bank95849 +95850 POINT(40.38329514668293 -73.94652496988633) bank95850 +95851 POINT(40.862094331639675 -74.8543306278297) bank95851 +95852 POINT(40.07135061037435 -74.30239863336018) bank95852 +95853 POINT(40.02491766228278 -74.26161290205029) bank95853 +95854 POINT(40.00775644989408 -74.88847757826177) bank95854 +95855 POINT(40.16690642944881 -73.72852248681139) bank95855 +95856 POINT(41.34794525447878 -73.44623888587813) bank95856 +95857 POINT(40.98675481281332 -74.13497194277694) bank95857 +95858 POINT(40.03142965171508 -74.82208505325536) bank95858 +95859 POINT(40.9569405782806 -74.07971608412926) bank95859 +95860 POINT(40.604541631799925 -73.31787794512583) bank95860 +95861 POINT(41.39104570906946 -74.87972195674966) bank95861 +95862 POINT(41.59229342330118 -73.71435700682706) bank95862 +95863 POINT(41.62322237034883 -73.70330040707361) bank95863 +95864 POINT(41.2174467308205 -74.77421743251423) bank95864 +95865 POINT(39.73647978355442 -74.21651991584359) bank95865 +95866 POINT(41.550216638632065 -74.77675129282181) bank95866 +95867 POINT(41.00262481692961 -74.15081045926404) bank95867 +95868 POINT(40.133725925984876 -73.7790068548576) bank95868 +95869 POINT(39.85281230591859 -74.34999269759207) bank95869 +95870 POINT(41.12094453144019 -74.69983611537175) bank95870 +95871 POINT(40.00240240836106 -73.94162206416755) bank95871 +95872 POINT(41.60369454280156 -74.37728276015437) bank95872 +95873 POINT(39.83469073129626 -74.11551991264758) bank95873 +95874 POINT(40.29690194932611 -74.14348680888945) bank95874 +95875 POINT(40.151759305831554 -74.40157797753865) bank95875 +95876 POINT(41.45315269203684 -73.35231724058654) bank95876 +95877 POINT(41.435176745313434 -73.74858055462359) bank95877 +95878 POINT(40.132293739968134 -74.94359953128736) bank95878 +95879 POINT(41.349568406324174 -73.0196218100747) bank95879 +95880 POINT(41.2730827236265 -74.27068459663487) bank95880 +95881 POINT(40.33217777053653 -73.23309451752937) bank95881 +95882 POINT(39.79074080373127 -74.55765254754415) bank95882 +95883 POINT(39.81293724933709 -74.21532677828037) bank95883 +95884 POINT(41.280593808614306 -73.45740461641576) bank95884 +95885 POINT(40.92090087254694 -74.70101031322558) bank95885 +95886 POINT(40.90376064273223 -74.66736437479753) bank95886 +95887 POINT(40.673441689050335 -74.98435529722427) bank95887 +95888 POINT(40.095311376625965 -74.48892073148751) bank95888 +95889 POINT(41.67355432412273 -73.22602798893986) bank95889 +95890 POINT(40.490534074946765 -74.58897214035422) bank95890 +95891 POINT(40.61432155800899 -74.21501806833896) bank95891 +95892 POINT(40.556247714259136 -73.64211378737821) bank95892 +95893 POINT(41.43959485623538 -73.70834333051236) bank95893 +95894 POINT(39.95463227239177 -74.18273339266476) bank95894 +95895 POINT(40.52650318012395 -73.06152923841961) bank95895 +95896 POINT(41.55946199335318 -73.12067986156156) bank95896 +95897 POINT(40.90669783381196 -73.01596624087014) bank95897 +95898 POINT(40.82422212407633 -73.80988020526493) bank95898 +95899 POINT(41.2706932859928 -74.85410083484349) bank95899 +95900 POINT(41.47853457217416 -73.02942716392853) bank95900 +95901 POINT(41.58656016759332 -73.79571077635234) bank95901 +95902 POINT(40.89642015275023 -73.74422525294678) bank95902 +95903 POINT(40.94058274374842 -73.87197480075716) bank95903 +95904 POINT(40.97842116587764 -73.98245971256236) bank95904 +95905 POINT(41.011146751506374 -74.6988987149171) bank95905 +95906 POINT(40.47288893540814 -73.39264801565334) bank95906 +95907 POINT(41.32567929499104 -74.88082194111107) bank95907 +95908 POINT(41.238420652318496 -74.8569483197117) bank95908 +95909 POINT(39.978931454767235 -74.24832334402527) bank95909 +95910 POINT(41.6847301272296 -73.59038799391816) bank95910 +95911 POINT(41.61719483432748 -73.05332882406087) bank95911 +95912 POINT(39.91913586756405 -74.23058430083465) bank95912 +95913 POINT(41.65862784649239 -74.97983082997763) bank95913 +95914 POINT(39.881271594217814 -73.75865771333112) bank95914 +95915 POINT(41.23242439438466 -74.96477461161528) bank95915 +95916 POINT(41.48816960605386 -73.17531040572423) bank95916 +95917 POINT(40.77385849071703 -74.32253522212122) bank95917 +95918 POINT(40.84041778834153 -74.99850500807625) bank95918 +95919 POINT(39.94921509051925 -73.93306730576492) bank95919 +95920 POINT(40.17120019124822 -74.12549182287422) bank95920 +95921 POINT(40.24523737697275 -73.43553674527834) bank95921 +95922 POINT(40.75723994818954 -74.29212867709724) bank95922 +95923 POINT(40.324830410823 -74.00578194271166) bank95923 +95924 POINT(40.63643555356353 -74.00805532169235) bank95924 +95925 POINT(40.81962590515993 -74.2497658324774) bank95925 +95926 POINT(40.666832539701176 -73.15222707951006) bank95926 +95927 POINT(40.10921500903041 -73.36090879223259) bank95927 +95928 POINT(39.84219843297531 -73.48914103023938) bank95928 +95929 POINT(39.94012947004296 -73.18774716602424) bank95929 +95930 POINT(40.48138208635836 -74.42296879341498) bank95930 +95931 POINT(39.93432860757144 -74.77457116516246) bank95931 +95932 POINT(40.73331023940395 -73.99916688496376) bank95932 +95933 POINT(39.97890033465038 -74.56308128241018) bank95933 +95934 POINT(40.92480497794529 -73.64031152303194) bank95934 +95935 POINT(40.541387641653344 -74.2829706824266) bank95935 +95936 POINT(40.422426578878415 -73.69220047389153) bank95936 +95937 POINT(41.23982448832012 -74.52283340033559) bank95937 +95938 POINT(41.66439964311825 -74.8619564311702) bank95938 +95939 POINT(41.04406268997287 -73.53275094287855) bank95939 +95940 POINT(40.283010282670304 -73.28075661383377) bank95940 +95941 POINT(40.45855080916641 -74.70314500526649) bank95941 +95942 POINT(40.18003111317574 -74.82338502195864) bank95942 +95943 POINT(40.147539924226066 -73.86266461729107) bank95943 +95944 POINT(40.91753795756708 -73.03395246429753) bank95944 +95945 POINT(40.96756398825538 -74.78323922897027) bank95945 +95946 POINT(41.677342128658864 -74.0469761225403) bank95946 +95947 POINT(41.28158831632114 -74.70617865378811) bank95947 +95948 POINT(41.32785823221292 -73.52148333921934) bank95948 +95949 POINT(41.015286383484344 -74.34598470559509) bank95949 +95950 POINT(40.760124356386115 -74.42020559401533) bank95950 +95951 POINT(39.95968942111976 -74.02218309603768) bank95951 +95952 POINT(40.48432761017415 -74.44323286997663) bank95952 +95953 POINT(40.607429788783804 -73.29714155290756) bank95953 +95954 POINT(40.83748013824401 -73.36913215932867) bank95954 +95955 POINT(41.59264200076505 -74.39983887306883) bank95955 +95956 POINT(41.50603660676848 -73.84662436441097) bank95956 +95957 POINT(40.93503622810428 -73.36062326170936) bank95957 +95958 POINT(41.12769372944578 -74.00351129118289) bank95958 +95959 POINT(40.71359836919248 -74.67684968926629) bank95959 +95960 POINT(40.732156752431585 -73.75654332773178) bank95960 +95961 POINT(40.24542498595989 -74.90916953018518) bank95961 +95962 POINT(39.91166075113574 -74.69271158658783) bank95962 +95963 POINT(40.874355021640426 -73.24907438895987) bank95963 +95964 POINT(40.057290165778056 -73.72315927725535) bank95964 +95965 POINT(39.784086251715415 -73.63358148027942) bank95965 +95966 POINT(40.60457798776466 -74.91181835939842) bank95966 +95967 POINT(40.49351512221124 -74.92805843790897) bank95967 +95968 POINT(40.05950862469253 -74.16081918818728) bank95968 +95969 POINT(40.96321570476089 -73.32701525873394) bank95969 +95970 POINT(40.34749638362341 -73.69815451054694) bank95970 +95971 POINT(40.85058335857176 -73.34822745988325) bank95971 +95972 POINT(40.884980984435636 -73.40483287810748) bank95972 +95973 POINT(41.490957729639874 -74.0960077965151) bank95973 +95974 POINT(39.99271456751044 -73.27555513163227) bank95974 +95975 POINT(40.212060600378834 -74.9280857099218) bank95975 +95976 POINT(40.12000417954859 -74.5704714847651) bank95976 +95977 POINT(40.667130703569846 -74.72981713972133) bank95977 +95978 POINT(41.16089551600411 -73.96511278428345) bank95978 +95979 POINT(39.80271396054543 -73.60136898538809) bank95979 +95980 POINT(41.377861421056174 -73.84614501982827) bank95980 +95981 POINT(40.25189860243864 -73.15805868751733) bank95981 +95982 POINT(41.100888532661706 -73.01549768738545) bank95982 +95983 POINT(39.993187842923255 -73.30679845161112) bank95983 +95984 POINT(41.60564022203971 -74.07768997437735) bank95984 +95985 POINT(39.92426541407626 -73.71405927782403) bank95985 +95986 POINT(39.979735021324856 -74.96493717259526) bank95986 +95987 POINT(40.23740860591379 -73.02008063193051) bank95987 +95988 POINT(40.14683008923808 -74.37109785746854) bank95988 +95989 POINT(40.38173341659993 -73.7291941656039) bank95989 +95990 POINT(40.93465928144801 -73.02308041226202) bank95990 +95991 POINT(40.43479725072291 -74.21642903317954) bank95991 +95992 POINT(41.12319735527233 -74.04855257585737) bank95992 +95993 POINT(41.41692579012631 -73.64225736175742) bank95993 +95994 POINT(40.87433007935668 -74.63421182042707) bank95994 +95995 POINT(40.35123657762312 -74.83588014166543) bank95995 +95996 POINT(41.07946859007047 -73.9106749483334) bank95996 +95997 POINT(39.93282225159214 -73.27185573923215) bank95997 +95998 POINT(41.270301022392516 -74.00558066461556) bank95998 +95999 POINT(40.07910847106913 -74.98578429300395) bank95999 +96000 POINT(40.851585768489656 -74.91357529140802) bank96000 +96001 POINT(41.61489694639778 -73.41379386710237) bank96001 +96002 POINT(40.00033769667796 -74.96283515254919) bank96002 +96003 POINT(41.381300733194536 -73.45798689142747) bank96003 +96004 POINT(40.127346958680015 -74.81682179845755) bank96004 +96005 POINT(40.70440316211982 -73.9633040531373) bank96005 +96006 POINT(40.208009128727475 -74.7776194812206) bank96006 +96007 POINT(39.798489120484774 -74.69112036673906) bank96007 +96008 POINT(40.886625696355445 -74.11445308132515) bank96008 +96009 POINT(41.34729765376552 -73.4625571258058) bank96009 +96010 POINT(41.562466517801916 -74.20748248476455) bank96010 +96011 POINT(40.726652257814706 -74.7626973570594) bank96011 +96012 POINT(39.79756397622854 -73.5973695126565) bank96012 +96013 POINT(40.10190993192755 -73.04722443753458) bank96013 +96014 POINT(39.85851955538583 -73.47005649370651) bank96014 +96015 POINT(39.931531351198124 -73.21476582661057) bank96015 +96016 POINT(40.17990312088533 -73.41540429993113) bank96016 +96017 POINT(39.99274053680112 -74.09653510360404) bank96017 +96018 POINT(39.97597659412445 -74.75058000524588) bank96018 +96019 POINT(41.02902552396499 -74.4441536752413) bank96019 +96020 POINT(39.892841319220764 -73.85012332779638) bank96020 +96021 POINT(41.054053838324236 -74.9323933793967) bank96021 +96022 POINT(40.17362048003048 -73.73173330279758) bank96022 +96023 POINT(40.23621017362424 -73.31066466309612) bank96023 +96024 POINT(41.21834970590782 -73.81625379787731) bank96024 +96025 POINT(40.575991925684264 -73.38144191068193) bank96025 +96026 POINT(41.27264904577475 -74.85717527130895) bank96026 +96027 POINT(41.424398135546376 -74.3698502028111) bank96027 +96028 POINT(40.78426260453423 -74.16986646190338) bank96028 +96029 POINT(41.37381298908287 -74.49241862140184) bank96029 +96030 POINT(41.577185290892004 -73.50833775908092) bank96030 +96031 POINT(41.25935318137567 -73.53997491523953) bank96031 +96032 POINT(39.74178395227363 -73.65768993342223) bank96032 +96033 POINT(41.68513030752891 -74.70982092119527) bank96033 +96034 POINT(41.56103027428668 -73.81245590590851) bank96034 +96035 POINT(40.86782252946317 -74.62216640858401) bank96035 +96036 POINT(41.65847397724202 -73.86372787619213) bank96036 +96037 POINT(41.15378030454416 -74.44977131597209) bank96037 +96038 POINT(41.588977365551564 -74.1572222798285) bank96038 +96039 POINT(41.3804954849732 -74.17307301721556) bank96039 +96040 POINT(40.61312300147403 -74.77795167453989) bank96040 +96041 POINT(40.260311683992214 -73.87275748103622) bank96041 +96042 POINT(40.57579750095304 -74.03424298838934) bank96042 +96043 POINT(40.73477642690911 -73.08202249842715) bank96043 +96044 POINT(39.77006902429186 -73.57732924293) bank96044 +96045 POINT(40.41677079297001 -73.4212490923714) bank96045 +96046 POINT(40.25580428831082 -73.45430616871843) bank96046 +96047 POINT(40.296585551332505 -73.28773960913307) bank96047 +96048 POINT(40.70491797113662 -73.82141450141364) bank96048 +96049 POINT(39.95873602409218 -74.05970716173626) bank96049 +96050 POINT(39.981513027907205 -74.8934315568019) bank96050 +96051 POINT(40.78715209743583 -73.85480643629181) bank96051 +96052 POINT(39.71984281003637 -74.19125650996303) bank96052 +96053 POINT(40.56100102475967 -73.83033876220063) bank96053 +96054 POINT(40.85720510868288 -73.10075868547091) bank96054 +96055 POINT(40.330177355122004 -74.6267029579695) bank96055 +96056 POINT(40.90472765813117 -73.35653920593444) bank96056 +96057 POINT(40.04134693164757 -73.98090710344884) bank96057 +96058 POINT(39.980393920074185 -74.20663572497827) bank96058 +96059 POINT(41.57987578798916 -73.23081523418418) bank96059 +96060 POINT(41.33994461217004 -73.65996745979525) bank96060 +96061 POINT(40.91654705758395 -73.07158499388085) bank96061 +96062 POINT(39.844991659420174 -73.502091591597) bank96062 +96063 POINT(41.19514726574077 -73.62320019055105) bank96063 +96064 POINT(41.521465431144605 -74.20302408644811) bank96064 +96065 POINT(40.793177219001755 -74.37362234598255) bank96065 +96066 POINT(40.307175980287504 -74.34436987130648) bank96066 +96067 POINT(40.728158766995186 -73.26352542046294) bank96067 +96068 POINT(40.21287670776363 -74.94346803386775) bank96068 +96069 POINT(41.19074193559734 -73.91804320321896) bank96069 +96070 POINT(41.61666082673938 -74.60214381186474) bank96070 +96071 POINT(39.71795011892512 -73.38027504209666) bank96071 +96072 POINT(41.58771718741209 -74.35059115910965) bank96072 +96073 POINT(40.996374817828546 -73.41149673758626) bank96073 +96074 POINT(41.39131721404114 -73.28893213369383) bank96074 +96075 POINT(39.7217661233963 -73.13638131930348) bank96075 +96076 POINT(40.86523721223808 -74.70936699133955) bank96076 +96077 POINT(40.20099760628324 -74.10940157650462) bank96077 +96078 POINT(41.01549362339729 -73.60821515766709) bank96078 +96079 POINT(40.857081912935605 -73.54226418961464) bank96079 +96080 POINT(40.05667570959515 -74.4409206734756) bank96080 +96081 POINT(40.50324150467125 -73.67556266501342) bank96081 +96082 POINT(40.616256622273745 -74.17114504331437) bank96082 +96083 POINT(41.64381329158138 -73.89638519264443) bank96083 +96084 POINT(40.6825536510541 -73.04985417888894) bank96084 +96085 POINT(41.59477810398783 -74.83352944190953) bank96085 +96086 POINT(40.94042924975258 -73.83964748550689) bank96086 +96087 POINT(40.64781735485988 -73.96760638819501) bank96087 +96088 POINT(40.18003067450344 -73.93833692134187) bank96088 +96089 POINT(40.52827794640527 -73.98797765751615) bank96089 +96090 POINT(40.455062183241445 -73.44036156378453) bank96090 +96091 POINT(40.05726281460762 -73.75903619951087) bank96091 +96092 POINT(41.46505379626718 -74.73629285823071) bank96092 +96093 POINT(40.46533476276862 -74.40849158623261) bank96093 +96094 POINT(41.6248591405645 -73.862827715751) bank96094 +96095 POINT(39.7641277705232 -74.67540983797457) bank96095 +96096 POINT(41.03408428070285 -73.17786158122453) bank96096 +96097 POINT(40.47390333526815 -73.52078406473049) bank96097 +96098 POINT(40.211058452498044 -73.98355602266896) bank96098 +96099 POINT(41.590336280430805 -74.96534487941697) bank96099 +96100 POINT(41.1539346162988 -74.55722836646544) bank96100 +96101 POINT(39.91999362972643 -74.0838497776951) bank96101 +96102 POINT(41.569571287078524 -73.21039036975307) bank96102 +96103 POINT(40.123544328390516 -73.27018695350964) bank96103 +96104 POINT(41.67991108814632 -73.65450601841576) bank96104 +96105 POINT(41.506290068577336 -74.69249203669982) bank96105 +96106 POINT(41.666947920246244 -74.7962071658099) bank96106 +96107 POINT(41.57381423667792 -73.35347428748733) bank96107 +96108 POINT(40.895050655737116 -74.95640924920446) bank96108 +96109 POINT(39.994618203713976 -74.098957524084) bank96109 +96110 POINT(40.62018961396762 -74.3430793372922) bank96110 +96111 POINT(39.7509627002466 -73.19007811110838) bank96111 +96112 POINT(41.27339446791297 -74.63630172076522) bank96112 +96113 POINT(41.16640580674002 -74.23103574968505) bank96113 +96114 POINT(41.35694833095066 -73.46326789939425) bank96114 +96115 POINT(41.479682005637926 -74.69172623231978) bank96115 +96116 POINT(41.05929040922201 -73.15985595212783) bank96116 +96117 POINT(41.61413584487062 -73.24500020871733) bank96117 +96118 POINT(41.43998317833376 -73.36808308229776) bank96118 +96119 POINT(40.287147436925046 -74.33082776086063) bank96119 +96120 POINT(40.17190027675821 -74.41240448447572) bank96120 +96121 POINT(40.65171232823121 -74.43512147879872) bank96121 +96122 POINT(39.86914457966794 -74.4683745709389) bank96122 +96123 POINT(41.535675331475844 -74.55914483730065) bank96123 +96124 POINT(40.68731942068354 -73.97655832685668) bank96124 +96125 POINT(39.86697110250612 -73.29769701323225) bank96125 +96126 POINT(40.84232578075615 -74.65068702006026) bank96126 +96127 POINT(40.61276378995269 -73.15409265107724) bank96127 +96128 POINT(41.202695252854994 -74.8588464772253) bank96128 +96129 POINT(40.3641897457516 -73.40186409219429) bank96129 +96130 POINT(41.59925329086961 -73.5475941269467) bank96130 +96131 POINT(40.71323467119378 -73.29760233704144) bank96131 +96132 POINT(40.77844378840948 -74.92365043128996) bank96132 +96133 POINT(40.697853354986464 -74.73984115778032) bank96133 +96134 POINT(41.23484766328917 -73.6376437565712) bank96134 +96135 POINT(40.07378715799725 -74.69196379311943) bank96135 +96136 POINT(40.60289139847367 -74.1750962421569) bank96136 +96137 POINT(39.96333892519605 -74.33078993680103) bank96137 +96138 POINT(41.18747368658416 -74.55932904407028) bank96138 +96139 POINT(41.2083982714085 -73.39418919908042) bank96139 +96140 POINT(39.95184384850446 -73.48055865818378) bank96140 +96141 POINT(41.362260364445916 -74.24773245237706) bank96141 +96142 POINT(40.139204150928585 -74.99139155295485) bank96142 +96143 POINT(40.62022485509656 -74.95569729988456) bank96143 +96144 POINT(39.91236695951204 -73.06356327112876) bank96144 +96145 POINT(40.41055476331997 -74.62727656614373) bank96145 +96146 POINT(41.5594634969772 -73.69203237492077) bank96146 +96147 POINT(40.76916898717934 -73.94689714192705) bank96147 +96148 POINT(41.33680084517415 -74.93615016838918) bank96148 +96149 POINT(39.968932688077935 -73.13295212531666) bank96149 +96150 POINT(40.64676349422676 -74.79180178319487) bank96150 +96151 POINT(41.18035292445732 -73.4017506377838) bank96151 +96152 POINT(40.287308952994074 -73.16233960805226) bank96152 +96153 POINT(40.049627119781206 -73.92659684029168) bank96153 +96154 POINT(39.937609884196654 -73.0084549139716) bank96154 +96155 POINT(41.17425437239356 -73.62700486138222) bank96155 +96156 POINT(39.92897315148761 -73.1202451418022) bank96156 +96157 POINT(41.10499334540431 -73.36071907789346) bank96157 +96158 POINT(41.069830482578986 -73.89424172044194) bank96158 +96159 POINT(40.90295451841869 -74.67517707882801) bank96159 +96160 POINT(40.440747301347336 -73.95070533060907) bank96160 +96161 POINT(41.25584134360615 -73.48917819821924) bank96161 +96162 POINT(41.25591387822655 -74.08197109879055) bank96162 +96163 POINT(39.95330625395851 -73.19388414000039) bank96163 +96164 POINT(41.205353379201675 -73.17537209716109) bank96164 +96165 POINT(41.39218450189197 -74.006499752649) bank96165 +96166 POINT(40.425734577169024 -74.94852675549183) bank96166 +96167 POINT(41.41282821597616 -74.19449429152645) bank96167 +96168 POINT(39.76718367782276 -73.50076056886404) bank96168 +96169 POINT(41.42022262617112 -74.8714616514828) bank96169 +96170 POINT(40.541844144539475 -73.75438822001729) bank96170 +96171 POINT(40.90973406136041 -74.66504241898691) bank96171 +96172 POINT(40.367693265111676 -74.02177475110747) bank96172 +96173 POINT(40.23779585037469 -73.98264408143176) bank96173 +96174 POINT(40.18441212513146 -73.8247835318227) bank96174 +96175 POINT(40.50119592697712 -73.63100463756047) bank96175 +96176 POINT(41.284036561740564 -74.51288089633871) bank96176 +96177 POINT(41.00313615534029 -74.59922532451723) bank96177 +96178 POINT(41.01012055972674 -74.16164412884478) bank96178 +96179 POINT(40.48331285914811 -74.41975967662931) bank96179 +96180 POINT(41.371787317181834 -74.86464749543403) bank96180 +96181 POINT(40.243946688606655 -73.24762568813992) bank96181 +96182 POINT(39.89581588599121 -73.275568187653) bank96182 +96183 POINT(41.348504426245455 -73.70394417963102) bank96183 +96184 POINT(40.512230369047415 -73.62377649289516) bank96184 +96185 POINT(39.88990865828543 -73.53242948942332) bank96185 +96186 POINT(40.1506928454753 -73.80342861961) bank96186 +96187 POINT(41.1744573746468 -74.2300157156483) bank96187 +96188 POINT(39.99190457656911 -73.1868748528198) bank96188 +96189 POINT(41.628189151716754 -74.74821019608818) bank96189 +96190 POINT(39.98389957098488 -74.98899690695555) bank96190 +96191 POINT(40.04688393602241 -73.77899514856556) bank96191 +96192 POINT(40.51492438443508 -73.62941457335248) bank96192 +96193 POINT(41.145015891677986 -74.36093739487855) bank96193 +96194 POINT(39.72014770654209 -73.65953983768148) bank96194 +96195 POINT(39.78069405594151 -74.8210116637407) bank96195 +96196 POINT(40.4430104777188 -73.99396089782307) bank96196 +96197 POINT(40.26677148651691 -73.1147534684077) bank96197 +96198 POINT(41.30249712967059 -74.42642164589158) bank96198 +96199 POINT(41.585974625095965 -74.86231203338197) bank96199 +96200 POINT(41.27584020665591 -74.49247028267415) bank96200 +96201 POINT(41.248826193668855 -74.58853984694393) bank96201 +96202 POINT(41.537628972657295 -73.55522487940877) bank96202 +96203 POINT(39.81947542765072 -73.68582180409568) bank96203 +96204 POINT(41.62999804753268 -73.92592631763863) bank96204 +96205 POINT(41.39038661946587 -74.46096683497399) bank96205 +96206 POINT(39.78545462227405 -74.08996888449106) bank96206 +96207 POINT(39.94146641065759 -74.61406447007052) bank96207 +96208 POINT(41.655068331954375 -73.05736951343695) bank96208 +96209 POINT(40.29680844978932 -74.4338511363767) bank96209 +96210 POINT(40.57807302505954 -74.04141432489413) bank96210 +96211 POINT(41.05334161892365 -74.12572517182885) bank96211 +96212 POINT(40.706803531349365 -73.42429645787058) bank96212 +96213 POINT(41.67378154181042 -74.17417123717489) bank96213 +96214 POINT(40.320343241718675 -73.47869843476157) bank96214 +96215 POINT(41.12505035644243 -74.82299491724751) bank96215 +96216 POINT(41.63722673791723 -73.04434691515127) bank96216 +96217 POINT(40.99216911502992 -73.21464188197524) bank96217 +96218 POINT(41.443063753644786 -73.08932214507404) bank96218 +96219 POINT(40.475471891436385 -73.48453416051768) bank96219 +96220 POINT(40.32695726451814 -74.20045059085659) bank96220 +96221 POINT(40.00237937688459 -73.98048097452464) bank96221 +96222 POINT(39.80708775110374 -73.66718794970048) bank96222 +96223 POINT(40.49527690218032 -73.28865547254803) bank96223 +96224 POINT(40.9136226531912 -73.48256911570091) bank96224 +96225 POINT(40.81975156205522 -74.21591101906039) bank96225 +96226 POINT(39.99996057190804 -74.94954600378081) bank96226 +96227 POINT(40.3106841410837 -74.73891691322078) bank96227 +96228 POINT(41.385902432137705 -73.68226927139501) bank96228 +96229 POINT(40.966890087773926 -73.13395553226087) bank96229 +96230 POINT(40.57431745263971 -74.35667918203215) bank96230 +96231 POINT(39.72571556365453 -74.31066941775951) bank96231 +96232 POINT(40.932031527270375 -73.45851768723723) bank96232 +96233 POINT(39.98266433004058 -74.90919730002706) bank96233 +96234 POINT(40.346802153220565 -73.19227196134217) bank96234 +96235 POINT(40.01758700971401 -73.41905196241512) bank96235 +96236 POINT(41.03853378643074 -74.7746073778655) bank96236 +96237 POINT(39.92030015151258 -73.16687918255631) bank96237 +96238 POINT(40.6730815947066 -73.87685826236913) bank96238 +96239 POINT(41.202339522333155 -73.14247434063516) bank96239 +96240 POINT(40.46066037971094 -74.26348396120663) bank96240 +96241 POINT(41.33264729710105 -74.84591296901502) bank96241 +96242 POINT(41.035368603265205 -74.78142193315166) bank96242 +96243 POINT(40.37788318071391 -73.59993727391006) bank96243 +96244 POINT(40.092599172637954 -74.51060124605085) bank96244 +96245 POINT(41.686386543184256 -73.50867066755215) bank96245 +96246 POINT(41.45302657899505 -73.5506153327937) bank96246 +96247 POINT(41.31186163675125 -73.91083980002765) bank96247 +96248 POINT(39.96423763944201 -73.84092085187139) bank96248 +96249 POINT(40.010326531909705 -74.82473176105742) bank96249 +96250 POINT(39.73433327639099 -74.11025243539356) bank96250 +96251 POINT(40.891149908683886 -73.6000585368259) bank96251 +96252 POINT(39.85040174143895 -73.9672755654168) bank96252 +96253 POINT(41.58167346854445 -74.10096619616957) bank96253 +96254 POINT(41.3831811909675 -73.87827235264801) bank96254 +96255 POINT(41.386787728706274 -73.12132877722225) bank96255 +96256 POINT(41.18129172060811 -73.0115681879965) bank96256 +96257 POINT(41.03735082832219 -73.19836612794808) bank96257 +96258 POINT(40.52159854215647 -74.64872304053304) bank96258 +96259 POINT(40.251206125537344 -74.08896909744698) bank96259 +96260 POINT(40.83390857382902 -74.3593291597153) bank96260 +96261 POINT(41.291388060463774 -74.43492221634449) bank96261 +96262 POINT(41.04844443033193 -74.46454466905189) bank96262 +96263 POINT(41.64751365937746 -74.10971844830247) bank96263 +96264 POINT(39.82347955368308 -73.90023854275265) bank96264 +96265 POINT(40.31293708845766 -73.14442583081046) bank96265 +96266 POINT(41.03014124027034 -73.86255261019205) bank96266 +96267 POINT(41.57733473909376 -74.53443795657162) bank96267 +96268 POINT(40.99558715766552 -73.16940174156704) bank96268 +96269 POINT(41.12341758941802 -73.57023389526333) bank96269 +96270 POINT(40.23363485106005 -74.76462860717247) bank96270 +96271 POINT(40.04053708943142 -73.12885826814905) bank96271 +96272 POINT(40.307777295519635 -74.70086143159392) bank96272 +96273 POINT(39.820780022446954 -74.45856120944669) bank96273 +96274 POINT(41.19421591226942 -73.178193977191) bank96274 +96275 POINT(40.313787749075225 -73.1385655415012) bank96275 +96276 POINT(40.87639660093484 -74.66873159965884) bank96276 +96277 POINT(40.77681007298957 -73.98247787851552) bank96277 +96278 POINT(40.29254866282842 -74.3798479816565) bank96278 +96279 POINT(40.93952438030817 -73.84947856305442) bank96279 +96280 POINT(41.34276697774324 -74.89453345353861) bank96280 +96281 POINT(39.80268953719204 -74.620693693687) bank96281 +96282 POINT(40.816648494643246 -74.67878956522203) bank96282 +96283 POINT(41.66618465485995 -74.54522263658983) bank96283 +96284 POINT(41.60904374866783 -74.61943751286928) bank96284 +96285 POINT(39.72280400603922 -73.36994513417555) bank96285 +96286 POINT(40.85664085989947 -74.93705580066408) bank96286 +96287 POINT(41.422898642643865 -73.80464031614594) bank96287 +96288 POINT(39.82103042863343 -73.19958201215057) bank96288 +96289 POINT(41.14520141271401 -73.14061697165266) bank96289 +96290 POINT(40.292868219047094 -74.68386406506902) bank96290 +96291 POINT(40.34172224302525 -73.91265702152313) bank96291 +96292 POINT(41.70466666349507 -73.12317822271939) bank96292 +96293 POINT(41.45560597403291 -74.14407261461677) bank96293 +96294 POINT(41.21485308639132 -74.15640708480811) bank96294 +96295 POINT(40.994986246056875 -73.16111089159243) bank96295 +96296 POINT(40.22812878589538 -74.20456406259834) bank96296 +96297 POINT(40.127487207011896 -74.78437172423436) bank96297 +96298 POINT(40.699072404758894 -73.85321199779256) bank96298 +96299 POINT(40.15272608267676 -74.282387682824) bank96299 +96300 POINT(40.544059114801016 -73.47657671273771) bank96300 +96301 POINT(40.31097247477549 -73.55068805973295) bank96301 +96302 POINT(40.571379001704315 -73.06486221648824) bank96302 +96303 POINT(40.60085168981823 -73.15451629979107) bank96303 +96304 POINT(40.50629861561005 -73.92195926369551) bank96304 +96305 POINT(39.95745295586259 -73.38819068296333) bank96305 +96306 POINT(40.8306850769203 -73.38743106467308) bank96306 +96307 POINT(41.13853034053315 -73.62894647023573) bank96307 +96308 POINT(40.81998359864414 -74.88142193013005) bank96308 +96309 POINT(40.84000657904504 -73.03143765443255) bank96309 +96310 POINT(40.14588937645026 -73.35095639811345) bank96310 +96311 POINT(41.377565888129 -73.9298345736431) bank96311 +96312 POINT(41.18384876053576 -73.27197421845877) bank96312 +96313 POINT(41.70570904946541 -73.27722698426024) bank96313 +96314 POINT(41.52636519455749 -74.46108916449275) bank96314 +96315 POINT(40.28090922696789 -74.73285469362307) bank96315 +96316 POINT(41.08678731616588 -74.23325273009749) bank96316 +96317 POINT(40.62145928584165 -73.22093659616944) bank96317 +96318 POINT(40.436326577316954 -74.86072979699956) bank96318 +96319 POINT(40.29304901547402 -73.30050601119869) bank96319 +96320 POINT(40.006640665943415 -73.63587908302586) bank96320 +96321 POINT(40.239373652841415 -73.59236163696224) bank96321 +96322 POINT(41.340029384886996 -74.57533616902252) bank96322 +96323 POINT(39.73957245795321 -74.19245626132098) bank96323 +96324 POINT(41.43548640001799 -73.58056017873943) bank96324 +96325 POINT(40.353336151562715 -73.33388590567994) bank96325 +96326 POINT(40.29141564781833 -73.1114867157297) bank96326 +96327 POINT(41.45702312338945 -73.7697999852772) bank96327 +96328 POINT(39.985080912299324 -74.21700709192378) bank96328 +96329 POINT(40.72742741865452 -73.13402287289573) bank96329 +96330 POINT(41.08421682939971 -73.47425352264733) bank96330 +96331 POINT(39.81300853099067 -73.3477370760075) bank96331 +96332 POINT(41.63653097044402 -73.47452555626538) bank96332 +96333 POINT(41.28730884517789 -74.15692844265182) bank96333 +96334 POINT(39.76074790131329 -74.1432330908625) bank96334 +96335 POINT(40.91096686250249 -73.50636401362304) bank96335 +96336 POINT(41.4796259660979 -73.7221907431394) bank96336 +96337 POINT(40.56108170727668 -73.78855109846015) bank96337 +96338 POINT(41.50069145459189 -73.55812705942975) bank96338 +96339 POINT(41.0718853613334 -73.63308389553275) bank96339 +96340 POINT(40.37886756568492 -74.68853601422141) bank96340 +96341 POINT(41.529699658170045 -74.63642176296658) bank96341 +96342 POINT(39.772063782360235 -74.79120290103423) bank96342 +96343 POINT(41.32712731645727 -73.11841436380043) bank96343 +96344 POINT(40.06132449687896 -73.44952494065524) bank96344 +96345 POINT(41.160797417531114 -74.79454106473537) bank96345 +96346 POINT(40.714559059590854 -73.39390460804734) bank96346 +96347 POINT(41.434478295075145 -73.76824016333381) bank96347 +96348 POINT(39.9870046048888 -73.08927730969202) bank96348 +96349 POINT(40.584298845978836 -74.76147241809444) bank96349 +96350 POINT(40.83729759392592 -73.90398929556561) bank96350 +96351 POINT(41.14856912977868 -73.17132069280169) bank96351 +96352 POINT(40.596959537432845 -74.91086154250475) bank96352 +96353 POINT(41.21469404425556 -74.97035310142603) bank96353 +96354 POINT(40.9736675383508 -74.74315850572533) bank96354 +96355 POINT(41.241171448994535 -73.46229057149722) bank96355 +96356 POINT(39.80389085512036 -74.08379359349367) bank96356 +96357 POINT(41.14931768959666 -74.54472646011496) bank96357 +96358 POINT(40.46643476909124 -74.925089134212) bank96358 +96359 POINT(40.96005546543693 -73.71810876213249) bank96359 +96360 POINT(39.76133651231987 -73.80152031424288) bank96360 +96361 POINT(41.578802618099665 -73.82445205926965) bank96361 +96362 POINT(40.20067065654909 -73.27714447536904) bank96362 +96363 POINT(40.03437154991693 -74.74903641155008) bank96363 +96364 POINT(40.119011309325636 -74.77178657002283) bank96364 +96365 POINT(41.37261949164889 -74.49526339565706) bank96365 +96366 POINT(40.7810163325296 -73.10824164837656) bank96366 +96367 POINT(41.47184520299737 -73.55811613795909) bank96367 +96368 POINT(39.74206596731474 -73.92196795760198) bank96368 +96369 POINT(41.42586585287253 -74.77922979661196) bank96369 +96370 POINT(40.93232541524413 -73.72555217190427) bank96370 +96371 POINT(39.99960126847503 -74.76504976708212) bank96371 +96372 POINT(40.499909182025895 -74.83071703988456) bank96372 +96373 POINT(41.49072626538349 -74.23206117716961) bank96373 +96374 POINT(40.1067606695321 -73.07889027204536) bank96374 +96375 POINT(41.53728116546279 -74.3341461149517) bank96375 +96376 POINT(40.60594522037788 -74.43115094177747) bank96376 +96377 POINT(41.640468917879986 -74.4160090418805) bank96377 +96378 POINT(40.43649776309444 -74.72438806860376) bank96378 +96379 POINT(41.677820328379276 -73.77985585158962) bank96379 +96380 POINT(41.302708674471155 -74.41131163542913) bank96380 +96381 POINT(40.047333109719474 -74.902659714263) bank96381 +96382 POINT(39.735236615139804 -74.66755957950474) bank96382 +96383 POINT(41.49507573341518 -74.58720792292883) bank96383 +96384 POINT(41.47908039554676 -73.50527015245721) bank96384 +96385 POINT(40.02694762419079 -74.53682740860602) bank96385 +96386 POINT(40.17246756733404 -74.61209530074187) bank96386 +96387 POINT(41.240247724108315 -73.05007453784997) bank96387 +96388 POINT(40.58833025041081 -74.13428854572386) bank96388 +96389 POINT(40.455874715721684 -74.48824003273224) bank96389 +96390 POINT(41.540797801263565 -74.79713076021055) bank96390 +96391 POINT(40.90929983606498 -73.27703176136937) bank96391 +96392 POINT(40.04762459475129 -73.06997005521019) bank96392 +96393 POINT(40.72129309514968 -74.76056270081928) bank96393 +96394 POINT(41.468845341722734 -74.83172012264069) bank96394 +96395 POINT(41.01214557182044 -74.7756689392136) bank96395 +96396 POINT(40.729487427749945 -74.73123268460756) bank96396 +96397 POINT(40.605756408830146 -74.52823174610351) bank96397 +96398 POINT(40.82138998738472 -73.79100155947039) bank96398 +96399 POINT(41.085708995312416 -73.47315118793507) bank96399 +96400 POINT(41.58893699894991 -74.8937486267807) bank96400 +96401 POINT(40.61782585578275 -73.48642906363989) bank96401 +96402 POINT(39.76905826656151 -73.93773559883289) bank96402 +96403 POINT(41.67597133892034 -73.37340130989892) bank96403 +96404 POINT(41.69435570910191 -73.6824617974735) bank96404 +96405 POINT(40.45975819882969 -74.90401772063136) bank96405 +96406 POINT(40.06711306052619 -74.40850068835059) bank96406 +96407 POINT(40.267078798986375 -73.29306629135502) bank96407 +96408 POINT(41.397288695907 -73.69507819542892) bank96408 +96409 POINT(40.85842060768436 -73.8543955860523) bank96409 +96410 POINT(40.86187329808579 -73.51701489050463) bank96410 +96411 POINT(41.3110657071737 -73.11573098579913) bank96411 +96412 POINT(41.26813784190221 -73.20724755510761) bank96412 +96413 POINT(40.09166268860903 -74.99186013988066) bank96413 +96414 POINT(40.84357885482179 -73.10298969015241) bank96414 +96415 POINT(39.9656342081567 -73.7471708051792) bank96415 +96416 POINT(40.73515714286948 -73.53918597005118) bank96416 +96417 POINT(40.86505856424003 -73.62667298408044) bank96417 +96418 POINT(41.41237799817584 -74.47896439537796) bank96418 +96419 POINT(39.8209452734658 -74.27556627518412) bank96419 +96420 POINT(40.40457335361127 -73.48403553092318) bank96420 +96421 POINT(39.81946785370417 -73.05461491080142) bank96421 +96422 POINT(40.984549933753506 -74.34736911300672) bank96422 +96423 POINT(39.963604432452996 -74.3109820494625) bank96423 +96424 POINT(40.78565744617899 -74.65846494184981) bank96424 +96425 POINT(40.09159752534132 -74.83233389623805) bank96425 +96426 POINT(41.67248681739586 -73.81712188818148) bank96426 +96427 POINT(40.197979561457764 -74.41161910311911) bank96427 +96428 POINT(41.08915050192897 -74.86254707327141) bank96428 +96429 POINT(40.924593218640716 -73.28460726843377) bank96429 +96430 POINT(40.32897072638671 -74.08968137116723) bank96430 +96431 POINT(40.37479833492064 -74.32657401661682) bank96431 +96432 POINT(41.485060037418776 -73.01526419424589) bank96432 +96433 POINT(39.727132600777225 -73.26737849362553) bank96433 +96434 POINT(41.60420270375568 -73.08553510647195) bank96434 +96435 POINT(41.4981535827214 -74.2280706575364) bank96435 +96436 POINT(40.4514198209056 -73.70974786735674) bank96436 +96437 POINT(40.3569911020208 -74.75437241621486) bank96437 +96438 POINT(40.27147205534599 -73.89664278985977) bank96438 +96439 POINT(40.804011748197496 -74.91614370046698) bank96439 +96440 POINT(41.1995197655742 -73.3884230936514) bank96440 +96441 POINT(40.76755747103674 -73.0462479194566) bank96441 +96442 POINT(41.68928190539938 -74.00318338593276) bank96442 +96443 POINT(41.30233322766227 -73.27155473279612) bank96443 +96444 POINT(39.99560480608332 -74.60982228096816) bank96444 +96445 POINT(40.067292332602385 -74.041836644764) bank96445 +96446 POINT(40.54015397579242 -74.12990888081619) bank96446 +96447 POINT(41.111315487201175 -74.63772058388224) bank96447 +96448 POINT(41.554215877699605 -73.42568737374326) bank96448 +96449 POINT(40.31423049874994 -73.63772798205933) bank96449 +96450 POINT(40.98770189552891 -73.2133760565835) bank96450 +96451 POINT(40.47779190818854 -73.85074760764066) bank96451 +96452 POINT(41.02983266492599 -73.46368955946839) bank96452 +96453 POINT(39.865074692499086 -73.97485334164452) bank96453 +96454 POINT(41.05015644500784 -73.92585363067798) bank96454 +96455 POINT(41.4227898051435 -74.53512486427931) bank96455 +96456 POINT(40.176110672415504 -74.77725158689012) bank96456 +96457 POINT(40.82215102404211 -74.31888328450012) bank96457 +96458 POINT(40.675260818620316 -73.32295030707483) bank96458 +96459 POINT(40.283833772540014 -73.14269710859563) bank96459 +96460 POINT(41.07283312337059 -73.13836903358687) bank96460 +96461 POINT(40.42629659019751 -73.67327908762006) bank96461 +96462 POINT(41.33211840347145 -74.81081830335509) bank96462 +96463 POINT(41.66011328734507 -73.45266476861048) bank96463 +96464 POINT(40.20952290448185 -74.30087180979608) bank96464 +96465 POINT(41.145857513442756 -74.64802808393935) bank96465 +96466 POINT(41.441290316923514 -73.18114321095746) bank96466 +96467 POINT(40.54868272957702 -73.90767560527982) bank96467 +96468 POINT(40.82810251844843 -73.53977768255227) bank96468 +96469 POINT(41.61867244539468 -74.61604285124285) bank96469 +96470 POINT(40.17628727222163 -73.654042359526) bank96470 +96471 POINT(39.84112690953749 -74.62061273721875) bank96471 +96472 POINT(40.07603247414992 -73.80677988256068) bank96472 +96473 POINT(41.422983665724985 -74.35314964384621) bank96473 +96474 POINT(40.241316534834816 -73.63819218249782) bank96474 +96475 POINT(41.04164614899536 -73.41176926195801) bank96475 +96476 POINT(40.44378241127355 -73.21923591986929) bank96476 +96477 POINT(41.321385471704566 -73.10504400871858) bank96477 +96478 POINT(39.9400082938622 -73.71733042198251) bank96478 +96479 POINT(40.33082829634227 -74.47359245729018) bank96479 +96480 POINT(41.17536466248474 -73.92510195840367) bank96480 +96481 POINT(41.457852350380044 -73.57721801713971) bank96481 +96482 POINT(40.18529657906098 -74.90517393803374) bank96482 +96483 POINT(40.173600311379246 -73.78538035382657) bank96483 +96484 POINT(40.86963621564361 -73.25217435781745) bank96484 +96485 POINT(41.588251516145974 -73.1349785244139) bank96485 +96486 POINT(41.13084719870104 -73.58086442023907) bank96486 +96487 POINT(39.86917508702089 -74.25698315821947) bank96487 +96488 POINT(41.67223956946544 -73.24217601266727) bank96488 +96489 POINT(40.257869180767045 -74.37987856874844) bank96489 +96490 POINT(41.05215178696258 -74.686703553015) bank96490 +96491 POINT(40.69941608585359 -73.48202867601815) bank96491 +96492 POINT(40.56900754537773 -73.0623075910492) bank96492 +96493 POINT(40.336912167300724 -74.6494914169153) bank96493 +96494 POINT(41.59683887901463 -74.34441401822617) bank96494 +96495 POINT(41.50751631698704 -74.76113247586241) bank96495 +96496 POINT(40.43066965488358 -74.98130575963592) bank96496 +96497 POINT(41.522750731591366 -73.10456038469341) bank96497 +96498 POINT(40.812654775266054 -73.61659676376377) bank96498 +96499 POINT(41.678389741448406 -73.67079048434442) bank96499 +96500 POINT(41.5546281274273 -73.94643988522935) bank96500 +96501 POINT(41.07913513722912 -73.7363982314575) bank96501 +96502 POINT(40.94514919774051 -73.37746377076378) bank96502 +96503 POINT(39.939010553697436 -74.35386374250871) bank96503 +96504 POINT(41.156647250009584 -74.1268225757522) bank96504 +96505 POINT(41.437501440072644 -73.87496995653606) bank96505 +96506 POINT(41.1053243990225 -73.10898349337884) bank96506 +96507 POINT(40.08405762576362 -74.33304098323252) bank96507 +96508 POINT(39.81211377040901 -73.96430261992374) bank96508 +96509 POINT(40.556654872395804 -73.15934716586337) bank96509 +96510 POINT(40.05359737564572 -74.37140487839139) bank96510 +96511 POINT(39.76631512951028 -73.37427349300626) bank96511 +96512 POINT(40.32130404800034 -73.09183797790702) bank96512 +96513 POINT(40.107373111513894 -73.22383684923152) bank96513 +96514 POINT(41.47840234966785 -73.50020385622939) bank96514 +96515 POINT(41.14931277841845 -73.5059239570856) bank96515 +96516 POINT(39.943817487534034 -73.07652962941602) bank96516 +96517 POINT(41.46643498020542 -73.85215479656564) bank96517 +96518 POINT(40.457578416071534 -74.80423634081117) bank96518 +96519 POINT(40.17326895993315 -73.9829129300606) bank96519 +96520 POINT(41.4542557946723 -74.8512007050619) bank96520 +96521 POINT(39.90956540668591 -74.35025511825862) bank96521 +96522 POINT(40.24963019338157 -73.90113310209234) bank96522 +96523 POINT(39.83723674056751 -74.746889422536) bank96523 +96524 POINT(40.946047053015825 -74.1282514000973) bank96524 +96525 POINT(40.68715431221857 -74.42868877321648) bank96525 +96526 POINT(41.44823429064401 -73.5837110602565) bank96526 +96527 POINT(40.19618956367168 -73.18713733724508) bank96527 +96528 POINT(40.62238250578345 -73.46857074954015) bank96528 +96529 POINT(40.5933842585992 -74.37423602245693) bank96529 +96530 POINT(40.27380045995964 -73.45979819195402) bank96530 +96531 POINT(41.31491438777931 -74.69387034304462) bank96531 +96532 POINT(40.423459649437014 -73.97112947713276) bank96532 +96533 POINT(41.060287660736996 -73.36838833404254) bank96533 +96534 POINT(40.45402185758805 -73.71475065406736) bank96534 +96535 POINT(41.368677548197105 -73.63209850799467) bank96535 +96536 POINT(40.84589054039967 -74.44941599067484) bank96536 +96537 POINT(39.77628812106082 -73.60999438701802) bank96537 +96538 POINT(40.9457824479342 -73.79791215570506) bank96538 +96539 POINT(41.058357320324355 -74.862437702391) bank96539 +96540 POINT(40.65414836466871 -74.94393917354505) bank96540 +96541 POINT(41.638368991757005 -74.49242028358344) bank96541 +96542 POINT(40.95727617211991 -73.3300110706756) bank96542 +96543 POINT(41.49123338140438 -73.52365699243575) bank96543 +96544 POINT(40.68805955994703 -74.17718073605437) bank96544 +96545 POINT(40.798022040150634 -73.71630256778502) bank96545 +96546 POINT(40.51616046993073 -74.7767460746534) bank96546 +96547 POINT(39.843400795585985 -74.17438095568836) bank96547 +96548 POINT(40.325348008438866 -73.84468683729875) bank96548 +96549 POINT(41.17370081084075 -73.23120076267712) bank96549 +96550 POINT(40.542437376416196 -73.7421761926879) bank96550 +96551 POINT(41.53365865901236 -73.60933805729171) bank96551 +96552 POINT(40.880068719411675 -74.07762750386044) bank96552 +96553 POINT(41.07557710129357 -73.92122254271078) bank96553 +96554 POINT(39.800104858921564 -73.02481852173064) bank96554 +96555 POINT(41.70315907392546 -73.52321673996404) bank96555 +96556 POINT(40.26881382958428 -74.89158440501237) bank96556 +96557 POINT(41.556141034449574 -74.25855541280875) bank96557 +96558 POINT(39.86249600527102 -73.38285523618207) bank96558 +96559 POINT(40.59026940839694 -73.27385017204884) bank96559 +96560 POINT(39.72089004506541 -74.63579382584629) bank96560 +96561 POINT(39.92198144491999 -74.1297690660098) bank96561 +96562 POINT(41.58821543146082 -74.22426322781828) bank96562 +96563 POINT(41.5246636469472 -74.37186066652944) bank96563 +96564 POINT(40.47240440084747 -73.66187869032507) bank96564 +96565 POINT(41.70973482158319 -74.54620756533474) bank96565 +96566 POINT(41.465817883846036 -74.26645464637117) bank96566 +96567 POINT(41.0572510873062 -74.43088909378908) bank96567 +96568 POINT(40.741452071578244 -74.11021007262704) bank96568 +96569 POINT(41.27272092991358 -74.69704143172761) bank96569 +96570 POINT(41.11937062953589 -74.65359745310668) bank96570 +96571 POINT(40.22466139631487 -73.92218093595265) bank96571 +96572 POINT(40.07207929186947 -73.66066167868863) bank96572 +96573 POINT(41.404295429502014 -73.66404290667337) bank96573 +96574 POINT(40.59419309753199 -73.29309843599025) bank96574 +96575 POINT(40.508621203289415 -74.16788785452964) bank96575 +96576 POINT(41.202849377786876 -74.94336314382952) bank96576 +96577 POINT(41.59761718756994 -74.23067992794317) bank96577 +96578 POINT(39.929823357492246 -74.37637566640747) bank96578 +96579 POINT(41.528691884209785 -73.84126776232428) bank96579 +96580 POINT(39.975103081724974 -73.75544249233401) bank96580 +96581 POINT(40.588298929565575 -73.98276733311758) bank96581 +96582 POINT(41.67486612047765 -73.49326067681412) bank96582 +96583 POINT(39.71330540939209 -74.58194834811344) bank96583 +96584 POINT(41.57381118266831 -73.31172526720907) bank96584 +96585 POINT(41.32323210239712 -73.40535294039668) bank96585 +96586 POINT(41.63978508575077 -74.08147017243896) bank96586 +96587 POINT(40.14000881738293 -73.50002125574136) bank96587 +96588 POINT(40.44469672603468 -73.55108462379337) bank96588 +96589 POINT(41.397368473521595 -73.26888771376501) bank96589 +96590 POINT(41.19263752441944 -73.7317343611315) bank96590 +96591 POINT(41.218344584970275 -73.4389076440742) bank96591 +96592 POINT(41.063772204341774 -73.9922787537858) bank96592 +96593 POINT(40.0943099316937 -74.86716661937297) bank96593 +96594 POINT(41.40632164488972 -73.72972371467976) bank96594 +96595 POINT(40.61306221738795 -73.88976231313906) bank96595 +96596 POINT(39.87297047655459 -74.03727662421655) bank96596 +96597 POINT(41.02872704754856 -74.77333814729222) bank96597 +96598 POINT(41.25171765893826 -73.07687725107006) bank96598 +96599 POINT(40.2209962981308 -74.37589047897842) bank96599 +96600 POINT(39.7284022254091 -73.80437438183694) bank96600 +96601 POINT(41.435671786007 -74.45932368705381) bank96601 +96602 POINT(41.236849115571644 -73.42011558289269) bank96602 +96603 POINT(41.28697973000884 -73.4922598175186) bank96603 +96604 POINT(41.31847598889547 -73.57345602517248) bank96604 +96605 POINT(41.145656503908874 -74.98832296291458) bank96605 +96606 POINT(40.988417806043735 -73.98331228602828) bank96606 +96607 POINT(41.19972859060726 -73.84672992061479) bank96607 +96608 POINT(39.95617210454022 -73.25080984427032) bank96608 +96609 POINT(40.80126217648126 -73.35807159175924) bank96609 +96610 POINT(41.43529574990356 -73.39247669260797) bank96610 +96611 POINT(39.94406646470159 -73.08492033161639) bank96611 +96612 POINT(41.33103199466482 -74.50366683056477) bank96612 +96613 POINT(40.46884803178176 -74.46943021865559) bank96613 +96614 POINT(41.0929626511392 -74.912721227203) bank96614 +96615 POINT(39.97210575624654 -74.58832358558266) bank96615 +96616 POINT(39.99483380666338 -73.54803376302112) bank96616 +96617 POINT(40.500839923393144 -73.15104836677847) bank96617 +96618 POINT(39.7522275346194 -74.92521371794983) bank96618 +96619 POINT(41.09530041102507 -74.7083736490656) bank96619 +96620 POINT(40.7103318433278 -74.06456993846543) bank96620 +96621 POINT(41.22132660065737 -73.28795331673533) bank96621 +96622 POINT(39.95336660964104 -74.64657310000679) bank96622 +96623 POINT(39.80545365169088 -74.40045843884045) bank96623 +96624 POINT(40.57217567080751 -75.00202826778184) bank96624 +96625 POINT(40.04202654554652 -74.9066846800282) bank96625 +96626 POINT(40.19703992262085 -73.79819495271259) bank96626 +96627 POINT(41.55911295745603 -73.89402608424834) bank96627 +96628 POINT(40.2973799533952 -73.34094344608509) bank96628 +96629 POINT(40.19356306017375 -73.14209416007601) bank96629 +96630 POINT(40.92304245895631 -75.0003520072397) bank96630 +96631 POINT(41.23299758780651 -74.7134578343742) bank96631 +96632 POINT(40.13642155884283 -74.55519061412569) bank96632 +96633 POINT(40.74512707653914 -74.32633033957964) bank96633 +96634 POINT(40.370309147349005 -73.71223054558668) bank96634 +96635 POINT(40.07737543397523 -74.42687838026262) bank96635 +96636 POINT(40.179408087984214 -73.68459994934513) bank96636 +96637 POINT(39.92436034587997 -74.05426598397864) bank96637 +96638 POINT(41.21772526717316 -73.84757074287357) bank96638 +96639 POINT(40.59153751641316 -74.7354279467053) bank96639 +96640 POINT(40.3309662200739 -73.25304569913584) bank96640 +96641 POINT(40.367094711524665 -73.48468493120171) bank96641 +96642 POINT(41.32774906039421 -73.3619052453953) bank96642 +96643 POINT(41.043195254554405 -73.46343982365808) bank96643 +96644 POINT(41.10841888246914 -74.3456203912829) bank96644 +96645 POINT(40.9803912749327 -74.56468129058545) bank96645 +96646 POINT(41.50427739725837 -73.09242999775387) bank96646 +96647 POINT(41.235650670734834 -73.59349618019499) bank96647 +96648 POINT(40.579407640372956 -74.27567929623756) bank96648 +96649 POINT(40.908021574940705 -73.99651948934269) bank96649 +96650 POINT(40.69771994888108 -73.98908581725331) bank96650 +96651 POINT(41.1427297724701 -73.69616467784465) bank96651 +96652 POINT(40.40089586668811 -74.62346662715574) bank96652 +96653 POINT(40.675977626734394 -74.3486243400031) bank96653 +96654 POINT(40.88138439757913 -73.77710889859772) bank96654 +96655 POINT(40.30902596572376 -74.9831797017154) bank96655 +96656 POINT(39.94298044452757 -74.21020974105802) bank96656 +96657 POINT(41.57897827135257 -73.64076995040043) bank96657 +96658 POINT(40.090233283730534 -73.19122923162959) bank96658 +96659 POINT(40.15531421182606 -74.65249602420839) bank96659 +96660 POINT(41.627831597138005 -74.95884769582526) bank96660 +96661 POINT(39.88484620937305 -73.79986308675348) bank96661 +96662 POINT(41.6272553234003 -74.85974002456557) bank96662 +96663 POINT(40.162327255288446 -74.38472181439414) bank96663 +96664 POINT(40.23534073928012 -73.47079968544337) bank96664 +96665 POINT(40.11854277899439 -74.82754978416753) bank96665 +96666 POINT(40.459680756829314 -74.04071523648369) bank96666 +96667 POINT(40.85976219710119 -73.41876209594739) bank96667 +96668 POINT(39.790156880691484 -74.1959790173575) bank96668 +96669 POINT(39.78076821205015 -74.73101728204655) bank96669 +96670 POINT(41.534168086689576 -74.39272063632761) bank96670 +96671 POINT(40.50404879446137 -73.91353070651701) bank96671 +96672 POINT(41.52229056029147 -73.42821814810414) bank96672 +96673 POINT(41.20077990087096 -73.87036104728101) bank96673 +96674 POINT(41.62870906151569 -74.15774933343513) bank96674 +96675 POINT(40.38558309250772 -74.6738381296546) bank96675 +96676 POINT(40.124064582460875 -74.61410978328328) bank96676 +96677 POINT(40.63116733173076 -74.64034714949638) bank96677 +96678 POINT(40.493392688772225 -74.8667645506713) bank96678 +96679 POINT(40.17905458332421 -74.5220250547087) bank96679 +96680 POINT(40.46851633830539 -74.40085392062362) bank96680 +96681 POINT(41.30040573139398 -73.99480446030705) bank96681 +96682 POINT(40.556590439503495 -73.9297729586968) bank96682 +96683 POINT(40.06492660694925 -73.70940236595638) bank96683 +96684 POINT(40.81210004321558 -73.72485593120645) bank96684 +96685 POINT(40.10823763274416 -74.77909886774549) bank96685 +96686 POINT(40.78906151704744 -73.18896473374112) bank96686 +96687 POINT(40.23311338914986 -74.83671691643794) bank96687 +96688 POINT(41.6760490345358 -74.14018247532286) bank96688 +96689 POINT(40.93963994555783 -73.26006064346055) bank96689 +96690 POINT(40.21155854971671 -73.39855333551253) bank96690 +96691 POINT(40.23686987125124 -73.52609499943307) bank96691 +96692 POINT(40.68951210004016 -74.64612174314921) bank96692 +96693 POINT(40.55195170642563 -74.91383750641864) bank96693 +96694 POINT(41.36548868960821 -73.07900958276863) bank96694 +96695 POINT(41.59493097036257 -74.12588696950118) bank96695 +96696 POINT(41.269708514987784 -73.40461755536514) bank96696 +96697 POINT(40.86998225305211 -73.2325924064454) bank96697 +96698 POINT(40.0931740331319 -73.81537321218119) bank96698 +96699 POINT(40.985350717430386 -74.82648509327396) bank96699 +96700 POINT(41.67528120698858 -73.35728323437748) bank96700 +96701 POINT(40.83172084324456 -73.76524510954671) bank96701 +96702 POINT(40.93416606760863 -74.24159776754227) bank96702 +96703 POINT(41.311244534260986 -74.14594478100786) bank96703 +96704 POINT(41.545794832443455 -73.90004473716665) bank96704 +96705 POINT(41.37424327987449 -74.91169958965239) bank96705 +96706 POINT(41.40807086540303 -74.906711258838) bank96706 +96707 POINT(40.91744259065658 -73.78789999622911) bank96707 +96708 POINT(41.443838145600814 -73.72037941332017) bank96708 +96709 POINT(40.606535348867524 -73.18901302998924) bank96709 +96710 POINT(40.01091734714126 -74.55653263124923) bank96710 +96711 POINT(39.90950941815791 -73.74089322218073) bank96711 +96712 POINT(40.79828495098733 -74.84296094530104) bank96712 +96713 POINT(40.79161282008643 -73.11069882823392) bank96713 +96714 POINT(40.863929953235946 -74.66851599995672) bank96714 +96715 POINT(40.24463281264275 -74.35159530701128) bank96715 +96716 POINT(40.64996788265767 -74.69281133578609) bank96716 +96717 POINT(41.06585046401899 -74.65953470641692) bank96717 +96718 POINT(41.31469474372767 -73.07916101246617) bank96718 +96719 POINT(40.50871849984122 -73.1575355367856) bank96719 +96720 POINT(39.724094731415455 -73.6121836091584) bank96720 +96721 POINT(41.01368223066095 -74.64325475627658) bank96721 +96722 POINT(41.42787337809741 -74.8417047531387) bank96722 +96723 POINT(40.36199731934454 -74.03347290862482) bank96723 +96724 POINT(41.14768439133084 -74.30747247199386) bank96724 +96725 POINT(40.086786242580835 -73.89466518361276) bank96725 +96726 POINT(41.29901871434543 -73.55545428067273) bank96726 +96727 POINT(39.75085920438819 -73.70102697916462) bank96727 +96728 POINT(41.68678682614944 -73.08240494234644) bank96728 +96729 POINT(40.92918368004178 -73.13840112995389) bank96729 +96730 POINT(40.61711163339989 -73.56628007197422) bank96730 +96731 POINT(40.296077724217675 -73.19084885481172) bank96731 +96732 POINT(41.63393483618625 -74.03787568082475) bank96732 +96733 POINT(39.98828608052424 -74.12098362678799) bank96733 +96734 POINT(40.12561259736733 -74.37664356517847) bank96734 +96735 POINT(40.43766230165716 -73.2132874362152) bank96735 +96736 POINT(40.12412264023031 -73.61569861750115) bank96736 +96737 POINT(40.88046928670778 -74.67601307302766) bank96737 +96738 POINT(40.684157977376536 -73.34635824064213) bank96738 +96739 POINT(40.44025225908048 -74.47501677598282) bank96739 +96740 POINT(40.8178318128548 -74.72180955282496) bank96740 +96741 POINT(40.17064634219147 -74.06692352447617) bank96741 +96742 POINT(40.271157895623915 -74.72707184079565) bank96742 +96743 POINT(40.267626533656305 -74.95393327148257) bank96743 +96744 POINT(41.65864286238279 -73.72028996067239) bank96744 +96745 POINT(41.20547248295076 -74.95582294016339) bank96745 +96746 POINT(40.84939285471214 -74.86276821616833) bank96746 +96747 POINT(40.802163249572516 -73.88810412655587) bank96747 +96748 POINT(40.470814851329955 -74.76851177807075) bank96748 +96749 POINT(40.047506724348324 -74.72432341832885) bank96749 +96750 POINT(41.317351736993416 -74.86100327857598) bank96750 +96751 POINT(41.13686470385835 -73.51340473988337) bank96751 +96752 POINT(41.15514658555665 -73.97198476733247) bank96752 +96753 POINT(39.785286885395614 -74.47655591676128) bank96753 +96754 POINT(40.19767510165684 -73.56248399240496) bank96754 +96755 POINT(41.022277894454255 -73.94020183373443) bank96755 +96756 POINT(39.898756327733466 -74.68129210800701) bank96756 +96757 POINT(41.595000814471554 -73.0495461188952) bank96757 +96758 POINT(41.09932382841469 -74.1503286604101) bank96758 +96759 POINT(40.44438340572224 -73.15895998854181) bank96759 +96760 POINT(40.189958314359416 -74.98416174428759) bank96760 +96761 POINT(41.32741184253309 -73.47059328862991) bank96761 +96762 POINT(41.640120419756485 -74.65999180350995) bank96762 +96763 POINT(41.01450362209436 -73.77133296905203) bank96763 +96764 POINT(40.39859158733503 -73.10911881869933) bank96764 +96765 POINT(41.574803050343625 -73.40413597307847) bank96765 +96766 POINT(40.89823146201261 -73.32256921664998) bank96766 +96767 POINT(40.46926332501801 -73.36437303106749) bank96767 +96768 POINT(40.461177789658024 -73.57022913510004) bank96768 +96769 POINT(40.48855157290258 -74.39364443617933) bank96769 +96770 POINT(40.95906193353634 -73.37989814296682) bank96770 +96771 POINT(40.83006047626218 -74.06148515768922) bank96771 +96772 POINT(40.25298333970848 -74.40050138315061) bank96772 +96773 POINT(40.946392824483716 -74.3614819132679) bank96773 +96774 POINT(40.23768523855678 -73.65133933849835) bank96774 +96775 POINT(39.76907375537068 -74.78127331422833) bank96775 +96776 POINT(40.364680127281986 -73.70299940000027) bank96776 +96777 POINT(41.33359460909153 -73.25804792199419) bank96777 +96778 POINT(41.6507142364043 -73.90896042429216) bank96778 +96779 POINT(39.79903821454125 -73.99422370773974) bank96779 +96780 POINT(39.81896105249057 -74.79696316142575) bank96780 +96781 POINT(40.51570715370111 -74.97472177604263) bank96781 +96782 POINT(39.97317224508311 -73.51721242622706) bank96782 +96783 POINT(41.61432249190877 -73.81698828686547) bank96783 +96784 POINT(40.89058037886265 -74.45277834809562) bank96784 +96785 POINT(40.88612877651401 -74.1202628881049) bank96785 +96786 POINT(40.00431912103365 -74.74754141680812) bank96786 +96787 POINT(40.803757483397376 -74.22902233886998) bank96787 +96788 POINT(40.307702125696665 -74.26444456115853) bank96788 +96789 POINT(40.53383679529398 -73.38688506001526) bank96789 +96790 POINT(39.89437647427203 -73.20708780257706) bank96790 +96791 POINT(41.00353810052082 -74.78811320934619) bank96791 +96792 POINT(41.41145350972942 -73.6326955339563) bank96792 +96793 POINT(40.86589993910982 -73.65096500609759) bank96793 +96794 POINT(41.31408671838307 -73.43231459825284) bank96794 +96795 POINT(41.31044974105791 -74.06026407610172) bank96795 +96796 POINT(40.983316291853 -73.80836946573069) bank96796 +96797 POINT(39.92742057929234 -73.12032772891351) bank96797 +96798 POINT(40.88600114955366 -74.57439205748616) bank96798 +96799 POINT(41.09153138695634 -73.8968597751056) bank96799 +96800 POINT(41.514127190325 -73.44137931572851) bank96800 +96801 POINT(40.50014276563227 -74.59057481109883) bank96801 +96802 POINT(40.65362316751006 -73.88691126047834) bank96802 +96803 POINT(40.447848820029435 -73.82240775496501) bank96803 +96804 POINT(41.188025457368624 -74.51281019787493) bank96804 +96805 POINT(40.16862492124271 -74.48054450496224) bank96805 +96806 POINT(40.70270173601101 -73.1617555132212) bank96806 +96807 POINT(40.35496908964389 -73.46942382705625) bank96807 +96808 POINT(40.758973904873706 -73.84399044967391) bank96808 +96809 POINT(40.85123360885822 -74.72273707493169) bank96809 +96810 POINT(40.217334730590835 -74.15196904240759) bank96810 +96811 POINT(41.11542177187059 -73.97249277063638) bank96811 +96812 POINT(41.66040449167907 -74.66574323318966) bank96812 +96813 POINT(39.718774883358904 -73.58567388317965) bank96813 +96814 POINT(41.68692436232212 -74.51992674908865) bank96814 +96815 POINT(40.26559980433692 -74.35214887859686) bank96815 +96816 POINT(40.597742764695994 -74.07876285459994) bank96816 +96817 POINT(41.45944501530251 -73.20347912728913) bank96817 +96818 POINT(39.97464873068721 -73.01544701748882) bank96818 +96819 POINT(40.252938060923505 -73.8130864094106) bank96819 +96820 POINT(41.16846714939101 -73.40522618101646) bank96820 +96821 POINT(40.03489801070959 -74.62654387611283) bank96821 +96822 POINT(40.55539037148953 -74.88618446658273) bank96822 +96823 POINT(40.43237222452833 -73.21361526356016) bank96823 +96824 POINT(40.69575324694411 -73.28050161080135) bank96824 +96825 POINT(39.947290238714615 -74.66294211359262) bank96825 +96826 POINT(39.92796726412347 -73.48395216518614) bank96826 +96827 POINT(40.467339605637804 -73.44138614798935) bank96827 +96828 POINT(41.64605751366411 -73.43111793984481) bank96828 +96829 POINT(40.43330623357099 -74.236568267055) bank96829 +96830 POINT(39.839837795923934 -74.77516899893891) bank96830 +96831 POINT(41.048430546768515 -73.29565910169764) bank96831 +96832 POINT(40.03200656776858 -73.67023047653718) bank96832 +96833 POINT(40.968970325949876 -73.70074462162452) bank96833 +96834 POINT(39.7392189819123 -73.96452208612823) bank96834 +96835 POINT(39.94502513209635 -74.65638332064628) bank96835 +96836 POINT(40.580445012208315 -74.45986961024879) bank96836 +96837 POINT(40.082490377610334 -73.67693268027855) bank96837 +96838 POINT(41.51386599478696 -73.23596208474675) bank96838 +96839 POINT(41.55747757369837 -74.50460239876809) bank96839 +96840 POINT(40.34545352555314 -74.30009304225852) bank96840 +96841 POINT(40.439166408938654 -73.33422017193409) bank96841 +96842 POINT(40.30974070539665 -74.2051316269803) bank96842 +96843 POINT(40.508918569954034 -74.90847067703095) bank96843 +96844 POINT(41.16422502767171 -73.73444155969968) bank96844 +96845 POINT(40.37512211784259 -74.2951274660717) bank96845 +96846 POINT(41.39546953987939 -73.18572760335996) bank96846 +96847 POINT(41.05989307318425 -73.3790624882301) bank96847 +96848 POINT(40.37922287885489 -74.07540164569734) bank96848 +96849 POINT(40.54554770360369 -74.69411909159722) bank96849 +96850 POINT(39.85305764460753 -73.04234653593579) bank96850 +96851 POINT(39.78505923745595 -73.35400527587859) bank96851 +96852 POINT(41.50608958569752 -73.77607898994896) bank96852 +96853 POINT(41.013224501132 -74.30297771849112) bank96853 +96854 POINT(40.53700240841175 -73.41614372101773) bank96854 +96855 POINT(39.852989598692666 -73.16148140813947) bank96855 +96856 POINT(40.700655793031956 -74.74093993353327) bank96856 +96857 POINT(39.77769253993246 -74.15247054314027) bank96857 +96858 POINT(39.78832656041224 -74.16772343416845) bank96858 +96859 POINT(41.25938653760479 -73.86063266688689) bank96859 +96860 POINT(41.326408098633166 -73.31352151419604) bank96860 +96861 POINT(40.97026181227347 -74.14741758272473) bank96861 +96862 POINT(40.07019008404838 -74.55282620489982) bank96862 +96863 POINT(40.07998862163189 -73.65599003494282) bank96863 +96864 POINT(40.54682878312084 -73.02060165970994) bank96864 +96865 POINT(41.035355046348876 -74.27693361748378) bank96865 +96866 POINT(41.554544846982125 -74.11523509794573) bank96866 +96867 POINT(40.36597518572653 -73.91043886403503) bank96867 +96868 POINT(40.13045657002082 -74.66644107638342) bank96868 +96869 POINT(41.05501826619001 -74.62116651521818) bank96869 +96870 POINT(41.550413812529605 -73.04556461137321) bank96870 +96871 POINT(40.279633267561096 -74.36932761397847) bank96871 +96872 POINT(41.414505051130426 -74.22392275559811) bank96872 +96873 POINT(41.64644874682725 -73.78723121146055) bank96873 +96874 POINT(40.331213955702374 -73.92013726052241) bank96874 +96875 POINT(40.342189099526095 -73.2619813890933) bank96875 +96876 POINT(40.17048357237412 -74.43188388716767) bank96876 +96877 POINT(41.240372651586775 -73.38975589043434) bank96877 +96878 POINT(41.558407268244615 -73.93270471259515) bank96878 +96879 POINT(40.58416258225399 -74.35301628529811) bank96879 +96880 POINT(40.71970886702117 -74.604912017985) bank96880 +96881 POINT(41.23408476651536 -74.5917332648733) bank96881 +96882 POINT(41.61692131725534 -74.01684503317408) bank96882 +96883 POINT(40.6362094719216 -73.04552238689722) bank96883 +96884 POINT(40.35198062280936 -73.70560611980227) bank96884 +96885 POINT(39.94215685455937 -73.14921082531495) bank96885 +96886 POINT(41.23325735981227 -74.6218862776587) bank96886 +96887 POINT(40.106501915280134 -74.93447115495647) bank96887 +96888 POINT(40.5935061708347 -73.1188633569318) bank96888 +96889 POINT(40.720707250012715 -74.53198313504352) bank96889 +96890 POINT(39.779174676951044 -73.35578030977405) bank96890 +96891 POINT(40.18401253377862 -73.22261147996748) bank96891 +96892 POINT(41.1233171534561 -74.26015675541053) bank96892 +96893 POINT(41.05518179216682 -73.86153791646754) bank96893 +96894 POINT(41.598194602546535 -74.91069157441838) bank96894 +96895 POINT(41.115879418160596 -74.01709084481578) bank96895 +96896 POINT(40.37400365990119 -73.6398529362151) bank96896 +96897 POINT(39.87412946866312 -73.33424426065547) bank96897 +96898 POINT(40.23022078208734 -74.35395474788137) bank96898 +96899 POINT(40.2693484683859 -73.59305973781575) bank96899 +96900 POINT(40.00975464990937 -73.33329187256925) bank96900 +96901 POINT(40.14025077653093 -73.92807392842822) bank96901 +96902 POINT(41.576628903333614 -73.09743197659047) bank96902 +96903 POINT(39.75384685603456 -73.05127543410441) bank96903 +96904 POINT(40.052377470605144 -74.20086997668832) bank96904 +96905 POINT(41.621882413674555 -74.56849075513065) bank96905 +96906 POINT(41.374039564462436 -74.55601823827377) bank96906 +96907 POINT(40.45282903052657 -73.7077824457947) bank96907 +96908 POINT(40.040503757659856 -73.85180797073885) bank96908 +96909 POINT(41.03922484084557 -73.59068551511359) bank96909 +96910 POINT(41.532094010549024 -74.98208076021417) bank96910 +96911 POINT(40.35626460200233 -74.46969731088423) bank96911 +96912 POINT(39.9880336622383 -74.48068066772235) bank96912 +96913 POINT(41.6909759973919 -74.59307898662114) bank96913 +96914 POINT(39.929981543570364 -73.25822324193831) bank96914 +96915 POINT(39.86540713694146 -74.26742005315319) bank96915 +96916 POINT(41.40637920435583 -74.13023135218373) bank96916 +96917 POINT(40.582172714290664 -73.36086119891799) bank96917 +96918 POINT(41.5824032457908 -74.95536892618213) bank96918 +96919 POINT(40.05137214779194 -74.51006167979338) bank96919 +96920 POINT(40.92179767113782 -74.12036551418154) bank96920 +96921 POINT(40.83272203517399 -74.2651973526074) bank96921 +96922 POINT(41.284396994749564 -74.3648450053813) bank96922 +96923 POINT(41.41896712139221 -73.10470263203028) bank96923 +96924 POINT(41.086460829299355 -73.7521850362926) bank96924 +96925 POINT(40.85345325287171 -73.08594834008295) bank96925 +96926 POINT(40.45345275551532 -73.72425253199346) bank96926 +96927 POINT(41.700216805615895 -74.97988313751934) bank96927 +96928 POINT(41.62736542044638 -73.09002907279009) bank96928 +96929 POINT(40.341734956901895 -73.6389103334132) bank96929 +96930 POINT(41.6125960066666 -73.06243068781077) bank96930 +96931 POINT(41.63872520467091 -74.64438938608762) bank96931 +96932 POINT(40.6579063316811 -74.43415715237813) bank96932 +96933 POINT(41.38181505994981 -73.04819984009467) bank96933 +96934 POINT(40.21171614157481 -73.3602579194868) bank96934 +96935 POINT(40.57222224741932 -74.67286687087568) bank96935 +96936 POINT(40.33758973957739 -74.04188404212097) bank96936 +96937 POINT(40.794537174235536 -74.74429339856242) bank96937 +96938 POINT(41.57858357196797 -74.11489549826797) bank96938 +96939 POINT(40.05200422792697 -74.87158683829813) bank96939 +96940 POINT(40.95115225665015 -74.90870537600698) bank96940 +96941 POINT(41.389718804974834 -74.27543706932923) bank96941 +96942 POINT(41.637610112179495 -74.54227969916799) bank96942 +96943 POINT(41.36267255750498 -74.15613558699124) bank96943 +96944 POINT(40.839874598031045 -73.14088371663095) bank96944 +96945 POINT(41.64691857243461 -74.930033335358) bank96945 +96946 POINT(40.779523936825576 -73.45067338400507) bank96946 +96947 POINT(41.556760385696265 -73.95336781974095) bank96947 +96948 POINT(40.79314590900924 -73.20021424822276) bank96948 +96949 POINT(41.043590139526 -73.13781100424094) bank96949 +96950 POINT(40.82960365573931 -73.12061102091735) bank96950 +96951 POINT(40.037271859985 -74.4348970260581) bank96951 +96952 POINT(41.235275036171934 -74.84153556497253) bank96952 +96953 POINT(41.55907492880707 -74.9187905955162) bank96953 +96954 POINT(40.73451931525469 -74.82164228297212) bank96954 +96955 POINT(41.687457645128816 -73.14496054980575) bank96955 +96956 POINT(41.090208171510724 -73.32635486165076) bank96956 +96957 POINT(41.37073354007404 -74.99292684138405) bank96957 +96958 POINT(40.600225950881175 -74.39577538058073) bank96958 +96959 POINT(40.43754306203182 -73.21925358881461) bank96959 +96960 POINT(40.20273685758778 -74.51147959640596) bank96960 +96961 POINT(41.162332967299676 -74.58833042120703) bank96961 +96962 POINT(40.66815650368357 -74.37647330365655) bank96962 +96963 POINT(41.189825010631104 -74.38351669515622) bank96963 +96964 POINT(39.8405975454659 -73.88760840795047) bank96964 +96965 POINT(41.54415156518307 -74.22920623762818) bank96965 +96966 POINT(41.25547406455509 -74.9401764464336) bank96966 +96967 POINT(41.079536590558 -74.70689600724153) bank96967 +96968 POINT(40.881836632188694 -74.46355758139416) bank96968 +96969 POINT(40.22126018662609 -74.32758851605234) bank96969 +96970 POINT(41.41722457595927 -73.17822295624629) bank96970 +96971 POINT(40.24530156114257 -73.19524378800644) bank96971 +96972 POINT(40.14208873748528 -74.40024675415361) bank96972 +96973 POINT(39.98108405809524 -73.70383632960602) bank96973 +96974 POINT(41.37977004154538 -73.95520073289197) bank96974 +96975 POINT(39.74194430474297 -73.35254273552216) bank96975 +96976 POINT(41.39424747881403 -74.55985265841107) bank96976 +96977 POINT(40.26152949818051 -73.04185832533362) bank96977 +96978 POINT(39.86326391620047 -74.40023728822513) bank96978 +96979 POINT(40.75709961555743 -74.57717378954518) bank96979 +96980 POINT(40.07065309779979 -74.38373376837792) bank96980 +96981 POINT(40.75251082450823 -74.82712432242405) bank96981 +96982 POINT(40.0079467509655 -74.56463740136506) bank96982 +96983 POINT(40.782895036104264 -74.17466018891669) bank96983 +96984 POINT(39.88715705549247 -74.89290255135745) bank96984 +96985 POINT(40.09521895172214 -73.56144716742371) bank96985 +96986 POINT(41.24034668508367 -73.5877277419828) bank96986 +96987 POINT(41.06622552377952 -74.36426182776) bank96987 +96988 POINT(40.75747084939652 -73.09210103210573) bank96988 +96989 POINT(41.67003557624193 -73.35013876231345) bank96989 +96990 POINT(41.29567879855319 -74.31529829678652) bank96990 +96991 POINT(41.17033220949576 -74.98460867870622) bank96991 +96992 POINT(40.12481273787691 -73.6997252430448) bank96992 +96993 POINT(40.988922548518 -73.54691675242049) bank96993 +96994 POINT(40.878382968967344 -74.75261655435393) bank96994 +96995 POINT(40.39041527005355 -74.68840709299369) bank96995 +96996 POINT(40.24484395125044 -73.76011758779853) bank96996 +96997 POINT(41.7020319811514 -74.22714399886888) bank96997 +96998 POINT(40.78594998167494 -73.85030321485053) bank96998 +96999 POINT(41.23125632789149 -74.26134628690167) bank96999 +97000 POINT(40.05375617408576 -74.9183500808667) bank97000 +97001 POINT(41.32335276236274 -74.99905125082942) bank97001 +97002 POINT(41.51586465380221 -73.52260400524959) bank97002 +97003 POINT(41.60683716673015 -74.96422544615885) bank97003 +97004 POINT(41.499142877167465 -73.37910631147176) bank97004 +97005 POINT(39.81880573582302 -74.76135864636089) bank97005 +97006 POINT(41.621644509296424 -73.8265152134741) bank97006 +97007 POINT(41.418695913951495 -74.12711365691858) bank97007 +97008 POINT(39.89972992727346 -73.62918602604577) bank97008 +97009 POINT(41.46368148043377 -74.48851221727315) bank97009 +97010 POINT(40.87042942623431 -74.14825728303428) bank97010 +97011 POINT(40.39346355089052 -73.96381544588485) bank97011 +97012 POINT(40.905198308247236 -73.37721101799626) bank97012 +97013 POINT(41.63271077692687 -74.25262060432304) bank97013 +97014 POINT(40.14198464922161 -73.24821911825539) bank97014 +97015 POINT(41.147346445321936 -73.17284431530848) bank97015 +97016 POINT(40.61829389132009 -74.3889386378418) bank97016 +97017 POINT(39.97770153958538 -73.76783058018927) bank97017 +97018 POINT(40.18992019570251 -73.68168259666498) bank97018 +97019 POINT(41.36963063472937 -73.87700759532872) bank97019 +97020 POINT(40.25744768290588 -73.95665790462232) bank97020 +97021 POINT(40.67698355207544 -73.12833128157486) bank97021 +97022 POINT(41.58680151060663 -73.47192904133236) bank97022 +97023 POINT(40.56769600642766 -74.49907905176879) bank97023 +97024 POINT(41.50594987911597 -73.64952237109078) bank97024 +97025 POINT(40.34736007224531 -73.02525138413529) bank97025 +97026 POINT(41.46791918419655 -73.16050899874811) bank97026 +97027 POINT(40.42297177189674 -74.4891770516622) bank97027 +97028 POINT(41.075253443973374 -74.22591660816025) bank97028 +97029 POINT(41.6892959941299 -74.49349707680526) bank97029 +97030 POINT(41.01291578714663 -73.87100176640779) bank97030 +97031 POINT(41.02819639577087 -74.8471088671361) bank97031 +97032 POINT(40.872279610978964 -74.21992688347338) bank97032 +97033 POINT(40.234092939831555 -74.04185190765564) bank97033 +97034 POINT(40.095577386327555 -74.03351619858599) bank97034 +97035 POINT(41.23507172748894 -73.51507532502903) bank97035 +97036 POINT(40.37586478090047 -74.21786113350082) bank97036 +97037 POINT(40.721450637020304 -74.02981408509237) bank97037 +97038 POINT(40.30356482803876 -74.75637318873318) bank97038 +97039 POINT(40.37975974288633 -74.85942103249576) bank97039 +97040 POINT(41.33822042382651 -73.71109422512733) bank97040 +97041 POINT(40.89172320628885 -73.74478682774419) bank97041 +97042 POINT(41.18333255581821 -73.63814418031937) bank97042 +97043 POINT(41.42030226809167 -73.19383886484933) bank97043 +97044 POINT(39.730755546368684 -74.15419848724859) bank97044 +97045 POINT(41.130940773166 -73.92201270691315) bank97045 +97046 POINT(41.69092327459464 -73.82508167074641) bank97046 +97047 POINT(41.70723230843626 -74.65202978606224) bank97047 +97048 POINT(40.54125354448334 -73.61366717395573) bank97048 +97049 POINT(40.016686746314136 -73.22209580449677) bank97049 +97050 POINT(41.10730274812873 -74.35851641987283) bank97050 +97051 POINT(40.192471116625796 -73.61902588849834) bank97051 +97052 POINT(40.773129302462074 -73.74606513906538) bank97052 +97053 POINT(40.19279717213659 -74.55146735601441) bank97053 +97054 POINT(40.933967375154786 -74.08859534187478) bank97054 +97055 POINT(39.7183856700625 -74.24624082489768) bank97055 +97056 POINT(39.893346816093015 -73.3470589907501) bank97056 +97057 POINT(40.21184055408111 -74.6149882052392) bank97057 +97058 POINT(41.445075654057305 -74.80561339607948) bank97058 +97059 POINT(41.09018661852131 -73.880890279771) bank97059 +97060 POINT(40.637560033260556 -73.19429821712342) bank97060 +97061 POINT(40.3277190003372 -73.60270394464703) bank97061 +97062 POINT(40.89665794212646 -74.12922717042375) bank97062 +97063 POINT(40.33902489601034 -73.44668862255357) bank97063 +97064 POINT(40.20214754510421 -73.27998866084522) bank97064 +97065 POINT(41.07507810721684 -73.03715304277928) bank97065 +97066 POINT(40.956988694876756 -73.89828763711084) bank97066 +97067 POINT(40.33620230445232 -74.14480727256195) bank97067 +97068 POINT(41.474022214295886 -73.46198752718533) bank97068 +97069 POINT(39.74204205243767 -74.1384795349374) bank97069 +97070 POINT(39.8602776297158 -73.6884637989111) bank97070 +97071 POINT(39.94720597723406 -74.70097527123151) bank97071 +97072 POINT(40.15998579060401 -73.36953621715158) bank97072 +97073 POINT(40.709348727887786 -73.55649271409614) bank97073 +97074 POINT(40.387872063194294 -74.86919979764522) bank97074 +97075 POINT(40.19549029690367 -74.31119527895864) bank97075 +97076 POINT(40.047610312562206 -73.69273203869766) bank97076 +97077 POINT(40.32051923376675 -73.30671056257549) bank97077 +97078 POINT(40.34743577032104 -74.81287863563755) bank97078 +97079 POINT(40.925214364954776 -74.26137844975784) bank97079 +97080 POINT(40.60647403151715 -73.71121185868353) bank97080 +97081 POINT(39.937193966097816 -74.06486439800626) bank97081 +97082 POINT(40.67141176702966 -74.49824891013434) bank97082 +97083 POINT(41.03405810074122 -73.1293298365217) bank97083 +97084 POINT(39.908786028591784 -73.48122766895524) bank97084 +97085 POINT(41.40310577898496 -74.05855277551763) bank97085 +97086 POINT(40.71421516168044 -73.90009008468773) bank97086 +97087 POINT(40.01590214709519 -73.29392048013642) bank97087 +97088 POINT(41.128706390059676 -73.91416101311316) bank97088 +97089 POINT(41.031910192994594 -73.3790159785579) bank97089 +97090 POINT(40.722759364159685 -73.34704644142022) bank97090 +97091 POINT(40.537122482353894 -74.9036280758336) bank97091 +97092 POINT(41.07947399621655 -74.482994904642) bank97092 +97093 POINT(40.45876395444963 -73.93105647766731) bank97093 +97094 POINT(40.58439136919435 -73.34844115998078) bank97094 +97095 POINT(41.396787324181176 -74.85260479075586) bank97095 +97096 POINT(40.97068390055269 -74.14098577067934) bank97096 +97097 POINT(41.19216386413959 -74.94538917295422) bank97097 +97098 POINT(40.231872249652646 -73.44672874270043) bank97098 +97099 POINT(41.536691495496534 -73.69597144242218) bank97099 +97100 POINT(41.67243947193146 -73.55441690389287) bank97100 +97101 POINT(40.81039137224376 -74.41217084090616) bank97101 +97102 POINT(40.485387894119945 -73.15391044360048) bank97102 +97103 POINT(41.687500923314765 -73.61999707811435) bank97103 +97104 POINT(40.250396660515705 -74.21750233608331) bank97104 +97105 POINT(40.61064384502389 -74.63745547264821) bank97105 +97106 POINT(40.3514273458458 -73.12336238531704) bank97106 +97107 POINT(40.03295804969608 -74.34090843289543) bank97107 +97108 POINT(39.830871863906594 -73.04082496867784) bank97108 +97109 POINT(39.916751403332626 -74.14347162568926) bank97109 +97110 POINT(41.505454828690695 -73.1785851591808) bank97110 +97111 POINT(41.6624154705845 -73.35178688793594) bank97111 +97112 POINT(40.870089317862444 -74.28965048313894) bank97112 +97113 POINT(41.184626546778325 -73.04923791625774) bank97113 +97114 POINT(41.459532724238755 -73.80841113095715) bank97114 +97115 POINT(41.6122334727903 -73.2336432188115) bank97115 +97116 POINT(40.183653037737564 -73.40804420354597) bank97116 +97117 POINT(40.62364433690921 -74.43539554286394) bank97117 +97118 POINT(40.65107516008792 -73.42119051975345) bank97118 +97119 POINT(41.63396260761388 -73.38954249365452) bank97119 +97120 POINT(40.392238848053594 -73.52976459163246) bank97120 +97121 POINT(40.85570723502088 -74.71763501479619) bank97121 +97122 POINT(41.617388143399175 -74.13803108329282) bank97122 +97123 POINT(39.999807409879445 -73.34522585885907) bank97123 +97124 POINT(41.40691015487566 -73.32155454603419) bank97124 +97125 POINT(41.12675621417847 -74.08800983311616) bank97125 +97126 POINT(40.57006790005217 -73.43816302177065) bank97126 +97127 POINT(40.262038345443486 -74.09788219335371) bank97127 +97128 POINT(40.355626826614746 -73.87918486875341) bank97128 +97129 POINT(40.81025191082461 -74.38265880520213) bank97129 +97130 POINT(40.94625898653499 -74.97062055597901) bank97130 +97131 POINT(41.37121226532113 -73.3016645539448) bank97131 +97132 POINT(40.30681911093746 -73.16913504819455) bank97132 +97133 POINT(40.01037380298097 -74.64901971811022) bank97133 +97134 POINT(40.124740077188875 -74.50249824252602) bank97134 +97135 POINT(39.865448897359876 -74.17491716664749) bank97135 +97136 POINT(40.72533439795863 -74.08707727480343) bank97136 +97137 POINT(39.752808769560644 -74.90088230007666) bank97137 +97138 POINT(41.51332466030225 -74.9870187080804) bank97138 +97139 POINT(40.772078646505115 -74.84825796960597) bank97139 +97140 POINT(39.739465058705704 -73.12612726485764) bank97140 +97141 POINT(40.18398269907066 -74.57847825596293) bank97141 +97142 POINT(41.51032711034339 -73.7890760148534) bank97142 +97143 POINT(41.61405462647709 -73.6354856118401) bank97143 +97144 POINT(41.42092402029684 -74.77552826441874) bank97144 +97145 POINT(40.836770163067285 -74.7612248496395) bank97145 +97146 POINT(40.460893758267495 -74.12948896873519) bank97146 +97147 POINT(41.06520344392362 -73.50799100842428) bank97147 +97148 POINT(41.28679250649111 -73.80822934640591) bank97148 +97149 POINT(41.192540555422056 -74.3221208936528) bank97149 +97150 POINT(41.28245482843387 -74.45384974921849) bank97150 +97151 POINT(41.49639615302823 -74.88981101435297) bank97151 +97152 POINT(40.18430663812679 -73.8305787991482) bank97152 +97153 POINT(40.173560123230175 -74.4347620428571) bank97153 +97154 POINT(41.56728477720786 -74.09510159183353) bank97154 +97155 POINT(40.23658714587826 -73.86467667111297) bank97155 +97156 POINT(41.26225164005769 -73.21406708005439) bank97156 +97157 POINT(41.0249910083586 -74.73412111773963) bank97157 +97158 POINT(41.25704326813927 -73.88179254323978) bank97158 +97159 POINT(40.82157419548536 -73.33769852601739) bank97159 +97160 POINT(40.52967970447296 -74.49509880988838) bank97160 +97161 POINT(39.90971521557842 -73.27043956457472) bank97161 +97162 POINT(40.573835582888925 -74.93200374463545) bank97162 +97163 POINT(41.436656395751825 -74.63414042117559) bank97163 +97164 POINT(40.761058430635714 -74.6013715871909) bank97164 +97165 POINT(40.82021378281945 -73.22177981272928) bank97165 +97166 POINT(39.97193832266449 -74.16853857877169) bank97166 +97167 POINT(40.99042631333542 -74.29413662720022) bank97167 +97168 POINT(39.78997980268452 -73.89646792526494) bank97168 +97169 POINT(39.9078989770819 -73.28313127042924) bank97169 +97170 POINT(40.18736889833045 -74.8384840700058) bank97170 +97171 POINT(40.33636329577069 -74.88497544124684) bank97171 +97172 POINT(40.60794168771676 -73.35081468073537) bank97172 +97173 POINT(41.14307680394852 -74.50070756008962) bank97173 +97174 POINT(40.42694263768506 -73.92938515179343) bank97174 +97175 POINT(41.54441037469162 -74.39531151835298) bank97175 +97176 POINT(41.18713080728086 -73.9148284192102) bank97176 +97177 POINT(41.54789998993442 -74.06692344874718) bank97177 +97178 POINT(41.00090360445305 -73.65656766765082) bank97178 +97179 POINT(40.97442685943502 -73.68004370763651) bank97179 +97180 POINT(41.231802984848926 -74.10651368724373) bank97180 +97181 POINT(41.25938427016588 -73.95744362895059) bank97181 +97182 POINT(39.94245689339287 -74.14457712141369) bank97182 +97183 POINT(40.756419233403385 -74.91053178312735) bank97183 +97184 POINT(40.64764976373535 -74.38830389314379) bank97184 +97185 POINT(41.156693988345836 -74.4292214809159) bank97185 +97186 POINT(41.16015103131347 -74.89150201279494) bank97186 +97187 POINT(40.63064770289128 -74.98382639462167) bank97187 +97188 POINT(41.072042466476496 -73.7643463271696) bank97188 +97189 POINT(40.805832015286995 -73.88664574319087) bank97189 +97190 POINT(39.76388347726 -73.21591622930693) bank97190 +97191 POINT(40.77429728400813 -73.08823637270022) bank97191 +97192 POINT(39.74764496312295 -74.07498473937166) bank97192 +97193 POINT(39.76306660974182 -74.7647310653018) bank97193 +97194 POINT(40.0987652740661 -74.30090757243096) bank97194 +97195 POINT(39.974224192817 -74.20888298546544) bank97195 +97196 POINT(41.63674486231865 -73.18206133873298) bank97196 +97197 POINT(40.24026681419471 -73.45984494162744) bank97197 +97198 POINT(41.07451149439165 -73.59421529623117) bank97198 +97199 POINT(40.46195555910745 -73.85596871092613) bank97199 +97200 POINT(41.52718610741463 -74.39508856038486) bank97200 +97201 POINT(41.40388823848597 -74.52312156407359) bank97201 +97202 POINT(41.63070625181386 -74.30650544994629) bank97202 +97203 POINT(39.74556532010317 -73.90025960739955) bank97203 +97204 POINT(40.366976251141686 -73.72743268119748) bank97204 +97205 POINT(39.93410408231629 -74.93202745379045) bank97205 +97206 POINT(41.15148638288847 -74.5565463391405) bank97206 +97207 POINT(40.69141409884218 -73.54232620560553) bank97207 +97208 POINT(40.64808637990811 -73.63636388315467) bank97208 +97209 POINT(40.57882687198685 -73.09850651731894) bank97209 +97210 POINT(40.524608497341845 -74.88063107830925) bank97210 +97211 POINT(41.18675828542815 -73.20477674346553) bank97211 +97212 POINT(39.722482577909666 -74.88581275056767) bank97212 +97213 POINT(41.62733024286267 -73.96337284358428) bank97213 +97214 POINT(40.46867433791471 -74.11018131222603) bank97214 +97215 POINT(41.43812382583246 -74.09306149469839) bank97215 +97216 POINT(39.9443021889507 -73.6459141777351) bank97216 +97217 POINT(40.49361411968508 -73.43319942455561) bank97217 +97218 POINT(40.07641329749562 -74.76603258795043) bank97218 +97219 POINT(40.056328916641284 -73.25464354368391) bank97219 +97220 POINT(41.46221931747172 -74.04015973994238) bank97220 +97221 POINT(40.139310684062266 -73.51742375151937) bank97221 +97222 POINT(41.36794797665475 -73.3531699068504) bank97222 +97223 POINT(41.69944232219614 -74.02790673855391) bank97223 +97224 POINT(40.17110959052807 -73.08312352113617) bank97224 +97225 POINT(40.28655353902748 -73.25715279963813) bank97225 +97226 POINT(40.00993588347007 -73.46445386760104) bank97226 +97227 POINT(41.10659465007783 -74.92511094322707) bank97227 +97228 POINT(39.994313494931056 -73.26275594836342) bank97228 +97229 POINT(40.63464048354682 -73.4598003759934) bank97229 +97230 POINT(40.211898023449365 -73.67897478244258) bank97230 +97231 POINT(40.17449983475626 -73.57695734462368) bank97231 +97232 POINT(39.79309808630443 -73.02683210435002) bank97232 +97233 POINT(41.41828837520427 -74.4267123768321) bank97233 +97234 POINT(40.674368881768224 -73.28849613241303) bank97234 +97235 POINT(40.590394857628695 -74.47412444760515) bank97235 +97236 POINT(41.526389541242324 -73.15832350199682) bank97236 +97237 POINT(39.781894350791504 -73.28142128131823) bank97237 +97238 POINT(41.51189999955814 -74.47855640642359) bank97238 +97239 POINT(41.23747119887267 -73.41394204578876) bank97239 +97240 POINT(40.192914573944144 -73.24784627512248) bank97240 +97241 POINT(41.47885208734989 -73.3792380308488) bank97241 +97242 POINT(40.353643019845855 -74.38265416059605) bank97242 +97243 POINT(41.36259636906035 -73.37489703790963) bank97243 +97244 POINT(39.75679488065139 -74.63081980893631) bank97244 +97245 POINT(41.43131071400019 -74.19535260413241) bank97245 +97246 POINT(39.90872122767714 -74.92150095549444) bank97246 +97247 POINT(39.79398535138242 -74.76925453107145) bank97247 +97248 POINT(40.147543778147266 -73.28057842128821) bank97248 +97249 POINT(40.68899888912523 -73.00894944019295) bank97249 +97250 POINT(39.83790426210375 -73.49111538614797) bank97250 +97251 POINT(40.2548376566862 -74.61950540822048) bank97251 +97252 POINT(41.41960596159957 -74.4383564696153) bank97252 +97253 POINT(41.02320388298867 -73.94237262595594) bank97253 +97254 POINT(39.74605249550252 -73.47214496047303) bank97254 +97255 POINT(40.33648994710088 -73.44206686381298) bank97255 +97256 POINT(39.96058925190793 -73.17819442106672) bank97256 +97257 POINT(41.31776117992655 -73.47104937474303) bank97257 +97258 POINT(41.03702104748812 -73.47872651752554) bank97258 +97259 POINT(40.89857187829968 -74.18651439937895) bank97259 +97260 POINT(41.1419178906911 -73.68050512072067) bank97260 +97261 POINT(41.69180246656926 -73.68113658913987) bank97261 +97262 POINT(40.209952415773145 -73.89999104321443) bank97262 +97263 POINT(39.96299739167355 -74.41198254335647) bank97263 +97264 POINT(40.12310472365293 -74.53293391068415) bank97264 +97265 POINT(41.1124826165892 -73.08663968964086) bank97265 +97266 POINT(41.58019298745168 -74.59136240474716) bank97266 +97267 POINT(40.40750160197738 -74.3627484740672) bank97267 +97268 POINT(41.67602155864198 -73.90175917717367) bank97268 +97269 POINT(41.21221014190859 -74.10163155554555) bank97269 +97270 POINT(40.613100623064874 -74.0706064452514) bank97270 +97271 POINT(39.86587283411732 -74.78893877617399) bank97271 +97272 POINT(40.039762874215825 -73.34388102467774) bank97272 +97273 POINT(40.76020582986936 -73.72765918848874) bank97273 +97274 POINT(39.87323567205284 -74.24064703066355) bank97274 +97275 POINT(40.29903681372912 -73.32490938338807) bank97275 +97276 POINT(40.20393326985836 -73.6518976815075) bank97276 +97277 POINT(40.44244196199463 -74.59063663420606) bank97277 +97278 POINT(41.63086936739338 -74.08552542273942) bank97278 +97279 POINT(41.51347498639011 -74.67246378193914) bank97279 +97280 POINT(41.170607370730615 -73.75728705558312) bank97280 +97281 POINT(40.64468523694546 -74.71598565066499) bank97281 +97282 POINT(41.123302282340546 -74.37993698267788) bank97282 +97283 POINT(41.09965098487402 -73.13581028690595) bank97283 +97284 POINT(41.12099893920275 -74.97671149149069) bank97284 +97285 POINT(41.22992909759659 -73.94491271306684) bank97285 +97286 POINT(39.74955292406004 -74.32181461365894) bank97286 +97287 POINT(40.9981553321589 -73.77240183496457) bank97287 +97288 POINT(39.881156145865845 -74.06181205276067) bank97288 +97289 POINT(40.63583861967858 -74.6267432373798) bank97289 +97290 POINT(40.0860615272965 -74.27124171306006) bank97290 +97291 POINT(40.82151881168794 -73.15833291302104) bank97291 +97292 POINT(41.2324443112189 -74.76172793579859) bank97292 +97293 POINT(40.623348227802964 -74.44871800829918) bank97293 +97294 POINT(39.967552454753324 -74.83543205378342) bank97294 +97295 POINT(41.25429640275307 -73.28116028346433) bank97295 +97296 POINT(40.33391437145584 -73.75494488184557) bank97296 +97297 POINT(39.84827718917809 -74.0928678213305) bank97297 +97298 POINT(41.39725224831469 -73.49377542669755) bank97298 +97299 POINT(41.56113622020602 -74.63451586471685) bank97299 +97300 POINT(41.31193759418622 -73.45108287436352) bank97300 +97301 POINT(41.038291918502395 -73.82110640589904) bank97301 +97302 POINT(40.52770356190853 -74.24597415408702) bank97302 +97303 POINT(40.870694230654344 -73.43662961472995) bank97303 +97304 POINT(40.25758006012511 -73.99511189277317) bank97304 +97305 POINT(40.957614897849226 -74.37907140235373) bank97305 +97306 POINT(40.15030524484752 -73.38235440903304) bank97306 +97307 POINT(41.05622174686027 -74.91154604740031) bank97307 +97308 POINT(41.50555517068611 -73.21185106398256) bank97308 +97309 POINT(40.868847856812636 -73.23408439210175) bank97309 +97310 POINT(40.943785834383306 -73.32202410951984) bank97310 +97311 POINT(40.378907104089414 -73.47052198434932) bank97311 +97312 POINT(40.04627046986235 -74.22896473749694) bank97312 +97313 POINT(41.22240037541942 -74.64820050260853) bank97313 +97314 POINT(41.69326992092701 -74.79888988861858) bank97314 +97315 POINT(40.03551690241984 -74.27596072504967) bank97315 +97316 POINT(40.99559683911679 -74.53128230635059) bank97316 +97317 POINT(40.24289038889903 -73.4146091310594) bank97317 +97318 POINT(40.94221827347649 -74.05395588724419) bank97318 +97319 POINT(40.55169476762819 -74.74665179612578) bank97319 +97320 POINT(41.70395176749951 -73.32044084457061) bank97320 +97321 POINT(41.47461416471199 -73.10176302316528) bank97321 +97322 POINT(40.943453286593 -74.07511105348873) bank97322 +97323 POINT(40.5806149348423 -73.19805331947815) bank97323 +97324 POINT(41.582518131965266 -73.99195921351664) bank97324 +97325 POINT(41.40856386530615 -74.04835957204276) bank97325 +97326 POINT(41.04282077229212 -73.92748448742206) bank97326 +97327 POINT(40.39851504721443 -73.47699168349547) bank97327 +97328 POINT(41.343068527843336 -74.19577062807848) bank97328 +97329 POINT(41.40849742431287 -73.7582925763346) bank97329 +97330 POINT(41.364763181042754 -73.34996349952398) bank97330 +97331 POINT(40.175494428402054 -74.55273104412348) bank97331 +97332 POINT(40.3113245023521 -74.15537274449082) bank97332 +97333 POINT(41.43080869306814 -73.80413732297251) bank97333 +97334 POINT(40.2841500286658 -73.26774119185664) bank97334 +97335 POINT(40.363182670667975 -74.1312933914084) bank97335 +97336 POINT(40.13647020839232 -74.3381858132716) bank97336 +97337 POINT(41.20817270012504 -73.37874303382888) bank97337 +97338 POINT(40.38816053923893 -73.08015533573764) bank97338 +97339 POINT(41.39558385769778 -74.59705028952982) bank97339 +97340 POINT(40.66544221254033 -74.11060417189009) bank97340 +97341 POINT(41.274438821154135 -73.9732672015313) bank97341 +97342 POINT(41.064808958166836 -73.4046279283731) bank97342 +97343 POINT(41.426564248430466 -74.88166807809091) bank97343 +97344 POINT(41.44566059841649 -74.21492252540058) bank97344 +97345 POINT(40.863794664059505 -74.5305574793983) bank97345 +97346 POINT(41.64810298917106 -73.79262759576716) bank97346 +97347 POINT(41.21712890913594 -74.61142388652996) bank97347 +97348 POINT(39.9742948961163 -74.26840239714903) bank97348 +97349 POINT(40.60581496015787 -74.60499555835109) bank97349 +97350 POINT(41.11587187236311 -73.09145662990078) bank97350 +97351 POINT(40.915755495030496 -73.89185390658562) bank97351 +97352 POINT(40.99671086835586 -74.5402563907647) bank97352 +97353 POINT(40.7314174810613 -74.3826508895317) bank97353 +97354 POINT(41.21230471316021 -73.31507734905381) bank97354 +97355 POINT(39.83779681665741 -73.85829938869615) bank97355 +97356 POINT(41.65771721661024 -74.05268912579913) bank97356 +97357 POINT(40.51262285554984 -74.52028954617305) bank97357 +97358 POINT(40.249962449450415 -74.36467557405567) bank97358 +97359 POINT(39.84026194434789 -73.21542803330532) bank97359 +97360 POINT(41.48505082457066 -75.0029493360596) bank97360 +97361 POINT(41.189352804058906 -74.37148229169071) bank97361 +97362 POINT(39.84410248186011 -73.1144489493956) bank97362 +97363 POINT(40.74280641007495 -73.48820408590484) bank97363 +97364 POINT(40.315513573858084 -74.25672220067317) bank97364 +97365 POINT(40.64583640775258 -73.50197093744133) bank97365 +97366 POINT(40.274757694731704 -73.94015825698159) bank97366 +97367 POINT(40.50016720983954 -73.26911576219177) bank97367 +97368 POINT(39.8175026485707 -74.33030643089091) bank97368 +97369 POINT(40.37610638292718 -73.52015549271196) bank97369 +97370 POINT(41.63515147144608 -74.07983819567764) bank97370 +97371 POINT(39.82715151427376 -74.31008906893015) bank97371 +97372 POINT(41.5446892084894 -74.05111136371636) bank97372 +97373 POINT(39.78476609862745 -73.84650270814048) bank97373 +97374 POINT(40.10149861423096 -73.53217105510043) bank97374 +97375 POINT(40.167719629026735 -74.21366116446502) bank97375 +97376 POINT(41.13311144119108 -73.4374397957086) bank97376 +97377 POINT(41.60775688583268 -73.55902340000314) bank97377 +97378 POINT(41.33588133545264 -74.85708090674221) bank97378 +97379 POINT(41.228400881560674 -73.73056149621438) bank97379 +97380 POINT(40.84708351184084 -74.73962245345831) bank97380 +97381 POINT(40.376959375410955 -73.75342480665623) bank97381 +97382 POINT(39.99489546868747 -74.78724975730997) bank97382 +97383 POINT(41.05181845879868 -74.24495655351231) bank97383 +97384 POINT(40.10534326357933 -73.47829068289957) bank97384 +97385 POINT(39.757046840585886 -74.66933559912586) bank97385 +97386 POINT(40.430186119584945 -74.60821293815631) bank97386 +97387 POINT(41.040626975344594 -73.6500644067535) bank97387 +97388 POINT(40.07105408731922 -73.4670739405995) bank97388 +97389 POINT(41.261786640259785 -73.66660866375118) bank97389 +97390 POINT(40.78384466384003 -74.22699649533601) bank97390 +97391 POINT(40.51661797701318 -74.1786438227779) bank97391 +97392 POINT(39.84766171560068 -74.77510940647728) bank97392 +97393 POINT(40.767180935945085 -73.6464611268848) bank97393 +97394 POINT(41.30907527813997 -74.84229225753165) bank97394 +97395 POINT(40.54829786460306 -73.57499204182716) bank97395 +97396 POINT(40.78797574729582 -74.62487747961733) bank97396 +97397 POINT(41.40804642947152 -74.04558938911649) bank97397 +97398 POINT(39.80670916473802 -74.47262918927255) bank97398 +97399 POINT(39.714740735218435 -73.73035171850738) bank97399 +97400 POINT(40.593498349628184 -73.72092254470803) bank97400 +97401 POINT(41.327378730705355 -74.40194637199886) bank97401 +97402 POINT(39.78070515423478 -74.52229066641974) bank97402 +97403 POINT(40.05705663009586 -74.75094795922762) bank97403 +97404 POINT(40.20349052259758 -73.60017481246767) bank97404 +97405 POINT(40.131464942704866 -73.57652910996525) bank97405 +97406 POINT(40.715440797598525 -74.71788411669696) bank97406 +97407 POINT(40.4297731699094 -74.03827552062616) bank97407 +97408 POINT(39.9629158235651 -73.988567110743) bank97408 +97409 POINT(41.013355617373755 -73.62732543417611) bank97409 +97410 POINT(40.625075439848345 -73.43891060614865) bank97410 +97411 POINT(41.60039337098366 -74.38234885130304) bank97411 +97412 POINT(41.05641604744022 -74.23251381419023) bank97412 +97413 POINT(40.73623263217847 -73.1514746779646) bank97413 +97414 POINT(40.97136965807276 -73.60456172772584) bank97414 +97415 POINT(40.52746945614831 -73.44877135590139) bank97415 +97416 POINT(41.39145459623235 -74.85648154673656) bank97416 +97417 POINT(41.23147668832865 -74.32537500796069) bank97417 +97418 POINT(41.073243842595055 -73.79179238235331) bank97418 +97419 POINT(40.75172319153555 -73.30844576896618) bank97419 +97420 POINT(41.05284056229367 -74.3111518542918) bank97420 +97421 POINT(39.7320522568726 -74.71463770651255) bank97421 +97422 POINT(40.748643150296154 -74.62002204685731) bank97422 +97423 POINT(40.40580314441202 -74.15161873228189) bank97423 +97424 POINT(41.099930842970195 -74.08460542122616) bank97424 +97425 POINT(40.73910059765789 -74.52237168329961) bank97425 +97426 POINT(40.49792790939327 -74.43003980757143) bank97426 +97427 POINT(41.637787892838936 -73.20408394891616) bank97427 +97428 POINT(41.35045791382051 -74.23055628321465) bank97428 +97429 POINT(40.87246529466322 -74.0928419360998) bank97429 +97430 POINT(41.366111636073335 -74.52185519301213) bank97430 +97431 POINT(41.41559650044997 -73.50906853013768) bank97431 +97432 POINT(39.742021288713396 -73.07046892683401) bank97432 +97433 POINT(40.86463951748697 -74.90925800365186) bank97433 +97434 POINT(40.37096237832772 -74.56962918804578) bank97434 +97435 POINT(41.046198932015685 -74.98398653014607) bank97435 +97436 POINT(39.842832681475706 -74.97672783504922) bank97436 +97437 POINT(40.05953679663261 -74.06834287623875) bank97437 +97438 POINT(40.09020627316623 -74.68565833474112) bank97438 +97439 POINT(40.09928110424545 -74.79318602518268) bank97439 +97440 POINT(39.85701632836633 -74.90159923801998) bank97440 +97441 POINT(40.611805520780706 -74.91032870456331) bank97441 +97442 POINT(40.15439881746616 -73.26534096954406) bank97442 +97443 POINT(39.895743577278566 -73.6407535951657) bank97443 +97444 POINT(40.236524948945736 -74.22101924759778) bank97444 +97445 POINT(40.453608316356195 -73.76855471386231) bank97445 +97446 POINT(40.57623102057587 -74.84827849247985) bank97446 +97447 POINT(40.13757304204681 -73.88168084686752) bank97447 +97448 POINT(40.39637795543116 -73.77340862226984) bank97448 +97449 POINT(40.65581371140157 -74.14215505273171) bank97449 +97450 POINT(40.30694918862035 -73.28988621536884) bank97450 +97451 POINT(41.36034667525854 -73.11930018733777) bank97451 +97452 POINT(40.98911161444239 -73.94562537750775) bank97452 +97453 POINT(41.069795307240696 -74.25431701809671) bank97453 +97454 POINT(39.927112999977574 -74.76061188348149) bank97454 +97455 POINT(39.759774064883736 -74.2543479081953) bank97455 +97456 POINT(40.20952626778375 -74.34956408841255) bank97456 +97457 POINT(41.19666508405655 -73.13195158328867) bank97457 +97458 POINT(41.25760809004963 -74.75005336857501) bank97458 +97459 POINT(41.533509583447675 -73.71825925887012) bank97459 +97460 POINT(40.403255993282514 -73.5575355529179) bank97460 +97461 POINT(40.8905650651278 -73.04777669641024) bank97461 +97462 POINT(40.04005190702205 -74.8352251500925) bank97462 +97463 POINT(41.072447089917276 -74.77375540280526) bank97463 +97464 POINT(40.92246714852907 -73.00676573300962) bank97464 +97465 POINT(40.340019880260016 -74.68024249441392) bank97465 +97466 POINT(41.16108940366987 -73.84829190781005) bank97466 +97467 POINT(40.83871972364384 -73.69877160407056) bank97467 +97468 POINT(40.85535951597879 -74.81881165352874) bank97468 +97469 POINT(40.04220838745155 -73.93512433108204) bank97469 +97470 POINT(39.97028570302793 -74.06920694538677) bank97470 +97471 POINT(41.63351744193541 -74.22777012461961) bank97471 +97472 POINT(41.30963326961476 -73.60469548229906) bank97472 +97473 POINT(40.91708593577272 -74.01419442070068) bank97473 +97474 POINT(41.181396854317455 -73.57417846208018) bank97474 +97475 POINT(40.62034569513771 -74.52688801010915) bank97475 +97476 POINT(40.83465493980508 -73.58154026483054) bank97476 +97477 POINT(40.2583194783509 -74.98937791968434) bank97477 +97478 POINT(41.16080911834444 -74.98273112689046) bank97478 +97479 POINT(39.83566895231223 -73.07208535127481) bank97479 +97480 POINT(40.30308458346635 -74.1251265247376) bank97480 +97481 POINT(39.765615452347106 -74.91540826253613) bank97481 +97482 POINT(39.942502054773335 -74.87496358145765) bank97482 +97483 POINT(40.52057449064654 -73.52444654862389) bank97483 +97484 POINT(39.90680511293921 -74.90970842963996) bank97484 +97485 POINT(41.66163572620173 -74.11599662724966) bank97485 +97486 POINT(40.283729180431344 -73.74351753451678) bank97486 +97487 POINT(39.96778452861513 -73.31412837450368) bank97487 +97488 POINT(39.918128605868674 -74.2652794639445) bank97488 +97489 POINT(40.94039344434758 -73.21791489994159) bank97489 +97490 POINT(40.81313437863435 -74.67745565504637) bank97490 +97491 POINT(40.06848378755005 -74.94656104084545) bank97491 +97492 POINT(41.26127633274698 -74.33833498520119) bank97492 +97493 POINT(41.41069151019995 -73.608717998887) bank97493 +97494 POINT(40.24514624534302 -74.71865028569265) bank97494 +97495 POINT(40.19757175087185 -73.88980859625974) bank97495 +97496 POINT(40.14789516809363 -73.73831410995497) bank97496 +97497 POINT(40.84584975927169 -73.21490651510125) bank97497 +97498 POINT(39.80159665042661 -73.08934165934049) bank97498 +97499 POINT(39.8364164204281 -73.14355005789864) bank97499 +97500 POINT(40.48370032391499 -74.62997420078928) bank97500 +97501 POINT(41.003763347665 -74.99969922145327) bank97501 +97502 POINT(41.0121043859951 -73.42907533127033) bank97502 +97503 POINT(40.260616927750156 -74.84731651426593) bank97503 +97504 POINT(40.35839288476282 -74.25288351597194) bank97504 +97505 POINT(39.80978893250996 -75.00335443145572) bank97505 +97506 POINT(41.308737868221804 -74.8702235874174) bank97506 +97507 POINT(41.431203789198136 -74.26432475233305) bank97507 +97508 POINT(41.08814855312857 -73.95772157830258) bank97508 +97509 POINT(40.69016163433515 -74.5380505408242) bank97509 +97510 POINT(39.95627041491239 -74.62318144232387) bank97510 +97511 POINT(39.821236143050996 -74.28282115718645) bank97511 +97512 POINT(39.919760269310125 -73.1530699066002) bank97512 +97513 POINT(41.52764281614887 -74.24426280959916) bank97513 +97514 POINT(41.46931271544684 -73.31439036939393) bank97514 +97515 POINT(40.813163431519364 -74.28180999611547) bank97515 +97516 POINT(41.067071859738824 -74.16459402163174) bank97516 +97517 POINT(40.60152227877545 -74.05196641923904) bank97517 +97518 POINT(40.60155031819746 -74.93496258444625) bank97518 +97519 POINT(40.063253986575646 -73.5189614893747) bank97519 +97520 POINT(40.24182741367982 -73.08476290646757) bank97520 +97521 POINT(40.28037409612374 -73.85785299393336) bank97521 +97522 POINT(40.600305932062064 -73.05612640751558) bank97522 +97523 POINT(39.98937517420701 -74.94420479293785) bank97523 +97524 POINT(41.158821151836996 -74.97164304231747) bank97524 +97525 POINT(40.069906716873874 -74.3215169568614) bank97525 +97526 POINT(40.16825219156515 -74.38364121114638) bank97526 +97527 POINT(41.124370963791534 -73.53009519361267) bank97527 +97528 POINT(39.83206958818111 -73.91462946758433) bank97528 +97529 POINT(41.17581160122552 -73.3485331854559) bank97529 +97530 POINT(40.78982762617311 -74.4928017298014) bank97530 +97531 POINT(40.48856159184416 -74.7879961450091) bank97531 +97532 POINT(41.41465860859857 -73.29141998224294) bank97532 +97533 POINT(41.157924674587385 -74.30077822914144) bank97533 +97534 POINT(41.01293423417786 -73.84055248864782) bank97534 +97535 POINT(39.72044770993509 -74.57485246198831) bank97535 +97536 POINT(39.800782481806394 -74.4318582717033) bank97536 +97537 POINT(41.70484810592261 -73.83540072262404) bank97537 +97538 POINT(39.917307825667415 -74.32782264391001) bank97538 +97539 POINT(40.96757051686166 -74.8317843860172) bank97539 +97540 POINT(41.68109371486445 -73.59431363872972) bank97540 +97541 POINT(40.73653657164486 -73.81766003510364) bank97541 +97542 POINT(40.44526964693983 -74.10966135881772) bank97542 +97543 POINT(40.2182225813237 -74.2238700977927) bank97543 +97544 POINT(40.49694849827979 -73.04037066181627) bank97544 +97545 POINT(40.057201249265155 -73.70520756623226) bank97545 +97546 POINT(40.009632769622314 -74.88571309057161) bank97546 +97547 POINT(40.31634004427455 -74.03242643776281) bank97547 +97548 POINT(41.25712364780249 -73.05480353397829) bank97548 +97549 POINT(41.61966211351861 -74.7937099217158) bank97549 +97550 POINT(40.86789044270737 -73.7446643954938) bank97550 +97551 POINT(41.626770753758855 -73.82281083419679) bank97551 +97552 POINT(41.33890786662923 -74.28364053028395) bank97552 +97553 POINT(41.18616456455449 -73.02247683032054) bank97553 +97554 POINT(39.815887607443344 -73.59196520588651) bank97554 +97555 POINT(40.67631774507077 -73.61637317454603) bank97555 +97556 POINT(40.22579200955636 -74.03861789279573) bank97556 +97557 POINT(40.27455455797154 -73.40650089601624) bank97557 +97558 POINT(39.882648262101725 -73.23211556849641) bank97558 +97559 POINT(41.54678855377414 -73.58684933956529) bank97559 +97560 POINT(41.395393580456705 -74.99100291318946) bank97560 +97561 POINT(41.70883821231971 -73.01450479328034) bank97561 +97562 POINT(41.542279703405875 -74.04714623936526) bank97562 +97563 POINT(40.69932968065257 -73.86464682002925) bank97563 +97564 POINT(41.238111563202914 -74.14554329668694) bank97564 +97565 POINT(40.07034293835434 -73.89251269734686) bank97565 +97566 POINT(40.95722397595703 -74.06086580676012) bank97566 +97567 POINT(39.98125904522359 -73.0880373583379) bank97567 +97568 POINT(41.510030906323614 -73.3979039090231) bank97568 +97569 POINT(39.91998811211993 -74.41579780484533) bank97569 +97570 POINT(39.95284811950736 -74.60471003511535) bank97570 +97571 POINT(39.816912467229635 -73.63102786403539) bank97571 +97572 POINT(40.68856138939906 -74.94610051759247) bank97572 +97573 POINT(40.47657387301958 -73.28338128797989) bank97573 +97574 POINT(41.07914026007777 -74.7290166199497) bank97574 +97575 POINT(39.79712866937026 -73.54065591277464) bank97575 +97576 POINT(41.02367556817328 -74.18604056637052) bank97576 +97577 POINT(40.70935328783047 -73.88047367397355) bank97577 +97578 POINT(40.60064475507186 -74.50707481294313) bank97578 +97579 POINT(40.81120523568585 -74.93766165272375) bank97579 +97580 POINT(40.896879511913966 -74.57816066467193) bank97580 +97581 POINT(40.578008132857185 -73.37082457221122) bank97581 +97582 POINT(41.105852768866825 -74.79591917609889) bank97582 +97583 POINT(41.61623004635839 -74.41936066320402) bank97583 +97584 POINT(40.41282851113347 -73.27648429595672) bank97584 +97585 POINT(40.757798806874206 -74.34242126516321) bank97585 +97586 POINT(40.8712923311486 -74.27723078269159) bank97586 +97587 POINT(40.936279484709736 -73.66100856163371) bank97587 +97588 POINT(39.749674843369995 -73.36826091459181) bank97588 +97589 POINT(41.04313866449476 -73.33225645788427) bank97589 +97590 POINT(41.24941207152182 -74.08402779382594) bank97590 +97591 POINT(41.16728562054186 -74.99666995396872) bank97591 +97592 POINT(39.73313597688994 -73.01465725230936) bank97592 +97593 POINT(40.96369566736503 -74.41834824723587) bank97593 +97594 POINT(40.213031135655726 -74.59000867712876) bank97594 +97595 POINT(41.45796192044994 -73.22353046440938) bank97595 +97596 POINT(41.404949856011186 -74.3554271752362) bank97596 +97597 POINT(39.721331412732454 -73.95889591108583) bank97597 +97598 POINT(40.73567838182586 -73.29004915654102) bank97598 +97599 POINT(40.466984055507645 -73.71863174663946) bank97599 +97600 POINT(40.14859785262884 -73.88172801735722) bank97600 +97601 POINT(41.22092229467147 -74.28177514577969) bank97601 +97602 POINT(41.58144208026725 -73.70150215175686) bank97602 +97603 POINT(39.814220066707804 -73.1744125680668) bank97603 +97604 POINT(39.80409950750293 -73.03736437384165) bank97604 +97605 POINT(39.72221242730878 -73.323382039753) bank97605 +97606 POINT(41.51429694357738 -73.34169637338229) bank97606 +97607 POINT(41.6517531134285 -74.52192085412665) bank97607 +97608 POINT(39.79776960523355 -73.20956828105251) bank97608 +97609 POINT(41.649091450641066 -73.89551633778878) bank97609 +97610 POINT(40.972845754474925 -74.14743377805739) bank97610 +97611 POINT(40.75868926489419 -74.42511144305806) bank97611 +97612 POINT(41.530189975357345 -74.25441928013655) bank97612 +97613 POINT(40.58525455672898 -73.68739627391012) bank97613 +97614 POINT(40.021994797902906 -73.13743435533583) bank97614 +97615 POINT(41.49034314292168 -74.63794583602802) bank97615 +97616 POINT(40.63177274736197 -73.5823800536509) bank97616 +97617 POINT(39.91372393694865 -74.99565179783457) bank97617 +97618 POINT(40.17425513641398 -74.08595954415007) bank97618 +97619 POINT(40.95214231269258 -74.17711719205055) bank97619 +97620 POINT(40.239669840602105 -74.18705305690136) bank97620 +97621 POINT(41.236414407729086 -74.7060754198805) bank97621 +97622 POINT(40.64288965526769 -74.64505950683346) bank97622 +97623 POINT(41.126687559495856 -73.70219358445271) bank97623 +97624 POINT(40.426830060028124 -73.14925326501722) bank97624 +97625 POINT(41.44278045262872 -73.30833552156403) bank97625 +97626 POINT(39.9959137138587 -74.70501663951191) bank97626 +97627 POINT(40.61042743853398 -73.57338230810642) bank97627 +97628 POINT(41.5187010600582 -73.84180103531662) bank97628 +97629 POINT(40.20664944549617 -73.97636936803777) bank97629 +97630 POINT(39.88616556780997 -74.38442382537546) bank97630 +97631 POINT(39.832925039396564 -73.12457143866001) bank97631 +97632 POINT(40.39281708297894 -74.972980723731) bank97632 +97633 POINT(41.218180165425274 -74.99663267922476) bank97633 +97634 POINT(41.35108301949924 -74.5693951360633) bank97634 +97635 POINT(40.14002864128417 -73.44076758996142) bank97635 +97636 POINT(39.78983749546241 -73.2623394493977) bank97636 +97637 POINT(40.30770505573336 -74.84879276716437) bank97637 +97638 POINT(40.98918321721084 -73.22246930753424) bank97638 +97639 POINT(40.15931978857817 -74.4326443882732) bank97639 +97640 POINT(41.20578784385156 -73.76011317556491) bank97640 +97641 POINT(39.75853697450917 -74.8965179350393) bank97641 +97642 POINT(40.40123795116361 -73.13498946951475) bank97642 +97643 POINT(40.75851380001979 -73.90162112117356) bank97643 +97644 POINT(41.45760627090658 -73.79042517696755) bank97644 +97645 POINT(40.71195850676527 -74.41368058598717) bank97645 +97646 POINT(40.414481190320714 -73.45766233965963) bank97646 +97647 POINT(41.627078536599676 -73.84888245253309) bank97647 +97648 POINT(40.701813544140805 -74.9195476692445) bank97648 +97649 POINT(40.037582058642435 -73.74155051519007) bank97649 +97650 POINT(39.963705156902826 -73.50803143185665) bank97650 +97651 POINT(40.33552932803511 -74.22268910736418) bank97651 +97652 POINT(41.432723433013145 -74.9858865795213) bank97652 +97653 POINT(39.933262701476345 -74.04480305524662) bank97653 +97654 POINT(40.84263190116848 -74.19987713545437) bank97654 +97655 POINT(40.487252196684864 -74.23965643989237) bank97655 +97656 POINT(41.70553262757699 -74.7763832276886) bank97656 +97657 POINT(39.855464888431484 -73.93900332669168) bank97657 +97658 POINT(41.68711530831794 -73.01229234676944) bank97658 +97659 POINT(41.036051331578165 -74.60246790586528) bank97659 +97660 POINT(40.819309121787335 -73.38433651987536) bank97660 +97661 POINT(41.413992531053275 -74.54688918053246) bank97661 +97662 POINT(40.15374848857455 -73.57650728062372) bank97662 +97663 POINT(40.49747629929246 -73.43005558078542) bank97663 +97664 POINT(40.1508819684757 -74.80658807728265) bank97664 +97665 POINT(41.45224152817251 -74.00982926301502) bank97665 +97666 POINT(40.32703729446558 -74.31368699720257) bank97666 +97667 POINT(40.04326957305306 -74.33150282530389) bank97667 +97668 POINT(40.78658638254398 -73.23660469752096) bank97668 +97669 POINT(40.92467417645514 -73.48308422983621) bank97669 +97670 POINT(40.95858855541667 -74.44508055842324) bank97670 +97671 POINT(40.15730692530852 -73.3711740091304) bank97671 +97672 POINT(40.634877907937835 -74.68667210950497) bank97672 +97673 POINT(40.31493243622963 -74.88569322888567) bank97673 +97674 POINT(40.6043618098212 -73.398039864425) bank97674 +97675 POINT(40.73480438379766 -74.75229542086632) bank97675 +97676 POINT(40.09895844744166 -74.17149043516751) bank97676 +97677 POINT(39.8690331382224 -73.49264987390193) bank97677 +97678 POINT(41.524122747257245 -74.87512131200218) bank97678 +97679 POINT(41.70243989526489 -73.46903564616501) bank97679 +97680 POINT(40.422340301607434 -74.14422081875216) bank97680 +97681 POINT(40.577698238980076 -73.71832519912422) bank97681 +97682 POINT(40.23392980724165 -74.11558905651954) bank97682 +97683 POINT(41.54193864817256 -74.23691938467304) bank97683 +97684 POINT(41.63974071042874 -73.47551637038755) bank97684 +97685 POINT(40.603598984835024 -73.44696418079023) bank97685 +97686 POINT(40.91226245359424 -74.16965987207352) bank97686 +97687 POINT(41.47839473458334 -73.45601892558795) bank97687 +97688 POINT(40.08714661009919 -74.79433795623564) bank97688 +97689 POINT(39.807020663955576 -74.04414308519843) bank97689 +97690 POINT(40.56882630338391 -74.65814531560075) bank97690 +97691 POINT(41.63096831807062 -74.11883254504994) bank97691 +97692 POINT(40.91794118057722 -74.49129266470435) bank97692 +97693 POINT(40.04996093935237 -74.45448183202038) bank97693 +97694 POINT(40.726686110626076 -73.78700762043304) bank97694 +97695 POINT(40.74597484556359 -74.81428723386232) bank97695 +97696 POINT(41.326445892416416 -74.60454654368567) bank97696 +97697 POINT(41.68652197216241 -73.87106116770559) bank97697 +97698 POINT(41.11360197551182 -73.93754902678702) bank97698 +97699 POINT(40.67442305481904 -73.21791516768161) bank97699 +97700 POINT(39.99087597677728 -74.24704231432615) bank97700 +97701 POINT(40.31113770389556 -74.32971539041783) bank97701 +97702 POINT(40.011492491837956 -73.24287924154332) bank97702 +97703 POINT(41.39586739222666 -74.58300957513978) bank97703 +97704 POINT(41.53279189960217 -74.44037879071315) bank97704 +97705 POINT(41.198694388952156 -74.4875389250649) bank97705 +97706 POINT(40.04714896149844 -74.03141837778418) bank97706 +97707 POINT(41.394221557355515 -74.82110756303689) bank97707 +97708 POINT(40.72581538059348 -73.26542078833332) bank97708 +97709 POINT(39.722364548857115 -73.96097773748295) bank97709 +97710 POINT(39.867491764174424 -73.3625359570054) bank97710 +97711 POINT(40.00956548997706 -73.15210223248538) bank97711 +97712 POINT(40.69674442287337 -74.14511630470271) bank97712 +97713 POINT(40.63516103596463 -74.21558717502171) bank97713 +97714 POINT(40.09109159556912 -74.67481686555527) bank97714 +97715 POINT(39.812793074351056 -74.37025087123533) bank97715 +97716 POINT(40.18224066691108 -74.1786933293445) bank97716 +97717 POINT(40.91416296948376 -74.02716814881487) bank97717 +97718 POINT(41.22519453905556 -73.22006746324327) bank97718 +97719 POINT(39.87971263439034 -74.5256996802313) bank97719 +97720 POINT(40.44492605226834 -74.95213498055648) bank97720 +97721 POINT(40.918280117503514 -74.38602466041579) bank97721 +97722 POINT(40.395228899201406 -74.5426135460748) bank97722 +97723 POINT(41.705994447751024 -74.82149910971445) bank97723 +97724 POINT(41.13709350034243 -74.18542139349525) bank97724 +97725 POINT(41.07939760341072 -73.84147044237055) bank97725 +97726 POINT(40.528039998972105 -73.6429654343658) bank97726 +97727 POINT(39.91148437604401 -73.26768733485667) bank97727 +97728 POINT(40.17313528192918 -74.85521177455668) bank97728 +97729 POINT(40.33154416392178 -73.28395940842792) bank97729 +97730 POINT(41.57056912025582 -73.45022827857333) bank97730 +97731 POINT(41.07337947609361 -73.77150962345769) bank97731 +97732 POINT(41.21176742533166 -73.66286967866064) bank97732 +97733 POINT(41.67596215751112 -73.08441504667081) bank97733 +97734 POINT(40.69941774256718 -73.01577981733483) bank97734 +97735 POINT(40.92704333304282 -73.15532455890542) bank97735 +97736 POINT(39.72342971616214 -73.43272143400128) bank97736 +97737 POINT(40.37352076956898 -74.05311542194526) bank97737 +97738 POINT(40.53348943238692 -74.1950477612434) bank97738 +97739 POINT(40.49352230561385 -74.60595418921139) bank97739 +97740 POINT(40.471556838776294 -74.69988461012856) bank97740 +97741 POINT(40.642098012161505 -74.18209028669868) bank97741 +97742 POINT(40.178928689292114 -73.64958141481316) bank97742 +97743 POINT(40.60784309186861 -74.00544089681425) bank97743 +97744 POINT(41.643494458306584 -74.67359905942614) bank97744 +97745 POINT(40.94569324763467 -74.71775400495466) bank97745 +97746 POINT(39.93399381754568 -73.61885475480183) bank97746 +97747 POINT(41.635575327120094 -73.72425422903643) bank97747 +97748 POINT(41.22302379111478 -73.72308979795967) bank97748 +97749 POINT(40.61883384679976 -73.32084843533684) bank97749 +97750 POINT(41.286202274579665 -74.67044213401162) bank97750 +97751 POINT(41.35982306136235 -73.12671299233719) bank97751 +97752 POINT(40.060312657707705 -73.4292369895722) bank97752 +97753 POINT(41.50573757669989 -74.61853986529718) bank97753 +97754 POINT(41.237277837886154 -74.8162989463045) bank97754 +97755 POINT(41.60838044752647 -73.42700295684722) bank97755 +97756 POINT(40.306021931351175 -73.60020699768906) bank97756 +97757 POINT(41.476614333893515 -74.81833932829363) bank97757 +97758 POINT(39.88276308633825 -74.56483456596234) bank97758 +97759 POINT(39.80606898506281 -73.73023379839991) bank97759 +97760 POINT(41.17755365293815 -73.1501909910109) bank97760 +97761 POINT(39.780977419969716 -74.4125933316325) bank97761 +97762 POINT(40.8928117233657 -74.43584661952681) bank97762 +97763 POINT(40.693149526494445 -73.40068331139715) bank97763 +97764 POINT(40.7453592286285 -73.46129191931664) bank97764 +97765 POINT(40.93488273085648 -73.32970489404039) bank97765 +97766 POINT(41.06447983536348 -73.84777571483497) bank97766 +97767 POINT(40.5369600398443 -73.27210813044347) bank97767 +97768 POINT(39.752308727024754 -73.14932199305122) bank97768 +97769 POINT(40.076712293657934 -73.92343275775211) bank97769 +97770 POINT(40.42336834595127 -74.21332806280776) bank97770 +97771 POINT(39.95738628332681 -74.29710644731767) bank97771 +97772 POINT(39.74384090016689 -73.69981508002851) bank97772 +97773 POINT(40.05958279729677 -73.77464274227319) bank97773 +97774 POINT(40.001959684700466 -73.99301139161358) bank97774 +97775 POINT(40.386243857084054 -74.08955429306161) bank97775 +97776 POINT(40.593751007108 -73.12634469119804) bank97776 +97777 POINT(40.343543024432584 -73.88488927210466) bank97777 +97778 POINT(41.06258013545252 -73.1934379578402) bank97778 +97779 POINT(40.3683894139835 -73.95012996947487) bank97779 +97780 POINT(41.529724833521186 -74.59068357149623) bank97780 +97781 POINT(41.61899781855534 -73.18856251729078) bank97781 +97782 POINT(41.56132367115762 -73.12913452930601) bank97782 +97783 POINT(40.07651773371735 -74.59867856244941) bank97783 +97784 POINT(40.19440139854978 -73.96123339771475) bank97784 +97785 POINT(41.112723001000795 -74.37059184641024) bank97785 +97786 POINT(41.19473073332423 -74.67642892192002) bank97786 +97787 POINT(39.72855632399409 -73.21597055282342) bank97787 +97788 POINT(41.242603962918814 -73.97786291179877) bank97788 +97789 POINT(40.908478221389906 -73.28326378688874) bank97789 +97790 POINT(40.56034034408662 -74.91765202821135) bank97790 +97791 POINT(40.50974830468821 -73.08830854951461) bank97791 +97792 POINT(39.822196413252485 -74.91782568281184) bank97792 +97793 POINT(41.1972080947831 -73.63295603895489) bank97793 +97794 POINT(41.059855580382326 -74.70920348949802) bank97794 +97795 POINT(41.541466522723745 -74.68726712725545) bank97795 +97796 POINT(40.28249080396304 -73.10347679347367) bank97796 +97797 POINT(40.191210347314936 -74.35150133248392) bank97797 +97798 POINT(41.32740084589461 -74.76275233539941) bank97798 +97799 POINT(39.94382014685852 -74.63559706145561) bank97799 +97800 POINT(40.0404326148429 -73.44365420670506) bank97800 +97801 POINT(40.28342006609493 -74.09589930392434) bank97801 +97802 POINT(40.87075626768299 -73.74678238363704) bank97802 +97803 POINT(40.44967094474056 -74.14183190029408) bank97803 +97804 POINT(40.91416714762806 -74.44904289412497) bank97804 +97805 POINT(41.41054820488635 -74.65145919080736) bank97805 +97806 POINT(41.05567659453292 -74.98323254823983) bank97806 +97807 POINT(39.7321683254724 -73.37352178749218) bank97807 +97808 POINT(40.90883260865901 -73.37442365004225) bank97808 +97809 POINT(41.43632851405165 -73.7897819767581) bank97809 +97810 POINT(41.672180165749594 -74.87937093582389) bank97810 +97811 POINT(40.875492827094064 -73.59036972939852) bank97811 +97812 POINT(39.96236684399182 -73.88484466093063) bank97812 +97813 POINT(41.37454781974922 -73.99620506773228) bank97813 +97814 POINT(41.33559601748213 -74.27867557849255) bank97814 +97815 POINT(39.98423281021734 -73.80652648096597) bank97815 +97816 POINT(41.01426587086004 -74.6928144105551) bank97816 +97817 POINT(40.75512956954682 -73.566691296792) bank97817 +97818 POINT(39.836831901275914 -74.60827027158307) bank97818 +97819 POINT(40.020910574784054 -74.5395316635153) bank97819 +97820 POINT(40.267911306700135 -74.05980317119894) bank97820 +97821 POINT(41.35102318942553 -74.78057182311983) bank97821 +97822 POINT(41.705934557892085 -74.63975214645356) bank97822 +97823 POINT(40.284688186749925 -74.87085776614275) bank97823 +97824 POINT(40.85457645838859 -74.41522301624812) bank97824 +97825 POINT(40.51459029764765 -74.08047220540406) bank97825 +97826 POINT(41.26270951934053 -74.25908677715843) bank97826 +97827 POINT(41.62127296211891 -73.57969043454753) bank97827 +97828 POINT(40.75992058054728 -73.143000081995) bank97828 +97829 POINT(40.36563989593341 -73.32272875438491) bank97829 +97830 POINT(41.06175096249373 -73.98136057106976) bank97830 +97831 POINT(39.87819378567793 -73.326513149678) bank97831 +97832 POINT(40.266803760112644 -74.71713084987375) bank97832 +97833 POINT(40.71505222349239 -73.3294067683811) bank97833 +97834 POINT(40.750494296282604 -73.64281603916898) bank97834 +97835 POINT(41.02264825623782 -73.85136064183682) bank97835 +97836 POINT(40.40873378984269 -73.06116501131292) bank97836 +97837 POINT(41.57631384948247 -73.96629734332478) bank97837 +97838 POINT(39.84526008535352 -74.83197027441358) bank97838 +97839 POINT(40.49868719379928 -74.30786088742488) bank97839 +97840 POINT(40.55652909104145 -74.08882126546317) bank97840 +97841 POINT(39.72627716185126 -74.67488965788155) bank97841 +97842 POINT(40.42153275611169 -73.14531281288008) bank97842 +97843 POINT(40.068482614955016 -73.30383626079109) bank97843 +97844 POINT(40.33367240827604 -74.342780343259) bank97844 +97845 POINT(41.302656665990554 -73.32645649085082) bank97845 +97846 POINT(41.674101676817955 -74.64831214236568) bank97846 +97847 POINT(41.61616695389042 -74.1167209398475) bank97847 +97848 POINT(41.223856245593005 -74.16327316382952) bank97848 +97849 POINT(40.51733630746444 -73.4376131106486) bank97849 +97850 POINT(40.38050664166259 -73.10364740810451) bank97850 +97851 POINT(39.913878417863984 -73.70966671751981) bank97851 +97852 POINT(41.07430993599153 -73.90962882654064) bank97852 +97853 POINT(40.77521399822096 -74.56235706975207) bank97853 +97854 POINT(41.12353295456269 -74.20054950562017) bank97854 +97855 POINT(40.74167213386364 -74.91151199381257) bank97855 +97856 POINT(40.66596378777367 -74.30136107238036) bank97856 +97857 POINT(41.487260150866454 -73.19870974119206) bank97857 +97858 POINT(41.33928041525364 -74.34720236563332) bank97858 +97859 POINT(40.094335073047446 -74.0973237479543) bank97859 +97860 POINT(40.15238775917711 -74.88622620043911) bank97860 +97861 POINT(40.33304830035034 -73.54998174757198) bank97861 +97862 POINT(40.203829611346634 -74.41936505349915) bank97862 +97863 POINT(40.6236205930788 -74.90921993441883) bank97863 +97864 POINT(41.66095762435786 -74.81605792501553) bank97864 +97865 POINT(40.02427069022078 -73.16227947687726) bank97865 +97866 POINT(39.86708889461719 -74.01394429119675) bank97866 +97867 POINT(40.58654937743066 -73.54633750970353) bank97867 +97868 POINT(40.00877834771017 -73.78943162054739) bank97868 +97869 POINT(41.611284375946276 -73.94743923310655) bank97869 +97870 POINT(40.93518605411747 -73.21621363258345) bank97870 +97871 POINT(40.78866224578788 -74.92624644253932) bank97871 +97872 POINT(40.36522250014392 -74.80844702528857) bank97872 +97873 POINT(41.54898280532168 -73.68373344459118) bank97873 +97874 POINT(41.51876171332759 -73.74933696371984) bank97874 +97875 POINT(41.55258705615008 -73.5026469775) bank97875 +97876 POINT(40.06471711496846 -73.69132369730872) bank97876 +97877 POINT(41.38878385214093 -74.71770005708923) bank97877 +97878 POINT(40.20494202045063 -74.26968214261535) bank97878 +97879 POINT(40.31719455697584 -73.21497047874742) bank97879 +97880 POINT(40.09178388610551 -73.09052781012507) bank97880 +97881 POINT(40.82300841375033 -73.54540513284721) bank97881 +97882 POINT(40.405720814861276 -74.83482529317867) bank97882 +97883 POINT(40.26044455588013 -74.51139844902033) bank97883 +97884 POINT(40.899052461964956 -73.78671892842448) bank97884 +97885 POINT(41.56690641869494 -74.4980983957542) bank97885 +97886 POINT(40.74424995346832 -74.5116328235105) bank97886 +97887 POINT(41.52135900992731 -73.32677830129259) bank97887 +97888 POINT(40.96558592158729 -73.41048564083172) bank97888 +97889 POINT(39.935441574912794 -74.37244867573334) bank97889 +97890 POINT(40.55361504057027 -73.16090225415545) bank97890 +97891 POINT(40.40556410871182 -74.69476578691238) bank97891 +97892 POINT(39.72369244761803 -73.24757053229541) bank97892 +97893 POINT(40.27363695918304 -73.65401726443804) bank97893 +97894 POINT(41.33259913012037 -73.0859810559823) bank97894 +97895 POINT(39.797761640751006 -74.87672979725144) bank97895 +97896 POINT(41.448566118746974 -74.73370731786626) bank97896 +97897 POINT(41.28009121147199 -74.47385590232268) bank97897 +97898 POINT(39.96918910554735 -74.25159077350702) bank97898 +97899 POINT(40.513600387598686 -73.10170528771158) bank97899 +97900 POINT(40.95701252346861 -74.71543646602555) bank97900 +97901 POINT(41.07643962422945 -73.73471723321535) bank97901 +97902 POINT(41.38969988195507 -73.67997148224099) bank97902 +97903 POINT(39.84340274703688 -73.74196939922587) bank97903 +97904 POINT(40.118393802185366 -73.03015409223117) bank97904 +97905 POINT(41.52672314004661 -73.48565326879543) bank97905 +97906 POINT(41.08887416063209 -73.31869698819132) bank97906 +97907 POINT(41.02734003807937 -73.09097544408363) bank97907 +97908 POINT(41.71085323245243 -74.89014281266913) bank97908 +97909 POINT(40.77096394706328 -74.3917251953789) bank97909 +97910 POINT(41.32409430824218 -73.62843022557762) bank97910 +97911 POINT(40.580395381997974 -73.10653561797726) bank97911 +97912 POINT(39.82810940061777 -75.00467137083113) bank97912 +97913 POINT(40.466397140472395 -74.52291682724271) bank97913 +97914 POINT(39.886634284074724 -74.38975701090254) bank97914 +97915 POINT(40.45104220557989 -73.70374435694347) bank97915 +97916 POINT(41.47600988868785 -74.10733859966803) bank97916 +97917 POINT(41.29177775114051 -74.60056575585637) bank97917 +97918 POINT(40.424506057050145 -74.8768002756171) bank97918 +97919 POINT(40.027154770944676 -73.87818485535004) bank97919 +97920 POINT(40.206974119141776 -74.22156901362416) bank97920 +97921 POINT(41.38036760246877 -74.51653634370845) bank97921 +97922 POINT(40.03389980492153 -74.85946219252486) bank97922 +97923 POINT(40.19927661459185 -73.64402718218864) bank97923 +97924 POINT(40.47654431206437 -73.73726377891296) bank97924 +97925 POINT(40.477851576919754 -73.61862309290822) bank97925 +97926 POINT(41.70163168539549 -74.40642235794814) bank97926 +97927 POINT(41.3255290935219 -74.20541494677276) bank97927 +97928 POINT(41.61668856066057 -73.94564031419971) bank97928 +97929 POINT(41.51214694058144 -73.85628748950403) bank97929 +97930 POINT(40.50555842921221 -74.0839854141469) bank97930 +97931 POINT(40.55079141472644 -74.64439399327287) bank97931 +97932 POINT(40.45030334263169 -73.34914745749572) bank97932 +97933 POINT(40.098391721809456 -74.46046394685973) bank97933 +97934 POINT(40.868832404797296 -73.39777334526323) bank97934 +97935 POINT(39.938411424339435 -74.17828723798014) bank97935 +97936 POINT(40.39059672846822 -73.67439756685165) bank97936 +97937 POINT(40.277844258570184 -74.62137027347343) bank97937 +97938 POINT(41.299097379025454 -74.97712073379495) bank97938 +97939 POINT(40.07993295156745 -74.92973103133228) bank97939 +97940 POINT(40.900961878799386 -74.34817120289262) bank97940 +97941 POINT(40.13448644178958 -73.64020427147118) bank97941 +97942 POINT(40.14080510222823 -74.44652087074394) bank97942 +97943 POINT(40.31616201702866 -74.94660836384735) bank97943 +97944 POINT(40.26175707548671 -73.68088158796506) bank97944 +97945 POINT(40.032370520289724 -74.26688544169727) bank97945 +97946 POINT(41.28008311757648 -73.95361333734787) bank97946 +97947 POINT(41.63356624937871 -74.40228457027382) bank97947 +97948 POINT(39.87811961370027 -73.25083094480962) bank97948 +97949 POINT(39.76951756606303 -74.2132020081904) bank97949 +97950 POINT(41.09222352389616 -73.53966402158562) bank97950 +97951 POINT(41.258609508824605 -74.53275915258946) bank97951 +97952 POINT(40.665552411892875 -74.56429546647266) bank97952 +97953 POINT(41.52875227294639 -74.11623987360625) bank97953 +97954 POINT(41.25054824637047 -74.62750726580325) bank97954 +97955 POINT(40.1819470103748 -74.6104235796749) bank97955 +97956 POINT(39.79228160005683 -74.0648188738324) bank97956 +97957 POINT(39.89313277777118 -73.90475310591158) bank97957 +97958 POINT(39.87423373270695 -73.20101709935467) bank97958 +97959 POINT(40.73678727043012 -74.36403246762481) bank97959 +97960 POINT(39.76117100438829 -73.97280997858167) bank97960 +97961 POINT(40.03587948648317 -74.64227577863272) bank97961 +97962 POINT(41.42211294745475 -73.36671565141266) bank97962 +97963 POINT(40.481702612858875 -74.30477592573192) bank97963 +97964 POINT(41.62919978164633 -74.94693344211105) bank97964 +97965 POINT(40.99198536111114 -74.36944660324802) bank97965 +97966 POINT(40.71196347967555 -74.38797938045317) bank97966 +97967 POINT(41.269503995060276 -74.10462124273057) bank97967 +97968 POINT(40.39795091508243 -73.61674158752406) bank97968 +97969 POINT(41.55509645948229 -74.9025277125351) bank97969 +97970 POINT(40.10422593423751 -73.92962673830087) bank97970 +97971 POINT(39.87494821196098 -74.31810376401957) bank97971 +97972 POINT(40.56064803214921 -73.46405635579694) bank97972 +97973 POINT(40.7427582868579 -73.03161630792214) bank97973 +97974 POINT(40.126698157417884 -74.6251372880145) bank97974 +97975 POINT(40.31413565753931 -74.16475917356293) bank97975 +97976 POINT(41.021721060971096 -73.22019108668546) bank97976 +97977 POINT(40.63496314969364 -74.8128071465316) bank97977 +97978 POINT(41.320965255840555 -74.51915376467878) bank97978 +97979 POINT(39.91386618765193 -74.04187009428618) bank97979 +97980 POINT(40.27846801263979 -74.06774723342353) bank97980 +97981 POINT(41.22556753165831 -73.80987050849468) bank97981 +97982 POINT(40.170331373064634 -74.62933595552269) bank97982 +97983 POINT(41.30582749090004 -74.1001240816261) bank97983 +97984 POINT(41.21785140186791 -73.39480441204398) bank97984 +97985 POINT(40.41064615959203 -73.78286882941823) bank97985 +97986 POINT(40.7015317390172 -74.52612084875722) bank97986 +97987 POINT(40.3412090126506 -73.66676753815543) bank97987 +97988 POINT(39.79273706323317 -73.38224389980925) bank97988 +97989 POINT(40.07305767895972 -74.82409620656303) bank97989 +97990 POINT(41.50865077006834 -73.27046231833359) bank97990 +97991 POINT(39.94591655103912 -74.17535646628227) bank97991 +97992 POINT(41.02394736061423 -73.34989913127008) bank97992 +97993 POINT(40.519310639471065 -74.48233243306979) bank97993 +97994 POINT(40.9812362752843 -73.06965913436414) bank97994 +97995 POINT(39.74092742245312 -74.80568722427414) bank97995 +97996 POINT(41.59986502196386 -73.57396607782128) bank97996 +97997 POINT(41.520317635910395 -74.08719763863436) bank97997 +97998 POINT(39.74963563422921 -73.33805140663807) bank97998 +97999 POINT(40.03755557241371 -74.93135815619196) bank97999 +98000 POINT(39.99486862069876 -74.91494567751772) bank98000 +98001 POINT(40.04180497723683 -74.1898171853689) bank98001 +98002 POINT(40.200502959817676 -74.37261103456073) bank98002 +98003 POINT(41.059181555204255 -73.21777965819712) bank98003 +98004 POINT(41.12274264479541 -73.60750615918846) bank98004 +98005 POINT(40.74583148785263 -74.11047795528266) bank98005 +98006 POINT(40.10905453754056 -74.73601853279278) bank98006 +98007 POINT(40.349181728137445 -73.57494935279597) bank98007 +98008 POINT(40.30269651320389 -73.52652735167563) bank98008 +98009 POINT(40.46871406910287 -74.9715387108825) bank98009 +98010 POINT(41.519175003774585 -74.09925282564478) bank98010 +98011 POINT(40.0877661486511 -74.81205346222521) bank98011 +98012 POINT(40.452798713502794 -73.15363558241239) bank98012 +98013 POINT(40.85983035039434 -74.95311962136888) bank98013 +98014 POINT(40.041751111835325 -74.57674282052007) bank98014 +98015 POINT(41.11931219765341 -73.76541500777941) bank98015 +98016 POINT(40.426231862752836 -74.15672877190714) bank98016 +98017 POINT(41.05035943993147 -73.02901274291492) bank98017 +98018 POINT(41.30157653767899 -74.90140312147967) bank98018 +98019 POINT(40.47619702967285 -73.5837454026982) bank98019 +98020 POINT(40.56665059855274 -74.76345621587105) bank98020 +98021 POINT(40.73009834684393 -74.0707531277959) bank98021 +98022 POINT(40.673461641631704 -73.92732119054867) bank98022 +98023 POINT(41.17174745887205 -73.67174851229889) bank98023 +98024 POINT(41.359619942497446 -74.66855336112256) bank98024 +98025 POINT(40.11955589667524 -74.95809952426738) bank98025 +98026 POINT(41.1799986637607 -73.57631847850584) bank98026 +98027 POINT(40.17648267212244 -74.5309092719089) bank98027 +98028 POINT(41.515854807531774 -74.37112838756286) bank98028 +98029 POINT(40.39055471528034 -74.84711170043455) bank98029 +98030 POINT(41.33839264023008 -73.28577855512714) bank98030 +98031 POINT(40.53519555208328 -73.59849696317688) bank98031 +98032 POINT(40.02434705516208 -73.38904471165317) bank98032 +98033 POINT(39.87151613629293 -74.76463105103902) bank98033 +98034 POINT(40.90426911713036 -74.14415243325186) bank98034 +98035 POINT(40.23978071739119 -74.43847786274317) bank98035 +98036 POINT(41.186672542201755 -74.81400058720077) bank98036 +98037 POINT(39.763773270007356 -73.5804769984558) bank98037 +98038 POINT(40.98915504503622 -73.99088252351964) bank98038 +98039 POINT(40.606887339316614 -73.76245187118644) bank98039 +98040 POINT(40.77359976877702 -74.05373451865617) bank98040 +98041 POINT(40.31359901427357 -74.14865581057877) bank98041 +98042 POINT(40.37540739451051 -73.40555824731035) bank98042 +98043 POINT(40.68110871514622 -73.67906683527951) bank98043 +98044 POINT(40.24681571307805 -74.77288387724475) bank98044 +98045 POINT(40.80361933759157 -73.51690375970225) bank98045 +98046 POINT(39.94215405063557 -74.19938700219654) bank98046 +98047 POINT(40.098962621807296 -74.83457763490091) bank98047 +98048 POINT(41.69020261820945 -74.4288105845003) bank98048 +98049 POINT(39.763259043186466 -74.88020431373778) bank98049 +98050 POINT(40.066546670412315 -74.25875421666751) bank98050 +98051 POINT(40.028052650775095 -74.0256651090269) bank98051 +98052 POINT(40.02159712435154 -74.06137588972895) bank98052 +98053 POINT(39.839051665992 -73.81035992058037) bank98053 +98054 POINT(40.00039576303357 -74.51706307090868) bank98054 +98055 POINT(41.60474151922806 -73.13442544835864) bank98055 +98056 POINT(39.95465538988938 -73.47631370252954) bank98056 +98057 POINT(39.878949489145 -74.79048642127174) bank98057 +98058 POINT(40.41398455959703 -74.76594704555443) bank98058 +98059 POINT(40.78312474324149 -74.22234322494867) bank98059 +98060 POINT(39.80153735722485 -74.23032196516648) bank98060 +98061 POINT(41.08872049100012 -73.02478773843825) bank98061 +98062 POINT(41.658212184504094 -73.30900729259277) bank98062 +98063 POINT(40.11613356418627 -73.38797511712583) bank98063 +98064 POINT(39.716938940718315 -74.1041511716061) bank98064 +98065 POINT(41.32039308901957 -74.18695807965342) bank98065 +98066 POINT(40.09545492117652 -73.81165398467824) bank98066 +98067 POINT(40.14668419363049 -73.40115458002458) bank98067 +98068 POINT(41.580478246721334 -73.98897063948489) bank98068 +98069 POINT(40.38700142261911 -74.55171784694097) bank98069 +98070 POINT(40.50050790427808 -73.86681608580345) bank98070 +98071 POINT(40.28997936394518 -73.77630404984527) bank98071 +98072 POINT(40.73911377528452 -74.16671666170251) bank98072 +98073 POINT(40.50413104594496 -73.09986936899568) bank98073 +98074 POINT(39.89832678774579 -73.6629892143708) bank98074 +98075 POINT(40.16440288419015 -73.6813343598525) bank98075 +98076 POINT(41.42417661141592 -73.59191288678005) bank98076 +98077 POINT(39.714557486606395 -73.99468645750603) bank98077 +98078 POINT(40.638742332853994 -74.42720699063815) bank98078 +98079 POINT(40.90557874345945 -73.92351599997465) bank98079 +98080 POINT(40.23998844330406 -73.91985033042184) bank98080 +98081 POINT(40.968871920100796 -73.68914139699501) bank98081 +98082 POINT(40.104708416490226 -73.43581143424262) bank98082 +98083 POINT(41.17079089425515 -73.97718211845266) bank98083 +98084 POINT(41.33289776510032 -74.1658280820295) bank98084 +98085 POINT(41.527645497483455 -73.22427009830731) bank98085 +98086 POINT(40.450270470392375 -74.50126179835935) bank98086 +98087 POINT(40.671851889202244 -74.80694694829805) bank98087 +98088 POINT(39.893395014961115 -74.44191934253845) bank98088 +98089 POINT(39.826316954749004 -73.13856717683642) bank98089 +98090 POINT(41.47591883298185 -74.54736067578122) bank98090 +98091 POINT(40.777949700504735 -73.83629974122613) bank98091 +98092 POINT(41.08689215169724 -74.10697332297363) bank98092 +98093 POINT(41.61635881353995 -73.5926195151519) bank98093 +98094 POINT(41.38257139045584 -73.14760038629662) bank98094 +98095 POINT(40.90360361172523 -73.21618776631685) bank98095 +98096 POINT(40.06484770235083 -73.15296846943026) bank98096 +98097 POINT(40.5440657545683 -74.97408415611675) bank98097 +98098 POINT(39.96634210405836 -73.18323437302926) bank98098 +98099 POINT(39.88802570636139 -74.38451110840946) bank98099 +98100 POINT(40.40836686248195 -74.25796798968113) bank98100 +98101 POINT(41.49748341912694 -73.43163356646684) bank98101 +98102 POINT(39.83981308874877 -73.20363901847097) bank98102 +98103 POINT(40.9004155661716 -74.6589269510523) bank98103 +98104 POINT(39.74850227593653 -73.07184385042335) bank98104 +98105 POINT(41.51227921752332 -74.0404902324661) bank98105 +98106 POINT(40.0679551581261 -74.87565000664593) bank98106 +98107 POINT(40.72143510606677 -74.13849694130289) bank98107 +98108 POINT(41.135724216028244 -73.98518160176648) bank98108 +98109 POINT(39.72151246765238 -73.85067776945066) bank98109 +98110 POINT(40.53484066094114 -73.61163896125811) bank98110 +98111 POINT(40.20274716179887 -74.2196793401479) bank98111 +98112 POINT(40.813855639964856 -73.46866225308023) bank98112 +98113 POINT(40.61323029012674 -73.32544212832607) bank98113 +98114 POINT(40.79630393825629 -74.03078880732001) bank98114 +98115 POINT(40.02967585276192 -74.5192970113923) bank98115 +98116 POINT(40.163213734242 -74.78633791373187) bank98116 +98117 POINT(40.17478835982045 -74.04816369930003) bank98117 +98118 POINT(41.55848059460455 -73.44941272448175) bank98118 +98119 POINT(39.92731334456282 -74.67466223020531) bank98119 +98120 POINT(41.64055575931239 -73.49950818994311) bank98120 +98121 POINT(40.89871749068915 -73.41354226575689) bank98121 +98122 POINT(41.44501801268355 -74.88510093358187) bank98122 +98123 POINT(39.82615457352942 -73.3247962447427) bank98123 +98124 POINT(40.89921611615708 -73.9103559198053) bank98124 +98125 POINT(41.52156831511521 -74.67345019341423) bank98125 +98126 POINT(40.89662157354205 -74.6408398133955) bank98126 +98127 POINT(40.71273215675043 -74.3083218600507) bank98127 +98128 POINT(40.37895187076967 -74.69081520186495) bank98128 +98129 POINT(41.4945217990645 -74.59972812306941) bank98129 +98130 POINT(39.85606669298506 -74.78098617174862) bank98130 +98131 POINT(40.03985148160276 -73.48055933147701) bank98131 +98132 POINT(40.96083624998695 -74.48759951241495) bank98132 +98133 POINT(41.43346708087614 -74.69200491404403) bank98133 +98134 POINT(40.11540295353626 -74.75518021810944) bank98134 +98135 POINT(40.5547638314876 -74.43094007401446) bank98135 +98136 POINT(39.969075539476584 -73.53426618337429) bank98136 +98137 POINT(40.64166494134771 -73.05310496432983) bank98137 +98138 POINT(41.23169750266129 -74.27579317375195) bank98138 +98139 POINT(40.69543621020266 -73.78527715205155) bank98139 +98140 POINT(40.85279800650168 -74.4418931298765) bank98140 +98141 POINT(40.39333553934875 -73.79207203934328) bank98141 +98142 POINT(41.05606212389223 -74.38914723543374) bank98142 +98143 POINT(40.80191171480698 -74.86540631995263) bank98143 +98144 POINT(40.80501784035385 -74.50036449097715) bank98144 +98145 POINT(40.59688200227188 -73.21970913564532) bank98145 +98146 POINT(40.22826236361419 -74.32117427674193) bank98146 +98147 POINT(40.990857084490436 -74.58545481061215) bank98147 +98148 POINT(41.58222765697452 -74.37581304578545) bank98148 +98149 POINT(41.59457848670629 -73.50129772210393) bank98149 +98150 POINT(41.061078796352085 -73.6648565443082) bank98150 +98151 POINT(41.09651234637939 -74.55588929112048) bank98151 +98152 POINT(39.85700969563936 -74.55840314358569) bank98152 +98153 POINT(41.31010313639009 -73.64091940900886) bank98153 +98154 POINT(40.46977228430705 -73.02461559388624) bank98154 +98155 POINT(40.11848061792559 -74.3712608272508) bank98155 +98156 POINT(41.63789492844792 -74.59929837618103) bank98156 +98157 POINT(40.99422208228118 -74.84067584890205) bank98157 +98158 POINT(40.174732480124014 -73.12243862613894) bank98158 +98159 POINT(41.4889253211693 -73.68010060422907) bank98159 +98160 POINT(41.17058422985309 -74.3304444461556) bank98160 +98161 POINT(40.737965959197965 -74.31776796850785) bank98161 +98162 POINT(40.78579733928798 -73.02249512891885) bank98162 +98163 POINT(40.61152006945745 -73.94201683692393) bank98163 +98164 POINT(40.29321520522432 -74.47649879113575) bank98164 +98165 POINT(39.98410422770871 -74.47283479971489) bank98165 +98166 POINT(41.454556162961616 -74.03825519814521) bank98166 +98167 POINT(41.708092578449524 -74.19787380856046) bank98167 +98168 POINT(40.58552094188764 -73.99472003809656) bank98168 +98169 POINT(40.23142743762456 -73.8682567235244) bank98169 +98170 POINT(40.196919973346 -73.98181234843479) bank98170 +98171 POINT(41.69655363835658 -74.38406131320042) bank98171 +98172 POINT(41.35052674851634 -73.65311865847401) bank98172 +98173 POINT(40.23317723734294 -73.58838607149669) bank98173 +98174 POINT(40.19030401671455 -73.35349984755246) bank98174 +98175 POINT(41.01964220354897 -73.03965887403982) bank98175 +98176 POINT(39.89860348062546 -74.54961870733199) bank98176 +98177 POINT(40.334347331708244 -74.54213607509757) bank98177 +98178 POINT(40.25353037440613 -74.9687636706893) bank98178 +98179 POINT(40.12056241672025 -74.76033571148734) bank98179 +98180 POINT(40.75148815587415 -74.73978964651765) bank98180 +98181 POINT(41.503708447828906 -73.95503293290847) bank98181 +98182 POINT(41.48392335631936 -73.25606127522136) bank98182 +98183 POINT(40.77102960761877 -73.39456172568512) bank98183 +98184 POINT(39.92485901243698 -74.03646935450209) bank98184 +98185 POINT(41.511075355561154 -74.96654943733782) bank98185 +98186 POINT(39.718834554292606 -73.47904637236564) bank98186 +98187 POINT(40.12369858417926 -74.123998603354) bank98187 +98188 POINT(40.63906102163084 -73.17884999938256) bank98188 +98189 POINT(39.90199598222454 -74.94427293386052) bank98189 +98190 POINT(40.33345220359238 -73.60792552255498) bank98190 +98191 POINT(40.8219368783826 -73.88462077083169) bank98191 +98192 POINT(39.782448580986106 -74.12155459193146) bank98192 +98193 POINT(40.22011863845017 -74.5264974469186) bank98193 +98194 POINT(41.70236924404483 -73.61493630446314) bank98194 +98195 POINT(40.71412969615761 -73.56074165647502) bank98195 +98196 POINT(41.38314894489503 -74.69751921869026) bank98196 +98197 POINT(40.314677912974815 -73.76908793322097) bank98197 +98198 POINT(41.05026047089793 -73.04132698575195) bank98198 +98199 POINT(41.59292901972451 -74.30421146093397) bank98199 +98200 POINT(40.436651076099956 -74.00748487494349) bank98200 +98201 POINT(40.78755032689587 -74.74785756944263) bank98201 +98202 POINT(40.71100258563435 -73.50136785050479) bank98202 +98203 POINT(40.77894520689348 -73.69711542264534) bank98203 +98204 POINT(39.768156302403746 -74.69432956322446) bank98204 +98205 POINT(40.797532123535504 -74.98198249316194) bank98205 +98206 POINT(41.68428571687036 -74.3138344652262) bank98206 +98207 POINT(40.45552740204053 -73.8597158753248) bank98207 +98208 POINT(41.42340227451261 -73.11733830147303) bank98208 +98209 POINT(41.48486501340476 -74.2029457943983) bank98209 +98210 POINT(39.786296652957134 -73.45321363553715) bank98210 +98211 POINT(41.33346127313691 -74.98970026573247) bank98211 +98212 POINT(40.59066866405708 -74.00452540321382) bank98212 +98213 POINT(41.32779205452497 -74.65621639667984) bank98213 +98214 POINT(40.03453295813955 -73.90202169081044) bank98214 +98215 POINT(41.183020920270515 -74.78024799311238) bank98215 +98216 POINT(41.61852315206341 -74.93794484988726) bank98216 +98217 POINT(40.949392825701196 -73.46669986488166) bank98217 +98218 POINT(41.13635725764159 -74.79313223774899) bank98218 +98219 POINT(40.6808542390018 -73.0362349038486) bank98219 +98220 POINT(41.43533096269686 -73.41895456341992) bank98220 +98221 POINT(41.639481797866466 -74.0888808616662) bank98221 +98222 POINT(40.144129877126424 -73.29718036186665) bank98222 +98223 POINT(40.56645534625154 -73.00799091191578) bank98223 +98224 POINT(40.31734270487631 -73.39719415762441) bank98224 +98225 POINT(41.42774063054603 -75.00534178785156) bank98225 +98226 POINT(39.94890028933869 -74.94523550344661) bank98226 +98227 POINT(40.84085393856295 -73.06644322687353) bank98227 +98228 POINT(40.63734980913373 -74.16691001476202) bank98228 +98229 POINT(40.0968818142821 -74.34693778298123) bank98229 +98230 POINT(41.24413899842783 -74.92308245430785) bank98230 +98231 POINT(39.96134866162726 -74.28787810206917) bank98231 +98232 POINT(40.491340959525346 -73.07107465919354) bank98232 +98233 POINT(39.79483001364519 -74.88676325464304) bank98233 +98234 POINT(40.501426884862134 -74.96529154806012) bank98234 +98235 POINT(41.53535501231387 -74.71605547574269) bank98235 +98236 POINT(40.917270620521585 -73.85132728501537) bank98236 +98237 POINT(40.78306828105805 -73.1617572732548) bank98237 +98238 POINT(41.5737540635998 -74.63613709297996) bank98238 +98239 POINT(41.12158993502306 -73.4069234579049) bank98239 +98240 POINT(40.138753628971436 -74.6650105374543) bank98240 +98241 POINT(41.022045021597975 -74.3733283510143) bank98241 +98242 POINT(40.62833641376621 -74.89617155751839) bank98242 +98243 POINT(39.966275419067756 -74.51469272182855) bank98243 +98244 POINT(39.83507620271484 -74.40524583349342) bank98244 +98245 POINT(40.90203577157875 -73.39568343857283) bank98245 +98246 POINT(40.3943783672245 -74.26893091947474) bank98246 +98247 POINT(40.37876139164039 -73.07399851132509) bank98247 +98248 POINT(40.88416451313538 -73.29209386257637) bank98248 +98249 POINT(40.89026801277119 -74.53183891911073) bank98249 +98250 POINT(40.58860731928221 -74.05058281261057) bank98250 +98251 POINT(39.94186592888312 -74.61236950084744) bank98251 +98252 POINT(40.64474698873935 -73.38515450076352) bank98252 +98253 POINT(41.48135941676574 -74.55669234762185) bank98253 +98254 POINT(41.196848220308105 -74.35008816935502) bank98254 +98255 POINT(39.965679505295505 -73.64675091915845) bank98255 +98256 POINT(41.71006705043785 -73.66953725466735) bank98256 +98257 POINT(41.14895935992121 -73.20271816646174) bank98257 +98258 POINT(40.83946310083027 -73.9789077331634) bank98258 +98259 POINT(40.50911516378108 -74.74079015363861) bank98259 +98260 POINT(41.323348529998036 -73.68343180014683) bank98260 +98261 POINT(40.5526130356948 -74.64384906605216) bank98261 +98262 POINT(41.60890963320018 -74.67689435385081) bank98262 +98263 POINT(40.55416115552702 -73.39044141721237) bank98263 +98264 POINT(40.927612980432 -74.19416230638208) bank98264 +98265 POINT(41.258672737656305 -74.82954456481302) bank98265 +98266 POINT(40.54242268859396 -73.68052591361929) bank98266 +98267 POINT(39.90620804032581 -74.68965619468312) bank98267 +98268 POINT(41.49783084254141 -74.00568209504972) bank98268 +98269 POINT(40.427913300711495 -73.56755573985953) bank98269 +98270 POINT(39.95020932156758 -74.66167710531869) bank98270 +98271 POINT(40.47575283012709 -74.08654886873657) bank98271 +98272 POINT(40.397089500749075 -74.55581216066332) bank98272 +98273 POINT(41.23590018477846 -74.15295273263354) bank98273 +98274 POINT(40.96728497839824 -74.36504556228259) bank98274 +98275 POINT(40.669460843561396 -73.13478323651616) bank98275 +98276 POINT(40.33689223749422 -73.07344530626582) bank98276 +98277 POINT(41.05039156512341 -74.9097175895801) bank98277 +98278 POINT(40.40223579119273 -74.21928465634473) bank98278 +98279 POINT(40.317979554170805 -74.25707807913666) bank98279 +98280 POINT(40.356113389622706 -73.76287414465075) bank98280 +98281 POINT(40.9425354946101 -74.54876197955475) bank98281 +98282 POINT(41.460976400576186 -73.67923254465032) bank98282 +98283 POINT(40.08811542782817 -73.16768604806339) bank98283 +98284 POINT(40.47644819057243 -73.26241535260502) bank98284 +98285 POINT(41.100233926411185 -74.15811194344076) bank98285 +98286 POINT(39.96412221946087 -73.75856546276876) bank98286 +98287 POINT(40.24609514528884 -74.13869375960677) bank98287 +98288 POINT(40.22614184652802 -73.58148173137879) bank98288 +98289 POINT(41.32786614382445 -74.43711696623777) bank98289 +98290 POINT(40.3955331940045 -74.28807732854347) bank98290 +98291 POINT(40.89044960768267 -74.25845382673025) bank98291 +98292 POINT(41.47831525417616 -74.67843002683692) bank98292 +98293 POINT(41.70469651751824 -74.47364176180176) bank98293 +98294 POINT(41.283766440737026 -74.56965848089676) bank98294 +98295 POINT(39.72616241890388 -74.90931542829941) bank98295 +98296 POINT(39.742523021613394 -74.52347994588169) bank98296 +98297 POINT(40.21375562262367 -73.87779049287585) bank98297 +98298 POINT(39.75668214395647 -74.45412823963026) bank98298 +98299 POINT(41.476974382145656 -73.54717216358978) bank98299 +98300 POINT(40.9927672013808 -73.12469798122662) bank98300 +98301 POINT(41.31595347987439 -74.24658629383099) bank98301 +98302 POINT(41.05894877197635 -74.88889154355084) bank98302 +98303 POINT(41.48614421217297 -74.47423272268972) bank98303 +98304 POINT(41.265438170362465 -73.20820927956294) bank98304 +98305 POINT(40.005398199000844 -74.5898027074935) bank98305 +98306 POINT(41.55615079139168 -73.15284693485964) bank98306 +98307 POINT(41.34513434154574 -74.99194317371719) bank98307 +98308 POINT(39.833386875186555 -73.57862739084871) bank98308 +98309 POINT(40.84830448400463 -74.13626206044503) bank98309 +98310 POINT(40.42477103374922 -74.14013370341806) bank98310 +98311 POINT(41.36534312416817 -73.42055506346894) bank98311 +98312 POINT(39.97090171740904 -74.55828915406104) bank98312 +98313 POINT(40.46346538580412 -74.6288500937853) bank98313 +98314 POINT(41.675935665748135 -74.85271404197042) bank98314 +98315 POINT(40.22705491066662 -73.85918774161583) bank98315 +98316 POINT(41.686314977530714 -73.50091895735275) bank98316 +98317 POINT(40.11798871980198 -73.940834910488) bank98317 +98318 POINT(40.87683214818291 -74.64677630688013) bank98318 +98319 POINT(41.370186806816974 -73.13084446907963) bank98319 +98320 POINT(40.06252032296152 -73.20464930974106) bank98320 +98321 POINT(40.072328910974655 -73.75144489957901) bank98321 +98322 POINT(41.63985242920126 -73.99788400931118) bank98322 +98323 POINT(39.870094958839076 -74.97117645596121) bank98323 +98324 POINT(41.01544109189322 -74.38823831222335) bank98324 +98325 POINT(39.72146649638675 -73.35152856303402) bank98325 +98326 POINT(40.1707858447806 -74.69220121370753) bank98326 +98327 POINT(39.74850536488561 -73.39754793711406) bank98327 +98328 POINT(41.295785905184836 -74.54853039462878) bank98328 +98329 POINT(40.148615673947674 -74.75553335738672) bank98329 +98330 POINT(40.77713627495054 -74.76906242777878) bank98330 +98331 POINT(40.12096918590291 -74.03343244814234) bank98331 +98332 POINT(40.99315959236832 -73.17386220464276) bank98332 +98333 POINT(41.41756889376071 -73.80736710104065) bank98333 +98334 POINT(40.97333619494457 -73.32118744336509) bank98334 +98335 POINT(40.899183586948794 -73.63941159370398) bank98335 +98336 POINT(40.434943360963196 -73.0377215021952) bank98336 +98337 POINT(41.1127010068408 -73.65281866813173) bank98337 +98338 POINT(41.08165774979739 -74.81775455172779) bank98338 +98339 POINT(40.62633419954918 -74.9905939863791) bank98339 +98340 POINT(41.333514204261434 -74.81605795413941) bank98340 +98341 POINT(41.2457046634439 -74.59685433362722) bank98341 +98342 POINT(40.72438525515383 -74.99952715787913) bank98342 +98343 POINT(41.26188813440098 -74.53681079469652) bank98343 +98344 POINT(40.69903965197105 -73.47150203592504) bank98344 +98345 POINT(41.639070278675774 -74.2223110188547) bank98345 +98346 POINT(41.37361144213831 -73.13820203869848) bank98346 +98347 POINT(40.10024564889243 -73.64891437519276) bank98347 +98348 POINT(41.28356338554197 -73.82342457088536) bank98348 +98349 POINT(40.53367163284508 -74.76018608607471) bank98349 +98350 POINT(39.981773579312005 -74.85403268766534) bank98350 +98351 POINT(40.22189973397856 -74.2526724823001) bank98351 +98352 POINT(40.20332653351065 -73.67564378794587) bank98352 +98353 POINT(40.0878435800964 -74.67615370489801) bank98353 +98354 POINT(39.85612114585328 -74.30208737144197) bank98354 +98355 POINT(40.841217564249135 -73.12630550821932) bank98355 +98356 POINT(40.29290853409577 -73.87263866618729) bank98356 +98357 POINT(40.68812853551604 -74.19571616918019) bank98357 +98358 POINT(41.078862301651455 -74.24928838914542) bank98358 +98359 POINT(40.84120624367833 -74.25573271868849) bank98359 +98360 POINT(40.78010598505133 -73.61273166260483) bank98360 +98361 POINT(40.41085875148263 -74.82459584245275) bank98361 +98362 POINT(40.035509414071335 -74.0081300565279) bank98362 +98363 POINT(39.9443595466406 -73.00636776525205) bank98363 +98364 POINT(40.104645920652615 -74.97363371852151) bank98364 +98365 POINT(41.65205378326647 -73.80970778168316) bank98365 +98366 POINT(40.54606553043462 -74.12922025678522) bank98366 +98367 POINT(41.0889892948752 -74.02191797540335) bank98367 +98368 POINT(40.58501530587895 -73.36934243551286) bank98368 +98369 POINT(40.5309019968566 -73.66790561711846) bank98369 +98370 POINT(40.263088191305954 -74.27336515979258) bank98370 +98371 POINT(40.47877255210083 -73.90241329621676) bank98371 +98372 POINT(39.78170611453633 -74.33354023156554) bank98372 +98373 POINT(41.07206881023686 -73.37288433478315) bank98373 +98374 POINT(40.05398866230573 -73.86947802569485) bank98374 +98375 POINT(41.01495061909385 -73.83652708239268) bank98375 +98376 POINT(40.51091437916675 -73.7246162709488) bank98376 +98377 POINT(40.11425635317439 -73.55936449361634) bank98377 +98378 POINT(40.632907367263186 -73.87780716889738) bank98378 +98379 POINT(40.77310456309184 -73.06827353968828) bank98379 +98380 POINT(40.890464161569575 -74.86167109315123) bank98380 +98381 POINT(40.6414683294912 -74.61689642530784) bank98381 +98382 POINT(40.424680564251474 -73.76557684370283) bank98382 +98383 POINT(41.517549052889144 -74.71146730735438) bank98383 +98384 POINT(40.83106325170715 -74.81724921733431) bank98384 +98385 POINT(40.38798742394688 -74.36192412119124) bank98385 +98386 POINT(40.76720792709827 -74.0748996112592) bank98386 +98387 POINT(41.371872469504915 -74.22970042212962) bank98387 +98388 POINT(39.92496976999125 -73.0464038235871) bank98388 +98389 POINT(40.51723717888927 -73.5859589653986) bank98389 +98390 POINT(40.51511287635069 -74.2272023909043) bank98390 +98391 POINT(39.71512135265224 -74.13615389062458) bank98391 +98392 POINT(40.01128999390458 -75.00212509654506) bank98392 +98393 POINT(40.90858495252966 -74.6821618772149) bank98393 +98394 POINT(40.44613479830248 -74.61316874764) bank98394 +98395 POINT(40.733091177643125 -74.13550000411408) bank98395 +98396 POINT(41.569473108910834 -74.06419882922016) bank98396 +98397 POINT(40.55642366706153 -74.33841314113623) bank98397 +98398 POINT(39.837029183987994 -74.96479293800252) bank98398 +98399 POINT(39.884361264119924 -73.70799972527597) bank98399 +98400 POINT(41.18983136442927 -74.64374241562685) bank98400 +98401 POINT(41.37039642034426 -74.3786187939631) bank98401 +98402 POINT(40.79163413810671 -73.19521555174994) bank98402 +98403 POINT(41.54075470222094 -74.78615850242598) bank98403 +98404 POINT(41.29083477003776 -74.34055974037963) bank98404 +98405 POINT(40.98532430363066 -74.14334286283793) bank98405 +98406 POINT(40.40085951519868 -74.45309621151286) bank98406 +98407 POINT(41.62854892265062 -74.53302344850037) bank98407 +98408 POINT(40.54670061172445 -74.53570969291681) bank98408 +98409 POINT(41.13716032545018 -74.084127260111) bank98409 +98410 POINT(40.781977486472556 -73.72805082852585) bank98410 +98411 POINT(39.888837962061814 -74.61575938752644) bank98411 +98412 POINT(41.455479767029196 -73.16451592581728) bank98412 +98413 POINT(40.503153173007654 -73.5904570583861) bank98413 +98414 POINT(41.09040090467314 -73.83472455088935) bank98414 +98415 POINT(40.227916041278036 -73.29080836910535) bank98415 +98416 POINT(41.25644194750447 -74.13236226573177) bank98416 +98417 POINT(41.444074177823616 -73.18687273494048) bank98417 +98418 POINT(40.965556940876134 -74.49753970284155) bank98418 +98419 POINT(40.7612567654054 -73.34721006787345) bank98419 +98420 POINT(40.42261594500421 -74.45077978517489) bank98420 +98421 POINT(39.83364646045703 -73.34474761936173) bank98421 +98422 POINT(39.938923689210974 -74.93102692762751) bank98422 +98423 POINT(41.63171350477809 -73.84142662645677) bank98423 +98424 POINT(41.052569345685384 -73.52917685140902) bank98424 +98425 POINT(40.20639430752956 -73.2439433077101) bank98425 +98426 POINT(39.98432887063997 -73.66823441296052) bank98426 +98427 POINT(41.23608848700123 -73.26896030869561) bank98427 +98428 POINT(39.93868173474041 -74.44277336804589) bank98428 +98429 POINT(40.7438973019349 -74.26175711921738) bank98429 +98430 POINT(40.11908170880634 -73.16121783949457) bank98430 +98431 POINT(41.383534350635585 -73.8594025863348) bank98431 +98432 POINT(40.90516114364578 -74.05427273967067) bank98432 +98433 POINT(41.44415154913007 -73.2936574648193) bank98433 +98434 POINT(40.07749737409824 -74.87645211587706) bank98434 +98435 POINT(39.82941682535566 -73.90871884100812) bank98435 +98436 POINT(41.01483648834797 -73.84220758120661) bank98436 +98437 POINT(41.240604700735005 -73.21403159258598) bank98437 +98438 POINT(41.0202053631719 -74.49978153778635) bank98438 +98439 POINT(41.27176240307772 -73.99212260045361) bank98439 +98440 POINT(39.749389844975994 -74.11898633578045) bank98440 +98441 POINT(40.65985978553484 -74.80921476789906) bank98441 +98442 POINT(40.655224567017676 -73.29101581676082) bank98442 +98443 POINT(40.47959068959087 -74.9480579893209) bank98443 +98444 POINT(40.40478817014138 -74.94945064865914) bank98444 +98445 POINT(40.76720807875667 -73.56858917486025) bank98445 +98446 POINT(40.69207644395303 -73.56219761387126) bank98446 +98447 POINT(41.03338629473987 -74.91911481332298) bank98447 +98448 POINT(40.053937304137825 -74.86244247698977) bank98448 +98449 POINT(40.41493031354507 -73.1213074479413) bank98449 +98450 POINT(41.21393775748752 -74.30861766164652) bank98450 +98451 POINT(41.3632496647017 -73.60527535513525) bank98451 +98452 POINT(40.06882088589124 -73.93208767269464) bank98452 +98453 POINT(41.33839106482988 -73.7045821962688) bank98453 +98454 POINT(41.56018893363776 -73.3576106213451) bank98454 +98455 POINT(40.127603962581794 -73.44319107984818) bank98455 +98456 POINT(40.31541352551718 -74.21440897707448) bank98456 +98457 POINT(40.6600477184267 -73.20499395426984) bank98457 +98458 POINT(41.58353141409952 -74.05322651808989) bank98458 +98459 POINT(41.0792965241006 -74.31690995645795) bank98459 +98460 POINT(40.09211839981861 -73.149059451537) bank98460 +98461 POINT(40.0669220005905 -74.30232119574804) bank98461 +98462 POINT(41.49239039694494 -74.03500545894009) bank98462 +98463 POINT(41.44567650903824 -73.07316554998042) bank98463 +98464 POINT(41.31387946064921 -74.87411293289435) bank98464 +98465 POINT(40.900011860783245 -73.28369504381897) bank98465 +98466 POINT(41.01653612731363 -74.38782212826933) bank98466 +98467 POINT(40.73182930263573 -74.47842315994706) bank98467 +98468 POINT(41.36085203432318 -74.49249756911448) bank98468 +98469 POINT(41.076435162422925 -74.03149571053376) bank98469 +98470 POINT(41.177331556243104 -74.4052992330844) bank98470 +98471 POINT(40.669721952414825 -74.61971802402807) bank98471 +98472 POINT(40.97404321612573 -74.28862167332238) bank98472 +98473 POINT(39.89291986212472 -73.87369729467395) bank98473 +98474 POINT(39.896497811040945 -74.15026440292316) bank98474 +98475 POINT(41.20348544600992 -74.27619917599218) bank98475 +98476 POINT(40.57374722812185 -73.48141569006155) bank98476 +98477 POINT(39.83168665942894 -73.12473574083734) bank98477 +98478 POINT(39.827237022056714 -74.61160611995797) bank98478 +98479 POINT(40.24518168582329 -73.9893799142506) bank98479 +98480 POINT(40.16834220747195 -74.5160684430243) bank98480 +98481 POINT(40.234590930181916 -73.43490686638447) bank98481 +98482 POINT(41.51054530733977 -74.91200963075822) bank98482 +98483 POINT(41.20211488291551 -73.51115276830348) bank98483 +98484 POINT(41.31498332903719 -74.62199644873733) bank98484 +98485 POINT(39.911816861445566 -73.47208843274124) bank98485 +98486 POINT(40.65818579319447 -73.08328821052535) bank98486 +98487 POINT(39.77002097160256 -73.78580958679969) bank98487 +98488 POINT(40.3090221642168 -74.49615655402648) bank98488 +98489 POINT(39.97187989962203 -74.7223092502786) bank98489 +98490 POINT(39.97552800367876 -73.08251619267597) bank98490 +98491 POINT(40.10720332428916 -74.90387340640491) bank98491 +98492 POINT(39.86194574981871 -74.00179241369267) bank98492 +98493 POINT(41.23200013236134 -74.85559111802992) bank98493 +98494 POINT(41.54609203125991 -74.36779259658026) bank98494 +98495 POINT(40.75872765728625 -73.45415138358369) bank98495 +98496 POINT(41.07802229849293 -73.02532574857045) bank98496 +98497 POINT(40.54606211706674 -74.24436215441624) bank98497 +98498 POINT(39.97339692029148 -74.07836912962368) bank98498 +98499 POINT(40.58147111621682 -73.34938410080892) bank98499 +98500 POINT(41.001777410506214 -74.16182485628518) bank98500 +98501 POINT(40.80993229630805 -73.66699824234175) bank98501 +98502 POINT(39.89141820288281 -73.86838193194104) bank98502 +98503 POINT(40.35943993893939 -73.61950059260796) bank98503 +98504 POINT(41.07256330349794 -73.069269358758) bank98504 +98505 POINT(40.63436386076859 -73.12867230437598) bank98505 +98506 POINT(40.99726986901911 -73.66208963948569) bank98506 +98507 POINT(40.22804660222273 -74.56174507324165) bank98507 +98508 POINT(40.64397118323917 -74.48486554710955) bank98508 +98509 POINT(40.70225024263534 -74.53743629187738) bank98509 +98510 POINT(40.26295533311743 -73.83040432982553) bank98510 +98511 POINT(40.55561825271899 -74.0901610593399) bank98511 +98512 POINT(41.243775045171176 -73.84914618067398) bank98512 +98513 POINT(41.191567184216495 -73.53260159879048) bank98513 +98514 POINT(39.868304535433204 -73.45379937411424) bank98514 +98515 POINT(39.73409262828798 -74.99688331765908) bank98515 +98516 POINT(41.54310370568377 -73.54235367778085) bank98516 +98517 POINT(40.734075021490575 -73.4186822758764) bank98517 +98518 POINT(40.35252765676565 -73.17134672711398) bank98518 +98519 POINT(40.63153987167805 -74.2439363466727) bank98519 +98520 POINT(40.07350228623224 -73.18373826220828) bank98520 +98521 POINT(41.646744882266276 -73.16051945609632) bank98521 +98522 POINT(40.405122168879885 -74.53110101747279) bank98522 +98523 POINT(40.24751184362093 -73.74624042985174) bank98523 +98524 POINT(41.51225623938187 -74.2925013029831) bank98524 +98525 POINT(41.47362365549799 -73.77514104044651) bank98525 +98526 POINT(41.66862052356041 -74.66376712714482) bank98526 +98527 POINT(40.55370182324883 -73.39769724257218) bank98527 +98528 POINT(39.765952146750536 -74.55300841190487) bank98528 +98529 POINT(40.9782896745224 -74.64388216768862) bank98529 +98530 POINT(40.94122085583783 -74.92066975882935) bank98530 +98531 POINT(40.88844130837538 -73.35017963516925) bank98531 +98532 POINT(40.20938138312389 -74.5017341498558) bank98532 +98533 POINT(41.6527614141391 -73.33236286241997) bank98533 +98534 POINT(40.47727625749654 -73.06316282264434) bank98534 +98535 POINT(41.508594659354046 -73.72234547236505) bank98535 +98536 POINT(41.544764143220654 -74.08256756740045) bank98536 +98537 POINT(41.571642303735025 -73.09557105556661) bank98537 +98538 POINT(40.00727170834387 -74.10569240170089) bank98538 +98539 POINT(41.18760071076061 -73.9611787841331) bank98539 +98540 POINT(41.4871441927402 -74.56796484980923) bank98540 +98541 POINT(40.266013747270165 -73.21501094384809) bank98541 +98542 POINT(40.742574895504404 -74.11249504350617) bank98542 +98543 POINT(39.816614464328175 -73.19457910577492) bank98543 +98544 POINT(40.140910769786416 -74.27528693125221) bank98544 +98545 POINT(39.97387888683266 -73.69675991485323) bank98545 +98546 POINT(41.547294150906346 -74.93623587858123) bank98546 +98547 POINT(41.693028898268416 -73.31324926626876) bank98547 +98548 POINT(41.60907725839714 -73.90049974677088) bank98548 +98549 POINT(41.37058042619681 -73.65414572317742) bank98549 +98550 POINT(40.087998005818214 -73.73550514041597) bank98550 +98551 POINT(41.15561162211669 -74.35832251388607) bank98551 +98552 POINT(41.5754125305854 -74.16305320205362) bank98552 +98553 POINT(41.58654808303949 -74.33441630964236) bank98553 +98554 POINT(40.26953413058428 -74.81985130969096) bank98554 +98555 POINT(40.27833114042321 -74.15520867450546) bank98555 +98556 POINT(40.716658568478486 -74.26972384604052) bank98556 +98557 POINT(39.757189917790484 -73.49130263379725) bank98557 +98558 POINT(40.19926555926374 -74.58727138035174) bank98558 +98559 POINT(40.95985141656018 -74.93865028261571) bank98559 +98560 POINT(40.01746444321262 -74.03806447478536) bank98560 +98561 POINT(41.559881366296 -74.13787735934986) bank98561 +98562 POINT(39.775775046423284 -73.29628207080675) bank98562 +98563 POINT(41.50025288292679 -73.33548855100915) bank98563 +98564 POINT(40.55496065719979 -73.64814387814462) bank98564 +98565 POINT(41.70135734640837 -73.7425818196206) bank98565 +98566 POINT(40.12920970111299 -73.05484053318494) bank98566 +98567 POINT(40.04857325155653 -74.00777528838606) bank98567 +98568 POINT(39.77783409503081 -73.77978142526901) bank98568 +98569 POINT(40.51667932138528 -73.01565766520702) bank98569 +98570 POINT(40.13918937755413 -74.04792114142943) bank98570 +98571 POINT(41.506402487034364 -74.99518722772592) bank98571 +98572 POINT(40.587085605104726 -74.27382880810354) bank98572 +98573 POINT(40.5944429535982 -74.2317167403193) bank98573 +98574 POINT(41.09828025957114 -74.92288724036632) bank98574 +98575 POINT(40.9244377876948 -73.09830097955322) bank98575 +98576 POINT(40.49413737587418 -74.22911219446527) bank98576 +98577 POINT(41.53675258821085 -73.26943146944596) bank98577 +98578 POINT(40.97920637764389 -73.20569530415358) bank98578 +98579 POINT(41.10404458769693 -73.84599875796417) bank98579 +98580 POINT(41.02016573298532 -73.66137467382644) bank98580 +98581 POINT(41.10016411612865 -73.77735956245242) bank98581 +98582 POINT(40.541559109765856 -74.95484668196633) bank98582 +98583 POINT(40.24282573353521 -73.61999998597805) bank98583 +98584 POINT(40.798548284108314 -74.94298810393556) bank98584 +98585 POINT(41.0174938465156 -73.98353722739454) bank98585 +98586 POINT(40.629082215308614 -73.90134263611942) bank98586 +98587 POINT(41.509372372338674 -74.138166158746) bank98587 +98588 POINT(41.08257845572738 -74.86629848491843) bank98588 +98589 POINT(41.47443649637171 -74.60351719117517) bank98589 +98590 POINT(40.1195661788424 -73.68383559199954) bank98590 +98591 POINT(39.798382099681724 -73.80258713883852) bank98591 +98592 POINT(40.62162188580167 -74.29529904930288) bank98592 +98593 POINT(39.80763552225855 -74.80055089530032) bank98593 +98594 POINT(40.72822685434374 -73.3791549907053) bank98594 +98595 POINT(39.77869744487182 -73.52276168625322) bank98595 +98596 POINT(40.64733994752667 -74.88722790206295) bank98596 +98597 POINT(40.48204570034914 -74.81452610070704) bank98597 +98598 POINT(39.917511756169816 -74.9068565652463) bank98598 +98599 POINT(40.28494057396453 -74.48483167033132) bank98599 +98600 POINT(41.64260412872099 -73.60205010308435) bank98600 +98601 POINT(40.485904606356186 -73.57897170942567) bank98601 +98602 POINT(39.82280459056746 -74.56106764306868) bank98602 +98603 POINT(39.89877681198962 -73.10274349723915) bank98603 +98604 POINT(40.132155732756125 -74.22448699651649) bank98604 +98605 POINT(39.97274818414432 -74.99047669928159) bank98605 +98606 POINT(41.564416871573 -74.30178744346578) bank98606 +98607 POINT(41.469847416264095 -74.56221843845711) bank98607 +98608 POINT(40.83767747629968 -73.85331667681498) bank98608 +98609 POINT(41.00211952515904 -73.47761965273942) bank98609 +98610 POINT(40.147132458677646 -73.78067694705544) bank98610 +98611 POINT(39.81821993503485 -73.54819155096487) bank98611 +98612 POINT(40.034858239022554 -74.37584701692617) bank98612 +98613 POINT(39.78728953329864 -73.56115064742393) bank98613 +98614 POINT(40.70708520990938 -74.06498871812744) bank98614 +98615 POINT(41.0333732231638 -73.38586856450608) bank98615 +98616 POINT(41.2915697789229 -74.35311056507634) bank98616 +98617 POINT(39.821789052016136 -74.06202493670493) bank98617 +98618 POINT(39.818670693000534 -74.57065571734971) bank98618 +98619 POINT(41.06886254126311 -73.10538540005084) bank98619 +98620 POINT(40.91678191140719 -74.41102735772586) bank98620 +98621 POINT(40.28637782141008 -74.5031623568459) bank98621 +98622 POINT(41.5146733767079 -74.0270886349551) bank98622 +98623 POINT(40.609284618727266 -73.59792455672022) bank98623 +98624 POINT(41.36712200008534 -73.28142335576469) bank98624 +98625 POINT(41.65386423935034 -73.49709903428963) bank98625 +98626 POINT(40.69143994421406 -74.21445425448933) bank98626 +98627 POINT(40.68594832657609 -73.89731954456279) bank98627 +98628 POINT(41.63114397851278 -74.86243155952965) bank98628 +98629 POINT(40.56412984176425 -74.37824556142937) bank98629 +98630 POINT(40.289328720759016 -74.48769464747134) bank98630 +98631 POINT(41.463989049034176 -74.88880028834099) bank98631 +98632 POINT(40.751703958833176 -73.43571054280453) bank98632 +98633 POINT(40.059663990869524 -73.75713141352531) bank98633 +98634 POINT(41.20783173774993 -74.54420806430079) bank98634 +98635 POINT(40.90577425933346 -74.04498951006053) bank98635 +98636 POINT(40.065683555399524 -73.86655123485912) bank98636 +98637 POINT(40.12278940955557 -74.09169559961846) bank98637 +98638 POINT(41.301895824526405 -73.40467932650817) bank98638 +98639 POINT(40.33726912972119 -73.91056055355683) bank98639 +98640 POINT(41.13674025321858 -74.49404865340631) bank98640 +98641 POINT(41.25469575095714 -73.5190991849505) bank98641 +98642 POINT(40.662791912396266 -73.7994374111438) bank98642 +98643 POINT(41.36575472000228 -74.2721225341114) bank98643 +98644 POINT(40.82388834735826 -74.53484018826587) bank98644 +98645 POINT(40.80575019318983 -74.56557083062123) bank98645 +98646 POINT(40.57186834454686 -74.61099221607459) bank98646 +98647 POINT(40.47882209503841 -73.90517026903306) bank98647 +98648 POINT(40.32013703490674 -73.94718555487516) bank98648 +98649 POINT(40.20015646918631 -73.97416299913434) bank98649 +98650 POINT(39.99097585847541 -74.90378866300863) bank98650 +98651 POINT(40.55351335000357 -74.13014980391125) bank98651 +98652 POINT(40.620561717976315 -73.25946298238372) bank98652 +98653 POINT(41.62904488860727 -73.81053269845602) bank98653 +98654 POINT(41.52276648393912 -73.20811272316314) bank98654 +98655 POINT(40.706918537803304 -74.76080493651271) bank98655 +98656 POINT(40.652554131717764 -74.49498008492274) bank98656 +98657 POINT(40.35170045708461 -73.71578117264913) bank98657 +98658 POINT(40.05021050792045 -74.85409371044688) bank98658 +98659 POINT(41.352717614257884 -73.93377878219056) bank98659 +98660 POINT(39.85689283175674 -74.54857789829931) bank98660 +98661 POINT(41.62650909384219 -73.5820857952517) bank98661 +98662 POINT(40.41800185772172 -73.47402659434906) bank98662 +98663 POINT(40.14777870405826 -73.44774486548764) bank98663 +98664 POINT(40.71484653865865 -73.6800623685411) bank98664 +98665 POINT(39.91536420895306 -74.21242943044345) bank98665 +98666 POINT(40.028233336439556 -73.25285375494393) bank98666 +98667 POINT(40.554467405851796 -74.67219418926997) bank98667 +98668 POINT(40.444573945165416 -73.25991857648799) bank98668 +98669 POINT(40.592849299596736 -74.80702278808637) bank98669 +98670 POINT(41.15251359673284 -73.04448112859423) bank98670 +98671 POINT(41.28997690896908 -74.68560776805586) bank98671 +98672 POINT(40.31388987070986 -74.81754094815317) bank98672 +98673 POINT(41.28278019567898 -73.48226003144003) bank98673 +98674 POINT(39.889718872121044 -74.75439074339863) bank98674 +98675 POINT(39.87936587208233 -74.14382422374392) bank98675 +98676 POINT(40.97099377578885 -74.75060659393635) bank98676 +98677 POINT(40.13753285315899 -74.63393861391346) bank98677 +98678 POINT(41.33366301066948 -74.59182727505818) bank98678 +98679 POINT(41.37514458770087 -74.59981040106167) bank98679 +98680 POINT(40.72354884182167 -73.50965423892453) bank98680 +98681 POINT(40.37005646623917 -74.00610090970102) bank98681 +98682 POINT(40.36153248507028 -74.52347868047214) bank98682 +98683 POINT(39.973060998275045 -73.21537572007742) bank98683 +98684 POINT(41.21442240262373 -74.0046734675346) bank98684 +98685 POINT(41.59588753189386 -74.49979720955994) bank98685 +98686 POINT(40.143753143827134 -73.42632142808571) bank98686 +98687 POINT(40.4813005151762 -74.67807530573917) bank98687 +98688 POINT(39.82991518708186 -73.22105700671041) bank98688 +98689 POINT(40.41732357557001 -73.84698426437997) bank98689 +98690 POINT(40.632635174117894 -74.6549124210322) bank98690 +98691 POINT(40.23948346710268 -74.18921042025993) bank98691 +98692 POINT(41.5606305807709 -73.13114610222394) bank98692 +98693 POINT(40.07377278185726 -74.56649941060839) bank98693 +98694 POINT(39.81432658761105 -73.8719476176339) bank98694 +98695 POINT(40.65293875841835 -73.88335231039684) bank98695 +98696 POINT(40.04846391909633 -73.76248677983706) bank98696 +98697 POINT(39.73750478060444 -74.86790526629132) bank98697 +98698 POINT(40.36404607532855 -74.29218336535033) bank98698 +98699 POINT(41.42985558939498 -73.34443664578696) bank98699 +98700 POINT(41.672484203736175 -73.172359539197) bank98700 +98701 POINT(40.912368452983046 -74.97508470545615) bank98701 +98702 POINT(41.52398715651102 -74.74245841063235) bank98702 +98703 POINT(40.93078696205983 -73.14506873953158) bank98703 +98704 POINT(40.90682451651629 -74.8107321702653) bank98704 +98705 POINT(41.69987062780318 -74.23695202256091) bank98705 +98706 POINT(41.40467493440462 -74.88872472719632) bank98706 +98707 POINT(41.38123411752306 -73.68100890019848) bank98707 +98708 POINT(40.73470721344682 -74.89313948382964) bank98708 +98709 POINT(40.11460634856933 -74.45453423977509) bank98709 +98710 POINT(39.841949295487304 -74.72512872208824) bank98710 +98711 POINT(39.884907587948426 -74.1435063957229) bank98711 +98712 POINT(39.93394211429193 -74.74903642847703) bank98712 +98713 POINT(40.54328760932915 -74.93014879711544) bank98713 +98714 POINT(40.35672048436553 -74.8513439981219) bank98714 +98715 POINT(39.83894519133763 -74.07323172936339) bank98715 +98716 POINT(40.38593600852532 -74.44303914638948) bank98716 +98717 POINT(39.918843655690736 -73.81769766834039) bank98717 +98718 POINT(40.32288131102531 -73.22577720327743) bank98718 +98719 POINT(40.408774126115794 -73.454994206474) bank98719 +98720 POINT(40.309417289057684 -74.71517934955627) bank98720 +98721 POINT(40.12209080241307 -74.66627942148348) bank98721 +98722 POINT(41.49392217221241 -73.47732196319481) bank98722 +98723 POINT(41.64327203725658 -74.09206178981674) bank98723 +98724 POINT(41.36074963174994 -74.8255606935798) bank98724 +98725 POINT(41.43397222120449 -74.10977826678527) bank98725 +98726 POINT(40.57290035685819 -73.10182557881951) bank98726 +98727 POINT(41.52598276296365 -73.7790464778366) bank98727 +98728 POINT(41.45313746289007 -74.48749283894485) bank98728 +98729 POINT(40.89761416382618 -73.04587122666337) bank98729 +98730 POINT(41.217662334239684 -73.5866772132796) bank98730 +98731 POINT(41.28457923958761 -74.09298590015064) bank98731 +98732 POINT(40.12838162909445 -74.9213544942734) bank98732 +98733 POINT(39.731546016020026 -73.87021531164059) bank98733 +98734 POINT(41.345524182467926 -73.65960633605432) bank98734 +98735 POINT(39.99769819789977 -73.73963770341823) bank98735 +98736 POINT(41.11144237785176 -73.89457473279698) bank98736 +98737 POINT(40.08486866424535 -73.30155345266049) bank98737 +98738 POINT(41.65542957720304 -74.06271144078148) bank98738 +98739 POINT(40.37909278728274 -73.90043730688825) bank98739 +98740 POINT(41.291933722248 -73.16842717121936) bank98740 +98741 POINT(41.071600018131605 -73.88347925837907) bank98741 +98742 POINT(40.56471526805621 -74.26972948448541) bank98742 +98743 POINT(41.07627367556149 -73.53411265925948) bank98743 +98744 POINT(40.126925109957284 -73.63707698451354) bank98744 +98745 POINT(41.55601951050413 -73.11439614261431) bank98745 +98746 POINT(40.67827045828153 -74.48291801621995) bank98746 +98747 POINT(41.594705451660964 -74.95549249977391) bank98747 +98748 POINT(40.62992747912724 -73.23818621264368) bank98748 +98749 POINT(40.59367175279888 -73.48113184350937) bank98749 +98750 POINT(41.06154972545433 -74.41119378029396) bank98750 +98751 POINT(41.270705546571115 -74.30942378366096) bank98751 +98752 POINT(41.10925149931552 -74.73448346083391) bank98752 +98753 POINT(39.88501523513896 -74.69943288478117) bank98753 +98754 POINT(40.724969911063376 -74.05334285959559) bank98754 +98755 POINT(40.82104347700128 -73.6919698919691) bank98755 +98756 POINT(40.754864979331884 -74.21898045867073) bank98756 +98757 POINT(41.29770001444114 -74.49153487218736) bank98757 +98758 POINT(39.97087436653019 -73.05430223453303) bank98758 +98759 POINT(40.14962720486223 -73.33426009662726) bank98759 +98760 POINT(41.04377534225516 -74.35135863405509) bank98760 +98761 POINT(40.5271072960247 -74.73707301112647) bank98761 +98762 POINT(41.19508805457909 -74.62925066963354) bank98762 +98763 POINT(40.202742753451396 -73.93996035115087) bank98763 +98764 POINT(41.61074233684847 -73.20195133611766) bank98764 +98765 POINT(40.24913070632813 -73.42930468526235) bank98765 +98766 POINT(41.65911502828871 -74.67943297447445) bank98766 +98767 POINT(39.92324088076148 -74.61398330526174) bank98767 +98768 POINT(39.761427494069196 -74.50053689643264) bank98768 +98769 POINT(40.88609300465882 -73.31404727470408) bank98769 +98770 POINT(40.797134931886106 -73.23201891156222) bank98770 +98771 POINT(41.05417609251686 -74.57176142476615) bank98771 +98772 POINT(41.373203179635134 -73.02628854620346) bank98772 +98773 POINT(41.062906816218614 -74.33953177537312) bank98773 +98774 POINT(39.97272366292206 -73.5323135063082) bank98774 +98775 POINT(40.79037098710316 -74.04276290072409) bank98775 +98776 POINT(40.46936776411705 -73.14995435361077) bank98776 +98777 POINT(41.45991427929525 -74.10869901190314) bank98777 +98778 POINT(40.42661144953318 -74.73189209109762) bank98778 +98779 POINT(40.53120671950909 -73.63872106016979) bank98779 +98780 POINT(40.799381331098985 -74.12851522791375) bank98780 +98781 POINT(39.736047988069814 -74.34759378685236) bank98781 +98782 POINT(40.930274348113386 -74.03423029518548) bank98782 +98783 POINT(39.842642751128885 -74.86516295034107) bank98783 +98784 POINT(40.771601272803615 -74.55362983636988) bank98784 +98785 POINT(39.78749462487503 -74.39892137231895) bank98785 +98786 POINT(40.28652635645393 -73.12826128452271) bank98786 +98787 POINT(41.22845549363292 -73.88347100959203) bank98787 +98788 POINT(41.636282352053065 -74.51539428708269) bank98788 +98789 POINT(40.047846941494754 -73.63903907319434) bank98789 +98790 POINT(41.60569219717355 -74.30865972185751) bank98790 +98791 POINT(41.272854892525324 -74.09199635867954) bank98791 +98792 POINT(41.3076148816374 -74.60559835668262) bank98792 +98793 POINT(40.10980109668576 -74.29670492567593) bank98793 +98794 POINT(41.26552401232232 -73.17575664157029) bank98794 +98795 POINT(41.60521899940691 -74.3810631251256) bank98795 +98796 POINT(40.25204941984321 -73.39939155426244) bank98796 +98797 POINT(40.493975430107454 -73.43588194226592) bank98797 +98798 POINT(40.309491110560565 -73.98913828255733) bank98798 +98799 POINT(40.23356808681577 -73.54632812198557) bank98799 +98800 POINT(41.08775675063721 -74.28379096671927) bank98800 +98801 POINT(39.86207141449116 -73.60761485842869) bank98801 +98802 POINT(41.59795954196252 -74.07199503051862) bank98802 +98803 POINT(41.15544787083025 -73.61944128508618) bank98803 +98804 POINT(40.09769670050665 -74.62311059830401) bank98804 +98805 POINT(40.63733419084393 -74.17694260921846) bank98805 +98806 POINT(39.719871903300394 -73.98143756712024) bank98806 +98807 POINT(40.06440464572235 -74.39445157218307) bank98807 +98808 POINT(41.68657276853842 -74.64066813246689) bank98808 +98809 POINT(41.561902229222014 -74.9494556530447) bank98809 +98810 POINT(39.75127293277094 -73.06725489632068) bank98810 +98811 POINT(40.333450951593484 -73.92204078434047) bank98811 +98812 POINT(41.67118335315856 -73.0564201795339) bank98812 +98813 POINT(39.80793585705355 -74.90933011204048) bank98813 +98814 POINT(41.241209987259005 -73.91296734105181) bank98814 +98815 POINT(40.833863369120465 -73.61989709678654) bank98815 +98816 POINT(40.35878267754221 -74.26406640636243) bank98816 +98817 POINT(40.50764056967801 -74.19574076573393) bank98817 +98818 POINT(40.025713002942794 -73.20397829560908) bank98818 +98819 POINT(40.64854143238999 -73.88322974032057) bank98819 +98820 POINT(40.19353041544886 -74.23878431500731) bank98820 +98821 POINT(41.347219683062946 -74.94299107198827) bank98821 +98822 POINT(39.73930582469173 -73.74529247984194) bank98822 +98823 POINT(41.1271326042765 -74.49814080245686) bank98823 +98824 POINT(40.57823026439544 -74.51260640804247) bank98824 +98825 POINT(41.453591342191054 -74.11963141161058) bank98825 +98826 POINT(39.92160456403205 -73.9838459671561) bank98826 +98827 POINT(41.41156528315021 -74.10008033895318) bank98827 +98828 POINT(40.22418295562738 -73.18310312658295) bank98828 +98829 POINT(41.61741781278518 -74.20388670679556) bank98829 +98830 POINT(39.95584089155176 -73.66199021074898) bank98830 +98831 POINT(40.99946890895455 -74.70733433815596) bank98831 +98832 POINT(41.21482772792238 -74.79772178387057) bank98832 +98833 POINT(40.89245488763256 -74.05687574896126) bank98833 +98834 POINT(40.53885798789079 -74.04822237117801) bank98834 +98835 POINT(40.218593391016476 -74.82921778029673) bank98835 +98836 POINT(40.381194869684435 -74.23615803824589) bank98836 +98837 POINT(40.14917920009584 -74.32897874499324) bank98837 +98838 POINT(41.2316610234178 -73.35251428902673) bank98838 +98839 POINT(40.80586249387589 -73.85846778656604) bank98839 +98840 POINT(40.50301164761394 -74.06706269530761) bank98840 +98841 POINT(40.939101580967595 -73.88865929864077) bank98841 +98842 POINT(41.07046773814257 -73.01196973667471) bank98842 +98843 POINT(41.55494343269134 -74.2922522159335) bank98843 +98844 POINT(40.816144855659886 -73.14226205706156) bank98844 +98845 POINT(41.38560438264066 -73.42415443932893) bank98845 +98846 POINT(41.170194457383346 -73.8412879262527) bank98846 +98847 POINT(41.48980226240441 -73.06905023884502) bank98847 +98848 POINT(40.981160787835975 -74.68408799292813) bank98848 +98849 POINT(41.559052489505035 -73.02055999159047) bank98849 +98850 POINT(39.92037405501341 -73.97493812006601) bank98850 +98851 POINT(41.10018853381688 -74.06500561092155) bank98851 +98852 POINT(41.623931725195426 -73.29244455889416) bank98852 +98853 POINT(40.385002592881165 -73.6031083749615) bank98853 +98854 POINT(40.519362524395966 -74.17777726610825) bank98854 +98855 POINT(40.23178503730137 -74.63408154878984) bank98855 +98856 POINT(39.94581038921987 -73.09485803383433) bank98856 +98857 POINT(41.47984697479107 -74.60493372668257) bank98857 +98858 POINT(40.61232047946526 -74.69282401278438) bank98858 +98859 POINT(41.54252990718038 -73.33715755703486) bank98859 +98860 POINT(40.17769254790862 -74.3190431306941) bank98860 +98861 POINT(40.58998785263397 -73.14339747098927) bank98861 +98862 POINT(40.12839228647415 -73.71029849234235) bank98862 +98863 POINT(41.32677488226555 -73.48713638027533) bank98863 +98864 POINT(40.2778783323168 -74.84720240750558) bank98864 +98865 POINT(39.775212752081025 -74.18068835991362) bank98865 +98866 POINT(40.04949060441194 -74.19901647533203) bank98866 +98867 POINT(40.393470435925444 -74.12466107963054) bank98867 +98868 POINT(41.084431561398986 -74.29050941229642) bank98868 +98869 POINT(41.59772636508308 -74.97504049163948) bank98869 +98870 POINT(40.21713946826417 -74.58083034418325) bank98870 +98871 POINT(40.17987432223983 -74.28491413904648) bank98871 +98872 POINT(41.54399701706569 -73.3618572012059) bank98872 +98873 POINT(40.86330442876198 -74.13198454022637) bank98873 +98874 POINT(40.410700885930666 -74.49560243149726) bank98874 +98875 POINT(41.004816500950895 -74.01464278768789) bank98875 +98876 POINT(40.24626307158413 -73.82833120336103) bank98876 +98877 POINT(40.42219360142798 -73.6273998160008) bank98877 +98878 POINT(41.1888552036156 -74.61461768719055) bank98878 +98879 POINT(41.21197062583265 -73.1136260064136) bank98879 +98880 POINT(41.65758513039447 -73.50898439421996) bank98880 +98881 POINT(39.72548421600755 -73.46368294690154) bank98881 +98882 POINT(40.75892621488075 -73.4523554864557) bank98882 +98883 POINT(41.3768773414053 -74.8356546176087) bank98883 +98884 POINT(41.06441562070191 -74.18362232312569) bank98884 +98885 POINT(41.02148935997765 -73.25238001562015) bank98885 +98886 POINT(41.443671963891155 -73.3672397271953) bank98886 +98887 POINT(40.39702831222422 -74.41874067870033) bank98887 +98888 POINT(41.14861005830331 -74.60448067250492) bank98888 +98889 POINT(40.421064634507104 -74.49330397620199) bank98889 +98890 POINT(41.02027139378838 -74.60250051399089) bank98890 +98891 POINT(39.955384178621735 -74.56058675217217) bank98891 +98892 POINT(41.23142130423291 -73.41336433300198) bank98892 +98893 POINT(40.44753682610276 -73.24046258897606) bank98893 +98894 POINT(41.06453896528178 -74.59408202716511) bank98894 +98895 POINT(40.442457378771294 -73.8202077296415) bank98895 +98896 POINT(40.10696573626849 -73.37284122904933) bank98896 +98897 POINT(41.612240233722126 -73.18756588268988) bank98897 +98898 POINT(40.227062344415 -73.1289116889477) bank98898 +98899 POINT(40.04128364569099 -74.4309900724243) bank98899 +98900 POINT(40.106131211436804 -74.59001136097713) bank98900 +98901 POINT(40.91075880303302 -73.22549441995447) bank98901 +98902 POINT(40.806419887371845 -73.8508512342646) bank98902 +98903 POINT(40.819743536605024 -74.52904953567082) bank98903 +98904 POINT(40.92083917720287 -74.7678742486057) bank98904 +98905 POINT(40.63834776550825 -73.02058431535721) bank98905 +98906 POINT(41.32774113149893 -74.72413221867134) bank98906 +98907 POINT(40.01767203046739 -74.21592197360818) bank98907 +98908 POINT(39.80363369633586 -74.4125363372961) bank98908 +98909 POINT(41.10962232766545 -74.86808474755104) bank98909 +98910 POINT(39.778714149574554 -73.63285388993772) bank98910 +98911 POINT(40.74843147157557 -74.20775243191119) bank98911 +98912 POINT(40.34157090736815 -74.27214075032383) bank98912 +98913 POINT(40.94356343010811 -73.92823698159818) bank98913 +98914 POINT(41.36858704321156 -74.54397062631699) bank98914 +98915 POINT(41.287643628358104 -74.13677932023383) bank98915 +98916 POINT(40.04977657809157 -73.73895588654649) bank98916 +98917 POINT(40.110239108476996 -74.63879881514755) bank98917 +98918 POINT(41.62320985756207 -73.2325422523965) bank98918 +98919 POINT(39.83209099709815 -74.67240368643874) bank98919 +98920 POINT(40.39362160160664 -73.84271955525283) bank98920 +98921 POINT(41.33424308618941 -74.11503548565716) bank98921 +98922 POINT(41.046856188851095 -73.98862243606322) bank98922 +98923 POINT(40.82953635826945 -73.10473827752288) bank98923 +98924 POINT(40.77658111857325 -74.20001303143596) bank98924 +98925 POINT(41.69335558827428 -73.86435420624149) bank98925 +98926 POINT(40.49905363778068 -74.94640381852207) bank98926 +98927 POINT(40.02880898129511 -73.9407553243567) bank98927 +98928 POINT(40.397244620588474 -73.28450057113918) bank98928 +98929 POINT(40.40055618789789 -73.8512309906212) bank98929 +98930 POINT(41.094575234236615 -74.77847540193271) bank98930 +98931 POINT(41.361547044168006 -73.70953738265054) bank98931 +98932 POINT(41.67333099104126 -73.04744497388559) bank98932 +98933 POINT(40.93524398712256 -73.32133490153076) bank98933 +98934 POINT(40.14452424070244 -74.3147667224779) bank98934 +98935 POINT(40.322343061524336 -73.71008351254022) bank98935 +98936 POINT(40.83969683955718 -73.05264696173577) bank98936 +98937 POINT(39.80907265997651 -74.50842132157881) bank98937 +98938 POINT(39.792620349226304 -73.5487624095472) bank98938 +98939 POINT(41.16024832966047 -74.99843745454201) bank98939 +98940 POINT(40.30878001506215 -74.31875771357244) bank98940 +98941 POINT(41.30377601301726 -73.64453669136613) bank98941 +98942 POINT(41.62404145380547 -73.76445443468256) bank98942 +98943 POINT(40.268829481150384 -75.00088613844402) bank98943 +98944 POINT(40.35786276839265 -73.36138305133277) bank98944 +98945 POINT(41.525054672555655 -74.8985594773017) bank98945 +98946 POINT(39.785566720172724 -74.57541737691783) bank98946 +98947 POINT(41.09524221079149 -74.6315990964833) bank98947 +98948 POINT(41.383097798367196 -74.9813113750377) bank98948 +98949 POINT(41.21622697116804 -73.04035327975463) bank98949 +98950 POINT(40.63121675309033 -73.7378394783356) bank98950 +98951 POINT(40.513007274809745 -74.44368207482563) bank98951 +98952 POINT(41.02701958743087 -74.2169686436531) bank98952 +98953 POINT(41.420835462065085 -73.09500998477957) bank98953 +98954 POINT(40.00160875929238 -74.78140465222853) bank98954 +98955 POINT(39.74680698632475 -73.31513319422864) bank98955 +98956 POINT(41.37390509713917 -75.0042528590518) bank98956 +98957 POINT(41.61789033698163 -74.12151279378955) bank98957 +98958 POINT(40.47168526120644 -73.18289772364801) bank98958 +98959 POINT(39.751333863042674 -74.09020437471413) bank98959 +98960 POINT(41.63940558547879 -74.39896609045394) bank98960 +98961 POINT(40.2474625867382 -74.63989573647005) bank98961 +98962 POINT(40.07385877059853 -74.05242671567288) bank98962 +98963 POINT(41.63774230439911 -74.37267175670141) bank98963 +98964 POINT(41.607047243504645 -74.64507818056077) bank98964 +98965 POINT(41.39177027472742 -74.74886836792795) bank98965 +98966 POINT(40.6612314560042 -73.24830857691047) bank98966 +98967 POINT(40.174980706090935 -74.17230315023161) bank98967 +98968 POINT(40.80404221890896 -74.21061667665606) bank98968 +98969 POINT(41.00862520019908 -74.80464937723184) bank98969 +98970 POINT(41.47958409445278 -73.99785379342963) bank98970 +98971 POINT(40.03936907453505 -74.97068222299444) bank98971 +98972 POINT(39.9062613147864 -73.65503925137389) bank98972 +98973 POINT(40.02452812424092 -73.63834053113425) bank98973 +98974 POINT(41.21386695848126 -73.86272689614214) bank98974 +98975 POINT(41.071210742409896 -73.7986389706) bank98975 +98976 POINT(41.61458815283681 -73.73134740667844) bank98976 +98977 POINT(41.36011272290018 -74.3045941671739) bank98977 +98978 POINT(41.38701911686239 -74.09645391113027) bank98978 +98979 POINT(41.16261980135911 -73.2128661434464) bank98979 +98980 POINT(41.21876356753257 -73.40725668609525) bank98980 +98981 POINT(40.710821100696585 -74.84917455907616) bank98981 +98982 POINT(40.4026786994716 -73.27557938354136) bank98982 +98983 POINT(40.57271803389686 -74.20610270421514) bank98983 +98984 POINT(40.01205378461823 -74.65106270499406) bank98984 +98985 POINT(41.46622693813967 -74.18826438283564) bank98985 +98986 POINT(40.91461946117124 -74.37277449079023) bank98986 +98987 POINT(40.70449430760985 -73.37252666628505) bank98987 +98988 POINT(40.67822624938224 -73.75140080822146) bank98988 +98989 POINT(40.96605076035988 -73.21190256350921) bank98989 +98990 POINT(40.85570312314038 -73.2279649893444) bank98990 +98991 POINT(40.670497118007766 -73.90338617107791) bank98991 +98992 POINT(41.131789602593145 -73.73200013470723) bank98992 +98993 POINT(41.2354883862195 -73.53603318871633) bank98993 +98994 POINT(39.906505358238675 -73.58092816763599) bank98994 +98995 POINT(41.20410703392107 -74.42148624118124) bank98995 +98996 POINT(40.70702197950833 -73.53221612770952) bank98996 +98997 POINT(39.82480394121011 -74.58959458087297) bank98997 +98998 POINT(40.751465521580954 -73.55150989416421) bank98998 +98999 POINT(40.63474248673282 -74.72882612961592) bank98999 +99000 POINT(39.73500225793309 -73.30633864990924) bank99000 +99001 POINT(40.10821559027178 -74.72411880743945) bank99001 +99002 POINT(40.397313213844164 -73.99342381523321) bank99002 +99003 POINT(39.742819044581175 -73.58102608293058) bank99003 +99004 POINT(41.12483892816085 -73.50581581526622) bank99004 +99005 POINT(41.33883010022313 -74.1486882724509) bank99005 +99006 POINT(39.89464416089478 -74.64800365316096) bank99006 +99007 POINT(39.81468035200102 -74.77091797258984) bank99007 +99008 POINT(40.68360827689961 -74.07180843789962) bank99008 +99009 POINT(39.83373173269172 -73.29015119914526) bank99009 +99010 POINT(40.94118911631186 -73.84476673497828) bank99010 +99011 POINT(39.75046173555131 -74.61685775456392) bank99011 +99012 POINT(40.04190779126064 -73.26266054016341) bank99012 +99013 POINT(40.49416298685083 -74.33271885005756) bank99013 +99014 POINT(40.586591553330216 -74.33889348801152) bank99014 +99015 POINT(40.960998281112964 -74.96660357729263) bank99015 +99016 POINT(41.627618409632994 -73.61191748057877) bank99016 +99017 POINT(40.1919850530232 -73.2229080954904) bank99017 +99018 POINT(40.34113666877148 -73.24368010995677) bank99018 +99019 POINT(40.05028527453028 -73.75360947453652) bank99019 +99020 POINT(40.23188849439021 -74.93443449997882) bank99020 +99021 POINT(41.169363949422674 -74.81024441027995) bank99021 +99022 POINT(39.84851732498226 -73.84289372292042) bank99022 +99023 POINT(39.867676439517965 -73.79272381413895) bank99023 +99024 POINT(41.353716397291244 -73.12350417212434) bank99024 +99025 POINT(41.528169696678866 -74.56833385868119) bank99025 +99026 POINT(40.34741810645842 -74.00873373902638) bank99026 +99027 POINT(40.200482492424655 -74.2103639854517) bank99027 +99028 POINT(40.791387343689976 -73.64979016746273) bank99028 +99029 POINT(40.58949526585514 -73.14458316607296) bank99029 +99030 POINT(41.27299837078101 -73.94231396356096) bank99030 +99031 POINT(40.884378567703 -74.25608942809025) bank99031 +99032 POINT(40.318303581140924 -73.63600712384623) bank99032 +99033 POINT(39.92388171843658 -74.0951229414713) bank99033 +99034 POINT(41.221754396772205 -74.6063960209184) bank99034 +99035 POINT(40.745804093343594 -73.5543684230526) bank99035 +99036 POINT(41.21858489681326 -74.09482036650799) bank99036 +99037 POINT(40.908640713242455 -73.88101233812489) bank99037 +99038 POINT(41.4890767290605 -73.3348179379132) bank99038 +99039 POINT(41.2956570926156 -74.54639133944775) bank99039 +99040 POINT(41.452736080053754 -73.66146529306467) bank99040 +99041 POINT(39.9610564759416 -73.64622616545905) bank99041 +99042 POINT(40.382422607503614 -73.17292119449029) bank99042 +99043 POINT(39.925900909745 -74.84050933866328) bank99043 +99044 POINT(41.256005683696166 -74.50475261146276) bank99044 +99045 POINT(39.91616392460284 -73.79741381756345) bank99045 +99046 POINT(40.856974148467394 -73.84704161624329) bank99046 +99047 POINT(40.694312133355744 -74.17588187986594) bank99047 +99048 POINT(41.26763738722847 -73.7547617261403) bank99048 +99049 POINT(40.66181024408583 -74.74382840126273) bank99049 +99050 POINT(39.9477306197201 -73.83470307024767) bank99050 +99051 POINT(40.47988201604481 -73.40553111629791) bank99051 +99052 POINT(39.9539451433263 -74.44005672464623) bank99052 +99053 POINT(39.970418928894816 -74.70687541637604) bank99053 +99054 POINT(40.644108646418715 -74.57780207594864) bank99054 +99055 POINT(41.382075355986316 -74.8106745129845) bank99055 +99056 POINT(40.27904417200714 -73.22282030054623) bank99056 +99057 POINT(41.21961028536501 -74.07423681804165) bank99057 +99058 POINT(41.280178428584016 -73.91060521271797) bank99058 +99059 POINT(40.69051736172787 -73.75135308352539) bank99059 +99060 POINT(40.786936521127195 -73.60828444768671) bank99060 +99061 POINT(41.321453806212915 -73.22604786020021) bank99061 +99062 POINT(40.29463325282719 -74.82836568959785) bank99062 +99063 POINT(41.452769193551624 -73.67950321230276) bank99063 +99064 POINT(39.74219315231232 -74.26499743744823) bank99064 +99065 POINT(40.37686747156822 -74.22339101119805) bank99065 +99066 POINT(41.20101881349319 -74.77238201271604) bank99066 +99067 POINT(41.27914120714715 -73.58718869971256) bank99067 +99068 POINT(39.986118740136355 -73.40629334206083) bank99068 +99069 POINT(40.01635241414595 -73.24709947033956) bank99069 +99070 POINT(40.57275658956796 -74.08989086905146) bank99070 +99071 POINT(40.3562480289302 -74.93365248734504) bank99071 +99072 POINT(41.294629369171744 -74.3429462580887) bank99072 +99073 POINT(39.77880973172755 -74.53727991296039) bank99073 +99074 POINT(41.02298798478785 -73.31326796096957) bank99074 +99075 POINT(40.468885476028426 -74.06851448003013) bank99075 +99076 POINT(40.03470349041789 -74.98212028935495) bank99076 +99077 POINT(41.08291925994495 -74.99869339484779) bank99077 +99078 POINT(41.46185816710485 -74.16557049640028) bank99078 +99079 POINT(40.543010104341064 -74.34715801902591) bank99079 +99080 POINT(40.750596240556995 -74.39815154478879) bank99080 +99081 POINT(40.70568893284946 -73.89428154184424) bank99081 +99082 POINT(40.222249910959555 -74.42634301709417) bank99082 +99083 POINT(41.35173760038842 -73.06754210518743) bank99083 +99084 POINT(41.56176532133536 -73.29229470470347) bank99084 +99085 POINT(39.873281126128425 -74.24397583314023) bank99085 +99086 POINT(39.85381790471151 -73.46590918013906) bank99086 +99087 POINT(40.33378828925526 -74.20855117223277) bank99087 +99088 POINT(40.62701586531146 -74.9592753924171) bank99088 +99089 POINT(41.246353064812716 -73.3129122440661) bank99089 +99090 POINT(39.7844558137722 -73.89385994161873) bank99090 +99091 POINT(41.12314527080599 -73.82226761765754) bank99091 +99092 POINT(41.00351789788343 -74.26705574813232) bank99092 +99093 POINT(40.69802068426207 -74.72174216672384) bank99093 +99094 POINT(41.2943781800523 -73.72247250426965) bank99094 +99095 POINT(40.25830753476471 -74.32484696765151) bank99095 +99096 POINT(40.52660816579126 -73.13611911372807) bank99096 +99097 POINT(40.3450449363543 -73.12566179845531) bank99097 +99098 POINT(39.805382435199704 -73.61758091202813) bank99098 +99099 POINT(40.9741030275441 -74.51535494275464) bank99099 +99100 POINT(41.32526960509871 -73.79248865302205) bank99100 +99101 POINT(41.40683300352221 -73.63853034716841) bank99101 +99102 POINT(40.97345765259936 -73.71060696153035) bank99102 +99103 POINT(40.11402422273455 -73.58300958328378) bank99103 +99104 POINT(40.32955298868016 -73.8075952161785) bank99104 +99105 POINT(40.83011680320755 -73.8737890694485) bank99105 +99106 POINT(40.946005697066056 -73.17180748782043) bank99106 +99107 POINT(39.731060105825165 -73.51547830527396) bank99107 +99108 POINT(40.71288649708817 -74.01129524362489) bank99108 +99109 POINT(40.477630731785474 -73.61264767916333) bank99109 +99110 POINT(39.96471469129213 -73.81281886939782) bank99110 +99111 POINT(41.540093562969034 -73.65882397156801) bank99111 +99112 POINT(39.86887498909554 -73.40648734551495) bank99112 +99113 POINT(40.630234044431155 -74.80432032560206) bank99113 +99114 POINT(40.8711188358869 -73.82984783928836) bank99114 +99115 POINT(41.09597099401297 -73.63224845803167) bank99115 +99116 POINT(41.53174627298879 -73.7205367983594) bank99116 +99117 POINT(40.510265720870585 -73.39926163655795) bank99117 +99118 POINT(41.05548123602914 -73.07753880485258) bank99118 +99119 POINT(41.38300586287486 -73.73943752603175) bank99119 +99120 POINT(41.457234729841744 -73.37982434255895) bank99120 +99121 POINT(40.34382209020967 -73.21935921858109) bank99121 +99122 POINT(39.82561522015785 -74.65978679093429) bank99122 +99123 POINT(39.84535415822787 -73.45916727209153) bank99123 +99124 POINT(40.26408828270274 -74.19380185202012) bank99124 +99125 POINT(39.9168486536492 -74.57073244052235) bank99125 +99126 POINT(41.707107165444654 -74.26678101769572) bank99126 +99127 POINT(40.25170955645446 -74.44711408267266) bank99127 +99128 POINT(39.82760071488476 -73.98702011709057) bank99128 +99129 POINT(41.61114385273403 -73.28653661518369) bank99129 +99130 POINT(41.089928030097965 -74.46048530482139) bank99130 +99131 POINT(40.71662756932963 -74.51153381610443) bank99131 +99132 POINT(41.0514373522955 -73.82034778761161) bank99132 +99133 POINT(40.42757774019234 -74.46811775721433) bank99133 +99134 POINT(40.84980114549793 -73.8433460998574) bank99134 +99135 POINT(40.42387935506876 -74.04252345687738) bank99135 +99136 POINT(40.51806600731914 -74.9800325888365) bank99136 +99137 POINT(39.82291407424764 -74.10416056296646) bank99137 +99138 POINT(40.9800851802419 -73.41258332774557) bank99138 +99139 POINT(40.67840472232979 -73.91297489183394) bank99139 +99140 POINT(40.59174765023747 -74.77498742161151) bank99140 +99141 POINT(40.023821605742526 -73.89551088056292) bank99141 +99142 POINT(41.03713197381669 -73.02075208608885) bank99142 +99143 POINT(40.770816765107135 -74.20851661010802) bank99143 +99144 POINT(41.51346404368238 -74.48077577128346) bank99144 +99145 POINT(40.089409198275845 -74.1928018914258) bank99145 +99146 POINT(41.4218044894152 -74.69210008769574) bank99146 +99147 POINT(41.19761083526562 -74.6545231384582) bank99147 +99148 POINT(40.64117235491316 -74.63427273214909) bank99148 +99149 POINT(41.400023618295165 -74.4120399200127) bank99149 +99150 POINT(41.03873377124484 -74.55432029052163) bank99150 +99151 POINT(41.51377251313847 -74.72737192024528) bank99151 +99152 POINT(41.23194472346982 -73.6467376808206) bank99152 +99153 POINT(39.88182061692481 -73.73463083657145) bank99153 +99154 POINT(41.28428667304117 -74.75624944598295) bank99154 +99155 POINT(41.2171981570644 -74.33823299530543) bank99155 +99156 POINT(39.772418885922185 -73.66997653412554) bank99156 +99157 POINT(41.59692040827791 -73.87725643904442) bank99157 +99158 POINT(40.51083736900768 -73.51515366592601) bank99158 +99159 POINT(41.02565382996627 -74.03346323254642) bank99159 +99160 POINT(39.716785717979775 -74.72919322975281) bank99160 +99161 POINT(41.1736386251921 -74.01002192338213) bank99161 +99162 POINT(41.70945021845177 -73.2997021791816) bank99162 +99163 POINT(40.8454636413785 -74.5232779348697) bank99163 +99164 POINT(41.27439703252862 -74.49421919428545) bank99164 +99165 POINT(41.060838852107246 -73.88680759690108) bank99165 +99166 POINT(40.73211544266274 -73.88385074676518) bank99166 +99167 POINT(41.41244694229948 -74.32457690689162) bank99167 +99168 POINT(40.904421900765435 -73.62354192876798) bank99168 +99169 POINT(40.848956128291334 -73.64155570448415) bank99169 +99170 POINT(39.994144605394595 -74.32684144103757) bank99170 +99171 POINT(40.964983560687955 -73.4711744910132) bank99171 +99172 POINT(41.42744413910301 -74.14891455056832) bank99172 +99173 POINT(40.95557360082674 -73.96720325924784) bank99173 +99174 POINT(40.180025162355 -73.40604291900617) bank99174 +99175 POINT(41.579299341319796 -74.54638661976401) bank99175 +99176 POINT(40.25952855588407 -73.9814527877925) bank99176 +99177 POINT(41.374004556973226 -74.19766286803996) bank99177 +99178 POINT(40.39208286963554 -74.39673225309188) bank99178 +99179 POINT(41.154436728291735 -73.2967376465483) bank99179 +99180 POINT(40.874645444650426 -74.20632504504766) bank99180 +99181 POINT(40.80672132122984 -73.6946234267906) bank99181 +99182 POINT(40.40868311601412 -73.56999885278069) bank99182 +99183 POINT(41.10377840896613 -74.48753139831999) bank99183 +99184 POINT(40.06891133713769 -74.9237959884444) bank99184 +99185 POINT(41.37894381951831 -73.4174827281728) bank99185 +99186 POINT(41.346237087362304 -74.84793462787277) bank99186 +99187 POINT(41.21217817422072 -73.85379168630723) bank99187 +99188 POINT(39.969467871704985 -73.16352137893752) bank99188 +99189 POINT(41.61463616943567 -73.66796973615759) bank99189 +99190 POINT(40.492031912262526 -74.03045947881671) bank99190 +99191 POINT(39.87843078865542 -73.90959245868326) bank99191 +99192 POINT(39.95897586679357 -74.76939116526776) bank99192 +99193 POINT(40.938786815293135 -74.99857765187582) bank99193 +99194 POINT(40.51017301355926 -75.00551980381373) bank99194 +99195 POINT(40.47137609270321 -73.60267534498516) bank99195 +99196 POINT(40.338608720547036 -73.59804309364185) bank99196 +99197 POINT(40.53901692141411 -74.74269864448362) bank99197 +99198 POINT(40.71310577540474 -73.16531820921576) bank99198 +99199 POINT(40.20884096996482 -73.35292061469923) bank99199 +99200 POINT(41.5210070065316 -74.24387310441935) bank99200 +99201 POINT(41.22137440417835 -73.23083090951859) bank99201 +99202 POINT(40.62335988693538 -74.58178394050421) bank99202 +99203 POINT(40.01522040736088 -73.92250513348533) bank99203 +99204 POINT(40.347029680326614 -73.9610702047817) bank99204 +99205 POINT(40.698270007862355 -74.88304669655129) bank99205 +99206 POINT(41.50753051174817 -74.1151340676112) bank99206 +99207 POINT(40.818992587871605 -74.4845372338977) bank99207 +99208 POINT(40.663622632190595 -74.45888772821674) bank99208 +99209 POINT(39.73733032198353 -74.18351766087305) bank99209 +99210 POINT(40.75460474932201 -74.36672501187195) bank99210 +99211 POINT(40.812380372033886 -74.26447344044458) bank99211 +99212 POINT(41.36330230413243 -74.66412936699359) bank99212 +99213 POINT(40.13793654723087 -74.23283956971571) bank99213 +99214 POINT(40.90410699655729 -74.65071593979555) bank99214 +99215 POINT(40.495511320881235 -74.63454660029502) bank99215 +99216 POINT(41.260133348459135 -74.43190262405108) bank99216 +99217 POINT(39.81472720317282 -74.6527635142993) bank99217 +99218 POINT(40.77448404753226 -73.78453611388679) bank99218 +99219 POINT(40.37864566068216 -74.19616119990286) bank99219 +99220 POINT(40.428002857115615 -73.08173946154304) bank99220 +99221 POINT(40.74284973886645 -74.53041875248495) bank99221 +99222 POINT(39.84143115190745 -74.61700722302336) bank99222 +99223 POINT(41.219787250840334 -73.05570881035017) bank99223 +99224 POINT(41.24838967638062 -73.52312084924638) bank99224 +99225 POINT(40.67561820039982 -74.76862770248682) bank99225 +99226 POINT(40.086167889117114 -74.86510818897874) bank99226 +99227 POINT(41.698537161961674 -73.15148921949196) bank99227 +99228 POINT(41.234943527624935 -74.80825930409854) bank99228 +99229 POINT(40.62482527457272 -73.26269867022107) bank99229 +99230 POINT(40.23906381008882 -74.42468146993366) bank99230 +99231 POINT(40.83913892204551 -74.2698386671525) bank99231 +99232 POINT(39.98488343441137 -73.4164733898766) bank99232 +99233 POINT(40.02591335128086 -73.21589412058287) bank99233 +99234 POINT(39.90786682536363 -74.35161869245108) bank99234 +99235 POINT(40.72758184256967 -74.67605823909425) bank99235 +99236 POINT(40.12871867905495 -74.2159464936952) bank99236 +99237 POINT(40.51128361481767 -73.32416828051028) bank99237 +99238 POINT(41.62068325652341 -74.54961149607334) bank99238 +99239 POINT(40.10307789674697 -74.41108316611655) bank99239 +99240 POINT(41.38076934030122 -73.19857996019124) bank99240 +99241 POINT(39.91766561291097 -73.14040156515998) bank99241 +99242 POINT(40.32899274729428 -74.98883120675933) bank99242 +99243 POINT(40.88564843271913 -73.91863204436913) bank99243 +99244 POINT(40.849974247072716 -73.18947241120767) bank99244 +99245 POINT(40.863985186498994 -74.07086596646747) bank99245 +99246 POINT(39.97670877565616 -73.28267007085913) bank99246 +99247 POINT(41.19875681195356 -74.0964714932731) bank99247 +99248 POINT(41.57591636986652 -74.3668214128265) bank99248 +99249 POINT(40.38058943234719 -74.70542882772666) bank99249 +99250 POINT(41.3132931256294 -74.891312308657) bank99250 +99251 POINT(40.31121235822327 -74.76948037356452) bank99251 +99252 POINT(41.31998325809783 -74.67438852046722) bank99252 +99253 POINT(41.0739882533947 -73.06962887743947) bank99253 +99254 POINT(41.17840174399008 -73.86789858302171) bank99254 +99255 POINT(41.05085750434136 -73.8761949828596) bank99255 +99256 POINT(41.07591164260838 -73.90345373346257) bank99256 +99257 POINT(41.2407450985983 -73.54656112337015) bank99257 +99258 POINT(41.01083351069892 -73.37490650697761) bank99258 +99259 POINT(41.232225037531904 -74.92843753489863) bank99259 +99260 POINT(39.747710607628505 -73.44677990568236) bank99260 +99261 POINT(40.83107319432933 -75.00495188311983) bank99261 +99262 POINT(40.09712543939464 -74.63918288205134) bank99262 +99263 POINT(40.811550551773074 -74.60797845268554) bank99263 +99264 POINT(40.05960426167189 -73.99086480975997) bank99264 +99265 POINT(40.98674785051145 -74.03941590850546) bank99265 +99266 POINT(40.684276647692776 -73.21753902852876) bank99266 +99267 POINT(41.59426756795029 -73.48106121700134) bank99267 +99268 POINT(41.4935371528551 -73.5927737366166) bank99268 +99269 POINT(40.22763308517929 -73.11343719383785) bank99269 +99270 POINT(40.172325552524185 -73.20389959323774) bank99270 +99271 POINT(40.40183768407415 -74.22026362668737) bank99271 +99272 POINT(41.39470010720892 -73.22475873989492) bank99272 +99273 POINT(41.30398930661398 -74.38189881430779) bank99273 +99274 POINT(40.221582935258546 -73.73785236380404) bank99274 +99275 POINT(40.43634449104017 -74.11370635391062) bank99275 +99276 POINT(41.65499513996078 -74.87514168801458) bank99276 +99277 POINT(41.23757395559993 -73.34992039851635) bank99277 +99278 POINT(40.68661182462977 -73.3552746343538) bank99278 +99279 POINT(40.929332326716604 -74.05642492260905) bank99279 +99280 POINT(39.820988761414334 -74.86226197281819) bank99280 +99281 POINT(40.13840573155947 -74.64797354236211) bank99281 +99282 POINT(39.79524847156903 -73.47294076814785) bank99282 +99283 POINT(40.011306538336115 -73.8806436366944) bank99283 +99284 POINT(40.842418676677944 -74.15204835774477) bank99284 +99285 POINT(40.52247135269559 -74.25817769817925) bank99285 +99286 POINT(39.729821369433566 -73.6549512435604) bank99286 +99287 POINT(41.631660859192266 -73.31559749800087) bank99287 +99288 POINT(41.336520729990475 -74.79016855114381) bank99288 +99289 POINT(40.535779519455 -74.93653062874732) bank99289 +99290 POINT(40.71217913571469 -73.04271615028956) bank99290 +99291 POINT(40.88320079750674 -74.74076783868513) bank99291 +99292 POINT(40.411800909364004 -73.09381203191343) bank99292 +99293 POINT(39.89805155287723 -73.19784113439844) bank99293 +99294 POINT(41.12854977390259 -73.57199685023335) bank99294 +99295 POINT(41.19319976392154 -74.98881595428794) bank99295 +99296 POINT(40.05776247772763 -74.4228659318556) bank99296 +99297 POINT(39.92069554528352 -73.37697302418682) bank99297 +99298 POINT(40.51913811654893 -73.39918639907397) bank99298 +99299 POINT(40.18506077223862 -74.84468789819861) bank99299 +99300 POINT(40.032014009485756 -74.50182321402781) bank99300 +99301 POINT(41.476774729570785 -73.58917637677314) bank99301 +99302 POINT(40.60016141659911 -74.04892136192197) bank99302 +99303 POINT(40.138540923325166 -73.18803756250107) bank99303 +99304 POINT(41.082260761381946 -73.14812643643761) bank99304 +99305 POINT(41.61872097876096 -74.00152903552373) bank99305 +99306 POINT(40.41883207753231 -74.85948510161853) bank99306 +99307 POINT(40.11025016479778 -74.49020771211956) bank99307 +99308 POINT(40.34219663303312 -74.4650645954959) bank99308 +99309 POINT(40.3048548008538 -74.15138811309942) bank99309 +99310 POINT(41.536639435857246 -74.10139853615388) bank99310 +99311 POINT(40.51008034403385 -74.78340297129964) bank99311 +99312 POINT(40.5979668857304 -74.20069499662335) bank99312 +99313 POINT(40.62040371090278 -73.10398177203179) bank99313 +99314 POINT(41.22486068853801 -73.04490093403973) bank99314 +99315 POINT(40.789875110009994 -73.04892305301806) bank99315 +99316 POINT(40.134747670657156 -74.10693631844492) bank99316 +99317 POINT(40.389782792744796 -73.46972236758552) bank99317 +99318 POINT(40.74455125339408 -74.41193082010982) bank99318 +99319 POINT(39.93121841337398 -74.5230640387421) bank99319 +99320 POINT(40.18662620800755 -74.79547692003398) bank99320 +99321 POINT(40.055631650608035 -73.36587553042095) bank99321 +99322 POINT(39.84204273983754 -74.84245386734882) bank99322 +99323 POINT(40.995398704476685 -73.98037804305174) bank99323 +99324 POINT(40.44850663097502 -73.82346033646735) bank99324 +99325 POINT(40.79486104970822 -74.53482086500074) bank99325 +99326 POINT(40.94556927954691 -74.00900737352356) bank99326 +99327 POINT(41.06777109948403 -73.65515141618975) bank99327 +99328 POINT(40.97228680791014 -74.5655447339702) bank99328 +99329 POINT(40.849602425649095 -73.29438244001348) bank99329 +99330 POINT(40.4251227035003 -73.9664750934014) bank99330 +99331 POINT(40.43285494999344 -73.0690548853016) bank99331 +99332 POINT(40.615595749463324 -74.3463371733211) bank99332 +99333 POINT(41.41475328585599 -73.54885804111112) bank99333 +99334 POINT(40.26489332495522 -73.47522187476429) bank99334 +99335 POINT(40.04180955847592 -74.44789342991554) bank99335 +99336 POINT(39.92977906144786 -74.95425674577969) bank99336 +99337 POINT(40.32893627259322 -73.04811685143112) bank99337 +99338 POINT(40.81912758771017 -74.5902004511955) bank99338 +99339 POINT(41.70567472782423 -74.21211386873127) bank99339 +99340 POINT(41.29023412491146 -74.27555536625552) bank99340 +99341 POINT(39.81225236722933 -73.74257416201966) bank99341 +99342 POINT(40.95717982761666 -74.37947004141131) bank99342 +99343 POINT(41.56190279328741 -73.09402057190383) bank99343 +99344 POINT(40.27692826633662 -74.36415142254941) bank99344 +99345 POINT(41.65065900417781 -74.54466823302315) bank99345 +99346 POINT(40.208921859056986 -74.96458563849247) bank99346 +99347 POINT(40.111215801883326 -74.76110046168773) bank99347 +99348 POINT(39.86385961442527 -74.85238451016741) bank99348 +99349 POINT(41.10766729036031 -73.04327958274372) bank99349 +99350 POINT(41.143624810132124 -73.33142938140233) bank99350 +99351 POINT(40.18280694985934 -73.59426291808124) bank99351 +99352 POINT(41.45767204893359 -74.23864277946839) bank99352 +99353 POINT(39.73049663742121 -73.79926259654047) bank99353 +99354 POINT(40.41563963765811 -74.13890779441057) bank99354 +99355 POINT(41.119539196695314 -73.661274860584) bank99355 +99356 POINT(40.74782007169954 -74.85986987796365) bank99356 +99357 POINT(40.25066739133419 -73.90868731030407) bank99357 +99358 POINT(41.41925251118928 -73.25386777253809) bank99358 +99359 POINT(41.591059456885155 -74.10033943707047) bank99359 +99360 POINT(41.347397502785874 -74.48776109728557) bank99360 +99361 POINT(40.80290899709293 -73.51200902323401) bank99361 +99362 POINT(40.84839659316302 -74.83481931666125) bank99362 +99363 POINT(40.2494845184377 -74.533801927896) bank99363 +99364 POINT(40.93184940467267 -74.08926057433382) bank99364 +99365 POINT(41.41661495262801 -73.54887885298315) bank99365 +99366 POINT(41.546009285141864 -73.78057979223269) bank99366 +99367 POINT(41.661218089855694 -73.06946262077233) bank99367 +99368 POINT(40.5983325313384 -73.43459069861366) bank99368 +99369 POINT(41.27238434489868 -73.59943415124835) bank99369 +99370 POINT(40.19062828555868 -73.51318403581604) bank99370 +99371 POINT(39.84186137651357 -74.88810235969163) bank99371 +99372 POINT(39.79197953287165 -74.27544334310328) bank99372 +99373 POINT(39.84671343266691 -74.42173986492878) bank99373 +99374 POINT(40.041299609856466 -74.97513380034535) bank99374 +99375 POINT(40.198630830935954 -73.22993091106325) bank99375 +99376 POINT(41.460403335027586 -73.63022925403858) bank99376 +99377 POINT(40.303609426463694 -73.79534189843751) bank99377 +99378 POINT(41.635101733286646 -74.89617645820448) bank99378 +99379 POINT(41.40268355871273 -73.01182123703181) bank99379 +99380 POINT(40.41659332398786 -74.13318545637186) bank99380 +99381 POINT(41.710468315865995 -73.87647136337628) bank99381 +99382 POINT(39.87284769017259 -74.40556694085248) bank99382 +99383 POINT(40.21330750016729 -73.78142647621843) bank99383 +99384 POINT(40.12535835654417 -73.85850806592987) bank99384 +99385 POINT(40.03099376210345 -74.2115154987317) bank99385 +99386 POINT(40.9154252917107 -74.20099294851566) bank99386 +99387 POINT(40.7779887695813 -74.07303868994272) bank99387 +99388 POINT(41.05696115456943 -74.6031203702619) bank99388 +99389 POINT(39.84733579484705 -73.0470758516778) bank99389 +99390 POINT(41.320132289111584 -73.61175728335054) bank99390 +99391 POINT(40.15557748846171 -73.04796742056001) bank99391 +99392 POINT(40.48955534589707 -74.3432332316917) bank99392 +99393 POINT(39.96225996320926 -74.07111549955209) bank99393 +99394 POINT(40.69795554207332 -73.2588945574719) bank99394 +99395 POINT(40.454172547969506 -73.37901061177637) bank99395 +99396 POINT(41.44344201039536 -74.7336958861167) bank99396 +99397 POINT(41.24081343230935 -74.91322284834494) bank99397 +99398 POINT(39.76255602780936 -73.97559452712355) bank99398 +99399 POINT(41.60309618898485 -73.20822786977638) bank99399 +99400 POINT(40.66206637668282 -74.47395289316057) bank99400 +99401 POINT(41.35967707386504 -73.75736311069294) bank99401 +99402 POINT(40.605238950078274 -74.19851445073449) bank99402 +99403 POINT(40.239966617275954 -73.99691251136619) bank99403 +99404 POINT(40.37862252440174 -74.506816453465) bank99404 +99405 POINT(40.40886088340606 -73.53050617960051) bank99405 +99406 POINT(41.42788584611277 -73.3025244195221) bank99406 +99407 POINT(39.82254278880599 -73.98049616350735) bank99407 +99408 POINT(40.93678216280658 -73.2816506721878) bank99408 +99409 POINT(40.72784663465082 -74.66770474084231) bank99409 +99410 POINT(40.38309574403028 -73.1647081005556) bank99410 +99411 POINT(39.82350326150435 -74.92269204884347) bank99411 +99412 POINT(40.47225740868757 -73.77988909744384) bank99412 +99413 POINT(39.85118371184258 -74.52626084407325) bank99413 +99414 POINT(40.702327098259474 -73.563274513905) bank99414 +99415 POINT(40.51779788178946 -74.0477798302106) bank99415 +99416 POINT(40.89920874512046 -73.05155227656228) bank99416 +99417 POINT(40.31012214997541 -73.06632956092278) bank99417 +99418 POINT(39.87241529672563 -73.48757095013951) bank99418 +99419 POINT(40.82384797121216 -74.43781812802305) bank99419 +99420 POINT(40.95047016567111 -73.57902894768984) bank99420 +99421 POINT(41.03999791369044 -74.90962080756438) bank99421 +99422 POINT(39.902452672370714 -74.57020824928185) bank99422 +99423 POINT(40.672440442205826 -73.78381558827948) bank99423 +99424 POINT(41.22558326956597 -74.6235666701001) bank99424 +99425 POINT(39.777824687284436 -73.16249534248334) bank99425 +99426 POINT(41.420046523923425 -73.2831742442758) bank99426 +99427 POINT(41.331584701657064 -74.58641273738034) bank99427 +99428 POINT(40.701642358436494 -74.26851250011757) bank99428 +99429 POINT(41.4880387647474 -73.44036438947374) bank99429 +99430 POINT(40.78959026890879 -74.41429860772148) bank99430 +99431 POINT(41.282221355802754 -74.69948523116987) bank99431 +99432 POINT(40.14139865100413 -74.26992743513041) bank99432 +99433 POINT(40.66529081675081 -74.64044759413949) bank99433 +99434 POINT(41.56080518427855 -74.15696753681121) bank99434 +99435 POINT(41.552594910376634 -74.87212108801265) bank99435 +99436 POINT(40.638233117205225 -74.17898770880957) bank99436 +99437 POINT(39.90714725711939 -74.71871745668355) bank99437 +99438 POINT(41.42981239297669 -74.921712667498) bank99438 +99439 POINT(40.93975271194277 -74.76140930176089) bank99439 +99440 POINT(40.185311469834176 -74.352606763441) bank99440 +99441 POINT(41.39518408058819 -74.47275557871068) bank99441 +99442 POINT(40.574162253148245 -73.28532133537229) bank99442 +99443 POINT(40.77114603184635 -74.1486030827462) bank99443 +99444 POINT(40.944853598568876 -74.20449676721354) bank99444 +99445 POINT(41.26266867899199 -74.16634598734983) bank99445 +99446 POINT(41.38704239897211 -73.41426562664395) bank99446 +99447 POINT(39.81209787205539 -73.2004054208419) bank99447 +99448 POINT(41.34553522517653 -73.58686069400338) bank99448 +99449 POINT(39.98237165901666 -73.13591302403479) bank99449 +99450 POINT(41.70443658259604 -74.38747231653615) bank99450 +99451 POINT(39.94710043388075 -74.67376800503006) bank99451 +99452 POINT(41.48385693098487 -74.20660401164177) bank99452 +99453 POINT(40.64227329492154 -74.05094317624125) bank99453 +99454 POINT(40.22218429466553 -73.79129367369113) bank99454 +99455 POINT(41.574658177723514 -73.72833362600228) bank99455 +99456 POINT(40.2153269504223 -74.0161872110305) bank99456 +99457 POINT(40.68799594491019 -74.85629774936153) bank99457 +99458 POINT(41.23702410956685 -74.16528179698136) bank99458 +99459 POINT(40.405837230635754 -74.61321257698718) bank99459 +99460 POINT(39.87083505937976 -74.31950077844134) bank99460 +99461 POINT(40.61632416546493 -73.5551006160987) bank99461 +99462 POINT(40.88907405291055 -73.73284438877046) bank99462 +99463 POINT(39.895364800571436 -73.73629042165499) bank99463 +99464 POINT(40.258886005784476 -73.5478010115231) bank99464 +99465 POINT(40.501127452705504 -73.62215004964094) bank99465 +99466 POINT(40.513244340546585 -74.99683193454379) bank99466 +99467 POINT(40.396871128001195 -74.88709585063306) bank99467 +99468 POINT(40.064478443018736 -73.21175225224484) bank99468 +99469 POINT(40.99025233991163 -74.64791815913708) bank99469 +99470 POINT(40.84294256825224 -73.71064921289845) bank99470 +99471 POINT(41.127363546458966 -73.90511762777768) bank99471 +99472 POINT(39.76613536892966 -74.84295072755367) bank99472 +99473 POINT(39.94034384391883 -73.55808901881525) bank99473 +99474 POINT(40.74296487794703 -74.4985214003504) bank99474 +99475 POINT(41.60798461017388 -74.3735960476875) bank99475 +99476 POINT(40.9806607877652 -73.62473343106002) bank99476 +99477 POINT(39.93051345933653 -73.57183619925182) bank99477 +99478 POINT(39.82710713900718 -73.96844034053628) bank99478 +99479 POINT(41.30484979794504 -73.8326519906818) bank99479 +99480 POINT(41.156472997563014 -73.71624397358654) bank99480 +99481 POINT(39.87650403050215 -74.17526025106007) bank99481 +99482 POINT(40.21460008996498 -74.78219214556039) bank99482 +99483 POINT(40.511710240486096 -73.14052700263056) bank99483 +99484 POINT(41.35009028072445 -73.53968474900768) bank99484 +99485 POINT(40.890914556819794 -74.28281421191986) bank99485 +99486 POINT(40.078040279850384 -74.46605692719638) bank99486 +99487 POINT(39.76885279036417 -73.65149364950967) bank99487 +99488 POINT(41.019420885517896 -74.94054964931964) bank99488 +99489 POINT(40.78985231354696 -73.76407934660743) bank99489 +99490 POINT(40.861940137790114 -73.58635422428685) bank99490 +99491 POINT(40.45450481768347 -73.57683962521492) bank99491 +99492 POINT(40.316065149215326 -73.59034366471495) bank99492 +99493 POINT(41.078900345871794 -73.83083225697546) bank99493 +99494 POINT(39.928912000456435 -73.05859278867906) bank99494 +99495 POINT(40.92220878455527 -74.8148482791178) bank99495 +99496 POINT(40.6106226067293 -73.31330099866075) bank99496 +99497 POINT(39.87686302693183 -74.8272530500102) bank99497 +99498 POINT(40.9980950336718 -74.66311220146797) bank99498 +99499 POINT(39.877551104932024 -74.58985420687904) bank99499 +99500 POINT(41.264086713427794 -73.88974825874206) bank99500 +99501 POINT(40.735965955079614 -73.13820944978382) bank99501 +99502 POINT(40.15288328485579 -74.7959938019258) bank99502 +99503 POINT(41.06009197998537 -74.01890029034737) bank99503 +99504 POINT(40.57153234279679 -73.46318809518827) bank99504 +99505 POINT(41.62776310042892 -73.52733852348395) bank99505 +99506 POINT(40.45901307569685 -73.33479732244287) bank99506 +99507 POINT(40.77427630560228 -74.28464858532952) bank99507 +99508 POINT(41.4901693655467 -73.96552612945598) bank99508 +99509 POINT(40.484174863957776 -74.85867347419091) bank99509 +99510 POINT(40.970646941996044 -73.38588637798061) bank99510 +99511 POINT(41.285176750153994 -73.36721923996446) bank99511 +99512 POINT(41.528476479406116 -73.03522483024601) bank99512 +99513 POINT(40.2294769879381 -73.77476859404665) bank99513 +99514 POINT(39.77588670168426 -73.25657508076009) bank99514 +99515 POINT(41.289564821683065 -74.47508105333254) bank99515 +99516 POINT(41.07241212195919 -73.16137555242204) bank99516 +99517 POINT(39.85351057465029 -74.83293411119517) bank99517 +99518 POINT(40.34084565870684 -74.39098192145165) bank99518 +99519 POINT(41.171699107224796 -73.98050206918566) bank99519 +99520 POINT(40.7438829414766 -74.95260986585356) bank99520 +99521 POINT(41.06754861636885 -73.11775217406854) bank99521 +99522 POINT(40.814044806926965 -74.82784717703272) bank99522 +99523 POINT(40.997534813366016 -73.44809768139231) bank99523 +99524 POINT(41.65181692518209 -74.3003042283553) bank99524 +99525 POINT(40.53109652647844 -73.32954038652426) bank99525 +99526 POINT(40.12954796080971 -74.19185404843127) bank99526 +99527 POINT(39.9439621913967 -73.24765465501825) bank99527 +99528 POINT(40.41314372421486 -73.42541244271564) bank99528 +99529 POINT(40.063363401826294 -73.805357381153) bank99529 +99530 POINT(41.37355520395323 -73.38407628248333) bank99530 +99531 POINT(40.0149110915662 -73.13731529269181) bank99531 +99532 POINT(40.21316053390061 -74.23430134265578) bank99532 +99533 POINT(39.875689255132485 -74.25854428692662) bank99533 +99534 POINT(40.342550611375685 -74.97410534337952) bank99534 +99535 POINT(40.77397249028209 -74.646387130736) bank99535 +99536 POINT(40.27218346502338 -73.26567776421022) bank99536 +99537 POINT(40.00089329638498 -73.1864164555646) bank99537 +99538 POINT(40.786250273623196 -73.17841075323182) bank99538 +99539 POINT(39.747716773789264 -73.20052197800031) bank99539 +99540 POINT(39.78444239752783 -74.77325277439296) bank99540 +99541 POINT(40.094165648573416 -74.88697928793114) bank99541 +99542 POINT(41.5655267520003 -73.33438820042214) bank99542 +99543 POINT(40.404373935370295 -74.83054528688936) bank99543 +99544 POINT(40.113646801371296 -73.59735170237924) bank99544 +99545 POINT(40.242393522989815 -74.41380115892954) bank99545 +99546 POINT(40.83904109553679 -73.66609917164897) bank99546 +99547 POINT(40.20453144259048 -74.34083460300629) bank99547 +99548 POINT(39.84974835068921 -74.2308724473548) bank99548 +99549 POINT(39.948730747292466 -74.05101573855477) bank99549 +99550 POINT(41.39831649257986 -74.50732010062067) bank99550 +99551 POINT(40.41359568419809 -74.23090032074049) bank99551 +99552 POINT(40.04469465605116 -74.5407529227384) bank99552 +99553 POINT(39.77698893555995 -73.56565714660718) bank99553 +99554 POINT(41.135069922599676 -73.99343022308865) bank99554 +99555 POINT(40.261650699582766 -74.28427587992233) bank99555 +99556 POINT(39.95729739782471 -74.66400216717477) bank99556 +99557 POINT(39.886312579956666 -73.07845526979827) bank99557 +99558 POINT(40.9416078153104 -74.92941591600456) bank99558 +99559 POINT(40.0458161407971 -74.80642198491984) bank99559 +99560 POINT(41.48665130727611 -74.04713484483149) bank99560 +99561 POINT(39.92429605761144 -74.65540230228524) bank99561 +99562 POINT(40.99914060826192 -74.3487442899147) bank99562 +99563 POINT(40.264875965969665 -73.37358706214063) bank99563 +99564 POINT(41.34585812468247 -74.79100923907419) bank99564 +99565 POINT(39.830585550755714 -73.17129042457951) bank99565 +99566 POINT(40.69854435200442 -73.7135697576173) bank99566 +99567 POINT(40.40675651703182 -73.60445776341135) bank99567 +99568 POINT(40.536549488452884 -74.38695549476962) bank99568 +99569 POINT(40.420229899592236 -73.5396140281667) bank99569 +99570 POINT(40.94904138438926 -73.3960565501206) bank99570 +99571 POINT(41.287617925689524 -74.70229481299687) bank99571 +99572 POINT(40.91963232275133 -74.272027395687) bank99572 +99573 POINT(39.81618756551555 -73.76580443909235) bank99573 +99574 POINT(40.67076336836311 -74.73514694235061) bank99574 +99575 POINT(40.09950076475618 -73.41829324798064) bank99575 +99576 POINT(41.22325895508338 -74.11921488206357) bank99576 +99577 POINT(39.85410073319427 -73.11147492946802) bank99577 +99578 POINT(41.621982988177535 -74.67688807045441) bank99578 +99579 POINT(40.80749596130568 -74.85350640949959) bank99579 +99580 POINT(40.672037786124505 -73.65264357959738) bank99580 +99581 POINT(40.066697194658964 -73.72658680135912) bank99581 +99582 POINT(40.610775149145155 -73.52655066438348) bank99582 +99583 POINT(41.10534763679658 -74.8298647778282) bank99583 +99584 POINT(40.538233969241766 -73.62853709880822) bank99584 +99585 POINT(40.58704953389669 -73.14854288373988) bank99585 +99586 POINT(39.962790199938915 -73.77224965422026) bank99586 +99587 POINT(41.2341800779176 -73.98716160252879) bank99587 +99588 POINT(40.610093015902414 -73.64526401481822) bank99588 +99589 POINT(41.49561163109227 -73.34130635821532) bank99589 +99590 POINT(41.077597568317174 -73.34078040297187) bank99590 +99591 POINT(40.41484585549709 -74.85509691779173) bank99591 +99592 POINT(41.65081961513069 -74.85233665806271) bank99592 +99593 POINT(40.0013607130432 -73.42590926193604) bank99593 +99594 POINT(41.55200855263213 -74.20225085448146) bank99594 +99595 POINT(40.5956312133705 -73.57580835910518) bank99595 +99596 POINT(40.943249423023296 -73.17189302212397) bank99596 +99597 POINT(41.52319142356563 -73.87821998483864) bank99597 +99598 POINT(39.982873613213116 -73.86838224701307) bank99598 +99599 POINT(40.68204882242807 -73.82744672409864) bank99599 +99600 POINT(39.79190218341319 -73.99845309652467) bank99600 +99601 POINT(39.9005445290853 -73.59277857154032) bank99601 +99602 POINT(41.58652498338722 -74.74604867705114) bank99602 +99603 POINT(40.96226183730254 -73.43722231310377) bank99603 +99604 POINT(40.756853991510525 -74.72669061714525) bank99604 +99605 POINT(41.33324625127243 -73.87037385859921) bank99605 +99606 POINT(41.34976109052021 -73.46133627765607) bank99606 +99607 POINT(41.40810993676251 -73.0070207162735) bank99607 +99608 POINT(40.93255614870687 -74.2299198056338) bank99608 +99609 POINT(40.45595721296719 -73.86960225789875) bank99609 +99610 POINT(40.927044469393685 -74.75520554768714) bank99610 +99611 POINT(40.95161928938847 -74.05185387288799) bank99611 +99612 POINT(40.29889161837951 -73.90197281740173) bank99612 +99613 POINT(41.00830619879921 -73.95698777298415) bank99613 +99614 POINT(41.255257066019965 -73.66759178493386) bank99614 +99615 POINT(41.13850204235644 -74.827518632046) bank99615 +99616 POINT(41.3585209131613 -73.22293276188427) bank99616 +99617 POINT(41.359494025239805 -73.23783994220143) bank99617 +99618 POINT(40.374618868802514 -74.78459922905249) bank99618 +99619 POINT(39.85211163450246 -74.99851107126325) bank99619 +99620 POINT(40.68634383486625 -73.52352814349493) bank99620 +99621 POINT(41.04717316461886 -74.77935932132704) bank99621 +99622 POINT(41.148983744461134 -73.98419139115506) bank99622 +99623 POINT(41.448287405095435 -74.91665017330202) bank99623 +99624 POINT(41.08124588259004 -74.52852070170137) bank99624 +99625 POINT(40.22113836824866 -74.11474221084909) bank99625 +99626 POINT(41.21157404579621 -74.0560766876849) bank99626 +99627 POINT(41.07917521811533 -74.2637626469851) bank99627 +99628 POINT(40.05606669188435 -73.7500142527489) bank99628 +99629 POINT(40.804038622885834 -74.62030639034423) bank99629 +99630 POINT(41.25131250874994 -73.27262048497954) bank99630 +99631 POINT(39.91055370910434 -73.2576210581154) bank99631 +99632 POINT(41.15980163379637 -73.58461175595018) bank99632 +99633 POINT(41.544814935596705 -74.79780498951085) bank99633 +99634 POINT(41.58307787099631 -73.65082460496477) bank99634 +99635 POINT(41.38851087155442 -74.26969320240015) bank99635 +99636 POINT(40.01687635681637 -74.93174659814319) bank99636 +99637 POINT(41.349470610725554 -73.40074043010314) bank99637 +99638 POINT(41.091493822082086 -74.0507824043679) bank99638 +99639 POINT(40.82313797970064 -73.28732717098566) bank99639 +99640 POINT(41.60606171709628 -74.97444171804676) bank99640 +99641 POINT(40.09728412967372 -73.04565253959028) bank99641 +99642 POINT(41.35451379701501 -74.14785590376576) bank99642 +99643 POINT(40.45368837166551 -74.69819496077369) bank99643 +99644 POINT(39.99901547217958 -73.10162419385895) bank99644 +99645 POINT(40.20510436732271 -73.16152478640996) bank99645 +99646 POINT(39.81320107795903 -73.50064818535404) bank99646 +99647 POINT(40.57043366625179 -74.6029143718946) bank99647 +99648 POINT(41.43784027713181 -73.50251385607928) bank99648 +99649 POINT(41.50192685965747 -73.52545382247172) bank99649 +99650 POINT(41.24532470321139 -73.0302996746622) bank99650 +99651 POINT(40.34194667925217 -73.14438862095876) bank99651 +99652 POINT(41.66205682687875 -74.05347591226119) bank99652 +99653 POINT(41.312984523126644 -73.52514057664656) bank99653 +99654 POINT(40.88055632541853 -74.34253844978994) bank99654 +99655 POINT(39.94256547654469 -74.8907241460129) bank99655 +99656 POINT(40.53629539737563 -73.88797430977144) bank99656 +99657 POINT(40.79449914104493 -73.48671637982197) bank99657 +99658 POINT(39.99822961306894 -74.97327050525676) bank99658 +99659 POINT(41.10048549596145 -74.47153223692875) bank99659 +99660 POINT(40.74404692286959 -74.02408900360216) bank99660 +99661 POINT(39.99762533703843 -74.20953368144417) bank99661 +99662 POINT(40.70810610710562 -74.04090059452413) bank99662 +99663 POINT(41.53423136996147 -74.29280227025676) bank99663 +99664 POINT(40.62381376369564 -73.40376779770877) bank99664 +99665 POINT(40.529358560514815 -73.10948162221574) bank99665 +99666 POINT(39.727632683451205 -74.63659837554634) bank99666 +99667 POINT(40.04680387609586 -73.49898042737512) bank99667 +99668 POINT(41.00536910977961 -74.80024882254585) bank99668 +99669 POINT(39.80584172080676 -74.28311701955336) bank99669 +99670 POINT(41.34086008743921 -74.83285593035676) bank99670 +99671 POINT(40.49729502117035 -73.94312476884264) bank99671 +99672 POINT(40.725422295872505 -74.12538589649476) bank99672 +99673 POINT(40.47180429712172 -74.76219845692228) bank99673 +99674 POINT(40.5989940823789 -74.11336050748078) bank99674 +99675 POINT(40.00803039078977 -74.96565364571916) bank99675 +99676 POINT(40.71640543420865 -74.6074985084498) bank99676 +99677 POINT(39.78178454010515 -74.03582802372463) bank99677 +99678 POINT(40.958195500051296 -73.41278856908623) bank99678 +99679 POINT(40.44744942826672 -73.02966064845944) bank99679 +99680 POINT(39.877192764790436 -73.56467075748554) bank99680 +99681 POINT(40.38752863114548 -74.44349153496994) bank99681 +99682 POINT(41.14441137356632 -73.4692979003428) bank99682 +99683 POINT(40.538930217709954 -73.29064250614496) bank99683 +99684 POINT(41.237413620027716 -73.946219852831) bank99684 +99685 POINT(41.344259751597434 -73.28517084382801) bank99685 +99686 POINT(40.32465697640497 -73.58628085641652) bank99686 +99687 POINT(40.042202223923404 -73.82913597781813) bank99687 +99688 POINT(41.65260850438927 -73.53882409167467) bank99688 +99689 POINT(39.877853971841404 -74.18494825062514) bank99689 +99690 POINT(41.565275199635906 -73.49560569786905) bank99690 +99691 POINT(40.17551358370122 -73.19509585459707) bank99691 +99692 POINT(40.83824071837827 -74.859922915125) bank99692 +99693 POINT(40.896861575792904 -74.88272115280263) bank99693 +99694 POINT(39.93518998427714 -74.79271350953938) bank99694 +99695 POINT(41.4876834947002 -73.95202104236992) bank99695 +99696 POINT(40.31888440999794 -73.1606330396823) bank99696 +99697 POINT(40.53427824740707 -73.78610243723327) bank99697 +99698 POINT(40.65907459000129 -73.07448620332241) bank99698 +99699 POINT(41.56017882442694 -74.63178644252545) bank99699 +99700 POINT(40.096682343269286 -74.05048980021483) bank99700 +99701 POINT(39.84851051386826 -73.52324126835029) bank99701 +99702 POINT(41.05816473606772 -73.23343099176572) bank99702 +99703 POINT(40.223907885268446 -74.77407446510605) bank99703 +99704 POINT(41.276249701404716 -73.7350785138336) bank99704 +99705 POINT(40.92634384563649 -73.34129014297636) bank99705 +99706 POINT(39.75392225424473 -74.42340882559424) bank99706 +99707 POINT(40.388155897871705 -74.56748553625187) bank99707 +99708 POINT(41.391474326274974 -74.90979364677119) bank99708 +99709 POINT(39.88455140221172 -73.11495449402167) bank99709 +99710 POINT(40.432647761464466 -73.23798119614462) bank99710 +99711 POINT(40.591743554838416 -74.75083091995563) bank99711 +99712 POINT(39.92413600281693 -74.3298327729394) bank99712 +99713 POINT(39.765533313020214 -73.1636175440491) bank99713 +99714 POINT(41.40871946691974 -73.16284737333017) bank99714 +99715 POINT(41.327866954629386 -73.19625092165523) bank99715 +99716 POINT(40.67988836649027 -74.72681570781467) bank99716 +99717 POINT(41.55941338971093 -73.85948814464889) bank99717 +99718 POINT(41.334696480805015 -73.5379136706258) bank99718 +99719 POINT(40.26058927321925 -74.50372485294886) bank99719 +99720 POINT(41.408558477266006 -73.4948796667) bank99720 +99721 POINT(40.34534443257244 -73.76259253335287) bank99721 +99722 POINT(40.41289965210158 -74.42140635218858) bank99722 +99723 POINT(40.1532576091985 -73.41444775792161) bank99723 +99724 POINT(41.30386386498609 -74.91355735862362) bank99724 +99725 POINT(40.09074152945643 -73.24617832764028) bank99725 +99726 POINT(40.222221443472634 -74.6944680765639) bank99726 +99727 POINT(40.49974389685745 -74.52095332744473) bank99727 +99728 POINT(40.74552995009715 -73.13384020275251) bank99728 +99729 POINT(41.0269666278368 -73.62675356660716) bank99729 +99730 POINT(39.93352839680383 -74.52489650464975) bank99730 +99731 POINT(41.119661142616316 -74.62459097493091) bank99731 +99732 POINT(41.67486251645386 -74.02990780845077) bank99732 +99733 POINT(39.891286655320314 -73.2470041766349) bank99733 +99734 POINT(40.5476923950372 -73.36121271063766) bank99734 +99735 POINT(41.03226777326884 -73.29113512677986) bank99735 +99736 POINT(40.83728657164972 -74.63743493753796) bank99736 +99737 POINT(41.29840285718566 -73.1978282032822) bank99737 +99738 POINT(41.2697716215088 -73.8625795054559) bank99738 +99739 POINT(40.86358939770402 -73.27806853789218) bank99739 +99740 POINT(41.313273055317396 -74.95182233341596) bank99740 +99741 POINT(41.17111862468429 -74.70032974597967) bank99741 +99742 POINT(40.059020536455726 -74.06814267127494) bank99742 +99743 POINT(40.55924509577417 -74.42549964495859) bank99743 +99744 POINT(39.719124155306275 -73.76490098988161) bank99744 +99745 POINT(40.78015857501769 -74.57653583535381) bank99745 +99746 POINT(39.770736962222934 -73.65325033174813) bank99746 +99747 POINT(41.396064575098556 -74.82699016874815) bank99747 +99748 POINT(40.97435437224961 -74.97186381050979) bank99748 +99749 POINT(40.27001689976307 -73.28053511767759) bank99749 +99750 POINT(40.420238403982374 -73.2346040290369) bank99750 +99751 POINT(41.46512083454438 -73.6188662852777) bank99751 +99752 POINT(40.072284692069374 -73.03840911556884) bank99752 +99753 POINT(40.308300328249175 -74.73604085253082) bank99753 +99754 POINT(41.528831879206614 -74.67468107438317) bank99754 +99755 POINT(39.792718125066 -73.83700098914196) bank99755 +99756 POINT(40.82452188296283 -73.8650530482808) bank99756 +99757 POINT(39.83420194076694 -74.01095547403975) bank99757 +99758 POINT(40.322018194673475 -74.48879649615252) bank99758 +99759 POINT(40.827072266087065 -74.22172318548441) bank99759 +99760 POINT(40.68984096864897 -73.63687139034298) bank99760 +99761 POINT(40.55820122202878 -73.51794866283814) bank99761 +99762 POINT(40.74765153041893 -73.70193094486999) bank99762 +99763 POINT(40.40571212617918 -73.83052000403895) bank99763 +99764 POINT(40.99828523288683 -73.84892128320368) bank99764 +99765 POINT(41.58180405330978 -73.14260085461288) bank99765 +99766 POINT(40.141328324992415 -73.71313014818392) bank99766 +99767 POINT(40.69191195714926 -74.48695472250992) bank99767 +99768 POINT(39.84836142774931 -74.79424869951941) bank99768 +99769 POINT(40.28746890357388 -74.12439656270527) bank99769 +99770 POINT(41.07219252590535 -73.43790432351602) bank99770 +99771 POINT(40.5233769325161 -73.40838110627712) bank99771 +99772 POINT(41.4192857225404 -74.50500825468212) bank99772 +99773 POINT(40.02125465769079 -74.42552513303607) bank99773 +99774 POINT(41.515862993355974 -73.04598289747693) bank99774 +99775 POINT(40.98360858831855 -73.1057613380231) bank99775 +99776 POINT(39.87428315437325 -74.63515966070598) bank99776 +99777 POINT(39.84175879364483 -74.19261651687036) bank99777 +99778 POINT(40.935143141849245 -73.57273595078635) bank99778 +99779 POINT(41.587914779471895 -74.70632007368927) bank99779 +99780 POINT(41.28858447944785 -74.77759780264084) bank99780 +99781 POINT(39.79104078668377 -73.10874354572302) bank99781 +99782 POINT(40.16403299748717 -74.50582247317108) bank99782 +99783 POINT(41.12659222663412 -73.50485622743204) bank99783 +99784 POINT(41.27802509429174 -74.5782416084075) bank99784 +99785 POINT(40.602484416210196 -74.19485140704859) bank99785 +99786 POINT(40.84517056321885 -74.91960154216267) bank99786 +99787 POINT(40.36711307828624 -73.40203759609692) bank99787 +99788 POINT(41.421828121076175 -74.69420066763779) bank99788 +99789 POINT(41.08403411370241 -73.48175244445939) bank99789 +99790 POINT(41.17260385669279 -73.5791098811191) bank99790 +99791 POINT(40.69467419777745 -73.2528871226698) bank99791 +99792 POINT(41.265822500775165 -74.03396981962224) bank99792 +99793 POINT(40.10124164653241 -73.17797096433068) bank99793 +99794 POINT(39.88181243255096 -74.95595157420233) bank99794 +99795 POINT(41.24544046887984 -74.21351137570917) bank99795 +99796 POINT(41.38497741148479 -73.70324843829776) bank99796 +99797 POINT(40.628280866432526 -73.12726724541996) bank99797 +99798 POINT(40.242232953566464 -73.38373684409174) bank99798 +99799 POINT(41.192817956984854 -74.2842718169039) bank99799 +99800 POINT(40.97546119571723 -74.54896206286939) bank99800 +99801 POINT(40.87312949690037 -74.11459394726323) bank99801 +99802 POINT(40.6561113225462 -73.91046996330488) bank99802 +99803 POINT(40.825544564718356 -73.84566048910344) bank99803 +99804 POINT(40.21504896728385 -73.93557281274117) bank99804 +99805 POINT(40.249137929927215 -74.6063225746343) bank99805 +99806 POINT(40.55810726971525 -74.63202655280655) bank99806 +99807 POINT(41.47833698889275 -74.70722659188678) bank99807 +99808 POINT(41.48037173462804 -73.58370670253427) bank99808 +99809 POINT(40.900905038680364 -74.87350657927685) bank99809 +99810 POINT(40.0255428485333 -73.67060423642882) bank99810 +99811 POINT(39.794807663299444 -74.33269092998455) bank99811 +99812 POINT(41.1199832539406 -74.76507097744644) bank99812 +99813 POINT(39.81793619249751 -73.76478610423379) bank99813 +99814 POINT(39.84728120111573 -73.22462457928373) bank99814 +99815 POINT(39.98843260304449 -74.5136533393086) bank99815 +99816 POINT(40.433366355954625 -74.6825184280738) bank99816 +99817 POINT(40.0881282566097 -74.88535428597812) bank99817 +99818 POINT(41.34471014266214 -73.3454735943041) bank99818 +99819 POINT(39.941362861649466 -73.3153889304899) bank99819 +99820 POINT(40.57387786809639 -73.0130658439161) bank99820 +99821 POINT(40.94831016648802 -73.9019704968493) bank99821 +99822 POINT(40.43800818956793 -73.52123596264539) bank99822 +99823 POINT(41.42751861107301 -74.87890415856133) bank99823 +99824 POINT(41.459295682212606 -74.46952492435139) bank99824 +99825 POINT(41.03468957641344 -74.94657390699942) bank99825 +99826 POINT(40.43619253130106 -73.07947526215479) bank99826 +99827 POINT(41.698783783284135 -73.882797872627) bank99827 +99828 POINT(40.762828033986786 -74.59461494686657) bank99828 +99829 POINT(41.41259897723826 -73.53875072645579) bank99829 +99830 POINT(39.80794514628293 -74.09495263023565) bank99830 +99831 POINT(40.486501680029974 -73.49518440397274) bank99831 +99832 POINT(40.370903683497275 -74.77068203479674) bank99832 +99833 POINT(40.233842514842124 -73.07928901186808) bank99833 +99834 POINT(41.380877452330644 -74.62217589551672) bank99834 +99835 POINT(39.99010055163053 -73.03317426724963) bank99835 +99836 POINT(40.87509333367177 -73.6840330168766) bank99836 +99837 POINT(40.012966018840174 -73.95695043537211) bank99837 +99838 POINT(41.5315478522307 -73.09537086350997) bank99838 +99839 POINT(41.69151662472468 -74.74925498569178) bank99839 +99840 POINT(40.86156686530661 -74.44083619354893) bank99840 +99841 POINT(40.93986827583895 -73.23004453659583) bank99841 +99842 POINT(41.57166776223785 -73.03897001128009) bank99842 +99843 POINT(40.92764934940827 -74.81824117970648) bank99843 +99844 POINT(41.11486862968102 -73.4141836525394) bank99844 +99845 POINT(39.855293769872006 -74.3184513166854) bank99845 +99846 POINT(40.812253326092865 -74.71052080056889) bank99846 +99847 POINT(39.85980442644599 -74.60617402109509) bank99847 +99848 POINT(40.92861372068565 -74.6770382531747) bank99848 +99849 POINT(40.36367314189683 -74.83152708781317) bank99849 +99850 POINT(40.099905754252276 -73.75386364549257) bank99850 +99851 POINT(39.96304928066931 -74.86426802144089) bank99851 +99852 POINT(40.69332440757546 -73.73551635685375) bank99852 +99853 POINT(41.453493484428456 -73.27410140037657) bank99853 +99854 POINT(40.41552258056349 -74.34430845402788) bank99854 +99855 POINT(41.020824284913665 -73.45528768059332) bank99855 +99856 POINT(40.69571039847148 -74.19904852761837) bank99856 +99857 POINT(41.57716162239041 -73.54730889324576) bank99857 +99858 POINT(40.88048470220807 -74.49342314316972) bank99858 +99859 POINT(41.62575194884055 -74.81685831352632) bank99859 +99860 POINT(41.292790638103654 -74.57540617246946) bank99860 +99861 POINT(39.921096511351415 -74.3704139123308) bank99861 +99862 POINT(40.99257022816168 -73.85296284111038) bank99862 +99863 POINT(40.977242760884444 -74.72942967145076) bank99863 +99864 POINT(40.680165870253205 -74.0619617291351) bank99864 +99865 POINT(40.36295966942948 -73.03559291824564) bank99865 +99866 POINT(40.13174893508432 -74.08090825317552) bank99866 +99867 POINT(41.2531285853074 -74.3060040614187) bank99867 +99868 POINT(41.155089620288784 -73.87307912056016) bank99868 +99869 POINT(40.77330788057323 -74.91777667392859) bank99869 +99870 POINT(40.93015009292411 -74.21782885803802) bank99870 +99871 POINT(40.180228565123286 -74.30633343951875) bank99871 +99872 POINT(39.724919826907666 -74.96240965548122) bank99872 +99873 POINT(40.09339126742276 -73.14300429405203) bank99873 +99874 POINT(41.280799607686404 -73.03471766621155) bank99874 +99875 POINT(39.72260157115876 -74.12138092170774) bank99875 +99876 POINT(40.162986342647876 -74.31592446757868) bank99876 +99877 POINT(41.71164115014737 -73.79495165591321) bank99877 +99878 POINT(39.893649039291134 -73.24007281133224) bank99878 +99879 POINT(40.89065620786683 -73.59847389570362) bank99879 +99880 POINT(39.71563829211701 -74.3545154306305) bank99880 +99881 POINT(41.64947101544249 -74.4402843565273) bank99881 +99882 POINT(39.88408319960437 -73.72808274088243) bank99882 +99883 POINT(40.104877850739925 -74.79363725072895) bank99883 +99884 POINT(40.20276756210611 -73.55217688867533) bank99884 +99885 POINT(40.08894033958048 -73.99578009396718) bank99885 +99886 POINT(41.54101075937573 -74.67032232151672) bank99886 +99887 POINT(41.25925902135832 -73.33299131888207) bank99887 +99888 POINT(41.16952842971482 -74.40761319287238) bank99888 +99889 POINT(40.40961683985445 -74.67854684898853) bank99889 +99890 POINT(40.21183743944153 -74.04622942199325) bank99890 +99891 POINT(39.784915333600736 -74.61540305772853) bank99891 +99892 POINT(41.43138351893029 -73.01630341736708) bank99892 +99893 POINT(40.78852713111547 -74.3413322129701) bank99893 +99894 POINT(41.555965240389284 -74.36229997414854) bank99894 +99895 POINT(40.420986542663925 -73.00837228586235) bank99895 +99896 POINT(41.11199399153275 -73.31101775382703) bank99896 +99897 POINT(40.07832009456875 -73.99989826025859) bank99897 +99898 POINT(40.63774651687773 -73.25574257534993) bank99898 +99899 POINT(40.502739121069204 -73.12179679857655) bank99899 +99900 POINT(41.45099996114034 -74.72213999260684) bank99900 +99901 POINT(41.17691451076335 -74.18609401386628) bank99901 +99902 POINT(40.639959545743814 -74.73466671374752) bank99902 +99903 POINT(40.01134004752739 -74.38283261256728) bank99903 +99904 POINT(40.787820214438604 -73.71884169238177) bank99904 +99905 POINT(40.839204633096365 -73.05081709481026) bank99905 +99906 POINT(39.86422025489428 -73.16344213696789) bank99906 +99907 POINT(40.667323591284486 -73.37010444997854) bank99907 +99908 POINT(41.534152262488675 -74.00695170675515) bank99908 +99909 POINT(40.83108329864587 -73.89814818859733) bank99909 +99910 POINT(41.473727987438586 -74.95830445061497) bank99910 +99911 POINT(41.386315314490666 -74.22652945277869) bank99911 +99912 POINT(40.82731872470544 -73.38990675091664) bank99912 +99913 POINT(41.63188012711599 -73.7303890955114) bank99913 +99914 POINT(41.65512204666743 -74.8728862370654) bank99914 +99915 POINT(40.73204494025977 -74.5986256871552) bank99915 +99916 POINT(39.932646831469576 -74.22871335917002) bank99916 +99917 POINT(40.884996061208625 -73.53358331370258) bank99917 +99918 POINT(40.87657243964014 -74.70085166122803) bank99918 +99919 POINT(41.156204583623506 -73.92505078637751) bank99919 +99920 POINT(39.81181219894851 -74.6186929330442) bank99920 +99921 POINT(41.262134448275944 -74.81875942269733) bank99921 +99922 POINT(40.98613799398625 -73.84184777596404) bank99922 +99923 POINT(40.97761373942353 -74.970120363936) bank99923 +99924 POINT(41.36279685297651 -73.37030094001695) bank99924 +99925 POINT(40.731632246473275 -73.24431899022368) bank99925 +99926 POINT(39.719239198608456 -73.67068895050863) bank99926 +99927 POINT(40.158538699232814 -73.33823177330022) bank99927 +99928 POINT(40.47227923156925 -73.99191775797682) bank99928 +99929 POINT(40.90088265561827 -74.96764611593603) bank99929 +99930 POINT(41.25244517369331 -74.35721940003813) bank99930 +99931 POINT(39.88984010066401 -73.6413028964686) bank99931 +99932 POINT(40.85811153476916 -73.22643341997606) bank99932 +99933 POINT(40.40240525351544 -74.0893150943775) bank99933 +99934 POINT(41.47868022527709 -73.68043293198376) bank99934 +99935 POINT(40.818915302401365 -74.38360079409864) bank99935 +99936 POINT(40.62528346573787 -74.44724519191278) bank99936 +99937 POINT(41.05767613105791 -74.26785833858378) bank99937 +99938 POINT(40.36861974167863 -73.55969690203372) bank99938 +99939 POINT(40.66496157254743 -73.72788567608839) bank99939 +99940 POINT(39.73737829394553 -74.99609151493654) bank99940 +99941 POINT(40.35735203932357 -74.83495527052423) bank99941 +99942 POINT(39.73144011944066 -74.06081681173292) bank99942 +99943 POINT(39.756610133577844 -74.66463177783854) bank99943 +99944 POINT(40.99533955997497 -74.4276401569896) bank99944 +99945 POINT(41.70943942879376 -74.30390628232509) bank99945 +99946 POINT(39.927932268535166 -73.35388350629458) bank99946 +99947 POINT(40.60042055295365 -73.81613863563926) bank99947 +99948 POINT(40.55244484017721 -74.02650540598808) bank99948 +99949 POINT(40.72320073963583 -73.16202539135314) bank99949 +99950 POINT(41.18084147951895 -73.12704721931594) bank99950 +99951 POINT(41.6244138476703 -74.33092516270992) bank99951 +99952 POINT(40.74739633163976 -73.39313983029692) bank99952 +99953 POINT(41.25320710584152 -74.66731549087682) bank99953 +99954 POINT(41.564365036211015 -74.5760253826341) bank99954 +99955 POINT(41.235324835231495 -74.79429513012425) bank99955 +99956 POINT(40.26304391474087 -74.52973777527698) bank99956 +99957 POINT(40.165667089970604 -73.71327306643475) bank99957 +99958 POINT(40.82251577347392 -74.6991297292024) bank99958 +99959 POINT(40.995354057656805 -73.21486251210759) bank99959 +99960 POINT(40.260613872180286 -73.9511248151401) bank99960 +99961 POINT(40.54980745113592 -73.0629113852445) bank99961 +99962 POINT(40.59747455663457 -73.14601083700454) bank99962 +99963 POINT(40.65862831336829 -73.88022075437593) bank99963 +99964 POINT(39.86386149946913 -74.51720743655399) bank99964 +99965 POINT(40.40423476790859 -74.97606933227983) bank99965 +99966 POINT(41.14395976343243 -74.80631970753375) bank99966 +99967 POINT(39.93762117264911 -73.59116855999788) bank99967 +99968 POINT(40.160317462186676 -74.95872986201701) bank99968 +99969 POINT(41.03542631428132 -74.41686103029373) bank99969 +99970 POINT(39.91253612537736 -73.56310472927075) bank99970 +99971 POINT(40.97775850633283 -73.40523850626523) bank99971 +99972 POINT(40.46152844043078 -73.56377536135224) bank99972 +99973 POINT(41.442638265951594 -73.15688705298433) bank99973 +99974 POINT(41.657650506941515 -73.86214780901342) bank99974 +99975 POINT(41.51930597404696 -74.94340249018147) bank99975 +99976 POINT(40.175842359143296 -74.4767026554321) bank99976 +99977 POINT(40.4686211164695 -73.72435232161759) bank99977 +99978 POINT(41.651732536662806 -73.21945768695656) bank99978 +99979 POINT(39.880958626171406 -73.4139807250971) bank99979 +99980 POINT(41.580614964383386 -74.33864468430995) bank99980 +99981 POINT(40.560346674263705 -73.48323715824131) bank99981 +99982 POINT(40.699696333338636 -74.31932040438058) bank99982 +99983 POINT(40.12569254277724 -74.92803011681056) bank99983 +99984 POINT(39.85992384672332 -73.24560617358115) bank99984 +99985 POINT(40.14611929931353 -73.93041807895902) bank99985 +99986 POINT(40.871300641160644 -73.92407431197397) bank99986 +99987 POINT(41.26387693332359 -74.88432293534963) bank99987 +99988 POINT(41.300118013813204 -74.45386206936891) bank99988 +99989 POINT(40.34086209713241 -73.1949707992193) bank99989 +99990 POINT(41.1665162047343 -73.05191556940673) bank99990 +99991 POINT(40.19896629906879 -73.10341969943983) bank99991 +99992 POINT(41.02393344113755 -74.53534413115699) bank99992 +99993 POINT(40.142789722230845 -74.963976666639) bank99993 +99994 POINT(40.14090643006112 -73.45167342648266) bank99994 +99995 POINT(41.36684781450654 -73.8060546703381) bank99995 +99996 POINT(40.974250476422526 -73.60122014435001) bank99996 +99997 POINT(40.177030552833465 -73.75296710057953) bank99997 +99998 POINT(40.251606054057824 -74.2366321160017) bank99998 +99999 POINT(41.70968400023791 -73.27397501759233) bank99999 +100000 POINT(40.06153742138935 -73.54751782432474) bank100000 +100001 POINT(41.20444344978369 -74.22613738880489) bank100001 +100002 POINT(41.21507233729217 -73.39536723351618) bank100002 +100003 POINT(40.60278533264826 -73.41501124588062) bank100003 +100004 POINT(40.96781319602984 -74.46882619730437) bank100004 +100005 POINT(41.20027465789161 -73.77855962651094) bank100005 +100006 POINT(40.959963791670155 -73.71523800366708) bank100006 +100007 POINT(39.99465874975504 -73.07836884654134) bank100007 +100008 POINT(40.573247667530055 -74.65751725615512) bank100008 +100009 POINT(41.522583572727115 -74.3250890784489) bank100009 +100010 POINT(47.269117314835206 -123.01114218780242) bank100010 +100011 POINT(47.05864976429022 -122.53632471694144) bank100011 +100012 POINT(47.033478358644786 -122.78565204027292) bank100012 +100013 POINT(47.23387010750525 -123.27586173141198) bank100013 +100014 POINT(48.22706090044882 -122.20450157191546) bank100014 +100015 POINT(47.85839421705628 -122.67346209140042) bank100015 +100016 POINT(47.373052044601216 -122.59532628603655) bank100016 +100017 POINT(48.45617879759831 -122.70943806476836) bank100017 +100018 POINT(47.149387416959534 -122.91371490729858) bank100018 +100019 POINT(47.42536137025007 -121.38610428355216) bank100019 +100020 POINT(47.62379854443945 -122.99870560177023) bank100020 +100021 POINT(47.39619487201478 -123.2549431418391) bank100021 +100022 POINT(48.23122753717395 -122.66178580682057) bank100022 +100023 POINT(48.06312709933675 -121.38321335113639) bank100023 +100024 POINT(48.26006894168785 -122.34863569733398) bank100024 +100025 POINT(46.68634419007067 -121.51154543068189) bank100025 +100026 POINT(47.67679422445991 -122.57785392685233) bank100026 +100027 POINT(47.110926875802036 -123.22640744434356) bank100027 +100028 POINT(48.110981346877836 -122.41661604370225) bank100028 +100029 POINT(46.795285490595944 -123.30889430962335) bank100029 +100030 POINT(48.243935137005685 -122.55669249410916) bank100030 +100031 POINT(48.569991691087154 -122.55291920727632) bank100031 +100032 POINT(47.89082644725466 -122.8847240880027) bank100032 +100033 POINT(46.916136961837715 -122.59443321269832) bank100033 +100034 POINT(46.708835502095205 -123.24340692166196) bank100034 +100035 POINT(47.71273396455742 -121.68778636549709) bank100035 +100036 POINT(48.22980778569165 -121.6683283840641) bank100036 +100037 POINT(46.86608341627442 -122.28063483985815) bank100037 +100038 POINT(48.287918199826365 -122.1622496748154) bank100038 +100039 POINT(46.9787886574868 -122.0373739717592) bank100039 +100040 POINT(47.09705690945493 -123.23274846168711) bank100040 +100041 POINT(47.68979419125097 -122.92799396580712) bank100041 +100042 POINT(47.27415889749442 -121.91052212957615) bank100042 +100043 POINT(47.85245025954643 -123.23377010453407) bank100043 +100044 POINT(46.898072938885186 -122.50737962749595) bank100044 +100045 POINT(47.41699712031226 -122.78882273006367) bank100045 +100046 POINT(47.67551473438241 -123.22984680313498) bank100046 +100047 POINT(48.36590913355194 -121.45019539029322) bank100047 +100048 POINT(46.8722682578347 -122.33809478716) bank100048 +100049 POINT(46.81583330335047 -122.47786245839872) bank100049 +100050 POINT(48.24528646487581 -121.60159543594202) bank100050 +100051 POINT(46.935050346086946 -123.29222164716197) bank100051 +100052 POINT(48.42113134905898 -121.58327883238613) bank100052 +100053 POINT(46.9430814723098 -122.60217216468345) bank100053 +100054 POINT(47.85352922528298 -123.00464755435057) bank100054 +100055 POINT(47.73912198406073 -122.01508673374761) bank100055 +100056 POINT(48.429220409018555 -122.89261302887422) bank100056 +100057 POINT(46.90987506794671 -121.74820582149643) bank100057 +100058 POINT(48.25681696934918 -121.41409691461871) bank100058 +100059 POINT(47.33135954085284 -121.3540457632806) bank100059 +100060 POINT(46.96382935038384 -122.55321609611093) bank100060 +100061 POINT(46.780005631124084 -122.19613495314378) bank100061 +100062 POINT(46.86032802071442 -122.3627625643425) bank100062 +100063 POINT(47.9631864641668 -123.1338921643436) bank100063 +100064 POINT(47.98181049498351 -122.06256740101165) bank100064 +100065 POINT(46.75434858194143 -122.63749878961457) bank100065 +100066 POINT(46.798969843005324 -121.82184614133742) bank100066 +100067 POINT(48.60022014523074 -123.07752959162495) bank100067 +100068 POINT(48.1555161602584 -122.52050081881703) bank100068 +100069 POINT(47.330018922466294 -121.3833260374213) bank100069 +100070 POINT(46.635812354386786 -121.5283368777283) bank100070 +100071 POINT(47.42418326852177 -121.65530314939662) bank100071 +100072 POINT(47.50065473666862 -122.88345769016244) bank100072 +100073 POINT(47.67014718571072 -123.29897314477175) bank100073 +100074 POINT(46.63903578730454 -121.46743797860513) bank100074 +100075 POINT(48.41520276993123 -121.45015641282126) bank100075 +100076 POINT(46.84618118026507 -122.40661548606651) bank100076 +100077 POINT(47.15135194414404 -123.13976355447197) bank100077 +100078 POINT(48.552894741765066 -122.97891053347327) bank100078 +100079 POINT(47.84575396601389 -123.09655411749723) bank100079 +100080 POINT(46.929873423462304 -122.49491571359157) bank100080 +100081 POINT(47.22337334182362 -123.21590774481723) bank100081 +100082 POINT(48.3503058797089 -122.57032486590339) bank100082 +100083 POINT(47.81950794127442 -122.68914905856938) bank100083 +100084 POINT(48.16777701894265 -122.9428172600911) bank100084 +100085 POINT(47.11561955509799 -123.12401164137762) bank100085 +100086 POINT(48.03860968852819 -121.43827272921006) bank100086 +100087 POINT(47.74490668465061 -122.70905655801666) bank100087 +100088 POINT(48.60526283676827 -122.66347972260013) bank100088 +100089 POINT(47.19673301574176 -122.10629627089966) bank100089 +100090 POINT(46.88439197125022 -122.31698197574069) bank100090 +100091 POINT(48.39718449505948 -122.8032048055857) bank100091 +100092 POINT(48.47000530528745 -122.31842109946305) bank100092 +100093 POINT(47.2820467574375 -121.74387265426954) bank100093 +100094 POINT(47.90043276757799 -122.3159691826907) bank100094 +100095 POINT(46.61531049136285 -123.0361910288495) bank100095 +100096 POINT(47.10554149939285 -122.67874412221153) bank100096 +100097 POINT(47.62075470980704 -121.70861092275618) bank100097 +100098 POINT(48.08297707896083 -122.33966268904183) bank100098 +100099 POINT(46.99255900771973 -123.08440973211201) bank100099 +100100 POINT(46.659862746465535 -122.62682488331677) bank100100 +100101 POINT(48.47602803883607 -123.31031976465857) bank100101 +100102 POINT(47.65195527250224 -123.0074967867504) bank100102 +100103 POINT(46.611936144949844 -122.1069631064499) bank100103 +100104 POINT(46.833222349096864 -121.40446265126528) bank100104 +100105 POINT(47.73877490926018 -122.7652813178496) bank100105 +100106 POINT(47.4959285139029 -122.35794635539429) bank100106 +100107 POINT(46.908728646246864 -122.14100246436593) bank100107 +100108 POINT(46.86914500451883 -123.12334772554487) bank100108 +100109 POINT(48.124731800157186 -123.13343772147999) bank100109 +100110 POINT(47.017574553453144 -123.19224739853311) bank100110 +100111 POINT(48.00862356540879 -122.9322906037277) bank100111 +100112 POINT(46.94759132236265 -122.84321705391226) bank100112 +100113 POINT(47.12722574634642 -121.75641406683796) bank100113 +100114 POINT(47.44750805328223 -122.99221530888607) bank100114 +100115 POINT(47.7060371704644 -121.46129312132511) bank100115 +100116 POINT(47.056842795192736 -123.09999170982574) bank100116 +100117 POINT(46.94790715090752 -122.81298972072584) bank100117 +100118 POINT(47.86871815081317 -121.87048923811959) bank100118 +100119 POINT(48.352617244532986 -121.80651293672274) bank100119 +100120 POINT(48.256850620656195 -122.789579918012) bank100120 +100121 POINT(47.70418088802416 -122.04263495351049) bank100121 +100122 POINT(46.97358046394465 -122.30547508092431) bank100122 +100123 POINT(46.703655918371474 -122.48487690477293) bank100123 +100124 POINT(47.00287117769818 -122.57959478529096) bank100124 +100125 POINT(47.97351834061758 -121.6139058650402) bank100125 +100126 POINT(47.25298495916973 -122.50040383855858) bank100126 +100127 POINT(47.071609219692654 -122.47604040646004) bank100127 +100128 POINT(47.587519959290496 -123.07250325199693) bank100128 +100129 POINT(47.05713356650551 -121.96919909348978) bank100129 +100130 POINT(46.846643016994314 -122.61010433419304) bank100130 +100131 POINT(46.662278492011474 -122.35419347780314) bank100131 +100132 POINT(48.10694145738226 -122.68221030363914) bank100132 +100133 POINT(47.04743625828275 -122.57998193708332) bank100133 +100134 POINT(47.376160701224116 -121.84457108663025) bank100134 +100135 POINT(48.115592512353636 -122.44100837639157) bank100135 +100136 POINT(47.474044988910016 -123.1830361695045) bank100136 +100137 POINT(46.75673444812502 -123.27265638249631) bank100137 +100138 POINT(48.552073081126565 -122.44600372295517) bank100138 +100139 POINT(47.97742077898778 -123.12298424188921) bank100139 +100140 POINT(48.520782811072195 -123.20776623831723) bank100140 +100141 POINT(47.49489403865712 -121.5125850666175) bank100141 +100142 POINT(47.068131007395166 -122.35146785384183) bank100142 +100143 POINT(48.15887647401287 -122.33338693466506) bank100143 +100144 POINT(47.27434326736275 -121.7210472524057) bank100144 +100145 POINT(47.1048306230103 -123.28499872638059) bank100145 +100146 POINT(47.212211805343166 -122.58965518202143) bank100146 +100147 POINT(48.498192128797605 -121.96755176995761) bank100147 +100148 POINT(48.27477704400902 -122.9933644349764) bank100148 +100149 POINT(47.61294489377304 -123.21072432347832) bank100149 +100150 POINT(48.324756568164425 -121.50559875885399) bank100150 +100151 POINT(48.5942715792896 -122.02932182263274) bank100151 +100152 POINT(47.94552255797711 -122.52251912197545) bank100152 +100153 POINT(47.024529603863606 -122.57762956886317) bank100153 +100154 POINT(46.80472073876638 -122.55159766887593) bank100154 +100155 POINT(47.86872805446483 -122.22175733232193) bank100155 +100156 POINT(47.9294560640542 -122.99670171331164) bank100156 +100157 POINT(47.1877698964478 -123.02366159113777) bank100157 +100158 POINT(47.39587867492346 -122.8104040715135) bank100158 +100159 POINT(48.01693603926578 -123.14630838458683) bank100159 +100160 POINT(47.08106780248847 -121.52788464069738) bank100160 +100161 POINT(47.935846807630284 -123.17402001707397) bank100161 +100162 POINT(47.47155992108925 -123.06455949369835) bank100162 +100163 POINT(47.600271889868104 -122.43402698503203) bank100163 +100164 POINT(47.142663984268786 -121.9728027780219) bank100164 +100165 POINT(46.87760255028737 -122.72140037108828) bank100165 +100166 POINT(47.06809590703486 -122.11596608788709) bank100166 +100167 POINT(47.49858625570492 -122.45847002479083) bank100167 +100168 POINT(47.30599063196679 -122.44188133580602) bank100168 +100169 POINT(47.06267792439408 -122.504320215443) bank100169 +100170 POINT(48.11673351859715 -122.2936228258076) bank100170 +100171 POINT(48.48709693924997 -122.2442101274702) bank100171 +100172 POINT(48.14800402154204 -121.7978314318497) bank100172 +100173 POINT(47.895583851397674 -121.68907754673695) bank100173 +100174 POINT(47.3570185455753 -122.64613264576391) bank100174 +100175 POINT(47.004579788260294 -121.6537092218698) bank100175 +100176 POINT(46.776670154479284 -122.12841196150848) bank100176 +100177 POINT(48.58605745066459 -121.85181060528765) bank100177 +100178 POINT(46.976467457988015 -122.54992586784705) bank100178 +100179 POINT(47.39392427029679 -122.24717991130343) bank100179 +100180 POINT(48.096351002434126 -121.6875022538426) bank100180 +100181 POINT(47.06761034967403 -122.287689894841) bank100181 +100182 POINT(46.99575183016367 -121.87553085606052) bank100182 +100183 POINT(47.80435994802781 -121.75999824685312) bank100183 +100184 POINT(48.36253148869054 -122.37366793888415) bank100184 +100185 POINT(47.080940681537484 -121.39674331567176) bank100185 +100186 POINT(47.065005257683644 -123.30283111367211) bank100186 +100187 POINT(46.67799346698413 -122.84494465450659) bank100187 +100188 POINT(46.83875262721784 -122.89759474429168) bank100188 +100189 POINT(47.00216072578035 -122.62426797240302) bank100189 +100190 POINT(46.60921747491768 -122.99916365460089) bank100190 +100191 POINT(47.40764663320267 -123.02291448311131) bank100191 +100192 POINT(48.11934903451486 -121.69711194463227) bank100192 +100193 POINT(46.70520872537254 -121.47837958001539) bank100193 +100194 POINT(47.09609018594424 -122.2075921325093) bank100194 +100195 POINT(47.456528245973225 -122.34855829603634) bank100195 +100196 POINT(48.44027197752568 -121.42454767535449) bank100196 +100197 POINT(48.58167068280885 -121.71480064483491) bank100197 +100198 POINT(48.04176751701162 -121.98442028673031) bank100198 +100199 POINT(48.02892467574416 -122.02145752433347) bank100199 +100200 POINT(47.9253632799091 -121.9642514191207) bank100200 +100201 POINT(47.50751726853167 -122.09417709753733) bank100201 +100202 POINT(48.33165130759131 -122.62138205665781) bank100202 +100203 POINT(47.41592577725646 -122.1006004211848) bank100203 +100204 POINT(47.12056738392882 -123.04547085246703) bank100204 +100205 POINT(48.4306646777743 -123.27762703604012) bank100205 +100206 POINT(48.26680644723374 -121.85793534497165) bank100206 +100207 POINT(46.88661643584256 -122.96642456688144) bank100207 +100208 POINT(47.92201247432701 -123.31071425681262) bank100208 +100209 POINT(47.07855685692179 -122.56728337548783) bank100209 +100210 POINT(47.36587403568811 -122.7750255143163) bank100210 +100211 POINT(46.66386748475533 -123.10906321950256) bank100211 +100212 POINT(47.37420910411127 -122.65921390677312) bank100212 +100213 POINT(47.79231988016496 -122.67550998268355) bank100213 +100214 POINT(47.80985335868946 -122.38968335993437) bank100214 +100215 POINT(48.17815225650996 -122.82526739323825) bank100215 +100216 POINT(47.625045827761255 -123.01062881890279) bank100216 +100217 POINT(48.49076156450197 -122.60154934751522) bank100217 +100218 POINT(48.46118712571788 -122.7669480455751) bank100218 +100219 POINT(47.47441194566438 -121.69258315885023) bank100219 +100220 POINT(47.10040689900955 -122.89872357641825) bank100220 +100221 POINT(47.87102249726457 -122.64217746534419) bank100221 +100222 POINT(46.60829821269371 -122.54428672227941) bank100222 +100223 POINT(48.49725578222668 -121.8565773944186) bank100223 +100224 POINT(46.75397002466167 -122.1431792478634) bank100224 +100225 POINT(47.37745238073786 -122.88146362934384) bank100225 +100226 POINT(46.616372978372446 -121.69192692468359) bank100226 +100227 POINT(47.465689484545834 -122.53936703966706) bank100227 +100228 POINT(48.16578354077931 -122.91833001860356) bank100228 +100229 POINT(48.3921054118938 -122.57659192298763) bank100229 +100230 POINT(47.947077350048374 -122.68279069087458) bank100230 +100231 POINT(46.79571001515108 -122.67202442129613) bank100231 +100232 POINT(47.732109369363876 -122.30994589306775) bank100232 +100233 POINT(46.90349810995152 -122.23207225165211) bank100233 +100234 POINT(47.43060771640324 -122.21905527765986) bank100234 +100235 POINT(48.00882639059182 -121.56272925503933) bank100235 +100236 POINT(47.537343275359994 -122.40318469497767) bank100236 +100237 POINT(47.35148213193353 -122.07692375165132) bank100237 +100238 POINT(46.70766585170698 -121.36772361190444) bank100238 +100239 POINT(48.182748763181856 -122.3446839395825) bank100239 +100240 POINT(46.9342360768598 -123.10692363028454) bank100240 +100241 POINT(46.752193048416636 -122.06211051478425) bank100241 +100242 POINT(48.413598942060105 -121.91687480808149) bank100242 +100243 POINT(47.38342610386212 -122.51246937810858) bank100243 +100244 POINT(48.57158909173217 -123.18030560565859) bank100244 +100245 POINT(47.02424243896897 -121.38725228178512) bank100245 +100246 POINT(47.07244020180801 -122.61870673548123) bank100246 +100247 POINT(47.629017815732674 -121.89481634092382) bank100247 +100248 POINT(46.95818654834437 -122.80830105949107) bank100248 +100249 POINT(47.12295746417594 -121.70743349494745) bank100249 +100250 POINT(47.108263737385144 -121.4724166648615) bank100250 +100251 POINT(46.72524295115498 -122.40715280546225) bank100251 +100252 POINT(48.17511697206033 -121.98411455902) bank100252 +100253 POINT(48.46090833610392 -121.72909823442147) bank100253 +100254 POINT(47.720457589684116 -122.5989884140756) bank100254 +100255 POINT(47.78410809059867 -122.03796717859711) bank100255 +100256 POINT(47.1817513884515 -122.93712503745701) bank100256 +100257 POINT(46.65236809123033 -123.19244500897899) bank100257 +100258 POINT(47.196833460862734 -122.2694957895619) bank100258 +100259 POINT(47.10818220621822 -121.57530277305956) bank100259 +100260 POINT(47.55508779949927 -121.67246543225464) bank100260 +100261 POINT(47.603038110555204 -121.84203933514682) bank100261 +100262 POINT(47.70421481311674 -123.05626611957332) bank100262 +100263 POINT(47.608114087545545 -122.63690509808202) bank100263 +100264 POINT(46.77019924036669 -122.66211602543372) bank100264 +100265 POINT(47.03650769014278 -122.21721699644776) bank100265 +100266 POINT(46.889159611272426 -122.00225354144152) bank100266 +100267 POINT(47.333150773304986 -122.42032638564481) bank100267 +100268 POINT(48.54144830882205 -122.12064373321692) bank100268 +100269 POINT(46.7825242035592 -122.87695964694588) bank100269 +100270 POINT(48.053062181380106 -121.89496679379387) bank100270 +100271 POINT(46.9623731858894 -123.27094671228107) bank100271 +100272 POINT(46.81683929765518 -122.40585963545216) bank100272 +100273 POINT(47.20699525898932 -122.52609594237306) bank100273 +100274 POINT(47.902107488255105 -121.96362686506184) bank100274 +100275 POINT(47.480562006311764 -122.33461290416795) bank100275 +100276 POINT(48.133774503240495 -122.37936389483237) bank100276 +100277 POINT(47.332050595760634 -122.92904646733213) bank100277 +100278 POINT(48.07874170442958 -122.16947633378007) bank100278 +100279 POINT(47.73256061714204 -123.16128566023993) bank100279 +100280 POINT(48.3467064360625 -121.3939147914374) bank100280 +100281 POINT(47.9873200936057 -122.7802563126019) bank100281 +100282 POINT(46.93584491963327 -122.2948994192853) bank100282 +100283 POINT(48.01751475091565 -122.32660191356854) bank100283 +100284 POINT(47.885312043844635 -122.9924139863872) bank100284 +100285 POINT(47.33321167861585 -121.88785840382072) bank100285 +100286 POINT(47.78131159894618 -122.89675683363913) bank100286 +100287 POINT(48.008004481272806 -121.68815293281664) bank100287 +100288 POINT(46.736391395381446 -121.53091311469746) bank100288 +100289 POINT(47.68126796425244 -122.65709517574633) bank100289 +100290 POINT(48.03844105926999 -122.96587108201436) bank100290 +100291 POINT(47.57671994508577 -122.52702991399411) bank100291 +100292 POINT(48.33090401535904 -122.15013004726882) bank100292 +100293 POINT(47.71233381130285 -121.53371712134728) bank100293 +100294 POINT(47.59720587958956 -121.44297345220154) bank100294 +100295 POINT(47.56900524685571 -121.44064309064524) bank100295 +100296 POINT(47.14487623199528 -122.0412496702478) bank100296 +100297 POINT(47.101161666766345 -121.94330373517283) bank100297 +100298 POINT(46.721443080142556 -122.69265199128337) bank100298 +100299 POINT(48.489952472571936 -122.44017830335491) bank100299 +100300 POINT(47.92860024836894 -123.02737221302289) bank100300 +100301 POINT(47.17069006465871 -123.15013865393833) bank100301 +100302 POINT(47.05203175865351 -121.87105254449766) bank100302 +100303 POINT(46.93859355775462 -123.16947284363091) bank100303 +100304 POINT(47.6855006107803 -121.76798888972647) bank100304 +100305 POINT(47.979654679740435 -123.30606947362733) bank100305 +100306 POINT(47.05219579878931 -122.83385476424492) bank100306 +100307 POINT(47.31246611088112 -122.8256615506899) bank100307 +100308 POINT(47.11133043652928 -121.93687915859076) bank100308 +100309 POINT(48.07099122596706 -122.43402718582917) bank100309 +100310 POINT(47.405948233887806 -123.26346768432977) bank100310 +100311 POINT(46.88535880497955 -122.37782923885179) bank100311 +100312 POINT(47.842402377724845 -123.10558637282824) bank100312 +100313 POINT(48.14387756424048 -122.65699249165077) bank100313 +100314 POINT(47.2840491693834 -122.40524262066015) bank100314 +100315 POINT(47.42159264271049 -123.06067668711634) bank100315 +100316 POINT(47.80711748133791 -123.20165420136985) bank100316 +100317 POINT(47.07695159440487 -122.66520542678813) bank100317 +100318 POINT(48.01458042846874 -121.45913963471781) bank100318 +100319 POINT(46.614288187330416 -122.73530892759828) bank100319 +100320 POINT(46.621726487068955 -122.47169339717144) bank100320 +100321 POINT(46.74093232667063 -121.35232447657657) bank100321 +100322 POINT(47.101305129884295 -121.35914983133509) bank100322 +100323 POINT(47.22732243331994 -122.95601952260903) bank100323 +100324 POINT(47.146313661915315 -122.8066719166069) bank100324 +100325 POINT(46.85041184117309 -121.74720892811811) bank100325 +100326 POINT(48.45762688953753 -122.61147119529585) bank100326 +100327 POINT(48.0427342454283 -122.58015634643776) bank100327 +100328 POINT(47.424756173848394 -121.4338002034977) bank100328 +100329 POINT(48.2208255282454 -122.22897216798353) bank100329 +100330 POINT(48.07960317918992 -122.72703079147044) bank100330 +100331 POINT(48.198860957764225 -122.25515610963987) bank100331 +100332 POINT(47.62428601340347 -122.08522733524534) bank100332 +100333 POINT(48.33880303550875 -122.18496056381514) bank100333 +100334 POINT(46.73285070913707 -121.85839490247147) bank100334 +100335 POINT(48.252070025877636 -122.58322446093652) bank100335 +100336 POINT(48.07175077406979 -122.95852019481464) bank100336 +100337 POINT(47.6129323714899 -121.33897178324744) bank100337 +100338 POINT(48.37210962277168 -122.51262342710136) bank100338 +100339 POINT(46.771006828134674 -122.54841136254184) bank100339 +100340 POINT(47.54101309582738 -123.09161479315057) bank100340 +100341 POINT(48.04889548173188 -123.2761763712244) bank100341 +100342 POINT(47.89533607066486 -122.53578180512237) bank100342 +100343 POINT(48.126937646911756 -121.42636392155228) bank100343 +100344 POINT(47.444051302129076 -121.86574906279856) bank100344 +100345 POINT(46.61865604845234 -122.49781866289915) bank100345 +100346 POINT(48.25037892456832 -122.22695089747873) bank100346 +100347 POINT(47.3747228268127 -122.39791346974198) bank100347 +100348 POINT(48.255100564257084 -122.80378516351549) bank100348 +100349 POINT(48.04404262757532 -121.74163242035654) bank100349 +100350 POINT(46.886643651615664 -122.44416727390225) bank100350 +100351 POINT(47.49161535280466 -122.5795804028176) bank100351 +100352 POINT(47.31209414997396 -122.63379279737798) bank100352 +100353 POINT(48.51079437955909 -122.88099827848059) bank100353 +100354 POINT(46.72774612722385 -122.67547757976747) bank100354 +100355 POINT(46.97212563103206 -121.76218261013763) bank100355 +100356 POINT(47.853487264953614 -122.95814537846212) bank100356 +100357 POINT(47.15581033778075 -121.88236228439432) bank100357 +100358 POINT(48.01258582512727 -121.69157120100168) bank100358 +100359 POINT(47.88506240361219 -122.19525060097776) bank100359 +100360 POINT(47.27586529524507 -122.2067661898764) bank100360 +100361 POINT(46.615031627613924 -123.15384525896714) bank100361 +100362 POINT(48.22978053753784 -121.87682071099017) bank100362 +100363 POINT(48.22696326351946 -122.82921802298208) bank100363 +100364 POINT(48.23062222139801 -121.65666959107519) bank100364 +100365 POINT(47.088216227827814 -121.5680594674944) bank100365 +100366 POINT(47.22719699119652 -121.8050843440673) bank100366 +100367 POINT(48.32492832995918 -122.75467000861656) bank100367 +100368 POINT(47.32991138053043 -122.27081885967407) bank100368 +100369 POINT(47.044441946086316 -123.24780341919437) bank100369 +100370 POINT(46.772834575774915 -122.26581849394081) bank100370 +100371 POINT(46.69156765895716 -121.81295027682074) bank100371 +100372 POINT(47.78425366313415 -121.7416733917565) bank100372 +100373 POINT(47.13149996236473 -122.03528569523358) bank100373 +100374 POINT(47.090781355358715 -122.82364366844455) bank100374 +100375 POINT(47.14454672068043 -122.90196230232334) bank100375 +100376 POINT(47.374384394017106 -121.64346309267272) bank100376 +100377 POINT(47.54109142521362 -121.3465532814034) bank100377 +100378 POINT(47.392399172517244 -122.60886193293376) bank100378 +100379 POINT(48.52761099165879 -123.00577682344523) bank100379 +100380 POINT(46.635276062917 -122.12308277924667) bank100380 +100381 POINT(48.434098904110336 -123.10910622698526) bank100381 +100382 POINT(47.938706270707165 -121.62922350739372) bank100382 +100383 POINT(46.79043055450331 -122.30687611687118) bank100383 +100384 POINT(48.47907571176741 -121.4078156949898) bank100384 +100385 POINT(47.392283988976075 -122.20601209330889) bank100385 +100386 POINT(47.53276761649291 -122.73286264268921) bank100386 +100387 POINT(47.077217165695785 -122.5302702582585) bank100387 +100388 POINT(46.95753544199773 -122.03567956956935) bank100388 +100389 POINT(46.61966547368317 -121.9397812738982) bank100389 +100390 POINT(48.38515280437576 -121.3873006575759) bank100390 +100391 POINT(48.41874521790118 -122.70336882522643) bank100391 +100392 POINT(46.64544847435931 -122.29358060747182) bank100392 +100393 POINT(48.60551557888362 -121.96585666652815) bank100393 +100394 POINT(47.67762691510552 -121.69509904359752) bank100394 +100395 POINT(48.132845899436056 -123.28714136973322) bank100395 +100396 POINT(48.5110606220952 -123.03773006007093) bank100396 +100397 POINT(47.51134827025478 -122.00717271660373) bank100397 +100398 POINT(47.94357962449676 -121.34733479770776) bank100398 +100399 POINT(47.841339299845444 -121.86807781543374) bank100399 +100400 POINT(46.728669937523726 -122.64730691921922) bank100400 +100401 POINT(48.435838891937436 -123.10897143759682) bank100401 +100402 POINT(48.276567361866654 -122.8424742279757) bank100402 +100403 POINT(47.961580448921566 -121.77740679803618) bank100403 +100404 POINT(47.650120138757146 -122.27833538694215) bank100404 +100405 POINT(46.66018678479037 -123.294563679933) bank100405 +100406 POINT(46.809218891268344 -121.38324970165178) bank100406 +100407 POINT(48.13466396651951 -122.965644240846) bank100407 +100408 POINT(48.33665293084191 -122.65606632843891) bank100408 +100409 POINT(48.47381793534252 -123.11645320165412) bank100409 +100410 POINT(47.84349890440134 -121.56191908499873) bank100410 +100411 POINT(47.016612657316855 -121.91721678243695) bank100411 +100412 POINT(47.043775608244445 -122.51889466030312) bank100412 +100413 POINT(47.27140956658483 -123.23701985227675) bank100413 +100414 POINT(47.83982008987565 -121.71170216215802) bank100414 +100415 POINT(47.02619126440534 -122.8653787102125) bank100415 +100416 POINT(46.835327304394255 -123.0226795181472) bank100416 +100417 POINT(48.363752433673184 -122.62997836037674) bank100417 +100418 POINT(47.150105612036405 -123.22403007123893) bank100418 +100419 POINT(47.85173320193335 -122.65857228959972) bank100419 +100420 POINT(47.718810936940926 -123.25846875704134) bank100420 +100421 POINT(48.089841635963566 -122.03830677778843) bank100421 +100422 POINT(47.9542147776136 -121.84942170385237) bank100422 +100423 POINT(47.661926713155864 -121.577705890182) bank100423 +100424 POINT(48.292120384043585 -121.45241696140091) bank100424 +100425 POINT(47.55574841814134 -123.31357068130414) bank100425 +100426 POINT(48.31034269642515 -122.02581570719471) bank100426 +100427 POINT(48.247549708189986 -122.82181010786805) bank100427 +100428 POINT(47.92546971882714 -122.987050092217) bank100428 +100429 POINT(47.54902930022674 -122.07137531476148) bank100429 +100430 POINT(47.02829327364716 -122.44390186088) bank100430 +100431 POINT(47.13624679535288 -123.09680122134925) bank100431 +100432 POINT(48.369284002698386 -122.28212633774106) bank100432 +100433 POINT(48.37788334588423 -123.08643164792724) bank100433 +100434 POINT(48.52270978995161 -123.1374539940765) bank100434 +100435 POINT(47.17846022251867 -122.93074470871913) bank100435 +100436 POINT(46.890486644698655 -121.57622502862723) bank100436 +100437 POINT(47.92122284592923 -121.5913408436473) bank100437 +100438 POINT(48.43750806526597 -122.13203404059539) bank100438 +100439 POINT(48.28458965658817 -122.58489787177675) bank100439 +100440 POINT(47.04500795941871 -121.81100239796993) bank100440 +100441 POINT(47.803366293418776 -122.8197068073715) bank100441 +100442 POINT(47.48217038195756 -122.39352208304228) bank100442 +100443 POINT(48.38450779608938 -123.18334702548356) bank100443 +100444 POINT(47.837390419447026 -122.49700222961057) bank100444 +100445 POINT(47.39607620341014 -121.59899926066697) bank100445 +100446 POINT(48.20190491840001 -122.29230405653045) bank100446 +100447 POINT(47.80595075346511 -122.49039082546068) bank100447 +100448 POINT(47.41458134130509 -121.56025251222243) bank100448 +100449 POINT(48.47493469046593 -122.05511720540764) bank100449 +100450 POINT(48.24649657296386 -121.87321215638781) bank100450 +100451 POINT(47.7823575498841 -121.79540109934337) bank100451 +100452 POINT(48.399661764646424 -122.30440595433825) bank100452 +100453 POINT(47.46582801786341 -123.08374007916444) bank100453 +100454 POINT(48.005693140593564 -121.64353256878779) bank100454 +100455 POINT(48.225447751655295 -122.77306584740434) bank100455 +100456 POINT(47.118174372050746 -122.11139523127456) bank100456 +100457 POINT(47.695599114315144 -121.82616737857369) bank100457 +100458 POINT(47.2953131870228 -123.0770256864304) bank100458 +100459 POINT(48.05822600258463 -122.14321767176372) bank100459 +100460 POINT(47.20427637522257 -121.60230912639345) bank100460 +100461 POINT(46.97149466004319 -122.01206941832562) bank100461 +100462 POINT(47.911879592152076 -123.12649214067211) bank100462 +100463 POINT(48.292590476186284 -122.04723825608806) bank100463 +100464 POINT(47.95201679071003 -122.38310581393687) bank100464 +100465 POINT(47.162856475509564 -123.28509739006427) bank100465 +100466 POINT(47.311028368062566 -123.20903333574454) bank100466 +100467 POINT(46.82162475969269 -123.25607725221131) bank100467 +100468 POINT(48.296784880453714 -123.15228089550303) bank100468 +100469 POINT(46.739915610686886 -123.03411260326331) bank100469 +100470 POINT(47.95577011465212 -123.11571236627724) bank100470 +100471 POINT(47.27854697443463 -122.44474602909611) bank100471 +100472 POINT(48.27353457481968 -121.45178265274787) bank100472 +100473 POINT(47.32966564753496 -121.58436804221384) bank100473 +100474 POINT(48.40391799791594 -122.54437783123781) bank100474 +100475 POINT(47.372809301505136 -122.20380920712708) bank100475 +100476 POINT(48.136756660703284 -121.79409069574035) bank100476 +100477 POINT(47.008404406314085 -122.89905683041505) bank100477 +100478 POINT(47.37036671367993 -121.56951176714581) bank100478 +100479 POINT(47.00806877484402 -122.21903082940771) bank100479 +100480 POINT(47.09141847657878 -122.08462109107921) bank100480 +100481 POINT(47.38565871539209 -121.33793940771906) bank100481 +100482 POINT(47.40589262544534 -121.51854784927322) bank100482 +100483 POINT(48.274830183368245 -122.15340323054782) bank100483 +100484 POINT(46.7908825041226 -122.83082617979017) bank100484 +100485 POINT(46.96373594364185 -122.526849841242) bank100485 +100486 POINT(48.34320434071782 -122.08312326583759) bank100486 +100487 POINT(47.08192397130833 -122.74703463236065) bank100487 +100488 POINT(46.91467946773312 -122.23715611245599) bank100488 +100489 POINT(47.24952842098627 -122.87328677660449) bank100489 +100490 POINT(46.70827991019472 -122.87199250045289) bank100490 +100491 POINT(47.72636588797169 -121.51368455683746) bank100491 +100492 POINT(47.41926026908839 -121.53870319716957) bank100492 +100493 POINT(46.8221549044768 -122.52348700305086) bank100493 +100494 POINT(48.361087242800174 -122.07968575017776) bank100494 +100495 POINT(48.45888651140963 -122.69629862280689) bank100495 +100496 POINT(47.580095482654826 -122.29065103068815) bank100496 +100497 POINT(46.92904148687399 -122.41609616086407) bank100497 +100498 POINT(46.667220385378435 -121.86799922123144) bank100498 +100499 POINT(47.5769750332605 -122.44652284257856) bank100499 +100500 POINT(48.367746781256216 -121.65705753124332) bank100500 +100501 POINT(47.61407067331041 -122.24640536037424) bank100501 +100502 POINT(48.5940829677158 -122.34225194439365) bank100502 +100503 POINT(47.93873611541454 -121.40743211469602) bank100503 +100504 POINT(47.54892042055131 -121.82261072094931) bank100504 +100505 POINT(48.56024261980588 -121.88081833410703) bank100505 +100506 POINT(48.271385348009666 -123.32108232475798) bank100506 +100507 POINT(47.96725095439435 -121.83930384525617) bank100507 +100508 POINT(48.59478229310123 -123.10449659994313) bank100508 +100509 POINT(47.64178053585792 -123.04864876906736) bank100509 +100510 POINT(47.17567696022777 -122.97413382804832) bank100510 +100511 POINT(47.552507478966405 -122.77808694206485) bank100511 +100512 POINT(46.82656052711019 -122.21463571459103) bank100512 +100513 POINT(48.211044004867155 -121.92436771509664) bank100513 +100514 POINT(48.4100878634703 -121.53722136273333) bank100514 +100515 POINT(48.59562962679397 -121.70466235691735) bank100515 +100516 POINT(47.70929317007163 -121.91427265472826) bank100516 +100517 POINT(47.97814418286671 -121.68275990909407) bank100517 +100518 POINT(46.823292079406905 -121.43501221143624) bank100518 +100519 POINT(47.03358114607409 -121.83788246334926) bank100519 +100520 POINT(48.388780129699754 -122.3416723168702) bank100520 +100521 POINT(48.45661406670337 -123.16739986911223) bank100521 +100522 POINT(48.165798961538044 -121.48339383429615) bank100522 +100523 POINT(47.58346692590197 -123.01137005435089) bank100523 +100524 POINT(46.67953879553996 -121.95672121489088) bank100524 +100525 POINT(47.776052498577194 -122.19766065147174) bank100525 +100526 POINT(47.56617907833958 -121.83820792964073) bank100526 +100527 POINT(47.92380684229508 -122.68298179079913) bank100527 +100528 POINT(47.4537236793242 -122.98908687325655) bank100528 +100529 POINT(46.77844001569921 -122.04996040794633) bank100529 +100530 POINT(47.43755649092153 -121.41564832190414) bank100530 +100531 POINT(48.123804347781615 -122.89184758753059) bank100531 +100532 POINT(48.23213816984194 -121.5770933955211) bank100532 +100533 POINT(48.22756303960425 -122.14044465139035) bank100533 +100534 POINT(46.955747398699536 -121.95181547532368) bank100534 +100535 POINT(47.407302819783084 -121.99397372110386) bank100535 +100536 POINT(48.27219602296152 -121.77272623294161) bank100536 +100537 POINT(46.663652438062634 -121.44220176997153) bank100537 +100538 POINT(48.29331556161358 -122.23950156147605) bank100538 +100539 POINT(47.32388641715847 -123.25269397783917) bank100539 +100540 POINT(47.60833184667694 -122.89369466251117) bank100540 +100541 POINT(46.84242806089869 -122.46195186059717) bank100541 +100542 POINT(47.43145346999647 -122.15636678856558) bank100542 +100543 POINT(46.89897156493899 -121.53612016437262) bank100543 +100544 POINT(47.04064918754803 -121.47477408777388) bank100544 +100545 POINT(46.8012373606044 -123.28746480782094) bank100545 +100546 POINT(48.43785245170551 -123.32917956636489) bank100546 +100547 POINT(46.953222164705664 -121.7723923562764) bank100547 +100548 POINT(48.44391257267825 -121.73682753876794) bank100548 +100549 POINT(48.32683790457824 -121.43927495288806) bank100549 +100550 POINT(47.88135794198994 -121.81917901401587) bank100550 +100551 POINT(46.89323978434964 -122.84188872059676) bank100551 +100552 POINT(46.65056050052767 -121.8223865749262) bank100552 +100553 POINT(46.998923363879925 -122.72901403402433) bank100553 +100554 POINT(48.271576451748864 -121.93179667543345) bank100554 +100555 POINT(48.23346164427183 -122.18358289886173) bank100555 +100556 POINT(48.436484701966634 -122.91873290564888) bank100556 +100557 POINT(48.00614352164904 -121.86497144002126) bank100557 +100558 POINT(48.568267394964685 -122.06588909428142) bank100558 +100559 POINT(48.00752330951372 -122.98292218654274) bank100559 +100560 POINT(46.74065336769974 -123.05843089645712) bank100560 +100561 POINT(48.06608052735063 -121.90946892858769) bank100561 +100562 POINT(47.2844243090345 -121.70195739761778) bank100562 +100563 POINT(46.69488997506896 -122.47360833724929) bank100563 +100564 POINT(48.49349815717215 -123.1218059413063) bank100564 +100565 POINT(47.67191479638436 -121.89422252957719) bank100565 +100566 POINT(48.175468068608794 -122.91715478429828) bank100566 +100567 POINT(47.616185150514646 -121.51697686309149) bank100567 +100568 POINT(48.33048931789285 -123.15264591312736) bank100568 +100569 POINT(46.75700059942387 -121.67363798891564) bank100569 +100570 POINT(47.358692598707314 -123.1281833820044) bank100570 +100571 POINT(46.831846501061904 -122.16162933998436) bank100571 +100572 POINT(48.60095212971322 -122.75957924932301) bank100572 +100573 POINT(47.99011408925935 -123.14038473061731) bank100573 +100574 POINT(46.88977870007891 -121.67019748280161) bank100574 +100575 POINT(48.37584771577473 -121.44006435538834) bank100575 +100576 POINT(47.4760351031329 -121.64628635187036) bank100576 +100577 POINT(47.50356035907703 -121.52703382659487) bank100577 +100578 POINT(48.242487656082446 -121.57196758623896) bank100578 +100579 POINT(48.41041371409138 -121.64447199385349) bank100579 +100580 POINT(47.99384040924571 -122.50989105870636) bank100580 +100581 POINT(47.38022106471904 -122.06057315543588) bank100581 +100582 POINT(47.636978996276504 -121.64913395671688) bank100582 +100583 POINT(47.62299649666652 -123.0595295442115) bank100583 +100584 POINT(46.81640743381104 -123.04085547152114) bank100584 +100585 POINT(47.016602263226346 -121.98315746546982) bank100585 +100586 POINT(47.23400019182969 -122.11290046380964) bank100586 +100587 POINT(48.58503714736751 -123.13665003435173) bank100587 +100588 POINT(47.5522485088882 -122.40695229961517) bank100588 +100589 POINT(48.30569668196729 -123.13477384462512) bank100589 +100590 POINT(47.84475094722072 -121.86513904430069) bank100590 +100591 POINT(46.97797024574253 -121.35315890065498) bank100591 +100592 POINT(48.39961010901517 -122.77423540042129) bank100592 +100593 POINT(47.55507020133417 -122.70804450969416) bank100593 +100594 POINT(46.79157680378309 -122.09236866818968) bank100594 +100595 POINT(48.55170759264378 -123.03289123836355) bank100595 +100596 POINT(47.54657453560332 -121.83594186843946) bank100596 +100597 POINT(47.36910358063875 -121.41988256605377) bank100597 +100598 POINT(48.25927892519642 -122.37328423056488) bank100598 +100599 POINT(48.13909552596118 -121.55938341094914) bank100599 +100600 POINT(48.13152026610293 -122.56689590743346) bank100600 +100601 POINT(46.70398738861369 -122.11674490034842) bank100601 +100602 POINT(46.81667591498318 -122.84003578323829) bank100602 +100603 POINT(47.34255896177305 -122.24325233248724) bank100603 +100604 POINT(47.672913026208 -122.18995717935681) bank100604 +100605 POINT(47.12098545248328 -123.15249286256501) bank100605 +100606 POINT(48.190205841976166 -121.7238966510528) bank100606 +100607 POINT(47.659805081958176 -122.13412789173685) bank100607 +100608 POINT(47.84161728369779 -122.45193376592594) bank100608 +100609 POINT(48.017115008875976 -122.20874173448503) bank100609 +100610 POINT(47.12573497893161 -121.76728576812006) bank100610 +100611 POINT(47.6850018700026 -123.16608994674691) bank100611 +100612 POINT(47.42640673710523 -121.5934607092925) bank100612 +100613 POINT(47.46573487313325 -121.42556765272307) bank100613 +100614 POINT(46.86186146951067 -122.97352062447828) bank100614 +100615 POINT(47.10747774034082 -123.31529522766539) bank100615 +100616 POINT(47.684058323862054 -122.7333932288342) bank100616 +100617 POINT(48.3029145519192 -121.98480536505939) bank100617 +100618 POINT(47.283234592351874 -121.82721314594943) bank100618 +100619 POINT(48.39327056639417 -122.07719165102765) bank100619 +100620 POINT(46.969622636186735 -123.26444770957474) bank100620 +100621 POINT(47.79510370682369 -121.91405203194861) bank100621 +100622 POINT(47.274778384149535 -123.00179191837368) bank100622 +100623 POINT(47.01102377846566 -121.99390374042557) bank100623 +100624 POINT(47.36207830637638 -121.48407463910182) bank100624 +100625 POINT(48.53508534785372 -121.70413667750866) bank100625 +100626 POINT(48.267481230282 -123.03126938945626) bank100626 +100627 POINT(47.94694907620941 -121.58093616658485) bank100627 +100628 POINT(46.85438305380352 -123.22116330263975) bank100628 +100629 POINT(48.10608809840921 -122.0628940879668) bank100629 +100630 POINT(47.63858833082548 -122.77694744538948) bank100630 +100631 POINT(47.26118464647119 -122.00687697260915) bank100631 +100632 POINT(46.62968348392603 -123.23599231606742) bank100632 +100633 POINT(48.19687758410939 -123.12886037968669) bank100633 +100634 POINT(47.50310105682366 -121.44810731928627) bank100634 +100635 POINT(47.84213372005967 -123.28680913567722) bank100635 +100636 POINT(48.58711633294974 -121.8400526068898) bank100636 +100637 POINT(46.61183100605167 -122.68534803215121) bank100637 +100638 POINT(48.033286782118154 -122.49652711378295) bank100638 +100639 POINT(47.28419642189069 -122.69513580705537) bank100639 +100640 POINT(48.30771646833592 -122.08466104602172) bank100640 +100641 POINT(47.26512049717235 -122.88257153181577) bank100641 +100642 POINT(46.92979818898619 -121.47671459944809) bank100642 +100643 POINT(48.08006587213527 -122.02846581480003) bank100643 +100644 POINT(46.924134845734535 -122.7504730582263) bank100644 +100645 POINT(47.06001883254477 -121.80206604085502) bank100645 +100646 POINT(47.686638889125376 -122.15374036249212) bank100646 +100647 POINT(46.90780873079779 -121.96452834862208) bank100647 +100648 POINT(47.023080593618616 -122.10015087232834) bank100648 +100649 POINT(46.768552601806185 -122.6058234742553) bank100649 +100650 POINT(47.788197094059704 -122.18779278595322) bank100650 +100651 POINT(47.42659450301714 -122.12360116905252) bank100651 +100652 POINT(46.75683425192638 -122.00448253302694) bank100652 +100653 POINT(47.28083925022235 -123.224238324061) bank100653 +100654 POINT(47.96191967706115 -122.34187841277473) bank100654 +100655 POINT(47.21812560841036 -121.76803137858091) bank100655 +100656 POINT(47.674367248881936 -123.08298242264541) bank100656 +100657 POINT(47.9677973326835 -123.0727220789531) bank100657 +100658 POINT(46.74812495236799 -122.61488219982394) bank100658 +100659 POINT(47.150506980509746 -122.25626296296718) bank100659 +100660 POINT(48.097485886140646 -122.14204359208296) bank100660 +100661 POINT(46.620976710845916 -122.34257682428517) bank100661 +100662 POINT(48.46270718460648 -121.67496891890852) bank100662 +100663 POINT(47.18368785959771 -123.25684580899338) bank100663 +100664 POINT(47.69799349663853 -121.84583434859168) bank100664 +100665 POINT(47.726382945788295 -122.93637447602599) bank100665 +100666 POINT(47.60171800551082 -123.13956695799725) bank100666 +100667 POINT(48.16820530189073 -121.60283508427023) bank100667 +100668 POINT(47.87574778080071 -122.54096779691511) bank100668 +100669 POINT(48.069327555231794 -122.0552797223391) bank100669 +100670 POINT(47.642697772460515 -123.11923928166082) bank100670 +100671 POINT(47.668772129968225 -123.08585162919688) bank100671 +100672 POINT(48.00355445796353 -122.77096821350105) bank100672 +100673 POINT(48.09443541189673 -122.68965788390764) bank100673 +100674 POINT(48.18344025390952 -121.8936973066953) bank100674 +100675 POINT(48.44816476081014 -122.22223992621281) bank100675 +100676 POINT(47.694623831161415 -122.3621342729963) bank100676 +100677 POINT(47.31123311737984 -122.34736212870197) bank100677 +100678 POINT(48.24643118655848 -121.51665293950627) bank100678 +100679 POINT(48.023004362424956 -122.84723662667187) bank100679 +100680 POINT(47.79664032753034 -122.70037666347747) bank100680 +100681 POINT(48.032109533060925 -122.63015309003696) bank100681 +100682 POINT(48.17472927510166 -121.63520715282603) bank100682 +100683 POINT(48.597944413703445 -121.3406123016517) bank100683 +100684 POINT(47.80623760579786 -121.47358488616284) bank100684 +100685 POINT(47.868076347846475 -122.26555172128492) bank100685 +100686 POINT(47.181511112841875 -121.82966979167014) bank100686 +100687 POINT(47.47110936010092 -123.15265156281795) bank100687 +100688 POINT(46.953278142323875 -121.57734010934931) bank100688 +100689 POINT(47.89886300363702 -122.03628058494716) bank100689 +100690 POINT(47.976064579345596 -121.47393139842153) bank100690 +100691 POINT(48.02387431485453 -121.61106468048499) bank100691 +100692 POINT(47.77657499369235 -121.35654149803999) bank100692 +100693 POINT(47.985624878313786 -122.07880145749836) bank100693 +100694 POINT(48.345991475557014 -122.03853578683996) bank100694 +100695 POINT(48.25889815319578 -122.47964639195774) bank100695 +100696 POINT(46.8351648627509 -122.31396644164307) bank100696 +100697 POINT(47.46227763827043 -122.41755705211689) bank100697 +100698 POINT(47.97686961222737 -122.05672533383583) bank100698 +100699 POINT(46.950562799404445 -122.96618068381511) bank100699 +100700 POINT(48.54320194358598 -122.30120841154252) bank100700 +100701 POINT(48.274598458136914 -121.92909095343957) bank100701 +100702 POINT(48.32426680309535 -121.48074993127248) bank100702 +100703 POINT(46.74741903880875 -122.25295764228845) bank100703 +100704 POINT(46.67470785506898 -122.48708143796524) bank100704 +100705 POINT(47.73552481613554 -121.67678701795006) bank100705 +100706 POINT(47.92505400053317 -121.93049075314927) bank100706 +100707 POINT(46.632279371806355 -121.69134496682014) bank100707 +100708 POINT(46.654326229109344 -123.10585257481155) bank100708 +100709 POINT(48.15590927921767 -122.91524708817569) bank100709 +100710 POINT(46.627817364184146 -122.90618611285018) bank100710 +100711 POINT(48.15082813819065 -123.28276343584037) bank100711 +100712 POINT(46.72795602098759 -121.74548935527406) bank100712 +100713 POINT(48.569752089480964 -121.83675035411517) bank100713 +100714 POINT(46.97121786081631 -122.51386709047127) bank100714 +100715 POINT(47.277166524298785 -121.46334305400313) bank100715 +100716 POINT(47.9704018049882 -122.52850200451665) bank100716 +100717 POINT(48.33334622175155 -121.35139631150886) bank100717 +100718 POINT(47.386564028635114 -121.40393083117993) bank100718 +100719 POINT(48.12131994607948 -122.26925748739072) bank100719 +100720 POINT(48.371291827999045 -121.64716331330952) bank100720 +100721 POINT(46.896425735992615 -123.25159518695695) bank100721 +100722 POINT(48.23407361779189 -121.44416643773945) bank100722 +100723 POINT(47.283374126903325 -122.43675692575441) bank100723 +100724 POINT(47.63365924683102 -122.87345944523537) bank100724 +100725 POINT(47.679557255857645 -122.64904423741335) bank100725 +100726 POINT(46.628175907795196 -121.45087832768766) bank100726 +100727 POINT(47.50050506818509 -122.14603043886359) bank100727 +100728 POINT(46.816076946548826 -122.14122249477538) bank100728 +100729 POINT(47.61021676464277 -122.10365395395043) bank100729 +100730 POINT(46.84651465159088 -122.60711991156668) bank100730 +100731 POINT(48.39152310203538 -122.45709127633893) bank100731 +100732 POINT(47.380667063096 -121.64817016084336) bank100732 +100733 POINT(47.271024181017914 -123.23283151932303) bank100733 +100734 POINT(47.821688702496914 -123.27752623909142) bank100734 +100735 POINT(46.92096758807642 -122.71775344440498) bank100735 +100736 POINT(48.067775648279856 -122.38210618613522) bank100736 +100737 POINT(46.839061121812435 -121.67149392360969) bank100737 +100738 POINT(48.51169473473734 -121.92949564106354) bank100738 +100739 POINT(46.78799314361717 -122.05100532889934) bank100739 +100740 POINT(48.0363779504125 -122.29809418435921) bank100740 +100741 POINT(46.780984632058065 -122.31925063944547) bank100741 +100742 POINT(47.60339703903627 -121.84720955934746) bank100742 +100743 POINT(47.483701995791016 -122.20900047765561) bank100743 +100744 POINT(47.95555684238605 -122.8192313395572) bank100744 +100745 POINT(48.59566130026259 -121.91452930182882) bank100745 +100746 POINT(46.66436881603298 -121.80562962291233) bank100746 +100747 POINT(48.395483835809685 -121.56385101263088) bank100747 +100748 POINT(46.79262979146422 -123.04890258909681) bank100748 +100749 POINT(47.72523033301518 -122.15485597064954) bank100749 +100750 POINT(46.926493476033556 -122.79838374121074) bank100750 +100751 POINT(47.49628676734248 -122.95805045051966) bank100751 +100752 POINT(47.79633704000039 -123.21800549429011) bank100752 +100753 POINT(47.2162936449512 -123.15877370203134) bank100753 +100754 POINT(47.86407472653859 -122.9021483144826) bank100754 +100755 POINT(47.44945153859852 -122.29911412370505) bank100755 +100756 POINT(47.65163435833357 -123.31814038329088) bank100756 +100757 POINT(48.53367326522778 -122.00927108575625) bank100757 +100758 POINT(47.731038820371936 -122.37932785586666) bank100758 +100759 POINT(46.839386182182025 -121.36348645685618) bank100759 +100760 POINT(47.47405780119714 -121.82121825969494) bank100760 +100761 POINT(47.36340207626096 -122.39774279727334) bank100761 +100762 POINT(47.80850551574339 -121.66391508656913) bank100762 +100763 POINT(48.4211544904351 -122.24283695301557) bank100763 +100764 POINT(48.02788827421405 -122.94981469193809) bank100764 +100765 POINT(46.803363517682605 -121.89508107507275) bank100765 +100766 POINT(47.6894795384847 -121.68591806756316) bank100766 +100767 POINT(46.68846420594375 -121.51130909847814) bank100767 +100768 POINT(47.910124378490046 -121.99845154035206) bank100768 +100769 POINT(48.31503119331805 -122.26193926335768) bank100769 +100770 POINT(47.15443431064839 -123.31117006482778) bank100770 +100771 POINT(47.89405696469083 -123.24156981740322) bank100771 +100772 POINT(46.65366801772131 -122.13755720401589) bank100772 +100773 POINT(47.08417600283325 -122.79176564075104) bank100773 +100774 POINT(48.15014905106163 -122.2039727084302) bank100774 +100775 POINT(47.193974995479856 -123.04364446803903) bank100775 +100776 POINT(48.23017386572034 -121.95320548615742) bank100776 +100777 POINT(47.18824312264114 -122.19725798417329) bank100777 +100778 POINT(48.00375438287026 -122.74800606153913) bank100778 +100779 POINT(46.75790944313543 -121.3602361963673) bank100779 +100780 POINT(47.08183798155605 -123.22594392257832) bank100780 +100781 POINT(47.58420849427748 -122.89981735483956) bank100781 +100782 POINT(48.075453651122665 -122.7813295645866) bank100782 +100783 POINT(47.52688854861371 -122.48063491283793) bank100783 +100784 POINT(46.66248232497653 -122.91736431972402) bank100784 +100785 POINT(47.036494815933665 -122.63033192064258) bank100785 +100786 POINT(48.18493916372954 -122.5336802534651) bank100786 +100787 POINT(47.36102837637023 -123.2536705997262) bank100787 +100788 POINT(47.250041506913945 -121.98870044680594) bank100788 +100789 POINT(46.92327730775626 -122.77374114739635) bank100789 +100790 POINT(47.518714883997234 -122.01064397700793) bank100790 +100791 POINT(47.66670294754264 -121.72959259978967) bank100791 +100792 POINT(46.60791446744003 -122.61633366603162) bank100792 +100793 POINT(47.24049711644857 -123.20386530965557) bank100793 +100794 POINT(47.48738781429035 -122.10862701765637) bank100794 +100795 POINT(46.894678582499296 -121.36605898904624) bank100795 +100796 POINT(47.11609107680273 -121.340664782669) bank100796 +100797 POINT(47.84621353087344 -121.6695832481453) bank100797 +100798 POINT(48.02957127847099 -123.2510258269605) bank100798 +100799 POINT(48.52385309667203 -121.64108554952067) bank100799 +100800 POINT(48.37714727830064 -121.99622301144878) bank100800 +100801 POINT(48.41597138077722 -122.97847137473511) bank100801 +100802 POINT(47.358175567232884 -121.95807402506284) bank100802 +100803 POINT(47.45663215994538 -122.42408662888374) bank100803 +100804 POINT(47.385216983210775 -121.78649717163032) bank100804 +100805 POINT(46.951626978062436 -122.50453281901144) bank100805 +100806 POINT(46.88029142702449 -122.60099185415565) bank100806 +100807 POINT(46.86814148576199 -122.33660976744713) bank100807 +100808 POINT(46.673955361231705 -122.19386434027348) bank100808 +100809 POINT(47.12427417737558 -121.48498110709025) bank100809 +100810 POINT(48.31907653519738 -121.87604343646737) bank100810 +100811 POINT(46.77393424544563 -122.73331062631912) bank100811 +100812 POINT(48.40087034836499 -121.79192303988994) bank100812 +100813 POINT(47.04085411643517 -122.52944546604262) bank100813 +100814 POINT(46.659244654534135 -122.62785912759709) bank100814 +100815 POINT(47.432893439589364 -121.91303204938892) bank100815 +100816 POINT(47.283081666139545 -122.12061267675321) bank100816 +100817 POINT(48.40246886553515 -122.51342112806034) bank100817 +100818 POINT(46.7800209428125 -122.31161347828412) bank100818 +100819 POINT(48.024120377197896 -123.32054405020799) bank100819 +100820 POINT(47.66599774069326 -122.77762879141207) bank100820 +100821 POINT(48.170755179709225 -121.91782222595619) bank100821 +100822 POINT(48.23211392704111 -122.68323361656485) bank100822 +100823 POINT(47.025986498394296 -122.52308147509065) bank100823 +100824 POINT(48.232380600852714 -122.41235917217638) bank100824 +100825 POINT(46.71068049802775 -122.35128447604315) bank100825 +100826 POINT(48.27405812787182 -122.7686158749624) bank100826 +100827 POINT(46.76115312835302 -122.83332438642782) bank100827 +100828 POINT(46.90590832391593 -123.03077432156486) bank100828 +100829 POINT(46.72126397625785 -123.26499467759285) bank100829 +100830 POINT(47.31126020855795 -121.40539723957737) bank100830 +100831 POINT(48.295374254698096 -121.63239629619086) bank100831 +100832 POINT(47.22467208614073 -121.49972001051215) bank100832 +100833 POINT(47.305585827032615 -123.01321829429357) bank100833 +100834 POINT(47.3871206205262 -121.64458091534225) bank100834 +100835 POINT(48.284115677609925 -123.33039817541295) bank100835 +100836 POINT(47.46188151986044 -122.31918537715106) bank100836 +100837 POINT(47.90671813577246 -121.55777024604501) bank100837 +100838 POINT(48.456182925766186 -123.0240049633109) bank100838 +100839 POINT(47.42734016432182 -121.49255961477357) bank100839 +100840 POINT(46.78289147442902 -122.14449229644417) bank100840 +100841 POINT(47.53613307271924 -122.50564215116177) bank100841 +100842 POINT(46.86442523138965 -122.12900163478851) bank100842 +100843 POINT(47.48268184502975 -121.8348164078642) bank100843 +100844 POINT(47.22898686375617 -122.38017306132049) bank100844 +100845 POINT(46.810211953674845 -123.069426654864) bank100845 +100846 POINT(47.032839740844985 -121.65231338927987) bank100846 +100847 POINT(47.06076368565068 -121.34391913979849) bank100847 +100848 POINT(47.404051321230085 -122.75472290594874) bank100848 +100849 POINT(48.09325391719141 -121.35470028521998) bank100849 +100850 POINT(46.95177098149273 -121.60703717653948) bank100850 +100851 POINT(46.7784930916736 -121.33641864835828) bank100851 +100852 POINT(48.33029613667458 -122.50519562024886) bank100852 +100853 POINT(47.48422150347895 -122.49847260744973) bank100853 +100854 POINT(46.8087973207719 -122.76494672370617) bank100854 +100855 POINT(46.790686725576954 -122.53772816215064) bank100855 +100856 POINT(47.15950648077681 -122.33845346739547) bank100856 +100857 POINT(47.786561715424426 -121.53853569793962) bank100857 +100858 POINT(48.12093033575236 -122.28800295297948) bank100858 +100859 POINT(46.795132763124165 -121.89825799112845) bank100859 +100860 POINT(48.052156858899494 -121.55526881201882) bank100860 +100861 POINT(46.76830955317053 -123.21026822095973) bank100861 +100862 POINT(46.971833327833295 -122.95806414469538) bank100862 +100863 POINT(46.70130180555089 -123.19830900676925) bank100863 +100864 POINT(47.958997871969935 -122.67898050831113) bank100864 +100865 POINT(46.91739981247954 -123.13214804424905) bank100865 +100866 POINT(47.71657544531858 -122.67647992356822) bank100866 +100867 POINT(48.287140053719135 -121.88381240023112) bank100867 +100868 POINT(47.61269351514336 -121.63908108354774) bank100868 +100869 POINT(47.697570288239454 -122.98674865436092) bank100869 +100870 POINT(47.941619016373146 -121.9669567234415) bank100870 +100871 POINT(48.54356553826699 -121.69628813407674) bank100871 +100872 POINT(47.35132715412145 -121.66617906458633) bank100872 +100873 POINT(47.79806836385081 -122.11356797065548) bank100873 +100874 POINT(47.61136272488051 -121.4471181429691) bank100874 +100875 POINT(47.612346090470005 -121.52150613401956) bank100875 +100876 POINT(46.754760304910995 -122.83185408226208) bank100876 +100877 POINT(47.11132949832923 -122.30026895136588) bank100877 +100878 POINT(47.52962504041202 -122.51871062474426) bank100878 +100879 POINT(48.2898805195209 -122.08752928781455) bank100879 +100880 POINT(47.528807256357936 -123.0973132773951) bank100880 +100881 POINT(46.9056643560489 -123.00173998444541) bank100881 +100882 POINT(47.57370508730593 -121.59422948605027) bank100882 +100883 POINT(47.51850033373498 -122.64577175251245) bank100883 +100884 POINT(47.54932573546961 -123.3288417986969) bank100884 +100885 POINT(46.62321767074458 -123.06819995463582) bank100885 +100886 POINT(48.33277777220729 -121.54198888189704) bank100886 +100887 POINT(48.39127600742265 -122.8651323192727) bank100887 +100888 POINT(48.53143343228949 -122.71163739631703) bank100888 +100889 POINT(47.029708061817004 -122.02405409492107) bank100889 +100890 POINT(47.281180079428204 -122.5109899260454) bank100890 +100891 POINT(47.643745552117664 -122.02660330906971) bank100891 +100892 POINT(46.89052650690019 -122.20624397533449) bank100892 +100893 POINT(47.81282209211588 -122.18177321156206) bank100893 +100894 POINT(47.169585260429535 -121.79238539906645) bank100894 +100895 POINT(47.75086921912969 -122.23561524916028) bank100895 +100896 POINT(47.15827874153077 -121.84501255012788) bank100896 +100897 POINT(47.410568643397724 -121.92204747121144) bank100897 +100898 POINT(47.595747201849015 -122.96207109639369) bank100898 +100899 POINT(47.85636312309659 -121.72419636585094) bank100899 +100900 POINT(47.138520077051254 -122.01461485188132) bank100900 +100901 POINT(47.95789768225689 -121.76291197705372) bank100901 +100902 POINT(47.69120775067143 -122.7874462362687) bank100902 +100903 POINT(46.81971614611957 -122.1526817498197) bank100903 +100904 POINT(47.30574275182953 -123.14187577842112) bank100904 +100905 POINT(47.42812420868246 -122.5953327910409) bank100905 +100906 POINT(48.36521785774165 -121.35666395335902) bank100906 +100907 POINT(46.87648292536733 -121.92224978283339) bank100907 +100908 POINT(48.37063551815229 -122.73229458167164) bank100908 +100909 POINT(47.892448797265345 -121.83901484141715) bank100909 +100910 POINT(48.12684689249926 -121.66106436782165) bank100910 +100911 POINT(47.443189279042386 -122.09086694676076) bank100911 +100912 POINT(48.34039556997891 -121.39505867889957) bank100912 +100913 POINT(47.20769259883398 -122.84425648688581) bank100913 +100914 POINT(47.695111367971464 -121.47390617937951) bank100914 +100915 POINT(46.9267172593627 -123.05918131861561) bank100915 +100916 POINT(46.99670438509292 -122.46519856255168) bank100916 +100917 POINT(46.94513692387684 -121.67309264512501) bank100917 +100918 POINT(48.58208131465424 -122.53900081793593) bank100918 +100919 POINT(46.85537322542203 -123.22924088206378) bank100919 +100920 POINT(47.420093390704125 -121.97457829002249) bank100920 +100921 POINT(46.91434337495117 -122.00282938296101) bank100921 +100922 POINT(47.47644540876172 -121.9063341550993) bank100922 +100923 POINT(48.397742922076766 -123.02271812392597) bank100923 +100924 POINT(48.06333462808042 -122.13321827419018) bank100924 +100925 POINT(47.75087694068724 -122.90787239430477) bank100925 +100926 POINT(47.73562177519898 -122.26584870682468) bank100926 +100927 POINT(48.431730602166766 -122.43071119149997) bank100927 +100928 POINT(48.51222687015496 -121.58797993747073) bank100928 +100929 POINT(48.032677803882116 -121.65043001165002) bank100929 +100930 POINT(48.5875171492402 -122.74085504075168) bank100930 +100931 POINT(47.3521980449724 -122.11654584658358) bank100931 +100932 POINT(47.53368471981981 -121.68315561400352) bank100932 +100933 POINT(46.980086373095126 -122.2740060213693) bank100933 +100934 POINT(46.782189389385294 -121.42058372278022) bank100934 +100935 POINT(48.1636040134057 -122.97986710646502) bank100935 +100936 POINT(48.26227211402868 -122.23551123186787) bank100936 +100937 POINT(48.120996858360535 -122.59502032678546) bank100937 +100938 POINT(47.06414510899211 -122.65288116258012) bank100938 +100939 POINT(48.21226269266319 -121.7343226845627) bank100939 +100940 POINT(47.98279568717088 -122.69236473847866) bank100940 +100941 POINT(46.90484295420059 -121.38929989867407) bank100941 +100942 POINT(48.596760651921805 -123.18157143180349) bank100942 +100943 POINT(46.66639133286717 -121.33540408153858) bank100943 +100944 POINT(48.220701335756814 -122.31441903292851) bank100944 +100945 POINT(48.340867564031115 -121.38019560462601) bank100945 +100946 POINT(47.04638274925645 -121.74513939503014) bank100946 +100947 POINT(47.72309470364282 -122.57094366742646) bank100947 +100948 POINT(47.08651731679835 -121.74910417873238) bank100948 +100949 POINT(46.90873456886229 -122.04067482399839) bank100949 +100950 POINT(48.12427274623192 -122.63550067735673) bank100950 +100951 POINT(48.184536115281105 -121.67036814746643) bank100951 +100952 POINT(48.20261345754925 -123.24941310195251) bank100952 +100953 POINT(48.266167551817254 -121.37318179822239) bank100953 +100954 POINT(48.46629447530305 -123.28077253286789) bank100954 +100955 POINT(47.66006940721107 -123.00372176194931) bank100955 +100956 POINT(47.8742536941537 -122.61544060778411) bank100956 +100957 POINT(48.18267678177673 -123.28413077721177) bank100957 +100958 POINT(47.940849079494996 -122.93308906469768) bank100958 +100959 POINT(46.8847238408741 -121.52030776634403) bank100959 +100960 POINT(47.825653899181965 -121.55101148018308) bank100960 +100961 POINT(48.129922554265654 -122.27479865820251) bank100961 +100962 POINT(47.95506015502531 -122.68389002760813) bank100962 +100963 POINT(48.00581693506421 -123.12810265702078) bank100963 +100964 POINT(47.8091295203501 -122.11569550520719) bank100964 +100965 POINT(47.61611004984143 -122.37452418180894) bank100965 +100966 POINT(46.89196395130944 -122.82903526704735) bank100966 +100967 POINT(47.42809441520134 -122.05089829977939) bank100967 +100968 POINT(47.00702177850047 -123.13881712626355) bank100968 +100969 POINT(48.1736920241015 -121.42704134761368) bank100969 +100970 POINT(47.855630195794944 -122.74011791130584) bank100970 +100971 POINT(47.09772112468348 -121.5579555097039) bank100971 +100972 POINT(47.72416087474818 -122.3302918310216) bank100972 +100973 POINT(48.22961316322917 -121.48336581855214) bank100973 +100974 POINT(47.31297816182353 -122.4916680446137) bank100974 +100975 POINT(47.2429677076246 -122.06771672414158) bank100975 +100976 POINT(47.50218281135299 -122.81271818956488) bank100976 +100977 POINT(48.569126580215425 -122.13904038332605) bank100977 +100978 POINT(47.48950860683296 -122.02873579152042) bank100978 +100979 POINT(47.5534326008628 -122.80564015342344) bank100979 +100980 POINT(47.638412426747465 -122.96865835348311) bank100980 +100981 POINT(47.28407035516901 -122.51132854518168) bank100981 +100982 POINT(46.86227277172314 -121.86181490391685) bank100982 +100983 POINT(48.53045737901362 -121.48083881607552) bank100983 +100984 POINT(47.232631630650175 -122.36863849955847) bank100984 +100985 POINT(47.514602528562406 -121.4232580637762) bank100985 +100986 POINT(46.788899931667366 -122.61171846989357) bank100986 +100987 POINT(48.5638615209864 -121.723046972959) bank100987 +100988 POINT(47.69523155511592 -123.17855325351012) bank100988 +100989 POINT(48.02992764807412 -122.81545756341768) bank100989 +100990 POINT(48.59066874453135 -123.05393499972408) bank100990 +100991 POINT(48.40319323106047 -122.39703812201168) bank100991 +100992 POINT(48.00473745829533 -121.51824722651088) bank100992 +100993 POINT(47.15971729359823 -123.27194635616927) bank100993 +100994 POINT(48.26242544835558 -123.24625871824125) bank100994 +100995 POINT(47.03303405047687 -122.2009615943442) bank100995 +100996 POINT(47.97373752243603 -123.23231902634998) bank100996 +100997 POINT(47.84462238584787 -122.37645922218519) bank100997 +100998 POINT(47.78624094094109 -123.17316527383299) bank100998 +100999 POINT(47.97486875406393 -122.94414661612957) bank100999 +101000 POINT(48.250628691792805 -122.65524728034445) bank101000 +101001 POINT(47.42517309044928 -122.00660955445551) bank101001 +101002 POINT(47.30529770404895 -123.06931267899566) bank101002 +101003 POINT(47.72522578426587 -121.37514688912597) bank101003 +101004 POINT(47.14098927935251 -122.6445611539319) bank101004 +101005 POINT(48.07753981199589 -122.4236896363571) bank101005 +101006 POINT(46.814844634109356 -123.20822877290763) bank101006 +101007 POINT(48.194814304452564 -121.36251656809252) bank101007 +101008 POINT(46.68047650455686 -122.13559910078007) bank101008 +101009 POINT(48.49352652974384 -122.04348020938903) bank101009 +101010 POINT(46.82600627155246 -121.6463120230595) bank101010 +101011 POINT(47.40120425968541 -122.42572571446048) bank101011 +101012 POINT(46.92038896957225 -123.03405028827255) bank101012 +101013 POINT(47.5474483840384 -121.56915985283072) bank101013 +101014 POINT(48.56123487663494 -121.98307822845192) bank101014 +101015 POINT(48.37218850592708 -122.02592055712745) bank101015 +101016 POINT(46.69364396905049 -122.4594031639041) bank101016 +101017 POINT(47.77568662386913 -121.77118953846299) bank101017 +101018 POINT(48.28485884138359 -122.78490798321815) bank101018 +101019 POINT(47.51891593291808 -122.27997755632238) bank101019 +101020 POINT(47.683332323695744 -122.4407396669625) bank101020 +101021 POINT(47.8925820610426 -122.50845622477418) bank101021 +101022 POINT(47.89071827086839 -121.70210967731306) bank101022 +101023 POINT(47.00093620007339 -123.30918682232206) bank101023 +101024 POINT(48.368933211145595 -122.45743441310695) bank101024 +101025 POINT(48.44013474428051 -122.91125915200332) bank101025 +101026 POINT(46.61249085473305 -121.83311963408497) bank101026 +101027 POINT(48.44622007259071 -121.49463926819669) bank101027 +101028 POINT(47.39089670915177 -123.32250583198956) bank101028 +101029 POINT(48.07442254637398 -121.4111777212075) bank101029 +101030 POINT(47.13231298478669 -122.92895907119328) bank101030 +101031 POINT(47.915610489828886 -122.05643906829124) bank101031 +101032 POINT(47.534079748613216 -123.08418478795073) bank101032 +101033 POINT(46.98954668315245 -122.21655765206845) bank101033 +101034 POINT(48.15610929452748 -122.71351423800695) bank101034 +101035 POINT(48.307369561050756 -122.45530408623499) bank101035 +101036 POINT(47.98910749259787 -122.21475560191986) bank101036 +101037 POINT(47.92979099181173 -121.63371585605444) bank101037 +101038 POINT(47.88736012524487 -121.47065936386582) bank101038 +101039 POINT(48.40353048514459 -122.91516593680434) bank101039 +101040 POINT(48.069029646787726 -123.03171761530065) bank101040 +101041 POINT(48.283254743136304 -122.70427127923769) bank101041 +101042 POINT(48.08946928890801 -121.42845827583034) bank101042 +101043 POINT(48.100603940853475 -122.95904956033489) bank101043 +101044 POINT(46.91350906696449 -122.6075548937129) bank101044 +101045 POINT(47.01566459515155 -122.67527522705434) bank101045 +101046 POINT(47.25924976850566 -122.46773277267859) bank101046 +101047 POINT(48.53498452725615 -122.9301325511795) bank101047 +101048 POINT(47.568385279641475 -121.56788434453391) bank101048 +101049 POINT(48.57689189677806 -122.58228802133178) bank101049 +101050 POINT(48.42664020396798 -122.19726803608079) bank101050 +101051 POINT(47.142960260474176 -122.65249049412122) bank101051 +101052 POINT(47.37549163502557 -122.88784570016271) bank101052 +101053 POINT(46.799042541983106 -122.98660499928467) bank101053 +101054 POINT(46.70392931305676 -122.83938529082721) bank101054 +101055 POINT(47.43351580345948 -122.31729041590597) bank101055 +101056 POINT(47.047451735632706 -123.14611173419539) bank101056 +101057 POINT(46.993785312482515 -121.40283507548016) bank101057 +101058 POINT(46.7440747999905 -123.04376320027906) bank101058 +101059 POINT(47.77706772776302 -121.66293668990409) bank101059 +101060 POINT(48.13943865708603 -123.32101903362873) bank101060 +101061 POINT(46.68677205579992 -122.97914618190585) bank101061 +101062 POINT(48.48106912150171 -121.45583794424276) bank101062 +101063 POINT(47.72443536951453 -121.62366854272393) bank101063 +101064 POINT(46.842562298850936 -121.64478075869494) bank101064 +101065 POINT(48.54891230459501 -123.05484357645356) bank101065 +101066 POINT(47.19703603655386 -121.3888586095302) bank101066 +101067 POINT(47.38978000601669 -122.29844951754325) bank101067 +101068 POINT(48.33486883873604 -121.98036377126985) bank101068 +101069 POINT(47.107278284314106 -122.46004174699172) bank101069 +101070 POINT(47.35356761688303 -121.99872073311971) bank101070 +101071 POINT(46.96892138503946 -121.47330843458613) bank101071 +101072 POINT(47.25048347270904 -123.30618729279408) bank101072 +101073 POINT(46.94820059106023 -121.37324425768335) bank101073 +101074 POINT(46.91648919382051 -121.70799252772902) bank101074 +101075 POINT(47.88737612136427 -123.06320572063888) bank101075 +101076 POINT(48.18696772626861 -123.01693006574683) bank101076 +101077 POINT(48.44522371719979 -121.886956482581) bank101077 +101078 POINT(47.63457067697354 -121.49247537369521) bank101078 +101079 POINT(47.61060257079943 -121.77524758787065) bank101079 +101080 POINT(47.64880853486279 -122.32405007415359) bank101080 +101081 POINT(48.507481245544696 -123.03492415452514) bank101081 +101082 POINT(48.39182034587146 -122.82686861607051) bank101082 +101083 POINT(48.27186331048113 -122.26345608985915) bank101083 +101084 POINT(46.697240172627644 -122.21909259175439) bank101084 +101085 POINT(47.65492475354076 -122.21416387061153) bank101085 +101086 POINT(48.193161044914454 -122.97551000928952) bank101086 +101087 POINT(47.27928602586327 -121.97096664867985) bank101087 +101088 POINT(46.63907261563613 -122.35458077528644) bank101088 +101089 POINT(48.46972193807567 -122.90937702125548) bank101089 +101090 POINT(47.10987394759199 -121.70167119807748) bank101090 +101091 POINT(46.84267334226827 -121.47022811260855) bank101091 +101092 POINT(47.72068799984798 -122.74018929048898) bank101092 +101093 POINT(48.39185615458869 -122.09613208080417) bank101093 +101094 POINT(47.53767833320327 -122.95442414585511) bank101094 +101095 POINT(47.073889442453215 -122.32714643210805) bank101095 +101096 POINT(47.787704139768245 -122.06455410137508) bank101096 +101097 POINT(47.390837126112736 -121.49342025519346) bank101097 +101098 POINT(47.421012782433266 -122.44150401227502) bank101098 +101099 POINT(47.072187251716684 -121.36275404821308) bank101099 +101100 POINT(47.74486298089822 -122.27780090915236) bank101100 +101101 POINT(47.96513002387977 -121.8595984347633) bank101101 +101102 POINT(47.49465808939479 -121.70801392364173) bank101102 +101103 POINT(48.503762187294726 -122.0714621076405) bank101103 +101104 POINT(48.321451596302765 -122.13035702720009) bank101104 +101105 POINT(46.91862856487292 -122.01745210904494) bank101105 +101106 POINT(47.25230419970112 -122.7492740473445) bank101106 +101107 POINT(47.26545063958118 -122.41806628329778) bank101107 +101108 POINT(47.061675186591295 -122.58720824112775) bank101108 +101109 POINT(48.55718831022333 -121.78599817271186) bank101109 +101110 POINT(46.69064003717939 -122.75607039716667) bank101110 +101111 POINT(47.998933813957315 -122.62754983952334) bank101111 +101112 POINT(47.99884959123036 -122.10898942709673) bank101112 +101113 POINT(47.436287325468264 -122.01574712239277) bank101113 +101114 POINT(47.701915413503585 -121.51522007574518) bank101114 +101115 POINT(47.25600736371788 -122.2508672527505) bank101115 +101116 POINT(48.56690640182371 -121.58573011312173) bank101116 +101117 POINT(47.364004824002606 -121.34106935250526) bank101117 +101118 POINT(47.89511765075 -122.59540933474845) bank101118 +101119 POINT(47.27082311710564 -122.10865272661846) bank101119 +101120 POINT(47.15048629954629 -121.48367497362084) bank101120 +101121 POINT(46.65198142494663 -122.11215347166932) bank101121 +101122 POINT(47.69389669662885 -122.02569855156851) bank101122 +101123 POINT(47.40561090679628 -122.51114909495485) bank101123 +101124 POINT(46.72786286456996 -122.52582356387775) bank101124 +101125 POINT(47.384407823194 -122.58707394451201) bank101125 +101126 POINT(48.14242902243822 -121.43609641223718) bank101126 +101127 POINT(48.59945935446117 -121.74343793072636) bank101127 +101128 POINT(47.39999640440319 -122.38068331027154) bank101128 +101129 POINT(48.596049448656025 -121.52038295431728) bank101129 +101130 POINT(48.322790560511244 -123.11238558697322) bank101130 +101131 POINT(48.29430362605589 -122.84942803318765) bank101131 +101132 POINT(47.82685458906344 -122.16451778176074) bank101132 +101133 POINT(46.65581699928889 -122.10588531893154) bank101133 +101134 POINT(47.758098057917834 -123.28756240448858) bank101134 +101135 POINT(47.917790398436935 -122.85868534075311) bank101135 +101136 POINT(46.64968468042995 -123.13400005135423) bank101136 +101137 POINT(46.94930229452578 -122.16314013413812) bank101137 +101138 POINT(48.50386419411332 -121.8832643986891) bank101138 +101139 POINT(47.08847688002635 -122.75668977483116) bank101139 +101140 POINT(47.89116262319325 -122.50188489732287) bank101140 +101141 POINT(46.990215128326646 -122.79307864473465) bank101141 +101142 POINT(47.96315257137431 -122.23429693673329) bank101142 +101143 POINT(47.58523034848128 -121.98909396161082) bank101143 +101144 POINT(46.86530356687664 -123.28177179064508) bank101144 +101145 POINT(47.26520955019423 -122.92140660617198) bank101145 +101146 POINT(48.04845216259841 -122.81200786486333) bank101146 +101147 POINT(47.387956891194605 -122.58789088453302) bank101147 +101148 POINT(46.907661685107975 -122.91345088988858) bank101148 +101149 POINT(47.191948585068346 -121.57842674795639) bank101149 +101150 POINT(47.076763414446276 -121.88498110625555) bank101150 +101151 POINT(47.74013038489772 -121.5859020574479) bank101151 +101152 POINT(48.28582707773412 -123.05395515471052) bank101152 +101153 POINT(47.81972038880735 -121.52871811398403) bank101153 +101154 POINT(46.75155454784682 -121.92503768173998) bank101154 +101155 POINT(47.95363282757997 -122.00719446896073) bank101155 +101156 POINT(47.59564110967537 -121.98698544023281) bank101156 +101157 POINT(47.67241228357447 -121.7760464514338) bank101157 +101158 POINT(47.138555734619175 -122.84078938716563) bank101158 +101159 POINT(47.23930900624755 -122.6407288579208) bank101159 +101160 POINT(47.07206774145101 -121.82769549302502) bank101160 +101161 POINT(47.27755338946913 -122.10948195787745) bank101161 +101162 POINT(47.17206714988888 -122.6223617803392) bank101162 +101163 POINT(48.592930372403224 -123.25758757338508) bank101163 +101164 POINT(47.04773145691207 -121.91697911617267) bank101164 +101165 POINT(47.6280288581033 -121.99785721236863) bank101165 +101166 POINT(47.981259597538255 -123.33130442441346) bank101166 +101167 POINT(46.83812159454857 -123.1160875551696) bank101167 +101168 POINT(47.05079383769044 -121.95029294842182) bank101168 +101169 POINT(47.54231326702546 -121.44244472157015) bank101169 +101170 POINT(48.272563612057574 -121.9616577017086) bank101170 +101171 POINT(47.567843028660086 -122.43577156205161) bank101171 +101172 POINT(47.57808432465503 -123.17615415096472) bank101172 +101173 POINT(48.511829221048885 -123.02580775807697) bank101173 +101174 POINT(46.945449683508386 -122.8723862794649) bank101174 +101175 POINT(46.70028590092412 -123.318501133443) bank101175 +101176 POINT(47.26280015152894 -122.69812859092328) bank101176 +101177 POINT(48.41772112171448 -123.25196277424511) bank101177 +101178 POINT(47.73410773110185 -123.32653428479242) bank101178 +101179 POINT(47.75379315577477 -122.10254502468719) bank101179 +101180 POINT(47.2471194558066 -123.07117901687974) bank101180 +101181 POINT(47.5305506141151 -123.21142164169272) bank101181 +101182 POINT(47.736378197307296 -122.50164163184589) bank101182 +101183 POINT(46.859292844946495 -122.61394574078899) bank101183 +101184 POINT(48.108896943860394 -122.35344227591658) bank101184 +101185 POINT(47.445125773492 -121.53073608962919) bank101185 +101186 POINT(47.22433387631372 -123.14223490224738) bank101186 +101187 POINT(47.707882384061044 -122.95428420619965) bank101187 +101188 POINT(46.71312257014668 -122.43218618702663) bank101188 +101189 POINT(46.7190147903011 -122.4181442238076) bank101189 +101190 POINT(48.303742808613954 -122.37490877921597) bank101190 +101191 POINT(48.500057349371126 -122.77963328490819) bank101191 +101192 POINT(47.49899895313647 -122.87057196919311) bank101192 +101193 POINT(47.39991027188337 -121.58175723588327) bank101193 +101194 POINT(46.72219046722261 -122.57394953828224) bank101194 +101195 POINT(47.423352032106635 -122.22536514335285) bank101195 +101196 POINT(47.56013456331404 -122.83952485562732) bank101196 +101197 POINT(48.33556535270803 -122.28466236392902) bank101197 +101198 POINT(47.557569193624765 -122.01802683250759) bank101198 +101199 POINT(47.032884666975654 -122.92550894649332) bank101199 +101200 POINT(46.70190898718995 -121.37781045790044) bank101200 +101201 POINT(46.78382406075337 -121.82379329177965) bank101201 +101202 POINT(48.425945064451 -122.45144448981999) bank101202 +101203 POINT(47.28147644654793 -122.6125090791145) bank101203 +101204 POINT(48.11887694460343 -121.97843035330524) bank101204 +101205 POINT(46.8051856363246 -122.29910277720892) bank101205 +101206 POINT(47.038593249348175 -122.11128720380289) bank101206 +101207 POINT(47.09349494388694 -122.083070375863) bank101207 +101208 POINT(48.57447343660637 -122.57661203915359) bank101208 +101209 POINT(47.61459045809871 -122.35677379005205) bank101209 +101210 POINT(48.43513044152838 -122.18195887083142) bank101210 +101211 POINT(47.76862932577855 -122.26452124106873) bank101211 +101212 POINT(48.447812603688604 -121.880429991834) bank101212 +101213 POINT(47.64048224217953 -122.53541672755418) bank101213 +101214 POINT(47.884179021908665 -122.88688542784202) bank101214 +101215 POINT(48.434916656767385 -123.32825199522541) bank101215 +101216 POINT(48.081986030346854 -121.53255087671835) bank101216 +101217 POINT(46.67072074085103 -122.65501555347265) bank101217 +101218 POINT(47.103377498477464 -122.03004587168002) bank101218 +101219 POINT(47.4045318447625 -121.72803869454253) bank101219 +101220 POINT(47.4157878855191 -121.55034592568396) bank101220 +101221 POINT(46.641572863610406 -122.96566300105971) bank101221 +101222 POINT(47.98342416810141 -122.8916015091224) bank101222 +101223 POINT(46.739254585963906 -123.29045272783738) bank101223 +101224 POINT(47.83789776490967 -123.27416915645111) bank101224 +101225 POINT(48.22608035069808 -122.60785015394733) bank101225 +101226 POINT(47.94080674130726 -121.7588938341442) bank101226 +101227 POINT(47.885051638912834 -123.28687153885424) bank101227 +101228 POINT(46.847243265130196 -122.46375547404925) bank101228 +101229 POINT(47.118627989224066 -122.7982487725973) bank101229 +101230 POINT(48.327530425476844 -122.86432422976534) bank101230 +101231 POINT(47.81208299931125 -121.85482064998317) bank101231 +101232 POINT(47.95057610558832 -122.00622632268073) bank101232 +101233 POINT(46.743870308674346 -122.00647622645008) bank101233 +101234 POINT(47.10366800836899 -122.89551606602524) bank101234 +101235 POINT(46.60721325609822 -122.1769900895132) bank101235 +101236 POINT(48.53189868244521 -121.949760313504) bank101236 +101237 POINT(48.38810879366981 -123.03982198599246) bank101237 +101238 POINT(47.861960882759 -122.2060130284075) bank101238 +101239 POINT(48.30368745817965 -122.6150245622358) bank101239 +101240 POINT(48.37854277746233 -123.03191834515363) bank101240 +101241 POINT(47.810794008036325 -123.07122237378388) bank101241 +101242 POINT(47.082064926210926 -122.11933800191132) bank101242 +101243 POINT(47.52461716770936 -122.34337103277956) bank101243 +101244 POINT(47.92833829824036 -121.82554727413441) bank101244 +101245 POINT(47.06062340340573 -122.38776704579955) bank101245 +101246 POINT(48.19100519874806 -121.80909138329105) bank101246 +101247 POINT(47.96387092369471 -121.47724969529312) bank101247 +101248 POINT(47.52966020413719 -122.41768048692634) bank101248 +101249 POINT(48.46555227260785 -121.57002638978108) bank101249 +101250 POINT(48.50858977171087 -122.83288472890844) bank101250 +101251 POINT(48.49300620050051 -121.57158006008831) bank101251 +101252 POINT(48.096115550911904 -123.1448193553633) bank101252 +101253 POINT(47.68116879124963 -123.32622592189816) bank101253 +101254 POINT(47.23856326637387 -122.46917907737101) bank101254 +101255 POINT(47.98850822036263 -122.66600746843926) bank101255 +101256 POINT(46.61958772168411 -122.03021694149558) bank101256 +101257 POINT(47.550320674463364 -121.38338190526925) bank101257 +101258 POINT(48.45892723567458 -121.50648286374629) bank101258 +101259 POINT(46.994062223083795 -121.47240045167266) bank101259 +101260 POINT(47.02307310268116 -121.83546595504853) bank101260 +101261 POINT(46.70661693581087 -121.37113359217757) bank101261 +101262 POINT(47.18732350131107 -122.47509157822812) bank101262 +101263 POINT(47.17669937950893 -122.85288522395476) bank101263 +101264 POINT(48.33276035080926 -121.86931510827796) bank101264 +101265 POINT(47.12887231427895 -123.26025143717386) bank101265 +101266 POINT(48.40963676829669 -122.45701494811347) bank101266 +101267 POINT(47.14822521214896 -123.11367645247388) bank101267 +101268 POINT(47.976562802343125 -122.19199762729457) bank101268 +101269 POINT(48.48344210798927 -123.17078637295303) bank101269 +101270 POINT(46.81316018348185 -122.10063807757403) bank101270 +101271 POINT(47.333969741303626 -121.88104159246444) bank101271 +101272 POINT(47.22311835388914 -122.81129011328125) bank101272 +101273 POINT(47.61222736729723 -121.84482701591351) bank101273 +101274 POINT(48.056842196098145 -121.77396745278199) bank101274 +101275 POINT(47.58626358091619 -122.32554435196356) bank101275 +101276 POINT(47.58362231189703 -121.82839743549779) bank101276 +101277 POINT(46.881947415610796 -121.82423297632708) bank101277 +101278 POINT(47.39159656406485 -122.16051764832913) bank101278 +101279 POINT(47.30406836289007 -122.50359028759645) bank101279 +101280 POINT(48.47833070427809 -122.44928751593223) bank101280 +101281 POINT(47.82264087791103 -123.04036760562809) bank101281 +101282 POINT(46.6856213091974 -121.38617239421473) bank101282 +101283 POINT(47.47673723513167 -122.4110241698616) bank101283 +101284 POINT(47.68254633874393 -123.29575872212123) bank101284 +101285 POINT(47.67739065396535 -121.46283161441072) bank101285 +101286 POINT(46.792007036929085 -122.95663588671428) bank101286 +101287 POINT(48.4192871827823 -121.48356376546774) bank101287 +101288 POINT(48.003701462707866 -122.46228434375281) bank101288 +101289 POINT(47.317198928953125 -122.14591177020347) bank101289 +101290 POINT(48.11584947315494 -122.05449677691979) bank101290 +101291 POINT(47.769067831282825 -122.2746294406693) bank101291 +101292 POINT(48.10938388354713 -121.96303541067954) bank101292 +101293 POINT(47.06600754508741 -121.6800468935551) bank101293 +101294 POINT(48.53874009699366 -122.91007629866094) bank101294 +101295 POINT(47.85798077617644 -122.62613814005597) bank101295 +101296 POINT(47.47209408549788 -121.60835749441387) bank101296 +101297 POINT(48.09125048261921 -121.78623448214697) bank101297 +101298 POINT(47.15561306757584 -123.00109019380675) bank101298 +101299 POINT(48.4819731304562 -122.59383377349725) bank101299 +101300 POINT(47.73025677370635 -122.87330639019545) bank101300 +101301 POINT(46.60984208597563 -122.66593353608805) bank101301 +101302 POINT(48.31391761171266 -121.8875069668023) bank101302 +101303 POINT(47.85326971044785 -122.67280779967763) bank101303 +101304 POINT(47.125189199652986 -121.51407585000024) bank101304 +101305 POINT(47.7957849612481 -122.24919617979712) bank101305 +101306 POINT(47.28220817715091 -121.81445455730478) bank101306 +101307 POINT(47.02269009652624 -121.3482373230926) bank101307 +101308 POINT(46.767787476683885 -122.25017120876088) bank101308 +101309 POINT(48.53400032694678 -122.29764251774407) bank101309 +101310 POINT(46.62051579945497 -122.58061142304578) bank101310 +101311 POINT(47.96510723923604 -121.34959429059181) bank101311 +101312 POINT(47.68810260121876 -122.68343070956688) bank101312 +101313 POINT(48.04871951612439 -122.2737470635495) bank101313 +101314 POINT(46.64762804322658 -121.85331403324295) bank101314 +101315 POINT(46.719085323740124 -122.94568917440023) bank101315 +101316 POINT(47.139055676078975 -122.57355316417771) bank101316 +101317 POINT(46.787611417974404 -122.6582832613173) bank101317 +101318 POINT(48.58964467080297 -122.21209564623337) bank101318 +101319 POINT(47.665856218752445 -121.91164582430713) bank101319 +101320 POINT(47.112787215023445 -122.86614758548978) bank101320 +101321 POINT(47.66399059926889 -123.2471489932984) bank101321 +101322 POINT(47.78294380244667 -123.16111037950346) bank101322 +101323 POINT(47.13628229265662 -121.8158222252792) bank101323 +101324 POINT(48.0873424228482 -122.40677969654801) bank101324 +101325 POINT(48.41089342484315 -122.31664155648188) bank101325 +101326 POINT(47.936596891561514 -122.95519217018793) bank101326 +101327 POINT(48.529200567599624 -121.34565509923807) bank101327 +101328 POINT(47.6145851637907 -121.95023151942739) bank101328 +101329 POINT(47.32593293098913 -122.55873428581678) bank101329 +101330 POINT(47.85713775962502 -121.91405739733968) bank101330 +101331 POINT(47.62997196048917 -122.17315088930182) bank101331 +101332 POINT(47.71801485701001 -122.05769872420251) bank101332 +101333 POINT(47.95523371110727 -123.14690394849436) bank101333 +101334 POINT(46.68162613447737 -121.98931670285728) bank101334 +101335 POINT(46.793610526061016 -123.32752760224129) bank101335 +101336 POINT(47.53718760700328 -122.47442149843337) bank101336 +101337 POINT(46.77324527934139 -121.64122409105808) bank101337 +101338 POINT(47.5022633310569 -121.51914018421996) bank101338 +101339 POINT(47.81941710475579 -121.76757756646907) bank101339 +101340 POINT(48.00508940357009 -121.607648510167) bank101340 +101341 POINT(48.16369647736923 -122.3763234530305) bank101341 +101342 POINT(48.030038351989134 -122.06323252417296) bank101342 +101343 POINT(47.28802493023615 -121.83694607396065) bank101343 +101344 POINT(47.113591235785925 -121.94666437068226) bank101344 +101345 POINT(46.72239631481202 -122.99235833416006) bank101345 +101346 POINT(48.405244844626864 -121.4014831398441) bank101346 +101347 POINT(47.452738224697626 -123.04093656409243) bank101347 +101348 POINT(47.29308844329338 -122.317915424068) bank101348 +101349 POINT(46.686299240830934 -121.65805749048982) bank101349 +101350 POINT(47.25876848481258 -122.62903455041854) bank101350 +101351 POINT(47.04229231471467 -121.61549291493915) bank101351 +101352 POINT(48.148832822451325 -122.39060713877447) bank101352 +101353 POINT(46.860977520250046 -122.8634210093222) bank101353 +101354 POINT(48.20322314753781 -122.8932282085394) bank101354 +101355 POINT(46.78487952247331 -121.84875750031436) bank101355 +101356 POINT(48.034216456419195 -121.86308902437305) bank101356 +101357 POINT(48.471889503265885 -122.21433085095475) bank101357 +101358 POINT(47.31613018516987 -121.90616617874085) bank101358 +101359 POINT(46.9501119374434 -122.86207120612875) bank101359 +101360 POINT(47.66966460733618 -123.23755675656817) bank101360 +101361 POINT(48.60151632374372 -122.27211698419991) bank101361 +101362 POINT(47.309252893362235 -123.3250332498152) bank101362 +101363 POINT(48.230198814941396 -121.70071808771154) bank101363 +101364 POINT(47.62123711092465 -121.86070687628924) bank101364 +101365 POINT(47.82937893027936 -121.45397898436782) bank101365 +101366 POINT(47.68608678028969 -121.40082307012621) bank101366 +101367 POINT(47.61695766953166 -121.80946847636989) bank101367 +101368 POINT(47.30905655092938 -123.07752878061314) bank101368 +101369 POINT(48.10140042341909 -121.88056488239732) bank101369 +101370 POINT(47.55841582344687 -121.3628251746179) bank101370 +101371 POINT(47.39583392928775 -121.5894948690633) bank101371 +101372 POINT(47.08107573185148 -122.83423665557251) bank101372 +101373 POINT(46.71743417974364 -121.85007170682665) bank101373 +101374 POINT(47.19313145958198 -121.40112815149118) bank101374 +101375 POINT(47.043720700308064 -122.92534618595279) bank101375 +101376 POINT(47.05532345920954 -121.74832629761435) bank101376 +101377 POINT(47.91840594196829 -123.29853996140544) bank101377 +101378 POINT(47.787561782378376 -121.58462982698373) bank101378 +101379 POINT(47.75999703119611 -122.52650790920872) bank101379 +101380 POINT(47.69323816912375 -122.82411625036467) bank101380 +101381 POINT(48.20408671938948 -122.36131926979705) bank101381 +101382 POINT(47.82012406330337 -122.34925876293714) bank101382 +101383 POINT(46.77368317416644 -121.97484677574136) bank101383 +101384 POINT(47.07736594154092 -121.59307817829992) bank101384 +101385 POINT(46.86178673633673 -122.86003019217345) bank101385 +101386 POINT(47.170442567942274 -122.46545259207065) bank101386 +101387 POINT(48.11199737878121 -123.16624780337142) bank101387 +101388 POINT(47.99784371252368 -122.75450445612165) bank101388 +101389 POINT(48.44205453330595 -123.24663178174042) bank101389 +101390 POINT(47.94383859527368 -121.61358333736536) bank101390 +101391 POINT(48.57959435767751 -121.33232151364021) bank101391 +101392 POINT(46.90662020989007 -121.76352808060999) bank101392 +101393 POINT(47.18201938224717 -121.63038694662332) bank101393 +101394 POINT(46.76310244559723 -121.99039391022599) bank101394 +101395 POINT(48.42684912990419 -122.96251110049049) bank101395 +101396 POINT(46.61407586087672 -121.9772865760718) bank101396 +101397 POINT(48.3189996564298 -121.77859307178406) bank101397 +101398 POINT(47.37360197200979 -122.14289244093507) bank101398 +101399 POINT(47.94396202285812 -121.66081266954396) bank101399 +101400 POINT(47.06351078368741 -122.37884767250027) bank101400 +101401 POINT(47.84158141097565 -122.61246621503481) bank101401 +101402 POINT(48.165200725611136 -123.03149515255241) bank101402 +101403 POINT(47.314934132310135 -121.50580513505415) bank101403 +101404 POINT(46.80591526974821 -122.47602858471197) bank101404 +101405 POINT(46.89300516246872 -123.32453077844822) bank101405 +101406 POINT(47.27694836901523 -123.2568304544495) bank101406 +101407 POINT(48.56722029084405 -121.97364750291) bank101407 +101408 POINT(47.903716056420755 -122.54005221628557) bank101408 +101409 POINT(47.49180684211477 -123.05904535522562) bank101409 +101410 POINT(48.20970217355261 -121.48304915273985) bank101410 +101411 POINT(47.78694870938965 -122.44351962013025) bank101411 +101412 POINT(48.26364732713399 -122.08729433218336) bank101412 +101413 POINT(47.12171758361271 -123.13709825845085) bank101413 +101414 POINT(47.5764217992174 -121.80365044838445) bank101414 +101415 POINT(47.757215078470686 -122.54180027649507) bank101415 +101416 POINT(47.297795147275075 -122.89789707680822) bank101416 +101417 POINT(48.01433548199739 -122.49844307626007) bank101417 +101418 POINT(48.5612415137256 -121.82165273296401) bank101418 +101419 POINT(47.27768263604252 -123.16610744576651) bank101419 +101420 POINT(48.117072363756584 -121.6807459868849) bank101420 +101421 POINT(46.98588374494082 -121.75828107789421) bank101421 +101422 POINT(46.79953701892706 -121.87576401655052) bank101422 +101423 POINT(47.34990945004029 -121.46909593786853) bank101423 +101424 POINT(48.1256403312815 -122.88167385556976) bank101424 +101425 POINT(48.15835127335455 -123.07099313596122) bank101425 +101426 POINT(48.32499762445389 -123.32532032407264) bank101426 +101427 POINT(47.70138144723936 -121.88383865823454) bank101427 +101428 POINT(47.38426773859725 -122.81861427041527) bank101428 +101429 POINT(47.48124431509366 -122.75307393736263) bank101429 +101430 POINT(48.1633859929058 -123.06004442411833) bank101430 +101431 POINT(48.06953144029239 -122.39469515947663) bank101431 +101432 POINT(48.100618182216486 -122.89847645219005) bank101432 +101433 POINT(47.210006196453996 -123.15835193351747) bank101433 +101434 POINT(46.76738228043372 -121.78762105370961) bank101434 +101435 POINT(47.25124613560935 -123.21001852105714) bank101435 +101436 POINT(48.09871821514419 -122.32870154383183) bank101436 +101437 POINT(47.07034490632067 -122.35569019201283) bank101437 +101438 POINT(46.90640872186107 -122.20352497471899) bank101438 +101439 POINT(47.56370140643422 -123.11077060169143) bank101439 +101440 POINT(47.18798286166997 -121.4860079227977) bank101440 +101441 POINT(47.66093573959208 -122.39151814870877) bank101441 +101442 POINT(47.27138591389199 -121.61816429752402) bank101442 +101443 POINT(48.1592039091486 -122.30207285993056) bank101443 +101444 POINT(47.95651084886869 -121.38410100570532) bank101444 +101445 POINT(47.78176208596064 -122.13184397198793) bank101445 +101446 POINT(47.34084918711695 -123.27888959958113) bank101446 +101447 POINT(48.26055880478089 -122.86004167914912) bank101447 +101448 POINT(48.243484794791655 -121.9927444275758) bank101448 +101449 POINT(48.32331367342059 -121.53553594804676) bank101449 +101450 POINT(48.01287845233115 -121.33681143585522) bank101450 +101451 POINT(47.247401288327886 -121.89711253337916) bank101451 +101452 POINT(48.05236150951696 -122.25997224652473) bank101452 +101453 POINT(47.27379518014833 -121.86975442149168) bank101453 +101454 POINT(47.0407869661692 -122.69914921623592) bank101454 +101455 POINT(48.38930885167962 -122.84643569157399) bank101455 +101456 POINT(46.956300489047436 -122.99650982021399) bank101456 +101457 POINT(48.578833587168134 -122.4068204716507) bank101457 +101458 POINT(48.02708286104518 -122.70463076986852) bank101458 +101459 POINT(47.093351849392384 -123.10542503859182) bank101459 +101460 POINT(47.45061692483279 -121.47315123441135) bank101460 +101461 POINT(48.08813334822932 -122.22260249861785) bank101461 +101462 POINT(47.58945970091082 -121.71545033830586) bank101462 +101463 POINT(46.78317041508553 -121.94214894161163) bank101463 +101464 POINT(46.68669115136592 -121.74322224156337) bank101464 +101465 POINT(47.05090950458182 -122.9201219617198) bank101465 +101466 POINT(47.50868505238591 -122.3074025191779) bank101466 +101467 POINT(48.08775062235267 -121.60885379302489) bank101467 +101468 POINT(47.86924613353183 -122.89055263575491) bank101468 +101469 POINT(48.47017073402738 -122.94689183944024) bank101469 +101470 POINT(46.964345476789234 -122.83263612442005) bank101470 +101471 POINT(47.70673856285891 -122.54542355549566) bank101471 +101472 POINT(48.042751196511894 -123.33136851086188) bank101472 +101473 POINT(47.316605307371155 -123.0033957586265) bank101473 +101474 POINT(46.8489373894119 -121.73837748064436) bank101474 +101475 POINT(47.87994232043217 -122.38630741092499) bank101475 +101476 POINT(48.28453174791352 -122.38316129496056) bank101476 +101477 POINT(47.92048094263957 -122.83789348650306) bank101477 +101478 POINT(48.04432810596818 -121.58728532349399) bank101478 +101479 POINT(46.75080645783063 -122.26848167182979) bank101479 +101480 POINT(47.62194211597144 -122.29510773665261) bank101480 +101481 POINT(47.03174994360197 -121.55084061206585) bank101481 +101482 POINT(47.35478760809153 -121.91895405901742) bank101482 +101483 POINT(47.54223258884149 -121.60360055908832) bank101483 +101484 POINT(48.38577697791728 -123.24544311726487) bank101484 +101485 POINT(47.60605310792625 -122.66572306784647) bank101485 +101486 POINT(47.464805345677625 -122.58333092192352) bank101486 +101487 POINT(46.87300012331773 -122.0753664101845) bank101487 +101488 POINT(48.46918880933037 -121.82686042998733) bank101488 +101489 POINT(48.13898465012397 -122.21832863141171) bank101489 +101490 POINT(46.71927494674764 -122.27469443304739) bank101490 +101491 POINT(48.18112919908265 -122.07554112822878) bank101491 +101492 POINT(47.17703982884955 -122.46580634119934) bank101492 +101493 POINT(47.55771299514433 -122.01735911311411) bank101493 +101494 POINT(46.80267264659055 -122.57604468894759) bank101494 +101495 POINT(46.92697148448264 -123.31578181684301) bank101495 +101496 POINT(47.263458872298166 -122.6922551371394) bank101496 +101497 POINT(48.0476285658615 -121.85519997148737) bank101497 +101498 POINT(47.791811496637486 -123.11529006661227) bank101498 +101499 POINT(46.77826690722241 -122.80741276968199) bank101499 +101500 POINT(47.30425197997192 -122.44165148705918) bank101500 +101501 POINT(46.60755270360894 -122.88825007429043) bank101501 +101502 POINT(47.6770477651673 -122.96375771018968) bank101502 +101503 POINT(47.57673002206551 -121.40550615905649) bank101503 +101504 POINT(48.38221917625588 -122.39414073477936) bank101504 +101505 POINT(48.24747882362668 -121.83294728214013) bank101505 +101506 POINT(46.85391289304995 -121.86998452214821) bank101506 +101507 POINT(47.24827645368993 -121.79947069097732) bank101507 +101508 POINT(46.9888921588704 -121.6674670690957) bank101508 +101509 POINT(46.785494606645656 -121.73619316868786) bank101509 +101510 POINT(48.41795053214658 -123.18965539734367) bank101510 +101511 POINT(48.50465521261096 -122.89602871687812) bank101511 +101512 POINT(47.912779262994235 -122.47962248768725) bank101512 +101513 POINT(46.82173201234427 -122.63707013863124) bank101513 +101514 POINT(47.25687228250377 -121.716401608064) bank101514 +101515 POINT(48.46430072867895 -123.1303121708179) bank101515 +101516 POINT(48.22902930507632 -122.37451805141394) bank101516 +101517 POINT(46.678944442100104 -123.02254105546173) bank101517 +101518 POINT(48.11320910605092 -121.98000775060487) bank101518 +101519 POINT(47.09857226927796 -121.52914050544746) bank101519 +101520 POINT(47.681149269197284 -122.43090970101669) bank101520 +101521 POINT(47.45945418576571 -123.29261290932428) bank101521 +101522 POINT(47.49526564746043 -121.33690807718523) bank101522 +101523 POINT(47.2553228275639 -122.34855467814295) bank101523 +101524 POINT(47.493049520968846 -122.7181864702994) bank101524 +101525 POINT(47.51857345022526 -121.78343208968445) bank101525 +101526 POINT(47.28071696461408 -121.54685611752426) bank101526 +101527 POINT(47.66190721291377 -123.02764048202953) bank101527 +101528 POINT(48.37361866660899 -122.26053655692685) bank101528 +101529 POINT(47.25762727951913 -122.32869227378644) bank101529 +101530 POINT(48.100246314085155 -122.22284138909268) bank101530 +101531 POINT(48.33725717808635 -122.17321373758641) bank101531 +101532 POINT(46.82096148994524 -121.86802510739565) bank101532 +101533 POINT(46.6472601903208 -121.40650341406605) bank101533 +101534 POINT(46.85029937599037 -122.23439148088116) bank101534 +101535 POINT(47.47891040947539 -122.77127713119661) bank101535 +101536 POINT(48.34089230846888 -122.69420520052147) bank101536 +101537 POINT(48.143466374932245 -121.75533274607379) bank101537 +101538 POINT(48.50717990771745 -122.84648052161438) bank101538 +101539 POINT(47.76544896607907 -121.59933580581898) bank101539 +101540 POINT(47.19787566988689 -121.90075004179837) bank101540 +101541 POINT(48.400136057330414 -121.78953823925437) bank101541 +101542 POINT(46.82668570190166 -121.90913289529847) bank101542 +101543 POINT(47.63228415351374 -122.241760708858) bank101543 +101544 POINT(47.93113196039041 -121.73428667705379) bank101544 +101545 POINT(48.45276672068141 -122.8517540251325) bank101545 +101546 POINT(47.59088281516286 -122.94482148948863) bank101546 +101547 POINT(47.43531155987594 -121.74208201668196) bank101547 +101548 POINT(48.11624345490893 -121.40109057105653) bank101548 +101549 POINT(46.91034904408439 -121.7246884800632) bank101549 +101550 POINT(47.74207730236325 -121.67714957396302) bank101550 +101551 POINT(47.75569934723778 -122.048100226506) bank101551 +101552 POINT(47.99999782144752 -123.06888604050717) bank101552 +101553 POINT(47.835757043232775 -121.65928184394015) bank101553 +101554 POINT(47.65977579343442 -121.51645095838384) bank101554 +101555 POINT(47.09030196237759 -121.70326016867797) bank101555 +101556 POINT(46.60781572394607 -122.25772134598597) bank101556 +101557 POINT(47.36834590553255 -121.53891467278518) bank101557 +101558 POINT(46.72315306016768 -122.04161030444635) bank101558 +101559 POINT(47.692125157875346 -121.75145197957032) bank101559 +101560 POINT(47.70785469818964 -121.9408264504747) bank101560 +101561 POINT(47.50996409449656 -122.54559854216303) bank101561 +101562 POINT(48.22940304997303 -122.32900994323954) bank101562 +101563 POINT(47.388823605970074 -121.39416672177651) bank101563 +101564 POINT(46.944328725962976 -121.7206220284665) bank101564 +101565 POINT(48.5961565923609 -122.91387455563626) bank101565 +101566 POINT(47.51607583574426 -123.13494524256822) bank101566 +101567 POINT(47.67399078287813 -122.50333704749156) bank101567 +101568 POINT(46.880432747951694 -123.00308356217455) bank101568 +101569 POINT(48.29746015859584 -122.66559591861429) bank101569 +101570 POINT(47.82911757021495 -122.0394124952699) bank101570 +101571 POINT(46.929675621499015 -121.92937526613575) bank101571 +101572 POINT(47.8216370528648 -122.98759631770463) bank101572 +101573 POINT(47.10012843353693 -122.71403935710705) bank101573 +101574 POINT(47.75823898049111 -122.65832925101046) bank101574 +101575 POINT(47.29723547126506 -123.22391175986199) bank101575 +101576 POINT(46.93896957469683 -121.47586158949485) bank101576 +101577 POINT(48.32397171494924 -121.47111786113567) bank101577 +101578 POINT(47.50896370684885 -123.33138454553645) bank101578 +101579 POINT(47.3350733588936 -122.51855687668646) bank101579 +101580 POINT(46.67144215784336 -121.4826911591279) bank101580 +101581 POINT(46.84696289308642 -123.23056826377545) bank101581 +101582 POINT(47.604490656490135 -122.75420099286643) bank101582 +101583 POINT(46.952369150578285 -122.05569315239579) bank101583 +101584 POINT(47.10711866579085 -121.96410800786673) bank101584 +101585 POINT(47.78296020883866 -122.2024382443369) bank101585 +101586 POINT(48.33128024541623 -121.37727970615158) bank101586 +101587 POINT(46.79009974077315 -121.86601621743445) bank101587 +101588 POINT(47.6375273323338 -121.68000635003392) bank101588 +101589 POINT(47.933582299106824 -121.59881741812305) bank101589 +101590 POINT(48.04547717807835 -122.3423129962335) bank101590 +101591 POINT(46.96039711341666 -121.64487474513783) bank101591 +101592 POINT(47.72840571214165 -122.6902042551851) bank101592 +101593 POINT(48.451833400749635 -122.23744843636545) bank101593 +101594 POINT(46.61341254921188 -122.4197828317231) bank101594 +101595 POINT(47.56234007424471 -122.75797015872242) bank101595 +101596 POINT(47.677094912058344 -122.36054919497981) bank101596 +101597 POINT(47.96993808689863 -121.47981739483691) bank101597 +101598 POINT(47.66917044748693 -122.86920461235745) bank101598 +101599 POINT(47.38815868538439 -122.50628650874825) bank101599 +101600 POINT(47.599512964482486 -122.90682832023431) bank101600 +101601 POINT(48.37940081071147 -122.79178299262456) bank101601 +101602 POINT(47.17367821365499 -122.04321378650204) bank101602 +101603 POINT(48.56282691772369 -122.159870761369) bank101603 +101604 POINT(47.131790020752454 -122.99289527708433) bank101604 +101605 POINT(48.48199741562205 -122.15464394787446) bank101605 +101606 POINT(48.6032242846738 -122.90904337151859) bank101606 +101607 POINT(48.338753916374046 -121.42165787648756) bank101607 +101608 POINT(47.16108794540046 -122.8803837334586) bank101608 +101609 POINT(46.71953631887049 -121.59389118900825) bank101609 +101610 POINT(47.77024431018698 -121.64705269227912) bank101610 +101611 POINT(47.02015672825111 -122.51197201943768) bank101611 +101612 POINT(47.40521951474797 -121.52828335460917) bank101612 +101613 POINT(47.22494524532418 -122.7317026553854) bank101613 +101614 POINT(48.02557585287723 -122.4216162683594) bank101614 +101615 POINT(48.04638643182632 -121.70689157200776) bank101615 +101616 POINT(47.388762747742284 -122.67191877833002) bank101616 +101617 POINT(47.799575002859456 -121.84869032838749) bank101617 +101618 POINT(48.34787365205343 -121.73466443583001) bank101618 +101619 POINT(46.960443468600666 -121.68640564414537) bank101619 +101620 POINT(47.16237597154559 -122.8164897873774) bank101620 +101621 POINT(47.04622061402209 -122.0105705819221) bank101621 +101622 POINT(46.74045108271995 -122.89086574254156) bank101622 +101623 POINT(46.620350979640875 -122.62622377074075) bank101623 +101624 POINT(47.89093255457788 -122.33883022383614) bank101624 +101625 POINT(46.6311904673604 -123.22839730908376) bank101625 +101626 POINT(47.78684047120528 -121.53114485619822) bank101626 +101627 POINT(48.57820411660186 -122.71734407388773) bank101627 +101628 POINT(47.19983848990612 -123.28182185416297) bank101628 +101629 POINT(46.88574271184096 -122.99790138633792) bank101629 +101630 POINT(46.866105185958915 -121.61126805557137) bank101630 +101631 POINT(47.73835681573563 -122.81345679948386) bank101631 +101632 POINT(47.10772986351086 -122.43801438897255) bank101632 +101633 POINT(47.731658013175306 -123.22791426372744) bank101633 +101634 POINT(47.41380352925585 -121.93342852293604) bank101634 +101635 POINT(47.91221658143325 -122.51923489341542) bank101635 +101636 POINT(47.49007066114573 -122.19806574452849) bank101636 +101637 POINT(47.36173733072775 -121.83563473660485) bank101637 +101638 POINT(47.51886194147376 -121.84839703856187) bank101638 +101639 POINT(47.11221490036471 -121.85024039631054) bank101639 +101640 POINT(47.40757309678132 -121.46224353258827) bank101640 +101641 POINT(48.074927231778986 -122.13290437290411) bank101641 +101642 POINT(47.5798042444111 -123.26683663415771) bank101642 +101643 POINT(47.80727032092314 -122.65721399017046) bank101643 +101644 POINT(48.170837140034685 -121.54929917554873) bank101644 +101645 POINT(48.56270042196281 -121.526229680284) bank101645 +101646 POINT(47.215497300000855 -121.94009337213373) bank101646 +101647 POINT(48.52697228904947 -122.2508629652785) bank101647 +101648 POINT(48.46137863981102 -122.42862575226226) bank101648 +101649 POINT(46.90344783471642 -122.8479742580749) bank101649 +101650 POINT(47.05961534940666 -121.36578887887701) bank101650 +101651 POINT(47.664887898643386 -121.45293035984567) bank101651 +101652 POINT(48.5861427068493 -122.53003068565447) bank101652 +101653 POINT(48.33560929355393 -121.74962985080334) bank101653 +101654 POINT(47.8013360887113 -123.09444826620076) bank101654 +101655 POINT(47.14470114766373 -121.6558959114301) bank101655 +101656 POINT(47.89313047604723 -121.64039302203082) bank101656 +101657 POINT(46.934527669168126 -122.27271805109598) bank101657 +101658 POINT(47.24235474683848 -121.53159993879322) bank101658 +101659 POINT(48.605304790289765 -121.52383233782164) bank101659 +101660 POINT(48.532706820828814 -122.68395283027795) bank101660 +101661 POINT(48.43532168387713 -121.6333447475812) bank101661 +101662 POINT(47.651140744542865 -122.43103138987601) bank101662 +101663 POINT(47.5342449035833 -123.2030741232007) bank101663 +101664 POINT(47.09625762242567 -122.96684054811756) bank101664 +101665 POINT(48.25485027798491 -121.98543000050675) bank101665 +101666 POINT(47.9429884678799 -122.91523037719861) bank101666 +101667 POINT(47.36225349153579 -122.6697411624944) bank101667 +101668 POINT(46.96026163876976 -123.00255151459989) bank101668 +101669 POINT(46.90660489951842 -122.95557800711958) bank101669 +101670 POINT(48.10011391141628 -121.5353850992424) bank101670 +101671 POINT(47.81434480948493 -121.56611296321253) bank101671 +101672 POINT(48.09434890446495 -122.95159128843345) bank101672 +101673 POINT(46.67823304247749 -123.29804029266485) bank101673 +101674 POINT(47.09752915417927 -121.76705447918279) bank101674 +101675 POINT(48.0649231802082 -122.77939312813893) bank101675 +101676 POINT(47.63826427896902 -123.0089855843216) bank101676 +101677 POINT(46.61696615595814 -121.5442554559998) bank101677 +101678 POINT(46.879974762565666 -123.29750837934594) bank101678 +101679 POINT(47.20574886274899 -122.10711989854535) bank101679 +101680 POINT(46.68050718501284 -121.63022691000127) bank101680 +101681 POINT(48.377406229048276 -122.4453563983697) bank101681 +101682 POINT(47.52683762389937 -121.9074214209959) bank101682 +101683 POINT(47.64677360259205 -121.96794280300873) bank101683 +101684 POINT(47.447822391417446 -121.63468240247451) bank101684 +101685 POINT(46.81247949999302 -123.26680965770257) bank101685 +101686 POINT(48.191200784868705 -123.27964359153594) bank101686 +101687 POINT(47.39490253445455 -122.44733272063533) bank101687 +101688 POINT(48.427822435196724 -122.14425240954833) bank101688 +101689 POINT(47.560976903977256 -122.00540265908377) bank101689 +101690 POINT(46.71252389963999 -121.5328127032143) bank101690 +101691 POINT(48.15760726412381 -122.39922777637736) bank101691 +101692 POINT(48.00066763466663 -123.17214871566892) bank101692 +101693 POINT(47.75507880418923 -122.75033047558283) bank101693 +101694 POINT(47.791521294997615 -122.98583193524146) bank101694 +101695 POINT(47.52673880711167 -122.00807640956243) bank101695 +101696 POINT(47.97330076448172 -122.82228687688658) bank101696 +101697 POINT(48.43240775387687 -121.50787531109452) bank101697 +101698 POINT(48.14488338546974 -122.28125895943428) bank101698 +101699 POINT(47.12332128976958 -122.84023167203117) bank101699 +101700 POINT(47.432108438812755 -122.08281327173316) bank101700 +101701 POINT(48.267890948190725 -121.68478634251753) bank101701 +101702 POINT(47.14025470644121 -122.19145325570317) bank101702 +101703 POINT(47.925582764458426 -121.4771666609538) bank101703 +101704 POINT(47.70644578175797 -122.13116545815731) bank101704 +101705 POINT(46.779813397582394 -122.3987420006509) bank101705 +101706 POINT(48.076316395365396 -122.20555275838372) bank101706 +101707 POINT(47.91242670368206 -121.60003677314428) bank101707 +101708 POINT(47.39894624375175 -123.21779590900321) bank101708 +101709 POINT(47.70142098552126 -123.04542623468039) bank101709 +101710 POINT(48.254005777595765 -121.75244911674507) bank101710 +101711 POINT(47.60666171441967 -121.68039903529689) bank101711 +101712 POINT(46.7820869038076 -123.24287168845743) bank101712 +101713 POINT(47.35490242808012 -122.04447142322269) bank101713 +101714 POINT(47.2514959162354 -121.48103735819524) bank101714 +101715 POINT(47.89510942680104 -122.39873034543862) bank101715 +101716 POINT(48.2673955355952 -121.42126937916895) bank101716 +101717 POINT(47.83833223086965 -122.90684228603413) bank101717 +101718 POINT(46.68634801556735 -121.43358610178235) bank101718 +101719 POINT(47.68780527973955 -121.91656795390465) bank101719 +101720 POINT(47.83487365373407 -122.48492302519938) bank101720 +101721 POINT(47.264906354522545 -122.1429332380365) bank101721 +101722 POINT(47.82329531945245 -122.52166965926813) bank101722 +101723 POINT(46.945366220726875 -122.40370066361231) bank101723 +101724 POINT(47.8197490084816 -122.9814957435602) bank101724 +101725 POINT(47.74223656234533 -122.70773237529896) bank101725 +101726 POINT(48.02619698523682 -122.68522377644558) bank101726 +101727 POINT(47.838777412833586 -122.37743445496166) bank101727 +101728 POINT(47.62530332317968 -122.62006778470784) bank101728 +101729 POINT(48.11873082765549 -122.64677031578499) bank101729 +101730 POINT(47.21643133935366 -121.71376480307579) bank101730 +101731 POINT(46.73780587683268 -122.33379000057602) bank101731 +101732 POINT(46.652152980783995 -121.97824292226265) bank101732 +101733 POINT(47.35299010586334 -123.15180651171195) bank101733 +101734 POINT(47.0107870237394 -121.45753344709152) bank101734 +101735 POINT(47.166140391130945 -122.93025510122115) bank101735 +101736 POINT(47.534657493015594 -121.79850882625541) bank101736 +101737 POINT(46.618425340469166 -121.96969867774416) bank101737 +101738 POINT(47.1195795407791 -121.88630214804172) bank101738 +101739 POINT(47.008919034343876 -122.97085827091888) bank101739 +101740 POINT(47.76847956953045 -123.03511039362334) bank101740 +101741 POINT(47.47501695347784 -121.4603114078965) bank101741 +101742 POINT(46.940189475477425 -121.46276394809446) bank101742 +101743 POINT(47.15453880769604 -121.56767363225497) bank101743 +101744 POINT(47.2626175330373 -122.08258112666911) bank101744 +101745 POINT(47.157467795831586 -122.8934019070781) bank101745 +101746 POINT(47.84084531567214 -121.33219336865025) bank101746 +101747 POINT(47.887603088791764 -121.84517561875688) bank101747 +101748 POINT(48.09621191551132 -121.96385588833994) bank101748 +101749 POINT(46.62504487426357 -122.8817319874962) bank101749 +101750 POINT(46.72895481279171 -123.31720040393606) bank101750 +101751 POINT(48.21964343905174 -121.44780600758068) bank101751 +101752 POINT(48.55441519663745 -122.6215213214802) bank101752 +101753 POINT(47.107268562401785 -122.20544044932427) bank101753 +101754 POINT(47.1966173373227 -121.70914468126563) bank101754 +101755 POINT(47.08939056104704 -122.72915073497923) bank101755 +101756 POINT(47.14199061512615 -121.81058139250084) bank101756 +101757 POINT(47.81679839594858 -122.25754165540943) bank101757 +101758 POINT(47.651685367137674 -122.38668906822771) bank101758 +101759 POINT(46.744413713711474 -121.82974463929004) bank101759 +101760 POINT(48.04242848959997 -122.28468059981128) bank101760 +101761 POINT(48.22590815583378 -121.96074872786278) bank101761 +101762 POINT(48.01981366853486 -121.51513756400406) bank101762 +101763 POINT(46.905783422755306 -122.55822940425706) bank101763 +101764 POINT(48.18702078921504 -122.36089717184585) bank101764 +101765 POINT(47.4396662422569 -121.50223025600893) bank101765 +101766 POINT(48.29625407230832 -122.48502769937909) bank101766 +101767 POINT(47.60181466346181 -122.04924272250531) bank101767 +101768 POINT(47.781157375566664 -122.36125076747417) bank101768 +101769 POINT(47.14385364804823 -121.98651489122841) bank101769 +101770 POINT(46.83316996592193 -121.58814281554376) bank101770 +101771 POINT(47.52212483358472 -123.33038709736333) bank101771 +101772 POINT(48.53881737747665 -122.56537102688424) bank101772 +101773 POINT(47.81353766932279 -122.39296245373879) bank101773 +101774 POINT(46.61189915211874 -122.851342532369) bank101774 +101775 POINT(48.19552287023402 -122.09016510029382) bank101775 +101776 POINT(48.512419541114895 -122.27957715254956) bank101776 +101777 POINT(47.49556825654182 -121.36810581459555) bank101777 +101778 POINT(47.537467442081905 -122.64508060247613) bank101778 +101779 POINT(48.02656925058623 -121.37881076314804) bank101779 +101780 POINT(47.79620655731407 -121.88361354567408) bank101780 +101781 POINT(47.110908610901944 -122.39248575409336) bank101781 +101782 POINT(47.00691006297488 -122.6307013971979) bank101782 +101783 POINT(46.85199609737714 -123.1664622041921) bank101783 +101784 POINT(47.26143759784863 -121.95985403861319) bank101784 +101785 POINT(46.80667856767683 -122.16581846554105) bank101785 +101786 POINT(47.31951835727015 -123.0123330215587) bank101786 +101787 POINT(47.11437868296554 -122.2283474887051) bank101787 +101788 POINT(47.901877435942865 -121.46953194535111) bank101788 +101789 POINT(47.988111272577086 -123.19634272614289) bank101789 +101790 POINT(47.001838808868406 -122.67359886974353) bank101790 +101791 POINT(48.385304838831004 -122.0798380081197) bank101791 +101792 POINT(48.14948448922334 -121.74290520795235) bank101792 +101793 POINT(47.303451356186564 -122.9419068781326) bank101793 +101794 POINT(47.62463441335102 -121.6959574871168) bank101794 +101795 POINT(46.72878382012851 -121.83318917710865) bank101795 +101796 POINT(48.087772929423636 -122.3202264191764) bank101796 +101797 POINT(47.357412864386255 -121.49778620875894) bank101797 +101798 POINT(47.83010042780678 -121.93829193417974) bank101798 +101799 POINT(47.64991552791994 -122.99437012549679) bank101799 +101800 POINT(46.820280847721826 -121.33970407724479) bank101800 +101801 POINT(48.54216374896238 -121.7265120186673) bank101801 +101802 POINT(48.575137150200064 -122.0603701392142) bank101802 +101803 POINT(46.99930109460795 -121.59216560474654) bank101803 +101804 POINT(48.22343023961086 -121.63589263841912) bank101804 +101805 POINT(48.292477166468636 -123.16746997054356) bank101805 +101806 POINT(46.78717722472248 -121.56090567976976) bank101806 +101807 POINT(46.636489387815615 -121.849449033732) bank101807 +101808 POINT(46.796151045619695 -122.24846008556749) bank101808 +101809 POINT(47.88545998369285 -122.50244654841251) bank101809 +101810 POINT(48.48640221821789 -123.18406524444997) bank101810 +101811 POINT(47.965699865239465 -123.23304605611416) bank101811 +101812 POINT(47.84004502793095 -122.9347136623242) bank101812 +101813 POINT(48.35070486351226 -122.86274316584958) bank101813 +101814 POINT(48.50961974268006 -122.8268130958177) bank101814 +101815 POINT(47.126020290319964 -122.88816784543117) bank101815 +101816 POINT(46.85989624970513 -121.54503623428917) bank101816 +101817 POINT(47.051484059878796 -121.5852432280622) bank101817 +101818 POINT(47.260092346033865 -123.02069768319723) bank101818 +101819 POINT(46.921737112815585 -121.38780948394772) bank101819 +101820 POINT(47.171696736115216 -123.12286033005844) bank101820 +101821 POINT(47.48805100325243 -122.17957060394674) bank101821 +101822 POINT(47.533725700219854 -122.79234498506617) bank101822 +101823 POINT(48.59497300449667 -122.82754928261251) bank101823 +101824 POINT(48.21757331559222 -122.72106217877129) bank101824 +101825 POINT(46.6433549522178 -122.45414930821822) bank101825 +101826 POINT(46.91178883176806 -121.56373364506305) bank101826 +101827 POINT(48.22656189802443 -122.93078842429004) bank101827 +101828 POINT(47.516429374907716 -122.7341546597142) bank101828 +101829 POINT(48.25874178431994 -122.54269804819467) bank101829 +101830 POINT(48.517776156404786 -122.40638834216762) bank101830 +101831 POINT(46.98211085585197 -121.9951462076298) bank101831 +101832 POINT(47.17952570483444 -121.94656134554286) bank101832 +101833 POINT(48.45324814365751 -121.56156904784797) bank101833 +101834 POINT(46.69157610408673 -121.55498293361286) bank101834 +101835 POINT(47.81721272594534 -121.86474577860922) bank101835 +101836 POINT(48.457225724731714 -122.58246941847716) bank101836 +101837 POINT(47.51215654139508 -121.94032534417485) bank101837 +101838 POINT(46.77557919482936 -121.99145710554365) bank101838 +101839 POINT(47.700524607296806 -121.40774219700954) bank101839 +101840 POINT(46.91107668614843 -122.06279833310955) bank101840 +101841 POINT(46.87501675050076 -122.9421794102867) bank101841 +101842 POINT(47.47834038879359 -122.74710218787622) bank101842 +101843 POINT(46.84592847624085 -121.76194943485868) bank101843 +101844 POINT(48.240146085914034 -123.2477987462212) bank101844 +101845 POINT(47.51087747879611 -122.457080242265) bank101845 +101846 POINT(46.86878000423424 -121.8944820390215) bank101846 +101847 POINT(47.499625665423316 -121.57123645541999) bank101847 +101848 POINT(46.71620992906537 -122.94507710643768) bank101848 +101849 POINT(47.98442699186675 -122.35105370473559) bank101849 +101850 POINT(48.00622441870125 -121.48035516671655) bank101850 +101851 POINT(47.529655566795576 -122.86019290905477) bank101851 +101852 POINT(47.92410108322992 -122.38255608874972) bank101852 +101853 POINT(47.703563450069566 -122.20205966333612) bank101853 +101854 POINT(47.21484147873943 -121.57685923301658) bank101854 +101855 POINT(47.60185592615754 -122.33641151416731) bank101855 +101856 POINT(47.56433121835667 -121.3895021905295) bank101856 +101857 POINT(47.393565559455475 -122.97944027859927) bank101857 +101858 POINT(47.153580698819034 -121.90358509219976) bank101858 +101859 POINT(48.390985271696515 -121.69469312247331) bank101859 +101860 POINT(48.22060270427277 -122.52868091696956) bank101860 +101861 POINT(46.69178185290003 -122.05266164527451) bank101861 +101862 POINT(47.53067480912314 -122.16886700785014) bank101862 +101863 POINT(47.40613194057769 -122.50169829519665) bank101863 +101864 POINT(47.58596934587202 -121.4644826843339) bank101864 +101865 POINT(47.71392509415404 -121.9591858201543) bank101865 +101866 POINT(47.03185642049934 -123.10811345150454) bank101866 +101867 POINT(47.78056522051523 -123.32310060026235) bank101867 +101868 POINT(48.46159727243879 -122.74017033889045) bank101868 +101869 POINT(47.40836207301126 -121.70459336156121) bank101869 +101870 POINT(47.24286796035059 -122.1234421395726) bank101870 +101871 POINT(47.85157075469657 -121.5217317042554) bank101871 +101872 POINT(47.39334495594997 -123.17942663973335) bank101872 +101873 POINT(48.05550864894142 -121.57558215951524) bank101873 +101874 POINT(48.10544016276197 -122.23908744659118) bank101874 +101875 POINT(48.15743894278181 -121.86347256471818) bank101875 +101876 POINT(48.48014904909991 -122.84302283227717) bank101876 +101877 POINT(47.85766634323787 -123.06671873249402) bank101877 +101878 POINT(47.1482157184707 -122.03247400643855) bank101878 +101879 POINT(48.50425739675467 -122.98922568280129) bank101879 +101880 POINT(48.436812127842295 -122.63414573202307) bank101880 +101881 POINT(47.31726076223983 -123.31908306757686) bank101881 +101882 POINT(47.01191250158458 -122.00914638001363) bank101882 +101883 POINT(47.61676365220249 -121.51442235922006) bank101883 +101884 POINT(46.999637397538606 -122.817419899878) bank101884 +101885 POINT(47.83392177316356 -122.83648035173631) bank101885 +101886 POINT(47.15052654730865 -121.91728540805258) bank101886 +101887 POINT(48.583626309300605 -121.47775066142108) bank101887 +101888 POINT(47.11314018523822 -122.88247027417296) bank101888 +101889 POINT(48.50728098122352 -121.416820347474) bank101889 +101890 POINT(47.87352845824128 -122.85437308856645) bank101890 +101891 POINT(48.151463676586715 -122.5588098232984) bank101891 +101892 POINT(47.251908642263096 -122.98766776269031) bank101892 +101893 POINT(47.09676884986069 -122.31947126541965) bank101893 +101894 POINT(47.84663239816366 -122.60953418534493) bank101894 +101895 POINT(47.41207174614643 -122.2957304160879) bank101895 +101896 POINT(47.072714573511924 -122.47885229659313) bank101896 +101897 POINT(48.26531389026255 -122.20344003800189) bank101897 +101898 POINT(48.606000762988835 -122.3145377902006) bank101898 +101899 POINT(47.21170476575528 -123.18100274471311) bank101899 +101900 POINT(47.73705219752479 -121.58572773689292) bank101900 +101901 POINT(47.68857736601429 -121.67942635544014) bank101901 +101902 POINT(47.22903648065971 -123.22810965213189) bank101902 +101903 POINT(47.054299286141976 -122.57790212116744) bank101903 +101904 POINT(47.563839932130335 -122.175724282209) bank101904 +101905 POINT(47.25002261765835 -122.89615190634159) bank101905 +101906 POINT(47.609407185119025 -122.687736930856) bank101906 +101907 POINT(46.95212776960128 -123.32528037788094) bank101907 +101908 POINT(48.29339464744623 -121.5284327406841) bank101908 +101909 POINT(46.696319432120916 -122.82659008808923) bank101909 +101910 POINT(47.0378088199038 -121.78368029926668) bank101910 +101911 POINT(47.35751840885386 -122.0435446161245) bank101911 +101912 POINT(47.8553779285177 -122.55350487600289) bank101912 +101913 POINT(46.70928827989508 -122.56354947877065) bank101913 +101914 POINT(47.95761821393477 -121.3709083116525) bank101914 +101915 POINT(47.29706872746808 -121.92525240700105) bank101915 +101916 POINT(48.078263584479245 -122.62905171375995) bank101916 +101917 POINT(46.99125952160567 -122.6010784594737) bank101917 +101918 POINT(47.13041854616279 -122.67426798761741) bank101918 +101919 POINT(48.00667908446888 -121.53812321092096) bank101919 +101920 POINT(48.0138547875605 -122.62424708005587) bank101920 +101921 POINT(47.95886826394153 -122.02511799217274) bank101921 +101922 POINT(46.67742780114261 -121.53294499096725) bank101922 +101923 POINT(47.602612485238744 -121.90940884592585) bank101923 +101924 POINT(48.33645896662342 -122.40946997464357) bank101924 +101925 POINT(46.88640408757194 -121.73177399185523) bank101925 +101926 POINT(47.19398294991004 -122.74166582786941) bank101926 +101927 POINT(47.87779602640526 -122.11388054527454) bank101927 +101928 POINT(48.221117858163346 -123.10491890053183) bank101928 +101929 POINT(47.31647706652874 -121.91901557178026) bank101929 +101930 POINT(47.96665464564676 -121.81755509088526) bank101930 +101931 POINT(47.60395356813077 -123.17903001276936) bank101931 +101932 POINT(47.099690266782666 -121.53092587106788) bank101932 +101933 POINT(47.90785839747464 -121.9164116084122) bank101933 +101934 POINT(47.73523289902229 -121.9848484135959) bank101934 +101935 POINT(46.93488827236873 -123.09969357051024) bank101935 +101936 POINT(47.23933973712892 -121.96497679167283) bank101936 +101937 POINT(47.43559231288257 -122.97257769973707) bank101937 +101938 POINT(46.90956070910547 -121.46569921809855) bank101938 +101939 POINT(47.78516456125071 -121.96675454948983) bank101939 +101940 POINT(47.1090403468444 -121.40206793136649) bank101940 +101941 POINT(47.838332556052684 -121.40263579349552) bank101941 +101942 POINT(48.02107218455474 -123.03112282665748) bank101942 +101943 POINT(47.281138254674225 -122.97884523907604) bank101943 +101944 POINT(47.686990274785636 -122.74303137716599) bank101944 +101945 POINT(47.680523543975994 -121.55452152368616) bank101945 +101946 POINT(46.95956532846991 -123.13707394384504) bank101946 +101947 POINT(47.56245538260971 -121.55566988845297) bank101947 +101948 POINT(48.018015158209614 -121.95936068138377) bank101948 +101949 POINT(47.836852937302446 -121.42168521005053) bank101949 +101950 POINT(47.35465960325395 -121.42615514233349) bank101950 +101951 POINT(48.135032683643 -121.44576672095974) bank101951 +101952 POINT(48.47159985802815 -122.84173797401338) bank101952 +101953 POINT(48.347550947506164 -122.06856472421616) bank101953 +101954 POINT(47.53836599478245 -122.35663731432665) bank101954 +101955 POINT(48.56804942217988 -121.39504475162363) bank101955 +101956 POINT(47.11406844971706 -122.84678358334872) bank101956 +101957 POINT(48.38307931437291 -122.76805846637969) bank101957 +101958 POINT(46.79118681308749 -122.26249231793558) bank101958 +101959 POINT(48.0747738402944 -122.03786348680795) bank101959 +101960 POINT(48.28710305177388 -121.83898262555068) bank101960 +101961 POINT(47.12048967575139 -121.58893313357133) bank101961 +101962 POINT(47.34266356005448 -121.33628967333465) bank101962 +101963 POINT(47.23027049144268 -123.25138520417407) bank101963 +101964 POINT(48.16961209454762 -123.14703992221055) bank101964 +101965 POINT(47.22060228763016 -122.96331563988767) bank101965 +101966 POINT(48.31555078963307 -122.91958797632864) bank101966 +101967 POINT(48.005113344013814 -121.9060122941341) bank101967 +101968 POINT(48.60189767683547 -122.00784685591722) bank101968 +101969 POINT(47.327851663014776 -123.07933280651791) bank101969 +101970 POINT(47.17649115844896 -121.46547038223403) bank101970 +101971 POINT(46.9710508191968 -121.40109763147343) bank101971 +101972 POINT(48.07701219054207 -122.24362983002638) bank101972 +101973 POINT(47.598779110910016 -122.26134335043544) bank101973 +101974 POINT(47.51071606968094 -123.08460807625575) bank101974 +101975 POINT(46.81584582765383 -121.360677518118) bank101975 +101976 POINT(47.97262197982801 -122.63993528787333) bank101976 +101977 POINT(48.05592546504582 -122.97379467386286) bank101977 +101978 POINT(48.27640371965927 -122.93227343884236) bank101978 +101979 POINT(46.89935251491022 -121.90026356171725) bank101979 +101980 POINT(47.86926503046481 -123.19603635309646) bank101980 +101981 POINT(46.73408717707758 -122.97757183867017) bank101981 +101982 POINT(46.898639313339466 -122.01219623651893) bank101982 +101983 POINT(48.54952314636292 -122.57779794608322) bank101983 +101984 POINT(47.686590715647576 -122.6554796125958) bank101984 +101985 POINT(46.64796374473668 -121.80889363313022) bank101985 +101986 POINT(47.857366386135006 -123.31741873534453) bank101986 +101987 POINT(48.18675413920342 -122.17331107894742) bank101987 +101988 POINT(48.06767709197601 -122.74541617797613) bank101988 +101989 POINT(46.62660223578558 -122.83282832115704) bank101989 +101990 POINT(46.860549139493116 -122.4606793626324) bank101990 +101991 POINT(47.24933867623744 -121.58715781757753) bank101991 +101992 POINT(47.8476428204535 -121.40974712399155) bank101992 +101993 POINT(46.95479211535532 -122.81071949062945) bank101993 +101994 POINT(47.532840490741044 -121.53172281141887) bank101994 +101995 POINT(47.03188775679018 -123.15253788313703) bank101995 +101996 POINT(47.69980220039284 -122.57173573344062) bank101996 +101997 POINT(47.968679045229095 -121.94749303508804) bank101997 +101998 POINT(47.602125598381896 -122.00220567954675) bank101998 +101999 POINT(46.95978838171706 -122.05483407389214) bank101999 +102000 POINT(46.967537725799914 -122.85727956565195) bank102000 +102001 POINT(47.56165919696376 -121.77530305147639) bank102001 +102002 POINT(47.028157646516185 -121.5193427864495) bank102002 +102003 POINT(47.183642014801684 -121.52140350062726) bank102003 +102004 POINT(46.99052715487137 -122.2426719888418) bank102004 +102005 POINT(47.00446195689401 -121.9228715062842) bank102005 +102006 POINT(48.08396086191071 -121.35165238155486) bank102006 +102007 POINT(47.80329712554744 -122.51685914444003) bank102007 +102008 POINT(48.237236397340624 -123.25563714315246) bank102008 +102009 POINT(47.175609312798414 -122.15902987455286) bank102009 +102010 POINT(47.7851978895191 -123.01259818609603) bank102010 +102011 POINT(46.86890212475807 -122.65478500785545) bank102011 +102012 POINT(48.40775338396741 -122.38738542415417) bank102012 +102013 POINT(47.29710004419043 -121.68309225948374) bank102013 +102014 POINT(46.62849887692109 -122.3007129734048) bank102014 +102015 POINT(46.90568204265942 -121.37811086648577) bank102015 +102016 POINT(46.964883740255104 -122.84192898598612) bank102016 +102017 POINT(47.23951906499409 -121.70634518093918) bank102017 +102018 POINT(47.70380010698447 -122.78273429797225) bank102018 +102019 POINT(48.005675778978095 -123.14263670733234) bank102019 +102020 POINT(48.1745737245276 -122.2854640431675) bank102020 +102021 POINT(47.667826812592914 -121.83953443586084) bank102021 +102022 POINT(48.11289119382123 -122.34763074758588) bank102022 +102023 POINT(48.053512216153344 -121.61613365328175) bank102023 +102024 POINT(48.59753901975946 -121.87054991214175) bank102024 +102025 POINT(46.67573690126589 -121.75275305990279) bank102025 +102026 POINT(46.99846350575649 -123.27256661736601) bank102026 +102027 POINT(47.58273839953488 -122.88587305572314) bank102027 +102028 POINT(48.01751779045197 -122.2291567062101) bank102028 +102029 POINT(47.832279185855114 -122.28652871749078) bank102029 +102030 POINT(46.69137656704318 -122.02118374615279) bank102030 +102031 POINT(47.50357942968888 -121.62067457690027) bank102031 +102032 POINT(48.539867030290274 -121.93130760792127) bank102032 +102033 POINT(48.041781952110554 -123.24464361900652) bank102033 +102034 POINT(46.70108476772725 -122.43831935212488) bank102034 +102035 POINT(48.50122748520837 -122.19993702720187) bank102035 +102036 POINT(48.203088804644786 -122.73339631132156) bank102036 +102037 POINT(47.98603428519071 -122.30227100419451) bank102037 +102038 POINT(48.29331838542345 -122.34649003324333) bank102038 +102039 POINT(48.531675143085565 -122.7830780045542) bank102039 +102040 POINT(47.94588723078539 -123.17093879593726) bank102040 +102041 POINT(48.4846249223675 -123.26114335121834) bank102041 +102042 POINT(47.36805604313243 -122.06859188372847) bank102042 +102043 POINT(46.73084102961497 -122.25377659008215) bank102043 +102044 POINT(46.71341991851368 -121.38700121046395) bank102044 +102045 POINT(47.46170676592608 -122.42254463060063) bank102045 +102046 POINT(46.78569739462621 -123.2885647954707) bank102046 +102047 POINT(48.40708703072559 -122.62960207097818) bank102047 +102048 POINT(47.32362030437111 -122.74436140438921) bank102048 +102049 POINT(47.46867669268692 -122.99235153829618) bank102049 +102050 POINT(46.902806789795136 -122.12149973686054) bank102050 +102051 POINT(46.98236865936972 -122.09842583912554) bank102051 +102052 POINT(48.594216759439504 -122.79239937180543) bank102052 +102053 POINT(47.385571072594395 -122.26184892234662) bank102053 +102054 POINT(46.762256858906966 -122.45388442998916) bank102054 +102055 POINT(47.23712789519651 -121.61987697299533) bank102055 +102056 POINT(48.1639179158789 -123.00249436796186) bank102056 +102057 POINT(47.55835526192824 -121.38520264183575) bank102057 +102058 POINT(46.77730967199835 -123.24741170602145) bank102058 +102059 POINT(47.76099641674832 -122.8668701945273) bank102059 +102060 POINT(46.82605332031648 -121.95885424259774) bank102060 +102061 POINT(48.152798307999355 -122.55713266982741) bank102061 +102062 POINT(48.311190999869495 -123.22047659776564) bank102062 +102063 POINT(47.12359426209664 -122.73290957675133) bank102063 +102064 POINT(47.46669460580998 -123.16327021803069) bank102064 +102065 POINT(46.79695715394361 -122.21977675069063) bank102065 +102066 POINT(47.24382029206839 -121.84412066444796) bank102066 +102067 POINT(46.964848540350616 -122.87199326276547) bank102067 +102068 POINT(48.00176443819895 -123.20318659285029) bank102068 +102069 POINT(48.50721242426567 -121.52990140638593) bank102069 +102070 POINT(48.41151894245009 -123.15939754564789) bank102070 +102071 POINT(47.232202670560646 -121.78722896468891) bank102071 +102072 POINT(47.766052409427225 -122.35287585783809) bank102072 +102073 POINT(46.63504563660339 -122.02873190342633) bank102073 +102074 POINT(48.226774633144075 -122.47666660991607) bank102074 +102075 POINT(47.36724724267555 -121.43899232159445) bank102075 +102076 POINT(48.41556369200258 -122.99233707822191) bank102076 +102077 POINT(47.084223036793844 -121.97005291022501) bank102077 +102078 POINT(48.51563958510873 -122.67652411854402) bank102078 +102079 POINT(48.0113518234499 -121.52663337893983) bank102079 +102080 POINT(48.3427567264807 -121.36375504882967) bank102080 +102081 POINT(48.18479197195156 -122.92737018963211) bank102081 +102082 POINT(47.31947603569868 -121.7363625012572) bank102082 +102083 POINT(47.717667295917465 -122.64355408695685) bank102083 +102084 POINT(48.038114391565394 -122.45717684208472) bank102084 +102085 POINT(47.03950594332583 -123.27200979725485) bank102085 +102086 POINT(47.57640043805027 -121.96068552666351) bank102086 +102087 POINT(47.71181864987704 -121.565869946945) bank102087 +102088 POINT(47.20140056935123 -122.41541214390489) bank102088 +102089 POINT(47.036612240104986 -122.08601798915869) bank102089 +102090 POINT(46.945999195088625 -122.06954967130152) bank102090 +102091 POINT(48.046761282954456 -122.75139461826515) bank102091 +102092 POINT(46.6127231902795 -122.25127012953334) bank102092 +102093 POINT(47.298693189798165 -122.21850985237941) bank102093 +102094 POINT(47.831721374813796 -121.49784125632989) bank102094 +102095 POINT(46.83502184315734 -121.81033463876972) bank102095 +102096 POINT(48.193197165213114 -122.9471360905293) bank102096 +102097 POINT(46.86711166454245 -122.0257417513434) bank102097 +102098 POINT(47.30819531656018 -122.55906886919838) bank102098 +102099 POINT(48.57617074344976 -122.90774583151021) bank102099 +102100 POINT(46.97838443859264 -122.52439133717056) bank102100 +102101 POINT(46.937683669093445 -123.30121408829247) bank102101 +102102 POINT(48.195635960194245 -122.62025641858506) bank102102 +102103 POINT(48.50985181417864 -122.09870275419009) bank102103 +102104 POINT(46.75820890018197 -121.5085525151221) bank102104 +102105 POINT(47.49868032726792 -123.25583798514789) bank102105 +102106 POINT(46.68809740178247 -121.5168853101123) bank102106 +102107 POINT(48.113041490600985 -121.44234861495079) bank102107 +102108 POINT(46.81150417249131 -121.86479502233223) bank102108 +102109 POINT(47.63261050990201 -121.7075396794664) bank102109 +102110 POINT(47.166063133146636 -122.76139829510987) bank102110 +102111 POINT(46.836419199276484 -122.63512649361246) bank102111 +102112 POINT(46.629252859672064 -121.98558056923044) bank102112 +102113 POINT(48.116846744081364 -122.63529748431131) bank102113 +102114 POINT(48.02445148528821 -122.10678594816063) bank102114 +102115 POINT(46.827845617437475 -121.9160307458234) bank102115 +102116 POINT(47.736843322916755 -122.70976418765996) bank102116 +102117 POINT(46.6723156999612 -122.83782528965276) bank102117 +102118 POINT(47.568297354321686 -121.71447062140791) bank102118 +102119 POINT(47.35691053542233 -122.02886669213947) bank102119 +102120 POINT(47.435942542895745 -123.17891312691513) bank102120 +102121 POINT(47.112688785578136 -121.48113153562976) bank102121 +102122 POINT(47.959960694757555 -121.66950983052759) bank102122 +102123 POINT(48.25208989771424 -122.17943455788472) bank102123 +102124 POINT(46.62324471118102 -123.2753062366106) bank102124 +102125 POINT(47.851821908759234 -123.05234145282536) bank102125 +102126 POINT(47.81623988833719 -123.13472839354023) bank102126 +102127 POINT(48.369959275961264 -123.29171096019604) bank102127 +102128 POINT(48.11761623537991 -122.27632137918008) bank102128 +102129 POINT(47.98233975242424 -122.78640031524947) bank102129 +102130 POINT(47.919831242997354 -121.98641851288717) bank102130 +102131 POINT(48.21421044638939 -122.98767805001576) bank102131 +102132 POINT(46.997611193092176 -122.18285890513697) bank102132 +102133 POINT(47.48463653581636 -122.7801729086754) bank102133 +102134 POINT(48.29675949347132 -122.05302528263911) bank102134 +102135 POINT(46.90358053882093 -121.68375162451318) bank102135 +102136 POINT(47.60871725874137 -123.25376229952688) bank102136 +102137 POINT(47.35495960654706 -122.69303507397177) bank102137 +102138 POINT(47.9880341480514 -121.85442445807185) bank102138 +102139 POINT(47.44853810237456 -122.79204582863923) bank102139 +102140 POINT(47.3967448602992 -123.32481468102559) bank102140 +102141 POINT(46.958905054129914 -122.39588107617624) bank102141 +102142 POINT(48.03997720869393 -122.45235595403659) bank102142 +102143 POINT(46.68197346749766 -121.73506030802643) bank102143 +102144 POINT(48.49764508667387 -121.85090024622464) bank102144 +102145 POINT(48.019730338500715 -122.36261372945458) bank102145 +102146 POINT(46.87431072389696 -121.39835569714123) bank102146 +102147 POINT(48.04999394505794 -122.20084248834792) bank102147 +102148 POINT(46.80505496554779 -123.3076717502082) bank102148 +102149 POINT(47.599719204273285 -121.50154010385569) bank102149 +102150 POINT(47.118270924532986 -121.96485548019402) bank102150 +102151 POINT(48.590404648117975 -121.7318122971303) bank102151 +102152 POINT(48.216536723705445 -123.19705484164446) bank102152 +102153 POINT(48.023681595543785 -122.03922061951377) bank102153 +102154 POINT(46.62706574137286 -121.83785417439412) bank102154 +102155 POINT(47.96510324355309 -122.06839618518663) bank102155 +102156 POINT(47.75437576407695 -121.49982081486046) bank102156 +102157 POINT(46.86133672535709 -122.113958256308) bank102157 +102158 POINT(47.393208385073635 -123.11372980661373) bank102158 +102159 POINT(47.44409614281869 -122.81768917742511) bank102159 +102160 POINT(48.55912832960788 -122.70784355065854) bank102160 +102161 POINT(47.37184944356114 -122.5207142761489) bank102161 +102162 POINT(47.17969961279331 -122.19896894965495) bank102162 +102163 POINT(47.3543448408908 -123.16074208539358) bank102163 +102164 POINT(47.57534625999113 -123.00239445601676) bank102164 +102165 POINT(46.66647609622011 -121.59913956141156) bank102165 +102166 POINT(47.49497173548297 -122.70551776037757) bank102166 +102167 POINT(47.22729967701307 -122.08230303421462) bank102167 +102168 POINT(47.835841879736 -121.6646193365166) bank102168 +102169 POINT(48.494281730688634 -122.52906616209381) bank102169 +102170 POINT(48.53736115821998 -123.16787478252093) bank102170 +102171 POINT(48.49143372213744 -121.67188487973814) bank102171 +102172 POINT(48.53778416734097 -122.91710899534256) bank102172 +102173 POINT(46.865975134047545 -122.37328446800282) bank102173 +102174 POINT(46.773871264093465 -122.30745255930354) bank102174 +102175 POINT(47.999120949841206 -123.25089967293728) bank102175 +102176 POINT(47.444721391229564 -122.62197552889717) bank102176 +102177 POINT(46.73768342734832 -121.73646588705789) bank102177 +102178 POINT(47.74753288264255 -123.21386620364987) bank102178 +102179 POINT(47.15093961743344 -121.78875629924116) bank102179 +102180 POINT(47.89627867394092 -121.76802036013105) bank102180 +102181 POINT(47.97574317522101 -122.2123085776512) bank102181 +102182 POINT(47.89827040294067 -121.70126599961405) bank102182 +102183 POINT(48.29347760107496 -122.75693220805343) bank102183 +102184 POINT(48.03358428634849 -122.56882690544295) bank102184 +102185 POINT(48.2686886145513 -122.03895215017606) bank102185 +102186 POINT(47.90797288314029 -122.08765799209009) bank102186 +102187 POINT(46.92707290788326 -123.11736611346299) bank102187 +102188 POINT(46.61572262371828 -121.52769227703827) bank102188 +102189 POINT(46.69476784864269 -122.7078478785437) bank102189 +102190 POINT(48.560339221949036 -121.43610794834044) bank102190 +102191 POINT(47.73861675775211 -122.20988115520828) bank102191 +102192 POINT(48.17642002244037 -122.19966706868315) bank102192 +102193 POINT(48.073361648921264 -122.65112706856145) bank102193 +102194 POINT(47.14218544297436 -123.08131534967747) bank102194 +102195 POINT(48.40562900467513 -121.72179109499118) bank102195 +102196 POINT(47.04874614734984 -122.04724406053379) bank102196 +102197 POINT(47.24885612929683 -122.03289336323533) bank102197 +102198 POINT(47.439224544423034 -123.31730483835055) bank102198 +102199 POINT(46.8413556778504 -121.95452276185874) bank102199 +102200 POINT(48.50403061893677 -122.6842436042388) bank102200 +102201 POINT(48.568701034715566 -122.10104976374222) bank102201 +102202 POINT(46.92732270473009 -122.36884528186361) bank102202 +102203 POINT(48.44743537193867 -122.57232889974313) bank102203 +102204 POINT(48.27953921374429 -122.06714703344026) bank102204 +102205 POINT(47.44579765263143 -122.65569770508824) bank102205 +102206 POINT(47.254135952057915 -122.76677163847414) bank102206 +102207 POINT(47.644016391244435 -121.5396181478022) bank102207 +102208 POINT(47.98515078629492 -121.39232274650796) bank102208 +102209 POINT(46.866339077126746 -122.5730118528937) bank102209 +102210 POINT(46.84505271490159 -121.48460444286101) bank102210 +102211 POINT(48.308772833688465 -122.77600046750267) bank102211 +102212 POINT(48.04906425878926 -121.7244331291549) bank102212 +102213 POINT(47.35686410852217 -122.13548233265324) bank102213 +102214 POINT(48.180364099617606 -121.5909199684172) bank102214 +102215 POINT(48.56385898517308 -122.36229003769033) bank102215 +102216 POINT(48.54519567468698 -123.1942487444867) bank102216 +102217 POINT(46.7497408284913 -121.7726318421812) bank102217 +102218 POINT(48.495651169444514 -122.34486172568853) bank102218 +102219 POINT(46.82733445346879 -122.79882064200166) bank102219 +102220 POINT(47.09103862186853 -122.01668609588916) bank102220 +102221 POINT(47.13184491034249 -121.39705273535712) bank102221 +102222 POINT(48.325776429932105 -121.62964991567463) bank102222 +102223 POINT(47.6903622541102 -122.02225538284783) bank102223 +102224 POINT(46.87312332142137 -121.72892740731362) bank102224 +102225 POINT(47.13517194322673 -122.33414829673187) bank102225 +102226 POINT(48.31722854942419 -122.89463349168072) bank102226 +102227 POINT(46.82319418586663 -121.98408816555842) bank102227 +102228 POINT(48.24301370824078 -122.28081585738195) bank102228 +102229 POINT(48.49509414969424 -123.05884849126629) bank102229 +102230 POINT(48.0132937344211 -121.70531481481186) bank102230 +102231 POINT(48.546334623721364 -122.60178206578499) bank102231 +102232 POINT(47.1389970386911 -121.38315880584857) bank102232 +102233 POINT(48.087510214838325 -121.36933903879095) bank102233 +102234 POINT(48.191937724822644 -123.17477339565863) bank102234 +102235 POINT(48.49110285343792 -122.37283684909931) bank102235 +102236 POINT(47.60451424730109 -123.27184166629452) bank102236 +102237 POINT(47.755972940436536 -121.97768475646704) bank102237 +102238 POINT(47.13010362961942 -122.04031621863956) bank102238 +102239 POINT(47.69389722772433 -121.84671405900785) bank102239 +102240 POINT(47.54788673661002 -122.82393057927389) bank102240 +102241 POINT(47.835972575583995 -122.08248706034445) bank102241 +102242 POINT(47.091179148084706 -122.3535779929047) bank102242 +102243 POINT(47.69930601411071 -122.46914428596435) bank102243 +102244 POINT(47.449166027607795 -122.07466190488364) bank102244 +102245 POINT(48.342000869744304 -122.06292261413482) bank102245 +102246 POINT(46.872985291213965 -122.31714977405683) bank102246 +102247 POINT(47.632719270055375 -123.15968184305677) bank102247 +102248 POINT(47.53615411965915 -121.67941231788721) bank102248 +102249 POINT(47.92840418448779 -121.58719532012518) bank102249 +102250 POINT(48.579581880650096 -122.63895176107513) bank102250 +102251 POINT(48.03463578959613 -122.05198877314614) bank102251 +102252 POINT(48.415113726269105 -121.69420865893849) bank102252 +102253 POINT(46.88255491832288 -122.46631722044665) bank102253 +102254 POINT(46.627468747507166 -121.90852277252212) bank102254 +102255 POINT(46.96346493093991 -123.19887135279575) bank102255 +102256 POINT(47.9354130956616 -121.94198604569823) bank102256 +102257 POINT(47.27813917029445 -123.04400695401634) bank102257 +102258 POINT(48.316695542967516 -122.30213526150663) bank102258 +102259 POINT(47.797806203429445 -122.45826136740402) bank102259 +102260 POINT(47.082995440463456 -122.70062224745831) bank102260 +102261 POINT(47.6413034871941 -123.11445786850301) bank102261 +102262 POINT(48.032836126826666 -121.6189306902245) bank102262 +102263 POINT(48.08070321153397 -121.54710747148468) bank102263 +102264 POINT(48.28929101126619 -121.54762840963531) bank102264 +102265 POINT(47.79972159586689 -122.19575893434835) bank102265 +102266 POINT(47.27767834265564 -122.9362019982119) bank102266 +102267 POINT(47.74339998132977 -121.76798492744648) bank102267 +102268 POINT(47.97533599573544 -122.06177724186587) bank102268 +102269 POINT(46.767500772533914 -123.22439021534433) bank102269 +102270 POINT(46.911430446499175 -123.10739587471512) bank102270 +102271 POINT(47.519600532410436 -121.96842338965536) bank102271 +102272 POINT(47.69150100757895 -121.8206083121801) bank102272 +102273 POINT(47.99928957561108 -122.86630587259093) bank102273 +102274 POINT(47.43201423383562 -121.73785860638473) bank102274 +102275 POINT(47.39891011760939 -122.88545150088386) bank102275 +102276 POINT(48.591266474307744 -122.40848528873829) bank102276 +102277 POINT(47.21878253868223 -121.80911935992232) bank102277 +102278 POINT(47.92402369949626 -121.95843554206486) bank102278 +102279 POINT(46.977615114515075 -123.17709011256618) bank102279 +102280 POINT(47.82855836279876 -122.17516109996436) bank102280 +102281 POINT(47.85361050204278 -122.63372033510899) bank102281 +102282 POINT(47.07450000853839 -122.85726907159584) bank102282 +102283 POINT(47.89913805402214 -121.77311284634087) bank102283 +102284 POINT(48.089173194400495 -121.6888831806191) bank102284 +102285 POINT(47.61827314920673 -122.17666015233361) bank102285 +102286 POINT(47.661770527616696 -122.68222181304476) bank102286 +102287 POINT(47.73755265239016 -122.50042945495188) bank102287 +102288 POINT(48.2840336696747 -122.40181784437512) bank102288 +102289 POINT(46.779142989795204 -121.72964306508698) bank102289 +102290 POINT(47.98549800134829 -122.54256585158711) bank102290 +102291 POINT(48.12818708452717 -121.58424496228896) bank102291 +102292 POINT(48.5927508564841 -121.97164913940871) bank102292 +102293 POINT(46.946131574561136 -122.39949743695237) bank102293 +102294 POINT(47.663571468687785 -121.89557508798568) bank102294 +102295 POINT(47.78767818574799 -122.63033617586257) bank102295 +102296 POINT(48.11459143012764 -123.08292557748395) bank102296 +102297 POINT(47.8128232322483 -123.29199315722606) bank102297 +102298 POINT(47.91897791386386 -123.02041234899221) bank102298 +102299 POINT(47.81395082788505 -121.47359448399322) bank102299 +102300 POINT(47.77309091838591 -122.88501951728331) bank102300 +102301 POINT(47.22753687772054 -122.31135223903046) bank102301 +102302 POINT(47.35591073598445 -122.13887445265888) bank102302 +102303 POINT(47.52957573064649 -122.8873474500707) bank102303 +102304 POINT(46.976562774016784 -122.08329229961984) bank102304 +102305 POINT(46.91543468730926 -122.60429027587784) bank102305 +102306 POINT(46.726274124775436 -122.67940075624146) bank102306 +102307 POINT(46.65230179791719 -121.71269640450414) bank102307 +102308 POINT(47.59417144523064 -121.62034451013301) bank102308 +102309 POINT(47.6428575421998 -121.58601587887415) bank102309 +102310 POINT(48.491066062511265 -122.29727520467276) bank102310 +102311 POINT(47.295014967007106 -123.10359597599157) bank102311 +102312 POINT(46.961728257656944 -121.67803384211669) bank102312 +102313 POINT(48.35566491365057 -121.39240296630324) bank102313 +102314 POINT(48.027609918020914 -122.00697089502408) bank102314 +102315 POINT(48.31527294923452 -122.2097142030382) bank102315 +102316 POINT(46.933643081688736 -121.79745147835249) bank102316 +102317 POINT(46.73051480672931 -121.54047529071289) bank102317 +102318 POINT(48.250668232422086 -121.8887602869854) bank102318 +102319 POINT(46.69302927345078 -122.80134103038047) bank102319 +102320 POINT(48.01383068418038 -123.2676313821906) bank102320 +102321 POINT(46.6739082569088 -122.61552206216794) bank102321 +102322 POINT(47.10625007601583 -122.15068624231287) bank102322 +102323 POINT(47.828762094337755 -122.84835194393327) bank102323 +102324 POINT(46.73367322614834 -121.7876162651284) bank102324 +102325 POINT(47.437770605042154 -123.00941944528766) bank102325 +102326 POINT(47.280635103180344 -122.71418666354934) bank102326 +102327 POINT(46.89442164217421 -123.0676549819738) bank102327 +102328 POINT(47.47539234525531 -121.68345003677457) bank102328 +102329 POINT(47.58592388580977 -122.21800775811158) bank102329 +102330 POINT(46.86833507985252 -121.71019212069153) bank102330 +102331 POINT(48.301990837461126 -123.12468791316364) bank102331 +102332 POINT(48.08533753235552 -122.51034660396817) bank102332 +102333 POINT(47.385076838351104 -121.7593600051469) bank102333 +102334 POINT(48.299824914226704 -122.81581305338682) bank102334 +102335 POINT(46.79643884961303 -123.02661426679589) bank102335 +102336 POINT(46.73828342783684 -122.31282820914241) bank102336 +102337 POINT(46.63089042402486 -122.80967240798182) bank102337 +102338 POINT(47.67118495013137 -122.66080447336691) bank102338 +102339 POINT(47.74174208282488 -121.71133570900176) bank102339 +102340 POINT(46.70897340187273 -122.69541351660688) bank102340 +102341 POINT(48.15198988639071 -122.21829077558428) bank102341 +102342 POINT(47.545630987218246 -122.07457737698678) bank102342 +102343 POINT(48.567426147014956 -121.72318412820783) bank102343 +102344 POINT(48.41691915876635 -123.16944238800016) bank102344 +102345 POINT(47.22131012796078 -123.03470999009997) bank102345 +102346 POINT(48.264080928431106 -123.23736686374812) bank102346 +102347 POINT(47.54631067598948 -122.9207943096084) bank102347 +102348 POINT(47.44278808407546 -123.10028844656716) bank102348 +102349 POINT(47.68640793862947 -123.04891132685655) bank102349 +102350 POINT(47.87504991560211 -121.98732028527115) bank102350 +102351 POINT(46.702760577756735 -122.00067885760232) bank102351 +102352 POINT(48.01896028950802 -122.0254832932729) bank102352 +102353 POINT(48.40368270060756 -122.43337299730723) bank102353 +102354 POINT(47.26314742924838 -121.7437777448409) bank102354 +102355 POINT(47.369063995977314 -121.63611624280196) bank102355 +102356 POINT(48.08250110831945 -122.88708963546205) bank102356 +102357 POINT(48.147101071001664 -123.14694317957498) bank102357 +102358 POINT(47.93831045834671 -123.09497072905626) bank102358 +102359 POINT(48.52469995789721 -122.78606180960129) bank102359 +102360 POINT(46.93528186236825 -121.95806786799291) bank102360 +102361 POINT(46.797185897939045 -122.16035363567894) bank102361 +102362 POINT(47.41192563602928 -122.95035464129728) bank102362 +102363 POINT(47.15838334700011 -123.28539632653637) bank102363 +102364 POINT(46.858480517537885 -122.32675196449155) bank102364 +102365 POINT(48.41860272794687 -123.28088754829992) bank102365 +102366 POINT(48.323253657162 -122.52100828512425) bank102366 +102367 POINT(47.961432334286606 -122.21955673075297) bank102367 +102368 POINT(46.63556009566556 -122.8255684830771) bank102368 +102369 POINT(48.44511250404119 -122.49871942618223) bank102369 +102370 POINT(48.538286181597734 -122.0669791075326) bank102370 +102371 POINT(47.82928096371255 -122.77692714328175) bank102371 +102372 POINT(48.134591744337584 -122.78625192339688) bank102372 +102373 POINT(48.03262296832696 -121.72134925383138) bank102373 +102374 POINT(47.646217328401164 -121.97543687185689) bank102374 +102375 POINT(46.77949090295301 -122.63041445504334) bank102375 +102376 POINT(48.4938157607986 -121.75303860995858) bank102376 +102377 POINT(47.8506699505977 -122.97896208365297) bank102377 +102378 POINT(47.67204997017493 -122.83798420582501) bank102378 +102379 POINT(46.942629090164026 -121.53588761366512) bank102379 +102380 POINT(47.63094041427552 -123.25000126961511) bank102380 +102381 POINT(48.11867363652853 -123.16132840404507) bank102381 +102382 POINT(48.531704086518054 -122.07392540797586) bank102382 +102383 POINT(46.83088566300989 -122.92589450405953) bank102383 +102384 POINT(46.70063070102728 -123.10883863235142) bank102384 +102385 POINT(46.7644073549968 -122.48703243560253) bank102385 +102386 POINT(48.39862602840642 -123.29684136484306) bank102386 +102387 POINT(48.059495003521484 -122.17909313146967) bank102387 +102388 POINT(46.834124405096816 -121.4283586264065) bank102388 +102389 POINT(48.60191431281018 -121.41745652132387) bank102389 +102390 POINT(46.95823541544778 -121.47481593417494) bank102390 +102391 POINT(46.93124510719129 -122.64861225681626) bank102391 +102392 POINT(47.43258938214005 -122.61803343233339) bank102392 +102393 POINT(47.70634025216408 -123.19320234754902) bank102393 +102394 POINT(47.60964426314666 -122.98634400414099) bank102394 +102395 POINT(48.02894238257986 -123.32458622751409) bank102395 +102396 POINT(46.72231955751666 -122.6521073429872) bank102396 +102397 POINT(48.160343305090684 -121.48604043163346) bank102397 +102398 POINT(47.11737654596431 -123.10472784309694) bank102398 +102399 POINT(46.68182048322617 -122.95034353982297) bank102399 +102400 POINT(48.3647302047636 -122.16119185960261) bank102400 +102401 POINT(47.50104460437924 -122.66746261825912) bank102401 +102402 POINT(47.51850436643847 -121.98320577191909) bank102402 +102403 POINT(48.01872677789719 -121.34623185350374) bank102403 +102404 POINT(46.968148225945214 -122.93870800267482) bank102404 +102405 POINT(46.645470076806724 -121.6005592371119) bank102405 +102406 POINT(46.77845880643176 -123.27368183693774) bank102406 +102407 POINT(47.41081911642341 -122.80691675311145) bank102407 +102408 POINT(47.799652505974834 -122.1271728602639) bank102408 +102409 POINT(47.18851424316761 -122.56785852860561) bank102409 +102410 POINT(48.23242625640178 -121.66063251589526) bank102410 +102411 POINT(47.423788312233505 -121.70402078584891) bank102411 +102412 POINT(47.52589632134618 -122.55891714083255) bank102412 +102413 POINT(47.54707856082438 -121.36706215441573) bank102413 +102414 POINT(48.58641490897303 -121.40300343865486) bank102414 +102415 POINT(47.596138315935974 -121.40015774632488) bank102415 +102416 POINT(47.320371610716506 -121.6797362350829) bank102416 +102417 POINT(47.17694932311495 -122.44669529953985) bank102417 +102418 POINT(48.09777122439758 -123.30132134654542) bank102418 +102419 POINT(47.60952379545648 -123.14551180806899) bank102419 +102420 POINT(48.56726623317201 -122.70213547658813) bank102420 +102421 POINT(47.21461703237969 -122.51575666106025) bank102421 +102422 POINT(48.270307654173685 -122.84252980144335) bank102422 +102423 POINT(46.7589256841315 -122.40691561266684) bank102423 +102424 POINT(46.876783195136525 -122.13841922232788) bank102424 +102425 POINT(47.708534492223365 -121.35797067685866) bank102425 +102426 POINT(48.24455649642895 -123.2293639957941) bank102426 +102427 POINT(47.10731878666473 -122.19284185443348) bank102427 +102428 POINT(46.7917633017899 -122.37390802862564) bank102428 +102429 POINT(47.30112742380867 -121.77826682972255) bank102429 +102430 POINT(47.15344659773625 -123.22012158063963) bank102430 +102431 POINT(47.34640476209246 -122.66395215598374) bank102431 +102432 POINT(47.14073367080474 -122.42664228450008) bank102432 +102433 POINT(48.43085234878861 -122.07182347268865) bank102433 +102434 POINT(47.22653469946031 -122.32277232933421) bank102434 +102435 POINT(47.599895284685836 -122.00402874156806) bank102435 +102436 POINT(46.73213803775592 -121.37436403712597) bank102436 +102437 POINT(47.03424557969888 -121.73606531845195) bank102437 +102438 POINT(48.60143794262747 -122.47270135010804) bank102438 +102439 POINT(48.60063806955623 -122.68593895656309) bank102439 +102440 POINT(47.496014454299015 -121.84174766232627) bank102440 +102441 POINT(47.43184671044722 -121.85698602966939) bank102441 +102442 POINT(47.24665717167685 -122.49081861186126) bank102442 +102443 POINT(47.89584503100822 -121.56596700379103) bank102443 +102444 POINT(47.741171532704094 -122.88225637500797) bank102444 +102445 POINT(48.5329420810314 -122.20282354954765) bank102445 +102446 POINT(46.68889619211665 -121.73076148684177) bank102446 +102447 POINT(47.983018148539074 -121.67985582118212) bank102447 +102448 POINT(48.0779456394952 -123.33014107234277) bank102448 +102449 POINT(47.026633586261816 -121.5845763595945) bank102449 +102450 POINT(47.48126344715209 -123.06857367391358) bank102450 +102451 POINT(47.441633442773956 -121.62565320676444) bank102451 +102452 POINT(47.82297525083395 -121.70588093123065) bank102452 +102453 POINT(47.53833785267554 -123.28602917541107) bank102453 +102454 POINT(47.90624271841017 -122.56077060638077) bank102454 +102455 POINT(47.209326896138116 -122.00071500523323) bank102455 +102456 POINT(47.80927554863974 -121.7649043971815) bank102456 +102457 POINT(47.81231978657381 -123.00352882413983) bank102457 +102458 POINT(47.09054647173865 -122.26342769431461) bank102458 +102459 POINT(48.372695126408125 -123.30404493864368) bank102459 +102460 POINT(46.68731285462197 -122.30002348945396) bank102460 +102461 POINT(48.246702071527565 -122.04105956773367) bank102461 +102462 POINT(48.32183223919112 -122.18440904958847) bank102462 +102463 POINT(48.298734521584684 -121.60783195901323) bank102463 +102464 POINT(48.50043905039872 -121.9122483320419) bank102464 +102465 POINT(47.88178171787065 -121.66829759758384) bank102465 +102466 POINT(47.05823738128933 -121.62617349478462) bank102466 +102467 POINT(47.59299124779257 -123.1425917766314) bank102467 +102468 POINT(47.996044289000174 -121.82252601968352) bank102468 +102469 POINT(47.42996331605916 -122.79795207919827) bank102469 +102470 POINT(47.64276155111155 -121.57905060921212) bank102470 +102471 POINT(48.23046063964311 -122.54763666468895) bank102471 +102472 POINT(47.72306306649337 -123.09383568961725) bank102472 +102473 POINT(47.51024825455893 -121.54294810243702) bank102473 +102474 POINT(47.73216471119213 -122.23489349706577) bank102474 +102475 POINT(46.88044441142473 -123.0531421674856) bank102475 +102476 POINT(48.28499735276032 -122.54654100946804) bank102476 +102477 POINT(48.198064878581256 -122.1346177029616) bank102477 +102478 POINT(46.99204700886238 -122.33757302132813) bank102478 +102479 POINT(48.02754946057447 -122.93702846490505) bank102479 +102480 POINT(46.98970700090494 -122.97325850313821) bank102480 +102481 POINT(47.32966520077199 -122.35935009858215) bank102481 +102482 POINT(47.65692559517112 -122.90201366521967) bank102482 +102483 POINT(48.175825621950224 -122.22250974993636) bank102483 +102484 POINT(47.99870174950727 -122.62524043940516) bank102484 +102485 POINT(47.34844019267042 -122.49438068612426) bank102485 +102486 POINT(47.14688388358867 -122.52306497141664) bank102486 +102487 POINT(47.13803829553663 -122.88394721653167) bank102487 +102488 POINT(46.75940956311552 -122.798471971016) bank102488 +102489 POINT(47.66803130399799 -121.8748425929815) bank102489 +102490 POINT(48.5502892409604 -121.81591545379233) bank102490 +102491 POINT(48.49718002159862 -122.50234396907018) bank102491 +102492 POINT(47.793169915984315 -123.10117888423075) bank102492 +102493 POINT(47.114477726020986 -122.88273175987793) bank102493 +102494 POINT(48.33901478162598 -121.39457631351057) bank102494 +102495 POINT(47.1033709017714 -122.95509504299757) bank102495 +102496 POINT(46.96496733699855 -122.19970604861577) bank102496 +102497 POINT(47.72136594807108 -123.23315437925542) bank102497 +102498 POINT(47.11153252187914 -122.64462466965094) bank102498 +102499 POINT(48.464601152661395 -121.82623549328436) bank102499 +102500 POINT(47.31661743249138 -121.4096970528665) bank102500 +102501 POINT(47.14713436483998 -122.76667855216776) bank102501 +102502 POINT(48.16751377966451 -122.34417207666138) bank102502 +102503 POINT(47.78937958346002 -123.22144861294858) bank102503 +102504 POINT(47.06009268504616 -121.58466821523153) bank102504 +102505 POINT(46.999399500922095 -122.06265981187802) bank102505 +102506 POINT(47.10660864817629 -122.33890896830104) bank102506 +102507 POINT(48.00970917752447 -121.82101262961145) bank102507 +102508 POINT(47.03921357913962 -122.15659788164824) bank102508 +102509 POINT(48.29289188761566 -121.6084498479052) bank102509 +102510 POINT(47.21047944048485 -122.64207775868707) bank102510 +102511 POINT(47.61800031211495 -123.27396508133091) bank102511 +102512 POINT(46.83431595860769 -122.33084691596858) bank102512 +102513 POINT(47.67684826110134 -122.28219763731298) bank102513 +102514 POINT(47.399050130747845 -121.6111796537222) bank102514 +102515 POINT(47.87707232250903 -122.07001836765171) bank102515 +102516 POINT(47.14240508714198 -123.06971865355837) bank102516 +102517 POINT(46.65310827119209 -121.39947942196342) bank102517 +102518 POINT(46.61101005524961 -121.64016870527178) bank102518 +102519 POINT(47.07881061414314 -122.40870318727957) bank102519 +102520 POINT(46.78108160031403 -121.9176878451486) bank102520 +102521 POINT(48.29649830498436 -121.64064317312432) bank102521 +102522 POINT(47.52474120884541 -122.44459558132245) bank102522 +102523 POINT(47.1438108669807 -122.4824459182363) bank102523 +102524 POINT(47.84105509821177 -121.45721545571395) bank102524 +102525 POINT(48.51860256708702 -122.921416330928) bank102525 +102526 POINT(47.49349447115771 -121.7504607866786) bank102526 +102527 POINT(48.294531217358646 -122.51554022276811) bank102527 +102528 POINT(47.371934797123856 -121.69818812156151) bank102528 +102529 POINT(46.674276164585734 -121.80468284764429) bank102529 +102530 POINT(48.29573729787843 -122.23417996447918) bank102530 +102531 POINT(48.181772918204004 -122.35463900737837) bank102531 +102532 POINT(47.1405128187192 -121.62550726511397) bank102532 +102533 POINT(47.31270326696838 -121.97935050612756) bank102533 +102534 POINT(46.949607213985686 -122.47232189754861) bank102534 +102535 POINT(47.214081092995194 -121.7428579236245) bank102535 +102536 POINT(47.93739199161304 -121.77829243493512) bank102536 +102537 POINT(47.74071335276764 -121.42076424180304) bank102537 +102538 POINT(46.6813068952496 -122.20715161928517) bank102538 +102539 POINT(46.736626757767226 -121.55582049876841) bank102539 +102540 POINT(47.21859470996598 -123.15282582366444) bank102540 +102541 POINT(48.381660639416154 -122.32073061874219) bank102541 +102542 POINT(48.57538422011929 -121.33578912145877) bank102542 +102543 POINT(48.12214125301893 -122.01695298977349) bank102543 +102544 POINT(46.70612855380213 -122.4949497416165) bank102544 +102545 POINT(46.73373049771248 -122.83087060987653) bank102545 +102546 POINT(47.00281516197547 -123.26598762852493) bank102546 +102547 POINT(48.552273720587166 -122.2249177992471) bank102547 +102548 POINT(48.053659911717745 -122.59810401302785) bank102548 +102549 POINT(47.98593676137453 -122.97543674520426) bank102549 +102550 POINT(48.27605666533488 -122.51021488133254) bank102550 +102551 POINT(48.36344340413151 -122.39270451197982) bank102551 +102552 POINT(48.49188194944867 -122.8235675645651) bank102552 +102553 POINT(47.1654240812939 -121.64561477602457) bank102553 +102554 POINT(47.60598384875478 -122.98620165829364) bank102554 +102555 POINT(48.120600108588 -121.36079586198612) bank102555 +102556 POINT(47.80501765513669 -121.96977442944618) bank102556 +102557 POINT(47.47999768731906 -122.0376605631807) bank102557 +102558 POINT(47.431174654772796 -123.1149851599154) bank102558 +102559 POINT(47.397443809377194 -121.42919845062426) bank102559 +102560 POINT(48.49588286610041 -122.28787412212505) bank102560 +102561 POINT(48.37258013062552 -121.8220194794732) bank102561 +102562 POINT(46.919340182240916 -121.93929191372432) bank102562 +102563 POINT(48.390775443445584 -122.82568233411979) bank102563 +102564 POINT(47.371737850125285 -122.07884446623521) bank102564 +102565 POINT(46.92258571689415 -122.58178477817161) bank102565 +102566 POINT(48.384383467498324 -123.32685197438389) bank102566 +102567 POINT(48.33912465102026 -122.73703531777677) bank102567 +102568 POINT(48.08240518522592 -121.79355686014301) bank102568 +102569 POINT(47.182965511349984 -121.62030602418805) bank102569 +102570 POINT(47.52053192042789 -122.473237351175) bank102570 +102571 POINT(48.562828156825375 -121.3997093450526) bank102571 +102572 POINT(47.927926084854434 -123.1456087294962) bank102572 +102573 POINT(47.47187634717559 -122.55741192878057) bank102573 +102574 POINT(46.764173318978806 -121.8527744336549) bank102574 +102575 POINT(48.113836422641725 -122.09896138511827) bank102575 +102576 POINT(46.72957084157232 -122.16753480110096) bank102576 +102577 POINT(47.074630247233195 -122.11597459245542) bank102577 +102578 POINT(47.85731450558913 -122.24157148758573) bank102578 +102579 POINT(48.184577571995746 -121.98715915000074) bank102579 +102580 POINT(48.116778330696704 -122.86022057843316) bank102580 +102581 POINT(46.91157735219193 -123.07237534036474) bank102581 +102582 POINT(46.6557699380601 -122.39985392238547) bank102582 +102583 POINT(48.07350775242915 -123.30039597673341) bank102583 +102584 POINT(48.10951955995825 -122.66993694423203) bank102584 +102585 POINT(47.985760134395896 -122.58659389266948) bank102585 +102586 POINT(47.07395673810586 -122.86555561488154) bank102586 +102587 POINT(48.0163338808178 -122.92638334538997) bank102587 +102588 POINT(47.463954730267716 -122.97431343887243) bank102588 +102589 POINT(48.589021537924005 -122.4366914285202) bank102589 +102590 POINT(48.560610513767685 -122.54078191924269) bank102590 +102591 POINT(47.1067100968079 -123.0243690819421) bank102591 +102592 POINT(47.4895958371971 -122.7808948523158) bank102592 +102593 POINT(48.51221251803826 -123.31123258840228) bank102593 +102594 POINT(48.158955026940006 -122.73116617648991) bank102594 +102595 POINT(48.43444984038402 -121.44511570396143) bank102595 +102596 POINT(47.17632868371911 -122.03526977735584) bank102596 +102597 POINT(47.301676903320555 -122.75641807241225) bank102597 +102598 POINT(47.99977525588422 -122.10909151825939) bank102598 +102599 POINT(48.504562600149946 -122.62965774914167) bank102599 +102600 POINT(47.605464413617135 -121.33501410187712) bank102600 +102601 POINT(47.428201832576015 -122.54169750701436) bank102601 +102602 POINT(47.7621617921128 -121.9904830665177) bank102602 +102603 POINT(46.81823244069399 -121.98619981369175) bank102603 +102604 POINT(47.8840247597692 -123.11202605268345) bank102604 +102605 POINT(46.853728663853126 -121.38231418313781) bank102605 +102606 POINT(47.72337063739979 -122.125154253704) bank102606 +102607 POINT(48.54235781482998 -121.54241300375833) bank102607 +102608 POINT(47.334063079873516 -121.59284114544805) bank102608 +102609 POINT(48.50992003560176 -122.47276713665651) bank102609 +102610 POINT(48.34805445780891 -123.06439603874016) bank102610 +102611 POINT(47.9742581675305 -122.91313770555304) bank102611 +102612 POINT(46.883889459798496 -123.29133570037327) bank102612 +102613 POINT(47.390757923080436 -121.9628014729067) bank102613 +102614 POINT(48.386160385560046 -121.76604594803841) bank102614 +102615 POINT(47.4077778369597 -122.80569182354735) bank102615 +102616 POINT(47.46943849416655 -121.35988078685418) bank102616 +102617 POINT(47.05755278193314 -122.98358824120058) bank102617 +102618 POINT(48.54800154184601 -121.48245065593801) bank102618 +102619 POINT(48.497739222679414 -122.48044573216734) bank102619 +102620 POINT(46.940666413665646 -121.97803777795983) bank102620 +102621 POINT(47.811002012990166 -122.19658320422883) bank102621 +102622 POINT(48.23381799975725 -121.97315960481417) bank102622 +102623 POINT(46.98060104111471 -121.87973664444874) bank102623 +102624 POINT(47.12020744428993 -121.63399827271341) bank102624 +102625 POINT(46.86226700804256 -122.14023772282317) bank102625 +102626 POINT(47.56643129920782 -122.45628771693116) bank102626 +102627 POINT(46.71519506613253 -122.58323772896496) bank102627 +102628 POINT(47.161718146961945 -121.94718673294665) bank102628 +102629 POINT(47.159627582712076 -122.47274150276924) bank102629 +102630 POINT(46.750919159948275 -123.10794703779194) bank102630 +102631 POINT(47.4401241755263 -122.13405640074777) bank102631 +102632 POINT(48.10010583788076 -123.29279489519854) bank102632 +102633 POINT(48.07132652215444 -122.39522053369708) bank102633 +102634 POINT(47.49915021988706 -121.75324904821707) bank102634 +102635 POINT(47.673176199976915 -122.7692261268242) bank102635 +102636 POINT(48.60361296731048 -121.96888748084359) bank102636 +102637 POINT(48.5369086779484 -121.74578494348387) bank102637 +102638 POINT(47.8139336353343 -121.71767377619261) bank102638 +102639 POINT(48.072905184417685 -121.9865851317916) bank102639 +102640 POINT(47.91288905766687 -122.39992759937087) bank102640 +102641 POINT(47.110034228895344 -121.70884398855573) bank102641 +102642 POINT(47.001068526498756 -121.77350062045305) bank102642 +102643 POINT(47.469062304018756 -121.61373093824734) bank102643 +102644 POINT(48.16811714771267 -122.45217257191577) bank102644 +102645 POINT(48.16192223925171 -122.09804202339033) bank102645 +102646 POINT(48.57853870036305 -122.01161394370283) bank102646 +102647 POINT(46.812799281647955 -121.40603743157604) bank102647 +102648 POINT(48.04898131016457 -122.04721936902891) bank102648 +102649 POINT(47.293877506848034 -122.42809001332903) bank102649 +102650 POINT(48.272563635006364 -122.42408740175384) bank102650 +102651 POINT(47.47297054950621 -121.5082285540512) bank102651 +102652 POINT(46.74232472775581 -122.08458803378105) bank102652 +102653 POINT(48.41367607818736 -121.68822155826406) bank102653 +102654 POINT(47.72427822246276 -122.08655435306966) bank102654 +102655 POINT(47.363860604514514 -121.58088688277329) bank102655 +102656 POINT(48.21274248148502 -121.93334992108106) bank102656 +102657 POINT(47.77706842141331 -123.32041339620636) bank102657 +102658 POINT(47.75644853697049 -121.76581941016163) bank102658 +102659 POINT(46.87552407289611 -122.15339813147455) bank102659 +102660 POINT(48.244291584235384 -122.5089617160401) bank102660 +102661 POINT(48.23174519959489 -122.51389958885005) bank102661 +102662 POINT(47.18936577045791 -121.61428095766519) bank102662 +102663 POINT(47.87616889283695 -122.76671030394645) bank102663 +102664 POINT(47.98268647111189 -121.41629335579339) bank102664 +102665 POINT(47.84675974504619 -123.31293381728103) bank102665 +102666 POINT(46.85814600017217 -121.91307353693833) bank102666 +102667 POINT(48.34759560677245 -122.75083281091354) bank102667 +102668 POINT(48.35655326605199 -121.90073879862526) bank102668 +102669 POINT(47.97226354948838 -122.54618474899948) bank102669 +102670 POINT(46.899856401024415 -121.94165111688946) bank102670 +102671 POINT(48.45900625514072 -122.79837040217564) bank102671 +102672 POINT(47.624887284048235 -122.16764646791455) bank102672 +102673 POINT(48.149776507826594 -122.85934195924649) bank102673 +102674 POINT(46.676691676209835 -122.6478983937064) bank102674 +102675 POINT(47.65916629966494 -121.88046441202604) bank102675 +102676 POINT(48.442964690788706 -121.41062582517249) bank102676 +102677 POINT(48.16368548807652 -121.80504585560416) bank102677 +102678 POINT(47.934618258495156 -121.91348472183721) bank102678 +102679 POINT(48.04290238314699 -122.93572339041931) bank102679 +102680 POINT(47.26383613258311 -123.31586416904943) bank102680 +102681 POINT(48.08424805591536 -122.64431754571312) bank102681 +102682 POINT(46.85459963662402 -121.5111039506976) bank102682 +102683 POINT(47.77950163006515 -122.65617146744714) bank102683 +102684 POINT(47.22758158787472 -121.90677598445092) bank102684 +102685 POINT(47.2377826858541 -123.05409516160827) bank102685 +102686 POINT(46.704434987785106 -121.60703759137127) bank102686 +102687 POINT(48.46421658102594 -122.97488516972548) bank102687 +102688 POINT(48.35832486259828 -123.21099248483446) bank102688 +102689 POINT(47.498738998755925 -123.23312394855056) bank102689 +102690 POINT(48.55894088641468 -123.13165983418445) bank102690 +102691 POINT(47.20869346653136 -122.5832586203229) bank102691 +102692 POINT(46.97819695998822 -122.01976287346524) bank102692 +102693 POINT(47.987882459388295 -122.65043126703121) bank102693 +102694 POINT(48.014648921090874 -121.45231900908009) bank102694 +102695 POINT(48.05306548829119 -122.89475059095936) bank102695 +102696 POINT(46.88415956672539 -121.39530990140034) bank102696 +102697 POINT(46.79807893785631 -122.2400099231694) bank102697 +102698 POINT(48.59023155154112 -122.88981867933464) bank102698 +102699 POINT(48.552967964662145 -122.13613418950517) bank102699 +102700 POINT(47.47547903173732 -122.35600978364903) bank102700 +102701 POINT(48.06814890319645 -122.77926188958199) bank102701 +102702 POINT(47.77948659794074 -121.55248782108879) bank102702 +102703 POINT(47.497483606258044 -123.20495776047852) bank102703 +102704 POINT(48.44271496777855 -122.30965416493906) bank102704 +102705 POINT(47.882025905554656 -123.14039055811946) bank102705 +102706 POINT(48.40597147349617 -122.57804850970835) bank102706 +102707 POINT(47.33621958207236 -121.51414014998664) bank102707 +102708 POINT(48.42577807626874 -123.18769973942717) bank102708 +102709 POINT(47.05739318287724 -121.72602618906107) bank102709 +102710 POINT(48.052913959433184 -122.6458175335203) bank102710 +102711 POINT(47.94107923212013 -121.49597132965636) bank102711 +102712 POINT(47.813409770423206 -122.29688780591425) bank102712 +102713 POINT(47.02684251123789 -122.90123923621381) bank102713 +102714 POINT(47.75771597882062 -122.43405438150171) bank102714 +102715 POINT(47.1401980970353 -122.00406642072586) bank102715 +102716 POINT(48.14194129867088 -122.17045928849697) bank102716 +102717 POINT(47.253648260240894 -121.90104010457442) bank102717 +102718 POINT(47.59391548840399 -121.67274948350709) bank102718 +102719 POINT(46.83732970483406 -121.78885939124609) bank102719 +102720 POINT(48.05147109130682 -122.87195724430907) bank102720 +102721 POINT(47.462254870787554 -123.12556093549513) bank102721 +102722 POINT(48.52885152813173 -121.7173948783792) bank102722 +102723 POINT(47.07287241638158 -122.64862529923366) bank102723 +102724 POINT(47.37134847774037 -122.10048292953961) bank102724 +102725 POINT(47.25460197369285 -122.34361570952883) bank102725 +102726 POINT(48.31591275267285 -121.69186058482994) bank102726 +102727 POINT(48.4622444958099 -123.0305443452534) bank102727 +102728 POINT(47.91953369784728 -123.24666266865123) bank102728 +102729 POINT(47.00071298940895 -122.49598831726783) bank102729 +102730 POINT(48.34917399296886 -122.32041439623151) bank102730 +102731 POINT(48.54823589471868 -121.72789922127832) bank102731 +102732 POINT(48.460981572891924 -122.87083719939871) bank102732 +102733 POINT(47.78334917106144 -121.35727753510899) bank102733 +102734 POINT(48.19430418422293 -121.99130952459524) bank102734 +102735 POINT(46.95872122547457 -122.86736392030512) bank102735 +102736 POINT(48.30903428672629 -122.78362960886315) bank102736 +102737 POINT(47.44769735958894 -121.51710083804059) bank102737 +102738 POINT(48.44884922516625 -122.16462550682857) bank102738 +102739 POINT(48.34982658368807 -123.26040162425546) bank102739 +102740 POINT(48.31613555079651 -121.35086293349315) bank102740 +102741 POINT(46.89829861957109 -122.5402252241809) bank102741 +102742 POINT(48.24408982224177 -121.52676073518694) bank102742 +102743 POINT(47.24690484709077 -123.32414990610597) bank102743 +102744 POINT(48.24750211663087 -122.96248313047684) bank102744 +102745 POINT(47.03941811992078 -122.91273099363696) bank102745 +102746 POINT(46.96418123121827 -122.31804664333711) bank102746 +102747 POINT(47.70722938382116 -122.76284598803142) bank102747 +102748 POINT(46.95111527565913 -122.89598565313347) bank102748 +102749 POINT(48.30859485885003 -122.43428063603935) bank102749 +102750 POINT(46.659558613193035 -121.5268015338116) bank102750 +102751 POINT(48.16738933402638 -122.67885034234808) bank102751 +102752 POINT(47.536234155446024 -122.65661639718425) bank102752 +102753 POINT(48.19352422100809 -122.30855665431426) bank102753 +102754 POINT(47.75196765132804 -122.4384191144372) bank102754 +102755 POINT(46.63550482613474 -122.11961338788207) bank102755 +102756 POINT(48.07629689315668 -123.26991255397843) bank102756 +102757 POINT(47.31802987934843 -121.89810841119723) bank102757 +102758 POINT(47.67144442382591 -122.36796618244757) bank102758 +102759 POINT(48.184042564245814 -121.79987820233023) bank102759 +102760 POINT(48.00731363098576 -122.83868522534375) bank102760 +102761 POINT(48.469413190591304 -122.5445364153455) bank102761 +102762 POINT(46.9812699047336 -122.21206215928012) bank102762 +102763 POINT(47.0948215593229 -122.5758263669068) bank102763 +102764 POINT(47.472532295160114 -122.17098533933444) bank102764 +102765 POINT(47.270677639678276 -122.9155691118911) bank102765 +102766 POINT(47.02660900638296 -122.15772425237158) bank102766 +102767 POINT(47.397721408692426 -123.27302211795549) bank102767 +102768 POINT(47.46389179318157 -123.05495961942962) bank102768 +102769 POINT(47.11982173736491 -122.0899752553957) bank102769 +102770 POINT(46.63687854625923 -122.80339460250795) bank102770 +102771 POINT(46.756457638831115 -122.52847065728794) bank102771 +102772 POINT(48.44671954908337 -123.25195290262033) bank102772 +102773 POINT(47.96553199081567 -121.68952053026742) bank102773 +102774 POINT(47.659057316783674 -122.69545458762893) bank102774 +102775 POINT(47.88880844137506 -122.89392142479015) bank102775 +102776 POINT(46.72294406554178 -122.09054527490987) bank102776 +102777 POINT(47.21143108683543 -122.8218499549807) bank102777 +102778 POINT(47.04871069941027 -122.8062508169007) bank102778 +102779 POINT(48.16172251476545 -123.13876310344475) bank102779 +102780 POINT(47.997009107278565 -122.40216073074792) bank102780 +102781 POINT(48.20977567020956 -123.14615822142916) bank102781 +102782 POINT(47.929228441114624 -121.42450396352099) bank102782 +102783 POINT(46.994349143885216 -122.83277818175996) bank102783 +102784 POINT(47.578155956421874 -123.12350290957521) bank102784 +102785 POINT(48.366388585768945 -123.27631091947777) bank102785 +102786 POINT(47.33855239340604 -122.69992457118002) bank102786 +102787 POINT(47.5816918066489 -123.29742476751825) bank102787 +102788 POINT(47.23164018211558 -123.29584582410314) bank102788 +102789 POINT(47.48474037863184 -121.5964484428041) bank102789 +102790 POINT(47.41111911043206 -122.61251851626997) bank102790 +102791 POINT(47.39170321704943 -122.68427237304248) bank102791 +102792 POINT(47.23833258436456 -122.69411303788115) bank102792 +102793 POINT(46.73809683837659 -123.014171473852) bank102793 +102794 POINT(47.77651614321866 -122.68638543959497) bank102794 +102795 POINT(47.902743357907546 -122.73171097959565) bank102795 +102796 POINT(46.652397826014024 -121.7581632038644) bank102796 +102797 POINT(47.71764202750778 -122.38073110158142) bank102797 +102798 POINT(47.61427282917342 -121.3719684725215) bank102798 +102799 POINT(47.583699693556255 -123.0842543927536) bank102799 +102800 POINT(47.6187255930236 -121.84675778580967) bank102800 +102801 POINT(48.47436592498254 -122.85437103422136) bank102801 +102802 POINT(47.79233441897663 -122.43066284811653) bank102802 +102803 POINT(47.37806342250616 -121.85185988233631) bank102803 +102804 POINT(46.90982399654288 -122.45371929819116) bank102804 +102805 POINT(47.276005574900374 -122.05342588468609) bank102805 +102806 POINT(48.1495071582344 -122.74049963353501) bank102806 +102807 POINT(48.26536193392865 -122.1226957667591) bank102807 +102808 POINT(46.80372575694609 -123.10866039275083) bank102808 +102809 POINT(46.99074469012054 -121.68307859862728) bank102809 +102810 POINT(48.46652846312852 -122.0331694959619) bank102810 +102811 POINT(47.70509447957404 -122.0841637985262) bank102811 +102812 POINT(47.54143356489345 -121.61518260455011) bank102812 +102813 POINT(47.837282405206565 -121.73725933458307) bank102813 +102814 POINT(47.51906800675096 -121.49675110970368) bank102814 +102815 POINT(47.660931347915906 -122.94943355390248) bank102815 +102816 POINT(47.74925102249167 -122.31249524864094) bank102816 +102817 POINT(47.14966127727412 -122.77335808848976) bank102817 +102818 POINT(48.19200725640795 -121.65773735607668) bank102818 +102819 POINT(47.57996064437574 -123.07815719057491) bank102819 +102820 POINT(47.162975772356724 -122.01107749939509) bank102820 +102821 POINT(47.6379170009153 -123.24677702118308) bank102821 +102822 POINT(48.36553689871503 -122.38851627044859) bank102822 +102823 POINT(46.61805732720507 -121.93371544076113) bank102823 +102824 POINT(47.477595047931906 -122.0416915143903) bank102824 +102825 POINT(48.18848572102566 -121.41880789081465) bank102825 +102826 POINT(47.276426945745115 -121.6484491087566) bank102826 +102827 POINT(48.14215582445301 -121.35965017590578) bank102827 +102828 POINT(47.65934536160753 -121.51838296912395) bank102828 +102829 POINT(48.116509381042576 -122.36403270510455) bank102829 +102830 POINT(47.947401498886855 -122.8034387884374) bank102830 +102831 POINT(47.334041394556955 -122.74807737896428) bank102831 +102832 POINT(46.79669548613422 -122.14927971919867) bank102832 +102833 POINT(47.6829756078937 -122.26331899705166) bank102833 +102834 POINT(47.785090576603324 -122.7909787486937) bank102834 +102835 POINT(48.408008864857926 -122.25453847434957) bank102835 +102836 POINT(46.88197402865165 -123.20786134785996) bank102836 +102837 POINT(48.57945563263824 -122.25216932506726) bank102837 +102838 POINT(47.636257758007574 -122.99302443881881) bank102838 +102839 POINT(47.21223992364177 -122.73931120885361) bank102839 +102840 POINT(47.3657592586144 -122.18055043315083) bank102840 +102841 POINT(46.71468828874146 -123.31619959718581) bank102841 +102842 POINT(47.97758668878107 -122.68608283451005) bank102842 +102843 POINT(46.68136391972074 -122.81742412630565) bank102843 +102844 POINT(47.94716752174378 -121.76502024823937) bank102844 +102845 POINT(47.42097330254835 -122.22407347656971) bank102845 +102846 POINT(48.53892486120245 -122.43826653192617) bank102846 +102847 POINT(47.908565705239624 -123.13604669036398) bank102847 +102848 POINT(47.9642031935814 -122.64284912282884) bank102848 +102849 POINT(47.53270861220136 -122.18535977651563) bank102849 +102850 POINT(46.934740512736525 -122.37528120665354) bank102850 +102851 POINT(47.71188863523624 -121.93929822504651) bank102851 +102852 POINT(48.59210975619013 -123.23061374614656) bank102852 +102853 POINT(48.382367903019194 -123.02295400821298) bank102853 +102854 POINT(47.112029605768726 -122.69517870773204) bank102854 +102855 POINT(47.12597873782016 -121.42555647701909) bank102855 +102856 POINT(47.26656562506011 -122.52534053882445) bank102856 +102857 POINT(46.62198552806075 -121.34218754788066) bank102857 +102858 POINT(48.47840441965319 -122.84895083492242) bank102858 +102859 POINT(47.890467312708246 -122.68662117598626) bank102859 +102860 POINT(47.070672903128305 -121.6271413273612) bank102860 +102861 POINT(47.630383396773944 -122.30815147980486) bank102861 +102862 POINT(48.31478067702079 -121.35211909186147) bank102862 +102863 POINT(48.08403214981569 -123.1087575257833) bank102863 +102864 POINT(47.72773381514593 -122.70156820463733) bank102864 +102865 POINT(47.6393869386888 -122.73201042749216) bank102865 +102866 POINT(46.9138453992945 -121.63558024797301) bank102866 +102867 POINT(46.69835314468223 -122.81893074490598) bank102867 +102868 POINT(47.67686793310588 -121.71447525210313) bank102868 +102869 POINT(48.36200631366676 -123.14653303733627) bank102869 +102870 POINT(46.88899348704725 -123.0933393752876) bank102870 +102871 POINT(47.39705478884274 -121.58134649007359) bank102871 +102872 POINT(46.65389113598218 -121.45422005703665) bank102872 +102873 POINT(47.562360977820134 -121.95747158313644) bank102873 +102874 POINT(46.95845180487601 -122.3857643141229) bank102874 +102875 POINT(48.14838507353532 -123.28560568225883) bank102875 +102876 POINT(47.477435836522965 -121.81210908666439) bank102876 +102877 POINT(47.04502165735431 -122.28405826359621) bank102877 +102878 POINT(47.752617577878084 -122.956786069477) bank102878 +102879 POINT(48.4540241723131 -122.35937539651034) bank102879 +102880 POINT(48.24288903786331 -121.71196130026547) bank102880 +102881 POINT(48.42933481686166 -121.96398690290658) bank102881 +102882 POINT(48.19765854720669 -121.58630883457498) bank102882 +102883 POINT(46.65356944022576 -121.92624289502018) bank102883 +102884 POINT(47.93880601067991 -122.96733879787539) bank102884 +102885 POINT(47.94572375408045 -122.56541292743817) bank102885 +102886 POINT(47.262593524770615 -122.95215645514152) bank102886 +102887 POINT(48.36325795215827 -122.98776042882132) bank102887 +102888 POINT(46.93827897691382 -121.69869242027622) bank102888 +102889 POINT(48.41515686913734 -122.06371394701031) bank102889 +102890 POINT(48.05481043214406 -122.58003450565197) bank102890 +102891 POINT(47.65851914567129 -122.3854740195947) bank102891 +102892 POINT(48.218955934507655 -121.93425170560964) bank102892 +102893 POINT(47.330265567773736 -122.48754998896177) bank102893 +102894 POINT(48.5908337118715 -123.03508781507774) bank102894 +102895 POINT(48.36113257034961 -122.5458399988386) bank102895 +102896 POINT(48.581417767214376 -122.12210111604412) bank102896 +102897 POINT(47.67579076652383 -122.02151659513157) bank102897 +102898 POINT(47.80742784006468 -121.97449476110711) bank102898 +102899 POINT(48.008401531053785 -123.08787358597435) bank102899 +102900 POINT(47.79267883259523 -122.60055554194135) bank102900 +102901 POINT(47.72995415283383 -123.03856022142656) bank102901 +102902 POINT(47.564508009476086 -123.11206092602387) bank102902 +102903 POINT(47.68230247711331 -122.97429604815912) bank102903 +102904 POINT(47.672535408215616 -122.48047545453508) bank102904 +102905 POINT(47.706497596509216 -121.82582190242177) bank102905 +102906 POINT(47.47106124191958 -122.26554795737111) bank102906 +102907 POINT(48.20884495294011 -121.90516407972159) bank102907 +102908 POINT(47.56759250191134 -122.35219966246393) bank102908 +102909 POINT(47.35117840224183 -123.2225530705241) bank102909 +102910 POINT(46.79207839941778 -123.00738859235838) bank102910 +102911 POINT(48.18461379896332 -122.3328136775602) bank102911 +102912 POINT(46.69866166847563 -122.45517007509861) bank102912 +102913 POINT(48.4345384279276 -123.03034482900826) bank102913 +102914 POINT(48.082906008004606 -122.29515290683199) bank102914 +102915 POINT(47.36034150781545 -123.28285917747165) bank102915 +102916 POINT(47.86041137522451 -121.86952107493921) bank102916 +102917 POINT(46.78344762102625 -121.96276751844314) bank102917 +102918 POINT(46.62887687679689 -122.13291614730376) bank102918 +102919 POINT(48.062325140043825 -121.42761585760441) bank102919 +102920 POINT(47.57879683434982 -121.87843457970494) bank102920 +102921 POINT(48.00856392444953 -122.39505732127097) bank102921 +102922 POINT(47.96208362737343 -122.47570362948964) bank102922 +102923 POINT(47.23257896202142 -122.59051177870103) bank102923 +102924 POINT(47.13106506783402 -121.87432589528302) bank102924 +102925 POINT(46.75940443764099 -122.96706369787705) bank102925 +102926 POINT(48.06858639761737 -121.46668934861809) bank102926 +102927 POINT(48.39512981001923 -122.630976655462) bank102927 +102928 POINT(46.825795930058085 -122.03148550136821) bank102928 +102929 POINT(48.47709785539684 -122.66233373027421) bank102929 +102930 POINT(48.35795379215238 -122.93150954293662) bank102930 +102931 POINT(47.486916526920595 -121.8864720794423) bank102931 +102932 POINT(48.04539357578871 -122.3011489352847) bank102932 +102933 POINT(47.02417902279208 -121.60696526165519) bank102933 +102934 POINT(46.9172833841971 -122.73407300937464) bank102934 +102935 POINT(48.43712123619538 -122.45525469040649) bank102935 +102936 POINT(48.33787376539227 -122.77187653353887) bank102936 +102937 POINT(48.13827246769314 -121.48563239960792) bank102937 +102938 POINT(46.78465408381283 -121.3464133042957) bank102938 +102939 POINT(47.49227929117226 -123.28116681772444) bank102939 +102940 POINT(47.26629708347045 -123.22919510473378) bank102940 +102941 POINT(48.516345733386345 -122.34288481232352) bank102941 +102942 POINT(47.55850953616192 -121.73868270507663) bank102942 +102943 POINT(48.28764284305042 -121.33224756973193) bank102943 +102944 POINT(47.25486829406898 -121.50326480903576) bank102944 +102945 POINT(47.987253585922005 -121.97214273033278) bank102945 +102946 POINT(47.41719661477001 -122.80442660516917) bank102946 +102947 POINT(47.61661620271501 -121.58864645667926) bank102947 +102948 POINT(47.54473347455524 -122.85229165940811) bank102948 +102949 POINT(47.32806001778622 -121.82413616852342) bank102949 +102950 POINT(48.451906044715045 -122.25376807253012) bank102950 +102951 POINT(48.27023437868854 -123.19987768873945) bank102951 +102952 POINT(47.92808941338139 -122.07502189907376) bank102952 +102953 POINT(48.507158440832306 -122.08728533128193) bank102953 +102954 POINT(48.14245943463649 -122.0583112950635) bank102954 +102955 POINT(47.03713837142928 -123.01997903561934) bank102955 +102956 POINT(48.321831950442196 -122.6980969291447) bank102956 +102957 POINT(46.69185041418476 -121.36737304704768) bank102957 +102958 POINT(48.52579192002167 -122.13347765181675) bank102958 +102959 POINT(47.61266410828268 -123.0795173139289) bank102959 +102960 POINT(46.66893082297739 -122.21272686523035) bank102960 +102961 POINT(48.5732753116019 -121.50116670588727) bank102961 +102962 POINT(48.316453016057835 -122.70476240261503) bank102962 +102963 POINT(46.862775079208056 -121.57071649402697) bank102963 +102964 POINT(47.79947218048307 -123.05839410129944) bank102964 +102965 POINT(48.47690984073439 -121.33251760324127) bank102965 +102966 POINT(47.31114142913514 -121.35755977918079) bank102966 +102967 POINT(46.735125912869584 -122.39229570329506) bank102967 +102968 POINT(48.26884411938761 -123.22790161982049) bank102968 +102969 POINT(46.99025946172729 -121.78613475763024) bank102969 +102970 POINT(48.45243392100268 -121.92995580132711) bank102970 +102971 POINT(46.994924943660216 -123.0813844555386) bank102971 +102972 POINT(47.083858824226304 -122.90034178000282) bank102972 +102973 POINT(47.33521030163254 -122.36847460817036) bank102973 +102974 POINT(47.52726413946739 -122.34814881309946) bank102974 +102975 POINT(48.12272577797534 -121.95161820035317) bank102975 +102976 POINT(47.34663910653195 -121.33500078799491) bank102976 +102977 POINT(48.586373111584514 -122.25143926304976) bank102977 +102978 POINT(47.35091022484532 -123.14148349175399) bank102978 +102979 POINT(48.077893596553665 -121.53564036032473) bank102979 +102980 POINT(46.61116025101571 -122.0711430461683) bank102980 +102981 POINT(46.87858362654038 -122.98365101608007) bank102981 +102982 POINT(48.470124111412765 -121.97123190478854) bank102982 +102983 POINT(47.802400889909 -121.87887883393928) bank102983 +102984 POINT(47.312486692544354 -121.35644077624052) bank102984 +102985 POINT(48.03043079159954 -122.04322903957303) bank102985 +102986 POINT(46.97738803814433 -122.92599402369495) bank102986 +102987 POINT(48.3193535206956 -123.21098009258544) bank102987 +102988 POINT(47.873207843868435 -121.59986931152982) bank102988 +102989 POINT(47.49154481194005 -121.88028258337765) bank102989 +102990 POINT(47.62947167764928 -122.91282071392806) bank102990 +102991 POINT(46.761414227625195 -122.45038627353313) bank102991 +102992 POINT(48.09715676771818 -122.8057243078249) bank102992 +102993 POINT(47.85260112667833 -122.97757579821297) bank102993 +102994 POINT(47.28301195038947 -121.50344755867575) bank102994 +102995 POINT(47.45967774568832 -122.35581936452427) bank102995 +102996 POINT(46.819483597143666 -123.11911560546702) bank102996 +102997 POINT(48.50570058468312 -122.40338847413712) bank102997 +102998 POINT(48.10020953319399 -121.43894156393566) bank102998 +102999 POINT(47.31421726087839 -122.3515238109084) bank102999 +103000 POINT(47.216223116261496 -121.36789119455756) bank103000 +103001 POINT(46.61408897342112 -122.1914612528799) bank103001 +103002 POINT(48.60548049453173 -123.1748215959078) bank103002 +103003 POINT(47.82415381013996 -121.33412418222633) bank103003 +103004 POINT(47.90364409309579 -122.89544087492422) bank103004 +103005 POINT(47.24677897950295 -122.55846557108254) bank103005 +103006 POINT(46.717656822294465 -121.49616281771624) bank103006 +103007 POINT(47.24682244749892 -122.96116530813246) bank103007 +103008 POINT(48.578364945002285 -122.11267218772346) bank103008 +103009 POINT(47.31574254053655 -121.94250918340977) bank103009 +103010 POINT(46.815808892558195 -122.90237250954023) bank103010 +103011 POINT(47.12877853827467 -121.7249477449194) bank103011 +103012 POINT(48.259686776798944 -122.66486497280152) bank103012 +103013 POINT(47.268860256419586 -122.79902923366754) bank103013 +103014 POINT(46.63389164976854 -122.62863288544058) bank103014 +103015 POINT(48.534898666713715 -121.78788441622325) bank103015 +103016 POINT(48.24656285992485 -122.70356583412205) bank103016 +103017 POINT(47.64874151164467 -121.52387969407366) bank103017 +103018 POINT(47.76539525800609 -122.11557555225264) bank103018 +103019 POINT(48.22746930164463 -121.43770283474676) bank103019 +103020 POINT(46.98267356991959 -123.31420447532389) bank103020 +103021 POINT(47.020400119429844 -121.57000285170854) bank103021 +103022 POINT(47.80847664259867 -122.21972625612678) bank103022 +103023 POINT(47.310138229221906 -121.57914395342267) bank103023 +103024 POINT(47.68416983824498 -122.40819034527959) bank103024 +103025 POINT(47.976932922827004 -122.33703948084926) bank103025 +103026 POINT(47.17837136538494 -122.01754932648122) bank103026 +103027 POINT(46.82557956509118 -122.64594124690761) bank103027 +103028 POINT(47.36681952507677 -122.49329239230055) bank103028 +103029 POINT(46.82702433016104 -122.45319775227667) bank103029 +103030 POINT(48.25694089659545 -121.362088884337) bank103030 +103031 POINT(48.5850052749281 -123.19362712269522) bank103031 +103032 POINT(46.83287906360095 -123.25013898207237) bank103032 +103033 POINT(47.967171008996345 -121.60025088927013) bank103033 +103034 POINT(48.022408445081446 -121.46584278416225) bank103034 +103035 POINT(48.1158796839593 -121.89084388877875) bank103035 +103036 POINT(48.261699597768896 -123.12316743889633) bank103036 +103037 POINT(48.00007588766309 -121.4776199278195) bank103037 +103038 POINT(48.09617027933363 -121.84806168641938) bank103038 +103039 POINT(48.29166804812778 -122.25151221308253) bank103039 +103040 POINT(48.53952395465459 -121.63526966019795) bank103040 +103041 POINT(48.494856182571674 -122.85167535820551) bank103041 +103042 POINT(47.87076375097702 -123.00254066111532) bank103042 +103043 POINT(47.67570400546203 -121.37649885807497) bank103043 +103044 POINT(48.17571915830218 -121.36696606495066) bank103044 +103045 POINT(46.87394929040334 -121.95906602015545) bank103045 +103046 POINT(47.99116104630232 -123.01757725325521) bank103046 +103047 POINT(47.05255106554704 -121.58982948369662) bank103047 +103048 POINT(47.540480361522754 -122.12370593039066) bank103048 +103049 POINT(46.76261320156991 -122.59400949641804) bank103049 +103050 POINT(47.76115038706229 -122.14970462322329) bank103050 +103051 POINT(46.78339846464276 -122.05222644264829) bank103051 +103052 POINT(47.755018649475886 -123.01035285538663) bank103052 +103053 POINT(48.27398101506003 -121.48052596535722) bank103053 +103054 POINT(48.40235234089002 -122.57832435369387) bank103054 +103055 POINT(48.23152855141226 -122.76993463481395) bank103055 +103056 POINT(48.23969075264161 -122.24257907639145) bank103056 +103057 POINT(47.65663307744837 -122.48814825706022) bank103057 +103058 POINT(47.62802570867497 -122.50508700736793) bank103058 +103059 POINT(47.745088889785855 -121.48016889388536) bank103059 +103060 POINT(47.54783226988028 -122.62574144778442) bank103060 +103061 POINT(48.22394541895392 -123.04030286453583) bank103061 +103062 POINT(48.44597195024519 -123.12301861325494) bank103062 +103063 POINT(46.732663955247176 -123.21830008955952) bank103063 +103064 POINT(48.077981983586774 -121.52187898183077) bank103064 +103065 POINT(48.399278834122605 -122.60097116506257) bank103065 +103066 POINT(47.46792853360249 -122.49592726910294) bank103066 +103067 POINT(46.98828141577548 -123.15582810624949) bank103067 +103068 POINT(48.43626929844374 -123.18658496278015) bank103068 +103069 POINT(48.46545205148397 -121.83548826537805) bank103069 +103070 POINT(47.001643999105575 -123.31939014526246) bank103070 +103071 POINT(47.03705325838866 -123.09111896517872) bank103071 +103072 POINT(48.142385271131054 -121.51710724854128) bank103072 +103073 POINT(48.14243152671068 -122.08901386588954) bank103073 +103074 POINT(48.39406488636772 -121.74352005050878) bank103074 +103075 POINT(46.71863038949648 -122.19587317171447) bank103075 +103076 POINT(48.43595733020707 -121.66452921794831) bank103076 +103077 POINT(47.65738953211015 -122.72445125815976) bank103077 +103078 POINT(47.416834727708846 -122.20760736896493) bank103078 +103079 POINT(48.20180387126315 -122.8050261756234) bank103079 +103080 POINT(48.281523381955765 -121.64295865084893) bank103080 +103081 POINT(48.26887744642457 -121.34262637004856) bank103081 +103082 POINT(47.225993044331446 -121.41586883854781) bank103082 +103083 POINT(48.30158620546406 -122.53035485356564) bank103083 +103084 POINT(47.800877210876656 -121.70560710714155) bank103084 +103085 POINT(47.869570493346416 -122.42981823870228) bank103085 +103086 POINT(47.18694684933269 -122.79136633179) bank103086 +103087 POINT(48.20416590286825 -122.99450966923187) bank103087 +103088 POINT(48.13365636278889 -122.94281047186772) bank103088 +103089 POINT(48.39409135509916 -122.65810541402645) bank103089 +103090 POINT(48.41454676447693 -123.30063713078545) bank103090 +103091 POINT(47.692705592167236 -121.59568789747269) bank103091 +103092 POINT(46.850862640184666 -121.37860616400746) bank103092 +103093 POINT(47.63036336249066 -122.89246635572121) bank103093 +103094 POINT(47.37348494297029 -121.54860989585296) bank103094 +103095 POINT(46.63899397987969 -121.69854707256884) bank103095 +103096 POINT(47.87700206715913 -121.36198455082116) bank103096 +103097 POINT(48.493452775762975 -123.21097618077648) bank103097 +103098 POINT(47.03865717096655 -121.68497206425695) bank103098 +103099 POINT(47.25904105753627 -121.51558130242464) bank103099 +103100 POINT(47.65948612335525 -123.29678565573406) bank103100 +103101 POINT(48.13436867671062 -122.62367067284693) bank103101 +103102 POINT(48.22391349738372 -122.57611951787023) bank103102 +103103 POINT(46.91002770525137 -121.83862413844304) bank103103 +103104 POINT(47.203819327263574 -122.01634777477324) bank103104 +103105 POINT(48.35596273943719 -121.72598167385433) bank103105 +103106 POINT(47.352054855807346 -123.08699340747538) bank103106 +103107 POINT(47.851577576054254 -122.07980277422308) bank103107 +103108 POINT(46.774263754174335 -123.0808645217224) bank103108 +103109 POINT(48.030674085724726 -122.67915312816946) bank103109 +103110 POINT(47.2233183464682 -121.98529098164671) bank103110 +103111 POINT(47.12380221917513 -122.25640243663292) bank103111 +103112 POINT(47.476015368038006 -122.11438776694867) bank103112 +103113 POINT(48.011683316279594 -121.86360027869645) bank103113 +103114 POINT(48.478976179950756 -123.24406313623163) bank103114 +103115 POINT(47.67173755001247 -122.36278454356113) bank103115 +103116 POINT(48.35358219753856 -121.4803649827252) bank103116 +103117 POINT(48.3328493912037 -121.54427512821215) bank103117 +103118 POINT(48.210364089548634 -121.47031357036603) bank103118 +103119 POINT(47.06982419051461 -122.96197323056917) bank103119 +103120 POINT(48.02347694878026 -121.60378472831117) bank103120 +103121 POINT(47.512559910375415 -122.89100394662269) bank103121 +103122 POINT(48.511424650953494 -123.13806381517341) bank103122 +103123 POINT(48.3447127283721 -123.31945517114508) bank103123 +103124 POINT(47.756911299075114 -123.29986454713263) bank103124 +103125 POINT(48.02722342430997 -122.39674727856588) bank103125 +103126 POINT(48.522892182033175 -121.44636755983655) bank103126 +103127 POINT(46.90153980428409 -121.4650716847307) bank103127 +103128 POINT(47.52396983724981 -122.55410044004567) bank103128 +103129 POINT(47.34400387132135 -121.71853314891194) bank103129 +103130 POINT(47.352694408819914 -122.80078249882641) bank103130 +103131 POINT(47.00477114868124 -123.22670816783697) bank103131 +103132 POINT(47.224516280914365 -122.7115488130904) bank103132 +103133 POINT(48.59844769755731 -122.85366272101858) bank103133 +103134 POINT(47.869590253731374 -123.20272222803057) bank103134 +103135 POINT(48.38128980195278 -122.9109729002549) bank103135 +103136 POINT(47.36717032485703 -122.02818786776822) bank103136 +103137 POINT(47.69707997622097 -122.46707192480902) bank103137 +103138 POINT(47.525484367032384 -121.90291134685586) bank103138 +103139 POINT(47.847693816114756 -121.49953927909019) bank103139 +103140 POINT(48.032179858384815 -122.8908656661925) bank103140 +103141 POINT(47.08267032378604 -122.9208388910112) bank103141 +103142 POINT(47.865008732866336 -123.00195979360741) bank103142 +103143 POINT(47.32341218872594 -121.57616766628442) bank103143 +103144 POINT(47.85052086819836 -122.21458257834426) bank103144 +103145 POINT(48.02982748330882 -122.36423694376751) bank103145 +103146 POINT(47.4103442675954 -122.25093148678056) bank103146 +103147 POINT(46.83474483016523 -123.18802459079424) bank103147 +103148 POINT(48.308410835609855 -122.2193051549132) bank103148 +103149 POINT(47.820854136391375 -122.60960748095474) bank103149 +103150 POINT(47.332135261169896 -122.63993050777869) bank103150 +103151 POINT(48.51096614902669 -121.90069237083085) bank103151 +103152 POINT(47.195758724687295 -122.72271291711905) bank103152 +103153 POINT(47.585234613780635 -121.69431124059467) bank103153 +103154 POINT(46.691081245989515 -121.52384815806658) bank103154 +103155 POINT(46.86906884674753 -121.44825659933576) bank103155 +103156 POINT(47.928905215422574 -121.63415212534517) bank103156 +103157 POINT(47.83646490444076 -122.59493674261998) bank103157 +103158 POINT(46.775248724593176 -123.03743901495365) bank103158 +103159 POINT(47.23613363816078 -122.17026065256033) bank103159 +103160 POINT(48.462910656497115 -122.8073455152796) bank103160 +103161 POINT(48.51494261603671 -122.02504809596297) bank103161 +103162 POINT(47.89563415334927 -123.08208875552727) bank103162 +103163 POINT(48.41994369921287 -121.4032613880237) bank103163 +103164 POINT(46.9676478404582 -121.43761202428995) bank103164 +103165 POINT(47.030045406589736 -122.24440162572313) bank103165 +103166 POINT(48.16896094115505 -121.88865629828364) bank103166 +103167 POINT(48.019462205847326 -122.36473716654947) bank103167 +103168 POINT(48.27153415605398 -122.41976503368976) bank103168 +103169 POINT(47.448852572382364 -123.24919607136104) bank103169 +103170 POINT(47.28935720260964 -121.9091143291425) bank103170 +103171 POINT(47.74285529682376 -121.74085026884659) bank103171 +103172 POINT(46.75120704417708 -122.99944359897705) bank103172 +103173 POINT(47.34284432740058 -122.30902379699467) bank103173 +103174 POINT(48.32753113195082 -122.22620509386412) bank103174 +103175 POINT(48.10153607802791 -122.10102373462972) bank103175 +103176 POINT(48.557501942028615 -121.63387101544572) bank103176 +103177 POINT(47.65241789675516 -121.88876428754081) bank103177 +103178 POINT(47.23045264610872 -122.48969820162395) bank103178 +103179 POINT(48.48614929446771 -121.33980413725496) bank103179 +103180 POINT(46.66573600525551 -122.1857180452615) bank103180 +103181 POINT(48.24647474359473 -121.8116198527815) bank103181 +103182 POINT(48.411916660214075 -122.58654339869236) bank103182 +103183 POINT(48.05100145598932 -123.02679265591401) bank103183 +103184 POINT(47.139056873144554 -123.24276930067248) bank103184 +103185 POINT(47.98592786388186 -123.22641224206599) bank103185 +103186 POINT(48.53392194499144 -121.676805977243) bank103186 +103187 POINT(47.67839937369209 -121.54364858511858) bank103187 +103188 POINT(48.357413560287696 -122.8980457069984) bank103188 +103189 POINT(47.88184690092722 -123.10394664589141) bank103189 +103190 POINT(46.931131500291364 -123.08365189376116) bank103190 +103191 POINT(47.37991108196459 -122.75023081801498) bank103191 +103192 POINT(47.41858681723079 -121.4326946915775) bank103192 +103193 POINT(46.78114842670499 -123.18977436651032) bank103193 +103194 POINT(47.363308236260245 -121.90260852608722) bank103194 +103195 POINT(48.349191278373766 -121.82431950885538) bank103195 +103196 POINT(47.15387475791406 -122.3518313039225) bank103196 +103197 POINT(48.45175316859676 -122.91349159193209) bank103197 +103198 POINT(47.79026125095806 -122.8308951258197) bank103198 +103199 POINT(47.99804656387149 -121.76758791494628) bank103199 +103200 POINT(48.4737884523453 -122.18825306668714) bank103200 +103201 POINT(48.44843471705129 -121.70328619384753) bank103201 +103202 POINT(48.408398818495286 -123.03082314788747) bank103202 +103203 POINT(47.16910234887053 -122.79919448529441) bank103203 +103204 POINT(48.30085447880583 -122.84804871249592) bank103204 +103205 POINT(47.396412824309465 -121.85902477846459) bank103205 +103206 POINT(48.40332177673792 -122.44615869270672) bank103206 +103207 POINT(47.43297316495621 -122.58542557989735) bank103207 +103208 POINT(46.78928345199826 -122.89443841039929) bank103208 +103209 POINT(47.153324462558885 -122.632106347486) bank103209 +103210 POINT(46.643592240520846 -122.97783340818542) bank103210 +103211 POINT(48.18861775319402 -122.28359759634913) bank103211 +103212 POINT(48.334772641705115 -122.37959915501997) bank103212 +103213 POINT(47.70472790110685 -121.98967055898088) bank103213 +103214 POINT(47.72039679171584 -121.59346045204198) bank103214 +103215 POINT(47.127851767769485 -121.74741778937343) bank103215 +103216 POINT(48.43938563917608 -122.66103642167464) bank103216 +103217 POINT(48.07606425926026 -122.46658395990416) bank103217 +103218 POINT(48.08966367688951 -122.35794033284095) bank103218 +103219 POINT(48.58421134257422 -123.1719818539954) bank103219 +103220 POINT(47.57341275064599 -122.52456144491553) bank103220 +103221 POINT(47.68602618883064 -122.66466616843792) bank103221 +103222 POINT(47.74349467206004 -122.5754138060492) bank103222 +103223 POINT(47.21165806572527 -121.78383129314147) bank103223 +103224 POINT(47.909313799563655 -121.62180138210113) bank103224 +103225 POINT(46.88740859173102 -121.61112700628847) bank103225 +103226 POINT(48.40248290452429 -121.37563293208274) bank103226 +103227 POINT(47.37679973903732 -123.07655620813202) bank103227 +103228 POINT(47.194949226043526 -122.2490225644281) bank103228 +103229 POINT(46.78858878629258 -123.32658324067994) bank103229 +103230 POINT(48.11950784234131 -121.84481497539393) bank103230 +103231 POINT(46.90045893130714 -122.45047512354317) bank103231 +103232 POINT(48.47411625375221 -123.20037988523268) bank103232 +103233 POINT(48.28634234276194 -122.22075007621743) bank103233 +103234 POINT(48.263973434394494 -122.80248105339716) bank103234 +103235 POINT(47.123801426498915 -121.46550482540395) bank103235 +103236 POINT(47.48659228893867 -121.63162203136838) bank103236 +103237 POINT(47.993110932026106 -122.42321928409619) bank103237 +103238 POINT(48.33766226917656 -123.06863921674514) bank103238 +103239 POINT(47.81565807273267 -121.99404087604636) bank103239 +103240 POINT(47.447797971426525 -121.66611513611743) bank103240 +103241 POINT(46.893320478669295 -122.42842394421768) bank103241 +103242 POINT(47.10392867469046 -122.89463470078087) bank103242 +103243 POINT(47.126102669531726 -122.03905401206654) bank103243 +103244 POINT(47.92631752935692 -123.04713453152439) bank103244 +103245 POINT(47.10655672872082 -122.28096046543611) bank103245 +103246 POINT(46.6487181935697 -122.58936524454452) bank103246 +103247 POINT(47.87147017390644 -122.98538305020149) bank103247 +103248 POINT(48.362166865560425 -121.5946104073608) bank103248 +103249 POINT(48.156572408215425 -123.13713560604448) bank103249 +103250 POINT(47.60896813906503 -121.35877571966721) bank103250 +103251 POINT(48.597013886261834 -122.13693368306508) bank103251 +103252 POINT(47.5874836296792 -121.41866654249783) bank103252 +103253 POINT(47.521019171300104 -123.11807540024667) bank103253 +103254 POINT(46.68318362737902 -122.41484641993713) bank103254 +103255 POINT(48.43077817567073 -121.6598417879293) bank103255 +103256 POINT(47.32978308197455 -122.03969717146349) bank103256 +103257 POINT(47.69899604640806 -122.54583773031868) bank103257 +103258 POINT(46.62740975284391 -122.51823348981817) bank103258 +103259 POINT(47.49252936119777 -121.38775973717487) bank103259 +103260 POINT(47.80793403999408 -122.99597883475761) bank103260 +103261 POINT(47.83982112104904 -122.59511914938089) bank103261 +103262 POINT(48.538844623722696 -122.32115606901142) bank103262 +103263 POINT(47.81687585810374 -121.57872349500074) bank103263 +103264 POINT(47.754464604328575 -121.51428079937685) bank103264 +103265 POINT(48.078020742581955 -123.06925577324995) bank103265 +103266 POINT(46.64040355022947 -123.26947390583626) bank103266 +103267 POINT(47.99049249323103 -121.96242048665147) bank103267 +103268 POINT(48.33120018808422 -122.35386308697213) bank103268 +103269 POINT(47.96296054510654 -121.90918321353246) bank103269 +103270 POINT(46.65544439049731 -122.07221460746099) bank103270 +103271 POINT(47.326101005416845 -121.601647311334) bank103271 +103272 POINT(48.30683643615163 -121.60529823459215) bank103272 +103273 POINT(47.94390482584519 -121.8329834088724) bank103273 +103274 POINT(48.39532272130444 -122.58798660149826) bank103274 +103275 POINT(48.42181241074773 -121.60077078767569) bank103275 +103276 POINT(47.32142521116557 -122.30436400498485) bank103276 +103277 POINT(46.88113305840181 -122.47336306073382) bank103277 +103278 POINT(48.41787430330293 -122.58837509148373) bank103278 +103279 POINT(48.217666682224774 -122.58148623950427) bank103279 +103280 POINT(48.19444512895632 -122.94091429152888) bank103280 +103281 POINT(47.87910015994412 -121.80964027011926) bank103281 +103282 POINT(47.351461437846936 -121.8717556703746) bank103282 +103283 POINT(48.07424763578749 -122.9558849122507) bank103283 +103284 POINT(47.15279958576797 -121.34646084499596) bank103284 +103285 POINT(47.27614902093798 -123.27960062238847) bank103285 +103286 POINT(47.788674453668314 -121.70028563204055) bank103286 +103287 POINT(47.64456197361316 -121.69417061306906) bank103287 +103288 POINT(47.443675752904625 -121.58179619713881) bank103288 +103289 POINT(48.463519063012 -122.93772003577917) bank103289 +103290 POINT(47.83077328740494 -122.79444832979118) bank103290 +103291 POINT(47.791525275696756 -122.68300646591155) bank103291 +103292 POINT(48.53371008484311 -121.54615385111931) bank103292 +103293 POINT(46.88845837616306 -122.87028533014681) bank103293 +103294 POINT(46.81377913772127 -123.19958874495521) bank103294 +103295 POINT(47.43014376135997 -122.32583605803056) bank103295 +103296 POINT(47.85192254630749 -121.63314735895162) bank103296 +103297 POINT(46.871083154827744 -122.94338507005027) bank103297 +103298 POINT(48.21043498611574 -122.85511286706911) bank103298 +103299 POINT(47.58313737970457 -121.51004013330086) bank103299 +103300 POINT(47.171530260283696 -121.85419605727908) bank103300 +103301 POINT(48.3581190763638 -122.46404168346912) bank103301 +103302 POINT(47.99870278636275 -123.11341046940142) bank103302 +103303 POINT(47.164444614545985 -122.48231757846737) bank103303 +103304 POINT(48.39034932580727 -121.39824580621499) bank103304 +103305 POINT(47.76333092852321 -121.91834958539111) bank103305 +103306 POINT(47.044618192401245 -121.50297847671497) bank103306 +103307 POINT(46.771277523443665 -123.17749562097444) bank103307 +103308 POINT(47.5460008697467 -122.10134302729614) bank103308 +103309 POINT(48.435848236206034 -122.02895648395696) bank103309 +103310 POINT(47.405805609889676 -123.15115392430717) bank103310 +103311 POINT(47.52222564439973 -122.75524212429566) bank103311 +103312 POINT(46.82996198180387 -122.32994091544946) bank103312 +103313 POINT(48.21888420442416 -121.7102917116764) bank103313 +103314 POINT(48.06588421040668 -122.13543046720764) bank103314 +103315 POINT(47.965247288152824 -122.11104339470491) bank103315 +103316 POINT(47.24627424177403 -121.36529232348198) bank103316 +103317 POINT(47.462490564043286 -122.23431706294681) bank103317 +103318 POINT(47.51829769135811 -123.18935307438414) bank103318 +103319 POINT(47.20926352382086 -123.30597536150266) bank103319 +103320 POINT(47.328475915582985 -122.23669829497841) bank103320 +103321 POINT(48.1119581568369 -121.49652110443671) bank103321 +103322 POINT(47.16506952775409 -122.09028040035544) bank103322 +103323 POINT(47.67030494741133 -122.72481971025604) bank103323 +103324 POINT(46.617290592372655 -122.347654478312) bank103324 +103325 POINT(47.997394205617134 -123.31771276347493) bank103325 +103326 POINT(47.07129702752352 -121.76143988855569) bank103326 +103327 POINT(46.609526340227895 -121.75223321709765) bank103327 +103328 POINT(48.080163204467176 -122.38321746182984) bank103328 +103329 POINT(48.31158055478176 -122.20365408001008) bank103329 +103330 POINT(47.3878223966778 -123.29622567670557) bank103330 +103331 POINT(46.966531452048855 -123.05932458396327) bank103331 +103332 POINT(46.949777463267765 -121.66307710374015) bank103332 +103333 POINT(48.04718344325699 -121.92324585821508) bank103333 +103334 POINT(47.901946681258124 -122.08328002530638) bank103334 +103335 POINT(48.112712626319485 -121.80711602665114) bank103335 +103336 POINT(47.61437132189044 -122.54243582773049) bank103336 +103337 POINT(48.58581899709989 -122.97320890828567) bank103337 +103338 POINT(48.404519468627214 -122.51177021371814) bank103338 +103339 POINT(48.1344238116767 -122.90569136870499) bank103339 +103340 POINT(48.05806919296258 -122.0455915063926) bank103340 +103341 POINT(48.33150977424977 -121.59305398328193) bank103341 +103342 POINT(46.727125764985765 -121.47499366846982) bank103342 +103343 POINT(48.06953865867208 -122.33695007214001) bank103343 +103344 POINT(47.41712388710341 -123.1720797994462) bank103344 +103345 POINT(47.42472031001922 -122.95074130975864) bank103345 +103346 POINT(47.067977756021 -121.9429407793732) bank103346 +103347 POINT(47.166076582087015 -121.62197288409398) bank103347 +103348 POINT(48.48226766241772 -122.2893650516363) bank103348 +103349 POINT(48.59400732886817 -122.27221339318544) bank103349 +103350 POINT(46.97812901979428 -122.06142075314438) bank103350 +103351 POINT(47.977672491354326 -121.6145458989668) bank103351 +103352 POINT(47.03481605619056 -122.92173797647447) bank103352 +103353 POINT(47.46526540688098 -121.79921562537623) bank103353 +103354 POINT(47.421310122019946 -122.1814648829767) bank103354 +103355 POINT(48.120651914318486 -122.85699127747272) bank103355 +103356 POINT(47.62005576448181 -121.85820059877324) bank103356 +103357 POINT(48.2138565795844 -123.058666115145) bank103357 +103358 POINT(46.91429569112611 -121.70170299652797) bank103358 +103359 POINT(47.653344151945575 -122.92148636377371) bank103359 +103360 POINT(47.230155324295225 -123.29443725965044) bank103360 +103361 POINT(47.39616298239557 -122.0180424619391) bank103361 +103362 POINT(47.00363377814755 -122.7053006569218) bank103362 +103363 POINT(47.4889185424755 -122.38176368952767) bank103363 +103364 POINT(47.88900288459679 -122.73951270345007) bank103364 +103365 POINT(46.86571406118637 -121.66625979052436) bank103365 +103366 POINT(47.78585820470791 -122.20796148922517) bank103366 +103367 POINT(48.316315886696344 -122.30175878024247) bank103367 +103368 POINT(48.459346312182916 -122.54217159282695) bank103368 +103369 POINT(48.0395385649365 -122.78517658542857) bank103369 +103370 POINT(47.705077118882066 -121.43499880666715) bank103370 +103371 POINT(48.027611490489754 -121.69615371677828) bank103371 +103372 POINT(47.061383469188534 -121.35163098167529) bank103372 +103373 POINT(48.58628120768567 -121.98619809492558) bank103373 +103374 POINT(48.39993599529111 -123.32655155952139) bank103374 +103375 POINT(47.48455229866861 -122.56973224673388) bank103375 +103376 POINT(47.86686898467803 -122.27576409429842) bank103376 +103377 POINT(46.98448060831487 -122.19220491846141) bank103377 +103378 POINT(47.09913630852297 -121.85001027001937) bank103378 +103379 POINT(46.7124461084207 -122.89444025141378) bank103379 +103380 POINT(48.41163836928582 -123.03103654453477) bank103380 +103381 POINT(47.271794674960454 -122.68640762652818) bank103381 +103382 POINT(48.4923656877377 -121.6777484225077) bank103382 +103383 POINT(46.83851797316488 -122.85962383899046) bank103383 +103384 POINT(48.41983985281809 -121.67702461947503) bank103384 +103385 POINT(47.35763460834627 -121.35484846346088) bank103385 +103386 POINT(48.32929831800443 -123.11066410164116) bank103386 +103387 POINT(47.17475181186976 -121.35905594852747) bank103387 +103388 POINT(46.908964099023365 -121.50972210569955) bank103388 +103389 POINT(46.919280451026424 -122.37727935644185) bank103389 +103390 POINT(46.909276913048366 -122.1406173440502) bank103390 +103391 POINT(46.87931833733592 -121.34287916392697) bank103391 +103392 POINT(46.682181883199185 -123.16260254076919) bank103392 +103393 POINT(47.207641690317224 -122.06187884197871) bank103393 +103394 POINT(47.71329282116726 -121.946267614433) bank103394 +103395 POINT(47.07848080647778 -121.75566044944813) bank103395 +103396 POINT(47.2620630559746 -123.02219366873405) bank103396 +103397 POINT(47.61661636300073 -123.14065430042825) bank103397 +103398 POINT(47.90087278170662 -121.58631815729827) bank103398 +103399 POINT(47.02134188973183 -122.85453558279548) bank103399 +103400 POINT(48.498140003624975 -123.04907648613788) bank103400 +103401 POINT(47.44384780817736 -121.70434892270994) bank103401 +103402 POINT(47.09045744725407 -122.40888519785852) bank103402 +103403 POINT(47.86282442956079 -122.37049464333998) bank103403 +103404 POINT(47.20365032673173 -121.59448694354329) bank103404 +103405 POINT(46.85289191048372 -122.25836520317478) bank103405 +103406 POINT(47.936192444162685 -121.81999660308263) bank103406 +103407 POINT(47.15940572912032 -122.58583776228886) bank103407 +103408 POINT(47.77333491659773 -121.62173138693363) bank103408 +103409 POINT(48.56769073230434 -122.18283369205932) bank103409 +103410 POINT(47.43028789664953 -122.90510164818548) bank103410 +103411 POINT(48.061467473063 -121.9111616573242) bank103411 +103412 POINT(47.304906974468615 -122.44187720087544) bank103412 +103413 POINT(46.99303580834761 -122.82777435564287) bank103413 +103414 POINT(46.98638687245496 -122.79372417142895) bank103414 +103415 POINT(47.1013272414498 -121.7223622085531) bank103415 +103416 POINT(47.2350569410608 -122.92007099616069) bank103416 +103417 POINT(47.04038784811017 -121.5493455169078) bank103417 +103418 POINT(46.632738823476956 -123.22735543466885) bank103418 +103419 POINT(47.409915302718645 -122.16967378795131) bank103419 +103420 POINT(48.07159077905011 -122.86136569931203) bank103420 +103421 POINT(46.903285366047655 -122.37109829431866) bank103421 +103422 POINT(47.047332551180354 -121.66584427723186) bank103422 +103423 POINT(46.658180501068856 -121.48973076865029) bank103423 +103424 POINT(46.82480606954676 -121.70620490550047) bank103424 +103425 POINT(47.0367355277007 -123.10660085244156) bank103425 +103426 POINT(46.818844771331804 -122.52965891880322) bank103426 +103427 POINT(47.5558750921793 -122.36929131188704) bank103427 +103428 POINT(46.761821619057365 -121.64542441842546) bank103428 +103429 POINT(48.2373142879735 -121.5691062891131) bank103429 +103430 POINT(47.63161699176116 -123.2774924290929) bank103430 +103431 POINT(47.7519124944985 -121.39629576172574) bank103431 +103432 POINT(48.2954160115303 -122.01352808414595) bank103432 +103433 POINT(48.48426806736067 -121.81954414080718) bank103433 +103434 POINT(47.56881582981251 -121.72331050403683) bank103434 +103435 POINT(48.23517961814109 -122.23776963971196) bank103435 +103436 POINT(47.62283956316011 -123.15065277473605) bank103436 +103437 POINT(48.403000450643184 -121.43299011647188) bank103437 +103438 POINT(47.404272029289494 -122.8770694251518) bank103438 +103439 POINT(47.85106516818189 -123.23744207864421) bank103439 +103440 POINT(46.61631708058291 -123.21590808499283) bank103440 +103441 POINT(46.785045459977276 -123.20225127326435) bank103441 +103442 POINT(47.12984857507398 -121.99630939674796) bank103442 +103443 POINT(48.06942481647697 -121.89525649658152) bank103443 +103444 POINT(48.24452633803412 -123.1993119383829) bank103444 +103445 POINT(46.92006609504281 -121.64316231798267) bank103445 +103446 POINT(46.743281863198156 -122.3038341408092) bank103446 +103447 POINT(47.515653862487525 -122.22676709277722) bank103447 +103448 POINT(46.89682680894009 -122.75350377111894) bank103448 +103449 POINT(48.416218347471975 -121.55605833667929) bank103449 +103450 POINT(47.922560256597485 -121.8966534642001) bank103450 +103451 POINT(46.699392272012 -122.14969709505378) bank103451 +103452 POINT(48.266842662527175 -122.84187926020088) bank103452 +103453 POINT(48.32271866850267 -121.74701884512628) bank103453 +103454 POINT(47.66180647777503 -122.72316333206211) bank103454 +103455 POINT(48.35370829454679 -122.6757249299246) bank103455 +103456 POINT(46.84728006452804 -121.87745831305463) bank103456 +103457 POINT(47.55792008450396 -122.75189948322817) bank103457 +103458 POINT(47.23374546882594 -122.23915082316353) bank103458 +103459 POINT(47.450019036431065 -122.30041773923132) bank103459 +103460 POINT(48.30428709864216 -121.63904551040052) bank103460 +103461 POINT(46.72928241903644 -121.74140429556454) bank103461 +103462 POINT(48.2847747707152 -122.62875772187174) bank103462 +103463 POINT(47.3628261595055 -121.58996407385332) bank103463 +103464 POINT(47.5562328996637 -121.90510844635479) bank103464 +103465 POINT(48.38081454987673 -121.39591715411585) bank103465 +103466 POINT(47.501648690284696 -122.1006366906369) bank103466 +103467 POINT(47.98827016891096 -122.01347637014071) bank103467 +103468 POINT(47.06787498553412 -122.39051108795469) bank103468 +103469 POINT(47.89162167367399 -122.16603559850753) bank103469 +103470 POINT(48.58621573091819 -123.06368409598366) bank103470 +103471 POINT(46.927892864101864 -122.98921939459585) bank103471 +103472 POINT(46.77198387391536 -123.00642795561531) bank103472 +103473 POINT(48.29030559348397 -122.22049091827539) bank103473 +103474 POINT(48.25374634597139 -122.30652543561759) bank103474 +103475 POINT(47.560666717997556 -123.02436299137064) bank103475 +103476 POINT(47.78243800837722 -121.96333429612748) bank103476 +103477 POINT(47.49270425982648 -122.4723008023443) bank103477 +103478 POINT(46.806763493974834 -122.38394370792635) bank103478 +103479 POINT(48.4874398057401 -122.01529843605107) bank103479 +103480 POINT(46.81498309019073 -123.17774510277208) bank103480 +103481 POINT(48.16309419735182 -122.81634075862449) bank103481 +103482 POINT(46.65313150352478 -122.94831370576931) bank103482 +103483 POINT(47.290116792014004 -121.9693946973646) bank103483 +103484 POINT(47.65319508016673 -122.00469733674328) bank103484 +103485 POINT(48.459522030330376 -122.05996239083632) bank103485 +103486 POINT(46.87961471537557 -121.86201379648179) bank103486 +103487 POINT(47.383477834694254 -123.2539886117106) bank103487 +103488 POINT(47.32829744685382 -122.37027437169763) bank103488 +103489 POINT(47.89080545092718 -122.99889007991464) bank103489 +103490 POINT(46.64070849834913 -123.28649382053335) bank103490 +103491 POINT(47.46214390238264 -121.53615240749996) bank103491 +103492 POINT(46.64764379450758 -122.6971027628191) bank103492 +103493 POINT(47.23562152261714 -122.31028826675757) bank103493 +103494 POINT(47.610466792219555 -122.15702780404852) bank103494 +103495 POINT(46.91053940189511 -122.98262233211747) bank103495 +103496 POINT(47.72682135025186 -122.90762701071786) bank103496 +103497 POINT(47.01581536710756 -121.51017310833923) bank103497 +103498 POINT(47.68950366627346 -123.00047682372845) bank103498 +103499 POINT(46.64129565712055 -122.45595829727955) bank103499 +103500 POINT(47.95582969492131 -122.77918036105748) bank103500 +103501 POINT(47.053734198361816 -122.26762163612733) bank103501 +103502 POINT(48.25130101890047 -122.12106271337656) bank103502 +103503 POINT(48.36738578057763 -122.16754142322843) bank103503 +103504 POINT(47.70270933730309 -122.04228088006936) bank103504 +103505 POINT(47.62675230334796 -122.77748411486441) bank103505 +103506 POINT(48.50873774929722 -123.261230110234) bank103506 +103507 POINT(48.42581452529277 -121.70389135418425) bank103507 +103508 POINT(47.433783773394275 -122.99414241159779) bank103508 +103509 POINT(47.76764469829234 -121.64039223484161) bank103509 +103510 POINT(48.5834608781411 -123.21407432313868) bank103510 +103511 POINT(46.81827405350245 -121.51221289616907) bank103511 +103512 POINT(47.28920740601238 -122.15150654496878) bank103512 +103513 POINT(48.105315716895205 -122.42824609246769) bank103513 +103514 POINT(46.78511015739268 -122.97776656972468) bank103514 +103515 POINT(47.130394105729295 -122.6461813356708) bank103515 +103516 POINT(46.62054962856621 -121.37452294313572) bank103516 +103517 POINT(47.46281334103677 -121.68116077662442) bank103517 +103518 POINT(47.86731481340646 -122.90546015068708) bank103518 +103519 POINT(48.39843541230626 -121.42440888912698) bank103519 +103520 POINT(47.433704687185056 -122.27914917407873) bank103520 +103521 POINT(46.867431761123505 -122.92374948469205) bank103521 +103522 POINT(47.33169651687672 -122.32222748647125) bank103522 +103523 POINT(47.99574631987568 -122.65426319469658) bank103523 +103524 POINT(48.09884258300232 -123.01861798510521) bank103524 +103525 POINT(48.58867451457763 -121.95443535407915) bank103525 +103526 POINT(46.63625873893891 -122.49066683272943) bank103526 +103527 POINT(48.035062209249894 -122.0795445423825) bank103527 +103528 POINT(47.788278447365535 -123.07322761532666) bank103528 +103529 POINT(47.3846829150717 -123.14393183121857) bank103529 +103530 POINT(48.488248842881504 -122.3250185034565) bank103530 +103531 POINT(48.438513221996054 -121.96281158166994) bank103531 +103532 POINT(48.337512254922586 -122.69041936379763) bank103532 +103533 POINT(46.99513247125791 -122.98259979859289) bank103533 +103534 POINT(48.41600535714223 -122.57314051744419) bank103534 +103535 POINT(48.211028701519496 -123.13086609358064) bank103535 +103536 POINT(47.09544056688985 -122.86818226863366) bank103536 +103537 POINT(48.28994255041614 -123.21371476139026) bank103537 +103538 POINT(48.27234316227053 -121.34873397531065) bank103538 +103539 POINT(48.56325299895713 -121.94819622081476) bank103539 +103540 POINT(48.38128066693393 -121.56849235620575) bank103540 +103541 POINT(47.31414381146525 -122.66202723213175) bank103541 +103542 POINT(47.5974285764601 -122.7417761711809) bank103542 +103543 POINT(47.002346118499304 -122.60538269395654) bank103543 +103544 POINT(48.15766144851511 -123.11928924385685) bank103544 +103545 POINT(48.295241188808696 -123.08632223218204) bank103545 +103546 POINT(48.57054912527484 -122.12315627205629) bank103546 +103547 POINT(46.84307495181394 -121.8913299185586) bank103547 +103548 POINT(47.64811712976024 -122.43471220465341) bank103548 +103549 POINT(48.265351990682944 -121.336224205013) bank103549 +103550 POINT(48.37783755187412 -122.12686979695148) bank103550 +103551 POINT(47.801341164713094 -121.75477544675637) bank103551 +103552 POINT(47.22425199344023 -122.24569829255138) bank103552 +103553 POINT(48.153892655904826 -122.35241716101527) bank103553 +103554 POINT(46.663329174248496 -122.55353641106808) bank103554 +103555 POINT(46.73920526681466 -121.37953514555824) bank103555 +103556 POINT(47.21994441440795 -123.12405593862208) bank103556 +103557 POINT(47.755722896016096 -122.0020023146898) bank103557 +103558 POINT(47.371689794552644 -121.8518318356603) bank103558 +103559 POINT(48.052230263850014 -121.8876990814617) bank103559 +103560 POINT(47.023149099999735 -122.03190628109547) bank103560 +103561 POINT(48.05023073133299 -122.97913545466079) bank103561 +103562 POINT(47.34232512196564 -122.12579826122948) bank103562 +103563 POINT(47.66277300068522 -122.5304354211424) bank103563 +103564 POINT(48.00582433875236 -123.08541802424931) bank103564 +103565 POINT(47.99567696352702 -122.72779235555265) bank103565 +103566 POINT(46.90397287969768 -121.70532170927014) bank103566 +103567 POINT(47.60802386256377 -122.86587586929232) bank103567 +103568 POINT(48.44933339140233 -121.79430757163713) bank103568 +103569 POINT(46.694191044328996 -122.28694686398734) bank103569 +103570 POINT(47.09386816024368 -121.76061657803524) bank103570 +103571 POINT(47.20377300499816 -121.96948135010248) bank103571 +103572 POINT(47.120915447624725 -122.06286911699482) bank103572 +103573 POINT(47.240058138432275 -123.09047035256064) bank103573 +103574 POINT(48.14368174571301 -122.45806488119051) bank103574 +103575 POINT(48.2360023629991 -122.52063203691216) bank103575 +103576 POINT(48.085589329829794 -121.48388761251204) bank103576 +103577 POINT(47.13514024622238 -121.95403528812685) bank103577 +103578 POINT(47.78808467281737 -121.52097121954247) bank103578 +103579 POINT(48.20480773297131 -122.35637028563622) bank103579 +103580 POINT(48.586325111147175 -122.12194123404693) bank103580 +103581 POINT(46.714186839063515 -121.5320220136406) bank103581 +103582 POINT(47.600061651282154 -122.12934865903229) bank103582 +103583 POINT(47.172461046110804 -121.8682663294165) bank103583 +103584 POINT(46.64003940818906 -123.14840591298756) bank103584 +103585 POINT(47.23846574855008 -121.60749857726617) bank103585 +103586 POINT(46.79895005782414 -123.13124002419823) bank103586 +103587 POINT(47.06467349075634 -122.5445820557851) bank103587 +103588 POINT(47.196839547924746 -122.28684798427423) bank103588 +103589 POINT(46.76134154220914 -122.78466299409656) bank103589 +103590 POINT(46.722273997263926 -123.02664950753581) bank103590 +103591 POINT(47.38676843210049 -122.03152203896751) bank103591 +103592 POINT(47.6656365419915 -121.40863216032807) bank103592 +103593 POINT(47.15361373444415 -122.98640132727708) bank103593 +103594 POINT(47.633161475632015 -121.73117892120776) bank103594 +103595 POINT(47.21281382521419 -121.491559578159) bank103595 +103596 POINT(46.64668151555709 -122.06435238766632) bank103596 +103597 POINT(46.71090893667943 -122.5348399661627) bank103597 +103598 POINT(47.762277866087544 -121.63227215281995) bank103598 +103599 POINT(47.221728415245494 -122.45295360628428) bank103599 +103600 POINT(48.01375844734764 -123.01188098327525) bank103600 +103601 POINT(47.51439127119859 -121.8081515238569) bank103601 +103602 POINT(46.862447637813155 -122.46393899105485) bank103602 +103603 POINT(47.310623652576886 -122.19543266686343) bank103603 +103604 POINT(48.246084226704575 -121.81056430161583) bank103604 +103605 POINT(46.77780955482167 -121.72871956977855) bank103605 +103606 POINT(47.55898939105608 -121.53983434687306) bank103606 +103607 POINT(48.08624851283374 -121.58626678510332) bank103607 +103608 POINT(48.25984957837048 -122.70109066068258) bank103608 +103609 POINT(48.03937773283072 -122.72995548576004) bank103609 +103610 POINT(48.13064173314986 -122.64703147138745) bank103610 +103611 POINT(47.9541141174576 -123.31654470072264) bank103611 +103612 POINT(48.03750936229149 -122.71421869598541) bank103612 +103613 POINT(48.034511639761384 -123.32005620232732) bank103613 +103614 POINT(48.35580230656012 -122.82444097823694) bank103614 +103615 POINT(48.00601562440066 -122.11577953364561) bank103615 +103616 POINT(47.68315707574016 -121.82214548524234) bank103616 +103617 POINT(48.05832531308169 -122.36999156300763) bank103617 +103618 POINT(47.93395818163343 -121.57478600750045) bank103618 +103619 POINT(46.99557210122367 -121.98891861291865) bank103619 +103620 POINT(48.4338940014713 -122.70394976683029) bank103620 +103621 POINT(48.299878415702516 -122.30270448479061) bank103621 +103622 POINT(47.989648548310726 -122.96072961083313) bank103622 +103623 POINT(48.58214289145812 -122.63008759510618) bank103623 +103624 POINT(47.695790297364304 -122.54531977581901) bank103624 +103625 POINT(47.5310171806463 -122.40082062432535) bank103625 +103626 POINT(48.535443666046795 -123.01023211723229) bank103626 +103627 POINT(48.46095704505933 -123.14802600842953) bank103627 +103628 POINT(47.98879556386746 -121.48474787124384) bank103628 +103629 POINT(47.55712008066602 -121.65492334839533) bank103629 +103630 POINT(47.52152756426648 -122.70030197205149) bank103630 +103631 POINT(47.11947110192946 -122.59307190828731) bank103631 +103632 POINT(47.56932703295475 -122.87537536648433) bank103632 +103633 POINT(47.8468748164977 -123.07528304149727) bank103633 +103634 POINT(47.948802762558216 -122.6179366260358) bank103634 +103635 POINT(47.89665531734107 -123.0997285245182) bank103635 +103636 POINT(48.37039387099367 -122.73967055643817) bank103636 +103637 POINT(46.846032945711606 -123.06816020848446) bank103637 +103638 POINT(47.547892502003386 -121.72147783885497) bank103638 +103639 POINT(48.54977428736316 -122.11841192777884) bank103639 +103640 POINT(47.75545607640675 -122.10766405781791) bank103640 +103641 POINT(48.112883619371274 -121.52398429716392) bank103641 +103642 POINT(48.19183621163151 -122.05998399949527) bank103642 +103643 POINT(47.34495147813131 -121.86912045450745) bank103643 +103644 POINT(47.99708894763725 -121.57222900589512) bank103644 +103645 POINT(47.28731056593513 -122.90773660285053) bank103645 +103646 POINT(47.132675743488385 -121.8871103286938) bank103646 +103647 POINT(47.01967793353377 -121.68846561492207) bank103647 +103648 POINT(47.23183779157347 -122.40687018278521) bank103648 +103649 POINT(47.51570776702738 -122.67130664598433) bank103649 +103650 POINT(48.392810624158166 -121.43143125919315) bank103650 +103651 POINT(48.525462318628584 -122.01440181248167) bank103651 +103652 POINT(47.89913021435712 -123.09926005185561) bank103652 +103653 POINT(47.40415610537823 -122.57725580598007) bank103653 +103654 POINT(48.559043385098235 -122.3285322660543) bank103654 +103655 POINT(46.77879626571314 -122.67544489884514) bank103655 +103656 POINT(48.42597580923247 -122.1617490386393) bank103656 +103657 POINT(47.40538142902119 -121.53896553412467) bank103657 +103658 POINT(47.14286204322749 -121.9227820801726) bank103658 +103659 POINT(48.07547719960718 -123.25913106407202) bank103659 +103660 POINT(47.28624809219075 -122.76448149648564) bank103660 +103661 POINT(46.61772977866164 -123.00835533822597) bank103661 +103662 POINT(47.77917637985126 -121.55259485509826) bank103662 +103663 POINT(46.982577017834195 -122.80876643196328) bank103663 +103664 POINT(47.32761083746619 -123.19873225758056) bank103664 +103665 POINT(48.5632769858386 -123.08115476746279) bank103665 +103666 POINT(48.12663721380354 -123.14478238121872) bank103666 +103667 POINT(48.52065379786821 -121.72648752568463) bank103667 +103668 POINT(47.16531572241337 -122.89719454836874) bank103668 +103669 POINT(47.71622356270589 -123.08081712578245) bank103669 +103670 POINT(47.55387872624117 -122.8585574748531) bank103670 +103671 POINT(47.14725278978108 -122.43213100425349) bank103671 +103672 POINT(48.19686722097386 -123.09253560182012) bank103672 +103673 POINT(48.45564877462607 -121.66964007566027) bank103673 +103674 POINT(48.1315916035058 -122.51167670626211) bank103674 +103675 POINT(47.64244603691234 -121.53447496711566) bank103675 +103676 POINT(48.30187811558142 -123.3189742939159) bank103676 +103677 POINT(47.248464226753974 -122.20713330412032) bank103677 +103678 POINT(46.83446059569954 -122.94938286959423) bank103678 +103679 POINT(47.990402470428066 -122.52345797024824) bank103679 +103680 POINT(47.91009527302147 -122.07795743261325) bank103680 +103681 POINT(47.07718813380019 -123.11224681468448) bank103681 +103682 POINT(47.27045427762258 -122.59577332384956) bank103682 +103683 POINT(47.747907617763836 -121.93220500654739) bank103683 +103684 POINT(48.510497517922055 -122.8061888468056) bank103684 +103685 POINT(47.877735636962534 -122.83553598013516) bank103685 +103686 POINT(47.37330202828597 -121.58592646957483) bank103686 +103687 POINT(47.94490085761274 -121.95987342238568) bank103687 +103688 POINT(47.24840265014079 -121.90123486521172) bank103688 +103689 POINT(47.2415523981373 -122.93701286408034) bank103689 +103690 POINT(46.91248344898656 -122.45641822185401) bank103690 +103691 POINT(47.597717458026544 -122.23609050720722) bank103691 +103692 POINT(47.26068605223579 -122.55439483190744) bank103692 +103693 POINT(48.199197001977645 -122.20041448519747) bank103693 +103694 POINT(47.07810784350802 -122.52996215762039) bank103694 +103695 POINT(46.905840058703966 -122.06281646012552) bank103695 +103696 POINT(47.66809473075997 -121.96186682092716) bank103696 +103697 POINT(47.15868860034563 -123.151678214065) bank103697 +103698 POINT(48.38567783556971 -122.37618917673763) bank103698 +103699 POINT(48.12760851759411 -121.70592919243609) bank103699 +103700 POINT(47.07613211569076 -122.10656298858761) bank103700 +103701 POINT(48.31163322937023 -122.43392081735007) bank103701 +103702 POINT(47.69672689838375 -121.96341909338054) bank103702 +103703 POINT(47.367012221489084 -123.20505905074974) bank103703 +103704 POINT(47.62942302213553 -122.47898965750714) bank103704 +103705 POINT(46.766380936653526 -122.87481799190557) bank103705 +103706 POINT(48.01585311644191 -122.4926802890136) bank103706 +103707 POINT(47.657166877189184 -121.79033267137444) bank103707 +103708 POINT(47.37121586238965 -122.2051505922378) bank103708 +103709 POINT(47.735399504833815 -122.0886082132212) bank103709 +103710 POINT(46.72980420875333 -122.38398653905212) bank103710 +103711 POINT(47.371486919116165 -122.69025757304156) bank103711 +103712 POINT(48.28059110246436 -122.16190745155684) bank103712 +103713 POINT(46.95284174190591 -121.65553608822002) bank103713 +103714 POINT(48.498714273317354 -121.36646443936921) bank103714 +103715 POINT(48.04460229677583 -122.7974339417743) bank103715 +103716 POINT(48.101129277540984 -123.03852945559476) bank103716 +103717 POINT(46.668246286596876 -122.79134828808125) bank103717 +103718 POINT(48.5747373881752 -121.34071693404422) bank103718 +103719 POINT(47.788083949389055 -121.62524814648596) bank103719 +103720 POINT(47.315004433293964 -122.33141609963468) bank103720 +103721 POINT(47.79736190584919 -123.14700208983542) bank103721 +103722 POINT(46.83183493994552 -123.134188266787) bank103722 +103723 POINT(47.596450923643324 -122.96105707986179) bank103723 +103724 POINT(47.847274177742946 -121.85405713910471) bank103724 +103725 POINT(47.019218517389874 -121.5178650176736) bank103725 +103726 POINT(47.4807180912485 -122.67674828237332) bank103726 +103727 POINT(47.740061632897365 -122.54371573665905) bank103727 +103728 POINT(48.0321961556645 -121.47454910799236) bank103728 +103729 POINT(48.33125842084496 -121.79945121383007) bank103729 +103730 POINT(48.44403536885647 -121.43494532730062) bank103730 +103731 POINT(46.769194516124415 -122.09247395020103) bank103731 +103732 POINT(47.77178757916795 -122.88894545294801) bank103732 +103733 POINT(47.22193515896181 -122.97540936181035) bank103733 +103734 POINT(48.00299609913953 -121.70065317014621) bank103734 +103735 POINT(48.226395300267875 -121.98201742895355) bank103735 +103736 POINT(46.94836356348178 -122.52044718735905) bank103736 +103737 POINT(47.03147266544054 -122.25221076684119) bank103737 +103738 POINT(47.02541050685428 -122.48440300138758) bank103738 +103739 POINT(46.981237748287576 -121.95530969256123) bank103739 +103740 POINT(48.42617553811646 -122.096698235662) bank103740 +103741 POINT(47.849711239021744 -122.74188395981514) bank103741 +103742 POINT(47.53597702035682 -122.81328272855805) bank103742 +103743 POINT(47.917255316573 -122.1843537181791) bank103743 +103744 POINT(48.10837036694168 -122.42952901369736) bank103744 +103745 POINT(47.16680782518901 -122.1562884318477) bank103745 +103746 POINT(47.77071944282338 -121.66419537717645) bank103746 +103747 POINT(48.02294964195334 -121.98870298188888) bank103747 +103748 POINT(47.21005432545463 -122.25200591309081) bank103748 +103749 POINT(47.73681906640623 -121.9500495417806) bank103749 +103750 POINT(47.315550016426364 -122.557618148612) bank103750 +103751 POINT(46.81687677679432 -123.0772839762983) bank103751 +103752 POINT(48.27308806222385 -122.78039829391662) bank103752 +103753 POINT(48.54847920343462 -123.00748941472642) bank103753 +103754 POINT(47.08380438831378 -122.45373383854943) bank103754 +103755 POINT(48.29528252380099 -122.90804427339489) bank103755 +103756 POINT(47.44706633564396 -122.8519820762043) bank103756 +103757 POINT(47.704015086355646 -122.42941509233844) bank103757 +103758 POINT(48.468312867236435 -121.54913711577181) bank103758 +103759 POINT(47.07231717779345 -121.39328934091772) bank103759 +103760 POINT(47.69701992288881 -121.7154975413235) bank103760 +103761 POINT(47.872476272574154 -122.97098156196722) bank103761 +103762 POINT(48.604427027907526 -123.0634418731837) bank103762 +103763 POINT(48.53271818783889 -122.34492780001376) bank103763 +103764 POINT(46.647003736287296 -122.01899687605578) bank103764 +103765 POINT(48.31148033499114 -122.42589986678959) bank103765 +103766 POINT(47.89506135743333 -122.39003238736433) bank103766 +103767 POINT(47.4847135592979 -123.26892602191428) bank103767 +103768 POINT(48.05710041280156 -122.21625536217562) bank103768 +103769 POINT(47.52141511203844 -122.86979388415011) bank103769 +103770 POINT(47.931293879044254 -122.89147078002117) bank103770 +103771 POINT(46.72318178622863 -122.35807915825062) bank103771 +103772 POINT(47.27540187594789 -121.62825599244935) bank103772 +103773 POINT(48.154280533514466 -122.98175424733168) bank103773 +103774 POINT(46.898983716371205 -121.36180388375682) bank103774 +103775 POINT(47.452611865114356 -121.4046132375858) bank103775 +103776 POINT(47.62073833044831 -123.1821915171855) bank103776 +103777 POINT(47.397229316932346 -121.84140560610318) bank103777 +103778 POINT(47.698461354346094 -122.14998541168748) bank103778 +103779 POINT(46.98618306430953 -122.60851056546304) bank103779 +103780 POINT(47.43399324677987 -121.8455980564517) bank103780 +103781 POINT(46.73796586062067 -122.54571541234435) bank103781 +103782 POINT(47.359043335346136 -121.491238297265) bank103782 +103783 POINT(48.16467374367953 -121.68996023687043) bank103783 +103784 POINT(47.32923704608304 -122.19115756730803) bank103784 +103785 POINT(47.21099431797425 -121.8430021302288) bank103785 +103786 POINT(46.75851360384683 -122.29023520744478) bank103786 +103787 POINT(46.76129712227951 -121.80827644499827) bank103787 +103788 POINT(48.36250622211191 -121.69949186936408) bank103788 +103789 POINT(46.97126097735445 -121.39348222757098) bank103789 +103790 POINT(47.322244052252564 -121.52730347108204) bank103790 +103791 POINT(47.370562894438415 -122.52912288200027) bank103791 +103792 POINT(48.16556670305617 -122.36455814622146) bank103792 +103793 POINT(47.31868240303177 -123.22768377078008) bank103793 +103794 POINT(48.21362419980132 -122.47943463443028) bank103794 +103795 POINT(46.68681930497181 -121.51191476610573) bank103795 +103796 POINT(48.59840493661385 -122.9928833934246) bank103796 +103797 POINT(46.94718746090938 -122.36899944247537) bank103797 +103798 POINT(46.86836661821825 -122.38745274590947) bank103798 +103799 POINT(46.74047994031426 -123.14216495778096) bank103799 +103800 POINT(47.44469057675857 -122.25952190648357) bank103800 +103801 POINT(48.3356615367427 -122.58060456002926) bank103801 +103802 POINT(47.914840400973475 -123.03390933705299) bank103802 +103803 POINT(48.52940779918615 -122.36202904546082) bank103803 +103804 POINT(48.49372903683389 -123.23793283592269) bank103804 +103805 POINT(46.96396863731456 -122.72391933221691) bank103805 +103806 POINT(47.327625744383795 -121.72605255254342) bank103806 +103807 POINT(47.404304287756695 -122.72646841218605) bank103807 +103808 POINT(46.85687655011979 -121.54390918200941) bank103808 +103809 POINT(46.9769643112403 -121.70913502248298) bank103809 +103810 POINT(47.10729232397596 -121.87753751088825) bank103810 +103811 POINT(47.31809140282203 -122.00364582863881) bank103811 +103812 POINT(46.68627228959092 -122.27234801206653) bank103812 +103813 POINT(46.95670337685503 -122.59345697096869) bank103813 +103814 POINT(46.90830245236037 -123.20833965032328) bank103814 +103815 POINT(47.71570109014651 -122.34201658652103) bank103815 +103816 POINT(47.280592379093534 -121.98181359616927) bank103816 +103817 POINT(48.158427429887034 -122.93318677230057) bank103817 +103818 POINT(47.4864325551232 -122.1268946631698) bank103818 +103819 POINT(46.80836146085702 -122.20344451718793) bank103819 +103820 POINT(47.38700314746695 -123.11065257015335) bank103820 +103821 POINT(48.166360379895465 -121.7767559263322) bank103821 +103822 POINT(46.90662634428467 -123.03794299227256) bank103822 +103823 POINT(48.05322691553267 -121.50016543233201) bank103823 +103824 POINT(47.47807825346259 -122.42359353941555) bank103824 +103825 POINT(47.69621986916305 -121.512655839209) bank103825 +103826 POINT(48.48973498037354 -122.46590721225894) bank103826 +103827 POINT(48.43823598540241 -121.56559228724794) bank103827 +103828 POINT(47.663583798014265 -121.95327836769913) bank103828 +103829 POINT(47.61324119027165 -123.14247250391763) bank103829 +103830 POINT(47.16004681072076 -122.47320978678857) bank103830 +103831 POINT(47.36437986224251 -121.40443610708812) bank103831 +103832 POINT(48.165754189935626 -121.35323687122316) bank103832 +103833 POINT(46.84490413332703 -122.57892583129474) bank103833 +103834 POINT(48.2208018785921 -123.21782023332577) bank103834 +103835 POINT(47.01010209125715 -123.31141885712746) bank103835 +103836 POINT(47.717928213303594 -121.8760590170918) bank103836 +103837 POINT(48.36749540941174 -122.45092470300813) bank103837 +103838 POINT(48.52201098149154 -123.29657143855013) bank103838 +103839 POINT(48.00738932706299 -122.39737054922767) bank103839 +103840 POINT(48.153861257405204 -122.20972068284989) bank103840 +103841 POINT(46.6378618339904 -122.81690105479419) bank103841 +103842 POINT(47.261032277056465 -121.45585921856949) bank103842 +103843 POINT(46.673699130660765 -122.4741371134855) bank103843 +103844 POINT(47.48690343022574 -123.20180881578418) bank103844 +103845 POINT(46.67357938059135 -121.39845132299571) bank103845 +103846 POINT(47.42671516259748 -122.22885740800578) bank103846 +103847 POINT(47.7051815835941 -121.38675217582657) bank103847 +103848 POINT(47.7609706708462 -122.05603085647432) bank103848 +103849 POINT(47.876306198931395 -121.61626754935911) bank103849 +103850 POINT(48.32712973675392 -122.29237847590018) bank103850 +103851 POINT(48.02991011396012 -121.96743226351609) bank103851 +103852 POINT(47.3362102179584 -123.31132991553747) bank103852 +103853 POINT(47.6138529844361 -122.83033904753373) bank103853 +103854 POINT(47.274074273730825 -122.59813118969255) bank103854 +103855 POINT(46.82427578050161 -122.74527160906074) bank103855 +103856 POINT(48.21844766864527 -121.88932655940589) bank103856 +103857 POINT(47.83539925180925 -121.52491954326969) bank103857 +103858 POINT(47.94284410465859 -122.897305174935) bank103858 +103859 POINT(47.59961690079095 -121.37430471888152) bank103859 +103860 POINT(47.72292440806325 -122.358272285159) bank103860 +103861 POINT(47.97148948677523 -122.1501325045615) bank103861 +103862 POINT(47.5731191029066 -122.29238239283184) bank103862 +103863 POINT(47.637507460866566 -122.67198486689993) bank103863 +103864 POINT(48.0011927805378 -122.4866776776151) bank103864 +103865 POINT(46.9622352967715 -122.63447628112381) bank103865 +103866 POINT(47.0677416790608 -122.40716639579715) bank103866 +103867 POINT(47.23416700654488 -122.73726218054404) bank103867 +103868 POINT(46.947648319490376 -122.43066194338945) bank103868 +103869 POINT(48.5071991629973 -121.86958023733474) bank103869 +103870 POINT(46.87232117260966 -123.31904156230586) bank103870 +103871 POINT(47.68825486110486 -121.37783781306068) bank103871 +103872 POINT(47.082325852008324 -122.05661292648043) bank103872 +103873 POINT(48.37619014934786 -121.76128584930001) bank103873 +103874 POINT(47.426914222998896 -122.28530164554243) bank103874 +103875 POINT(48.267988737333674 -122.39635690370491) bank103875 +103876 POINT(47.74588117368432 -123.02409224368263) bank103876 +103877 POINT(48.36405767693284 -122.99145902146665) bank103877 +103878 POINT(47.26377611612694 -122.25079341396257) bank103878 +103879 POINT(46.74592671708422 -123.02405748147325) bank103879 +103880 POINT(46.86343261482833 -122.25568065212667) bank103880 +103881 POINT(47.29120165866485 -122.52185661001901) bank103881 +103882 POINT(47.57643774479293 -121.5722058113909) bank103882 +103883 POINT(47.88563919328188 -123.03696837750465) bank103883 +103884 POINT(47.954638413722144 -123.2017330151926) bank103884 +103885 POINT(47.49153210276732 -122.94829812820268) bank103885 +103886 POINT(48.37648086901254 -121.66695343830462) bank103886 +103887 POINT(47.89502602606141 -122.16198802751246) bank103887 +103888 POINT(46.77277484327796 -122.77615739726062) bank103888 +103889 POINT(48.56509113808502 -122.74085350098098) bank103889 +103890 POINT(48.309295646964074 -121.98777693285605) bank103890 +103891 POINT(47.08249593367503 -121.70219385478796) bank103891 +103892 POINT(47.874206648659765 -123.27328674711336) bank103892 +103893 POINT(47.70685655267951 -122.62659255476385) bank103893 +103894 POINT(48.49481266982825 -123.20868137895937) bank103894 +103895 POINT(48.47764068843614 -121.40083683149095) bank103895 +103896 POINT(47.60040820300068 -123.23524606702394) bank103896 +103897 POINT(48.077073409875766 -122.14127006395005) bank103897 +103898 POINT(47.9360903136559 -121.79191573627047) bank103898 +103899 POINT(46.82143922817401 -121.52962547162274) bank103899 +103900 POINT(47.20006502316674 -123.2691937860792) bank103900 +103901 POINT(48.27826398333165 -123.09484326631674) bank103901 +103902 POINT(46.91090036776973 -121.62107530857912) bank103902 +103903 POINT(48.26743606442129 -122.02274206575899) bank103903 +103904 POINT(47.377417018968 -122.19567458136022) bank103904 +103905 POINT(47.328039928507266 -123.19652661360333) bank103905 +103906 POINT(46.889779406400265 -122.79300241754478) bank103906 +103907 POINT(47.16568333349218 -122.58133249664384) bank103907 +103908 POINT(48.28495507625863 -123.00247267553922) bank103908 +103909 POINT(47.12071205563375 -123.01270354132785) bank103909 +103910 POINT(48.269970979810836 -123.04141074802922) bank103910 +103911 POINT(48.41295393466187 -122.67922831750695) bank103911 +103912 POINT(48.32163666628889 -122.8499554151284) bank103912 +103913 POINT(48.504394594871954 -123.30195553677541) bank103913 +103914 POINT(47.38381634235323 -121.67571019602234) bank103914 +103915 POINT(48.386264542689624 -122.35660212313614) bank103915 +103916 POINT(47.81974849153047 -123.22670746973873) bank103916 +103917 POINT(46.77015193386671 -122.39376318005372) bank103917 +103918 POINT(47.889607268197075 -122.22008549479773) bank103918 +103919 POINT(47.45759057756396 -121.59228063741911) bank103919 +103920 POINT(46.67839324568475 -122.88357982138542) bank103920 +103921 POINT(48.50919753871101 -121.93488539663626) bank103921 +103922 POINT(48.186138667508686 -122.69247457214574) bank103922 +103923 POINT(46.71922995325385 -121.88833229113744) bank103923 +103924 POINT(47.29695100611766 -123.28934518901629) bank103924 +103925 POINT(47.132155349632306 -121.93728419082217) bank103925 +103926 POINT(47.52031443519288 -122.37567771757205) bank103926 +103927 POINT(47.36297696566334 -121.35385752167562) bank103927 +103928 POINT(47.59319930903826 -122.11769934411755) bank103928 +103929 POINT(47.31560687387612 -123.076733643774) bank103929 +103930 POINT(47.1663576243798 -122.60217367008022) bank103930 +103931 POINT(47.03860736417898 -121.48023244597043) bank103931 +103932 POINT(47.829319448750816 -122.40574984038814) bank103932 +103933 POINT(48.445490491670654 -122.87130022610347) bank103933 +103934 POINT(48.50629684457818 -122.32868206834102) bank103934 +103935 POINT(46.7935102599935 -122.96459555343245) bank103935 +103936 POINT(46.792240246395416 -122.1406259726681) bank103936 +103937 POINT(48.475358297271654 -121.35702205487345) bank103937 +103938 POINT(47.07631160414397 -123.31696786610738) bank103938 +103939 POINT(48.135440124303436 -121.51186192795123) bank103939 +103940 POINT(47.575106569316446 -122.45500503608524) bank103940 +103941 POINT(47.81072252134582 -121.66082456550346) bank103941 +103942 POINT(46.669010514509324 -122.67636433521903) bank103942 +103943 POINT(47.67870309217056 -122.7843144728692) bank103943 +103944 POINT(47.431567219835415 -121.7170437503123) bank103944 +103945 POINT(47.259561062308585 -123.29688007356962) bank103945 +103946 POINT(48.20821396247993 -123.21226827280228) bank103946 +103947 POINT(48.02630129493029 -123.12337695416997) bank103947 +103948 POINT(47.39789828569332 -122.7785223217258) bank103948 +103949 POINT(47.76100451074526 -122.20833438364463) bank103949 +103950 POINT(48.154529256106215 -122.47096116358102) bank103950 +103951 POINT(47.56213510978806 -122.77848038938492) bank103951 +103952 POINT(47.443720601809844 -121.77291316012709) bank103952 +103953 POINT(48.59973331735382 -121.99568867723858) bank103953 +103954 POINT(46.853878526544925 -121.75709927682895) bank103954 +103955 POINT(47.63086790572337 -122.6362077732153) bank103955 +103956 POINT(47.40483874793706 -123.23723973559544) bank103956 +103957 POINT(47.29605779091117 -121.38755060865685) bank103957 +103958 POINT(46.632235713952 -122.35343056675194) bank103958 +103959 POINT(46.89347785075282 -123.17127037330422) bank103959 +103960 POINT(48.443419183865764 -123.1018250437638) bank103960 +103961 POINT(47.612947983651594 -121.33378193787898) bank103961 +103962 POINT(47.27437024720733 -122.21903898227002) bank103962 +103963 POINT(46.832565318913524 -123.19140230044489) bank103963 +103964 POINT(48.028058765869176 -121.82973722975528) bank103964 +103965 POINT(48.22417434755018 -122.99494006338422) bank103965 +103966 POINT(47.66677034404928 -123.20364027744381) bank103966 +103967 POINT(48.348511171106516 -121.69807362151268) bank103967 +103968 POINT(48.39936976272521 -122.05040164631275) bank103968 +103969 POINT(48.00637229818445 -121.36962316385397) bank103969 +103970 POINT(48.14266031871118 -123.06859488218937) bank103970 +103971 POINT(48.46210813925162 -121.61447818385194) bank103971 +103972 POINT(48.241355797872544 -122.15830064843243) bank103972 +103973 POINT(46.941592286668225 -122.7770705599399) bank103973 +103974 POINT(48.441037107895724 -121.98749939415453) bank103974 +103975 POINT(47.3585175474547 -123.28953256385014) bank103975 +103976 POINT(48.598562669753825 -122.57157839748987) bank103976 +103977 POINT(47.36599617421645 -121.47188162110176) bank103977 +103978 POINT(46.718063950765966 -122.943666727179) bank103978 +103979 POINT(48.581228933627465 -122.25923254575177) bank103979 +103980 POINT(47.00232746264562 -121.84598394771702) bank103980 +103981 POINT(47.827790163948954 -121.80111325664288) bank103981 +103982 POINT(48.463954207447706 -122.38472513253096) bank103982 +103983 POINT(47.797072924552545 -123.13062570328431) bank103983 +103984 POINT(48.16919628002088 -122.95745580374626) bank103984 +103985 POINT(47.769205570272945 -121.75522327769808) bank103985 +103986 POINT(46.91656118783257 -121.91703924808218) bank103986 +103987 POINT(47.872679655620374 -121.6199818794356) bank103987 +103988 POINT(46.941195260127344 -122.98535480894446) bank103988 +103989 POINT(47.18696711427036 -122.80163289798516) bank103989 +103990 POINT(47.16742351453665 -121.6219612882514) bank103990 +103991 POINT(48.33604590607984 -122.35218916034628) bank103991 +103992 POINT(48.298292506211546 -122.20801892640668) bank103992 +103993 POINT(47.94283402660316 -121.47684447886118) bank103993 +103994 POINT(47.25656034866636 -122.64375391845073) bank103994 +103995 POINT(47.68321436543153 -122.08466717913424) bank103995 +103996 POINT(48.35665003980772 -122.69754879802909) bank103996 +103997 POINT(46.76825974216404 -122.36508852461573) bank103997 +103998 POINT(47.69599892927805 -122.86578455645329) bank103998 +103999 POINT(47.362910777099295 -121.4065961860717) bank103999 +104000 POINT(47.907519545016854 -121.37103671066403) bank104000 +104001 POINT(47.461916600172266 -123.06698602964421) bank104001 +104002 POINT(48.28485006384967 -121.72143678051587) bank104002 +104003 POINT(47.45123048587478 -122.46209801419437) bank104003 +104004 POINT(47.06043937580201 -122.91958473817435) bank104004 +104005 POINT(47.5156145839256 -122.73113454135866) bank104005 +104006 POINT(47.93570598584331 -121.7501223022106) bank104006 +104007 POINT(47.81494876228916 -122.63476613875767) bank104007 +104008 POINT(46.7231208000416 -121.34176091648756) bank104008 +104009 POINT(47.98966580318563 -121.56019145010325) bank104009 +104010 POINT(47.85289017042776 -121.82361880257336) bank104010 +104011 POINT(47.91147859431383 -122.83793585808218) bank104011 +104012 POINT(48.51468147478377 -121.94192571044778) bank104012 +104013 POINT(47.007632482381744 -121.34952895304085) bank104013 +104014 POINT(47.74491299999451 -122.72456966014103) bank104014 +104015 POINT(48.43689342397703 -121.68720455131908) bank104015 +104016 POINT(47.16033314996295 -122.49531060151324) bank104016 +104017 POINT(46.74921290722122 -121.97966197334705) bank104017 +104018 POINT(47.14930030598519 -122.04314707349445) bank104018 +104019 POINT(47.792347864921695 -123.31749693033893) bank104019 +104020 POINT(48.11907497741153 -121.87887056348707) bank104020 +104021 POINT(47.60985334748586 -122.03212299321277) bank104021 +104022 POINT(47.98269974361606 -121.67446706162008) bank104022 +104023 POINT(47.28186937876212 -122.09157726255923) bank104023 +104024 POINT(47.49280144021032 -122.36590656441084) bank104024 +104025 POINT(47.92795611177431 -121.62983411322827) bank104025 +104026 POINT(48.25561179632106 -122.84124580075546) bank104026 +104027 POINT(47.12466136934309 -121.98113375973115) bank104027 +104028 POINT(48.187399289780956 -121.60943214641604) bank104028 +104029 POINT(47.84697477170386 -121.43478077088356) bank104029 +104030 POINT(46.81911265200862 -122.54254016256796) bank104030 +104031 POINT(47.13104036803538 -123.3287169173691) bank104031 +104032 POINT(47.640515662228935 -123.20317219243081) bank104032 +104033 POINT(48.53720495044784 -122.11157345375854) bank104033 +104034 POINT(47.208961940408486 -122.1609943390085) bank104034 +104035 POINT(48.279226195403254 -122.42843047614137) bank104035 +104036 POINT(47.41784721257502 -121.52478464457081) bank104036 +104037 POINT(47.7655640663792 -121.9199536874993) bank104037 +104038 POINT(48.051686361297826 -122.20760405732398) bank104038 +104039 POINT(47.34085028277269 -121.78045339860162) bank104039 +104040 POINT(47.92930949569214 -122.80403217407525) bank104040 +104041 POINT(46.93792773523161 -122.91460037503404) bank104041 +104042 POINT(47.528875653989196 -121.84668946878219) bank104042 +104043 POINT(47.07856692644537 -121.67518856917961) bank104043 +104044 POINT(48.15719793344214 -122.4448234762346) bank104044 +104045 POINT(47.81309817026881 -123.00710727571258) bank104045 +104046 POINT(48.115501237151705 -122.93984768524574) bank104046 +104047 POINT(47.66045828511354 -122.40514537537297) bank104047 +104048 POINT(46.92499027349642 -123.15600987150505) bank104048 +104049 POINT(47.16672711535937 -122.56149284503662) bank104049 +104050 POINT(47.61053294864717 -121.44822559559283) bank104050 +104051 POINT(48.30132935618452 -122.27303914758053) bank104051 +104052 POINT(46.61110448372173 -122.38930163439149) bank104052 +104053 POINT(46.730990745797385 -123.30077067913504) bank104053 +104054 POINT(46.798494771747606 -121.91403581498668) bank104054 +104055 POINT(47.90335150868807 -122.45162299487674) bank104055 +104056 POINT(48.4986966667439 -123.30616693654365) bank104056 +104057 POINT(48.59650457100704 -122.77744173792271) bank104057 +104058 POINT(47.972030739577114 -123.16516615115337) bank104058 +104059 POINT(46.994407597267205 -123.00936907407157) bank104059 +104060 POINT(47.31896245652878 -121.63529423549424) bank104060 +104061 POINT(47.92653904681161 -122.13771674308501) bank104061 +104062 POINT(47.81278168570247 -121.41531535160658) bank104062 +104063 POINT(47.15483283383945 -123.21324356529476) bank104063 +104064 POINT(46.64546393536396 -122.42101453165964) bank104064 +104065 POINT(47.14655057384867 -122.18587501814149) bank104065 +104066 POINT(47.82960799412655 -121.95897133492248) bank104066 +104067 POINT(47.65601456745685 -121.67605045706458) bank104067 +104068 POINT(48.05095479687655 -123.1273271170204) bank104068 +104069 POINT(47.341609476815925 -123.26095596552098) bank104069 +104070 POINT(48.47328897955027 -121.78610148860436) bank104070 +104071 POINT(48.365373941826455 -122.36234988686574) bank104071 +104072 POINT(47.640367382816216 -121.39580003224616) bank104072 +104073 POINT(48.495138374040216 -122.41090637795956) bank104073 +104074 POINT(48.04030059596736 -121.42190422978736) bank104074 +104075 POINT(47.767450277915565 -121.97198037833687) bank104075 +104076 POINT(47.82628921973263 -123.07214594208402) bank104076 +104077 POINT(47.8655545614077 -121.33527487725812) bank104077 +104078 POINT(47.142080207939 -123.32237472353891) bank104078 +104079 POINT(48.208125673040016 -121.90928579069652) bank104079 +104080 POINT(48.23030103062617 -122.42450385230781) bank104080 +104081 POINT(48.22570445993075 -121.53136528976322) bank104081 +104082 POINT(48.220901666974186 -122.6544599957161) bank104082 +104083 POINT(47.09477867033106 -121.42374585472938) bank104083 +104084 POINT(48.3259449408349 -123.13090623610545) bank104084 +104085 POINT(47.435475117395725 -122.24589571110207) bank104085 +104086 POINT(48.179303115983814 -123.07784303385635) bank104086 +104087 POINT(47.79046420111927 -122.20370495513754) bank104087 +104088 POINT(47.75894125996691 -122.19055646297396) bank104088 +104089 POINT(47.558906150912506 -122.64873925167808) bank104089 +104090 POINT(48.55293254453423 -122.24784133507514) bank104090 +104091 POINT(47.01682593171304 -122.82566470386219) bank104091 +104092 POINT(47.402321058497094 -123.18509967167954) bank104092 +104093 POINT(48.523769419038516 -121.81632404637735) bank104093 +104094 POINT(48.011397188877694 -122.4897265286942) bank104094 +104095 POINT(48.60432703836102 -123.32925917328521) bank104095 +104096 POINT(47.82908507423664 -123.1377091999162) bank104096 +104097 POINT(48.46127076588625 -121.37446432190981) bank104097 +104098 POINT(47.73621607817971 -122.74202738588806) bank104098 +104099 POINT(47.819899076119704 -122.90116679576346) bank104099 +104100 POINT(48.30234087777784 -122.67775134577703) bank104100 +104101 POINT(46.806943885091265 -122.15480179179156) bank104101 +104102 POINT(47.18719657474256 -122.1944179693732) bank104102 +104103 POINT(47.741469313245624 -122.81081001488505) bank104103 +104104 POINT(46.6622831841931 -121.97532845450205) bank104104 +104105 POINT(48.07734539777436 -122.7975077296653) bank104105 +104106 POINT(47.803249613395124 -122.81609282766745) bank104106 +104107 POINT(48.579158375429785 -121.34296160302496) bank104107 +104108 POINT(48.050721254549934 -121.84654799809717) bank104108 +104109 POINT(48.320039801776296 -122.61588491261183) bank104109 +104110 POINT(48.52717645864736 -122.92526175482298) bank104110 +104111 POINT(48.45197227271258 -123.17034768491972) bank104111 +104112 POINT(47.13977342655875 -122.17217519943492) bank104112 +104113 POINT(47.10298179118988 -122.06352018068354) bank104113 +104114 POINT(46.664981545610566 -122.39016742784119) bank104114 +104115 POINT(47.54979006466123 -122.60476538500596) bank104115 +104116 POINT(47.33971672862496 -122.35971327438398) bank104116 +104117 POINT(46.702425652955554 -122.12785809680906) bank104117 +104118 POINT(48.23081266382376 -122.91710773655768) bank104118 +104119 POINT(47.137014875604045 -122.99249567350616) bank104119 +104120 POINT(47.30470492327583 -121.6426211863797) bank104120 +104121 POINT(47.429653382969214 -121.56705940590915) bank104121 +104122 POINT(48.24394628311995 -122.7223868008343) bank104122 +104123 POINT(48.321489858085336 -122.396178883409) bank104123 +104124 POINT(46.79496402805082 -121.6794185927998) bank104124 +104125 POINT(47.58504326851004 -122.86420638781904) bank104125 +104126 POINT(48.322132647822684 -121.49149014567492) bank104126 +104127 POINT(46.739328998244325 -121.38986811513611) bank104127 +104128 POINT(48.08735290913207 -123.10178999310328) bank104128 +104129 POINT(47.71695956841491 -121.4066321093132) bank104129 +104130 POINT(47.38585244724115 -122.56326439828972) bank104130 +104131 POINT(48.452721930253624 -122.1594639265874) bank104131 +104132 POINT(47.850583281381134 -122.91052705791421) bank104132 +104133 POINT(48.5819239783199 -122.73515040228277) bank104133 +104134 POINT(47.81492862740366 -122.17167070099775) bank104134 +104135 POINT(47.88252206732789 -122.02116038739646) bank104135 +104136 POINT(48.20808152583633 -123.22949604827812) bank104136 +104137 POINT(48.222037938226144 -123.26100052388286) bank104137 +104138 POINT(47.36622309789745 -121.47871994344095) bank104138 +104139 POINT(47.50279454286492 -122.6286129740283) bank104139 +104140 POINT(47.08672427468199 -121.76635840170067) bank104140 +104141 POINT(47.574200892725926 -121.7814183617508) bank104141 +104142 POINT(48.15913773532684 -122.59469608687948) bank104142 +104143 POINT(46.91516782750719 -122.89603646485497) bank104143 +104144 POINT(46.95747590459135 -122.65511735158537) bank104144 +104145 POINT(47.637401656689335 -123.1213430353254) bank104145 +104146 POINT(48.243292486854905 -121.33502267276963) bank104146 +104147 POINT(48.516733740677296 -123.04240152412044) bank104147 +104148 POINT(48.54698777482497 -122.51856032119814) bank104148 +104149 POINT(48.010162269317 -123.0645354388105) bank104149 +104150 POINT(46.760151484096205 -122.09961333915447) bank104150 +104151 POINT(48.03747197268419 -122.53638582035543) bank104151 +104152 POINT(47.62075332614773 -121.77484943415422) bank104152 +104153 POINT(47.73394776034019 -122.44790247329917) bank104153 +104154 POINT(47.93883071084465 -122.99360181116343) bank104154 +104155 POINT(47.4919500618077 -122.1658410212902) bank104155 +104156 POINT(48.45303023723746 -123.22957775895489) bank104156 +104157 POINT(48.55225298459093 -122.65967479031468) bank104157 +104158 POINT(47.41436265267138 -121.6282216557489) bank104158 +104159 POINT(46.703078901818714 -122.32668472200098) bank104159 +104160 POINT(46.899552355917145 -123.16269032712482) bank104160 +104161 POINT(47.90116099668365 -122.42041585864659) bank104161 +104162 POINT(48.43885211830946 -121.71231306418869) bank104162 +104163 POINT(47.9501371170922 -121.83919219767557) bank104163 +104164 POINT(47.16710787047873 -122.66085205855921) bank104164 +104165 POINT(46.83651150036551 -123.20507775192974) bank104165 +104166 POINT(47.13239056451851 -121.40686150296459) bank104166 +104167 POINT(48.24825653979545 -121.80072257328221) bank104167 +104168 POINT(47.46374818760793 -122.01204558702126) bank104168 +104169 POINT(47.73111285705825 -122.06220704775217) bank104169 +104170 POINT(47.77947670044667 -121.47002959238468) bank104170 +104171 POINT(47.934285213205584 -121.95019082426168) bank104171 +104172 POINT(46.63500364781647 -123.16674926477602) bank104172 +104173 POINT(46.76809737903413 -123.11797933458223) bank104173 +104174 POINT(47.60419448568955 -121.4268895611588) bank104174 +104175 POINT(46.678680290881594 -122.98819159138007) bank104175 +104176 POINT(47.041171000405456 -123.18452736012844) bank104176 +104177 POINT(47.40343531197666 -122.24774116897493) bank104177 +104178 POINT(47.29011877145082 -122.01495753204955) bank104178 +104179 POINT(47.467551715992855 -121.55227420397061) bank104179 +104180 POINT(48.42519202788596 -122.5976483912516) bank104180 +104181 POINT(47.27618758376903 -122.03766699143054) bank104181 +104182 POINT(47.59232593151622 -121.74868157131807) bank104182 +104183 POINT(47.769988912962106 -123.22174638242178) bank104183 +104184 POINT(47.328985417676975 -122.56259943570288) bank104184 +104185 POINT(47.03373827462679 -123.13912976695173) bank104185 +104186 POINT(47.26216352916435 -122.30640621117526) bank104186 +104187 POINT(47.96494245607711 -121.80119107804744) bank104187 +104188 POINT(48.16729555750106 -122.82435938044623) bank104188 +104189 POINT(48.165617177168684 -122.55407614706968) bank104189 +104190 POINT(48.04309941860176 -121.83395580363518) bank104190 +104191 POINT(47.441524626386155 -121.63484288054676) bank104191 +104192 POINT(47.045156719783236 -121.95041805222866) bank104192 +104193 POINT(47.85650588922879 -121.40628212022959) bank104193 +104194 POINT(47.099435986446885 -123.08182141393372) bank104194 +104195 POINT(46.87205378093781 -122.05845146773694) bank104195 +104196 POINT(47.97368925331504 -122.54493468504234) bank104196 +104197 POINT(47.50740760066722 -121.9964919417661) bank104197 +104198 POINT(47.22858602237565 -122.93618153694939) bank104198 +104199 POINT(47.22376243349099 -121.3908419263852) bank104199 +104200 POINT(47.32460286679503 -122.04362382993335) bank104200 +104201 POINT(47.36812245955943 -123.05565981127295) bank104201 +104202 POINT(46.98840264180075 -123.28825176025993) bank104202 +104203 POINT(47.89612673457526 -121.64117299569828) bank104203 +104204 POINT(48.21080431832389 -122.57970467467047) bank104204 +104205 POINT(47.79569573184983 -121.59738315337067) bank104205 +104206 POINT(46.69030479592484 -121.5575128165037) bank104206 +104207 POINT(47.921837227869986 -122.73970103283443) bank104207 +104208 POINT(48.15613335297347 -123.27705680647715) bank104208 +104209 POINT(47.28479040853599 -121.82203495169112) bank104209 +104210 POINT(47.46006426628095 -121.81831189540041) bank104210 +104211 POINT(47.50241476439552 -121.66780220451129) bank104211 +104212 POINT(46.77604211615203 -122.61187734383365) bank104212 +104213 POINT(47.81980717902377 -121.95070965443747) bank104213 +104214 POINT(48.02815126504581 -122.47281144237297) bank104214 +104215 POINT(47.96728354830852 -121.55816026805687) bank104215 +104216 POINT(48.023966228100655 -122.79526752519801) bank104216 +104217 POINT(46.684469338111086 -122.23830874117618) bank104217 +104218 POINT(47.24852156899122 -121.86102335910645) bank104218 +104219 POINT(48.36265341391054 -122.65797091647511) bank104219 +104220 POINT(47.55074420161244 -121.53555073385029) bank104220 +104221 POINT(47.85876028778625 -122.91834384465747) bank104221 +104222 POINT(47.06011632850498 -121.96474757165429) bank104222 +104223 POINT(47.053647874345366 -121.78582624827602) bank104223 +104224 POINT(47.744126985543915 -122.60581385571909) bank104224 +104225 POINT(47.78458349215901 -122.10433461173879) bank104225 +104226 POINT(47.46205753533738 -121.9736202663205) bank104226 +104227 POINT(47.40754505682732 -123.09023959593668) bank104227 +104228 POINT(47.411163742271455 -121.87868819250039) bank104228 +104229 POINT(48.30751511776337 -122.52120612491517) bank104229 +104230 POINT(46.936786376330296 -121.3467136119985) bank104230 +104231 POINT(46.73892022027933 -122.23258299811869) bank104231 +104232 POINT(46.949627359671105 -121.84102350944174) bank104232 +104233 POINT(47.33812067261772 -122.57853417124062) bank104233 +104234 POINT(47.10369741659807 -122.75085078349717) bank104234 +104235 POINT(47.33868878045703 -121.66803421491426) bank104235 +104236 POINT(46.770183155685146 -122.26437606929353) bank104236 +104237 POINT(48.4277769912045 -122.85497178698823) bank104237 +104238 POINT(47.52191767901561 -122.70723228168994) bank104238 +104239 POINT(46.85486271092229 -122.38882696840516) bank104239 +104240 POINT(48.564132771945424 -122.02700790429941) bank104240 +104241 POINT(47.940496372009235 -121.70855261278554) bank104241 +104242 POINT(47.77161142627313 -122.75599082850766) bank104242 +104243 POINT(47.902102247473614 -123.27338665551076) bank104243 +104244 POINT(47.59085199073014 -122.92043885558462) bank104244 +104245 POINT(47.664997195160794 -121.67361037813522) bank104245 +104246 POINT(47.94247400452684 -123.17398043435168) bank104246 +104247 POINT(46.68001815881376 -122.16281617615735) bank104247 +104248 POINT(46.98127063604447 -121.39003363119542) bank104248 +104249 POINT(47.82734775162624 -122.33710509450552) bank104249 +104250 POINT(46.801659734086776 -122.8604737906547) bank104250 +104251 POINT(46.86288960620925 -122.82072478759017) bank104251 +104252 POINT(47.020202923993416 -122.63385976297786) bank104252 +104253 POINT(47.08178785834551 -122.76716428025) bank104253 +104254 POINT(47.63349515448915 -122.83660668647926) bank104254 +104255 POINT(47.29862573592624 -122.13455743021515) bank104255 +104256 POINT(47.40181190132544 -121.50994997064679) bank104256 +104257 POINT(48.20024760558325 -121.73968442989943) bank104257 +104258 POINT(47.550473311682836 -121.77510620700255) bank104258 +104259 POINT(48.40227376329718 -121.78835635874184) bank104259 +104260 POINT(48.088106462981365 -121.67654547872115) bank104260 +104261 POINT(47.14517057217139 -122.66339248105287) bank104261 +104262 POINT(48.32860575280079 -122.86146575859223) bank104262 +104263 POINT(46.931724436852434 -122.76282300060979) bank104263 +104264 POINT(48.45236862262642 -123.14655244842744) bank104264 +104265 POINT(47.103904850376665 -122.60038237130733) bank104265 +104266 POINT(47.96996467361169 -121.93784240904247) bank104266 +104267 POINT(48.603245384457196 -122.40459635971574) bank104267 +104268 POINT(47.14648896540167 -123.10760473414359) bank104268 +104269 POINT(47.197310221384456 -122.5944887550445) bank104269 +104270 POINT(48.22164875940999 -122.77466288173386) bank104270 +104271 POINT(47.05074232391333 -122.38427914026086) bank104271 +104272 POINT(47.17649506581488 -123.18404229954115) bank104272 +104273 POINT(48.33060405971361 -123.11752480269026) bank104273 +104274 POINT(47.88494548649189 -123.07968007817492) bank104274 +104275 POINT(47.319283753229165 -122.74188399046423) bank104275 +104276 POINT(48.011851634881715 -123.30104155412923) bank104276 +104277 POINT(48.44537270053718 -121.82037796105881) bank104277 +104278 POINT(48.12031677523507 -122.10396953252985) bank104278 +104279 POINT(47.053441678390044 -121.44985232509063) bank104279 +104280 POINT(46.7746920010753 -122.68074862027724) bank104280 +104281 POINT(47.79500278364579 -122.03072384218952) bank104281 +104282 POINT(46.70726922189491 -121.76067453368842) bank104282 +104283 POINT(48.33273277762713 -122.91592611502008) bank104283 +104284 POINT(48.54951434085158 -122.47518098867442) bank104284 +104285 POINT(47.83737787439778 -122.48892524395796) bank104285 +104286 POINT(46.95289829256402 -121.60483477834198) bank104286 +104287 POINT(48.43239073423689 -122.14320283864883) bank104287 +104288 POINT(48.50183164214367 -121.81328851450186) bank104288 +104289 POINT(47.02127818940485 -122.32872830913202) bank104289 +104290 POINT(46.952893869137704 -123.10434997647002) bank104290 +104291 POINT(48.34949254439424 -122.83150003854396) bank104291 +104292 POINT(46.93161537323313 -122.27300521461736) bank104292 +104293 POINT(47.58414062829854 -122.64965135415552) bank104293 +104294 POINT(47.13587570406528 -122.16311814906518) bank104294 +104295 POINT(47.05635454637642 -123.28880961247738) bank104295 +104296 POINT(47.290053771902734 -122.85718694404542) bank104296 +104297 POINT(46.948089387722504 -121.67547057045142) bank104297 +104298 POINT(47.7038549892674 -122.29491563611796) bank104298 +104299 POINT(48.1908076435917 -123.1949835638643) bank104299 +104300 POINT(47.60810706232185 -122.21977589869987) bank104300 +104301 POINT(48.523088379088286 -121.4565592591477) bank104301 +104302 POINT(47.917659384178975 -123.17614944605478) bank104302 +104303 POINT(48.4998102594868 -122.05056569100383) bank104303 +104304 POINT(47.5515568415428 -122.40501950112527) bank104304 +104305 POINT(48.18082137969575 -121.3798827017075) bank104305 +104306 POINT(47.14883215392391 -121.63811724389025) bank104306 +104307 POINT(47.63943434446144 -122.59893147149114) bank104307 +104308 POINT(46.782333983430014 -122.30255738773984) bank104308 +104309 POINT(46.78961235816583 -121.84243515587394) bank104309 +104310 POINT(48.11746600976795 -121.57424574404402) bank104310 +104311 POINT(47.30063218358731 -123.07418895995306) bank104311 +104312 POINT(47.43446220690846 -123.11342507193568) bank104312 +104313 POINT(48.544720258663396 -122.30830233466413) bank104313 +104314 POINT(47.19096832555115 -121.44245193263556) bank104314 +104315 POINT(46.7864514936357 -121.80591571530621) bank104315 +104316 POINT(48.20833961069536 -122.45580790492902) bank104316 +104317 POINT(48.14052802383137 -123.03208667748558) bank104317 +104318 POINT(48.16202728363502 -122.60351386456671) bank104318 +104319 POINT(47.82330195582749 -122.6405191945667) bank104319 +104320 POINT(47.66371682815086 -122.60163325291731) bank104320 +104321 POINT(48.458797369245 -122.4593823748054) bank104321 +104322 POINT(47.48720615722621 -123.11362287272688) bank104322 +104323 POINT(47.84390740488872 -122.05037433697105) bank104323 +104324 POINT(46.88506800649841 -123.27834658740602) bank104324 +104325 POINT(48.04509018913199 -123.32270132970088) bank104325 +104326 POINT(46.97103689744517 -123.32924394325072) bank104326 +104327 POINT(47.66857153829356 -122.23872857677576) bank104327 +104328 POINT(47.00421973856339 -122.31727716052491) bank104328 +104329 POINT(48.23747369373939 -123.0220222672089) bank104329 +104330 POINT(47.238400005667614 -122.47558985863499) bank104330 +104331 POINT(46.92361706617787 -121.39791766633277) bank104331 +104332 POINT(47.60203031098872 -121.87862059501204) bank104332 +104333 POINT(47.126681281287226 -122.3399190077391) bank104333 +104334 POINT(46.700366813251755 -122.07234274952059) bank104334 +104335 POINT(47.34688388519254 -122.74470780024888) bank104335 +104336 POINT(46.740098854710126 -121.94701116007371) bank104336 +104337 POINT(46.71919511582108 -123.14555080947157) bank104337 +104338 POINT(47.81453193827713 -122.57358996018701) bank104338 +104339 POINT(48.06976004628349 -123.03633888509616) bank104339 +104340 POINT(46.987793641569056 -123.14227055300978) bank104340 +104341 POINT(47.138964268879036 -122.85370875701415) bank104341 +104342 POINT(47.79010456344596 -121.47265516754815) bank104342 +104343 POINT(47.44308211009036 -121.36652021677234) bank104343 +104344 POINT(47.18348330450082 -121.6580091835103) bank104344 +104345 POINT(48.47410286186856 -121.50048762719992) bank104345 +104346 POINT(48.075818335259086 -121.47824021560118) bank104346 +104347 POINT(47.43579903880917 -121.59987399626816) bank104347 +104348 POINT(47.607435829348354 -122.51844146375727) bank104348 +104349 POINT(47.683434318371 -122.47112960118116) bank104349 +104350 POINT(46.68275612650867 -123.26933978880767) bank104350 +104351 POINT(48.528432772758585 -121.75693693240056) bank104351 +104352 POINT(47.21417902575421 -123.21951570494465) bank104352 +104353 POINT(48.031902287033425 -122.81188563867319) bank104353 +104354 POINT(47.17073841021498 -121.77736382009735) bank104354 +104355 POINT(46.90514827004788 -123.17286393954687) bank104355 +104356 POINT(47.27274095600627 -122.72306003289457) bank104356 +104357 POINT(47.42164254310583 -122.6121067698312) bank104357 +104358 POINT(48.34122766713528 -122.29649828041413) bank104358 +104359 POINT(48.54369973464393 -122.19504298957419) bank104359 +104360 POINT(46.75917127828993 -123.14926726132995) bank104360 +104361 POINT(46.62169482576524 -122.0741623389391) bank104361 +104362 POINT(48.45388682235028 -122.10906282055592) bank104362 +104363 POINT(47.4311263753286 -122.54711221250054) bank104363 +104364 POINT(46.89591367209724 -123.0084305664551) bank104364 +104365 POINT(47.26721904326809 -122.42331370432147) bank104365 +104366 POINT(48.16544295671595 -121.45216565449599) bank104366 +104367 POINT(48.38661760854839 -121.80727597715932) bank104367 +104368 POINT(48.53952243644567 -121.75339378585399) bank104368 +104369 POINT(47.38033769398251 -121.94818086533753) bank104369 +104370 POINT(47.86723773152982 -122.97701952533217) bank104370 +104371 POINT(47.55379049766965 -122.0979846012682) bank104371 +104372 POINT(47.83326299234016 -121.43684041801727) bank104372 +104373 POINT(48.05561778184913 -121.66438978898441) bank104373 +104374 POINT(46.80974667840684 -123.13382681308713) bank104374 +104375 POINT(46.90956113759266 -121.33422274133558) bank104375 +104376 POINT(48.01696772987712 -122.30405756809218) bank104376 +104377 POINT(47.131688899190735 -122.26969253282198) bank104377 +104378 POINT(47.24733257982428 -121.85283276873055) bank104378 +104379 POINT(47.14696871014406 -121.74225666316606) bank104379 +104380 POINT(47.59652937530089 -123.02807799655626) bank104380 +104381 POINT(47.326182260741604 -121.41905558493285) bank104381 +104382 POINT(47.23524640037403 -122.95946321642815) bank104382 +104383 POINT(47.411108622323546 -122.22323220715913) bank104383 +104384 POINT(47.07651052293923 -121.46901683808913) bank104384 +104385 POINT(46.62931210430625 -122.6031245291374) bank104385 +104386 POINT(48.251872033341364 -121.80661186428655) bank104386 +104387 POINT(46.72084654245863 -123.03803305322288) bank104387 +104388 POINT(47.4266978258088 -123.02936014078045) bank104388 +104389 POINT(46.85848288032435 -121.52349698699526) bank104389 +104390 POINT(47.180777435346926 -123.21360491657494) bank104390 +104391 POINT(47.68642049968817 -123.20838045004739) bank104391 +104392 POINT(47.722751536821214 -122.50312661228855) bank104392 +104393 POINT(47.37427627013655 -121.74390717614538) bank104393 +104394 POINT(48.58714721740145 -121.52632580961483) bank104394 +104395 POINT(46.6152619453492 -122.33155723686211) bank104395 +104396 POINT(47.8428364310494 -122.53440525360885) bank104396 +104397 POINT(48.20771295265855 -122.95674950114288) bank104397 +104398 POINT(47.23164412246556 -122.13017227915125) bank104398 +104399 POINT(46.61375844527376 -122.06000591313962) bank104399 +104400 POINT(48.05614701334341 -121.93328651226373) bank104400 +104401 POINT(48.18761489612995 -121.37967266436281) bank104401 +104402 POINT(47.95632889597891 -122.2914127331931) bank104402 +104403 POINT(47.71630197606473 -122.76410011112691) bank104403 +104404 POINT(48.592533065360726 -121.51186891566056) bank104404 +104405 POINT(47.506448775375375 -122.35285891168391) bank104405 +104406 POINT(47.54199693303322 -122.75425487437242) bank104406 +104407 POINT(46.73767601110379 -122.03403361635436) bank104407 +104408 POINT(48.34600912812877 -122.77865393265229) bank104408 +104409 POINT(46.716741751400235 -123.1233345103717) bank104409 +104410 POINT(47.27122355752724 -122.00491591869874) bank104410 +104411 POINT(46.92758041406718 -122.66793000830052) bank104411 +104412 POINT(47.72142897853211 -122.14828890524971) bank104412 +104413 POINT(47.389689845320596 -123.13632930715924) bank104413 +104414 POINT(47.39398620378579 -121.78848455415795) bank104414 +104415 POINT(47.39201201154718 -122.01913384363223) bank104415 +104416 POINT(47.319648802976936 -122.83565142721842) bank104416 +104417 POINT(48.017498212022495 -122.44780708713515) bank104417 +104418 POINT(47.08636104574793 -121.53384252956288) bank104418 +104419 POINT(48.50001759043845 -122.09680140489623) bank104419 +104420 POINT(48.44434406684709 -121.6610231648688) bank104420 +104421 POINT(46.647590824821044 -122.26958248015724) bank104421 +104422 POINT(47.7326881704041 -121.49853115685627) bank104422 +104423 POINT(46.721530646563934 -122.6791052243806) bank104423 +104424 POINT(48.31670291694748 -121.9917164915147) bank104424 +104425 POINT(46.984735583027096 -121.93472744671372) bank104425 +104426 POINT(46.90467236469105 -122.78779096602175) bank104426 +104427 POINT(46.777264444393744 -122.36926795642357) bank104427 +104428 POINT(46.89455351130981 -121.57762757620605) bank104428 +104429 POINT(46.75546419190763 -121.46397594649761) bank104429 +104430 POINT(47.37059674578217 -121.69436678749997) bank104430 +104431 POINT(47.536460911651425 -121.72185734944794) bank104431 +104432 POINT(47.63244813246185 -122.83508598257853) bank104432 +104433 POINT(47.518753208512535 -121.75094604560222) bank104433 +104434 POINT(46.83819284558005 -122.64976739586115) bank104434 +104435 POINT(47.387090498572654 -122.88123974218726) bank104435 +104436 POINT(48.52842300696498 -123.01606990518536) bank104436 +104437 POINT(46.76516808198072 -122.77672575004013) bank104437 +104438 POINT(48.24670469256826 -121.67001335165658) bank104438 +104439 POINT(48.24591257148139 -122.25945814783972) bank104439 +104440 POINT(47.53872429547494 -122.19679133132689) bank104440 +104441 POINT(47.315469642193236 -121.52535081855946) bank104441 +104442 POINT(47.32931517512709 -122.48203708259223) bank104442 +104443 POINT(47.676718074002146 -123.16097184925573) bank104443 +104444 POINT(47.76866532905713 -123.26792255473276) bank104444 +104445 POINT(48.458171601793666 -121.83087062777328) bank104445 +104446 POINT(48.29508833882102 -122.78976332143993) bank104446 +104447 POINT(48.02117945064928 -121.69638792585438) bank104447 +104448 POINT(46.998627012476156 -122.99178755301625) bank104448 +104449 POINT(47.90960500609859 -122.38338393043377) bank104449 +104450 POINT(47.900813588977854 -121.4397887235151) bank104450 +104451 POINT(48.44697465090997 -121.86051256258088) bank104451 +104452 POINT(47.81555488278498 -122.92230851589746) bank104452 +104453 POINT(48.20509367781499 -123.25448162449746) bank104453 +104454 POINT(47.14313694705035 -122.7818677473076) bank104454 +104455 POINT(48.12441749137949 -122.7980363012736) bank104455 +104456 POINT(48.248750369728505 -122.28323126300215) bank104456 +104457 POINT(47.6732110256864 -121.77552431376736) bank104457 +104458 POINT(47.289487390066185 -122.11980649519239) bank104458 +104459 POINT(47.012367407160006 -121.77951593852838) bank104459 +104460 POINT(46.87408877494119 -121.50946343508905) bank104460 +104461 POINT(46.72644975042438 -122.59807062870964) bank104461 +104462 POINT(47.42884477743222 -122.4325404342779) bank104462 +104463 POINT(46.76031937333195 -121.5345985820518) bank104463 +104464 POINT(47.03651254462414 -121.62626788800782) bank104464 +104465 POINT(48.51728640240658 -122.12290956347036) bank104465 +104466 POINT(47.370035492726096 -122.89286695688746) bank104466 +104467 POINT(48.39122412233649 -121.6853559307937) bank104467 +104468 POINT(46.89038556446325 -122.0557850032883) bank104468 +104469 POINT(47.22621258091222 -122.51296938961748) bank104469 +104470 POINT(47.55395948741383 -122.18171077808202) bank104470 +104471 POINT(47.87276510578403 -121.79810939927474) bank104471 +104472 POINT(47.73261282895866 -121.65032412675696) bank104472 +104473 POINT(48.41448303262469 -122.01698574909335) bank104473 +104474 POINT(47.74213968527938 -121.35666517427018) bank104474 +104475 POINT(46.845620753690106 -121.40644440943476) bank104475 +104476 POINT(48.30386840276832 -121.51007910803996) bank104476 +104477 POINT(47.43642271716685 -121.4775199170513) bank104477 +104478 POINT(47.947746485898996 -122.16336810159821) bank104478 +104479 POINT(47.55285143519797 -122.94104082581963) bank104479 +104480 POINT(47.86854608356223 -122.92556470560247) bank104480 +104481 POINT(48.37938859104755 -122.12858811021627) bank104481 +104482 POINT(46.93861106836425 -121.56412751056669) bank104482 +104483 POINT(46.646223396904865 -123.12717027078423) bank104483 +104484 POINT(46.698800442039385 -121.8029762787421) bank104484 +104485 POINT(46.85085432295967 -121.57242707422996) bank104485 +104486 POINT(47.02439028135191 -122.32537768930884) bank104486 +104487 POINT(46.621860026597155 -122.19204850403068) bank104487 +104488 POINT(47.17700010576266 -122.84125606043598) bank104488 +104489 POINT(47.63181884519715 -122.26961073128281) bank104489 +104490 POINT(48.4061088785866 -123.0480625070228) bank104490 +104491 POINT(47.889502306792046 -122.51882617931739) bank104491 +104492 POINT(47.71622560917223 -122.97933517948891) bank104492 +104493 POINT(47.808761849601964 -123.27924258059811) bank104493 +104494 POINT(48.18317319941678 -123.03918555511567) bank104494 +104495 POINT(47.600425244617554 -121.57689975972842) bank104495 +104496 POINT(47.793533708891296 -121.85840043224222) bank104496 +104497 POINT(47.59485578220655 -122.28022107864878) bank104497 +104498 POINT(47.525214118400235 -122.0592666834627) bank104498 +104499 POINT(46.87935380363898 -121.8014646880232) bank104499 +104500 POINT(48.170366588442754 -122.42410915957298) bank104500 +104501 POINT(48.179572779704635 -121.52609339375017) bank104501 +104502 POINT(48.53364572162907 -122.25688657580642) bank104502 +104503 POINT(47.52460446254291 -122.70577564485315) bank104503 +104504 POINT(46.99693083647332 -123.12323721813117) bank104504 +104505 POINT(46.96684443141898 -121.6591954736927) bank104505 +104506 POINT(47.234031211496145 -122.40814843432186) bank104506 +104507 POINT(46.62105165241674 -122.22198176629334) bank104507 +104508 POINT(48.27858584316071 -122.79530036771098) bank104508 +104509 POINT(47.54657455583881 -122.04937407352857) bank104509 +104510 POINT(47.60384621724552 -121.76284113763676) bank104510 +104511 POINT(47.43447665971139 -123.12809783914156) bank104511 +104512 POINT(47.02376683493956 -122.96310378080652) bank104512 +104513 POINT(48.017429210180104 -122.51164398470215) bank104513 +104514 POINT(48.05441922432538 -122.18622371403573) bank104514 +104515 POINT(46.806475076424604 -122.2958789339211) bank104515 +104516 POINT(48.104963921012335 -122.41670525000598) bank104516 +104517 POINT(47.3965060349867 -121.64144857757884) bank104517 +104518 POINT(48.50255397070071 -122.4020788162511) bank104518 +104519 POINT(47.81212512274434 -123.30636043151065) bank104519 +104520 POINT(47.88467405219796 -123.17216544770491) bank104520 +104521 POINT(47.461327489821855 -122.74766076403981) bank104521 +104522 POINT(48.2649302599155 -123.18127460101714) bank104522 +104523 POINT(47.54105848290617 -121.87566766080612) bank104523 +104524 POINT(47.73571387289502 -122.3186004217982) bank104524 +104525 POINT(47.306855997587576 -121.6414941375946) bank104525 +104526 POINT(47.687661574202295 -122.54727060038736) bank104526 +104527 POINT(47.81992707191694 -121.77320100495065) bank104527 +104528 POINT(46.71300146127574 -122.25548575804059) bank104528 +104529 POINT(48.35060834911515 -121.43612495017129) bank104529 +104530 POINT(48.05339311565701 -123.02682892687282) bank104530 +104531 POINT(48.50814022953321 -122.11497018508747) bank104531 +104532 POINT(46.67772322150175 -122.25288728315857) bank104532 +104533 POINT(47.71182530413985 -122.57899441745197) bank104533 +104534 POINT(47.0723534939674 -122.07446190940844) bank104534 +104535 POINT(47.491658462078306 -121.37781622635852) bank104535 +104536 POINT(46.89952690159781 -122.0532579767916) bank104536 +104537 POINT(47.92962936415419 -121.78751502132573) bank104537 +104538 POINT(46.62169242733454 -121.7916046855808) bank104538 +104539 POINT(47.03855611280956 -121.98173074964215) bank104539 +104540 POINT(47.070498994065 -122.39806944744413) bank104540 +104541 POINT(46.989600045017156 -122.33767905772552) bank104541 +104542 POINT(47.58329567917691 -122.33488493887874) bank104542 +104543 POINT(48.267794080446976 -122.15031709973243) bank104543 +104544 POINT(48.206071429495665 -123.26150632843087) bank104544 +104545 POINT(48.54066439312786 -123.30342641654356) bank104545 +104546 POINT(47.231123438607646 -121.84014303820751) bank104546 +104547 POINT(47.28976089164123 -121.90586824129696) bank104547 +104548 POINT(47.38152840535619 -121.64913693751984) bank104548 +104549 POINT(46.92779988150189 -121.46644890987251) bank104549 +104550 POINT(48.21145933994744 -122.71197507473968) bank104550 +104551 POINT(46.92341030034974 -123.01493756960443) bank104551 +104552 POINT(47.158398575930434 -121.80029019867641) bank104552 +104553 POINT(47.441228179322856 -123.29250396581072) bank104553 +104554 POINT(48.40202015382321 -121.6419607779776) bank104554 +104555 POINT(48.46848076683846 -122.02248518202256) bank104555 +104556 POINT(48.3701230542279 -121.64346869310495) bank104556 +104557 POINT(46.671220223926724 -123.08142832043589) bank104557 +104558 POINT(46.97817662498661 -122.48541425273785) bank104558 +104559 POINT(46.65505032507589 -123.0252656261089) bank104559 +104560 POINT(46.75523067512264 -122.85154503907567) bank104560 +104561 POINT(47.700504714528805 -123.2119606503534) bank104561 +104562 POINT(48.4789566415728 -122.41475393730833) bank104562 +104563 POINT(47.45920421181913 -122.03305089744433) bank104563 +104564 POINT(47.047911655297774 -122.72548046334447) bank104564 +104565 POINT(47.42529789825247 -121.8126853717868) bank104565 +104566 POINT(46.61406147565357 -121.69288320377662) bank104566 +104567 POINT(48.283663364166074 -122.05942503207787) bank104567 +104568 POINT(48.14995266664286 -121.97227451878737) bank104568 +104569 POINT(48.083740301557924 -121.44825911546712) bank104569 +104570 POINT(48.371257582782114 -122.31662919589567) bank104570 +104571 POINT(48.51623025826139 -122.02827340855059) bank104571 +104572 POINT(48.4045907516017 -122.49702623504818) bank104572 +104573 POINT(47.65104773925787 -122.25513692144423) bank104573 +104574 POINT(47.341229509090546 -122.26603485737795) bank104574 +104575 POINT(46.951611678824534 -122.59907061918909) bank104575 +104576 POINT(47.97784897987894 -123.21921248149957) bank104576 +104577 POINT(48.061620665075985 -122.96915268560755) bank104577 +104578 POINT(46.72651170776643 -122.15015664215498) bank104578 +104579 POINT(47.92925419955762 -122.79660754347749) bank104579 +104580 POINT(46.99222004130532 -121.5182725618714) bank104580 +104581 POINT(47.933130362689205 -121.50699300841663) bank104581 +104582 POINT(48.5208765085948 -123.30680859898027) bank104582 +104583 POINT(46.89590432015042 -122.93046676309544) bank104583 +104584 POINT(46.99671796367356 -121.7391033918174) bank104584 +104585 POINT(47.23791835164252 -123.10133340276857) bank104585 +104586 POINT(46.768133912432155 -121.95346555972925) bank104586 +104587 POINT(46.694855946307996 -123.0667535605769) bank104587 +104588 POINT(47.439918146552465 -122.01199488126704) bank104588 +104589 POINT(48.269283934869236 -121.873677796384) bank104589 +104590 POINT(47.00705724730415 -122.93520677809279) bank104590 +104591 POINT(47.68584795652948 -121.81220773972987) bank104591 +104592 POINT(47.680854614598594 -122.30284056017312) bank104592 +104593 POINT(47.93437928295593 -122.04013967924486) bank104593 +104594 POINT(47.40331521010687 -121.39783281792842) bank104594 +104595 POINT(47.709706602073624 -121.7261226678222) bank104595 +104596 POINT(47.72442859640399 -122.2624560123764) bank104596 +104597 POINT(48.29601986810736 -121.43276966791665) bank104597 +104598 POINT(46.68865714760731 -121.75479794265237) bank104598 +104599 POINT(47.20992764477618 -121.85521836066428) bank104599 +104600 POINT(47.699419281779136 -122.34632306872544) bank104600 +104601 POINT(47.865971222682965 -122.93622657915547) bank104601 +104602 POINT(48.605826047031854 -123.00125822470908) bank104602 +104603 POINT(47.708771689889716 -122.10150523202832) bank104603 +104604 POINT(46.796973522793955 -123.12954508416098) bank104604 +104605 POINT(47.768582147040284 -122.72801498654202) bank104605 +104606 POINT(47.617829207704006 -121.99447224919071) bank104606 +104607 POINT(48.03318253411176 -123.21917730016823) bank104607 +104608 POINT(48.12498342023369 -122.08225439132254) bank104608 +104609 POINT(48.00596982770512 -122.49769505516035) bank104609 +104610 POINT(47.64682419132926 -123.11606510356303) bank104610 +104611 POINT(46.760551508427994 -122.69705849856008) bank104611 +104612 POINT(48.542831441685806 -122.87424231872252) bank104612 +104613 POINT(48.180524610990744 -122.87390801551406) bank104613 +104614 POINT(47.32823833989824 -121.37534356784988) bank104614 +104615 POINT(46.67465288248389 -122.49946656602144) bank104615 +104616 POINT(47.11128205261139 -122.49558735356085) bank104616 +104617 POINT(46.77763971932571 -122.89549720139664) bank104617 +104618 POINT(48.376842859109594 -122.71382717158913) bank104618 +104619 POINT(48.601926575761475 -122.09667168536758) bank104619 +104620 POINT(47.752927694772524 -121.76870589895543) bank104620 +104621 POINT(48.29351772078469 -122.4366267023095) bank104621 +104622 POINT(47.79419472693353 -123.31528821244714) bank104622 +104623 POINT(48.103982710686935 -121.54321633009788) bank104623 +104624 POINT(48.5334033553086 -121.36253169167894) bank104624 +104625 POINT(47.956343043985825 -122.54411473777542) bank104625 +104626 POINT(48.196257515287456 -123.07815219415436) bank104626 +104627 POINT(48.58899848239533 -121.80258949944809) bank104627 +104628 POINT(48.582123480341735 -123.06780510470027) bank104628 +104629 POINT(47.04043962843916 -122.78518654988513) bank104629 +104630 POINT(47.93621786066961 -122.01908336676703) bank104630 +104631 POINT(48.21696857871174 -122.46736266880716) bank104631 +104632 POINT(47.61694921608928 -121.44667935269385) bank104632 +104633 POINT(47.00752743478114 -121.60213577504855) bank104633 +104634 POINT(46.71017754244954 -121.59323497731263) bank104634 +104635 POINT(48.16811860012157 -121.97870887484511) bank104635 +104636 POINT(47.290975049933124 -122.66345881046085) bank104636 +104637 POINT(47.691175352975016 -123.03033654668306) bank104637 +104638 POINT(48.13838760432975 -122.21089695240147) bank104638 +104639 POINT(47.63440422049454 -121.94135205969009) bank104639 +104640 POINT(48.31644578470998 -122.54081122391716) bank104640 +104641 POINT(47.416389777335155 -121.70225610229949) bank104641 +104642 POINT(47.43693805949519 -121.42381086500636) bank104642 +104643 POINT(48.19432923379309 -121.9588082699212) bank104643 +104644 POINT(48.15933160378542 -122.39467916956927) bank104644 +104645 POINT(47.32961082619033 -122.23894830645864) bank104645 +104646 POINT(47.95142363290964 -121.38563251743574) bank104646 +104647 POINT(48.05242954706225 -123.12853640795765) bank104647 +104648 POINT(46.640588513827126 -123.18596639334407) bank104648 +104649 POINT(47.01797116126287 -122.78229427005864) bank104649 +104650 POINT(48.384778416181895 -122.47310224355353) bank104650 +104651 POINT(47.92895722493975 -122.05374731136445) bank104651 +104652 POINT(46.80413289207989 -121.89890326650462) bank104652 +104653 POINT(47.753371893291416 -121.87948476496531) bank104653 +104654 POINT(48.43173695397741 -121.43538606650199) bank104654 +104655 POINT(46.901350314219314 -122.27445208765013) bank104655 +104656 POINT(47.42869826925277 -121.63963903865888) bank104656 +104657 POINT(46.771079279493705 -121.70091680088412) bank104657 +104658 POINT(47.399336124336116 -121.43133430699643) bank104658 +104659 POINT(46.691706465886774 -121.5613247771747) bank104659 +104660 POINT(47.654640407626424 -122.55294602174794) bank104660 +104661 POINT(47.3845165341316 -122.0274366624436) bank104661 +104662 POINT(48.259836946939906 -122.98641692475735) bank104662 +104663 POINT(47.05664483493164 -122.0112633480343) bank104663 +104664 POINT(47.898685495441356 -123.17008769351274) bank104664 +104665 POINT(47.050370386606915 -122.13518794939355) bank104665 +104666 POINT(48.32534583965277 -121.37034880373542) bank104666 +104667 POINT(46.86874354498689 -122.11120083277495) bank104667 +104668 POINT(48.08347675385886 -121.53733324417773) bank104668 +104669 POINT(48.58653378571701 -122.82719063136231) bank104669 +104670 POINT(47.8221645399509 -121.83860596585113) bank104670 +104671 POINT(47.74971453213385 -122.84307085785147) bank104671 +104672 POINT(48.41473273341739 -121.73664120350755) bank104672 +104673 POINT(47.04268647424805 -122.90201868444979) bank104673 +104674 POINT(46.985969663747575 -121.81466971806151) bank104674 +104675 POINT(47.81680287252448 -121.44365137872747) bank104675 +104676 POINT(47.56051186426126 -123.02691737012178) bank104676 +104677 POINT(46.637404250093496 -122.79535524043885) bank104677 +104678 POINT(48.197634897670184 -123.07572942464779) bank104678 +104679 POINT(47.20980891349086 -122.92014865218776) bank104679 +104680 POINT(47.97868305793132 -123.01408350773053) bank104680 +104681 POINT(46.82145440508747 -121.54488330581788) bank104681 +104682 POINT(47.18390977631427 -121.49737720986359) bank104682 +104683 POINT(48.54532499629578 -121.71729309217746) bank104683 +104684 POINT(47.44407270135461 -122.60791660603013) bank104684 +104685 POINT(46.70872540525097 -122.42803415791266) bank104685 +104686 POINT(47.400075106095805 -121.35197033170739) bank104686 +104687 POINT(47.67872184117073 -121.6152088825332) bank104687 +104688 POINT(46.776291948834725 -122.13033799830494) bank104688 +104689 POINT(48.039256496954856 -123.10891222271468) bank104689 +104690 POINT(47.41672306853106 -122.81859964623182) bank104690 +104691 POINT(48.37298740367771 -121.52921481399926) bank104691 +104692 POINT(47.01802246233815 -122.60589192195889) bank104692 +104693 POINT(47.15202000965428 -121.78510194018433) bank104693 +104694 POINT(46.86125729884103 -122.31684076862183) bank104694 +104695 POINT(47.95342403117471 -122.82899849586708) bank104695 +104696 POINT(47.14071030863895 -123.2342360347663) bank104696 +104697 POINT(47.78979470974785 -122.53849046236137) bank104697 +104698 POINT(47.0750074640018 -122.02186027180474) bank104698 +104699 POINT(47.34406096316447 -122.52219817202094) bank104699 +104700 POINT(47.053366125997115 -122.09373432553213) bank104700 +104701 POINT(47.45239345510922 -123.2200805198542) bank104701 +104702 POINT(46.740435544341686 -122.64332928513734) bank104702 +104703 POINT(48.011429198319085 -122.1261683634358) bank104703 +104704 POINT(47.115511033942816 -122.52283472949213) bank104704 +104705 POINT(47.6848637472896 -121.59199904223628) bank104705 +104706 POINT(47.49103519517932 -123.24993762763467) bank104706 +104707 POINT(48.373565082828165 -122.99564184156918) bank104707 +104708 POINT(48.0391955580372 -121.96023871535493) bank104708 +104709 POINT(47.8095440092676 -121.97366352500298) bank104709 +104710 POINT(46.847717145032746 -122.81967573866564) bank104710 +104711 POINT(48.26455569950077 -121.90631127217894) bank104711 +104712 POINT(48.54291581292921 -121.76260722604161) bank104712 +104713 POINT(47.54459414772572 -121.41407420244484) bank104713 +104714 POINT(47.01685678724386 -123.25600020483361) bank104714 +104715 POINT(48.04082500772186 -122.82069129869402) bank104715 +104716 POINT(47.60335523189239 -122.82021635305597) bank104716 +104717 POINT(46.695005757356576 -121.77705608903668) bank104717 +104718 POINT(47.85528595075787 -121.63009260918864) bank104718 +104719 POINT(48.077956478207476 -122.92392867216584) bank104719 +104720 POINT(48.40416385720968 -123.27117309202393) bank104720 +104721 POINT(46.820965950647995 -122.35563454880165) bank104721 +104722 POINT(46.83779146183233 -122.94081189238722) bank104722 +104723 POINT(47.04395865715285 -121.69267196670954) bank104723 +104724 POINT(48.4146558920913 -123.31078907010638) bank104724 +104725 POINT(47.48456608271753 -121.35409181874853) bank104725 +104726 POINT(47.35500218685498 -121.5465575616002) bank104726 +104727 POINT(48.26839810500372 -122.90841393059148) bank104727 +104728 POINT(47.45318987078066 -122.66367841468126) bank104728 +104729 POINT(48.01123655149467 -122.90243659113885) bank104729 +104730 POINT(47.37071975726984 -121.39842313872899) bank104730 +104731 POINT(48.25399579183559 -121.64420630364279) bank104731 +104732 POINT(48.27273776089587 -122.9033177723624) bank104732 +104733 POINT(48.388271180842644 -123.30264541106202) bank104733 +104734 POINT(47.46007157245079 -123.00653999303698) bank104734 +104735 POINT(47.951398822549464 -122.26767454405429) bank104735 +104736 POINT(48.31433533991058 -121.5792016224874) bank104736 +104737 POINT(47.50074880318664 -121.80620414119511) bank104737 +104738 POINT(47.669415475595756 -122.14743869409641) bank104738 +104739 POINT(47.12220410464789 -121.76808197620264) bank104739 +104740 POINT(46.93758033729721 -121.84765341348832) bank104740 +104741 POINT(47.92106386389126 -121.38681267469302) bank104741 +104742 POINT(46.65850613876064 -121.82319262504919) bank104742 +104743 POINT(47.49096808445714 -123.02133984716137) bank104743 +104744 POINT(48.188504100360134 -121.84703626827879) bank104744 +104745 POINT(46.93767316654875 -122.7919936472971) bank104745 +104746 POINT(47.46877912702752 -121.88299269149185) bank104746 +104747 POINT(47.933951586093045 -121.52122490124464) bank104747 +104748 POINT(47.08406415926539 -122.21502102176935) bank104748 +104749 POINT(47.822633726849254 -123.21733456765406) bank104749 +104750 POINT(47.75912815594668 -122.82086690130762) bank104750 +104751 POINT(47.537990363031916 -123.14825658396344) bank104751 +104752 POINT(46.81337191900289 -121.64495188703123) bank104752 +104753 POINT(47.793866829605825 -122.71275395932126) bank104753 +104754 POINT(47.21891877939238 -122.16361833815581) bank104754 +104755 POINT(47.14790666420784 -121.98934020665163) bank104755 +104756 POINT(47.76852063622592 -121.39399492847609) bank104756 +104757 POINT(46.96182695434255 -122.21910658981108) bank104757 +104758 POINT(47.87659393267326 -123.31662252105241) bank104758 +104759 POINT(47.8938502256203 -122.71383917589918) bank104759 +104760 POINT(47.50386571512269 -123.08296081137277) bank104760 +104761 POINT(47.68224305421983 -123.13456184428034) bank104761 +104762 POINT(48.16624296765036 -121.76012801564377) bank104762 +104763 POINT(48.133277353593776 -122.92256605613018) bank104763 +104764 POINT(47.60772075254364 -121.44587170714225) bank104764 +104765 POINT(48.16480164454667 -122.1523449013329) bank104765 +104766 POINT(47.142083165970405 -121.54700740775496) bank104766 +104767 POINT(48.33377113042153 -122.78251548519395) bank104767 +104768 POINT(48.19696209132433 -122.71928256663358) bank104768 +104769 POINT(47.78027740501171 -123.09361340229763) bank104769 +104770 POINT(46.60901215015851 -122.58229062779554) bank104770 +104771 POINT(47.31904291195937 -123.20344050616625) bank104771 +104772 POINT(48.37239802294898 -121.39187531759066) bank104772 +104773 POINT(47.157229105178054 -122.81621814278465) bank104773 +104774 POINT(48.51869992573876 -121.90279951441872) bank104774 +104775 POINT(48.33246101344377 -121.70965801501903) bank104775 +104776 POINT(47.64735390397708 -122.41588966807316) bank104776 +104777 POINT(47.33342301340261 -122.98296812045949) bank104777 +104778 POINT(48.49597085579062 -122.76432073698398) bank104778 +104779 POINT(47.42126263407967 -122.55340323458574) bank104779 +104780 POINT(48.19350911306158 -121.44447999033194) bank104780 +104781 POINT(46.86534264634634 -123.29043520057033) bank104781 +104782 POINT(48.39695853641943 -121.5015959049996) bank104782 +104783 POINT(48.300217592851695 -123.06077302069572) bank104783 +104784 POINT(46.93335380061392 -121.74938824350843) bank104784 +104785 POINT(48.037744486817815 -122.10835673194622) bank104785 +104786 POINT(47.33397737385587 -121.66032726205073) bank104786 +104787 POINT(47.77393165078597 -122.86132006062226) bank104787 +104788 POINT(46.61433016638071 -123.02760764040549) bank104788 +104789 POINT(46.71125507132488 -122.8115526576279) bank104789 +104790 POINT(48.29687649876107 -121.96710053921218) bank104790 +104791 POINT(47.34541333916567 -122.56959916798468) bank104791 +104792 POINT(47.702449190351594 -121.8994426416581) bank104792 +104793 POINT(46.82244393935355 -122.09808975305528) bank104793 +104794 POINT(46.607786521580984 -122.34371495319716) bank104794 +104795 POINT(47.79637245955719 -122.74141263762) bank104795 +104796 POINT(46.73249103366208 -122.18149986101909) bank104796 +104797 POINT(46.900120101878564 -121.81580932587754) bank104797 +104798 POINT(47.73831603999877 -121.5260561109644) bank104798 +104799 POINT(48.39809604067031 -122.8896094034675) bank104799 +104800 POINT(48.446130071476496 -122.43499731910747) bank104800 +104801 POINT(48.23685882066483 -123.1109157207968) bank104801 +104802 POINT(47.364788958730614 -123.29729448801608) bank104802 +104803 POINT(47.738031350698755 -122.56456808288621) bank104803 +104804 POINT(48.32781205578548 -122.691992420841) bank104804 +104805 POINT(47.384925961340066 -122.32069647247846) bank104805 +104806 POINT(48.519257664635596 -122.63609966728798) bank104806 +104807 POINT(47.0849424400652 -122.11161847567702) bank104807 +104808 POINT(46.792186068149825 -122.11327041007505) bank104808 +104809 POINT(47.56674222387705 -122.14697344008103) bank104809 +104810 POINT(47.70174878548303 -122.52271761270734) bank104810 +104811 POINT(47.4219812286639 -122.35941793215932) bank104811 +104812 POINT(47.04560994873095 -122.45177605966047) bank104812 +104813 POINT(47.42363093569055 -121.76622094837818) bank104813 +104814 POINT(47.638096448458086 -123.31783254013028) bank104814 +104815 POINT(47.47922320599817 -121.97463887129283) bank104815 +104816 POINT(47.65265725491669 -122.63532155305941) bank104816 +104817 POINT(48.57677374926055 -123.15678784860334) bank104817 +104818 POINT(48.0657550228233 -122.6678633347648) bank104818 +104819 POINT(47.72488132583028 -121.7857200637181) bank104819 +104820 POINT(47.69322339982191 -123.23954133101635) bank104820 +104821 POINT(47.0791099255635 -122.89417650826935) bank104821 +104822 POINT(48.25877102564871 -121.38995214523247) bank104822 +104823 POINT(47.94326722943623 -122.31221592344627) bank104823 +104824 POINT(48.15442180668251 -123.16545600054648) bank104824 +104825 POINT(48.10345352078866 -121.36022623617202) bank104825 +104826 POINT(47.634197454577034 -121.71524446924867) bank104826 +104827 POINT(47.39663021000899 -121.68634734575298) bank104827 +104828 POINT(47.285531735347305 -121.43575461218394) bank104828 +104829 POINT(47.85534226250195 -121.93549878620357) bank104829 +104830 POINT(47.10983187565214 -123.16279426057166) bank104830 +104831 POINT(48.4635209329796 -121.50446075299222) bank104831 +104832 POINT(47.66946561751662 -123.10854296296696) bank104832 +104833 POINT(47.822314254830644 -121.74644616085921) bank104833 +104834 POINT(48.293197442734325 -121.37297712658315) bank104834 +104835 POINT(48.235745667314276 -122.69548362142474) bank104835 +104836 POINT(48.425584052374404 -121.73804253131729) bank104836 +104837 POINT(47.547068263110546 -122.42391193413852) bank104837 +104838 POINT(47.647550934427635 -122.80923111414366) bank104838 +104839 POINT(47.7148136375592 -121.7085085227111) bank104839 +104840 POINT(48.54213401117399 -122.57907991940371) bank104840 +104841 POINT(48.33942012187349 -122.43930678782466) bank104841 +104842 POINT(47.95506839217594 -122.43129742114712) bank104842 +104843 POINT(47.13613922178762 -122.23116142599459) bank104843 +104844 POINT(47.27905616680673 -123.04683333648269) bank104844 +104845 POINT(47.56498462958224 -123.31899226819206) bank104845 +104846 POINT(48.42288015100326 -122.37441307413008) bank104846 +104847 POINT(47.10877321377623 -122.25147435417436) bank104847 +104848 POINT(47.352911015775874 -122.49725124307284) bank104848 +104849 POINT(47.75764276440428 -121.4783007939609) bank104849 +104850 POINT(48.144225379090514 -122.91520650609483) bank104850 +104851 POINT(48.176457551025365 -122.3009907906214) bank104851 +104852 POINT(46.79595904640509 -122.57664608659536) bank104852 +104853 POINT(48.21282064366588 -121.46046125104772) bank104853 +104854 POINT(47.12422355533302 -121.75669117466947) bank104854 +104855 POINT(47.24918759566909 -122.22426925460304) bank104855 +104856 POINT(47.2392448629159 -121.3603196990652) bank104856 +104857 POINT(46.81964175453283 -122.87972707733552) bank104857 +104858 POINT(47.75944564560993 -122.60133110877076) bank104858 +104859 POINT(48.11294231955532 -121.64937155784114) bank104859 +104860 POINT(47.74730166502399 -121.7614737730705) bank104860 +104861 POINT(48.23951340725281 -123.3196127023411) bank104861 +104862 POINT(47.89145314818966 -122.49009126040126) bank104862 +104863 POINT(48.42663912064198 -121.78566790900362) bank104863 +104864 POINT(48.355381171479344 -122.02756488172174) bank104864 +104865 POINT(47.08044246840413 -122.558402795052) bank104865 +104866 POINT(48.096684620729064 -122.08648297146947) bank104866 +104867 POINT(46.89052396953977 -123.29103716609443) bank104867 +104868 POINT(48.1223982330039 -121.89662454963958) bank104868 +104869 POINT(47.84587908942606 -122.08155517515542) bank104869 +104870 POINT(47.24927889128914 -122.26073715581538) bank104870 +104871 POINT(47.409526212410235 -121.66156770331223) bank104871 +104872 POINT(46.69954622622546 -121.6269775630429) bank104872 +104873 POINT(46.74089144284405 -123.23406459911129) bank104873 +104874 POINT(47.88106510679587 -122.07658286974494) bank104874 +104875 POINT(47.79652521336177 -123.12311469048778) bank104875 +104876 POINT(47.10305456526709 -121.6541500093318) bank104876 +104877 POINT(47.15044465920289 -122.78265871022788) bank104877 +104878 POINT(47.85932968481305 -121.46040968477716) bank104878 +104879 POINT(48.05960494724594 -122.594744515616) bank104879 +104880 POINT(46.94397563430368 -123.13010770698286) bank104880 +104881 POINT(46.635630522589594 -121.9562977150614) bank104881 +104882 POINT(47.28213895503999 -122.69161574923784) bank104882 +104883 POINT(47.38693455805638 -123.10101226799927) bank104883 +104884 POINT(47.139225505864765 -122.76494523646919) bank104884 +104885 POINT(48.17576376540191 -122.45132589625425) bank104885 +104886 POINT(48.210218585089464 -122.25092599699842) bank104886 +104887 POINT(46.981422023704376 -121.76258017399469) bank104887 +104888 POINT(46.92688303992751 -122.02060337179783) bank104888 +104889 POINT(47.64466492957868 -121.69882750685292) bank104889 +104890 POINT(46.78984786531646 -121.84937644330795) bank104890 +104891 POINT(47.088194942322325 -122.63368357200783) bank104891 +104892 POINT(47.58630144670786 -123.33113784839269) bank104892 +104893 POINT(47.11492806582872 -122.94032165510959) bank104893 +104894 POINT(47.750547828270406 -122.0257283248985) bank104894 +104895 POINT(46.892641935127514 -121.64922024831198) bank104895 +104896 POINT(47.70600635099403 -122.16114038688171) bank104896 +104897 POINT(46.62279315946026 -122.45417325165518) bank104897 +104898 POINT(47.2766377680972 -122.49711897414024) bank104898 +104899 POINT(48.43797850322209 -122.6714705371337) bank104899 +104900 POINT(47.84915038583574 -122.2437800087884) bank104900 +104901 POINT(48.072510596942735 -122.14989284728338) bank104901 +104902 POINT(47.876905297260976 -123.28971066215993) bank104902 +104903 POINT(48.11212404737337 -123.19840524895848) bank104903 +104904 POINT(48.41405942775428 -121.63152195397882) bank104904 +104905 POINT(46.975969894736345 -121.47662880166601) bank104905 +104906 POINT(47.970932758253305 -121.58124669539676) bank104906 +104907 POINT(47.29849922440021 -122.49016005400954) bank104907 +104908 POINT(47.10709255827859 -122.60592240311291) bank104908 +104909 POINT(47.85807240586636 -122.48590430636689) bank104909 +104910 POINT(48.23225572907248 -122.1585244675264) bank104910 +104911 POINT(46.76164108774216 -121.68326376715756) bank104911 +104912 POINT(48.24187716107471 -121.40621819723988) bank104912 +104913 POINT(46.94273244706809 -122.23680501526741) bank104913 +104914 POINT(48.407367179320886 -122.33389161014173) bank104914 +104915 POINT(46.939682153511455 -122.46868226572334) bank104915 +104916 POINT(47.57523592338937 -122.76368424748182) bank104916 +104917 POINT(46.8448644558183 -122.3450330308686) bank104917 +104918 POINT(48.07522856822796 -123.18113512152578) bank104918 +104919 POINT(46.715935234665594 -121.72836826088239) bank104919 +104920 POINT(48.569025067678744 -121.48287282697342) bank104920 +104921 POINT(47.56435017517202 -122.36659510539148) bank104921 +104922 POINT(48.16841261303968 -121.59613540107883) bank104922 +104923 POINT(47.244727118979704 -121.54935125400486) bank104923 +104924 POINT(48.594120344741974 -123.16226605232252) bank104924 +104925 POINT(46.829658952009055 -122.21323999783854) bank104925 +104926 POINT(46.684434608141736 -122.43980217702887) bank104926 +104927 POINT(47.028249894124365 -122.03830722358856) bank104927 +104928 POINT(48.53611224805688 -122.35293389473912) bank104928 +104929 POINT(48.005735837003755 -122.51699410012438) bank104929 +104930 POINT(47.550075834521074 -122.05381175206412) bank104930 +104931 POINT(47.64522999276651 -122.2152928172215) bank104931 +104932 POINT(46.89181163421361 -122.99150555679243) bank104932 +104933 POINT(48.45906524860189 -122.4220132928475) bank104933 +104934 POINT(48.338528224098624 -122.64169119987994) bank104934 +104935 POINT(47.286212230906884 -122.85416535525923) bank104935 +104936 POINT(46.84477890322742 -122.38788034297738) bank104936 +104937 POINT(47.47160432654725 -122.78388203625724) bank104937 +104938 POINT(47.894521088908334 -121.80556360853623) bank104938 +104939 POINT(46.830302984533674 -122.75797852829321) bank104939 +104940 POINT(46.624674815709255 -123.14240047387331) bank104940 +104941 POINT(47.7500220853938 -121.90813485535342) bank104941 +104942 POINT(48.181146796768715 -123.07357906694709) bank104942 +104943 POINT(48.02687414021084 -122.0856403261917) bank104943 +104944 POINT(47.44644077466527 -123.04136752015289) bank104944 +104945 POINT(48.07260900094781 -123.29868571065828) bank104945 +104946 POINT(48.47421642155793 -121.76041947535413) bank104946 +104947 POINT(46.89609920595376 -121.45402865891359) bank104947 +104948 POINT(47.116732532820826 -121.71581612899308) bank104948 +104949 POINT(47.88852684388787 -123.20524309614746) bank104949 +104950 POINT(48.57041649981657 -122.60216956590735) bank104950 +104951 POINT(48.32385028678422 -121.90705355612724) bank104951 +104952 POINT(48.41172163693886 -123.095886719374) bank104952 +104953 POINT(47.843030737249414 -122.05598127336863) bank104953 +104954 POINT(46.70704788969386 -121.80347265886242) bank104954 +104955 POINT(48.340998702591776 -122.35017858947764) bank104955 +104956 POINT(48.19350336160798 -122.54239270162834) bank104956 +104957 POINT(48.5200742106323 -122.0047888833091) bank104957 +104958 POINT(47.79418113088707 -121.37287086841631) bank104958 +104959 POINT(47.56871137116679 -123.2797480435727) bank104959 +104960 POINT(46.71046361762869 -122.72490991600199) bank104960 +104961 POINT(46.82972940892234 -122.66928908878933) bank104961 +104962 POINT(47.10448072684519 -122.21096194496687) bank104962 +104963 POINT(47.11026500205225 -122.00843956123133) bank104963 +104964 POINT(47.41517981717681 -122.79565657254152) bank104964 +104965 POINT(47.92710571421491 -122.16592697367892) bank104965 +104966 POINT(47.824764119536454 -121.9033221963125) bank104966 +104967 POINT(48.053086065804685 -122.6324926209391) bank104967 +104968 POINT(47.80076713276185 -121.97780180055635) bank104968 +104969 POINT(47.94254316125117 -122.76540826875568) bank104969 +104970 POINT(47.09827178249581 -122.29005862804657) bank104970 +104971 POINT(48.38689378663663 -122.91746846909957) bank104971 +104972 POINT(48.230769194568495 -122.23588284938197) bank104972 +104973 POINT(47.1805139058491 -122.3432873877443) bank104973 +104974 POINT(47.779069091082015 -122.79955920264213) bank104974 +104975 POINT(47.09150386443869 -122.14566130212242) bank104975 +104976 POINT(46.87040753546048 -122.13982804336379) bank104976 +104977 POINT(48.398700331750184 -122.36482538693343) bank104977 +104978 POINT(47.85197925384849 -121.94377757420068) bank104978 +104979 POINT(46.68622832584529 -122.58829216250771) bank104979 +104980 POINT(47.874513285686774 -121.89800641722424) bank104980 +104981 POINT(48.125184817276406 -121.96480147685428) bank104981 +104982 POINT(48.44961605207816 -121.77264624614388) bank104982 +104983 POINT(47.03222775987387 -122.27832736210534) bank104983 +104984 POINT(47.831835277320714 -123.08442331644864) bank104984 +104985 POINT(47.20063239854034 -122.38340563769133) bank104985 +104986 POINT(47.553774338898364 -122.1867994588422) bank104986 +104987 POINT(47.631687033527356 -121.62555892725098) bank104987 +104988 POINT(47.883951573614276 -121.3714320239829) bank104988 +104989 POINT(47.82900057507943 -122.70111391898165) bank104989 +104990 POINT(47.23696057288807 -122.1409023155933) bank104990 +104991 POINT(47.13261829509874 -123.01619605395757) bank104991 +104992 POINT(48.32013344424042 -121.97647972001917) bank104992 +104993 POINT(48.57070684480671 -121.45949381360246) bank104993 +104994 POINT(48.02569584989505 -122.44251527152046) bank104994 +104995 POINT(48.583081371800816 -122.54113904779854) bank104995 +104996 POINT(47.247478307518996 -123.17750920436987) bank104996 +104997 POINT(47.125306878815955 -122.31937029381152) bank104997 +104998 POINT(47.35074894780784 -123.22548396748152) bank104998 +104999 POINT(48.117667778677244 -122.39401512866414) bank104999 +105000 POINT(47.50433374464068 -122.03160109995541) bank105000 +105001 POINT(48.149950608218276 -123.28676128110013) bank105001 +105002 POINT(48.526093038397875 -121.88384471098409) bank105002 +105003 POINT(47.4307646063097 -122.05309737090069) bank105003 +105004 POINT(47.454903659305614 -123.0339581267777) bank105004 +105005 POINT(46.770885696432465 -123.14864952054933) bank105005 +105006 POINT(47.088960405394616 -122.54971518220529) bank105006 +105007 POINT(47.775369450400675 -122.0847762416455) bank105007 +105008 POINT(46.73800275828276 -122.18622399038043) bank105008 +105009 POINT(48.17391042144416 -121.8861744003465) bank105009 +105010 POINT(47.46466256461354 -123.06101319018528) bank105010 +105011 POINT(48.3286778841575 -122.89305656953644) bank105011 +105012 POINT(47.48875569946143 -121.54923632094119) bank105012 +105013 POINT(48.46432473036182 -121.59609940033941) bank105013 +105014 POINT(47.49462331617763 -121.6366768355386) bank105014 +105015 POINT(48.09473831852693 -122.39366760111878) bank105015 +105016 POINT(47.16869412897298 -121.89927137963375) bank105016 +105017 POINT(47.76414913230101 -122.69750903436378) bank105017 +105018 POINT(47.59357944156985 -122.7164605069345) bank105018 +105019 POINT(46.9883783502236 -122.39906542581826) bank105019 +105020 POINT(47.74706621445907 -122.1389622795879) bank105020 +105021 POINT(48.30636044400739 -121.7517208598688) bank105021 +105022 POINT(48.094140926364595 -123.0499551401921) bank105022 +105023 POINT(47.370555216895774 -121.70961600142793) bank105023 +105024 POINT(47.32230266156332 -122.05782232041656) bank105024 +105025 POINT(47.183557517024795 -121.93328496477989) bank105025 +105026 POINT(47.30805940254072 -123.14681131586018) bank105026 +105027 POINT(47.17240657376791 -122.11678004416355) bank105027 +105028 POINT(48.01784252118309 -123.14863211634565) bank105028 +105029 POINT(46.72065674901113 -122.86795461284315) bank105029 +105030 POINT(46.781884743908826 -123.05952497249633) bank105030 +105031 POINT(46.895891590563274 -121.50236435206607) bank105031 +105032 POINT(47.43871311631098 -122.03637178060814) bank105032 +105033 POINT(48.24518402919079 -121.71884433317729) bank105033 +105034 POINT(48.50051120875947 -122.47289547477857) bank105034 +105035 POINT(48.441012759625956 -122.17254543091836) bank105035 +105036 POINT(47.69298735543651 -121.90786335088576) bank105036 +105037 POINT(47.899199430909114 -121.58214373929184) bank105037 +105038 POINT(46.850064768285804 -123.20574539755897) bank105038 +105039 POINT(48.3821872895586 -122.90515866891549) bank105039 +105040 POINT(46.64998331824576 -122.07551544327774) bank105040 +105041 POINT(46.78303345061012 -122.26245359861703) bank105041 +105042 POINT(48.28561465683883 -122.1756878682352) bank105042 +105043 POINT(47.3848807901055 -122.70724756431534) bank105043 +105044 POINT(48.26858180109727 -122.50505979115319) bank105044 +105045 POINT(48.2626714423335 -122.19675056638016) bank105045 +105046 POINT(47.186467171520675 -121.72199491477794) bank105046 +105047 POINT(46.78162949067856 -121.7406183032701) bank105047 +105048 POINT(46.9425847427699 -122.90565146900748) bank105048 +105049 POINT(48.14813632552942 -121.36036308174788) bank105049 +105050 POINT(47.97630185414033 -122.01432193687987) bank105050 +105051 POINT(48.049157104445904 -122.05591983973055) bank105051 +105052 POINT(47.38348846307236 -123.02704104765682) bank105052 +105053 POINT(48.39479366552594 -121.57152880307169) bank105053 +105054 POINT(48.37864053285821 -122.4693301528104) bank105054 +105055 POINT(47.81430049722408 -122.82001440341337) bank105055 +105056 POINT(48.09056421174126 -123.19979968870886) bank105056 +105057 POINT(48.57224726827366 -122.99937736574113) bank105057 +105058 POINT(47.02251713214387 -122.6014369557837) bank105058 +105059 POINT(47.66438566692663 -121.45622970371107) bank105059 +105060 POINT(48.24462327330877 -122.54536653988369) bank105060 +105061 POINT(48.53116810705504 -121.87502711635459) bank105061 +105062 POINT(47.73663856531242 -121.6053645217215) bank105062 +105063 POINT(47.665647569003454 -122.1967832616244) bank105063 +105064 POINT(48.02196031233628 -122.51410425758725) bank105064 +105065 POINT(46.76963442881065 -121.89464673863966) bank105065 +105066 POINT(46.85185344462872 -123.0179755533084) bank105066 +105067 POINT(46.71240357077271 -123.21855456138785) bank105067 +105068 POINT(48.03638925974481 -121.37143180080346) bank105068 +105069 POINT(46.70912569460651 -122.2021991087224) bank105069 +105070 POINT(47.957086169971696 -122.45579163825143) bank105070 +105071 POINT(48.07322922710989 -121.78121411430428) bank105071 +105072 POINT(47.59184922851264 -122.19305378683173) bank105072 +105073 POINT(48.54698024706758 -121.38987105608932) bank105073 +105074 POINT(46.69176229030835 -121.84289076698681) bank105074 +105075 POINT(47.7036021445624 -122.86417375032575) bank105075 +105076 POINT(48.4567412683892 -122.03088367655508) bank105076 +105077 POINT(47.350546990277884 -121.80921345160691) bank105077 +105078 POINT(48.4286027971058 -122.29278643417103) bank105078 +105079 POINT(48.20951046494268 -122.16304892898408) bank105079 +105080 POINT(48.43313054693926 -121.34803008732003) bank105080 +105081 POINT(46.945504461678645 -121.65569879537347) bank105081 +105082 POINT(47.150623269369824 -122.91341423736006) bank105082 +105083 POINT(47.45972787275775 -121.95418740459574) bank105083 +105084 POINT(47.220854226392426 -122.24541890700621) bank105084 +105085 POINT(46.809695903867585 -122.75368795448536) bank105085 +105086 POINT(47.70590456708902 -122.13457076749418) bank105086 +105087 POINT(47.30599092861161 -121.35583608660734) bank105087 +105088 POINT(47.188638681782365 -122.80266992647908) bank105088 +105089 POINT(47.35829877700927 -121.43156503040869) bank105089 +105090 POINT(48.535114036469146 -121.48367707677299) bank105090 +105091 POINT(48.160338359440765 -121.68233897725325) bank105091 +105092 POINT(47.2326284462187 -122.1762079471602) bank105092 +105093 POINT(48.37196070013575 -121.4819425964272) bank105093 +105094 POINT(46.71226345042786 -123.14002019032084) bank105094 +105095 POINT(48.22212419045579 -123.19781866637852) bank105095 +105096 POINT(47.63726326165999 -123.01051518972085) bank105096 +105097 POINT(48.404249608701335 -121.84563831302546) bank105097 +105098 POINT(47.992234350381445 -122.37932283013781) bank105098 +105099 POINT(46.612432376829865 -121.44667377671774) bank105099 +105100 POINT(47.82769089528188 -121.55844338180516) bank105100 +105101 POINT(47.436516065462556 -122.06227845732465) bank105101 +105102 POINT(47.687474675457004 -123.02094583274666) bank105102 +105103 POINT(48.55057209406319 -122.0017926360575) bank105103 +105104 POINT(48.528742592746504 -122.22091960720532) bank105104 +105105 POINT(46.64017564294739 -122.32076704679257) bank105105 +105106 POINT(47.7016958677177 -121.80168756439117) bank105106 +105107 POINT(48.14549585596731 -122.23133693556369) bank105107 +105108 POINT(48.58192226118625 -121.54159030509622) bank105108 +105109 POINT(48.526899732264376 -122.47639719002578) bank105109 +105110 POINT(47.963639326318216 -122.79280955162254) bank105110 +105111 POINT(47.9966830588549 -122.6501822836409) bank105111 +105112 POINT(47.74317026231676 -123.06668427520474) bank105112 +105113 POINT(48.43387322783971 -121.64085466152412) bank105113 +105114 POINT(47.739715749263794 -122.4873013453813) bank105114 +105115 POINT(47.95784831149375 -121.33878667140206) bank105115 +105116 POINT(46.71299242881855 -121.53883706499319) bank105116 +105117 POINT(46.86008652414031 -121.75832417237822) bank105117 +105118 POINT(48.56106575215567 -122.46824889142152) bank105118 +105119 POINT(47.625305646728286 -122.34822026854513) bank105119 +105120 POINT(48.060245885444374 -121.5601178296969) bank105120 +105121 POINT(46.72834597409841 -122.57259836154623) bank105121 +105122 POINT(47.49844403840677 -123.27217257036864) bank105122 +105123 POINT(47.88918711779857 -122.76675549435667) bank105123 +105124 POINT(47.44189826123995 -121.78925946736004) bank105124 +105125 POINT(47.02247943418718 -121.7214896925452) bank105125 +105126 POINT(47.27023876165691 -122.01234361565407) bank105126 +105127 POINT(48.179666530060246 -121.81090691500974) bank105127 +105128 POINT(47.076357264288184 -122.49523463702877) bank105128 +105129 POINT(47.75674403741469 -123.10671524783594) bank105129 +105130 POINT(47.708597619551554 -121.72537058225642) bank105130 +105131 POINT(48.42686520664897 -121.44741852657708) bank105131 +105132 POINT(47.43571193421532 -123.26967673433522) bank105132 +105133 POINT(47.89997404328262 -122.9845937188485) bank105133 +105134 POINT(47.315203136274654 -122.40665518277464) bank105134 +105135 POINT(46.84491160887444 -121.99172963125747) bank105135 +105136 POINT(46.81269364630728 -122.45746729831458) bank105136 +105137 POINT(46.9604305270899 -122.80998959821372) bank105137 +105138 POINT(47.053678913536594 -122.34012775350514) bank105138 +105139 POINT(46.802357147452526 -123.26687461652976) bank105139 +105140 POINT(48.01835271510758 -122.75540740261746) bank105140 +105141 POINT(46.68106980183339 -122.4435108777798) bank105141 +105142 POINT(47.752247935277765 -122.90003486573276) bank105142 +105143 POINT(47.9870968199279 -122.15953890112517) bank105143 +105144 POINT(47.96146116392837 -122.42803470482983) bank105144 +105145 POINT(47.85005466691986 -121.71856427569985) bank105145 +105146 POINT(47.05338687032533 -122.33624369657127) bank105146 +105147 POINT(47.198643840452725 -122.02207245476534) bank105147 +105148 POINT(48.0969928647913 -122.40633318256745) bank105148 +105149 POINT(47.46084051445419 -123.19715040367409) bank105149 +105150 POINT(48.53596786718502 -123.0495637029384) bank105150 +105151 POINT(47.89948562115936 -122.95422183489572) bank105151 +105152 POINT(46.71702594948934 -122.79901337810442) bank105152 +105153 POINT(46.934355572618024 -122.34515152365651) bank105153 +105154 POINT(47.59992527885881 -122.70488877902318) bank105154 +105155 POINT(47.7141046760551 -121.9300915445106) bank105155 +105156 POINT(47.25904832083737 -123.27162020343255) bank105156 +105157 POINT(48.432988805479674 -121.4784846331547) bank105157 +105158 POINT(47.78179396775851 -122.50442014556681) bank105158 +105159 POINT(48.207182732844025 -121.75742944002725) bank105159 +105160 POINT(47.51337654151588 -121.47272292783745) bank105160 +105161 POINT(47.095941500929584 -123.1877612635957) bank105161 +105162 POINT(46.734157033706914 -122.02954140361287) bank105162 +105163 POINT(48.57495717370834 -123.04976585922347) bank105163 +105164 POINT(47.900430089508106 -122.9264362764224) bank105164 +105165 POINT(47.59345335072847 -122.96582000268617) bank105165 +105166 POINT(47.35852830902015 -121.64319114040411) bank105166 +105167 POINT(47.180496168670196 -121.4843329508004) bank105167 +105168 POINT(46.757801061650824 -123.21288543272864) bank105168 +105169 POINT(47.80059508701789 -121.92471297319635) bank105169 +105170 POINT(46.94465732652558 -122.64504264043973) bank105170 +105171 POINT(46.6937334202802 -121.53774321308926) bank105171 +105172 POINT(47.45895577285177 -122.67705259882251) bank105172 +105173 POINT(48.45986912272746 -123.16673475440196) bank105173 +105174 POINT(48.01456118417529 -123.23524824183188) bank105174 +105175 POINT(47.95387344051092 -123.14972566624583) bank105175 +105176 POINT(47.21823823764992 -121.34036724567244) bank105176 +105177 POINT(47.1251843175865 -122.70915229148191) bank105177 +105178 POINT(47.44396965097574 -121.49096044814827) bank105178 +105179 POINT(47.36187497904783 -121.88331554542285) bank105179 +105180 POINT(47.789585438411166 -121.4481089895057) bank105180 +105181 POINT(48.23749363976755 -121.9044302487098) bank105181 +105182 POINT(48.020482790992254 -122.79568729301238) bank105182 +105183 POINT(46.65350674415527 -123.28855733153537) bank105183 +105184 POINT(46.64836052964357 -122.20983217370892) bank105184 +105185 POINT(48.03997664995056 -122.9413480439727) bank105185 +105186 POINT(46.989508542961666 -121.55995923021754) bank105186 +105187 POINT(46.92835294873426 -121.34251570078368) bank105187 +105188 POINT(47.1347147868923 -122.03423996529234) bank105188 +105189 POINT(47.098389279533016 -122.08400883913437) bank105189 +105190 POINT(47.88645697185854 -121.60088417746726) bank105190 +105191 POINT(48.48426937067724 -123.0573019460553) bank105191 +105192 POINT(46.96162025617253 -122.01396880253851) bank105192 +105193 POINT(48.343588524813356 -122.85295147469367) bank105193 +105194 POINT(48.132743874954095 -122.34993504577345) bank105194 +105195 POINT(47.17107888502721 -122.39370013235241) bank105195 +105196 POINT(47.54349774661531 -122.18783457712921) bank105196 +105197 POINT(48.16952981492638 -121.9006405520342) bank105197 +105198 POINT(47.94289038456908 -122.15487395460798) bank105198 +105199 POINT(47.45498951230327 -122.61179580768913) bank105199 +105200 POINT(48.16407320978847 -122.3696610932814) bank105200 +105201 POINT(46.894402411498675 -122.16647112066822) bank105201 +105202 POINT(46.635249910461965 -121.84572140206097) bank105202 +105203 POINT(47.36037207143416 -122.91926167178204) bank105203 +105204 POINT(48.22913023520583 -122.69240165533938) bank105204 +105205 POINT(48.495435011080055 -121.97776693694162) bank105205 +105206 POINT(47.79840054726296 -122.32731848202165) bank105206 +105207 POINT(47.22781954869398 -121.60262948888142) bank105207 +105208 POINT(47.38943733155197 -121.7631392764196) bank105208 +105209 POINT(47.58036920642138 -122.99887154456701) bank105209 +105210 POINT(48.13979797817181 -123.18590897674748) bank105210 +105211 POINT(46.796011074571034 -122.536749677195) bank105211 +105212 POINT(46.90912536249114 -123.27493376083434) bank105212 +105213 POINT(48.426886396723916 -122.81651491350172) bank105213 +105214 POINT(47.62713408659204 -121.91617950958805) bank105214 +105215 POINT(48.05150955623029 -121.48348041085761) bank105215 +105216 POINT(47.87551754139298 -123.13038030719986) bank105216 +105217 POINT(47.08733798508665 -121.82674399658163) bank105217 +105218 POINT(48.31370754786503 -121.75660941066492) bank105218 +105219 POINT(46.64171554846429 -122.45013950332054) bank105219 +105220 POINT(48.02654925687952 -122.6316820651379) bank105220 +105221 POINT(46.69092988660241 -122.0587884510755) bank105221 +105222 POINT(48.36525649114134 -122.79153947802695) bank105222 +105223 POINT(46.66182463559701 -122.46975393715962) bank105223 +105224 POINT(47.89791356137772 -122.12931563502703) bank105224 +105225 POINT(48.03599710875519 -122.84151495549214) bank105225 +105226 POINT(47.7488579919661 -123.29704591342731) bank105226 +105227 POINT(47.08278052103202 -121.83538591894329) bank105227 +105228 POINT(47.06840320730988 -122.09155648529169) bank105228 +105229 POINT(46.96819977360769 -122.79649314693161) bank105229 +105230 POINT(47.37801026136327 -123.14786862118874) bank105230 +105231 POINT(47.44582620262817 -121.81749211318257) bank105231 +105232 POINT(48.473688590975925 -121.96700599781104) bank105232 +105233 POINT(47.54093812188428 -123.01096426789276) bank105233 +105234 POINT(47.37860720552354 -121.96462800501642) bank105234 +105235 POINT(48.221197163553434 -121.75421401952967) bank105235 +105236 POINT(48.06311188275799 -122.02751352789683) bank105236 +105237 POINT(48.15746896333079 -122.36771801120602) bank105237 +105238 POINT(47.99328304900729 -122.54218209595642) bank105238 +105239 POINT(47.86436860134039 -121.59309144822673) bank105239 +105240 POINT(47.06215539299042 -123.28687970718391) bank105240 +105241 POINT(48.026380371102405 -122.15916364446151) bank105241 +105242 POINT(47.87332576380309 -122.63194699746833) bank105242 +105243 POINT(48.240009141896905 -122.48922510851392) bank105243 +105244 POINT(46.632988244561155 -122.4878387642475) bank105244 +105245 POINT(47.98345957804487 -121.66813704966177) bank105245 +105246 POINT(48.426543376547315 -122.14058400332007) bank105246 +105247 POINT(47.632376484696124 -122.97936431822097) bank105247 +105248 POINT(47.07384017757682 -122.93822532592114) bank105248 +105249 POINT(48.36694512898973 -121.85097586722807) bank105249 +105250 POINT(47.40138391257927 -122.44697788899514) bank105250 +105251 POINT(47.95034708805893 -121.7806673104576) bank105251 +105252 POINT(46.80325743053452 -123.1440337452916) bank105252 +105253 POINT(47.86467450215545 -122.20297998023231) bank105253 +105254 POINT(47.2376418627863 -123.04313841679942) bank105254 +105255 POINT(47.42695390932733 -121.95951317932327) bank105255 +105256 POINT(48.46570453517338 -121.86716581641589) bank105256 +105257 POINT(48.241828270723005 -122.63677889138187) bank105257 +105258 POINT(48.16994802395313 -121.9009202753185) bank105258 +105259 POINT(47.73743934349543 -122.93360952629763) bank105259 +105260 POINT(47.73731400342705 -123.02272281603075) bank105260 +105261 POINT(47.87216871562145 -121.5973041574301) bank105261 +105262 POINT(47.58124682169041 -123.16620794863583) bank105262 +105263 POINT(48.59469493758382 -122.61386064570537) bank105263 +105264 POINT(47.22236183932124 -123.16505107397536) bank105264 +105265 POINT(48.288291943394945 -122.56955868857408) bank105265 +105266 POINT(46.93567344320835 -121.46939730901929) bank105266 +105267 POINT(47.00815997578385 -121.97809102515926) bank105267 +105268 POINT(47.25032310231925 -121.64200008678094) bank105268 +105269 POINT(48.28122067200313 -121.53931095132049) bank105269 +105270 POINT(48.057896963577846 -122.33494137719534) bank105270 +105271 POINT(47.300677545148595 -122.73143720946769) bank105271 +105272 POINT(47.84314949657088 -122.67031076753325) bank105272 +105273 POINT(48.196309676572106 -121.93952100958819) bank105273 +105274 POINT(47.08287511956468 -122.3038716027813) bank105274 +105275 POINT(48.537020503120154 -121.51982468833727) bank105275 +105276 POINT(47.44909083855433 -121.4478326993497) bank105276 +105277 POINT(47.41790661347334 -123.26715575943408) bank105277 +105278 POINT(47.17954212775596 -123.14049015957849) bank105278 +105279 POINT(47.37249443206454 -122.09094994662982) bank105279 +105280 POINT(48.4494865344612 -121.60184104442476) bank105280 +105281 POINT(46.87376575039408 -122.4564006102528) bank105281 +105282 POINT(47.02298188400086 -121.57817737043385) bank105282 +105283 POINT(48.48372421336163 -122.96624037082138) bank105283 +105284 POINT(47.40395516145947 -122.65712631571907) bank105284 +105285 POINT(47.565552181202605 -121.94763186487745) bank105285 +105286 POINT(48.01599624081714 -123.07089355970493) bank105286 +105287 POINT(47.05890280372809 -122.87981510682994) bank105287 +105288 POINT(47.62734325037763 -121.51181221599144) bank105288 +105289 POINT(47.446919967348585 -121.47277485875955) bank105289 +105290 POINT(47.54589176820445 -122.95583959114953) bank105290 +105291 POINT(47.55045232640071 -121.73317949573375) bank105291 +105292 POINT(48.56409126540977 -122.9435911448459) bank105292 +105293 POINT(47.990886755688265 -121.68558760728067) bank105293 +105294 POINT(47.973228956175106 -122.15084168921958) bank105294 +105295 POINT(46.94722908688701 -121.51090723871305) bank105295 +105296 POINT(48.46618933205937 -123.31339399781407) bank105296 +105297 POINT(47.011868274972905 -122.49404542951473) bank105297 +105298 POINT(46.641189083477514 -121.37122590548847) bank105298 +105299 POINT(47.76374531109989 -122.21983381590778) bank105299 +105300 POINT(48.369309307954666 -123.22493087376964) bank105300 +105301 POINT(46.84277313417204 -122.84411979548064) bank105301 +105302 POINT(47.199786639296946 -123.22629164746567) bank105302 +105303 POINT(46.919246031974616 -122.18399322340228) bank105303 +105304 POINT(48.42795728562261 -122.08680874574769) bank105304 +105305 POINT(48.458236133996095 -123.27187626957459) bank105305 +105306 POINT(47.808576764578305 -121.55340014071965) bank105306 +105307 POINT(47.3301192636505 -122.78502245897708) bank105307 +105308 POINT(47.642465512811285 -123.07639628514653) bank105308 +105309 POINT(47.60284807800681 -122.82164612352743) bank105309 +105310 POINT(47.391854338680744 -123.16674284301845) bank105310 +105311 POINT(47.22707189648803 -122.78234080641727) bank105311 +105312 POINT(47.206954725406895 -121.55501723841529) bank105312 +105313 POINT(47.425277726022685 -121.6374378646807) bank105313 +105314 POINT(47.597622371275015 -122.86492842392899) bank105314 +105315 POINT(47.852929377777365 -122.55955631375487) bank105315 +105316 POINT(47.02323633720764 -122.96519786829458) bank105316 +105317 POINT(47.906616868644825 -123.20629071848178) bank105317 +105318 POINT(47.342695197968204 -121.71157220935781) bank105318 +105319 POINT(47.38384545866163 -121.66374294838228) bank105319 +105320 POINT(48.496412548896345 -121.55242170840012) bank105320 +105321 POINT(48.06059289354025 -121.94745652798663) bank105321 +105322 POINT(46.91269393810469 -122.14815900325918) bank105322 +105323 POINT(48.33543173616076 -122.1733995893632) bank105323 +105324 POINT(47.8681773769269 -121.75307956118944) bank105324 +105325 POINT(46.641503060419936 -122.2085934993005) bank105325 +105326 POINT(48.11766474267241 -121.9121149703419) bank105326 +105327 POINT(47.74510480958878 -122.25425418443943) bank105327 +105328 POINT(47.39476345385216 -121.95210931520903) bank105328 +105329 POINT(47.53614313717902 -122.63125300961165) bank105329 +105330 POINT(47.48165608535792 -123.23795522345065) bank105330 +105331 POINT(47.42558523422186 -122.2683046081778) bank105331 +105332 POINT(47.10996736162824 -122.72882209747242) bank105332 +105333 POINT(48.15609654974961 -121.43885893224386) bank105333 +105334 POINT(47.74625968791104 -122.68952357733626) bank105334 +105335 POINT(47.17015929477856 -122.0642285244905) bank105335 +105336 POINT(48.01555024071411 -121.87790983080428) bank105336 +105337 POINT(47.40801952363747 -121.94069950013353) bank105337 +105338 POINT(46.88572109297447 -122.03213407801461) bank105338 +105339 POINT(47.12758532447305 -121.69581105106967) bank105339 +105340 POINT(47.053705509220976 -123.20328706690437) bank105340 +105341 POINT(47.17782323293747 -122.61018995015054) bank105341 +105342 POINT(48.37288775904445 -121.94951324322814) bank105342 +105343 POINT(47.995584633893564 -122.91056561435106) bank105343 +105344 POINT(46.943784023847854 -123.24549819845156) bank105344 +105345 POINT(48.4783296273264 -123.07691865947392) bank105345 +105346 POINT(47.80385119471962 -122.0146608363606) bank105346 +105347 POINT(48.056078262772374 -121.9194904947131) bank105347 +105348 POINT(47.350040160656754 -121.42586533915124) bank105348 +105349 POINT(47.13478534799652 -122.75034851578727) bank105349 +105350 POINT(47.20518737985448 -122.02365100662678) bank105350 +105351 POINT(47.937428411017265 -121.98080250567254) bank105351 +105352 POINT(47.70482631589195 -123.01808003931477) bank105352 +105353 POINT(47.61793716897038 -121.39008301390113) bank105353 +105354 POINT(46.74002029070813 -122.80305042973667) bank105354 +105355 POINT(47.05026758817249 -121.54034828340195) bank105355 +105356 POINT(46.87461246265784 -121.39548664441848) bank105356 +105357 POINT(47.01885468662207 -122.2640307979829) bank105357 +105358 POINT(48.242339022961744 -123.25904298114366) bank105358 +105359 POINT(48.04253425398558 -122.52828874406346) bank105359 +105360 POINT(48.49722032039005 -123.28139003509219) bank105360 +105361 POINT(47.11710381234331 -123.1905322154056) bank105361 +105362 POINT(47.14124464707271 -122.49797696041355) bank105362 +105363 POINT(47.95798616377095 -123.29642266783303) bank105363 +105364 POINT(47.416454677831496 -122.07995887699444) bank105364 +105365 POINT(47.46521345600301 -121.82630658581397) bank105365 +105366 POINT(47.95402362180452 -121.66320308423066) bank105366 +105367 POINT(47.816125020059296 -123.0612288573813) bank105367 +105368 POINT(47.17976980114494 -122.2248822167678) bank105368 +105369 POINT(46.792698814260575 -122.78962775158662) bank105369 +105370 POINT(47.0199201307442 -122.12586318252447) bank105370 +105371 POINT(47.84246852389004 -122.64773905165917) bank105371 +105372 POINT(47.216502269396244 -122.70885380102983) bank105372 +105373 POINT(48.238687578972616 -122.61444441186046) bank105373 +105374 POINT(47.59278253467088 -122.30173769089821) bank105374 +105375 POINT(47.849387732715414 -123.14819574433655) bank105375 +105376 POINT(48.601127358572896 -123.09954179909134) bank105376 +105377 POINT(47.03818771377185 -121.41595515285238) bank105377 +105378 POINT(47.443259729392594 -122.65602342415178) bank105378 +105379 POINT(47.14676076988395 -121.6600331770055) bank105379 +105380 POINT(47.938804288365674 -122.82659594136415) bank105380 +105381 POINT(46.693006723424354 -123.25819972503757) bank105381 +105382 POINT(48.40198647644237 -122.6467113330263) bank105382 +105383 POINT(47.61270919604435 -121.33466896749175) bank105383 +105384 POINT(46.91543974426242 -121.60306492170648) bank105384 +105385 POINT(47.596119492887105 -122.44117604147462) bank105385 +105386 POINT(46.74496654682203 -122.59958957814013) bank105386 +105387 POINT(47.98479405150157 -123.18722978152192) bank105387 +105388 POINT(46.888559530107756 -122.4593462505881) bank105388 +105389 POINT(47.28850896867496 -122.25025486445912) bank105389 +105390 POINT(47.85606171978928 -123.20007090576007) bank105390 +105391 POINT(47.886752524355224 -122.52208040542195) bank105391 +105392 POINT(46.85863105358807 -122.25607007834128) bank105392 +105393 POINT(47.17136070902955 -123.33078564123181) bank105393 +105394 POINT(47.82205746283414 -121.68908160683841) bank105394 +105395 POINT(48.599878595768025 -122.46560027357113) bank105395 +105396 POINT(47.52337362135936 -121.99892716820501) bank105396 +105397 POINT(48.42036418943096 -122.56852432882708) bank105397 +105398 POINT(47.29829795274157 -123.25963831739209) bank105398 +105399 POINT(47.19289789924024 -122.77280910553425) bank105399 +105400 POINT(46.74417487172722 -121.83039940460718) bank105400 +105401 POINT(47.383742478793124 -122.18545684578098) bank105401 +105402 POINT(47.34655211359679 -122.92354205928163) bank105402 +105403 POINT(46.87987788756712 -122.23907226471198) bank105403 +105404 POINT(48.4284489923236 -121.97750117482897) bank105404 +105405 POINT(47.33740538312882 -122.68260644287943) bank105405 +105406 POINT(47.40287302337064 -122.63949661064652) bank105406 +105407 POINT(46.620125224947635 -122.19598655878441) bank105407 +105408 POINT(47.39819827257593 -123.04847628216523) bank105408 +105409 POINT(48.500465673134435 -122.23715955807815) bank105409 +105410 POINT(48.41904269299348 -121.5518089257234) bank105410 +105411 POINT(47.18797170739958 -122.39698967064882) bank105411 +105412 POINT(47.22940802578946 -122.34945874616466) bank105412 +105413 POINT(47.56351199913597 -121.65389997653361) bank105413 +105414 POINT(47.41915028244854 -121.50618014897903) bank105414 +105415 POINT(47.28547921259892 -123.30578437511171) bank105415 +105416 POINT(47.65382513528469 -121.97687959234908) bank105416 +105417 POINT(48.11072353345369 -122.90476506423468) bank105417 +105418 POINT(46.94634780110606 -122.32557263476177) bank105418 +105419 POINT(47.060439958074646 -121.7589596014438) bank105419 +105420 POINT(47.54167124964147 -122.5388431066303) bank105420 +105421 POINT(47.87272912447331 -122.25562369566524) bank105421 +105422 POINT(47.84753367183781 -123.0268195761191) bank105422 +105423 POINT(48.11298032389281 -121.974269419881) bank105423 +105424 POINT(47.331934998906526 -121.71480554270643) bank105424 +105425 POINT(47.94604675403648 -123.07366642781795) bank105425 +105426 POINT(47.395308392160764 -121.87367642616496) bank105426 +105427 POINT(47.93508317258266 -121.57183778117634) bank105427 +105428 POINT(47.2746470614076 -122.96878400894809) bank105428 +105429 POINT(47.14083369667934 -123.04028397390275) bank105429 +105430 POINT(47.93739688242399 -122.89226891905786) bank105430 +105431 POINT(47.29719231471011 -123.01004295650792) bank105431 +105432 POINT(48.15836188912838 -121.79488139134517) bank105432 +105433 POINT(48.16970337996813 -121.51488687581546) bank105433 +105434 POINT(48.377984101429206 -122.7384479323191) bank105434 +105435 POINT(47.37371949807629 -122.37632176826743) bank105435 +105436 POINT(47.029272248853374 -122.4251730226664) bank105436 +105437 POINT(47.20887578460521 -121.40606711643085) bank105437 +105438 POINT(47.345839973536094 -122.63133087210338) bank105438 +105439 POINT(48.315299447046534 -123.18938107368895) bank105439 +105440 POINT(46.93962721697297 -121.56328163208511) bank105440 +105441 POINT(47.820869702402746 -121.83798221373574) bank105441 +105442 POINT(47.00183623086422 -121.72241450187647) bank105442 +105443 POINT(47.29792050928065 -121.85709897094193) bank105443 +105444 POINT(46.7341060186708 -123.18845241124262) bank105444 +105445 POINT(47.57379375572863 -121.55800229521269) bank105445 +105446 POINT(46.689485933547864 -122.47474880427066) bank105446 +105447 POINT(46.847105908066915 -123.18545219302106) bank105447 +105448 POINT(46.92222036514116 -121.7571667443973) bank105448 +105449 POINT(48.07367303106539 -121.92684460573183) bank105449 +105450 POINT(47.21244461238009 -121.55730289848059) bank105450 +105451 POINT(47.344644533691785 -123.29316564388668) bank105451 +105452 POINT(47.527477469159926 -123.09921782027125) bank105452 +105453 POINT(47.90528371854557 -122.32740768642827) bank105453 +105454 POINT(46.99786427121633 -122.85168708660215) bank105454 +105455 POINT(48.006174954136064 -122.32356270560777) bank105455 +105456 POINT(46.66321270142316 -122.76159597037021) bank105456 +105457 POINT(47.87840761566538 -121.72427848023517) bank105457 +105458 POINT(46.912421657675246 -121.84040180720754) bank105458 +105459 POINT(48.50539614839787 -122.2760101672344) bank105459 +105460 POINT(48.314715400725426 -121.74872009975397) bank105460 +105461 POINT(48.4511846116027 -121.95242510398582) bank105461 +105462 POINT(47.24311072648567 -122.4647241182791) bank105462 +105463 POINT(47.273912429533745 -121.48303672797421) bank105463 +105464 POINT(48.213929902075854 -121.65810477842487) bank105464 +105465 POINT(46.79575099661394 -122.70027519332115) bank105465 +105466 POINT(48.09416822916293 -123.12313849527813) bank105466 +105467 POINT(47.3262446550772 -123.14303047702832) bank105467 +105468 POINT(47.76009121640323 -122.00686556934774) bank105468 +105469 POINT(48.51874618067148 -122.2368973299679) bank105469 +105470 POINT(47.77502734280695 -121.77787264740155) bank105470 +105471 POINT(46.85092476286016 -121.62010083467797) bank105471 +105472 POINT(47.36280445498358 -121.92546943645999) bank105472 +105473 POINT(47.31119415033496 -122.57384796576504) bank105473 +105474 POINT(46.684843234456686 -122.95363305878) bank105474 +105475 POINT(47.31295832290127 -121.83101824943992) bank105475 +105476 POINT(47.85608148111057 -122.42153345697274) bank105476 +105477 POINT(47.36440419885967 -122.02034553943021) bank105477 +105478 POINT(48.29008949275437 -121.9452063291808) bank105478 +105479 POINT(46.60839138407709 -122.61497532214666) bank105479 +105480 POINT(46.880548220435685 -122.21895814130426) bank105480 +105481 POINT(46.82127783399607 -123.00909385723557) bank105481 +105482 POINT(47.928162776671016 -123.27281962010868) bank105482 +105483 POINT(47.647887048676246 -122.86279090578928) bank105483 +105484 POINT(46.76393650369543 -123.32534325085652) bank105484 +105485 POINT(47.92080542534191 -121.53169136627562) bank105485 +105486 POINT(47.73600057934043 -122.19230080728966) bank105486 +105487 POINT(48.56690452780522 -122.97955001995837) bank105487 +105488 POINT(48.482875671450095 -122.2304736469495) bank105488 +105489 POINT(48.35378729160925 -122.36874462695334) bank105489 +105490 POINT(46.80036647060804 -121.88385692255318) bank105490 +105491 POINT(47.26227423262074 -121.97269514068965) bank105491 +105492 POINT(47.82332357475351 -122.83157826230257) bank105492 +105493 POINT(48.58477496907179 -123.20539939950542) bank105493 +105494 POINT(48.21293409999894 -121.75249942529634) bank105494 +105495 POINT(47.5746901484352 -122.51325723515441) bank105495 +105496 POINT(46.771729801691876 -122.70187281446127) bank105496 +105497 POINT(47.434200084101406 -122.63138898461219) bank105497 +105498 POINT(47.15395406494309 -121.62014518196375) bank105498 +105499 POINT(47.62622749338448 -122.22306950823375) bank105499 +105500 POINT(47.386663742397616 -121.41853770608482) bank105500 +105501 POINT(46.898592070086906 -123.0567142456394) bank105501 +105502 POINT(47.42637945537967 -122.95370465671147) bank105502 +105503 POINT(46.685384700038334 -122.31846800743159) bank105503 +105504 POINT(48.121429461576305 -122.33819131223055) bank105504 +105505 POINT(48.28182783661867 -123.30076530472755) bank105505 +105506 POINT(47.4328298566561 -121.54352764094635) bank105506 +105507 POINT(47.01891270479329 -122.69244887029238) bank105507 +105508 POINT(48.46610777520645 -121.61248762315051) bank105508 +105509 POINT(47.884566819700396 -122.60157455674494) bank105509 +105510 POINT(47.389367027728156 -122.97752453474777) bank105510 +105511 POINT(47.54234725628468 -122.28615371581529) bank105511 +105512 POINT(48.441680834807435 -123.23925523879028) bank105512 +105513 POINT(48.42267391031966 -122.09027251957238) bank105513 +105514 POINT(47.64370190335114 -122.77290717353395) bank105514 +105515 POINT(48.259370338981384 -122.12013474131031) bank105515 +105516 POINT(47.8129847078339 -122.10743344463212) bank105516 +105517 POINT(47.39349579793382 -122.39629861062372) bank105517 +105518 POINT(47.986764307832 -123.18639576702601) bank105518 +105519 POINT(46.860589519338376 -123.2258414161508) bank105519 +105520 POINT(48.09169545994694 -121.922237979892) bank105520 +105521 POINT(47.216698740937716 -122.88007084644862) bank105521 +105522 POINT(47.72207468574539 -122.87866155365586) bank105522 +105523 POINT(46.622807649727605 -122.29290944447109) bank105523 +105524 POINT(47.34927686960807 -121.74695343977481) bank105524 +105525 POINT(47.549818955175766 -121.33272953998667) bank105525 +105526 POINT(47.69581916227081 -121.91092251360811) bank105526 +105527 POINT(47.02406538565059 -122.11283056588445) bank105527 +105528 POINT(47.19205834018622 -123.006373006053) bank105528 +105529 POINT(48.41258435057109 -122.47830591089608) bank105529 +105530 POINT(47.011690615012334 -122.46685990377244) bank105530 +105531 POINT(47.358211732848346 -122.3442257361354) bank105531 +105532 POINT(46.92189809250694 -122.76547225105881) bank105532 +105533 POINT(48.54225062283076 -122.95829239672514) bank105533 +105534 POINT(47.90801429018459 -122.14063248666824) bank105534 +105535 POINT(47.28129823782386 -121.95211320724579) bank105535 +105536 POINT(48.13214580541242 -121.69507234754043) bank105536 +105537 POINT(46.606536633630625 -121.94950304418371) bank105537 +105538 POINT(48.506169284173154 -122.45096479319606) bank105538 +105539 POINT(47.8780786880081 -122.125832617291) bank105539 +105540 POINT(48.173791708789196 -121.65967954312252) bank105540 +105541 POINT(47.08216064605336 -122.19939685623726) bank105541 +105542 POINT(48.41260849550038 -121.5295348075841) bank105542 +105543 POINT(47.637012969670295 -122.07192779971525) bank105543 +105544 POINT(47.82582764683658 -121.97130236954557) bank105544 +105545 POINT(47.209153716914315 -121.34858794244418) bank105545 +105546 POINT(48.57213060342938 -121.94937157837516) bank105546 +105547 POINT(47.50013680399329 -122.63836969415476) bank105547 +105548 POINT(46.90959001349555 -121.5840283007392) bank105548 +105549 POINT(47.52983298360651 -121.88453706516195) bank105549 +105550 POINT(48.04615810611091 -122.8469838264646) bank105550 +105551 POINT(47.49234394638505 -122.22694180735151) bank105551 +105552 POINT(47.47509347859746 -123.09134262913499) bank105552 +105553 POINT(48.26856142231694 -123.02528537806856) bank105553 +105554 POINT(48.41515551220674 -123.29649293837568) bank105554 +105555 POINT(48.349822625997575 -122.05929638175665) bank105555 +105556 POINT(48.09434443297696 -122.25051916479754) bank105556 +105557 POINT(46.824050584627805 -121.86851893272316) bank105557 +105558 POINT(46.903201409766076 -121.93236515913419) bank105558 +105559 POINT(47.17716338295355 -122.52817018563218) bank105559 +105560 POINT(47.56526740154477 -123.31460306613185) bank105560 +105561 POINT(47.40666946420701 -122.40073466715788) bank105561 +105562 POINT(48.52857143863427 -122.95231337129623) bank105562 +105563 POINT(46.901863539152835 -121.45194800045843) bank105563 +105564 POINT(47.43330280488474 -123.19216262847334) bank105564 +105565 POINT(47.86164601749002 -122.59608053050461) bank105565 +105566 POINT(47.0744980503749 -121.46348878685473) bank105566 +105567 POINT(48.5778087706505 -122.27496449732432) bank105567 +105568 POINT(46.792344716878404 -122.40276022870908) bank105568 +105569 POINT(47.28263372176902 -123.19393162916062) bank105569 +105570 POINT(46.783384430628516 -123.2025567202548) bank105570 +105571 POINT(47.30110267642046 -121.81166916695989) bank105571 +105572 POINT(48.22531293950695 -123.31251623813966) bank105572 +105573 POINT(46.903140821895626 -122.71359708359965) bank105573 +105574 POINT(47.83893766086977 -121.48392719593218) bank105574 +105575 POINT(47.428484144738206 -121.57480975105577) bank105575 +105576 POINT(47.09759578903018 -122.09957713542168) bank105576 +105577 POINT(46.931068059909734 -122.73381998935356) bank105577 +105578 POINT(48.26299815477196 -122.07709997004828) bank105578 +105579 POINT(47.16946272034306 -123.10178469754611) bank105579 +105580 POINT(48.06982155246925 -121.62585327036875) bank105580 +105581 POINT(48.290673696766255 -121.91177477620312) bank105581 +105582 POINT(47.80900999267357 -121.75625648581459) bank105582 +105583 POINT(47.18953000242253 -123.12265548373254) bank105583 +105584 POINT(47.37169456072907 -122.4287250437606) bank105584 +105585 POINT(47.88668763763553 -122.8537486217273) bank105585 +105586 POINT(47.53322483123113 -122.79698474110687) bank105586 +105587 POINT(47.09284267274344 -121.77752821882176) bank105587 +105588 POINT(46.613072433101095 -122.99769419401676) bank105588 +105589 POINT(47.442917083956594 -122.5361195788351) bank105589 +105590 POINT(47.692752420601195 -123.29341146296058) bank105590 +105591 POINT(47.66932655439835 -122.42128236374556) bank105591 +105592 POINT(47.55796871142825 -122.39487593157335) bank105592 +105593 POINT(47.04553134497198 -121.50812889439159) bank105593 +105594 POINT(47.413546760684355 -121.99760598159088) bank105594 +105595 POINT(48.24249570808165 -122.49809165755383) bank105595 +105596 POINT(47.514305897482124 -123.28799720185381) bank105596 +105597 POINT(47.5367069386555 -123.31227247876674) bank105597 +105598 POINT(47.07358928088893 -123.27500722378372) bank105598 +105599 POINT(47.63001430901896 -121.57260991241155) bank105599 +105600 POINT(47.66555599992915 -122.71750609930878) bank105600 +105601 POINT(48.35512161441573 -121.7133686764842) bank105601 +105602 POINT(47.76841648754885 -121.48228023121357) bank105602 +105603 POINT(47.08613176123562 -122.36304594588375) bank105603 +105604 POINT(48.539900178905604 -122.30470137960036) bank105604 +105605 POINT(47.43629310147815 -121.8567256079576) bank105605 +105606 POINT(46.678947331409425 -122.21900176647857) bank105606 +105607 POINT(48.0853471624104 -122.35870313440924) bank105607 +105608 POINT(47.99426034364447 -122.73120882765723) bank105608 +105609 POINT(48.373136883581246 -122.34964366836749) bank105609 +105610 POINT(47.17201810143062 -122.32479015451173) bank105610 +105611 POINT(48.01177627141086 -122.42769125652663) bank105611 +105612 POINT(46.75332743387348 -121.33758436776499) bank105612 +105613 POINT(48.12955763986482 -121.6999233591303) bank105613 +105614 POINT(48.100053699676394 -123.02950490397915) bank105614 +105615 POINT(46.95796360333743 -122.6662559514705) bank105615 +105616 POINT(47.672342980379156 -122.29771639139217) bank105616 +105617 POINT(46.64129884862121 -122.46054214801025) bank105617 +105618 POINT(48.156496645269 -122.43992589915418) bank105618 +105619 POINT(46.74805738471398 -122.10151762062681) bank105619 +105620 POINT(46.72970309451661 -121.87629736356689) bank105620 +105621 POINT(46.992404409492735 -121.96947579286001) bank105621 +105622 POINT(46.75276512695949 -122.09096127539691) bank105622 +105623 POINT(47.76914880590884 -123.25682908190196) bank105623 +105624 POINT(46.892467031323775 -121.58729629209128) bank105624 +105625 POINT(47.62009762242463 -122.7843084108401) bank105625 +105626 POINT(48.36918518116723 -123.19481046108206) bank105626 +105627 POINT(47.159550377385 -121.41159455259412) bank105627 +105628 POINT(46.84130316933992 -121.63297980703953) bank105628 +105629 POINT(46.84485921072453 -122.79862926546565) bank105629 +105630 POINT(48.098655331872685 -123.13380799976808) bank105630 +105631 POINT(47.38496128758941 -122.51291967166071) bank105631 +105632 POINT(47.12728518622426 -123.20238813969372) bank105632 +105633 POINT(47.05093547603924 -121.7534653229549) bank105633 +105634 POINT(48.22749213958969 -122.23630097818967) bank105634 +105635 POINT(47.85400255851008 -122.99768765461566) bank105635 +105636 POINT(48.00329449149634 -122.71792413530653) bank105636 +105637 POINT(48.09536528753553 -122.33214489629748) bank105637 +105638 POINT(46.92551934990457 -122.25776592765457) bank105638 +105639 POINT(48.45972146438902 -122.76364246942349) bank105639 +105640 POINT(47.26302022468737 -122.19719971653285) bank105640 +105641 POINT(46.734409669659264 -121.87367336368064) bank105641 +105642 POINT(47.01182673390632 -122.94022365494551) bank105642 +105643 POINT(46.834199983783165 -122.58547860738733) bank105643 +105644 POINT(46.762293704288425 -122.7238336288996) bank105644 +105645 POINT(47.850355679303604 -121.65358097222749) bank105645 +105646 POINT(47.34765198955994 -121.34576753977129) bank105646 +105647 POINT(47.60036514827581 -122.19947289432352) bank105647 +105648 POINT(46.86242761294088 -121.79696052369428) bank105648 +105649 POINT(47.40654790519688 -122.00497684665143) bank105649 +105650 POINT(47.353603401620255 -122.91881723751118) bank105650 +105651 POINT(47.70806771675483 -121.79770440376576) bank105651 +105652 POINT(47.32446892401405 -122.21703958858245) bank105652 +105653 POINT(47.60235228557551 -122.52233801094799) bank105653 +105654 POINT(47.6889686311826 -122.59501620493924) bank105654 +105655 POINT(48.21818628277951 -123.1570132705649) bank105655 +105656 POINT(46.700115423122824 -123.31142692014956) bank105656 +105657 POINT(47.70509831678962 -122.28131437738821) bank105657 +105658 POINT(47.75882229128693 -121.8201674838909) bank105658 +105659 POINT(47.27534547556145 -121.91140609646574) bank105659 +105660 POINT(47.02052493316765 -122.08243332089359) bank105660 +105661 POINT(48.42572613203362 -121.78836907054756) bank105661 +105662 POINT(47.40649107473767 -122.63692597398352) bank105662 +105663 POINT(47.18789654119181 -122.71706332193833) bank105663 +105664 POINT(47.543902712895076 -121.35073400563097) bank105664 +105665 POINT(46.697048119385386 -122.76226145789683) bank105665 +105666 POINT(47.994298559365056 -121.98411343828444) bank105666 +105667 POINT(47.760313778619704 -121.87368625887768) bank105667 +105668 POINT(47.00053149478887 -121.90198548007864) bank105668 +105669 POINT(46.995223558348016 -122.44436033720241) bank105669 +105670 POINT(47.111203993524306 -122.9584508087164) bank105670 +105671 POINT(47.07436826771599 -122.64132636305281) bank105671 +105672 POINT(48.476132372416 -122.26494015043802) bank105672 +105673 POINT(48.239283605861296 -122.15261499392965) bank105673 +105674 POINT(47.46314640293226 -123.10462763895703) bank105674 +105675 POINT(47.10159118182342 -122.47989676295167) bank105675 +105676 POINT(47.61925282961964 -122.04453345219824) bank105676 +105677 POINT(46.97693129157941 -122.10228754813672) bank105677 +105678 POINT(46.83443194614099 -122.15277004857774) bank105678 +105679 POINT(46.887392493500634 -121.49433213639288) bank105679 +105680 POINT(48.50042542994207 -121.99839238823719) bank105680 +105681 POINT(47.97155065931279 -121.55858355786033) bank105681 +105682 POINT(47.554745281269206 -122.49711907862803) bank105682 +105683 POINT(46.78204477179253 -123.31924641472001) bank105683 +105684 POINT(47.42793668650384 -122.14936916355424) bank105684 +105685 POINT(47.96404098629271 -121.54720033994029) bank105685 +105686 POINT(46.87008803275632 -121.81738557735575) bank105686 +105687 POINT(48.193349015723726 -122.38613828636517) bank105687 +105688 POINT(47.90063252685057 -122.89918345430084) bank105688 +105689 POINT(46.88437153032557 -122.79364796669404) bank105689 +105690 POINT(47.11380824553298 -122.7413752680008) bank105690 +105691 POINT(47.26263828695476 -121.88025509048637) bank105691 +105692 POINT(47.426196354540245 -121.81113935550151) bank105692 +105693 POINT(47.5972530758306 -121.79043710848335) bank105693 +105694 POINT(46.805611691540626 -121.55328878198205) bank105694 +105695 POINT(47.27695770621281 -122.48201602121307) bank105695 +105696 POINT(48.216128790109956 -121.63879295805425) bank105696 +105697 POINT(47.13174920771228 -122.3542566695856) bank105697 +105698 POINT(47.46615458514484 -122.08953899517785) bank105698 +105699 POINT(46.73003284343872 -123.12471258863881) bank105699 +105700 POINT(47.144535379701075 -122.51467950725029) bank105700 +105701 POINT(47.14913574148813 -121.37422806876828) bank105701 +105702 POINT(47.95722639410594 -121.35936331656781) bank105702 +105703 POINT(47.42003568759053 -121.98991317778365) bank105703 +105704 POINT(46.84779754740391 -123.18058380896024) bank105704 +105705 POINT(48.541473058371494 -122.59288487360024) bank105705 +105706 POINT(47.42187900640446 -121.93092960915244) bank105706 +105707 POINT(48.34288539984486 -123.19402100159306) bank105707 +105708 POINT(47.13521738163103 -122.38520370504806) bank105708 +105709 POINT(48.57746059672034 -121.58481282330516) bank105709 +105710 POINT(47.306316170276254 -122.17886755260258) bank105710 +105711 POINT(47.42444582274991 -123.28595053080753) bank105711 +105712 POINT(47.04450100963993 -122.81175855425383) bank105712 +105713 POINT(48.253352883975516 -122.39236567630198) bank105713 +105714 POINT(47.48350135286473 -122.13868570810199) bank105714 +105715 POINT(47.9572992028039 -121.90972658458703) bank105715 +105716 POINT(48.434164461328244 -122.2719867766515) bank105716 +105717 POINT(47.55842658926326 -121.6132306534267) bank105717 +105718 POINT(47.901215209492975 -123.13345373509733) bank105718 +105719 POINT(48.228013833626456 -122.31437825346269) bank105719 +105720 POINT(48.514020713002935 -122.11028459033967) bank105720 +105721 POINT(46.99218859896386 -122.85222327379869) bank105721 +105722 POINT(46.92126089283026 -122.37219988674259) bank105722 +105723 POINT(47.91329059277889 -121.77483895159105) bank105723 +105724 POINT(47.213116718348466 -121.37009526602712) bank105724 +105725 POINT(48.16507607157501 -122.5063562727627) bank105725 +105726 POINT(46.64471339082274 -123.24701745767841) bank105726 +105727 POINT(46.91104081855461 -122.3081918480766) bank105727 +105728 POINT(46.94474030540367 -121.6653319481013) bank105728 +105729 POINT(47.43396940568068 -122.55383290875886) bank105729 +105730 POINT(47.260822148185866 -122.30078665039558) bank105730 +105731 POINT(46.82987101235906 -122.60588508321918) bank105731 +105732 POINT(47.60923485424538 -122.58470553012454) bank105732 +105733 POINT(48.26717476007857 -121.92226164016444) bank105733 +105734 POINT(48.44226676792764 -121.46926603609694) bank105734 +105735 POINT(47.00690636405095 -122.10097561433125) bank105735 +105736 POINT(48.199547389299866 -122.6521684071165) bank105736 +105737 POINT(47.89920894921436 -121.74018990312678) bank105737 +105738 POINT(46.8234368463906 -121.92325334806493) bank105738 +105739 POINT(47.52848199767777 -122.53809613998304) bank105739 +105740 POINT(46.82971553648535 -121.6798640800213) bank105740 +105741 POINT(47.159412011237436 -122.65674952256356) bank105741 +105742 POINT(47.21196528352347 -122.84280537943982) bank105742 +105743 POINT(48.28163745284665 -123.04510710609597) bank105743 +105744 POINT(46.69224745782149 -122.92645941269983) bank105744 +105745 POINT(48.208864760983836 -122.08246173593345) bank105745 +105746 POINT(48.06264153071764 -121.99263934447985) bank105746 +105747 POINT(47.69122210116319 -122.49361436568387) bank105747 +105748 POINT(47.739024236712964 -122.25146625926207) bank105748 +105749 POINT(47.43425949938802 -121.61496596726246) bank105749 +105750 POINT(48.381691487975345 -121.89203421794099) bank105750 +105751 POINT(47.607384614277656 -122.46095152426786) bank105751 +105752 POINT(47.575645121795844 -122.72160870713428) bank105752 +105753 POINT(47.55344192336432 -121.96525364892769) bank105753 +105754 POINT(46.92940581979912 -121.7337389392716) bank105754 +105755 POINT(47.91723000626978 -121.90898393648362) bank105755 +105756 POINT(47.83355303864243 -121.3490330837586) bank105756 +105757 POINT(47.08003776786313 -122.05588165850985) bank105757 +105758 POINT(48.02608049528424 -121.70236577818245) bank105758 +105759 POINT(46.93963582322594 -121.4335372981594) bank105759 +105760 POINT(46.798754267242316 -122.2849607617096) bank105760 +105761 POINT(47.874943676601475 -122.41483761562007) bank105761 +105762 POINT(47.72890396382977 -121.62664052808918) bank105762 +105763 POINT(47.4179810101537 -121.39644255661217) bank105763 +105764 POINT(47.217472106911025 -121.70303011164019) bank105764 +105765 POINT(47.848147776292954 -122.11417230628847) bank105765 +105766 POINT(47.62263710309404 -121.48386314749133) bank105766 +105767 POINT(48.47007013839838 -123.21988608777491) bank105767 +105768 POINT(47.84423466621321 -121.64473751827518) bank105768 +105769 POINT(48.4514551919118 -121.4926423070695) bank105769 +105770 POINT(48.46389645069411 -121.99936370020062) bank105770 +105771 POINT(47.00868557041363 -123.25833347511383) bank105771 +105772 POINT(46.86694842857663 -121.93696158978713) bank105772 +105773 POINT(48.25884280703748 -122.50243203555479) bank105773 +105774 POINT(47.831171759755264 -122.24108591460302) bank105774 +105775 POINT(46.675462368552694 -123.29298768352659) bank105775 +105776 POINT(47.313969715406785 -122.91968499202457) bank105776 +105777 POINT(47.660118652846705 -122.96626243904129) bank105777 +105778 POINT(48.09999469325368 -122.43010254457928) bank105778 +105779 POINT(48.53710861161455 -121.67446551889815) bank105779 +105780 POINT(47.79003867989045 -123.13224153138212) bank105780 +105781 POINT(46.82303115069544 -122.25170667617097) bank105781 +105782 POINT(46.99518217464666 -123.3199919363441) bank105782 +105783 POINT(46.83651202603747 -121.813720118194) bank105783 +105784 POINT(48.501220407325654 -123.05014975901572) bank105784 +105785 POINT(47.935109207905015 -122.27522893208678) bank105785 +105786 POINT(47.5015537051111 -121.51190476080565) bank105786 +105787 POINT(47.40758990407948 -122.16776739238554) bank105787 +105788 POINT(48.16836281571184 -122.17086258833578) bank105788 +105789 POINT(47.75912276431418 -121.78777732686548) bank105789 +105790 POINT(48.16278166791902 -121.78884859512671) bank105790 +105791 POINT(48.25223492617366 -121.40176757799576) bank105791 +105792 POINT(48.23890916881688 -121.46464337667068) bank105792 +105793 POINT(47.5045636896154 -122.80170421796626) bank105793 +105794 POINT(46.91800932428035 -122.74293775385047) bank105794 +105795 POINT(48.100441425310535 -121.38345478290455) bank105795 +105796 POINT(47.42295706548815 -122.38942790192782) bank105796 +105797 POINT(48.604520349252944 -122.29605806363735) bank105797 +105798 POINT(46.79276022514025 -122.30363804237008) bank105798 +105799 POINT(47.655485989284756 -122.35846044602623) bank105799 +105800 POINT(48.2162844935557 -121.6655262116937) bank105800 +105801 POINT(46.972665941208696 -122.13021415288178) bank105801 +105802 POINT(47.82020880509269 -121.55288976853612) bank105802 +105803 POINT(47.16756447548322 -122.99115259076555) bank105803 +105804 POINT(46.7252290902682 -122.53766618693126) bank105804 +105805 POINT(47.535041911404136 -122.65808136518778) bank105805 +105806 POINT(47.60292615954003 -122.65464485721793) bank105806 +105807 POINT(46.89277345061864 -123.05321197370445) bank105807 +105808 POINT(46.9471179646459 -122.7868162207714) bank105808 +105809 POINT(46.876377764234924 -121.4629727955674) bank105809 +105810 POINT(46.890463050888364 -122.963191524225) bank105810 +105811 POINT(47.108033242750906 -121.8570922399711) bank105811 +105812 POINT(48.133795604831995 -122.62861430798473) bank105812 +105813 POINT(47.82959591935913 -122.39700308422165) bank105813 +105814 POINT(47.42116222873605 -122.99105753249904) bank105814 +105815 POINT(48.42998271467925 -123.03664924978293) bank105815 +105816 POINT(47.3355640949293 -121.44660609058283) bank105816 +105817 POINT(48.197269797724 -122.58347569260245) bank105817 +105818 POINT(47.284659619847744 -122.96437680551614) bank105818 +105819 POINT(46.81784929505865 -123.27124896781648) bank105819 +105820 POINT(48.003532289933204 -122.12146500989218) bank105820 +105821 POINT(48.52034142402336 -121.65797281384113) bank105821 +105822 POINT(48.456972897662645 -121.65871800305356) bank105822 +105823 POINT(47.9610198879573 -122.86613744284313) bank105823 +105824 POINT(48.31845902065851 -122.83893368522848) bank105824 +105825 POINT(47.306904723431494 -122.98250970249505) bank105825 +105826 POINT(47.62942757196931 -122.58785214767636) bank105826 +105827 POINT(48.46212353967762 -123.05650792294999) bank105827 +105828 POINT(48.40174673624445 -121.33964350298272) bank105828 +105829 POINT(48.11445010103843 -122.5856992864282) bank105829 +105830 POINT(47.829832960467286 -121.40611877997081) bank105830 +105831 POINT(48.190489951816474 -123.15871080507904) bank105831 +105832 POINT(47.50182940482669 -121.56327934473903) bank105832 +105833 POINT(46.91545285459972 -123.2451557422561) bank105833 +105834 POINT(48.23654806109219 -122.26395682483455) bank105834 +105835 POINT(48.42360388723604 -121.50912962753746) bank105835 +105836 POINT(47.78229100715458 -123.25940869967346) bank105836 +105837 POINT(47.25300450779626 -121.35341960229758) bank105837 +105838 POINT(48.5431167275071 -122.73302384921337) bank105838 +105839 POINT(47.538961409139624 -121.55831303871399) bank105839 +105840 POINT(47.538279333327075 -122.20043917926533) bank105840 +105841 POINT(47.16380428080478 -121.913554318558) bank105841 +105842 POINT(47.030143866586556 -123.21758535732116) bank105842 +105843 POINT(48.55244671627158 -121.77292827758829) bank105843 +105844 POINT(47.838455885927765 -122.2404306310938) bank105844 +105845 POINT(46.97450002073185 -123.25764598514957) bank105845 +105846 POINT(46.77802211693849 -121.54326945385398) bank105846 +105847 POINT(46.78231318406743 -121.65078287589122) bank105847 +105848 POINT(46.643650216213196 -122.99341218738674) bank105848 +105849 POINT(47.16648419700161 -123.03528048016508) bank105849 +105850 POINT(48.21786469895506 -121.42325700067055) bank105850 +105851 POINT(48.33217853626702 -122.39672486287469) bank105851 +105852 POINT(46.92358633536586 -122.87674427728982) bank105852 +105853 POINT(48.50496128398626 -122.14896047629152) bank105853 +105854 POINT(47.91920760310451 -121.35872042222545) bank105854 +105855 POINT(48.30999256673052 -121.4017660380787) bank105855 +105856 POINT(48.56281141691274 -123.33155985657666) bank105856 +105857 POINT(48.202512995175944 -121.407836593863) bank105857 +105858 POINT(47.79357770337892 -122.50324941135734) bank105858 +105859 POINT(46.920432721911126 -121.75828793264107) bank105859 +105860 POINT(46.75102597332951 -121.43040624065256) bank105860 +105861 POINT(46.90586064926915 -122.07379840723148) bank105861 +105862 POINT(47.24636321165917 -123.05979693160526) bank105862 +105863 POINT(48.14604461724175 -122.08030170716515) bank105863 +105864 POINT(47.67124315071679 -122.52328573070895) bank105864 +105865 POINT(46.644413018265055 -122.4898692418454) bank105865 +105866 POINT(47.41646358027318 -121.36474450522122) bank105866 +105867 POINT(47.8546665009046 -121.54802444218153) bank105867 +105868 POINT(47.02751748423604 -122.48844784460148) bank105868 +105869 POINT(48.50341197485082 -122.65457802164218) bank105869 +105870 POINT(48.10981990358408 -122.97844302407012) bank105870 +105871 POINT(47.108203464484085 -122.45175339900251) bank105871 +105872 POINT(48.20294542094827 -122.55150742871443) bank105872 +105873 POINT(47.95972883947692 -121.33915792813256) bank105873 +105874 POINT(48.30877770929471 -121.80191271808712) bank105874 +105875 POINT(47.67488330440197 -122.44177845000425) bank105875 +105876 POINT(48.401828007303 -122.9606665210206) bank105876 +105877 POINT(47.6573855626727 -121.82360743905792) bank105877 +105878 POINT(47.73676141395244 -122.57262586789278) bank105878 +105879 POINT(47.3369170732233 -121.86976702414427) bank105879 +105880 POINT(46.787485044368964 -123.08328256991128) bank105880 +105881 POINT(47.68580121068796 -122.45373145642341) bank105881 +105882 POINT(46.680774332005015 -122.69396898770071) bank105882 +105883 POINT(46.81904132320714 -121.52873333667655) bank105883 +105884 POINT(48.21809811999728 -122.36014823936439) bank105884 +105885 POINT(48.216163073411 -121.79594833376038) bank105885 +105886 POINT(47.24239197005777 -121.36519878611296) bank105886 +105887 POINT(46.992574598827325 -123.0390811155849) bank105887 +105888 POINT(48.01326877204893 -122.63065112135583) bank105888 +105889 POINT(48.19975211356699 -122.68038798826613) bank105889 +105890 POINT(46.76147278167779 -123.02606612375405) bank105890 +105891 POINT(46.73461473505785 -121.51142841096538) bank105891 +105892 POINT(47.27984192138301 -121.36636575548438) bank105892 +105893 POINT(47.95691641029631 -121.38201313642169) bank105893 +105894 POINT(47.56478707905503 -121.47366291007812) bank105894 +105895 POINT(47.56548582670161 -123.08496756579723) bank105895 +105896 POINT(47.26687910915379 -122.50495019420728) bank105896 +105897 POINT(46.94015683076532 -121.70604128669422) bank105897 +105898 POINT(46.89421952624481 -122.77495323814617) bank105898 +105899 POINT(47.06908162368684 -121.91129783763311) bank105899 +105900 POINT(47.43356110199214 -122.32513811550267) bank105900 +105901 POINT(48.497576511957284 -121.99186885215549) bank105901 +105902 POINT(48.33070627839318 -122.61991469119762) bank105902 +105903 POINT(48.495761952161665 -121.81733310144924) bank105903 +105904 POINT(48.36848315824332 -121.87737692021132) bank105904 +105905 POINT(47.10323181446954 -121.49359553112583) bank105905 +105906 POINT(47.25382155767572 -122.7756402418794) bank105906 +105907 POINT(47.038539440393635 -122.13464340464162) bank105907 +105908 POINT(46.64418657498134 -123.16950729146055) bank105908 +105909 POINT(47.63819836854375 -123.33063550692638) bank105909 +105910 POINT(47.517991683635984 -123.31679218878016) bank105910 +105911 POINT(47.414871883563976 -123.18540012957189) bank105911 +105912 POINT(48.49977471571365 -122.56735108227623) bank105912 +105913 POINT(47.043694229979735 -123.08341286119928) bank105913 +105914 POINT(47.07098626007763 -121.47706030432356) bank105914 +105915 POINT(46.76401982913703 -121.52153993868811) bank105915 +105916 POINT(48.189256849705714 -121.34684158603518) bank105916 +105917 POINT(48.596222770178834 -123.28209227418951) bank105917 +105918 POINT(48.30280923258289 -122.21462882683801) bank105918 +105919 POINT(48.41184936180914 -122.94061671701061) bank105919 +105920 POINT(47.249619612131234 -123.25945076359156) bank105920 +105921 POINT(47.84502949832452 -122.57748228170433) bank105921 +105922 POINT(46.91681258528574 -122.91603706755922) bank105922 +105923 POINT(46.683808012237925 -121.40303687847103) bank105923 +105924 POINT(46.83109691193111 -122.52741830850168) bank105924 +105925 POINT(48.265358859404664 -123.01834564246636) bank105925 +105926 POINT(47.063758435742535 -122.31848049878253) bank105926 +105927 POINT(47.16044856472948 -122.39306885673497) bank105927 +105928 POINT(47.44066360320665 -122.66012220122222) bank105928 +105929 POINT(46.83917196138887 -122.4098868470474) bank105929 +105930 POINT(47.64538107837194 -121.37005236831705) bank105930 +105931 POINT(48.55208169613343 -121.70640506822389) bank105931 +105932 POINT(47.49278742022809 -121.42206809380532) bank105932 +105933 POINT(46.626301345162574 -121.63655336366713) bank105933 +105934 POINT(47.69790289920453 -122.01806960156996) bank105934 +105935 POINT(48.31510103724446 -121.77313551936686) bank105935 +105936 POINT(47.659889624115536 -122.23319262792349) bank105936 +105937 POINT(47.57922240909449 -123.1940343879603) bank105937 +105938 POINT(48.0490641635167 -122.12815694443921) bank105938 +105939 POINT(48.49288697144041 -122.16941423939568) bank105939 +105940 POINT(47.154997103851805 -121.72139182683757) bank105940 +105941 POINT(46.68418592790158 -122.20213948424049) bank105941 +105942 POINT(46.91564371334349 -122.20043639508123) bank105942 +105943 POINT(46.83758199761252 -122.6548524345969) bank105943 +105944 POINT(47.550298322774616 -122.32587270889294) bank105944 +105945 POINT(48.49447655803406 -122.60677268776614) bank105945 +105946 POINT(48.30427838057501 -122.69058496850712) bank105946 +105947 POINT(46.69297423912686 -122.3302219653012) bank105947 +105948 POINT(47.38570538768476 -122.61869840736323) bank105948 +105949 POINT(47.697500869323754 -121.99968835717645) bank105949 +105950 POINT(48.06526040026783 -123.27942701954072) bank105950 +105951 POINT(48.332859161342874 -122.69430859451087) bank105951 +105952 POINT(48.378185350905774 -121.50597092963852) bank105952 +105953 POINT(47.513034799130494 -122.02551433522068) bank105953 +105954 POINT(48.400344622181024 -123.17119593055928) bank105954 +105955 POINT(48.25555003619343 -122.3606575856146) bank105955 +105956 POINT(48.13530413930197 -122.9925388901485) bank105956 +105957 POINT(47.144084773397566 -122.54472825354156) bank105957 +105958 POINT(46.815661895571964 -121.64653195904275) bank105958 +105959 POINT(47.268882356740576 -122.66386280421746) bank105959 +105960 POINT(47.81940061985374 -122.83018175843281) bank105960 +105961 POINT(47.14476805675519 -122.95113976382311) bank105961 +105962 POINT(47.61382933218638 -123.20253077945756) bank105962 +105963 POINT(48.160515977230176 -121.74669502551966) bank105963 +105964 POINT(47.19409333823358 -121.95816378473137) bank105964 +105965 POINT(48.36499320874015 -122.68771113437187) bank105965 +105966 POINT(47.14680349406504 -122.18744797802877) bank105966 +105967 POINT(48.07118425112396 -121.77035879568709) bank105967 +105968 POINT(47.347245574931485 -122.10887337833388) bank105968 +105969 POINT(48.42072145590833 -121.73857300413994) bank105969 +105970 POINT(46.98987186058918 -122.56152713059225) bank105970 +105971 POINT(46.72244747108019 -122.9691787131665) bank105971 +105972 POINT(47.18002389961304 -123.31853943557351) bank105972 +105973 POINT(47.9623496940559 -122.2035602977452) bank105973 +105974 POINT(47.90232490329858 -122.65577649331011) bank105974 +105975 POINT(47.947102146980974 -122.12438668702889) bank105975 +105976 POINT(47.77837853641297 -121.37696573399197) bank105976 +105977 POINT(47.95104642655682 -121.81083521353827) bank105977 +105978 POINT(48.23173158954414 -121.62441767489528) bank105978 +105979 POINT(46.67803739692017 -121.90415836577054) bank105979 +105980 POINT(47.06451516834204 -121.36740461967229) bank105980 +105981 POINT(47.92342623081199 -123.00922796875683) bank105981 +105982 POINT(48.068199080951295 -122.63884701675914) bank105982 +105983 POINT(48.32826843865146 -122.87128858124589) bank105983 +105984 POINT(48.51706388084451 -122.36472967209009) bank105984 +105985 POINT(47.4064523430523 -121.83908996174965) bank105985 +105986 POINT(47.62576574060196 -122.27326177754138) bank105986 +105987 POINT(47.74361560674869 -122.976378510928) bank105987 +105988 POINT(47.62217801259275 -122.09505334056428) bank105988 +105989 POINT(47.12446736876248 -121.6266770219523) bank105989 +105990 POINT(48.56337937107616 -122.86715671181697) bank105990 +105991 POINT(47.1384755011418 -122.43362532423242) bank105991 +105992 POINT(47.82063006916704 -123.22572031753744) bank105992 +105993 POINT(47.865279271831085 -121.78116542591947) bank105993 +105994 POINT(47.65473741758892 -122.7492618281303) bank105994 +105995 POINT(46.87688933384635 -122.66380204240593) bank105995 +105996 POINT(48.31475899236368 -122.1674350016177) bank105996 +105997 POINT(47.70735945903138 -121.46869730175435) bank105997 +105998 POINT(47.508108102728364 -122.26226187457543) bank105998 +105999 POINT(46.814655457178745 -122.60708424026282) bank105999 +106000 POINT(48.30829097291644 -121.37644048548715) bank106000 +106001 POINT(48.58833801591407 -122.43184172533829) bank106001 +106002 POINT(47.14781815419433 -122.32020306995808) bank106002 +106003 POINT(46.64704845842977 -122.07419737308003) bank106003 +106004 POINT(47.7813882129106 -122.79123383235739) bank106004 +106005 POINT(47.38827155438488 -121.51640264528751) bank106005 +106006 POINT(47.636167210733014 -121.77492222047194) bank106006 +106007 POINT(48.232682140015925 -122.45575218295707) bank106007 +106008 POINT(47.395113970805774 -123.16828466116058) bank106008 +106009 POINT(48.31605556696748 -122.30157064795372) bank106009 +106010 POINT(48.38247428943405 -122.24731136833718) bank106010 +106011 POINT(48.29450264065021 -123.06375758964862) bank106011 +106012 POINT(48.53726052534001 -122.20190044821199) bank106012 +106013 POINT(47.817951356338334 -122.14396407864153) bank106013 +106014 POINT(46.80262505818806 -121.49430002222795) bank106014 +106015 POINT(47.02358755848928 -121.36749608493066) bank106015 +106016 POINT(48.53668667977447 -122.98276911206176) bank106016 +106017 POINT(47.48947553570089 -122.5999668893124) bank106017 +106018 POINT(48.23037590215494 -123.05756233491599) bank106018 +106019 POINT(46.62573416237406 -121.80767061381991) bank106019 +106020 POINT(48.25590507773547 -122.93725195441357) bank106020 +106021 POINT(48.28266326975946 -122.5354650491475) bank106021 +106022 POINT(47.833469757261255 -121.59799606708924) bank106022 +106023 POINT(47.22517817888831 -121.5169071225069) bank106023 +106024 POINT(46.81924258397691 -122.18044064087609) bank106024 +106025 POINT(48.543354139592786 -121.42616398743178) bank106025 +106026 POINT(47.587164051826775 -122.37826123836284) bank106026 +106027 POINT(47.582922527282044 -121.3356603428241) bank106027 +106028 POINT(47.92519441286702 -121.82805863116077) bank106028 +106029 POINT(47.14263630484997 -122.7714937693757) bank106029 +106030 POINT(47.250906980380336 -121.94001171619227) bank106030 +106031 POINT(47.606289537220185 -123.01811935793096) bank106031 +106032 POINT(48.4163677446573 -122.18249920988119) bank106032 +106033 POINT(47.541141244392065 -122.100934144762) bank106033 +106034 POINT(47.881039423568204 -122.5858390662868) bank106034 +106035 POINT(47.15963938063112 -122.24947059940189) bank106035 +106036 POINT(46.82232348942647 -123.2224953041091) bank106036 +106037 POINT(47.218821063093834 -122.224485288528) bank106037 +106038 POINT(48.59550401020196 -121.57908574463872) bank106038 +106039 POINT(46.61387071086838 -122.19947243371016) bank106039 +106040 POINT(48.16167039809671 -123.00707714084231) bank106040 +106041 POINT(46.87721670892125 -121.3997756274785) bank106041 +106042 POINT(48.428402746341185 -122.31834430483008) bank106042 +106043 POINT(48.147438652227656 -122.47388217570608) bank106043 +106044 POINT(48.355330832692964 -122.12640295696251) bank106044 +106045 POINT(47.6280245979186 -121.89100383838358) bank106045 +106046 POINT(46.81022851250879 -122.55631633663413) bank106046 +106047 POINT(47.1035921799925 -121.72588160953468) bank106047 +106048 POINT(47.0070079519205 -121.5530829586138) bank106048 +106049 POINT(47.169340412492026 -123.0296427376941) bank106049 +106050 POINT(48.5598655608038 -122.78290691558921) bank106050 +106051 POINT(47.50318929879283 -122.0579905974204) bank106051 +106052 POINT(47.65625800222839 -123.10430995761048) bank106052 +106053 POINT(47.37429326771134 -122.52259571314607) bank106053 +106054 POINT(47.35400374544743 -122.59614200935269) bank106054 +106055 POINT(46.96286574199821 -121.38069692700542) bank106055 +106056 POINT(47.22238736613618 -122.12239469482826) bank106056 +106057 POINT(46.76911941742544 -121.40270374505268) bank106057 +106058 POINT(47.4294994335546 -121.9746065181802) bank106058 +106059 POINT(47.21868281065211 -121.6007175929667) bank106059 +106060 POINT(46.84839511090032 -122.25846988087152) bank106060 +106061 POINT(48.543870481909735 -122.78830566155257) bank106061 +106062 POINT(47.73704657984799 -122.20072707775932) bank106062 +106063 POINT(48.54954562581687 -122.67644794406172) bank106063 +106064 POINT(47.87446885451252 -123.25109101718769) bank106064 +106065 POINT(47.432449671548284 -121.62836220671467) bank106065 +106066 POINT(46.913083678014665 -122.57898519536869) bank106066 +106067 POINT(47.23459156084772 -121.63713446796639) bank106067 +106068 POINT(46.75675215722883 -121.36050866707497) bank106068 +106069 POINT(48.332154274658386 -121.50236299570278) bank106069 +106070 POINT(48.51264235710952 -122.72761850042956) bank106070 +106071 POINT(47.47065229954999 -122.60553541605647) bank106071 +106072 POINT(48.49716109283102 -122.9381088711176) bank106072 +106073 POINT(46.79340970469428 -121.50666500637118) bank106073 +106074 POINT(46.60806302946445 -121.70247160431657) bank106074 +106075 POINT(47.84741900394114 -121.98530181939344) bank106075 +106076 POINT(47.554515138396255 -122.18573435524331) bank106076 +106077 POINT(47.140208705293624 -123.22228250236033) bank106077 +106078 POINT(46.67168590054592 -121.4843707364256) bank106078 +106079 POINT(47.23479382143293 -122.8506537139881) bank106079 +106080 POINT(47.112174955515115 -122.7609889028889) bank106080 +106081 POINT(48.32724492850217 -122.90669537876597) bank106081 +106082 POINT(46.759284765584404 -121.92452719049506) bank106082 +106083 POINT(47.15800392493146 -122.00327031250818) bank106083 +106084 POINT(48.191174922582306 -122.12172226842745) bank106084 +106085 POINT(47.38810123454796 -121.72179304517712) bank106085 +106086 POINT(47.69653190853055 -121.48078943434919) bank106086 +106087 POINT(48.20987613746965 -122.32688326549795) bank106087 +106088 POINT(46.7703649292881 -122.53644848641598) bank106088 +106089 POINT(47.11685935651994 -121.65343190381344) bank106089 +106090 POINT(48.57175231450332 -122.60465174320144) bank106090 +106091 POINT(46.899793686528 -123.19386942125718) bank106091 +106092 POINT(48.359486657602254 -121.36244645198389) bank106092 +106093 POINT(48.44295647457823 -122.89095030766059) bank106093 +106094 POINT(47.577711531680364 -122.71021977636592) bank106094 +106095 POINT(47.0893861530041 -122.26405995769676) bank106095 +106096 POINT(47.25027790176956 -121.76806567081505) bank106096 +106097 POINT(47.979080072527886 -122.01896574120147) bank106097 +106098 POINT(46.86864615010666 -122.55111653407289) bank106098 +106099 POINT(47.5554019253328 -122.6610369187108) bank106099 +106100 POINT(47.5028653014333 -122.99872325876824) bank106100 +106101 POINT(47.60978502635387 -123.06623394477462) bank106101 +106102 POINT(46.86920674237782 -122.7213860601501) bank106102 +106103 POINT(48.37108555711144 -123.25184334348464) bank106103 +106104 POINT(48.018465130431736 -123.2588277092122) bank106104 +106105 POINT(46.92808232888891 -121.73694650126343) bank106105 +106106 POINT(48.10652258520116 -122.14246355134567) bank106106 +106107 POINT(46.664759848889304 -123.32034746347523) bank106107 +106108 POINT(47.41873437258661 -121.7381553891445) bank106108 +106109 POINT(47.514727092536056 -123.15716809329138) bank106109 +106110 POINT(46.626553964179905 -122.91678090550586) bank106110 +106111 POINT(47.07770476781192 -121.78036223249349) bank106111 +106112 POINT(46.831338689824086 -122.61341438156943) bank106112 +106113 POINT(48.22962580770959 -121.90803575637746) bank106113 +106114 POINT(47.46377343841529 -121.73557218974675) bank106114 +106115 POINT(47.36533209838106 -121.52200822094842) bank106115 +106116 POINT(47.298672759207285 -122.20851173740616) bank106116 +106117 POINT(47.061066082803414 -122.8874097005463) bank106117 +106118 POINT(47.50144513985824 -122.9757263673817) bank106118 +106119 POINT(48.48541758555096 -121.36538871409748) bank106119 +106120 POINT(48.04331633384632 -122.97678657619923) bank106120 +106121 POINT(46.733888918299854 -122.91964713629928) bank106121 +106122 POINT(47.53729610020822 -123.26904386601363) bank106122 +106123 POINT(47.55681003478036 -123.03146812659053) bank106123 +106124 POINT(48.584614224836486 -122.7093389229146) bank106124 +106125 POINT(48.39732777107826 -121.33795040981106) bank106125 +106126 POINT(47.36869234840917 -121.34548070146208) bank106126 +106127 POINT(47.42699898219697 -121.4417073809482) bank106127 +106128 POINT(47.411690671734284 -123.18817525966696) bank106128 +106129 POINT(46.87908684092745 -122.9379282310034) bank106129 +106130 POINT(47.46584877145927 -122.17168106123405) bank106130 +106131 POINT(48.37500833307702 -122.90491698593624) bank106131 +106132 POINT(47.6811433068531 -122.09248079656238) bank106132 +106133 POINT(48.095327207375945 -121.83143398415383) bank106133 +106134 POINT(47.892426627887275 -122.31022196074333) bank106134 +106135 POINT(48.2870289452477 -122.83222166157442) bank106135 +106136 POINT(47.87853215239241 -121.9128514864172) bank106136 +106137 POINT(47.8273674967514 -122.54500765896039) bank106137 +106138 POINT(48.24020238889934 -122.9808290212956) bank106138 +106139 POINT(47.00659471466292 -123.1557797023889) bank106139 +106140 POINT(48.56944780964424 -123.0401628917158) bank106140 +106141 POINT(46.96070705404335 -122.07113445701707) bank106141 +106142 POINT(48.23560593588337 -122.15138879241896) bank106142 +106143 POINT(47.245231480687934 -121.59174352860813) bank106143 +106144 POINT(47.76512194683854 -122.04924531166881) bank106144 +106145 POINT(48.3373954828289 -121.41423034105533) bank106145 +106146 POINT(47.422159582417 -121.53841769399334) bank106146 +106147 POINT(48.51531168156893 -122.42375839551707) bank106147 +106148 POINT(47.4512863028612 -121.51855781303136) bank106148 +106149 POINT(48.182856246702286 -121.55924255547112) bank106149 +106150 POINT(47.091746114920966 -123.24831686236256) bank106150 +106151 POINT(47.457293509648615 -122.73544627427562) bank106151 +106152 POINT(46.768352913808734 -122.24979078951655) bank106152 +106153 POINT(47.01160922104602 -121.63918731565438) bank106153 +106154 POINT(46.90323532137724 -121.35192440544355) bank106154 +106155 POINT(47.387159769333806 -121.83340173145183) bank106155 +106156 POINT(47.700820019325825 -123.28822541172542) bank106156 +106157 POINT(47.4527032279688 -122.24305460769708) bank106157 +106158 POINT(46.778964459294706 -121.57406433239633) bank106158 +106159 POINT(47.17353532586314 -122.56645048167981) bank106159 +106160 POINT(46.65107343078857 -122.07862296612716) bank106160 +106161 POINT(48.30099576333316 -121.62628668994297) bank106161 +106162 POINT(48.17941815240127 -123.15141859266626) bank106162 +106163 POINT(46.81576846154063 -123.10941944339602) bank106163 +106164 POINT(47.84787087732292 -122.9760472404512) bank106164 +106165 POINT(47.95875561464679 -121.82057468484841) bank106165 +106166 POINT(48.183339558261274 -121.65054560665997) bank106166 +106167 POINT(47.68124779028912 -121.64426737884457) bank106167 +106168 POINT(47.825731364525915 -122.7314975507743) bank106168 +106169 POINT(47.82277883769118 -121.63137345074685) bank106169 +106170 POINT(46.7009753583185 -121.85694755241788) bank106170 +106171 POINT(47.320211565099974 -122.7960249272092) bank106171 +106172 POINT(46.93977404879628 -122.51089325720976) bank106172 +106173 POINT(47.65205191577482 -121.59159243418496) bank106173 +106174 POINT(47.86209049137253 -121.85748441004283) bank106174 +106175 POINT(46.97307136002959 -122.76951898124986) bank106175 +106176 POINT(48.58929346246802 -122.91201255691233) bank106176 +106177 POINT(47.34065697725416 -123.03032709772373) bank106177 +106178 POINT(48.29681249399937 -123.32826297381393) bank106178 +106179 POINT(47.00914758583619 -122.88438298207068) bank106179 +106180 POINT(47.67589817295611 -121.4017198454713) bank106180 +106181 POINT(46.80145433364032 -123.08094240933623) bank106181 +106182 POINT(48.050602681265104 -121.92220332381206) bank106182 +106183 POINT(48.26928152563602 -121.91214598969778) bank106183 +106184 POINT(47.74166885414627 -122.99355203654916) bank106184 +106185 POINT(48.02346877955857 -121.49247429181005) bank106185 +106186 POINT(47.105105099086416 -123.18767494868837) bank106186 +106187 POINT(46.770505244602205 -122.56230066793805) bank106187 +106188 POINT(47.013749595172484 -121.71779413583478) bank106188 +106189 POINT(47.92014703451626 -122.59949577360982) bank106189 +106190 POINT(48.38848307844164 -121.80988846901744) bank106190 +106191 POINT(47.30214072672537 -121.39434225436901) bank106191 +106192 POINT(47.0140835895822 -121.98302530912099) bank106192 +106193 POINT(48.328671927767445 -122.2061129235277) bank106193 +106194 POINT(47.348232670398566 -122.21151932316526) bank106194 +106195 POINT(48.30813228581169 -121.43763571983128) bank106195 +106196 POINT(47.40568656326706 -123.05063426225517) bank106196 +106197 POINT(47.272226019909205 -122.93769917126431) bank106197 +106198 POINT(47.820981141295945 -122.60288314413445) bank106198 +106199 POINT(47.898175732463514 -123.08310969556557) bank106199 +106200 POINT(46.81582553920292 -122.56184939532999) bank106200 +106201 POINT(47.587282369798906 -122.68139797974328) bank106201 +106202 POINT(47.98231676291499 -122.47265761634418) bank106202 +106203 POINT(47.163284777738426 -123.04287743498499) bank106203 +106204 POINT(47.63028988655931 -123.27147163556681) bank106204 +106205 POINT(46.87294978777747 -123.33153781555455) bank106205 +106206 POINT(47.19823927887847 -121.89135279546085) bank106206 +106207 POINT(48.04643552167628 -121.42368740622747) bank106207 +106208 POINT(46.82605807196209 -121.79917414966916) bank106208 +106209 POINT(46.8088777166331 -121.73253084208153) bank106209 +106210 POINT(48.124797397089175 -123.03622997845477) bank106210 +106211 POINT(48.10795862377791 -122.94803935488952) bank106211 +106212 POINT(46.96505168305613 -121.33380355542947) bank106212 +106213 POINT(46.69738905770829 -122.63421615724806) bank106213 +106214 POINT(48.19256542276523 -123.24617400405106) bank106214 +106215 POINT(48.16307533705785 -122.56549962647523) bank106215 +106216 POINT(47.50699680843529 -121.79311473578679) bank106216 +106217 POINT(46.62516134580911 -122.33308571978786) bank106217 +106218 POINT(46.689654782228956 -121.8070476732948) bank106218 +106219 POINT(48.39887517360329 -122.83310669605507) bank106219 +106220 POINT(47.37177281925337 -122.05214924482478) bank106220 +106221 POINT(46.63874695862115 -122.65496553934682) bank106221 +106222 POINT(46.96170507495047 -123.04320748589916) bank106222 +106223 POINT(47.89251696858565 -122.9963681843511) bank106223 +106224 POINT(47.65600027634156 -122.82430601376959) bank106224 +106225 POINT(48.3545043482473 -121.89560033467994) bank106225 +106226 POINT(47.66659097169804 -121.95437141171395) bank106226 +106227 POINT(48.123676857994084 -121.39254865297022) bank106227 +106228 POINT(48.24171056618082 -122.51995186405449) bank106228 +106229 POINT(48.284534625487254 -121.68692752690865) bank106229 +106230 POINT(48.00592664866136 -123.08158901865261) bank106230 +106231 POINT(48.052456455902615 -122.80502720768285) bank106231 +106232 POINT(46.97328838283893 -123.1352573262504) bank106232 +106233 POINT(46.98900688454993 -122.24172480496225) bank106233 +106234 POINT(47.864895802324654 -121.37287942019248) bank106234 +106235 POINT(46.79516170256742 -122.60766219540484) bank106235 +106236 POINT(47.03157997318582 -123.02675309708526) bank106236 +106237 POINT(48.53224059112347 -122.9348980760055) bank106237 +106238 POINT(48.23351508298048 -122.59007360326004) bank106238 +106239 POINT(47.375842332077546 -122.09492321593144) bank106239 +106240 POINT(47.21297255253309 -122.794052813858) bank106240 +106241 POINT(47.91578200922285 -123.20420367060903) bank106241 +106242 POINT(47.61281598532148 -122.53134132061894) bank106242 +106243 POINT(47.28942216517806 -122.18050464200432) bank106243 +106244 POINT(46.975806031217566 -122.9758504362944) bank106244 +106245 POINT(47.49417026359757 -122.52801602756041) bank106245 +106246 POINT(47.526517768092205 -122.57312443133951) bank106246 +106247 POINT(47.8136144784249 -123.26770927674008) bank106247 +106248 POINT(48.09218587708248 -122.55020410898165) bank106248 +106249 POINT(48.22314114887089 -122.4443651605767) bank106249 +106250 POINT(48.25217652552455 -122.92374014622808) bank106250 +106251 POINT(46.996553260183795 -122.72515993330371) bank106251 +106252 POINT(47.63466987653853 -121.78696264494555) bank106252 +106253 POINT(47.73781373105329 -122.48589362381416) bank106253 +106254 POINT(47.392826017067165 -121.99304490117423) bank106254 +106255 POINT(47.59542657305313 -123.06021451622306) bank106255 +106256 POINT(47.47514693642533 -122.2765913671325) bank106256 +106257 POINT(46.76024784725199 -121.37265795270937) bank106257 +106258 POINT(48.32241306106358 -122.0446292513292) bank106258 +106259 POINT(48.32613061443671 -122.82360099284628) bank106259 +106260 POINT(47.369748188634006 -122.25358904373272) bank106260 +106261 POINT(48.18958750942346 -122.46142985350052) bank106261 +106262 POINT(48.3448577959439 -121.63065990093656) bank106262 +106263 POINT(46.62479421134471 -122.71895939769261) bank106263 +106264 POINT(47.363550019527985 -122.11393641853256) bank106264 +106265 POINT(47.17066639447886 -122.12397878622146) bank106265 +106266 POINT(46.750671624233625 -122.44881934748615) bank106266 +106267 POINT(48.60440913108318 -122.93725790295615) bank106267 +106268 POINT(47.97875287258825 -123.23915220563755) bank106268 +106269 POINT(47.03167638602843 -122.69028908436488) bank106269 +106270 POINT(46.9039842016139 -121.675537734382) bank106270 +106271 POINT(47.9257768888451 -121.42337433794486) bank106271 +106272 POINT(48.214480817103166 -122.29651099153996) bank106272 +106273 POINT(47.95216953743936 -121.58775600577783) bank106273 +106274 POINT(48.24850409445055 -122.50270764163987) bank106274 +106275 POINT(48.57269629768734 -122.48449518825451) bank106275 +106276 POINT(47.064574035794934 -122.45346230920016) bank106276 +106277 POINT(47.503630551158366 -121.49393452994639) bank106277 +106278 POINT(46.97599227857753 -122.31506466079641) bank106278 +106279 POINT(48.12751280351559 -122.11449932833214) bank106279 +106280 POINT(48.23678063504485 -121.74382362705872) bank106280 +106281 POINT(47.33304914486022 -122.53754774599801) bank106281 +106282 POINT(48.02238744348249 -123.02315735034327) bank106282 +106283 POINT(48.349536884835274 -121.48705610181452) bank106283 +106284 POINT(47.47620120054477 -121.4865079339459) bank106284 +106285 POINT(46.74680869812954 -121.98790148062862) bank106285 +106286 POINT(47.47528670174656 -122.88802423816449) bank106286 +106287 POINT(47.3909384669283 -121.68056095304765) bank106287 +106288 POINT(47.56222873202461 -122.48904316901711) bank106288 +106289 POINT(48.242886328704124 -121.82987812645337) bank106289 +106290 POINT(46.898388658349425 -122.01079177872633) bank106290 +106291 POINT(48.00841047930217 -122.00043519759629) bank106291 +106292 POINT(48.509713754577334 -121.34528002260693) bank106292 +106293 POINT(47.90134316762262 -121.4208922152561) bank106293 +106294 POINT(48.39129098070635 -121.67755688749969) bank106294 +106295 POINT(47.24865837806164 -122.84885873071477) bank106295 +106296 POINT(46.70672175037347 -123.1362985938792) bank106296 +106297 POINT(47.01049182893892 -122.11289762535208) bank106297 +106298 POINT(47.49012197667231 -122.96793179255256) bank106298 +106299 POINT(48.317199684099045 -121.4284868145711) bank106299 +106300 POINT(47.438550029699435 -122.67244511455733) bank106300 +106301 POINT(47.58541876862271 -121.91645034576582) bank106301 +106302 POINT(46.979894295181715 -122.67412700984319) bank106302 +106303 POINT(48.01938748293822 -122.38691897540117) bank106303 +106304 POINT(47.405762365717116 -121.59075817831791) bank106304 +106305 POINT(47.81074981219571 -121.36616115581825) bank106305 +106306 POINT(48.18383310138746 -121.70979010522814) bank106306 +106307 POINT(47.51149197205458 -122.19631884300632) bank106307 +106308 POINT(48.28676672653776 -122.91704843059833) bank106308 +106309 POINT(47.83147787225824 -122.488541815382) bank106309 +106310 POINT(48.273550096620745 -122.21690931644805) bank106310 +106311 POINT(47.909636753100166 -123.24144694385447) bank106311 +106312 POINT(46.97374601725882 -123.3309586768601) bank106312 +106313 POINT(46.88893230481322 -121.37825701821612) bank106313 +106314 POINT(48.52395567268232 -121.77814332136226) bank106314 +106315 POINT(48.27414383666642 -121.83623087909203) bank106315 +106316 POINT(46.95916615550916 -122.61907586019849) bank106316 +106317 POINT(47.24959536356128 -121.45890771934717) bank106317 +106318 POINT(47.92311581081412 -121.96504630989602) bank106318 +106319 POINT(47.87975490785037 -121.60261534018291) bank106319 +106320 POINT(46.73885177784567 -121.90602840400965) bank106320 +106321 POINT(46.78905013045313 -122.33833608789709) bank106321 +106322 POINT(48.335034307266326 -121.52522620411037) bank106322 +106323 POINT(47.278832359896406 -122.00283639568997) bank106323 +106324 POINT(48.06160432343839 -123.00481010572805) bank106324 +106325 POINT(48.10216952964001 -122.55384682850232) bank106325 +106326 POINT(47.496894095391355 -123.10778396806981) bank106326 +106327 POINT(47.61548994049577 -121.72274723491266) bank106327 +106328 POINT(46.93639654157398 -122.81770121013457) bank106328 +106329 POINT(48.11571924285724 -121.52435792971575) bank106329 +106330 POINT(47.53724597634422 -123.09602237578451) bank106330 +106331 POINT(47.348328089907625 -121.69082197538405) bank106331 +106332 POINT(48.58523279538134 -123.16721147796126) bank106332 +106333 POINT(46.93061073131861 -122.10611830201658) bank106333 +106334 POINT(47.371558170048935 -121.93512899134443) bank106334 +106335 POINT(46.67390472586566 -121.76758958179158) bank106335 +106336 POINT(46.90782316089457 -122.10907209341568) bank106336 +106337 POINT(47.482029635915964 -121.48082651659217) bank106337 +106338 POINT(46.82171938029734 -123.19424258776549) bank106338 +106339 POINT(47.8829382409845 -123.26813016286131) bank106339 +106340 POINT(48.56736578512502 -122.79610526572502) bank106340 +106341 POINT(47.59582043482669 -121.40266106176418) bank106341 +106342 POINT(47.901424447177284 -122.5115440985226) bank106342 +106343 POINT(46.65311570383946 -123.0193278952103) bank106343 +106344 POINT(47.78377618188592 -122.7486486791627) bank106344 +106345 POINT(47.01707862712891 -123.09751966058279) bank106345 +106346 POINT(47.8052911813051 -122.85032305494614) bank106346 +106347 POINT(46.615978477070634 -123.11518144545242) bank106347 +106348 POINT(46.65965015420387 -122.13250409821384) bank106348 +106349 POINT(46.69387524600585 -122.9855144068928) bank106349 +106350 POINT(48.00120756002544 -122.62161077312915) bank106350 +106351 POINT(47.990608491778 -121.5835224152639) bank106351 +106352 POINT(46.926916761205774 -121.65471818158733) bank106352 +106353 POINT(48.00906334888368 -121.43354980255423) bank106353 +106354 POINT(47.57631706424024 -122.86166980769565) bank106354 +106355 POINT(48.42646088743741 -122.17331934166963) bank106355 +106356 POINT(46.89681943874597 -121.5686813564514) bank106356 +106357 POINT(47.026072425056995 -121.81565493069415) bank106357 +106358 POINT(48.02176710280155 -123.02970245229582) bank106358 +106359 POINT(46.83742437598515 -122.29425172369601) bank106359 +106360 POINT(47.119692832468225 -123.04134900998284) bank106360 +106361 POINT(47.77109386913946 -122.43006110186747) bank106361 +106362 POINT(48.288342654937566 -123.14489259389345) bank106362 +106363 POINT(46.78353316144728 -121.99033291868051) bank106363 +106364 POINT(47.011655131132954 -123.20217511651522) bank106364 +106365 POINT(47.438713509285165 -122.82193402030654) bank106365 +106366 POINT(48.18424080152684 -121.44417786615263) bank106366 +106367 POINT(48.27211439793701 -122.44780743242649) bank106367 +106368 POINT(46.96167493649568 -122.16703404267977) bank106368 +106369 POINT(46.68830512816139 -123.0106867309672) bank106369 +106370 POINT(47.04021523175563 -121.82153983941454) bank106370 +106371 POINT(46.81721931845823 -121.61312772498165) bank106371 +106372 POINT(48.32283550686722 -122.14541471624747) bank106372 +106373 POINT(46.718907612623 -121.46660927112467) bank106373 +106374 POINT(48.081025493690845 -121.58604108296328) bank106374 +106375 POINT(46.985495371603335 -123.07409558900926) bank106375 +106376 POINT(47.34577896173422 -122.14056290883879) bank106376 +106377 POINT(46.704419100942424 -122.09095501586334) bank106377 +106378 POINT(47.047467661954904 -122.6954681903905) bank106378 +106379 POINT(48.20687217486627 -122.39418634300593) bank106379 +106380 POINT(47.410745311982154 -122.28063837673218) bank106380 +106381 POINT(47.79050327319 -122.56914792669895) bank106381 +106382 POINT(48.5394093984679 -123.21614704146202) bank106382 +106383 POINT(47.390458148731696 -122.13392676022877) bank106383 +106384 POINT(47.45725957867292 -123.13177469785246) bank106384 +106385 POINT(47.83455380833303 -122.91284043354891) bank106385 +106386 POINT(46.74410825809968 -122.73308410090533) bank106386 +106387 POINT(47.86619821960657 -121.89280406870411) bank106387 +106388 POINT(48.273082237087586 -122.659863039794) bank106388 +106389 POINT(46.71920900020964 -122.13964494802076) bank106389 +106390 POINT(47.27130698247984 -121.92613857952729) bank106390 +106391 POINT(48.315744345947806 -122.91562956334758) bank106391 +106392 POINT(48.326571570948865 -122.8127805522375) bank106392 +106393 POINT(48.13450488420531 -122.2216491829742) bank106393 +106394 POINT(48.112407585665686 -122.02853555285085) bank106394 +106395 POINT(48.447816969792896 -122.55256601046133) bank106395 +106396 POINT(47.60279897301896 -122.61262481662783) bank106396 +106397 POINT(48.50905525818804 -122.2237935312392) bank106397 +106398 POINT(47.599786121059886 -122.29232022993742) bank106398 +106399 POINT(46.66660600675971 -121.6754429477383) bank106399 +106400 POINT(46.67167146397244 -121.91128121792585) bank106400 +106401 POINT(46.83552859048288 -121.86035915862978) bank106401 +106402 POINT(47.19032510226285 -122.298715197494) bank106402 +106403 POINT(48.04440661601265 -122.62557195753915) bank106403 +106404 POINT(47.529217963856446 -121.68263250521098) bank106404 +106405 POINT(47.655929657136994 -123.28052022176176) bank106405 +106406 POINT(47.795340257910375 -122.72166557572717) bank106406 +106407 POINT(46.854091816000214 -123.13083263970735) bank106407 +106408 POINT(47.522572351352494 -123.20398762478973) bank106408 +106409 POINT(48.03294512775499 -121.57735780277264) bank106409 +106410 POINT(48.28953357342782 -122.70192417731916) bank106410 +106411 POINT(47.73700108487966 -121.42314099069539) bank106411 +106412 POINT(47.226777961591644 -121.61610190515422) bank106412 +106413 POINT(47.63515065518495 -121.65045072782924) bank106413 +106414 POINT(48.24716874634718 -121.3854728406899) bank106414 +106415 POINT(48.559675692167744 -122.0238998610929) bank106415 +106416 POINT(48.30382113521259 -122.16941777205948) bank106416 +106417 POINT(48.5922908774192 -122.27398639131573) bank106417 +106418 POINT(47.21469750613297 -121.8710771060146) bank106418 +106419 POINT(47.86595782494961 -122.42963399975841) bank106419 +106420 POINT(47.362796429766306 -121.54418726667951) bank106420 +106421 POINT(46.97825282033893 -122.15557249031673) bank106421 +106422 POINT(48.29977616412418 -122.29904372437689) bank106422 +106423 POINT(46.779139818609444 -121.77604223968724) bank106423 +106424 POINT(47.086647696477414 -122.56186828407296) bank106424 +106425 POINT(47.21896724203966 -122.2720803820002) bank106425 +106426 POINT(47.99532282563935 -122.56899922298662) bank106426 +106427 POINT(47.040433161281456 -122.5025786395436) bank106427 +106428 POINT(47.9063080457684 -121.61839674538147) bank106428 +106429 POINT(47.47535571498162 -122.10794521335052) bank106429 +106430 POINT(48.18807666004297 -122.59820446050273) bank106430 +106431 POINT(46.717127978164605 -123.28568210808493) bank106431 +106432 POINT(46.640036489717836 -122.15118053912758) bank106432 +106433 POINT(47.45499773636375 -122.49723069242536) bank106433 +106434 POINT(46.828847021264764 -122.54139375324579) bank106434 +106435 POINT(47.03564015726067 -122.50203074862902) bank106435 +106436 POINT(47.80116509906192 -122.99738600563137) bank106436 +106437 POINT(47.68800394532575 -123.08708881049157) bank106437 +106438 POINT(48.06978308140754 -121.8011927405248) bank106438 +106439 POINT(47.040122265440026 -121.56685878581514) bank106439 +106440 POINT(47.88291287746371 -122.8202439990385) bank106440 +106441 POINT(47.38154913078638 -121.47184308112317) bank106441 +106442 POINT(46.96504564665374 -121.33922846712332) bank106442 +106443 POINT(47.64969156238782 -121.93943626967682) bank106443 +106444 POINT(47.159107835686505 -122.9180379179721) bank106444 +106445 POINT(48.044990310675885 -123.02851032770833) bank106445 +106446 POINT(48.44643507199344 -122.71469782232539) bank106446 +106447 POINT(47.85588631628362 -122.92641575155562) bank106447 +106448 POINT(47.399130101887536 -122.3021495580162) bank106448 +106449 POINT(47.08045950954687 -123.29246882458173) bank106449 +106450 POINT(48.01863694969056 -121.85983867289966) bank106450 +106451 POINT(48.19303735364849 -121.92220590328468) bank106451 +106452 POINT(48.60246559061369 -121.5619679037692) bank106452 +106453 POINT(48.48365100194429 -122.87179204489996) bank106453 +106454 POINT(47.380311563014224 -122.99472810969935) bank106454 +106455 POINT(48.401786955275604 -121.6884579508879) bank106455 +106456 POINT(47.71135218524963 -121.55530294751999) bank106456 +106457 POINT(47.96325207956921 -121.39654519704735) bank106457 +106458 POINT(48.290250143124986 -121.59514747207612) bank106458 +106459 POINT(46.669533079386056 -121.33649748521353) bank106459 +106460 POINT(48.40515675101182 -122.44817064423195) bank106460 +106461 POINT(48.384808570588206 -122.92361542735937) bank106461 +106462 POINT(48.55576235179754 -123.26255572180798) bank106462 +106463 POINT(48.35209045130394 -122.99143515463336) bank106463 +106464 POINT(46.873985498283396 -122.55468248379977) bank106464 +106465 POINT(47.17056373674428 -122.99159300066768) bank106465 +106466 POINT(48.13888986509925 -122.01109781581309) bank106466 +106467 POINT(46.64878042463562 -123.30991616993683) bank106467 +106468 POINT(47.59016985490814 -121.98994915984814) bank106468 +106469 POINT(47.2681098129947 -121.44388706071874) bank106469 +106470 POINT(46.6355356506935 -122.22843762613928) bank106470 +106471 POINT(46.60832365009005 -121.96370575606) bank106471 +106472 POINT(47.357293766098806 -123.31421402899198) bank106472 +106473 POINT(48.59077110811461 -122.0984786445672) bank106473 +106474 POINT(47.326970788072664 -121.34833758316597) bank106474 +106475 POINT(47.87444396986434 -122.45246728803524) bank106475 +106476 POINT(47.517567313831776 -121.91406699910338) bank106476 +106477 POINT(47.51566443848692 -122.8992067009709) bank106477 +106478 POINT(47.51069220404081 -121.59381466259515) bank106478 +106479 POINT(47.30616560297135 -122.8259915633815) bank106479 +106480 POINT(47.67917043370508 -123.22135437399973) bank106480 +106481 POINT(47.27408877156156 -122.57642917267216) bank106481 +106482 POINT(46.915746770639444 -122.6112733130294) bank106482 +106483 POINT(47.13700977808445 -123.22260650099595) bank106483 +106484 POINT(47.207477046228576 -122.76952498417458) bank106484 +106485 POINT(48.54803337696663 -122.82436137028567) bank106485 +106486 POINT(47.22545360733637 -123.23779878068072) bank106486 +106487 POINT(48.469916916102854 -122.7159681626116) bank106487 +106488 POINT(46.785831356438344 -121.98463369120115) bank106488 +106489 POINT(46.67948853791524 -121.37851502531703) bank106489 +106490 POINT(48.28880857308024 -122.58371565942947) bank106490 +106491 POINT(47.814335022351315 -122.79272914873498) bank106491 +106492 POINT(47.62681180856287 -122.96944807410576) bank106492 +106493 POINT(47.43320076209799 -122.23449942271907) bank106493 +106494 POINT(47.08929150427911 -122.222273100577) bank106494 +106495 POINT(48.045037838015475 -122.9407555920138) bank106495 +106496 POINT(47.683405336761425 -122.99581880180251) bank106496 +106497 POINT(48.55681543766472 -121.9470761172017) bank106497 +106498 POINT(46.641987825228355 -121.80243406623246) bank106498 +106499 POINT(48.255448818358964 -122.5392454149709) bank106499 +106500 POINT(48.54270288823422 -123.30976147566707) bank106500 +106501 POINT(47.69502805971489 -123.18328167014502) bank106501 +106502 POINT(48.12302531446961 -122.18752836282557) bank106502 +106503 POINT(48.29815023192718 -123.2758155511436) bank106503 +106504 POINT(47.481641474260954 -121.83512394398323) bank106504 +106505 POINT(47.97222898037942 -122.18233457639431) bank106505 +106506 POINT(46.94928690496156 -121.76132882367527) bank106506 +106507 POINT(47.64161547304948 -122.94857863349492) bank106507 +106508 POINT(48.41515192308435 -121.92774116759747) bank106508 +106509 POINT(47.8571492142001 -121.52627988550753) bank106509 +106510 POINT(47.343480730943355 -121.63429964268877) bank106510 +106511 POINT(47.9110204395072 -122.20988741804001) bank106511 +106512 POINT(46.79100291442587 -122.71533487755352) bank106512 +106513 POINT(48.564055374621375 -122.46891808809139) bank106513 +106514 POINT(48.12701077511403 -121.71138025339715) bank106514 +106515 POINT(47.042852487066305 -121.72823453426697) bank106515 +106516 POINT(48.46346224520338 -121.77644332077189) bank106516 +106517 POINT(47.563582841826836 -121.45952960661637) bank106517 +106518 POINT(47.68151309533351 -122.30540625874207) bank106518 +106519 POINT(47.88665298221843 -123.12650044887596) bank106519 +106520 POINT(47.307532697744506 -122.83082132761297) bank106520 +106521 POINT(47.7350525063537 -123.30078475493694) bank106521 +106522 POINT(47.241809584016025 -121.96368789842349) bank106522 +106523 POINT(47.554113616182384 -121.54653638893643) bank106523 +106524 POINT(48.25922235635107 -122.3001449401938) bank106524 +106525 POINT(48.485984685394016 -121.68208333331565) bank106525 +106526 POINT(48.472483846979124 -122.34443760152102) bank106526 +106527 POINT(47.48465158123762 -122.29232535844027) bank106527 +106528 POINT(47.654875591214804 -121.95437071013973) bank106528 +106529 POINT(46.62003484439423 -121.8580751040413) bank106529 +106530 POINT(46.6389179880944 -121.76526737461049) bank106530 +106531 POINT(47.06378652364418 -122.19795806319307) bank106531 +106532 POINT(47.69580777411841 -122.15056678663744) bank106532 +106533 POINT(47.20164500232902 -121.81339993585105) bank106533 +106534 POINT(46.659313991591205 -123.19752842270367) bank106534 +106535 POINT(47.549271497510595 -121.68576867522863) bank106535 +106536 POINT(48.30784358219401 -122.52145459888499) bank106536 +106537 POINT(46.65166561974271 -122.02722147960658) bank106537 +106538 POINT(47.27037460556727 -122.65642212867937) bank106538 +106539 POINT(47.00137516240734 -123.32969824056521) bank106539 +106540 POINT(48.56075072213597 -121.79908171627201) bank106540 +106541 POINT(47.535669996222396 -123.304099294054) bank106541 +106542 POINT(46.893108201115915 -122.18843348553533) bank106542 +106543 POINT(47.901147820176085 -122.28570251313589) bank106543 +106544 POINT(47.66392048519341 -122.09452398159459) bank106544 +106545 POINT(48.29395128576261 -121.34883798596097) bank106545 +106546 POINT(48.411832569080715 -123.28138693466997) bank106546 +106547 POINT(48.18021357618061 -122.49221768545495) bank106547 +106548 POINT(47.03986034034032 -122.29200505416553) bank106548 +106549 POINT(47.69736765604533 -121.50309974057669) bank106549 +106550 POINT(48.373180452649315 -122.61217188034804) bank106550 +106551 POINT(48.18794234999698 -122.01306850804133) bank106551 +106552 POINT(48.090994356317545 -121.43375492692424) bank106552 +106553 POINT(48.109085991241635 -123.08641683515683) bank106553 +106554 POINT(47.875686513455136 -121.64334140811508) bank106554 +106555 POINT(47.23453539731129 -122.15209197406027) bank106555 +106556 POINT(48.39516424285881 -121.92771159425975) bank106556 +106557 POINT(47.064425980244046 -121.4687960482938) bank106557 +106558 POINT(46.858614763761636 -122.33490667196088) bank106558 +106559 POINT(47.11572791199971 -122.83519404815168) bank106559 +106560 POINT(48.36032546955144 -122.42567543745133) bank106560 +106561 POINT(47.589091581009505 -122.19260831074166) bank106561 +106562 POINT(47.34739068440569 -121.91721724972257) bank106562 +106563 POINT(47.76785442107436 -122.75235356725028) bank106563 +106564 POINT(48.08568756810249 -121.85957745736417) bank106564 +106565 POINT(48.18197314379468 -121.71982804083027) bank106565 +106566 POINT(48.5412580169497 -122.47469165116173) bank106566 +106567 POINT(47.769375002928854 -122.3135376831007) bank106567 +106568 POINT(47.14108754397719 -123.08892051435626) bank106568 +106569 POINT(46.661645600536644 -122.11858772187833) bank106569 +106570 POINT(46.79607574694571 -121.85561095563796) bank106570 +106571 POINT(47.95078312419557 -121.43877991311078) bank106571 +106572 POINT(46.80929925062016 -123.01487730275284) bank106572 +106573 POINT(47.105743867570254 -121.46982632107519) bank106573 +106574 POINT(46.689700381479334 -122.54397326273113) bank106574 +106575 POINT(47.2352815089708 -122.69901798533525) bank106575 +106576 POINT(48.16151618157008 -123.20671184698831) bank106576 +106577 POINT(48.2222798506093 -121.35457937696272) bank106577 +106578 POINT(47.27397718604772 -122.53471849070984) bank106578 +106579 POINT(48.08869860930193 -123.00644234712445) bank106579 +106580 POINT(47.082377057186655 -122.7228518292824) bank106580 +106581 POINT(46.98299581886081 -122.1546571582947) bank106581 +106582 POINT(47.32301654357136 -121.81031382547884) bank106582 +106583 POINT(47.58150478243809 -121.91101139677677) bank106583 +106584 POINT(48.26218557562358 -122.46544146767118) bank106584 +106585 POINT(46.867569456042226 -123.1923045411877) bank106585 +106586 POINT(47.52702010563994 -122.47583013049783) bank106586 +106587 POINT(47.494975607785534 -122.99440262473335) bank106587 +106588 POINT(48.119131556388844 -122.09902164557352) bank106588 +106589 POINT(48.43102741505791 -121.4035314866148) bank106589 +106590 POINT(47.72889943431001 -121.4032829888361) bank106590 +106591 POINT(46.698534643636854 -122.2828457302801) bank106591 +106592 POINT(47.321239486986784 -122.66017455530866) bank106592 +106593 POINT(46.619438211648074 -121.78139542426702) bank106593 +106594 POINT(47.38089491286776 -121.728570078914) bank106594 +106595 POINT(47.226235149313204 -122.12613306232505) bank106595 +106596 POINT(47.22901094647549 -123.1811049033829) bank106596 +106597 POINT(47.563722367138354 -121.45867055960738) bank106597 +106598 POINT(48.17322618169406 -122.1678500285454) bank106598 +106599 POINT(48.32595207438205 -122.43335231426084) bank106599 +106600 POINT(48.28371857260893 -122.54997020572146) bank106600 +106601 POINT(47.57682203584974 -121.52216123379776) bank106601 +106602 POINT(47.60046402132722 -122.25517052858683) bank106602 +106603 POINT(48.4131679931333 -122.32415922781537) bank106603 +106604 POINT(48.24317656454902 -122.79663659508581) bank106604 +106605 POINT(47.27010471416728 -121.91320749506937) bank106605 +106606 POINT(46.75341671742064 -123.27326453261013) bank106606 +106607 POINT(47.14981465567136 -122.35673237929402) bank106607 +106608 POINT(46.82039255068801 -121.78515652190352) bank106608 +106609 POINT(46.87360238237987 -122.05241261189052) bank106609 +106610 POINT(47.44490876933836 -122.84077834808134) bank106610 +106611 POINT(47.20915017873721 -121.60016531248895) bank106611 +106612 POINT(47.89267259068226 -123.17778362249842) bank106612 +106613 POINT(47.59217224939804 -122.44118837138619) bank106613 +106614 POINT(48.02499377050655 -123.18782530704357) bank106614 +106615 POINT(47.75238517273769 -123.10031438140965) bank106615 +106616 POINT(48.309168423314304 -122.10288414139708) bank106616 +106617 POINT(47.73456525304472 -122.89630033637346) bank106617 +106618 POINT(46.92897539566849 -121.784417902127) bank106618 +106619 POINT(46.77842506322782 -122.54628951277297) bank106619 +106620 POINT(47.90525804059829 -121.86968074899005) bank106620 +106621 POINT(47.56304208663348 -122.87045541379541) bank106621 +106622 POINT(47.14245531935522 -123.18064458511876) bank106622 +106623 POINT(47.54917277768907 -122.98434105414256) bank106623 +106624 POINT(47.809006122942776 -121.50671616644331) bank106624 +106625 POINT(47.79732553092968 -122.53370385133208) bank106625 +106626 POINT(47.52454214911955 -122.6273126599953) bank106626 +106627 POINT(47.900861882800456 -121.54048152367452) bank106627 +106628 POINT(48.081492705176466 -121.33647332857082) bank106628 +106629 POINT(47.61209255647304 -121.48304837313886) bank106629 +106630 POINT(47.29450457067866 -121.38981382956811) bank106630 +106631 POINT(46.615985908556596 -122.54510902428957) bank106631 +106632 POINT(47.22942808143797 -121.38335293964518) bank106632 +106633 POINT(47.201353206989474 -121.89445580199185) bank106633 +106634 POINT(47.89107813593187 -122.79625823623854) bank106634 +106635 POINT(48.25352547421291 -122.24262455675303) bank106635 +106636 POINT(48.39462956523896 -121.43380009975128) bank106636 +106637 POINT(47.564316946887494 -122.82913672163288) bank106637 +106638 POINT(48.188654482179125 -123.24899631442322) bank106638 +106639 POINT(47.84876640195843 -122.07325649556446) bank106639 +106640 POINT(47.375571310117465 -121.81473525379158) bank106640 +106641 POINT(46.717297362991246 -121.96402876213664) bank106641 +106642 POINT(47.99092807827201 -121.39751499455764) bank106642 +106643 POINT(48.10529383766037 -122.46537824771853) bank106643 +106644 POINT(46.97466986432513 -123.1177828124039) bank106644 +106645 POINT(47.43100780648193 -122.71858366729835) bank106645 +106646 POINT(47.64078010879568 -122.67087138337375) bank106646 +106647 POINT(46.61500004703235 -122.171968396643) bank106647 +106648 POINT(46.72090719859993 -122.15458230856392) bank106648 +106649 POINT(47.26389150481337 -123.02156663385513) bank106649 +106650 POINT(47.485156179356785 -121.57013979608081) bank106650 +106651 POINT(48.34516172421258 -121.72238913754371) bank106651 +106652 POINT(47.88418355171331 -122.43447971573603) bank106652 +106653 POINT(46.88707645458637 -122.73853458909541) bank106653 +106654 POINT(47.2008929633646 -123.3244980972543) bank106654 +106655 POINT(47.184889076394384 -121.51954926317039) bank106655 +106656 POINT(47.17453312958664 -122.10617218312692) bank106656 +106657 POINT(47.29664683091234 -122.85383164939535) bank106657 +106658 POINT(48.55801375362078 -121.76857323349576) bank106658 +106659 POINT(48.034397009237985 -122.3204365346725) bank106659 +106660 POINT(48.43142244602131 -123.32392019212519) bank106660 +106661 POINT(47.081729149993336 -121.92118307298755) bank106661 +106662 POINT(47.91817448656728 -123.0882252345366) bank106662 +106663 POINT(47.07652166891215 -122.60608565584862) bank106663 +106664 POINT(47.434770878624626 -123.19573316810211) bank106664 +106665 POINT(47.07891743979618 -121.89292694254966) bank106665 +106666 POINT(46.94194830626853 -121.69796528669113) bank106666 +106667 POINT(46.797167747106954 -123.16463788756373) bank106667 +106668 POINT(47.10949454679977 -122.41070308398857) bank106668 +106669 POINT(47.17151025696322 -121.6116067461618) bank106669 +106670 POINT(47.861604965056436 -123.27102648283865) bank106670 +106671 POINT(48.603650147433115 -121.81509925859399) bank106671 +106672 POINT(47.069931667446696 -122.93401548412524) bank106672 +106673 POINT(47.5103067636894 -123.18124354650428) bank106673 +106674 POINT(47.745279053995716 -123.27368664869783) bank106674 +106675 POINT(47.30988033935849 -122.65957909558057) bank106675 +106676 POINT(47.95009466156165 -122.76017451829345) bank106676 +106677 POINT(46.92340542527649 -122.15885094617173) bank106677 +106678 POINT(47.15757547871445 -122.48586401093176) bank106678 +106679 POINT(47.201066413459465 -121.87485679388034) bank106679 +106680 POINT(48.01976012538501 -121.93055797187127) bank106680 +106681 POINT(46.73980979000133 -122.47669869892309) bank106681 +106682 POINT(47.516220613837035 -122.84938201104421) bank106682 +106683 POINT(47.490130257451355 -123.31759979441951) bank106683 +106684 POINT(47.15222653316772 -122.31625163436091) bank106684 +106685 POINT(47.541233287758374 -121.43666610708192) bank106685 +106686 POINT(46.84600627489246 -121.65741643134737) bank106686 +106687 POINT(47.34190342810916 -121.6094846369326) bank106687 +106688 POINT(47.99117046191613 -121.75405834001) bank106688 +106689 POINT(46.925327929027084 -122.25702653060556) bank106689 +106690 POINT(48.087243014017204 -123.24135017372505) bank106690 +106691 POINT(47.711960806562786 -122.37317268037687) bank106691 +106692 POINT(47.3843389643153 -122.50103277911452) bank106692 +106693 POINT(47.6603314660845 -122.16122549735215) bank106693 +106694 POINT(47.10630308134091 -122.79164153977509) bank106694 +106695 POINT(46.76197601223527 -122.4829684797337) bank106695 +106696 POINT(48.46738088147663 -121.44492845183206) bank106696 +106697 POINT(48.29983139580468 -121.4773587447016) bank106697 +106698 POINT(48.485904540212054 -121.84695343589766) bank106698 +106699 POINT(47.54791771997977 -122.27978908521776) bank106699 +106700 POINT(48.27718319036052 -122.34686426023428) bank106700 +106701 POINT(48.34211221668534 -122.56048358117155) bank106701 +106702 POINT(46.83283490805762 -121.89116990872188) bank106702 +106703 POINT(46.62983058772861 -121.78028754465726) bank106703 +106704 POINT(46.72098678468814 -122.43437628863428) bank106704 +106705 POINT(47.991107101250385 -123.13353806240565) bank106705 +106706 POINT(47.03966278679172 -121.47576242985903) bank106706 +106707 POINT(48.187767807945924 -122.13063360446681) bank106707 +106708 POINT(46.78154261862449 -121.44667739707816) bank106708 +106709 POINT(46.942969637665534 -123.02482188743656) bank106709 +106710 POINT(47.37643885220797 -122.70747872743563) bank106710 +106711 POINT(47.4186361277066 -122.63627850881694) bank106711 +106712 POINT(48.233043913212356 -121.68689902191734) bank106712 +106713 POINT(46.709204018507336 -123.15370431522429) bank106713 +106714 POINT(47.03549023154737 -122.01637649717097) bank106714 +106715 POINT(48.003917978029975 -122.89076000659335) bank106715 +106716 POINT(46.854850459853566 -122.68457230081135) bank106716 +106717 POINT(48.380934095720505 -121.42575479442341) bank106717 +106718 POINT(46.72122668954749 -121.35444304980956) bank106718 +106719 POINT(47.876243784137145 -123.27499965387754) bank106719 +106720 POINT(47.39045909268423 -122.70953039484834) bank106720 +106721 POINT(47.50137356336301 -122.51044518919353) bank106721 +106722 POINT(47.72784564735585 -122.54207678302286) bank106722 +106723 POINT(47.08773878074535 -121.84983902793088) bank106723 +106724 POINT(47.21870404734031 -122.33779758903485) bank106724 +106725 POINT(46.94857631726327 -122.34026770821671) bank106725 +106726 POINT(47.03471828747689 -122.45638698788898) bank106726 +106727 POINT(48.54356319829064 -123.11489511446277) bank106727 +106728 POINT(47.582050889046194 -123.16292496382344) bank106728 +106729 POINT(47.23584555688069 -122.22291302049854) bank106729 +106730 POINT(47.698425655152896 -121.88254586530167) bank106730 +106731 POINT(48.2426207034736 -122.88950631316973) bank106731 +106732 POINT(47.09142488901842 -121.86301087412126) bank106732 +106733 POINT(47.97760937722515 -123.06897070507415) bank106733 +106734 POINT(47.74694878329187 -121.92874550205461) bank106734 +106735 POINT(47.687107127898344 -122.29046986448859) bank106735 +106736 POINT(46.905863211308215 -121.37630073421921) bank106736 +106737 POINT(48.367461963375504 -121.94767324061557) bank106737 +106738 POINT(46.719261710216564 -122.68099968300878) bank106738 +106739 POINT(47.381854017789685 -122.71837414709012) bank106739 +106740 POINT(47.679195110873266 -123.14186566012872) bank106740 +106741 POINT(48.21101338339003 -122.9854986918723) bank106741 +106742 POINT(47.36109201349296 -121.53033536123712) bank106742 +106743 POINT(48.210035654367196 -122.86250667197386) bank106743 +106744 POINT(46.64358193927261 -122.67030570837727) bank106744 +106745 POINT(47.174442749382024 -122.49108708196742) bank106745 +106746 POINT(48.269064398366424 -122.03307724831438) bank106746 +106747 POINT(48.3601906851202 -121.73898306682527) bank106747 +106748 POINT(47.672229171544004 -123.15572965538666) bank106748 +106749 POINT(46.80789030315047 -121.69464280241384) bank106749 +106750 POINT(47.74751318573118 -122.73068582455845) bank106750 +106751 POINT(47.719615670043574 -121.36938971616118) bank106751 +106752 POINT(46.814626936291866 -122.32525626524013) bank106752 +106753 POINT(46.77313327575796 -123.1156927372025) bank106753 +106754 POINT(48.372977695858594 -122.66273033045759) bank106754 +106755 POINT(47.03667844369369 -121.59184309771929) bank106755 +106756 POINT(48.063704298192796 -123.15030922898636) bank106756 +106757 POINT(48.25237234693227 -122.4440173755761) bank106757 +106758 POINT(48.55078018234845 -121.92542989149257) bank106758 +106759 POINT(48.036172522498596 -121.74695893896825) bank106759 +106760 POINT(48.51973755319378 -121.76077034968175) bank106760 +106761 POINT(47.51137818957744 -122.88528863236147) bank106761 +106762 POINT(47.79000524301295 -122.37313354790018) bank106762 +106763 POINT(47.88489899362566 -123.22165676280329) bank106763 +106764 POINT(48.28475900544759 -123.01443639949652) bank106764 +106765 POINT(47.83854110640362 -122.95106283022622) bank106765 +106766 POINT(48.54901009222651 -121.44887352537455) bank106766 +106767 POINT(48.32952415928793 -121.59488066109827) bank106767 +106768 POINT(48.070592097812515 -122.44182463558573) bank106768 +106769 POINT(47.378915975601764 -123.16004049894077) bank106769 +106770 POINT(46.620699279167034 -122.63459426398879) bank106770 +106771 POINT(47.289384144324856 -123.13575925599216) bank106771 +106772 POINT(48.18210572341482 -121.93773248222583) bank106772 +106773 POINT(48.13075085738676 -123.13111635902118) bank106773 +106774 POINT(47.784197078455165 -121.66114514319855) bank106774 +106775 POINT(48.497056213019604 -122.098565510283) bank106775 +106776 POINT(48.03768512328892 -122.15911408008215) bank106776 +106777 POINT(47.892455269287645 -122.70865403860074) bank106777 +106778 POINT(48.16427914665308 -122.35156011919644) bank106778 +106779 POINT(47.23523975055091 -122.82829466437465) bank106779 +106780 POINT(47.974630153548205 -122.48446821634418) bank106780 +106781 POINT(48.525805257298124 -122.58622218299935) bank106781 +106782 POINT(46.719730344776046 -121.74367188237031) bank106782 +106783 POINT(48.50718240634156 -122.55963011867459) bank106783 +106784 POINT(47.37951637913135 -121.89708293703312) bank106784 +106785 POINT(47.73143667407104 -121.39913238249845) bank106785 +106786 POINT(46.85378210793175 -122.20757097192049) bank106786 +106787 POINT(46.793264473121255 -121.72719692454373) bank106787 +106788 POINT(47.379831301281975 -123.22357261503682) bank106788 +106789 POINT(47.187206804977485 -122.26784773531713) bank106789 +106790 POINT(46.78806411157341 -122.03497287251142) bank106790 +106791 POINT(48.4787916459371 -121.95629801967299) bank106791 +106792 POINT(46.86801131456611 -122.72848947133133) bank106792 +106793 POINT(47.90127511766893 -121.62167314825875) bank106793 +106794 POINT(47.24223582470488 -121.39950169789843) bank106794 +106795 POINT(47.1619322394806 -122.89583903957747) bank106795 +106796 POINT(46.900019632860705 -121.57850703349757) bank106796 +106797 POINT(48.40175236125434 -122.1670417321241) bank106797 +106798 POINT(48.44472492153221 -123.15931315167938) bank106798 +106799 POINT(47.379344976733755 -122.2258809436503) bank106799 +106800 POINT(46.6066081045215 -122.76040752205367) bank106800 +106801 POINT(47.16368080782018 -122.09310101197285) bank106801 +106802 POINT(48.25759387017372 -121.52251378471311) bank106802 +106803 POINT(47.638449625360636 -122.09603842421053) bank106803 +106804 POINT(48.332008081474626 -122.6428248065379) bank106804 +106805 POINT(46.775166880746156 -122.35772005763822) bank106805 +106806 POINT(47.288627441113704 -122.72236273648984) bank106806 +106807 POINT(48.515673262326786 -122.03891278707103) bank106807 +106808 POINT(47.284756846124054 -122.15018031889963) bank106808 +106809 POINT(47.537537394006236 -122.03477134244663) bank106809 +106810 POINT(48.16941190127638 -122.92412780268184) bank106810 +106811 POINT(47.220132420670026 -121.76695505652388) bank106811 +106812 POINT(48.17937924639306 -122.46759025324977) bank106812 +106813 POINT(48.38337150314765 -122.53801983249996) bank106813 +106814 POINT(47.470219136269236 -123.04071639618758) bank106814 +106815 POINT(46.91124918717696 -121.69411576645035) bank106815 +106816 POINT(47.131305292280736 -121.65663354965243) bank106816 +106817 POINT(47.276245130251574 -122.67986215054067) bank106817 +106818 POINT(48.045367341210905 -123.03117312978911) bank106818 +106819 POINT(47.54084526440351 -121.79383413795573) bank106819 +106820 POINT(46.71291451339808 -121.39471858693045) bank106820 +106821 POINT(48.45337404833494 -121.85620728150074) bank106821 +106822 POINT(48.06157367800267 -121.94845862098491) bank106822 +106823 POINT(48.1687574610714 -122.16647430298885) bank106823 +106824 POINT(46.95768100196193 -121.4326216545434) bank106824 +106825 POINT(47.676723509885306 -122.88365227363917) bank106825 +106826 POINT(48.48481137709966 -123.01377962352564) bank106826 +106827 POINT(48.51889689879111 -123.18333953397715) bank106827 +106828 POINT(47.68269536058751 -122.4240300673775) bank106828 +106829 POINT(47.324364314929994 -122.5999544672147) bank106829 +106830 POINT(48.29192887630231 -121.7430109444809) bank106830 +106831 POINT(47.762697277411576 -121.74078043986646) bank106831 +106832 POINT(47.40987479909883 -121.59832570427926) bank106832 +106833 POINT(46.6358662810912 -122.65528171559679) bank106833 +106834 POINT(47.7712364484152 -122.83997165990309) bank106834 +106835 POINT(48.34998583413228 -122.67929323443066) bank106835 +106836 POINT(48.181595617297695 -122.74041263909474) bank106836 +106837 POINT(48.16429860251013 -121.43611166623701) bank106837 +106838 POINT(48.424119838379085 -122.08288996280143) bank106838 +106839 POINT(47.51976945339248 -122.47429088797864) bank106839 +106840 POINT(47.56987889873758 -122.56767659081969) bank106840 +106841 POINT(48.486291892036455 -123.21380096145982) bank106841 +106842 POINT(47.485263055679894 -122.31338330026495) bank106842 +106843 POINT(48.088306927034914 -121.70653282633032) bank106843 +106844 POINT(46.726516165717634 -121.43889328759066) bank106844 +106845 POINT(47.87152465666345 -123.28690731211344) bank106845 +106846 POINT(47.518075206168014 -123.27516570981037) bank106846 +106847 POINT(47.10006845061331 -121.6747562947752) bank106847 +106848 POINT(46.93572613044491 -121.37158823640003) bank106848 +106849 POINT(47.51679465904147 -122.7312957374688) bank106849 +106850 POINT(48.139855925044536 -121.74427431504039) bank106850 +106851 POINT(47.661774452098285 -122.1662960628341) bank106851 +106852 POINT(46.768873054928974 -122.59867528353485) bank106852 +106853 POINT(47.7866065889506 -123.1315217381082) bank106853 +106854 POINT(48.208352506418024 -123.08414462915012) bank106854 +106855 POINT(48.38258558946457 -122.80171208895428) bank106855 +106856 POINT(47.85012250598903 -121.89438099200545) bank106856 +106857 POINT(46.607850883034814 -123.21851783009654) bank106857 +106858 POINT(47.99843935135628 -123.28396809961193) bank106858 +106859 POINT(48.19972843266651 -122.37924884664196) bank106859 +106860 POINT(47.725976694574776 -122.6115590068929) bank106860 +106861 POINT(48.21627676442845 -122.81194130330823) bank106861 +106862 POINT(48.0395519905717 -121.45106000200784) bank106862 +106863 POINT(47.64073634731413 -123.25903824114216) bank106863 +106864 POINT(48.37078536931325 -122.1731470973932) bank106864 +106865 POINT(48.37228021956542 -121.72802006600543) bank106865 +106866 POINT(47.440768125102636 -121.3866963043901) bank106866 +106867 POINT(47.31871977358078 -123.14356629451969) bank106867 +106868 POINT(46.80949466683631 -122.09594686138067) bank106868 +106869 POINT(47.80925879975798 -122.48569651545854) bank106869 +106870 POINT(47.58392075323798 -123.0336764248054) bank106870 +106871 POINT(48.48267011115154 -122.0615531344292) bank106871 +106872 POINT(47.45712181621721 -121.33974969631805) bank106872 +106873 POINT(47.19647760608783 -123.07007126738264) bank106873 +106874 POINT(46.6997456085265 -123.26070675690912) bank106874 +106875 POINT(48.38091761755211 -121.89451358494156) bank106875 +106876 POINT(47.14395094419537 -122.88678947251154) bank106876 +106877 POINT(48.52017458183088 -122.88306406722626) bank106877 +106878 POINT(46.67917126031959 -122.99052295728922) bank106878 +106879 POINT(47.51284988413531 -122.21293404267995) bank106879 +106880 POINT(48.49315995061258 -122.59893398323285) bank106880 +106881 POINT(47.200586561802204 -122.2524149561001) bank106881 +106882 POINT(47.798408115779694 -122.4824125749552) bank106882 +106883 POINT(48.251340388501404 -123.09088843879182) bank106883 +106884 POINT(48.5512006471451 -122.6017677977687) bank106884 +106885 POINT(46.91787129104284 -121.43918446432494) bank106885 +106886 POINT(48.33627668271135 -121.92762505029854) bank106886 +106887 POINT(46.71918254264263 -122.41297711790833) bank106887 +106888 POINT(48.56869110784865 -121.72807497299362) bank106888 +106889 POINT(46.60635325310704 -123.0618748677395) bank106889 +106890 POINT(47.67592827053335 -121.71390803978326) bank106890 +106891 POINT(47.73318126036653 -121.44557325712613) bank106891 +106892 POINT(47.98861830065829 -121.49015742413687) bank106892 +106893 POINT(47.51631034474799 -122.95171643060185) bank106893 +106894 POINT(47.60485204357326 -122.44112773164879) bank106894 +106895 POINT(47.438319625516435 -121.91931068775652) bank106895 +106896 POINT(48.00458829808541 -122.60591352900782) bank106896 +106897 POINT(48.14159591593033 -123.27205477715279) bank106897 +106898 POINT(47.689252894514695 -121.4579222264549) bank106898 +106899 POINT(46.69430205737767 -121.8999771718577) bank106899 +106900 POINT(47.310081762098235 -121.73057741525956) bank106900 +106901 POINT(47.410285515327594 -122.5033135663926) bank106901 +106902 POINT(47.68314475274105 -121.86445833228358) bank106902 +106903 POINT(47.61828559246758 -121.83897574294721) bank106903 +106904 POINT(48.442660814513296 -122.40723918696615) bank106904 +106905 POINT(47.8182750169026 -123.1567306222004) bank106905 +106906 POINT(47.591592597789614 -121.5662368170557) bank106906 +106907 POINT(48.54576175099236 -121.78586040670288) bank106907 +106908 POINT(48.4278883658187 -122.09347301524969) bank106908 +106909 POINT(46.99708516226325 -121.55024494026077) bank106909 +106910 POINT(47.82709083904135 -122.76535920165016) bank106910 +106911 POINT(47.59184160169574 -123.30930137007277) bank106911 +106912 POINT(46.74919355377182 -122.4336017753175) bank106912 +106913 POINT(47.49362458643584 -123.12794431632885) bank106913 +106914 POINT(48.4934283150212 -122.83611377184651) bank106914 +106915 POINT(47.61904141570436 -122.12428683652935) bank106915 +106916 POINT(47.12001339858751 -122.96523762058239) bank106916 +106917 POINT(47.57399392146876 -122.67298709853945) bank106917 +106918 POINT(47.55696295342682 -123.06440851856486) bank106918 +106919 POINT(47.52681054936252 -121.49705195337825) bank106919 +106920 POINT(48.2970550073335 -122.96153268095745) bank106920 +106921 POINT(47.23269426458121 -121.74740683102453) bank106921 +106922 POINT(47.84385852833489 -123.11120497896626) bank106922 +106923 POINT(47.91398722757899 -121.37385438178055) bank106923 +106924 POINT(47.95721712309734 -121.86201688364567) bank106924 +106925 POINT(47.905589158820156 -121.49094555732204) bank106925 +106926 POINT(48.15297729224247 -122.02570314639182) bank106926 +106927 POINT(47.472239471883995 -122.61531545433091) bank106927 +106928 POINT(47.263594281136974 -123.15238678859971) bank106928 +106929 POINT(48.52802693563623 -123.1484981379324) bank106929 +106930 POINT(46.91563581460772 -121.54885932638645) bank106930 +106931 POINT(46.95855599845154 -122.80318835509546) bank106931 +106932 POINT(47.99479937655297 -121.4959108884451) bank106932 +106933 POINT(48.24267978056095 -122.28760734715891) bank106933 +106934 POINT(47.88037492362053 -121.50949167128692) bank106934 +106935 POINT(47.875937806198735 -122.80132386617484) bank106935 +106936 POINT(47.93793965878517 -122.6170594749474) bank106936 +106937 POINT(47.582317062242716 -121.95356852495519) bank106937 +106938 POINT(47.82972849253841 -122.90515793456859) bank106938 +106939 POINT(47.14762897700149 -122.91401143932741) bank106939 +106940 POINT(48.45096189608052 -122.4307714846771) bank106940 +106941 POINT(46.78843988899586 -121.6478404112179) bank106941 +106942 POINT(47.66405976301795 -121.52156701808232) bank106942 +106943 POINT(47.98980664087441 -122.50310755517283) bank106943 +106944 POINT(47.57293925278974 -122.16504130185616) bank106944 +106945 POINT(47.50031530614947 -121.45682474725497) bank106945 +106946 POINT(47.38724837939352 -123.23761993847371) bank106946 +106947 POINT(47.08037048928825 -121.65228520297322) bank106947 +106948 POINT(47.502539459888254 -121.88826291213017) bank106948 +106949 POINT(47.25086620731839 -122.00771340894117) bank106949 +106950 POINT(47.94762465109613 -122.3017183530255) bank106950 +106951 POINT(47.41747568142768 -122.88862344619533) bank106951 +106952 POINT(48.391000170903716 -121.40629739808838) bank106952 +106953 POINT(48.33073919577197 -122.17118655131821) bank106953 +106954 POINT(48.120154520399566 -121.48711201723587) bank106954 +106955 POINT(46.71441760739848 -122.25686438804826) bank106955 +106956 POINT(48.08639192623943 -122.28663001692453) bank106956 +106957 POINT(47.84272521310462 -123.07553709841635) bank106957 +106958 POINT(46.8952451326596 -121.8748333212583) bank106958 +106959 POINT(46.72509971956986 -122.68303821487922) bank106959 +106960 POINT(48.377062027471084 -122.32585941828201) bank106960 +106961 POINT(46.74579721306628 -121.82612809138091) bank106961 +106962 POINT(48.21365661396324 -121.4796709389195) bank106962 +106963 POINT(48.2118631761421 -122.6466486597387) bank106963 +106964 POINT(46.996786488216316 -122.00602150513417) bank106964 +106965 POINT(47.12256634128071 -121.56972109957367) bank106965 +106966 POINT(47.957844912775855 -123.23529874634063) bank106966 +106967 POINT(48.27539897773279 -121.45568614178592) bank106967 +106968 POINT(47.67422756007958 -121.69421676387283) bank106968 +106969 POINT(47.68352485870955 -121.59647310859381) bank106969 +106970 POINT(47.267116152409905 -121.36955512240854) bank106970 +106971 POINT(46.81336967023703 -122.69397222940577) bank106971 +106972 POINT(48.1792524364707 -122.97436990073656) bank106972 +106973 POINT(46.92855595234435 -122.20547509822481) bank106973 +106974 POINT(47.09041453790632 -121.63274994047843) bank106974 +106975 POINT(48.0597284481589 -122.44617435973646) bank106975 +106976 POINT(47.16680813233013 -122.84359726254523) bank106976 +106977 POINT(46.803478653754546 -122.82899711464188) bank106977 +106978 POINT(46.632169956250536 -121.5652969095065) bank106978 +106979 POINT(46.765792943587776 -121.95377428666623) bank106979 +106980 POINT(47.99895027000227 -123.31057502973724) bank106980 +106981 POINT(48.039761698529524 -121.54068007489019) bank106981 +106982 POINT(48.14955344830012 -121.71976263422938) bank106982 +106983 POINT(46.91714830212446 -121.49231291997647) bank106983 +106984 POINT(47.48128991904819 -121.76474004991825) bank106984 +106985 POINT(46.60724267087765 -122.40168618294672) bank106985 +106986 POINT(47.318432362135496 -121.3328386511399) bank106986 +106987 POINT(47.94692950475785 -122.55710162868486) bank106987 +106988 POINT(47.39773741269897 -121.53043748973413) bank106988 +106989 POINT(47.348637441270306 -122.76621889156449) bank106989 +106990 POINT(47.89897173347259 -123.23401477669307) bank106990 +106991 POINT(48.36299271940237 -121.8425571535081) bank106991 +106992 POINT(47.097615232901376 -122.76864534543428) bank106992 +106993 POINT(46.645486164084495 -123.03252522853313) bank106993 +106994 POINT(46.78962652245618 -122.84357495576536) bank106994 +106995 POINT(47.30048966563356 -121.65887229427787) bank106995 +106996 POINT(47.408472436967664 -122.54659249153991) bank106996 +106997 POINT(47.27138115068268 -122.00612784827311) bank106997 +106998 POINT(47.58633561082554 -121.82010578034745) bank106998 +106999 POINT(48.59317386721057 -122.97637501319947) bank106999 +107000 POINT(48.497064733680716 -121.41145827444024) bank107000 +107001 POINT(48.077816975388 -123.23525539337271) bank107001 +107002 POINT(48.42025045410315 -121.71436761101592) bank107002 +107003 POINT(47.997148231351225 -122.6303790779147) bank107003 +107004 POINT(48.57269816938865 -123.09785672544879) bank107004 +107005 POINT(47.692342144627 -121.7556509176907) bank107005 +107006 POINT(48.09766667398528 -122.54902045896729) bank107006 +107007 POINT(47.633650925218426 -121.68312934064723) bank107007 +107008 POINT(46.938683931751264 -121.40838713388295) bank107008 +107009 POINT(47.05921602748096 -121.69723547078998) bank107009 +107010 POINT(47.28977500938186 -122.0277166920345) bank107010 +107011 POINT(47.77344383977099 -123.20359916463863) bank107011 +107012 POINT(48.26487983623444 -121.42898429651787) bank107012 +107013 POINT(47.65676111242937 -122.34794492269538) bank107013 +107014 POINT(46.689562574491866 -122.64806782996742) bank107014 +107015 POINT(48.30425743167601 -121.74829555933006) bank107015 +107016 POINT(47.79478530610104 -121.89448798068018) bank107016 +107017 POINT(46.7301038055134 -122.72243720162719) bank107017 +107018 POINT(48.44452461932894 -123.11820616574651) bank107018 +107019 POINT(46.990505361622056 -121.61309786611602) bank107019 +107020 POINT(48.441418244686204 -123.30623100404034) bank107020 +107021 POINT(47.256451505601724 -122.4759392235902) bank107021 +107022 POINT(46.72925644685018 -122.63740135058029) bank107022 +107023 POINT(47.07895013113529 -121.72322252922659) bank107023 +107024 POINT(47.4992980388266 -121.93669287701633) bank107024 +107025 POINT(47.87127789161119 -122.39053734717463) bank107025 +107026 POINT(46.95002867025674 -122.5131707610844) bank107026 +107027 POINT(46.72417998597797 -123.24322667219636) bank107027 +107028 POINT(48.59499257200673 -122.81190036107773) bank107028 +107029 POINT(48.60253200916399 -122.6735713312638) bank107029 +107030 POINT(47.625673556824815 -122.17769946218209) bank107030 +107031 POINT(46.64115090801173 -122.22661523254219) bank107031 +107032 POINT(47.3927137153129 -122.89610244389642) bank107032 +107033 POINT(47.56710063388635 -122.51653815508166) bank107033 +107034 POINT(46.730975154546485 -122.34105005215264) bank107034 +107035 POINT(47.03471657094064 -122.1495312212329) bank107035 +107036 POINT(47.09012500817484 -121.38346241946861) bank107036 +107037 POINT(47.297381759496474 -121.72092981560615) bank107037 +107038 POINT(48.53610013502913 -123.01307061528597) bank107038 +107039 POINT(46.686585569857094 -122.41574074204877) bank107039 +107040 POINT(47.91567797486926 -123.19241546192266) bank107040 +107041 POINT(47.762778465382816 -121.96500190647049) bank107041 +107042 POINT(47.9551789217305 -122.23725136754284) bank107042 +107043 POINT(46.724445952543036 -122.9002307545031) bank107043 +107044 POINT(47.20317957895044 -122.00483996887046) bank107044 +107045 POINT(47.45741633902244 -122.52645341944157) bank107045 +107046 POINT(46.75481454826062 -122.45324382786237) bank107046 +107047 POINT(46.714903863287184 -122.01379423973387) bank107047 +107048 POINT(47.013882491297494 -122.45453975034177) bank107048 +107049 POINT(48.20956899202723 -122.49540862913446) bank107049 +107050 POINT(47.66559549875592 -121.86779896881579) bank107050 +107051 POINT(46.904819258935966 -122.74716143670493) bank107051 +107052 POINT(46.89432393486862 -122.34481235255345) bank107052 +107053 POINT(47.67372812281244 -121.4424218370631) bank107053 +107054 POINT(46.67439738694292 -121.41946979074181) bank107054 +107055 POINT(47.100981874696586 -121.867545677809) bank107055 +107056 POINT(46.70682481321842 -121.93502459719983) bank107056 +107057 POINT(48.22344948459407 -122.43927278106338) bank107057 +107058 POINT(47.120796059217035 -122.92557102750472) bank107058 +107059 POINT(48.38390319345543 -122.89566619474073) bank107059 +107060 POINT(47.395298870932194 -121.78008042905444) bank107060 +107061 POINT(47.13597798999596 -122.57235957563519) bank107061 +107062 POINT(48.166623992012774 -121.79847684811271) bank107062 +107063 POINT(47.77930097044595 -122.77997321976599) bank107063 +107064 POINT(47.233463267199696 -122.0896895925185) bank107064 +107065 POINT(47.16665735181969 -123.10208630037222) bank107065 +107066 POINT(47.26938232334596 -122.23332868861674) bank107066 +107067 POINT(47.35967515210826 -121.89450456137085) bank107067 +107068 POINT(47.18473847696532 -122.83337767861998) bank107068 +107069 POINT(48.115375572228196 -122.25645026326445) bank107069 +107070 POINT(47.422390245366756 -122.8625998765808) bank107070 +107071 POINT(47.64328233482737 -123.12341937693095) bank107071 +107072 POINT(47.75539800340777 -121.8445550517427) bank107072 +107073 POINT(48.43895191420255 -122.11860116479433) bank107073 +107074 POINT(48.30550035900606 -122.73272451062735) bank107074 +107075 POINT(46.976921278260654 -122.10408922822671) bank107075 +107076 POINT(48.046909157761554 -123.1507301626604) bank107076 +107077 POINT(48.55203412097668 -123.32598393711456) bank107077 +107078 POINT(47.70101908810391 -122.9474575101075) bank107078 +107079 POINT(48.39982908813681 -122.4984348155855) bank107079 +107080 POINT(47.96808310252127 -123.2614687437006) bank107080 +107081 POINT(47.726908165089114 -122.32087112073366) bank107081 +107082 POINT(47.35734352992003 -122.5198404662099) bank107082 +107083 POINT(47.977735609853156 -122.5421089265659) bank107083 +107084 POINT(46.96494910107455 -121.57279363938056) bank107084 +107085 POINT(47.37676952354324 -121.62643636922087) bank107085 +107086 POINT(47.97463259922551 -123.22212981488526) bank107086 +107087 POINT(47.47770822934472 -122.09855850916159) bank107087 +107088 POINT(47.76519067434712 -121.39296108489715) bank107088 +107089 POINT(47.867381058665224 -123.00700191455331) bank107089 +107090 POINT(48.50117260206263 -121.82061580515727) bank107090 +107091 POINT(47.32260308693111 -121.82974358677336) bank107091 +107092 POINT(46.981937650843875 -121.59239900428813) bank107092 +107093 POINT(47.70379725185927 -122.07290903661153) bank107093 +107094 POINT(46.69588346067903 -122.25593727070365) bank107094 +107095 POINT(47.58569476703977 -122.68097011242273) bank107095 +107096 POINT(47.65806371537719 -121.52323371791684) bank107096 +107097 POINT(48.36262055364396 -121.6018247781544) bank107097 +107098 POINT(48.17840231359795 -122.58740772665774) bank107098 +107099 POINT(47.196334354684076 -122.18230201318626) bank107099 +107100 POINT(46.783787855160135 -121.34007001270552) bank107100 +107101 POINT(46.762378809546604 -121.78267851974867) bank107101 +107102 POINT(48.35486258526067 -121.82878689257024) bank107102 +107103 POINT(48.43304887728359 -121.86153240499877) bank107103 +107104 POINT(47.79807915006358 -122.92263211753757) bank107104 +107105 POINT(47.410678756861266 -122.8491940952895) bank107105 +107106 POINT(47.354705005322394 -121.66585816332379) bank107106 +107107 POINT(48.00003314176503 -121.44201600671299) bank107107 +107108 POINT(47.841280780361345 -122.76170946657304) bank107108 +107109 POINT(48.33835936960309 -122.78377822446005) bank107109 +107110 POINT(48.06487336233756 -122.32084630275266) bank107110 +107111 POINT(47.153080559682564 -121.4365246996376) bank107111 +107112 POINT(48.54197100825964 -122.16896566533798) bank107112 +107113 POINT(46.66535947984692 -121.45123526788812) bank107113 +107114 POINT(47.31393684464732 -121.69099384743033) bank107114 +107115 POINT(47.774080169701904 -122.0524263958889) bank107115 +107116 POINT(46.969372519387846 -122.22179538978361) bank107116 +107117 POINT(46.68828440300671 -122.4403316368882) bank107117 +107118 POINT(47.72707188582471 -122.68145280394862) bank107118 +107119 POINT(47.03236741989127 -123.1460143024286) bank107119 +107120 POINT(47.575077779594245 -122.32363577050063) bank107120 +107121 POINT(46.73919211602587 -121.77078394328441) bank107121 +107122 POINT(46.869354606353085 -122.37740692936868) bank107122 +107123 POINT(47.17117650401145 -123.33036715227043) bank107123 +107124 POINT(48.59612996022643 -122.04830977380482) bank107124 +107125 POINT(47.126965496098315 -121.63740330166418) bank107125 +107126 POINT(47.471157476374465 -121.39272443803742) bank107126 +107127 POINT(48.11813104872027 -123.04438394045238) bank107127 +107128 POINT(48.48120131548818 -121.78118161546304) bank107128 +107129 POINT(48.273409906140124 -123.25536955130765) bank107129 +107130 POINT(48.484321557440964 -123.25495143738628) bank107130 +107131 POINT(47.52973234753692 -121.39060110790768) bank107131 +107132 POINT(47.7310699482749 -122.36261829189043) bank107132 +107133 POINT(46.7353492016769 -123.04265147188306) bank107133 +107134 POINT(47.5527306736236 -121.46163490236478) bank107134 +107135 POINT(46.73522745880291 -122.99452818525832) bank107135 +107136 POINT(47.05005366619644 -122.42401142155161) bank107136 +107137 POINT(48.09304990117045 -121.86657830800469) bank107137 +107138 POINT(47.42817552837463 -121.79014552733102) bank107138 +107139 POINT(47.51984364273792 -121.58015406449714) bank107139 +107140 POINT(48.34076347504017 -121.71524342579986) bank107140 +107141 POINT(46.79238180700326 -123.14616937483892) bank107141 +107142 POINT(48.35778814140154 -122.66333346293756) bank107142 +107143 POINT(47.81990175250021 -122.62839244656921) bank107143 +107144 POINT(46.618349201517006 -123.10135568859917) bank107144 +107145 POINT(46.73761962145752 -122.6342989847385) bank107145 +107146 POINT(46.87832061774038 -122.99352687624793) bank107146 +107147 POINT(48.140254928869375 -123.02925626637196) bank107147 +107148 POINT(48.05040668168715 -121.36217873816754) bank107148 +107149 POINT(48.51933529560137 -121.43200510262416) bank107149 +107150 POINT(48.23066358740153 -122.39413083447208) bank107150 +107151 POINT(47.68168250489998 -123.2245724986991) bank107151 +107152 POINT(47.60147127565064 -122.69397145424087) bank107152 +107153 POINT(47.716505715204576 -122.83808696536896) bank107153 +107154 POINT(47.59607502055047 -121.90470931980222) bank107154 +107155 POINT(48.26763763333819 -123.09571064820274) bank107155 +107156 POINT(46.717099847759364 -122.75014694259102) bank107156 +107157 POINT(47.700824638866344 -121.91988649958141) bank107157 +107158 POINT(48.303658469887566 -123.21834657748573) bank107158 +107159 POINT(48.31642441467013 -123.10820232659228) bank107159 +107160 POINT(47.43723997080626 -123.26678256184677) bank107160 +107161 POINT(47.9032707987741 -121.98581162585619) bank107161 +107162 POINT(48.51146308271844 -122.22385771284144) bank107162 +107163 POINT(48.30791559243458 -122.43968457761156) bank107163 +107164 POINT(47.94904483455659 -122.39697911201833) bank107164 +107165 POINT(48.287133806509246 -121.95200047407417) bank107165 +107166 POINT(47.280390525731995 -123.29469099119923) bank107166 +107167 POINT(48.365379268861595 -123.32523848143096) bank107167 +107168 POINT(47.7771830794425 -121.73955002173201) bank107168 +107169 POINT(47.390650159987295 -122.87618636046078) bank107169 +107170 POINT(47.529803702178064 -122.17038004440397) bank107170 +107171 POINT(47.11375115956529 -122.63478269650561) bank107171 +107172 POINT(48.40928269767781 -121.40706819353086) bank107172 +107173 POINT(47.30466219195358 -121.80812604559749) bank107173 +107174 POINT(48.16739971956049 -123.23587055652389) bank107174 +107175 POINT(46.79353612285482 -123.30920662113344) bank107175 +107176 POINT(47.53050886687758 -121.37496643035864) bank107176 +107177 POINT(46.95928970059716 -121.50263161404003) bank107177 +107178 POINT(47.360254081331135 -122.87779250231218) bank107178 +107179 POINT(47.59707936700129 -121.6254529264242) bank107179 +107180 POINT(47.22166978516845 -121.57249443831637) bank107180 +107181 POINT(47.409376214280876 -121.4642549941938) bank107181 +107182 POINT(48.37341726735955 -122.84066558894042) bank107182 +107183 POINT(48.32159995154698 -122.64218421996117) bank107183 +107184 POINT(48.05666185077287 -122.87172767511355) bank107184 +107185 POINT(48.1257061048224 -122.7566039579866) bank107185 +107186 POINT(47.35901283727724 -121.92315804217259) bank107186 +107187 POINT(47.28550379726774 -122.59448053071034) bank107187 +107188 POINT(47.835837242278366 -122.60818398262805) bank107188 +107189 POINT(46.80117937488332 -122.59257468613221) bank107189 +107190 POINT(46.71942277077715 -122.21173051337851) bank107190 +107191 POINT(48.52713618412457 -121.59135455571806) bank107191 +107192 POINT(47.17698357741263 -122.51642106191184) bank107192 +107193 POINT(47.77328244562081 -122.2277622839641) bank107193 +107194 POINT(46.905054748646556 -121.51115549027412) bank107194 +107195 POINT(48.5068554561959 -121.48143047959192) bank107195 +107196 POINT(47.00731414550786 -121.68820863543422) bank107196 +107197 POINT(47.91003388752835 -122.20551140347132) bank107197 +107198 POINT(47.72702220790654 -122.76299805963409) bank107198 +107199 POINT(47.72175607204775 -122.0349386941278) bank107199 +107200 POINT(47.38201729496158 -121.40440660388997) bank107200 +107201 POINT(48.28129701677557 -121.70140304449474) bank107201 +107202 POINT(47.131947013847636 -121.77725900909356) bank107202 +107203 POINT(46.61464350815472 -122.49183450671141) bank107203 +107204 POINT(47.145433594548635 -123.21067782016107) bank107204 +107205 POINT(47.70213779007777 -122.95313999975295) bank107205 +107206 POINT(46.86134031639138 -122.20997102697389) bank107206 +107207 POINT(46.610679978614144 -121.84045138511792) bank107207 +107208 POINT(48.338462944138065 -121.80202210281936) bank107208 +107209 POINT(46.76216975463158 -122.03022166103489) bank107209 +107210 POINT(47.47459571381858 -121.69973750633976) bank107210 +107211 POINT(48.311568538737774 -121.36745418513212) bank107211 +107212 POINT(48.129868276488885 -122.73579323845084) bank107212 +107213 POINT(47.90067092801455 -123.04132315401492) bank107213 +107214 POINT(47.10688148240771 -121.48149759972893) bank107214 +107215 POINT(46.82552006282561 -123.186649968006) bank107215 +107216 POINT(47.69526234434702 -122.11255964712117) bank107216 +107217 POINT(47.287124950701156 -122.5780556566854) bank107217 +107218 POINT(46.96985114222753 -121.67351720155024) bank107218 +107219 POINT(48.05491146978021 -122.382240214628) bank107219 +107220 POINT(47.8198378310095 -123.13650984169469) bank107220 +107221 POINT(48.50624413392764 -121.65529192828458) bank107221 +107222 POINT(48.059769473429675 -121.79312803808787) bank107222 +107223 POINT(47.854971893846546 -121.52135508331607) bank107223 +107224 POINT(47.51662664243416 -121.58369256110657) bank107224 +107225 POINT(47.669381948970056 -122.8638958795353) bank107225 +107226 POINT(48.419761964077885 -122.62475134767165) bank107226 +107227 POINT(47.21249165837565 -123.00541784157005) bank107227 +107228 POINT(48.20195197334666 -122.97960396659604) bank107228 +107229 POINT(46.897553923884914 -122.1682981126958) bank107229 +107230 POINT(48.37058305265626 -121.55263799410645) bank107230 +107231 POINT(48.3970458839502 -123.03520482142169) bank107231 +107232 POINT(48.353626244521415 -122.28389757094597) bank107232 +107233 POINT(47.696359767988824 -122.08017814089631) bank107233 +107234 POINT(46.79678648520663 -121.8522800291691) bank107234 +107235 POINT(46.654309856446076 -121.6593034354711) bank107235 +107236 POINT(47.40864658079896 -122.93989080804582) bank107236 +107237 POINT(47.09138234530817 -123.16204530944215) bank107237 +107238 POINT(47.0442347813392 -122.7803902146461) bank107238 +107239 POINT(47.1787419845236 -123.02408170822044) bank107239 +107240 POINT(47.003103628846965 -123.09256579977877) bank107240 +107241 POINT(47.508411310634514 -122.5674634105177) bank107241 +107242 POINT(46.99053264011415 -122.15747411063677) bank107242 +107243 POINT(46.83726407761659 -121.80770363305687) bank107243 +107244 POINT(46.91944862917416 -123.27854922820553) bank107244 +107245 POINT(47.14990135275801 -121.99280628417314) bank107245 +107246 POINT(47.12659877605282 -121.55697168501685) bank107246 +107247 POINT(47.18992185928504 -121.72311302811367) bank107247 +107248 POINT(48.27868230821158 -121.44643795351321) bank107248 +107249 POINT(48.40956149627421 -122.54355410486092) bank107249 +107250 POINT(48.203819389279246 -121.47393354144457) bank107250 +107251 POINT(48.45698700016973 -122.85017488655379) bank107251 +107252 POINT(47.67350541289846 -121.37165659945448) bank107252 +107253 POINT(47.56957182958487 -122.23737296853679) bank107253 +107254 POINT(47.29794816974802 -123.07590275302422) bank107254 +107255 POINT(48.025052261902864 -123.1891285901018) bank107255 +107256 POINT(48.19435273594661 -123.14871474410403) bank107256 +107257 POINT(47.5266442009645 -121.37979999241166) bank107257 +107258 POINT(48.586829732742984 -121.92841307373892) bank107258 +107259 POINT(47.17919942249414 -123.07037959963978) bank107259 +107260 POINT(47.55869686884965 -121.76177429720002) bank107260 +107261 POINT(46.62079717810546 -121.82634297501713) bank107261 +107262 POINT(47.5614306351173 -122.77451080496988) bank107262 +107263 POINT(47.404342712239384 -122.84129729969948) bank107263 +107264 POINT(47.9271629765042 -123.02662892700417) bank107264 +107265 POINT(48.33698813362452 -123.30125917269136) bank107265 +107266 POINT(48.12951230407129 -121.94040804109312) bank107266 +107267 POINT(47.95459481466665 -122.02483668036967) bank107267 +107268 POINT(46.77663218085625 -121.4292764728188) bank107268 +107269 POINT(48.44075482909818 -122.231420032017) bank107269 +107270 POINT(47.152993915734946 -121.62252282486543) bank107270 +107271 POINT(46.87895993063092 -122.86256859380795) bank107271 +107272 POINT(46.9490951532943 -122.2854315239055) bank107272 +107273 POINT(47.86261574404314 -121.34001417653266) bank107273 +107274 POINT(48.51392641586034 -122.87924033228386) bank107274 +107275 POINT(48.220980057664725 -122.38907943916278) bank107275 +107276 POINT(48.169123326437635 -123.1688545032653) bank107276 +107277 POINT(47.98013730710534 -122.74346854950873) bank107277 +107278 POINT(48.39088426481219 -122.7906730253182) bank107278 +107279 POINT(46.70205430339408 -122.19327479459909) bank107279 +107280 POINT(48.3421758967265 -121.4817462719641) bank107280 +107281 POINT(48.46439101798502 -122.32023052936107) bank107281 +107282 POINT(46.92673340195827 -121.34234321159927) bank107282 +107283 POINT(47.65537910670868 -122.2940615019116) bank107283 +107284 POINT(47.776608008907964 -121.74330319534533) bank107284 +107285 POINT(47.45581566802076 -122.49684481085117) bank107285 +107286 POINT(47.42936825034517 -122.03778052835982) bank107286 +107287 POINT(47.1474414477352 -122.91422152363322) bank107287 +107288 POINT(48.2484960704636 -123.25582940906632) bank107288 +107289 POINT(47.40318523446095 -123.05653701658284) bank107289 +107290 POINT(48.26054975003817 -122.90789380176892) bank107290 +107291 POINT(48.105792647180415 -122.1442073253122) bank107291 +107292 POINT(47.794106110284254 -122.37498443612755) bank107292 +107293 POINT(48.12729385557922 -122.57639300546067) bank107293 +107294 POINT(46.82726873847568 -123.2873033293442) bank107294 +107295 POINT(48.3659278415069 -121.94595847674749) bank107295 +107296 POINT(46.609918001871755 -122.9857224147893) bank107296 +107297 POINT(48.38661581984768 -121.39319953312753) bank107297 +107298 POINT(47.25302236893527 -122.18741094288802) bank107298 +107299 POINT(47.306937536786705 -121.76908710496241) bank107299 +107300 POINT(48.116264558472274 -122.42144985376099) bank107300 +107301 POINT(47.797059687752075 -121.69712299654375) bank107301 +107302 POINT(46.608115533103465 -121.86992401361728) bank107302 +107303 POINT(48.03523716087143 -122.01821029646106) bank107303 +107304 POINT(47.32420222736343 -122.61771150916483) bank107304 +107305 POINT(46.942619293363364 -122.87671451994626) bank107305 +107306 POINT(46.94016970471176 -121.33509164592346) bank107306 +107307 POINT(48.47426183772244 -121.97428748488922) bank107307 +107308 POINT(47.45624937541077 -122.81914872616592) bank107308 +107309 POINT(47.15818148998103 -122.02608298551931) bank107309 +107310 POINT(47.86928612869393 -122.81089116388667) bank107310 +107311 POINT(46.94784089725366 -121.87758100665219) bank107311 +107312 POINT(48.37756508500931 -121.40718466852886) bank107312 +107313 POINT(48.20475567667219 -121.95074024158643) bank107313 +107314 POINT(46.80254984498681 -121.90151585889554) bank107314 +107315 POINT(47.24282504143886 -122.02399170882673) bank107315 +107316 POINT(47.774204804371266 -123.23820923624834) bank107316 +107317 POINT(48.51909189070704 -122.8388189659517) bank107317 +107318 POINT(48.59986221464824 -122.99862993810939) bank107318 +107319 POINT(48.2162530285632 -122.68083189737743) bank107319 +107320 POINT(47.13566670109236 -121.68751665906417) bank107320 +107321 POINT(48.05969266076993 -121.43326489532016) bank107321 +107322 POINT(48.041122738660505 -123.12683721934097) bank107322 +107323 POINT(47.289020917971285 -122.83779317456207) bank107323 +107324 POINT(46.936877320741786 -122.68300527838399) bank107324 +107325 POINT(46.94347567741789 -121.4906551570411) bank107325 +107326 POINT(48.50900817768746 -122.26254751312247) bank107326 +107327 POINT(47.56559343288344 -122.68957556174132) bank107327 +107328 POINT(47.27522391474717 -122.14288145122346) bank107328 +107329 POINT(47.05449833581438 -122.35468288928331) bank107329 +107330 POINT(46.675428543234936 -122.10147794233805) bank107330 +107331 POINT(46.756629561299874 -122.79875850800234) bank107331 +107332 POINT(47.2557262871759 -121.91939784687804) bank107332 +107333 POINT(47.6506586145771 -122.6370161689745) bank107333 +107334 POINT(47.195117881316264 -122.09104287786734) bank107334 +107335 POINT(46.664935602740556 -122.921670244421) bank107335 +107336 POINT(47.70269477383707 -123.10325972939847) bank107336 +107337 POINT(47.292495831241226 -121.35843386009182) bank107337 +107338 POINT(48.4950391259206 -122.38787821078704) bank107338 +107339 POINT(47.56340954371025 -123.30915758989619) bank107339 +107340 POINT(47.282608448773374 -121.64325789233489) bank107340 +107341 POINT(48.54911342629172 -122.22074810777352) bank107341 +107342 POINT(48.56387176175407 -121.92262408745123) bank107342 +107343 POINT(48.38700765777033 -121.76106965819916) bank107343 +107344 POINT(46.92366354130531 -121.9634291598499) bank107344 +107345 POINT(48.05353036272632 -122.90632986515683) bank107345 +107346 POINT(47.67871744489483 -122.8991374220778) bank107346 +107347 POINT(47.689341574602004 -123.17636410466659) bank107347 +107348 POINT(48.00465531318901 -122.06019839811154) bank107348 +107349 POINT(47.061390983822825 -121.95574190536831) bank107349 +107350 POINT(48.01178502949124 -121.48543214612748) bank107350 +107351 POINT(46.835167281491934 -122.81858927226367) bank107351 +107352 POINT(47.45235235247118 -123.28235821129772) bank107352 +107353 POINT(47.22656636355859 -121.98524117833833) bank107353 +107354 POINT(48.14400522187607 -122.94359489387594) bank107354 +107355 POINT(46.8914107558753 -123.26562069936593) bank107355 +107356 POINT(48.44670138683595 -121.80907170115803) bank107356 +107357 POINT(48.19007382804744 -122.33188273433605) bank107357 +107358 POINT(46.67232625261471 -121.92113765921236) bank107358 +107359 POINT(48.03233131192203 -122.13290585706383) bank107359 +107360 POINT(48.00018220759034 -122.68927386730037) bank107360 +107361 POINT(48.482793378226944 -121.58988368276566) bank107361 +107362 POINT(48.445826124964164 -122.14612304599802) bank107362 +107363 POINT(46.85914005602841 -123.08296983017999) bank107363 +107364 POINT(48.26914610516973 -123.1396601809988) bank107364 +107365 POINT(48.08066965928123 -123.05657492684783) bank107365 +107366 POINT(46.83623349641619 -122.22400798977141) bank107366 +107367 POINT(47.718249750220316 -121.399701391594) bank107367 +107368 POINT(47.904598918534006 -122.53178184445058) bank107368 +107369 POINT(46.93250718305059 -122.23046013822734) bank107369 +107370 POINT(48.48457373863266 -121.91287008618484) bank107370 +107371 POINT(48.463476398478186 -122.3231945394294) bank107371 +107372 POINT(47.915868084908 -121.61568274620787) bank107372 +107373 POINT(48.0434908207274 -122.02169278320355) bank107373 +107374 POINT(47.562297954166425 -122.111177671921) bank107374 +107375 POINT(46.679665258638906 -121.37690627028982) bank107375 +107376 POINT(48.12144879716527 -121.93051414216328) bank107376 +107377 POINT(47.73234420042894 -121.80166417804512) bank107377 +107378 POINT(47.261307986521935 -122.03303975541327) bank107378 +107379 POINT(48.350117409287876 -121.738197190186) bank107379 +107380 POINT(47.111833345822824 -122.1902821897124) bank107380 +107381 POINT(48.37364187821019 -121.66505660002943) bank107381 +107382 POINT(46.80937804622148 -122.30920657531917) bank107382 +107383 POINT(47.256207972048 -121.60235418522672) bank107383 +107384 POINT(48.38808378433753 -121.35828801998896) bank107384 +107385 POINT(48.41946355126746 -122.25080912095194) bank107385 +107386 POINT(46.77767979807302 -123.02120522764648) bank107386 +107387 POINT(47.39371404061193 -123.26617987041125) bank107387 +107388 POINT(48.05299845802553 -122.95072509028664) bank107388 +107389 POINT(47.26739248389954 -122.04808521600623) bank107389 +107390 POINT(48.568220453755444 -122.2177337064837) bank107390 +107391 POINT(47.64784941739 -123.00800381982569) bank107391 +107392 POINT(47.61710278096916 -122.40850428734396) bank107392 +107393 POINT(46.64745397282615 -122.9646966112239) bank107393 +107394 POINT(47.06848907126688 -122.10635568551254) bank107394 +107395 POINT(48.39718450533222 -122.73678735296912) bank107395 +107396 POINT(47.111803438864506 -123.11583767355783) bank107396 +107397 POINT(47.4975380666981 -123.33022637684519) bank107397 +107398 POINT(46.72997577799346 -123.06216075802067) bank107398 +107399 POINT(47.77371852753434 -121.86047275404263) bank107399 +107400 POINT(48.50348970512542 -121.8483111232345) bank107400 +107401 POINT(48.076873264509565 -122.40513956879226) bank107401 +107402 POINT(47.920961526774285 -121.35456301827755) bank107402 +107403 POINT(48.01646574631671 -121.86978955099873) bank107403 +107404 POINT(48.59131357067996 -122.69006162513298) bank107404 +107405 POINT(47.34409610599949 -122.13550925869644) bank107405 +107406 POINT(48.521133489013536 -122.78321546155435) bank107406 +107407 POINT(47.412369609627184 -122.99992586134255) bank107407 +107408 POINT(46.85479910494708 -121.54842063217855) bank107408 +107409 POINT(46.89630261494436 -121.73314573325227) bank107409 +107410 POINT(46.686274301443 -121.53574105777264) bank107410 +107411 POINT(48.208230813307324 -123.2151987412024) bank107411 +107412 POINT(47.5113409732124 -122.14942599285804) bank107412 +107413 POINT(47.60129946403819 -121.81315607636093) bank107413 +107414 POINT(46.80100966917163 -121.67675505887108) bank107414 +107415 POINT(48.53762258483457 -122.34055192103014) bank107415 +107416 POINT(46.997975332090455 -122.08582648621758) bank107416 +107417 POINT(46.6483251782814 -122.80904670041639) bank107417 +107418 POINT(47.227904603667774 -122.5670364329903) bank107418 +107419 POINT(48.154483912619675 -122.58606438031431) bank107419 +107420 POINT(47.19778323925869 -122.86075380569636) bank107420 +107421 POINT(47.62150496616961 -122.52406247768494) bank107421 +107422 POINT(48.4281953769508 -122.56253612189752) bank107422 +107423 POINT(47.634233654068 -122.81198956539812) bank107423 +107424 POINT(48.16348686203514 -121.73579507298605) bank107424 +107425 POINT(48.00372810032129 -122.60453270542503) bank107425 +107426 POINT(48.393623712215025 -123.08540311749243) bank107426 +107427 POINT(47.7122233350101 -121.80728860234109) bank107427 +107428 POINT(48.193149438656214 -123.03436706565387) bank107428 +107429 POINT(46.83636244313161 -122.38427256228795) bank107429 +107430 POINT(48.33295949412363 -122.23955952392635) bank107430 +107431 POINT(47.43738410366791 -121.65053536082192) bank107431 +107432 POINT(47.05565196105666 -122.75640966831153) bank107432 +107433 POINT(47.06139535611243 -122.01359601988068) bank107433 +107434 POINT(47.11844047969829 -123.10145119907298) bank107434 +107435 POINT(48.07099547011976 -121.3484335877972) bank107435 +107436 POINT(48.350979283281006 -121.64096056941486) bank107436 +107437 POINT(48.121342020004846 -121.83268811027239) bank107437 +107438 POINT(48.34535811943369 -122.50672297095431) bank107438 +107439 POINT(48.428796380909176 -122.79422476580505) bank107439 +107440 POINT(47.02202156708797 -122.23107721347581) bank107440 +107441 POINT(47.752527886210395 -123.14045349833198) bank107441 +107442 POINT(47.7910697885186 -122.76676057440058) bank107442 +107443 POINT(46.839758694760015 -123.26809612617217) bank107443 +107444 POINT(48.3683500918946 -122.39528713085396) bank107444 +107445 POINT(47.683381002583204 -122.95532765596748) bank107445 +107446 POINT(46.862342809485654 -121.39472515584454) bank107446 +107447 POINT(46.73484520948674 -121.82886381086585) bank107447 +107448 POINT(47.09781084326486 -121.98706642827244) bank107448 +107449 POINT(46.67983093336345 -122.66224614544291) bank107449 +107450 POINT(47.89812062598513 -123.12028988028078) bank107450 +107451 POINT(47.43524291400082 -123.19722520882912) bank107451 +107452 POINT(46.98622199427161 -122.23437174828172) bank107452 +107453 POINT(47.829446040999834 -121.65703049568246) bank107453 +107454 POINT(47.96861647379446 -121.64832184538739) bank107454 +107455 POINT(46.60898953335623 -121.61176603232137) bank107455 +107456 POINT(46.97683815746402 -122.87206389744955) bank107456 +107457 POINT(47.51848659092715 -121.5127967444726) bank107457 +107458 POINT(48.52863523340818 -122.65253218940998) bank107458 +107459 POINT(48.14241473783449 -121.95916282232517) bank107459 +107460 POINT(47.180625164925466 -122.96223809671241) bank107460 +107461 POINT(47.98044757291926 -123.0891268028445) bank107461 +107462 POINT(48.11656456232814 -122.11127288568225) bank107462 +107463 POINT(47.59925512512643 -121.95963660761727) bank107463 +107464 POINT(47.67745373571812 -122.4172055469843) bank107464 +107465 POINT(48.58932038336942 -122.07588669156803) bank107465 +107466 POINT(47.327811204983426 -121.33845646760884) bank107466 +107467 POINT(46.66569165850245 -123.0620262091178) bank107467 +107468 POINT(47.115899386838564 -123.0004684960027) bank107468 +107469 POINT(48.13356132242454 -122.18614618637352) bank107469 +107470 POINT(48.39391246980736 -121.8937348800974) bank107470 +107471 POINT(46.968039829936494 -122.40773800954653) bank107471 +107472 POINT(47.80859035029789 -121.35733180062942) bank107472 +107473 POINT(48.373385621653476 -122.59937691288714) bank107473 +107474 POINT(47.68721320967934 -121.60799079847641) bank107474 +107475 POINT(47.50313362944673 -123.02753486085804) bank107475 +107476 POINT(48.0433133563009 -121.50033361362279) bank107476 +107477 POINT(47.14149198107916 -122.12279296106742) bank107477 +107478 POINT(46.9172933721113 -123.08593267577814) bank107478 +107479 POINT(48.16691324322531 -122.37722670731043) bank107479 +107480 POINT(46.75372138132606 -122.48810931208806) bank107480 +107481 POINT(46.760616686827646 -121.58256402104088) bank107481 +107482 POINT(47.22773859327092 -123.07738538685524) bank107482 +107483 POINT(47.566581297605055 -122.94445983487024) bank107483 +107484 POINT(48.035522742880715 -121.6992511485369) bank107484 +107485 POINT(47.475038568357114 -123.17388762075238) bank107485 +107486 POINT(47.507053360776496 -121.94151038796404) bank107486 +107487 POINT(47.69611115249078 -122.99716739304888) bank107487 +107488 POINT(47.57382229855641 -121.87831338403167) bank107488 +107489 POINT(46.797140819178644 -122.52730177631067) bank107489 +107490 POINT(46.83379881826933 -122.54993263672895) bank107490 +107491 POINT(48.08828798787708 -122.63269371473366) bank107491 +107492 POINT(47.41562865964318 -122.77004782890015) bank107492 +107493 POINT(47.14271593986814 -121.83065345249938) bank107493 +107494 POINT(48.27394086195061 -121.51814693451261) bank107494 +107495 POINT(47.14756036158661 -122.4303345993495) bank107495 +107496 POINT(48.47255500628658 -122.71935646930639) bank107496 +107497 POINT(47.044793229339376 -122.46889755764752) bank107497 +107498 POINT(48.42746820330297 -122.19687468446095) bank107498 +107499 POINT(47.894275098309436 -122.50657212506992) bank107499 +107500 POINT(48.50100820383506 -123.07404934161043) bank107500 +107501 POINT(47.093479134045616 -122.06634542347403) bank107501 +107502 POINT(48.38111873858906 -121.90346986625106) bank107502 +107503 POINT(47.98488460756547 -121.46259560708744) bank107503 +107504 POINT(48.23850392228836 -123.31579538433967) bank107504 +107505 POINT(47.64331382558038 -121.38282934759185) bank107505 +107506 POINT(47.80931605502541 -122.42017498819588) bank107506 +107507 POINT(46.8790315144322 -122.16695256384213) bank107507 +107508 POINT(47.99873160146105 -123.06312423627301) bank107508 +107509 POINT(46.769422996486 -122.41523656902329) bank107509 +107510 POINT(48.32640873248806 -122.50837979338783) bank107510 +107511 POINT(47.17162660432617 -121.97843482977243) bank107511 +107512 POINT(47.82321051450941 -123.06977449846562) bank107512 +107513 POINT(46.9303430096295 -122.21168425790655) bank107513 +107514 POINT(46.919062854403606 -122.53975409781565) bank107514 +107515 POINT(47.62524702121073 -122.73815599651654) bank107515 +107516 POINT(48.20349857710378 -122.25813474051857) bank107516 +107517 POINT(47.31478644526134 -122.13739388064661) bank107517 +107518 POINT(47.51195863474136 -122.9670102663167) bank107518 +107519 POINT(47.19949459384036 -123.10938099851056) bank107519 +107520 POINT(46.671217170497215 -121.6753517406201) bank107520 +107521 POINT(47.47780590348757 -122.76992876591136) bank107521 +107522 POINT(46.85376569075715 -123.277376242715) bank107522 +107523 POINT(46.80036928677477 -122.84385737340564) bank107523 +107524 POINT(48.34413430936173 -123.22491048697954) bank107524 +107525 POINT(47.40390982021771 -121.94080755832431) bank107525 +107526 POINT(48.52669249765048 -123.3179531884746) bank107526 +107527 POINT(47.78200013500796 -121.41638562429638) bank107527 +107528 POINT(47.43904104944322 -121.96650192492551) bank107528 +107529 POINT(48.37184188589193 -123.05383344857393) bank107529 +107530 POINT(47.39578924881654 -121.42509062325149) bank107530 +107531 POINT(47.84401597986201 -122.21606049968617) bank107531 +107532 POINT(48.51167098604923 -122.44970033944) bank107532 +107533 POINT(47.93977253124409 -122.45771653786113) bank107533 +107534 POINT(47.73327431950246 -122.9036451612196) bank107534 +107535 POINT(47.98597488091182 -122.36516597285761) bank107535 +107536 POINT(48.03498843211227 -121.74187192078124) bank107536 +107537 POINT(48.51857014073499 -121.978064374623) bank107537 +107538 POINT(48.19403418906422 -122.30537055263102) bank107538 +107539 POINT(47.49110431530779 -122.35861788836057) bank107539 +107540 POINT(48.07715798960474 -122.37294451396917) bank107540 +107541 POINT(47.969301750420726 -122.42272937611317) bank107541 +107542 POINT(47.81087074243612 -122.72739425944302) bank107542 +107543 POINT(46.70104164197604 -122.92210707394807) bank107543 +107544 POINT(48.383743383572586 -122.06480067951978) bank107544 +107545 POINT(48.02326748261083 -122.86006521198682) bank107545 +107546 POINT(48.03777583673292 -122.17328083219442) bank107546 +107547 POINT(48.21665545286772 -121.87520556697442) bank107547 +107548 POINT(48.56871670671001 -122.60609439511585) bank107548 +107549 POINT(47.93645471068733 -121.56282790836728) bank107549 +107550 POINT(47.431990296027806 -122.09237972660235) bank107550 +107551 POINT(47.77334253544121 -123.29828777534485) bank107551 +107552 POINT(46.77678545314433 -121.90604623538349) bank107552 +107553 POINT(47.81470647314417 -121.39627886180529) bank107553 +107554 POINT(47.56056746328896 -122.32240501685729) bank107554 +107555 POINT(46.863769684104135 -122.18407713609321) bank107555 +107556 POINT(48.342185507183466 -121.47717402162858) bank107556 +107557 POINT(47.27016152733854 -122.86575094725868) bank107557 +107558 POINT(47.40490584184358 -121.60133835304187) bank107558 +107559 POINT(47.35911009253562 -122.29084780849125) bank107559 +107560 POINT(47.55805145029378 -123.22676699410876) bank107560 +107561 POINT(47.938514040853484 -122.66846642875034) bank107561 +107562 POINT(47.66307325745856 -123.13628040159008) bank107562 +107563 POINT(48.08059186764345 -121.52277485722878) bank107563 +107564 POINT(47.93961578412991 -122.50682452023018) bank107564 +107565 POINT(47.38854232203611 -121.40488554647433) bank107565 +107566 POINT(46.80035193544723 -123.15147812622821) bank107566 +107567 POINT(48.18236006467681 -122.8294712215565) bank107567 +107568 POINT(47.268567995377666 -122.94699290701514) bank107568 +107569 POINT(48.574366343496905 -121.41444010700359) bank107569 +107570 POINT(47.34895205740481 -122.16501009534537) bank107570 +107571 POINT(47.99950857620408 -122.12924875205117) bank107571 +107572 POINT(47.279686911863976 -122.58739117760258) bank107572 +107573 POINT(46.78325471177585 -121.48469494284069) bank107573 +107574 POINT(47.134594550354535 -121.86661881152222) bank107574 +107575 POINT(48.421780641409256 -123.09013776731159) bank107575 +107576 POINT(47.51268812819019 -121.80158929552628) bank107576 +107577 POINT(48.224693193527 -121.8598718571916) bank107577 +107578 POINT(47.56798085974794 -123.23034636450062) bank107578 +107579 POINT(47.623582856816235 -122.00340072488562) bank107579 +107580 POINT(47.15329285705398 -121.45677511979567) bank107580 +107581 POINT(46.86645615693072 -123.11035844862268) bank107581 +107582 POINT(48.32582751797676 -121.49867414486577) bank107582 +107583 POINT(48.011967476694494 -122.61637746066842) bank107583 +107584 POINT(46.97823160614983 -122.75466120085012) bank107584 +107585 POINT(46.73129302135891 -123.13490148809989) bank107585 +107586 POINT(46.934158463845684 -121.40146111798879) bank107586 +107587 POINT(47.336332199805504 -122.02447188534747) bank107587 +107588 POINT(48.49183079370793 -122.94187489010253) bank107588 +107589 POINT(47.757181669775626 -123.1480400646946) bank107589 +107590 POINT(48.262931719318196 -122.4660227974126) bank107590 +107591 POINT(46.63445144039478 -122.5403369097395) bank107591 +107592 POINT(47.9380037148477 -121.75869322580714) bank107592 +107593 POINT(47.889849257341396 -122.92625711628523) bank107593 +107594 POINT(47.34998623024273 -121.6804373510802) bank107594 +107595 POINT(47.93786782380702 -122.24097315887744) bank107595 +107596 POINT(47.5450983327809 -122.78588086227737) bank107596 +107597 POINT(46.9960413771919 -122.99329911476863) bank107597 +107598 POINT(48.18861197258121 -122.11066557757795) bank107598 +107599 POINT(46.73284064822681 -122.3761721249333) bank107599 +107600 POINT(48.197127861589266 -121.76685144119736) bank107600 +107601 POINT(47.1970274154352 -122.22802363074635) bank107601 +107602 POINT(47.06328748664791 -122.93864903895881) bank107602 +107603 POINT(48.45454742609549 -122.89701123314447) bank107603 +107604 POINT(47.65314385555477 -121.92760220090688) bank107604 +107605 POINT(47.24309193688677 -121.57949026326708) bank107605 +107606 POINT(47.7788325406289 -123.32251975964536) bank107606 +107607 POINT(47.58415934914725 -121.52397448591678) bank107607 +107608 POINT(46.85017088255246 -122.31495557093119) bank107608 +107609 POINT(47.11486488985534 -121.34637023327012) bank107609 +107610 POINT(48.47448042986402 -121.37177239366576) bank107610 +107611 POINT(48.26669861084876 -121.64009476570219) bank107611 +107612 POINT(47.61257297319911 -121.58974688210128) bank107612 +107613 POINT(47.988554544522074 -123.00523879637223) bank107613 +107614 POINT(46.88250560381331 -121.4704216496837) bank107614 +107615 POINT(47.09941733946925 -121.75101204830527) bank107615 +107616 POINT(47.212673268682195 -123.1202855462598) bank107616 +107617 POINT(48.11331020565631 -122.80192999331628) bank107617 +107618 POINT(47.04917554898671 -122.96452658412692) bank107618 +107619 POINT(47.826708509012 -122.39139498188588) bank107619 +107620 POINT(47.315647820117 -121.79006183203609) bank107620 +107621 POINT(46.800856379547085 -122.33090204899753) bank107621 +107622 POINT(47.48753652816179 -123.0182256438294) bank107622 +107623 POINT(48.3914426863431 -122.4189614646855) bank107623 +107624 POINT(47.232642180762824 -123.20715526792449) bank107624 +107625 POINT(47.830525992839846 -122.75094563147552) bank107625 +107626 POINT(48.47507399355339 -121.33408846607811) bank107626 +107627 POINT(46.75891560256383 -122.4047354397592) bank107627 +107628 POINT(46.97120488408547 -121.53800894134557) bank107628 +107629 POINT(46.60724255251995 -121.79992327921639) bank107629 +107630 POINT(48.55771354128701 -122.41887193707598) bank107630 +107631 POINT(48.206001776948 -122.66575604394706) bank107631 +107632 POINT(47.61012218003977 -122.93581942844527) bank107632 +107633 POINT(47.83631721770821 -123.11121995876472) bank107633 +107634 POINT(48.552780574545835 -122.24272107940136) bank107634 +107635 POINT(48.08241471819083 -122.5590110674049) bank107635 +107636 POINT(47.40504627829275 -122.5845502574255) bank107636 +107637 POINT(47.46007190610364 -121.3814653817977) bank107637 +107638 POINT(47.651190994760306 -122.89391955394524) bank107638 +107639 POINT(47.68438866708616 -121.61725348604185) bank107639 +107640 POINT(48.32201832841879 -122.94942190993534) bank107640 +107641 POINT(47.46710235296808 -123.06144382928564) bank107641 +107642 POINT(47.92850324094303 -121.45835816608101) bank107642 +107643 POINT(48.17178858709931 -121.60995964544851) bank107643 +107644 POINT(47.348055117442065 -121.72094274766752) bank107644 +107645 POINT(47.017528204980806 -122.87806208747543) bank107645 +107646 POINT(48.15584346335516 -122.21912453982175) bank107646 +107647 POINT(47.29317344336075 -122.9254485787361) bank107647 +107648 POINT(48.28187493999609 -122.84432501427443) bank107648 +107649 POINT(48.583276293103225 -122.13162624765441) bank107649 +107650 POINT(47.2373313745416 -121.34280468598301) bank107650 +107651 POINT(47.59128741589539 -122.49239666112669) bank107651 +107652 POINT(47.60755695210791 -121.97637328641088) bank107652 +107653 POINT(46.94191549620329 -122.77513123356563) bank107653 +107654 POINT(47.67923101872898 -122.32759915185942) bank107654 +107655 POINT(47.17413622619036 -123.03728092640596) bank107655 +107656 POINT(47.047577545987 -123.03680188995573) bank107656 +107657 POINT(46.78788999893984 -123.26588613676307) bank107657 +107658 POINT(47.01852989245703 -122.19394273483302) bank107658 +107659 POINT(47.88225756668313 -123.00984511536089) bank107659 +107660 POINT(48.60421682177632 -123.30842831535898) bank107660 +107661 POINT(47.025652163168914 -122.30018906483637) bank107661 +107662 POINT(46.85504663061895 -122.68216505081175) bank107662 +107663 POINT(47.77695237956714 -121.62568246428098) bank107663 +107664 POINT(47.58676734515116 -121.92933176145291) bank107664 +107665 POINT(47.49001491365858 -122.41552459460037) bank107665 +107666 POINT(47.426618791584666 -121.50936237721587) bank107666 +107667 POINT(46.7073001984441 -122.81235359008934) bank107667 +107668 POINT(47.058523542682046 -122.82658021683775) bank107668 +107669 POINT(46.614655971482996 -122.4383923371093) bank107669 +107670 POINT(47.8640242683977 -122.61187796489865) bank107670 +107671 POINT(48.23843196320529 -122.45002321520866) bank107671 +107672 POINT(48.45522231206192 -123.096088577891) bank107672 +107673 POINT(47.045594262833966 -122.91428486334767) bank107673 +107674 POINT(48.52644205469499 -121.35874727602283) bank107674 +107675 POINT(46.797894549419524 -123.23256848920404) bank107675 +107676 POINT(46.815677430665815 -122.15281462735672) bank107676 +107677 POINT(48.04317279730929 -123.24399572457334) bank107677 +107678 POINT(48.428494137837106 -122.20176558804532) bank107678 +107679 POINT(47.203306115750244 -122.60651218695895) bank107679 +107680 POINT(47.40639334929558 -122.4950204508091) bank107680 +107681 POINT(47.024138258326566 -123.02944903137683) bank107681 +107682 POINT(46.713923708205066 -122.790970070789) bank107682 +107683 POINT(48.148841739569185 -123.08455158470196) bank107683 +107684 POINT(46.96630411665315 -122.42841150298344) bank107684 +107685 POINT(47.09743093812097 -122.29751967465488) bank107685 +107686 POINT(48.45578363030193 -123.00988335002714) bank107686 +107687 POINT(48.09058136668284 -121.88874955160617) bank107687 +107688 POINT(48.26113396687454 -121.64504565889119) bank107688 +107689 POINT(47.09716943780336 -123.2790021780088) bank107689 +107690 POINT(48.343716910876175 -121.92242758524465) bank107690 +107691 POINT(48.54942728441904 -123.2183293504013) bank107691 +107692 POINT(47.84059473045174 -122.69895613596205) bank107692 +107693 POINT(47.47802985657129 -121.72182047702906) bank107693 +107694 POINT(48.0579354861244 -122.33143091250577) bank107694 +107695 POINT(48.43533678724279 -122.49473346380992) bank107695 +107696 POINT(47.53021062381385 -122.56838155342243) bank107696 +107697 POINT(46.82582737123075 -123.23269068592215) bank107697 +107698 POINT(48.599988991916916 -121.65956450673383) bank107698 +107699 POINT(47.68909758431342 -121.54330386586624) bank107699 +107700 POINT(48.58065955642066 -122.17147216761134) bank107700 +107701 POINT(46.93351040644354 -122.63720839954668) bank107701 +107702 POINT(47.769970179342636 -121.74110730442099) bank107702 +107703 POINT(48.31044883943957 -123.18708178936558) bank107703 +107704 POINT(47.25618947777046 -122.84656489325685) bank107704 +107705 POINT(47.296545257613154 -122.21066987835407) bank107705 +107706 POINT(46.767066189156274 -121.93875309542598) bank107706 +107707 POINT(46.67404005100281 -121.78022290675351) bank107707 +107708 POINT(47.26877616995024 -122.57837058749114) bank107708 +107709 POINT(46.66171119564977 -121.90858952150435) bank107709 +107710 POINT(46.98769454652156 -121.86765529689941) bank107710 +107711 POINT(48.09949151363028 -122.86055527669058) bank107711 +107712 POINT(46.60890548385034 -122.48979912364678) bank107712 +107713 POINT(48.40878401324329 -122.36186257812045) bank107713 +107714 POINT(46.626547734545355 -122.57291977738771) bank107714 +107715 POINT(46.943358631396435 -123.02635842538339) bank107715 +107716 POINT(46.967674864339415 -122.56407655569869) bank107716 +107717 POINT(48.2020158855934 -121.37970137971053) bank107717 +107718 POINT(47.04010997828364 -121.41630732135839) bank107718 +107719 POINT(46.73724589439064 -123.15862476805391) bank107719 +107720 POINT(47.103850799954365 -121.96693774472088) bank107720 +107721 POINT(48.179856003979076 -122.59019990264179) bank107721 +107722 POINT(47.320675356233004 -122.30992834742872) bank107722 +107723 POINT(48.57911719304484 -122.52895924652454) bank107723 +107724 POINT(46.74938422978261 -122.48208268225612) bank107724 +107725 POINT(47.51176732119722 -122.4804534194645) bank107725 +107726 POINT(46.657759670838644 -121.6933200430419) bank107726 +107727 POINT(47.51103453582988 -121.94198650341268) bank107727 +107728 POINT(46.75723565876983 -123.07633279683068) bank107728 +107729 POINT(46.83312865728979 -121.92514703740689) bank107729 +107730 POINT(47.025720847833114 -122.28984233783392) bank107730 +107731 POINT(47.33700712786168 -121.89299122076412) bank107731 +107732 POINT(47.06117877840776 -122.85890839186924) bank107732 +107733 POINT(48.31893230694498 -121.716016284041) bank107733 +107734 POINT(47.414775609543376 -121.5648098480508) bank107734 +107735 POINT(47.50982890982288 -123.1371259370844) bank107735 +107736 POINT(48.51398722937694 -121.92632187957362) bank107736 +107737 POINT(48.51254735725329 -121.405808098147) bank107737 +107738 POINT(46.66002563360725 -123.0280440734302) bank107738 +107739 POINT(48.40778943123745 -122.27634207213391) bank107739 +107740 POINT(47.1214711319995 -121.43733468362217) bank107740 +107741 POINT(47.02151321408379 -121.57292171650553) bank107741 +107742 POINT(47.20083988765125 -122.80803836085576) bank107742 +107743 POINT(47.25557077816362 -122.40818886381658) bank107743 +107744 POINT(48.33139022296016 -121.88595010255774) bank107744 +107745 POINT(47.07905972706807 -122.49074866294725) bank107745 +107746 POINT(48.358016320172496 -121.56389681609576) bank107746 +107747 POINT(47.126826859889555 -123.28355208840465) bank107747 +107748 POINT(47.235194130516795 -121.44041178376277) bank107748 +107749 POINT(47.77365907186458 -122.23181753179473) bank107749 +107750 POINT(46.8603211168735 -121.6351253432048) bank107750 +107751 POINT(47.55250549126041 -123.00199859926391) bank107751 +107752 POINT(47.59736583477803 -122.96652736767177) bank107752 +107753 POINT(47.17106885514978 -121.3790267561061) bank107753 +107754 POINT(47.64194221495817 -123.11559921101647) bank107754 +107755 POINT(47.714443167718386 -122.56834899219376) bank107755 +107756 POINT(48.271738451861864 -121.88933086776711) bank107756 +107757 POINT(47.334240742927896 -121.73508801242741) bank107757 +107758 POINT(47.31167128877111 -123.29197793785606) bank107758 +107759 POINT(47.761932826744946 -121.82520477868351) bank107759 +107760 POINT(46.89578139393375 -122.13812634906861) bank107760 +107761 POINT(48.40940416515493 -122.73348219784963) bank107761 +107762 POINT(48.33800773461948 -122.2653033677189) bank107762 +107763 POINT(48.33406455479154 -121.83958755593491) bank107763 +107764 POINT(48.065623527824506 -122.03223702531848) bank107764 +107765 POINT(46.766785672808695 -122.9784269787758) bank107765 +107766 POINT(47.80657601520272 -122.79277865967083) bank107766 +107767 POINT(47.334376641829344 -122.91825440133564) bank107767 +107768 POINT(47.18308439878661 -122.36428556451317) bank107768 +107769 POINT(48.02924705137013 -123.30590820355502) bank107769 +107770 POINT(47.694743631229215 -122.3047249645081) bank107770 +107771 POINT(48.02307957834188 -122.57319381939635) bank107771 +107772 POINT(46.721575776336145 -122.39537375950776) bank107772 +107773 POINT(47.542123062363125 -121.41472661397498) bank107773 +107774 POINT(47.63001302101792 -122.95728124611792) bank107774 +107775 POINT(48.297548393500826 -122.39736713786777) bank107775 +107776 POINT(46.936913943482466 -122.75895173241206) bank107776 +107777 POINT(48.446715232294956 -121.60328726423882) bank107777 +107778 POINT(47.90163449104042 -121.58354457733715) bank107778 +107779 POINT(48.295236390268094 -121.94478805744876) bank107779 +107780 POINT(48.155327693463995 -123.09410464228094) bank107780 +107781 POINT(48.56397292342176 -121.72213285686428) bank107781 +107782 POINT(47.16008814963105 -122.47589419330133) bank107782 +107783 POINT(47.58013346245976 -121.55077958713278) bank107783 +107784 POINT(48.22663805232607 -123.15251782257184) bank107784 +107785 POINT(47.49918507528252 -121.36299152603124) bank107785 +107786 POINT(47.90427522572057 -122.14559865327163) bank107786 +107787 POINT(48.211337239829525 -122.75962037751549) bank107787 +107788 POINT(46.870708007810535 -122.34555391930357) bank107788 +107789 POINT(47.88570944652634 -122.87585290791735) bank107789 +107790 POINT(47.28972376241917 -122.04589176402901) bank107790 +107791 POINT(47.33343640737539 -121.89034669544581) bank107791 +107792 POINT(47.355600194793666 -121.85619666578687) bank107792 +107793 POINT(47.35250416994638 -122.48945490352241) bank107793 +107794 POINT(46.61036389522045 -121.4517497581086) bank107794 +107795 POINT(48.53995980848469 -122.56328094862029) bank107795 +107796 POINT(47.30604819377075 -122.34330159319336) bank107796 +107797 POINT(46.64413189374348 -122.21974644332941) bank107797 +107798 POINT(48.12049307734075 -122.71656205631076) bank107798 +107799 POINT(47.58314424684699 -123.08359930455404) bank107799 +107800 POINT(47.1303941153811 -121.37881240439006) bank107800 +107801 POINT(48.10040440441569 -122.0376431263091) bank107801 +107802 POINT(48.02933922957345 -123.1011579041731) bank107802 +107803 POINT(47.235257120635055 -122.72097506373925) bank107803 +107804 POINT(47.71169044625903 -121.56928794269966) bank107804 +107805 POINT(47.504823616305714 -122.8000213055873) bank107805 +107806 POINT(46.68948226459005 -122.28799008937877) bank107806 +107807 POINT(48.262110845522336 -122.54804379390961) bank107807 +107808 POINT(47.0003429030344 -122.35867450100439) bank107808 +107809 POINT(46.98999838102376 -122.83120046217884) bank107809 +107810 POINT(48.03702945692624 -122.59533528404171) bank107810 +107811 POINT(48.02078687394171 -121.41328733576039) bank107811 +107812 POINT(46.75802486329194 -122.6631983483762) bank107812 +107813 POINT(47.782265454313254 -123.14315615910772) bank107813 +107814 POINT(47.11192351246685 -122.90802109151164) bank107814 +107815 POINT(48.09180756192745 -122.12472108910856) bank107815 +107816 POINT(48.01817804188297 -122.77847413063805) bank107816 +107817 POINT(46.83998256264135 -121.98714171062164) bank107817 +107818 POINT(48.10299057370105 -123.15057521607488) bank107818 +107819 POINT(47.09662821614867 -121.9904486888601) bank107819 +107820 POINT(47.686799677294836 -122.6415659521423) bank107820 +107821 POINT(48.262269607092584 -121.46530154662054) bank107821 +107822 POINT(48.573817552637855 -122.94857789910704) bank107822 +107823 POINT(47.492302220907746 -123.09867530241343) bank107823 +107824 POINT(48.5357820061702 -123.10191596644636) bank107824 +107825 POINT(46.88951664856467 -122.30861677300553) bank107825 +107826 POINT(46.71658115302315 -122.88632506295986) bank107826 +107827 POINT(47.98223880124676 -121.76455302779996) bank107827 +107828 POINT(48.362206126271325 -121.4463805393891) bank107828 +107829 POINT(47.29017086626989 -121.50713835784381) bank107829 +107830 POINT(48.48499826892631 -121.67497460329709) bank107830 +107831 POINT(46.62065785444824 -121.88215873568343) bank107831 +107832 POINT(47.975596806824925 -122.9908550393234) bank107832 +107833 POINT(47.06139175354404 -122.00339716340535) bank107833 +107834 POINT(47.761866482052305 -121.84660371054854) bank107834 +107835 POINT(48.03284348286316 -121.93068118062436) bank107835 +107836 POINT(47.52761501232962 -123.18363486002309) bank107836 +107837 POINT(46.80746836185512 -121.78197425802118) bank107837 +107838 POINT(48.57040494442263 -121.67054146712363) bank107838 +107839 POINT(47.99412876872853 -121.55763161104305) bank107839 +107840 POINT(48.411838874707385 -122.7191928210404) bank107840 +107841 POINT(47.749476602404634 -122.1992237692118) bank107841 +107842 POINT(47.35657507157597 -122.40653804882774) bank107842 +107843 POINT(47.530772738483854 -121.40780768609767) bank107843 +107844 POINT(47.49256343074808 -121.51406726391943) bank107844 +107845 POINT(47.53844910044786 -121.45824440973585) bank107845 +107846 POINT(48.145873499419295 -122.76254573705239) bank107846 +107847 POINT(48.24970966693685 -122.04943334771475) bank107847 +107848 POINT(48.60452749348123 -122.52393553431436) bank107848 +107849 POINT(48.173058215566584 -122.36593832726865) bank107849 +107850 POINT(47.99278847938295 -122.11679846339894) bank107850 +107851 POINT(47.28915468025389 -121.48053014143002) bank107851 +107852 POINT(47.04062529600348 -121.87632830378146) bank107852 +107853 POINT(48.108674810954376 -122.74863226879278) bank107853 +107854 POINT(47.37615493835462 -121.53313471116927) bank107854 +107855 POINT(47.56834480125025 -122.48923932458294) bank107855 +107856 POINT(46.86414395578495 -123.12624174781195) bank107856 +107857 POINT(47.37359043127939 -121.4733893283543) bank107857 +107858 POINT(48.605189189197546 -121.6102198507351) bank107858 +107859 POINT(47.45614822749114 -122.21693669330901) bank107859 +107860 POINT(48.17753536156867 -121.487161021418) bank107860 +107861 POINT(47.42530374963682 -121.97631499729376) bank107861 +107862 POINT(48.20863247864075 -122.68918584798045) bank107862 +107863 POINT(47.95829555083319 -122.83817653886697) bank107863 +107864 POINT(48.25039751396166 -121.46950074435986) bank107864 +107865 POINT(46.721087454216644 -122.9607415319655) bank107865 +107866 POINT(47.24746539341128 -121.99170258333635) bank107866 +107867 POINT(48.17310553502618 -122.37459106264404) bank107867 +107868 POINT(47.73410452005525 -122.1735893404167) bank107868 +107869 POINT(47.606921345286345 -121.39387863540769) bank107869 +107870 POINT(48.25950046603542 -122.40575600735963) bank107870 +107871 POINT(48.369652714651025 -121.9363538058722) bank107871 +107872 POINT(46.91385877119884 -121.57565124853615) bank107872 +107873 POINT(48.53699431281127 -122.20367404415504) bank107873 +107874 POINT(48.02124841665437 -122.25067137836244) bank107874 +107875 POINT(47.60560940385116 -121.91797138809233) bank107875 +107876 POINT(48.429150671191124 -121.95586065305959) bank107876 +107877 POINT(47.031467894639306 -123.26762714413351) bank107877 +107878 POINT(47.20852423167681 -122.98258430197207) bank107878 +107879 POINT(48.526659515147436 -123.20552710574371) bank107879 +107880 POINT(47.849404905900904 -123.11519387196876) bank107880 +107881 POINT(47.49893856202494 -123.27229720943912) bank107881 +107882 POINT(46.607141513944484 -123.05680413521839) bank107882 +107883 POINT(48.08881021999609 -121.50833632950618) bank107883 +107884 POINT(46.75981060760237 -122.55015427713771) bank107884 +107885 POINT(47.54956594463641 -122.95553138879464) bank107885 +107886 POINT(46.63718419752436 -121.6605891899761) bank107886 +107887 POINT(47.57003558798464 -121.8592919508435) bank107887 +107888 POINT(48.09440737971 -122.20850551138896) bank107888 +107889 POINT(47.64704939401251 -122.28156090883587) bank107889 +107890 POINT(48.22205579520752 -122.18631771758542) bank107890 +107891 POINT(48.153434890563055 -121.70886754830582) bank107891 +107892 POINT(46.8725089988724 -122.2682230136096) bank107892 +107893 POINT(48.05862740466635 -122.10109160689449) bank107893 +107894 POINT(47.868421308503706 -122.61614360442302) bank107894 +107895 POINT(48.237532852737715 -122.28349200055698) bank107895 +107896 POINT(48.22684876949481 -122.05603887933154) bank107896 +107897 POINT(47.362345057008 -121.5234480013825) bank107897 +107898 POINT(48.330526339015854 -122.8628099366075) bank107898 +107899 POINT(48.1532879610528 -121.65845155008128) bank107899 +107900 POINT(47.612455532100284 -121.97011117419085) bank107900 +107901 POINT(47.45785407567634 -122.5581630842941) bank107901 +107902 POINT(47.29178771706736 -121.76781165534602) bank107902 +107903 POINT(47.38372170693627 -123.29010727567017) bank107903 +107904 POINT(47.78131671477888 -121.64346780584052) bank107904 +107905 POINT(47.602263889112685 -122.37975752798732) bank107905 +107906 POINT(48.27115805368472 -122.8920141009051) bank107906 +107907 POINT(46.89050285233232 -122.49091029135617) bank107907 +107908 POINT(46.84043178617246 -121.60135272601632) bank107908 +107909 POINT(47.87843087111135 -122.17086620903503) bank107909 +107910 POINT(47.11426008696102 -122.58757618410375) bank107910 +107911 POINT(48.55718044644017 -123.33018384456318) bank107911 +107912 POINT(46.60768452237735 -122.69646957186106) bank107912 +107913 POINT(47.06462951286752 -121.62936097496672) bank107913 +107914 POINT(48.37781374276395 -123.15414115021593) bank107914 +107915 POINT(47.667233823611554 -121.47960106337598) bank107915 +107916 POINT(47.821985048052866 -122.89932918015377) bank107916 +107917 POINT(47.92488745875628 -122.46566947146665) bank107917 +107918 POINT(47.31094580922976 -122.19277800241622) bank107918 +107919 POINT(48.144650899997444 -122.43685938594548) bank107919 +107920 POINT(47.33363671736013 -123.2528718788549) bank107920 +107921 POINT(48.22418296152013 -122.08931413487122) bank107921 +107922 POINT(48.518205084262156 -122.55858361358891) bank107922 +107923 POINT(46.79865370752991 -123.27147882803257) bank107923 +107924 POINT(47.36477731904497 -122.4092054352194) bank107924 +107925 POINT(48.294320124697684 -121.35332867654785) bank107925 +107926 POINT(47.33822756601516 -122.63345078380715) bank107926 +107927 POINT(48.59359601768504 -121.69377881717843) bank107927 +107928 POINT(47.687985063687776 -121.9200861985396) bank107928 +107929 POINT(47.17790695402659 -121.51374088901275) bank107929 +107930 POINT(47.197655703489666 -123.26424739233677) bank107930 +107931 POINT(47.30410312886151 -123.29197062589955) bank107931 +107932 POINT(46.76217576034416 -122.49108219625238) bank107932 +107933 POINT(46.81999355475821 -121.76440764211947) bank107933 +107934 POINT(46.854902173100044 -122.97086982035918) bank107934 +107935 POINT(47.812261297413876 -122.65031117484128) bank107935 +107936 POINT(46.77783450700101 -121.93558271024482) bank107936 +107937 POINT(48.47770436549338 -121.77895813841334) bank107937 +107938 POINT(47.102356628351686 -121.64522539739988) bank107938 +107939 POINT(47.90695137697156 -123.03207627587724) bank107939 +107940 POINT(47.40444860321308 -121.4082330528411) bank107940 +107941 POINT(47.278632812247494 -122.60430369831934) bank107941 +107942 POINT(46.982200134614374 -122.16314057248465) bank107942 +107943 POINT(46.97409286600919 -122.98305027796336) bank107943 +107944 POINT(48.552317987859 -121.97935783584906) bank107944 +107945 POINT(48.03319482261588 -121.4582810426965) bank107945 +107946 POINT(47.45418584553178 -122.88433836987953) bank107946 +107947 POINT(48.01880747272258 -121.9609130934551) bank107947 +107948 POINT(47.03065302909041 -122.93928209368863) bank107948 +107949 POINT(48.21386989046754 -122.11884780490352) bank107949 +107950 POINT(46.90459251410489 -123.18564656078001) bank107950 +107951 POINT(48.5806419890591 -122.4015573854193) bank107951 +107952 POINT(46.95874659775731 -121.6964958995747) bank107952 +107953 POINT(48.2186080596526 -121.56218740098338) bank107953 +107954 POINT(48.39483199522668 -123.04547719281895) bank107954 +107955 POINT(48.07187784526415 -123.18263717044192) bank107955 +107956 POINT(47.96310287702624 -121.42247659478502) bank107956 +107957 POINT(47.1233395407597 -123.03563013220841) bank107957 +107958 POINT(46.867424725638145 -121.3650823818394) bank107958 +107959 POINT(48.58489866302042 -122.14958493329316) bank107959 +107960 POINT(46.65504141263603 -121.71526439263631) bank107960 +107961 POINT(47.274597782516096 -122.08476844421993) bank107961 +107962 POINT(47.803344853811396 -122.76705217723779) bank107962 +107963 POINT(47.80404122119037 -123.28066097357691) bank107963 +107964 POINT(47.74993030305888 -122.31596234582342) bank107964 +107965 POINT(47.969849924905255 -122.48213449851156) bank107965 +107966 POINT(47.974109211313845 -122.83587973434429) bank107966 +107967 POINT(48.54319967673131 -122.76740300943152) bank107967 +107968 POINT(48.37858987821318 -121.84264857088904) bank107968 +107969 POINT(46.67094211882741 -121.35131180380897) bank107969 +107970 POINT(48.36112025327193 -121.99719969598107) bank107970 +107971 POINT(48.50506045320444 -121.43872306764253) bank107971 +107972 POINT(47.59362278041107 -122.46891832556402) bank107972 +107973 POINT(47.46143037191292 -123.07379440916706) bank107973 +107974 POINT(48.48865865861892 -123.16216447619153) bank107974 +107975 POINT(48.42969267273385 -121.6024243036838) bank107975 +107976 POINT(48.60351485891898 -122.76536605587447) bank107976 +107977 POINT(48.107941290655404 -122.96115487629825) bank107977 +107978 POINT(47.68208751680741 -121.70678818585795) bank107978 +107979 POINT(47.19221982203289 -122.38404563674405) bank107979 +107980 POINT(47.17161973065443 -122.65117222821263) bank107980 +107981 POINT(48.28987108410953 -122.94885022984778) bank107981 +107982 POINT(47.225141747005146 -122.13854817560504) bank107982 +107983 POINT(47.25632269022585 -122.92861065082947) bank107983 +107984 POINT(47.686604576295565 -121.6550278120073) bank107984 +107985 POINT(47.10715645103018 -122.91070484628702) bank107985 +107986 POINT(47.20967281332484 -122.622459924273) bank107986 +107987 POINT(47.375599210776656 -122.87240795648488) bank107987 +107988 POINT(48.52790367630815 -123.31809907540398) bank107988 +107989 POINT(47.93112453013999 -122.54342042003532) bank107989 +107990 POINT(47.93906161946218 -122.0223923812847) bank107990 +107991 POINT(46.74077317120754 -121.49977969621476) bank107991 +107992 POINT(47.35567233717693 -121.59877772803463) bank107992 +107993 POINT(48.27907281357464 -121.88466688960949) bank107993 +107994 POINT(47.19353193197501 -121.4549641145769) bank107994 +107995 POINT(46.73239512255063 -123.07610612722235) bank107995 +107996 POINT(47.19584979411813 -122.32303335355994) bank107996 +107997 POINT(47.07758147291127 -122.50180532422137) bank107997 +107998 POINT(48.153462524852685 -122.59021351806958) bank107998 +107999 POINT(48.2145675591242 -121.46270569078327) bank107999 +108000 POINT(46.89699230537483 -121.40254863289523) bank108000 +108001 POINT(48.10531007120778 -121.68730437201876) bank108001 +108002 POINT(47.292481673288144 -121.33941252860336) bank108002 +108003 POINT(47.344958814605185 -121.85555347244681) bank108003 +108004 POINT(47.985466890994296 -123.32055017497925) bank108004 +108005 POINT(48.01727594720093 -121.92244448738332) bank108005 +108006 POINT(47.21199060408973 -123.08412801426934) bank108006 +108007 POINT(48.044176166160945 -121.67053854612608) bank108007 +108008 POINT(48.40837734233213 -122.65602570850756) bank108008 +108009 POINT(46.84684363320259 -122.47072381807541) bank108009 +108010 POINT(48.01610470262073 -121.40978275282832) bank108010 +108011 POINT(47.95500787808858 -121.71078561788318) bank108011 +108012 POINT(47.61809537176531 -122.27982434844125) bank108012 +108013 POINT(48.4836509673095 -121.73639915610497) bank108013 +108014 POINT(48.194009152892804 -122.19088415109002) bank108014 +108015 POINT(48.417525113748994 -121.99358969391518) bank108015 +108016 POINT(46.635280440523616 -122.6398613084335) bank108016 +108017 POINT(47.96140748711339 -122.08914489373032) bank108017 +108018 POINT(48.34116605193187 -122.08238723451608) bank108018 +108019 POINT(47.55084747588888 -121.96067746257975) bank108019 +108020 POINT(48.325128605983295 -121.47905536250234) bank108020 +108021 POINT(47.54529945765112 -121.79802419537744) bank108021 +108022 POINT(47.6693036323874 -122.32596631161435) bank108022 +108023 POINT(48.17168761614847 -123.20573540472003) bank108023 +108024 POINT(47.064937678437545 -123.10706757704784) bank108024 +108025 POINT(47.792832370217965 -121.84045479312772) bank108025 +108026 POINT(47.02828698796639 -122.08824286636437) bank108026 +108027 POINT(48.41458713867564 -121.5747738999365) bank108027 +108028 POINT(46.66342516454276 -122.79272492879254) bank108028 +108029 POINT(46.9069694142157 -122.65057707978349) bank108029 +108030 POINT(46.97565753699842 -123.13669016329338) bank108030 +108031 POINT(47.4648551428828 -122.71169734330158) bank108031 +108032 POINT(47.493371346774126 -122.57791166385846) bank108032 +108033 POINT(48.07691482046076 -123.07575915488205) bank108033 +108034 POINT(48.082891704221545 -123.23607491319576) bank108034 +108035 POINT(47.695374180023116 -122.35716774978052) bank108035 +108036 POINT(47.44260399897743 -123.19988555634936) bank108036 +108037 POINT(48.45991782534096 -122.60998781012293) bank108037 +108038 POINT(48.38088495410372 -122.61092324666244) bank108038 +108039 POINT(47.628917968768604 -122.98749267499541) bank108039 +108040 POINT(47.8995655410349 -121.3651557593759) bank108040 +108041 POINT(48.04653403499379 -122.15084637939476) bank108041 +108042 POINT(48.22768096362652 -122.48132474296304) bank108042 +108043 POINT(46.644015177386436 -121.92996985294927) bank108043 +108044 POINT(48.059071342004074 -123.26580688143727) bank108044 +108045 POINT(47.751112466038144 -122.78968599149499) bank108045 +108046 POINT(47.03512676810976 -121.67878758526444) bank108046 +108047 POINT(48.461113305000566 -122.04797199027091) bank108047 +108048 POINT(48.58667781383025 -122.82505545888189) bank108048 +108049 POINT(47.793344859170794 -121.57056333700837) bank108049 +108050 POINT(48.54490067787321 -122.55044948620608) bank108050 +108051 POINT(46.6878621249759 -121.51441242616353) bank108051 +108052 POINT(48.429637481959105 -122.44009217901787) bank108052 +108053 POINT(48.35867845850133 -123.16374643907692) bank108053 +108054 POINT(47.941896941938815 -121.41322171383906) bank108054 +108055 POINT(48.324718578764916 -121.57216674912745) bank108055 +108056 POINT(48.243466249770165 -123.10455675800752) bank108056 +108057 POINT(46.617033633746445 -122.50961665790022) bank108057 +108058 POINT(48.40614193025262 -123.0660547493195) bank108058 +108059 POINT(47.72510971911426 -122.65175920555454) bank108059 +108060 POINT(47.489826161701295 -123.06785815430663) bank108060 +108061 POINT(47.136099493277555 -121.94827031515376) bank108061 +108062 POINT(48.464169562942786 -122.8726002491369) bank108062 +108063 POINT(47.22708144408393 -122.92872581181115) bank108063 +108064 POINT(47.569698605701994 -123.02020792852367) bank108064 +108065 POINT(46.74198960340853 -122.04506541644608) bank108065 +108066 POINT(47.25025400075394 -122.44586333587166) bank108066 +108067 POINT(48.44434577217539 -121.4755277775805) bank108067 +108068 POINT(47.87323151525844 -121.79377674984742) bank108068 +108069 POINT(46.72224771255736 -121.99810508071113) bank108069 +108070 POINT(46.72895196731991 -122.73902293133543) bank108070 +108071 POINT(47.17089172762896 -122.48404483195459) bank108071 +108072 POINT(47.93668113107988 -122.76789048089275) bank108072 +108073 POINT(47.27890404119484 -122.08178232039374) bank108073 +108074 POINT(47.10487727430159 -122.88986223292112) bank108074 +108075 POINT(47.56374529438116 -121.82628335984126) bank108075 +108076 POINT(47.183633284769655 -123.27517989757833) bank108076 +108077 POINT(47.47980516175395 -121.90121604769331) bank108077 +108078 POINT(47.043421931388515 -121.874581571817) bank108078 +108079 POINT(47.61716044228297 -122.2356201511285) bank108079 +108080 POINT(47.25584204016277 -122.877953227985) bank108080 +108081 POINT(47.409770532376314 -121.62465043919) bank108081 +108082 POINT(48.30194067240305 -123.16091414525218) bank108082 +108083 POINT(46.729099624666574 -123.25728178130788) bank108083 +108084 POINT(47.17889575497008 -123.18488584326248) bank108084 +108085 POINT(47.44203070859038 -121.40740977117451) bank108085 +108086 POINT(47.31937121625757 -121.78300126606202) bank108086 +108087 POINT(47.75897111742959 -122.66673636942807) bank108087 +108088 POINT(48.05221914412592 -121.58821180033688) bank108088 +108089 POINT(47.38410317062958 -121.40987536508678) bank108089 +108090 POINT(46.93257058763114 -122.97171846907872) bank108090 +108091 POINT(48.27475817654683 -121.99745729052076) bank108091 +108092 POINT(46.671291493352406 -122.26298424609533) bank108092 +108093 POINT(48.068813430790186 -122.87312881781462) bank108093 +108094 POINT(47.65569268961599 -121.5315055700651) bank108094 +108095 POINT(48.217535220865784 -121.81735355115036) bank108095 +108096 POINT(48.26785753289107 -122.68040722921829) bank108096 +108097 POINT(47.0800755170811 -123.04067768482281) bank108097 +108098 POINT(48.16792525982878 -122.10235936138419) bank108098 +108099 POINT(48.14506136411753 -121.894549395281) bank108099 +108100 POINT(47.026920149047086 -123.03873380710885) bank108100 +108101 POINT(46.7225904610352 -121.91428230609296) bank108101 +108102 POINT(48.149045715767485 -123.0613996242123) bank108102 +108103 POINT(48.47293221149115 -123.28199743763956) bank108103 +108104 POINT(47.92683161187376 -122.21765298310078) bank108104 +108105 POINT(47.55122990651028 -121.56689244159455) bank108105 +108106 POINT(47.64587530144619 -122.96253612741265) bank108106 +108107 POINT(47.78011464838576 -122.07649516880743) bank108107 +108108 POINT(48.01634261930511 -123.18031930299492) bank108108 +108109 POINT(47.89830514011888 -121.95756320459105) bank108109 +108110 POINT(48.185794963309796 -122.39233364010856) bank108110 +108111 POINT(47.19900910368293 -121.57416368623332) bank108111 +108112 POINT(47.69805055796403 -123.31095585262698) bank108112 +108113 POINT(47.35480478143152 -121.56738048247054) bank108113 +108114 POINT(47.78684575367116 -123.235051674353) bank108114 +108115 POINT(47.88263259112452 -121.33598205965792) bank108115 +108116 POINT(47.93054866595331 -122.04864266622955) bank108116 +108117 POINT(48.552006938344014 -122.34430347860533) bank108117 +108118 POINT(48.17561378919666 -121.9635826373697) bank108118 +108119 POINT(47.5850841152288 -122.80756830434122) bank108119 +108120 POINT(46.77581814551484 -122.72090220053953) bank108120 +108121 POINT(48.14472100274507 -122.13246493039003) bank108121 +108122 POINT(47.72416469052467 -123.18259745935495) bank108122 +108123 POINT(47.2735091104775 -122.4114135856174) bank108123 +108124 POINT(48.53223521584704 -121.37112812074702) bank108124 +108125 POINT(46.966936650182646 -121.94007970734665) bank108125 +108126 POINT(46.80110166197892 -122.39626666935293) bank108126 +108127 POINT(47.50103001696451 -122.19744140127436) bank108127 +108128 POINT(48.350677545988766 -122.80286384519772) bank108128 +108129 POINT(47.29682396006187 -122.85768641258827) bank108129 +108130 POINT(47.62688314668871 -123.11233956258195) bank108130 +108131 POINT(46.8727652551493 -122.41115697252225) bank108131 +108132 POINT(47.99411684490003 -121.72446354837125) bank108132 +108133 POINT(46.79316473425858 -123.11644530196753) bank108133 +108134 POINT(48.026831563917206 -121.81463654207687) bank108134 +108135 POINT(48.60449503886596 -123.00769624882874) bank108135 +108136 POINT(48.2707358068375 -121.40780534412723) bank108136 +108137 POINT(48.09623971334002 -122.99715347087492) bank108137 +108138 POINT(48.08719239266427 -123.08031055937049) bank108138 +108139 POINT(48.25813888785596 -122.43408899719346) bank108139 +108140 POINT(48.329877790556544 -123.24001126439147) bank108140 +108141 POINT(48.478196990947445 -123.19989274376852) bank108141 +108142 POINT(46.91198946748791 -121.80285328726058) bank108142 +108143 POINT(47.282190907213504 -123.12216471684162) bank108143 +108144 POINT(47.844334813509064 -121.5279268503146) bank108144 +108145 POINT(47.51461512748652 -122.29282089778239) bank108145 +108146 POINT(48.15454576268431 -121.50917140252717) bank108146 +108147 POINT(47.53588662083116 -122.24165918518833) bank108147 +108148 POINT(47.92959839002873 -121.34638721297704) bank108148 +108149 POINT(47.05250264881525 -122.57119464324637) bank108149 +108150 POINT(47.396438717744665 -122.07098298621182) bank108150 +108151 POINT(47.15252085962571 -122.51228247501408) bank108151 +108152 POINT(48.455202088100734 -121.82610946165782) bank108152 +108153 POINT(47.34500227335757 -122.55381952269815) bank108153 +108154 POINT(46.813080010527855 -121.44055815264663) bank108154 +108155 POINT(47.91802151438935 -123.18617247572372) bank108155 +108156 POINT(48.37483948883898 -121.58628986810318) bank108156 +108157 POINT(48.540804165737484 -121.95951914850882) bank108157 +108158 POINT(47.037880007305944 -123.24550239448013) bank108158 +108159 POINT(47.114816475044776 -121.91803910523183) bank108159 +108160 POINT(47.55986650290067 -122.91808866989219) bank108160 +108161 POINT(47.11019667944997 -121.62526772806441) bank108161 +108162 POINT(47.87111397360684 -121.85718476209482) bank108162 +108163 POINT(48.50605079498133 -123.06541057668176) bank108163 +108164 POINT(48.13035518793718 -122.42080947939117) bank108164 +108165 POINT(47.413642275385605 -122.77115655909985) bank108165 +108166 POINT(47.25940091002896 -123.04098655885198) bank108166 +108167 POINT(47.21197191807291 -121.66315852350618) bank108167 +108168 POINT(48.14381273962511 -122.11685112837958) bank108168 +108169 POINT(46.82923938817062 -121.9168389761122) bank108169 +108170 POINT(47.86286539273649 -122.341380972986) bank108170 +108171 POINT(47.29075047194747 -122.04284675525837) bank108171 +108172 POINT(47.59553211118126 -121.44293288312295) bank108172 +108173 POINT(46.985126879954855 -122.3059408107226) bank108173 +108174 POINT(47.38221501677097 -121.39279711950252) bank108174 +108175 POINT(47.434486668133985 -121.8765043562212) bank108175 +108176 POINT(47.22168135009199 -122.9489390079277) bank108176 +108177 POINT(47.78085387247477 -122.47215230108279) bank108177 +108178 POINT(48.43184270159137 -123.24171390259922) bank108178 +108179 POINT(47.076629572688994 -121.4960281969239) bank108179 +108180 POINT(48.27107271745714 -122.94083073148796) bank108180 +108181 POINT(47.716112099559695 -121.71211229368654) bank108181 +108182 POINT(47.84451386451433 -121.89989032370434) bank108182 +108183 POINT(48.432010329831925 -122.3384822899204) bank108183 +108184 POINT(48.29284864924427 -123.12651571622517) bank108184 +108185 POINT(47.99756180564757 -122.90397443135015) bank108185 +108186 POINT(48.3257116705479 -122.3220242970228) bank108186 +108187 POINT(46.93574000577139 -122.03062084169824) bank108187 +108188 POINT(48.12048020040496 -122.04510685728232) bank108188 +108189 POINT(48.33772072742046 -122.29697824802109) bank108189 +108190 POINT(47.39494269946808 -122.90344342090282) bank108190 +108191 POINT(47.102117406148274 -122.62854277201622) bank108191 +108192 POINT(47.153676944452094 -122.08259943268516) bank108192 +108193 POINT(47.4476909712142 -121.64013871810285) bank108193 +108194 POINT(47.12896085577697 -121.8548268663398) bank108194 +108195 POINT(46.632065874079395 -121.34535590957768) bank108195 +108196 POINT(47.98977928168405 -121.56071342169757) bank108196 +108197 POINT(47.154659882499374 -121.82331881226429) bank108197 +108198 POINT(47.104521497484626 -121.4230862437557) bank108198 +108199 POINT(48.08228517319421 -122.65589574224656) bank108199 +108200 POINT(48.44166380348392 -122.94128152222036) bank108200 +108201 POINT(47.77436051497713 -122.05071334373827) bank108201 +108202 POINT(47.53805891968885 -121.89597756413905) bank108202 +108203 POINT(46.78050332842738 -121.7659621389946) bank108203 +108204 POINT(47.39687960660253 -122.85563201732302) bank108204 +108205 POINT(47.7056318953253 -121.45969782528883) bank108205 +108206 POINT(47.07155491872353 -122.82597563673508) bank108206 +108207 POINT(47.314799591009645 -122.72349670536636) bank108207 +108208 POINT(48.477972860375765 -121.45742901259632) bank108208 +108209 POINT(46.92851835425469 -123.18792193766754) bank108209 +108210 POINT(47.602303970291516 -121.79720637510098) bank108210 +108211 POINT(46.84041497558579 -123.29563753938523) bank108211 +108212 POINT(47.475017893959894 -123.19395272374726) bank108212 +108213 POINT(47.60515715403928 -121.51246474453629) bank108213 +108214 POINT(48.12418539854345 -122.00534953117428) bank108214 +108215 POINT(48.119636592735056 -122.79110488773038) bank108215 +108216 POINT(46.7034764510461 -121.37850705513993) bank108216 +108217 POINT(47.308448486109235 -122.2036614872127) bank108217 +108218 POINT(48.095986568172904 -122.852587171191) bank108218 +108219 POINT(47.479878290530856 -122.07822241843562) bank108219 +108220 POINT(47.088813305282365 -123.22443405774968) bank108220 +108221 POINT(46.94332835797549 -122.42880424899408) bank108221 +108222 POINT(48.40237463920081 -121.37958479622618) bank108222 +108223 POINT(48.32089706547064 -122.75532154817382) bank108223 +108224 POINT(46.80575016212195 -122.59796132406039) bank108224 +108225 POINT(48.03032831087213 -122.25399607852766) bank108225 +108226 POINT(48.28562719766143 -122.81034443417104) bank108226 +108227 POINT(47.95820413529541 -123.03554147952048) bank108227 +108228 POINT(47.30113128051256 -123.12795266890522) bank108228 +108229 POINT(48.07469905436918 -121.49880167740274) bank108229 +108230 POINT(47.874962547497496 -121.67969910190428) bank108230 +108231 POINT(47.025680042369494 -121.43176347608168) bank108231 +108232 POINT(48.15759426275017 -121.39425258498464) bank108232 +108233 POINT(46.64731327961629 -122.23973049690652) bank108233 +108234 POINT(48.51008263220444 -121.79201963988486) bank108234 +108235 POINT(48.262250926489045 -122.90373864557507) bank108235 +108236 POINT(47.06078033133092 -122.60281460870416) bank108236 +108237 POINT(47.36909486626624 -123.06544067466383) bank108237 +108238 POINT(47.642331553319984 -121.91574749251335) bank108238 +108239 POINT(47.604954648555285 -121.86865194726228) bank108239 +108240 POINT(46.92098986992426 -121.43354546438177) bank108240 +108241 POINT(48.49489748236533 -122.61195217195946) bank108241 +108242 POINT(47.9136396280542 -122.25486592359314) bank108242 +108243 POINT(47.36814454197635 -122.75429267754565) bank108243 +108244 POINT(47.256088721830515 -122.94079699513912) bank108244 +108245 POINT(47.118389251892104 -121.79855814862718) bank108245 +108246 POINT(48.032207019919866 -122.78967332124677) bank108246 +108247 POINT(48.395408516353676 -122.22332700069107) bank108247 +108248 POINT(48.34795158096286 -121.43436007201753) bank108248 +108249 POINT(48.55292048792795 -123.13166351294988) bank108249 +108250 POINT(46.739211839822374 -122.25369086755305) bank108250 +108251 POINT(48.295782485689514 -121.4477188663758) bank108251 +108252 POINT(46.72707197677111 -123.15065665649142) bank108252 +108253 POINT(46.78276879596408 -122.5546739076815) bank108253 +108254 POINT(47.5280621916941 -121.4435561957072) bank108254 +108255 POINT(48.39703757875782 -122.72453752311361) bank108255 +108256 POINT(46.97200072249937 -121.55996486035774) bank108256 +108257 POINT(47.23067755759246 -122.52972621238015) bank108257 +108258 POINT(46.94547996607307 -122.47333066054931) bank108258 +108259 POINT(47.9578326193498 -122.28295393997476) bank108259 +108260 POINT(47.57222317512015 -121.62662970368156) bank108260 +108261 POINT(47.67904353983873 -122.25607429033765) bank108261 +108262 POINT(47.266951662248815 -121.42663166936258) bank108262 +108263 POINT(48.25365853137705 -122.44186150932657) bank108263 +108264 POINT(47.6535338416458 -121.65724207870514) bank108264 +108265 POINT(47.90123322001933 -123.27439735442483) bank108265 +108266 POINT(47.44541810439662 -122.27967427632697) bank108266 +108267 POINT(47.98367813578527 -122.7961136438643) bank108267 +108268 POINT(48.40938368947222 -122.08931144282354) bank108268 +108269 POINT(47.91867092911849 -122.41141926203927) bank108269 +108270 POINT(47.78735748699562 -122.7468709502309) bank108270 +108271 POINT(48.08850333139755 -121.98683990560984) bank108271 +108272 POINT(46.81014788372866 -122.022790974732) bank108272 +108273 POINT(48.39016162995562 -123.1784680784499) bank108273 +108274 POINT(47.11483160974843 -121.85173261374833) bank108274 +108275 POINT(47.8729866504124 -123.32523720367668) bank108275 +108276 POINT(48.41828645382631 -121.88953187292857) bank108276 +108277 POINT(47.51883607516517 -123.31252781603152) bank108277 +108278 POINT(46.8793989663063 -123.03325210853994) bank108278 +108279 POINT(46.697534815053935 -121.44900474265984) bank108279 +108280 POINT(47.153866610968834 -121.88374029888128) bank108280 +108281 POINT(48.03408087564997 -122.76414441065948) bank108281 +108282 POINT(47.498028248540905 -122.34226614324736) bank108282 +108283 POINT(47.631103935806344 -122.23552661119903) bank108283 +108284 POINT(47.1957544296888 -123.01379870568091) bank108284 +108285 POINT(47.27776895355918 -121.59112223155822) bank108285 +108286 POINT(47.552523327876486 -122.15972531035746) bank108286 +108287 POINT(48.55453079282602 -122.62827403027647) bank108287 +108288 POINT(47.55518508343237 -122.40092344592988) bank108288 +108289 POINT(48.42694446932148 -122.52670626412426) bank108289 +108290 POINT(47.16385942521795 -123.00395634097502) bank108290 +108291 POINT(46.98207541541305 -121.53825333948103) bank108291 +108292 POINT(46.75024101676749 -122.02400721168874) bank108292 +108293 POINT(48.33947354591777 -122.7873859586378) bank108293 +108294 POINT(47.51221368456181 -122.47220928038425) bank108294 +108295 POINT(47.95878260413621 -122.47853724348491) bank108295 +108296 POINT(47.86593241072654 -122.41595880952772) bank108296 +108297 POINT(47.02354685794386 -121.70388160434273) bank108297 +108298 POINT(48.32160798106147 -122.53183257921951) bank108298 +108299 POINT(46.768897290306704 -122.56010276472666) bank108299 +108300 POINT(47.51148714334569 -122.31825322685593) bank108300 +108301 POINT(48.4332672737081 -122.48367777584241) bank108301 +108302 POINT(48.32567398798341 -123.15046243911898) bank108302 +108303 POINT(48.48731479587859 -122.87403487862916) bank108303 +108304 POINT(46.74188223072501 -123.06429690746619) bank108304 +108305 POINT(46.94991832864321 -122.27371422160961) bank108305 +108306 POINT(48.44406180217116 -122.0495656116707) bank108306 +108307 POINT(48.25727675296299 -121.46327803316002) bank108307 +108308 POINT(47.936844282136185 -122.00485593460395) bank108308 +108309 POINT(47.708720932386285 -121.87782780344416) bank108309 +108310 POINT(47.65076454304073 -122.2836629763139) bank108310 +108311 POINT(47.01366487897468 -122.60359798918347) bank108311 +108312 POINT(47.81698160531844 -121.75381050823364) bank108312 +108313 POINT(48.28119577063004 -123.09091747299486) bank108313 +108314 POINT(47.78168876085432 -123.11835911206894) bank108314 +108315 POINT(47.34965465786172 -121.42203735874476) bank108315 +108316 POINT(46.69709316286896 -121.6732975922242) bank108316 +108317 POINT(46.97188319770922 -122.30000415906846) bank108317 +108318 POINT(47.634006197479565 -121.76921015420172) bank108318 +108319 POINT(47.80548308890566 -122.70442770652465) bank108319 +108320 POINT(46.70957429461938 -122.1562345717578) bank108320 +108321 POINT(46.88304259890174 -123.00355465150461) bank108321 +108322 POINT(47.964833016056026 -123.12614323737236) bank108322 +108323 POINT(48.09747338996616 -122.60149645182221) bank108323 +108324 POINT(47.90782952064313 -122.96257613150661) bank108324 +108325 POINT(48.122039286798646 -122.12721038038723) bank108325 +108326 POINT(47.31204125051729 -123.17514826478529) bank108326 +108327 POINT(47.64468433387831 -122.15700896837613) bank108327 +108328 POINT(47.80228940588873 -121.94813502824798) bank108328 +108329 POINT(46.780907698962835 -123.0939627960388) bank108329 +108330 POINT(47.162705033048084 -121.67440116685474) bank108330 +108331 POINT(48.33663265022269 -122.01289486690972) bank108331 +108332 POINT(47.69376211982041 -123.05620124308261) bank108332 +108333 POINT(47.280013285956656 -121.4191880098932) bank108333 +108334 POINT(47.271888372926966 -122.36221613247955) bank108334 +108335 POINT(47.256325994595215 -122.98439446675701) bank108335 +108336 POINT(47.76309805757027 -122.73086781469915) bank108336 +108337 POINT(47.767878182712835 -122.81050193335824) bank108337 +108338 POINT(48.45753460994635 -121.6684850513049) bank108338 +108339 POINT(47.52858330814561 -121.41420362149576) bank108339 +108340 POINT(48.13275130284605 -122.49141910789038) bank108340 +108341 POINT(47.801018344632425 -121.77586677423028) bank108341 +108342 POINT(48.22654711088882 -123.27291460804281) bank108342 +108343 POINT(47.34859233740156 -121.48594741781059) bank108343 +108344 POINT(47.6440977536599 -123.10304893886332) bank108344 +108345 POINT(46.88887408987947 -122.621170628899) bank108345 +108346 POINT(47.56369616135585 -122.6763785953078) bank108346 +108347 POINT(47.030824854356624 -122.9236122068967) bank108347 +108348 POINT(47.96151145655788 -121.62348719096693) bank108348 +108349 POINT(47.05652577512411 -122.03813279127553) bank108349 +108350 POINT(46.780152933961965 -122.70176557191597) bank108350 +108351 POINT(46.82794006169188 -122.343991491511) bank108351 +108352 POINT(48.01411755720543 -122.17589687857024) bank108352 +108353 POINT(47.65924287975507 -121.8086995247198) bank108353 +108354 POINT(47.34460715604587 -122.76029501586164) bank108354 +108355 POINT(47.4242778607738 -123.30459207609239) bank108355 +108356 POINT(47.041652418357636 -123.08065853774104) bank108356 +108357 POINT(47.68797213631982 -121.43097296225568) bank108357 +108358 POINT(47.6853986767133 -123.12183100742308) bank108358 +108359 POINT(47.42565764695309 -123.25751992810359) bank108359 +108360 POINT(47.35500308900372 -123.19693039598623) bank108360 +108361 POINT(48.29101787382326 -122.54326329547024) bank108361 +108362 POINT(48.07823742874731 -122.24888419624178) bank108362 +108363 POINT(48.49197197909102 -122.77870070318312) bank108363 +108364 POINT(48.16011899065473 -123.02027111619516) bank108364 +108365 POINT(48.5967643069022 -122.76825987056853) bank108365 +108366 POINT(47.86911223757943 -122.29622069615323) bank108366 +108367 POINT(46.68078353587146 -122.45550691997659) bank108367 +108368 POINT(47.788711312508795 -122.94058352878089) bank108368 +108369 POINT(47.56386489334242 -121.68412853193941) bank108369 +108370 POINT(47.38660634763153 -122.81374642099286) bank108370 +108371 POINT(48.0112228000832 -121.75032479013397) bank108371 +108372 POINT(47.900409531308306 -123.24304785963072) bank108372 +108373 POINT(48.12756672672486 -121.33800753431086) bank108373 +108374 POINT(47.622136554111016 -123.13984446947747) bank108374 +108375 POINT(48.49176196622308 -123.30957053853113) bank108375 +108376 POINT(47.81548659486836 -122.51648962566357) bank108376 +108377 POINT(47.834449702272 -121.88128489268236) bank108377 +108378 POINT(47.40368691112539 -123.072354987045) bank108378 +108379 POINT(47.72059505144452 -123.10068136532237) bank108379 +108380 POINT(48.13339385646075 -121.50861130255369) bank108380 +108381 POINT(47.10297036264137 -121.46048946535886) bank108381 +108382 POINT(47.628877816113544 -122.9526822396927) bank108382 +108383 POINT(48.232531755222766 -122.75240078584937) bank108383 +108384 POINT(48.064404338806966 -122.28801601807281) bank108384 +108385 POINT(48.45994832555182 -122.03472298633744) bank108385 +108386 POINT(47.47312955897567 -121.92377772148942) bank108386 +108387 POINT(47.16323809546675 -122.08134646376931) bank108387 +108388 POINT(46.71530151839717 -122.47042228105421) bank108388 +108389 POINT(47.16740997318626 -122.50440412383988) bank108389 +108390 POINT(48.50217215262059 -122.94760694495966) bank108390 +108391 POINT(47.33732786429268 -123.08659746743932) bank108391 +108392 POINT(48.44535011403901 -122.19540493818741) bank108392 +108393 POINT(48.1455031459555 -122.97616622272191) bank108393 +108394 POINT(47.145770643107674 -122.00733045934035) bank108394 +108395 POINT(48.10032631448208 -122.3974542007622) bank108395 +108396 POINT(48.11246253273165 -123.12414372870542) bank108396 +108397 POINT(46.650082709857514 -123.04952007419621) bank108397 +108398 POINT(47.3801150107341 -122.37954006087023) bank108398 +108399 POINT(46.83548020205998 -122.73084036006813) bank108399 +108400 POINT(47.16433792881029 -122.47408243007183) bank108400 +108401 POINT(47.50062379158501 -122.23795723378677) bank108401 +108402 POINT(46.89537614882353 -122.07669148018859) bank108402 +108403 POINT(48.12577335374621 -121.96109067857053) bank108403 +108404 POINT(47.47938202500318 -121.93260382095787) bank108404 +108405 POINT(47.2308829346689 -121.61257190613414) bank108405 +108406 POINT(46.729560425557466 -122.35204920462823) bank108406 +108407 POINT(47.00323028485208 -122.64320246559878) bank108407 +108408 POINT(47.559884349805 -123.00044405175454) bank108408 +108409 POINT(48.36243017028855 -121.956328571878) bank108409 +108410 POINT(48.26581858487403 -123.27040500853954) bank108410 +108411 POINT(47.826141471134534 -122.05291304738091) bank108411 +108412 POINT(48.53665319313243 -121.54316601197961) bank108412 +108413 POINT(47.66215251490625 -121.70294441660214) bank108413 +108414 POINT(47.84102457370367 -123.28055060391537) bank108414 +108415 POINT(47.35942549340303 -122.5737811035637) bank108415 +108416 POINT(47.67590554208151 -122.16527802569037) bank108416 +108417 POINT(46.659986361414305 -122.91362724730357) bank108417 +108418 POINT(48.17561355964826 -121.43728969236132) bank108418 +108419 POINT(48.3463445215603 -122.41914365553501) bank108419 +108420 POINT(47.29423527765103 -121.66483185305401) bank108420 +108421 POINT(46.784570367279954 -122.11353730292825) bank108421 +108422 POINT(47.08660351925201 -122.0798516657362) bank108422 +108423 POINT(47.26376097279226 -122.27253993164284) bank108423 +108424 POINT(46.716763046822415 -122.54848224835222) bank108424 +108425 POINT(46.806437175374555 -122.59117003254019) bank108425 +108426 POINT(46.96977325665184 -122.01927730520542) bank108426 +108427 POINT(47.23953881036348 -123.19418886413473) bank108427 +108428 POINT(47.02271246775852 -122.53871578554077) bank108428 +108429 POINT(47.639888624489 -122.62470741336841) bank108429 +108430 POINT(47.69208703988426 -122.1068085746364) bank108430 +108431 POINT(48.35359902495914 -122.0822242588359) bank108431 +108432 POINT(46.87249661426867 -121.5645662013145) bank108432 +108433 POINT(48.00649375246895 -122.76599094041977) bank108433 +108434 POINT(47.271255006753805 -123.02033026607134) bank108434 +108435 POINT(47.042907734585455 -122.38685610728493) bank108435 +108436 POINT(48.49775526838484 -122.85675662405663) bank108436 +108437 POINT(47.56385788497498 -122.01446087308722) bank108437 +108438 POINT(48.32165312169741 -121.47626721815006) bank108438 +108439 POINT(47.338147775977205 -121.77124056339157) bank108439 +108440 POINT(47.68175522046677 -123.30657993790982) bank108440 +108441 POINT(46.79161404742855 -121.85173004175232) bank108441 +108442 POINT(46.83435610026632 -122.50012478499029) bank108442 +108443 POINT(47.47634305338554 -123.09701251638691) bank108443 +108444 POINT(47.45855873879498 -122.62141028405972) bank108444 +108445 POINT(47.17416011118976 -121.88091463354044) bank108445 +108446 POINT(47.41071915793455 -122.95584027325832) bank108446 +108447 POINT(47.008581692392035 -121.37787391976903) bank108447 +108448 POINT(47.67739833513446 -122.73153093234822) bank108448 +108449 POINT(47.94166578555522 -122.67258276143234) bank108449 +108450 POINT(47.89123141853527 -121.91715594403688) bank108450 +108451 POINT(48.01230745121651 -122.21595478188334) bank108451 +108452 POINT(47.00552479922473 -123.09489259880219) bank108452 +108453 POINT(47.56834358942565 -122.71184012659282) bank108453 +108454 POINT(46.695673870555844 -122.77300186738914) bank108454 +108455 POINT(47.324279908170446 -122.55189848927235) bank108455 +108456 POINT(47.2232938714785 -121.92343973455615) bank108456 +108457 POINT(46.77587231131981 -122.90384742853188) bank108457 +108458 POINT(47.9328381334135 -121.78180117884702) bank108458 +108459 POINT(47.8483311532098 -123.10783593652623) bank108459 +108460 POINT(46.64521713415416 -122.8240125302901) bank108460 +108461 POINT(48.498500367951046 -121.59850382646) bank108461 +108462 POINT(47.1071652470949 -121.65745544122358) bank108462 +108463 POINT(47.23990997611562 -122.92046104316901) bank108463 +108464 POINT(47.62696464382766 -122.1837512963595) bank108464 +108465 POINT(47.26598897337867 -122.86710231925379) bank108465 +108466 POINT(46.72525117926355 -123.04450155231368) bank108466 +108467 POINT(47.12363606470095 -123.13824436727242) bank108467 +108468 POINT(47.974025732006226 -122.87397628969482) bank108468 +108469 POINT(47.862691772379435 -121.41486168954424) bank108469 +108470 POINT(46.919296256219724 -121.88628308475342) bank108470 +108471 POINT(47.539141667571386 -121.42577222341775) bank108471 +108472 POINT(47.947576298733004 -122.91638087732142) bank108472 +108473 POINT(46.76792811177686 -122.16527485759848) bank108473 +108474 POINT(48.014221371527924 -122.58145200130203) bank108474 +108475 POINT(47.619086052148134 -122.72522289766377) bank108475 +108476 POINT(47.889792456313 -122.26332428643227) bank108476 +108477 POINT(47.68643872219976 -121.6832577915219) bank108477 +108478 POINT(46.90201475367937 -121.73625964702742) bank108478 +108479 POINT(47.00558598307391 -122.47902162963385) bank108479 +108480 POINT(47.60774564350685 -122.05209645933623) bank108480 +108481 POINT(47.44854549557177 -122.21228727306496) bank108481 +108482 POINT(47.39630065600479 -122.53290551765025) bank108482 +108483 POINT(47.19485447585902 -121.84446771934651) bank108483 +108484 POINT(48.142700214497125 -122.98866684368478) bank108484 +108485 POINT(48.41683402205357 -121.64064091373174) bank108485 +108486 POINT(47.46914929007854 -121.94200254740294) bank108486 +108487 POINT(47.041424670143336 -123.0061076124528) bank108487 +108488 POINT(48.56334203087809 -123.21966681691251) bank108488 +108489 POINT(47.70792254493135 -121.75815347949707) bank108489 +108490 POINT(47.075279782478745 -122.66250235501226) bank108490 +108491 POINT(47.91182668691301 -122.88744460149712) bank108491 +108492 POINT(47.96640562097935 -122.3424693879466) bank108492 +108493 POINT(46.745116646761 -121.74302510539657) bank108493 +108494 POINT(47.06473484621012 -121.36233304198709) bank108494 +108495 POINT(47.26197673980298 -121.97388182978379) bank108495 +108496 POINT(46.874355932569536 -123.31964568594286) bank108496 +108497 POINT(47.18768869244112 -122.32477282698333) bank108497 +108498 POINT(47.55834729054958 -122.25513068260895) bank108498 +108499 POINT(47.198893539796536 -122.0860204274274) bank108499 +108500 POINT(47.780941207126155 -122.38906834129594) bank108500 +108501 POINT(46.8085837531702 -121.57938855755778) bank108501 +108502 POINT(47.56107814491835 -122.68167391705086) bank108502 +108503 POINT(47.52522784343646 -121.88617525328434) bank108503 +108504 POINT(47.1628320050935 -121.92320254816032) bank108504 +108505 POINT(48.123566852630674 -121.80505672261499) bank108505 +108506 POINT(47.40339654618306 -121.80370102005058) bank108506 +108507 POINT(46.65525799262331 -122.20293074121147) bank108507 +108508 POINT(47.08773222331191 -121.43123187266119) bank108508 +108509 POINT(47.95546397048957 -122.89260199374726) bank108509 +108510 POINT(47.926766274117355 -121.78301014141245) bank108510 +108511 POINT(48.290253300015486 -121.96070495111853) bank108511 +108512 POINT(47.78217216720732 -122.75214983794363) bank108512 +108513 POINT(47.77470875411532 -122.29857699276248) bank108513 +108514 POINT(47.785410603124326 -122.76431955818052) bank108514 +108515 POINT(48.55587183868926 -121.45154656027648) bank108515 +108516 POINT(47.639549258560756 -122.76789579444805) bank108516 +108517 POINT(47.49547358366627 -122.685201998911) bank108517 +108518 POINT(47.88303289283775 -123.13697032325403) bank108518 +108519 POINT(47.26163905267578 -122.40906162334944) bank108519 +108520 POINT(46.81751942772628 -122.78997600358659) bank108520 +108521 POINT(47.438471294166895 -121.92128202949193) bank108521 +108522 POINT(47.79021398807667 -122.8655790592158) bank108522 +108523 POINT(48.2285143020449 -121.77090593288209) bank108523 +108524 POINT(47.714469090192644 -121.40036978911176) bank108524 +108525 POINT(48.379683805035405 -121.81469696087179) bank108525 +108526 POINT(47.078826815424826 -121.34028658719893) bank108526 +108527 POINT(48.1985199777563 -121.65374655270384) bank108527 +108528 POINT(48.09129349803927 -123.09427230029246) bank108528 +108529 POINT(46.607099782700146 -121.98948593483027) bank108529 +108530 POINT(48.16304876830111 -123.30701228011384) bank108530 +108531 POINT(47.44980555992518 -122.18324880639567) bank108531 +108532 POINT(47.059063812293665 -122.04140274140306) bank108532 +108533 POINT(47.776266510998376 -121.78919608757768) bank108533 +108534 POINT(47.312742539880645 -122.08159847313735) bank108534 +108535 POINT(48.059194903564695 -122.33191458460666) bank108535 +108536 POINT(47.1031580927357 -122.3146390289947) bank108536 +108537 POINT(46.74176198983872 -123.3042629052053) bank108537 +108538 POINT(47.128881167020076 -122.49550394670617) bank108538 +108539 POINT(47.04994019689607 -122.76338283278177) bank108539 +108540 POINT(48.32315166813215 -121.71694412719104) bank108540 +108541 POINT(47.74673447208819 -123.16388903473104) bank108541 +108542 POINT(48.29294827615762 -122.30403784865226) bank108542 +108543 POINT(46.68425163386351 -122.74324276323469) bank108543 +108544 POINT(46.903898036581566 -123.22436018207209) bank108544 +108545 POINT(48.057075409146876 -122.69232728335888) bank108545 +108546 POINT(47.903780250747175 -122.75104032984014) bank108546 +108547 POINT(47.33117370742858 -122.05805704446733) bank108547 +108548 POINT(47.096412213019825 -121.65648387567356) bank108548 +108549 POINT(47.87169813253313 -121.74500885975198) bank108549 +108550 POINT(47.977497513392834 -122.75854472307569) bank108550 +108551 POINT(47.895503321641534 -121.94683976269674) bank108551 +108552 POINT(47.7411726853285 -123.08966722610087) bank108552 +108553 POINT(46.678037901338435 -122.3323844833886) bank108553 +108554 POINT(46.70865092206722 -121.54355428016702) bank108554 +108555 POINT(47.9713171097993 -123.23393998732405) bank108555 +108556 POINT(47.71801012105714 -123.06304664676048) bank108556 +108557 POINT(47.13573684882783 -121.71151319908587) bank108557 +108558 POINT(47.618263114943645 -122.10048749657196) bank108558 +108559 POINT(47.95495113414466 -122.01055960693068) bank108559 +108560 POINT(47.29513746871491 -121.88948522107749) bank108560 +108561 POINT(48.53782156196711 -122.63036916290163) bank108561 +108562 POINT(47.52828572735541 -123.12043162086383) bank108562 +108563 POINT(48.55577411900778 -123.14436503446524) bank108563 +108564 POINT(47.74569662828618 -121.40857012064721) bank108564 +108565 POINT(48.171720280042024 -121.63012499959358) bank108565 +108566 POINT(47.65781583749252 -121.99950029781559) bank108566 +108567 POINT(47.93942848221471 -122.11899591607039) bank108567 +108568 POINT(46.904244669600764 -121.73327509591464) bank108568 +108569 POINT(47.36832923694509 -122.03147202737146) bank108569 +108570 POINT(46.66907638789653 -121.4620144131672) bank108570 +108571 POINT(47.760446561708335 -121.89381469627652) bank108571 +108572 POINT(47.908816053465415 -122.78713370959255) bank108572 +108573 POINT(47.969093421037904 -123.20062384125708) bank108573 +108574 POINT(47.40171004415334 -122.09960848788346) bank108574 +108575 POINT(47.577829114867775 -122.9317586527116) bank108575 +108576 POINT(47.24674275010112 -121.82576856347958) bank108576 +108577 POINT(46.635455851895074 -121.79245847770771) bank108577 +108578 POINT(48.571445339832444 -122.72609335349132) bank108578 +108579 POINT(47.82514588317939 -121.76826332317535) bank108579 +108580 POINT(47.20917779388674 -121.89722048512827) bank108580 +108581 POINT(46.826342929900356 -123.32982637794665) bank108581 +108582 POINT(48.15386945164802 -123.16098494107966) bank108582 +108583 POINT(47.26902437653904 -121.85272755453225) bank108583 +108584 POINT(48.317292407436696 -121.73342341241819) bank108584 +108585 POINT(47.77539750473622 -122.03027221623188) bank108585 +108586 POINT(47.92645314311066 -122.55260133143453) bank108586 +108587 POINT(48.25249765960632 -121.85960244606292) bank108587 +108588 POINT(46.76179899808978 -122.67885963300544) bank108588 +108589 POINT(48.117568887143975 -121.66155988808435) bank108589 +108590 POINT(47.62700578741976 -121.87765873844651) bank108590 +108591 POINT(48.26145391436348 -122.44050037889242) bank108591 +108592 POINT(47.08625498260095 -121.8141612299252) bank108592 +108593 POINT(47.76862657992782 -122.73686852223864) bank108593 +108594 POINT(46.90704137615652 -121.46041534832081) bank108594 +108595 POINT(47.62867410116011 -121.4612943409606) bank108595 +108596 POINT(48.46493223163984 -122.51809078642799) bank108596 +108597 POINT(46.73070535765428 -122.95147320650294) bank108597 +108598 POINT(47.95178502306313 -123.20041530567869) bank108598 +108599 POINT(47.49354934488224 -122.30497970078954) bank108599 +108600 POINT(47.43058606666343 -123.00249493564465) bank108600 +108601 POINT(48.26313328123441 -121.84640615722837) bank108601 +108602 POINT(47.743313134826906 -122.16667897766524) bank108602 +108603 POINT(47.13636892493996 -122.7126008866685) bank108603 +108604 POINT(48.10947153339391 -122.22158107503847) bank108604 +108605 POINT(47.55157289825798 -121.52000465907582) bank108605 +108606 POINT(47.69500487398968 -123.29055419718239) bank108606 +108607 POINT(48.17806713276105 -122.99305736962073) bank108607 +108608 POINT(47.75494323400829 -121.80556063116731) bank108608 +108609 POINT(47.99919286569194 -122.31691434573148) bank108609 +108610 POINT(47.2442616948299 -123.30588650222482) bank108610 +108611 POINT(47.35742366443549 -123.03454723025921) bank108611 +108612 POINT(47.195651118110135 -122.5676502184916) bank108612 +108613 POINT(48.42209509028426 -121.73513707098138) bank108613 +108614 POINT(47.03410072886526 -121.49043707879837) bank108614 +108615 POINT(47.18679037952922 -122.83101205068472) bank108615 +108616 POINT(46.71787631189515 -122.78446755386744) bank108616 +108617 POINT(47.103017359069625 -122.48294840723307) bank108617 +108618 POINT(47.424505453727754 -122.90874657210118) bank108618 +108619 POINT(48.315789665468586 -121.66414326483323) bank108619 +108620 POINT(46.69228809441201 -122.70605160968206) bank108620 +108621 POINT(47.421950552026885 -122.62365419372722) bank108621 +108622 POINT(47.57681215583112 -123.2087514519737) bank108622 +108623 POINT(48.23663688251319 -122.7681548099022) bank108623 +108624 POINT(48.15251387481802 -122.59310426585886) bank108624 +108625 POINT(47.432424806352635 -123.0949097653062) bank108625 +108626 POINT(48.0066370297196 -122.51457230117548) bank108626 +108627 POINT(48.397874901626075 -122.10738030278546) bank108627 +108628 POINT(47.80589204025898 -122.4419752510484) bank108628 +108629 POINT(46.79597943695639 -123.26284954720772) bank108629 +108630 POINT(48.224015135540974 -123.09724042859537) bank108630 +108631 POINT(46.98418180609231 -122.89509736757451) bank108631 +108632 POINT(47.6385751748232 -121.45655963049292) bank108632 +108633 POINT(47.117857134562946 -122.95115943262913) bank108633 +108634 POINT(47.04921603512602 -122.5537716362456) bank108634 +108635 POINT(46.9905646590785 -122.81392337167432) bank108635 +108636 POINT(48.12649409492458 -122.71555137853858) bank108636 +108637 POINT(48.284791559209076 -121.58084023465867) bank108637 +108638 POINT(47.586392414721296 -123.19026443909974) bank108638 +108639 POINT(47.96392761817697 -122.93638485504313) bank108639 +108640 POINT(47.591815627401154 -122.97989811014062) bank108640 +108641 POINT(47.396184747815646 -121.43918134365927) bank108641 +108642 POINT(48.05104225715062 -122.22621542254527) bank108642 +108643 POINT(47.57831783307276 -121.6630930166784) bank108643 +108644 POINT(47.6848767353782 -122.18446894437655) bank108644 +108645 POINT(47.70199729898268 -122.76002709373901) bank108645 +108646 POINT(47.505665695727174 -122.75119772871277) bank108646 +108647 POINT(47.69474839680964 -121.51175518252143) bank108647 +108648 POINT(47.213915001488985 -121.85777117298854) bank108648 +108649 POINT(46.92921446096546 -122.83365684216713) bank108649 +108650 POINT(47.59859394202672 -122.09309906691277) bank108650 +108651 POINT(47.5550841712652 -123.14531153687564) bank108651 +108652 POINT(48.19336506612161 -122.79436360157928) bank108652 +108653 POINT(48.008158736541795 -122.82573381072224) bank108653 +108654 POINT(46.972365567583296 -121.70361370790702) bank108654 +108655 POINT(47.07627185760228 -123.27931663969206) bank108655 +108656 POINT(48.038382133707756 -121.83755374394796) bank108656 +108657 POINT(46.622933958398924 -122.355746375766) bank108657 +108658 POINT(48.14488648758379 -122.7320255445142) bank108658 +108659 POINT(47.63313929197474 -123.00124841697702) bank108659 +108660 POINT(48.55843091997295 -122.31039290178379) bank108660 +108661 POINT(48.04719090675697 -123.11047800475829) bank108661 +108662 POINT(47.40485517230074 -122.41064320277863) bank108662 +108663 POINT(47.47035747641288 -122.56798855481041) bank108663 +108664 POINT(46.68163318581252 -121.7762178447821) bank108664 +108665 POINT(48.45786276422483 -121.75347683822504) bank108665 +108666 POINT(48.37233048878291 -122.71462913716154) bank108666 +108667 POINT(47.1734053485503 -122.56261489221731) bank108667 +108668 POINT(48.241485797364774 -121.80067278921014) bank108668 +108669 POINT(47.832882352740526 -122.1908794309059) bank108669 +108670 POINT(47.517257131748124 -121.89164735019672) bank108670 +108671 POINT(47.345660625422646 -121.56415830016178) bank108671 +108672 POINT(48.32516065105947 -123.30293039494236) bank108672 +108673 POINT(47.252803060659005 -121.89640476396303) bank108673 +108674 POINT(48.39495720101154 -121.49658821851008) bank108674 +108675 POINT(47.839224680611004 -121.38906363665673) bank108675 +108676 POINT(46.75835027828124 -122.66572608688404) bank108676 +108677 POINT(46.87432784690283 -121.49155895191083) bank108677 +108678 POINT(47.19528524985563 -122.91423383104778) bank108678 +108679 POINT(48.36281029828747 -122.5267375843799) bank108679 +108680 POINT(48.20022254942189 -122.25111808855148) bank108680 +108681 POINT(46.64815813309293 -121.80117401606331) bank108681 +108682 POINT(47.46085842486138 -121.42685060746989) bank108682 +108683 POINT(47.63606465184623 -122.719095898713) bank108683 +108684 POINT(47.63471806460463 -122.18281360288437) bank108684 +108685 POINT(48.048901035272095 -122.40345972427875) bank108685 +108686 POINT(48.602819560530634 -123.28044456454082) bank108686 +108687 POINT(47.93061889426832 -123.17924689048091) bank108687 +108688 POINT(47.83448589637614 -122.80657798244579) bank108688 +108689 POINT(48.10653684110096 -122.56015416186318) bank108689 +108690 POINT(48.350612177350314 -121.56358765520629) bank108690 +108691 POINT(47.59397132518699 -123.07052337232903) bank108691 +108692 POINT(47.82717975315204 -122.81814434542657) bank108692 +108693 POINT(47.62288236297493 -123.2300721638695) bank108693 +108694 POINT(48.21595972156811 -121.78819500224459) bank108694 +108695 POINT(46.911914380208 -123.28019013521498) bank108695 +108696 POINT(48.262519009145734 -122.3122258436492) bank108696 +108697 POINT(47.64882610267751 -123.1561322769209) bank108697 +108698 POINT(47.72495937747365 -121.48936191625857) bank108698 +108699 POINT(47.28973634713447 -122.30733270147499) bank108699 +108700 POINT(48.34772813741667 -121.96382608463476) bank108700 +108701 POINT(47.78056805826797 -121.8482343836) bank108701 +108702 POINT(47.896445445055676 -122.21656164562422) bank108702 +108703 POINT(46.851740082090444 -121.65830654910361) bank108703 +108704 POINT(48.11298642754907 -122.209895675658) bank108704 +108705 POINT(47.02155803017192 -122.81352685986563) bank108705 +108706 POINT(46.85812881574894 -121.40735325684818) bank108706 +108707 POINT(48.03249937860832 -122.89738479841505) bank108707 +108708 POINT(47.73355070189168 -122.3897956101596) bank108708 +108709 POINT(46.88716295391762 -122.45820775317887) bank108709 +108710 POINT(46.99861434539406 -123.19870470473587) bank108710 +108711 POINT(46.91733258683621 -121.49331957942535) bank108711 +108712 POINT(48.07167154885871 -121.63239115941393) bank108712 +108713 POINT(47.28258240328602 -122.46607648210133) bank108713 +108714 POINT(47.46268928228642 -122.92814356140033) bank108714 +108715 POINT(48.345974773340735 -121.39678445970809) bank108715 +108716 POINT(48.20315320247711 -122.38143436687349) bank108716 +108717 POINT(48.32335008851895 -122.88815848278678) bank108717 +108718 POINT(48.01172916078249 -121.38595042281565) bank108718 +108719 POINT(46.7218768598935 -122.07240661752994) bank108719 +108720 POINT(47.15210520395022 -122.26357589361862) bank108720 +108721 POINT(48.22293873231042 -122.13806087359686) bank108721 +108722 POINT(47.90800677587527 -121.72669675554062) bank108722 +108723 POINT(48.52152141999505 -122.35519909872998) bank108723 +108724 POINT(47.80323575327987 -122.84116971313136) bank108724 +108725 POINT(48.10898851236744 -122.71607866275824) bank108725 +108726 POINT(47.42780121594157 -122.27612601531763) bank108726 +108727 POINT(48.35318400998943 -122.51085076981012) bank108727 +108728 POINT(46.987272138221044 -123.28103570090403) bank108728 +108729 POINT(47.826254301006195 -122.61685896181145) bank108729 +108730 POINT(48.273588623899684 -121.85488622972223) bank108730 +108731 POINT(47.328018225918285 -121.68829369152313) bank108731 +108732 POINT(47.20016925028402 -121.35978962102469) bank108732 +108733 POINT(47.622351307990364 -123.05856859494176) bank108733 +108734 POINT(46.63824902908836 -122.76368990625627) bank108734 +108735 POINT(46.6079306493484 -122.50592914200058) bank108735 +108736 POINT(46.82174640353652 -122.42915790483949) bank108736 +108737 POINT(47.631363936509246 -122.41049189952066) bank108737 +108738 POINT(47.17771324181522 -123.21849292252037) bank108738 +108739 POINT(47.233564833946986 -123.13341834272619) bank108739 +108740 POINT(47.98611839296199 -121.33670419112511) bank108740 +108741 POINT(46.71272520045269 -122.2884365610357) bank108741 +108742 POINT(48.54021190736401 -122.53334610737677) bank108742 +108743 POINT(48.11148439402878 -121.91840199024736) bank108743 +108744 POINT(46.832520811962844 -123.14880071476391) bank108744 +108745 POINT(48.338523794634995 -123.14723081824772) bank108745 +108746 POINT(47.15452663985285 -121.70473878205854) bank108746 +108747 POINT(48.07149127432855 -122.75786920808311) bank108747 +108748 POINT(46.759279646745675 -121.67862556740647) bank108748 +108749 POINT(48.12870609481489 -122.10829008260336) bank108749 +108750 POINT(47.154255689525456 -121.91415096289393) bank108750 +108751 POINT(47.172406514412636 -121.66571938562726) bank108751 +108752 POINT(48.21716315714428 -121.70899312141333) bank108752 +108753 POINT(46.62560907395211 -122.16728144292175) bank108753 +108754 POINT(47.25427018811899 -121.72779329934619) bank108754 +108755 POINT(47.88669311538278 -122.36783777536576) bank108755 +108756 POINT(47.501446094627255 -122.04714849155584) bank108756 +108757 POINT(48.45893098334503 -122.7568364857449) bank108757 +108758 POINT(47.935402512968494 -123.31439411842042) bank108758 +108759 POINT(47.93284683964162 -123.07509975329538) bank108759 +108760 POINT(47.93026491125095 -121.45034934914752) bank108760 +108761 POINT(48.38638256288736 -121.77532071935745) bank108761 +108762 POINT(48.33493423966596 -122.2363766428991) bank108762 +108763 POINT(47.87945114882103 -123.15424259355827) bank108763 +108764 POINT(46.80628321589791 -122.96268818672159) bank108764 +108765 POINT(46.960510553237754 -122.49979566650799) bank108765 +108766 POINT(48.10144125275339 -121.55671725923916) bank108766 +108767 POINT(48.53694063492736 -121.63487105855555) bank108767 +108768 POINT(47.16702062518525 -122.30209982713984) bank108768 +108769 POINT(48.47865390662859 -123.17355846796694) bank108769 +108770 POINT(47.20489190282278 -122.64613305417225) bank108770 +108771 POINT(47.80820213151206 -122.96740967023528) bank108771 +108772 POINT(46.721476925834466 -121.59318343675322) bank108772 +108773 POINT(47.578597475920354 -121.42297916730257) bank108773 +108774 POINT(46.627947550973715 -122.30471987925343) bank108774 +108775 POINT(47.06616672287664 -123.09574605700615) bank108775 +108776 POINT(47.02891684018584 -122.04491722666253) bank108776 +108777 POINT(47.40500210924573 -121.64972146125545) bank108777 +108778 POINT(48.212688112816814 -121.6878252951652) bank108778 +108779 POINT(47.34512952708021 -122.5749188326974) bank108779 +108780 POINT(46.93291740227676 -122.85798627484887) bank108780 +108781 POINT(48.0136647112127 -121.75025542128715) bank108781 +108782 POINT(48.24751700775123 -121.85661968046206) bank108782 +108783 POINT(48.01851258020912 -122.06340865256828) bank108783 +108784 POINT(47.68783531121594 -122.46168938574161) bank108784 +108785 POINT(47.311010291696036 -122.64808503394148) bank108785 +108786 POINT(47.87006808033233 -122.77305170055658) bank108786 +108787 POINT(46.90128868643085 -122.09779048618161) bank108787 +108788 POINT(47.46343289768767 -121.67279761715082) bank108788 +108789 POINT(47.99817263717831 -122.96217705527665) bank108789 +108790 POINT(47.76354331073225 -122.8199456065518) bank108790 +108791 POINT(47.000291870362524 -123.07332250782976) bank108791 +108792 POINT(48.43133473534235 -123.13130868644429) bank108792 +108793 POINT(47.77667105113386 -122.68342557736798) bank108793 +108794 POINT(47.43059317190832 -121.47430082088103) bank108794 +108795 POINT(48.25213670964116 -121.787581015842) bank108795 +108796 POINT(48.402132099842085 -122.54366133203914) bank108796 +108797 POINT(47.95674609443302 -122.54606357713354) bank108797 +108798 POINT(48.319876018055304 -122.19208757183816) bank108798 +108799 POINT(46.773524729387645 -123.10910052317205) bank108799 +108800 POINT(47.421733744830064 -121.769240857763) bank108800 +108801 POINT(48.19139901489897 -122.72929543882819) bank108801 +108802 POINT(46.721215657674755 -122.77465694372215) bank108802 +108803 POINT(46.997656630229315 -121.39914951833484) bank108803 +108804 POINT(47.61010881639175 -122.00573913229864) bank108804 +108805 POINT(47.903634314518364 -122.35656323165921) bank108805 +108806 POINT(47.044161195431656 -121.83185027196103) bank108806 +108807 POINT(47.97805939363762 -121.36664205036156) bank108807 +108808 POINT(48.45991181736505 -122.55552957556513) bank108808 +108809 POINT(47.52428834779723 -123.28853429620546) bank108809 +108810 POINT(47.47207612388568 -122.35423292471242) bank108810 +108811 POINT(47.83625688655159 -121.88135868647149) bank108811 +108812 POINT(47.94924159907564 -122.32642019720014) bank108812 +108813 POINT(47.73368498646665 -122.00020666047344) bank108813 +108814 POINT(47.22446131909048 -121.54093707323347) bank108814 +108815 POINT(48.28503538111386 -121.80434599571551) bank108815 +108816 POINT(47.21261952146803 -121.86303318181601) bank108816 +108817 POINT(47.9918432470481 -121.33511763658922) bank108817 +108818 POINT(48.225463297419054 -121.46126753671786) bank108818 +108819 POINT(48.04448949214258 -121.44727696712657) bank108819 +108820 POINT(47.166893951291 -121.61082194393109) bank108820 +108821 POINT(47.74173744023786 -122.48129543184501) bank108821 +108822 POINT(46.76567036475324 -121.42617136924056) bank108822 +108823 POINT(48.201999268676396 -122.89459441570897) bank108823 +108824 POINT(47.516979637705525 -121.90661669354346) bank108824 +108825 POINT(48.55028978270017 -122.97320824213463) bank108825 +108826 POINT(47.30215036046948 -123.07203535057737) bank108826 +108827 POINT(47.27880457658364 -121.5646825149225) bank108827 +108828 POINT(46.93951074661081 -121.60744164566844) bank108828 +108829 POINT(48.094576190876815 -122.9302486284993) bank108829 +108830 POINT(48.387939712969064 -121.96855801736106) bank108830 +108831 POINT(47.964086260654526 -121.94611831849974) bank108831 +108832 POINT(48.533055412291354 -122.5750785086504) bank108832 +108833 POINT(47.032397751984846 -122.73278774887385) bank108833 +108834 POINT(46.87205839441519 -122.89648219212881) bank108834 +108835 POINT(48.10597374944124 -122.65702778805542) bank108835 +108836 POINT(48.464755433844005 -123.24496126919544) bank108836 +108837 POINT(46.953965151068424 -122.85538144308336) bank108837 +108838 POINT(48.1935424702696 -123.1270073506836) bank108838 +108839 POINT(48.42020266282323 -123.11187489883315) bank108839 +108840 POINT(47.299096429927246 -122.09199756998372) bank108840 +108841 POINT(47.97353798456209 -123.29292823712032) bank108841 +108842 POINT(48.30826256657903 -121.44688671153415) bank108842 +108843 POINT(47.41161144709484 -122.26920534952899) bank108843 +108844 POINT(46.804557368072246 -121.49178841029025) bank108844 +108845 POINT(47.010809467744195 -122.50181529419727) bank108845 +108846 POINT(48.577335976958345 -122.49749365149312) bank108846 +108847 POINT(48.53890617551855 -122.18940611786573) bank108847 +108848 POINT(47.748697735159595 -122.44152917308828) bank108848 +108849 POINT(47.49780548229351 -122.25138290783485) bank108849 +108850 POINT(46.92342649380927 -123.31152713340835) bank108850 +108851 POINT(46.60696698061918 -122.47132554486647) bank108851 +108852 POINT(47.38155916894917 -122.77812444103851) bank108852 +108853 POINT(48.22377351665347 -122.07586014095342) bank108853 +108854 POINT(47.269885256043025 -122.6468360744613) bank108854 +108855 POINT(47.280702322113 -122.41833715611762) bank108855 +108856 POINT(47.82796146959711 -122.76742424322248) bank108856 +108857 POINT(47.99478286040312 -121.94667071464423) bank108857 +108858 POINT(46.653807902461196 -121.39824014407019) bank108858 +108859 POINT(47.420109602948294 -122.11230533873558) bank108859 +108860 POINT(48.027151192822416 -122.87342415539106) bank108860 +108861 POINT(47.20684186797959 -122.41979254367048) bank108861 +108862 POINT(46.673398505123146 -122.41153225569896) bank108862 +108863 POINT(47.236348585729694 -122.51891176925744) bank108863 +108864 POINT(46.9522526186985 -122.8210285222228) bank108864 +108865 POINT(46.86577958551803 -122.22961233815006) bank108865 +108866 POINT(46.75895022565951 -123.13781854432553) bank108866 +108867 POINT(48.40566798455784 -121.83114889424941) bank108867 +108868 POINT(46.704306878993954 -122.84912268974526) bank108868 +108869 POINT(47.281512310149786 -122.46176405657505) bank108869 +108870 POINT(47.32519291122013 -123.07011047118905) bank108870 +108871 POINT(48.077873307584966 -121.85507089020433) bank108871 +108872 POINT(48.447216683889316 -121.50301085023753) bank108872 +108873 POINT(47.859398029394434 -122.5811423108044) bank108873 +108874 POINT(47.556805554400455 -123.00720890265384) bank108874 +108875 POINT(47.14443427553194 -121.4938530667839) bank108875 +108876 POINT(46.941095376583306 -122.71585435113188) bank108876 +108877 POINT(47.69603377217172 -122.10795216721264) bank108877 +108878 POINT(48.056201770216745 -122.66729428916899) bank108878 +108879 POINT(46.68750090002949 -121.50505501263977) bank108879 +108880 POINT(47.569146383736005 -121.4465316894039) bank108880 +108881 POINT(47.202140676641875 -123.24242650134433) bank108881 +108882 POINT(47.18415300594558 -121.764258897868) bank108882 +108883 POINT(47.6980735501223 -121.59018376216459) bank108883 +108884 POINT(47.55612431305816 -122.22738057483978) bank108884 +108885 POINT(47.42374133521504 -122.83426839042436) bank108885 +108886 POINT(46.99614320564339 -122.34269713627631) bank108886 +108887 POINT(47.63979993296607 -123.23375582323507) bank108887 +108888 POINT(47.430336617317394 -122.64508574761734) bank108888 +108889 POINT(47.87825119133466 -122.0347480369413) bank108889 +108890 POINT(47.61890319672605 -122.57739486680143) bank108890 +108891 POINT(48.55349661196375 -122.20788766566932) bank108891 +108892 POINT(47.267530571071696 -122.95751875672642) bank108892 +108893 POINT(46.684888848664315 -123.27130909236789) bank108893 +108894 POINT(47.14796372865174 -122.26704949337153) bank108894 +108895 POINT(47.1342942540186 -122.5346492339209) bank108895 +108896 POINT(48.12101054977439 -121.93904144375976) bank108896 +108897 POINT(48.242908639708986 -121.8739744894723) bank108897 +108898 POINT(48.145293448389246 -122.37940615888623) bank108898 +108899 POINT(47.641259476498234 -121.74271198639369) bank108899 +108900 POINT(47.65809887053782 -121.58437647862164) bank108900 +108901 POINT(48.37969116012062 -122.30937928286693) bank108901 +108902 POINT(47.942404988574076 -123.16688260019552) bank108902 +108903 POINT(46.7445753082046 -122.46076990235655) bank108903 +108904 POINT(48.43714057116537 -122.23205056752468) bank108904 +108905 POINT(47.75168509244198 -121.52277543141338) bank108905 +108906 POINT(47.82204133220353 -122.53738216982022) bank108906 +108907 POINT(48.14387012810777 -123.17258198777932) bank108907 +108908 POINT(46.736257798990486 -122.40136400392906) bank108908 +108909 POINT(48.24982434980869 -121.98801973963722) bank108909 +108910 POINT(47.69388988910587 -121.49503815024005) bank108910 +108911 POINT(47.48659832922943 -121.40966414713964) bank108911 +108912 POINT(47.99743328949149 -122.00124295239111) bank108912 +108913 POINT(47.31572442413785 -122.39658892927788) bank108913 +108914 POINT(46.70234636683231 -122.26364459743056) bank108914 +108915 POINT(47.47454841424661 -122.99233637726718) bank108915 +108916 POINT(46.82406071186459 -122.22189008880838) bank108916 +108917 POINT(47.541568219715984 -121.59116721306788) bank108917 +108918 POINT(47.58677895597229 -121.3975318692636) bank108918 +108919 POINT(46.613901566199424 -122.38160339297191) bank108919 +108920 POINT(46.85188694347146 -121.85941562848365) bank108920 +108921 POINT(48.29833152136981 -121.87961903112341) bank108921 +108922 POINT(48.37498768058173 -122.0354302123151) bank108922 +108923 POINT(48.48448640162008 -121.87605583991454) bank108923 +108924 POINT(48.0826204165692 -121.41916442179927) bank108924 +108925 POINT(48.17837998233861 -122.72680745245363) bank108925 +108926 POINT(46.98021789098857 -121.36984752988155) bank108926 +108927 POINT(48.233209906774135 -121.4695731041167) bank108927 +108928 POINT(47.90323359779344 -121.35508316745229) bank108928 +108929 POINT(47.28638240305536 -122.60716521960394) bank108929 +108930 POINT(46.78202204738565 -121.85544198687077) bank108930 +108931 POINT(48.23100734649827 -121.54805210847883) bank108931 +108932 POINT(47.53991170657805 -122.85725157141525) bank108932 +108933 POINT(48.5638773078817 -122.80579896312237) bank108933 +108934 POINT(47.30627352135001 -121.91819660069298) bank108934 +108935 POINT(48.02487847894937 -122.45123405552428) bank108935 +108936 POINT(48.528982804941805 -121.5361925298026) bank108936 +108937 POINT(46.94256299116921 -121.38206021586805) bank108937 +108938 POINT(48.58233600744229 -122.19759937291067) bank108938 +108939 POINT(47.12900796664013 -122.09098094430391) bank108939 +108940 POINT(48.248298317526675 -121.60825660993792) bank108940 +108941 POINT(48.53378040299974 -123.2866484251186) bank108941 +108942 POINT(47.60527148077372 -122.11368942347607) bank108942 +108943 POINT(47.68069568104047 -122.00943843042346) bank108943 +108944 POINT(47.29110612909827 -122.7624136392326) bank108944 +108945 POINT(47.06943006882146 -123.06188375091372) bank108945 +108946 POINT(47.12236756494932 -123.1557144095716) bank108946 +108947 POINT(47.91676514612783 -121.68528012615847) bank108947 +108948 POINT(46.703615577369845 -123.09117966977277) bank108948 +108949 POINT(47.26112253603065 -122.37233847161896) bank108949 +108950 POINT(48.38929659202247 -121.3820044237884) bank108950 +108951 POINT(47.78528593167779 -123.18811751948714) bank108951 +108952 POINT(46.72012996832418 -121.85412824674742) bank108952 +108953 POINT(46.96504911352419 -122.15069989092815) bank108953 +108954 POINT(48.50802806275912 -123.27030621982863) bank108954 +108955 POINT(47.865025719684304 -122.72004499087554) bank108955 +108956 POINT(46.86014165735039 -122.56975012812505) bank108956 +108957 POINT(46.62630227405296 -121.68450055128578) bank108957 +108958 POINT(47.52403074734606 -123.04585671833004) bank108958 +108959 POINT(47.56578243185139 -123.06859031183757) bank108959 +108960 POINT(47.108847970624815 -122.59278183891327) bank108960 +108961 POINT(48.178629678014126 -123.14003524254406) bank108961 +108962 POINT(48.37547633805714 -122.47847216077501) bank108962 +108963 POINT(47.52149655894862 -122.21396450095499) bank108963 +108964 POINT(48.1948650163265 -123.21637257448558) bank108964 +108965 POINT(46.64124600100663 -121.95932935761098) bank108965 +108966 POINT(48.00902470007236 -121.81127197141276) bank108966 +108967 POINT(48.405510595979884 -122.00413741890674) bank108967 +108968 POINT(48.39437162968243 -123.18399688879359) bank108968 +108969 POINT(48.578680948259255 -122.42511798094517) bank108969 +108970 POINT(48.57817373111099 -122.23822272081865) bank108970 +108971 POINT(47.91200846918523 -121.42950287231072) bank108971 +108972 POINT(47.5481965491017 -122.24156812530462) bank108972 +108973 POINT(47.105386747121855 -122.26972514868672) bank108973 +108974 POINT(47.729340266940206 -122.48654320947554) bank108974 +108975 POINT(47.87136114988288 -121.79391859220219) bank108975 +108976 POINT(48.54045022617229 -122.44449744513206) bank108976 +108977 POINT(47.57659133602157 -122.445187562227) bank108977 +108978 POINT(48.14373397040268 -122.4526474450275) bank108978 +108979 POINT(47.573117554605346 -121.69394796507753) bank108979 +108980 POINT(47.61031761685627 -123.21776872265414) bank108980 +108981 POINT(48.3736584262251 -122.98057117408352) bank108981 +108982 POINT(46.87626685928543 -121.44072149063194) bank108982 +108983 POINT(47.35717798801782 -122.84346469720161) bank108983 +108984 POINT(47.76737974543936 -123.25309493690106) bank108984 +108985 POINT(47.25792617088581 -122.79007922899521) bank108985 +108986 POINT(47.77018830234543 -122.44695973665506) bank108986 +108987 POINT(47.20044587332328 -122.82527613113123) bank108987 +108988 POINT(47.70082210413774 -123.03819074364147) bank108988 +108989 POINT(47.76845022590559 -122.5138055090852) bank108989 +108990 POINT(46.84038003319244 -122.65968373220512) bank108990 +108991 POINT(48.085055694456756 -122.44230715678509) bank108991 +108992 POINT(47.8860572393705 -121.83876698117618) bank108992 +108993 POINT(47.75778354180609 -121.45953732206526) bank108993 +108994 POINT(48.35861433198464 -122.77068348086969) bank108994 +108995 POINT(46.73917627729279 -122.84970693538177) bank108995 +108996 POINT(47.80258765993729 -122.3310304748941) bank108996 +108997 POINT(47.86774127559299 -121.61626822558412) bank108997 +108998 POINT(47.5886430168099 -122.71236974559649) bank108998 +108999 POINT(47.659091513402394 -122.89819623800118) bank108999 +109000 POINT(47.99256623387114 -122.40012697564445) bank109000 +109001 POINT(47.72385721155026 -122.60878055014689) bank109001 +109002 POINT(47.491184506572026 -122.15326649613141) bank109002 +109003 POINT(48.55373682770068 -122.90300353477896) bank109003 +109004 POINT(47.59961438790922 -122.85259795220976) bank109004 +109005 POINT(47.10055481510346 -121.82383066310506) bank109005 +109006 POINT(47.192684729168654 -123.20532688009011) bank109006 +109007 POINT(48.582531022092226 -122.68575117506214) bank109007 +109008 POINT(47.087476916031605 -121.6615209176907) bank109008 +109009 POINT(47.218841731725874 -122.70412133877826) bank109009 +109010 POINT(47.816323851703146 -122.6885977379528) bank109010 +109011 POINT(46.91918431061705 -123.07128522005277) bank109011 +109012 POINT(48.299942601333925 -121.67785687219198) bank109012 +109013 POINT(48.06632419497356 -122.37863930374593) bank109013 +109014 POINT(46.990459646477774 -122.41090943870547) bank109014 +109015 POINT(47.45665150822759 -123.2605134590014) bank109015 +109016 POINT(46.8148270050791 -121.50342115948953) bank109016 +109017 POINT(48.32993944488363 -121.34150654187597) bank109017 +109018 POINT(48.11626762815834 -122.79454023494033) bank109018 +109019 POINT(48.07958155746809 -122.5639081773444) bank109019 +109020 POINT(47.42125083566878 -122.66792875329647) bank109020 +109021 POINT(46.79352174693366 -121.86100122724451) bank109021 +109022 POINT(47.16486303877993 -122.18302608659187) bank109022 +109023 POINT(46.73133363985543 -122.94184502068944) bank109023 +109024 POINT(47.8220819410441 -122.76916253017963) bank109024 +109025 POINT(46.83948379731982 -122.33202715767403) bank109025 +109026 POINT(47.28978171461543 -122.83064914121488) bank109026 +109027 POINT(47.97590214316469 -121.659611548149) bank109027 +109028 POINT(48.29084441431227 -122.74508687425947) bank109028 +109029 POINT(47.48390367886567 -122.00886753949843) bank109029 +109030 POINT(47.226542841311144 -122.65255763326118) bank109030 +109031 POINT(47.32930260365426 -122.01919154953397) bank109031 +109032 POINT(47.46670574576834 -122.596412445512) bank109032 +109033 POINT(48.39484239135649 -122.88854677804416) bank109033 +109034 POINT(47.84572754129485 -121.36318771643008) bank109034 +109035 POINT(46.823525725958234 -123.07291308132179) bank109035 +109036 POINT(48.0618471816032 -121.51872642444708) bank109036 +109037 POINT(47.5522359493548 -121.46447402474134) bank109037 +109038 POINT(47.44818398972089 -121.69913652926773) bank109038 +109039 POINT(47.62919290141953 -123.27448296877128) bank109039 +109040 POINT(47.176664705157364 -123.0270454373501) bank109040 +109041 POINT(48.505484996560035 -123.31424203542694) bank109041 +109042 POINT(47.97021050523955 -123.1505558634881) bank109042 +109043 POINT(46.64800900395234 -123.08334850238055) bank109043 +109044 POINT(48.400238942860135 -121.97882360127684) bank109044 +109045 POINT(46.83646419958046 -121.47393285096) bank109045 +109046 POINT(47.21085058540869 -122.96633176926412) bank109046 +109047 POINT(48.23824548624095 -122.84714351114414) bank109047 +109048 POINT(48.00889974244771 -121.44482699875671) bank109048 +109049 POINT(46.7059363749101 -122.51822551675579) bank109049 +109050 POINT(48.022326494725576 -123.24199278013006) bank109050 +109051 POINT(46.64825475491104 -121.57531447759405) bank109051 +109052 POINT(46.97630121911885 -121.7114012730463) bank109052 +109053 POINT(47.32650521176352 -122.66463286171535) bank109053 +109054 POINT(47.14533765839774 -122.08693951459583) bank109054 +109055 POINT(47.12879391637077 -121.836955542108) bank109055 +109056 POINT(46.74096739103515 -122.03061172687782) bank109056 +109057 POINT(47.06639406819715 -121.87411945070751) bank109057 +109058 POINT(47.07120230082353 -122.53317559630874) bank109058 +109059 POINT(46.729703081209784 -122.1811865080246) bank109059 +109060 POINT(47.46285368328666 -121.41312944246816) bank109060 +109061 POINT(46.656111417635046 -122.49916911632643) bank109061 +109062 POINT(46.86745156858298 -121.41439383868368) bank109062 +109063 POINT(47.639470618520626 -122.5116141173657) bank109063 +109064 POINT(47.19988087593424 -122.66186192254146) bank109064 +109065 POINT(46.96827259894251 -123.171508997775) bank109065 +109066 POINT(47.61079453602196 -121.38487356950249) bank109066 +109067 POINT(46.81863407326075 -122.60056531937505) bank109067 +109068 POINT(46.65859382345679 -121.43102650238212) bank109068 +109069 POINT(48.53810716898499 -121.45762633567574) bank109069 +109070 POINT(48.29984373603277 -121.57984536772022) bank109070 +109071 POINT(47.16472377786181 -121.35103822493889) bank109071 +109072 POINT(47.521914209311376 -121.40848156812156) bank109072 +109073 POINT(47.09253713856184 -122.39687169274629) bank109073 +109074 POINT(48.507261923809956 -122.19531814858081) bank109074 +109075 POINT(47.29835436941263 -121.95254801183778) bank109075 +109076 POINT(47.0204779814575 -123.29129916201485) bank109076 +109077 POINT(47.4373467531625 -122.30105841264385) bank109077 +109078 POINT(47.232764884255246 -121.44800084248122) bank109078 +109079 POINT(47.52517375893734 -122.92855818325394) bank109079 +109080 POINT(48.251132469495 -123.02679914951563) bank109080 +109081 POINT(47.795260693598266 -121.59323454034792) bank109081 +109082 POINT(47.52786755256788 -122.7569826016602) bank109082 +109083 POINT(48.28399921431619 -122.92789261049353) bank109083 +109084 POINT(46.68726671704232 -121.36205985357012) bank109084 +109085 POINT(46.728718525866896 -121.83540083670808) bank109085 +109086 POINT(47.590243229139666 -122.65924038304438) bank109086 +109087 POINT(47.29349154917824 -121.4982027351021) bank109087 +109088 POINT(47.440525909985254 -122.79682361231981) bank109088 +109089 POINT(48.340244644647726 -122.17499382834961) bank109089 +109090 POINT(46.695283960256624 -122.56802309682504) bank109090 +109091 POINT(47.921933950304805 -121.59153737541854) bank109091 +109092 POINT(47.43786699427237 -121.48850518023472) bank109092 +109093 POINT(48.18621828796012 -121.80434129316349) bank109093 +109094 POINT(46.85287699911571 -121.84110274532479) bank109094 +109095 POINT(47.07125671691271 -121.96854410302554) bank109095 +109096 POINT(46.6600475764038 -122.17779699540681) bank109096 +109097 POINT(48.332101866170134 -123.01984005036293) bank109097 +109098 POINT(47.70651194989995 -122.13147062500701) bank109098 +109099 POINT(48.524649359055296 -121.9247906769527) bank109099 +109100 POINT(47.50786539289182 -122.33272587656676) bank109100 +109101 POINT(47.34106727050322 -121.77519408695356) bank109101 +109102 POINT(46.817397759212085 -122.99684644214969) bank109102 +109103 POINT(47.06367169687405 -122.01035339006457) bank109103 +109104 POINT(46.777902816558694 -121.44186910009094) bank109104 +109105 POINT(47.21327862592468 -121.90986687076332) bank109105 +109106 POINT(47.874205833801206 -122.52447082009715) bank109106 +109107 POINT(46.95633439395275 -122.97042377086589) bank109107 +109108 POINT(48.27737025539696 -121.38477780249518) bank109108 +109109 POINT(48.29808128435502 -122.63055693867685) bank109109 +109110 POINT(47.30789221729925 -121.7439649949436) bank109110 +109111 POINT(48.03652785848087 -121.53500162401687) bank109111 +109112 POINT(47.324797392801116 -122.5822610620083) bank109112 +109113 POINT(47.99598514488574 -122.27172796894939) bank109113 +109114 POINT(46.89082204511919 -121.98088107081453) bank109114 +109115 POINT(47.801802359217945 -123.06343158591113) bank109115 +109116 POINT(47.97084984991572 -122.51096937764517) bank109116 +109117 POINT(47.56933278721221 -121.80756888509534) bank109117 +109118 POINT(47.901745086584896 -122.26493322203662) bank109118 +109119 POINT(46.81659953406542 -121.44682042619121) bank109119 +109120 POINT(48.19472557867406 -123.28630263381463) bank109120 +109121 POINT(48.575174608329355 -123.1997242884268) bank109121 +109122 POINT(47.96435159666436 -121.73689037320156) bank109122 +109123 POINT(48.08895874554144 -121.40719981245499) bank109123 +109124 POINT(47.55278163264704 -121.8355892519577) bank109124 +109125 POINT(48.55199629403647 -123.20800154752388) bank109125 +109126 POINT(48.356536624542755 -121.90080906995954) bank109126 +109127 POINT(46.876490330532434 -122.63434141210203) bank109127 +109128 POINT(48.53259829780143 -123.04438342369355) bank109128 +109129 POINT(47.08341019067324 -121.68940812040584) bank109129 +109130 POINT(46.83205356423098 -122.42764292247426) bank109130 +109131 POINT(47.80511023825158 -121.85389607430689) bank109131 +109132 POINT(47.22598772083781 -122.25719114262013) bank109132 +109133 POINT(46.725069714599 -122.29269247602055) bank109133 +109134 POINT(48.60235998857072 -121.67408425017514) bank109134 +109135 POINT(48.56966821123853 -122.76228783250455) bank109135 +109136 POINT(47.070819124822066 -122.00094045282461) bank109136 +109137 POINT(48.52252885367959 -121.97555918201606) bank109137 +109138 POINT(47.798568516068144 -121.63776725894428) bank109138 +109139 POINT(47.15464419441864 -121.54550681580275) bank109139 +109140 POINT(47.047261070337 -121.6763111102272) bank109140 +109141 POINT(47.27913056439846 -121.59824456904614) bank109141 +109142 POINT(46.932351424555044 -121.92553048541403) bank109142 +109143 POINT(46.931576033421365 -121.47583470756994) bank109143 +109144 POINT(47.88885543703826 -121.44234547250947) bank109144 +109145 POINT(47.81069598018867 -122.10884589862394) bank109145 +109146 POINT(47.799266979977034 -121.77730366896766) bank109146 +109147 POINT(47.00642192338331 -121.59687551451394) bank109147 +109148 POINT(47.15276968033332 -121.85117807927261) bank109148 +109149 POINT(47.74091775025317 -122.53303334081245) bank109149 +109150 POINT(46.754118868117025 -121.71937293479651) bank109150 +109151 POINT(47.88899267590594 -121.67924043370053) bank109151 +109152 POINT(48.3780973661047 -123.11542639220657) bank109152 +109153 POINT(47.98787992898826 -122.10455145306365) bank109153 +109154 POINT(48.46105125674879 -122.97473136447498) bank109154 +109155 POINT(48.05134252471785 -121.99274412552721) bank109155 +109156 POINT(47.07663669150062 -121.56622267721889) bank109156 +109157 POINT(46.62739165905435 -121.92632268639143) bank109157 +109158 POINT(48.22865881713261 -121.33583565693014) bank109158 +109159 POINT(48.28892628839313 -123.32429751440715) bank109159 +109160 POINT(48.49359048373298 -122.01053455062724) bank109160 +109161 POINT(47.26875088579019 -121.48525051937209) bank109161 +109162 POINT(48.46664041060214 -122.6870587096062) bank109162 +109163 POINT(47.230122824744186 -121.40168633188085) bank109163 +109164 POINT(47.7534914452778 -121.37331975583514) bank109164 +109165 POINT(47.291210505503216 -121.44311877532537) bank109165 +109166 POINT(47.38788770141683 -121.60209620633093) bank109166 +109167 POINT(48.37754515715033 -123.16282384918445) bank109167 +109168 POINT(47.87241218707692 -123.23438195389163) bank109168 +109169 POINT(47.627076888739374 -121.4149893338745) bank109169 +109170 POINT(48.548401076922 -122.89540918614121) bank109170 +109171 POINT(48.29937999469556 -123.14367451378014) bank109171 +109172 POINT(48.34376264120879 -122.16832023118478) bank109172 +109173 POINT(48.3653687205089 -121.99378847320602) bank109173 +109174 POINT(47.896201429828814 -122.35015512146123) bank109174 +109175 POINT(47.9503737073154 -122.41288129590966) bank109175 +109176 POINT(47.63136489775675 -123.21580612722643) bank109176 +109177 POINT(48.4470667199652 -122.47196563420698) bank109177 +109178 POINT(47.06307448689406 -121.6876403895576) bank109178 +109179 POINT(48.414169054782896 -123.01528934745228) bank109179 +109180 POINT(46.998336506597454 -121.9809700029997) bank109180 +109181 POINT(48.38195066018329 -121.47445432297887) bank109181 +109182 POINT(46.948603739598845 -122.18013005320311) bank109182 +109183 POINT(47.44490579891795 -122.71904091256182) bank109183 +109184 POINT(46.80361714285741 -121.86081257623798) bank109184 +109185 POINT(46.6459524475156 -121.49558601562278) bank109185 +109186 POINT(47.24338017980531 -122.51326066700686) bank109186 +109187 POINT(47.222233918779445 -122.55061420843188) bank109187 +109188 POINT(48.02929728584408 -121.72585689679124) bank109188 +109189 POINT(48.119600626496506 -122.14144541677783) bank109189 +109190 POINT(47.969673677296555 -123.14433711966294) bank109190 +109191 POINT(48.431290022373126 -122.86142045367947) bank109191 +109192 POINT(47.039308981557504 -123.05442554236627) bank109192 +109193 POINT(46.77720874307476 -122.24464486601084) bank109193 +109194 POINT(48.09281477982327 -122.19528950743049) bank109194 +109195 POINT(47.9826538205902 -122.46890619467345) bank109195 +109196 POINT(46.95637908874439 -122.79243725646292) bank109196 +109197 POINT(48.07550698992108 -121.60472216507263) bank109197 +109198 POINT(46.95066858121578 -121.79223310700763) bank109198 +109199 POINT(46.87409314733322 -121.63090459868982) bank109199 +109200 POINT(47.692790011343526 -122.47198903125259) bank109200 +109201 POINT(47.10777574310431 -121.61893862284059) bank109201 +109202 POINT(47.02895228343043 -122.51761630168423) bank109202 +109203 POINT(47.10976118518067 -123.18856461344832) bank109203 +109204 POINT(46.65868571534058 -122.39162329298655) bank109204 +109205 POINT(48.09957765138503 -122.01694884438153) bank109205 +109206 POINT(47.916200077487645 -122.18252061212891) bank109206 +109207 POINT(48.20298504889812 -121.46245874447276) bank109207 +109208 POINT(46.837970081446656 -122.28919481777557) bank109208 +109209 POINT(47.882592889033894 -122.36362654798697) bank109209 +109210 POINT(48.26001849264026 -121.43011621496089) bank109210 +109211 POINT(48.13713941609983 -122.29194930114662) bank109211 +109212 POINT(47.906789394798366 -122.24516937300328) bank109212 +109213 POINT(47.64743787142007 -122.52676595050376) bank109213 +109214 POINT(47.14868276957284 -121.47899295880485) bank109214 +109215 POINT(46.75722811161134 -122.16799363636751) bank109215 +109216 POINT(47.72868616946041 -122.24223083805605) bank109216 +109217 POINT(47.7973330260599 -122.59735319296591) bank109217 +109218 POINT(48.44400193470196 -122.45679467707717) bank109218 +109219 POINT(47.51266721314768 -121.62735923626293) bank109219 +109220 POINT(47.959214093223316 -123.05331734321818) bank109220 +109221 POINT(47.62028068461837 -123.28182217304426) bank109221 +109222 POINT(47.634764369038976 -122.80358012074879) bank109222 +109223 POINT(47.53922814961855 -121.39340191542642) bank109223 +109224 POINT(47.951488231893336 -121.6356576932292) bank109224 +109225 POINT(48.412671756486155 -123.21512535496453) bank109225 +109226 POINT(47.83850937335585 -122.93585134823195) bank109226 +109227 POINT(47.171253494552445 -122.56050572140946) bank109227 +109228 POINT(47.039810949191995 -122.875462565657) bank109228 +109229 POINT(48.027533173363885 -122.29299320735727) bank109229 +109230 POINT(47.999351901010094 -122.2537363165867) bank109230 +109231 POINT(48.53540378968577 -121.45553450377115) bank109231 +109232 POINT(47.966912103967886 -122.96476896425172) bank109232 +109233 POINT(48.329412447649226 -121.85213858904056) bank109233 +109234 POINT(47.1877574072932 -123.1340144852621) bank109234 +109235 POINT(47.7183568122908 -122.76653145356187) bank109235 +109236 POINT(46.95634922356221 -121.45862470879337) bank109236 +109237 POINT(47.30201768818691 -122.91194444998995) bank109237 +109238 POINT(47.84825007292544 -122.2224138513447) bank109238 +109239 POINT(46.85413182728556 -122.4224092646246) bank109239 +109240 POINT(46.80397083185995 -123.19941702957004) bank109240 +109241 POINT(48.22423324041792 -122.81861641842788) bank109241 +109242 POINT(47.47902430641521 -122.06931660983157) bank109242 +109243 POINT(48.14602854108821 -122.67670597964903) bank109243 +109244 POINT(48.232938708904065 -122.82444617126373) bank109244 +109245 POINT(48.487246864176846 -121.95077518744012) bank109245 +109246 POINT(48.2336946680468 -121.77269837188331) bank109246 +109247 POINT(46.77099880376099 -122.06314761651407) bank109247 +109248 POINT(47.667787450737364 -122.6014182891462) bank109248 +109249 POINT(47.64453384741867 -122.70872767179617) bank109249 +109250 POINT(46.63408806784449 -122.84594857165504) bank109250 +109251 POINT(47.95323994686322 -121.39806114811512) bank109251 +109252 POINT(46.900477630537125 -121.86764688158377) bank109252 +109253 POINT(48.563921613350075 -122.77071897974896) bank109253 +109254 POINT(47.59093301184214 -121.47105493810682) bank109254 +109255 POINT(47.145338562843925 -122.31895545024406) bank109255 +109256 POINT(48.16116657894338 -121.7591827540323) bank109256 +109257 POINT(48.3513414978367 -122.04298520874067) bank109257 +109258 POINT(47.7869423213236 -121.72884067958488) bank109258 +109259 POINT(47.28213713633973 -123.24947273973042) bank109259 +109260 POINT(47.30125761174809 -123.1236925839338) bank109260 +109261 POINT(46.918652344933236 -121.54166533608692) bank109261 +109262 POINT(48.490430107316264 -122.18177095727712) bank109262 +109263 POINT(46.69006447982881 -122.8872209197059) bank109263 +109264 POINT(47.292964300258944 -122.2900959943113) bank109264 +109265 POINT(47.51104793168087 -123.06647806498637) bank109265 +109266 POINT(47.452583424515176 -123.06929475288494) bank109266 +109267 POINT(46.991906798390076 -121.54590365965983) bank109267 +109268 POINT(48.03736981269499 -122.67763594780267) bank109268 +109269 POINT(47.829784923866704 -122.79147835194671) bank109269 +109270 POINT(48.03778580740798 -121.90124826648) bank109270 +109271 POINT(46.75410595605688 -121.76357799700361) bank109271 +109272 POINT(47.44435084211894 -122.01491228186924) bank109272 +109273 POINT(47.65659639272913 -123.11253263226821) bank109273 +109274 POINT(48.27036673809289 -122.2814313780698) bank109274 +109275 POINT(47.221517786436436 -122.84652513316927) bank109275 +109276 POINT(47.50742081439247 -122.84017255028061) bank109276 +109277 POINT(46.724249737855615 -121.84029082822914) bank109277 +109278 POINT(47.203497215689 -121.36688322671671) bank109278 +109279 POINT(46.671680038259275 -122.43561926919216) bank109279 +109280 POINT(48.086517762116316 -122.93153353100894) bank109280 +109281 POINT(47.91163820962708 -121.6401664445844) bank109281 +109282 POINT(47.87862825844002 -122.26231663614132) bank109282 +109283 POINT(47.618764836057196 -122.01159394467315) bank109283 +109284 POINT(47.85124823477703 -121.49974487973687) bank109284 +109285 POINT(48.51543478344375 -123.17623528796162) bank109285 +109286 POINT(46.8383498361413 -122.4102287730763) bank109286 +109287 POINT(47.09873659988673 -121.69704539012075) bank109287 +109288 POINT(47.85888139927552 -123.03221202996662) bank109288 +109289 POINT(47.44913154004613 -123.26959198949882) bank109289 +109290 POINT(47.8769431739316 -122.46342349727006) bank109290 +109291 POINT(47.93484806729187 -122.9632520540755) bank109291 +109292 POINT(48.33083674811784 -122.97431191739133) bank109292 +109293 POINT(47.812750624746855 -121.62123669866318) bank109293 +109294 POINT(48.1636552936162 -121.80513761198375) bank109294 +109295 POINT(48.14529940769733 -122.89211755733935) bank109295 +109296 POINT(46.808715088437275 -122.58434027898106) bank109296 +109297 POINT(47.73738730662161 -121.5815601913175) bank109297 +109298 POINT(47.14221990099869 -121.80204895245058) bank109298 +109299 POINT(47.800089453250465 -123.19098773252986) bank109299 +109300 POINT(47.468209933347694 -122.2139447953641) bank109300 +109301 POINT(47.82145500366152 -122.7532475112903) bank109301 +109302 POINT(48.03908239828753 -122.82019220025589) bank109302 +109303 POINT(47.80328885967068 -123.27727624451184) bank109303 +109304 POINT(48.49031280324285 -122.09124535856603) bank109304 +109305 POINT(46.702916321589555 -121.9631525405345) bank109305 +109306 POINT(48.0649934866998 -122.64828235075187) bank109306 +109307 POINT(47.05681787635641 -123.019408506713) bank109307 +109308 POINT(46.999127739808785 -121.60335746479167) bank109308 +109309 POINT(47.619919324548896 -121.57263493419437) bank109309 +109310 POINT(47.8113498668603 -122.781977681417) bank109310 +109311 POINT(48.43474918522253 -121.60324219643437) bank109311 +109312 POINT(47.12710648425964 -121.60485878012982) bank109312 +109313 POINT(46.6737663941679 -122.83552269044345) bank109313 +109314 POINT(46.69016425935586 -123.16015627556497) bank109314 +109315 POINT(47.16220085380207 -123.1824740012966) bank109315 +109316 POINT(48.45144159589866 -121.43142617601733) bank109316 +109317 POINT(48.24975041994548 -122.91648416356674) bank109317 +109318 POINT(47.816335035640115 -123.19824715753268) bank109318 +109319 POINT(46.64038334563461 -123.11624953027341) bank109319 +109320 POINT(47.20752445809606 -122.5826482504058) bank109320 +109321 POINT(46.73621784345014 -122.6395266463229) bank109321 +109322 POINT(48.17369640924926 -121.74573754291268) bank109322 +109323 POINT(47.421046083392426 -121.58688522110805) bank109323 +109324 POINT(47.537520470550255 -122.37542176113818) bank109324 +109325 POINT(48.45490840946008 -121.93908085642933) bank109325 +109326 POINT(46.80327514954936 -121.57434271938338) bank109326 +109327 POINT(46.86187006508021 -121.60148171774424) bank109327 +109328 POINT(47.31923221188131 -122.33499944403772) bank109328 +109329 POINT(46.61371113948754 -121.4794130074957) bank109329 +109330 POINT(46.88021077661431 -121.37468809998647) bank109330 +109331 POINT(46.607188079440064 -122.51951362431296) bank109331 +109332 POINT(46.637785628494676 -122.87243602387892) bank109332 +109333 POINT(48.22092510749929 -122.34654273286047) bank109333 +109334 POINT(47.44382737652074 -122.7072645262434) bank109334 +109335 POINT(47.84863380435054 -122.17569167246064) bank109335 +109336 POINT(47.180955818123756 -123.11676120166737) bank109336 +109337 POINT(48.426675195095335 -122.69531458668419) bank109337 +109338 POINT(47.26893538310895 -121.82287187368094) bank109338 +109339 POINT(47.867938694293485 -122.13714227183115) bank109339 +109340 POINT(47.546050634599226 -123.21771963999906) bank109340 +109341 POINT(47.419103284090134 -123.3143057938528) bank109341 +109342 POINT(48.06575333831846 -122.72597863180987) bank109342 +109343 POINT(47.025903586651616 -123.32889861906393) bank109343 +109344 POINT(48.538477370720194 -122.02350160550967) bank109344 +109345 POINT(47.131986359643676 -121.89142107817388) bank109345 +109346 POINT(47.98745807269499 -121.4788062613876) bank109346 +109347 POINT(47.870352473198444 -121.74473182539386) bank109347 +109348 POINT(47.49387626046232 -122.9685672336438) bank109348 +109349 POINT(47.96820512434271 -121.3651723990625) bank109349 +109350 POINT(47.719706568704076 -122.49037045861152) bank109350 +109351 POINT(48.5696750311621 -122.99842106458645) bank109351 +109352 POINT(46.977263372522565 -121.97333045195344) bank109352 +109353 POINT(47.40098291907824 -121.88374950879049) bank109353 +109354 POINT(46.78645831175138 -123.17271433995286) bank109354 +109355 POINT(46.950317574572175 -122.36229149458727) bank109355 +109356 POINT(47.95004845318279 -122.60132822432666) bank109356 +109357 POINT(48.061403632717344 -122.17588428975418) bank109357 +109358 POINT(48.0359094584905 -122.49256250805509) bank109358 +109359 POINT(48.38341410096942 -123.21219646491801) bank109359 +109360 POINT(47.622843801399895 -122.20658802280303) bank109360 +109361 POINT(47.046757870445084 -121.78430333254539) bank109361 +109362 POINT(47.45390400882802 -121.4944902734766) bank109362 +109363 POINT(47.305378585199364 -122.12331531334586) bank109363 +109364 POINT(48.38954075405293 -122.60446702698799) bank109364 +109365 POINT(47.23352792198272 -122.07984622478234) bank109365 +109366 POINT(46.62412468683153 -122.10468272482925) bank109366 +109367 POINT(47.60593283136547 -123.29717516552319) bank109367 +109368 POINT(47.366762351448145 -121.68412947531839) bank109368 +109369 POINT(48.599826481831045 -122.19574275263369) bank109369 +109370 POINT(47.3841675201357 -122.0373103038758) bank109370 +109371 POINT(48.129827085548015 -121.8218167935718) bank109371 +109372 POINT(47.58110872743648 -122.72772805033337) bank109372 +109373 POINT(46.83300807812527 -122.85649801453685) bank109373 +109374 POINT(47.45458314075985 -122.58949563213314) bank109374 +109375 POINT(48.23767756068278 -121.53717756139537) bank109375 +109376 POINT(47.97561503487568 -123.1344375522714) bank109376 +109377 POINT(46.73729052367129 -121.50919438574233) bank109377 +109378 POINT(46.90674927092882 -122.81664803463279) bank109378 +109379 POINT(46.65175602511929 -122.8162711483784) bank109379 +109380 POINT(47.8694742409674 -122.24724442853541) bank109380 +109381 POINT(48.415455859464984 -123.18882897275617) bank109381 +109382 POINT(47.044035883135784 -122.06798506254917) bank109382 +109383 POINT(47.722982804364534 -123.31829797765334) bank109383 +109384 POINT(47.16176723013041 -121.42511300898005) bank109384 +109385 POINT(46.960215066714376 -121.6638382874595) bank109385 +109386 POINT(47.934311371240554 -123.2183614414953) bank109386 +109387 POINT(48.22848103100049 -121.96177274259513) bank109387 +109388 POINT(48.138308365912145 -121.47018136811853) bank109388 +109389 POINT(47.49252522983235 -121.4513366131788) bank109389 +109390 POINT(47.92659802089694 -122.6740066106853) bank109390 +109391 POINT(47.49648709544982 -123.14409115670946) bank109391 +109392 POINT(46.66035522676732 -122.01470691557687) bank109392 +109393 POINT(48.08181176918687 -122.47826424289057) bank109393 +109394 POINT(47.45369642598841 -121.92419412770775) bank109394 +109395 POINT(48.31730857106614 -121.64673913330537) bank109395 +109396 POINT(47.18875072698449 -122.85329601058477) bank109396 +109397 POINT(48.1929385673484 -123.27589922308553) bank109397 +109398 POINT(48.139290284652574 -121.75899485466427) bank109398 +109399 POINT(46.62424714287799 -122.81303186545884) bank109399 +109400 POINT(48.42586312942757 -122.6732290679449) bank109400 +109401 POINT(47.80707333033424 -122.03271464068548) bank109401 +109402 POINT(47.309720150382404 -122.80946750109892) bank109402 +109403 POINT(47.03481687822414 -121.36873713672861) bank109403 +109404 POINT(48.54268060251421 -121.69133922843166) bank109404 +109405 POINT(46.815865806172276 -121.81552379001857) bank109405 +109406 POINT(46.642515983091464 -121.55098962319255) bank109406 +109407 POINT(47.97181379809051 -122.28235653998557) bank109407 +109408 POINT(48.33307697555628 -122.04888509662433) bank109408 +109409 POINT(46.70109648616986 -122.4841444603683) bank109409 +109410 POINT(47.161188200497556 -122.7853028507463) bank109410 +109411 POINT(47.03114833716868 -122.20757220776304) bank109411 +109412 POINT(47.6120974121543 -121.54369853700163) bank109412 +109413 POINT(47.17799416658333 -123.12139158979346) bank109413 +109414 POINT(48.501283858383715 -123.1156134734587) bank109414 +109415 POINT(47.9393250218799 -123.32024278580077) bank109415 +109416 POINT(48.41533005508442 -122.7492074009989) bank109416 +109417 POINT(47.79498101867612 -122.79606660281615) bank109417 +109418 POINT(47.104338065072156 -123.26715101648286) bank109418 +109419 POINT(47.195276663727185 -122.39850202067954) bank109419 +109420 POINT(46.637020644258406 -121.80576054740284) bank109420 +109421 POINT(46.892000930194406 -122.0090939865176) bank109421 +109422 POINT(47.98898862817721 -121.57367071428892) bank109422 +109423 POINT(47.63375575453663 -121.93673981300903) bank109423 +109424 POINT(47.488199048805676 -121.48165916394849) bank109424 +109425 POINT(47.167534859602505 -123.28626162738924) bank109425 +109426 POINT(46.975376298328364 -122.97334935794386) bank109426 +109427 POINT(46.83229464520523 -122.17129253043072) bank109427 +109428 POINT(47.54052413115801 -123.06355644381533) bank109428 +109429 POINT(47.374008677183426 -122.0096067001854) bank109429 +109430 POINT(47.96918290792533 -121.44745353463465) bank109430 +109431 POINT(48.587423579901795 -123.07166560478225) bank109431 +109432 POINT(46.9812456148821 -121.95639368694766) bank109432 +109433 POINT(47.2532282281178 -122.01467230197555) bank109433 +109434 POINT(48.031511808501065 -122.72042373256683) bank109434 +109435 POINT(47.11614862091763 -121.35982958163758) bank109435 +109436 POINT(47.65803537451216 -122.52529900365545) bank109436 +109437 POINT(47.92886063483316 -122.47801899721753) bank109437 +109438 POINT(47.26891385803763 -122.67750618742372) bank109438 +109439 POINT(47.56990964985632 -121.75293851095336) bank109439 +109440 POINT(47.941866100749486 -121.72694717086824) bank109440 +109441 POINT(47.6652858175708 -122.00158284490055) bank109441 +109442 POINT(47.716278234676864 -122.64355115375062) bank109442 +109443 POINT(48.49021766614689 -121.53079417646724) bank109443 +109444 POINT(47.23688222982866 -122.7319271711856) bank109444 +109445 POINT(46.90385447770552 -122.4192532223425) bank109445 +109446 POINT(46.68024836966133 -122.92937390223545) bank109446 +109447 POINT(47.75266744402052 -121.96273753701394) bank109447 +109448 POINT(48.38118504762379 -122.6417952276113) bank109448 +109449 POINT(46.72656294099739 -122.14829057723179) bank109449 +109450 POINT(48.509809064527396 -122.93991124716855) bank109450 +109451 POINT(48.02790796038929 -122.33632946737494) bank109451 +109452 POINT(47.974046204063136 -122.95114203098983) bank109452 +109453 POINT(46.94561176967062 -122.32898829071458) bank109453 +109454 POINT(48.29017224924612 -121.37021816758302) bank109454 +109455 POINT(46.82524230269318 -123.32243932265193) bank109455 +109456 POINT(48.47381758558514 -123.27367710038924) bank109456 +109457 POINT(46.64630134373147 -121.39700815786495) bank109457 +109458 POINT(47.958437023191536 -121.51075456302644) bank109458 +109459 POINT(48.60450800329552 -121.91008955992578) bank109459 +109460 POINT(48.302909354571504 -122.14097057069363) bank109460 +109461 POINT(47.812174522364714 -123.25285611612117) bank109461 +109462 POINT(47.785414205964145 -123.25552479396438) bank109462 +109463 POINT(46.91992200090521 -121.5119902535668) bank109463 +109464 POINT(47.093493307824666 -122.65242198739566) bank109464 +109465 POINT(47.89917994882958 -122.1110673821756) bank109465 +109466 POINT(47.5183949527044 -122.94621481718734) bank109466 +109467 POINT(47.393750413968505 -121.60468272030595) bank109467 +109468 POINT(46.70905635323348 -122.32246630252014) bank109468 +109469 POINT(47.87125995369898 -121.97352418860306) bank109469 +109470 POINT(46.698502473285615 -122.04418986566112) bank109470 +109471 POINT(48.323067062079446 -122.44245550204559) bank109471 +109472 POINT(47.25361978247129 -122.81363870298516) bank109472 +109473 POINT(48.30551456168114 -121.34333733610214) bank109473 +109474 POINT(47.906840439326814 -122.83025754447952) bank109474 +109475 POINT(47.70949029893416 -121.78569133483424) bank109475 +109476 POINT(47.000524717256425 -123.22858730341217) bank109476 +109477 POINT(46.689286868055234 -121.74608446048944) bank109477 +109478 POINT(48.302666205771324 -121.65449123077813) bank109478 +109479 POINT(46.824487126085444 -122.34854583839207) bank109479 +109480 POINT(46.81965907459478 -121.53026441777861) bank109480 +109481 POINT(47.24924714755492 -122.80657074328583) bank109481 +109482 POINT(48.13983003627004 -122.21674595479254) bank109482 +109483 POINT(47.67403850421848 -122.8057786147017) bank109483 +109484 POINT(48.60357094671425 -121.48809895405698) bank109484 +109485 POINT(47.75115120227054 -121.42830385393013) bank109485 +109486 POINT(48.33613860687464 -121.69128263912985) bank109486 +109487 POINT(48.110213266864456 -122.22772021442066) bank109487 +109488 POINT(48.47640856969389 -121.94143885690772) bank109488 +109489 POINT(47.49252559363998 -121.7988047505156) bank109489 +109490 POINT(48.51708974663259 -121.94277818769679) bank109490 +109491 POINT(46.91896847998666 -122.04561191374123) bank109491 +109492 POINT(47.46421633422222 -122.23565644703757) bank109492 +109493 POINT(47.64987130346438 -121.54196708784214) bank109493 +109494 POINT(46.79814905386162 -122.66389080709324) bank109494 +109495 POINT(47.663714565795324 -122.262527006948) bank109495 +109496 POINT(47.86959771252706 -122.24657295305072) bank109496 +109497 POINT(47.83986781539741 -121.74296055375633) bank109497 +109498 POINT(47.3837927310855 -121.90527650253708) bank109498 +109499 POINT(46.9480809890692 -122.42887151459612) bank109499 +109500 POINT(47.6334479901036 -122.87105221022259) bank109500 +109501 POINT(48.42946369498773 -122.30250833606634) bank109501 +109502 POINT(47.0312141533873 -123.07293773483066) bank109502 +109503 POINT(48.46063975078245 -121.5270126871207) bank109503 +109504 POINT(48.52253553024495 -121.60902486056875) bank109504 +109505 POINT(46.680321130846096 -122.45245805489931) bank109505 +109506 POINT(47.88841479484946 -122.89179968488078) bank109506 +109507 POINT(48.60290431152335 -122.48827640216788) bank109507 +109508 POINT(47.93361250540714 -122.64367903228982) bank109508 +109509 POINT(47.34064330947856 -122.07543755107406) bank109509 +109510 POINT(47.04772026704017 -122.96484939651944) bank109510 +109511 POINT(46.710750380209156 -122.4153467996715) bank109511 +109512 POINT(48.02769436627747 -122.33180869568666) bank109512 +109513 POINT(47.68757795476073 -122.1155460863446) bank109513 +109514 POINT(48.3761377618378 -122.78196880540139) bank109514 +109515 POINT(47.646163593324616 -121.83969292717364) bank109515 +109516 POINT(47.14506822972437 -121.65630519791284) bank109516 +109517 POINT(48.45351573593691 -121.39276891716703) bank109517 +109518 POINT(47.77849945668412 -123.22980968922484) bank109518 +109519 POINT(48.45570166789696 -121.68548519605258) bank109519 +109520 POINT(46.642126588913925 -122.9489296995348) bank109520 +109521 POINT(47.40240114285933 -121.55787365747781) bank109521 +109522 POINT(48.06354876002952 -122.22526941099851) bank109522 +109523 POINT(47.03377295385949 -123.12014347498548) bank109523 +109524 POINT(46.95397213086465 -122.49359498787032) bank109524 +109525 POINT(47.76517533279119 -122.3721539373867) bank109525 +109526 POINT(46.72829418482206 -123.3046601334102) bank109526 +109527 POINT(48.538163752169226 -121.48436291690297) bank109527 +109528 POINT(46.894427743975804 -122.60903313795) bank109528 +109529 POINT(48.16225176880937 -122.39742296274899) bank109529 +109530 POINT(47.16174668583306 -122.96731530162046) bank109530 +109531 POINT(48.45218985199154 -121.85983544713588) bank109531 +109532 POINT(47.12660705782674 -121.53140305281566) bank109532 +109533 POINT(46.898233211859065 -123.04412002313337) bank109533 +109534 POINT(47.867803855742906 -121.8823706769029) bank109534 +109535 POINT(48.53875850030289 -122.92167310573456) bank109535 +109536 POINT(46.98664021964297 -122.95417636843818) bank109536 +109537 POINT(47.00140479468754 -121.70818626460442) bank109537 +109538 POINT(46.68186099857696 -121.70983519385143) bank109538 +109539 POINT(48.37009427064025 -121.34307962719413) bank109539 +109540 POINT(47.60196831118951 -121.96541348740595) bank109540 +109541 POINT(47.806868460650385 -121.55562123859114) bank109541 +109542 POINT(48.308367799747344 -123.10507711185795) bank109542 +109543 POINT(46.786910591718154 -123.18113283360361) bank109543 +109544 POINT(48.178319604255435 -122.99641881954902) bank109544 +109545 POINT(47.538311545509686 -122.96726336503329) bank109545 +109546 POINT(46.967977328233026 -122.58944594659276) bank109546 +109547 POINT(46.73183295692257 -123.19698946417513) bank109547 +109548 POINT(47.75379130883508 -123.30361612576986) bank109548 +109549 POINT(48.208804810962164 -122.43038545865467) bank109549 +109550 POINT(46.662109435508675 -121.97436557689683) bank109550 +109551 POINT(47.71171750491941 -122.99390450439506) bank109551 +109552 POINT(48.04022089080796 -122.45751463799999) bank109552 +109553 POINT(48.37928187300009 -122.39039493199552) bank109553 +109554 POINT(47.287481618275805 -122.08492709279544) bank109554 +109555 POINT(47.33767399126125 -122.88598470514084) bank109555 +109556 POINT(48.47939274923125 -122.61319514806087) bank109556 +109557 POINT(48.28706217126115 -121.93206208076609) bank109557 +109558 POINT(47.06125936427794 -122.02770377962581) bank109558 +109559 POINT(47.80605204329286 -121.83805495646018) bank109559 +109560 POINT(47.269430436671456 -121.88074109828386) bank109560 +109561 POINT(47.785472811434815 -122.75971396587472) bank109561 +109562 POINT(48.2386652989368 -121.65419610729613) bank109562 +109563 POINT(48.194371944437705 -123.23709115155437) bank109563 +109564 POINT(47.19946631966388 -122.46339276634832) bank109564 +109565 POINT(46.82742116078724 -122.00952445499986) bank109565 +109566 POINT(47.966809497412164 -121.49521815989382) bank109566 +109567 POINT(46.818639722267115 -122.16396387654538) bank109567 +109568 POINT(47.2985157990133 -122.97811301812999) bank109568 +109569 POINT(47.34821714845828 -121.63558583833868) bank109569 +109570 POINT(47.93445173899501 -122.78790721823623) bank109570 +109571 POINT(48.29581073427627 -121.65331288241363) bank109571 +109572 POINT(47.957025368549154 -121.69907847927777) bank109572 +109573 POINT(47.80570159461554 -122.1118435731341) bank109573 +109574 POINT(47.42050667362004 -122.6069136823713) bank109574 +109575 POINT(47.721242860307036 -121.37099910797595) bank109575 +109576 POINT(48.36757628952591 -122.67077300754619) bank109576 +109577 POINT(47.470610542364184 -122.01134158059764) bank109577 +109578 POINT(47.068931899485214 -122.0226838917519) bank109578 +109579 POINT(48.21553830664332 -122.32635407476974) bank109579 +109580 POINT(46.73229091770591 -122.82500620965477) bank109580 +109581 POINT(46.76276234424306 -121.53715337016907) bank109581 +109582 POINT(46.73163617588818 -121.46307768398042) bank109582 +109583 POINT(47.079982106182854 -122.00688106989047) bank109583 +109584 POINT(48.54854845599856 -122.6954960661174) bank109584 +109585 POINT(48.075355158606925 -121.92818260771286) bank109585 +109586 POINT(48.56822967291142 -122.2700390203788) bank109586 +109587 POINT(47.41011318739209 -123.03505363251578) bank109587 +109588 POINT(46.81704927769094 -121.35867873537134) bank109588 +109589 POINT(46.80514332357688 -122.79516948859826) bank109589 +109590 POINT(47.31366332721487 -121.66519805574264) bank109590 +109591 POINT(48.28826719723393 -123.28280185318951) bank109591 +109592 POINT(48.323867585970866 -123.15148295603284) bank109592 +109593 POINT(46.92599471696579 -122.18478714122242) bank109593 +109594 POINT(47.157967703295874 -121.62440228671167) bank109594 +109595 POINT(47.19121441806163 -123.14087117191005) bank109595 +109596 POINT(47.18996766599784 -122.02528901072932) bank109596 +109597 POINT(46.74831773939803 -122.13642768552492) bank109597 +109598 POINT(47.218998231330644 -123.06659979441521) bank109598 +109599 POINT(47.971198827570895 -121.87421062142786) bank109599 +109600 POINT(48.25338564147763 -121.6614725806949) bank109600 +109601 POINT(47.36732914691575 -122.98960291793308) bank109601 +109602 POINT(47.01822314156728 -123.00730707996766) bank109602 +109603 POINT(46.91179257250564 -121.38569893323101) bank109603 +109604 POINT(47.605829192582554 -122.73012363454613) bank109604 +109605 POINT(47.75285478244089 -123.09888254124415) bank109605 +109606 POINT(46.80327072943757 -122.84964168140922) bank109606 +109607 POINT(48.09844139842844 -121.36866151310161) bank109607 +109608 POINT(47.20148429519224 -121.39357620883871) bank109608 +109609 POINT(47.796491381821625 -121.34638155226716) bank109609 +109610 POINT(48.30808077142706 -122.40861900855893) bank109610 +109611 POINT(46.83508732140875 -122.04112184221017) bank109611 +109612 POINT(48.337471391232896 -121.65358269643642) bank109612 +109613 POINT(47.45619154565116 -122.88316783148066) bank109613 +109614 POINT(47.05159662744194 -121.53060491814433) bank109614 +109615 POINT(47.778461433432845 -122.78235147230677) bank109615 +109616 POINT(48.23850921621271 -123.02023446482214) bank109616 +109617 POINT(46.81946018681591 -122.6050505841361) bank109617 +109618 POINT(47.85797621918349 -123.24682679587502) bank109618 +109619 POINT(47.092192818343136 -122.03777307011477) bank109619 +109620 POINT(47.62781798208849 -121.68164447773363) bank109620 +109621 POINT(47.849137091011 -121.42882743857737) bank109621 +109622 POINT(47.66314684248986 -121.52317297034102) bank109622 +109623 POINT(46.69820215600608 -121.75199653325532) bank109623 +109624 POINT(47.41993020338018 -122.14323869677132) bank109624 +109625 POINT(47.581635389553604 -123.02774151363425) bank109625 +109626 POINT(47.19797354229123 -121.65535687914493) bank109626 +109627 POINT(47.616382173284244 -121.6287480127614) bank109627 +109628 POINT(48.08880084938007 -122.52066458703409) bank109628 +109629 POINT(47.262775205927 -121.63833847779162) bank109629 +109630 POINT(48.546322874600044 -122.27330408303925) bank109630 +109631 POINT(48.31184162754475 -121.37649157329096) bank109631 +109632 POINT(48.505872960567274 -122.00238814800852) bank109632 +109633 POINT(47.274373578455915 -121.58039374357143) bank109633 +109634 POINT(48.20676282834899 -122.38464763828998) bank109634 +109635 POINT(48.04160985041003 -123.25992715467086) bank109635 +109636 POINT(48.29939410267789 -122.69124778139019) bank109636 +109637 POINT(47.76664691857706 -122.0343453278423) bank109637 +109638 POINT(46.86108041615381 -122.13220406937113) bank109638 +109639 POINT(47.08155383122216 -121.44654538307567) bank109639 +109640 POINT(48.16466330496166 -121.7581920083516) bank109640 +109641 POINT(48.591132028517656 -121.82275462414533) bank109641 +109642 POINT(46.97788054481751 -122.45816446303093) bank109642 +109643 POINT(48.23491241570035 -122.2627517165702) bank109643 +109644 POINT(47.60357927754911 -122.67535369053277) bank109644 +109645 POINT(47.4500558835944 -121.69299134111873) bank109645 +109646 POINT(48.341644309296846 -122.2925899675183) bank109646 +109647 POINT(47.23593373288414 -121.59336593482155) bank109647 +109648 POINT(48.248228081632256 -122.08510232831323) bank109648 +109649 POINT(46.85736887922123 -122.92012307602114) bank109649 +109650 POINT(47.437708067831366 -122.71106105559338) bank109650 +109651 POINT(47.744894078030136 -123.00909896722146) bank109651 +109652 POINT(47.569502242986765 -121.43272803584047) bank109652 +109653 POINT(47.67735739406588 -122.33845000910662) bank109653 +109654 POINT(47.75245905930775 -122.21631912801897) bank109654 +109655 POINT(48.523232970700796 -122.47700876479314) bank109655 +109656 POINT(48.23822658518966 -121.9965731619586) bank109656 +109657 POINT(47.153163490622326 -122.22132887761873) bank109657 +109658 POINT(47.3415818875231 -121.74701987337342) bank109658 +109659 POINT(46.947952598984976 -122.32232498689903) bank109659 +109660 POINT(47.39434063311126 -122.38028262063806) bank109660 +109661 POINT(48.42572580956859 -121.77928651443429) bank109661 +109662 POINT(47.33123385450589 -122.46719286432706) bank109662 +109663 POINT(48.526996627287645 -121.91419481561402) bank109663 +109664 POINT(47.891781907890596 -121.91018444257475) bank109664 +109665 POINT(47.98616484388634 -121.527174927501) bank109665 +109666 POINT(48.144491193812655 -122.14534555014143) bank109666 +109667 POINT(48.45745115285443 -123.16993995986803) bank109667 +109668 POINT(46.84670328332934 -122.5426331895517) bank109668 +109669 POINT(46.89083547763231 -121.9670501995028) bank109669 +109670 POINT(48.12489525632165 -122.60921067365913) bank109670 +109671 POINT(46.9934415385858 -123.00605098111473) bank109671 +109672 POINT(48.300965515946935 -121.70994258300783) bank109672 +109673 POINT(46.70580773684135 -121.48123185865305) bank109673 +109674 POINT(47.365215382045434 -121.96451452109734) bank109674 +109675 POINT(47.410624643104555 -121.68348272741848) bank109675 +109676 POINT(48.45343694759492 -122.18505992335199) bank109676 +109677 POINT(47.799330332167216 -121.36751988399567) bank109677 +109678 POINT(48.35622893162718 -123.2665415795694) bank109678 +109679 POINT(48.37057016846634 -122.19006525058725) bank109679 +109680 POINT(47.986464867064 -122.23504614838973) bank109680 +109681 POINT(46.9842444564648 -122.51832403751875) bank109681 +109682 POINT(47.055093547471785 -122.60880580180805) bank109682 +109683 POINT(47.2669176179779 -121.56141484284363) bank109683 +109684 POINT(48.129925967613126 -122.69967308111646) bank109684 +109685 POINT(47.20791168533165 -122.88113226835343) bank109685 +109686 POINT(47.30810464963261 -122.50172740635028) bank109686 +109687 POINT(47.07330096770601 -121.62671091345025) bank109687 +109688 POINT(48.10663650792065 -121.6522968752329) bank109688 +109689 POINT(47.96519962983818 -122.59397373551202) bank109689 +109690 POINT(48.11915174403284 -122.15061167402189) bank109690 +109691 POINT(47.115532871339404 -121.74473472287373) bank109691 +109692 POINT(47.37961981889785 -121.66748070930657) bank109692 +109693 POINT(47.96387766172165 -122.88862688855437) bank109693 +109694 POINT(47.53358629007525 -123.200767447831) bank109694 +109695 POINT(48.603065934371664 -122.51498588190135) bank109695 +109696 POINT(48.32204639507246 -122.96594307309545) bank109696 +109697 POINT(47.221651037413025 -122.83124541508202) bank109697 +109698 POINT(48.43437711842427 -121.71369389681489) bank109698 +109699 POINT(48.307322597046074 -122.89251750159482) bank109699 +109700 POINT(48.17342295863049 -121.76982804750804) bank109700 +109701 POINT(47.47942151100015 -123.20457067581113) bank109701 +109702 POINT(47.74004557284032 -121.53820161754442) bank109702 +109703 POINT(47.44433623585963 -122.52839581596118) bank109703 +109704 POINT(48.49516375417077 -122.76428319244148) bank109704 +109705 POINT(47.98294059775586 -122.83272217307209) bank109705 +109706 POINT(47.00290318523178 -123.14591333516188) bank109706 +109707 POINT(46.84881170038276 -122.26896856683885) bank109707 +109708 POINT(48.142097789021015 -121.41440928764065) bank109708 +109709 POINT(47.51063531512833 -121.35881630129046) bank109709 +109710 POINT(47.437904195396904 -121.72946880941252) bank109710 +109711 POINT(48.01891952199281 -121.44767858059105) bank109711 +109712 POINT(47.54872965944845 -122.60978257767022) bank109712 +109713 POINT(47.11957379345385 -123.24198508596928) bank109713 +109714 POINT(47.94051006202884 -121.40463430868961) bank109714 +109715 POINT(46.613774096841155 -121.43853121215061) bank109715 +109716 POINT(48.41805900229422 -122.01470844740865) bank109716 +109717 POINT(47.678732413460246 -121.90606856641644) bank109717 +109718 POINT(48.243954481570256 -121.9163261047537) bank109718 +109719 POINT(48.1011355907654 -122.86433313749538) bank109719 +109720 POINT(46.81089663067764 -122.42886015058637) bank109720 +109721 POINT(48.13340245724942 -122.96405417148006) bank109721 +109722 POINT(47.69810477699574 -122.87183400714213) bank109722 +109723 POINT(48.557635176344974 -121.67057090725945) bank109723 +109724 POINT(48.435294769881814 -122.51446261512024) bank109724 +109725 POINT(48.52369089772958 -123.0314749178319) bank109725 +109726 POINT(47.95395840661948 -122.79679163533761) bank109726 +109727 POINT(48.56706479908318 -122.22687336058613) bank109727 +109728 POINT(46.74950528333748 -121.84893525232793) bank109728 +109729 POINT(46.89886382550643 -123.3010851981536) bank109729 +109730 POINT(48.423098277117326 -123.0155572625286) bank109730 +109731 POINT(47.38478386144098 -122.77278364481829) bank109731 +109732 POINT(47.029651657085786 -122.01738068881363) bank109732 +109733 POINT(48.03676715549655 -122.15767109953717) bank109733 +109734 POINT(48.02112883045965 -121.40603966465268) bank109734 +109735 POINT(46.87209462703491 -121.7823167992647) bank109735 +109736 POINT(48.56913536358902 -123.2311695188413) bank109736 +109737 POINT(46.63832444359294 -122.88004982136628) bank109737 +109738 POINT(48.339216868878566 -122.44499590355417) bank109738 +109739 POINT(48.207819618051985 -121.55034128636066) bank109739 +109740 POINT(47.110367921910665 -122.22559354870918) bank109740 +109741 POINT(48.31213113768393 -122.74328788050366) bank109741 +109742 POINT(47.8402302284302 -121.81751394521905) bank109742 +109743 POINT(47.764503547484935 -122.3078767179865) bank109743 +109744 POINT(46.85680831966984 -121.65299899812727) bank109744 +109745 POINT(46.751413554378594 -121.55284549201815) bank109745 +109746 POINT(47.676342155817345 -122.17936998734591) bank109746 +109747 POINT(47.47219025184525 -122.5263000788778) bank109747 +109748 POINT(48.393582936669524 -122.35769312494355) bank109748 +109749 POINT(48.232097014230106 -122.29771492560093) bank109749 +109750 POINT(48.44952802203276 -121.89128422977983) bank109750 +109751 POINT(47.39398210361524 -121.7609170141617) bank109751 +109752 POINT(47.825455045009704 -121.96906885282287) bank109752 +109753 POINT(47.25015076045073 -123.24869246341046) bank109753 +109754 POINT(47.645349950550774 -122.07279110565281) bank109754 +109755 POINT(47.43236775270023 -122.23946980059974) bank109755 +109756 POINT(47.60718237319839 -121.83732544865943) bank109756 +109757 POINT(46.654473537068 -122.84146719699632) bank109757 +109758 POINT(48.12092080386663 -123.25165802969929) bank109758 +109759 POINT(48.157745979179566 -122.78983059904098) bank109759 +109760 POINT(47.16942445992872 -123.12102373478494) bank109760 +109761 POINT(48.29799698120529 -122.75669536465675) bank109761 +109762 POINT(46.89967961562776 -122.08882814191496) bank109762 +109763 POINT(48.52426382969698 -121.68457023235375) bank109763 +109764 POINT(48.4797142391478 -121.93588058191524) bank109764 +109765 POINT(47.13495176426826 -122.17816739632363) bank109765 +109766 POINT(48.5154859845763 -122.96302113796303) bank109766 +109767 POINT(48.476596395705805 -123.19114771767083) bank109767 +109768 POINT(46.98647539038342 -121.37518763169359) bank109768 +109769 POINT(46.78873974416168 -121.65061214850056) bank109769 +109770 POINT(48.03738983288447 -122.82739915832684) bank109770 +109771 POINT(47.108291912792474 -121.36680929997611) bank109771 +109772 POINT(47.764053861156846 -121.72967073492067) bank109772 +109773 POINT(48.33813827461905 -122.0368965804649) bank109773 +109774 POINT(48.510887979555875 -121.6922416593591) bank109774 +109775 POINT(46.79243737825519 -122.52917169682195) bank109775 +109776 POINT(48.00588364422565 -122.59486176965723) bank109776 +109777 POINT(47.05287489319705 -121.35841919527901) bank109777 +109778 POINT(48.56522909645149 -122.06398098204805) bank109778 +109779 POINT(47.16402887094496 -122.92163620604694) bank109779 +109780 POINT(47.26873968674795 -122.53371641434342) bank109780 +109781 POINT(48.059690323292806 -123.24304331882094) bank109781 +109782 POINT(46.676876581300085 -123.1717862146557) bank109782 +109783 POINT(46.94430960621977 -121.3576391659044) bank109783 +109784 POINT(46.67208026367868 -122.71483200153297) bank109784 +109785 POINT(46.93577818488523 -122.45915450387719) bank109785 +109786 POINT(46.814519139639806 -121.92070946012291) bank109786 +109787 POINT(48.056421550395314 -121.83498953741875) bank109787 +109788 POINT(48.39772469743392 -122.05494309916031) bank109788 +109789 POINT(48.439842347403385 -121.76186758425696) bank109789 +109790 POINT(47.580312600183234 -121.9817773217968) bank109790 +109791 POINT(46.99777046099855 -122.56319542637738) bank109791 +109792 POINT(47.17603287041092 -122.44738503658013) bank109792 +109793 POINT(46.89259015417999 -122.61177572129252) bank109793 +109794 POINT(47.66068724074543 -123.31835435254872) bank109794 +109795 POINT(48.156940124695836 -121.52593751165402) bank109795 +109796 POINT(48.52593878625366 -123.16831737325965) bank109796 +109797 POINT(47.48374317306213 -123.21786771385894) bank109797 +109798 POINT(46.72145740824778 -122.07164988110956) bank109798 +109799 POINT(46.98610778393903 -122.99805559664176) bank109799 +109800 POINT(48.32169326850839 -121.99751236379929) bank109800 +109801 POINT(47.11099270515184 -121.66516784140643) bank109801 +109802 POINT(47.81237831933639 -123.26951925492159) bank109802 +109803 POINT(47.803623583060364 -122.77781457422657) bank109803 +109804 POINT(48.3523036028306 -123.01757642182903) bank109804 +109805 POINT(46.74925757620017 -123.15157253163177) bank109805 +109806 POINT(48.54885098787813 -121.54056492715657) bank109806 +109807 POINT(48.35898101243403 -122.65199027797027) bank109807 +109808 POINT(47.37790500882289 -121.78196968882266) bank109808 +109809 POINT(48.21347623873175 -121.76399661610564) bank109809 +109810 POINT(47.35171822551318 -123.13106871558233) bank109810 +109811 POINT(48.43719591762195 -121.92098654457345) bank109811 +109812 POINT(47.50696174867738 -121.5413862983297) bank109812 +109813 POINT(47.71772357960058 -121.91537003134765) bank109813 +109814 POINT(47.29462733342159 -122.95375371496942) bank109814 +109815 POINT(46.70065051740274 -121.6583388415516) bank109815 +109816 POINT(48.39798650240026 -122.88359848818276) bank109816 +109817 POINT(47.806379489118015 -122.246997475771) bank109817 +109818 POINT(47.72589532851054 -123.30564425649244) bank109818 +109819 POINT(48.57177602582223 -121.53747103747361) bank109819 +109820 POINT(46.611782274478536 -121.82659122617652) bank109820 +109821 POINT(46.67475802244155 -121.49257075126826) bank109821 +109822 POINT(47.50702911062924 -121.52734360569136) bank109822 +109823 POINT(47.515355811835946 -121.67823492447545) bank109823 +109824 POINT(47.61665666953296 -121.54801060812409) bank109824 +109825 POINT(48.015035523152726 -123.2711406439819) bank109825 +109826 POINT(47.13754254798984 -122.16723141768891) bank109826 +109827 POINT(46.89864002062329 -122.34633233358583) bank109827 +109828 POINT(48.01204308560585 -121.7579660918954) bank109828 +109829 POINT(47.732793249339046 -121.80147471131482) bank109829 +109830 POINT(47.41738302479077 -123.20503106523162) bank109830 +109831 POINT(47.037330945069776 -121.42755264222335) bank109831 +109832 POINT(47.91778167638823 -122.24511264587125) bank109832 +109833 POINT(48.02969582159893 -122.77468256172568) bank109833 +109834 POINT(48.5439679846587 -123.14634480391604) bank109834 +109835 POINT(48.14247781170078 -122.67265160057748) bank109835 +109836 POINT(47.11532873390906 -121.34674148864444) bank109836 +109837 POINT(48.417532811310025 -122.41666290388252) bank109837 +109838 POINT(48.22554571421323 -123.12736389365035) bank109838 +109839 POINT(47.46357102256906 -121.7929095511155) bank109839 +109840 POINT(47.93106939946786 -123.31354214298608) bank109840 +109841 POINT(47.426914272871926 -121.7614315862184) bank109841 +109842 POINT(47.39290252359501 -121.5120723513709) bank109842 +109843 POINT(46.79174934439422 -123.08186249319857) bank109843 +109844 POINT(47.22096074085053 -122.48048461819168) bank109844 +109845 POINT(47.0677592428873 -121.41280765371884) bank109845 +109846 POINT(48.214388420898494 -123.33206325352505) bank109846 +109847 POINT(47.2772938241166 -121.97417794794315) bank109847 +109848 POINT(47.71593717581262 -122.49478641091952) bank109848 +109849 POINT(48.194266744786304 -123.23331455023641) bank109849 +109850 POINT(47.976610944156235 -121.60988468537529) bank109850 +109851 POINT(47.53756147052251 -123.2402402189881) bank109851 +109852 POINT(48.36363577619163 -122.54742310314244) bank109852 +109853 POINT(47.523518603595235 -121.87217513150236) bank109853 +109854 POINT(47.39003474858186 -121.68915348226841) bank109854 +109855 POINT(47.15628028736753 -122.63121761966148) bank109855 +109856 POINT(47.18799914279165 -122.10144564391399) bank109856 +109857 POINT(47.93814111507078 -123.20411389494684) bank109857 +109858 POINT(46.82058233057271 -122.78467802786632) bank109858 +109859 POINT(47.35551738745351 -122.87611179807341) bank109859 +109860 POINT(48.47325714821323 -122.44299252781752) bank109860 +109861 POINT(46.6293771017889 -123.1320625547699) bank109861 +109862 POINT(48.07599216166767 -122.70137828773436) bank109862 +109863 POINT(48.24175488744662 -121.9103441539294) bank109863 +109864 POINT(47.83342463259778 -122.61251639224258) bank109864 +109865 POINT(48.31921933705694 -122.90300207025378) bank109865 +109866 POINT(47.4982033638864 -121.52815335694163) bank109866 +109867 POINT(48.33036412467554 -122.01632740669577) bank109867 +109868 POINT(48.19730454448882 -121.63103871612631) bank109868 +109869 POINT(48.15690069348209 -122.97853051641502) bank109869 +109870 POINT(48.407669877374154 -123.02021120068441) bank109870 +109871 POINT(47.314129833364085 -121.50519133929308) bank109871 +109872 POINT(48.02153690037506 -123.24002419403791) bank109872 +109873 POINT(48.22213016567026 -122.85760431934804) bank109873 +109874 POINT(47.36397154352727 -122.9287962522433) bank109874 +109875 POINT(48.54597903373167 -122.4853401782154) bank109875 +109876 POINT(47.47805984161703 -121.8667479651161) bank109876 +109877 POINT(48.247766189241716 -122.78707332253283) bank109877 +109878 POINT(47.21848521467304 -122.22700373443486) bank109878 +109879 POINT(47.17950299525037 -121.64199761773558) bank109879 +109880 POINT(47.40683268507648 -122.01643405152116) bank109880 +109881 POINT(48.12031471804715 -121.6308075669861) bank109881 +109882 POINT(46.95442785669061 -123.0162663601419) bank109882 +109883 POINT(46.92970516309013 -123.07293795370072) bank109883 +109884 POINT(46.979518516487474 -123.12252779089647) bank109884 +109885 POINT(47.55273577044427 -121.77603076410243) bank109885 +109886 POINT(46.909099925918454 -121.99138424350849) bank109886 +109887 POINT(47.15176516644993 -121.36811927926558) bank109887 +109888 POINT(47.53805682127304 -122.910836692703) bank109888 +109889 POINT(47.94531260842689 -121.44571310258686) bank109889 +109890 POINT(48.53345393422797 -122.69639279433537) bank109890 +109891 POINT(48.184706957169865 -121.79209128057387) bank109891 +109892 POINT(47.34485122780307 -122.00433871103368) bank109892 +109893 POINT(46.7049132174821 -122.86081499599844) bank109893 +109894 POINT(47.1171171181372 -121.78776495105515) bank109894 +109895 POINT(47.609365212961755 -122.72492613513464) bank109895 +109896 POINT(47.90545705882391 -123.31410505414158) bank109896 +109897 POINT(47.782516799956845 -123.19129568912307) bank109897 +109898 POINT(48.20094042117284 -121.97123506925644) bank109898 +109899 POINT(47.29054971949529 -122.64490549223702) bank109899 +109900 POINT(46.796647150810266 -122.63534516512605) bank109900 +109901 POINT(47.46584172109198 -122.89144884977108) bank109901 +109902 POINT(48.32308079622932 -121.7868884105548) bank109902 +109903 POINT(47.1852508854604 -123.25862679036233) bank109903 +109904 POINT(47.19785603362405 -122.6484312017295) bank109904 +109905 POINT(48.505922781080564 -121.56437722519046) bank109905 +109906 POINT(48.32219792053833 -122.2916141148403) bank109906 +109907 POINT(47.924306859667475 -122.2963228182545) bank109907 +109908 POINT(47.168608817544104 -123.31389993414733) bank109908 +109909 POINT(46.7247638470983 -121.44476161192135) bank109909 +109910 POINT(47.839502145869474 -121.66958932356088) bank109910 +109911 POINT(48.43169421411303 -121.92413766393355) bank109911 +109912 POINT(46.710695345944124 -122.91526332574206) bank109912 +109913 POINT(48.35641930299881 -121.3893500103825) bank109913 +109914 POINT(48.58292763902556 -121.89171464122666) bank109914 +109915 POINT(47.66605657563623 -122.85880734612272) bank109915 +109916 POINT(47.878224397916156 -121.73212252812125) bank109916 +109917 POINT(47.69998665236442 -122.32669314126197) bank109917 +109918 POINT(47.45147182832799 -121.49409826543437) bank109918 +109919 POINT(46.90796959275059 -121.71666641507942) bank109919 +109920 POINT(47.188798787633466 -122.69390089784126) bank109920 +109921 POINT(48.53385028475706 -122.85955402285163) bank109921 +109922 POINT(47.723738925204486 -121.39904359965671) bank109922 +109923 POINT(47.593772794996205 -121.53852811107534) bank109923 +109924 POINT(46.91516250229968 -123.16946969525368) bank109924 +109925 POINT(47.40907193012933 -122.46098890766511) bank109925 +109926 POINT(48.260764324343405 -122.28851112094198) bank109926 +109927 POINT(47.11965984732237 -122.3213078374863) bank109927 +109928 POINT(47.39773874408423 -122.17094126892621) bank109928 +109929 POINT(46.78736986164129 -122.71529521382793) bank109929 +109930 POINT(48.5916359402693 -122.30896243451441) bank109930 +109931 POINT(46.907366595192364 -123.22777362443958) bank109931 +109932 POINT(47.25984739461138 -122.14531584633599) bank109932 +109933 POINT(46.95359949528606 -122.30812588836095) bank109933 +109934 POINT(48.154264262049644 -123.247644009428) bank109934 +109935 POINT(47.64242646330335 -122.00274153966882) bank109935 +109936 POINT(47.851878527010946 -121.91504324304555) bank109936 +109937 POINT(48.12520819163757 -122.67803696628087) bank109937 +109938 POINT(48.294170080222464 -122.89758414560437) bank109938 +109939 POINT(48.379717738170385 -122.78179372289173) bank109939 +109940 POINT(47.42140877609061 -122.95202282542219) bank109940 +109941 POINT(47.64696401046339 -122.75531165799724) bank109941 +109942 POINT(47.70284245068312 -121.66062364842959) bank109942 +109943 POINT(47.094768822943465 -121.44817451780493) bank109943 +109944 POINT(48.226630881050795 -122.72985971453434) bank109944 +109945 POINT(47.7922900124078 -121.34704638407925) bank109945 +109946 POINT(48.41123659281829 -122.42637507214894) bank109946 +109947 POINT(47.69909498399199 -121.9647862274706) bank109947 +109948 POINT(48.24112466883655 -122.91107029993631) bank109948 +109949 POINT(47.55362482727555 -121.47553283338723) bank109949 +109950 POINT(48.129161637354734 -122.42288379129482) bank109950 +109951 POINT(47.362012726046295 -121.84562564848503) bank109951 +109952 POINT(48.40741873577487 -122.34380026382485) bank109952 +109953 POINT(47.89154660532503 -123.00651590871367) bank109953 +109954 POINT(46.953066474472415 -122.31176208813197) bank109954 +109955 POINT(47.47671494513601 -123.16189939980727) bank109955 +109956 POINT(46.61715781912821 -123.13854802007073) bank109956 +109957 POINT(47.95173366616074 -122.28575828323287) bank109957 +109958 POINT(47.289710029253605 -122.76495800751422) bank109958 +109959 POINT(47.61046372054724 -123.0321169357475) bank109959 +109960 POINT(47.762782593557674 -121.92689193456947) bank109960 +109961 POINT(48.141848020205366 -122.91301704000868) bank109961 +109962 POINT(47.691481282105336 -122.6386862037969) bank109962 +109963 POINT(48.546399929788016 -121.64400033306326) bank109963 +109964 POINT(47.28752071868107 -122.17588801308226) bank109964 +109965 POINT(48.13665844678726 -121.68773599707256) bank109965 +109966 POINT(47.48601402881068 -122.91156371286435) bank109966 +109967 POINT(48.34363889127189 -122.13247536792578) bank109967 +109968 POINT(48.013349194793626 -121.58933679084038) bank109968 +109969 POINT(48.48526213548314 -121.74345650588845) bank109969 +109970 POINT(48.25519968851255 -122.37957569106997) bank109970 +109971 POINT(46.97412023078468 -123.25478758670427) bank109971 +109972 POINT(47.48912620655178 -122.92192923787731) bank109972 +109973 POINT(47.38161224999234 -121.90134824374117) bank109973 +109974 POINT(47.45513312971135 -121.86172782216421) bank109974 +109975 POINT(48.03447720167219 -122.42860613443233) bank109975 +109976 POINT(48.364328035803105 -121.80469666527517) bank109976 +109977 POINT(47.6695986714611 -122.76026456226907) bank109977 +109978 POINT(47.493245693961754 -122.3171220982446) bank109978 +109979 POINT(46.652132514628704 -123.30817975468517) bank109979 +109980 POINT(48.007243515335276 -122.93899873730238) bank109980 +109981 POINT(48.318025713961234 -122.56695224802732) bank109981 +109982 POINT(46.98709687596373 -123.17945233545228) bank109982 +109983 POINT(48.48736677960918 -121.44672031275974) bank109983 +109984 POINT(47.61513129858443 -122.16955746976492) bank109984 +109985 POINT(48.015002934274825 -122.37360729001495) bank109985 +109986 POINT(46.65315852371079 -121.35922962040007) bank109986 +109987 POINT(47.978209499913376 -121.67957902879799) bank109987 +109988 POINT(47.15561491238911 -123.06767001731471) bank109988 +109989 POINT(48.24115121571311 -122.24206338716472) bank109989 +109990 POINT(47.508881972471606 -121.36284900923772) bank109990 +109991 POINT(47.19853539182089 -121.49192756819357) bank109991 +109992 POINT(47.84223226863385 -121.9138427966463) bank109992 +109993 POINT(47.47047384267681 -123.14074261489358) bank109993 +109994 POINT(46.796720070698136 -123.30437603168866) bank109994 +109995 POINT(48.46911756593371 -122.90304952654714) bank109995 +109996 POINT(46.61960987409768 -123.25100226165873) bank109996 +109997 POINT(48.39293591187954 -121.72249598064693) bank109997 +109998 POINT(47.47770571682002 -122.08734773033217) bank109998 +109999 POINT(48.30243091652623 -122.92177651904845) bank109999 +110000 POINT(47.948819782964776 -122.5843359764007) bank110000 +110001 POINT(48.461929731038666 -121.61170525741537) bank110001 +110002 POINT(46.73721108700142 -122.20642289795983) bank110002 +110003 POINT(46.7437716484613 -122.06700688739517) bank110003 +110004 POINT(48.15063860313902 -122.67083913747572) bank110004 +110005 POINT(47.82748824295662 -121.80727153647061) bank110005 +110006 POINT(46.73009472331845 -122.83715834278296) bank110006 +110007 POINT(48.22261444932629 -123.2286721392301) bank110007 +110008 POINT(48.36498474415225 -122.60308667900381) bank110008 +110009 POINT(47.20950378524383 -122.17316059580406) bank110009 +110010 POINT(47.37797639295072 -123.29514592246989) bank110010 +110011 POINT(47.1164581704438 -123.23791110047945) bank110011 +110012 POINT(47.36816547428285 -122.20196951336418) bank110012 +110013 POINT(47.6872620594859 -121.68886442267129) bank110013 +110014 POINT(47.15319121393217 -123.19333976281837) bank110014 +110015 POINT(48.13469681980616 -121.91797335809675) bank110015 +110016 POINT(47.23844886636251 -122.17099280300532) bank110016 +110017 POINT(47.56486968235484 -122.99393622496541) bank110017 +110018 POINT(48.25577902774028 -121.34999422689343) bank110018 +110019 POINT(46.652172659916424 -121.43781841389381) bank110019 +110020 POINT(47.527814749497324 -122.57048273797832) bank110020 +110021 POINT(47.12681689916215 -122.81918025792766) bank110021 +110022 POINT(46.701306678032296 -122.84910938422125) bank110022 +110023 POINT(48.46126908022633 -122.6239922310469) bank110023 +110024 POINT(48.0175418035584 -123.26757161570508) bank110024 +110025 POINT(47.786556962791565 -121.77626599227706) bank110025 +110026 POINT(46.653477511010855 -123.07682725474095) bank110026 +110027 POINT(46.860356622667695 -122.27328305387803) bank110027 +110028 POINT(47.71500554150624 -122.54570330658565) bank110028 +110029 POINT(48.484715900555656 -123.22555081496596) bank110029 +110030 POINT(47.784966869534024 -122.2830058483316) bank110030 +110031 POINT(47.85600320102917 -123.03310039913201) bank110031 +110032 POINT(48.08543196532786 -121.53284878142016) bank110032 +110033 POINT(47.31487680583602 -122.39266992393081) bank110033 +110034 POINT(47.17816573205559 -121.4344304997644) bank110034 +110035 POINT(46.87901193541685 -121.62627672128635) bank110035 +110036 POINT(48.0986670010017 -122.50790147600395) bank110036 +110037 POINT(47.76090599037195 -123.06866344016063) bank110037 +110038 POINT(47.988687330657825 -122.78939270608356) bank110038 +110039 POINT(48.37941330683212 -122.85857988145328) bank110039 +110040 POINT(48.32558356086703 -123.23079482109382) bank110040 +110041 POINT(48.40706371372588 -121.52233880301748) bank110041 +110042 POINT(46.80292108236873 -121.74130884408359) bank110042 +110043 POINT(48.327754494935895 -122.79164551456188) bank110043 +110044 POINT(47.882859067727814 -122.75177555547467) bank110044 +110045 POINT(47.4233281069611 -121.56077244510504) bank110045 +110046 POINT(47.427546860259646 -122.87394228233893) bank110046 +110047 POINT(48.30675762920586 -122.94071640648761) bank110047 +110048 POINT(46.64218000933225 -122.92864739101518) bank110048 +110049 POINT(48.506593110228856 -122.43928294153143) bank110049 +110050 POINT(47.14488344223039 -123.09025214964556) bank110050 +110051 POINT(47.627876442578085 -121.85440011196425) bank110051 +110052 POINT(47.306601267961504 -121.5711179132894) bank110052 +110053 POINT(47.92987075176114 -123.23568291211316) bank110053 +110054 POINT(47.050402519668054 -123.27163163391052) bank110054 +110055 POINT(46.99142706825413 -122.31679804906301) bank110055 +110056 POINT(47.23279285386629 -122.35143568785435) bank110056 +110057 POINT(47.22952903971961 -121.53165093851541) bank110057 +110058 POINT(48.10094031664724 -121.65918140364751) bank110058 +110059 POINT(47.87365364736091 -121.99899204199986) bank110059 +110060 POINT(47.84552274624287 -123.00392723203323) bank110060 +110061 POINT(46.6913418905575 -121.35350422475405) bank110061 +110062 POINT(48.472604982871175 -122.6175545885624) bank110062 +110063 POINT(47.5375927719748 -121.923384727755) bank110063 +110064 POINT(47.3409209705175 -122.15559561055488) bank110064 +110065 POINT(47.943537289741805 -122.58940064658886) bank110065 +110066 POINT(47.38278119424636 -122.98516304126507) bank110066 +110067 POINT(47.29970825179744 -122.79228654917533) bank110067 +110068 POINT(47.207630093815006 -121.8677650447572) bank110068 +110069 POINT(47.39894463819112 -123.06020967090657) bank110069 +110070 POINT(48.24642269735984 -122.77630551164292) bank110070 +110071 POINT(47.307270262924305 -122.53370245781716) bank110071 +110072 POINT(47.03326277476326 -122.24443630327565) bank110072 +110073 POINT(48.53898600906054 -121.84765328064591) bank110073 +110074 POINT(48.15757543485902 -123.09578934379157) bank110074 +110075 POINT(47.61713136766876 -121.89213053980743) bank110075 +110076 POINT(46.844108167038435 -123.04106908765063) bank110076 +110077 POINT(47.40802706855476 -122.9549684963536) bank110077 +110078 POINT(46.762908736477016 -121.98819615103379) bank110078 +110079 POINT(47.86549929820731 -122.65212760363478) bank110079 +110080 POINT(46.854825384336934 -122.25760027184855) bank110080 +110081 POINT(47.295737505478314 -122.35584857520935) bank110081 +110082 POINT(47.38756346780046 -123.02248308055019) bank110082 +110083 POINT(48.26222955776112 -122.46258153203904) bank110083 +110084 POINT(47.940348985839435 -121.96311522846507) bank110084 +110085 POINT(47.71899858578199 -122.36896371230225) bank110085 +110086 POINT(48.571576003767014 -121.46522685969393) bank110086 +110087 POINT(48.15564750837556 -121.98004598308724) bank110087 +110088 POINT(46.61023241540701 -121.84231806007999) bank110088 +110089 POINT(48.23217434077537 -122.80028119765358) bank110089 +110090 POINT(47.00847962805932 -121.86569837354156) bank110090 +110091 POINT(48.378351860947014 -123.16573455815742) bank110091 +110092 POINT(47.7701717106499 -122.79048400796408) bank110092 +110093 POINT(47.62187682266327 -121.57343078008353) bank110093 +110094 POINT(46.97761908810181 -121.8877552742854) bank110094 +110095 POINT(47.397650592227485 -122.64925965196693) bank110095 +110096 POINT(47.3829041733667 -123.09855329936896) bank110096 +110097 POINT(47.28300537961385 -122.60088752885882) bank110097 +110098 POINT(47.40304768759787 -123.03808886740727) bank110098 +110099 POINT(46.777456954045974 -123.21651969300034) bank110099 +110100 POINT(46.97420683717895 -121.68510536993128) bank110100 +110101 POINT(47.84905223208645 -121.8635414908356) bank110101 +110102 POINT(47.08959263253865 -121.65848790127167) bank110102 +110103 POINT(46.79654489240263 -122.48193987941617) bank110103 +110104 POINT(47.2913064658658 -122.66147080743745) bank110104 +110105 POINT(46.6107180282761 -122.52523986964444) bank110105 +110106 POINT(48.14841964240151 -123.23507014671574) bank110106 +110107 POINT(48.472457013080074 -123.06132647517356) bank110107 +110108 POINT(47.306245275323384 -122.82167927227867) bank110108 +110109 POINT(46.87504241919422 -123.278424079201) bank110109 +110110 POINT(47.77751107059445 -121.55765124815304) bank110110 +110111 POINT(46.63305098760557 -123.16344701898859) bank110111 +110112 POINT(47.0948964780972 -123.11439728715646) bank110112 +110113 POINT(47.847933193602444 -122.52545897140597) bank110113 +110114 POINT(46.837599778105314 -122.97339042360143) bank110114 +110115 POINT(46.98538767672649 -121.85192033110555) bank110115 +110116 POINT(47.04948330941265 -121.88257729208517) bank110116 +110117 POINT(47.27327915347304 -123.13199846779055) bank110117 +110118 POINT(47.32593246727349 -122.13562051906007) bank110118 +110119 POINT(48.155391053641125 -122.7448187607574) bank110119 +110120 POINT(46.780213406805885 -122.08667311056453) bank110120 +110121 POINT(47.512852676697676 -121.52560285427832) bank110121 +110122 POINT(48.56318358540528 -123.3254598265127) bank110122 +110123 POINT(48.04840354518692 -121.41120255832298) bank110123 +110124 POINT(47.065724819939135 -122.95761277063237) bank110124 +110125 POINT(47.56010881818581 -122.08418626358949) bank110125 +110126 POINT(46.923712621094204 -122.44678761665163) bank110126 +110127 POINT(47.204018419085486 -121.9395635333095) bank110127 +110128 POINT(47.827256546158694 -123.02623779176805) bank110128 +110129 POINT(47.188750946425316 -121.94985213598758) bank110129 +110130 POINT(47.92960289606759 -122.61004292092801) bank110130 +110131 POINT(47.052210855015126 -123.23160750801536) bank110131 +110132 POINT(47.5777576160182 -123.002246996487) bank110132 +110133 POINT(48.084539191840626 -121.65334806854968) bank110133 +110134 POINT(48.58062125115163 -121.85508267089087) bank110134 +110135 POINT(48.554163206910644 -122.07635745756794) bank110135 +110136 POINT(47.53490292882525 -121.78341478413222) bank110136 +110137 POINT(47.48881832124231 -122.17072813490253) bank110137 +110138 POINT(48.551567390139226 -122.87232347794381) bank110138 +110139 POINT(46.87371934868394 -121.53531133363956) bank110139 +110140 POINT(47.38484399331534 -123.2583641487652) bank110140 +110141 POINT(48.22296560476409 -122.10489892333736) bank110141 +110142 POINT(47.46118322795886 -122.09130272766747) bank110142 +110143 POINT(46.97012633138681 -121.53249422426664) bank110143 +110144 POINT(47.14841493605313 -122.9734594616977) bank110144 +110145 POINT(47.06805554206801 -123.23215497013686) bank110145 +110146 POINT(46.97392180196776 -122.54681122988357) bank110146 +110147 POINT(47.81764544800244 -121.51503539397277) bank110147 +110148 POINT(48.01711993779494 -123.07571446693656) bank110148 +110149 POINT(48.10543499696264 -122.0195209306632) bank110149 +110150 POINT(46.658244962984696 -121.34729108832356) bank110150 +110151 POINT(47.44706022077492 -121.51885892857692) bank110151 +110152 POINT(46.70042692404212 -122.05841305892035) bank110152 +110153 POINT(47.12882664376347 -122.0510578038549) bank110153 +110154 POINT(47.05560877586306 -122.20585193175631) bank110154 +110155 POINT(47.85775426716669 -122.89661611587128) bank110155 +110156 POINT(46.78262518498406 -121.57633233521578) bank110156 +110157 POINT(48.45687548480299 -122.33703833853981) bank110157 +110158 POINT(47.9373280377543 -121.83608690573438) bank110158 +110159 POINT(47.85588063566045 -121.77576607688006) bank110159 +110160 POINT(47.83429624423246 -121.74905721224292) bank110160 +110161 POINT(47.187676581778184 -122.16469712483314) bank110161 +110162 POINT(47.35931699396007 -121.82471737468694) bank110162 +110163 POINT(47.38495100303304 -122.91391835421464) bank110163 +110164 POINT(47.938496538878574 -121.75970484725521) bank110164 +110165 POINT(48.34326586355725 -123.25221722393088) bank110165 +110166 POINT(46.943111660628546 -121.99681816819331) bank110166 +110167 POINT(46.81486505610093 -122.04871825781538) bank110167 +110168 POINT(47.964851297135766 -121.7885614851279) bank110168 +110169 POINT(47.17566338142463 -122.1723974609295) bank110169 +110170 POINT(47.12332813180415 -121.96661405878602) bank110170 +110171 POINT(47.89047009933094 -122.81747555276527) bank110171 +110172 POINT(47.433639031617844 -122.76524975337455) bank110172 +110173 POINT(47.179528363649375 -122.5919918260819) bank110173 +110174 POINT(48.13313639309736 -123.18173618848164) bank110174 +110175 POINT(48.591419706530125 -121.9714378114389) bank110175 +110176 POINT(46.708623026779954 -122.05781553274892) bank110176 +110177 POINT(48.28215459819577 -123.03218793630914) bank110177 +110178 POINT(48.18593848903738 -123.23317955872672) bank110178 +110179 POINT(46.942566718386395 -122.67655144652605) bank110179 +110180 POINT(47.672791684893774 -122.59434112682456) bank110180 +110181 POINT(47.40113519971539 -122.78469542858875) bank110181 +110182 POINT(47.97485809911369 -123.20359162925546) bank110182 +110183 POINT(47.72926910238883 -123.3105675134758) bank110183 +110184 POINT(47.21339550182533 -122.69228880440012) bank110184 +110185 POINT(47.978782078076726 -121.5677445955586) bank110185 +110186 POINT(48.04054398101458 -121.47631326119271) bank110186 +110187 POINT(47.03824029792495 -121.4409963907395) bank110187 +110188 POINT(47.629527847837394 -122.54789766418237) bank110188 +110189 POINT(47.01649606815074 -122.92941970229991) bank110189 +110190 POINT(47.37886676020474 -122.93654933343251) bank110190 +110191 POINT(47.704808446842435 -122.55906380638217) bank110191 +110192 POINT(48.079458743707335 -122.74707409917218) bank110192 +110193 POINT(47.05636192346391 -123.2096015521944) bank110193 +110194 POINT(48.58555213143611 -121.51892769486808) bank110194 +110195 POINT(46.795935818751516 -123.25645140391721) bank110195 +110196 POINT(47.87199916472743 -122.21885334842378) bank110196 +110197 POINT(47.74183020086714 -122.4128152722284) bank110197 +110198 POINT(48.54478550887129 -123.20473347004268) bank110198 +110199 POINT(47.76845592478435 -122.49095664985857) bank110199 +110200 POINT(46.93170631545052 -121.86383601123151) bank110200 +110201 POINT(46.89704521317661 -122.94382814650996) bank110201 +110202 POINT(47.373175078078944 -121.6258285156117) bank110202 +110203 POINT(48.42892454040968 -122.67007025533648) bank110203 +110204 POINT(47.16274574193177 -122.95979200379332) bank110204 +110205 POINT(48.23213381851324 -123.15777589484749) bank110205 +110206 POINT(47.017022576988815 -122.8062777104819) bank110206 +110207 POINT(47.32527983275438 -123.2679676082383) bank110207 +110208 POINT(48.395902271475734 -122.76485686845987) bank110208 +110209 POINT(47.480726857218286 -121.57216729358974) bank110209 +110210 POINT(48.41311685795447 -121.53881572684011) bank110210 +110211 POINT(48.18514148899019 -121.79920193897631) bank110211 +110212 POINT(48.15510051934501 -121.48598809943553) bank110212 +110213 POINT(46.945876124099776 -122.0718887884505) bank110213 +110214 POINT(47.734036902708745 -121.71146139885276) bank110214 +110215 POINT(48.54733334439273 -122.13611677563185) bank110215 +110216 POINT(46.95891472577234 -122.91120534404895) bank110216 +110217 POINT(47.19327299494612 -121.67296481872629) bank110217 +110218 POINT(47.27600913039543 -122.1660595108497) bank110218 +110219 POINT(48.28198388443281 -121.56434238806145) bank110219 +110220 POINT(47.81955694620312 -121.64691851558565) bank110220 +110221 POINT(47.69526048945178 -122.04870366612177) bank110221 +110222 POINT(47.588656967906324 -121.7613858641) bank110222 +110223 POINT(48.41616632889565 -121.91775184365358) bank110223 +110224 POINT(47.97265106711539 -122.55380647720276) bank110224 +110225 POINT(47.563945334634035 -122.86856811063024) bank110225 +110226 POINT(47.21862028859108 -121.56250666646912) bank110226 +110227 POINT(47.428767462197804 -123.15156871559726) bank110227 +110228 POINT(46.724476523586375 -122.65905312723494) bank110228 +110229 POINT(48.48731422263763 -121.5678979829385) bank110229 +110230 POINT(48.53507908330938 -122.77174939234355) bank110230 +110231 POINT(48.48541793904047 -122.26523144446206) bank110231 +110232 POINT(47.989875407845695 -122.26996232502057) bank110232 +110233 POINT(47.17824012497733 -121.53772663689809) bank110233 +110234 POINT(48.42161385046231 -122.7983972586797) bank110234 +110235 POINT(48.305243191399335 -122.66827934644262) bank110235 +110236 POINT(47.926785221162724 -123.0183144379886) bank110236 +110237 POINT(47.584299995645914 -123.23185538863919) bank110237 +110238 POINT(48.45102952876689 -121.50533035441778) bank110238 +110239 POINT(47.78895463061922 -121.96614966631881) bank110239 +110240 POINT(48.22875636864453 -122.22066901159249) bank110240 +110241 POINT(47.86269071823074 -122.72859733863203) bank110241 +110242 POINT(47.029164091049815 -122.22437182676393) bank110242 +110243 POINT(47.043072285065094 -121.65552569467211) bank110243 +110244 POINT(46.664745592639726 -121.38064698987809) bank110244 +110245 POINT(47.57684362871173 -123.11465916803449) bank110245 +110246 POINT(47.40510795715403 -121.33441980308285) bank110246 +110247 POINT(47.86358560523419 -123.29840939918627) bank110247 +110248 POINT(46.94368642037691 -122.78214248949668) bank110248 +110249 POINT(48.01921479064669 -122.49398136080099) bank110249 +110250 POINT(47.49020013485729 -121.86753947970183) bank110250 +110251 POINT(46.98237369005127 -122.2512607384341) bank110251 +110252 POINT(48.05272821288392 -123.17325153710884) bank110252 +110253 POINT(47.122420662953324 -123.00692005091088) bank110253 +110254 POINT(47.87030374435257 -122.9083246203552) bank110254 +110255 POINT(47.567253849701125 -122.5835384421016) bank110255 +110256 POINT(47.16616989135946 -122.60883729868942) bank110256 +110257 POINT(46.99291919873594 -123.29041821482299) bank110257 +110258 POINT(47.12118425779609 -121.58745311729398) bank110258 +110259 POINT(48.518548369191755 -122.48474080408111) bank110259 +110260 POINT(47.93943316123934 -122.88081284182915) bank110260 +110261 POINT(48.12569052041926 -122.01273330064375) bank110261 +110262 POINT(47.56784581284094 -122.3530799934086) bank110262 +110263 POINT(46.73538275837334 -122.66324786191143) bank110263 +110264 POINT(48.54206266050139 -122.4188280867305) bank110264 +110265 POINT(46.9325781560028 -121.43629161289186) bank110265 +110266 POINT(48.267550270022916 -122.37397922127877) bank110266 +110267 POINT(48.492564345521664 -122.55307051524477) bank110267 +110268 POINT(46.67256113152128 -123.02910446725637) bank110268 +110269 POINT(48.244716621737304 -123.01334081253555) bank110269 +110270 POINT(46.633080681439765 -122.06670054927282) bank110270 +110271 POINT(46.92085469294373 -123.26177390817705) bank110271 +110272 POINT(48.534495461291236 -122.88331032729158) bank110272 +110273 POINT(46.75098271474737 -122.06802138246323) bank110273 +110274 POINT(48.16664863882558 -122.98665443324255) bank110274 +110275 POINT(48.485489088139346 -121.79288840326971) bank110275 +110276 POINT(47.45332790298499 -122.29665712162578) bank110276 +110277 POINT(46.65334234999049 -122.29102231719735) bank110277 +110278 POINT(47.22719635925374 -122.32309836649338) bank110278 +110279 POINT(46.97817160837198 -122.27677157893069) bank110279 +110280 POINT(47.63152041570285 -122.24205326146878) bank110280 +110281 POINT(48.088224238345084 -121.41707277049757) bank110281 +110282 POINT(48.53709058366284 -122.69808386876149) bank110282 +110283 POINT(47.954060082717135 -121.4751562892641) bank110283 +110284 POINT(47.31277700128254 -123.13966618519702) bank110284 +110285 POINT(48.37321916946983 -123.18737210508483) bank110285 +110286 POINT(46.93221439074431 -122.99059477072952) bank110286 +110287 POINT(47.28466519744188 -122.51219938873663) bank110287 +110288 POINT(48.50945450159114 -123.01788813558504) bank110288 +110289 POINT(46.86685024626171 -123.27375623869557) bank110289 +110290 POINT(47.50056599351955 -122.03488286782837) bank110290 +110291 POINT(47.62811163462047 -122.64580570991512) bank110291 +110292 POINT(48.29457829640544 -122.08579461724078) bank110292 +110293 POINT(47.33516770447621 -122.20300776656704) bank110293 +110294 POINT(46.87629613710888 -121.67824973822916) bank110294 +110295 POINT(47.09918900145033 -122.00333968708931) bank110295 +110296 POINT(47.37161039873905 -121.44824938961555) bank110296 +110297 POINT(47.176913741912024 -123.25500777252408) bank110297 +110298 POINT(46.96795600904123 -122.72452166721143) bank110298 +110299 POINT(47.278011899566515 -122.72238292032041) bank110299 +110300 POINT(46.6095771132383 -122.84992623853) bank110300 +110301 POINT(48.18523018768264 -121.4365255955252) bank110301 +110302 POINT(47.14716140440356 -121.71228698742058) bank110302 +110303 POINT(48.44151189812654 -123.1163641267985) bank110303 +110304 POINT(46.87267736477527 -122.70252370980498) bank110304 +110305 POINT(48.21656438406839 -122.35288568913491) bank110305 +110306 POINT(47.43755861330683 -123.03840926872167) bank110306 +110307 POINT(48.16792620480385 -122.70661760179053) bank110307 +110308 POINT(47.34623327924447 -121.35751226496771) bank110308 +110309 POINT(47.04693379883401 -122.17808778369596) bank110309 +110310 POINT(47.254877915732 -123.08654970880185) bank110310 +110311 POINT(47.71768563397493 -122.33537958624295) bank110311 +110312 POINT(48.51189454320179 -121.95874128901418) bank110312 +110313 POINT(47.769271967997334 -122.42614505563441) bank110313 +110314 POINT(48.34638182956402 -123.19842258872951) bank110314 +110315 POINT(48.02645517607323 -123.24834559769842) bank110315 +110316 POINT(47.44058558926183 -122.66521766800196) bank110316 +110317 POINT(46.6276638785534 -122.93015856956177) bank110317 +110318 POINT(47.01966245583639 -121.60334969004677) bank110318 +110319 POINT(48.151716942848964 -123.03100096741056) bank110319 +110320 POINT(48.045587534036336 -122.44754475942293) bank110320 +110321 POINT(47.783080049659134 -121.61457562388689) bank110321 +110322 POINT(47.44603988959247 -121.67948810523968) bank110322 +110323 POINT(48.26958741356844 -123.01914464440605) bank110323 +110324 POINT(47.92434768394601 -122.96724953230122) bank110324 +110325 POINT(46.781549553758644 -121.40328233425335) bank110325 +110326 POINT(47.268200126051575 -121.3373087083477) bank110326 +110327 POINT(48.13210216617176 -122.44648908492321) bank110327 +110328 POINT(48.38298818254868 -121.43003275590821) bank110328 +110329 POINT(48.25361672359721 -123.26249343708308) bank110329 +110330 POINT(48.5540892507397 -121.45876736967706) bank110330 +110331 POINT(46.90323670385796 -122.45696893647772) bank110331 +110332 POINT(47.352379356278334 -122.82104435992701) bank110332 +110333 POINT(47.21805741647723 -122.09138974957956) bank110333 +110334 POINT(47.94488888605343 -122.66374648085151) bank110334 +110335 POINT(47.01764292211065 -122.92439945876168) bank110335 +110336 POINT(48.212786473793514 -122.44306403103494) bank110336 +110337 POINT(47.43528047006128 -122.79428418556402) bank110337 +110338 POINT(47.13358156571313 -121.78082667027572) bank110338 +110339 POINT(46.70479078934665 -122.2146023957213) bank110339 +110340 POINT(47.93175787815392 -122.89203227610287) bank110340 +110341 POINT(48.1194465793547 -121.92516513342102) bank110341 +110342 POINT(46.72692236161329 -121.67249667309665) bank110342 +110343 POINT(48.00598432140617 -122.67917660623456) bank110343 +110344 POINT(46.76386737050579 -122.99630243783452) bank110344 +110345 POINT(48.16210566902542 -122.29840019608172) bank110345 +110346 POINT(46.799541538211194 -121.86403394242497) bank110346 +110347 POINT(47.38466973328884 -123.01134118436745) bank110347 +110348 POINT(48.4409524645475 -121.46027498920252) bank110348 +110349 POINT(48.49101111502665 -123.07394361362662) bank110349 +110350 POINT(46.942592266515646 -121.7255327515752) bank110350 +110351 POINT(47.92474009226844 -121.4455342572401) bank110351 +110352 POINT(47.89733422813486 -123.2526897005468) bank110352 +110353 POINT(47.68570222473037 -122.51044302004217) bank110353 +110354 POINT(48.45094532039814 -122.36522683262007) bank110354 +110355 POINT(47.76345784916918 -122.67294937148962) bank110355 +110356 POINT(47.10891373555649 -122.8197927199864) bank110356 +110357 POINT(47.344044925609865 -121.82991411952402) bank110357 +110358 POINT(48.495347852044425 -122.6279985464695) bank110358 +110359 POINT(48.01996554422324 -122.73858964539792) bank110359 +110360 POINT(47.537444892939476 -123.16191116207494) bank110360 +110361 POINT(48.2591447251356 -123.0371437280328) bank110361 +110362 POINT(47.573771389093636 -122.71933720733041) bank110362 +110363 POINT(48.36914751715627 -122.34231575442371) bank110363 +110364 POINT(47.32118775696189 -122.81399910527259) bank110364 +110365 POINT(46.622840826505325 -121.67557568366834) bank110365 +110366 POINT(47.52113837220012 -122.66086362118442) bank110366 +110367 POINT(47.75757051132125 -121.91790324401056) bank110367 +110368 POINT(48.45080319121238 -121.79822783395112) bank110368 +110369 POINT(47.64097890892671 -121.78361259849247) bank110369 +110370 POINT(47.770375645177 -121.59674614900804) bank110370 +110371 POINT(48.04086876161816 -123.31030176458735) bank110371 +110372 POINT(47.35798106842982 -121.99829186808017) bank110372 +110373 POINT(47.823137909511836 -121.93521374682756) bank110373 +110374 POINT(46.87726062019352 -122.31410086713589) bank110374 +110375 POINT(47.72136298809766 -121.6931290085054) bank110375 +110376 POINT(47.547490712162556 -122.05784547893266) bank110376 +110377 POINT(47.019950456000444 -121.53822167437346) bank110377 +110378 POINT(47.04490124704109 -122.0925579845658) bank110378 +110379 POINT(48.52041924830162 -122.43354283560987) bank110379 +110380 POINT(47.78522406557847 -122.27107798133146) bank110380 +110381 POINT(46.98353258939927 -122.89276456760481) bank110381 +110382 POINT(48.23227390770531 -121.35803809190084) bank110382 +110383 POINT(47.36610683597519 -122.64925006527659) bank110383 +110384 POINT(47.41371171745811 -122.67091529671285) bank110384 +110385 POINT(47.66706900311474 -122.22654078746552) bank110385 +110386 POINT(48.19276332814163 -122.39904096951507) bank110386 +110387 POINT(47.05969276179629 -122.89133888428276) bank110387 +110388 POINT(48.0427041972874 -121.73599810168251) bank110388 +110389 POINT(47.937608746351785 -122.77568228313005) bank110389 +110390 POINT(47.28739049820013 -121.80904837749229) bank110390 +110391 POINT(47.44202963174089 -122.39824231677689) bank110391 +110392 POINT(48.38468655600376 -122.38302169460846) bank110392 +110393 POINT(46.730943122222534 -122.80748144243572) bank110393 +110394 POINT(46.78690603255892 -122.05362381339111) bank110394 +110395 POINT(48.55020276097675 -122.57145664952971) bank110395 +110396 POINT(48.16178608975822 -121.89788096251714) bank110396 +110397 POINT(48.09393271798136 -122.67994512741198) bank110397 +110398 POINT(46.90544970236393 -123.30337943618805) bank110398 +110399 POINT(47.90297936722912 -122.11143004090266) bank110399 +110400 POINT(47.30675814098909 -122.57072984668173) bank110400 +110401 POINT(47.7382271557118 -122.1900135738556) bank110401 +110402 POINT(47.13526371379911 -121.4246627505106) bank110402 +110403 POINT(48.59742399018903 -122.28273899690312) bank110403 +110404 POINT(48.08836336866038 -121.76110158305997) bank110404 +110405 POINT(46.84936216844967 -121.79442896759333) bank110405 +110406 POINT(48.44775728210618 -121.71817234362379) bank110406 +110407 POINT(48.513717865410136 -121.50727919294212) bank110407 +110408 POINT(48.481768794919084 -121.38687598980742) bank110408 +110409 POINT(48.204735285680336 -122.03793991564375) bank110409 +110410 POINT(47.80063791507974 -122.53718257892483) bank110410 +110411 POINT(47.2917899469347 -121.79480404673298) bank110411 +110412 POINT(47.898830800639494 -121.60425701436701) bank110412 +110413 POINT(46.86923455875878 -122.29606224616215) bank110413 +110414 POINT(47.57487317065235 -122.71533129667128) bank110414 +110415 POINT(47.10710232583046 -121.658462854699) bank110415 +110416 POINT(48.112479531556744 -122.86702054537867) bank110416 +110417 POINT(48.30763313924128 -123.25102449963764) bank110417 +110418 POINT(47.34572242705963 -122.2667056805731) bank110418 +110419 POINT(46.971779756172076 -123.07635752020457) bank110419 +110420 POINT(46.969922003420244 -121.76648240959426) bank110420 +110421 POINT(48.41788512425682 -121.40338414964395) bank110421 +110422 POINT(48.147833683962915 -122.40469753944845) bank110422 +110423 POINT(47.211115006899476 -121.94752038708123) bank110423 +110424 POINT(47.04064929887906 -122.08034872486007) bank110424 +110425 POINT(46.80835909154173 -122.61276893987093) bank110425 +110426 POINT(47.75994673861445 -123.11537423040308) bank110426 +110427 POINT(48.08360218858147 -122.91639519362866) bank110427 +110428 POINT(47.655306542497094 -121.74327667920393) bank110428 +110429 POINT(48.32284362727088 -121.99881518452196) bank110429 +110430 POINT(48.41868630836304 -122.76662768814472) bank110430 +110431 POINT(47.03875165145662 -122.33330304937473) bank110431 +110432 POINT(47.70820348969919 -122.95470993486958) bank110432 +110433 POINT(47.86264946843762 -122.08868931636385) bank110433 +110434 POINT(47.30654706432399 -122.17340582914021) bank110434 +110435 POINT(47.648868497554524 -123.09074506976506) bank110435 +110436 POINT(47.73888551890397 -123.07953935100326) bank110436 +110437 POINT(47.89885419443461 -122.84157104927519) bank110437 +110438 POINT(47.56923035165063 -122.61514907084937) bank110438 +110439 POINT(47.2619811476304 -122.42723574993926) bank110439 +110440 POINT(46.94554711414054 -122.26542348615493) bank110440 +110441 POINT(48.51551767015447 -121.8327547297467) bank110441 +110442 POINT(47.89224688944809 -121.71637148448822) bank110442 +110443 POINT(47.51417410979944 -121.44173936192804) bank110443 +110444 POINT(48.33556933957523 -122.3935525856247) bank110444 +110445 POINT(47.88569374661132 -121.4926754697383) bank110445 +110446 POINT(46.60695275225678 -123.01158255121118) bank110446 +110447 POINT(48.41506449857201 -121.6455898479397) bank110447 +110448 POINT(46.996262089588555 -122.23407789906766) bank110448 +110449 POINT(46.99139818361364 -121.88727406610101) bank110449 +110450 POINT(48.42983781518903 -123.09260269286462) bank110450 +110451 POINT(47.26263141297548 -121.38368247340368) bank110451 +110452 POINT(48.320320919616385 -121.71512914270308) bank110452 +110453 POINT(46.63737317451407 -121.38122705126318) bank110453 +110454 POINT(48.1386032373032 -122.19393769220073) bank110454 +110455 POINT(48.24016234948129 -121.49544139306077) bank110455 +110456 POINT(47.01700375261333 -122.13632296359133) bank110456 +110457 POINT(48.27471748698663 -122.9706932369645) bank110457 +110458 POINT(48.193559560603234 -121.40893498780227) bank110458 +110459 POINT(47.5225229594692 -121.42699125952147) bank110459 +110460 POINT(48.27068991533258 -121.45904637770333) bank110460 +110461 POINT(46.88647954042976 -121.45754428185569) bank110461 +110462 POINT(46.73773966518941 -121.82752394616743) bank110462 +110463 POINT(47.74641835664288 -121.62371847507366) bank110463 +110464 POINT(48.53037892651073 -122.98597471350003) bank110464 +110465 POINT(46.9083103752689 -123.09757683396305) bank110465 +110466 POINT(46.65762780670876 -123.1991219879905) bank110466 +110467 POINT(46.80716548239913 -121.87589117810609) bank110467 +110468 POINT(46.71524168750224 -122.03432499130764) bank110468 +110469 POINT(46.80301763864399 -121.82208809965172) bank110469 +110470 POINT(46.75095412310466 -121.8858580842304) bank110470 +110471 POINT(46.767712797565224 -123.055336632317) bank110471 +110472 POINT(48.11802650636843 -121.78345781931895) bank110472 +110473 POINT(47.83642588130916 -121.90219237389641) bank110473 +110474 POINT(47.792842841091066 -121.36833537452969) bank110474 +110475 POINT(47.87075393506486 -123.01817987952164) bank110475 +110476 POINT(47.088882223803935 -122.2562249901858) bank110476 +110477 POINT(47.42932543167108 -121.48168252794281) bank110477 +110478 POINT(46.97113273892139 -122.300831110119) bank110478 +110479 POINT(48.527779528668006 -121.60780094811874) bank110479 +110480 POINT(48.199600023685136 -121.48347278126633) bank110480 +110481 POINT(47.175210362358726 -122.61151872532697) bank110481 +110482 POINT(46.94587712016605 -122.34715306710785) bank110482 +110483 POINT(47.618470844301015 -122.86617378629796) bank110483 +110484 POINT(47.56533802479885 -122.6628588701048) bank110484 +110485 POINT(47.64763712958446 -121.91941821215353) bank110485 +110486 POINT(47.68973831330366 -123.18613147124209) bank110486 +110487 POINT(48.556414465117385 -121.61889865654794) bank110487 +110488 POINT(47.7996946735075 -122.3919253923464) bank110488 +110489 POINT(47.477532970626655 -122.60596416573281) bank110489 +110490 POINT(46.95880007784869 -122.54454521452305) bank110490 +110491 POINT(48.16628412925794 -123.14797961366975) bank110491 +110492 POINT(48.51982621439535 -121.6002056928547) bank110492 +110493 POINT(47.6358272484641 -122.27677160566834) bank110493 +110494 POINT(48.29744988632908 -121.81327265170894) bank110494 +110495 POINT(47.150336638838134 -122.86966256715816) bank110495 +110496 POINT(48.064666042174174 -122.53736599846296) bank110496 +110497 POINT(48.00528643103557 -123.04065149846646) bank110497 +110498 POINT(46.8494991859586 -122.15289031872432) bank110498 +110499 POINT(47.61800381054763 -122.13000893595091) bank110499 +110500 POINT(46.615732041448425 -121.49450609778303) bank110500 +110501 POINT(46.83467906720037 -121.3850040280487) bank110501 +110502 POINT(48.535363122879254 -122.47728790147542) bank110502 +110503 POINT(48.15035545700081 -121.50332517816646) bank110503 +110504 POINT(46.92178857474692 -122.53625483499972) bank110504 +110505 POINT(46.93410989172129 -122.93374514919469) bank110505 +110506 POINT(47.91152950131773 -121.71245668073152) bank110506 +110507 POINT(47.121324940964655 -122.41418236779487) bank110507 +110508 POINT(46.620123888540135 -122.07968788096463) bank110508 +110509 POINT(47.75174430149382 -122.72178109275357) bank110509 +110510 POINT(47.533757592691465 -121.93695879841229) bank110510 +110511 POINT(47.805086868153666 -122.16121956394701) bank110511 +110512 POINT(47.72082242463308 -121.61752322224054) bank110512 +110513 POINT(48.20134741751494 -121.59999627742909) bank110513 +110514 POINT(47.14622434511629 -122.54796688782922) bank110514 +110515 POINT(46.81890952928463 -121.880611892327) bank110515 +110516 POINT(48.35170200818811 -122.92619393813703) bank110516 +110517 POINT(47.08696823939053 -121.56647689377354) bank110517 +110518 POINT(48.49702202630282 -123.2861629682524) bank110518 +110519 POINT(47.74090360803377 -123.02455318185672) bank110519 +110520 POINT(46.995187584182176 -122.04795511935986) bank110520 +110521 POINT(47.8901959215881 -122.18451309299518) bank110521 +110522 POINT(46.79340027938553 -122.96301664463392) bank110522 +110523 POINT(47.52061804602887 -122.68106998135906) bank110523 +110524 POINT(46.67447878643307 -122.14854491712465) bank110524 +110525 POINT(47.543312307930165 -122.1053616462532) bank110525 +110526 POINT(48.09647067101155 -122.35409196740734) bank110526 +110527 POINT(46.699106973551295 -122.15079548013406) bank110527 +110528 POINT(47.08082882731346 -123.16233805045187) bank110528 +110529 POINT(46.85772450789278 -122.98564179515209) bank110529 +110530 POINT(47.56424835703812 -121.5493143610567) bank110530 +110531 POINT(47.39834720345181 -121.59186189042354) bank110531 +110532 POINT(48.56958728869198 -121.9182307187733) bank110532 +110533 POINT(48.416756606101444 -122.80146224764947) bank110533 +110534 POINT(47.69515827027876 -122.53054769524749) bank110534 +110535 POINT(47.76615723021866 -121.57611936792087) bank110535 +110536 POINT(48.567760643986105 -122.92642803819977) bank110536 +110537 POINT(46.706780273892754 -121.565749414721) bank110537 +110538 POINT(48.011760218164696 -122.55737400982771) bank110538 +110539 POINT(47.41926372550834 -121.73669107539764) bank110539 +110540 POINT(47.24976195291193 -122.6237815484421) bank110540 +110541 POINT(47.08349098856382 -123.03174274329325) bank110541 +110542 POINT(47.38495776637012 -121.80035689964609) bank110542 +110543 POINT(47.62242248375354 -121.8183890537037) bank110543 +110544 POINT(47.12357647293959 -122.00969581619303) bank110544 +110545 POINT(48.52374053413923 -122.39945037157028) bank110545 +110546 POINT(46.77552001709623 -122.00874084361315) bank110546 +110547 POINT(47.02013749295562 -123.07398961221779) bank110547 +110548 POINT(47.06998187335276 -122.78226259341642) bank110548 +110549 POINT(47.491586042075525 -121.88118655721324) bank110549 +110550 POINT(47.01362869224387 -122.47332941548177) bank110550 +110551 POINT(47.24610495178533 -121.62506498298976) bank110551 +110552 POINT(47.32218405958575 -121.45945289871838) bank110552 +110553 POINT(47.53851731877005 -123.22499577892282) bank110553 +110554 POINT(46.78078115235874 -122.80198246314484) bank110554 +110555 POINT(47.41071239144258 -121.50652483025492) bank110555 +110556 POINT(46.7874987098178 -121.75313529682131) bank110556 +110557 POINT(48.154839767788 -122.60325686699062) bank110557 +110558 POINT(48.12997618270162 -121.89051386788297) bank110558 +110559 POINT(47.530160929787485 -122.79360095877216) bank110559 +110560 POINT(46.736071933500234 -122.62392860223419) bank110560 +110561 POINT(47.3552127917905 -122.20643590332058) bank110561 +110562 POINT(48.27254525494358 -122.57667865184995) bank110562 +110563 POINT(47.351884314749135 -121.47310583743986) bank110563 +110564 POINT(47.46794325128027 -121.39746887061885) bank110564 +110565 POINT(48.195704549466605 -123.28515171618432) bank110565 +110566 POINT(46.648035750094735 -122.79844342172564) bank110566 +110567 POINT(47.288198611264995 -122.84312510451565) bank110567 +110568 POINT(47.35085971303078 -121.7076726043257) bank110568 +110569 POINT(48.13788101970649 -121.40579055346545) bank110569 +110570 POINT(47.820274065385135 -123.27119520584836) bank110570 +110571 POINT(47.26350436533012 -122.61056668121512) bank110571 +110572 POINT(48.32301726092584 -122.72779284130893) bank110572 +110573 POINT(46.86351652107219 -123.0220344048637) bank110573 +110574 POINT(46.742382747764715 -123.21529762808603) bank110574 +110575 POINT(47.70249927876792 -121.55535954955482) bank110575 +110576 POINT(48.24119339940027 -122.14913858062555) bank110576 +110577 POINT(48.39335103351351 -121.38007499716402) bank110577 +110578 POINT(48.153116538179475 -123.11404110431019) bank110578 +110579 POINT(48.23550682846504 -121.8913893778223) bank110579 +110580 POINT(48.59157383678996 -122.1504230587551) bank110580 +110581 POINT(47.733336538456335 -122.9168200500502) bank110581 +110582 POINT(46.88589063494755 -122.38457835772272) bank110582 +110583 POINT(47.8758864746679 -121.87197898911715) bank110583 +110584 POINT(48.091776222783395 -122.60273127368065) bank110584 +110585 POINT(47.28962511485111 -121.90203149829982) bank110585 +110586 POINT(48.14956888033403 -122.72170957740916) bank110586 +110587 POINT(47.448765492571 -122.63146349070449) bank110587 +110588 POINT(47.09703272991935 -121.54347907575026) bank110588 +110589 POINT(47.38635198291673 -122.64322858611953) bank110589 +110590 POINT(47.25967426247018 -122.35747225784294) bank110590 +110591 POINT(47.962613793129364 -121.58004541597619) bank110591 +110592 POINT(47.91015410672777 -121.98284313018377) bank110592 +110593 POINT(46.74264727483939 -123.1935352088053) bank110593 +110594 POINT(46.690809516639455 -121.7481139337221) bank110594 +110595 POINT(47.53653187119177 -122.21516766389833) bank110595 +110596 POINT(48.56212431660414 -121.46297026357458) bank110596 +110597 POINT(47.11289734083512 -123.11339130362859) bank110597 +110598 POINT(48.46968912499196 -121.71402005436285) bank110598 +110599 POINT(48.18389709505337 -121.34560192452052) bank110599 +110600 POINT(47.569178003465794 -122.63614744491193) bank110600 +110601 POINT(46.768071855876656 -121.95566343625772) bank110601 +110602 POINT(47.42967861708628 -122.05017433136774) bank110602 +110603 POINT(46.79183236205806 -122.2106020714962) bank110603 +110604 POINT(48.436241383206266 -122.35799979227878) bank110604 +110605 POINT(46.657631779708126 -121.6164820686549) bank110605 +110606 POINT(48.01723757483763 -123.02552201371918) bank110606 +110607 POINT(46.63369708158194 -122.14993823856956) bank110607 +110608 POINT(46.61681610665421 -121.95723879299344) bank110608 +110609 POINT(48.338510519842934 -122.92280184191416) bank110609 +110610 POINT(46.843735808550015 -122.91355832232156) bank110610 +110611 POINT(48.20275195423977 -122.93188223789949) bank110611 +110612 POINT(48.124165991942675 -122.61259158831002) bank110612 +110613 POINT(48.474732630692024 -121.7339729617697) bank110613 +110614 POINT(46.77060974407161 -121.5522836480871) bank110614 +110615 POINT(47.663144540774184 -122.87391394845324) bank110615 +110616 POINT(47.04060412800504 -122.980816462483) bank110616 +110617 POINT(47.72266157057472 -121.78893337338994) bank110617 +110618 POINT(47.10293892098125 -122.34320653352513) bank110618 +110619 POINT(48.359034066922455 -121.66052184246199) bank110619 +110620 POINT(48.25621305257751 -121.39120940865661) bank110620 +110621 POINT(47.64695640765999 -122.82972097337763) bank110621 +110622 POINT(47.40741473835634 -121.46430266452381) bank110622 +110623 POINT(47.089026638087226 -123.1933405755682) bank110623 +110624 POINT(47.60933438185726 -122.70287866934954) bank110624 +110625 POINT(48.29108066608521 -123.05184888028944) bank110625 +110626 POINT(47.54124942813646 -121.34578036724925) bank110626 +110627 POINT(48.50867547528852 -121.98349022931251) bank110627 +110628 POINT(47.2378039650859 -122.37234289239194) bank110628 +110629 POINT(47.49826259280037 -122.5380951775226) bank110629 +110630 POINT(48.59783772395517 -121.79243731664725) bank110630 +110631 POINT(47.80800381107641 -123.15078987059218) bank110631 +110632 POINT(47.732807910209345 -123.14979465101418) bank110632 +110633 POINT(47.38993886465259 -122.57167096704686) bank110633 +110634 POINT(48.03748695153587 -121.57752176803962) bank110634 +110635 POINT(46.717914663692746 -123.1628733878345) bank110635 +110636 POINT(48.20104602022237 -121.86480328819758) bank110636 +110637 POINT(47.858558759009924 -121.35774348976477) bank110637 +110638 POINT(48.34330375295459 -122.33398821852303) bank110638 +110639 POINT(46.92024348329689 -122.45495859494419) bank110639 +110640 POINT(48.113396927099565 -121.67897937631191) bank110640 +110641 POINT(47.4104213263065 -122.88996328497468) bank110641 +110642 POINT(47.103913365493426 -122.92349311502518) bank110642 +110643 POINT(46.85354311736609 -122.25102146289892) bank110643 +110644 POINT(47.932587979257875 -122.82381194114045) bank110644 +110645 POINT(46.94905005344898 -122.72454091178399) bank110645 +110646 POINT(47.43283724451964 -122.07549234935748) bank110646 +110647 POINT(46.716704529151556 -121.39765063810053) bank110647 +110648 POINT(48.281257923958876 -121.45400344395279) bank110648 +110649 POINT(48.39460297674933 -122.1293435013406) bank110649 +110650 POINT(47.52064672609795 -121.97679549725022) bank110650 +110651 POINT(48.504402855616235 -122.90270318732284) bank110651 +110652 POINT(46.81936313329269 -121.94608877461954) bank110652 +110653 POINT(47.373477079521074 -122.57713852060922) bank110653 +110654 POINT(46.899109095720576 -121.81942059376087) bank110654 +110655 POINT(47.55668306468194 -121.70585056414653) bank110655 +110656 POINT(47.07559270062 -121.48202724952905) bank110656 +110657 POINT(47.816018030034236 -121.93885494797364) bank110657 +110658 POINT(48.44101486626219 -121.41898226226344) bank110658 +110659 POINT(47.47377364458068 -122.64880016828373) bank110659 +110660 POINT(47.920309665121565 -121.8134041941457) bank110660 +110661 POINT(46.86007045193131 -123.07472019919233) bank110661 +110662 POINT(47.24908930015349 -121.46413116228182) bank110662 +110663 POINT(48.60264284435381 -122.36312438222349) bank110663 +110664 POINT(48.36059479076351 -122.73394124717014) bank110664 +110665 POINT(47.614464476301876 -122.94921258835058) bank110665 +110666 POINT(47.79592684807387 -121.39035580246218) bank110666 +110667 POINT(48.158313036758095 -121.80793951452124) bank110667 +110668 POINT(46.7088832427101 -123.1632419886047) bank110668 +110669 POINT(46.81631794672226 -122.2191917861842) bank110669 +110670 POINT(47.53100639341472 -122.22707811157419) bank110670 +110671 POINT(47.89708025020977 -122.33671604514856) bank110671 +110672 POINT(47.74247636437772 -121.39014429807517) bank110672 +110673 POINT(48.453046093755226 -121.98687511154974) bank110673 +110674 POINT(46.96693533321023 -122.6946431558693) bank110674 +110675 POINT(48.34417057590017 -123.1988265136778) bank110675 +110676 POINT(48.13248645812969 -122.59915047981723) bank110676 +110677 POINT(47.53988130375374 -122.33452697939182) bank110677 +110678 POINT(47.62995435314574 -121.47383309033086) bank110678 +110679 POINT(48.34296605290325 -121.504384579561) bank110679 +110680 POINT(47.896326003666886 -122.46108702575532) bank110680 +110681 POINT(48.09015487222262 -121.44788802178567) bank110681 +110682 POINT(48.46794587702192 -123.27091248116996) bank110682 +110683 POINT(48.47442804757966 -123.14820181857722) bank110683 +110684 POINT(47.9907174933353 -122.2812558586809) bank110684 +110685 POINT(48.19524353258649 -123.27010856462218) bank110685 +110686 POINT(47.86143253263145 -122.76514097629972) bank110686 +110687 POINT(48.436719260118174 -121.92598443465135) bank110687 +110688 POINT(47.09265046901785 -122.07082694966726) bank110688 +110689 POINT(47.536496619984554 -121.440782494553) bank110689 +110690 POINT(46.880620975777134 -122.84659935059766) bank110690 +110691 POINT(47.62061459294404 -122.96116181204103) bank110691 +110692 POINT(47.70009863696584 -121.37885309507604) bank110692 +110693 POINT(47.80891174870127 -122.88863010205797) bank110693 +110694 POINT(46.98964751356169 -122.64029324522409) bank110694 +110695 POINT(47.40051533620406 -121.5123928756355) bank110695 +110696 POINT(48.174785317278236 -122.92830650890635) bank110696 +110697 POINT(46.65191503851126 -122.20041859019658) bank110697 +110698 POINT(47.19807788071122 -122.84095133973749) bank110698 +110699 POINT(47.06850771670748 -123.05522976805426) bank110699 +110700 POINT(46.73482409636462 -121.4512742050782) bank110700 +110701 POINT(46.8825063226784 -122.19401593851721) bank110701 +110702 POINT(47.707069596253056 -121.57022344721024) bank110702 +110703 POINT(46.66025799592661 -122.54117590394796) bank110703 +110704 POINT(47.442001691070224 -122.23186174869812) bank110704 +110705 POINT(46.63005872843014 -122.03801794193232) bank110705 +110706 POINT(46.84091913675439 -122.84720454566637) bank110706 +110707 POINT(47.849971734191975 -122.61239222558584) bank110707 +110708 POINT(48.58438276806415 -121.62222467954008) bank110708 +110709 POINT(47.72724837908515 -123.0076394096838) bank110709 +110710 POINT(46.974000091673155 -122.24139753726507) bank110710 +110711 POINT(46.80205705368225 -122.37770053925732) bank110711 +110712 POINT(48.024683116984455 -121.66405813088988) bank110712 +110713 POINT(47.4286815037128 -122.19555811652829) bank110713 +110714 POINT(47.68469681910529 -122.82098447332895) bank110714 +110715 POINT(48.587633133681834 -121.83520727191078) bank110715 +110716 POINT(47.952885774000386 -122.08550058704702) bank110716 +110717 POINT(47.2604294701329 -122.99615490111186) bank110717 +110718 POINT(47.8184692847883 -122.31352638846765) bank110718 +110719 POINT(48.05661410283245 -121.91176504765728) bank110719 +110720 POINT(48.102899846263895 -121.79859111747629) bank110720 +110721 POINT(46.79821435435572 -122.41852481859766) bank110721 +110722 POINT(46.76197363292395 -122.63618525020766) bank110722 +110723 POINT(47.80793653280041 -121.39540719523283) bank110723 +110724 POINT(48.431604150654685 -121.65442584888024) bank110724 +110725 POINT(48.51214510549341 -122.89766576993192) bank110725 +110726 POINT(47.63944824032105 -123.2777063928705) bank110726 +110727 POINT(48.27762062062351 -122.83617852819637) bank110727 +110728 POINT(47.48087553455502 -122.37331651784363) bank110728 +110729 POINT(47.643242937503565 -122.5433284872758) bank110729 +110730 POINT(47.620904480530285 -121.50007833432448) bank110730 +110731 POINT(47.44238468873338 -121.73691491648967) bank110731 +110732 POINT(47.563086836822315 -122.7771028099125) bank110732 +110733 POINT(47.30917921419199 -123.16348103096915) bank110733 +110734 POINT(47.71253208930813 -123.01814125977235) bank110734 +110735 POINT(48.04382731617949 -122.43794262691682) bank110735 +110736 POINT(47.96487984801045 -121.43104107819356) bank110736 +110737 POINT(46.73636517193432 -122.601770491585) bank110737 +110738 POINT(47.36228595825786 -121.60628338344434) bank110738 +110739 POINT(46.846363981591765 -122.82027750015996) bank110739 +110740 POINT(47.86891269234837 -122.50331525071324) bank110740 +110741 POINT(47.55783095455671 -121.73942801894113) bank110741 +110742 POINT(47.90836088916611 -122.85425619962467) bank110742 +110743 POINT(47.9601724397625 -122.88576220310851) bank110743 +110744 POINT(47.47685925853864 -121.4417347510592) bank110744 +110745 POINT(48.09666628324577 -121.41916570687675) bank110745 +110746 POINT(46.66082793648199 -123.24598418913233) bank110746 +110747 POINT(46.77316199231028 -122.70550756681091) bank110747 +110748 POINT(48.28992855613448 -122.63204181260743) bank110748 +110749 POINT(47.84082373661196 -123.0979869397323) bank110749 +110750 POINT(46.80433713936528 -122.61464673417801) bank110750 +110751 POINT(47.21191179473574 -122.12993478886402) bank110751 +110752 POINT(46.96395882052516 -123.01920631799268) bank110752 +110753 POINT(48.53851144626876 -121.58890311545814) bank110753 +110754 POINT(48.20709573989576 -122.54744016476644) bank110754 +110755 POINT(46.621373526543515 -123.25892705472467) bank110755 +110756 POINT(47.978631501884315 -121.73372313388894) bank110756 +110757 POINT(47.13470340719046 -122.48406687081595) bank110757 +110758 POINT(46.98883524127654 -122.82329782261142) bank110758 +110759 POINT(47.63544810467562 -121.45478411392719) bank110759 +110760 POINT(47.63930936004234 -122.39201040436312) bank110760 +110761 POINT(46.88089508624703 -121.86118371820118) bank110761 +110762 POINT(47.61333468713252 -121.97469134955479) bank110762 +110763 POINT(48.5397650929821 -121.64308420405006) bank110763 +110764 POINT(47.371110984723025 -123.13150206750753) bank110764 +110765 POINT(48.119942533889585 -123.16135321143013) bank110765 +110766 POINT(48.21723378659161 -122.096298797736) bank110766 +110767 POINT(48.198403298978214 -121.98570510415934) bank110767 +110768 POINT(47.51903033944005 -122.97133519795281) bank110768 +110769 POINT(47.134334609414324 -122.88709448544382) bank110769 +110770 POINT(46.90738810643717 -122.59444044295279) bank110770 +110771 POINT(47.260970467821274 -121.86731834453775) bank110771 +110772 POINT(48.33403136671508 -123.06634605670797) bank110772 +110773 POINT(48.34241936162619 -122.12234112563367) bank110773 +110774 POINT(47.547163954724624 -123.30878054469808) bank110774 +110775 POINT(46.793998741593654 -122.97952447285452) bank110775 +110776 POINT(47.90563681289366 -123.15028660857783) bank110776 +110777 POINT(48.60067392332651 -122.4841388364503) bank110777 +110778 POINT(48.49686485165836 -122.59885511104784) bank110778 +110779 POINT(48.24959814428352 -122.40176167069536) bank110779 +110780 POINT(48.202819290632156 -123.02797432336854) bank110780 +110781 POINT(48.293942457132026 -122.01069330592914) bank110781 +110782 POINT(47.27234200341517 -122.05987312030453) bank110782 +110783 POINT(48.16548543041879 -121.57298169942115) bank110783 +110784 POINT(46.703546306960604 -122.50188494182433) bank110784 +110785 POINT(47.879089656977094 -123.05107781733386) bank110785 +110786 POINT(46.93368618685079 -122.5426200298538) bank110786 +110787 POINT(47.40277959035607 -122.31798232105444) bank110787 +110788 POINT(47.053715182318356 -121.88050717080942) bank110788 +110789 POINT(46.674012655009086 -122.84106526566886) bank110789 +110790 POINT(47.66688763206135 -122.91228970774179) bank110790 +110791 POINT(47.287232321814955 -121.92839018704011) bank110791 +110792 POINT(48.19907406396759 -122.87582681095934) bank110792 +110793 POINT(47.49011739797145 -121.77189768500219) bank110793 +110794 POINT(46.88712620581174 -121.54587677470163) bank110794 +110795 POINT(47.839698727282 -122.40193888973343) bank110795 +110796 POINT(47.671069135173454 -122.08272873684462) bank110796 +110797 POINT(48.306477725541335 -121.73438912505637) bank110797 +110798 POINT(47.54099706801205 -123.10169077256356) bank110798 +110799 POINT(47.731640357008594 -121.93058098548184) bank110799 +110800 POINT(48.213889988188626 -121.84191086337367) bank110800 +110801 POINT(47.12853804863122 -122.51579570183375) bank110801 +110802 POINT(46.612226023792104 -121.35836904731607) bank110802 +110803 POINT(48.35062269374621 -122.33404217684573) bank110803 +110804 POINT(48.08384232636723 -121.74959010681737) bank110804 +110805 POINT(46.98159485118429 -121.83659655464045) bank110805 +110806 POINT(47.22156379533588 -121.58817599612684) bank110806 +110807 POINT(47.63461718434781 -122.11921150849287) bank110807 +110808 POINT(48.43334250458591 -122.074413137903) bank110808 +110809 POINT(46.99819089056916 -123.06258060140993) bank110809 +110810 POINT(47.5409613349988 -123.13631426289534) bank110810 +110811 POINT(48.59872948887047 -122.33265108477234) bank110811 +110812 POINT(48.16271272367123 -121.66710282647539) bank110812 +110813 POINT(46.955404642819985 -122.23388309230904) bank110813 +110814 POINT(47.186257603192104 -123.25289469815156) bank110814 +110815 POINT(48.218623060676265 -121.5537257930465) bank110815 +110816 POINT(47.83480480327653 -122.36799660112156) bank110816 +110817 POINT(47.45733593786697 -121.76758030355572) bank110817 +110818 POINT(48.34447496597565 -122.20911262341288) bank110818 +110819 POINT(48.510458612767195 -122.61331048288181) bank110819 +110820 POINT(46.64776543040391 -121.3507317989102) bank110820 +110821 POINT(47.36838554227147 -121.87093940788661) bank110821 +110822 POINT(47.76717537449593 -121.36629389766341) bank110822 +110823 POINT(47.023084217792984 -121.52439079789423) bank110823 +110824 POINT(46.84251592161632 -123.20085328163603) bank110824 +110825 POINT(48.01981415799407 -122.46472818508826) bank110825 +110826 POINT(48.05371560774369 -122.22715879019884) bank110826 +110827 POINT(48.191160580244166 -123.05967333656521) bank110827 +110828 POINT(46.90484757750478 -123.13576512077098) bank110828 +110829 POINT(47.959932336622536 -123.20111571862195) bank110829 +110830 POINT(47.697512363940724 -122.03044750412536) bank110830 +110831 POINT(48.15735215321095 -121.96507637876834) bank110831 +110832 POINT(47.32404510351856 -122.79966106265731) bank110832 +110833 POINT(48.534950944346186 -122.82221395191512) bank110833 +110834 POINT(46.789678563203225 -121.92608977934515) bank110834 +110835 POINT(47.69345236349088 -122.28821521047351) bank110835 +110836 POINT(46.90332739663966 -121.79444619446521) bank110836 +110837 POINT(47.61528501480552 -123.02148820777784) bank110837 +110838 POINT(48.15320805817635 -122.80007310890691) bank110838 +110839 POINT(47.23560181475652 -123.10312724650072) bank110839 +110840 POINT(46.93838144388211 -121.69322331122912) bank110840 +110841 POINT(48.3566983157994 -122.23528412600913) bank110841 +110842 POINT(48.21277085662504 -122.04475385847942) bank110842 +110843 POINT(48.160584153898014 -122.52582106956186) bank110843 +110844 POINT(47.18165182808289 -121.92171558993114) bank110844 +110845 POINT(46.63056103957595 -122.97958429208829) bank110845 +110846 POINT(47.02019172121013 -122.20024011917576) bank110846 +110847 POINT(46.72923696525283 -121.64967302904343) bank110847 +110848 POINT(48.324537935879675 -122.53728702097962) bank110848 +110849 POINT(47.30323102971714 -122.01516246534041) bank110849 +110850 POINT(48.47819893363876 -123.12167189661432) bank110850 +110851 POINT(47.304966566154 -123.02015004148323) bank110851 +110852 POINT(47.73886845594697 -121.81085479074939) bank110852 +110853 POINT(48.19635696346261 -121.39434643759294) bank110853 +110854 POINT(47.25998609100758 -121.6354720876142) bank110854 +110855 POINT(47.61146303657154 -122.56423801167153) bank110855 +110856 POINT(47.98328958868877 -122.27892781373416) bank110856 +110857 POINT(47.49488933017363 -122.86343817934673) bank110857 +110858 POINT(47.038196779741526 -122.52336106255775) bank110858 +110859 POINT(48.138707946798185 -121.69895544547452) bank110859 +110860 POINT(48.22438658661283 -122.80641121371873) bank110860 +110861 POINT(48.136678045519126 -123.27200688426883) bank110861 +110862 POINT(46.68356318442767 -122.44578392191161) bank110862 +110863 POINT(48.01576459956355 -121.45601839742206) bank110863 +110864 POINT(47.90097743731968 -122.49831560401276) bank110864 +110865 POINT(48.41653711459284 -122.85356394505925) bank110865 +110866 POINT(47.843801256585145 -123.03517671435748) bank110866 +110867 POINT(48.01287267873989 -122.34144181661723) bank110867 +110868 POINT(46.94361987903818 -121.80514068212031) bank110868 +110869 POINT(46.87324444803149 -122.636583453175) bank110869 +110870 POINT(47.472872407894414 -122.91434533241069) bank110870 +110871 POINT(47.4746600684669 -122.04216930668541) bank110871 +110872 POINT(47.49610460118281 -121.95461065565443) bank110872 +110873 POINT(48.52562747400934 -122.1906251049807) bank110873 +110874 POINT(47.06133881998334 -122.62922328831917) bank110874 +110875 POINT(47.50676468156015 -123.32085828872295) bank110875 +110876 POINT(46.778612333941126 -121.58490171189241) bank110876 +110877 POINT(46.75476972617848 -123.1764160798945) bank110877 +110878 POINT(46.6646129157338 -123.32610094735399) bank110878 +110879 POINT(46.714581227043524 -123.2485756859221) bank110879 +110880 POINT(47.89098768729689 -122.98052366565891) bank110880 +110881 POINT(46.70611664030536 -123.01831386059328) bank110881 +110882 POINT(47.30978971821534 -121.53728642162069) bank110882 +110883 POINT(47.7676781809979 -122.0671148368714) bank110883 +110884 POINT(47.609821085843755 -121.52885831059051) bank110884 +110885 POINT(46.965821298965864 -122.22565442911049) bank110885 +110886 POINT(47.534322298051386 -123.11350389284851) bank110886 +110887 POINT(48.291722111543464 -123.0100595729807) bank110887 +110888 POINT(47.68439667785712 -123.31035589063735) bank110888 +110889 POINT(47.69194651140295 -122.67337012304544) bank110889 +110890 POINT(48.51052204466472 -123.04424874394375) bank110890 +110891 POINT(47.57744337355414 -121.93962388516749) bank110891 +110892 POINT(48.1405643634339 -121.92902238769348) bank110892 +110893 POINT(48.16947276227838 -122.21357344941376) bank110893 +110894 POINT(47.28068439922459 -122.40493492183401) bank110894 +110895 POINT(48.36428720322724 -122.7461400293347) bank110895 +110896 POINT(47.39604937523154 -121.70263799884215) bank110896 +110897 POINT(47.94931683663777 -121.46831561703964) bank110897 +110898 POINT(48.38606988685868 -121.90845142359456) bank110898 +110899 POINT(47.22455840826775 -123.08348199065333) bank110899 +110900 POINT(48.39009625300863 -123.11544711392503) bank110900 +110901 POINT(47.405481217915906 -123.28856709826319) bank110901 +110902 POINT(47.86137672487408 -122.4413096481799) bank110902 +110903 POINT(47.36075867863842 -122.88149147362893) bank110903 +110904 POINT(47.94079994526392 -122.24426514680354) bank110904 +110905 POINT(47.83365685101526 -122.79017213233794) bank110905 +110906 POINT(46.90600346264701 -123.14152125851267) bank110906 +110907 POINT(47.01494920158608 -122.71720854014404) bank110907 +110908 POINT(47.66539715958878 -123.02647413683846) bank110908 +110909 POINT(47.72010118753739 -123.18252626385913) bank110909 +110910 POINT(47.75713210419208 -123.0395280472052) bank110910 +110911 POINT(47.262916356573164 -121.67487561399625) bank110911 +110912 POINT(48.00393776079939 -123.21602305262387) bank110912 +110913 POINT(46.67175455673482 -121.6205248767684) bank110913 +110914 POINT(48.57834765150585 -123.13635830391726) bank110914 +110915 POINT(47.40912296766663 -123.31161439603083) bank110915 +110916 POINT(47.728976228282 -121.33292617785496) bank110916 +110917 POINT(48.00165270462989 -121.7368316601551) bank110917 +110918 POINT(47.10415151621604 -121.68732875361457) bank110918 +110919 POINT(48.277835261932694 -122.24516543075843) bank110919 +110920 POINT(47.23570500238929 -122.6618857097237) bank110920 +110921 POINT(48.13613776850892 -121.56130402833924) bank110921 +110922 POINT(48.32265524390056 -121.99372031097919) bank110922 +110923 POINT(47.05441347451662 -122.69244703215637) bank110923 +110924 POINT(48.53465985952331 -122.62226831441656) bank110924 +110925 POINT(47.123678550993574 -123.07827816562076) bank110925 +110926 POINT(48.07765021338793 -122.64085979496632) bank110926 +110927 POINT(47.56559779918073 -122.84365939863655) bank110927 +110928 POINT(47.08169545211173 -122.15953107991712) bank110928 +110929 POINT(46.69367747822421 -122.85094208815529) bank110929 +110930 POINT(47.42705164054283 -122.76628199010695) bank110930 +110931 POINT(48.520031011746305 -122.06019157386856) bank110931 +110932 POINT(48.30569534563524 -121.98434447239012) bank110932 +110933 POINT(47.903761797122606 -122.6685864875932) bank110933 +110934 POINT(48.489556308945 -122.2555022026136) bank110934 +110935 POINT(47.67362504760107 -122.10841567017337) bank110935 +110936 POINT(47.18774865942468 -122.13266749846456) bank110936 +110937 POINT(46.76995520135792 -121.95554946214642) bank110937 +110938 POINT(48.17192692524674 -121.81248900285205) bank110938 +110939 POINT(47.744106018890946 -121.64596771101489) bank110939 +110940 POINT(48.06287257676656 -122.9546483220947) bank110940 +110941 POINT(47.148453451165885 -123.01018549503635) bank110941 +110942 POINT(47.91265885902954 -121.71870589702831) bank110942 +110943 POINT(47.79278407447677 -123.30924058734654) bank110943 +110944 POINT(47.61769558913149 -123.25263996860609) bank110944 +110945 POINT(48.54608863451467 -123.31223842529562) bank110945 +110946 POINT(47.046962549006324 -122.89295805186288) bank110946 +110947 POINT(48.29454983295535 -122.36811806293117) bank110947 +110948 POINT(47.79642890199835 -123.14172372837302) bank110948 +110949 POINT(47.56695602451346 -122.0242793713271) bank110949 +110950 POINT(48.35823796076657 -122.56790385235294) bank110950 +110951 POINT(46.83596571931872 -121.78682875939147) bank110951 +110952 POINT(47.08504894578421 -121.71416731406374) bank110952 +110953 POINT(48.46241334149488 -122.95585683650057) bank110953 +110954 POINT(47.25460968647421 -121.62655976422255) bank110954 +110955 POINT(47.35053552084607 -121.651143504991) bank110955 +110956 POINT(48.025350947271804 -122.04755646359206) bank110956 +110957 POINT(47.0344836717821 -122.407832904238) bank110957 +110958 POINT(47.64785840843526 -122.98503002300713) bank110958 +110959 POINT(46.721751012940956 -122.21167532681989) bank110959 +110960 POINT(47.33710964677701 -122.88853799009166) bank110960 +110961 POINT(47.17688694016649 -122.90385452044036) bank110961 +110962 POINT(47.14265545021344 -122.21353248014454) bank110962 +110963 POINT(47.104293673928474 -122.32994614187275) bank110963 +110964 POINT(47.08240916970319 -122.77374575476769) bank110964 +110965 POINT(47.55744067212359 -122.86577831803162) bank110965 +110966 POINT(48.4578819640018 -121.52731807199437) bank110966 +110967 POINT(46.71070757976602 -122.79643034985585) bank110967 +110968 POINT(48.10385085813691 -123.06905815260177) bank110968 +110969 POINT(48.233977674487704 -121.4221208379674) bank110969 +110970 POINT(48.5695506486113 -122.9036506552105) bank110970 +110971 POINT(46.6660925379347 -122.61296827673178) bank110971 +110972 POINT(48.09224181890208 -122.66318549225014) bank110972 +110973 POINT(46.75317696236659 -122.36388877329928) bank110973 +110974 POINT(47.99849702404956 -123.2155657296438) bank110974 +110975 POINT(46.61623695797743 -122.56308659794604) bank110975 +110976 POINT(47.43550331570836 -122.9229166361399) bank110976 +110977 POINT(47.88044481659285 -121.55006640956623) bank110977 +110978 POINT(48.17291238958581 -122.93664828022632) bank110978 +110979 POINT(47.650119664306374 -122.65569626054217) bank110979 +110980 POINT(48.374941852237114 -122.00523841186805) bank110980 +110981 POINT(46.70987047158016 -122.88750101426227) bank110981 +110982 POINT(48.517873579953935 -121.68512353751424) bank110982 +110983 POINT(47.6074128111123 -121.5414401523053) bank110983 +110984 POINT(48.53692022183223 -122.20661567721898) bank110984 +110985 POINT(46.91778768070124 -121.62953198439428) bank110985 +110986 POINT(48.245843184750285 -122.60496181842198) bank110986 +110987 POINT(47.838691755369396 -121.33763602175276) bank110987 +110988 POINT(48.09337198947419 -122.94060267228839) bank110988 +110989 POINT(47.89836623993314 -122.63088719903816) bank110989 +110990 POINT(48.03208913717736 -122.65919271636511) bank110990 +110991 POINT(47.126534081391746 -122.09010986113466) bank110991 +110992 POINT(48.18360512790889 -122.99872704812894) bank110992 +110993 POINT(48.368578714959014 -123.11551276420596) bank110993 +110994 POINT(47.84197115986917 -121.55952059399232) bank110994 +110995 POINT(47.20590393428951 -121.47468859733833) bank110995 +110996 POINT(47.27761139287915 -122.803540014431) bank110996 +110997 POINT(47.88494115334315 -121.34499422663482) bank110997 +110998 POINT(47.260288349192706 -122.27620260777344) bank110998 +110999 POINT(48.04856526546653 -123.22758736247178) bank110999 +111000 POINT(47.279551573574174 -122.88809395817262) bank111000 +111001 POINT(47.360906583752616 -122.97700752412301) bank111001 +111002 POINT(47.004618782301286 -122.24326436825488) bank111002 +111003 POINT(48.28036553825393 -122.30648062697676) bank111003 +111004 POINT(48.493936949007534 -121.71888669319202) bank111004 +111005 POINT(46.82002716226101 -122.18013266544004) bank111005 +111006 POINT(48.260390330171106 -122.90770301917932) bank111006 +111007 POINT(48.277325829263496 -121.39861564443358) bank111007 +111008 POINT(47.65592162119703 -122.63192685206407) bank111008 +111009 POINT(47.37825127766043 -122.46096343991098) bank111009 +111010 POINT(48.479828841198994 -121.35858286954506) bank111010 +111011 POINT(47.79894865975694 -121.51860932858685) bank111011 +111012 POINT(47.53306746655502 -121.98881816633332) bank111012 +111013 POINT(46.73083529822333 -122.90676086788642) bank111013 +111014 POINT(46.69824970798854 -122.18144389542677) bank111014 +111015 POINT(46.93487252517147 -122.2498167162462) bank111015 +111016 POINT(47.56594326029146 -121.54612125798708) bank111016 +111017 POINT(47.11810219467773 -123.10586403643683) bank111017 +111018 POINT(48.13422235965083 -123.013243611247) bank111018 +111019 POINT(48.140344568850296 -123.1892422869219) bank111019 +111020 POINT(48.56638488636516 -122.29043185355361) bank111020 +111021 POINT(47.50843345578883 -122.87710808077674) bank111021 +111022 POINT(47.268001900594506 -122.57693393165607) bank111022 +111023 POINT(46.65367370967857 -123.12802256437706) bank111023 +111024 POINT(47.890205543027896 -121.95958050892101) bank111024 +111025 POINT(48.31926328469616 -122.25906789883064) bank111025 +111026 POINT(46.99429385976653 -123.02437964972306) bank111026 +111027 POINT(48.60156029614523 -122.04273133852097) bank111027 +111028 POINT(48.16756393406099 -121.41896376923471) bank111028 +111029 POINT(48.301100442021784 -121.48181125414139) bank111029 +111030 POINT(47.579128564931835 -122.52606747942231) bank111030 +111031 POINT(47.95389190788449 -121.9694022209232) bank111031 +111032 POINT(48.533067879663385 -121.58283802019645) bank111032 +111033 POINT(47.88592195301661 -122.09165322246214) bank111033 +111034 POINT(47.772500621008206 -122.53135643335622) bank111034 +111035 POINT(47.65693163937353 -123.14258971792763) bank111035 +111036 POINT(46.885740802720946 -123.19000903297513) bank111036 +111037 POINT(48.047656897966796 -122.44474341269489) bank111037 +111038 POINT(48.47508752885546 -121.52583867838632) bank111038 +111039 POINT(47.738519606394824 -122.34454953747652) bank111039 +111040 POINT(46.73185146303015 -122.54344313467529) bank111040 +111041 POINT(48.33725647480893 -121.94811866462682) bank111041 +111042 POINT(47.911271735963936 -121.33299953399396) bank111042 +111043 POINT(46.82194638225328 -122.06473243947286) bank111043 +111044 POINT(47.382557917092605 -121.43392405599333) bank111044 +111045 POINT(47.86325591840061 -121.77391786583078) bank111045 +111046 POINT(47.61334660391775 -121.76306596443895) bank111046 +111047 POINT(46.887311357122115 -121.86548567944992) bank111047 +111048 POINT(47.654349528847575 -122.1700759412787) bank111048 +111049 POINT(47.45023431588962 -122.05097064667603) bank111049 +111050 POINT(48.37853717516838 -122.78459645985632) bank111050 +111051 POINT(46.688412614170744 -122.80177616874495) bank111051 +111052 POINT(47.83280512455852 -122.27003503546489) bank111052 +111053 POINT(48.24646994615971 -122.94957586910647) bank111053 +111054 POINT(48.09309406155783 -122.53444066116666) bank111054 +111055 POINT(47.1055598989874 -122.22020843551877) bank111055 +111056 POINT(47.909245906829334 -121.80898853460167) bank111056 +111057 POINT(47.53178411481708 -122.86324134828084) bank111057 +111058 POINT(48.007122131172274 -121.6674925197953) bank111058 +111059 POINT(47.516737940850454 -122.15090934904927) bank111059 +111060 POINT(47.242427833437404 -122.46317437642375) bank111060 +111061 POINT(46.897070883018735 -121.72780249290847) bank111061 +111062 POINT(46.7899634347689 -122.21523978936408) bank111062 +111063 POINT(46.862063541466384 -123.2145875645908) bank111063 +111064 POINT(46.887696158005895 -122.4274637860716) bank111064 +111065 POINT(48.097006302766886 -121.7109043419944) bank111065 +111066 POINT(46.62536323886329 -123.11952810659855) bank111066 +111067 POINT(48.30901735694588 -122.88992588105798) bank111067 +111068 POINT(47.7079179271755 -121.54419044314834) bank111068 +111069 POINT(48.337938791874464 -121.47838111300736) bank111069 +111070 POINT(47.940043882863144 -122.54809159856197) bank111070 +111071 POINT(47.33309927203906 -122.18305540629014) bank111071 +111072 POINT(48.40634890782019 -122.04880321399546) bank111072 +111073 POINT(47.955465063579496 -123.24001966794813) bank111073 +111074 POINT(46.733245186191226 -121.37543061028667) bank111074 +111075 POINT(47.08708986742894 -122.17034550373423) bank111075 +111076 POINT(46.91903169818705 -121.5424754860305) bank111076 +111077 POINT(47.75683759165884 -121.78185193043228) bank111077 +111078 POINT(47.73174702931469 -122.95799866744949) bank111078 +111079 POINT(47.27899336403712 -122.65389577256906) bank111079 +111080 POINT(48.0393401303397 -122.15636900367176) bank111080 +111081 POINT(47.422163605164215 -122.72327043447079) bank111081 +111082 POINT(46.61935025365491 -121.48320896981394) bank111082 +111083 POINT(48.15505914793124 -123.18469759117622) bank111083 +111084 POINT(47.016514295634785 -122.68546056631263) bank111084 +111085 POINT(47.14786684135657 -122.02887263859368) bank111085 +111086 POINT(47.84971865076495 -121.61518357139956) bank111086 +111087 POINT(46.62196589773009 -122.60369047794413) bank111087 +111088 POINT(47.30935222586465 -122.85143951727028) bank111088 +111089 POINT(48.22637076253995 -122.32492934838227) bank111089 +111090 POINT(46.849263981930335 -122.23619778870386) bank111090 +111091 POINT(47.56209493160074 -123.17216782812929) bank111091 +111092 POINT(48.359387875497866 -121.81690157900157) bank111092 +111093 POINT(46.61443485535269 -123.20081271427098) bank111093 +111094 POINT(48.36267287170764 -122.1001778907123) bank111094 +111095 POINT(47.74998415392216 -121.6416618884455) bank111095 +111096 POINT(48.0698090377221 -121.89254817717928) bank111096 +111097 POINT(48.15628383754958 -121.49485461594833) bank111097 +111098 POINT(48.47850603697002 -121.43171089459602) bank111098 +111099 POINT(46.75949533578756 -122.05136026407769) bank111099 +111100 POINT(47.73767278701681 -122.22456753193401) bank111100 +111101 POINT(47.01747627605193 -122.06227731754261) bank111101 +111102 POINT(48.345265959653126 -123.10201748568689) bank111102 +111103 POINT(47.03237418608774 -122.91298915989071) bank111103 +111104 POINT(48.47446253415973 -122.25610433930666) bank111104 +111105 POINT(48.54315065042741 -122.96620838919102) bank111105 +111106 POINT(46.79465448670232 -122.55006232321168) bank111106 +111107 POINT(46.63515380381042 -122.55693942236836) bank111107 +111108 POINT(48.580586921692415 -121.74094370253339) bank111108 +111109 POINT(47.720964129439395 -123.12323609600968) bank111109 +111110 POINT(47.509560308506586 -122.83423921074537) bank111110 +111111 POINT(47.38992621724034 -122.42786179934357) bank111111 +111112 POINT(48.57219037169717 -123.08800546427697) bank111112 +111113 POINT(48.115269938877496 -121.68689365278854) bank111113 +111114 POINT(48.499297304313885 -123.10161767199627) bank111114 +111115 POINT(47.98297209226737 -121.40688809704567) bank111115 +111116 POINT(47.866316872398315 -122.16247004941785) bank111116 +111117 POINT(47.75342716120339 -122.07410955884242) bank111117 +111118 POINT(47.360504848302426 -121.91108460012319) bank111118 +111119 POINT(47.9777085286987 -123.30274747048858) bank111119 +111120 POINT(48.48811890985313 -122.3543183385793) bank111120 +111121 POINT(48.091438364593124 -121.98583837658735) bank111121 +111122 POINT(47.70157842205703 -122.5157156825035) bank111122 +111123 POINT(47.76391432820004 -122.07478381376427) bank111123 +111124 POINT(46.881000083423 -123.00399818839884) bank111124 +111125 POINT(48.403929103151455 -122.73669090474154) bank111125 +111126 POINT(47.85574797730036 -122.95301263431826) bank111126 +111127 POINT(46.807009681203894 -122.61979763642957) bank111127 +111128 POINT(48.17459169200005 -123.27380424502533) bank111128 +111129 POINT(46.766012242614 -122.03610995724081) bank111129 +111130 POINT(47.644160467873604 -122.3563114339324) bank111130 +111131 POINT(47.77522019366029 -121.52670365081167) bank111131 +111132 POINT(47.35706330096861 -121.69330390565229) bank111132 +111133 POINT(47.324169531773016 -123.2826459813895) bank111133 +111134 POINT(48.12690836455419 -122.62052540251818) bank111134 +111135 POINT(47.739542483179264 -123.12205250384996) bank111135 +111136 POINT(48.336349401582915 -122.45588685038699) bank111136 +111137 POINT(47.41084847834698 -121.6600025928912) bank111137 +111138 POINT(48.20782463296881 -122.54542142480867) bank111138 +111139 POINT(47.575013477235814 -122.40069256346915) bank111139 +111140 POINT(46.98257698496589 -121.7059124433063) bank111140 +111141 POINT(47.09445525026755 -123.10544112424044) bank111141 +111142 POINT(47.7257311088924 -121.9195592134728) bank111142 +111143 POINT(48.15290466568197 -121.4892502068075) bank111143 +111144 POINT(47.21928689187046 -123.18137225036708) bank111144 +111145 POINT(48.37960914739043 -121.3814060865026) bank111145 +111146 POINT(47.53526854471243 -121.66507294507075) bank111146 +111147 POINT(48.166415869987595 -122.67500447097996) bank111147 +111148 POINT(47.375370412294885 -121.56971869739534) bank111148 +111149 POINT(46.8293624172157 -121.75013369668808) bank111149 +111150 POINT(46.86000792912905 -122.74029211036583) bank111150 +111151 POINT(47.53677283358121 -122.74865117193282) bank111151 +111152 POINT(47.09777312328261 -122.2124826252794) bank111152 +111153 POINT(48.10358743675965 -121.93687816100147) bank111153 +111154 POINT(47.01870152491195 -121.54074724408144) bank111154 +111155 POINT(48.517951619933186 -122.60166400835605) bank111155 +111156 POINT(46.946630967999994 -123.3257562679962) bank111156 +111157 POINT(48.60011212500165 -122.7511639132403) bank111157 +111158 POINT(48.421747560626045 -122.70306004059293) bank111158 +111159 POINT(46.76983663446103 -121.69396468167331) bank111159 +111160 POINT(46.643962152939636 -121.90476209803755) bank111160 +111161 POINT(48.19056817307978 -121.93700585809962) bank111161 +111162 POINT(46.77383501031465 -122.63160506113367) bank111162 +111163 POINT(47.882076546893124 -121.86992176784065) bank111163 +111164 POINT(46.777076511616364 -122.4455061780052) bank111164 +111165 POINT(47.002620624138835 -122.79346424880643) bank111165 +111166 POINT(46.9365691901535 -121.8486000118289) bank111166 +111167 POINT(47.332694253841105 -121.39580363109701) bank111167 +111168 POINT(47.93769004706943 -121.92246140475068) bank111168 +111169 POINT(47.752598096974616 -121.39473618154682) bank111169 +111170 POINT(47.0619374798971 -122.15857106136056) bank111170 +111171 POINT(46.92457938628418 -121.62310435805028) bank111171 +111172 POINT(47.34774707774185 -121.37837814380075) bank111172 +111173 POINT(47.823104399652884 -121.59390801937579) bank111173 +111174 POINT(47.2003872321815 -121.43253815888151) bank111174 +111175 POINT(46.84369432130085 -123.05520466501768) bank111175 +111176 POINT(48.49447762976884 -122.0184535644069) bank111176 +111177 POINT(47.62621127757708 -122.32792452310683) bank111177 +111178 POINT(47.16255721559977 -121.42399543080442) bank111178 +111179 POINT(46.68983581163499 -121.60766078050048) bank111179 +111180 POINT(48.336208502216174 -122.14026328990846) bank111180 +111181 POINT(47.13105975678359 -122.77400231543298) bank111181 +111182 POINT(47.84051563720511 -122.83992491945617) bank111182 +111183 POINT(46.63972453981836 -123.05212087460929) bank111183 +111184 POINT(47.760624133309136 -122.55846541675403) bank111184 +111185 POINT(47.1585797539025 -121.48215110786074) bank111185 +111186 POINT(46.7812902825812 -123.07336471705005) bank111186 +111187 POINT(46.615577018156834 -121.6716274431999) bank111187 +111188 POINT(46.80709779224826 -121.84635380500613) bank111188 +111189 POINT(48.46955100475311 -123.20333557354002) bank111189 +111190 POINT(46.74377924361945 -121.3897854831882) bank111190 +111191 POINT(48.04839864895047 -121.97386087921251) bank111191 +111192 POINT(46.68917914538994 -123.09467048009178) bank111192 +111193 POINT(46.93860956371751 -123.18981773437818) bank111193 +111194 POINT(47.71651634837966 -122.63695761770948) bank111194 +111195 POINT(48.47980545671934 -123.03290526962384) bank111195 +111196 POINT(48.36125282980903 -121.99299482838578) bank111196 +111197 POINT(46.97477971169371 -122.77068926400517) bank111197 +111198 POINT(47.15902358088721 -122.07845368359136) bank111198 +111199 POINT(47.959457898815806 -123.12396455953011) bank111199 +111200 POINT(46.93069272314957 -123.28978204900112) bank111200 +111201 POINT(47.64661866144109 -121.51684120963588) bank111201 +111202 POINT(48.47987900563253 -122.02643073587063) bank111202 +111203 POINT(48.20128725197107 -122.13199605563717) bank111203 +111204 POINT(48.31404789043017 -121.71589119312078) bank111204 +111205 POINT(47.53539947267819 -122.89188564136556) bank111205 +111206 POINT(46.68810675517878 -122.63068896873105) bank111206 +111207 POINT(47.6094662896705 -122.00887581008101) bank111207 +111208 POINT(47.13336556993145 -122.12556357111869) bank111208 +111209 POINT(47.86012371966992 -122.23441658661784) bank111209 +111210 POINT(47.929960888780855 -122.40984191442467) bank111210 +111211 POINT(48.26323368031556 -123.27802506186967) bank111211 +111212 POINT(47.018081365022965 -122.5523195115123) bank111212 +111213 POINT(48.07386106897372 -122.31796602987325) bank111213 +111214 POINT(47.92822369859989 -122.93589257354554) bank111214 +111215 POINT(48.27904844342228 -122.37204664911566) bank111215 +111216 POINT(46.687195664668366 -122.78396653848937) bank111216 +111217 POINT(47.971402807347125 -121.63512973735584) bank111217 +111218 POINT(48.209372838964754 -123.30108095307027) bank111218 +111219 POINT(48.18030632573135 -123.32778370438659) bank111219 +111220 POINT(47.39162953532384 -121.71969068253989) bank111220 +111221 POINT(46.898916636274734 -123.21241988895483) bank111221 +111222 POINT(46.908234477236746 -122.66781691957851) bank111222 +111223 POINT(47.94971665921952 -121.49874441717952) bank111223 +111224 POINT(48.024396313310106 -123.04526097849141) bank111224 +111225 POINT(48.46467730783271 -122.32971676920482) bank111225 +111226 POINT(47.4179889792062 -122.60507162447922) bank111226 +111227 POINT(48.524039709903775 -121.60614695487872) bank111227 +111228 POINT(46.93801865520692 -122.57779896041482) bank111228 +111229 POINT(47.23848104053389 -123.20640323743368) bank111229 +111230 POINT(46.79591874829006 -122.84256871809627) bank111230 +111231 POINT(48.56415869615955 -122.62260785046416) bank111231 +111232 POINT(48.01968800077772 -122.93490855686723) bank111232 +111233 POINT(47.704159271867745 -121.69128106156089) bank111233 +111234 POINT(47.40718550515993 -122.94667067141653) bank111234 +111235 POINT(47.646484388580255 -122.34556535049026) bank111235 +111236 POINT(47.06209406994877 -123.01382693255897) bank111236 +111237 POINT(47.84361899263736 -122.05505175700313) bank111237 +111238 POINT(46.98011840691129 -122.56527080479293) bank111238 +111239 POINT(47.83206573405211 -122.18007952606912) bank111239 +111240 POINT(47.23626971982737 -121.75011095008513) bank111240 +111241 POINT(48.50077566263636 -122.40977484224585) bank111241 +111242 POINT(47.087759817144644 -123.10066597516929) bank111242 +111243 POINT(47.55871809971735 -122.78247515582609) bank111243 +111244 POINT(48.341370669713655 -121.80522049035004) bank111244 +111245 POINT(47.50555596290725 -122.14550531472976) bank111245 +111246 POINT(47.285387503826065 -122.9409437996054) bank111246 +111247 POINT(47.99584302449498 -123.2160589252824) bank111247 +111248 POINT(47.375225983907 -122.96575055568262) bank111248 +111249 POINT(48.17825502973885 -121.81628536110225) bank111249 +111250 POINT(46.860434276170864 -121.53947459240904) bank111250 +111251 POINT(48.25011627669958 -123.21010020754058) bank111251 +111252 POINT(46.88556538571206 -123.11287197185217) bank111252 +111253 POINT(47.924510552138436 -121.93561695444856) bank111253 +111254 POINT(46.71228418080709 -122.53088297406885) bank111254 +111255 POINT(47.84156749314343 -122.23810737266116) bank111255 +111256 POINT(47.8885872807673 -121.63491849293518) bank111256 +111257 POINT(47.93445928351744 -122.80702823052492) bank111257 +111258 POINT(46.71004929245553 -123.17299643670644) bank111258 +111259 POINT(47.098740376612184 -122.14308050210448) bank111259 +111260 POINT(47.84141217238054 -122.90624946591417) bank111260 +111261 POINT(47.77048335250472 -121.62334019971632) bank111261 +111262 POINT(47.32199931357704 -122.54182807720059) bank111262 +111263 POINT(46.613126055034655 -123.1094819845946) bank111263 +111264 POINT(46.67499614044077 -122.21796679603722) bank111264 +111265 POINT(46.75779737574929 -123.25159102438197) bank111265 +111266 POINT(47.13122413358852 -122.44293866844505) bank111266 +111267 POINT(48.55963791831764 -122.11587967971244) bank111267 +111268 POINT(47.39786024205094 -121.47577860040943) bank111268 +111269 POINT(47.024906268885275 -121.69299169327904) bank111269 +111270 POINT(46.84570815159809 -122.38599778825485) bank111270 +111271 POINT(47.00868430593657 -122.5153529105404) bank111271 +111272 POINT(46.94136092723448 -123.01763873939053) bank111272 +111273 POINT(46.900929733354374 -123.03622308102432) bank111273 +111274 POINT(47.171728083967736 -123.30337277359682) bank111274 +111275 POINT(47.682405764342775 -122.42146211619739) bank111275 +111276 POINT(48.53561087763498 -122.91231467390277) bank111276 +111277 POINT(47.173537474567844 -123.25132823148303) bank111277 +111278 POINT(47.234317695327576 -121.4038402686309) bank111278 +111279 POINT(46.983760540752485 -122.50789277105804) bank111279 +111280 POINT(48.18210034970948 -121.94609909058192) bank111280 +111281 POINT(48.271674815348256 -123.05467290347278) bank111281 +111282 POINT(47.088827665040924 -123.050233403273) bank111282 +111283 POINT(47.64092624419699 -121.36585158337553) bank111283 +111284 POINT(48.50820386923614 -121.58025254025821) bank111284 +111285 POINT(47.19190842873865 -122.24739527488964) bank111285 +111286 POINT(47.16966219879229 -122.60621620967834) bank111286 +111287 POINT(47.310750381373815 -121.71423138496782) bank111287 +111288 POINT(46.9719042032816 -121.81536067015027) bank111288 +111289 POINT(48.42426961657361 -122.98125815725719) bank111289 +111290 POINT(48.52423756805165 -121.92666245185711) bank111290 +111291 POINT(47.58844311440735 -121.55081682640973) bank111291 +111292 POINT(47.678380375548464 -121.8615724996012) bank111292 +111293 POINT(46.92487620929356 -121.73796537477804) bank111293 +111294 POINT(47.62362736135429 -122.6988539659637) bank111294 +111295 POINT(46.936574213933675 -121.52850359492776) bank111295 +111296 POINT(47.33443322329471 -121.82350208752362) bank111296 +111297 POINT(46.64216251691522 -122.41760416741646) bank111297 +111298 POINT(48.05123755112269 -121.7574692057826) bank111298 +111299 POINT(47.70733604737874 -123.15470945435564) bank111299 +111300 POINT(47.44272698274017 -122.3666321498581) bank111300 +111301 POINT(48.445047163760876 -122.53364112738737) bank111301 +111302 POINT(47.93164989079213 -121.50856438790755) bank111302 +111303 POINT(47.86649549408505 -123.19658074381897) bank111303 +111304 POINT(46.967598703564455 -122.64359606489347) bank111304 +111305 POINT(46.81966073903376 -123.11184151078155) bank111305 +111306 POINT(48.35860465335786 -121.62197639310945) bank111306 +111307 POINT(48.40236112633081 -122.23739098765832) bank111307 +111308 POINT(47.28894068055315 -121.45262468835068) bank111308 +111309 POINT(47.39116047805646 -122.10574064900821) bank111309 +111310 POINT(46.99767306184513 -121.57902015216706) bank111310 +111311 POINT(46.694468398556 -121.44017670424823) bank111311 +111312 POINT(48.07586300803486 -122.59724800214971) bank111312 +111313 POINT(48.385503664294596 -122.3675560940919) bank111313 +111314 POINT(47.3431032351883 -122.62711771910375) bank111314 +111315 POINT(48.5933054809642 -123.01525688783696) bank111315 +111316 POINT(48.53311834277433 -122.10809618614303) bank111316 +111317 POINT(48.22099817058742 -122.54165634010009) bank111317 +111318 POINT(48.33607658573245 -121.757699833456) bank111318 +111319 POINT(46.762249570403405 -122.50425880911085) bank111319 +111320 POINT(47.557617232130184 -121.52389540125897) bank111320 +111321 POINT(47.76408912241908 -121.52322954232015) bank111321 +111322 POINT(46.652102363361 -121.41987171751603) bank111322 +111323 POINT(47.67553280535584 -122.49389996067892) bank111323 +111324 POINT(47.70915923356385 -122.31009135776912) bank111324 +111325 POINT(47.66636948486628 -122.17337311103556) bank111325 +111326 POINT(47.411372656938305 -122.52953018100939) bank111326 +111327 POINT(47.6513261787766 -123.23419760056198) bank111327 +111328 POINT(47.39668386321733 -122.0847492865177) bank111328 +111329 POINT(47.59381916236502 -121.89907322736482) bank111329 +111330 POINT(48.077059448378044 -122.79969932205863) bank111330 +111331 POINT(47.7578760327208 -123.24592696586365) bank111331 +111332 POINT(47.84769565314612 -123.2885607362977) bank111332 +111333 POINT(47.420741441927795 -122.4643078512211) bank111333 +111334 POINT(46.623940654143155 -122.42101259151285) bank111334 +111335 POINT(48.064026401814466 -121.81665981441007) bank111335 +111336 POINT(48.559536535984726 -123.00434821348696) bank111336 +111337 POINT(47.1196379881636 -123.30907262190304) bank111337 +111338 POINT(47.49793557813736 -122.5978540969789) bank111338 +111339 POINT(46.63743211920852 -122.02605624534132) bank111339 +111340 POINT(47.31789921981696 -121.93181403562967) bank111340 +111341 POINT(46.86922034248225 -122.57246709931651) bank111341 +111342 POINT(48.20130092776222 -121.501666900609) bank111342 +111343 POINT(48.43801786731525 -122.00910374157814) bank111343 +111344 POINT(48.50822052868367 -122.96194777748518) bank111344 +111345 POINT(47.98739404800891 -122.8232484396733) bank111345 +111346 POINT(48.52225140882936 -122.3514448795676) bank111346 +111347 POINT(48.0776344859278 -121.46758397108935) bank111347 +111348 POINT(48.441299225151205 -122.61204436554004) bank111348 +111349 POINT(48.27832264174649 -121.88244876743374) bank111349 +111350 POINT(46.81843615433145 -121.5254894092021) bank111350 +111351 POINT(48.441979608067975 -123.29584307832582) bank111351 +111352 POINT(47.522101484881006 -121.76455022145743) bank111352 +111353 POINT(47.46256937904338 -122.8184980946869) bank111353 +111354 POINT(47.31854146367761 -121.38513889937578) bank111354 +111355 POINT(47.036537366446275 -121.98762995362537) bank111355 +111356 POINT(47.98599328559926 -122.85183506291206) bank111356 +111357 POINT(46.61275797497422 -121.88619153343603) bank111357 +111358 POINT(47.82329597887211 -122.53597699858486) bank111358 +111359 POINT(46.72857952546471 -123.02418138753036) bank111359 +111360 POINT(47.936713686052684 -121.66095557853359) bank111360 +111361 POINT(48.20410306198218 -121.8374820763796) bank111361 +111362 POINT(48.12833380875685 -122.40994921006309) bank111362 +111363 POINT(46.89142378370696 -122.7037005880879) bank111363 +111364 POINT(46.898768996257566 -122.37426316152074) bank111364 +111365 POINT(46.851553840524375 -121.87294959437696) bank111365 +111366 POINT(46.923094760461574 -121.35409592832498) bank111366 +111367 POINT(47.15459412447612 -121.69366158711509) bank111367 +111368 POINT(47.76758105819905 -121.57205502846006) bank111368 +111369 POINT(48.20156993958683 -122.130514623501) bank111369 +111370 POINT(46.96556621647091 -122.28773889017464) bank111370 +111371 POINT(46.99836671912778 -122.56586110042556) bank111371 +111372 POINT(46.85204735385942 -123.14386542323467) bank111372 +111373 POINT(48.454440715810335 -121.42940249962226) bank111373 +111374 POINT(46.78893402920876 -121.47912127671813) bank111374 +111375 POINT(48.007819379941616 -122.74194839338634) bank111375 +111376 POINT(47.818504498335436 -121.81751568622413) bank111376 +111377 POINT(47.48720188682234 -123.2694705003098) bank111377 +111378 POINT(47.21030213306679 -122.82000784683228) bank111378 +111379 POINT(47.38100503046447 -122.68586068884557) bank111379 +111380 POINT(47.90646680352234 -123.08298886571129) bank111380 +111381 POINT(48.57762005904022 -122.78964175405434) bank111381 +111382 POINT(47.14134205303666 -123.11291755658056) bank111382 +111383 POINT(47.58442600221831 -122.71692252169754) bank111383 +111384 POINT(48.23474060999945 -121.92887791826756) bank111384 +111385 POINT(47.078153461162614 -122.02393490671349) bank111385 +111386 POINT(47.74300824679325 -121.40034643941642) bank111386 +111387 POINT(46.96409780040719 -123.18754561394614) bank111387 +111388 POINT(46.61398492593021 -122.20806840657995) bank111388 +111389 POINT(47.988075783429736 -121.89487436333069) bank111389 +111390 POINT(47.614244755632804 -122.84106099809999) bank111390 +111391 POINT(47.96631654424593 -122.85248134657608) bank111391 +111392 POINT(47.35784216531488 -123.26015646074278) bank111392 +111393 POINT(48.39563625006591 -121.76897020469823) bank111393 +111394 POINT(47.98885181420945 -123.23974020659294) bank111394 +111395 POINT(46.89544547567738 -122.90008891245147) bank111395 +111396 POINT(48.50460845815185 -122.02579674412229) bank111396 +111397 POINT(48.10322489121193 -122.91341113005222) bank111397 +111398 POINT(48.10805058494874 -122.70101045356594) bank111398 +111399 POINT(47.16699261409686 -122.27726613836813) bank111399 +111400 POINT(47.79098030847341 -122.80702187810749) bank111400 +111401 POINT(47.868396840838486 -121.52356815393065) bank111401 +111402 POINT(48.537334468690354 -122.78058527352174) bank111402 +111403 POINT(46.806548994015294 -121.55480931078057) bank111403 +111404 POINT(47.10723976431413 -122.5703251150313) bank111404 +111405 POINT(48.118143373109895 -122.20975058877879) bank111405 +111406 POINT(47.50297352777829 -122.74576212122011) bank111406 +111407 POINT(48.570496380435934 -122.03875814829047) bank111407 +111408 POINT(47.73692385787129 -122.72451379144795) bank111408 +111409 POINT(48.22195339806532 -122.00083510784971) bank111409 +111410 POINT(47.17703037335374 -123.33031693248746) bank111410 +111411 POINT(47.26433106818113 -123.11849209814554) bank111411 +111412 POINT(46.93638762597135 -121.82167879191124) bank111412 +111413 POINT(47.10598965747821 -121.96614558939312) bank111413 +111414 POINT(48.58529002628996 -122.53738991918412) bank111414 +111415 POINT(48.08443399269549 -122.17086635591588) bank111415 +111416 POINT(48.36961541501734 -121.46927041761455) bank111416 +111417 POINT(46.882752600057515 -122.6034065917192) bank111417 +111418 POINT(47.88745682514802 -122.66396095122536) bank111418 +111419 POINT(48.24050997623536 -121.45614742257384) bank111419 +111420 POINT(47.488545559223944 -123.0181317919841) bank111420 +111421 POINT(46.6396717221018 -122.9900360880968) bank111421 +111422 POINT(46.9652366297212 -123.06622123726534) bank111422 +111423 POINT(47.65969092655137 -123.14734946755948) bank111423 +111424 POINT(47.251892814823606 -123.086711385053) bank111424 +111425 POINT(46.93310969013248 -122.48615985314913) bank111425 +111426 POINT(46.89604020093101 -122.78049271853614) bank111426 +111427 POINT(47.019070591893474 -122.46169331165059) bank111427 +111428 POINT(48.293192798866826 -122.58745553204221) bank111428 +111429 POINT(47.297215566815005 -121.81374232413386) bank111429 +111430 POINT(47.79949323010704 -121.67978571390222) bank111430 +111431 POINT(47.181936707403146 -123.26910994459728) bank111431 +111432 POINT(48.580804123273694 -121.77932704266834) bank111432 +111433 POINT(48.284667845143495 -122.72351046478765) bank111433 +111434 POINT(47.169625861854826 -122.15257886111634) bank111434 +111435 POINT(46.990646758547136 -123.00001367442773) bank111435 +111436 POINT(46.96972771590802 -122.73494987433254) bank111436 +111437 POINT(48.36006422831137 -121.757848528518) bank111437 +111438 POINT(47.649240669840175 -121.99509036140392) bank111438 +111439 POINT(47.52387660415682 -121.34909580165545) bank111439 +111440 POINT(47.43670557222368 -122.2119973019879) bank111440 +111441 POINT(46.750022954477124 -122.21597276723317) bank111441 +111442 POINT(48.152337147091615 -121.86512907959965) bank111442 +111443 POINT(48.189539710085455 -122.83339876881833) bank111443 +111444 POINT(47.53027679007648 -121.78020168166583) bank111444 +111445 POINT(47.45811589030291 -123.0293720538237) bank111445 +111446 POINT(47.208900194756495 -123.22649527802118) bank111446 +111447 POINT(47.50857983576677 -121.46334755405356) bank111447 +111448 POINT(48.480910636028895 -121.35630268963477) bank111448 +111449 POINT(48.43748093465522 -121.40258256052779) bank111449 +111450 POINT(46.634221234274904 -121.92375992484548) bank111450 +111451 POINT(46.903207627581544 -122.40938938457525) bank111451 +111452 POINT(47.6515954273014 -121.8113928562117) bank111452 +111453 POINT(48.22495041432406 -121.78678396443756) bank111453 +111454 POINT(47.38955303587513 -122.96416663999992) bank111454 +111455 POINT(47.950916569665495 -123.07567391320164) bank111455 +111456 POINT(47.47071751474889 -122.08456384792683) bank111456 +111457 POINT(47.78582484181938 -122.11043303530059) bank111457 +111458 POINT(46.909574675114975 -122.53912899253687) bank111458 +111459 POINT(46.61731248284903 -122.34767621348443) bank111459 +111460 POINT(47.64953634476564 -123.22872470272459) bank111460 +111461 POINT(46.86767949421242 -121.6521106701173) bank111461 +111462 POINT(47.39480763702567 -121.53341394298306) bank111462 +111463 POINT(47.98762093440889 -121.83520434023612) bank111463 +111464 POINT(47.73058186498777 -122.76245376291513) bank111464 +111465 POINT(47.50032496337009 -122.88017573462203) bank111465 +111466 POINT(47.67255746830288 -121.53830230923126) bank111466 +111467 POINT(47.45813186211341 -121.90599187968982) bank111467 +111468 POINT(47.354836050372725 -122.68541759290662) bank111468 +111469 POINT(48.25192334799994 -122.33641517763314) bank111469 +111470 POINT(47.366148943017635 -122.26835418744483) bank111470 +111471 POINT(47.9202582064118 -123.22725175692595) bank111471 +111472 POINT(48.13112168582875 -123.06957695343488) bank111472 +111473 POINT(48.04164097020073 -122.21352342237847) bank111473 +111474 POINT(48.530332025122505 -121.74151142172019) bank111474 +111475 POINT(47.669273708563544 -122.40849516836026) bank111475 +111476 POINT(46.94512695091831 -122.17710208356335) bank111476 +111477 POINT(46.981488809885306 -122.81035571411468) bank111477 +111478 POINT(48.07241787718827 -122.4658383448203) bank111478 +111479 POINT(47.14651795804121 -122.78245531931519) bank111479 +111480 POINT(47.263557024946394 -122.63936590656797) bank111480 +111481 POINT(46.906257574109965 -121.60538316116794) bank111481 +111482 POINT(46.8955482577177 -121.4369763789645) bank111482 +111483 POINT(47.2031988782573 -121.38496978913096) bank111483 +111484 POINT(46.75749114129159 -122.15071736837912) bank111484 +111485 POINT(48.02973474755581 -122.53792953946646) bank111485 +111486 POINT(46.81642749155551 -122.25593876885283) bank111486 +111487 POINT(47.086110537883904 -122.41806731030401) bank111487 +111488 POINT(48.60508649091452 -122.769009987067) bank111488 +111489 POINT(47.24560834836197 -122.2323104895872) bank111489 +111490 POINT(48.48751716777981 -121.78486847417987) bank111490 +111491 POINT(48.19249753861488 -121.86853283658739) bank111491 +111492 POINT(47.10001802771026 -121.39169257365404) bank111492 +111493 POINT(47.977605135490094 -122.9423182357271) bank111493 +111494 POINT(47.066952713237754 -122.7122044138636) bank111494 +111495 POINT(48.24277774830717 -121.68219077387792) bank111495 +111496 POINT(48.08052242254479 -122.43321630239255) bank111496 +111497 POINT(47.136334604638925 -121.52863979676364) bank111497 +111498 POINT(48.44598466406525 -121.619803093055) bank111498 +111499 POINT(47.83903290490266 -122.0140081601671) bank111499 +111500 POINT(47.26414120425838 -123.21420777569725) bank111500 +111501 POINT(46.87913560456938 -123.00786490246811) bank111501 +111502 POINT(46.6221258151518 -123.18707988495643) bank111502 +111503 POINT(48.359530232880395 -122.56722565431049) bank111503 +111504 POINT(47.58664022904658 -121.80020758245935) bank111504 +111505 POINT(48.02959607437355 -122.62208747849195) bank111505 +111506 POINT(48.50284519878411 -122.54151906231046) bank111506 +111507 POINT(46.823386119907774 -121.73666288626684) bank111507 +111508 POINT(48.476474408714125 -122.71462995886691) bank111508 +111509 POINT(47.676230488050834 -121.73224813349799) bank111509 +111510 POINT(47.5481183979213 -121.76429042904257) bank111510 +111511 POINT(48.05092205400231 -121.74851167407738) bank111511 +111512 POINT(47.73623010516392 -122.01792254018751) bank111512 +111513 POINT(47.2885550302547 -122.42591189709178) bank111513 +111514 POINT(47.02658156460335 -123.24728007474901) bank111514 +111515 POINT(46.76600555607887 -123.21213826094275) bank111515 +111516 POINT(47.87728831624908 -122.77422016137076) bank111516 +111517 POINT(48.47102674671846 -122.732638051701) bank111517 +111518 POINT(48.31513333761558 -121.74242843971076) bank111518 +111519 POINT(48.35311677391959 -123.16489654892983) bank111519 +111520 POINT(48.25163407873587 -122.8716483577532) bank111520 +111521 POINT(46.71641374339601 -122.7060601768746) bank111521 +111522 POINT(47.31996293472241 -122.29461885772142) bank111522 +111523 POINT(46.96896177437288 -121.38344191418807) bank111523 +111524 POINT(47.522218818136466 -122.31014970950386) bank111524 +111525 POINT(48.554218675226814 -121.9130572007875) bank111525 +111526 POINT(47.49101569087634 -121.34897287628704) bank111526 +111527 POINT(47.97081008830033 -121.46499287744123) bank111527 +111528 POINT(47.60976261058948 -121.38962623545373) bank111528 +111529 POINT(48.34909807511488 -122.3826979767458) bank111529 +111530 POINT(48.49039218740609 -122.18837296887477) bank111530 +111531 POINT(48.019183894781904 -122.73321126542369) bank111531 +111532 POINT(48.194655131696166 -121.87995324439697) bank111532 +111533 POINT(48.53062539963033 -123.14284660797588) bank111533 +111534 POINT(47.17024278531101 -122.19474994833547) bank111534 +111535 POINT(47.3849037716599 -121.97344928803237) bank111535 +111536 POINT(47.20695419404087 -122.61149363849725) bank111536 +111537 POINT(46.91705660829281 -122.39025212304466) bank111537 +111538 POINT(47.37547890571822 -122.56291198313536) bank111538 +111539 POINT(47.97364325976182 -122.59097559503248) bank111539 +111540 POINT(46.87413535620194 -122.4556322921783) bank111540 +111541 POINT(46.76502272621666 -123.00539648165099) bank111541 +111542 POINT(47.20616701243162 -121.86421839105893) bank111542 +111543 POINT(47.73582803984647 -122.17125468678123) bank111543 +111544 POINT(48.166198697698036 -121.3849729760019) bank111544 +111545 POINT(47.807079450788024 -121.76930353194429) bank111545 +111546 POINT(48.45490100655442 -122.7865922768347) bank111546 +111547 POINT(48.46225746791834 -123.14222928448733) bank111547 +111548 POINT(47.76158390838593 -121.82346043734677) bank111548 +111549 POINT(47.49114460325421 -122.53814173316368) bank111549 +111550 POINT(47.61131194564174 -122.86873021729754) bank111550 +111551 POINT(48.22666060310839 -121.88595309209079) bank111551 +111552 POINT(47.273875021047836 -121.63815126992752) bank111552 +111553 POINT(47.96393244647431 -121.85691493554471) bank111553 +111554 POINT(47.438958850940494 -123.03348426919588) bank111554 +111555 POINT(47.90510813338283 -121.57394226555138) bank111555 +111556 POINT(47.144260276585875 -121.49034148607623) bank111556 +111557 POINT(46.87499625881286 -121.46696551411581) bank111557 +111558 POINT(46.611988193845775 -122.861833640913) bank111558 +111559 POINT(47.28251698878244 -121.97899958105921) bank111559 +111560 POINT(47.77529473271024 -123.14823191252279) bank111560 +111561 POINT(47.811197120566845 -123.12008421272289) bank111561 +111562 POINT(47.038150402022 -121.54608584549942) bank111562 +111563 POINT(47.687552868893206 -122.93610559736317) bank111563 +111564 POINT(47.4419891793559 -122.5395569182663) bank111564 +111565 POINT(47.13395724446938 -122.90400816523132) bank111565 +111566 POINT(48.27684084474203 -122.61249142580911) bank111566 +111567 POINT(47.95045455600759 -122.80117906560852) bank111567 +111568 POINT(48.56389086023416 -122.67332987369291) bank111568 +111569 POINT(47.149999466122104 -122.21173621277498) bank111569 +111570 POINT(47.813382835782924 -121.66698399983905) bank111570 +111571 POINT(47.50756765357647 -122.47448943618342) bank111571 +111572 POINT(47.2769408576768 -122.5322293103141) bank111572 +111573 POINT(47.41733820765549 -122.49850646665622) bank111573 +111574 POINT(48.10686324152961 -123.16934169035514) bank111574 +111575 POINT(46.72205933809966 -122.8552909878607) bank111575 +111576 POINT(48.253253408945135 -123.17702833797235) bank111576 +111577 POINT(48.07574382123738 -121.55885969401369) bank111577 +111578 POINT(47.99692536954674 -121.93720852284557) bank111578 +111579 POINT(47.89440335254837 -121.61405436409835) bank111579 +111580 POINT(48.1072856559519 -121.71215146659627) bank111580 +111581 POINT(47.01462035976521 -122.97490356643392) bank111581 +111582 POINT(48.45301611080611 -123.14153068918988) bank111582 +111583 POINT(48.59557929915909 -123.22818973395935) bank111583 +111584 POINT(48.494957925651505 -121.3965551048344) bank111584 +111585 POINT(47.59721236870108 -122.26555251816436) bank111585 +111586 POINT(47.26025350844896 -122.72855595694917) bank111586 +111587 POINT(47.41502359987856 -123.09463739083687) bank111587 +111588 POINT(47.37644617247225 -123.29758247223903) bank111588 +111589 POINT(48.01813056210814 -122.56472935780042) bank111589 +111590 POINT(48.078540640570495 -123.23608893170523) bank111590 +111591 POINT(48.53955182240213 -122.82859374991902) bank111591 +111592 POINT(47.9919684669061 -121.66110713072007) bank111592 +111593 POINT(48.03200618781892 -122.52492542957805) bank111593 +111594 POINT(47.82759499765869 -122.2986746328648) bank111594 +111595 POINT(47.771425784424174 -122.14056070396731) bank111595 +111596 POINT(48.38791062765786 -121.97421023907793) bank111596 +111597 POINT(47.73535034666764 -123.20058629462137) bank111597 +111598 POINT(48.18736583861354 -121.71966546853893) bank111598 +111599 POINT(48.19206200997416 -122.29076088566357) bank111599 +111600 POINT(47.65830199780352 -121.4235323268156) bank111600 +111601 POINT(48.11644607260432 -121.75849840676062) bank111601 +111602 POINT(47.907627078577065 -122.97863265737348) bank111602 +111603 POINT(47.35491974993093 -122.88312672558806) bank111603 +111604 POINT(48.033719524008 -122.62117059790246) bank111604 +111605 POINT(47.26073218810055 -122.83706165045874) bank111605 +111606 POINT(48.07370008304225 -121.90005403542015) bank111606 +111607 POINT(47.616816978368774 -123.07004358887353) bank111607 +111608 POINT(47.0650036650453 -121.7745290641103) bank111608 +111609 POINT(47.23353832191058 -123.12503725605373) bank111609 +111610 POINT(47.81998662990572 -121.80300127153538) bank111610 +111611 POINT(47.5964972595635 -121.65770554966481) bank111611 +111612 POINT(47.69007071665836 -121.37536999873537) bank111612 +111613 POINT(47.36586287311927 -122.51313146416108) bank111613 +111614 POINT(47.295094315567006 -122.98008544921387) bank111614 +111615 POINT(48.446106347973426 -121.85347818938843) bank111615 +111616 POINT(47.75136707521843 -121.96309496883318) bank111616 +111617 POINT(48.2751703961008 -121.87392967822414) bank111617 +111618 POINT(47.01505127314238 -122.70594687728813) bank111618 +111619 POINT(47.3742883033519 -121.96204886131544) bank111619 +111620 POINT(48.076606518562855 -122.9721485203223) bank111620 +111621 POINT(46.84047984702697 -121.88610039818239) bank111621 +111622 POINT(48.03022878589272 -123.3131952728672) bank111622 +111623 POINT(46.8199827990606 -122.77654610601671) bank111623 +111624 POINT(47.58110455836822 -122.30768120920935) bank111624 +111625 POINT(48.32340404115943 -123.04201265912819) bank111625 +111626 POINT(47.31901227438216 -122.40104159473513) bank111626 +111627 POINT(47.35848614026031 -121.73731411557226) bank111627 +111628 POINT(48.25383205133045 -122.99186807054636) bank111628 +111629 POINT(48.2987439290182 -121.4959668181742) bank111629 +111630 POINT(47.279269037488596 -121.44519396774781) bank111630 +111631 POINT(47.68047734956795 -122.30888214489133) bank111631 +111632 POINT(47.127461914637145 -122.03883068224637) bank111632 +111633 POINT(46.938693361226356 -121.72717789061466) bank111633 +111634 POINT(47.02013942655725 -122.09195155275066) bank111634 +111635 POINT(48.50637496618687 -122.713046378505) bank111635 +111636 POINT(48.496295690817625 -122.71168356310454) bank111636 +111637 POINT(47.347886211252195 -121.77396042248739) bank111637 +111638 POINT(47.87862688920764 -122.38206636015488) bank111638 +111639 POINT(47.389832027219825 -121.6034193559693) bank111639 +111640 POINT(47.75300799890633 -121.79667589908884) bank111640 +111641 POINT(47.46791509849486 -121.7701273446209) bank111641 +111642 POINT(47.13820987114413 -122.901072951593) bank111642 +111643 POINT(48.36945305192552 -121.39625045857272) bank111643 +111644 POINT(47.8866729120512 -122.07214296034357) bank111644 +111645 POINT(48.07973255407732 -122.32390838198744) bank111645 +111646 POINT(46.83200472863054 -122.38910901357353) bank111646 +111647 POINT(47.14345511240885 -123.00364770644435) bank111647 +111648 POINT(46.82461939736664 -122.54663822196954) bank111648 +111649 POINT(47.677717299328656 -122.14197718769742) bank111649 +111650 POINT(46.60731269189478 -121.96234010674401) bank111650 +111651 POINT(47.88491951843736 -122.68987005553495) bank111651 +111652 POINT(47.443769533317734 -122.66983867398643) bank111652 +111653 POINT(48.297674675216534 -122.81714823190984) bank111653 +111654 POINT(48.340186372459755 -122.26689963115993) bank111654 +111655 POINT(47.994543231786096 -122.54881550378809) bank111655 +111656 POINT(47.33255291816638 -122.09198411283907) bank111656 +111657 POINT(46.90294207504467 -122.10322319448474) bank111657 +111658 POINT(47.94995635858402 -122.90439677987142) bank111658 +111659 POINT(47.361449042026564 -123.25827341446661) bank111659 +111660 POINT(48.00659005369435 -121.49274279983175) bank111660 +111661 POINT(46.60842731128309 -122.66074129286811) bank111661 +111662 POINT(47.05404543616727 -123.19956542970857) bank111662 +111663 POINT(46.7439923977121 -123.13542194624333) bank111663 +111664 POINT(48.18947713482046 -121.42772074419675) bank111664 +111665 POINT(47.1161045374586 -122.32553026061505) bank111665 +111666 POINT(47.024813632650144 -122.24144949693468) bank111666 +111667 POINT(48.01725483755623 -122.60999142187858) bank111667 +111668 POINT(48.53033935595058 -121.690012142585) bank111668 +111669 POINT(48.42103390865789 -122.25641755017159) bank111669 +111670 POINT(48.052761506750386 -122.9633125681649) bank111670 +111671 POINT(46.69708802929798 -122.5098605028317) bank111671 +111672 POINT(47.22099718467918 -123.28730659533237) bank111672 +111673 POINT(47.11866388205432 -121.41308076597281) bank111673 +111674 POINT(48.58799120532837 -121.8185630468744) bank111674 +111675 POINT(47.835940794961395 -122.10316713471107) bank111675 +111676 POINT(48.090469972790295 -122.1043194245594) bank111676 +111677 POINT(47.88939596367715 -121.44563867967837) bank111677 +111678 POINT(47.05896803676729 -122.76802121702735) bank111678 +111679 POINT(47.28518469394064 -121.66256469854555) bank111679 +111680 POINT(46.88335177959612 -122.56046348448966) bank111680 +111681 POINT(46.90109190448964 -122.4655965942858) bank111681 +111682 POINT(46.6627226947421 -122.59090057100505) bank111682 +111683 POINT(47.851035459168656 -122.74086862237202) bank111683 +111684 POINT(47.2352888523012 -121.36226914857205) bank111684 +111685 POINT(47.0097194970982 -122.9609094950131) bank111685 +111686 POINT(47.78067176457863 -123.00592842659029) bank111686 +111687 POINT(48.146719752198784 -121.4458585919599) bank111687 +111688 POINT(47.52789592423315 -122.9525513741936) bank111688 +111689 POINT(48.40105016891919 -121.80194270163946) bank111689 +111690 POINT(48.50131249623157 -122.09041100434305) bank111690 +111691 POINT(47.33802275628475 -123.10611178384349) bank111691 +111692 POINT(47.65638241849553 -121.78538321335928) bank111692 +111693 POINT(47.77567584225884 -122.44111616171696) bank111693 +111694 POINT(47.82475660945371 -121.5054753001465) bank111694 +111695 POINT(47.38298935230245 -121.5803486952626) bank111695 +111696 POINT(47.8024670146234 -121.91564698924326) bank111696 +111697 POINT(48.13402160975224 -122.06549453016385) bank111697 +111698 POINT(47.068850782751355 -121.92888215271061) bank111698 +111699 POINT(47.390088311926135 -122.62769835379898) bank111699 +111700 POINT(47.776001496395345 -121.72640445698295) bank111700 +111701 POINT(47.740981564048674 -122.40372107636078) bank111701 +111702 POINT(48.51993840568015 -122.19145565914769) bank111702 +111703 POINT(47.33366926861637 -123.03141850749027) bank111703 +111704 POINT(47.532617470501 -122.96598089047237) bank111704 +111705 POINT(48.47066093825376 -122.1780729851849) bank111705 +111706 POINT(47.103787455930785 -123.01629578240315) bank111706 +111707 POINT(47.80299327054442 -122.14958506704373) bank111707 +111708 POINT(47.10811520210555 -121.75949188768304) bank111708 +111709 POINT(47.47486828046968 -123.28979276306086) bank111709 +111710 POINT(46.62925879917548 -121.92153421014937) bank111710 +111711 POINT(48.36877980487384 -121.50180462390126) bank111711 +111712 POINT(48.262426786890806 -122.25161193429096) bank111712 +111713 POINT(48.2593370266116 -123.03053671015665) bank111713 +111714 POINT(47.130774448524534 -123.27438217831673) bank111714 +111715 POINT(46.606211998827014 -123.23341973814081) bank111715 +111716 POINT(47.57525026901815 -121.59395060625432) bank111716 +111717 POINT(46.63389853546446 -122.11266499470074) bank111717 +111718 POINT(46.92252928356736 -121.76426675609216) bank111718 +111719 POINT(48.467856568494824 -122.23774070525012) bank111719 +111720 POINT(48.56280630520651 -122.73699928975326) bank111720 +111721 POINT(48.18742565690456 -123.08605527448412) bank111721 +111722 POINT(47.96862430269062 -122.29486743565585) bank111722 +111723 POINT(48.15758472297387 -123.25562220115668) bank111723 +111724 POINT(47.038699255240594 -122.91759355887362) bank111724 +111725 POINT(47.776289626773824 -121.81973331670525) bank111725 +111726 POINT(48.19234023277882 -122.2744075437172) bank111726 +111727 POINT(47.026151852988754 -122.45198745538607) bank111727 +111728 POINT(48.43586500847055 -121.97678977638792) bank111728 +111729 POINT(47.92006883192161 -123.03993414447145) bank111729 +111730 POINT(48.340498334597534 -121.54006939111598) bank111730 +111731 POINT(47.497822445692385 -122.42679247441549) bank111731 +111732 POINT(47.327058379005535 -121.62691021440402) bank111732 +111733 POINT(46.60971901548437 -123.22244967439438) bank111733 +111734 POINT(47.54487265992067 -121.45207352742628) bank111734 +111735 POINT(48.083326276940774 -123.31410300505419) bank111735 +111736 POINT(48.213604061906565 -122.37493350312941) bank111736 +111737 POINT(47.6939276211891 -122.09224588365713) bank111737 +111738 POINT(48.45072493722627 -122.49251326547794) bank111738 +111739 POINT(47.624723695895675 -122.11444667161314) bank111739 +111740 POINT(48.234173058050274 -121.70402383564071) bank111740 +111741 POINT(48.118783209689454 -121.63430198329664) bank111741 +111742 POINT(48.184714179394014 -121.67196763827296) bank111742 +111743 POINT(46.87547993620099 -121.55616960506848) bank111743 +111744 POINT(47.62966674763902 -122.98231219186138) bank111744 +111745 POINT(47.17084873378446 -122.61973179075231) bank111745 +111746 POINT(46.71359416742059 -122.53905234471793) bank111746 +111747 POINT(47.527300268145666 -123.25341851895983) bank111747 +111748 POINT(46.68519794349852 -121.56707445247405) bank111748 +111749 POINT(47.68117492991058 -121.87822746710584) bank111749 +111750 POINT(48.271412063828436 -122.00712847271998) bank111750 +111751 POINT(47.854964253952666 -122.84285563938275) bank111751 +111752 POINT(47.58549687000198 -121.45410884064894) bank111752 +111753 POINT(47.82239209916684 -122.28787386509757) bank111753 +111754 POINT(47.78835733987583 -122.45770122068865) bank111754 +111755 POINT(48.34575160811265 -122.19823394803227) bank111755 +111756 POINT(47.952418615870926 -123.28706894944568) bank111756 +111757 POINT(47.81232586242977 -123.06536293355533) bank111757 +111758 POINT(47.26966473316625 -121.71876681548058) bank111758 +111759 POINT(46.87631370920343 -122.37907074623804) bank111759 +111760 POINT(46.72266644732142 -121.56581936905951) bank111760 +111761 POINT(48.06587174439172 -123.2426160151305) bank111761 +111762 POINT(47.65776344553556 -121.97017841028784) bank111762 +111763 POINT(47.58572300892914 -122.10977559245048) bank111763 +111764 POINT(47.25203892811988 -123.05182005685077) bank111764 +111765 POINT(48.06333056345667 -122.14348052118149) bank111765 +111766 POINT(48.45703881099576 -122.21046864695064) bank111766 +111767 POINT(47.570543236615976 -122.93410890129798) bank111767 +111768 POINT(48.39639004158777 -123.30909799167019) bank111768 +111769 POINT(48.38454923282182 -122.47525443106788) bank111769 +111770 POINT(48.23488023366665 -121.6876365912871) bank111770 +111771 POINT(47.10294310690291 -122.2225961489048) bank111771 +111772 POINT(46.76785128002986 -121.96467966083547) bank111772 +111773 POINT(48.17841902300207 -122.32417426616458) bank111773 +111774 POINT(47.70024275564434 -121.53862467913272) bank111774 +111775 POINT(48.157588642677446 -122.89397959705218) bank111775 +111776 POINT(47.18284233750061 -122.30777293161107) bank111776 +111777 POINT(47.601164751093314 -121.44356991231139) bank111777 +111778 POINT(46.977365073619325 -122.92428548768459) bank111778 +111779 POINT(46.781457109504345 -122.57654918308147) bank111779 +111780 POINT(47.90250885813586 -122.34950911981623) bank111780 +111781 POINT(48.38097538079324 -122.81754026791224) bank111781 +111782 POINT(47.45848435128867 -122.17170945926532) bank111782 +111783 POINT(46.69248539156789 -121.98714122905338) bank111783 +111784 POINT(46.66476302046846 -123.29289789378052) bank111784 +111785 POINT(48.578794860490184 -123.28337589546203) bank111785 +111786 POINT(47.669741925932634 -122.13677533240494) bank111786 +111787 POINT(48.51314677723731 -121.64711367246747) bank111787 +111788 POINT(46.973726117426224 -123.28315607951826) bank111788 +111789 POINT(47.00170558836668 -123.11895984716489) bank111789 +111790 POINT(47.90909679504622 -123.02577573548197) bank111790 +111791 POINT(47.112830575406086 -122.53024263476915) bank111791 +111792 POINT(48.49992066275473 -121.53531443444162) bank111792 +111793 POINT(47.89322011726735 -121.44618666144238) bank111793 +111794 POINT(47.2690337112578 -122.93594835165963) bank111794 +111795 POINT(48.55806773911867 -123.21060696070548) bank111795 +111796 POINT(46.67046661640882 -123.28844720037301) bank111796 +111797 POINT(46.88879864951926 -121.98761282366952) bank111797 +111798 POINT(48.46858421093162 -121.84349456789249) bank111798 +111799 POINT(46.611555802056266 -121.66574133396641) bank111799 +111800 POINT(47.27942969794692 -122.39610379389399) bank111800 +111801 POINT(47.89480221408968 -122.46229884859453) bank111801 +111802 POINT(47.56656426294486 -123.22062295497119) bank111802 +111803 POINT(48.34134318669218 -123.03676112951887) bank111803 +111804 POINT(47.81776112089527 -123.0719746643209) bank111804 +111805 POINT(47.67025733347064 -122.67981145378263) bank111805 +111806 POINT(46.89974400204992 -122.07181668721634) bank111806 +111807 POINT(47.49032505813277 -123.22134025967026) bank111807 +111808 POINT(47.35818631858379 -121.41266808393031) bank111808 +111809 POINT(47.663314560117286 -122.837510986939) bank111809 +111810 POINT(48.163084556445064 -122.41481648575879) bank111810 +111811 POINT(47.66744674952745 -121.75128824316754) bank111811 +111812 POINT(47.50158692086629 -122.50213579164688) bank111812 +111813 POINT(46.64842491624426 -121.85022126841278) bank111813 +111814 POINT(48.34283154282583 -122.08553173560972) bank111814 +111815 POINT(47.636831811569316 -121.59815501157627) bank111815 +111816 POINT(48.2945237077893 -123.06471750834645) bank111816 +111817 POINT(48.382114824200315 -121.69718546582142) bank111817 +111818 POINT(47.50143542511154 -123.13268467378735) bank111818 +111819 POINT(47.81142838052161 -121.63371870946747) bank111819 +111820 POINT(48.04491722440855 -121.70691905144982) bank111820 +111821 POINT(48.40465026959911 -123.14015548199347) bank111821 +111822 POINT(48.29704460811429 -122.58769235016095) bank111822 +111823 POINT(47.97895603275466 -122.03733579406946) bank111823 +111824 POINT(47.77481909679806 -122.45354449447531) bank111824 +111825 POINT(47.08825679396765 -121.76186553826652) bank111825 +111826 POINT(47.93828177243364 -123.05407961069005) bank111826 +111827 POINT(48.3843738563411 -122.40950592265602) bank111827 +111828 POINT(47.05811747319578 -121.72658735924932) bank111828 +111829 POINT(48.005282576245776 -122.73043059598145) bank111829 +111830 POINT(46.95440929691926 -122.67079045331666) bank111830 +111831 POINT(47.93900316233136 -121.89112927437432) bank111831 +111832 POINT(47.2374985351429 -121.45999565478567) bank111832 +111833 POINT(47.97334775111747 -123.16086493254743) bank111833 +111834 POINT(47.50836384261818 -121.78678453743241) bank111834 +111835 POINT(48.19446853247322 -122.85194359641493) bank111835 +111836 POINT(48.26473935193869 -123.01000054478871) bank111836 +111837 POINT(48.41819520043303 -121.70182879222945) bank111837 +111838 POINT(47.344559754831934 -122.01267763980609) bank111838 +111839 POINT(47.90154827535079 -121.34453293315812) bank111839 +111840 POINT(47.26883408250248 -121.9592652565293) bank111840 +111841 POINT(47.71557973347673 -122.49503827405465) bank111841 +111842 POINT(46.63788091293306 -122.82781808161036) bank111842 +111843 POINT(47.35088730184007 -122.63426043816335) bank111843 +111844 POINT(47.82928837060838 -122.22522803380683) bank111844 +111845 POINT(48.093083295100044 -122.05003860467167) bank111845 +111846 POINT(48.35370587044114 -122.35006994669102) bank111846 +111847 POINT(48.25992154073946 -123.30205365784661) bank111847 +111848 POINT(46.8141821865813 -121.49087810566543) bank111848 +111849 POINT(47.65079414856529 -121.48638858728312) bank111849 +111850 POINT(47.8865688449708 -121.43309284237571) bank111850 +111851 POINT(47.978775987734586 -122.81205214497493) bank111851 +111852 POINT(47.99641746166512 -122.79519596493488) bank111852 +111853 POINT(47.177845011695474 -122.32606485350715) bank111853 +111854 POINT(47.22555437679068 -123.21941072396613) bank111854 +111855 POINT(47.440709332026955 -122.9967513107169) bank111855 +111856 POINT(47.55206442802752 -122.54337917654934) bank111856 +111857 POINT(47.668923155188104 -121.56400395667376) bank111857 +111858 POINT(47.70586942204229 -121.8608835128739) bank111858 +111859 POINT(48.02483733802925 -121.54301139502657) bank111859 +111860 POINT(47.191826546659364 -121.79712661123004) bank111860 +111861 POINT(47.10169743832347 -122.16271657155298) bank111861 +111862 POINT(47.764242791849625 -121.78127947261731) bank111862 +111863 POINT(47.27687099036262 -121.36574858072562) bank111863 +111864 POINT(46.622810281069256 -123.22548219755646) bank111864 +111865 POINT(48.09102625910866 -121.42204917419454) bank111865 +111866 POINT(47.66724365384725 -122.3850501900827) bank111866 +111867 POINT(47.134033499173114 -122.61585951641351) bank111867 +111868 POINT(48.142654409484926 -122.03392420515019) bank111868 +111869 POINT(46.76507785360428 -121.81506205337038) bank111869 +111870 POINT(46.703364932967176 -121.49406502276899) bank111870 +111871 POINT(47.2565876298621 -121.70593622430574) bank111871 +111872 POINT(47.04965838680894 -121.73985570928983) bank111872 +111873 POINT(47.04831087181951 -123.29286928015726) bank111873 +111874 POINT(47.98939090900232 -122.526290131551) bank111874 +111875 POINT(46.924768206622666 -122.50792674278865) bank111875 +111876 POINT(48.33120334093888 -121.73336659112482) bank111876 +111877 POINT(47.20000263370884 -122.50904459750747) bank111877 +111878 POINT(47.40644557256404 -121.6272599490052) bank111878 +111879 POINT(46.80294037842051 -122.93129736887444) bank111879 +111880 POINT(47.718098564840155 -122.56789899126026) bank111880 +111881 POINT(48.25167704225299 -121.68266181865651) bank111881 +111882 POINT(47.13342259427053 -122.03438884721218) bank111882 +111883 POINT(48.17126216944773 -122.59642687825425) bank111883 +111884 POINT(47.29815270145641 -123.04215963380916) bank111884 +111885 POINT(48.06510798099441 -121.6729923794737) bank111885 +111886 POINT(47.04869889002284 -121.34318083384844) bank111886 +111887 POINT(48.086064409077025 -123.20889846926009) bank111887 +111888 POINT(47.88743666012821 -122.24925276057597) bank111888 +111889 POINT(48.10266784105775 -122.49760679984485) bank111889 +111890 POINT(47.3199661794572 -123.3149526487759) bank111890 +111891 POINT(48.5423940175171 -122.04327165312455) bank111891 +111892 POINT(47.81329832588365 -122.13375477644672) bank111892 +111893 POINT(46.776358731123324 -122.62987303621578) bank111893 +111894 POINT(46.66888300226385 -121.49689660647812) bank111894 +111895 POINT(47.19121988539979 -122.08945606319857) bank111895 +111896 POINT(46.80851509492786 -123.18328449229581) bank111896 +111897 POINT(47.07519547000505 -121.89893800791631) bank111897 +111898 POINT(47.89640906603039 -121.55888930281552) bank111898 +111899 POINT(47.66663739286947 -121.53689725271909) bank111899 +111900 POINT(47.59090490121569 -121.57703060135205) bank111900 +111901 POINT(48.40954209399164 -123.2495441080321) bank111901 +111902 POINT(46.927405817529056 -123.16680350917738) bank111902 +111903 POINT(48.33296486359276 -122.47845325051033) bank111903 +111904 POINT(47.607994485429195 -121.79689013778257) bank111904 +111905 POINT(47.16626222088699 -122.887246367982) bank111905 +111906 POINT(47.04396371459601 -122.19750125600295) bank111906 +111907 POINT(46.80248660006071 -122.38318930745304) bank111907 +111908 POINT(48.55530205260443 -121.57001069167883) bank111908 +111909 POINT(47.91142813093812 -122.99165969803379) bank111909 +111910 POINT(46.99875937402445 -122.81310845144803) bank111910 +111911 POINT(47.91226159161781 -122.40079055576426) bank111911 +111912 POINT(47.233298118265786 -122.9759038953755) bank111912 +111913 POINT(47.713340054062066 -122.97715055195802) bank111913 +111914 POINT(46.63712171222483 -122.41710650578986) bank111914 +111915 POINT(47.4288802934879 -121.46988852406088) bank111915 +111916 POINT(47.57018307629491 -122.18051678510916) bank111916 +111917 POINT(47.940665730992876 -122.94148246876902) bank111917 +111918 POINT(47.550844883332886 -122.66985174579108) bank111918 +111919 POINT(48.5453415535584 -122.46008295109412) bank111919 +111920 POINT(47.8734009730326 -121.89648389858436) bank111920 +111921 POINT(47.22090505782544 -122.2918107501334) bank111921 +111922 POINT(48.282418804626985 -122.86422798590814) bank111922 +111923 POINT(48.370419759435705 -122.42748033057194) bank111923 +111924 POINT(47.38922896338321 -122.2609479225447) bank111924 +111925 POINT(48.103658283473 -121.72064171492757) bank111925 +111926 POINT(48.10472212256482 -122.92902454482007) bank111926 +111927 POINT(48.05418767896237 -122.51076587608257) bank111927 +111928 POINT(46.70540663905902 -122.88991879856259) bank111928 +111929 POINT(48.140966953613564 -123.0716093280738) bank111929 +111930 POINT(47.954109770551675 -122.71507323022186) bank111930 +111931 POINT(48.10081503854448 -121.436250424036) bank111931 +111932 POINT(47.85020771497309 -122.61394834332101) bank111932 +111933 POINT(47.57782042268612 -121.54954236340484) bank111933 +111934 POINT(48.48707106139646 -122.63276949466074) bank111934 +111935 POINT(48.228921053466074 -122.87026834830422) bank111935 +111936 POINT(48.23774820392068 -121.91199027135036) bank111936 +111937 POINT(46.901176456006546 -122.82561523721071) bank111937 +111938 POINT(46.65351549935785 -121.34657159353084) bank111938 +111939 POINT(47.84026518074821 -122.71370045077099) bank111939 +111940 POINT(47.569696592872155 -122.39056950938254) bank111940 +111941 POINT(47.69166252563529 -121.49435898580649) bank111941 +111942 POINT(46.735213363042384 -122.73602432524008) bank111942 +111943 POINT(47.31078657177595 -121.46712904064667) bank111943 +111944 POINT(48.13343004780905 -123.13755941176049) bank111944 +111945 POINT(46.76151180277275 -121.80023404025384) bank111945 +111946 POINT(46.77976504919668 -123.0401536831066) bank111946 +111947 POINT(46.667764595104885 -121.53877777083471) bank111947 +111948 POINT(46.99822771188481 -122.61715983503336) bank111948 +111949 POINT(47.064041895915594 -123.04932035080817) bank111949 +111950 POINT(47.76099366402447 -122.86414281835886) bank111950 +111951 POINT(46.87110090435426 -122.93571777966261) bank111951 +111952 POINT(48.388959799331566 -121.86390490495684) bank111952 +111953 POINT(47.349541917389566 -121.85634238990983) bank111953 +111954 POINT(47.57623440157109 -122.39912232050438) bank111954 +111955 POINT(47.98724271505675 -122.0345446240625) bank111955 +111956 POINT(47.795759330463085 -122.67427823797458) bank111956 +111957 POINT(46.825768119668396 -122.13375313943621) bank111957 +111958 POINT(47.20731002012566 -122.81840734475468) bank111958 +111959 POINT(47.44485591464563 -121.60713956036957) bank111959 +111960 POINT(47.20005626165042 -121.3585288974106) bank111960 +111961 POINT(47.63516151001465 -122.55837437666878) bank111961 +111962 POINT(48.3177172003854 -122.85893335745538) bank111962 +111963 POINT(48.182605011077506 -121.97947051774925) bank111963 +111964 POINT(46.95499804739737 -122.41679716722149) bank111964 +111965 POINT(47.26019291070563 -121.69262755245056) bank111965 +111966 POINT(47.26123987185132 -121.38917373576896) bank111966 +111967 POINT(47.15400789325953 -122.24404919037441) bank111967 +111968 POINT(48.13644279013453 -123.2025722361998) bank111968 +111969 POINT(47.04515225813023 -122.44082453043207) bank111969 +111970 POINT(47.5943578275443 -122.11656154285939) bank111970 +111971 POINT(47.470757693534175 -123.2020596063649) bank111971 +111972 POINT(47.21181767153422 -122.51755104067155) bank111972 +111973 POINT(46.92445082645441 -122.78137501460414) bank111973 +111974 POINT(46.62939565701653 -123.25055892133291) bank111974 +111975 POINT(47.75609998254057 -123.09161430302663) bank111975 +111976 POINT(48.49583892205057 -122.14882493898577) bank111976 +111977 POINT(46.923409432201986 -122.44878311911592) bank111977 +111978 POINT(47.17873234170929 -121.68169205340874) bank111978 +111979 POINT(46.66862952407512 -123.13797961426054) bank111979 +111980 POINT(47.09331445828364 -122.97112168712627) bank111980 +111981 POINT(48.3083733705002 -121.95781048690844) bank111981 +111982 POINT(48.55183639464684 -122.41329138315884) bank111982 +111983 POINT(47.92461432285894 -121.40369023798281) bank111983 +111984 POINT(48.4186042071271 -122.52883510068808) bank111984 +111985 POINT(47.96203323351581 -121.82617047583446) bank111985 +111986 POINT(46.87732458761365 -122.65098326430763) bank111986 +111987 POINT(46.76748267101032 -122.43270393980329) bank111987 +111988 POINT(47.128697331685046 -122.54685387350703) bank111988 +111989 POINT(46.836832902159614 -122.9932763712583) bank111989 +111990 POINT(46.770904932265395 -121.73006024237677) bank111990 +111991 POINT(46.69953745271776 -121.54408603392883) bank111991 +111992 POINT(47.834839280229566 -122.08511644631731) bank111992 +111993 POINT(47.53552884232907 -122.56333673820242) bank111993 +111994 POINT(47.025663560363284 -122.29412491805088) bank111994 +111995 POINT(47.16003727792083 -122.75066696463006) bank111995 +111996 POINT(46.89461508572547 -122.08689635940699) bank111996 +111997 POINT(48.31586145887911 -121.33315734988751) bank111997 +111998 POINT(47.30591444176589 -123.22018577795143) bank111998 +111999 POINT(48.10937142877416 -121.40705357724353) bank111999 +112000 POINT(48.452159014450885 -122.92743820528408) bank112000 +112001 POINT(48.184476405886016 -122.85796737015045) bank112001 +112002 POINT(48.58151426364786 -123.15472519986382) bank112002 +112003 POINT(47.75486887695738 -122.38692526042867) bank112003 +112004 POINT(47.84256467218305 -123.1924659735525) bank112004 +112005 POINT(47.14060160984547 -121.88262671859177) bank112005 +112006 POINT(48.40992982372425 -121.51955269341394) bank112006 +112007 POINT(46.79186299753908 -122.30985071767995) bank112007 +112008 POINT(48.27715045465889 -122.62886559918216) bank112008 +112009 POINT(46.75689830399694 -121.60497812603953) bank112009 +112010 POINT(47.693402498433876 -121.84872772087591) bank112010 +112011 POINT(47.92879245595048 -123.14381486134042) bank112011 +112012 POINT(47.23585877289753 -121.80599772526244) bank112012 +112013 POINT(48.04324644440704 -122.89105779718548) bank112013 +112014 POINT(47.91038305243066 -122.12086878353554) bank112014 +112015 POINT(48.47632210691284 -122.82246479495032) bank112015 +112016 POINT(47.26786251872642 -121.56351378720866) bank112016 +112017 POINT(46.778664968346334 -121.49430119312841) bank112017 +112018 POINT(47.10988190927139 -121.49599125522485) bank112018 +112019 POINT(47.5679164979814 -123.02359522240472) bank112019 +112020 POINT(48.58609989170979 -122.75368348541238) bank112020 +112021 POINT(48.21128035830689 -122.2524549692622) bank112021 +112022 POINT(47.3007651105093 -121.3579034601064) bank112022 +112023 POINT(48.27775403555578 -121.36420866031486) bank112023 +112024 POINT(47.54016288751654 -122.52030757019051) bank112024 +112025 POINT(47.073921314621195 -121.34811735829442) bank112025 +112026 POINT(46.79431169891178 -121.86936171377097) bank112026 +112027 POINT(46.60691807916104 -122.96610904077782) bank112027 +112028 POINT(46.62853020767858 -122.83793663900273) bank112028 +112029 POINT(47.85734853473 -122.36569878627915) bank112029 +112030 POINT(48.105892871913646 -123.28688615788694) bank112030 +112031 POINT(47.10798348920566 -123.25307901937882) bank112031 +112032 POINT(48.07057498197825 -121.36380068493352) bank112032 +112033 POINT(46.92622607355684 -121.88099546219289) bank112033 +112034 POINT(47.219464301109134 -121.79052827273969) bank112034 +112035 POINT(47.856457089583714 -122.09656225108296) bank112035 +112036 POINT(47.24722957838909 -122.14414294289658) bank112036 +112037 POINT(47.368187258617134 -122.80705599828774) bank112037 +112038 POINT(47.78695160151159 -121.37145198113305) bank112038 +112039 POINT(47.38590368780785 -123.06130274220298) bank112039 +112040 POINT(47.70633232606839 -121.71756535583299) bank112040 +112041 POINT(47.63792955184992 -122.35304243366207) bank112041 +112042 POINT(47.05586820589666 -122.20016887677623) bank112042 +112043 POINT(46.942002899018156 -121.88179714959885) bank112043 +112044 POINT(48.323673771818825 -121.4756141450623) bank112044 +112045 POINT(47.77140539930055 -122.23426354802601) bank112045 +112046 POINT(48.537938511695025 -123.1956070519824) bank112046 +112047 POINT(48.00374377753722 -121.72797738407485) bank112047 +112048 POINT(47.45654068568462 -121.57785171030947) bank112048 +112049 POINT(46.62973513611648 -121.66287655647555) bank112049 +112050 POINT(48.31746681185941 -121.93228888894025) bank112050 +112051 POINT(46.76819201347732 -123.09917728495978) bank112051 +112052 POINT(46.647872257357164 -121.64185744501145) bank112052 +112053 POINT(47.964730338380285 -122.52898528659998) bank112053 +112054 POINT(46.97567246522493 -122.40733539272253) bank112054 +112055 POINT(48.04888560058795 -121.89198261552045) bank112055 +112056 POINT(48.21419183448839 -122.93670129512573) bank112056 +112057 POINT(47.55613655061606 -121.36627038851263) bank112057 +112058 POINT(48.041805059167615 -121.80414304779792) bank112058 +112059 POINT(47.509515906103864 -121.87972330904522) bank112059 +112060 POINT(48.2459860297821 -121.93365989160411) bank112060 +112061 POINT(47.779030053762675 -121.65613048399618) bank112061 +112062 POINT(48.09339542534077 -123.00438867218273) bank112062 +112063 POINT(48.121024730703326 -123.07113309826491) bank112063 +112064 POINT(46.93303877354676 -122.01702271169412) bank112064 +112065 POINT(47.73779234721402 -122.45035648442816) bank112065 +112066 POINT(47.5282039845774 -122.0335077978873) bank112066 +112067 POINT(47.987701591858375 -123.054492179799) bank112067 +112068 POINT(46.92346644365456 -122.52538461377422) bank112068 +112069 POINT(47.719108304268836 -121.5196401319741) bank112069 +112070 POINT(47.29585690045652 -121.49515752118107) bank112070 +112071 POINT(47.10731235911013 -123.1185324255459) bank112071 +112072 POINT(46.721760323767576 -122.60811979888864) bank112072 +112073 POINT(47.16741019817708 -122.6739402678169) bank112073 +112074 POINT(48.14717696489513 -122.75832516034443) bank112074 +112075 POINT(46.64856305743197 -122.37639927058719) bank112075 +112076 POINT(48.5348585989183 -122.19391699702179) bank112076 +112077 POINT(48.211951142502244 -122.99142566346264) bank112077 +112078 POINT(46.77629842306188 -123.23271198315493) bank112078 +112079 POINT(48.139728056706595 -122.63114191884277) bank112079 +112080 POINT(47.425521727841755 -121.54452449446256) bank112080 +112081 POINT(48.53616114102518 -121.68878863997217) bank112081 +112082 POINT(48.60229375539983 -122.89639168989358) bank112082 +112083 POINT(47.32139680141984 -121.60676186902552) bank112083 +112084 POINT(48.25267364688806 -121.46196042903989) bank112084 +112085 POINT(47.302016530110606 -123.26716410176043) bank112085 +112086 POINT(47.45565267464361 -121.93631354766592) bank112086 +112087 POINT(47.44501165916795 -122.37370224392389) bank112087 +112088 POINT(47.8556691998221 -122.44153262764002) bank112088 +112089 POINT(46.636478852535625 -122.90141910786002) bank112089 +112090 POINT(48.28537941864364 -122.59806430792632) bank112090 +112091 POINT(47.17934997543415 -123.32602306220653) bank112091 +112092 POINT(46.919052170398864 -122.24946087832288) bank112092 +112093 POINT(47.26664434068294 -121.73788173369361) bank112093 +112094 POINT(47.76461039683291 -122.68078877575216) bank112094 +112095 POINT(46.903155481573016 -122.36584476533511) bank112095 +112096 POINT(46.8962150886541 -123.25623091019568) bank112096 +112097 POINT(48.00454160646946 -122.04626199729435) bank112097 +112098 POINT(48.39113621586855 -122.84193629441643) bank112098 +112099 POINT(47.02348374715653 -121.61060087929037) bank112099 +112100 POINT(47.58344843441442 -122.59831846945296) bank112100 +112101 POINT(48.34976418184297 -121.94666975576314) bank112101 +112102 POINT(47.79385097640584 -121.55392247647777) bank112102 +112103 POINT(47.19312252423696 -123.01769809738565) bank112103 +112104 POINT(48.576587491356726 -121.41222290113778) bank112104 +112105 POINT(48.5504837997899 -121.89880576010802) bank112105 +112106 POINT(48.36764161938091 -122.43815578913224) bank112106 +112107 POINT(48.372145276751155 -122.74809041656043) bank112107 +112108 POINT(47.32536647163424 -122.2536235706375) bank112108 +112109 POINT(47.71810940896984 -122.99696719601816) bank112109 +112110 POINT(47.879748292326916 -122.18930123821914) bank112110 +112111 POINT(47.7705615347513 -122.45584399876337) bank112111 +112112 POINT(47.929606365892596 -121.62666253251659) bank112112 +112113 POINT(47.0461024294226 -122.25818942582468) bank112113 +112114 POINT(47.639788458168695 -122.06468398127494) bank112114 +112115 POINT(46.79038391509763 -123.12612168839796) bank112115 +112116 POINT(46.78491186839283 -122.71599852129461) bank112116 +112117 POINT(47.050456996927906 -122.77511636780736) bank112117 +112118 POINT(48.02348463458444 -122.21839462861419) bank112118 +112119 POINT(46.91154104201151 -121.40758227428175) bank112119 +112120 POINT(47.16385077223685 -122.50146862311605) bank112120 +112121 POINT(48.54747589361306 -121.87444876037644) bank112121 +112122 POINT(48.53819951127893 -121.48666388278801) bank112122 +112123 POINT(48.28791090806495 -122.44537914029121) bank112123 +112124 POINT(47.50331281211549 -123.03751712673868) bank112124 +112125 POINT(46.94745338950503 -123.23619069694912) bank112125 +112126 POINT(48.27014028284669 -122.43212255766024) bank112126 +112127 POINT(48.544404755251755 -123.12127737601527) bank112127 +112128 POINT(47.4005887516938 -122.53894536310095) bank112128 +112129 POINT(47.63342758913093 -123.06440681318865) bank112129 +112130 POINT(48.59702542893635 -122.0401449684628) bank112130 +112131 POINT(48.45481099486943 -122.15233966906715) bank112131 +112132 POINT(47.528801655288866 -122.76727652861463) bank112132 +112133 POINT(46.76021947827788 -122.83580714931307) bank112133 +112134 POINT(47.652733258144785 -123.26470571228062) bank112134 +112135 POINT(48.352528591707355 -122.2727138817892) bank112135 +112136 POINT(46.829243910344424 -123.22884866405828) bank112136 +112137 POINT(47.80253791741999 -122.90270277560028) bank112137 +112138 POINT(47.94137129852706 -122.23628266547638) bank112138 +112139 POINT(47.76604463160171 -121.82654910646993) bank112139 +112140 POINT(47.67547861488744 -122.93328728186474) bank112140 +112141 POINT(48.18625068916091 -123.30040016998338) bank112141 +112142 POINT(48.50162042625712 -122.83855977223634) bank112142 +112143 POINT(46.88014043768882 -122.99042239653897) bank112143 +112144 POINT(47.08863735818928 -121.9893759410783) bank112144 +112145 POINT(47.104859226752204 -122.92595437876413) bank112145 +112146 POINT(46.984478757557724 -123.06995332104697) bank112146 +112147 POINT(48.06445068773079 -121.56151332876757) bank112147 +112148 POINT(48.13693368543096 -121.48598684985993) bank112148 +112149 POINT(48.238681248647964 -122.71808800563274) bank112149 +112150 POINT(47.4538539607359 -121.53440609736384) bank112150 +112151 POINT(47.240342390259194 -121.74456530697) bank112151 +112152 POINT(47.031741223499274 -121.78812717327837) bank112152 +112153 POINT(48.46485782656603 -122.92285831475571) bank112153 +112154 POINT(47.06714160325801 -122.7670402707115) bank112154 +112155 POINT(47.16679834449467 -123.15058421027827) bank112155 +112156 POINT(47.18405919612172 -122.36223390162868) bank112156 +112157 POINT(46.70078373296732 -122.35593871039919) bank112157 +112158 POINT(47.82395955266276 -121.37057577263442) bank112158 +112159 POINT(47.13344335534303 -122.32993544310287) bank112159 +112160 POINT(47.643369420635516 -122.06315185181322) bank112160 +112161 POINT(48.03877532476813 -121.9695958056416) bank112161 +112162 POINT(48.226599540869614 -122.71067564788908) bank112162 +112163 POINT(46.93296145680793 -122.6499902346108) bank112163 +112164 POINT(47.744089392739546 -122.72034378326642) bank112164 +112165 POINT(47.737164366668175 -121.68564739859144) bank112165 +112166 POINT(47.064726952113126 -122.60570550167833) bank112166 +112167 POINT(47.12058260179974 -121.41768960443164) bank112167 +112168 POINT(47.42745914062269 -122.38647296952072) bank112168 +112169 POINT(47.317200419030165 -122.17052422182383) bank112169 +112170 POINT(48.20464635934987 -122.46174026305403) bank112170 +112171 POINT(47.27366758276558 -122.31896127939366) bank112171 +112172 POINT(47.20423403307866 -121.73852868130399) bank112172 +112173 POINT(47.673924295789845 -121.5663471295199) bank112173 +112174 POINT(46.842229658957486 -122.10551662714877) bank112174 +112175 POINT(47.54685890803076 -121.38328565675035) bank112175 +112176 POINT(46.76067473770772 -122.15641284328338) bank112176 +112177 POINT(47.23569756170067 -122.53975607884196) bank112177 +112178 POINT(47.5877437147311 -123.32750196002505) bank112178 +112179 POINT(48.57627400349073 -121.92148860847931) bank112179 +112180 POINT(47.11230724916009 -123.21682828139664) bank112180 +112181 POINT(48.25999676667654 -122.24732262927917) bank112181 +112182 POINT(48.562383764946816 -122.98252753349014) bank112182 +112183 POINT(46.79368746432282 -122.52606337644674) bank112183 +112184 POINT(47.50452218065019 -122.49691268614603) bank112184 +112185 POINT(47.71843443024631 -122.28225644928871) bank112185 +112186 POINT(47.834407319996224 -122.35985202154717) bank112186 +112187 POINT(46.72524780755581 -122.63665339634197) bank112187 +112188 POINT(48.12972371896351 -122.49918858539236) bank112188 +112189 POINT(46.77789349449284 -123.11652554124748) bank112189 +112190 POINT(47.61728529046226 -122.60043304265054) bank112190 +112191 POINT(48.030795929618286 -121.571973477018) bank112191 +112192 POINT(47.55777603033282 -121.63708153370241) bank112192 +112193 POINT(48.23430151189665 -121.63101321412552) bank112193 +112194 POINT(47.68363859353118 -121.463055534561) bank112194 +112195 POINT(48.2988794065162 -123.23206159053888) bank112195 +112196 POINT(47.04610735888952 -121.83630416594154) bank112196 +112197 POINT(47.008678040972036 -122.04477565548773) bank112197 +112198 POINT(47.09724995057681 -122.00096143799276) bank112198 +112199 POINT(48.569431174625336 -122.86132318515912) bank112199 +112200 POINT(47.73088847771496 -122.40251934356655) bank112200 +112201 POINT(46.65637617816664 -121.4360759288063) bank112201 +112202 POINT(48.240668277817065 -122.16657059941163) bank112202 +112203 POINT(48.42486899986469 -122.37544327528576) bank112203 +112204 POINT(48.254929644031016 -121.62926722601526) bank112204 +112205 POINT(47.50181041044454 -121.47043561451049) bank112205 +112206 POINT(48.28775966433425 -123.10780900661199) bank112206 +112207 POINT(48.01925511602946 -121.465941378034) bank112207 +112208 POINT(48.4562760468697 -122.18291920078218) bank112208 +112209 POINT(47.976885449167305 -122.90527156805103) bank112209 +112210 POINT(47.65487931994895 -121.36875739214884) bank112210 +112211 POINT(48.246124155696 -122.78408563449749) bank112211 +112212 POINT(47.16185294717345 -121.53430685854813) bank112212 +112213 POINT(48.24621761753883 -121.47754436452003) bank112213 +112214 POINT(47.30474988108038 -122.70337374051304) bank112214 +112215 POINT(47.52531085343042 -122.86511296034195) bank112215 +112216 POINT(48.09770474688774 -121.99112192011677) bank112216 +112217 POINT(48.10868984125826 -122.35800280107016) bank112217 +112218 POINT(47.911389043089315 -121.7703265140967) bank112218 +112219 POINT(47.77047097530659 -123.12802046499093) bank112219 +112220 POINT(48.070507456936255 -122.4902431752889) bank112220 +112221 POINT(47.469241167607585 -121.90432398609882) bank112221 +112222 POINT(47.291524735352326 -122.9076335340648) bank112222 +112223 POINT(47.479016361655994 -122.44430783197772) bank112223 +112224 POINT(46.809429604862544 -123.05252569305364) bank112224 +112225 POINT(48.19731736625528 -121.83133772741547) bank112225 +112226 POINT(47.29721263036621 -121.82225286094283) bank112226 +112227 POINT(47.937508457188 -121.42029237652316) bank112227 +112228 POINT(46.65781538624943 -121.61437770336272) bank112228 +112229 POINT(47.214800009123685 -121.51536164113058) bank112229 +112230 POINT(47.18391614935755 -122.99687154504896) bank112230 +112231 POINT(47.549846611846064 -121.39157353539295) bank112231 +112232 POINT(48.44078394882059 -121.73913842957742) bank112232 +112233 POINT(46.94609568820127 -122.58206134908427) bank112233 +112234 POINT(48.0799625431467 -122.92593767256334) bank112234 +112235 POINT(47.641354479442306 -122.93014051402392) bank112235 +112236 POINT(47.21997976566791 -123.0046216393387) bank112236 +112237 POINT(47.990101863473775 -122.06728977533584) bank112237 +112238 POINT(48.06409309345694 -122.16890241534368) bank112238 +112239 POINT(47.20867878688069 -122.773524726397) bank112239 +112240 POINT(46.65335463888254 -122.64927226779015) bank112240 +112241 POINT(48.038678306247704 -122.38952482872375) bank112241 +112242 POINT(47.41410468504978 -122.49359259491453) bank112242 +112243 POINT(48.22082837313019 -121.8625677791711) bank112243 +112244 POINT(47.025061070096136 -122.83851094048677) bank112244 +112245 POINT(47.32445254208071 -122.17776550724624) bank112245 +112246 POINT(47.02607674838479 -123.06097733447884) bank112246 +112247 POINT(48.17839687675336 -122.8286101463344) bank112247 +112248 POINT(47.48356510852453 -123.16009688745744) bank112248 +112249 POINT(46.681687002482185 -122.215591241963) bank112249 +112250 POINT(46.94911763083209 -123.19151988637569) bank112250 +112251 POINT(46.917670617962635 -121.43617223860338) bank112251 +112252 POINT(47.915165336085465 -121.52976780650548) bank112252 +112253 POINT(47.38380545214448 -123.12317383882487) bank112253 +112254 POINT(48.03219368270544 -123.26206833633644) bank112254 +112255 POINT(47.97948377983553 -123.11507206443005) bank112255 +112256 POINT(47.05492753509589 -121.95777034588934) bank112256 +112257 POINT(47.83235779279338 -122.75451445420258) bank112257 +112258 POINT(46.7613900094684 -121.4688822346973) bank112258 +112259 POINT(47.4467135402509 -122.6897628563327) bank112259 +112260 POINT(46.903244032474205 -121.61943617379919) bank112260 +112261 POINT(48.10698611230775 -122.32493981908442) bank112261 +112262 POINT(48.08825455322978 -121.63098267885474) bank112262 +112263 POINT(48.412573462407586 -123.1558188530265) bank112263 +112264 POINT(47.83832316919391 -121.64549734756132) bank112264 +112265 POINT(48.587924090963156 -121.77830295541762) bank112265 +112266 POINT(46.83962520949153 -122.33336294473905) bank112266 +112267 POINT(47.675838486475406 -122.60353225568353) bank112267 +112268 POINT(48.505719294865614 -121.6639873964419) bank112268 +112269 POINT(47.4143182643004 -121.99724097992443) bank112269 +112270 POINT(47.10037823802929 -122.76355720184995) bank112270 +112271 POINT(48.06605461503282 -122.65138374032999) bank112271 +112272 POINT(47.463125295415544 -121.95525338751722) bank112272 +112273 POINT(48.126255123132744 -122.03592415440616) bank112273 +112274 POINT(47.2434567021379 -121.46284121051488) bank112274 +112275 POINT(47.62092256627971 -121.53774166711334) bank112275 +112276 POINT(46.845136267638324 -121.63108844459052) bank112276 +112277 POINT(48.06965375629861 -121.84033615800588) bank112277 +112278 POINT(48.402460770678466 -122.15099240859557) bank112278 +112279 POINT(47.17970431064705 -122.58722714084801) bank112279 +112280 POINT(48.00499726003578 -122.2918692150823) bank112280 +112281 POINT(47.44748031576568 -122.63674348794724) bank112281 +112282 POINT(46.66994188141638 -122.29575120744914) bank112282 +112283 POINT(47.903473789305686 -121.58933227432898) bank112283 +112284 POINT(47.395580036447974 -122.52976728196127) bank112284 +112285 POINT(47.07974742895896 -122.41327712365452) bank112285 +112286 POINT(47.79757842682835 -121.95225370443838) bank112286 +112287 POINT(47.103873576100334 -122.64983297983788) bank112287 +112288 POINT(47.99741426133806 -121.55547666927967) bank112288 +112289 POINT(47.2968050023795 -122.98374424672298) bank112289 +112290 POINT(48.12721316244839 -121.5037698083316) bank112290 +112291 POINT(48.19779877481724 -123.2462536708343) bank112291 +112292 POINT(47.91811050005522 -121.48598262293282) bank112292 +112293 POINT(47.69251146977821 -123.04108513304315) bank112293 +112294 POINT(47.726849768777214 -122.36211241970942) bank112294 +112295 POINT(46.680419136485284 -122.67165528165448) bank112295 +112296 POINT(46.67377583249053 -122.95122722498326) bank112296 +112297 POINT(48.264870519875956 -121.85097145272937) bank112297 +112298 POINT(47.04801835642381 -122.24921206489792) bank112298 +112299 POINT(47.08280718424404 -122.2677400682287) bank112299 +112300 POINT(48.42070677579575 -122.55926870311805) bank112300 +112301 POINT(46.747100550807055 -123.06703447610168) bank112301 +112302 POINT(48.349320506608564 -122.69660627414818) bank112302 +112303 POINT(46.828134080405135 -121.9560723332342) bank112303 +112304 POINT(48.271105613605215 -123.13896833894661) bank112304 +112305 POINT(48.525920293906985 -121.40793701692924) bank112305 +112306 POINT(47.999121783067665 -122.56617480810625) bank112306 +112307 POINT(46.998230113017314 -121.68242152412647) bank112307 +112308 POINT(47.12592997256488 -121.61405454181063) bank112308 +112309 POINT(48.494090844931755 -121.77161404273967) bank112309 +112310 POINT(46.82663759587538 -122.8812753399828) bank112310 +112311 POINT(46.820144107354714 -122.96184560542) bank112311 +112312 POINT(47.200451281189906 -122.16550415780128) bank112312 +112313 POINT(48.37904347679005 -121.96181397703542) bank112313 +112314 POINT(48.60076084788321 -121.47403189581586) bank112314 +112315 POINT(47.251059763040374 -122.6958328633938) bank112315 +112316 POINT(48.44618668598098 -122.14451720495619) bank112316 +112317 POINT(47.97751116058849 -122.58588866111505) bank112317 +112318 POINT(47.82046502408422 -122.13032517786664) bank112318 +112319 POINT(47.98982457054032 -121.34723827613186) bank112319 +112320 POINT(46.73645376113534 -121.88941551362412) bank112320 +112321 POINT(46.98640899408168 -121.61890979708309) bank112321 +112322 POINT(47.40018191248207 -122.33760539111638) bank112322 +112323 POINT(47.907137090905486 -122.052034618953) bank112323 +112324 POINT(47.93463267464266 -121.7226537981928) bank112324 +112325 POINT(46.66546800470301 -122.94531585340754) bank112325 +112326 POINT(48.07433623965685 -123.16243060005232) bank112326 +112327 POINT(47.671727115081744 -121.87667382854715) bank112327 +112328 POINT(47.00368421140624 -121.82694712279125) bank112328 +112329 POINT(47.125230407007294 -122.21083664153619) bank112329 +112330 POINT(48.09819023270139 -121.7579669450131) bank112330 +112331 POINT(47.1525545736077 -122.54262488064596) bank112331 +112332 POINT(47.54522534568824 -121.54487453582956) bank112332 +112333 POINT(47.48131183276453 -122.83372057277062) bank112333 +112334 POINT(46.84131448974365 -122.51516221023525) bank112334 +112335 POINT(47.075419073339845 -123.24434898137326) bank112335 +112336 POINT(48.318807754006365 -122.28914408635123) bank112336 +112337 POINT(47.67178371223842 -122.20351886274702) bank112337 +112338 POINT(48.153822543535135 -123.09725556786537) bank112338 +112339 POINT(47.021351539145435 -121.74830510804067) bank112339 +112340 POINT(47.79607624607001 -122.29586928415813) bank112340 +112341 POINT(47.382587525308374 -121.36076198053443) bank112341 +112342 POINT(48.24815018458493 -123.18573493228335) bank112342 +112343 POINT(48.347714040735205 -121.65182373715349) bank112343 +112344 POINT(48.45677468776735 -122.97762759990725) bank112344 +112345 POINT(48.06280585762132 -123.15990911816625) bank112345 +112346 POINT(48.51043320468557 -122.4256552028653) bank112346 +112347 POINT(47.54537120701166 -121.55193953432803) bank112347 +112348 POINT(47.366579288481425 -122.63236432795311) bank112348 +112349 POINT(47.16040860337177 -122.28105124621365) bank112349 +112350 POINT(47.202845639858374 -121.461978530404) bank112350 +112351 POINT(47.744470997690215 -122.58005461173319) bank112351 +112352 POINT(47.47213214830868 -121.34331208852198) bank112352 +112353 POINT(46.67508737610667 -121.51253536446252) bank112353 +112354 POINT(46.73829320391381 -122.2315582502655) bank112354 +112355 POINT(46.92798932805131 -121.601743823164) bank112355 +112356 POINT(47.015368246915855 -121.91460003397772) bank112356 +112357 POINT(47.78712389483634 -122.39902580646617) bank112357 +112358 POINT(47.083130016477 -121.8309712170896) bank112358 +112359 POINT(48.43429537413253 -122.14265288460591) bank112359 +112360 POINT(48.39153213603842 -122.25121532737886) bank112360 +112361 POINT(47.59281245930233 -122.12120401612445) bank112361 +112362 POINT(46.81048407587927 -122.97368620465306) bank112362 +112363 POINT(47.53977079537326 -122.09070953356144) bank112363 +112364 POINT(46.800952324126015 -123.09409051465421) bank112364 +112365 POINT(47.2998336779835 -122.2064107509561) bank112365 +112366 POINT(48.13848620985397 -122.70340190394825) bank112366 +112367 POINT(47.518148580876506 -122.74345523740261) bank112367 +112368 POINT(48.50122474461766 -121.61279403261007) bank112368 +112369 POINT(46.66423266072477 -122.11833464380193) bank112369 +112370 POINT(47.12546950989955 -123.29834772069188) bank112370 +112371 POINT(47.001189317983474 -122.42026837501228) bank112371 +112372 POINT(47.20779618709072 -122.6784611136916) bank112372 +112373 POINT(47.84398402791552 -121.50191783072948) bank112373 +112374 POINT(47.20617998011472 -122.95426903675488) bank112374 +112375 POINT(46.73959092898972 -121.8942002638941) bank112375 +112376 POINT(47.86288290639082 -121.69551446587369) bank112376 +112377 POINT(46.760591116805834 -122.77862222474957) bank112377 +112378 POINT(47.27885985687963 -122.57205673273371) bank112378 +112379 POINT(48.53479428837759 -121.59877784614707) bank112379 +112380 POINT(47.578117423293286 -122.70429561399783) bank112380 +112381 POINT(47.29656452190971 -122.90125658362378) bank112381 +112382 POINT(47.459730145475866 -122.61288710050412) bank112382 +112383 POINT(48.13196391498086 -122.68192236526495) bank112383 +112384 POINT(48.05980488610038 -122.06013786319258) bank112384 +112385 POINT(47.08430515932723 -122.56075735675574) bank112385 +112386 POINT(48.043671904554486 -122.94255698789541) bank112386 +112387 POINT(47.02789714941477 -121.52798365380212) bank112387 +112388 POINT(47.60424928580709 -122.0416958359679) bank112388 +112389 POINT(48.117700265771305 -123.24752491469017) bank112389 +112390 POINT(48.30929589740359 -121.84587698607143) bank112390 +112391 POINT(47.837400535443145 -122.10137994205547) bank112391 +112392 POINT(48.13331028064698 -122.47972214516818) bank112392 +112393 POINT(48.06196056886228 -122.04251100030274) bank112393 +112394 POINT(47.95531466387073 -122.25476615144811) bank112394 +112395 POINT(48.3889894711704 -123.26369192714962) bank112395 +112396 POINT(46.622482269237956 -122.88312949004796) bank112396 +112397 POINT(48.08922246438227 -121.9144783588496) bank112397 +112398 POINT(47.30591624844732 -123.29240586887445) bank112398 +112399 POINT(47.03631024401831 -121.82104842395803) bank112399 +112400 POINT(47.817873467846226 -121.84328596489144) bank112400 +112401 POINT(47.581563748160704 -121.79222538075788) bank112401 +112402 POINT(48.251778110838664 -121.44650406647318) bank112402 +112403 POINT(46.6112631457082 -121.59790200631203) bank112403 +112404 POINT(47.93273287465364 -122.40820452114815) bank112404 +112405 POINT(47.03023703066099 -121.56429236514718) bank112405 +112406 POINT(47.92688960399492 -122.87534732302201) bank112406 +112407 POINT(47.129705039938976 -123.1366270167584) bank112407 +112408 POINT(46.75433604127904 -121.78886892513914) bank112408 +112409 POINT(47.67466015410067 -122.61565754991356) bank112409 +112410 POINT(46.98877631562968 -121.51840243310565) bank112410 +112411 POINT(46.85956191398213 -122.25052242911188) bank112411 +112412 POINT(48.0852994897839 -122.19084665279408) bank112412 +112413 POINT(48.24056233679446 -122.59562374616931) bank112413 +112414 POINT(48.13839700801649 -121.5942252669101) bank112414 +112415 POINT(46.929594757747374 -122.8148335021648) bank112415 +112416 POINT(46.896670614354775 -121.47103197373815) bank112416 +112417 POINT(48.43366674758718 -122.81222789387911) bank112417 +112418 POINT(48.00722417820514 -123.00628640188546) bank112418 +112419 POINT(46.96612190387841 -122.60598490588478) bank112419 +112420 POINT(48.55124845872692 -122.52803547456188) bank112420 +112421 POINT(47.11395908367678 -123.31194500056485) bank112421 +112422 POINT(48.497447526381045 -121.66679100270292) bank112422 +112423 POINT(47.320920020748474 -121.44932752640047) bank112423 +112424 POINT(47.8442391678508 -122.14112627286201) bank112424 +112425 POINT(47.14392001088666 -122.03605779317918) bank112425 +112426 POINT(48.131634680358665 -122.28966958702462) bank112426 +112427 POINT(47.12046318136079 -122.39141267271884) bank112427 +112428 POINT(47.3422013456181 -122.36991661271378) bank112428 +112429 POINT(48.340996295573284 -122.61747199498247) bank112429 +112430 POINT(47.14673459406071 -122.55342938374051) bank112430 +112431 POINT(48.54025432560017 -122.06103942687074) bank112431 +112432 POINT(46.73988091711302 -122.16782603967866) bank112432 +112433 POINT(47.87203511240915 -121.6288179702618) bank112433 +112434 POINT(48.44675009230989 -122.65689454641225) bank112434 +112435 POINT(47.47130154673085 -121.52357665305769) bank112435 +112436 POINT(48.39374496018742 -123.28329743097582) bank112436 +112437 POINT(46.945241077447115 -122.43215137853728) bank112437 +112438 POINT(48.351953789761296 -122.81127641578823) bank112438 +112439 POINT(48.064867696621285 -122.41397471621617) bank112439 +112440 POINT(46.87977903236568 -122.67595393644379) bank112440 +112441 POINT(46.95563351449924 -123.17430535638749) bank112441 +112442 POINT(47.80384354203024 -122.00466210492465) bank112442 +112443 POINT(47.15278446287877 -121.99303571843365) bank112443 +112444 POINT(48.59327331154024 -122.09128136308833) bank112444 +112445 POINT(47.51146969441076 -122.47308245486298) bank112445 +112446 POINT(48.422187013691804 -122.84978238371527) bank112446 +112447 POINT(48.46397261310378 -122.42354318094648) bank112447 +112448 POINT(47.27431191360028 -121.51756862687662) bank112448 +112449 POINT(47.75941650340509 -121.77455591434108) bank112449 +112450 POINT(46.938261787022256 -122.28729341173796) bank112450 +112451 POINT(46.66243746100547 -122.97161655235746) bank112451 +112452 POINT(48.33549554701692 -122.74646995720926) bank112452 +112453 POINT(47.85224627646961 -122.19036867448861) bank112453 +112454 POINT(46.97835538483644 -122.23520491884946) bank112454 +112455 POINT(48.13334680030229 -121.5530073198717) bank112455 +112456 POINT(48.202719449527876 -121.62799990622548) bank112456 +112457 POINT(47.55930298875515 -121.9093737720573) bank112457 +112458 POINT(48.17365849397925 -121.92901079680558) bank112458 +112459 POINT(47.591931181093116 -122.33794098818132) bank112459 +112460 POINT(48.488585411619894 -123.0624904510724) bank112460 +112461 POINT(46.96265411706491 -122.60165569772904) bank112461 +112462 POINT(48.13937660972274 -122.56987280350224) bank112462 +112463 POINT(47.57252378245357 -123.0678407423251) bank112463 +112464 POINT(46.88031639562539 -122.62722331550655) bank112464 +112465 POINT(48.489744091927186 -122.90354675217898) bank112465 +112466 POINT(47.348206136093026 -122.16790710570997) bank112466 +112467 POINT(47.02514517203706 -123.06502137501056) bank112467 +112468 POINT(47.317351489420574 -121.83000068891171) bank112468 +112469 POINT(47.05159214021594 -122.72833834557782) bank112469 +112470 POINT(48.12807159576463 -121.97482336186374) bank112470 +112471 POINT(47.21511828044225 -121.85135319608383) bank112471 +112472 POINT(46.966952252800155 -122.66749995529865) bank112472 +112473 POINT(46.97361830970024 -123.22004240403596) bank112473 +112474 POINT(47.237454344400895 -123.15690263950164) bank112474 +112475 POINT(48.231511584436106 -121.91713428316123) bank112475 +112476 POINT(47.389395953489064 -123.31175456052861) bank112476 +112477 POINT(48.0131217149813 -122.12047481947525) bank112477 +112478 POINT(47.474265781696836 -122.9095630039402) bank112478 +112479 POINT(47.37498168617724 -123.20969977143245) bank112479 +112480 POINT(48.23453567447154 -121.98507615744577) bank112480 +112481 POINT(46.655328192158635 -122.9892979311005) bank112481 +112482 POINT(48.23369788221851 -121.88100850544882) bank112482 +112483 POINT(47.88684238983318 -121.84747780599382) bank112483 +112484 POINT(47.52393960234841 -121.67739399760019) bank112484 +112485 POINT(48.25279132704096 -121.6995332992578) bank112485 +112486 POINT(47.619909664392544 -123.21726290568031) bank112486 +112487 POINT(46.9233601903938 -121.97946326987406) bank112487 +112488 POINT(46.821414163040686 -121.40444071402071) bank112488 +112489 POINT(48.111204174284815 -122.67090528078907) bank112489 +112490 POINT(48.127327642362054 -121.69970223424443) bank112490 +112491 POINT(47.34520339675158 -122.7032328429371) bank112491 +112492 POINT(47.75281162177751 -123.10321293021906) bank112492 +112493 POINT(47.135004363522455 -122.2074369442463) bank112493 +112494 POINT(47.21340956579718 -122.56424173553239) bank112494 +112495 POINT(47.1313528877772 -122.74842902135188) bank112495 +112496 POINT(47.36043959314283 -122.70948323546342) bank112496 +112497 POINT(47.12551309339512 -123.27082085887403) bank112497 +112498 POINT(48.58799980847864 -121.37447208794553) bank112498 +112499 POINT(47.0771254395203 -122.9190718686382) bank112499 +112500 POINT(47.944770861774835 -122.96546359516142) bank112500 +112501 POINT(47.789082633720774 -121.68494767345244) bank112501 +112502 POINT(47.91126812156636 -122.25513324704504) bank112502 +112503 POINT(48.31852024823015 -123.28836909662219) bank112503 +112504 POINT(48.08078665808884 -121.78812987952952) bank112504 +112505 POINT(47.88627982668076 -121.48815525040216) bank112505 +112506 POINT(46.74776409194032 -121.50821117550609) bank112506 +112507 POINT(47.64616325876438 -121.57397696986463) bank112507 +112508 POINT(46.83063893041904 -121.5536453646109) bank112508 +112509 POINT(47.584451162177544 -121.61267513979332) bank112509 +112510 POINT(47.62045964779017 -121.91062209019962) bank112510 +112511 POINT(47.83662259424389 -121.83838364310547) bank112511 +112512 POINT(46.82654373443104 -122.86088961721462) bank112512 +112513 POINT(46.823252506357626 -122.96420672779833) bank112513 +112514 POINT(48.09042637778309 -123.19652710816581) bank112514 +112515 POINT(47.498895302829915 -122.47591829125032) bank112515 +112516 POINT(47.69335551232947 -122.979074353314) bank112516 +112517 POINT(47.093604355927404 -122.99605025790775) bank112517 +112518 POINT(46.79273299044032 -121.45806709710814) bank112518 +112519 POINT(47.51633851240934 -121.64900662278076) bank112519 +112520 POINT(47.65159265459175 -122.77370914576669) bank112520 +112521 POINT(47.589500978605784 -123.21285213008017) bank112521 +112522 POINT(47.24838175453673 -123.14852400359564) bank112522 +112523 POINT(47.96627865448416 -122.52368987960952) bank112523 +112524 POINT(46.96143448379635 -121.3713044807834) bank112524 +112525 POINT(48.20326543343891 -121.90330123183978) bank112525 +112526 POINT(48.20315759354294 -123.33084196011684) bank112526 +112527 POINT(47.24612497558094 -121.92766008373056) bank112527 +112528 POINT(47.63043640117378 -122.56273285102067) bank112528 +112529 POINT(47.092437836431394 -122.890258947862) bank112529 +112530 POINT(48.405286937033516 -121.84739469396645) bank112530 +112531 POINT(47.32366806200119 -121.48450400340991) bank112531 +112532 POINT(48.345542606377165 -121.80872011992034) bank112532 +112533 POINT(47.347803496135725 -122.11348212296835) bank112533 +112534 POINT(48.06650693260748 -122.91997127058558) bank112534 +112535 POINT(47.48924040330991 -122.2664649627137) bank112535 +112536 POINT(47.834054427811004 -123.10857356782222) bank112536 +112537 POINT(46.78071905041982 -121.80854722337936) bank112537 +112538 POINT(47.278199298821086 -122.35476636371004) bank112538 +112539 POINT(48.43375221208537 -122.78024962724591) bank112539 +112540 POINT(48.43106320384681 -123.02963236891657) bank112540 +112541 POINT(47.61127887044826 -122.02617495337839) bank112541 +112542 POINT(47.24296958703176 -123.08198678191464) bank112542 +112543 POINT(48.48993852704629 -122.83309731647972) bank112543 +112544 POINT(47.64361530938675 -122.76635512148742) bank112544 +112545 POINT(47.35853247143535 -121.34582371586441) bank112545 +112546 POINT(47.62247731682609 -122.61176557261469) bank112546 +112547 POINT(47.13173041937909 -122.71610969794061) bank112547 +112548 POINT(46.70292619452777 -123.2574294536513) bank112548 +112549 POINT(47.91606415340886 -121.39660516219622) bank112549 +112550 POINT(47.923032326999355 -122.92234838796594) bank112550 +112551 POINT(47.60881364344053 -122.79196962126208) bank112551 +112552 POINT(47.450666445284746 -122.89480619497431) bank112552 +112553 POINT(47.07537544431519 -121.98636436250044) bank112553 +112554 POINT(46.75581554929917 -123.18037531052242) bank112554 +112555 POINT(47.14174271789989 -123.06515210941998) bank112555 +112556 POINT(48.11718835660231 -121.89326064019184) bank112556 +112557 POINT(47.08400937331759 -123.19577953191893) bank112557 +112558 POINT(48.18454424212207 -122.38944078466152) bank112558 +112559 POINT(48.12965860726064 -121.36497295282703) bank112559 +112560 POINT(46.75688880777626 -122.56276279879336) bank112560 +112561 POINT(48.057304194916625 -122.53064499483763) bank112561 +112562 POINT(47.95312145517088 -121.4110326200845) bank112562 +112563 POINT(47.34303168373312 -122.07483813009172) bank112563 +112564 POINT(46.840153873676016 -122.88221238513945) bank112564 +112565 POINT(47.93864120435263 -121.68490046914827) bank112565 +112566 POINT(47.410951296987186 -121.91439037816271) bank112566 +112567 POINT(46.9466851406793 -122.7581278604086) bank112567 +112568 POINT(48.494973533650345 -121.96865196126328) bank112568 +112569 POINT(46.890171530298794 -123.07994067907018) bank112569 +112570 POINT(46.703863605704754 -121.37177154280414) bank112570 +112571 POINT(48.14708988227221 -122.21721589140337) bank112571 +112572 POINT(47.944223588350184 -122.59963150237708) bank112572 +112573 POINT(48.48767750883104 -122.79168679370557) bank112573 +112574 POINT(48.33726772021577 -122.2751702464975) bank112574 +112575 POINT(48.16813048915847 -121.83683984166376) bank112575 +112576 POINT(48.541706034867154 -121.9298014642638) bank112576 +112577 POINT(46.809786221093795 -122.70524580385597) bank112577 +112578 POINT(47.22284109901068 -123.31359338089482) bank112578 +112579 POINT(48.328215876127025 -122.05229560417675) bank112579 +112580 POINT(46.801157581465425 -123.16688094722997) bank112580 +112581 POINT(47.61072854980046 -122.56025538954374) bank112581 +112582 POINT(47.905510474323975 -121.46729035027987) bank112582 +112583 POINT(48.46283770749231 -121.68797941472637) bank112583 +112584 POINT(46.859751739054175 -122.1636595536054) bank112584 +112585 POINT(47.45371884056294 -122.20636824710007) bank112585 +112586 POINT(48.35938485281938 -122.46835907348773) bank112586 +112587 POINT(47.23469553437743 -122.71690404238296) bank112587 +112588 POINT(47.51148559119882 -121.79305565175085) bank112588 +112589 POINT(47.825586999822036 -121.99131723348671) bank112589 +112590 POINT(47.6538832929462 -122.23224736407845) bank112590 +112591 POINT(46.97918632003958 -123.10929253949922) bank112591 +112592 POINT(47.87960095885463 -121.89765832010336) bank112592 +112593 POINT(47.50839452650143 -121.36649610755603) bank112593 +112594 POINT(47.236753051820436 -122.85902100921041) bank112594 +112595 POINT(47.68174187640241 -123.09004276209828) bank112595 +112596 POINT(47.931814573353435 -122.75670762785423) bank112596 +112597 POINT(47.05699818182331 -121.42233923762737) bank112597 +112598 POINT(46.817296662886164 -121.94835973222472) bank112598 +112599 POINT(48.44053097580839 -122.73247515611779) bank112599 +112600 POINT(47.57406332948368 -122.41406183306702) bank112600 +112601 POINT(47.93197713225843 -121.97224530934898) bank112601 +112602 POINT(47.86028153052971 -122.60666929748734) bank112602 +112603 POINT(48.02743100108802 -122.17930958128821) bank112603 +112604 POINT(47.52349718585621 -121.82520785075174) bank112604 +112605 POINT(48.333600184626135 -123.06923602350723) bank112605 +112606 POINT(48.10664016876607 -122.58522538814182) bank112606 +112607 POINT(47.90508136760836 -122.25308373427254) bank112607 +112608 POINT(47.17461144181863 -121.85518311103019) bank112608 +112609 POINT(47.89792444174633 -121.89425899138314) bank112609 +112610 POINT(47.95142588475788 -123.25141605276818) bank112610 +112611 POINT(47.28375951776291 -123.07524552653713) bank112611 +112612 POINT(47.32319347721602 -123.1227010360455) bank112612 +112613 POINT(47.58448615560239 -123.2900430063698) bank112613 +112614 POINT(48.378294293236166 -121.48060223194261) bank112614 +112615 POINT(48.16323312410995 -122.30952171884057) bank112615 +112616 POINT(47.225099007909584 -123.19980122121814) bank112616 +112617 POINT(48.4368577708714 -122.63561348286657) bank112617 +112618 POINT(46.61879848165177 -121.66266743802252) bank112618 +112619 POINT(48.315241590224524 -122.79211768503143) bank112619 +112620 POINT(46.66720152065036 -122.01702108368002) bank112620 +112621 POINT(48.291790627903026 -123.32030801766196) bank112621 +112622 POINT(47.895815591486034 -123.28688069538643) bank112622 +112623 POINT(47.98044048771863 -122.19456166748124) bank112623 +112624 POINT(47.39024478991515 -122.00879615362639) bank112624 +112625 POINT(48.18827332648466 -122.01845891048126) bank112625 +112626 POINT(48.11486654780097 -123.32736164017943) bank112626 +112627 POINT(47.95654304407014 -122.31668500902909) bank112627 +112628 POINT(48.21590712968549 -122.92816098415084) bank112628 +112629 POINT(48.40449563908462 -122.77465481198512) bank112629 +112630 POINT(48.13636362676203 -121.57394368610598) bank112630 +112631 POINT(46.76744830501305 -122.33069115091604) bank112631 +112632 POINT(48.17551856819294 -122.05946392763842) bank112632 +112633 POINT(47.59512289793028 -122.65962917026587) bank112633 +112634 POINT(47.78246481025283 -122.18589914405864) bank112634 +112635 POINT(46.83338901724898 -122.60397962747513) bank112635 +112636 POINT(47.24830397335878 -122.32691669360428) bank112636 +112637 POINT(46.715266916078455 -122.65160401291368) bank112637 +112638 POINT(46.77816219996162 -122.0058231264569) bank112638 +112639 POINT(46.643129972394135 -122.67458162752297) bank112639 +112640 POINT(47.32262338303697 -121.8091579095616) bank112640 +112641 POINT(46.84131915298138 -121.70910499101979) bank112641 +112642 POINT(47.97465955510293 -122.69139266264753) bank112642 +112643 POINT(47.025224832854676 -122.39498183717191) bank112643 +112644 POINT(48.08262920224032 -123.2362127376485) bank112644 +112645 POINT(47.98686537839147 -122.97718428791174) bank112645 +112646 POINT(48.55227553771265 -123.2318948047371) bank112646 +112647 POINT(47.89649681506553 -121.3483837126428) bank112647 +112648 POINT(46.86123607264862 -122.78813512049746) bank112648 +112649 POINT(48.23177695449674 -122.36999069669452) bank112649 +112650 POINT(48.11133389707516 -122.28397599707257) bank112650 +112651 POINT(48.53630582989002 -123.27514865863048) bank112651 +112652 POINT(48.24985095445047 -121.65501993711887) bank112652 +112653 POINT(47.7175848921498 -121.99824345225825) bank112653 +112654 POINT(47.75532847849983 -122.9602680837107) bank112654 +112655 POINT(47.39164563787738 -122.26751770180758) bank112655 +112656 POINT(46.68756425907348 -121.3411832640462) bank112656 +112657 POINT(48.14767126034533 -122.94747073119477) bank112657 +112658 POINT(48.14626783693348 -121.52845911140454) bank112658 +112659 POINT(48.478548301196675 -123.15131149445537) bank112659 +112660 POINT(46.70532470765416 -122.60886336729874) bank112660 +112661 POINT(47.21705505342948 -122.45156109499632) bank112661 +112662 POINT(47.85855626847101 -122.49646058063827) bank112662 +112663 POINT(48.32360328919242 -122.19490619474628) bank112663 +112664 POINT(46.740335803775295 -122.65378410376806) bank112664 +112665 POINT(48.38326989484825 -122.92246381451996) bank112665 +112666 POINT(47.118441138843316 -122.06026924141354) bank112666 +112667 POINT(48.08020859698327 -122.6928412454616) bank112667 +112668 POINT(47.456878869709364 -122.16692577350203) bank112668 +112669 POINT(47.66283399344242 -122.7343552476515) bank112669 +112670 POINT(46.688671889250095 -121.98742845022886) bank112670 +112671 POINT(47.40588575298044 -121.50537735374753) bank112671 +112672 POINT(48.35810384463146 -122.37960655035673) bank112672 +112673 POINT(47.3776248094145 -121.89115793086104) bank112673 +112674 POINT(47.12226005354263 -122.35738703665884) bank112674 +112675 POINT(48.21382902637034 -122.42134125254303) bank112675 +112676 POINT(48.105172291356205 -122.74561581927735) bank112676 +112677 POINT(48.485246552627075 -122.97781894035941) bank112677 +112678 POINT(47.061094865064746 -122.19516567316525) bank112678 +112679 POINT(47.26137107082238 -122.12980033523448) bank112679 +112680 POINT(46.95030069029659 -122.08840039036463) bank112680 +112681 POINT(46.81375889839504 -122.34897952414973) bank112681 +112682 POINT(47.001485800238235 -123.05570596146342) bank112682 +112683 POINT(47.481901560307186 -122.38475141684285) bank112683 +112684 POINT(48.29265044640922 -122.24300901889521) bank112684 +112685 POINT(48.2990591958247 -122.31109432462084) bank112685 +112686 POINT(48.506667840327005 -121.85063543226556) bank112686 +112687 POINT(47.59884813953008 -121.87077854182782) bank112687 +112688 POINT(47.391798632202175 -122.53576860121774) bank112688 +112689 POINT(47.26101671180029 -122.71774643314303) bank112689 +112690 POINT(46.70376438033075 -122.17446983951037) bank112690 +112691 POINT(47.57996990555612 -122.80638478253626) bank112691 +112692 POINT(48.364835036630666 -121.57937246464152) bank112692 +112693 POINT(48.19599367799464 -123.10506305808892) bank112693 +112694 POINT(48.29827917062145 -122.87559183074828) bank112694 +112695 POINT(46.789301113269126 -121.64962384899627) bank112695 +112696 POINT(46.61458033205028 -122.90410435764326) bank112696 +112697 POINT(47.32078328433717 -121.95201026232765) bank112697 +112698 POINT(48.23670219577741 -121.99472734672852) bank112698 +112699 POINT(47.5729445091307 -121.84434396737963) bank112699 +112700 POINT(47.97502964242397 -122.91550498037688) bank112700 +112701 POINT(47.97617431895812 -121.64388009444241) bank112701 +112702 POINT(47.0993151029361 -121.3335194737283) bank112702 +112703 POINT(48.4546289903415 -121.68292269966868) bank112703 +112704 POINT(48.25011453384488 -122.71943639463144) bank112704 +112705 POINT(47.469325850405696 -121.88881972673843) bank112705 +112706 POINT(47.883453764248145 -122.62942147519101) bank112706 +112707 POINT(46.79645300679282 -122.00843866562555) bank112707 +112708 POINT(46.76776734476837 -121.38723778202808) bank112708 +112709 POINT(47.91058155443325 -121.89061883977938) bank112709 +112710 POINT(48.08805484103719 -122.80892959053686) bank112710 +112711 POINT(47.905670705952915 -122.66553806787617) bank112711 +112712 POINT(47.40141264390371 -121.94528396560716) bank112712 +112713 POINT(47.00321639730679 -121.94759250885197) bank112713 +112714 POINT(48.28670166059536 -121.34509453700488) bank112714 +112715 POINT(48.406978243547634 -122.6297551031144) bank112715 +112716 POINT(48.39030529236745 -121.53249934616244) bank112716 +112717 POINT(47.47190572701909 -121.91238049437328) bank112717 +112718 POINT(47.168829917416346 -123.19031439462938) bank112718 +112719 POINT(48.29722182763711 -122.85959333232441) bank112719 +112720 POINT(46.71049379369118 -122.39638913984817) bank112720 +112721 POINT(47.937777359847246 -122.1558827896216) bank112721 +112722 POINT(47.11801691045111 -121.39268826999614) bank112722 +112723 POINT(47.085953365217264 -122.32556647567627) bank112723 +112724 POINT(48.15894414672444 -122.03436160100728) bank112724 +112725 POINT(47.61913337280194 -121.55807429327695) bank112725 +112726 POINT(46.95706635659606 -122.40944600524904) bank112726 +112727 POINT(46.961354825848424 -121.62548144460906) bank112727 +112728 POINT(47.17682531104138 -122.0404729507251) bank112728 +112729 POINT(47.861199481871665 -121.60312436065765) bank112729 +112730 POINT(46.61063227860576 -121.68173116771592) bank112730 +112731 POINT(47.54149285318098 -122.25317698870936) bank112731 +112732 POINT(47.55372475238324 -121.97537022807781) bank112732 +112733 POINT(47.81270615636197 -121.95329229160775) bank112733 +112734 POINT(47.153351432455345 -121.48277867090083) bank112734 +112735 POINT(47.46466067057836 -123.07554784841686) bank112735 +112736 POINT(47.647242716849725 -121.67421639703053) bank112736 +112737 POINT(47.10642239600091 -121.89891080219392) bank112737 +112738 POINT(47.945374912629646 -122.62502910601512) bank112738 +112739 POINT(48.18800984263098 -122.62288370222393) bank112739 +112740 POINT(48.20185165397304 -123.23183314322493) bank112740 +112741 POINT(47.86172881725901 -122.07769306591317) bank112741 +112742 POINT(47.84824145341613 -121.66833916865669) bank112742 +112743 POINT(48.15699649281247 -121.64965320597345) bank112743 +112744 POINT(48.15605647575014 -122.65262615486873) bank112744 +112745 POINT(47.767454878062644 -123.10834158204815) bank112745 +112746 POINT(47.609265965973194 -122.85416560820413) bank112746 +112747 POINT(47.12608774772832 -122.07650422850163) bank112747 +112748 POINT(46.98383466086129 -121.49091106500018) bank112748 +112749 POINT(47.56641180615322 -123.13663350852859) bank112749 +112750 POINT(48.573836470681734 -122.91139935938978) bank112750 +112751 POINT(48.0657655621442 -121.39470539312728) bank112751 +112752 POINT(46.848727931745586 -121.87806854224384) bank112752 +112753 POINT(46.88651029940181 -122.42286366055083) bank112753 +112754 POINT(46.72455357331716 -121.39945808136196) bank112754 +112755 POINT(46.97086002099756 -121.8994066515846) bank112755 +112756 POINT(47.16209862100274 -123.31469211825956) bank112756 +112757 POINT(46.755840525626446 -122.60328623329612) bank112757 +112758 POINT(48.35417245690683 -121.59890053785159) bank112758 +112759 POINT(47.03621014433342 -121.99468399337634) bank112759 +112760 POINT(47.47680340205165 -122.04884113876649) bank112760 +112761 POINT(46.87550897005837 -122.14570815704401) bank112761 +112762 POINT(47.716461020947754 -121.54905525930397) bank112762 +112763 POINT(46.999005665386754 -122.8520458040636) bank112763 +112764 POINT(48.34246017498549 -123.32399635749135) bank112764 +112765 POINT(47.160975998495296 -122.0265314186781) bank112765 +112766 POINT(47.35100945744822 -122.70438340554877) bank112766 +112767 POINT(48.055848580440006 -123.26538226403662) bank112767 +112768 POINT(47.31709254503311 -121.44586050805538) bank112768 +112769 POINT(47.16097847524549 -121.54025096258736) bank112769 +112770 POINT(47.911575627961895 -122.13861292686335) bank112770 +112771 POINT(48.491219074119044 -122.51250441877197) bank112771 +112772 POINT(48.52234833248305 -122.63591324228392) bank112772 +112773 POINT(47.451630382643906 -122.51393745456221) bank112773 +112774 POINT(48.212805899280035 -123.08555129172815) bank112774 +112775 POINT(47.709601626185865 -123.19158614502764) bank112775 +112776 POINT(47.539904826783015 -121.83656390379356) bank112776 +112777 POINT(47.8717024891611 -121.40017160630096) bank112777 +112778 POINT(47.56545512550493 -121.959483260454) bank112778 +112779 POINT(47.915364575872836 -122.71191400720969) bank112779 +112780 POINT(48.44858332296985 -122.06405890668506) bank112780 +112781 POINT(48.410720608438474 -122.10054629729474) bank112781 +112782 POINT(47.34948762610181 -122.7292343496181) bank112782 +112783 POINT(46.62419438634085 -123.01591686638766) bank112783 +112784 POINT(48.526512196701695 -122.34232469772941) bank112784 +112785 POINT(47.405382349976215 -122.51544566449559) bank112785 +112786 POINT(48.1399236527001 -123.24509287458235) bank112786 +112787 POINT(48.32306259844536 -122.54814471068671) bank112787 +112788 POINT(48.21584622343947 -122.31726762056861) bank112788 +112789 POINT(47.85366346158969 -123.1554854106121) bank112789 +112790 POINT(48.08309457048906 -121.6807702976339) bank112790 +112791 POINT(47.5158494671506 -122.3753299391157) bank112791 +112792 POINT(46.767963181213815 -123.3147045025852) bank112792 +112793 POINT(46.95829506590587 -122.06649270298632) bank112793 +112794 POINT(47.39693758887076 -122.19838998940133) bank112794 +112795 POINT(47.70808336948514 -122.50719564062732) bank112795 +112796 POINT(48.16391805863872 -122.23893068491894) bank112796 +112797 POINT(47.58579266368909 -122.83658133163188) bank112797 +112798 POINT(47.914594745354485 -122.27443192181542) bank112798 +112799 POINT(47.78341900737136 -121.38136043120383) bank112799 +112800 POINT(48.35511182863618 -122.42017673984066) bank112800 +112801 POINT(47.59265697518655 -123.01285092583232) bank112801 +112802 POINT(46.80244168168123 -122.09867556989951) bank112802 +112803 POINT(48.13929387568826 -121.763975264671) bank112803 +112804 POINT(47.53340903233674 -122.51467031139873) bank112804 +112805 POINT(46.72807572846264 -123.09082723499904) bank112805 +112806 POINT(47.88875135700294 -121.48297884517396) bank112806 +112807 POINT(47.47061555078547 -122.0954308190009) bank112807 +112808 POINT(47.01709406195922 -122.26772839780057) bank112808 +112809 POINT(47.00339912549394 -122.44011805952103) bank112809 +112810 POINT(47.75525760574756 -122.78646149183056) bank112810 +112811 POINT(47.55551464919645 -121.38502182022486) bank112811 +112812 POINT(47.645336070285175 -123.18816587234366) bank112812 +112813 POINT(48.04998643983658 -121.68080063684886) bank112813 +112814 POINT(47.19832456992185 -122.56893481479315) bank112814 +112815 POINT(47.89280981096879 -123.10132976024657) bank112815 +112816 POINT(47.641769946737426 -122.36617011802713) bank112816 +112817 POINT(47.895329833773395 -121.44827555278945) bank112817 +112818 POINT(46.89763676326064 -122.89721663916143) bank112818 +112819 POINT(47.20590971504383 -121.61125142763272) bank112819 +112820 POINT(47.221803868701166 -122.14662728446936) bank112820 +112821 POINT(48.409671787032806 -122.10852700916719) bank112821 +112822 POINT(47.87625099704274 -121.76271121995272) bank112822 +112823 POINT(47.07057228152144 -123.11224975704545) bank112823 +112824 POINT(48.03990816831541 -121.99402315223928) bank112824 +112825 POINT(48.3141271764559 -121.99668952776497) bank112825 +112826 POINT(48.1672172304246 -121.51713864832496) bank112826 +112827 POINT(47.401424841564285 -122.56580679438895) bank112827 +112828 POINT(47.18186496529699 -122.53123152426618) bank112828 +112829 POINT(46.70270459079671 -122.34569041554467) bank112829 +112830 POINT(48.20756483766235 -121.35584139414611) bank112830 +112831 POINT(48.034688962885994 -122.45987427256237) bank112831 +112832 POINT(47.93333261156915 -122.60842315296773) bank112832 +112833 POINT(46.8866459524885 -121.63888546592432) bank112833 +112834 POINT(46.77638202228332 -121.72928178258313) bank112834 +112835 POINT(46.72738857325061 -123.11129214133501) bank112835 +112836 POINT(46.63723135196811 -122.5421234790865) bank112836 +112837 POINT(46.97304434567652 -121.8937876419478) bank112837 +112838 POINT(48.253259555749516 -121.75015001176111) bank112838 +112839 POINT(47.002930503405324 -123.32061872987862) bank112839 +112840 POINT(47.927452149555066 -121.45873924787858) bank112840 +112841 POINT(48.26815547544908 -122.70282481473768) bank112841 +112842 POINT(47.935085619792716 -122.45757920635758) bank112842 +112843 POINT(47.93607454366906 -121.58889120360372) bank112843 +112844 POINT(47.79904952979585 -121.3406237887153) bank112844 +112845 POINT(46.706653532844456 -122.2752041312708) bank112845 +112846 POINT(48.08063361890148 -122.28585044605697) bank112846 +112847 POINT(48.29397332179898 -121.78698887424711) bank112847 +112848 POINT(47.6209452531699 -122.50689825950722) bank112848 +112849 POINT(47.55686860093695 -122.7935387763198) bank112849 +112850 POINT(46.942958026128984 -121.7302202656082) bank112850 +112851 POINT(48.46989112353588 -121.774192003956) bank112851 +112852 POINT(48.12575306297368 -122.65997960554031) bank112852 +112853 POINT(46.76543841597257 -122.77528166082163) bank112853 +112854 POINT(46.675566084783185 -122.29583184770607) bank112854 +112855 POINT(46.94674662055072 -122.15134481779265) bank112855 +112856 POINT(46.76788696124553 -122.1850063009958) bank112856 +112857 POINT(46.65813657885609 -122.72313373025617) bank112857 +112858 POINT(48.03367155226108 -122.38783308202602) bank112858 +112859 POINT(47.003339166741576 -122.2434066963354) bank112859 +112860 POINT(47.57515127004993 -122.04336911727515) bank112860 +112861 POINT(48.33657193437128 -121.37064908799452) bank112861 +112862 POINT(48.48234439939587 -122.48045743323941) bank112862 +112863 POINT(48.170664295034 -122.44462707140194) bank112863 +112864 POINT(48.096027624249004 -122.31266935287358) bank112864 +112865 POINT(47.927306950955646 -121.63149315140248) bank112865 +112866 POINT(48.380737331814984 -122.08642027326282) bank112866 +112867 POINT(47.39021419261235 -121.61016013331952) bank112867 +112868 POINT(47.30492564318288 -123.30626316192902) bank112868 +112869 POINT(47.84211986531331 -122.18747504969588) bank112869 +112870 POINT(46.88764832389036 -121.71893624098348) bank112870 +112871 POINT(46.851170158416544 -121.37224058675321) bank112871 +112872 POINT(47.71477637010469 -122.6149768297359) bank112872 +112873 POINT(47.35237329399267 -123.15898650007287) bank112873 +112874 POINT(47.442514347671406 -121.76336697359824) bank112874 +112875 POINT(46.90238526577787 -121.93711662901767) bank112875 +112876 POINT(48.39854936224423 -122.15952000485507) bank112876 +112877 POINT(46.9418440713016 -121.9415958794506) bank112877 +112878 POINT(48.079909389777846 -122.18079935545454) bank112878 +112879 POINT(47.84180963261666 -122.43374247295381) bank112879 +112880 POINT(46.802132899598476 -121.9827789179624) bank112880 +112881 POINT(47.6041416340901 -122.04168156108702) bank112881 +112882 POINT(47.67028957711053 -122.34424347688139) bank112882 +112883 POINT(47.8437465878923 -122.60573051290335) bank112883 +112884 POINT(47.79678295505298 -122.3552201584267) bank112884 +112885 POINT(48.14118545068464 -121.34883458617684) bank112885 +112886 POINT(48.38275149178154 -122.46789952520751) bank112886 +112887 POINT(47.00832179239327 -122.08728217715651) bank112887 +112888 POINT(48.03034836154606 -122.89640981838262) bank112888 +112889 POINT(47.42867121417607 -121.37605792521596) bank112889 +112890 POINT(47.46784425250355 -121.74203666877926) bank112890 +112891 POINT(48.009475822358176 -123.19286779338745) bank112891 +112892 POINT(47.75086292314174 -121.46171935632509) bank112892 +112893 POINT(47.46921121342564 -122.02690517774998) bank112893 +112894 POINT(46.66040054830456 -121.58628574888279) bank112894 +112895 POINT(47.73882739308885 -123.02702274270796) bank112895 +112896 POINT(47.138164006233936 -122.00773289173635) bank112896 +112897 POINT(48.39626137049002 -122.04320189890211) bank112897 +112898 POINT(46.95294745924598 -122.04451600923075) bank112898 +112899 POINT(46.88420237285697 -122.69243076259764) bank112899 +112900 POINT(47.75927446903401 -122.14315743862385) bank112900 +112901 POINT(48.23110786073807 -122.20167909055351) bank112901 +112902 POINT(47.760511099858824 -122.61434065393125) bank112902 +112903 POINT(48.60431128735263 -122.02126721795162) bank112903 +112904 POINT(46.70835968818578 -122.990129429926) bank112904 +112905 POINT(48.32922157681181 -122.60646528556791) bank112905 +112906 POINT(47.3365465978218 -121.74909811938319) bank112906 +112907 POINT(47.63820779731653 -123.30067351627521) bank112907 +112908 POINT(47.802618595303784 -122.16756643996085) bank112908 +112909 POINT(48.49446217628185 -123.06248619633008) bank112909 +112910 POINT(46.95478995530196 -123.31541169555956) bank112910 +112911 POINT(47.253616740779364 -122.81235721001771) bank112911 +112912 POINT(47.139108501451084 -123.10064242261195) bank112912 +112913 POINT(46.882232219972316 -121.90165988080786) bank112913 +112914 POINT(47.56551394268215 -122.48458583674754) bank112914 +112915 POINT(47.006905303126324 -123.0717766637664) bank112915 +112916 POINT(47.15985031571416 -122.22032193892079) bank112916 +112917 POINT(47.45915182433207 -122.7720852180059) bank112917 +112918 POINT(47.36976362383886 -122.41154637356126) bank112918 +112919 POINT(46.748704903006995 -122.48656716598406) bank112919 +112920 POINT(47.69449277696519 -121.86763498235518) bank112920 +112921 POINT(48.30623978302131 -122.0809604990057) bank112921 +112922 POINT(47.914157577320154 -122.88956480167006) bank112922 +112923 POINT(47.08788450667123 -122.82759116819325) bank112923 +112924 POINT(46.61480641273581 -123.2949560246552) bank112924 +112925 POINT(47.66000519039661 -121.62087847722066) bank112925 +112926 POINT(47.10706532811894 -121.37475270576105) bank112926 +112927 POINT(47.17675030138157 -123.1839233354141) bank112927 +112928 POINT(48.025484236841194 -122.64992114347605) bank112928 +112929 POINT(47.52108526300744 -122.53813784678675) bank112929 +112930 POINT(47.68749439956321 -122.21817409709145) bank112930 +112931 POINT(48.0177334082616 -122.2358771943646) bank112931 +112932 POINT(47.86952101928248 -123.0136942371191) bank112932 +112933 POINT(47.762445368224455 -122.72694999554007) bank112933 +112934 POINT(48.111593973568056 -122.37037490464155) bank112934 +112935 POINT(46.71930692991588 -121.70502538672491) bank112935 +112936 POINT(46.894309553471174 -122.06018260827209) bank112936 +112937 POINT(47.17054710836484 -121.674649442955) bank112937 +112938 POINT(48.58127069379029 -122.57049741171531) bank112938 +112939 POINT(48.23995679382634 -123.05536820022635) bank112939 +112940 POINT(47.14682008001597 -123.03639760400364) bank112940 +112941 POINT(48.24636002273029 -123.11003348291122) bank112941 +112942 POINT(47.096786674820464 -122.91761313488271) bank112942 +112943 POINT(47.86386214968363 -123.12628423492843) bank112943 +112944 POINT(46.89998067055735 -122.69416590880891) bank112944 +112945 POINT(46.69836748502821 -122.49859224210182) bank112945 +112946 POINT(46.73474222505679 -122.36631653630894) bank112946 +112947 POINT(48.07659367716213 -122.5553481623937) bank112947 +112948 POINT(47.32140838139568 -121.95055090432054) bank112948 +112949 POINT(48.0074994036519 -122.78862039177741) bank112949 +112950 POINT(47.1002236221434 -121.77726254107625) bank112950 +112951 POINT(46.729071465033385 -121.8496207372411) bank112951 +112952 POINT(48.07004215122326 -122.00012431714099) bank112952 +112953 POINT(46.73891859408692 -122.89121209602646) bank112953 +112954 POINT(47.02094193968561 -122.03219573521953) bank112954 +112955 POINT(48.38305956645953 -122.6434636825016) bank112955 +112956 POINT(46.654451160071034 -122.47758423364508) bank112956 +112957 POINT(46.94382821593276 -121.75700924494474) bank112957 +112958 POINT(47.54516473616596 -121.94656284590118) bank112958 +112959 POINT(46.72934572059635 -122.63153510530489) bank112959 +112960 POINT(48.112094262820705 -121.38057634568152) bank112960 +112961 POINT(47.191277008851436 -121.79400975956911) bank112961 +112962 POINT(47.472718290003115 -123.06424307869571) bank112962 +112963 POINT(46.996187551023965 -121.52154301574923) bank112963 +112964 POINT(46.76903847623174 -121.89668147817257) bank112964 +112965 POINT(46.81327074531477 -122.89173417517269) bank112965 +112966 POINT(48.43248628731392 -121.99277318389677) bank112966 +112967 POINT(48.5184508339591 -121.3553382237258) bank112967 +112968 POINT(46.833840365173046 -121.72315796707996) bank112968 +112969 POINT(47.54263762929669 -122.73887320052998) bank112969 +112970 POINT(47.57787829941817 -121.89762274687077) bank112970 +112971 POINT(46.95165215645663 -121.44082842630604) bank112971 +112972 POINT(48.394647955483705 -122.38045027992423) bank112972 +112973 POINT(48.16822934840824 -122.53810848736445) bank112973 +112974 POINT(47.93788765843263 -122.34710517010964) bank112974 +112975 POINT(46.9855492234469 -122.63517479347773) bank112975 +112976 POINT(47.14403017880376 -121.6255008071622) bank112976 +112977 POINT(47.62086170177019 -123.30373185010053) bank112977 +112978 POINT(48.580534038908915 -122.27071019119158) bank112978 +112979 POINT(48.32528971868399 -122.01234089418554) bank112979 +112980 POINT(47.20464264113274 -122.71267303192451) bank112980 +112981 POINT(48.138264709144075 -121.36123796026578) bank112981 +112982 POINT(46.70906863988091 -122.45772008606238) bank112982 +112983 POINT(46.68155352834538 -121.51933889133913) bank112983 +112984 POINT(46.675711763056206 -122.55125807264962) bank112984 +112985 POINT(48.22463906243323 -122.2903973817471) bank112985 +112986 POINT(47.37035189713018 -122.93274882712603) bank112986 +112987 POINT(47.045781395043775 -121.91578221005068) bank112987 +112988 POINT(46.87752303565137 -122.94875210600519) bank112988 +112989 POINT(46.733768304606976 -121.38641960276317) bank112989 +112990 POINT(47.54968074424789 -122.01507688430155) bank112990 +112991 POINT(47.75746668736094 -123.0900216090131) bank112991 +112992 POINT(48.14283293292125 -122.89353627045081) bank112992 +112993 POINT(48.21063836661928 -122.03144700948127) bank112993 +112994 POINT(47.64198147508891 -121.84180990902468) bank112994 +112995 POINT(46.843792834352584 -121.43256743370857) bank112995 +112996 POINT(47.465195221892984 -121.90756996729938) bank112996 +112997 POINT(48.12400978476686 -122.40901616301547) bank112997 +112998 POINT(47.591478693257386 -122.34775208817257) bank112998 +112999 POINT(47.806650805955265 -121.48106328113421) bank112999 +113000 POINT(47.91461467261157 -121.34836412283433) bank113000 +113001 POINT(47.94076105409958 -122.2435362523108) bank113001 +113002 POINT(47.36424689468025 -122.01014392517871) bank113002 +113003 POINT(47.205415822952396 -121.77245100781148) bank113003 +113004 POINT(47.5560313246022 -122.06972219505943) bank113004 +113005 POINT(48.185767998207545 -122.61472428168936) bank113005 +113006 POINT(47.47015184665358 -122.03877362393398) bank113006 +113007 POINT(47.95526220323412 -121.6160482896748) bank113007 +113008 POINT(48.11395881940295 -121.46084148845834) bank113008 +113009 POINT(47.28052036352496 -122.29037908436432) bank113009 +113010 POINT(48.55917514080832 -122.0104402634469) bank113010 +113011 POINT(48.44591211236773 -123.05372438052618) bank113011 +113012 POINT(48.3318461875664 -122.80134383837247) bank113012 +113013 POINT(48.14169869885103 -121.93461670662431) bank113013 +113014 POINT(47.45174888593309 -121.68378458303853) bank113014 +113015 POINT(47.03459181506453 -122.08009971112858) bank113015 +113016 POINT(47.64750651054537 -121.81414159307587) bank113016 +113017 POINT(47.39098833429876 -121.3959763585421) bank113017 +113018 POINT(47.49482515489322 -123.14601797290474) bank113018 +113019 POINT(46.97727023303416 -122.3953260642409) bank113019 +113020 POINT(47.08903833791672 -121.85079262845952) bank113020 +113021 POINT(47.856946999351216 -121.39070477303562) bank113021 +113022 POINT(47.588310279377694 -122.21243691974621) bank113022 +113023 POINT(47.48530067924603 -121.89065262503682) bank113023 +113024 POINT(46.89126628531216 -123.07403440733391) bank113024 +113025 POINT(47.68829140372339 -123.0274387250585) bank113025 +113026 POINT(47.425288093669586 -122.95095200068653) bank113026 +113027 POINT(48.16277106683945 -122.70593549099934) bank113027 +113028 POINT(48.55837745718495 -122.70351929338612) bank113028 +113029 POINT(46.74412441307118 -123.15561515271784) bank113029 +113030 POINT(48.1079309584235 -122.09381487748213) bank113030 +113031 POINT(47.55578640282762 -122.84765845106034) bank113031 +113032 POINT(48.27346885489364 -122.62546339052163) bank113032 +113033 POINT(48.241205726289515 -122.78564434106902) bank113033 +113034 POINT(47.3408519029475 -122.51555177470483) bank113034 +113035 POINT(47.518710219790584 -122.91379392152288) bank113035 +113036 POINT(46.63284777437085 -123.17438076174538) bank113036 +113037 POINT(48.423263328941424 -122.27343109257306) bank113037 +113038 POINT(47.89981651791914 -121.46490542689313) bank113038 +113039 POINT(47.69739611801634 -122.89990031955622) bank113039 +113040 POINT(48.357377655725756 -123.25635578196142) bank113040 +113041 POINT(48.21568981943971 -121.41767790112308) bank113041 +113042 POINT(47.62262972054574 -122.5988534719412) bank113042 +113043 POINT(48.29123445760317 -121.80049693706653) bank113043 +113044 POINT(46.64968693336377 -123.2660836189461) bank113044 +113045 POINT(48.46593315358395 -121.62659984301759) bank113045 +113046 POINT(47.162044139455354 -121.57122776023358) bank113046 +113047 POINT(46.60705403200814 -121.3924146996419) bank113047 +113048 POINT(47.482203332555756 -123.0251524857302) bank113048 +113049 POINT(47.31135152688579 -122.29810464076533) bank113049 +113050 POINT(47.09791606947779 -122.33241302986764) bank113050 +113051 POINT(48.11931471020252 -121.85626716594145) bank113051 +113052 POINT(48.249295357313194 -123.0679854738843) bank113052 +113053 POINT(48.08390271796364 -121.9882213954535) bank113053 +113054 POINT(48.24158870890544 -121.4927729128881) bank113054 +113055 POINT(47.755449308517136 -122.94770370134998) bank113055 +113056 POINT(47.99217620881837 -122.51730968558172) bank113056 +113057 POINT(47.12119874597052 -122.3098935418041) bank113057 +113058 POINT(48.56258653770394 -122.58568207969932) bank113058 +113059 POINT(47.081173745540134 -122.1105820484258) bank113059 +113060 POINT(47.163756615196704 -122.01738677964816) bank113060 +113061 POINT(46.87601438907604 -122.23565210494144) bank113061 +113062 POINT(48.46803059157861 -121.5820212853713) bank113062 +113063 POINT(48.003811147049454 -121.52239674286031) bank113063 +113064 POINT(48.02575943938277 -122.02552849988056) bank113064 +113065 POINT(48.05827334762243 -121.80466755699507) bank113065 +113066 POINT(46.627210562061556 -121.97011363456309) bank113066 +113067 POINT(48.43235597063307 -121.9426243218462) bank113067 +113068 POINT(47.226308216856005 -122.23398814735039) bank113068 +113069 POINT(47.737752260902646 -122.65510948024149) bank113069 +113070 POINT(48.55260831088111 -123.03159023283705) bank113070 +113071 POINT(48.362686598362046 -121.66186095575951) bank113071 +113072 POINT(47.04186438131879 -122.0889899034268) bank113072 +113073 POINT(47.22657364517696 -123.0220915857929) bank113073 +113074 POINT(48.20761702080605 -121.4460084374681) bank113074 +113075 POINT(47.722597908967366 -121.70645635993002) bank113075 +113076 POINT(48.03121509857263 -122.67930054821886) bank113076 +113077 POINT(47.973446289980636 -121.43985215987567) bank113077 +113078 POINT(48.42096921898816 -123.27643447633372) bank113078 +113079 POINT(48.38668136120288 -122.9457801396265) bank113079 +113080 POINT(47.57374302785631 -123.27349271384504) bank113080 +113081 POINT(47.22910069250116 -121.37443533301658) bank113081 +113082 POINT(47.88061512706065 -121.61272234533313) bank113082 +113083 POINT(48.55900555484291 -122.48395806007387) bank113083 +113084 POINT(47.495700081603765 -123.32076855302759) bank113084 +113085 POINT(47.90716797853458 -121.85015844778336) bank113085 +113086 POINT(48.33611813891252 -123.20347849714844) bank113086 +113087 POINT(48.04814562683592 -123.26540512252295) bank113087 +113088 POINT(46.6971264213416 -122.93012473572144) bank113088 +113089 POINT(48.20201472323871 -122.71698141892523) bank113089 +113090 POINT(47.93349104062367 -123.26542971085914) bank113090 +113091 POINT(47.19668002779812 -123.06657935672861) bank113091 +113092 POINT(47.79223073174221 -123.26677904563465) bank113092 +113093 POINT(47.76495134416011 -122.99958033965864) bank113093 +113094 POINT(47.30974783651944 -121.39478766108364) bank113094 +113095 POINT(48.0382257905347 -121.94099207197668) bank113095 +113096 POINT(48.11022575274842 -122.81822611198372) bank113096 +113097 POINT(48.03424218614906 -121.46468600056313) bank113097 +113098 POINT(48.06145399063039 -122.3621701359206) bank113098 +113099 POINT(47.7213799912622 -121.95200470866158) bank113099 +113100 POINT(47.1175650027597 -122.78961005196321) bank113100 +113101 POINT(46.96254970002442 -121.91568858541021) bank113101 +113102 POINT(48.02583985276954 -123.06352364441953) bank113102 +113103 POINT(46.67120091312795 -123.18572887343628) bank113103 +113104 POINT(47.1250852365656 -122.5860334533886) bank113104 +113105 POINT(47.88214047704741 -122.55544420540875) bank113105 +113106 POINT(47.71487833395384 -122.65043839010552) bank113106 +113107 POINT(46.82186425040817 -122.62008372577296) bank113107 +113108 POINT(47.71560984742921 -122.9569412111983) bank113108 +113109 POINT(47.19137598111196 -122.58943022283758) bank113109 +113110 POINT(48.11983213975916 -122.02446737666973) bank113110 +113111 POINT(48.20209723072267 -123.00513121263207) bank113111 +113112 POINT(46.89215354156654 -122.82605110064101) bank113112 +113113 POINT(47.26166208001806 -121.92039695351393) bank113113 +113114 POINT(48.37462293370054 -121.49174613442075) bank113114 +113115 POINT(47.67826308417143 -121.48991521602989) bank113115 +113116 POINT(46.664406299811986 -122.15118044636125) bank113116 +113117 POINT(46.75679456269279 -122.7707232156619) bank113117 +113118 POINT(48.185152044510254 -121.47891858937113) bank113118 +113119 POINT(48.107635562850234 -122.22045048919045) bank113119 +113120 POINT(47.3659965702236 -122.45713408886095) bank113120 +113121 POINT(47.5491989123429 -122.01656453559525) bank113121 +113122 POINT(47.788541335496866 -121.53394921695423) bank113122 +113123 POINT(46.68814082966727 -121.38208408440796) bank113123 +113124 POINT(47.324288992610356 -122.9376226836574) bank113124 +113125 POINT(47.93798797094383 -123.05621548654437) bank113125 +113126 POINT(47.71114056030683 -121.75544440263417) bank113126 +113127 POINT(47.266688165230356 -122.86271487345896) bank113127 +113128 POINT(47.68579820642206 -123.0224447926442) bank113128 +113129 POINT(48.19648182615707 -122.65666968597111) bank113129 +113130 POINT(48.30354077951225 -122.77397355647479) bank113130 +113131 POINT(48.2030195543144 -121.56978504259266) bank113131 +113132 POINT(46.76418152722813 -122.9822085174797) bank113132 +113133 POINT(48.284363492811266 -121.7571526529265) bank113133 +113134 POINT(48.49717043194148 -121.50415681837184) bank113134 +113135 POINT(47.75793291985836 -122.41320922726439) bank113135 +113136 POINT(47.81761085603249 -122.09721530274173) bank113136 +113137 POINT(47.108345697430494 -123.01363337903454) bank113137 +113138 POINT(47.19910738550584 -122.29076928999679) bank113138 +113139 POINT(47.273292035493114 -121.98198705735558) bank113139 +113140 POINT(48.256206420858135 -122.99494732163991) bank113140 +113141 POINT(47.86777911368079 -122.68577198912878) bank113141 +113142 POINT(48.17636722991632 -121.98806391424718) bank113142 +113143 POINT(48.158025466612195 -123.28999257249365) bank113143 +113144 POINT(46.88629068931938 -121.34490096466158) bank113144 +113145 POINT(48.28489674499907 -121.72313065474626) bank113145 +113146 POINT(46.88754268819393 -123.22084158721778) bank113146 +113147 POINT(48.03367936647379 -122.83018586714104) bank113147 +113148 POINT(46.79635680938158 -123.05513561284087) bank113148 +113149 POINT(48.34272029581219 -122.28493548126477) bank113149 +113150 POINT(47.64784441106697 -122.81000825827576) bank113150 +113151 POINT(47.489367136746615 -123.30530879973654) bank113151 +113152 POINT(47.65623945538514 -122.37093140570848) bank113152 +113153 POINT(47.07318890247235 -121.80788797388101) bank113153 +113154 POINT(47.49088805619156 -122.17453400417712) bank113154 +113155 POINT(47.49521403484762 -122.87858130708392) bank113155 +113156 POINT(47.655822661299325 -122.0882857434117) bank113156 +113157 POINT(47.09342546050673 -121.7101437313142) bank113157 +113158 POINT(47.06314106638999 -122.57331318190894) bank113158 +113159 POINT(48.11857088494196 -122.34483536073111) bank113159 +113160 POINT(47.26198898659288 -122.53867859710863) bank113160 +113161 POINT(47.38311767086665 -121.7498970762653) bank113161 +113162 POINT(46.72402164644439 -122.94012463503141) bank113162 +113163 POINT(48.393240971982166 -122.24044482166873) bank113163 +113164 POINT(46.89451144404864 -121.57036317219294) bank113164 +113165 POINT(47.86662859207863 -122.74470560973309) bank113165 +113166 POINT(47.72780378930428 -122.50887479151596) bank113166 +113167 POINT(46.64596275376407 -123.32758332903903) bank113167 +113168 POINT(47.298821495992705 -122.51735639096918) bank113168 +113169 POINT(48.542521397385855 -123.27858593145753) bank113169 +113170 POINT(47.0008988962534 -122.47540353180781) bank113170 +113171 POINT(47.20320267877727 -122.8471802915902) bank113171 +113172 POINT(47.90642318809905 -122.3855210180727) bank113172 +113173 POINT(47.0133850084634 -121.84813303861321) bank113173 +113174 POINT(48.18701798231939 -122.5396865168479) bank113174 +113175 POINT(48.601800214033226 -121.52576970809613) bank113175 +113176 POINT(46.75412109580581 -123.3177405467956) bank113176 +113177 POINT(47.61134439205326 -121.82380400236774) bank113177 +113178 POINT(47.1159533969689 -122.15072712448202) bank113178 +113179 POINT(47.71731477028219 -121.44671840960933) bank113179 +113180 POINT(48.49012531007542 -122.91269302803134) bank113180 +113181 POINT(47.75722478060817 -122.42399100134203) bank113181 +113182 POINT(47.71168975182615 -122.14505928025773) bank113182 +113183 POINT(47.77469174322301 -123.19149395108671) bank113183 +113184 POINT(47.14889129715296 -122.57567602285881) bank113184 +113185 POINT(46.7879214996458 -122.20330406877507) bank113185 +113186 POINT(47.02457431713629 -121.67270384967267) bank113186 +113187 POINT(48.273812674405505 -122.02456500076161) bank113187 +113188 POINT(47.00502068725927 -123.29647672924814) bank113188 +113189 POINT(47.595090289092596 -123.14882829888698) bank113189 +113190 POINT(47.62205983985667 -123.26979769917376) bank113190 +113191 POINT(46.68419805967617 -123.01470479163814) bank113191 +113192 POINT(48.07816276021207 -121.48031227816767) bank113192 +113193 POINT(47.00583376872173 -122.29181523637604) bank113193 +113194 POINT(46.64127214477116 -122.10361008316866) bank113194 +113195 POINT(47.152459424285304 -121.4468597700669) bank113195 +113196 POINT(48.181482984809236 -122.53405066593177) bank113196 +113197 POINT(48.2577948966245 -122.00613388620833) bank113197 +113198 POINT(47.15149157204785 -122.52194830123972) bank113198 +113199 POINT(48.062241643878835 -121.82878141710937) bank113199 +113200 POINT(47.75111029361513 -121.40957369516791) bank113200 +113201 POINT(47.85221514252974 -122.63411276795038) bank113201 +113202 POINT(47.64502146031203 -121.39101108525021) bank113202 +113203 POINT(48.26745081995008 -122.1012501190489) bank113203 +113204 POINT(47.45740569472992 -121.67437034276628) bank113204 +113205 POINT(48.39046697018273 -123.32403882138044) bank113205 +113206 POINT(48.25314871931961 -122.71970536783985) bank113206 +113207 POINT(47.033863799531005 -123.31879982194795) bank113207 +113208 POINT(46.99609401855653 -122.32950909467681) bank113208 +113209 POINT(48.285732271940795 -122.8003529937085) bank113209 +113210 POINT(46.64982161512143 -122.85031789762232) bank113210 +113211 POINT(46.77987539735085 -121.50651165922736) bank113211 +113212 POINT(46.7321583412538 -122.29747673937185) bank113212 +113213 POINT(47.690744383752964 -123.0695117880791) bank113213 +113214 POINT(47.92083051153925 -122.97852292800785) bank113214 +113215 POINT(47.30374841501791 -122.76723192324407) bank113215 +113216 POINT(47.1233832487826 -122.03221409908642) bank113216 +113217 POINT(47.41901065107119 -122.21799871312405) bank113217 +113218 POINT(47.63500332628777 -121.3975484628959) bank113218 +113219 POINT(47.87230361822764 -121.52254702774862) bank113219 +113220 POINT(48.59429920437228 -122.41920278442721) bank113220 +113221 POINT(47.20577076658861 -121.90905515485099) bank113221 +113222 POINT(48.271307190619545 -122.43301635294085) bank113222 +113223 POINT(47.489636801801815 -121.94951052810943) bank113223 +113224 POINT(48.456574580250305 -122.60855987361063) bank113224 +113225 POINT(47.34691899025016 -122.22674846480575) bank113225 +113226 POINT(48.10167843039766 -122.85321227312875) bank113226 +113227 POINT(48.25704835900837 -122.29338251945342) bank113227 +113228 POINT(46.704762417546924 -122.40731468188804) bank113228 +113229 POINT(47.58495663409955 -123.21200275514222) bank113229 +113230 POINT(47.030662399166886 -121.91075216007684) bank113230 +113231 POINT(46.85508536561935 -121.3540558334905) bank113231 +113232 POINT(48.13434454760495 -122.8382846256024) bank113232 +113233 POINT(47.69466316532538 -121.8967777293064) bank113233 +113234 POINT(48.151730870026995 -123.19248871283658) bank113234 +113235 POINT(47.45801834835638 -122.65752289770703) bank113235 +113236 POINT(46.98430479645914 -122.62940631166894) bank113236 +113237 POINT(48.18802714625784 -121.8245276591023) bank113237 +113238 POINT(47.8632884799558 -122.58406803246088) bank113238 +113239 POINT(47.99947528456086 -123.20926539669233) bank113239 +113240 POINT(46.71950002147991 -121.47838966757314) bank113240 +113241 POINT(47.4010376695353 -122.66567084607533) bank113241 +113242 POINT(46.68216919935382 -123.10571390349544) bank113242 +113243 POINT(48.4006843109408 -122.58412978894046) bank113243 +113244 POINT(46.936926768008526 -122.6142631229597) bank113244 +113245 POINT(47.16267387664675 -121.56975790140338) bank113245 +113246 POINT(48.400221028775846 -121.36631504757362) bank113246 +113247 POINT(47.74336818549456 -122.55312752222108) bank113247 +113248 POINT(47.513943500370324 -122.30083228197888) bank113248 +113249 POINT(48.14587572677333 -121.71301075425325) bank113249 +113250 POINT(47.71693800867343 -122.66006067584162) bank113250 +113251 POINT(47.83744822725216 -122.70526183406777) bank113251 +113252 POINT(47.6282553262989 -123.044855851482) bank113252 +113253 POINT(47.2242814240553 -122.80172043248004) bank113253 +113254 POINT(46.84521837809856 -121.87311851494712) bank113254 +113255 POINT(48.18559396986586 -122.1256545413001) bank113255 +113256 POINT(46.761337956079565 -121.38315088986805) bank113256 +113257 POINT(47.05079786649401 -122.4764609191173) bank113257 +113258 POINT(47.65370758399416 -122.07519846888705) bank113258 +113259 POINT(46.926747736542886 -122.51597805643) bank113259 +113260 POINT(47.73462533201679 -121.35889201652193) bank113260 +113261 POINT(47.0042967649387 -122.12504865353033) bank113261 +113262 POINT(48.55391952234194 -122.04055517177429) bank113262 +113263 POINT(46.867804219535884 -121.99425929079781) bank113263 +113264 POINT(47.95761108666229 -122.86462618864059) bank113264 +113265 POINT(47.57608611601229 -122.33588225501174) bank113265 +113266 POINT(47.93219389404204 -122.04877746088135) bank113266 +113267 POINT(47.698430462728815 -123.00471040109349) bank113267 +113268 POINT(48.068377636595415 -121.89431715079952) bank113268 +113269 POINT(47.41275238148029 -122.05850438601951) bank113269 +113270 POINT(47.37535451993425 -121.75789485250567) bank113270 +113271 POINT(47.75607225578247 -122.08924148740275) bank113271 +113272 POINT(47.36609048955391 -121.94277272981847) bank113272 +113273 POINT(47.41796516615069 -121.4877071278997) bank113273 +113274 POINT(48.10364373299589 -121.84108356963134) bank113274 +113275 POINT(47.865979689632056 -122.16306762008759) bank113275 +113276 POINT(48.270094360852646 -122.19270769904138) bank113276 +113277 POINT(48.11353181758233 -121.66043277760119) bank113277 +113278 POINT(48.292496737720086 -122.93354277995637) bank113278 +113279 POINT(47.71413209350082 -121.62357696334627) bank113279 +113280 POINT(48.14935860462434 -123.08089009037666) bank113280 +113281 POINT(48.372695990232344 -122.73461364538548) bank113281 +113282 POINT(47.44562254354989 -123.25532593764011) bank113282 +113283 POINT(48.25078180103118 -122.0674765421039) bank113283 +113284 POINT(47.35206043509486 -123.18801309841713) bank113284 +113285 POINT(47.23893368359394 -122.02388655248018) bank113285 +113286 POINT(48.35383250274014 -123.04791682170752) bank113286 +113287 POINT(48.412140789345365 -122.2638152299361) bank113287 +113288 POINT(47.380755524479255 -121.98218994483175) bank113288 +113289 POINT(46.8678593693952 -122.61880693636382) bank113289 +113290 POINT(47.830268108318364 -122.5768394888965) bank113290 +113291 POINT(46.861724281864994 -122.2672998665433) bank113291 +113292 POINT(47.96656664851376 -122.8587097527424) bank113292 +113293 POINT(47.7963467281492 -121.3354410034659) bank113293 +113294 POINT(47.853598078884794 -122.19278478484304) bank113294 +113295 POINT(48.07492610670069 -122.45131872657421) bank113295 +113296 POINT(47.098173107945456 -121.6788180803782) bank113296 +113297 POINT(47.79072195827723 -122.0999004392483) bank113297 +113298 POINT(47.75284784104303 -121.44167221790603) bank113298 +113299 POINT(48.549262947670414 -121.8778682110515) bank113299 +113300 POINT(47.31033465050911 -122.93773972136353) bank113300 +113301 POINT(46.87815605416654 -122.04839899039705) bank113301 +113302 POINT(46.74559906308958 -121.86646388698085) bank113302 +113303 POINT(47.10156533954085 -121.58525873970927) bank113303 +113304 POINT(47.25987607450114 -122.65118252658118) bank113304 +113305 POINT(47.91513392057883 -122.1574478237327) bank113305 +113306 POINT(48.33064530305741 -122.69209515434984) bank113306 +113307 POINT(46.85245495633966 -122.6196517184033) bank113307 +113308 POINT(47.046877536348426 -121.94897932341114) bank113308 +113309 POINT(47.82010451222904 -122.49748668887837) bank113309 +113310 POINT(47.340635080776636 -121.9257228826332) bank113310 +113311 POINT(46.646997688480845 -121.35130623286274) bank113311 +113312 POINT(47.30214738698469 -122.91227573723687) bank113312 +113313 POINT(48.20860707046247 -122.56764355948398) bank113313 +113314 POINT(47.37107740372045 -121.52339326370874) bank113314 +113315 POINT(47.69386056139171 -121.37631760509564) bank113315 +113316 POINT(48.573345904657664 -121.35945079175303) bank113316 +113317 POINT(47.73189477117017 -121.53366067359434) bank113317 +113318 POINT(48.276251966814804 -121.73705025675372) bank113318 +113319 POINT(47.742331238898686 -121.84016010396915) bank113319 +113320 POINT(47.338346364719385 -121.34474873608868) bank113320 +113321 POINT(46.84654126650542 -122.13277565431548) bank113321 +113322 POINT(48.10593787366567 -121.3576408898795) bank113322 +113323 POINT(47.383795345857244 -122.83772732959905) bank113323 +113324 POINT(46.66475494491721 -121.4911391345837) bank113324 +113325 POINT(46.806208609158375 -122.1510419693787) bank113325 +113326 POINT(47.975292243299826 -122.57822123362297) bank113326 +113327 POINT(47.24730169369868 -122.29923512836443) bank113327 +113328 POINT(48.41181697721513 -122.72679367290613) bank113328 +113329 POINT(46.794475963593236 -121.76671751485985) bank113329 +113330 POINT(47.56602968367903 -123.03485433824959) bank113330 +113331 POINT(47.54672876174914 -122.48619536381618) bank113331 +113332 POINT(48.20579189172195 -121.43001163594874) bank113332 +113333 POINT(47.074580814712846 -121.43700446931558) bank113333 +113334 POINT(48.281766742065145 -122.2343335129163) bank113334 +113335 POINT(47.84484155959487 -122.00663252218985) bank113335 +113336 POINT(46.94662196671821 -122.7314831438699) bank113336 +113337 POINT(47.11318956956478 -122.25002675531375) bank113337 +113338 POINT(48.58321580208629 -122.2605077373014) bank113338 +113339 POINT(48.4922617334391 -122.81014011071677) bank113339 +113340 POINT(47.45213618642671 -121.97005923092121) bank113340 +113341 POINT(47.48686109931372 -122.1715659727677) bank113341 +113342 POINT(48.12841383212236 -122.43351045899678) bank113342 +113343 POINT(48.1664784815274 -121.89667598407688) bank113343 +113344 POINT(47.90327948028676 -121.98273550830764) bank113344 +113345 POINT(47.29561175050549 -121.54901359286445) bank113345 +113346 POINT(47.17968595591651 -122.23334968528405) bank113346 +113347 POINT(47.658154227154455 -122.61020604879319) bank113347 +113348 POINT(47.7276747850202 -122.4335243064406) bank113348 +113349 POINT(48.56132285501738 -122.02841105522133) bank113349 +113350 POINT(46.76788546058623 -122.4140627407047) bank113350 +113351 POINT(47.909390850757276 -123.0802826598059) bank113351 +113352 POINT(47.9195485606435 -121.38337049150762) bank113352 +113353 POINT(48.594895568298966 -121.33883851605424) bank113353 +113354 POINT(46.67861426702264 -122.62571271013897) bank113354 +113355 POINT(47.6632411639888 -121.47601091882204) bank113355 +113356 POINT(48.00401299091628 -122.00726074726768) bank113356 +113357 POINT(47.66788351844351 -122.58612910112733) bank113357 +113358 POINT(46.91332286001355 -122.49993401387506) bank113358 +113359 POINT(47.3511544492763 -122.7951300617026) bank113359 +113360 POINT(48.54681305809331 -121.6174330439276) bank113360 +113361 POINT(48.225526541982106 -121.60286773453954) bank113361 +113362 POINT(47.46310387538055 -121.71959040325397) bank113362 +113363 POINT(47.6558626971815 -123.28939902221195) bank113363 +113364 POINT(47.55160758662156 -122.70061674863243) bank113364 +113365 POINT(46.664890093081134 -121.91355365490958) bank113365 +113366 POINT(48.05153746296772 -122.14740668036585) bank113366 +113367 POINT(47.84094048074206 -122.33667714047371) bank113367 +113368 POINT(47.10164687392889 -121.41425045993651) bank113368 +113369 POINT(47.029831169324495 -123.12224637788336) bank113369 +113370 POINT(48.32585267283333 -122.05528783367266) bank113370 +113371 POINT(47.18475764719005 -123.30409870340745) bank113371 +113372 POINT(48.47346725297725 -122.65920478097254) bank113372 +113373 POINT(47.00606380654268 -123.1675855938111) bank113373 +113374 POINT(48.48044328694429 -121.5467981538884) bank113374 +113375 POINT(48.56286243166428 -122.20091185473997) bank113375 +113376 POINT(47.051278965279835 -123.287032214192) bank113376 +113377 POINT(47.00701935582944 -123.1010374176751) bank113377 +113378 POINT(46.68563110189336 -122.8498010215862) bank113378 +113379 POINT(46.61724928043309 -121.72122025416003) bank113379 +113380 POINT(47.525541472732066 -121.33991241502652) bank113380 +113381 POINT(46.88800040800224 -122.99552462640128) bank113381 +113382 POINT(47.27135249868326 -122.1895137221798) bank113382 +113383 POINT(46.71541848331773 -123.06220718188659) bank113383 +113384 POINT(48.32395841693204 -123.07783760160082) bank113384 +113385 POINT(47.190261259407094 -122.12346955516345) bank113385 +113386 POINT(47.18109938927482 -121.36530748322976) bank113386 +113387 POINT(48.26271730292178 -121.56340499628467) bank113387 +113388 POINT(48.26936427551444 -121.67631983326523) bank113388 +113389 POINT(47.75639603333097 -122.12806051366695) bank113389 +113390 POINT(48.204941348572504 -122.88610159448808) bank113390 +113391 POINT(48.59519098685148 -121.497619081004) bank113391 +113392 POINT(47.91691646798487 -121.76586240984433) bank113392 +113393 POINT(47.60452706286029 -122.43803951620085) bank113393 +113394 POINT(47.142694120518236 -122.40109723392342) bank113394 +113395 POINT(48.38236802061984 -123.19968501953136) bank113395 +113396 POINT(47.74724624141905 -123.14880982482619) bank113396 +113397 POINT(46.84323189365643 -121.4992578447857) bank113397 +113398 POINT(46.66565699494487 -122.45689082431541) bank113398 +113399 POINT(46.6430697849256 -123.29498754960461) bank113399 +113400 POINT(48.011130425035404 -123.13623582756479) bank113400 +113401 POINT(47.33539979858474 -122.86705019966948) bank113401 +113402 POINT(48.13038355711846 -121.92815218175501) bank113402 +113403 POINT(47.69195961876603 -123.28524755004901) bank113403 +113404 POINT(47.437115091858345 -121.52378841004463) bank113404 +113405 POINT(47.501904178913286 -122.20789108783936) bank113405 +113406 POINT(47.02708736350054 -122.54209422933289) bank113406 +113407 POINT(47.86954777632512 -121.69751583713553) bank113407 +113408 POINT(47.311592285598024 -121.3988066092794) bank113408 +113409 POINT(47.433896569504114 -122.20648383913048) bank113409 +113410 POINT(46.882040176203624 -121.8503816213655) bank113410 +113411 POINT(46.989734855413346 -122.11350156950817) bank113411 +113412 POINT(48.33980481932169 -122.61308025724595) bank113412 +113413 POINT(46.86810980030535 -123.12735277749348) bank113413 +113414 POINT(46.747084822541474 -122.21985218851549) bank113414 +113415 POINT(47.095485173187285 -123.05902401692482) bank113415 +113416 POINT(46.80309312350969 -122.72081364265604) bank113416 +113417 POINT(48.5042851592515 -122.61358673488364) bank113417 +113418 POINT(48.38374348849136 -122.35101629451445) bank113418 +113419 POINT(47.550564686482716 -121.5231466579375) bank113419 +113420 POINT(46.88415161395717 -122.11406783958557) bank113420 +113421 POINT(47.57748286468246 -121.63837216040788) bank113421 +113422 POINT(47.793630036076145 -122.4970606054085) bank113422 +113423 POINT(48.16916472764698 -123.22107251296765) bank113423 +113424 POINT(46.82197830125208 -121.82923431359974) bank113424 +113425 POINT(48.28134957204195 -122.82745732255644) bank113425 +113426 POINT(48.19105019475827 -122.61730932869028) bank113426 +113427 POINT(48.60197161879479 -121.46148931148451) bank113427 +113428 POINT(47.26852550018127 -122.0082625570453) bank113428 +113429 POINT(47.93586630285082 -121.56955072490035) bank113429 +113430 POINT(48.26623432329524 -121.61691781915191) bank113430 +113431 POINT(48.30171046687115 -123.06184608859131) bank113431 +113432 POINT(47.8510560058716 -122.63567743091666) bank113432 +113433 POINT(48.49973680661032 -122.36171530984835) bank113433 +113434 POINT(48.5136915306422 -121.69552644643942) bank113434 +113435 POINT(47.61997052339317 -122.43554640396522) bank113435 +113436 POINT(47.21662546822068 -123.1207521641382) bank113436 +113437 POINT(47.85720608797273 -122.52757797352945) bank113437 +113438 POINT(47.95779891561474 -122.30749655624466) bank113438 +113439 POINT(47.337896753338164 -121.35480755914887) bank113439 +113440 POINT(48.51873182500061 -123.22221771762418) bank113440 +113441 POINT(46.960049521593085 -123.1113327472063) bank113441 +113442 POINT(46.88269240464018 -122.93907025577502) bank113442 +113443 POINT(48.051841949123954 -122.90041801481036) bank113443 +113444 POINT(47.81831472029311 -121.74297129508906) bank113444 +113445 POINT(46.823483539885416 -123.29401823716192) bank113445 +113446 POINT(46.64830253031077 -122.43648518635453) bank113446 +113447 POINT(48.49907187022568 -122.54893813809934) bank113447 +113448 POINT(48.33022852334752 -121.51825516194809) bank113448 +113449 POINT(47.70659786335436 -122.96237245551748) bank113449 +113450 POINT(47.148950895032634 -122.206972704918) bank113450 +113451 POINT(48.601853062453415 -122.97445907340231) bank113451 +113452 POINT(48.54542645376274 -123.06656451455596) bank113452 +113453 POINT(48.05307140629701 -122.20353327468611) bank113453 +113454 POINT(47.419858301295825 -121.71678241762508) bank113454 +113455 POINT(46.69364758501354 -122.16377520819444) bank113455 +113456 POINT(47.782826171706255 -122.39671414432001) bank113456 +113457 POINT(48.44157558906021 -122.80772172857017) bank113457 +113458 POINT(47.521269035278316 -122.55593768925544) bank113458 +113459 POINT(48.22173438975348 -121.6536314930307) bank113459 +113460 POINT(47.773258919025906 -122.7978332756749) bank113460 +113461 POINT(48.498333591161426 -122.51032549088187) bank113461 +113462 POINT(47.50761944896353 -123.0824988981649) bank113462 +113463 POINT(48.59579846806532 -122.06785358929092) bank113463 +113464 POINT(48.05934666502219 -121.70425799808761) bank113464 +113465 POINT(46.69453416584365 -122.88732955456398) bank113465 +113466 POINT(46.80261430081392 -122.61663989517055) bank113466 +113467 POINT(48.097329512060725 -122.67104065004338) bank113467 +113468 POINT(48.33882978510172 -121.70967612831784) bank113468 +113469 POINT(47.11612575906588 -122.75058868316597) bank113469 +113470 POINT(47.706289010826424 -121.39734883026469) bank113470 +113471 POINT(47.576965974547015 -121.63874676257223) bank113471 +113472 POINT(46.952621019629476 -122.60536309996431) bank113472 +113473 POINT(47.607929924458574 -121.6857396276135) bank113473 +113474 POINT(48.55883637314536 -122.87441732691101) bank113474 +113475 POINT(48.515491821872665 -123.19337175364822) bank113475 +113476 POINT(46.74519665856085 -122.29162134775704) bank113476 +113477 POINT(47.14214371960883 -121.50491257333441) bank113477 +113478 POINT(47.08913911149416 -122.53692352604463) bank113478 +113479 POINT(47.47534864267969 -122.502767709654) bank113479 +113480 POINT(48.384151362900205 -122.49175175990894) bank113480 +113481 POINT(47.397959848299514 -121.74745286359224) bank113481 +113482 POINT(47.62883681284876 -123.0295953287901) bank113482 +113483 POINT(46.98310195426318 -122.10338950142207) bank113483 +113484 POINT(48.16911720165826 -122.69886023782475) bank113484 +113485 POINT(48.077545306293565 -122.32360647239034) bank113485 +113486 POINT(47.90880144375692 -122.99689344763091) bank113486 +113487 POINT(48.47280224306073 -122.08697754302686) bank113487 +113488 POINT(46.778139020816766 -122.37832188302238) bank113488 +113489 POINT(46.78686635826497 -121.45093484526615) bank113489 +113490 POINT(47.58459459370196 -122.71818987839617) bank113490 +113491 POINT(48.09191329159909 -122.8212249819463) bank113491 +113492 POINT(48.37838841097304 -121.74333635521488) bank113492 +113493 POINT(48.44151054990836 -121.393686372092) bank113493 +113494 POINT(47.40875975686399 -123.23434960357112) bank113494 +113495 POINT(47.10310891088954 -122.59990636546436) bank113495 +113496 POINT(47.90004485155332 -122.14847498899162) bank113496 +113497 POINT(47.88848589991515 -121.7585674414343) bank113497 +113498 POINT(47.773529699316754 -122.70777710371782) bank113498 +113499 POINT(47.17105194092986 -122.60637833847093) bank113499 +113500 POINT(46.82981709071507 -121.63372109251455) bank113500 +113501 POINT(46.94479183223468 -122.69888888051138) bank113501 +113502 POINT(48.07989018149102 -122.24403619232498) bank113502 +113503 POINT(47.804761575193474 -122.51674046596617) bank113503 +113504 POINT(47.63227276774787 -122.44464182325542) bank113504 +113505 POINT(46.685795158899964 -123.12903733008848) bank113505 +113506 POINT(47.29744946917524 -122.7547373621924) bank113506 +113507 POINT(47.12685130862181 -121.36171459083296) bank113507 +113508 POINT(48.38017133189613 -123.26838639414906) bank113508 +113509 POINT(46.94497493406554 -123.04261991081586) bank113509 +113510 POINT(48.269742820404836 -121.48904811486) bank113510 +113511 POINT(48.051471584693616 -121.88948117581545) bank113511 +113512 POINT(47.51887570188486 -121.38572563736655) bank113512 +113513 POINT(47.79744732549731 -121.882719510031) bank113513 +113514 POINT(48.3997795824558 -121.62032628247572) bank113514 +113515 POINT(47.43276402093165 -122.62729361141433) bank113515 +113516 POINT(47.025137456878625 -122.57160993455282) bank113516 +113517 POINT(47.145880915957314 -122.37357622407697) bank113517 +113518 POINT(47.056764029680615 -121.81592900912122) bank113518 +113519 POINT(47.76327807499997 -122.08098547885012) bank113519 +113520 POINT(46.90273234909126 -121.78955075559703) bank113520 +113521 POINT(48.05756444009525 -122.96292089387683) bank113521 +113522 POINT(47.29300841808874 -122.48836302435679) bank113522 +113523 POINT(48.58941007851646 -122.19600417171033) bank113523 +113524 POINT(48.41625882255055 -122.2805769136784) bank113524 +113525 POINT(46.83402009613662 -122.19395797706065) bank113525 +113526 POINT(47.62122027344857 -121.81736832319044) bank113526 +113527 POINT(48.38679320038727 -121.91854246480236) bank113527 +113528 POINT(48.42851310915359 -122.81422025258382) bank113528 +113529 POINT(46.79322177007855 -122.40629653445403) bank113529 +113530 POINT(47.7698537579071 -121.95825948671805) bank113530 +113531 POINT(48.061317218223785 -121.60319452025828) bank113531 +113532 POINT(47.678897446181935 -121.88072364829127) bank113532 +113533 POINT(47.12455622904677 -123.33130782741931) bank113533 +113534 POINT(47.213327552794624 -122.72628320191949) bank113534 +113535 POINT(48.49018567561365 -122.23639917700245) bank113535 +113536 POINT(46.93438992015037 -122.79793704953966) bank113536 +113537 POINT(47.86820418319564 -121.99386275763138) bank113537 +113538 POINT(47.0541220557163 -121.7491693238608) bank113538 +113539 POINT(47.555770463395916 -123.19590649569446) bank113539 +113540 POINT(48.42265946636973 -121.66646989700793) bank113540 +113541 POINT(47.283150864895546 -122.15637701421049) bank113541 +113542 POINT(47.86356984711642 -122.90203467319797) bank113542 +113543 POINT(47.965424636634694 -122.60453908273902) bank113543 +113544 POINT(47.420392825082615 -122.86902897422715) bank113544 +113545 POINT(46.998072381997055 -123.05944137756804) bank113545 +113546 POINT(47.708191374013495 -122.02440778340976) bank113546 +113547 POINT(47.08051667980159 -123.1830162645959) bank113547 +113548 POINT(46.776731854368656 -122.68514173157605) bank113548 +113549 POINT(47.18323543710679 -123.07506864288682) bank113549 +113550 POINT(48.311560400237326 -122.56751385857982) bank113550 +113551 POINT(47.602493937584335 -121.3818258052062) bank113551 +113552 POINT(48.52707999118249 -121.3429712725814) bank113552 +113553 POINT(46.67807410903056 -121.63394744829309) bank113553 +113554 POINT(47.91118444980069 -122.1552846352588) bank113554 +113555 POINT(47.203362734610735 -122.07819384961843) bank113555 +113556 POINT(46.9884646603821 -123.0095789239624) bank113556 +113557 POINT(48.42773724871293 -121.75629891872005) bank113557 +113558 POINT(46.7241049588513 -123.0792135032177) bank113558 +113559 POINT(47.43617310915962 -123.3224287415182) bank113559 +113560 POINT(47.591205963068546 -121.95600983237314) bank113560 +113561 POINT(47.85035093910355 -121.92216956633169) bank113561 +113562 POINT(47.84477481827959 -121.4530418811115) bank113562 +113563 POINT(48.59127825128115 -122.22551359449008) bank113563 +113564 POINT(48.05868050794113 -122.32700130826024) bank113564 +113565 POINT(47.425036840367625 -121.33231851649876) bank113565 +113566 POINT(47.165104616632725 -122.5738946044362) bank113566 +113567 POINT(47.38040851901539 -121.53540110758779) bank113567 +113568 POINT(46.896614981745344 -122.13862959164845) bank113568 +113569 POINT(48.17969505402194 -123.1386051912799) bank113569 +113570 POINT(48.07869682596564 -122.29355222860322) bank113570 +113571 POINT(47.03898165263585 -123.28019781199897) bank113571 +113572 POINT(46.81013728901664 -123.27136907464075) bank113572 +113573 POINT(48.48896356384799 -123.1623313064374) bank113573 +113574 POINT(48.55683477473908 -121.34428634566541) bank113574 +113575 POINT(46.912159099513595 -122.25099183796583) bank113575 +113576 POINT(48.526931805520604 -123.19501469166542) bank113576 +113577 POINT(46.754000975530914 -122.24715034290237) bank113577 +113578 POINT(47.04213277570883 -123.15647841604479) bank113578 +113579 POINT(47.76378134151223 -122.08999597112957) bank113579 +113580 POINT(48.15649592195292 -123.0973907622854) bank113580 +113581 POINT(48.22595850371249 -121.95206721849125) bank113581 +113582 POINT(47.61088901226837 -122.96025171490261) bank113582 +113583 POINT(46.9345541559528 -122.93046434033185) bank113583 +113584 POINT(47.30738950158826 -123.26184979500162) bank113584 +113585 POINT(46.802579543906155 -121.71479100892206) bank113585 +113586 POINT(47.613047656056345 -122.55375506251066) bank113586 +113587 POINT(47.656913305977994 -123.12180932685229) bank113587 +113588 POINT(48.06713023809826 -121.81531116881435) bank113588 +113589 POINT(46.61721154052664 -121.9333907987869) bank113589 +113590 POINT(47.7224971154339 -121.8798049929208) bank113590 +113591 POINT(46.988710971859945 -123.05073652383209) bank113591 +113592 POINT(47.20412191892114 -123.26405558853862) bank113592 +113593 POINT(48.513003360994716 -122.68365306328211) bank113593 +113594 POINT(47.84986254915814 -123.09952221169904) bank113594 +113595 POINT(48.33222586177084 -121.66593730930254) bank113595 +113596 POINT(46.790770216471365 -121.93606044729341) bank113596 +113597 POINT(48.234169334803006 -123.15151129625822) bank113597 +113598 POINT(48.23673866006398 -122.26434167636536) bank113598 +113599 POINT(47.81940071353135 -121.74078925183142) bank113599 +113600 POINT(47.136745231356116 -121.94884415274537) bank113600 +113601 POINT(46.735030280111175 -121.48063696179602) bank113601 +113602 POINT(47.76001645439152 -121.60719667650834) bank113602 +113603 POINT(47.01156985898096 -121.52590549285752) bank113603 +113604 POINT(48.42013876291045 -123.02875484177966) bank113604 +113605 POINT(48.335960306892076 -121.82083904583197) bank113605 +113606 POINT(46.66614903187592 -122.99133287926601) bank113606 +113607 POINT(48.10360220533647 -121.63723435424339) bank113607 +113608 POINT(47.77466545141459 -122.66883382612451) bank113608 +113609 POINT(48.12503054508294 -122.336905517023) bank113609 +113610 POINT(48.30365256592329 -122.10514916938881) bank113610 +113611 POINT(46.73939825594644 -123.26584639705413) bank113611 +113612 POINT(46.627907333501675 -122.63609204823402) bank113612 +113613 POINT(48.57659919735032 -122.47774478341881) bank113613 +113614 POINT(46.980990810874864 -121.73322857886603) bank113614 +113615 POINT(47.364129171281135 -123.04033053042755) bank113615 +113616 POINT(47.34765774217141 -122.37148761584318) bank113616 +113617 POINT(47.78416413121952 -123.03742048233428) bank113617 +113618 POINT(46.69251214196727 -122.28613226265054) bank113618 +113619 POINT(48.09916435971271 -122.75268493184298) bank113619 +113620 POINT(47.94895500072434 -121.92551239720137) bank113620 +113621 POINT(46.667625188112765 -122.79697631998471) bank113621 +113622 POINT(46.93794927844472 -122.16476532463336) bank113622 +113623 POINT(47.22948543022179 -121.6864087927923) bank113623 +113624 POINT(47.39917045318613 -122.37239272461514) bank113624 +113625 POINT(47.85728649828914 -122.75427299467363) bank113625 +113626 POINT(48.57376435057695 -122.47122308102503) bank113626 +113627 POINT(47.16099961455965 -122.73003494085413) bank113627 +113628 POINT(46.654950755443295 -121.59759345382305) bank113628 +113629 POINT(46.73041503918982 -122.62992168409129) bank113629 +113630 POINT(46.62541332045012 -122.33468654172533) bank113630 +113631 POINT(48.26786668580521 -121.74969113885948) bank113631 +113632 POINT(47.98446682295756 -122.9988043993742) bank113632 +113633 POINT(48.52065380473667 -122.8569400562207) bank113633 +113634 POINT(47.68003285288428 -122.22179365263975) bank113634 +113635 POINT(46.74724434247689 -122.24325269721238) bank113635 +113636 POINT(47.4994174588266 -122.55052688017167) bank113636 +113637 POINT(47.44669380889781 -121.9314873437009) bank113637 +113638 POINT(48.60108159032219 -122.28137221323189) bank113638 +113639 POINT(46.88946149062715 -122.18661888633137) bank113639 +113640 POINT(46.78799118149511 -122.43527620862088) bank113640 +113641 POINT(46.78123647155877 -122.27444629123787) bank113641 +113642 POINT(47.65786938639372 -122.45105391531895) bank113642 +113643 POINT(47.570091520861645 -122.45981916240972) bank113643 +113644 POINT(47.15461842882993 -122.38714306580064) bank113644 +113645 POINT(47.79441908163965 -121.43484290693053) bank113645 +113646 POINT(47.55347731566617 -122.14320563420914) bank113646 +113647 POINT(47.36431141209899 -121.50980241498782) bank113647 +113648 POINT(47.40362914309818 -122.21785289454229) bank113648 +113649 POINT(47.75775708257891 -122.28506510336993) bank113649 +113650 POINT(46.8727435602323 -122.02457822175876) bank113650 +113651 POINT(47.28933948511754 -121.36878882254827) bank113651 +113652 POINT(48.4549514951778 -121.91469214404071) bank113652 +113653 POINT(47.81411426628806 -123.20972626246791) bank113653 +113654 POINT(48.39337538658562 -121.7898275488745) bank113654 +113655 POINT(46.69691454977512 -123.19711528962888) bank113655 +113656 POINT(48.0035681362395 -121.7967378006574) bank113656 +113657 POINT(47.615448078987576 -121.7082547749131) bank113657 +113658 POINT(48.52591570966265 -122.60095941564711) bank113658 +113659 POINT(46.63105585996445 -122.05794310021535) bank113659 +113660 POINT(48.46284519640672 -122.18997758064387) bank113660 +113661 POINT(48.317643404757426 -122.71586074160615) bank113661 +113662 POINT(47.414491697872336 -122.54246768772465) bank113662 +113663 POINT(48.41874210904167 -122.06351182629717) bank113663 +113664 POINT(48.31699852461814 -122.86962197802761) bank113664 +113665 POINT(46.77528687697524 -123.02846099241049) bank113665 +113666 POINT(47.300484620782676 -122.61772186235456) bank113666 +113667 POINT(47.828302567906626 -121.83873512521973) bank113667 +113668 POINT(47.31713995000075 -121.61918228297905) bank113668 +113669 POINT(47.120558144772204 -121.89951227114057) bank113669 +113670 POINT(47.692715741204985 -122.99899446007804) bank113670 +113671 POINT(48.11238176740917 -123.02203237389539) bank113671 +113672 POINT(48.33585056982067 -122.4981823440857) bank113672 +113673 POINT(47.22353992822637 -123.22930256805323) bank113673 +113674 POINT(47.27448020994745 -123.1266331497593) bank113674 +113675 POINT(47.9239220961378 -122.10492363091947) bank113675 +113676 POINT(46.77518611453894 -121.4316070169886) bank113676 +113677 POINT(46.80047853851582 -121.6852104661709) bank113677 +113678 POINT(47.995156927979274 -121.82353349080921) bank113678 +113679 POINT(48.4765493604134 -122.7516928378144) bank113679 +113680 POINT(47.35992606679764 -122.60012255026528) bank113680 +113681 POINT(48.46009686140859 -122.75312242416489) bank113681 +113682 POINT(46.95964634534915 -122.02650885094624) bank113682 +113683 POINT(47.19823683345501 -121.79213747997278) bank113683 +113684 POINT(47.2872336702148 -123.21856387678845) bank113684 +113685 POINT(47.72919184986055 -123.12171082728621) bank113685 +113686 POINT(48.559557624067644 -122.56143288263135) bank113686 +113687 POINT(48.152967167463515 -121.83331425503931) bank113687 +113688 POINT(47.104862622980285 -121.70268726451097) bank113688 +113689 POINT(47.35262774365636 -122.74181862097723) bank113689 +113690 POINT(47.80447360058382 -122.8743606915297) bank113690 +113691 POINT(47.19503074190865 -122.97153912783823) bank113691 +113692 POINT(47.62804201698989 -122.29232298173841) bank113692 +113693 POINT(47.88446469113174 -122.20228743502557) bank113693 +113694 POINT(46.88803557393209 -121.89381125548) bank113694 +113695 POINT(47.319424761572265 -121.64565847276423) bank113695 +113696 POINT(47.64018773727672 -122.751565612088) bank113696 +113697 POINT(47.32383988303583 -122.86873517731382) bank113697 +113698 POINT(47.17846924322031 -122.42585394065657) bank113698 +113699 POINT(48.45286694217266 -122.27596981979696) bank113699 +113700 POINT(46.84017473564903 -122.5906207722535) bank113700 +113701 POINT(46.672879280579714 -122.13878650483775) bank113701 +113702 POINT(48.172108455192955 -123.01619638571647) bank113702 +113703 POINT(48.22965597373894 -121.57978174563598) bank113703 +113704 POINT(46.74172718740455 -123.20284115963041) bank113704 +113705 POINT(46.87261331969584 -122.79927018622396) bank113705 +113706 POINT(46.62377452103523 -123.09331387931442) bank113706 +113707 POINT(46.93034902879581 -122.616393281626) bank113707 +113708 POINT(47.12055394653689 -122.75200331362377) bank113708 +113709 POINT(46.81746611096333 -121.6585225560145) bank113709 +113710 POINT(48.285109778845936 -122.65410643592985) bank113710 +113711 POINT(47.16757265932728 -122.42767557772777) bank113711 +113712 POINT(46.986776671189865 -121.84829898836603) bank113712 +113713 POINT(47.955065832862175 -121.88779226007252) bank113713 +113714 POINT(48.226642464902945 -122.111972537538) bank113714 +113715 POINT(47.94792502520249 -122.60284483618972) bank113715 +113716 POINT(47.53129253486278 -121.5539750897437) bank113716 +113717 POINT(48.407903646902135 -121.69715730049134) bank113717 +113718 POINT(48.27561107653747 -121.50920851958728) bank113718 +113719 POINT(48.44341573601972 -123.08712993821777) bank113719 +113720 POINT(47.64399502267214 -122.45826075793927) bank113720 +113721 POINT(47.70089690665594 -121.52049602987587) bank113721 +113722 POINT(47.829235461636436 -122.54038797471169) bank113722 +113723 POINT(48.297087145448465 -121.36001594885865) bank113723 +113724 POINT(47.055515834062184 -121.34552713615017) bank113724 +113725 POINT(48.601844504096135 -121.3842966315576) bank113725 +113726 POINT(47.88778357329065 -121.39760762921834) bank113726 +113727 POINT(48.15885735912727 -122.7218298594267) bank113727 +113728 POINT(48.04063099112676 -122.96639920427559) bank113728 +113729 POINT(48.357018514033506 -123.00542723190566) bank113729 +113730 POINT(47.98401273386827 -121.46079399310298) bank113730 +113731 POINT(47.56381683091921 -121.35475605544633) bank113731 +113732 POINT(47.75213664968266 -122.84350813744001) bank113732 +113733 POINT(47.008211519602455 -122.6579616477277) bank113733 +113734 POINT(47.51627509837062 -123.19779198007097) bank113734 +113735 POINT(47.33701612148436 -122.04165462757754) bank113735 +113736 POINT(46.968396923276934 -121.79742516616895) bank113736 +113737 POINT(48.25662887513616 -121.9245536341832) bank113737 +113738 POINT(47.66806682751729 -121.91121529227941) bank113738 +113739 POINT(47.37350044389249 -121.41978746299093) bank113739 +113740 POINT(46.72540526803015 -122.28061908948925) bank113740 +113741 POINT(47.71660108425673 -121.89140824960323) bank113741 +113742 POINT(48.30291206227407 -122.99192611118151) bank113742 +113743 POINT(47.89181054202253 -121.5992848446566) bank113743 +113744 POINT(47.44145540430994 -123.24351149555825) bank113744 +113745 POINT(46.63968836157683 -121.33956354451323) bank113745 +113746 POINT(48.449052050873576 -122.66965791213893) bank113746 +113747 POINT(47.882504833984584 -122.3638313247872) bank113747 +113748 POINT(47.65147289939194 -121.68496298587779) bank113748 +113749 POINT(48.12763831542045 -123.06035060829494) bank113749 +113750 POINT(46.85945326474986 -121.75905738606801) bank113750 +113751 POINT(47.59655169518567 -121.88346451220986) bank113751 +113752 POINT(47.15079334359083 -122.21645659644206) bank113752 +113753 POINT(47.278031082213204 -122.82335927572383) bank113753 +113754 POINT(47.68743885683711 -121.74502263706977) bank113754 +113755 POINT(47.42502689488727 -121.51235423445209) bank113755 +113756 POINT(48.21741671600867 -123.06364712600265) bank113756 +113757 POINT(47.80278543807137 -123.15163806441637) bank113757 +113758 POINT(48.494590380469596 -121.70237777677343) bank113758 +113759 POINT(48.494311610428504 -122.22139178390832) bank113759 +113760 POINT(47.75797573808823 -122.05346473680014) bank113760 +113761 POINT(46.64335927615884 -123.25062301587602) bank113761 +113762 POINT(47.03415777880895 -121.54896488079241) bank113762 +113763 POINT(48.3312177033915 -122.02509875239038) bank113763 +113764 POINT(48.39038865618816 -123.16977535266702) bank113764 +113765 POINT(46.63442252628617 -121.46234571409937) bank113765 +113766 POINT(46.77612975408418 -122.70413811252367) bank113766 +113767 POINT(47.18061966872359 -121.61282124934657) bank113767 +113768 POINT(48.03938942202581 -121.67236147174017) bank113768 +113769 POINT(48.333361714336924 -122.79287290576315) bank113769 +113770 POINT(47.72351661992071 -121.6620198009089) bank113770 +113771 POINT(48.15514089406376 -122.94370861423084) bank113771 +113772 POINT(47.109220221727526 -123.28050315008613) bank113772 +113773 POINT(47.42527730177817 -122.03887313602662) bank113773 +113774 POINT(47.84560124889141 -122.19868384646979) bank113774 +113775 POINT(47.26859309634552 -122.15044155018468) bank113775 +113776 POINT(46.82010358239496 -123.22462703617416) bank113776 +113777 POINT(48.04983640587489 -121.45093850580113) bank113777 +113778 POINT(46.71335340153373 -122.92521804344551) bank113778 +113779 POINT(47.589744720083495 -122.93907343636265) bank113779 +113780 POINT(47.965396676989855 -123.16536993727019) bank113780 +113781 POINT(48.404850346214985 -121.66826512548269) bank113781 +113782 POINT(46.730021311659236 -121.5152046488461) bank113782 +113783 POINT(46.721628345724355 -121.4416731451652) bank113783 +113784 POINT(48.018270995556946 -123.21885675722963) bank113784 +113785 POINT(47.11391309542418 -121.59016112941417) bank113785 +113786 POINT(47.46280019799471 -122.93423119353193) bank113786 +113787 POINT(48.177975599703565 -122.88431182441337) bank113787 +113788 POINT(46.963765636236644 -122.01566625647489) bank113788 +113789 POINT(46.77607486549481 -121.48823368759102) bank113789 +113790 POINT(47.86004637677896 -121.77978073895693) bank113790 +113791 POINT(48.07133162759332 -122.76275519436055) bank113791 +113792 POINT(46.95745574990382 -122.32128854870713) bank113792 +113793 POINT(48.32915396999276 -122.66848137071337) bank113793 +113794 POINT(48.447265014120084 -122.17270759023042) bank113794 +113795 POINT(47.87513077663887 -123.1520601318966) bank113795 +113796 POINT(47.9294526007245 -123.2073280684718) bank113796 +113797 POINT(47.27493366942261 -122.67721847329221) bank113797 +113798 POINT(46.71749186784789 -121.7665833085432) bank113798 +113799 POINT(47.96277495626193 -122.90768724030411) bank113799 +113800 POINT(47.896783617314135 -122.48103034203501) bank113800 +113801 POINT(47.49565384213125 -121.3650053099739) bank113801 +113802 POINT(47.75450748238379 -121.84798646573556) bank113802 +113803 POINT(46.702989710977775 -122.73855209107683) bank113803 +113804 POINT(47.019554757473905 -123.23219901461746) bank113804 +113805 POINT(47.161221307686276 -122.57361437614095) bank113805 +113806 POINT(48.09454653495735 -122.18124240544091) bank113806 +113807 POINT(48.340957786472515 -122.41284686723559) bank113807 +113808 POINT(47.96742923274742 -121.73863563437895) bank113808 +113809 POINT(46.811525205055865 -122.51429668212177) bank113809 +113810 POINT(46.70036304528005 -121.78394437609879) bank113810 +113811 POINT(47.54336586520518 -122.4347073905761) bank113811 +113812 POINT(47.60812095157104 -122.20419425304465) bank113812 +113813 POINT(47.48141966841278 -121.7645358754021) bank113813 +113814 POINT(47.12045717165642 -122.48005091292488) bank113814 +113815 POINT(47.893068390316905 -121.5019533438936) bank113815 +113816 POINT(46.7656710991823 -121.66637723744347) bank113816 +113817 POINT(48.05802116638988 -122.74706675312828) bank113817 +113818 POINT(47.42833126662268 -122.88367607860732) bank113818 +113819 POINT(46.76579073424629 -122.49055803704795) bank113819 +113820 POINT(47.21536667214863 -121.45896262316262) bank113820 +113821 POINT(47.63800842432076 -122.8436926884849) bank113821 +113822 POINT(47.16577126283541 -121.68232459340523) bank113822 +113823 POINT(48.369314367775154 -122.85231467193539) bank113823 +113824 POINT(46.89012112617134 -121.67900652366345) bank113824 +113825 POINT(47.39422331982349 -122.45117429979149) bank113825 +113826 POINT(46.702079960211876 -122.55143322358593) bank113826 +113827 POINT(48.541207606636014 -121.9443282516125) bank113827 +113828 POINT(48.52641553282009 -122.4956266046062) bank113828 +113829 POINT(48.293476942368656 -123.2835808612036) bank113829 +113830 POINT(46.67105901790686 -122.49211707317086) bank113830 +113831 POINT(47.83801774920613 -121.58291905111128) bank113831 +113832 POINT(47.46644933818748 -122.01086581590467) bank113832 +113833 POINT(48.18054943063116 -122.89435351976472) bank113833 +113834 POINT(47.611643755489474 -121.49289993285873) bank113834 +113835 POINT(47.93115224137686 -122.89817613020531) bank113835 +113836 POINT(47.63129038655176 -122.61720092876524) bank113836 +113837 POINT(48.53562693069904 -121.71940253713258) bank113837 +113838 POINT(46.91436123608184 -121.68489480647128) bank113838 +113839 POINT(47.328234688282166 -121.38270255804665) bank113839 +113840 POINT(46.779584296420644 -121.86682157229885) bank113840 +113841 POINT(48.491698962563405 -122.88446891344589) bank113841 +113842 POINT(48.11597187561818 -122.98251461275156) bank113842 +113843 POINT(48.005810522896404 -123.04697386696704) bank113843 +113844 POINT(48.54631178167733 -122.6050454029475) bank113844 +113845 POINT(47.89277514383516 -122.39629679866182) bank113845 +113846 POINT(48.485983755421366 -123.12144112073254) bank113846 +113847 POINT(48.35484134262956 -122.83239598514714) bank113847 +113848 POINT(46.82927011913217 -121.54197563115466) bank113848 +113849 POINT(47.17380800837297 -122.72633972901768) bank113849 +113850 POINT(48.401746593145106 -122.92805270895016) bank113850 +113851 POINT(48.4863117031699 -121.35028859083451) bank113851 +113852 POINT(48.20329225178162 -121.71349288088766) bank113852 +113853 POINT(47.9632249508143 -121.95850493175531) bank113853 +113854 POINT(47.9130540568804 -122.12924676512525) bank113854 +113855 POINT(47.109032387955075 -122.83228478981025) bank113855 +113856 POINT(47.572529585984284 -123.18581601862259) bank113856 +113857 POINT(47.11601155132184 -122.53724890159495) bank113857 +113858 POINT(48.05649188386623 -123.09440520835906) bank113858 +113859 POINT(48.390261538798434 -122.02200496495759) bank113859 +113860 POINT(47.26248384358748 -123.24180402229639) bank113860 +113861 POINT(48.60177515345102 -122.16776823648823) bank113861 +113862 POINT(48.47038479297149 -123.21051039348436) bank113862 +113863 POINT(48.109744517950155 -122.13450064077476) bank113863 +113864 POINT(47.66362832803907 -122.64352145723802) bank113864 +113865 POINT(47.31624068214838 -121.64622663098832) bank113865 +113866 POINT(47.06518832382053 -121.69386189925704) bank113866 +113867 POINT(46.66128600507557 -121.55062790128184) bank113867 +113868 POINT(47.8151328549736 -122.23961025660196) bank113868 +113869 POINT(48.13901461045659 -121.38506687135337) bank113869 +113870 POINT(46.95922918007508 -122.39843036005344) bank113870 +113871 POINT(48.35143613645801 -121.91018115032053) bank113871 +113872 POINT(46.81784387568817 -122.71038578602429) bank113872 +113873 POINT(47.77128533927271 -123.32488445136681) bank113873 +113874 POINT(46.82897080169582 -122.74026678446994) bank113874 +113875 POINT(47.114925585942196 -122.14962407554228) bank113875 +113876 POINT(46.79977859027056 -123.3232443332827) bank113876 +113877 POINT(47.08402174886872 -122.82033762860739) bank113877 +113878 POINT(47.35548195234579 -121.3952584113332) bank113878 +113879 POINT(48.21657050551611 -121.51990050475469) bank113879 +113880 POINT(46.73424257071648 -122.3535145183217) bank113880 +113881 POINT(48.30111895833439 -122.03601271642205) bank113881 +113882 POINT(47.328229542792144 -122.2845265184985) bank113882 +113883 POINT(48.41944690623773 -122.4979402222492) bank113883 +113884 POINT(48.27852475416735 -121.91069402675694) bank113884 +113885 POINT(47.92945268169049 -122.19739224717226) bank113885 +113886 POINT(48.301318433902075 -123.09067571239002) bank113886 +113887 POINT(46.82169162547481 -122.05842251109362) bank113887 +113888 POINT(48.458749708850334 -123.32592588824195) bank113888 +113889 POINT(47.65510965439693 -121.72505560517783) bank113889 +113890 POINT(47.053690489582195 -122.77557359394038) bank113890 +113891 POINT(47.70382520731288 -123.09884903918626) bank113891 +113892 POINT(46.91155292132641 -121.6890127939226) bank113892 +113893 POINT(47.366617354868744 -121.717084581293) bank113893 +113894 POINT(47.47505982417802 -122.46721262147145) bank113894 +113895 POINT(48.50986794382942 -121.7250700438361) bank113895 +113896 POINT(48.520579917481584 -122.51846323517685) bank113896 +113897 POINT(47.402948217680624 -121.90053938265756) bank113897 +113898 POINT(48.17770411154979 -122.03457963804041) bank113898 +113899 POINT(48.053226544206275 -122.24422127055897) bank113899 +113900 POINT(48.16425665637436 -123.1196980542125) bank113900 +113901 POINT(48.04710447347378 -121.50679143121717) bank113901 +113902 POINT(47.96250908882752 -122.67478379769942) bank113902 +113903 POINT(47.075829270125816 -121.91829093514335) bank113903 +113904 POINT(46.90010343554112 -122.72276892572117) bank113904 +113905 POINT(47.85547994805812 -122.73464349235824) bank113905 +113906 POINT(46.92896624611595 -122.27549465114514) bank113906 +113907 POINT(48.47768844438799 -123.08363237517915) bank113907 +113908 POINT(47.18847611818518 -122.43348004535883) bank113908 +113909 POINT(47.139461225779215 -123.14959979362915) bank113909 +113910 POINT(47.829786458030014 -123.13574749097113) bank113910 +113911 POINT(48.13810732863295 -121.64369076288666) bank113911 +113912 POINT(48.0622236841308 -122.61402968311204) bank113912 +113913 POINT(47.26749375304769 -121.38748935581734) bank113913 +113914 POINT(48.367614794927995 -122.34130078230929) bank113914 +113915 POINT(48.39457044770007 -123.10364227460934) bank113915 +113916 POINT(47.48484909625947 -122.27949475452131) bank113916 +113917 POINT(47.380058556041234 -123.20203796109323) bank113917 +113918 POINT(47.39368415163589 -121.49771163640612) bank113918 +113919 POINT(48.52288427438339 -121.67420263550295) bank113919 +113920 POINT(46.66121062957192 -122.74498330436649) bank113920 +113921 POINT(47.73291747129199 -123.16793226433168) bank113921 +113922 POINT(48.44167500414558 -121.68764463061862) bank113922 +113923 POINT(46.96369571142597 -121.88482805472869) bank113923 +113924 POINT(47.40047042442048 -121.8316278149318) bank113924 +113925 POINT(48.431947281975155 -121.80960098842988) bank113925 +113926 POINT(48.26725005398428 -122.91036776308276) bank113926 +113927 POINT(47.46146685181808 -122.89774483965003) bank113927 +113928 POINT(48.00301265529292 -123.0887670881724) bank113928 +113929 POINT(48.496845913785755 -121.89708057690737) bank113929 +113930 POINT(48.53674589093525 -121.33280008536815) bank113930 +113931 POINT(47.20390343214413 -122.06844709619483) bank113931 +113932 POINT(48.09987675858819 -123.28904427418968) bank113932 +113933 POINT(46.66697851677875 -121.81189125945949) bank113933 +113934 POINT(46.64612051385037 -123.0116650883464) bank113934 +113935 POINT(47.281980149127456 -122.54017599427421) bank113935 +113936 POINT(46.71965864192422 -122.7835025818219) bank113936 +113937 POINT(47.99053578998858 -121.47568693895546) bank113937 +113938 POINT(48.41597869799724 -122.77760754801015) bank113938 +113939 POINT(48.53340359455498 -123.11884859059391) bank113939 +113940 POINT(48.33834926183275 -121.81992969511734) bank113940 +113941 POINT(46.66108091965189 -122.92732120866201) bank113941 +113942 POINT(47.22474856399463 -122.6218018239387) bank113942 +113943 POINT(46.93948677036429 -121.43319834373186) bank113943 +113944 POINT(47.337640894976545 -122.68535141445797) bank113944 +113945 POINT(47.278405280967405 -121.99564144773458) bank113945 +113946 POINT(47.2038221756992 -122.933804831638) bank113946 +113947 POINT(48.20042891725848 -121.54579746076574) bank113947 +113948 POINT(47.36323119857082 -121.51963482220104) bank113948 +113949 POINT(47.99731912286927 -122.65801689275743) bank113949 +113950 POINT(46.77098803521002 -121.96216288951875) bank113950 +113951 POINT(47.773887656797704 -122.41514848085116) bank113951 +113952 POINT(48.446968191641076 -123.06571686069137) bank113952 +113953 POINT(48.52995378754052 -121.61194400462894) bank113953 +113954 POINT(47.842412322594676 -122.04842563674455) bank113954 +113955 POINT(47.32194741022909 -123.21132695795022) bank113955 +113956 POINT(47.415433960887405 -122.14381404450315) bank113956 +113957 POINT(46.6975335380365 -122.95328653035848) bank113957 +113958 POINT(48.413875729809384 -122.89054251823691) bank113958 +113959 POINT(46.88809536491809 -121.67678786574014) bank113959 +113960 POINT(46.63649264158708 -121.42833931546224) bank113960 +113961 POINT(48.28706765546848 -121.66206471591597) bank113961 +113962 POINT(47.42600145057025 -122.3337117337775) bank113962 +113963 POINT(47.8663046981992 -123.00415975359998) bank113963 +113964 POINT(47.89180148465024 -122.75796159098459) bank113964 +113965 POINT(46.70685203926627 -122.62412436294316) bank113965 +113966 POINT(48.003496588998345 -122.89910995290228) bank113966 +113967 POINT(46.650491955692 -122.67292127477808) bank113967 +113968 POINT(46.89650014006498 -123.1573909905767) bank113968 +113969 POINT(48.564928642036854 -122.93771583351854) bank113969 +113970 POINT(47.0978528723068 -121.4053043247179) bank113970 +113971 POINT(47.39556250390812 -123.08312358769162) bank113971 +113972 POINT(47.59887052307465 -122.4456273090254) bank113972 +113973 POINT(47.12543308366368 -121.53556413217342) bank113973 +113974 POINT(47.42235414513363 -122.70215619360494) bank113974 +113975 POINT(46.72655146710256 -121.7081908815932) bank113975 +113976 POINT(46.87737765294832 -122.51131753311819) bank113976 +113977 POINT(47.19572943236175 -122.26358673025001) bank113977 +113978 POINT(47.05296663549131 -122.87007255789398) bank113978 +113979 POINT(48.13604139186322 -123.18799145394343) bank113979 +113980 POINT(48.556539654562464 -122.33607877118075) bank113980 +113981 POINT(46.75052253783018 -122.94221430107983) bank113981 +113982 POINT(48.397196777630384 -123.27577747764686) bank113982 +113983 POINT(47.65973340809837 -121.7403350553855) bank113983 +113984 POINT(48.374286585396405 -122.23795953999809) bank113984 +113985 POINT(46.93681260797738 -122.53189342337346) bank113985 +113986 POINT(48.513619138174704 -121.70066570203743) bank113986 +113987 POINT(47.68231424961704 -122.36027356518932) bank113987 +113988 POINT(46.641568458297456 -122.9485578949542) bank113988 +113989 POINT(48.36785238265678 -123.02038849339455) bank113989 +113990 POINT(47.54202383318623 -121.37056430863092) bank113990 +113991 POINT(48.29451388246405 -122.3705067237404) bank113991 +113992 POINT(47.3420428943578 -122.15501135967287) bank113992 +113993 POINT(48.503364322799065 -121.91328640400289) bank113993 +113994 POINT(48.34346827474388 -121.55462688882733) bank113994 +113995 POINT(47.61304067693522 -121.44860479528558) bank113995 +113996 POINT(47.78373583321629 -122.11532069322249) bank113996 +113997 POINT(46.79958902220663 -121.78718436804867) bank113997 +113998 POINT(46.69958609236398 -121.3876323049583) bank113998 +113999 POINT(48.579638078020686 -121.34041083816973) bank113999 +114000 POINT(46.65263387701308 -122.72684237451543) bank114000 +114001 POINT(47.82681460263644 -122.8195431478011) bank114001 +114002 POINT(47.501373947127746 -122.35672886282323) bank114002 +114003 POINT(47.93167320486144 -123.10413636757245) bank114003 +114004 POINT(47.28846643418366 -121.66829014591829) bank114004 +114005 POINT(46.95188184960676 -122.35065966784558) bank114005 +114006 POINT(48.47817951533547 -123.17042731652404) bank114006 +114007 POINT(46.93723624808403 -122.60785713893301) bank114007 +114008 POINT(47.834707816158875 -122.06696727772993) bank114008 +114009 POINT(47.684409346337766 -121.71212261998976) bank114009 +114010 POINT(47.572524993137186 -122.32778811071734) bank114010 +114011 POINT(47.493842760487844 -122.24612655530876) bank114011 +114012 POINT(47.96467589615878 -122.0840751721965) bank114012 +114013 POINT(48.0255135126483 -122.93927997854871) bank114013 +114014 POINT(48.442044620746465 -122.22712669452159) bank114014 +114015 POINT(47.36522550871488 -122.01884810726013) bank114015 +114016 POINT(47.50103850211073 -122.27174909112648) bank114016 +114017 POINT(48.133257139156406 -123.31575556002508) bank114017 +114018 POINT(46.610426562631986 -122.7997540841526) bank114018 +114019 POINT(47.9922354950148 -121.33905796470384) bank114019 +114020 POINT(46.91648365751235 -122.78343262387548) bank114020 +114021 POINT(47.77668822977257 -121.95966205320063) bank114021 +114022 POINT(47.624968884497214 -121.84886811391267) bank114022 +114023 POINT(48.303361888769935 -123.07570778936643) bank114023 +114024 POINT(46.73020207071223 -122.26870868184645) bank114024 +114025 POINT(47.08386102889171 -121.82117435427043) bank114025 +114026 POINT(48.34384218022679 -123.28819441891693) bank114026 +114027 POINT(47.041592010185234 -122.0689539676531) bank114027 +114028 POINT(48.57150487711784 -122.24021056905856) bank114028 +114029 POINT(47.370954560354555 -121.96362541428726) bank114029 +114030 POINT(46.63289628188746 -121.67990079627262) bank114030 +114031 POINT(48.133961985086955 -123.18939729761492) bank114031 +114032 POINT(46.73934685820312 -122.74764982336438) bank114032 +114033 POINT(47.88778600047105 -122.20820905267682) bank114033 +114034 POINT(48.07200928325155 -121.66903103899698) bank114034 +114035 POINT(47.96285293375254 -121.94177962864677) bank114035 +114036 POINT(46.93116390083304 -122.98840678300067) bank114036 +114037 POINT(47.993133305711176 -122.15287316847291) bank114037 +114038 POINT(48.008191702891665 -122.20102430394054) bank114038 +114039 POINT(48.07753618864195 -122.63325608151099) bank114039 +114040 POINT(48.261637687179466 -121.6012888442547) bank114040 +114041 POINT(47.869882617742014 -121.6425672679143) bank114041 +114042 POINT(47.49123965956085 -122.17721582464775) bank114042 +114043 POINT(47.150837380621795 -122.34362673525526) bank114043 +114044 POINT(48.402536602406414 -121.43415925041954) bank114044 +114045 POINT(46.89882804482573 -121.64767999173844) bank114045 +114046 POINT(48.255956149748336 -121.39606854813431) bank114046 +114047 POINT(47.86250611785882 -122.30754196475604) bank114047 +114048 POINT(48.32042285445189 -122.19363917804127) bank114048 +114049 POINT(47.19504689643503 -122.88763975910183) bank114049 +114050 POINT(47.578827351430704 -122.08904195066633) bank114050 +114051 POINT(48.25610487534825 -121.45572486611243) bank114051 +114052 POINT(47.68348623284089 -122.01226134969862) bank114052 +114053 POINT(47.281077241947 -123.32575871727693) bank114053 +114054 POINT(48.25065903267994 -123.22523026782996) bank114054 +114055 POINT(47.96346477822154 -122.96029244400184) bank114055 +114056 POINT(48.48186718973215 -122.62187437847955) bank114056 +114057 POINT(46.852517532487695 -121.64658233885247) bank114057 +114058 POINT(46.702676120089144 -121.53914528519) bank114058 +114059 POINT(47.30558408458518 -122.05558833027443) bank114059 +114060 POINT(48.05554029932899 -121.87767331433665) bank114060 +114061 POINT(48.327191353462666 -122.95140593193965) bank114061 +114062 POINT(46.66083222971865 -121.56533015617322) bank114062 +114063 POINT(46.74560078486697 -123.05198365629242) bank114063 +114064 POINT(48.08661034936677 -122.63371857589026) bank114064 +114065 POINT(47.74245346667292 -121.82108849756915) bank114065 +114066 POINT(47.100740850909645 -122.26421295526994) bank114066 +114067 POINT(48.135764299718204 -123.13581815375848) bank114067 +114068 POINT(47.37655345032343 -122.0039820341265) bank114068 +114069 POINT(47.6908948653784 -122.41972790757447) bank114069 +114070 POINT(48.581357565066625 -122.43879091552222) bank114070 +114071 POINT(47.62526213380778 -122.72386586149041) bank114071 +114072 POINT(47.52170092201034 -121.46861448537001) bank114072 +114073 POINT(48.387480319353074 -122.69585643203054) bank114073 +114074 POINT(47.02373320131986 -122.5661356527098) bank114074 +114075 POINT(47.92575994643045 -122.60290439214401) bank114075 +114076 POINT(46.773096839373466 -121.93722062287449) bank114076 +114077 POINT(47.38038222793598 -121.42954946705817) bank114077 +114078 POINT(46.77439938102336 -121.91217964300922) bank114078 +114079 POINT(46.97781059115553 -123.13520667935732) bank114079 +114080 POINT(47.89731743286174 -121.38487983148143) bank114080 +114081 POINT(46.904002023504155 -123.21514104419929) bank114081 +114082 POINT(47.470595279205575 -122.18154479212411) bank114082 +114083 POINT(47.684323463412156 -122.0512698112922) bank114083 +114084 POINT(46.69586461783998 -122.96193172783987) bank114084 +114085 POINT(46.826070448436106 -122.71955309126008) bank114085 +114086 POINT(46.85997144880836 -122.11504802861724) bank114086 +114087 POINT(46.83200532687452 -121.8643532676695) bank114087 +114088 POINT(47.589355662078255 -122.03609279732464) bank114088 +114089 POINT(46.65202420565907 -122.32738216936188) bank114089 +114090 POINT(47.03592921297104 -122.06173222024307) bank114090 +114091 POINT(47.24826783172422 -122.1036666488385) bank114091 +114092 POINT(47.43966425613816 -121.9324541219141) bank114092 +114093 POINT(47.43448047923213 -123.10921934821201) bank114093 +114094 POINT(47.35251806207912 -123.32140419792898) bank114094 +114095 POINT(48.12154564334296 -121.84194730063302) bank114095 +114096 POINT(47.4569897356586 -121.35712417091618) bank114096 +114097 POINT(47.21826603353234 -123.23771450148897) bank114097 +114098 POINT(47.811226610751035 -122.61180230189913) bank114098 +114099 POINT(48.456114677051175 -122.82054906974635) bank114099 +114100 POINT(46.669573486405866 -122.40698540570935) bank114100 +114101 POINT(47.12119771560887 -122.08303549382921) bank114101 +114102 POINT(48.03463693269713 -122.37246730569763) bank114102 +114103 POINT(47.27906459626183 -121.7214887531294) bank114103 +114104 POINT(47.62493360559694 -121.9510084418249) bank114104 +114105 POINT(47.45348630521582 -122.42140628339789) bank114105 +114106 POINT(47.35949057690035 -123.15214138939692) bank114106 +114107 POINT(47.0702571297022 -122.00476665037229) bank114107 +114108 POINT(47.610321991619465 -122.32113913061534) bank114108 +114109 POINT(47.952042590317475 -122.30199248348264) bank114109 +114110 POINT(47.16947456876114 -121.71330158874484) bank114110 +114111 POINT(47.138463633622386 -121.82364573989403) bank114111 +114112 POINT(46.66696907911755 -122.22341300712392) bank114112 +114113 POINT(47.36713116426057 -123.1557873067232) bank114113 +114114 POINT(46.6470417431303 -122.0466462555823) bank114114 +114115 POINT(48.21381359040362 -123.15684829651403) bank114115 +114116 POINT(47.37674796111831 -123.21955796187625) bank114116 +114117 POINT(47.187772987325275 -122.90723242496209) bank114117 +114118 POINT(47.89398531274515 -122.38971812938482) bank114118 +114119 POINT(47.573363454095386 -121.97231425402919) bank114119 +114120 POINT(48.32920701281243 -122.82900621226085) bank114120 +114121 POINT(48.42596062830317 -122.39913518467611) bank114121 +114122 POINT(48.44796377997869 -123.29063740730955) bank114122 +114123 POINT(47.627226641819306 -121.72415822331358) bank114123 +114124 POINT(47.10759462114757 -122.09681781913213) bank114124 +114125 POINT(47.26411143095669 -122.82059663445315) bank114125 +114126 POINT(48.58589052286249 -122.12094731389772) bank114126 +114127 POINT(47.365542435423805 -122.49524146403684) bank114127 +114128 POINT(47.388982729026566 -122.85710429331253) bank114128 +114129 POINT(48.124943018375106 -121.68106810454591) bank114129 +114130 POINT(46.938768115242794 -121.59765318802697) bank114130 +114131 POINT(48.217407829309025 -121.63687080006792) bank114131 +114132 POINT(48.11854997144458 -122.55262183895888) bank114132 +114133 POINT(47.70717015736598 -121.84965003590588) bank114133 +114134 POINT(47.88823479423326 -122.27728589706832) bank114134 +114135 POINT(47.826841479751316 -122.4182816185922) bank114135 +114136 POINT(47.16956316130181 -121.47652522087067) bank114136 +114137 POINT(47.39747338307275 -121.95302485520884) bank114137 +114138 POINT(47.50039462156585 -121.90814848860892) bank114138 +114139 POINT(48.4136089920288 -121.71401292338825) bank114139 +114140 POINT(47.44494798252185 -121.80772418018368) bank114140 +114141 POINT(46.97792769907265 -122.10534499982056) bank114141 +114142 POINT(47.67811327875072 -122.56053457069606) bank114142 +114143 POINT(46.679472922722226 -122.75943358429635) bank114143 +114144 POINT(47.16689578944468 -123.33013150639712) bank114144 +114145 POINT(48.349706369895465 -121.87730159231674) bank114145 +114146 POINT(47.1112103207743 -122.16343355582175) bank114146 +114147 POINT(47.72250617782924 -123.04566131909637) bank114147 +114148 POINT(48.27795469385984 -122.75422188701828) bank114148 +114149 POINT(47.26300157993104 -122.28747185538732) bank114149 +114150 POINT(46.698540187599136 -122.9323719691458) bank114150 +114151 POINT(48.5933454358007 -122.09628441147352) bank114151 +114152 POINT(48.0544515503412 -121.47336737044222) bank114152 +114153 POINT(46.760992809835464 -121.46201440046714) bank114153 +114154 POINT(47.12706695040581 -122.56967216358532) bank114154 +114155 POINT(47.39669328781539 -122.82829966540423) bank114155 +114156 POINT(48.18375896444467 -122.07704911389234) bank114156 +114157 POINT(47.38399604601519 -122.70534919513649) bank114157 +114158 POINT(47.09891387698249 -121.85234908027576) bank114158 +114159 POINT(47.93888786651368 -123.27923801749779) bank114159 +114160 POINT(46.75714451218372 -121.68809101977837) bank114160 +114161 POINT(47.94885383621893 -122.58171078349993) bank114161 +114162 POINT(47.27819040294716 -122.64756821026174) bank114162 +114163 POINT(47.66601605737066 -121.99153750645634) bank114163 +114164 POINT(47.42429504018548 -121.71718151860192) bank114164 +114165 POINT(47.11383128665779 -121.5495696031178) bank114165 +114166 POINT(46.73107900687648 -121.57304754884869) bank114166 +114167 POINT(48.316574480020144 -121.39383440227492) bank114167 +114168 POINT(47.3544755971995 -121.44618514260642) bank114168 +114169 POINT(47.35064158506258 -122.92470162521225) bank114169 +114170 POINT(47.55274258661477 -122.95475774093845) bank114170 +114171 POINT(48.53920088673994 -121.7158917119554) bank114171 +114172 POINT(48.20398541131389 -121.42447356920825) bank114172 +114173 POINT(46.76836963125803 -121.3951551954316) bank114173 +114174 POINT(48.23408077866433 -122.87167223670569) bank114174 +114175 POINT(47.75125190430744 -122.60944013677248) bank114175 +114176 POINT(47.1010705619549 -122.70337612489651) bank114176 +114177 POINT(47.77192117288655 -122.73041002836526) bank114177 +114178 POINT(48.11465684459471 -121.79619419577871) bank114178 +114179 POINT(46.91244324558351 -123.29802346370519) bank114179 +114180 POINT(47.7569397653316 -122.67863597237493) bank114180 +114181 POINT(46.799051062244594 -122.46789737312598) bank114181 +114182 POINT(47.22754309785301 -122.8203709175707) bank114182 +114183 POINT(47.55101031199863 -122.9079484944894) bank114183 +114184 POINT(46.883091583930714 -121.88568672029828) bank114184 +114185 POINT(46.98907425718455 -122.17444925300342) bank114185 +114186 POINT(46.84368880221848 -122.12454636158806) bank114186 +114187 POINT(48.46797112856697 -122.80191940133841) bank114187 +114188 POINT(48.2735802477016 -122.05277941585973) bank114188 +114189 POINT(47.84123592013988 -121.99674069343058) bank114189 +114190 POINT(48.57523154296304 -121.69560586192107) bank114190 +114191 POINT(47.667519658915374 -122.3222620153516) bank114191 +114192 POINT(47.17800303461822 -122.45291467611045) bank114192 +114193 POINT(46.65889147265444 -121.86079839585591) bank114193 +114194 POINT(47.91485457974643 -122.77906994422429) bank114194 +114195 POINT(47.610032086869964 -122.29156262647882) bank114195 +114196 POINT(47.41831637001607 -122.94173178395563) bank114196 +114197 POINT(47.253241578368005 -122.68417932518109) bank114197 +114198 POINT(47.34858888457585 -121.39775697640484) bank114198 +114199 POINT(47.77521231442643 -122.37321193371118) bank114199 +114200 POINT(47.83020192798772 -122.36901020998341) bank114200 +114201 POINT(47.18776965315988 -121.8767562230802) bank114201 +114202 POINT(47.29313494567755 -122.91144840448317) bank114202 +114203 POINT(47.350650595065424 -121.98132976277705) bank114203 +114204 POINT(47.81250794225624 -122.74006288744059) bank114204 +114205 POINT(46.868987009380454 -123.29006688751927) bank114205 +114206 POINT(46.80864062565878 -121.4557757724031) bank114206 +114207 POINT(47.50847726005508 -122.62250242829386) bank114207 +114208 POINT(47.090473915329305 -123.05526615320868) bank114208 +114209 POINT(48.57129891335837 -123.02589591166456) bank114209 +114210 POINT(47.753824215819115 -122.3612071105442) bank114210 +114211 POINT(47.49415091307027 -122.40789396144409) bank114211 +114212 POINT(48.515606562613456 -122.22585317555911) bank114212 +114213 POINT(48.091544777155995 -122.87379583388082) bank114213 +114214 POINT(47.89567071188035 -122.41525777617174) bank114214 +114215 POINT(48.251811001281865 -122.08739669057753) bank114215 +114216 POINT(47.744907904203814 -121.97321013882971) bank114216 +114217 POINT(48.33886076820576 -123.07454264832889) bank114217 +114218 POINT(47.896275894177855 -123.02790014882854) bank114218 +114219 POINT(47.10905200943289 -122.42539914171515) bank114219 +114220 POINT(47.34864135014873 -122.51259280351097) bank114220 +114221 POINT(46.792529890633844 -122.50327466558501) bank114221 +114222 POINT(47.29841758954194 -122.06765265260321) bank114222 +114223 POINT(47.7041302838146 -121.97816577932652) bank114223 +114224 POINT(48.163953209895794 -122.69891658756517) bank114224 +114225 POINT(47.38382237948464 -122.47138607572035) bank114225 +114226 POINT(48.60225861154673 -121.51995039923754) bank114226 +114227 POINT(47.796777688330785 -121.91595961718734) bank114227 +114228 POINT(48.20533009626408 -122.97786378460509) bank114228 +114229 POINT(48.41710310448107 -122.69756239342499) bank114229 +114230 POINT(46.75856071863448 -121.68879723797822) bank114230 +114231 POINT(46.73352597838339 -122.29641527281926) bank114231 +114232 POINT(47.52246270949054 -122.50580526958335) bank114232 +114233 POINT(48.17211013451219 -123.12521714531532) bank114233 +114234 POINT(48.536914089436586 -122.81696835146386) bank114234 +114235 POINT(47.11880930224106 -121.48516750590292) bank114235 +114236 POINT(46.66678447297755 -121.46400995314332) bank114236 +114237 POINT(47.548208616921514 -122.44536823773966) bank114237 +114238 POINT(48.160435382906996 -122.95817607436952) bank114238 +114239 POINT(47.41388177755846 -121.76491338265484) bank114239 +114240 POINT(48.484630953717016 -121.94893800251754) bank114240 +114241 POINT(48.46860109994177 -123.21206986406106) bank114241 +114242 POINT(47.926254947798675 -123.10964302169816) bank114242 +114243 POINT(46.88865442335535 -122.55401169506176) bank114243 +114244 POINT(47.41977091936262 -123.24648765777289) bank114244 +114245 POINT(47.32574838662086 -121.9577436729727) bank114245 +114246 POINT(48.401887089358134 -123.26174136247248) bank114246 +114247 POINT(47.698922342438046 -121.33368539931519) bank114247 +114248 POINT(47.86545312765742 -122.35730297656234) bank114248 +114249 POINT(47.450719570719265 -121.97545400676871) bank114249 +114250 POINT(46.67007127023632 -122.49597668371693) bank114250 +114251 POINT(47.66689695848336 -122.3989505471797) bank114251 +114252 POINT(48.566824460005165 -121.6810475784042) bank114252 +114253 POINT(46.69728393843927 -122.58355318853648) bank114253 +114254 POINT(48.253292294694766 -123.14326957523534) bank114254 +114255 POINT(48.27734344927976 -122.61864027370271) bank114255 +114256 POINT(47.67742773073566 -122.35157221908017) bank114256 +114257 POINT(48.09415397849847 -122.83744513832906) bank114257 +114258 POINT(47.13808541156531 -122.14440853478597) bank114258 +114259 POINT(48.51500251088222 -122.35913427383404) bank114259 +114260 POINT(48.557033363611154 -122.35955092031362) bank114260 +114261 POINT(48.499973585325236 -122.25964654236121) bank114261 +114262 POINT(46.81051958784807 -122.54832720466102) bank114262 +114263 POINT(48.3529529055902 -121.81081006270857) bank114263 +114264 POINT(48.37918806320362 -122.97946753981122) bank114264 +114265 POINT(47.829929090521574 -121.3979855855605) bank114265 +114266 POINT(47.93385766531048 -122.69877937819598) bank114266 +114267 POINT(47.16409202504435 -123.04223684889357) bank114267 +114268 POINT(46.848595290164624 -121.63115775929414) bank114268 +114269 POINT(47.23437958126753 -122.61533589736811) bank114269 +114270 POINT(48.05286953714487 -122.54121840198097) bank114270 +114271 POINT(46.62753202652263 -121.71634186431872) bank114271 +114272 POINT(47.88384094310385 -121.66398901359625) bank114272 +114273 POINT(46.8681804460471 -121.56899112407001) bank114273 +114274 POINT(46.686448634842876 -121.95966054565184) bank114274 +114275 POINT(47.620803326546806 -122.85550204767158) bank114275 +114276 POINT(48.48272032531794 -122.59611852844296) bank114276 +114277 POINT(47.256661641672714 -121.64851112492957) bank114277 +114278 POINT(48.52377800481224 -122.85037165551987) bank114278 +114279 POINT(48.55525934318018 -123.27978675826546) bank114279 +114280 POINT(48.49887104377751 -122.560376751273) bank114280 +114281 POINT(47.46070516780025 -122.33640322531024) bank114281 +114282 POINT(48.088048872067716 -122.50146081352578) bank114282 +114283 POINT(48.46222707461444 -121.48567810123646) bank114283 +114284 POINT(48.23509971147272 -122.40532144241723) bank114284 +114285 POINT(47.880309927431405 -122.36082470148122) bank114285 +114286 POINT(48.412128693652015 -122.78401903653588) bank114286 +114287 POINT(46.92116259320679 -123.0223879865229) bank114287 +114288 POINT(47.35724109141411 -122.55518346502234) bank114288 +114289 POINT(47.483147865931414 -122.61700200761854) bank114289 +114290 POINT(48.01060504790503 -123.32044877561788) bank114290 +114291 POINT(46.98695889195064 -122.68752273331039) bank114291 +114292 POINT(47.89751105122953 -122.41909810764065) bank114292 +114293 POINT(47.069042514482305 -121.34276102660004) bank114293 +114294 POINT(47.5962158436716 -122.18963004517497) bank114294 +114295 POINT(47.95575278555289 -123.21530818397979) bank114295 +114296 POINT(48.42996196367881 -123.28289098193129) bank114296 +114297 POINT(48.218768988329664 -121.54365275445744) bank114297 +114298 POINT(47.94614091595035 -122.47880196637107) bank114298 +114299 POINT(47.93070313374351 -122.2042359795275) bank114299 +114300 POINT(48.0627423030542 -122.56113236475656) bank114300 +114301 POINT(47.71437718987588 -122.63142905720686) bank114301 +114302 POINT(48.35375584776997 -121.96894042034224) bank114302 +114303 POINT(46.94391413543797 -122.23800221388129) bank114303 +114304 POINT(48.08605441977881 -122.92354942386463) bank114304 +114305 POINT(47.77514293992697 -122.41076701522827) bank114305 +114306 POINT(46.901707677011984 -123.05587654215913) bank114306 +114307 POINT(47.742575456586856 -122.32402108296438) bank114307 +114308 POINT(47.49803689319139 -121.64063495618902) bank114308 +114309 POINT(47.59744401102819 -123.0986593250559) bank114309 +114310 POINT(47.134919273765135 -122.29085322240313) bank114310 +114311 POINT(47.85269935012272 -122.38656049252415) bank114311 +114312 POINT(48.281549715696514 -122.98846089741785) bank114312 +114313 POINT(47.71727373054089 -121.710333520649) bank114313 +114314 POINT(47.64743940702462 -122.08289627010228) bank114314 +114315 POINT(48.59457476584732 -123.06433707366277) bank114315 +114316 POINT(48.418721206365674 -122.53581624495014) bank114316 +114317 POINT(48.145663415506355 -121.99360495372711) bank114317 +114318 POINT(48.191199811457025 -121.51964599278236) bank114318 +114319 POINT(48.513695706391246 -122.38554601097019) bank114319 +114320 POINT(48.024705004440015 -121.74571794381237) bank114320 +114321 POINT(48.47329402175422 -123.25263050446287) bank114321 +114322 POINT(47.350144076891965 -122.62335436300734) bank114322 +114323 POINT(47.69237668719894 -122.02185999875125) bank114323 +114324 POINT(47.142123469002044 -122.73906053766707) bank114324 +114325 POINT(46.816377499849295 -122.59071410986913) bank114325 +114326 POINT(46.84936318383616 -122.63259127790141) bank114326 +114327 POINT(48.20472579316941 -121.4931874916707) bank114327 +114328 POINT(47.47185833057129 -121.65816146584399) bank114328 +114329 POINT(46.968411226637066 -121.64818802034766) bank114329 +114330 POINT(47.50094582184314 -121.68028540387718) bank114330 +114331 POINT(46.64066944530735 -122.82451491481655) bank114331 +114332 POINT(47.39005598630944 -122.6450344501935) bank114332 +114333 POINT(47.72191256887225 -123.01641629366651) bank114333 +114334 POINT(48.47209383444114 -122.4657326834376) bank114334 +114335 POINT(48.36691412644558 -121.49642975312038) bank114335 +114336 POINT(47.67481081393618 -121.4012803497082) bank114336 +114337 POINT(47.62662859108583 -122.204163874269) bank114337 +114338 POINT(47.69080191011723 -122.64145500540879) bank114338 +114339 POINT(47.947571439850144 -122.1328257264141) bank114339 +114340 POINT(47.24441061263015 -122.27642632260346) bank114340 +114341 POINT(48.18842406701086 -121.4560635162625) bank114341 +114342 POINT(46.71736255953309 -121.8209141858793) bank114342 +114343 POINT(47.5947061966057 -122.41900213904808) bank114343 +114344 POINT(47.89909866939959 -121.87366191049054) bank114344 +114345 POINT(48.13720484572206 -122.54448273518393) bank114345 +114346 POINT(46.98778674776323 -123.16852596995027) bank114346 +114347 POINT(46.86569813634361 -122.37964393930818) bank114347 +114348 POINT(48.31630382872906 -121.82989913449026) bank114348 +114349 POINT(46.88502602658117 -122.82825767283097) bank114349 +114350 POINT(46.90118199337789 -122.50099827152151) bank114350 +114351 POINT(48.31575583989936 -122.07467860224982) bank114351 +114352 POINT(47.33431525757431 -121.65247450619349) bank114352 +114353 POINT(48.57284329259515 -122.60271011189784) bank114353 +114354 POINT(46.68989451425882 -121.60729127074225) bank114354 +114355 POINT(48.526741307698906 -122.54714995584833) bank114355 +114356 POINT(47.173660589845376 -121.56766548059129) bank114356 +114357 POINT(46.927874972004396 -121.76065876201345) bank114357 +114358 POINT(48.02608521355586 -123.28626654518962) bank114358 +114359 POINT(48.49353856723904 -122.62402323317941) bank114359 +114360 POINT(47.91148224028034 -122.38812538543917) bank114360 +114361 POINT(47.64356290041369 -121.38640932159252) bank114361 +114362 POINT(47.412653342299265 -121.64016359250168) bank114362 +114363 POINT(46.60932707498327 -122.04793915097946) bank114363 +114364 POINT(47.69994649981284 -121.73872674611182) bank114364 +114365 POINT(48.289463108180094 -122.85763713855691) bank114365 +114366 POINT(47.64016909894316 -122.18621912477916) bank114366 +114367 POINT(48.328004454856526 -122.49639403267463) bank114367 +114368 POINT(48.027726203188365 -122.95498961524788) bank114368 +114369 POINT(47.709406229837 -122.95570764571606) bank114369 +114370 POINT(47.732297221608235 -122.98864996949695) bank114370 +114371 POINT(48.075012356289555 -123.23099144426561) bank114371 +114372 POINT(47.84241044595658 -123.01970677596385) bank114372 +114373 POINT(47.08437570146258 -122.8695271219823) bank114373 +114374 POINT(47.90290410139191 -122.46627837509705) bank114374 +114375 POINT(47.66018021765399 -122.46914914054848) bank114375 +114376 POINT(47.27694015049968 -122.39638932659167) bank114376 +114377 POINT(48.31350949300496 -122.6550404764353) bank114377 +114378 POINT(47.52564712682762 -121.55291519966097) bank114378 +114379 POINT(48.03860926580276 -121.91524902756844) bank114379 +114380 POINT(48.340564276955284 -121.90728335892194) bank114380 +114381 POINT(47.301375589092906 -121.5128007609838) bank114381 +114382 POINT(47.52023942386189 -122.24556020731752) bank114382 +114383 POINT(47.940044551396866 -123.23939971060462) bank114383 +114384 POINT(47.462261628124054 -123.08201843106173) bank114384 +114385 POINT(48.183926850937645 -122.54796226588797) bank114385 +114386 POINT(46.93071050590375 -122.88471913578844) bank114386 +114387 POINT(47.01181317604392 -122.57184083266694) bank114387 +114388 POINT(46.89456774835951 -121.88700834839004) bank114388 +114389 POINT(47.06781971901713 -121.68195461492569) bank114389 +114390 POINT(47.52585011623417 -122.78837205472493) bank114390 +114391 POINT(47.72869509800735 -122.61537738629976) bank114391 +114392 POINT(47.5803815672687 -122.21838371981589) bank114392 +114393 POINT(47.40274296890466 -121.54155250349022) bank114393 +114394 POINT(48.26368525738687 -122.92200989755239) bank114394 +114395 POINT(48.3193886626164 -122.93855246414533) bank114395 +114396 POINT(47.287528853804 -122.12059874245772) bank114396 +114397 POINT(47.32589032459941 -122.47892490624315) bank114397 +114398 POINT(47.88660543412632 -121.75941426959514) bank114398 +114399 POINT(47.04512333948547 -122.27771976922354) bank114399 +114400 POINT(48.43247351693445 -123.02899688773756) bank114400 +114401 POINT(48.40050053575877 -122.51930909431734) bank114401 +114402 POINT(46.868974631857654 -121.60826356971938) bank114402 +114403 POINT(47.20096836251688 -122.15664110109608) bank114403 +114404 POINT(47.88089641674686 -121.35367480340469) bank114404 +114405 POINT(47.19569354456157 -122.78507952224683) bank114405 +114406 POINT(47.99959647079105 -122.68852961398981) bank114406 +114407 POINT(47.79214896791963 -122.34168476367293) bank114407 +114408 POINT(47.59926643839925 -121.48382660203073) bank114408 +114409 POINT(47.80301810301386 -122.13979202777111) bank114409 +114410 POINT(48.58466609142708 -122.22699954383526) bank114410 +114411 POINT(47.448822149217385 -121.44550194788569) bank114411 +114412 POINT(47.66464092300731 -121.52655529176994) bank114412 +114413 POINT(47.34492930487359 -121.87143261379285) bank114413 +114414 POINT(47.68661316854203 -122.67077064830002) bank114414 +114415 POINT(48.308374096283764 -122.30606077796493) bank114415 +114416 POINT(47.99841005404918 -122.98417614973272) bank114416 +114417 POINT(48.43724031044097 -122.15310082106905) bank114417 +114418 POINT(48.088598153992926 -122.25220067665705) bank114418 +114419 POINT(46.700038303562856 -123.23694723554632) bank114419 +114420 POINT(46.718985522566165 -121.99523982661646) bank114420 +114421 POINT(47.63367530943926 -123.04750318525056) bank114421 +114422 POINT(47.52416508205024 -121.43090362830957) bank114422 +114423 POINT(48.12078991804419 -121.86640463954798) bank114423 +114424 POINT(48.54563189925743 -122.11871038788631) bank114424 +114425 POINT(48.20986918151677 -122.16418084864493) bank114425 +114426 POINT(46.978414702856696 -121.66178221955106) bank114426 +114427 POINT(48.03518248737658 -121.48786572951647) bank114427 +114428 POINT(48.28640214987417 -121.38571967296402) bank114428 +114429 POINT(46.835861685228345 -121.42276835480666) bank114429 +114430 POINT(48.58903840251065 -123.04807917382048) bank114430 +114431 POINT(48.369537292102144 -121.60651284749616) bank114431 +114432 POINT(48.182859891531145 -122.46001437911924) bank114432 +114433 POINT(48.24498655123601 -122.04463288681484) bank114433 +114434 POINT(47.456797076638445 -121.7394753344205) bank114434 +114435 POINT(47.880299760602384 -122.4739535288951) bank114435 +114436 POINT(48.47418567568232 -122.80121766859347) bank114436 +114437 POINT(47.40588817146432 -122.81699390693889) bank114437 +114438 POINT(47.98407917865584 -121.33355329891388) bank114438 +114439 POINT(47.19363118783885 -121.83305104535286) bank114439 +114440 POINT(46.838391921197356 -122.53909977142149) bank114440 +114441 POINT(48.0112343510726 -122.26291020169455) bank114441 +114442 POINT(46.80523463851648 -123.16700048819378) bank114442 +114443 POINT(47.73238157560816 -121.99820389329562) bank114443 +114444 POINT(47.330020519867794 -122.44412974323882) bank114444 +114445 POINT(48.01943279642574 -121.37027782874581) bank114445 +114446 POINT(47.68951081881498 -123.30128790375048) bank114446 +114447 POINT(47.89770335155281 -122.93131345343475) bank114447 +114448 POINT(47.59920475768851 -122.28578714133741) bank114448 +114449 POINT(47.9795555731591 -121.38596182825552) bank114449 +114450 POINT(48.32332499440573 -121.46598922748426) bank114450 +114451 POINT(46.83550526298611 -122.4350505878599) bank114451 +114452 POINT(48.30578065563697 -123.02174323102986) bank114452 +114453 POINT(47.1636615955816 -122.9912265343193) bank114453 +114454 POINT(48.29596285771037 -122.27763196163696) bank114454 +114455 POINT(47.19306331798221 -121.36539328788854) bank114455 +114456 POINT(47.43688990760267 -122.32343387118891) bank114456 +114457 POINT(47.666477063520624 -122.27983426242388) bank114457 +114458 POINT(48.06133650018815 -121.478128339075) bank114458 +114459 POINT(48.22208315296083 -123.31554172759562) bank114459 +114460 POINT(47.602815134901824 -122.70540100052003) bank114460 +114461 POINT(47.14869152032249 -122.88743105381674) bank114461 +114462 POINT(48.583899663900326 -123.1234215976279) bank114462 +114463 POINT(48.46005189929757 -122.92031347448776) bank114463 +114464 POINT(47.0543864287767 -121.65210302200126) bank114464 +114465 POINT(46.755125465482976 -121.90816123315757) bank114465 +114466 POINT(47.125924643980376 -122.85408142023763) bank114466 +114467 POINT(47.97501216724317 -122.56148447066113) bank114467 +114468 POINT(46.96511300359904 -122.57582088818515) bank114468 +114469 POINT(47.60368810211417 -121.36349111957378) bank114469 +114470 POINT(46.92819258375375 -123.14850473781048) bank114470 +114471 POINT(47.75227118841227 -123.01521535095769) bank114471 +114472 POINT(47.289319117415964 -122.838015751478) bank114472 +114473 POINT(47.60861260427904 -121.574345900092) bank114473 +114474 POINT(47.815908319901894 -122.3082416979817) bank114474 +114475 POINT(48.501196736609245 -121.63718524582627) bank114475 +114476 POINT(48.52223573991715 -122.15920009513684) bank114476 +114477 POINT(46.77326550436791 -121.61867649509759) bank114477 +114478 POINT(47.43049471200338 -122.70561230119138) bank114478 +114479 POINT(47.39916737614854 -122.33670851859031) bank114479 +114480 POINT(47.21292427232835 -122.1122242969836) bank114480 +114481 POINT(46.705833833876795 -121.85267543700186) bank114481 +114482 POINT(47.861050507079085 -121.38868982646602) bank114482 +114483 POINT(48.261527539736505 -121.72989030849615) bank114483 +114484 POINT(47.41352482949167 -121.6985112911444) bank114484 +114485 POINT(48.54331647085226 -123.31144400955627) bank114485 +114486 POINT(47.66198381500477 -121.33235951445798) bank114486 +114487 POINT(48.292231500140026 -122.52432635900182) bank114487 +114488 POINT(47.216373222083675 -121.64622266925912) bank114488 +114489 POINT(46.80007491589448 -121.51858593073342) bank114489 +114490 POINT(46.6772359579195 -122.46593467667743) bank114490 +114491 POINT(46.73121807159413 -122.02870863923515) bank114491 +114492 POINT(47.664774993231845 -122.39341620742096) bank114492 +114493 POINT(47.94956902790918 -122.07619077048288) bank114493 +114494 POINT(47.86609225923526 -122.87174523354491) bank114494 +114495 POINT(47.04148887837831 -123.01183562187526) bank114495 +114496 POINT(47.346810300907826 -121.52274698154379) bank114496 +114497 POINT(46.93625453129955 -123.07617370184683) bank114497 +114498 POINT(47.13695283656877 -121.7646076285945) bank114498 +114499 POINT(48.04671776844106 -121.83276306429246) bank114499 +114500 POINT(46.87886474059655 -122.56804378736325) bank114500 +114501 POINT(47.37203096457236 -123.00477759680577) bank114501 +114502 POINT(48.23077663669563 -122.91900574206517) bank114502 +114503 POINT(46.70420912722942 -121.440555700694) bank114503 +114504 POINT(47.8849250434303 -121.6684539124795) bank114504 +114505 POINT(48.60446993322262 -122.73143486909774) bank114505 +114506 POINT(48.52593526927629 -122.89357921530323) bank114506 +114507 POINT(48.25445075410493 -122.23143308711974) bank114507 +114508 POINT(47.723334713786166 -121.61015096139654) bank114508 +114509 POINT(47.09373724994088 -122.11101528078142) bank114509 +114510 POINT(46.95909712572665 -121.38328878970741) bank114510 +114511 POINT(47.837699084029836 -122.3657231010906) bank114511 +114512 POINT(46.73098761810503 -121.8294621871504) bank114512 +114513 POINT(47.63694251249325 -122.8681960870121) bank114513 +114514 POINT(47.90097943026018 -122.02709222585939) bank114514 +114515 POINT(46.98081198212414 -122.85169793744593) bank114515 +114516 POINT(47.571515528454256 -121.8202672111345) bank114516 +114517 POINT(48.31821143774638 -121.37914528697172) bank114517 +114518 POINT(47.14912780391555 -122.87068989397231) bank114518 +114519 POINT(48.39730666483221 -121.80166686530868) bank114519 +114520 POINT(46.78258422810657 -123.22260702827447) bank114520 +114521 POINT(48.55724156415022 -122.22779445357969) bank114521 +114522 POINT(47.706068458005355 -122.66843001120775) bank114522 +114523 POINT(47.73597732832043 -123.25481195313309) bank114523 +114524 POINT(48.36919954923446 -122.11970349797663) bank114524 +114525 POINT(46.77242271032687 -122.54093649060677) bank114525 +114526 POINT(48.591346285783516 -122.99407998106007) bank114526 +114527 POINT(47.774563259532734 -121.64870597486889) bank114527 +114528 POINT(47.993860546123614 -121.96840175376822) bank114528 +114529 POINT(48.08270611283164 -122.61060305587988) bank114529 +114530 POINT(47.45372675887623 -121.91287875927492) bank114530 +114531 POINT(48.025969556438504 -121.87925742259434) bank114531 +114532 POINT(47.4705467149802 -121.54912162182538) bank114532 +114533 POINT(46.88952899428275 -122.57257973285733) bank114533 +114534 POINT(47.762196428309004 -122.81317850000244) bank114534 +114535 POINT(47.42291242789074 -123.28570777458914) bank114535 +114536 POINT(47.28548334162149 -122.50112946673288) bank114536 +114537 POINT(46.684735939039214 -122.74294125284833) bank114537 +114538 POINT(47.90472167046967 -122.91265536914845) bank114538 +114539 POINT(47.4537559573929 -121.62989540332798) bank114539 +114540 POINT(47.80251445540048 -121.86914787472047) bank114540 +114541 POINT(48.120733931452115 -121.3768714215218) bank114541 +114542 POINT(48.00929306239011 -123.1907782641887) bank114542 +114543 POINT(48.052274532621794 -121.99260272048394) bank114543 +114544 POINT(48.468333862561884 -121.41903774058979) bank114544 +114545 POINT(47.18151417639343 -121.36360156908628) bank114545 +114546 POINT(47.63326372621313 -121.75823161666612) bank114546 +114547 POINT(47.97770562921333 -121.63170636082128) bank114547 +114548 POINT(47.987013798075814 -122.50136738812242) bank114548 +114549 POINT(48.22860684421581 -122.43546075426906) bank114549 +114550 POINT(47.947813620388985 -123.26118845270399) bank114550 +114551 POINT(47.49015240070919 -121.72203597944248) bank114551 +114552 POINT(46.71588116987076 -121.73630388021577) bank114552 +114553 POINT(47.038733208420986 -121.65479630585581) bank114553 +114554 POINT(47.62309561085502 -122.03581555668438) bank114554 +114555 POINT(47.704944024072624 -122.0675938671814) bank114555 +114556 POINT(46.82390139672651 -122.62520266669648) bank114556 +114557 POINT(47.23573366941081 -123.07390604025633) bank114557 +114558 POINT(46.79276870205012 -122.66658291489776) bank114558 +114559 POINT(48.28393332729322 -121.96411515827135) bank114559 +114560 POINT(46.995967453018864 -122.23909999799541) bank114560 +114561 POINT(47.41081397171358 -121.6313605699185) bank114561 +114562 POINT(47.63804040383593 -122.51540242408495) bank114562 +114563 POINT(46.809498168202005 -121.35274541939005) bank114563 +114564 POINT(47.9844386073834 -122.85972777999456) bank114564 +114565 POINT(47.1440300384543 -122.26928816644262) bank114565 +114566 POINT(48.227424960569195 -122.99473729646307) bank114566 +114567 POINT(47.453034114212684 -122.96457747897428) bank114567 +114568 POINT(47.5856641940485 -122.34450234904916) bank114568 +114569 POINT(47.77576934877152 -123.05339870932598) bank114569 +114570 POINT(47.12948622334049 -123.2531318300069) bank114570 +114571 POINT(47.87780414562285 -122.78191727794639) bank114571 +114572 POINT(48.34341792915265 -123.14891989379826) bank114572 +114573 POINT(47.48984633176825 -122.63329078894733) bank114573 +114574 POINT(48.18814703374777 -122.02358721284575) bank114574 +114575 POINT(48.025747246357625 -121.98845489903096) bank114575 +114576 POINT(48.46894098533234 -122.51734338534546) bank114576 +114577 POINT(47.59124428161299 -122.82163780926496) bank114577 +114578 POINT(47.49315146201911 -122.97081869604963) bank114578 +114579 POINT(46.702310370191014 -123.28479357359247) bank114579 +114580 POINT(47.35447404555137 -122.98775171392361) bank114580 +114581 POINT(46.69296824275793 -122.37274585537055) bank114581 +114582 POINT(46.88319094605359 -122.57833992780948) bank114582 +114583 POINT(47.458679518405155 -123.0558931379196) bank114583 +114584 POINT(47.046558754231896 -122.00043159722361) bank114584 +114585 POINT(48.573560863735544 -123.04627883779388) bank114585 +114586 POINT(47.745455009365266 -122.25400027892164) bank114586 +114587 POINT(46.763843819436616 -122.09724148895359) bank114587 +114588 POINT(47.6484272603893 -121.3401467272188) bank114588 +114589 POINT(46.895610224362606 -122.23708439039308) bank114589 +114590 POINT(46.90880149342409 -122.1002205017038) bank114590 +114591 POINT(47.57193223870593 -121.8191831905676) bank114591 +114592 POINT(47.991124675594094 -121.69785184963166) bank114592 +114593 POINT(46.9015678826941 -121.67766987274014) bank114593 +114594 POINT(46.75888151973576 -122.22295561745146) bank114594 +114595 POINT(47.443313240884876 -122.32727606861096) bank114595 +114596 POINT(47.15350212702457 -121.64141697422075) bank114596 +114597 POINT(47.818351428058044 -122.16494032012272) bank114597 +114598 POINT(46.95417542719702 -121.87346551185881) bank114598 +114599 POINT(48.32316841584349 -122.9902335207344) bank114599 +114600 POINT(48.14328515567859 -122.54604390408109) bank114600 +114601 POINT(47.343212135156236 -122.39471414355769) bank114601 +114602 POINT(47.022379647187385 -121.56564465986091) bank114602 +114603 POINT(47.022908318680905 -122.6858350478186) bank114603 +114604 POINT(47.51534414678304 -123.22129782568666) bank114604 +114605 POINT(46.787345084504544 -122.41111833949822) bank114605 +114606 POINT(47.562441583248194 -123.01763326223403) bank114606 +114607 POINT(46.760535097211 -121.67296079436379) bank114607 +114608 POINT(47.07126139441144 -121.66425046686996) bank114608 +114609 POINT(47.56245467889447 -123.05439744437912) bank114609 +114610 POINT(47.99919366182703 -121.48699045717726) bank114610 +114611 POINT(47.96734374575677 -122.61033299499972) bank114611 +114612 POINT(47.66510869879555 -123.3205504968007) bank114612 +114613 POINT(47.46657062175487 -122.31722524467578) bank114613 +114614 POINT(46.71183016972353 -122.47760350573637) bank114614 +114615 POINT(47.72775261796168 -123.28642404606857) bank114615 +114616 POINT(47.00531883778195 -122.58381098002866) bank114616 +114617 POINT(47.17944085441192 -121.3466466247247) bank114617 +114618 POINT(47.34906397250456 -122.29306955765949) bank114618 +114619 POINT(46.776654431188355 -122.21599568629037) bank114619 +114620 POINT(47.1116272029945 -122.53289787554881) bank114620 +114621 POINT(47.40585300474303 -121.96420796838864) bank114621 +114622 POINT(47.59398830085451 -122.92527731415801) bank114622 +114623 POINT(47.41328308558836 -122.2961991553761) bank114623 +114624 POINT(48.372388726285834 -122.23411891390013) bank114624 +114625 POINT(47.76187231318783 -121.36152953625361) bank114625 +114626 POINT(48.002204073566965 -121.49105824189661) bank114626 +114627 POINT(47.5017352017344 -123.15588850184193) bank114627 +114628 POINT(46.625246836199885 -123.03782655964008) bank114628 +114629 POINT(48.59869415457857 -122.77649842002786) bank114629 +114630 POINT(47.590705014951325 -123.32904330786667) bank114630 +114631 POINT(48.011070608438544 -121.86668454315559) bank114631 +114632 POINT(47.56163797214062 -121.95353387111048) bank114632 +114633 POINT(47.759588245090285 -122.61980548483459) bank114633 +114634 POINT(48.180798888741805 -121.63887019188823) bank114634 +114635 POINT(46.79672096816501 -122.31271648074615) bank114635 +114636 POINT(46.89708595351971 -122.17511449532454) bank114636 +114637 POINT(46.96300847593061 -122.57190760455065) bank114637 +114638 POINT(47.78837156941362 -122.12355179506142) bank114638 +114639 POINT(48.143010767588024 -121.99008554342774) bank114639 +114640 POINT(48.35059736496962 -121.57040190433972) bank114640 +114641 POINT(47.460176658758 -121.40727658578989) bank114641 +114642 POINT(47.64684393027638 -123.26611824073277) bank114642 +114643 POINT(48.162942796173915 -122.55882673225167) bank114643 +114644 POINT(48.51851200451899 -123.1800065921265) bank114644 +114645 POINT(48.00359820899784 -122.58164059019803) bank114645 +114646 POINT(47.91303654624227 -122.5503453556471) bank114646 +114647 POINT(47.50086851959965 -121.56064648020283) bank114647 +114648 POINT(47.79075426977041 -122.28069374492108) bank114648 +114649 POINT(47.459811995837335 -121.64445673801447) bank114649 +114650 POINT(47.276195928520046 -122.00212693115752) bank114650 +114651 POINT(48.24465234546431 -122.27777722984757) bank114651 +114652 POINT(47.88894505996244 -123.23547432852665) bank114652 +114653 POINT(48.13090964224034 -122.12395798864281) bank114653 +114654 POINT(47.90399638900751 -121.84583471931519) bank114654 +114655 POINT(47.24105895003206 -121.96715170208924) bank114655 +114656 POINT(48.221232646782944 -123.32583908235611) bank114656 +114657 POINT(47.454043632801856 -122.49799391940347) bank114657 +114658 POINT(47.00819699548232 -122.8840674544823) bank114658 +114659 POINT(47.98561979291547 -121.63768087665704) bank114659 +114660 POINT(47.43984687039042 -123.26097380170002) bank114660 +114661 POINT(46.93826621082857 -123.08344250494366) bank114661 +114662 POINT(47.03188449172905 -122.03060488258149) bank114662 +114663 POINT(46.99376869301328 -122.22101467157607) bank114663 +114664 POINT(46.96222748025965 -122.27979240751051) bank114664 +114665 POINT(47.94506409198164 -122.53159486818299) bank114665 +114666 POINT(48.3727379370223 -123.32515071782572) bank114666 +114667 POINT(48.26556773046194 -122.47222404588585) bank114667 +114668 POINT(48.446192181208716 -122.88821535607632) bank114668 +114669 POINT(46.98692373539924 -121.54502901513557) bank114669 +114670 POINT(47.20243490535573 -122.71011859144613) bank114670 +114671 POINT(46.97494871998006 -122.70529486940124) bank114671 +114672 POINT(47.810606924417385 -121.4028274767628) bank114672 +114673 POINT(48.434460561805174 -122.71098118318072) bank114673 +114674 POINT(47.866817389763185 -121.68117093929595) bank114674 +114675 POINT(48.284697987189304 -122.8988177697298) bank114675 +114676 POINT(47.645036090168986 -121.77672472330333) bank114676 +114677 POINT(47.54512139957506 -122.07297230602705) bank114677 +114678 POINT(47.94953105879785 -122.05513225295769) bank114678 +114679 POINT(46.80141604185088 -122.30253155060086) bank114679 +114680 POINT(47.09320418833361 -121.62886598160476) bank114680 +114681 POINT(47.52047949096705 -121.42754205266593) bank114681 +114682 POINT(47.65027421092903 -122.79370557596224) bank114682 +114683 POINT(48.01372726528818 -121.69066102707536) bank114683 +114684 POINT(47.9915355117475 -121.64321699270639) bank114684 +114685 POINT(48.09866887343639 -123.30312355801739) bank114685 +114686 POINT(46.733570833166674 -122.73041134246021) bank114686 +114687 POINT(47.9744465164167 -121.82912305080578) bank114687 +114688 POINT(47.142607143114795 -121.37858644284367) bank114688 +114689 POINT(48.47320109272363 -122.14914674960184) bank114689 +114690 POINT(48.204690515996184 -123.3116000212735) bank114690 +114691 POINT(46.65443664100007 -123.02735535327733) bank114691 +114692 POINT(46.78188798745523 -122.25881608493829) bank114692 +114693 POINT(47.04323673395985 -122.40803540562192) bank114693 +114694 POINT(46.744145180041585 -121.75200637477201) bank114694 +114695 POINT(47.10251996328568 -122.95501280902948) bank114695 +114696 POINT(47.62511136256025 -121.45677936116815) bank114696 +114697 POINT(46.72590017778343 -122.27383808143182) bank114697 +114698 POINT(48.51888832370977 -123.32864091224931) bank114698 +114699 POINT(47.93030925041354 -121.9530174004408) bank114699 +114700 POINT(46.83693344634209 -123.10937092973441) bank114700 +114701 POINT(47.07916405874403 -122.8447946382203) bank114701 +114702 POINT(47.01346837595239 -122.20709441964839) bank114702 +114703 POINT(47.313672862438914 -122.871189999147) bank114703 +114704 POINT(47.33711958542647 -121.33877826311013) bank114704 +114705 POINT(47.73032923780162 -122.49636010287858) bank114705 +114706 POINT(48.33311825410127 -123.11267477673931) bank114706 +114707 POINT(46.79166433318197 -123.08699193820854) bank114707 +114708 POINT(47.01937201569196 -122.87468317629629) bank114708 +114709 POINT(48.210484675359886 -123.00299437314635) bank114709 +114710 POINT(47.17111861182229 -122.55272597609957) bank114710 +114711 POINT(47.05322665291076 -123.13807023388925) bank114711 +114712 POINT(47.30772692758136 -121.91381969246167) bank114712 +114713 POINT(48.13980012903937 -122.31334451740918) bank114713 +114714 POINT(47.167649324846074 -121.70310669907875) bank114714 +114715 POINT(46.83127609801284 -121.77263019412469) bank114715 +114716 POINT(47.14533459048204 -122.41031292617876) bank114716 +114717 POINT(48.23953304972765 -122.80374190523315) bank114717 +114718 POINT(47.99754406809664 -121.57808205536129) bank114718 +114719 POINT(48.2405300268536 -122.08189170953399) bank114719 +114720 POINT(48.560279828832286 -123.21516415565051) bank114720 +114721 POINT(47.22250784385957 -122.98582477471454) bank114721 +114722 POINT(47.47466500322074 -122.36577472745698) bank114722 +114723 POINT(47.00059000971956 -123.0102133679734) bank114723 +114724 POINT(48.077849653851835 -122.73172097621752) bank114724 +114725 POINT(47.371300846859604 -121.38607887853787) bank114725 +114726 POINT(47.887414108600126 -122.36979819627165) bank114726 +114727 POINT(46.94490994226937 -122.5496081914354) bank114727 +114728 POINT(48.389890457621014 -122.14943013591967) bank114728 +114729 POINT(47.16739243301895 -122.70078722747097) bank114729 +114730 POINT(47.18172049394708 -121.52799519546521) bank114730 +114731 POINT(48.207022135902 -122.87803469761747) bank114731 +114732 POINT(48.55223468940785 -122.20806862967574) bank114732 +114733 POINT(47.895901382663794 -122.6036870962371) bank114733 +114734 POINT(46.79155205306376 -123.05911796579316) bank114734 +114735 POINT(47.753308174444285 -121.78578985157294) bank114735 +114736 POINT(46.71089324747382 -122.47696977508734) bank114736 +114737 POINT(47.88158795917323 -121.33641741053712) bank114737 +114738 POINT(47.995688389737026 -123.2243175400656) bank114738 +114739 POINT(46.79568089019086 -123.10820584595345) bank114739 +114740 POINT(47.886776561349734 -121.79853328983461) bank114740 +114741 POINT(46.718952672820464 -121.36966833693137) bank114741 +114742 POINT(48.028854630292464 -122.0780472411638) bank114742 +114743 POINT(48.55764846169507 -122.38929162776887) bank114743 +114744 POINT(47.16636140213624 -122.23622321334332) bank114744 +114745 POINT(47.82708180252432 -122.62926267332199) bank114745 +114746 POINT(47.23773834550358 -122.16230405039369) bank114746 +114747 POINT(47.31478080973837 -122.74257325391106) bank114747 +114748 POINT(46.92996219075013 -123.07020833940418) bank114748 +114749 POINT(47.99555291841995 -122.88789127054831) bank114749 +114750 POINT(47.81178486807535 -121.86203380136841) bank114750 +114751 POINT(48.09584457602681 -122.77079506315776) bank114751 +114752 POINT(47.555902973893 -121.59205148088779) bank114752 +114753 POINT(48.53968455144878 -122.56784266874736) bank114753 +114754 POINT(48.26614185795732 -121.58590330130721) bank114754 +114755 POINT(47.31173533309569 -122.58365045375655) bank114755 +114756 POINT(48.59644600488316 -122.41047363642592) bank114756 +114757 POINT(47.66958506806282 -122.56139517237924) bank114757 +114758 POINT(47.40553757050369 -123.28305222750382) bank114758 +114759 POINT(46.785856178826485 -123.21262503053359) bank114759 +114760 POINT(47.8066425950263 -121.83572993634186) bank114760 +114761 POINT(46.61794542026341 -121.3476314052815) bank114761 +114762 POINT(46.79081569485964 -121.80396335381909) bank114762 +114763 POINT(47.818007948337765 -122.03671561041102) bank114763 +114764 POINT(48.28162144663901 -121.46547271986913) bank114764 +114765 POINT(47.55422431209995 -122.0763191597928) bank114765 +114766 POINT(47.774326417640914 -121.34331781500855) bank114766 +114767 POINT(46.625161800175796 -121.84207407167574) bank114767 +114768 POINT(47.324749852530715 -122.47372301480404) bank114768 +114769 POINT(47.30608685787632 -122.26244330395727) bank114769 +114770 POINT(48.2463790608547 -122.60225066356828) bank114770 +114771 POINT(46.66745902147291 -121.57699578796037) bank114771 +114772 POINT(46.61342771421292 -121.51546186970519) bank114772 +114773 POINT(47.266689874327795 -121.97036637930462) bank114773 +114774 POINT(48.24506187958648 -121.34210784420785) bank114774 +114775 POINT(47.46202796458538 -123.18845113995552) bank114775 +114776 POINT(48.222783951596085 -122.38146440121832) bank114776 +114777 POINT(46.655655734582616 -121.57788202306328) bank114777 +114778 POINT(48.25005794143725 -121.75462481514786) bank114778 +114779 POINT(48.469448702934656 -123.15311532650212) bank114779 +114780 POINT(47.07425593876039 -121.87112907741901) bank114780 +114781 POINT(47.136449199917266 -122.6134137481021) bank114781 +114782 POINT(48.12972507366209 -122.67796314367831) bank114782 +114783 POINT(47.81318014439276 -121.6961128360373) bank114783 +114784 POINT(47.22162860939415 -122.53106095079269) bank114784 +114785 POINT(48.0378967527189 -122.26330774690061) bank114785 +114786 POINT(48.01201689572565 -122.77516870389054) bank114786 +114787 POINT(47.904654591687226 -122.05236033780105) bank114787 +114788 POINT(47.576322122186326 -121.54912203402907) bank114788 +114789 POINT(48.41262582540306 -122.32702356536578) bank114789 +114790 POINT(48.128437359567684 -121.95016324328871) bank114790 +114791 POINT(47.92104118927648 -123.28412832283847) bank114791 +114792 POINT(47.76522052303516 -122.40720707676803) bank114792 +114793 POINT(47.2425723261868 -121.84040102842262) bank114793 +114794 POINT(47.714297236467516 -122.63227247542716) bank114794 +114795 POINT(47.07685462467755 -122.25548079623188) bank114795 +114796 POINT(47.159165700619326 -122.35357178719329) bank114796 +114797 POINT(47.01783430288264 -122.41248460837801) bank114797 +114798 POINT(48.47624863762547 -123.20060133514895) bank114798 +114799 POINT(47.2635429018989 -122.33099037681433) bank114799 +114800 POINT(47.2818535121464 -122.7024875847471) bank114800 +114801 POINT(46.99777687883799 -123.13211001424287) bank114801 +114802 POINT(47.911004140591174 -122.38033457920982) bank114802 +114803 POINT(47.91660035581027 -121.44120665748346) bank114803 +114804 POINT(47.78616900585127 -123.2175608588006) bank114804 +114805 POINT(48.331109336206175 -121.47710109639273) bank114805 +114806 POINT(48.368938483485366 -122.84819971280227) bank114806 +114807 POINT(48.584183778746564 -121.90103684288316) bank114807 +114808 POINT(48.3355750254959 -122.03985324542039) bank114808 +114809 POINT(48.47483168512182 -122.38528722940582) bank114809 +114810 POINT(47.20185205888968 -122.70762709086218) bank114810 +114811 POINT(48.27257963033337 -123.09105713040988) bank114811 +114812 POINT(46.94018424546743 -122.25371951012106) bank114812 +114813 POINT(48.07795591814408 -122.56758262117312) bank114813 +114814 POINT(46.65227278894595 -121.49206946596293) bank114814 +114815 POINT(47.73100718675694 -122.99007167952195) bank114815 +114816 POINT(48.30103682225724 -122.4680711961923) bank114816 +114817 POINT(46.9950054591147 -122.12351190501762) bank114817 +114818 POINT(48.41109060317063 -122.68602818322317) bank114818 +114819 POINT(47.104892306354564 -123.23417334876183) bank114819 +114820 POINT(47.00382978412136 -121.7000924926808) bank114820 +114821 POINT(46.90148325281766 -122.5297717263749) bank114821 +114822 POINT(47.980883847393336 -121.47201516269925) bank114822 +114823 POINT(47.79638021327402 -121.5117433898131) bank114823 +114824 POINT(47.81108664934883 -122.87057589381273) bank114824 +114825 POINT(47.5112742536947 -122.35851673826978) bank114825 +114826 POINT(47.533538334470634 -122.3554614778812) bank114826 +114827 POINT(47.196281339008486 -122.34989851841134) bank114827 +114828 POINT(47.45074622541825 -121.66356018278552) bank114828 +114829 POINT(47.85765613511821 -123.15044439804956) bank114829 +114830 POINT(48.36628457522391 -122.78033176559414) bank114830 +114831 POINT(48.11705126773518 -123.22652348463333) bank114831 +114832 POINT(47.950568451388094 -121.58627147191632) bank114832 +114833 POINT(46.790025924693296 -123.3249012424382) bank114833 +114834 POINT(47.89082659234713 -121.77360707496197) bank114834 +114835 POINT(47.53830787598493 -121.65331264891151) bank114835 +114836 POINT(47.0009053931497 -122.70243914965708) bank114836 +114837 POINT(48.24344822889948 -123.0210447164365) bank114837 +114838 POINT(47.542353610172604 -122.37604692049283) bank114838 +114839 POINT(47.24794964418054 -121.50406206068867) bank114839 +114840 POINT(46.76055321895535 -122.80097656151891) bank114840 +114841 POINT(46.915141877975415 -121.51055751907468) bank114841 +114842 POINT(46.7039863814202 -122.5082584257575) bank114842 +114843 POINT(46.801786181439084 -121.90722695826531) bank114843 +114844 POINT(47.01392049731496 -123.32481275791362) bank114844 +114845 POINT(48.257142878258314 -122.25512614762323) bank114845 +114846 POINT(47.986938106931554 -122.4118248591918) bank114846 +114847 POINT(46.78518492689629 -123.18972352770844) bank114847 +114848 POINT(47.592126214636565 -122.15735719380204) bank114848 +114849 POINT(48.137719598239336 -122.09098761299316) bank114849 +114850 POINT(47.96657596046475 -122.29159495977963) bank114850 +114851 POINT(46.9360421330853 -122.67731140138588) bank114851 +114852 POINT(46.8168739684003 -122.64127468705587) bank114852 +114853 POINT(46.70281647839303 -122.38448604585786) bank114853 +114854 POINT(47.53302368066602 -121.48754350288594) bank114854 +114855 POINT(48.06076274213039 -122.9669067589501) bank114855 +114856 POINT(46.804524056973456 -121.621661363555) bank114856 +114857 POINT(47.876981053224945 -122.38833724707646) bank114857 +114858 POINT(47.42102367285558 -121.42342354093681) bank114858 +114859 POINT(47.04465633134096 -122.8102398205302) bank114859 +114860 POINT(46.758740383706055 -122.45702813317129) bank114860 +114861 POINT(48.431516683335836 -121.79254578653814) bank114861 +114862 POINT(46.607552317003716 -121.64547578657395) bank114862 +114863 POINT(48.161949722014604 -123.29260558199074) bank114863 +114864 POINT(47.953260747850166 -122.38817387299021) bank114864 +114865 POINT(48.129433332760726 -122.65668867379188) bank114865 +114866 POINT(47.98758189694131 -121.73147256294656) bank114866 +114867 POINT(48.50399087528767 -121.48408912254025) bank114867 +114868 POINT(48.41878385814122 -123.03453561659113) bank114868 +114869 POINT(47.734229909735724 -123.05419831736997) bank114869 +114870 POINT(48.12950269186905 -122.75639196403372) bank114870 +114871 POINT(47.145855203779156 -121.38301436547289) bank114871 +114872 POINT(47.38482452175247 -122.67673134241899) bank114872 +114873 POINT(47.8715097025234 -121.77354241560293) bank114873 +114874 POINT(48.1560772403343 -122.26389320930988) bank114874 +114875 POINT(46.914289236793266 -122.38541714084427) bank114875 +114876 POINT(47.037312664013946 -122.12356835723841) bank114876 +114877 POINT(48.12017586395383 -121.79810479713937) bank114877 +114878 POINT(46.712429927786765 -121.6917072085969) bank114878 +114879 POINT(48.09016619650083 -122.84667694197617) bank114879 +114880 POINT(46.82901382572866 -121.50613493814694) bank114880 +114881 POINT(46.74938911356245 -121.73847889007598) bank114881 +114882 POINT(47.37012064628866 -122.05985306356392) bank114882 +114883 POINT(47.333774762139086 -122.19256812098207) bank114883 +114884 POINT(47.99060269777061 -121.47212386491977) bank114884 +114885 POINT(48.59698126144138 -122.64929706955603) bank114885 +114886 POINT(46.86350822705549 -121.33951192537984) bank114886 +114887 POINT(48.330502282809604 -121.6392847998782) bank114887 +114888 POINT(48.31721918187857 -121.6124776983842) bank114888 +114889 POINT(47.27913471648314 -122.6894100780847) bank114889 +114890 POINT(47.795410191395156 -123.13943926067216) bank114890 +114891 POINT(48.20477827077029 -122.85402538012339) bank114891 +114892 POINT(48.04310030625824 -121.92153525111549) bank114892 +114893 POINT(48.60160192879803 -122.00051340607642) bank114893 +114894 POINT(48.44695385211143 -122.64018327068058) bank114894 +114895 POINT(47.939466028853325 -121.86522863792328) bank114895 +114896 POINT(47.84403844613471 -122.80419363785225) bank114896 +114897 POINT(48.258527251507346 -121.97346024983416) bank114897 +114898 POINT(48.585565427053915 -122.90312155169443) bank114898 +114899 POINT(46.888227199348485 -121.35815579230365) bank114899 +114900 POINT(48.44241746498815 -121.68643866148872) bank114900 +114901 POINT(46.80403699978399 -121.36659569384393) bank114901 +114902 POINT(46.63504301192821 -122.97784990362658) bank114902 +114903 POINT(47.12088549699968 -121.53492041829735) bank114903 +114904 POINT(48.26264117880277 -122.64721774288053) bank114904 +114905 POINT(46.85583207817219 -122.31207216361915) bank114905 +114906 POINT(46.97233150338742 -122.54467016729309) bank114906 +114907 POINT(47.17697211911517 -123.30058663625675) bank114907 +114908 POINT(47.76185091771317 -122.23448474117238) bank114908 +114909 POINT(46.64985619102279 -121.52201949127038) bank114909 +114910 POINT(46.96619612150904 -122.95151331120921) bank114910 +114911 POINT(46.70167977327882 -122.96785013362519) bank114911 +114912 POINT(47.97294572215609 -123.17971666329518) bank114912 +114913 POINT(47.61096889015073 -121.61640478206824) bank114913 +114914 POINT(47.24799916716122 -121.85007511746309) bank114914 +114915 POINT(48.50506592078336 -123.26943076353452) bank114915 +114916 POINT(47.67798982212082 -121.59145791071757) bank114916 +114917 POINT(48.283795267842464 -122.95165798547117) bank114917 +114918 POINT(47.702439272394166 -122.07937316421729) bank114918 +114919 POINT(47.923398842932144 -121.46739290316775) bank114919 +114920 POINT(46.720257116287655 -122.46500294528155) bank114920 +114921 POINT(46.73053890104138 -122.63947305145778) bank114921 +114922 POINT(47.00723497645857 -122.3231907667618) bank114922 +114923 POINT(48.12889527756128 -122.36280823205254) bank114923 +114924 POINT(46.87140500392567 -121.54318084321996) bank114924 +114925 POINT(46.82586473149888 -123.25908222887055) bank114925 +114926 POINT(48.53320487379619 -121.60837614478908) bank114926 +114927 POINT(47.79887125428724 -121.98536635593102) bank114927 +114928 POINT(48.32528344865981 -122.75125448604138) bank114928 +114929 POINT(48.02834164567135 -122.05639890670497) bank114929 +114930 POINT(48.11248591348248 -122.44310823654796) bank114930 +114931 POINT(48.17702125559763 -122.85574416417857) bank114931 +114932 POINT(46.937886142493376 -122.65450500721684) bank114932 +114933 POINT(47.63171705322166 -122.07372999569361) bank114933 +114934 POINT(46.620544854425106 -122.21547224626983) bank114934 +114935 POINT(48.15071851045414 -121.41247895366885) bank114935 +114936 POINT(48.186811712854734 -121.40601153760758) bank114936 +114937 POINT(47.60736902284831 -121.81775023976228) bank114937 +114938 POINT(47.068494036222354 -122.04395349136811) bank114938 +114939 POINT(47.09914210229582 -122.99017101667538) bank114939 +114940 POINT(47.25885121104819 -121.93711968092747) bank114940 +114941 POINT(47.59516008507085 -121.35317518070013) bank114941 +114942 POINT(48.45599745631276 -122.60715044337383) bank114942 +114943 POINT(48.223789941606945 -123.31301155457699) bank114943 +114944 POINT(48.05152203497317 -121.53770106218327) bank114944 +114945 POINT(47.08963766445926 -123.30719817368562) bank114945 +114946 POINT(47.86342164968244 -122.99717273228576) bank114946 +114947 POINT(47.746193115902145 -122.9725438311777) bank114947 +114948 POINT(46.70022729168405 -121.39504970643502) bank114948 +114949 POINT(46.78767499324339 -123.03183092423843) bank114949 +114950 POINT(47.95209293297371 -121.98005568705473) bank114950 +114951 POINT(47.53533545575428 -122.65770229089998) bank114951 +114952 POINT(47.371716030226686 -121.8338637961798) bank114952 +114953 POINT(48.316491391773056 -121.50716385669523) bank114953 +114954 POINT(46.60793202474469 -121.6571028879086) bank114954 +114955 POINT(46.99999125597454 -121.5001513700162) bank114955 +114956 POINT(46.9092814985282 -122.2036909020635) bank114956 +114957 POINT(46.9987734916616 -121.42466591842408) bank114957 +114958 POINT(48.01335199500225 -121.99061502925424) bank114958 +114959 POINT(47.97369261980007 -121.69652513604663) bank114959 +114960 POINT(48.00767971145658 -121.7392476959061) bank114960 +114961 POINT(48.00831134429026 -121.42840760984456) bank114961 +114962 POINT(46.995121536934384 -122.72394477016724) bank114962 +114963 POINT(47.385000700662204 -122.94720375143935) bank114963 +114964 POINT(47.993224467791784 -122.62342847757466) bank114964 +114965 POINT(46.72845623905542 -123.30876046936122) bank114965 +114966 POINT(47.2772436516752 -122.35883533929749) bank114966 +114967 POINT(47.99720414873578 -122.43058810669194) bank114967 +114968 POINT(48.19265544967974 -122.80732338096962) bank114968 +114969 POINT(46.82794310538134 -123.01409837752037) bank114969 +114970 POINT(47.72115136689925 -123.20137325527493) bank114970 +114971 POINT(46.60882063639742 -123.21008969316402) bank114971 +114972 POINT(46.63161663042646 -123.11812332654314) bank114972 +114973 POINT(48.045649339193204 -122.30197370983586) bank114973 +114974 POINT(47.452920927264834 -121.5486494820748) bank114974 +114975 POINT(46.77992688060147 -121.33751077913888) bank114975 +114976 POINT(47.09441856084075 -122.798132733964) bank114976 +114977 POINT(47.664600147446116 -122.9110220363234) bank114977 +114978 POINT(48.41404113208254 -123.1188308950748) bank114978 +114979 POINT(48.03145080496224 -121.98249283196958) bank114979 +114980 POINT(46.833486899932645 -121.89234488313777) bank114980 +114981 POINT(46.85571208628457 -123.18991649636503) bank114981 +114982 POINT(47.38832091035023 -122.54808342799203) bank114982 +114983 POINT(48.16167159904852 -122.94869597112503) bank114983 +114984 POINT(47.10905579420099 -122.53109247104734) bank114984 +114985 POINT(48.54183978780776 -122.57152184809945) bank114985 +114986 POINT(48.0573137870818 -122.19934111559769) bank114986 +114987 POINT(47.59401901008578 -122.46318968941546) bank114987 +114988 POINT(47.835261946052604 -123.20444169750868) bank114988 +114989 POINT(47.66127667838742 -122.18791937780173) bank114989 +114990 POINT(47.30673011653243 -122.78998931011651) bank114990 +114991 POINT(47.063634640529315 -122.82660866276744) bank114991 +114992 POINT(48.279748607869564 -123.24572836600603) bank114992 +114993 POINT(47.80701680438835 -123.2578293347353) bank114993 +114994 POINT(48.42808049611066 -122.06728074049299) bank114994 +114995 POINT(48.069828918089854 -123.0435229014813) bank114995 +114996 POINT(47.93340136791788 -122.4718482592008) bank114996 +114997 POINT(48.11673789384152 -122.66632752828325) bank114997 +114998 POINT(48.510838888537506 -123.32365588732762) bank114998 +114999 POINT(48.60298255693426 -122.74268867151882) bank114999 +115000 POINT(48.289156487062186 -122.73903075174151) bank115000 +115001 POINT(47.408477040581325 -122.74384177049592) bank115001 +115002 POINT(48.17322180469533 -122.03993446519513) bank115002 +115003 POINT(48.05004496085461 -122.84914932168206) bank115003 +115004 POINT(48.45411340992491 -121.66363394431527) bank115004 +115005 POINT(47.32798324803029 -122.87945631885191) bank115005 +115006 POINT(47.92323529949527 -122.82603520453513) bank115006 +115007 POINT(48.131275872873 -122.32985276680155) bank115007 +115008 POINT(47.53852919866045 -123.28077485844572) bank115008 +115009 POINT(46.95983281848638 -123.1918966439728) bank115009 +115010 POINT(48.106940483768724 -123.05662242035757) bank115010 +115011 POINT(47.972197811389606 -122.21893423319447) bank115011 +115012 POINT(48.37522761836212 -121.49635794049675) bank115012 +115013 POINT(47.31476811178473 -121.59603356051359) bank115013 +115014 POINT(47.600078962254855 -123.32968404817203) bank115014 +115015 POINT(46.68179311996581 -123.27079740072395) bank115015 +115016 POINT(47.79501102115451 -121.57411912237784) bank115016 +115017 POINT(48.59560323536821 -121.81061146204686) bank115017 +115018 POINT(47.626749476186426 -122.65097358988979) bank115018 +115019 POINT(47.41903922803821 -121.83561832044775) bank115019 +115020 POINT(48.41771784824052 -123.00375816723404) bank115020 +115021 POINT(47.90674035297576 -122.8724454015714) bank115021 +115022 POINT(47.8117290816556 -122.10570014226226) bank115022 +115023 POINT(46.789501839588794 -121.46798499022229) bank115023 +115024 POINT(46.8683024727183 -122.36751051055907) bank115024 +115025 POINT(46.61825598693013 -121.90984465062276) bank115025 +115026 POINT(48.554289785076094 -123.32543559552386) bank115026 +115027 POINT(47.085295024677265 -123.33182057037355) bank115027 +115028 POINT(48.21767699575994 -122.26082618004808) bank115028 +115029 POINT(47.00251696637355 -123.05809957168594) bank115029 +115030 POINT(47.983419061821344 -121.51651429039788) bank115030 +115031 POINT(48.57756284614618 -121.81754727033318) bank115031 +115032 POINT(46.83415926263942 -122.64535413796185) bank115032 +115033 POINT(47.322843420011154 -122.552114133532) bank115033 +115034 POINT(48.16362236463041 -122.22210161195635) bank115034 +115035 POINT(47.673351588654796 -123.30279254047142) bank115035 +115036 POINT(47.90726049696841 -121.52386227603547) bank115036 +115037 POINT(47.315642622020775 -121.78203205150442) bank115037 +115038 POINT(47.22106967570037 -121.9821280054113) bank115038 +115039 POINT(47.86866337646089 -121.56401176454226) bank115039 +115040 POINT(47.09384557543633 -122.2905058668798) bank115040 +115041 POINT(47.0906854268767 -121.93691705274487) bank115041 +115042 POINT(48.067166856403645 -122.89816549992648) bank115042 +115043 POINT(48.47856202307267 -122.3978809537709) bank115043 +115044 POINT(47.06891257115999 -121.43897773692218) bank115044 +115045 POINT(48.24688553297364 -122.9786807032473) bank115045 +115046 POINT(47.832388928511115 -121.61277818775108) bank115046 +115047 POINT(47.115926093701276 -122.30380788076843) bank115047 +115048 POINT(47.56284830170105 -122.71987146065759) bank115048 +115049 POINT(46.813047704321036 -122.8925558195929) bank115049 +115050 POINT(48.321841395610015 -122.47901628493558) bank115050 +115051 POINT(47.28080532823102 -123.08512670266803) bank115051 +115052 POINT(48.113296391880986 -121.79268129792399) bank115052 +115053 POINT(47.95165342799133 -121.41059786726322) bank115053 +115054 POINT(48.44615366818707 -122.5571828640752) bank115054 +115055 POINT(47.8631755704535 -123.19083052509282) bank115055 +115056 POINT(48.361826898575245 -122.0146540610626) bank115056 +115057 POINT(47.68369749767939 -122.69156040751139) bank115057 +115058 POINT(47.51957060168119 -122.15152701551737) bank115058 +115059 POINT(46.69478814586354 -123.11990319694598) bank115059 +115060 POINT(48.580990185766744 -122.22832882595942) bank115060 +115061 POINT(47.07857590422535 -122.23987413546499) bank115061 +115062 POINT(46.835363352121355 -122.8211703181073) bank115062 +115063 POINT(48.44006165285878 -121.46396599233591) bank115063 +115064 POINT(47.5335716129584 -121.84655473820096) bank115064 +115065 POINT(47.812089872347975 -121.53160889906742) bank115065 +115066 POINT(47.91168077267857 -121.97739688500882) bank115066 +115067 POINT(47.5793970188741 -122.1079409429722) bank115067 +115068 POINT(47.49076151702023 -123.25727646310744) bank115068 +115069 POINT(48.586076821832776 -122.54445961549041) bank115069 +115070 POINT(47.99810372470763 -121.99082027101066) bank115070 +115071 POINT(48.573636178768986 -123.05780046403711) bank115071 +115072 POINT(47.19649311186528 -122.4093557064607) bank115072 +115073 POINT(48.1570217741991 -122.26403210591108) bank115073 +115074 POINT(47.56463573018683 -122.93187884417718) bank115074 +115075 POINT(48.027974565306785 -121.71030971296402) bank115075 +115076 POINT(47.18957030948567 -123.283452226387) bank115076 +115077 POINT(46.848494868605215 -122.13520651183235) bank115077 +115078 POINT(47.015257491953456 -122.29139089566456) bank115078 +115079 POINT(48.2057998380385 -122.28677582019282) bank115079 +115080 POINT(46.77898843413961 -122.03413246540437) bank115080 +115081 POINT(48.482189779605626 -122.1715355536533) bank115081 +115082 POINT(48.06023700104227 -122.1080417718632) bank115082 +115083 POINT(48.02104033039688 -123.09994049350057) bank115083 +115084 POINT(47.59433782216851 -121.76068004220016) bank115084 +115085 POINT(47.7109347393197 -122.76903949156059) bank115085 +115086 POINT(48.230739699970435 -121.6110305320265) bank115086 +115087 POINT(48.56266434612129 -121.69836797932639) bank115087 +115088 POINT(47.34848421022649 -121.77002877337027) bank115088 +115089 POINT(47.398413730909105 -121.35686575120353) bank115089 +115090 POINT(48.500602286505725 -121.43903946265922) bank115090 +115091 POINT(48.0980567230686 -122.37775661407652) bank115091 +115092 POINT(46.8807876556872 -121.84798921943884) bank115092 +115093 POINT(48.511543518151846 -123.1292534792458) bank115093 +115094 POINT(48.0946512046383 -121.94881810057582) bank115094 +115095 POINT(48.60523070353984 -122.10643177159469) bank115095 +115096 POINT(48.127898321391214 -122.38594165362747) bank115096 +115097 POINT(47.116916564783004 -122.60990448736318) bank115097 +115098 POINT(47.70963083484849 -122.61216712912668) bank115098 +115099 POINT(47.28690765616833 -122.06090580063177) bank115099 +115100 POINT(46.86115695917854 -121.51363975395913) bank115100 +115101 POINT(48.074690619513774 -123.21241798022055) bank115101 +115102 POINT(48.13458524677992 -123.08887231740769) bank115102 +115103 POINT(47.158914345636646 -123.09278737173392) bank115103 +115104 POINT(47.045760182681654 -122.65668997376345) bank115104 +115105 POINT(47.83238535895524 -122.1997671198915) bank115105 +115106 POINT(48.03652643708988 -122.91453788785216) bank115106 +115107 POINT(46.829096006907854 -121.48155968831165) bank115107 +115108 POINT(48.3998815982588 -122.13482256148711) bank115108 +115109 POINT(47.107654888949845 -121.56244327221988) bank115109 +115110 POINT(47.54077295572343 -122.14773141323971) bank115110 +115111 POINT(46.72752421974246 -123.12647514744955) bank115111 +115112 POINT(47.05949792078437 -121.41643154755218) bank115112 +115113 POINT(47.38285227931928 -122.01761016546301) bank115113 +115114 POINT(48.40236691619124 -121.51127936268477) bank115114 +115115 POINT(47.566968708889995 -122.07875967180809) bank115115 +115116 POINT(47.95851586989002 -122.93483256226745) bank115116 +115117 POINT(47.32407885484352 -122.096826605149) bank115117 +115118 POINT(48.505981561126035 -121.67482357069619) bank115118 +115119 POINT(48.27573517294167 -123.16996126402061) bank115119 +115120 POINT(47.36264477181717 -122.15522906886068) bank115120 +115121 POINT(47.99778514014143 -122.67325258412032) bank115121 +115122 POINT(47.06525690399733 -122.83695374269188) bank115122 +115123 POINT(47.44635282374075 -123.0070488330543) bank115123 +115124 POINT(47.66019241539964 -121.64891025109779) bank115124 +115125 POINT(48.26394664794636 -122.42744553858775) bank115125 +115126 POINT(47.85689465938595 -122.84782678756304) bank115126 +115127 POINT(47.29356730550685 -122.17558875117386) bank115127 +115128 POINT(46.91832512134226 -122.24272186655668) bank115128 +115129 POINT(46.91832654398394 -123.17174789038287) bank115129 +115130 POINT(47.39855673209848 -122.65726571333435) bank115130 +115131 POINT(48.47006970428175 -122.69853888172264) bank115131 +115132 POINT(47.64085175068324 -122.98099122728236) bank115132 +115133 POINT(48.2869575819012 -122.35870363847658) bank115133 +115134 POINT(47.29518279381956 -121.94748054098513) bank115134 +115135 POINT(47.8991459804502 -123.29016676088686) bank115135 +115136 POINT(46.944483241453675 -121.68657587187215) bank115136 +115137 POINT(47.503284348542884 -121.71273271214672) bank115137 +115138 POINT(47.92468522758192 -123.21818133115578) bank115138 +115139 POINT(47.74666050469474 -123.32255092239826) bank115139 +115140 POINT(46.88329411413853 -122.14498479958702) bank115140 +115141 POINT(46.95831137492212 -122.61257620511843) bank115141 +115142 POINT(47.59598329720049 -122.16512925087586) bank115142 +115143 POINT(48.1972718096909 -121.91133357523353) bank115143 +115144 POINT(48.549051683079455 -122.52537425700791) bank115144 +115145 POINT(47.73788077074567 -122.953767246789) bank115145 +115146 POINT(48.15055245700353 -122.24705507506556) bank115146 +115147 POINT(48.52875427105147 -122.71504861656335) bank115147 +115148 POINT(47.87399943451396 -121.93920509632412) bank115148 +115149 POINT(48.479715720046556 -122.91134691905023) bank115149 +115150 POINT(46.825904620894164 -123.15082570507063) bank115150 +115151 POINT(48.07238295968326 -122.02505954394942) bank115151 +115152 POINT(47.819467889268026 -122.8781790049557) bank115152 +115153 POINT(47.60294262431581 -122.61397006829121) bank115153 +115154 POINT(47.3954449674019 -123.2115107399644) bank115154 +115155 POINT(48.39331273084401 -122.71707743275556) bank115155 +115156 POINT(47.13104486006506 -122.99298468483101) bank115156 +115157 POINT(47.61877490830287 -123.15680272984757) bank115157 +115158 POINT(46.65909568519877 -122.3130862567268) bank115158 +115159 POINT(47.62521995642093 -121.53547794530988) bank115159 +115160 POINT(47.20741540272938 -122.93292644495818) bank115160 +115161 POINT(48.1247206430949 -122.20459138752139) bank115161 +115162 POINT(47.427436360352374 -122.33765653123896) bank115162 +115163 POINT(46.791439814653685 -123.29748301242068) bank115163 +115164 POINT(46.73648441701431 -121.55477514797514) bank115164 +115165 POINT(48.22743675473336 -123.28264678554429) bank115165 +115166 POINT(47.63041549686797 -122.05115447621678) bank115166 +115167 POINT(48.46234771207348 -121.55895267864297) bank115167 +115168 POINT(48.32443833424816 -122.07530537700525) bank115168 +115169 POINT(47.95366503611035 -122.65879032517786) bank115169 +115170 POINT(47.26272796216585 -122.3634159121636) bank115170 +115171 POINT(47.73063836388157 -121.88900805478974) bank115171 +115172 POINT(48.40329007434559 -121.42030937981862) bank115172 +115173 POINT(48.484745243855244 -121.39345926231437) bank115173 +115174 POINT(47.70007980844976 -122.43446507365768) bank115174 +115175 POINT(47.699144733097675 -121.84853143653685) bank115175 +115176 POINT(47.73495846856475 -123.03294354696901) bank115176 +115177 POINT(47.445565130820064 -121.95825394184378) bank115177 +115178 POINT(47.31628928708958 -122.28843838799783) bank115178 +115179 POINT(47.591258674417126 -122.78218263262171) bank115179 +115180 POINT(48.09040014185694 -121.38720684241805) bank115180 +115181 POINT(47.19437595966472 -123.15763905307116) bank115181 +115182 POINT(47.95047796158075 -122.64339825612511) bank115182 +115183 POINT(48.489360512132976 -122.08681344049111) bank115183 +115184 POINT(47.382327414220214 -122.2343955730712) bank115184 +115185 POINT(47.88412842495994 -122.38556022864562) bank115185 +115186 POINT(47.90873127431337 -122.37986108810881) bank115186 +115187 POINT(47.41892529191979 -122.95815577332651) bank115187 +115188 POINT(47.48034246098795 -122.76935950225149) bank115188 +115189 POINT(47.731557320094474 -122.97150051379045) bank115189 +115190 POINT(48.42769162073264 -121.3511193394846) bank115190 +115191 POINT(46.95496648010956 -122.96556372096371) bank115191 +115192 POINT(47.30406876309788 -123.11846641791048) bank115192 +115193 POINT(47.13297724286528 -123.09225056983485) bank115193 +115194 POINT(47.56442945993712 -123.10963956348118) bank115194 +115195 POINT(48.22977898241291 -122.6779401462074) bank115195 +115196 POINT(46.84519803615241 -122.33412310088262) bank115196 +115197 POINT(48.16820789100897 -122.2313957428517) bank115197 +115198 POINT(48.222047319699115 -121.96751874851654) bank115198 +115199 POINT(47.345308825156216 -121.96421057267558) bank115199 +115200 POINT(48.32737370729565 -122.12788062260294) bank115200 +115201 POINT(48.028830395076966 -121.84144278598144) bank115201 +115202 POINT(47.652616883385505 -121.94701608754431) bank115202 +115203 POINT(48.076142561438104 -122.4698793533494) bank115203 +115204 POINT(47.451947618691634 -122.77768397697128) bank115204 +115205 POINT(47.43606360602844 -123.17892083868075) bank115205 +115206 POINT(47.52950942123124 -121.53764332768526) bank115206 +115207 POINT(47.47543477365913 -123.32262470748834) bank115207 +115208 POINT(48.33826052441572 -121.75374058810391) bank115208 +115209 POINT(48.442774355273734 -121.58128691291066) bank115209 +115210 POINT(47.11673717598885 -123.02222689205693) bank115210 +115211 POINT(47.87756532855026 -122.89874140509622) bank115211 +115212 POINT(47.28644014129319 -122.54353339870477) bank115212 +115213 POINT(47.04819859199122 -122.18647674239108) bank115213 +115214 POINT(47.49821743418866 -123.32981380820554) bank115214 +115215 POINT(46.73674259128153 -122.37054334215608) bank115215 +115216 POINT(48.245722896025335 -121.46679379421586) bank115216 +115217 POINT(46.65574828879158 -121.67757528676431) bank115217 +115218 POINT(48.21155445451841 -122.01700951571189) bank115218 +115219 POINT(48.10549036542715 -122.16070020930084) bank115219 +115220 POINT(47.15370139773478 -122.84832018562626) bank115220 +115221 POINT(48.20120844245058 -121.74741267055754) bank115221 +115222 POINT(47.777340142949996 -122.58426525967923) bank115222 +115223 POINT(47.58317481175257 -121.49103546932878) bank115223 +115224 POINT(47.96966395919123 -122.02103577868583) bank115224 +115225 POINT(46.9701473202421 -122.12853235201352) bank115225 +115226 POINT(48.02623536333214 -121.45902333514191) bank115226 +115227 POINT(48.29398536208891 -122.65487249477358) bank115227 +115228 POINT(47.14345750856475 -121.50342536066667) bank115228 +115229 POINT(46.677550913108206 -122.63016078704486) bank115229 +115230 POINT(46.94792492713683 -122.58806606754177) bank115230 +115231 POINT(47.978722526674694 -122.28708338326986) bank115231 +115232 POINT(46.66822773249643 -123.26230969714537) bank115232 +115233 POINT(46.89059584666741 -122.07408759358157) bank115233 +115234 POINT(48.15441921865082 -123.07740120782263) bank115234 +115235 POINT(47.346111010493935 -123.12237070234036) bank115235 +115236 POINT(47.7674249628556 -123.28819914810501) bank115236 +115237 POINT(47.112225153977676 -123.32260834549247) bank115237 +115238 POINT(47.71364147006196 -122.82651645184342) bank115238 +115239 POINT(47.83777264662658 -122.88774532231194) bank115239 +115240 POINT(47.6651530256983 -121.8299805650372) bank115240 +115241 POINT(47.04484410406486 -121.57467250748111) bank115241 +115242 POINT(48.602656410262185 -123.04897564987797) bank115242 +115243 POINT(47.73768847542053 -122.18961538582806) bank115243 +115244 POINT(47.4015999354142 -123.28502051546808) bank115244 +115245 POINT(47.456481334308634 -122.44512098566749) bank115245 +115246 POINT(48.541035077434366 -121.42169982109468) bank115246 +115247 POINT(46.835862192548674 -122.18144609881531) bank115247 +115248 POINT(48.0145174076882 -123.28837073122007) bank115248 +115249 POINT(47.83773344385728 -122.38824306139743) bank115249 +115250 POINT(48.543856434884674 -123.00504693074404) bank115250 +115251 POINT(48.2202657785289 -122.42319766873669) bank115251 +115252 POINT(46.89887889623949 -121.8686198260132) bank115252 +115253 POINT(46.7261849604989 -122.39541456432194) bank115253 +115254 POINT(47.01998789403388 -122.54948476629156) bank115254 +115255 POINT(48.27237409872698 -122.93803415527856) bank115255 +115256 POINT(47.46256351372399 -121.81255685155456) bank115256 +115257 POINT(47.98955648833319 -122.59037000870573) bank115257 +115258 POINT(47.27994889193755 -122.36205160920892) bank115258 +115259 POINT(47.44800769871352 -122.73371919888345) bank115259 +115260 POINT(47.458365128297004 -121.41646129054233) bank115260 +115261 POINT(48.019971352276585 -121.99394099249761) bank115261 +115262 POINT(48.56755020836652 -122.73557702156423) bank115262 +115263 POINT(47.15382534450596 -121.39421951097354) bank115263 +115264 POINT(48.37706841884695 -122.90281787829791) bank115264 +115265 POINT(46.87646549054573 -121.52088181549432) bank115265 +115266 POINT(47.60406697353051 -122.13769173345534) bank115266 +115267 POINT(47.94890659851991 -122.45343186618553) bank115267 +115268 POINT(48.18447650196169 -121.6337460395158) bank115268 +115269 POINT(46.85057404568838 -122.51150088837204) bank115269 +115270 POINT(46.705652548060584 -122.03884902298323) bank115270 +115271 POINT(48.55188350874955 -122.78064059295976) bank115271 +115272 POINT(46.67632194824491 -121.3947709068014) bank115272 +115273 POINT(48.21831970385883 -123.0531544652915) bank115273 +115274 POINT(47.754490892245265 -121.92265281685378) bank115274 +115275 POINT(47.214226109553785 -122.36231900719952) bank115275 +115276 POINT(46.62039308087566 -123.09072932426568) bank115276 +115277 POINT(47.88011958318965 -122.61280580997294) bank115277 +115278 POINT(48.5291002275582 -121.35968825020325) bank115278 +115279 POINT(47.67139280058902 -121.88465613075364) bank115279 +115280 POINT(47.07310455323213 -121.91755735431066) bank115280 +115281 POINT(48.20872217457024 -122.82154311973372) bank115281 +115282 POINT(46.96267496160747 -122.39181661368629) bank115282 +115283 POINT(46.74981787935709 -123.31738629199603) bank115283 +115284 POINT(48.035172115788335 -122.15979222127463) bank115284 +115285 POINT(47.036341218470305 -122.52195251930084) bank115285 +115286 POINT(47.87106443016599 -123.26156120735718) bank115286 +115287 POINT(48.21058584508357 -123.08452342140761) bank115287 +115288 POINT(47.547231091998775 -122.70328310236758) bank115288 +115289 POINT(46.97400574309808 -121.96687583071031) bank115289 +115290 POINT(47.68289686460114 -121.39146173860895) bank115290 +115291 POINT(47.33792600288127 -122.47449191441642) bank115291 +115292 POINT(47.136509527354626 -121.67545300358454) bank115292 +115293 POINT(48.48445309029771 -121.87010540008029) bank115293 +115294 POINT(48.50158438547873 -123.09740071965777) bank115294 +115295 POINT(48.409257596178676 -121.60556365425013) bank115295 +115296 POINT(46.78928680223649 -122.77655607552406) bank115296 +115297 POINT(47.09522344748903 -123.1560003980203) bank115297 +115298 POINT(46.96392578847778 -121.54782538539278) bank115298 +115299 POINT(47.31873048745279 -122.52768403890565) bank115299 +115300 POINT(46.8746932963086 -121.68990711283934) bank115300 +115301 POINT(47.76933102668195 -123.16126220154753) bank115301 +115302 POINT(48.44747735355971 -122.0727577932386) bank115302 +115303 POINT(47.7700727218645 -122.28362265772317) bank115303 +115304 POINT(46.765384097464 -122.19070102720463) bank115304 +115305 POINT(48.58772666556776 -123.2392104840362) bank115305 +115306 POINT(47.00744492262899 -123.04437554844058) bank115306 +115307 POINT(47.080181404709066 -121.80553064868563) bank115307 +115308 POINT(46.64979380431887 -122.74531211410991) bank115308 +115309 POINT(48.43837619933914 -122.60337433746865) bank115309 +115310 POINT(48.256758449714845 -122.02358460049217) bank115310 +115311 POINT(48.090728760397056 -123.2415715874199) bank115311 +115312 POINT(47.28167636157087 -123.24995111887105) bank115312 +115313 POINT(48.071755047703725 -122.8727950652229) bank115313 +115314 POINT(48.56407040806667 -123.08214430035144) bank115314 +115315 POINT(48.4159767243148 -122.20686436403528) bank115315 +115316 POINT(47.34606920696857 -122.34547000462943) bank115316 +115317 POINT(47.316618465395635 -121.75915032767408) bank115317 +115318 POINT(47.94609155174033 -123.10262013823342) bank115318 +115319 POINT(46.84648758149523 -123.19000395223075) bank115319 +115320 POINT(47.59568804292183 -121.8026420252948) bank115320 +115321 POINT(48.53540127468885 -122.97612432956619) bank115321 +115322 POINT(46.835276546247066 -123.11101101069262) bank115322 +115323 POINT(47.342729506544636 -121.98070613583667) bank115323 +115324 POINT(47.92383269174399 -121.67208539925115) bank115324 +115325 POINT(46.78666752964335 -122.55845451132619) bank115325 +115326 POINT(47.421462870115974 -122.8762224053831) bank115326 +115327 POINT(47.47992813511713 -123.13200987407446) bank115327 +115328 POINT(48.41891041798378 -122.41290430934399) bank115328 +115329 POINT(47.761686028564085 -122.49995075997754) bank115329 +115330 POINT(48.57887980692546 -123.04102880419367) bank115330 +115331 POINT(47.06325393366979 -122.64876198530649) bank115331 +115332 POINT(47.16857404954115 -122.25018573583952) bank115332 +115333 POINT(47.08933540074271 -122.6122321756028) bank115333 +115334 POINT(48.076251360024884 -121.70169178407886) bank115334 +115335 POINT(48.07346491400435 -121.67243112481164) bank115335 +115336 POINT(48.05852556010188 -121.53916196925667) bank115336 +115337 POINT(47.45457755720573 -122.78409375559322) bank115337 +115338 POINT(46.787590180062274 -123.16915347177553) bank115338 +115339 POINT(47.59390739074404 -121.9903326322496) bank115339 +115340 POINT(47.81244721901832 -121.36244514444061) bank115340 +115341 POINT(46.73299040884757 -122.35019194691827) bank115341 +115342 POINT(47.18765801372887 -122.55144851574676) bank115342 +115343 POINT(48.256681702962055 -123.123526099709) bank115343 +115344 POINT(47.53234797929287 -122.28539513951907) bank115344 +115345 POINT(47.16453642917694 -122.73404718608444) bank115345 +115346 POINT(47.82413380940484 -121.44217198246078) bank115346 +115347 POINT(46.88196338323763 -121.55207754372218) bank115347 +115348 POINT(47.07196363288358 -122.3092421409955) bank115348 +115349 POINT(48.51951868867609 -123.32215146526345) bank115349 +115350 POINT(47.76638791359471 -121.8264607657821) bank115350 +115351 POINT(46.61830622879275 -121.96982626044773) bank115351 +115352 POINT(47.374435319024485 -123.07777655886139) bank115352 +115353 POINT(48.372846157422124 -123.17431495986656) bank115353 +115354 POINT(47.36469274777669 -122.19806114101443) bank115354 +115355 POINT(47.054269624463 -121.7907961437348) bank115355 +115356 POINT(48.30339325642765 -122.87486830738104) bank115356 +115357 POINT(47.32843633397227 -123.05178592489843) bank115357 +115358 POINT(47.61835324549919 -121.99727301871215) bank115358 +115359 POINT(48.39053903894221 -122.15768561384083) bank115359 +115360 POINT(47.74419319294288 -121.76741480269217) bank115360 +115361 POINT(48.55588790676566 -121.47487000999557) bank115361 +115362 POINT(48.43680334254385 -122.81456504488662) bank115362 +115363 POINT(47.395308270731725 -122.90680207067598) bank115363 +115364 POINT(46.87189166772206 -123.25840206810689) bank115364 +115365 POINT(47.93222572563664 -122.43068750221771) bank115365 +115366 POINT(48.003641425477504 -122.32919492085165) bank115366 +115367 POINT(48.078829713631315 -123.14364421829868) bank115367 +115368 POINT(48.55676947162724 -123.16998266704967) bank115368 +115369 POINT(47.12080003109883 -123.0859289419202) bank115369 +115370 POINT(47.60284421078014 -121.992913479023) bank115370 +115371 POINT(48.280432100452394 -122.6427064855376) bank115371 +115372 POINT(47.90192084986553 -123.21302909599939) bank115372 +115373 POINT(48.43311397780196 -122.68747864601902) bank115373 +115374 POINT(47.07127504133681 -121.48741035136382) bank115374 +115375 POINT(46.925311826834005 -122.36768626549026) bank115375 +115376 POINT(48.25169217654301 -121.38056645773648) bank115376 +115377 POINT(47.255745416755026 -123.07642102822943) bank115377 +115378 POINT(47.14081747182597 -122.22023502485362) bank115378 +115379 POINT(47.133347663116986 -121.60407556218172) bank115379 +115380 POINT(47.793284938931805 -122.44971272259741) bank115380 +115381 POINT(48.050376694551154 -121.9741419060435) bank115381 +115382 POINT(47.21326436370918 -122.46700158548244) bank115382 +115383 POINT(47.8244642446401 -122.17460581636168) bank115383 +115384 POINT(48.37244858980441 -121.63374689915631) bank115384 +115385 POINT(47.353618473299896 -123.23108452381429) bank115385 +115386 POINT(47.736399814156655 -122.98411707789232) bank115386 +115387 POINT(48.378429719136236 -121.81751252133677) bank115387 +115388 POINT(47.57473512590485 -122.71987392670403) bank115388 +115389 POINT(47.530703851884375 -121.95378023160798) bank115389 +115390 POINT(47.47931165631826 -122.86577331127806) bank115390 +115391 POINT(48.33248642035194 -123.136031974144) bank115391 +115392 POINT(47.19041034964307 -121.35157902108409) bank115392 +115393 POINT(47.37427370313596 -123.20848452824983) bank115393 +115394 POINT(48.436827445189884 -121.61874761727911) bank115394 +115395 POINT(46.77605254007561 -122.1490988818486) bank115395 +115396 POINT(47.940070754639535 -122.50033790537377) bank115396 +115397 POINT(47.42702883998871 -123.12238208168435) bank115397 +115398 POINT(47.509789979638924 -123.14497111787877) bank115398 +115399 POINT(48.598589632131635 -122.5758980341577) bank115399 +115400 POINT(47.317398504138815 -122.2363962257075) bank115400 +115401 POINT(46.951929684066435 -123.23507365365224) bank115401 +115402 POINT(48.516494528744445 -121.48691569727768) bank115402 +115403 POINT(46.62070325736196 -123.31264773149687) bank115403 +115404 POINT(48.318605110075815 -121.44028257773878) bank115404 +115405 POINT(48.130356894196424 -122.39782692141505) bank115405 +115406 POINT(48.14580693458223 -122.88964894183931) bank115406 +115407 POINT(47.36464331437537 -123.09158308401113) bank115407 +115408 POINT(48.12887143933659 -122.26572112558061) bank115408 +115409 POINT(47.94071970323143 -121.45698343983855) bank115409 +115410 POINT(48.418222447833735 -122.95843315647346) bank115410 +115411 POINT(47.38521172966089 -121.43334029535201) bank115411 +115412 POINT(48.337648212178316 -122.40174747259212) bank115412 +115413 POINT(46.859874927490125 -122.78331370680392) bank115413 +115414 POINT(48.43463451286229 -122.51671286573962) bank115414 +115415 POINT(47.78371413007738 -122.63317324682123) bank115415 +115416 POINT(46.966961068511765 -121.42786454917919) bank115416 +115417 POINT(47.99264557304625 -121.4646623455153) bank115417 +115418 POINT(48.41006335409522 -122.872004049605) bank115418 +115419 POINT(48.36215255668004 -122.49826103763328) bank115419 +115420 POINT(47.34570675402088 -122.10227383218333) bank115420 +115421 POINT(46.7243713466103 -121.50228701968885) bank115421 +115422 POINT(47.130934558095966 -121.87374041774086) bank115422 +115423 POINT(47.259209995135166 -121.40170066406434) bank115423 +115424 POINT(47.092313092390526 -121.34590696659541) bank115424 +115425 POINT(46.97928362844027 -122.72293329993231) bank115425 +115426 POINT(46.967327379380826 -123.14597655382491) bank115426 +115427 POINT(47.929243396009966 -122.67098583853186) bank115427 +115428 POINT(47.90964457825384 -122.03774905979188) bank115428 +115429 POINT(47.96322464036686 -122.11227022250495) bank115429 +115430 POINT(46.71782943622238 -122.39597408603198) bank115430 +115431 POINT(46.68616887043479 -121.58720153861641) bank115431 +115432 POINT(47.032102800809454 -121.52826654532652) bank115432 +115433 POINT(48.231935882964535 -123.08066964605301) bank115433 +115434 POINT(47.965840994723884 -121.35164570737862) bank115434 +115435 POINT(48.29610148515267 -122.43040637299019) bank115435 +115436 POINT(47.16446747281727 -121.71588190960028) bank115436 +115437 POINT(47.21007919285962 -121.56140127327804) bank115437 +115438 POINT(47.36720539268209 -121.60056086954528) bank115438 +115439 POINT(48.09535249036422 -121.62990498070002) bank115439 +115440 POINT(48.47511114542178 -123.28629384656688) bank115440 +115441 POINT(47.77244336700083 -123.31023580374412) bank115441 +115442 POINT(48.14535800785136 -121.60127465804736) bank115442 +115443 POINT(47.4717154220597 -121.86004610478025) bank115443 +115444 POINT(47.3347390650619 -121.89682628564657) bank115444 +115445 POINT(46.91414803645763 -122.44943569085326) bank115445 +115446 POINT(47.624491867382176 -122.80977198513727) bank115446 +115447 POINT(47.090800961865014 -122.56796460832672) bank115447 +115448 POINT(47.60903198206286 -122.00518061123779) bank115448 +115449 POINT(48.579239195502396 -122.75930222167894) bank115449 +115450 POINT(47.51200516395072 -123.02597469646211) bank115450 +115451 POINT(48.346373683604774 -122.71428213640245) bank115451 +115452 POINT(47.7069175138241 -123.26596792903088) bank115452 +115453 POINT(48.36916309903805 -122.66572957392775) bank115453 +115454 POINT(47.375924918133045 -121.80319626326775) bank115454 +115455 POINT(47.88384847918211 -123.19926516434468) bank115455 +115456 POINT(47.64214150402448 -123.02213582440783) bank115456 +115457 POINT(48.024107507347544 -123.22794846549223) bank115457 +115458 POINT(47.46893829510234 -121.8685257332965) bank115458 +115459 POINT(47.00219801256539 -121.93330946442099) bank115459 +115460 POINT(47.92460316839162 -121.70797709016071) bank115460 +115461 POINT(47.28398517838009 -123.22590757889361) bank115461 +115462 POINT(46.80613125553159 -121.4781310645161) bank115462 +115463 POINT(47.08784824221618 -123.11551888618355) bank115463 +115464 POINT(47.22877038775197 -122.56691279051266) bank115464 +115465 POINT(47.122935791815415 -123.26401098614542) bank115465 +115466 POINT(48.425049581596355 -121.4832025986872) bank115466 +115467 POINT(47.8076066280517 -121.37685003642483) bank115467 +115468 POINT(46.731820209855634 -122.9316920207525) bank115468 +115469 POINT(48.45381508616679 -122.51786435668923) bank115469 +115470 POINT(46.70014404939933 -121.72005337505928) bank115470 +115471 POINT(48.312446112140776 -122.68259699798438) bank115471 +115472 POINT(47.17760302557721 -122.55824395046481) bank115472 +115473 POINT(48.401929415113095 -122.9173588402915) bank115473 +115474 POINT(47.53821552413861 -123.02368656833991) bank115474 +115475 POINT(47.696537392953815 -121.66011222821054) bank115475 +115476 POINT(46.93109046155803 -121.90219999944625) bank115476 +115477 POINT(48.05460500975376 -121.7690322351974) bank115477 +115478 POINT(47.244090603524604 -122.40244905810634) bank115478 +115479 POINT(47.60710163050179 -122.65750372381507) bank115479 +115480 POINT(48.50511195955514 -123.1249319371772) bank115480 +115481 POINT(48.37672752062964 -123.06978745577025) bank115481 +115482 POINT(46.91752349982951 -121.8247316475749) bank115482 +115483 POINT(46.697922583647156 -122.06664311376652) bank115483 +115484 POINT(48.31118797699773 -121.8010432323122) bank115484 +115485 POINT(47.50451061763702 -121.57268712796534) bank115485 +115486 POINT(47.74159673609125 -122.28325193311345) bank115486 +115487 POINT(47.198976189020314 -121.49981924540269) bank115487 +115488 POINT(47.61556541033318 -121.61492064045272) bank115488 +115489 POINT(48.42007705686009 -122.88469450151861) bank115489 +115490 POINT(46.72847951947698 -121.81200626270869) bank115490 +115491 POINT(47.069278992295445 -121.9445116741855) bank115491 +115492 POINT(48.194289337701086 -121.48150163509568) bank115492 +115493 POINT(46.8094157230543 -121.64371111144519) bank115493 +115494 POINT(47.23365857861396 -122.49572030006905) bank115494 +115495 POINT(48.55337816739661 -122.53864280400121) bank115495 +115496 POINT(47.90429880685118 -122.08388996919736) bank115496 +115497 POINT(46.80966606274815 -122.79269390269184) bank115497 +115498 POINT(48.05397920385305 -122.60289224499185) bank115498 +115499 POINT(48.41509873473712 -122.04811783224072) bank115499 +115500 POINT(47.10046454256941 -123.051211952704) bank115500 +115501 POINT(48.015580588622115 -121.87284978937664) bank115501 +115502 POINT(47.07697466897345 -123.25612264934355) bank115502 +115503 POINT(48.104457094019324 -122.98338454386452) bank115503 +115504 POINT(47.32380565264735 -122.6737042067666) bank115504 +115505 POINT(46.88494760930915 -122.35669267631683) bank115505 +115506 POINT(48.32224315771508 -121.91741062301656) bank115506 +115507 POINT(48.04561330995609 -121.91893406485146) bank115507 +115508 POINT(47.68017564694544 -121.39889736413384) bank115508 +115509 POINT(48.05996531644625 -121.88782169601606) bank115509 +115510 POINT(48.28563216108 -123.19496392069283) bank115510 +115511 POINT(47.197113660314685 -122.33453214731313) bank115511 +115512 POINT(48.28657558280516 -121.471865232907) bank115512 +115513 POINT(47.238578398572535 -122.16523534632746) bank115513 +115514 POINT(48.27155202970431 -122.20814882545405) bank115514 +115515 POINT(47.99622662647664 -123.11328410052002) bank115515 +115516 POINT(47.27959436469122 -121.79581684505895) bank115516 +115517 POINT(47.395163304426454 -122.40157688924978) bank115517 +115518 POINT(47.90098857301858 -121.99532259015965) bank115518 +115519 POINT(47.91478290701263 -123.02843515657706) bank115519 +115520 POINT(47.45403495194138 -121.78712524714109) bank115520 +115521 POINT(46.62308942028819 -121.81518569254544) bank115521 +115522 POINT(48.33578165504024 -122.18174667518774) bank115522 +115523 POINT(47.102765289670025 -122.41141805923009) bank115523 +115524 POINT(47.24181913659288 -121.7346939413232) bank115524 +115525 POINT(47.13948126403803 -122.2083370775254) bank115525 +115526 POINT(46.921894973212 -121.41317867249569) bank115526 +115527 POINT(47.19109893533802 -122.8590883314997) bank115527 +115528 POINT(47.596885090391694 -121.59562190772941) bank115528 +115529 POINT(47.45056609932087 -121.43309493877835) bank115529 +115530 POINT(48.01282596084507 -121.343860316449) bank115530 +115531 POINT(48.08906223917395 -122.19199781474278) bank115531 +115532 POINT(48.03924816752202 -122.95595690274676) bank115532 +115533 POINT(48.583490256976134 -121.57529508600116) bank115533 +115534 POINT(48.100551460825905 -123.10084493215017) bank115534 +115535 POINT(46.812672628850194 -121.59945485781456) bank115535 +115536 POINT(46.65352622426085 -122.00788652319092) bank115536 +115537 POINT(47.30079764681324 -122.96774113349593) bank115537 +115538 POINT(47.99957613865964 -122.37593226201302) bank115538 +115539 POINT(46.98838586277586 -121.66198206104798) bank115539 +115540 POINT(48.23555018750257 -121.88496539320153) bank115540 +115541 POINT(47.716554070556846 -121.52730613225351) bank115541 +115542 POINT(46.96704747412188 -121.4949323648418) bank115542 +115543 POINT(47.80857620757409 -122.85151840456109) bank115543 +115544 POINT(48.250369671428736 -121.44647152493788) bank115544 +115545 POINT(47.48566326343075 -122.28581087041171) bank115545 +115546 POINT(47.253565476594225 -121.85509936488474) bank115546 +115547 POINT(48.58327193961907 -122.53639696054316) bank115547 +115548 POINT(47.06490366110016 -123.30203074453463) bank115548 +115549 POINT(47.53414763688197 -123.3041248378419) bank115549 +115550 POINT(47.166667752897254 -121.96739073908621) bank115550 +115551 POINT(47.267098349192324 -121.58453007809966) bank115551 +115552 POINT(47.237936209838956 -121.78955603994264) bank115552 +115553 POINT(46.85088149617332 -122.98232308141549) bank115553 +115554 POINT(48.21827716655013 -122.94924489384242) bank115554 +115555 POINT(48.37113013196265 -122.82177850162783) bank115555 +115556 POINT(47.56824064843993 -123.01185850882732) bank115556 +115557 POINT(47.4457698663343 -122.89721923126532) bank115557 +115558 POINT(48.098354102831564 -121.9322814072574) bank115558 +115559 POINT(47.26233106641095 -121.66289924968693) bank115559 +115560 POINT(47.53344669990315 -122.27269891253772) bank115560 +115561 POINT(46.88738821859376 -122.07384355585573) bank115561 +115562 POINT(47.48790619512726 -122.11678687311932) bank115562 +115563 POINT(47.19039951868414 -121.63500242579043) bank115563 +115564 POINT(47.57416677100773 -122.42658207211012) bank115564 +115565 POINT(47.59076284144648 -121.94236595497641) bank115565 +115566 POINT(47.21576317572944 -121.56123239806641) bank115566 +115567 POINT(47.54332664255439 -123.31194121060686) bank115567 +115568 POINT(48.00828708118164 -123.17718562816967) bank115568 +115569 POINT(47.1190450532369 -122.61598768792226) bank115569 +115570 POINT(48.35984519421734 -122.5137608234661) bank115570 +115571 POINT(46.71614449548085 -122.27236197025685) bank115571 +115572 POINT(47.3515449631653 -121.97473247083893) bank115572 +115573 POINT(47.55851958131414 -121.96501144677318) bank115573 +115574 POINT(47.55646519367123 -122.2549929285786) bank115574 +115575 POINT(47.52586040760889 -122.3746513572065) bank115575 +115576 POINT(47.076745430458004 -122.08442528560838) bank115576 +115577 POINT(46.844742850858204 -122.40867077787127) bank115577 +115578 POINT(47.508533011780905 -121.34170687388854) bank115578 +115579 POINT(47.508412435747594 -123.16197104063191) bank115579 +115580 POINT(48.591011807820614 -121.58866661290128) bank115580 +115581 POINT(47.57463075677566 -122.29042824680005) bank115581 +115582 POINT(48.437261790105765 -121.4850551223036) bank115582 +115583 POINT(47.059869971475244 -122.60319089959992) bank115583 +115584 POINT(48.43933313799673 -122.48913280920446) bank115584 +115585 POINT(48.21010682352127 -122.86616108641984) bank115585 +115586 POINT(47.646864023353444 -122.54114126303134) bank115586 +115587 POINT(47.87711860016395 -121.48650550430837) bank115587 +115588 POINT(46.82792065824935 -122.8179193402623) bank115588 +115589 POINT(47.23096344193462 -122.30311402352254) bank115589 +115590 POINT(48.0279768089958 -122.59842183981766) bank115590 +115591 POINT(47.77670608124909 -122.19673546230214) bank115591 +115592 POINT(48.15613035078591 -122.37889251872943) bank115592 +115593 POINT(48.360883530417354 -122.57733673266065) bank115593 +115594 POINT(47.08915491155116 -121.34993938191872) bank115594 +115595 POINT(48.14330548747207 -121.5043823148653) bank115595 +115596 POINT(47.911726421200555 -122.99511824116645) bank115596 +115597 POINT(48.34413124992442 -122.6449589823065) bank115597 +115598 POINT(47.60336421152568 -121.4339646323027) bank115598 +115599 POINT(48.515167152267615 -123.31429490737952) bank115599 +115600 POINT(47.87787169405431 -122.79385262821711) bank115600 +115601 POINT(47.35766392297962 -122.11483138339192) bank115601 +115602 POINT(48.14271328298661 -121.68493333197945) bank115602 +115603 POINT(48.217518731049964 -122.02688147846212) bank115603 +115604 POINT(47.63222229500775 -122.48447816600329) bank115604 +115605 POINT(46.868961668869474 -122.45587872062993) bank115605 +115606 POINT(46.666499389930834 -121.609536698939) bank115606 +115607 POINT(47.49982628763996 -123.20040874475247) bank115607 +115608 POINT(46.86095956056132 -122.63478765658913) bank115608 +115609 POINT(47.809445191407704 -121.57737943856054) bank115609 +115610 POINT(48.34302412338805 -123.16265396160861) bank115610 +115611 POINT(47.6490206820348 -122.75473496521053) bank115611 +115612 POINT(47.41584537406781 -123.16696911303946) bank115612 +115613 POINT(48.06837363755285 -122.66169305639771) bank115613 +115614 POINT(47.042209527190884 -123.1650682617564) bank115614 +115615 POINT(47.06817320420404 -121.47632644562293) bank115615 +115616 POINT(47.263805049199796 -123.19782368491857) bank115616 +115617 POINT(47.75140741567548 -122.33757116154673) bank115617 +115618 POINT(47.98691320739491 -121.50960060145054) bank115618 +115619 POINT(48.1073636440368 -121.68594032632242) bank115619 +115620 POINT(47.11394001611241 -122.14984875397175) bank115620 +115621 POINT(48.21550356380203 -122.82036770950656) bank115621 +115622 POINT(47.56992454137267 -122.44671343554622) bank115622 +115623 POINT(47.44682628749471 -121.71464668605512) bank115623 +115624 POINT(48.17620376856435 -121.72507058603287) bank115624 +115625 POINT(48.40284730681273 -123.306489836728) bank115625 +115626 POINT(47.35323628287869 -122.76595634082307) bank115626 +115627 POINT(47.49742049478074 -123.08537353273499) bank115627 +115628 POINT(48.191496958754264 -121.38107297899514) bank115628 +115629 POINT(47.86476250975448 -121.80236669818906) bank115629 +115630 POINT(47.359362484741595 -121.65741748716489) bank115630 +115631 POINT(46.80286085261071 -122.6480825203298) bank115631 +115632 POINT(47.618250555822655 -121.48000349224085) bank115632 +115633 POINT(47.86335524979243 -122.2886056576522) bank115633 +115634 POINT(48.40612229622542 -123.31728206441028) bank115634 +115635 POINT(47.924899620162186 -121.8163093047634) bank115635 +115636 POINT(47.603275221022564 -122.29639739062692) bank115636 +115637 POINT(48.37816208402937 -123.06254290391789) bank115637 +115638 POINT(48.35073836095471 -122.19365109369042) bank115638 +115639 POINT(47.93126674811736 -123.2660595952281) bank115639 +115640 POINT(47.464113101932526 -122.33092552716705) bank115640 +115641 POINT(47.46412713091739 -122.14832914076419) bank115641 +115642 POINT(47.312231776433414 -121.70358589188653) bank115642 +115643 POINT(47.206200886301566 -122.61243267151295) bank115643 +115644 POINT(48.46152985129677 -122.72682729576854) bank115644 +115645 POINT(46.966379596629544 -121.42202139981914) bank115645 +115646 POINT(46.873951780502566 -122.8512075046086) bank115646 +115647 POINT(47.457869463785116 -121.86726325862429) bank115647 +115648 POINT(48.4706420127894 -123.0729489195266) bank115648 +115649 POINT(46.682539617731074 -123.05919967028433) bank115649 +115650 POINT(46.724859414766996 -122.64804047681712) bank115650 +115651 POINT(47.90054809035676 -121.79708478304336) bank115651 +115652 POINT(47.6619000000728 -122.47694341191306) bank115652 +115653 POINT(47.79350669111942 -121.68038587648252) bank115653 +115654 POINT(48.573224684589924 -121.7085900390407) bank115654 +115655 POINT(47.13010089267265 -122.5964631127992) bank115655 +115656 POINT(48.20411724953953 -121.33493553559998) bank115656 +115657 POINT(47.21189044427744 -122.95335785127621) bank115657 +115658 POINT(47.35940115545523 -121.66896842848878) bank115658 +115659 POINT(46.96004624781542 -123.20113267019536) bank115659 +115660 POINT(47.01909264822099 -122.35357890283997) bank115660 +115661 POINT(48.131235323822104 -122.5222377051208) bank115661 +115662 POINT(47.349643840978715 -122.22604745388625) bank115662 +115663 POINT(47.22139135384662 -122.84035921384975) bank115663 +115664 POINT(46.84314025865397 -122.87061808509768) bank115664 +115665 POINT(47.35501686551665 -122.66969370716114) bank115665 +115666 POINT(48.012977401034 -122.82542664498352) bank115666 +115667 POINT(47.339718731545375 -121.85384705787071) bank115667 +115668 POINT(47.12650520818053 -122.02048616456479) bank115668 +115669 POINT(47.26460605650313 -121.55899456576006) bank115669 +115670 POINT(47.81072797810596 -121.61304025832908) bank115670 +115671 POINT(47.828569587969795 -121.64557584142672) bank115671 +115672 POINT(48.492933395860746 -122.1216050217774) bank115672 +115673 POINT(47.1597828094829 -122.43922835467329) bank115673 +115674 POINT(47.764008761502076 -122.01688874973881) bank115674 +115675 POINT(47.550932525635865 -121.82419573665167) bank115675 +115676 POINT(47.98705399894652 -122.14843774341435) bank115676 +115677 POINT(48.096645357673346 -122.82217814183517) bank115677 +115678 POINT(47.101274409049104 -121.53677968968384) bank115678 +115679 POINT(47.784212574928716 -121.39927518692335) bank115679 +115680 POINT(47.875969677619395 -121.7862264655017) bank115680 +115681 POINT(48.27472653923504 -122.78583900651601) bank115681 +115682 POINT(47.94771630925658 -122.43543514009322) bank115682 +115683 POINT(47.91876838279968 -121.52812378622069) bank115683 +115684 POINT(47.24419060505137 -123.12211826174828) bank115684 +115685 POINT(47.636794722162406 -122.10370244058281) bank115685 +115686 POINT(47.689261219134345 -121.64270008569105) bank115686 +115687 POINT(48.24362695631514 -122.32267932149138) bank115687 +115688 POINT(47.1373335951118 -122.69262121486466) bank115688 +115689 POINT(47.29274536310038 -121.71491377661738) bank115689 +115690 POINT(47.68978749660729 -122.67624108665592) bank115690 +115691 POINT(47.33285632259447 -123.2451100856124) bank115691 +115692 POINT(48.27522833187078 -123.15318206929662) bank115692 +115693 POINT(48.13832878870938 -122.60228313450064) bank115693 +115694 POINT(46.68996338563922 -123.08544110659994) bank115694 +115695 POINT(47.52880087699944 -121.6551842433021) bank115695 +115696 POINT(47.655014896696734 -123.26189302339336) bank115696 +115697 POINT(47.123576811555495 -122.49015994153831) bank115697 +115698 POINT(47.26945300583542 -123.05567199637608) bank115698 +115699 POINT(47.32499437188195 -121.6222124803818) bank115699 +115700 POINT(47.707804195773285 -122.22755708248087) bank115700 +115701 POINT(46.66471429283101 -121.89736497068118) bank115701 +115702 POINT(48.407962913173115 -121.53083845737467) bank115702 +115703 POINT(47.98176905794066 -122.94576348449988) bank115703 +115704 POINT(46.95189846849655 -121.3400893960835) bank115704 +115705 POINT(48.04608349140285 -122.53853713342855) bank115705 +115706 POINT(48.189256405471305 -122.48723947324483) bank115706 +115707 POINT(47.21199429210329 -122.58021635146515) bank115707 +115708 POINT(48.11185062449936 -121.39079426658225) bank115708 +115709 POINT(48.508747713518645 -122.00788589840167) bank115709 +115710 POINT(47.4665857538636 -121.9990573180306) bank115710 +115711 POINT(46.72321886140288 -121.62565386519195) bank115711 +115712 POINT(48.33874420443398 -122.87261719275021) bank115712 +115713 POINT(47.36525241234328 -122.97545699161056) bank115713 +115714 POINT(47.29513948529255 -122.3259655500602) bank115714 +115715 POINT(48.18366545320588 -121.93544121427766) bank115715 +115716 POINT(47.262709031219 -122.65198802943534) bank115716 +115717 POINT(47.90805658009929 -122.62005110549181) bank115717 +115718 POINT(47.28447945281676 -122.94072641605433) bank115718 +115719 POINT(47.260698732694294 -122.66163470819485) bank115719 +115720 POINT(48.22021124535806 -122.48210216195406) bank115720 +115721 POINT(47.111539255429015 -121.56726628647858) bank115721 +115722 POINT(46.833518183607765 -121.90823805595792) bank115722 +115723 POINT(46.93119542332384 -121.438078032999) bank115723 +115724 POINT(46.6957795854369 -122.0748679417085) bank115724 +115725 POINT(47.551946340204225 -123.1616604833154) bank115725 +115726 POINT(47.56734134532504 -122.34760199755033) bank115726 +115727 POINT(48.10115183065783 -122.66534593920922) bank115727 +115728 POINT(47.26729135263703 -121.3810711370362) bank115728 +115729 POINT(47.74612963656868 -122.97045822980854) bank115729 +115730 POINT(47.16567045903209 -121.99225909672681) bank115730 +115731 POINT(47.35032176957943 -122.2062152873362) bank115731 +115732 POINT(48.11773102254432 -122.49468178647174) bank115732 +115733 POINT(47.81774890587189 -121.33403467736984) bank115733 +115734 POINT(48.08938896196256 -122.58346125411211) bank115734 +115735 POINT(48.43364877620038 -121.80026076752117) bank115735 +115736 POINT(47.204458594597696 -122.5362031135583) bank115736 +115737 POINT(47.81335995448988 -122.52105379219752) bank115737 +115738 POINT(47.420329654532495 -122.29061833035509) bank115738 +115739 POINT(48.00783607277922 -122.4813484149366) bank115739 +115740 POINT(46.673680923721356 -121.47963376887321) bank115740 +115741 POINT(47.194752678025196 -122.74419388266024) bank115741 +115742 POINT(48.41258571354655 -122.49514227906403) bank115742 +115743 POINT(46.911996537112174 -122.93666770963908) bank115743 +115744 POINT(47.462395803109665 -122.3436892519811) bank115744 +115745 POINT(48.412463445610925 -123.01169545357924) bank115745 +115746 POINT(46.99246825949203 -122.7919349807127) bank115746 +115747 POINT(47.256985755082574 -123.18243735165159) bank115747 +115748 POINT(48.50980087424749 -122.49108413943392) bank115748 +115749 POINT(46.86870806040309 -122.10385176620025) bank115749 +115750 POINT(47.412666819048965 -122.2320935018771) bank115750 +115751 POINT(46.7453213213066 -123.01509385687544) bank115751 +115752 POINT(48.278910345557016 -121.95215579815239) bank115752 +115753 POINT(47.04457197219769 -122.85298710000865) bank115753 +115754 POINT(46.75118572551661 -123.20762551098109) bank115754 +115755 POINT(48.19355074091756 -123.00634501680761) bank115755 +115756 POINT(47.36812736818911 -121.42476396883129) bank115756 +115757 POINT(47.031722119734006 -122.82690830283768) bank115757 +115758 POINT(48.25795162123184 -121.83985342303768) bank115758 +115759 POINT(48.43989974340975 -121.39075800960983) bank115759 +115760 POINT(48.567108896488676 -122.5351354342387) bank115760 +115761 POINT(46.65834407084466 -121.7867197635404) bank115761 +115762 POINT(48.313565439856305 -121.7783941950843) bank115762 +115763 POINT(48.047406933189976 -122.77714117624284) bank115763 +115764 POINT(48.32393437131376 -123.03063263767845) bank115764 +115765 POINT(46.82732658747775 -123.16627255383845) bank115765 +115766 POINT(48.41693637889751 -122.47804502885721) bank115766 +115767 POINT(48.01092132025674 -121.80488391475909) bank115767 +115768 POINT(47.91608479331251 -122.87590701876422) bank115768 +115769 POINT(48.20598103363111 -123.15110289292613) bank115769 +115770 POINT(46.60666439560155 -122.32503892503506) bank115770 +115771 POINT(48.52440843611402 -122.13560428042491) bank115771 +115772 POINT(48.57786427998449 -121.48700580637008) bank115772 +115773 POINT(47.68039630294748 -122.4832322367585) bank115773 +115774 POINT(47.829432434376486 -122.38312188682816) bank115774 +115775 POINT(47.97007183646007 -122.69245984219073) bank115775 +115776 POINT(47.836921413392204 -122.95927332678235) bank115776 +115777 POINT(46.746055461744625 -121.84342338195752) bank115777 +115778 POINT(47.178876389866474 -122.03900232427823) bank115778 +115779 POINT(48.46574139628659 -122.36912014995849) bank115779 +115780 POINT(47.35389162038112 -121.87215482354165) bank115780 +115781 POINT(46.70411497415196 -122.93798539540457) bank115781 +115782 POINT(47.30723947637129 -121.37048972010633) bank115782 +115783 POINT(46.98365201139327 -121.53860216886176) bank115783 +115784 POINT(48.00091462579415 -121.94575990788779) bank115784 +115785 POINT(47.65576280310082 -122.99409223909026) bank115785 +115786 POINT(46.99386601798072 -122.68011364553328) bank115786 +115787 POINT(47.432301015643255 -123.29770141979233) bank115787 +115788 POINT(48.391270595397806 -122.52904538417661) bank115788 +115789 POINT(47.75465791259887 -121.44482858074227) bank115789 +115790 POINT(47.20109398274917 -121.81568806554519) bank115790 +115791 POINT(47.85660886585305 -122.28762637069448) bank115791 +115792 POINT(46.63568495468597 -122.43374320333587) bank115792 +115793 POINT(47.340037215909184 -121.62604451694607) bank115793 +115794 POINT(47.46851416557287 -121.70539182596728) bank115794 +115795 POINT(48.51712204395041 -123.21228203639113) bank115795 +115796 POINT(48.01265274102039 -122.46607514027309) bank115796 +115797 POINT(46.94223269554203 -122.71510200655099) bank115797 +115798 POINT(46.98880103419632 -121.4204912346779) bank115798 +115799 POINT(48.46478795042135 -121.41932795719688) bank115799 +115800 POINT(48.129040663492084 -122.47994146399299) bank115800 +115801 POINT(47.33067745205388 -121.720643238268) bank115801 +115802 POINT(47.11269806615814 -123.13642258210177) bank115802 +115803 POINT(46.904878329986055 -121.79577644475044) bank115803 +115804 POINT(48.3414929415169 -122.73018220913457) bank115804 +115805 POINT(47.67850533483423 -122.82290167293412) bank115805 +115806 POINT(46.9803018103036 -122.55535283719296) bank115806 +115807 POINT(46.928380646462145 -121.82372574485585) bank115807 +115808 POINT(48.44458388153926 -122.47932062274039) bank115808 +115809 POINT(47.67129030209563 -121.4877176542209) bank115809 +115810 POINT(47.27050508531371 -121.39195459350972) bank115810 +115811 POINT(46.66527608314456 -123.3175392324442) bank115811 +115812 POINT(47.59364832924928 -121.6890583999496) bank115812 +115813 POINT(47.54238809706396 -122.82107010706527) bank115813 +115814 POINT(46.99740881162836 -122.76198248193046) bank115814 +115815 POINT(47.081814675709595 -122.0449359466995) bank115815 +115816 POINT(46.94885983425714 -121.8648021095498) bank115816 +115817 POINT(46.886961387652136 -121.89551609762569) bank115817 +115818 POINT(47.58708038247269 -121.61359658916831) bank115818 +115819 POINT(47.387755626609085 -121.5000595791584) bank115819 +115820 POINT(47.711472728435425 -121.43645783434515) bank115820 +115821 POINT(46.9121744552145 -121.60521348527587) bank115821 +115822 POINT(46.93684745501554 -122.00997180377517) bank115822 +115823 POINT(46.84961375784031 -122.92423293900195) bank115823 +115824 POINT(46.69366539572033 -122.21715630183687) bank115824 +115825 POINT(48.23995726660271 -122.26846863891907) bank115825 +115826 POINT(47.171553331441544 -122.66539031471854) bank115826 +115827 POINT(48.206095183895386 -123.25682010536921) bank115827 +115828 POINT(47.02550688911843 -121.40716206552246) bank115828 +115829 POINT(47.45137258505204 -122.64593143268978) bank115829 +115830 POINT(46.8223876818471 -122.80573440608671) bank115830 +115831 POINT(47.0608092615697 -123.23769460883095) bank115831 +115832 POINT(47.0754288421818 -122.73569323548595) bank115832 +115833 POINT(46.633484208609595 -122.55648883380876) bank115833 +115834 POINT(47.37558650458506 -122.86696849893553) bank115834 +115835 POINT(47.81246574905198 -121.71313979755864) bank115835 +115836 POINT(47.56417130835981 -122.82087474079644) bank115836 +115837 POINT(47.638725269911575 -122.7625355471393) bank115837 +115838 POINT(47.45982603261378 -121.67504895066223) bank115838 +115839 POINT(48.11604174770868 -121.63799439718616) bank115839 +115840 POINT(48.21406505315062 -121.65382814995849) bank115840 +115841 POINT(47.308289978266124 -123.16864203217294) bank115841 +115842 POINT(46.776628107079155 -121.68608978416962) bank115842 +115843 POINT(48.305473458639476 -121.4914514684926) bank115843 +115844 POINT(47.52383450503055 -121.92031967359229) bank115844 +115845 POINT(48.12001853772561 -121.38791820771085) bank115845 +115846 POINT(48.601029531162816 -123.00339920312844) bank115846 +115847 POINT(48.254389105627304 -122.0863162139211) bank115847 +115848 POINT(48.28557889260553 -123.0229154137617) bank115848 +115849 POINT(47.603131219492575 -121.59858467233849) bank115849 +115850 POINT(48.27640332349354 -122.0359134346691) bank115850 +115851 POINT(46.70787778041685 -121.52038132492217) bank115851 +115852 POINT(47.52209735067866 -122.02813500229043) bank115852 +115853 POINT(48.42247273204304 -122.3152913897266) bank115853 +115854 POINT(47.722887136908675 -122.35460194322961) bank115854 +115855 POINT(48.047280371399424 -122.60354345040129) bank115855 +115856 POINT(47.08258395797874 -122.79891935149718) bank115856 +115857 POINT(48.3587643417623 -122.49562443754024) bank115857 +115858 POINT(47.26938346067426 -121.96214952194654) bank115858 +115859 POINT(47.82952831788369 -121.3644883386174) bank115859 +115860 POINT(46.8041047074247 -122.93679596682169) bank115860 +115861 POINT(47.47781280306074 -121.98289105236559) bank115861 +115862 POINT(48.42622740049656 -122.34846344991932) bank115862 +115863 POINT(48.23804886408974 -122.68078177151261) bank115863 +115864 POINT(48.48401089990356 -121.57205371669446) bank115864 +115865 POINT(48.36169417198603 -122.68967319534399) bank115865 +115866 POINT(46.682468803433984 -121.89795702700087) bank115866 +115867 POINT(46.961227675513314 -122.05480766485837) bank115867 +115868 POINT(46.7241951418752 -123.25701376524783) bank115868 +115869 POINT(48.289762207517846 -122.82183943247392) bank115869 +115870 POINT(47.85871780517998 -122.91460858939355) bank115870 +115871 POINT(46.828621855381755 -122.66145365609584) bank115871 +115872 POINT(48.18744293059055 -122.78164031158508) bank115872 +115873 POINT(47.641183035754246 -121.47143738397818) bank115873 +115874 POINT(47.74812472330348 -123.26308581285875) bank115874 +115875 POINT(48.218176929649545 -122.23773344100361) bank115875 +115876 POINT(47.383723159500214 -122.85509929887797) bank115876 +115877 POINT(47.87865319744463 -121.52150834635175) bank115877 +115878 POINT(47.17224971528555 -123.15005358596073) bank115878 +115879 POINT(46.6507700692673 -121.4598603510729) bank115879 +115880 POINT(47.386945733297104 -122.38131908122686) bank115880 +115881 POINT(47.46269140443985 -122.74125941800978) bank115881 +115882 POINT(48.568920092500335 -122.71483597997194) bank115882 +115883 POINT(48.21339383554054 -121.74937072680639) bank115883 +115884 POINT(47.74834226367462 -122.33137041333686) bank115884 +115885 POINT(48.04727621986534 -122.93099175428758) bank115885 +115886 POINT(47.46398917540384 -122.79918277748924) bank115886 +115887 POINT(47.76866670362735 -122.12392888921292) bank115887 +115888 POINT(46.870899328913445 -121.70109901926028) bank115888 +115889 POINT(47.14628628465982 -123.20296901854812) bank115889 +115890 POINT(46.92696473751726 -122.57942481307201) bank115890 +115891 POINT(46.92967001542355 -121.92356529136373) bank115891 +115892 POINT(47.02721026678581 -122.96908110081479) bank115892 +115893 POINT(47.72182111761455 -122.09839783973716) bank115893 +115894 POINT(47.66152970257767 -121.86961594098419) bank115894 +115895 POINT(46.64510077791639 -122.66613128996057) bank115895 +115896 POINT(47.116826415399125 -121.98689936642167) bank115896 +115897 POINT(48.232426372581955 -122.26571365587066) bank115897 +115898 POINT(47.45730123282576 -121.70490695487386) bank115898 +115899 POINT(46.720094206626705 -121.43575502071899) bank115899 +115900 POINT(48.51310102960705 -121.5408850906616) bank115900 +115901 POINT(48.4329587283702 -122.42652365427485) bank115901 +115902 POINT(47.88457610744686 -122.70487956789644) bank115902 +115903 POINT(46.94443626687089 -122.68314689739883) bank115903 +115904 POINT(47.661352641484996 -121.96187053575498) bank115904 +115905 POINT(47.852998036878425 -122.92807391788178) bank115905 +115906 POINT(47.23709253679906 -121.42041345967854) bank115906 +115907 POINT(48.36122705882823 -121.65986977657946) bank115907 +115908 POINT(47.23259937813432 -122.37966160447718) bank115908 +115909 POINT(46.6987956696392 -123.12282402269703) bank115909 +115910 POINT(46.87537085736916 -121.60070943156587) bank115910 +115911 POINT(48.28992118949795 -121.88006119507978) bank115911 +115912 POINT(47.17636689725489 -121.79585702268807) bank115912 +115913 POINT(47.50145167971141 -121.74740981035532) bank115913 +115914 POINT(48.39170434213747 -122.70490119922354) bank115914 +115915 POINT(48.263501981828654 -123.07038518878086) bank115915 +115916 POINT(46.66777842195812 -122.65792826334012) bank115916 +115917 POINT(47.218884469767325 -123.03220801680841) bank115917 +115918 POINT(46.775010902989315 -121.5631018023386) bank115918 +115919 POINT(48.3624368364154 -121.61629133307555) bank115919 +115920 POINT(47.99308604754947 -121.40696155800383) bank115920 +115921 POINT(48.17836210277097 -121.56448577258142) bank115921 +115922 POINT(48.522777948194474 -121.66281493576929) bank115922 +115923 POINT(47.02041057131911 -123.14426021085472) bank115923 +115924 POINT(48.45476899097938 -121.81648770225212) bank115924 +115925 POINT(47.55376335220445 -123.1313114448953) bank115925 +115926 POINT(47.964520119340605 -123.29469363695301) bank115926 +115927 POINT(46.9037851804214 -122.8659898914503) bank115927 +115928 POINT(47.46152127769768 -122.36138277099317) bank115928 +115929 POINT(47.46095223873666 -122.40832512292911) bank115929 +115930 POINT(48.212006788519545 -122.14090439027845) bank115930 +115931 POINT(48.09047230638899 -121.8284866606385) bank115931 +115932 POINT(47.214514997221926 -122.54779358480988) bank115932 +115933 POINT(48.2883700047238 -122.62696135090054) bank115933 +115934 POINT(48.17684461847146 -121.65777179719876) bank115934 +115935 POINT(47.14096969823104 -121.76499205789011) bank115935 +115936 POINT(47.84595796421547 -122.53573535616343) bank115936 +115937 POINT(46.95420522262304 -122.46117508626494) bank115937 +115938 POINT(47.87214443866088 -122.31627599471652) bank115938 +115939 POINT(48.50150055185936 -123.1420153002082) bank115939 +115940 POINT(46.772557229616616 -121.45737540544256) bank115940 +115941 POINT(47.40480203142839 -122.77181561812405) bank115941 +115942 POINT(47.27071342019599 -122.10035754854718) bank115942 +115943 POINT(48.0065097715935 -122.49704273027658) bank115943 +115944 POINT(47.46456488978564 -122.38761418554486) bank115944 +115945 POINT(46.79523123693833 -121.6483490637484) bank115945 +115946 POINT(47.40319453825124 -121.99360950763361) bank115946 +115947 POINT(48.329446189527374 -121.61924959550073) bank115947 +115948 POINT(46.66370324175851 -123.26680804486655) bank115948 +115949 POINT(48.578946877049205 -122.60012599409906) bank115949 +115950 POINT(46.85583851609928 -122.49211536192713) bank115950 +115951 POINT(48.246580983913546 -122.70081139589392) bank115951 +115952 POINT(48.46977729860368 -121.83160079411104) bank115952 +115953 POINT(47.38366420055609 -121.84660007250974) bank115953 +115954 POINT(46.926954672482104 -121.73348752630856) bank115954 +115955 POINT(48.10668363238931 -121.78325215578349) bank115955 +115956 POINT(46.86158317749363 -121.56521279869966) bank115956 +115957 POINT(47.56241922641876 -121.538718536969) bank115957 +115958 POINT(47.08177742215951 -121.74982604087249) bank115958 +115959 POINT(46.93654238293627 -122.11124828363921) bank115959 +115960 POINT(47.64324331234263 -122.77557655918756) bank115960 +115961 POINT(46.89602335360995 -123.12283275948023) bank115961 +115962 POINT(46.93732566219293 -121.57373449734601) bank115962 +115963 POINT(48.32114611349498 -122.38014110039138) bank115963 +115964 POINT(47.358036633281436 -121.62069573521612) bank115964 +115965 POINT(47.682208826751925 -122.00209429001555) bank115965 +115966 POINT(46.88694565383497 -121.59310729103419) bank115966 +115967 POINT(47.59055941514746 -122.27552851817148) bank115967 +115968 POINT(47.679573937380724 -121.47499778508278) bank115968 +115969 POINT(48.5699150647358 -121.65266641370954) bank115969 +115970 POINT(47.73876816132455 -122.47320192969318) bank115970 +115971 POINT(48.094006982907516 -121.66804651488471) bank115971 +115972 POINT(46.734692728302534 -121.3742183092363) bank115972 +115973 POINT(48.44606947782766 -121.37512615892352) bank115973 +115974 POINT(48.37755441466946 -122.40099640138925) bank115974 +115975 POINT(47.20494234351417 -122.46106653803601) bank115975 +115976 POINT(47.3519578777767 -122.9512246871998) bank115976 +115977 POINT(46.74761167661383 -122.99817871906481) bank115977 +115978 POINT(48.560019315329896 -121.83288128829228) bank115978 +115979 POINT(47.97490277950246 -123.10043442907919) bank115979 +115980 POINT(47.986434445449326 -121.66693852771196) bank115980 +115981 POINT(47.7103882429138 -121.84033557714467) bank115981 +115982 POINT(48.26947044323787 -122.46977359670329) bank115982 +115983 POINT(46.966054401381754 -122.62066275446188) bank115983 +115984 POINT(47.653813867027246 -122.47081081459336) bank115984 +115985 POINT(47.91785936147375 -122.81753483312053) bank115985 +115986 POINT(48.441808741848654 -121.78784328789926) bank115986 +115987 POINT(47.90946747698507 -121.91903027367368) bank115987 +115988 POINT(47.42053700799237 -121.94052482506092) bank115988 +115989 POINT(48.40745066383304 -123.12275393576577) bank115989 +115990 POINT(47.92941653118472 -123.07848769749195) bank115990 +115991 POINT(47.58685117509691 -122.39966803384623) bank115991 +115992 POINT(46.80110250310897 -123.28261889022951) bank115992 +115993 POINT(48.174596103870286 -121.60417531955656) bank115993 +115994 POINT(48.357669818223414 -122.0809943805753) bank115994 +115995 POINT(48.4606225669038 -121.54583576159332) bank115995 +115996 POINT(48.45118598344221 -122.13918400663032) bank115996 +115997 POINT(48.372986337187264 -123.05885427460984) bank115997 +115998 POINT(47.53390731669459 -122.73159439040812) bank115998 +115999 POINT(47.0721855359172 -122.73632525553086) bank115999 +116000 POINT(48.13430983499636 -123.26687235002379) bank116000 +116001 POINT(47.51811920084302 -121.83805952422323) bank116001 +116002 POINT(47.625036200651195 -122.404972986601) bank116002 +116003 POINT(48.10847772916119 -122.47661969927617) bank116003 +116004 POINT(48.26528674757844 -121.55981028438299) bank116004 +116005 POINT(48.04101166175484 -121.53505040010931) bank116005 +116006 POINT(48.09443940420616 -121.75696378566397) bank116006 +116007 POINT(47.007423214105046 -123.0571412183168) bank116007 +116008 POINT(48.37090437867187 -122.46934660462145) bank116008 +116009 POINT(47.639697628794316 -122.43760135226584) bank116009 +116010 POINT(46.98421489066801 -121.90912102946227) bank116010 +116011 POINT(48.33920888211701 -122.85873428241989) bank116011 +116012 POINT(47.413880580679326 -122.64997596796582) bank116012 +116013 POINT(48.23142836833435 -122.39826529692691) bank116013 +116014 POINT(46.92498045404251 -121.94279451741319) bank116014 +116015 POINT(48.510283938864106 -122.3590743687962) bank116015 +116016 POINT(47.28076321306524 -121.42965740256464) bank116016 +116017 POINT(47.24039029119106 -121.54568318943574) bank116017 +116018 POINT(46.631623931285226 -122.47017431160027) bank116018 +116019 POINT(47.78316275510216 -122.2936201234928) bank116019 +116020 POINT(47.163241874109765 -121.39328757147325) bank116020 +116021 POINT(46.75623843901214 -122.32435688316357) bank116021 +116022 POINT(47.55038492205446 -121.83337345008887) bank116022 +116023 POINT(47.6052684233045 -121.77577019444166) bank116023 +116024 POINT(46.76303966554042 -122.00260564204311) bank116024 +116025 POINT(48.28847919708671 -121.67190185430854) bank116025 +116026 POINT(47.50217187398892 -121.97346850432628) bank116026 +116027 POINT(46.70823462677312 -122.26895298754214) bank116027 +116028 POINT(48.31490574379177 -122.60906635392706) bank116028 +116029 POINT(47.99091711665489 -121.83940812009845) bank116029 +116030 POINT(48.57887388319285 -121.45545475049016) bank116030 +116031 POINT(46.86025600046544 -123.28819554465029) bank116031 +116032 POINT(48.29340915100429 -122.60211821000536) bank116032 +116033 POINT(48.14905505114218 -122.86575618239814) bank116033 +116034 POINT(47.03283767709317 -121.33774239048104) bank116034 +116035 POINT(46.63435346121363 -122.19487824725985) bank116035 +116036 POINT(46.63417597037792 -121.82454758469675) bank116036 +116037 POINT(46.665361218343975 -122.25467565514626) bank116037 +116038 POINT(46.88756917431347 -122.88696307951729) bank116038 +116039 POINT(48.28427130379642 -122.70466127619231) bank116039 +116040 POINT(47.787635894961745 -123.23400526881842) bank116040 +116041 POINT(47.980201316970636 -122.10362498350953) bank116041 +116042 POINT(47.890713700916685 -122.5304930618631) bank116042 +116043 POINT(47.40983158101588 -121.47564938301004) bank116043 +116044 POINT(47.208228924370054 -122.02328080864031) bank116044 +116045 POINT(48.380836064414794 -122.32438048119114) bank116045 +116046 POINT(48.546557194360645 -122.16390475403965) bank116046 +116047 POINT(46.76392009823419 -121.878514294556) bank116047 +116048 POINT(47.075818972323674 -123.27063448323617) bank116048 +116049 POINT(48.071141335430376 -122.99576353433402) bank116049 +116050 POINT(46.93410651466296 -123.22314881568495) bank116050 +116051 POINT(48.36261074477896 -121.86853189342625) bank116051 +116052 POINT(48.47053276552997 -122.9504480626753) bank116052 +116053 POINT(47.80241184027837 -122.57667174289249) bank116053 +116054 POINT(47.04984385542934 -122.85365264845689) bank116054 +116055 POINT(47.61311872691592 -122.7055677270454) bank116055 +116056 POINT(47.94089031350481 -121.53218388819147) bank116056 +116057 POINT(47.90057300171721 -122.86988368022652) bank116057 +116058 POINT(47.98039478053814 -121.67364527435139) bank116058 +116059 POINT(47.21842307405657 -123.03527279919456) bank116059 +116060 POINT(47.42669140581525 -122.64459092259585) bank116060 +116061 POINT(47.54826381754981 -122.83084100778214) bank116061 +116062 POINT(47.53719054110906 -123.19169066013815) bank116062 +116063 POINT(48.52019620335114 -122.2988553963937) bank116063 +116064 POINT(48.59406736600637 -121.794732090158) bank116064 +116065 POINT(46.697376593298486 -123.13240916593533) bank116065 +116066 POINT(48.26325503276411 -121.5190787950952) bank116066 +116067 POINT(47.93594523061462 -122.96411873315772) bank116067 +116068 POINT(46.72949423875471 -121.71864381946969) bank116068 +116069 POINT(47.42452143405148 -122.44086185196197) bank116069 +116070 POINT(47.08793548521552 -121.74640041754543) bank116070 +116071 POINT(48.00450626102844 -122.91941268958283) bank116071 +116072 POINT(47.68626353033193 -121.7341179130237) bank116072 +116073 POINT(47.26324003539508 -121.46873020361251) bank116073 +116074 POINT(47.505211111383964 -122.79941931629398) bank116074 +116075 POINT(47.90402265254601 -123.11891186576621) bank116075 +116076 POINT(48.10478260538586 -121.99295219634985) bank116076 +116077 POINT(47.98230925840875 -122.36473098195755) bank116077 +116078 POINT(46.737380018355125 -123.15658854468231) bank116078 +116079 POINT(48.32660611424741 -121.60214134387431) bank116079 +116080 POINT(46.61006428608781 -122.49716072631954) bank116080 +116081 POINT(48.04237524217888 -122.95231919769432) bank116081 +116082 POINT(47.04424859738372 -122.59162773431973) bank116082 +116083 POINT(47.16644898326108 -121.78492875261756) bank116083 +116084 POINT(48.09665540665351 -121.66373529266843) bank116084 +116085 POINT(48.20556041884057 -121.71453469967825) bank116085 +116086 POINT(47.56849524592565 -122.25205764595086) bank116086 +116087 POINT(47.75284116811155 -122.25001801270254) bank116087 +116088 POINT(47.73867614106088 -121.89462043954516) bank116088 +116089 POINT(47.99482669608503 -122.06312396700142) bank116089 +116090 POINT(47.84255537557737 -122.182491360267) bank116090 +116091 POINT(47.062468605524735 -122.28911586300413) bank116091 +116092 POINT(46.62019345834596 -121.68815783760432) bank116092 +116093 POINT(46.818949860355985 -123.20024552371875) bank116093 +116094 POINT(47.88314303208608 -122.60628070136815) bank116094 +116095 POINT(47.61167567661336 -121.57346931233162) bank116095 +116096 POINT(47.15041436918309 -122.69265001587627) bank116096 +116097 POINT(47.537483175329434 -123.30666353648058) bank116097 +116098 POINT(48.07185403145992 -123.14451973100257) bank116098 +116099 POINT(47.12329011338941 -123.27452596040288) bank116099 +116100 POINT(48.38414394359556 -122.2489970572115) bank116100 +116101 POINT(47.649403810303724 -121.47113694832306) bank116101 +116102 POINT(46.67863932559801 -123.23522898856618) bank116102 +116103 POINT(48.342952481152 -122.0427208177305) bank116103 +116104 POINT(47.618268031301696 -121.72035103581447) bank116104 +116105 POINT(48.000376210341024 -121.96084787090548) bank116105 +116106 POINT(47.226799162076546 -121.44218484231675) bank116106 +116107 POINT(48.19803171980138 -121.87336502151193) bank116107 +116108 POINT(48.135177528971084 -123.05402852301593) bank116108 +116109 POINT(48.600026304908646 -122.59383649400716) bank116109 +116110 POINT(48.37927140697069 -122.48438496281412) bank116110 +116111 POINT(48.0089602199932 -122.55091732913719) bank116111 +116112 POINT(48.042297425808236 -123.20269020205322) bank116112 +116113 POINT(48.35676143783584 -121.81782090529576) bank116113 +116114 POINT(48.19488207964805 -123.1833388553941) bank116114 +116115 POINT(48.27127427732566 -122.20405643098032) bank116115 +116116 POINT(47.098347513778215 -121.37642777850476) bank116116 +116117 POINT(47.677843756756936 -122.04365253667626) bank116117 +116118 POINT(47.43720921544527 -121.45155719337956) bank116118 +116119 POINT(46.717302229230064 -121.56847432869692) bank116119 +116120 POINT(48.43641367895743 -121.79021358482206) bank116120 +116121 POINT(48.54113155660132 -123.30925758632773) bank116121 +116122 POINT(48.40041187040575 -121.56228602461483) bank116122 +116123 POINT(48.36759073702185 -121.56079693781776) bank116123 +116124 POINT(46.8973072156995 -122.24809381043283) bank116124 +116125 POINT(47.46935823743377 -122.94247197067658) bank116125 +116126 POINT(47.43506430930555 -123.18077320742098) bank116126 +116127 POINT(48.49484641869178 -122.5750096947126) bank116127 +116128 POINT(46.89546643771479 -121.71529621218724) bank116128 +116129 POINT(48.26665452715713 -122.31120435186905) bank116129 +116130 POINT(48.02842604763725 -121.86907700313712) bank116130 +116131 POINT(46.76336501805299 -122.17668584994946) bank116131 +116132 POINT(48.064556702387485 -121.37078901123861) bank116132 +116133 POINT(46.70063564177849 -122.64240941846897) bank116133 +116134 POINT(47.59904296649398 -121.8301826544043) bank116134 +116135 POINT(48.33917962108575 -121.34794789920937) bank116135 +116136 POINT(47.436773984179865 -122.76458378873369) bank116136 +116137 POINT(48.255257137890844 -121.53370768802256) bank116137 +116138 POINT(47.49079334635477 -123.17311010682025) bank116138 +116139 POINT(46.979057526758034 -122.089771059003) bank116139 +116140 POINT(48.13288652836748 -121.74535635215487) bank116140 +116141 POINT(47.72214336201462 -122.63130976198752) bank116141 +116142 POINT(46.68043385258088 -121.94120649053033) bank116142 +116143 POINT(48.21335702265469 -122.32019158532036) bank116143 +116144 POINT(47.274017032465515 -122.46146745635092) bank116144 +116145 POINT(46.886602796808766 -123.21716132516245) bank116145 +116146 POINT(47.04225917011615 -123.08270992874331) bank116146 +116147 POINT(47.61036060285597 -121.83354127839155) bank116147 +116148 POINT(48.42890491936063 -122.87573285827384) bank116148 +116149 POINT(47.95800599931966 -122.74931915631073) bank116149 +116150 POINT(48.115643020920224 -122.65566672855876) bank116150 +116151 POINT(47.88534498150631 -122.69037379117442) bank116151 +116152 POINT(48.02923241550229 -122.81127288794335) bank116152 +116153 POINT(46.72207965335157 -121.34153845366149) bank116153 +116154 POINT(47.304871050673796 -123.20006410613979) bank116154 +116155 POINT(47.2688818330753 -122.0245080215659) bank116155 +116156 POINT(48.190113377325716 -121.79415177872276) bank116156 +116157 POINT(47.717124849796114 -122.78208307230769) bank116157 +116158 POINT(46.76144222738489 -122.45679455045467) bank116158 +116159 POINT(47.37560907433433 -121.75319992870058) bank116159 +116160 POINT(47.03159480997772 -121.91491339178619) bank116160 +116161 POINT(47.305563758738266 -122.73489720082264) bank116161 +116162 POINT(48.26871243953614 -123.18471102393713) bank116162 +116163 POINT(47.27714565971005 -121.34078578150228) bank116163 +116164 POINT(47.71683184065526 -121.52824549010742) bank116164 +116165 POINT(47.94273403705268 -121.46258378072268) bank116165 +116166 POINT(48.24993710992462 -121.39148750790297) bank116166 +116167 POINT(47.89226140161136 -123.3131304615245) bank116167 +116168 POINT(47.73329531292012 -122.80573921960911) bank116168 +116169 POINT(48.258710183330685 -122.22567990408605) bank116169 +116170 POINT(46.95106148461653 -122.13813737572636) bank116170 +116171 POINT(47.52129575550503 -122.55781342325312) bank116171 +116172 POINT(47.4965674842899 -122.84062069398084) bank116172 +116173 POINT(48.440730309181305 -122.78184066193738) bank116173 +116174 POINT(48.21829476138959 -121.45018872146977) bank116174 +116175 POINT(48.38593076343767 -122.2709787515195) bank116175 +116176 POINT(48.353058082272575 -122.34689381462516) bank116176 +116177 POINT(47.431950523283874 -121.95024300352897) bank116177 +116178 POINT(47.58987868526058 -122.18937168282304) bank116178 +116179 POINT(47.861091269543806 -121.5744644726003) bank116179 +116180 POINT(48.43595331894496 -123.15023001063025) bank116180 +116181 POINT(47.95052797576527 -123.20379883521059) bank116181 +116182 POINT(47.35023454749003 -122.65878150493899) bank116182 +116183 POINT(46.760417900518824 -123.29116737100227) bank116183 +116184 POINT(47.11042925896723 -122.34635716280805) bank116184 +116185 POINT(48.40827406369418 -123.1206485252424) bank116185 +116186 POINT(47.552940068598744 -121.87964286616247) bank116186 +116187 POINT(47.14322923719355 -121.4944216645759) bank116187 +116188 POINT(48.15464176576202 -122.30156815801014) bank116188 +116189 POINT(48.573922663402996 -121.6471018586946) bank116189 +116190 POINT(46.85355409350769 -121.57091443458734) bank116190 +116191 POINT(48.05175589930548 -123.07839717843677) bank116191 +116192 POINT(47.317753084538495 -121.84627144198583) bank116192 +116193 POINT(47.29405299969664 -122.03892891408353) bank116193 +116194 POINT(47.207426523290096 -121.88281374797404) bank116194 +116195 POINT(47.05758770154456 -123.29586424038843) bank116195 +116196 POINT(47.50231517536397 -121.37331968247669) bank116196 +116197 POINT(47.07110477034109 -122.39285189597493) bank116197 +116198 POINT(48.11154406558137 -123.04566824355516) bank116198 +116199 POINT(47.71074360963686 -122.39935047496304) bank116199 +116200 POINT(47.24414990052213 -122.28205548829014) bank116200 +116201 POINT(47.746571253184946 -122.50788931139168) bank116201 +116202 POINT(47.62122864390687 -122.78131155519398) bank116202 +116203 POINT(47.567415634374306 -122.99746777009165) bank116203 +116204 POINT(47.60911188300578 -123.25970590453184) bank116204 +116205 POINT(48.41677532629018 -123.2179079861643) bank116205 +116206 POINT(48.420822090563085 -122.6620370750124) bank116206 +116207 POINT(48.20178808217816 -121.70527545442044) bank116207 +116208 POINT(47.38884859258479 -121.51562679320894) bank116208 +116209 POINT(47.634195621197044 -122.39896536779025) bank116209 +116210 POINT(47.48394943095844 -122.13676077820814) bank116210 +116211 POINT(48.49951806854623 -121.54145777541561) bank116211 +116212 POINT(47.19471031732213 -122.9236963082458) bank116212 +116213 POINT(46.96240747255644 -122.34915464858359) bank116213 +116214 POINT(47.045776574492436 -122.59765426307469) bank116214 +116215 POINT(47.0135976930067 -121.9028533931602) bank116215 +116216 POINT(48.57171808658556 -123.16069543999949) bank116216 +116217 POINT(47.70336599241835 -121.98354442042829) bank116217 +116218 POINT(48.56693413834179 -121.74537268972621) bank116218 +116219 POINT(47.744644639570915 -121.63049989016638) bank116219 +116220 POINT(48.0437617528995 -122.01859656987442) bank116220 +116221 POINT(46.82546135534434 -121.86941377549405) bank116221 +116222 POINT(47.81077255343197 -121.92868079515702) bank116222 +116223 POINT(47.9375787427461 -121.349509952833) bank116223 +116224 POINT(46.999438003916225 -122.44767138298742) bank116224 +116225 POINT(48.2415205065762 -121.48937772389525) bank116225 +116226 POINT(48.36193394919728 -123.26562244290442) bank116226 +116227 POINT(47.62092714656919 -122.4907017775497) bank116227 +116228 POINT(47.74979945877404 -122.21426548866864) bank116228 +116229 POINT(48.37724532791421 -121.8962591678193) bank116229 +116230 POINT(47.93360794429552 -121.69024290436985) bank116230 +116231 POINT(46.82591195493259 -122.64228562816017) bank116231 +116232 POINT(47.94290230384058 -122.76884579300058) bank116232 +116233 POINT(48.27629614139666 -121.63304334772681) bank116233 +116234 POINT(47.44736190761264 -122.86585108708077) bank116234 +116235 POINT(48.32173529065106 -123.08158970666025) bank116235 +116236 POINT(47.420638995802825 -121.84519254848091) bank116236 +116237 POINT(47.70214031872575 -122.96360448103133) bank116237 +116238 POINT(47.125515786525305 -122.90074334013866) bank116238 +116239 POINT(48.4417909379107 -122.8157123314505) bank116239 +116240 POINT(47.188747848941205 -121.43450779313252) bank116240 +116241 POINT(48.032581795453424 -121.99745899537093) bank116241 +116242 POINT(47.71332866739505 -122.42980964623021) bank116242 +116243 POINT(47.220186773001146 -122.69603824557649) bank116243 +116244 POINT(47.51343892172428 -123.33119369271502) bank116244 +116245 POINT(47.2812183165535 -122.56585319220409) bank116245 +116246 POINT(47.96762554604436 -121.98680588999068) bank116246 +116247 POINT(47.00258461740468 -121.95416102032135) bank116247 +116248 POINT(47.29622152949382 -122.7765663456823) bank116248 +116249 POINT(46.758353591617734 -121.76344911423203) bank116249 +116250 POINT(47.35997340656575 -122.33897694085768) bank116250 +116251 POINT(46.64519298680901 -122.99893798663652) bank116251 +116252 POINT(48.14725708406807 -122.65338968025313) bank116252 +116253 POINT(47.56727580362185 -122.36467188688094) bank116253 +116254 POINT(46.970281560583054 -122.44878827386911) bank116254 +116255 POINT(47.4219701611387 -122.31998810126858) bank116255 +116256 POINT(47.94727130149031 -121.54952802812275) bank116256 +116257 POINT(47.73254177890028 -122.42762028954905) bank116257 +116258 POINT(48.17870206310005 -122.1253411002176) bank116258 +116259 POINT(48.47583784346375 -123.09641201189028) bank116259 +116260 POINT(48.31838752860606 -121.99305086845509) bank116260 +116261 POINT(48.60292015438299 -122.67198911074087) bank116261 +116262 POINT(48.43683177428922 -121.40453396422657) bank116262 +116263 POINT(46.77733694167282 -122.517709707581) bank116263 +116264 POINT(48.40943492516189 -122.64814449371453) bank116264 +116265 POINT(47.58479616417656 -122.0096830930329) bank116265 +116266 POINT(48.587387546177396 -122.86466013605832) bank116266 +116267 POINT(47.09083244478522 -121.90223837591206) bank116267 +116268 POINT(47.85767649020718 -121.91309511823258) bank116268 +116269 POINT(47.224950975927 -123.01365105863488) bank116269 +116270 POINT(46.7959266105156 -122.13557735296035) bank116270 +116271 POINT(46.70831478286309 -122.88550297629429) bank116271 +116272 POINT(48.54176071329167 -123.28983234109272) bank116272 +116273 POINT(47.18817330698382 -122.97394511943868) bank116273 +116274 POINT(48.55094259168639 -121.8243957990368) bank116274 +116275 POINT(47.88340600469116 -122.02462308342267) bank116275 +116276 POINT(47.73997687855318 -121.7220727847109) bank116276 +116277 POINT(47.08038937951749 -122.03492073352835) bank116277 +116278 POINT(47.8592960669229 -122.44465009670157) bank116278 +116279 POINT(48.41283266804838 -122.14638140343327) bank116279 +116280 POINT(47.6900012978561 -122.3193679491136) bank116280 +116281 POINT(46.82177390290431 -123.08701338235754) bank116281 +116282 POINT(47.050308140129935 -121.98639197396528) bank116282 +116283 POINT(47.62186590379233 -122.9803963277418) bank116283 +116284 POINT(47.04036606445506 -122.82821724887906) bank116284 +116285 POINT(47.092462217674495 -121.36338566057348) bank116285 +116286 POINT(47.77451824847614 -122.54296193738273) bank116286 +116287 POINT(47.495504521209575 -122.53871900320173) bank116287 +116288 POINT(48.03274493612022 -122.64927970397426) bank116288 +116289 POINT(48.52268528935774 -121.62599396675121) bank116289 +116290 POINT(47.18666584370862 -122.35825930441865) bank116290 +116291 POINT(46.97207535580863 -122.15600879955142) bank116291 +116292 POINT(48.567030572355456 -121.56046942969384) bank116292 +116293 POINT(46.866426439787766 -122.17149788912998) bank116293 +116294 POINT(46.96173672508176 -123.26138384702399) bank116294 +116295 POINT(47.008573172967964 -121.84128772134095) bank116295 +116296 POINT(46.75713778274329 -122.12631932976007) bank116296 +116297 POINT(48.31921712678294 -123.0048193378325) bank116297 +116298 POINT(47.72062329414766 -123.20251710042211) bank116298 +116299 POINT(47.65385035780926 -121.80558228385354) bank116299 +116300 POINT(46.99782910268375 -122.97682727822814) bank116300 +116301 POINT(47.26184040766331 -121.66581677467875) bank116301 +116302 POINT(48.364527775299585 -123.17032386330887) bank116302 +116303 POINT(48.00457504598451 -122.03652725210087) bank116303 +116304 POINT(48.272338198452516 -121.67731057308214) bank116304 +116305 POINT(48.395050721880665 -121.85922100618917) bank116305 +116306 POINT(46.64957378902713 -122.75142760020762) bank116306 +116307 POINT(48.1538743163922 -121.51745076614682) bank116307 +116308 POINT(46.824439378882595 -123.2679357942984) bank116308 +116309 POINT(48.3195338061124 -122.16086226334475) bank116309 +116310 POINT(47.376132675570624 -121.7189392578695) bank116310 +116311 POINT(47.102943449674704 -121.95918100947588) bank116311 +116312 POINT(47.630619544606176 -123.22112662533003) bank116312 +116313 POINT(47.69826639512932 -122.2325471681191) bank116313 +116314 POINT(47.878563893751384 -121.41520578034685) bank116314 +116315 POINT(47.31916570480261 -122.54593072515317) bank116315 +116316 POINT(47.22381458581835 -122.27336790249167) bank116316 +116317 POINT(47.67872960726487 -122.3277719166989) bank116317 +116318 POINT(47.28861586022839 -121.93464999274875) bank116318 +116319 POINT(47.25712246893565 -121.39731194467323) bank116319 +116320 POINT(47.73280430067858 -122.84430109515463) bank116320 +116321 POINT(47.04005601031566 -121.7817506109048) bank116321 +116322 POINT(47.726843176521434 -122.56791962563524) bank116322 +116323 POINT(46.728763071978904 -123.04174516695318) bank116323 +116324 POINT(46.80668363897909 -121.86810627962167) bank116324 +116325 POINT(48.18987565029495 -122.4839949141427) bank116325 +116326 POINT(48.13451546745572 -121.9066432068925) bank116326 +116327 POINT(47.77275862629917 -121.59657463898732) bank116327 +116328 POINT(47.71848155636205 -121.77640574598026) bank116328 +116329 POINT(47.942070327599716 -121.99449730863925) bank116329 +116330 POINT(47.34148016104799 -121.84670147366445) bank116330 +116331 POINT(48.08731892432379 -123.2820283018145) bank116331 +116332 POINT(46.63820140849811 -123.11231271466768) bank116332 +116333 POINT(46.979553275115244 -122.18841224628288) bank116333 +116334 POINT(47.91094632215055 -122.26039299260651) bank116334 +116335 POINT(48.45067385359829 -121.9337189634914) bank116335 +116336 POINT(48.24460490870626 -121.7166625576453) bank116336 +116337 POINT(47.29883178223992 -121.8269763317657) bank116337 +116338 POINT(48.03798518556198 -123.30786060476231) bank116338 +116339 POINT(47.50794584358071 -121.74084889728631) bank116339 +116340 POINT(47.58984419493633 -122.54825848746326) bank116340 +116341 POINT(46.652770916370216 -122.33677176684307) bank116341 +116342 POINT(47.198161234355105 -122.17255551308618) bank116342 +116343 POINT(48.49533744828854 -123.28823214895607) bank116343 +116344 POINT(46.72787253670989 -121.93058173147229) bank116344 +116345 POINT(47.87505126775484 -121.51455674252976) bank116345 +116346 POINT(46.63154345791864 -121.589285398626) bank116346 +116347 POINT(47.61048762846672 -122.96374220275857) bank116347 +116348 POINT(48.01537598945152 -122.8442617771597) bank116348 +116349 POINT(46.652661745628365 -121.80970592260273) bank116349 +116350 POINT(47.898871028749646 -121.88940374328357) bank116350 +116351 POINT(46.7088077130079 -122.46663103816073) bank116351 +116352 POINT(48.02568761854302 -121.77278266763143) bank116352 +116353 POINT(46.92478294158005 -121.39989600480286) bank116353 +116354 POINT(47.64124354127943 -122.222954159327) bank116354 +116355 POINT(47.23389274300732 -121.36142028065056) bank116355 +116356 POINT(47.83471544229614 -122.57179202043613) bank116356 +116357 POINT(47.138580021859475 -121.65925859580872) bank116357 +116358 POINT(48.58801994962066 -121.37042370970028) bank116358 +116359 POINT(47.50213481092113 -121.78120555905187) bank116359 +116360 POINT(48.448937927772945 -122.68918121634984) bank116360 +116361 POINT(48.57029653043376 -121.50749308805763) bank116361 +116362 POINT(47.55663908366843 -122.83728703231233) bank116362 +116363 POINT(47.187244102677695 -123.06895616590143) bank116363 +116364 POINT(47.044817615942364 -121.34747266508782) bank116364 +116365 POINT(47.784209874027184 -121.93981050395692) bank116365 +116366 POINT(46.763557935232384 -122.01647014851414) bank116366 +116367 POINT(47.26346275731203 -122.21054739416107) bank116367 +116368 POINT(48.564927297151826 -121.88602182111478) bank116368 +116369 POINT(47.97070321416936 -123.04497729296733) bank116369 +116370 POINT(46.62017456840076 -122.09006750689662) bank116370 +116371 POINT(47.640295910239516 -121.99747323042337) bank116371 +116372 POINT(47.781266312704005 -121.42730241607968) bank116372 +116373 POINT(47.811702828866736 -122.52205916942299) bank116373 +116374 POINT(47.50033321053719 -122.33878101566535) bank116374 +116375 POINT(47.24666530397817 -123.12965029824299) bank116375 +116376 POINT(47.38319446083651 -122.7622229621826) bank116376 +116377 POINT(47.64859682789694 -122.00258855678508) bank116377 +116378 POINT(47.73756855433576 -121.90307416973) bank116378 +116379 POINT(47.72997753863889 -121.96424054368663) bank116379 +116380 POINT(47.389689712630215 -121.62023526797523) bank116380 +116381 POINT(48.19829118791209 -122.79589215529889) bank116381 +116382 POINT(48.190127542297134 -123.21455844990243) bank116382 +116383 POINT(47.8711306987915 -123.30317427605505) bank116383 +116384 POINT(47.94233757318671 -123.20119692698017) bank116384 +116385 POINT(48.454767099129256 -122.47716060703964) bank116385 +116386 POINT(46.621015445764215 -121.72827345263997) bank116386 +116387 POINT(47.882805704559495 -123.32585078737885) bank116387 +116388 POINT(48.03241422209391 -123.0506566103484) bank116388 +116389 POINT(48.48643629899054 -122.09750599428475) bank116389 +116390 POINT(47.44501860632868 -121.60276499045172) bank116390 +116391 POINT(47.80240344456981 -122.13313043521192) bank116391 +116392 POINT(47.63597856139462 -122.97568097398727) bank116392 +116393 POINT(48.078108614040616 -121.44286585143956) bank116393 +116394 POINT(46.900541741830395 -122.73858215074313) bank116394 +116395 POINT(48.4584315674464 -122.90360790335126) bank116395 +116396 POINT(48.49159369719787 -123.12540970160347) bank116396 +116397 POINT(46.78944672477046 -123.06520720120508) bank116397 +116398 POINT(48.09879736964655 -123.04362902885127) bank116398 +116399 POINT(48.20441235875512 -122.87732172316744) bank116399 +116400 POINT(47.05377057208892 -121.66874802475711) bank116400 +116401 POINT(47.73758151836987 -122.85243754383575) bank116401 +116402 POINT(48.09987687561526 -123.29838355421613) bank116402 +116403 POINT(48.100565681882095 -121.378609477319) bank116403 +116404 POINT(47.33862914343296 -121.50711840754833) bank116404 +116405 POINT(48.082305780788204 -121.42722157547325) bank116405 +116406 POINT(47.27039341735791 -123.0230919310989) bank116406 +116407 POINT(48.5229851313365 -123.28261427612051) bank116407 +116408 POINT(48.464958530062695 -121.5382131807005) bank116408 +116409 POINT(47.377187735662325 -121.35012217686) bank116409 +116410 POINT(48.335612882349395 -122.08766493055363) bank116410 +116411 POINT(47.44420154671934 -122.99163261724291) bank116411 +116412 POINT(46.93736774281656 -121.69651646055195) bank116412 +116413 POINT(47.35454809794981 -122.80551895358802) bank116413 +116414 POINT(47.61421840411925 -123.03499667845368) bank116414 +116415 POINT(48.11660750403762 -122.97426259018434) bank116415 +116416 POINT(46.873393492089455 -122.06721618460816) bank116416 +116417 POINT(47.69506020399284 -122.68935269818051) bank116417 +116418 POINT(46.78420723103613 -121.77657779746413) bank116418 +116419 POINT(48.14067506149137 -123.25136718397623) bank116419 +116420 POINT(46.84849612645557 -121.55962843044713) bank116420 +116421 POINT(48.38232261457296 -122.8056163224829) bank116421 +116422 POINT(46.951305470904714 -122.55164226966804) bank116422 +116423 POINT(47.59267524582749 -121.40539978254786) bank116423 +116424 POINT(46.67835953799387 -122.81206357098898) bank116424 +116425 POINT(48.28706731130097 -121.90099073302375) bank116425 +116426 POINT(48.40136446166808 -122.12564607969037) bank116426 +116427 POINT(48.11617302827874 -121.89706004061638) bank116427 +116428 POINT(47.69265089295606 -123.27874447840068) bank116428 +116429 POINT(48.58977225966676 -121.46797345206657) bank116429 +116430 POINT(48.407778722192326 -123.20057225457094) bank116430 +116431 POINT(47.51492790637961 -122.64292281641049) bank116431 +116432 POINT(48.60139492991414 -121.75626675080171) bank116432 +116433 POINT(48.261704635629776 -122.77515315052483) bank116433 +116434 POINT(48.302865814627694 -121.33385589785468) bank116434 +116435 POINT(47.98151492501008 -121.58295721294972) bank116435 +116436 POINT(48.445271044731015 -122.14491370003545) bank116436 +116437 POINT(48.36948785237317 -121.92446779626569) bank116437 +116438 POINT(47.557394706917876 -122.04220892247366) bank116438 +116439 POINT(47.96827624144748 -122.85383347702727) bank116439 +116440 POINT(48.04351804788132 -122.67435373250373) bank116440 +116441 POINT(46.91766825647916 -123.11133902671834) bank116441 +116442 POINT(47.75815120403163 -122.70279514446247) bank116442 +116443 POINT(47.21923296548259 -123.12565186134738) bank116443 +116444 POINT(47.29440782054111 -121.41776258698859) bank116444 +116445 POINT(48.48160243748335 -122.36835338621664) bank116445 +116446 POINT(47.38244802930322 -122.32058878970753) bank116446 +116447 POINT(48.28966192863427 -123.28234602753835) bank116447 +116448 POINT(47.097786253387895 -122.96823712061031) bank116448 +116449 POINT(47.84118567484167 -122.3057246272571) bank116449 +116450 POINT(48.404647932411095 -123.2201993519121) bank116450 +116451 POINT(48.44693053826824 -122.41117675155651) bank116451 +116452 POINT(47.10681314345184 -121.78014345355052) bank116452 +116453 POINT(48.05195020084869 -123.28461282614242) bank116453 +116454 POINT(47.48096710445221 -122.96172850745326) bank116454 +116455 POINT(46.89424682111677 -121.74001969404263) bank116455 +116456 POINT(47.91829785717097 -122.11131801542102) bank116456 +116457 POINT(48.580825051312985 -121.33679440679005) bank116457 +116458 POINT(46.93012289097189 -122.4592254077076) bank116458 +116459 POINT(46.99817568148841 -121.82298159056762) bank116459 +116460 POINT(48.113715611300435 -121.55674617004682) bank116460 +116461 POINT(46.79133277715355 -123.07567510301246) bank116461 +116462 POINT(48.55700093403843 -122.42537077316) bank116462 +116463 POINT(48.08019207295735 -121.70710433189201) bank116463 +116464 POINT(48.51884520305885 -121.54218121660591) bank116464 +116465 POINT(47.433923401099115 -122.67886427175182) bank116465 +116466 POINT(46.88917065461145 -122.37970750271782) bank116466 +116467 POINT(47.12034813898948 -122.06490454520062) bank116467 +116468 POINT(47.0727480405645 -121.402484876838) bank116468 +116469 POINT(47.21449719190357 -122.58407118402455) bank116469 +116470 POINT(47.80041377995944 -122.79028514379058) bank116470 +116471 POINT(46.67695114151803 -122.88787305979463) bank116471 +116472 POINT(47.387146343322165 -122.32892854572093) bank116472 +116473 POINT(47.23092581378762 -121.60296486236098) bank116473 +116474 POINT(47.74428949998535 -122.15740989468293) bank116474 +116475 POINT(47.5118485865921 -122.67798818969763) bank116475 +116476 POINT(46.75621614373812 -122.81758842975397) bank116476 +116477 POINT(48.05290947503083 -123.02687889360591) bank116477 +116478 POINT(47.17425434571807 -121.36781589184989) bank116478 +116479 POINT(46.87764065367014 -122.08747105212365) bank116479 +116480 POINT(47.516847735715984 -122.38412698099414) bank116480 +116481 POINT(47.220876777676104 -122.02007201804686) bank116481 +116482 POINT(48.18790283050243 -122.49840341371116) bank116482 +116483 POINT(47.2365965043544 -121.96860514346031) bank116483 +116484 POINT(48.01336769412367 -121.6182425679277) bank116484 +116485 POINT(47.659345633425005 -122.05934731706155) bank116485 +116486 POINT(47.424499520215605 -122.46427650282287) bank116486 +116487 POINT(47.484414358992495 -122.68199315597941) bank116487 +116488 POINT(48.15043947651507 -122.51187870910843) bank116488 +116489 POINT(48.19959156567857 -121.35290868931156) bank116489 +116490 POINT(46.88864902499787 -121.45555511001902) bank116490 +116491 POINT(48.37894846038852 -122.41671503171547) bank116491 +116492 POINT(46.93189926925353 -122.80192772111353) bank116492 +116493 POINT(48.05050880877724 -121.80351289730311) bank116493 +116494 POINT(48.09306743378866 -122.98689226208928) bank116494 +116495 POINT(48.480141211092494 -123.06318754102617) bank116495 +116496 POINT(47.07776507162199 -121.39119908801236) bank116496 +116497 POINT(46.88538840086339 -121.70118768482942) bank116497 +116498 POINT(47.38918804608301 -122.0533675272379) bank116498 +116499 POINT(46.921244487263515 -123.30762829933474) bank116499 +116500 POINT(48.36391969490005 -121.4532299256676) bank116500 +116501 POINT(47.196038625801805 -121.45743927754471) bank116501 +116502 POINT(48.01314478224845 -122.9014574372054) bank116502 +116503 POINT(47.09612524459527 -121.92148986857282) bank116503 +116504 POINT(48.445302310270385 -122.45830012160337) bank116504 +116505 POINT(46.68479123794665 -121.65923612318224) bank116505 +116506 POINT(48.26039325853854 -121.67477316048007) bank116506 +116507 POINT(47.20409518898676 -122.57665955431426) bank116507 +116508 POINT(48.43729536893653 -122.48202532871038) bank116508 +116509 POINT(47.46836918720258 -122.11496625427107) bank116509 +116510 POINT(48.59746736590416 -122.68566544829285) bank116510 +116511 POINT(46.77405724071725 -121.80405806893324) bank116511 +116512 POINT(46.65819115682471 -123.17851457068761) bank116512 +116513 POINT(47.1508191375515 -122.17199382746425) bank116513 +116514 POINT(48.45846958315426 -121.44107773726398) bank116514 +116515 POINT(46.78612000588048 -122.05958295095039) bank116515 +116516 POINT(47.182761214927055 -121.58247647451039) bank116516 +116517 POINT(47.87555122684895 -122.47093898560277) bank116517 +116518 POINT(47.02072602795057 -122.44622417076292) bank116518 +116519 POINT(48.31230688872326 -122.14603066467099) bank116519 +116520 POINT(47.01728985297563 -122.66824924485846) bank116520 +116521 POINT(48.475882412677485 -121.35753572587123) bank116521 +116522 POINT(46.71406563184795 -123.25682158651165) bank116522 +116523 POINT(47.1205114226577 -123.0191021927154) bank116523 +116524 POINT(48.357406597211885 -123.21917195927243) bank116524 +116525 POINT(47.49374674112676 -122.20666183771387) bank116525 +116526 POINT(47.482172947687836 -122.41681981545456) bank116526 +116527 POINT(48.34100249939241 -121.93084836023408) bank116527 +116528 POINT(46.811823625462 -121.34656667963019) bank116528 +116529 POINT(48.150524994621364 -122.95755799783579) bank116529 +116530 POINT(47.412090007867896 -122.1736476233957) bank116530 +116531 POINT(47.335481981130336 -121.77678379844728) bank116531 +116532 POINT(47.005820332400965 -122.53492801272391) bank116532 +116533 POINT(48.393122110258034 -121.77938122326067) bank116533 +116534 POINT(47.98201223314685 -121.39016475689606) bank116534 +116535 POINT(46.67566424371427 -123.10186520960819) bank116535 +116536 POINT(47.14486482542998 -121.43677113958766) bank116536 +116537 POINT(47.43013544247753 -121.5019901365241) bank116537 +116538 POINT(48.52686749013938 -121.58837540839333) bank116538 +116539 POINT(47.56033537756317 -121.38792019457041) bank116539 +116540 POINT(47.3243082521046 -121.78133015119168) bank116540 +116541 POINT(48.17749017201438 -123.08059499559738) bank116541 +116542 POINT(48.04495832512984 -122.1856608082047) bank116542 +116543 POINT(47.10416687921216 -122.29075937863077) bank116543 +116544 POINT(47.32987160791696 -123.03817668327058) bank116544 +116545 POINT(47.48914280252118 -122.87644576093979) bank116545 +116546 POINT(47.98792042177787 -121.33433557796215) bank116546 +116547 POINT(47.26308729367984 -121.58645906789958) bank116547 +116548 POINT(47.98550366523438 -123.30653801556554) bank116548 +116549 POINT(47.38674016258545 -121.86874986472148) bank116549 +116550 POINT(46.789808273339965 -122.17095829366022) bank116550 +116551 POINT(48.15711631776898 -122.85439969308247) bank116551 +116552 POINT(47.372848016621575 -121.847307183528) bank116552 +116553 POINT(48.44053623032555 -121.42898345850702) bank116553 +116554 POINT(46.6852061661355 -121.84551846231331) bank116554 +116555 POINT(47.28933253374011 -122.89104432281644) bank116555 +116556 POINT(48.483558379484016 -121.45110419434131) bank116556 +116557 POINT(48.429003096856114 -123.25955399527086) bank116557 +116558 POINT(47.94159346525487 -122.03336159265136) bank116558 +116559 POINT(47.15788167078257 -123.19375165933317) bank116559 +116560 POINT(48.563418025037976 -122.15973373734374) bank116560 +116561 POINT(48.505279745448284 -123.04264088524806) bank116561 +116562 POINT(46.84266416205627 -121.76670432737859) bank116562 +116563 POINT(47.76213871884631 -122.40148309532543) bank116563 +116564 POINT(48.09445862815428 -122.9386507789542) bank116564 +116565 POINT(47.33385999737511 -121.91105199991787) bank116565 +116566 POINT(47.05962931115466 -122.4452089065716) bank116566 +116567 POINT(47.11766332048032 -121.97462629526609) bank116567 +116568 POINT(46.84916586300032 -122.98708806452068) bank116568 +116569 POINT(47.23219960007612 -121.63926179928256) bank116569 +116570 POINT(48.25351968588828 -122.385177870385) bank116570 +116571 POINT(46.814207940876095 -121.59389194964231) bank116571 +116572 POINT(48.48057454008518 -123.09669834771815) bank116572 +116573 POINT(47.776509641606594 -122.35359398329635) bank116573 +116574 POINT(47.25271142327056 -122.53168119295238) bank116574 +116575 POINT(47.09317547715158 -122.94941999288932) bank116575 +116576 POINT(48.22423122404978 -122.14531329719624) bank116576 +116577 POINT(47.03939306207118 -121.54182454660085) bank116577 +116578 POINT(47.857538314271174 -121.63064038384077) bank116578 +116579 POINT(48.355055124209095 -123.0042727355517) bank116579 +116580 POINT(47.12899119440445 -122.02112896837613) bank116580 +116581 POINT(48.47332777540935 -122.29367726563036) bank116581 +116582 POINT(47.515355958189176 -121.67152129346995) bank116582 +116583 POINT(48.359659424462016 -123.03737331599768) bank116583 +116584 POINT(47.51392677094414 -121.5937301498607) bank116584 +116585 POINT(47.5248779463238 -122.71304898013622) bank116585 +116586 POINT(46.67212687633245 -121.68370838619013) bank116586 +116587 POINT(47.45119815862502 -122.79376529481955) bank116587 +116588 POINT(46.6994086134245 -121.51328507340575) bank116588 +116589 POINT(47.21546950672978 -122.16481485311638) bank116589 +116590 POINT(48.27363586529104 -122.0797438591113) bank116590 +116591 POINT(48.56441884224947 -121.42461412076666) bank116591 +116592 POINT(46.992851355162216 -123.05091380347751) bank116592 +116593 POINT(47.87905491017324 -123.09214265532236) bank116593 +116594 POINT(48.57010032398516 -122.22540281329712) bank116594 +116595 POINT(47.669307542129715 -121.49300924202423) bank116595 +116596 POINT(47.32579971619838 -122.28980076226316) bank116596 +116597 POINT(48.31577953165413 -121.93102056257062) bank116597 +116598 POINT(48.441355792128185 -121.37383248348296) bank116598 +116599 POINT(47.514283272277915 -121.75369536577774) bank116599 +116600 POINT(46.65449323475687 -123.18281456612597) bank116600 +116601 POINT(46.65966182030472 -121.88808117822394) bank116601 +116602 POINT(47.896218182042745 -122.45954328461583) bank116602 +116603 POINT(47.01962315641558 -122.01977012345513) bank116603 +116604 POINT(47.20942836624038 -123.11388168931987) bank116604 +116605 POINT(48.37000970325852 -122.31467007831915) bank116605 +116606 POINT(48.40090167908696 -122.6925199576366) bank116606 +116607 POINT(47.08098953023764 -122.4917005676381) bank116607 +116608 POINT(47.05054472189442 -121.4818935257189) bank116608 +116609 POINT(48.60242619704912 -121.90817016237371) bank116609 +116610 POINT(48.27508892192752 -122.77164679575183) bank116610 +116611 POINT(48.05127709536637 -123.287076642478) bank116611 +116612 POINT(48.317236000200005 -123.17742482921277) bank116612 +116613 POINT(48.42730443574563 -121.74831287259147) bank116613 +116614 POINT(48.14120631498179 -121.58765294019726) bank116614 +116615 POINT(47.52698425068178 -122.5226307341931) bank116615 +116616 POINT(47.7078504513096 -123.31488676550003) bank116616 +116617 POINT(47.758064922415876 -123.01732357839042) bank116617 +116618 POINT(46.736285847499765 -121.95778998796143) bank116618 +116619 POINT(46.94722670367318 -121.93499422508353) bank116619 +116620 POINT(48.11311791657852 -121.39484922048132) bank116620 +116621 POINT(47.93175014595775 -123.08540664644794) bank116621 +116622 POINT(47.32635085957392 -123.22261475921373) bank116622 +116623 POINT(48.11419370697275 -122.47130119398578) bank116623 +116624 POINT(48.0152930473515 -121.36782005946208) bank116624 +116625 POINT(47.812505290699846 -121.49372302664077) bank116625 +116626 POINT(48.369672240626144 -123.00066995596187) bank116626 +116627 POINT(48.19309255481231 -123.07396845785563) bank116627 +116628 POINT(48.05531769211406 -122.90780250899694) bank116628 +116629 POINT(47.023652874796056 -121.34176123242071) bank116629 +116630 POINT(48.26709104187788 -123.07256656373046) bank116630 +116631 POINT(47.5110698695991 -122.87621255133526) bank116631 +116632 POINT(48.27076953861069 -121.48660092289768) bank116632 +116633 POINT(47.467858962377896 -122.57831979958428) bank116633 +116634 POINT(46.62088285164452 -121.61634457414553) bank116634 +116635 POINT(46.95103326216837 -121.51076871654475) bank116635 +116636 POINT(46.96026955714499 -121.60930968364076) bank116636 +116637 POINT(46.92078042372306 -122.45207678237551) bank116637 +116638 POINT(47.491643566704646 -121.99974197389766) bank116638 +116639 POINT(48.53787054885735 -121.8041755418305) bank116639 +116640 POINT(48.5482908874078 -121.79876998017235) bank116640 +116641 POINT(46.69100697396733 -121.6579483929297) bank116641 +116642 POINT(48.104791552545 -122.89780392425064) bank116642 +116643 POINT(46.9547982122905 -123.20272765903987) bank116643 +116644 POINT(48.09053820127336 -122.6514858544255) bank116644 +116645 POINT(47.83581020341215 -121.39326329982195) bank116645 +116646 POINT(47.45259323450991 -122.35365749439626) bank116646 +116647 POINT(48.384855103216 -121.87303895025725) bank116647 +116648 POINT(46.72528118022267 -121.52877811673989) bank116648 +116649 POINT(47.18028016747626 -122.54763617532109) bank116649 +116650 POINT(46.68438903475927 -123.16346216924255) bank116650 +116651 POINT(48.415398971331626 -121.62438305055417) bank116651 +116652 POINT(48.29489441456238 -122.3263934745704) bank116652 +116653 POINT(47.12636966390737 -122.9476473986132) bank116653 +116654 POINT(47.198025431071386 -122.46823406388636) bank116654 +116655 POINT(46.835038705007825 -122.45464434175312) bank116655 +116656 POINT(48.13790619053105 -122.04215487296112) bank116656 +116657 POINT(48.50525512174838 -123.01450162862663) bank116657 +116658 POINT(47.27498309875141 -122.00030006030552) bank116658 +116659 POINT(47.37841539020373 -122.5797226385425) bank116659 +116660 POINT(47.639980226478656 -123.24874367412102) bank116660 +116661 POINT(47.414890652106315 -122.41056389344575) bank116661 +116662 POINT(48.50914444141289 -123.25986728032605) bank116662 +116663 POINT(46.99710211400701 -122.61308104931673) bank116663 +116664 POINT(48.55885645136155 -121.56754581987737) bank116664 +116665 POINT(48.013462509949434 -122.99914999551063) bank116665 +116666 POINT(46.774110804563556 -123.2077032789617) bank116666 +116667 POINT(47.5758041166463 -121.9691664377079) bank116667 +116668 POINT(47.31120350526311 -121.3853376036366) bank116668 +116669 POINT(47.448749559369325 -121.77357325906162) bank116669 +116670 POINT(47.772817414203075 -122.82936623311863) bank116670 +116671 POINT(47.80482176818588 -122.2550954649283) bank116671 +116672 POINT(48.2257526890962 -121.44415756415924) bank116672 +116673 POINT(47.27626685539418 -121.75896728194945) bank116673 +116674 POINT(47.18600118764068 -123.05673105585959) bank116674 +116675 POINT(47.53543189586242 -122.20486707377471) bank116675 +116676 POINT(48.55070070972476 -122.36116964883915) bank116676 +116677 POINT(46.92840682242579 -121.95466561998117) bank116677 +116678 POINT(47.27436342659566 -122.62656633757892) bank116678 +116679 POINT(47.18682863423768 -121.59192917391997) bank116679 +116680 POINT(48.517098334762025 -122.41491095670095) bank116680 +116681 POINT(47.43507530920631 -123.02328722979817) bank116681 +116682 POINT(48.139198344997915 -122.57899162471219) bank116682 +116683 POINT(48.499133194819244 -121.54147594669266) bank116683 +116684 POINT(47.03363344501442 -122.78559132993132) bank116684 +116685 POINT(48.54797565585358 -121.92572221448948) bank116685 +116686 POINT(48.1473625856507 -121.95578406835045) bank116686 +116687 POINT(47.41217762813001 -121.37789997335955) bank116687 +116688 POINT(47.731851849875 -122.60071916453649) bank116688 +116689 POINT(47.25568972387488 -123.19597906687555) bank116689 +116690 POINT(48.29667176900947 -121.93085277906826) bank116690 +116691 POINT(47.8065538487411 -121.78536642516562) bank116691 +116692 POINT(47.34568955364123 -122.04752781429849) bank116692 +116693 POINT(48.43299240980747 -123.29105286004787) bank116693 +116694 POINT(47.96705016627452 -123.31170804673286) bank116694 +116695 POINT(48.122166968514684 -122.02240012337124) bank116695 +116696 POINT(48.117104663195256 -122.07982822055934) bank116696 +116697 POINT(47.636652495235204 -122.31947744412776) bank116697 +116698 POINT(46.99580335944963 -121.83311504256508) bank116698 +116699 POINT(47.626861387316254 -121.90355143090072) bank116699 +116700 POINT(48.234187722648215 -121.33428506563611) bank116700 +116701 POINT(48.270638760912796 -122.68362985988405) bank116701 +116702 POINT(47.61676069404101 -121.49942304137656) bank116702 +116703 POINT(46.937891377834916 -123.0634965158353) bank116703 +116704 POINT(47.13139367470106 -122.27707465216702) bank116704 +116705 POINT(47.869488565935455 -123.28491432380044) bank116705 +116706 POINT(47.44649083734622 -122.14977933305357) bank116706 +116707 POINT(47.850393041767 -122.9136939226023) bank116707 +116708 POINT(47.82235561707012 -122.34874901427702) bank116708 +116709 POINT(46.986245296100364 -122.45996644354106) bank116709 +116710 POINT(47.78082087324309 -121.56447426881782) bank116710 +116711 POINT(47.39862368374452 -122.56605040466347) bank116711 +116712 POINT(47.696379261183566 -122.24908048682926) bank116712 +116713 POINT(47.96321159828004 -121.84185174558081) bank116713 +116714 POINT(47.29605641623147 -121.61575606553501) bank116714 +116715 POINT(46.89410298550113 -121.6477197500464) bank116715 +116716 POINT(47.16322442981213 -123.27845392532836) bank116716 +116717 POINT(48.54294099745726 -122.66007774743696) bank116717 +116718 POINT(48.20185219629992 -122.89996610620787) bank116718 +116719 POINT(47.90172742667704 -123.2494489933448) bank116719 +116720 POINT(47.14255697679304 -123.00140303839825) bank116720 +116721 POINT(47.7960975224726 -122.1373377916276) bank116721 +116722 POINT(47.28817719194168 -122.25222189269266) bank116722 +116723 POINT(47.53291159992542 -122.21408779687125) bank116723 +116724 POINT(46.65376442897332 -121.48199551073415) bank116724 +116725 POINT(48.00684797943973 -122.8453599878985) bank116725 +116726 POINT(47.9278248888574 -121.983806353985) bank116726 +116727 POINT(47.857090314726285 -122.64719640668254) bank116727 +116728 POINT(47.35391379647607 -121.45615101357158) bank116728 +116729 POINT(47.509324276715034 -122.9417067053724) bank116729 +116730 POINT(47.99467780332288 -121.92700544417369) bank116730 +116731 POINT(47.21559892689193 -123.20705190085017) bank116731 +116732 POINT(46.926285171588326 -121.54750518437169) bank116732 +116733 POINT(46.91334555789495 -122.48148123234903) bank116733 +116734 POINT(47.82741481817706 -121.72318988224481) bank116734 +116735 POINT(46.841929268398964 -122.15443026676603) bank116735 +116736 POINT(47.497418413455705 -122.20009585377119) bank116736 +116737 POINT(47.05405423757869 -122.10465148353272) bank116737 +116738 POINT(47.755487711091824 -122.2509664892055) bank116738 +116739 POINT(48.459477619082875 -122.96869046889393) bank116739 +116740 POINT(47.811330684429954 -122.64463520822922) bank116740 +116741 POINT(48.00128047810709 -122.68668224758463) bank116741 +116742 POINT(47.732265454417686 -123.22746153928564) bank116742 +116743 POINT(47.94904259803228 -123.0002515412293) bank116743 +116744 POINT(47.14804172154897 -123.01365013877545) bank116744 +116745 POINT(48.17626570211841 -121.3563536059464) bank116745 +116746 POINT(47.45032915240114 -121.57779999975615) bank116746 +116747 POINT(48.20022616083758 -121.70980186956392) bank116747 +116748 POINT(46.999917626073476 -121.44987378602782) bank116748 +116749 POINT(48.0804259423358 -121.88366250786157) bank116749 +116750 POINT(48.23532584597659 -123.06635054462375) bank116750 +116751 POINT(48.076395713474646 -122.88818746923921) bank116751 +116752 POINT(47.225699722611 -123.23314532319272) bank116752 +116753 POINT(48.328336616921014 -122.74171443204784) bank116753 +116754 POINT(48.301181652353534 -122.74161354452593) bank116754 +116755 POINT(48.2509739976583 -121.98168925604264) bank116755 +116756 POINT(48.31992083846572 -121.50841690086166) bank116756 +116757 POINT(47.071382885981215 -122.38649856373921) bank116757 +116758 POINT(47.02996880678659 -121.4496932742283) bank116758 +116759 POINT(47.63740039260366 -123.30792731878722) bank116759 +116760 POINT(47.26146956485331 -121.97490936304159) bank116760 +116761 POINT(47.25631545579853 -122.49276171623441) bank116761 +116762 POINT(48.19243155341826 -121.79620359496816) bank116762 +116763 POINT(47.897102680231725 -123.26169547495024) bank116763 +116764 POINT(48.20501489535524 -121.702854646121) bank116764 +116765 POINT(47.98427403147192 -122.29100837033022) bank116765 +116766 POINT(47.2272815595403 -121.48295060388641) bank116766 +116767 POINT(47.234715979081116 -121.81415858651833) bank116767 +116768 POINT(47.05630588593266 -122.10189643859347) bank116768 +116769 POINT(47.3168852324759 -122.29232203587392) bank116769 +116770 POINT(47.62968740245502 -122.14622620790334) bank116770 +116771 POINT(46.619411102654425 -122.63641695862768) bank116771 +116772 POINT(48.16358394518163 -122.18495120516802) bank116772 +116773 POINT(47.366578256019636 -122.39396872213672) bank116773 +116774 POINT(47.92424647329734 -121.8207506725228) bank116774 +116775 POINT(48.20306866630891 -122.14197545998486) bank116775 +116776 POINT(48.25434967779749 -121.47352242147352) bank116776 +116777 POINT(47.92191700286839 -121.37884820785737) bank116777 +116778 POINT(46.873914862628176 -121.85305615005369) bank116778 +116779 POINT(46.96768370056257 -122.88900323909876) bank116779 +116780 POINT(47.20664263291066 -123.31539120578516) bank116780 +116781 POINT(46.86239333225031 -123.05371902042889) bank116781 +116782 POINT(48.28487279292647 -122.42379343830486) bank116782 +116783 POINT(47.438754427795246 -122.01298556590163) bank116783 +116784 POINT(46.91059514064218 -121.92198209198384) bank116784 +116785 POINT(46.92350360979864 -122.3496605760625) bank116785 +116786 POINT(47.495159005077205 -122.16748860798971) bank116786 +116787 POINT(48.476157686138166 -121.52222740721712) bank116787 +116788 POINT(46.84571520505108 -122.32803094723154) bank116788 +116789 POINT(47.8754527992839 -123.18997045760203) bank116789 +116790 POINT(46.80024155828292 -122.82833181804357) bank116790 +116791 POINT(48.32057113910649 -121.55315901051462) bank116791 +116792 POINT(47.27361810942866 -122.43615062854711) bank116792 +116793 POINT(47.06060457894977 -122.44651973655799) bank116793 +116794 POINT(47.12658810066299 -123.14920928594194) bank116794 +116795 POINT(46.67110450629116 -122.09001053254592) bank116795 +116796 POINT(47.85224960099814 -121.49623385488471) bank116796 +116797 POINT(47.08344029278871 -123.13402922010818) bank116797 +116798 POINT(47.64841320862582 -122.7047536971964) bank116798 +116799 POINT(48.471306655783856 -122.55644097486558) bank116799 +116800 POINT(47.584148382296924 -121.93364064135207) bank116800 +116801 POINT(48.60474270237265 -121.4059599063683) bank116801 +116802 POINT(46.66339822649486 -123.28069395407434) bank116802 +116803 POINT(46.82398001002799 -121.62743103338936) bank116803 +116804 POINT(48.48179290911196 -121.41016680735108) bank116804 +116805 POINT(47.562385269080956 -121.44044906338918) bank116805 +116806 POINT(46.946442897446815 -121.44538743186403) bank116806 +116807 POINT(46.70797316798395 -121.90091649060284) bank116807 +116808 POINT(46.632611143380764 -121.69452804245377) bank116808 +116809 POINT(47.37125960016665 -121.39434303158686) bank116809 +116810 POINT(46.763951972486 -122.49686760905492) bank116810 +116811 POINT(46.86090903062085 -122.15212661187307) bank116811 +116812 POINT(47.24175867634601 -122.01604001003794) bank116812 +116813 POINT(46.90426486926922 -122.86712185826194) bank116813 +116814 POINT(47.47852558353478 -122.90867461384833) bank116814 +116815 POINT(46.811420079689874 -121.72084830658456) bank116815 +116816 POINT(47.98855061573592 -121.47027021690994) bank116816 +116817 POINT(48.094387054969545 -122.95556191596184) bank116817 +116818 POINT(47.93405246486071 -121.80501526448269) bank116818 +116819 POINT(48.196245618194986 -121.81811267781906) bank116819 +116820 POINT(48.190126471084845 -121.985479108808) bank116820 +116821 POINT(46.81379753423778 -123.3191643015769) bank116821 +116822 POINT(47.95344443704762 -123.13204650454725) bank116822 +116823 POINT(47.79690442021354 -122.62014104286403) bank116823 +116824 POINT(48.45366296950088 -122.23163373808906) bank116824 +116825 POINT(47.10278827537324 -122.43244093160384) bank116825 +116826 POINT(47.39928326509239 -122.83430163127333) bank116826 +116827 POINT(47.900450928436676 -123.14154734912724) bank116827 +116828 POINT(47.796410644205174 -123.18060277773043) bank116828 +116829 POINT(46.63396492097576 -121.57527547421954) bank116829 +116830 POINT(46.808090098191826 -123.01764829888165) bank116830 +116831 POINT(47.548378439690644 -123.21078457374777) bank116831 +116832 POINT(46.94517873852011 -121.44628442949882) bank116832 +116833 POINT(48.389206271463536 -122.89826983314217) bank116833 +116834 POINT(47.40622490235775 -121.96836436171544) bank116834 +116835 POINT(47.251878711527404 -123.30414599957219) bank116835 +116836 POINT(47.97116383387528 -121.84711302918676) bank116836 +116837 POINT(47.25486532432032 -121.44404431012882) bank116837 +116838 POINT(48.37842661978745 -121.87996741756197) bank116838 +116839 POINT(47.82844544030176 -123.29996417858013) bank116839 +116840 POINT(47.52871260486085 -123.28724428908873) bank116840 +116841 POINT(47.237833278482306 -122.66332639650884) bank116841 +116842 POINT(48.59156856914754 -122.25470356422274) bank116842 +116843 POINT(46.925942381358034 -122.80063053176221) bank116843 +116844 POINT(47.03434474717808 -123.21707363988506) bank116844 +116845 POINT(47.349076352139306 -122.66033352618942) bank116845 +116846 POINT(47.69143840804086 -122.3903796059529) bank116846 +116847 POINT(47.61678630738276 -122.73072005438422) bank116847 +116848 POINT(46.66466125552594 -122.45574612221007) bank116848 +116849 POINT(48.53629611957209 -121.76093922656065) bank116849 +116850 POINT(46.87982463738232 -121.65449809086094) bank116850 +116851 POINT(46.74848950996683 -121.93874732824865) bank116851 +116852 POINT(48.079878463459764 -122.7038082435716) bank116852 +116853 POINT(47.67213448455741 -122.40623246884498) bank116853 +116854 POINT(46.77091481405577 -121.54454816364112) bank116854 +116855 POINT(47.14994375817822 -123.17776437304532) bank116855 +116856 POINT(47.25109263675783 -122.10082115308875) bank116856 +116857 POINT(48.60047302510715 -121.33918389137474) bank116857 +116858 POINT(47.887929508570025 -121.83619492704692) bank116858 +116859 POINT(47.67491360919993 -122.26110794936424) bank116859 +116860 POINT(47.094427175015845 -123.12430532379123) bank116860 +116861 POINT(48.4244959734232 -122.36182924844654) bank116861 +116862 POINT(47.660135265393684 -122.25033643094793) bank116862 +116863 POINT(48.49034619093881 -122.15849201132623) bank116863 +116864 POINT(48.03291973897878 -121.79274069752486) bank116864 +116865 POINT(47.03566252678341 -121.66056047500423) bank116865 +116866 POINT(47.941869739837784 -121.61505558503731) bank116866 +116867 POINT(48.02284330898933 -121.45367472053698) bank116867 +116868 POINT(47.98672258149483 -121.59476863689375) bank116868 +116869 POINT(47.56997563038122 -122.58490754704336) bank116869 +116870 POINT(46.73317031323664 -121.87889718453356) bank116870 +116871 POINT(47.00244005107929 -122.05632928186643) bank116871 +116872 POINT(46.61970619259263 -122.23567087152973) bank116872 +116873 POINT(48.35408752936043 -122.20858350330133) bank116873 +116874 POINT(48.431008210733104 -121.60451858945923) bank116874 +116875 POINT(46.783374487701344 -121.89716383142152) bank116875 +116876 POINT(48.24023154005189 -121.93181007165819) bank116876 +116877 POINT(47.5244945550354 -121.9285183925209) bank116877 +116878 POINT(48.51799608581465 -122.92083334088682) bank116878 +116879 POINT(47.601375284477314 -122.18527048011747) bank116879 +116880 POINT(48.478716748237694 -123.11503527723312) bank116880 +116881 POINT(48.26555759166801 -123.2123223882894) bank116881 +116882 POINT(47.294686597714225 -122.45746239454432) bank116882 +116883 POINT(47.49758548528845 -122.56276796675593) bank116883 +116884 POINT(46.91715575509596 -121.42037973247989) bank116884 +116885 POINT(46.94793627543069 -122.82726955862557) bank116885 +116886 POINT(48.31081577356601 -122.53124901306236) bank116886 +116887 POINT(48.25938862662154 -123.23260012836208) bank116887 +116888 POINT(48.02804774497814 -123.00104540353952) bank116888 +116889 POINT(48.26118046050881 -122.89176151197495) bank116889 +116890 POINT(48.60461462492447 -122.0547734898847) bank116890 +116891 POINT(46.970420540208416 -122.93044898860225) bank116891 +116892 POINT(47.8152826786777 -123.1135939290708) bank116892 +116893 POINT(46.84893424789203 -122.35630721022427) bank116893 +116894 POINT(47.89342717013506 -122.17214876520416) bank116894 +116895 POINT(46.76792172747174 -121.60677260218995) bank116895 +116896 POINT(48.444591382167125 -122.84086735149147) bank116896 +116897 POINT(46.63980987200871 -123.14330641135007) bank116897 +116898 POINT(47.16847332195657 -122.68277381314722) bank116898 +116899 POINT(46.785774194703706 -121.71220992389244) bank116899 +116900 POINT(47.78984813921498 -121.90322987165439) bank116900 +116901 POINT(48.288935236365745 -122.34361991995404) bank116901 +116902 POINT(47.71403507807725 -122.66771945195401) bank116902 +116903 POINT(48.49416809063248 -122.30129060593232) bank116903 +116904 POINT(47.55514706424474 -122.12466043379044) bank116904 +116905 POINT(47.84556297629837 -121.76255627097125) bank116905 +116906 POINT(48.306229828361936 -123.20792290983651) bank116906 +116907 POINT(47.10583779120768 -121.95133873393561) bank116907 +116908 POINT(47.035161324087646 -121.99398522900397) bank116908 +116909 POINT(48.40195755646457 -122.88933839080566) bank116909 +116910 POINT(47.74357053990078 -123.28530329768371) bank116910 +116911 POINT(48.36911172821943 -122.15591709021548) bank116911 +116912 POINT(46.87676284633355 -122.38712007213257) bank116912 +116913 POINT(47.324359896042445 -122.72108524797832) bank116913 +116914 POINT(46.69636943212337 -122.43390325150486) bank116914 +116915 POINT(47.945159887965005 -123.18781651854158) bank116915 +116916 POINT(47.62820031443883 -122.44535562583762) bank116916 +116917 POINT(48.12301854971296 -121.72798797477361) bank116917 +116918 POINT(46.71538350557117 -122.05731597197916) bank116918 +116919 POINT(48.29781449767676 -122.58321997926352) bank116919 +116920 POINT(46.88675153193703 -122.24036049665445) bank116920 +116921 POINT(46.84608065269525 -122.360757621784) bank116921 +116922 POINT(47.88822866425901 -123.08394023582014) bank116922 +116923 POINT(48.15068421274229 -122.74894792763126) bank116923 +116924 POINT(48.49347822098682 -122.81142704303082) bank116924 +116925 POINT(47.32453848079079 -122.58143846228128) bank116925 +116926 POINT(46.80624464757854 -122.7056211953003) bank116926 +116927 POINT(47.381729539186594 -121.96088199611943) bank116927 +116928 POINT(47.02969289375187 -121.43021961190307) bank116928 +116929 POINT(47.307460243153436 -122.29925622674432) bank116929 +116930 POINT(47.32531784244667 -122.43209087694781) bank116930 +116931 POINT(46.66710422427757 -122.15109488309075) bank116931 +116932 POINT(47.121827610559336 -121.65486260809924) bank116932 +116933 POINT(48.06958538189119 -122.15620728360923) bank116933 +116934 POINT(47.467371650234874 -122.5817132344424) bank116934 +116935 POINT(46.945106173838184 -121.74508671077186) bank116935 +116936 POINT(46.95895806802108 -122.86683763003656) bank116936 +116937 POINT(46.67814810626331 -122.01079865153531) bank116937 +116938 POINT(47.979439328212905 -122.83412335968792) bank116938 +116939 POINT(46.840936552944 -122.49407371618754) bank116939 +116940 POINT(47.18042002822442 -122.60998619157658) bank116940 +116941 POINT(47.09790230956128 -121.65812461011475) bank116941 +116942 POINT(47.65465178099592 -122.95157007680852) bank116942 +116943 POINT(48.2927644524647 -122.08319085176258) bank116943 +116944 POINT(48.36193346238608 -122.40763110700262) bank116944 +116945 POINT(47.641822980381946 -122.29529349547919) bank116945 +116946 POINT(48.44783175269263 -122.69403271692109) bank116946 +116947 POINT(48.17082242450667 -122.29628524861504) bank116947 +116948 POINT(46.73070631290301 -122.89955531847178) bank116948 +116949 POINT(47.661314549111346 -122.6964096486879) bank116949 +116950 POINT(48.46483176839188 -123.08879678036692) bank116950 +116951 POINT(48.27334554896858 -122.62064616421952) bank116951 +116952 POINT(46.98456835895696 -122.09979447853976) bank116952 +116953 POINT(46.682453730266616 -121.82707282152958) bank116953 +116954 POINT(47.812251994401564 -121.71790245584172) bank116954 +116955 POINT(47.164560830523286 -123.07501053683006) bank116955 +116956 POINT(47.255251552858184 -123.09431555213173) bank116956 +116957 POINT(48.59492023636881 -122.92814509737622) bank116957 +116958 POINT(47.59674068633348 -121.9203322512457) bank116958 +116959 POINT(47.23245288695343 -122.8855080738519) bank116959 +116960 POINT(47.15353289194273 -122.645649350716) bank116960 +116961 POINT(46.742777427450775 -122.80108808923403) bank116961 +116962 POINT(47.262822891269735 -122.52276770059585) bank116962 +116963 POINT(46.850909185762156 -123.08536550761981) bank116963 +116964 POINT(47.01013646220301 -121.80704231705244) bank116964 +116965 POINT(46.95266458484126 -122.39461712042917) bank116965 +116966 POINT(47.92947163881707 -121.56620147901211) bank116966 +116967 POINT(47.827091402368666 -121.98053308021547) bank116967 +116968 POINT(46.769938809268815 -121.78884699843667) bank116968 +116969 POINT(46.68381728462752 -121.80062798863126) bank116969 +116970 POINT(47.173032117696806 -121.54175720597591) bank116970 +116971 POINT(48.42292155584308 -121.84303270392586) bank116971 +116972 POINT(47.90712942970224 -121.94324498469042) bank116972 +116973 POINT(47.815463596644356 -122.57136305775245) bank116973 +116974 POINT(48.1830362500239 -122.36488809534471) bank116974 +116975 POINT(47.44372360547167 -123.17300147057279) bank116975 +116976 POINT(48.528642970149136 -122.21562577077003) bank116976 +116977 POINT(48.215317231593524 -122.20356508883437) bank116977 +116978 POINT(48.2084638469505 -122.7800973188384) bank116978 +116979 POINT(47.78822815061185 -121.55182377330104) bank116979 +116980 POINT(47.912107464970866 -123.19910945223101) bank116980 +116981 POINT(46.85555536753241 -122.53828709179713) bank116981 +116982 POINT(47.24375393738831 -123.20668197951514) bank116982 +116983 POINT(47.48881503920312 -121.74830425192336) bank116983 +116984 POINT(48.54420369807035 -122.29412360627869) bank116984 +116985 POINT(47.55123820502719 -122.56940692021904) bank116985 +116986 POINT(47.22285083386812 -121.59565544892658) bank116986 +116987 POINT(46.95901425828957 -123.2204366296886) bank116987 +116988 POINT(48.13878257458977 -122.50582308227243) bank116988 +116989 POINT(47.530956751972674 -121.67443956361807) bank116989 +116990 POINT(47.400782591824964 -122.2974534016348) bank116990 +116991 POINT(48.42023779651897 -122.04749372782592) bank116991 +116992 POINT(47.818580399760485 -122.82325387550542) bank116992 +116993 POINT(48.29789659222161 -123.18870387299624) bank116993 +116994 POINT(46.68327207445586 -122.16124331709956) bank116994 +116995 POINT(48.02592998179574 -121.39000305368101) bank116995 +116996 POINT(47.68689401692932 -121.69795132250184) bank116996 +116997 POINT(47.33928505128416 -121.9003407108977) bank116997 +116998 POINT(46.99011409793576 -121.92138979979389) bank116998 +116999 POINT(46.94797511022182 -121.77427498574507) bank116999 +117000 POINT(47.040034277558846 -121.63748264093826) bank117000 +117001 POINT(47.43798414118058 -123.09235695683154) bank117001 +117002 POINT(47.9145730333532 -122.61291957741257) bank117002 +117003 POINT(47.39069266671001 -122.64789378274409) bank117003 +117004 POINT(47.449865472600194 -123.08890419906655) bank117004 +117005 POINT(47.02652641900529 -122.13585926162483) bank117005 +117006 POINT(47.61909969803131 -123.17153485714371) bank117006 +117007 POINT(48.516630211251524 -122.9689423127244) bank117007 +117008 POINT(47.371076583348135 -122.16652475235921) bank117008 +117009 POINT(48.12301315871848 -122.67418692170166) bank117009 +117010 POINT(48.260631433979086 -121.52524746293743) bank117010 +117011 POINT(48.09238889249702 -121.38067288828992) bank117011 +117012 POINT(47.30707499552413 -121.65358315477326) bank117012 +117013 POINT(46.733334211952126 -121.86745877892959) bank117013 +117014 POINT(46.843629830383506 -121.61031368464006) bank117014 +117015 POINT(47.70344696998636 -122.56979348350978) bank117015 +117016 POINT(47.965216755221036 -121.63748248432711) bank117016 +117017 POINT(48.393181742788336 -123.17200072344843) bank117017 +117018 POINT(48.0909866096976 -122.0858838086427) bank117018 +117019 POINT(48.17147751777517 -121.34522359463404) bank117019 +117020 POINT(47.163957803504005 -121.75671319099625) bank117020 +117021 POINT(47.69578500795356 -121.49556820061284) bank117021 +117022 POINT(47.96065039297943 -122.39016590126703) bank117022 +117023 POINT(48.39855466371196 -121.62114249579552) bank117023 +117024 POINT(48.37154398278784 -121.63179189317567) bank117024 +117025 POINT(47.30870947398425 -121.43812895049689) bank117025 +117026 POINT(47.52445151961146 -121.75410987675114) bank117026 +117027 POINT(46.79658491922897 -121.60994862942877) bank117027 +117028 POINT(47.41181484277396 -121.87912421273086) bank117028 +117029 POINT(47.3484213085391 -121.71459273428472) bank117029 +117030 POINT(48.03188111303847 -121.90050768594408) bank117030 +117031 POINT(48.57838592369093 -121.35401004670298) bank117031 +117032 POINT(46.890356166153175 -123.06761065156147) bank117032 +117033 POINT(48.20854128276911 -122.48192999014802) bank117033 +117034 POINT(48.57951240533197 -121.42919079800906) bank117034 +117035 POINT(47.06697407502044 -123.32530684763839) bank117035 +117036 POINT(47.81018854573061 -121.34214390308057) bank117036 +117037 POINT(48.594100331962395 -122.9420965177755) bank117037 +117038 POINT(48.430845574279175 -121.5336186520126) bank117038 +117039 POINT(48.437480590672294 -121.88971791119376) bank117039 +117040 POINT(47.41707338796472 -122.89569981726065) bank117040 +117041 POINT(46.80205395828208 -122.53281503046894) bank117041 +117042 POINT(47.664209451955195 -122.747907767306) bank117042 +117043 POINT(47.34459264303752 -123.22789080522445) bank117043 +117044 POINT(48.07745733448105 -122.50363618885514) bank117044 +117045 POINT(47.48986850829422 -121.74167358526516) bank117045 +117046 POINT(48.59144023129378 -121.58992363087857) bank117046 +117047 POINT(47.94674709075213 -121.59262135288297) bank117047 +117048 POINT(48.050113864221544 -122.43731384726304) bank117048 +117049 POINT(48.34105842017872 -122.22815923164252) bank117049 +117050 POINT(47.16465847658839 -121.72034008485096) bank117050 +117051 POINT(48.51862815365212 -122.45890561177981) bank117051 +117052 POINT(47.741139427613824 -121.97711906799246) bank117052 +117053 POINT(48.42009020112245 -121.83402143980788) bank117053 +117054 POINT(47.37465168868712 -122.5493454595241) bank117054 +117055 POINT(47.531886062421826 -122.42138424473903) bank117055 +117056 POINT(48.084524792054204 -122.47710273537662) bank117056 +117057 POINT(47.45012478455803 -121.56048585617305) bank117057 +117058 POINT(47.139656810125956 -122.4973606353745) bank117058 +117059 POINT(48.52569598259075 -122.6622044351879) bank117059 +117060 POINT(47.15564685872061 -121.55738512018722) bank117060 +117061 POINT(47.79620477956069 -121.92493597959266) bank117061 +117062 POINT(47.952587078309 -122.84238139818672) bank117062 +117063 POINT(47.06101231377277 -122.32672525495015) bank117063 +117064 POINT(47.7453618012841 -122.08342780150655) bank117064 +117065 POINT(48.42143105685875 -122.24173075056443) bank117065 +117066 POINT(48.395589458823956 -122.92231639515178) bank117066 +117067 POINT(48.108961329593875 -122.74843249600545) bank117067 +117068 POINT(48.46880423168523 -121.77510959719731) bank117068 +117069 POINT(48.048640821430034 -122.04850840166321) bank117069 +117070 POINT(48.24824837333512 -121.94761027418934) bank117070 +117071 POINT(48.25551141934311 -123.19929872943418) bank117071 +117072 POINT(47.83956176750971 -123.13094578026504) bank117072 +117073 POINT(47.86910172837765 -122.8241961766311) bank117073 +117074 POINT(47.26217721230089 -122.88802811233577) bank117074 +117075 POINT(47.155989120906746 -122.30353123207213) bank117075 +117076 POINT(47.2286797406826 -121.63957274348849) bank117076 +117077 POINT(46.893719865783794 -121.33382779414252) bank117077 +117078 POINT(47.1191028237588 -121.51617077781167) bank117078 +117079 POINT(46.799942852983875 -122.23783171279409) bank117079 +117080 POINT(47.73656818589566 -121.73893703866851) bank117080 +117081 POINT(48.32529722747228 -121.59045488551038) bank117081 +117082 POINT(47.73647870835097 -122.27187747306381) bank117082 +117083 POINT(48.17325054080522 -122.33363705801517) bank117083 +117084 POINT(46.60666987584513 -122.16462357836535) bank117084 +117085 POINT(47.9489785401359 -121.82742576065756) bank117085 +117086 POINT(48.47501951235689 -122.3377215730186) bank117086 +117087 POINT(47.95592884715341 -121.5350908009461) bank117087 +117088 POINT(47.22138024285782 -123.01735744987141) bank117088 +117089 POINT(47.11801169604225 -122.67381622668037) bank117089 +117090 POINT(47.944737326226274 -122.61095518720161) bank117090 +117091 POINT(48.054427675820136 -123.2639612339609) bank117091 +117092 POINT(48.15811895723038 -121.56564866198754) bank117092 +117093 POINT(48.02664681067222 -123.06811545493895) bank117093 +117094 POINT(47.70215353563832 -121.76808026934143) bank117094 +117095 POINT(47.65621005866618 -122.88124714126184) bank117095 +117096 POINT(48.20708230142812 -121.74146607713746) bank117096 +117097 POINT(46.88727517867158 -121.48984347681187) bank117097 +117098 POINT(47.835144342178566 -123.31942339441581) bank117098 +117099 POINT(47.8364472686604 -121.94934597270817) bank117099 +117100 POINT(46.68169087582103 -121.5103051563142) bank117100 +117101 POINT(48.26631811542985 -122.23863609790058) bank117101 +117102 POINT(47.72078700786352 -122.77289314159974) bank117102 +117103 POINT(48.603727730533286 -122.45902015278881) bank117103 +117104 POINT(47.37171712176184 -123.27578315317236) bank117104 +117105 POINT(47.563281735684996 -122.98148630870507) bank117105 +117106 POINT(48.50290883239925 -123.10910558046773) bank117106 +117107 POINT(47.870668080716115 -121.66272073689805) bank117107 +117108 POINT(48.04834115980909 -121.70021427670731) bank117108 +117109 POINT(46.68523495156615 -122.90026018974858) bank117109 +117110 POINT(48.57757898855236 -121.72239971509781) bank117110 +117111 POINT(48.050765074324296 -122.05877008779429) bank117111 +117112 POINT(47.2736645393389 -123.0245046999282) bank117112 +117113 POINT(47.409654508175194 -122.38017662935093) bank117113 +117114 POINT(47.07516480730667 -123.18168700184225) bank117114 +117115 POINT(47.771040050600014 -121.86864924737438) bank117115 +117116 POINT(47.16959764262457 -122.24125024002241) bank117116 +117117 POINT(47.00877178792894 -121.88737369897494) bank117117 +117118 POINT(47.80699224784518 -123.05727681596899) bank117118 +117119 POINT(48.361066051868924 -122.26123287303521) bank117119 +117120 POINT(47.419188993829735 -121.87755155732775) bank117120 +117121 POINT(48.18066633095023 -121.88132332867148) bank117121 +117122 POINT(48.34210086117418 -122.36016744648086) bank117122 +117123 POINT(48.505187444750895 -122.74048997761459) bank117123 +117124 POINT(47.05481425595479 -122.22029554084936) bank117124 +117125 POINT(47.0951361461723 -123.06979792084248) bank117125 +117126 POINT(47.67414339533993 -122.70674112929488) bank117126 +117127 POINT(46.899272503110765 -121.76708874544364) bank117127 +117128 POINT(46.975199114846774 -122.27561380124645) bank117128 +117129 POINT(46.72403630623816 -123.11716591327448) bank117129 +117130 POINT(47.69638241969705 -122.16125014440426) bank117130 +117131 POINT(47.12937459803929 -121.5095771617578) bank117131 +117132 POINT(47.07748787323525 -121.37716010096078) bank117132 +117133 POINT(48.45362812415484 -122.65865461643779) bank117133 +117134 POINT(48.40578584294622 -121.51773662130668) bank117134 +117135 POINT(48.559154084329975 -121.41650293538281) bank117135 +117136 POINT(48.19861468334482 -122.37556886080137) bank117136 +117137 POINT(48.36827376318393 -123.2877391104111) bank117137 +117138 POINT(48.1678865038833 -121.90681736463083) bank117138 +117139 POINT(47.258775967532586 -121.82231661012784) bank117139 +117140 POINT(46.864337565423185 -122.61848776856033) bank117140 +117141 POINT(48.18499049893166 -122.06474573594923) bank117141 +117142 POINT(48.389065319771326 -122.74957081608025) bank117142 +117143 POINT(48.07619912704338 -122.7040362969629) bank117143 +117144 POINT(48.60499255976641 -121.72162001128405) bank117144 +117145 POINT(48.33576593611015 -121.52434301722157) bank117145 +117146 POINT(46.98814016124491 -122.18513498508113) bank117146 +117147 POINT(47.28530558918671 -123.20558229018599) bank117147 +117148 POINT(47.84142433115663 -123.2276765169435) bank117148 +117149 POINT(48.253008247665555 -121.46967024535564) bank117149 +117150 POINT(46.83084770628441 -122.17485389019502) bank117150 +117151 POINT(46.717083408148056 -122.51010278169156) bank117151 +117152 POINT(48.577386017031486 -121.81185400886903) bank117152 +117153 POINT(48.24882904943663 -122.81105780345231) bank117153 +117154 POINT(48.168969354216536 -121.59663840610924) bank117154 +117155 POINT(48.51094104838954 -121.67039925486559) bank117155 +117156 POINT(47.456003248107855 -123.17224847528034) bank117156 +117157 POINT(47.021345326243186 -123.23364960420528) bank117157 +117158 POINT(47.76961877242669 -121.69271085307773) bank117158 +117159 POINT(47.64280727484913 -121.71822625998203) bank117159 +117160 POINT(47.29150409631356 -121.48315829159961) bank117160 +117161 POINT(48.08980592846315 -121.83552007917946) bank117161 +117162 POINT(48.52521754804571 -121.47323191889298) bank117162 +117163 POINT(47.74919699822026 -123.29750683603942) bank117163 +117164 POINT(48.54073516420872 -121.53773416596754) bank117164 +117165 POINT(46.78912175386815 -122.8557950792655) bank117165 +117166 POINT(47.56230548483733 -121.98930352552713) bank117166 +117167 POINT(48.594727671604936 -121.88442722277036) bank117167 +117168 POINT(46.92588495514215 -121.46716098023545) bank117168 +117169 POINT(48.06793157260696 -121.9495894878271) bank117169 +117170 POINT(47.271200725143345 -123.07533131263963) bank117170 +117171 POINT(47.12410431027964 -123.31395700924017) bank117171 +117172 POINT(47.281398931327246 -122.68418113992698) bank117172 +117173 POINT(47.81223738357235 -123.28624389171841) bank117173 +117174 POINT(47.44380544506613 -122.76846730555513) bank117174 +117175 POINT(47.014262815307255 -121.42700896231385) bank117175 +117176 POINT(47.45495873898999 -122.17815033887247) bank117176 +117177 POINT(48.41726800321608 -121.41106981068035) bank117177 +117178 POINT(48.22629986359221 -122.09665443606518) bank117178 +117179 POINT(46.74952016605625 -122.7746524128525) bank117179 +117180 POINT(47.84780298208904 -123.19312528043062) bank117180 +117181 POINT(48.339526578432 -122.23507780531789) bank117181 +117182 POINT(47.88627744230608 -121.54949657021297) bank117182 +117183 POINT(47.02860477025031 -122.99817830925876) bank117183 +117184 POINT(48.12644456437814 -121.41258235557072) bank117184 +117185 POINT(47.29961820099073 -123.26960026526619) bank117185 +117186 POINT(47.73074994364828 -121.97792968730604) bank117186 +117187 POINT(47.34649847356738 -121.92810449398985) bank117187 +117188 POINT(46.62080264090623 -123.07393592413796) bank117188 +117189 POINT(48.59769522103702 -122.30630613100892) bank117189 +117190 POINT(47.15937786819802 -122.24781853406546) bank117190 +117191 POINT(48.34351071280693 -122.6750981878728) bank117191 +117192 POINT(48.30121304074967 -123.09266972568011) bank117192 +117193 POINT(47.90918046758648 -121.35948267523993) bank117193 +117194 POINT(47.29100412074931 -122.94508633384868) bank117194 +117195 POINT(47.45168467035075 -121.84078239777392) bank117195 +117196 POINT(46.619238185816585 -121.550130226827) bank117196 +117197 POINT(47.468631427606304 -122.95300163783754) bank117197 +117198 POINT(48.2051184624433 -122.59542389525588) bank117198 +117199 POINT(48.43646752285393 -122.88373569309798) bank117199 +117200 POINT(48.49998526008247 -121.70972769157726) bank117200 +117201 POINT(48.19711744872742 -121.35613705501272) bank117201 +117202 POINT(47.45068032863584 -122.37079703471427) bank117202 +117203 POINT(48.192240250776365 -121.92373049701202) bank117203 +117204 POINT(48.468667856892054 -121.70099180504386) bank117204 +117205 POINT(47.8636108829588 -121.48226479429906) bank117205 +117206 POINT(48.305640890353025 -121.51852768995981) bank117206 +117207 POINT(47.16419935697652 -121.63922357016176) bank117207 +117208 POINT(48.45613416732441 -122.09510973779336) bank117208 +117209 POINT(48.44581473068662 -121.92194774120736) bank117209 +117210 POINT(47.55706805002599 -121.84866590358119) bank117210 +117211 POINT(47.25709672802003 -121.8677929398273) bank117211 +117212 POINT(47.08044912460487 -121.47693310960825) bank117212 +117213 POINT(47.97324052343087 -121.65438338048772) bank117213 +117214 POINT(47.026392790909625 -122.97871900472428) bank117214 +117215 POINT(47.27753004679974 -121.76485296091948) bank117215 +117216 POINT(48.29224832384523 -122.99763398146546) bank117216 +117217 POINT(47.35992922171874 -121.98812208080011) bank117217 +117218 POINT(47.04339131901062 -122.66275956321589) bank117218 +117219 POINT(46.703552609810416 -122.04414871222994) bank117219 +117220 POINT(46.951851634130634 -121.90721840358692) bank117220 +117221 POINT(48.20040503648801 -122.98254469900198) bank117221 +117222 POINT(46.7443032653376 -122.22818137788424) bank117222 +117223 POINT(48.21838196695148 -123.29224591431445) bank117223 +117224 POINT(48.43880746866398 -121.7301256252267) bank117224 +117225 POINT(47.98568614987811 -122.0891988647184) bank117225 +117226 POINT(47.954709006154424 -122.22177151186096) bank117226 +117227 POINT(48.171427899114825 -121.62469162197746) bank117227 +117228 POINT(48.54858718777536 -122.76762063238394) bank117228 +117229 POINT(47.7450675105121 -122.72317908616219) bank117229 +117230 POINT(46.86928784302753 -122.22374070226843) bank117230 +117231 POINT(48.56348630068331 -122.1762737160375) bank117231 +117232 POINT(47.209669827522944 -122.37867454422872) bank117232 +117233 POINT(47.910491891657095 -122.08156369659694) bank117233 +117234 POINT(47.59281607548011 -122.52156610107836) bank117234 +117235 POINT(46.91134541800321 -122.33279111566061) bank117235 +117236 POINT(46.60686486756538 -123.32450645607005) bank117236 +117237 POINT(47.21671511210791 -122.18374534448232) bank117237 +117238 POINT(46.673040515274316 -121.63856063580019) bank117238 +117239 POINT(46.87315371799318 -123.13451619995315) bank117239 +117240 POINT(47.71886513909317 -121.69719099576523) bank117240 +117241 POINT(48.15708990381923 -121.74270769124986) bank117241 +117242 POINT(48.17162603087935 -122.05160921083386) bank117242 +117243 POINT(46.697032874648265 -122.9477040061016) bank117243 +117244 POINT(47.213873401680615 -121.66679876874512) bank117244 +117245 POINT(47.391339886834835 -122.05385841744723) bank117245 +117246 POINT(47.424902375485324 -122.89977356369475) bank117246 +117247 POINT(47.96482123895836 -122.93043565599356) bank117247 +117248 POINT(46.75151325953667 -121.43324775692014) bank117248 +117249 POINT(48.48929657228493 -121.58851091886304) bank117249 +117250 POINT(47.87789889528948 -121.85985393540204) bank117250 +117251 POINT(48.05750140480501 -123.23099611331719) bank117251 +117252 POINT(48.13760705972208 -121.55999253874303) bank117252 +117253 POINT(46.8111130079923 -123.02121653532755) bank117253 +117254 POINT(48.05956215581318 -122.57906635097021) bank117254 +117255 POINT(46.95131168619774 -122.11145742552527) bank117255 +117256 POINT(48.10684193561761 -121.68403726841032) bank117256 +117257 POINT(46.83432538588489 -122.28329952326138) bank117257 +117258 POINT(47.97774855220056 -122.68102574863207) bank117258 +117259 POINT(48.0427991657914 -122.5002877022704) bank117259 +117260 POINT(46.85245260659622 -122.34593003203291) bank117260 +117261 POINT(47.895149560867615 -121.93976700874286) bank117261 +117262 POINT(46.75865344281052 -123.08811248795944) bank117262 +117263 POINT(48.25844311977651 -122.31799587276694) bank117263 +117264 POINT(47.96226177758709 -121.48172008172911) bank117264 +117265 POINT(47.78225303312835 -121.36315544791022) bank117265 +117266 POINT(48.561550254620315 -121.41604123286966) bank117266 +117267 POINT(47.620960525195784 -121.61032098744484) bank117267 +117268 POINT(48.495586044416946 -122.61689291784508) bank117268 +117269 POINT(47.3089618849373 -122.91490589157387) bank117269 +117270 POINT(48.2281126259065 -122.881983975392) bank117270 +117271 POINT(48.5946214483064 -121.47970751770335) bank117271 +117272 POINT(47.63545643152405 -123.06331119416895) bank117272 +117273 POINT(46.90222375875051 -122.50949570160772) bank117273 +117274 POINT(46.73608710791219 -122.15057454969943) bank117274 +117275 POINT(47.98689527918197 -121.95427451302797) bank117275 +117276 POINT(48.298939531455375 -121.9115873934616) bank117276 +117277 POINT(46.84169788370273 -122.71512344417211) bank117277 +117278 POINT(48.348455100254185 -122.38989327584197) bank117278 +117279 POINT(47.00851491261106 -122.02761708706977) bank117279 +117280 POINT(47.30968025583759 -121.40743028101298) bank117280 +117281 POINT(48.128592399393895 -121.34633228138105) bank117281 +117282 POINT(48.23988158906096 -122.27983022041133) bank117282 +117283 POINT(48.15901664212937 -122.45271854714686) bank117283 +117284 POINT(47.44072984850343 -122.92052605738273) bank117284 +117285 POINT(48.55506274670604 -121.61637234831197) bank117285 +117286 POINT(48.187629848607806 -122.96767780599019) bank117286 +117287 POINT(47.77605249921682 -122.88086406517135) bank117287 +117288 POINT(48.5385361352487 -121.54793859538773) bank117288 +117289 POINT(48.362681989063944 -121.98810675214888) bank117289 +117290 POINT(46.985762229139446 -122.96755928126258) bank117290 +117291 POINT(47.53468071391104 -122.64795224505833) bank117291 +117292 POINT(46.78385854480302 -123.03636503557861) bank117292 +117293 POINT(46.834412368999864 -122.06100265854583) bank117293 +117294 POINT(48.01056993947209 -122.90263433342784) bank117294 +117295 POINT(47.43054017265696 -122.32228954612599) bank117295 +117296 POINT(47.324644389827 -123.02921975473677) bank117296 +117297 POINT(47.0723413107583 -122.91516932134212) bank117297 +117298 POINT(48.49245139656624 -122.41190518926366) bank117298 +117299 POINT(47.06244553440416 -123.04786381300593) bank117299 +117300 POINT(48.10060627852486 -122.93449219070315) bank117300 +117301 POINT(47.24370176230352 -122.18534133164222) bank117301 +117302 POINT(46.6879214426277 -123.30038684334214) bank117302 +117303 POINT(47.294070124529135 -122.2199871858408) bank117303 +117304 POINT(46.79884213042416 -121.45870158895076) bank117304 +117305 POINT(46.61277752456333 -121.47181992082719) bank117305 +117306 POINT(48.12764612300397 -122.77678661237637) bank117306 +117307 POINT(48.04493019168476 -121.64751307361077) bank117307 +117308 POINT(47.90816061944111 -123.24242577997826) bank117308 +117309 POINT(46.66814255456796 -123.0357075394047) bank117309 +117310 POINT(46.80821840528311 -122.45926146574034) bank117310 +117311 POINT(47.51271407307005 -121.58101111717264) bank117311 +117312 POINT(48.40363562417414 -123.01582446286467) bank117312 +117313 POINT(47.40669028686213 -122.38873039989666) bank117313 +117314 POINT(47.89300100092769 -122.33723277612286) bank117314 +117315 POINT(48.6005117473132 -122.98246887972077) bank117315 +117316 POINT(46.82410093075335 -121.40805804622065) bank117316 +117317 POINT(48.301165768237944 -122.61228601501472) bank117317 +117318 POINT(47.262526824768095 -122.50795511652254) bank117318 +117319 POINT(46.98656113755621 -121.33251164940039) bank117319 +117320 POINT(47.43343650617726 -122.30253293642743) bank117320 +117321 POINT(46.752952202219696 -121.61617560442247) bank117321 +117322 POINT(48.365487255201 -121.3377620971018) bank117322 +117323 POINT(47.05141142160909 -123.12621197603909) bank117323 +117324 POINT(48.301858708987346 -121.57843635039728) bank117324 +117325 POINT(48.0902203478541 -122.43422265500651) bank117325 +117326 POINT(47.82884881261976 -122.19609065518951) bank117326 +117327 POINT(48.24404772101762 -123.27485270327357) bank117327 +117328 POINT(47.04547081178775 -121.45960409525728) bank117328 +117329 POINT(47.45632464632899 -122.01906149751422) bank117329 +117330 POINT(47.732998380568304 -123.11777623852839) bank117330 +117331 POINT(47.61411436178393 -123.09633520752) bank117331 +117332 POINT(47.997803071203116 -121.6137382946748) bank117332 +117333 POINT(47.63890236348584 -122.0992196247923) bank117333 +117334 POINT(46.63217036097285 -123.17175558073164) bank117334 +117335 POINT(47.402510770854185 -121.66726610910243) bank117335 +117336 POINT(48.3959906131487 -122.34893357079645) bank117336 +117337 POINT(47.36221446053133 -122.18464072535747) bank117337 +117338 POINT(48.085976793823384 -121.51043140762027) bank117338 +117339 POINT(47.88360226903615 -122.82749905189664) bank117339 +117340 POINT(47.09353604544808 -122.84006892330964) bank117340 +117341 POINT(46.73216519279116 -121.95808238680077) bank117341 +117342 POINT(47.45535381811274 -122.76435991638574) bank117342 +117343 POINT(47.27106533502715 -122.37004795020579) bank117343 +117344 POINT(47.889075437010106 -122.05902160823858) bank117344 +117345 POINT(47.71739205546385 -121.87077388412172) bank117345 +117346 POINT(47.407313837800544 -123.29482186697908) bank117346 +117347 POINT(47.10461504537568 -122.7528543240157) bank117347 +117348 POINT(46.892400626806165 -122.25476705812835) bank117348 +117349 POINT(47.1506006730445 -122.57395290047963) bank117349 +117350 POINT(48.4054200258577 -122.84134336686263) bank117350 +117351 POINT(48.583409143313204 -123.26027574461509) bank117351 +117352 POINT(47.33141836162688 -121.40025105889785) bank117352 +117353 POINT(48.14165330324787 -123.13641342649161) bank117353 +117354 POINT(47.10131707594895 -121.51669471087445) bank117354 +117355 POINT(47.87713815794635 -122.4771328314069) bank117355 +117356 POINT(48.18466300667274 -122.85598350514155) bank117356 +117357 POINT(48.36787378635462 -122.64536419983067) bank117357 +117358 POINT(46.91186409423672 -122.70733989507717) bank117358 +117359 POINT(47.82499570097764 -122.97572533105644) bank117359 +117360 POINT(47.92924424218566 -121.38364629566622) bank117360 +117361 POINT(46.88458475916325 -122.98888729586791) bank117361 +117362 POINT(47.02049820309388 -122.59764456416104) bank117362 +117363 POINT(48.15147597222212 -122.34790492142072) bank117363 +117364 POINT(47.693216646348716 -123.215053011556) bank117364 +117365 POINT(47.11548382332749 -122.43485458004893) bank117365 +117366 POINT(47.38262189915437 -121.63474258514822) bank117366 +117367 POINT(47.683729450108935 -122.88194128450137) bank117367 +117368 POINT(47.54797351689122 -122.23555556732342) bank117368 +117369 POINT(47.40883248056893 -122.68564859433944) bank117369 +117370 POINT(46.650601564740576 -122.1574448927286) bank117370 +117371 POINT(48.344757479455374 -121.72424509585298) bank117371 +117372 POINT(46.832457568046436 -122.55157157870657) bank117372 +117373 POINT(47.72062796016757 -121.58341723427694) bank117373 +117374 POINT(48.036502868197545 -122.5771270317292) bank117374 +117375 POINT(46.95906761168098 -122.2462774562911) bank117375 +117376 POINT(47.804056402839755 -121.36445044501971) bank117376 +117377 POINT(47.30425448959607 -122.60148121790074) bank117377 +117378 POINT(47.732062877086676 -122.46134623331304) bank117378 +117379 POINT(47.913423763549424 -122.40021069290587) bank117379 +117380 POINT(48.0223046832507 -122.02357219737584) bank117380 +117381 POINT(47.44754730039272 -121.8885856858363) bank117381 +117382 POINT(47.08731175833593 -121.37235859504399) bank117382 +117383 POINT(48.33428112574495 -122.68744563663375) bank117383 +117384 POINT(46.94115984027341 -122.7500736281056) bank117384 +117385 POINT(48.248863990437066 -123.06190244441045) bank117385 +117386 POINT(48.59561140088845 -122.17770599053325) bank117386 +117387 POINT(46.76337654108249 -121.55663992301766) bank117387 +117388 POINT(48.29966851759325 -121.56920374393466) bank117388 +117389 POINT(48.46829296639919 -121.53719865789748) bank117389 +117390 POINT(46.62028187010684 -121.60813831887869) bank117390 +117391 POINT(47.420446970984656 -122.38672982301769) bank117391 +117392 POINT(48.456632273032234 -122.37982062379618) bank117392 +117393 POINT(48.024979917967556 -122.24980478393363) bank117393 +117394 POINT(48.4599687841569 -121.97235681464535) bank117394 +117395 POINT(46.984332451009564 -122.6769382866043) bank117395 +117396 POINT(47.23391645688356 -121.95167756360362) bank117396 +117397 POINT(47.029108496620466 -121.94270571141634) bank117397 +117398 POINT(47.30646577138174 -122.4723435224137) bank117398 +117399 POINT(47.33904427353545 -123.16338713450597) bank117399 +117400 POINT(48.51270379525939 -123.26556862381545) bank117400 +117401 POINT(47.510266313825525 -122.51304960301718) bank117401 +117402 POINT(46.816285901507136 -123.22108823348698) bank117402 +117403 POINT(46.882376076909296 -121.49091811609222) bank117403 +117404 POINT(48.35030700042788 -121.54777376597531) bank117404 +117405 POINT(48.449584499048576 -122.86875635868623) bank117405 +117406 POINT(46.862601250080964 -122.9643654403002) bank117406 +117407 POINT(47.1754404276601 -121.35983768005347) bank117407 +117408 POINT(47.746632375487444 -123.13916845426641) bank117408 +117409 POINT(47.5394134684756 -122.09219245245762) bank117409 +117410 POINT(47.592111847933985 -122.9781437752098) bank117410 +117411 POINT(46.71442787186805 -122.03232354900094) bank117411 +117412 POINT(48.273076144872824 -121.84339007952906) bank117412 +117413 POINT(48.36515914827481 -122.2177936840609) bank117413 +117414 POINT(47.08561045499626 -123.03509949743209) bank117414 +117415 POINT(47.60266865248248 -122.85033281426816) bank117415 +117416 POINT(47.12304273955833 -121.69110247884066) bank117416 +117417 POINT(48.22635764316711 -121.63288642531913) bank117417 +117418 POINT(47.61197115930017 -123.1409439529807) bank117418 +117419 POINT(47.61928397180559 -122.48067200195764) bank117419 +117420 POINT(47.126516660575135 -121.6625021186873) bank117420 +117421 POINT(46.78194312166795 -121.8376773885852) bank117421 +117422 POINT(47.05917246990109 -123.07244590352721) bank117422 +117423 POINT(48.43783709952745 -122.7220490518263) bank117423 +117424 POINT(47.94152239746352 -123.25742090018392) bank117424 +117425 POINT(47.68871361426644 -123.3211777397008) bank117425 +117426 POINT(47.07270161581432 -121.89162199915623) bank117426 +117427 POINT(47.15619671857857 -123.01952368478374) bank117427 +117428 POINT(47.85622463636572 -121.5593917236196) bank117428 +117429 POINT(46.62280990080514 -121.91418082191487) bank117429 +117430 POINT(47.694480451839866 -121.88933141952242) bank117430 +117431 POINT(47.52287359549619 -121.49797440227907) bank117431 +117432 POINT(48.542289580637735 -123.06992470237329) bank117432 +117433 POINT(47.96884386725611 -121.43970550808048) bank117433 +117434 POINT(47.49174961532211 -122.66349755612586) bank117434 +117435 POINT(48.23584175314267 -122.5052591683883) bank117435 +117436 POINT(47.9028399505101 -121.8681000083668) bank117436 +117437 POINT(48.51805641958517 -121.62683015554224) bank117437 +117438 POINT(46.63936804031082 -122.16138621367891) bank117438 +117439 POINT(46.92536726056809 -122.07541656434665) bank117439 +117440 POINT(47.755734113245026 -122.32406984797173) bank117440 +117441 POINT(47.7427471633049 -122.24552186507186) bank117441 +117442 POINT(47.32407574881542 -122.06522131152383) bank117442 +117443 POINT(47.41472526089817 -122.44302029245839) bank117443 +117444 POINT(48.44493302263066 -121.99685218103063) bank117444 +117445 POINT(46.84967289912086 -122.93410974024277) bank117445 +117446 POINT(47.93281759024107 -122.6286428147001) bank117446 +117447 POINT(47.65043656195535 -122.69261498587763) bank117447 +117448 POINT(47.96497844779566 -122.64260277966768) bank117448 +117449 POINT(48.06533581863657 -122.72939467654903) bank117449 +117450 POINT(47.59546752249239 -122.1196481041939) bank117450 +117451 POINT(47.988944754124965 -121.98085034648555) bank117451 +117452 POINT(47.37545549982297 -122.77050244027014) bank117452 +117453 POINT(47.464407597846055 -122.61341399150555) bank117453 +117454 POINT(47.366402275262125 -122.79356933079295) bank117454 +117455 POINT(47.50307804327636 -123.04714791285383) bank117455 +117456 POINT(46.74940301853781 -122.65550981317983) bank117456 +117457 POINT(47.23149579843175 -122.49531863272651) bank117457 +117458 POINT(47.57050026553733 -122.63919555976209) bank117458 +117459 POINT(46.806544612562725 -121.80275908285485) bank117459 +117460 POINT(48.266769129681165 -121.52165833853626) bank117460 +117461 POINT(46.739096220961464 -123.22411560638407) bank117461 +117462 POINT(46.73186621098107 -122.9864669861161) bank117462 +117463 POINT(47.54511824986109 -122.55790411993279) bank117463 +117464 POINT(46.677301875047014 -121.92737393478954) bank117464 +117465 POINT(48.07189352974085 -122.64087390585492) bank117465 +117466 POINT(48.552012354360755 -122.33783169877798) bank117466 +117467 POINT(47.64785968499394 -122.48403241751465) bank117467 +117468 POINT(46.76669510458455 -122.70873278348395) bank117468 +117469 POINT(48.44249299718352 -122.51757452812444) bank117469 +117470 POINT(46.766624238918375 -122.00091716934347) bank117470 +117471 POINT(48.47206908970022 -122.23378224507574) bank117471 +117472 POINT(47.85155562621941 -121.77204789814112) bank117472 +117473 POINT(47.89233113472131 -123.01856289856049) bank117473 +117474 POINT(46.790945900210886 -123.19240935344664) bank117474 +117475 POINT(47.624494001154126 -122.63370374087967) bank117475 +117476 POINT(47.448899215917976 -122.76284703508549) bank117476 +117477 POINT(47.12459374876173 -122.12046429589607) bank117477 +117478 POINT(48.469532154476084 -122.77124809059661) bank117478 +117479 POINT(47.11153998265671 -122.65775204892343) bank117479 +117480 POINT(47.79086277832839 -122.6010129334088) bank117480 +117481 POINT(48.153888188277904 -122.1985754873932) bank117481 +117482 POINT(47.207850043257416 -122.7826518786459) bank117482 +117483 POINT(47.88906628665443 -121.58610779832703) bank117483 +117484 POINT(47.37346917224478 -122.89703681950904) bank117484 +117485 POINT(46.6478091359729 -122.78763184880175) bank117485 +117486 POINT(48.2859247764634 -123.10752314040428) bank117486 +117487 POINT(46.956004845631305 -122.22801498530296) bank117487 +117488 POINT(46.85479233973011 -122.49820343466727) bank117488 +117489 POINT(48.47864798838195 -122.19812541285629) bank117489 +117490 POINT(48.5360602923751 -121.48178210604908) bank117490 +117491 POINT(46.73305923006963 -123.00289761525265) bank117491 +117492 POINT(48.161004541827 -121.8710335136583) bank117492 +117493 POINT(48.33911216959507 -121.35437752079034) bank117493 +117494 POINT(47.82076192399996 -122.02849787828741) bank117494 +117495 POINT(47.13286818822121 -122.44299831163939) bank117495 +117496 POINT(47.85870973495972 -122.43041814066704) bank117496 +117497 POINT(46.66483090273598 -121.55818881165287) bank117497 +117498 POINT(47.58500847413886 -122.63979601686911) bank117498 +117499 POINT(47.419494778782656 -122.8126624721827) bank117499 +117500 POINT(47.84779492673928 -121.38524139982577) bank117500 +117501 POINT(48.446687581985024 -121.3502765216815) bank117501 +117502 POINT(48.075534940807955 -123.09034296742148) bank117502 +117503 POINT(48.3153657852533 -122.74653773191184) bank117503 +117504 POINT(46.69648937612469 -122.79522787378333) bank117504 +117505 POINT(47.15424232194308 -121.91503647340195) bank117505 +117506 POINT(46.95871510237521 -121.85404222640734) bank117506 +117507 POINT(47.280418488752055 -123.13865731092736) bank117507 +117508 POINT(48.455356390004106 -121.9561304169839) bank117508 +117509 POINT(46.676508485468275 -122.058749397869) bank117509 +117510 POINT(47.63417691040064 -121.81941324563715) bank117510 +117511 POINT(48.4313702596594 -122.20387919416652) bank117511 +117512 POINT(48.468843614059715 -123.04026492019183) bank117512 +117513 POINT(48.021509647557686 -121.64853188828118) bank117513 +117514 POINT(48.04113144400204 -122.74199963979986) bank117514 +117515 POINT(47.859064342204675 -123.31606955706866) bank117515 +117516 POINT(47.95342447765444 -122.50508016239839) bank117516 +117517 POINT(48.163231899491606 -121.901323624556) bank117517 +117518 POINT(47.52662182297103 -122.11139956009974) bank117518 +117519 POINT(47.5074400285417 -121.91785935053683) bank117519 +117520 POINT(47.71927442104168 -123.1376644952103) bank117520 +117521 POINT(48.16913213867987 -123.2328930578196) bank117521 +117522 POINT(47.49287167526337 -121.82661068710193) bank117522 +117523 POINT(46.99934708340338 -122.43791750639862) bank117523 +117524 POINT(46.73917142822503 -121.94841712658993) bank117524 +117525 POINT(47.95138013147057 -121.50426155971026) bank117525 +117526 POINT(46.647546446800646 -121.44301610214693) bank117526 +117527 POINT(46.76894402835931 -122.459532367518) bank117527 +117528 POINT(46.92202536154873 -123.32613003307931) bank117528 +117529 POINT(48.364484958110154 -121.91938370034337) bank117529 +117530 POINT(47.89055887699477 -121.8425117207459) bank117530 +117531 POINT(48.47179196385956 -121.35763441295128) bank117531 +117532 POINT(47.86772139373582 -122.36251922463711) bank117532 +117533 POINT(48.35881330995005 -121.65272256851092) bank117533 +117534 POINT(48.30653104110392 -121.75194179859197) bank117534 +117535 POINT(47.05198203356823 -122.9979405349645) bank117535 +117536 POINT(48.53943958542027 -122.01990961649065) bank117536 +117537 POINT(47.956476230810026 -122.34882091019966) bank117537 +117538 POINT(48.01495216563089 -122.26993537588528) bank117538 +117539 POINT(48.50095570373535 -122.02289890556595) bank117539 +117540 POINT(46.77887854053552 -121.49953726470538) bank117540 +117541 POINT(48.58101305824465 -122.62493027807051) bank117541 +117542 POINT(47.12226235502194 -122.06465515281712) bank117542 +117543 POINT(47.14795935927396 -121.39505480818035) bank117543 +117544 POINT(48.091325090436996 -122.92248977348694) bank117544 +117545 POINT(48.163269296716294 -121.54484463089814) bank117545 +117546 POINT(47.92969334169086 -122.16685908072617) bank117546 +117547 POINT(46.863395186255175 -122.26857156451945) bank117547 +117548 POINT(48.470725108151505 -121.6324610176033) bank117548 +117549 POINT(47.73222531320831 -121.54811041366672) bank117549 +117550 POINT(47.154818096448246 -122.28576476895005) bank117550 +117551 POINT(47.61257398972793 -121.37210010608703) bank117551 +117552 POINT(48.242708386281066 -122.4642083207013) bank117552 +117553 POINT(47.57441512197348 -122.67467549553405) bank117553 +117554 POINT(46.68050551734349 -122.0449942671974) bank117554 +117555 POINT(46.81532061084556 -122.05799196544172) bank117555 +117556 POINT(46.833468033588936 -122.51698303417899) bank117556 +117557 POINT(47.63917887709437 -121.63668757554608) bank117557 +117558 POINT(46.890405682239226 -122.71236367440733) bank117558 +117559 POINT(47.58761300387067 -122.04831120500491) bank117559 +117560 POINT(47.228952051983114 -123.19478774040401) bank117560 +117561 POINT(47.3676299987946 -122.48184840917554) bank117561 +117562 POINT(47.75571783644109 -123.29677541665296) bank117562 +117563 POINT(46.82889448878107 -121.84927556264793) bank117563 +117564 POINT(48.303621372999054 -121.8327571712758) bank117564 +117565 POINT(46.83388596034004 -123.32720711295653) bank117565 +117566 POINT(46.71105647012075 -122.75363794272539) bank117566 +117567 POINT(48.23742862436758 -121.62748201656908) bank117567 +117568 POINT(48.00965185702054 -122.56538531506378) bank117568 +117569 POINT(46.853684180780846 -122.9234818374784) bank117569 +117570 POINT(47.03347016526268 -122.64966094809697) bank117570 +117571 POINT(47.75422277087832 -121.35259166957371) bank117571 +117572 POINT(47.40475322608218 -123.22597613270455) bank117572 +117573 POINT(47.783457097101575 -123.06645858712012) bank117573 +117574 POINT(48.42379318914119 -121.70379733729749) bank117574 +117575 POINT(48.254875686550875 -122.74821431306778) bank117575 +117576 POINT(48.35155685600272 -121.60414988831161) bank117576 +117577 POINT(46.79296047007947 -122.98928155377853) bank117577 +117578 POINT(46.766993445362765 -122.23768220817105) bank117578 +117579 POINT(46.84014309824505 -121.51854757598524) bank117579 +117580 POINT(48.55880464850216 -122.66641376649191) bank117580 +117581 POINT(48.01966591545956 -122.40010193168725) bank117581 +117582 POINT(47.35413921023296 -122.68750130648958) bank117582 +117583 POINT(47.322779078082746 -122.187230863317) bank117583 +117584 POINT(46.97607704864086 -122.37683220461622) bank117584 +117585 POINT(48.45531751464213 -123.24600262495248) bank117585 +117586 POINT(48.11699714932501 -122.87511807191449) bank117586 +117587 POINT(46.90634989338239 -122.83622774011579) bank117587 +117588 POINT(47.33001505020674 -123.05282935819056) bank117588 +117589 POINT(47.034834964552736 -121.42457787523999) bank117589 +117590 POINT(48.50808495798219 -123.00703528172374) bank117590 +117591 POINT(47.431414284671305 -122.62291271939799) bank117591 +117592 POINT(46.820382401254804 -123.13492228346502) bank117592 +117593 POINT(47.20833187155912 -121.61944368951411) bank117593 +117594 POINT(48.443055496866734 -122.80508467239504) bank117594 +117595 POINT(48.03231429265223 -122.25615342075785) bank117595 +117596 POINT(46.94458151139053 -122.98164953655214) bank117596 +117597 POINT(48.22293015573434 -123.07358776693592) bank117597 +117598 POINT(47.807220874025916 -122.75494187839445) bank117598 +117599 POINT(48.58332024614645 -121.93898584876803) bank117599 +117600 POINT(47.95385952629762 -123.27529533038197) bank117600 +117601 POINT(46.76621790328293 -123.18293516095106) bank117601 +117602 POINT(47.05465105979754 -123.30345660686578) bank117602 +117603 POINT(46.815740634689874 -122.94220330315095) bank117603 +117604 POINT(48.45594031618487 -121.57039506454836) bank117604 +117605 POINT(48.20373261708304 -122.65145941601962) bank117605 +117606 POINT(47.93505354294388 -121.71949885918845) bank117606 +117607 POINT(46.68633141287861 -122.20843064825225) bank117607 +117608 POINT(47.16509083937162 -123.1496661494229) bank117608 +117609 POINT(46.955136467762024 -121.86457763048207) bank117609 +117610 POINT(46.767110608771404 -121.79546958026303) bank117610 +117611 POINT(46.610041068947766 -122.68724301505526) bank117611 +117612 POINT(47.44037102278312 -122.55043950783246) bank117612 +117613 POINT(46.99674211849837 -121.81778025437671) bank117613 +117614 POINT(48.08533951025027 -122.45215134619507) bank117614 +117615 POINT(47.32932757185577 -122.06814911308237) bank117615 +117616 POINT(48.40369719133574 -122.42394637722411) bank117616 +117617 POINT(47.11768297124021 -122.87472127480312) bank117617 +117618 POINT(46.61287134034189 -121.49298143231808) bank117618 +117619 POINT(47.040879862431396 -121.67811619434848) bank117619 +117620 POINT(48.139464038171134 -122.09209284623998) bank117620 +117621 POINT(47.35611568828401 -122.9631662463703) bank117621 +117622 POINT(47.64341980734423 -122.79428965745919) bank117622 +117623 POINT(47.02124406455766 -121.9049914043166) bank117623 +117624 POINT(48.28326646878927 -122.4085989354094) bank117624 +117625 POINT(47.52083256192953 -122.79486565818456) bank117625 +117626 POINT(47.46319439755732 -121.58953324605898) bank117626 +117627 POINT(48.31184381895561 -121.96508623027873) bank117627 +117628 POINT(46.8420535499264 -123.25760374976711) bank117628 +117629 POINT(47.34721207402913 -123.31314545210212) bank117629 +117630 POINT(46.70531942238768 -122.98997505521895) bank117630 +117631 POINT(46.672991893237565 -122.22203883482764) bank117631 +117632 POINT(46.72140371488448 -123.07055240759492) bank117632 +117633 POINT(46.992169855143004 -121.81678265984118) bank117633 +117634 POINT(46.64162341520273 -122.98355403577955) bank117634 +117635 POINT(48.15053005432547 -121.76988660405216) bank117635 +117636 POINT(48.10126661087069 -122.87640496095996) bank117636 +117637 POINT(47.81888805761345 -122.5919407065214) bank117637 +117638 POINT(46.796769408189036 -123.30590591492137) bank117638 +117639 POINT(47.39170173098984 -122.87570707226891) bank117639 +117640 POINT(47.02903941508015 -123.24617805527777) bank117640 +117641 POINT(47.30070231680811 -122.80488543829662) bank117641 +117642 POINT(47.78119922390321 -123.04668274796965) bank117642 +117643 POINT(47.4019879572917 -122.6477847604263) bank117643 +117644 POINT(47.75247099837837 -122.03558778396976) bank117644 +117645 POINT(48.5313434636326 -123.06380989978132) bank117645 +117646 POINT(46.9061091056581 -123.0042048676549) bank117646 +117647 POINT(48.30868530807004 -122.03774716412022) bank117647 +117648 POINT(47.41034308059868 -123.24077214427521) bank117648 +117649 POINT(48.20493677885399 -122.1657829454912) bank117649 +117650 POINT(47.06309846356761 -121.3365648505086) bank117650 +117651 POINT(46.756719457890426 -122.30042309177564) bank117651 +117652 POINT(46.63575110183708 -123.15901577715303) bank117652 +117653 POINT(46.72418761375713 -121.39006386822331) bank117653 +117654 POINT(47.25781830908382 -121.48134142214619) bank117654 +117655 POINT(47.11499424897185 -122.01830408352669) bank117655 +117656 POINT(47.56703920884176 -121.82958744952279) bank117656 +117657 POINT(48.38003195168656 -123.14053196507642) bank117657 +117658 POINT(47.43823075840735 -122.44081265439011) bank117658 +117659 POINT(46.62607774513038 -121.89612592322973) bank117659 +117660 POINT(48.00396204554301 -122.38602691390241) bank117660 +117661 POINT(48.426377274631136 -121.95112228535366) bank117661 +117662 POINT(47.36606262189402 -122.78437419020287) bank117662 +117663 POINT(47.45488208731109 -123.10256995018584) bank117663 +117664 POINT(47.49776674806075 -122.2768351736718) bank117664 +117665 POINT(46.814410291516765 -122.40005958275574) bank117665 +117666 POINT(47.195479367130496 -122.72380213529891) bank117666 +117667 POINT(48.086833775305585 -121.85024293452513) bank117667 +117668 POINT(48.17046952519027 -122.5030117699182) bank117668 +117669 POINT(48.21497383546687 -121.64697357773957) bank117669 +117670 POINT(46.855765742043864 -123.10606866463978) bank117670 +117671 POINT(46.936851241312134 -122.50592122405318) bank117671 +117672 POINT(47.92489647280984 -121.79335499217616) bank117672 +117673 POINT(47.67818995221647 -121.51074645058982) bank117673 +117674 POINT(48.3039351107761 -121.50550061515668) bank117674 +117675 POINT(47.244844774313776 -123.16968224998756) bank117675 +117676 POINT(48.194822339231415 -122.41817115930665) bank117676 +117677 POINT(47.33047809106483 -122.86279142267767) bank117677 +117678 POINT(48.08020570447832 -121.60434799706823) bank117678 +117679 POINT(48.12829010792473 -121.70741253875177) bank117679 +117680 POINT(46.74851726809897 -123.1386021053523) bank117680 +117681 POINT(46.766500440094234 -121.47888445082313) bank117681 +117682 POINT(47.84207160413485 -122.71566012459385) bank117682 +117683 POINT(48.13458091143652 -122.043045809269) bank117683 +117684 POINT(48.20570372252612 -121.36305954114394) bank117684 +117685 POINT(47.79803345280793 -121.71428650572354) bank117685 +117686 POINT(47.76881878188907 -121.77346241576504) bank117686 +117687 POINT(47.43705214613873 -122.78059806280247) bank117687 +117688 POINT(48.11357179407403 -122.45295694701866) bank117688 +117689 POINT(48.396393754197824 -122.85922621644453) bank117689 +117690 POINT(47.35125592701849 -122.73887434384122) bank117690 +117691 POINT(47.62195054699905 -121.83530647856658) bank117691 +117692 POINT(47.46761652533394 -122.49088206091399) bank117692 +117693 POINT(48.37183091429609 -121.76711639838373) bank117693 +117694 POINT(48.278349597265 -121.99264474277483) bank117694 +117695 POINT(47.75500568373809 -122.07467096451381) bank117695 +117696 POINT(48.12696630636756 -121.81918079848971) bank117696 +117697 POINT(47.647947469604 -122.36326985578087) bank117697 +117698 POINT(47.66045494518232 -123.04617082277073) bank117698 +117699 POINT(48.36214496905582 -123.13502892851854) bank117699 +117700 POINT(47.61652739221904 -121.43290067563893) bank117700 +117701 POINT(48.081095065029565 -122.21578672327901) bank117701 +117702 POINT(47.53646018672158 -122.13396667309269) bank117702 +117703 POINT(47.12250835664604 -122.33514437795576) bank117703 +117704 POINT(48.4916703792959 -122.86672242292016) bank117704 +117705 POINT(47.43652306687632 -121.6164639086199) bank117705 +117706 POINT(47.66278497947835 -122.24080677068369) bank117706 +117707 POINT(48.219131374214406 -122.08734615534391) bank117707 +117708 POINT(47.89266546701102 -122.73916210875349) bank117708 +117709 POINT(48.52568574465525 -122.89094210266096) bank117709 +117710 POINT(47.61103857925761 -123.19631140464932) bank117710 +117711 POINT(48.40595961016732 -121.61214842762384) bank117711 +117712 POINT(46.797920493915136 -122.14238511036628) bank117712 +117713 POINT(47.51674560626825 -121.53822332467387) bank117713 +117714 POINT(47.52366835093328 -121.60362555395488) bank117714 +117715 POINT(47.48821038630256 -122.88379109352485) bank117715 +117716 POINT(48.54782277095652 -121.9979448033386) bank117716 +117717 POINT(47.01317023520823 -121.40032537265766) bank117717 +117718 POINT(47.718160777134386 -121.5282584273813) bank117718 +117719 POINT(48.55027158387774 -122.2906662371589) bank117719 +117720 POINT(46.651051990207925 -122.97965879060361) bank117720 +117721 POINT(47.32556889357678 -122.60643020978031) bank117721 +117722 POINT(47.619346215494346 -122.70868247595803) bank117722 +117723 POINT(47.27096954323393 -123.25873051494264) bank117723 +117724 POINT(47.395567093925976 -122.83514612532751) bank117724 +117725 POINT(47.59143777804316 -122.57913898733243) bank117725 +117726 POINT(48.30055442494162 -122.92577161408019) bank117726 +117727 POINT(48.11621131941898 -123.03810275621368) bank117727 +117728 POINT(46.61829816594191 -121.92317635125286) bank117728 +117729 POINT(47.38785335888311 -122.18392501459172) bank117729 +117730 POINT(46.882331648599 -121.84371079034467) bank117730 +117731 POINT(48.33348507872756 -121.43407208793404) bank117731 +117732 POINT(47.83089968350626 -122.2072364863168) bank117732 +117733 POINT(47.797482548085696 -122.17023648207991) bank117733 +117734 POINT(48.495660794002994 -122.7963563831122) bank117734 +117735 POINT(46.79423947118525 -121.67300774772961) bank117735 +117736 POINT(47.69743375090821 -122.57198970733936) bank117736 +117737 POINT(48.12980836125059 -123.22352849660926) bank117737 +117738 POINT(47.2279010334041 -122.55482753195885) bank117738 +117739 POINT(47.87047721876844 -122.78269920847147) bank117739 +117740 POINT(46.842378229983794 -121.99435344028944) bank117740 +117741 POINT(46.6313991817783 -123.22572085774755) bank117741 +117742 POINT(47.336499596201236 -123.12956834851516) bank117742 +117743 POINT(47.27665344439419 -122.24459982773823) bank117743 +117744 POINT(48.06962018643022 -123.27720786346177) bank117744 +117745 POINT(47.01980441580535 -122.54211288796694) bank117745 +117746 POINT(48.143804993656694 -122.21015821763093) bank117746 +117747 POINT(47.711336471353434 -122.05228604979706) bank117747 +117748 POINT(47.63445386334514 -122.95976732763295) bank117748 +117749 POINT(47.5267913684434 -122.31989521249075) bank117749 +117750 POINT(47.18489826540554 -123.29249077163786) bank117750 +117751 POINT(48.066181617813996 -122.4588790128542) bank117751 +117752 POINT(47.35248515455938 -123.30373376243821) bank117752 +117753 POINT(46.910579142835054 -122.21479996092305) bank117753 +117754 POINT(47.09001637871835 -121.68332212190882) bank117754 +117755 POINT(47.286271020155155 -121.89850942822223) bank117755 +117756 POINT(47.498581909052874 -123.0085180501987) bank117756 +117757 POINT(46.929295185407355 -122.27442489658542) bank117757 +117758 POINT(47.579256017377055 -121.69140231272641) bank117758 +117759 POINT(48.45908429304052 -122.03456747993523) bank117759 +117760 POINT(47.26585729057305 -121.44135400151603) bank117760 +117761 POINT(47.24335938507288 -121.79219608881462) bank117761 +117762 POINT(48.196910542753784 -122.2317783879662) bank117762 +117763 POINT(48.386208241214895 -123.28155432457412) bank117763 +117764 POINT(47.67101618231384 -121.8156083643056) bank117764 +117765 POINT(47.121174855088235 -122.61482733575332) bank117765 +117766 POINT(47.50872672347845 -121.98029558534648) bank117766 +117767 POINT(48.39413274137999 -122.00712266748259) bank117767 +117768 POINT(46.847797592736946 -122.70303777801286) bank117768 +117769 POINT(47.82296954337611 -122.23808577204967) bank117769 +117770 POINT(46.91538550300903 -121.53876348079497) bank117770 +117771 POINT(47.000601994065725 -121.3722104047913) bank117771 +117772 POINT(47.34790735961925 -122.2247972886422) bank117772 +117773 POINT(48.4991887468432 -122.66017855738193) bank117773 +117774 POINT(48.20403837354403 -123.06188270500627) bank117774 +117775 POINT(47.22485009102691 -122.39947299432804) bank117775 +117776 POINT(47.40115733303403 -122.23394783727305) bank117776 +117777 POINT(48.50913231061125 -122.81902039903201) bank117777 +117778 POINT(46.65927231951033 -121.684020116671) bank117778 +117779 POINT(47.82108050942047 -122.8158031163345) bank117779 +117780 POINT(47.01286159460694 -123.07317407783277) bank117780 +117781 POINT(48.585339751213645 -122.09781312161125) bank117781 +117782 POINT(48.45628139105115 -122.29872958577177) bank117782 +117783 POINT(48.5084161336461 -122.35967641712097) bank117783 +117784 POINT(48.17170042200336 -122.43277896668715) bank117784 +117785 POINT(48.23005322479784 -122.59859394107309) bank117785 +117786 POINT(47.80153690237134 -122.04881584200831) bank117786 +117787 POINT(47.42774749531941 -123.20372187204183) bank117787 +117788 POINT(46.88406734857874 -122.20858004708035) bank117788 +117789 POINT(48.39923217283197 -121.6119458217286) bank117789 +117790 POINT(46.90793955841036 -121.35682879879076) bank117790 +117791 POINT(47.53244214645423 -122.00648639725623) bank117791 +117792 POINT(47.86466621556224 -122.33887887072449) bank117792 +117793 POINT(48.57460270623311 -123.32461188893569) bank117793 +117794 POINT(46.71278377905521 -122.12691612589123) bank117794 +117795 POINT(46.90938196612245 -121.60158849040097) bank117795 +117796 POINT(48.25685389375264 -122.98879858751472) bank117796 +117797 POINT(47.98050436762348 -122.01083258821131) bank117797 +117798 POINT(48.40887897518717 -121.60585364914641) bank117798 +117799 POINT(47.134772298071134 -123.1062546705722) bank117799 +117800 POINT(48.17884337687079 -122.63174075713401) bank117800 +117801 POINT(47.29805665913868 -121.83713901953571) bank117801 +117802 POINT(48.24068164589405 -122.26875321578713) bank117802 +117803 POINT(46.9034249654103 -122.37822463421767) bank117803 +117804 POINT(47.17911568807866 -122.26185319060244) bank117804 +117805 POINT(46.62751968198565 -122.80621949404866) bank117805 +117806 POINT(47.966450840646935 -122.697185500016) bank117806 +117807 POINT(47.444124620819366 -122.79518520753375) bank117807 +117808 POINT(48.47572233091496 -122.78376963063714) bank117808 +117809 POINT(47.6287942056546 -121.80389933779875) bank117809 +117810 POINT(47.00303466090359 -123.15393586170073) bank117810 +117811 POINT(47.666400211183706 -122.04020804765331) bank117811 +117812 POINT(48.01284040687056 -123.00645669625386) bank117812 +117813 POINT(47.91338199419339 -122.47198415525804) bank117813 +117814 POINT(48.47568627412307 -122.73206743477121) bank117814 +117815 POINT(47.35788183038735 -122.85445582725761) bank117815 +117816 POINT(47.96547565356157 -121.92504089955436) bank117816 +117817 POINT(47.97740209971662 -123.0289877650291) bank117817 +117818 POINT(47.38614895459 -122.92632598856743) bank117818 +117819 POINT(46.878865228928824 -121.79372538751892) bank117819 +117820 POINT(48.38855978169488 -122.30667998697191) bank117820 +117821 POINT(46.73031257427543 -121.8657525868079) bank117821 +117822 POINT(47.30563581923523 -122.59396123283038) bank117822 +117823 POINT(48.10023684865772 -121.34510040160764) bank117823 +117824 POINT(48.039641635937286 -121.93628817422653) bank117824 +117825 POINT(47.20413704145939 -121.53303967103363) bank117825 +117826 POINT(48.1159060654849 -121.79304903214242) bank117826 +117827 POINT(48.133355689931214 -123.17299187543445) bank117827 +117828 POINT(47.38542646044586 -122.33899408443015) bank117828 +117829 POINT(46.657188799354905 -121.9846948997926) bank117829 +117830 POINT(46.648744121980585 -122.42282219214218) bank117830 +117831 POINT(46.64414709605246 -122.75812531698794) bank117831 +117832 POINT(46.997898723252845 -121.5418701165357) bank117832 +117833 POINT(48.19480045921385 -121.55541059278622) bank117833 +117834 POINT(48.308188397362194 -122.67012389228694) bank117834 +117835 POINT(46.78791951572069 -122.69504079344593) bank117835 +117836 POINT(47.84693655313733 -122.26223895130848) bank117836 +117837 POINT(48.05510495589495 -122.03019879770298) bank117837 +117838 POINT(48.22374573626029 -121.56627299672417) bank117838 +117839 POINT(48.292576025261795 -121.5823138247545) bank117839 +117840 POINT(46.86340815761455 -123.14762921035202) bank117840 +117841 POINT(47.129335423563965 -122.4434494087299) bank117841 +117842 POINT(47.300484403895865 -122.2362984358679) bank117842 +117843 POINT(47.92522233874933 -121.49940917927701) bank117843 +117844 POINT(48.58269466563741 -122.61493087944186) bank117844 +117845 POINT(48.582989372041304 -121.98213526166026) bank117845 +117846 POINT(47.69169160451341 -121.54509805262862) bank117846 +117847 POINT(48.45438443350437 -123.21920286173622) bank117847 +117848 POINT(47.71151305573562 -121.71933995965593) bank117848 +117849 POINT(48.20251023411408 -122.69429533699021) bank117849 +117850 POINT(46.98609849060152 -122.67316607777873) bank117850 +117851 POINT(48.039975627881084 -122.88031142255984) bank117851 +117852 POINT(47.62601707166395 -121.99139585553536) bank117852 +117853 POINT(46.64037081466017 -122.70177463432714) bank117853 +117854 POINT(46.85770356729724 -121.34546636973559) bank117854 +117855 POINT(47.90204420633771 -121.53160178671078) bank117855 +117856 POINT(48.49239067163603 -121.44069393828887) bank117856 +117857 POINT(48.58127782413603 -122.94366796011755) bank117857 +117858 POINT(47.25383153520476 -122.39748184449606) bank117858 +117859 POINT(48.58709159248448 -122.27930262961154) bank117859 +117860 POINT(47.28218114687347 -122.21140315463852) bank117860 +117861 POINT(48.56195194410162 -122.50878341285993) bank117861 +117862 POINT(46.625328660388085 -121.67612769450884) bank117862 +117863 POINT(47.802739942425795 -122.08071520708742) bank117863 +117864 POINT(47.02623626750636 -123.32283500081829) bank117864 +117865 POINT(47.193240516552315 -123.09252376943502) bank117865 +117866 POINT(48.00912556232232 -122.52162469409686) bank117866 +117867 POINT(48.2213354544539 -123.17824820973667) bank117867 +117868 POINT(47.115534684126025 -122.5349673704979) bank117868 +117869 POINT(46.82435859255839 -122.98001424080026) bank117869 +117870 POINT(46.785735616089724 -122.73333503804649) bank117870 +117871 POINT(47.91861571245811 -122.00842621369635) bank117871 +117872 POINT(46.95275804942085 -121.65045234421633) bank117872 +117873 POINT(48.08466278861354 -122.98932664693451) bank117873 +117874 POINT(47.63651291685571 -123.31914101732147) bank117874 +117875 POINT(47.725885749990766 -123.02012602231395) bank117875 +117876 POINT(47.421818499581875 -122.81774031299172) bank117876 +117877 POINT(47.89513934947616 -122.32066251069575) bank117877 +117878 POINT(47.18179520568899 -122.10982366924448) bank117878 +117879 POINT(48.00831515819352 -122.19054846611085) bank117879 +117880 POINT(47.35322349743049 -121.36033431190658) bank117880 +117881 POINT(47.6906720721767 -121.96323456419746) bank117881 +117882 POINT(48.49528939318885 -121.83805699737741) bank117882 +117883 POINT(46.765659951913676 -121.72986202733014) bank117883 +117884 POINT(46.62965263495459 -122.63559659640003) bank117884 +117885 POINT(48.291388474217094 -121.76555375045892) bank117885 +117886 POINT(47.803133667734606 -123.08941137640714) bank117886 +117887 POINT(47.0145142315709 -121.53656544721217) bank117887 +117888 POINT(46.64417561988793 -122.79927605815709) bank117888 +117889 POINT(48.19225062111031 -122.52838707984614) bank117889 +117890 POINT(47.43593079837031 -121.6315267725341) bank117890 +117891 POINT(47.32119174231792 -121.88672397679963) bank117891 +117892 POINT(48.58879189763576 -122.46904385069465) bank117892 +117893 POINT(47.3414498733112 -122.09501340148991) bank117893 +117894 POINT(47.98511530510699 -122.5069202014735) bank117894 +117895 POINT(47.86418889558181 -122.51486607217467) bank117895 +117896 POINT(47.22850705305571 -121.6172720310057) bank117896 +117897 POINT(46.80399111139652 -121.4211967911096) bank117897 +117898 POINT(46.61250595831948 -121.6976952879412) bank117898 +117899 POINT(47.68618881124208 -122.43387461316404) bank117899 +117900 POINT(47.72157149629718 -121.54680224659954) bank117900 +117901 POINT(47.12924261807071 -123.13530499685845) bank117901 +117902 POINT(47.798315712191794 -121.92028262191798) bank117902 +117903 POINT(48.444231994086685 -121.87667080329999) bank117903 +117904 POINT(47.19754130917811 -122.52631556460146) bank117904 +117905 POINT(46.96950443907406 -121.53387468723142) bank117905 +117906 POINT(48.38795168356417 -121.76545077524248) bank117906 +117907 POINT(47.61360911559103 -122.78086720045792) bank117907 +117908 POINT(46.700664056559575 -123.21805014109557) bank117908 +117909 POINT(48.599289622358285 -122.65405076815658) bank117909 +117910 POINT(48.43690027387487 -122.94220707804256) bank117910 +117911 POINT(48.31950656800428 -121.63989470785256) bank117911 +117912 POINT(46.761957559638056 -123.00529417273438) bank117912 +117913 POINT(47.96491705328676 -122.3760534551986) bank117913 +117914 POINT(46.726801953394435 -121.73537733893433) bank117914 +117915 POINT(47.127458146794176 -121.88238616053198) bank117915 +117916 POINT(47.51320923510539 -121.64924220575887) bank117916 +117917 POINT(47.59239978872685 -121.3946359461618) bank117917 +117918 POINT(47.04022273074829 -121.78916095120177) bank117918 +117919 POINT(48.364774826643796 -122.52427166583726) bank117919 +117920 POINT(47.24781754377578 -122.33013616484767) bank117920 +117921 POINT(47.984175650151045 -121.86203350216572) bank117921 +117922 POINT(47.745882246610904 -122.5187597127821) bank117922 +117923 POINT(46.81660525340064 -122.78567554030386) bank117923 +117924 POINT(46.94927872451104 -122.27210547078337) bank117924 +117925 POINT(46.82605106423852 -123.10437223164521) bank117925 +117926 POINT(47.84325598545027 -123.22843795514342) bank117926 +117927 POINT(47.790616867355155 -123.28109999410577) bank117927 +117928 POINT(47.76684394964922 -121.36788408310281) bank117928 +117929 POINT(48.04635565896692 -123.17953714065406) bank117929 +117930 POINT(47.62603824132438 -121.98654961792685) bank117930 +117931 POINT(47.11598111565289 -121.40151081364213) bank117931 +117932 POINT(48.50167692428285 -123.29358723817352) bank117932 +117933 POINT(48.12267846974342 -121.77936018044234) bank117933 +117934 POINT(47.502788072745815 -122.25040410714881) bank117934 +117935 POINT(48.02902320281525 -122.20044760413228) bank117935 +117936 POINT(47.15753916557596 -121.57170492777057) bank117936 +117937 POINT(48.16690108567496 -121.57883045174168) bank117937 +117938 POINT(46.91934443217926 -121.88180897289585) bank117938 +117939 POINT(47.57534523587233 -122.49180066080139) bank117939 +117940 POINT(47.94319359393973 -122.95164810679206) bank117940 +117941 POINT(47.01588802297778 -122.48430358404671) bank117941 +117942 POINT(47.76430625728838 -122.99536219182467) bank117942 +117943 POINT(47.17238226988723 -121.4998821913656) bank117943 +117944 POINT(46.700224599234815 -122.85103980923142) bank117944 +117945 POINT(47.57844289336883 -123.01881229363536) bank117945 +117946 POINT(47.628519669796425 -122.08323966559803) bank117946 +117947 POINT(48.07743559747015 -123.02557580780122) bank117947 +117948 POINT(47.27914709398549 -122.01294965781327) bank117948 +117949 POINT(47.83150878048288 -123.3288518633589) bank117949 +117950 POINT(46.789110462672866 -122.57263090608124) bank117950 +117951 POINT(47.30877673304503 -123.22999065414551) bank117951 +117952 POINT(46.97726908825416 -121.59431762435581) bank117952 +117953 POINT(47.880685806622395 -121.76411184332837) bank117953 +117954 POINT(47.60498286805902 -122.36370807728751) bank117954 +117955 POINT(47.38916397554936 -122.55573054036711) bank117955 +117956 POINT(47.88394790078893 -122.38257262787484) bank117956 +117957 POINT(46.84079569807622 -122.06465080206281) bank117957 +117958 POINT(46.71543645946651 -121.50383016543726) bank117958 +117959 POINT(48.28610632664052 -122.55615745297337) bank117959 +117960 POINT(47.74345537301123 -121.87360202735252) bank117960 +117961 POINT(48.228039532345456 -122.66250236575051) bank117961 +117962 POINT(47.235318549172895 -122.04107834356527) bank117962 +117963 POINT(48.07487650354399 -123.26203922415546) bank117963 +117964 POINT(46.65184969158457 -121.7232084370361) bank117964 +117965 POINT(48.1588240399897 -122.58623577553263) bank117965 +117966 POINT(47.17577789329034 -122.50485582372278) bank117966 +117967 POINT(46.92208318934527 -123.25828611254882) bank117967 +117968 POINT(47.20409160725847 -122.43632620169556) bank117968 +117969 POINT(47.86954004222565 -122.57631928265089) bank117969 +117970 POINT(47.72812951130546 -122.0047302503815) bank117970 +117971 POINT(48.274617381913316 -123.06570547383589) bank117971 +117972 POINT(47.0656618430075 -121.4613228405974) bank117972 +117973 POINT(46.991907122001535 -121.37690987649064) bank117973 +117974 POINT(47.85923859839626 -122.12097248955608) bank117974 +117975 POINT(48.4384867707117 -122.39182664625828) bank117975 +117976 POINT(48.5231654634419 -122.61801993191342) bank117976 +117977 POINT(47.01633700237462 -122.94202920463565) bank117977 +117978 POINT(47.52908113420728 -122.6235888868141) bank117978 +117979 POINT(47.273571927642074 -123.20158327149025) bank117979 +117980 POINT(47.913121618183 -123.29460376535015) bank117980 +117981 POINT(47.35330624854153 -121.57754142644207) bank117981 +117982 POINT(46.65819435980194 -122.79468005708526) bank117982 +117983 POINT(47.53958428000924 -122.14086268255437) bank117983 +117984 POINT(48.232713410571854 -121.80679993587226) bank117984 +117985 POINT(47.85391520264868 -122.51221248506943) bank117985 +117986 POINT(48.23032733671827 -122.72297873661272) bank117986 +117987 POINT(46.768374582931564 -121.53087964812794) bank117987 +117988 POINT(46.7167222719857 -123.18364952974821) bank117988 +117989 POINT(47.555435427173684 -123.10796741372492) bank117989 +117990 POINT(46.6728624795696 -121.77882141468233) bank117990 +117991 POINT(46.65013288663891 -122.627990608411) bank117991 +117992 POINT(48.22224722036819 -122.78531292368109) bank117992 +117993 POINT(48.424317374464586 -122.61140625859512) bank117993 +117994 POINT(47.11875972839233 -122.68017902189742) bank117994 +117995 POINT(46.625608421607566 -122.68352052195763) bank117995 +117996 POINT(48.50941329291724 -122.21418798638788) bank117996 +117997 POINT(47.6523816342692 -122.72807087850217) bank117997 +117998 POINT(47.63150484829953 -123.27913203827374) bank117998 +117999 POINT(47.091942947433694 -122.93453217266907) bank117999 +118000 POINT(47.1618042295063 -121.62685090176184) bank118000 +118001 POINT(47.33169386699809 -121.45549804504579) bank118001 +118002 POINT(48.053864511154586 -123.22425759002158) bank118002 +118003 POINT(48.202982220481395 -121.50940404201549) bank118003 +118004 POINT(48.54744253146432 -121.68650131707393) bank118004 +118005 POINT(46.884457826439 -122.16035848507539) bank118005 +118006 POINT(48.506312170096486 -122.09207756931983) bank118006 +118007 POINT(48.05581838030346 -123.1030377424967) bank118007 +118008 POINT(47.21423762126543 -122.37359545894915) bank118008 +118009 POINT(47.70036286300031 -121.97544788523729) bank118009 +118010 POINT(48.25044839201716 -122.15262790669676) bank118010 +118011 POINT(46.910571660690294 -122.16090843802252) bank118011 +118012 POINT(47.560146926015754 -122.57535662516514) bank118012 +118013 POINT(47.23292443108931 -122.30837176605564) bank118013 +118014 POINT(47.321015191412286 -122.60641237870459) bank118014 +118015 POINT(47.74554285492523 -121.76425416136131) bank118015 +118016 POINT(47.31571909132936 -122.55251886917775) bank118016 +118017 POINT(47.24368523530849 -123.08180025594987) bank118017 +118018 POINT(48.16937311213249 -121.72172364214738) bank118018 +118019 POINT(46.93422755649679 -122.41665681281185) bank118019 +118020 POINT(46.750322705480905 -121.36961492983407) bank118020 +118021 POINT(47.114811685642096 -121.8104738084859) bank118021 +118022 POINT(47.62488297042205 -122.03036856270468) bank118022 +118023 POINT(48.03616977898786 -122.14089440116093) bank118023 +118024 POINT(46.68307255825098 -122.92771348447069) bank118024 +118025 POINT(48.20589642631636 -121.63275894816233) bank118025 +118026 POINT(47.197796531031905 -123.17974460966275) bank118026 +118027 POINT(47.60868200890126 -122.31924331543092) bank118027 +118028 POINT(47.2241808083374 -121.91861482742038) bank118028 +118029 POINT(48.19134103701376 -122.34664795190514) bank118029 +118030 POINT(48.262263940337235 -123.04350067960434) bank118030 +118031 POINT(48.37839242420247 -122.10079069078857) bank118031 +118032 POINT(47.79426365912159 -121.55591685835755) bank118032 +118033 POINT(46.808860124138064 -121.39518601331693) bank118033 +118034 POINT(47.88590950155537 -122.02026148269353) bank118034 +118035 POINT(46.93637114054374 -122.27955882154788) bank118035 +118036 POINT(48.557768302739454 -122.27907138155024) bank118036 +118037 POINT(46.66092551709131 -122.29512003497935) bank118037 +118038 POINT(46.9198443047747 -121.7017078220922) bank118038 +118039 POINT(48.548357976476055 -121.35095742828217) bank118039 +118040 POINT(48.22784539803912 -121.63417273263921) bank118040 +118041 POINT(47.95634452251641 -121.79146002186607) bank118041 +118042 POINT(47.72713526561486 -122.31430408076804) bank118042 +118043 POINT(47.25720403506888 -121.84614143757496) bank118043 +118044 POINT(47.17323060836216 -122.3853080168338) bank118044 +118045 POINT(47.91151272163919 -123.10600312127839) bank118045 +118046 POINT(46.63286368473378 -122.88480516494046) bank118046 +118047 POINT(46.63971163518617 -122.56308659875648) bank118047 +118048 POINT(46.94709628640139 -122.92020170378173) bank118048 +118049 POINT(48.505448087509684 -122.66348309186081) bank118049 +118050 POINT(46.97226272307261 -122.60722933995928) bank118050 +118051 POINT(47.274457556401174 -121.52035056720403) bank118051 +118052 POINT(47.92806862200101 -122.1102314152244) bank118052 +118053 POINT(47.710034051751 -122.42892661463969) bank118053 +118054 POINT(47.53284603663869 -122.84754393585126) bank118054 +118055 POINT(47.419678877035075 -123.06131962325966) bank118055 +118056 POINT(48.525311944770245 -123.24984671592834) bank118056 +118057 POINT(47.56609635014527 -122.17029414637244) bank118057 +118058 POINT(47.05014251022142 -121.51382103533129) bank118058 +118059 POINT(47.35445745085685 -122.29113870090454) bank118059 +118060 POINT(48.34285704199585 -123.2988985869854) bank118060 +118061 POINT(47.178857094064924 -122.23452909540747) bank118061 +118062 POINT(48.48947250046304 -121.35314880685745) bank118062 +118063 POINT(48.30586586473211 -121.96119219037365) bank118063 +118064 POINT(47.65599781563598 -122.97574238505484) bank118064 +118065 POINT(47.30905286396126 -123.30026092441494) bank118065 +118066 POINT(48.501698401629305 -121.49630237091283) bank118066 +118067 POINT(48.54262916329851 -123.27005907395176) bank118067 +118068 POINT(47.3369519473005 -123.17855228721963) bank118068 +118069 POINT(48.56397180933524 -121.93863126669734) bank118069 +118070 POINT(48.3291378401909 -121.7484482095213) bank118070 +118071 POINT(47.79734621472112 -121.69989232031651) bank118071 +118072 POINT(47.81498720150162 -121.73151467849159) bank118072 +118073 POINT(47.18732272791449 -123.1886532660394) bank118073 +118074 POINT(47.605825460581414 -121.67039018838773) bank118074 +118075 POINT(48.07117547356227 -121.64514179636156) bank118075 +118076 POINT(46.88481863028067 -122.30530065503143) bank118076 +118077 POINT(47.97789776538665 -123.10877564019451) bank118077 +118078 POINT(47.20138311430574 -123.2680929324492) bank118078 +118079 POINT(47.15656957412199 -122.74778693817025) bank118079 +118080 POINT(48.54350388808012 -122.09929978551827) bank118080 +118081 POINT(48.16029837101715 -121.43162508722973) bank118081 +118082 POINT(48.34783180628213 -123.10877397332737) bank118082 +118083 POINT(47.06893239767128 -121.59302107387664) bank118083 +118084 POINT(47.96207556977392 -121.58017952558627) bank118084 +118085 POINT(48.294154346641335 -123.11813163659286) bank118085 +118086 POINT(48.238674008388394 -121.84760704850301) bank118086 +118087 POINT(47.51165966376178 -121.79721761485192) bank118087 +118088 POINT(47.119562057091194 -123.0653644025882) bank118088 +118089 POINT(47.2191075711051 -121.52488512520776) bank118089 +118090 POINT(48.056709028410275 -122.46774149699534) bank118090 +118091 POINT(47.74115520729171 -123.05393490409688) bank118091 +118092 POINT(47.45696059510042 -122.48527771518758) bank118092 +118093 POINT(46.687354264836586 -122.18289223788422) bank118093 +118094 POINT(48.204590866860734 -121.36629641051371) bank118094 +118095 POINT(47.139572418711154 -121.40197910946178) bank118095 +118096 POINT(46.890939813527254 -122.78335671605137) bank118096 +118097 POINT(47.862159977266465 -121.7184693841722) bank118097 +118098 POINT(48.36895494225205 -122.02258683973214) bank118098 +118099 POINT(46.780415960477356 -123.03407252753875) bank118099 +118100 POINT(47.645388750422576 -122.26843776093357) bank118100 +118101 POINT(46.7733917809079 -122.28972706328814) bank118101 +118102 POINT(47.52396644392922 -123.18349858481434) bank118102 +118103 POINT(47.60807560236219 -121.34036812138498) bank118103 +118104 POINT(46.830345070846946 -123.24158665335761) bank118104 +118105 POINT(46.674860053006284 -121.8360127815171) bank118105 +118106 POINT(47.261422809521335 -122.78450436479748) bank118106 +118107 POINT(48.55792419747577 -122.09657058496205) bank118107 +118108 POINT(47.05121407153452 -122.33446742918979) bank118108 +118109 POINT(48.398043285852424 -121.7153891188942) bank118109 +118110 POINT(48.3631446477921 -122.91683208053867) bank118110 +118111 POINT(47.8119382482663 -121.91261581179906) bank118111 +118112 POINT(47.224193585082794 -121.47550876170311) bank118112 +118113 POINT(47.37664653367883 -122.80870095809018) bank118113 +118114 POINT(47.64771878753972 -121.70048176302087) bank118114 +118115 POINT(48.161341371797135 -121.44539483447467) bank118115 +118116 POINT(47.16876619836093 -122.86318741215179) bank118116 +118117 POINT(46.94441761068061 -123.15731992697123) bank118117 +118118 POINT(48.502854919384845 -123.254138987048) bank118118 +118119 POINT(46.935589273645704 -122.98968620418677) bank118119 +118120 POINT(46.93494497606182 -121.91634783942078) bank118120 +118121 POINT(48.18330561716416 -121.4655888188233) bank118121 +118122 POINT(47.76585161443077 -121.42008411693486) bank118122 +118123 POINT(48.10203899569179 -121.68946836058169) bank118123 +118124 POINT(48.55414475001795 -123.28476367938752) bank118124 +118125 POINT(47.828210396065536 -123.01264551687908) bank118125 +118126 POINT(47.42185197862017 -121.41898674554668) bank118126 +118127 POINT(48.402086283115615 -122.88839331105954) bank118127 +118128 POINT(47.32787800425554 -121.66953947289866) bank118128 +118129 POINT(48.08950431070973 -122.49584801538367) bank118129 +118130 POINT(46.75461753260564 -123.09396567297286) bank118130 +118131 POINT(48.3121835872377 -122.55206583355508) bank118131 +118132 POINT(46.70412416663052 -121.35957197360554) bank118132 +118133 POINT(48.585856009935796 -121.81597434749318) bank118133 +118134 POINT(48.26069391717509 -121.48853397955429) bank118134 +118135 POINT(48.40497869585301 -123.12391406107245) bank118135 +118136 POINT(47.09019309539006 -121.70380506666872) bank118136 +118137 POINT(48.165466071580376 -123.29680833094599) bank118137 +118138 POINT(47.74712174459941 -122.4447935094901) bank118138 +118139 POINT(47.22566369767291 -123.26244946081627) bank118139 +118140 POINT(48.04340369569117 -122.64503415174303) bank118140 +118141 POINT(46.99650527107612 -122.89811875928807) bank118141 +118142 POINT(48.139032312867556 -121.91256273143507) bank118142 +118143 POINT(47.240937472946705 -122.02406477121119) bank118143 +118144 POINT(47.1133427995708 -121.58971471748522) bank118144 +118145 POINT(48.41730923505482 -121.66570803843666) bank118145 +118146 POINT(46.847548261126605 -121.34095520385456) bank118146 +118147 POINT(47.18699296795354 -122.96113865398422) bank118147 +118148 POINT(47.95269214607765 -122.52805695416322) bank118148 +118149 POINT(47.15243128513764 -122.65368877773403) bank118149 +118150 POINT(47.19257961059784 -122.5951698100939) bank118150 +118151 POINT(47.59369075256279 -123.31018283964704) bank118151 +118152 POINT(47.818238729381015 -122.5576059959043) bank118152 +118153 POINT(48.12464275378235 -122.30074314320667) bank118153 +118154 POINT(47.353188736934676 -122.63750265888144) bank118154 +118155 POINT(48.294364329974336 -123.27089486306214) bank118155 +118156 POINT(47.28987208953919 -121.50511251433824) bank118156 +118157 POINT(47.155850425508895 -121.54094446620607) bank118157 +118158 POINT(47.690445460122575 -122.68439902722615) bank118158 +118159 POINT(46.97024410696419 -123.00616135810274) bank118159 +118160 POINT(47.03247727753354 -122.46337978983874) bank118160 +118161 POINT(48.052055051253944 -122.11377882916243) bank118161 +118162 POINT(47.518925832782635 -121.33613973081444) bank118162 +118163 POINT(46.725284574488924 -122.2093715852281) bank118163 +118164 POINT(47.445999329550915 -122.27620941403) bank118164 +118165 POINT(47.53298038392494 -121.71381591720716) bank118165 +118166 POINT(47.84399744758399 -122.3705734565318) bank118166 +118167 POINT(46.995963488091185 -123.01863975107693) bank118167 +118168 POINT(47.04743166390741 -123.1088685156397) bank118168 +118169 POINT(48.07618157896607 -122.45290485831445) bank118169 +118170 POINT(47.35023870751134 -121.8394395832218) bank118170 +118171 POINT(47.49553945124851 -122.805551765307) bank118171 +118172 POINT(48.359265560625396 -121.8871122613515) bank118172 +118173 POINT(47.534308810878876 -121.62403972883025) bank118173 +118174 POINT(48.35548762547724 -121.60022006198452) bank118174 +118175 POINT(48.5652662429192 -123.31497782407853) bank118175 +118176 POINT(47.01011601333814 -121.75351176832238) bank118176 +118177 POINT(46.77718324230126 -122.77424729530414) bank118177 +118178 POINT(47.176909290390356 -121.98933854596515) bank118178 +118179 POINT(47.100809354235395 -121.37792167408972) bank118179 +118180 POINT(47.89969324322673 -122.6366428573963) bank118180 +118181 POINT(46.73538653981695 -121.88739184913845) bank118181 +118182 POINT(47.78521889855882 -123.06073383137878) bank118182 +118183 POINT(48.08647675877319 -122.44621910879809) bank118183 +118184 POINT(47.8751237777231 -121.72564921057955) bank118184 +118185 POINT(46.71389359632303 -122.3423173138654) bank118185 +118186 POINT(46.96716646962316 -123.06064265682022) bank118186 +118187 POINT(47.610562090462345 -122.24592226415673) bank118187 +118188 POINT(47.032658262692166 -122.29088001655532) bank118188 +118189 POINT(48.38982502457775 -121.93435030776723) bank118189 +118190 POINT(48.36035969275182 -123.20266570933212) bank118190 +118191 POINT(47.20772842802635 -122.12573741683515) bank118191 +118192 POINT(46.94256578632984 -122.45879647011064) bank118192 +118193 POINT(48.00675266225191 -122.28401464329448) bank118193 +118194 POINT(48.542054074450384 -121.69434545657286) bank118194 +118195 POINT(47.46123275148449 -122.36579909788118) bank118195 +118196 POINT(46.97876535441968 -122.2763543762335) bank118196 +118197 POINT(47.78719394456052 -122.64141121237782) bank118197 +118198 POINT(47.35908580982194 -121.80463099719192) bank118198 +118199 POINT(47.189894800228316 -121.72433543578315) bank118199 +118200 POINT(48.13046710255132 -121.70702926670536) bank118200 +118201 POINT(46.69887360471853 -122.86870069046667) bank118201 +118202 POINT(46.90253073752194 -122.37725326140026) bank118202 +118203 POINT(48.399195237727135 -122.11825863688487) bank118203 +118204 POINT(46.980067634858464 -121.59517600803902) bank118204 +118205 POINT(48.23369192208829 -121.4857683950244) bank118205 +118206 POINT(48.413865913538885 -122.75320616357483) bank118206 +118207 POINT(47.71824918584482 -122.04553392915653) bank118207 +118208 POINT(47.78594040043366 -121.50799057490974) bank118208 +118209 POINT(47.2437524939241 -122.93769152321705) bank118209 +118210 POINT(47.684790507164365 -122.04355534877222) bank118210 +118211 POINT(47.207718496195056 -121.39376356575694) bank118211 +118212 POINT(46.60941895900036 -122.2947598513725) bank118212 +118213 POINT(47.211590035545605 -123.32027866650326) bank118213 +118214 POINT(46.76360824303081 -122.60231192965034) bank118214 +118215 POINT(47.266948704679145 -122.35159249403229) bank118215 +118216 POINT(46.89013225918723 -121.3343477935867) bank118216 +118217 POINT(47.01688034428869 -122.2664661128413) bank118217 +118218 POINT(46.62320747428596 -122.31472739576928) bank118218 +118219 POINT(48.492320149096805 -123.09734287463108) bank118219 +118220 POINT(48.46377168846238 -122.49883222934594) bank118220 +118221 POINT(48.14969523742304 -122.6527214495901) bank118221 +118222 POINT(47.30355573263316 -122.25487745522057) bank118222 +118223 POINT(48.40450039926359 -121.92348721120702) bank118223 +118224 POINT(47.04261665109914 -123.17333720965898) bank118224 +118225 POINT(47.899225715473655 -122.83745264163126) bank118225 +118226 POINT(48.38227055815542 -123.05482941446928) bank118226 +118227 POINT(47.198618805431025 -121.50922888671928) bank118227 +118228 POINT(48.60352010282508 -122.0511492835742) bank118228 +118229 POINT(47.32176910279847 -123.04591623913974) bank118229 +118230 POINT(47.8456919883417 -121.73680452403326) bank118230 +118231 POINT(48.11735279284954 -123.11769642553598) bank118231 +118232 POINT(46.62208959343724 -122.52709794604732) bank118232 +118233 POINT(47.46208817394872 -122.74531218300928) bank118233 +118234 POINT(47.4389286078522 -122.62326003811825) bank118234 +118235 POINT(48.167372951416745 -121.3600804692914) bank118235 +118236 POINT(48.36477866278314 -122.43751485852604) bank118236 +118237 POINT(47.95449138846281 -122.87033071850313) bank118237 +118238 POINT(46.71054592511237 -122.91429547678193) bank118238 +118239 POINT(47.72666307064088 -123.18498439600178) bank118239 +118240 POINT(47.102533331832824 -121.69824165004952) bank118240 +118241 POINT(48.09674325321763 -122.7038763652863) bank118241 +118242 POINT(48.19625636536332 -122.90121540222263) bank118242 +118243 POINT(48.32353859044386 -123.2346737063665) bank118243 +118244 POINT(47.32762291974157 -122.03609112324843) bank118244 +118245 POINT(47.777915391927536 -122.34784629776777) bank118245 +118246 POINT(48.43814128710002 -122.21828219301987) bank118246 +118247 POINT(46.67691317702623 -121.51613749002324) bank118247 +118248 POINT(47.54259232341412 -121.44181069611994) bank118248 +118249 POINT(46.62316185508323 -122.25645973562528) bank118249 +118250 POINT(48.15475172005214 -121.39720169646836) bank118250 +118251 POINT(48.18843351255382 -122.92240180858484) bank118251 +118252 POINT(48.41774112651909 -121.92678565196081) bank118252 +118253 POINT(47.351961997020176 -122.07359602285793) bank118253 +118254 POINT(47.16717600815616 -121.43863537401799) bank118254 +118255 POINT(48.30484662018901 -122.59442876652047) bank118255 +118256 POINT(47.20836080832346 -123.0602188184272) bank118256 +118257 POINT(46.62247934373649 -122.06700158110242) bank118257 +118258 POINT(48.20786372942048 -122.5467894040038) bank118258 +118259 POINT(47.75807337379389 -121.53330727269626) bank118259 +118260 POINT(47.21888717016587 -122.30629881679185) bank118260 +118261 POINT(47.42437411879539 -122.15317571596198) bank118261 +118262 POINT(47.740799095212175 -121.61072946810978) bank118262 +118263 POINT(48.58878026566991 -121.64394834391058) bank118263 +118264 POINT(48.416095093299774 -122.05235466047519) bank118264 +118265 POINT(47.76483020680525 -121.54668830726715) bank118265 +118266 POINT(47.9746506138273 -121.45821303422748) bank118266 +118267 POINT(47.52861509850129 -123.19560154043103) bank118267 +118268 POINT(47.592520644601 -121.76770496343094) bank118268 +118269 POINT(47.720260223116824 -123.31121834357953) bank118269 +118270 POINT(47.3075666048291 -121.76852395010607) bank118270 +118271 POINT(46.6227622168935 -122.99612381416812) bank118271 +118272 POINT(47.097435704299976 -121.90019879702193) bank118272 +118273 POINT(48.594249878623515 -123.16617886424737) bank118273 +118274 POINT(47.50646513003548 -122.87209712611966) bank118274 +118275 POINT(48.28111370368068 -122.53819030804654) bank118275 +118276 POINT(48.263738414607694 -122.44509975206712) bank118276 +118277 POINT(47.82740230219438 -122.14814397968988) bank118277 +118278 POINT(46.94127627366303 -122.82553530398128) bank118278 +118279 POINT(48.02017296462028 -121.34789892671874) bank118279 +118280 POINT(48.350579006244544 -122.97064670034094) bank118280 +118281 POINT(46.88640999534473 -122.1389803701422) bank118281 +118282 POINT(48.57725199801901 -122.72848436736872) bank118282 +118283 POINT(47.383144117592956 -121.8169482953124) bank118283 +118284 POINT(46.905226321687984 -123.32407318776669) bank118284 +118285 POINT(47.97857223466422 -122.90871481692123) bank118285 +118286 POINT(46.96719626174036 -121.82158832641214) bank118286 +118287 POINT(47.460261709484165 -122.54658346410127) bank118287 +118288 POINT(47.40564357626026 -122.54862620879686) bank118288 +118289 POINT(47.49513649723246 -122.85661576626218) bank118289 +118290 POINT(48.60451299051748 -123.1852014782702) bank118290 +118291 POINT(46.806243746368835 -121.58809797118766) bank118291 +118292 POINT(47.38658030739747 -123.21751166430009) bank118292 +118293 POINT(48.017814346777435 -122.88481489752263) bank118293 +118294 POINT(48.26310717980441 -121.47849904312612) bank118294 +118295 POINT(47.906911220791315 -121.79292476273544) bank118295 +118296 POINT(48.07800368721725 -123.26118439052175) bank118296 +118297 POINT(46.622367049548515 -122.89635786329946) bank118297 +118298 POINT(48.40875466364044 -122.69279844180109) bank118298 +118299 POINT(47.35966141928218 -123.11746691010862) bank118299 +118300 POINT(48.1672776667526 -122.84856681170162) bank118300 +118301 POINT(48.23768561585287 -122.42410304201775) bank118301 +118302 POINT(47.67240081681485 -122.41693511578976) bank118302 +118303 POINT(46.981094419160314 -122.48570124654252) bank118303 +118304 POINT(47.239503011213905 -122.80498555367488) bank118304 +118305 POINT(48.31343491853983 -122.21063009303583) bank118305 +118306 POINT(48.35988564587737 -122.89006599722259) bank118306 +118307 POINT(48.33458642125985 -121.45147680954771) bank118307 +118308 POINT(48.342279345911884 -121.38200186133903) bank118308 +118309 POINT(48.096675627275836 -122.5791335279285) bank118309 +118310 POINT(46.80328887084471 -122.61393399751398) bank118310 +118311 POINT(48.076434679146374 -122.41142258151844) bank118311 +118312 POINT(47.31504713104803 -121.78697805567361) bank118312 +118313 POINT(46.7104346070854 -122.59579270729152) bank118313 +118314 POINT(47.43028186689622 -121.54678798986056) bank118314 +118315 POINT(46.774685719567685 -122.43963443006338) bank118315 +118316 POINT(47.87620230123148 -121.54929467815838) bank118316 +118317 POINT(47.0962810788582 -123.23123938126989) bank118317 +118318 POINT(47.43227909372393 -121.95476087335675) bank118318 +118319 POINT(47.22901396209061 -122.92089288014441) bank118319 +118320 POINT(48.54684694710739 -122.05936179732282) bank118320 +118321 POINT(48.209955950051025 -121.57485876522597) bank118321 +118322 POINT(47.69885964141797 -122.63707966144318) bank118322 +118323 POINT(47.80274685272404 -122.14014680698382) bank118323 +118324 POINT(48.23280078669895 -122.93737180528339) bank118324 +118325 POINT(48.53798705118476 -122.74464717764351) bank118325 +118326 POINT(48.280045442846195 -123.16368790102817) bank118326 +118327 POINT(48.58855054585862 -122.5602540586212) bank118327 +118328 POINT(47.79528146318164 -122.42513485112502) bank118328 +118329 POINT(47.57072036565627 -123.16229280794529) bank118329 +118330 POINT(47.37975381340611 -123.0243055607928) bank118330 +118331 POINT(46.865281174535745 -121.64382880250027) bank118331 +118332 POINT(47.64867836274206 -121.38778236626628) bank118332 +118333 POINT(47.670914531938095 -123.22620155869039) bank118333 +118334 POINT(47.64631969740978 -123.18621399669637) bank118334 +118335 POINT(48.17468157372952 -121.68879010566977) bank118335 +118336 POINT(48.04517571046765 -121.43749950102611) bank118336 +118337 POINT(48.44361536833712 -123.24500850811533) bank118337 +118338 POINT(48.47678256342996 -122.8105809457873) bank118338 +118339 POINT(47.74715117319913 -122.75971217383984) bank118339 +118340 POINT(47.67251980526928 -121.99459446458508) bank118340 +118341 POINT(47.16862967627218 -121.35514515959677) bank118341 +118342 POINT(48.379823587087 -122.14782795649366) bank118342 +118343 POINT(48.31580048040929 -122.84149821722839) bank118343 +118344 POINT(47.87456571585362 -121.72608111202031) bank118344 +118345 POINT(47.205273214312896 -122.54914720862284) bank118345 +118346 POINT(46.65263616344779 -123.03292199872641) bank118346 +118347 POINT(47.31808891745479 -121.47739966195645) bank118347 +118348 POINT(47.984352940514846 -122.25566767146348) bank118348 +118349 POINT(47.26697893397007 -122.29083128947224) bank118349 +118350 POINT(48.53214260588695 -121.90266008995775) bank118350 +118351 POINT(46.82523491650676 -122.01874400771966) bank118351 +118352 POINT(46.9137838688979 -122.49440692730586) bank118352 +118353 POINT(46.669174307377624 -121.50782480251418) bank118353 +118354 POINT(47.40503563653113 -121.47129494596385) bank118354 +118355 POINT(46.674819580684385 -121.57859307531793) bank118355 +118356 POINT(47.051800382586826 -121.587940795794) bank118356 +118357 POINT(46.69671773310664 -122.78196128836194) bank118357 +118358 POINT(48.155737105045645 -121.99959551464333) bank118358 +118359 POINT(47.405967585092895 -121.89468943956136) bank118359 +118360 POINT(47.6970315199298 -121.51413307049809) bank118360 +118361 POINT(47.18975953981832 -121.92928844508302) bank118361 +118362 POINT(47.12274673902557 -121.37316302072472) bank118362 +118363 POINT(47.03990462171834 -123.10005233682784) bank118363 +118364 POINT(46.61319217346171 -122.81770126007692) bank118364 +118365 POINT(48.076754329056804 -121.72153949450268) bank118365 +118366 POINT(47.29488512210443 -122.21491212615877) bank118366 +118367 POINT(47.812180455582336 -121.7707367925037) bank118367 +118368 POINT(46.81821816896724 -122.12738139409518) bank118368 +118369 POINT(47.601646830313065 -122.32956047737763) bank118369 +118370 POINT(47.31562290167907 -122.87653881297908) bank118370 +118371 POINT(47.49899332010764 -121.95246441045045) bank118371 +118372 POINT(48.57196842407238 -122.32762516056873) bank118372 +118373 POINT(47.64122990404815 -122.29481082957923) bank118373 +118374 POINT(46.83980728504104 -122.98664622045352) bank118374 +118375 POINT(47.06225903635557 -122.652625758212) bank118375 +118376 POINT(48.43460996592309 -121.57253387365574) bank118376 +118377 POINT(47.68332002831127 -122.86918977807194) bank118377 +118378 POINT(47.33200201408001 -122.20904304326878) bank118378 +118379 POINT(48.58823860649536 -122.59443294370347) bank118379 +118380 POINT(46.7181529331983 -122.65125577414166) bank118380 +118381 POINT(47.422372646814416 -121.42868268249934) bank118381 +118382 POINT(48.346389010253624 -122.172843728083) bank118382 +118383 POINT(47.69094405106037 -123.01524206731013) bank118383 +118384 POINT(47.18145735984775 -121.80700677730817) bank118384 +118385 POINT(48.42967683769633 -123.23281431477692) bank118385 +118386 POINT(48.468176215383046 -122.82082355721043) bank118386 +118387 POINT(47.73478919673515 -122.72994003125618) bank118387 +118388 POINT(46.994400982229095 -122.1925832163236) bank118388 +118389 POINT(46.91335033717261 -121.84289536250009) bank118389 +118390 POINT(46.97620207546283 -123.02401547476221) bank118390 +118391 POINT(47.945850348089834 -122.18741364912317) bank118391 +118392 POINT(47.78463273413746 -121.95615646689102) bank118392 +118393 POINT(46.81859555347112 -122.4995861869246) bank118393 +118394 POINT(47.890373292845084 -123.03774825019727) bank118394 +118395 POINT(47.69108298732388 -123.03777103249807) bank118395 +118396 POINT(48.1298378846051 -121.50287646950571) bank118396 +118397 POINT(48.43733780449613 -121.74450256864611) bank118397 +118398 POINT(47.50032361996515 -122.02322706021862) bank118398 +118399 POINT(47.744661263501236 -121.90335938319939) bank118399 +118400 POINT(47.07356187986984 -123.15614289905284) bank118400 +118401 POINT(48.219706056174076 -122.78887241504641) bank118401 +118402 POINT(48.14121749081519 -123.05434620157831) bank118402 +118403 POINT(46.87670257324403 -122.68048230937306) bank118403 +118404 POINT(47.37160794455486 -121.42185351193216) bank118404 +118405 POINT(48.562923837245854 -123.15530148147978) bank118405 +118406 POINT(48.4645116954296 -121.90622157823745) bank118406 +118407 POINT(48.0963138217568 -121.99937145770127) bank118407 +118408 POINT(46.877445427595184 -123.23839222165047) bank118408 +118409 POINT(48.28204190319512 -122.62328157900359) bank118409 +118410 POINT(46.62840197841199 -121.9813143777508) bank118410 +118411 POINT(47.24675174633163 -122.78241188596525) bank118411 +118412 POINT(47.97381576870492 -121.92894204209776) bank118412 +118413 POINT(48.11740544212796 -122.26578538816676) bank118413 +118414 POINT(48.12059993443087 -122.16036819349486) bank118414 +118415 POINT(47.43790005934154 -122.65887279538552) bank118415 +118416 POINT(47.93512926290212 -121.61440896013559) bank118416 +118417 POINT(46.938694350337464 -122.30105036316098) bank118417 +118418 POINT(47.529818143538726 -121.99938286555275) bank118418 +118419 POINT(47.30573169718517 -121.78014216604062) bank118419 +118420 POINT(47.10264776436376 -121.65576657281441) bank118420 +118421 POINT(47.046835679765685 -122.25492300077917) bank118421 +118422 POINT(47.12956010363157 -122.6487304201253) bank118422 +118423 POINT(47.39296920669694 -122.50533541478106) bank118423 +118424 POINT(48.21747686484418 -122.7593779031014) bank118424 +118425 POINT(48.001382660176226 -122.39818474799378) bank118425 +118426 POINT(47.05396517052278 -121.59310679843905) bank118426 +118427 POINT(46.916729396581786 -121.67278524308607) bank118427 +118428 POINT(47.88603924458418 -122.05549108488552) bank118428 +118429 POINT(47.37626728709802 -123.26069261720686) bank118429 +118430 POINT(47.53761807304046 -123.18273466529898) bank118430 +118431 POINT(46.814783801744056 -122.32935310416266) bank118431 +118432 POINT(47.920085967942754 -122.11650836671856) bank118432 +118433 POINT(46.755526083312844 -121.40757683656945) bank118433 +118434 POINT(47.19530651678279 -122.39516117595699) bank118434 +118435 POINT(46.92336273267351 -121.7665171855834) bank118435 +118436 POINT(48.05613821267833 -121.48033083513486) bank118436 +118437 POINT(48.3142267967144 -122.50222503900476) bank118437 +118438 POINT(47.65399695145434 -122.98894451816453) bank118438 +118439 POINT(46.9247166446588 -123.25440102327052) bank118439 +118440 POINT(47.94197275419816 -122.14852661039367) bank118440 +118441 POINT(47.23314853176649 -123.13395942748888) bank118441 +118442 POINT(46.667424507812186 -122.10866668631688) bank118442 +118443 POINT(46.67625977951004 -123.05055355885696) bank118443 +118444 POINT(47.713807624650066 -123.12949148331062) bank118444 +118445 POINT(46.65050982352201 -121.48768919347366) bank118445 +118446 POINT(47.2477910912409 -121.8715723901903) bank118446 +118447 POINT(46.801991471153826 -122.82169976264375) bank118447 +118448 POINT(47.13482582522626 -122.91827725821955) bank118448 +118449 POINT(47.76427043863287 -122.19996002639853) bank118449 +118450 POINT(47.045962223957474 -122.89005374176419) bank118450 +118451 POINT(47.84685885876931 -122.58796350143386) bank118451 +118452 POINT(47.88976032228388 -122.93296945587906) bank118452 +118453 POINT(47.942018715499074 -121.44718620396033) bank118453 +118454 POINT(48.58018845912057 -122.76468832432799) bank118454 +118455 POINT(47.05540614341293 -122.7083610681665) bank118455 +118456 POINT(46.79135066801439 -122.19566453800921) bank118456 +118457 POINT(48.14777639416831 -121.60227807686364) bank118457 +118458 POINT(46.73205581422066 -122.17142414325923) bank118458 +118459 POINT(46.88377657098463 -122.76110331809116) bank118459 +118460 POINT(46.7868521962224 -121.91002185369673) bank118460 +118461 POINT(48.298609150767874 -122.81124221389668) bank118461 +118462 POINT(47.05775483573003 -121.98259555199296) bank118462 +118463 POINT(48.19653569659887 -123.09219630626279) bank118463 +118464 POINT(47.86643970725388 -122.0736792055888) bank118464 +118465 POINT(47.93013614292388 -122.02774816644758) bank118465 +118466 POINT(47.326527544308334 -121.6152482633636) bank118466 +118467 POINT(48.136159228083045 -123.29182972031258) bank118467 +118468 POINT(47.82558107315874 -121.78442971792506) bank118468 +118469 POINT(46.71130950011481 -121.87308524752278) bank118469 +118470 POINT(46.67753595248544 -121.74191267524898) bank118470 +118471 POINT(48.26712843321917 -121.90705943731831) bank118471 +118472 POINT(47.61116832356893 -122.21987819810806) bank118472 +118473 POINT(48.573086444697964 -122.73758259754133) bank118473 +118474 POINT(48.52793033529157 -122.94295357158788) bank118474 +118475 POINT(47.65620180150672 -122.31905482931835) bank118475 +118476 POINT(47.57556216395086 -121.88840812033031) bank118476 +118477 POINT(48.199697877452785 -122.14924826042895) bank118477 +118478 POINT(47.60418663644818 -122.90856911609693) bank118478 +118479 POINT(48.428224385609816 -121.97284320082973) bank118479 +118480 POINT(48.27086302398259 -123.2477952915149) bank118480 +118481 POINT(47.32168793711228 -123.25569759056583) bank118481 +118482 POINT(47.87736174174645 -122.26012241961082) bank118482 +118483 POINT(47.92946529321808 -121.51188039729557) bank118483 +118484 POINT(47.427269962634284 -122.76998720860202) bank118484 +118485 POINT(47.74566086977675 -122.1292065951906) bank118485 +118486 POINT(46.94702910438693 -123.22862003507412) bank118486 +118487 POINT(47.9012202007941 -122.94531499612235) bank118487 +118488 POINT(46.94593138697536 -122.50655222026049) bank118488 +118489 POINT(47.35558776898425 -122.95621527054061) bank118489 +118490 POINT(46.62893338993516 -122.19168601790105) bank118490 +118491 POINT(47.11154342389378 -123.19416602384678) bank118491 +118492 POINT(47.33863942667182 -121.55648324483205) bank118492 +118493 POINT(47.924208153338526 -122.34305746860343) bank118493 +118494 POINT(46.612951367415235 -122.26972093633374) bank118494 +118495 POINT(48.18830802896546 -121.51138920485026) bank118495 +118496 POINT(46.816725112178204 -123.07616660984938) bank118496 +118497 POINT(46.77642896936461 -123.05445693653927) bank118497 +118498 POINT(46.853198679898235 -121.44975970075521) bank118498 +118499 POINT(46.83814227514839 -122.03057373453615) bank118499 +118500 POINT(47.50938283021993 -121.96890567043064) bank118500 +118501 POINT(48.48702716398655 -122.89077517923879) bank118501 +118502 POINT(46.8939310794069 -122.24687249995533) bank118502 +118503 POINT(46.9702742998529 -122.58056031763944) bank118503 +118504 POINT(47.26397795322185 -123.26569110151026) bank118504 +118505 POINT(47.736170659181916 -121.60163238546231) bank118505 +118506 POINT(47.2877468950981 -123.3034924566274) bank118506 +118507 POINT(47.50781614557679 -122.1453391039962) bank118507 +118508 POINT(47.209248787820584 -121.72991032857225) bank118508 +118509 POINT(47.0177375673162 -122.7709779750191) bank118509 +118510 POINT(47.235642152918295 -122.5677212655222) bank118510 +118511 POINT(47.307928995742444 -122.30879088220408) bank118511 +118512 POINT(48.05045094912623 -121.3550515516718) bank118512 +118513 POINT(47.0993841149173 -122.28859878556555) bank118513 +118514 POINT(48.27282509616331 -122.28302139443613) bank118514 +118515 POINT(47.51262420806021 -121.93069862717039) bank118515 +118516 POINT(47.42608877398509 -121.49087375544217) bank118516 +118517 POINT(48.13423737803376 -121.4965977609376) bank118517 +118518 POINT(47.87417459365313 -123.23095255945611) bank118518 +118519 POINT(47.637968048602495 -121.60984767859918) bank118519 +118520 POINT(46.772879939182296 -123.07592848924304) bank118520 +118521 POINT(47.968769995363644 -122.11880368544423) bank118521 +118522 POINT(47.45916161925812 -122.51289730818499) bank118522 +118523 POINT(48.241940264233286 -123.31103175714635) bank118523 +118524 POINT(48.49871332875723 -123.07256308886812) bank118524 +118525 POINT(47.360090652799826 -123.2146875764578) bank118525 +118526 POINT(48.42260835772491 -122.92363723820108) bank118526 +118527 POINT(48.167613143203624 -122.27722335992148) bank118527 +118528 POINT(47.86623287761218 -122.53727079411759) bank118528 +118529 POINT(48.561981606466226 -121.90409681013335) bank118529 +118530 POINT(48.2893010148995 -122.43811166759957) bank118530 +118531 POINT(47.587024818748986 -122.13416198544891) bank118531 +118532 POINT(47.190197098801065 -122.70873993789257) bank118532 +118533 POINT(48.57000373354935 -121.98213587646212) bank118533 +118534 POINT(47.24062218628433 -121.52111363156126) bank118534 +118535 POINT(47.44070445299799 -122.8014526555129) bank118535 +118536 POINT(47.86881304229894 -123.11934124792494) bank118536 +118537 POINT(48.45386207839615 -122.228420594051) bank118537 +118538 POINT(46.95923089456471 -121.38377842838362) bank118538 +118539 POINT(47.91826874818103 -121.58359630677177) bank118539 +118540 POINT(47.68772898897729 -122.14685962237162) bank118540 +118541 POINT(48.386222251221604 -121.55495828488344) bank118541 +118542 POINT(46.67905152125787 -122.59671606561848) bank118542 +118543 POINT(48.06365546738618 -122.47722373218808) bank118543 +118544 POINT(46.87002277801974 -122.19460827297208) bank118544 +118545 POINT(46.63997637445405 -121.86668191242212) bank118545 +118546 POINT(48.1180740747318 -123.26935784261904) bank118546 +118547 POINT(47.1611918659287 -123.32774846542425) bank118547 +118548 POINT(48.136400890831624 -121.43176809279265) bank118548 +118549 POINT(48.128851863445085 -122.4430095534496) bank118549 +118550 POINT(48.26677249119433 -121.35228513782998) bank118550 +118551 POINT(47.371907492249974 -121.40290605111467) bank118551 +118552 POINT(48.02282340995558 -123.18948395576014) bank118552 +118553 POINT(46.838336035195596 -122.64716307705) bank118553 +118554 POINT(47.39756513499219 -123.07251037277051) bank118554 +118555 POINT(46.7024803926624 -122.07054298756934) bank118555 +118556 POINT(47.453250269499335 -122.85282619640448) bank118556 +118557 POINT(47.97048413035012 -122.79886115016224) bank118557 +118558 POINT(48.51137657767635 -122.42840907275476) bank118558 +118559 POINT(48.12052758995475 -122.27499449558253) bank118559 +118560 POINT(47.19735737600615 -123.07362892225369) bank118560 +118561 POINT(47.18632379619558 -123.22866407635875) bank118561 +118562 POINT(47.89470490353109 -122.63650062488641) bank118562 +118563 POINT(47.27271816734173 -121.49061750518925) bank118563 +118564 POINT(46.72085439713902 -123.25080089066695) bank118564 +118565 POINT(47.92067928716019 -121.51889878846255) bank118565 +118566 POINT(48.32488419576538 -121.88999152667081) bank118566 +118567 POINT(46.64528301766622 -123.2998999120113) bank118567 +118568 POINT(47.80887574078398 -122.5594294290104) bank118568 +118569 POINT(48.15777294968465 -121.58683398393995) bank118569 +118570 POINT(47.31902377916585 -122.72938902037745) bank118570 +118571 POINT(48.59147213089802 -122.1316419576711) bank118571 +118572 POINT(48.22080306238259 -122.02646964426275) bank118572 +118573 POINT(47.06890258256968 -122.70038653297273) bank118573 +118574 POINT(47.8818985494009 -121.52043813393588) bank118574 +118575 POINT(47.46451277713398 -122.60068688424157) bank118575 +118576 POINT(47.831228668191976 -122.97022827188299) bank118576 +118577 POINT(47.93844547869368 -122.20776907397696) bank118577 +118578 POINT(47.79551092912706 -121.72452553151872) bank118578 +118579 POINT(48.37443217606377 -121.88741642781828) bank118579 +118580 POINT(46.62841522113388 -121.72430231874712) bank118580 +118581 POINT(48.08071665590727 -121.70104341205372) bank118581 +118582 POINT(47.29480492478377 -123.19977961740399) bank118582 +118583 POINT(48.26274552519316 -122.46505825663864) bank118583 +118584 POINT(47.094208442457756 -123.25929462259606) bank118584 +118585 POINT(46.88875900187404 -121.97908225163253) bank118585 +118586 POINT(48.18052829793591 -122.19767536821988) bank118586 +118587 POINT(48.28091826461973 -122.24865500535608) bank118587 +118588 POINT(47.67296293077162 -123.24519165364566) bank118588 +118589 POINT(47.02041212248605 -121.9576556823013) bank118589 +118590 POINT(47.10229441560737 -121.34242416767445) bank118590 +118591 POINT(47.0130935110556 -121.41091915796916) bank118591 +118592 POINT(46.680199642724666 -122.29137534637411) bank118592 +118593 POINT(46.99100457805449 -123.11868194689934) bank118593 +118594 POINT(48.01003262267666 -122.64890448300406) bank118594 +118595 POINT(48.397128979803384 -122.0043032841004) bank118595 +118596 POINT(46.974904222090544 -122.97776750636235) bank118596 +118597 POINT(47.344759693002786 -123.02352080752942) bank118597 +118598 POINT(47.58374106918808 -123.26226495653022) bank118598 +118599 POINT(48.44721294031651 -122.93561297946306) bank118599 +118600 POINT(47.2741126010569 -121.73631910726554) bank118600 +118601 POINT(46.784456897586075 -121.87183380782989) bank118601 +118602 POINT(48.51146617458199 -123.27983390583401) bank118602 +118603 POINT(48.529046102443814 -123.32648110888492) bank118603 +118604 POINT(48.58089232623441 -121.65285898666427) bank118604 +118605 POINT(48.50140305835347 -121.88086482338738) bank118605 +118606 POINT(47.99755227114279 -121.45452850211105) bank118606 +118607 POINT(48.17065037443713 -121.93158713413727) bank118607 +118608 POINT(48.38119794496181 -122.57759492637979) bank118608 +118609 POINT(47.32558737187168 -122.4307745270876) bank118609 +118610 POINT(47.149217330218484 -123.31000299145988) bank118610 +118611 POINT(47.9730185577739 -123.1163791087937) bank118611 +118612 POINT(47.214415004384335 -122.48553590269648) bank118612 +118613 POINT(47.86232293470907 -122.39791880520625) bank118613 +118614 POINT(46.789626868267405 -122.7542497860238) bank118614 +118615 POINT(46.637022216397924 -122.11614489575751) bank118615 +118616 POINT(47.87547869649081 -121.42186120288106) bank118616 +118617 POINT(47.64697459440131 -122.38902558524951) bank118617 +118618 POINT(48.41415343509381 -122.62745749167675) bank118618 +118619 POINT(48.099552001063486 -122.6148654885808) bank118619 +118620 POINT(48.214108848028616 -122.42197109124976) bank118620 +118621 POINT(47.52408397448501 -121.59791528046726) bank118621 +118622 POINT(47.47101629547298 -122.0249309754701) bank118622 +118623 POINT(47.40615567540888 -121.82575045035009) bank118623 +118624 POINT(48.00521420149153 -122.29165391408527) bank118624 +118625 POINT(47.61562515492433 -122.0736442266091) bank118625 +118626 POINT(47.03614233572093 -123.33031916929417) bank118626 +118627 POINT(47.51369506439462 -122.99252285698239) bank118627 +118628 POINT(48.18736954986023 -122.7951851747341) bank118628 +118629 POINT(47.79660166256909 -121.40949862158945) bank118629 +118630 POINT(47.34143999061675 -121.54225285118663) bank118630 +118631 POINT(48.45531851388638 -122.64443754858183) bank118631 +118632 POINT(46.75338576251704 -122.38553894420662) bank118632 +118633 POINT(47.89403458053866 -122.49779269033685) bank118633 +118634 POINT(48.553207184789436 -123.28622960795889) bank118634 +118635 POINT(48.59588436334658 -122.32151823940245) bank118635 +118636 POINT(47.05676667509014 -122.51481970465005) bank118636 +118637 POINT(48.35258723080736 -121.69066299678697) bank118637 +118638 POINT(46.898181087606616 -122.66785293871325) bank118638 +118639 POINT(47.40996324862333 -122.39037887337813) bank118639 +118640 POINT(47.99090178562345 -121.92247972797502) bank118640 +118641 POINT(48.38203816037556 -122.64230242749004) bank118641 +118642 POINT(47.15113970141124 -122.53166058256032) bank118642 +118643 POINT(47.68117545618787 -122.83627119565219) bank118643 +118644 POINT(47.95092661258503 -122.91336830062569) bank118644 +118645 POINT(46.78130124261161 -122.94959229219889) bank118645 +118646 POINT(46.70920824857705 -122.58203091609398) bank118646 +118647 POINT(47.629481910823394 -122.40702399205502) bank118647 +118648 POINT(47.44218357399755 -121.42300550576536) bank118648 +118649 POINT(46.88581650986739 -122.26186141358555) bank118649 +118650 POINT(47.42829499632252 -122.42295990913571) bank118650 +118651 POINT(46.636340209737476 -122.14833408369385) bank118651 +118652 POINT(48.14722668641825 -122.78743503746196) bank118652 +118653 POINT(47.43800628972563 -121.91794278842202) bank118653 +118654 POINT(46.84074354884092 -121.35229229770702) bank118654 +118655 POINT(47.93304817251729 -121.84175914096859) bank118655 +118656 POINT(47.83974047321254 -122.40352297173368) bank118656 +118657 POINT(47.9597011974314 -121.67636396750565) bank118657 +118658 POINT(48.09205671546366 -121.67822317581752) bank118658 +118659 POINT(47.707233025714004 -122.65649478383394) bank118659 +118660 POINT(48.19067048057561 -123.08247707788219) bank118660 +118661 POINT(46.70762768715374 -122.95735515187587) bank118661 +118662 POINT(48.40356055682362 -122.74938302809615) bank118662 +118663 POINT(46.97828994936678 -122.48027140179063) bank118663 +118664 POINT(46.94361151065595 -122.39396501830899) bank118664 +118665 POINT(46.698050531303736 -122.93424045826217) bank118665 +118666 POINT(46.6518652619504 -122.07117843890585) bank118666 +118667 POINT(46.980535967045334 -121.77960976230057) bank118667 +118668 POINT(47.239569692641105 -122.14816685935934) bank118668 +118669 POINT(47.291590584006705 -122.0783811796463) bank118669 +118670 POINT(47.658640405649116 -122.1329655855791) bank118670 +118671 POINT(48.53772053724482 -122.64673049624434) bank118671 +118672 POINT(46.9126708044418 -123.00664456834457) bank118672 +118673 POINT(47.92969808325177 -122.91435658530186) bank118673 +118674 POINT(47.066025379251656 -122.11762608029382) bank118674 +118675 POINT(47.445635054617256 -122.42092795940718) bank118675 +118676 POINT(47.96102017141424 -123.17164012101016) bank118676 +118677 POINT(48.30884034420779 -122.16823761412645) bank118677 +118678 POINT(47.8883496833474 -122.65280103738628) bank118678 +118679 POINT(47.71518072666328 -122.84317074420726) bank118679 +118680 POINT(47.987136387022105 -122.66815941870334) bank118680 +118681 POINT(47.89690326966319 -121.40920009880149) bank118681 +118682 POINT(47.807725430382916 -123.21336796050757) bank118682 +118683 POINT(46.69392892779693 -122.93277644186642) bank118683 +118684 POINT(48.42568807849602 -121.69812676503022) bank118684 +118685 POINT(48.574026549340864 -122.68388989043501) bank118685 +118686 POINT(46.9459877261262 -122.6660988492912) bank118686 +118687 POINT(48.456175737989916 -122.24973189040148) bank118687 +118688 POINT(48.3864600622527 -122.1776181640417) bank118688 +118689 POINT(48.07512420853514 -122.25086279050642) bank118689 +118690 POINT(47.84709712583894 -121.37539660322358) bank118690 +118691 POINT(48.5926474846206 -122.80341596852931) bank118691 +118692 POINT(47.846598974841555 -123.18730135321269) bank118692 +118693 POINT(48.40230991156615 -122.59465667372733) bank118693 +118694 POINT(47.64980775252219 -121.99345929061167) bank118694 +118695 POINT(47.95604843097916 -121.57173041215061) bank118695 +118696 POINT(47.813177002740304 -122.18769511106834) bank118696 +118697 POINT(47.165248987935826 -122.56780011808875) bank118697 +118698 POINT(47.06792418020728 -122.58190735846145) bank118698 +118699 POINT(46.821309524769255 -122.5400746480938) bank118699 +118700 POINT(48.57635065881172 -123.09589476986204) bank118700 +118701 POINT(47.2018104797935 -123.19418019270044) bank118701 +118702 POINT(48.20301642826005 -122.74786978435424) bank118702 +118703 POINT(47.57496960168621 -122.50492924500965) bank118703 +118704 POINT(46.92541950057095 -121.78926611850576) bank118704 +118705 POINT(48.556704779820784 -122.49693485867446) bank118705 +118706 POINT(47.30111855986448 -122.09713963433485) bank118706 +118707 POINT(47.15753735835977 -121.97818243895823) bank118707 +118708 POINT(47.82392725763841 -121.34624993158269) bank118708 +118709 POINT(46.832328336263686 -121.67901052548308) bank118709 +118710 POINT(46.70802444142706 -122.36031809823952) bank118710 +118711 POINT(48.007646166083475 -121.73256334580857) bank118711 +118712 POINT(47.86055967676743 -122.68023926375915) bank118712 +118713 POINT(48.214453339180054 -122.32465827932154) bank118713 +118714 POINT(46.82082143367739 -121.9907215686321) bank118714 +118715 POINT(47.340947168598845 -121.73868589615248) bank118715 +118716 POINT(46.664063815130106 -122.98882940869382) bank118716 +118717 POINT(48.55588177744792 -121.43065204790807) bank118717 +118718 POINT(47.5475418712898 -122.04569820887757) bank118718 +118719 POINT(48.213214897889614 -122.35283897593814) bank118719 +118720 POINT(48.285647250442906 -122.26500606745054) bank118720 +118721 POINT(47.75182853257765 -122.8591856623625) bank118721 +118722 POINT(48.385506229895036 -122.34317578551894) bank118722 +118723 POINT(47.01825551427319 -122.33560820218868) bank118723 +118724 POINT(48.40135275042666 -122.50692649904347) bank118724 +118725 POINT(48.54103563963515 -123.1331702186219) bank118725 +118726 POINT(48.18712865934633 -121.79144490971564) bank118726 +118727 POINT(47.77510033584742 -122.54861384232287) bank118727 +118728 POINT(48.11119739548635 -121.36181470155041) bank118728 +118729 POINT(47.800784308577235 -122.68915432447757) bank118729 +118730 POINT(48.05666725863999 -122.68672558517017) bank118730 +118731 POINT(47.45470319858434 -122.64461963260221) bank118731 +118732 POINT(47.83843417858053 -121.41874551372216) bank118732 +118733 POINT(47.029690716219704 -122.89317731674616) bank118733 +118734 POINT(47.06843124898159 -122.0674007336217) bank118734 +118735 POINT(47.65885413645746 -122.83920751417665) bank118735 +118736 POINT(46.656625465247444 -122.41066256675192) bank118736 +118737 POINT(46.710222235191445 -123.25733831673288) bank118737 +118738 POINT(47.159924758357306 -123.14613534782394) bank118738 +118739 POINT(46.82541317644302 -122.29917758318396) bank118739 +118740 POINT(48.59795282419048 -122.20230554269838) bank118740 +118741 POINT(46.69498031937321 -123.15885052407002) bank118741 +118742 POINT(48.560307444456576 -121.98489492381152) bank118742 +118743 POINT(48.185706807615674 -123.21261369292063) bank118743 +118744 POINT(48.434654405408104 -123.07665137418772) bank118744 +118745 POINT(47.07952702055191 -122.17827947136152) bank118745 +118746 POINT(47.37584132988642 -122.95569448680844) bank118746 +118747 POINT(47.41849547435767 -121.84167086257614) bank118747 +118748 POINT(48.575827408685264 -121.7095705706101) bank118748 +118749 POINT(48.0031488113124 -121.62666150214423) bank118749 +118750 POINT(48.01605547275996 -121.35586980290579) bank118750 +118751 POINT(47.688241675077975 -122.93829779780859) bank118751 +118752 POINT(47.53027990133761 -121.48965731187903) bank118752 +118753 POINT(48.58397084154515 -121.48720092983261) bank118753 +118754 POINT(46.73438258982396 -122.42247984448575) bank118754 +118755 POINT(47.54537719618286 -122.94688203007802) bank118755 +118756 POINT(47.17384537385108 -123.07794604146746) bank118756 +118757 POINT(47.53643375954382 -121.63957223604672) bank118757 +118758 POINT(47.309840279526135 -121.8034372863982) bank118758 +118759 POINT(47.834754943264976 -122.65948174783848) bank118759 +118760 POINT(47.480431683950556 -122.22846645806452) bank118760 +118761 POINT(48.595378935782726 -121.84316502396372) bank118761 +118762 POINT(47.48910241436817 -121.88233303577738) bank118762 +118763 POINT(47.09619814429171 -123.01722648618411) bank118763 +118764 POINT(47.855116219097894 -122.09884308439949) bank118764 +118765 POINT(47.068462913894194 -122.54117242030443) bank118765 +118766 POINT(48.04032974548136 -122.79211107152813) bank118766 +118767 POINT(47.56406423408558 -122.00096902788421) bank118767 +118768 POINT(47.8668030771492 -122.02109857644881) bank118768 +118769 POINT(48.107821207033 -123.16749343358161) bank118769 +118770 POINT(47.55545099274321 -123.13358776430857) bank118770 +118771 POINT(48.26067127058893 -123.16271969133783) bank118771 +118772 POINT(46.714430335858076 -121.92899127795482) bank118772 +118773 POINT(46.79798455425856 -121.6182394995887) bank118773 +118774 POINT(47.57238832301805 -123.22808508612097) bank118774 +118775 POINT(47.914170192946735 -121.57474264320857) bank118775 +118776 POINT(47.54678280240768 -122.94508912015905) bank118776 +118777 POINT(47.08729172250571 -122.24038424160219) bank118777 +118778 POINT(48.24631645268792 -122.80085027152066) bank118778 +118779 POINT(48.2741848026453 -123.30228641113546) bank118779 +118780 POINT(48.06809729504383 -122.64764604026081) bank118780 +118781 POINT(48.351077349805664 -123.1895853062051) bank118781 +118782 POINT(47.08330915854518 -122.55963929773348) bank118782 +118783 POINT(48.07174436123481 -122.7252348344024) bank118783 +118784 POINT(47.11615762077299 -122.36262177699494) bank118784 +118785 POINT(48.075310753236984 -121.9488651217235) bank118785 +118786 POINT(47.618934916803 -121.59464282818684) bank118786 +118787 POINT(47.00328032823665 -122.30697683524016) bank118787 +118788 POINT(48.17891172797403 -121.71920643668919) bank118788 +118789 POINT(47.96423566270136 -122.69729277151069) bank118789 +118790 POINT(47.7217506023785 -123.3188438094575) bank118790 +118791 POINT(46.94077416682659 -121.712894741671) bank118791 +118792 POINT(48.48777399741898 -123.05344579537457) bank118792 +118793 POINT(46.715249656109236 -122.78546183463249) bank118793 +118794 POINT(47.91246968977196 -122.74911484511122) bank118794 +118795 POINT(48.51395823314012 -122.48862955940987) bank118795 +118796 POINT(46.86511155348905 -121.43257455488467) bank118796 +118797 POINT(46.902886439404774 -122.89201111475116) bank118797 +118798 POINT(47.655332898869865 -122.47315750436213) bank118798 +118799 POINT(47.2115340337672 -122.8948171805288) bank118799 +118800 POINT(46.79637026926247 -123.20246785883192) bank118800 +118801 POINT(47.287286260333644 -122.98710534735373) bank118801 +118802 POINT(48.46745323178273 -122.3284918304375) bank118802 +118803 POINT(47.21777683579951 -123.09035107501226) bank118803 +118804 POINT(47.26984143259944 -121.7977296251147) bank118804 +118805 POINT(48.16443366077751 -123.23802766916002) bank118805 +118806 POINT(47.68228590428672 -122.83210975669378) bank118806 +118807 POINT(47.9871247361186 -122.77393991234946) bank118807 +118808 POINT(47.51103522270793 -123.0805462471852) bank118808 +118809 POINT(48.03207951030038 -123.18323019114789) bank118809 +118810 POINT(48.46446664245608 -121.69708120100573) bank118810 +118811 POINT(48.10612950206408 -122.66539710889992) bank118811 +118812 POINT(48.22955813398282 -122.6359988464468) bank118812 +118813 POINT(47.171051032089004 -121.9651186927131) bank118813 +118814 POINT(46.79642728293235 -123.2185119104203) bank118814 +118815 POINT(47.67100228030616 -122.34266412458504) bank118815 +118816 POINT(47.54612223796178 -121.660492494185) bank118816 +118817 POINT(47.732686319211325 -122.71805439180487) bank118817 +118818 POINT(48.03757726346169 -121.78785246631436) bank118818 +118819 POINT(47.58512763333057 -123.2414124645741) bank118819 +118820 POINT(48.15218565534062 -121.59510639628661) bank118820 +118821 POINT(47.55666655030308 -121.53510155038452) bank118821 +118822 POINT(46.810642645050415 -121.59219502700148) bank118822 +118823 POINT(46.61103647582085 -122.14613432323384) bank118823 +118824 POINT(47.58231175397965 -122.24126455995592) bank118824 +118825 POINT(47.8867112854968 -123.00600357059656) bank118825 +118826 POINT(47.70982435641593 -122.12269450150463) bank118826 +118827 POINT(46.89731462433486 -122.59853537064059) bank118827 +118828 POINT(46.82719255315238 -122.48108300177458) bank118828 +118829 POINT(48.12136820966178 -121.52175145419734) bank118829 +118830 POINT(48.082484336330914 -122.13363504309899) bank118830 +118831 POINT(48.25014580038014 -122.34206193884111) bank118831 +118832 POINT(46.726735921948 -122.59724606110252) bank118832 +118833 POINT(47.47415641433519 -122.62651859543905) bank118833 +118834 POINT(47.90388279604052 -122.79586134022162) bank118834 +118835 POINT(47.1806297040043 -122.30009523637855) bank118835 +118836 POINT(48.01421242284666 -121.647251272251) bank118836 +118837 POINT(47.950435490750856 -122.43001098152908) bank118837 +118838 POINT(48.440439792169705 -121.90629662715001) bank118838 +118839 POINT(47.2085737343975 -121.35437032168582) bank118839 +118840 POINT(46.622160265026935 -122.05241180899402) bank118840 +118841 POINT(48.18573974591858 -121.77601830877838) bank118841 +118842 POINT(46.885458394633716 -121.5417064374185) bank118842 +118843 POINT(47.11422065773727 -121.42905760179485) bank118843 +118844 POINT(47.47866499188025 -122.14028822776343) bank118844 +118845 POINT(47.664201171643576 -122.74616660472944) bank118845 +118846 POINT(48.34942577140523 -122.68323642232853) bank118846 +118847 POINT(47.23660274738958 -121.53500796550414) bank118847 +118848 POINT(47.34925566915991 -122.20405269132463) bank118848 +118849 POINT(48.339366243398906 -122.35308826328996) bank118849 +118850 POINT(47.234458637025405 -123.00670688895292) bank118850 +118851 POINT(47.73519144048276 -123.06137707593555) bank118851 +118852 POINT(47.400568192500785 -122.2143338994348) bank118852 +118853 POINT(46.74334426930068 -121.58087086932261) bank118853 +118854 POINT(46.6295682862544 -123.28925162240935) bank118854 +118855 POINT(46.65691165188768 -123.1410938456174) bank118855 +118856 POINT(48.38974512097645 -122.14656280533775) bank118856 +118857 POINT(47.53911985035488 -122.36025498487123) bank118857 +118858 POINT(47.07621307861771 -121.69916133268042) bank118858 +118859 POINT(47.303717383507546 -122.86868564656675) bank118859 +118860 POINT(46.9299168062242 -122.08038439015408) bank118860 +118861 POINT(47.86999434167521 -122.6234739034216) bank118861 +118862 POINT(46.687983041442465 -122.9950951700371) bank118862 +118863 POINT(48.26646704237017 -122.23480074385763) bank118863 +118864 POINT(46.82661930126668 -122.54014397920976) bank118864 +118865 POINT(47.063010131188044 -121.66512309041518) bank118865 +118866 POINT(46.62355789865608 -121.9249668663093) bank118866 +118867 POINT(48.24437598886401 -123.32662829335253) bank118867 +118868 POINT(46.74669085253061 -122.8525605427984) bank118868 +118869 POINT(47.60727387680017 -121.53012716166936) bank118869 +118870 POINT(48.247752329017636 -123.16635282773821) bank118870 +118871 POINT(47.22353351279 -122.29888026362997) bank118871 +118872 POINT(47.23492711243066 -121.37401390491833) bank118872 +118873 POINT(47.78474024740153 -121.48080127088944) bank118873 +118874 POINT(47.14128438648304 -121.4041245019128) bank118874 +118875 POINT(46.8841824358673 -121.63682817799318) bank118875 +118876 POINT(47.56204155144753 -122.55217761298825) bank118876 +118877 POINT(48.144402950934555 -122.90630486701193) bank118877 +118878 POINT(48.03945254535452 -123.2854762645605) bank118878 +118879 POINT(46.98382694455533 -122.58468801269409) bank118879 +118880 POINT(46.74332002877275 -123.32499705894915) bank118880 +118881 POINT(48.087128414967054 -122.11224148745933) bank118881 +118882 POINT(48.19895866175321 -121.3917690059457) bank118882 +118883 POINT(47.4658840206126 -122.88630576065746) bank118883 +118884 POINT(47.76585610404388 -121.69541699662446) bank118884 +118885 POINT(47.8081984186414 -123.24610167885237) bank118885 +118886 POINT(47.4911642357721 -122.89393948979112) bank118886 +118887 POINT(48.58940691103617 -122.67404523864043) bank118887 +118888 POINT(47.316342481170246 -121.82724757008776) bank118888 +118889 POINT(46.67010147589243 -122.50470368302072) bank118889 +118890 POINT(46.736525602051145 -121.46575034131642) bank118890 +118891 POINT(48.30294882105866 -122.57306475420243) bank118891 +118892 POINT(47.3045452984053 -121.92618869884913) bank118892 +118893 POINT(46.971485716967216 -122.87718732313205) bank118893 +118894 POINT(48.302906748882194 -121.8182894234429) bank118894 +118895 POINT(46.6092728081733 -122.30801298815256) bank118895 +118896 POINT(46.8319317729303 -121.53954149269944) bank118896 +118897 POINT(47.59422815179955 -123.29845764554688) bank118897 +118898 POINT(48.21512798325722 -123.245716092476) bank118898 +118899 POINT(48.10396662369687 -123.10678103794062) bank118899 +118900 POINT(47.63045002071992 -123.21499149574471) bank118900 +118901 POINT(46.989063467234814 -122.43066592938403) bank118901 +118902 POINT(48.06344913620003 -122.51913483932833) bank118902 +118903 POINT(48.20204775528904 -121.9535844312305) bank118903 +118904 POINT(47.632446914490615 -123.10033108313225) bank118904 +118905 POINT(47.269934089746975 -121.9963802147396) bank118905 +118906 POINT(47.594177325347744 -122.9794516867628) bank118906 +118907 POINT(47.49523578339689 -122.01711651453736) bank118907 +118908 POINT(47.18667972569917 -121.73229507049457) bank118908 +118909 POINT(47.40688148267744 -121.85353272845485) bank118909 +118910 POINT(48.16303866615933 -122.87846152397039) bank118910 +118911 POINT(48.31620662446055 -123.11757657207218) bank118911 +118912 POINT(47.204371257344896 -122.29744866385484) bank118912 +118913 POINT(46.86047811210017 -121.98535256261151) bank118913 +118914 POINT(47.531965405572066 -123.29422345839014) bank118914 +118915 POINT(48.5016079005651 -122.95047418323442) bank118915 +118916 POINT(47.09066056639754 -121.71520193304882) bank118916 +118917 POINT(48.156975142365944 -121.92049494345481) bank118917 +118918 POINT(48.37368467257123 -122.99710421623513) bank118918 +118919 POINT(48.02270202411669 -122.06287007297611) bank118919 +118920 POINT(48.26730887724396 -123.2297232217785) bank118920 +118921 POINT(48.17636042734913 -121.72033174829927) bank118921 +118922 POINT(47.407893590231645 -122.84350094628982) bank118922 +118923 POINT(47.05412501347018 -121.91799163174747) bank118923 +118924 POINT(46.94454666304888 -121.94095400405992) bank118924 +118925 POINT(46.65236046258625 -122.4357784349052) bank118925 +118926 POINT(46.95453277473267 -122.32670395704902) bank118926 +118927 POINT(47.131987387478 -121.896590039972) bank118927 +118928 POINT(48.40811210894245 -123.0489346284797) bank118928 +118929 POINT(46.61429968611454 -122.55273412753594) bank118929 +118930 POINT(47.346670367514704 -122.42485025999235) bank118930 +118931 POINT(48.158238797896516 -122.26927272252739) bank118931 +118932 POINT(46.93630935235864 -121.7591341348527) bank118932 +118933 POINT(46.85324314116835 -122.56061718714837) bank118933 +118934 POINT(46.77975879448748 -122.97693362932768) bank118934 +118935 POINT(46.992000633383796 -121.66368317898325) bank118935 +118936 POINT(47.77743217096151 -122.46477425707629) bank118936 +118937 POINT(47.79402605604038 -122.42212876858991) bank118937 +118938 POINT(46.854643704869815 -121.33855606940895) bank118938 +118939 POINT(47.58985798791063 -122.76552455956349) bank118939 +118940 POINT(46.60869609220743 -122.06996030030142) bank118940 +118941 POINT(46.642732026096354 -121.45142930417185) bank118941 +118942 POINT(47.74798474338459 -122.25203065458514) bank118942 +118943 POINT(47.9078995539492 -123.2703342579272) bank118943 +118944 POINT(48.48817011198519 -123.2467333805417) bank118944 +118945 POINT(47.67652935292355 -123.03411056614416) bank118945 +118946 POINT(47.04580458209284 -122.01255964982039) bank118946 +118947 POINT(48.31121610902555 -121.43940930345687) bank118947 +118948 POINT(47.16837468054714 -122.59646245169662) bank118948 +118949 POINT(47.921044972483166 -121.54062048809688) bank118949 +118950 POINT(48.213532346503115 -121.73363494730226) bank118950 +118951 POINT(47.680313170939414 -123.16883667671864) bank118951 +118952 POINT(47.3203125847605 -123.04598508234382) bank118952 +118953 POINT(48.00338270088784 -121.76416874794033) bank118953 +118954 POINT(47.988612223431346 -121.97564140038706) bank118954 +118955 POINT(48.2287396639724 -122.88698862476322) bank118955 +118956 POINT(47.76907512121019 -121.78181346565846) bank118956 +118957 POINT(47.238046007727874 -122.61722923082222) bank118957 +118958 POINT(48.12479626409364 -122.68936086056854) bank118958 +118959 POINT(47.261103950798486 -121.8790012281375) bank118959 +118960 POINT(48.14617164689057 -122.63115454599838) bank118960 +118961 POINT(48.03207995889769 -121.4174174963369) bank118961 +118962 POINT(46.8958597263363 -123.09722927575912) bank118962 +118963 POINT(46.84618801354926 -122.41283262934242) bank118963 +118964 POINT(46.79572980437311 -121.4919990349653) bank118964 +118965 POINT(48.3939196876483 -122.40328513981436) bank118965 +118966 POINT(47.59159827880929 -122.66541665193968) bank118966 +118967 POINT(46.61971907535163 -123.26330244090107) bank118967 +118968 POINT(46.65719280763166 -121.35796428059362) bank118968 +118969 POINT(47.01772762295229 -122.34944993134202) bank118969 +118970 POINT(48.298686522614425 -122.01585441307193) bank118970 +118971 POINT(47.025645918199494 -122.23420189934039) bank118971 +118972 POINT(46.78130011230073 -121.9579571810328) bank118972 +118973 POINT(46.853737815778715 -121.47312702003032) bank118973 +118974 POINT(46.78661511322729 -121.53558463396777) bank118974 +118975 POINT(47.89205337065597 -122.35418627002673) bank118975 +118976 POINT(46.97502930637726 -121.9061956280842) bank118976 +118977 POINT(47.007161543860065 -122.709345705524) bank118977 +118978 POINT(47.66032449771157 -122.39948917040236) bank118978 +118979 POINT(46.73309379942539 -121.71372325889705) bank118979 +118980 POINT(47.93052479235803 -123.00996919744878) bank118980 +118981 POINT(47.43494137592298 -123.04714635663885) bank118981 +118982 POINT(46.909807429591105 -122.19178937747174) bank118982 +118983 POINT(47.552122452112606 -122.2038897890384) bank118983 +118984 POINT(47.619307124060796 -121.83839671764356) bank118984 +118985 POINT(48.32425194643034 -121.916756336422) bank118985 +118986 POINT(48.53484162474897 -121.55840894863273) bank118986 +118987 POINT(47.064304801898814 -122.67754787991092) bank118987 +118988 POINT(48.03318107902136 -121.95355757179131) bank118988 +118989 POINT(47.87911529628145 -122.42528436837125) bank118989 +118990 POINT(48.16044776710295 -122.81904218571582) bank118990 +118991 POINT(47.21861872422609 -123.19119842536074) bank118991 +118992 POINT(46.97348009535115 -122.52402563567804) bank118992 +118993 POINT(47.61034211246176 -121.63235610153703) bank118993 +118994 POINT(48.09798995828463 -123.05967590249934) bank118994 +118995 POINT(47.67643945776601 -121.3876730569967) bank118995 +118996 POINT(47.863067142675675 -123.2267805618012) bank118996 +118997 POINT(48.37666229221991 -122.64247908626923) bank118997 +118998 POINT(46.994578096565256 -122.54635893416295) bank118998 +118999 POINT(47.78769644996995 -121.47913014756678) bank118999 +119000 POINT(47.26878873694691 -123.30398620196226) bank119000 +119001 POINT(46.78556772392912 -122.76957353698891) bank119001 +119002 POINT(48.56126642674188 -121.85406597530974) bank119002 +119003 POINT(48.093843700830305 -122.62339471224504) bank119003 +119004 POINT(47.261215932491886 -122.57904608504883) bank119004 +119005 POINT(46.90149394209187 -122.17042710366779) bank119005 +119006 POINT(47.481298011617085 -122.64535495915591) bank119006 +119007 POINT(46.83608066041829 -122.67424686331087) bank119007 +119008 POINT(48.50786392618735 -123.1253243675176) bank119008 +119009 POINT(47.99216809432156 -123.06502746900262) bank119009 +119010 POINT(47.099172578215075 -123.28617046204346) bank119010 +119011 POINT(46.61785769373469 -121.44190462015723) bank119011 +119012 POINT(47.25805666719348 -122.56021519758889) bank119012 +119013 POINT(48.30440358538433 -121.85333255871997) bank119013 +119014 POINT(48.00507516516682 -121.41656857678723) bank119014 +119015 POINT(46.967596992917564 -123.28771088321385) bank119015 +119016 POINT(47.63828091643053 -122.3517683903956) bank119016 +119017 POINT(46.79948302291805 -122.11979570759108) bank119017 +119018 POINT(47.10891872360343 -122.85633927527265) bank119018 +119019 POINT(47.34083343598656 -122.77013294900809) bank119019 +119020 POINT(47.76766476920918 -121.74952246132904) bank119020 +119021 POINT(47.11025412064517 -122.60741291472327) bank119021 +119022 POINT(48.56599113257255 -123.18339331431979) bank119022 +119023 POINT(48.56337217811201 -123.2574416731508) bank119023 +119024 POINT(46.67643718602003 -121.45533090757871) bank119024 +119025 POINT(46.75896072634969 -121.4284758419973) bank119025 +119026 POINT(47.71017197546794 -121.9371336962479) bank119026 +119027 POINT(48.582046134458615 -122.72162128461825) bank119027 +119028 POINT(48.581437563158204 -122.3131049784016) bank119028 +119029 POINT(47.31377832610151 -121.77811196192488) bank119029 +119030 POINT(48.26128710135561 -122.64159074100208) bank119030 +119031 POINT(46.635413644420154 -121.61225209207015) bank119031 +119032 POINT(47.04474604625975 -122.98836739962316) bank119032 +119033 POINT(46.859171195362826 -121.78100944835242) bank119033 +119034 POINT(47.32012704382762 -122.06108666935921) bank119034 +119035 POINT(47.46513537946249 -123.16970498681255) bank119035 +119036 POINT(48.32659995960175 -121.61983978469644) bank119036 +119037 POINT(48.47608778412819 -122.52557852413858) bank119037 +119038 POINT(47.16997990278454 -122.82385627324986) bank119038 +119039 POINT(46.95948664191347 -122.76576057074928) bank119039 +119040 POINT(47.3042938800254 -122.6865221157243) bank119040 +119041 POINT(47.75550780647788 -121.533381201276) bank119041 +119042 POINT(47.17395297975125 -123.09151984553256) bank119042 +119043 POINT(48.071896341653286 -122.79141834234288) bank119043 +119044 POINT(48.40866138359136 -122.13159605983833) bank119044 +119045 POINT(48.18207475762548 -122.07775912238301) bank119045 +119046 POINT(47.50906747151504 -121.84880245462102) bank119046 +119047 POINT(46.720440577889036 -122.49876598265915) bank119047 +119048 POINT(47.824204812124584 -123.29071506449735) bank119048 +119049 POINT(46.687200829633156 -121.81116702909449) bank119049 +119050 POINT(47.007342960646874 -122.04808374412148) bank119050 +119051 POINT(47.88132753320191 -123.21923420641485) bank119051 +119052 POINT(48.311590448263594 -121.97642847625163) bank119052 +119053 POINT(47.27860146222836 -122.05856388193304) bank119053 +119054 POINT(48.594605996781475 -122.20729356342588) bank119054 +119055 POINT(47.821413634072385 -122.37880480287916) bank119055 +119056 POINT(47.84368377677898 -122.51878089018093) bank119056 +119057 POINT(47.05617954542112 -122.49678246797005) bank119057 +119058 POINT(48.37522587760052 -123.2374432950204) bank119058 +119059 POINT(47.43761453373112 -121.34411072856891) bank119059 +119060 POINT(48.58183920655972 -122.33565215194987) bank119060 +119061 POINT(47.6179284002602 -121.60871281366893) bank119061 +119062 POINT(47.19496259783978 -123.05414080371553) bank119062 +119063 POINT(46.74762202399534 -122.58037705954911) bank119063 +119064 POINT(48.47635854790008 -121.89865334598171) bank119064 +119065 POINT(47.06733560549465 -121.59922260042002) bank119065 +119066 POINT(47.21038602705675 -121.50217300757737) bank119066 +119067 POINT(48.4830310606613 -122.44761678756657) bank119067 +119068 POINT(48.25088317620371 -123.10077848087619) bank119068 +119069 POINT(48.282039536657884 -122.53076903840493) bank119069 +119070 POINT(46.69304178677252 -123.23772253221736) bank119070 +119071 POINT(47.13328830069103 -121.59060678856981) bank119071 +119072 POINT(48.17146237298554 -122.5204621481192) bank119072 +119073 POINT(46.61470761317878 -121.76904490071202) bank119073 +119074 POINT(47.91255935009666 -123.19061999465269) bank119074 +119075 POINT(48.33074820001246 -121.75667868857282) bank119075 +119076 POINT(48.36323529582656 -122.89450118473886) bank119076 +119077 POINT(46.98770138549409 -122.67565332395327) bank119077 +119078 POINT(47.052849723997234 -122.38411455305767) bank119078 +119079 POINT(47.82332860146923 -122.7598841479739) bank119079 +119080 POINT(47.04839410030726 -122.45891680619683) bank119080 +119081 POINT(47.92597272188772 -121.51459405258362) bank119081 +119082 POINT(48.20468274702585 -121.9315452042299) bank119082 +119083 POINT(47.67211311440927 -123.18788080253056) bank119083 +119084 POINT(47.47753036509469 -122.15875774166084) bank119084 +119085 POINT(47.524754064460225 -122.58495550475477) bank119085 +119086 POINT(48.03347294214229 -121.80366305084837) bank119086 +119087 POINT(48.00259531021329 -122.5930160123201) bank119087 +119088 POINT(47.06614291007369 -121.71409327092329) bank119088 +119089 POINT(48.583097559216085 -122.58813273154904) bank119089 +119090 POINT(47.07957457883735 -123.08118334473913) bank119090 +119091 POINT(47.317033773912655 -122.68764702233095) bank119091 +119092 POINT(47.191576637649725 -121.71099372493217) bank119092 +119093 POINT(47.211996172346254 -123.28503719928766) bank119093 +119094 POINT(47.426800526807625 -121.84651458397504) bank119094 +119095 POINT(48.40207031221209 -123.15452287296583) bank119095 +119096 POINT(48.072535148900165 -123.238165600245) bank119096 +119097 POINT(47.05580015531025 -123.19899456662559) bank119097 +119098 POINT(47.3005122673766 -122.87220992930422) bank119098 +119099 POINT(46.65244529212427 -123.30988405302335) bank119099 +119100 POINT(47.562100010359806 -123.15435982796603) bank119100 +119101 POINT(47.05336654102576 -122.52641506188795) bank119101 +119102 POINT(47.43936450995167 -121.60311938634831) bank119102 +119103 POINT(46.92400277483579 -121.39842981145028) bank119103 +119104 POINT(48.42289515403909 -123.21252525432014) bank119104 +119105 POINT(47.6631672382648 -121.64922288648646) bank119105 +119106 POINT(48.45748816753648 -121.48618388643098) bank119106 +119107 POINT(47.508169030967856 -122.74999164508765) bank119107 +119108 POINT(47.54816203946516 -122.33023447625656) bank119108 +119109 POINT(48.413187415481346 -122.25915902899116) bank119109 +119110 POINT(46.74663204272569 -122.60062892454886) bank119110 +119111 POINT(48.15508543899439 -123.32751969820967) bank119111 +119112 POINT(46.92877930757829 -123.01722581321606) bank119112 +119113 POINT(47.188196606016824 -122.78465827308831) bank119113 +119114 POINT(47.69252132414009 -121.97255135966836) bank119114 +119115 POINT(46.97385686341106 -122.05525029137749) bank119115 +119116 POINT(47.292878693393114 -123.22945523728912) bank119116 +119117 POINT(47.593107755192136 -121.959917861891) bank119117 +119118 POINT(46.83381510899767 -122.19149430797285) bank119118 +119119 POINT(47.08576781799499 -122.72953018657961) bank119119 +119120 POINT(47.77276625767949 -122.89173996518164) bank119120 +119121 POINT(47.55185036209417 -121.47441720849586) bank119121 +119122 POINT(48.59642312044808 -121.87658422190484) bank119122 +119123 POINT(48.20562463534534 -122.86064519393628) bank119123 +119124 POINT(48.153221242518846 -122.55673588716195) bank119124 +119125 POINT(47.58825764948007 -121.61588595310639) bank119125 +119126 POINT(47.76847673630499 -122.28682870113613) bank119126 +119127 POINT(46.834382350886514 -121.70807409091451) bank119127 +119128 POINT(47.74448942730239 -123.10453201257873) bank119128 +119129 POINT(47.59706362138264 -122.34325104053016) bank119129 +119130 POINT(46.75971782908986 -123.1654483084135) bank119130 +119131 POINT(47.67791460694495 -122.459543025356) bank119131 +119132 POINT(47.45017263524622 -122.70275002918517) bank119132 +119133 POINT(48.101438539361496 -121.45706197998238) bank119133 +119134 POINT(47.57061771835313 -121.89407572891417) bank119134 +119135 POINT(48.0497146979243 -121.55992955470232) bank119135 +119136 POINT(48.11111950363245 -121.8125553837732) bank119136 +119137 POINT(47.51577932684652 -121.63592201570654) bank119137 +119138 POINT(48.103326015313115 -122.77550756229604) bank119138 +119139 POINT(46.78491396296907 -121.49312020420217) bank119139 +119140 POINT(46.87651466899668 -123.17714978018105) bank119140 +119141 POINT(46.673387236133756 -122.3313300272705) bank119141 +119142 POINT(47.098122970189856 -122.18816385753543) bank119142 +119143 POINT(46.76024553216519 -122.14429949375845) bank119143 +119144 POINT(46.611081216398084 -121.8819218959727) bank119144 +119145 POINT(47.72169803672073 -122.19510889604098) bank119145 +119146 POINT(47.3345398405253 -122.166645871241) bank119146 +119147 POINT(46.72602382098753 -122.70372594289171) bank119147 +119148 POINT(47.294850461904716 -121.46678718675935) bank119148 +119149 POINT(47.2692315224196 -122.32452170900378) bank119149 +119150 POINT(47.29007837533708 -122.47197798413495) bank119150 +119151 POINT(46.80224072082379 -121.95261673504156) bank119151 +119152 POINT(47.360679915110175 -122.47189452569619) bank119152 +119153 POINT(46.62101287413299 -122.46371898856174) bank119153 +119154 POINT(48.42698092835678 -122.6858561169894) bank119154 +119155 POINT(48.16608635695673 -123.0523646202071) bank119155 +119156 POINT(47.21668294454149 -122.19823177714025) bank119156 +119157 POINT(47.52969242975026 -121.9123358816274) bank119157 +119158 POINT(47.94124178099929 -121.97317316954816) bank119158 +119159 POINT(48.24876757993842 -121.43411820418835) bank119159 +119160 POINT(47.49988183399811 -122.15626157545744) bank119160 +119161 POINT(47.45355692985726 -122.31579152050259) bank119161 +119162 POINT(47.19136877714208 -121.85760512945143) bank119162 +119163 POINT(47.028946134856696 -122.72182367553813) bank119163 +119164 POINT(47.527049441687595 -122.33278810633341) bank119164 +119165 POINT(48.09948077913321 -121.47842390438593) bank119165 +119166 POINT(46.75625630348425 -122.49613341035308) bank119166 +119167 POINT(48.38304238290928 -122.98449154739079) bank119167 +119168 POINT(47.30903197489472 -122.4439365316795) bank119168 +119169 POINT(48.01344642008618 -121.6637242095549) bank119169 +119170 POINT(46.83371783687928 -122.7937339970981) bank119170 +119171 POINT(48.46553114801043 -123.12567206151431) bank119171 +119172 POINT(46.61758008646987 -122.19402584459309) bank119172 +119173 POINT(47.36321350996548 -121.99767883496494) bank119173 +119174 POINT(48.128112500897366 -122.50331573309157) bank119174 +119175 POINT(47.226091372394016 -121.79195177279348) bank119175 +119176 POINT(47.0472208504312 -121.87232624954869) bank119176 +119177 POINT(48.25630292282232 -123.05967480470265) bank119177 +119178 POINT(47.87383253413609 -122.66044339369196) bank119178 +119179 POINT(47.229303443525666 -123.21440887657093) bank119179 +119180 POINT(48.571177761234274 -123.20456751216358) bank119180 +119181 POINT(47.82867411999026 -121.57106875702533) bank119181 +119182 POINT(47.015852134332526 -121.84300114181) bank119182 +119183 POINT(46.84192315964197 -122.28692353609503) bank119183 +119184 POINT(47.7087489847258 -122.87292088430164) bank119184 +119185 POINT(47.5735565982182 -122.0515534851227) bank119185 +119186 POINT(46.65981369870969 -122.11642929683875) bank119186 +119187 POINT(47.056126514761026 -123.3316669743612) bank119187 +119188 POINT(47.673147345867335 -122.48468302522284) bank119188 +119189 POINT(47.457688349336 -123.26665180915116) bank119189 +119190 POINT(48.160435598738026 -121.68423674676387) bank119190 +119191 POINT(47.23535684899146 -122.22549567876908) bank119191 +119192 POINT(46.782040992908044 -122.39810237970633) bank119192 +119193 POINT(47.683177284223646 -122.90825915161278) bank119193 +119194 POINT(47.481233577340944 -122.95662906035973) bank119194 +119195 POINT(47.88688279422522 -122.08617754726006) bank119195 +119196 POINT(46.836200894836644 -123.23622966659849) bank119196 +119197 POINT(47.67190519661593 -122.91196002026632) bank119197 +119198 POINT(47.373178519835406 -121.53882866565485) bank119198 +119199 POINT(47.01039665116514 -122.4771370564461) bank119199 +119200 POINT(48.241304600889194 -122.5443236531308) bank119200 +119201 POINT(47.04168887116684 -121.62613189647479) bank119201 +119202 POINT(46.63726135123729 -121.77000476608511) bank119202 +119203 POINT(47.92574583976159 -122.99063647313919) bank119203 +119204 POINT(47.15797715691453 -122.93472325143246) bank119204 +119205 POINT(47.34047157487353 -121.42140789547163) bank119205 +119206 POINT(47.517955383449916 -121.53177809053454) bank119206 +119207 POINT(48.36891136404372 -121.3925198267689) bank119207 +119208 POINT(47.84529320549957 -122.22182658839107) bank119208 +119209 POINT(47.839962602422126 -122.53103028283971) bank119209 +119210 POINT(48.14397363242208 -121.34774183330227) bank119210 +119211 POINT(46.62409879499156 -122.59195454760369) bank119211 +119212 POINT(46.72600372518346 -121.37852425811812) bank119212 +119213 POINT(46.94399095139541 -122.91223189983565) bank119213 +119214 POINT(46.962086065939545 -121.75926016771903) bank119214 +119215 POINT(48.28578193937421 -122.60256971409574) bank119215 +119216 POINT(47.82601122345903 -122.35553546226079) bank119216 +119217 POINT(48.577358079853184 -123.11666083930149) bank119217 +119218 POINT(48.56097086409307 -122.59048813082478) bank119218 +119219 POINT(46.609113173764115 -121.97313473605526) bank119219 +119220 POINT(48.033671175389046 -123.11148714527386) bank119220 +119221 POINT(46.62396861928923 -121.7622429248701) bank119221 +119222 POINT(48.07102349888336 -121.92258169267858) bank119222 +119223 POINT(47.443074555073665 -121.47159641983228) bank119223 +119224 POINT(46.84365070193405 -121.72131513128316) bank119224 +119225 POINT(47.426027900914896 -121.481769395119) bank119225 +119226 POINT(48.261076150355485 -122.32066187440337) bank119226 +119227 POINT(47.070887287157994 -121.95401137107308) bank119227 +119228 POINT(48.199343198020884 -122.21191504191586) bank119228 +119229 POINT(48.59851987667744 -122.75350427815779) bank119229 +119230 POINT(48.46260817922254 -121.33771031665081) bank119230 +119231 POINT(47.649496130923964 -123.03667484070694) bank119231 +119232 POINT(48.29379609310973 -122.00089984371866) bank119232 +119233 POINT(47.26720360181802 -122.85888560806026) bank119233 +119234 POINT(46.91648482981969 -121.77821094268667) bank119234 +119235 POINT(48.52924774592887 -123.06593376169525) bank119235 +119236 POINT(47.65725905210839 -123.1748616935406) bank119236 +119237 POINT(47.46665378510107 -122.04639193698317) bank119237 +119238 POINT(48.246670553230594 -121.72696296210407) bank119238 +119239 POINT(46.7589294802893 -123.31766378710677) bank119239 +119240 POINT(48.06856669680911 -121.43243012320993) bank119240 +119241 POINT(47.48804779150173 -122.99988054668725) bank119241 +119242 POINT(47.57350180165746 -123.1549951301525) bank119242 +119243 POINT(46.82838001766828 -122.04017465766955) bank119243 +119244 POINT(48.57700610282294 -121.82945593641398) bank119244 +119245 POINT(46.99099693484145 -122.83659205802746) bank119245 +119246 POINT(47.23866428919501 -122.17465158265682) bank119246 +119247 POINT(48.295516959043475 -123.12439333887892) bank119247 +119248 POINT(48.21385884988585 -122.2862945081512) bank119248 +119249 POINT(48.176460884102454 -123.1716671370468) bank119249 +119250 POINT(46.92963597450777 -123.10551628124715) bank119250 +119251 POINT(47.86319464982369 -121.98732004142899) bank119251 +119252 POINT(48.371060895986425 -122.57884044550103) bank119252 +119253 POINT(47.57075029189933 -121.80892928763808) bank119253 +119254 POINT(47.991664740266266 -122.29239017164713) bank119254 +119255 POINT(46.88035118198341 -122.78173484811455) bank119255 +119256 POINT(48.27354012020314 -122.17336155146245) bank119256 +119257 POINT(47.94700444267653 -122.07317617981978) bank119257 +119258 POINT(48.53293512755461 -122.12593666640637) bank119258 +119259 POINT(48.41356154944361 -121.61530890679396) bank119259 +119260 POINT(46.72623117445144 -122.65364026736724) bank119260 +119261 POINT(46.82311257954803 -121.33795085145296) bank119261 +119262 POINT(46.91755743049951 -122.50781820870323) bank119262 +119263 POINT(46.827086482977755 -121.93631332557153) bank119263 +119264 POINT(47.13503595458596 -123.20457532215636) bank119264 +119265 POINT(47.006177267364386 -121.88586349255797) bank119265 +119266 POINT(47.48586094973502 -121.9662089115682) bank119266 +119267 POINT(46.67839881508929 -121.37069481164566) bank119267 +119268 POINT(47.16248647593953 -121.96008105283893) bank119268 +119269 POINT(46.736431468633555 -122.22917430064422) bank119269 +119270 POINT(48.280471847895456 -123.15440190271362) bank119270 +119271 POINT(46.91100648248401 -122.56096698106832) bank119271 +119272 POINT(48.23354753288036 -121.93247573305823) bank119272 +119273 POINT(46.74596039128229 -121.79707405652209) bank119273 +119274 POINT(46.71725224765058 -122.82326493442591) bank119274 +119275 POINT(47.43333973649317 -123.12679496151438) bank119275 +119276 POINT(46.65970076586684 -122.0083614060058) bank119276 +119277 POINT(47.76907119755001 -122.56712152815244) bank119277 +119278 POINT(46.717559557275436 -122.6393109690072) bank119278 +119279 POINT(48.4426207617709 -122.14245253792775) bank119279 +119280 POINT(48.0932624656114 -121.80207433373833) bank119280 +119281 POINT(47.593809046947996 -121.8510785398587) bank119281 +119282 POINT(47.60975696483025 -121.95769102843471) bank119282 +119283 POINT(48.1946008419588 -122.39279485736357) bank119283 +119284 POINT(48.51420947535441 -123.32499182583994) bank119284 +119285 POINT(46.66110950070862 -122.48213625411977) bank119285 +119286 POINT(47.460863351427925 -122.27679943630582) bank119286 +119287 POINT(48.4535993372358 -122.78145460438408) bank119287 +119288 POINT(47.916593531424105 -121.57086601844198) bank119288 +119289 POINT(47.84333094811054 -122.64099332075732) bank119289 +119290 POINT(47.761403728207505 -123.20149574785655) bank119290 +119291 POINT(48.41532340095886 -122.61393871255453) bank119291 +119292 POINT(48.55419852347372 -123.23101636728613) bank119292 +119293 POINT(47.04583909391741 -123.30920662731205) bank119293 +119294 POINT(47.78668028847371 -121.52101989566891) bank119294 +119295 POINT(47.0552675573131 -121.76385283226172) bank119295 +119296 POINT(46.68727050455494 -122.05187133031538) bank119296 +119297 POINT(47.57627338089139 -121.9198865593915) bank119297 +119298 POINT(47.05026625690665 -121.75101793856071) bank119298 +119299 POINT(47.6753724585869 -122.49721358200419) bank119299 +119300 POINT(48.08213747551719 -121.8326360858008) bank119300 +119301 POINT(47.9548835104181 -123.11473956410114) bank119301 +119302 POINT(48.24302192447141 -122.4827980098214) bank119302 +119303 POINT(48.38908243861389 -123.02032133725001) bank119303 +119304 POINT(47.35980638279369 -122.89088228821875) bank119304 +119305 POINT(47.83321837677856 -121.54613663566887) bank119305 +119306 POINT(46.93476984855526 -122.49348479342558) bank119306 +119307 POINT(47.46882515760316 -121.55480579199141) bank119307 +119308 POINT(46.804942255087234 -122.2079925415413) bank119308 +119309 POINT(47.90904806236062 -122.08145540477271) bank119309 +119310 POINT(47.016369577805676 -122.07734374938167) bank119310 +119311 POINT(47.38722520468221 -122.3300547206936) bank119311 +119312 POINT(48.38296070420871 -121.52449260638913) bank119312 +119313 POINT(46.67498103550359 -122.01548976383371) bank119313 +119314 POINT(48.04737487433241 -122.25042080815716) bank119314 +119315 POINT(47.48920308928379 -123.01760117331521) bank119315 +119316 POINT(47.92161197820208 -121.86274239933675) bank119316 +119317 POINT(48.49301064922441 -123.12755127964687) bank119317 +119318 POINT(48.231775901216864 -122.0436248444029) bank119318 +119319 POINT(47.58129073626051 -122.40215970729605) bank119319 +119320 POINT(48.18755587656767 -122.70263252183777) bank119320 +119321 POINT(46.99109066042111 -122.53037051190249) bank119321 +119322 POINT(48.01869670913841 -121.6180494984635) bank119322 +119323 POINT(47.2761086418463 -121.47334209659996) bank119323 +119324 POINT(47.81664592904916 -122.49073373226913) bank119324 +119325 POINT(47.76143277050777 -123.0677602273342) bank119325 +119326 POINT(46.68784700137373 -121.51623248906505) bank119326 +119327 POINT(46.676864868639235 -122.6240103692795) bank119327 +119328 POINT(48.362613216866116 -121.61700956365289) bank119328 +119329 POINT(47.24934326303275 -122.77379871608163) bank119329 +119330 POINT(46.73634821452057 -123.19269197488302) bank119330 +119331 POINT(46.97785683049181 -123.06241768469484) bank119331 +119332 POINT(46.68631182912951 -123.21057779568126) bank119332 +119333 POINT(46.86731892366586 -123.10208796824799) bank119333 +119334 POINT(46.64141355821278 -122.74475667239524) bank119334 +119335 POINT(47.473913212225106 -121.40694747648863) bank119335 +119336 POINT(48.11484821860795 -121.5922260533751) bank119336 +119337 POINT(47.45077076905606 -121.67226938234742) bank119337 +119338 POINT(48.49524306494213 -122.39164316004808) bank119338 +119339 POINT(47.198816725373895 -122.69994680411241) bank119339 +119340 POINT(48.160765533298175 -121.67935622810197) bank119340 +119341 POINT(47.64456570449826 -122.00357850632946) bank119341 +119342 POINT(46.96598387058162 -122.85035052387984) bank119342 +119343 POINT(47.092903575055615 -121.70223963899332) bank119343 +119344 POINT(47.12680756578973 -122.58132027968523) bank119344 +119345 POINT(47.48408158395553 -121.3358293430741) bank119345 +119346 POINT(48.32376365998712 -122.17093393508692) bank119346 +119347 POINT(47.198829920808826 -122.45866592004336) bank119347 +119348 POINT(47.8101590838472 -121.60327969471302) bank119348 +119349 POINT(48.53522744892336 -121.9072882038946) bank119349 +119350 POINT(46.76281416091153 -121.63713101433372) bank119350 +119351 POINT(46.88333164776256 -122.41943182517602) bank119351 +119352 POINT(47.296607884140386 -121.57387657751563) bank119352 +119353 POINT(47.18992649134845 -122.93794495158188) bank119353 +119354 POINT(46.622247961340285 -123.24217941815085) bank119354 +119355 POINT(48.2020585258466 -122.1748890238326) bank119355 +119356 POINT(46.67341763504699 -121.37631075517331) bank119356 +119357 POINT(48.324800416876755 -123.17351873733554) bank119357 +119358 POINT(46.79204381071842 -122.73002613917072) bank119358 +119359 POINT(46.67213898578897 -121.53636636256857) bank119359 +119360 POINT(47.04214342028437 -121.77347404485609) bank119360 +119361 POINT(47.55807686820671 -122.3546209476642) bank119361 +119362 POINT(48.510665765073504 -122.78942398393596) bank119362 +119363 POINT(47.96456774811031 -121.3925560562811) bank119363 +119364 POINT(48.246006710708755 -123.16081649570852) bank119364 +119365 POINT(46.909341118036046 -121.75951466409619) bank119365 +119366 POINT(48.485161085012834 -122.67844639393458) bank119366 +119367 POINT(47.549461977657806 -122.89685296907135) bank119367 +119368 POINT(48.30323130290439 -121.91301995270729) bank119368 +119369 POINT(47.53617502194541 -123.08463157564803) bank119369 +119370 POINT(48.040339662520225 -122.48419508265194) bank119370 +119371 POINT(48.596655356902225 -122.56609228984433) bank119371 +119372 POINT(47.372927285757214 -123.07147993484503) bank119372 +119373 POINT(48.55425053258846 -121.59912324530066) bank119373 +119374 POINT(48.255933901774874 -122.64753500637079) bank119374 +119375 POINT(46.98150385135778 -121.83253492426702) bank119375 +119376 POINT(48.21428354208222 -122.34781453711624) bank119376 +119377 POINT(48.53213604892963 -121.85016871210206) bank119377 +119378 POINT(48.57373722335353 -121.68847406262798) bank119378 +119379 POINT(48.560420371911185 -122.01945805843962) bank119379 +119380 POINT(48.55692569131921 -123.26221077323714) bank119380 +119381 POINT(47.051349066580926 -122.61322724847358) bank119381 +119382 POINT(47.67542718728199 -121.56322530472269) bank119382 +119383 POINT(48.15881178616394 -122.03579840527081) bank119383 +119384 POINT(48.05130970479059 -121.79296233796434) bank119384 +119385 POINT(48.09970838511947 -122.48311841106582) bank119385 +119386 POINT(48.54845053818863 -122.14392167371304) bank119386 +119387 POINT(47.100472636474294 -121.5786229068112) bank119387 +119388 POINT(47.295081477291596 -123.27275998372531) bank119388 +119389 POINT(47.380214500761184 -122.26404373227466) bank119389 +119390 POINT(48.43179762859008 -122.62406998603493) bank119390 +119391 POINT(48.49397633725391 -122.46904932507341) bank119391 +119392 POINT(46.85310298261998 -121.81447434983706) bank119392 +119393 POINT(48.09329904008192 -122.57712173793402) bank119393 +119394 POINT(46.643571194316365 -122.25364827165836) bank119394 +119395 POINT(47.02359210005698 -122.66794271670072) bank119395 +119396 POINT(46.99466052957184 -122.18876774740878) bank119396 +119397 POINT(48.08138235063785 -121.97897938617434) bank119397 +119398 POINT(47.838247016557645 -122.64047660917596) bank119398 +119399 POINT(46.67675791667586 -121.3490449363307) bank119399 +119400 POINT(47.86671845557567 -122.62104707065004) bank119400 +119401 POINT(48.169072029973066 -121.63432212647028) bank119401 +119402 POINT(46.80992414106155 -121.98290495176795) bank119402 +119403 POINT(48.32089330808771 -121.71182747853396) bank119403 +119404 POINT(48.14093851758674 -122.1905221217329) bank119404 +119405 POINT(47.01113736826621 -122.63751533441372) bank119405 +119406 POINT(48.25657551668745 -122.9166713275517) bank119406 +119407 POINT(46.72334276397138 -121.8412676994215) bank119407 +119408 POINT(47.62828515032675 -122.43385251944075) bank119408 +119409 POINT(47.7046417221819 -123.18785667344665) bank119409 +119410 POINT(48.54280600279537 -122.5065520435292) bank119410 +119411 POINT(48.355110800839974 -122.92008811020932) bank119411 +119412 POINT(48.129299058129924 -123.05507055217494) bank119412 +119413 POINT(47.77009684757967 -123.28911096955154) bank119413 +119414 POINT(47.87242483141207 -123.17474754059754) bank119414 +119415 POINT(47.18730955308133 -122.35245373602272) bank119415 +119416 POINT(47.558239019487274 -121.35358343693552) bank119416 +119417 POINT(47.2559867663407 -121.35272922443123) bank119417 +119418 POINT(46.836966074405495 -121.87802012323245) bank119418 +119419 POINT(48.27899850163881 -123.2783867195698) bank119419 +119420 POINT(47.049359133611155 -123.00437875268277) bank119420 +119421 POINT(46.82436331146282 -122.6411411045402) bank119421 +119422 POINT(47.249351382509786 -122.37217031609484) bank119422 +119423 POINT(47.537662898750824 -122.61297112820749) bank119423 +119424 POINT(47.49906130891269 -122.39664320810549) bank119424 +119425 POINT(48.560941518013195 -122.32414054018457) bank119425 +119426 POINT(46.67620459399239 -122.7072722168323) bank119426 +119427 POINT(47.965347708763396 -122.176729701446) bank119427 +119428 POINT(48.27654588211921 -122.47731758104104) bank119428 +119429 POINT(47.363138396756675 -121.37906116100577) bank119429 +119430 POINT(47.836338983025485 -123.09919234447293) bank119430 +119431 POINT(47.074252803380276 -123.28945403840737) bank119431 +119432 POINT(47.78553540177849 -122.63910355720219) bank119432 +119433 POINT(47.62237580393901 -122.18040684012611) bank119433 +119434 POINT(46.97903723350675 -121.6737738478859) bank119434 +119435 POINT(47.39149920533003 -123.12549000883062) bank119435 +119436 POINT(46.97975034964407 -122.53787656689862) bank119436 +119437 POINT(47.36474541025132 -122.44937534056726) bank119437 +119438 POINT(47.498928069839934 -123.10025771870475) bank119438 +119439 POINT(48.60068763918239 -122.37818294981973) bank119439 +119440 POINT(47.65884717868937 -122.23270958849524) bank119440 +119441 POINT(46.850959528865225 -122.94936190980643) bank119441 +119442 POINT(46.778754950979916 -121.77843482533977) bank119442 +119443 POINT(47.06974206038121 -123.3156011988828) bank119443 +119444 POINT(47.06845440974179 -122.05613974079635) bank119444 +119445 POINT(47.78353367780783 -122.68752944467066) bank119445 +119446 POINT(48.10585658102324 -122.83057121728258) bank119446 +119447 POINT(47.66834124684025 -121.77046128928909) bank119447 +119448 POINT(47.830221257030274 -121.9898288349268) bank119448 +119449 POINT(47.04558740013778 -123.09909867237221) bank119449 +119450 POINT(47.51060487067924 -121.53325494416555) bank119450 +119451 POINT(48.42929520100002 -121.67228347574388) bank119451 +119452 POINT(48.08679156116497 -122.33545157188496) bank119452 +119453 POINT(48.37090187239863 -122.02023817617595) bank119453 +119454 POINT(48.484231236463785 -121.7378429487138) bank119454 +119455 POINT(47.199001876969625 -122.58958214060502) bank119455 +119456 POINT(47.825592385737586 -122.67402798680276) bank119456 +119457 POINT(48.123759106244265 -121.47589404355237) bank119457 +119458 POINT(48.37951781410359 -122.67117325410224) bank119458 +119459 POINT(46.687501299942596 -122.70934399334068) bank119459 +119460 POINT(47.55475625359536 -123.01378300452693) bank119460 +119461 POINT(47.14707812109725 -122.03996212715464) bank119461 +119462 POINT(48.40438630337017 -123.19631598581128) bank119462 +119463 POINT(47.90847685620117 -122.91647278291116) bank119463 +119464 POINT(47.937483557055344 -122.96097309833279) bank119464 +119465 POINT(47.976252419162904 -121.9320621327561) bank119465 +119466 POINT(48.0492224324616 -123.03491038108373) bank119466 +119467 POINT(48.01931141604805 -122.47972777563314) bank119467 +119468 POINT(47.901108252815526 -122.63988927513363) bank119468 +119469 POINT(47.626029462113515 -122.64235161351047) bank119469 +119470 POINT(46.821080362251884 -121.9760217741894) bank119470 +119471 POINT(47.7436284868536 -123.0239141087922) bank119471 +119472 POINT(47.42139214890356 -122.86009345741027) bank119472 +119473 POINT(46.823139874567076 -121.44307029593405) bank119473 +119474 POINT(47.3981135188983 -121.81560255962637) bank119474 +119475 POINT(46.924143908464664 -122.00472699501188) bank119475 +119476 POINT(48.1317899570672 -122.57750728509977) bank119476 +119477 POINT(47.64083638795122 -122.60481999161634) bank119477 +119478 POINT(48.21307337801777 -122.4606869934503) bank119478 +119479 POINT(46.78628435929828 -123.29090396483447) bank119479 +119480 POINT(47.18855138149671 -123.14415091859375) bank119480 +119481 POINT(48.24154099148865 -122.49911586983269) bank119481 +119482 POINT(46.83522176657868 -122.22847256805615) bank119482 +119483 POINT(48.03782959746862 -122.52063341850834) bank119483 +119484 POINT(47.57430111787983 -121.7307941327856) bank119484 +119485 POINT(47.54974385241104 -122.75921364749428) bank119485 +119486 POINT(47.56005073099569 -121.7121735397776) bank119486 +119487 POINT(46.937847699788605 -122.85217285979866) bank119487 +119488 POINT(47.58723064686052 -121.92662362396301) bank119488 +119489 POINT(47.46684760454911 -121.49767872025997) bank119489 +119490 POINT(47.9457022310857 -122.27716250055785) bank119490 +119491 POINT(47.38624225541001 -121.38475645602956) bank119491 +119492 POINT(47.20872663398108 -122.1091040482317) bank119492 +119493 POINT(48.22318083696529 -121.89216074312318) bank119493 +119494 POINT(48.44238705750859 -123.00908976191084) bank119494 +119495 POINT(46.957252188727495 -121.44917393963837) bank119495 +119496 POINT(47.067293828970655 -122.6262638108957) bank119496 +119497 POINT(46.852651461666476 -122.75212677741021) bank119497 +119498 POINT(47.18598301740659 -123.25695816114647) bank119498 +119499 POINT(47.75808706287215 -121.92661955175019) bank119499 +119500 POINT(47.02204367424881 -122.13489255674219) bank119500 +119501 POINT(47.772948363891835 -121.56953731868177) bank119501 +119502 POINT(47.234928008522004 -121.80814924839538) bank119502 +119503 POINT(46.75787846291094 -122.34063320047154) bank119503 +119504 POINT(48.03929125716586 -123.19112098934494) bank119504 +119505 POINT(48.098749592303776 -121.82463114214242) bank119505 +119506 POINT(46.83085290033314 -122.29508880588818) bank119506 +119507 POINT(48.57184051931129 -123.1339654420664) bank119507 +119508 POINT(47.826192259109206 -121.9898095185999) bank119508 +119509 POINT(46.93280069431497 -122.59230633282681) bank119509 +119510 POINT(46.6525164435067 -121.46263912099029) bank119510 +119511 POINT(48.12760920796113 -122.55951447010737) bank119511 +119512 POINT(46.96328290081226 -121.86937338219751) bank119512 +119513 POINT(46.776595287865625 -122.01543975164589) bank119513 +119514 POINT(48.022813042544996 -121.38486962682646) bank119514 +119515 POINT(46.98404384331546 -122.44132763030684) bank119515 +119516 POINT(46.76814183610955 -122.73750020732479) bank119516 +119517 POINT(46.93786009429937 -122.48090605585388) bank119517 +119518 POINT(48.060173656028084 -121.63199189565687) bank119518 +119519 POINT(46.78245999949823 -121.87667983919249) bank119519 +119520 POINT(48.2154887746596 -122.062224503914) bank119520 +119521 POINT(47.93362359507134 -122.14853641310651) bank119521 +119522 POINT(48.02267171373384 -123.28880629384555) bank119522 +119523 POINT(48.22480649807592 -121.43265995005609) bank119523 +119524 POINT(47.60590026244913 -122.7323064279407) bank119524 +119525 POINT(48.53699990296132 -122.45925436298438) bank119525 +119526 POINT(48.38527502606006 -121.68864546611628) bank119526 +119527 POINT(48.03434886608483 -122.08126019890165) bank119527 +119528 POINT(47.18832075009166 -122.81135694264681) bank119528 +119529 POINT(46.88633507260243 -122.71943606206433) bank119529 +119530 POINT(48.60156332342988 -121.39848924415989) bank119530 +119531 POINT(47.81956031005747 -121.82984650114732) bank119531 +119532 POINT(47.545217105352776 -121.41512928384472) bank119532 +119533 POINT(47.76172203105878 -121.70558012399391) bank119533 +119534 POINT(47.7456269108814 -123.009081331979) bank119534 +119535 POINT(47.57053813385969 -122.28462220601288) bank119535 +119536 POINT(47.1279353225558 -123.0942863250043) bank119536 +119537 POINT(47.87700209402755 -122.94815091622702) bank119537 +119538 POINT(47.19152617638731 -122.15067136737851) bank119538 +119539 POINT(47.48040293500104 -122.68741847512793) bank119539 +119540 POINT(47.69667649168692 -122.92437790008172) bank119540 +119541 POINT(47.60406433278424 -123.2391243830129) bank119541 +119542 POINT(48.033017167756206 -121.66469368368935) bank119542 +119543 POINT(47.08200872439327 -122.70664025062793) bank119543 +119544 POINT(48.05521134631668 -121.55576495711) bank119544 +119545 POINT(47.00707115883802 -121.92960608671532) bank119545 +119546 POINT(48.551102507457344 -123.04301784291188) bank119546 +119547 POINT(48.105231755397085 -122.02912758461963) bank119547 +119548 POINT(47.24707675468424 -122.33123680501421) bank119548 +119549 POINT(46.90690924834179 -122.41696629205187) bank119549 +119550 POINT(47.66129070981671 -121.41747176022852) bank119550 +119551 POINT(46.75834918262857 -122.15933293672445) bank119551 +119552 POINT(47.485283294827305 -122.91661865206936) bank119552 +119553 POINT(46.65186097776296 -121.56224829168225) bank119553 +119554 POINT(46.72298150554112 -122.89523133863719) bank119554 +119555 POINT(47.07566359442781 -122.93441762348036) bank119555 +119556 POINT(46.77961965580572 -122.50673352962778) bank119556 +119557 POINT(46.98105786039244 -122.23736751471277) bank119557 +119558 POINT(46.87959399695184 -121.95961042448587) bank119558 +119559 POINT(47.676305837998946 -121.40222561537367) bank119559 +119560 POINT(48.598802721676236 -123.09670210435219) bank119560 +119561 POINT(46.87692562375572 -121.80238402495958) bank119561 +119562 POINT(46.73704605238981 -122.89351571218482) bank119562 +119563 POINT(48.47864883640921 -122.35222103748947) bank119563 +119564 POINT(47.77881483090706 -121.4438803997271) bank119564 +119565 POINT(48.030661695559196 -122.8878403480564) bank119565 +119566 POINT(46.752864398069406 -122.20257367930323) bank119566 +119567 POINT(46.827580546712994 -122.41387202691472) bank119567 +119568 POINT(48.48806538822616 -122.76767212034717) bank119568 +119569 POINT(46.95958816191089 -122.23402390466204) bank119569 +119570 POINT(48.2612946565878 -122.3692881577913) bank119570 +119571 POINT(48.0658420645309 -123.2314697657602) bank119571 +119572 POINT(46.90007233744206 -123.11779927968496) bank119572 +119573 POINT(47.90888205618518 -123.14843155400044) bank119573 +119574 POINT(48.221606907936945 -122.87571797282567) bank119574 +119575 POINT(48.40178823333084 -122.79165866407344) bank119575 +119576 POINT(48.33535074229267 -123.26017891704123) bank119576 +119577 POINT(47.000793370975025 -123.05628564629478) bank119577 +119578 POINT(48.072695019367615 -122.52135846176056) bank119578 +119579 POINT(47.720035731041676 -121.52700352757032) bank119579 +119580 POINT(47.79140330245387 -122.67733084685855) bank119580 +119581 POINT(47.06597004243983 -123.22165627596243) bank119581 +119582 POINT(47.60392335373459 -122.85254342927142) bank119582 +119583 POINT(47.4231236631443 -122.07473328824985) bank119583 +119584 POINT(48.25905018123787 -123.2294450263161) bank119584 +119585 POINT(46.69747379578479 -122.30177344880322) bank119585 +119586 POINT(48.57114853098963 -122.62812642287795) bank119586 +119587 POINT(48.08567365209262 -122.19157416595719) bank119587 +119588 POINT(47.336528234082536 -121.5912342391078) bank119588 +119589 POINT(47.48345408702163 -121.68433090441734) bank119589 +119590 POINT(46.89484066302013 -122.56676064656143) bank119590 +119591 POINT(47.28212967188479 -122.24931944959721) bank119591 +119592 POINT(47.36861602003882 -121.52756311387533) bank119592 +119593 POINT(48.21460765623233 -121.68795340069398) bank119593 +119594 POINT(47.873952761674545 -123.22156517725507) bank119594 +119595 POINT(47.11479677998051 -122.53631935927925) bank119595 +119596 POINT(48.1342711382162 -122.16852028951735) bank119596 +119597 POINT(47.508788896256114 -122.41854783743263) bank119597 +119598 POINT(48.074503676556 -123.29667704374718) bank119598 +119599 POINT(48.34513997004838 -122.44639097382517) bank119599 +119600 POINT(47.16887397472324 -122.39531782321791) bank119600 +119601 POINT(48.22086938602668 -121.50137488343593) bank119601 +119602 POINT(47.713174823605065 -122.65498509237673) bank119602 +119603 POINT(47.81653972013382 -122.26965107760564) bank119603 +119604 POINT(48.166958752536175 -122.29271495643358) bank119604 +119605 POINT(46.69922405341635 -122.05735012365624) bank119605 +119606 POINT(47.001913911672325 -122.01229977144484) bank119606 +119607 POINT(48.24288605598348 -122.92107480647232) bank119607 +119608 POINT(47.30571246660516 -122.22986103946182) bank119608 +119609 POINT(47.297095526371486 -121.51278922028396) bank119609 +119610 POINT(48.09323281920147 -121.33377170046413) bank119610 +119611 POINT(47.870528034519886 -122.70528755608258) bank119611 +119612 POINT(47.98966379323594 -122.03384591974796) bank119612 +119613 POINT(48.21323638128618 -123.11519367109993) bank119613 +119614 POINT(46.645974732909735 -123.06611168295333) bank119614 +119615 POINT(47.102020303757506 -122.41481087233518) bank119615 +119616 POINT(46.78128903027766 -122.71631573474063) bank119616 +119617 POINT(47.79099522554152 -122.392437256077) bank119617 +119618 POINT(47.85082235115382 -122.34099512159655) bank119618 +119619 POINT(47.066139604631715 -123.05880862178614) bank119619 +119620 POINT(47.34672772927498 -122.53641337399272) bank119620 +119621 POINT(48.587767539165554 -122.76514890718698) bank119621 +119622 POINT(48.31236945871045 -122.01700454416869) bank119622 +119623 POINT(47.30729009416507 -122.85062942790468) bank119623 +119624 POINT(47.50788512263571 -122.1343641642107) bank119624 +119625 POINT(47.38247499172063 -121.5599193707684) bank119625 +119626 POINT(47.541244068018294 -123.11793467727387) bank119626 +119627 POINT(48.10649762993822 -123.08612391346898) bank119627 +119628 POINT(48.54676436553893 -122.95272437229214) bank119628 +119629 POINT(47.882769303900645 -121.88439136261405) bank119629 +119630 POINT(48.185585377112936 -123.16450702576799) bank119630 +119631 POINT(47.361712335584464 -123.19988656365811) bank119631 +119632 POINT(46.830672937943085 -122.32180705126369) bank119632 +119633 POINT(46.926369617841914 -123.18993566645467) bank119633 +119634 POINT(48.50113022796037 -122.27590329681266) bank119634 +119635 POINT(46.819053507596365 -122.23292541774872) bank119635 +119636 POINT(47.46019156085246 -122.28952527503105) bank119636 +119637 POINT(47.0470821748601 -122.72582457868691) bank119637 +119638 POINT(47.25794253608239 -122.05562401486127) bank119638 +119639 POINT(47.73544826565155 -123.2800042707609) bank119639 +119640 POINT(47.028901278553775 -121.63583658926602) bank119640 +119641 POINT(48.27120232041393 -122.94666832337522) bank119641 +119642 POINT(47.280378375734955 -123.25799694106205) bank119642 +119643 POINT(47.15056299599553 -121.49178992561704) bank119643 +119644 POINT(46.9121911086492 -122.79978850443382) bank119644 +119645 POINT(48.027260780576036 -122.29879922127486) bank119645 +119646 POINT(46.79879637901286 -122.24321181416708) bank119646 +119647 POINT(46.84695297681188 -121.66235593215808) bank119647 +119648 POINT(48.33042141453508 -122.18158146474194) bank119648 +119649 POINT(47.256411363658735 -121.59631313035486) bank119649 +119650 POINT(48.3205302180374 -122.88317403829338) bank119650 +119651 POINT(46.79310476061144 -122.92026061051182) bank119651 +119652 POINT(48.550889054172124 -122.664577177238) bank119652 +119653 POINT(48.567538719246826 -123.04918541160161) bank119653 +119654 POINT(46.92063258526156 -122.52081729719659) bank119654 +119655 POINT(47.226450755985844 -122.2289128854793) bank119655 +119656 POINT(47.28572141495479 -122.9662460178026) bank119656 +119657 POINT(47.253553055005575 -123.05251765047822) bank119657 +119658 POINT(48.18728386562157 -122.26422009747002) bank119658 +119659 POINT(48.154006778901966 -121.6046821321855) bank119659 +119660 POINT(46.7965482963837 -123.14905024920213) bank119660 +119661 POINT(47.57834320472965 -121.33336083494257) bank119661 +119662 POINT(47.88700614899635 -121.37442712676975) bank119662 +119663 POINT(47.39392311988465 -123.00217242701062) bank119663 +119664 POINT(47.37578185839764 -121.81500834485915) bank119664 +119665 POINT(47.18131891739634 -122.4371350531057) bank119665 +119666 POINT(48.2606311131196 -122.7578452887929) bank119666 +119667 POINT(46.85638268214344 -122.71051735868222) bank119667 +119668 POINT(48.113950698785494 -123.15028609038539) bank119668 +119669 POINT(47.514452611422364 -122.75077071258693) bank119669 +119670 POINT(46.619420701359594 -121.76479952360368) bank119670 +119671 POINT(48.28524923996017 -121.33531954503812) bank119671 +119672 POINT(47.25265640978 -121.44772356362164) bank119672 +119673 POINT(48.28777392164055 -121.50326733845951) bank119673 +119674 POINT(47.017952053697776 -122.08464141297652) bank119674 +119675 POINT(46.912433189814905 -122.08758144345218) bank119675 +119676 POINT(48.35329485152308 -121.88893407011518) bank119676 +119677 POINT(48.371789472938445 -122.73589417256198) bank119677 +119678 POINT(47.729659217478215 -122.56618960138812) bank119678 +119679 POINT(47.792513935759196 -121.37935522607202) bank119679 +119680 POINT(47.07868670169527 -123.11425854752112) bank119680 +119681 POINT(47.13177969543482 -122.43080940213036) bank119681 +119682 POINT(46.938398166248206 -122.25349011853997) bank119682 +119683 POINT(46.95481903810611 -122.71690087435648) bank119683 +119684 POINT(47.442606640091874 -121.59459831660575) bank119684 +119685 POINT(47.881523325149324 -122.98416709740636) bank119685 +119686 POINT(46.646058865081145 -121.98765180804702) bank119686 +119687 POINT(47.21398325364616 -122.3580024149513) bank119687 +119688 POINT(47.54814005371059 -123.02607057661449) bank119688 +119689 POINT(47.53294489709655 -122.75089625671878) bank119689 +119690 POINT(46.60662497682553 -122.05975836541828) bank119690 +119691 POINT(48.276701462970145 -122.63044354792831) bank119691 +119692 POINT(48.478905908197476 -122.54729568938578) bank119692 +119693 POINT(46.80047071000597 -122.73591350365498) bank119693 +119694 POINT(48.21974979475938 -123.1601389927453) bank119694 +119695 POINT(47.28721743336774 -122.64232688458614) bank119695 +119696 POINT(47.24466733657339 -122.58499241826281) bank119696 +119697 POINT(47.57938284740081 -122.49343661296683) bank119697 +119698 POINT(47.50459217565855 -122.33836368358932) bank119698 +119699 POINT(47.90776882407567 -121.59282246216951) bank119699 +119700 POINT(46.99105061092048 -122.07758987482103) bank119700 +119701 POINT(47.68532794691805 -122.83695315136892) bank119701 +119702 POINT(46.62093039879688 -122.95856544424885) bank119702 +119703 POINT(47.79667634701701 -123.0607420626314) bank119703 +119704 POINT(46.679924710585475 -122.06464768014611) bank119704 +119705 POINT(47.992636912899755 -121.35226428998811) bank119705 +119706 POINT(46.949303535526546 -121.50334010768186) bank119706 +119707 POINT(48.33697206156722 -122.40678816110787) bank119707 +119708 POINT(47.165494709961834 -123.17188685803433) bank119708 +119709 POINT(46.636793465820666 -121.81192197098278) bank119709 +119710 POINT(47.4689637144993 -121.89377653258552) bank119710 +119711 POINT(48.00427424119157 -121.52980338744113) bank119711 +119712 POINT(47.44012530393931 -122.06902117179685) bank119712 +119713 POINT(48.02896481146934 -122.93089074823867) bank119713 +119714 POINT(48.5645903428328 -121.8717060397851) bank119714 +119715 POINT(46.66404088295862 -122.11652096219179) bank119715 +119716 POINT(48.0293367713033 -122.77601086993178) bank119716 +119717 POINT(48.03585274763595 -121.37927120085251) bank119717 +119718 POINT(47.008310104643996 -121.68315210666016) bank119718 +119719 POINT(47.873753122422414 -122.68123098259689) bank119719 +119720 POINT(46.670233055686275 -122.48919830305024) bank119720 +119721 POINT(48.4363367395447 -122.89073235711996) bank119721 +119722 POINT(48.528832613494096 -122.77841048201292) bank119722 +119723 POINT(48.41873950185478 -121.83240621097265) bank119723 +119724 POINT(47.22703335415029 -121.58787722911033) bank119724 +119725 POINT(48.095492847726966 -123.12728836584688) bank119725 +119726 POINT(48.17818138414556 -121.94803687074482) bank119726 +119727 POINT(48.553158626230115 -122.36463642227099) bank119727 +119728 POINT(47.55286751332909 -121.63102929932334) bank119728 +119729 POINT(47.25848029393786 -122.34121878005698) bank119729 +119730 POINT(48.48041304321125 -122.66600214446824) bank119730 +119731 POINT(47.150643815059766 -122.88608805183263) bank119731 +119732 POINT(48.26712816892014 -121.55470213705665) bank119732 +119733 POINT(48.18658386242978 -121.96941832543652) bank119733 +119734 POINT(47.604854008967855 -121.49152930914903) bank119734 +119735 POINT(48.18975421689391 -121.36639321058587) bank119735 +119736 POINT(47.36189683275705 -122.0649862137719) bank119736 +119737 POINT(48.14802541436574 -123.25121917745986) bank119737 +119738 POINT(48.05252547105654 -122.274287282438) bank119738 +119739 POINT(47.06737865015581 -122.43134501736483) bank119739 +119740 POINT(48.32384707596225 -122.00166242775953) bank119740 +119741 POINT(47.77378144787901 -121.64253548189981) bank119741 +119742 POINT(47.60922037903935 -121.9408639091117) bank119742 +119743 POINT(48.54370322434348 -122.65550424023536) bank119743 +119744 POINT(46.757174359062326 -122.2205494814) bank119744 +119745 POINT(47.892342369701176 -121.47437565835001) bank119745 +119746 POINT(46.95818879514749 -121.94377866846955) bank119746 +119747 POINT(48.45134894651713 -122.69354418076352) bank119747 +119748 POINT(47.72218011360819 -121.62388134710973) bank119748 +119749 POINT(47.359670416466514 -122.35666728869383) bank119749 +119750 POINT(47.580256841264614 -122.37625111319879) bank119750 +119751 POINT(46.92758983421008 -123.24149169316786) bank119751 +119752 POINT(47.086394656061 -122.30429219842453) bank119752 +119753 POINT(47.40471233782531 -123.24114221257967) bank119753 +119754 POINT(47.15017345816822 -122.71022169365803) bank119754 +119755 POINT(47.28750921556846 -123.12098840763866) bank119755 +119756 POINT(47.428626037701406 -121.89651599336862) bank119756 +119757 POINT(47.56246721889723 -122.24708160769045) bank119757 +119758 POINT(47.923468124278315 -121.54867371976927) bank119758 +119759 POINT(47.965255126840276 -123.28426303960453) bank119759 +119760 POINT(47.760552628595974 -122.4181284490839) bank119760 +119761 POINT(47.15201992341654 -122.81269764050381) bank119761 +119762 POINT(47.18674202953579 -122.17253558724467) bank119762 +119763 POINT(47.395753588912626 -122.99749494285223) bank119763 +119764 POINT(46.90206694436372 -122.77660995491777) bank119764 +119765 POINT(48.189851768747225 -121.74737451420916) bank119765 +119766 POINT(47.233712023678024 -122.54830467518367) bank119766 +119767 POINT(48.523897370937455 -121.46296738944113) bank119767 +119768 POINT(47.013967401423706 -121.42218784178077) bank119768 +119769 POINT(47.47477107450912 -122.38802163843269) bank119769 +119770 POINT(47.927448049045516 -121.77587609392413) bank119770 +119771 POINT(48.31888148989626 -122.99912016251018) bank119771 +119772 POINT(46.846781229409444 -123.25676824216754) bank119772 +119773 POINT(47.129501857677255 -121.9880883738639) bank119773 +119774 POINT(47.18317211820652 -123.12449259810394) bank119774 +119775 POINT(48.552756069466675 -121.45807676451116) bank119775 +119776 POINT(48.15750012359244 -122.57872157502055) bank119776 +119777 POINT(48.59198339201409 -123.261541004886) bank119777 +119778 POINT(47.557338022952614 -122.82653776885323) bank119778 +119779 POINT(47.71860182987241 -123.16816254099389) bank119779 +119780 POINT(47.67382854677988 -122.66097716591605) bank119780 +119781 POINT(46.71428215130848 -122.71279662452459) bank119781 +119782 POINT(46.79680693446494 -121.57218410207574) bank119782 +119783 POINT(47.75476604445965 -121.54878167073593) bank119783 +119784 POINT(46.79065826961301 -122.90429562682243) bank119784 +119785 POINT(47.710918314861026 -123.20124022434705) bank119785 +119786 POINT(47.20909537565682 -121.59901845381665) bank119786 +119787 POINT(47.90093838297476 -122.69779215010912) bank119787 +119788 POINT(47.943789409817626 -122.03759339661708) bank119788 +119789 POINT(47.827370843100056 -121.82048755556366) bank119789 +119790 POINT(47.99474660456479 -121.39800960280033) bank119790 +119791 POINT(47.02451258273047 -122.19246305947878) bank119791 +119792 POINT(46.850187953702 -122.4565158887931) bank119792 +119793 POINT(48.456835199653774 -121.427698881776) bank119793 +119794 POINT(48.20117704426865 -123.20264103366105) bank119794 +119795 POINT(47.125133543671055 -122.81638259777037) bank119795 +119796 POINT(47.74963431651611 -122.98597386829668) bank119796 +119797 POINT(46.66125661066092 -122.40487802008974) bank119797 +119798 POINT(48.27285723576767 -121.34786132669906) bank119798 +119799 POINT(47.25083892561997 -122.09514069625473) bank119799 +119800 POINT(48.56343277242439 -121.75144902456012) bank119800 +119801 POINT(46.80793546872507 -121.95053318148778) bank119801 +119802 POINT(48.022348046053544 -121.74749990020102) bank119802 +119803 POINT(47.996074767653965 -121.39620257574573) bank119803 +119804 POINT(48.38341528246499 -122.8717021413381) bank119804 +119805 POINT(47.28130333010664 -122.72739711772711) bank119805 +119806 POINT(48.35629399057847 -122.41448695404621) bank119806 +119807 POINT(47.684081539971885 -122.85975113023711) bank119807 +119808 POINT(48.30049388526752 -121.5771466629595) bank119808 +119809 POINT(48.234752202490085 -121.40377876987489) bank119809 +119810 POINT(47.320081477907536 -122.1930039502925) bank119810 +119811 POINT(46.94032563536594 -122.16101296245043) bank119811 +119812 POINT(46.7975178458598 -122.66237701842317) bank119812 +119813 POINT(48.243254843050984 -122.09989552064555) bank119813 +119814 POINT(48.362743810280996 -123.288310486981) bank119814 +119815 POINT(46.8151227138221 -122.87488508941422) bank119815 +119816 POINT(46.73757877765555 -121.96609644765103) bank119816 +119817 POINT(48.287354516322594 -122.43189133153689) bank119817 +119818 POINT(47.184368784721265 -122.14652847849675) bank119818 +119819 POINT(46.895100711355724 -122.08373858202245) bank119819 +119820 POINT(48.11297625018539 -122.79006644608394) bank119820 +119821 POINT(46.97634569561403 -123.12363397689742) bank119821 +119822 POINT(47.89029384744885 -122.68440920356741) bank119822 +119823 POINT(47.09208039197547 -122.65784787965465) bank119823 +119824 POINT(48.290643191345445 -121.78524205635048) bank119824 +119825 POINT(47.23137038728381 -121.50200559886095) bank119825 +119826 POINT(47.169808391710525 -122.6571020544678) bank119826 +119827 POINT(46.93136335299662 -122.33570036188455) bank119827 +119828 POINT(46.80355418295231 -122.93988940988277) bank119828 +119829 POINT(48.09171720553024 -121.44092079215251) bank119829 +119830 POINT(48.326684931386445 -122.751783928478) bank119830 +119831 POINT(48.13441681474524 -121.77169901194533) bank119831 +119832 POINT(48.320986714125176 -122.99469552852803) bank119832 +119833 POINT(48.28807936700768 -122.99991459792415) bank119833 +119834 POINT(47.57911700588522 -121.70124277875418) bank119834 +119835 POINT(48.34552472434438 -122.98648863703936) bank119835 +119836 POINT(46.70125027480513 -121.98848666698986) bank119836 +119837 POINT(48.511324866244856 -122.91423598923382) bank119837 +119838 POINT(48.39231049574559 -122.76899038402057) bank119838 +119839 POINT(48.247512825179 -121.61447675539364) bank119839 +119840 POINT(48.54548846824458 -121.76062975188728) bank119840 +119841 POINT(48.2714608763669 -121.8171601191601) bank119841 +119842 POINT(47.49676141571408 -121.75166972818879) bank119842 +119843 POINT(47.488387588934515 -121.60289708894302) bank119843 +119844 POINT(47.93572353342635 -122.035669794553) bank119844 +119845 POINT(47.90907002140722 -122.88577745227585) bank119845 +119846 POINT(47.87143415356877 -122.3141473112919) bank119846 +119847 POINT(47.293533110894245 -122.19398979678556) bank119847 +119848 POINT(47.78609035676137 -122.87977809755547) bank119848 +119849 POINT(47.95604796022414 -122.16822388289195) bank119849 +119850 POINT(47.074191951276006 -122.42857346027682) bank119850 +119851 POINT(47.60197115510392 -123.00491494749683) bank119851 +119852 POINT(48.52107824848497 -121.35796026397415) bank119852 +119853 POINT(47.99029153435336 -122.1507602818271) bank119853 +119854 POINT(46.972694693805856 -122.54747229825169) bank119854 +119855 POINT(47.650677823329794 -123.1256918215589) bank119855 +119856 POINT(46.6319510817762 -121.71344138119547) bank119856 +119857 POINT(47.93527848761715 -122.97111442636262) bank119857 +119858 POINT(46.74327847577932 -122.1563991225548) bank119858 +119859 POINT(48.41514806155009 -121.68665291655515) bank119859 +119860 POINT(48.08789226504811 -122.77121561005524) bank119860 +119861 POINT(47.81182229793741 -122.76494455354847) bank119861 +119862 POINT(48.1570057036852 -121.96950876695404) bank119862 +119863 POINT(48.45967905570604 -121.96395275046272) bank119863 +119864 POINT(47.05764666880284 -122.10700580627947) bank119864 +119865 POINT(47.85367558507912 -122.22511979811726) bank119865 +119866 POINT(47.512267229139304 -122.7925729698471) bank119866 +119867 POINT(47.020271264439884 -121.653314679211) bank119867 +119868 POINT(48.28958470326894 -122.53189171439689) bank119868 +119869 POINT(47.219914318010915 -122.2601982139925) bank119869 +119870 POINT(48.44652663265998 -122.38461317591876) bank119870 +119871 POINT(47.957395304124994 -121.91708787104626) bank119871 +119872 POINT(47.9177584894692 -122.39852276392772) bank119872 +119873 POINT(48.023691529508355 -121.99421843221941) bank119873 +119874 POINT(47.53728803891844 -122.14405823814961) bank119874 +119875 POINT(47.57874162466135 -121.50259286388228) bank119875 +119876 POINT(47.89773013709269 -122.35752895588945) bank119876 +119877 POINT(47.513410331259394 -123.32758277903989) bank119877 +119878 POINT(47.74767024236443 -123.32876873912682) bank119878 +119879 POINT(48.391152773176366 -121.50204455993698) bank119879 +119880 POINT(48.220887907653555 -121.71717195967584) bank119880 +119881 POINT(47.64928368425379 -121.98426651990991) bank119881 +119882 POINT(47.96844451350557 -122.97308697184874) bank119882 +119883 POINT(48.22319357092896 -121.98899550595202) bank119883 +119884 POINT(48.20713164868269 -122.22447474140658) bank119884 +119885 POINT(48.514888718598115 -122.9395680992103) bank119885 +119886 POINT(48.02356505658977 -123.02537917634969) bank119886 +119887 POINT(48.11560320761383 -122.22509247823734) bank119887 +119888 POINT(48.286319205913 -122.80143693652369) bank119888 +119889 POINT(48.214990523584895 -122.5023576836715) bank119889 +119890 POINT(47.97243572011245 -122.36222724486144) bank119890 +119891 POINT(48.36915922121669 -121.62024213460569) bank119891 +119892 POINT(47.65566110146362 -123.173624977008) bank119892 +119893 POINT(46.84657635753049 -122.11253889172457) bank119893 +119894 POINT(47.547713262106825 -122.99689408899917) bank119894 +119895 POINT(48.05014598633126 -123.0235843384485) bank119895 +119896 POINT(46.79723773266682 -122.30278687815866) bank119896 +119897 POINT(47.88397854694819 -122.06333514963362) bank119897 +119898 POINT(47.10323664378299 -121.45121803274581) bank119898 +119899 POINT(47.615074102520495 -122.221456958485) bank119899 +119900 POINT(48.04419270923068 -121.50503167889958) bank119900 +119901 POINT(48.11476881495438 -122.59932447537942) bank119901 +119902 POINT(47.69491768097767 -121.36998981129607) bank119902 +119903 POINT(47.52296641402992 -121.79080913675675) bank119903 +119904 POINT(48.36047453466295 -122.53999043212787) bank119904 +119905 POINT(48.253495974283084 -122.46942553209614) bank119905 +119906 POINT(48.14798183233631 -122.56943220849436) bank119906 +119907 POINT(47.664677232682884 -121.66330390105267) bank119907 +119908 POINT(47.40268001294854 -121.56386502964834) bank119908 +119909 POINT(46.647858129361495 -123.17864422269636) bank119909 +119910 POINT(46.75709237928529 -121.50994049433493) bank119910 +119911 POINT(48.259351494066756 -122.60921924077323) bank119911 +119912 POINT(47.10549694058004 -121.54486431935015) bank119912 +119913 POINT(47.668943691985284 -121.36035785408085) bank119913 +119914 POINT(47.09792331100056 -122.12434302095883) bank119914 +119915 POINT(48.439426123348966 -122.82785014471841) bank119915 +119916 POINT(47.79280922340554 -121.88012120359963) bank119916 +119917 POINT(46.65399917879843 -122.22125336854504) bank119917 +119918 POINT(47.85818844698371 -123.30037173879293) bank119918 +119919 POINT(46.75701596011343 -121.74430869353827) bank119919 +119920 POINT(46.70192649176494 -121.96621679125853) bank119920 +119921 POINT(47.93538095242216 -122.20538326307496) bank119921 +119922 POINT(47.782517332729505 -121.90032036640497) bank119922 +119923 POINT(47.582063990443174 -123.32599968605209) bank119923 +119924 POINT(47.284491896481775 -122.34546044803294) bank119924 +119925 POINT(47.52839270601314 -121.60756193928924) bank119925 +119926 POINT(47.054213610761465 -123.23735286380715) bank119926 +119927 POINT(46.9026751323288 -121.37871733862222) bank119927 +119928 POINT(47.04965649396607 -121.84598279355428) bank119928 +119929 POINT(48.129656295511396 -123.26976090091972) bank119929 +119930 POINT(48.41729635209932 -122.83426279202189) bank119930 +119931 POINT(48.20142759402119 -121.8661989014808) bank119931 +119932 POINT(47.51990893735348 -121.94630958340947) bank119932 +119933 POINT(46.989745198439564 -122.2013866392726) bank119933 +119934 POINT(47.14231968267598 -122.73690308105662) bank119934 +119935 POINT(47.871979241960894 -122.76776798388426) bank119935 +119936 POINT(47.9092379991297 -122.59845093794503) bank119936 +119937 POINT(47.363999506751476 -122.47496678020248) bank119937 +119938 POINT(46.899677661830665 -121.8849010122237) bank119938 +119939 POINT(47.70426327158737 -123.31638073778629) bank119939 +119940 POINT(48.06644104773341 -123.04448617640399) bank119940 +119941 POINT(47.431904951273125 -122.70968863459741) bank119941 +119942 POINT(48.568140968342696 -122.65199483116633) bank119942 +119943 POINT(46.8811111450743 -121.56603324504616) bank119943 +119944 POINT(46.880220421240494 -122.56639111424448) bank119944 +119945 POINT(48.315311797924316 -123.23118486115072) bank119945 +119946 POINT(48.434972581230554 -122.85861814748928) bank119946 +119947 POINT(47.93271812699929 -121.75243804739303) bank119947 +119948 POINT(48.15566892594355 -122.71993365969237) bank119948 +119949 POINT(47.2325881142676 -122.69384486912968) bank119949 +119950 POINT(47.360817460240455 -121.76146995282062) bank119950 +119951 POINT(47.474655607052235 -123.03509051998499) bank119951 +119952 POINT(46.76740628041767 -123.02865273371327) bank119952 +119953 POINT(47.905582683751874 -122.86278031781256) bank119953 +119954 POINT(47.837925679895704 -122.76688137094062) bank119954 +119955 POINT(47.39967036835874 -121.8734471784308) bank119955 +119956 POINT(47.86882402683021 -122.18760056345133) bank119956 +119957 POINT(46.72420324281854 -122.46487971592846) bank119957 +119958 POINT(48.28960549424644 -122.43142007352233) bank119958 +119959 POINT(47.4402171252806 -123.23435644261832) bank119959 +119960 POINT(47.98455731103013 -122.67708494274859) bank119960 +119961 POINT(48.04462348002347 -121.57787653252187) bank119961 +119962 POINT(48.10852029358852 -121.87285271062255) bank119962 +119963 POINT(46.99006751147619 -122.97901845962144) bank119963 +119964 POINT(47.260619184443065 -121.44873623223857) bank119964 +119965 POINT(47.103844929254755 -121.59601103114987) bank119965 +119966 POINT(47.26612372210228 -121.97283979369615) bank119966 +119967 POINT(47.83092123659149 -122.7981592188805) bank119967 +119968 POINT(47.229403415947075 -122.00485361379754) bank119968 +119969 POINT(46.768730875177 -121.90239649374374) bank119969 +119970 POINT(46.908029235094524 -122.75218217153189) bank119970 +119971 POINT(47.108194900770776 -121.6582113416186) bank119971 +119972 POINT(48.43567758192621 -121.829288684191) bank119972 +119973 POINT(46.73835654976425 -122.44618520520616) bank119973 +119974 POINT(47.83377653005422 -121.40679926292968) bank119974 +119975 POINT(46.625596475295445 -122.8355298373565) bank119975 +119976 POINT(47.390169858273154 -122.06547435264042) bank119976 +119977 POINT(46.73898984285797 -121.8837140825512) bank119977 +119978 POINT(47.802648713453856 -121.53512869627615) bank119978 +119979 POINT(47.652300108902246 -121.91028307732985) bank119979 +119980 POINT(47.63204622530232 -121.95562652582962) bank119980 +119981 POINT(47.92335514387855 -122.20453743200567) bank119981 +119982 POINT(46.78787782743245 -122.3117437137078) bank119982 +119983 POINT(47.1726851322196 -121.91449683427012) bank119983 +119984 POINT(48.2015938373824 -122.23779956679024) bank119984 +119985 POINT(48.06982538959249 -121.62881810027046) bank119985 +119986 POINT(48.34007571514775 -121.92837755845346) bank119986 +119987 POINT(48.374247625655514 -122.8229736480813) bank119987 +119988 POINT(46.892106617261604 -121.40254523190752) bank119988 +119989 POINT(46.72973236877282 -123.03645001663362) bank119989 +119990 POINT(48.421753912320796 -122.6188997654884) bank119990 +119991 POINT(47.442843891165005 -121.62549168350854) bank119991 +119992 POINT(47.574549761280146 -121.49501700094415) bank119992 +119993 POINT(47.03960990076681 -122.13300754541144) bank119993 +119994 POINT(47.65122273614281 -123.02166432002977) bank119994 +119995 POINT(47.10235643060946 -122.42793490930805) bank119995 +119996 POINT(48.15992063854033 -122.72716542400092) bank119996 +119997 POINT(47.951484315794204 -122.82937939796541) bank119997 +119998 POINT(47.529469661451046 -122.73980832193939) bank119998 +119999 POINT(47.35919002819972 -121.57571497821498) bank119999 +120000 POINT(48.52509935368721 -123.04376023283737) bank120000 +120001 POINT(47.52547282257277 -122.75311636113028) bank120001 +120002 POINT(47.21488945465379 -121.84770319720599) bank120002 +120003 POINT(48.54716830484336 -122.00723641207738) bank120003 +120004 POINT(46.91587786017433 -121.787709043521) bank120004 +120005 POINT(47.9333110424066 -122.61579989888382) bank120005 +120006 POINT(47.246873997129114 -122.55493911810449) bank120006 +120007 POINT(47.666100605190316 -122.53662803162776) bank120007 +120008 POINT(48.13352788788756 -123.16457794557063) bank120008 +120009 POINT(48.10662110546302 -121.84550554227867) bank120009 +120010 POINT(48.284672776552426 -121.85425538548574) bank120010 +120011 POINT(47.547237445093 -122.55691973762842) bank120011 +120012 POINT(46.882442728717415 -122.98965128016513) bank120012 +120013 POINT(48.53246865099986 -122.4303756937822) bank120013 +120014 POINT(46.8556999298759 -122.94658570837305) bank120014 +120015 POINT(46.692105638626686 -122.23160842242046) bank120015 +120016 POINT(48.597389020323035 -122.53866854699268) bank120016 +120017 POINT(48.5916227921171 -121.62292772625521) bank120017 +120018 POINT(47.663429119449624 -122.21180974519686) bank120018 +120019 POINT(47.437061492433486 -121.94580247500389) bank120019 +120020 POINT(47.34554713154957 -121.58850114841887) bank120020 +120021 POINT(48.0449380099516 -123.10401131688546) bank120021 +120022 POINT(47.04630261469539 -121.8953271546611) bank120022 +120023 POINT(47.70293120003433 -122.26607495399259) bank120023 +120024 POINT(46.77087445460065 -122.33827764584592) bank120024 +120025 POINT(48.279434728520926 -122.33519388789875) bank120025 +120026 POINT(46.640213951973614 -122.10057958763791) bank120026 +120027 POINT(47.11382780704474 -121.35544854019439) bank120027 +120028 POINT(48.56685186245085 -121.75339939129702) bank120028 +120029 POINT(48.23859704905322 -123.09957501450317) bank120029 +120030 POINT(48.40926608688068 -121.58075131960527) bank120030 +120031 POINT(47.392644705873685 -123.04372819038788) bank120031 +120032 POINT(47.20203747215053 -123.28751286651898) bank120032 +120033 POINT(46.862880261045746 -121.42552683118359) bank120033 +120034 POINT(47.660354496144336 -121.34004150493982) bank120034 +120035 POINT(46.90544250660926 -121.44483615969148) bank120035 +120036 POINT(47.73924582236395 -121.56432700449747) bank120036 +120037 POINT(48.366959328327994 -123.2421819597037) bank120037 +120038 POINT(47.67643350177313 -122.40849756883578) bank120038 +120039 POINT(47.92386563427958 -121.49795024469766) bank120039 +120040 POINT(48.00333022563709 -121.5826753150622) bank120040 +120041 POINT(46.62523620213666 -122.99151140943296) bank120041 +120042 POINT(47.592514483541 -121.51282908738214) bank120042 +120043 POINT(47.197677613150475 -123.07153563375167) bank120043 +120044 POINT(48.24757734774553 -121.72799958289635) bank120044 +120045 POINT(48.39465867769101 -123.18743099070866) bank120045 +120046 POINT(47.86282816512107 -121.97061566861971) bank120046 +120047 POINT(46.70153100548244 -123.31708585131358) bank120047 +120048 POINT(47.284294724195284 -122.93262368080943) bank120048 +120049 POINT(46.93676903319374 -121.63873555817302) bank120049 +120050 POINT(46.785680838386114 -123.32228393591721) bank120050 +120051 POINT(47.853537532774645 -121.41265170729383) bank120051 +120052 POINT(47.46462265538324 -122.13668769739151) bank120052 +120053 POINT(47.71325785717663 -121.95339064455187) bank120053 +120054 POINT(47.57570713685607 -121.6112289902328) bank120054 +120055 POINT(47.28249425343122 -121.4161010921262) bank120055 +120056 POINT(48.083451695955574 -122.20086775388398) bank120056 +120057 POINT(48.46679588412647 -123.18347183275684) bank120057 +120058 POINT(46.65982226595738 -123.16676658067769) bank120058 +120059 POINT(46.988638958236564 -122.65797830980463) bank120059 +120060 POINT(47.187923181247825 -123.2317414730932) bank120060 +120061 POINT(47.83296822665548 -122.36063615830912) bank120061 +120062 POINT(48.465682965907675 -122.66641308616444) bank120062 +120063 POINT(46.69351088222023 -121.34780291977945) bank120063 +120064 POINT(48.15497275384087 -122.45741421269547) bank120064 +120065 POINT(47.877055474014206 -121.49074602451361) bank120065 +120066 POINT(48.10081600646061 -121.92116638307813) bank120066 +120067 POINT(47.546266788965546 -122.23490412332096) bank120067 +120068 POINT(48.19535702903841 -123.09897256162542) bank120068 +120069 POINT(48.366325008476565 -121.61860234472597) bank120069 +120070 POINT(47.976184603725905 -123.00940436220593) bank120070 +120071 POINT(47.32614779350941 -122.45967506921716) bank120071 +120072 POINT(48.51498641992616 -122.8414594820878) bank120072 +120073 POINT(48.159758771063835 -121.78812411719572) bank120073 +120074 POINT(46.689678043953656 -122.74483068405121) bank120074 +120075 POINT(47.96603040048654 -121.6926499946365) bank120075 +120076 POINT(47.71800217938156 -122.34505132310318) bank120076 +120077 POINT(48.11697985655394 -121.4029324936404) bank120077 +120078 POINT(47.9729785100718 -121.37567522758151) bank120078 +120079 POINT(47.83399244705055 -122.89845903652768) bank120079 +120080 POINT(47.26416670071379 -122.83912455077312) bank120080 +120081 POINT(48.13860770191598 -121.70021973158794) bank120081 +120082 POINT(47.91666182645226 -122.45117732094144) bank120082 +120083 POINT(47.49871565881086 -123.14019534459518) bank120083 +120084 POINT(48.22724215780346 -122.6553224511881) bank120084 +120085 POINT(47.4415020907027 -122.92519445451128) bank120085 +120086 POINT(47.080873499669245 -121.80394468333141) bank120086 +120087 POINT(46.830224324244504 -122.70428284558378) bank120087 +120088 POINT(48.20966063613914 -121.78798741963054) bank120088 +120089 POINT(47.34568933694355 -122.60500476273018) bank120089 +120090 POINT(47.19180010225895 -122.4483691905237) bank120090 +120091 POINT(47.53951868039609 -123.3166195249031) bank120091 +120092 POINT(47.82180521611277 -123.16171775596067) bank120092 +120093 POINT(47.164661916667185 -122.161941292518) bank120093 +120094 POINT(48.32785674915568 -122.84038841744021) bank120094 +120095 POINT(47.848236166773695 -122.27038488120819) bank120095 +120096 POINT(47.11541606613842 -121.61503184017958) bank120096 +120097 POINT(46.63247238528774 -121.47880771742429) bank120097 +120098 POINT(47.40744070146884 -122.92793812002047) bank120098 +120099 POINT(47.59180757428701 -122.43547474184062) bank120099 +120100 POINT(48.40672924902473 -121.84495078526494) bank120100 +120101 POINT(47.021549131593446 -122.28889435207937) bank120101 +120102 POINT(47.317069333003516 -122.80642100242474) bank120102 +120103 POINT(47.67164604320314 -122.74904416392137) bank120103 +120104 POINT(46.906492261519794 -122.38853344055752) bank120104 +120105 POINT(46.963866485009746 -122.97803739721482) bank120105 +120106 POINT(48.465822007436564 -123.28442560967045) bank120106 +120107 POINT(47.46943402466612 -122.53026129877374) bank120107 +120108 POINT(46.61708385407537 -121.53059715171932) bank120108 +120109 POINT(47.85851526485018 -123.01006619106239) bank120109 +120110 POINT(48.004850548793144 -121.83632951233143) bank120110 +120111 POINT(47.933447403491805 -123.07271329721148) bank120111 +120112 POINT(48.11124288741186 -122.59419287610669) bank120112 +120113 POINT(46.93398216860325 -121.43337453971711) bank120113 +120114 POINT(46.98088755426661 -121.40184711788858) bank120114 +120115 POINT(47.36485040843389 -122.35297618150007) bank120115 +120116 POINT(46.97335036796697 -122.55726687034819) bank120116 +120117 POINT(46.836241480773516 -121.87400235771844) bank120117 +120118 POINT(47.80275830756028 -122.00893999280213) bank120118 +120119 POINT(47.797544742846995 -122.54084276662795) bank120119 +120120 POINT(46.92995656526243 -121.56019401098244) bank120120 +120121 POINT(47.03537855773646 -121.4405983915576) bank120121 +120122 POINT(47.806496199318346 -121.56292167214214) bank120122 +120123 POINT(46.67010701491106 -122.08282032386188) bank120123 +120124 POINT(47.68061273270142 -122.9351693000583) bank120124 +120125 POINT(46.93240128967599 -121.66889655643985) bank120125 +120126 POINT(46.95045309018519 -121.68530094952929) bank120126 +120127 POINT(48.04045230995355 -122.45630588629427) bank120127 +120128 POINT(48.15626110981387 -121.37122925741477) bank120128 +120129 POINT(48.05674521917838 -122.49272192113276) bank120129 +120130 POINT(47.63204031450762 -122.54393408581332) bank120130 +120131 POINT(46.983810540083915 -121.81558219201798) bank120131 +120132 POINT(47.24831556613003 -121.67533896071245) bank120132 +120133 POINT(48.432858454645135 -122.16491626344869) bank120133 +120134 POINT(48.02584229261529 -121.40012838425031) bank120134 +120135 POINT(48.0746220670832 -121.96903821084115) bank120135 +120136 POINT(46.61254212767308 -121.34920236380377) bank120136 +120137 POINT(46.69638966212305 -122.56956347298564) bank120137 +120138 POINT(48.51869625652225 -121.55195913347453) bank120138 +120139 POINT(47.19129065676139 -122.76436741045265) bank120139 +120140 POINT(47.088284198012154 -121.94333524044134) bank120140 +120141 POINT(47.06684336324456 -122.97378165709657) bank120141 +120142 POINT(47.288016430626605 -122.40094337123062) bank120142 +120143 POINT(47.3734268633093 -123.00046067270284) bank120143 +120144 POINT(48.132404218654095 -121.86237578942905) bank120144 +120145 POINT(47.9108617058738 -122.67666574225088) bank120145 +120146 POINT(47.9386919170148 -121.67116315782772) bank120146 +120147 POINT(46.62456701315774 -121.9518775111949) bank120147 +120148 POINT(47.446090516881995 -121.92226527690828) bank120148 +120149 POINT(47.306266540967314 -122.89144091453639) bank120149 +120150 POINT(47.63784929235433 -121.48671189673396) bank120150 +120151 POINT(47.9491974689505 -122.01757621605536) bank120151 +120152 POINT(47.03393159389904 -123.0411537241701) bank120152 +120153 POINT(46.85548698531231 -122.48764394435734) bank120153 +120154 POINT(48.04415204367945 -123.20881267441331) bank120154 +120155 POINT(46.88196514303006 -123.20987021459148) bank120155 +120156 POINT(47.156749426124556 -121.33754917718797) bank120156 +120157 POINT(47.07576265888409 -122.67157467753182) bank120157 +120158 POINT(47.27811334874725 -121.89071461240279) bank120158 +120159 POINT(47.20666951548873 -122.75254399456539) bank120159 +120160 POINT(48.332569332975204 -122.00386023847574) bank120160 +120161 POINT(47.95658623009385 -121.77273783899341) bank120161 +120162 POINT(48.09286439717213 -122.59359603209793) bank120162 +120163 POINT(47.61434774456983 -122.82194345366811) bank120163 +120164 POINT(48.50615536795476 -121.40784648503089) bank120164 +120165 POINT(48.52959255847887 -121.81646914120279) bank120165 +120166 POINT(46.679162648319696 -123.07787710811787) bank120166 +120167 POINT(48.04345339353895 -122.70314896965924) bank120167 +120168 POINT(47.63540558202938 -122.9332494577916) bank120168 +120169 POINT(46.743269172164005 -122.54472232464414) bank120169 +120170 POINT(47.35501014257511 -121.4128986572948) bank120170 +120171 POINT(47.49602895248352 -121.46993613732865) bank120171 +120172 POINT(47.441898257990474 -121.4781023751344) bank120172 +120173 POINT(48.43972935364512 -121.48458113507957) bank120173 +120174 POINT(48.23465602126508 -122.19987163753171) bank120174 +120175 POINT(47.57048566808953 -123.18242232039215) bank120175 +120176 POINT(48.5769095568249 -122.64448308372306) bank120176 +120177 POINT(47.89667679041089 -122.56734926433987) bank120177 +120178 POINT(47.30341659269137 -121.77658986003979) bank120178 +120179 POINT(47.30929545464844 -122.56358474458162) bank120179 +120180 POINT(48.145642608419934 -122.4904998034301) bank120180 +120181 POINT(47.520380049333575 -121.70949372155955) bank120181 +120182 POINT(47.02871958132301 -121.67998671407078) bank120182 +120183 POINT(47.646139577471516 -123.06265192021924) bank120183 +120184 POINT(47.20546605540175 -123.10384395405895) bank120184 +120185 POINT(48.34319463599589 -122.65904179725459) bank120185 +120186 POINT(47.54604726713663 -122.81605727843424) bank120186 +120187 POINT(47.534474600437484 -121.43827383369391) bank120187 +120188 POINT(48.406253483755876 -123.05637840593981) bank120188 +120189 POINT(48.20920321664876 -123.24014245903024) bank120189 +120190 POINT(47.12091124489893 -121.90940899401268) bank120190 +120191 POINT(48.59180823810702 -122.90192778565243) bank120191 +120192 POINT(47.951981801023344 -122.43982674360946) bank120192 +120193 POINT(48.5993965113869 -121.8627573530469) bank120193 +120194 POINT(47.33191068604519 -123.20734645485445) bank120194 +120195 POINT(48.19039244469125 -123.10621360411072) bank120195 +120196 POINT(48.23012026277809 -122.19035668570501) bank120196 +120197 POINT(48.09761618044742 -122.05521965358568) bank120197 +120198 POINT(48.006146138596236 -122.32723666198335) bank120198 +120199 POINT(47.73275189244927 -122.83398345849314) bank120199 +120200 POINT(47.50029025021944 -121.38544910175402) bank120200 +120201 POINT(46.63857120712561 -121.41550229099326) bank120201 +120202 POINT(47.36061591979719 -122.2891074053501) bank120202 +120203 POINT(48.041979298523636 -121.69614605940167) bank120203 +120204 POINT(48.52767233793045 -122.71271357704201) bank120204 +120205 POINT(46.89589256004859 -122.31358003658566) bank120205 +120206 POINT(47.8726376429009 -122.0702402721718) bank120206 +120207 POINT(48.415358731714356 -121.64516259078226) bank120207 +120208 POINT(47.6395699372183 -121.82255384466299) bank120208 +120209 POINT(48.27889835880063 -122.47641591296622) bank120209 +120210 POINT(48.419671843168935 -122.05107983715202) bank120210 +120211 POINT(48.0323344936962 -122.28824676227319) bank120211 +120212 POINT(47.48730500106959 -122.8495532499388) bank120212 +120213 POINT(47.74034053186418 -121.5836748159147) bank120213 +120214 POINT(47.300750836617766 -122.88383439377185) bank120214 +120215 POINT(47.14464927096742 -121.89238813139588) bank120215 +120216 POINT(46.89684434697849 -121.62017304043361) bank120216 +120217 POINT(48.58140115498844 -122.35889763605157) bank120217 +120218 POINT(48.12196993412026 -122.64601091264429) bank120218 +120219 POINT(48.11414441662936 -122.21005912670729) bank120219 +120220 POINT(46.67444086865223 -122.15027307956024) bank120220 +120221 POINT(47.61727954376525 -121.9364455331247) bank120221 +120222 POINT(47.7026466228226 -122.15498820888752) bank120222 +120223 POINT(48.240434916013065 -122.00556932155644) bank120223 +120224 POINT(48.20902872167695 -122.32060069731638) bank120224 +120225 POINT(47.142701452306675 -123.10611784309664) bank120225 +120226 POINT(46.72106913916711 -123.28719279002853) bank120226 +120227 POINT(47.28807929159307 -122.31039963372656) bank120227 +120228 POINT(47.19179384767119 -122.93778388050599) bank120228 +120229 POINT(48.259843483462255 -122.73918038303079) bank120229 +120230 POINT(48.354205414479246 -123.19599076532832) bank120230 +120231 POINT(48.091675959581416 -122.68781188146879) bank120231 +120232 POINT(46.70712898423908 -121.93680265564278) bank120232 +120233 POINT(47.78726258186406 -121.54873454912645) bank120233 +120234 POINT(47.74263462481299 -121.8475906337833) bank120234 +120235 POINT(47.27177304730593 -121.80389145475989) bank120235 +120236 POINT(47.28214514154 -122.5338961572227) bank120236 +120237 POINT(48.295081695148994 -121.42478217449379) bank120237 +120238 POINT(48.58726025171544 -121.7779782116779) bank120238 +120239 POINT(47.96087859220889 -122.6817646760708) bank120239 +120240 POINT(46.9948248576909 -121.81106178870931) bank120240 +120241 POINT(47.882448762001594 -122.40347829127849) bank120241 +120242 POINT(48.4431980136058 -122.49676998679088) bank120242 +120243 POINT(48.241458802309516 -123.20995195830359) bank120243 +120244 POINT(46.65657364456559 -122.11748603372124) bank120244 +120245 POINT(48.348679016565264 -121.43682661960007) bank120245 +120246 POINT(47.76828353014736 -123.23934691933455) bank120246 +120247 POINT(47.06025246983902 -122.5948622180395) bank120247 +120248 POINT(48.248985776178536 -121.53251228691121) bank120248 +120249 POINT(47.49347000266006 -122.41945155921643) bank120249 +120250 POINT(48.27034287560844 -122.4851208526313) bank120250 +120251 POINT(47.56812495545301 -122.26430046125623) bank120251 +120252 POINT(46.67056914753473 -122.35096272347022) bank120252 +120253 POINT(47.39513235742375 -121.55304059703282) bank120253 +120254 POINT(47.242060403322874 -121.79538215796313) bank120254 +120255 POINT(48.06841943525865 -122.26042448354609) bank120255 +120256 POINT(46.68779061015459 -122.39768504443805) bank120256 +120257 POINT(46.71336390001922 -121.48740619774121) bank120257 +120258 POINT(48.2087973945562 -122.38795223637001) bank120258 +120259 POINT(47.59025462014332 -121.90265255206165) bank120259 +120260 POINT(47.13998901900192 -123.23720316353247) bank120260 +120261 POINT(48.45950721290681 -123.32230308524217) bank120261 +120262 POINT(46.90089412207976 -122.8985067393972) bank120262 +120263 POINT(46.71529375192847 -122.84186768109235) bank120263 +120264 POINT(46.667304171499126 -122.26614017578584) bank120264 +120265 POINT(47.50501165579816 -121.90858873474664) bank120265 +120266 POINT(47.32240003475028 -121.8055031015263) bank120266 +120267 POINT(47.47675011568438 -123.22871952542292) bank120267 +120268 POINT(47.45974091170466 -121.77409159112453) bank120268 +120269 POINT(46.8062653035552 -123.09861647085835) bank120269 +120270 POINT(48.512660405327026 -122.07938949900561) bank120270 +120271 POINT(47.386709462742644 -121.6423068333539) bank120271 +120272 POINT(46.915191893534754 -122.69063878834378) bank120272 +120273 POINT(47.20663315671321 -122.86659013761889) bank120273 +120274 POINT(47.22503850361284 -121.9707328964708) bank120274 +120275 POINT(47.30319761111145 -121.54218763476861) bank120275 +120276 POINT(47.81969732207 -122.74309658156643) bank120276 +120277 POINT(48.527220279981094 -122.98423599150902) bank120277 +120278 POINT(48.58805377170552 -122.72432830429541) bank120278 +120279 POINT(47.44946473298908 -122.49437413569116) bank120279 +120280 POINT(47.97875170191263 -122.90844822354298) bank120280 +120281 POINT(47.96853386072078 -122.39810432220413) bank120281 +120282 POINT(47.721092812889026 -121.90313447754876) bank120282 +120283 POINT(48.03043272812699 -122.93954333087558) bank120283 +120284 POINT(48.354823239116406 -122.51910891083371) bank120284 +120285 POINT(48.25448057287266 -121.77702225847985) bank120285 +120286 POINT(46.704618281765114 -121.43644326952605) bank120286 +120287 POINT(48.35445568974938 -122.45091233263945) bank120287 +120288 POINT(47.647917275842055 -121.76981305725016) bank120288 +120289 POINT(47.80188338985975 -122.99376623194297) bank120289 +120290 POINT(47.89665642933093 -122.62042045929523) bank120290 +120291 POINT(46.73483522488325 -122.1989855450605) bank120291 +120292 POINT(47.45064488097465 -122.44864912195207) bank120292 +120293 POINT(47.63629662316529 -121.95943662899835) bank120293 +120294 POINT(47.157195093690866 -122.54165057900242) bank120294 +120295 POINT(46.99969865512597 -123.09553663545859) bank120295 +120296 POINT(47.4269122195315 -121.41662043781076) bank120296 +120297 POINT(46.91448249888381 -121.83733363681885) bank120297 +120298 POINT(47.08300558606629 -121.87972861551518) bank120298 +120299 POINT(47.29207025266117 -122.21537278779671) bank120299 +120300 POINT(46.90264956181229 -123.22913850458093) bank120300 +120301 POINT(47.07894691314274 -121.74665227348113) bank120301 +120302 POINT(47.82269728415428 -122.32395350839107) bank120302 +120303 POINT(47.060418007847765 -121.44457673095937) bank120303 +120304 POINT(46.64279493887219 -122.18062554808601) bank120304 +120305 POINT(47.08643346297874 -121.86972510524014) bank120305 +120306 POINT(47.10514918937263 -122.82134566387597) bank120306 +120307 POINT(48.41279627749964 -121.53350273340646) bank120307 +120308 POINT(47.270353569181005 -122.81995718567043) bank120308 +120309 POINT(47.33889889175663 -122.7536682096628) bank120309 +120310 POINT(47.82282015551217 -123.07604383535461) bank120310 +120311 POINT(46.7450470386427 -122.23178388603624) bank120311 +120312 POINT(48.23714395124707 -121.97494086043903) bank120312 +120313 POINT(48.01304328239884 -122.90444426522768) bank120313 +120314 POINT(48.422414317783165 -123.09541215024379) bank120314 +120315 POINT(47.5871697595677 -122.1979088496137) bank120315 +120316 POINT(47.85317711335217 -122.64766612442021) bank120316 +120317 POINT(47.340473495894564 -123.22324918915862) bank120317 +120318 POINT(46.67730631108705 -121.71654624131902) bank120318 +120319 POINT(47.162273156134766 -122.02570680592785) bank120319 +120320 POINT(48.043138117887274 -122.13749061575813) bank120320 +120321 POINT(48.554744220655365 -122.9800480658301) bank120321 +120322 POINT(48.479073001003755 -122.48354660215456) bank120322 +120323 POINT(47.97526907342798 -121.66836911074212) bank120323 +120324 POINT(48.065135621156266 -122.58491987370411) bank120324 +120325 POINT(47.977143959485474 -121.89927647954464) bank120325 +120326 POINT(48.52528374165862 -121.53291478697346) bank120326 +120327 POINT(46.82996762385396 -121.364519830537) bank120327 +120328 POINT(47.948626573976995 -122.62466055770903) bank120328 +120329 POINT(46.89091561828425 -121.83859515653485) bank120329 +120330 POINT(47.86098564176302 -122.62175218714346) bank120330 +120331 POINT(48.01026163638514 -121.55847871772505) bank120331 +120332 POINT(47.033692447229086 -121.94183500111593) bank120332 +120333 POINT(47.206357380722075 -123.25877878203536) bank120333 +120334 POINT(48.393142936782276 -121.69790904890029) bank120334 +120335 POINT(46.698608456908374 -122.20833224185918) bank120335 +120336 POINT(47.045939011442606 -122.6785243587749) bank120336 +120337 POINT(46.64666076384441 -121.87887225099004) bank120337 +120338 POINT(46.87047898806173 -122.25403485484712) bank120338 +120339 POINT(47.84054105111867 -122.6672147046993) bank120339 +120340 POINT(47.34100680194468 -122.94403477248893) bank120340 +120341 POINT(48.35926911692694 -122.45167966044534) bank120341 +120342 POINT(47.210271695069736 -123.24863717574972) bank120342 +120343 POINT(48.445972845999464 -121.56252651899797) bank120343 +120344 POINT(47.27043322195775 -122.34973127327133) bank120344 +120345 POINT(48.13186609338581 -123.20364164503843) bank120345 +120346 POINT(46.90934671779343 -122.31753901527537) bank120346 +120347 POINT(48.035480018569636 -121.44849964534939) bank120347 +120348 POINT(46.940934509556655 -121.68928094380266) bank120348 +120349 POINT(47.67074824951843 -121.81814889819483) bank120349 +120350 POINT(46.80365494493575 -123.02446372900745) bank120350 +120351 POINT(48.26428646412343 -122.84238211596097) bank120351 +120352 POINT(46.76095596332216 -121.6509360235776) bank120352 +120353 POINT(48.088320355557265 -121.85101504779088) bank120353 +120354 POINT(48.56316536775748 -121.38351500129286) bank120354 +120355 POINT(46.71632473445933 -122.69268684042548) bank120355 +120356 POINT(48.106768798936585 -123.14527640237843) bank120356 +120357 POINT(48.399729009774134 -122.08608882737722) bank120357 +120358 POINT(48.56953736580213 -122.87372982204089) bank120358 +120359 POINT(48.58602729111767 -121.60540737376542) bank120359 +120360 POINT(46.77596543859045 -122.65960020024299) bank120360 +120361 POINT(47.62694952947414 -122.71686713192335) bank120361 +120362 POINT(48.192588022008465 -121.75865732581532) bank120362 +120363 POINT(48.31118123169453 -123.03054983047487) bank120363 +120364 POINT(46.619696980691806 -123.09336592988248) bank120364 +120365 POINT(47.12023569568609 -122.84368038119588) bank120365 +120366 POINT(46.836533719470445 -122.07655469511967) bank120366 +120367 POINT(48.47716905237172 -122.72732695103383) bank120367 +120368 POINT(48.07435192503508 -122.32872771550404) bank120368 +120369 POINT(48.43536085907739 -122.18816748790877) bank120369 +120370 POINT(46.90377838931759 -122.1397158184848) bank120370 +120371 POINT(47.94872290350039 -122.4116855778612) bank120371 +120372 POINT(47.11638450834565 -121.85794043119458) bank120372 +120373 POINT(47.42531933333242 -121.44203486919193) bank120373 +120374 POINT(48.30866882601454 -123.19630909557483) bank120374 +120375 POINT(48.447408549026974 -122.73606580123042) bank120375 +120376 POINT(48.600118536921535 -122.89062364146615) bank120376 +120377 POINT(47.179662118215205 -121.74165157150172) bank120377 +120378 POINT(48.42949632045393 -122.57452204772096) bank120378 +120379 POINT(47.81625991857484 -122.84838940496853) bank120379 +120380 POINT(48.30787747438045 -122.11244273836898) bank120380 +120381 POINT(47.757698544323674 -123.10580697792128) bank120381 +120382 POINT(47.96871108330718 -122.81051735722293) bank120382 +120383 POINT(46.962254766713585 -122.64256942347447) bank120383 +120384 POINT(46.62588983819675 -123.18832412388716) bank120384 +120385 POINT(47.454902198210064 -121.50124507070703) bank120385 +120386 POINT(46.628342711388036 -122.44850771554951) bank120386 +120387 POINT(47.81392251530351 -123.04663280718286) bank120387 +120388 POINT(48.17826574221374 -123.10592957311837) bank120388 +120389 POINT(48.27363137141278 -122.73826742206083) bank120389 +120390 POINT(47.19691313831561 -122.3201306484183) bank120390 +120391 POINT(48.40908917229991 -121.4813539664139) bank120391 +120392 POINT(46.69858030893918 -122.64209859783153) bank120392 +120393 POINT(46.86801982927001 -121.33512856010839) bank120393 +120394 POINT(48.162078302261754 -121.72114546289338) bank120394 +120395 POINT(46.74109272126931 -122.12220604841399) bank120395 +120396 POINT(48.30907856974461 -123.11482496331287) bank120396 +120397 POINT(47.337196545296734 -121.53832420203405) bank120397 +120398 POINT(47.4813579527149 -123.07829854791683) bank120398 +120399 POINT(47.662146973061304 -123.08310118872328) bank120399 +120400 POINT(46.74813829120141 -122.44932216534797) bank120400 +120401 POINT(47.65446093301017 -122.15929151371195) bank120401 +120402 POINT(47.76065030952021 -123.19693909642953) bank120402 +120403 POINT(46.956154440798336 -121.44412081674795) bank120403 +120404 POINT(48.47611828744331 -123.13443008594649) bank120404 +120405 POINT(48.13942811299965 -121.60174815628024) bank120405 +120406 POINT(47.34432669617626 -121.4578277056203) bank120406 +120407 POINT(46.68309017588545 -121.84115553384814) bank120407 +120408 POINT(47.48820618028884 -121.65383947109197) bank120408 +120409 POINT(47.30344530682967 -122.3899611555351) bank120409 +120410 POINT(46.743747210569154 -122.80630433553875) bank120410 +120411 POINT(47.61398592096644 -123.1290111835396) bank120411 +120412 POINT(46.75967312873212 -122.29017048565906) bank120412 +120413 POINT(47.270328325360865 -122.70145736475138) bank120413 +120414 POINT(47.24751428360426 -121.89486100577858) bank120414 +120415 POINT(48.04871429234178 -122.76529959596847) bank120415 +120416 POINT(47.992782114383886 -122.54069467584235) bank120416 +120417 POINT(46.687241161316045 -121.3503229736908) bank120417 +120418 POINT(47.99023371565434 -122.39974074856441) bank120418 +120419 POINT(48.41923954697798 -121.77595159720981) bank120419 +120420 POINT(47.25723490587922 -122.95112452426368) bank120420 +120421 POINT(46.85517115704975 -121.95791958623845) bank120421 +120422 POINT(47.54505458976859 -121.96631914711526) bank120422 +120423 POINT(48.09313129030467 -122.72994061465366) bank120423 +120424 POINT(48.25577249017318 -122.69949770501364) bank120424 +120425 POINT(47.12050654757364 -122.27945188438551) bank120425 +120426 POINT(47.780646584467114 -123.059415978536) bank120426 +120427 POINT(46.89827140818869 -121.80200938462303) bank120427 +120428 POINT(47.43686526799107 -122.47281431366969) bank120428 +120429 POINT(47.69825680847614 -123.09402070578281) bank120429 +120430 POINT(48.320860223475606 -122.61825392416588) bank120430 +120431 POINT(48.19854049859976 -121.44380766738244) bank120431 +120432 POINT(46.86506455705 -123.10060398153172) bank120432 +120433 POINT(47.584644513347385 -121.33717138713394) bank120433 +120434 POINT(47.4195402260478 -122.40460172681652) bank120434 +120435 POINT(48.60184917296914 -122.26476092719666) bank120435 +120436 POINT(47.12154349034696 -123.07041527069916) bank120436 +120437 POINT(48.356918927234176 -123.2725881579786) bank120437 +120438 POINT(47.37451284458396 -121.63659204813379) bank120438 +120439 POINT(46.99655008292724 -121.5387800168193) bank120439 +120440 POINT(48.41349932892354 -122.26708076156494) bank120440 +120441 POINT(48.026872921858406 -123.32287195867443) bank120441 +120442 POINT(47.28034879128309 -122.09601300440715) bank120442 +120443 POINT(47.02372880115507 -121.76941025181935) bank120443 +120444 POINT(48.10767865643792 -122.39272123044263) bank120444 +120445 POINT(48.42918473763897 -121.36467528636511) bank120445 +120446 POINT(48.10198872653878 -121.54613169034282) bank120446 +120447 POINT(48.01470356888293 -122.21564769701702) bank120447 +120448 POINT(46.85148232390591 -121.97388671471634) bank120448 +120449 POINT(48.52271681789451 -123.26030092585935) bank120449 +120450 POINT(47.437429680075695 -122.60557228787488) bank120450 +120451 POINT(47.41337157610348 -122.01395441873814) bank120451 +120452 POINT(47.922859242094866 -123.23807320075544) bank120452 +120453 POINT(47.47432374778192 -122.02767307539852) bank120453 +120454 POINT(48.27276185655414 -122.7823372423869) bank120454 +120455 POINT(46.88102018915009 -121.62924320889516) bank120455 +120456 POINT(48.383726347890445 -121.3511600435048) bank120456 +120457 POINT(46.93019971537383 -122.52863004497225) bank120457 +120458 POINT(47.59251123070835 -122.26783086979474) bank120458 +120459 POINT(48.08655063789255 -121.4666671130122) bank120459 +120460 POINT(47.81123742953112 -122.82432165076025) bank120460 +120461 POINT(48.38685373932263 -122.03196036924027) bank120461 +120462 POINT(46.82236219345804 -121.88403323126822) bank120462 +120463 POINT(47.2319643688011 -122.53217089248383) bank120463 +120464 POINT(48.54065473348721 -122.99670494493483) bank120464 +120465 POINT(47.86015836586644 -121.50642929565691) bank120465 +120466 POINT(48.465708022695715 -122.68022214412855) bank120466 +120467 POINT(48.27261090589437 -122.9058926629397) bank120467 +120468 POINT(47.14632490885497 -121.7245370836442) bank120468 +120469 POINT(48.35149696235348 -122.21254247834675) bank120469 +120470 POINT(48.340711742638874 -122.47731146932932) bank120470 +120471 POINT(47.6987596268696 -122.949700096669) bank120471 +120472 POINT(46.87095447848766 -122.20006003935127) bank120472 +120473 POINT(47.8666004044094 -122.27208111608482) bank120473 +120474 POINT(47.43139915267318 -123.22502418606477) bank120474 +120475 POINT(46.749839977834206 -121.61577004181136) bank120475 +120476 POINT(47.72921318810662 -122.43105083801612) bank120476 +120477 POINT(47.76409157196859 -121.7691497486985) bank120477 +120478 POINT(46.87996855747168 -121.92308203362136) bank120478 +120479 POINT(48.336111774992126 -121.53377088141048) bank120479 +120480 POINT(47.082386234829485 -122.85203381149175) bank120480 +120481 POINT(47.61606197394081 -122.14227313502492) bank120481 +120482 POINT(48.11982988385361 -122.87713717263834) bank120482 +120483 POINT(48.54233333557471 -123.09415941615325) bank120483 +120484 POINT(47.34889958030562 -122.04434827323867) bank120484 +120485 POINT(48.288933187388515 -123.20016846249948) bank120485 +120486 POINT(48.419201739487534 -122.55402895664423) bank120486 +120487 POINT(48.41392478428955 -123.05554962448612) bank120487 +120488 POINT(48.10475834903489 -122.45316941177995) bank120488 +120489 POINT(47.451623741077405 -122.31422039670744) bank120489 +120490 POINT(48.446091591056316 -122.86589123782667) bank120490 +120491 POINT(47.19511478233782 -122.83139844383723) bank120491 +120492 POINT(46.68817730878623 -122.77930801073097) bank120492 +120493 POINT(48.2273540183654 -123.11496748518228) bank120493 +120494 POINT(47.83055459400174 -121.86640384969202) bank120494 +120495 POINT(47.17877030532907 -122.17331054200984) bank120495 +120496 POINT(47.46537286668539 -122.65015100641614) bank120496 +120497 POINT(46.996573005350946 -122.03526345706943) bank120497 +120498 POINT(47.10492947380068 -122.86645891785827) bank120498 +120499 POINT(48.033585947089236 -122.29960697800509) bank120499 +120500 POINT(47.510593859678636 -122.54476138183956) bank120500 +120501 POINT(47.77626717003182 -121.97686918054741) bank120501 +120502 POINT(47.99581468527173 -123.31581544492663) bank120502 +120503 POINT(47.683256663763 -121.95266968274979) bank120503 +120504 POINT(47.88839397472486 -123.29322880501198) bank120504 +120505 POINT(46.889773007497475 -122.48936278820186) bank120505 +120506 POINT(47.367011306348765 -122.38123549208541) bank120506 +120507 POINT(47.60588756593209 -122.72093052381659) bank120507 +120508 POINT(47.306092269548195 -122.178110542999) bank120508 +120509 POINT(47.18174984999865 -122.19445396773808) bank120509 +120510 POINT(47.61863051828138 -121.72176903465501) bank120510 +120511 POINT(47.014662983796725 -121.40149098736336) bank120511 +120512 POINT(48.067773210626505 -122.68056149957187) bank120512 +120513 POINT(48.27785611918857 -122.55771818252275) bank120513 +120514 POINT(47.11265262052278 -121.52656484088875) bank120514 +120515 POINT(48.393925256305074 -122.75103755282888) bank120515 +120516 POINT(46.697933117525956 -122.92928674544656) bank120516 +120517 POINT(46.731041311652476 -122.42208421081976) bank120517 +120518 POINT(47.126152126984714 -122.0056332291346) bank120518 +120519 POINT(46.76842194315481 -121.73476975878332) bank120519 +120520 POINT(47.889289852330194 -121.81839678176446) bank120520 +120521 POINT(47.314228638199054 -121.6393396181352) bank120521 +120522 POINT(46.69793056368328 -122.89118543841617) bank120522 +120523 POINT(48.205757473868694 -123.21821866821115) bank120523 +120524 POINT(47.04471821217264 -122.74666722309692) bank120524 +120525 POINT(47.3100445700355 -122.70315394480728) bank120525 +120526 POINT(47.79221291031726 -122.58709709227378) bank120526 +120527 POINT(47.004811405669635 -122.26083977132274) bank120527 +120528 POINT(48.50709327920773 -122.93945538246106) bank120528 +120529 POINT(48.468349501057986 -122.56943305099715) bank120529 +120530 POINT(47.548223697940095 -122.67873072437291) bank120530 +120531 POINT(46.64500828378374 -121.60763227262518) bank120531 +120532 POINT(47.68853323449818 -122.56111017425496) bank120532 +120533 POINT(46.63447730927959 -123.1415322847914) bank120533 +120534 POINT(46.830365328525964 -123.28867520966241) bank120534 +120535 POINT(47.19505030497163 -122.06194253919512) bank120535 +120536 POINT(46.818645469411074 -122.91103602759033) bank120536 +120537 POINT(47.72235278381251 -122.07602278471272) bank120537 +120538 POINT(46.93730732547597 -122.24987486292716) bank120538 +120539 POINT(48.44146269410979 -122.15071241343797) bank120539 +120540 POINT(47.76207071985105 -123.00025217423966) bank120540 +120541 POINT(47.99245822148162 -121.64169674356566) bank120541 +120542 POINT(47.527428156309476 -121.8203399117491) bank120542 +120543 POINT(47.287961982429486 -122.40784209872527) bank120543 +120544 POINT(48.28760183809927 -123.06320569599399) bank120544 +120545 POINT(48.50770500221673 -121.75499223021365) bank120545 +120546 POINT(47.39148329040478 -121.45839432088974) bank120546 +120547 POINT(46.98343190868031 -122.892935167523) bank120547 +120548 POINT(46.96143945547595 -121.40031002479161) bank120548 +120549 POINT(47.787761763493435 -122.54689014678682) bank120549 +120550 POINT(47.96959627077099 -122.22186723742567) bank120550 +120551 POINT(46.62344462718581 -122.37810149479661) bank120551 +120552 POINT(48.418424909879754 -121.4002428846037) bank120552 +120553 POINT(46.742490011463055 -122.29673944208547) bank120553 +120554 POINT(47.38528952892535 -121.82957364977) bank120554 +120555 POINT(47.841337775469796 -122.13123002964423) bank120555 +120556 POINT(48.31862101269187 -122.29995702808236) bank120556 +120557 POINT(46.84524721797646 -122.51730082550385) bank120557 +120558 POINT(48.15840503514901 -121.50963458885614) bank120558 +120559 POINT(46.83854795773704 -122.84880694419921) bank120559 +120560 POINT(48.403560442168214 -123.0839564035739) bank120560 +120561 POINT(47.00194119459081 -121.696436815123) bank120561 +120562 POINT(46.809788604687654 -121.48458345631512) bank120562 +120563 POINT(48.42919748140213 -122.93400156588787) bank120563 +120564 POINT(48.4743718751231 -122.28988011001272) bank120564 +120565 POINT(48.1728782742713 -121.80516287979867) bank120565 +120566 POINT(48.4057204716837 -122.9980173005936) bank120566 +120567 POINT(47.92473763417442 -122.0335208977429) bank120567 +120568 POINT(48.27811104322244 -122.92476693087981) bank120568 +120569 POINT(47.47052576743465 -122.8210556515701) bank120569 +120570 POINT(47.05135052556883 -122.5153308489114) bank120570 +120571 POINT(46.95770359672541 -122.87913017937493) bank120571 +120572 POINT(46.97626917656887 -122.90443293957141) bank120572 +120573 POINT(47.906655526813886 -121.4143514870847) bank120573 +120574 POINT(46.95544431998093 -121.7308317800838) bank120574 +120575 POINT(48.37039119488608 -122.4921816790804) bank120575 +120576 POINT(47.570788419597484 -122.93520164449436) bank120576 +120577 POINT(47.589033059808614 -121.73132384015773) bank120577 +120578 POINT(46.80066039229163 -122.94000164248209) bank120578 +120579 POINT(48.03839549315681 -122.52510748223418) bank120579 +120580 POINT(48.40073946081117 -122.8413495317097) bank120580 +120581 POINT(47.983206373749184 -122.01752425959663) bank120581 +120582 POINT(47.938937371533186 -121.84700501448955) bank120582 +120583 POINT(47.76782591496203 -123.06323652866747) bank120583 +120584 POINT(46.89674927265947 -122.3549672210708) bank120584 +120585 POINT(47.04027762780833 -122.4256297584395) bank120585 +120586 POINT(47.29956803543691 -121.43794641158112) bank120586 +120587 POINT(46.78285803313214 -122.90063268390118) bank120587 +120588 POINT(47.92117464153369 -121.68598738317164) bank120588 +120589 POINT(46.71294928776858 -122.46189070448257) bank120589 +120590 POINT(48.10990426360124 -122.67647778298017) bank120590 +120591 POINT(46.765862977961966 -122.1605056479527) bank120591 +120592 POINT(46.725997730916816 -123.3183011186927) bank120592 +120593 POINT(47.48683340285807 -121.44141624202236) bank120593 +120594 POINT(47.2913125528766 -122.30594974809745) bank120594 +120595 POINT(47.31220216324947 -121.69053994172143) bank120595 +120596 POINT(47.59848393383168 -122.34822972123901) bank120596 +120597 POINT(47.14233764495666 -122.17574703287845) bank120597 +120598 POINT(47.59815874118327 -121.43448816171754) bank120598 +120599 POINT(48.296118077182896 -122.59684280075275) bank120599 +120600 POINT(47.55652835393247 -122.24628991198722) bank120600 +120601 POINT(46.785410844518275 -123.05505287846862) bank120601 +120602 POINT(48.39348409589953 -122.63907257655225) bank120602 +120603 POINT(47.819794126681515 -122.8879616612517) bank120603 +120604 POINT(46.81687053601271 -122.1917376113937) bank120604 +120605 POINT(48.253847697112136 -122.17444436694691) bank120605 +120606 POINT(47.18556481654197 -122.56340633811645) bank120606 +120607 POINT(47.50239782346504 -122.65871940492833) bank120607 +120608 POINT(47.62369948190785 -122.23776358526979) bank120608 +120609 POINT(47.67534045551097 -121.93310043301766) bank120609 +120610 POINT(48.38809613690338 -121.80325438603315) bank120610 +120611 POINT(47.55770409938499 -122.17589923955309) bank120611 +120612 POINT(47.15139866366837 -121.78994127219764) bank120612 +120613 POINT(48.3849410521562 -123.27515666510604) bank120613 +120614 POINT(47.96530118865247 -121.44377225277648) bank120614 +120615 POINT(47.276832301735425 -122.27059377583757) bank120615 +120616 POINT(47.22258638193464 -121.39036821087618) bank120616 +120617 POINT(48.013798172102206 -122.48476013656294) bank120617 +120618 POINT(47.81848208349966 -122.47136767610287) bank120618 +120619 POINT(47.8400832401598 -121.50634472280971) bank120619 +120620 POINT(46.90828689165679 -122.89203519899104) bank120620 +120621 POINT(48.250140129050806 -122.11155713941399) bank120621 +120622 POINT(47.24399625182503 -121.44548726275563) bank120622 +120623 POINT(47.01881491827126 -121.89879808473495) bank120623 +120624 POINT(48.38441582452281 -122.56558197344232) bank120624 +120625 POINT(46.740364929318915 -122.02395141130265) bank120625 +120626 POINT(47.50185031604998 -123.04364178173806) bank120626 +120627 POINT(46.76830092027112 -121.38057209246212) bank120627 +120628 POINT(47.87344243115614 -123.12375832557377) bank120628 +120629 POINT(47.54883396668273 -122.52052480938785) bank120629 +120630 POINT(48.22029003359274 -123.04393501787456) bank120630 +120631 POINT(47.95670004178644 -123.2652445264635) bank120631 +120632 POINT(48.06055369557063 -123.16419891392452) bank120632 +120633 POINT(46.75130066707321 -121.58100523737933) bank120633 +120634 POINT(46.90068647514196 -121.55540543715655) bank120634 +120635 POINT(47.416355469207744 -122.49198909672977) bank120635 +120636 POINT(47.829166003294404 -122.81780611325725) bank120636 +120637 POINT(47.7436808925151 -121.83085382064772) bank120637 +120638 POINT(47.663893258616795 -122.23880874776279) bank120638 +120639 POINT(47.84970985223384 -122.07937951164794) bank120639 +120640 POINT(47.012893111862795 -121.51232482317829) bank120640 +120641 POINT(47.5325897699337 -122.71492029422998) bank120641 +120642 POINT(47.494224349894296 -121.4714004781939) bank120642 +120643 POINT(46.71261081993702 -121.72562826676432) bank120643 +120644 POINT(47.90006534309063 -122.85799812727888) bank120644 +120645 POINT(48.51958737495044 -121.87912693254842) bank120645 +120646 POINT(46.65314526725928 -123.22924018649735) bank120646 +120647 POINT(47.294001367691585 -122.7923386940777) bank120647 +120648 POINT(48.27657474889865 -121.62834987102681) bank120648 +120649 POINT(48.05221025703427 -121.7508411287323) bank120649 +120650 POINT(47.019336017790515 -122.3588989530983) bank120650 +120651 POINT(48.21325050547016 -123.0420848950633) bank120651 +120652 POINT(46.80237236696423 -122.15911696399837) bank120652 +120653 POINT(47.154461500656765 -122.73591691446765) bank120653 +120654 POINT(46.751876172430904 -121.64713286566408) bank120654 +120655 POINT(47.830996991026446 -121.83173818253557) bank120655 +120656 POINT(48.08702932444711 -121.67271430336365) bank120656 +120657 POINT(48.02253362100567 -121.86075579936468) bank120657 +120658 POINT(47.40450611302266 -122.49570958544233) bank120658 +120659 POINT(47.70819235018734 -121.34730096766674) bank120659 +120660 POINT(48.311787075675255 -121.63002780196724) bank120660 +120661 POINT(46.69888312181162 -121.64271993046448) bank120661 +120662 POINT(48.07049808386216 -123.0401732572352) bank120662 +120663 POINT(47.00440216574233 -121.6685166179897) bank120663 +120664 POINT(47.08470732558757 -121.60960396827046) bank120664 +120665 POINT(47.58587688443262 -123.32583800743048) bank120665 +120666 POINT(46.799830399119 -122.50772651328914) bank120666 +120667 POINT(47.47233525093738 -122.45871657188833) bank120667 +120668 POINT(46.78462474817012 -122.77756419679146) bank120668 +120669 POINT(48.37583185884475 -122.01631198947824) bank120669 +120670 POINT(47.102251222502375 -122.5607614343301) bank120670 +120671 POINT(47.18151156995673 -123.22383461406605) bank120671 +120672 POINT(47.453937068801736 -123.25506921356039) bank120672 +120673 POINT(48.39467854015683 -123.30466820245685) bank120673 +120674 POINT(48.54066972004608 -121.99430460293733) bank120674 +120675 POINT(47.457353910222594 -122.5917745148901) bank120675 +120676 POINT(48.59892206780282 -121.76580630242808) bank120676 +120677 POINT(47.75173036718635 -122.75883461044504) bank120677 +120678 POINT(48.591904945062204 -122.43429138708952) bank120678 +120679 POINT(47.46215183493677 -122.46883274713262) bank120679 +120680 POINT(48.32016522154966 -122.8760833275115) bank120680 +120681 POINT(48.287398237723664 -123.29338840712941) bank120681 +120682 POINT(47.03613700704772 -123.06544299951322) bank120682 +120683 POINT(47.493573251575434 -121.43337550904269) bank120683 +120684 POINT(48.25608004157019 -121.50286638189198) bank120684 +120685 POINT(48.384037546263244 -123.06501923485469) bank120685 +120686 POINT(48.40691267120693 -122.56223802428416) bank120686 +120687 POINT(48.56324283239889 -122.98908561750252) bank120687 +120688 POINT(47.43263758368884 -122.00761202250472) bank120688 +120689 POINT(47.50557772528262 -121.8610861037267) bank120689 +120690 POINT(47.94043343101259 -122.24885277949417) bank120690 +120691 POINT(46.803453093600865 -123.25270326506129) bank120691 +120692 POINT(48.35787117677191 -121.45145575025329) bank120692 +120693 POINT(46.88884829022772 -121.58094519006596) bank120693 +120694 POINT(47.60512998085172 -123.04240929224775) bank120694 +120695 POINT(47.56556003059232 -122.72779088602445) bank120695 +120696 POINT(47.72426751829329 -122.45180325198697) bank120696 +120697 POINT(47.900920137090225 -122.50364380492016) bank120697 +120698 POINT(47.83102736607009 -122.7366272292659) bank120698 +120699 POINT(47.556613019740006 -122.56496117649094) bank120699 +120700 POINT(48.43663790559726 -122.82341107507064) bank120700 +120701 POINT(48.212740625496345 -122.64916547916086) bank120701 +120702 POINT(47.16163357307495 -122.37920472652964) bank120702 +120703 POINT(46.741942368407706 -121.70516552423601) bank120703 +120704 POINT(48.015836857062 -121.69474925117075) bank120704 +120705 POINT(47.41322228202979 -121.51691630670604) bank120705 +120706 POINT(47.91783622659104 -123.27075183852492) bank120706 +120707 POINT(46.78338387420361 -122.60411392818712) bank120707 +120708 POINT(48.23398490412213 -122.8208891896195) bank120708 +120709 POINT(47.75856521799964 -122.43033627703436) bank120709 +120710 POINT(48.07679526714885 -122.68556564161582) bank120710 +120711 POINT(46.606503804328824 -122.61750604991384) bank120711 +120712 POINT(48.52799899435517 -123.32758785871742) bank120712 +120713 POINT(47.38197297798112 -122.06163922404942) bank120713 +120714 POINT(48.02269513151107 -121.55162054997659) bank120714 +120715 POINT(48.34211607389619 -123.20768674619812) bank120715 +120716 POINT(47.58737351203417 -123.29060317765146) bank120716 +120717 POINT(46.61337410326947 -122.16020897026131) bank120717 +120718 POINT(48.17218590371445 -121.38889375568289) bank120718 +120719 POINT(46.73381587766653 -122.62609495682864) bank120719 +120720 POINT(47.7044494020876 -123.0924426021841) bank120720 +120721 POINT(47.442695243982904 -122.68198024556092) bank120721 +120722 POINT(47.65119461121723 -122.1117773675876) bank120722 +120723 POINT(47.34184602112267 -122.8405667729883) bank120723 +120724 POINT(47.785605616234655 -123.31388100739458) bank120724 +120725 POINT(47.338872329297196 -123.25309371860598) bank120725 +120726 POINT(48.376621400886634 -121.66433676101475) bank120726 +120727 POINT(48.23263846202484 -122.30506429458148) bank120727 +120728 POINT(47.5206046026407 -121.89672744575836) bank120728 +120729 POINT(46.900309892341134 -121.77323778849079) bank120729 +120730 POINT(48.41036345412505 -122.30196926133246) bank120730 +120731 POINT(46.91610942832425 -122.79502482557886) bank120731 +120732 POINT(47.92612235876036 -123.01033700289912) bank120732 +120733 POINT(46.98188577118137 -122.44083131912095) bank120733 +120734 POINT(47.39036325906354 -122.27882996410486) bank120734 +120735 POINT(48.124185514084864 -122.82317142342697) bank120735 +120736 POINT(46.778728108761925 -122.66529268011263) bank120736 +120737 POINT(48.54581314668471 -122.7023088479016) bank120737 +120738 POINT(48.4816189973005 -121.85133028995058) bank120738 +120739 POINT(48.15091259533011 -121.33491490790821) bank120739 +120740 POINT(47.66185035833844 -122.73120004299817) bank120740 +120741 POINT(48.024035480888976 -123.10259492121881) bank120741 +120742 POINT(48.33769418353187 -122.17591295417411) bank120742 +120743 POINT(48.33403549630197 -122.51245494412265) bank120743 +120744 POINT(47.55636937536441 -123.31663793085345) bank120744 +120745 POINT(47.74526530456666 -123.18618527965752) bank120745 +120746 POINT(48.5087388247785 -121.73281034762543) bank120746 +120747 POINT(48.21638243641785 -121.73331508686094) bank120747 +120748 POINT(47.58769562343385 -122.87237555754812) bank120748 +120749 POINT(48.0770223577193 -121.42969071925934) bank120749 +120750 POINT(47.49649476749435 -123.20913758588338) bank120750 +120751 POINT(47.95120832710176 -123.11982774954193) bank120751 +120752 POINT(48.14663136407226 -121.42637777782788) bank120752 +120753 POINT(47.1159158067976 -122.80271424130476) bank120753 +120754 POINT(47.34176718647981 -122.32007003660397) bank120754 +120755 POINT(47.82083825540344 -123.0381546375494) bank120755 +120756 POINT(48.41331425261716 -122.64969095077174) bank120756 +120757 POINT(48.332385824573336 -121.76578051987683) bank120757 +120758 POINT(47.93193778086963 -123.20979420625034) bank120758 +120759 POINT(46.63659598073237 -122.36442526256064) bank120759 +120760 POINT(47.27444804910279 -123.32870632360066) bank120760 +120761 POINT(48.022855430603556 -123.02756978300192) bank120761 +120762 POINT(47.95639729431113 -122.20658824865035) bank120762 +120763 POINT(46.69294521672192 -122.96840741257849) bank120763 +120764 POINT(46.75102483418651 -122.04795911428867) bank120764 +120765 POINT(47.698967801963036 -122.31148062601147) bank120765 +120766 POINT(48.10495601538073 -123.19285628330536) bank120766 +120767 POINT(48.346542615534354 -122.09734833046582) bank120767 +120768 POINT(48.60354767849657 -122.58728173104541) bank120768 +120769 POINT(47.317526131271805 -121.3490037782361) bank120769 +120770 POINT(48.1703908951008 -122.74673143560963) bank120770 +120771 POINT(47.65395183913317 -122.18050483757226) bank120771 +120772 POINT(48.03480544598858 -122.29382258749443) bank120772 +120773 POINT(47.72208467234064 -122.52516160110758) bank120773 +120774 POINT(47.91656495345421 -122.59603622830726) bank120774 +120775 POINT(46.75451511227989 -122.40430046912168) bank120775 +120776 POINT(48.54945817816728 -122.2172441748439) bank120776 +120777 POINT(46.70072889127132 -121.8494094490224) bank120777 +120778 POINT(48.41581621688326 -122.0625878900087) bank120778 +120779 POINT(47.49372878816978 -121.68079579471784) bank120779 +120780 POINT(47.180586426250514 -121.8831828695935) bank120780 +120781 POINT(48.580260116023865 -122.43955683079112) bank120781 +120782 POINT(46.6722902897658 -122.80277089198684) bank120782 +120783 POINT(46.93032907294946 -122.14474617250146) bank120783 +120784 POINT(47.652080834359055 -123.22569538458141) bank120784 +120785 POINT(46.90429574560276 -122.05344083257414) bank120785 +120786 POINT(47.704534208254245 -121.33530443994918) bank120786 +120787 POINT(46.969254688782044 -121.79327064607905) bank120787 +120788 POINT(48.071305452921436 -121.3936295368331) bank120788 +120789 POINT(48.47450546777627 -123.29899301074123) bank120789 +120790 POINT(48.09184813214458 -122.66283531022607) bank120790 +120791 POINT(46.84016902390808 -122.41004897784764) bank120791 +120792 POINT(47.33400729297234 -121.59185166161139) bank120792 +120793 POINT(46.73148458414292 -122.45350018842129) bank120793 +120794 POINT(48.5101656147659 -122.46431324662854) bank120794 +120795 POINT(46.85255786028598 -122.91004663107539) bank120795 +120796 POINT(48.36205770833485 -123.21166169646443) bank120796 +120797 POINT(46.761613532120265 -122.0519006298947) bank120797 +120798 POINT(47.97685588214142 -123.00412788514731) bank120798 +120799 POINT(46.808750106441856 -122.94142681091711) bank120799 +120800 POINT(47.8618770574754 -121.35672388509903) bank120800 +120801 POINT(46.75718895313134 -123.19501353968158) bank120801 +120802 POINT(46.82942424371517 -121.59266521370185) bank120802 +120803 POINT(46.771061722269124 -122.25784678218959) bank120803 +120804 POINT(47.14811906595278 -122.196317632619) bank120804 +120805 POINT(46.98510217423269 -122.6307433526345) bank120805 +120806 POINT(47.4812756420165 -121.43329558076547) bank120806 +120807 POINT(47.74724707317202 -121.96852844167853) bank120807 +120808 POINT(48.538517266486245 -122.55742136643778) bank120808 +120809 POINT(47.01627798441165 -121.77772411793542) bank120809 +120810 POINT(46.91992310213008 -122.35952881501221) bank120810 +120811 POINT(48.571062058107046 -122.90605759031354) bank120811 +120812 POINT(46.914491869440624 -122.25682389495775) bank120812 +120813 POINT(47.71149844070504 -122.29369712398938) bank120813 +120814 POINT(47.50541623695111 -122.29137018619292) bank120814 +120815 POINT(46.75610623964113 -122.07806757799395) bank120815 +120816 POINT(47.45272323702314 -123.00961211560845) bank120816 +120817 POINT(47.70355474703584 -122.53230138455103) bank120817 +120818 POINT(48.23360562879185 -121.58028852356729) bank120818 +120819 POINT(46.607646605606256 -121.79201539916234) bank120819 +120820 POINT(47.29973471035493 -121.59589557724347) bank120820 +120821 POINT(47.587055136101746 -121.69549352236604) bank120821 +120822 POINT(48.05412152193216 -121.92105754011065) bank120822 +120823 POINT(47.69253670448584 -122.56689498342817) bank120823 +120824 POINT(46.63472053892387 -122.25519151562331) bank120824 +120825 POINT(47.89868209817103 -122.43470388219313) bank120825 +120826 POINT(46.85084828195113 -121.66751333432293) bank120826 +120827 POINT(46.84356893022344 -122.43137540075935) bank120827 +120828 POINT(47.70794723872362 -122.2849579156028) bank120828 +120829 POINT(46.69677036138849 -121.90412090394035) bank120829 +120830 POINT(47.560483913674354 -123.05571019333148) bank120830 +120831 POINT(48.517769736552 -122.21738594257235) bank120831 +120832 POINT(47.63187752626886 -122.78266666014927) bank120832 +120833 POINT(46.93205952858908 -122.80638423161314) bank120833 +120834 POINT(48.13359196341541 -121.75908960232996) bank120834 +120835 POINT(48.450311220632656 -122.58054347897507) bank120835 +120836 POINT(47.86410606409596 -122.25959357158072) bank120836 +120837 POINT(48.54772786979617 -121.3483710202974) bank120837 +120838 POINT(47.285617215587 -123.07312730403225) bank120838 +120839 POINT(48.48746679314218 -122.16755232033168) bank120839 +120840 POINT(47.815971371280504 -121.50662985580908) bank120840 +120841 POINT(46.67717983767606 -122.87070528806933) bank120841 +120842 POINT(47.90495713389932 -122.95308612739835) bank120842 +120843 POINT(48.44488779013445 -123.14201010843352) bank120843 +120844 POINT(47.083258894225104 -123.27427239823055) bank120844 +120845 POINT(48.08677293215508 -123.3017783073921) bank120845 +120846 POINT(48.562689920408516 -121.7884667609653) bank120846 +120847 POINT(47.843262770240855 -122.49713477588871) bank120847 +120848 POINT(48.273370134066745 -121.88795869075827) bank120848 +120849 POINT(48.140324831522385 -122.45470212582251) bank120849 +120850 POINT(48.19454269960802 -122.67657085893806) bank120850 +120851 POINT(46.76827472964901 -123.1826109064648) bank120851 +120852 POINT(46.7220708110536 -121.5670920232221) bank120852 +120853 POINT(46.63605298573703 -122.12242438629963) bank120853 +120854 POINT(46.646796238641784 -123.29514444425978) bank120854 +120855 POINT(46.61304086101178 -122.29718735722291) bank120855 +120856 POINT(47.03514249149058 -122.17178912672625) bank120856 +120857 POINT(47.66167091202765 -122.36306153180836) bank120857 +120858 POINT(47.46155461912942 -122.80323620867642) bank120858 +120859 POINT(46.95424325794103 -122.49125327818733) bank120859 +120860 POINT(46.8027063831751 -121.49717795324602) bank120860 +120861 POINT(47.62594699172945 -123.02714435821555) bank120861 +120862 POINT(47.95345620483593 -123.18591996130814) bank120862 +120863 POINT(47.634467044594814 -121.78591463139072) bank120863 +120864 POINT(46.88020623444616 -122.27377916244662) bank120864 +120865 POINT(48.00255206714232 -122.96644019344699) bank120865 +120866 POINT(47.7160817893372 -121.37186308911862) bank120866 +120867 POINT(48.155249915900704 -121.78938922650684) bank120867 +120868 POINT(47.85944804771152 -121.55562998385776) bank120868 +120869 POINT(48.32236998233614 -122.42944745118572) bank120869 +120870 POINT(47.67256711405349 -121.41198373684009) bank120870 +120871 POINT(47.34853922355067 -122.14058237838599) bank120871 +120872 POINT(47.54592688318985 -123.29906759487636) bank120872 +120873 POINT(47.23434371548409 -121.75504300008899) bank120873 +120874 POINT(48.343611508173225 -121.81914498353044) bank120874 +120875 POINT(47.98249019355588 -123.15278906680686) bank120875 +120876 POINT(47.248531841982285 -122.84494474938992) bank120876 +120877 POINT(48.363813045177565 -122.25281308334307) bank120877 +120878 POINT(48.259899881607 -121.56359931334427) bank120878 +120879 POINT(48.52796406442679 -121.76867828235959) bank120879 +120880 POINT(47.87768457621914 -123.23082194484815) bank120880 +120881 POINT(47.86394203889355 -121.92257736787369) bank120881 +120882 POINT(48.56551124670461 -123.02611078868355) bank120882 +120883 POINT(46.62279591188975 -122.7943181650142) bank120883 +120884 POINT(48.46512512693252 -122.34720364936554) bank120884 +120885 POINT(46.75536077989734 -122.36161258442236) bank120885 +120886 POINT(47.79016146637093 -121.42596243321958) bank120886 +120887 POINT(46.88213604690835 -122.67554336221221) bank120887 +120888 POINT(47.09961323000694 -122.46801140646969) bank120888 +120889 POINT(47.02450176327605 -121.77623011429154) bank120889 +120890 POINT(48.337752671388195 -121.56868251131846) bank120890 +120891 POINT(48.07592512732605 -122.10909856692376) bank120891 +120892 POINT(48.229895090994 -122.47580900417469) bank120892 +120893 POINT(46.80510777937755 -121.40024208044579) bank120893 +120894 POINT(47.56058572385788 -123.12207891491506) bank120894 +120895 POINT(47.361328810419074 -122.56858011165951) bank120895 +120896 POINT(47.63209622615341 -123.2596540190994) bank120896 +120897 POINT(47.807225418692724 -121.36011669014435) bank120897 +120898 POINT(47.3686221048207 -122.20490738790805) bank120898 +120899 POINT(47.02670695402961 -122.84164900450475) bank120899 +120900 POINT(48.22485592615658 -122.3423103273171) bank120900 +120901 POINT(48.57960720698659 -121.52196920257556) bank120901 +120902 POINT(48.10918661431242 -123.28364040581124) bank120902 +120903 POINT(47.158603685697614 -122.07340649891131) bank120903 +120904 POINT(46.663142613401796 -122.05204818095812) bank120904 +120905 POINT(48.07025802548065 -122.62303173397524) bank120905 +120906 POINT(47.424976191569165 -122.92066033966331) bank120906 +120907 POINT(47.8370651896198 -122.35456658655872) bank120907 +120908 POINT(48.555419684258894 -122.86020888798335) bank120908 +120909 POINT(46.86599026571304 -121.89268211606866) bank120909 +120910 POINT(48.21837838115323 -122.01883034184758) bank120910 +120911 POINT(48.346637842220794 -122.86722501031196) bank120911 +120912 POINT(48.08135579624114 -122.86220693306323) bank120912 +120913 POINT(47.32225060073321 -123.31541912095854) bank120913 +120914 POINT(47.25305005490114 -122.30814502210306) bank120914 +120915 POINT(47.46772341761395 -122.11663879776141) bank120915 +120916 POINT(46.73202949457451 -121.51984197703496) bank120916 +120917 POINT(47.74843121264025 -123.16060841769765) bank120917 +120918 POINT(48.34774377850107 -122.85262571432875) bank120918 +120919 POINT(48.27376159482863 -123.0388657297799) bank120919 +120920 POINT(47.93398942907099 -122.32206250553622) bank120920 +120921 POINT(46.63755590745758 -122.85112474200699) bank120921 +120922 POINT(47.59215367490755 -121.4651034725621) bank120922 +120923 POINT(47.05863802801768 -122.68692739508445) bank120923 +120924 POINT(47.591450662758085 -121.62353732611871) bank120924 +120925 POINT(48.051092956959884 -121.93225205478822) bank120925 +120926 POINT(47.12786532960993 -122.86056166615809) bank120926 +120927 POINT(47.10409796904979 -122.14970385548146) bank120927 +120928 POINT(48.54118266179522 -123.11368932034704) bank120928 +120929 POINT(47.28952341744904 -121.90428998609687) bank120929 +120930 POINT(47.722292692403094 -123.29357881669212) bank120930 +120931 POINT(47.36053363380151 -123.1527141489581) bank120931 +120932 POINT(47.217069961178666 -123.21308727100028) bank120932 +120933 POINT(47.41406596568925 -122.9628407803109) bank120933 +120934 POINT(47.49371768061572 -122.35173033854457) bank120934 +120935 POINT(47.702638569672835 -123.32184041506405) bank120935 +120936 POINT(47.42445726878441 -123.13949072756742) bank120936 +120937 POINT(47.657143863918094 -123.31562886277388) bank120937 +120938 POINT(48.566252101779035 -122.02852703235139) bank120938 +120939 POINT(47.79186638221414 -122.16902378693965) bank120939 +120940 POINT(47.27888115328733 -123.11060110122611) bank120940 +120941 POINT(47.89409176161865 -122.97810247283773) bank120941 +120942 POINT(46.90477802078975 -123.0874929975869) bank120942 +120943 POINT(48.230062706318 -123.01669422907922) bank120943 +120944 POINT(47.177078944002496 -121.48374454719882) bank120944 +120945 POINT(48.34529846805371 -123.29244507289461) bank120945 +120946 POINT(48.43793460130175 -123.22750766049197) bank120946 +120947 POINT(47.64293496936197 -122.79300810622671) bank120947 +120948 POINT(47.367403342757825 -121.59267562598585) bank120948 +120949 POINT(48.089012433829055 -123.16300493988973) bank120949 +120950 POINT(48.43739287462574 -121.75068011028603) bank120950 +120951 POINT(48.46834563122268 -121.99933987440653) bank120951 +120952 POINT(46.68573679152577 -122.57963457837604) bank120952 +120953 POINT(46.656157171767376 -122.80784419194241) bank120953 +120954 POINT(47.133586458187224 -122.99234226588254) bank120954 +120955 POINT(47.35274665797878 -122.27916810599314) bank120955 +120956 POINT(48.010990349149466 -121.64068895308608) bank120956 +120957 POINT(47.40783627742251 -122.69498284919068) bank120957 +120958 POINT(46.70469627754286 -122.06680220492603) bank120958 +120959 POINT(47.22717033763576 -121.34068399728203) bank120959 +120960 POINT(47.11061536357544 -123.1594518475627) bank120960 +120961 POINT(47.80955232579192 -123.12270837190665) bank120961 +120962 POINT(47.44531319154865 -122.51947091894074) bank120962 +120963 POINT(47.34969483020932 -121.8678581756479) bank120963 +120964 POINT(47.91308538719822 -122.45355691939957) bank120964 +120965 POINT(46.70936021930475 -122.52430025373009) bank120965 +120966 POINT(47.94201103320603 -122.16934160589982) bank120966 +120967 POINT(47.195942990557164 -121.98548896849935) bank120967 +120968 POINT(48.334791885322154 -122.48148712821612) bank120968 +120969 POINT(48.03514146534668 -121.89685733335051) bank120969 +120970 POINT(48.40368238456197 -121.51037910770785) bank120970 +120971 POINT(46.81700383866754 -122.13590722214572) bank120971 +120972 POINT(48.11377077261923 -122.56366219666803) bank120972 +120973 POINT(47.06162052772998 -122.0931806120753) bank120973 +120974 POINT(46.82497790262048 -121.54708097690984) bank120974 +120975 POINT(47.47225264799581 -122.06780790876019) bank120975 +120976 POINT(47.281881022624255 -121.7513349118913) bank120976 +120977 POINT(46.95734420471653 -122.74284984281152) bank120977 +120978 POINT(47.20747990744204 -121.71715344771171) bank120978 +120979 POINT(48.09118603219705 -122.73737825304605) bank120979 +120980 POINT(46.82835140513015 -123.29439107198647) bank120980 +120981 POINT(46.97022299336809 -121.8109888249376) bank120981 +120982 POINT(48.21506730489332 -121.64157585080966) bank120982 +120983 POINT(46.933133683699175 -121.34738078058477) bank120983 +120984 POINT(47.590565057771144 -122.9973140440904) bank120984 +120985 POINT(47.83967813712109 -122.36752302804345) bank120985 +120986 POINT(47.06882602247822 -122.52548285294269) bank120986 +120987 POINT(48.26368415195414 -122.92250793872924) bank120987 +120988 POINT(47.354192491128245 -123.15498772654271) bank120988 +120989 POINT(47.72345496733112 -122.59383201749698) bank120989 +120990 POINT(48.14990652084062 -121.71916728036356) bank120990 +120991 POINT(48.06400579268174 -123.29683611493103) bank120991 +120992 POINT(46.74224394021148 -122.36204464340655) bank120992 +120993 POINT(48.49698129660688 -123.06105962857515) bank120993 +120994 POINT(46.95705518421685 -122.4540317652937) bank120994 +120995 POINT(46.73046979743794 -122.70141346315428) bank120995 +120996 POINT(46.998631999709445 -122.54390573180108) bank120996 +120997 POINT(48.20162417629901 -121.56687456804335) bank120997 +120998 POINT(47.51815489567049 -121.58226473916629) bank120998 +120999 POINT(46.83475695138913 -121.65732657025418) bank120999 +121000 POINT(46.82881999178834 -121.4611953317855) bank121000 +121001 POINT(48.02199057982544 -121.69433394573885) bank121001 +121002 POINT(48.10737405407541 -122.16008243473965) bank121002 +121003 POINT(46.908358691475854 -122.78978626226228) bank121003 +121004 POINT(47.22029496637398 -122.01343233057207) bank121004 +121005 POINT(47.313171320105624 -122.80186774976485) bank121005 +121006 POINT(48.10260284180787 -121.52978822557712) bank121006 +121007 POINT(48.36303720312431 -121.92712404920687) bank121007 +121008 POINT(47.55518463238811 -121.93296999929386) bank121008 +121009 POINT(48.30308599887344 -122.83270536346812) bank121009 +121010 POINT(47.93577717466947 -122.01621548152217) bank121010 +121011 POINT(46.95186246492985 -122.47219377610986) bank121011 +121012 POINT(47.94101303585316 -121.82181430924483) bank121012 +121013 POINT(47.41472635725307 -121.42747890806231) bank121013 +121014 POINT(47.2029011506481 -121.92839445552477) bank121014 +121015 POINT(48.45204239997356 -122.49866997330165) bank121015 +121016 POINT(47.962141408079944 -122.85458041520471) bank121016 +121017 POINT(47.12119437941153 -123.04621049840291) bank121017 +121018 POINT(47.65578059450859 -122.10528157316114) bank121018 +121019 POINT(46.99570466388411 -122.05398506305444) bank121019 +121020 POINT(47.87967466053301 -122.94153751546055) bank121020 +121021 POINT(46.937352368346396 -122.41375920348754) bank121021 +121022 POINT(47.79404050948514 -122.24552679567692) bank121022 +121023 POINT(48.24233621143592 -123.17828930976228) bank121023 +121024 POINT(48.375144458999706 -122.27429666933905) bank121024 +121025 POINT(48.044415684047046 -122.63857344390819) bank121025 +121026 POINT(46.89934599931067 -122.02858761674878) bank121026 +121027 POINT(47.2309996190675 -121.93211838037348) bank121027 +121028 POINT(47.12241125072278 -122.03854299054032) bank121028 +121029 POINT(47.95251576071993 -121.67510943438293) bank121029 +121030 POINT(47.77227514659736 -121.4928812642035) bank121030 +121031 POINT(46.624569475482645 -122.80307439242993) bank121031 +121032 POINT(46.9962949274283 -122.4788646662372) bank121032 +121033 POINT(48.07558343046657 -123.28670114195491) bank121033 +121034 POINT(48.069925534949576 -123.09588531720313) bank121034 +121035 POINT(47.8185665204022 -122.37939897286014) bank121035 +121036 POINT(48.063925704641804 -122.7185915780802) bank121036 +121037 POINT(46.63535025824551 -121.67701518747552) bank121037 +121038 POINT(47.229010837964985 -122.54161358037874) bank121038 +121039 POINT(47.33546008802025 -122.8975712504634) bank121039 +121040 POINT(47.1672809345797 -121.33358110756889) bank121040 +121041 POINT(47.31257002057339 -121.85155540003389) bank121041 +121042 POINT(47.09930988859033 -122.8362311030173) bank121042 +121043 POINT(47.36046206584445 -122.92210055107209) bank121043 +121044 POINT(47.08324703550218 -122.77396985148242) bank121044 +121045 POINT(47.133887625582 -121.75782422971291) bank121045 +121046 POINT(46.710363935701146 -123.03305179134706) bank121046 +121047 POINT(46.96989337965746 -121.53260610638576) bank121047 +121048 POINT(47.51072495465476 -121.78630008527225) bank121048 +121049 POINT(48.130267618160865 -121.40088094731337) bank121049 +121050 POINT(46.992333378429706 -122.9529602126677) bank121050 +121051 POINT(48.50203360288441 -121.5665577811682) bank121051 +121052 POINT(48.32500928882719 -121.71850454563364) bank121052 +121053 POINT(48.22150994075478 -123.26239638659423) bank121053 +121054 POINT(48.38268514097452 -121.79248648266783) bank121054 +121055 POINT(46.64153476966119 -122.71826166693788) bank121055 +121056 POINT(47.06353597416786 -122.59876610844687) bank121056 +121057 POINT(47.56194892620494 -122.90458495032652) bank121057 +121058 POINT(48.47712693394924 -123.0290247106681) bank121058 +121059 POINT(46.895868569483454 -121.82634820399102) bank121059 +121060 POINT(47.74598259249353 -122.31135458286855) bank121060 +121061 POINT(47.634446532779364 -121.82353454246203) bank121061 +121062 POINT(47.05561119839854 -123.32798264126174) bank121062 +121063 POINT(47.38557627512406 -121.50465572789197) bank121063 +121064 POINT(46.61414792484821 -122.28294138882003) bank121064 +121065 POINT(47.58074513580073 -122.76444043655434) bank121065 +121066 POINT(47.5517128221845 -122.38103590119165) bank121066 +121067 POINT(48.24142730152901 -121.410325666657) bank121067 +121068 POINT(47.25675658899075 -121.43199634351372) bank121068 +121069 POINT(46.61145804951443 -121.93938218246896) bank121069 +121070 POINT(47.53584034006304 -122.9609273560653) bank121070 +121071 POINT(47.76125272990797 -121.4680866357484) bank121071 +121072 POINT(47.28654016687207 -121.85541858924492) bank121072 +121073 POINT(48.46144852730282 -122.80263143213148) bank121073 +121074 POINT(47.285405065817656 -122.70451916332856) bank121074 +121075 POINT(46.946814892713824 -122.75297146505137) bank121075 +121076 POINT(48.41799770831043 -121.76352695119418) bank121076 +121077 POINT(47.244759380660625 -121.6050852416468) bank121077 +121078 POINT(48.24838221380162 -123.00522013221338) bank121078 +121079 POINT(48.14351040203805 -122.45052669359939) bank121079 +121080 POINT(48.26885183348082 -122.57694818625748) bank121080 +121081 POINT(48.26907479642178 -121.68176291856808) bank121081 +121082 POINT(47.862832564902384 -123.26864262473407) bank121082 +121083 POINT(47.95530097910689 -122.48683375623844) bank121083 +121084 POINT(47.06324746046443 -122.24562935099874) bank121084 +121085 POINT(48.38415404005044 -122.84132766759923) bank121085 +121086 POINT(48.197494346982225 -122.14788409692896) bank121086 +121087 POINT(46.98478616884263 -121.67173043805035) bank121087 +121088 POINT(47.325408139237524 -122.97495407553822) bank121088 +121089 POINT(46.68969617075317 -122.13684889990128) bank121089 +121090 POINT(47.44350239114871 -122.05322094494612) bank121090 +121091 POINT(48.23379473932646 -122.56144176449757) bank121091 +121092 POINT(47.46082130408607 -122.3668921388072) bank121092 +121093 POINT(48.585738531506074 -122.05491963388867) bank121093 +121094 POINT(46.64300350837749 -121.97425160959797) bank121094 +121095 POINT(47.418280076437455 -122.09957748342453) bank121095 +121096 POINT(46.61096442450177 -121.35716716546047) bank121096 +121097 POINT(47.88081320813977 -122.47676196797653) bank121097 +121098 POINT(47.90065680935816 -121.37508506634373) bank121098 +121099 POINT(46.887719802306236 -121.97624127189371) bank121099 +121100 POINT(47.87983474498348 -122.35322844833124) bank121100 +121101 POINT(48.26674578483924 -121.53919555826687) bank121101 +121102 POINT(47.755015812375895 -122.14868872177856) bank121102 +121103 POINT(48.30759439989681 -123.25747222343476) bank121103 +121104 POINT(47.00996926533294 -122.46116323077649) bank121104 +121105 POINT(47.542574947523754 -122.35614953704392) bank121105 +121106 POINT(47.0677694465273 -122.00322928542211) bank121106 +121107 POINT(46.878959096102285 -122.68025254698915) bank121107 +121108 POINT(48.269669906647444 -121.61412790394893) bank121108 +121109 POINT(47.95942017127283 -121.72688302958352) bank121109 +121110 POINT(46.833820630110665 -122.77893976416364) bank121110 +121111 POINT(47.223209556542685 -123.12910668714976) bank121111 +121112 POINT(46.678874879516556 -122.12112842033324) bank121112 +121113 POINT(48.472508793251954 -122.28899874590002) bank121113 +121114 POINT(47.5391943324765 -121.35932426625334) bank121114 +121115 POINT(48.21216500683028 -122.94138378496905) bank121115 +121116 POINT(48.19854272500293 -122.17801893448672) bank121116 +121117 POINT(46.9508076302788 -121.87874464963164) bank121117 +121118 POINT(47.31687017863822 -122.99540980247987) bank121118 +121119 POINT(47.11792062931614 -122.24741689379132) bank121119 +121120 POINT(47.706649313242515 -121.77974703612847) bank121120 +121121 POINT(48.00254541162986 -123.19075560280044) bank121121 +121122 POINT(46.758965324559334 -121.4388287064623) bank121122 +121123 POINT(47.166962144911224 -121.51409876474295) bank121123 +121124 POINT(47.204022560484056 -122.37636921346416) bank121124 +121125 POINT(48.22131352480787 -123.03738010730719) bank121125 +121126 POINT(48.58206635885948 -122.62595809376852) bank121126 +121127 POINT(48.473609106252795 -122.3705362389027) bank121127 +121128 POINT(47.73893956818297 -122.88634648218851) bank121128 +121129 POINT(47.86251921807083 -121.52481527049684) bank121129 +121130 POINT(46.9017960271934 -122.74957267408777) bank121130 +121131 POINT(47.068186349692944 -123.19387097540462) bank121131 +121132 POINT(48.324402319649266 -122.19807119925997) bank121132 +121133 POINT(48.551689310578915 -122.78077703912795) bank121133 +121134 POINT(47.86611930488053 -122.40639820672386) bank121134 +121135 POINT(46.81332206434432 -121.42899245365822) bank121135 +121136 POINT(47.04657417862304 -122.29814243127892) bank121136 +121137 POINT(47.751211937370435 -122.22102589941754) bank121137 +121138 POINT(47.726838233671174 -122.63889766276853) bank121138 +121139 POINT(47.958629342576906 -121.44501811303591) bank121139 +121140 POINT(48.24347085924701 -121.76196449851292) bank121140 +121141 POINT(48.528528335341065 -123.06043246738098) bank121141 +121142 POINT(47.606094147615636 -122.09065780107575) bank121142 +121143 POINT(46.619561150807975 -121.60580613541997) bank121143 +121144 POINT(47.73051276924507 -122.01397286864804) bank121144 +121145 POINT(47.568591673672174 -122.00533414054084) bank121145 +121146 POINT(47.53808750888361 -123.06478047165508) bank121146 +121147 POINT(46.93467400914576 -123.07842584216068) bank121147 +121148 POINT(47.06415126905953 -122.59113748582638) bank121148 +121149 POINT(47.53072174510714 -122.59682528480924) bank121149 +121150 POINT(47.63165200600937 -122.57610640240506) bank121150 +121151 POINT(48.228602756054556 -121.6768027229344) bank121151 +121152 POINT(46.84392852039112 -122.73760831870135) bank121152 +121153 POINT(46.97974523683304 -121.36550139789618) bank121153 +121154 POINT(47.56708977686652 -122.14512774455122) bank121154 +121155 POINT(47.63435348494296 -122.02674603486051) bank121155 +121156 POINT(47.72614722218333 -122.25466753775899) bank121156 +121157 POINT(48.40678578274085 -121.47009187229916) bank121157 +121158 POINT(48.39908500726632 -121.78692935912134) bank121158 +121159 POINT(47.81389594465135 -121.85479266549042) bank121159 +121160 POINT(47.211914589342506 -122.38667813348334) bank121160 +121161 POINT(47.01831160307594 -122.45371698260575) bank121161 +121162 POINT(48.50715812270956 -121.58590392746062) bank121162 +121163 POINT(47.23990693531277 -121.93832951133678) bank121163 +121164 POINT(48.10526056809239 -122.47676491216102) bank121164 +121165 POINT(47.284539386388495 -122.67822722087396) bank121165 +121166 POINT(47.889195116574165 -123.02871597130574) bank121166 +121167 POINT(47.759425408616 -121.33652809313294) bank121167 +121168 POINT(46.98688091829702 -121.7591546611006) bank121168 +121169 POINT(47.80189291819698 -122.50703592112855) bank121169 +121170 POINT(47.20165985220541 -123.04642412118123) bank121170 +121171 POINT(46.91708772334612 -122.104658741451) bank121171 +121172 POINT(46.79898813936218 -122.32756397995347) bank121172 +121173 POINT(47.719314886409094 -122.92527792645245) bank121173 +121174 POINT(48.210516932611924 -122.1489861983161) bank121174 +121175 POINT(46.975464328849405 -122.88922388039045) bank121175 +121176 POINT(47.639535479337994 -123.07963076911668) bank121176 +121177 POINT(48.563789469704815 -121.63604229379251) bank121177 +121178 POINT(47.65695984451851 -121.46897771027413) bank121178 +121179 POINT(47.493632704215834 -121.96301351650634) bank121179 +121180 POINT(47.16903189521551 -121.96135164853061) bank121180 +121181 POINT(48.4476678675415 -121.50440430266417) bank121181 +121182 POINT(46.730978329406234 -123.22586373895274) bank121182 +121183 POINT(47.49122514683906 -121.74532562211928) bank121183 +121184 POINT(48.56520503091926 -121.57068435668134) bank121184 +121185 POINT(47.198511212915605 -122.08690800002421) bank121185 +121186 POINT(47.56822614066322 -123.17614839492843) bank121186 +121187 POINT(46.65123176369852 -122.89257581717067) bank121187 +121188 POINT(47.25179505937573 -122.79614092673508) bank121188 +121189 POINT(46.70725895636597 -121.669961266052) bank121189 +121190 POINT(47.66412444718424 -121.76185191399136) bank121190 +121191 POINT(46.69253917575092 -121.42741793030876) bank121191 +121192 POINT(46.752104789942585 -122.70769397763735) bank121192 +121193 POINT(46.95869983078845 -122.68071391583041) bank121193 +121194 POINT(47.58593588942152 -122.56584692875309) bank121194 +121195 POINT(46.63705356394364 -121.4121009794964) bank121195 +121196 POINT(47.297791531443785 -123.13162995527091) bank121196 +121197 POINT(47.971639007228426 -122.78665085595269) bank121197 +121198 POINT(46.682771708478406 -122.51703909925776) bank121198 +121199 POINT(46.820614444494076 -122.31132135851908) bank121199 +121200 POINT(47.90550751596485 -122.70018860240897) bank121200 +121201 POINT(47.6058682416719 -123.00480948027689) bank121201 +121202 POINT(46.70307216451067 -121.78644379114179) bank121202 +121203 POINT(46.63686254092409 -122.6385504312923) bank121203 +121204 POINT(47.10089965960221 -122.5306915972191) bank121204 +121205 POINT(47.13213858067484 -122.36252154129154) bank121205 +121206 POINT(47.719702024374826 -121.90513563801164) bank121206 +121207 POINT(48.081615097629296 -122.30603922111388) bank121207 +121208 POINT(48.57431185669136 -122.56084800406312) bank121208 +121209 POINT(46.66481996735159 -123.20021609041476) bank121209 +121210 POINT(47.173774917494086 -121.3817489291278) bank121210 +121211 POINT(48.46303497126352 -122.18395902703944) bank121211 +121212 POINT(46.76023960298789 -123.30497322652435) bank121212 +121213 POINT(47.3275801895979 -122.15896289232826) bank121213 +121214 POINT(47.86098196020046 -121.68641398884654) bank121214 +121215 POINT(48.151483995363144 -121.55957792602298) bank121215 +121216 POINT(46.695961247261955 -122.62710842078313) bank121216 +121217 POINT(47.12472006297532 -123.14694128510857) bank121217 +121218 POINT(47.1666575693954 -122.52460124751646) bank121218 +121219 POINT(48.25962684126358 -121.51453152688737) bank121219 +121220 POINT(48.444436231619285 -122.33988279675594) bank121220 +121221 POINT(48.51799655908528 -123.12843866757387) bank121221 +121222 POINT(47.95636791551751 -122.80653871500249) bank121222 +121223 POINT(48.60062787373478 -122.41276474783861) bank121223 +121224 POINT(47.48692166250402 -122.23644583172934) bank121224 +121225 POINT(47.49295144114512 -122.36706504697094) bank121225 +121226 POINT(47.711971690746054 -123.28589525412923) bank121226 +121227 POINT(47.13984212230528 -122.69420627057471) bank121227 +121228 POINT(47.38578753959544 -121.69910713443413) bank121228 +121229 POINT(47.40218742855171 -121.48557710147276) bank121229 +121230 POINT(48.36421847890475 -121.59014869543412) bank121230 +121231 POINT(47.42605485627842 -121.80027711734277) bank121231 +121232 POINT(48.36504712545751 -122.92290265876868) bank121232 +121233 POINT(47.65106261349202 -121.46344839126698) bank121233 +121234 POINT(46.96927536784386 -122.00010259545957) bank121234 +121235 POINT(47.57699613038697 -123.29411571820052) bank121235 +121236 POINT(47.68271924561253 -122.54823882444987) bank121236 +121237 POINT(47.77199461097522 -121.6169311544142) bank121237 +121238 POINT(47.23966576844585 -122.61334278494857) bank121238 +121239 POINT(48.400709868409905 -121.73285482096442) bank121239 +121240 POINT(46.65849823342003 -122.76567379639388) bank121240 +121241 POINT(48.39216153086785 -122.4998955580171) bank121241 +121242 POINT(48.34450679975501 -121.7814176653006) bank121242 +121243 POINT(47.88504988330318 -122.1994018765105) bank121243 +121244 POINT(48.256866944578704 -121.37542051912452) bank121244 +121245 POINT(47.20480569485872 -122.68348138947684) bank121245 +121246 POINT(47.918803308275486 -122.66636028524522) bank121246 +121247 POINT(47.47362580222655 -122.40072279306234) bank121247 +121248 POINT(47.33869720370737 -122.26256291180592) bank121248 +121249 POINT(47.97251590537228 -122.46763484778499) bank121249 +121250 POINT(48.15451693560336 -121.52993239090517) bank121250 +121251 POINT(47.271532897721244 -121.4789640062374) bank121251 +121252 POINT(46.97194715545223 -123.1635594730822) bank121252 +121253 POINT(47.529766750656115 -121.99736407173243) bank121253 +121254 POINT(47.067748551325785 -123.16698210666638) bank121254 +121255 POINT(47.209207593786395 -121.77714543487184) bank121255 +121256 POINT(47.16207884819191 -122.14061624563567) bank121256 +121257 POINT(47.42057815155527 -123.1931584034519) bank121257 +121258 POINT(46.7082609042115 -122.52772305198907) bank121258 +121259 POINT(46.73852053906636 -122.80106814641438) bank121259 +121260 POINT(47.534508072847245 -122.34804991105337) bank121260 +121261 POINT(47.46559818301683 -121.58129443076874) bank121261 +121262 POINT(47.321928221913325 -123.18737454445184) bank121262 +121263 POINT(47.371673489051645 -122.32105136964174) bank121263 +121264 POINT(46.79352622432898 -122.6249496997718) bank121264 +121265 POINT(47.13486443987352 -121.6887793193213) bank121265 +121266 POINT(46.60670536937718 -122.24827291317503) bank121266 +121267 POINT(46.78704522995775 -122.3885663403745) bank121267 +121268 POINT(48.26546282760677 -122.9985684807744) bank121268 +121269 POINT(46.82894514271912 -123.26486571234285) bank121269 +121270 POINT(47.716134767162 -122.54334753224215) bank121270 +121271 POINT(48.315297249241056 -123.07767399198198) bank121271 +121272 POINT(47.66382868481385 -121.77802850581267) bank121272 +121273 POINT(47.51292619502247 -121.41471284872829) bank121273 +121274 POINT(48.19586543052179 -122.3575621853515) bank121274 +121275 POINT(46.75808921086905 -122.5843617080777) bank121275 +121276 POINT(48.44268234932995 -122.93954215194678) bank121276 +121277 POINT(47.572028091687585 -122.89094868282572) bank121277 +121278 POINT(46.80825841881895 -122.57379532016179) bank121278 +121279 POINT(47.03106486016102 -121.40679421599877) bank121279 +121280 POINT(47.80441930969821 -121.51298528030125) bank121280 +121281 POINT(47.41519126292254 -122.41000340182171) bank121281 +121282 POINT(48.53060214447827 -122.41932969155953) bank121282 +121283 POINT(47.30440423869243 -122.59897272130956) bank121283 +121284 POINT(47.090128085713424 -122.90766372893245) bank121284 +121285 POINT(48.22421554421653 -122.87075113960391) bank121285 +121286 POINT(47.59727184483267 -121.35041936796217) bank121286 +121287 POINT(46.738317760986256 -121.70840702850411) bank121287 +121288 POINT(47.616900318913785 -123.29874872653001) bank121288 +121289 POINT(46.68291539914641 -122.35678571284816) bank121289 +121290 POINT(48.02709048413151 -122.47753432875743) bank121290 +121291 POINT(48.526268247636914 -121.98639396981288) bank121291 +121292 POINT(47.56195823842762 -122.92459152206378) bank121292 +121293 POINT(48.595054251081066 -121.8550927611733) bank121293 +121294 POINT(47.27158266758151 -121.77389285764436) bank121294 +121295 POINT(48.5043226127744 -122.86310476553867) bank121295 +121296 POINT(48.23266492718258 -122.1418074320941) bank121296 +121297 POINT(46.92855135752382 -122.86561489538967) bank121297 +121298 POINT(48.50679260708417 -122.27700406765923) bank121298 +121299 POINT(47.19793583544765 -123.32670705005938) bank121299 +121300 POINT(48.067310972815946 -122.84973572997839) bank121300 +121301 POINT(48.34063927679391 -123.22291439008531) bank121301 +121302 POINT(48.47247515383042 -121.43740771355354) bank121302 +121303 POINT(46.95539494900429 -122.19332492471861) bank121303 +121304 POINT(47.534627736088396 -123.2520376386786) bank121304 +121305 POINT(47.76783194387974 -122.92789750445266) bank121305 +121306 POINT(47.97239752036869 -122.55136940374831) bank121306 +121307 POINT(46.691987651824505 -121.33219781044367) bank121307 +121308 POINT(48.229949063254246 -123.1859927947876) bank121308 +121309 POINT(48.30394493053897 -122.76515260135004) bank121309 +121310 POINT(46.628535237420586 -123.10622933083961) bank121310 +121311 POINT(47.52570875584937 -122.05416630873738) bank121311 +121312 POINT(47.28747599869474 -121.71293093731249) bank121312 +121313 POINT(47.25571918114322 -122.58533174458908) bank121313 +121314 POINT(47.36058075623619 -122.76752643196264) bank121314 +121315 POINT(47.39693547169589 -123.27690995668058) bank121315 +121316 POINT(47.52819445370637 -122.63022573205875) bank121316 +121317 POINT(46.72261720168903 -123.2526069952381) bank121317 +121318 POINT(47.333724513741785 -122.21014939539249) bank121318 +121319 POINT(47.68640350798751 -123.31730598166592) bank121319 +121320 POINT(47.4945982737628 -122.26277866317176) bank121320 +121321 POINT(46.93503009529311 -122.22183477368044) bank121321 +121322 POINT(47.35184553466913 -123.27096847435291) bank121322 +121323 POINT(47.98301251775094 -122.59512029414842) bank121323 +121324 POINT(48.493468347493135 -122.31726129530749) bank121324 +121325 POINT(46.67739095667703 -122.22536424589217) bank121325 +121326 POINT(47.60749611277453 -122.4714876092304) bank121326 +121327 POINT(47.654012691859826 -122.60956435965878) bank121327 +121328 POINT(48.582163566036485 -122.99941351443064) bank121328 +121329 POINT(46.81744860742568 -122.14728542110156) bank121329 +121330 POINT(47.64962052275522 -121.5081821085611) bank121330 +121331 POINT(48.48141070004264 -121.87240028431691) bank121331 +121332 POINT(46.79541619382996 -122.40571361699942) bank121332 +121333 POINT(47.08173835663648 -122.68576324280458) bank121333 +121334 POINT(47.93845868074292 -121.65941374598418) bank121334 +121335 POINT(47.663946838199394 -121.81592139144416) bank121335 +121336 POINT(47.16612166916113 -123.16191668459689) bank121336 +121337 POINT(47.86822828480041 -122.12444673762445) bank121337 +121338 POINT(47.33197035577217 -121.94338256553799) bank121338 +121339 POINT(46.66033517189866 -123.20258101099728) bank121339 +121340 POINT(47.54662987142969 -123.04052113068603) bank121340 +121341 POINT(46.999000069716004 -122.8258144874275) bank121341 +121342 POINT(48.23597802140693 -122.1490645811869) bank121342 +121343 POINT(47.66385913650173 -121.96265188924336) bank121343 +121344 POINT(48.52116847636776 -122.52968803133346) bank121344 +121345 POINT(46.70238730072047 -122.46095804837655) bank121345 +121346 POINT(47.25972101145996 -122.46861913913493) bank121346 +121347 POINT(46.81384529788531 -122.72554910293896) bank121347 +121348 POINT(46.75490813568734 -121.85022751833988) bank121348 +121349 POINT(47.94410231879978 -121.7253981628287) bank121349 +121350 POINT(46.9791624570403 -121.93575272935384) bank121350 +121351 POINT(47.41515771224161 -121.50452573449067) bank121351 +121352 POINT(46.854986742681824 -121.78165490954632) bank121352 +121353 POINT(48.304925226739776 -121.99038481972005) bank121353 +121354 POINT(47.38260671195603 -121.44785120351239) bank121354 +121355 POINT(47.81800442652038 -121.81525763343424) bank121355 +121356 POINT(48.113935021725 -123.02274443536237) bank121356 +121357 POINT(46.65890744889345 -121.71310833931588) bank121357 +121358 POINT(47.62861362484674 -121.74306424977455) bank121358 +121359 POINT(47.56639970510159 -122.91670620431255) bank121359 +121360 POINT(48.32603809489152 -122.5620606008395) bank121360 +121361 POINT(47.03538764178539 -122.7067002861224) bank121361 +121362 POINT(47.19292643323714 -123.25425874478138) bank121362 +121363 POINT(48.44459048137753 -121.61577547070853) bank121363 +121364 POINT(47.82370564993611 -122.61764784297225) bank121364 +121365 POINT(48.503792756748204 -122.9278744752168) bank121365 +121366 POINT(47.03373545766279 -122.57180590651126) bank121366 +121367 POINT(48.08352503546346 -122.77971161289123) bank121367 +121368 POINT(48.47526734055098 -121.71037896392535) bank121368 +121369 POINT(48.52399726712971 -121.4793090300969) bank121369 +121370 POINT(47.92532392542269 -122.303837326212) bank121370 +121371 POINT(46.77146904825561 -121.58219195503227) bank121371 +121372 POINT(47.09646483738229 -121.701875663589) bank121372 +121373 POINT(47.39390246740067 -121.43329016029122) bank121373 +121374 POINT(47.47504033502098 -122.15624946645408) bank121374 +121375 POINT(47.99414761663699 -123.12147757831221) bank121375 +121376 POINT(47.67993270421485 -121.59980464954421) bank121376 +121377 POINT(47.06669478946182 -122.30745352745176) bank121377 +121378 POINT(47.76483348805585 -122.43355393258014) bank121378 +121379 POINT(47.53239528043032 -122.17395189076625) bank121379 +121380 POINT(47.095657343113466 -121.5900254951978) bank121380 +121381 POINT(47.30400092221519 -123.24791544051723) bank121381 +121382 POINT(48.14893756216487 -121.48316286318673) bank121382 +121383 POINT(47.65989273963673 -121.72505156960757) bank121383 +121384 POINT(46.81665323403619 -121.60092142215065) bank121384 +121385 POINT(48.19509213343958 -123.07325442264818) bank121385 +121386 POINT(47.21739453506278 -121.768402304591) bank121386 +121387 POINT(47.120251143176965 -122.46654019524097) bank121387 +121388 POINT(46.926052148329184 -121.71277133911805) bank121388 +121389 POINT(46.72533604387906 -122.88009706905225) bank121389 +121390 POINT(48.004690178331856 -121.98221394607576) bank121390 +121391 POINT(48.50122156507592 -122.65425950190536) bank121391 +121392 POINT(47.481666943157244 -122.36473968222406) bank121392 +121393 POINT(46.81930770098589 -122.92804857870131) bank121393 +121394 POINT(48.554314425797756 -122.35222565110381) bank121394 +121395 POINT(47.564457647127746 -122.0141255245129) bank121395 +121396 POINT(48.58787797300089 -123.12274233779311) bank121396 +121397 POINT(46.70409205976709 -122.77240258397978) bank121397 +121398 POINT(48.27208933940396 -122.21382078552001) bank121398 +121399 POINT(48.51996283434209 -122.73762112197899) bank121399 +121400 POINT(47.524394969801236 -122.57318944614148) bank121400 +121401 POINT(47.48808899507481 -123.11737265618927) bank121401 +121402 POINT(48.579105332087714 -122.6529973379832) bank121402 +121403 POINT(47.220744446675695 -121.88885558509665) bank121403 +121404 POINT(47.06408887189126 -122.9224136855979) bank121404 +121405 POINT(47.54463872508627 -122.00823947363634) bank121405 +121406 POINT(47.8093469431486 -121.7105848075785) bank121406 +121407 POINT(47.286662979640276 -121.7036997740916) bank121407 +121408 POINT(46.690874290794305 -121.99846441454567) bank121408 +121409 POINT(48.30303401455828 -123.31442780910923) bank121409 +121410 POINT(48.187256063719836 -121.92453702836389) bank121410 +121411 POINT(47.44138723730009 -122.09239094704041) bank121411 +121412 POINT(48.030563885298065 -122.8974449618967) bank121412 +121413 POINT(47.61716214796105 -122.20896401413192) bank121413 +121414 POINT(46.818076692873696 -121.55804867051228) bank121414 +121415 POINT(47.74002646009606 -121.50053641350627) bank121415 +121416 POINT(48.392215049806936 -121.6585921719275) bank121416 +121417 POINT(48.515191583504794 -121.48748054040016) bank121417 +121418 POINT(48.175752384732476 -121.51501588872766) bank121418 +121419 POINT(47.60067076345093 -121.4165729665212) bank121419 +121420 POINT(48.23080140028459 -122.9939470080078) bank121420 +121421 POINT(48.38871301946861 -122.25719807796942) bank121421 +121422 POINT(48.45217641739221 -123.06393334860314) bank121422 +121423 POINT(47.718479415556615 -121.78269594599269) bank121423 +121424 POINT(47.193184345181635 -122.77706965885653) bank121424 +121425 POINT(46.85673085771497 -123.06130209034745) bank121425 +121426 POINT(48.57473657930697 -122.6992528571108) bank121426 +121427 POINT(47.31155240562923 -122.38701178647506) bank121427 +121428 POINT(46.89775876737413 -122.10582896614508) bank121428 +121429 POINT(47.97026646151916 -122.71693631661627) bank121429 +121430 POINT(47.78177722193538 -122.1595236515037) bank121430 +121431 POINT(47.86856810082664 -123.11267908155695) bank121431 +121432 POINT(47.14279204755544 -123.06228050062468) bank121432 +121433 POINT(47.03512357413931 -123.13735534134398) bank121433 +121434 POINT(46.97956895423471 -121.78191431314507) bank121434 +121435 POINT(47.07116912913963 -122.52231398910696) bank121435 +121436 POINT(46.751125226963 -121.39566160542289) bank121436 +121437 POINT(48.51018074083999 -122.2818416724193) bank121437 +121438 POINT(47.388463960840305 -121.81300392446522) bank121438 +121439 POINT(48.133754069852245 -121.95728710761622) bank121439 +121440 POINT(47.91200631651221 -122.85264321800318) bank121440 +121441 POINT(48.08031195052296 -121.6678687640995) bank121441 +121442 POINT(48.421417099504815 -122.1097757757056) bank121442 +121443 POINT(47.44576679701839 -123.0398514192383) bank121443 +121444 POINT(47.18217195467398 -121.91052948348077) bank121444 +121445 POINT(47.024435765986695 -122.68873091415067) bank121445 +121446 POINT(48.45964466770117 -121.44211742144277) bank121446 +121447 POINT(46.856657282598185 -121.70080751875595) bank121447 +121448 POINT(48.016216238715195 -122.4206872572454) bank121448 +121449 POINT(46.89702240422069 -122.06901155042314) bank121449 +121450 POINT(47.498257012956884 -123.08477509843253) bank121450 +121451 POINT(47.66831232939093 -122.9579281350856) bank121451 +121452 POINT(47.92151466872572 -123.08369165650794) bank121452 +121453 POINT(48.49920956735313 -122.4461547806288) bank121453 +121454 POINT(47.27172871355702 -121.81709125931735) bank121454 +121455 POINT(46.716541424593295 -122.13648985029485) bank121455 +121456 POINT(47.43258801827164 -122.37852941430516) bank121456 +121457 POINT(46.749989397344336 -122.72989657848453) bank121457 +121458 POINT(48.58063992708361 -123.01211550657688) bank121458 +121459 POINT(47.808292559552484 -121.82742470964966) bank121459 +121460 POINT(47.48685056580873 -121.47528726105357) bank121460 +121461 POINT(48.49193499950536 -122.83614296200574) bank121461 +121462 POINT(48.14152857566622 -122.89471450751208) bank121462 +121463 POINT(47.64201471088215 -123.03102621977429) bank121463 +121464 POINT(46.73282240157048 -123.10939207899217) bank121464 +121465 POINT(46.69595116654094 -123.16025789750212) bank121465 +121466 POINT(47.852349990495696 -121.62201257896596) bank121466 +121467 POINT(47.640295689881334 -121.97502467889012) bank121467 +121468 POINT(47.41795031236347 -123.12600180327128) bank121468 +121469 POINT(47.27607044174229 -122.54270702591583) bank121469 +121470 POINT(46.792501884271594 -122.76816658762937) bank121470 +121471 POINT(47.53347578358816 -121.92326425318882) bank121471 +121472 POINT(46.73146058574577 -121.64942949064961) bank121472 +121473 POINT(47.66735129401504 -121.59114562993169) bank121473 +121474 POINT(47.54680438266604 -123.17285144305627) bank121474 +121475 POINT(47.08805156303227 -122.4576461411071) bank121475 +121476 POINT(47.72564550820623 -122.88013012777131) bank121476 +121477 POINT(46.691050919946555 -121.45142719053781) bank121477 +121478 POINT(47.29145484430498 -121.33808274081181) bank121478 +121479 POINT(47.71026334326462 -122.12261795625128) bank121479 +121480 POINT(48.39200371832863 -122.7163603036879) bank121480 +121481 POINT(46.976091485763824 -123.08626919813582) bank121481 +121482 POINT(47.455005093707136 -121.65404749175354) bank121482 +121483 POINT(48.24455790016681 -122.84521558394444) bank121483 +121484 POINT(47.29167412439621 -122.21513257524293) bank121484 +121485 POINT(48.294837894054965 -121.67046863320316) bank121485 +121486 POINT(46.61932372618266 -122.53023671221517) bank121486 +121487 POINT(47.2358878002542 -123.15606192203434) bank121487 +121488 POINT(48.01519433612468 -121.60141722334471) bank121488 +121489 POINT(47.098927505235864 -121.79578233684025) bank121489 +121490 POINT(48.25244518356418 -123.1286472557277) bank121490 +121491 POINT(47.73865506873074 -121.47376456169206) bank121491 +121492 POINT(47.00165135646509 -121.98402574833078) bank121492 +121493 POINT(46.997999275052656 -121.75259869971872) bank121493 +121494 POINT(47.401434670092094 -121.93606568558724) bank121494 +121495 POINT(46.70588828104016 -121.99913695954619) bank121495 +121496 POINT(48.491343607688464 -122.5811575958446) bank121496 +121497 POINT(47.66404688504328 -122.64140276570849) bank121497 +121498 POINT(46.65742773363426 -121.74205243367803) bank121498 +121499 POINT(48.414495849301844 -121.7522797620365) bank121499 +121500 POINT(47.70882823595504 -122.15119585547434) bank121500 +121501 POINT(47.379469866281084 -122.68242882086092) bank121501 +121502 POINT(46.72484247725221 -122.56252206579379) bank121502 +121503 POINT(47.24269945946365 -122.69506599122384) bank121503 +121504 POINT(47.90089698112422 -121.47670219485583) bank121504 +121505 POINT(46.88771897092176 -121.60049232074184) bank121505 +121506 POINT(46.77307942810276 -122.22850130110142) bank121506 +121507 POINT(48.29552105040005 -122.81892663396445) bank121507 +121508 POINT(47.66655565219965 -121.75613723324238) bank121508 +121509 POINT(47.16806834509533 -121.38264983773402) bank121509 +121510 POINT(46.85517359847252 -122.74668072713793) bank121510 +121511 POINT(48.00398657041548 -121.41015940622776) bank121511 +121512 POINT(47.70499411384004 -122.0650442148248) bank121512 +121513 POINT(47.53659009924845 -122.14274358178127) bank121513 +121514 POINT(46.64162026375245 -122.95926430868286) bank121514 +121515 POINT(47.34831536453536 -122.60105622697706) bank121515 +121516 POINT(48.578302631184634 -122.83765902974434) bank121516 +121517 POINT(46.76024728252653 -121.65213935214767) bank121517 +121518 POINT(48.53612018057486 -121.91821378889577) bank121518 +121519 POINT(47.13652836363396 -121.38991808926593) bank121519 +121520 POINT(47.1113175603063 -121.90937824465372) bank121520 +121521 POINT(48.37579404010833 -122.77130802804707) bank121521 +121522 POINT(48.28567583342995 -121.58747122919242) bank121522 +121523 POINT(46.657439329726834 -123.06062784604673) bank121523 +121524 POINT(46.90030041486802 -121.61817067616857) bank121524 +121525 POINT(48.17377520510095 -122.28697502123221) bank121525 +121526 POINT(46.63163387212611 -121.91796586411009) bank121526 +121527 POINT(47.97409566221253 -121.80468341874331) bank121527 +121528 POINT(46.85230005736033 -121.45908214996336) bank121528 +121529 POINT(48.04540659422697 -122.68565252618032) bank121529 +121530 POINT(48.088263768485284 -123.04108589509521) bank121530 +121531 POINT(46.678842500686095 -122.89003030279677) bank121531 +121532 POINT(46.73062574424299 -121.6773742623286) bank121532 +121533 POINT(48.33631245406643 -122.81664977195256) bank121533 +121534 POINT(48.035613520139904 -122.73306819436084) bank121534 +121535 POINT(47.49543474239061 -122.20397218471798) bank121535 +121536 POINT(47.6536455924016 -121.36295620110025) bank121536 +121537 POINT(47.33652240386095 -121.73786860965934) bank121537 +121538 POINT(47.7782834335265 -121.83467737104199) bank121538 +121539 POINT(47.96054152675619 -122.2806659915595) bank121539 +121540 POINT(47.90035703313298 -122.93708511496953) bank121540 +121541 POINT(48.099421532688986 -122.44896311509845) bank121541 +121542 POINT(47.89371309839408 -122.28826992283778) bank121542 +121543 POINT(47.310790037098975 -121.74050292743469) bank121543 +121544 POINT(47.961539956298544 -121.91453472403137) bank121544 +121545 POINT(47.17197477189143 -121.50194621106556) bank121545 +121546 POINT(46.8181469910645 -121.41911037133725) bank121546 +121547 POINT(47.46287023864913 -122.22636654768279) bank121547 +121548 POINT(48.44729502029058 -122.62875566471583) bank121548 +121549 POINT(47.0340850469694 -121.82400974979305) bank121549 +121550 POINT(47.13538328669976 -122.46410308380656) bank121550 +121551 POINT(47.89544000009638 -122.0002933194731) bank121551 +121552 POINT(48.605537199629985 -121.47820629327985) bank121552 +121553 POINT(47.97765226508261 -121.89819788957759) bank121553 +121554 POINT(48.01756505075129 -122.74164485233528) bank121554 +121555 POINT(47.668129212661476 -122.62345419618669) bank121555 +121556 POINT(48.49477455463536 -122.68156785068716) bank121556 +121557 POINT(48.54520828552827 -121.96474936345597) bank121557 +121558 POINT(46.933632229638526 -121.90645106931058) bank121558 +121559 POINT(47.22822178252851 -122.71986061949676) bank121559 +121560 POINT(48.0127781197953 -122.27749563505098) bank121560 +121561 POINT(47.41658402971608 -122.79621656592256) bank121561 +121562 POINT(48.332946867550355 -121.81380635879823) bank121562 +121563 POINT(47.75518194772341 -122.51459272106271) bank121563 +121564 POINT(47.211965174342446 -123.24930181952173) bank121564 +121565 POINT(48.303994730623586 -121.66426409981443) bank121565 +121566 POINT(47.238886901170616 -122.92443532880606) bank121566 +121567 POINT(47.25682006209139 -123.11884966027799) bank121567 +121568 POINT(47.30411677489315 -123.14796315938158) bank121568 +121569 POINT(48.29958974477302 -121.71572152308936) bank121569 +121570 POINT(48.544243745646895 -122.94732399558987) bank121570 +121571 POINT(48.10745967010932 -121.84268703981608) bank121571 +121572 POINT(47.41053048172681 -121.98992791644015) bank121572 +121573 POINT(46.66110296381774 -122.09869639370403) bank121573 +121574 POINT(48.361052989113524 -122.86269446647584) bank121574 +121575 POINT(47.59145287416647 -121.55152444273301) bank121575 +121576 POINT(48.101468468253884 -121.55008346370187) bank121576 +121577 POINT(47.36128463457897 -121.51229950914066) bank121577 +121578 POINT(47.913662530495166 -121.83995907042204) bank121578 +121579 POINT(47.83640761606754 -121.43383912501373) bank121579 +121580 POINT(48.39046170292002 -122.41650628217731) bank121580 +121581 POINT(47.667805719314536 -122.31624390510503) bank121581 +121582 POINT(47.432326400840836 -121.45549609249692) bank121582 +121583 POINT(48.146916372864155 -122.30093260951529) bank121583 +121584 POINT(48.316364434449035 -123.31451816318948) bank121584 +121585 POINT(47.72627371825743 -122.8753117616212) bank121585 +121586 POINT(47.62431575416408 -121.4493086127339) bank121586 +121587 POINT(46.91320930823984 -122.16601651516868) bank121587 +121588 POINT(47.94541755302959 -121.49836181593557) bank121588 +121589 POINT(47.26858227448012 -122.36794721509081) bank121589 +121590 POINT(47.742806038664064 -121.45224178125598) bank121590 +121591 POINT(48.32688527084853 -121.83062412678908) bank121591 +121592 POINT(47.98858034866111 -121.41546046513446) bank121592 +121593 POINT(47.71004475925423 -123.2190730785515) bank121593 +121594 POINT(48.45296458271737 -121.56840328018966) bank121594 +121595 POINT(47.71133525563036 -121.63777851166242) bank121595 +121596 POINT(47.43157011175912 -123.02875490000332) bank121596 +121597 POINT(47.94421415822745 -122.79199741402452) bank121597 +121598 POINT(47.60674683762339 -122.0463562104522) bank121598 +121599 POINT(46.62822358727376 -121.72667517401882) bank121599 +121600 POINT(47.66692930787501 -122.93687515030842) bank121600 +121601 POINT(47.20415892815084 -122.93223758814644) bank121601 +121602 POINT(46.89458987278257 -121.7718218887905) bank121602 +121603 POINT(48.03375187069278 -121.83226046086968) bank121603 +121604 POINT(48.471421848643445 -122.77141406324144) bank121604 +121605 POINT(47.19815550435543 -122.68777649589633) bank121605 +121606 POINT(48.13119881254582 -122.62071106502155) bank121606 +121607 POINT(46.81011502385226 -121.33737700044394) bank121607 +121608 POINT(48.53663169526373 -122.46236730543475) bank121608 +121609 POINT(47.32524182600512 -122.05602894347956) bank121609 +121610 POINT(47.30786013567494 -122.140169693488) bank121610 +121611 POINT(47.021991098861065 -123.0963962232916) bank121611 +121612 POINT(48.2646009870055 -121.55410863566871) bank121612 +121613 POINT(47.296212658832 -122.12799922557126) bank121613 +121614 POINT(47.11845226056025 -121.47676968305083) bank121614 +121615 POINT(47.170537368484176 -122.08538255741509) bank121615 +121616 POINT(47.45453707099086 -121.5550012739564) bank121616 +121617 POINT(46.6729177761946 -122.26378615647309) bank121617 +121618 POINT(46.79581837047257 -122.63981513997551) bank121618 +121619 POINT(47.926242924570026 -121.8244182668724) bank121619 +121620 POINT(47.88028913380574 -122.82298604319209) bank121620 +121621 POINT(47.811261933352476 -123.30322487548187) bank121621 +121622 POINT(46.73922043656566 -123.16935421044383) bank121622 +121623 POINT(48.373170543969536 -121.83648732356356) bank121623 +121624 POINT(48.471710675104504 -121.40976793119795) bank121624 +121625 POINT(47.23448854670649 -122.20387536466795) bank121625 +121626 POINT(48.42811698257952 -123.21050288581573) bank121626 +121627 POINT(47.81101238527243 -122.50354506480069) bank121627 +121628 POINT(47.578660200331605 -123.18442272505435) bank121628 +121629 POINT(47.0933636454672 -122.24025069850835) bank121629 +121630 POINT(48.23756828350073 -122.48531824677828) bank121630 +121631 POINT(48.26867816293326 -122.4645261942632) bank121631 +121632 POINT(47.85907878252973 -121.46743099380302) bank121632 +121633 POINT(47.371932011395145 -123.12219332716823) bank121633 +121634 POINT(48.27391466101618 -122.37264366579191) bank121634 +121635 POINT(46.60736325928106 -121.5524363424703) bank121635 +121636 POINT(47.54673724674202 -123.05463765960697) bank121636 +121637 POINT(48.21595991107335 -122.92737454135202) bank121637 +121638 POINT(47.24240145917211 -122.86919233119991) bank121638 +121639 POINT(46.78722489892268 -122.64957717246955) bank121639 +121640 POINT(47.669678704890764 -123.2740928903106) bank121640 +121641 POINT(47.09418616134619 -123.18020448806337) bank121641 +121642 POINT(47.89600508080828 -122.07544404503008) bank121642 +121643 POINT(48.41527946228787 -121.52448106490088) bank121643 +121644 POINT(48.08169288667611 -122.84207831323967) bank121644 +121645 POINT(47.513745731028536 -123.32035142717204) bank121645 +121646 POINT(46.80638244508277 -121.50124868581992) bank121646 +121647 POINT(48.06838852184893 -122.24808642118008) bank121647 +121648 POINT(48.34199001087867 -122.82705734555398) bank121648 +121649 POINT(48.27526698689779 -122.23638397731078) bank121649 +121650 POINT(48.00878687313393 -122.34615979174332) bank121650 +121651 POINT(47.5935712825084 -123.3080147900959) bank121651 +121652 POINT(47.650336224211685 -121.393240307625) bank121652 +121653 POINT(48.010132165311646 -121.78669493969689) bank121653 +121654 POINT(46.95755512504307 -122.47443024068586) bank121654 +121655 POINT(46.863996145157145 -122.88273761362262) bank121655 +121656 POINT(47.57549347319005 -123.15909839825454) bank121656 +121657 POINT(47.92788048205641 -123.15035205468068) bank121657 +121658 POINT(46.78495812420089 -123.04991022384115) bank121658 +121659 POINT(47.31256757983038 -122.10213213167579) bank121659 +121660 POINT(47.32274868395677 -121.63137199745127) bank121660 +121661 POINT(48.41257750198627 -123.15129272743278) bank121661 +121662 POINT(46.64645462929512 -122.03148884776688) bank121662 +121663 POINT(48.511976742638005 -122.60815105791234) bank121663 +121664 POINT(46.79855069433494 -121.44696234496286) bank121664 +121665 POINT(47.20723892988539 -121.81833276366616) bank121665 +121666 POINT(48.12482969045619 -122.77879085662435) bank121666 +121667 POINT(47.849043187409 -121.36964836435085) bank121667 +121668 POINT(46.99174138314433 -122.43552740393685) bank121668 +121669 POINT(47.03326927161527 -121.4859215264114) bank121669 +121670 POINT(48.438199671092484 -121.46448520368217) bank121670 +121671 POINT(47.07373729199625 -122.60903539697586) bank121671 +121672 POINT(46.963208998582104 -122.62015408034041) bank121672 +121673 POINT(46.97587529059248 -122.30826755919374) bank121673 +121674 POINT(47.51118499920624 -121.711203083192) bank121674 +121675 POINT(47.57229419759978 -122.96974490726822) bank121675 +121676 POINT(47.159563359923794 -122.80627528727543) bank121676 +121677 POINT(47.61907955350725 -122.33838705216633) bank121677 +121678 POINT(46.9249403237488 -123.14322444479272) bank121678 +121679 POINT(48.303721198090564 -122.75035821274676) bank121679 +121680 POINT(47.059156090863716 -122.07655443359211) bank121680 +121681 POINT(47.84531703326773 -122.21057436425535) bank121681 +121682 POINT(46.753540206681464 -122.9859245635435) bank121682 +121683 POINT(46.90372902333134 -121.62876910284393) bank121683 +121684 POINT(48.13197958674262 -122.03150938892503) bank121684 +121685 POINT(47.14739088071167 -122.2471779459065) bank121685 +121686 POINT(48.17282836212207 -122.34236056407728) bank121686 +121687 POINT(48.19047745374574 -123.11524573201665) bank121687 +121688 POINT(47.68918151115851 -122.30670966992763) bank121688 +121689 POINT(47.80216410169857 -123.13573121060098) bank121689 +121690 POINT(46.74787242848799 -121.50089905967543) bank121690 +121691 POINT(48.5937300293896 -121.76898056611225) bank121691 +121692 POINT(47.22112450159374 -121.99431351292746) bank121692 +121693 POINT(46.86186489599838 -122.59998016176678) bank121693 +121694 POINT(48.031331443238315 -122.67187323083542) bank121694 +121695 POINT(47.7597094056064 -121.36538768141635) bank121695 +121696 POINT(48.595545426354576 -122.40070386938541) bank121696 +121697 POINT(48.38765422261539 -121.73695357541584) bank121697 +121698 POINT(47.156396805249535 -122.07312662990932) bank121698 +121699 POINT(46.96098125444659 -122.38400795095833) bank121699 +121700 POINT(48.12665037688381 -122.67932459732035) bank121700 +121701 POINT(47.24471982070565 -121.88778441979443) bank121701 +121702 POINT(47.14283829811802 -122.88049440870307) bank121702 +121703 POINT(47.21575193731554 -122.0282516749412) bank121703 +121704 POINT(48.44176275777456 -122.19017324212902) bank121704 +121705 POINT(48.02754589187235 -121.95686582629862) bank121705 +121706 POINT(47.35497912542324 -121.65914822062518) bank121706 +121707 POINT(48.13603195237471 -121.87745011116476) bank121707 +121708 POINT(47.05717799877227 -121.59401890200887) bank121708 +121709 POINT(48.06470731464043 -121.67698041289849) bank121709 +121710 POINT(46.73205982865703 -121.8557104072918) bank121710 +121711 POINT(48.52355155556874 -122.37040377724269) bank121711 +121712 POINT(48.57838210543808 -121.77051742522664) bank121712 +121713 POINT(47.521420620903505 -121.4860003977822) bank121713 +121714 POINT(47.78383441292529 -122.76531307290674) bank121714 +121715 POINT(47.00354446195654 -121.41938723304912) bank121715 +121716 POINT(46.65739818912375 -122.67741203833408) bank121716 +121717 POINT(48.216975124762506 -121.4519129874081) bank121717 +121718 POINT(46.66501392393552 -122.74473629251835) bank121718 +121719 POINT(48.074229012232934 -121.48233672051343) bank121719 +121720 POINT(48.145607710183384 -121.45683409209686) bank121720 +121721 POINT(46.737120118400156 -121.5569310129828) bank121721 +121722 POINT(48.502366317938 -121.51488630542099) bank121722 +121723 POINT(48.46473216632687 -122.75537828289487) bank121723 +121724 POINT(47.822365857788476 -122.22743805888531) bank121724 +121725 POINT(48.01634204609734 -121.79469117133729) bank121725 +121726 POINT(47.571573432522534 -121.87333986430401) bank121726 +121727 POINT(47.59102189533564 -123.09304295289579) bank121727 +121728 POINT(48.55596911228291 -121.36681445696622) bank121728 +121729 POINT(47.77120611186297 -122.29555041816646) bank121729 +121730 POINT(47.68588833707113 -121.88601705917291) bank121730 +121731 POINT(46.61590561914816 -121.6044937784311) bank121731 +121732 POINT(47.766236121488376 -121.39561780619991) bank121732 +121733 POINT(47.40684417458344 -122.10172612922013) bank121733 +121734 POINT(46.88333645250518 -121.46332085256353) bank121734 +121735 POINT(46.84880297283654 -121.6793463061826) bank121735 +121736 POINT(47.39217530775069 -122.80477313919602) bank121736 +121737 POINT(47.58392622235646 -121.8772632521667) bank121737 +121738 POINT(48.4723395634302 -122.79939558209132) bank121738 +121739 POINT(47.530529033194334 -122.96651334179344) bank121739 +121740 POINT(48.00507598500667 -121.63747585090591) bank121740 +121741 POINT(47.27493115539475 -122.41092402550441) bank121741 +121742 POINT(48.20829838894665 -121.4724271275087) bank121742 +121743 POINT(47.96698173175538 -122.84143475267821) bank121743 +121744 POINT(48.38673893595232 -123.16735052909304) bank121744 +121745 POINT(48.32604186173432 -123.02049780636548) bank121745 +121746 POINT(46.989547718866554 -123.1888717378327) bank121746 +121747 POINT(47.61204637722141 -122.72051554803195) bank121747 +121748 POINT(47.50022409547083 -122.4794555533712) bank121748 +121749 POINT(47.76175537393875 -123.00832330406826) bank121749 +121750 POINT(47.719264462677664 -122.76522002058985) bank121750 +121751 POINT(47.24449726753362 -122.61680974690044) bank121751 +121752 POINT(47.9585658167533 -122.30776613325355) bank121752 +121753 POINT(47.67304870043052 -123.17480053228007) bank121753 +121754 POINT(48.28589219817605 -123.08296554632996) bank121754 +121755 POINT(46.936975547953296 -121.65279023898913) bank121755 +121756 POINT(48.34250673360215 -121.89129595800556) bank121756 +121757 POINT(47.43579049192368 -123.25847978674084) bank121757 +121758 POINT(46.65392895524507 -122.67237571223427) bank121758 +121759 POINT(48.55343101284839 -123.1469340377703) bank121759 +121760 POINT(47.20883492176803 -121.58412925296346) bank121760 +121761 POINT(47.81023333538967 -123.22372127912193) bank121761 +121762 POINT(47.17432719675689 -122.20204526896121) bank121762 +121763 POINT(47.877914436029904 -123.02468702238649) bank121763 +121764 POINT(46.67724146340931 -121.73678484216556) bank121764 +121765 POINT(46.962471805554365 -123.23366828347224) bank121765 +121766 POINT(47.09591063048553 -122.01648096362952) bank121766 +121767 POINT(47.36345790992553 -123.07837017573223) bank121767 +121768 POINT(48.03503436776886 -123.30971462042464) bank121768 +121769 POINT(46.916318699513056 -122.15838911483807) bank121769 +121770 POINT(47.80372745124014 -121.62271861532652) bank121770 +121771 POINT(47.890613616161765 -121.35509798765808) bank121771 +121772 POINT(48.393374261177776 -123.07090342984782) bank121772 +121773 POINT(46.61903361561731 -122.13386481916888) bank121773 +121774 POINT(48.154681703186014 -122.15087063699123) bank121774 +121775 POINT(47.86364435774276 -121.84191249461622) bank121775 +121776 POINT(47.81055640742375 -123.26044659011323) bank121776 +121777 POINT(48.16451985025239 -122.84329184188691) bank121777 +121778 POINT(46.6289988980774 -122.03039834601337) bank121778 +121779 POINT(47.44320087554959 -122.73914978566874) bank121779 +121780 POINT(48.03655088750521 -122.84299284535726) bank121780 +121781 POINT(47.085823800709655 -122.4297688726324) bank121781 +121782 POINT(46.721145907817025 -121.59059995229464) bank121782 +121783 POINT(48.55118110486422 -122.13388464357696) bank121783 +121784 POINT(48.234891729711116 -122.90611381439328) bank121784 +121785 POINT(48.31159835414754 -122.07055206513556) bank121785 +121786 POINT(48.392962320629664 -121.5255079467091) bank121786 +121787 POINT(46.73777911090689 -122.7244284224058) bank121787 +121788 POINT(48.068129565660435 -121.55020816641543) bank121788 +121789 POINT(47.405335080377604 -122.85999212941974) bank121789 +121790 POINT(46.63037004280874 -121.86441847861211) bank121790 +121791 POINT(46.806958255747105 -122.53225889735975) bank121791 +121792 POINT(46.97284370274989 -123.1048946843425) bank121792 +121793 POINT(46.81982568494166 -121.35901215857349) bank121793 +121794 POINT(47.9078794689275 -121.90230966611249) bank121794 +121795 POINT(47.35606608217848 -121.33673486836024) bank121795 +121796 POINT(47.1117801563936 -122.04505049471226) bank121796 +121797 POINT(47.75426310758527 -122.46887070538142) bank121797 +121798 POINT(48.31470745597976 -122.97704949388564) bank121798 +121799 POINT(47.16677171789338 -123.32808567537087) bank121799 +121800 POINT(47.578361126631 -122.44005294038264) bank121800 +121801 POINT(46.83588693845676 -121.63106805650692) bank121801 +121802 POINT(48.26210033379581 -123.08643431803794) bank121802 +121803 POINT(48.0990455293039 -122.08779381587055) bank121803 +121804 POINT(48.48421955986056 -123.01667713670997) bank121804 +121805 POINT(47.8022913943985 -123.05306780049453) bank121805 +121806 POINT(47.967726788209 -121.60431026930313) bank121806 +121807 POINT(48.08014756681516 -121.42491926154183) bank121807 +121808 POINT(47.439337323195986 -122.61468668685147) bank121808 +121809 POINT(48.49529403709485 -121.98346183319681) bank121809 +121810 POINT(46.69131588771139 -121.84341765039267) bank121810 +121811 POINT(47.67163402605101 -122.84001239706164) bank121811 +121812 POINT(48.016446050366966 -122.9568759754768) bank121812 +121813 POINT(46.78315985256321 -122.88846384366269) bank121813 +121814 POINT(46.812554547558946 -123.04513578206088) bank121814 +121815 POINT(47.97789067778165 -122.44508614966584) bank121815 +121816 POINT(47.149567222858664 -122.8023947666692) bank121816 +121817 POINT(46.69103558437029 -121.69696371958635) bank121817 +121818 POINT(47.73362658631889 -123.24334847296001) bank121818 +121819 POINT(48.120020379395534 -121.64209066902696) bank121819 +121820 POINT(47.775147579750865 -122.8591551324112) bank121820 +121821 POINT(47.77606232621786 -121.37218976748397) bank121821 +121822 POINT(46.76354185914908 -121.75396806227023) bank121822 +121823 POINT(47.67542647847032 -122.5208232579672) bank121823 +121824 POINT(47.21190545851345 -122.91324161329842) bank121824 +121825 POINT(48.37980607712755 -121.8780516926887) bank121825 +121826 POINT(48.16661493869536 -122.78124926614298) bank121826 +121827 POINT(48.19724362862118 -122.60210684416178) bank121827 +121828 POINT(47.531279701971435 -121.53231128327477) bank121828 +121829 POINT(48.29393617544485 -122.99615252727668) bank121829 +121830 POINT(47.34286696588282 -122.22527122532102) bank121830 +121831 POINT(47.02301020409573 -123.14289696585222) bank121831 +121832 POINT(46.87320707962886 -123.01770110514221) bank121832 +121833 POINT(46.6685010486996 -122.23056795676598) bank121833 +121834 POINT(47.23543367457737 -121.71911287408214) bank121834 +121835 POINT(46.994915293540366 -121.70023234070597) bank121835 +121836 POINT(47.96946350378989 -122.93184567604308) bank121836 +121837 POINT(47.029263205866954 -122.47646709991973) bank121837 +121838 POINT(47.635520508742175 -121.87128900323248) bank121838 +121839 POINT(48.352975049785684 -122.31121389965459) bank121839 +121840 POINT(48.58791120611786 -122.86752719785774) bank121840 +121841 POINT(47.142585998669105 -122.98629027170976) bank121841 +121842 POINT(47.86630999793334 -122.19737537338005) bank121842 +121843 POINT(48.29879000092762 -121.61341382413367) bank121843 +121844 POINT(48.143352989868816 -122.06355428073526) bank121844 +121845 POINT(47.99269480438397 -121.77238298030736) bank121845 +121846 POINT(46.71176830152665 -121.8092150309905) bank121846 +121847 POINT(48.273646200245835 -122.47753622061204) bank121847 +121848 POINT(47.55971329923955 -123.1575442758402) bank121848 +121849 POINT(47.81810153919846 -122.23113788629853) bank121849 +121850 POINT(46.9004887460631 -122.97664712636471) bank121850 +121851 POINT(46.74696651966528 -122.23122316333507) bank121851 +121852 POINT(48.52280911193599 -122.23861220913815) bank121852 +121853 POINT(48.002695166849854 -122.86700801291927) bank121853 +121854 POINT(48.27295969943059 -123.21761333490267) bank121854 +121855 POINT(47.843364119431115 -123.20760881893779) bank121855 +121856 POINT(46.886512521488136 -122.77752012474409) bank121856 +121857 POINT(47.26090913061802 -122.25992725659451) bank121857 +121858 POINT(47.9624701553648 -121.44405164700714) bank121858 +121859 POINT(47.717468100477724 -122.01770984374296) bank121859 +121860 POINT(48.480405012004425 -122.0728612819917) bank121860 +121861 POINT(46.83403554761346 -121.93524062103747) bank121861 +121862 POINT(46.90022203428086 -122.486208034693) bank121862 +121863 POINT(48.27373749103366 -122.46266798408023) bank121863 +121864 POINT(47.253099160595916 -123.01396669425627) bank121864 +121865 POINT(46.931018555236534 -121.46544712266845) bank121865 +121866 POINT(48.04579351364943 -122.95616811873067) bank121866 +121867 POINT(47.76681856373503 -121.71955660689086) bank121867 +121868 POINT(46.690582166957725 -122.48045921822283) bank121868 +121869 POINT(48.6019463311821 -121.5865084636086) bank121869 +121870 POINT(47.752102288207844 -123.07523742786395) bank121870 +121871 POINT(48.414602552838296 -122.96694082934663) bank121871 +121872 POINT(48.06852253802099 -122.03451076346403) bank121872 +121873 POINT(46.936390633881096 -122.5528343370762) bank121873 +121874 POINT(48.52912263528567 -123.1758464363542) bank121874 +121875 POINT(48.55465405799483 -122.35988450294099) bank121875 +121876 POINT(47.3024159304679 -122.82576320810287) bank121876 +121877 POINT(47.84274282394086 -121.50529653876812) bank121877 +121878 POINT(48.06232610926736 -122.3734067112449) bank121878 +121879 POINT(48.58565986727219 -122.10416792815853) bank121879 +121880 POINT(47.226469677202076 -122.9010259872946) bank121880 +121881 POINT(47.90496053751715 -122.9697924891528) bank121881 +121882 POINT(47.46043345625164 -123.04795248966302) bank121882 +121883 POINT(47.86730588565661 -122.69048476846184) bank121883 +121884 POINT(46.97995452988169 -122.15985280555333) bank121884 +121885 POINT(48.55376834584291 -122.89853380276116) bank121885 +121886 POINT(48.41910589454378 -121.6654728314235) bank121886 +121887 POINT(47.31642198191729 -122.28183470560299) bank121887 +121888 POINT(47.66686196477877 -122.22595628251418) bank121888 +121889 POINT(48.446216930621446 -122.34821231565029) bank121889 +121890 POINT(47.47722352513124 -121.92497697846191) bank121890 +121891 POINT(47.26404789694228 -122.50395326803829) bank121891 +121892 POINT(48.316669958443455 -121.84461881860986) bank121892 +121893 POINT(48.39523236891239 -121.89265268333331) bank121893 +121894 POINT(47.58882577434386 -122.55425682609155) bank121894 +121895 POINT(47.98052747974849 -121.40984819508368) bank121895 +121896 POINT(47.763128075681045 -122.42671978728816) bank121896 +121897 POINT(47.0593928636925 -123.32836273552248) bank121897 +121898 POINT(46.82528188809944 -121.75215943356523) bank121898 +121899 POINT(46.679618626410594 -122.12638258539266) bank121899 +121900 POINT(47.59512595524191 -122.23029108237056) bank121900 +121901 POINT(47.1967608381034 -122.81184676398922) bank121901 +121902 POINT(48.028419053920096 -122.62898705340469) bank121902 +121903 POINT(46.825113247803486 -123.18232853716304) bank121903 +121904 POINT(46.8523567718245 -121.84127419611518) bank121904 +121905 POINT(46.97214614534344 -121.52818506946652) bank121905 +121906 POINT(46.701259872860234 -122.30924555298856) bank121906 +121907 POINT(47.03931932470959 -121.81121616570877) bank121907 +121908 POINT(48.206900718964555 -121.7115892603941) bank121908 +121909 POINT(47.50051430487173 -123.28348598654408) bank121909 +121910 POINT(46.91795740845425 -121.59746286025707) bank121910 +121911 POINT(47.77532180757678 -122.46797861423667) bank121911 +121912 POINT(48.23777835109738 -122.73849843324493) bank121912 +121913 POINT(46.863857634109394 -121.73947617642675) bank121913 +121914 POINT(48.411038281894115 -122.74371380768687) bank121914 +121915 POINT(47.122358605466644 -121.78598284401971) bank121915 +121916 POINT(47.45678017969442 -123.19242990178007) bank121916 +121917 POINT(48.403629637719554 -121.53669778502609) bank121917 +121918 POINT(47.97419850302913 -123.19792872227946) bank121918 +121919 POINT(47.40124012534318 -122.12029791741786) bank121919 +121920 POINT(47.90633681014657 -123.00265234935014) bank121920 +121921 POINT(47.55894570898301 -123.2282450048006) bank121921 +121922 POINT(47.77444607699997 -122.40646755249273) bank121922 +121923 POINT(47.441557311689834 -121.62561146216257) bank121923 +121924 POINT(47.73903316742122 -122.17904991973576) bank121924 +121925 POINT(48.28503192045222 -122.05164579551511) bank121925 +121926 POINT(48.075956704637775 -122.70513037236066) bank121926 +121927 POINT(47.71192700548814 -122.98249761973526) bank121927 +121928 POINT(47.129128138066555 -121.76612080314695) bank121928 +121929 POINT(48.47131837738314 -122.49660161854497) bank121929 +121930 POINT(47.543070055093075 -122.68203256444637) bank121930 +121931 POINT(47.14151447719827 -122.64852366553079) bank121931 +121932 POINT(47.57187111289766 -122.73157658500169) bank121932 +121933 POINT(48.04550044055188 -122.29663934127836) bank121933 +121934 POINT(46.740493218380394 -121.96518623067018) bank121934 +121935 POINT(47.831153325144165 -122.68393415923157) bank121935 +121936 POINT(48.07181602556108 -122.5428012566395) bank121936 +121937 POINT(48.11649732113729 -122.91877092643784) bank121937 +121938 POINT(47.439433228589124 -122.13426037524634) bank121938 +121939 POINT(48.12178009350775 -121.53182627567026) bank121939 +121940 POINT(48.11413334938887 -122.56996371716257) bank121940 +121941 POINT(47.09040270026927 -123.10205128416234) bank121941 +121942 POINT(47.81182406633411 -122.59835567714174) bank121942 +121943 POINT(48.521324406562066 -121.71013764752307) bank121943 +121944 POINT(47.43996227521105 -121.65130301854767) bank121944 +121945 POINT(46.847161598263725 -122.95183090960825) bank121945 +121946 POINT(47.922631900354865 -123.28271801707393) bank121946 +121947 POINT(48.42979471134551 -122.759686996579) bank121947 +121948 POINT(48.551776354471244 -123.31114085887914) bank121948 +121949 POINT(47.62233077787455 -123.08312207014316) bank121949 +121950 POINT(47.563000180468684 -121.90914238498266) bank121950 +121951 POINT(48.35684675157598 -122.09879826862125) bank121951 +121952 POINT(48.43020349102486 -122.4565235568489) bank121952 +121953 POINT(47.79939725318773 -121.48256008737559) bank121953 +121954 POINT(47.28763524541952 -123.07857547597514) bank121954 +121955 POINT(47.03906515928881 -122.5858213601584) bank121955 +121956 POINT(46.95445475272712 -123.04579395215808) bank121956 +121957 POINT(47.85129502192694 -121.54454033038851) bank121957 +121958 POINT(46.83627372564127 -121.81211548422831) bank121958 +121959 POINT(48.51066893572555 -122.82344483705606) bank121959 +121960 POINT(48.025500755548805 -122.64302352654369) bank121960 +121961 POINT(47.00839963835504 -122.70182603199721) bank121961 +121962 POINT(46.69587642141671 -122.77194270992457) bank121962 +121963 POINT(47.45027524749233 -122.21420617886112) bank121963 +121964 POINT(47.31945821104963 -121.48521936458224) bank121964 +121965 POINT(48.49561091461003 -121.74744445133202) bank121965 +121966 POINT(46.66967864843704 -121.67686339480566) bank121966 +121967 POINT(46.877768914228156 -121.3714196914926) bank121967 +121968 POINT(48.22170797599813 -123.1692314062539) bank121968 +121969 POINT(46.72939672877475 -122.92277654718598) bank121969 +121970 POINT(47.582740230652114 -122.97607318422446) bank121970 +121971 POINT(47.590092770118865 -121.7531053512128) bank121971 +121972 POINT(48.327599385716226 -122.3435615300285) bank121972 +121973 POINT(47.73320153432457 -123.04436404573083) bank121973 +121974 POINT(47.94612620366665 -122.44134536704685) bank121974 +121975 POINT(47.32656389749949 -123.14705671856854) bank121975 +121976 POINT(48.277450308026225 -121.7774140091554) bank121976 +121977 POINT(48.065532486982136 -122.902935342552) bank121977 +121978 POINT(47.00509148289213 -122.48967862371819) bank121978 +121979 POINT(47.06793825667921 -123.05864288233133) bank121979 +121980 POINT(47.78041176242803 -121.52852977481446) bank121980 +121981 POINT(46.84403158007298 -123.25898121012611) bank121981 +121982 POINT(46.93496864494627 -122.51836445818473) bank121982 +121983 POINT(47.699380277552805 -122.47618832720175) bank121983 +121984 POINT(46.76364798746707 -121.57204334840401) bank121984 +121985 POINT(46.88473834105038 -122.31920419512825) bank121985 +121986 POINT(47.10348287574096 -122.12749602982717) bank121986 +121987 POINT(47.31581995737327 -122.42089591707865) bank121987 +121988 POINT(47.720037791034464 -121.46386216086997) bank121988 +121989 POINT(47.20525151421044 -123.1525632034671) bank121989 +121990 POINT(48.44309331181319 -121.36131842726365) bank121990 +121991 POINT(47.27424141351933 -122.37513056916455) bank121991 +121992 POINT(48.206996409389426 -121.57966149773793) bank121992 +121993 POINT(48.116765761734506 -122.01944360317532) bank121993 +121994 POINT(48.34581230097981 -121.91979802387641) bank121994 +121995 POINT(48.467697950431464 -122.03997658700567) bank121995 +121996 POINT(46.83837149705566 -122.88411757453552) bank121996 +121997 POINT(46.94184880607548 -123.30924418491864) bank121997 +121998 POINT(47.77012580210854 -121.5531657474274) bank121998 +121999 POINT(48.14900154812507 -123.04632883417192) bank121999 +122000 POINT(47.43267775467908 -122.08985520883364) bank122000 +122001 POINT(48.081592230239835 -122.94231591022984) bank122001 +122002 POINT(47.942417345571826 -122.58932497664081) bank122002 +122003 POINT(46.78904896471923 -121.740510677925) bank122003 +122004 POINT(47.87087644790156 -122.91005818171065) bank122004 +122005 POINT(46.862889170081246 -121.87952354956774) bank122005 +122006 POINT(47.427936150205554 -122.38179845633913) bank122006 +122007 POINT(46.732092232564895 -122.48098394834881) bank122007 +122008 POINT(47.31975876399965 -122.28197115669279) bank122008 +122009 POINT(47.15139002933536 -122.4668614552539) bank122009 +122010 POINT(48.316302013787066 -122.33449655299391) bank122010 +122011 POINT(47.625223177375666 -122.98105775893484) bank122011 +122012 POINT(47.718234087829174 -121.96061745303972) bank122012 +122013 POINT(48.07408490781271 -121.8605133227869) bank122013 +122014 POINT(46.82165033651698 -123.23846943044354) bank122014 +122015 POINT(46.60763779398274 -122.69774423284048) bank122015 +122016 POINT(48.05929006600436 -122.69889996578652) bank122016 +122017 POINT(47.94900788331356 -122.89499243712761) bank122017 +122018 POINT(47.13764345201338 -121.91962878303107) bank122018 +122019 POINT(47.94338969182735 -122.02419627319654) bank122019 +122020 POINT(47.70282254648442 -121.81060521693915) bank122020 +122021 POINT(47.36402979996881 -122.85514910499381) bank122021 +122022 POINT(47.432085408323324 -122.4529471338671) bank122022 +122023 POINT(47.23091331290027 -121.96862483042496) bank122023 +122024 POINT(47.89217159968208 -121.55880323925317) bank122024 +122025 POINT(46.854399247197584 -121.59437835029792) bank122025 +122026 POINT(48.55614513774637 -123.15121629972029) bank122026 +122027 POINT(48.52177926949569 -121.55357369379438) bank122027 +122028 POINT(48.27917845220202 -121.90155879367376) bank122028 +122029 POINT(47.36679537683723 -122.43655181607792) bank122029 +122030 POINT(47.80428694287613 -123.07218153463012) bank122030 +122031 POINT(47.13250184689189 -122.76893710072011) bank122031 +122032 POINT(48.516268473914295 -121.67275069352029) bank122032 +122033 POINT(47.480369000481126 -122.48425357935544) bank122033 +122034 POINT(46.755005767593275 -121.3988574742995) bank122034 +122035 POINT(47.360666388685466 -121.60372684580399) bank122035 +122036 POINT(48.14036756194549 -121.59321710126825) bank122036 +122037 POINT(48.44710489291217 -122.53760522775117) bank122037 +122038 POINT(48.34739741037515 -121.43202635371397) bank122038 +122039 POINT(47.53913767985041 -121.85203813028808) bank122039 +122040 POINT(47.965024052944784 -121.73511976775475) bank122040 +122041 POINT(46.80820853747627 -121.41305808466458) bank122041 +122042 POINT(47.537745944513425 -123.09302513004805) bank122042 +122043 POINT(47.62698832776624 -121.93934795972146) bank122043 +122044 POINT(48.34013053287856 -123.09452769578087) bank122044 +122045 POINT(47.144875962511456 -121.62399420447959) bank122045 +122046 POINT(46.96879884323832 -121.53390262942406) bank122046 +122047 POINT(47.066658302773284 -122.14901487834535) bank122047 +122048 POINT(48.30526154339957 -121.51909340738736) bank122048 +122049 POINT(47.836567085965235 -121.62985274575334) bank122049 +122050 POINT(47.83717783663169 -121.765724285809) bank122050 +122051 POINT(47.397088734516295 -123.24945855127123) bank122051 +122052 POINT(48.26652446551967 -121.79014088860464) bank122052 +122053 POINT(47.004231245621355 -123.11478017039033) bank122053 +122054 POINT(47.5986570752408 -122.8081270326225) bank122054 +122055 POINT(48.25527279718081 -121.51649275058233) bank122055 +122056 POINT(48.42280624749655 -122.18635677725739) bank122056 +122057 POINT(46.86354404006406 -122.76174797886323) bank122057 +122058 POINT(47.962471529866356 -122.46999539440478) bank122058 +122059 POINT(48.46428077079247 -123.1865489771084) bank122059 +122060 POINT(48.25772039681047 -122.51330848216743) bank122060 +122061 POINT(47.64254258356351 -122.76469795524228) bank122061 +122062 POINT(46.66951400875785 -122.39864898817171) bank122062 +122063 POINT(46.882931150212144 -122.20225173579115) bank122063 +122064 POINT(47.46438617609263 -121.99415705300845) bank122064 +122065 POINT(47.97610627529103 -121.66767473691856) bank122065 +122066 POINT(47.21308880267202 -121.36522462648566) bank122066 +122067 POINT(48.5708632903638 -123.22342366439024) bank122067 +122068 POINT(46.83365985044678 -123.18561688464705) bank122068 +122069 POINT(46.85570632940046 -122.18635391820747) bank122069 +122070 POINT(48.10202596114911 -121.95992431097086) bank122070 +122071 POINT(47.75284916648394 -122.04062255223126) bank122071 +122072 POINT(47.639622496583044 -121.51186923543291) bank122072 +122073 POINT(47.615827409909755 -121.45397343706554) bank122073 +122074 POINT(47.80312178091 -121.5309344178168) bank122074 +122075 POINT(47.79372253300542 -121.72300710930467) bank122075 +122076 POINT(48.430497967312405 -123.15906727879961) bank122076 +122077 POINT(47.599242199277526 -121.53620042210882) bank122077 +122078 POINT(46.9875220954132 -121.35463357869436) bank122078 +122079 POINT(47.280585123859034 -122.49649629925062) bank122079 +122080 POINT(48.48509784763192 -122.20065898247428) bank122080 +122081 POINT(47.675125669708855 -122.20793108483363) bank122081 +122082 POINT(47.421911894108995 -121.42576961988931) bank122082 +122083 POINT(47.28277378109922 -121.48223161482116) bank122083 +122084 POINT(48.14490534800143 -121.52329617787426) bank122084 +122085 POINT(47.666932797338944 -122.35696958260519) bank122085 +122086 POINT(48.02187130046063 -121.98893245270325) bank122086 +122087 POINT(48.49703835441203 -121.39138865567026) bank122087 +122088 POINT(47.82487928946139 -121.82614823563986) bank122088 +122089 POINT(47.27966629737052 -121.74858744714123) bank122089 +122090 POINT(48.46550548002796 -122.35360886679409) bank122090 +122091 POINT(48.46518946212018 -122.27489499425403) bank122091 +122092 POINT(48.272995908776416 -122.26932916872205) bank122092 +122093 POINT(47.425683631886976 -122.91573959132329) bank122093 +122094 POINT(48.320401324365854 -121.34573699134673) bank122094 +122095 POINT(47.20153423315055 -122.33746640823904) bank122095 +122096 POINT(48.46037777462777 -121.60903059759633) bank122096 +122097 POINT(47.14850679923028 -121.97945065844006) bank122097 +122098 POINT(47.719866435506844 -122.697844948795) bank122098 +122099 POINT(48.25019856121721 -122.94112204820982) bank122099 +122100 POINT(47.52281367361407 -122.33862704999875) bank122100 +122101 POINT(46.61655381884766 -123.19896695497422) bank122101 +122102 POINT(47.27519079021717 -122.88734807911479) bank122102 +122103 POINT(47.47717197047298 -121.86716911384335) bank122103 +122104 POINT(48.17378882576814 -121.45410652239939) bank122104 +122105 POINT(47.568599907274034 -123.10862465481924) bank122105 +122106 POINT(48.552821653822974 -121.89954267442204) bank122106 +122107 POINT(48.593199660852044 -121.96152827235692) bank122107 +122108 POINT(46.65468958405437 -122.89322624863675) bank122108 +122109 POINT(47.042347197971935 -121.3364033087208) bank122109 +122110 POINT(47.740885173472876 -122.95186863032883) bank122110 +122111 POINT(46.72498427815355 -121.44507896065969) bank122111 +122112 POINT(47.99686805763854 -122.60427333998024) bank122112 +122113 POINT(46.870823907917945 -122.25958644719528) bank122113 +122114 POINT(48.2467671807042 -122.08499695039126) bank122114 +122115 POINT(47.72787625698302 -122.72784660626621) bank122115 +122116 POINT(46.83505686496796 -122.05057297308545) bank122116 +122117 POINT(47.150100311017745 -121.61302817997087) bank122117 +122118 POINT(48.58142347207493 -122.49173770287666) bank122118 +122119 POINT(47.19968015726251 -122.42456365648839) bank122119 +122120 POINT(47.20912810075853 -122.21402194673553) bank122120 +122121 POINT(46.93647127977189 -121.91203283476378) bank122121 +122122 POINT(47.53468106534798 -122.44911766504288) bank122122 +122123 POINT(48.42063425837477 -122.39540552644353) bank122123 +122124 POINT(47.08782584329933 -122.50932210226019) bank122124 +122125 POINT(47.80124791922001 -121.75999120053915) bank122125 +122126 POINT(47.84694381096423 -121.59032945055996) bank122126 +122127 POINT(47.480406272131646 -122.03491026847458) bank122127 +122128 POINT(48.25074639710814 -121.78584777646428) bank122128 +122129 POINT(47.13503500171463 -121.5318542786695) bank122129 +122130 POINT(47.49819456555281 -122.25170035997282) bank122130 +122131 POINT(47.7492981604274 -122.7020382068495) bank122131 +122132 POINT(47.01591127456672 -121.72443384449852) bank122132 +122133 POINT(48.15886499850197 -122.55342891142513) bank122133 +122134 POINT(48.40808190463168 -121.83104137848566) bank122134 +122135 POINT(48.60091052634973 -122.2561181251265) bank122135 +122136 POINT(47.54128529480039 -122.91359726001784) bank122136 +122137 POINT(47.34795218808345 -121.5679297687384) bank122137 +122138 POINT(46.997613851271396 -121.81763351454764) bank122138 +122139 POINT(46.67670602701679 -123.27088525861524) bank122139 +122140 POINT(46.91179921409236 -122.11325025372585) bank122140 +122141 POINT(48.16143051103887 -123.21028707670845) bank122141 +122142 POINT(48.41986296682159 -122.37915119153581) bank122142 +122143 POINT(47.326625562603816 -123.31770717096022) bank122143 +122144 POINT(46.80558948745411 -122.44170935272554) bank122144 +122145 POINT(46.70271124235748 -121.40272623831939) bank122145 +122146 POINT(48.33429023476828 -122.03962777559062) bank122146 +122147 POINT(47.39993795096982 -121.59734603941561) bank122147 +122148 POINT(48.53037284842695 -122.22095993648848) bank122148 +122149 POINT(47.437658822920646 -121.37017885650582) bank122149 +122150 POINT(47.82222634262156 -122.10251450253963) bank122150 +122151 POINT(48.20035838632398 -122.41434120049709) bank122151 +122152 POINT(47.690821089983245 -123.32611493282016) bank122152 +122153 POINT(47.18936937743464 -121.82891331428667) bank122153 +122154 POINT(48.14879276712088 -122.49547380103981) bank122154 +122155 POINT(47.70261557738041 -121.72952159445157) bank122155 +122156 POINT(48.28852294550083 -122.09053491794666) bank122156 +122157 POINT(47.508665512955915 -123.24530599590281) bank122157 +122158 POINT(46.89517699382453 -122.44359733470706) bank122158 +122159 POINT(47.71834916560679 -121.8929073334121) bank122159 +122160 POINT(47.376724831104134 -123.16585578909827) bank122160 +122161 POINT(46.84113950391257 -121.78266621344552) bank122161 +122162 POINT(47.72460267747671 -123.16563019005957) bank122162 +122163 POINT(47.87928096367326 -122.48129130683866) bank122163 +122164 POINT(48.373089883589245 -122.42153029272548) bank122164 +122165 POINT(46.903584649111224 -122.89280286011247) bank122165 +122166 POINT(46.75515668530198 -123.00353284074828) bank122166 +122167 POINT(47.61136616230331 -122.54761238254041) bank122167 +122168 POINT(47.31942170335326 -122.81463507495978) bank122168 +122169 POINT(47.23304761586364 -121.6795181538174) bank122169 +122170 POINT(47.00072901877191 -122.06166521740975) bank122170 +122171 POINT(47.00671842815568 -122.47531104007095) bank122171 +122172 POINT(48.44054828749538 -121.54968508453824) bank122172 +122173 POINT(48.42871005465484 -122.00964562522877) bank122173 +122174 POINT(46.941777167615925 -121.33813550661381) bank122174 +122175 POINT(47.780483119994315 -123.26321926425618) bank122175 +122176 POINT(48.5855351717912 -121.72849549444354) bank122176 +122177 POINT(46.90127253036488 -122.28050150609592) bank122177 +122178 POINT(47.06783156153945 -121.99630483372717) bank122178 +122179 POINT(48.5853801257573 -121.39674704256241) bank122179 +122180 POINT(46.930122053134205 -122.37392828180957) bank122180 +122181 POINT(47.52790390369116 -122.54956617750477) bank122181 +122182 POINT(47.660319776471475 -122.34382643151365) bank122182 +122183 POINT(47.53750008375352 -121.46741669588441) bank122183 +122184 POINT(48.0114860480491 -121.99515364703772) bank122184 +122185 POINT(48.46186773377649 -121.84086775698175) bank122185 +122186 POINT(48.561073037701895 -122.015134130199) bank122186 +122187 POINT(47.00746546138019 -123.24158755083997) bank122187 +122188 POINT(47.460524402146916 -122.54244939771405) bank122188 +122189 POINT(46.62726846883676 -123.14830388122247) bank122189 +122190 POINT(47.459860111394775 -123.10996267407005) bank122190 +122191 POINT(46.74509539949142 -122.39555802042825) bank122191 +122192 POINT(47.1070974269343 -122.79289583407659) bank122192 +122193 POINT(48.28681566804576 -123.27596501929125) bank122193 +122194 POINT(47.78506953563053 -121.49698814233864) bank122194 +122195 POINT(46.80945129064396 -122.83000279597623) bank122195 +122196 POINT(48.00789943366032 -123.12736293463018) bank122196 +122197 POINT(47.61728350189255 -121.98085006906223) bank122197 +122198 POINT(48.231151439368816 -122.26293361340696) bank122198 +122199 POINT(47.46741529689785 -121.46957248308493) bank122199 +122200 POINT(47.949168879312985 -122.40952542699364) bank122200 +122201 POINT(47.25766658477204 -122.70154162510569) bank122201 +122202 POINT(48.06054366446859 -121.5029194922153) bank122202 +122203 POINT(48.458850984449654 -122.57114452991927) bank122203 +122204 POINT(48.59952152796009 -122.50170400940547) bank122204 +122205 POINT(48.51358168531527 -122.50804556714307) bank122205 +122206 POINT(47.10631943823639 -122.90513083841084) bank122206 +122207 POINT(48.1363789268274 -123.15855281639148) bank122207 +122208 POINT(48.237402036996926 -122.33873157355292) bank122208 +122209 POINT(47.026218353583495 -122.25117366289938) bank122209 +122210 POINT(47.90274785663929 -122.87429421715954) bank122210 +122211 POINT(47.49554407728699 -121.84303219589704) bank122211 +122212 POINT(48.169467138764475 -121.70921182846192) bank122212 +122213 POINT(47.50895487408789 -122.59390214544901) bank122213 +122214 POINT(48.190284932685366 -122.7757795708932) bank122214 +122215 POINT(46.9329435577886 -122.47982879298175) bank122215 +122216 POINT(47.10096604245384 -123.1635174647417) bank122216 +122217 POINT(48.048169289382706 -121.95144157299852) bank122217 +122218 POINT(47.798399232871404 -122.54263489903884) bank122218 +122219 POINT(48.49198688364296 -122.36895818954343) bank122219 +122220 POINT(47.240725201189164 -121.97167501970952) bank122220 +122221 POINT(47.37264582695402 -122.9264045183545) bank122221 +122222 POINT(47.43757963423123 -122.9533534816864) bank122222 +122223 POINT(47.65847714340294 -121.61775304667701) bank122223 +122224 POINT(47.621917380266446 -122.34236231473466) bank122224 +122225 POINT(47.32765130953885 -121.70132196806225) bank122225 +122226 POINT(47.29999093264192 -122.59236275158975) bank122226 +122227 POINT(46.83510946922202 -122.48680102840004) bank122227 +122228 POINT(47.27469540757656 -123.30026538481208) bank122228 +122229 POINT(46.80652525713815 -123.01386030517341) bank122229 +122230 POINT(48.27344262958477 -121.5534572042499) bank122230 +122231 POINT(47.155703897385266 -122.44469195512679) bank122231 +122232 POINT(48.156967706731265 -121.585625766688) bank122232 +122233 POINT(48.38072899546485 -122.6867255814683) bank122233 +122234 POINT(48.40752983570062 -123.09525426939257) bank122234 +122235 POINT(48.2066660709072 -122.50782431809402) bank122235 +122236 POINT(47.862186582361765 -122.78805122866689) bank122236 +122237 POINT(48.36000997353127 -122.57521012312947) bank122237 +122238 POINT(46.638523629267866 -121.87061105251605) bank122238 +122239 POINT(47.84168677946111 -121.44992355235524) bank122239 +122240 POINT(47.89762988041937 -122.90587799452916) bank122240 +122241 POINT(48.151930497974604 -122.50748199688354) bank122241 +122242 POINT(47.70662737487407 -121.89437489141552) bank122242 +122243 POINT(47.166221436675066 -122.64857749138673) bank122243 +122244 POINT(47.86480400989223 -121.78906574593309) bank122244 +122245 POINT(46.80804717137186 -121.43855359505314) bank122245 +122246 POINT(46.63105538446817 -122.8905861118205) bank122246 +122247 POINT(47.26649204308726 -121.92992796581105) bank122247 +122248 POINT(47.11097534258396 -122.32282597408964) bank122248 +122249 POINT(47.86949620545687 -122.39286810538) bank122249 +122250 POINT(46.87732086181312 -121.92463954152545) bank122250 +122251 POINT(48.314949679741204 -122.69874122010226) bank122251 +122252 POINT(46.98901548367314 -122.03972791264789) bank122252 +122253 POINT(48.402731963384014 -122.74377313451312) bank122253 +122254 POINT(47.04043461195913 -123.31337387823332) bank122254 +122255 POINT(48.45463970305121 -123.20318456027616) bank122255 +122256 POINT(47.61314621420514 -123.15912454783154) bank122256 +122257 POINT(47.24803681927454 -123.14192394318702) bank122257 +122258 POINT(47.33091689635461 -121.37699391432443) bank122258 +122259 POINT(47.30090620423814 -121.35440299741266) bank122259 +122260 POINT(48.498970157854615 -122.73120545108218) bank122260 +122261 POINT(46.98267993047496 -122.89680045585614) bank122261 +122262 POINT(48.41164569481831 -121.57974931176992) bank122262 +122263 POINT(48.52484140693638 -123.04652961066876) bank122263 +122264 POINT(47.43805854623248 -123.18075617512876) bank122264 +122265 POINT(47.20337369311444 -122.74050384435066) bank122265 +122266 POINT(47.84674391363388 -122.68527169672709) bank122266 +122267 POINT(47.540231678294596 -122.46114562749194) bank122267 +122268 POINT(47.38309766426352 -121.86263866782483) bank122268 +122269 POINT(47.83839238780218 -121.49866324128679) bank122269 +122270 POINT(48.16048871210786 -122.32858474463143) bank122270 +122271 POINT(48.56778627506518 -122.87425710643015) bank122271 +122272 POINT(47.741663959445034 -121.44677249133568) bank122272 +122273 POINT(48.15908951615638 -122.51309577095209) bank122273 +122274 POINT(48.03498794997534 -122.78732257368218) bank122274 +122275 POINT(46.740522373088396 -121.87560791300933) bank122275 +122276 POINT(48.3173277530255 -122.39373054115885) bank122276 +122277 POINT(48.551924622277895 -122.99719804371776) bank122277 +122278 POINT(47.280386130254435 -122.0893349082289) bank122278 +122279 POINT(46.67262501832987 -122.83297934956165) bank122279 +122280 POINT(47.386023704804884 -121.88989514720402) bank122280 +122281 POINT(47.88701866399449 -121.37313858255962) bank122281 +122282 POINT(47.36321485112381 -122.25621914334707) bank122282 +122283 POINT(48.11817316330239 -123.30189214890267) bank122283 +122284 POINT(47.249935595053394 -121.50334352520422) bank122284 +122285 POINT(48.10125003825179 -123.22094753708595) bank122285 +122286 POINT(48.32304234613126 -122.60413939342062) bank122286 +122287 POINT(47.83483005522728 -122.57900722882778) bank122287 +122288 POINT(48.541124881898384 -121.98830529787136) bank122288 +122289 POINT(47.98183487545553 -122.48787617975206) bank122289 +122290 POINT(47.95073393179413 -121.4961513891023) bank122290 +122291 POINT(47.102904329485746 -122.0778436638366) bank122291 +122292 POINT(47.45591377753109 -123.1980535619812) bank122292 +122293 POINT(48.0486291531158 -123.02294758188353) bank122293 +122294 POINT(46.65246019835393 -122.44746641113701) bank122294 +122295 POINT(47.04168528867283 -121.61067334272157) bank122295 +122296 POINT(46.73380995593637 -122.21519097279683) bank122296 +122297 POINT(48.42197419622792 -121.69542457601929) bank122297 +122298 POINT(47.98492448923292 -122.2881948210945) bank122298 +122299 POINT(47.11415867142122 -121.58729347210571) bank122299 +122300 POINT(46.91244029998573 -121.91048023960258) bank122300 +122301 POINT(46.69103110935398 -122.34444242477467) bank122301 +122302 POINT(48.25783439554184 -121.69365755472296) bank122302 +122303 POINT(47.1961217603464 -121.36620541458696) bank122303 +122304 POINT(47.618016567447846 -122.45486862553878) bank122304 +122305 POINT(47.36659262809272 -121.65650974415131) bank122305 +122306 POINT(47.6272676652737 -122.79767654876744) bank122306 +122307 POINT(46.81910423620268 -121.62227524995957) bank122307 +122308 POINT(48.552259826003436 -123.01502970597535) bank122308 +122309 POINT(48.06613705067079 -122.86145303349556) bank122309 +122310 POINT(47.40862273930925 -122.20505391677435) bank122310 +122311 POINT(48.57696615062326 -122.67401005264311) bank122311 +122312 POINT(47.94871519933331 -123.2176528061854) bank122312 +122313 POINT(47.664169347742764 -122.16963369105412) bank122313 +122314 POINT(48.528909787622034 -121.89607195134654) bank122314 +122315 POINT(47.68537045159073 -122.2375708982109) bank122315 +122316 POINT(48.482573355785824 -122.1776462793569) bank122316 +122317 POINT(48.29227659265363 -121.72091328168976) bank122317 +122318 POINT(47.995051055428725 -123.3310037543797) bank122318 +122319 POINT(46.97257324987173 -121.41011194592059) bank122319 +122320 POINT(47.58468579985454 -122.34072903002895) bank122320 +122321 POINT(47.865139003850224 -122.7289666980741) bank122321 +122322 POINT(46.98749951362568 -121.75603293001933) bank122322 +122323 POINT(47.5691827173895 -123.21638097850042) bank122323 +122324 POINT(47.66575208085664 -121.67481673746478) bank122324 +122325 POINT(47.55451997116098 -122.62546706256255) bank122325 +122326 POINT(47.89180686551843 -122.56543345493571) bank122326 +122327 POINT(47.60196984858342 -121.52442191220646) bank122327 +122328 POINT(46.93654265162478 -121.37189217608885) bank122328 +122329 POINT(47.649729385068056 -122.43038201769562) bank122329 +122330 POINT(48.46829997925097 -122.37302170935502) bank122330 +122331 POINT(47.76671325909364 -122.50797262132325) bank122331 +122332 POINT(46.69568513101949 -121.64893151526348) bank122332 +122333 POINT(46.770486383976966 -121.79261458935538) bank122333 +122334 POINT(46.94763427939114 -121.48441564703191) bank122334 +122335 POINT(46.90945339049705 -123.15405162694937) bank122335 +122336 POINT(46.97140537545804 -121.49692658560068) bank122336 +122337 POINT(47.9746325886206 -121.6722265201797) bank122337 +122338 POINT(47.0468189150879 -122.11905492593412) bank122338 +122339 POINT(47.36449386290108 -122.20034660421098) bank122339 +122340 POINT(47.28722444890367 -122.47101756350713) bank122340 +122341 POINT(47.89577302152472 -122.95896524922829) bank122341 +122342 POINT(47.88950555881252 -121.85436857374758) bank122342 +122343 POINT(46.609180746818026 -122.99774910139111) bank122343 +122344 POINT(48.02934667670234 -122.28305279535643) bank122344 +122345 POINT(47.90141615990464 -121.37074961923409) bank122345 +122346 POINT(47.45854179823114 -123.29224629255039) bank122346 +122347 POINT(47.07206540633959 -122.59823949950257) bank122347 +122348 POINT(46.81298229881225 -122.03335019661417) bank122348 +122349 POINT(46.83252820938232 -122.73455861781972) bank122349 +122350 POINT(46.694215597151974 -122.61173890551183) bank122350 +122351 POINT(47.450274873478385 -122.6553860873495) bank122351 +122352 POINT(48.519742940498524 -122.81259136111932) bank122352 +122353 POINT(48.450932324960434 -121.41816090204327) bank122353 +122354 POINT(47.470007214814835 -121.90166739841763) bank122354 +122355 POINT(47.049258724472445 -121.68821691110408) bank122355 +122356 POINT(47.033726596979044 -123.1834409172851) bank122356 +122357 POINT(47.399936229789134 -122.1853084755737) bank122357 +122358 POINT(47.92882887543354 -121.8668570789748) bank122358 +122359 POINT(48.4569908883999 -122.08912920443555) bank122359 +122360 POINT(47.75765344851171 -123.25602366777977) bank122360 +122361 POINT(47.93650186095427 -121.3945783985624) bank122361 +122362 POINT(48.224945025661256 -122.77533805902523) bank122362 +122363 POINT(47.9091028359727 -122.47651996919438) bank122363 +122364 POINT(46.876112061899896 -122.74101898045737) bank122364 +122365 POINT(47.27864639107424 -121.42987808551268) bank122365 +122366 POINT(48.00776754441596 -122.82173090608917) bank122366 +122367 POINT(47.65726610204058 -122.74714345100476) bank122367 +122368 POINT(48.41619779334875 -122.41050067065078) bank122368 +122369 POINT(48.118383366473026 -123.26720476202516) bank122369 +122370 POINT(48.383137526436144 -122.5647335420261) bank122370 +122371 POINT(47.11817810797171 -121.80588633307376) bank122371 +122372 POINT(48.17051426408399 -121.44443616085466) bank122372 +122373 POINT(47.811636961073646 -123.13823533865919) bank122373 +122374 POINT(47.526245596382154 -122.35868688137046) bank122374 +122375 POINT(47.47573269106552 -122.36083438343995) bank122375 +122376 POINT(48.04133392631315 -122.56738359758621) bank122376 +122377 POINT(47.82467868027139 -121.64941287591147) bank122377 +122378 POINT(47.78147401556186 -122.42105326394224) bank122378 +122379 POINT(46.98223638323677 -121.6898140942326) bank122379 +122380 POINT(48.1723125125241 -122.4052666141624) bank122380 +122381 POINT(46.93810992195414 -121.3579619496382) bank122381 +122382 POINT(48.13611567101163 -121.98475987378748) bank122382 +122383 POINT(48.06017214856224 -121.61678462562837) bank122383 +122384 POINT(48.56606846004644 -121.70484105095996) bank122384 +122385 POINT(47.50639538750671 -122.64445731092948) bank122385 +122386 POINT(47.012654253206975 -123.29468226024453) bank122386 +122387 POINT(47.97296717619283 -122.78733719962652) bank122387 +122388 POINT(48.234915793086934 -121.68446040468562) bank122388 +122389 POINT(47.20367827754412 -122.47696560180549) bank122389 +122390 POINT(48.035663491719994 -122.04734545166163) bank122390 +122391 POINT(47.398728380281185 -123.01959168115495) bank122391 +122392 POINT(47.18878391134926 -122.79445696405809) bank122392 +122393 POINT(47.424860200860046 -123.07116680283086) bank122393 +122394 POINT(47.86265649620445 -122.76546903071834) bank122394 +122395 POINT(46.811403075261914 -122.5241514721858) bank122395 +122396 POINT(47.76225377484128 -122.13043706117239) bank122396 +122397 POINT(48.34927912162822 -122.81689285765552) bank122397 +122398 POINT(48.35891169725198 -122.10123525742824) bank122398 +122399 POINT(47.224451867160305 -121.89470339201615) bank122399 +122400 POINT(47.61298750785364 -122.6142502542805) bank122400 +122401 POINT(48.38004205285173 -122.11393002238069) bank122401 +122402 POINT(48.50301213000237 -122.59982407669278) bank122402 +122403 POINT(48.09202158986718 -122.53475383944628) bank122403 +122404 POINT(48.35353665580038 -122.32567522408723) bank122404 +122405 POINT(47.01520598922033 -123.14249194829652) bank122405 +122406 POINT(47.46082588395231 -122.41513733081555) bank122406 +122407 POINT(46.81010499063861 -123.10895839154519) bank122407 +122408 POINT(48.52409871510083 -121.94869195169791) bank122408 +122409 POINT(46.8075746547996 -121.57080040942907) bank122409 +122410 POINT(47.680941784951784 -122.8058363273178) bank122410 +122411 POINT(47.46163113508012 -122.50238442399842) bank122411 +122412 POINT(47.7122783864993 -121.74351119484831) bank122412 +122413 POINT(46.90841037510442 -123.2774522484753) bank122413 +122414 POINT(47.76447442445442 -122.1432178443864) bank122414 +122415 POINT(48.44878628946369 -123.28796199450022) bank122415 +122416 POINT(48.01767461417224 -123.16098836745044) bank122416 +122417 POINT(46.809603758593575 -122.98136154197762) bank122417 +122418 POINT(47.92572461537126 -123.03213969937774) bank122418 +122419 POINT(46.93591949795102 -122.7656532723756) bank122419 +122420 POINT(47.68056397834972 -123.02837840426008) bank122420 +122421 POINT(47.43343539053972 -122.6747428165767) bank122421 +122422 POINT(46.78078636936266 -122.44314231065965) bank122422 +122423 POINT(46.91651304079371 -121.3990709496623) bank122423 +122424 POINT(48.1494688027239 -122.75232198870388) bank122424 +122425 POINT(47.12552068045679 -123.1569069286337) bank122425 +122426 POINT(46.76041130296608 -121.67050584268205) bank122426 +122427 POINT(46.73647559947599 -123.07455457384383) bank122427 +122428 POINT(46.75448152027358 -122.82374400008753) bank122428 +122429 POINT(46.797499699915555 -123.24697210372045) bank122429 +122430 POINT(46.82326312819757 -121.98011408158446) bank122430 +122431 POINT(46.972531314398324 -122.76769947405796) bank122431 +122432 POINT(47.97789824673924 -123.04105014464464) bank122432 +122433 POINT(47.995422527616014 -122.08113139120739) bank122433 +122434 POINT(48.14728869780181 -123.09150342603581) bank122434 +122435 POINT(47.530000879875516 -121.49400394072464) bank122435 +122436 POINT(48.077929696741236 -121.46470683452907) bank122436 +122437 POINT(47.46980957031632 -123.06085465055045) bank122437 +122438 POINT(46.85098119954224 -122.67178506072408) bank122438 +122439 POINT(48.19239041305404 -122.53475364253335) bank122439 +122440 POINT(46.92649882590529 -122.27043971804619) bank122440 +122441 POINT(48.21463476060166 -121.80752706266372) bank122441 +122442 POINT(48.16732358427688 -121.35232182585632) bank122442 +122443 POINT(47.42414031029823 -121.67809956243379) bank122443 +122444 POINT(47.9796483306655 -122.11826212842828) bank122444 +122445 POINT(48.34084910041103 -122.01925458757455) bank122445 +122446 POINT(46.95406998505358 -122.28737401066573) bank122446 +122447 POINT(47.06016557124436 -122.95253272663318) bank122447 +122448 POINT(47.933971651404185 -122.38346386378716) bank122448 +122449 POINT(47.80588140936791 -121.40459133168727) bank122449 +122450 POINT(47.79809422105237 -121.35125593124515) bank122450 +122451 POINT(46.90395196732119 -121.62316288429767) bank122451 +122452 POINT(48.107904250512796 -121.4227053359983) bank122452 +122453 POINT(48.13910468011767 -121.66818277821652) bank122453 +122454 POINT(48.03781185797678 -121.44065018197968) bank122454 +122455 POINT(47.380159309068674 -121.78396642937929) bank122455 +122456 POINT(48.593048470379536 -123.19799245777197) bank122456 +122457 POINT(47.59146684921486 -122.41273506075328) bank122457 +122458 POINT(46.988019229951284 -122.67687883556475) bank122458 +122459 POINT(47.63141815986946 -122.88283231663661) bank122459 +122460 POINT(48.1716978746734 -122.36543474337222) bank122460 +122461 POINT(47.40229391398947 -122.99065624598741) bank122461 +122462 POINT(46.730638717228146 -121.77604206947287) bank122462 +122463 POINT(48.33200145825382 -122.08688812494795) bank122463 +122464 POINT(47.848324727645625 -123.1567101309877) bank122464 +122465 POINT(47.75637451173803 -121.66379227553995) bank122465 +122466 POINT(47.01183668013455 -121.5026101409112) bank122466 +122467 POINT(48.2934675971613 -122.41998656394327) bank122467 +122468 POINT(47.53962236420101 -121.51997705992044) bank122468 +122469 POINT(47.46144735522776 -122.06620634178782) bank122469 +122470 POINT(47.767881426647364 -122.2457434958311) bank122470 +122471 POINT(48.267272161128815 -123.1746160789217) bank122471 +122472 POINT(47.094594129226586 -122.84626640533274) bank122472 +122473 POINT(47.06870045901897 -122.87528882728412) bank122473 +122474 POINT(47.1593990519343 -122.34716294328746) bank122474 +122475 POINT(46.87799869789357 -123.14470058657261) bank122475 +122476 POINT(48.44514979712468 -122.71178868423128) bank122476 +122477 POINT(47.039057261348646 -123.06228086742178) bank122477 +122478 POINT(47.57979013452369 -121.44766181243448) bank122478 +122479 POINT(47.33045040957991 -122.37344767751293) bank122479 +122480 POINT(47.39324334310538 -122.68140229093642) bank122480 +122481 POINT(46.801051460673456 -122.97367746866904) bank122481 +122482 POINT(47.13570570531717 -122.76232740273633) bank122482 +122483 POINT(47.64060470280374 -121.91185654421334) bank122483 +122484 POINT(47.93777312569794 -122.68998159837338) bank122484 +122485 POINT(48.527173539245425 -122.58454733271634) bank122485 +122486 POINT(47.881718131036415 -123.2236022772844) bank122486 +122487 POINT(48.24403205005593 -122.98239965116417) bank122487 +122488 POINT(47.0963562654888 -121.61048897133121) bank122488 +122489 POINT(47.61041616395808 -123.10279892666465) bank122489 +122490 POINT(48.12259584728426 -121.96637843223894) bank122490 +122491 POINT(48.39397238582928 -121.77375827344794) bank122491 +122492 POINT(47.360584726135016 -122.66122463894064) bank122492 +122493 POINT(47.94881038826414 -123.16588967244164) bank122493 +122494 POINT(47.50337337630962 -123.2266268772296) bank122494 +122495 POINT(47.77388487761736 -123.06633984011727) bank122495 +122496 POINT(47.40358982939116 -122.11680883933246) bank122496 +122497 POINT(47.462385622954194 -122.9036155252254) bank122497 +122498 POINT(47.50552979435324 -123.08078342166881) bank122498 +122499 POINT(48.06537167164124 -122.22834403105368) bank122499 +122500 POINT(47.63906679011038 -122.58926041672268) bank122500 +122501 POINT(47.56368067819176 -123.10695467647054) bank122501 +122502 POINT(46.94670165152605 -123.29674755016983) bank122502 +122503 POINT(46.879308074519216 -122.8473051870657) bank122503 +122504 POINT(47.22729621774709 -122.2440111810639) bank122504 +122505 POINT(47.195358347537166 -121.6958948841783) bank122505 +122506 POINT(46.811858051515614 -122.47923216758134) bank122506 +122507 POINT(46.61053288981641 -122.17750169983576) bank122507 +122508 POINT(48.33410797637862 -122.35426175729808) bank122508 +122509 POINT(47.49189402381911 -122.44157693578119) bank122509 +122510 POINT(48.40895847659561 -122.89338410784896) bank122510 +122511 POINT(47.86245591908814 -122.95079329413629) bank122511 +122512 POINT(47.76365015619783 -121.5586972356811) bank122512 +122513 POINT(48.060439521456274 -122.86631238818545) bank122513 +122514 POINT(47.15991799039643 -121.49286940853827) bank122514 +122515 POINT(47.18520052156177 -121.89418324191459) bank122515 +122516 POINT(47.16738652511777 -122.30634655635593) bank122516 +122517 POINT(46.863603062191636 -122.1297225003636) bank122517 +122518 POINT(46.90840338026417 -121.47347996623837) bank122518 +122519 POINT(46.831959327504606 -121.83680700579991) bank122519 +122520 POINT(48.16199525603693 -121.91227334066093) bank122520 +122521 POINT(46.66087500066186 -122.4197841635974) bank122521 +122522 POINT(46.72588936314587 -122.76791001337004) bank122522 +122523 POINT(46.7309422245572 -122.97383712089548) bank122523 +122524 POINT(46.913527063895785 -121.83302345041582) bank122524 +122525 POINT(46.94281685521211 -121.50895878801961) bank122525 +122526 POINT(46.98734106583216 -122.29109873334168) bank122526 +122527 POINT(48.5708718137074 -123.1506806386252) bank122527 +122528 POINT(47.67783574979799 -122.03051477903976) bank122528 +122529 POINT(46.61165011888867 -123.16996912979116) bank122529 +122530 POINT(47.53738672928665 -121.55548715664045) bank122530 +122531 POINT(47.298132484310905 -121.7498412008797) bank122531 +122532 POINT(46.91990672637841 -122.45145239841341) bank122532 +122533 POINT(47.11504201756278 -122.58319090921167) bank122533 +122534 POINT(48.51986957437573 -122.38240781742216) bank122534 +122535 POINT(48.57096235900961 -122.00475598414877) bank122535 +122536 POINT(46.635182443163856 -123.11590128043366) bank122536 +122537 POINT(48.28048500980852 -121.49355958682565) bank122537 +122538 POINT(47.549476916512646 -122.46639593477875) bank122538 +122539 POINT(47.042383454734875 -122.52292001459631) bank122539 +122540 POINT(46.91499037926027 -122.85574236776061) bank122540 +122541 POINT(48.1228640330359 -121.68965812124375) bank122541 +122542 POINT(47.310170532232384 -122.40099959309563) bank122542 +122543 POINT(47.26441987939034 -122.2274656220127) bank122543 +122544 POINT(47.69438395865139 -123.16478552603671) bank122544 +122545 POINT(46.62517833871718 -121.66452088032197) bank122545 +122546 POINT(47.07752548939194 -122.23541018776336) bank122546 +122547 POINT(46.631956786368754 -121.33747123649422) bank122547 +122548 POINT(47.99739468787464 -122.50316129797027) bank122548 +122549 POINT(46.8976785784076 -122.72844538429887) bank122549 +122550 POINT(48.50999213908738 -121.91316252525965) bank122550 +122551 POINT(46.837544164552774 -122.45421545343189) bank122551 +122552 POINT(47.86594481078241 -123.0827494021683) bank122552 +122553 POINT(47.58604185198209 -123.14550926554561) bank122553 +122554 POINT(47.28456884368848 -121.84186293737882) bank122554 +122555 POINT(48.21740766103731 -122.62641483606079) bank122555 +122556 POINT(47.166341829230234 -122.31615802264464) bank122556 +122557 POINT(48.49236103301748 -122.0275456987315) bank122557 +122558 POINT(48.40473736402036 -122.57045452730175) bank122558 +122559 POINT(47.89150928747963 -122.25298473502534) bank122559 +122560 POINT(48.1872919699119 -121.78162016220314) bank122560 +122561 POINT(48.133855148870374 -122.1586160982759) bank122561 +122562 POINT(47.01358989587908 -123.08402216245173) bank122562 +122563 POINT(46.80837206245164 -121.37512336895664) bank122563 +122564 POINT(46.81187891066128 -122.58254813961976) bank122564 +122565 POINT(48.411260982074474 -122.00325816181551) bank122565 +122566 POINT(47.31885526018137 -121.5167059939251) bank122566 +122567 POINT(46.97175817261661 -122.71755926876229) bank122567 +122568 POINT(48.06351610180563 -121.79708604625519) bank122568 +122569 POINT(48.59634365498422 -123.01506445323832) bank122569 +122570 POINT(47.49740778288688 -121.91200689763048) bank122570 +122571 POINT(47.425843949634306 -123.29584563361516) bank122571 +122572 POINT(48.59549364032226 -122.57895679524836) bank122572 +122573 POINT(47.11517049419981 -122.79648707863932) bank122573 +122574 POINT(47.570139786285 -123.18175650514773) bank122574 +122575 POINT(47.49937659756155 -122.60072227184364) bank122575 +122576 POINT(47.896525910549435 -123.16836761925063) bank122576 +122577 POINT(47.67280646356611 -123.20479473571073) bank122577 +122578 POINT(47.082889004080485 -121.840098384236) bank122578 +122579 POINT(47.21659514215499 -122.67327943059951) bank122579 +122580 POINT(48.373288955048785 -123.12293639084143) bank122580 +122581 POINT(48.20422407620427 -121.39323537347504) bank122581 +122582 POINT(48.46010132039144 -122.11099648747106) bank122582 +122583 POINT(47.23681533514396 -122.24409987193053) bank122583 +122584 POINT(46.78394275753301 -122.22225068018373) bank122584 +122585 POINT(48.14291417125955 -121.81233554906119) bank122585 +122586 POINT(48.25416202346664 -121.57002194720619) bank122586 +122587 POINT(46.642274715915896 -122.36964762618939) bank122587 +122588 POINT(47.059614306028266 -121.53438490241761) bank122588 +122589 POINT(46.64346872111103 -122.42127464563868) bank122589 +122590 POINT(47.012196794413754 -122.87439831820534) bank122590 +122591 POINT(48.57833872267238 -122.27941322147517) bank122591 +122592 POINT(48.05622161744474 -121.74383097383608) bank122592 +122593 POINT(48.01883486123178 -123.27800235272392) bank122593 +122594 POINT(48.482534619249705 -122.8163072850747) bank122594 +122595 POINT(46.921841730291554 -123.29766950908471) bank122595 +122596 POINT(47.65929424326444 -122.62110574044098) bank122596 +122597 POINT(48.288002967047355 -122.63584694099538) bank122597 +122598 POINT(48.564601103753404 -122.80458883308734) bank122598 +122599 POINT(46.68621158808878 -121.35427035174914) bank122599 +122600 POINT(46.81115073166427 -121.76521842404335) bank122600 +122601 POINT(47.34127736860583 -122.90829201493229) bank122601 +122602 POINT(47.152096547668144 -121.7769174935194) bank122602 +122603 POINT(47.03952478437608 -121.50542200852604) bank122603 +122604 POINT(47.36853105867631 -123.31348589151457) bank122604 +122605 POINT(47.63946846648443 -121.96521020423832) bank122605 +122606 POINT(46.614090203026315 -122.98204937486759) bank122606 +122607 POINT(48.41289247610009 -123.19257632345243) bank122607 +122608 POINT(47.0575199683424 -121.8635363378891) bank122608 +122609 POINT(47.61249190108675 -122.91189280853835) bank122609 +122610 POINT(48.548034875691926 -122.17774579396102) bank122610 +122611 POINT(47.065312664760484 -122.90103626948003) bank122611 +122612 POINT(47.59849624925084 -121.90769024045898) bank122612 +122613 POINT(47.70183626756307 -121.38071896274298) bank122613 +122614 POINT(47.282319906769885 -122.78760323698015) bank122614 +122615 POINT(46.77300332260701 -121.69043176868243) bank122615 +122616 POINT(47.76525208347773 -122.0977588009895) bank122616 +122617 POINT(48.22440347708809 -122.94151837952944) bank122617 +122618 POINT(48.31243878540429 -122.97657508150006) bank122618 +122619 POINT(46.84009225868416 -123.14684870878042) bank122619 +122620 POINT(47.06418261501495 -122.8678699281152) bank122620 +122621 POINT(46.65384806991561 -123.02440279616485) bank122621 +122622 POINT(48.51244053553677 -122.83002838101623) bank122622 +122623 POINT(48.06254712777845 -122.98715168824118) bank122623 +122624 POINT(47.09879990145104 -122.11461466360903) bank122624 +122625 POINT(47.422296386408 -122.83577816954957) bank122625 +122626 POINT(48.10393137139897 -122.00967101621437) bank122626 +122627 POINT(46.801469608794626 -122.9388685905231) bank122627 +122628 POINT(48.15398692570404 -122.5669641428241) bank122628 +122629 POINT(47.70486291752926 -122.43920542973481) bank122629 +122630 POINT(47.98613944500969 -122.15529862142246) bank122630 +122631 POINT(47.506654383333554 -121.98726051911868) bank122631 +122632 POINT(46.67547187431223 -121.6321891695568) bank122632 +122633 POINT(48.0016395775001 -122.26936143870681) bank122633 +122634 POINT(46.96195673253917 -123.20717278261786) bank122634 +122635 POINT(47.591352710399995 -122.74841433071914) bank122635 +122636 POINT(47.34423914051928 -121.89958060939853) bank122636 +122637 POINT(48.31979610700783 -122.59497514423781) bank122637 +122638 POINT(47.36403729575885 -122.74876838964623) bank122638 +122639 POINT(47.08711771479732 -122.09445280253543) bank122639 +122640 POINT(48.11792082639499 -122.21145530010017) bank122640 +122641 POINT(48.07779133131891 -122.39548004144793) bank122641 +122642 POINT(46.969341656696976 -122.38260880820361) bank122642 +122643 POINT(47.046786264771704 -122.69566613886556) bank122643 +122644 POINT(47.786920124245945 -122.75995761572693) bank122644 +122645 POINT(46.903591839773135 -122.35206608269125) bank122645 +122646 POINT(47.377116561992324 -121.93105797041704) bank122646 +122647 POINT(46.82197465591264 -121.63123930310212) bank122647 +122648 POINT(47.07954094958874 -122.9705261867402) bank122648 +122649 POINT(47.51703358552004 -121.96220364863358) bank122649 +122650 POINT(47.36205767676582 -121.6335979872804) bank122650 +122651 POINT(47.96546076224818 -122.42388974216774) bank122651 +122652 POINT(46.97715049487809 -122.17561242752897) bank122652 +122653 POINT(47.54098732068134 -122.28138686126482) bank122653 +122654 POINT(48.09559364322622 -122.33168374319777) bank122654 +122655 POINT(46.65032522427693 -122.77187139407451) bank122655 +122656 POINT(46.86934800935605 -123.15644356174974) bank122656 +122657 POINT(47.07089618837919 -122.52251441436317) bank122657 +122658 POINT(48.15416143326182 -122.21196650369347) bank122658 +122659 POINT(47.987055865075135 -123.22083091688113) bank122659 +122660 POINT(47.28725160609008 -121.52737984818405) bank122660 +122661 POINT(47.998393587763076 -123.2148566555009) bank122661 +122662 POINT(46.75205919161153 -121.71670543365856) bank122662 +122663 POINT(48.3677155065872 -122.75153601555456) bank122663 +122664 POINT(47.830175168678196 -121.67891639820239) bank122664 +122665 POINT(47.40149558920208 -122.46924124198985) bank122665 +122666 POINT(47.506617908188076 -122.11280494387006) bank122666 +122667 POINT(48.12127304821948 -123.09614276102188) bank122667 +122668 POINT(47.64496838770467 -121.67518552648538) bank122668 +122669 POINT(47.55531081545681 -121.37052415087666) bank122669 +122670 POINT(47.974704712430416 -122.39138167499853) bank122670 +122671 POINT(46.87669947331257 -122.45226428303569) bank122671 +122672 POINT(48.534502384690725 -121.383510980272) bank122672 +122673 POINT(47.435962171420336 -121.51723753087198) bank122673 +122674 POINT(47.977576022838505 -122.24615307486538) bank122674 +122675 POINT(47.92130967557676 -121.59263790835892) bank122675 +122676 POINT(48.04137622883133 -122.51820702091297) bank122676 +122677 POINT(47.66640098011152 -121.96578366407387) bank122677 +122678 POINT(46.68502395336035 -121.89752420252557) bank122678 +122679 POINT(47.24219855166029 -121.68971294400896) bank122679 +122680 POINT(47.78294772174484 -122.04239064387966) bank122680 +122681 POINT(48.45199214326979 -122.25010806391046) bank122681 +122682 POINT(47.78303708933071 -121.55631256605461) bank122682 +122683 POINT(48.244280144878175 -122.64882353973383) bank122683 +122684 POINT(48.04270605965494 -122.79900651608388) bank122684 +122685 POINT(46.822945605514875 -123.01096257111895) bank122685 +122686 POINT(46.661696937152506 -123.14463630128061) bank122686 +122687 POINT(46.70710574135844 -123.11536796342318) bank122687 +122688 POINT(46.95382529132928 -122.27283705270762) bank122688 +122689 POINT(48.01355396636457 -121.59608024247794) bank122689 +122690 POINT(48.25698877415893 -121.4315577015368) bank122690 +122691 POINT(48.158063160386064 -122.01980437132993) bank122691 +122692 POINT(48.5325420991817 -122.30367670051814) bank122692 +122693 POINT(47.6280493686376 -122.15119287337762) bank122693 +122694 POINT(48.571990280187286 -122.77348777613194) bank122694 +122695 POINT(47.03384525451461 -121.69307667541258) bank122695 +122696 POINT(47.02298277096608 -121.93533800553074) bank122696 +122697 POINT(48.355235062337535 -122.46097678570453) bank122697 +122698 POINT(48.51490725858613 -122.438073796506) bank122698 +122699 POINT(47.2966199590058 -121.63836436795049) bank122699 +122700 POINT(47.08361063962331 -122.1406494423159) bank122700 +122701 POINT(47.968582301473106 -123.01197765809735) bank122701 +122702 POINT(48.14082057511989 -121.67232595738105) bank122702 +122703 POINT(47.385928072540814 -123.03754512858306) bank122703 +122704 POINT(46.70152571638393 -122.8972604024604) bank122704 +122705 POINT(47.66398137152896 -122.20968097061241) bank122705 +122706 POINT(46.838846091706706 -121.72367304123149) bank122706 +122707 POINT(47.27696909687005 -123.17376780458223) bank122707 +122708 POINT(47.015620323875346 -122.46638624216956) bank122708 +122709 POINT(46.797222268098906 -122.19542072836691) bank122709 +122710 POINT(47.72483812972235 -122.54093616742287) bank122710 +122711 POINT(48.24188219850399 -122.56595288703862) bank122711 +122712 POINT(48.518742203553906 -122.35151325596784) bank122712 +122713 POINT(47.60354628122982 -123.25859451554562) bank122713 +122714 POINT(48.57431224359749 -122.1182199040585) bank122714 +122715 POINT(48.506923097040485 -122.67223048527404) bank122715 +122716 POINT(46.853390791520106 -122.51414688543885) bank122716 +122717 POINT(46.88875237382863 -121.56897346121058) bank122717 +122718 POINT(47.63894017779031 -123.18415289619811) bank122718 +122719 POINT(47.71118228971656 -122.87503365625473) bank122719 +122720 POINT(47.76357107329117 -121.88499217138882) bank122720 +122721 POINT(47.06308609161417 -121.66330813294445) bank122721 +122722 POINT(47.59147157699921 -122.99882047639663) bank122722 +122723 POINT(47.67400790624808 -122.92825761617058) bank122723 +122724 POINT(46.64342844086609 -123.23946983895536) bank122724 +122725 POINT(48.18287734615701 -122.63477970258299) bank122725 +122726 POINT(47.44148785238938 -122.92523955837576) bank122726 +122727 POINT(48.163590955113186 -121.4664780308224) bank122727 +122728 POINT(48.12763408455949 -121.33954188910903) bank122728 +122729 POINT(48.488600523462 -123.1647903939751) bank122729 +122730 POINT(46.97968503196233 -121.98252679220168) bank122730 +122731 POINT(47.26627467731233 -122.88158968373212) bank122731 +122732 POINT(47.48130843899492 -122.11817199268778) bank122732 +122733 POINT(48.48474484914239 -121.67252712129837) bank122733 +122734 POINT(47.40628613812696 -123.323017918214) bank122734 +122735 POINT(47.802564290373574 -122.30717205670929) bank122735 +122736 POINT(47.12209132478123 -121.49470285173999) bank122736 +122737 POINT(48.20326224926021 -123.08029708673533) bank122737 +122738 POINT(46.741191877755185 -121.58096825498279) bank122738 +122739 POINT(48.48744426895663 -121.84708039224365) bank122739 +122740 POINT(47.81727115531237 -122.29378159897223) bank122740 +122741 POINT(46.953250893410306 -123.2215061440396) bank122741 +122742 POINT(48.04267602626716 -123.17250231948071) bank122742 +122743 POINT(47.44947210934691 -123.23628058650058) bank122743 +122744 POINT(47.91980363903018 -121.70168538324981) bank122744 +122745 POINT(48.29911282606794 -121.48287103414037) bank122745 +122746 POINT(48.22508460866605 -121.62048849774662) bank122746 +122747 POINT(46.7822882436241 -121.97405593584085) bank122747 +122748 POINT(48.16987648487397 -121.8098768984398) bank122748 +122749 POINT(47.731818980539735 -122.55658825375903) bank122749 +122750 POINT(47.20406721536681 -122.80568850836684) bank122750 +122751 POINT(47.366686006999075 -122.46646930117689) bank122751 +122752 POINT(48.27985912432678 -121.45062931116422) bank122752 +122753 POINT(47.467574516177876 -122.29474724809077) bank122753 +122754 POINT(47.58986443422302 -123.3150036378094) bank122754 +122755 POINT(46.95657169817459 -123.13469000107781) bank122755 +122756 POINT(47.133062689943024 -123.23204610263244) bank122756 +122757 POINT(47.163937510578144 -121.91743880377061) bank122757 +122758 POINT(46.67136640643564 -122.60114220507866) bank122758 +122759 POINT(47.328190443656 -122.67415862384395) bank122759 +122760 POINT(47.79455986097221 -122.89407156414875) bank122760 +122761 POINT(48.45329898483006 -122.24950642911661) bank122761 +122762 POINT(47.7448946158817 -122.3773540979879) bank122762 +122763 POINT(46.81590885355003 -122.05384153186749) bank122763 +122764 POINT(47.75209122258102 -121.46327865752245) bank122764 +122765 POINT(47.12105350299824 -122.66831686846164) bank122765 +122766 POINT(48.25941612173025 -122.96451487216791) bank122766 +122767 POINT(47.29138569079561 -122.08449671968832) bank122767 +122768 POINT(48.05028987378155 -123.19155279285287) bank122768 +122769 POINT(47.444851357695036 -121.70362191992456) bank122769 +122770 POINT(47.54068373325268 -123.0379665591862) bank122770 +122771 POINT(47.529782811653014 -122.83647424172518) bank122771 +122772 POINT(48.590299116141274 -122.89374214320847) bank122772 +122773 POINT(48.29413301773988 -122.27207304457488) bank122773 +122774 POINT(46.888487842317794 -122.25004098120307) bank122774 +122775 POINT(46.62745809714432 -122.18533943010183) bank122775 +122776 POINT(46.73370246701305 -121.75074909115438) bank122776 +122777 POINT(48.55261128369211 -123.07771652732362) bank122777 +122778 POINT(46.82396144610962 -122.72305502654186) bank122778 +122779 POINT(47.85875825654107 -122.5581606735954) bank122779 +122780 POINT(47.39690473989694 -122.63359884340338) bank122780 +122781 POINT(46.705990750487544 -121.4772113487261) bank122781 +122782 POINT(46.879442663194794 -122.8222843922014) bank122782 +122783 POINT(48.54926629970751 -123.22102719516897) bank122783 +122784 POINT(46.80400079280974 -122.92658493096448) bank122784 +122785 POINT(47.94362971758605 -121.33475355912556) bank122785 +122786 POINT(47.743280205086194 -121.53657527651055) bank122786 +122787 POINT(47.10710705275034 -121.73377907684726) bank122787 +122788 POINT(47.78781923694226 -122.77376264444975) bank122788 +122789 POINT(47.801380514921796 -122.27401197916862) bank122789 +122790 POINT(46.82743477470449 -122.1259491248966) bank122790 +122791 POINT(47.05889092829373 -121.57546378220016) bank122791 +122792 POINT(48.05816153764747 -122.74714483429798) bank122792 +122793 POINT(46.64358048356212 -122.84046462668395) bank122793 +122794 POINT(47.908763530433035 -123.18211770904925) bank122794 +122795 POINT(48.549798383236684 -121.56590562630905) bank122795 +122796 POINT(46.730729027617485 -122.85042783428257) bank122796 +122797 POINT(47.3347286883569 -122.35973987862619) bank122797 +122798 POINT(46.65456129825685 -122.4962201580566) bank122798 +122799 POINT(48.23701730800009 -121.91289873315347) bank122799 +122800 POINT(46.73687329635265 -121.87463823100035) bank122800 +122801 POINT(48.35580301762354 -123.25426800145021) bank122801 +122802 POINT(47.69856423922725 -121.69771565456317) bank122802 +122803 POINT(46.695010710937744 -122.17617907359156) bank122803 +122804 POINT(48.291867656574894 -122.4476641730539) bank122804 +122805 POINT(48.44623787536843 -121.4952905625531) bank122805 +122806 POINT(47.00089814189487 -121.78601823293019) bank122806 +122807 POINT(47.19806084552164 -122.59057508344704) bank122807 +122808 POINT(46.663943635001026 -123.25819891106019) bank122808 +122809 POINT(48.48988940580616 -121.6825931518408) bank122809 +122810 POINT(46.61073030303568 -122.70410889462784) bank122810 +122811 POINT(46.84883151951357 -121.33369738155233) bank122811 +122812 POINT(46.84820345266752 -122.16924367602367) bank122812 +122813 POINT(48.12668188455447 -123.19760229895394) bank122813 +122814 POINT(47.811473467117736 -122.34904736390861) bank122814 +122815 POINT(48.20484818299213 -123.22418075218718) bank122815 +122816 POINT(48.39995233601037 -123.07603306220405) bank122816 +122817 POINT(47.16895167143283 -122.33104423022552) bank122817 +122818 POINT(48.26300759845147 -121.71508623606276) bank122818 +122819 POINT(47.911484489595196 -123.26239004276124) bank122819 +122820 POINT(46.97405512052842 -123.07794052416477) bank122820 +122821 POINT(47.480674820608954 -122.14158752467746) bank122821 +122822 POINT(48.53313814528263 -123.13754189700205) bank122822 +122823 POINT(47.52712085485398 -121.77386143263764) bank122823 +122824 POINT(47.49960179333522 -121.73665792602276) bank122824 +122825 POINT(47.40021939027912 -121.45952712032643) bank122825 +122826 POINT(48.49691862352054 -122.47323001168449) bank122826 +122827 POINT(47.450546789542 -122.87864609798062) bank122827 +122828 POINT(46.93166667632534 -121.59269647486167) bank122828 +122829 POINT(46.905918835191265 -122.7278953049102) bank122829 +122830 POINT(47.452914249056626 -123.1473595291148) bank122830 +122831 POINT(48.08577173219664 -122.83476544710942) bank122831 +122832 POINT(47.330963511748806 -122.20724015141165) bank122832 +122833 POINT(48.33939693898686 -121.6271249733717) bank122833 +122834 POINT(47.82460822654423 -123.22790958904085) bank122834 +122835 POINT(47.70078435838852 -123.03939313701919) bank122835 +122836 POINT(46.792090218549156 -121.34094451547742) bank122836 +122837 POINT(47.07057010380054 -123.10724869945035) bank122837 +122838 POINT(47.523313679594594 -121.68999962058562) bank122838 +122839 POINT(47.22150708240688 -122.23245814991628) bank122839 +122840 POINT(46.997049841867785 -122.17629395980762) bank122840 +122841 POINT(47.20582102872157 -123.07319105115367) bank122841 +122842 POINT(48.24223768350852 -122.48892283183338) bank122842 +122843 POINT(46.79312679203224 -121.78085928471295) bank122843 +122844 POINT(48.35912422008264 -122.94178838300137) bank122844 +122845 POINT(46.64405134228923 -123.09367025922603) bank122845 +122846 POINT(48.43884379842332 -122.02613265883332) bank122846 +122847 POINT(47.18655533801385 -122.23357506886852) bank122847 +122848 POINT(47.82461400259466 -122.57351772221729) bank122848 +122849 POINT(48.17514853415537 -121.6702338503847) bank122849 +122850 POINT(46.72990738774738 -121.97469512340535) bank122850 +122851 POINT(46.97714426282613 -122.54728188002542) bank122851 +122852 POINT(47.52594259666426 -122.65786960570912) bank122852 +122853 POINT(46.86717264585407 -123.32639404371028) bank122853 +122854 POINT(47.40074550982004 -121.80426403465734) bank122854 +122855 POINT(48.17568770964393 -123.26052263507572) bank122855 +122856 POINT(48.09704339147392 -122.88294423644699) bank122856 +122857 POINT(48.059721102196185 -121.74243477814757) bank122857 +122858 POINT(47.558694524256005 -122.99751841440835) bank122858 +122859 POINT(48.36393391691215 -122.6680573984845) bank122859 +122860 POINT(48.55847501580798 -122.78917629223368) bank122860 +122861 POINT(46.93816234334911 -121.71630742510868) bank122861 +122862 POINT(48.153284751731796 -121.35333631640202) bank122862 +122863 POINT(46.77582467315456 -122.50543497990793) bank122863 +122864 POINT(47.52330125522844 -123.25434729984323) bank122864 +122865 POINT(46.79765133725279 -121.94938356247565) bank122865 +122866 POINT(48.04297703060771 -122.2577011005163) bank122866 +122867 POINT(48.56278287495638 -123.25553733160962) bank122867 +122868 POINT(48.33293409781911 -122.22062997165509) bank122868 +122869 POINT(46.94213239517082 -122.51574730452133) bank122869 +122870 POINT(47.06595459430912 -123.17810284484254) bank122870 +122871 POINT(47.20985693929945 -121.97491048909106) bank122871 +122872 POINT(48.07982911271628 -122.94772558624993) bank122872 +122873 POINT(47.89613633104379 -122.62504213870734) bank122873 +122874 POINT(48.37913354415595 -122.10537639912972) bank122874 +122875 POINT(47.44613688965034 -122.52147727363787) bank122875 +122876 POINT(47.9161863832945 -122.97843220883446) bank122876 +122877 POINT(46.7600038415187 -121.43134723781219) bank122877 +122878 POINT(48.16859235895907 -121.845314601132) bank122878 +122879 POINT(47.061478101500775 -122.78290924426771) bank122879 +122880 POINT(48.371488247065685 -121.73872078905377) bank122880 +122881 POINT(46.81169603575237 -122.09832953904365) bank122881 +122882 POINT(46.61933028775982 -121.95801319991087) bank122882 +122883 POINT(48.130394682790474 -121.65817167619323) bank122883 +122884 POINT(47.195075403341555 -123.24573395311243) bank122884 +122885 POINT(47.50583567067718 -122.75838350452541) bank122885 +122886 POINT(48.59333501986852 -121.42680072517035) bank122886 +122887 POINT(47.95189913378445 -122.89350649631994) bank122887 +122888 POINT(48.42978500607446 -122.52774963123913) bank122888 +122889 POINT(47.498933402049026 -121.87841399927945) bank122889 +122890 POINT(48.393777745728954 -121.41352715218322) bank122890 +122891 POINT(47.061116579633236 -121.40254754114603) bank122891 +122892 POINT(47.205323841204645 -123.16525332723617) bank122892 +122893 POINT(47.65903023299084 -122.8043779435164) bank122893 +122894 POINT(48.491533029693755 -122.30190137018128) bank122894 +122895 POINT(47.25245988704661 -121.60062703718853) bank122895 +122896 POINT(46.6570223593275 -121.52612434093348) bank122896 +122897 POINT(46.79096907775558 -123.16605280457233) bank122897 +122898 POINT(47.468604013381736 -122.79652401980225) bank122898 +122899 POINT(47.62519699338263 -122.04029786199119) bank122899 +122900 POINT(47.40836401621097 -122.82476939001887) bank122900 +122901 POINT(46.68565362553777 -122.59588254741777) bank122901 +122902 POINT(47.07475565387713 -122.89666631702126) bank122902 +122903 POINT(47.06784994380208 -123.26329261770785) bank122903 +122904 POINT(47.6775257855781 -121.5755270639813) bank122904 +122905 POINT(46.671908890309716 -122.23625769298289) bank122905 +122906 POINT(48.06667702188885 -121.37505687331999) bank122906 +122907 POINT(47.84180511621905 -123.08149204622283) bank122907 +122908 POINT(46.6311906186536 -122.8640318435602) bank122908 +122909 POINT(47.86254833387248 -123.17709575926906) bank122909 +122910 POINT(47.017231399791456 -122.77378624260881) bank122910 +122911 POINT(48.32030624275932 -121.75174180416113) bank122911 +122912 POINT(46.756925988559445 -122.34265156105376) bank122912 +122913 POINT(47.78187879363775 -122.08275166635731) bank122913 +122914 POINT(47.22957217136321 -123.31085801919212) bank122914 +122915 POINT(47.899824774423195 -121.34652343808983) bank122915 +122916 POINT(47.31258483374638 -121.93823142565675) bank122916 +122917 POINT(48.4321119040199 -122.5883269032878) bank122917 +122918 POINT(46.872228664664725 -121.37552234039345) bank122918 +122919 POINT(47.52146204697501 -122.97029009501097) bank122919 +122920 POINT(47.14775093956158 -121.72650256021198) bank122920 +122921 POINT(47.23241402830404 -123.16537619561659) bank122921 +122922 POINT(47.89528035055349 -122.70590819511324) bank122922 +122923 POINT(47.6951694832322 -122.51322608188767) bank122923 +122924 POINT(48.32307048404074 -121.94357934439178) bank122924 +122925 POINT(46.60631303780905 -122.94222313414711) bank122925 +122926 POINT(47.15204504481583 -122.45256393140811) bank122926 +122927 POINT(47.551547496588924 -121.98414466628056) bank122927 +122928 POINT(46.819657730276 -122.50055832172401) bank122928 +122929 POINT(46.73567494245987 -122.76366067793619) bank122929 +122930 POINT(48.46589671261561 -122.37187307585084) bank122930 +122931 POINT(48.46637756177987 -121.72494320497381) bank122931 +122932 POINT(47.47368626671245 -121.98927816296478) bank122932 +122933 POINT(48.484330913883134 -122.10147295889217) bank122933 +122934 POINT(47.5186952913884 -122.09583879837511) bank122934 +122935 POINT(47.04627135200428 -121.8367667559525) bank122935 +122936 POINT(47.53359525228764 -122.52005708796872) bank122936 +122937 POINT(46.77241963570665 -123.15710086356478) bank122937 +122938 POINT(47.52579559905681 -121.7308737393404) bank122938 +122939 POINT(47.81041552775693 -122.19321779469671) bank122939 +122940 POINT(47.7210268058945 -121.68841399371246) bank122940 +122941 POINT(47.48941139571717 -123.16094796068228) bank122941 +122942 POINT(47.83796024887369 -122.43425358376089) bank122942 +122943 POINT(47.70000527357236 -123.2656373366971) bank122943 +122944 POINT(48.09641504175178 -123.30334972100594) bank122944 +122945 POINT(47.576011404812725 -123.1669757018347) bank122945 +122946 POINT(47.69487897957342 -123.11780994589645) bank122946 +122947 POINT(47.50684233070155 -122.9527689949971) bank122947 +122948 POINT(47.62052231385161 -122.02440644952284) bank122948 +122949 POINT(46.93755970337921 -122.87309329700051) bank122949 +122950 POINT(47.36535247926103 -122.32669132364934) bank122950 +122951 POINT(47.1401416589336 -122.11455493256739) bank122951 +122952 POINT(47.431857074648406 -122.50637339049246) bank122952 +122953 POINT(48.247309072062414 -121.83868726808159) bank122953 +122954 POINT(47.54733051257066 -123.08426995683097) bank122954 +122955 POINT(48.21905351974128 -123.28411270597769) bank122955 +122956 POINT(48.19448480232923 -123.00463845258419) bank122956 +122957 POINT(47.45096824876084 -122.50988902707111) bank122957 +122958 POINT(47.097262972283374 -122.14334696915967) bank122958 +122959 POINT(48.1710606270742 -123.05033223987608) bank122959 +122960 POINT(47.43869935714722 -123.01778363716694) bank122960 +122961 POINT(47.97773060283184 -122.38768346813285) bank122961 +122962 POINT(48.035830613364574 -122.67013728584736) bank122962 +122963 POINT(47.75608835682186 -122.17730509969883) bank122963 +122964 POINT(47.360262714716086 -121.34639857390177) bank122964 +122965 POINT(46.94644343582995 -122.59915426184963) bank122965 +122966 POINT(47.099267619169666 -122.84613336118491) bank122966 +122967 POINT(46.94529737951801 -123.13316311041129) bank122967 +122968 POINT(47.527615875002134 -122.33217398733493) bank122968 +122969 POINT(48.259778392805586 -121.59410046564363) bank122969 +122970 POINT(47.790982638532356 -122.28000521442394) bank122970 +122971 POINT(48.14539106499521 -121.51547682080329) bank122971 +122972 POINT(46.78651469823569 -122.77921030758043) bank122972 +122973 POINT(47.39968517628541 -121.93865204684539) bank122973 +122974 POINT(46.741861919636165 -122.72365616204466) bank122974 +122975 POINT(46.74413490542944 -122.39383824982302) bank122975 +122976 POINT(48.53290699305591 -121.7650331405667) bank122976 +122977 POINT(48.41337685236694 -122.9484972444766) bank122977 +122978 POINT(48.49816321129616 -122.86222797119768) bank122978 +122979 POINT(47.975742078452356 -121.70317658004711) bank122979 +122980 POINT(48.56247972511849 -121.77491764756232) bank122980 +122981 POINT(47.85376838866693 -121.9344020638098) bank122981 +122982 POINT(48.42623759714529 -122.1857526089335) bank122982 +122983 POINT(47.513257729510265 -122.64430819492789) bank122983 +122984 POINT(48.44376617613837 -121.70374461099743) bank122984 +122985 POINT(47.803777428229814 -122.80512964188411) bank122985 +122986 POINT(47.73953230732397 -122.65886753504803) bank122986 +122987 POINT(47.43341560032914 -122.78170309072546) bank122987 +122988 POINT(47.30896131455866 -122.49299248079251) bank122988 +122989 POINT(47.683533815039674 -122.73848818874681) bank122989 +122990 POINT(47.747957958162665 -122.13397479154297) bank122990 +122991 POINT(48.456441937328805 -123.0153969605679) bank122991 +122992 POINT(48.56898021898309 -122.82802619544705) bank122992 +122993 POINT(47.930088958820825 -121.97447879120138) bank122993 +122994 POINT(48.147066099680785 -122.24900842677408) bank122994 +122995 POINT(46.74080998566306 -121.92328925411373) bank122995 +122996 POINT(47.95491160267831 -122.52003743828486) bank122996 +122997 POINT(47.38829413360653 -122.79306812733137) bank122997 +122998 POINT(47.031977725311485 -121.41668718671883) bank122998 +122999 POINT(47.107838660459656 -121.87782698419386) bank122999 +123000 POINT(48.2190750725364 -122.10715279909424) bank123000 +123001 POINT(48.193398277572335 -122.54734569184907) bank123001 +123002 POINT(48.47726736139678 -121.42739419235876) bank123002 +123003 POINT(47.91984887646937 -122.04026664076484) bank123003 +123004 POINT(47.972695838930214 -123.13529119857876) bank123004 +123005 POINT(47.18082367142546 -121.90444354046666) bank123005 +123006 POINT(48.23041042465397 -121.53951337001479) bank123006 +123007 POINT(46.644677792041584 -122.67869676839638) bank123007 +123008 POINT(46.930095265745805 -122.80305564028784) bank123008 +123009 POINT(46.98943804234113 -121.80418281466206) bank123009 +123010 POINT(46.975394602131466 -122.29947585583737) bank123010 +123011 POINT(48.094068398368755 -121.95382487419542) bank123011 +123012 POINT(46.96627211244371 -123.1394441013415) bank123012 +123013 POINT(48.20837811631048 -122.16464433073793) bank123013 +123014 POINT(48.17300521702203 -122.90696616681635) bank123014 +123015 POINT(48.26326953466167 -122.86926149295388) bank123015 +123016 POINT(47.52614137649428 -121.8750943793247) bank123016 +123017 POINT(47.50822268371722 -122.49689444616268) bank123017 +123018 POINT(46.844609345716506 -121.42907847657087) bank123018 +123019 POINT(48.38215020183998 -123.15078069554457) bank123019 +123020 POINT(46.82165090125993 -122.87556023188311) bank123020 +123021 POINT(47.282937240427664 -121.83294030004744) bank123021 +123022 POINT(48.09318059258014 -122.01912342747208) bank123022 +123023 POINT(48.12603810889315 -122.43070362421788) bank123023 +123024 POINT(47.727816295889205 -121.84416138223271) bank123024 +123025 POINT(47.1370887033897 -121.81258224074918) bank123025 +123026 POINT(46.91478393403296 -122.93622648559159) bank123026 +123027 POINT(47.77879426127467 -122.09945737247223) bank123027 +123028 POINT(48.06615450771383 -123.14281835135374) bank123028 +123029 POINT(46.66130670971349 -122.46953798169059) bank123029 +123030 POINT(46.676292378565776 -121.59425641295259) bank123030 +123031 POINT(46.68087493057902 -122.0514301700512) bank123031 +123032 POINT(47.29870110575259 -123.13265694359121) bank123032 +123033 POINT(46.89480884737942 -121.4557427555515) bank123033 +123034 POINT(48.398536860895966 -123.22120798206284) bank123034 +123035 POINT(48.21361651238028 -121.6755245781527) bank123035 +123036 POINT(48.3200579836342 -122.48206003294119) bank123036 +123037 POINT(47.76485623324322 -122.7936432456732) bank123037 +123038 POINT(47.88664406000642 -121.72473975664721) bank123038 +123039 POINT(47.519858606675804 -122.71452318004555) bank123039 +123040 POINT(47.322004061189915 -122.42254270954251) bank123040 +123041 POINT(47.08092141260303 -122.14589154656399) bank123041 +123042 POINT(48.31803757477472 -121.56583921859814) bank123042 +123043 POINT(47.95253591009704 -122.81918968243411) bank123043 +123044 POINT(47.49950568617011 -122.631714145879) bank123044 +123045 POINT(47.44383152685741 -122.21791154573168) bank123045 +123046 POINT(48.352314454635646 -122.39085733178761) bank123046 +123047 POINT(48.23172921573688 -122.3579077091841) bank123047 +123048 POINT(48.5782751564551 -121.4020807086268) bank123048 +123049 POINT(47.249171794278496 -122.34923690609193) bank123049 +123050 POINT(47.12420887887008 -121.93705375963741) bank123050 +123051 POINT(48.009458605055 -121.70955172024303) bank123051 +123052 POINT(46.732641154432116 -122.41074208838278) bank123052 +123053 POINT(48.46217431211265 -121.84279145976544) bank123053 +123054 POINT(47.10092081346298 -121.43073415626931) bank123054 +123055 POINT(47.72608376135289 -122.67364652815164) bank123055 +123056 POINT(46.81589314562582 -121.42783304937204) bank123056 +123057 POINT(47.75686141848322 -121.6399497475956) bank123057 +123058 POINT(48.051046911652215 -123.3169246975711) bank123058 +123059 POINT(48.59199958208119 -123.31138429943724) bank123059 +123060 POINT(47.76700421456879 -122.49369176389665) bank123060 +123061 POINT(48.38624186099103 -121.87864016503143) bank123061 +123062 POINT(47.24593063562256 -121.78392978271815) bank123062 +123063 POINT(47.30437666067133 -121.75030402558198) bank123063 +123064 POINT(47.135581361106844 -121.66883397201485) bank123064 +123065 POINT(47.14083855309276 -122.12436788253352) bank123065 +123066 POINT(46.73247952629906 -123.2024604753113) bank123066 +123067 POINT(46.98565298784444 -122.55954043507526) bank123067 +123068 POINT(47.53404383400402 -122.9423388327929) bank123068 +123069 POINT(48.421939608153195 -122.68773655704895) bank123069 +123070 POINT(47.80467607495556 -122.46072943467492) bank123070 +123071 POINT(47.02422052350511 -121.9643371215137) bank123071 +123072 POINT(47.83853833374293 -123.00237266332128) bank123072 +123073 POINT(47.76122536429933 -122.96073983260017) bank123073 +123074 POINT(48.032841255017765 -122.04622061452338) bank123074 +123075 POINT(47.71739701296795 -122.07278383394045) bank123075 +123076 POINT(46.879884089251334 -121.73851232392886) bank123076 +123077 POINT(47.403778188457906 -122.88587091200924) bank123077 +123078 POINT(46.94109496403635 -122.98998263931881) bank123078 +123079 POINT(47.33229704246865 -122.64253424637124) bank123079 +123080 POINT(48.01566210176937 -122.51111005985935) bank123080 +123081 POINT(48.46315662522738 -122.0903928808341) bank123081 +123082 POINT(48.45977494821337 -121.86386840896657) bank123082 +123083 POINT(47.03183788608785 -122.67300778293222) bank123083 +123084 POINT(47.21560990094431 -122.75996867447344) bank123084 +123085 POINT(47.37441452720223 -122.45813625060495) bank123085 +123086 POINT(46.86738465936833 -122.76325743459094) bank123086 +123087 POINT(47.26907700422748 -121.83651196289517) bank123087 +123088 POINT(48.54895649356631 -123.3168679584849) bank123088 +123089 POINT(46.810677335950324 -123.05693759447925) bank123089 +123090 POINT(46.680068058816055 -122.25224692372427) bank123090 +123091 POINT(47.86448836447211 -122.90381387222901) bank123091 +123092 POINT(48.33681850642176 -121.57637602529698) bank123092 +123093 POINT(48.3405789422623 -122.15018582405783) bank123093 +123094 POINT(46.88913942039515 -122.66234191845342) bank123094 +123095 POINT(47.359497649495765 -121.54671428118833) bank123095 +123096 POINT(48.21530746780686 -123.02619087196705) bank123096 +123097 POINT(47.93493979980962 -122.75042599795304) bank123097 +123098 POINT(47.154011907804694 -121.47561394659944) bank123098 +123099 POINT(48.3515188698093 -121.64447542065487) bank123099 +123100 POINT(47.995188591292205 -122.3960081069092) bank123100 +123101 POINT(48.23395495183329 -123.14298941345734) bank123101 +123102 POINT(46.849568343448944 -122.33746916344623) bank123102 +123103 POINT(47.59885794595999 -122.14602004886855) bank123103 +123104 POINT(48.18976085586577 -123.19906663286669) bank123104 +123105 POINT(48.0048510492668 -121.71972162097305) bank123105 +123106 POINT(47.505699429858986 -122.90162032365441) bank123106 +123107 POINT(48.49496896071596 -121.36709163019565) bank123107 +123108 POINT(47.49788813875787 -121.85027267558021) bank123108 +123109 POINT(48.169444006503774 -122.79229559718893) bank123109 +123110 POINT(48.48256582106115 -121.63472053869452) bank123110 +123111 POINT(48.3839501596489 -122.08263426446926) bank123111 +123112 POINT(48.52985857432146 -121.8068947728641) bank123112 +123113 POINT(46.82697114969344 -121.58175348491008) bank123113 +123114 POINT(47.875507587113795 -121.42237843724405) bank123114 +123115 POINT(46.94433482519094 -122.11735393193449) bank123115 +123116 POINT(47.80846076665157 -121.71334848820196) bank123116 +123117 POINT(47.90211279178719 -122.2742049728307) bank123117 +123118 POINT(48.05214213430498 -121.88567571846875) bank123118 +123119 POINT(47.38639021552109 -121.42883620675049) bank123119 +123120 POINT(46.71404113700636 -121.4456600535189) bank123120 +123121 POINT(48.43821761140864 -122.43816479893388) bank123121 +123122 POINT(47.93434087023574 -122.19036938125338) bank123122 +123123 POINT(47.24697071692058 -122.84043136565322) bank123123 +123124 POINT(48.250017713314996 -121.77309654347397) bank123124 +123125 POINT(46.71030780810016 -123.11539407531863) bank123125 +123126 POINT(48.03234390700999 -121.35181394364179) bank123126 +123127 POINT(47.306001178583855 -121.47960910683761) bank123127 +123128 POINT(47.57997772567421 -123.00572590648949) bank123128 +123129 POINT(46.96623046745963 -122.80733773420484) bank123129 +123130 POINT(48.08253763458102 -123.14523258242141) bank123130 +123131 POINT(47.50068552227643 -122.83418748030302) bank123131 +123132 POINT(47.974285312860765 -122.9868153673665) bank123132 +123133 POINT(48.21265044372994 -122.23040847598512) bank123133 +123134 POINT(47.41661487141487 -121.76025966697587) bank123134 +123135 POINT(48.270825763144636 -121.61427053033324) bank123135 +123136 POINT(48.29135382710808 -122.49488929816258) bank123136 +123137 POINT(47.07327818398072 -121.95367196752194) bank123137 +123138 POINT(48.027681632976915 -122.33894815062408) bank123138 +123139 POINT(48.389828965712006 -123.01121273173514) bank123139 +123140 POINT(47.30402315002783 -122.46734052244037) bank123140 +123141 POINT(47.56737175790442 -122.82548890764778) bank123141 +123142 POINT(47.99322293045229 -122.67514744634848) bank123142 +123143 POINT(48.524305167554246 -121.54576177474664) bank123143 +123144 POINT(46.82167832169955 -121.67602177223111) bank123144 +123145 POINT(48.34728255410516 -121.74313533001101) bank123145 +123146 POINT(48.251559171292136 -122.11482188094402) bank123146 +123147 POINT(47.21013843272711 -121.83424348845024) bank123147 +123148 POINT(47.651629076693595 -123.25855274320449) bank123148 +123149 POINT(48.59242155212264 -123.32530398392535) bank123149 +123150 POINT(47.959150190463994 -121.62317404965472) bank123150 +123151 POINT(48.270662492465526 -122.35810924699051) bank123151 +123152 POINT(47.38649206959832 -121.4046054598097) bank123152 +123153 POINT(47.99024972198631 -122.8280077354753) bank123153 +123154 POINT(48.000453633606774 -121.86778135819577) bank123154 +123155 POINT(47.49083756245446 -121.80200032436531) bank123155 +123156 POINT(48.195773460417904 -121.78521155797779) bank123156 +123157 POINT(47.38582382641728 -122.22566452812273) bank123157 +123158 POINT(48.153475827099754 -121.484676251156) bank123158 +123159 POINT(48.56883734267167 -122.00274311601933) bank123159 +123160 POINT(47.29162139496471 -123.31498137250307) bank123160 +123161 POINT(48.008731908955085 -122.36611363762515) bank123161 +123162 POINT(46.99546494476288 -122.2480519643741) bank123162 +123163 POINT(48.5028420073937 -121.56169358507584) bank123163 +123164 POINT(47.680821837464485 -122.65687657955756) bank123164 +123165 POINT(48.155687577105304 -123.2244178089162) bank123165 +123166 POINT(48.22022870257919 -121.77614025829172) bank123166 +123167 POINT(47.08780726968825 -122.45534683258185) bank123167 +123168 POINT(48.31376598820523 -122.41540303832575) bank123168 +123169 POINT(48.06373846033228 -122.1759682807213) bank123169 +123170 POINT(47.417841691877214 -121.61098161060406) bank123170 +123171 POINT(48.213590753472545 -121.53237256126002) bank123171 +123172 POINT(46.85686321919398 -122.28799315717394) bank123172 +123173 POINT(46.64941633659158 -122.24483375938566) bank123173 +123174 POINT(47.36279704097474 -121.89565333469392) bank123174 +123175 POINT(46.75094565154239 -121.9851708998594) bank123175 +123176 POINT(47.62424475215422 -122.42146771103093) bank123176 +123177 POINT(47.95605915134777 -122.32078621013075) bank123177 +123178 POINT(47.83660139908815 -121.92029759881599) bank123178 +123179 POINT(47.52368006862381 -121.51112597715182) bank123179 +123180 POINT(48.010254583654536 -122.1825215318251) bank123180 +123181 POINT(48.35166409100514 -122.56877808556168) bank123181 +123182 POINT(47.923226499940625 -123.06195421580114) bank123182 +123183 POINT(47.17340280721227 -121.89618608901397) bank123183 +123184 POINT(48.601162956025576 -121.83689304621056) bank123184 +123185 POINT(48.59605136963797 -122.10159944349378) bank123185 +123186 POINT(46.682946074874394 -122.84116382836544) bank123186 +123187 POINT(47.712641069342816 -122.34982555031822) bank123187 +123188 POINT(47.80920541972808 -121.53184993968925) bank123188 +123189 POINT(47.1969300609251 -122.86455363833649) bank123189 +123190 POINT(47.916284242459625 -121.61533899466694) bank123190 +123191 POINT(46.6998334285149 -121.98596129796131) bank123191 +123192 POINT(46.83807666906387 -122.17656307009736) bank123192 +123193 POINT(47.09776073288037 -122.17777073383677) bank123193 +123194 POINT(47.61296860341244 -122.04479276461142) bank123194 +123195 POINT(47.454655313094314 -123.22838490767202) bank123195 +123196 POINT(47.553828982351085 -121.94489517830544) bank123196 +123197 POINT(48.459043783399665 -122.963045400791) bank123197 +123198 POINT(47.58701748794775 -122.56527003153533) bank123198 +123199 POINT(46.659541892444864 -121.44555296337293) bank123199 +123200 POINT(48.60482228294034 -122.5468005701008) bank123200 +123201 POINT(48.60577832924765 -122.8402986372283) bank123201 +123202 POINT(47.50771186173051 -122.35871386219343) bank123202 +123203 POINT(48.296333105097126 -121.51388898058175) bank123203 +123204 POINT(48.110418828554415 -123.2908968106091) bank123204 +123205 POINT(47.77390724504589 -122.34793114691657) bank123205 +123206 POINT(47.82616882701046 -121.7369146498545) bank123206 +123207 POINT(46.75198897021301 -122.37013744869601) bank123207 +123208 POINT(47.70447609814502 -122.255949516825) bank123208 +123209 POINT(48.23429948126982 -122.76922325236823) bank123209 +123210 POINT(46.694814769996675 -122.17257104436882) bank123210 +123211 POINT(46.689228617147556 -122.13616900535628) bank123211 +123212 POINT(46.61264403027243 -121.75370952621542) bank123212 +123213 POINT(46.792239035811576 -123.29707674121833) bank123213 +123214 POINT(46.69182778365848 -122.45100765252924) bank123214 +123215 POINT(47.42017200248094 -122.28265340047068) bank123215 +123216 POINT(46.89293786347441 -121.7063881650841) bank123216 +123217 POINT(46.89718256655797 -121.93981362869513) bank123217 +123218 POINT(47.273045315732965 -123.08309162243656) bank123218 +123219 POINT(47.79277920106764 -121.3703733851144) bank123219 +123220 POINT(47.57955958505071 -122.8756393902322) bank123220 +123221 POINT(47.44148490649646 -122.10130262735024) bank123221 +123222 POINT(48.59848660242942 -122.34290460825036) bank123222 +123223 POINT(46.6232902433384 -121.99546839447832) bank123223 +123224 POINT(46.82196995095596 -122.75907903538553) bank123224 +123225 POINT(48.296655293936986 -122.77648937941478) bank123225 +123226 POINT(46.97827339501855 -122.88824646282181) bank123226 +123227 POINT(47.463688181163874 -121.73920261329049) bank123227 +123228 POINT(48.42034879911279 -121.90693036652318) bank123228 +123229 POINT(46.981598885300535 -121.80663687948798) bank123229 +123230 POINT(47.33991224075509 -123.08261636520842) bank123230 +123231 POINT(47.54581582669824 -122.966919719221) bank123231 +123232 POINT(47.49840480164308 -122.87990005819262) bank123232 +123233 POINT(48.09770744790144 -123.10977770339548) bank123233 +123234 POINT(47.80360587944962 -123.0069464758167) bank123234 +123235 POINT(47.41642953777666 -122.06486180546838) bank123235 +123236 POINT(48.05003155591675 -123.0582146592756) bank123236 +123237 POINT(48.18519554206028 -122.9097523918856) bank123237 +123238 POINT(47.66876339194868 -122.49184595631951) bank123238 +123239 POINT(48.53051865957856 -121.43240870059137) bank123239 +123240 POINT(47.1656119848193 -121.7570394169532) bank123240 +123241 POINT(47.09817376092843 -122.30172107897553) bank123241 +123242 POINT(46.79721070356729 -122.24499329616499) bank123242 +123243 POINT(47.01194536320797 -123.24795126694241) bank123243 +123244 POINT(47.382459678147214 -123.0149399399194) bank123244 +123245 POINT(47.64927360029076 -123.08963179294027) bank123245 +123246 POINT(47.926232135600564 -121.51028639500576) bank123246 +123247 POINT(48.2526962336903 -121.7207884471085) bank123247 +123248 POINT(48.49650636364766 -121.98728641668087) bank123248 +123249 POINT(47.59682168678992 -122.62157008590167) bank123249 +123250 POINT(47.96265166935684 -122.56760043321306) bank123250 +123251 POINT(48.134060636305335 -122.8572427972543) bank123251 +123252 POINT(47.03414971700931 -123.0560520726592) bank123252 +123253 POINT(47.739171139407645 -122.35562077731403) bank123253 +123254 POINT(47.990866283778345 -122.32630430964667) bank123254 +123255 POINT(47.96408391717534 -123.26952307345333) bank123255 +123256 POINT(46.62043052997973 -122.00971063032961) bank123256 +123257 POINT(46.972386800326156 -122.87319522376953) bank123257 +123258 POINT(48.495717785166384 -122.76237648925874) bank123258 +123259 POINT(46.67725025327148 -121.90085099321794) bank123259 +123260 POINT(47.62108488208303 -122.59588247494723) bank123260 +123261 POINT(48.3912106495331 -123.0739876567828) bank123261 +123262 POINT(47.490802842194284 -123.09297999033497) bank123262 +123263 POINT(47.05945787156809 -121.83175725321351) bank123263 +123264 POINT(48.571306354264586 -121.92258532100797) bank123264 +123265 POINT(47.67824641207198 -121.67390105314213) bank123265 +123266 POINT(48.110755499494836 -121.84780505424806) bank123266 +123267 POINT(48.483395746750155 -121.89436975626515) bank123267 +123268 POINT(48.57085356541071 -122.13016693470898) bank123268 +123269 POINT(47.79812085045501 -122.49030255500877) bank123269 +123270 POINT(48.43710662042839 -121.77929572155914) bank123270 +123271 POINT(48.29146072049344 -122.26061348288609) bank123271 +123272 POINT(48.54793505374668 -122.4115727976166) bank123272 +123273 POINT(48.05222841352245 -122.75222935206521) bank123273 +123274 POINT(47.76016229176017 -122.52539294955695) bank123274 +123275 POINT(47.525113692442154 -121.69985938819123) bank123275 +123276 POINT(46.65235776801142 -121.90958655315261) bank123276 +123277 POINT(47.138381391610956 -121.82617080149159) bank123277 +123278 POINT(47.86374848231822 -122.66676768069331) bank123278 +123279 POINT(47.53742066648456 -123.04923587779903) bank123279 +123280 POINT(47.84783117675029 -121.35238757424794) bank123280 +123281 POINT(47.79042365164638 -122.40290534283014) bank123281 +123282 POINT(47.47079483435674 -122.3744139550105) bank123282 +123283 POINT(47.105946456242485 -121.34796623800781) bank123283 +123284 POINT(48.15297163782697 -122.06564285364571) bank123284 +123285 POINT(47.97400857348377 -121.37988137682463) bank123285 +123286 POINT(47.21154435130673 -121.35571776049038) bank123286 +123287 POINT(46.663827128660806 -122.99927606545741) bank123287 +123288 POINT(47.84397286536624 -121.64768128661936) bank123288 +123289 POINT(47.589115477737565 -121.84346062544016) bank123289 +123290 POINT(48.567337834131735 -122.70192678498385) bank123290 +123291 POINT(46.68762837242628 -122.53720946744085) bank123291 +123292 POINT(47.83378707336832 -121.64190462727088) bank123292 +123293 POINT(47.351590694365065 -122.86660707502242) bank123293 +123294 POINT(46.939816554818414 -123.1523446423166) bank123294 +123295 POINT(46.61947925449224 -122.8126940435283) bank123295 +123296 POINT(46.844942587948914 -123.11370754510635) bank123296 +123297 POINT(46.95044864043297 -121.77245691341847) bank123297 +123298 POINT(46.90587739973722 -121.4764893560271) bank123298 +123299 POINT(47.90956778585143 -122.69921032198866) bank123299 +123300 POINT(48.27489754241109 -121.36068409795469) bank123300 +123301 POINT(47.689055529555986 -122.26218073217667) bank123301 +123302 POINT(47.07580831225782 -122.67214569076879) bank123302 +123303 POINT(47.283466943363706 -122.19332607760336) bank123303 +123304 POINT(47.57548295730093 -122.26149700866114) bank123304 +123305 POINT(48.57005869629722 -122.60337275705409) bank123305 +123306 POINT(48.493105300118785 -122.15849484036539) bank123306 +123307 POINT(48.388941831587466 -123.15724622886161) bank123307 +123308 POINT(48.58030850919083 -122.86189273010909) bank123308 +123309 POINT(46.75844353911037 -122.48292225557512) bank123309 +123310 POINT(47.17887961766104 -121.71542288891314) bank123310 +123311 POINT(46.808679150068 -122.31847561621503) bank123311 +123312 POINT(47.96617166229206 -122.89313526247174) bank123312 +123313 POINT(46.80344575687585 -122.03651087040086) bank123313 +123314 POINT(46.92374068478689 -123.21468391254426) bank123314 +123315 POINT(47.31171203062722 -122.79813500374063) bank123315 +123316 POINT(47.61051321833136 -121.72411930837276) bank123316 +123317 POINT(48.22360846912513 -122.82591092405474) bank123317 +123318 POINT(46.691403012950374 -121.88264238022875) bank123318 +123319 POINT(48.0444778118364 -122.14075627298648) bank123319 +123320 POINT(47.94216709259049 -122.93478877379702) bank123320 +123321 POINT(46.823191908433266 -122.92200885366337) bank123321 +123322 POINT(47.713137382759385 -122.53540988524406) bank123322 +123323 POINT(47.73437108551192 -122.23323428155527) bank123323 +123324 POINT(48.142640417606074 -122.96838740081472) bank123324 +123325 POINT(46.63492362605688 -123.17200742372134) bank123325 +123326 POINT(46.99713003921264 -121.93988339832681) bank123326 +123327 POINT(48.50786854100058 -122.00647516012526) bank123327 +123328 POINT(47.007327467234816 -121.8699251457181) bank123328 +123329 POINT(46.975851733584086 -122.13752295322587) bank123329 +123330 POINT(48.2361853947324 -121.59514374403994) bank123330 +123331 POINT(48.10716626922497 -123.26467876287444) bank123331 +123332 POINT(48.21593660467575 -121.77047725302265) bank123332 +123333 POINT(47.62169734852679 -122.74586878259338) bank123333 +123334 POINT(48.58357250783116 -122.11311397554425) bank123334 +123335 POINT(48.37399831556038 -123.28237742849845) bank123335 +123336 POINT(48.23867441489261 -122.91046879390036) bank123336 +123337 POINT(47.06612218652705 -121.93597233615029) bank123337 +123338 POINT(48.015374965713704 -123.16940713962057) bank123338 +123339 POINT(46.77381683886577 -121.39295928571664) bank123339 +123340 POINT(47.845222802740835 -122.7616567873464) bank123340 +123341 POINT(48.028078044823985 -121.91406350265798) bank123341 +123342 POINT(46.77049793116003 -121.36865921748611) bank123342 +123343 POINT(47.896135298767796 -123.17101581732966) bank123343 +123344 POINT(48.198039178491676 -122.62446454902279) bank123344 +123345 POINT(48.300821501410624 -122.55468406781462) bank123345 +123346 POINT(48.00554271146503 -122.96944716920214) bank123346 +123347 POINT(47.733338304449155 -123.2518135110322) bank123347 +123348 POINT(48.31159444202305 -122.38462190595627) bank123348 +123349 POINT(47.53473527291852 -122.54865347506869) bank123349 +123350 POINT(46.76658893099685 -123.00000681821383) bank123350 +123351 POINT(47.47101687407689 -121.52864492288943) bank123351 +123352 POINT(48.54090355597823 -122.77028340941371) bank123352 +123353 POINT(48.29912417461351 -123.04028987177105) bank123353 +123354 POINT(47.348870394189646 -121.57927783350604) bank123354 +123355 POINT(47.63983473532156 -122.89059474229965) bank123355 +123356 POINT(47.76200934147598 -122.12487113991034) bank123356 +123357 POINT(46.67205147672183 -121.44670461113373) bank123357 +123358 POINT(47.19828815765541 -122.10373720316392) bank123358 +123359 POINT(47.92061803035492 -123.16556542287258) bank123359 +123360 POINT(47.92370021812025 -122.43161142320929) bank123360 +123361 POINT(47.78248276600106 -123.21286309023105) bank123361 +123362 POINT(47.47826556653453 -122.643759201493) bank123362 +123363 POINT(47.31872709010734 -121.39449442768422) bank123363 +123364 POINT(48.524356270874534 -122.05626326737573) bank123364 +123365 POINT(48.010591959449606 -123.03233214070798) bank123365 +123366 POINT(47.31781381860506 -121.35099926581495) bank123366 +123367 POINT(47.23527457534772 -122.88152196290024) bank123367 +123368 POINT(47.774189273984156 -123.3246015690314) bank123368 +123369 POINT(48.12625665456416 -121.62605078431363) bank123369 +123370 POINT(48.02314122250647 -122.63573318564536) bank123370 +123371 POINT(47.02612507924711 -122.19162279252119) bank123371 +123372 POINT(47.11704236284188 -122.75670948925564) bank123372 +123373 POINT(47.52815529143269 -122.8170502977421) bank123373 +123374 POINT(48.58452652320018 -122.47459695162429) bank123374 +123375 POINT(48.19272145873785 -122.98628112070924) bank123375 +123376 POINT(47.1085519835318 -123.13960996857863) bank123376 +123377 POINT(47.66294098580749 -122.06757716859984) bank123377 +123378 POINT(46.70173535036584 -122.893560192603) bank123378 +123379 POINT(47.74910056712857 -122.93247849688514) bank123379 +123380 POINT(48.35074532040817 -122.3436032901482) bank123380 +123381 POINT(46.8969189771545 -123.02806774224308) bank123381 +123382 POINT(46.718545550113475 -121.33788614646144) bank123382 +123383 POINT(48.09055525324081 -122.09286092063651) bank123383 +123384 POINT(47.164672984619386 -123.08952254503583) bank123384 +123385 POINT(46.99822067604726 -122.82874601503238) bank123385 +123386 POINT(47.69593879651876 -123.25339780771394) bank123386 +123387 POINT(46.65787286043792 -122.09593029752438) bank123387 +123388 POINT(47.75082811482199 -122.53802586859153) bank123388 +123389 POINT(47.488544620824996 -121.62275881603406) bank123389 +123390 POINT(48.38275161991226 -122.52178359324633) bank123390 +123391 POINT(48.158289880311344 -121.87248017781349) bank123391 +123392 POINT(48.093908417421844 -122.66762964735636) bank123392 +123393 POINT(47.604388761248174 -122.95499058843635) bank123393 +123394 POINT(47.00962215439537 -122.53479750037134) bank123394 +123395 POINT(48.08838233079344 -122.14670786395038) bank123395 +123396 POINT(46.72290910424379 -122.6383283194367) bank123396 +123397 POINT(46.73288595753871 -122.88330385299237) bank123397 +123398 POINT(47.20111145289724 -123.32576001059094) bank123398 +123399 POINT(48.57369495400543 -122.70428506278617) bank123399 +123400 POINT(47.567073635641364 -123.147130269285) bank123400 +123401 POINT(47.11857045520227 -123.05183385679871) bank123401 +123402 POINT(47.191552571407605 -122.63565611592998) bank123402 +123403 POINT(46.73118792688128 -122.8843816200681) bank123403 +123404 POINT(47.47357001886682 -122.7187532433945) bank123404 +123405 POINT(47.17989578667718 -122.36568014523955) bank123405 +123406 POINT(47.143532518753645 -122.35958328447924) bank123406 +123407 POINT(47.62156012224697 -121.94320355894797) bank123407 +123408 POINT(47.897719314300105 -121.69722497577865) bank123408 +123409 POINT(48.550942384585255 -123.2023633436385) bank123409 +123410 POINT(47.05064606002262 -123.09800776011032) bank123410 +123411 POINT(47.063956485292415 -122.03681461123273) bank123411 +123412 POINT(47.64437763549096 -122.06593216345547) bank123412 +123413 POINT(47.45461097163461 -122.38251691189699) bank123413 +123414 POINT(46.965624946006685 -123.15850178280591) bank123414 +123415 POINT(48.173173613262676 -123.33093309982159) bank123415 +123416 POINT(46.794894602515534 -123.25614896370821) bank123416 +123417 POINT(47.45265803919723 -122.642218345594) bank123417 +123418 POINT(47.866011219825495 -121.62292594958872) bank123418 +123419 POINT(47.28534971479428 -122.44975804030449) bank123419 +123420 POINT(48.562867665440216 -122.0762407104836) bank123420 +123421 POINT(47.46427158165648 -122.09501979809649) bank123421 +123422 POINT(48.09857081252543 -122.28395629576471) bank123422 +123423 POINT(46.72994275536828 -122.01318560616217) bank123423 +123424 POINT(47.28276742137604 -121.66025098986361) bank123424 +123425 POINT(48.523609526058074 -121.73601760229523) bank123425 +123426 POINT(47.03034112101249 -122.67882346886012) bank123426 +123427 POINT(47.16725160785741 -122.52552438638598) bank123427 +123428 POINT(46.652607014307556 -123.07202429622434) bank123428 +123429 POINT(47.91150027304451 -121.54210031364943) bank123429 +123430 POINT(46.6363413767881 -122.55472425250981) bank123430 +123431 POINT(48.35902951588908 -123.04240973231606) bank123431 +123432 POINT(46.77159262029856 -123.15537475177968) bank123432 +123433 POINT(47.1596451786059 -122.18846772499414) bank123433 +123434 POINT(46.997729040424865 -122.14495341288996) bank123434 +123435 POINT(46.815007179082706 -122.23641318066845) bank123435 +123436 POINT(46.939580842674516 -123.1668309469016) bank123436 +123437 POINT(47.39784426748031 -122.16750488029965) bank123437 +123438 POINT(46.86814703293327 -121.3341807820798) bank123438 +123439 POINT(48.343100430254054 -122.29705898384987) bank123439 +123440 POINT(47.27519668163859 -121.52061121086342) bank123440 +123441 POINT(47.16171563537139 -121.79657087118983) bank123441 +123442 POINT(47.58706233469009 -123.04633047180673) bank123442 +123443 POINT(47.64161652691693 -122.2065851742925) bank123443 +123444 POINT(48.255195279481555 -122.39123259533878) bank123444 +123445 POINT(46.811286183465164 -121.54259808654017) bank123445 +123446 POINT(47.25980790811117 -122.70617298726046) bank123446 +123447 POINT(47.382451867361176 -122.25022552465202) bank123447 +123448 POINT(47.11410151703354 -121.83145423838363) bank123448 +123449 POINT(46.86433615036972 -123.18924480601366) bank123449 +123450 POINT(48.16123958595858 -122.96082159516179) bank123450 +123451 POINT(48.46126834116378 -121.342527121471) bank123451 +123452 POINT(47.09228415674764 -121.63529243759317) bank123452 +123453 POINT(46.64769245320379 -123.16631899437766) bank123453 +123454 POINT(47.388420209513555 -122.9723000404145) bank123454 +123455 POINT(47.46096469304709 -122.01320168192626) bank123455 +123456 POINT(48.29313107220737 -122.46189571966724) bank123456 +123457 POINT(47.85675140852756 -121.41150091872116) bank123457 +123458 POINT(47.46626709720903 -122.49094328985339) bank123458 +123459 POINT(47.33698136997799 -122.36385425432769) bank123459 +123460 POINT(47.61954379380678 -121.53649210932343) bank123460 +123461 POINT(46.73306548172876 -121.51010561201805) bank123461 +123462 POINT(46.752247114834525 -122.88938338956021) bank123462 +123463 POINT(46.66571890161178 -122.22713162200891) bank123463 +123464 POINT(48.15893982635091 -123.07044269844644) bank123464 +123465 POINT(46.657698205047375 -123.30130407723055) bank123465 +123466 POINT(46.78860000827927 -121.69749885774553) bank123466 +123467 POINT(48.32626205516462 -123.06361171890714) bank123467 +123468 POINT(46.814337832720554 -122.1733909767246) bank123468 +123469 POINT(47.915386778590445 -122.34434698475268) bank123469 +123470 POINT(48.08268348925629 -121.87912387080878) bank123470 +123471 POINT(47.979129662260526 -122.9122177335285) bank123471 +123472 POINT(47.28734232938209 -122.80972407276664) bank123472 +123473 POINT(46.8118534570011 -122.66988625945716) bank123473 +123474 POINT(46.87119703055257 -121.7891110858074) bank123474 +123475 POINT(47.17141799586866 -122.68718379801629) bank123475 +123476 POINT(47.568919223136504 -121.46806274107405) bank123476 +123477 POINT(47.85160107873414 -122.94045145716791) bank123477 +123478 POINT(47.618166773386285 -122.49198297666351) bank123478 +123479 POINT(48.274304948836246 -122.62838383829244) bank123479 +123480 POINT(48.44426233018861 -121.64054655036912) bank123480 +123481 POINT(46.787781482966054 -121.61502545319313) bank123481 +123482 POINT(47.99188584380389 -122.9180385638405) bank123482 +123483 POINT(48.21273738886205 -121.6232079449368) bank123483 +123484 POINT(46.924737024521214 -122.80766892990432) bank123484 +123485 POINT(46.771085453471684 -121.86486409625584) bank123485 +123486 POINT(48.39370122724927 -122.02611364122842) bank123486 +123487 POINT(47.20886618021717 -122.10103436886205) bank123487 +123488 POINT(47.52445269284916 -122.02970351345981) bank123488 +123489 POINT(46.61678776187247 -121.60951816234758) bank123489 +123490 POINT(46.99142091132234 -121.7895529510356) bank123490 +123491 POINT(46.622843751872374 -122.3503263334814) bank123491 +123492 POINT(47.31553800262964 -122.2030859607212) bank123492 +123493 POINT(47.033771278933834 -121.68421121146446) bank123493 +123494 POINT(48.40997758641218 -121.56958140708751) bank123494 +123495 POINT(47.904012267252625 -123.18350564567193) bank123495 +123496 POINT(47.854056904301444 -121.52385606625882) bank123496 +123497 POINT(48.01959506810443 -122.8100667639919) bank123497 +123498 POINT(47.010593670892085 -122.35493433820108) bank123498 +123499 POINT(48.29323781196957 -122.57984364909349) bank123499 +123500 POINT(48.252431361264186 -122.27450343055833) bank123500 +123501 POINT(47.911604174387655 -122.48981842063715) bank123501 +123502 POINT(47.679975638060974 -122.32661388177884) bank123502 +123503 POINT(46.7715444281447 -123.11856231423576) bank123503 +123504 POINT(47.582945836708774 -122.90263980045847) bank123504 +123505 POINT(48.278490174465134 -121.78945866882921) bank123505 +123506 POINT(48.17558738155205 -122.40587806837927) bank123506 +123507 POINT(46.977221695788344 -122.92103105743077) bank123507 +123508 POINT(46.60691412230851 -122.13674733436504) bank123508 +123509 POINT(47.047464518924635 -121.82726604676654) bank123509 +123510 POINT(48.17054626727956 -122.6100739947274) bank123510 +123511 POINT(47.0016849231887 -123.14807364018291) bank123511 +123512 POINT(48.16649320761879 -121.95145355013801) bank123512 +123513 POINT(48.27710725629594 -121.9454807503671) bank123513 +123514 POINT(48.45691856112365 -121.39361750047357) bank123514 +123515 POINT(47.0372608573193 -121.4637407503291) bank123515 +123516 POINT(47.18838621775575 -122.99929667532764) bank123516 +123517 POINT(47.56984623597217 -121.6575178955746) bank123517 +123518 POINT(48.14006054701861 -123.11092026990056) bank123518 +123519 POINT(47.44349729993526 -122.77347266704562) bank123519 +123520 POINT(48.398491948345786 -121.51929836601074) bank123520 +123521 POINT(46.62784169982087 -122.13170245511435) bank123521 +123522 POINT(46.69590264361749 -123.29126625816771) bank123522 +123523 POINT(48.486039586918494 -122.27491328122797) bank123523 +123524 POINT(47.643773433298485 -122.4942223503438) bank123524 +123525 POINT(47.72424313756526 -123.10565084830785) bank123525 +123526 POINT(47.17149341783627 -122.44480239562844) bank123526 +123527 POINT(46.74381874548994 -122.26708994495249) bank123527 +123528 POINT(47.02509912163556 -121.71643958424976) bank123528 +123529 POINT(47.992836805412594 -122.50377031227634) bank123529 +123530 POINT(46.6190292747366 -122.12347573832577) bank123530 +123531 POINT(46.88846331497983 -123.06088848749341) bank123531 +123532 POINT(47.84903227634284 -122.1674602297127) bank123532 +123533 POINT(48.43534849407206 -122.40555920935374) bank123533 +123534 POINT(46.81870615286533 -122.40792571540126) bank123534 +123535 POINT(47.88613924578284 -123.041344207499) bank123535 +123536 POINT(46.96935298800601 -122.64738677097579) bank123536 +123537 POINT(47.321870790959515 -121.987643923056) bank123537 +123538 POINT(48.440137129492754 -122.13459147891886) bank123538 +123539 POINT(47.18178586489907 -121.83135245154973) bank123539 +123540 POINT(48.49501691714859 -122.66695074580167) bank123540 +123541 POINT(46.76104027596246 -121.37671351244909) bank123541 +123542 POINT(47.16789777447517 -122.78422067728373) bank123542 +123543 POINT(47.35103491621186 -122.57479083113277) bank123543 +123544 POINT(47.15065500886734 -121.74044778477287) bank123544 +123545 POINT(46.71798002385838 -122.09802860483754) bank123545 +123546 POINT(47.47291494865251 -122.64200802847928) bank123546 +123547 POINT(48.56164831343628 -122.23759021336309) bank123547 +123548 POINT(47.76547373220089 -122.37832299253961) bank123548 +123549 POINT(47.44340492574493 -123.16617207828354) bank123549 +123550 POINT(47.99577009569718 -123.02334295575432) bank123550 +123551 POINT(47.4120864587464 -121.51375847604479) bank123551 +123552 POINT(48.175273136864035 -121.39481475517944) bank123552 +123553 POINT(48.04982575103506 -122.59381963415906) bank123553 +123554 POINT(46.720623176810406 -122.51406962964158) bank123554 +123555 POINT(47.36516562781359 -123.2705466108718) bank123555 +123556 POINT(48.052186103212364 -122.14017281341773) bank123556 +123557 POINT(48.33875405338103 -122.9038816572479) bank123557 +123558 POINT(46.8996037500073 -122.91523620771922) bank123558 +123559 POINT(46.844245408189465 -123.17294099348113) bank123559 +123560 POINT(47.828799073709746 -123.13206223066355) bank123560 +123561 POINT(47.00271089097588 -123.09165556202917) bank123561 +123562 POINT(47.96528947153237 -121.80349592401114) bank123562 +123563 POINT(46.65944580776559 -122.9929522588097) bank123563 +123564 POINT(47.502888047806415 -122.3221742021085) bank123564 +123565 POINT(46.63853880354 -122.17686469898953) bank123565 +123566 POINT(47.12325766865248 -121.83126161378001) bank123566 +123567 POINT(46.750455162167505 -121.84734466775602) bank123567 +123568 POINT(46.76065683861587 -121.52317636105883) bank123568 +123569 POINT(48.54798478316831 -123.07145328234373) bank123569 +123570 POINT(46.8901274373017 -121.72938228117462) bank123570 +123571 POINT(47.69201423967649 -121.47317539395831) bank123571 +123572 POINT(48.381289474512585 -122.26014655571036) bank123572 +123573 POINT(47.2589747109122 -121.95494652763817) bank123573 +123574 POINT(48.007254201461166 -121.33619542367929) bank123574 +123575 POINT(46.684070300751294 -122.42844921528217) bank123575 +123576 POINT(47.12204006370913 -121.39328716312379) bank123576 +123577 POINT(47.926753598888375 -123.0445928666269) bank123577 +123578 POINT(47.3227068841052 -122.43168410465283) bank123578 +123579 POINT(47.77817756099011 -122.51407381264073) bank123579 +123580 POINT(47.1697392206794 -122.67113076278979) bank123580 +123581 POINT(48.06003452849034 -122.33429354131678) bank123581 +123582 POINT(46.8485318174904 -122.95655161084412) bank123582 +123583 POINT(47.806871743933435 -123.28139869357803) bank123583 +123584 POINT(46.80387136006781 -123.07941980098703) bank123584 +123585 POINT(47.47061736465062 -123.03089404766462) bank123585 +123586 POINT(47.105985934569944 -121.80867484429957) bank123586 +123587 POINT(47.46818964498338 -122.77499430631079) bank123587 +123588 POINT(47.14104217236818 -121.45107130539708) bank123588 +123589 POINT(47.88469255949186 -123.30326316215991) bank123589 +123590 POINT(47.82163823433259 -122.44145369541991) bank123590 +123591 POINT(47.66141493550696 -122.2780684855723) bank123591 +123592 POINT(47.552287849374025 -123.014766201776) bank123592 +123593 POINT(46.74096377206707 -123.22395715110888) bank123593 +123594 POINT(48.44305526350764 -123.04362581428884) bank123594 +123595 POINT(46.98837639911387 -122.73510001412485) bank123595 +123596 POINT(46.85287811124365 -122.62123397362335) bank123596 +123597 POINT(46.97549500715917 -121.72054262936217) bank123597 +123598 POINT(47.19746619474744 -121.69169515761305) bank123598 +123599 POINT(46.90632458214687 -122.28403271443375) bank123599 +123600 POINT(47.0786424981198 -121.71607850711244) bank123600 +123601 POINT(46.98306406080302 -121.37111601263433) bank123601 +123602 POINT(47.51222097473649 -122.7390608481917) bank123602 +123603 POINT(47.62531873928376 -122.53793751622885) bank123603 +123604 POINT(47.37741080162854 -121.7189746656877) bank123604 +123605 POINT(48.23451498996932 -121.72287722351962) bank123605 +123606 POINT(48.59056029055862 -121.61970487885435) bank123606 +123607 POINT(47.80044959741192 -121.47106205461027) bank123607 +123608 POINT(46.87531935140593 -121.67869608539698) bank123608 +123609 POINT(46.97893073190748 -122.94968090293204) bank123609 +123610 POINT(47.46324222664964 -123.11496147719338) bank123610 +123611 POINT(48.03149735761082 -122.64624584322887) bank123611 +123612 POINT(48.006231956289035 -122.84860272212981) bank123612 +123613 POINT(47.42054278945752 -122.92092803240564) bank123613 +123614 POINT(46.72274580151841 -121.59180087390513) bank123614 +123615 POINT(48.168913777850975 -122.73517404237214) bank123615 +123616 POINT(47.324446517242784 -122.96744148795442) bank123616 +123617 POINT(46.68627099065738 -121.64380708139342) bank123617 +123618 POINT(47.65097758114548 -122.43790979791503) bank123618 +123619 POINT(46.79885131108702 -122.63802444616287) bank123619 +123620 POINT(47.58119892351948 -123.15982559182757) bank123620 +123621 POINT(47.75384382077342 -123.27659271999282) bank123621 +123622 POINT(48.123392155556466 -123.25912613284544) bank123622 +123623 POINT(48.52579960929029 -121.84825682664872) bank123623 +123624 POINT(48.53894850312086 -122.80487274730346) bank123624 +123625 POINT(47.519019995435436 -121.85886432347108) bank123625 +123626 POINT(48.00923368937572 -121.37806609095146) bank123626 +123627 POINT(46.65268572914617 -121.79221964740451) bank123627 +123628 POINT(47.04993678229756 -122.09409651490454) bank123628 +123629 POINT(48.27116479669916 -122.36406975932343) bank123629 +123630 POINT(47.585887353345484 -121.84250215392181) bank123630 +123631 POINT(46.92790700070383 -122.17842295423748) bank123631 +123632 POINT(46.80808284166363 -121.53429011371901) bank123632 +123633 POINT(47.903924898104165 -121.81124590011716) bank123633 +123634 POINT(48.605765774533594 -122.43539229152925) bank123634 +123635 POINT(47.03458893837643 -122.58919058497646) bank123635 +123636 POINT(47.29623629638648 -122.05901804901704) bank123636 +123637 POINT(48.19928953333434 -122.03162908251488) bank123637 +123638 POINT(47.12532177080205 -122.28526335062274) bank123638 +123639 POINT(47.451042692002005 -121.75775378484362) bank123639 +123640 POINT(47.82297076952571 -121.97774541270464) bank123640 +123641 POINT(47.4951435341359 -122.02155888303608) bank123641 +123642 POINT(48.188395199557135 -122.74979947937051) bank123642 +123643 POINT(48.045899360024265 -121.49139782529501) bank123643 +123644 POINT(48.346311954824984 -122.83200780408119) bank123644 +123645 POINT(47.23177710135056 -122.95540794145657) bank123645 +123646 POINT(47.026722212352006 -123.10872727141586) bank123646 +123647 POINT(46.96853336060825 -122.3103569505952) bank123647 +123648 POINT(47.63743163822452 -123.03489370767839) bank123648 +123649 POINT(47.23167962110394 -123.1449916784192) bank123649 +123650 POINT(48.23915873610588 -122.52234592220034) bank123650 +123651 POINT(47.936622602400305 -122.91762323170383) bank123651 +123652 POINT(48.449039163869664 -123.08898611213148) bank123652 +123653 POINT(48.457744650666086 -121.39003211673446) bank123653 +123654 POINT(47.372367708004724 -122.33220026164054) bank123654 +123655 POINT(48.051834963009654 -121.92859537634305) bank123655 +123656 POINT(48.07364475763408 -121.78968844516747) bank123656 +123657 POINT(48.13126043257284 -121.36170457811431) bank123657 +123658 POINT(47.93923350166634 -122.39678596479375) bank123658 +123659 POINT(47.791944620690074 -122.93845332998805) bank123659 +123660 POINT(47.57225768005301 -123.08179909904635) bank123660 +123661 POINT(48.34970323330479 -122.34876057989746) bank123661 +123662 POINT(47.02213444430018 -122.53298684512721) bank123662 +123663 POINT(47.66405929804973 -122.63710631342725) bank123663 +123664 POINT(46.99789024419973 -121.63609886340856) bank123664 +123665 POINT(48.03550298477574 -121.3354761311493) bank123665 +123666 POINT(47.09986776453496 -122.73231929760735) bank123666 +123667 POINT(46.9591408390208 -122.27765860889166) bank123667 +123668 POINT(47.939155327244954 -121.83540976442256) bank123668 +123669 POINT(48.287519909864194 -122.08406106569437) bank123669 +123670 POINT(47.598907728730936 -122.1107213603174) bank123670 +123671 POINT(48.30131072952974 -121.81397930173138) bank123671 +123672 POINT(48.49294755071079 -122.53708540377731) bank123672 +123673 POINT(48.57033057639284 -121.56462087164527) bank123673 +123674 POINT(48.032990809205764 -121.99743739790273) bank123674 +123675 POINT(48.12220870290311 -121.61497015264402) bank123675 +123676 POINT(48.3909516990276 -122.64016477699954) bank123676 +123677 POINT(47.74622005969648 -121.54378282720043) bank123677 +123678 POINT(46.8861142763002 -121.47222162982375) bank123678 +123679 POINT(48.01247124398238 -123.25409422972581) bank123679 +123680 POINT(47.36277477547956 -122.64489240265112) bank123680 +123681 POINT(47.11483624453273 -123.04710593630556) bank123681 +123682 POINT(47.56219672820068 -121.951234577329) bank123682 +123683 POINT(48.06168163481537 -122.81173799906969) bank123683 +123684 POINT(47.15093686339767 -122.35708886442917) bank123684 +123685 POINT(47.64136983723574 -121.63322882715303) bank123685 +123686 POINT(47.82816111127192 -121.36260240577899) bank123686 +123687 POINT(47.380188431106966 -122.89755660606062) bank123687 +123688 POINT(48.528257182879926 -123.08916690167526) bank123688 +123689 POINT(47.975710302369706 -122.70594056380793) bank123689 +123690 POINT(48.352875587554315 -123.16418505836786) bank123690 +123691 POINT(47.27210107069404 -122.97010967213089) bank123691 +123692 POINT(47.71978604395092 -123.24962630246469) bank123692 +123693 POINT(47.3095362919251 -121.98792472879197) bank123693 +123694 POINT(48.112192660930525 -121.3416394362558) bank123694 +123695 POINT(47.13986337066731 -123.17305307072462) bank123695 +123696 POINT(47.27510776535085 -122.45403952104782) bank123696 +123697 POINT(47.64673998286703 -122.99801982132361) bank123697 +123698 POINT(47.0642377631648 -121.69763667701552) bank123698 +123699 POINT(47.892798035439384 -123.2173480379973) bank123699 +123700 POINT(48.03799578183443 -122.4278478030969) bank123700 +123701 POINT(48.142569120265364 -122.9383039471848) bank123701 +123702 POINT(47.19821734227409 -121.97773690723385) bank123702 +123703 POINT(48.58442211166516 -123.15965176339377) bank123703 +123704 POINT(48.577827773060996 -121.65899679297215) bank123704 +123705 POINT(48.01146810620849 -122.15847643987098) bank123705 +123706 POINT(47.433766554197945 -122.25334636817482) bank123706 +123707 POINT(46.72602616698774 -123.10486844728149) bank123707 +123708 POINT(47.531150642395154 -122.56997172226929) bank123708 +123709 POINT(47.84445312256486 -122.000180015557) bank123709 +123710 POINT(48.24334447289004 -122.49441552387837) bank123710 +123711 POINT(46.859127193716226 -121.39618426232526) bank123711 +123712 POINT(47.94721462066062 -123.18422877638115) bank123712 +123713 POINT(47.316328830157 -122.57996131904899) bank123713 +123714 POINT(46.99210898992393 -122.67782126501875) bank123714 +123715 POINT(47.47333421669116 -122.16862784830242) bank123715 +123716 POINT(48.17670655529398 -122.80227104916166) bank123716 +123717 POINT(47.84470567872551 -121.81598142609765) bank123717 +123718 POINT(47.145051325713155 -121.9109284954136) bank123718 +123719 POINT(47.39307088775688 -122.41026758586084) bank123719 +123720 POINT(47.48380333287256 -122.85825198764331) bank123720 +123721 POINT(47.64834293919832 -123.03229038168836) bank123721 +123722 POINT(46.72601182267367 -122.50548320496284) bank123722 +123723 POINT(46.9900551934409 -121.41103711886954) bank123723 +123724 POINT(47.72594248637368 -121.7417123378645) bank123724 +123725 POINT(48.031679527204155 -121.5614218263509) bank123725 +123726 POINT(48.290610324226975 -121.93610666401689) bank123726 +123727 POINT(48.16607331909244 -123.31499017293581) bank123727 +123728 POINT(46.73054397726376 -122.71412608174707) bank123728 +123729 POINT(47.15103842402611 -123.32514812597424) bank123729 +123730 POINT(47.07796046246202 -122.40508225461664) bank123730 +123731 POINT(47.88542875984528 -123.28376859720692) bank123731 +123732 POINT(46.71025672781933 -122.72801832625696) bank123732 +123733 POINT(46.97912860348194 -121.73767264050188) bank123733 +123734 POINT(46.674428073256394 -122.67779586765519) bank123734 +123735 POINT(47.71737221821457 -121.61907246155025) bank123735 +123736 POINT(47.0203512319012 -121.51681391019723) bank123736 +123737 POINT(47.979509426754696 -121.76332518943462) bank123737 +123738 POINT(47.19828223774061 -122.01951433142985) bank123738 +123739 POINT(47.795028931791336 -122.22334618414965) bank123739 +123740 POINT(47.19709463284462 -122.48374196910018) bank123740 +123741 POINT(47.332427461198805 -121.59443403606068) bank123741 +123742 POINT(47.32234307922113 -121.46689623743757) bank123742 +123743 POINT(47.01352668439837 -121.84733903520997) bank123743 +123744 POINT(47.25265227957452 -121.89591194441084) bank123744 +123745 POINT(48.023208161582595 -122.22322075138688) bank123745 +123746 POINT(48.195408593728374 -122.31366396739115) bank123746 +123747 POINT(48.12289588150084 -122.31105793573492) bank123747 +123748 POINT(46.87589117708982 -121.9965828004848) bank123748 +123749 POINT(47.77263847434181 -123.23527396597862) bank123749 +123750 POINT(46.792931169284316 -122.97426510550675) bank123750 +123751 POINT(46.70613395311352 -123.24392211828697) bank123751 +123752 POINT(48.06889452939382 -122.4891583536958) bank123752 +123753 POINT(47.06656581154304 -122.72844930409619) bank123753 +123754 POINT(47.49669534289001 -122.21823925877216) bank123754 +123755 POINT(47.578284632078045 -123.02879562521038) bank123755 +123756 POINT(47.34533670275521 -121.60149891600993) bank123756 +123757 POINT(47.125643085272486 -123.1173000825752) bank123757 +123758 POINT(48.25574084073005 -122.35665921951936) bank123758 +123759 POINT(47.20329401339266 -122.80192996536658) bank123759 +123760 POINT(47.11810151624456 -122.2927188166174) bank123760 +123761 POINT(47.6347032446571 -122.88354752325249) bank123761 +123762 POINT(47.919688437734365 -122.46858686596423) bank123762 +123763 POINT(47.45858507904374 -121.42510216231801) bank123763 +123764 POINT(47.24909772553429 -122.87075580161441) bank123764 +123765 POINT(47.38952538055983 -123.13917748366357) bank123765 +123766 POINT(46.73194387177378 -121.8593964865325) bank123766 +123767 POINT(48.472648225004455 -121.80194917589507) bank123767 +123768 POINT(46.964820733395456 -122.46896649060596) bank123768 +123769 POINT(47.28388530481538 -121.68878788508334) bank123769 +123770 POINT(47.0606562214733 -122.95881447462972) bank123770 +123771 POINT(47.213631419554034 -122.53660195637643) bank123771 +123772 POINT(47.959554873895755 -121.81019873944462) bank123772 +123773 POINT(46.6322777795593 -122.40732655878624) bank123773 +123774 POINT(46.94719989337752 -121.55396162817306) bank123774 +123775 POINT(48.09927406158983 -122.88866202103699) bank123775 +123776 POINT(48.3360210744829 -122.98754571611111) bank123776 +123777 POINT(46.62912986144466 -122.4560904932405) bank123777 +123778 POINT(48.35007775261676 -122.33759469309527) bank123778 +123779 POINT(46.752394980942604 -121.75406032685294) bank123779 +123780 POINT(46.86276864821883 -122.52696498394872) bank123780 +123781 POINT(48.448791599549956 -122.07952842146341) bank123781 +123782 POINT(47.90274944504561 -122.58234063950671) bank123782 +123783 POINT(46.83584200318824 -121.56905877414746) bank123783 +123784 POINT(48.25975339308162 -123.26308537090927) bank123784 +123785 POINT(47.019993066170464 -122.86493519652369) bank123785 +123786 POINT(47.02718926530318 -121.89458005369542) bank123786 +123787 POINT(47.27085005154655 -121.77950021455446) bank123787 +123788 POINT(48.507476764400735 -121.54279036269901) bank123788 +123789 POINT(46.691954383664594 -122.13652390893131) bank123789 +123790 POINT(46.72057922022352 -122.48151844787644) bank123790 +123791 POINT(48.00509093039801 -121.89541849904496) bank123791 +123792 POINT(48.52041826155129 -122.65029042270183) bank123792 +123793 POINT(47.85488685508291 -122.01115246734712) bank123793 +123794 POINT(48.276562342402606 -121.50673509283584) bank123794 +123795 POINT(47.8154436203283 -123.06579054996502) bank123795 +123796 POINT(46.939787527373106 -122.98246358800078) bank123796 +123797 POINT(47.96720062449708 -122.49947889642009) bank123797 +123798 POINT(48.571927711349865 -123.12944652625262) bank123798 +123799 POINT(48.30315144336884 -123.27500071159902) bank123799 +123800 POINT(47.703193093103884 -122.94555709858187) bank123800 +123801 POINT(47.78748526891944 -121.92684062341354) bank123801 +123802 POINT(47.66707516919961 -121.86801626046297) bank123802 +123803 POINT(47.57205876628014 -122.22478532918826) bank123803 +123804 POINT(47.6972689073885 -122.32795097517835) bank123804 +123805 POINT(48.10941311596075 -122.8832549968925) bank123805 +123806 POINT(47.5825022245263 -122.77830192291022) bank123806 +123807 POINT(46.66046806228718 -121.35990396348276) bank123807 +123808 POINT(48.08053650542677 -121.48897066841673) bank123808 +123809 POINT(47.40962887972204 -121.41956357095592) bank123809 +123810 POINT(48.24272986711929 -122.66262972778338) bank123810 +123811 POINT(48.51182293640033 -122.81558795556735) bank123811 +123812 POINT(48.1814884295454 -123.19351827174326) bank123812 +123813 POINT(48.28945687628019 -123.14724231040346) bank123813 +123814 POINT(48.547036170045 -123.25995488024776) bank123814 +123815 POINT(48.47914204650468 -122.62916936096778) bank123815 +123816 POINT(47.07988872192816 -122.17954666926066) bank123816 +123817 POINT(47.20208156487868 -121.87606396849546) bank123817 +123818 POINT(46.97279546362092 -121.85089259641848) bank123818 +123819 POINT(46.930129393617804 -122.96057192256978) bank123819 +123820 POINT(47.81994691934699 -122.31776947577511) bank123820 +123821 POINT(47.35477921171394 -122.79889641201522) bank123821 +123822 POINT(48.396599401445336 -121.91518970460977) bank123822 +123823 POINT(47.2668026871235 -121.86563750729425) bank123823 +123824 POINT(48.21501873799321 -122.32747576525833) bank123824 +123825 POINT(47.7157164684223 -121.91772177930437) bank123825 +123826 POINT(47.96800974915913 -122.78105567718194) bank123826 +123827 POINT(48.28463395801249 -123.1330126280496) bank123827 +123828 POINT(47.22298148313001 -123.17408797577083) bank123828 +123829 POINT(48.52246878020141 -123.08042883456211) bank123829 +123830 POINT(48.02985178528453 -122.24832278799796) bank123830 +123831 POINT(47.55772532771472 -121.63607582734467) bank123831 +123832 POINT(47.89492862725039 -123.17730958578731) bank123832 +123833 POINT(47.45296977355676 -121.4505178523708) bank123833 +123834 POINT(47.4412297892436 -123.20784085383369) bank123834 +123835 POINT(46.68428158740394 -121.67559056766937) bank123835 +123836 POINT(46.62804243783908 -122.74257169898563) bank123836 +123837 POINT(46.95639001216435 -121.46726172886545) bank123837 +123838 POINT(47.00929930223714 -122.1092214009251) bank123838 +123839 POINT(47.7005942889936 -121.45612194055751) bank123839 +123840 POINT(47.1793351770566 -123.26338931000984) bank123840 +123841 POINT(47.72028939617444 -122.85646034248035) bank123841 +123842 POINT(47.825533914609935 -123.0523650880848) bank123842 +123843 POINT(48.07921377202749 -123.1450354790701) bank123843 +123844 POINT(46.730990709316764 -123.12909614475046) bank123844 +123845 POINT(48.44727785428703 -122.22881300430497) bank123845 +123846 POINT(48.30465495830624 -121.77745112957781) bank123846 +123847 POINT(46.74876323186132 -122.08397180497381) bank123847 +123848 POINT(47.75408753611571 -121.34427996184908) bank123848 +123849 POINT(48.50677938753801 -121.9086051364631) bank123849 +123850 POINT(47.27519472165694 -121.78248685756402) bank123850 +123851 POINT(46.748576090325294 -122.42941470658221) bank123851 +123852 POINT(47.57522797789856 -123.08668959767077) bank123852 +123853 POINT(47.34522154728014 -121.66849742957903) bank123853 +123854 POINT(48.58233996630039 -122.76892198001603) bank123854 +123855 POINT(47.700802588785514 -122.7981646940128) bank123855 +123856 POINT(47.02234845408756 -121.82813655931176) bank123856 +123857 POINT(47.30069637255075 -121.80375962839919) bank123857 +123858 POINT(46.725410498626054 -122.29146569162948) bank123858 +123859 POINT(48.02697315552406 -123.04725511302806) bank123859 +123860 POINT(46.82754383486677 -121.6413813740897) bank123860 +123861 POINT(48.48197230288669 -122.77565835840042) bank123861 +123862 POINT(48.172098980239866 -123.26329983197058) bank123862 +123863 POINT(48.59499045671599 -122.9981367961152) bank123863 +123864 POINT(46.73643300906338 -123.03408675426964) bank123864 +123865 POINT(48.20695926851265 -122.83904576706759) bank123865 +123866 POINT(48.58622570512961 -123.02498090485548) bank123866 +123867 POINT(47.49849691342703 -122.20385129498428) bank123867 +123868 POINT(46.8012303331062 -122.67134571932293) bank123868 +123869 POINT(48.37967294035984 -121.39419599723725) bank123869 +123870 POINT(47.5332197038496 -122.92001265036268) bank123870 +123871 POINT(48.539875776560834 -123.19892887905935) bank123871 +123872 POINT(47.41720500003581 -122.5958730567139) bank123872 +123873 POINT(48.027663339574296 -121.85725010470048) bank123873 +123874 POINT(47.537864423824594 -122.96229050845544) bank123874 +123875 POINT(47.412983133782554 -121.5262963494698) bank123875 +123876 POINT(48.60429941581533 -122.92162118998101) bank123876 +123877 POINT(47.758265592708696 -121.61936279584882) bank123877 +123878 POINT(48.31620465177679 -122.43647371444636) bank123878 +123879 POINT(48.48966375204603 -123.14223625324524) bank123879 +123880 POINT(47.040908402377006 -122.71440251305441) bank123880 +123881 POINT(47.6011928751792 -122.18069263563841) bank123881 +123882 POINT(47.696028465738586 -121.51659656848862) bank123882 +123883 POINT(46.853125221085435 -122.71616180383303) bank123883 +123884 POINT(47.935654842257726 -121.45888728860487) bank123884 +123885 POINT(47.15549565270828 -122.00885558268409) bank123885 +123886 POINT(47.26507446473021 -122.87212758027944) bank123886 +123887 POINT(47.05531376494492 -122.72939483311755) bank123887 +123888 POINT(46.723904481487054 -121.58889940438407) bank123888 +123889 POINT(47.23236088110929 -121.57827566865664) bank123889 +123890 POINT(47.66665791147169 -122.58099218128086) bank123890 +123891 POINT(46.701149898693195 -122.4632410976626) bank123891 +123892 POINT(47.871425663629 -122.41087938116176) bank123892 +123893 POINT(47.756336594941295 -121.52554672428768) bank123893 +123894 POINT(48.549602550381366 -121.37320910448872) bank123894 +123895 POINT(47.382092715858455 -122.93213245578826) bank123895 +123896 POINT(47.372592996095634 -122.98835112898331) bank123896 +123897 POINT(47.877682608737864 -121.67488634620463) bank123897 +123898 POINT(48.59628081935691 -123.13416143402561) bank123898 +123899 POINT(48.218961508425615 -122.54373831653137) bank123899 +123900 POINT(47.75879060625101 -122.60278703770618) bank123900 +123901 POINT(47.12052980344905 -122.30808227821584) bank123901 +123902 POINT(47.212137081367146 -122.75207098951176) bank123902 +123903 POINT(47.07506379407211 -122.72468662135704) bank123903 +123904 POINT(47.0771029716051 -123.31730529067904) bank123904 +123905 POINT(46.712635293818465 -122.73796114917681) bank123905 +123906 POINT(47.488558585237534 -122.40555345181983) bank123906 +123907 POINT(47.77035653610018 -122.08236006977776) bank123907 +123908 POINT(46.82603699385408 -122.99858044844608) bank123908 +123909 POINT(46.84096412527804 -121.4566677414209) bank123909 +123910 POINT(47.54892673598415 -122.3542946832323) bank123910 +123911 POINT(47.221291566433486 -122.65634530505488) bank123911 +123912 POINT(47.46745596021919 -121.87007304471008) bank123912 +123913 POINT(47.796799973569385 -121.6952939291158) bank123913 +123914 POINT(47.9964045482443 -122.97616163851382) bank123914 +123915 POINT(47.42635804204029 -121.99224072492622) bank123915 +123916 POINT(48.26625637404623 -121.73036124183884) bank123916 +123917 POINT(48.094975468645046 -122.02700007685708) bank123917 +123918 POINT(47.86707357251899 -123.03776827845785) bank123918 +123919 POINT(48.35716127686108 -122.25091859028673) bank123919 +123920 POINT(46.98824006413987 -122.20833391853878) bank123920 +123921 POINT(48.45513490795002 -121.79123821687358) bank123921 +123922 POINT(47.82011931158522 -122.20681320247417) bank123922 +123923 POINT(47.126777133340106 -123.06617882197742) bank123923 +123924 POINT(46.657467864313645 -121.96760654662947) bank123924 +123925 POINT(47.943891557993354 -121.92235938155095) bank123925 +123926 POINT(47.99769203425603 -122.94470655849695) bank123926 +123927 POINT(47.811774750094216 -121.7138373923327) bank123927 +123928 POINT(46.65003983556986 -122.48660116056483) bank123928 +123929 POINT(48.02812549082277 -122.30091439055738) bank123929 +123930 POINT(47.53847689937583 -122.44671892725893) bank123930 +123931 POINT(47.436112182751245 -121.74163020469426) bank123931 +123932 POINT(47.652118026856485 -121.50928947087094) bank123932 +123933 POINT(48.00546585166631 -122.32771207811354) bank123933 +123934 POINT(47.08778812831689 -122.25939025776817) bank123934 +123935 POINT(47.569232175745036 -123.1916297636902) bank123935 +123936 POINT(47.73595331182007 -123.1547930374593) bank123936 +123937 POINT(47.27689365193634 -122.01866939144588) bank123937 +123938 POINT(47.112214302619755 -122.97940766493669) bank123938 +123939 POINT(48.1940899733567 -122.8754287316251) bank123939 +123940 POINT(47.56386096441578 -123.16153194310456) bank123940 +123941 POINT(48.258117289367334 -122.52289074249063) bank123941 +123942 POINT(47.26154856719529 -122.72463639931055) bank123942 +123943 POINT(46.75259272892095 -123.24771535277704) bank123943 +123944 POINT(48.055849460375775 -122.1631388672918) bank123944 +123945 POINT(47.305173063062355 -122.06311351331792) bank123945 +123946 POINT(46.730871139811555 -122.70691830884533) bank123946 +123947 POINT(46.94551371326712 -123.00165444612465) bank123947 +123948 POINT(48.31474273399569 -122.4844505856658) bank123948 +123949 POINT(47.00723241896434 -122.91763159911122) bank123949 +123950 POINT(46.766440322069535 -122.94095723975715) bank123950 +123951 POINT(47.08055644897025 -121.59664213428324) bank123951 +123952 POINT(47.44576768997082 -122.92223984664989) bank123952 +123953 POINT(47.36880720711921 -123.00505259186693) bank123953 +123954 POINT(46.89708380458032 -121.66041190425872) bank123954 +123955 POINT(47.7422841651875 -123.29398576175515) bank123955 +123956 POINT(47.80499125353627 -122.41831427991872) bank123956 +123957 POINT(47.089474437786805 -122.04506986995025) bank123957 +123958 POINT(47.873896752387395 -123.23094380006312) bank123958 +123959 POINT(47.823969199692385 -122.98810542654374) bank123959 +123960 POINT(47.78225611245008 -122.05391350883048) bank123960 +123961 POINT(46.83022192966589 -123.07809788758166) bank123961 +123962 POINT(47.75130310064484 -121.33268064323067) bank123962 +123963 POINT(47.76740789959958 -122.74518228698668) bank123963 +123964 POINT(47.23968753325776 -123.06240157195468) bank123964 +123965 POINT(48.47464384285984 -123.15696864140553) bank123965 +123966 POINT(47.14076717444161 -122.21490975275681) bank123966 +123967 POINT(47.09791666543512 -122.17688092658003) bank123967 +123968 POINT(47.248865593453765 -122.23564976437731) bank123968 +123969 POINT(46.99320352220519 -121.63187184060288) bank123969 +123970 POINT(47.06914235783256 -123.1731683928628) bank123970 +123971 POINT(48.006162389429925 -122.39805040681466) bank123971 +123972 POINT(46.88170621138725 -121.44369015386962) bank123972 +123973 POINT(48.13200971961679 -122.14493426302995) bank123973 +123974 POINT(46.897439982782856 -121.61198665951939) bank123974 +123975 POINT(48.414730545781566 -122.78782246165123) bank123975 +123976 POINT(47.851689518914725 -121.41833542467488) bank123976 +123977 POINT(48.000652548108036 -123.28658976012053) bank123977 +123978 POINT(47.13072502922173 -122.22963786434386) bank123978 +123979 POINT(48.371946020063476 -123.17303467983879) bank123979 +123980 POINT(47.490786299874934 -122.2951664232607) bank123980 +123981 POINT(47.8372266501581 -122.7865884752622) bank123981 +123982 POINT(47.89229443822924 -121.8006675504858) bank123982 +123983 POINT(48.22763124394029 -122.18575693878122) bank123983 +123984 POINT(48.41483122362085 -122.73727482667276) bank123984 +123985 POINT(47.46754280572175 -121.44873349360809) bank123985 +123986 POINT(46.79350111671964 -121.66037911459227) bank123986 +123987 POINT(47.0643202520719 -122.84770288054563) bank123987 +123988 POINT(47.34817070637189 -122.69830602853875) bank123988 +123989 POINT(47.81262556785621 -122.00107685594782) bank123989 +123990 POINT(48.16347982204963 -122.04473528815721) bank123990 +123991 POINT(48.101126057894994 -122.47312137819054) bank123991 +123992 POINT(48.520296794203404 -121.72438166869205) bank123992 +123993 POINT(48.4832047911452 -122.26537342348837) bank123993 +123994 POINT(46.95392968285506 -123.10728704925162) bank123994 +123995 POINT(47.43694711311589 -122.68244966327451) bank123995 +123996 POINT(48.067868390220994 -121.60199591092905) bank123996 +123997 POINT(48.378776369052005 -121.34848022456795) bank123997 +123998 POINT(48.212260258985786 -121.47518973473852) bank123998 +123999 POINT(46.74194873305816 -123.28593412034762) bank123999 +124000 POINT(47.97181920298881 -122.4823807134993) bank124000 +124001 POINT(47.174636819368274 -122.08192091590242) bank124001 +124002 POINT(47.535608410636264 -122.78120049736052) bank124002 +124003 POINT(47.063013734819705 -121.95456898357094) bank124003 +124004 POINT(48.04522412043318 -122.36004187526677) bank124004 +124005 POINT(47.94720461348607 -121.76341519884933) bank124005 +124006 POINT(46.8554297378157 -121.43496941312871) bank124006 +124007 POINT(47.67666651794444 -121.71468992718532) bank124007 +124008 POINT(46.86167047998209 -122.45135830866435) bank124008 +124009 POINT(48.43489835930498 -122.92443541245088) bank124009 +124010 POINT(46.835636010818305 -121.77600996293513) bank124010 +124011 POINT(47.754571799165035 -122.5733554834576) bank124011 +124012 POINT(47.09882627986341 -121.94271652802757) bank124012 +124013 POINT(47.19318896711226 -122.2945325033636) bank124013 +124014 POINT(48.17396422013431 -122.72113524055398) bank124014 +124015 POINT(48.40170007712833 -123.03691043551076) bank124015 +124016 POINT(48.22799570599091 -121.53521223872875) bank124016 +124017 POINT(46.76872518075715 -122.0445161000844) bank124017 +124018 POINT(47.79299963210922 -122.67283245786086) bank124018 +124019 POINT(48.28168155988504 -122.00829388974878) bank124019 +124020 POINT(47.26660922034908 -121.61546285090876) bank124020 +124021 POINT(48.60034536049995 -122.55778820909822) bank124021 +124022 POINT(47.03865038135443 -121.88877761524789) bank124022 +124023 POINT(48.006201112107874 -121.92746160652797) bank124023 +124024 POINT(46.93543884755307 -121.87923327689863) bank124024 +124025 POINT(46.7663636435274 -123.09270851060099) bank124025 +124026 POINT(47.1700210600704 -121.45575696191767) bank124026 +124027 POINT(48.24490715668097 -121.8476827410075) bank124027 +124028 POINT(47.85963436391349 -122.4215165729267) bank124028 +124029 POINT(47.739547958053166 -122.27285982202297) bank124029 +124030 POINT(46.71845388079484 -121.42428049950259) bank124030 +124031 POINT(48.029751026688615 -121.40968935732033) bank124031 +124032 POINT(48.110068792639325 -123.0392386590859) bank124032 +124033 POINT(48.13163326757865 -123.167957430249) bank124033 +124034 POINT(48.13013115239992 -121.51879243608549) bank124034 +124035 POINT(46.81111808104841 -121.76347996953555) bank124035 +124036 POINT(48.112091073544335 -122.95435827006723) bank124036 +124037 POINT(47.90475346335809 -122.45940406993168) bank124037 +124038 POINT(48.08946920160782 -122.06798560848989) bank124038 +124039 POINT(47.535532949328 -121.50767917241957) bank124039 +124040 POINT(47.59358670378 -122.00233242204533) bank124040 +124041 POINT(47.97289382670967 -122.37821977726962) bank124041 +124042 POINT(48.03249445384112 -121.36028024874692) bank124042 +124043 POINT(47.388361752313266 -121.4445848984272) bank124043 +124044 POINT(48.041347711847024 -123.23638874714557) bank124044 +124045 POINT(47.01264652460089 -121.98770616358165) bank124045 +124046 POINT(47.06304665308308 -122.34346781430324) bank124046 +124047 POINT(48.31685762769913 -121.44309549960585) bank124047 +124048 POINT(47.78494438713464 -122.37642367559286) bank124048 +124049 POINT(48.58299728762427 -122.74196639947183) bank124049 +124050 POINT(46.86512751011553 -122.78538035153515) bank124050 +124051 POINT(47.65449591201397 -121.8008278687064) bank124051 +124052 POINT(48.53984772988898 -123.31522993066665) bank124052 +124053 POINT(47.637624695127776 -121.52828503369163) bank124053 +124054 POINT(46.855819516968054 -122.15993442911824) bank124054 +124055 POINT(48.3060106718548 -122.04776277610392) bank124055 +124056 POINT(46.736222980897715 -123.01678330887819) bank124056 +124057 POINT(47.14388022254681 -121.51434327976908) bank124057 +124058 POINT(46.651179472297535 -121.66175848350784) bank124058 +124059 POINT(46.982541124227 -121.81480074501147) bank124059 +124060 POINT(48.169090862963266 -122.47925573413379) bank124060 +124061 POINT(46.87853664034798 -122.77400758083444) bank124061 +124062 POINT(47.78869804892276 -122.37204925827938) bank124062 +124063 POINT(47.72405049827381 -121.49676936344378) bank124063 +124064 POINT(46.63272860409462 -121.85225459967361) bank124064 +124065 POINT(48.11262315737742 -122.80100683703192) bank124065 +124066 POINT(47.79789839634338 -122.21516711061436) bank124066 +124067 POINT(48.0278053434437 -122.3554567835038) bank124067 +124068 POINT(47.337707807888165 -122.3567702936976) bank124068 +124069 POINT(48.21653533741074 -122.7083635228546) bank124069 +124070 POINT(47.14830506446695 -123.29843755803414) bank124070 +124071 POINT(46.64110848179496 -122.13023098131647) bank124071 +124072 POINT(47.705469112766586 -123.11958868228103) bank124072 +124073 POINT(48.09038813856495 -123.32136771772795) bank124073 +124074 POINT(47.01950730204826 -122.78596644766499) bank124074 +124075 POINT(46.725699440679136 -123.22755155889155) bank124075 +124076 POINT(47.605155113336586 -122.75665236132087) bank124076 +124077 POINT(47.646165053281884 -123.26334310472141) bank124077 +124078 POINT(46.9135157440455 -122.1560784819825) bank124078 +124079 POINT(47.99607597670533 -121.66628901409312) bank124079 +124080 POINT(48.22876050500981 -121.84420875009971) bank124080 +124081 POINT(47.27383402388202 -121.94582042571149) bank124081 +124082 POINT(48.51510561775066 -122.49825162603236) bank124082 +124083 POINT(48.232394410371015 -122.96539215921356) bank124083 +124084 POINT(48.44507674355431 -122.37370467119038) bank124084 +124085 POINT(47.46573580309126 -122.54718113318327) bank124085 +124086 POINT(48.44790176476205 -121.64345934114483) bank124086 +124087 POINT(47.617046138588854 -123.2288902440063) bank124087 +124088 POINT(47.6733366321004 -122.94573940934089) bank124088 +124089 POINT(48.58612207720761 -123.29434052073702) bank124089 +124090 POINT(47.901488897418574 -122.28145484718034) bank124090 +124091 POINT(48.530559982046896 -122.75460945832722) bank124091 +124092 POINT(46.825978448212105 -123.0548620704073) bank124092 +124093 POINT(47.22596138709994 -121.56208087684871) bank124093 +124094 POINT(48.35537287207001 -123.32678984836727) bank124094 +124095 POINT(48.01654492475581 -121.64883986302458) bank124095 +124096 POINT(48.021533203651586 -121.74271293358383) bank124096 +124097 POINT(48.53589721944883 -122.45525466867767) bank124097 +124098 POINT(47.83740051563601 -121.7969781805149) bank124098 +124099 POINT(48.43360389502023 -123.14201567474365) bank124099 +124100 POINT(47.17440255451133 -122.77460332538477) bank124100 +124101 POINT(47.80058749333892 -123.289754457794) bank124101 +124102 POINT(46.64312787359821 -123.22680080433827) bank124102 +124103 POINT(47.568567510994306 -122.05720588862147) bank124103 +124104 POINT(48.09739412831698 -122.27133863363872) bank124104 +124105 POINT(46.81949346326216 -122.5761131885539) bank124105 +124106 POINT(48.59815252297872 -122.67356083675958) bank124106 +124107 POINT(47.92530357242589 -122.8598548709992) bank124107 +124108 POINT(48.12299188760683 -122.00552893661782) bank124108 +124109 POINT(48.51015422009482 -121.36403561044621) bank124109 +124110 POINT(47.91616677074019 -121.81048964479301) bank124110 +124111 POINT(47.4452020305381 -121.56131249107685) bank124111 +124112 POINT(48.33662786796441 -121.78783304113341) bank124112 +124113 POINT(48.22660238689266 -123.31178794560432) bank124113 +124114 POINT(48.52421445369966 -121.80279557274854) bank124114 +124115 POINT(47.285284132277475 -122.0529412819656) bank124115 +124116 POINT(46.68626681251316 -122.03594387206917) bank124116 +124117 POINT(46.682864988853574 -122.83259523410914) bank124117 +124118 POINT(47.19232311477687 -123.2498352017441) bank124118 +124119 POINT(47.173475879287516 -122.85296488146335) bank124119 +124120 POINT(48.32757002621178 -122.09385771012222) bank124120 +124121 POINT(46.69563283312024 -121.43965792163732) bank124121 +124122 POINT(47.146947341765376 -121.87299837041252) bank124122 +124123 POINT(46.64158841096521 -123.00332826024568) bank124123 +124124 POINT(48.252932870879185 -122.30027976230508) bank124124 +124125 POINT(48.11654205599794 -123.22197337934104) bank124125 +124126 POINT(48.56231846836274 -122.27317933490407) bank124126 +124127 POINT(47.84926184246881 -121.88609022644485) bank124127 +124128 POINT(47.54539091627754 -123.26748872619807) bank124128 +124129 POINT(47.94815328812368 -121.5165886245631) bank124129 +124130 POINT(46.9868866125741 -122.76521828044545) bank124130 +124131 POINT(47.78949005246901 -121.8626221040312) bank124131 +124132 POINT(48.47230057764492 -122.08292709726805) bank124132 +124133 POINT(46.834496420372126 -122.84177466319353) bank124133 +124134 POINT(48.58680042199704 -121.43282184667785) bank124134 +124135 POINT(46.64728093573007 -122.34755870232418) bank124135 +124136 POINT(48.17829528058539 -121.74381007519148) bank124136 +124137 POINT(47.98680677414315 -121.79106479285919) bank124137 +124138 POINT(47.40600578897018 -121.3723926469873) bank124138 +124139 POINT(47.117790488699725 -121.89856594640848) bank124139 +124140 POINT(46.64931724356194 -121.68278730668659) bank124140 +124141 POINT(48.501054394760835 -121.61769747136387) bank124141 +124142 POINT(46.703405821854325 -122.24930046775994) bank124142 +124143 POINT(48.2377523380626 -122.11286995526092) bank124143 +124144 POINT(47.05062287867236 -121.56349728763803) bank124144 +124145 POINT(47.46809296318017 -122.98492472782374) bank124145 +124146 POINT(47.387168643612746 -121.98741011285982) bank124146 +124147 POINT(47.394512857397714 -121.78731281063106) bank124147 +124148 POINT(47.443761470187155 -121.43764839457363) bank124148 +124149 POINT(47.92253076689949 -122.10321403082615) bank124149 +124150 POINT(47.836995239717155 -121.34635503646042) bank124150 +124151 POINT(47.591964920532995 -121.78810519375568) bank124151 +124152 POINT(47.86999942633865 -121.91622806936564) bank124152 +124153 POINT(47.70339471602741 -122.95536948599617) bank124153 +124154 POINT(48.368735447380864 -122.31804200820119) bank124154 +124155 POINT(47.56613413075841 -122.90094091857827) bank124155 +124156 POINT(47.29412418562524 -121.7705584383768) bank124156 +124157 POINT(48.439272963688445 -122.9124765645545) bank124157 +124158 POINT(46.9973728782435 -122.84023171836472) bank124158 +124159 POINT(47.09627473720559 -122.72468392266798) bank124159 +124160 POINT(48.20743190811375 -121.65465418182929) bank124160 +124161 POINT(47.87335529994634 -122.43528739301196) bank124161 +124162 POINT(47.54730054483404 -123.28143166373553) bank124162 +124163 POINT(47.5922452868089 -123.23022980079213) bank124163 +124164 POINT(48.56435953871723 -121.58440260284435) bank124164 +124165 POINT(46.70084099174002 -121.61971160637222) bank124165 +124166 POINT(47.98842945658514 -122.06707437064814) bank124166 +124167 POINT(47.35300825005038 -121.4416989053929) bank124167 +124168 POINT(47.367020989954014 -121.72183430860703) bank124168 +124169 POINT(47.55209948495987 -122.80856886376786) bank124169 +124170 POINT(48.0911274465963 -122.86279115625918) bank124170 +124171 POINT(47.73843954343978 -122.78890729605013) bank124171 +124172 POINT(48.17145426116501 -122.8059406612369) bank124172 +124173 POINT(48.14415196285384 -122.59155579642486) bank124173 +124174 POINT(46.768273841396436 -121.80073563001677) bank124174 +124175 POINT(48.169850969145166 -121.84470189088549) bank124175 +124176 POINT(48.47784658555863 -121.3703904485032) bank124176 +124177 POINT(47.42327584833994 -122.4853805586527) bank124177 +124178 POINT(48.17548714326919 -121.76101670521015) bank124178 +124179 POINT(47.02504905028048 -121.56864475319038) bank124179 +124180 POINT(47.347118281711055 -122.46102730823009) bank124180 +124181 POINT(48.54309342894924 -122.20927276748093) bank124181 +124182 POINT(47.909666401533464 -121.99101788838844) bank124182 +124183 POINT(47.567686368827644 -121.76333450308083) bank124183 +124184 POINT(46.75470470330176 -121.59405539901302) bank124184 +124185 POINT(47.200363382213546 -123.25965609194024) bank124185 +124186 POINT(47.64914139308402 -122.69259877841289) bank124186 +124187 POINT(46.833452455137376 -123.23144902978616) bank124187 +124188 POINT(47.07131396295852 -121.94777603939566) bank124188 +124189 POINT(48.21978674151354 -121.7040209465481) bank124189 +124190 POINT(47.56109414650525 -122.19395450706368) bank124190 +124191 POINT(46.7337502014598 -121.65225196619475) bank124191 +124192 POINT(47.26547316548759 -123.02782813849143) bank124192 +124193 POINT(47.393839804026186 -123.23688111355071) bank124193 +124194 POINT(46.85148721378142 -121.74792609878605) bank124194 +124195 POINT(46.84450850906948 -122.29459735986727) bank124195 +124196 POINT(48.05934474684776 -122.48560731891216) bank124196 +124197 POINT(48.13806467142026 -121.50879083014992) bank124197 +124198 POINT(47.314953310320746 -121.71821359610074) bank124198 +124199 POINT(47.70781760052602 -122.49850776511026) bank124199 +124200 POINT(47.70396251740909 -121.94420220241294) bank124200 +124201 POINT(47.25839669825804 -121.33813773097376) bank124201 +124202 POINT(47.665051025615575 -123.32474701637378) bank124202 +124203 POINT(47.585508740311695 -122.7708637139979) bank124203 +124204 POINT(48.23739382766568 -123.10043548679842) bank124204 +124205 POINT(46.646473648169504 -122.09384848378207) bank124205 +124206 POINT(48.44587770276241 -123.14790851541186) bank124206 +124207 POINT(46.91870066973382 -122.47863438709247) bank124207 +124208 POINT(47.34961564705533 -121.80410396562698) bank124208 +124209 POINT(48.2949477506397 -123.26172678738384) bank124209 +124210 POINT(48.254106241165104 -123.29453973041194) bank124210 +124211 POINT(48.229897349132735 -121.3972024387032) bank124211 +124212 POINT(47.06738846702251 -121.78267084910645) bank124212 +124213 POINT(48.08463193654864 -122.00719048540438) bank124213 +124214 POINT(47.663763014444534 -122.54870846152141) bank124214 +124215 POINT(48.2007373252191 -123.14408144499004) bank124215 +124216 POINT(48.231052010338274 -121.93889311207167) bank124216 +124217 POINT(46.95577739655414 -122.02073129810178) bank124217 +124218 POINT(48.00347365625076 -122.27091664708557) bank124218 +124219 POINT(47.574863437987275 -121.88215232770347) bank124219 +124220 POINT(47.9156484957836 -121.60494957187119) bank124220 +124221 POINT(48.39897689494849 -121.52976427736178) bank124221 +124222 POINT(47.99339719274788 -123.25165045887685) bank124222 +124223 POINT(47.420733403017564 -122.82829312057142) bank124223 +124224 POINT(48.09754304247994 -122.71747592644931) bank124224 +124225 POINT(47.60720759942783 -123.31286628061484) bank124225 +124226 POINT(47.40089851536479 -121.58916018336244) bank124226 +124227 POINT(46.89064556065716 -121.94326764109657) bank124227 +124228 POINT(47.83534394616063 -122.46861464222566) bank124228 +124229 POINT(47.06380213502083 -122.19373984756548) bank124229 +124230 POINT(46.752622621315815 -122.4665792125187) bank124230 +124231 POINT(46.75145899766872 -121.9244286015607) bank124231 +124232 POINT(46.689553197742775 -123.05509737602816) bank124232 +124233 POINT(48.365249076792786 -122.62724981452489) bank124233 +124234 POINT(48.050274191008576 -121.79854962498548) bank124234 +124235 POINT(47.21642327882827 -123.14161616712975) bank124235 +124236 POINT(48.07456546566765 -122.3687401739642) bank124236 +124237 POINT(48.2139760176747 -122.53928071127291) bank124237 +124238 POINT(48.121992343118535 -121.89339756042722) bank124238 +124239 POINT(46.93234747414256 -122.85528520328234) bank124239 +124240 POINT(46.97828781782187 -122.55972648925409) bank124240 +124241 POINT(46.915985664080374 -121.90477272560739) bank124241 +124242 POINT(48.45150158142519 -121.72283067849209) bank124242 +124243 POINT(48.00131421857636 -122.25980373202401) bank124243 +124244 POINT(48.11111507586489 -121.77345114628042) bank124244 +124245 POINT(47.216094860188676 -121.41650920275268) bank124245 +124246 POINT(47.374629535781786 -123.03145102002243) bank124246 +124247 POINT(47.452157392690985 -122.55089406739535) bank124247 +124248 POINT(48.39909097750328 -123.2940101008032) bank124248 +124249 POINT(48.349060347384636 -121.81150355169503) bank124249 +124250 POINT(47.6762311462301 -122.00083168278566) bank124250 +124251 POINT(46.70810126922991 -122.23007835461358) bank124251 +124252 POINT(48.40517155246592 -122.3454564817924) bank124252 +124253 POINT(47.2242018180259 -122.83630761891371) bank124253 +124254 POINT(47.043518872831214 -122.9788127286502) bank124254 +124255 POINT(48.46718657159743 -123.10508800701317) bank124255 +124256 POINT(46.75388901838257 -122.92621682765892) bank124256 +124257 POINT(47.74122420547189 -122.47730231440933) bank124257 +124258 POINT(46.91801352452473 -122.01343538301356) bank124258 +124259 POINT(47.1624406416607 -122.46096315526677) bank124259 +124260 POINT(46.694297401823036 -121.76142776682758) bank124260 +124261 POINT(47.72314347617263 -122.9904846525261) bank124261 +124262 POINT(46.768374782027536 -122.66819612857199) bank124262 +124263 POINT(47.996822725187734 -122.74916325554607) bank124263 +124264 POINT(47.42809279787393 -122.60960362347588) bank124264 +124265 POINT(46.92669698880821 -122.7499315452299) bank124265 +124266 POINT(48.31429136268234 -122.40099853868121) bank124266 +124267 POINT(47.087792676180136 -122.84642642011578) bank124267 +124268 POINT(47.13407502031619 -122.76163507407043) bank124268 +124269 POINT(48.20382071928384 -122.96801415704994) bank124269 +124270 POINT(47.08261757642103 -122.74321868713675) bank124270 +124271 POINT(47.78995514674182 -121.3555264812599) bank124271 +124272 POINT(46.67773145463078 -121.93729105123418) bank124272 +124273 POINT(46.73792256027436 -122.93751011321075) bank124273 +124274 POINT(48.252811763215014 -121.87595605201354) bank124274 +124275 POINT(47.61987921542907 -123.05442949663012) bank124275 +124276 POINT(48.25587799491627 -123.21080229847445) bank124276 +124277 POINT(46.92986029344426 -121.8573199178847) bank124277 +124278 POINT(47.52555454174738 -122.24083037839863) bank124278 +124279 POINT(46.74557061837307 -121.72588987760932) bank124279 +124280 POINT(47.760734471433864 -122.50192436515201) bank124280 +124281 POINT(47.48590244248036 -121.61034368460423) bank124281 +124282 POINT(48.17050091387043 -122.18887471341576) bank124282 +124283 POINT(48.59966184427337 -121.79608534675837) bank124283 +124284 POINT(46.74163300716425 -123.01488721332237) bank124284 +124285 POINT(47.89233781658357 -121.38153082591201) bank124285 +124286 POINT(47.16471767034329 -121.89403618394029) bank124286 +124287 POINT(48.03370478850146 -122.39863496858428) bank124287 +124288 POINT(48.07438450062388 -122.32782070919335) bank124288 +124289 POINT(46.68276611703545 -121.44318469198754) bank124289 +124290 POINT(48.558915416107844 -122.10416734490536) bank124290 +124291 POINT(48.059467526675746 -122.70450447589957) bank124291 +124292 POINT(47.110400954914084 -121.42394513582707) bank124292 +124293 POINT(48.34467889261873 -121.82722752890461) bank124293 +124294 POINT(47.15211755212344 -122.13393208470256) bank124294 +124295 POINT(47.65741230512571 -121.93820254308321) bank124295 +124296 POINT(46.82601563462548 -122.34255501780113) bank124296 +124297 POINT(47.90635785712904 -122.93161932076444) bank124297 +124298 POINT(46.86224103176169 -122.83248181566908) bank124298 +124299 POINT(48.211242159596225 -123.16055761223551) bank124299 +124300 POINT(48.24069135657628 -122.91484782958193) bank124300 +124301 POINT(47.59246803224833 -122.99023519965017) bank124301 +124302 POINT(46.74999875608096 -122.00302925976582) bank124302 +124303 POINT(48.288633033552806 -121.99096347980884) bank124303 +124304 POINT(47.284012676269995 -123.11573805333957) bank124304 +124305 POINT(47.187818906856805 -121.88543735637715) bank124305 +124306 POINT(47.097731824820464 -121.42448813911088) bank124306 +124307 POINT(48.3161202468257 -121.83316564128724) bank124307 +124308 POINT(48.03739012868076 -122.02920641265175) bank124308 +124309 POINT(47.60431103563413 -121.73620674793453) bank124309 +124310 POINT(47.76042445138673 -122.79421889897567) bank124310 +124311 POINT(48.069220747057464 -122.0918503872816) bank124311 +124312 POINT(48.04369976364673 -121.74176887964722) bank124312 +124313 POINT(47.78166797331594 -121.9662704052247) bank124313 +124314 POINT(47.599950191221424 -122.59024024365706) bank124314 +124315 POINT(47.6982142365633 -121.6703031180122) bank124315 +124316 POINT(48.482225884327384 -122.49991865161337) bank124316 +124317 POINT(47.14870830864222 -123.24427282347254) bank124317 +124318 POINT(48.569786162882004 -122.82380765686057) bank124318 +124319 POINT(47.371145807137616 -123.25768613288093) bank124319 +124320 POINT(47.76290421116321 -121.99355812015774) bank124320 +124321 POINT(46.97066640655548 -121.7063673198571) bank124321 +124322 POINT(47.972508757581075 -121.42815157217403) bank124322 +124323 POINT(47.37681008056751 -122.64866260474307) bank124323 +124324 POINT(46.9049004016727 -121.41492870821442) bank124324 +124325 POINT(47.774638734088605 -121.76368462101625) bank124325 +124326 POINT(47.289787595611095 -123.11708073168528) bank124326 +124327 POINT(48.35785592531892 -123.29318223199999) bank124327 +124328 POINT(47.6943222013745 -122.66383044644871) bank124328 +124329 POINT(46.841826609141265 -122.10271674986313) bank124329 +124330 POINT(48.0880098155631 -121.50048465769585) bank124330 +124331 POINT(48.01370737842028 -123.06485113545054) bank124331 +124332 POINT(48.60459418618395 -122.18955979646859) bank124332 +124333 POINT(47.651986779909805 -121.84286774894782) bank124333 +124334 POINT(48.00655774983507 -122.7150036114422) bank124334 +124335 POINT(47.18032974882683 -121.74189118433941) bank124335 +124336 POINT(47.6394999649127 -122.23500673856138) bank124336 +124337 POINT(47.101641552569596 -121.9438015998976) bank124337 +124338 POINT(47.03511143089994 -123.24459299723584) bank124338 +124339 POINT(48.49219188046882 -122.9974337539661) bank124339 +124340 POINT(46.631283864714725 -121.44783807007268) bank124340 +124341 POINT(48.50670633381042 -121.45924193371405) bank124341 +124342 POINT(46.80528056981569 -123.32864959672554) bank124342 +124343 POINT(47.261475269563924 -121.3672437820791) bank124343 +124344 POINT(47.9700460567782 -122.29643288671663) bank124344 +124345 POINT(48.18134030403203 -122.3002179997779) bank124345 +124346 POINT(47.321716058386684 -122.97793504876336) bank124346 +124347 POINT(48.50694419624354 -122.15016475497893) bank124347 +124348 POINT(48.54665504285147 -122.99889658840551) bank124348 +124349 POINT(47.58721430932106 -121.73435827023687) bank124349 +124350 POINT(46.65222801946223 -122.49151732287449) bank124350 +124351 POINT(48.48903872444018 -123.12956544362724) bank124351 +124352 POINT(46.899572821901124 -121.33771587371756) bank124352 +124353 POINT(46.67132399125445 -121.65681107988499) bank124353 +124354 POINT(47.38283777834565 -122.77289880889838) bank124354 +124355 POINT(47.044524374616266 -123.10226177647017) bank124355 +124356 POINT(48.28426609487081 -121.78381033137413) bank124356 +124357 POINT(48.381735714589425 -121.3957231130123) bank124357 +124358 POINT(47.9646697737156 -122.37254667730723) bank124358 +124359 POINT(48.06396824988919 -122.6665209236386) bank124359 +124360 POINT(48.46809940794462 -121.95010870989047) bank124360 +124361 POINT(47.15845597826136 -122.65596297924169) bank124361 +124362 POINT(47.15811969547871 -122.36063078983891) bank124362 +124363 POINT(48.48551141184067 -122.5500717278952) bank124363 +124364 POINT(46.92168392598639 -122.88465398426463) bank124364 +124365 POINT(46.65450651131841 -121.73164878853221) bank124365 +124366 POINT(47.63605704514322 -122.5386510626229) bank124366 +124367 POINT(47.713824547586285 -121.70773201563084) bank124367 +124368 POINT(47.288375500336954 -121.93906420669941) bank124368 +124369 POINT(46.60645711970072 -121.53591446099111) bank124369 +124370 POINT(47.603661258669526 -122.68954857105676) bank124370 +124371 POINT(47.27570654919183 -122.00514181050416) bank124371 +124372 POINT(47.91814373499904 -121.55727486417341) bank124372 +124373 POINT(47.95094040165006 -121.94776491987103) bank124373 +124374 POINT(47.882611072847745 -121.53991121772691) bank124374 +124375 POINT(46.94811993528124 -121.94633143707708) bank124375 +124376 POINT(47.850959584483974 -122.44371483761763) bank124376 +124377 POINT(47.12421579406285 -121.80454750977363) bank124377 +124378 POINT(47.70060126031098 -121.75413568913669) bank124378 +124379 POINT(47.60294335454759 -122.25490019069423) bank124379 +124380 POINT(48.31760638432046 -122.43302153225979) bank124380 +124381 POINT(48.46272341090852 -122.9261771504267) bank124381 +124382 POINT(47.11169302232299 -123.11572262937958) bank124382 +124383 POINT(47.88453601062379 -122.94622365797092) bank124383 +124384 POINT(46.78573107210232 -121.60624743103376) bank124384 +124385 POINT(47.93847447696513 -121.54456046406614) bank124385 +124386 POINT(47.324890235367526 -122.99862800755267) bank124386 +124387 POINT(47.3904569824402 -121.9036990052263) bank124387 +124388 POINT(48.52187818215743 -122.48074371986579) bank124388 +124389 POINT(48.21741389370304 -122.38741200666247) bank124389 +124390 POINT(48.27709672909519 -122.75228368021736) bank124390 +124391 POINT(47.77084408443044 -121.42649577521888) bank124391 +124392 POINT(47.00006207371106 -121.83568442134582) bank124392 +124393 POINT(47.82923450065729 -121.34828873172795) bank124393 +124394 POINT(47.725965059717915 -123.00285240306972) bank124394 +124395 POINT(47.26475919728232 -122.98302601599302) bank124395 +124396 POINT(46.67145822028739 -122.49333227975801) bank124396 +124397 POINT(48.48161711725277 -123.32303851366669) bank124397 +124398 POINT(47.040011683980985 -122.48747157025731) bank124398 +124399 POINT(47.046060220389364 -121.50376108751468) bank124399 +124400 POINT(48.32998993744764 -123.206738212801) bank124400 +124401 POINT(48.45723569414519 -123.15209944354761) bank124401 +124402 POINT(48.30217072960342 -121.84004746457389) bank124402 +124403 POINT(48.1365801802051 -122.17251037260539) bank124403 +124404 POINT(47.40405827635478 -122.9242373676352) bank124404 +124405 POINT(47.95666422080255 -122.83941341097551) bank124405 +124406 POINT(48.43373640886824 -121.37453633261163) bank124406 +124407 POINT(48.01016852828746 -122.39335398688037) bank124407 +124408 POINT(47.37580641667152 -122.57820293160714) bank124408 +124409 POINT(47.02497785979295 -122.18091069767713) bank124409 +124410 POINT(47.48044510551795 -123.22823682609223) bank124410 +124411 POINT(47.16408012214244 -121.45178368637761) bank124411 +124412 POINT(47.7122805684359 -123.31504616781675) bank124412 +124413 POINT(47.13537473971204 -121.80676668907257) bank124413 +124414 POINT(47.24195021045655 -122.87362940945447) bank124414 +124415 POINT(47.68208584139063 -121.68876332323568) bank124415 +124416 POINT(47.14940090375905 -123.08299192982892) bank124416 +124417 POINT(47.88850349141024 -122.88341980176243) bank124417 +124418 POINT(46.649518089653 -122.43020702966382) bank124418 +124419 POINT(48.43231953805589 -122.00365503956692) bank124419 +124420 POINT(47.4924410304808 -121.46669628456317) bank124420 +124421 POINT(47.70115333895171 -122.61751848949463) bank124421 +124422 POINT(46.718996012017435 -123.20927258737042) bank124422 +124423 POINT(46.78991948809795 -122.84477462568492) bank124423 +124424 POINT(48.11066371619957 -122.88856589973615) bank124424 +124425 POINT(47.42884670470344 -122.61356081368862) bank124425 +124426 POINT(48.07242459047366 -122.45440673311963) bank124426 +124427 POINT(46.67068807096734 -121.76808478525514) bank124427 +124428 POINT(46.906853496458176 -122.32064802240707) bank124428 +124429 POINT(46.71158699017209 -122.82482945057593) bank124429 +124430 POINT(48.520987101073125 -121.90267943648352) bank124430 +124431 POINT(48.500452382305305 -123.13880860972338) bank124431 +124432 POINT(46.71846179640337 -122.305345515602) bank124432 +124433 POINT(47.806078913357126 -123.27712756995516) bank124433 +124434 POINT(48.126470902782664 -122.63287705783412) bank124434 +124435 POINT(47.06502367364229 -122.77352711129689) bank124435 +124436 POINT(48.24872991673276 -121.6411177217988) bank124436 +124437 POINT(47.718558063165865 -122.51377678056072) bank124437 +124438 POINT(47.0048287880125 -122.91685721399007) bank124438 +124439 POINT(48.34312717448949 -121.45603289568962) bank124439 +124440 POINT(47.55577853423575 -121.5919705494863) bank124440 +124441 POINT(46.79288306154462 -122.85173262085218) bank124441 +124442 POINT(47.47429413709504 -122.400577651435) bank124442 +124443 POINT(47.445429756054125 -122.29163610012773) bank124443 +124444 POINT(48.56213988329657 -122.84215699742154) bank124444 +124445 POINT(46.99708769064706 -121.758165695987) bank124445 +124446 POINT(47.68040849066016 -121.33851443827814) bank124446 +124447 POINT(46.954101926489855 -122.5142667965194) bank124447 +124448 POINT(47.092429198339566 -122.01762039230941) bank124448 +124449 POINT(46.702221461428586 -121.86016130058735) bank124449 +124450 POINT(48.080416814623646 -123.20437270661688) bank124450 +124451 POINT(47.06602940639902 -122.61927358755561) bank124451 +124452 POINT(47.914272569251324 -121.97741565989956) bank124452 +124453 POINT(47.65606045587514 -121.5285410195105) bank124453 +124454 POINT(47.31780448294438 -122.8676133065136) bank124454 +124455 POINT(47.84131978415047 -121.71781979149952) bank124455 +124456 POINT(48.35830613877813 -122.73913703696479) bank124456 +124457 POINT(47.13859516514976 -122.50883157664512) bank124457 +124458 POINT(46.75107251143722 -122.59746184447478) bank124458 +124459 POINT(48.05456512996546 -122.77761743816026) bank124459 +124460 POINT(48.1925849783937 -122.20364573946156) bank124460 +124461 POINT(47.889409206611084 -122.38087125576344) bank124461 +124462 POINT(47.90807078771822 -123.0849802184295) bank124462 +124463 POINT(47.90582674014482 -122.48631309639947) bank124463 +124464 POINT(47.407605940310056 -122.80755411732585) bank124464 +124465 POINT(48.02132187842155 -121.4772449046993) bank124465 +124466 POINT(47.34557040339261 -121.92231498505397) bank124466 +124467 POINT(48.28274480323806 -121.36849749271296) bank124467 +124468 POINT(47.885479480362015 -122.73441825614151) bank124468 +124469 POINT(48.074765134127915 -122.35619135665517) bank124469 +124470 POINT(47.85362005470053 -122.29143092002013) bank124470 +124471 POINT(47.137440865174746 -123.0567249773796) bank124471 +124472 POINT(47.149324687816986 -122.76274586885773) bank124472 +124473 POINT(46.80896136168798 -122.9389126876833) bank124473 +124474 POINT(48.438988698146694 -122.38684433962779) bank124474 +124475 POINT(47.96706622267725 -121.60725078982006) bank124475 +124476 POINT(48.52039503758455 -122.36358201151516) bank124476 +124477 POINT(47.12656282859756 -121.64100551849202) bank124477 +124478 POINT(48.181999124851224 -123.30850438727265) bank124478 +124479 POINT(47.585778288200366 -123.14939550224612) bank124479 +124480 POINT(46.776564131581864 -121.93049077501003) bank124480 +124481 POINT(48.37922578954094 -122.28515167441755) bank124481 +124482 POINT(47.74750844809756 -121.62730303465783) bank124482 +124483 POINT(46.90512302771965 -122.3087790142678) bank124483 +124484 POINT(47.89601291827733 -123.29634762098362) bank124484 +124485 POINT(47.52753812830482 -122.18529796424761) bank124485 +124486 POINT(46.80118939001382 -121.53655898580054) bank124486 +124487 POINT(48.46249809285152 -121.49972020951508) bank124487 +124488 POINT(47.47859867982067 -122.58936522910138) bank124488 +124489 POINT(46.796703978731216 -121.36347144519947) bank124489 +124490 POINT(47.577021332807014 -122.78450146943548) bank124490 +124491 POINT(46.72281596944688 -121.76376022771957) bank124491 +124492 POINT(47.49441005206674 -121.79727999894139) bank124492 +124493 POINT(47.20326090705019 -122.47301401087398) bank124493 +124494 POINT(48.264864277546216 -123.23669965546286) bank124494 +124495 POINT(47.73994815434986 -123.22073504982478) bank124495 +124496 POINT(47.46133503445024 -122.76220633808194) bank124496 +124497 POINT(47.793180321518136 -121.3679353559174) bank124497 +124498 POINT(48.32508442067297 -122.9027820915341) bank124498 +124499 POINT(46.95342777554295 -122.67511015726795) bank124499 +124500 POINT(47.1111674206449 -121.89512060907222) bank124500 +124501 POINT(47.50158922279892 -122.55537783647023) bank124501 +124502 POINT(47.95348812539143 -122.30572612109621) bank124502 +124503 POINT(48.13517441470849 -122.1316767513095) bank124503 +124504 POINT(48.27115026551565 -122.55082798384736) bank124504 +124505 POINT(46.85837130670659 -123.19609100197206) bank124505 +124506 POINT(47.3942401135613 -122.74517257993705) bank124506 +124507 POINT(48.2816503670368 -121.5806108228027) bank124507 +124508 POINT(46.976148561592694 -123.23668107536763) bank124508 +124509 POINT(48.13681176469989 -122.52034253464187) bank124509 +124510 POINT(47.88570067916637 -121.93840473760511) bank124510 +124511 POINT(46.86774082994146 -122.32984563479746) bank124511 +124512 POINT(47.752493831979926 -122.74403789462185) bank124512 +124513 POINT(47.99894040186366 -122.99800035301915) bank124513 +124514 POINT(46.76285290205824 -121.66012228750647) bank124514 +124515 POINT(47.81018097237187 -121.52596627328249) bank124515 +124516 POINT(47.52713499896926 -121.79676812214929) bank124516 +124517 POINT(46.93123701051874 -121.57942566039294) bank124517 +124518 POINT(48.239916519787656 -122.32638197633688) bank124518 +124519 POINT(47.923902749508066 -122.36030348636442) bank124519 +124520 POINT(47.293046477317006 -122.8797100148056) bank124520 +124521 POINT(47.39478111595456 -122.39388133230533) bank124521 +124522 POINT(46.66867897779426 -121.43202261792848) bank124522 +124523 POINT(48.08064409029963 -121.8480107528881) bank124523 +124524 POINT(46.87336728406578 -122.86008970591621) bank124524 +124525 POINT(46.73997918445441 -121.63420766909327) bank124525 +124526 POINT(47.73405891004544 -123.09433480715673) bank124526 +124527 POINT(48.51536375202671 -122.67857260740244) bank124527 +124528 POINT(47.80984022507196 -123.23023273632248) bank124528 +124529 POINT(48.230977931345464 -123.14450386029945) bank124529 +124530 POINT(46.91740707173062 -121.95169596391263) bank124530 +124531 POINT(48.15509215617326 -123.10138211885972) bank124531 +124532 POINT(48.4820491358227 -121.50465432295424) bank124532 +124533 POINT(47.026456577864515 -123.01082823959548) bank124533 +124534 POINT(47.02318980143255 -121.57092813825699) bank124534 +124535 POINT(46.737068382369955 -121.84068340384529) bank124535 +124536 POINT(46.74228722625634 -122.6561322746571) bank124536 +124537 POINT(47.08903845585027 -123.25143915397756) bank124537 +124538 POINT(46.95799916536787 -122.01686928216148) bank124538 +124539 POINT(48.27707961242183 -121.7853314369252) bank124539 +124540 POINT(47.03230854356079 -121.5437030900923) bank124540 +124541 POINT(46.64368268658382 -122.99509785304987) bank124541 +124542 POINT(47.179136197483 -123.00109990550943) bank124542 +124543 POINT(48.19536923450303 -122.79544641095642) bank124543 +124544 POINT(48.50886913327681 -121.91881010107336) bank124544 +124545 POINT(47.70067537213317 -121.36694453158869) bank124545 +124546 POINT(46.83751723937081 -123.09792456336562) bank124546 +124547 POINT(46.80610460895152 -121.87231624850754) bank124547 +124548 POINT(47.6162041789015 -121.55239260295643) bank124548 +124549 POINT(47.8178596002679 -123.234176323174) bank124549 +124550 POINT(47.318135313228815 -122.53025597149758) bank124550 +124551 POINT(46.70101310830107 -122.14767110584526) bank124551 +124552 POINT(48.17204292599675 -122.61910716330141) bank124552 +124553 POINT(47.97678259225217 -122.89375691457364) bank124553 +124554 POINT(48.51269374002449 -122.03656744354123) bank124554 +124555 POINT(48.386758102939595 -121.52502154719387) bank124555 +124556 POINT(48.59065933420563 -121.74668703351803) bank124556 +124557 POINT(48.24637416426677 -122.4112860383418) bank124557 +124558 POINT(48.52094724467603 -122.35924523693768) bank124558 +124559 POINT(47.02962916584381 -123.18306359993343) bank124559 +124560 POINT(47.83679177776715 -122.86588075133777) bank124560 +124561 POINT(48.53545324268213 -122.49270112595875) bank124561 +124562 POINT(47.82846868695197 -122.88830057042492) bank124562 +124563 POINT(47.00640902356333 -123.2032040878138) bank124563 +124564 POINT(48.58044984470244 -122.23977153438969) bank124564 +124565 POINT(48.59355155960441 -121.53815847379677) bank124565 +124566 POINT(47.82391847337532 -121.58052147738287) bank124566 +124567 POINT(48.035625291988225 -122.31733251046442) bank124567 +124568 POINT(47.80191699982865 -122.31018277440198) bank124568 +124569 POINT(46.61714325789945 -123.18105991167465) bank124569 +124570 POINT(47.18893808413528 -122.43034585860146) bank124570 +124571 POINT(48.3306331503292 -121.64915075863873) bank124571 +124572 POINT(48.59087602713044 -122.67390570623432) bank124572 +124573 POINT(47.16590123554016 -122.69861407688396) bank124573 +124574 POINT(47.982970251356704 -121.75057212377192) bank124574 +124575 POINT(48.33967799976506 -122.46367849309313) bank124575 +124576 POINT(47.470570108506486 -121.49143335432862) bank124576 +124577 POINT(47.513119409036015 -122.9417024247088) bank124577 +124578 POINT(46.925987658213 -121.69386131059296) bank124578 +124579 POINT(48.59634285155699 -122.27883672023682) bank124579 +124580 POINT(47.89111194831504 -122.76033506507702) bank124580 +124581 POINT(48.19409264888057 -122.5535252876637) bank124581 +124582 POINT(47.64124791864553 -121.95688191650653) bank124582 +124583 POINT(47.67844967057033 -123.05586505873535) bank124583 +124584 POINT(48.50479407670782 -121.66533610260362) bank124584 +124585 POINT(46.89462290553839 -122.97492213283222) bank124585 +124586 POINT(48.35606444579036 -123.05428109774866) bank124586 +124587 POINT(47.30216348181705 -121.89208353891983) bank124587 +124588 POINT(47.22806018683328 -123.23822764979134) bank124588 +124589 POINT(47.50979591332171 -122.36984269808077) bank124589 +124590 POINT(48.36014431698692 -123.31497641164488) bank124590 +124591 POINT(48.57655268103099 -123.09879997283325) bank124591 +124592 POINT(47.13994365554664 -122.0488605234152) bank124592 +124593 POINT(47.31220772492306 -121.99724489812819) bank124593 +124594 POINT(47.876856766851304 -121.4079764036898) bank124594 +124595 POINT(48.58913265634934 -122.93200147063754) bank124595 +124596 POINT(48.39231225758937 -122.19093584279003) bank124596 +124597 POINT(46.673104510954445 -122.69574452645057) bank124597 +124598 POINT(47.47799663525259 -122.18690920556021) bank124598 +124599 POINT(47.535800986579986 -122.5541697156111) bank124599 +124600 POINT(47.34288556576921 -121.64832770135314) bank124600 +124601 POINT(47.61251197216635 -122.36630483955855) bank124601 +124602 POINT(47.869238067478015 -122.94016701018198) bank124602 +124603 POINT(47.53887590183129 -121.92879244114582) bank124603 +124604 POINT(47.65442152700932 -121.72357209235163) bank124604 +124605 POINT(47.994916452671625 -122.05969141406452) bank124605 +124606 POINT(46.729581048833474 -122.11129705727672) bank124606 +124607 POINT(47.187319271192955 -123.05239546054136) bank124607 +124608 POINT(48.1806911377705 -122.24363433238113) bank124608 +124609 POINT(48.055088982328954 -121.42487986469264) bank124609 +124610 POINT(46.94803674165358 -122.27813902974178) bank124610 +124611 POINT(46.672868993863624 -122.86517907251968) bank124611 +124612 POINT(46.81810781112296 -122.45623807268936) bank124612 +124613 POINT(46.79181176280434 -122.3410730719247) bank124613 +124614 POINT(47.2827821293771 -123.24781843509629) bank124614 +124615 POINT(47.62722799790273 -121.87136498825515) bank124615 +124616 POINT(46.7888452727253 -122.80903462825988) bank124616 +124617 POINT(47.9771537637989 -122.40312621574678) bank124617 +124618 POINT(48.12723819730655 -122.10579904189672) bank124618 +124619 POINT(47.26415819701345 -123.08517798219555) bank124619 +124620 POINT(48.567839557307735 -122.67927490378851) bank124620 +124621 POINT(48.58104185366672 -121.3461186868248) bank124621 +124622 POINT(46.97798538270953 -122.24017985999288) bank124622 +124623 POINT(48.33447248230336 -121.55401527355743) bank124623 +124624 POINT(47.78789191747341 -122.61288854096296) bank124624 +124625 POINT(46.780738450844986 -122.83905111426874) bank124625 +124626 POINT(47.460908416213165 -122.95093837718105) bank124626 +124627 POINT(47.86779201153621 -121.62006675699892) bank124627 +124628 POINT(47.22144435069294 -121.8858115654134) bank124628 +124629 POINT(47.84032553832341 -122.41638004394123) bank124629 +124630 POINT(46.97201491065757 -122.69950920902284) bank124630 +124631 POINT(48.36660549883859 -122.94368496008981) bank124631 +124632 POINT(48.33360336559964 -122.8568957533899) bank124632 +124633 POINT(48.52396818633498 -123.19933004804336) bank124633 +124634 POINT(47.10086993543506 -122.31025640719527) bank124634 +124635 POINT(46.73089245215928 -123.17953662829628) bank124635 +124636 POINT(47.37423507741047 -121.9499706386396) bank124636 +124637 POINT(47.81298045297239 -121.96280486625669) bank124637 +124638 POINT(47.2826421103716 -123.28031397675566) bank124638 +124639 POINT(48.51369738136785 -121.98931708366536) bank124639 +124640 POINT(47.976402186343705 -122.11720363232779) bank124640 +124641 POINT(46.766749024246465 -122.38478318705748) bank124641 +124642 POINT(48.348940380678016 -123.19847514982747) bank124642 +124643 POINT(47.635666378115644 -121.97680519981824) bank124643 +124644 POINT(47.68145418582326 -122.27329734934213) bank124644 +124645 POINT(46.60648791027429 -122.84971246636688) bank124645 +124646 POINT(47.64803121490723 -122.03086041048162) bank124646 +124647 POINT(48.21458922162677 -121.40074646257858) bank124647 +124648 POINT(48.06607417109808 -122.40465616349756) bank124648 +124649 POINT(47.564181532336214 -122.16559473385925) bank124649 +124650 POINT(47.13678148968806 -123.0824767911272) bank124650 +124651 POINT(46.72075816044974 -122.06683993834311) bank124651 +124652 POINT(47.30194904103555 -121.66672147833381) bank124652 +124653 POINT(48.14084132147176 -121.50927367942457) bank124653 +124654 POINT(46.775390321465736 -122.44722349348119) bank124654 +124655 POINT(48.06004268606878 -121.84349402842427) bank124655 +124656 POINT(47.254737026463374 -122.2805951929551) bank124656 +124657 POINT(46.85692123692863 -121.55751997263353) bank124657 +124658 POINT(46.753313290298856 -121.38828433550621) bank124658 +124659 POINT(46.741778566911265 -122.18468972844843) bank124659 +124660 POINT(48.1677037676097 -121.92399832888626) bank124660 +124661 POINT(48.4506978532767 -122.87932744085046) bank124661 +124662 POINT(48.21687031361523 -121.71543810927773) bank124662 +124663 POINT(46.69729982560968 -121.46143917715368) bank124663 +124664 POINT(47.731910026115216 -121.73789958022556) bank124664 +124665 POINT(48.05060843915613 -121.60659057362429) bank124665 +124666 POINT(46.702114855726904 -122.48551231610026) bank124666 +124667 POINT(47.694060253688576 -122.80549925902362) bank124667 +124668 POINT(48.29905964374329 -122.5665968700765) bank124668 +124669 POINT(47.23194553727038 -122.02414617596511) bank124669 +124670 POINT(48.60297076888423 -122.12153962547734) bank124670 +124671 POINT(47.89813603354085 -122.42675323968406) bank124671 +124672 POINT(47.9052710543266 -121.5775581281455) bank124672 +124673 POINT(46.6158248703265 -121.81068376151842) bank124673 +124674 POINT(47.36537268766765 -122.23643306969616) bank124674 +124675 POINT(47.881474056880805 -122.86034086556545) bank124675 +124676 POINT(46.82200031536016 -122.09665624324033) bank124676 +124677 POINT(46.65193409249136 -122.7968745671587) bank124677 +124678 POINT(47.13145177329595 -121.87884645960295) bank124678 +124679 POINT(47.46270005476532 -122.04962164011923) bank124679 +124680 POINT(48.184145329938495 -122.62013882769892) bank124680 +124681 POINT(47.69221082876022 -123.20602399813154) bank124681 +124682 POINT(46.874191021956676 -121.44337850010183) bank124682 +124683 POINT(48.456705261773095 -121.70535340779323) bank124683 +124684 POINT(48.15674587864499 -121.84835877077596) bank124684 +124685 POINT(46.962979713958696 -122.17650394352063) bank124685 +124686 POINT(48.3443919785371 -122.80801787178953) bank124686 +124687 POINT(48.19564316923168 -123.07434402161795) bank124687 +124688 POINT(48.38671425206331 -121.38244606275714) bank124688 +124689 POINT(46.797698307679916 -122.49119820727631) bank124689 +124690 POINT(48.454576661353244 -121.96237651242629) bank124690 +124691 POINT(47.03477232086787 -121.77289700517781) bank124691 +124692 POINT(48.22593746517887 -123.12091060920538) bank124692 +124693 POINT(47.14634455092948 -122.83972271307377) bank124693 +124694 POINT(48.29395572812762 -122.17862066475443) bank124694 +124695 POINT(47.96370335754223 -122.16360220810564) bank124695 +124696 POINT(46.75291851305912 -122.24581643710981) bank124696 +124697 POINT(48.20052379042691 -122.34103797109262) bank124697 +124698 POINT(47.86042977106374 -122.9155222270324) bank124698 +124699 POINT(47.674994302769555 -121.66128875796115) bank124699 +124700 POINT(47.962018359419254 -122.92936658738766) bank124700 +124701 POINT(47.651780092896274 -122.61177258135264) bank124701 +124702 POINT(47.20391872416004 -122.47515012214186) bank124702 +124703 POINT(46.743164219676565 -123.12373994546256) bank124703 +124704 POINT(47.25883201923841 -123.18935535858104) bank124704 +124705 POINT(46.937597694960715 -122.9655018998306) bank124705 +124706 POINT(48.15495489175957 -123.00300513880715) bank124706 +124707 POINT(47.51099920099848 -122.90120624348012) bank124707 +124708 POINT(47.6362371350707 -123.07650886046584) bank124708 +124709 POINT(48.50046390374639 -122.04267697535563) bank124709 +124710 POINT(48.381601544850426 -121.36110838160414) bank124710 +124711 POINT(47.3356846759282 -123.27220922712478) bank124711 +124712 POINT(47.96667044061806 -122.38213145923567) bank124712 +124713 POINT(47.87037826322111 -121.53053476081651) bank124713 +124714 POINT(47.997524965287575 -123.24834633289232) bank124714 +124715 POINT(47.196002882180835 -122.54817183269223) bank124715 +124716 POINT(47.06233846896349 -123.31858307654694) bank124716 +124717 POINT(47.444637757231966 -121.8090399378223) bank124717 +124718 POINT(47.182984719195346 -123.04524195390836) bank124718 +124719 POINT(48.458461853284675 -122.80340645671987) bank124719 +124720 POINT(48.17407677319562 -123.28708889106998) bank124720 +124721 POINT(46.80010254058217 -122.89372037125241) bank124721 +124722 POINT(46.896388601008795 -121.67484185560383) bank124722 +124723 POINT(47.37714946124518 -122.9456771675974) bank124723 +124724 POINT(48.57681420445399 -122.60819543010162) bank124724 +124725 POINT(48.226384206050795 -121.65914363002705) bank124725 +124726 POINT(47.63670428032484 -122.6233881897212) bank124726 +124727 POINT(47.676199596439375 -122.51965991119268) bank124727 +124728 POINT(48.061459085181 -121.59479751969297) bank124728 +124729 POINT(46.65890232057171 -122.22697096158997) bank124729 +124730 POINT(47.442720642471286 -122.9625143454342) bank124730 +124731 POINT(47.01416969181884 -123.11162496441557) bank124731 +124732 POINT(48.398827343952306 -121.78044655813554) bank124732 +124733 POINT(47.765854522570585 -122.15521947244866) bank124733 +124734 POINT(48.3128293436654 -122.13474261367476) bank124734 +124735 POINT(47.95321889943406 -122.40624471791055) bank124735 +124736 POINT(48.51871801035561 -123.06465393598498) bank124736 +124737 POINT(47.81159117062425 -121.69152891586718) bank124737 +124738 POINT(48.299121201215215 -122.92607021665818) bank124738 +124739 POINT(47.54198603069578 -122.05945399597802) bank124739 +124740 POINT(48.1476977235369 -122.35411155112052) bank124740 +124741 POINT(46.73313120367045 -122.50218342636256) bank124741 +124742 POINT(47.34489344774377 -122.0404081186135) bank124742 +124743 POINT(48.51009469910849 -122.42156284676301) bank124743 +124744 POINT(48.040348703466336 -122.3873917611121) bank124744 +124745 POINT(46.95920510341738 -121.80081873369488) bank124745 +124746 POINT(46.770649496209856 -122.98667277088906) bank124746 +124747 POINT(47.15779350450205 -121.38874870368284) bank124747 +124748 POINT(46.98835526940505 -121.42559180386343) bank124748 +124749 POINT(48.162480706513264 -122.9116697364323) bank124749 +124750 POINT(48.26336351495181 -121.33838137140542) bank124750 +124751 POINT(48.398646635744605 -123.26807056621314) bank124751 +124752 POINT(47.46525704882529 -121.87152612640278) bank124752 +124753 POINT(46.99804574172048 -122.75131675589492) bank124753 +124754 POINT(48.11993057442291 -121.97237992245734) bank124754 +124755 POINT(48.10814377942235 -122.00200625076961) bank124755 +124756 POINT(46.75885828186011 -122.9920012620772) bank124756 +124757 POINT(47.38076152453078 -121.51929319213056) bank124757 +124758 POINT(46.92279214600104 -122.52670402316518) bank124758 +124759 POINT(46.83414120719024 -121.87226010085337) bank124759 +124760 POINT(47.022730378095616 -121.65236796064322) bank124760 +124761 POINT(46.95452628109927 -122.34275535934972) bank124761 +124762 POINT(47.29371226949377 -122.79706412674307) bank124762 +124763 POINT(47.29245107535356 -122.66155304126383) bank124763 +124764 POINT(47.688424817489896 -122.93196143123855) bank124764 +124765 POINT(47.98220964519073 -123.32950078742648) bank124765 +124766 POINT(47.220534059268005 -122.3636613955177) bank124766 +124767 POINT(48.36217986549324 -122.80230511957986) bank124767 +124768 POINT(47.48732615391595 -123.16576034871171) bank124768 +124769 POINT(46.872976309012806 -121.85093223692587) bank124769 +124770 POINT(46.64426925995723 -122.75621359563291) bank124770 +124771 POINT(48.58247929035828 -122.03542865754655) bank124771 +124772 POINT(48.464915726368 -123.20375112594922) bank124772 +124773 POINT(47.94584529599691 -122.70736648363659) bank124773 +124774 POINT(47.57696986122412 -122.60452648279167) bank124774 +124775 POINT(46.880610164118515 -121.78290122455373) bank124775 +124776 POINT(46.7176242915847 -122.49326077885696) bank124776 +124777 POINT(46.79372331486964 -122.0815297656959) bank124777 +124778 POINT(46.75215898999667 -122.39856349067551) bank124778 +124779 POINT(47.63060319372711 -122.87536761975082) bank124779 +124780 POINT(48.46513161404271 -122.50965565317411) bank124780 +124781 POINT(48.515224462985806 -122.94977123238306) bank124781 +124782 POINT(47.72855268811987 -122.2547714550397) bank124782 +124783 POINT(47.44050175438602 -122.41856836013342) bank124783 +124784 POINT(47.70059114393399 -122.87862051666421) bank124784 +124785 POINT(47.79335429428042 -121.94559434093614) bank124785 +124786 POINT(48.27243911990057 -122.70816012990568) bank124786 +124787 POINT(47.77569208884194 -121.46846305259379) bank124787 +124788 POINT(48.38620197838025 -122.9725855474456) bank124788 +124789 POINT(47.82833223199959 -123.1193954002131) bank124789 +124790 POINT(48.45204893113255 -121.7329474336984) bank124790 +124791 POINT(47.59528830523887 -122.1527099847591) bank124791 +124792 POINT(48.04547993059671 -122.51898362675713) bank124792 +124793 POINT(46.790973577869615 -121.69136718648245) bank124793 +124794 POINT(47.16443266760969 -122.68935151589142) bank124794 +124795 POINT(48.44200479745033 -122.82359593047364) bank124795 +124796 POINT(48.483490581905414 -122.30309112513757) bank124796 +124797 POINT(47.20514068148952 -123.27961040339713) bank124797 +124798 POINT(47.532966959938285 -121.8692853988597) bank124798 +124799 POINT(47.71182903741119 -122.9625600736085) bank124799 +124800 POINT(47.75517429413728 -122.69593119654078) bank124800 +124801 POINT(48.375130426022594 -122.25847391557124) bank124801 +124802 POINT(47.27982218633069 -122.35114754487854) bank124802 +124803 POINT(48.50330494604302 -121.96662959663003) bank124803 +124804 POINT(48.56993470569826 -123.08486453620051) bank124804 +124805 POINT(47.390514638287996 -123.13036664059288) bank124805 +124806 POINT(46.69702840731964 -121.33414558634226) bank124806 +124807 POINT(48.104430410185664 -122.19926631668962) bank124807 +124808 POINT(47.45082075238587 -122.42436376260032) bank124808 +124809 POINT(48.24021522340985 -121.64818235469501) bank124809 +124810 POINT(47.21252692128228 -122.71156244637325) bank124810 +124811 POINT(48.545278155481206 -122.18338127041864) bank124811 +124812 POINT(47.81392973354257 -122.07387979240096) bank124812 +124813 POINT(46.84599837198427 -121.70816590075896) bank124813 +124814 POINT(48.215283375192776 -121.53660366474283) bank124814 +124815 POINT(47.018689100225934 -121.57656943267756) bank124815 +124816 POINT(48.38047150837666 -122.71569817886487) bank124816 +124817 POINT(46.63329594361661 -121.52577766991084) bank124817 +124818 POINT(47.9506827568139 -123.19085766158094) bank124818 +124819 POINT(47.845120600086126 -121.45003912245427) bank124819 +124820 POINT(47.542849717398596 -122.89957658667709) bank124820 +124821 POINT(48.196480357357046 -122.89943462003903) bank124821 +124822 POINT(47.042091458712626 -121.82893750068276) bank124822 +124823 POINT(48.24612928712819 -123.00344499212116) bank124823 +124824 POINT(46.89874795062383 -122.53639782927122) bank124824 +124825 POINT(48.57227511648952 -123.30985261818343) bank124825 +124826 POINT(47.50866671467248 -122.18472980768924) bank124826 +124827 POINT(48.062244532457996 -121.9593542423172) bank124827 +124828 POINT(46.8454371191087 -122.30581129932102) bank124828 +124829 POINT(47.073880052557094 -122.90273266152722) bank124829 +124830 POINT(47.705237949160356 -122.40439803591393) bank124830 +124831 POINT(47.492991959383396 -122.00303637836284) bank124831 +124832 POINT(47.79946040366886 -121.52392942240091) bank124832 +124833 POINT(46.90971319459517 -123.15027412228402) bank124833 +124834 POINT(48.29686401726046 -121.46552177113875) bank124834 +124835 POINT(47.3487150892747 -121.46933203808914) bank124835 +124836 POINT(47.829272388526675 -121.86571670345322) bank124836 +124837 POINT(47.50351285057828 -123.02338400329954) bank124837 +124838 POINT(47.05874323117776 -122.48756192523989) bank124838 +124839 POINT(47.06505889654964 -121.36835791108778) bank124839 +124840 POINT(47.21948077531181 -121.56334592323839) bank124840 +124841 POINT(48.564464559226074 -121.53713867711663) bank124841 +124842 POINT(48.398709835398755 -122.23113106051463) bank124842 +124843 POINT(46.75003458366588 -122.00176936362678) bank124843 +124844 POINT(48.42739328763753 -123.27048796258993) bank124844 +124845 POINT(47.79988738305584 -123.07715373243272) bank124845 +124846 POINT(48.26064471338566 -122.49553156514253) bank124846 +124847 POINT(47.65985592344709 -122.90307936160568) bank124847 +124848 POINT(46.8688857614109 -122.15993564111257) bank124848 +124849 POINT(48.53314217321437 -122.97032208276546) bank124849 +124850 POINT(48.27152258324758 -123.29892138521038) bank124850 +124851 POINT(48.47733506615944 -122.66913120688265) bank124851 +124852 POINT(48.227193470383405 -122.64086849990849) bank124852 +124853 POINT(48.126555086966334 -121.35982863106543) bank124853 +124854 POINT(47.490353918304706 -122.95973403419934) bank124854 +124855 POINT(47.73727409315996 -121.94015434805578) bank124855 +124856 POINT(47.88279680944029 -122.83795908321149) bank124856 +124857 POINT(47.09650916236089 -122.83946273989714) bank124857 +124858 POINT(48.28455374937307 -121.64795208254411) bank124858 +124859 POINT(47.399037500915036 -122.05551320060195) bank124859 +124860 POINT(48.30382719624128 -121.9104295584937) bank124860 +124861 POINT(48.39070842028105 -122.11562474367314) bank124861 +124862 POINT(47.07225311787791 -121.85621286014212) bank124862 +124863 POINT(48.08316721197675 -122.7944744811899) bank124863 +124864 POINT(47.802031130589185 -121.79828722038069) bank124864 +124865 POINT(47.238675918049324 -121.35138385725253) bank124865 +124866 POINT(47.974655805681245 -122.3619989368176) bank124866 +124867 POINT(46.94339966103418 -123.02353852835088) bank124867 +124868 POINT(48.34157081240331 -122.71275217663845) bank124868 +124869 POINT(48.20091335727542 -122.85705395816946) bank124869 +124870 POINT(47.744895717839505 -122.20038515752843) bank124870 +124871 POINT(48.269954689698025 -121.45770407839005) bank124871 +124872 POINT(47.04955892368927 -121.80739889561153) bank124872 +124873 POINT(47.163472161338056 -122.82179699974033) bank124873 +124874 POINT(47.02684920363928 -123.27288094804673) bank124874 +124875 POINT(47.01917821335747 -121.83743930142509) bank124875 +124876 POINT(48.39316222275004 -121.86140402070642) bank124876 +124877 POINT(47.64761894555179 -121.97337201443223) bank124877 +124878 POINT(46.89829427396092 -123.30030084265971) bank124878 +124879 POINT(47.39694394109082 -122.61132889947473) bank124879 +124880 POINT(46.61147209476086 -121.45661717898135) bank124880 +124881 POINT(47.10471430899656 -121.9304860823159) bank124881 +124882 POINT(48.33084024949568 -121.70175841460316) bank124882 +124883 POINT(47.83332239080787 -122.38779874815509) bank124883 +124884 POINT(47.126206133187914 -123.28586373727865) bank124884 +124885 POINT(48.38347075542985 -121.62330720060861) bank124885 +124886 POINT(48.38287038029652 -122.20742573354481) bank124886 +124887 POINT(46.977623613843 -122.31939971618705) bank124887 +124888 POINT(48.56488543525693 -122.43162637091164) bank124888 +124889 POINT(47.642311529523894 -122.60762472018558) bank124889 +124890 POINT(46.801136503107124 -121.43946936737316) bank124890 +124891 POINT(47.22943954498361 -123.1070894339315) bank124891 +124892 POINT(47.21186435920106 -121.78361666474392) bank124892 +124893 POINT(48.5907685173448 -121.52813932757195) bank124893 +124894 POINT(48.493231372002064 -122.44247391790465) bank124894 +124895 POINT(47.91501980403701 -122.20110155283105) bank124895 +124896 POINT(46.84283882052812 -121.63399432844524) bank124896 +124897 POINT(47.28674268985199 -121.54938646551366) bank124897 +124898 POINT(47.16686885722114 -121.67425892005006) bank124898 +124899 POINT(48.25304642596543 -121.77706122172769) bank124899 +124900 POINT(47.62523207461534 -123.15589582262265) bank124900 +124901 POINT(47.683186949508176 -121.7929652329206) bank124901 +124902 POINT(48.236813490776 -122.57195223900248) bank124902 +124903 POINT(48.04457579969153 -122.89861283493106) bank124903 +124904 POINT(46.979681168937354 -122.99545328452787) bank124904 +124905 POINT(46.64917205562551 -123.28914494490651) bank124905 +124906 POINT(47.626371477005264 -122.08478055422081) bank124906 +124907 POINT(46.66911180512007 -122.83635030244807) bank124907 +124908 POINT(46.8623325737296 -122.28227520995323) bank124908 +124909 POINT(48.177878859942965 -123.06754601535644) bank124909 +124910 POINT(47.898083264939935 -122.19002648186965) bank124910 +124911 POINT(48.59790231314507 -121.61767254150568) bank124911 +124912 POINT(47.473137163326676 -122.68390328449071) bank124912 +124913 POINT(47.85849502150245 -122.59117188616861) bank124913 +124914 POINT(48.09581294619749 -122.73532692816742) bank124914 +124915 POINT(48.31702175656376 -123.04154900918573) bank124915 +124916 POINT(47.3735423152709 -122.04755373843274) bank124916 +124917 POINT(47.54522855884417 -122.88523408304485) bank124917 +124918 POINT(48.48599245246995 -121.42283551628077) bank124918 +124919 POINT(48.37558942077547 -121.67211619053883) bank124919 +124920 POINT(46.788046518150814 -123.2704941219159) bank124920 +124921 POINT(47.77276054750502 -122.43707678310456) bank124921 +124922 POINT(47.8952044057179 -122.93058978283949) bank124922 +124923 POINT(48.32025195728588 -123.00987474907902) bank124923 +124924 POINT(48.22312737138877 -121.88913189695074) bank124924 +124925 POINT(46.817395727463435 -121.84128675394645) bank124925 +124926 POINT(47.787214929446904 -121.5300047071197) bank124926 +124927 POINT(47.13429782563996 -122.70392840306353) bank124927 +124928 POINT(47.78769186375059 -123.19928988919597) bank124928 +124929 POINT(47.141613074572824 -122.02684434865658) bank124929 +124930 POINT(48.13733292652481 -122.6171080165926) bank124930 +124931 POINT(46.97050199230599 -121.51354654158565) bank124931 +124932 POINT(46.956222903416396 -122.36119930743543) bank124932 +124933 POINT(47.18160309846078 -122.08950957896108) bank124933 +124934 POINT(48.569261422560885 -122.48929086845511) bank124934 +124935 POINT(47.459223073936606 -121.98468751532114) bank124935 +124936 POINT(48.23919313515161 -121.96699558736559) bank124936 +124937 POINT(47.3387089192884 -122.29578962982947) bank124937 +124938 POINT(48.48084494404075 -123.07192776283334) bank124938 +124939 POINT(48.483010267918935 -122.52138954044065) bank124939 +124940 POINT(48.16024363572336 -121.68163907353673) bank124940 +124941 POINT(47.122321089319826 -122.66055119318685) bank124941 +124942 POINT(47.37360759786191 -122.34240678695275) bank124942 +124943 POINT(47.35853275162655 -122.0432671007605) bank124943 +124944 POINT(47.17544315348906 -123.2431966303306) bank124944 +124945 POINT(46.995207999920254 -122.33971028338439) bank124945 +124946 POINT(47.71376942017005 -122.97464943205178) bank124946 +124947 POINT(47.72013689125243 -123.06803405140245) bank124947 +124948 POINT(46.858382424854234 -122.44286335495225) bank124948 +124949 POINT(47.4854529738656 -122.49608003842239) bank124949 +124950 POINT(47.73829037008974 -122.23478178075366) bank124950 +124951 POINT(47.60987477149901 -123.3093614632759) bank124951 +124952 POINT(47.27927217445862 -122.64288023761814) bank124952 +124953 POINT(46.68268369841123 -123.29386310775091) bank124953 +124954 POINT(47.77618779981309 -122.3534330559348) bank124954 +124955 POINT(48.33092435802761 -121.56658084429107) bank124955 +124956 POINT(47.473640198032854 -121.78102526885854) bank124956 +124957 POINT(47.19775408592994 -123.11639350306562) bank124957 +124958 POINT(47.6980517454171 -121.39221616061585) bank124958 +124959 POINT(48.226439862320134 -121.96482014491568) bank124959 +124960 POINT(47.746613292351654 -121.82194866912643) bank124960 +124961 POINT(47.83447977860036 -122.36677655480136) bank124961 +124962 POINT(48.28590301410226 -122.91866167169171) bank124962 +124963 POINT(47.2666828787461 -123.28580816947762) bank124963 +124964 POINT(47.83878587004125 -123.27855253633798) bank124964 +124965 POINT(48.24427146578595 -122.83909515791636) bank124965 +124966 POINT(47.828920354046794 -122.68617430599332) bank124966 +124967 POINT(48.5327977050694 -122.12621927408802) bank124967 +124968 POINT(48.53163687516676 -122.41631925913609) bank124968 +124969 POINT(48.29794512090402 -122.52610718985167) bank124969 +124970 POINT(47.745096418262825 -123.24811250874654) bank124970 +124971 POINT(46.84162778975773 -123.18774540816571) bank124971 +124972 POINT(46.74233824430651 -121.92970587189456) bank124972 +124973 POINT(47.730312053267504 -121.4544117648891) bank124973 +124974 POINT(48.39483340199496 -123.14802844481576) bank124974 +124975 POINT(48.33005505036698 -122.52197626467803) bank124975 +124976 POINT(47.19648234842687 -122.9581398534056) bank124976 +124977 POINT(48.300554884710934 -121.93136377079709) bank124977 +124978 POINT(47.199872441489624 -121.47920650334018) bank124978 +124979 POINT(48.261432479858584 -121.79048932335925) bank124979 +124980 POINT(47.833501130672374 -121.85639537676947) bank124980 +124981 POINT(47.65940200423316 -121.37638297598038) bank124981 +124982 POINT(48.1954280884976 -122.73141795877095) bank124982 +124983 POINT(48.56386318147382 -122.28835677120409) bank124983 +124984 POINT(48.20926300943809 -121.8145612596345) bank124984 +124985 POINT(48.01910349737446 -122.16613123281377) bank124985 +124986 POINT(48.15817174384524 -122.28678133426332) bank124986 +124987 POINT(47.33234056152221 -121.9733281727136) bank124987 +124988 POINT(47.961951722562915 -121.5955049428069) bank124988 +124989 POINT(46.77263180489064 -122.95135255866246) bank124989 +124990 POINT(48.31558901457751 -121.86781044108952) bank124990 +124991 POINT(47.47306459934027 -122.08993686192053) bank124991 +124992 POINT(47.1034549625759 -121.57657162988491) bank124992 +124993 POINT(47.4079322469271 -123.06843086330736) bank124993 +124994 POINT(47.119536607774194 -121.58239242133946) bank124994 +124995 POINT(47.19955293749417 -122.81359951050683) bank124995 +124996 POINT(47.97832593793946 -122.68178298208521) bank124996 +124997 POINT(48.58648800865188 -122.17757684598524) bank124997 +124998 POINT(47.079545138962835 -123.07752161646133) bank124998 +124999 POINT(48.262958402313814 -121.83965029647145) bank124999 +125000 POINT(47.46089588719113 -121.77566617535297) bank125000 +125001 POINT(48.22462612429649 -121.80885128335217) bank125001 +125002 POINT(48.09747287326748 -121.8340829979309) bank125002 +125003 POINT(48.30984490344763 -122.07091759287204) bank125003 +125004 POINT(47.25273159469475 -123.30088793883004) bank125004 +125005 POINT(46.978254665009686 -122.9534010534051) bank125005 +125006 POINT(47.877545979845706 -122.41976757819138) bank125006 +125007 POINT(48.135869546209136 -121.67284321112858) bank125007 +125008 POINT(48.45097714701557 -123.02399679127531) bank125008 +125009 POINT(47.34181609069928 -121.41871030160918) bank125009 +125010 POINT(47.99954937401384 -123.00837680479839) bank125010 +125011 POINT(48.53220903112803 -122.54927437305798) bank125011 +125012 POINT(48.30786226623379 -122.46438474726946) bank125012 +125013 POINT(47.459572383262405 -122.92486426447924) bank125013 +125014 POINT(47.50746507126894 -121.90136089252417) bank125014 +125015 POINT(48.0885587743637 -121.38084511416719) bank125015 +125016 POINT(46.6744680373102 -123.31375693478547) bank125016 +125017 POINT(47.49411307761447 -121.925861275794) bank125017 +125018 POINT(47.40891227056667 -122.18281131073742) bank125018 +125019 POINT(46.965366123588886 -121.40928138645263) bank125019 +125020 POINT(47.831691076893904 -123.13430257001139) bank125020 +125021 POINT(46.752213007374976 -123.04209318163664) bank125021 +125022 POINT(48.0626022047371 -122.2797100766881) bank125022 +125023 POINT(46.91352918410153 -122.90044749123605) bank125023 +125024 POINT(47.00619373628977 -121.34576489609898) bank125024 +125025 POINT(47.73286935746516 -122.31980093296033) bank125025 +125026 POINT(48.057545113018776 -121.92178279301673) bank125026 +125027 POINT(48.17303859253742 -123.15103312414455) bank125027 +125028 POINT(46.94060169703472 -122.96657032654957) bank125028 +125029 POINT(47.24482449251656 -122.38090262418301) bank125029 +125030 POINT(47.79468906856052 -123.25675513772195) bank125030 +125031 POINT(48.08648045486499 -121.39975414820375) bank125031 +125032 POINT(47.34803678289765 -121.41441410202509) bank125032 +125033 POINT(47.06079766079371 -123.28639813967521) bank125033 +125034 POINT(47.973260498976096 -122.27596065300145) bank125034 +125035 POINT(46.76098053719813 -121.98483047665829) bank125035 +125036 POINT(48.21040779303156 -123.28327302387315) bank125036 +125037 POINT(47.18592522358601 -122.95114444242458) bank125037 +125038 POINT(46.76174908423301 -122.5343602610367) bank125038 +125039 POINT(47.222490164437495 -123.18590154801899) bank125039 +125040 POINT(47.88576106504053 -122.50457764318026) bank125040 +125041 POINT(47.05593104262895 -121.82576065927128) bank125041 +125042 POINT(48.484309225233396 -121.63932523274254) bank125042 +125043 POINT(48.19163107291478 -121.6998808491716) bank125043 +125044 POINT(48.23397910926259 -121.39969841696404) bank125044 +125045 POINT(48.52195853834602 -122.3274500891008) bank125045 +125046 POINT(47.97639725158567 -121.8558748134652) bank125046 +125047 POINT(48.3905747641802 -122.22963668574144) bank125047 +125048 POINT(47.29539626625087 -122.955382786471) bank125048 +125049 POINT(48.367248064863325 -122.59234214396076) bank125049 +125050 POINT(47.704230744109225 -122.2104241627673) bank125050 +125051 POINT(47.08100588574175 -123.2249666021842) bank125051 +125052 POINT(47.326015460899804 -122.936822600611) bank125052 +125053 POINT(46.9793504128106 -121.71007384684178) bank125053 +125054 POINT(47.92186847483618 -123.21116660266614) bank125054 +125055 POINT(48.08052585624929 -123.19244629160335) bank125055 +125056 POINT(47.25766864817696 -121.92042535000884) bank125056 +125057 POINT(48.06527547976471 -122.76869142225459) bank125057 +125058 POINT(48.109555821803575 -122.05935612357331) bank125058 +125059 POINT(47.055040718217825 -121.71440621366459) bank125059 +125060 POINT(48.12718332020993 -123.18630860064974) bank125060 +125061 POINT(47.570412017490796 -121.85657169610495) bank125061 +125062 POINT(46.94385477279464 -123.08677235643106) bank125062 +125063 POINT(47.430486506427556 -122.13559169469282) bank125063 +125064 POINT(47.27861962770512 -121.3380087782646) bank125064 +125065 POINT(46.61717366997417 -123.24624314253417) bank125065 +125066 POINT(48.5509170888045 -121.39972204831678) bank125066 +125067 POINT(46.992715970284706 -122.93318571188223) bank125067 +125068 POINT(46.79919869670901 -123.14924399316357) bank125068 +125069 POINT(46.83816307775353 -123.04850582224078) bank125069 +125070 POINT(46.9908182497143 -122.67644409062005) bank125070 +125071 POINT(47.51439563830644 -121.93543307038391) bank125071 +125072 POINT(47.283528220915926 -122.36773545266072) bank125072 +125073 POINT(48.53331407434625 -122.91613963591682) bank125073 +125074 POINT(47.23449832311928 -122.84211206016124) bank125074 +125075 POINT(46.90692574286551 -122.45198289794833) bank125075 +125076 POINT(47.62053605867109 -122.37681166303707) bank125076 +125077 POINT(48.182090726356286 -122.86489211240053) bank125077 +125078 POINT(47.83543954953807 -122.43211038628597) bank125078 +125079 POINT(48.097050221164615 -122.41112331734911) bank125079 +125080 POINT(48.38962977814972 -122.58181421454233) bank125080 +125081 POINT(48.10800033699546 -121.63084793226014) bank125081 +125082 POINT(46.69635504549034 -123.02014060298642) bank125082 +125083 POINT(48.157553428047336 -123.0814919167782) bank125083 +125084 POINT(47.749603382160245 -122.1321406694918) bank125084 +125085 POINT(47.19909316027614 -121.91231449689072) bank125085 +125086 POINT(48.31390226103265 -122.40693736884056) bank125086 +125087 POINT(47.86580149317896 -122.34344427623527) bank125087 +125088 POINT(46.860928476644915 -122.18489338773384) bank125088 +125089 POINT(47.20840432647954 -121.47174754903745) bank125089 +125090 POINT(48.58303275666198 -122.452331548102) bank125090 +125091 POINT(46.712805480764175 -123.14760782103126) bank125091 +125092 POINT(46.862343658568264 -122.80726727761703) bank125092 +125093 POINT(46.96328931714725 -122.79085307571111) bank125093 +125094 POINT(46.85912886123324 -121.4368558411765) bank125094 +125095 POINT(46.761014476777504 -122.33278032360592) bank125095 +125096 POINT(47.085320386400824 -122.01209731877422) bank125096 +125097 POINT(47.583597022734715 -122.89810916447576) bank125097 +125098 POINT(48.09649503508039 -121.99596684622718) bank125098 +125099 POINT(47.73739854662889 -123.04323732934313) bank125099 +125100 POINT(47.31925317983769 -122.9867551227598) bank125100 +125101 POINT(47.247525480540666 -123.20741161905868) bank125101 +125102 POINT(46.7408309614115 -121.33598089640678) bank125102 +125103 POINT(48.10168144040978 -122.03280328077169) bank125103 +125104 POINT(47.77619364447649 -121.6127402989077) bank125104 +125105 POINT(47.99691174926682 -122.51539436471069) bank125105 +125106 POINT(47.485003611354614 -123.15334205609433) bank125106 +125107 POINT(47.91908746171816 -121.60283179207042) bank125107 +125108 POINT(47.72817205075843 -123.25245755078599) bank125108 +125109 POINT(47.22570021375546 -123.15416913810421) bank125109 +125110 POINT(46.63376232061103 -122.93037764352592) bank125110 +125111 POINT(47.97555962612776 -121.66364934106335) bank125111 +125112 POINT(46.609112186921344 -122.9936442671873) bank125112 +125113 POINT(47.62135737435179 -122.80198941199565) bank125113 +125114 POINT(46.76846632572547 -121.38333131942672) bank125114 +125115 POINT(48.120228118398195 -122.54289987106424) bank125115 +125116 POINT(48.349220232381306 -122.57933945245102) bank125116 +125117 POINT(48.09013762489216 -122.75229368126436) bank125117 +125118 POINT(46.83740802001239 -121.64151650162525) bank125118 +125119 POINT(47.87860392542372 -122.58440534672634) bank125119 +125120 POINT(47.39382709947456 -122.10263407923843) bank125120 +125121 POINT(46.93126794090421 -122.4920495101944) bank125121 +125122 POINT(48.219411562314924 -123.30062733132215) bank125122 +125123 POINT(47.42335574182044 -122.49319174815729) bank125123 +125124 POINT(47.888698293083905 -122.02123006520243) bank125124 +125125 POINT(47.63139564686316 -122.85485431345724) bank125125 +125126 POINT(47.07145427146366 -122.77287914245912) bank125126 +125127 POINT(47.878383426665955 -123.11544209388694) bank125127 +125128 POINT(46.76760939123445 -121.54010534970408) bank125128 +125129 POINT(48.042215512310776 -122.11208491103675) bank125129 +125130 POINT(47.11869856819807 -122.3627981996259) bank125130 +125131 POINT(47.11830126570927 -121.66953443809578) bank125131 +125132 POINT(46.790187985438834 -121.96525546607654) bank125132 +125133 POINT(48.10601003763177 -123.2746393539325) bank125133 +125134 POINT(48.335330021382475 -122.35530853682003) bank125134 +125135 POINT(46.6466564881407 -123.23373393549473) bank125135 +125136 POINT(47.165149630489296 -122.96432901561622) bank125136 +125137 POINT(47.961222164436 -121.70007058785683) bank125137 +125138 POINT(47.31344670186922 -123.07879952362595) bank125138 +125139 POINT(48.60598154889445 -121.96935545899892) bank125139 +125140 POINT(47.23721114545905 -123.22808846769672) bank125140 +125141 POINT(48.511328437441605 -122.34256862812921) bank125141 +125142 POINT(47.17937539025834 -123.13978627124514) bank125142 +125143 POINT(48.01361329788897 -123.33035392232314) bank125143 +125144 POINT(46.907108396671 -123.19089462996486) bank125144 +125145 POINT(48.29503521879621 -123.22699927058086) bank125145 +125146 POINT(46.81411769067104 -122.8421357773707) bank125146 +125147 POINT(47.943853635954625 -121.74937533983083) bank125147 +125148 POINT(47.5967693127413 -121.620959279505) bank125148 +125149 POINT(47.82986406567879 -121.726623225229) bank125149 +125150 POINT(47.79106420732373 -121.81879508403826) bank125150 +125151 POINT(47.775793158475246 -121.86060843649332) bank125151 +125152 POINT(48.12299325274529 -123.205692979739) bank125152 +125153 POINT(47.027532497275814 -122.91024831070989) bank125153 +125154 POINT(47.608873818399005 -121.58508518953217) bank125154 +125155 POINT(48.2265910194572 -122.96644211817252) bank125155 +125156 POINT(47.5528667045836 -122.6327975013065) bank125156 +125157 POINT(46.78168997912751 -121.79045252346914) bank125157 +125158 POINT(46.846186811697336 -123.11692151469356) bank125158 +125159 POINT(46.980838892787794 -123.02934234443741) bank125159 +125160 POINT(48.43996298187529 -122.2719988418494) bank125160 +125161 POINT(47.18244986771563 -123.30470083268635) bank125161 +125162 POINT(47.29780812447955 -122.54151326572817) bank125162 +125163 POINT(47.48443447793457 -122.95619008745793) bank125163 +125164 POINT(48.43302169742075 -122.96280334129598) bank125164 +125165 POINT(47.12440728048752 -121.8870714005929) bank125165 +125166 POINT(48.12116506247408 -122.35418159456339) bank125166 +125167 POINT(47.496365086559734 -122.69908843696777) bank125167 +125168 POINT(47.68439479837456 -123.14240776744218) bank125168 +125169 POINT(48.063236396822354 -123.31506114322953) bank125169 +125170 POINT(47.160818075637714 -121.59780707744152) bank125170 +125171 POINT(48.12156980382686 -121.86355401191489) bank125171 +125172 POINT(46.89525192795786 -122.91127092452501) bank125172 +125173 POINT(47.54093684580655 -122.40071063503758) bank125173 +125174 POINT(46.70998842427875 -121.68742555327277) bank125174 +125175 POINT(46.7171253345563 -122.1904488609372) bank125175 +125176 POINT(47.570108314126394 -121.7981217127112) bank125176 +125177 POINT(47.49790898934079 -123.31991614770271) bank125177 +125178 POINT(47.95039181885894 -123.11592024485782) bank125178 +125179 POINT(46.7456876972469 -122.39886539997029) bank125179 +125180 POINT(47.594509922006274 -121.67115672442138) bank125180 +125181 POINT(46.6096794193805 -123.00029509717311) bank125181 +125182 POINT(48.503796327478724 -122.6962505059232) bank125182 +125183 POINT(47.43048986743143 -122.6076947879016) bank125183 +125184 POINT(46.713175495701655 -123.11990510612503) bank125184 +125185 POINT(47.84700740152999 -123.24053129465186) bank125185 +125186 POINT(47.644376199226464 -123.19390016697541) bank125186 +125187 POINT(47.14271875093861 -122.65049555443728) bank125187 +125188 POINT(47.933826187208965 -122.42319006051915) bank125188 +125189 POINT(48.33576592897156 -122.95676495452294) bank125189 +125190 POINT(46.73793737881301 -122.41236307593314) bank125190 +125191 POINT(46.98786518506569 -121.8663535947615) bank125191 +125192 POINT(47.152278511606134 -122.87766761124058) bank125192 +125193 POINT(48.397029177181125 -122.68724553478215) bank125193 +125194 POINT(47.0951698702256 -121.53037279021774) bank125194 +125195 POINT(48.552316614699535 -122.92303989625321) bank125195 +125196 POINT(47.4527904439883 -123.21987335393004) bank125196 +125197 POINT(46.6867607550472 -123.2534227971722) bank125197 +125198 POINT(48.34152460717985 -121.4323429831635) bank125198 +125199 POINT(47.58285793429268 -122.09529997222262) bank125199 +125200 POINT(46.611848897092784 -122.51153649269706) bank125200 +125201 POINT(48.434454744286775 -122.2527423298368) bank125201 +125202 POINT(47.35922545388537 -121.54078233604629) bank125202 +125203 POINT(47.97962300693618 -123.06457004548889) bank125203 +125204 POINT(47.535665697847996 -123.11835537645293) bank125204 +125205 POINT(48.33646375401336 -121.97524443698414) bank125205 +125206 POINT(48.35788743153415 -123.25758369489162) bank125206 +125207 POINT(47.070775416110145 -122.72436846311663) bank125207 +125208 POINT(47.52889248823629 -121.84240467377236) bank125208 +125209 POINT(46.70180660365189 -122.93724865395757) bank125209 +125210 POINT(47.32659597798242 -122.88921540276657) bank125210 +125211 POINT(47.90705890009221 -122.02471165164148) bank125211 +125212 POINT(47.97349013756829 -122.75495770113449) bank125212 +125213 POINT(47.12166965359439 -122.27963230197092) bank125213 +125214 POINT(48.015871106992556 -123.11057803338541) bank125214 +125215 POINT(47.80672485780676 -122.96198770698943) bank125215 +125216 POINT(46.722308684851356 -121.55698713431724) bank125216 +125217 POINT(47.054252384593646 -122.7296726347233) bank125217 +125218 POINT(46.97697390140499 -123.16292170199156) bank125218 +125219 POINT(48.558893745478166 -121.76028622926898) bank125219 +125220 POINT(47.85607251944869 -122.65169777044721) bank125220 +125221 POINT(46.907528558003214 -122.8894888408583) bank125221 +125222 POINT(47.75269419947448 -121.91376682772636) bank125222 +125223 POINT(47.39435112327148 -121.4676163897683) bank125223 +125224 POINT(47.743171709114144 -122.99129256766739) bank125224 +125225 POINT(47.78414774254893 -123.27438851152044) bank125225 +125226 POINT(46.99241615493912 -121.99980435221941) bank125226 +125227 POINT(48.01195984134104 -122.7884168055575) bank125227 +125228 POINT(47.54479605272163 -122.0630121902185) bank125228 +125229 POINT(48.46400359404978 -121.67504841467506) bank125229 +125230 POINT(47.8558024127805 -121.55395835272746) bank125230 +125231 POINT(46.691749774680204 -121.51309454766626) bank125231 +125232 POINT(47.97639248491938 -123.05323297496989) bank125232 +125233 POINT(46.68652881757584 -122.2059773776709) bank125233 +125234 POINT(47.83396154114596 -122.8510031522823) bank125234 +125235 POINT(47.557371794146356 -123.13427400330181) bank125235 +125236 POINT(48.3488055003848 -122.63821295971908) bank125236 +125237 POINT(47.48621693714812 -122.90516349655334) bank125237 +125238 POINT(47.59297687908085 -122.36404054771442) bank125238 +125239 POINT(47.38984668819779 -122.82058824254356) bank125239 +125240 POINT(47.01917725500124 -121.72916783830031) bank125240 +125241 POINT(47.566552198968004 -123.2491979211074) bank125241 +125242 POINT(47.22999702601108 -123.30490905101252) bank125242 +125243 POINT(48.3671325924619 -122.53962882797381) bank125243 +125244 POINT(48.44142630020092 -121.8568775632162) bank125244 +125245 POINT(47.753978196826694 -122.66241997408625) bank125245 +125246 POINT(48.219763987480356 -121.81859094639685) bank125246 +125247 POINT(47.05140088981717 -122.15226554184956) bank125247 +125248 POINT(46.946933391155 -122.22113093020798) bank125248 +125249 POINT(46.9282653432044 -121.86035614215164) bank125249 +125250 POINT(46.79386144406977 -122.21948102696525) bank125250 +125251 POINT(47.67070887237837 -121.98405269400862) bank125251 +125252 POINT(48.458320451076276 -121.47028155453029) bank125252 +125253 POINT(47.78544191513236 -122.54723414374097) bank125253 +125254 POINT(46.650030558753954 -122.69897942488353) bank125254 +125255 POINT(47.10960171147435 -122.15510386557875) bank125255 +125256 POINT(48.13694969501107 -122.88091312097693) bank125256 +125257 POINT(47.16937490434031 -122.370428544966) bank125257 +125258 POINT(47.17921096969076 -122.67818838633056) bank125258 +125259 POINT(48.470615058262744 -123.18285573284282) bank125259 +125260 POINT(47.98853579073817 -122.21453161048201) bank125260 +125261 POINT(47.99666657581147 -122.1003939083233) bank125261 +125262 POINT(47.538524844840026 -121.89672520693871) bank125262 +125263 POINT(47.469633226828336 -121.65319881826825) bank125263 +125264 POINT(46.98973138442069 -122.97659830202386) bank125264 +125265 POINT(48.356275824422276 -123.03783443294324) bank125265 +125266 POINT(48.172822834867716 -122.14629571520072) bank125266 +125267 POINT(47.82754941356236 -123.09111996633949) bank125267 +125268 POINT(47.27746957389432 -122.40578339759098) bank125268 +125269 POINT(47.635306377294526 -123.09745326845537) bank125269 +125270 POINT(48.59430918365645 -121.84145784733336) bank125270 +125271 POINT(48.347272868158655 -123.08197113090091) bank125271 +125272 POINT(48.22149512648886 -123.3027761138035) bank125272 +125273 POINT(48.20687971889812 -122.98565814266992) bank125273 +125274 POINT(48.200759044340536 -121.42397963679892) bank125274 +125275 POINT(48.569695791703836 -122.66646435088217) bank125275 +125276 POINT(47.64680221913882 -121.54354853318664) bank125276 +125277 POINT(47.981210961600524 -122.47896741255214) bank125277 +125278 POINT(47.129284256000375 -122.17830342766712) bank125278 +125279 POINT(47.01666371581992 -123.25067333001837) bank125279 +125280 POINT(48.51737251102705 -121.8739369627586) bank125280 +125281 POINT(48.132079041931775 -123.19639606760742) bank125281 +125282 POINT(46.69530609702182 -122.70430601283158) bank125282 +125283 POINT(46.86902716016639 -123.33045171339765) bank125283 +125284 POINT(48.222761682139996 -122.11289017566696) bank125284 +125285 POINT(47.11086144592445 -122.81422245784228) bank125285 +125286 POINT(47.28063134732552 -122.87312447140545) bank125286 +125287 POINT(47.604151361420755 -122.23824843343814) bank125287 +125288 POINT(47.563548057759206 -122.80832273409592) bank125288 +125289 POINT(47.24483832843347 -122.9910950412383) bank125289 +125290 POINT(47.69862297986717 -121.3745246484391) bank125290 +125291 POINT(48.460626827253 -122.35254550663322) bank125291 +125292 POINT(47.967376980635095 -121.36294495152391) bank125292 +125293 POINT(48.4703332594014 -122.48822646049013) bank125293 +125294 POINT(47.26497267807677 -122.33167736787341) bank125294 +125295 POINT(48.35944778048471 -123.12936043120452) bank125295 +125296 POINT(48.296295956742405 -122.90106851299517) bank125296 +125297 POINT(46.87974496307567 -121.37995375148013) bank125297 +125298 POINT(47.34871279225511 -121.75150712861556) bank125298 +125299 POINT(48.00067726807834 -121.48439855642711) bank125299 +125300 POINT(48.50560754258945 -122.60665033788928) bank125300 +125301 POINT(48.057902788147125 -123.16316695413246) bank125301 +125302 POINT(47.8604026073614 -121.48970176980534) bank125302 +125303 POINT(46.76055445476215 -122.37037296249807) bank125303 +125304 POINT(48.0266048098294 -121.36771392749252) bank125304 +125305 POINT(48.561314684398084 -122.61769909300217) bank125305 +125306 POINT(46.87265239765561 -123.12855807420351) bank125306 +125307 POINT(48.47104232954481 -121.37776079145618) bank125307 +125308 POINT(47.60769612545879 -122.84165309123733) bank125308 +125309 POINT(47.90267742537626 -121.38745747456585) bank125309 +125310 POINT(48.49646809494064 -122.2088287692195) bank125310 +125311 POINT(47.49055936975806 -123.29712630972732) bank125311 +125312 POINT(48.00595360651184 -122.03167756822462) bank125312 +125313 POINT(48.42369091434733 -121.72609069576139) bank125313 +125314 POINT(48.048293774192224 -122.51978417272187) bank125314 +125315 POINT(47.90343067437857 -121.46101556585567) bank125315 +125316 POINT(47.56217660746356 -123.0282386646294) bank125316 +125317 POINT(48.19092670422397 -122.66105677255806) bank125317 +125318 POINT(46.817150548111435 -123.28219020506826) bank125318 +125319 POINT(46.65493609346064 -122.79363813205347) bank125319 +125320 POINT(47.71936068418474 -121.9592312996326) bank125320 +125321 POINT(48.535894158308736 -121.58825798219442) bank125321 +125322 POINT(47.88950654915163 -121.69262085884377) bank125322 +125323 POINT(47.58880033554182 -122.68251200222154) bank125323 +125324 POINT(47.00056822624608 -121.66245856527266) bank125324 +125325 POINT(47.292050970240545 -122.74253490212722) bank125325 +125326 POINT(48.362957960140626 -121.96601982896658) bank125326 +125327 POINT(46.9870992818031 -121.63861847108524) bank125327 +125328 POINT(47.03426288519031 -121.3479506921077) bank125328 +125329 POINT(47.19146545992108 -122.24516245892295) bank125329 +125330 POINT(47.826027757134405 -121.83965368832213) bank125330 +125331 POINT(48.58509254099965 -121.42868810499839) bank125331 +125332 POINT(48.53727859031823 -122.22095071413895) bank125332 +125333 POINT(48.091721608431264 -122.33400862395675) bank125333 +125334 POINT(47.12135390783424 -122.48383325563394) bank125334 +125335 POINT(47.90702047107189 -121.7293736754419) bank125335 +125336 POINT(47.45132563965439 -122.73849220972414) bank125336 +125337 POINT(48.404737298693625 -123.31196339919485) bank125337 +125338 POINT(47.550704929726336 -121.85685253574066) bank125338 +125339 POINT(47.26831250279052 -121.76343167854259) bank125339 +125340 POINT(48.60195336882453 -122.28206139574904) bank125340 +125341 POINT(47.73929425890972 -122.97462066271567) bank125341 +125342 POINT(48.03657630944589 -122.98486856274283) bank125342 +125343 POINT(48.38365361897394 -122.77424051957321) bank125343 +125344 POINT(47.60116328199031 -121.47554987769432) bank125344 +125345 POINT(48.597306777109395 -123.1221574693954) bank125345 +125346 POINT(47.5499595189196 -121.34442937957473) bank125346 +125347 POINT(47.80737429195694 -123.32234750298522) bank125347 +125348 POINT(47.96534654702086 -122.88205903521758) bank125348 +125349 POINT(48.356807787386884 -121.44951768438337) bank125349 +125350 POINT(47.91083363108394 -123.07894838461512) bank125350 +125351 POINT(48.10299940484965 -121.5050433307594) bank125351 +125352 POINT(47.72099891413161 -122.27267958028432) bank125352 +125353 POINT(48.328883986874295 -122.70298690984204) bank125353 +125354 POINT(46.87648572208802 -122.29404531898078) bank125354 +125355 POINT(47.38852304978568 -121.80351518080195) bank125355 +125356 POINT(47.53414351520753 -121.91322849280678) bank125356 +125357 POINT(47.04289532135764 -121.63693810265046) bank125357 +125358 POINT(47.54359684295096 -121.42820896023004) bank125358 +125359 POINT(47.839630263357336 -122.0704586862617) bank125359 +125360 POINT(46.85586693089198 -123.16644183958792) bank125360 +125361 POINT(48.12690631851607 -123.27729102886232) bank125361 +125362 POINT(47.24027266899701 -123.208903955176) bank125362 +125363 POINT(47.24868137210108 -121.47911018886795) bank125363 +125364 POINT(48.25297539392787 -121.41953752957839) bank125364 +125365 POINT(47.82889928916548 -121.706590829618) bank125365 +125366 POINT(48.335638549574895 -122.75625680313117) bank125366 +125367 POINT(48.128425723348045 -122.87675612999423) bank125367 +125368 POINT(48.11091715584235 -122.2174176427211) bank125368 +125369 POINT(48.446859915030586 -121.64072009903558) bank125369 +125370 POINT(46.979756077769174 -122.44368128183689) bank125370 +125371 POINT(47.925376367393 -123.03484997556446) bank125371 +125372 POINT(48.38319658154045 -122.1310210200385) bank125372 +125373 POINT(48.38878742179489 -123.18342452708714) bank125373 +125374 POINT(47.7941433696799 -121.36655946152405) bank125374 +125375 POINT(48.02480370504392 -123.11599591081003) bank125375 +125376 POINT(47.24338375956027 -122.21367688209729) bank125376 +125377 POINT(48.10018566263589 -122.03482907352567) bank125377 +125378 POINT(48.18134179269183 -121.70236488697415) bank125378 +125379 POINT(48.20303642567931 -123.27253937924114) bank125379 +125380 POINT(47.91036390106707 -122.1007679518131) bank125380 +125381 POINT(47.86090654071214 -122.05777678378404) bank125381 +125382 POINT(48.54685406094154 -122.26672500466887) bank125382 +125383 POINT(48.29628440361836 -122.87903553003576) bank125383 +125384 POINT(48.12395636680068 -121.65828359557482) bank125384 +125385 POINT(47.346593054883066 -122.97714100585326) bank125385 +125386 POINT(48.09177915546434 -122.48761314876721) bank125386 +125387 POINT(47.14973618862381 -122.54661227938075) bank125387 +125388 POINT(47.44072813115766 -122.73666292249489) bank125388 +125389 POINT(47.251191063033964 -121.8767603862948) bank125389 +125390 POINT(48.584821577224055 -123.30130074500252) bank125390 +125391 POINT(48.0358858533367 -122.97807423768833) bank125391 +125392 POINT(48.05964863214166 -122.26315747681937) bank125392 +125393 POINT(47.04979880555493 -122.62533828624379) bank125393 +125394 POINT(47.087506393473014 -121.46078044676185) bank125394 +125395 POINT(48.40437125619474 -122.03838497259379) bank125395 +125396 POINT(47.368849966258914 -122.50788363684408) bank125396 +125397 POINT(48.319045393033704 -122.08774535989173) bank125397 +125398 POINT(47.214218495151634 -121.9671582536565) bank125398 +125399 POINT(46.61666193524264 -122.76221794873975) bank125399 +125400 POINT(46.96628822831412 -123.32889343844572) bank125400 +125401 POINT(47.56692355953052 -121.99448017099228) bank125401 +125402 POINT(48.31949862696957 -122.96087869876493) bank125402 +125403 POINT(47.181481615738996 -123.00952350689751) bank125403 +125404 POINT(48.269392100014365 -122.34249334872105) bank125404 +125405 POINT(48.09668896600914 -122.68115234259429) bank125405 +125406 POINT(46.648411182837044 -123.22211522368347) bank125406 +125407 POINT(47.00757980548262 -122.3782012876168) bank125407 +125408 POINT(47.977579955501 -122.48809758987619) bank125408 +125409 POINT(47.7640721535431 -122.91329745084856) bank125409 +125410 POINT(48.14330148098571 -121.66409380365786) bank125410 +125411 POINT(48.58517784036782 -121.99898404003079) bank125411 +125412 POINT(47.15013422798007 -121.8836056864127) bank125412 +125413 POINT(46.64327614607031 -123.0666399352006) bank125413 +125414 POINT(48.208872084501984 -121.50375844773603) bank125414 +125415 POINT(48.10522251041066 -123.1846377795873) bank125415 +125416 POINT(48.28112106691491 -123.09643986198911) bank125416 +125417 POINT(47.69129843877938 -122.88173544425558) bank125417 +125418 POINT(47.44377373403416 -122.61744849851789) bank125418 +125419 POINT(46.92418780326449 -122.55617755308955) bank125419 +125420 POINT(47.427583210027066 -122.81253202304218) bank125420 +125421 POINT(48.354674633133484 -122.37687216790887) bank125421 +125422 POINT(47.6793705048257 -121.70608794973099) bank125422 +125423 POINT(47.493783551558955 -122.98894844616741) bank125423 +125424 POINT(47.95562624110746 -122.99200863322874) bank125424 +125425 POINT(48.27108075295403 -122.76599565098162) bank125425 +125426 POINT(46.820603758702454 -121.44278913262394) bank125426 +125427 POINT(46.80877277093898 -121.58187623988319) bank125427 +125428 POINT(47.85129516210132 -123.05906717200223) bank125428 +125429 POINT(46.74117967924733 -122.41873436704135) bank125429 +125430 POINT(46.626979395297646 -122.89473344156798) bank125430 +125431 POINT(47.37892502569223 -122.213150559759) bank125431 +125432 POINT(48.12785254793654 -122.19125654023264) bank125432 +125433 POINT(47.50432326324142 -121.71046482156939) bank125433 +125434 POINT(47.383613719874944 -121.5499726135301) bank125434 +125435 POINT(47.64714432900361 -122.88572908278023) bank125435 +125436 POINT(48.03012941159526 -122.28458949341557) bank125436 +125437 POINT(48.04412044523404 -122.89374686762147) bank125437 +125438 POINT(47.25931662014363 -122.27295714171785) bank125438 +125439 POINT(48.45373393864235 -122.84741058515698) bank125439 +125440 POINT(47.375343634167066 -122.18568554820436) bank125440 +125441 POINT(47.23584250625649 -123.31418934440173) bank125441 +125442 POINT(46.917524233888905 -122.47322533334867) bank125442 +125443 POINT(47.43947631765992 -122.44653888395615) bank125443 +125444 POINT(47.233502145043204 -122.86949102890118) bank125444 +125445 POINT(47.33448764699137 -121.8851751782077) bank125445 +125446 POINT(47.80389575054137 -123.05573685243743) bank125446 +125447 POINT(47.114596643007175 -121.6296965608486) bank125447 +125448 POINT(47.23304829422415 -122.23885735536989) bank125448 +125449 POINT(47.52880013795013 -121.43569433535411) bank125449 +125450 POINT(47.0211264890478 -122.58171988669073) bank125450 +125451 POINT(48.60029345844693 -122.7137937861789) bank125451 +125452 POINT(47.330672571188714 -121.35241243229977) bank125452 +125453 POINT(47.04563291018364 -122.69325159709682) bank125453 +125454 POINT(47.32586548070295 -123.17897502029695) bank125454 +125455 POINT(48.36955350480322 -121.83605939749239) bank125455 +125456 POINT(47.30227907237317 -121.48583265427403) bank125456 +125457 POINT(47.302153905223896 -121.89011941195044) bank125457 +125458 POINT(48.018573915125295 -121.52284071405748) bank125458 +125459 POINT(46.94110227928775 -122.20312299078968) bank125459 +125460 POINT(47.80480849552732 -123.04503937413268) bank125460 +125461 POINT(47.157907891393414 -123.33208281014218) bank125461 +125462 POINT(47.92242892366641 -122.44364596041626) bank125462 +125463 POINT(46.67744959255446 -123.07442209411496) bank125463 +125464 POINT(48.0341455135421 -123.20522543708016) bank125464 +125465 POINT(47.99577736948216 -122.98547703992533) bank125465 +125466 POINT(47.86766969203882 -121.84634869886997) bank125466 +125467 POINT(47.58810735206307 -122.0690601234881) bank125467 +125468 POINT(48.14106902484635 -121.86960299848265) bank125468 +125469 POINT(47.374821362996606 -121.37728562223465) bank125469 +125470 POINT(47.08966783951697 -121.98503562245925) bank125470 +125471 POINT(48.195278347537965 -121.45851612080845) bank125471 +125472 POINT(48.420919885034166 -123.11293481315562) bank125472 +125473 POINT(46.8516291184472 -121.52033639408154) bank125473 +125474 POINT(47.940978371388006 -122.94182472693247) bank125474 +125475 POINT(47.97158423729539 -121.86790850597001) bank125475 +125476 POINT(48.55623797686704 -122.26718903033517) bank125476 +125477 POINT(46.78311257910206 -123.29592942191405) bank125477 +125478 POINT(48.32043512522609 -121.90698929478391) bank125478 +125479 POINT(47.951142614944246 -121.99129105828172) bank125479 +125480 POINT(47.773479763209984 -123.09310421835204) bank125480 +125481 POINT(48.29259676080358 -121.5755722169239) bank125481 +125482 POINT(48.128660880011346 -123.32718257428618) bank125482 +125483 POINT(47.20606895420365 -121.69074470525221) bank125483 +125484 POINT(47.82852608437081 -121.67400341898178) bank125484 +125485 POINT(47.19351559637063 -122.47560578559217) bank125485 +125486 POINT(46.96623012414497 -122.13889350179522) bank125486 +125487 POINT(47.00209512863883 -122.48948083422763) bank125487 +125488 POINT(47.167803602468666 -122.54673402995076) bank125488 +125489 POINT(47.67677034673073 -121.45884370093847) bank125489 +125490 POINT(47.94450464977872 -122.22392761691455) bank125490 +125491 POINT(48.15487249434311 -123.05986040620621) bank125491 +125492 POINT(46.70123061926518 -122.23345210498442) bank125492 +125493 POINT(47.02063733079449 -122.64561020360777) bank125493 +125494 POINT(47.02165185938342 -122.55329615886413) bank125494 +125495 POINT(47.05759720343983 -122.26732515592377) bank125495 +125496 POINT(46.614487118629555 -123.32800637798138) bank125496 +125497 POINT(47.20468432429912 -122.99933598142404) bank125497 +125498 POINT(48.272098027419126 -122.19869605110232) bank125498 +125499 POINT(47.38806006193296 -123.32804965229428) bank125499 +125500 POINT(48.57301466185428 -122.38767009211757) bank125500 +125501 POINT(47.51814878363396 -121.92563356646036) bank125501 +125502 POINT(47.973716945650985 -121.53151970046832) bank125502 +125503 POINT(48.602924640078655 -122.77761104054346) bank125503 +125504 POINT(47.8976549380088 -121.9130138378821) bank125504 +125505 POINT(48.5601095263644 -122.81642920472494) bank125505 +125506 POINT(48.56734152180835 -121.96915619498893) bank125506 +125507 POINT(47.14780064279971 -122.76861810029664) bank125507 +125508 POINT(48.01308727927831 -121.64207419619287) bank125508 +125509 POINT(47.29230707462895 -122.69743271768145) bank125509 +125510 POINT(46.7217972808336 -123.30424652699487) bank125510 +125511 POINT(47.620484443807186 -123.31805271861442) bank125511 +125512 POINT(48.17663402449557 -123.15416626424047) bank125512 +125513 POINT(48.587548927501395 -122.45053648408236) bank125513 +125514 POINT(46.78095775764299 -122.61790776483444) bank125514 +125515 POINT(47.47162124357641 -121.8260145996982) bank125515 +125516 POINT(46.8272016293744 -122.11615964400782) bank125516 +125517 POINT(48.009559060674555 -123.09167333396866) bank125517 +125518 POINT(48.29488141453394 -122.5544762043738) bank125518 +125519 POINT(46.721833831639 -121.96070375938336) bank125519 +125520 POINT(47.535519963941084 -122.14842343543978) bank125520 +125521 POINT(47.858706951265006 -121.91224510380033) bank125521 +125522 POINT(48.15021899610631 -121.45397042058009) bank125522 +125523 POINT(47.13985824798176 -122.87710988329476) bank125523 +125524 POINT(47.862465972884046 -122.7537864912923) bank125524 +125525 POINT(47.464948780706145 -121.7593304423801) bank125525 +125526 POINT(46.627473195153605 -122.56376521331889) bank125526 +125527 POINT(46.61053346445782 -121.61094122785093) bank125527 +125528 POINT(47.4416982872315 -121.68771992644294) bank125528 +125529 POINT(47.88474055643554 -121.4312093994644) bank125529 +125530 POINT(47.986634084561764 -121.3673432497238) bank125530 +125531 POINT(46.738997628366974 -122.19115842705772) bank125531 +125532 POINT(47.17824278125519 -122.16714385959072) bank125532 +125533 POINT(48.34918321176986 -121.52095808238799) bank125533 +125534 POINT(47.68257741120989 -122.52986188718033) bank125534 +125535 POINT(47.43041749718566 -122.07777547516847) bank125535 +125536 POINT(48.4216403057834 -122.89361746330462) bank125536 +125537 POINT(46.98208367387376 -122.58303179011705) bank125537 +125538 POINT(47.19077790783352 -123.02524398696947) bank125538 +125539 POINT(48.23328817389625 -122.20338244987089) bank125539 +125540 POINT(47.59604583179066 -121.84199149886763) bank125540 +125541 POINT(48.5444782890114 -121.64090192420878) bank125541 +125542 POINT(47.947815581286434 -121.77024275613621) bank125542 +125543 POINT(47.03248227613179 -121.87662316805947) bank125543 +125544 POINT(48.1987573739228 -122.47388616029465) bank125544 +125545 POINT(47.938020234381455 -121.91309773524632) bank125545 +125546 POINT(46.83888804039857 -122.3703219811645) bank125546 +125547 POINT(46.71837023761494 -122.3105919374469) bank125547 +125548 POINT(47.762168245371534 -121.63103091893448) bank125548 +125549 POINT(46.81996093161671 -123.0182381673708) bank125549 +125550 POINT(46.912161183518045 -122.32692158733683) bank125550 +125551 POINT(48.094161436675336 -121.96704887291216) bank125551 +125552 POINT(48.26220779661173 -121.42719908166684) bank125552 +125553 POINT(48.418471446374454 -122.4184878237586) bank125553 +125554 POINT(46.86487436373349 -122.91952885668695) bank125554 +125555 POINT(47.00448792793588 -121.62905458057904) bank125555 +125556 POINT(47.09987038878332 -121.99176908709615) bank125556 +125557 POINT(47.963901442890794 -121.73479156364635) bank125557 +125558 POINT(47.17045153931284 -121.71178051194552) bank125558 +125559 POINT(47.40995123238806 -123.11443624435543) bank125559 +125560 POINT(48.60570180060805 -123.19626045856036) bank125560 +125561 POINT(47.4664288375727 -121.89867770916834) bank125561 +125562 POINT(46.655903964006626 -122.41434039201829) bank125562 +125563 POINT(47.179960351620764 -123.05940972831276) bank125563 +125564 POINT(47.54671013302782 -122.14204457635537) bank125564 +125565 POINT(47.45353201258313 -123.2162059479119) bank125565 +125566 POINT(48.53552706561555 -122.6350148860729) bank125566 +125567 POINT(48.58044339983511 -122.38716558540287) bank125567 +125568 POINT(47.07943755193937 -121.51939927540779) bank125568 +125569 POINT(46.623569626789596 -121.62719941240073) bank125569 +125570 POINT(48.161357882109066 -122.53640442999368) bank125570 +125571 POINT(46.72900723883776 -122.72360887676886) bank125571 +125572 POINT(47.46074197490008 -121.36288845153872) bank125572 +125573 POINT(48.34400560887547 -123.20603445288128) bank125573 +125574 POINT(47.82783712797401 -121.65689681461241) bank125574 +125575 POINT(48.02921671767334 -123.00470002798046) bank125575 +125576 POINT(46.652442923805125 -122.45440758085921) bank125576 +125577 POINT(48.424490132409005 -123.30642378939218) bank125577 +125578 POINT(48.363717998534675 -122.37990277339539) bank125578 +125579 POINT(46.65621497216586 -122.43116064528313) bank125579 +125580 POINT(47.47135894055323 -122.48920367603) bank125580 +125581 POINT(46.61521937962009 -123.04953241768668) bank125581 +125582 POINT(48.3784173749951 -122.66472202128811) bank125582 +125583 POINT(46.839033081293124 -121.7797153313564) bank125583 +125584 POINT(47.352722079200724 -121.77096669644068) bank125584 +125585 POINT(47.28763395882155 -123.1671687634849) bank125585 +125586 POINT(47.18913255594684 -123.07589402247363) bank125586 +125587 POINT(48.55721896316618 -121.55938472680855) bank125587 +125588 POINT(47.936145558657294 -121.83347573844925) bank125588 +125589 POINT(47.56017419043879 -121.73361156153251) bank125589 +125590 POINT(47.94344031102482 -121.95134125035749) bank125590 +125591 POINT(46.80209144855768 -121.64717654843002) bank125591 +125592 POINT(47.60705025907057 -123.00097267771069) bank125592 +125593 POINT(48.343325127150614 -122.38611710046048) bank125593 +125594 POINT(47.42918485601044 -123.16187450073315) bank125594 +125595 POINT(47.725462623284656 -121.42007809467904) bank125595 +125596 POINT(47.05721690502984 -121.99585535191075) bank125596 +125597 POINT(47.99187619321753 -121.46112737074483) bank125597 +125598 POINT(48.00959597994659 -122.77051252361233) bank125598 +125599 POINT(48.070195136468314 -122.24010961221735) bank125599 +125600 POINT(47.86027386590725 -121.51128485798766) bank125600 +125601 POINT(46.69063146985835 -121.86264410672847) bank125601 +125602 POINT(48.467018248549074 -122.82610705478818) bank125602 +125603 POINT(47.39337454842941 -122.09126031295148) bank125603 +125604 POINT(48.60272270724683 -121.90011331883943) bank125604 +125605 POINT(46.67152795108668 -122.45532035352491) bank125605 +125606 POINT(47.00166620000406 -122.08467166830896) bank125606 +125607 POINT(48.54400701778737 -121.39638020049954) bank125607 +125608 POINT(46.95820834866388 -121.63322068095165) bank125608 +125609 POINT(48.565068953437084 -121.40724591871668) bank125609 +125610 POINT(46.88864879759865 -122.14670352261054) bank125610 +125611 POINT(47.344926811456965 -121.35191263465845) bank125611 +125612 POINT(47.7640378553016 -121.75914393828423) bank125612 +125613 POINT(47.08645846964487 -122.47609679200899) bank125613 +125614 POINT(47.70068471655397 -122.09665854799947) bank125614 +125615 POINT(47.40304548719229 -121.74357814835784) bank125615 +125616 POINT(48.510943521054735 -122.23417942039532) bank125616 +125617 POINT(48.403748970995565 -121.75180429356186) bank125617 +125618 POINT(47.13967082108349 -122.37400709972655) bank125618 +125619 POINT(47.30678069104653 -122.06167633470073) bank125619 +125620 POINT(48.16670123551246 -123.12057377720463) bank125620 +125621 POINT(48.255866477777175 -122.40207415239347) bank125621 +125622 POINT(47.84802296516074 -121.60375865770877) bank125622 +125623 POINT(47.163702975002224 -121.74352871890838) bank125623 +125624 POINT(48.05241316019173 -122.90018254970487) bank125624 +125625 POINT(47.901323896474544 -122.45311024681382) bank125625 +125626 POINT(47.998383954417925 -123.04309048734153) bank125626 +125627 POINT(47.51852055636759 -122.82579282455612) bank125627 +125628 POINT(47.369013408057874 -121.87385828130496) bank125628 +125629 POINT(48.09145358868172 -122.85446009659206) bank125629 +125630 POINT(47.28097437636278 -122.31742244781431) bank125630 +125631 POINT(47.112906863035505 -123.04407502834025) bank125631 +125632 POINT(47.46277422841054 -123.14539980428121) bank125632 +125633 POINT(47.9500336697691 -123.0495076667344) bank125633 +125634 POINT(47.60086802654624 -121.39555829124973) bank125634 +125635 POINT(48.541743319318726 -123.09058496948063) bank125635 +125636 POINT(46.79726387830621 -122.39746751793427) bank125636 +125637 POINT(46.85405301455943 -121.37360890147183) bank125637 +125638 POINT(47.088243528941746 -121.9984420424339) bank125638 +125639 POINT(47.01947366612304 -123.10489802878965) bank125639 +125640 POINT(47.22226336367646 -123.07231583205109) bank125640 +125641 POINT(48.48978446068843 -121.70076151383024) bank125641 +125642 POINT(47.24597029211166 -122.05591967171546) bank125642 +125643 POINT(48.03596345481756 -122.7111654978902) bank125643 +125644 POINT(47.342484076344846 -122.9876274734993) bank125644 +125645 POINT(48.07994980721128 -121.76199391143838) bank125645 +125646 POINT(48.37709767240412 -122.70637563452335) bank125646 +125647 POINT(47.532670044318955 -122.31738547865396) bank125647 +125648 POINT(47.427236559084434 -122.21254361492176) bank125648 +125649 POINT(46.777484065532114 -122.60304937812364) bank125649 +125650 POINT(47.6134840265097 -122.93474194808174) bank125650 +125651 POINT(48.01025506535203 -122.32855424867286) bank125651 +125652 POINT(47.5832638679394 -123.23186154678447) bank125652 +125653 POINT(46.87772310633756 -122.30894158411512) bank125653 +125654 POINT(46.99550705840954 -121.70156455814441) bank125654 +125655 POINT(47.059863329548755 -121.36686298454693) bank125655 +125656 POINT(47.207994871221935 -121.8751612129218) bank125656 +125657 POINT(47.757417569605046 -122.03719607265037) bank125657 +125658 POINT(48.22372899822023 -123.16886361090866) bank125658 +125659 POINT(46.64428636287697 -122.7220523355653) bank125659 +125660 POINT(48.16647254705311 -122.3687365431199) bank125660 +125661 POINT(48.44925586628155 -122.35200595490124) bank125661 +125662 POINT(48.39404637932242 -122.69676819470209) bank125662 +125663 POINT(48.59010977706133 -121.92053481811307) bank125663 +125664 POINT(47.38667209370885 -121.46924020935897) bank125664 +125665 POINT(47.71817983043464 -122.63136990778325) bank125665 +125666 POINT(47.02697151456161 -122.34295179116064) bank125666 +125667 POINT(46.869538960550614 -123.00435387383064) bank125667 +125668 POINT(48.40688114956429 -122.68455783691964) bank125668 +125669 POINT(46.88382970340626 -122.64319053650546) bank125669 +125670 POINT(48.32468368026846 -122.41033917743049) bank125670 +125671 POINT(48.23500793971104 -121.76087939045722) bank125671 +125672 POINT(48.420004139623764 -122.95696571730154) bank125672 +125673 POINT(46.870116106098955 -121.33374378467079) bank125673 +125674 POINT(47.76192213318505 -122.79068725205603) bank125674 +125675 POINT(46.65607788334205 -123.31094781650394) bank125675 +125676 POINT(47.07570809986173 -121.62426513437909) bank125676 +125677 POINT(48.22532736346019 -122.21250816332905) bank125677 +125678 POINT(48.2646822266831 -122.53104454897175) bank125678 +125679 POINT(46.86191365102441 -122.87863185582516) bank125679 +125680 POINT(46.954488482181034 -121.65259859224578) bank125680 +125681 POINT(47.68017460286005 -122.36104799446538) bank125681 +125682 POINT(46.957610426442365 -121.51085236462201) bank125682 +125683 POINT(46.921693283204355 -123.22071883350078) bank125683 +125684 POINT(48.181395985072825 -121.6122238277812) bank125684 +125685 POINT(47.51919330969776 -121.7488028865859) bank125685 +125686 POINT(46.796144897660724 -122.53753704075149) bank125686 +125687 POINT(47.72332257992722 -122.16371715107992) bank125687 +125688 POINT(47.30922516705755 -123.00870677567318) bank125688 +125689 POINT(47.336274542488496 -121.74134970491079) bank125689 +125690 POINT(47.087713377960284 -121.7198739663963) bank125690 +125691 POINT(48.02457384979307 -123.31820314081848) bank125691 +125692 POINT(46.863686044511326 -122.43404589211022) bank125692 +125693 POINT(48.00186225039412 -122.02350971797634) bank125693 +125694 POINT(47.174921497406196 -121.94995543410941) bank125694 +125695 POINT(48.48670888198832 -122.15162258554815) bank125695 +125696 POINT(47.679967656287076 -121.57151502165347) bank125696 +125697 POINT(47.77827636408073 -121.85073885054354) bank125697 +125698 POINT(46.843422616327814 -122.99375820873051) bank125698 +125699 POINT(47.96615443766031 -122.85916391694244) bank125699 +125700 POINT(48.14752711965459 -122.57605450588773) bank125700 +125701 POINT(47.865834196400485 -121.57462609304608) bank125701 +125702 POINT(48.34620396643966 -123.09053296581286) bank125702 +125703 POINT(47.91854701863261 -122.42567633961785) bank125703 +125704 POINT(47.94829217627268 -121.57769354745402) bank125704 +125705 POINT(48.449890218715815 -121.8341187311174) bank125705 +125706 POINT(46.79526025702489 -121.63184343868838) bank125706 +125707 POINT(47.25432313548101 -122.45620262721849) bank125707 +125708 POINT(48.499628821603636 -123.21054994356795) bank125708 +125709 POINT(48.288577278861275 -121.55464429812031) bank125709 +125710 POINT(48.13863429475373 -122.33492088647849) bank125710 +125711 POINT(46.836197039220544 -123.2422530081641) bank125711 +125712 POINT(48.2955144838205 -121.42851699859956) bank125712 +125713 POINT(46.77743879608224 -122.12533831251835) bank125713 +125714 POINT(47.00739524993402 -121.83783958409633) bank125714 +125715 POINT(47.878179261803005 -121.37891037652277) bank125715 +125716 POINT(47.458135721867755 -123.17687308693415) bank125716 +125717 POINT(48.56218006115515 -121.65538524293079) bank125717 +125718 POINT(47.945843105421346 -121.78452154869123) bank125718 +125719 POINT(47.855725752227045 -121.80848679667974) bank125719 +125720 POINT(48.10631108647799 -122.64781998777224) bank125720 +125721 POINT(48.32731312753626 -121.64602638550917) bank125721 +125722 POINT(47.205401076011235 -122.01269562669141) bank125722 +125723 POINT(46.89101606912504 -122.68733560158758) bank125723 +125724 POINT(47.910363380200074 -122.59923626980513) bank125724 +125725 POINT(48.35995121121358 -123.19282728170096) bank125725 +125726 POINT(47.45724851801656 -122.61419688048005) bank125726 +125727 POINT(46.76738974444841 -122.02607973838073) bank125727 +125728 POINT(46.751433339278655 -121.43792643833713) bank125728 +125729 POINT(47.583181413636375 -122.7421136226552) bank125729 +125730 POINT(46.79511869290225 -123.32682627774138) bank125730 +125731 POINT(48.14933632429146 -122.10617875179031) bank125731 +125732 POINT(47.95320407958855 -122.24490048207282) bank125732 +125733 POINT(48.512550635671346 -121.40654111267021) bank125733 +125734 POINT(47.70543249815182 -121.80792917408078) bank125734 +125735 POINT(47.26417782573929 -122.28666120344445) bank125735 +125736 POINT(46.89579623574198 -122.92295873690553) bank125736 +125737 POINT(47.83651023923405 -121.77616262532858) bank125737 +125738 POINT(48.09674882431294 -121.76091948634688) bank125738 +125739 POINT(47.57617034148708 -123.00952934598554) bank125739 +125740 POINT(47.60217859876666 -122.83922778255936) bank125740 +125741 POINT(46.74451971282863 -123.01500061239466) bank125741 +125742 POINT(47.92533840637701 -121.43786842475284) bank125742 +125743 POINT(48.514207771190726 -122.94488882945282) bank125743 +125744 POINT(47.36635653690574 -123.31152359135305) bank125744 +125745 POINT(46.694433536914474 -121.86857469060624) bank125745 +125746 POINT(47.70806971108907 -122.01927748251612) bank125746 +125747 POINT(48.45031380207185 -122.37867569682454) bank125747 +125748 POINT(47.00609502955182 -123.2867083684328) bank125748 +125749 POINT(46.98542948732491 -123.05926507019397) bank125749 +125750 POINT(46.79121007430783 -123.15971107583334) bank125750 +125751 POINT(47.44979640436927 -121.35630213777279) bank125751 +125752 POINT(48.18908250489664 -121.6601083666236) bank125752 +125753 POINT(47.307860844928726 -123.26334423436266) bank125753 +125754 POINT(48.21581459850796 -121.3810530131735) bank125754 +125755 POINT(47.0436403444994 -123.21532813689444) bank125755 +125756 POINT(48.0551366695986 -121.59115581708603) bank125756 +125757 POINT(47.622695645853256 -122.20949147435088) bank125757 +125758 POINT(48.42457152644977 -123.24417997518431) bank125758 +125759 POINT(47.170525609735385 -121.9375230949905) bank125759 +125760 POINT(48.4017019249777 -122.18416383662374) bank125760 +125761 POINT(46.85067920279259 -122.24763298471005) bank125761 +125762 POINT(47.32281871464808 -121.50077798955519) bank125762 +125763 POINT(46.636938440894475 -122.61370269670036) bank125763 +125764 POINT(46.88668403889553 -121.58251109907842) bank125764 +125765 POINT(47.48981351467977 -122.25235775774865) bank125765 +125766 POINT(47.70649875962833 -121.47721509140142) bank125766 +125767 POINT(46.61808517984887 -122.34059628461553) bank125767 +125768 POINT(47.89661855815107 -122.48932919998727) bank125768 +125769 POINT(48.185082503941196 -121.48556367507864) bank125769 +125770 POINT(47.43968061599754 -121.51116696629089) bank125770 +125771 POINT(47.02974674175566 -123.18495649402877) bank125771 +125772 POINT(47.480244731999605 -122.7011315583626) bank125772 +125773 POINT(48.06516282921731 -121.80555681193314) bank125773 +125774 POINT(47.533342420113044 -122.97842968749336) bank125774 +125775 POINT(48.430715174649464 -122.26088468068964) bank125775 +125776 POINT(47.32445658773235 -122.63241992850699) bank125776 +125777 POINT(47.06053188075171 -122.27376465893315) bank125777 +125778 POINT(47.28169485391809 -122.23406723628524) bank125778 +125779 POINT(46.60964227824086 -122.77842772626813) bank125779 +125780 POINT(47.819457698330076 -121.90223186696785) bank125780 +125781 POINT(46.845933466580675 -122.86824606386146) bank125781 +125782 POINT(47.819341870879256 -122.23856841827796) bank125782 +125783 POINT(47.122285426171295 -121.59261582651145) bank125783 +125784 POINT(47.34450298944028 -122.96360983500057) bank125784 +125785 POINT(47.16909680777154 -122.63433475002479) bank125785 +125786 POINT(46.70305857221958 -122.75270345225401) bank125786 +125787 POINT(46.84615109354619 -121.50058577903292) bank125787 +125788 POINT(47.30119537404807 -123.19227087284291) bank125788 +125789 POINT(48.02572350984545 -121.49244813178423) bank125789 +125790 POINT(47.45582146700744 -122.74181559085736) bank125790 +125791 POINT(46.8081226950364 -121.51013555559376) bank125791 +125792 POINT(46.78131372700645 -121.93090069073334) bank125792 +125793 POINT(48.3749124052644 -122.05530107852044) bank125793 +125794 POINT(48.05294334848991 -121.62747697856038) bank125794 +125795 POINT(47.75169950173424 -122.94235221970978) bank125795 +125796 POINT(47.853104225247314 -122.93116411029733) bank125796 +125797 POINT(48.34318892607492 -122.37639981492563) bank125797 +125798 POINT(47.34294737232825 -122.317154250492) bank125798 +125799 POINT(46.97124270027697 -122.56126029590253) bank125799 +125800 POINT(46.83619671982655 -122.08838372181613) bank125800 +125801 POINT(47.25538087801909 -121.93302267427028) bank125801 +125802 POINT(47.032928894446826 -121.80760610069761) bank125802 +125803 POINT(48.242591423989694 -123.19064857644821) bank125803 +125804 POINT(47.31878778592411 -122.47368970604896) bank125804 +125805 POINT(48.35955875796546 -123.32999459268598) bank125805 +125806 POINT(47.82983004562133 -122.08881283632164) bank125806 +125807 POINT(48.22183444536556 -121.4360621993693) bank125807 +125808 POINT(48.15708093598381 -122.0953594766971) bank125808 +125809 POINT(48.482517850270746 -121.44146106723842) bank125809 +125810 POINT(46.92005679774016 -122.42497220325826) bank125810 +125811 POINT(48.49134998430613 -122.98894798482102) bank125811 +125812 POINT(47.20441520287457 -121.97627778799148) bank125812 +125813 POINT(48.33946925033103 -123.29225425322882) bank125813 +125814 POINT(47.943561167123526 -122.8693347372878) bank125814 +125815 POINT(47.85266409531749 -122.53937974175587) bank125815 +125816 POINT(46.6208729301214 -122.93042818617772) bank125816 +125817 POINT(47.33738656809876 -121.39009777710211) bank125817 +125818 POINT(46.662250172834625 -121.6962108995386) bank125818 +125819 POINT(47.392596357366436 -121.46290700925854) bank125819 +125820 POINT(48.29333983306998 -122.4253413601363) bank125820 +125821 POINT(46.665391268258624 -122.83582610740105) bank125821 +125822 POINT(48.580859059920606 -122.32529839443418) bank125822 +125823 POINT(47.73002928451354 -123.01705816917114) bank125823 +125824 POINT(48.273857739603685 -122.62502421190237) bank125824 +125825 POINT(47.850277814501986 -122.94632361509586) bank125825 +125826 POINT(47.125183550395406 -122.7556336356672) bank125826 +125827 POINT(47.20243774694321 -122.05304668828413) bank125827 +125828 POINT(48.40790045947706 -121.6292681842947) bank125828 +125829 POINT(48.45101931664825 -121.66833574662571) bank125829 +125830 POINT(47.87773044134917 -123.2190282236705) bank125830 +125831 POINT(48.48999554916945 -123.22630351503125) bank125831 +125832 POINT(47.78574729205961 -121.58554244257725) bank125832 +125833 POINT(47.13227234143866 -121.41390316473087) bank125833 +125834 POINT(47.39250301197094 -121.35451506048958) bank125834 +125835 POINT(46.73520934563623 -122.49877584388612) bank125835 +125836 POINT(46.635481850863506 -122.12318533012082) bank125836 +125837 POINT(48.4390009181459 -121.56353539155272) bank125837 +125838 POINT(48.589283117105715 -122.56559377187281) bank125838 +125839 POINT(48.4695232596884 -122.26590248808637) bank125839 +125840 POINT(48.487852837401554 -122.80109399280862) bank125840 +125841 POINT(47.581993768794845 -122.22300973911895) bank125841 +125842 POINT(47.77618467345676 -122.81596955786397) bank125842 +125843 POINT(48.57118945613815 -123.00298937300806) bank125843 +125844 POINT(47.432085092738795 -121.36578614514517) bank125844 +125845 POINT(47.85023924856098 -122.64034484939181) bank125845 +125846 POINT(47.61323110542519 -122.28668500417372) bank125846 +125847 POINT(47.79013936658657 -123.22070704578823) bank125847 +125848 POINT(48.43195545537407 -122.38776697735278) bank125848 +125849 POINT(47.984889829686416 -122.90274269933502) bank125849 +125850 POINT(47.09518799741569 -122.15196722286635) bank125850 +125851 POINT(48.507645451593774 -121.64370580299259) bank125851 +125852 POINT(47.136151064302275 -122.13874983852875) bank125852 +125853 POINT(48.25031274697306 -123.05024214170832) bank125853 +125854 POINT(47.89727457220882 -121.33910992718688) bank125854 +125855 POINT(47.191263991930875 -122.33814119020553) bank125855 +125856 POINT(47.048122194169956 -122.04558539120684) bank125856 +125857 POINT(48.196668221886846 -123.22860863431204) bank125857 +125858 POINT(47.0240037785246 -122.00076555003474) bank125858 +125859 POINT(46.755900008929174 -122.99092866924201) bank125859 +125860 POINT(47.31131202818764 -121.46345466713835) bank125860 +125861 POINT(47.12305162489924 -123.22329075889506) bank125861 +125862 POINT(46.629730277407944 -123.01460368766168) bank125862 +125863 POINT(48.35470586120874 -122.96110132873888) bank125863 +125864 POINT(46.8769123297619 -121.76250650406367) bank125864 +125865 POINT(48.51462202937616 -122.88866280409606) bank125865 +125866 POINT(47.19941294514763 -121.97955595648268) bank125866 +125867 POINT(48.402809045000794 -121.56737601972904) bank125867 +125868 POINT(48.25072643607305 -122.5903391264296) bank125868 +125869 POINT(47.03298438592539 -121.87428154287163) bank125869 +125870 POINT(48.04928920909322 -122.59599678306303) bank125870 +125871 POINT(48.58352300889429 -121.95188909094576) bank125871 +125872 POINT(47.729285297152394 -123.16806047849445) bank125872 +125873 POINT(47.52114322214751 -123.18586638628695) bank125873 +125874 POINT(47.95328512212199 -122.54627480306407) bank125874 +125875 POINT(47.40112272488059 -121.45150209173673) bank125875 +125876 POINT(48.288806902295015 -122.10701662349972) bank125876 +125877 POINT(47.171595318159895 -122.55577926919271) bank125877 +125878 POINT(48.28025018231653 -122.49134156081298) bank125878 +125879 POINT(47.60557065372008 -121.54307395640411) bank125879 +125880 POINT(46.85621729600058 -122.10005804556351) bank125880 +125881 POINT(47.23795473711893 -123.21906681896586) bank125881 +125882 POINT(47.298886768356965 -122.33244444696348) bank125882 +125883 POINT(48.24588755164026 -122.8381718820554) bank125883 +125884 POINT(47.0741768990888 -121.40177526341289) bank125884 +125885 POINT(48.33990728462105 -121.70902357019975) bank125885 +125886 POINT(47.40453579997993 -121.8358505509017) bank125886 +125887 POINT(48.22052703552942 -123.06322124286433) bank125887 +125888 POINT(47.90997878083414 -122.33741769666568) bank125888 +125889 POINT(47.017339373817855 -122.30075443832911) bank125889 +125890 POINT(47.600841937927974 -122.06006519980973) bank125890 +125891 POINT(47.6583058531389 -122.88684194725164) bank125891 +125892 POINT(47.76879335709587 -122.45034918334954) bank125892 +125893 POINT(47.20928796569569 -123.09775056722188) bank125893 +125894 POINT(47.968400814645584 -122.08062308810864) bank125894 +125895 POINT(46.7394463856786 -122.69476072925306) bank125895 +125896 POINT(48.458276060192716 -122.89634468660013) bank125896 +125897 POINT(48.15691725588723 -123.24866909388076) bank125897 +125898 POINT(48.30243025449634 -122.86757249609343) bank125898 +125899 POINT(46.75457181883223 -122.29061046034359) bank125899 +125900 POINT(47.886647968837295 -122.05805400010479) bank125900 +125901 POINT(47.497487688914575 -123.27683441157863) bank125901 +125902 POINT(48.4155614665528 -121.76720338654111) bank125902 +125903 POINT(47.0505358245146 -121.53743963188164) bank125903 +125904 POINT(47.0154321058863 -122.58318836955577) bank125904 +125905 POINT(47.90566227497442 -122.0629750451979) bank125905 +125906 POINT(48.0889594160846 -121.42341306279687) bank125906 +125907 POINT(47.668628225727716 -121.4561134454805) bank125907 +125908 POINT(47.71979409132597 -122.29691030758673) bank125908 +125909 POINT(47.35425013408479 -123.28331775029922) bank125909 +125910 POINT(47.922673896888476 -122.59440363385455) bank125910 +125911 POINT(48.12789861538332 -123.12906010207149) bank125911 +125912 POINT(47.08884603175583 -121.61772500815677) bank125912 +125913 POINT(48.44897631379158 -121.90973218169547) bank125913 +125914 POINT(48.03135824822465 -123.24466416498089) bank125914 +125915 POINT(48.14972575054158 -121.86938677951582) bank125915 +125916 POINT(47.53669486936259 -122.59141533086829) bank125916 +125917 POINT(47.393480452972234 -123.23999123755333) bank125917 +125918 POINT(47.38040379843034 -122.46372289694075) bank125918 +125919 POINT(47.36205206593541 -122.6800003975782) bank125919 +125920 POINT(47.09844242649997 -121.65508270523252) bank125920 +125921 POINT(48.268681568590914 -123.03552621830147) bank125921 +125922 POINT(47.970795985045875 -123.09810698912916) bank125922 +125923 POINT(48.0724933654151 -122.27461832539512) bank125923 +125924 POINT(47.57401880811229 -121.81662637406272) bank125924 +125925 POINT(47.714519231649206 -121.90217701250985) bank125925 +125926 POINT(48.124430728069655 -122.41720153442003) bank125926 +125927 POINT(48.445323233241716 -122.29092036935589) bank125927 +125928 POINT(47.8285576145229 -122.11665354736533) bank125928 +125929 POINT(48.08791199004308 -121.58355717251663) bank125929 +125930 POINT(48.29708642737171 -123.03087839849836) bank125930 +125931 POINT(47.76397830323844 -122.88807626344502) bank125931 +125932 POINT(46.73250197877487 -122.76473381575312) bank125932 +125933 POINT(47.99617431045528 -121.65243614124167) bank125933 +125934 POINT(48.273981581476704 -122.78035680043625) bank125934 +125935 POINT(47.498325316902275 -122.44708501039814) bank125935 +125936 POINT(46.8867943007631 -122.65574738882053) bank125936 +125937 POINT(48.446359903235354 -121.56237524167493) bank125937 +125938 POINT(48.60212151669916 -121.78155488337417) bank125938 +125939 POINT(46.78432150205585 -122.76920482035261) bank125939 +125940 POINT(46.691026061967854 -122.01716262382902) bank125940 +125941 POINT(47.068823231776996 -122.64315917940812) bank125941 +125942 POINT(46.97810918783854 -121.63461640974427) bank125942 +125943 POINT(46.858391042564946 -123.23786988005455) bank125943 +125944 POINT(47.257647126322205 -121.53147632284586) bank125944 +125945 POINT(47.82185099730958 -122.84522973677181) bank125945 +125946 POINT(48.29441238037341 -123.04466619893644) bank125946 +125947 POINT(46.977108925991544 -123.28812794259731) bank125947 +125948 POINT(47.471424438898076 -123.20098750653997) bank125948 +125949 POINT(46.80901128045068 -122.84120426575122) bank125949 +125950 POINT(46.78368514410203 -122.56192565349926) bank125950 +125951 POINT(46.79746497771238 -122.76313585476704) bank125951 +125952 POINT(48.01563591564324 -122.76386865952368) bank125952 +125953 POINT(47.570829446249775 -121.72438154889348) bank125953 +125954 POINT(46.69862110966848 -121.81616274709587) bank125954 +125955 POINT(47.63200360289268 -122.76084857532051) bank125955 +125956 POINT(48.57589984912045 -122.71891494421847) bank125956 +125957 POINT(48.545690149284226 -123.020885177424) bank125957 +125958 POINT(48.33168839458452 -123.31108896469694) bank125958 +125959 POINT(48.57338728931004 -122.95039469365926) bank125959 +125960 POINT(47.10073579084879 -122.62944720721535) bank125960 +125961 POINT(47.478617762439086 -121.65792449945772) bank125961 +125962 POINT(46.890359445200836 -123.0689367471027) bank125962 +125963 POINT(47.06448667609272 -122.99097729025931) bank125963 +125964 POINT(47.355142400800545 -122.86035461601874) bank125964 +125965 POINT(48.2780798704271 -122.84393121798539) bank125965 +125966 POINT(48.285046912463805 -122.03596009651308) bank125966 +125967 POINT(47.601090241564826 -122.80430055674772) bank125967 +125968 POINT(46.87919754774385 -123.12276680035343) bank125968 +125969 POINT(47.84879726271203 -121.37066370303936) bank125969 +125970 POINT(46.91613807085931 -122.87605837942753) bank125970 +125971 POINT(47.28494639637105 -122.1033734207695) bank125971 +125972 POINT(48.023304711830114 -121.75942335424935) bank125972 +125973 POINT(47.0402655393886 -121.72978238080674) bank125973 +125974 POINT(47.93551490494899 -121.9369372890741) bank125974 +125975 POINT(48.456605497391216 -122.83484993387043) bank125975 +125976 POINT(47.000957641274695 -123.05301843986352) bank125976 +125977 POINT(48.59533297884732 -121.3759188988827) bank125977 +125978 POINT(47.5134800263651 -123.12875256767023) bank125978 +125979 POINT(48.35943821645632 -123.20207118691634) bank125979 +125980 POINT(47.71292934452562 -122.35390543948573) bank125980 +125981 POINT(47.404281992528155 -122.99301348083135) bank125981 +125982 POINT(47.28473158040686 -122.64945361325972) bank125982 +125983 POINT(48.555217665092776 -121.49776260127503) bank125983 +125984 POINT(47.00625143295966 -121.580538261892) bank125984 +125985 POINT(47.62622159159106 -122.7675171613942) bank125985 +125986 POINT(47.01135590324879 -122.58144096029439) bank125986 +125987 POINT(48.16553916709819 -122.25047681554618) bank125987 +125988 POINT(47.55774464604709 -121.50331597341352) bank125988 +125989 POINT(48.29760823404013 -122.66621082350866) bank125989 +125990 POINT(48.156284153756516 -122.45259658763622) bank125990 +125991 POINT(48.4584888492919 -123.0152635398263) bank125991 +125992 POINT(47.8176715132847 -121.5849228876595) bank125992 +125993 POINT(48.51462735539466 -122.86706257526008) bank125993 +125994 POINT(46.692343778043636 -122.85594587199152) bank125994 +125995 POINT(48.10378254689664 -122.6021674317085) bank125995 +125996 POINT(47.41121677433844 -121.55022591736629) bank125996 +125997 POINT(48.586231367280305 -121.8018930790745) bank125997 +125998 POINT(47.41505340907116 -122.88282760038639) bank125998 +125999 POINT(47.546720710523026 -122.23659498710444) bank125999 +126000 POINT(47.10410306260994 -123.22725013001525) bank126000 +126001 POINT(47.17613688350948 -122.09282167988601) bank126001 +126002 POINT(47.641646478773424 -121.78052443714694) bank126002 +126003 POINT(48.03517600696037 -121.88467171860509) bank126003 +126004 POINT(46.720372238614324 -122.19670912752866) bank126004 +126005 POINT(47.814373522769955 -122.18258164564197) bank126005 +126006 POINT(47.755005579845175 -122.67145218677342) bank126006 +126007 POINT(48.50405415612774 -122.125304186859) bank126007 +126008 POINT(47.9270018865423 -121.48678609065665) bank126008 +126009 POINT(47.110398733516625 -122.58347649028386) bank126009 +126010 POINT(48.309539697880226 -121.45683200256885) bank126010 +126011 POINT(47.97719171610734 -121.57976948427856) bank126011 +126012 POINT(47.996735907578866 -122.81324037351479) bank126012 +126013 POINT(47.52498817825139 -121.71413619268888) bank126013 +126014 POINT(47.8608648431159 -123.08167013934624) bank126014 +126015 POINT(46.92903114095423 -122.80728293776116) bank126015 +126016 POINT(48.33457977834698 -121.85926697295108) bank126016 +126017 POINT(47.00407903436937 -122.24422101278547) bank126017 +126018 POINT(47.844272057077845 -121.86003308519913) bank126018 +126019 POINT(47.3228014196429 -122.91014854238843) bank126019 +126020 POINT(47.21541056712548 -122.56281892376545) bank126020 +126021 POINT(46.976364671059116 -121.479448150876) bank126021 +126022 POINT(47.98772991594787 -121.87978963757031) bank126022 +126023 POINT(48.00266150414742 -121.95994762680213) bank126023 +126024 POINT(48.50791292303761 -123.11845466815524) bank126024 +126025 POINT(48.22913387242153 -122.21217434156918) bank126025 +126026 POINT(47.29697145338908 -122.82211543461138) bank126026 +126027 POINT(48.36145313298001 -121.95771519953901) bank126027 +126028 POINT(48.39377185954291 -121.67477578352212) bank126028 +126029 POINT(48.13559876261228 -122.1562340321549) bank126029 +126030 POINT(46.87349185569292 -122.60977901522405) bank126030 +126031 POINT(48.114794862461416 -123.28727184772417) bank126031 +126032 POINT(48.55572762807556 -121.550288898874) bank126032 +126033 POINT(47.453320881820936 -123.01338215469397) bank126033 +126034 POINT(48.09773353169758 -122.72340534506672) bank126034 +126035 POINT(47.39973689423202 -122.77603039972784) bank126035 +126036 POINT(46.76763691015243 -123.00623975272848) bank126036 +126037 POINT(48.18142313911558 -121.511181945626) bank126037 +126038 POINT(47.597744326490115 -122.36639478450822) bank126038 +126039 POINT(47.6808326168352 -122.36996155993498) bank126039 +126040 POINT(47.395527957633725 -121.95921180316539) bank126040 +126041 POINT(46.84438464954053 -123.25989213697387) bank126041 +126042 POINT(48.44113261986217 -123.2447638445704) bank126042 +126043 POINT(46.95246143818291 -122.21513335508143) bank126043 +126044 POINT(47.62000166912728 -121.88783726924147) bank126044 +126045 POINT(46.722888884923854 -121.51205685715753) bank126045 +126046 POINT(47.52232930820763 -123.28324474606053) bank126046 +126047 POINT(46.752681821597285 -122.86455636026695) bank126047 +126048 POINT(48.5796732887157 -122.33971215979241) bank126048 +126049 POINT(47.863186486506606 -123.19252538866958) bank126049 +126050 POINT(48.5104057334928 -122.8122069678553) bank126050 +126051 POINT(47.799877568788915 -122.14989345854828) bank126051 +126052 POINT(47.98551931897688 -121.52901140092864) bank126052 +126053 POINT(47.56215898498252 -123.05709284936442) bank126053 +126054 POINT(46.96454284533816 -123.25275952196476) bank126054 +126055 POINT(48.267609666590964 -121.80021748815409) bank126055 +126056 POINT(48.489558062722296 -122.66863101734492) bank126056 +126057 POINT(48.495868412576584 -121.94792463215306) bank126057 +126058 POINT(47.287279278243695 -122.02451260077909) bank126058 +126059 POINT(48.27012677248911 -122.32086368413873) bank126059 +126060 POINT(48.60010741916452 -122.87183948846094) bank126060 +126061 POINT(47.01232539931714 -122.19711244654279) bank126061 +126062 POINT(47.19476691693999 -121.85121320130989) bank126062 +126063 POINT(48.60452603254902 -122.11190540209213) bank126063 +126064 POINT(46.867833647839205 -123.15119756767778) bank126064 +126065 POINT(48.25894931647784 -122.40337514810025) bank126065 +126066 POINT(48.479525138514816 -122.23528693081991) bank126066 +126067 POINT(47.37558829098967 -122.88188663734384) bank126067 +126068 POINT(47.45367635919766 -122.57997860326245) bank126068 +126069 POINT(46.917736071273914 -122.8855373558151) bank126069 +126070 POINT(48.23653337444727 -121.77948431255803) bank126070 +126071 POINT(48.114435259250016 -122.00877934877089) bank126071 +126072 POINT(47.836092890420694 -123.15373586422768) bank126072 +126073 POINT(47.35355325880828 -121.64361457705618) bank126073 +126074 POINT(46.791698509612274 -122.01913985313779) bank126074 +126075 POINT(46.62210109205612 -122.86936993444186) bank126075 +126076 POINT(47.48125877813903 -123.17091452558536) bank126076 +126077 POINT(46.7924879226401 -122.47211557746706) bank126077 +126078 POINT(46.76486114278557 -121.77207437458438) bank126078 +126079 POINT(47.457186976484884 -121.61426857752433) bank126079 +126080 POINT(47.652923092286535 -122.23550379502456) bank126080 +126081 POINT(48.207796613414885 -122.09023410508478) bank126081 +126082 POINT(46.96319251768307 -123.09508773982729) bank126082 +126083 POINT(48.04309632931605 -122.34067858005164) bank126083 +126084 POINT(48.26747072998636 -122.39572206953366) bank126084 +126085 POINT(48.052812920651974 -121.45226981113571) bank126085 +126086 POINT(47.54533381080147 -122.83305831143336) bank126086 +126087 POINT(47.17641809556331 -121.48892087832255) bank126087 +126088 POINT(47.99742764669046 -122.0828198943411) bank126088 +126089 POINT(48.2989549221807 -123.3064326657737) bank126089 +126090 POINT(47.60974068589775 -121.38401025261811) bank126090 +126091 POINT(46.612919863506576 -122.892065341729) bank126091 +126092 POINT(46.76876574616236 -123.0456840820354) bank126092 +126093 POINT(48.117758603165356 -121.40662239298753) bank126093 +126094 POINT(47.610530095953905 -122.330895450619) bank126094 +126095 POINT(46.7978490176322 -121.77882340306908) bank126095 +126096 POINT(46.68372326357254 -123.0621538713112) bank126096 +126097 POINT(47.165616249593924 -122.22050584134101) bank126097 +126098 POINT(46.62338552467498 -121.37993739541122) bank126098 +126099 POINT(48.571775465160336 -122.75493444259126) bank126099 +126100 POINT(48.234909759931305 -123.27039662815163) bank126100 +126101 POINT(48.58023953128522 -122.61561950644963) bank126101 +126102 POINT(46.925441718854444 -121.47050596569763) bank126102 +126103 POINT(47.468413756651266 -121.64935339285832) bank126103 +126104 POINT(48.07592597935615 -122.54127663993695) bank126104 +126105 POINT(47.305963905817165 -123.18664634397186) bank126105 +126106 POINT(46.667685400818165 -121.87439964129888) bank126106 +126107 POINT(46.639549335486336 -123.2923436057531) bank126107 +126108 POINT(47.08965615157514 -122.29780018401253) bank126108 +126109 POINT(47.8169723434152 -122.84795607438275) bank126109 +126110 POINT(48.207213935782256 -122.65947044034209) bank126110 +126111 POINT(47.49719448964824 -121.45920548183315) bank126111 +126112 POINT(48.36881534238939 -123.07657013749899) bank126112 +126113 POINT(47.49455191856867 -121.4379828041064) bank126113 +126114 POINT(46.63058423544568 -122.14738452579998) bank126114 +126115 POINT(48.161388203056795 -122.87524516278589) bank126115 +126116 POINT(47.61796657804635 -122.4034631588481) bank126116 +126117 POINT(48.01997011857013 -122.51321899458608) bank126117 +126118 POINT(47.40850842810845 -122.23049551062617) bank126118 +126119 POINT(46.75127349565552 -122.16307016180706) bank126119 +126120 POINT(48.33021523074035 -122.3125722163573) bank126120 +126121 POINT(48.58294709453863 -122.87208740234372) bank126121 +126122 POINT(46.87535659731155 -121.58574527805278) bank126122 +126123 POINT(48.58102558328492 -121.77466990939557) bank126123 +126124 POINT(47.102731178461234 -123.10891954351584) bank126124 +126125 POINT(48.325672700757806 -123.20177366052181) bank126125 +126126 POINT(48.289770286988514 -122.01947844358959) bank126126 +126127 POINT(47.36876393165628 -122.55778328104286) bank126127 +126128 POINT(47.02964297443226 -122.63896475946989) bank126128 +126129 POINT(48.01424013114674 -122.02050163186345) bank126129 +126130 POINT(47.94649773573212 -121.48688547935681) bank126130 +126131 POINT(48.36477984150911 -122.87031550605239) bank126131 +126132 POINT(47.19073026166821 -121.83609642004356) bank126132 +126133 POINT(48.51296986572805 -122.55485042052574) bank126133 +126134 POINT(48.208016723551765 -122.96384659055153) bank126134 +126135 POINT(47.208297771157895 -122.32379080616414) bank126135 +126136 POINT(47.522877713379835 -122.7186497499477) bank126136 +126137 POINT(47.051149930605284 -121.87046817039653) bank126137 +126138 POINT(47.401458698949625 -121.36763297760476) bank126138 +126139 POINT(46.935585332150886 -123.28486755873936) bank126139 +126140 POINT(48.320800908443935 -121.4429274477579) bank126140 +126141 POINT(47.266201215698715 -121.93298936429375) bank126141 +126142 POINT(47.497939551558595 -122.0405015618265) bank126142 +126143 POINT(47.02451493979724 -121.80400907583974) bank126143 +126144 POINT(48.24319220567188 -122.3387218231231) bank126144 +126145 POINT(46.779446079886974 -122.48458107801133) bank126145 +126146 POINT(47.25179869358923 -122.45264587721462) bank126146 +126147 POINT(48.20346266881356 -123.22557025080769) bank126147 +126148 POINT(46.94240803416454 -122.62147194692636) bank126148 +126149 POINT(46.770806438846364 -121.51983999090571) bank126149 +126150 POINT(47.71666935740137 -121.87545084774739) bank126150 +126151 POINT(46.933332909881116 -123.11260303102094) bank126151 +126152 POINT(47.7519913913077 -121.62714607752957) bank126152 +126153 POINT(47.95620140830689 -122.5396839297609) bank126153 +126154 POINT(48.3808951199471 -121.3960410352735) bank126154 +126155 POINT(47.19032954822943 -122.6574853807211) bank126155 +126156 POINT(47.72197453903415 -122.30370152277658) bank126156 +126157 POINT(47.36315501641171 -121.6500071734532) bank126157 +126158 POINT(46.640745185297625 -122.72453596611334) bank126158 +126159 POINT(48.325565336027964 -122.25185189485123) bank126159 +126160 POINT(46.76514565656885 -121.68637094704043) bank126160 +126161 POINT(48.26621263277891 -122.21968767327557) bank126161 +126162 POINT(47.911506567789594 -121.87777443573218) bank126162 +126163 POINT(47.98802449792354 -122.93479215710067) bank126163 +126164 POINT(47.9729713509499 -121.61619175079139) bank126164 +126165 POINT(46.75702899123385 -122.45635854477312) bank126165 +126166 POINT(47.63030209395838 -123.11179351124191) bank126166 +126167 POINT(47.980133232145874 -122.04320672379305) bank126167 +126168 POINT(48.35647606161034 -123.02740750989929) bank126168 +126169 POINT(48.245693985643186 -121.73095331993522) bank126169 +126170 POINT(47.95918882573582 -121.9982937952631) bank126170 +126171 POINT(46.8487681805902 -121.34093033877559) bank126171 +126172 POINT(48.06014735411754 -123.20166251743167) bank126172 +126173 POINT(47.98877366885128 -121.43648517816145) bank126173 +126174 POINT(46.7442449900787 -122.12112889597746) bank126174 +126175 POINT(48.15957996465801 -122.25468513124281) bank126175 +126176 POINT(47.89322384063735 -121.8451862346686) bank126176 +126177 POINT(47.03893253519539 -122.4525022167553) bank126177 +126178 POINT(47.02009547935051 -123.20327788519825) bank126178 +126179 POINT(47.44131016120851 -123.20963076979281) bank126179 +126180 POINT(46.67893794085623 -121.59412296828046) bank126180 +126181 POINT(48.57481484092033 -123.15382919372583) bank126181 +126182 POINT(47.42934841022485 -122.35438962751128) bank126182 +126183 POINT(48.51766522188136 -122.9355049475673) bank126183 +126184 POINT(47.16270803527018 -123.0885659665586) bank126184 +126185 POINT(47.90678692269102 -122.58611846129969) bank126185 +126186 POINT(47.867092327126194 -123.04595748861922) bank126186 +126187 POINT(48.31199619191692 -122.10545573698869) bank126187 +126188 POINT(47.70618316105236 -122.19107796278051) bank126188 +126189 POINT(48.147104409219246 -122.89102262935727) bank126189 +126190 POINT(47.71207284216792 -122.53709971235399) bank126190 +126191 POINT(47.41896778325099 -122.0579217208521) bank126191 +126192 POINT(47.571941704500695 -122.20263608718876) bank126192 +126193 POINT(46.765172597231896 -123.21978705225631) bank126193 +126194 POINT(46.68383114750333 -123.3256472774911) bank126194 +126195 POINT(47.3534571402525 -122.50200416654256) bank126195 +126196 POINT(46.69249010006105 -123.0469114046482) bank126196 +126197 POINT(46.64382925680232 -122.87850947428055) bank126197 +126198 POINT(47.015704754511766 -122.49470636350178) bank126198 +126199 POINT(47.6760821475891 -121.34059492200147) bank126199 +126200 POINT(47.89261836724455 -123.1729600660207) bank126200 +126201 POINT(47.375554051576174 -122.21798919393923) bank126201 +126202 POINT(47.02132439616597 -122.98693476958728) bank126202 +126203 POINT(46.826020876406425 -122.37467073939092) bank126203 +126204 POINT(48.05160678193517 -122.53241650914491) bank126204 +126205 POINT(46.678314979994205 -121.91652868915425) bank126205 +126206 POINT(47.10673713996748 -122.51964083482342) bank126206 +126207 POINT(46.61269188840791 -121.39252904562628) bank126207 +126208 POINT(46.77277873734547 -121.83685594581605) bank126208 +126209 POINT(47.887335265981186 -121.71052591845742) bank126209 +126210 POINT(48.48695387692858 -122.96374394209386) bank126210 +126211 POINT(48.249734038902 -122.20939931271903) bank126211 +126212 POINT(47.72416156212438 -122.90951668270806) bank126212 +126213 POINT(48.53368609555975 -122.64404320438027) bank126213 +126214 POINT(47.000123019730616 -122.00725648156468) bank126214 +126215 POINT(47.64879046632228 -121.49079478046923) bank126215 +126216 POINT(47.14900860894525 -123.10351147870514) bank126216 +126217 POINT(48.18478422671616 -121.42517963884526) bank126217 +126218 POINT(48.57588713545037 -123.22371700387441) bank126218 +126219 POINT(47.540072324162104 -122.9272230070391) bank126219 +126220 POINT(48.25603295118945 -121.89379413773321) bank126220 +126221 POINT(48.496361729711126 -121.50543652411899) bank126221 +126222 POINT(48.269724329291016 -122.1988736913932) bank126222 +126223 POINT(46.824621509711605 -122.17159042770139) bank126223 +126224 POINT(47.23502329209789 -122.8505780541197) bank126224 +126225 POINT(46.91268788737456 -121.50309757988235) bank126225 +126226 POINT(48.11198566808467 -122.55398484193991) bank126226 +126227 POINT(47.47907357545204 -122.00506805188311) bank126227 +126228 POINT(48.096950673825305 -123.18228463890772) bank126228 +126229 POINT(48.43099862757739 -123.23331790881618) bank126229 +126230 POINT(47.02150267408905 -121.39075912544912) bank126230 +126231 POINT(47.505046861707854 -122.92934760443181) bank126231 +126232 POINT(48.28155676188129 -122.66518388078045) bank126232 +126233 POINT(48.1275016292427 -122.92675276363494) bank126233 +126234 POINT(47.75526697227184 -122.33227944010177) bank126234 +126235 POINT(47.456887689018956 -122.54185645908407) bank126235 +126236 POINT(46.968518197740096 -122.68020972638222) bank126236 +126237 POINT(46.92231904329789 -122.7567104518186) bank126237 +126238 POINT(47.59856807249759 -121.87205876958048) bank126238 +126239 POINT(46.73905331714133 -122.53336976930218) bank126239 +126240 POINT(48.088759613599265 -121.45591311560395) bank126240 +126241 POINT(47.96388025863485 -122.1486752099726) bank126241 +126242 POINT(48.11752982905801 -122.8808114457848) bank126242 +126243 POINT(47.418986450419816 -123.0618379113104) bank126243 +126244 POINT(48.1196788313166 -122.34820620016511) bank126244 +126245 POINT(48.24328432397726 -122.26281908442157) bank126245 +126246 POINT(48.32375125838199 -122.5390178132244) bank126246 +126247 POINT(46.734158858370655 -122.1009710171496) bank126247 +126248 POINT(46.67194426445184 -122.25280372120912) bank126248 +126249 POINT(46.69462213966639 -122.25630813945439) bank126249 +126250 POINT(46.79605017995854 -123.13808461017784) bank126250 +126251 POINT(46.68323546537661 -122.26608898980878) bank126251 +126252 POINT(47.032766934085195 -122.1079063298549) bank126252 +126253 POINT(47.26951240606805 -122.9576234413087) bank126253 +126254 POINT(47.47733535672225 -123.30237576732245) bank126254 +126255 POINT(47.75812398647576 -121.54871209029808) bank126255 +126256 POINT(47.35571547968211 -122.21014525139879) bank126256 +126257 POINT(47.42246015766354 -122.27310421739223) bank126257 +126258 POINT(46.86731484405309 -122.1296008976251) bank126258 +126259 POINT(48.34034100893335 -123.07254717344328) bank126259 +126260 POINT(46.6340447523406 -121.5933078364829) bank126260 +126261 POINT(47.280332112356035 -122.59053124218563) bank126261 +126262 POINT(48.522191711505656 -121.45440224822192) bank126262 +126263 POINT(48.51881299531775 -122.75127741321342) bank126263 +126264 POINT(48.21162668147054 -121.58869314942686) bank126264 +126265 POINT(47.98144872579891 -122.77986217176318) bank126265 +126266 POINT(47.46450202935744 -122.77157230372877) bank126266 +126267 POINT(48.100910595472044 -122.427825616463) bank126267 +126268 POINT(47.762021668672936 -121.46111330336369) bank126268 +126269 POINT(48.58520638781246 -122.53773087744932) bank126269 +126270 POINT(47.534994235264904 -121.99019779206562) bank126270 +126271 POINT(47.96675538025184 -122.36137254737781) bank126271 +126272 POINT(48.373455793952104 -122.5549363293821) bank126272 +126273 POINT(47.75247849639837 -121.60492272892665) bank126273 +126274 POINT(47.56951810280505 -121.76422001998368) bank126274 +126275 POINT(47.604165374364456 -121.76046498035052) bank126275 +126276 POINT(47.80625192207574 -123.16029948698242) bank126276 +126277 POINT(46.6230574267966 -121.5179126453829) bank126277 +126278 POINT(48.172179027380174 -121.44441879704313) bank126278 +126279 POINT(46.89720444401089 -121.78488172210328) bank126279 +126280 POINT(47.73713428513796 -121.78815462898626) bank126280 +126281 POINT(46.74104517690877 -121.93277617867425) bank126281 +126282 POINT(47.59582720339834 -121.4457497585314) bank126282 +126283 POINT(47.09133915320095 -121.69823934076578) bank126283 +126284 POINT(46.79111376479783 -122.3164290908741) bank126284 +126285 POINT(48.2625113806213 -122.45052575628145) bank126285 +126286 POINT(47.88283868108408 -122.32329059615736) bank126286 +126287 POINT(47.58770924131654 -121.48922373215247) bank126287 +126288 POINT(48.319880603068526 -122.67175393363293) bank126288 +126289 POINT(46.7105187165135 -122.38719375767373) bank126289 +126290 POINT(46.959558903107116 -122.12981610150761) bank126290 +126291 POINT(48.05556987002694 -123.12495191904625) bank126291 +126292 POINT(46.72989299283313 -123.18338007513906) bank126292 +126293 POINT(46.654236460093834 -121.41653250052846) bank126293 +126294 POINT(47.66701843770285 -121.82854107894681) bank126294 +126295 POINT(47.82587342792367 -123.18511851260136) bank126295 +126296 POINT(48.19577438059759 -122.54371348478924) bank126296 +126297 POINT(48.25506045347565 -122.53209937761683) bank126297 +126298 POINT(46.98320536945371 -121.9338155223007) bank126298 +126299 POINT(48.080637313913904 -122.65830633912545) bank126299 +126300 POINT(48.06755049081588 -122.41387195178469) bank126300 +126301 POINT(48.09848823946884 -122.50942993870905) bank126301 +126302 POINT(46.82526610072402 -123.22707955227021) bank126302 +126303 POINT(47.766875502158584 -122.03447839057833) bank126303 +126304 POINT(47.88296887236856 -122.7948658107793) bank126304 +126305 POINT(47.55402230716921 -122.36772722208148) bank126305 +126306 POINT(47.81998204847969 -121.7561377358055) bank126306 +126307 POINT(47.6470439720245 -123.28193911985466) bank126307 +126308 POINT(47.397305242542 -122.02154936796074) bank126308 +126309 POINT(48.16653430310263 -123.21426671642097) bank126309 +126310 POINT(46.99279567120832 -122.0477203896144) bank126310 +126311 POINT(47.399029879909754 -121.72520345315522) bank126311 +126312 POINT(46.645822351709356 -122.62233959087132) bank126312 +126313 POINT(47.60681868478844 -122.35001045436752) bank126313 +126314 POINT(48.60530043552039 -122.09979598443567) bank126314 +126315 POINT(47.51145825193271 -121.39111574100606) bank126315 +126316 POINT(46.71959875829873 -122.35799304239129) bank126316 +126317 POINT(47.35052514362499 -123.0137058460697) bank126317 +126318 POINT(48.26198422561939 -122.59864301208484) bank126318 +126319 POINT(47.07336652848801 -121.80375218775406) bank126319 +126320 POINT(46.866070363500235 -122.23806349990484) bank126320 +126321 POINT(47.02857446013889 -122.79510032997564) bank126321 +126322 POINT(46.919264508930816 -123.20668350797497) bank126322 +126323 POINT(48.33169241046815 -121.66799136626905) bank126323 +126324 POINT(48.19164393756656 -122.3761895222715) bank126324 +126325 POINT(48.090374361813645 -123.1034693363316) bank126325 +126326 POINT(48.37708243219549 -122.36632936705368) bank126326 +126327 POINT(48.25488497457173 -122.60499087656227) bank126327 +126328 POINT(47.56240587659755 -121.34976365389664) bank126328 +126329 POINT(47.75834674250872 -121.82457065332345) bank126329 +126330 POINT(47.31025729138408 -122.48119222265471) bank126330 +126331 POINT(48.3102186644449 -121.60194320979899) bank126331 +126332 POINT(46.957268230434224 -122.59960848870975) bank126332 +126333 POINT(47.75863129400581 -122.88276886598813) bank126333 +126334 POINT(47.694311754405824 -122.3508485902577) bank126334 +126335 POINT(46.892236871077074 -121.43003077975227) bank126335 +126336 POINT(47.29212140074762 -121.80863026038045) bank126336 +126337 POINT(47.89361117316392 -123.02353590783588) bank126337 +126338 POINT(48.00755135845985 -121.99328299938604) bank126338 +126339 POINT(47.0700757193506 -121.48577843854505) bank126339 +126340 POINT(47.043389291007955 -122.3910952081951) bank126340 +126341 POINT(48.05032462410246 -122.6148719952429) bank126341 +126342 POINT(47.33255604103894 -123.09895854370758) bank126342 +126343 POINT(47.08699765058017 -121.71072236456315) bank126343 +126344 POINT(47.85151715092009 -123.3273869923402) bank126344 +126345 POINT(46.888184527254104 -121.90517552763725) bank126345 +126346 POINT(47.67973320808509 -122.69299243747952) bank126346 +126347 POINT(46.807058261917895 -123.21289447044) bank126347 +126348 POINT(47.6965038301533 -123.00393183652993) bank126348 +126349 POINT(48.219692523283015 -121.6561895178038) bank126349 +126350 POINT(47.99883364168334 -121.90904964422826) bank126350 +126351 POINT(46.61631779016425 -121.48866589683611) bank126351 +126352 POINT(47.05834701511417 -123.18038414622059) bank126352 +126353 POINT(48.0123855205246 -122.66182115899855) bank126353 +126354 POINT(46.839575178536 -122.41007183041471) bank126354 +126355 POINT(47.51015885359891 -122.41347638824764) bank126355 +126356 POINT(47.06223767469927 -122.70998864897606) bank126356 +126357 POINT(47.09036129645824 -122.97697023424429) bank126357 +126358 POINT(47.324755100983786 -123.18947753775443) bank126358 +126359 POINT(48.22771916316154 -122.24185841387023) bank126359 +126360 POINT(48.11291978807744 -122.30605425077647) bank126360 +126361 POINT(47.86728621082523 -123.04873291541321) bank126361 +126362 POINT(47.644927312849966 -123.29397957769415) bank126362 +126363 POINT(47.309997645370466 -122.30082459209447) bank126363 +126364 POINT(47.36789777217044 -122.7186847693131) bank126364 +126365 POINT(48.14007571991104 -122.74521063342233) bank126365 +126366 POINT(47.907411686882725 -123.08432047244816) bank126366 +126367 POINT(47.59167292702206 -121.89543060090695) bank126367 +126368 POINT(47.32483399546199 -121.78994452182958) bank126368 +126369 POINT(47.19703681307166 -122.00829918831677) bank126369 +126370 POINT(47.72073837289034 -122.75404303348851) bank126370 +126371 POINT(46.633582442058696 -121.53075186003666) bank126371 +126372 POINT(48.05697690863784 -121.61882159880909) bank126372 +126373 POINT(46.96698012878862 -121.55597629653062) bank126373 +126374 POINT(47.702317596971234 -123.20333158554288) bank126374 +126375 POINT(47.00828196272631 -122.98482507737076) bank126375 +126376 POINT(46.648860784642466 -122.52654492640164) bank126376 +126377 POINT(46.85168950676944 -122.48591104665024) bank126377 +126378 POINT(48.177785377681204 -123.31088733149056) bank126378 +126379 POINT(46.65659521254424 -123.08949227672207) bank126379 +126380 POINT(48.34924285281695 -122.22055421263063) bank126380 +126381 POINT(46.85728845231333 -122.62081514874272) bank126381 +126382 POINT(46.972916708588414 -122.14115280580354) bank126382 +126383 POINT(47.001280940130265 -122.63591184230474) bank126383 +126384 POINT(46.96674879707276 -122.98813596679295) bank126384 +126385 POINT(47.22963941222136 -122.90751461285566) bank126385 +126386 POINT(47.99491711463522 -122.37559896829451) bank126386 +126387 POINT(48.301621223479195 -122.84785082516781) bank126387 +126388 POINT(47.564875314221624 -123.13191180715035) bank126388 +126389 POINT(47.79910860671618 -122.28888593874332) bank126389 +126390 POINT(47.00048686725593 -121.96065294973943) bank126390 +126391 POINT(48.11523745401362 -122.04888571917554) bank126391 +126392 POINT(47.23374453950556 -123.05577250180104) bank126392 +126393 POINT(47.418105332917456 -123.00965942563249) bank126393 +126394 POINT(47.15432505915038 -123.03466725192185) bank126394 +126395 POINT(47.36229120583919 -123.14890517273066) bank126395 +126396 POINT(46.84145310227155 -122.67625587216723) bank126396 +126397 POINT(47.929749946670825 -122.72244551672212) bank126397 +126398 POINT(47.211952778688136 -122.72395324069066) bank126398 +126399 POINT(48.23352947718074 -122.32370553149435) bank126399 +126400 POINT(48.278446684547575 -121.4564605057022) bank126400 +126401 POINT(47.852671309868285 -121.37053137949309) bank126401 +126402 POINT(47.5158726637071 -122.2621257735827) bank126402 +126403 POINT(48.302886100441654 -122.16767642641325) bank126403 +126404 POINT(48.08443910206594 -123.29727390302672) bank126404 +126405 POINT(46.884741972215956 -121.54940978431183) bank126405 +126406 POINT(47.03426299343328 -121.5700740782351) bank126406 +126407 POINT(47.45202885531471 -122.52110286367673) bank126407 +126408 POINT(46.810072733464956 -122.72006751958746) bank126408 +126409 POINT(47.21100130238833 -122.75623739427584) bank126409 +126410 POINT(47.978655883622594 -123.31301072988454) bank126410 +126411 POINT(46.851905217311426 -121.88330214067328) bank126411 +126412 POINT(47.75852242417234 -122.54018441738289) bank126412 +126413 POINT(47.72067975799604 -122.14085494693302) bank126413 +126414 POINT(47.405008158869656 -123.13803312818213) bank126414 +126415 POINT(47.770015026501376 -122.61757916366184) bank126415 +126416 POINT(48.38350628004049 -123.2005604194892) bank126416 +126417 POINT(47.88932304189777 -123.30357660560315) bank126417 +126418 POINT(47.25362938180085 -121.38815415381028) bank126418 +126419 POINT(47.48838360245929 -121.97334277491632) bank126419 +126420 POINT(47.156164590807926 -121.93014377241228) bank126420 +126421 POINT(47.10250692193344 -122.82507626630651) bank126421 +126422 POINT(46.80990669484931 -122.4912983997072) bank126422 +126423 POINT(48.30542690444423 -121.94912469429406) bank126423 +126424 POINT(47.3726320990486 -121.58161982786405) bank126424 +126425 POINT(47.88848970793576 -123.17787116327075) bank126425 +126426 POINT(47.56086218983361 -123.2003260236775) bank126426 +126427 POINT(46.94741147033856 -122.70147547427281) bank126427 +126428 POINT(46.74762267483663 -122.08427480774417) bank126428 +126429 POINT(47.902896055812725 -121.66192889391097) bank126429 +126430 POINT(47.7428719645701 -122.62988627805638) bank126430 +126431 POINT(47.43382287757915 -122.65166020818522) bank126431 +126432 POINT(47.30577158757473 -122.16489507235714) bank126432 +126433 POINT(47.903960764279034 -121.92234319962097) bank126433 +126434 POINT(47.67794992405845 -121.77022850830636) bank126434 +126435 POINT(47.10043542823245 -122.61487052240044) bank126435 +126436 POINT(48.5668889848436 -121.73278544640553) bank126436 +126437 POINT(47.07665440424197 -121.60542809216979) bank126437 +126438 POINT(48.15869234988847 -122.0737509464777) bank126438 +126439 POINT(47.24409269241077 -121.59076363647338) bank126439 +126440 POINT(48.500802211446285 -122.7519587654093) bank126440 +126441 POINT(46.63236868922031 -122.0998415845003) bank126441 +126442 POINT(46.6726265242143 -122.04483771944925) bank126442 +126443 POINT(47.10124795238816 -121.58507421266184) bank126443 +126444 POINT(46.71521671183176 -122.02268800562939) bank126444 +126445 POINT(47.42273133429211 -123.27090095117067) bank126445 +126446 POINT(47.942071298753376 -121.77408706344673) bank126446 +126447 POINT(48.00543340670777 -123.09506267082651) bank126447 +126448 POINT(47.02718610082536 -122.31125234461885) bank126448 +126449 POINT(46.834313710334655 -121.9036110505836) bank126449 +126450 POINT(46.790613557701406 -122.63770010683558) bank126450 +126451 POINT(46.719663557982976 -121.92336630459724) bank126451 +126452 POINT(47.22555813117769 -121.9980907182214) bank126452 +126453 POINT(46.66335019021849 -123.16773481458209) bank126453 +126454 POINT(48.38239685922093 -121.41121532043907) bank126454 +126455 POINT(48.54248838461245 -121.92074709266261) bank126455 +126456 POINT(46.90284828504839 -121.83468407824769) bank126456 +126457 POINT(47.3632283319677 -123.0531387970616) bank126457 +126458 POINT(46.79815174383416 -122.73708802187618) bank126458 +126459 POINT(47.067874714205146 -122.20604853592376) bank126459 +126460 POINT(48.11531827535019 -121.48250136616525) bank126460 +126461 POINT(47.29813879784255 -121.54340567593391) bank126461 +126462 POINT(47.69478653132274 -122.35890006909769) bank126462 +126463 POINT(48.575702360259434 -122.09990528220695) bank126463 +126464 POINT(47.34126340196428 -122.2894869380798) bank126464 +126465 POINT(47.154812026622636 -122.16122093601604) bank126465 +126466 POINT(46.84674020612435 -121.88838013766434) bank126466 +126467 POINT(47.897565380750194 -122.14006140321946) bank126467 +126468 POINT(47.88012117621224 -122.46397264342282) bank126468 +126469 POINT(48.01315144891008 -123.18727040758051) bank126469 +126470 POINT(47.54032673571597 -121.33227187012773) bank126470 +126471 POINT(47.5533064538509 -122.65032621475386) bank126471 +126472 POINT(48.377847087109785 -122.95878317397221) bank126472 +126473 POINT(47.36963541847421 -122.35008886619914) bank126473 +126474 POINT(46.974252443962676 -121.38065390480203) bank126474 +126475 POINT(47.21207866989644 -123.2770714231068) bank126475 +126476 POINT(47.378007410474495 -122.69808858211253) bank126476 +126477 POINT(46.6264018714532 -121.46746737295692) bank126477 +126478 POINT(48.041266905055345 -122.77447729524012) bank126478 +126479 POINT(47.70123359358251 -121.530646637908) bank126479 +126480 POINT(47.99936652852717 -122.37661388922183) bank126480 +126481 POINT(47.92605773691865 -121.81165641222032) bank126481 +126482 POINT(47.88929659608963 -121.3767638911283) bank126482 +126483 POINT(46.79503849518604 -122.28684113135112) bank126483 +126484 POINT(48.256641265366525 -121.89255716304365) bank126484 +126485 POINT(48.47036325756228 -122.88791083231652) bank126485 +126486 POINT(48.464634616539016 -121.83050662050907) bank126486 +126487 POINT(46.893128352782355 -123.06225935868501) bank126487 +126488 POINT(47.592259288080484 -122.02990280969048) bank126488 +126489 POINT(47.835152368627476 -123.04960374228905) bank126489 +126490 POINT(47.760177792973835 -123.14193466364715) bank126490 +126491 POINT(47.325230106088156 -121.38944523936395) bank126491 +126492 POINT(46.635384287296276 -122.28898273054719) bank126492 +126493 POINT(46.7688933018197 -122.70460249268463) bank126493 +126494 POINT(46.81383144253895 -122.98415322486291) bank126494 +126495 POINT(47.25982754120761 -121.75897907082279) bank126495 +126496 POINT(47.34348469655643 -122.07690359105548) bank126496 +126497 POINT(47.526364114882085 -122.69691285218168) bank126497 +126498 POINT(47.74116825702196 -122.00469910994204) bank126498 +126499 POINT(47.50810674058511 -121.55766545617841) bank126499 +126500 POINT(46.61169252256903 -122.24712302214108) bank126500 +126501 POINT(47.922982468792334 -121.8732378638925) bank126501 +126502 POINT(46.621661250839146 -122.68110978233176) bank126502 +126503 POINT(47.62842642052118 -121.84487605886795) bank126503 +126504 POINT(47.38449618793624 -121.64545712946743) bank126504 +126505 POINT(47.30044119820578 -123.20349683354132) bank126505 +126506 POINT(47.25733055189177 -122.39934428814072) bank126506 +126507 POINT(47.966528049897406 -121.7868750306899) bank126507 +126508 POINT(47.852824986901865 -122.09492046973023) bank126508 +126509 POINT(47.36890721473461 -122.4406915433902) bank126509 +126510 POINT(46.80601525869392 -121.80780293269721) bank126510 +126511 POINT(48.024426423473436 -121.66871109046042) bank126511 +126512 POINT(47.44691343306955 -122.26763092789699) bank126512 +126513 POINT(47.371486523538124 -123.28235128306021) bank126513 +126514 POINT(47.50203910171385 -123.05399382132416) bank126514 +126515 POINT(47.02276800684396 -121.68534837262744) bank126515 +126516 POINT(47.708822143721775 -122.03261389303168) bank126516 +126517 POINT(47.13522756703315 -122.84990994382086) bank126517 +126518 POINT(47.61523123693401 -122.19609834384453) bank126518 +126519 POINT(47.12431603977137 -122.2182654585207) bank126519 +126520 POINT(47.391345336015675 -121.47180251420755) bank126520 +126521 POINT(47.296215467434685 -121.73900334934342) bank126521 +126522 POINT(47.246535466680896 -122.43684523229359) bank126522 +126523 POINT(48.50289884911494 -122.35221026177693) bank126523 +126524 POINT(48.01078517349116 -122.65951903766565) bank126524 +126525 POINT(47.069939692826246 -122.3881408836301) bank126525 +126526 POINT(47.224018703006905 -121.97944233281119) bank126526 +126527 POINT(47.25336947309511 -122.19171034103601) bank126527 +126528 POINT(47.67843119085374 -122.41537417596905) bank126528 +126529 POINT(48.0131712520801 -123.24352192095506) bank126529 +126530 POINT(48.551854403008726 -122.77725957095852) bank126530 +126531 POINT(48.0568239373488 -122.0238664330899) bank126531 +126532 POINT(48.44226596076215 -122.0847122455591) bank126532 +126533 POINT(48.2625278977357 -122.2426513372618) bank126533 +126534 POINT(46.92396696281531 -122.46367662315184) bank126534 +126535 POINT(47.68099491851514 -122.87139830775902) bank126535 +126536 POINT(47.655011864555156 -121.51886086870246) bank126536 +126537 POINT(46.969781984018425 -121.92912661548016) bank126537 +126538 POINT(47.2120589083855 -123.1612255839722) bank126538 +126539 POINT(47.18625628947635 -121.68769474752526) bank126539 +126540 POINT(47.66746737046573 -122.05754898361468) bank126540 +126541 POINT(47.91334619734386 -121.66996986633093) bank126541 +126542 POINT(48.339036735211636 -121.48542149337135) bank126542 +126543 POINT(48.0526772220305 -121.52063075866523) bank126543 +126544 POINT(48.39318399706404 -122.98320764845431) bank126544 +126545 POINT(47.350942907967216 -123.31395036112642) bank126545 +126546 POINT(48.594914172758145 -123.26197386190769) bank126546 +126547 POINT(48.576562129496764 -121.63140054222454) bank126547 +126548 POINT(47.78910449764774 -122.60016159064993) bank126548 +126549 POINT(48.00753863252106 -122.62774375907239) bank126549 +126550 POINT(47.35789497053511 -121.88707110366369) bank126550 +126551 POINT(48.38647876990591 -122.34522973686441) bank126551 +126552 POINT(48.288991094701046 -121.89040970641915) bank126552 +126553 POINT(48.342836954642685 -122.30137297850835) bank126553 +126554 POINT(47.34148948401067 -122.74628768365524) bank126554 +126555 POINT(46.75268374408953 -122.75673458465444) bank126555 +126556 POINT(48.35675699050468 -122.2752716657379) bank126556 +126557 POINT(48.08632587183194 -122.38022646289323) bank126557 +126558 POINT(47.67527603447113 -122.56186832046427) bank126558 +126559 POINT(46.75123134350756 -121.9797711605991) bank126559 +126560 POINT(48.590321381532185 -123.32631832849475) bank126560 +126561 POINT(47.32082584918103 -122.65702490703498) bank126561 +126562 POINT(47.02646604425608 -121.54296192485872) bank126562 +126563 POINT(47.89595584939193 -122.68328300496235) bank126563 +126564 POINT(47.974581633059 -122.41834364706529) bank126564 +126565 POINT(47.5982852991076 -121.68933840086245) bank126565 +126566 POINT(47.95983372393143 -122.97742013537847) bank126566 +126567 POINT(48.43629585491813 -122.24093051295455) bank126567 +126568 POINT(48.44066065828514 -122.60486217587615) bank126568 +126569 POINT(48.057278253663036 -122.82217078036305) bank126569 +126570 POINT(48.15569341042528 -121.43244976728177) bank126570 +126571 POINT(46.64156323833937 -123.19914151139878) bank126571 +126572 POINT(48.30197369238651 -122.49717665690878) bank126572 +126573 POINT(47.51769720218609 -121.54028086890565) bank126573 +126574 POINT(47.6471315024416 -122.23713219141018) bank126574 +126575 POINT(47.12558817756545 -123.03385692588871) bank126575 +126576 POINT(48.47491687542859 -123.23918654978412) bank126576 +126577 POINT(47.16193626242279 -123.1672780241923) bank126577 +126578 POINT(46.93731205474764 -123.09418895228532) bank126578 +126579 POINT(48.05235318537252 -122.00364590448449) bank126579 +126580 POINT(46.68114199949351 -122.17587601739092) bank126580 +126581 POINT(48.121482839453364 -121.89035399817901) bank126581 +126582 POINT(47.54336527901703 -122.91403473525068) bank126582 +126583 POINT(48.56750427590507 -121.59706877773097) bank126583 +126584 POINT(47.51786380299507 -122.2398085587254) bank126584 +126585 POINT(47.16023198740234 -123.2942903303898) bank126585 +126586 POINT(46.65890564526112 -123.2039234137845) bank126586 +126587 POINT(48.477796329652044 -122.74749586916055) bank126587 +126588 POINT(47.71116680572854 -122.61601275974284) bank126588 +126589 POINT(47.170910311588194 -123.12230602862016) bank126589 +126590 POINT(48.50727113290384 -122.68436334193548) bank126590 +126591 POINT(47.60272657132618 -123.19352855172883) bank126591 +126592 POINT(47.704759908675484 -121.56846890057591) bank126592 +126593 POINT(48.46850012548114 -122.15022730459954) bank126593 +126594 POINT(48.49617025808306 -122.56397574732975) bank126594 +126595 POINT(47.13816846594905 -122.52788675426123) bank126595 +126596 POINT(48.501246026965156 -123.21653717787417) bank126596 +126597 POINT(46.97968130487802 -123.04122321392576) bank126597 +126598 POINT(47.36002208197121 -121.9580619255301) bank126598 +126599 POINT(47.34743104293633 -121.62868484992555) bank126599 +126600 POINT(46.72438251269221 -122.75069237104822) bank126600 +126601 POINT(46.702095759039764 -122.2843208532382) bank126601 +126602 POINT(48.053960173891575 -121.74333422810987) bank126602 +126603 POINT(48.22508445239796 -121.75573723584655) bank126603 +126604 POINT(47.84242633616746 -123.20125887410428) bank126604 +126605 POINT(46.84837339792258 -122.76559072630401) bank126605 +126606 POINT(48.44093009514893 -121.5743404234547) bank126606 +126607 POINT(47.16435169341359 -122.17162303927135) bank126607 +126608 POINT(47.80536541391367 -121.38420710097515) bank126608 +126609 POINT(48.544998783428554 -122.25201877461112) bank126609 +126610 POINT(47.4763966688574 -122.21089131861714) bank126610 +126611 POINT(48.5787955869331 -122.80110952968758) bank126611 +126612 POINT(48.57355000500229 -122.21713211832514) bank126612 +126613 POINT(48.161516929584174 -123.21872839142802) bank126613 +126614 POINT(47.16534005896469 -123.19069777062487) bank126614 +126615 POINT(48.337880147851195 -122.49324174461034) bank126615 +126616 POINT(48.09278830710995 -122.1687001231967) bank126616 +126617 POINT(47.29993480788259 -121.83019762875296) bank126617 +126618 POINT(47.18688958622736 -121.66493958946751) bank126618 +126619 POINT(46.78933482424888 -121.87350067716676) bank126619 +126620 POINT(48.41865330391878 -123.03519160307906) bank126620 +126621 POINT(47.782240229331784 -121.94377850377897) bank126621 +126622 POINT(47.74279684395799 -122.64467831415193) bank126622 +126623 POINT(46.63913767492561 -122.4620558901612) bank126623 +126624 POINT(48.60153013393768 -121.91091206271201) bank126624 +126625 POINT(48.42281610298945 -122.56264411280722) bank126625 +126626 POINT(48.373695167402836 -122.75932987410671) bank126626 +126627 POINT(46.933285781621755 -122.2180829647128) bank126627 +126628 POINT(47.87398162525555 -121.91942860492638) bank126628 +126629 POINT(47.13149055307268 -121.90179197090687) bank126629 +126630 POINT(47.15954090647215 -123.19307742655735) bank126630 +126631 POINT(46.93126832639602 -122.18213671881084) bank126631 +126632 POINT(46.7618905472523 -122.4374564334542) bank126632 +126633 POINT(46.647300370465096 -122.05358056607955) bank126633 +126634 POINT(48.078460834663815 -122.11475675951482) bank126634 +126635 POINT(46.95422422805264 -122.83033180954868) bank126635 +126636 POINT(48.525613302909655 -121.53853239264646) bank126636 +126637 POINT(48.1644202699239 -122.29030069383879) bank126637 +126638 POINT(46.656469911724855 -122.33187948017382) bank126638 +126639 POINT(47.658154334778914 -123.08784241557935) bank126639 +126640 POINT(46.83050673121746 -122.17138199893365) bank126640 +126641 POINT(47.804919677929064 -123.1601848503766) bank126641 +126642 POINT(48.52724611709014 -121.57255804325146) bank126642 +126643 POINT(48.37406344420439 -122.76478625890311) bank126643 +126644 POINT(48.09037490688906 -122.91258833638975) bank126644 +126645 POINT(47.14452644176568 -123.09086311250502) bank126645 +126646 POINT(47.664192627683235 -123.32751445466863) bank126646 +126647 POINT(47.14685514121618 -121.78729616821036) bank126647 +126648 POINT(46.9570891517672 -123.07370826877448) bank126648 +126649 POINT(47.94737683250601 -122.02312792660582) bank126649 +126650 POINT(48.218587772414544 -122.03108528081373) bank126650 +126651 POINT(48.48124511779206 -123.21116019884288) bank126651 +126652 POINT(48.475857054648436 -121.50372518710998) bank126652 +126653 POINT(47.29400341639597 -122.7885535429406) bank126653 +126654 POINT(47.60629211193475 -123.22678244279352) bank126654 +126655 POINT(46.895364747945834 -121.91911353045958) bank126655 +126656 POINT(46.6260697265182 -122.53899130529012) bank126656 +126657 POINT(48.32642116924939 -122.42125244546143) bank126657 +126658 POINT(46.96183180363015 -122.50144142685384) bank126658 +126659 POINT(46.69021482879672 -121.70043375128466) bank126659 +126660 POINT(48.26704658122042 -121.39848344099603) bank126660 +126661 POINT(48.14147366708776 -122.89871267304648) bank126661 +126662 POINT(48.60333087257562 -121.66715547177145) bank126662 +126663 POINT(48.377815095816544 -122.82898756963168) bank126663 +126664 POINT(47.491312586267085 -122.38383873202696) bank126664 +126665 POINT(47.96546956541301 -121.9777922475215) bank126665 +126666 POINT(46.73714733436259 -122.43848082418683) bank126666 +126667 POINT(47.71362551388923 -122.38444641772138) bank126667 +126668 POINT(46.97406582144903 -122.00903362958893) bank126668 +126669 POINT(46.75013463995546 -121.34071534302436) bank126669 +126670 POINT(47.35923795807097 -123.08824914779473) bank126670 +126671 POINT(46.674169811852956 -121.54392474616634) bank126671 +126672 POINT(47.33322546577868 -122.96608846126706) bank126672 +126673 POINT(48.426703706983496 -122.90815613450465) bank126673 +126674 POINT(47.88208944932157 -121.99251109210932) bank126674 +126675 POINT(48.12422438560154 -121.80694091559356) bank126675 +126676 POINT(47.6316791571424 -122.2203401111766) bank126676 +126677 POINT(46.62881944017211 -122.85583916673046) bank126677 +126678 POINT(48.042319697648814 -121.65753249432838) bank126678 +126679 POINT(46.9366681040567 -121.87273058364357) bank126679 +126680 POINT(48.12833512777001 -122.96743497154574) bank126680 +126681 POINT(47.44586267529798 -122.06557143677685) bank126681 +126682 POINT(47.72882595185508 -122.64922909836046) bank126682 +126683 POINT(47.53842447869269 -123.23475705208477) bank126683 +126684 POINT(47.90441390540965 -121.52456093044351) bank126684 +126685 POINT(48.031582284407285 -122.095317817125) bank126685 +126686 POINT(48.29732957192526 -122.56561416978985) bank126686 +126687 POINT(47.27619055491506 -122.80417839702483) bank126687 +126688 POINT(47.431229176890554 -121.61583137083753) bank126688 +126689 POINT(47.57302479543932 -121.75316696448725) bank126689 +126690 POINT(47.29892702340157 -122.40135091575208) bank126690 +126691 POINT(47.23281247595672 -122.93195608263844) bank126691 +126692 POINT(48.52515259161309 -121.47848037853916) bank126692 +126693 POINT(47.49748758165556 -121.52260674507347) bank126693 +126694 POINT(47.59957611656924 -121.61050391181335) bank126694 +126695 POINT(46.63896514507399 -121.68159777473898) bank126695 +126696 POINT(47.30970367008495 -122.70437288721813) bank126696 +126697 POINT(47.94051687503403 -122.91582868643056) bank126697 +126698 POINT(48.09088018362969 -121.57993276151086) bank126698 +126699 POINT(47.07679686382578 -122.06628145376516) bank126699 +126700 POINT(48.43529640496565 -122.01610767826395) bank126700 +126701 POINT(47.751193904530986 -123.24035453427219) bank126701 +126702 POINT(47.74205300759974 -122.58382772947036) bank126702 +126703 POINT(47.54162775584098 -122.38070088140772) bank126703 +126704 POINT(48.16143169065201 -121.70604810465716) bank126704 +126705 POINT(47.844388866548854 -123.24926639569124) bank126705 +126706 POINT(47.06689575530392 -121.43698132980998) bank126706 +126707 POINT(47.50196200499717 -123.11160748181251) bank126707 +126708 POINT(48.56437268786945 -122.04071208798457) bank126708 +126709 POINT(46.711161597872426 -122.43097244145787) bank126709 +126710 POINT(48.27272842573526 -121.78342860272497) bank126710 +126711 POINT(47.39800556537977 -122.70659108804857) bank126711 +126712 POINT(47.8537682118759 -122.54929694213932) bank126712 +126713 POINT(47.896420821051954 -122.28499664171964) bank126713 +126714 POINT(48.41559613243358 -121.39955435003543) bank126714 +126715 POINT(48.465979924851595 -122.21766340296838) bank126715 +126716 POINT(47.721250730226764 -121.9714952988021) bank126716 +126717 POINT(47.72315943964394 -121.95248084358295) bank126717 +126718 POINT(46.68625988189115 -121.48986421848478) bank126718 +126719 POINT(47.198930818911904 -122.51486264773564) bank126719 +126720 POINT(47.780870203472546 -121.81576884049993) bank126720 +126721 POINT(47.377806642576175 -123.00006591785802) bank126721 +126722 POINT(47.24585065801621 -122.87292093754405) bank126722 +126723 POINT(47.891496915189116 -122.85706149559908) bank126723 +126724 POINT(46.66100784421101 -122.4498463608) bank126724 +126725 POINT(47.555815515269295 -123.21253536712962) bank126725 +126726 POINT(47.8671351576243 -121.55322106725225) bank126726 +126727 POINT(46.69705669569141 -121.44987600616881) bank126727 +126728 POINT(46.775953990061815 -121.5526362478227) bank126728 +126729 POINT(47.39173121849362 -122.52198709472987) bank126729 +126730 POINT(47.16914988768245 -122.5592785373615) bank126730 +126731 POINT(48.21921198377253 -121.84577994299006) bank126731 +126732 POINT(48.023698359717905 -122.34403933889725) bank126732 +126733 POINT(47.91129015794792 -123.23216084496815) bank126733 +126734 POINT(46.70486626695697 -123.16083508840701) bank126734 +126735 POINT(47.858382485851 -122.02902063768671) bank126735 +126736 POINT(48.051417412366085 -122.81795382725345) bank126736 +126737 POINT(48.493919753445475 -122.88921199312873) bank126737 +126738 POINT(46.821705813347556 -122.08739368295177) bank126738 +126739 POINT(46.804886296573464 -121.85043940292769) bank126739 +126740 POINT(47.216253357944865 -121.8853424372032) bank126740 +126741 POINT(47.896669836477194 -122.17161061363271) bank126741 +126742 POINT(48.30278494578556 -122.65519448009215) bank126742 +126743 POINT(46.85167635067849 -122.72811496313928) bank126743 +126744 POINT(47.18599693081124 -123.07433090435089) bank126744 +126745 POINT(48.56021309036688 -121.9315719711798) bank126745 +126746 POINT(46.822828194185234 -122.24604870418847) bank126746 +126747 POINT(47.78319564547981 -122.66621944571463) bank126747 +126748 POINT(46.84027178687211 -122.64116230868095) bank126748 +126749 POINT(47.19050785585557 -121.98676997136769) bank126749 +126750 POINT(46.71024277782977 -121.86616454109318) bank126750 +126751 POINT(46.66975392356308 -121.51243473600329) bank126751 +126752 POINT(47.8728073662197 -122.47678214855493) bank126752 +126753 POINT(48.295830554829784 -122.07783464710349) bank126753 +126754 POINT(47.32571418324893 -121.62629543324363) bank126754 +126755 POINT(47.85718761784818 -121.7734674308681) bank126755 +126756 POINT(47.43504049168562 -123.3120436583241) bank126756 +126757 POINT(48.1357659644964 -121.59844693892015) bank126757 +126758 POINT(47.16542476622059 -122.714883124198) bank126758 +126759 POINT(47.35877477296262 -123.11035706508912) bank126759 +126760 POINT(47.2158426538645 -122.32237466602271) bank126760 +126761 POINT(48.57874265006538 -122.2777431935485) bank126761 +126762 POINT(48.23052906862174 -123.06793503357241) bank126762 +126763 POINT(46.87232883232842 -121.87675776431897) bank126763 +126764 POINT(48.39324183266728 -121.57700298201542) bank126764 +126765 POINT(48.14339814011384 -122.33466243721344) bank126765 +126766 POINT(47.36631572878737 -122.85955117672434) bank126766 +126767 POINT(48.281419708447125 -122.17599515736856) bank126767 +126768 POINT(47.35038960079298 -122.35319207969886) bank126768 +126769 POINT(48.133906313110366 -121.36571923577391) bank126769 +126770 POINT(47.583253508445644 -122.13061649557334) bank126770 +126771 POINT(46.73490633069206 -123.22128422486895) bank126771 +126772 POINT(47.39616145921002 -121.33378601368078) bank126772 +126773 POINT(47.44965843166662 -121.78355536992659) bank126773 +126774 POINT(48.007262884155345 -121.51941797100787) bank126774 +126775 POINT(46.80462670047988 -122.99701032340653) bank126775 +126776 POINT(47.721913897290364 -122.39555750092035) bank126776 +126777 POINT(48.40132108473084 -121.81547699342588) bank126777 +126778 POINT(48.57662735031886 -122.0937801152066) bank126778 +126779 POINT(47.86348071881759 -123.20044376186371) bank126779 +126780 POINT(48.29403055386185 -122.10787270680044) bank126780 +126781 POINT(46.80455736602482 -122.15648631682281) bank126781 +126782 POINT(47.94643138808027 -122.98692747540467) bank126782 +126783 POINT(48.48984185576234 -121.83466459765073) bank126783 +126784 POINT(47.960809770418834 -122.13375090130636) bank126784 +126785 POINT(47.28081899015591 -122.71922443707764) bank126785 +126786 POINT(48.582958216908644 -122.42885205657379) bank126786 +126787 POINT(47.570649357429076 -122.30941297607876) bank126787 +126788 POINT(46.970420971050245 -123.2011628301348) bank126788 +126789 POINT(47.76836198672136 -122.05701027574038) bank126789 +126790 POINT(47.94608125599953 -121.43031849892417) bank126790 +126791 POINT(47.482972865895434 -121.8259200513429) bank126791 +126792 POINT(47.781452824672584 -122.20597260365084) bank126792 +126793 POINT(48.2437246827162 -122.18821016972298) bank126793 +126794 POINT(47.73002169447765 -122.64283180333715) bank126794 +126795 POINT(47.655776249859294 -121.38982870664248) bank126795 +126796 POINT(48.27889993509025 -121.87222612682332) bank126796 +126797 POINT(48.474695325833764 -121.87795124351365) bank126797 +126798 POINT(46.94126927041614 -123.26010557081287) bank126798 +126799 POINT(47.36421880172823 -121.44938897668935) bank126799 +126800 POINT(48.13955583404157 -121.68031685888889) bank126800 +126801 POINT(47.77549570474825 -121.98086511839495) bank126801 +126802 POINT(48.450148169530145 -123.08382590306167) bank126802 +126803 POINT(47.10071119439448 -122.83283072115678) bank126803 +126804 POINT(48.14940691615611 -123.32047967996765) bank126804 +126805 POINT(47.18552412893532 -122.20281733075628) bank126805 +126806 POINT(47.38100931565096 -121.94937019538992) bank126806 +126807 POINT(47.70219984965367 -122.86899064847721) bank126807 +126808 POINT(47.52869275005734 -123.12450247550062) bank126808 +126809 POINT(47.9139728968392 -123.09257976653164) bank126809 +126810 POINT(48.06012583778325 -123.0809733233371) bank126810 +126811 POINT(48.456814603732916 -122.1651776186023) bank126811 +126812 POINT(47.94562377022237 -122.62283768145156) bank126812 +126813 POINT(47.27758885396154 -122.84307039008863) bank126813 +126814 POINT(47.68220759798663 -123.22492012447444) bank126814 +126815 POINT(47.07232583033876 -121.4102251866282) bank126815 +126816 POINT(48.05075343767989 -122.1288522711104) bank126816 +126817 POINT(48.22659513466865 -122.63649840367808) bank126817 +126818 POINT(47.206273714530994 -121.77272960953755) bank126818 +126819 POINT(47.76281863359786 -122.47806483097918) bank126819 +126820 POINT(46.93063002353076 -122.85897589361073) bank126820 +126821 POINT(47.896874879731875 -122.5578193179594) bank126821 +126822 POINT(47.29311458717604 -122.35485365173223) bank126822 +126823 POINT(48.12666951489953 -122.06735046258811) bank126823 +126824 POINT(47.07322915782576 -122.79008456823945) bank126824 +126825 POINT(48.05271137226716 -121.57854747204598) bank126825 +126826 POINT(48.09447083658607 -121.997965573328) bank126826 +126827 POINT(46.6949649814578 -122.48141310072806) bank126827 +126828 POINT(46.89733389257615 -122.899767603872) bank126828 +126829 POINT(46.79311235354589 -123.30330930475995) bank126829 +126830 POINT(47.59950378092584 -122.26191867987735) bank126830 +126831 POINT(47.0025826923772 -121.80696309434028) bank126831 +126832 POINT(48.44093122405531 -122.89299603818519) bank126832 +126833 POINT(47.185167337119225 -121.78357184387662) bank126833 +126834 POINT(48.11123652513117 -122.34739483740006) bank126834 +126835 POINT(48.11219396530726 -121.7556615809287) bank126835 +126836 POINT(47.69554176160448 -122.05975708328427) bank126836 +126837 POINT(47.067876528276074 -122.50691866449274) bank126837 +126838 POINT(47.26008742891872 -122.24781349294719) bank126838 +126839 POINT(47.704940300997244 -121.8382445537895) bank126839 +126840 POINT(47.82548488235375 -123.11367800288652) bank126840 +126841 POINT(47.79016773576012 -122.71911043083846) bank126841 +126842 POINT(47.29457089861583 -123.11193635660234) bank126842 +126843 POINT(46.72884511397218 -123.02540608755575) bank126843 +126844 POINT(48.10519242184525 -122.46240402391604) bank126844 +126845 POINT(46.83348178311888 -123.32384399288303) bank126845 +126846 POINT(48.2565031934977 -121.42675639460958) bank126846 +126847 POINT(47.534005839523765 -123.19640763113144) bank126847 +126848 POINT(47.96491657087007 -122.12601505659477) bank126848 +126849 POINT(48.07695369178184 -122.6512625722749) bank126849 +126850 POINT(47.49395969316639 -122.03908359769852) bank126850 +126851 POINT(46.72211750344498 -121.9159023564085) bank126851 +126852 POINT(47.24739482084239 -122.88788917181016) bank126852 +126853 POINT(48.477322377213746 -123.05083272905541) bank126853 +126854 POINT(47.201577450040624 -122.49888853311722) bank126854 +126855 POINT(47.39968307243721 -122.26689853947478) bank126855 +126856 POINT(47.9065777351937 -123.00081996203595) bank126856 +126857 POINT(46.61853771755809 -123.08116675221883) bank126857 +126858 POINT(48.40082731144184 -122.8347765659804) bank126858 +126859 POINT(47.755030938079024 -121.92359781458428) bank126859 +126860 POINT(48.342676927546684 -122.5217754666574) bank126860 +126861 POINT(48.193355326064946 -121.39754323011107) bank126861 +126862 POINT(47.98864259680557 -122.94300696868908) bank126862 +126863 POINT(46.918931162556056 -121.50131098040045) bank126863 +126864 POINT(46.71549684994461 -121.80910740997754) bank126864 +126865 POINT(48.35373460519223 -123.20585131833973) bank126865 +126866 POINT(47.85182880723961 -122.86582038125634) bank126866 +126867 POINT(46.61328029568183 -121.35714238722473) bank126867 +126868 POINT(48.05824464125077 -122.5245786651988) bank126868 +126869 POINT(48.19341622512023 -122.92179156642428) bank126869 +126870 POINT(47.03231198634344 -121.97309742919172) bank126870 +126871 POINT(47.93040396997058 -122.13514164871862) bank126871 +126872 POINT(47.62431389839529 -123.0869068717628) bank126872 +126873 POINT(47.92108213425464 -122.86651478833579) bank126873 +126874 POINT(46.774390236669 -122.72642210958729) bank126874 +126875 POINT(46.691490525134185 -121.73669914945881) bank126875 +126876 POINT(48.57192569662175 -121.88523033372601) bank126876 +126877 POINT(47.35133436899569 -121.44629245628644) bank126877 +126878 POINT(47.280894191622636 -121.37970709130359) bank126878 +126879 POINT(47.69509542281657 -123.20417766501996) bank126879 +126880 POINT(47.92919182562583 -121.44078449179706) bank126880 +126881 POINT(46.73680784618627 -122.5490662109862) bank126881 +126882 POINT(48.25586935919499 -122.8982721125342) bank126882 +126883 POINT(46.796319760742584 -121.60651469615581) bank126883 +126884 POINT(47.78232611507421 -121.94223111548752) bank126884 +126885 POINT(46.92158091060877 -123.23012567279605) bank126885 +126886 POINT(47.111910410479105 -123.11966170514775) bank126886 +126887 POINT(46.79835050953269 -121.86183918812321) bank126887 +126888 POINT(46.66731764497129 -123.18696321829032) bank126888 +126889 POINT(47.644122828424706 -122.29653750332432) bank126889 +126890 POINT(46.61522202807506 -121.77954288104156) bank126890 +126891 POINT(47.58300646074228 -122.47878589946542) bank126891 +126892 POINT(47.013935677663675 -121.85753663975426) bank126892 +126893 POINT(47.524262299142286 -121.53306530953316) bank126893 +126894 POINT(47.705906175619255 -122.63698411804651) bank126894 +126895 POINT(46.80589963728804 -121.70501962084055) bank126895 +126896 POINT(47.94631638456836 -122.82662538263546) bank126896 +126897 POINT(47.81814697631235 -123.20124118682254) bank126897 +126898 POINT(47.39603248986902 -122.84396747042749) bank126898 +126899 POINT(47.87407112035777 -122.28259223014639) bank126899 +126900 POINT(47.802555103018136 -122.16789352346171) bank126900 +126901 POINT(47.5291113849646 -122.60507112414705) bank126901 +126902 POINT(46.80301765590062 -123.21385842416922) bank126902 +126903 POINT(47.12623401270241 -122.37859426714513) bank126903 +126904 POINT(47.52404933511019 -121.7434668463727) bank126904 +126905 POINT(46.93418152132106 -122.61586166277219) bank126905 +126906 POINT(48.4356240974981 -121.74865525078144) bank126906 +126907 POINT(46.72275530416396 -123.27393572574609) bank126907 +126908 POINT(47.103873120243236 -122.25999204802366) bank126908 +126909 POINT(48.453271215419676 -122.13026990086077) bank126909 +126910 POINT(47.12564737638856 -122.8778051906648) bank126910 +126911 POINT(48.006543327067696 -122.2323914148668) bank126911 +126912 POINT(48.5462127795707 -121.39270095466883) bank126912 +126913 POINT(48.42572259839235 -122.6489281541875) bank126913 +126914 POINT(47.40915299602011 -122.13418393135642) bank126914 +126915 POINT(47.28333568632711 -122.09336693486767) bank126915 +126916 POINT(46.86855641493456 -122.90151151007005) bank126916 +126917 POINT(46.833971222332195 -122.19462444666662) bank126917 +126918 POINT(47.29717907052096 -121.82147077777299) bank126918 +126919 POINT(47.105048425197765 -122.61164247665667) bank126919 +126920 POINT(48.28631189675575 -122.99527000804535) bank126920 +126921 POINT(48.04805712450004 -121.87617073114168) bank126921 +126922 POINT(48.36339571158455 -122.01815773064794) bank126922 +126923 POINT(48.22535507251963 -122.19285863779629) bank126923 +126924 POINT(47.28398696040669 -121.65483859523324) bank126924 +126925 POINT(47.53932650614608 -121.4070098137566) bank126925 +126926 POINT(47.4299095378859 -121.7054362806766) bank126926 +126927 POINT(48.58108542023353 -121.49373861453473) bank126927 +126928 POINT(47.5455328841506 -122.36583523641076) bank126928 +126929 POINT(47.38606475536271 -121.6355698983958) bank126929 +126930 POINT(47.951207419373894 -121.73324613615928) bank126930 +126931 POINT(47.018390715476684 -121.64840700036481) bank126931 +126932 POINT(47.24992372823579 -122.14380628376311) bank126932 +126933 POINT(46.69224195655836 -122.9008823852821) bank126933 +126934 POINT(48.067554703996876 -122.10830646005188) bank126934 +126935 POINT(46.95765176294757 -122.68191740797555) bank126935 +126936 POINT(47.48486970818986 -122.1330836697297) bank126936 +126937 POINT(47.28153209759596 -121.51615264736435) bank126937 +126938 POINT(46.957411200513725 -122.61269145784078) bank126938 +126939 POINT(46.70507290270008 -122.98138800785824) bank126939 +126940 POINT(48.5954263468344 -121.4412492631533) bank126940 +126941 POINT(47.98531717016912 -122.58926231891816) bank126941 +126942 POINT(47.779650339599044 -121.65347646120078) bank126942 +126943 POINT(47.96519230302934 -121.90933724401837) bank126943 +126944 POINT(46.654903312211694 -122.15072887128328) bank126944 +126945 POINT(48.0795140118523 -122.24501316458519) bank126945 +126946 POINT(47.70256919270746 -123.18785069886954) bank126946 +126947 POINT(47.50237753971127 -121.5925672283081) bank126947 +126948 POINT(47.28732179861436 -123.1311123453307) bank126948 +126949 POINT(47.39427808897397 -123.18182669229691) bank126949 +126950 POINT(47.6204508817837 -122.58516796523378) bank126950 +126951 POINT(47.646477848923226 -123.30809710103595) bank126951 +126952 POINT(46.71105222194599 -123.1377616760648) bank126952 +126953 POINT(47.7354253170286 -122.2026983111329) bank126953 +126954 POINT(46.702052222612345 -122.0814664081779) bank126954 +126955 POINT(47.43431023365443 -121.44081470528995) bank126955 +126956 POINT(46.738391448719376 -122.00626116612747) bank126956 +126957 POINT(48.28140541098228 -122.33509599358084) bank126957 +126958 POINT(46.79746103590006 -122.82323169234832) bank126958 +126959 POINT(47.19770287822739 -122.13196335579578) bank126959 +126960 POINT(47.0605329660426 -122.35806519078223) bank126960 +126961 POINT(47.746882387000895 -122.39781245704175) bank126961 +126962 POINT(47.738282350600976 -122.28777455102657) bank126962 +126963 POINT(48.10484716009125 -122.97342258719297) bank126963 +126964 POINT(46.72732339097812 -122.37921560231436) bank126964 +126965 POINT(47.433345753164744 -122.03923631963384) bank126965 +126966 POINT(46.90514038202169 -121.69923729951846) bank126966 +126967 POINT(47.74076127764814 -122.36767364924455) bank126967 +126968 POINT(47.141556828470314 -121.70133427771528) bank126968 +126969 POINT(47.670657447895294 -121.41503856203634) bank126969 +126970 POINT(47.05917133954732 -123.17698838539958) bank126970 +126971 POINT(47.55291697127344 -121.6623410087067) bank126971 +126972 POINT(47.9702077714386 -122.03899807221572) bank126972 +126973 POINT(47.21766513496751 -123.13302999479778) bank126973 +126974 POINT(48.415646844958744 -121.95688037091234) bank126974 +126975 POINT(47.704870706500046 -122.15203304475715) bank126975 +126976 POINT(48.5328529957142 -122.32016647945785) bank126976 +126977 POINT(47.406760532737025 -122.06112816072981) bank126977 +126978 POINT(47.281342152651604 -123.31097866595617) bank126978 +126979 POINT(47.9502745140431 -122.16151872716303) bank126979 +126980 POINT(46.94482449637027 -122.16755739975629) bank126980 +126981 POINT(46.803442539674435 -121.77887208212897) bank126981 +126982 POINT(47.06146650370041 -121.54211636235748) bank126982 +126983 POINT(47.1024166532736 -122.94019907379067) bank126983 +126984 POINT(47.724380538984164 -122.99411079451816) bank126984 +126985 POINT(47.657868882266456 -122.4326820575893) bank126985 +126986 POINT(47.631765162272714 -123.21180404385173) bank126986 +126987 POINT(47.33264894202282 -123.22308566689037) bank126987 +126988 POINT(47.62655248529694 -121.86001970057622) bank126988 +126989 POINT(48.377214429271895 -123.16438613016864) bank126989 +126990 POINT(48.14471596416309 -121.99761872296459) bank126990 +126991 POINT(47.089649904192484 -121.55891128898402) bank126991 +126992 POINT(47.83534925995241 -122.55246502402296) bank126992 +126993 POINT(47.3246648176907 -123.24057197853567) bank126993 +126994 POINT(46.672037002668795 -121.60210571323532) bank126994 +126995 POINT(48.23187741490462 -122.94486829940368) bank126995 +126996 POINT(47.769852225243724 -123.0197222794306) bank126996 +126997 POINT(47.18290243022279 -123.17370345521674) bank126997 +126998 POINT(47.784551468681784 -122.20943187917544) bank126998 +126999 POINT(47.046744440986565 -121.35091144249819) bank126999 +127000 POINT(47.03904744341224 -122.38266659705675) bank127000 +127001 POINT(48.60130358550483 -122.4514704060411) bank127001 +127002 POINT(48.49882683757249 -122.30122990376286) bank127002 +127003 POINT(48.25830363002821 -122.93177406305587) bank127003 +127004 POINT(47.859297459477936 -123.31918729428524) bank127004 +127005 POINT(48.57799465676587 -121.46402338246054) bank127005 +127006 POINT(47.88300036522859 -122.98932775764439) bank127006 +127007 POINT(48.332334448394754 -123.08060213462389) bank127007 +127008 POINT(48.4894361142796 -121.39116608437769) bank127008 +127009 POINT(47.180611225066315 -122.27411737286896) bank127009 +127010 POINT(46.74339143288003 -121.84301020250462) bank127010 +127011 POINT(47.288869970003695 -121.93545601309656) bank127011 +127012 POINT(47.63060217185898 -122.94495569550062) bank127012 +127013 POINT(46.71179043191651 -122.80491142254952) bank127013 +127014 POINT(48.228987249958074 -122.75467744332133) bank127014 +127015 POINT(47.845415942466694 -121.92927517082218) bank127015 +127016 POINT(47.13639056444371 -122.25654793616476) bank127016 +127017 POINT(46.771078692415074 -122.940069341114) bank127017 +127018 POINT(46.73386934909117 -122.25752370997594) bank127018 +127019 POINT(48.06459935951028 -122.14362791268516) bank127019 +127020 POINT(47.253978781585396 -121.4298359342319) bank127020 +127021 POINT(47.82078536771427 -121.80176267279448) bank127021 +127022 POINT(48.384999864954224 -122.97970312119207) bank127022 +127023 POINT(46.76656453199258 -122.56107057685294) bank127023 +127024 POINT(47.058072301097916 -121.478016424843) bank127024 +127025 POINT(47.50829927922846 -122.13268212585228) bank127025 +127026 POINT(46.85256520814048 -122.88026497093249) bank127026 +127027 POINT(48.22385223086317 -122.9139674120078) bank127027 +127028 POINT(47.64772203811044 -121.94979408345435) bank127028 +127029 POINT(46.81597488598624 -121.57362875732645) bank127029 +127030 POINT(48.570927322736736 -122.0191699948011) bank127030 +127031 POINT(48.06704122271242 -122.6117893500851) bank127031 +127032 POINT(46.87658411436723 -122.16932780192614) bank127032 +127033 POINT(48.287408265857096 -121.73780438777099) bank127033 +127034 POINT(47.320142234650206 -122.2773464790883) bank127034 +127035 POINT(47.86456906913539 -121.97010193338076) bank127035 +127036 POINT(47.65153841008578 -122.32126759058312) bank127036 +127037 POINT(47.900032132096065 -122.78534936468921) bank127037 +127038 POINT(47.7270500998827 -122.3603854219495) bank127038 +127039 POINT(47.52984027753534 -121.68994223346554) bank127039 +127040 POINT(47.349912757891936 -122.91774125062747) bank127040 +127041 POINT(47.23090198858859 -121.55662540574308) bank127041 +127042 POINT(47.80134629774035 -121.7261514143137) bank127042 +127043 POINT(47.05315213135472 -121.41778200846572) bank127043 +127044 POINT(46.93037890141603 -122.34951326986659) bank127044 +127045 POINT(48.00954001222878 -122.06147031226722) bank127045 +127046 POINT(46.77335161330912 -122.78582751350132) bank127046 +127047 POINT(46.918180787972716 -123.07221376005599) bank127047 +127048 POINT(48.00610978239496 -121.78950959585832) bank127048 +127049 POINT(47.05211822027432 -123.09727144704999) bank127049 +127050 POINT(47.75578615742559 -121.86330988794006) bank127050 +127051 POINT(48.36558141269577 -123.28779921894507) bank127051 +127052 POINT(47.20701007685737 -122.39532174207571) bank127052 +127053 POINT(48.47148558447193 -121.63423044537981) bank127053 +127054 POINT(48.18798235112193 -122.0361690847763) bank127054 +127055 POINT(46.92320414596451 -122.98929262119056) bank127055 +127056 POINT(47.033307045794814 -123.1250345093968) bank127056 +127057 POINT(48.33823771065319 -121.41239583374065) bank127057 +127058 POINT(46.861048792243274 -122.92542774268479) bank127058 +127059 POINT(47.215810124516395 -123.12842123764685) bank127059 +127060 POINT(47.949396219029566 -122.64458429486801) bank127060 +127061 POINT(47.25485903424399 -121.95489951720735) bank127061 +127062 POINT(48.58097367763338 -121.34073983160178) bank127062 +127063 POINT(46.68333655812922 -121.76095815723167) bank127063 +127064 POINT(47.84846550239018 -121.9254779873587) bank127064 +127065 POINT(47.01291996279293 -121.42811930607054) bank127065 +127066 POINT(47.10677663811373 -122.2105570605228) bank127066 +127067 POINT(46.9947442677823 -122.32216366067725) bank127067 +127068 POINT(46.77873534905446 -121.6775324762383) bank127068 +127069 POINT(47.935415360659285 -122.7067548358032) bank127069 +127070 POINT(47.253113614171426 -122.98699022150078) bank127070 +127071 POINT(47.17333136411467 -121.65808482075543) bank127071 +127072 POINT(48.45573394809091 -121.87556481767554) bank127072 +127073 POINT(46.63371020305327 -122.21277232837066) bank127073 +127074 POINT(46.68514810109289 -122.54502404379862) bank127074 +127075 POINT(47.742106232044065 -123.24577135224993) bank127075 +127076 POINT(48.12141812707383 -122.09516668161052) bank127076 +127077 POINT(46.669520767025574 -122.68749483119335) bank127077 +127078 POINT(47.06455465367958 -121.72577083296007) bank127078 +127079 POINT(47.532764125787885 -123.02431470565297) bank127079 +127080 POINT(47.240679779833755 -123.0256395534881) bank127080 +127081 POINT(48.23955947606045 -121.58982158843108) bank127081 +127082 POINT(47.996710264039656 -121.96870710800802) bank127082 +127083 POINT(46.74018455171355 -122.556957298332) bank127083 +127084 POINT(46.94659545244133 -122.44782964923031) bank127084 +127085 POINT(47.18257420255511 -123.25281613806533) bank127085 +127086 POINT(47.08317503983174 -122.85939863564738) bank127086 +127087 POINT(46.822445631982696 -121.809773902086) bank127087 +127088 POINT(46.89506293826189 -122.05147320599691) bank127088 +127089 POINT(47.41781355508479 -122.85004382676179) bank127089 +127090 POINT(47.5252973198185 -122.68833458086895) bank127090 +127091 POINT(48.186274804122135 -121.8497754400096) bank127091 +127092 POINT(47.19226321937029 -122.73633112928366) bank127092 +127093 POINT(47.432677423274264 -123.00391428929996) bank127093 +127094 POINT(46.977467728775885 -121.72946613229564) bank127094 +127095 POINT(48.33531137313895 -122.86243211353498) bank127095 +127096 POINT(47.00455933001877 -121.98721273206787) bank127096 +127097 POINT(46.96724525325292 -121.35667098835641) bank127097 +127098 POINT(47.271546451857404 -122.81251073359695) bank127098 +127099 POINT(48.19296961522614 -122.83477068212198) bank127099 +127100 POINT(47.983954212745275 -122.19496796526232) bank127100 +127101 POINT(47.95784833368848 -121.81633272673231) bank127101 +127102 POINT(48.29318214510878 -122.28586287245038) bank127102 +127103 POINT(46.795700935476106 -121.66672695273472) bank127103 +127104 POINT(48.266392383951235 -122.23524932166613) bank127104 +127105 POINT(48.37625575406253 -122.2968649462115) bank127105 +127106 POINT(48.18677750293842 -122.79377638483018) bank127106 +127107 POINT(48.219858238035954 -123.31258899977631) bank127107 +127108 POINT(47.154634423515915 -123.19587909364193) bank127108 +127109 POINT(46.86033379038022 -122.6511272920811) bank127109 +127110 POINT(47.25158565348419 -123.26270687350906) bank127110 +127111 POINT(46.984852571414706 -121.42332055400271) bank127111 +127112 POINT(48.06608896101371 -121.47439672826987) bank127112 +127113 POINT(47.6207589366532 -122.88543679603504) bank127113 +127114 POINT(47.618754801723455 -122.15770515493887) bank127114 +127115 POINT(48.36505871039037 -122.94827721798865) bank127115 +127116 POINT(47.54708761952592 -121.63073206761395) bank127116 +127117 POINT(47.641135786181444 -123.0267121171097) bank127117 +127118 POINT(47.75256622122984 -121.6061664142579) bank127118 +127119 POINT(46.96192840803663 -121.785675677998) bank127119 +127120 POINT(46.806390523208975 -123.07528503218661) bank127120 +127121 POINT(47.76304615866828 -121.767633300459) bank127121 +127122 POINT(48.23965972901692 -121.9501019917146) bank127122 +127123 POINT(47.673385204906864 -122.2390573263728) bank127123 +127124 POINT(46.86926450436597 -123.2180108865632) bank127124 +127125 POINT(47.56546481971755 -121.61888444642501) bank127125 +127126 POINT(48.45768644907876 -122.4768615721894) bank127126 +127127 POINT(47.113280378297596 -121.99698045274226) bank127127 +127128 POINT(47.12140504939915 -121.8242449553411) bank127128 +127129 POINT(48.47727131407671 -121.92698920379773) bank127129 +127130 POINT(46.621460333115195 -122.09381534384858) bank127130 +127131 POINT(48.29322515773903 -122.83936109039655) bank127131 +127132 POINT(48.480021727726 -122.76200623132952) bank127132 +127133 POINT(48.48779876023526 -122.81016973537541) bank127133 +127134 POINT(47.67509222664344 -121.64547862905195) bank127134 +127135 POINT(48.29074994852577 -122.54814439141734) bank127135 +127136 POINT(46.79952693258722 -121.98245520358877) bank127136 +127137 POINT(48.075542806942124 -123.11104269704096) bank127137 +127138 POINT(48.18555794237851 -122.69679840174383) bank127138 +127139 POINT(48.08950184175621 -122.86742260437231) bank127139 +127140 POINT(47.85693920459687 -122.5481440395897) bank127140 +127141 POINT(47.94482090178581 -122.98457274152754) bank127141 +127142 POINT(48.06221152216182 -121.73502704659889) bank127142 +127143 POINT(48.28650531184193 -121.53074237297496) bank127143 +127144 POINT(47.39049465752849 -122.73464851113391) bank127144 +127145 POINT(48.26351242440568 -121.68962856453217) bank127145 +127146 POINT(47.110462186394294 -122.41707925962726) bank127146 +127147 POINT(48.26858615134343 -121.53358011204023) bank127147 +127148 POINT(47.90077031535129 -123.19229532992168) bank127148 +127149 POINT(48.36468608883565 -123.25330652350341) bank127149 +127150 POINT(47.37678400086099 -122.96562679336557) bank127150 +127151 POINT(48.30321144948782 -121.61285539857566) bank127151 +127152 POINT(47.07416258325833 -122.19269039105133) bank127152 +127153 POINT(47.82691796409161 -122.1011285678918) bank127153 +127154 POINT(46.96936377158381 -121.96898180428272) bank127154 +127155 POINT(47.348619252198155 -123.31361302019721) bank127155 +127156 POINT(47.43020039739486 -121.79291128845814) bank127156 +127157 POINT(48.36906709272534 -123.04956134806908) bank127157 +127158 POINT(47.88386439498871 -122.84657136342159) bank127158 +127159 POINT(47.74783037098121 -122.58021152574223) bank127159 +127160 POINT(48.2242369418882 -123.29644636326495) bank127160 +127161 POINT(47.29749511272024 -121.537997409562) bank127161 +127162 POINT(48.1447268707216 -123.05949289185291) bank127162 +127163 POINT(47.254949499853026 -121.35457443643843) bank127163 +127164 POINT(47.49326517409074 -123.13195288627676) bank127164 +127165 POINT(47.62146561965482 -122.01748927058115) bank127165 +127166 POINT(48.60230177421516 -122.62147532388803) bank127166 +127167 POINT(47.69461643183392 -121.66511584667774) bank127167 +127168 POINT(47.277853147836765 -122.54477375528604) bank127168 +127169 POINT(47.91199543003949 -122.88381362767986) bank127169 +127170 POINT(48.450916113827816 -122.0437312815345) bank127170 +127171 POINT(47.40815108444374 -122.47840775192356) bank127171 +127172 POINT(47.90084198776969 -123.30981376725266) bank127172 +127173 POINT(47.63344391399964 -121.56430289082168) bank127173 +127174 POINT(47.012546380724125 -123.13761384660035) bank127174 +127175 POINT(48.085143687357665 -121.49216577860292) bank127175 +127176 POINT(46.85244181776726 -121.3335272594549) bank127176 +127177 POINT(48.46917810076077 -122.88145249163904) bank127177 +127178 POINT(46.94422281409199 -122.75232846582962) bank127178 +127179 POINT(46.730025853640576 -121.99925536101166) bank127179 +127180 POINT(47.42525942455294 -122.73247768097984) bank127180 +127181 POINT(48.27413568004504 -122.27129540283416) bank127181 +127182 POINT(46.64806302711482 -121.36751201005251) bank127182 +127183 POINT(47.85400581925883 -121.83094419193242) bank127183 +127184 POINT(48.21668646971192 -122.10230448819654) bank127184 +127185 POINT(48.40179837592366 -123.2455555803202) bank127185 +127186 POINT(48.55393170647569 -121.90309940194086) bank127186 +127187 POINT(47.94531525591963 -121.82058973623595) bank127187 +127188 POINT(48.464348313918144 -121.87911943714043) bank127188 +127189 POINT(48.317106208021805 -121.50937059504854) bank127189 +127190 POINT(48.48674668908391 -121.76884614370873) bank127190 +127191 POINT(47.20438314303835 -121.4048969458519) bank127191 +127192 POINT(47.217732737421635 -122.92100460306283) bank127192 +127193 POINT(47.52272963228436 -122.9484214090564) bank127193 +127194 POINT(48.53227935594161 -122.68012448098092) bank127194 +127195 POINT(47.815344638072744 -121.91651960566065) bank127195 +127196 POINT(48.19782791627809 -121.88051446332017) bank127196 +127197 POINT(47.875345392721584 -122.00613615966559) bank127197 +127198 POINT(47.86508761120174 -122.22823243724297) bank127198 +127199 POINT(47.39962986795997 -122.47421420492824) bank127199 +127200 POINT(47.5209209359958 -122.05700134524422) bank127200 +127201 POINT(47.070156109146076 -122.94858237311819) bank127201 +127202 POINT(46.92126238706847 -122.6413034675432) bank127202 +127203 POINT(46.87076057083256 -122.25762248906761) bank127203 +127204 POINT(48.567168070642936 -121.5686225370572) bank127204 +127205 POINT(47.27894074009125 -122.53920505896284) bank127205 +127206 POINT(47.383170604411404 -121.65155732350755) bank127206 +127207 POINT(47.88241496539964 -121.88891643107543) bank127207 +127208 POINT(48.235733727894335 -122.51294966590713) bank127208 +127209 POINT(48.44010338388862 -122.97870169007257) bank127209 +127210 POINT(47.73424458593989 -122.22042382026005) bank127210 +127211 POINT(48.13792621674318 -121.96206864836287) bank127211 +127212 POINT(48.34694259056349 -121.64319810142186) bank127212 +127213 POINT(47.08391941521468 -122.4585824303837) bank127213 +127214 POINT(47.59739705381205 -122.0153545667273) bank127214 +127215 POINT(48.17271829173082 -122.45900765736943) bank127215 +127216 POINT(46.79453114552966 -122.8962217215637) bank127216 +127217 POINT(47.545306544905266 -123.00071029781027) bank127217 +127218 POINT(47.01228779468341 -122.93903280124348) bank127218 +127219 POINT(46.95854188756954 -121.43895035810027) bank127219 +127220 POINT(48.11525032715882 -122.38232171280863) bank127220 +127221 POINT(47.043392495429615 -121.92299078179298) bank127221 +127222 POINT(48.32619418083944 -123.01150727324067) bank127222 +127223 POINT(46.80150663183641 -122.9721838507204) bank127223 +127224 POINT(48.242115251485586 -121.46005887851067) bank127224 +127225 POINT(47.59865724504344 -122.09058439191139) bank127225 +127226 POINT(47.068412296055996 -122.28724509397587) bank127226 +127227 POINT(47.415479032528914 -122.07778937100383) bank127227 +127228 POINT(47.52553658126843 -121.43253521471048) bank127228 +127229 POINT(47.10024081024211 -122.15066806776613) bank127229 +127230 POINT(47.55465855812984 -123.19883905026747) bank127230 +127231 POINT(46.99574175959109 -122.25279773584437) bank127231 +127232 POINT(47.219366854544575 -122.06338975728698) bank127232 +127233 POINT(47.64746760559801 -121.90457743758577) bank127233 +127234 POINT(47.154130068949 -121.60905106283714) bank127234 +127235 POINT(48.28745325493806 -122.09367832707916) bank127235 +127236 POINT(47.06591996835212 -123.25466640369305) bank127236 +127237 POINT(46.74231550100307 -122.15439642759502) bank127237 +127238 POINT(47.00289984635434 -123.20665319457639) bank127238 +127239 POINT(47.763570723546046 -122.11976380154069) bank127239 +127240 POINT(47.09104129913353 -121.60702404920045) bank127240 +127241 POINT(46.970662263253104 -121.35370327107238) bank127241 +127242 POINT(46.77829000686257 -122.29915953312631) bank127242 +127243 POINT(48.05662075554336 -123.32358805920241) bank127243 +127244 POINT(47.3655351463599 -122.81465897434877) bank127244 +127245 POINT(48.19945129129706 -122.59140098021503) bank127245 +127246 POINT(46.943672661387474 -121.62608590436953) bank127246 +127247 POINT(47.19788336655847 -121.56178312920248) bank127247 +127248 POINT(46.62592267235315 -123.25165169289353) bank127248 +127249 POINT(47.841359110162784 -123.15548241103413) bank127249 +127250 POINT(47.829185262333404 -122.1678586411539) bank127250 +127251 POINT(47.02129432460678 -121.71024142316276) bank127251 +127252 POINT(48.41322193661357 -122.79728703157899) bank127252 +127253 POINT(46.778488106703946 -123.14039830045473) bank127253 +127254 POINT(48.5656062129338 -123.00674927903376) bank127254 +127255 POINT(48.49182504765355 -122.10301512008547) bank127255 +127256 POINT(48.05868242817433 -121.93221806084722) bank127256 +127257 POINT(48.510360753295515 -123.02447839509067) bank127257 +127258 POINT(48.46228245238345 -122.14025251811613) bank127258 +127259 POINT(46.616552317514746 -122.33544048947694) bank127259 +127260 POINT(48.5557535378705 -122.15762500646687) bank127260 +127261 POINT(47.72060958953565 -122.29408899966788) bank127261 +127262 POINT(46.79275519704621 -121.92009044967881) bank127262 +127263 POINT(47.13008799230169 -121.42664383557097) bank127263 +127264 POINT(48.468729767490444 -121.5389803452166) bank127264 +127265 POINT(46.94954729010877 -121.38833696905995) bank127265 +127266 POINT(47.40349185105621 -122.65808148175685) bank127266 +127267 POINT(47.278997001999315 -121.64116200939246) bank127267 +127268 POINT(47.205047735020386 -122.80400835087296) bank127268 +127269 POINT(47.60841406990433 -122.77632801259502) bank127269 +127270 POINT(47.032963880073645 -122.95184283537264) bank127270 +127271 POINT(47.96441677210119 -121.57740161597302) bank127271 +127272 POINT(46.84906242039389 -121.34506958543345) bank127272 +127273 POINT(48.38787705377595 -122.06224829653397) bank127273 +127274 POINT(48.212146951622614 -121.6425934289842) bank127274 +127275 POINT(48.1284822729644 -122.76506412353766) bank127275 +127276 POINT(48.11970072946683 -122.4745323433325) bank127276 +127277 POINT(47.00628814796441 -121.8247568936378) bank127277 +127278 POINT(47.180091050154246 -121.588738816091) bank127278 +127279 POINT(47.54519796039216 -123.26552470154114) bank127279 +127280 POINT(47.1781708598538 -121.79842639394053) bank127280 +127281 POINT(47.84012343148788 -123.08085320557626) bank127281 +127282 POINT(47.93606511950162 -123.07146718483149) bank127282 +127283 POINT(46.80875507946638 -121.92524372341691) bank127283 +127284 POINT(47.55781991569078 -123.19614953066078) bank127284 +127285 POINT(47.138421046520314 -121.63025112763427) bank127285 +127286 POINT(47.73645487301199 -123.30102424710057) bank127286 +127287 POINT(47.69288197665685 -122.33684429545897) bank127287 +127288 POINT(47.23998799021091 -121.53114342396958) bank127288 +127289 POINT(47.46459141607206 -121.42977958905796) bank127289 +127290 POINT(46.701888231002776 -121.79339955077295) bank127290 +127291 POINT(46.7495616786433 -122.31527267404141) bank127291 +127292 POINT(47.14065810595918 -122.92449756261155) bank127292 +127293 POINT(48.37481546071734 -121.55913521509966) bank127293 +127294 POINT(46.9321274758982 -122.79761967422614) bank127294 +127295 POINT(47.54789350079297 -122.4970106606032) bank127295 +127296 POINT(47.38222028021298 -121.76576992031599) bank127296 +127297 POINT(48.1253962109471 -123.32947794293385) bank127297 +127298 POINT(47.88330189440627 -122.7303984822609) bank127298 +127299 POINT(48.023353417189234 -123.06459487033646) bank127299 +127300 POINT(47.04043698031779 -123.08219324425792) bank127300 +127301 POINT(47.76213115370682 -121.53899800978445) bank127301 +127302 POINT(48.063476752235225 -122.49877414052452) bank127302 +127303 POINT(47.55210408691348 -122.65870593822267) bank127303 +127304 POINT(47.59587944235821 -121.33332593716803) bank127304 +127305 POINT(48.38280715885454 -121.68175276722707) bank127305 +127306 POINT(47.61748038190326 -122.17337136286055) bank127306 +127307 POINT(48.18420230378184 -122.98575269775264) bank127307 +127308 POINT(47.27808346067703 -122.83886155847262) bank127308 +127309 POINT(47.138261300174875 -123.12415975319543) bank127309 +127310 POINT(48.54263168119306 -122.81032507599897) bank127310 +127311 POINT(48.564432619181034 -123.24472065976997) bank127311 +127312 POINT(47.68514302504293 -122.62472687473101) bank127312 +127313 POINT(46.82666298858626 -121.66390099416962) bank127313 +127314 POINT(47.94729618796697 -122.71174665259825) bank127314 +127315 POINT(47.502071561185 -122.63549376164802) bank127315 +127316 POINT(47.728513722566575 -123.23775513228294) bank127316 +127317 POINT(47.78324671769008 -123.24146615590463) bank127317 +127318 POINT(47.2191601744697 -121.49515216849522) bank127318 +127319 POINT(48.14081238422933 -121.65120327655606) bank127319 +127320 POINT(47.47409281854078 -122.68019918155541) bank127320 +127321 POINT(48.56513349696195 -123.19482497627325) bank127321 +127322 POINT(47.36460726352651 -121.65595556695817) bank127322 +127323 POINT(47.57986954653359 -121.46923315770199) bank127323 +127324 POINT(46.6212199383674 -122.08195650572148) bank127324 +127325 POINT(48.51190972777573 -121.84913261383842) bank127325 +127326 POINT(46.78651283116183 -121.76760340512163) bank127326 +127327 POINT(48.038198302754104 -122.69613493285324) bank127327 +127328 POINT(48.33184427192881 -121.5295395846013) bank127328 +127329 POINT(48.43303762663004 -121.4343691021556) bank127329 +127330 POINT(47.82491257359754 -123.18470002274202) bank127330 +127331 POINT(48.563633733922416 -122.18019098578372) bank127331 +127332 POINT(47.72102857116815 -121.4430146364293) bank127332 +127333 POINT(48.44801113879894 -122.1367452045796) bank127333 +127334 POINT(46.7185211620253 -122.01349143204209) bank127334 +127335 POINT(47.220032980537965 -122.84715104006347) bank127335 +127336 POINT(47.80309670473509 -121.52155950784277) bank127336 +127337 POINT(48.29014894124725 -122.54099556216808) bank127337 +127338 POINT(47.637088173082034 -122.41923265373404) bank127338 +127339 POINT(47.923222903780704 -122.66717703712041) bank127339 +127340 POINT(47.777566771553815 -121.65469705799018) bank127340 +127341 POINT(47.258186774971364 -122.31555971460956) bank127341 +127342 POINT(47.669950521084 -122.47425755580021) bank127342 +127343 POINT(48.230304583287975 -122.37024865436592) bank127343 +127344 POINT(47.99123087184158 -122.04820808556761) bank127344 +127345 POINT(47.45887005483453 -122.3646847092376) bank127345 +127346 POINT(47.13661489550216 -122.27956793608574) bank127346 +127347 POINT(47.55733449714542 -122.37413804479615) bank127347 +127348 POINT(47.033838551963896 -121.74327491846181) bank127348 +127349 POINT(46.88318857229004 -121.70832178197385) bank127349 +127350 POINT(47.66609733344743 -122.5268081639547) bank127350 +127351 POINT(47.27491800028117 -122.80232512564977) bank127351 +127352 POINT(47.92095066600159 -122.83512625764527) bank127352 +127353 POINT(48.25374430886 -122.95464398406314) bank127353 +127354 POINT(47.74080405560127 -121.75509824551627) bank127354 +127355 POINT(47.55975778548519 -123.04116831758785) bank127355 +127356 POINT(48.362088527285536 -122.18717142160962) bank127356 +127357 POINT(47.1423581320174 -122.14950568325) bank127357 +127358 POINT(46.667525758225885 -123.29844917098988) bank127358 +127359 POINT(48.250987369471794 -122.15754987841404) bank127359 +127360 POINT(48.35031934162269 -121.36134902712074) bank127360 +127361 POINT(48.071290093314914 -123.03286213330715) bank127361 +127362 POINT(47.5194838024945 -122.72644674698157) bank127362 +127363 POINT(48.59381538538736 -121.38189417996036) bank127363 +127364 POINT(47.02476368332069 -122.1080070283791) bank127364 +127365 POINT(47.29447387027171 -121.64186802891129) bank127365 +127366 POINT(47.114365283096895 -123.11229799859369) bank127366 +127367 POINT(47.43976279514355 -121.3540571726859) bank127367 +127368 POINT(48.08413500213986 -121.41738161397599) bank127368 +127369 POINT(46.73791204661394 -122.82482218286201) bank127369 +127370 POINT(48.11201878956509 -123.23105281623745) bank127370 +127371 POINT(47.89866542421951 -121.92293175579151) bank127371 +127372 POINT(46.633107678522165 -121.61128224543566) bank127372 +127373 POINT(47.79322406380382 -121.4430959281731) bank127373 +127374 POINT(47.81682662040278 -122.02879181927763) bank127374 +127375 POINT(47.31852485525832 -122.38882799154764) bank127375 +127376 POINT(46.61922450881013 -121.78922376679046) bank127376 +127377 POINT(46.6683541424966 -122.13576189861014) bank127377 +127378 POINT(47.84506212984355 -121.60604525044667) bank127378 +127379 POINT(47.05959681711971 -122.89405687560449) bank127379 +127380 POINT(48.154246431959436 -123.15683479892222) bank127380 +127381 POINT(47.88569159052682 -122.8765867370475) bank127381 +127382 POINT(47.86701409229019 -121.35292376063659) bank127382 +127383 POINT(48.03335263390001 -122.74372557438898) bank127383 +127384 POINT(47.830222859093745 -121.66309081206795) bank127384 +127385 POINT(48.29565838866463 -122.44702373700277) bank127385 +127386 POINT(46.784866144456934 -122.75829251438346) bank127386 +127387 POINT(47.74997362235541 -122.34080857967909) bank127387 +127388 POINT(47.19846370478284 -123.1005253366609) bank127388 +127389 POINT(47.649491807652126 -121.71724977206098) bank127389 +127390 POINT(47.821263619830894 -123.07318129255289) bank127390 +127391 POINT(47.28035887636606 -122.44513947497052) bank127391 +127392 POINT(47.36031583944391 -121.84605276886516) bank127392 +127393 POINT(47.14213885865178 -122.77457323302346) bank127393 +127394 POINT(48.346780441322515 -122.53963313989107) bank127394 +127395 POINT(47.13734793632047 -123.08967065259166) bank127395 +127396 POINT(47.82551287522618 -122.93400688275128) bank127396 +127397 POINT(48.590664722091084 -122.04085014202442) bank127397 +127398 POINT(48.60570969811179 -121.47323331340442) bank127398 +127399 POINT(47.553050241315084 -123.19173892856881) bank127399 +127400 POINT(47.74585852498251 -122.09564955930614) bank127400 +127401 POINT(47.003027786185186 -123.26892848709184) bank127401 +127402 POINT(48.37114083544556 -122.07575698800039) bank127402 +127403 POINT(48.431329252930176 -123.29102179414306) bank127403 +127404 POINT(48.4326437936632 -123.04489453492843) bank127404 +127405 POINT(48.061464391671535 -121.858963093289) bank127405 +127406 POINT(47.3769162240369 -121.99755864442481) bank127406 +127407 POINT(47.328772005472004 -121.39799346289004) bank127407 +127408 POINT(47.8545115466231 -121.97788427413259) bank127408 +127409 POINT(47.79212684782514 -122.94836491113878) bank127409 +127410 POINT(48.38106050307597 -122.04403443817586) bank127410 +127411 POINT(48.44101776835251 -121.82780345340609) bank127411 +127412 POINT(47.06147435640803 -123.28561932134869) bank127412 +127413 POINT(47.61166561967963 -122.31863454468117) bank127413 +127414 POINT(47.373792480684486 -122.27361391674754) bank127414 +127415 POINT(47.2910239567992 -121.33768881448643) bank127415 +127416 POINT(46.652674265997966 -122.64033127020674) bank127416 +127417 POINT(48.20293343335536 -121.36690035203024) bank127417 +127418 POINT(47.22609750771756 -121.35191306082135) bank127418 +127419 POINT(48.471613595682 -122.62075563499063) bank127419 +127420 POINT(48.02103602602507 -121.60615133799504) bank127420 +127421 POINT(47.692676019865935 -122.8479723735679) bank127421 +127422 POINT(48.289532342834384 -121.96526736931281) bank127422 +127423 POINT(47.812614414119416 -121.59625092554388) bank127423 +127424 POINT(46.80481820713103 -121.46247829800912) bank127424 +127425 POINT(47.20830089776131 -122.18151594581451) bank127425 +127426 POINT(47.1025554029929 -122.11391323574705) bank127426 +127427 POINT(48.46693585670582 -121.42970017537577) bank127427 +127428 POINT(47.791321674255705 -123.24857300569415) bank127428 +127429 POINT(48.4349332159285 -121.60054160306596) bank127429 +127430 POINT(47.03014253065909 -122.6517667348744) bank127430 +127431 POINT(48.354857472437615 -123.1815687538838) bank127431 +127432 POINT(47.54823035916483 -121.94587527083486) bank127432 +127433 POINT(47.74034085718463 -121.9324372115352) bank127433 +127434 POINT(48.603875827480536 -123.29788708067088) bank127434 +127435 POINT(47.35357035626178 -122.40209484583147) bank127435 +127436 POINT(47.42423434979614 -122.36740927289652) bank127436 +127437 POINT(47.73430612037119 -123.120938879021) bank127437 +127438 POINT(48.16536274324416 -122.97313060983339) bank127438 +127439 POINT(47.01493172714994 -122.40651315170116) bank127439 +127440 POINT(47.222595634707055 -121.52937491919504) bank127440 +127441 POINT(47.584065469047694 -121.9551929015832) bank127441 +127442 POINT(47.84650528405552 -121.92966989799376) bank127442 +127443 POINT(47.56531882970404 -121.53821559144944) bank127443 +127444 POINT(47.14775587683023 -122.59988100136593) bank127444 +127445 POINT(47.16377406076743 -122.95103868832187) bank127445 +127446 POINT(47.049525780184474 -121.79417745035573) bank127446 +127447 POINT(48.27636829034426 -123.15259661967528) bank127447 +127448 POINT(47.321801683030415 -121.78655417594749) bank127448 +127449 POINT(47.64756260361262 -122.96518481513512) bank127449 +127450 POINT(48.540701063191456 -121.58277554866922) bank127450 +127451 POINT(47.7456105660863 -122.53674317950279) bank127451 +127452 POINT(47.75498107242212 -122.95449145652725) bank127452 +127453 POINT(48.429440911042384 -122.19874684769637) bank127453 +127454 POINT(47.899033795040985 -122.74029931835997) bank127454 +127455 POINT(47.96929326612464 -123.1439988633257) bank127455 +127456 POINT(48.07397864782299 -122.62487936792257) bank127456 +127457 POINT(47.352042090513656 -123.27444005168873) bank127457 +127458 POINT(48.24000047728865 -121.73724616464273) bank127458 +127459 POINT(48.2581415312288 -121.5915722907855) bank127459 +127460 POINT(47.04076839825251 -122.2834187157139) bank127460 +127461 POINT(47.14925631486686 -121.88598145195799) bank127461 +127462 POINT(48.19923158967785 -123.32836476145724) bank127462 +127463 POINT(47.31590382096172 -121.75091123865153) bank127463 +127464 POINT(48.02007849981355 -121.7168230686647) bank127464 +127465 POINT(48.018335109296416 -122.522237495773) bank127465 +127466 POINT(47.206164774688176 -122.67676770780969) bank127466 +127467 POINT(47.4176097914073 -122.49424193919744) bank127467 +127468 POINT(46.84148435778439 -122.91594540408144) bank127468 +127469 POINT(48.06495566455042 -121.77567007017213) bank127469 +127470 POINT(47.310387622003304 -122.07485958897237) bank127470 +127471 POINT(48.397616497252514 -122.60912636653899) bank127471 +127472 POINT(47.66417599650627 -122.43425387271077) bank127472 +127473 POINT(46.65014617821998 -121.75023364276913) bank127473 +127474 POINT(47.43904115102755 -123.17694340747873) bank127474 +127475 POINT(46.7010720474894 -122.63274453842216) bank127475 +127476 POINT(47.15271010289739 -122.96167643577034) bank127476 +127477 POINT(47.36840444813913 -122.26898924563834) bank127477 +127478 POINT(46.69025477738543 -121.70080117816178) bank127478 +127479 POINT(46.68584425697294 -123.27500267697464) bank127479 +127480 POINT(48.409248175113554 -121.6363192524692) bank127480 +127481 POINT(48.000474582772895 -123.09314046879767) bank127481 +127482 POINT(48.26448599223843 -123.23950468740233) bank127482 +127483 POINT(47.02119403795085 -121.34505452866581) bank127483 +127484 POINT(47.32416459677001 -122.37330412897941) bank127484 +127485 POINT(47.64860923393412 -122.76487325050931) bank127485 +127486 POINT(47.180687794252655 -122.00795079117499) bank127486 +127487 POINT(47.99260093187187 -123.06720640209802) bank127487 +127488 POINT(47.71845181723404 -122.57116718001352) bank127488 +127489 POINT(47.825327811787204 -121.35997878822127) bank127489 +127490 POINT(48.043375669173244 -121.83055985864095) bank127490 +127491 POINT(48.249255496175294 -122.96552686951622) bank127491 +127492 POINT(46.69649191480796 -122.43115240381469) bank127492 +127493 POINT(48.35496169188055 -121.58617388017268) bank127493 +127494 POINT(46.75059855976468 -123.18337720294225) bank127494 +127495 POINT(46.838977530687785 -122.26213616902062) bank127495 +127496 POINT(47.50105773292403 -121.40079676459484) bank127496 +127497 POINT(48.16655146701128 -122.72502201910171) bank127497 +127498 POINT(46.84581311309712 -122.78974071505561) bank127498 +127499 POINT(47.62011629926583 -121.41779048131048) bank127499 +127500 POINT(47.30285057635228 -121.3469128816509) bank127500 +127501 POINT(47.9244007526787 -122.8694465933157) bank127501 +127502 POINT(47.1489116970747 -122.43338295526068) bank127502 +127503 POINT(48.55866457504599 -123.18023953080314) bank127503 +127504 POINT(48.601468938378645 -122.85831787800284) bank127504 +127505 POINT(47.73206994836539 -122.00961932634158) bank127505 +127506 POINT(47.175103088672394 -123.00650311723267) bank127506 +127507 POINT(46.90422339970054 -123.09572038148055) bank127507 +127508 POINT(47.30865355499398 -121.85776433817297) bank127508 +127509 POINT(48.46749883023901 -122.35121645711322) bank127509 +127510 POINT(46.82890364144118 -122.06028803347088) bank127510 +127511 POINT(48.19819503784328 -122.23872786803834) bank127511 +127512 POINT(48.580796676474826 -122.03636260522438) bank127512 +127513 POINT(47.94598086177416 -122.1563524938892) bank127513 +127514 POINT(47.22673357895913 -123.2284153390897) bank127514 +127515 POINT(48.17444329566084 -121.51959054577746) bank127515 +127516 POINT(48.44025162477211 -121.38011015474932) bank127516 +127517 POINT(47.60570931889072 -122.86751717036091) bank127517 +127518 POINT(47.039952033490586 -122.58218693500878) bank127518 +127519 POINT(48.5176855667409 -122.73163125091307) bank127519 +127520 POINT(47.49549813982276 -121.44286862406503) bank127520 +127521 POINT(48.04280132839114 -122.33987662982219) bank127521 +127522 POINT(47.1658953935671 -122.2041220354101) bank127522 +127523 POINT(48.51557165462848 -122.47988304941248) bank127523 +127524 POINT(47.147728264527075 -121.40664688388861) bank127524 +127525 POINT(47.1118343968034 -122.75012812264578) bank127525 +127526 POINT(47.22705194782649 -121.46850915620794) bank127526 +127527 POINT(47.437899267796 -121.86086129223733) bank127527 +127528 POINT(48.392721924173124 -121.73307599467324) bank127528 +127529 POINT(47.06109099873908 -122.2889679193578) bank127529 +127530 POINT(46.848179055680255 -122.31145491840682) bank127530 +127531 POINT(48.52442938576345 -123.00504407999267) bank127531 +127532 POINT(47.95024355763349 -122.48848506354938) bank127532 +127533 POINT(47.26285263631839 -121.4475819349694) bank127533 +127534 POINT(47.070800680647466 -122.16393985776436) bank127534 +127535 POINT(47.46388507986844 -121.77382514825307) bank127535 +127536 POINT(48.21603476128907 -121.82798211244028) bank127536 +127537 POINT(48.22323968920197 -121.39534444747379) bank127537 +127538 POINT(47.08323828927473 -122.75940220281639) bank127538 +127539 POINT(47.51712325405953 -122.26202384742733) bank127539 +127540 POINT(48.40145507898539 -121.3594989496037) bank127540 +127541 POINT(47.08854245311488 -121.99355124901629) bank127541 +127542 POINT(48.55713121320487 -121.63940137482523) bank127542 +127543 POINT(47.15209600309671 -122.23187139304852) bank127543 +127544 POINT(48.039916389066875 -122.5816764992074) bank127544 +127545 POINT(47.609982162978156 -122.68739044311965) bank127545 +127546 POINT(48.24729667190134 -121.98289464350249) bank127546 +127547 POINT(46.97314755537848 -121.74889313483519) bank127547 +127548 POINT(47.89546557790221 -122.44761332307792) bank127548 +127549 POINT(47.219207259764104 -122.75956496463314) bank127549 +127550 POINT(47.979764922691814 -121.54656504122597) bank127550 +127551 POINT(46.94807977208984 -122.97971849008307) bank127551 +127552 POINT(47.96923079518308 -121.69016959510121) bank127552 +127553 POINT(48.290826668999586 -121.96645483570795) bank127553 +127554 POINT(47.04902803143075 -123.2648493529659) bank127554 +127555 POINT(48.261026749352126 -121.88198099079737) bank127555 +127556 POINT(47.06780966023231 -122.83031972465145) bank127556 +127557 POINT(48.22860920370477 -121.53522447368447) bank127557 +127558 POINT(47.78231229410828 -123.00765850197412) bank127558 +127559 POINT(47.88701541638207 -123.11087336597123) bank127559 +127560 POINT(48.536481956000976 -121.81711008747074) bank127560 +127561 POINT(47.35869976828044 -122.96807884296561) bank127561 +127562 POINT(48.37166081646366 -121.74676038379248) bank127562 +127563 POINT(48.27441084550281 -122.98286333995611) bank127563 +127564 POINT(47.88803633106243 -123.14491644826896) bank127564 +127565 POINT(47.658431993697285 -121.79635832622071) bank127565 +127566 POINT(47.957520153173256 -123.21155440915007) bank127566 +127567 POINT(47.930849131006745 -121.57872361721707) bank127567 +127568 POINT(47.55613612234651 -121.55312359059016) bank127568 +127569 POINT(47.23369203101586 -123.06013140105814) bank127569 +127570 POINT(48.374452733487914 -122.40431861969749) bank127570 +127571 POINT(47.85171391540762 -123.20347395043171) bank127571 +127572 POINT(46.82421324267939 -121.36330641752588) bank127572 +127573 POINT(46.722870326290106 -121.40746249259593) bank127573 +127574 POINT(48.52542856703463 -122.14088076078802) bank127574 +127575 POINT(48.54172489436412 -121.62134723324087) bank127575 +127576 POINT(48.02733911525861 -123.21867489048661) bank127576 +127577 POINT(48.45727960053964 -122.6786661180855) bank127577 +127578 POINT(46.60984909259206 -122.32713262484188) bank127578 +127579 POINT(47.54215736104847 -122.72716649640306) bank127579 +127580 POINT(47.2352459540629 -122.01862142871349) bank127580 +127581 POINT(48.08533100915733 -121.39877694098483) bank127581 +127582 POINT(48.00714612747239 -121.58852955768187) bank127582 +127583 POINT(47.26269660138347 -122.11885082004042) bank127583 +127584 POINT(46.836810097985556 -121.60229419409312) bank127584 +127585 POINT(47.050308677791584 -122.4904848368946) bank127585 +127586 POINT(48.33038413284886 -121.76030294443333) bank127586 +127587 POINT(48.405444736126555 -121.43420625412263) bank127587 +127588 POINT(48.55147430759118 -122.05149334615649) bank127588 +127589 POINT(48.374469239055735 -122.51368532493635) bank127589 +127590 POINT(47.80249526359265 -122.08656753401266) bank127590 +127591 POINT(48.474256604629176 -122.69568279155517) bank127591 +127592 POINT(46.94671824188053 -122.31901619511834) bank127592 +127593 POINT(47.707249419863324 -122.90258994384526) bank127593 +127594 POINT(47.46435323892054 -121.88766178110698) bank127594 +127595 POINT(48.24846933546728 -123.30775590443172) bank127595 +127596 POINT(48.12614201252592 -121.54447770071293) bank127596 +127597 POINT(47.12206637956431 -122.3282958666624) bank127597 +127598 POINT(47.499085964530295 -121.88415659404993) bank127598 +127599 POINT(46.69542747037997 -121.37663947149655) bank127599 +127600 POINT(48.57106228873317 -122.6252579463204) bank127600 +127601 POINT(46.650955599417195 -122.28672298043995) bank127601 +127602 POINT(47.540004121906065 -121.81204170093574) bank127602 +127603 POINT(47.259123929551016 -121.38048263556729) bank127603 +127604 POINT(47.04190159746902 -123.30718293135102) bank127604 +127605 POINT(48.581649246079856 -122.54166845823137) bank127605 +127606 POINT(48.27867042007104 -122.9803288975669) bank127606 +127607 POINT(46.81060436271632 -122.09231884923211) bank127607 +127608 POINT(47.73328322699302 -122.09649461461893) bank127608 +127609 POINT(47.55515205255568 -123.22558976239256) bank127609 +127610 POINT(46.61382027842168 -123.17753806397292) bank127610 +127611 POINT(47.37528374351352 -122.58182060379852) bank127611 +127612 POINT(48.17650491762856 -122.6470608777161) bank127612 +127613 POINT(46.77765467007106 -121.78682009173716) bank127613 +127614 POINT(46.858739432354746 -122.31286386102978) bank127614 +127615 POINT(46.9332437385079 -122.94828186716757) bank127615 +127616 POINT(47.09357340923506 -123.08083841886716) bank127616 +127617 POINT(47.868809071206364 -122.65967135976098) bank127617 +127618 POINT(47.63118028877751 -122.58151066401966) bank127618 +127619 POINT(46.63881238968555 -122.25795114226209) bank127619 +127620 POINT(46.9122819077268 -121.93061248448045) bank127620 +127621 POINT(47.15990568091038 -121.67770499650577) bank127621 +127622 POINT(48.383563064923365 -123.21266974926411) bank127622 +127623 POINT(46.62641076345692 -122.78834114227934) bank127623 +127624 POINT(47.27617746720809 -122.89450482531626) bank127624 +127625 POINT(47.45832024488796 -121.44268608481848) bank127625 +127626 POINT(47.29985325788646 -123.21743794813767) bank127626 +127627 POINT(47.7534079577266 -122.22838696175646) bank127627 +127628 POINT(48.58128763088787 -122.88992316782435) bank127628 +127629 POINT(48.54787076130431 -122.7696269475242) bank127629 +127630 POINT(48.093628650157235 -121.66805820833503) bank127630 +127631 POINT(47.59106445074786 -122.09473073134723) bank127631 +127632 POINT(47.034283614925364 -122.8175084866629) bank127632 +127633 POINT(48.366339779496066 -123.1202002924373) bank127633 +127634 POINT(47.991893832361654 -122.22832376805884) bank127634 +127635 POINT(47.76853330482156 -123.28581826452788) bank127635 +127636 POINT(46.69535273189398 -122.84472087918718) bank127636 +127637 POINT(48.37956747768007 -121.53358819713844) bank127637 +127638 POINT(48.14776948949725 -122.18527232393251) bank127638 +127639 POINT(47.24497341531824 -122.4501368685456) bank127639 +127640 POINT(46.687204050443796 -121.4868908247694) bank127640 +127641 POINT(47.200196196984834 -121.6157616764666) bank127641 +127642 POINT(47.75748389298136 -121.56173258373607) bank127642 +127643 POINT(47.04563066621355 -121.84408618585361) bank127643 +127644 POINT(48.369722841619236 -121.3534833498461) bank127644 +127645 POINT(48.16311595046273 -123.166289432032) bank127645 +127646 POINT(48.5510218555855 -122.22776719183518) bank127646 +127647 POINT(47.16223600359518 -122.12217606074141) bank127647 +127648 POINT(47.749613708234854 -122.30898136304988) bank127648 +127649 POINT(47.805294903330235 -123.2032805965061) bank127649 +127650 POINT(47.62269644331197 -122.04937830129067) bank127650 +127651 POINT(46.74337381323693 -122.31205244125113) bank127651 +127652 POINT(47.72849998882301 -121.94081550902627) bank127652 +127653 POINT(47.235267979735625 -121.76896663373265) bank127653 +127654 POINT(47.37919920174489 -121.68515404022844) bank127654 +127655 POINT(48.0319267401325 -123.2836291388252) bank127655 +127656 POINT(47.32567009210623 -121.77244226457513) bank127656 +127657 POINT(48.1615771954742 -121.46225276076783) bank127657 +127658 POINT(48.32972216449794 -121.34820987189077) bank127658 +127659 POINT(47.87524797978222 -123.28949961338125) bank127659 +127660 POINT(47.637701078378484 -121.73228959750662) bank127660 +127661 POINT(48.33392178783694 -123.15042503155894) bank127661 +127662 POINT(47.89724748571223 -123.0996953460204) bank127662 +127663 POINT(47.56255678566965 -121.33961903999243) bank127663 +127664 POINT(48.00531723998955 -122.99890739309532) bank127664 +127665 POINT(48.414553509503065 -121.91998473319562) bank127665 +127666 POINT(47.63098145654111 -122.86623456607462) bank127666 +127667 POINT(47.249176022161876 -122.84385202286101) bank127667 +127668 POINT(47.39146697806195 -122.72537934834891) bank127668 +127669 POINT(47.71608371727224 -123.23856684251277) bank127669 +127670 POINT(47.49651052260167 -122.14144659327431) bank127670 +127671 POINT(48.207601128306685 -123.21147542971546) bank127671 +127672 POINT(47.47021927499129 -121.72124946382334) bank127672 +127673 POINT(47.983856874036015 -122.3380124186597) bank127673 +127674 POINT(47.68801721661531 -123.17569760493554) bank127674 +127675 POINT(47.32399398464836 -121.60558320118183) bank127675 +127676 POINT(46.714082211307975 -122.38618960157224) bank127676 +127677 POINT(48.276343140431905 -122.91939692640305) bank127677 +127678 POINT(48.5364847369775 -122.61419869163488) bank127678 +127679 POINT(48.062118516069866 -121.89854469896703) bank127679 +127680 POINT(46.84029364684526 -122.44681508910092) bank127680 +127681 POINT(48.314878675550915 -122.4283528670658) bank127681 +127682 POINT(46.88208731661072 -123.01505975317359) bank127682 +127683 POINT(47.116603550555 -121.4347489675204) bank127683 +127684 POINT(47.51848254311314 -122.8216825585258) bank127684 +127685 POINT(47.67633999966565 -121.36685108436416) bank127685 +127686 POINT(47.04104953901691 -122.97549844789202) bank127686 +127687 POINT(47.68702725830975 -122.07985816635014) bank127687 +127688 POINT(48.31183195292689 -122.26161473994453) bank127688 +127689 POINT(48.41295850378711 -122.42190271097161) bank127689 +127690 POINT(47.665290117788054 -122.11779659581214) bank127690 +127691 POINT(47.19943405972556 -123.08574427175152) bank127691 +127692 POINT(47.81224145723505 -121.37690740348057) bank127692 +127693 POINT(47.133029671522515 -122.53623665781876) bank127693 +127694 POINT(47.817814470773826 -122.40251523477306) bank127694 +127695 POINT(46.98686023144336 -122.3854452769728) bank127695 +127696 POINT(47.29321559889939 -121.4652514998058) bank127696 +127697 POINT(46.75898737064592 -123.2076687834006) bank127697 +127698 POINT(47.77001675467223 -122.96505372787016) bank127698 +127699 POINT(47.76818773685806 -123.32599686727087) bank127699 +127700 POINT(47.11227644019715 -122.18911512142425) bank127700 +127701 POINT(48.25268948045169 -122.10292276675438) bank127701 +127702 POINT(46.95020395456319 -122.37088271382159) bank127702 +127703 POINT(47.71707879606518 -122.5891838525438) bank127703 +127704 POINT(46.65901415550886 -122.32496493812046) bank127704 +127705 POINT(48.07906120920706 -122.97279826201344) bank127705 +127706 POINT(46.78750360868364 -122.84964369197068) bank127706 +127707 POINT(47.476449715115095 -122.37616142064698) bank127707 +127708 POINT(47.3355550712462 -121.65601295723964) bank127708 +127709 POINT(47.99103985019684 -122.38547635168531) bank127709 +127710 POINT(46.98714984627498 -123.01997600514456) bank127710 +127711 POINT(46.609824514891734 -122.5242142114584) bank127711 +127712 POINT(47.27229302612495 -121.3715626621816) bank127712 +127713 POINT(47.484845155175826 -121.35696716583449) bank127713 +127714 POINT(46.77180898291222 -121.81994544413855) bank127714 +127715 POINT(46.96723331310238 -122.97144984953613) bank127715 +127716 POINT(48.08161414334811 -122.41483701315067) bank127716 +127717 POINT(47.917875000908495 -122.77656097058116) bank127717 +127718 POINT(47.81707225843102 -121.37749189389847) bank127718 +127719 POINT(46.81933438663918 -122.57942030387576) bank127719 +127720 POINT(48.31183738786911 -122.00375103924586) bank127720 +127721 POINT(46.607170528218894 -121.80031869635637) bank127721 +127722 POINT(47.180380114022064 -122.5803067146113) bank127722 +127723 POINT(48.13351718822848 -121.72026812066345) bank127723 +127724 POINT(48.04430560998004 -123.05885737325778) bank127724 +127725 POINT(47.79306005392435 -122.44869455552404) bank127725 +127726 POINT(48.53432251826522 -121.97048696135172) bank127726 +127727 POINT(48.165549095846615 -122.54112665839841) bank127727 +127728 POINT(47.6256394282883 -122.83244481259848) bank127728 +127729 POINT(47.66973649994635 -122.53621913883711) bank127729 +127730 POINT(46.74968351351187 -122.32111151308936) bank127730 +127731 POINT(46.66838522587458 -121.45200195566338) bank127731 +127732 POINT(47.806717885815864 -121.76304067003899) bank127732 +127733 POINT(48.23477829649591 -122.49146626645924) bank127733 +127734 POINT(46.97969813047255 -121.5113524146999) bank127734 +127735 POINT(48.19974273422576 -122.39633110377491) bank127735 +127736 POINT(47.18321991567485 -122.43087150873595) bank127736 +127737 POINT(47.49776781562122 -123.16348437280593) bank127737 +127738 POINT(48.5240639858244 -122.52867332410956) bank127738 +127739 POINT(47.322593961416665 -121.38165847754206) bank127739 +127740 POINT(48.418421498695174 -121.48083384622007) bank127740 +127741 POINT(48.052512380621096 -122.00536225600504) bank127741 +127742 POINT(46.90810415684581 -122.65556782079929) bank127742 +127743 POINT(48.277894058781385 -123.10283050764977) bank127743 +127744 POINT(48.3004349069119 -121.54100598777761) bank127744 +127745 POINT(48.603797396120655 -122.30490893993617) bank127745 +127746 POINT(47.766273536336655 -121.60856057757242) bank127746 +127747 POINT(48.539393547468514 -121.98271199711365) bank127747 +127748 POINT(46.64137377056455 -121.89939283831211) bank127748 +127749 POINT(48.53988510709936 -121.95242882582778) bank127749 +127750 POINT(48.34613647765335 -122.71637641656123) bank127750 +127751 POINT(48.28991836078333 -122.85434625104794) bank127751 +127752 POINT(46.775961412516246 -122.95067826864546) bank127752 +127753 POINT(47.883179508485995 -121.5909223052709) bank127753 +127754 POINT(47.49809973852716 -122.78795601815426) bank127754 +127755 POINT(47.017036761496456 -122.41045110573404) bank127755 +127756 POINT(48.54321749152389 -121.9253819685612) bank127756 +127757 POINT(48.04598368444739 -122.48447828516478) bank127757 +127758 POINT(48.58119545048878 -121.49651270367018) bank127758 +127759 POINT(47.50654126190971 -123.04419213476189) bank127759 +127760 POINT(48.53527926540791 -121.97373638141718) bank127760 +127761 POINT(47.738929023259345 -123.32910943769608) bank127761 +127762 POINT(47.22480091258109 -123.00033552435764) bank127762 +127763 POINT(46.681217877021716 -122.49102985115034) bank127763 +127764 POINT(47.82586533380213 -122.82629915218803) bank127764 +127765 POINT(47.809503901315104 -122.96412388761414) bank127765 +127766 POINT(47.02867523907507 -122.76264309727325) bank127766 +127767 POINT(46.988068743412164 -122.54957927947311) bank127767 +127768 POINT(48.24528225590173 -122.4251484172282) bank127768 +127769 POINT(46.86372414323205 -121.44521485907435) bank127769 +127770 POINT(47.13194787420003 -123.28068943936917) bank127770 +127771 POINT(46.858562053104045 -122.98825687225545) bank127771 +127772 POINT(48.41309306837265 -122.46056537231435) bank127772 +127773 POINT(46.939089625236555 -123.08724817170553) bank127773 +127774 POINT(47.96490374980298 -123.10898791040263) bank127774 +127775 POINT(47.52824221974632 -122.22625805558343) bank127775 +127776 POINT(46.912327437284965 -122.26023373422161) bank127776 +127777 POINT(46.76680086104458 -121.43992737936499) bank127777 +127778 POINT(48.1565256994934 -123.1646337179678) bank127778 +127779 POINT(47.16179239600166 -121.49956608086515) bank127779 +127780 POINT(48.125041477042124 -121.84947790380174) bank127780 +127781 POINT(46.65996374025572 -122.89841862446706) bank127781 +127782 POINT(46.978928755172184 -122.60816683193264) bank127782 +127783 POINT(46.825473276391264 -122.93579615686724) bank127783 +127784 POINT(46.94291070515907 -122.13890167145456) bank127784 +127785 POINT(46.875136510331494 -121.60145001082739) bank127785 +127786 POINT(46.8624389437882 -122.17592733112906) bank127786 +127787 POINT(47.47590620841702 -122.54560534137785) bank127787 +127788 POINT(48.48397824637755 -123.1323531679552) bank127788 +127789 POINT(46.70351722381245 -121.68786017147845) bank127789 +127790 POINT(48.246135506557835 -122.23250112300383) bank127790 +127791 POINT(47.389281784485846 -122.96408941577869) bank127791 +127792 POINT(46.98482981653871 -121.68284759511691) bank127792 +127793 POINT(47.66750181608451 -122.70855001222667) bank127793 +127794 POINT(48.33378448691252 -121.46259301006083) bank127794 +127795 POINT(46.93355585606696 -122.36094587743355) bank127795 +127796 POINT(48.43557721861904 -122.9378561856026) bank127796 +127797 POINT(46.691226269803494 -122.41772245134473) bank127797 +127798 POINT(48.29947855737353 -122.84863889714363) bank127798 +127799 POINT(46.91419684261534 -123.30736820965619) bank127799 +127800 POINT(47.424456122217435 -121.48371047997388) bank127800 +127801 POINT(47.5694283511826 -121.79335351661331) bank127801 +127802 POINT(46.82835413656915 -122.382910552681) bank127802 +127803 POINT(47.22385217819152 -121.81555380253201) bank127803 +127804 POINT(47.641760655934206 -122.03649362708653) bank127804 +127805 POINT(47.70330901916037 -121.84229610583387) bank127805 +127806 POINT(47.13642709572696 -122.43349319933256) bank127806 +127807 POINT(47.835106015665914 -121.35447586380313) bank127807 +127808 POINT(47.91489028708021 -122.26073643957612) bank127808 +127809 POINT(47.170020461330914 -121.67891725060991) bank127809 +127810 POINT(47.21218810325642 -122.96107303528356) bank127810 +127811 POINT(47.577463442076954 -122.04030307083886) bank127811 +127812 POINT(48.20495507464674 -121.50831158670107) bank127812 +127813 POINT(48.56629872664123 -122.80826144753351) bank127813 +127814 POINT(48.521194989359074 -121.68994923775448) bank127814 +127815 POINT(47.88168812641596 -123.0496837878224) bank127815 +127816 POINT(47.36423630719317 -122.29593023265285) bank127816 +127817 POINT(47.10052807056633 -123.21925566470496) bank127817 +127818 POINT(46.79146868611603 -122.18656147364524) bank127818 +127819 POINT(48.21227482596846 -121.67113535924086) bank127819 +127820 POINT(46.70669899553227 -123.1686610401151) bank127820 +127821 POINT(46.747576070582724 -121.76302879191604) bank127821 +127822 POINT(47.86871069308305 -123.01881924441953) bank127822 +127823 POINT(48.16758557992977 -122.93899405052413) bank127823 +127824 POINT(47.81282427525321 -122.85601376643955) bank127824 +127825 POINT(48.45069369075053 -123.00030863685028) bank127825 +127826 POINT(48.028106209045745 -122.19841637617839) bank127826 +127827 POINT(48.28774759892644 -122.53577443614334) bank127827 +127828 POINT(47.71466542443846 -121.88173514673358) bank127828 +127829 POINT(46.74804879328073 -122.0849436008237) bank127829 +127830 POINT(47.71124807321935 -123.12887817153776) bank127830 +127831 POINT(46.928698822303026 -122.64341609490853) bank127831 +127832 POINT(47.03462064740044 -122.90820995952335) bank127832 +127833 POINT(46.98445306965054 -121.40706340754794) bank127833 +127834 POINT(47.60230390055007 -121.87810696478715) bank127834 +127835 POINT(47.73610241183547 -121.68619705932059) bank127835 +127836 POINT(46.61407527060248 -122.56201346004094) bank127836 +127837 POINT(48.07937018448881 -122.84599419441207) bank127837 +127838 POINT(48.57268791401247 -122.78731843260293) bank127838 +127839 POINT(47.65312608206599 -122.78751658560078) bank127839 +127840 POINT(47.188516538677995 -123.00410916296418) bank127840 +127841 POINT(46.81032659313065 -122.14656983998299) bank127841 +127842 POINT(47.27845335447941 -121.59519191340216) bank127842 +127843 POINT(48.08726071353808 -123.16081644925666) bank127843 +127844 POINT(47.42861193862063 -122.62568937854152) bank127844 +127845 POINT(48.221714377322925 -122.77308910573623) bank127845 +127846 POINT(46.73715407766905 -122.91399225598167) bank127846 +127847 POINT(48.27620210344028 -121.486345220093) bank127847 +127848 POINT(47.68383562746667 -121.8705314376658) bank127848 +127849 POINT(46.97173867121913 -121.93703364106204) bank127849 +127850 POINT(48.40476983782876 -123.2657176433202) bank127850 +127851 POINT(46.63426798839612 -121.79953965070156) bank127851 +127852 POINT(47.220958101170346 -121.49280632737141) bank127852 +127853 POINT(47.778264119592336 -123.03599701707827) bank127853 +127854 POINT(48.602120900014164 -121.46859938319359) bank127854 +127855 POINT(46.80986397235652 -122.66236786950834) bank127855 +127856 POINT(47.169415053286315 -123.29507408479891) bank127856 +127857 POINT(48.079563945636465 -122.90998582636327) bank127857 +127858 POINT(47.693167457674356 -123.28060347666047) bank127858 +127859 POINT(46.861974509490594 -122.57800892431034) bank127859 +127860 POINT(47.42503165474952 -121.71251180490111) bank127860 +127861 POINT(48.10505658000715 -122.36919552473262) bank127861 +127862 POINT(47.91731818132576 -121.9810604601403) bank127862 +127863 POINT(46.83815035747229 -122.31361206533774) bank127863 +127864 POINT(46.94563103345858 -121.98782389367798) bank127864 +127865 POINT(47.74212949429407 -122.76201602069565) bank127865 +127866 POINT(48.38899723779148 -121.40565303841618) bank127866 +127867 POINT(47.81158957377866 -122.8977416602232) bank127867 +127868 POINT(46.96031902020555 -122.0548970248546) bank127868 +127869 POINT(46.98645113187012 -121.59843992717335) bank127869 +127870 POINT(48.411705930721105 -122.97875917097403) bank127870 +127871 POINT(48.21151328703373 -121.97979847765237) bank127871 +127872 POINT(48.19686270698496 -121.43817391399953) bank127872 +127873 POINT(47.41964488827493 -122.23013629336438) bank127873 +127874 POINT(47.65608164899272 -121.6394887164842) bank127874 +127875 POINT(48.420148415178836 -121.84775828422674) bank127875 +127876 POINT(47.50372714889433 -122.4731057031197) bank127876 +127877 POINT(47.734947703436895 -123.01850609256378) bank127877 +127878 POINT(47.02382907856273 -123.12355362471949) bank127878 +127879 POINT(47.36563640021454 -122.50706994155853) bank127879 +127880 POINT(47.680131756588324 -122.50701195754955) bank127880 +127881 POINT(47.49167714556773 -122.04543499204316) bank127881 +127882 POINT(48.3326462222595 -122.32829990623247) bank127882 +127883 POINT(47.53732635834743 -123.02797556553801) bank127883 +127884 POINT(47.02753418959529 -123.07884470760939) bank127884 +127885 POINT(47.490049259825334 -122.54517595535948) bank127885 +127886 POINT(46.835021502027104 -122.56867366639052) bank127886 +127887 POINT(48.19392503991004 -122.24144214893414) bank127887 +127888 POINT(47.47447178965112 -121.9025102249727) bank127888 +127889 POINT(47.66466005884158 -122.92468786910159) bank127889 +127890 POINT(48.40911054878063 -121.36303122347364) bank127890 +127891 POINT(48.16041217221154 -122.42517642997781) bank127891 +127892 POINT(48.24750929350231 -122.01908669112095) bank127892 +127893 POINT(47.746561838282126 -121.9590398751014) bank127893 +127894 POINT(47.840929607562884 -121.96701685494195) bank127894 +127895 POINT(47.0551635516656 -122.8259696439888) bank127895 +127896 POINT(47.09948754237184 -122.08403944126006) bank127896 +127897 POINT(46.6955666437366 -121.42766068334412) bank127897 +127898 POINT(48.000101223646254 -122.64591588511067) bank127898 +127899 POINT(48.05814878639645 -123.18993954430306) bank127899 +127900 POINT(47.02237245745172 -122.74598956629416) bank127900 +127901 POINT(48.239006601236895 -122.9782968334383) bank127901 +127902 POINT(46.797361033615495 -123.11821027766393) bank127902 +127903 POINT(47.918706517473474 -122.58019697915034) bank127903 +127904 POINT(47.81783414691099 -122.18975902813268) bank127904 +127905 POINT(47.73392454677502 -121.86365545275372) bank127905 +127906 POINT(47.05396312312245 -122.21770420508551) bank127906 +127907 POINT(48.561085935543844 -122.0144602977918) bank127907 +127908 POINT(47.852261224479385 -122.93199179599472) bank127908 +127909 POINT(46.67468805295118 -122.98186609499118) bank127909 +127910 POINT(48.02155142828058 -122.8013292439695) bank127910 +127911 POINT(47.30386939056689 -122.65077757807529) bank127911 +127912 POINT(47.973724598592334 -122.31803219386022) bank127912 +127913 POINT(47.89274184756524 -122.83376381296138) bank127913 +127914 POINT(46.75007965196427 -122.7758609868917) bank127914 +127915 POINT(46.83938352050661 -122.01374398441055) bank127915 +127916 POINT(46.726983044645905 -121.34195333896334) bank127916 +127917 POINT(47.86835490203679 -122.2630012986882) bank127917 +127918 POINT(47.4981438644966 -121.48016690319507) bank127918 +127919 POINT(47.677046221424234 -121.33589700722531) bank127919 +127920 POINT(48.12293715155659 -122.83339009376604) bank127920 +127921 POINT(47.499715996814366 -122.71803471686658) bank127921 +127922 POINT(47.82625800958225 -122.39629054366111) bank127922 +127923 POINT(47.54113665905091 -121.41105471629726) bank127923 +127924 POINT(47.84726428986868 -122.15797682919481) bank127924 +127925 POINT(47.38547490678 -122.99640875577568) bank127925 +127926 POINT(48.46850090186156 -123.02306261205305) bank127926 +127927 POINT(48.33387358999294 -123.25152661971393) bank127927 +127928 POINT(48.2841528401961 -122.44578207519652) bank127928 +127929 POINT(47.236232542668716 -123.02219593391656) bank127929 +127930 POINT(48.58890248239231 -122.42795987389609) bank127930 +127931 POINT(47.514550953591694 -122.35556491562895) bank127931 +127932 POINT(48.59981986817733 -123.18333906902129) bank127932 +127933 POINT(46.893899874970224 -123.06360561094623) bank127933 +127934 POINT(47.345019036317325 -121.75558095768358) bank127934 +127935 POINT(47.51097002821791 -121.7252546533408) bank127935 +127936 POINT(47.156898626126384 -122.38000342631709) bank127936 +127937 POINT(47.04917254103365 -122.46136612363111) bank127937 +127938 POINT(47.40897725022202 -123.00442158102808) bank127938 +127939 POINT(46.73099979586921 -122.43485190934696) bank127939 +127940 POINT(47.423175270283075 -122.56898843455103) bank127940 +127941 POINT(48.2333244361899 -121.94784927619358) bank127941 +127942 POINT(47.5457736397362 -121.45527804942742) bank127942 +127943 POINT(47.995491520440744 -122.38730373731688) bank127943 +127944 POINT(48.43641753870536 -123.22195763446622) bank127944 +127945 POINT(48.04603600487322 -121.46428542040911) bank127945 +127946 POINT(46.839560173864136 -121.60982085824091) bank127946 +127947 POINT(48.5619261220458 -121.92686028100643) bank127947 +127948 POINT(48.570385155779455 -123.30772499937555) bank127948 +127949 POINT(47.4029902004483 -121.49565392485867) bank127949 +127950 POINT(47.97654674224711 -121.47767546720618) bank127950 +127951 POINT(47.71197405492532 -123.20076464290892) bank127951 +127952 POINT(48.31860917935812 -121.9602066940185) bank127952 +127953 POINT(48.010213142332155 -123.31598586214199) bank127953 +127954 POINT(47.44632668744582 -121.98964840840074) bank127954 +127955 POINT(47.66808775930143 -122.03536764059324) bank127955 +127956 POINT(47.428821482141906 -122.54745890706221) bank127956 +127957 POINT(48.36632092495549 -123.03794484455794) bank127957 +127958 POINT(48.549825304025816 -123.14136425076882) bank127958 +127959 POINT(46.85226698530122 -122.27929891398013) bank127959 +127960 POINT(46.666913158918526 -121.38193903380666) bank127960 +127961 POINT(48.2146365018211 -123.22744281620518) bank127961 +127962 POINT(48.38326720499235 -122.12197989281206) bank127962 +127963 POINT(47.75149889396589 -123.20632578175295) bank127963 +127964 POINT(47.899974453344356 -122.70254716715935) bank127964 +127965 POINT(47.27631782409711 -121.82635683642697) bank127965 +127966 POINT(48.49465614316566 -121.37600800957989) bank127966 +127967 POINT(46.9814643482817 -123.06123317947201) bank127967 +127968 POINT(47.99861387955356 -121.36279382751327) bank127968 +127969 POINT(47.616025758747085 -122.07621528854291) bank127969 +127970 POINT(47.55832715953494 -121.90151404126458) bank127970 +127971 POINT(46.859462857010335 -123.11623297081141) bank127971 +127972 POINT(46.95226372127186 -122.57459879625894) bank127972 +127973 POINT(47.22782323014846 -122.50925718601181) bank127973 +127974 POINT(48.200206159477 -121.94366597132215) bank127974 +127975 POINT(47.2192147854879 -122.79792007974018) bank127975 +127976 POINT(47.366071629170584 -123.07234076148796) bank127976 +127977 POINT(46.67345253198182 -123.08798531917493) bank127977 +127978 POINT(47.78471053863115 -122.3085046104475) bank127978 +127979 POINT(46.99871468239914 -122.11455258719211) bank127979 +127980 POINT(47.087935100774935 -121.88081248822486) bank127980 +127981 POINT(47.144858138042366 -122.14131674134623) bank127981 +127982 POINT(47.090637783549504 -121.5523747447462) bank127982 +127983 POINT(47.33889954196989 -122.60279897429342) bank127983 +127984 POINT(46.91678718396048 -122.86527899707954) bank127984 +127985 POINT(48.41061273168158 -122.43742961091594) bank127985 +127986 POINT(47.28307685038831 -121.62535231145729) bank127986 +127987 POINT(47.70007305217636 -121.74157344244854) bank127987 +127988 POINT(47.21023838863997 -123.24887505714771) bank127988 +127989 POINT(47.56639346390032 -121.52099805539396) bank127989 +127990 POINT(47.0823835364963 -121.46612678979419) bank127990 +127991 POINT(47.92445376378581 -122.17330860051446) bank127991 +127992 POINT(47.93177163549535 -121.58878642638945) bank127992 +127993 POINT(47.333163117579566 -122.24989897451182) bank127993 +127994 POINT(47.56440969484102 -121.69271741017333) bank127994 +127995 POINT(48.015545546309525 -122.95058760104807) bank127995 +127996 POINT(48.58017969425632 -122.28104314724743) bank127996 +127997 POINT(46.920875446304635 -122.19642951188968) bank127997 +127998 POINT(47.557827397254826 -121.57205403037982) bank127998 +127999 POINT(48.58373222934272 -121.95043362064058) bank127999 +128000 POINT(48.40388156434746 -123.09351271074397) bank128000 +128001 POINT(47.255356638379055 -122.30790106461082) bank128001 +128002 POINT(47.3546047992536 -122.02161102596266) bank128002 +128003 POINT(47.933119920231455 -122.56610007698298) bank128003 +128004 POINT(47.04321140613048 -123.25062349061058) bank128004 +128005 POINT(46.99211473367628 -122.97632513946711) bank128005 +128006 POINT(46.66445963606859 -123.0961223441572) bank128006 +128007 POINT(47.2191494348566 -121.84436552095285) bank128007 +128008 POINT(47.40957004800461 -122.37066321299996) bank128008 +128009 POINT(47.22717800445357 -122.932016137334) bank128009 +128010 POINT(48.55427911336238 -121.71889016976041) bank128010 +128011 POINT(47.53500050918875 -122.9758930456055) bank128011 +128012 POINT(47.9502490761345 -122.45017204426698) bank128012 +128013 POINT(48.124579778697324 -122.66335647203205) bank128013 +128014 POINT(46.7511556057553 -122.33657363998002) bank128014 +128015 POINT(46.734847597458426 -121.89094132713453) bank128015 +128016 POINT(46.917686300482835 -122.63227466553698) bank128016 +128017 POINT(47.146596484126675 -122.30160532870777) bank128017 +128018 POINT(46.838333208552555 -121.8145061718553) bank128018 +128019 POINT(48.54417801040729 -121.64385882866014) bank128019 +128020 POINT(47.66533664849159 -122.43991080888769) bank128020 +128021 POINT(46.74233628183336 -122.23138104541167) bank128021 +128022 POINT(47.51758532569782 -122.29126227141897) bank128022 +128023 POINT(47.8165005407629 -121.53194104822248) bank128023 +128024 POINT(48.490264250562255 -122.19135857864707) bank128024 +128025 POINT(47.00284710333408 -122.32625072086785) bank128025 +128026 POINT(46.834551045094074 -122.85839702256449) bank128026 +128027 POINT(46.73097978544811 -122.07946469242661) bank128027 +128028 POINT(47.199756356777954 -121.50286360113122) bank128028 +128029 POINT(47.80247457467582 -122.6419005813738) bank128029 +128030 POINT(47.72151316121691 -122.84322256070632) bank128030 +128031 POINT(48.32832364734192 -121.72390472625663) bank128031 +128032 POINT(47.410120160846304 -121.70086493917707) bank128032 +128033 POINT(46.68823638840154 -123.2368120539093) bank128033 +128034 POINT(48.393079991166715 -122.45167091598996) bank128034 +128035 POINT(47.16305784207021 -123.11335590001505) bank128035 +128036 POINT(46.75339868067427 -123.0011054848325) bank128036 +128037 POINT(46.99767099227219 -122.94575736212136) bank128037 +128038 POINT(47.42401962280419 -122.51069705848035) bank128038 +128039 POINT(47.61201834418869 -122.28890734488243) bank128039 +128040 POINT(48.047783365338255 -122.27760880690607) bank128040 +128041 POINT(47.518817040560705 -123.22999793768888) bank128041 +128042 POINT(47.0861273115474 -122.65019667871343) bank128042 +128043 POINT(48.398154768757585 -122.09823502152732) bank128043 +128044 POINT(48.00747595241543 -122.30561897896358) bank128044 +128045 POINT(47.60464866780057 -123.21724152093572) bank128045 +128046 POINT(46.77697239859495 -121.66807879844498) bank128046 +128047 POINT(47.853030738734475 -123.07519893631525) bank128047 +128048 POINT(47.90062453984404 -123.10255472565142) bank128048 +128049 POINT(47.69412172268342 -122.79300020201599) bank128049 +128050 POINT(48.04698129087278 -121.81729883190188) bank128050 +128051 POINT(47.27103039592806 -121.6644150997334) bank128051 +128052 POINT(48.41250937438124 -121.656404220527) bank128052 +128053 POINT(48.37119512285668 -122.0666535016068) bank128053 +128054 POINT(46.86056465454463 -122.66867892857562) bank128054 +128055 POINT(47.36836547794507 -122.62101514829122) bank128055 +128056 POINT(46.82581897493326 -121.44593525763872) bank128056 +128057 POINT(48.3094482762695 -121.47135883976306) bank128057 +128058 POINT(46.872141371627436 -121.91274518717843) bank128058 +128059 POINT(48.105685965241655 -122.84032847894565) bank128059 +128060 POINT(47.87917252157808 -122.83678032676097) bank128060 +128061 POINT(48.34739270218631 -123.22528515641005) bank128061 +128062 POINT(46.685781998368576 -122.76361916433459) bank128062 +128063 POINT(48.08704226591852 -122.58787523346429) bank128063 +128064 POINT(46.71992135712705 -122.74170698447304) bank128064 +128065 POINT(48.391723487138584 -121.46060620665075) bank128065 +128066 POINT(47.57679435847436 -122.25133126431706) bank128066 +128067 POINT(46.985196354209265 -121.45671461380539) bank128067 +128068 POINT(48.00043995542193 -123.02874270900251) bank128068 +128069 POINT(48.187978217283465 -122.10893551952358) bank128069 +128070 POINT(48.04881544215689 -122.41441218690288) bank128070 +128071 POINT(47.795136538834015 -122.1430617248757) bank128071 +128072 POINT(47.47587311507531 -122.64185088404874) bank128072 +128073 POINT(48.1731343552496 -123.0388508182939) bank128073 +128074 POINT(46.98862300219225 -122.37770678657623) bank128074 +128075 POINT(47.250019991678826 -123.14147424244368) bank128075 +128076 POINT(48.28087997444427 -122.08367689107229) bank128076 +128077 POINT(47.62012603596168 -121.4326219753945) bank128077 +128078 POINT(46.910691347688704 -121.4203713634036) bank128078 +128079 POINT(46.9013019756901 -121.44769310124207) bank128079 +128080 POINT(47.52459549047531 -123.10249955217675) bank128080 +128081 POINT(47.99744170961256 -121.57143547671502) bank128081 +128082 POINT(47.7589743394749 -123.28648836896937) bank128082 +128083 POINT(48.52759673451614 -121.51258309038175) bank128083 +128084 POINT(48.56393858745104 -122.27886618688076) bank128084 +128085 POINT(48.01910890521643 -122.62633949303961) bank128085 +128086 POINT(47.74092823864717 -122.31566781417973) bank128086 +128087 POINT(47.55729236331759 -122.26798265773583) bank128087 +128088 POINT(48.04577607051186 -122.05355056397077) bank128088 +128089 POINT(47.13330361765883 -122.66501788814699) bank128089 +128090 POINT(46.809497902994565 -121.37159157236779) bank128090 +128091 POINT(46.791721740383515 -121.79116436862944) bank128091 +128092 POINT(48.53556854129657 -121.97266766022932) bank128092 +128093 POINT(48.39391784689507 -122.0585574871294) bank128093 +128094 POINT(48.32497171574544 -121.90440324015465) bank128094 +128095 POINT(47.9192973649565 -121.62609963562666) bank128095 +128096 POINT(48.02828768354637 -123.15497121297727) bank128096 +128097 POINT(48.07081369502165 -122.43702674840877) bank128097 +128098 POINT(48.08798076778049 -121.57600253624433) bank128098 +128099 POINT(47.27022360694965 -122.6383913795459) bank128099 +128100 POINT(47.75099691116414 -122.57445416287368) bank128100 +128101 POINT(47.5329795302545 -123.11368004369163) bank128101 +128102 POINT(47.660486654736836 -121.66784824521862) bank128102 +128103 POINT(47.24044580494819 -122.3729330236515) bank128103 +128104 POINT(46.65832346446917 -122.48563156316804) bank128104 +128105 POINT(47.120570969957996 -122.37270521780795) bank128105 +128106 POINT(47.84699004114986 -122.35572561191687) bank128106 +128107 POINT(46.791176208769315 -122.72978613552095) bank128107 +128108 POINT(47.632731526370854 -122.77042376457699) bank128108 +128109 POINT(46.721272222724004 -122.80111710011101) bank128109 +128110 POINT(48.163408494320834 -121.55729859876965) bank128110 +128111 POINT(46.761985572703594 -123.30741524074428) bank128111 +128112 POINT(48.16988765527665 -123.1537937075425) bank128112 +128113 POINT(46.93738926916037 -122.95997892418366) bank128113 +128114 POINT(48.489392943586324 -122.81492406736025) bank128114 +128115 POINT(48.11008041716595 -121.36485860729586) bank128115 +128116 POINT(46.79525595198895 -121.36260046664101) bank128116 +128117 POINT(47.59550344930013 -122.8410517354065) bank128117 +128118 POINT(46.77139917713072 -121.40113190133704) bank128118 +128119 POINT(47.520727319757384 -123.06735776758454) bank128119 +128120 POINT(46.725954236508706 -121.94323620501042) bank128120 +128121 POINT(48.46978241288701 -122.82567323258816) bank128121 +128122 POINT(47.19543749712463 -121.42407155531829) bank128122 +128123 POINT(47.19074283433908 -121.98744254472975) bank128123 +128124 POINT(48.26712063336313 -123.17111209697629) bank128124 +128125 POINT(47.52612799700093 -121.73783984324957) bank128125 +128126 POINT(48.43882817137356 -123.23661079572904) bank128126 +128127 POINT(47.31517617751612 -123.04603042873438) bank128127 +128128 POINT(47.40658967340207 -121.44208613651762) bank128128 +128129 POINT(47.49199501479146 -123.31300490687696) bank128129 +128130 POINT(47.294491848906816 -121.5430327679768) bank128130 +128131 POINT(47.843435995935266 -122.27040369799714) bank128131 +128132 POINT(47.18973024549781 -122.0648344860863) bank128132 +128133 POINT(48.38778662942346 -123.13995359956014) bank128133 +128134 POINT(46.6135069839199 -122.57218442394114) bank128134 +128135 POINT(47.13955447572597 -121.86347186288157) bank128135 +128136 POINT(47.00373077048064 -121.37253154689417) bank128136 +128137 POINT(47.75128971110672 -122.27552454094626) bank128137 +128138 POINT(47.08069706322597 -123.19730618746692) bank128138 +128139 POINT(48.0381851534437 -122.47222873030638) bank128139 +128140 POINT(47.70733105710536 -122.09683501911698) bank128140 +128141 POINT(47.36666787144827 -121.3528975450939) bank128141 +128142 POINT(46.81158436045023 -123.24212445994638) bank128142 +128143 POINT(47.26577122455397 -123.26397095627715) bank128143 +128144 POINT(48.03574104309893 -122.6183463841714) bank128144 +128145 POINT(48.55939354228153 -121.5877853845065) bank128145 +128146 POINT(46.92854158728121 -122.444630439671) bank128146 +128147 POINT(48.40837404657555 -122.9086884413536) bank128147 +128148 POINT(47.7037530671845 -121.4437606058875) bank128148 +128149 POINT(47.695212931451685 -123.27610011719379) bank128149 +128150 POINT(46.74890817803448 -122.94056414476736) bank128150 +128151 POINT(48.40169899540888 -122.5067582970977) bank128151 +128152 POINT(47.85654224756172 -122.5528736933039) bank128152 +128153 POINT(46.95862458674862 -121.48704928365285) bank128153 +128154 POINT(46.96374817094221 -121.54754470862552) bank128154 +128155 POINT(46.91621435155357 -121.71593370311145) bank128155 +128156 POINT(48.03730667199253 -122.54724262341077) bank128156 +128157 POINT(47.86096653091786 -122.5010404688639) bank128157 +128158 POINT(48.083075691938234 -123.30589176125241) bank128158 +128159 POINT(47.538619717253574 -121.39981633881185) bank128159 +128160 POINT(46.65741285238256 -122.75563178032556) bank128160 +128161 POINT(47.22183206188493 -123.0569875579961) bank128161 +128162 POINT(47.18082284336193 -121.75382373441622) bank128162 +128163 POINT(47.44503818791378 -121.98139585674812) bank128163 +128164 POINT(47.54332965352045 -122.6868782616271) bank128164 +128165 POINT(47.33927767941646 -122.97089900318443) bank128165 +128166 POINT(47.0910743286212 -121.67671425861055) bank128166 +128167 POINT(48.385469546058495 -122.69515644515018) bank128167 +128168 POINT(47.94976165036912 -123.08397055872562) bank128168 +128169 POINT(47.35832782049513 -121.7995562764649) bank128169 +128170 POINT(47.11050259607346 -122.84027254406672) bank128170 +128171 POINT(48.07841668065723 -121.63674553644898) bank128171 +128172 POINT(47.044591964095346 -122.64283335464718) bank128172 +128173 POINT(46.852553481571846 -121.5632208858001) bank128173 +128174 POINT(46.801292922033255 -123.0347604611816) bank128174 +128175 POINT(47.566615000935116 -123.04457897430787) bank128175 +128176 POINT(47.353878627250744 -122.06674254643755) bank128176 +128177 POINT(48.53193787478961 -123.31801487628539) bank128177 +128178 POINT(48.50251652654395 -121.6648541552109) bank128178 +128179 POINT(47.02232046860822 -122.6199007586946) bank128179 +128180 POINT(46.743890477506056 -122.71962274925035) bank128180 +128181 POINT(47.00574003303111 -123.20923749325902) bank128181 +128182 POINT(47.37977229502941 -122.44518644009945) bank128182 +128183 POINT(47.51694662250869 -121.55907765079174) bank128183 +128184 POINT(48.47807914468716 -121.45694234751336) bank128184 +128185 POINT(47.93331020339144 -122.93224078025636) bank128185 +128186 POINT(46.92897081976405 -121.39988620860113) bank128186 +128187 POINT(46.91639002318785 -123.2338135500687) bank128187 +128188 POINT(48.574700315508025 -122.64842173346277) bank128188 +128189 POINT(47.13487369726252 -121.89215508978683) bank128189 +128190 POINT(46.93471533898213 -122.5432682104497) bank128190 +128191 POINT(46.709823563339825 -121.93302881810408) bank128191 +128192 POINT(46.91300037953395 -121.87096285652576) bank128192 +128193 POINT(46.64161571546315 -122.05055826438183) bank128193 +128194 POINT(47.95137748073479 -122.76798117510678) bank128194 +128195 POINT(48.47410000509587 -123.0238380495968) bank128195 +128196 POINT(48.40114705296851 -123.16184755230952) bank128196 +128197 POINT(48.293199891138215 -122.55472201694172) bank128197 +128198 POINT(47.40841023838859 -122.17296384914845) bank128198 +128199 POINT(46.662297311362636 -121.61968513371477) bank128199 +128200 POINT(47.79535069697808 -122.18029955596094) bank128200 +128201 POINT(46.62957134972909 -123.2279334890205) bank128201 +128202 POINT(47.682680149620666 -121.88664796305872) bank128202 +128203 POINT(46.85061161294445 -122.79440555877086) bank128203 +128204 POINT(47.85248699718763 -122.90187931283195) bank128204 +128205 POINT(47.40215451291398 -122.49983085876849) bank128205 +128206 POINT(47.101876709863376 -123.07407661638302) bank128206 +128207 POINT(47.48531023567516 -122.7296059802766) bank128207 +128208 POINT(47.155955420487984 -121.69512930217327) bank128208 +128209 POINT(47.73704733827662 -121.72755676990765) bank128209 +128210 POINT(46.928024655046116 -121.57233486223019) bank128210 +128211 POINT(47.081122798485055 -121.99833714212993) bank128211 +128212 POINT(48.3857478127756 -121.39436724372673) bank128212 +128213 POINT(48.520463542518506 -123.13986503994994) bank128213 +128214 POINT(48.30289423736475 -121.91455052099053) bank128214 +128215 POINT(47.8076197261409 -121.88761490440858) bank128215 +128216 POINT(48.292965356034784 -122.4148347266582) bank128216 +128217 POINT(46.773398492501094 -122.23021348036055) bank128217 +128218 POINT(48.403433839680325 -121.53322744087035) bank128218 +128219 POINT(48.26097840752362 -122.41285128893796) bank128219 +128220 POINT(48.32526652345471 -122.88098032400852) bank128220 +128221 POINT(47.11094593685549 -121.37493980155276) bank128221 +128222 POINT(47.403221984606354 -122.6145556422373) bank128222 +128223 POINT(46.68956945317502 -123.1523770784442) bank128223 +128224 POINT(48.34217075650711 -121.66076711001735) bank128224 +128225 POINT(47.061502315146605 -123.3205898383041) bank128225 +128226 POINT(46.887260907595035 -122.10241086999561) bank128226 +128227 POINT(47.31099918467018 -122.981539169441) bank128227 +128228 POINT(46.92450608795097 -121.4711769665969) bank128228 +128229 POINT(46.85786873115407 -122.28239096022818) bank128229 +128230 POINT(47.811477705528596 -121.75355195293733) bank128230 +128231 POINT(47.54860797701351 -122.38921557196964) bank128231 +128232 POINT(48.538741018615866 -121.67137284027709) bank128232 +128233 POINT(46.81778253564026 -122.98459092617004) bank128233 +128234 POINT(47.735006308853926 -121.92276467537079) bank128234 +128235 POINT(48.05439041333344 -122.98345449215738) bank128235 +128236 POINT(47.613887064613465 -122.19547474272899) bank128236 +128237 POINT(48.60345699961412 -122.97663537287137) bank128237 +128238 POINT(47.30511966441192 -121.98737744633911) bank128238 +128239 POINT(46.82383922613793 -121.97670591366136) bank128239 +128240 POINT(47.67975155688161 -122.69885051391284) bank128240 +128241 POINT(48.00300479776047 -121.92648984511126) bank128241 +128242 POINT(48.25064778583739 -122.33770383680533) bank128242 +128243 POINT(47.45144156970378 -122.48126942928812) bank128243 +128244 POINT(48.257829454578065 -122.9038059884427) bank128244 +128245 POINT(48.18264305640499 -122.0139976977671) bank128245 +128246 POINT(47.67464418110658 -122.47935473443478) bank128246 +128247 POINT(47.70772058864118 -121.71207203597292) bank128247 +128248 POINT(46.759228188511386 -121.72655944220126) bank128248 +128249 POINT(46.625567728149555 -121.8949521412067) bank128249 +128250 POINT(48.285520988933015 -121.61920604212607) bank128250 +128251 POINT(48.11643323366648 -122.99207452151954) bank128251 +128252 POINT(48.243640969680214 -121.48974732487146) bank128252 +128253 POINT(46.98032492905631 -123.13267277109577) bank128253 +128254 POINT(47.780710486018904 -122.4580813273879) bank128254 +128255 POINT(47.68476050882202 -122.62120370828673) bank128255 +128256 POINT(48.58508686469095 -121.79339349403875) bank128256 +128257 POINT(48.57634533272586 -123.29640495516314) bank128257 +128258 POINT(48.440171858240305 -123.09016052833354) bank128258 +128259 POINT(47.89953107394797 -123.05259865693452) bank128259 +128260 POINT(48.02083673487925 -123.05399916356933) bank128260 +128261 POINT(48.53691160440085 -122.34245850688956) bank128261 +128262 POINT(48.398226535143024 -122.35241226488809) bank128262 +128263 POINT(48.5797286726107 -122.56996706144436) bank128263 +128264 POINT(48.08070770066347 -122.78669132675047) bank128264 +128265 POINT(47.46379277418233 -122.97910066867522) bank128265 +128266 POINT(46.60644235220426 -122.38303028384176) bank128266 +128267 POINT(48.400292963902054 -123.12057465931312) bank128267 +128268 POINT(47.25529165679727 -121.8022668252387) bank128268 +128269 POINT(46.78292061772884 -122.39764322714328) bank128269 +128270 POINT(47.86172475857619 -121.7885295896552) bank128270 +128271 POINT(47.53289463435544 -122.45777935534899) bank128271 +128272 POINT(46.872182704171976 -122.60551306786319) bank128272 +128273 POINT(48.460923652248695 -121.76345403657487) bank128273 +128274 POINT(47.43458090252248 -122.57543624739478) bank128274 +128275 POINT(48.20593289754244 -122.7001222058304) bank128275 +128276 POINT(47.734806391526675 -121.69911366617514) bank128276 +128277 POINT(48.60260855576161 -121.9276948222185) bank128277 +128278 POINT(47.898508619725945 -122.27273770064075) bank128278 +128279 POINT(46.925563986854364 -121.48106042700482) bank128279 +128280 POINT(46.829277987806904 -121.33493974805658) bank128280 +128281 POINT(48.582978166420055 -122.93899096161174) bank128281 +128282 POINT(46.606410286026794 -122.10428182700774) bank128282 +128283 POINT(47.249982800404176 -122.49977852979191) bank128283 +128284 POINT(46.61955279340978 -122.35189121359183) bank128284 +128285 POINT(48.34725304900768 -121.73647259911102) bank128285 +128286 POINT(48.52411549603722 -121.73106590959632) bank128286 +128287 POINT(47.24880323188326 -121.35704923052732) bank128287 +128288 POINT(46.634125455389956 -121.92436396776355) bank128288 +128289 POINT(46.80132566679249 -122.66577197556185) bank128289 +128290 POINT(46.70625388605717 -123.13304657711373) bank128290 +128291 POINT(46.66430293704154 -121.41670300342037) bank128291 +128292 POINT(47.52979497976022 -121.60179425255406) bank128292 +128293 POINT(47.62558112970225 -122.99264206458962) bank128293 +128294 POINT(46.86342933689542 -122.89277252600318) bank128294 +128295 POINT(48.00873966483416 -121.96948081130122) bank128295 +128296 POINT(48.3136550443583 -121.94798896033109) bank128296 +128297 POINT(46.66499618697098 -122.24932094418098) bank128297 +128298 POINT(48.32095495333985 -121.58895277331364) bank128298 +128299 POINT(48.07852744651596 -121.989572684681) bank128299 +128300 POINT(48.54877522453802 -121.95184233909004) bank128300 +128301 POINT(48.01044582381199 -121.87593390727743) bank128301 +128302 POINT(47.50446383701078 -122.41714050956232) bank128302 +128303 POINT(47.94174409609588 -122.02896193012391) bank128303 +128304 POINT(47.139932718491515 -123.14164594054377) bank128304 +128305 POINT(47.15584508977624 -122.68603355000323) bank128305 +128306 POINT(46.955182219237145 -122.84181016795853) bank128306 +128307 POINT(47.192372703124775 -121.70748815729404) bank128307 +128308 POINT(48.04541627801569 -121.8206020466616) bank128308 +128309 POINT(46.72644779346743 -121.53355536742274) bank128309 +128310 POINT(46.90458019793206 -122.63825842535037) bank128310 +128311 POINT(47.78964804893978 -123.30688352820766) bank128311 +128312 POINT(46.7720906643388 -121.87238577734578) bank128312 +128313 POINT(46.94013647284963 -121.5900510055408) bank128313 +128314 POINT(48.021794600813166 -122.25807068355337) bank128314 +128315 POINT(48.453944453275014 -123.28581674811707) bank128315 +128316 POINT(48.49541273001848 -123.04955936291013) bank128316 +128317 POINT(47.42123838276072 -122.56358668652781) bank128317 +128318 POINT(47.898114277062426 -121.51584705404281) bank128318 +128319 POINT(46.89634767783913 -121.46445136247145) bank128319 +128320 POINT(47.2198078995341 -121.54078636126097) bank128320 +128321 POINT(46.679609769681306 -122.86161573907263) bank128321 +128322 POINT(48.215513306113564 -121.43036817041326) bank128322 +128323 POINT(48.25495541564083 -121.94188647171467) bank128323 +128324 POINT(47.3816679823139 -122.78259968623723) bank128324 +128325 POINT(47.530366913653175 -123.01810018273775) bank128325 +128326 POINT(48.031125346767546 -122.70699890185846) bank128326 +128327 POINT(46.62925220352883 -121.39100134827714) bank128327 +128328 POINT(46.85896207647035 -122.90944581387656) bank128328 +128329 POINT(47.77671466154162 -121.64017490101821) bank128329 +128330 POINT(48.209262737517484 -121.35355184704389) bank128330 +128331 POINT(46.92184839575713 -121.89382304786862) bank128331 +128332 POINT(46.898895267762484 -121.65014205833506) bank128332 +128333 POINT(48.577600277565494 -122.37799962198183) bank128333 +128334 POINT(47.2547835092621 -122.0432597128241) bank128334 +128335 POINT(46.84734833668221 -121.58765667165592) bank128335 +128336 POINT(48.24111587960287 -121.7740813935945) bank128336 +128337 POINT(46.87098664667997 -122.41387811864308) bank128337 +128338 POINT(47.421990960906214 -121.78365910759808) bank128338 +128339 POINT(48.49528460502791 -122.46588415135167) bank128339 +128340 POINT(48.11235328364388 -122.24552135316983) bank128340 +128341 POINT(46.73220352197253 -122.65781679828655) bank128341 +128342 POINT(47.439237790266084 -122.37264770791937) bank128342 +128343 POINT(46.88438834492865 -121.78305928492453) bank128343 +128344 POINT(47.79146646336235 -123.25768911898075) bank128344 +128345 POINT(48.05307738854787 -122.69881784487256) bank128345 +128346 POINT(47.27421423383193 -122.32594837019634) bank128346 +128347 POINT(47.82201233228029 -123.10505500849479) bank128347 +128348 POINT(48.25820203920648 -121.38656670205978) bank128348 +128349 POINT(48.00868579953173 -121.49068645363874) bank128349 +128350 POINT(47.44018367124674 -121.57951710597348) bank128350 +128351 POINT(48.260730564062726 -123.00373597400393) bank128351 +128352 POINT(47.22413658005855 -122.20029944444404) bank128352 +128353 POINT(47.99696001923293 -121.53676660994293) bank128353 +128354 POINT(48.589388038991686 -122.59469084617761) bank128354 +128355 POINT(47.59605910604435 -122.91390384175504) bank128355 +128356 POINT(47.299727152796606 -121.43970307026088) bank128356 +128357 POINT(48.33711913234689 -122.28576267337586) bank128357 +128358 POINT(48.06621529641191 -122.62572346116411) bank128358 +128359 POINT(48.585060781970974 -121.47182578021304) bank128359 +128360 POINT(46.708791614403346 -122.06602838710579) bank128360 +128361 POINT(47.05021374980669 -122.20574226300563) bank128361 +128362 POINT(47.4737062736883 -121.80604377381617) bank128362 +128363 POINT(46.64403358694187 -122.29358061035589) bank128363 +128364 POINT(47.259382024088474 -122.61801529436693) bank128364 +128365 POINT(48.193842015477834 -122.36047840393552) bank128365 +128366 POINT(46.7927597860296 -121.35096692997942) bank128366 +128367 POINT(47.70321712999773 -121.65571405421998) bank128367 +128368 POINT(46.631460342919546 -121.91161264432382) bank128368 +128369 POINT(47.43081790795653 -122.07113162357673) bank128369 +128370 POINT(48.552308293035416 -122.21892109403808) bank128370 +128371 POINT(48.0942851498567 -121.46362373312144) bank128371 +128372 POINT(48.55660383165298 -121.82584162810785) bank128372 +128373 POINT(46.90551556791842 -122.92201132352987) bank128373 +128374 POINT(47.7235718618026 -122.65873923420632) bank128374 +128375 POINT(47.2159074223969 -122.7628397315628) bank128375 +128376 POINT(48.556560747869376 -122.927261708987) bank128376 +128377 POINT(46.72369313726876 -121.48227023312734) bank128377 +128378 POINT(48.343091998992215 -122.35593495117816) bank128378 +128379 POINT(46.93122505826789 -122.44751126549421) bank128379 +128380 POINT(48.08384520841998 -122.71051589363712) bank128380 +128381 POINT(48.32196044977047 -123.26688987027713) bank128381 +128382 POINT(47.81022196368686 -122.63554080018523) bank128382 +128383 POINT(48.47969415450854 -123.08701509068376) bank128383 +128384 POINT(47.04049648866147 -121.51162710101013) bank128384 +128385 POINT(48.3317492020806 -121.61207364239175) bank128385 +128386 POINT(48.46165560091558 -122.71573317443736) bank128386 +128387 POINT(47.983097351387634 -122.57072952601966) bank128387 +128388 POINT(48.586040940417256 -122.45800986325916) bank128388 +128389 POINT(48.43175409783643 -121.88722810632989) bank128389 +128390 POINT(46.89310205778375 -121.88614040546243) bank128390 +128391 POINT(48.34264259245903 -122.91327377563636) bank128391 +128392 POINT(46.757559516133405 -122.04373145220059) bank128392 +128393 POINT(46.7814062079156 -121.59379158516742) bank128393 +128394 POINT(48.56423090688214 -122.21397147407708) bank128394 +128395 POINT(46.90025397298014 -123.31291337940566) bank128395 +128396 POINT(46.83111627665944 -122.11332448853463) bank128396 +128397 POINT(47.82204970536007 -121.64495020490286) bank128397 +128398 POINT(47.585653420116685 -121.62992545762528) bank128398 +128399 POINT(47.175484863841824 -122.6030744726521) bank128399 +128400 POINT(48.20264653244883 -122.5602914561862) bank128400 +128401 POINT(47.61495961234938 -121.78080432291775) bank128401 +128402 POINT(46.84347084869957 -121.39168208544999) bank128402 +128403 POINT(48.38792502758043 -122.77252403220552) bank128403 +128404 POINT(47.43464475259559 -122.08142273946811) bank128404 +128405 POINT(46.79582346905081 -122.6200339787534) bank128405 +128406 POINT(46.97845871527791 -122.50140090108046) bank128406 +128407 POINT(47.603928012914785 -122.04011733058998) bank128407 +128408 POINT(47.17996645696312 -122.09057853667363) bank128408 +128409 POINT(47.236373464803535 -122.07496435890256) bank128409 +128410 POINT(47.52698419152976 -121.41961151907601) bank128410 +128411 POINT(47.559207482694326 -122.12640767120833) bank128411 +128412 POINT(47.23891135886891 -122.78070433987222) bank128412 +128413 POINT(48.470967958522884 -123.09056644356751) bank128413 +128414 POINT(48.21257848349876 -122.38247523866754) bank128414 +128415 POINT(47.294345446184444 -122.7892561621315) bank128415 +128416 POINT(47.28984341835789 -121.41993884174224) bank128416 +128417 POINT(47.10530851180258 -122.94865096262045) bank128417 +128418 POINT(47.13678293792606 -123.02662383689697) bank128418 +128419 POINT(46.69332434349154 -122.57540310339225) bank128419 +128420 POINT(47.70499197113626 -122.19445344141424) bank128420 +128421 POINT(46.91131121104876 -122.90801451424129) bank128421 +128422 POINT(47.51960149788354 -121.97346434388847) bank128422 +128423 POINT(48.01201764429953 -122.51852530510195) bank128423 +128424 POINT(47.75653913907357 -121.95006825632134) bank128424 +128425 POINT(47.00953484620048 -122.56248487262326) bank128425 +128426 POINT(48.02521445333504 -122.08334250615513) bank128426 +128427 POINT(48.42180633361157 -123.32179289146053) bank128427 +128428 POINT(48.25240653710945 -121.90697690145424) bank128428 +128429 POINT(48.433518774209595 -122.71947332507796) bank128429 +128430 POINT(46.69533122063603 -122.4697205394668) bank128430 +128431 POINT(48.401208285237 -121.79808277957879) bank128431 +128432 POINT(48.572095080225345 -122.14041750855532) bank128432 +128433 POINT(47.789349121572634 -122.06077398256845) bank128433 +128434 POINT(48.55269652816394 -122.33692233866896) bank128434 +128435 POINT(48.526865197737564 -123.04981968647735) bank128435 +128436 POINT(47.447769325837605 -121.40870197081195) bank128436 +128437 POINT(48.33611150203936 -121.80685282822226) bank128437 +128438 POINT(46.86842105333098 -122.05041386685008) bank128438 +128439 POINT(47.55547270075027 -122.89674130864202) bank128439 +128440 POINT(48.475394174964904 -121.51217364757811) bank128440 +128441 POINT(47.36150107663417 -122.4292180695342) bank128441 +128442 POINT(48.1238678877889 -121.40663082867326) bank128442 +128443 POINT(47.52939098469843 -122.32800641791879) bank128443 +128444 POINT(47.25702552394512 -122.14274382058015) bank128444 +128445 POINT(46.863059203633796 -122.39851256466851) bank128445 +128446 POINT(48.27796202741251 -121.81307597733665) bank128446 +128447 POINT(47.602766612101 -122.73812765251664) bank128447 +128448 POINT(47.24335540329165 -121.9807352229433) bank128448 +128449 POINT(47.56849292650862 -121.59072342917327) bank128449 +128450 POINT(47.11622327180282 -122.8599899746287) bank128450 +128451 POINT(47.56955704838649 -121.35252892096693) bank128451 +128452 POINT(46.804814988064514 -121.70248333466421) bank128452 +128453 POINT(47.666827607921064 -121.75347524350933) bank128453 +128454 POINT(47.424161423900806 -121.93437341309087) bank128454 +128455 POINT(47.85115761866291 -122.20698250573612) bank128455 +128456 POINT(48.340886092609885 -121.61057086758694) bank128456 +128457 POINT(46.701801068203835 -121.87133528711276) bank128457 +128458 POINT(47.75499087528698 -121.33963800901456) bank128458 +128459 POINT(47.393178505409864 -122.78302455745265) bank128459 +128460 POINT(47.40785524057143 -123.0738475447187) bank128460 +128461 POINT(47.22869154282654 -122.62773934116939) bank128461 +128462 POINT(48.30410812308794 -122.06681579346333) bank128462 +128463 POINT(47.495669560245275 -121.44391391419741) bank128463 +128464 POINT(48.17247062751189 -123.1458714560864) bank128464 +128465 POINT(46.823748982983666 -123.11040249130889) bank128465 +128466 POINT(46.80341719503918 -121.66811046981218) bank128466 +128467 POINT(46.62198223543715 -123.24223295593785) bank128467 +128468 POINT(47.52038239660557 -122.1574596578831) bank128468 +128469 POINT(47.5764037514091 -122.5177185255049) bank128469 +128470 POINT(47.793697816980185 -123.2229682024149) bank128470 +128471 POINT(47.442321797682666 -121.59766250273721) bank128471 +128472 POINT(47.91346755735458 -123.24709468194064) bank128472 +128473 POINT(47.30359953550968 -121.3996795516531) bank128473 +128474 POINT(48.11233538182441 -122.48015671057001) bank128474 +128475 POINT(47.14026454727383 -121.96143388626521) bank128475 +128476 POINT(47.29667638354329 -122.17708403886866) bank128476 +128477 POINT(48.08966199465624 -121.45376982698426) bank128477 +128478 POINT(47.47071069347359 -121.863888528756) bank128478 +128479 POINT(48.35611481509422 -123.12466166407648) bank128479 +128480 POINT(48.37708824684838 -122.38094044756153) bank128480 +128481 POINT(47.57321708482139 -121.79731015283183) bank128481 +128482 POINT(47.31220116020574 -121.53669454294034) bank128482 +128483 POINT(48.17311573062158 -123.20326541988439) bank128483 +128484 POINT(47.249105114094654 -123.2637525329971) bank128484 +128485 POINT(47.788843221962324 -122.96184189627508) bank128485 +128486 POINT(47.79583759243517 -122.20235605250487) bank128486 +128487 POINT(47.09920710612569 -122.96167382504679) bank128487 +128488 POINT(48.079773787615274 -121.91500277683873) bank128488 +128489 POINT(48.4563375517533 -123.11069330272866) bank128489 +128490 POINT(48.15749604387928 -123.22003910538211) bank128490 +128491 POINT(48.51034997596923 -123.22866924259841) bank128491 +128492 POINT(46.75201555131995 -122.04598237123447) bank128492 +128493 POINT(48.30568486406903 -122.79376675579674) bank128493 +128494 POINT(46.63642051933699 -121.62564695537208) bank128494 +128495 POINT(46.92090539567988 -121.83891902643819) bank128495 +128496 POINT(46.814948099442695 -122.54296947995975) bank128496 +128497 POINT(48.484068068756294 -121.59212129467487) bank128497 +128498 POINT(47.25769132952745 -122.68771723109954) bank128498 +128499 POINT(48.346913798069814 -122.62190922058) bank128499 +128500 POINT(47.52372496623107 -123.15478704880842) bank128500 +128501 POINT(48.49602086803644 -121.74458881751238) bank128501 +128502 POINT(48.19570181890431 -122.9317380692563) bank128502 +128503 POINT(48.536799926186305 -123.1368650411889) bank128503 +128504 POINT(47.65785300906947 -121.35619668867933) bank128504 +128505 POINT(48.42259944604871 -121.4935545178517) bank128505 +128506 POINT(46.75870856997072 -123.29019260657292) bank128506 +128507 POINT(47.17140247741 -123.20119961825678) bank128507 +128508 POINT(46.63682462619721 -122.40217894186378) bank128508 +128509 POINT(46.75075233822794 -122.72596936907593) bank128509 +128510 POINT(48.56853775196531 -121.36226112240045) bank128510 +128511 POINT(47.07053870020184 -122.50991978361675) bank128511 +128512 POINT(47.91448341422191 -122.58894441645081) bank128512 +128513 POINT(47.11334863077311 -123.23949058038474) bank128513 +128514 POINT(47.694139898157914 -122.17999478232397) bank128514 +128515 POINT(48.313697180114225 -123.12226994530953) bank128515 +128516 POINT(47.37761536722018 -122.58759312496593) bank128516 +128517 POINT(47.432763089267866 -122.95936158070806) bank128517 +128518 POINT(48.532227320432234 -121.74129166475257) bank128518 +128519 POINT(48.21701354821148 -122.8247543741594) bank128519 +128520 POINT(47.61252998295538 -122.73256049981794) bank128520 +128521 POINT(48.11863133048588 -122.38257591535789) bank128521 +128522 POINT(47.4564189406354 -122.84867838067771) bank128522 +128523 POINT(47.887400450220504 -122.40135046747862) bank128523 +128524 POINT(46.78934176688301 -122.54829288261975) bank128524 +128525 POINT(46.95153903953988 -122.16360999935809) bank128525 +128526 POINT(47.548145733742565 -122.33611948410275) bank128526 +128527 POINT(47.70575663818312 -121.36159753240753) bank128527 +128528 POINT(47.14943229553011 -121.88039511924872) bank128528 +128529 POINT(48.30255579100276 -121.77078335675053) bank128529 +128530 POINT(47.992755859030275 -122.29117324709792) bank128530 +128531 POINT(47.46517605831811 -122.72198666435784) bank128531 +128532 POINT(46.621316952092194 -121.57170903507176) bank128532 +128533 POINT(47.8585446126298 -122.42274858549726) bank128533 +128534 POINT(47.22963669109973 -122.24347847773826) bank128534 +128535 POINT(48.352137172361054 -122.38597746493257) bank128535 +128536 POINT(47.44030595477805 -122.91507814268921) bank128536 +128537 POINT(48.07987143656495 -121.81932314420081) bank128537 +128538 POINT(46.97128205095213 -122.53610100909293) bank128538 +128539 POINT(47.250559685736036 -121.58385862670319) bank128539 +128540 POINT(48.204838750224276 -121.98286014888497) bank128540 +128541 POINT(47.610646678096224 -121.83990931655933) bank128541 +128542 POINT(47.99615534356788 -123.1378226722103) bank128542 +128543 POINT(47.89165849696344 -123.0448682953413) bank128543 +128544 POINT(47.794698188581926 -122.59295691289323) bank128544 +128545 POINT(47.59901403735222 -122.89340239050313) bank128545 +128546 POINT(46.76954931866761 -122.19235286559207) bank128546 +128547 POINT(47.041141156072655 -122.9686496748773) bank128547 +128548 POINT(47.338751598905986 -122.85164858051887) bank128548 +128549 POINT(47.219717134357865 -122.52618110750385) bank128549 +128550 POINT(46.900243170105966 -123.03334960764646) bank128550 +128551 POINT(46.816140251693696 -122.84275253322866) bank128551 +128552 POINT(48.00270189562796 -121.60777625736354) bank128552 +128553 POINT(48.128993056118624 -121.86776678084185) bank128553 +128554 POINT(47.89029055145081 -123.2813915346092) bank128554 +128555 POINT(47.357473321024955 -122.34791625743934) bank128555 +128556 POINT(46.76974113475758 -122.54363365185888) bank128556 +128557 POINT(48.27398128169096 -122.76120769185628) bank128557 +128558 POINT(47.147837261379046 -122.10794661634169) bank128558 +128559 POINT(48.23029101109734 -122.62681801610586) bank128559 +128560 POINT(48.512890267491926 -122.40288989578295) bank128560 +128561 POINT(48.32587085791784 -122.63036620600592) bank128561 +128562 POINT(47.16322099697781 -121.75004229970757) bank128562 +128563 POINT(48.37212450863166 -123.18261396512233) bank128563 +128564 POINT(47.77148826752568 -122.3260384075495) bank128564 +128565 POINT(47.20976438417541 -122.49586608525127) bank128565 +128566 POINT(47.55008668100604 -122.84991232392609) bank128566 +128567 POINT(47.31013188644008 -122.42205135100448) bank128567 +128568 POINT(48.48754001102736 -122.42881677714567) bank128568 +128569 POINT(47.10008147165708 -122.08804979175093) bank128569 +128570 POINT(47.128952671425125 -123.28751083252325) bank128570 +128571 POINT(46.80087992597139 -122.01640371465115) bank128571 +128572 POINT(47.967148094725474 -121.38103532478927) bank128572 +128573 POINT(48.286590459912105 -122.3091652052382) bank128573 +128574 POINT(48.40579480561089 -122.35485398288337) bank128574 +128575 POINT(46.663948142545884 -121.92398938944328) bank128575 +128576 POINT(46.77180407291009 -123.17539796573546) bank128576 +128577 POINT(48.11225909471638 -123.21324022442015) bank128577 +128578 POINT(48.57387970114341 -121.33293970291629) bank128578 +128579 POINT(46.899085953322384 -122.51027710611629) bank128579 +128580 POINT(46.69436902713334 -123.32874807609439) bank128580 +128581 POINT(48.10212383579629 -121.58389768694171) bank128581 +128582 POINT(47.06645389002518 -123.29774308919599) bank128582 +128583 POINT(47.74953856680323 -121.41076410479249) bank128583 +128584 POINT(46.920377126515376 -122.3300245065043) bank128584 +128585 POINT(47.46700079673666 -122.0262879172715) bank128585 +128586 POINT(47.97486314060646 -121.56468642862367) bank128586 +128587 POINT(47.43157208557131 -121.35283462307218) bank128587 +128588 POINT(46.78874956802748 -122.57189976315371) bank128588 +128589 POINT(47.52801281832506 -122.01419878368874) bank128589 +128590 POINT(48.46294557352367 -121.74525744490434) bank128590 +128591 POINT(47.58910443883495 -123.07829526549028) bank128591 +128592 POINT(47.9082318842671 -122.36483754329053) bank128592 +128593 POINT(48.46225459312374 -121.72904750623083) bank128593 +128594 POINT(46.86228374212587 -122.47527642672131) bank128594 +128595 POINT(47.59695116263485 -121.48158540973712) bank128595 +128596 POINT(47.2640346270026 -122.75213654542097) bank128596 +128597 POINT(47.714649636642015 -123.12656629332457) bank128597 +128598 POINT(47.25284263232693 -121.78612774470324) bank128598 +128599 POINT(46.995041771162235 -122.59741579981045) bank128599 +128600 POINT(46.83852971259985 -122.83448870329475) bank128600 +128601 POINT(48.02578589478778 -123.19997978174216) bank128601 +128602 POINT(48.04840990594406 -123.08912323248187) bank128602 +128603 POINT(48.54072496303773 -122.8240216501121) bank128603 +128604 POINT(46.88950613933901 -123.05793735566021) bank128604 +128605 POINT(48.557838291981234 -122.65669552550062) bank128605 +128606 POINT(47.84521767951027 -123.28407082821612) bank128606 +128607 POINT(46.86887432693588 -122.97829789546309) bank128607 +128608 POINT(46.97820952489997 -123.19288995457609) bank128608 +128609 POINT(47.66858308086439 -122.06699569329878) bank128609 +128610 POINT(47.10182689937403 -122.86176122249391) bank128610 +128611 POINT(46.625693201791 -122.05807059698401) bank128611 +128612 POINT(47.2163375217431 -123.21593352043563) bank128612 +128613 POINT(46.95947675515384 -122.91181238717532) bank128613 +128614 POINT(48.023292247874295 -121.94962693516787) bank128614 +128615 POINT(48.38503196925194 -122.18626098372039) bank128615 +128616 POINT(48.53174793447072 -123.2196462965244) bank128616 +128617 POINT(47.04179025574519 -123.10180853411512) bank128617 +128618 POINT(47.42604371085529 -122.14907225613312) bank128618 +128619 POINT(46.91310293867703 -122.74608850720224) bank128619 +128620 POINT(47.20012852444917 -122.02714858204979) bank128620 +128621 POINT(48.310595024994534 -121.88771150770745) bank128621 +128622 POINT(47.0960163378367 -123.02923730016815) bank128622 +128623 POINT(48.16951781461639 -123.2941775813795) bank128623 +128624 POINT(48.51584842952099 -122.19808821774458) bank128624 +128625 POINT(48.153467603533734 -122.31234571701572) bank128625 +128626 POINT(47.93985591014285 -122.90365338563886) bank128626 +128627 POINT(48.433246575054866 -121.92667691682723) bank128627 +128628 POINT(46.65334519114011 -122.28696343776477) bank128628 +128629 POINT(48.386357091846 -121.69209976462476) bank128629 +128630 POINT(47.30219874395972 -121.66470010551208) bank128630 +128631 POINT(48.28839328264515 -121.89188676599603) bank128631 +128632 POINT(48.30581333320083 -122.80193130458028) bank128632 +128633 POINT(46.70861974859763 -121.73657880707879) bank128633 +128634 POINT(47.290399307820735 -122.92029846329552) bank128634 +128635 POINT(47.349271812189315 -123.26924851881604) bank128635 +128636 POINT(46.67753546038709 -121.40485617355436) bank128636 +128637 POINT(47.724424274872625 -122.08433087876875) bank128637 +128638 POINT(46.773978721101116 -123.10398815367701) bank128638 +128639 POINT(48.43023255679267 -121.40640488357168) bank128639 +128640 POINT(47.578938557468206 -122.46536399173297) bank128640 +128641 POINT(47.40325170193801 -122.93932583274467) bank128641 +128642 POINT(47.04366240655194 -121.4306807585148) bank128642 +128643 POINT(47.93109400184559 -122.58427333083439) bank128643 +128644 POINT(47.83381622552865 -123.11236388036505) bank128644 +128645 POINT(47.51686080534136 -121.57702378608765) bank128645 +128646 POINT(48.02749043018539 -122.56148352119008) bank128646 +128647 POINT(46.92943712989993 -121.63183316092979) bank128647 +128648 POINT(47.14669900467519 -122.96978731724948) bank128648 +128649 POINT(47.76422452320921 -121.81711966236743) bank128649 +128650 POINT(48.58176321552993 -122.50399433738562) bank128650 +128651 POINT(48.18640427832431 -121.36077797073229) bank128651 +128652 POINT(48.170300650150764 -122.60062250188047) bank128652 +128653 POINT(47.14797289889549 -122.66952105191419) bank128653 +128654 POINT(46.71462325219907 -122.4471066543159) bank128654 +128655 POINT(47.892209988709226 -123.14625192039873) bank128655 +128656 POINT(48.25313316160211 -122.05805456919592) bank128656 +128657 POINT(46.67939922508849 -122.99683142497375) bank128657 +128658 POINT(46.76274173859334 -122.03298032537032) bank128658 +128659 POINT(47.48472691974168 -123.20414509876102) bank128659 +128660 POINT(46.7425248588969 -122.67295615729137) bank128660 +128661 POINT(47.187761093876595 -122.23471560161451) bank128661 +128662 POINT(48.42199340557673 -121.7992036475871) bank128662 +128663 POINT(47.80967971405979 -121.6225165376897) bank128663 +128664 POINT(47.0881120165692 -121.75951003084106) bank128664 +128665 POINT(47.19195149833561 -122.7334543383964) bank128665 +128666 POINT(47.34639988209223 -122.14332376611225) bank128666 +128667 POINT(46.71917507986228 -122.3464517786222) bank128667 +128668 POINT(48.1917771470042 -122.06609549843895) bank128668 +128669 POINT(46.95359387338893 -121.6307311730719) bank128669 +128670 POINT(46.90133064493056 -122.55181318865303) bank128670 +128671 POINT(47.33997750384886 -122.66512663876823) bank128671 +128672 POINT(48.17298410449137 -122.41190873429767) bank128672 +128673 POINT(46.65781184156589 -121.50581395312098) bank128673 +128674 POINT(48.14900215270562 -123.18046099202083) bank128674 +128675 POINT(48.245331164223394 -123.24525632583692) bank128675 +128676 POINT(47.44424985524931 -123.10084992340644) bank128676 +128677 POINT(47.35149488017223 -122.30468126990989) bank128677 +128678 POINT(48.14714110411381 -121.83775566530116) bank128678 +128679 POINT(47.17393461655418 -122.65886746200995) bank128679 +128680 POINT(47.010066011299415 -121.71738123865816) bank128680 +128681 POINT(46.615277827316 -123.21063181369276) bank128681 +128682 POINT(47.83686016682993 -123.03753697681184) bank128682 +128683 POINT(47.44903392301435 -122.15573732462767) bank128683 +128684 POINT(47.66041431363157 -121.5642538015861) bank128684 +128685 POINT(48.38774851307281 -121.38918980314403) bank128685 +128686 POINT(48.39053241306571 -121.370836278115) bank128686 +128687 POINT(46.72779010741294 -122.92449212270797) bank128687 +128688 POINT(46.98368962191738 -122.61558398987911) bank128688 +128689 POINT(47.4761724498811 -122.75163057324572) bank128689 +128690 POINT(47.77984507992769 -121.57434057643349) bank128690 +128691 POINT(47.51432240743911 -122.8245221142661) bank128691 +128692 POINT(48.17564249075508 -123.24777958651391) bank128692 +128693 POINT(47.22601309042569 -122.3159343364386) bank128693 +128694 POINT(47.16776017194357 -121.7196041789547) bank128694 +128695 POINT(47.73662319887945 -123.24239042887335) bank128695 +128696 POINT(48.586863387573345 -121.76985935372669) bank128696 +128697 POINT(47.477020199560855 -123.04802339541213) bank128697 +128698 POINT(47.23136448824306 -121.62723535250797) bank128698 +128699 POINT(46.90294095157184 -121.66061287034965) bank128699 +128700 POINT(48.39733274061319 -122.94298678016159) bank128700 +128701 POINT(47.11869109218744 -123.2471681109563) bank128701 +128702 POINT(46.67265152289784 -121.85535557154097) bank128702 +128703 POINT(47.254442584628606 -122.13455486546498) bank128703 +128704 POINT(47.6634579137048 -121.67918696283171) bank128704 +128705 POINT(47.952977558117254 -122.17613622884856) bank128705 +128706 POINT(47.684332685623 -122.51847344109673) bank128706 +128707 POINT(47.19392359214537 -121.46436573171947) bank128707 +128708 POINT(47.6844281910538 -122.2722490505097) bank128708 +128709 POINT(47.02745693329692 -121.76782313481193) bank128709 +128710 POINT(47.12413900332936 -121.36227778117949) bank128710 +128711 POINT(46.93058238257066 -121.62412872448606) bank128711 +128712 POINT(47.951066125431026 -121.33242322779505) bank128712 +128713 POINT(46.621628135889345 -122.55838641748296) bank128713 +128714 POINT(46.69534552495281 -123.32023996097891) bank128714 +128715 POINT(48.15927744422367 -121.41842445149813) bank128715 +128716 POINT(48.34664469688598 -122.83474723295969) bank128716 +128717 POINT(47.83633958274041 -122.30685899727877) bank128717 +128718 POINT(48.22036467848721 -122.50791701480135) bank128718 +128719 POINT(48.234599211120866 -123.32884922908661) bank128719 +128720 POINT(47.754804750622135 -121.95329033573721) bank128720 +128721 POINT(46.76078765999655 -121.82350304507163) bank128721 +128722 POINT(47.35415280789827 -122.31318690106048) bank128722 +128723 POINT(47.061262986924774 -123.10277418947705) bank128723 +128724 POINT(47.58036747993431 -121.39522474128314) bank128724 +128725 POINT(46.9375730369007 -122.59716741466872) bank128725 +128726 POINT(46.66765692143879 -121.36043289060211) bank128726 +128727 POINT(48.22520784542013 -122.9470053470304) bank128727 +128728 POINT(47.24529853092567 -123.30048232818898) bank128728 +128729 POINT(47.0201365980771 -121.43314112027066) bank128729 +128730 POINT(48.49419138199223 -122.95587735322779) bank128730 +128731 POINT(47.37752221293739 -121.83745443434805) bank128731 +128732 POINT(47.99430486786425 -122.1061247323502) bank128732 +128733 POINT(47.561010268568864 -121.5110327143406) bank128733 +128734 POINT(47.70819248051985 -123.06927417682817) bank128734 +128735 POINT(48.421543273734265 -122.4432954089644) bank128735 +128736 POINT(48.30976777184469 -121.79122831299611) bank128736 +128737 POINT(47.680386946750325 -123.09448408982907) bank128737 +128738 POINT(48.14765337617521 -122.27571897013948) bank128738 +128739 POINT(47.3382123584207 -122.0844933725641) bank128739 +128740 POINT(48.39468220183981 -121.98511920453878) bank128740 +128741 POINT(48.231078951894354 -122.27245001132435) bank128741 +128742 POINT(47.11941952201023 -121.73964660482918) bank128742 +128743 POINT(48.119884699109484 -121.9454850383326) bank128743 +128744 POINT(48.46040378539088 -121.53548134291071) bank128744 +128745 POINT(48.41546567225753 -121.48287014334247) bank128745 +128746 POINT(47.43102054517435 -123.19738397749308) bank128746 +128747 POINT(48.428760755490444 -123.18849271735546) bank128747 +128748 POINT(47.91792099581622 -121.95302643781254) bank128748 +128749 POINT(47.902965144975795 -122.70931884456058) bank128749 +128750 POINT(48.336171209978595 -123.02327246777574) bank128750 +128751 POINT(48.29803650024714 -122.8701873699216) bank128751 +128752 POINT(46.696259920512105 -122.8119916143869) bank128752 +128753 POINT(48.289809625348155 -122.44149456929192) bank128753 +128754 POINT(48.5162702918359 -122.25871615379455) bank128754 +128755 POINT(47.25179705645542 -122.09825353699063) bank128755 +128756 POINT(47.985284560565155 -122.30323541059403) bank128756 +128757 POINT(48.27509534997358 -123.06598110930794) bank128757 +128758 POINT(48.16825427361403 -121.46125919034526) bank128758 +128759 POINT(47.7121818138127 -121.80562220238592) bank128759 +128760 POINT(47.30080259195468 -122.29725878392563) bank128760 +128761 POINT(47.14590029544129 -123.15397539154873) bank128761 +128762 POINT(47.86728997620022 -121.77035539551099) bank128762 +128763 POINT(47.47965826284527 -121.35456809583351) bank128763 +128764 POINT(47.353776394824806 -122.41150278986265) bank128764 +128765 POINT(47.57171950087481 -121.50862862804264) bank128765 +128766 POINT(47.9781408739153 -121.459898329131) bank128766 +128767 POINT(47.628012170717355 -122.81947952977461) bank128767 +128768 POINT(46.78789139503123 -121.37975722750794) bank128768 +128769 POINT(46.6684412745984 -122.2142768624273) bank128769 +128770 POINT(47.07741106327733 -122.39242018362073) bank128770 +128771 POINT(47.826268496541864 -122.16804232275518) bank128771 +128772 POINT(47.40868997298249 -121.60609883099846) bank128772 +128773 POINT(47.11021414403676 -123.202575322035) bank128773 +128774 POINT(47.57844945109571 -122.10129640273696) bank128774 +128775 POINT(46.76799550670201 -121.84745728481295) bank128775 +128776 POINT(47.659870526867095 -121.47074380743172) bank128776 +128777 POINT(48.60181320045173 -122.57153243211071) bank128777 +128778 POINT(48.60263777403031 -122.7068257192102) bank128778 +128779 POINT(47.92564939945 -122.78662306674093) bank128779 +128780 POINT(46.6972829534576 -122.80109045438039) bank128780 +128781 POINT(48.59833973967607 -122.62512390629999) bank128781 +128782 POINT(46.68037498014788 -121.82375071698912) bank128782 +128783 POINT(47.20391955632947 -123.07332898284426) bank128783 +128784 POINT(48.493942048015306 -121.94097738190396) bank128784 +128785 POINT(47.996960327195374 -121.4239463110972) bank128785 +128786 POINT(47.39901150921949 -122.01148061050398) bank128786 +128787 POINT(48.42440115976181 -122.98142212707361) bank128787 +128788 POINT(47.9760929064197 -121.35742141672253) bank128788 +128789 POINT(47.83977585942997 -122.92867628876243) bank128789 +128790 POINT(48.07825271947808 -122.846510559146) bank128790 +128791 POINT(47.02047771099785 -122.9582088814855) bank128791 +128792 POINT(47.51759044904476 -121.51305587055076) bank128792 +128793 POINT(47.28728702663094 -121.46364261822386) bank128793 +128794 POINT(48.05701931443676 -121.51686211248997) bank128794 +128795 POINT(48.15692897617011 -122.95033971578032) bank128795 +128796 POINT(48.39806856505189 -122.25067469068951) bank128796 +128797 POINT(48.33757309358844 -122.36214221624333) bank128797 +128798 POINT(46.81213816565691 -121.66656770276988) bank128798 +128799 POINT(48.14700462562949 -123.04243771668038) bank128799 +128800 POINT(48.39761802029261 -122.1563195005815) bank128800 +128801 POINT(47.93489886588953 -121.78831411034966) bank128801 +128802 POINT(48.54542221291221 -122.21378617699419) bank128802 +128803 POINT(46.8577956807137 -121.89851965089795) bank128803 +128804 POINT(48.281347985466205 -122.52683271903749) bank128804 +128805 POINT(46.97640152999934 -121.59150222112655) bank128805 +128806 POINT(47.14127307865277 -121.68603673875968) bank128806 +128807 POINT(48.12293905566612 -122.31605355486658) bank128807 +128808 POINT(47.48263188272917 -121.55436667457907) bank128808 +128809 POINT(47.202579878186185 -122.28151217803823) bank128809 +128810 POINT(46.695538712540404 -121.95304785871723) bank128810 +128811 POINT(46.786718037565464 -121.55584573246647) bank128811 +128812 POINT(48.577845251505096 -121.91583999750254) bank128812 +128813 POINT(47.259171215563384 -122.38093396119531) bank128813 +128814 POINT(48.588929057551745 -122.17483982686524) bank128814 +128815 POINT(48.489420768405736 -122.30003712472912) bank128815 +128816 POINT(47.99005545310345 -122.84225840092053) bank128816 +128817 POINT(48.475177841905236 -121.43569524700933) bank128817 +128818 POINT(48.56731925575236 -123.07722767086909) bank128818 +128819 POINT(48.25749414005778 -122.97521602843251) bank128819 +128820 POINT(47.511902532989964 -122.50214473740662) bank128820 +128821 POINT(47.47413441576808 -122.941155986635) bank128821 +128822 POINT(46.91824703443765 -121.66432812578492) bank128822 +128823 POINT(48.45978066147323 -122.07347617980845) bank128823 +128824 POINT(48.176904948397215 -123.20982137992578) bank128824 +128825 POINT(46.64927894551618 -123.24012582243401) bank128825 +128826 POINT(48.20647760752478 -122.02969431389101) bank128826 +128827 POINT(47.41292929752691 -121.67661911445072) bank128827 +128828 POINT(47.33354790646171 -122.40867055321226) bank128828 +128829 POINT(47.638462578554496 -122.92167636883899) bank128829 +128830 POINT(48.069374129936755 -122.81055395868394) bank128830 +128831 POINT(48.354679324998784 -121.4670286448197) bank128831 +128832 POINT(46.94468255695139 -121.91507179652099) bank128832 +128833 POINT(48.390012295200016 -122.86189611768549) bank128833 +128834 POINT(46.87412452774286 -122.01850212704827) bank128834 +128835 POINT(46.838353541370495 -122.72394644109015) bank128835 +128836 POINT(46.796071690692884 -122.58562406263576) bank128836 +128837 POINT(47.787257042699295 -122.52939198822217) bank128837 +128838 POINT(48.30952637628777 -121.72541025269146) bank128838 +128839 POINT(48.09955289524859 -121.99479503410817) bank128839 +128840 POINT(48.41076016089665 -122.04462517049652) bank128840 +128841 POINT(48.17620660634774 -123.08154966835046) bank128841 +128842 POINT(46.76055855426945 -123.28677588675542) bank128842 +128843 POINT(46.913349307898656 -122.22672337525152) bank128843 +128844 POINT(48.31368836972524 -121.92972582334816) bank128844 +128845 POINT(46.82690603785023 -121.43611734726073) bank128845 +128846 POINT(46.82524140586424 -123.31212249620793) bank128846 +128847 POINT(47.18950933900613 -121.91870236130305) bank128847 +128848 POINT(48.455818066970146 -122.8011215461523) bank128848 +128849 POINT(48.09144511336832 -122.04691127784413) bank128849 +128850 POINT(47.247959602655925 -121.89807156336606) bank128850 +128851 POINT(47.0132855663964 -122.45308771825049) bank128851 +128852 POINT(48.14432620050775 -122.45496593492382) bank128852 +128853 POINT(47.11680987284094 -121.52276975759654) bank128853 +128854 POINT(46.97874991989866 -121.88045019266598) bank128854 +128855 POINT(48.33106946985582 -122.55891108349145) bank128855 +128856 POINT(47.69369721267038 -121.9290383191171) bank128856 +128857 POINT(46.862498831109384 -122.32212393796584) bank128857 +128858 POINT(47.11977063112352 -121.42387801102275) bank128858 +128859 POINT(47.320319453719435 -122.18437836953797) bank128859 +128860 POINT(48.21066683747261 -121.74991413808664) bank128860 +128861 POINT(47.946467009511 -123.03178236139716) bank128861 +128862 POINT(48.60555054034982 -122.97232467832431) bank128862 +128863 POINT(46.91843968624759 -121.78213252044361) bank128863 +128864 POINT(47.6112718300888 -123.15867499323748) bank128864 +128865 POINT(47.368574820669615 -122.88777660662988) bank128865 +128866 POINT(46.71975826527631 -122.50252246347027) bank128866 +128867 POINT(47.2800423793975 -122.0123977319535) bank128867 +128868 POINT(48.49908965536338 -121.35006279426096) bank128868 +128869 POINT(47.64395232064209 -121.47711116827138) bank128869 +128870 POINT(46.72635225529417 -122.21373555425595) bank128870 +128871 POINT(46.64522642833687 -123.08222147585893) bank128871 +128872 POINT(47.85452700720642 -121.87808134881911) bank128872 +128873 POINT(48.25084934997962 -122.46862801852653) bank128873 +128874 POINT(48.15980791902595 -123.11348352921193) bank128874 +128875 POINT(47.13265981465107 -123.24267544294136) bank128875 +128876 POINT(46.938208219967194 -122.87086449759568) bank128876 +128877 POINT(46.75152577549697 -123.19948213523344) bank128877 +128878 POINT(47.67638392373372 -121.46498848396298) bank128878 +128879 POINT(48.1161523351654 -122.85703934427134) bank128879 +128880 POINT(48.224897795022535 -121.78168255696248) bank128880 +128881 POINT(48.59850667821367 -122.73296190411939) bank128881 +128882 POINT(47.2653033860913 -122.36623297713095) bank128882 +128883 POINT(47.336122876605934 -122.2804349495939) bank128883 +128884 POINT(47.33466108261539 -122.86622404089259) bank128884 +128885 POINT(46.88825326322584 -122.56213742444086) bank128885 +128886 POINT(47.53207669017821 -121.37912894570226) bank128886 +128887 POINT(47.54564188669305 -123.3140929278208) bank128887 +128888 POINT(47.56933151410483 -121.55201209927853) bank128888 +128889 POINT(47.957156540185785 -121.53111265057214) bank128889 +128890 POINT(47.51847330756926 -122.02383318341234) bank128890 +128891 POINT(48.09032801034518 -122.18470597341577) bank128891 +128892 POINT(48.24306117846643 -122.52431189526062) bank128892 +128893 POINT(48.019051461261924 -121.41104592856898) bank128893 +128894 POINT(47.0779378093658 -123.2741874782912) bank128894 +128895 POINT(47.00328548153214 -123.30711448673209) bank128895 +128896 POINT(47.13802950482485 -121.62323420533858) bank128896 +128897 POINT(48.15433024575126 -121.76065815575076) bank128897 +128898 POINT(48.49712947045825 -123.2962428584372) bank128898 +128899 POINT(47.97875971317175 -123.2034488908906) bank128899 +128900 POINT(48.10923260592638 -121.75093177189753) bank128900 +128901 POINT(47.500088176982175 -121.65466808636138) bank128901 +128902 POINT(46.946100193668606 -123.07294446211262) bank128902 +128903 POINT(48.59662399783478 -121.80179987472468) bank128903 +128904 POINT(48.02914147305752 -122.72658407084752) bank128904 +128905 POINT(47.18413015516023 -123.29316033621848) bank128905 +128906 POINT(47.91461034509804 -122.94356664142498) bank128906 +128907 POINT(48.5992086087189 -121.52248850793991) bank128907 +128908 POINT(47.72038745958083 -121.42632459052372) bank128908 +128909 POINT(48.07042188000652 -122.34833986934362) bank128909 +128910 POINT(46.88519947327917 -121.67727533509141) bank128910 +128911 POINT(47.769161449436204 -121.8017732330194) bank128911 +128912 POINT(46.64949525070248 -122.80959766783872) bank128912 +128913 POINT(47.235028639192116 -121.54422171195847) bank128913 +128914 POINT(46.922680568084836 -121.9928905247095) bank128914 +128915 POINT(46.84765775786404 -121.50862309370915) bank128915 +128916 POINT(47.9397833630384 -123.2425884995101) bank128916 +128917 POINT(47.52481171254155 -122.34988874756495) bank128917 +128918 POINT(47.95205449135381 -122.9088484288662) bank128918 +128919 POINT(47.15938784499877 -121.79265641923841) bank128919 +128920 POINT(46.98573895451383 -121.95571840567831) bank128920 +128921 POINT(48.164722475958676 -122.8255211385359) bank128921 +128922 POINT(47.587132491823155 -122.97610255297141) bank128922 +128923 POINT(48.18521449468629 -122.1535664537226) bank128923 +128924 POINT(46.986190442502476 -122.7407710337847) bank128924 +128925 POINT(46.75304534263018 -123.27466700160784) bank128925 +128926 POINT(46.65066549246461 -123.33002486310092) bank128926 +128927 POINT(46.99502872289711 -121.8382460047964) bank128927 +128928 POINT(48.2831536669267 -121.61820259489787) bank128928 +128929 POINT(48.24572287782245 -123.03062712551537) bank128929 +128930 POINT(46.70560988585096 -121.88067468938955) bank128930 +128931 POINT(46.900140146909514 -122.99766562157036) bank128931 +128932 POINT(48.05427974809786 -121.79613620750796) bank128932 +128933 POINT(47.1355875127512 -122.55687710294681) bank128933 +128934 POINT(48.37839580150022 -121.92167964126521) bank128934 +128935 POINT(46.79094108727583 -122.63399115530544) bank128935 +128936 POINT(47.24174279418691 -122.57070508829953) bank128936 +128937 POINT(47.724403337620934 -122.19222763507292) bank128937 +128938 POINT(47.91920882194531 -123.00652580979887) bank128938 +128939 POINT(48.18212792323659 -121.43654501238451) bank128939 +128940 POINT(47.41369534237315 -121.72643017561929) bank128940 +128941 POINT(47.07482828879893 -121.53587157076184) bank128941 +128942 POINT(47.820084193568846 -122.38725040053765) bank128942 +128943 POINT(46.64108632246566 -121.65768717829266) bank128943 +128944 POINT(47.60578477935081 -122.3194066603556) bank128944 +128945 POINT(47.173552386664646 -121.8075446283894) bank128945 +128946 POINT(48.457764229220444 -123.30803782233619) bank128946 +128947 POINT(48.236828845335914 -122.07321901410359) bank128947 +128948 POINT(47.85290652444982 -121.81045666042561) bank128948 +128949 POINT(47.68684797218658 -121.41413409755394) bank128949 +128950 POINT(47.42886877675495 -122.18192117689601) bank128950 +128951 POINT(47.632026910752614 -121.91282840098424) bank128951 +128952 POINT(47.29828311097332 -121.96423038433026) bank128952 +128953 POINT(48.27155541697266 -121.70750452300099) bank128953 +128954 POINT(47.59076197983023 -121.50267807667244) bank128954 +128955 POINT(47.071514610245245 -123.08163071884071) bank128955 +128956 POINT(47.92649782782669 -121.57421144618596) bank128956 +128957 POINT(48.14125433051025 -122.74738679076776) bank128957 +128958 POINT(48.067404648869356 -122.71590643180252) bank128958 +128959 POINT(48.225030380476575 -123.27853815168963) bank128959 +128960 POINT(47.503493822801545 -122.09435244890193) bank128960 +128961 POINT(48.45748179723488 -121.75700027799469) bank128961 +128962 POINT(47.148589723167916 -122.6492706231851) bank128962 +128963 POINT(47.544012133660424 -122.90129868179328) bank128963 +128964 POINT(46.8041389663487 -121.47229821633958) bank128964 +128965 POINT(48.589500160619664 -122.03713415141198) bank128965 +128966 POINT(47.82048249048106 -122.84790623910563) bank128966 +128967 POINT(47.109737333350886 -121.50471766470181) bank128967 +128968 POINT(48.54078779657783 -121.39263335993867) bank128968 +128969 POINT(48.02618052239315 -122.53677882609573) bank128969 +128970 POINT(47.04717447009666 -122.1167782332242) bank128970 +128971 POINT(47.320011358925555 -122.22538609522731) bank128971 +128972 POINT(48.56569353909239 -123.11708275712358) bank128972 +128973 POINT(47.53214872729673 -121.83882596963844) bank128973 +128974 POINT(47.443221446050984 -122.47560585462497) bank128974 +128975 POINT(47.73410915614781 -122.63117739934258) bank128975 +128976 POINT(48.45412461733965 -122.44864605480673) bank128976 +128977 POINT(47.20937810484035 -121.36640728564863) bank128977 +128978 POINT(48.41746467610196 -122.05298939358879) bank128978 +128979 POINT(47.88698472971097 -123.3256524541667) bank128979 +128980 POINT(46.609179173038434 -121.39699491577214) bank128980 +128981 POINT(48.3270893077804 -122.4357031301698) bank128981 +128982 POINT(46.777803918020794 -122.06291149881895) bank128982 +128983 POINT(47.15808563049517 -121.75090536153924) bank128983 +128984 POINT(47.42736088091358 -123.26653947345699) bank128984 +128985 POINT(48.21853901840112 -121.84173509225003) bank128985 +128986 POINT(47.721339908044236 -121.76279429316811) bank128986 +128987 POINT(48.02282448694537 -122.32110237754812) bank128987 +128988 POINT(47.219342635087024 -122.35864477476363) bank128988 +128989 POINT(47.868251633760146 -122.70584714362501) bank128989 +128990 POINT(48.30406088302186 -122.99193051701661) bank128990 +128991 POINT(46.81659022573437 -121.6878666001468) bank128991 +128992 POINT(47.576696126136255 -122.8412721060027) bank128992 +128993 POINT(48.35697738295352 -121.99259346201445) bank128993 +128994 POINT(47.27330330759562 -122.21591812654623) bank128994 +128995 POINT(47.53697210867511 -121.5393285523763) bank128995 +128996 POINT(47.10833557946031 -123.31898643801219) bank128996 +128997 POINT(46.93704135820548 -121.65286763487396) bank128997 +128998 POINT(47.98157483367623 -122.21665493450914) bank128998 +128999 POINT(48.01116294654403 -121.37867255089259) bank128999 +129000 POINT(47.30038923786888 -122.47945667351642) bank129000 +129001 POINT(46.61719830090601 -121.51628787170011) bank129001 +129002 POINT(46.778396261348675 -121.70290698719307) bank129002 +129003 POINT(47.02344312685456 -121.8791302228805) bank129003 +129004 POINT(47.40781173938388 -123.15878874220357) bank129004 +129005 POINT(47.3764257863769 -121.58339905426782) bank129005 +129006 POINT(47.52030394195409 -123.10452941538709) bank129006 +129007 POINT(48.36224173696088 -121.49349645805627) bank129007 +129008 POINT(47.644719877956774 -121.50109547974343) bank129008 +129009 POINT(47.98079950312836 -121.9360331209523) bank129009 +129010 POINT(48.29314505953589 -121.6496720284522) bank129010 +129011 POINT(46.74868483028235 -121.80810393609678) bank129011 +129012 POINT(46.68892255439526 -122.32673379032033) bank129012 +129013 POINT(47.64219129587009 -121.96331018232826) bank129013 +129014 POINT(47.86535012588915 -121.89808073554919) bank129014 +129015 POINT(47.44454617090447 -121.88774184154528) bank129015 +129016 POINT(46.87581491849889 -123.28767136771582) bank129016 +129017 POINT(48.280482362333856 -122.1440514134067) bank129017 +129018 POINT(48.30313356285052 -122.23252748169865) bank129018 +129019 POINT(48.3394805805417 -122.0035209943472) bank129019 +129020 POINT(46.83028691680005 -122.07958888294948) bank129020 +129021 POINT(47.2965499592699 -122.11382699428422) bank129021 +129022 POINT(48.41299298761067 -122.9807992795382) bank129022 +129023 POINT(47.293805662738315 -122.08820936529341) bank129023 +129024 POINT(48.086312481234515 -121.6856536618094) bank129024 +129025 POINT(47.48990069596066 -121.69329694536313) bank129025 +129026 POINT(46.920400945995986 -123.14423389036213) bank129026 +129027 POINT(48.29279652191451 -122.95443592200499) bank129027 +129028 POINT(48.3401777173962 -123.19453101515121) bank129028 +129029 POINT(46.986938163848606 -122.92385057365432) bank129029 +129030 POINT(48.45122708581295 -121.82235248542933) bank129030 +129031 POINT(47.71839261620518 -121.50973019987867) bank129031 +129032 POINT(48.38115183000075 -122.69004088850615) bank129032 +129033 POINT(47.409283374850816 -123.24304880314098) bank129033 +129034 POINT(47.00505969760132 -121.88111860769394) bank129034 +129035 POINT(47.21266664821712 -122.37709507331532) bank129035 +129036 POINT(47.024058180334414 -122.43508838982765) bank129036 +129037 POINT(47.27697243126922 -122.52888822309043) bank129037 +129038 POINT(47.36865185628605 -121.95196263997511) bank129038 +129039 POINT(47.53310294617486 -122.14461224124676) bank129039 +129040 POINT(47.96721546147672 -121.84448486883251) bank129040 +129041 POINT(47.69233781820748 -122.4095345153793) bank129041 +129042 POINT(48.539242684200296 -121.87623324276025) bank129042 +129043 POINT(47.93102032851071 -122.25573690634242) bank129043 +129044 POINT(48.090900845855074 -121.54004050445442) bank129044 +129045 POINT(48.072835816014994 -122.56532024394899) bank129045 +129046 POINT(47.0204917633707 -121.7717051304175) bank129046 +129047 POINT(47.71236764332063 -122.0194492842602) bank129047 +129048 POINT(47.152190966870535 -121.33798803098) bank129048 +129049 POINT(47.33732900884512 -121.39241435577) bank129049 +129050 POINT(47.10670930130287 -122.60219446483647) bank129050 +129051 POINT(46.822648680524075 -121.40157724470119) bank129051 +129052 POINT(47.29188545184756 -121.45723523309704) bank129052 +129053 POINT(47.23244671048693 -122.69372540423944) bank129053 +129054 POINT(48.504580549357016 -122.95217125468598) bank129054 +129055 POINT(46.66056304308717 -123.03019472651238) bank129055 +129056 POINT(47.57999059849259 -122.65892808204529) bank129056 +129057 POINT(47.141757755923926 -121.5571010527821) bank129057 +129058 POINT(47.84765463936437 -121.40467617469353) bank129058 +129059 POINT(46.833691991190264 -122.9893298401165) bank129059 +129060 POINT(48.406114202879294 -121.98579602413996) bank129060 +129061 POINT(47.23637854144298 -122.83407378707935) bank129061 +129062 POINT(47.90947695337562 -122.39016417171585) bank129062 +129063 POINT(48.41489655145251 -123.31257901192832) bank129063 +129064 POINT(46.90470901761991 -121.3722343806133) bank129064 +129065 POINT(48.23311391627773 -122.68119308054163) bank129065 +129066 POINT(46.93527255901553 -122.03996581259337) bank129066 +129067 POINT(48.18872908044307 -123.24814586952643) bank129067 +129068 POINT(46.80780636933598 -122.59434871029238) bank129068 +129069 POINT(46.9619021171319 -121.4316779651416) bank129069 +129070 POINT(47.41663059232651 -123.15145277955827) bank129070 +129071 POINT(48.455357232339836 -122.227198914185) bank129071 +129072 POINT(47.48638045093147 -123.02151428876334) bank129072 +129073 POINT(48.24353947339567 -123.10879207543924) bank129073 +129074 POINT(48.318535676130885 -121.62498914877443) bank129074 +129075 POINT(47.03564461471604 -122.02622799924116) bank129075 +129076 POINT(47.466925300239005 -121.91883002303916) bank129076 +129077 POINT(47.836782579667705 -122.67921561394158) bank129077 +129078 POINT(46.6436985794915 -121.5474323390071) bank129078 +129079 POINT(47.18211645385157 -121.35808897919065) bank129079 +129080 POINT(47.65105940999334 -122.45801510808472) bank129080 +129081 POINT(48.56639357448621 -123.08697225012409) bank129081 +129082 POINT(48.260562241574725 -122.14099406554693) bank129082 +129083 POINT(47.15622511292499 -122.60536332008145) bank129083 +129084 POINT(47.60331470938757 -121.56742221940101) bank129084 +129085 POINT(47.828263053835094 -123.22830463366236) bank129085 +129086 POINT(46.89793518469927 -122.99931902543861) bank129086 +129087 POINT(47.613407780691716 -122.68569524022996) bank129087 +129088 POINT(48.154220296211896 -121.99144569241585) bank129088 +129089 POINT(46.85173040263835 -121.77891977531999) bank129089 +129090 POINT(48.084054277865775 -123.26046625001375) bank129090 +129091 POINT(47.85921646174369 -121.38378946915304) bank129091 +129092 POINT(47.02962521058679 -123.24454068032757) bank129092 +129093 POINT(46.858889122261246 -122.71786833971062) bank129093 +129094 POINT(47.153703388212506 -121.58783720336714) bank129094 +129095 POINT(46.766688179175894 -121.44114603941819) bank129095 +129096 POINT(47.14866981432936 -122.76819823813871) bank129096 +129097 POINT(47.77148571140401 -122.9035430931198) bank129097 +129098 POINT(47.627981316665085 -122.08728190519153) bank129098 +129099 POINT(47.45470149099433 -121.74409235603765) bank129099 +129100 POINT(46.784549636853725 -121.49515971478138) bank129100 +129101 POINT(47.121448856722814 -121.50970460095635) bank129101 +129102 POINT(46.9956730262114 -121.89498960888841) bank129102 +129103 POINT(47.20332124928546 -122.66243818044494) bank129103 +129104 POINT(48.26666317093921 -122.52863623942402) bank129104 +129105 POINT(48.50306048156986 -123.00779078890217) bank129105 +129106 POINT(46.71866554622661 -121.43766870802862) bank129106 +129107 POINT(46.87245503154473 -122.69172670322577) bank129107 +129108 POINT(46.91717138692898 -121.7260257888448) bank129108 +129109 POINT(47.3003625955298 -121.47882076663085) bank129109 +129110 POINT(47.56612907064574 -122.51785275573636) bank129110 +129111 POINT(47.95187949804501 -121.90146625130691) bank129111 +129112 POINT(46.742826192155384 -123.2897087787012) bank129112 +129113 POINT(47.52313010067207 -122.18391874759816) bank129113 +129114 POINT(46.88700179848891 -122.26723486093982) bank129114 +129115 POINT(47.36142771094696 -123.16108511926763) bank129115 +129116 POINT(48.36194094195222 -121.38282120455206) bank129116 +129117 POINT(47.09337495950129 -121.75947463737207) bank129117 +129118 POINT(46.6809468231384 -122.81749254650931) bank129118 +129119 POINT(47.213262971871174 -122.86689083262381) bank129119 +129120 POINT(46.93800762176293 -121.8549843472398) bank129120 +129121 POINT(48.403363205500014 -121.73664457486997) bank129121 +129122 POINT(46.81365430910592 -122.31801942842958) bank129122 +129123 POINT(47.61927716917125 -122.25567814447665) bank129123 +129124 POINT(46.81771622569899 -121.48993232721632) bank129124 +129125 POINT(46.72126768903247 -123.21390909959322) bank129125 +129126 POINT(48.0302573481646 -122.45271510271077) bank129126 +129127 POINT(47.81676747476638 -122.33954834248117) bank129127 +129128 POINT(47.58497863008597 -122.11751682617269) bank129128 +129129 POINT(46.869379462670665 -122.22359001803972) bank129129 +129130 POINT(46.93560561640422 -121.3672700241058) bank129130 +129131 POINT(48.48280607021976 -121.98405891794982) bank129131 +129132 POINT(48.55070164524226 -123.09254373857942) bank129132 +129133 POINT(47.474014552331404 -122.3950907682067) bank129133 +129134 POINT(47.85039895359046 -122.30885205911791) bank129134 +129135 POINT(47.808765261957305 -121.83941564356098) bank129135 +129136 POINT(47.059304230896394 -123.22833669804241) bank129136 +129137 POINT(48.20554074366969 -121.43843452284877) bank129137 +129138 POINT(47.825738395141215 -122.70201255413167) bank129138 +129139 POINT(47.84249663596241 -122.02629456244553) bank129139 +129140 POINT(46.96131808733597 -122.14079670505517) bank129140 +129141 POINT(47.813750758864145 -122.48300934638254) bank129141 +129142 POINT(47.5987945735139 -121.37204268725334) bank129142 +129143 POINT(47.48482914789424 -122.71805890270026) bank129143 +129144 POINT(46.76268703697597 -123.27388581616891) bank129144 +129145 POINT(48.1650011373908 -123.15989158563147) bank129145 +129146 POINT(48.01830011832454 -123.18633590860496) bank129146 +129147 POINT(47.617439274417194 -121.55165091164214) bank129147 +129148 POINT(48.261656513258714 -122.75566852526923) bank129148 +129149 POINT(47.652691761713456 -121.95244646014957) bank129149 +129150 POINT(48.11364602313078 -122.85016398914529) bank129150 +129151 POINT(47.86756565558579 -123.26413998977131) bank129151 +129152 POINT(48.12442487857612 -122.73776535808103) bank129152 +129153 POINT(47.437084057266986 -122.7534828316717) bank129153 +129154 POINT(46.96100674488633 -122.75033530010906) bank129154 +129155 POINT(47.277680454141 -121.36768183374534) bank129155 +129156 POINT(48.23994123954733 -121.65719811777551) bank129156 +129157 POINT(46.921549034710715 -122.77663745492075) bank129157 +129158 POINT(47.384646985233665 -122.49838277629465) bank129158 +129159 POINT(47.692158904021476 -121.44157794192378) bank129159 +129160 POINT(48.069329610786234 -122.02163590640481) bank129160 +129161 POINT(47.63750900271341 -121.37802440251282) bank129161 +129162 POINT(47.84759633389196 -121.97343980522248) bank129162 +129163 POINT(48.563149604730846 -121.5668611402725) bank129163 +129164 POINT(47.641641835902156 -122.48873431234765) bank129164 +129165 POINT(48.06315585572701 -121.99788492363331) bank129165 +129166 POINT(48.05097826143728 -122.35184674080814) bank129166 +129167 POINT(47.29928179639891 -122.35426034261955) bank129167 +129168 POINT(48.008293053370124 -122.64614643595597) bank129168 +129169 POINT(47.744071712776474 -121.35134297820053) bank129169 +129170 POINT(47.9097860764625 -121.93361632970807) bank129170 +129171 POINT(46.77203070211388 -123.13468763716807) bank129171 +129172 POINT(48.02611540593511 -121.93773740806154) bank129172 +129173 POINT(47.036554805663606 -122.91737357708338) bank129173 +129174 POINT(47.89029198552957 -122.0668683428583) bank129174 +129175 POINT(48.55143511612325 -122.20625745226606) bank129175 +129176 POINT(46.74682987624675 -121.7108231358809) bank129176 +129177 POINT(47.19469411403759 -121.7450599136557) bank129177 +129178 POINT(46.993411915746506 -121.90151284029348) bank129178 +129179 POINT(48.021483262029534 -122.05997606621374) bank129179 +129180 POINT(47.25318802747459 -122.21985320380493) bank129180 +129181 POINT(47.947468130373466 -122.95446266209757) bank129181 +129182 POINT(47.07927065126627 -122.81101875021096) bank129182 +129183 POINT(47.10598440131383 -122.53432474201288) bank129183 +129184 POINT(47.25584237280495 -122.35781290708127) bank129184 +129185 POINT(48.548160114664825 -121.99851914147945) bank129185 +129186 POINT(46.85182618064809 -122.6963748021636) bank129186 +129187 POINT(47.86123215549963 -122.04847470571113) bank129187 +129188 POINT(47.48147494937736 -121.5473360828761) bank129188 +129189 POINT(47.66804984900785 -122.72460632442692) bank129189 +129190 POINT(46.94429211076372 -123.08872642738574) bank129190 +129191 POINT(47.37789123389523 -121.86262822392553) bank129191 +129192 POINT(46.798365066206784 -122.81161260401026) bank129192 +129193 POINT(48.44427501169379 -123.23828589556591) bank129193 +129194 POINT(47.85316501502739 -123.14347094751177) bank129194 +129195 POINT(48.08963264437138 -121.7306138501937) bank129195 +129196 POINT(47.712875723330626 -123.1092613182858) bank129196 +129197 POINT(47.05494233558532 -121.52625200574693) bank129197 +129198 POINT(48.579337127305195 -122.75502598128884) bank129198 +129199 POINT(47.2826881160216 -121.81227780060824) bank129199 +129200 POINT(47.011082083546114 -122.46944957148071) bank129200 +129201 POINT(48.387685612345585 -122.14128620766915) bank129201 +129202 POINT(48.574097894070306 -122.19799531023145) bank129202 +129203 POINT(48.53797756076446 -122.29542846028608) bank129203 +129204 POINT(47.509840082104034 -122.40354185204068) bank129204 +129205 POINT(47.94849067741983 -122.48065259822089) bank129205 +129206 POINT(47.35862618164532 -122.20119364219697) bank129206 +129207 POINT(48.433019657457415 -122.54812929850058) bank129207 +129208 POINT(48.301981578495315 -122.04985695314062) bank129208 +129209 POINT(46.63180822315621 -123.07245372411725) bank129209 +129210 POINT(47.403894548918736 -123.32573792966173) bank129210 +129211 POINT(47.48882395403818 -122.41773857826665) bank129211 +129212 POINT(47.62305351746047 -121.7804518468468) bank129212 +129213 POINT(47.13709723436675 -122.29980557238939) bank129213 +129214 POINT(47.296270402350274 -121.90515692123351) bank129214 +129215 POINT(48.108670269736244 -121.7505076970809) bank129215 +129216 POINT(48.28091426517073 -123.23628865934184) bank129216 +129217 POINT(48.03745912965767 -122.71578629999189) bank129217 +129218 POINT(48.1571725079791 -122.28945296603474) bank129218 +129219 POINT(47.175161096664006 -122.69678569515699) bank129219 +129220 POINT(46.97234341994887 -122.4848173252254) bank129220 +129221 POINT(47.038211996186035 -122.32047149018327) bank129221 +129222 POINT(47.59322217460113 -122.07103540983444) bank129222 +129223 POINT(47.21421064815458 -123.31532718011668) bank129223 +129224 POINT(46.908043166877256 -121.79458070952919) bank129224 +129225 POINT(47.35199420342608 -121.33425668623072) bank129225 +129226 POINT(47.604386517773946 -123.01817609816499) bank129226 +129227 POINT(48.08231746008276 -123.13594929628619) bank129227 +129228 POINT(46.64506087183348 -122.56480253194538) bank129228 +129229 POINT(48.38222034215573 -122.2926106225501) bank129229 +129230 POINT(47.719206058434615 -121.77999296713377) bank129230 +129231 POINT(48.55936643075221 -122.28552080559854) bank129231 +129232 POINT(48.220054379134574 -121.33309399589237) bank129232 +129233 POINT(47.93928168033825 -122.44150059715498) bank129233 +129234 POINT(48.60273457560548 -121.63296645515085) bank129234 +129235 POINT(46.70046974018426 -121.81922074440837) bank129235 +129236 POINT(47.43502680433315 -122.57323283921379) bank129236 +129237 POINT(46.80869648297352 -121.82684021830396) bank129237 +129238 POINT(46.7669494532235 -123.30716382954542) bank129238 +129239 POINT(47.429262329513314 -121.64409336929349) bank129239 +129240 POINT(47.55693806300162 -122.97892826247755) bank129240 +129241 POINT(48.27729471035394 -122.99473572615014) bank129241 +129242 POINT(48.488736792060344 -122.92604199126092) bank129242 +129243 POINT(47.273205683715155 -123.24841239549609) bank129243 +129244 POINT(48.28631012583319 -122.82736326537345) bank129244 +129245 POINT(47.41104138540236 -121.6657507293584) bank129245 +129246 POINT(48.36318342911022 -121.59622889411438) bank129246 +129247 POINT(48.22820749931831 -123.22127988698536) bank129247 +129248 POINT(47.14335506863955 -122.73207646388461) bank129248 +129249 POINT(47.16524823492764 -121.86906866091749) bank129249 +129250 POINT(47.463218569920855 -123.19502042782987) bank129250 +129251 POINT(48.388262085332634 -121.71349729972874) bank129251 +129252 POINT(48.560070496341915 -121.64967402049325) bank129252 +129253 POINT(47.12653925832591 -121.90589659076318) bank129253 +129254 POINT(47.891036301155076 -122.38121940270052) bank129254 +129255 POINT(47.3282494234271 -122.98847123681679) bank129255 +129256 POINT(47.531560231713584 -122.55495491737894) bank129256 +129257 POINT(47.75805441146409 -122.68991541653739) bank129257 +129258 POINT(47.7845440387963 -122.36571875512597) bank129258 +129259 POINT(47.272806973505055 -122.1434947338572) bank129259 +129260 POINT(47.90605240645604 -122.41910406632962) bank129260 +129261 POINT(46.68466867187383 -121.60551659278285) bank129261 +129262 POINT(48.42692342967625 -122.05769474703516) bank129262 +129263 POINT(48.55033212466989 -121.80190327990874) bank129263 +129264 POINT(47.03260299091054 -122.81011968169966) bank129264 +129265 POINT(48.32271210603824 -121.51617544546984) bank129265 +129266 POINT(47.739951014626186 -122.5473420370466) bank129266 +129267 POINT(47.22437096345495 -123.11827415528813) bank129267 +129268 POINT(46.97071205414158 -123.03841697732017) bank129268 +129269 POINT(46.78688919656123 -121.7297327585115) bank129269 +129270 POINT(48.54653584908102 -123.08900035311999) bank129270 +129271 POINT(47.267283984325225 -123.2815220082622) bank129271 +129272 POINT(46.837311856204 -123.12151592387782) bank129272 +129273 POINT(46.85686138944776 -122.7881783083087) bank129273 +129274 POINT(47.48378735610551 -121.54411580641883) bank129274 +129275 POINT(47.59874166031688 -122.68390978598052) bank129275 +129276 POINT(47.06876324487414 -122.39350875501371) bank129276 +129277 POINT(46.75797878340554 -121.71818080332488) bank129277 +129278 POINT(46.680720926534306 -122.82872693049104) bank129278 +129279 POINT(47.588279198640244 -122.1951896744351) bank129279 +129280 POINT(47.31938554931518 -121.66837424617515) bank129280 +129281 POINT(48.22309414871218 -122.79588060111702) bank129281 +129282 POINT(46.63749569336827 -122.88023724551415) bank129282 +129283 POINT(47.07211115862902 -122.24723742496987) bank129283 +129284 POINT(48.4242310436321 -121.93818509007666) bank129284 +129285 POINT(48.097045953498544 -122.08252768434637) bank129285 +129286 POINT(48.324746695507756 -121.54046011837157) bank129286 +129287 POINT(48.20308812485737 -121.61650529854923) bank129287 +129288 POINT(48.215546008283056 -122.29787311277614) bank129288 +129289 POINT(48.163252069382224 -122.26051411455498) bank129289 +129290 POINT(48.339359917317246 -121.71082203820856) bank129290 +129291 POINT(46.93242647117769 -121.38843552198877) bank129291 +129292 POINT(48.176339276848495 -121.85491112237683) bank129292 +129293 POINT(47.58638069475556 -123.07368917967298) bank129293 +129294 POINT(48.5851198123601 -121.93093961452732) bank129294 +129295 POINT(47.44119393378233 -121.56020795973015) bank129295 +129296 POINT(48.486043238192764 -121.36632520413028) bank129296 +129297 POINT(48.367090570490944 -123.26227372329481) bank129297 +129298 POINT(46.763713568607486 -121.52779919503233) bank129298 +129299 POINT(47.97712911048547 -121.74146240772927) bank129299 +129300 POINT(48.260643128176184 -122.00880291878167) bank129300 +129301 POINT(48.45763588487401 -123.26869382954214) bank129301 +129302 POINT(47.83327418586824 -121.33257343421202) bank129302 +129303 POINT(47.0152740543228 -122.31841426711685) bank129303 +129304 POINT(48.07376470621482 -122.42700143798896) bank129304 +129305 POINT(46.981693428567844 -122.04553162452565) bank129305 +129306 POINT(48.41263344647552 -122.0690095972519) bank129306 +129307 POINT(47.955598072116175 -122.8663780450713) bank129307 +129308 POINT(48.22047913320692 -121.50053335736766) bank129308 +129309 POINT(48.09209898601328 -121.33226438086257) bank129309 +129310 POINT(48.346651946932454 -123.10632778275657) bank129310 +129311 POINT(46.74537818677736 -122.34372277998625) bank129311 +129312 POINT(46.69356724081508 -122.5667283229222) bank129312 +129313 POINT(47.10167074998173 -121.47921997095314) bank129313 +129314 POINT(48.03941008454328 -122.3368266445633) bank129314 +129315 POINT(47.36427981293001 -123.08410934335257) bank129315 +129316 POINT(47.17337650393865 -122.57234408990882) bank129316 +129317 POINT(47.63734273229639 -121.38650162574754) bank129317 +129318 POINT(46.65105233894659 -121.607252632077) bank129318 +129319 POINT(48.01800574940654 -123.01108317478854) bank129319 +129320 POINT(47.798981618318315 -121.46542290845534) bank129320 +129321 POINT(48.435228493980084 -122.45358894824932) bank129321 +129322 POINT(47.86034212791341 -122.4295638823043) bank129322 +129323 POINT(47.91855334777718 -123.08577165512064) bank129323 +129324 POINT(48.437648886963444 -122.61719427523047) bank129324 +129325 POINT(46.85688539159008 -122.14889914710282) bank129325 +129326 POINT(48.547136924251944 -122.89382730579608) bank129326 +129327 POINT(48.16549751276161 -122.12592571485676) bank129327 +129328 POINT(48.55926320749958 -121.94652524330478) bank129328 +129329 POINT(47.159111971875284 -122.8895744088229) bank129329 +129330 POINT(48.44607010322465 -122.09340084804853) bank129330 +129331 POINT(47.34022159459014 -123.08747025872577) bank129331 +129332 POINT(47.26943260934302 -121.37521242266757) bank129332 +129333 POINT(47.79150288557498 -122.3995982246243) bank129333 +129334 POINT(47.99831699587833 -122.82910396864398) bank129334 +129335 POINT(48.367458413832445 -123.33117198244855) bank129335 +129336 POINT(47.457375478952024 -121.61608334987373) bank129336 +129337 POINT(47.26444166244207 -123.01546602212998) bank129337 +129338 POINT(47.30622388697829 -123.1250703485279) bank129338 +129339 POINT(48.15168868746837 -123.03071793571203) bank129339 +129340 POINT(47.22071030671949 -122.92987729117873) bank129340 +129341 POINT(47.045679506658715 -121.64516736395306) bank129341 +129342 POINT(48.49185030861969 -123.24677691191698) bank129342 +129343 POINT(48.216047162604255 -123.11117013514807) bank129343 +129344 POINT(48.44972154676236 -121.86532415284566) bank129344 +129345 POINT(47.09971413212388 -121.61228276996178) bank129345 +129346 POINT(46.78857088211527 -123.28625664966917) bank129346 +129347 POINT(47.95156718366813 -122.23386586583376) bank129347 +129348 POINT(47.82410158813164 -121.9218496281249) bank129348 +129349 POINT(47.31632140302142 -122.13964516887471) bank129349 +129350 POINT(47.951166960281164 -121.93349699344127) bank129350 +129351 POINT(46.88069200295526 -123.0777630084249) bank129351 +129352 POINT(48.16690665826902 -122.47892839603618) bank129352 +129353 POINT(46.945221129480906 -122.74968318016154) bank129353 +129354 POINT(48.28877796799879 -122.78040679608324) bank129354 +129355 POINT(48.4773850019055 -123.28508936118081) bank129355 +129356 POINT(46.615277552506655 -122.38250364273411) bank129356 +129357 POINT(46.66740938052019 -122.6740935618197) bank129357 +129358 POINT(47.67088277274366 -121.87125166395047) bank129358 +129359 POINT(47.01689690839544 -121.52888441068494) bank129359 +129360 POINT(46.79607809566176 -123.31052517398865) bank129360 +129361 POINT(48.34399437385357 -122.10307433966464) bank129361 +129362 POINT(47.83004922904455 -122.87064000125191) bank129362 +129363 POINT(47.4779088409105 -122.47469576728531) bank129363 +129364 POINT(48.13897837050842 -122.04987344659803) bank129364 +129365 POINT(48.50853714387386 -121.44945051071213) bank129365 +129366 POINT(46.719387222664395 -122.83062187688205) bank129366 +129367 POINT(48.519252307922024 -121.89373006381712) bank129367 +129368 POINT(46.84918001039515 -122.9025412631826) bank129368 +129369 POINT(47.166314612045134 -122.54631761900882) bank129369 +129370 POINT(48.278151115991406 -122.73754677271239) bank129370 +129371 POINT(47.498126790309 -123.05238378842273) bank129371 +129372 POINT(47.63900413759559 -123.25875775153015) bank129372 +129373 POINT(48.21997373018845 -121.9007117600542) bank129373 +129374 POINT(48.29652190342631 -122.7958109333853) bank129374 +129375 POINT(48.53073919579501 -122.62451530766334) bank129375 +129376 POINT(46.85313958808296 -122.7192042439146) bank129376 +129377 POINT(47.42294697781808 -122.11824251284185) bank129377 +129378 POINT(48.206727526506015 -122.93339433049049) bank129378 +129379 POINT(48.43795313149275 -122.02691348518499) bank129379 +129380 POINT(46.675995028124255 -122.93243094094571) bank129380 +129381 POINT(47.5325213281651 -122.72013751221299) bank129381 +129382 POINT(47.91876261306035 -122.84727434896055) bank129382 +129383 POINT(46.95963840764964 -123.2671690954716) bank129383 +129384 POINT(47.22864406502598 -122.11328589231792) bank129384 +129385 POINT(48.44250308628885 -122.09012509522978) bank129385 +129386 POINT(48.05115125735193 -121.63993012757665) bank129386 +129387 POINT(47.64866396452388 -123.05262506988826) bank129387 +129388 POINT(46.908401124212745 -121.5503003789845) bank129388 +129389 POINT(48.37757538813155 -121.71319460141432) bank129389 +129390 POINT(47.830368880374216 -123.01309858870421) bank129390 +129391 POINT(48.417683331343056 -122.06750615656087) bank129391 +129392 POINT(47.544130112874164 -123.09549055346545) bank129392 +129393 POINT(48.48461378360446 -122.8413949143674) bank129393 +129394 POINT(48.51407476991284 -121.54277910767104) bank129394 +129395 POINT(48.133944188592594 -123.26046145650001) bank129395 +129396 POINT(48.42987386697437 -122.92810778306688) bank129396 +129397 POINT(48.32867359562965 -122.69252102279245) bank129397 +129398 POINT(47.38151008994716 -122.50898999882962) bank129398 +129399 POINT(47.484331119862304 -122.67928011850516) bank129399 +129400 POINT(48.186265810728486 -121.46280211959929) bank129400 +129401 POINT(47.0983056830223 -122.65084295129468) bank129401 +129402 POINT(46.93413113157772 -121.96271434383785) bank129402 +129403 POINT(47.73235786240352 -122.29438722158193) bank129403 +129404 POINT(48.268128338743644 -123.18840017616566) bank129404 +129405 POINT(46.708192381879535 -123.05443374330211) bank129405 +129406 POINT(47.539836066967595 -122.00496382592073) bank129406 +129407 POINT(47.379107655691016 -121.90965780495333) bank129407 +129408 POINT(48.089728697928855 -122.71294299627209) bank129408 +129409 POINT(47.61205170395457 -121.56594827778606) bank129409 +129410 POINT(47.60228587763159 -123.17712826610664) bank129410 +129411 POINT(48.28120675426961 -122.06333600361202) bank129411 +129412 POINT(47.3285466755735 -122.13224413545176) bank129412 +129413 POINT(48.47144995320634 -121.43507035764188) bank129413 +129414 POINT(47.892374654698706 -123.27791193366268) bank129414 +129415 POINT(46.63762249574324 -122.33716920017461) bank129415 +129416 POINT(48.051598439484714 -122.17936172309632) bank129416 +129417 POINT(47.62950897100803 -121.8584784328297) bank129417 +129418 POINT(47.20630354469968 -122.48459609745716) bank129418 +129419 POINT(47.01065124077124 -121.46030913838868) bank129419 +129420 POINT(48.279283600702584 -121.604919013351) bank129420 +129421 POINT(47.0448849571114 -121.33940822928666) bank129421 +129422 POINT(47.139333284165005 -121.82347889017608) bank129422 +129423 POINT(48.27113589333806 -121.39143888022095) bank129423 +129424 POINT(48.27400718934023 -122.67773749875926) bank129424 +129425 POINT(47.41240615254667 -123.21324598246069) bank129425 +129426 POINT(47.870375180632394 -122.30346791367027) bank129426 +129427 POINT(46.914538681595104 -122.96319403173356) bank129427 +129428 POINT(47.04732854966373 -123.2223748767814) bank129428 +129429 POINT(46.672432719157634 -123.08075846716434) bank129429 +129430 POINT(48.318219896107976 -122.85033144003242) bank129430 +129431 POINT(46.94688473821109 -122.63024544063872) bank129431 +129432 POINT(48.118919250425535 -122.93832583826038) bank129432 +129433 POINT(47.909333810353786 -121.56864380284455) bank129433 +129434 POINT(46.95988074854288 -122.30045374310158) bank129434 +129435 POINT(47.576944985495984 -121.44827171065154) bank129435 +129436 POINT(47.549714191505096 -123.05293560032634) bank129436 +129437 POINT(47.852564611174024 -121.4672849831779) bank129437 +129438 POINT(48.01170437284266 -122.82370390409739) bank129438 +129439 POINT(48.25483632641357 -122.75061708480375) bank129439 +129440 POINT(47.05345116281876 -123.04281446768783) bank129440 +129441 POINT(47.3743803613042 -121.77384755769434) bank129441 +129442 POINT(46.880922826863674 -121.84140443799572) bank129442 +129443 POINT(48.40619408408389 -122.4264638429907) bank129443 +129444 POINT(46.63145083867519 -122.72650802004581) bank129444 +129445 POINT(48.37232150169019 -122.14173603902871) bank129445 +129446 POINT(46.85031860927913 -121.66421667058188) bank129446 +129447 POINT(46.90220414665024 -123.06904247846708) bank129447 +129448 POINT(48.13354603105072 -122.63441401624648) bank129448 +129449 POINT(47.33540558781663 -121.74947912442502) bank129449 +129450 POINT(48.08024679193831 -121.37647881568205) bank129450 +129451 POINT(47.493178536403065 -121.35917062321307) bank129451 +129452 POINT(48.55014386367321 -122.82730474002412) bank129452 +129453 POINT(47.51871829345549 -121.62534638301669) bank129453 +129454 POINT(47.4035354850293 -122.46635663054103) bank129454 +129455 POINT(48.31308900331147 -122.744829726451) bank129455 +129456 POINT(47.445522510269235 -122.78818030749935) bank129456 +129457 POINT(48.15778774419699 -122.44464872087657) bank129457 +129458 POINT(48.570785289563304 -121.76552255026616) bank129458 +129459 POINT(47.194745357656316 -123.03621322839325) bank129459 +129460 POINT(47.086807904597656 -121.50799798392315) bank129460 +129461 POINT(48.39985330894371 -121.39336374016841) bank129461 +129462 POINT(47.38513581402738 -122.88387097915752) bank129462 +129463 POINT(46.812630813003196 -122.17508295060208) bank129463 +129464 POINT(46.70476701564404 -122.73597675503943) bank129464 +129465 POINT(46.69101765853522 -121.605226446792) bank129465 +129466 POINT(47.83551129577048 -122.45542085162771) bank129466 +129467 POINT(48.22824230509217 -121.4292224506894) bank129467 +129468 POINT(47.75431164659186 -122.57225428001392) bank129468 +129469 POINT(47.22594168919637 -122.43900495757603) bank129469 +129470 POINT(46.76850951100219 -122.04243092954846) bank129470 +129471 POINT(47.58717850808817 -121.36374219832562) bank129471 +129472 POINT(46.863880119788845 -121.55813177115978) bank129472 +129473 POINT(46.67268893609789 -121.44450122014771) bank129473 +129474 POINT(47.95138246415447 -122.90840401690988) bank129474 +129475 POINT(47.42860379952046 -122.83307020180965) bank129475 +129476 POINT(47.70604493868867 -121.44265001391474) bank129476 +129477 POINT(46.78638672135663 -121.50620980135108) bank129477 +129478 POINT(47.765579269971624 -122.09059381442661) bank129478 +129479 POINT(47.50700176629368 -122.0374228999123) bank129479 +129480 POINT(47.5348495441572 -121.92813563262749) bank129480 +129481 POINT(47.088759095460574 -122.39983031341566) bank129481 +129482 POINT(48.284279236507 -122.43290206055066) bank129482 +129483 POINT(46.75226308998448 -121.74969205836504) bank129483 +129484 POINT(47.704920666690136 -122.25477668133873) bank129484 +129485 POINT(47.046691854116844 -123.26713525668836) bank129485 +129486 POINT(47.314890351589476 -122.06275342729099) bank129486 +129487 POINT(48.55293999749614 -121.39087158678132) bank129487 +129488 POINT(47.893481578212096 -123.26860591118259) bank129488 +129489 POINT(47.6859508048731 -121.508781187151) bank129489 +129490 POINT(46.69541240554905 -122.94917497940554) bank129490 +129491 POINT(47.152707783564495 -121.89632529025435) bank129491 +129492 POINT(48.38072668860485 -121.87590464740606) bank129492 +129493 POINT(47.8381426891168 -123.15297879801331) bank129493 +129494 POINT(47.45578285941194 -121.33310897246365) bank129494 +129495 POINT(47.71754128548684 -123.08950286429182) bank129495 +129496 POINT(48.26577492309123 -123.29432030300244) bank129496 +129497 POINT(46.62291058544518 -122.2261987736822) bank129497 +129498 POINT(47.50736040904599 -121.46897007103699) bank129498 +129499 POINT(47.520133107917985 -121.3681507459445) bank129499 +129500 POINT(48.46362520361304 -122.70822386317317) bank129500 +129501 POINT(48.583546811858085 -123.13177710909622) bank129501 +129502 POINT(47.389383424605576 -121.66965170883157) bank129502 +129503 POINT(48.221554540726395 -121.64836946888629) bank129503 +129504 POINT(47.48156939443551 -122.96414886770933) bank129504 +129505 POINT(46.94839554010542 -122.26537179050426) bank129505 +129506 POINT(47.33773239234965 -122.8347555440658) bank129506 +129507 POINT(47.31657260102997 -121.73986531479663) bank129507 +129508 POINT(47.26171450980756 -122.3767947932453) bank129508 +129509 POINT(47.121251566944224 -122.6221640170005) bank129509 +129510 POINT(46.950177416846365 -121.79392674062687) bank129510 +129511 POINT(47.3175878279221 -122.1795888237572) bank129511 +129512 POINT(47.41669735271174 -123.2233153780211) bank129512 +129513 POINT(47.95460534707236 -123.0816067519902) bank129513 +129514 POINT(47.93166985312626 -121.55003733021906) bank129514 +129515 POINT(47.96866941125842 -122.4259446345582) bank129515 +129516 POINT(47.1200557163345 -122.68569038805715) bank129516 +129517 POINT(47.277117554665494 -123.21526960743284) bank129517 +129518 POINT(47.81285846936854 -123.20317713666687) bank129518 +129519 POINT(48.49872165670687 -122.51768546981539) bank129519 +129520 POINT(47.56534562876233 -121.48684357368549) bank129520 +129521 POINT(47.735323081287234 -121.35364019335599) bank129521 +129522 POINT(48.2966812747888 -122.97872360014959) bank129522 +129523 POINT(47.37327720124501 -121.67189498048678) bank129523 +129524 POINT(47.43065797902338 -121.81137671070998) bank129524 +129525 POINT(47.15902570005297 -121.54097308683585) bank129525 +129526 POINT(47.91772244851707 -123.3282679719499) bank129526 +129527 POINT(47.71582425921872 -121.98265495626374) bank129527 +129528 POINT(47.980838521798944 -121.42309567577605) bank129528 +129529 POINT(46.782366636761054 -122.83462871370976) bank129529 +129530 POINT(47.38499558611196 -121.88639159430699) bank129530 +129531 POINT(47.09828768963456 -122.98035896879217) bank129531 +129532 POINT(47.88320279143209 -122.87387501746723) bank129532 +129533 POINT(47.972292091460254 -122.94766156787678) bank129533 +129534 POINT(47.64109395183013 -122.92989963031727) bank129534 +129535 POINT(48.02614454199947 -122.74641206112898) bank129535 +129536 POINT(48.2776991280936 -121.5694619508913) bank129536 +129537 POINT(48.00282306687744 -121.96050139079797) bank129537 +129538 POINT(48.39906139461799 -121.46139775210666) bank129538 +129539 POINT(47.1018831397971 -121.86017744110354) bank129539 +129540 POINT(47.83906492129827 -121.50691510105013) bank129540 +129541 POINT(46.91037579765306 -122.79865778182834) bank129541 +129542 POINT(47.79662993479462 -121.35310434882463) bank129542 +129543 POINT(46.66787301379439 -123.2486220970014) bank129543 +129544 POINT(47.384665737138405 -123.04025690512127) bank129544 +129545 POINT(48.18463560594145 -121.69465547496583) bank129545 +129546 POINT(47.59838414480503 -121.73972076669736) bank129546 +129547 POINT(48.47675087074522 -121.39393624634397) bank129547 +129548 POINT(47.880266870475765 -122.87106936315224) bank129548 +129549 POINT(48.233331686400945 -123.09145963239982) bank129549 +129550 POINT(47.973471536466825 -121.8409210289541) bank129550 +129551 POINT(47.009864270544355 -122.28440256807032) bank129551 +129552 POINT(46.66593585824027 -121.74857200676976) bank129552 +129553 POINT(47.744889963996435 -122.11875099606954) bank129553 +129554 POINT(48.57016138569038 -122.03723538920363) bank129554 +129555 POINT(47.17776168860844 -122.25106647729618) bank129555 +129556 POINT(46.99280193618174 -121.99869149970368) bank129556 +129557 POINT(47.30528335891725 -121.81521204767762) bank129557 +129558 POINT(48.10132371439283 -122.37598323954627) bank129558 +129559 POINT(47.59013180677768 -122.9778481665081) bank129559 +129560 POINT(48.59678066490071 -121.47573440652832) bank129560 +129561 POINT(46.89317642984045 -122.10328217304453) bank129561 +129562 POINT(48.44450396924894 -121.56008088276977) bank129562 +129563 POINT(46.79624054571988 -121.95189297634819) bank129563 +129564 POINT(48.290052062772276 -123.080837843886) bank129564 +129565 POINT(48.13578960529256 -123.3283467007772) bank129565 +129566 POINT(47.06509877143018 -123.307605600742) bank129566 +129567 POINT(47.01426044214555 -121.93196155565657) bank129567 +129568 POINT(48.34691668218383 -121.83336254669331) bank129568 +129569 POINT(47.33391298617743 -122.52005573878935) bank129569 +129570 POINT(46.72922885959787 -121.37468996788724) bank129570 +129571 POINT(47.6964523975319 -121.73241698313352) bank129571 +129572 POINT(47.03800839758723 -122.01030384503156) bank129572 +129573 POINT(48.129579339016104 -123.2276376990274) bank129573 +129574 POINT(48.343622521255526 -122.12445875869915) bank129574 +129575 POINT(47.90301590546644 -122.38288473415238) bank129575 +129576 POINT(47.15238248421552 -122.7655510087817) bank129576 +129577 POINT(47.775301989355874 -122.51190318716088) bank129577 +129578 POINT(48.28238590756017 -121.5546366011584) bank129578 +129579 POINT(47.12765863128887 -123.25680743719484) bank129579 +129580 POINT(48.302221300766526 -122.17377531645839) bank129580 +129581 POINT(47.563494128923075 -121.572282367859) bank129581 +129582 POINT(46.72800933491592 -123.1417090459063) bank129582 +129583 POINT(47.00130770594318 -122.3802175174497) bank129583 +129584 POINT(46.8194189033628 -121.85443215987237) bank129584 +129585 POINT(48.48753896944348 -122.60014856870029) bank129585 +129586 POINT(47.20369448256625 -122.76302876659518) bank129586 +129587 POINT(48.31429597896387 -121.95121815683474) bank129587 +129588 POINT(47.558082877900105 -122.62487487015744) bank129588 +129589 POINT(46.876548311296055 -121.84847234528961) bank129589 +129590 POINT(46.70710350702276 -122.9215345041824) bank129590 +129591 POINT(48.43290643206472 -121.72273300262117) bank129591 +129592 POINT(47.815429866275984 -122.68057547778065) bank129592 +129593 POINT(47.613555001051964 -122.73399715157727) bank129593 +129594 POINT(48.51842949770706 -122.06698804547432) bank129594 +129595 POINT(47.56524510789315 -122.30976090174407) bank129595 +129596 POINT(47.752689276890635 -123.07682419330575) bank129596 +129597 POINT(46.812722831274556 -122.09564384701041) bank129597 +129598 POINT(47.65546808709271 -122.77262333625737) bank129598 +129599 POINT(46.648449817461305 -121.69396253745946) bank129599 +129600 POINT(46.957257697994685 -122.97852690076132) bank129600 +129601 POINT(47.49699112106423 -121.68909633868792) bank129601 +129602 POINT(48.162511452118856 -121.95352543998202) bank129602 +129603 POINT(46.938188532734586 -121.95546843496943) bank129603 +129604 POINT(47.90477493732547 -122.7347698105167) bank129604 +129605 POINT(47.4106862908843 -121.67909415730945) bank129605 +129606 POINT(48.593428759203604 -122.3243371094892) bank129606 +129607 POINT(47.66545157545137 -122.657705854622) bank129607 +129608 POINT(46.7321207110525 -122.88907745686559) bank129608 +129609 POINT(47.704248581321785 -121.55851153178999) bank129609 +129610 POINT(46.64177238309402 -121.60545019186529) bank129610 +129611 POINT(47.99189363477064 -123.00169256508771) bank129611 +129612 POINT(47.80067975371445 -122.42647754895088) bank129612 +129613 POINT(46.92074943867493 -122.92924183424168) bank129613 +129614 POINT(48.59681798814236 -122.34839676312488) bank129614 +129615 POINT(46.867535124521275 -122.01767923804675) bank129615 +129616 POINT(47.52308388040913 -121.67374718146648) bank129616 +129617 POINT(47.54380767446405 -122.65444058319277) bank129617 +129618 POINT(47.32528952417463 -122.75917610231508) bank129618 +129619 POINT(47.84308397896211 -122.06948721516598) bank129619 +129620 POINT(47.09730009094152 -122.5863023923704) bank129620 +129621 POINT(47.03711739294403 -123.17080087435497) bank129621 +129622 POINT(48.38046215931105 -121.37632313274278) bank129622 +129623 POINT(46.805775644565465 -122.47130345462811) bank129623 +129624 POINT(46.87153807246788 -121.76788304916752) bank129624 +129625 POINT(47.12341970142645 -122.34316599364435) bank129625 +129626 POINT(48.30910442601951 -123.31973798443349) bank129626 +129627 POINT(46.67204007360468 -122.35321807378922) bank129627 +129628 POINT(46.803462481375036 -121.86212374162012) bank129628 +129629 POINT(48.04004879426553 -122.87439172429606) bank129629 +129630 POINT(47.578013613545004 -122.94653637048772) bank129630 +129631 POINT(46.915404536886506 -121.60688533721631) bank129631 +129632 POINT(47.52375977052275 -122.77697839795071) bank129632 +129633 POINT(47.432545351291026 -121.36758161890674) bank129633 +129634 POINT(47.98567158762412 -122.14262296073694) bank129634 +129635 POINT(47.14744849501016 -122.0398240253449) bank129635 +129636 POINT(47.099881165083076 -121.8521988616291) bank129636 +129637 POINT(48.325912736017294 -121.88403109865779) bank129637 +129638 POINT(46.70219010271485 -122.58646469025538) bank129638 +129639 POINT(48.4158332205912 -122.44697585483313) bank129639 +129640 POINT(46.79438389988157 -122.73226219746657) bank129640 +129641 POINT(48.34357049861123 -121.92787796108715) bank129641 +129642 POINT(48.1786715474347 -123.08185621980113) bank129642 +129643 POINT(48.60153289832143 -122.69319456681842) bank129643 +129644 POINT(48.00898375259223 -123.11359527601131) bank129644 +129645 POINT(47.56307463677128 -122.01400848086696) bank129645 +129646 POINT(47.77468642522634 -122.76834701682044) bank129646 +129647 POINT(47.84558626706258 -122.32014705016742) bank129647 +129648 POINT(47.48703841420082 -121.3955025024134) bank129648 +129649 POINT(48.02579195110061 -123.18317254748695) bank129649 +129650 POINT(47.54374186200353 -123.20389351677554) bank129650 +129651 POINT(46.607682098858106 -123.2999617845447) bank129651 +129652 POINT(47.543580857014454 -122.8591008343534) bank129652 +129653 POINT(48.1381441625372 -122.96911306340581) bank129653 +129654 POINT(46.75285167917741 -121.34708212734304) bank129654 +129655 POINT(48.07478701116713 -122.63901188965342) bank129655 +129656 POINT(48.01456072196166 -123.05963922009305) bank129656 +129657 POINT(48.229626625347315 -122.5350136582899) bank129657 +129658 POINT(48.00931218199362 -121.79867397737286) bank129658 +129659 POINT(48.37271709004885 -122.28509278536715) bank129659 +129660 POINT(46.82107200461864 -122.39700103568852) bank129660 +129661 POINT(47.691134854626924 -122.59200694395875) bank129661 +129662 POINT(47.38877343747782 -123.31382751640265) bank129662 +129663 POINT(47.50274779280213 -122.58495367743748) bank129663 +129664 POINT(47.826921917964114 -123.26370043145465) bank129664 +129665 POINT(47.586721465875144 -122.04954860595632) bank129665 +129666 POINT(47.8561216277004 -123.13543491161849) bank129666 +129667 POINT(48.35912428444039 -123.04442849267723) bank129667 +129668 POINT(48.49819364423385 -121.41848084901537) bank129668 +129669 POINT(48.3003374744527 -122.2087427601702) bank129669 +129670 POINT(47.96042918603096 -123.1866164071465) bank129670 +129671 POINT(47.077884959953046 -121.67610607438633) bank129671 +129672 POINT(48.4601971816267 -121.37418082114374) bank129672 +129673 POINT(47.85645191420276 -122.9271102949829) bank129673 +129674 POINT(46.8269366778638 -122.23285359824347) bank129674 +129675 POINT(47.11707376534547 -122.54441924585339) bank129675 +129676 POINT(47.73233482454487 -121.61491668722732) bank129676 +129677 POINT(46.94928714735482 -123.13003069329467) bank129677 +129678 POINT(46.804938765811976 -122.53271439677563) bank129678 +129679 POINT(47.612477819637085 -123.17739535595828) bank129679 +129680 POINT(47.651059035329695 -122.61194055537658) bank129680 +129681 POINT(48.41208826077917 -121.7882284467098) bank129681 +129682 POINT(47.408022651330405 -121.35853820216316) bank129682 +129683 POINT(47.70845146382561 -123.28394934515447) bank129683 +129684 POINT(47.590797741892295 -122.88969830107264) bank129684 +129685 POINT(47.66479754759295 -123.23779515871593) bank129685 +129686 POINT(47.52139009548455 -123.07780773028828) bank129686 +129687 POINT(47.290931806599744 -121.89977428447371) bank129687 +129688 POINT(46.84713790959368 -122.49519461136266) bank129688 +129689 POINT(47.61816275906816 -121.78475438379512) bank129689 +129690 POINT(47.31377533786894 -121.41045229369568) bank129690 +129691 POINT(46.85713616876215 -122.13986852286516) bank129691 +129692 POINT(47.75720044198406 -122.90099116416093) bank129692 +129693 POINT(48.204167248039866 -121.35032132138944) bank129693 +129694 POINT(47.00426782608502 -121.60277598003395) bank129694 +129695 POINT(48.38491766978818 -121.83508744497858) bank129695 +129696 POINT(47.28013937509048 -122.59785333348819) bank129696 +129697 POINT(48.2251719732587 -123.25737558359099) bank129697 +129698 POINT(47.86672708812091 -122.6725149836366) bank129698 +129699 POINT(48.46470788992955 -122.70245966298889) bank129699 +129700 POINT(47.77121313587335 -123.28892957086747) bank129700 +129701 POINT(46.98324776807053 -122.34587997531403) bank129701 +129702 POINT(48.170612520405314 -121.66505080016906) bank129702 +129703 POINT(47.48121161256923 -123.00538253893033) bank129703 +129704 POINT(46.89543569003578 -122.53765508002597) bank129704 +129705 POINT(48.51502088208032 -121.4265970526566) bank129705 +129706 POINT(47.907807352144246 -121.5274624783067) bank129706 +129707 POINT(47.42972375348031 -123.19084831627583) bank129707 +129708 POINT(47.422746379638824 -122.95659005772579) bank129708 +129709 POINT(47.92685517947693 -122.07745604549969) bank129709 +129710 POINT(48.44928342366481 -122.61144521469454) bank129710 +129711 POINT(46.885196338311665 -123.13011459186473) bank129711 +129712 POINT(47.80107376881145 -122.4658079858899) bank129712 +129713 POINT(46.81484298889991 -121.9992548613699) bank129713 +129714 POINT(46.794651915187465 -121.94404191403521) bank129714 +129715 POINT(46.77066605422956 -121.34262888450816) bank129715 +129716 POINT(48.228461115934216 -122.56331625947365) bank129716 +129717 POINT(48.31403327452788 -122.2868587323819) bank129717 +129718 POINT(47.16050436621595 -122.962072536441) bank129718 +129719 POINT(46.67901977824574 -123.19449967633004) bank129719 +129720 POINT(48.15760519260124 -122.03460139460677) bank129720 +129721 POINT(47.851880770073905 -122.95010265719408) bank129721 +129722 POINT(47.34223552651879 -121.93101007331866) bank129722 +129723 POINT(48.308770497283035 -121.68039884831269) bank129723 +129724 POINT(48.01634739779841 -121.75514819612981) bank129724 +129725 POINT(48.22333486089221 -122.78400971198865) bank129725 +129726 POINT(48.07574738200284 -122.59243436674078) bank129726 +129727 POINT(46.975141438837916 -122.47874262535312) bank129727 +129728 POINT(47.75817388816945 -121.87880113694128) bank129728 +129729 POINT(46.95028306601897 -122.02458953437632) bank129729 +129730 POINT(47.240640595035515 -121.76488745102303) bank129730 +129731 POINT(47.23724710260013 -122.82158507651707) bank129731 +129732 POINT(47.505012794965204 -122.07529281788823) bank129732 +129733 POINT(47.988322106462284 -123.01314737998749) bank129733 +129734 POINT(47.16622265826879 -122.01030409117284) bank129734 +129735 POINT(46.907814368187196 -121.99009028221529) bank129735 +129736 POINT(48.284254228961345 -123.12238708809397) bank129736 +129737 POINT(47.35772280528729 -121.50795157760197) bank129737 +129738 POINT(48.13877811132592 -121.65022241650094) bank129738 +129739 POINT(47.93275551900063 -121.72533489140321) bank129739 +129740 POINT(47.43949217472023 -123.05005785515611) bank129740 +129741 POINT(47.55411396394085 -122.11939003674132) bank129741 +129742 POINT(48.30986134333518 -121.84048817159497) bank129742 +129743 POINT(48.57102255823937 -123.11916898033758) bank129743 +129744 POINT(47.778230447602205 -122.55552307942322) bank129744 +129745 POINT(46.730569485573426 -121.97891337908585) bank129745 +129746 POINT(47.15339059198985 -121.58208260280293) bank129746 +129747 POINT(48.18055749523638 -122.87133056322173) bank129747 +129748 POINT(47.25178552655166 -123.09193815694405) bank129748 +129749 POINT(47.02104596283737 -121.39132296426894) bank129749 +129750 POINT(48.41896699559216 -121.76901300974151) bank129750 +129751 POINT(48.43978334996706 -122.02530295700744) bank129751 +129752 POINT(48.01519238582721 -122.97269893664249) bank129752 +129753 POINT(47.972049632072945 -121.55710261764727) bank129753 +129754 POINT(46.97408009114765 -122.83894904143098) bank129754 +129755 POINT(48.130741806371155 -121.42709607232264) bank129755 +129756 POINT(47.94463246997489 -122.36731788736817) bank129756 +129757 POINT(47.072862221210116 -123.09414485799117) bank129757 +129758 POINT(47.355915011634316 -122.64683322684324) bank129758 +129759 POINT(48.347701494419866 -122.18865475174938) bank129759 +129760 POINT(47.58491802834602 -121.59155695381513) bank129760 +129761 POINT(47.206427927161215 -121.39268270074005) bank129761 +129762 POINT(47.13137665914913 -122.10075335667379) bank129762 +129763 POINT(47.67549052350882 -121.89528689893226) bank129763 +129764 POINT(46.97608264139814 -121.97007907099655) bank129764 +129765 POINT(47.76525217086329 -121.36967544760937) bank129765 +129766 POINT(46.79297695521773 -122.6129883254684) bank129766 +129767 POINT(48.35029561495527 -121.83062899520458) bank129767 +129768 POINT(47.884772693026186 -122.6939660334649) bank129768 +129769 POINT(48.56744085235199 -122.83582494696185) bank129769 +129770 POINT(46.68556331373113 -122.83155779466152) bank129770 +129771 POINT(47.77583062598036 -122.101721999472) bank129771 +129772 POINT(48.05655718013348 -123.05755391800803) bank129772 +129773 POINT(47.762272827327436 -122.51710528320976) bank129773 +129774 POINT(46.647296359635504 -123.02113857406964) bank129774 +129775 POINT(48.28439819772075 -121.7432532330365) bank129775 +129776 POINT(48.15554908059429 -121.34997022406704) bank129776 +129777 POINT(46.714055237445365 -122.76915009614392) bank129777 +129778 POINT(47.27136288456379 -121.71858102914251) bank129778 +129779 POINT(47.39410483760353 -121.37256953913828) bank129779 +129780 POINT(47.314793103294306 -122.17316005749689) bank129780 +129781 POINT(46.76863098576459 -123.10209421672228) bank129781 +129782 POINT(47.363050459361304 -122.4194111658406) bank129782 +129783 POINT(47.38558360342137 -121.90168497333444) bank129783 +129784 POINT(47.118282800460705 -121.77761869870065) bank129784 +129785 POINT(48.25999104516973 -122.27234171047657) bank129785 +129786 POINT(47.145159195311976 -122.18872230916666) bank129786 +129787 POINT(47.21230105852182 -122.5346152474515) bank129787 +129788 POINT(47.85371530371088 -122.88917332415544) bank129788 +129789 POINT(46.78229574420011 -122.02032417877751) bank129789 +129790 POINT(46.8997846076243 -123.31938649286414) bank129790 +129791 POINT(46.94217690532829 -122.87584910357637) bank129791 +129792 POINT(46.79922553606435 -123.14993781667921) bank129792 +129793 POINT(47.6469823705395 -122.27298062500701) bank129793 +129794 POINT(47.52632198610398 -122.56408280930258) bank129794 +129795 POINT(47.004429011746346 -121.84886938876737) bank129795 +129796 POINT(47.83363570438423 -121.59357882049574) bank129796 +129797 POINT(47.66524768523779 -122.41043624356931) bank129797 +129798 POINT(48.48812252901372 -122.33890312232721) bank129798 +129799 POINT(48.045053999218794 -122.4339718712842) bank129799 +129800 POINT(47.66022897388343 -122.31728626025328) bank129800 +129801 POINT(48.55922099626745 -121.37318674565317) bank129801 +129802 POINT(48.233676627624774 -122.47892163214519) bank129802 +129803 POINT(46.86862268052588 -122.00783496154389) bank129803 +129804 POINT(47.96359072544273 -122.57277707481065) bank129804 +129805 POINT(47.39661111523353 -122.26488043322549) bank129805 +129806 POINT(46.61747553529926 -122.29836303721363) bank129806 +129807 POINT(47.075027657979355 -121.58944889646523) bank129807 +129808 POINT(47.26184651064823 -121.64512022451348) bank129808 +129809 POINT(47.85578886104716 -122.63741427735646) bank129809 +129810 POINT(47.87752958945937 -123.08488611687348) bank129810 +129811 POINT(47.387173440287604 -122.94596752633011) bank129811 +129812 POINT(47.60942440168247 -121.54363467503885) bank129812 +129813 POINT(48.32892983547277 -121.72777684764414) bank129813 +129814 POINT(48.00371244980602 -123.26294994831143) bank129814 +129815 POINT(48.209571484325 -121.60813354243082) bank129815 +129816 POINT(47.83862351881517 -123.21260899498341) bank129816 +129817 POINT(46.66011727016055 -121.85465440061205) bank129817 +129818 POINT(47.19775121832515 -122.38250642635926) bank129818 +129819 POINT(47.08530838054733 -122.73797574163648) bank129819 +129820 POINT(47.26871120080701 -122.85350359657895) bank129820 +129821 POINT(48.15064091411021 -122.69225090279625) bank129821 +129822 POINT(46.9453577766425 -123.12492469719493) bank129822 +129823 POINT(47.5583816941876 -122.66797619614411) bank129823 +129824 POINT(46.614294832223635 -122.06421865624156) bank129824 +129825 POINT(47.18749022549987 -121.64249544606201) bank129825 +129826 POINT(47.85946521642621 -121.5225552342732) bank129826 +129827 POINT(46.73470896387145 -122.21786154339254) bank129827 +129828 POINT(46.635868963694975 -121.63400596613667) bank129828 +129829 POINT(48.150739101838695 -122.40829301948204) bank129829 +129830 POINT(47.207739185205064 -121.97364738795993) bank129830 +129831 POINT(47.50265916535554 -122.036719688044) bank129831 +129832 POINT(46.728713341197306 -121.96828927043134) bank129832 +129833 POINT(48.569587939383894 -122.56865790424854) bank129833 +129834 POINT(47.569018064134355 -121.38344295312923) bank129834 +129835 POINT(48.390656844541766 -121.73146840502999) bank129835 +129836 POINT(47.05677906689565 -122.59840238288227) bank129836 +129837 POINT(47.14425241014747 -122.69046879273881) bank129837 +129838 POINT(48.08148480689304 -123.06318093329739) bank129838 +129839 POINT(46.71963171397461 -121.39763690954999) bank129839 +129840 POINT(46.99310489752723 -122.57689389475505) bank129840 +129841 POINT(46.75350422776108 -122.20754956784421) bank129841 +129842 POINT(47.2334169223704 -121.81628708640848) bank129842 +129843 POINT(48.21910673643105 -121.72698074816562) bank129843 +129844 POINT(48.58022848531949 -121.39863317262352) bank129844 +129845 POINT(47.817890176406664 -123.17162752903414) bank129845 +129846 POINT(48.56027200238902 -121.81869989763453) bank129846 +129847 POINT(48.480151771536214 -121.61834676084473) bank129847 +129848 POINT(48.56266722954691 -121.58086101308716) bank129848 +129849 POINT(46.692052647604704 -121.53743769026254) bank129849 +129850 POINT(48.020045958765294 -121.70846930113385) bank129850 +129851 POINT(47.8636570755177 -121.48584273678618) bank129851 +129852 POINT(47.37451913835881 -122.38307372028707) bank129852 +129853 POINT(47.454264879553875 -122.59164904545554) bank129853 +129854 POINT(48.31006219767191 -122.83343505894399) bank129854 +129855 POINT(47.729110215072424 -121.39798213091808) bank129855 +129856 POINT(47.88190222508386 -121.35881692973065) bank129856 +129857 POINT(47.42070501035406 -123.13052766289532) bank129857 +129858 POINT(48.4222394351267 -122.77512262152958) bank129858 +129859 POINT(47.63148471617685 -123.15253518947685) bank129859 +129860 POINT(47.25657240428852 -123.054831767438) bank129860 +129861 POINT(47.39269046551479 -123.05514904772583) bank129861 +129862 POINT(47.12051012684176 -122.17452760209949) bank129862 +129863 POINT(47.91449663545711 -122.65737854826281) bank129863 +129864 POINT(47.62600367348705 -122.12845001154056) bank129864 +129865 POINT(47.24329515029857 -122.41960397089267) bank129865 +129866 POINT(48.33913150246696 -122.32391760059033) bank129866 +129867 POINT(47.9145057282586 -123.1817099266768) bank129867 +129868 POINT(48.41463205878205 -122.7524806265329) bank129868 +129869 POINT(47.23369014747613 -121.55298581339592) bank129869 +129870 POINT(46.711713953931906 -121.71553291051839) bank129870 +129871 POINT(46.70521507605365 -123.18023165138052) bank129871 +129872 POINT(47.86212662430933 -121.47964058434584) bank129872 +129873 POINT(48.28167971684794 -122.63992858807973) bank129873 +129874 POINT(47.84028147820864 -121.80755911301907) bank129874 +129875 POINT(48.007403501953554 -122.97545247890967) bank129875 +129876 POINT(48.50626347586781 -123.06839445089656) bank129876 +129877 POINT(48.56791158009898 -121.89068731578728) bank129877 +129878 POINT(46.99402243292623 -123.32885099105131) bank129878 +129879 POINT(47.82437159326418 -123.18328120356783) bank129879 +129880 POINT(47.205363741361545 -122.12158460617795) bank129880 +129881 POINT(47.787852686714984 -122.61150587941722) bank129881 +129882 POINT(46.838632124028386 -122.63435174070516) bank129882 +129883 POINT(47.58482005808111 -122.3488457188632) bank129883 +129884 POINT(48.10008123967521 -123.14988354818553) bank129884 +129885 POINT(47.737166841683916 -122.2065906320836) bank129885 +129886 POINT(48.298325872496015 -121.94684824925609) bank129886 +129887 POINT(48.568158257452396 -122.9710336550776) bank129887 +129888 POINT(47.356121076513226 -121.51869375604029) bank129888 +129889 POINT(47.20323650341113 -122.89695132705785) bank129889 +129890 POINT(47.48298930538287 -122.3613673916037) bank129890 +129891 POINT(47.62565590374875 -123.31216938217575) bank129891 +129892 POINT(47.152603307809464 -123.09877106019992) bank129892 +129893 POINT(47.853716236396465 -121.78578638159429) bank129893 +129894 POINT(46.83514883970046 -122.16149848319381) bank129894 +129895 POINT(47.261306465002185 -121.92364018139865) bank129895 +129896 POINT(47.94199403929523 -122.80291025032925) bank129896 +129897 POINT(46.93772976312348 -121.66407077360196) bank129897 +129898 POINT(47.76209531020776 -123.24174853028181) bank129898 +129899 POINT(47.55091701664921 -123.12633297898277) bank129899 +129900 POINT(47.913447623088295 -121.74005293234534) bank129900 +129901 POINT(46.87805988866407 -121.9812593205779) bank129901 +129902 POINT(48.59946896540055 -123.31280569477495) bank129902 +129903 POINT(47.13095798484598 -123.28353243531122) bank129903 +129904 POINT(47.67628993760805 -122.66432145205923) bank129904 +129905 POINT(48.07700945793446 -122.86428369122073) bank129905 +129906 POINT(48.37684400936931 -122.41598389182117) bank129906 +129907 POINT(48.03833382810544 -122.94524797464952) bank129907 +129908 POINT(48.29880822548967 -121.79246990641445) bank129908 +129909 POINT(48.00786066926063 -123.11744783110262) bank129909 +129910 POINT(47.71763126748276 -121.71514039199384) bank129910 +129911 POINT(48.59647022956733 -122.64411394836263) bank129911 +129912 POINT(48.57982723226341 -122.24451851210156) bank129912 +129913 POINT(48.18555376396942 -121.37442839812404) bank129913 +129914 POINT(46.8948303651178 -121.88668410110056) bank129914 +129915 POINT(47.33376373698629 -121.72869740322732) bank129915 +129916 POINT(48.53205092439531 -121.83179561326813) bank129916 +129917 POINT(46.956155053319364 -121.62859782019275) bank129917 +129918 POINT(48.5499877522466 -122.5923370007787) bank129918 +129919 POINT(48.091539007403384 -121.65646753463595) bank129919 +129920 POINT(47.20467772018744 -121.80036114402039) bank129920 +129921 POINT(47.06196381973032 -122.16692569749118) bank129921 +129922 POINT(48.0981310983254 -121.75157772269402) bank129922 +129923 POINT(47.57584295704343 -123.2383229346141) bank129923 +129924 POINT(47.24226149217442 -123.27696440307145) bank129924 +129925 POINT(47.16367144173448 -122.9258587589641) bank129925 +129926 POINT(47.47052447128758 -122.27426202402165) bank129926 +129927 POINT(47.40682554931885 -121.53736142758677) bank129927 +129928 POINT(47.26081332563518 -121.69073534181081) bank129928 +129929 POINT(47.086714252851394 -123.07369330212988) bank129929 +129930 POINT(46.819231632148366 -121.81903896134425) bank129930 +129931 POINT(47.10345611279709 -122.88659056248457) bank129931 +129932 POINT(47.88802701165809 -121.85338226443567) bank129932 +129933 POINT(48.411791647483305 -122.8275235693306) bank129933 +129934 POINT(47.86208797468262 -122.5697577976869) bank129934 +129935 POINT(47.71853121242446 -123.10556238749237) bank129935 +129936 POINT(48.04612919549598 -122.40959137223659) bank129936 +129937 POINT(46.88029645237841 -122.54843435307805) bank129937 +129938 POINT(47.26246801073715 -122.01647067447837) bank129938 +129939 POINT(47.50314236975118 -121.72587499439867) bank129939 +129940 POINT(47.77560968411054 -121.81726709748177) bank129940 +129941 POINT(47.1883883535594 -122.7079331710505) bank129941 +129942 POINT(46.82444738723944 -122.86346218920497) bank129942 +129943 POINT(46.63029203067889 -121.69834256344873) bank129943 +129944 POINT(47.41159529146387 -122.22116260647533) bank129944 +129945 POINT(48.4471606789986 -122.54470188879861) bank129945 +129946 POINT(47.90828368365112 -121.9411950485734) bank129946 +129947 POINT(48.41331534614612 -122.56004349904384) bank129947 +129948 POINT(46.90525369885743 -121.45698116526863) bank129948 +129949 POINT(47.75191807853049 -122.73137353619063) bank129949 +129950 POINT(47.17496214677891 -121.40333368106633) bank129950 +129951 POINT(48.300883995236035 -122.50032774521763) bank129951 +129952 POINT(48.40705909569626 -122.40842045737) bank129952 +129953 POINT(48.43601448598365 -122.19600593757774) bank129953 +129954 POINT(47.465711863653475 -122.83635602999514) bank129954 +129955 POINT(48.59034531312353 -121.7390256441722) bank129955 +129956 POINT(47.052008161258854 -122.5627251074669) bank129956 +129957 POINT(46.60985071060505 -122.84826192536417) bank129957 +129958 POINT(46.704991639448856 -123.18070704495283) bank129958 +129959 POINT(46.79811701138582 -121.99501797494825) bank129959 +129960 POINT(46.75706924019582 -122.04905109471382) bank129960 +129961 POINT(48.440950734673834 -121.92162717856259) bank129961 +129962 POINT(46.693669173221444 -122.49051458072735) bank129962 +129963 POINT(47.60677763228087 -122.05750627661907) bank129963 +129964 POINT(47.00032015880117 -122.30120273174532) bank129964 +129965 POINT(48.3505434912379 -122.27493433386424) bank129965 +129966 POINT(48.48403051045797 -121.47580811497019) bank129966 +129967 POINT(48.574252981337104 -122.95878353213753) bank129967 +129968 POINT(48.58499192043127 -121.83740824005045) bank129968 +129969 POINT(48.44754936295559 -121.94284997596267) bank129969 +129970 POINT(46.96345802006574 -121.99847866694758) bank129970 +129971 POINT(47.982747090641965 -122.18161734021096) bank129971 +129972 POINT(48.58224250015415 -121.38075115325981) bank129972 +129973 POINT(47.55572084267121 -122.10014220456394) bank129973 +129974 POINT(47.41749193347093 -121.66165771989262) bank129974 +129975 POINT(48.39584338060513 -122.48039719376013) bank129975 +129976 POINT(47.82356158106464 -121.75711230281281) bank129976 +129977 POINT(47.28500697022105 -121.44714573124439) bank129977 +129978 POINT(47.660145227933334 -121.54533145137715) bank129978 +129979 POINT(46.99452745347802 -121.90077925711888) bank129979 +129980 POINT(48.59648881611878 -121.7945001866364) bank129980 +129981 POINT(48.214745916388026 -122.70081796819233) bank129981 +129982 POINT(46.838436421911425 -123.31618896095311) bank129982 +129983 POINT(48.46214680388144 -121.60306575413234) bank129983 +129984 POINT(46.76756190985075 -122.46011143036334) bank129984 +129985 POINT(47.890797048120255 -121.91471446793277) bank129985 +129986 POINT(48.55591300090354 -123.26533759479267) bank129986 +129987 POINT(47.26696541595285 -121.61965476622699) bank129987 +129988 POINT(46.962184016560556 -122.20216729177858) bank129988 +129989 POINT(48.48497081194991 -122.3893362853122) bank129989 +129990 POINT(47.19712958913837 -122.90131171846124) bank129990 +129991 POINT(48.3419244267735 -122.42351228689938) bank129991 +129992 POINT(47.95553530264771 -123.15918158970416) bank129992 +129993 POINT(47.11704433166482 -123.1246826765382) bank129993 +129994 POINT(47.953140633143974 -122.78008593934337) bank129994 +129995 POINT(46.879837494579085 -122.37043453453819) bank129995 +129996 POINT(47.70183729131174 -122.01267742560088) bank129996 +129997 POINT(48.18408500814513 -122.50196671265103) bank129997 +129998 POINT(47.069606446469336 -122.12223460229293) bank129998 +129999 POINT(46.912896196298554 -122.53904059992702) bank129999 +130000 POINT(48.3175048178696 -121.68015117987551) bank130000 +130001 POINT(47.151402738070715 -122.88509895321927) bank130001 +130002 POINT(47.14302355983116 -123.10574339536124) bank130002 +130003 POINT(48.490602799421424 -122.31655937391845) bank130003 +130004 POINT(47.2508548767796 -122.13744331977199) bank130004 +130005 POINT(47.27979843948631 -122.0595539009373) bank130005 +130006 POINT(47.32368880534052 -123.17079176930166) bank130006 +130007 POINT(48.23462696541961 -122.26999877785292) bank130007 +130008 POINT(47.182662647872206 -122.4578324006797) bank130008 +130009 POINT(47.95080587653185 -122.28666080375663) bank130009 +130010 POINT(48.44936278802568 -122.40388568345391) bank130010 +130011 POINT(46.73357597059378 -122.48139584942865) bank130011 +130012 POINT(48.328593640928844 -123.3252516936721) bank130012 +130013 POINT(48.44260159523064 -123.22449077531857) bank130013 +130014 POINT(47.749415107472686 -121.95444577050144) bank130014 +130015 POINT(47.10842034709697 -122.33172366842427) bank130015 +130016 POINT(46.92017206147054 -122.36779741489099) bank130016 +130017 POINT(47.483536603634725 -121.43897275923433) bank130017 +130018 POINT(48.404339670983546 -122.89641354046185) bank130018 +130019 POINT(47.76499597950417 -122.73124721514968) bank130019 +130020 POINT(48.180450124098165 -121.50334301100963) bank130020 +130021 POINT(48.023784492819296 -122.65218269907334) bank130021 +130022 POINT(48.143119514134526 -121.51054104383462) bank130022 +130023 POINT(48.179958412707165 -123.32342117748729) bank130023 +130024 POINT(47.3485950776215 -121.97813397799504) bank130024 +130025 POINT(47.20336996637842 -121.75536883581881) bank130025 +130026 POINT(47.92406863481371 -122.41317876573119) bank130026 +130027 POINT(47.42857125269279 -121.92786619488858) bank130027 +130028 POINT(47.85645180145993 -123.15730115503038) bank130028 +130029 POINT(47.96704117004918 -122.79475711036544) bank130029 +130030 POINT(48.27892798141733 -122.40556778084341) bank130030 +130031 POINT(47.9201534651973 -122.09923303570797) bank130031 +130032 POINT(47.05574267989396 -122.48190692876801) bank130032 +130033 POINT(47.855987112080555 -121.53399470454535) bank130033 +130034 POINT(48.6040523651278 -121.59288262620596) bank130034 +130035 POINT(46.84933840994569 -121.875917454477) bank130035 +130036 POINT(47.85554166135182 -122.92639149215904) bank130036 +130037 POINT(47.547714598652526 -122.3328648083163) bank130037 +130038 POINT(47.183842942117394 -123.11271679418283) bank130038 +130039 POINT(48.19849950608997 -121.40554187825083) bank130039 +130040 POINT(47.695458757981314 -122.73880736265244) bank130040 +130041 POINT(47.63391170028818 -121.94900660467515) bank130041 +130042 POINT(47.72458225042705 -121.8515081240173) bank130042 +130043 POINT(47.497338098503974 -122.41090055965634) bank130043 +130044 POINT(48.10068265250391 -121.39098709933155) bank130044 +130045 POINT(47.95151641596189 -121.81685358660073) bank130045 +130046 POINT(47.057236814497685 -122.22292605390945) bank130046 +130047 POINT(46.81762839855923 -121.54983727385371) bank130047 +130048 POINT(46.93453181221497 -122.29281983839851) bank130048 +130049 POINT(47.37644924226076 -121.78037583808431) bank130049 +130050 POINT(46.64602336922096 -121.83614989924135) bank130050 +130051 POINT(47.11616119277476 -121.630564652137) bank130051 +130052 POINT(46.94775514175115 -122.66144385153667) bank130052 +130053 POINT(47.06294050113548 -123.18026220354628) bank130053 +130054 POINT(47.294019728381784 -122.65903493904455) bank130054 +130055 POINT(48.114585212062316 -122.11500840927529) bank130055 +130056 POINT(46.87738677912132 -121.51914915267835) bank130056 +130057 POINT(46.997309336918036 -121.84162642287136) bank130057 +130058 POINT(48.36386933292501 -122.12194105294041) bank130058 +130059 POINT(46.624380609132594 -122.63404162562995) bank130059 +130060 POINT(47.4309576909451 -123.19071021932108) bank130060 +130061 POINT(48.49709512631824 -123.20528318251716) bank130061 +130062 POINT(47.565102430845656 -122.61707469083223) bank130062 +130063 POINT(48.46650575512971 -122.02816988186842) bank130063 +130064 POINT(46.87540681174747 -121.8289115228824) bank130064 +130065 POINT(48.45524998917686 -121.87883879258428) bank130065 +130066 POINT(47.66926360706431 -123.00212257094371) bank130066 +130067 POINT(47.19219732683208 -122.03192384228906) bank130067 +130068 POINT(46.8409101801899 -122.11202947002639) bank130068 +130069 POINT(47.61646398987543 -122.47302954290859) bank130069 +130070 POINT(47.78483882852865 -122.15966953169728) bank130070 +130071 POINT(48.12269586644885 -121.7772692196166) bank130071 +130072 POINT(46.984267331441586 -122.15037165389285) bank130072 +130073 POINT(47.70766295510926 -121.97599371885343) bank130073 +130074 POINT(47.739595919079385 -121.98666652893935) bank130074 +130075 POINT(48.60151803339556 -121.93805502979042) bank130075 +130076 POINT(47.899925150056866 -122.96155146800253) bank130076 +130077 POINT(47.57595356596778 -122.08964847909577) bank130077 +130078 POINT(46.681875473069084 -121.55518282783433) bank130078 +130079 POINT(46.91972583027171 -121.45084063600086) bank130079 +130080 POINT(48.49586999839735 -122.91175352477882) bank130080 +130081 POINT(47.7171645761942 -121.36973086868183) bank130081 +130082 POINT(46.66317176447112 -122.28881058238451) bank130082 +130083 POINT(48.37173993593248 -122.95166961376101) bank130083 +130084 POINT(46.81940748881498 -122.5743460428672) bank130084 +130085 POINT(46.62278763172043 -123.06762866587131) bank130085 +130086 POINT(46.874191211227796 -122.04410462252719) bank130086 +130087 POINT(48.067018044540376 -123.04336807863305) bank130087 +130088 POINT(47.519497856303445 -121.86617093738636) bank130088 +130089 POINT(46.69415669785614 -121.93542244941061) bank130089 +130090 POINT(46.62980217670888 -122.48537711276799) bank130090 +130091 POINT(46.8536192113691 -122.64810703199038) bank130091 +130092 POINT(46.92210761693254 -121.46637849263217) bank130092 +130093 POINT(47.812609782543184 -123.13214890359848) bank130093 +130094 POINT(48.236147776149885 -122.00694578929199) bank130094 +130095 POINT(47.09167066422498 -123.16400712722623) bank130095 +130096 POINT(47.22508441309047 -123.22343110351991) bank130096 +130097 POINT(48.29721398331913 -121.4188360530391) bank130097 +130098 POINT(48.414225582671925 -122.84012731335314) bank130098 +130099 POINT(47.23962056956718 -121.99912902097687) bank130099 +130100 POINT(47.402231479334496 -122.78263219488977) bank130100 +130101 POINT(48.058681740883614 -121.36023489010209) bank130101 +130102 POINT(47.30725519563777 -122.3261844016783) bank130102 +130103 POINT(46.96738122364889 -122.88241990658312) bank130103 +130104 POINT(46.716278540800715 -122.13951169454037) bank130104 +130105 POINT(46.74030303061985 -121.53921667519758) bank130105 +130106 POINT(47.88998579152137 -121.4163227095236) bank130106 +130107 POINT(48.1860382437943 -121.57752400862219) bank130107 +130108 POINT(47.59896034056795 -123.27152631640796) bank130108 +130109 POINT(47.48968694120612 -122.59860195340126) bank130109 +130110 POINT(47.11526777567458 -122.38457134827564) bank130110 +130111 POINT(48.40506732384415 -123.28109330441063) bank130111 +130112 POINT(46.98541748270491 -122.70781781569822) bank130112 +130113 POINT(47.82024157343478 -123.16817047306668) bank130113 +130114 POINT(48.05630287813132 -122.23621101342005) bank130114 +130115 POINT(46.965077096386715 -122.96822327481092) bank130115 +130116 POINT(48.375273269557326 -121.51399034939017) bank130116 +130117 POINT(48.00321935265814 -123.31327602564997) bank130117 +130118 POINT(48.16450427288946 -122.97888877065526) bank130118 +130119 POINT(47.10401724245069 -121.63532144532412) bank130119 +130120 POINT(46.97518294232227 -122.42323200528784) bank130120 +130121 POINT(48.45817312525872 -122.14531416607646) bank130121 +130122 POINT(47.77669510879577 -122.43990250116745) bank130122 +130123 POINT(46.96427623768894 -123.26048046393798) bank130123 +130124 POINT(48.22151876703979 -121.75805878884525) bank130124 +130125 POINT(47.64105658465491 -122.6964346336789) bank130125 +130126 POINT(48.36350578311295 -122.7825696125602) bank130126 +130127 POINT(46.91551441682612 -122.11978111539972) bank130127 +130128 POINT(47.41883349092737 -122.02139710433842) bank130128 +130129 POINT(47.934613979088674 -122.07413806670611) bank130129 +130130 POINT(47.52545937162743 -122.62878939645097) bank130130 +130131 POINT(46.71404396274535 -122.50809280674395) bank130131 +130132 POINT(47.41896494374873 -122.18397760106245) bank130132 +130133 POINT(47.073357095907184 -122.24443281867855) bank130133 +130134 POINT(48.31409666407365 -121.86926551452022) bank130134 +130135 POINT(48.25380232890505 -121.5946697367373) bank130135 +130136 POINT(47.43233490421349 -122.98279816360844) bank130136 +130137 POINT(48.184428456097244 -121.81045963578715) bank130137 +130138 POINT(47.04750515068834 -122.88798061888221) bank130138 +130139 POINT(47.41820135239885 -121.47924952608214) bank130139 +130140 POINT(48.41562562017934 -122.99274687621784) bank130140 +130141 POINT(47.58618947126859 -123.0260394392522) bank130141 +130142 POINT(46.61113467044781 -121.5354129008084) bank130142 +130143 POINT(47.076259539910126 -121.48615666549534) bank130143 +130144 POINT(47.966145677651745 -121.68364098908116) bank130144 +130145 POINT(47.00094460068706 -123.26089663191185) bank130145 +130146 POINT(47.238831610009605 -121.6563990877408) bank130146 +130147 POINT(47.502757909944386 -121.3766546168602) bank130147 +130148 POINT(47.17608192366193 -122.14275592682469) bank130148 +130149 POINT(47.849387187759 -121.52153941725443) bank130149 +130150 POINT(46.660587112908715 -122.64155012104071) bank130150 +130151 POINT(48.10394257551332 -123.01153745783692) bank130151 +130152 POINT(46.63729230958997 -123.30961773247735) bank130152 +130153 POINT(47.06434861723468 -122.69767455426371) bank130153 +130154 POINT(46.67825982052181 -122.43050260361217) bank130154 +130155 POINT(48.00479253456434 -122.71412069659485) bank130155 +130156 POINT(48.03036875211503 -123.14779845148658) bank130156 +130157 POINT(48.416158492589524 -121.42575404034432) bank130157 +130158 POINT(47.34247478569418 -122.98141384211125) bank130158 +130159 POINT(47.723581414236094 -122.82936228360562) bank130159 +130160 POINT(46.74158825544688 -122.96402469442336) bank130160 +130161 POINT(46.75811967231409 -122.11733106923828) bank130161 +130162 POINT(46.889412199194574 -122.94431926164884) bank130162 +130163 POINT(48.052289054382584 -122.08900114366155) bank130163 +130164 POINT(47.94539194006202 -121.63567751334988) bank130164 +130165 POINT(47.29503119720756 -121.67864466904051) bank130165 +130166 POINT(47.97143224352872 -122.82399724137485) bank130166 +130167 POINT(47.70019524927441 -122.1797180836026) bank130167 +130168 POINT(46.90303996311064 -122.17850854626379) bank130168 +130169 POINT(48.51920749864291 -122.92245413518162) bank130169 +130170 POINT(47.86093256193516 -122.91984002883018) bank130170 +130171 POINT(47.39328992816789 -121.72051226886927) bank130171 +130172 POINT(48.430261598737815 -121.70000588972069) bank130172 +130173 POINT(48.11274911560489 -122.26211646885986) bank130173 +130174 POINT(47.212853434482525 -121.91351868631084) bank130174 +130175 POINT(47.716001307416086 -122.85465278467926) bank130175 +130176 POINT(47.024164982890206 -123.2935608554598) bank130176 +130177 POINT(48.45417000005913 -122.32568359865817) bank130177 +130178 POINT(47.18290760074709 -122.58911247850648) bank130178 +130179 POINT(48.23198808470496 -121.97172480470218) bank130179 +130180 POINT(48.36108915300075 -123.22845865730754) bank130180 +130181 POINT(47.2157758064031 -123.15058402433012) bank130181 +130182 POINT(47.79968690226786 -122.36976434128538) bank130182 +130183 POINT(46.71843178643449 -121.39090225235324) bank130183 +130184 POINT(46.80606090413245 -122.4773206019922) bank130184 +130185 POINT(47.76645392493983 -122.05055387215727) bank130185 +130186 POINT(47.70935820757603 -122.57457154902572) bank130186 +130187 POINT(47.42657155455156 -121.3452807146868) bank130187 +130188 POINT(48.336872356868135 -122.07813028257577) bank130188 +130189 POINT(47.52044514702549 -121.38783128102342) bank130189 +130190 POINT(47.96688315494175 -122.51530552285843) bank130190 +130191 POINT(48.10076613579443 -122.58762400653389) bank130191 +130192 POINT(48.05379712636106 -122.42715961918472) bank130192 +130193 POINT(46.677023026026596 -122.0621123278441) bank130193 +130194 POINT(46.75602804914112 -121.65167799276999) bank130194 +130195 POINT(47.57742319395277 -122.81064679594684) bank130195 +130196 POINT(46.83361301086379 -122.60538679506729) bank130196 +130197 POINT(48.490605535485386 -122.77100039190535) bank130197 +130198 POINT(47.50012726846774 -123.10200165598366) bank130198 +130199 POINT(47.444813445814475 -122.9660312184183) bank130199 +130200 POINT(47.282198146741834 -121.47098085240582) bank130200 +130201 POINT(47.746136985321975 -121.88451453664108) bank130201 +130202 POINT(47.87222562941722 -123.12768450743312) bank130202 +130203 POINT(47.774239248712156 -122.78190538267485) bank130203 +130204 POINT(48.13908053234694 -121.89463845182007) bank130204 +130205 POINT(48.45051032184806 -121.92890894391664) bank130205 +130206 POINT(47.744587883852894 -123.1308292962978) bank130206 +130207 POINT(47.63210556253504 -122.53836654767694) bank130207 +130208 POINT(48.540990025831256 -123.17152191118547) bank130208 +130209 POINT(48.34401167134045 -123.19894071360436) bank130209 +130210 POINT(48.51406086079737 -122.1817412269869) bank130210 +130211 POINT(46.920173341155305 -121.7036355018732) bank130211 +130212 POINT(47.23219805958581 -122.14739705539601) bank130212 +130213 POINT(47.82012333657706 -122.818861148751) bank130213 +130214 POINT(47.64003739228791 -121.95659180439162) bank130214 +130215 POINT(47.90557778827951 -123.0711260719666) bank130215 +130216 POINT(47.84637329074001 -121.54233460652699) bank130216 +130217 POINT(48.09539670205936 -122.52070248820228) bank130217 +130218 POINT(46.93891946128707 -121.81748359650034) bank130218 +130219 POINT(46.82461594812641 -122.49976542258757) bank130219 +130220 POINT(48.45428544975026 -123.24998028126909) bank130220 +130221 POINT(47.29597860971874 -122.03181574175932) bank130221 +130222 POINT(47.413010856748265 -122.89295653151954) bank130222 +130223 POINT(47.768159826133804 -121.81495899125713) bank130223 +130224 POINT(47.959816301511665 -121.62142000404351) bank130224 +130225 POINT(47.97658848856763 -122.95229222599784) bank130225 +130226 POINT(47.24332211357361 -123.0975278719988) bank130226 +130227 POINT(47.927689914009456 -121.72941178046945) bank130227 +130228 POINT(48.576044904515406 -123.01474894903316) bank130228 +130229 POINT(48.19764381436184 -122.21533239170037) bank130229 +130230 POINT(48.30009253879638 -122.83174866734923) bank130230 +130231 POINT(47.32783544955475 -121.9911530937552) bank130231 +130232 POINT(47.91808054761463 -122.7275643105594) bank130232 +130233 POINT(47.25604772475329 -123.31267715344265) bank130233 +130234 POINT(48.019866125731674 -121.40399365164113) bank130234 +130235 POINT(47.89536065289423 -122.81877182643078) bank130235 +130236 POINT(47.172841335799625 -122.84010336278423) bank130236 +130237 POINT(46.77534865473842 -121.43841280308989) bank130237 +130238 POINT(46.80117269692994 -123.14200688140798) bank130238 +130239 POINT(47.859477099274024 -121.98558376078341) bank130239 +130240 POINT(48.01535640466213 -123.20193991174166) bank130240 +130241 POINT(47.72593747459851 -121.90089176381596) bank130241 +130242 POINT(47.94469976176512 -121.79585296041945) bank130242 +130243 POINT(48.46780027867016 -121.82452090726953) bank130243 +130244 POINT(47.78499551255441 -123.25806477386195) bank130244 +130245 POINT(47.82654011666783 -121.66897107409132) bank130245 +130246 POINT(48.11044155462703 -122.94751149161442) bank130246 +130247 POINT(48.04202755109051 -121.71172370480532) bank130247 +130248 POINT(46.80855300541168 -122.0296310284729) bank130248 +130249 POINT(48.415270293310826 -121.9815748743788) bank130249 +130250 POINT(47.78934862083688 -121.84519570262894) bank130250 +130251 POINT(48.591138190775716 -121.89819295393319) bank130251 +130252 POINT(47.84146255351792 -123.24448564674928) bank130252 +130253 POINT(48.16169769691517 -122.09933813243336) bank130253 +130254 POINT(47.89181579245431 -122.57771999459364) bank130254 +130255 POINT(47.99979253791329 -122.40484646767999) bank130255 +130256 POINT(47.08002031085421 -122.58623201021864) bank130256 +130257 POINT(46.794199971416546 -122.88730989916594) bank130257 +130258 POINT(47.70705938096202 -122.16415553896059) bank130258 +130259 POINT(47.082785461796064 -123.18821543697658) bank130259 +130260 POINT(48.034039399280644 -122.88294938683762) bank130260 +130261 POINT(47.32533922333967 -122.45853454216157) bank130261 +130262 POINT(48.30748703577108 -122.42610531786988) bank130262 +130263 POINT(47.845818327900695 -122.76376825874837) bank130263 +130264 POINT(46.663082116454966 -122.19857018851172) bank130264 +130265 POINT(48.50163065570495 -122.33931674025797) bank130265 +130266 POINT(47.030023889364244 -122.86670540089236) bank130266 +130267 POINT(47.384076464711804 -121.55408073424587) bank130267 +130268 POINT(47.16440244436292 -121.90128597649733) bank130268 +130269 POINT(46.69818661078903 -121.9634411362947) bank130269 +130270 POINT(46.87549673699553 -122.94897588247196) bank130270 +130271 POINT(47.97244443481817 -122.77407194307965) bank130271 +130272 POINT(47.381817199765976 -121.99551076792777) bank130272 +130273 POINT(47.22417803406094 -122.46194555519605) bank130273 +130274 POINT(46.766229057992376 -122.10118740256831) bank130274 +130275 POINT(47.39975008791524 -122.606036667049) bank130275 +130276 POINT(48.53106322087694 -122.76200405174285) bank130276 +130277 POINT(48.426264101671215 -121.72407227178478) bank130277 +130278 POINT(48.58352510524722 -122.3411395057196) bank130278 +130279 POINT(47.74064714805934 -123.20654900043321) bank130279 +130280 POINT(48.034762552778545 -123.16672620359522) bank130280 +130281 POINT(47.931658544868895 -121.65240924109652) bank130281 +130282 POINT(47.1856529716827 -122.14623907695673) bank130282 +130283 POINT(48.23999445566118 -122.86353980066718) bank130283 +130284 POINT(47.948461103962785 -121.58621808014504) bank130284 +130285 POINT(46.994393153971494 -121.73327353345798) bank130285 +130286 POINT(47.59315502316335 -122.60372953304923) bank130286 +130287 POINT(48.396782571032176 -121.7824072194163) bank130287 +130288 POINT(47.266986724823276 -121.53161000635522) bank130288 +130289 POINT(47.28467305711206 -122.41650965827921) bank130289 +130290 POINT(47.13410577909359 -122.6942196432013) bank130290 +130291 POINT(48.191160383995445 -122.96156561259559) bank130291 +130292 POINT(48.32157895034049 -122.19665647748572) bank130292 +130293 POINT(46.69319892795171 -123.20173939177567) bank130293 +130294 POINT(47.46600307377722 -123.0718477339341) bank130294 +130295 POINT(46.657686660325176 -123.21662160297141) bank130295 +130296 POINT(46.73094870831186 -122.57552626820616) bank130296 +130297 POINT(48.56623134368994 -122.16149300475685) bank130297 +130298 POINT(47.25454857185496 -122.9864976167747) bank130298 +130299 POINT(47.64551177057615 -121.75357255055452) bank130299 +130300 POINT(48.20608050495762 -122.7345117841131) bank130300 +130301 POINT(48.10680449670077 -121.42334489800984) bank130301 +130302 POINT(47.76913386043293 -123.02735388593862) bank130302 +130303 POINT(48.206266544328926 -122.69977872589664) bank130303 +130304 POINT(48.338965256671024 -121.67314273792425) bank130304 +130305 POINT(47.123618537900974 -121.65523731005227) bank130305 +130306 POINT(47.924946724109546 -122.0786531204608) bank130306 +130307 POINT(48.44899882723841 -121.67253674958735) bank130307 +130308 POINT(47.6541026348053 -122.30720062475226) bank130308 +130309 POINT(47.114273489388744 -122.24476393525035) bank130309 +130310 POINT(46.76989990718648 -121.85168970746939) bank130310 +130311 POINT(46.96641417925732 -122.80849065761436) bank130311 +130312 POINT(47.85317631376728 -121.47970493526654) bank130312 +130313 POINT(48.474797311479634 -122.17188210020974) bank130313 +130314 POINT(46.71750771200764 -121.918452881225) bank130314 +130315 POINT(48.16562193950218 -121.58604781787136) bank130315 +130316 POINT(48.30064372618984 -123.19995203035317) bank130316 +130317 POINT(46.939344754086136 -123.24025272415615) bank130317 +130318 POINT(47.855242008210034 -121.81429086399906) bank130318 +130319 POINT(47.01236976069543 -121.33360116900823) bank130319 +130320 POINT(48.547119665062766 -123.13605802027243) bank130320 +130321 POINT(48.58092591910803 -121.5084511338756) bank130321 +130322 POINT(48.04977426016826 -122.90691179839624) bank130322 +130323 POINT(47.01977920386541 -122.73623089314681) bank130323 +130324 POINT(48.17374834841428 -121.50275558369492) bank130324 +130325 POINT(47.16146491092674 -122.00504660782612) bank130325 +130326 POINT(46.94349724300624 -121.75289390248552) bank130326 +130327 POINT(47.03474319210857 -122.32477996225623) bank130327 +130328 POINT(47.48355996932135 -122.49068360953973) bank130328 +130329 POINT(47.69342174254265 -122.23028900347074) bank130329 +130330 POINT(48.47485658674461 -121.74754343617182) bank130330 +130331 POINT(47.20508933941402 -122.5338891281358) bank130331 +130332 POINT(47.05231255481208 -123.329520962578) bank130332 +130333 POINT(47.01308476074067 -122.0453358090258) bank130333 +130334 POINT(47.11619633270754 -122.17014420105026) bank130334 +130335 POINT(47.947734115593484 -123.23655319147082) bank130335 +130336 POINT(47.591648181561524 -121.91747853275874) bank130336 +130337 POINT(48.21012678444361 -121.42660024801889) bank130337 +130338 POINT(47.845560273769095 -121.39169522545028) bank130338 +130339 POINT(47.56307692763774 -122.80581716351594) bank130339 +130340 POINT(47.46968880497108 -122.93128345625585) bank130340 +130341 POINT(48.00847875387327 -121.68697072498391) bank130341 +130342 POINT(47.01218533712598 -121.34862232504172) bank130342 +130343 POINT(47.96237373823348 -121.3552167690985) bank130343 +130344 POINT(47.72160803949461 -123.10680977062194) bank130344 +130345 POINT(47.97120239881423 -122.88937397429484) bank130345 +130346 POINT(47.67369670745565 -123.14290702253355) bank130346 +130347 POINT(47.85118330965131 -121.8856089341446) bank130347 +130348 POINT(47.04068833227076 -122.1525720064666) bank130348 +130349 POINT(47.69299752314894 -121.83872719453585) bank130349 +130350 POINT(47.33727214055391 -122.16899858324572) bank130350 +130351 POINT(48.028807859957745 -121.85009414470642) bank130351 +130352 POINT(48.248892905920265 -122.74916001272753) bank130352 +130353 POINT(46.854689468997854 -121.91337095013651) bank130353 +130354 POINT(47.02874058390931 -123.18487406804023) bank130354 +130355 POINT(47.81987426625806 -121.78650063203602) bank130355 +130356 POINT(48.39007835870855 -121.93355922921408) bank130356 +130357 POINT(48.10436752211842 -122.25273165557698) bank130357 +130358 POINT(48.19207018651512 -122.44100378397994) bank130358 +130359 POINT(48.01226609256792 -122.3456993792099) bank130359 +130360 POINT(48.38015831495518 -121.4791991766921) bank130360 +130361 POINT(47.58601248752018 -123.18085317985025) bank130361 +130362 POINT(47.038303329760886 -122.22274543765076) bank130362 +130363 POINT(48.32674283539821 -122.13701993490422) bank130363 +130364 POINT(47.550232277955054 -123.05799063461339) bank130364 +130365 POINT(46.77745651327181 -122.48459882085216) bank130365 +130366 POINT(47.893766089481225 -121.5855215450201) bank130366 +130367 POINT(48.06055591450945 -122.667460393168) bank130367 +130368 POINT(48.27883182660513 -122.9948820115603) bank130368 +130369 POINT(48.32468773045485 -121.99835911276286) bank130369 +130370 POINT(47.05203293547902 -123.2142922998429) bank130370 +130371 POINT(47.18647878034135 -123.13975848383178) bank130371 +130372 POINT(47.69940657681102 -121.83895623230008) bank130372 +130373 POINT(48.17931544839007 -121.33307152910092) bank130373 +130374 POINT(47.25306272200408 -122.59219759852118) bank130374 +130375 POINT(47.268805001825875 -121.50430939124463) bank130375 +130376 POINT(46.88447423014233 -123.1846236361637) bank130376 +130377 POINT(46.834756267134864 -121.54564446410858) bank130377 +130378 POINT(47.72400217902731 -123.2873338117038) bank130378 +130379 POINT(48.55873599246755 -122.84282798170436) bank130379 +130380 POINT(48.08235690042294 -122.82335288274462) bank130380 +130381 POINT(48.58069382216407 -121.79213073308651) bank130381 +130382 POINT(46.73143239151168 -122.69259369535321) bank130382 +130383 POINT(47.02660247156155 -123.04850785284334) bank130383 +130384 POINT(48.504643143924895 -123.08038197308022) bank130384 +130385 POINT(46.880350811356394 -121.82318760751657) bank130385 +130386 POINT(47.484746848204296 -122.34846468341982) bank130386 +130387 POINT(48.17409737906976 -123.07406766683903) bank130387 +130388 POINT(46.916050286773306 -122.43547258141648) bank130388 +130389 POINT(47.31853711450866 -121.92668606266476) bank130389 +130390 POINT(47.22743619091089 -122.10235750925052) bank130390 +130391 POINT(47.58082895020625 -121.36492712748736) bank130391 +130392 POINT(46.92364861819318 -122.00870403743971) bank130392 +130393 POINT(46.66806629988883 -122.50396350678763) bank130393 +130394 POINT(46.62481855206201 -122.94595145003397) bank130394 +130395 POINT(47.96407609548424 -122.14565642542374) bank130395 +130396 POINT(47.05291295262519 -123.03807378760968) bank130396 +130397 POINT(47.538506197862844 -123.02732983939298) bank130397 +130398 POINT(47.337767037669316 -123.21512535658424) bank130398 +130399 POINT(47.6290835056636 -123.03332840927035) bank130399 +130400 POINT(47.06641391029658 -123.13642238513415) bank130400 +130401 POINT(47.100278401247714 -123.18971479709232) bank130401 +130402 POINT(47.894445571684415 -123.21660736243032) bank130402 +130403 POINT(47.367615477249196 -121.84029556434747) bank130403 +130404 POINT(48.09494208768754 -123.17568357305056) bank130404 +130405 POINT(47.84880638246692 -123.17865779181237) bank130405 +130406 POINT(47.95369620792406 -123.28299962988355) bank130406 +130407 POINT(47.36613924086172 -121.58405205852164) bank130407 +130408 POINT(47.53633220233572 -122.69174920432293) bank130408 +130409 POINT(46.89740954498013 -121.59142606305865) bank130409 +130410 POINT(47.41217238234711 -121.9162606170586) bank130410 +130411 POINT(48.11897240177412 -121.3559577468704) bank130411 +130412 POINT(47.898639192600136 -123.08946272781728) bank130412 +130413 POINT(47.16300321341229 -122.37964328185085) bank130413 +130414 POINT(46.91016826238136 -121.73661718472626) bank130414 +130415 POINT(46.98847652082429 -122.70345447911403) bank130415 +130416 POINT(47.8362180844196 -122.74862325381201) bank130416 +130417 POINT(47.87426955541725 -122.0150110208254) bank130417 +130418 POINT(48.03243796057209 -121.87968639020805) bank130418 +130419 POINT(46.7925749821187 -122.72278145199158) bank130419 +130420 POINT(47.588633826203356 -122.68556440787255) bank130420 +130421 POINT(47.752777806395265 -122.93424745054665) bank130421 +130422 POINT(47.60231527897383 -123.08299918814672) bank130422 +130423 POINT(47.924888617673865 -122.06898799982733) bank130423 +130424 POINT(47.08921320372922 -122.9973041576229) bank130424 +130425 POINT(47.23646572654059 -122.62761928139001) bank130425 +130426 POINT(48.158989539705814 -122.14936608588405) bank130426 +130427 POINT(47.99581593119341 -121.81954805940802) bank130427 +130428 POINT(47.159329396250676 -122.21137055947143) bank130428 +130429 POINT(47.13380040796089 -122.40444047363319) bank130429 +130430 POINT(48.56937545306294 -123.004351055024) bank130430 +130431 POINT(48.10037946670077 -121.9601829014909) bank130431 +130432 POINT(47.11572197859743 -122.67480215271668) bank130432 +130433 POINT(47.38919176122507 -123.24366573244617) bank130433 +130434 POINT(48.26992648708336 -122.151364259705) bank130434 +130435 POINT(47.60070238096523 -123.30290690280364) bank130435 +130436 POINT(47.37605500411952 -121.94464511953647) bank130436 +130437 POINT(47.98671892889108 -121.59913685212109) bank130437 +130438 POINT(47.65457465708262 -122.5938716207155) bank130438 +130439 POINT(47.108445881977445 -123.22417938793652) bank130439 +130440 POINT(46.68704327406668 -122.69318883025876) bank130440 +130441 POINT(48.14590313006113 -121.81650674177011) bank130441 +130442 POINT(47.88462515713833 -122.85838829475288) bank130442 +130443 POINT(47.03147854724247 -121.47820006893761) bank130443 +130444 POINT(47.815950719683286 -121.61435113776906) bank130444 +130445 POINT(47.32870842805172 -122.98488009824878) bank130445 +130446 POINT(47.97492862052395 -121.57436805058582) bank130446 +130447 POINT(46.67983198554531 -122.084571842658) bank130447 +130448 POINT(46.677089480169016 -121.39829500524428) bank130448 +130449 POINT(48.13762564998528 -121.48674269938836) bank130449 +130450 POINT(48.09523644845161 -121.70687642442687) bank130450 +130451 POINT(48.58082967166743 -121.72274676783262) bank130451 +130452 POINT(48.13643571665042 -122.27090415622837) bank130452 +130453 POINT(48.38292322657638 -121.619886526398) bank130453 +130454 POINT(46.71287731626375 -121.73161701219453) bank130454 +130455 POINT(46.98937127510669 -122.0337963353424) bank130455 +130456 POINT(46.76423537607572 -122.22147724984691) bank130456 +130457 POINT(48.03964326587506 -122.15546486494546) bank130457 +130458 POINT(47.112351904687685 -122.49892971622675) bank130458 +130459 POINT(47.68622830298752 -121.49979172847213) bank130459 +130460 POINT(47.786005615956725 -122.44129053767952) bank130460 +130461 POINT(48.46058579858238 -123.25252243915565) bank130461 +130462 POINT(48.043988838986834 -121.79822608156479) bank130462 +130463 POINT(46.75397327856527 -121.92088276355649) bank130463 +130464 POINT(46.88165438416025 -122.37530992452464) bank130464 +130465 POINT(46.82488944416906 -122.66481071064769) bank130465 +130466 POINT(47.67847376147588 -121.65304576452974) bank130466 +130467 POINT(47.712859565664466 -121.36380048002022) bank130467 +130468 POINT(46.71468601870475 -123.0874882688501) bank130468 +130469 POINT(48.06018710829856 -121.56746905982189) bank130469 +130470 POINT(47.154840693678864 -121.40072091153544) bank130470 +130471 POINT(47.804627474482636 -121.97080167316288) bank130471 +130472 POINT(46.927556341733634 -123.14239320390575) bank130472 +130473 POINT(48.26415008939851 -122.01293573203705) bank130473 +130474 POINT(47.172235293380766 -121.99264320597605) bank130474 +130475 POINT(48.445427424514044 -123.01334972583864) bank130475 +130476 POINT(46.74116358343747 -123.1845715736713) bank130476 +130477 POINT(46.82557050623448 -122.14580416309292) bank130477 +130478 POINT(47.72776909076096 -121.40711882123925) bank130478 +130479 POINT(48.43718973856439 -121.4781336565377) bank130479 +130480 POINT(46.71091957102621 -122.40707432529753) bank130480 +130481 POINT(48.21136429885678 -121.71598718191917) bank130481 +130482 POINT(47.732304174631075 -122.26536921576043) bank130482 +130483 POINT(48.36518684910541 -122.54127101326614) bank130483 +130484 POINT(47.10496592117775 -123.25453345514055) bank130484 +130485 POINT(48.101859404221365 -121.99480360347569) bank130485 +130486 POINT(47.137409421558814 -123.27930866738377) bank130486 +130487 POINT(48.23715294344471 -122.21479845113386) bank130487 +130488 POINT(48.370156391510506 -121.89529112245502) bank130488 +130489 POINT(48.06613853239779 -122.52484672400516) bank130489 +130490 POINT(48.44871211950942 -121.93662696253676) bank130490 +130491 POINT(46.88611777678146 -123.18504303002614) bank130491 +130492 POINT(46.70987443352148 -122.09936192151586) bank130492 +130493 POINT(46.77413897196576 -121.5371649505296) bank130493 +130494 POINT(48.23126023298934 -122.30446606558229) bank130494 +130495 POINT(47.577796497168954 -122.43155477621775) bank130495 +130496 POINT(46.83649511279385 -123.0507814044974) bank130496 +130497 POINT(48.03055204666685 -122.0498069524505) bank130497 +130498 POINT(47.661926332666646 -122.02806707341811) bank130498 +130499 POINT(47.531527559067065 -122.69955923696793) bank130499 +130500 POINT(47.11712098417357 -123.16023793892742) bank130500 +130501 POINT(46.72940212899247 -122.85710735523828) bank130501 +130502 POINT(48.60162848336926 -121.37415805221976) bank130502 +130503 POINT(46.938964958463664 -121.96308491652387) bank130503 +130504 POINT(47.49906996701813 -122.36975804120459) bank130504 +130505 POINT(48.212504663106714 -122.60136955030129) bank130505 +130506 POINT(47.702094483841044 -121.70902062657093) bank130506 +130507 POINT(47.66736272512122 -122.92063207280944) bank130507 +130508 POINT(46.80614978693304 -121.8520679329669) bank130508 +130509 POINT(48.00363772290446 -121.7001639073865) bank130509 +130510 POINT(47.347266365072606 -122.96412031661608) bank130510 +130511 POINT(48.506554116452634 -121.91337231887707) bank130511 +130512 POINT(47.58300859673792 -121.40808769354507) bank130512 +130513 POINT(48.427453894086746 -121.39941026837371) bank130513 +130514 POINT(46.907612709105884 -122.44302659591098) bank130514 +130515 POINT(47.47374681044704 -122.62537281263933) bank130515 +130516 POINT(48.56702211303549 -122.654066345053) bank130516 +130517 POINT(48.32109804230965 -123.09665400867918) bank130517 +130518 POINT(46.63985175392387 -122.21449877482328) bank130518 +130519 POINT(48.39015934853458 -121.40084831863659) bank130519 +130520 POINT(46.84116292530738 -122.55951531369845) bank130520 +130521 POINT(47.226580696251546 -122.84437841077289) bank130521 +130522 POINT(47.08964344901688 -122.05401943090803) bank130522 +130523 POINT(48.014084636255085 -122.55626180178429) bank130523 +130524 POINT(46.73991530009619 -122.81407913904091) bank130524 +130525 POINT(47.93138939641437 -121.61053218078713) bank130525 +130526 POINT(47.47415992184699 -121.9142105655468) bank130526 +130527 POINT(48.52754461157518 -122.55452531224552) bank130527 +130528 POINT(47.62659370565028 -123.1603447209451) bank130528 +130529 POINT(47.33230665056137 -122.40448114044183) bank130529 +130530 POINT(47.36685796488208 -121.45768628863324) bank130530 +130531 POINT(47.6127453973653 -122.26136177501266) bank130531 +130532 POINT(47.67696466913755 -122.63776338344637) bank130532 +130533 POINT(47.293503590124494 -122.86955732591913) bank130533 +130534 POINT(46.687309943026726 -122.53600642896443) bank130534 +130535 POINT(48.533997315384894 -122.54269694584194) bank130535 +130536 POINT(47.9569391222169 -122.12824384498452) bank130536 +130537 POINT(47.12727955989375 -122.5691453227307) bank130537 +130538 POINT(47.505791499684 -122.35625502726971) bank130538 +130539 POINT(48.15747539174759 -121.9145588513492) bank130539 +130540 POINT(47.6539127155517 -121.55091179237617) bank130540 +130541 POINT(48.40557919964428 -121.88905913721378) bank130541 +130542 POINT(47.093019820589205 -122.73671427040082) bank130542 +130543 POINT(48.40647002216058 -122.83509061313306) bank130543 +130544 POINT(48.01648122617483 -122.06659344792205) bank130544 +130545 POINT(47.602973634260756 -122.96008031740347) bank130545 +130546 POINT(48.28061472270366 -121.7238391027163) bank130546 +130547 POINT(47.99603403341561 -122.36223089699166) bank130547 +130548 POINT(46.886133226460345 -122.40483257799599) bank130548 +130549 POINT(48.32655167087656 -122.89944053228149) bank130549 +130550 POINT(47.89012493100891 -121.9489048661443) bank130550 +130551 POINT(48.22561783575744 -121.89003892896307) bank130551 +130552 POINT(47.68235605108962 -122.26906432590843) bank130552 +130553 POINT(46.82501031490043 -122.74391302011618) bank130553 +130554 POINT(46.99027701675531 -123.15707449970648) bank130554 +130555 POINT(48.24358243356817 -122.88508631270759) bank130555 +130556 POINT(46.638072444837704 -121.45510304688901) bank130556 +130557 POINT(47.88441704680198 -123.1303558115228) bank130557 +130558 POINT(47.494514858633025 -122.45621019812049) bank130558 +130559 POINT(46.684233092998554 -121.82513981220228) bank130559 +130560 POINT(47.89569280999338 -122.20513706637034) bank130560 +130561 POINT(48.158057674354794 -123.31154713931912) bank130561 +130562 POINT(48.49469837500829 -122.87567966266948) bank130562 +130563 POINT(47.6720849941334 -122.72410546270717) bank130563 +130564 POINT(48.53808308967518 -123.13829208071692) bank130564 +130565 POINT(47.05795585207461 -122.88003603800172) bank130565 +130566 POINT(46.81505000474534 -122.72336381516374) bank130566 +130567 POINT(47.46555697396746 -122.91693008813283) bank130567 +130568 POINT(48.41005195726902 -121.91998359736294) bank130568 +130569 POINT(47.07601213749023 -121.68574936563233) bank130569 +130570 POINT(47.4753810752014 -122.99449037435751) bank130570 +130571 POINT(47.95302688915168 -122.06280648058996) bank130571 +130572 POINT(47.352239363753 -122.48075669725142) bank130572 +130573 POINT(47.55344318746527 -121.8991958306269) bank130573 +130574 POINT(47.18365955014223 -122.01705516896939) bank130574 +130575 POINT(47.015001120961216 -123.32280313631667) bank130575 +130576 POINT(48.412367761299734 -122.37993764267983) bank130576 +130577 POINT(47.38447477539659 -123.29028614106366) bank130577 +130578 POINT(48.416514748209735 -121.69384242039546) bank130578 +130579 POINT(48.06044985419856 -121.79047130734813) bank130579 +130580 POINT(46.71601407509314 -122.30111575819559) bank130580 +130581 POINT(48.41044951298472 -123.25332428254299) bank130581 +130582 POINT(47.68411237564599 -122.13847735229798) bank130582 +130583 POINT(47.428910623089166 -122.43122766733447) bank130583 +130584 POINT(47.79932285299872 -121.49900085413798) bank130584 +130585 POINT(48.20215640182297 -122.15831880852573) bank130585 +130586 POINT(46.615689873435606 -122.86250737352336) bank130586 +130587 POINT(47.23260696041261 -123.29599047326815) bank130587 +130588 POINT(48.54747441310666 -122.59029084428558) bank130588 +130589 POINT(46.86368315759374 -123.28040742766811) bank130589 +130590 POINT(47.02016686817471 -122.45079244791829) bank130590 +130591 POINT(47.7456512155622 -122.42479048405188) bank130591 +130592 POINT(48.43693148701933 -121.61593175171518) bank130592 +130593 POINT(47.52593718435961 -121.57048120781617) bank130593 +130594 POINT(47.260762790686634 -121.72700966086518) bank130594 +130595 POINT(48.56737926979247 -121.75293752388734) bank130595 +130596 POINT(48.377650247673245 -123.04135705659506) bank130596 +130597 POINT(47.683617947774856 -123.16346193022986) bank130597 +130598 POINT(46.7855416406547 -122.73912697486924) bank130598 +130599 POINT(48.37255749727533 -121.76631470069535) bank130599 +130600 POINT(48.59451153808879 -123.16493561622138) bank130600 +130601 POINT(47.34726683611358 -122.64074093782371) bank130601 +130602 POINT(47.42935749849044 -123.29958951647103) bank130602 +130603 POINT(47.98342283453552 -122.45186032156718) bank130603 +130604 POINT(47.9464644310544 -122.54641317301702) bank130604 +130605 POINT(46.95812689656381 -121.5592947029828) bank130605 +130606 POINT(48.0343129064878 -121.67122695746046) bank130606 +130607 POINT(47.12869533835135 -122.59299804542289) bank130607 +130608 POINT(47.85339483828632 -123.11399063273284) bank130608 +130609 POINT(48.4225723586854 -122.28909638438634) bank130609 +130610 POINT(47.121635394953486 -122.75672597006492) bank130610 +130611 POINT(47.40508182892246 -121.47315515739159) bank130611 +130612 POINT(47.721974236247576 -122.13276998255577) bank130612 +130613 POINT(48.43410202675913 -122.21998217430254) bank130613 +130614 POINT(47.42942583040644 -121.95525241641766) bank130614 +130615 POINT(47.47076146624121 -122.31875859904724) bank130615 +130616 POINT(48.59992525884463 -122.21616060882555) bank130616 +130617 POINT(47.97623118774177 -122.82917097423339) bank130617 +130618 POINT(47.401516280839324 -123.07735709020176) bank130618 +130619 POINT(48.58007074528321 -123.32365377549849) bank130619 +130620 POINT(47.579947623239704 -123.01751190704624) bank130620 +130621 POINT(47.27716906828372 -122.90497970636592) bank130621 +130622 POINT(46.99653107512812 -121.62249376699734) bank130622 +130623 POINT(46.86838448727354 -121.6822792598182) bank130623 +130624 POINT(48.08317773890695 -122.44334640943677) bank130624 +130625 POINT(48.099524710968396 -121.43023968382761) bank130625 +130626 POINT(47.50429952698518 -122.09881325784251) bank130626 +130627 POINT(46.672245660865286 -123.31364414744108) bank130627 +130628 POINT(48.26772447946285 -122.17451791456365) bank130628 +130629 POINT(48.517463920976276 -121.41953768389249) bank130629 +130630 POINT(46.98619581467223 -122.68235830302395) bank130630 +130631 POINT(47.366254921563396 -121.4596697860673) bank130631 +130632 POINT(46.99080789214566 -122.83180968027504) bank130632 +130633 POINT(48.19862542797722 -122.58673615351607) bank130633 +130634 POINT(47.840958135923024 -122.163967341891) bank130634 +130635 POINT(47.226758952884154 -122.68109361873469) bank130635 +130636 POINT(47.49510487966902 -121.68301592190177) bank130636 +130637 POINT(46.992509665083546 -121.78298160234641) bank130637 +130638 POINT(47.9823648982324 -121.5355930511892) bank130638 +130639 POINT(48.13677527063192 -122.14782015360292) bank130639 +130640 POINT(47.89158102364325 -122.66987856459558) bank130640 +130641 POINT(47.58762394042956 -121.62745567563596) bank130641 +130642 POINT(46.876530862849194 -121.50531919372568) bank130642 +130643 POINT(46.62615930702374 -122.10886121096021) bank130643 +130644 POINT(47.79586464101782 -122.7773243219473) bank130644 +130645 POINT(47.45834925810389 -121.44757266764138) bank130645 +130646 POINT(47.92579565891966 -123.33090689633846) bank130646 +130647 POINT(47.29306294936441 -121.73078337809928) bank130647 +130648 POINT(47.95620963219363 -122.18244735681407) bank130648 +130649 POINT(47.25442376424262 -121.788542249891) bank130649 +130650 POINT(46.61750967523121 -121.84214407504265) bank130650 +130651 POINT(48.45241448637951 -123.32574554217462) bank130651 +130652 POINT(47.26187870961282 -122.74441623331239) bank130652 +130653 POINT(47.329180955167665 -123.07504887417444) bank130653 +130654 POINT(47.679024503701285 -122.93461070898464) bank130654 +130655 POINT(46.66281874450397 -122.12738736004779) bank130655 +130656 POINT(47.1940685986491 -122.52789901711635) bank130656 +130657 POINT(47.43535670987587 -121.60302230539561) bank130657 +130658 POINT(48.04587022372478 -122.43220702833396) bank130658 +130659 POINT(48.05287126753851 -121.49534089526347) bank130659 +130660 POINT(47.33162367545544 -122.56408166577819) bank130660 +130661 POINT(46.93791881682938 -121.5430733987929) bank130661 +130662 POINT(47.95312048952699 -122.65463827746711) bank130662 +130663 POINT(46.85905228246909 -122.19078521138854) bank130663 +130664 POINT(47.668025441512484 -122.2224215389464) bank130664 +130665 POINT(47.121449021252346 -122.8346257366086) bank130665 +130666 POINT(48.202811274510246 -122.41026760923927) bank130666 +130667 POINT(47.10793049157345 -122.65478036536271) bank130667 +130668 POINT(47.26198904962607 -121.959447328493) bank130668 +130669 POINT(47.06029711767005 -122.58556347506955) bank130669 +130670 POINT(47.70627315768923 -122.64091259872261) bank130670 +130671 POINT(48.5061375705687 -121.45700258485805) bank130671 +130672 POINT(47.332658284504895 -121.56380870189575) bank130672 +130673 POINT(48.08649637026482 -121.73409051089929) bank130673 +130674 POINT(47.19390528244635 -121.64399934237285) bank130674 +130675 POINT(47.31764077073637 -121.60848656027568) bank130675 +130676 POINT(46.63068670863554 -122.93584485638539) bank130676 +130677 POINT(48.20786512078317 -121.65669089752527) bank130677 +130678 POINT(47.53472863097463 -121.35313417648415) bank130678 +130679 POINT(46.9025272292716 -122.93249886122668) bank130679 +130680 POINT(47.91526063778076 -122.70702539043923) bank130680 +130681 POINT(48.44125677463961 -123.15537319572577) bank130681 +130682 POINT(48.36133371711489 -123.24673544806987) bank130682 +130683 POINT(47.527062263537445 -122.97817164831838) bank130683 +130684 POINT(48.58482132617254 -122.75870393503111) bank130684 +130685 POINT(47.37193350976837 -122.10485566367544) bank130685 +130686 POINT(48.352902585898825 -122.1212261715564) bank130686 +130687 POINT(47.31535352315091 -122.99157951619762) bank130687 +130688 POINT(46.80214355836519 -122.62955921490472) bank130688 +130689 POINT(47.37998174821875 -122.79034440693125) bank130689 +130690 POINT(48.27565461437778 -122.5938453753081) bank130690 +130691 POINT(47.66938211351335 -122.98970191516192) bank130691 +130692 POINT(47.49862307184039 -122.67381515633038) bank130692 +130693 POINT(47.85524578157193 -121.95658899509455) bank130693 +130694 POINT(48.482116198732044 -121.67441344258071) bank130694 +130695 POINT(46.896204805307235 -121.90562970103568) bank130695 +130696 POINT(48.21849679672445 -122.22487772879137) bank130696 +130697 POINT(47.491074671647034 -122.58468295857132) bank130697 +130698 POINT(46.804066086724134 -123.26994108763809) bank130698 +130699 POINT(48.31007774230073 -122.47866849231586) bank130699 +130700 POINT(47.38440410343363 -122.0091704035343) bank130700 +130701 POINT(48.387539758156684 -121.55084881086246) bank130701 +130702 POINT(46.83830332135453 -122.77899633311772) bank130702 +130703 POINT(46.93097839844039 -122.91792277052927) bank130703 +130704 POINT(48.02407249359662 -123.16733604348035) bank130704 +130705 POINT(46.907824777304846 -121.68435259030569) bank130705 +130706 POINT(47.25999457691496 -121.8627703906019) bank130706 +130707 POINT(47.54708009883085 -123.24003423536276) bank130707 +130708 POINT(46.88313552522568 -123.17465571067467) bank130708 +130709 POINT(46.853770597918206 -123.19781038743552) bank130709 +130710 POINT(47.858662592759906 -123.04444568949607) bank130710 +130711 POINT(46.90255322827067 -122.76885467609743) bank130711 +130712 POINT(47.69438282466361 -121.78000886340531) bank130712 +130713 POINT(47.43737353178898 -122.67572705175986) bank130713 +130714 POINT(47.46131592540346 -121.70041753185694) bank130714 +130715 POINT(48.087760208137276 -122.15283757808946) bank130715 +130716 POINT(48.43612508786108 -122.10785151405061) bank130716 +130717 POINT(47.444892465177844 -121.7896911374259) bank130717 +130718 POINT(47.13039351099693 -121.51477233623761) bank130718 +130719 POINT(47.332393892165456 -123.14051425592227) bank130719 +130720 POINT(47.08894383956757 -121.37789918344679) bank130720 +130721 POINT(47.13471517965559 -122.53903236175671) bank130721 +130722 POINT(48.59026212053885 -122.24075036460896) bank130722 +130723 POINT(48.0857820617945 -121.63899857513367) bank130723 +130724 POINT(48.46689559097402 -121.86545890130517) bank130724 +130725 POINT(47.47547116063856 -122.6731327862333) bank130725 +130726 POINT(47.47548397583915 -121.84176796532748) bank130726 +130727 POINT(46.623009356254585 -123.15686112203785) bank130727 +130728 POINT(46.63127582677994 -123.07888276153015) bank130728 +130729 POINT(47.753855977785 -121.56957194613973) bank130729 +130730 POINT(46.78091590363755 -122.79759646078105) bank130730 +130731 POINT(47.66764948805933 -122.4805251997392) bank130731 +130732 POINT(46.79349003906437 -123.17339392256801) bank130732 +130733 POINT(47.79617466890524 -123.25829695228678) bank130733 +130734 POINT(46.97074153128172 -122.91450819872416) bank130734 +130735 POINT(47.27604796984399 -121.46815043431107) bank130735 +130736 POINT(48.15779781450099 -122.70273177979948) bank130736 +130737 POINT(46.81692886925545 -121.94746734235014) bank130737 +130738 POINT(48.398320922023395 -122.43335566005909) bank130738 +130739 POINT(48.39283359647203 -121.92499703190838) bank130739 +130740 POINT(47.467190224422076 -122.20918199925757) bank130740 +130741 POINT(47.38844136786633 -122.6201498517166) bank130741 +130742 POINT(46.8714807246334 -123.26747232673524) bank130742 +130743 POINT(47.70609914814738 -121.77143666073674) bank130743 +130744 POINT(46.73902625605113 -121.39133986767204) bank130744 +130745 POINT(47.84971434315246 -121.7197875918504) bank130745 +130746 POINT(47.504099555748354 -121.62960925298935) bank130746 +130747 POINT(47.821121939475375 -121.92773013086445) bank130747 +130748 POINT(47.57444460309765 -121.56900894725578) bank130748 +130749 POINT(46.669853600853656 -122.25314754080372) bank130749 +130750 POINT(47.50337229213768 -123.09363602047772) bank130750 +130751 POINT(47.92838460497626 -121.6380475662485) bank130751 +130752 POINT(48.502563513630086 -121.47699454162684) bank130752 +130753 POINT(48.51817312084497 -122.93179331312012) bank130753 +130754 POINT(48.17462448202063 -121.33792110666424) bank130754 +130755 POINT(46.76309972138841 -121.6266018858419) bank130755 +130756 POINT(48.23733801742963 -121.47993506272523) bank130756 +130757 POINT(47.141969157966535 -123.24448175403394) bank130757 +130758 POINT(47.110860656658495 -121.45279295759188) bank130758 +130759 POINT(48.50709360275661 -123.10508548978238) bank130759 +130760 POINT(48.139086287264114 -122.42658065290401) bank130760 +130761 POINT(46.81556387506132 -122.68640702005065) bank130761 +130762 POINT(47.2123768131332 -122.19242359025418) bank130762 +130763 POINT(48.50322747885442 -121.94566761190364) bank130763 +130764 POINT(48.09129874887011 -122.97443225443871) bank130764 +130765 POINT(47.70219113548304 -122.11337714415401) bank130765 +130766 POINT(48.5904956841626 -122.81445293570856) bank130766 +130767 POINT(47.752625406564505 -123.06141354395623) bank130767 +130768 POINT(47.18986549160429 -123.04498113913134) bank130768 +130769 POINT(47.42356174962188 -121.95511459553494) bank130769 +130770 POINT(47.25894420107437 -121.52955796302565) bank130770 +130771 POINT(47.545974436890305 -122.85993678616397) bank130771 +130772 POINT(48.0357690204153 -122.48843251113468) bank130772 +130773 POINT(48.416831286129316 -121.36384362945726) bank130773 +130774 POINT(46.71124983723304 -123.1173216359402) bank130774 +130775 POINT(47.62396881222934 -122.50596351837103) bank130775 +130776 POINT(47.15526614119982 -122.42322598446087) bank130776 +130777 POINT(48.01421120026255 -122.48680824245918) bank130777 +130778 POINT(47.37824253449959 -122.90749717752139) bank130778 +130779 POINT(47.12575797577549 -122.53787768738256) bank130779 +130780 POINT(48.1128443104239 -122.8489021521677) bank130780 +130781 POINT(47.317699417103114 -123.24998232966196) bank130781 +130782 POINT(47.00032532174861 -122.75218136857548) bank130782 +130783 POINT(46.728258649434544 -122.06151785148955) bank130783 +130784 POINT(48.48544112067263 -122.1612667474638) bank130784 +130785 POINT(48.05180194355106 -121.66212387703544) bank130785 +130786 POINT(47.451588335457295 -122.71850556520258) bank130786 +130787 POINT(48.137616550897484 -121.94535824077289) bank130787 +130788 POINT(47.70046686341661 -121.36339245528643) bank130788 +130789 POINT(46.886513696178476 -122.01865402336124) bank130789 +130790 POINT(46.8402124759136 -122.21436694965752) bank130790 +130791 POINT(48.50098106348448 -121.63562205286374) bank130791 +130792 POINT(47.03731961106566 -121.51320050542789) bank130792 +130793 POINT(48.39854281930888 -121.88881614808413) bank130793 +130794 POINT(47.214151183022764 -122.63612272714387) bank130794 +130795 POINT(47.4537496138503 -122.84665004120362) bank130795 +130796 POINT(46.96986378707076 -122.1319406518577) bank130796 +130797 POINT(47.22039468056747 -122.6897000389898) bank130797 +130798 POINT(48.39320527392812 -123.27711921293098) bank130798 +130799 POINT(47.56699449747857 -122.93603412729328) bank130799 +130800 POINT(46.953050182104086 -122.77221522053789) bank130800 +130801 POINT(46.99158871972755 -122.92605144177713) bank130801 +130802 POINT(47.57005765066271 -121.95865862761761) bank130802 +130803 POINT(47.784551435138404 -122.45336903642826) bank130803 +130804 POINT(47.12500445232726 -122.97319076424151) bank130804 +130805 POINT(46.756123602690074 -123.03134006817754) bank130805 +130806 POINT(48.08748479648199 -121.61965962308037) bank130806 +130807 POINT(47.31671257251577 -122.28051417432789) bank130807 +130808 POINT(47.38128242445011 -121.65120620705554) bank130808 +130809 POINT(47.42832619019745 -121.68342113909033) bank130809 +130810 POINT(47.56272541961412 -123.12863087866857) bank130810 +130811 POINT(47.825383652338495 -122.39825242592157) bank130811 +130812 POINT(48.42200518796692 -121.88973478253153) bank130812 +130813 POINT(47.27405811700042 -122.81533919774454) bank130813 +130814 POINT(48.25945648036372 -122.3923017440131) bank130814 +130815 POINT(47.50260256109607 -122.94767791011036) bank130815 +130816 POINT(47.559709992318865 -122.28021285100237) bank130816 +130817 POINT(48.3275540943203 -121.53625168350553) bank130817 +130818 POINT(47.189334079977534 -122.3765594958523) bank130818 +130819 POINT(47.32400836670806 -122.26894213751562) bank130819 +130820 POINT(47.43838327013387 -121.98765114215828) bank130820 +130821 POINT(46.82515625685741 -122.56367475888483) bank130821 +130822 POINT(48.066610342542 -122.83778336063473) bank130822 +130823 POINT(47.93690149494319 -121.71521256450694) bank130823 +130824 POINT(47.803286428875445 -121.56649997098432) bank130824 +130825 POINT(46.9248484093925 -123.15808604899964) bank130825 +130826 POINT(47.65885630550943 -122.63606943904477) bank130826 +130827 POINT(47.69480704329927 -122.3681497330446) bank130827 +130828 POINT(47.624673326697526 -122.68093068423941) bank130828 +130829 POINT(47.28088004675826 -122.8952667391089) bank130829 +130830 POINT(46.63751657460286 -122.08489874989222) bank130830 +130831 POINT(47.1716859191237 -123.06483304339781) bank130831 +130832 POINT(47.115640986634325 -121.70208247672906) bank130832 +130833 POINT(48.308095328206925 -122.10869827508002) bank130833 +130834 POINT(48.061251323459814 -122.93009115220876) bank130834 +130835 POINT(48.39620676667574 -123.05074655411217) bank130835 +130836 POINT(47.00088592301031 -123.17251969040771) bank130836 +130837 POINT(46.76180746728438 -121.51880580166979) bank130837 +130838 POINT(47.56131763139691 -121.77209873772456) bank130838 +130839 POINT(48.47019948061317 -122.56268341874122) bank130839 +130840 POINT(47.71383210817798 -121.7948103683883) bank130840 +130841 POINT(46.91906310232376 -121.8297027952745) bank130841 +130842 POINT(47.96194051058384 -122.32119989513261) bank130842 +130843 POINT(47.91063509805205 -122.7391687479644) bank130843 +130844 POINT(46.95652068591821 -121.34768289760807) bank130844 +130845 POINT(46.78028975578188 -121.35048416337852) bank130845 +130846 POINT(46.682356135470236 -121.5277841500311) bank130846 +130847 POINT(48.03767531237484 -121.77247896137133) bank130847 +130848 POINT(47.795763833251335 -122.44836788632642) bank130848 +130849 POINT(47.31652085128919 -121.7771023941413) bank130849 +130850 POINT(48.272774423654674 -123.2170980268227) bank130850 +130851 POINT(47.40809484358154 -121.9933715806435) bank130851 +130852 POINT(47.772951480705586 -121.3888039750075) bank130852 +130853 POINT(46.90839897179722 -122.29644719191907) bank130853 +130854 POINT(48.52357991234616 -122.91429324310381) bank130854 +130855 POINT(47.537855472323805 -122.2621652633367) bank130855 +130856 POINT(47.85030465934491 -122.39806056660751) bank130856 +130857 POINT(46.710986304656416 -123.07510554288072) bank130857 +130858 POINT(46.84485298262475 -121.38313679601063) bank130858 +130859 POINT(47.70855271574599 -121.88453358032676) bank130859 +130860 POINT(48.28331241827706 -121.77759341753253) bank130860 +130861 POINT(47.83500202229946 -121.74751451844396) bank130861 +130862 POINT(46.8908895873541 -122.87866849351363) bank130862 +130863 POINT(46.82610896695458 -122.15168014078782) bank130863 +130864 POINT(48.178769562997516 -123.22687522135881) bank130864 +130865 POINT(47.36598816420313 -122.27517454829058) bank130865 +130866 POINT(47.05121153679672 -121.409540941041) bank130866 +130867 POINT(48.339170846819954 -123.19270033724419) bank130867 +130868 POINT(47.21332311136391 -121.98877495316748) bank130868 +130869 POINT(46.61866009604792 -122.96278272615149) bank130869 +130870 POINT(48.536518814197 -122.35161135312195) bank130870 +130871 POINT(48.3392164144819 -121.57815960800639) bank130871 +130872 POINT(47.16018937379277 -123.2673553135771) bank130872 +130873 POINT(47.05262946210934 -122.60678223582246) bank130873 +130874 POINT(47.40454373250337 -123.16621051402734) bank130874 +130875 POINT(47.13181569426927 -121.73793120850651) bank130875 +130876 POINT(48.24786422821522 -121.7121949360011) bank130876 +130877 POINT(48.27153093225514 -122.33188514088135) bank130877 +130878 POINT(48.29387238874888 -121.45020483706014) bank130878 +130879 POINT(47.14423910158763 -121.64166957043805) bank130879 +130880 POINT(47.0141692021408 -123.04584341024035) bank130880 +130881 POINT(47.63468233598329 -123.24092827959657) bank130881 +130882 POINT(47.33895409547463 -121.88264368300362) bank130882 +130883 POINT(47.65761042971901 -122.91994155871554) bank130883 +130884 POINT(47.250043641304984 -122.63693998513848) bank130884 +130885 POINT(48.48706649387055 -123.03913302917411) bank130885 +130886 POINT(47.960078575673016 -122.59651913354988) bank130886 +130887 POINT(48.023808324492116 -121.49278825094568) bank130887 +130888 POINT(47.99068933362426 -123.20644704143031) bank130888 +130889 POINT(47.29308310260645 -121.35426554178738) bank130889 +130890 POINT(47.86666266841482 -123.29565689665999) bank130890 +130891 POINT(48.02711253201631 -122.01337016223235) bank130891 +130892 POINT(48.28258638864733 -121.99016100602249) bank130892 +130893 POINT(46.85083812000881 -121.60548523944492) bank130893 +130894 POINT(47.49720715981218 -122.67005701483177) bank130894 +130895 POINT(46.768566241932646 -121.34350984749581) bank130895 +130896 POINT(47.16042898589207 -121.48670299372958) bank130896 +130897 POINT(46.915917967018125 -122.95508179908123) bank130897 +130898 POINT(48.08829096698192 -121.48240935137093) bank130898 +130899 POINT(47.89341161842338 -121.47363459041584) bank130899 +130900 POINT(46.76790141076187 -121.63285984781942) bank130900 +130901 POINT(46.660196406945495 -122.13287147356502) bank130901 +130902 POINT(47.95608944942059 -122.45933869718237) bank130902 +130903 POINT(47.68050608676423 -122.73720488573838) bank130903 +130904 POINT(48.43217208543975 -123.21970422818067) bank130904 +130905 POINT(46.93511640364737 -122.7131726374964) bank130905 +130906 POINT(47.42215556453349 -123.31366892353634) bank130906 +130907 POINT(48.59642627303436 -121.56587136870965) bank130907 +130908 POINT(47.69340037150395 -122.66896084091238) bank130908 +130909 POINT(46.864981942809486 -122.91739975164141) bank130909 +130910 POINT(47.02040972215086 -123.24006204297396) bank130910 +130911 POINT(48.214900865957205 -121.77539168417864) bank130911 +130912 POINT(47.19212981291972 -122.11850295105424) bank130912 +130913 POINT(47.94730900468578 -123.33092247544494) bank130913 +130914 POINT(47.21723110560829 -123.09216113602368) bank130914 +130915 POINT(46.70435464909389 -122.7186875863122) bank130915 +130916 POINT(46.63662780509878 -122.19830716851145) bank130916 +130917 POINT(48.21832142686517 -122.81957526597255) bank130917 +130918 POINT(47.03235990183327 -122.11819878185854) bank130918 +130919 POINT(47.91369624507095 -122.29849495073947) bank130919 +130920 POINT(47.966260332898365 -121.64155478489384) bank130920 +130921 POINT(47.33668107024275 -121.75464989046895) bank130921 +130922 POINT(47.94984018576376 -122.28328068929821) bank130922 +130923 POINT(47.49249826617932 -121.85313653861073) bank130923 +130924 POINT(46.78068912680784 -122.9121631043135) bank130924 +130925 POINT(46.75865006305766 -121.50018748728854) bank130925 +130926 POINT(48.51311418162406 -122.60891229706934) bank130926 +130927 POINT(47.792199765736825 -122.95398591678138) bank130927 +130928 POINT(47.83406630777343 -121.6409707772051) bank130928 +130929 POINT(47.118500230630005 -122.89721935092578) bank130929 +130930 POINT(46.91741870548585 -121.95954360893737) bank130930 +130931 POINT(46.96475354170871 -121.41384489477277) bank130931 +130932 POINT(48.582671708792304 -122.06094247956662) bank130932 +130933 POINT(47.49179191339249 -122.85763628840985) bank130933 +130934 POINT(47.07187078381445 -122.93266056403432) bank130934 +130935 POINT(47.020845364336694 -122.12436556589365) bank130935 +130936 POINT(47.866014639549796 -122.32629418086583) bank130936 +130937 POINT(46.90874371063104 -122.4722936715297) bank130937 +130938 POINT(47.624487549285284 -121.50090534057493) bank130938 +130939 POINT(48.50371658777853 -122.7971507986711) bank130939 +130940 POINT(47.83264984021432 -122.98989606756516) bank130940 +130941 POINT(47.91027498324234 -121.76479673773099) bank130941 +130942 POINT(48.35061672132797 -122.74833056461077) bank130942 +130943 POINT(47.11653359836183 -121.68663260649282) bank130943 +130944 POINT(47.017296083650045 -121.6704250491715) bank130944 +130945 POINT(47.98045162278776 -123.1555412516646) bank130945 +130946 POINT(47.90526440297627 -123.28298652910847) bank130946 +130947 POINT(47.84443208846327 -121.52868668913099) bank130947 +130948 POINT(47.43446525598222 -122.73771959324023) bank130948 +130949 POINT(46.6338969121295 -123.11225728397135) bank130949 +130950 POINT(47.14118189503565 -122.07772866754009) bank130950 +130951 POINT(46.95435778046499 -122.94627279292693) bank130951 +130952 POINT(48.474248715110576 -121.8327474929272) bank130952 +130953 POINT(48.34594404039169 -121.60306020158916) bank130953 +130954 POINT(47.46985953116013 -122.78335834080946) bank130954 +130955 POINT(47.50271910044654 -121.93288203651393) bank130955 +130956 POINT(47.1523391232966 -122.96499416858113) bank130956 +130957 POINT(46.81017705695519 -122.10913793008002) bank130957 +130958 POINT(46.85705032535916 -122.03504462608227) bank130958 +130959 POINT(48.37140202129673 -122.01331104692744) bank130959 +130960 POINT(47.19822176403923 -121.99013084927572) bank130960 +130961 POINT(47.62611891195697 -121.362982370586) bank130961 +130962 POINT(47.149294457800835 -122.73089150444508) bank130962 +130963 POINT(47.749983301228255 -122.67811801066294) bank130963 +130964 POINT(47.82452552772316 -121.85200525231623) bank130964 +130965 POINT(48.19616420957834 -122.8022194076521) bank130965 +130966 POINT(47.4533433904066 -123.05853540698477) bank130966 +130967 POINT(47.92457551702645 -121.6369886837799) bank130967 +130968 POINT(48.40705299192174 -121.38181574165542) bank130968 +130969 POINT(47.46516025016015 -122.43966067785847) bank130969 +130970 POINT(47.87464867834175 -121.51009707882558) bank130970 +130971 POINT(46.61557058817663 -121.62096599224785) bank130971 +130972 POINT(47.92586914005258 -122.33369473218542) bank130972 +130973 POINT(48.45964642079894 -121.9321297915217) bank130973 +130974 POINT(46.6935849303025 -122.65424095807208) bank130974 +130975 POINT(46.985240047510445 -123.31677862568102) bank130975 +130976 POINT(48.183676891378205 -122.64354513658735) bank130976 +130977 POINT(48.546315057183634 -122.2147734757291) bank130977 +130978 POINT(47.63506965493787 -121.88784164092873) bank130978 +130979 POINT(46.982535045731396 -121.7956632646227) bank130979 +130980 POINT(46.66394705858609 -121.82164499688356) bank130980 +130981 POINT(47.14513729709449 -123.10659969712094) bank130981 +130982 POINT(47.30534369618339 -122.41387895227618) bank130982 +130983 POINT(46.80924245866104 -122.81083270303233) bank130983 +130984 POINT(47.47999110330037 -122.46379040492718) bank130984 +130985 POINT(47.24904823421841 -123.29590930537715) bank130985 +130986 POINT(47.092716465729985 -122.38660222699991) bank130986 +130987 POINT(48.08720526350239 -122.5887551827116) bank130987 +130988 POINT(46.95856394841134 -122.68545793111856) bank130988 +130989 POINT(46.85523619005311 -121.82841345397777) bank130989 +130990 POINT(47.02803863410934 -122.279554376964) bank130990 +130991 POINT(47.770777325575175 -123.00039436544867) bank130991 +130992 POINT(47.91679003589822 -122.26595049738894) bank130992 +130993 POINT(48.105482091761665 -122.70821818099752) bank130993 +130994 POINT(47.24102805873517 -121.68489396295882) bank130994 +130995 POINT(46.76157081020892 -123.01204304132126) bank130995 +130996 POINT(48.472520732802586 -122.28409675984625) bank130996 +130997 POINT(47.41804754078283 -122.99954958209658) bank130997 +130998 POINT(47.1404233033311 -122.4982411526414) bank130998 +130999 POINT(46.65932803262745 -122.84381443494455) bank130999 +131000 POINT(46.72180540752244 -122.77515139855596) bank131000 +131001 POINT(46.76917887974876 -122.235358312329) bank131001 +131002 POINT(47.80584730605469 -122.2413062733384) bank131002 +131003 POINT(47.65212183741177 -121.76401829071831) bank131003 +131004 POINT(48.56808254030292 -121.73744651189753) bank131004 +131005 POINT(47.14136869959538 -123.25329961964924) bank131005 +131006 POINT(47.15650849585008 -122.10707683567404) bank131006 +131007 POINT(46.769580620240966 -122.28875476159186) bank131007 +131008 POINT(47.779993970937355 -123.1364116403395) bank131008 +131009 POINT(47.00552862564339 -122.2527350622417) bank131009 +131010 POINT(47.46423617288239 -122.99070131475074) bank131010 +131011 POINT(48.58611206505173 -122.41844098866444) bank131011 +131012 POINT(47.31506838585656 -123.22048005699602) bank131012 +131013 POINT(47.33741763543911 -121.60728722197088) bank131013 +131014 POINT(48.43561927838447 -123.1582926846288) bank131014 +131015 POINT(48.488383367081866 -122.60165494823926) bank131015 +131016 POINT(48.46123011437129 -122.5484847406843) bank131016 +131017 POINT(48.1686996948446 -121.49169552715863) bank131017 +131018 POINT(47.371895733406646 -121.98817544701211) bank131018 +131019 POINT(47.73422179125504 -121.41024140562281) bank131019 +131020 POINT(48.255257217020684 -121.37335514111774) bank131020 +131021 POINT(46.78098828322465 -121.79193723022884) bank131021 +131022 POINT(48.45621126778216 -122.86945553541645) bank131022 +131023 POINT(47.39998310253777 -122.72708376227939) bank131023 +131024 POINT(47.41826713853479 -122.73865730497846) bank131024 +131025 POINT(47.24978708749272 -121.70360413786794) bank131025 +131026 POINT(47.78823263292647 -123.25832694815921) bank131026 +131027 POINT(47.30116444049307 -122.20619031849424) bank131027 +131028 POINT(48.42996805017115 -123.22028846293857) bank131028 +131029 POINT(47.79604447198148 -123.16875029020633) bank131029 +131030 POINT(48.52962602476903 -121.79846706624807) bank131030 +131031 POINT(46.60882619620346 -122.98725651267056) bank131031 +131032 POINT(47.110494184173426 -123.31585243616026) bank131032 +131033 POINT(47.35268299613616 -121.83979609118418) bank131033 +131034 POINT(47.39604477623533 -121.60763593257745) bank131034 +131035 POINT(47.88445488472867 -122.4900935279682) bank131035 +131036 POINT(47.247906571360694 -121.49944880033836) bank131036 +131037 POINT(46.6983207420708 -122.33188662527634) bank131037 +131038 POINT(47.728661001681836 -122.33185927737232) bank131038 +131039 POINT(48.06489777819973 -121.83690264678442) bank131039 +131040 POINT(47.632087785230794 -121.5585094567502) bank131040 +131041 POINT(46.83324231038731 -122.08141626427988) bank131041 +131042 POINT(47.00468732701305 -121.49958682317157) bank131042 +131043 POINT(47.20197779279445 -122.89562807509233) bank131043 +131044 POINT(47.14115144266089 -121.44259751049776) bank131044 +131045 POINT(47.769750511284265 -123.32160236190316) bank131045 +131046 POINT(48.15938253192433 -122.23221587731243) bank131046 +131047 POINT(47.16052479775861 -122.44002117009683) bank131047 +131048 POINT(48.135569860218844 -123.27669311276534) bank131048 +131049 POINT(48.031613958572976 -122.27073603205855) bank131049 +131050 POINT(48.42618350382235 -121.41504281834324) bank131050 +131051 POINT(48.43983771405098 -122.0545565893169) bank131051 +131052 POINT(47.90261545699789 -123.04155132566876) bank131052 +131053 POINT(48.49955890083287 -122.07353279620665) bank131053 +131054 POINT(47.73571053938821 -122.09027167330343) bank131054 +131055 POINT(47.0620696620759 -122.41355237962216) bank131055 +131056 POINT(48.18561091300955 -123.0813591281677) bank131056 +131057 POINT(47.66377628971282 -122.90815552847099) bank131057 +131058 POINT(47.57540738004769 -122.03429067689312) bank131058 +131059 POINT(48.52266186694609 -122.48615010333104) bank131059 +131060 POINT(47.37277716488189 -122.53497657557328) bank131060 +131061 POINT(47.84778363481816 -122.60011457295948) bank131061 +131062 POINT(48.34130462118186 -121.74687224139564) bank131062 +131063 POINT(47.627535167120705 -122.36063778268499) bank131063 +131064 POINT(47.37060763647132 -122.40349316478533) bank131064 +131065 POINT(47.93465234067985 -121.5672151847651) bank131065 +131066 POINT(46.891751099907154 -122.1850165234676) bank131066 +131067 POINT(47.446920868088334 -122.68573206842437) bank131067 +131068 POINT(47.62400607116411 -121.70504576407865) bank131068 +131069 POINT(46.99186238893448 -122.76577710235202) bank131069 +131070 POINT(47.55661928202719 -122.67000852136236) bank131070 +131071 POINT(48.41730878142154 -122.15853119822928) bank131071 +131072 POINT(46.878655850826284 -121.36224651423038) bank131072 +131073 POINT(48.022927979304875 -122.13655224084505) bank131073 +131074 POINT(47.26494192773625 -122.6191187761245) bank131074 +131075 POINT(48.02819477803936 -121.7495928777569) bank131075 +131076 POINT(48.137343762924814 -121.65330712677383) bank131076 +131077 POINT(48.30072160962355 -121.52757182288275) bank131077 +131078 POINT(47.043192283020716 -122.83469223695072) bank131078 +131079 POINT(46.69798528277286 -122.31765954773606) bank131079 +131080 POINT(48.09776545551514 -123.16680173432074) bank131080 +131081 POINT(47.195091341487704 -122.04093890097705) bank131081 +131082 POINT(46.74911980159516 -122.53848741629012) bank131082 +131083 POINT(47.696584090304604 -123.23268968433915) bank131083 +131084 POINT(47.91853764813062 -122.43849274900131) bank131084 +131085 POINT(48.26753247010088 -122.50979408126312) bank131085 +131086 POINT(48.45299058410999 -122.47107479326843) bank131086 +131087 POINT(48.59794049914402 -122.97821461824527) bank131087 +131088 POINT(46.8077808119046 -122.54040481802488) bank131088 +131089 POINT(48.54222284729708 -122.89395939407375) bank131089 +131090 POINT(47.73754654558727 -121.83909377423007) bank131090 +131091 POINT(46.98549417711954 -122.21566123823807) bank131091 +131092 POINT(48.42885388572131 -123.21806401651328) bank131092 +131093 POINT(47.83144304850297 -121.45059871146736) bank131093 +131094 POINT(47.67297860783846 -122.30290998434631) bank131094 +131095 POINT(48.44529822973744 -121.41130732755492) bank131095 +131096 POINT(47.88852604858094 -122.68588765893854) bank131096 +131097 POINT(46.61559488770141 -122.56396234132761) bank131097 +131098 POINT(47.059426192545324 -122.60087501837229) bank131098 +131099 POINT(46.872484447656 -121.88564445611738) bank131099 +131100 POINT(47.4112662928644 -122.64164054238526) bank131100 +131101 POINT(46.70381837978975 -122.73062833920632) bank131101 +131102 POINT(47.56771929317573 -123.05948237383706) bank131102 +131103 POINT(47.53267457426076 -122.06806082561613) bank131103 +131104 POINT(48.343481363784015 -123.31924712582666) bank131104 +131105 POINT(47.91529812534172 -121.58722740604658) bank131105 +131106 POINT(48.09916498687883 -122.37230790241252) bank131106 +131107 POINT(47.45587822956769 -122.70334922991644) bank131107 +131108 POINT(47.96264736162489 -123.09421296452643) bank131108 +131109 POINT(48.54439610145125 -122.89511677601199) bank131109 +131110 POINT(47.19200566708853 -123.20232041850645) bank131110 +131111 POINT(48.45336728266506 -123.31284040781303) bank131111 +131112 POINT(48.2252802409812 -122.00982395127342) bank131112 +131113 POINT(48.13780859538672 -122.53988448659116) bank131113 +131114 POINT(47.95320358832833 -123.18774793897552) bank131114 +131115 POINT(47.13380883965376 -121.3496546500409) bank131115 +131116 POINT(46.998686053471296 -122.86767510365526) bank131116 +131117 POINT(47.167298717991194 -123.26641907123492) bank131117 +131118 POINT(47.060299850410544 -121.49934111492274) bank131118 +131119 POINT(48.01361946002535 -121.92667980700503) bank131119 +131120 POINT(48.188459252070736 -121.65191309871436) bank131120 +131121 POINT(46.63862533291115 -121.83910750780673) bank131121 +131122 POINT(47.63811866936882 -121.39355133850599) bank131122 +131123 POINT(48.2384988711163 -122.14342999004022) bank131123 +131124 POINT(46.79130963685307 -121.50392008495226) bank131124 +131125 POINT(47.24660431484244 -121.98262120031913) bank131125 +131126 POINT(46.79011545250374 -122.56596986871374) bank131126 +131127 POINT(46.65939789760718 -122.49248808728615) bank131127 +131128 POINT(48.5346656013638 -123.21501700279504) bank131128 +131129 POINT(46.709188495119136 -121.76220102663231) bank131129 +131130 POINT(48.49836371149488 -122.91839327420969) bank131130 +131131 POINT(48.5070687120628 -122.59647634195733) bank131131 +131132 POINT(48.09637630506541 -121.64746901350142) bank131132 +131133 POINT(47.10755446173674 -122.57514624716278) bank131133 +131134 POINT(46.83794819906468 -122.44980886557767) bank131134 +131135 POINT(46.876127922783354 -122.57749068145294) bank131135 +131136 POINT(46.764109804149456 -123.29159848327419) bank131136 +131137 POINT(47.0224967118148 -121.63678171159896) bank131137 +131138 POINT(48.17999145412746 -121.74566974649471) bank131138 +131139 POINT(46.92622433507839 -123.0967884857523) bank131139 +131140 POINT(46.687155843529105 -122.92282364657873) bank131140 +131141 POINT(46.77060253659045 -122.02697887822546) bank131141 +131142 POINT(46.85257179973463 -123.22241146449446) bank131142 +131143 POINT(47.652896535247876 -121.90855078819804) bank131143 +131144 POINT(47.321004085476865 -122.91635235703592) bank131144 +131145 POINT(46.63152961728686 -122.45539298083463) bank131145 +131146 POINT(47.84307165573815 -122.71224597828149) bank131146 +131147 POINT(46.85644801361415 -121.83073254421426) bank131147 +131148 POINT(46.92057043744846 -121.77762650654907) bank131148 +131149 POINT(48.43101834780136 -122.52167534650246) bank131149 +131150 POINT(48.12300220014539 -122.88975720605738) bank131150 +131151 POINT(48.32127652512117 -122.97438536057469) bank131151 +131152 POINT(47.454617876203166 -121.7696027136167) bank131152 +131153 POINT(47.38406935186378 -122.29078265537443) bank131153 +131154 POINT(48.13303363125394 -122.67255850845419) bank131154 +131155 POINT(47.56525383329347 -122.79242303547763) bank131155 +131156 POINT(46.622142657422714 -122.08149841767728) bank131156 +131157 POINT(47.88169149610339 -123.23419142694586) bank131157 +131158 POINT(46.82382623953357 -122.05970768723542) bank131158 +131159 POINT(46.97680161602238 -123.16209780702965) bank131159 +131160 POINT(47.929322181576985 -122.82839374917451) bank131160 +131161 POINT(47.947816130094374 -123.17450441875668) bank131161 +131162 POINT(47.96851141827597 -121.58204121422281) bank131162 +131163 POINT(47.00662843722134 -122.54388115771572) bank131163 +131164 POINT(47.60704151477718 -121.6123687403975) bank131164 +131165 POINT(47.23653093668695 -123.25338969161578) bank131165 +131166 POINT(48.20940874555778 -123.22469644715709) bank131166 +131167 POINT(48.122344143336555 -121.7736147493515) bank131167 +131168 POINT(48.50319114705992 -122.07094951386959) bank131168 +131169 POINT(47.628861466124285 -122.58670511944122) bank131169 +131170 POINT(48.60312822438574 -122.6684073746416) bank131170 +131171 POINT(48.2729261521385 -121.8129141028427) bank131171 +131172 POINT(47.237682472630794 -121.49825092736536) bank131172 +131173 POINT(48.08324333050366 -122.20186692255334) bank131173 +131174 POINT(48.24094421661619 -121.60542291007805) bank131174 +131175 POINT(48.08234868293522 -122.01298462122902) bank131175 +131176 POINT(46.88696227349179 -122.2240715104854) bank131176 +131177 POINT(47.80134390277468 -121.59604006849297) bank131177 +131178 POINT(48.513468747429755 -122.97363598594148) bank131178 +131179 POINT(47.31220561662868 -122.40819897132003) bank131179 +131180 POINT(47.899205639166105 -123.20272329814748) bank131180 +131181 POINT(46.85215659889014 -121.73628097904925) bank131181 +131182 POINT(48.16896489001214 -121.35615235739661) bank131182 +131183 POINT(46.617655188276494 -121.71554028182115) bank131183 +131184 POINT(47.45781721302565 -121.83504211034102) bank131184 +131185 POINT(48.22637909194999 -121.67672903084242) bank131185 +131186 POINT(48.129585517576515 -122.12370807100798) bank131186 +131187 POINT(47.34088015415322 -121.95350445264918) bank131187 +131188 POINT(47.49969978235629 -123.0664327385967) bank131188 +131189 POINT(48.43744023303515 -122.83161133631906) bank131189 +131190 POINT(48.47454921370098 -121.76449501232176) bank131190 +131191 POINT(46.911855153267396 -122.7497607000537) bank131191 +131192 POINT(48.5089757301878 -121.6333318537233) bank131192 +131193 POINT(47.2118729006461 -121.76984669220997) bank131193 +131194 POINT(47.42100683215853 -121.42918048715832) bank131194 +131195 POINT(46.6796426014762 -122.07975551028228) bank131195 +131196 POINT(47.31051090850718 -121.5068016538708) bank131196 +131197 POINT(48.09200769120928 -122.89743245954907) bank131197 +131198 POINT(48.24396169631437 -121.74391809624329) bank131198 +131199 POINT(46.68133205818697 -122.75726493777037) bank131199 +131200 POINT(47.12399319568531 -122.19634601840332) bank131200 +131201 POINT(47.270465716319514 -121.57789138202044) bank131201 +131202 POINT(47.6738553086898 -122.69092876887495) bank131202 +131203 POINT(47.5070845518681 -122.8421662204334) bank131203 +131204 POINT(47.71870022541311 -121.45393959924795) bank131204 +131205 POINT(47.382517667394794 -121.95394001946573) bank131205 +131206 POINT(47.307902058085006 -122.55269774986616) bank131206 +131207 POINT(48.53340059847992 -121.72424083517234) bank131207 +131208 POINT(46.84341133958373 -123.26714123153538) bank131208 +131209 POINT(48.17028881030496 -122.67922458524305) bank131209 +131210 POINT(48.04093892333641 -121.85795121278117) bank131210 +131211 POINT(47.50475562739411 -123.1606232586236) bank131211 +131212 POINT(48.21254884063277 -121.48993230279169) bank131212 +131213 POINT(48.01407187903046 -121.41238909563411) bank131213 +131214 POINT(47.840396563319985 -121.52729744093003) bank131214 +131215 POINT(47.90428045374887 -121.39692789542818) bank131215 +131216 POINT(47.116027291975314 -123.23874324105768) bank131216 +131217 POINT(48.048249468702664 -121.56656782014194) bank131217 +131218 POINT(47.405267423130674 -121.63318855322674) bank131218 +131219 POINT(48.3705600099644 -121.5118136197422) bank131219 +131220 POINT(46.99794035671378 -121.98446380360245) bank131220 +131221 POINT(48.306073385063534 -122.14836915516966) bank131221 +131222 POINT(48.40330754427749 -121.8261651371105) bank131222 +131223 POINT(48.48936910937051 -121.7821732856819) bank131223 +131224 POINT(48.54146192180211 -122.88608370683275) bank131224 +131225 POINT(47.699309706588636 -122.30448140551123) bank131225 +131226 POINT(48.47067993720216 -122.43805800710363) bank131226 +131227 POINT(47.895581981682625 -121.4703519457875) bank131227 +131228 POINT(47.76511599582599 -122.8202578515037) bank131228 +131229 POINT(48.18532608597598 -121.72876647574883) bank131229 +131230 POINT(47.20449886130514 -122.76889440093935) bank131230 +131231 POINT(47.17303089439653 -122.8482757507624) bank131231 +131232 POINT(46.69961046900325 -122.97684797831153) bank131232 +131233 POINT(47.8894379346142 -122.61733851369196) bank131233 +131234 POINT(47.05157354874204 -122.58566216051827) bank131234 +131235 POINT(46.996592372473394 -121.9171721086987) bank131235 +131236 POINT(46.90523318771381 -122.41212866367964) bank131236 +131237 POINT(47.904077984898684 -121.49069567523105) bank131237 +131238 POINT(46.614004659174554 -122.99711877412511) bank131238 +131239 POINT(46.82233343043917 -122.61647325031895) bank131239 +131240 POINT(48.51397823492687 -122.38141128327389) bank131240 +131241 POINT(46.61686248479287 -122.59192544585912) bank131241 +131242 POINT(46.63817428152922 -123.28966794940543) bank131242 +131243 POINT(47.21595862679963 -122.52821810296903) bank131243 +131244 POINT(48.55203748297816 -122.96662796649433) bank131244 +131245 POINT(47.75103164410177 -122.26652187135024) bank131245 +131246 POINT(47.972467144999776 -121.67228545581227) bank131246 +131247 POINT(48.5137682588868 -122.02597867438871) bank131247 +131248 POINT(48.49518036016905 -122.03260391414071) bank131248 +131249 POINT(47.802243526574 -121.80782950316384) bank131249 +131250 POINT(47.98994875464881 -123.09444314743655) bank131250 +131251 POINT(47.910701513332526 -122.01488819742649) bank131251 +131252 POINT(47.40747902539802 -121.51049622883916) bank131252 +131253 POINT(47.787287024164335 -122.47893376595019) bank131253 +131254 POINT(47.3444437523471 -121.9818881012179) bank131254 +131255 POINT(48.48442026175585 -122.81447728826579) bank131255 +131256 POINT(48.51502959074178 -121.49047243070837) bank131256 +131257 POINT(47.13406761862238 -121.42597223755952) bank131257 +131258 POINT(47.058683046018956 -123.00163235419811) bank131258 +131259 POINT(48.354979992786404 -122.25287940755624) bank131259 +131260 POINT(46.80992033579744 -121.8278660689267) bank131260 +131261 POINT(46.79422701536767 -121.64706779522183) bank131261 +131262 POINT(46.9885024644569 -123.24204122286716) bank131262 +131263 POINT(46.87201927109365 -121.89033193341089) bank131263 +131264 POINT(46.80378100793229 -122.63375238431797) bank131264 +131265 POINT(48.49283927983262 -122.49189166526808) bank131265 +131266 POINT(48.35231464315538 -122.5860985274796) bank131266 +131267 POINT(48.07898080331395 -122.33450798353067) bank131267 +131268 POINT(47.99276387593399 -122.28956450032007) bank131268 +131269 POINT(47.421758089301356 -123.01141114106625) bank131269 +131270 POINT(46.6929119084044 -122.55117946325947) bank131270 +131271 POINT(47.579733948794775 -122.9690174946716) bank131271 +131272 POINT(46.62967617555318 -123.12881705628647) bank131272 +131273 POINT(48.29978203363109 -121.73971217334041) bank131273 +131274 POINT(47.82594494903872 -122.4042540338928) bank131274 +131275 POINT(47.125156855407205 -121.99021363714947) bank131275 +131276 POINT(47.78346449970177 -122.5052610821692) bank131276 +131277 POINT(48.27726411233334 -122.87509292797841) bank131277 +131278 POINT(48.165999107687796 -122.38056934654063) bank131278 +131279 POINT(47.41710952132316 -122.59563165955448) bank131279 +131280 POINT(46.786645308409405 -123.04293979734702) bank131280 +131281 POINT(47.17836747694597 -121.79480799530091) bank131281 +131282 POINT(47.55854970454681 -121.33819582732743) bank131282 +131283 POINT(48.107944768983735 -122.18598985770589) bank131283 +131284 POINT(47.562102168957125 -121.55639153023532) bank131284 +131285 POINT(47.22895020064805 -123.30306263216085) bank131285 +131286 POINT(47.32545101430457 -123.12235295460297) bank131286 +131287 POINT(46.609458706732845 -122.0382040842779) bank131287 +131288 POINT(48.538425493908676 -122.44757916097338) bank131288 +131289 POINT(47.1642469803624 -122.63580952200124) bank131289 +131290 POINT(47.51043175546565 -122.74187690840583) bank131290 +131291 POINT(47.19948119220498 -121.70692045326139) bank131291 +131292 POINT(47.02670983783183 -122.51479076636257) bank131292 +131293 POINT(48.13497373662651 -121.76393765195057) bank131293 +131294 POINT(47.706064138112374 -121.84500221246788) bank131294 +131295 POINT(47.558381113617266 -122.12130138501321) bank131295 +131296 POINT(47.305046006464806 -122.78432749069518) bank131296 +131297 POINT(48.408930176294085 -123.25313254415714) bank131297 +131298 POINT(47.3977310041748 -122.82999069371063) bank131298 +131299 POINT(47.53146840033982 -122.94616893353606) bank131299 +131300 POINT(48.53252609449937 -121.45764103359143) bank131300 +131301 POINT(48.035885354671976 -121.68936192312952) bank131301 +131302 POINT(48.49308515399664 -122.58779681835169) bank131302 +131303 POINT(47.63864455470169 -121.84991143209375) bank131303 +131304 POINT(47.02022581981264 -121.45297096958582) bank131304 +131305 POINT(48.10973230740993 -122.90897925580758) bank131305 +131306 POINT(47.49809630517971 -122.92413432967695) bank131306 +131307 POINT(48.04911801681165 -123.17737953631274) bank131307 +131308 POINT(48.40301318896234 -121.64721483533734) bank131308 +131309 POINT(46.62066185791438 -122.69157258054298) bank131309 +131310 POINT(47.99018079161941 -121.66613727621447) bank131310 +131311 POINT(47.229588997988685 -122.97686591833805) bank131311 +131312 POINT(47.369419824363774 -122.09515218091418) bank131312 +131313 POINT(47.75864549946652 -123.05544943818902) bank131313 +131314 POINT(46.86420594024008 -122.14517412102064) bank131314 +131315 POINT(46.80017672146389 -121.95593934189041) bank131315 +131316 POINT(47.75658257360616 -121.80399735805547) bank131316 +131317 POINT(48.24142375920382 -123.11105603533606) bank131317 +131318 POINT(46.855178423790804 -121.52955045603841) bank131318 +131319 POINT(46.72916065298654 -121.77810469308145) bank131319 +131320 POINT(46.7065112403596 -122.08576670493365) bank131320 +131321 POINT(46.65364311408958 -123.09558427130243) bank131321 +131322 POINT(47.7283696908476 -121.78823869310182) bank131322 +131323 POINT(47.06017323381647 -121.74837270472504) bank131323 +131324 POINT(47.808922370761415 -122.3992802349476) bank131324 +131325 POINT(47.06624101392047 -121.4427717630105) bank131325 +131326 POINT(48.11202794860148 -123.24171221930489) bank131326 +131327 POINT(48.29252544025119 -121.72758220518574) bank131327 +131328 POINT(46.664407655718406 -122.69240399801272) bank131328 +131329 POINT(47.221191687010005 -122.04815816749407) bank131329 +131330 POINT(47.60039143358742 -122.02805237875755) bank131330 +131331 POINT(47.51087083931265 -122.44296104499877) bank131331 +131332 POINT(47.93772703091894 -121.9823048325528) bank131332 +131333 POINT(48.49633451399683 -121.92299825505427) bank131333 +131334 POINT(48.2020232078295 -121.85904581071054) bank131334 +131335 POINT(47.89176553185514 -122.11891955774239) bank131335 +131336 POINT(46.73624814103012 -122.10196187861168) bank131336 +131337 POINT(48.05397061287198 -122.76951830535516) bank131337 +131338 POINT(48.36710931620829 -122.76664442666427) bank131338 +131339 POINT(48.12476185368433 -122.95389877878664) bank131339 +131340 POINT(48.07174241357314 -122.76764855008142) bank131340 +131341 POINT(47.35542541536552 -121.44297167949287) bank131341 +131342 POINT(47.559198141548954 -122.32171554293116) bank131342 +131343 POINT(47.22059113729004 -122.56120394725146) bank131343 +131344 POINT(46.79915869287952 -121.67824653135867) bank131344 +131345 POINT(48.18125030270487 -122.94265321463077) bank131345 +131346 POINT(47.29285869647609 -122.73325467087069) bank131346 +131347 POINT(47.56725594522634 -121.89562338777539) bank131347 +131348 POINT(48.25880017099733 -123.11597196936397) bank131348 +131349 POINT(46.88239219357959 -122.43541425423524) bank131349 +131350 POINT(46.6869235841943 -122.61713938812007) bank131350 +131351 POINT(47.15305802386874 -121.36119587951184) bank131351 +131352 POINT(47.29419471508444 -122.49502881858562) bank131352 +131353 POINT(47.20307772946493 -121.39371838919091) bank131353 +131354 POINT(46.65950132714543 -122.44267547657503) bank131354 +131355 POINT(47.46181648615456 -121.73232074705217) bank131355 +131356 POINT(48.10879542738587 -123.0085295894243) bank131356 +131357 POINT(46.978978996712506 -121.39651207397937) bank131357 +131358 POINT(47.418708824460104 -121.95600538116958) bank131358 +131359 POINT(47.01734401025157 -122.69772505321365) bank131359 +131360 POINT(47.45806905352561 -121.51148812843908) bank131360 +131361 POINT(47.99138111696776 -122.7389078445244) bank131361 +131362 POINT(47.45965781864135 -122.1692864221764) bank131362 +131363 POINT(48.27227406030306 -122.43252079066816) bank131363 +131364 POINT(47.30795792332047 -121.70943453200105) bank131364 +131365 POINT(47.63839980338583 -122.46845321924205) bank131365 +131366 POINT(47.13218443332457 -122.97936642672931) bank131366 +131367 POINT(47.15186702809389 -121.4945790533633) bank131367 +131368 POINT(47.484658775846746 -122.1145328918974) bank131368 +131369 POINT(47.67946903280743 -122.86238077293547) bank131369 +131370 POINT(47.98669425060405 -122.56430690259704) bank131370 +131371 POINT(48.27435833472303 -122.28708280674324) bank131371 +131372 POINT(47.45457469695574 -121.56416012038136) bank131372 +131373 POINT(47.883478511762924 -121.49146927532495) bank131373 +131374 POINT(47.258570669255114 -122.30266017389351) bank131374 +131375 POINT(47.84852035710489 -121.51666434159142) bank131375 +131376 POINT(48.21129532689549 -122.65077367874385) bank131376 +131377 POINT(46.64263945932857 -123.29748670211248) bank131377 +131378 POINT(48.29486680012152 -122.05818703271387) bank131378 +131379 POINT(47.236720901064984 -122.54913315241433) bank131379 +131380 POINT(47.16820387182536 -122.93336608621235) bank131380 +131381 POINT(47.43797403240589 -122.67946269291306) bank131381 +131382 POINT(47.02293100843554 -122.42385283184535) bank131382 +131383 POINT(48.47079592626545 -121.72130004578425) bank131383 +131384 POINT(46.85376253490328 -122.18116017193688) bank131384 +131385 POINT(47.6695669219899 -121.62238103356339) bank131385 +131386 POINT(46.840700055764486 -122.96413977018533) bank131386 +131387 POINT(47.181328117561705 -122.88379476674844) bank131387 +131388 POINT(47.55026170387087 -122.2412995068724) bank131388 +131389 POINT(47.79422691457039 -122.4816237596539) bank131389 +131390 POINT(46.914703448912356 -121.96920758068147) bank131390 +131391 POINT(47.26350151183853 -122.04405746009546) bank131391 +131392 POINT(48.01228925785405 -122.39804785836844) bank131392 +131393 POINT(48.06687349478021 -121.76526254936499) bank131393 +131394 POINT(46.71489392267343 -122.43057366859748) bank131394 +131395 POINT(48.10864064263688 -123.09969433858136) bank131395 +131396 POINT(47.64909189226143 -121.7694977084936) bank131396 +131397 POINT(47.90040383886222 -121.6546932787474) bank131397 +131398 POINT(47.98991111941549 -122.26423993570064) bank131398 +131399 POINT(47.42022221813706 -122.55726931550589) bank131399 +131400 POINT(47.688414533664755 -122.28665932397283) bank131400 +131401 POINT(47.09219014056924 -123.14953161512004) bank131401 +131402 POINT(47.25397672686546 -122.94153855386625) bank131402 +131403 POINT(47.77599927482485 -121.81048988057053) bank131403 +131404 POINT(48.13328819144151 -123.1688004239854) bank131404 +131405 POINT(46.88897964087539 -122.29798954270912) bank131405 +131406 POINT(48.01843566404189 -122.2254264301668) bank131406 +131407 POINT(47.623700876858706 -122.80429413733535) bank131407 +131408 POINT(48.23080077014903 -122.46173941409324) bank131408 +131409 POINT(47.82903568329733 -121.39506003200464) bank131409 +131410 POINT(46.611703913913914 -122.4342794836625) bank131410 +131411 POINT(46.92795534504403 -122.14203639006571) bank131411 +131412 POINT(47.7472761728394 -122.78584274444285) bank131412 +131413 POINT(47.941672868985094 -122.78479248751871) bank131413 +131414 POINT(48.03519640628268 -121.45931201589747) bank131414 +131415 POINT(46.96408311034485 -121.71793615897238) bank131415 +131416 POINT(47.70922653672525 -121.64044026009081) bank131416 +131417 POINT(48.29308653683561 -121.79185810456077) bank131417 +131418 POINT(47.09938443855654 -122.1621350299759) bank131418 +131419 POINT(47.36918925939308 -121.76443267056091) bank131419 +131420 POINT(47.9763504366686 -122.33883547418046) bank131420 +131421 POINT(47.68251088746628 -122.96282474655595) bank131421 +131422 POINT(47.774173413453255 -121.90896142897711) bank131422 +131423 POINT(47.16556699189317 -121.72972114646757) bank131423 +131424 POINT(48.12515206066998 -122.87089266638158) bank131424 +131425 POINT(47.37737181264195 -121.39803369766176) bank131425 +131426 POINT(48.2332472887558 -122.64839056358542) bank131426 +131427 POINT(47.956039793007015 -122.21281915986428) bank131427 +131428 POINT(48.60361827014522 -121.34582526287684) bank131428 +131429 POINT(47.12690265645031 -121.82508844888989) bank131429 +131430 POINT(47.94369381792891 -121.76645350110213) bank131430 +131431 POINT(47.11920423406137 -121.55923029217047) bank131431 +131432 POINT(47.88633496636719 -122.28769776524953) bank131432 +131433 POINT(48.062644552510314 -122.32029311532922) bank131433 +131434 POINT(46.756864273290084 -122.8698551145015) bank131434 +131435 POINT(46.725391603074854 -121.6700013752251) bank131435 +131436 POINT(47.71713356247968 -121.75899220354079) bank131436 +131437 POINT(48.09169465777459 -122.01674746661062) bank131437 +131438 POINT(46.6168690598016 -123.31883038812785) bank131438 +131439 POINT(48.15805749021811 -122.97401464743399) bank131439 +131440 POINT(46.72299742617011 -122.6720643225644) bank131440 +131441 POINT(48.189560686035385 -121.73589366663049) bank131441 +131442 POINT(48.22567593106309 -122.05388437777292) bank131442 +131443 POINT(47.433590312519236 -122.48477375708025) bank131443 +131444 POINT(48.0562017949699 -123.1124934648652) bank131444 +131445 POINT(48.31646161928603 -122.41626191650184) bank131445 +131446 POINT(48.12502622817021 -122.39584390684506) bank131446 +131447 POINT(48.40825200516554 -122.69847046356541) bank131447 +131448 POINT(46.91528690689938 -122.51663699875597) bank131448 +131449 POINT(47.31350133319299 -121.58069945784476) bank131449 +131450 POINT(48.08441801526124 -121.91362879511948) bank131450 +131451 POINT(48.141245683914924 -121.84789678401769) bank131451 +131452 POINT(47.49129456508918 -121.76392734366023) bank131452 +131453 POINT(46.68552987045884 -122.1950013919012) bank131453 +131454 POINT(48.534943930755894 -121.80482684258361) bank131454 +131455 POINT(46.998456776879614 -121.90531119766028) bank131455 +131456 POINT(48.07275631744476 -123.2960715905538) bank131456 +131457 POINT(48.02101532255283 -121.73264952873716) bank131457 +131458 POINT(47.28458634106598 -122.597609718674) bank131458 +131459 POINT(47.90165298926643 -121.57244505169332) bank131459 +131460 POINT(47.65677398234029 -121.34292636691622) bank131460 +131461 POINT(47.82863548292019 -121.67155458815101) bank131461 +131462 POINT(48.0380181096337 -121.91619537512776) bank131462 +131463 POINT(47.381360640800594 -121.4123862638856) bank131463 +131464 POINT(47.752214752604765 -121.40718671383657) bank131464 +131465 POINT(48.590946178561445 -121.87119980667114) bank131465 +131466 POINT(48.048885830054815 -122.16890523578166) bank131466 +131467 POINT(46.95927486565554 -123.03440079265519) bank131467 +131468 POINT(48.216261073415545 -123.08242720834905) bank131468 +131469 POINT(48.3390206102964 -121.84670122387814) bank131469 +131470 POINT(47.80629933963495 -121.85327404257669) bank131470 +131471 POINT(48.42605890413593 -122.85792347707148) bank131471 +131472 POINT(46.72436157498593 -121.64313793775648) bank131472 +131473 POINT(48.16636595750018 -122.50925442565693) bank131473 +131474 POINT(46.97906462857843 -121.76229620134609) bank131474 +131475 POINT(46.75086560061993 -122.41183511165782) bank131475 +131476 POINT(48.55249677813863 -121.91376147663253) bank131476 +131477 POINT(47.326780841217555 -122.7446414697025) bank131477 +131478 POINT(47.27172567177063 -122.73697810195317) bank131478 +131479 POINT(48.490853600265616 -122.6813796922517) bank131479 +131480 POINT(47.832614028841036 -122.11590147100411) bank131480 +131481 POINT(47.051101975665134 -123.12616710519053) bank131481 +131482 POINT(47.63771786021472 -121.55130301356454) bank131482 +131483 POINT(47.76187276916069 -122.23151167388771) bank131483 +131484 POINT(48.32805080358472 -122.99380781751526) bank131484 +131485 POINT(48.236350637647334 -123.07578162312775) bank131485 +131486 POINT(47.87128986278505 -122.73841267156921) bank131486 +131487 POINT(48.034740118070346 -122.09161015279963) bank131487 +131488 POINT(47.96025289986438 -122.87365754516746) bank131488 +131489 POINT(47.11076281335623 -123.02756100599339) bank131489 +131490 POINT(47.0154675418951 -122.49144685435162) bank131490 +131491 POINT(47.97541843452704 -122.12021244634649) bank131491 +131492 POINT(46.70657083075514 -121.83044550540747) bank131492 +131493 POINT(48.391143629676854 -122.04946490728052) bank131493 +131494 POINT(47.18754520293116 -122.46992387040221) bank131494 +131495 POINT(48.37699509075652 -121.88412673806232) bank131495 +131496 POINT(47.65989444312826 -121.52584664166982) bank131496 +131497 POINT(47.445245834074505 -121.54193869176206) bank131497 +131498 POINT(48.405920860070495 -121.7394606361086) bank131498 +131499 POINT(47.732722908696005 -122.2168996382106) bank131499 +131500 POINT(46.918350318455595 -121.72625888695549) bank131500 +131501 POINT(46.84194138359443 -123.20769363724577) bank131501 +131502 POINT(47.52336698564388 -122.4715474924451) bank131502 +131503 POINT(46.632673589899476 -123.11933113827749) bank131503 +131504 POINT(47.96160956894046 -122.78906775733013) bank131504 +131505 POINT(47.99147462927369 -122.04967910444363) bank131505 +131506 POINT(47.049755575646316 -122.21200097756066) bank131506 +131507 POINT(46.68201873323119 -122.59850733936412) bank131507 +131508 POINT(48.376396276226025 -121.34940203022337) bank131508 +131509 POINT(48.345888276550184 -122.01079761854054) bank131509 +131510 POINT(46.8705080311099 -122.45569847000303) bank131510 +131511 POINT(47.42094894625135 -121.79931131432559) bank131511 +131512 POINT(47.53392973728625 -122.46431410329396) bank131512 +131513 POINT(47.65009664531216 -122.45335147637395) bank131513 +131514 POINT(47.18753094036545 -122.46455734594358) bank131514 +131515 POINT(47.46258857770973 -122.36735397244365) bank131515 +131516 POINT(46.846147763579744 -122.55820449942667) bank131516 +131517 POINT(48.4376253998346 -123.3187544812157) bank131517 +131518 POINT(47.13825284898155 -123.09706310077671) bank131518 +131519 POINT(48.349392394982864 -122.38302073035142) bank131519 +131520 POINT(47.41993567045501 -122.97268451413989) bank131520 +131521 POINT(46.99333833871268 -121.8282987050169) bank131521 +131522 POINT(46.952729638430355 -122.41565902339339) bank131522 +131523 POINT(48.580154475576066 -121.47554499847557) bank131523 +131524 POINT(48.26510966712736 -121.64017727751052) bank131524 +131525 POINT(47.87762409585075 -123.1617158450481) bank131525 +131526 POINT(47.86768559683991 -123.21552975201692) bank131526 +131527 POINT(47.65302130755356 -122.6233753799768) bank131527 +131528 POINT(47.19761376633217 -123.00636831792913) bank131528 +131529 POINT(47.252400464609025 -122.37423296899648) bank131529 +131530 POINT(48.184429945383606 -121.53570976155902) bank131530 +131531 POINT(47.428658288565146 -122.995404425915) bank131531 +131532 POINT(47.836788838142965 -121.9205287306312) bank131532 +131533 POINT(48.11565355527368 -121.93068760667765) bank131533 +131534 POINT(48.378467022813155 -122.94312237121815) bank131534 +131535 POINT(47.58773974165871 -122.44857931536983) bank131535 +131536 POINT(47.908618111478546 -123.28163916774741) bank131536 +131537 POINT(47.28618914058932 -122.78949047419808) bank131537 +131538 POINT(47.44165338140568 -122.8372116715244) bank131538 +131539 POINT(46.8717235548113 -122.76182623100898) bank131539 +131540 POINT(48.545192731061846 -121.33793693749341) bank131540 +131541 POINT(46.94631520752846 -121.78036700611902) bank131541 +131542 POINT(47.41514165999095 -121.41302647290082) bank131542 +131543 POINT(47.940527636878045 -123.15630388362615) bank131543 +131544 POINT(47.242141437128325 -123.2042064811944) bank131544 +131545 POINT(46.76966300177861 -121.97485555617638) bank131545 +131546 POINT(48.110286569037285 -122.33538883096416) bank131546 +131547 POINT(48.44318257435505 -122.5463396426324) bank131547 +131548 POINT(48.179262539966786 -122.17634086481024) bank131548 +131549 POINT(47.888614231623386 -122.06914656242412) bank131549 +131550 POINT(47.335084826885875 -121.92730859471624) bank131550 +131551 POINT(47.496244546479986 -122.43752908099593) bank131551 +131552 POINT(47.854207607056416 -123.27933876920834) bank131552 +131553 POINT(47.477214349707815 -123.17165026158003) bank131553 +131554 POINT(48.273037584976436 -122.17035718790855) bank131554 +131555 POINT(47.61762305802075 -121.95109673385292) bank131555 +131556 POINT(47.86831201431784 -121.77298947830813) bank131556 +131557 POINT(47.30024698388449 -121.8347992631084) bank131557 +131558 POINT(47.11022632437621 -122.65821461653286) bank131558 +131559 POINT(47.181198716331124 -121.83983135469003) bank131559 +131560 POINT(48.50086501230226 -122.2650575760314) bank131560 +131561 POINT(47.78378764170891 -123.32510343239207) bank131561 +131562 POINT(47.74679042948786 -122.39011754814354) bank131562 +131563 POINT(46.78778303264913 -122.72720816790557) bank131563 +131564 POINT(46.68386163180273 -121.4340281115334) bank131564 +131565 POINT(47.72475704480398 -121.70730363467338) bank131565 +131566 POINT(47.08006575930356 -122.97778100453174) bank131566 +131567 POINT(48.39761035344849 -122.44853178752842) bank131567 +131568 POINT(48.14333080365774 -121.71463698823536) bank131568 +131569 POINT(47.645869561542725 -122.94671595001029) bank131569 +131570 POINT(48.12596956805188 -123.06557296946387) bank131570 +131571 POINT(47.75115618115443 -122.94753420270455) bank131571 +131572 POINT(48.33712774309043 -123.2547268820108) bank131572 +131573 POINT(48.48589566917863 -121.69411485029167) bank131573 +131574 POINT(47.08065084775248 -122.29027326833685) bank131574 +131575 POINT(47.98700413319103 -121.87808129841498) bank131575 +131576 POINT(47.97946095672148 -121.831777233199) bank131576 +131577 POINT(47.88088127561148 -121.39670769052154) bank131577 +131578 POINT(47.56964934686949 -121.38333910213909) bank131578 +131579 POINT(48.434655077556506 -122.31506136371789) bank131579 +131580 POINT(48.16591228030414 -122.91911082103493) bank131580 +131581 POINT(46.663018220250734 -121.80267401257005) bank131581 +131582 POINT(47.73597471419583 -121.34993127239834) bank131582 +131583 POINT(47.73230783566494 -122.95487290157162) bank131583 +131584 POINT(47.36778948544111 -121.58896796887923) bank131584 +131585 POINT(48.453700657570415 -122.75710124358129) bank131585 +131586 POINT(46.76641597277726 -122.66923892789092) bank131586 +131587 POINT(48.21779548726252 -122.41584337611327) bank131587 +131588 POINT(47.871295387469225 -121.76589101259769) bank131588 +131589 POINT(48.18080375548346 -123.19450542423984) bank131589 +131590 POINT(48.36862669658443 -122.21951362789812) bank131590 +131591 POINT(48.386589927069835 -122.43422450444852) bank131591 +131592 POINT(48.17629138072093 -121.86017001033434) bank131592 +131593 POINT(47.56706760189653 -121.69257290049195) bank131593 +131594 POINT(47.598337296778574 -123.19733172073926) bank131594 +131595 POINT(47.87686895986616 -122.49405819583205) bank131595 +131596 POINT(48.60124632908425 -122.04010746643205) bank131596 +131597 POINT(47.88437572874217 -122.44357408053051) bank131597 +131598 POINT(46.683434681371466 -122.89742726864898) bank131598 +131599 POINT(47.78551269059351 -122.66443351525243) bank131599 +131600 POINT(47.5305645538985 -122.63654984626926) bank131600 +131601 POINT(47.68471622480059 -122.78456760330639) bank131601 +131602 POINT(47.73177358873333 -122.12170921614612) bank131602 +131603 POINT(47.12376552678304 -123.00751115089861) bank131603 +131604 POINT(46.63938643463108 -122.22072644389426) bank131604 +131605 POINT(48.14069419685009 -121.6502529972965) bank131605 +131606 POINT(47.404472823699116 -122.30165661563935) bank131606 +131607 POINT(46.63002856709602 -122.90667950094367) bank131607 +131608 POINT(47.710407772035346 -122.26928622177934) bank131608 +131609 POINT(47.47601090311723 -121.78912791430507) bank131609 +131610 POINT(47.06367288286384 -121.57233003947633) bank131610 +131611 POINT(47.36465225826073 -123.29549372121852) bank131611 +131612 POINT(47.14193605923942 -122.46206183260512) bank131612 +131613 POINT(48.580730361368026 -123.22798848697853) bank131613 +131614 POINT(47.212899696318544 -121.88367532780225) bank131614 +131615 POINT(47.2260458145601 -121.80183451259695) bank131615 +131616 POINT(47.94304959925081 -122.80368501790097) bank131616 +131617 POINT(48.395110754813494 -123.15263328528756) bank131617 +131618 POINT(47.893932604135095 -122.84690385248327) bank131618 +131619 POINT(47.4712312881102 -121.45022204739082) bank131619 +131620 POINT(48.134983761426334 -122.4895682810185) bank131620 +131621 POINT(46.786593387802746 -122.96524898301098) bank131621 +131622 POINT(46.90290524826037 -121.96153704181889) bank131622 +131623 POINT(47.42054808210197 -121.67955205550807) bank131623 +131624 POINT(47.07457350908248 -122.51429972389684) bank131624 +131625 POINT(47.5903946837909 -122.65684427714682) bank131625 +131626 POINT(46.99707861442719 -123.17490127328625) bank131626 +131627 POINT(47.40809669988232 -122.20519967996928) bank131627 +131628 POINT(47.46992654318365 -121.79470813784002) bank131628 +131629 POINT(46.72235169786859 -122.70194263706902) bank131629 +131630 POINT(47.57726040383825 -122.17195910049419) bank131630 +131631 POINT(47.65003631611907 -122.53565311337034) bank131631 +131632 POINT(46.97607576249395 -122.75546405780355) bank131632 +131633 POINT(48.43397681245492 -121.5767268279516) bank131633 +131634 POINT(47.42099689314933 -121.56454595696233) bank131634 +131635 POINT(47.29264195750852 -121.93170853696861) bank131635 +131636 POINT(48.37920166201409 -122.7190625853888) bank131636 +131637 POINT(48.13394773615332 -123.10937765790175) bank131637 +131638 POINT(46.75594172394069 -121.67321242924729) bank131638 +131639 POINT(47.403512814703454 -122.05713465671903) bank131639 +131640 POINT(48.12172642082772 -122.78446122217018) bank131640 +131641 POINT(48.09172870646534 -121.75062055586154) bank131641 +131642 POINT(47.61764920753191 -122.16274225513409) bank131642 +131643 POINT(46.85991331693928 -121.33700860184263) bank131643 +131644 POINT(47.94661912032648 -121.82615420263606) bank131644 +131645 POINT(47.90369201534546 -121.4037443824104) bank131645 +131646 POINT(47.597891353810994 -121.77434525955948) bank131646 +131647 POINT(47.32155048164669 -122.86703372365241) bank131647 +131648 POINT(47.25480635917679 -122.90099445421257) bank131648 +131649 POINT(46.806512156294296 -122.20719216765097) bank131649 +131650 POINT(47.41552590581255 -123.09873498129498) bank131650 +131651 POINT(47.70861541503261 -122.55486188255362) bank131651 +131652 POINT(48.33640338023622 -121.63658926202226) bank131652 +131653 POINT(46.98036449789725 -122.45904823687219) bank131653 +131654 POINT(47.57238863175278 -122.16812127013715) bank131654 +131655 POINT(47.60109432572221 -122.2131067851769) bank131655 +131656 POINT(48.5708419310948 -123.15767535719537) bank131656 +131657 POINT(46.89094269115761 -121.64616502747099) bank131657 +131658 POINT(47.31267818890503 -122.41936687617462) bank131658 +131659 POINT(47.75843472339704 -122.27219742389663) bank131659 +131660 POINT(48.466370089473884 -121.87713074301318) bank131660 +131661 POINT(47.541982602436846 -121.79865109659058) bank131661 +131662 POINT(47.453838035404715 -123.05248945149313) bank131662 +131663 POINT(48.50453284812941 -123.21774081232672) bank131663 +131664 POINT(47.07376240761947 -122.74144896112024) bank131664 +131665 POINT(48.1943594462055 -121.91771389850719) bank131665 +131666 POINT(48.28972399418838 -121.67501170728485) bank131666 +131667 POINT(47.24113936347866 -123.24960941221246) bank131667 +131668 POINT(47.17535814776117 -123.01597772476741) bank131668 +131669 POINT(46.931632914438794 -121.62575697425136) bank131669 +131670 POINT(48.35948216838618 -123.15568550820302) bank131670 +131671 POINT(47.101451869714495 -121.5123045289691) bank131671 +131672 POINT(47.92554290276093 -121.68118851801393) bank131672 +131673 POINT(46.75730736830586 -121.45360047455692) bank131673 +131674 POINT(47.9235002463556 -122.41917597738656) bank131674 +131675 POINT(46.64784177528906 -123.11123868119589) bank131675 +131676 POINT(48.047140790526846 -121.55911612338204) bank131676 +131677 POINT(47.1958753609647 -122.45309155217895) bank131677 +131678 POINT(47.72923483808713 -123.06012428544504) bank131678 +131679 POINT(48.05159405365278 -122.24525859798177) bank131679 +131680 POINT(47.42547099982094 -122.94829584961673) bank131680 +131681 POINT(47.403262103767055 -121.98914411129672) bank131681 +131682 POINT(46.79565223623376 -121.70803166908016) bank131682 +131683 POINT(47.812481123199 -123.13355920865223) bank131683 +131684 POINT(48.160370524751 -121.83752959448084) bank131684 +131685 POINT(48.12803983576186 -121.3944173212713) bank131685 +131686 POINT(48.3755810142032 -123.16912196968136) bank131686 +131687 POINT(46.87535009930879 -122.4157218055932) bank131687 +131688 POINT(48.52309861158135 -122.19774701577367) bank131688 +131689 POINT(48.032999322148065 -121.54828550584529) bank131689 +131690 POINT(47.1274564913077 -121.78405287873056) bank131690 +131691 POINT(47.59137891315202 -123.08612841250088) bank131691 +131692 POINT(47.17363316229615 -121.86184865740448) bank131692 +131693 POINT(47.68137214791341 -122.7855679966195) bank131693 +131694 POINT(48.38782022915083 -121.99706597537882) bank131694 +131695 POINT(48.42252335763464 -122.39503294872522) bank131695 +131696 POINT(47.56282322704651 -122.41287549991439) bank131696 +131697 POINT(48.22217458149611 -122.163460971247) bank131697 +131698 POINT(47.94035893210691 -122.31518960377859) bank131698 +131699 POINT(48.582218236551014 -122.90671759078442) bank131699 +131700 POINT(48.18548617829469 -122.24400900866327) bank131700 +131701 POINT(47.158656260466664 -123.07746019606827) bank131701 +131702 POINT(46.978720302681424 -121.83581266494046) bank131702 +131703 POINT(47.06567449951029 -121.47313402851039) bank131703 +131704 POINT(47.94294398957475 -122.24881606247634) bank131704 +131705 POINT(48.6041443872116 -122.07699595850016) bank131705 +131706 POINT(47.93422290959623 -122.21922360733863) bank131706 +131707 POINT(47.78270981962181 -123.16197706861401) bank131707 +131708 POINT(48.13684654699503 -123.1091629509277) bank131708 +131709 POINT(48.37065878213403 -121.74514356884234) bank131709 +131710 POINT(48.24893803228548 -122.32210928481557) bank131710 +131711 POINT(47.07567592125608 -122.03273262899364) bank131711 +131712 POINT(47.63283241844041 -123.0437874378998) bank131712 +131713 POINT(48.02689041144892 -122.13660675954782) bank131713 +131714 POINT(47.75506506597029 -122.08670259502941) bank131714 +131715 POINT(48.47572038871671 -123.25570612509554) bank131715 +131716 POINT(48.03457878237548 -123.32119723444836) bank131716 +131717 POINT(46.7138737184504 -122.96982049919886) bank131717 +131718 POINT(48.60154266704039 -122.17635969385526) bank131718 +131719 POINT(47.35606145950781 -123.01711293448072) bank131719 +131720 POINT(46.64214325626683 -122.83204064896648) bank131720 +131721 POINT(48.49132238344329 -121.5959494599586) bank131721 +131722 POINT(48.23148730820957 -122.96683856696599) bank131722 +131723 POINT(48.154433715699206 -122.9986230575613) bank131723 +131724 POINT(47.273395159658946 -123.18716072230477) bank131724 +131725 POINT(46.673424129475116 -123.3281429800172) bank131725 +131726 POINT(47.446268355511585 -122.96068562878666) bank131726 +131727 POINT(47.081538601952175 -122.82271787515603) bank131727 +131728 POINT(47.563447626232936 -122.01211224382241) bank131728 +131729 POINT(48.41748160332716 -121.86495959348575) bank131729 +131730 POINT(47.78612863433817 -121.62714652519279) bank131730 +131731 POINT(46.82528588333672 -121.97277297724317) bank131731 +131732 POINT(48.0260692140078 -121.37553094159875) bank131732 +131733 POINT(46.846666298898256 -121.65310823973763) bank131733 +131734 POINT(47.878612828133235 -122.15733769250365) bank131734 +131735 POINT(47.387480556270965 -122.8687297587474) bank131735 +131736 POINT(48.35319084792803 -123.07820146988762) bank131736 +131737 POINT(47.0495621168064 -121.54495701401451) bank131737 +131738 POINT(47.56849785018741 -121.88659379511236) bank131738 +131739 POINT(48.482477222017685 -122.89855047163762) bank131739 +131740 POINT(48.196645033462794 -121.4196053040909) bank131740 +131741 POINT(47.07984170032611 -122.41952044618289) bank131741 +131742 POINT(47.7508466348256 -121.53689959459388) bank131742 +131743 POINT(46.92860154213937 -121.36282835920434) bank131743 +131744 POINT(46.704838107236085 -122.5475502481537) bank131744 +131745 POINT(47.73064766267347 -122.88701715463853) bank131745 +131746 POINT(47.98922436307394 -121.75813367204316) bank131746 +131747 POINT(47.06327228703966 -121.71650893884271) bank131747 +131748 POINT(48.131126781403154 -121.95686876682261) bank131748 +131749 POINT(46.866811861479896 -121.91624098417171) bank131749 +131750 POINT(47.242268703333195 -122.08192586483615) bank131750 +131751 POINT(48.49297937845046 -121.74828254098708) bank131751 +131752 POINT(47.66975030511228 -122.56120349491152) bank131752 +131753 POINT(48.06735471464991 -122.62377069996576) bank131753 +131754 POINT(47.80239341984465 -122.08896746057029) bank131754 +131755 POINT(48.036990027549486 -121.82788073436492) bank131755 +131756 POINT(47.23749980931934 -121.90835745993027) bank131756 +131757 POINT(47.30781724398825 -122.39827735114152) bank131757 +131758 POINT(47.26604230258333 -122.06704372758695) bank131758 +131759 POINT(47.68257536553139 -123.20086784604251) bank131759 +131760 POINT(47.141169378009245 -121.8821239517432) bank131760 +131761 POINT(46.810192739843814 -122.11421538631315) bank131761 +131762 POINT(48.43926929215462 -122.81688390580449) bank131762 +131763 POINT(48.04686596367754 -121.43049067136087) bank131763 +131764 POINT(47.77343446475573 -122.4828764621232) bank131764 +131765 POINT(48.457981333284216 -122.40657410220604) bank131765 +131766 POINT(47.12939765959704 -123.1494444330881) bank131766 +131767 POINT(47.424389620002 -122.00345860572413) bank131767 +131768 POINT(46.914711011207416 -122.36623816674262) bank131768 +131769 POINT(48.31977754938459 -123.10072865958627) bank131769 +131770 POINT(48.327356078009245 -121.73065048902184) bank131770 +131771 POINT(47.74558594378137 -121.5510018401527) bank131771 +131772 POINT(46.88095459560338 -123.29543349298663) bank131772 +131773 POINT(48.26558948064078 -123.17047844804335) bank131773 +131774 POINT(47.29231488994843 -122.92622391662441) bank131774 +131775 POINT(47.368824332466204 -122.00106134481075) bank131775 +131776 POINT(47.92092362012263 -122.6911183061471) bank131776 +131777 POINT(48.17845984739187 -122.17922950893228) bank131777 +131778 POINT(47.31482926543315 -121.46899436058524) bank131778 +131779 POINT(47.36985611289755 -122.4577779596351) bank131779 +131780 POINT(47.14708076838295 -123.14073206122795) bank131780 +131781 POINT(47.9971871127169 -122.62843766040666) bank131781 +131782 POINT(46.96565638829577 -122.54188962929324) bank131782 +131783 POINT(48.47589858176731 -122.77740363268686) bank131783 +131784 POINT(48.54709664191328 -121.86029865925859) bank131784 +131785 POINT(47.77417889054618 -123.1311095768127) bank131785 +131786 POINT(47.44730001791303 -123.12292812574441) bank131786 +131787 POINT(47.2773822020894 -121.57727600251955) bank131787 +131788 POINT(46.6524457578926 -122.20548227841827) bank131788 +131789 POINT(48.317525735943654 -121.71298724531584) bank131789 +131790 POINT(48.48973329973529 -121.95051518464993) bank131790 +131791 POINT(47.95543851745338 -123.24836322756323) bank131791 +131792 POINT(48.48566002666381 -122.53312060631303) bank131792 +131793 POINT(47.48263993736991 -122.82415921517989) bank131793 +131794 POINT(48.57814726223093 -123.23873283359299) bank131794 +131795 POINT(46.8466508174014 -121.68844674558268) bank131795 +131796 POINT(47.291785431439195 -121.52493930912537) bank131796 +131797 POINT(47.53033665958979 -123.28634672958546) bank131797 +131798 POINT(47.71877886748935 -122.30105000209733) bank131798 +131799 POINT(47.51500012156611 -121.7475988350799) bank131799 +131800 POINT(48.15804355818835 -123.25975251069859) bank131800 +131801 POINT(48.36783792309831 -123.27207958485836) bank131801 +131802 POINT(46.8041417394157 -122.91319806632399) bank131802 +131803 POINT(47.73378127522166 -122.11116554930867) bank131803 +131804 POINT(46.883814360323136 -122.05099111885073) bank131804 +131805 POINT(46.740598461483124 -122.34463341600146) bank131805 +131806 POINT(47.82514234905342 -121.44291133105061) bank131806 +131807 POINT(48.3442348800688 -122.37547957167841) bank131807 +131808 POINT(48.55212815435874 -123.20032284529539) bank131808 +131809 POINT(48.46802792580564 -122.34947508414646) bank131809 +131810 POINT(47.99322388715319 -122.43806249171277) bank131810 +131811 POINT(47.560673565680844 -123.30465997034426) bank131811 +131812 POINT(47.456302196201634 -123.15645663841896) bank131812 +131813 POINT(47.62036704031833 -122.3344563842484) bank131813 +131814 POINT(47.563799085883254 -121.42488047563887) bank131814 +131815 POINT(47.340972146804965 -122.30158636752141) bank131815 +131816 POINT(48.550613530015 -121.42225301333825) bank131816 +131817 POINT(46.77284792711411 -123.07726817352702) bank131817 +131818 POINT(48.16831363122495 -121.4089052509042) bank131818 +131819 POINT(47.72619985301025 -122.2603980623674) bank131819 +131820 POINT(47.55681707763399 -122.78933797042819) bank131820 +131821 POINT(47.29397778044737 -121.79335507817473) bank131821 +131822 POINT(48.302279752406236 -121.62746533634137) bank131822 +131823 POINT(47.16357606643028 -122.57264808168324) bank131823 +131824 POINT(47.28749383233627 -122.63324484276687) bank131824 +131825 POINT(48.28131844823256 -121.62803086708622) bank131825 +131826 POINT(48.15878577628742 -121.50798270896816) bank131826 +131827 POINT(46.772573554907346 -122.47296002712534) bank131827 +131828 POINT(47.49683374524539 -121.97999603346466) bank131828 +131829 POINT(47.150010213463126 -123.10644887661573) bank131829 +131830 POINT(48.45278972426186 -123.0000612437491) bank131830 +131831 POINT(47.29781136606223 -122.6048715672185) bank131831 +131832 POINT(47.385051612741634 -122.99127614051329) bank131832 +131833 POINT(47.11238058305408 -121.78692922112643) bank131833 +131834 POINT(47.692432672890256 -122.79624497195863) bank131834 +131835 POINT(47.02314596870391 -123.22311159191474) bank131835 +131836 POINT(48.32120239463191 -122.41815247755542) bank131836 +131837 POINT(47.390190189091726 -121.47813137679952) bank131837 +131838 POINT(47.39166630700006 -122.86769801523681) bank131838 +131839 POINT(47.04565278269543 -123.30896712135122) bank131839 +131840 POINT(47.67424947796803 -122.80164912136766) bank131840 +131841 POINT(48.43910257817796 -122.54017317113494) bank131841 +131842 POINT(47.38117015717141 -122.24828095807102) bank131842 +131843 POINT(48.11308161222885 -121.94724669874196) bank131843 +131844 POINT(47.036759641094115 -123.03853652521255) bank131844 +131845 POINT(46.97251201662619 -121.53612599337009) bank131845 +131846 POINT(47.63047727688465 -122.06903948388353) bank131846 +131847 POINT(48.603384320279346 -122.05631986546986) bank131847 +131848 POINT(47.943197970050385 -121.85108813328235) bank131848 +131849 POINT(46.8329360503577 -121.34059531117569) bank131849 +131850 POINT(47.28026728703906 -122.45579904761965) bank131850 +131851 POINT(48.20156377387687 -122.50023740170646) bank131851 +131852 POINT(47.07774496562178 -122.31282094273595) bank131852 +131853 POINT(47.722477765150096 -122.54718554241379) bank131853 +131854 POINT(47.76335421422379 -123.18126614420729) bank131854 +131855 POINT(46.69600991553319 -121.91500891126185) bank131855 +131856 POINT(47.961540467098956 -122.45064779079009) bank131856 +131857 POINT(46.9218549712733 -121.45626866629536) bank131857 +131858 POINT(47.09271606172856 -122.20916148750966) bank131858 +131859 POINT(47.218598586046475 -123.03201953441561) bank131859 +131860 POINT(48.24922078562809 -122.8091651138383) bank131860 +131861 POINT(46.909911502859714 -121.81093292536153) bank131861 +131862 POINT(48.46659009110979 -123.27150660442479) bank131862 +131863 POINT(48.08240513192147 -122.82092862833589) bank131863 +131864 POINT(46.84235886408469 -121.36693021164189) bank131864 +131865 POINT(46.90933887182181 -123.11006371910533) bank131865 +131866 POINT(47.34229386318483 -123.22714523259549) bank131866 +131867 POINT(47.985483084676694 -121.61027579122339) bank131867 +131868 POINT(47.633835706935926 -121.96601848043487) bank131868 +131869 POINT(47.36299201304586 -122.17552280718051) bank131869 +131870 POINT(47.58472148799937 -121.87197020602322) bank131870 +131871 POINT(48.05658812527251 -122.6676061277422) bank131871 +131872 POINT(47.21342366237863 -123.25746625263803) bank131872 +131873 POINT(46.8041521945636 -122.74393099506636) bank131873 +131874 POINT(48.13719779121481 -122.48682509168916) bank131874 +131875 POINT(46.76593718891014 -121.95650622933633) bank131875 +131876 POINT(48.138750898219875 -122.86589155410017) bank131876 +131877 POINT(47.818745161300626 -122.16168838139653) bank131877 +131878 POINT(47.29366468744384 -121.67862338835242) bank131878 +131879 POINT(47.929019209009745 -122.65482787301958) bank131879 +131880 POINT(47.20468333094491 -122.67169601607306) bank131880 +131881 POINT(48.18628355888514 -121.67198901731987) bank131881 +131882 POINT(48.151475129504604 -122.049518535299) bank131882 +131883 POINT(47.437553626711676 -122.20145083022439) bank131883 +131884 POINT(48.11600643246469 -122.94295095934429) bank131884 +131885 POINT(47.45043447968894 -123.09757070946736) bank131885 +131886 POINT(47.94028261131153 -121.52588472431178) bank131886 +131887 POINT(48.16312206526035 -122.54470511454055) bank131887 +131888 POINT(46.7405636161793 -123.15486577498251) bank131888 +131889 POINT(47.90401347788374 -121.39018674845126) bank131889 +131890 POINT(47.63734976618327 -122.39131004068827) bank131890 +131891 POINT(47.322118156767516 -121.83736901923106) bank131891 +131892 POINT(47.87722815583881 -122.28232936974125) bank131892 +131893 POINT(46.61576033216165 -121.3599710188963) bank131893 +131894 POINT(46.76405272777177 -121.35488569686247) bank131894 +131895 POINT(46.98497477837084 -122.94531728661605) bank131895 +131896 POINT(47.25790941903265 -122.00548180259193) bank131896 +131897 POINT(46.80505049498752 -123.11706968999908) bank131897 +131898 POINT(48.23773430985176 -123.16608356601495) bank131898 +131899 POINT(47.68633181547355 -122.62634671602211) bank131899 +131900 POINT(46.90674830461298 -123.32601205831142) bank131900 +131901 POINT(47.44150778854199 -121.52059152626003) bank131901 +131902 POINT(47.97145921218677 -121.75715506201136) bank131902 +131903 POINT(46.94921240220302 -122.38369452973042) bank131903 +131904 POINT(48.32447715296636 -122.90576942605117) bank131904 +131905 POINT(47.61324444428244 -122.03962040676542) bank131905 +131906 POINT(47.23551546114494 -122.03247601747803) bank131906 +131907 POINT(47.48041009737987 -122.5653676380868) bank131907 +131908 POINT(47.29256838985821 -121.88610855162952) bank131908 +131909 POINT(48.314329717142996 -122.72973824083412) bank131909 +131910 POINT(47.71263241876423 -122.05353548873606) bank131910 +131911 POINT(47.216808925575194 -122.51874133550649) bank131911 +131912 POINT(47.1555920201577 -121.6839416188489) bank131912 +131913 POINT(47.090504044980754 -122.75909952722296) bank131913 +131914 POINT(46.65757296761021 -121.91928928354193) bank131914 +131915 POINT(48.236826097075514 -122.64805012113585) bank131915 +131916 POINT(47.28977386433634 -121.80102414321088) bank131916 +131917 POINT(47.16460533111326 -121.67126923370782) bank131917 +131918 POINT(47.60043163343733 -123.29995503045826) bank131918 +131919 POINT(48.162959469487404 -121.7185301545002) bank131919 +131920 POINT(47.16981645359255 -122.22227191083576) bank131920 +131921 POINT(48.507052349942946 -121.86320716407437) bank131921 +131922 POINT(47.56281208160599 -121.73497657754014) bank131922 +131923 POINT(47.47632642789817 -121.96025566547368) bank131923 +131924 POINT(48.27570737915 -121.92841079642325) bank131924 +131925 POINT(46.88669720606825 -121.38296425454668) bank131925 +131926 POINT(48.00993725645741 -121.52591853892076) bank131926 +131927 POINT(48.29443260818205 -122.62190327525667) bank131927 +131928 POINT(47.80861630016171 -123.31323769213546) bank131928 +131929 POINT(47.46356251406901 -121.84386343467855) bank131929 +131930 POINT(47.02032606134769 -122.33433874478403) bank131930 +131931 POINT(47.140517339032435 -122.40158122478516) bank131931 +131932 POINT(46.64585268238163 -122.84217574760612) bank131932 +131933 POINT(47.60861762933614 -123.06836449078116) bank131933 +131934 POINT(48.32264407353564 -122.7791465401034) bank131934 +131935 POINT(47.02217191599213 -122.23504318335969) bank131935 +131936 POINT(46.880391168547455 -121.34814048431083) bank131936 +131937 POINT(47.533518273039334 -121.36073124461664) bank131937 +131938 POINT(47.190019006817636 -123.3057832038898) bank131938 +131939 POINT(47.92216126247446 -122.42550132034616) bank131939 +131940 POINT(48.173136549076546 -121.36840679660449) bank131940 +131941 POINT(47.54210516522435 -122.4565668329317) bank131941 +131942 POINT(48.40804881063067 -122.61999458983327) bank131942 +131943 POINT(46.95273494077929 -121.85993619117015) bank131943 +131944 POINT(46.70591604217302 -123.3141155982576) bank131944 +131945 POINT(48.59007872436807 -122.18518678530535) bank131945 +131946 POINT(47.566683275485815 -123.25610918971846) bank131946 +131947 POINT(47.63158022505513 -121.5043307806052) bank131947 +131948 POINT(47.17460425459213 -121.99374998275687) bank131948 +131949 POINT(48.442081190715726 -123.24097639269515) bank131949 +131950 POINT(46.90555110314343 -122.67833674360006) bank131950 +131951 POINT(47.19270954496599 -123.20285462873345) bank131951 +131952 POINT(48.35574855492729 -122.60532272664044) bank131952 +131953 POINT(48.27368870061717 -121.4107425795264) bank131953 +131954 POINT(47.43627652371242 -122.07240537717654) bank131954 +131955 POINT(48.52713605540703 -123.1715070316218) bank131955 +131956 POINT(47.09917360550523 -123.20135937719124) bank131956 +131957 POINT(47.63911687392154 -122.6068422157715) bank131957 +131958 POINT(46.78681187737687 -122.07243347243038) bank131958 +131959 POINT(47.59282827645137 -123.21559118759166) bank131959 +131960 POINT(46.867546223890315 -122.63050940232623) bank131960 +131961 POINT(48.06780700207158 -122.25340254563953) bank131961 +131962 POINT(48.4112513184232 -122.44261896754868) bank131962 +131963 POINT(47.67726348689246 -122.27899485307572) bank131963 +131964 POINT(48.51841221702215 -122.02175824324226) bank131964 +131965 POINT(48.573363762940275 -122.13555322075379) bank131965 +131966 POINT(47.13420293780187 -121.98350485030892) bank131966 +131967 POINT(47.793558967997356 -123.28510965396724) bank131967 +131968 POINT(48.256845624312284 -121.4901784904466) bank131968 +131969 POINT(47.508029006557734 -122.88999882313126) bank131969 +131970 POINT(47.378696343873294 -123.05021988105077) bank131970 +131971 POINT(47.077342407644245 -122.81235605345452) bank131971 +131972 POINT(47.38225439432955 -122.43436539758316) bank131972 +131973 POINT(48.004061520815625 -123.02593584643851) bank131973 +131974 POINT(48.47263958810694 -122.6307006789729) bank131974 +131975 POINT(47.78762974254078 -121.66793492609685) bank131975 +131976 POINT(46.7184748908453 -122.21100978690257) bank131976 +131977 POINT(47.36093654162059 -121.61532005392391) bank131977 +131978 POINT(47.12458771521882 -123.2967925239366) bank131978 +131979 POINT(48.53668350957441 -122.8353875377736) bank131979 +131980 POINT(47.35740617649369 -122.7500212303891) bank131980 +131981 POINT(48.375722929434055 -122.66417359347939) bank131981 +131982 POINT(46.770388143539414 -122.30962517691646) bank131982 +131983 POINT(46.619631358414104 -122.99649391625015) bank131983 +131984 POINT(48.04485224202524 -122.8474602923866) bank131984 +131985 POINT(46.75728003584571 -123.04546071261284) bank131985 +131986 POINT(48.532982312584615 -121.79582270567384) bank131986 +131987 POINT(47.651800452609244 -122.02994227635524) bank131987 +131988 POINT(48.48781038574107 -122.71742950765189) bank131988 +131989 POINT(48.17346495740514 -123.01501683290725) bank131989 +131990 POINT(46.832441272123006 -122.04044818205206) bank131990 +131991 POINT(47.42461361225807 -121.42868545849716) bank131991 +131992 POINT(48.03150367672079 -121.50191024627904) bank131992 +131993 POINT(46.994961206410984 -123.19793115245794) bank131993 +131994 POINT(47.22702028896243 -121.55297154026032) bank131994 +131995 POINT(46.87539788226598 -123.3232052958913) bank131995 +131996 POINT(47.9461416912133 -122.81272944023095) bank131996 +131997 POINT(46.84915197306183 -121.81187084753157) bank131997 +131998 POINT(48.423181814104026 -121.58433774689794) bank131998 +131999 POINT(46.81118493818119 -122.17682471917759) bank131999 +132000 POINT(47.42197223948878 -122.80790343597198) bank132000 +132001 POINT(47.46672212298719 -122.72166540457495) bank132001 +132002 POINT(47.56907639682503 -122.07330713452237) bank132002 +132003 POINT(47.001906994211865 -122.72132141934388) bank132003 +132004 POINT(48.60218655832973 -121.73337260086211) bank132004 +132005 POINT(48.16789080808157 -122.72498648504079) bank132005 +132006 POINT(47.92743486159278 -121.59809231515214) bank132006 +132007 POINT(48.28814584746134 -121.7306895495879) bank132007 +132008 POINT(48.167290039554345 -121.92307867964887) bank132008 +132009 POINT(48.23716922881075 -122.56380272798772) bank132009 +132010 POINT(46.75523513406775 -122.24222313178852) bank132010 +132011 POINT(46.68975583158506 -122.12947678176982) bank132011 +132012 POINT(47.47824608674355 -122.16819064445342) bank132012 +132013 POINT(46.99461626629215 -121.5224738281824) bank132013 +132014 POINT(48.35892617981625 -121.87901333092368) bank132014 +132015 POINT(47.18482865213054 -122.0752078845136) bank132015 +132016 POINT(47.176380750273175 -123.19218695848942) bank132016 +132017 POINT(47.04942476399256 -122.3059410863855) bank132017 +132018 POINT(47.97088825696795 -122.85496418643974) bank132018 +132019 POINT(47.4407226268544 -121.51699779626863) bank132019 +132020 POINT(48.05251280771982 -122.65713875794013) bank132020 +132021 POINT(46.75964516142129 -122.48697951662757) bank132021 +132022 POINT(48.01391270015683 -121.3625772912611) bank132022 +132023 POINT(47.59976455385173 -121.91203400684971) bank132023 +132024 POINT(48.1765904018122 -122.31892341307278) bank132024 +132025 POINT(48.605992897222315 -122.89457793299462) bank132025 +132026 POINT(47.25738279133189 -122.92428877909248) bank132026 +132027 POINT(47.17717830979883 -121.49461832446423) bank132027 +132028 POINT(47.928471318814445 -122.40464405294097) bank132028 +132029 POINT(48.548586269171736 -122.65323602487646) bank132029 +132030 POINT(47.180784857818004 -121.9668302916263) bank132030 +132031 POINT(47.095050817923315 -122.45620609824839) bank132031 +132032 POINT(48.15779929197739 -121.99401801627536) bank132032 +132033 POINT(48.16863726617377 -121.80229296971764) bank132033 +132034 POINT(47.62964376556568 -122.4284525930206) bank132034 +132035 POINT(48.5983698313335 -122.92549393001349) bank132035 +132036 POINT(47.405017056263524 -123.11209336222956) bank132036 +132037 POINT(47.71545088933521 -122.15531445080002) bank132037 +132038 POINT(48.134161682256256 -122.63931603738078) bank132038 +132039 POINT(47.415813159408636 -121.87654124647908) bank132039 +132040 POINT(48.26869699125841 -123.23330214549468) bank132040 +132041 POINT(47.12305828402634 -122.35724366461814) bank132041 +132042 POINT(48.224990892909446 -122.63126988873402) bank132042 +132043 POINT(46.6285527706139 -121.82835407324605) bank132043 +132044 POINT(48.10557203396491 -122.41405056967346) bank132044 +132045 POINT(47.3142514635185 -122.37329561756714) bank132045 +132046 POINT(48.302473950045865 -121.90063501106206) bank132046 +132047 POINT(48.18630142688366 -121.98641435792733) bank132047 +132048 POINT(48.09288424092651 -121.74246403059465) bank132048 +132049 POINT(48.40213000934583 -121.71006307299463) bank132049 +132050 POINT(47.8184315863878 -122.11657368869781) bank132050 +132051 POINT(47.418274745019694 -122.9613431193104) bank132051 +132052 POINT(47.308279764852394 -121.9615103043672) bank132052 +132053 POINT(47.06062823385535 -122.87937459442196) bank132053 +132054 POINT(47.48961233610422 -121.41048876881716) bank132054 +132055 POINT(47.143664771659594 -123.08425593728124) bank132055 +132056 POINT(46.852072694321585 -121.40330465891151) bank132056 +132057 POINT(48.19184091626441 -121.78787312922371) bank132057 +132058 POINT(47.338276969823326 -121.74729184266836) bank132058 +132059 POINT(48.197552304893684 -121.52333705799477) bank132059 +132060 POINT(47.03196293505379 -123.02425762719122) bank132060 +132061 POINT(46.830286215319035 -123.25189817069455) bank132061 +132062 POINT(47.646710806266796 -122.54636467514874) bank132062 +132063 POINT(46.73775617248087 -122.4387073906996) bank132063 +132064 POINT(47.96853688074491 -123.18181023010845) bank132064 +132065 POINT(47.37710918118386 -121.90333441689907) bank132065 +132066 POINT(47.03444400708093 -123.14805986007549) bank132066 +132067 POINT(47.78126716880503 -121.47006841518058) bank132067 +132068 POINT(48.46550429431758 -121.95379552774969) bank132068 +132069 POINT(47.983949794659665 -121.7978500135256) bank132069 +132070 POINT(47.98113597033001 -122.36494911124247) bank132070 +132071 POINT(47.74137043721497 -123.2695940920206) bank132071 +132072 POINT(48.09935414655672 -123.06290901385115) bank132072 +132073 POINT(46.84646237997058 -121.86663897128226) bank132073 +132074 POINT(47.87160104337153 -122.65277963592122) bank132074 +132075 POINT(47.78859434268667 -121.49744842408684) bank132075 +132076 POINT(48.05222614853456 -122.54407029471571) bank132076 +132077 POINT(48.35992539971746 -122.47866404002394) bank132077 +132078 POINT(46.737942350959756 -121.99731279215155) bank132078 +132079 POINT(47.02337367520768 -121.34285240676476) bank132079 +132080 POINT(46.71582630887715 -122.50752280444182) bank132080 +132081 POINT(48.54955408579492 -122.70928837425897) bank132081 +132082 POINT(47.705019500741834 -123.26117072212513) bank132082 +132083 POINT(46.705880271952914 -122.19843924996441) bank132083 +132084 POINT(47.27200805145047 -122.56297233957778) bank132084 +132085 POINT(48.078711889288044 -121.43099018279611) bank132085 +132086 POINT(48.50266050522578 -123.04891476315258) bank132086 +132087 POINT(48.25107663647091 -122.97772414804038) bank132087 +132088 POINT(47.52123239226149 -122.7193340021718) bank132088 +132089 POINT(47.160131565667015 -122.18454249803872) bank132089 +132090 POINT(46.7185308770445 -122.43226873270012) bank132090 +132091 POINT(48.162506564622284 -121.61114467653503) bank132091 +132092 POINT(48.47212724988036 -122.36751339808212) bank132092 +132093 POINT(48.48730373586829 -122.21283483662089) bank132093 +132094 POINT(48.07767711142185 -122.46004426407208) bank132094 +132095 POINT(46.88043106902664 -123.14776611533506) bank132095 +132096 POINT(47.42869101587064 -122.17142790190344) bank132096 +132097 POINT(47.13361630001809 -122.33523859469926) bank132097 +132098 POINT(48.11549557506774 -121.66548171978944) bank132098 +132099 POINT(46.82848438489654 -123.27042660186882) bank132099 +132100 POINT(47.555260971789515 -121.56296954817384) bank132100 +132101 POINT(46.97918400876261 -121.68333809559117) bank132101 +132102 POINT(48.33763307619816 -122.59256873803793) bank132102 +132103 POINT(48.53892185969238 -122.3863172444553) bank132103 +132104 POINT(46.744889682703594 -123.12727053451948) bank132104 +132105 POINT(46.8050507031194 -122.26499427043572) bank132105 +132106 POINT(48.42498294332619 -121.94814228220862) bank132106 +132107 POINT(47.464168985432764 -122.69410543117809) bank132107 +132108 POINT(47.89959559161064 -122.7211740690327) bank132108 +132109 POINT(47.98081191936822 -121.66736251345067) bank132109 +132110 POINT(48.494582716701046 -122.96655561419554) bank132110 +132111 POINT(46.904830935880675 -122.24041970785443) bank132111 +132112 POINT(47.51084423719615 -122.94347726256994) bank132112 +132113 POINT(47.42345445823654 -122.28047581242224) bank132113 +132114 POINT(48.0473275473079 -122.0686165325065) bank132114 +132115 POINT(47.99735582557876 -122.57854596182752) bank132115 +132116 POINT(46.90091499171209 -123.12646033358988) bank132116 +132117 POINT(48.10278531417503 -122.00023189895268) bank132117 +132118 POINT(47.943685244216404 -121.6303527883313) bank132118 +132119 POINT(47.661777077539455 -123.22066478909635) bank132119 +132120 POINT(47.76548458557116 -121.6496034060824) bank132120 +132121 POINT(48.109988854785165 -121.39614334811267) bank132121 +132122 POINT(48.57775652052859 -121.66712899694454) bank132122 +132123 POINT(48.33417036153259 -122.11151271369101) bank132123 +132124 POINT(46.70302222189272 -122.18523883297928) bank132124 +132125 POINT(48.530507419584396 -123.24668630051872) bank132125 +132126 POINT(47.09015480586462 -122.78173468281585) bank132126 +132127 POINT(47.72607029998569 -123.33181622626147) bank132127 +132128 POINT(48.24829800037541 -122.0779782469598) bank132128 +132129 POINT(46.88906526056882 -123.15423929871272) bank132129 +132130 POINT(48.30766222661943 -122.74455659813951) bank132130 +132131 POINT(47.309344729469665 -121.7532048777048) bank132131 +132132 POINT(48.18626270006589 -121.60987711475687) bank132132 +132133 POINT(47.36826520236568 -122.43798086521532) bank132133 +132134 POINT(47.529256606197116 -121.54370392552543) bank132134 +132135 POINT(46.63064462625004 -121.63721195178832) bank132135 +132136 POINT(46.63486288853323 -122.54127405597652) bank132136 +132137 POINT(46.85283264531286 -122.60612096398627) bank132137 +132138 POINT(48.51033383906285 -122.37150675360327) bank132138 +132139 POINT(46.93316265102619 -122.69049423910627) bank132139 +132140 POINT(48.37486709988013 -121.60105591194421) bank132140 +132141 POINT(47.181262081415774 -122.02504185339414) bank132141 +132142 POINT(46.675206438438614 -122.88068191662913) bank132142 +132143 POINT(48.306177496429164 -121.52036860985571) bank132143 +132144 POINT(48.56115475475943 -123.05920283500221) bank132144 +132145 POINT(48.19420273204608 -123.11306735535445) bank132145 +132146 POINT(47.71880667437663 -122.44723483308336) bank132146 +132147 POINT(46.84450670242251 -121.5468924326689) bank132147 +132148 POINT(47.516124422942454 -123.01370421239129) bank132148 +132149 POINT(48.33613422527252 -122.25831600647865) bank132149 +132150 POINT(47.59404964288196 -121.59355060129528) bank132150 +132151 POINT(47.427232103388 -123.03235691226942) bank132151 +132152 POINT(46.78453831199864 -122.60648126458659) bank132152 +132153 POINT(46.93881559605905 -122.24303543790411) bank132153 +132154 POINT(47.7397822358594 -121.83423242777926) bank132154 +132155 POINT(47.579724392294885 -121.75473910909378) bank132155 +132156 POINT(46.823386308968864 -121.45591709149501) bank132156 +132157 POINT(47.24823693852119 -123.22853364178188) bank132157 +132158 POINT(46.82049798015474 -122.55826329332419) bank132158 +132159 POINT(48.00805659489119 -122.42616103443443) bank132159 +132160 POINT(48.184311568437856 -122.45784512835806) bank132160 +132161 POINT(47.38704954055248 -122.66027306863178) bank132161 +132162 POINT(47.50024821251186 -122.36646036519598) bank132162 +132163 POINT(47.62440274040195 -121.55463082617459) bank132163 +132164 POINT(48.45438240367364 -122.73117460487288) bank132164 +132165 POINT(47.191480479250735 -121.45830059533378) bank132165 +132166 POINT(47.56951677187571 -122.47254187788407) bank132166 +132167 POINT(47.4895602615477 -121.93825594681086) bank132167 +132168 POINT(46.63287645041912 -122.79936379717732) bank132168 +132169 POINT(47.32894993415478 -121.76761677166975) bank132169 +132170 POINT(47.000875682608765 -121.74351937915664) bank132170 +132171 POINT(47.42980186544213 -121.93245968816746) bank132171 +132172 POINT(47.37783425156582 -123.17406301369319) bank132172 +132173 POINT(48.04655196969027 -122.57319775875459) bank132173 +132174 POINT(46.95857369519358 -122.03478604618623) bank132174 +132175 POINT(46.677725905282365 -122.05275848968348) bank132175 +132176 POINT(47.337325468577376 -122.48164814800327) bank132176 +132177 POINT(46.66989863723964 -122.42198483386325) bank132177 +132178 POINT(48.29168170053081 -122.74187738488808) bank132178 +132179 POINT(48.20739414206631 -122.74013971254381) bank132179 +132180 POINT(46.99457481932286 -122.8818945777548) bank132180 +132181 POINT(48.57642923663675 -122.58949823416046) bank132181 +132182 POINT(46.61387782250253 -121.8706124924213) bank132182 +132183 POINT(47.66959471715313 -122.07520804861184) bank132183 +132184 POINT(47.25137909438866 -122.72497809609878) bank132184 +132185 POINT(46.774699456114796 -121.48288723515799) bank132185 +132186 POINT(47.05635540617735 -121.51144803292836) bank132186 +132187 POINT(48.08005215305486 -123.14155638067723) bank132187 +132188 POINT(47.78795239963116 -122.46849295734758) bank132188 +132189 POINT(47.61316849665907 -121.55267731714434) bank132189 +132190 POINT(47.80188706470149 -122.30648453092213) bank132190 +132191 POINT(46.82665437938626 -121.37391668061164) bank132191 +132192 POINT(47.35842512609579 -122.07712953448825) bank132192 +132193 POINT(47.86147425317186 -122.89698773344469) bank132193 +132194 POINT(47.737127497840724 -121.50003875839839) bank132194 +132195 POINT(46.768539413634244 -121.91325244996347) bank132195 +132196 POINT(47.638388316832824 -123.26195120392603) bank132196 +132197 POINT(48.5268398376029 -122.48975756932803) bank132197 +132198 POINT(48.550061783335345 -122.41960358592394) bank132198 +132199 POINT(46.748019021261435 -121.38246129838751) bank132199 +132200 POINT(47.387238759622996 -122.30295831107539) bank132200 +132201 POINT(47.096664616075934 -122.78261167892936) bank132201 +132202 POINT(48.45073599405608 -123.29459459842191) bank132202 +132203 POINT(46.93927968433948 -121.50239584641594) bank132203 +132204 POINT(46.6948099849016 -123.05876325549913) bank132204 +132205 POINT(48.411191422045164 -122.33406408005149) bank132205 +132206 POINT(48.44222689500404 -122.98243557731598) bank132206 +132207 POINT(48.11627266382821 -123.32489841887116) bank132207 +132208 POINT(47.538456404465194 -122.97952503492327) bank132208 +132209 POINT(46.859992986506995 -121.78747384338769) bank132209 +132210 POINT(48.09850596377173 -121.44706951614074) bank132210 +132211 POINT(46.78697086581139 -122.10870856035046) bank132211 +132212 POINT(47.12571070412407 -123.16158811931189) bank132212 +132213 POINT(47.45820877137059 -121.80325253942473) bank132213 +132214 POINT(47.975529809616575 -121.43899546742666) bank132214 +132215 POINT(46.80763164642129 -121.52025724016421) bank132215 +132216 POINT(47.809272360898376 -123.19915309366452) bank132216 +132217 POINT(47.51775327670368 -122.55830347821919) bank132217 +132218 POINT(47.571237518292286 -122.28630070574638) bank132218 +132219 POINT(46.67003893982265 -122.87282750688209) bank132219 +132220 POINT(46.91900980992437 -122.0818870417223) bank132220 +132221 POINT(47.302184808879694 -121.9088334788673) bank132221 +132222 POINT(48.36363081826628 -121.75986994591563) bank132222 +132223 POINT(48.35736808579229 -122.4299651173804) bank132223 +132224 POINT(48.477703567229455 -121.61341787707612) bank132224 +132225 POINT(47.77825644748054 -123.07605219477963) bank132225 +132226 POINT(47.43236614105598 -123.03905999635556) bank132226 +132227 POINT(46.68827021214158 -123.22193493932566) bank132227 +132228 POINT(46.93065189700223 -122.7421599617332) bank132228 +132229 POINT(47.00915252083691 -121.94408568554454) bank132229 +132230 POINT(47.692091890734524 -122.02933014000857) bank132230 +132231 POINT(46.79649536364777 -122.55162342423611) bank132231 +132232 POINT(47.15158884587215 -122.70592955965344) bank132232 +132233 POINT(48.601014368192686 -121.75387647830499) bank132233 +132234 POINT(48.4503454332852 -122.01282042091839) bank132234 +132235 POINT(47.1353889193177 -122.59894999033516) bank132235 +132236 POINT(48.351134765172574 -122.25044474888132) bank132236 +132237 POINT(48.22468170867434 -121.41292017066799) bank132237 +132238 POINT(46.818468544632914 -123.29582023291886) bank132238 +132239 POINT(47.564510755893394 -122.62839207517989) bank132239 +132240 POINT(47.277793834180976 -123.18005935450537) bank132240 +132241 POINT(47.63158413084955 -122.28772708752658) bank132241 +132242 POINT(47.896841020756476 -122.1928056965615) bank132242 +132243 POINT(47.63800942363599 -122.03071486901425) bank132243 +132244 POINT(47.17501685385317 -122.32836144000049) bank132244 +132245 POINT(46.909460946624485 -121.38509600937586) bank132245 +132246 POINT(48.44882893412607 -122.53523406326613) bank132246 +132247 POINT(48.18868905401212 -123.08147868446547) bank132247 +132248 POINT(47.57723144469811 -121.54408540178002) bank132248 +132249 POINT(47.272057420497916 -121.648073736293) bank132249 +132250 POINT(47.00045104919274 -121.74104839184511) bank132250 +132251 POINT(48.130913451428036 -123.0048716842804) bank132251 +132252 POINT(46.99172783672129 -122.28229554272461) bank132252 +132253 POINT(47.09853140859788 -123.0194555624432) bank132253 +132254 POINT(48.001954613962276 -121.57233892742794) bank132254 +132255 POINT(46.68944968064104 -122.49234144113716) bank132255 +132256 POINT(47.54793620743596 -123.26260048501699) bank132256 +132257 POINT(48.0807974146053 -121.6918506828968) bank132257 +132258 POINT(47.12480316486233 -121.60739711917763) bank132258 +132259 POINT(47.7491346121566 -121.82335770366132) bank132259 +132260 POINT(47.65945186802035 -121.49559247191611) bank132260 +132261 POINT(48.46610336545927 -122.03584057121323) bank132261 +132262 POINT(48.46102002445237 -123.31008356157433) bank132262 +132263 POINT(48.448647626270315 -121.5810572729927) bank132263 +132264 POINT(47.31242166599084 -122.68135784530497) bank132264 +132265 POINT(47.25491051357122 -121.87138065028894) bank132265 +132266 POINT(46.80229690329574 -121.52556430679546) bank132266 +132267 POINT(47.31249984259161 -122.16870664756608) bank132267 +132268 POINT(47.02251727765519 -122.39900614975926) bank132268 +132269 POINT(47.393197732256304 -123.09753071140969) bank132269 +132270 POINT(47.0468774780618 -122.91914602106885) bank132270 +132271 POINT(48.51109931205108 -121.37923194455111) bank132271 +132272 POINT(48.4368211552859 -123.02350292666466) bank132272 +132273 POINT(48.18853669506184 -122.32039815771661) bank132273 +132274 POINT(47.82097065197957 -122.92301801125632) bank132274 +132275 POINT(46.72458314076491 -122.21899901451863) bank132275 +132276 POINT(47.041695674334804 -122.42058988975393) bank132276 +132277 POINT(48.41386649240136 -122.71126693991376) bank132277 +132278 POINT(47.487347467392 -123.29057390821917) bank132278 +132279 POINT(48.09537648087911 -122.87674249546598) bank132279 +132280 POINT(47.58858620165101 -123.0099437549799) bank132280 +132281 POINT(47.72967747710289 -123.29319386320265) bank132281 +132282 POINT(47.504783899145124 -123.0214448560342) bank132282 +132283 POINT(47.47961699023346 -122.10812683961265) bank132283 +132284 POINT(48.2641799144215 -122.49467858990795) bank132284 +132285 POINT(47.93461414758538 -122.31477098867366) bank132285 +132286 POINT(48.37437831181068 -121.71694970772181) bank132286 +132287 POINT(48.324746746499095 -121.87936656632488) bank132287 +132288 POINT(48.40327352361701 -121.91759745148957) bank132288 +132289 POINT(46.790261335973476 -122.70399624112214) bank132289 +132290 POINT(47.69378008503557 -122.12152364495162) bank132290 +132291 POINT(47.02534812917654 -122.58534715099405) bank132291 +132292 POINT(47.302773316343504 -122.35338703069176) bank132292 +132293 POINT(47.08608583857972 -121.60450823397349) bank132293 +132294 POINT(46.70585701746344 -122.17563049142869) bank132294 +132295 POINT(47.0695894609789 -122.06352951999666) bank132295 +132296 POINT(46.9677423174749 -122.99955698482026) bank132296 +132297 POINT(47.45706309317727 -121.49526399579595) bank132297 +132298 POINT(47.03884784422411 -122.5383627079897) bank132298 +132299 POINT(47.09484554818822 -122.18565641521123) bank132299 +132300 POINT(46.82308134534688 -122.48536326073295) bank132300 +132301 POINT(47.604114055101675 -122.95732091031752) bank132301 +132302 POINT(47.57062421403611 -122.84248175904747) bank132302 +132303 POINT(46.79957420529933 -122.06993834327216) bank132303 +132304 POINT(48.01140419181544 -122.7545024679755) bank132304 +132305 POINT(47.12941372160266 -122.11454717861453) bank132305 +132306 POINT(46.758509596233964 -122.65535170549776) bank132306 +132307 POINT(47.671759244731845 -122.07234502229753) bank132307 +132308 POINT(47.79329579974668 -121.62220968294238) bank132308 +132309 POINT(46.692397631964866 -121.77698216378629) bank132309 +132310 POINT(48.139469506346735 -121.9751905482085) bank132310 +132311 POINT(46.76367771127864 -122.15150000095879) bank132311 +132312 POINT(47.474226779360755 -122.9025540973309) bank132312 +132313 POINT(48.24541125528882 -121.45016915684064) bank132313 +132314 POINT(47.957705717139106 -122.93812820989808) bank132314 +132315 POINT(46.80181953693229 -123.18674838603621) bank132315 +132316 POINT(46.66114514424562 -122.11896072369883) bank132316 +132317 POINT(47.85846192942232 -122.01830583979921) bank132317 +132318 POINT(47.916410495434626 -121.88686431488802) bank132318 +132319 POINT(47.97420752392279 -122.40747157373872) bank132319 +132320 POINT(48.200891522577194 -121.3429521498447) bank132320 +132321 POINT(47.294905744461076 -121.38182985978685) bank132321 +132322 POINT(47.485873216434776 -121.61318093792303) bank132322 +132323 POINT(48.54884177612248 -122.6109050076381) bank132323 +132324 POINT(47.40241603737739 -121.33659601599244) bank132324 +132325 POINT(47.944515516570945 -123.10768806263172) bank132325 +132326 POINT(48.220290934506956 -122.23612217408808) bank132326 +132327 POINT(46.780503015625214 -122.31785555466361) bank132327 +132328 POINT(48.04959989656068 -123.1348880443463) bank132328 +132329 POINT(48.20563839515529 -122.50880055212029) bank132329 +132330 POINT(46.96384632801075 -122.14233187727336) bank132330 +132331 POINT(48.39198034468422 -123.24462393923568) bank132331 +132332 POINT(47.216846708674474 -122.32640927961245) bank132332 +132333 POINT(46.6144886351583 -123.03912119131022) bank132333 +132334 POINT(47.6885485824125 -121.52847471353631) bank132334 +132335 POINT(48.48680706298989 -122.40854120601988) bank132335 +132336 POINT(47.08715904500113 -122.20980621325369) bank132336 +132337 POINT(46.79835629051784 -123.10911062110802) bank132337 +132338 POINT(47.093508967024 -121.81419097595324) bank132338 +132339 POINT(46.74546921817805 -121.4297961897633) bank132339 +132340 POINT(47.54809176820646 -122.76296158358777) bank132340 +132341 POINT(48.12399439808745 -121.96580568045079) bank132341 +132342 POINT(47.11902200219412 -123.29497221230768) bank132342 +132343 POINT(48.30177679708003 -122.58827603515577) bank132343 +132344 POINT(48.124553314231946 -123.27813092869035) bank132344 +132345 POINT(47.09888570678209 -122.56742558918909) bank132345 +132346 POINT(48.08471767755921 -122.8256305917718) bank132346 +132347 POINT(47.27335168373278 -122.7161283266608) bank132347 +132348 POINT(46.74516350169342 -122.26825016179781) bank132348 +132349 POINT(46.9575083354694 -121.98057547281341) bank132349 +132350 POINT(47.557342201599056 -121.91974978168447) bank132350 +132351 POINT(47.49746408879026 -121.45865614706997) bank132351 +132352 POINT(47.414940915340246 -121.54501063433828) bank132352 +132353 POINT(48.51804083029728 -122.69098067654791) bank132353 +132354 POINT(47.28472547358219 -123.1307390572411) bank132354 +132355 POINT(48.34310662260036 -123.00114310119845) bank132355 +132356 POINT(46.68920706558909 -122.70759927149783) bank132356 +132357 POINT(48.044341172741 -122.38252671213627) bank132357 +132358 POINT(47.13981938504603 -122.91706445724708) bank132358 +132359 POINT(46.98391017147094 -123.20062969083801) bank132359 +132360 POINT(47.59571370816602 -122.18816847721975) bank132360 +132361 POINT(48.43566911431846 -123.19654763489672) bank132361 +132362 POINT(47.85679966108373 -121.56124131618824) bank132362 +132363 POINT(46.868754228251284 -121.50563505838745) bank132363 +132364 POINT(47.92434575271846 -123.00160737349348) bank132364 +132365 POINT(47.692755534930626 -121.38429444212382) bank132365 +132366 POINT(47.13959639128463 -121.5304741061375) bank132366 +132367 POINT(47.922830261596864 -122.94811790516191) bank132367 +132368 POINT(47.785985320431934 -122.74125764832863) bank132368 +132369 POINT(48.30723604441802 -122.2066079492235) bank132369 +132370 POINT(47.079341099478874 -123.06639409623119) bank132370 +132371 POINT(48.349182880300845 -121.80117897490082) bank132371 +132372 POINT(46.68022498448466 -122.99076440804087) bank132372 +132373 POINT(46.81938365769789 -122.86334234119958) bank132373 +132374 POINT(47.38228397190727 -121.66569039875031) bank132374 +132375 POINT(47.52471135874577 -122.72594259531492) bank132375 +132376 POINT(47.59096608495882 -122.64321929373801) bank132376 +132377 POINT(48.05382488439419 -121.45006875949454) bank132377 +132378 POINT(47.82571426157677 -122.20087518070537) bank132378 +132379 POINT(47.154942687704704 -121.69369893270972) bank132379 +132380 POINT(47.31995204557329 -121.75040569810055) bank132380 +132381 POINT(48.60194309002252 -121.56946574863015) bank132381 +132382 POINT(48.23057783135677 -121.42211402430044) bank132382 +132383 POINT(47.14309654661634 -121.41872230158685) bank132383 +132384 POINT(46.706697711614304 -122.93272781649199) bank132384 +132385 POINT(46.953009288910074 -122.63243053967932) bank132385 +132386 POINT(46.89270300478883 -121.87565838050698) bank132386 +132387 POINT(47.01868288121455 -122.86819559225839) bank132387 +132388 POINT(47.33936373669772 -121.73880738622172) bank132388 +132389 POINT(47.611869214270534 -122.94957135059701) bank132389 +132390 POINT(47.896862236200654 -122.56000194114068) bank132390 +132391 POINT(46.71890439056156 -121.96803839933735) bank132391 +132392 POINT(47.52950408560013 -122.33684194669877) bank132392 +132393 POINT(48.21683649949743 -122.82007283685466) bank132393 +132394 POINT(46.97207346729698 -122.01555623637252) bank132394 +132395 POINT(47.1883478328632 -122.27422444507324) bank132395 +132396 POINT(47.75932242349185 -122.24249271590251) bank132396 +132397 POINT(48.089187712525515 -121.91689360787466) bank132397 +132398 POINT(46.75553200559103 -122.93188418202982) bank132398 +132399 POINT(47.91941767050882 -121.64093662391247) bank132399 +132400 POINT(48.23732565764108 -123.11245812832124) bank132400 +132401 POINT(46.919397073450206 -123.30434873476567) bank132401 +132402 POINT(48.25808070542745 -122.07604507786623) bank132402 +132403 POINT(47.88065481698138 -122.80118558470065) bank132403 +132404 POINT(48.14099996349056 -122.90948822878958) bank132404 +132405 POINT(48.142735600045064 -123.04964982593401) bank132405 +132406 POINT(46.8681636001573 -121.47788669083805) bank132406 +132407 POINT(48.56542796800854 -122.98570008289413) bank132407 +132408 POINT(47.24992853337993 -122.86455194022983) bank132408 +132409 POINT(47.73606105605972 -123.09140123806183) bank132409 +132410 POINT(47.81086959547016 -122.14982382198511) bank132410 +132411 POINT(47.54536257458387 -122.4299733703496) bank132411 +132412 POINT(46.756768482624906 -121.63988050351179) bank132412 +132413 POINT(48.27083603201663 -121.92323865665294) bank132413 +132414 POINT(46.639065823753256 -121.68511177186592) bank132414 +132415 POINT(47.04584692209085 -121.85955779759684) bank132415 +132416 POINT(47.839739349776714 -122.48037651023813) bank132416 +132417 POINT(47.40728913256481 -122.41114858286744) bank132417 +132418 POINT(46.80502406100826 -122.65592849748369) bank132418 +132419 POINT(48.43617766932792 -122.5311804297559) bank132419 +132420 POINT(47.47355817886741 -122.43337095877118) bank132420 +132421 POINT(48.547810264384914 -121.76808523731687) bank132421 +132422 POINT(47.33500631731433 -122.45947544602723) bank132422 +132423 POINT(47.84910705165729 -121.99115782757225) bank132423 +132424 POINT(48.52052048691283 -122.85135564920523) bank132424 +132425 POINT(46.82700742049259 -122.07767198239371) bank132425 +132426 POINT(47.13773985622568 -122.33640808591082) bank132426 +132427 POINT(47.7837368733792 -123.22042041400194) bank132427 +132428 POINT(48.070885859681596 -121.57688185908096) bank132428 +132429 POINT(47.618228773798336 -122.63758578071072) bank132429 +132430 POINT(47.13244333578677 -123.04458902620168) bank132430 +132431 POINT(46.69635577655604 -122.80487696741261) bank132431 +132432 POINT(48.39943481975405 -121.59498498388015) bank132432 +132433 POINT(48.069903045154135 -122.74121040279832) bank132433 +132434 POINT(46.63290140929051 -122.10435851469447) bank132434 +132435 POINT(48.00939677524868 -123.14342132389028) bank132435 +132436 POINT(46.857144493381185 -123.22667280156155) bank132436 +132437 POINT(47.11825871904061 -121.44551536841605) bank132437 +132438 POINT(47.44221536458778 -122.52296648542539) bank132438 +132439 POINT(47.632986520721296 -122.48859558230106) bank132439 +132440 POINT(47.214144866879884 -122.797036024274) bank132440 +132441 POINT(48.0467370772073 -121.82540364259592) bank132441 +132442 POINT(47.81709765617235 -122.59686504634215) bank132442 +132443 POINT(47.18772457070821 -122.86007512834819) bank132443 +132444 POINT(47.255409276464015 -122.93165579739986) bank132444 +132445 POINT(48.30085687518756 -123.1242498213578) bank132445 +132446 POINT(47.78504720779866 -122.98657779440272) bank132446 +132447 POINT(47.54914172772076 -123.24864842399823) bank132447 +132448 POINT(46.98117639860578 -122.09479475306986) bank132448 +132449 POINT(48.34846940420594 -122.41901678130378) bank132449 +132450 POINT(48.00371225733234 -121.5411044841641) bank132450 +132451 POINT(47.76792846792237 -122.58519416808109) bank132451 +132452 POINT(47.34041571485463 -122.34138646390076) bank132452 +132453 POINT(46.61617580466343 -123.31176865960889) bank132453 +132454 POINT(47.21690558069385 -121.62066837604401) bank132454 +132455 POINT(47.30803377834186 -121.75069811424706) bank132455 +132456 POINT(46.76027584043205 -122.58893044818689) bank132456 +132457 POINT(48.38981439095756 -122.94374337255537) bank132457 +132458 POINT(46.62359913318412 -122.27832446439284) bank132458 +132459 POINT(47.54049374100796 -121.44316731101675) bank132459 +132460 POINT(46.88293058735632 -122.37414918868093) bank132460 +132461 POINT(46.86728132679318 -122.06095811985956) bank132461 +132462 POINT(47.997627215779474 -122.53647335706876) bank132462 +132463 POINT(46.93348005349459 -123.21602162215576) bank132463 +132464 POINT(47.14365450743447 -122.09128292174609) bank132464 +132465 POINT(48.02000342330523 -122.71028769513345) bank132465 +132466 POINT(48.16810687106929 -123.04100753530807) bank132466 +132467 POINT(46.96467084425821 -123.05891173649216) bank132467 +132468 POINT(46.90341320606225 -122.3842127760192) bank132468 +132469 POINT(47.383986228703726 -121.49265092171791) bank132469 +132470 POINT(47.707293128708415 -123.05977775224754) bank132470 +132471 POINT(48.43518858705798 -121.40433740303054) bank132471 +132472 POINT(47.731222635199465 -123.14103900038353) bank132472 +132473 POINT(46.63143881434388 -122.31148192459695) bank132473 +132474 POINT(47.933626294184464 -122.99025819491328) bank132474 +132475 POINT(47.74500816612029 -122.32805106446361) bank132475 +132476 POINT(47.60572117404716 -122.75372372274799) bank132476 +132477 POINT(47.67628338920149 -123.23895938582562) bank132477 +132478 POINT(47.45750313481322 -123.18855078462984) bank132478 +132479 POINT(47.604287092758696 -122.62739303223921) bank132479 +132480 POINT(47.36562191580068 -122.60604435964206) bank132480 +132481 POINT(46.74699139143706 -121.45539697231683) bank132481 +132482 POINT(46.69066812125818 -121.42533912326074) bank132482 +132483 POINT(47.706634528936654 -122.91879214360216) bank132483 +132484 POINT(48.14472831929608 -121.53628493623921) bank132484 +132485 POINT(47.60407094674932 -122.43501923764158) bank132485 +132486 POINT(48.12162717695903 -121.82184544657815) bank132486 +132487 POINT(47.32076157180111 -122.24483057715851) bank132487 +132488 POINT(47.07333688165944 -121.80013422761505) bank132488 +132489 POINT(46.97284572370347 -121.34548603735956) bank132489 +132490 POINT(47.76218406382406 -121.6829073217018) bank132490 +132491 POINT(48.100284086789685 -121.61134290474618) bank132491 +132492 POINT(47.90864859045605 -123.06336598547583) bank132492 +132493 POINT(48.14415066925961 -123.19816347983095) bank132493 +132494 POINT(46.99938855035335 -122.21901700789799) bank132494 +132495 POINT(47.38661699815435 -121.68269000965286) bank132495 +132496 POINT(47.908499462339215 -122.37791218027843) bank132496 +132497 POINT(48.15107085944027 -121.7079150007863) bank132497 +132498 POINT(48.441017032947606 -121.79374485665106) bank132498 +132499 POINT(47.06534284844025 -121.80475064877528) bank132499 +132500 POINT(47.266665287813005 -122.95101534636613) bank132500 +132501 POINT(47.88195099781046 -122.59292349699246) bank132501 +132502 POINT(46.71923593070049 -121.9454375361392) bank132502 +132503 POINT(47.58944782423663 -122.13919811517172) bank132503 +132504 POINT(48.3511658275534 -122.0720365493145) bank132504 +132505 POINT(48.340017604175884 -122.83531633078859) bank132505 +132506 POINT(47.020480850674765 -122.49976276040852) bank132506 +132507 POINT(47.04487225147362 -122.86073107781367) bank132507 +132508 POINT(48.464060947877186 -122.36742719472659) bank132508 +132509 POINT(47.121502201849715 -122.0507562406429) bank132509 +132510 POINT(48.164526436462914 -122.70115263673368) bank132510 +132511 POINT(47.623588743381575 -122.12217332210471) bank132511 +132512 POINT(47.66647503226519 -122.63565576918839) bank132512 +132513 POINT(47.72819908380032 -121.80396872361665) bank132513 +132514 POINT(48.09433252046874 -122.48025942516215) bank132514 +132515 POINT(48.18742384563855 -123.30228389328006) bank132515 +132516 POINT(48.57502205037746 -123.30914037537316) bank132516 +132517 POINT(48.54534192607817 -122.79164033330372) bank132517 +132518 POINT(46.63183629320202 -121.82205214740789) bank132518 +132519 POINT(46.82747951253974 -122.0086245902261) bank132519 +132520 POINT(46.657643960269816 -122.6137747309868) bank132520 +132521 POINT(48.23717323227442 -123.1868964605176) bank132521 +132522 POINT(47.90392241641315 -122.71183160564829) bank132522 +132523 POINT(47.93938574677652 -121.44708094643238) bank132523 +132524 POINT(48.46203054958701 -122.12012830890093) bank132524 +132525 POINT(48.547091139814796 -122.8407809859407) bank132525 +132526 POINT(47.57315866885502 -122.69266159309693) bank132526 +132527 POINT(47.31125419375216 -122.36287832852649) bank132527 +132528 POINT(46.62446364052671 -121.87726601288067) bank132528 +132529 POINT(48.318571251908736 -121.4592798266374) bank132529 +132530 POINT(46.710842098485685 -121.65208122335703) bank132530 +132531 POINT(47.405729645593254 -121.87076822789777) bank132531 +132532 POINT(46.99085622214512 -121.60595137742203) bank132532 +132533 POINT(47.4888501406836 -122.40133509297944) bank132533 +132534 POINT(47.97110632537575 -121.9294982458381) bank132534 +132535 POINT(46.769044505557495 -122.68057899390833) bank132535 +132536 POINT(47.53147212166216 -121.97792133240532) bank132536 +132537 POINT(47.92999093346913 -121.71659847409695) bank132537 +132538 POINT(46.956604412988604 -122.24964551393708) bank132538 +132539 POINT(47.97466056757291 -122.65089951827787) bank132539 +132540 POINT(48.21436058789332 -122.098896042035) bank132540 +132541 POINT(47.02229324587946 -121.9982609398741) bank132541 +132542 POINT(47.22657699395221 -121.76891740350106) bank132542 +132543 POINT(47.10577047630583 -122.72067586700537) bank132543 +132544 POINT(47.72018383162339 -122.32508522247204) bank132544 +132545 POINT(48.33775869022158 -122.16832484195535) bank132545 +132546 POINT(48.560396381628 -122.49450981116213) bank132546 +132547 POINT(48.332110149558765 -123.11644336417244) bank132547 +132548 POINT(47.76436505221635 -121.74189887810756) bank132548 +132549 POINT(47.90144377222415 -122.52963688648595) bank132549 +132550 POINT(47.47804919373109 -122.5274480531585) bank132550 +132551 POINT(46.94056208775165 -123.1918441238214) bank132551 +132552 POINT(47.44022722762793 -121.58229782593716) bank132552 +132553 POINT(46.91428008184331 -121.61820592244025) bank132553 +132554 POINT(48.52004326430621 -122.80967290270843) bank132554 +132555 POINT(47.18486545970134 -121.81326190641921) bank132555 +132556 POINT(46.738267562543086 -121.49742706074102) bank132556 +132557 POINT(47.195923690655746 -123.3135067903292) bank132557 +132558 POINT(47.46388118520977 -122.15945409390352) bank132558 +132559 POINT(46.702949048466216 -122.29603503815302) bank132559 +132560 POINT(47.423000334450045 -122.5320057941916) bank132560 +132561 POINT(47.924069569928314 -122.48592770477573) bank132561 +132562 POINT(47.59593099507401 -123.30300935049945) bank132562 +132563 POINT(47.9971575231859 -122.88959790123025) bank132563 +132564 POINT(47.154968203975166 -121.77620724300441) bank132564 +132565 POINT(47.293972193052774 -122.64562630708728) bank132565 +132566 POINT(47.251168558579096 -123.03199636967872) bank132566 +132567 POINT(47.19599675643842 -122.72696131188748) bank132567 +132568 POINT(48.092488101409955 -122.14419038562549) bank132568 +132569 POINT(47.124485535757515 -123.29425975834192) bank132569 +132570 POINT(47.648887735972465 -123.24453840485101) bank132570 +132571 POINT(47.62176728317088 -122.59724149007573) bank132571 +132572 POINT(48.56837304833914 -121.68251714571227) bank132572 +132573 POINT(47.32414227531012 -123.00350589972763) bank132573 +132574 POINT(46.9090070364966 -121.84369875184932) bank132574 +132575 POINT(47.51771358241202 -122.15270819768664) bank132575 +132576 POINT(47.42771019951697 -123.08679393904502) bank132576 +132577 POINT(46.660417120845075 -122.48848754278508) bank132577 +132578 POINT(47.198037592794215 -122.2149029613015) bank132578 +132579 POINT(47.68495707622857 -123.09962249643716) bank132579 +132580 POINT(47.76895120525002 -122.09819238710809) bank132580 +132581 POINT(47.817370209296485 -121.97343726864037) bank132581 +132582 POINT(47.57419231904315 -122.04130775949668) bank132582 +132583 POINT(47.206179005369655 -122.95457988787048) bank132583 +132584 POINT(48.59516381273564 -122.87419209068145) bank132584 +132585 POINT(47.3257675911954 -122.64532606564039) bank132585 +132586 POINT(48.10525188171452 -122.58456935046044) bank132586 +132587 POINT(47.19273955784855 -121.7192511518488) bank132587 +132588 POINT(46.799389670264816 -122.16232732588657) bank132588 +132589 POINT(48.47017174990837 -122.5106883022602) bank132589 +132590 POINT(46.88823837076572 -122.20324280382981) bank132590 +132591 POINT(48.12967540605942 -123.17628568344941) bank132591 +132592 POINT(47.31516922880962 -122.36012185610711) bank132592 +132593 POINT(48.34214208580869 -123.12241066368115) bank132593 +132594 POINT(47.57470586372052 -123.3303715444782) bank132594 +132595 POINT(47.045017302305915 -121.57450517400204) bank132595 +132596 POINT(47.50996800155192 -121.90646247275471) bank132596 +132597 POINT(48.44011085787376 -121.53149288875152) bank132597 +132598 POINT(47.46794494739176 -123.25422905335522) bank132598 +132599 POINT(47.20148557560593 -121.86449504039432) bank132599 +132600 POINT(48.25760914391502 -122.67642830694408) bank132600 +132601 POINT(47.62336867115901 -121.67788767801886) bank132601 +132602 POINT(46.738997533121015 -122.80837841654683) bank132602 +132603 POINT(48.11174368923496 -123.11882982864795) bank132603 +132604 POINT(46.60749055866248 -123.30404963171785) bank132604 +132605 POINT(48.471660448406006 -122.12595831704479) bank132605 +132606 POINT(47.587581409795106 -123.29291655252702) bank132606 +132607 POINT(47.42671767498411 -121.83931867514951) bank132607 +132608 POINT(48.29999265880597 -122.60061989769497) bank132608 +132609 POINT(47.00986922039338 -121.44925733245229) bank132609 +132610 POINT(47.70930295893837 -122.44216661107895) bank132610 +132611 POINT(47.87720040229185 -123.07407181019084) bank132611 +132612 POINT(48.31840085997102 -122.6046655619618) bank132612 +132613 POINT(47.250651264149866 -123.29806862409777) bank132613 +132614 POINT(47.52891452653871 -122.10787621041696) bank132614 +132615 POINT(46.65552882285385 -122.47158733401288) bank132615 +132616 POINT(48.597551878649405 -121.35388067696275) bank132616 +132617 POINT(47.431960601107534 -122.06635639315438) bank132617 +132618 POINT(48.19291323355484 -122.84661863866202) bank132618 +132619 POINT(48.290421186767325 -123.18751359438626) bank132619 +132620 POINT(47.91081342492412 -123.02165832780587) bank132620 +132621 POINT(47.67812786159277 -122.64995532320376) bank132621 +132622 POINT(47.11548272441972 -123.13545435274435) bank132622 +132623 POINT(47.63162620954169 -122.1542179405075) bank132623 +132624 POINT(47.34771502850037 -121.46875832275411) bank132624 +132625 POINT(48.57117189283372 -122.86457921403456) bank132625 +132626 POINT(48.535165622394665 -121.7057194454091) bank132626 +132627 POINT(47.94900033316281 -122.65518907362132) bank132627 +132628 POINT(46.953203373576585 -121.4992726300708) bank132628 +132629 POINT(48.460208859797596 -122.19700169892657) bank132629 +132630 POINT(46.76545730086044 -122.46658288344744) bank132630 +132631 POINT(47.06722192100617 -122.19041989115112) bank132631 +132632 POINT(47.01231304446976 -122.72768817543701) bank132632 +132633 POINT(48.10018643993438 -122.42366844337822) bank132633 +132634 POINT(48.03892663365279 -122.04628057923011) bank132634 +132635 POINT(47.06264533214403 -122.44130908888233) bank132635 +132636 POINT(46.66217067845356 -121.96821012963224) bank132636 +132637 POINT(48.33244187144409 -123.07160132809886) bank132637 +132638 POINT(46.84473378380178 -122.3707874750762) bank132638 +132639 POINT(48.1422666800225 -122.19431588503222) bank132639 +132640 POINT(47.453333234917594 -122.48083099646186) bank132640 +132641 POINT(47.09731966630415 -123.268092904024) bank132641 +132642 POINT(47.72330421415692 -123.2219228952895) bank132642 +132643 POINT(46.62974175434929 -121.46570898444872) bank132643 +132644 POINT(47.51637621268857 -122.18193382462843) bank132644 +132645 POINT(47.75023338612886 -122.14597949625096) bank132645 +132646 POINT(46.64231119502589 -122.61944207605207) bank132646 +132647 POINT(48.3046149370182 -122.21855694476572) bank132647 +132648 POINT(47.74331480159543 -121.90447733791827) bank132648 +132649 POINT(46.856290141322596 -123.21857321713131) bank132649 +132650 POINT(47.992019182809486 -121.66869092351202) bank132650 +132651 POINT(48.040806890128394 -122.59352354562394) bank132651 +132652 POINT(48.4036539723982 -122.8368833675842) bank132652 +132653 POINT(47.836436011568466 -121.57428240588013) bank132653 +132654 POINT(48.57391354390709 -123.31531878071685) bank132654 +132655 POINT(48.441108986995445 -122.968678455292) bank132655 +132656 POINT(47.330577160138525 -123.07348021780454) bank132656 +132657 POINT(46.62979062258611 -121.51639454417464) bank132657 +132658 POINT(48.02353987134044 -123.15559078744494) bank132658 +132659 POINT(47.460073848207315 -121.64983645841424) bank132659 +132660 POINT(47.91115535937763 -121.67055619966699) bank132660 +132661 POINT(46.647010652883075 -121.40722427335584) bank132661 +132662 POINT(47.607611462170404 -122.76721916998255) bank132662 +132663 POINT(48.07786139823497 -122.38941264631585) bank132663 +132664 POINT(47.33448241160128 -123.08070433901389) bank132664 +132665 POINT(46.79992973901657 -122.26841320024297) bank132665 +132666 POINT(46.611163592904596 -122.86847315217061) bank132666 +132667 POINT(47.60743711719808 -123.04826814448872) bank132667 +132668 POINT(47.299988237213505 -122.99810055874846) bank132668 +132669 POINT(46.8400444671517 -121.95669367323397) bank132669 +132670 POINT(47.09940435576078 -121.4618420001552) bank132670 +132671 POINT(47.68501037339453 -121.64545205894719) bank132671 +132672 POINT(47.624533477156426 -122.44437114440272) bank132672 +132673 POINT(46.65356608389363 -122.40433270084903) bank132673 +132674 POINT(47.20714192708129 -122.43449033458703) bank132674 +132675 POINT(48.59404003745656 -122.63995943504466) bank132675 +132676 POINT(47.51165851993634 -123.08714807733845) bank132676 +132677 POINT(46.68212835566813 -123.0356420144324) bank132677 +132678 POINT(47.47874798480047 -123.2687324315611) bank132678 +132679 POINT(48.442663777413934 -123.10497775063763) bank132679 +132680 POINT(46.993986775422776 -122.9060022698157) bank132680 +132681 POINT(48.036089600538375 -123.17406043769707) bank132681 +132682 POINT(47.6387651648454 -121.46558974562055) bank132682 +132683 POINT(46.79613039400336 -122.47137950481252) bank132683 +132684 POINT(46.7766976833312 -121.72351425221294) bank132684 +132685 POINT(46.81275862841913 -122.70708036769315) bank132685 +132686 POINT(47.32767455068829 -122.65496334797791) bank132686 +132687 POINT(47.16049819120417 -122.38658576089068) bank132687 +132688 POINT(47.99953856701435 -122.1190146028715) bank132688 +132689 POINT(46.93404953857893 -121.3960261158772) bank132689 +132690 POINT(47.51894388015913 -122.63256744446822) bank132690 +132691 POINT(47.28619178997826 -121.59147738017006) bank132691 +132692 POINT(48.35472514482728 -121.40366782945173) bank132692 +132693 POINT(48.36023829390134 -122.06965239428858) bank132693 +132694 POINT(48.41060190222704 -122.32737026158404) bank132694 +132695 POINT(47.3304576939599 -122.66342385833286) bank132695 +132696 POINT(46.89441409207662 -122.24455357457963) bank132696 +132697 POINT(48.27784785840022 -122.2845208288061) bank132697 +132698 POINT(47.62929827045095 -122.97726901239996) bank132698 +132699 POINT(47.54695498978868 -121.98630804224798) bank132699 +132700 POINT(47.44897488829134 -122.54945912664742) bank132700 +132701 POINT(48.560598604639445 -121.83117394310193) bank132701 +132702 POINT(47.12293643346739 -122.99438932698864) bank132702 +132703 POINT(48.13262998280319 -123.17364678129368) bank132703 +132704 POINT(47.28515406772272 -122.4748282322627) bank132704 +132705 POINT(47.16270900648007 -121.67626903372) bank132705 +132706 POINT(47.521718921786245 -122.38310698091097) bank132706 +132707 POINT(48.34123103721989 -123.25495295492976) bank132707 +132708 POINT(48.06041602545297 -121.79480289593064) bank132708 +132709 POINT(48.37125494516135 -122.8432116942527) bank132709 +132710 POINT(47.678078096214655 -123.17818155537294) bank132710 +132711 POINT(48.033463392639774 -122.70987406845288) bank132711 +132712 POINT(46.62004394738964 -122.58455138142133) bank132712 +132713 POINT(48.415021917936194 -121.72788621130724) bank132713 +132714 POINT(47.12842760895937 -121.87750953530649) bank132714 +132715 POINT(48.575222593665714 -122.19776354311993) bank132715 +132716 POINT(47.9878185439169 -122.36032649877845) bank132716 +132717 POINT(47.859519260526966 -123.32440055327932) bank132717 +132718 POINT(47.89867671879766 -122.0435474108138) bank132718 +132719 POINT(46.76246554047235 -121.53496856551241) bank132719 +132720 POINT(47.417399017223545 -122.45624031866734) bank132720 +132721 POINT(48.36545304940527 -121.4497729988951) bank132721 +132722 POINT(47.23962510016668 -122.80893982626239) bank132722 +132723 POINT(47.42563266409131 -122.32128751406728) bank132723 +132724 POINT(47.31123118256448 -122.5698411620674) bank132724 +132725 POINT(47.345835551021786 -121.41348826755716) bank132725 +132726 POINT(46.656284372899336 -122.40326874927372) bank132726 +132727 POINT(48.319818667423895 -121.64472101487081) bank132727 +132728 POINT(48.53263872897802 -122.86641264429463) bank132728 +132729 POINT(48.4816003942813 -122.34836811727196) bank132729 +132730 POINT(48.50111783662795 -122.5236693087574) bank132730 +132731 POINT(47.575168202290655 -122.3691889965778) bank132731 +132732 POINT(47.054432147704034 -121.38227465250918) bank132732 +132733 POINT(47.82279981834951 -123.0000274517332) bank132733 +132734 POINT(48.41415026950592 -121.33728591347601) bank132734 +132735 POINT(46.76578348320944 -122.67962885111152) bank132735 +132736 POINT(47.27138882666452 -121.99241786212255) bank132736 +132737 POINT(46.87050381085315 -122.96956121947592) bank132737 +132738 POINT(47.12907980808036 -121.97245471114714) bank132738 +132739 POINT(47.15524392534178 -121.75881498207382) bank132739 +132740 POINT(46.85875417208317 -121.43102409673484) bank132740 +132741 POINT(48.23251386387127 -122.61776003844582) bank132741 +132742 POINT(47.79905833040322 -122.10889665561218) bank132742 +132743 POINT(48.20828844285764 -121.54431642787188) bank132743 +132744 POINT(48.431672029031176 -121.50990952919362) bank132744 +132745 POINT(46.925442519233215 -122.84275369872294) bank132745 +132746 POINT(47.728128091004365 -122.83123100061395) bank132746 +132747 POINT(47.183920928149774 -123.03741307965684) bank132747 +132748 POINT(47.37237310284217 -122.28342554562067) bank132748 +132749 POINT(47.29690588124832 -121.42506153981617) bank132749 +132750 POINT(47.4827087759894 -123.21230676786728) bank132750 +132751 POINT(48.3472982995961 -121.90870737849636) bank132751 +132752 POINT(47.3039149768072 -122.55038900446833) bank132752 +132753 POINT(47.51651082554595 -122.9995912586373) bank132753 +132754 POINT(47.046519255760174 -121.70296042776427) bank132754 +132755 POINT(47.0837984993926 -122.4925571226611) bank132755 +132756 POINT(48.07931439469636 -122.63289451740317) bank132756 +132757 POINT(48.09294388971536 -121.86473704721014) bank132757 +132758 POINT(47.7364012437783 -122.48790464690059) bank132758 +132759 POINT(48.24645036914988 -121.76030099109295) bank132759 +132760 POINT(48.56798916537613 -121.6557516430825) bank132760 +132761 POINT(48.58261543298683 -122.91745912337642) bank132761 +132762 POINT(47.01077062115481 -121.57231509890678) bank132762 +132763 POINT(47.13157964874206 -121.7896663796846) bank132763 +132764 POINT(47.26587700263184 -122.3734014034624) bank132764 +132765 POINT(47.908753541956926 -121.54912266746565) bank132765 +132766 POINT(48.17970003700107 -122.86739980110247) bank132766 +132767 POINT(47.06309532742277 -122.50507777436155) bank132767 +132768 POINT(47.182323040157165 -122.94059941390225) bank132768 +132769 POINT(48.07784312197081 -123.08511663801129) bank132769 +132770 POINT(46.63603043203488 -121.52787528945089) bank132770 +132771 POINT(46.85032717687769 -122.22321955691595) bank132771 +132772 POINT(46.68814870968658 -122.58520587936385) bank132772 +132773 POINT(48.32653155265692 -123.1740284278318) bank132773 +132774 POINT(48.302892471882444 -122.39836084934569) bank132774 +132775 POINT(47.30925243157949 -123.01870278595874) bank132775 +132776 POINT(47.0052514594693 -122.10230820104921) bank132776 +132777 POINT(47.78086457552862 -122.88530241131113) bank132777 +132778 POINT(47.50236427025282 -121.66786105821825) bank132778 +132779 POINT(47.677092794871896 -121.82065458045903) bank132779 +132780 POINT(48.012998642775344 -122.88167627357144) bank132780 +132781 POINT(47.502800854599585 -122.25860431627748) bank132781 +132782 POINT(48.16135435671133 -123.15501156046793) bank132782 +132783 POINT(48.24460166670641 -121.89478899629655) bank132783 +132784 POINT(46.68635586858118 -122.48090211562454) bank132784 +132785 POINT(46.92716212463424 -123.13408947441579) bank132785 +132786 POINT(48.57725899413112 -121.71939373880716) bank132786 +132787 POINT(47.871026358371466 -122.78576991730695) bank132787 +132788 POINT(48.2184186861367 -123.1419010998579) bank132788 +132789 POINT(47.72712525757661 -122.73406444313713) bank132789 +132790 POINT(47.25466033230051 -122.53327792417329) bank132790 +132791 POINT(47.317272706429854 -122.97773674978689) bank132791 +132792 POINT(47.70643103955907 -121.81914096545225) bank132792 +132793 POINT(47.17785268480203 -121.7048216939649) bank132793 +132794 POINT(47.18650589840066 -121.61276690278697) bank132794 +132795 POINT(48.00605041465031 -121.4197906664053) bank132795 +132796 POINT(46.74619075933947 -121.51611937287929) bank132796 +132797 POINT(47.058195534957605 -122.55593311404076) bank132797 +132798 POINT(47.12234557083874 -122.17672469358342) bank132798 +132799 POINT(48.1319486394884 -122.89430222930265) bank132799 +132800 POINT(47.49505220246791 -122.02415392256952) bank132800 +132801 POINT(46.99622996779896 -121.95545373886347) bank132801 +132802 POINT(47.87826746676178 -121.57661342779224) bank132802 +132803 POINT(47.36101009088011 -123.20477601396261) bank132803 +132804 POINT(47.072569590732 -123.22496460247388) bank132804 +132805 POINT(47.12737753438617 -121.77225818199315) bank132805 +132806 POINT(47.146989135643324 -122.77869832808277) bank132806 +132807 POINT(47.46467186025213 -121.76129688558835) bank132807 +132808 POINT(48.070838279079375 -123.21518069832835) bank132808 +132809 POINT(47.250705520330094 -121.37774520319455) bank132809 +132810 POINT(48.08731765091019 -121.37435807012022) bank132810 +132811 POINT(46.635457749450936 -122.90454109258172) bank132811 +132812 POINT(47.84695257682845 -122.24139003700797) bank132812 +132813 POINT(46.674676023420226 -121.4552379524239) bank132813 +132814 POINT(48.323019442820005 -123.14080640210402) bank132814 +132815 POINT(47.44337967324528 -122.06233131444134) bank132815 +132816 POINT(48.068285386817124 -122.98454888008301) bank132816 +132817 POINT(47.769948593418746 -122.75864534627426) bank132817 +132818 POINT(47.04888133119076 -122.46010868792237) bank132818 +132819 POINT(47.797359853132676 -121.64470333742634) bank132819 +132820 POINT(47.889248653376235 -122.91205297210203) bank132820 +132821 POINT(47.407674241450174 -122.2884278126939) bank132821 +132822 POINT(48.160179831923756 -122.9473094297182) bank132822 +132823 POINT(47.71040877303922 -122.0689739998949) bank132823 +132824 POINT(46.83210928419045 -122.21583855955322) bank132824 +132825 POINT(47.692392419388625 -122.38468219986599) bank132825 +132826 POINT(47.838541648530054 -122.21309622994724) bank132826 +132827 POINT(47.66791260239983 -122.50402736947073) bank132827 +132828 POINT(46.65647515324623 -121.37157075395527) bank132828 +132829 POINT(48.567192396160216 -121.80125613923043) bank132829 +132830 POINT(46.73200346737685 -122.28089175231183) bank132830 +132831 POINT(47.927361875355174 -122.22122882986018) bank132831 +132832 POINT(48.22005684836111 -122.35087960713896) bank132832 +132833 POINT(48.31475394187216 -123.23159056809806) bank132833 +132834 POINT(46.80458302447876 -121.91437397791411) bank132834 +132835 POINT(48.586472597559215 -123.15227126524056) bank132835 +132836 POINT(46.75254246964907 -122.6522313456617) bank132836 +132837 POINT(47.7935229316821 -123.11500881082348) bank132837 +132838 POINT(46.984714638081925 -121.68479698653589) bank132838 +132839 POINT(47.20186995054171 -121.93061675799864) bank132839 +132840 POINT(48.17774885285496 -123.12313683800289) bank132840 +132841 POINT(48.54317853981454 -123.09454288700535) bank132841 +132842 POINT(46.94987659872955 -122.06201457279924) bank132842 +132843 POINT(46.8632349765885 -122.6051680100783) bank132843 +132844 POINT(48.36688398484734 -122.48961386010156) bank132844 +132845 POINT(48.089255207318445 -122.1404190282877) bank132845 +132846 POINT(47.289611750042866 -123.0602592158004) bank132846 +132847 POINT(47.360033186795434 -122.79539922461018) bank132847 +132848 POINT(46.86553230139218 -122.44846188735636) bank132848 +132849 POINT(47.068804102259435 -122.28955013503894) bank132849 +132850 POINT(48.39954272402154 -123.17844807884882) bank132850 +132851 POINT(46.9269563151093 -122.7680076433858) bank132851 +132852 POINT(47.60035865535684 -122.64966690291081) bank132852 +132853 POINT(46.92351711789832 -121.39164202819393) bank132853 +132854 POINT(47.16339025860654 -121.83792944609131) bank132854 +132855 POINT(47.95021853007919 -122.15954392746639) bank132855 +132856 POINT(47.26351724009155 -123.11552573881848) bank132856 +132857 POINT(47.30188160309698 -121.91411556723502) bank132857 +132858 POINT(48.32110074035184 -122.81899027691465) bank132858 +132859 POINT(47.57335128677258 -122.12642984777615) bank132859 +132860 POINT(47.88048369479667 -122.05322181536295) bank132860 +132861 POINT(46.6791161698267 -121.40883961753285) bank132861 +132862 POINT(47.691856317774175 -123.30264260728487) bank132862 +132863 POINT(47.336037772427154 -122.66670521308137) bank132863 +132864 POINT(47.15929357147923 -122.15557685185526) bank132864 +132865 POINT(47.28581060945289 -122.00700657252003) bank132865 +132866 POINT(48.5417727437314 -121.83930621988469) bank132866 +132867 POINT(47.18962694579618 -121.80079998826058) bank132867 +132868 POINT(47.00239093540846 -122.56784866698275) bank132868 +132869 POINT(48.15057387843757 -121.58631096259515) bank132869 +132870 POINT(48.195881493675415 -122.05418831860204) bank132870 +132871 POINT(48.15031040164541 -122.15554678077812) bank132871 +132872 POINT(46.95182734564563 -121.96128755265022) bank132872 +132873 POINT(46.7620238141429 -123.0065422206791) bank132873 +132874 POINT(46.998531866718686 -122.80220365668323) bank132874 +132875 POINT(48.174535978311155 -122.93920402906443) bank132875 +132876 POINT(48.28929609470298 -122.1916217480484) bank132876 +132877 POINT(48.31894963764842 -122.38089538188339) bank132877 +132878 POINT(48.22650490193116 -121.50295264978138) bank132878 +132879 POINT(48.3263618145372 -122.40016399807267) bank132879 +132880 POINT(46.629981206469935 -122.18714219738366) bank132880 +132881 POINT(46.84232944222235 -122.9167847057934) bank132881 +132882 POINT(47.33303538604953 -121.34684534068097) bank132882 +132883 POINT(47.32653767996166 -121.35164493925423) bank132883 +132884 POINT(46.78092420938742 -123.151290988984) bank132884 +132885 POINT(47.902251181816666 -122.08016337881176) bank132885 +132886 POINT(47.61296425442277 -121.37184673204322) bank132886 +132887 POINT(48.480255599771574 -121.75728687473045) bank132887 +132888 POINT(47.60196118550403 -122.91488753684227) bank132888 +132889 POINT(46.60799833371227 -123.00961098658257) bank132889 +132890 POINT(47.15376858981718 -123.16148443911007) bank132890 +132891 POINT(47.46723256613726 -122.3610295674851) bank132891 +132892 POINT(48.045475894443875 -122.75432521013317) bank132892 +132893 POINT(48.133800312242414 -123.07688551660229) bank132893 +132894 POINT(48.530033750748125 -121.87428404026281) bank132894 +132895 POINT(47.59529179760856 -121.45419825231792) bank132895 +132896 POINT(47.26979681598743 -123.2894977182507) bank132896 +132897 POINT(46.92864770035948 -122.06324480706982) bank132897 +132898 POINT(48.230853368345514 -122.28600982352232) bank132898 +132899 POINT(47.848356648723545 -122.40252654930079) bank132899 +132900 POINT(47.20099875633308 -121.87558364013783) bank132900 +132901 POINT(46.71011973838083 -121.70161863032209) bank132901 +132902 POINT(48.31610199109076 -121.41601046610553) bank132902 +132903 POINT(48.141171824384806 -122.36273886017065) bank132903 +132904 POINT(47.713556239012625 -122.46663278325947) bank132904 +132905 POINT(47.89625082789679 -121.79638446170102) bank132905 +132906 POINT(46.667796486918995 -121.60297950065683) bank132906 +132907 POINT(48.302574223746866 -122.66573962189227) bank132907 +132908 POINT(47.161632010572156 -121.93860964390001) bank132908 +132909 POINT(47.19841639164381 -122.31184689249334) bank132909 +132910 POINT(47.748127334818165 -122.73542202600179) bank132910 +132911 POINT(47.503782649346554 -121.96569178244073) bank132911 +132912 POINT(48.06298245717029 -122.51473539164193) bank132912 +132913 POINT(48.31257918030247 -123.09210868365426) bank132913 +132914 POINT(46.96743470369011 -122.03803860344628) bank132914 +132915 POINT(47.699026684496815 -122.53926290557978) bank132915 +132916 POINT(47.58224882615956 -122.83528836266576) bank132916 +132917 POINT(47.343331548009516 -122.41055255664483) bank132917 +132918 POINT(47.88785670223931 -121.63491365252328) bank132918 +132919 POINT(46.79762361843695 -121.85802780539171) bank132919 +132920 POINT(47.0316381671356 -122.69614606375842) bank132920 +132921 POINT(48.01314133144307 -121.9622015594869) bank132921 +132922 POINT(47.448260827703876 -122.78776687599047) bank132922 +132923 POINT(47.30907198819786 -122.54361931121574) bank132923 +132924 POINT(48.522701009689875 -121.62556340027405) bank132924 +132925 POINT(46.73688711336176 -122.73111884170306) bank132925 +132926 POINT(48.113444548481425 -121.37177389243448) bank132926 +132927 POINT(47.90732498731263 -121.95190924558598) bank132927 +132928 POINT(47.72156034892365 -122.99060034201219) bank132928 +132929 POINT(47.110063528139314 -121.51444897846021) bank132929 +132930 POINT(47.00434776643644 -123.2468218177308) bank132930 +132931 POINT(48.45979826320634 -122.2039410429808) bank132931 +132932 POINT(47.950159978890696 -122.12826317922308) bank132932 +132933 POINT(46.62076639829191 -122.63933417813345) bank132933 +132934 POINT(48.45349902279963 -122.23906525666794) bank132934 +132935 POINT(48.12233492151014 -121.76776325645291) bank132935 +132936 POINT(47.56046783732563 -121.81387377792355) bank132936 +132937 POINT(46.645987472874175 -122.54192748755375) bank132937 +132938 POINT(47.77955456112262 -122.63118682655626) bank132938 +132939 POINT(48.15287924695482 -123.2187798638702) bank132939 +132940 POINT(47.01622375749592 -122.87563045376533) bank132940 +132941 POINT(46.83944546665924 -123.11373725439276) bank132941 +132942 POINT(47.08900719363663 -122.4039370566031) bank132942 +132943 POINT(46.70614910357001 -122.70494318022698) bank132943 +132944 POINT(46.79320530328936 -123.29332633794935) bank132944 +132945 POINT(46.71668087221821 -121.80716236353294) bank132945 +132946 POINT(47.926891088739865 -122.26551521266855) bank132946 +132947 POINT(48.34549153595535 -121.58947848363424) bank132947 +132948 POINT(47.6890017011939 -122.36825676612862) bank132948 +132949 POINT(46.85166692574258 -121.63964181261429) bank132949 +132950 POINT(47.880424753939664 -123.31649261873613) bank132950 +132951 POINT(47.501542631818765 -121.6685357101196) bank132951 +132952 POINT(46.82387117940829 -122.78552430262901) bank132952 +132953 POINT(47.17143097168834 -121.88239712996551) bank132953 +132954 POINT(47.52058524172524 -122.16005221404447) bank132954 +132955 POINT(47.09022059757084 -123.15465930764519) bank132955 +132956 POINT(46.9314536440121 -122.14651447325912) bank132956 +132957 POINT(47.4268905752289 -122.24189702784764) bank132957 +132958 POINT(47.46733034447735 -122.10951599220564) bank132958 +132959 POINT(48.409871102665015 -122.73601750390031) bank132959 +132960 POINT(47.49876769506693 -122.61052104665234) bank132960 +132961 POINT(48.21198647722465 -122.40389571204443) bank132961 +132962 POINT(47.4435313375069 -122.00752536426215) bank132962 +132963 POINT(46.73951157019699 -121.56668396047418) bank132963 +132964 POINT(48.01003740892921 -121.50780783753449) bank132964 +132965 POINT(48.558570447001344 -122.0022367676112) bank132965 +132966 POINT(48.34491410636789 -122.07164810156569) bank132966 +132967 POINT(47.089147051039575 -122.51831537216084) bank132967 +132968 POINT(48.503133565444976 -122.61877463857911) bank132968 +132969 POINT(47.73260451831788 -121.76329537566727) bank132969 +132970 POINT(47.1693360531169 -122.77895467406897) bank132970 +132971 POINT(47.556202759987556 -123.2375301365427) bank132971 +132972 POINT(48.180400387810224 -122.57768142428563) bank132972 +132973 POINT(47.48281859898181 -121.51998390512088) bank132973 +132974 POINT(47.35332194791939 -121.70430892994037) bank132974 +132975 POINT(47.31241146321534 -121.47697184927809) bank132975 +132976 POINT(48.546991865838145 -121.42320548914014) bank132976 +132977 POINT(46.85482813336058 -122.78803605397928) bank132977 +132978 POINT(47.87802373667896 -121.59644583922957) bank132978 +132979 POINT(48.267191602568765 -122.07031898292827) bank132979 +132980 POINT(47.3454939150695 -122.6564993814587) bank132980 +132981 POINT(46.647805848149794 -122.54175946892644) bank132981 +132982 POINT(46.689843853321285 -121.55526420913485) bank132982 +132983 POINT(48.19176543767753 -123.05938025967528) bank132983 +132984 POINT(47.311819067987955 -121.39102295221582) bank132984 +132985 POINT(47.73076687326484 -122.02023893370435) bank132985 +132986 POINT(47.26624386478746 -121.63900740169238) bank132986 +132987 POINT(47.5191765582028 -122.68445824325119) bank132987 +132988 POINT(47.6116393139339 -123.2237300585826) bank132988 +132989 POINT(48.53740865070856 -122.37472630987465) bank132989 +132990 POINT(46.9315131814494 -122.6091662410612) bank132990 +132991 POINT(47.49676008566936 -121.75509373856657) bank132991 +132992 POINT(48.136942484369705 -121.40824378169705) bank132992 +132993 POINT(47.708478808672545 -123.16021640769812) bank132993 +132994 POINT(46.94136669964599 -123.06316963212625) bank132994 +132995 POINT(47.343704513711636 -121.60808058516416) bank132995 +132996 POINT(47.7242351937196 -122.28486583756317) bank132996 +132997 POINT(46.98720535314529 -123.21468084599667) bank132997 +132998 POINT(47.25173551406296 -123.23742056939498) bank132998 +132999 POINT(48.2280819770418 -122.71304632076978) bank132999 +133000 POINT(48.072420262315866 -122.84313607474462) bank133000 +133001 POINT(48.59483824317822 -121.44174018627439) bank133001 +133002 POINT(47.17920319359016 -122.70871640586152) bank133002 +133003 POINT(47.33799252437967 -122.59657131205243) bank133003 +133004 POINT(48.0444216691668 -123.29640629843907) bank133004 +133005 POINT(48.54007919084745 -122.72557085741374) bank133005 +133006 POINT(48.257585243042 -122.17087729065089) bank133006 +133007 POINT(47.535059589933795 -122.143488440504) bank133007 +133008 POINT(48.24331138480942 -121.64282009738804) bank133008 +133009 POINT(47.065373253570236 -122.14764425247674) bank133009 +133010 POINT(48.02953476623305 -121.85068775779774) bank133010 +133011 POINT(48.22563039094049 -122.08302397985993) bank133011 +133012 POINT(48.04243925163456 -122.29519328438703) bank133012 +133013 POINT(48.39691027637699 -121.5837420144494) bank133013 +133014 POINT(47.56584970326013 -123.20645267088496) bank133014 +133015 POINT(47.720498104322836 -123.06624360591559) bank133015 +133016 POINT(47.561561159656186 -122.27158907633206) bank133016 +133017 POINT(47.78243827840182 -123.31964675653515) bank133017 +133018 POINT(47.48514676292016 -121.39191745278465) bank133018 +133019 POINT(48.182591136427725 -121.695825295567) bank133019 +133020 POINT(47.01059546386622 -123.06408297955446) bank133020 +133021 POINT(46.863651588006114 -121.89852588849031) bank133021 +133022 POINT(47.004178703377036 -122.15033244857209) bank133022 +133023 POINT(48.419111688093736 -123.1738527727788) bank133023 +133024 POINT(48.16497356627373 -121.53246946335848) bank133024 +133025 POINT(47.10245852968258 -121.74427509139127) bank133025 +133026 POINT(47.13432694321826 -121.43223370524821) bank133026 +133027 POINT(48.49407105059937 -121.65539843532623) bank133027 +133028 POINT(46.77312116682884 -121.36283707087416) bank133028 +133029 POINT(47.05369162628777 -121.66299602195987) bank133029 +133030 POINT(47.25445507131999 -123.2868834129752) bank133030 +133031 POINT(48.314732022172734 -122.48339397155947) bank133031 +133032 POINT(47.351456835913424 -121.56994743691632) bank133032 +133033 POINT(46.76769346125072 -122.15669261272753) bank133033 +133034 POINT(47.352279983709025 -122.88518936293022) bank133034 +133035 POINT(47.35960792939819 -122.22836839042931) bank133035 +133036 POINT(46.95319471922157 -122.21626990001397) bank133036 +133037 POINT(48.5460744064468 -121.59620922739023) bank133037 +133038 POINT(47.12487157843009 -123.3252872519265) bank133038 +133039 POINT(47.26133265566018 -123.25766179043865) bank133039 +133040 POINT(47.21156651140415 -121.4370796095797) bank133040 +133041 POINT(47.12721288432026 -123.15992044210621) bank133041 +133042 POINT(47.437581237722554 -122.99524448014007) bank133042 +133043 POINT(48.2627750114001 -122.08618166498061) bank133043 +133044 POINT(47.867252992599695 -122.23498202933581) bank133044 +133045 POINT(47.66278167705611 -122.21909217714519) bank133045 +133046 POINT(46.62936728769814 -122.50764455354036) bank133046 +133047 POINT(47.08571512335252 -121.85559914645641) bank133047 +133048 POINT(47.67832788903239 -122.42276980200927) bank133048 +133049 POINT(47.344515601100625 -122.39774022522035) bank133049 +133050 POINT(48.2723382248973 -122.57726009480186) bank133050 +133051 POINT(47.49332354316536 -122.55919688539312) bank133051 +133052 POINT(48.217700964022164 -121.33755958855598) bank133052 +133053 POINT(48.311644044191105 -121.62669688033147) bank133053 +133054 POINT(47.38802824875062 -123.09323275391874) bank133054 +133055 POINT(47.688458808375245 -123.13992551931473) bank133055 +133056 POINT(48.409903763087584 -121.44140867419499) bank133056 +133057 POINT(48.0851496398573 -122.90319737895601) bank133057 +133058 POINT(46.96700945060425 -122.19090475609717) bank133058 +133059 POINT(47.325617434577346 -123.01678717100799) bank133059 +133060 POINT(47.47048343707239 -122.40264186591041) bank133060 +133061 POINT(46.85572656903372 -121.8570957571371) bank133061 +133062 POINT(47.99428659491361 -122.03553649965863) bank133062 +133063 POINT(46.99683226697848 -122.05127089662362) bank133063 +133064 POINT(46.88274010644581 -122.53105792627987) bank133064 +133065 POINT(47.082104686489416 -123.27591000129841) bank133065 +133066 POINT(46.86071068086635 -122.4144897540418) bank133066 +133067 POINT(47.574658007596476 -122.71852602671038) bank133067 +133068 POINT(47.78214339478303 -123.24186694769229) bank133068 +133069 POINT(47.29002591903252 -122.48610885655778) bank133069 +133070 POINT(47.93845203282293 -122.34844017935089) bank133070 +133071 POINT(47.42399599755573 -121.4717396520343) bank133071 +133072 POINT(47.329525673845055 -122.97788105618908) bank133072 +133073 POINT(48.324673071282646 -122.29451804262499) bank133073 +133074 POINT(46.84264372333266 -122.12873629010488) bank133074 +133075 POINT(47.27172545385703 -121.37967352013875) bank133075 +133076 POINT(46.68897512793776 -123.31037803119652) bank133076 +133077 POINT(47.91195152272625 -121.8641368060278) bank133077 +133078 POINT(47.327707283644266 -121.88707879887919) bank133078 +133079 POINT(48.481765348270244 -123.12107551362807) bank133079 +133080 POINT(46.85844742357466 -122.2565161841198) bank133080 +133081 POINT(47.710881238766085 -122.85657348342019) bank133081 +133082 POINT(47.42436973446024 -121.75402358394115) bank133082 +133083 POINT(47.627467450891054 -121.66604223090059) bank133083 +133084 POINT(47.88462005066078 -121.8872307773914) bank133084 +133085 POINT(47.73808474000326 -122.04073615818471) bank133085 +133086 POINT(47.05144133812144 -122.85917747744324) bank133086 +133087 POINT(47.44082738945928 -122.64115266207547) bank133087 +133088 POINT(47.273848031527116 -121.82627628464463) bank133088 +133089 POINT(47.92653325232912 -122.03514980720739) bank133089 +133090 POINT(46.85702203244839 -122.06646930606394) bank133090 +133091 POINT(47.735223461331614 -121.47536083341579) bank133091 +133092 POINT(48.56396215929951 -123.27617479652831) bank133092 +133093 POINT(47.241361419508095 -122.00744625942157) bank133093 +133094 POINT(47.03486455777477 -121.50884300192287) bank133094 +133095 POINT(48.535571496612754 -122.55936492415127) bank133095 +133096 POINT(46.62246289763249 -122.64852980914245) bank133096 +133097 POINT(47.136961151754875 -123.02564731359983) bank133097 +133098 POINT(47.17515674491901 -123.09931659419416) bank133098 +133099 POINT(46.70242898268517 -123.30088531002511) bank133099 +133100 POINT(47.02052167153947 -121.68111901244795) bank133100 +133101 POINT(47.88553829199466 -121.75272263610765) bank133101 +133102 POINT(47.521069817407344 -121.43198949367905) bank133102 +133103 POINT(47.13791004918742 -122.93009737057912) bank133103 +133104 POINT(47.498295102329266 -122.51274396378692) bank133104 +133105 POINT(48.30699704007275 -121.6430480445715) bank133105 +133106 POINT(47.58554911455364 -122.19064527053783) bank133106 +133107 POINT(47.55174740069253 -122.69581963883216) bank133107 +133108 POINT(47.2680980974469 -122.20605197550434) bank133108 +133109 POINT(48.32391025385435 -123.13030284474716) bank133109 +133110 POINT(47.82290589507938 -122.54157587820501) bank133110 +133111 POINT(48.00895178540982 -121.69653385710909) bank133111 +133112 POINT(48.03367396458273 -121.81296650737978) bank133112 +133113 POINT(47.49767389447947 -122.69524625465144) bank133113 +133114 POINT(48.182695812217965 -122.10674834989935) bank133114 +133115 POINT(48.262404527863346 -121.92022212695576) bank133115 +133116 POINT(46.956210385762084 -121.60733150518143) bank133116 +133117 POINT(47.51901558660599 -123.24729320825256) bank133117 +133118 POINT(48.369455438849165 -122.89436559277249) bank133118 +133119 POINT(47.61460059197873 -122.54715944154414) bank133119 +133120 POINT(48.24622883363387 -123.3005432385975) bank133120 +133121 POINT(46.96904976613198 -122.4389273488862) bank133121 +133122 POINT(48.062265565053444 -121.92608061187212) bank133122 +133123 POINT(47.43723259246279 -122.74343812096) bank133123 +133124 POINT(48.14258169884665 -122.00967188798239) bank133124 +133125 POINT(48.30107443288899 -121.70983891405469) bank133125 +133126 POINT(46.906498394302425 -121.64206453809663) bank133126 +133127 POINT(47.4081913882984 -122.11235799579684) bank133127 +133128 POINT(47.98157820230965 -121.8099918831928) bank133128 +133129 POINT(47.53203993941018 -122.7159901839961) bank133129 +133130 POINT(46.82591578440054 -121.5428161712899) bank133130 +133131 POINT(47.01115076294885 -122.07180445434727) bank133131 +133132 POINT(48.0604890940794 -121.7897548199263) bank133132 +133133 POINT(46.818390763002085 -121.71305931478452) bank133133 +133134 POINT(47.29805696207258 -122.83782638048692) bank133134 +133135 POINT(47.80245205750126 -122.4376879634738) bank133135 +133136 POINT(47.165776299791524 -121.78146560345013) bank133136 +133137 POINT(47.616460856651685 -122.47299592345863) bank133137 +133138 POINT(48.426815332249475 -122.8995321046983) bank133138 +133139 POINT(48.32891331219434 -122.09469261182036) bank133139 +133140 POINT(47.8410623791606 -122.33859204441904) bank133140 +133141 POINT(47.0219385185327 -122.2428782321608) bank133141 +133142 POINT(47.57567037904483 -122.96318632165831) bank133142 +133143 POINT(48.34821729658225 -122.32155531782486) bank133143 +133144 POINT(46.66250689118195 -122.51294329771454) bank133144 +133145 POINT(47.14419264598401 -123.23909243506093) bank133145 +133146 POINT(46.68437141463852 -122.23628160352935) bank133146 +133147 POINT(48.4305519230687 -123.10280492927134) bank133147 +133148 POINT(47.62425437993223 -123.20605031588903) bank133148 +133149 POINT(47.34006416822993 -121.64194659791654) bank133149 +133150 POINT(47.171432884581755 -121.91345591149417) bank133150 +133151 POINT(47.090947361392935 -121.45019388600186) bank133151 +133152 POINT(47.37832174142722 -122.10659305306079) bank133152 +133153 POINT(46.73780554656912 -123.29348542628667) bank133153 +133154 POINT(47.92613302615806 -122.59414947323776) bank133154 +133155 POINT(46.70447379740155 -121.840164130152) bank133155 +133156 POINT(48.527847674310784 -121.47341033147285) bank133156 +133157 POINT(48.349000295712344 -122.92641283262368) bank133157 +133158 POINT(48.37586787624471 -121.49544057260368) bank133158 +133159 POINT(47.45857982483748 -121.43065661376323) bank133159 +133160 POINT(47.77550286384066 -122.9771154841143) bank133160 +133161 POINT(46.62184975031945 -122.54207289030525) bank133161 +133162 POINT(47.60612517606459 -122.93166216177389) bank133162 +133163 POINT(47.098757160631614 -122.34447244851532) bank133163 +133164 POINT(47.02701373160557 -121.49119240310607) bank133164 +133165 POINT(46.74289771666037 -122.12884642893745) bank133165 +133166 POINT(47.8166940923626 -121.60279516609457) bank133166 +133167 POINT(47.23790956618428 -123.31570826481718) bank133167 +133168 POINT(47.56677976882219 -122.6424023780739) bank133168 +133169 POINT(47.41762682927084 -123.03258171236733) bank133169 +133170 POINT(46.663006827853785 -121.4836269562492) bank133170 +133171 POINT(46.779502434417125 -121.88917241157259) bank133171 +133172 POINT(47.08171344113602 -122.53694074250785) bank133172 +133173 POINT(48.38710228535045 -121.56967155012642) bank133173 +133174 POINT(47.52086953879879 -122.60384398236904) bank133174 +133175 POINT(47.430721340500185 -123.0488859259195) bank133175 +133176 POINT(48.56935073032965 -121.64812241581691) bank133176 +133177 POINT(48.592291844482524 -123.29445489833203) bank133177 +133178 POINT(46.89687847845405 -122.57244842555045) bank133178 +133179 POINT(47.022081677528405 -121.68045302327359) bank133179 +133180 POINT(47.268527615652765 -121.87360275577541) bank133180 +133181 POINT(47.288355223752326 -121.61379266767436) bank133181 +133182 POINT(48.27816134828934 -122.96689379874343) bank133182 +133183 POINT(46.892040961741245 -122.10266797061823) bank133183 +133184 POINT(48.43915736793532 -123.1517386147396) bank133184 +133185 POINT(48.27951424514907 -122.71452242359335) bank133185 +133186 POINT(46.70337944241652 -121.89005668484836) bank133186 +133187 POINT(48.009291358826076 -121.34275604180668) bank133187 +133188 POINT(47.282620357473675 -122.80141600062632) bank133188 +133189 POINT(46.70242723747188 -122.87611698052461) bank133189 +133190 POINT(46.8694804466667 -122.96438831878893) bank133190 +133191 POINT(48.064910120051394 -122.3241275993458) bank133191 +133192 POINT(47.74430856592451 -121.77457895022029) bank133192 +133193 POINT(47.919584167760206 -121.60372366562623) bank133193 +133194 POINT(47.68120779432848 -121.44094879082665) bank133194 +133195 POINT(47.80863936526622 -122.42675495117118) bank133195 +133196 POINT(46.83841406932042 -122.1911945310185) bank133196 +133197 POINT(48.126423620959585 -122.03593780645829) bank133197 +133198 POINT(46.882328547163034 -121.90537943641907) bank133198 +133199 POINT(48.16297591293372 -121.69835554863026) bank133199 +133200 POINT(47.475278465103045 -121.57713993572442) bank133200 +133201 POINT(47.086705875358426 -121.95520377302512) bank133201 +133202 POINT(48.275899146362505 -123.03606830602669) bank133202 +133203 POINT(46.96768156461184 -122.28106537224218) bank133203 +133204 POINT(48.09215595213836 -123.0311470873192) bank133204 +133205 POINT(47.810973894085286 -121.88952908718096) bank133205 +133206 POINT(48.22063624409536 -123.12188412796543) bank133206 +133207 POINT(47.16093137731646 -122.82529187593596) bank133207 +133208 POINT(48.067178233799545 -122.20944764919005) bank133208 +133209 POINT(47.453186018392934 -123.13385677770773) bank133209 +133210 POINT(46.85892182344614 -122.6106303583562) bank133210 +133211 POINT(48.21257573016935 -121.7620870432558) bank133211 +133212 POINT(47.31864175140392 -122.67174013588256) bank133212 +133213 POINT(48.34646831571352 -121.75735377303572) bank133213 +133214 POINT(48.524613525117715 -121.70585853412796) bank133214 +133215 POINT(47.24136968228255 -122.72259334594874) bank133215 +133216 POINT(47.6704589884566 -122.95324348295973) bank133216 +133217 POINT(48.33381281005867 -122.56964167168555) bank133217 +133218 POINT(48.18946520719311 -122.3471914620061) bank133218 +133219 POINT(47.958290883628365 -121.39403079786223) bank133219 +133220 POINT(47.962617469319525 -122.12571308339281) bank133220 +133221 POINT(47.19426667664009 -121.3697419742962) bank133221 +133222 POINT(47.96655037979667 -122.48930736417114) bank133222 +133223 POINT(48.1412514167212 -122.96257547940301) bank133223 +133224 POINT(48.54806019856149 -121.96296875701121) bank133224 +133225 POINT(46.76207215167463 -122.67746919898657) bank133225 +133226 POINT(46.79850537578628 -122.56368559134998) bank133226 +133227 POINT(47.90063867135706 -121.65499602319923) bank133227 +133228 POINT(48.433826778724935 -123.2105334090556) bank133228 +133229 POINT(47.336288549780896 -122.22591256205486) bank133229 +133230 POINT(48.40260752695753 -122.69658290071087) bank133230 +133231 POINT(46.88503890296589 -122.21898143537861) bank133231 +133232 POINT(48.0139889573214 -122.86455422238906) bank133232 +133233 POINT(46.65344206782594 -122.20619594195597) bank133233 +133234 POINT(48.46818631029349 -122.44787835133026) bank133234 +133235 POINT(47.08434954688273 -121.98358933436386) bank133235 +133236 POINT(47.40664419670708 -122.99254341275116) bank133236 +133237 POINT(47.767379950047406 -122.19786883995718) bank133237 +133238 POINT(47.65936752153553 -123.00459264032519) bank133238 +133239 POINT(47.83139182217334 -122.97451905668726) bank133239 +133240 POINT(47.84580914890748 -122.66280156093) bank133240 +133241 POINT(48.10823415839917 -123.11195759501994) bank133241 +133242 POINT(48.32151589629364 -122.44458611688839) bank133242 +133243 POINT(47.32769940098813 -123.27177329794688) bank133243 +133244 POINT(48.54741557316323 -122.19375350993167) bank133244 +133245 POINT(48.11293449148003 -121.70280688678041) bank133245 +133246 POINT(47.36920896723796 -121.48816513726827) bank133246 +133247 POINT(47.8800845101539 -122.02701627680139) bank133247 +133248 POINT(47.40071919813663 -123.03112626774475) bank133248 +133249 POINT(46.97033711895581 -121.90114411809586) bank133249 +133250 POINT(47.9699485378302 -123.15548051815647) bank133250 +133251 POINT(48.57879234744769 -121.77419796609546) bank133251 +133252 POINT(48.43211827165736 -122.98044248560771) bank133252 +133253 POINT(48.31661193248672 -121.43905278740402) bank133253 +133254 POINT(46.68008525222408 -121.38353658850293) bank133254 +133255 POINT(48.02492255382217 -123.26782339682602) bank133255 +133256 POINT(48.100290279303216 -122.15043196812776) bank133256 +133257 POINT(47.60360584674389 -122.95680091911147) bank133257 +133258 POINT(48.03722769642047 -122.89702903513049) bank133258 +133259 POINT(47.69999744650841 -122.26971785989382) bank133259 +133260 POINT(47.870330328224384 -122.05293090514077) bank133260 +133261 POINT(47.63862991483994 -121.51545802405242) bank133261 +133262 POINT(47.72219744367475 -122.5011540838424) bank133262 +133263 POINT(47.3071035862897 -122.72307901790273) bank133263 +133264 POINT(48.34265533661183 -121.97714205500411) bank133264 +133265 POINT(47.43444991421003 -121.45396681211537) bank133265 +133266 POINT(46.88750336496942 -121.79669882182414) bank133266 +133267 POINT(46.73206229361192 -123.2333740995356) bank133267 +133268 POINT(46.64753443438531 -122.94556505975325) bank133268 +133269 POINT(46.651498242928184 -121.9010797400957) bank133269 +133270 POINT(48.16628716324045 -123.23621823564184) bank133270 +133271 POINT(47.29720427587768 -121.67815264256112) bank133271 +133272 POINT(47.60599884994235 -122.9926285377403) bank133272 +133273 POINT(48.291645082342285 -122.18507869988258) bank133273 +133274 POINT(48.53164319123863 -122.00992102477431) bank133274 +133275 POINT(47.930128256440604 -121.581587989813) bank133275 +133276 POINT(47.10042599692677 -122.80946894329601) bank133276 +133277 POINT(48.31024190713503 -123.32559992218088) bank133277 +133278 POINT(47.897666043483945 -122.61773621081993) bank133278 +133279 POINT(46.822679411148286 -122.36205522299755) bank133279 +133280 POINT(48.08140738622944 -121.6479266677479) bank133280 +133281 POINT(47.509451355270876 -121.81045249275554) bank133281 +133282 POINT(48.413538927208705 -122.97298045044003) bank133282 +133283 POINT(47.14315631474166 -122.12917945169568) bank133283 +133284 POINT(47.22762531107664 -121.42009042107581) bank133284 +133285 POINT(47.593741068869576 -122.38122810857067) bank133285 +133286 POINT(47.199569184131384 -122.082768040797) bank133286 +133287 POINT(47.372193875712995 -122.48775993572346) bank133287 +133288 POINT(47.28566186284605 -121.49480370412952) bank133288 +133289 POINT(48.52536799056211 -122.22183736248759) bank133289 +133290 POINT(48.42027768667841 -122.52348507186578) bank133290 +133291 POINT(47.40406054869183 -123.2595709325801) bank133291 +133292 POINT(46.94641228689239 -121.5705200810559) bank133292 +133293 POINT(48.55352862646083 -122.2632173104298) bank133293 +133294 POINT(48.44916030317589 -122.2089360728119) bank133294 +133295 POINT(46.74678256791132 -123.21148462243627) bank133295 +133296 POINT(47.842916820384524 -121.81491151616515) bank133296 +133297 POINT(47.23778078653881 -122.4640353804395) bank133297 +133298 POINT(47.01119135940181 -121.79211700288616) bank133298 +133299 POINT(48.00154131823224 -121.61475166778125) bank133299 +133300 POINT(48.079687639674454 -121.57717019839431) bank133300 +133301 POINT(47.976279132395014 -123.19236588845628) bank133301 +133302 POINT(47.29296463358407 -122.20801558171246) bank133302 +133303 POINT(47.121885138572175 -122.97731922227098) bank133303 +133304 POINT(48.22873515626328 -121.55754826224268) bank133304 +133305 POINT(48.156467543266004 -123.07163631489223) bank133305 +133306 POINT(46.81700744410287 -122.07130331536776) bank133306 +133307 POINT(47.26450977955362 -121.72484878017737) bank133307 +133308 POINT(47.03950942832637 -122.41769376763216) bank133308 +133309 POINT(47.215541727342284 -121.62052424593584) bank133309 +133310 POINT(47.43494401645588 -122.96773706467326) bank133310 +133311 POINT(47.320055017326226 -121.86943632872921) bank133311 +133312 POINT(48.32373807116808 -122.50884575800241) bank133312 +133313 POINT(47.58047094097799 -122.80243847403739) bank133313 +133314 POINT(47.85662064656454 -122.84858749022113) bank133314 +133315 POINT(47.92179331638801 -122.98973432911204) bank133315 +133316 POINT(48.256957315038825 -123.07613920258186) bank133316 +133317 POINT(47.22996708978106 -122.19889916272689) bank133317 +133318 POINT(47.179155545441965 -122.12196487115057) bank133318 +133319 POINT(47.310340052397095 -122.52483161374336) bank133319 +133320 POINT(48.55276898829084 -121.63353439272) bank133320 +133321 POINT(48.57905939603997 -121.4246086620988) bank133321 +133322 POINT(48.09578390261099 -123.30811008739872) bank133322 +133323 POINT(47.56721158603325 -122.09595088840832) bank133323 +133324 POINT(47.40734138984707 -123.30000519358929) bank133324 +133325 POINT(48.32983243312709 -122.23496192962952) bank133325 +133326 POINT(48.04189989475851 -121.43104191214701) bank133326 +133327 POINT(47.951206589589184 -121.69876087381033) bank133327 +133328 POINT(48.017626255211056 -121.99623487013217) bank133328 +133329 POINT(47.791660669799995 -121.88863408687232) bank133329 +133330 POINT(48.056014997644645 -122.89146107584807) bank133330 +133331 POINT(46.69923077165897 -122.62941648297466) bank133331 +133332 POINT(47.27368766624179 -122.36893602177757) bank133332 +133333 POINT(48.261340913480424 -121.3998873975632) bank133333 +133334 POINT(48.08384265172225 -122.56837105518592) bank133334 +133335 POINT(47.06946101764346 -122.63012856014323) bank133335 +133336 POINT(47.687935792741904 -123.05454416258748) bank133336 +133337 POINT(47.5129753248649 -123.11418800745633) bank133337 +133338 POINT(47.71494356069075 -122.1725360441984) bank133338 +133339 POINT(47.42022986483807 -123.23355690511194) bank133339 +133340 POINT(42.782767810332714 -86.91613216722628) bank133340 +133341 POINT(41.75553102744402 -87.56524017246402) bank133341 +133342 POINT(42.30937359010812 -86.86475547416677) bank133342 +133343 POINT(42.7260652091971 -88.4106460781458) bank133343 +133344 POINT(41.80723381209703 -88.3282076649118) bank133344 +133345 POINT(42.038715132925745 -87.44546451749311) bank133345 +133346 POINT(41.52052159500971 -88.62397412650033) bank133346 +133347 POINT(41.7790689016738 -87.01065141759636) bank133347 +133348 POINT(42.70999283933322 -87.15583783150807) bank133348 +133349 POINT(40.91253722077688 -86.75297888127155) bank133349 +133350 POINT(41.38437156867957 -87.9012944589702) bank133350 +133351 POINT(40.96844901154365 -88.59555404555148) bank133351 +133352 POINT(42.872877716504874 -88.4587861966459) bank133352 +133353 POINT(41.53403250859883 -87.38798944332089) bank133353 +133354 POINT(41.93036779452349 -87.29135764367969) bank133354 +133355 POINT(41.36880445205467 -87.99367458895341) bank133355 +133356 POINT(42.40427355976798 -88.17028156096254) bank133356 +133357 POINT(41.52904920860535 -88.39471718902244) bank133357 +133358 POINT(41.23525508580092 -86.90521929114216) bank133358 +133359 POINT(41.8135598233219 -87.23111246298718) bank133359 +133360 POINT(41.23994850726314 -88.15759212370085) bank133360 +133361 POINT(40.942905995898464 -87.56392377035648) bank133361 +133362 POINT(41.85663499788881 -88.14439201235923) bank133362 +133363 POINT(42.78499955268968 -87.08793246408774) bank133363 +133364 POINT(41.835091589777 -87.8491392735372) bank133364 +133365 POINT(41.871213021588545 -88.42539536731547) bank133365 +133366 POINT(42.40302201665834 -88.48456921026518) bank133366 +133367 POINT(40.92791046953543 -87.66555283390815) bank133367 +133368 POINT(41.70009568308769 -87.76686954494693) bank133368 +133369 POINT(41.68163264142556 -87.10472854342129) bank133369 +133370 POINT(42.71976865306214 -86.65867336248202) bank133370 +133371 POINT(42.0515077289361 -87.93696289215387) bank133371 +133372 POINT(41.75415169678881 -87.11949238759568) bank133372 +133373 POINT(42.516116111074815 -87.06035317990928) bank133373 +133374 POINT(42.10438008659261 -87.02126546816748) bank133374 +133375 POINT(42.358397669723736 -87.64504969017203) bank133375 +133376 POINT(41.27743843762308 -87.58142132203885) bank133376 +133377 POINT(41.54446569506663 -87.00591275621615) bank133377 +133378 POINT(41.823173811242164 -86.78619000464451) bank133378 +133379 POINT(41.1810981223203 -88.55657480903051) bank133379 +133380 POINT(42.020903655834296 -87.5830968488793) bank133380 +133381 POINT(41.49327389656814 -87.81501168817634) bank133381 +133382 POINT(42.08791358679777 -88.38854564986126) bank133382 +133383 POINT(42.70146855003571 -87.1222769301551) bank133383 +133384 POINT(41.38715728301986 -88.36786844942172) bank133384 +133385 POINT(42.86586351162265 -88.62062573562378) bank133385 +133386 POINT(42.461073257399924 -87.59953066553801) bank133386 +133387 POINT(41.54722778388851 -87.49122315846367) bank133387 +133388 POINT(42.1488393239705 -87.76455104036683) bank133388 +133389 POINT(41.40660943456393 -87.97089292336615) bank133389 +133390 POINT(42.56049259111141 -87.05161522660359) bank133390 +133391 POINT(41.06766026947009 -86.90497917565727) bank133391 +133392 POINT(42.23513648053048 -87.57371867890281) bank133392 +133393 POINT(42.62785701190034 -87.28947096046049) bank133393 +133394 POINT(41.66685946811135 -88.58129902252982) bank133394 +133395 POINT(41.701836547989345 -87.83839345800004) bank133395 +133396 POINT(41.3754752267648 -87.15405521595005) bank133396 +133397 POINT(42.3198520210648 -87.86903745000366) bank133397 +133398 POINT(42.52187643897259 -87.03809342025062) bank133398 +133399 POINT(41.54809833560535 -88.35729224918929) bank133399 +133400 POINT(42.36351688252832 -88.10389327474134) bank133400 +133401 POINT(41.54668167295355 -88.50600890307182) bank133401 +133402 POINT(41.69899835246596 -88.38769612744093) bank133402 +133403 POINT(42.24763505464916 -88.44629963207115) bank133403 +133404 POINT(41.14591021692001 -86.71339203148929) bank133404 +133405 POINT(41.87316366356782 -87.48408849200256) bank133405 +133406 POINT(41.77766740877417 -87.40936727733938) bank133406 +133407 POINT(41.19944802870387 -86.63911688930348) bank133407 +133408 POINT(42.19659181127482 -87.96328942813606) bank133408 +133409 POINT(42.175481835772345 -87.52228388597679) bank133409 +133410 POINT(42.79873144791107 -87.86246716491895) bank133410 +133411 POINT(41.010234918562425 -87.00794404863245) bank133411 +133412 POINT(42.22598218742398 -87.41437555978199) bank133412 +133413 POINT(41.32306119501392 -88.31603851125813) bank133413 +133414 POINT(41.32050031935831 -88.62465028881442) bank133414 +133415 POINT(42.04277541610134 -87.17346987999238) bank133415 +133416 POINT(41.76226151473146 -87.24074924718066) bank133416 +133417 POINT(41.911806473352115 -88.0303963690802) bank133417 +133418 POINT(42.28051331358489 -87.94556696059453) bank133418 +133419 POINT(40.93251958533478 -87.29328691052586) bank133419 +133420 POINT(41.5413902969428 -88.32405917188711) bank133420 +133421 POINT(41.41952145165778 -87.39819153927934) bank133421 +133422 POINT(42.41812595723469 -87.49245233238196) bank133422 +133423 POINT(42.78752001853455 -86.96496873529577) bank133423 +133424 POINT(41.40336341176381 -88.44840482534948) bank133424 +133425 POINT(40.92247031532504 -88.00816991440387) bank133425 +133426 POINT(41.53318941136161 -88.15251239469364) bank133426 +133427 POINT(40.97979834993176 -88.12884097536278) bank133427 +133428 POINT(42.79851312087426 -87.66727561354651) bank133428 +133429 POINT(42.65000301599036 -88.0448030999821) bank133429 +133430 POINT(42.67359456106786 -86.94080465182607) bank133430 +133431 POINT(41.49343871315222 -87.98680957678742) bank133431 +133432 POINT(41.08421061856833 -87.5303864978667) bank133432 +133433 POINT(42.5638971608706 -86.6456846504011) bank133433 +133434 POINT(41.06091517881593 -88.19298348353107) bank133434 +133435 POINT(41.14995067578067 -86.73347938947029) bank133435 +133436 POINT(41.20672420839748 -87.44727683435109) bank133436 +133437 POINT(42.08805141862268 -88.17048702465742) bank133437 +133438 POINT(41.24145375658153 -86.73355709482925) bank133438 +133439 POINT(41.1984440239538 -87.73078629871898) bank133439 +133440 POINT(41.25460261823107 -87.03217956058121) bank133440 +133441 POINT(42.725915522065144 -87.44909190417567) bank133441 +133442 POINT(41.706311364623744 -88.36379930283448) bank133442 +133443 POINT(42.043524969511765 -88.26509424370096) bank133443 +133444 POINT(41.245474722261804 -87.2917855327729) bank133444 +133445 POINT(41.35538453313983 -88.2592155475391) bank133445 +133446 POINT(41.07017677831048 -88.12130901893192) bank133446 +133447 POINT(42.21457624261833 -87.67272866063132) bank133447 +133448 POINT(41.93964603964388 -88.24227560449769) bank133448 +133449 POINT(41.10595876813295 -87.88248937475191) bank133449 +133450 POINT(41.468240862653055 -87.91257970554102) bank133450 +133451 POINT(41.71427767474366 -87.35660065667464) bank133451 +133452 POINT(41.34397877718758 -86.94858766818605) bank133452 +133453 POINT(41.13536296342526 -88.17729345819673) bank133453 +133454 POINT(42.095037383540564 -88.34063326227913) bank133454 +133455 POINT(42.10513623205904 -88.20434582815756) bank133455 +133456 POINT(41.36346840627306 -87.95134622288586) bank133456 +133457 POINT(40.91487707443088 -88.11283556624639) bank133457 +133458 POINT(41.88681304440618 -87.22168603290676) bank133458 +133459 POINT(40.92195752661471 -87.98651508452386) bank133459 +133460 POINT(42.2522716082064 -87.11847690278022) bank133460 +133461 POINT(41.09994857105545 -88.22702928853296) bank133461 +133462 POINT(41.326865286825814 -87.4950554211321) bank133462 +133463 POINT(42.35951108915305 -87.72580247377628) bank133463 +133464 POINT(41.585809027921684 -88.37877721493189) bank133464 +133465 POINT(42.568373084510334 -86.80349481924075) bank133465 +133466 POINT(42.47766058912706 -88.02112612452885) bank133466 +133467 POINT(41.28112823964144 -87.7610889030481) bank133467 +133468 POINT(41.43251469125677 -87.84494326781315) bank133468 +133469 POINT(40.96521026532428 -87.07803595716287) bank133469 +133470 POINT(41.85192288283894 -87.24848130350358) bank133470 +133471 POINT(40.95149333406829 -87.52767049233336) bank133471 +133472 POINT(41.54307798925924 -87.28002377006628) bank133472 +133473 POINT(42.748509089946985 -87.15378356144393) bank133473 +133474 POINT(41.393348572050975 -87.02968202611831) bank133474 +133475 POINT(42.19214942694414 -88.51007367808452) bank133475 +133476 POINT(41.5238431831543 -88.1874050686628) bank133476 +133477 POINT(42.858903037023154 -87.46153171024436) bank133477 +133478 POINT(42.36317377091988 -86.94146623643819) bank133478 +133479 POINT(42.19089349184043 -87.42116190958144) bank133479 +133480 POINT(41.9147388921847 -87.49412179463882) bank133480 +133481 POINT(42.20322584259247 -86.90063435939669) bank133481 +133482 POINT(41.3735742193208 -88.49512387860523) bank133482 +133483 POINT(40.94413360619302 -86.90984635185919) bank133483 +133484 POINT(41.69799011587911 -86.8640529276743) bank133484 +133485 POINT(41.102301214673965 -87.26807080342074) bank133485 +133486 POINT(42.160013951952635 -87.96707051862003) bank133486 +133487 POINT(41.08295943184801 -87.2396454131604) bank133487 +133488 POINT(41.35273625475746 -86.86528989670127) bank133488 +133489 POINT(41.592255497338535 -86.66632579229854) bank133489 +133490 POINT(42.19514253574332 -86.87803045451781) bank133490 +133491 POINT(41.21370727402024 -88.48025728135796) bank133491 +133492 POINT(41.10278793415342 -87.14977555913578) bank133492 +133493 POINT(41.61913174509325 -88.50786259239939) bank133493 +133494 POINT(41.15664616396087 -88.60275346132795) bank133494 +133495 POINT(41.1716983614387 -88.30820459103546) bank133495 +133496 POINT(41.72326704379329 -87.79248000057491) bank133496 +133497 POINT(41.37055608858499 -88.15756819113639) bank133497 +133498 POINT(41.05572198689565 -88.35434382072197) bank133498 +133499 POINT(41.79278712024416 -87.29136926453657) bank133499 +133500 POINT(41.31874018823561 -86.86701712969685) bank133500 +133501 POINT(42.62212128572695 -87.65286530331245) bank133501 +133502 POINT(42.50211620043301 -87.40284519446544) bank133502 +133503 POINT(41.58437120825193 -88.13455554792796) bank133503 +133504 POINT(40.91378252945943 -87.20390327915133) bank133504 +133505 POINT(41.164775440411354 -88.33940210440555) bank133505 +133506 POINT(42.28789703495536 -86.72353640488141) bank133506 +133507 POINT(42.466843953139794 -87.97372208436106) bank133507 +133508 POINT(41.14526082560825 -87.45462243985915) bank133508 +133509 POINT(41.83056715498434 -88.22246798335324) bank133509 +133510 POINT(42.73147349949336 -87.50207407714336) bank133510 +133511 POINT(42.159476198150436 -87.54288116506162) bank133511 +133512 POINT(40.94172449275084 -87.51059131750759) bank133512 +133513 POINT(40.87969486127656 -87.41185018384823) bank133513 +133514 POINT(41.34291710383081 -87.17575319234862) bank133514 +133515 POINT(42.445982223229755 -87.00406036764774) bank133515 +133516 POINT(42.75368053867748 -87.59075367157428) bank133516 +133517 POINT(41.2972962268878 -87.8404884677472) bank133517 +133518 POINT(42.24804963714192 -88.06577721886177) bank133518 +133519 POINT(41.75506234975847 -88.20143925623742) bank133519 +133520 POINT(42.79042965337547 -87.2705818301048) bank133520 +133521 POINT(42.81575156218212 -86.96154797898386) bank133521 +133522 POINT(42.15721454942724 -88.14704014382717) bank133522 +133523 POINT(42.74238911180327 -87.62211610735446) bank133523 +133524 POINT(41.82368948143263 -88.2580644173667) bank133524 +133525 POINT(41.90584672999453 -87.51359526218803) bank133525 +133526 POINT(42.58181854247775 -88.22031639093109) bank133526 +133527 POINT(42.85121086049277 -87.40118505716357) bank133527 +133528 POINT(42.76644385203989 -88.37724007876825) bank133528 +133529 POINT(41.49499744656941 -87.22353129612634) bank133529 +133530 POINT(41.204556017863396 -86.82112456674176) bank133530 +133531 POINT(42.45366240820306 -88.5958060873422) bank133531 +133532 POINT(41.48162386692093 -88.48666917683335) bank133532 +133533 POINT(42.305764768454324 -87.21899812645644) bank133533 +133534 POINT(41.523233949072214 -87.31545998521857) bank133534 +133535 POINT(41.27399074947905 -88.35227851921327) bank133535 +133536 POINT(42.342459513649224 -86.7745268020929) bank133536 +133537 POINT(42.0056744349819 -86.94662039999773) bank133537 +133538 POINT(42.09956831319576 -88.3584562743183) bank133538 +133539 POINT(42.54220010948298 -86.77435669280757) bank133539 +133540 POINT(41.206074144638244 -86.69102013374393) bank133540 +133541 POINT(42.68533993442004 -88.09345774341732) bank133541 +133542 POINT(41.235066071419624 -87.63235652268146) bank133542 +133543 POINT(42.41519356683654 -87.3955464097883) bank133543 +133544 POINT(41.00693616564565 -87.57047841404717) bank133544 +133545 POINT(41.078127173422104 -88.19419342881253) bank133545 +133546 POINT(41.65111682818566 -86.8739729991451) bank133546 +133547 POINT(41.99947173229911 -88.09200929652997) bank133547 +133548 POINT(42.667117631549026 -87.14862637464121) bank133548 +133549 POINT(41.87628570494647 -88.3798664093239) bank133549 +133550 POINT(42.43510219265961 -88.54102890488137) bank133550 +133551 POINT(41.32305507310536 -88.42446792746432) bank133551 +133552 POINT(42.77436842247916 -87.82844834114447) bank133552 +133553 POINT(41.71905012280336 -87.5584396173953) bank133553 +133554 POINT(41.90834386793336 -88.35657562261312) bank133554 +133555 POINT(41.3534543975967 -88.5439785112239) bank133555 +133556 POINT(41.999756321870535 -87.87444698445769) bank133556 +133557 POINT(42.76735023458494 -87.23415380641586) bank133557 +133558 POINT(41.47627154777443 -86.72298885488956) bank133558 +133559 POINT(42.69937430601823 -88.44950024992505) bank133559 +133560 POINT(41.024864151366714 -86.72396348840546) bank133560 +133561 POINT(42.34958978682043 -87.15584922209402) bank133561 +133562 POINT(41.34911372894364 -87.46235884937606) bank133562 +133563 POINT(42.50570752232397 -88.33484555786856) bank133563 +133564 POINT(41.038494727319566 -87.72986233662218) bank133564 +133565 POINT(41.22268511415759 -88.02593643549993) bank133565 +133566 POINT(41.43185410261298 -86.71312136477128) bank133566 +133567 POINT(42.69681806633487 -86.70393895309984) bank133567 +133568 POINT(41.11039404256794 -87.19304233464845) bank133568 +133569 POINT(42.327305876229964 -86.88887523204163) bank133569 +133570 POINT(41.800954191924994 -88.1580766318797) bank133570 +133571 POINT(41.04834882696672 -88.27536934290372) bank133571 +133572 POINT(42.22250239841612 -87.10475239831067) bank133572 +133573 POINT(42.18579767847628 -88.37691654852547) bank133573 +133574 POINT(42.38627364064261 -88.4823439997089) bank133574 +133575 POINT(42.71005666697445 -86.76991977850061) bank133575 +133576 POINT(42.02301516921862 -87.6962652406431) bank133576 +133577 POINT(41.32925780837982 -87.53854716282014) bank133577 +133578 POINT(42.102248078668 -87.4686431781517) bank133578 +133579 POINT(42.683352628265446 -88.28933545129631) bank133579 +133580 POINT(40.88315541356439 -88.26615559994438) bank133580 +133581 POINT(41.2183488815989 -88.26046219107778) bank133581 +133582 POINT(42.6533906005388 -87.37698853980366) bank133582 +133583 POINT(41.09870264158327 -88.44604888376347) bank133583 +133584 POINT(42.10703399175879 -88.16003384913154) bank133584 +133585 POINT(42.24992471550019 -88.21944941134662) bank133585 +133586 POINT(42.79557996272747 -87.59142215084046) bank133586 +133587 POINT(42.676941839836225 -87.04494665309987) bank133587 +133588 POINT(41.77387297194312 -87.69613241853041) bank133588 +133589 POINT(42.27879343542894 -88.02984379067564) bank133589 +133590 POINT(42.48858142742146 -87.57248496149955) bank133590 +133591 POINT(42.49355120569442 -87.7423727605981) bank133591 +133592 POINT(42.511200375690294 -88.59907293455247) bank133592 +133593 POINT(41.59095430080144 -86.8490388052231) bank133593 +133594 POINT(42.264181468201464 -88.21047574425643) bank133594 +133595 POINT(40.98255048541385 -88.42344722392384) bank133595 +133596 POINT(41.70002143168081 -87.6223510807984) bank133596 +133597 POINT(41.72472728317481 -87.943000427592) bank133597 +133598 POINT(41.84247502139831 -86.73455674758836) bank133598 +133599 POINT(41.88228698228116 -88.53264354192204) bank133599 +133600 POINT(41.560866654645594 -87.34782032068283) bank133600 +133601 POINT(41.94972721271099 -86.91030569572467) bank133601 +133602 POINT(42.65871800963551 -87.74477092244723) bank133602 +133603 POINT(41.831181869744995 -88.40611396907407) bank133603 +133604 POINT(42.17829370541172 -87.56857380556517) bank133604 +133605 POINT(41.94638674629838 -86.99872745990952) bank133605 +133606 POINT(41.73121676655885 -87.99990891866858) bank133606 +133607 POINT(41.03665068938983 -87.28159031604332) bank133607 +133608 POINT(41.05968154547693 -87.33143159310191) bank133608 +133609 POINT(42.663677164422445 -88.58461757016757) bank133609 +133610 POINT(41.049611618285944 -87.22795844022663) bank133610 +133611 POINT(41.383288137388334 -86.84425468453655) bank133611 +133612 POINT(41.91981617884427 -87.2630798206814) bank133612 +133613 POINT(42.21648195656608 -86.78493045012263) bank133613 +133614 POINT(42.26599871318354 -87.52637726770901) bank133614 +133615 POINT(42.100377670543025 -87.38777727801309) bank133615 +133616 POINT(42.49105105181579 -87.03087948344923) bank133616 +133617 POINT(42.25923385962811 -87.78947495793864) bank133617 +133618 POINT(41.27634773315748 -87.88148437884671) bank133618 +133619 POINT(40.96177279711212 -88.4831332471084) bank133619 +133620 POINT(42.472145691090596 -87.49689497468307) bank133620 +133621 POINT(42.63472339061747 -88.34658794338546) bank133621 +133622 POINT(42.80353483851781 -87.13564621523876) bank133622 +133623 POINT(40.919948857042826 -87.26880570728636) bank133623 +133624 POINT(41.22371039998712 -87.07245622877505) bank133624 +133625 POINT(42.19167755246342 -86.94053603588975) bank133625 +133626 POINT(41.46885662092994 -87.26474343963383) bank133626 +133627 POINT(42.599686669653266 -87.96433720419388) bank133627 +133628 POINT(41.110838084043436 -87.00817378929453) bank133628 +133629 POINT(41.14661392243925 -87.70346816869886) bank133629 +133630 POINT(41.23470462803673 -86.84935101935007) bank133630 +133631 POINT(41.911040716804315 -88.24538134353723) bank133631 +133632 POINT(41.97454038778125 -87.39144053507627) bank133632 +133633 POINT(42.117061490068814 -88.44277516358241) bank133633 +133634 POINT(41.34546127867067 -86.84444143711058) bank133634 +133635 POINT(41.45391108530239 -86.98683395108273) bank133635 +133636 POINT(40.94272489092221 -87.15610812671648) bank133636 +133637 POINT(41.71745197889896 -87.22735754397341) bank133637 +133638 POINT(42.24866319098898 -87.0308269359018) bank133638 +133639 POINT(41.88502903024631 -87.12029118804413) bank133639 +133640 POINT(42.262627880682096 -88.38139359350969) bank133640 +133641 POINT(41.44234603214859 -86.87285843615045) bank133641 +133642 POINT(42.31636255931701 -87.75063681932293) bank133642 +133643 POINT(42.62947993253146 -88.60861137947461) bank133643 +133644 POINT(41.600023246573514 -87.38338043007992) bank133644 +133645 POINT(42.13217216528392 -88.1287746689626) bank133645 +133646 POINT(41.00446195602779 -87.44805478255327) bank133646 +133647 POINT(41.05661475218241 -88.29365190697735) bank133647 +133648 POINT(41.572444753508435 -88.45819018783281) bank133648 +133649 POINT(41.59273722254906 -88.1660141750276) bank133649 +133650 POINT(40.98920970225696 -87.67708932022721) bank133650 +133651 POINT(41.347165868736155 -87.95219017105295) bank133651 +133652 POINT(42.00883892191508 -87.8991046707032) bank133652 +133653 POINT(42.055220225890075 -86.88941978921356) bank133653 +133654 POINT(41.12783196943859 -86.8860882864517) bank133654 +133655 POINT(41.056811487974535 -87.51324999535726) bank133655 +133656 POINT(42.290543062293956 -87.96008172604223) bank133656 +133657 POINT(42.062986801683124 -87.27588294580694) bank133657 +133658 POINT(41.450649283248346 -88.42038006398762) bank133658 +133659 POINT(41.65396961158581 -87.80692585945256) bank133659 +133660 POINT(42.73325201820023 -87.7249038190134) bank133660 +133661 POINT(41.1075291919029 -87.39442724479102) bank133661 +133662 POINT(42.32496770250285 -88.55840718568547) bank133662 +133663 POINT(41.84356491101149 -87.61939519097456) bank133663 +133664 POINT(42.758138644943195 -88.32764019333351) bank133664 +133665 POINT(42.812206065833664 -88.56528049220628) bank133665 +133666 POINT(42.70981910293638 -87.71248364070506) bank133666 +133667 POINT(42.701469120250266 -87.15510111686896) bank133667 +133668 POINT(41.04924224921558 -88.28325878899139) bank133668 +133669 POINT(41.95895219897679 -87.64920747006927) bank133669 +133670 POINT(41.73237830829291 -87.33654136073177) bank133670 +133671 POINT(41.397628662083086 -88.38127303074266) bank133671 +133672 POINT(41.05127250650814 -87.52637046757809) bank133672 +133673 POINT(42.75750360939874 -87.19629093650505) bank133673 +133674 POINT(41.909734701950214 -88.37012765859049) bank133674 +133675 POINT(40.899277296826675 -87.33758231721568) bank133675 +133676 POINT(41.13362112372635 -87.30758338544956) bank133676 +133677 POINT(41.67221105749254 -88.04530512311686) bank133677 +133678 POINT(41.56663789363496 -87.74094886540092) bank133678 +133679 POINT(41.85542513891396 -87.81858281701439) bank133679 +133680 POINT(41.15872117438026 -88.55960264127998) bank133680 +133681 POINT(42.75629510279497 -87.19472448425161) bank133681 +133682 POINT(42.69072447679023 -87.23095677938447) bank133682 +133683 POINT(41.41568587146025 -86.9612005397987) bank133683 +133684 POINT(41.42273985103594 -87.0048342747419) bank133684 +133685 POINT(42.77110276882447 -87.93059239670234) bank133685 +133686 POINT(41.34296596184501 -87.66268962504411) bank133686 +133687 POINT(42.39568702435974 -86.74998809723084) bank133687 +133688 POINT(42.709380703874345 -87.51191566010353) bank133688 +133689 POINT(42.06413503553932 -88.46989678778715) bank133689 +133690 POINT(41.90679047560537 -87.88453992134907) bank133690 +133691 POINT(40.94970249546959 -88.59127672248425) bank133691 +133692 POINT(40.885093046306515 -87.55414438781565) bank133692 +133693 POINT(41.20840332863771 -88.19573344335961) bank133693 +133694 POINT(42.19389960987402 -88.577942379918) bank133694 +133695 POINT(42.49864341159411 -86.83620248654339) bank133695 +133696 POINT(41.85069363357632 -87.92628737260955) bank133696 +133697 POINT(42.022752065825514 -88.12893369088762) bank133697 +133698 POINT(41.904112366643055 -86.82758126511425) bank133698 +133699 POINT(41.58203906189056 -88.4580570532956) bank133699 +133700 POINT(41.36749195537911 -87.70947937284811) bank133700 +133701 POINT(40.97068336808514 -87.07586976167032) bank133701 +133702 POINT(40.98617860535391 -87.91122900223814) bank133702 +133703 POINT(42.090460273690475 -87.47606650540195) bank133703 +133704 POINT(42.79083339394861 -87.17931119975961) bank133704 +133705 POINT(41.58388157501062 -87.67263537453469) bank133705 +133706 POINT(41.424441838507605 -88.3487680953233) bank133706 +133707 POINT(42.85800784303566 -87.12213499867522) bank133707 +133708 POINT(41.38573663990395 -87.39298808719856) bank133708 +133709 POINT(42.769207074626394 -88.07185050786971) bank133709 +133710 POINT(42.832585268324664 -86.79711232956272) bank133710 +133711 POINT(41.73210192895485 -86.79341836944769) bank133711 +133712 POINT(42.390794952190795 -87.71860309328605) bank133712 +133713 POINT(42.082215609771374 -87.21410923316661) bank133713 +133714 POINT(41.887795265209746 -87.7591632793244) bank133714 +133715 POINT(42.20197052204885 -87.48338710976107) bank133715 +133716 POINT(41.00692544694493 -87.61919562252649) bank133716 +133717 POINT(42.348961964833904 -86.80330049603262) bank133717 +133718 POINT(41.71830654718247 -86.67426481201348) bank133718 +133719 POINT(42.83257725440361 -87.70517198391883) bank133719 +133720 POINT(41.43012608413977 -88.47177216063369) bank133720 +133721 POINT(41.710757531450895 -88.39186959547698) bank133721 +133722 POINT(41.59540770145699 -87.45285407810529) bank133722 +133723 POINT(42.23818792559099 -86.9989186594112) bank133723 +133724 POINT(42.34477014944553 -86.88153307770474) bank133724 +133725 POINT(41.219220876660884 -87.67995180623853) bank133725 +133726 POINT(41.076842274470714 -88.62859759147715) bank133726 +133727 POINT(41.85515346214473 -87.94733286771026) bank133727 +133728 POINT(40.92673836381315 -87.57688891887672) bank133728 +133729 POINT(40.998757122233286 -88.40493721118563) bank133729 +133730 POINT(41.711229667047625 -87.17586895075048) bank133730 +133731 POINT(42.64020964965666 -87.62313918427157) bank133731 +133732 POINT(41.825974444520654 -88.35710767655371) bank133732 +133733 POINT(42.42631830263716 -87.88742219470774) bank133733 +133734 POINT(41.28838904280002 -87.29914255812791) bank133734 +133735 POINT(41.76421834727778 -87.47064146398476) bank133735 +133736 POINT(41.919004992674374 -86.8388047194327) bank133736 +133737 POINT(41.232371395825304 -87.86572927274334) bank133737 +133738 POINT(41.35139172197224 -87.39689522783077) bank133738 +133739 POINT(41.15656410590922 -87.75821801345468) bank133739 +133740 POINT(42.5303227865261 -87.35266690383976) bank133740 +133741 POINT(42.00097928697683 -88.17575432954422) bank133741 +133742 POINT(42.48250215902644 -86.92034684409687) bank133742 +133743 POINT(41.3371650781661 -88.21985394676611) bank133743 +133744 POINT(42.62415472414275 -86.9806253181636) bank133744 +133745 POINT(42.57983849720656 -87.9675424298292) bank133745 +133746 POINT(40.99294752767016 -87.24731286947176) bank133746 +133747 POINT(40.90958070380491 -87.06830552058986) bank133747 +133748 POINT(41.70215048159385 -86.7360793164908) bank133748 +133749 POINT(42.59913500104018 -86.71052555631769) bank133749 +133750 POINT(41.669314700650894 -87.0877677080494) bank133750 +133751 POINT(42.19753592361979 -88.23417862295783) bank133751 +133752 POINT(41.97317146125392 -86.87005431122088) bank133752 +133753 POINT(42.448412702249406 -87.72416145532877) bank133753 +133754 POINT(41.93991922335649 -88.15950006871348) bank133754 +133755 POINT(42.03587745810619 -87.60971516826416) bank133755 +133756 POINT(42.481664652405364 -87.58731104763918) bank133756 +133757 POINT(42.8780062959223 -88.50838214779401) bank133757 +133758 POINT(42.237960083855285 -88.16269829089701) bank133758 +133759 POINT(41.4266699405743 -86.85099252721201) bank133759 +133760 POINT(42.47879682513409 -86.94102153512154) bank133760 +133761 POINT(42.07443136969018 -87.90184258564001) bank133761 +133762 POINT(41.37266686797319 -87.04367989116518) bank133762 +133763 POINT(40.892268325507914 -86.97888777499432) bank133763 +133764 POINT(41.77857029010487 -87.68476431747895) bank133764 +133765 POINT(41.69533140054496 -87.87426653251337) bank133765 +133766 POINT(41.58902894018775 -88.5253174340392) bank133766 +133767 POINT(41.392037103742375 -87.30626876242957) bank133767 +133768 POINT(42.75320981242118 -88.54296773152116) bank133768 +133769 POINT(40.88974497899692 -87.35112106047548) bank133769 +133770 POINT(42.534799015478946 -88.47610036275987) bank133770 +133771 POINT(41.52517330571568 -88.13413536865431) bank133771 +133772 POINT(41.49749354336435 -88.5866391571593) bank133772 +133773 POINT(42.28629460804692 -87.4883095287583) bank133773 +133774 POINT(41.379951980514015 -87.27526445193557) bank133774 +133775 POINT(42.785737671587384 -86.70397662878271) bank133775 +133776 POINT(42.27662615422788 -87.20263685431622) bank133776 +133777 POINT(41.89990323423749 -88.55971036258703) bank133777 +133778 POINT(42.71760312506823 -86.85237977006439) bank133778 +133779 POINT(42.21780514386546 -88.39261146746517) bank133779 +133780 POINT(41.099943352951215 -86.88622699646729) bank133780 +133781 POINT(42.090461801937856 -88.53884404195256) bank133781 +133782 POINT(41.05747156670927 -87.8155803333346) bank133782 +133783 POINT(40.93528084747311 -87.83266220985705) bank133783 +133784 POINT(42.34114088033687 -87.21767835606138) bank133784 +133785 POINT(41.2332018065804 -86.77938040326528) bank133785 +133786 POINT(42.58308014001517 -87.51385921683831) bank133786 +133787 POINT(42.692628421131964 -86.9335443420736) bank133787 +133788 POINT(41.32522755134186 -87.8026971053825) bank133788 +133789 POINT(40.90469521088347 -88.60820520915361) bank133789 +133790 POINT(41.324277699675356 -87.07518386951872) bank133790 +133791 POINT(41.05510456106849 -87.64111750936807) bank133791 +133792 POINT(42.13594470236854 -88.38362834373328) bank133792 +133793 POINT(41.82254685251072 -87.63661934585949) bank133793 +133794 POINT(42.364132953171655 -86.72912434966267) bank133794 +133795 POINT(42.06592223705027 -87.59633433871508) bank133795 +133796 POINT(42.852930741269006 -88.21953944380535) bank133796 +133797 POINT(41.211895322693806 -88.48705630641176) bank133797 +133798 POINT(41.805210980399096 -88.37279645002026) bank133798 +133799 POINT(40.93082207058096 -87.83273114849355) bank133799 +133800 POINT(42.41132258008887 -88.32075928044144) bank133800 +133801 POINT(42.43702178918012 -87.16114008451666) bank133801 +133802 POINT(41.09144220418445 -87.52815403357279) bank133802 +133803 POINT(41.05837282588171 -86.7823336267046) bank133803 +133804 POINT(41.39408834900991 -88.53896081602319) bank133804 +133805 POINT(41.9492144824871 -88.18150605384561) bank133805 +133806 POINT(41.64534208880299 -88.30178050747074) bank133806 +133807 POINT(41.871705798456574 -88.05304938403685) bank133807 +133808 POINT(41.8713967233777 -86.89587498301502) bank133808 +133809 POINT(41.56104504426402 -87.18266854784035) bank133809 +133810 POINT(42.46859312685475 -87.09480007473198) bank133810 +133811 POINT(42.493126814989466 -87.73031571148543) bank133811 +133812 POINT(40.8788342123929 -88.11474074532484) bank133812 +133813 POINT(42.62835356148082 -86.75921204346022) bank133813 +133814 POINT(42.174213711835016 -86.83165337069184) bank133814 +133815 POINT(41.67287110711604 -87.82170748231182) bank133815 +133816 POINT(41.82938690378722 -87.39565602109467) bank133816 +133817 POINT(42.633743898159565 -87.03613481144542) bank133817 +133818 POINT(41.25908672373847 -86.63478792012062) bank133818 +133819 POINT(41.40733844380073 -88.02600124445112) bank133819 +133820 POINT(41.0086092905202 -87.84911795229036) bank133820 +133821 POINT(42.25183764425794 -87.99845592075206) bank133821 +133822 POINT(42.14844047339048 -87.0687614263229) bank133822 +133823 POINT(40.945296973613225 -87.89228631540206) bank133823 +133824 POINT(41.96091167819687 -88.35254902860963) bank133824 +133825 POINT(42.76743308903393 -87.52058838106845) bank133825 +133826 POINT(41.677487718921576 -86.7966030008785) bank133826 +133827 POINT(41.57186149539529 -87.82006100457887) bank133827 +133828 POINT(42.16746397171135 -87.94426988766435) bank133828 +133829 POINT(42.48938048940238 -87.1858655220458) bank133829 +133830 POINT(41.923521484780814 -87.10595238602353) bank133830 +133831 POINT(41.77758701312321 -88.11220915920613) bank133831 +133832 POINT(42.041661208085095 -88.37470123272605) bank133832 +133833 POINT(42.17783666105391 -86.89902178096068) bank133833 +133834 POINT(42.42696362625002 -88.53855150715549) bank133834 +133835 POINT(41.86908107564147 -88.29760991567377) bank133835 +133836 POINT(41.45145129421976 -86.69993556761577) bank133836 +133837 POINT(42.35237411066388 -86.75064812815131) bank133837 +133838 POINT(42.718519429955215 -87.1561736046785) bank133838 +133839 POINT(41.775651932030804 -86.70639841102206) bank133839 +133840 POINT(42.561409803743814 -87.623492116075) bank133840 +133841 POINT(42.44149492304021 -87.8058299962501) bank133841 +133842 POINT(42.47513584448612 -86.90163431302268) bank133842 +133843 POINT(42.45746876875448 -87.51434853311567) bank133843 +133844 POINT(42.175066313843416 -86.80256774453579) bank133844 +133845 POINT(42.69837696021062 -87.17221335746339) bank133845 +133846 POINT(41.923584876468546 -87.61073633293657) bank133846 +133847 POINT(41.02489284834284 -86.63419469799126) bank133847 +133848 POINT(42.56698406565359 -88.43000095849659) bank133848 +133849 POINT(42.7504258657229 -88.05036405068232) bank133849 +133850 POINT(42.81004034314482 -86.85029745612928) bank133850 +133851 POINT(42.02249079432542 -87.89143109906044) bank133851 +133852 POINT(41.06291491915821 -88.47759893043377) bank133852 +133853 POINT(42.675545247506456 -87.34752385341852) bank133853 +133854 POINT(41.717858514103185 -87.27764205169012) bank133854 +133855 POINT(42.50743632467932 -87.44602522163896) bank133855 +133856 POINT(42.655234146353706 -87.97787650242785) bank133856 +133857 POINT(41.41941313680819 -87.37524628123846) bank133857 +133858 POINT(41.69601593308938 -88.04101748692145) bank133858 +133859 POINT(42.09285141014106 -87.20934171526865) bank133859 +133860 POINT(41.04944616971756 -87.61058716978889) bank133860 +133861 POINT(40.885832021549525 -87.48097535082188) bank133861 +133862 POINT(42.87790521909968 -87.26309718936697) bank133862 +133863 POINT(41.05503783276667 -88.05263837719961) bank133863 +133864 POINT(42.745643500596536 -88.43781661638154) bank133864 +133865 POINT(41.784607107989736 -88.07146793772938) bank133865 +133866 POINT(42.33475908422546 -87.74292330200814) bank133866 +133867 POINT(41.005059348786396 -88.14454614196973) bank133867 +133868 POINT(41.487818891097525 -88.61580517906775) bank133868 +133869 POINT(41.759035051219534 -87.42539334997157) bank133869 +133870 POINT(42.095447859506336 -86.77101111120805) bank133870 +133871 POINT(41.28291617731977 -87.31498132468498) bank133871 +133872 POINT(42.163371911865184 -88.50066733648104) bank133872 +133873 POINT(40.90712413167975 -88.11288442140189) bank133873 +133874 POINT(40.95824328951339 -88.30261973807112) bank133874 +133875 POINT(42.729658457084454 -87.00953060516017) bank133875 +133876 POINT(42.07519435449535 -86.98089195903118) bank133876 +133877 POINT(41.721069254817785 -86.90355827184042) bank133877 +133878 POINT(42.554480792124956 -88.0321234819825) bank133878 +133879 POINT(41.98910938615384 -87.0569275407248) bank133879 +133880 POINT(42.659125269989815 -87.46772385804582) bank133880 +133881 POINT(41.60680405575884 -88.53860758528289) bank133881 +133882 POINT(42.0696946453737 -87.75734950710127) bank133882 +133883 POINT(41.12253249840776 -88.13545970646281) bank133883 +133884 POINT(40.95693779802619 -87.21923176140389) bank133884 +133885 POINT(41.42760674990835 -88.12030916040149) bank133885 +133886 POINT(42.72150245004922 -86.93865810396753) bank133886 +133887 POINT(41.435102634904226 -87.38729551029138) bank133887 +133888 POINT(40.97472776494023 -87.07843586828483) bank133888 +133889 POINT(41.73244338768083 -87.9036149658901) bank133889 +133890 POINT(42.05008087780274 -87.5774685582985) bank133890 +133891 POINT(42.368754479550326 -87.69785891977969) bank133891 +133892 POINT(42.7210774829321 -87.93128745793142) bank133892 +133893 POINT(42.63580801079125 -88.48607003003967) bank133893 +133894 POINT(40.894025966371366 -87.2530574122347) bank133894 +133895 POINT(42.06782137376141 -87.5869154928441) bank133895 +133896 POINT(42.45055405004499 -88.54492216067926) bank133896 +133897 POINT(42.498315801151534 -87.80865889500977) bank133897 +133898 POINT(41.142030084659616 -87.96422803811845) bank133898 +133899 POINT(42.14598634229062 -88.52003337359304) bank133899 +133900 POINT(41.25660972678423 -86.71848400034705) bank133900 +133901 POINT(42.770125858320924 -86.6906596476966) bank133901 +133902 POINT(41.71660256341059 -88.60235265767093) bank133902 +133903 POINT(42.078354449981944 -86.88598331022499) bank133903 +133904 POINT(42.22520131777059 -88.24858212370589) bank133904 +133905 POINT(42.38690200110946 -86.66876790172284) bank133905 +133906 POINT(42.38140915255946 -86.79947846061522) bank133906 +133907 POINT(42.17262001359927 -87.33887069089681) bank133907 +133908 POINT(41.69141328443223 -87.66044046402772) bank133908 +133909 POINT(41.25810335784218 -86.99876080788925) bank133909 +133910 POINT(42.12141688046684 -87.05702356814415) bank133910 +133911 POINT(41.781428754333156 -87.67095501658495) bank133911 +133912 POINT(41.9089349776437 -86.73744635904941) bank133912 +133913 POINT(41.59062747314041 -88.30707567754293) bank133913 +133914 POINT(41.40012817000023 -86.72499850817876) bank133914 +133915 POINT(41.05897858879293 -88.17766785747465) bank133915 +133916 POINT(41.10184882710971 -87.42529483803408) bank133916 +133917 POINT(42.79711129703965 -87.14440555800132) bank133917 +133918 POINT(42.139687857758915 -87.48134447921392) bank133918 +133919 POINT(41.145936929513184 -87.9078479182651) bank133919 +133920 POINT(42.52709803360897 -87.6757208359722) bank133920 +133921 POINT(42.51916734487189 -87.98280417513791) bank133921 +133922 POINT(42.70861810189289 -87.11386164966407) bank133922 +133923 POINT(41.55175427692841 -87.98588660425975) bank133923 +133924 POINT(42.59364018836783 -88.37127564120038) bank133924 +133925 POINT(42.68255387425849 -88.19300331001159) bank133925 +133926 POINT(42.55300894960477 -87.45302073494886) bank133926 +133927 POINT(42.146633804280754 -88.3796185352929) bank133927 +133928 POINT(42.87514628098975 -88.20667740087144) bank133928 +133929 POINT(41.1747744627994 -87.30966880403126) bank133929 +133930 POINT(42.132093458433836 -86.81624709044453) bank133930 +133931 POINT(41.03077000180063 -87.82275974006718) bank133931 +133932 POINT(42.84643102058849 -86.96823520415539) bank133932 +133933 POINT(41.54452120454571 -88.46942347906052) bank133933 +133934 POINT(41.10801844713542 -88.30274171002263) bank133934 +133935 POINT(41.5747069396 -87.96742250134982) bank133935 +133936 POINT(42.363657742472355 -87.7837006053146) bank133936 +133937 POINT(41.28980981405357 -87.00778598391196) bank133937 +133938 POINT(42.436138519988866 -88.2918642612043) bank133938 +133939 POINT(41.75355764659456 -88.36937619323211) bank133939 +133940 POINT(42.09004158701521 -86.67962822758494) bank133940 +133941 POINT(41.102117741638274 -86.71715790326033) bank133941 +133942 POINT(41.85131428526 -86.79900767483527) bank133942 +133943 POINT(41.27048935838064 -88.50300021850501) bank133943 +133944 POINT(42.436085552186995 -87.41243552107186) bank133944 +133945 POINT(42.62396053896611 -88.08315764192449) bank133945 +133946 POINT(41.659141252690205 -88.44338650073122) bank133946 +133947 POINT(41.816010064699164 -87.75799223222042) bank133947 +133948 POINT(42.67413746603068 -87.86244756800482) bank133948 +133949 POINT(41.16178849315832 -87.2379694128419) bank133949 +133950 POINT(41.16171182484289 -88.02655235943465) bank133950 +133951 POINT(42.28952198108697 -87.85850072253317) bank133951 +133952 POINT(41.813793010428796 -88.21470214288664) bank133952 +133953 POINT(40.98213908806096 -86.65174686865517) bank133953 +133954 POINT(41.32960039414318 -86.81338303144445) bank133954 +133955 POINT(42.42517285421574 -87.11243442611901) bank133955 +133956 POINT(41.23326651678115 -87.75515440306157) bank133956 +133957 POINT(41.41000142468848 -88.61954854715214) bank133957 +133958 POINT(42.48088319094623 -88.0837995221333) bank133958 +133959 POINT(41.38957997035954 -88.22461709764342) bank133959 +133960 POINT(41.656276759191904 -86.8866548239604) bank133960 +133961 POINT(41.09771570175303 -87.95530019185132) bank133961 +133962 POINT(42.079558775507394 -86.72737200381539) bank133962 +133963 POINT(41.49060649901457 -87.1846532632096) bank133963 +133964 POINT(41.08092353815933 -87.25024400613138) bank133964 +133965 POINT(41.02040799784963 -87.00702840561287) bank133965 +133966 POINT(42.010579548262186 -87.46889927569293) bank133966 +133967 POINT(41.64694405851818 -87.01425517653917) bank133967 +133968 POINT(42.116938984021466 -88.47119317329802) bank133968 +133969 POINT(42.251548862221455 -87.98847832259162) bank133969 +133970 POINT(42.85287273703198 -87.77130631361948) bank133970 +133971 POINT(41.520799953134 -88.31039219765788) bank133971 +133972 POINT(42.62356506420215 -87.45854127424423) bank133972 +133973 POINT(42.599084471503076 -88.46691631572082) bank133973 +133974 POINT(41.62715204907225 -88.27349896276024) bank133974 +133975 POINT(41.7913272192016 -88.17855349217234) bank133975 +133976 POINT(41.042938533226156 -88.0533594456655) bank133976 +133977 POINT(41.75228897253226 -87.16235454570995) bank133977 +133978 POINT(41.40031789191765 -88.00951527241845) bank133978 +133979 POINT(41.59762863751609 -88.28221196554746) bank133979 +133980 POINT(42.58825416667159 -87.59161213226282) bank133980 +133981 POINT(42.86870623284346 -88.40818139074948) bank133981 +133982 POINT(41.712162342329044 -87.31446950222309) bank133982 +133983 POINT(40.93846036417243 -88.03143934346782) bank133983 +133984 POINT(41.84498785378811 -86.82683258492621) bank133984 +133985 POINT(42.718888648720124 -86.96824335972263) bank133985 +133986 POINT(42.41876360817561 -88.47374867909797) bank133986 +133987 POINT(41.76362096106818 -87.03126024852975) bank133987 +133988 POINT(41.97443385351551 -86.7503817611893) bank133988 +133989 POINT(42.093096805904246 -86.80080890100756) bank133989 +133990 POINT(41.08318412733733 -87.32082538606325) bank133990 +133991 POINT(41.337289949000215 -87.83634721840767) bank133991 +133992 POINT(41.71899608172287 -86.76154534227994) bank133992 +133993 POINT(41.69026857853342 -88.07990097549471) bank133993 +133994 POINT(41.58504244039858 -86.87007306529256) bank133994 +133995 POINT(41.91219358619124 -88.5850170889851) bank133995 +133996 POINT(42.792122724830556 -88.47516752512375) bank133996 +133997 POINT(42.30566732683014 -87.05849500426679) bank133997 +133998 POINT(42.682416823538 -87.87205163095264) bank133998 +133999 POINT(41.53966212466465 -87.20130052868826) bank133999 +134000 POINT(41.80427223992058 -87.12180378771279) bank134000 +134001 POINT(42.42468824902604 -88.38182091643792) bank134001 +134002 POINT(42.64456250190373 -87.91114886780129) bank134002 +134003 POINT(42.31808993556493 -88.56667114078125) bank134003 +134004 POINT(40.917846018895204 -87.13578702970196) bank134004 +134005 POINT(41.20366978485006 -88.33290890935196) bank134005 +134006 POINT(42.14135116451138 -87.18119998355428) bank134006 +134007 POINT(40.90516226429831 -87.36631080933753) bank134007 +134008 POINT(41.18922185246666 -86.63737870480101) bank134008 +134009 POINT(41.35122458323603 -88.44652598989589) bank134009 +134010 POINT(41.376371430765396 -88.25126031605413) bank134010 +134011 POINT(42.19243589236602 -87.77915556736652) bank134011 +134012 POINT(42.70088729605832 -88.11993159877652) bank134012 +134013 POINT(41.60026137968116 -88.1547106947535) bank134013 +134014 POINT(42.46484099400628 -86.93098259870328) bank134014 +134015 POINT(41.95679672561668 -87.01421053784856) bank134015 +134016 POINT(41.79958848830216 -88.09588687192954) bank134016 +134017 POINT(42.493406921326184 -87.68415943599379) bank134017 +134018 POINT(42.11974216861356 -87.66662115699877) bank134018 +134019 POINT(41.90495695886905 -87.4318663105765) bank134019 +134020 POINT(42.20339634990297 -86.90063791802358) bank134020 +134021 POINT(41.95152902937528 -88.39920079159857) bank134021 +134022 POINT(42.44947687257423 -87.31336342971647) bank134022 +134023 POINT(42.20666926502713 -86.66587182984945) bank134023 +134024 POINT(42.523599177820515 -87.62880947046341) bank134024 +134025 POINT(42.06028285137088 -86.73569480579788) bank134025 +134026 POINT(41.378033337155756 -87.31938179751398) bank134026 +134027 POINT(42.57267519720522 -87.56427999668152) bank134027 +134028 POINT(42.33459756975116 -87.37436242098929) bank134028 +134029 POINT(41.99732879053436 -88.6226139704459) bank134029 +134030 POINT(42.49394665613807 -88.09782330803131) bank134030 +134031 POINT(41.80608805736619 -87.44939532268637) bank134031 +134032 POINT(41.86239575604231 -87.75564444290076) bank134032 +134033 POINT(42.048868568736395 -87.44643548750513) bank134033 +134034 POINT(41.002729293519494 -88.40029681887016) bank134034 +134035 POINT(41.706666516657826 -86.90359328373057) bank134035 +134036 POINT(41.02210467228682 -87.34351887732382) bank134036 +134037 POINT(42.01739857285077 -88.15716974534828) bank134037 +134038 POINT(41.02541698743182 -88.00471981353542) bank134038 +134039 POINT(40.89030637974704 -87.60411931382265) bank134039 +134040 POINT(42.572422336458466 -88.4572522254845) bank134040 +134041 POINT(42.021108218080414 -88.30341499292983) bank134041 +134042 POINT(41.24334086095523 -87.98215266401101) bank134042 +134043 POINT(40.89019016778823 -87.29830484707189) bank134043 +134044 POINT(41.22144854263655 -88.38203571051837) bank134044 +134045 POINT(41.45185646133644 -87.21499498337887) bank134045 +134046 POINT(41.965465582666496 -88.00559225793944) bank134046 +134047 POINT(41.068177958144254 -88.19241042084786) bank134047 +134048 POINT(42.73684076916138 -87.3425610285978) bank134048 +134049 POINT(42.28130553402601 -87.28843197385173) bank134049 +134050 POINT(41.58818490631877 -86.79618224345624) bank134050 +134051 POINT(41.71211140623323 -87.13387175455959) bank134051 +134052 POINT(42.48734720093881 -88.5605084771474) bank134052 +134053 POINT(42.266913674463495 -86.63741652955898) bank134053 +134054 POINT(41.33887847708597 -87.87944941954758) bank134054 +134055 POINT(42.21976702113878 -87.67305138678675) bank134055 +134056 POINT(40.98710402876418 -86.88856578152573) bank134056 +134057 POINT(41.30840831092614 -87.18495811716596) bank134057 +134058 POINT(41.116707525675146 -87.51857230088373) bank134058 +134059 POINT(42.566595200912964 -87.27535949486482) bank134059 +134060 POINT(42.468663895384594 -87.59682130610986) bank134060 +134061 POINT(42.460251604901586 -88.23083387492991) bank134061 +134062 POINT(41.53722637865422 -87.3721914499143) bank134062 +134063 POINT(42.41015075152069 -87.64665652823597) bank134063 +134064 POINT(42.422877077100644 -87.63510951866319) bank134064 +134065 POINT(40.979215593717534 -88.022830416056) bank134065 +134066 POINT(41.887864164154145 -86.969481844106) bank134066 +134067 POINT(42.11604594918999 -87.45316856580511) bank134067 +134068 POINT(42.07953990025274 -88.44759818486402) bank134068 +134069 POINT(40.99766219077031 -87.17533723159862) bank134069 +134070 POINT(41.25499444621983 -87.5872505417682) bank134070 +134071 POINT(41.64747276634408 -87.93501554231052) bank134071 +134072 POINT(42.0367456690437 -88.25664162473) bank134072 +134073 POINT(41.990621439197525 -86.73990523498702) bank134073 +134074 POINT(41.304961535306674 -88.20964020614043) bank134074 +134075 POINT(41.2628438745604 -88.4978803725254) bank134075 +134076 POINT(41.138563671813635 -87.8880544284722) bank134076 +134077 POINT(42.27509846780792 -87.81538755086116) bank134077 +134078 POINT(41.19589545346666 -88.57645407837946) bank134078 +134079 POINT(41.66712047214491 -87.47797790971516) bank134079 +134080 POINT(41.34041337294741 -86.77922486019453) bank134080 +134081 POINT(42.04706227994933 -87.11116301114734) bank134081 +134082 POINT(42.032429142784146 -86.98855855111033) bank134082 +134083 POINT(42.169543275845776 -87.43349280381528) bank134083 +134084 POINT(42.40515104365685 -86.7493856588618) bank134084 +134085 POINT(40.92973093591286 -88.59484535815051) bank134085 +134086 POINT(41.63738279830349 -88.39660467587781) bank134086 +134087 POINT(41.967535839182695 -87.72724918435036) bank134087 +134088 POINT(41.06259746196947 -86.80777742396796) bank134088 +134089 POINT(42.763451757869085 -88.24005927274416) bank134089 +134090 POINT(42.84570874840551 -87.98713986800169) bank134090 +134091 POINT(42.0206790147155 -88.03977195888714) bank134091 +134092 POINT(42.86967917889225 -88.09071746451376) bank134092 +134093 POINT(42.45974686316037 -87.97196721284767) bank134093 +134094 POINT(42.16302250382111 -87.8449213537942) bank134094 +134095 POINT(42.70542816021852 -86.71949590243183) bank134095 +134096 POINT(42.838319276872745 -88.16234476067676) bank134096 +134097 POINT(42.38264998862047 -87.7314353888504) bank134097 +134098 POINT(42.626693816044785 -87.33925077987745) bank134098 +134099 POINT(41.56207922746594 -87.21547249073312) bank134099 +134100 POINT(42.61807245439579 -87.93521726531534) bank134100 +134101 POINT(41.52195973542905 -87.43037900207877) bank134101 +134102 POINT(41.07538266163941 -87.5082684864158) bank134102 +134103 POINT(42.33340470349479 -87.23579305585001) bank134103 +134104 POINT(41.63234019218458 -87.98036214189746) bank134104 +134105 POINT(41.07862060492383 -87.32104443439502) bank134105 +134106 POINT(42.16361981694327 -87.67560684736617) bank134106 +134107 POINT(42.76324709514688 -88.38149218072338) bank134107 +134108 POINT(41.130482622060775 -87.33380821986202) bank134108 +134109 POINT(41.552452041908914 -86.85788797477078) bank134109 +134110 POINT(42.40134068799482 -87.64308984216093) bank134110 +134111 POINT(40.97659977906852 -87.76451127906886) bank134111 +134112 POINT(42.660630763867424 -86.84533742192511) bank134112 +134113 POINT(41.2935269504189 -88.33766446606312) bank134113 +134114 POINT(42.10334178079863 -86.89033337324464) bank134114 +134115 POINT(42.41884101968173 -87.53403554749636) bank134115 +134116 POINT(40.95909075210562 -87.36478401713342) bank134116 +134117 POINT(42.171940606111164 -87.47699456206716) bank134117 +134118 POINT(41.705348597850666 -87.47826682051843) bank134118 +134119 POINT(42.733266836753764 -87.78641439247615) bank134119 +134120 POINT(41.21449705625296 -87.22945196375238) bank134120 +134121 POINT(42.57505001726681 -87.09698159650056) bank134121 +134122 POINT(42.80068674404753 -87.77918927467167) bank134122 +134123 POINT(41.472767448459855 -88.53290387184225) bank134123 +134124 POINT(41.644533435477484 -87.63837395828119) bank134124 +134125 POINT(42.47725466241428 -87.46587214191989) bank134125 +134126 POINT(41.42264726612623 -87.27155441518143) bank134126 +134127 POINT(40.97046457640834 -86.99159389752924) bank134127 +134128 POINT(42.471692707045925 -88.51035717494707) bank134128 +134129 POINT(41.65535417067838 -87.347452888266) bank134129 +134130 POINT(42.67533471870338 -88.36229538132537) bank134130 +134131 POINT(40.931439243880625 -87.19469972462763) bank134131 +134132 POINT(42.01505815242628 -88.45161328829488) bank134132 +134133 POINT(42.438868414582785 -87.95861650412412) bank134133 +134134 POINT(41.352299352261 -86.91002251902978) bank134134 +134135 POINT(41.9121661919013 -87.201245830935) bank134135 +134136 POINT(41.5182109715263 -87.79501286644438) bank134136 +134137 POINT(42.51969637280029 -87.43179629235064) bank134137 +134138 POINT(41.77151962682937 -88.56682812725657) bank134138 +134139 POINT(42.862237762960646 -86.92262353578876) bank134139 +134140 POINT(42.3144130952922 -87.73930596680093) bank134140 +134141 POINT(41.235322977929584 -87.14047124664607) bank134141 +134142 POINT(42.183318949083535 -87.32860130008497) bank134142 +134143 POINT(42.85946947079224 -87.63092816513287) bank134143 +134144 POINT(40.88996402197541 -86.81400234764327) bank134144 +134145 POINT(42.582685318094946 -86.88102469295225) bank134145 +134146 POINT(42.44278667185627 -86.75809726376045) bank134146 +134147 POINT(41.63627393258081 -87.96643380269548) bank134147 +134148 POINT(41.31309714746947 -87.83465947187317) bank134148 +134149 POINT(42.71892331745894 -88.31105101135398) bank134149 +134150 POINT(42.82483669319629 -87.17765967810539) bank134150 +134151 POINT(41.647432798371796 -88.14496947992926) bank134151 +134152 POINT(42.85443150162915 -87.59327518691309) bank134152 +134153 POINT(40.941953810367096 -86.92023761842931) bank134153 +134154 POINT(42.505933635135285 -88.48421978675887) bank134154 +134155 POINT(42.5951882100467 -88.2347948478928) bank134155 +134156 POINT(41.706346153035895 -88.03301042928265) bank134156 +134157 POINT(42.55921185090102 -88.02745741236284) bank134157 +134158 POINT(41.791616864070726 -87.8877023318235) bank134158 +134159 POINT(41.89672165491769 -88.33027979722071) bank134159 +134160 POINT(42.6116932637777 -87.77225026173863) bank134160 +134161 POINT(42.20056397178487 -88.47114914148166) bank134161 +134162 POINT(41.44579532476887 -88.02510811708608) bank134162 +134163 POINT(41.2431261629672 -88.26710518840382) bank134163 +134164 POINT(42.608374989823425 -86.83538210673656) bank134164 +134165 POINT(41.75495399133969 -86.9039590845845) bank134165 +134166 POINT(41.774376480144774 -87.77547238120039) bank134166 +134167 POINT(42.792683812551694 -88.09865617181255) bank134167 +134168 POINT(41.83480045928055 -88.4996436658305) bank134168 +134169 POINT(41.51743408992934 -87.34792627942872) bank134169 +134170 POINT(42.774271988214835 -86.95447327209145) bank134170 +134171 POINT(42.72683877778243 -88.16452757854668) bank134171 +134172 POINT(42.20701915821785 -88.29849451190191) bank134172 +134173 POINT(41.97446664428728 -87.35062036894729) bank134173 +134174 POINT(42.281677993956606 -87.56040891335417) bank134174 +134175 POINT(41.0380815698647 -86.85817696047647) bank134175 +134176 POINT(41.5166675203035 -87.1312206031352) bank134176 +134177 POINT(42.69906208174381 -86.90167650329374) bank134177 +134178 POINT(41.34028876469469 -87.38393076732706) bank134178 +134179 POINT(42.52446116058348 -88.07744428252659) bank134179 +134180 POINT(41.69644764401251 -87.05257508665004) bank134180 +134181 POINT(40.93936770285534 -87.26720528671987) bank134181 +134182 POINT(42.33970831349353 -88.04518745538061) bank134182 +134183 POINT(42.47627487953866 -86.6981500056827) bank134183 +134184 POINT(42.33909883409949 -87.59855325255829) bank134184 +134185 POINT(41.14135912549657 -88.16993397779622) bank134185 +134186 POINT(42.715879386923284 -87.21438328108226) bank134186 +134187 POINT(41.04722342040138 -88.60299400991804) bank134187 +134188 POINT(41.906732777930415 -87.953811049268) bank134188 +134189 POINT(42.43381128793578 -87.42147788049789) bank134189 +134190 POINT(41.144151660527896 -86.89013947176052) bank134190 +134191 POINT(42.413483791290865 -86.84548711347641) bank134191 +134192 POINT(41.412811188007844 -88.3507823455237) bank134192 +134193 POINT(41.4322849107133 -87.70538989157981) bank134193 +134194 POINT(41.01469165910728 -87.47938054200378) bank134194 +134195 POINT(42.630849515026405 -87.97141383199664) bank134195 +134196 POINT(42.26427954880941 -87.50176157056714) bank134196 +134197 POINT(40.97890928571516 -88.52573030349551) bank134197 +134198 POINT(42.02605398129283 -87.13036237481933) bank134198 +134199 POINT(41.49154870920485 -87.07549832833762) bank134199 +134200 POINT(40.91759765922074 -88.56195758083997) bank134200 +134201 POINT(41.18296147350475 -88.25653108395419) bank134201 +134202 POINT(41.49300675561221 -87.2495192422417) bank134202 +134203 POINT(41.781020874780396 -87.3524671298637) bank134203 +134204 POINT(42.1666246362328 -88.46576553008616) bank134204 +134205 POINT(41.3664233247072 -86.76424863224173) bank134205 +134206 POINT(42.77916306303119 -86.75278443215284) bank134206 +134207 POINT(42.869582983389236 -87.52718313650915) bank134207 +134208 POINT(41.656958400534805 -86.8703266469799) bank134208 +134209 POINT(42.624241710389576 -86.96467917125901) bank134209 +134210 POINT(41.100393275023364 -87.96865378851032) bank134210 +134211 POINT(42.27027561098092 -88.59108817192089) bank134211 +134212 POINT(42.790681003530516 -87.654036300392) bank134212 +134213 POINT(41.86172392624808 -87.45631843850903) bank134213 +134214 POINT(41.231937250842876 -86.74080848919738) bank134214 +134215 POINT(40.89850937763247 -88.52069256405835) bank134215 +134216 POINT(41.17225250100556 -87.83230551863475) bank134216 +134217 POINT(41.60568874347446 -86.69713462545985) bank134217 +134218 POINT(42.83928377780119 -86.99489954693769) bank134218 +134219 POINT(40.91207342706299 -86.70218388927556) bank134219 +134220 POINT(42.47832773346316 -87.2343339614286) bank134220 +134221 POINT(42.68628536278386 -86.90660446383838) bank134221 +134222 POINT(42.1143581669084 -88.09139593350844) bank134222 +134223 POINT(41.26564998969661 -87.68443596139396) bank134223 +134224 POINT(42.759066331292104 -88.30693293354183) bank134224 +134225 POINT(41.5472152747346 -87.79122565470502) bank134225 +134226 POINT(41.74670009146403 -88.25803799632783) bank134226 +134227 POINT(41.62361382035056 -87.37673940821628) bank134227 +134228 POINT(42.77025269129681 -88.5876496588056) bank134228 +134229 POINT(42.438680660173226 -87.11855690215742) bank134229 +134230 POINT(42.559098782327126 -87.64780605266118) bank134230 +134231 POINT(42.432897662522436 -87.27415016698015) bank134231 +134232 POINT(42.356950376635766 -87.44164215113393) bank134232 +134233 POINT(41.64386027799593 -88.61308018833198) bank134233 +134234 POINT(41.32566050803465 -86.71066738494385) bank134234 +134235 POINT(42.29659023054653 -87.47168385456536) bank134235 +134236 POINT(41.11986854366414 -86.98760463319029) bank134236 +134237 POINT(42.799329315894624 -87.21200443649231) bank134237 +134238 POINT(40.87875232366792 -87.72088758643854) bank134238 +134239 POINT(41.453092820748765 -86.96442476528418) bank134239 +134240 POINT(41.517595246311046 -88.221609721039) bank134240 +134241 POINT(41.54257595824915 -87.38338861546514) bank134241 +134242 POINT(41.4203519387435 -86.90484679721526) bank134242 +134243 POINT(40.974950672333414 -88.08783346523727) bank134243 +134244 POINT(42.2684803329581 -87.75099783064384) bank134244 +134245 POINT(41.54586261004241 -87.16007682081622) bank134245 +134246 POINT(42.83634159662508 -87.68967719358706) bank134246 +134247 POINT(40.982209408142324 -88.43226172537629) bank134247 +134248 POINT(41.25010727532727 -87.12291434806404) bank134248 +134249 POINT(42.81450344690384 -87.12644453601729) bank134249 +134250 POINT(42.67196653790863 -86.79206567165737) bank134250 +134251 POINT(41.84125595265875 -87.63626414864908) bank134251 +134252 POINT(42.55882689112188 -88.4221995832104) bank134252 +134253 POINT(41.58472392263431 -88.34829365335354) bank134253 +134254 POINT(41.42933536244232 -87.0311144374843) bank134254 +134255 POINT(41.710491398844475 -88.57845186693677) bank134255 +134256 POINT(41.52863528223179 -86.63098075940952) bank134256 +134257 POINT(42.844888617466914 -87.39727168838297) bank134257 +134258 POINT(41.059265884446695 -86.67870278998237) bank134258 +134259 POINT(42.51932036761067 -87.97411068306982) bank134259 +134260 POINT(41.37715163127489 -87.40771366655314) bank134260 +134261 POINT(41.5126757139083 -88.17943850024416) bank134261 +134262 POINT(41.43612115868617 -88.41341675327558) bank134262 +134263 POINT(40.96884158294358 -88.28415190655149) bank134263 +134264 POINT(40.91217148330351 -88.25394686375597) bank134264 +134265 POINT(41.90184677420925 -87.10065371863818) bank134265 +134266 POINT(42.30730829184762 -88.18304513896868) bank134266 +134267 POINT(42.69610687928319 -88.30991051351546) bank134267 +134268 POINT(42.38480272821098 -87.04461587009588) bank134268 +134269 POINT(42.63435960784763 -87.91971507225647) bank134269 +134270 POINT(41.941637059064675 -88.16801303156302) bank134270 +134271 POINT(41.16206653792836 -88.26415011091692) bank134271 +134272 POINT(41.26293670479574 -87.72418288049197) bank134272 +134273 POINT(42.16106804176923 -86.96997475094332) bank134273 +134274 POINT(41.51709528140478 -88.53085043114957) bank134274 +134275 POINT(41.332255206943884 -86.87113927615252) bank134275 +134276 POINT(42.69908271217624 -87.61872714649554) bank134276 +134277 POINT(41.99249714517984 -86.9958662827177) bank134277 +134278 POINT(41.98059724935716 -87.07482308417606) bank134278 +134279 POINT(41.786973632124166 -87.98053162264252) bank134279 +134280 POINT(42.73998890168266 -87.92490519728162) bank134280 +134281 POINT(41.954351865288906 -88.48055397998192) bank134281 +134282 POINT(42.55283175516191 -88.22655543960295) bank134282 +134283 POINT(42.84069138975173 -86.72069498567386) bank134283 +134284 POINT(42.0157844949153 -88.40215197109815) bank134284 +134285 POINT(41.58948978777772 -86.77829204822004) bank134285 +134286 POINT(41.41746602307456 -87.65179014832097) bank134286 +134287 POINT(41.29328522747844 -88.61219678025687) bank134287 +134288 POINT(41.760807959327536 -87.32972291412133) bank134288 +134289 POINT(42.70272373375461 -87.65770642294518) bank134289 +134290 POINT(42.237129653016524 -87.12650716473732) bank134290 +134291 POINT(41.09043875245821 -86.79446104909302) bank134291 +134292 POINT(40.903567623691586 -87.68093966332754) bank134292 +134293 POINT(41.952534052079855 -88.05939135531933) bank134293 +134294 POINT(41.11504685699065 -86.9749990632387) bank134294 +134295 POINT(41.973929005049044 -87.74648564410272) bank134295 +134296 POINT(41.00162683886766 -88.17723337637194) bank134296 +134297 POINT(41.65474254441623 -87.94245560435992) bank134297 +134298 POINT(42.132607210318625 -88.14937357493241) bank134298 +134299 POINT(41.49405405593593 -88.54780391645122) bank134299 +134300 POINT(41.179803605414335 -87.56679091808644) bank134300 +134301 POINT(42.159175473664725 -88.1338879886116) bank134301 +134302 POINT(42.116516149699656 -88.03501709750917) bank134302 +134303 POINT(41.52698175643886 -88.35468623326668) bank134303 +134304 POINT(41.26022508910462 -86.90282832266088) bank134304 +134305 POINT(41.988117912096264 -87.63241851593348) bank134305 +134306 POINT(42.11025665645409 -86.83735901768135) bank134306 +134307 POINT(42.03881090078222 -87.27835812718654) bank134307 +134308 POINT(42.10434247597493 -87.63588666140348) bank134308 +134309 POINT(41.715857047839926 -88.06510951365276) bank134309 +134310 POINT(42.145232192703 -88.45310319157144) bank134310 +134311 POINT(41.85122853606012 -86.96823845269576) bank134311 +134312 POINT(41.88255635721069 -87.02311252807336) bank134312 +134313 POINT(42.71532305752134 -87.93016695061769) bank134313 +134314 POINT(41.99769242675865 -87.16073937901643) bank134314 +134315 POINT(41.9566774396736 -87.09032162953838) bank134315 +134316 POINT(41.088421616799046 -86.93556867772442) bank134316 +134317 POINT(41.23898523376684 -87.44458300783116) bank134317 +134318 POINT(42.34451075217916 -87.83900223707423) bank134318 +134319 POINT(41.77784215135871 -87.57503086445423) bank134319 +134320 POINT(40.98296654445253 -88.60628359268232) bank134320 +134321 POINT(42.28428049118223 -87.96647848799685) bank134321 +134322 POINT(41.15376446206389 -87.6246031163858) bank134322 +134323 POINT(41.626366235303074 -88.27595275674796) bank134323 +134324 POINT(40.980302250962254 -86.8884149148983) bank134324 +134325 POINT(42.75603034551719 -87.98426825053335) bank134325 +134326 POINT(41.83457117844751 -88.15206113053323) bank134326 +134327 POINT(41.188467741045585 -87.44152527139697) bank134327 +134328 POINT(41.31622025248126 -88.17919213437135) bank134328 +134329 POINT(40.899219832239815 -87.31659087965959) bank134329 +134330 POINT(41.87791291681614 -87.56627649781993) bank134330 +134331 POINT(42.25636475775265 -87.39775005127554) bank134331 +134332 POINT(40.991147482287296 -87.41495437488071) bank134332 +134333 POINT(41.759858437630015 -86.91051131923867) bank134333 +134334 POINT(42.58000686941478 -88.39955146545155) bank134334 +134335 POINT(42.253827267614476 -87.6129735572201) bank134335 +134336 POINT(42.013149772533936 -87.67146239022293) bank134336 +134337 POINT(40.88499567249613 -87.38166519309682) bank134337 +134338 POINT(42.81441379454298 -87.66147015185508) bank134338 +134339 POINT(41.484110905422604 -87.11482915946759) bank134339 +134340 POINT(41.055503122143776 -87.92436295646624) bank134340 +134341 POINT(41.20797627609412 -86.93634872901255) bank134341 +134342 POINT(41.9697405177064 -88.41768318922139) bank134342 +134343 POINT(41.971104233265784 -88.06721565654631) bank134343 +134344 POINT(41.180666330839486 -88.25622497010576) bank134344 +134345 POINT(42.0003967325132 -87.95115853778917) bank134345 +134346 POINT(41.1447688203209 -86.72500359888681) bank134346 +134347 POINT(41.28760082155654 -88.34375084218806) bank134347 +134348 POINT(42.773167476131604 -87.45811660267863) bank134348 +134349 POINT(42.858139508807845 -88.44620311630592) bank134349 +134350 POINT(41.374452387144046 -86.8258225535853) bank134350 +134351 POINT(42.015747110972406 -88.16469510650866) bank134351 +134352 POINT(42.454753173578396 -86.70866510867144) bank134352 +134353 POINT(42.206560275368744 -86.8348762276457) bank134353 +134354 POINT(42.198623546589495 -88.15758540538003) bank134354 +134355 POINT(41.65159315125206 -87.56694832478003) bank134355 +134356 POINT(42.646263042241934 -87.41161137048223) bank134356 +134357 POINT(42.70516679110622 -86.70132328033611) bank134357 +134358 POINT(41.08070221001391 -87.45727559572083) bank134358 +134359 POINT(41.21144721656618 -87.25497034444979) bank134359 +134360 POINT(42.05358677654098 -86.75742257605744) bank134360 +134361 POINT(41.50075028790521 -87.76873239840994) bank134361 +134362 POINT(42.4384873822971 -87.87350379259597) bank134362 +134363 POINT(41.293982228446076 -88.49140433375439) bank134363 +134364 POINT(41.77299722657846 -88.36304985493884) bank134364 +134365 POINT(42.37311863885958 -87.5849726438204) bank134365 +134366 POINT(41.766358340742215 -88.23239986127851) bank134366 +134367 POINT(40.93354385415006 -87.89283727689093) bank134367 +134368 POINT(41.34007168641906 -86.91420445394732) bank134368 +134369 POINT(42.18015796151515 -88.34532018055431) bank134369 +134370 POINT(42.51202432072869 -87.32107529450055) bank134370 +134371 POINT(42.22962934908215 -86.88531885559115) bank134371 +134372 POINT(40.95971369296887 -88.28771280260676) bank134372 +134373 POINT(41.01240766762353 -88.40576558599135) bank134373 +134374 POINT(41.33772974937026 -88.3485697365011) bank134374 +134375 POINT(41.80206592892668 -87.14101303636821) bank134375 +134376 POINT(41.62150797580739 -88.23126513763002) bank134376 +134377 POINT(41.37184931000591 -87.28663917572423) bank134377 +134378 POINT(42.38876015366403 -88.0747462272377) bank134378 +134379 POINT(40.884868107971414 -87.82242448107306) bank134379 +134380 POINT(42.781357809393754 -86.71465927052951) bank134380 +134381 POINT(41.41305920587507 -87.56387033586499) bank134381 +134382 POINT(42.65919924984166 -87.11071201444622) bank134382 +134383 POINT(42.613820229967075 -87.12329175245026) bank134383 +134384 POINT(40.96833722235306 -87.73466291451616) bank134384 +134385 POINT(41.799390227498556 -87.46516559735598) bank134385 +134386 POINT(42.588086791959945 -87.22826907976717) bank134386 +134387 POINT(41.85409734382478 -88.37714497479743) bank134387 +134388 POINT(41.393352866814816 -88.18687856365771) bank134388 +134389 POINT(42.845559270860356 -87.77875963994015) bank134389 +134390 POINT(41.15785135778762 -87.93674970597623) bank134390 +134391 POINT(41.88397485367712 -88.45697107945607) bank134391 +134392 POINT(42.44399722701837 -88.13551954949241) bank134392 +134393 POINT(41.48278510762732 -86.9529433198415) bank134393 +134394 POINT(42.26095900868445 -87.53047136065868) bank134394 +134395 POINT(41.164076305131914 -86.70325314737642) bank134395 +134396 POINT(41.017179283648744 -86.85491040615663) bank134396 +134397 POINT(41.437494006778216 -87.23098955982975) bank134397 +134398 POINT(41.259678402560745 -87.5399478972933) bank134398 +134399 POINT(41.71808750635657 -87.77245344553138) bank134399 +134400 POINT(40.96984926941217 -88.22191637228563) bank134400 +134401 POINT(41.770367923224995 -88.05466674295153) bank134401 +134402 POINT(41.340700504378326 -88.40991480501877) bank134402 +134403 POINT(42.48877636114751 -88.07025673863792) bank134403 +134404 POINT(41.4997810411347 -88.55826128697657) bank134404 +134405 POINT(41.690865430443914 -86.63258713078947) bank134405 +134406 POINT(41.66672533629383 -87.18867943829254) bank134406 +134407 POINT(42.3458491344601 -87.16590374138285) bank134407 +134408 POINT(42.19503016342657 -87.26911983254227) bank134408 +134409 POINT(41.403638979879915 -87.51053074141849) bank134409 +134410 POINT(42.20682491796576 -87.45395512558767) bank134410 +134411 POINT(41.791631621580265 -88.00660613791489) bank134411 +134412 POINT(42.275618860397024 -87.90060859005622) bank134412 +134413 POINT(41.58992949249886 -87.21013319487741) bank134413 +134414 POINT(41.35066696579369 -88.08655852707561) bank134414 +134415 POINT(42.14140925027006 -86.95374361967778) bank134415 +134416 POINT(41.39726924970443 -87.07105977871765) bank134416 +134417 POINT(41.06479839760521 -87.75288946388474) bank134417 +134418 POINT(40.971448795903335 -88.29241140542591) bank134418 +134419 POINT(40.879856598550454 -87.25737665020236) bank134419 +134420 POINT(42.748245038014964 -87.38253211104585) bank134420 +134421 POINT(42.54873216108084 -86.88883347330916) bank134421 +134422 POINT(41.73275884084379 -87.9123859048937) bank134422 +134423 POINT(41.17309805021904 -87.07504267124413) bank134423 +134424 POINT(41.350336948891666 -86.98910120215425) bank134424 +134425 POINT(42.51830468618213 -86.81314568863101) bank134425 +134426 POINT(41.39360231962991 -86.73077053586566) bank134426 +134427 POINT(41.07875656847814 -87.33229964616729) bank134427 +134428 POINT(41.15876399139225 -86.72915204508236) bank134428 +134429 POINT(41.01222622529315 -88.57991624654096) bank134429 +134430 POINT(41.9321396608633 -88.33980722999246) bank134430 +134431 POINT(41.96946817100896 -86.94746329890921) bank134431 +134432 POINT(42.37181414648466 -87.98388149308276) bank134432 +134433 POINT(41.300104893343494 -88.3967501261199) bank134433 +134434 POINT(41.40497532234496 -87.7265929141059) bank134434 +134435 POINT(41.80708349048238 -87.19659343433545) bank134435 +134436 POINT(42.13543291717189 -87.77309605192687) bank134436 +134437 POINT(42.84634736595198 -87.45296940280022) bank134437 +134438 POINT(42.485309222799216 -87.54800698212871) bank134438 +134439 POINT(42.69619588223503 -86.90197959561392) bank134439 +134440 POINT(41.29767766391942 -87.24279040670142) bank134440 +134441 POINT(41.16310286128303 -86.78986965001968) bank134441 +134442 POINT(41.638292032035785 -88.53959981071478) bank134442 +134443 POINT(42.47800499981628 -87.09363368909332) bank134443 +134444 POINT(42.602898535731754 -88.11692877929532) bank134444 +134445 POINT(42.75288761832502 -88.23657668502676) bank134445 +134446 POINT(42.850166931292335 -87.4493242027025) bank134446 +134447 POINT(41.742592150226024 -87.51594356204437) bank134447 +134448 POINT(41.26247059585661 -87.91287175712053) bank134448 +134449 POINT(42.372738618630265 -88.54774482607594) bank134449 +134450 POINT(42.85887205878926 -87.23542628056157) bank134450 +134451 POINT(40.890563083378105 -87.3840884986923) bank134451 +134452 POINT(42.183671042418844 -87.07331342185302) bank134452 +134453 POINT(41.471910146090906 -88.13868997383987) bank134453 +134454 POINT(41.97463486222237 -88.42056062261388) bank134454 +134455 POINT(42.41719608694937 -87.35408561849607) bank134455 +134456 POINT(42.41650220878311 -88.09114159413551) bank134456 +134457 POINT(40.92088320194665 -88.15287044982556) bank134457 +134458 POINT(42.407183914414865 -86.63990812599043) bank134458 +134459 POINT(41.6524265965696 -87.1243908436956) bank134459 +134460 POINT(41.93475842618963 -87.67720045717557) bank134460 +134461 POINT(41.644705555183926 -87.49097768722102) bank134461 +134462 POINT(41.562013356540994 -88.31328547635565) bank134462 +134463 POINT(41.6697255530649 -87.51270877120037) bank134463 +134464 POINT(41.942416897565586 -87.89115088621011) bank134464 +134465 POINT(42.6817826390847 -86.89798960954683) bank134465 +134466 POINT(41.18904567886195 -87.62212178793291) bank134466 +134467 POINT(42.25373749977757 -87.79288656943142) bank134467 +134468 POINT(41.846534873733034 -86.91732592766095) bank134468 +134469 POINT(41.412792789395745 -88.13285603338227) bank134469 +134470 POINT(41.95748397325424 -87.7687426199796) bank134470 +134471 POINT(42.591597782215594 -88.43529241553543) bank134471 +134472 POINT(42.31181777885916 -87.00352062167698) bank134472 +134473 POINT(41.92026354545999 -87.2365554191884) bank134473 +134474 POINT(42.171709552613926 -87.09091306932889) bank134474 +134475 POINT(41.80851484116695 -86.7288613082825) bank134475 +134476 POINT(42.653390078614564 -87.1377815222894) bank134476 +134477 POINT(42.40441470857709 -87.29540096974773) bank134477 +134478 POINT(41.44040618929248 -88.12925337144974) bank134478 +134479 POINT(42.349268937363384 -87.68753467919214) bank134479 +134480 POINT(41.22674600611177 -86.70234671195284) bank134480 +134481 POINT(41.387624678940824 -88.04121250524913) bank134481 +134482 POINT(42.30111633767653 -88.26623058462953) bank134482 +134483 POINT(41.04993098478834 -88.02725776782195) bank134483 +134484 POINT(42.206279506242666 -86.6659604942053) bank134484 +134485 POINT(41.80125166749995 -88.18905370370129) bank134485 +134486 POINT(41.77913854146879 -87.85087972180794) bank134486 +134487 POINT(42.712609504812306 -88.12963934840427) bank134487 +134488 POINT(41.345555108085804 -87.3902067595567) bank134488 +134489 POINT(41.76901185887101 -88.18359805694521) bank134489 +134490 POINT(42.62207758491898 -87.58481965066376) bank134490 +134491 POINT(42.394595682977396 -87.83937100307304) bank134491 +134492 POINT(41.65980475183762 -87.58702700231252) bank134492 +134493 POINT(42.126691185964596 -86.8115075425709) bank134493 +134494 POINT(41.53923654672746 -87.94655994654164) bank134494 +134495 POINT(41.2319769120684 -88.02952051977861) bank134495 +134496 POINT(41.69204613888687 -87.5808856465408) bank134496 +134497 POINT(41.51296838561221 -87.77647790605894) bank134497 +134498 POINT(42.49528421691275 -88.33634436006426) bank134498 +134499 POINT(42.652176168081496 -87.06708367584524) bank134499 +134500 POINT(41.37291419640852 -87.08803241626237) bank134500 +134501 POINT(41.444558092715035 -87.34757225056686) bank134501 +134502 POINT(42.07709021221868 -87.36309320821537) bank134502 +134503 POINT(42.14515529951594 -87.67010695866176) bank134503 +134504 POINT(41.153382802884344 -86.86314653750226) bank134504 +134505 POINT(42.85223261835439 -86.99051222582183) bank134505 +134506 POINT(41.521200744740355 -87.05659703752623) bank134506 +134507 POINT(42.631159977949196 -86.68808746097629) bank134507 +134508 POINT(41.54451489916024 -87.5268893212362) bank134508 +134509 POINT(41.97806314712839 -87.37588964899528) bank134509 +134510 POINT(42.18234198460408 -87.29577495109035) bank134510 +134511 POINT(42.76232116394958 -86.92423606152255) bank134511 +134512 POINT(41.45182149896983 -87.59415784058879) bank134512 +134513 POINT(42.5379273015638 -87.7407480086117) bank134513 +134514 POINT(41.19420980761373 -88.50512405339677) bank134514 +134515 POINT(42.59202641659694 -86.89032612190047) bank134515 +134516 POINT(42.41113709660382 -86.86899148354917) bank134516 +134517 POINT(41.03189444819591 -87.06730414823176) bank134517 +134518 POINT(41.67116721572283 -87.93575045246878) bank134518 +134519 POINT(41.004996275938126 -87.76699632362808) bank134519 +134520 POINT(42.602105947483494 -87.79513973875538) bank134520 +134521 POINT(42.03405588435408 -86.87579481251456) bank134521 +134522 POINT(41.89230911068555 -87.16383296582609) bank134522 +134523 POINT(42.29973128487326 -88.12435739957817) bank134523 +134524 POINT(42.6727209469664 -88.37423735170428) bank134524 +134525 POINT(42.484730662359226 -87.56722165939826) bank134525 +134526 POINT(42.72800250508768 -87.53634237157512) bank134526 +134527 POINT(41.83125041567624 -87.59005606428076) bank134527 +134528 POINT(41.98989563288332 -88.13744290443077) bank134528 +134529 POINT(41.01730749837503 -88.2725258786131) bank134529 +134530 POINT(41.28738469157515 -88.04973149037572) bank134530 +134531 POINT(41.16722570914691 -87.91011075413688) bank134531 +134532 POINT(42.047403951232376 -87.83542906352277) bank134532 +134533 POINT(42.450476428674385 -87.9814927303862) bank134533 +134534 POINT(41.750203715332475 -87.82503906933455) bank134534 +134535 POINT(41.96026267911477 -87.02463124737008) bank134535 +134536 POINT(42.1670144508602 -88.42883282783936) bank134536 +134537 POINT(41.002555057902235 -86.64617433683351) bank134537 +134538 POINT(41.34133918846496 -86.77499961252651) bank134538 +134539 POINT(42.4522857163329 -88.3631539647339) bank134539 +134540 POINT(42.15808271163638 -87.93633770907451) bank134540 +134541 POINT(42.33168670221564 -86.85118027333682) bank134541 +134542 POINT(41.90997652077612 -87.22604936796021) bank134542 +134543 POINT(40.976943563715636 -87.87222184610194) bank134543 +134544 POINT(40.988476006334565 -87.98338869087404) bank134544 +134545 POINT(42.20456025015865 -87.78684918673213) bank134545 +134546 POINT(42.38356569544133 -87.3963285427053) bank134546 +134547 POINT(41.035718776938054 -87.66397542321059) bank134547 +134548 POINT(41.14835145205927 -87.288560743274) bank134548 +134549 POINT(41.508149639286174 -88.43718979943745) bank134549 +134550 POINT(41.31000432247087 -88.26631619727411) bank134550 +134551 POINT(41.5551206517546 -87.09647784309234) bank134551 +134552 POINT(42.83817878040049 -88.36115779244507) bank134552 +134553 POINT(41.17389476324689 -86.89439765197208) bank134553 +134554 POINT(42.71001070970605 -86.79163601581547) bank134554 +134555 POINT(41.560690517693246 -87.8360071889747) bank134555 +134556 POINT(41.363393225270386 -88.26055675396879) bank134556 +134557 POINT(41.46836691101806 -88.3919031121568) bank134557 +134558 POINT(41.15928760411261 -87.856104055714) bank134558 +134559 POINT(41.848072362354145 -88.42669978585789) bank134559 +134560 POINT(42.84043126883818 -87.88094443098319) bank134560 +134561 POINT(41.383267951727085 -87.41150162546354) bank134561 +134562 POINT(41.03272210853251 -88.23660110344477) bank134562 +134563 POINT(41.62307556739411 -87.34799584211225) bank134563 +134564 POINT(41.97232734635607 -88.00755310683506) bank134564 +134565 POINT(42.35977093138618 -88.61737438393645) bank134565 +134566 POINT(41.34010957335742 -87.49685916006253) bank134566 +134567 POINT(42.7924141309562 -88.35317819994356) bank134567 +134568 POINT(42.19511623839569 -87.06948324910607) bank134568 +134569 POINT(42.63983489498035 -88.35217764480672) bank134569 +134570 POINT(41.59181440001852 -87.0457631837866) bank134570 +134571 POINT(41.12256772581429 -86.7527505184071) bank134571 +134572 POINT(41.25266919176063 -87.69350169441364) bank134572 +134573 POINT(41.634074528204536 -87.25903255174043) bank134573 +134574 POINT(42.864569575714434 -88.36096447112186) bank134574 +134575 POINT(41.23306708642102 -87.98521050005543) bank134575 +134576 POINT(42.09461037936945 -87.45503067117839) bank134576 +134577 POINT(42.697170961736305 -87.56750376948212) bank134577 +134578 POINT(41.13614928334598 -88.38691088736914) bank134578 +134579 POINT(42.76206335353175 -87.037221761717) bank134579 +134580 POINT(42.316785811507394 -88.37252728810255) bank134580 +134581 POINT(42.30319783007304 -88.43662370114643) bank134581 +134582 POINT(41.770457598613596 -87.26196681734497) bank134582 +134583 POINT(41.08035271946028 -87.7445873034937) bank134583 +134584 POINT(42.526415667523864 -87.5996947963605) bank134584 +134585 POINT(41.89825684123873 -86.85614904317946) bank134585 +134586 POINT(41.60930147736491 -87.23152361913733) bank134586 +134587 POINT(41.22256883149393 -86.97549975373333) bank134587 +134588 POINT(41.864595221382714 -87.62689255842598) bank134588 +134589 POINT(41.65117708462378 -87.97456053739405) bank134589 +134590 POINT(42.51139066886387 -87.84944739844502) bank134590 +134591 POINT(41.315415558025265 -87.050208925981) bank134591 +134592 POINT(42.698775343411896 -88.14605797252048) bank134592 +134593 POINT(42.19616186069961 -87.41258518017715) bank134593 +134594 POINT(41.40391916668099 -87.94682992485187) bank134594 +134595 POINT(41.128808935167285 -87.83749769873715) bank134595 +134596 POINT(42.80249907090713 -87.61643143865676) bank134596 +134597 POINT(41.65773602359527 -86.89145202132579) bank134597 +134598 POINT(42.121462915249666 -88.11922600080214) bank134598 +134599 POINT(41.87253674995125 -87.99983195710844) bank134599 +134600 POINT(41.58201254253265 -87.75829376850231) bank134600 +134601 POINT(41.21126780654333 -87.34084395253504) bank134601 +134602 POINT(40.94358705815396 -88.62468045939887) bank134602 +134603 POINT(41.58053535801452 -87.98508083215751) bank134603 +134604 POINT(40.9246504294901 -87.99933156965099) bank134604 +134605 POINT(40.92341949716396 -87.47341075768448) bank134605 +134606 POINT(41.564480081410146 -87.06811627603709) bank134606 +134607 POINT(42.298756777324456 -87.32574296630797) bank134607 +134608 POINT(41.45493575606765 -87.4575768501063) bank134608 +134609 POINT(41.90268615255031 -88.56073013636941) bank134609 +134610 POINT(42.21057832732497 -88.16711016628413) bank134610 +134611 POINT(42.53263025690459 -88.10372663358301) bank134611 +134612 POINT(41.715846651582595 -87.14484632728801) bank134612 +134613 POINT(41.66646530094046 -86.94423157335954) bank134613 +134614 POINT(41.87345802621097 -88.28497942531637) bank134614 +134615 POINT(41.54990673279262 -87.96199274073241) bank134615 +134616 POINT(41.59256409957722 -87.54431223584525) bank134616 +134617 POINT(40.94773986357056 -88.27724658265825) bank134617 +134618 POINT(41.170007026933135 -88.19483104870858) bank134618 +134619 POINT(41.04260754589194 -87.11442279571848) bank134619 +134620 POINT(42.52612849706078 -88.51134492922021) bank134620 +134621 POINT(41.90437203504553 -87.35745441472257) bank134621 +134622 POINT(42.65837110431268 -88.04905089475035) bank134622 +134623 POINT(41.78554065206403 -86.89593877496854) bank134623 +134624 POINT(42.65411304287642 -86.6483173291247) bank134624 +134625 POINT(42.84724223144562 -88.22582118582362) bank134625 +134626 POINT(41.83041990057936 -87.59600112366566) bank134626 +134627 POINT(42.029479450031786 -87.47737038175369) bank134627 +134628 POINT(41.52483969817301 -87.39774685355553) bank134628 +134629 POINT(41.93948827071471 -86.89958021709096) bank134629 +134630 POINT(42.64151481317885 -86.94864583056639) bank134630 +134631 POINT(41.60473291255241 -87.58584631052969) bank134631 +134632 POINT(40.89594180839844 -86.78125248065122) bank134632 +134633 POINT(42.323222916498615 -87.04030575914028) bank134633 +134634 POINT(42.73328761834006 -87.12627234928414) bank134634 +134635 POINT(42.86371348604899 -87.23570382583613) bank134635 +134636 POINT(42.250365537929376 -86.82388154561777) bank134636 +134637 POINT(41.287149725372906 -88.2391631541092) bank134637 +134638 POINT(41.72061272006728 -86.8899693994488) bank134638 +134639 POINT(41.64158797291431 -87.75142259379876) bank134639 +134640 POINT(42.773511494561674 -87.32809688068848) bank134640 +134641 POINT(42.42014581136813 -86.73981080878427) bank134641 +134642 POINT(42.040798464029315 -88.06084019101425) bank134642 +134643 POINT(42.580169565102715 -87.76199455195254) bank134643 +134644 POINT(41.447282913675295 -87.83700528063818) bank134644 +134645 POINT(41.615304126637746 -87.08637285259454) bank134645 +134646 POINT(41.389080875270146 -88.3809918333679) bank134646 +134647 POINT(41.16510890675009 -87.09089027925175) bank134647 +134648 POINT(42.61066356236156 -88.06584016057325) bank134648 +134649 POINT(42.09836086643054 -87.79096962128118) bank134649 +134650 POINT(42.73293534464838 -87.44916117015892) bank134650 +134651 POINT(42.774205579531724 -88.16170084424563) bank134651 +134652 POINT(42.18932574264044 -88.09197135835383) bank134652 +134653 POINT(42.871012400828505 -87.26432718401315) bank134653 +134654 POINT(42.263425894642964 -88.40499558868648) bank134654 +134655 POINT(42.34748122038817 -87.93089004305197) bank134655 +134656 POINT(41.623221990246925 -87.9738586204745) bank134656 +134657 POINT(41.22806469555736 -88.01488390509932) bank134657 +134658 POINT(42.15177896426 -86.79421197820396) bank134658 +134659 POINT(42.12371866923493 -88.30969761709854) bank134659 +134660 POINT(42.49541201260551 -87.46790175049912) bank134660 +134661 POINT(42.56142828129999 -87.32112504441531) bank134661 +134662 POINT(42.16135812204119 -87.39472882809166) bank134662 +134663 POINT(41.27301569452706 -86.65269739941472) bank134663 +134664 POINT(42.02591848173499 -88.16363191637635) bank134664 +134665 POINT(40.98492968139585 -87.34437257552483) bank134665 +134666 POINT(41.95276167435945 -87.94791597508464) bank134666 +134667 POINT(40.98008089947772 -87.5685048108368) bank134667 +134668 POINT(41.881612633003094 -87.67897466795185) bank134668 +134669 POINT(40.99115258319796 -88.35413753164218) bank134669 +134670 POINT(42.55074444992038 -87.30978847634907) bank134670 +134671 POINT(41.90909169511387 -87.9835403797319) bank134671 +134672 POINT(42.53698425987841 -87.75636402010198) bank134672 +134673 POINT(42.26667297599917 -87.03983922191097) bank134673 +134674 POINT(41.66670345996842 -86.67733034579233) bank134674 +134675 POINT(42.49219926345181 -87.35752221866606) bank134675 +134676 POINT(41.428039051932274 -88.3291217330825) bank134676 +134677 POINT(40.89826955281065 -86.95300573910077) bank134677 +134678 POINT(41.27879111275044 -87.17629076655875) bank134678 +134679 POINT(41.38003300840042 -86.6467617959887) bank134679 +134680 POINT(42.63130610445194 -86.93732333622471) bank134680 +134681 POINT(41.91374791515512 -86.79788432965205) bank134681 +134682 POINT(42.51685273660184 -86.782923179486) bank134682 +134683 POINT(42.09486710250977 -88.21258985100056) bank134683 +134684 POINT(42.46986667591295 -87.48177010810865) bank134684 +134685 POINT(41.365139085844525 -88.2682913077144) bank134685 +134686 POINT(41.30622500591196 -86.63816045066922) bank134686 +134687 POINT(42.36938410340451 -86.88351494784915) bank134687 +134688 POINT(40.96148726602192 -88.05460896530747) bank134688 +134689 POINT(42.13784003253424 -86.68168714278963) bank134689 +134690 POINT(42.28407848378648 -88.44400840604092) bank134690 +134691 POINT(42.11709876634995 -88.04533681126476) bank134691 +134692 POINT(41.86767530079088 -88.17397773552786) bank134692 +134693 POINT(42.581072848640105 -88.07470308284223) bank134693 +134694 POINT(41.96670546892727 -88.28944115261291) bank134694 +134695 POINT(42.566545294319546 -86.74648822112762) bank134695 +134696 POINT(40.93930009269885 -88.03911665721724) bank134696 +134697 POINT(42.18600498624003 -86.80742868693386) bank134697 +134698 POINT(42.392020482594006 -87.3464466314914) bank134698 +134699 POINT(40.98269053491573 -88.18577758076309) bank134699 +134700 POINT(40.89206000477564 -87.90391144902568) bank134700 +134701 POINT(42.735955559177135 -87.81162905369135) bank134701 +134702 POINT(41.12906211258564 -88.27789350715314) bank134702 +134703 POINT(41.52455384914648 -88.4664216167949) bank134703 +134704 POINT(41.41877000031709 -87.61051988999178) bank134704 +134705 POINT(42.39975431607409 -87.20884442620356) bank134705 +134706 POINT(40.89649104142133 -86.92818587618656) bank134706 +134707 POINT(41.55334291909501 -87.32573334223356) bank134707 +134708 POINT(41.649331203941145 -87.49148564465854) bank134708 +134709 POINT(42.168752917454555 -88.53214624000934) bank134709 +134710 POINT(41.66008930428859 -87.55926461272081) bank134710 +134711 POINT(42.27571568496563 -88.26550603773151) bank134711 +134712 POINT(41.770024298314084 -87.58360770783123) bank134712 +134713 POINT(42.456911689091626 -88.58335103925921) bank134713 +134714 POINT(42.709387923949535 -87.43322307679739) bank134714 +134715 POINT(42.49847215469667 -87.68024671516139) bank134715 +134716 POINT(41.85236624657559 -87.58571929903415) bank134716 +134717 POINT(41.691752262264934 -86.99304842032107) bank134717 +134718 POINT(42.54344905463506 -87.67836492285873) bank134718 +134719 POINT(42.05840516088504 -86.6506451095573) bank134719 +134720 POINT(41.28212328016475 -87.47269006309192) bank134720 +134721 POINT(42.41666014551244 -88.3516943112526) bank134721 +134722 POINT(42.86208337060005 -87.49718350211027) bank134722 +134723 POINT(41.595559289454556 -87.8801611695237) bank134723 +134724 POINT(41.61589771132638 -88.07578283395797) bank134724 +134725 POINT(42.81915767256662 -87.5057559836733) bank134725 +134726 POINT(42.069302897132 -87.89219054242518) bank134726 +134727 POINT(42.19814286063955 -86.93125717580556) bank134727 +134728 POINT(42.20443440430842 -88.4301953567427) bank134728 +134729 POINT(40.95416777293933 -87.64288595737123) bank134729 +134730 POINT(40.96418494816778 -88.49038428926613) bank134730 +134731 POINT(42.22073381050558 -87.35703471760573) bank134731 +134732 POINT(41.05498791477063 -87.72251934816987) bank134732 +134733 POINT(41.690551961976794 -88.55396602363297) bank134733 +134734 POINT(41.03103456527036 -86.7273424537794) bank134734 +134735 POINT(41.77421919279888 -87.08450509051727) bank134735 +134736 POINT(42.321958416227694 -87.11504051108436) bank134736 +134737 POINT(41.08467439774228 -88.01255517182717) bank134737 +134738 POINT(42.47647123820553 -87.64019158159776) bank134738 +134739 POINT(42.099602878435384 -88.04323789874728) bank134739 +134740 POINT(41.47081104541318 -87.47542232260994) bank134740 +134741 POINT(42.0997456308838 -88.59338798336749) bank134741 +134742 POINT(42.83126974071683 -86.95544045856978) bank134742 +134743 POINT(42.1292638419483 -88.24950645475757) bank134743 +134744 POINT(41.536027673261295 -88.00619434038433) bank134744 +134745 POINT(42.64798111944945 -86.98514117936885) bank134745 +134746 POINT(41.287244725458734 -88.24842783535927) bank134746 +134747 POINT(42.289951486636625 -87.05741077274347) bank134747 +134748 POINT(41.18606439219092 -86.860126641453) bank134748 +134749 POINT(41.303776055649074 -86.92634798179013) bank134749 +134750 POINT(41.3660071157861 -88.09122659278259) bank134750 +134751 POINT(41.26201037296626 -88.57039503349324) bank134751 +134752 POINT(41.28522761142576 -87.22600047128444) bank134752 +134753 POINT(42.09883349465935 -87.91326914223299) bank134753 +134754 POINT(41.951626172866 -86.77936280746688) bank134754 +134755 POINT(41.25643608691895 -88.35677513052045) bank134755 +134756 POINT(42.36724051358702 -87.38137246286315) bank134756 +134757 POINT(41.127504647487726 -88.61563980329197) bank134757 +134758 POINT(42.5766178184394 -87.87224527754259) bank134758 +134759 POINT(42.8141191735026 -88.04871338025187) bank134759 +134760 POINT(41.923243539005014 -87.53215106774962) bank134760 +134761 POINT(42.573767018930894 -87.94391192168325) bank134761 +134762 POINT(41.63323350166584 -88.45745561276074) bank134762 +134763 POINT(42.78952071195937 -86.81356586764448) bank134763 +134764 POINT(42.71316943291566 -88.55428206667726) bank134764 +134765 POINT(42.2275160834105 -87.76807361443633) bank134765 +134766 POINT(41.35390200225292 -88.42978804164609) bank134766 +134767 POINT(40.99413690480644 -87.83239181431732) bank134767 +134768 POINT(40.978834164844244 -88.54493407697065) bank134768 +134769 POINT(42.69272124558749 -87.06301956380148) bank134769 +134770 POINT(41.09368639196568 -88.13015312415466) bank134770 +134771 POINT(41.15200644093493 -88.02638525984186) bank134771 +134772 POINT(41.580341771497494 -88.34768486133673) bank134772 +134773 POINT(41.098685231883856 -87.90470542470051) bank134773 +134774 POINT(42.66142969976468 -87.63889202747943) bank134774 +134775 POINT(42.19908413624571 -88.33660942868649) bank134775 +134776 POINT(41.071052508388576 -87.39135189323046) bank134776 +134777 POINT(42.638080491640096 -88.20912805068836) bank134777 +134778 POINT(41.519259802094034 -87.75703791740054) bank134778 +134779 POINT(41.20084275677819 -87.42695109373273) bank134779 +134780 POINT(41.06015009618708 -88.03367434618379) bank134780 +134781 POINT(41.946189377574655 -87.1183018145738) bank134781 +134782 POINT(42.49644930585305 -87.91207286677877) bank134782 +134783 POINT(42.00302184144337 -87.7342450859917) bank134783 +134784 POINT(40.94602888468535 -87.63443058748825) bank134784 +134785 POINT(42.6239783374021 -86.66212675858) bank134785 +134786 POINT(41.11777880458532 -87.94986693186493) bank134786 +134787 POINT(41.57924466053839 -87.50060669954578) bank134787 +134788 POINT(42.46949383798813 -87.89353667962) bank134788 +134789 POINT(42.734369875008525 -87.33572738655701) bank134789 +134790 POINT(41.96324786438221 -88.48010742056071) bank134790 +134791 POINT(42.33946946040079 -87.55155341367333) bank134791 +134792 POINT(41.17694656244348 -87.92065398286991) bank134792 +134793 POINT(41.88341381732546 -87.26236031498749) bank134793 +134794 POINT(42.215870583888 -87.23141103277759) bank134794 +134795 POINT(42.164552939124206 -87.31139966297201) bank134795 +134796 POINT(41.77175544598911 -87.83689177021333) bank134796 +134797 POINT(41.95852499825406 -88.34500391192218) bank134797 +134798 POINT(40.97357985172117 -88.5724540519064) bank134798 +134799 POINT(41.650375487402 -87.10651233500484) bank134799 +134800 POINT(42.324441270020635 -88.24288861327277) bank134800 +134801 POINT(41.57794387032202 -88.31241541411072) bank134801 +134802 POINT(42.10721085819238 -87.56391250886982) bank134802 +134803 POINT(40.99235850213552 -88.180060375397) bank134803 +134804 POINT(41.87831610428691 -88.463499945603) bank134804 +134805 POINT(41.622783739986 -87.66479928610282) bank134805 +134806 POINT(42.00331520479605 -86.9834312308294) bank134806 +134807 POINT(42.66168260043125 -86.63272547554311) bank134807 +134808 POINT(41.33507807687512 -87.67201164502264) bank134808 +134809 POINT(42.374848497250156 -87.18792248945238) bank134809 +134810 POINT(42.025128445869235 -87.15580022293085) bank134810 +134811 POINT(41.782201391028124 -87.35947643715969) bank134811 +134812 POINT(42.81724990147505 -88.06684054523812) bank134812 +134813 POINT(41.65042204284909 -87.96820671472261) bank134813 +134814 POINT(41.255317187601534 -88.17897451018825) bank134814 +134815 POINT(42.02900698662073 -87.38109161902143) bank134815 +134816 POINT(41.188064220351734 -87.90526718303138) bank134816 +134817 POINT(40.98237124073398 -86.73159618319526) bank134817 +134818 POINT(41.56782390233361 -87.59699889025497) bank134818 +134819 POINT(42.003789198718835 -87.9138284730044) bank134819 +134820 POINT(41.16383100800001 -86.79030599829586) bank134820 +134821 POINT(41.1619459904264 -86.89265951378275) bank134821 +134822 POINT(42.389958747185645 -88.24758282557933) bank134822 +134823 POINT(41.62286841571723 -86.89926437606742) bank134823 +134824 POINT(41.90239053916875 -87.20849522369984) bank134824 +134825 POINT(42.15277495533148 -88.15713548129521) bank134825 +134826 POINT(41.180952378633314 -86.66316574919063) bank134826 +134827 POINT(42.08533390784743 -87.30241948603783) bank134827 +134828 POINT(42.058862797320856 -88.52412939690035) bank134828 +134829 POINT(42.715454923621635 -88.60539800125707) bank134829 +134830 POINT(42.66362423619698 -87.0829347928045) bank134830 +134831 POINT(41.363816540881785 -87.57388445183605) bank134831 +134832 POINT(42.23452673230867 -88.03169190057847) bank134832 +134833 POINT(41.736676647811976 -87.24897821637963) bank134833 +134834 POINT(42.36989447345143 -87.08997177821486) bank134834 +134835 POINT(41.42654494403368 -86.83925301061126) bank134835 +134836 POINT(42.07610822181496 -88.11901578291133) bank134836 +134837 POINT(42.675013105417364 -86.73944243574215) bank134837 +134838 POINT(42.59857906173935 -87.9992602676191) bank134838 +134839 POINT(41.85208625684736 -88.44295795838968) bank134839 +134840 POINT(42.468438246003295 -87.95585850850253) bank134840 +134841 POINT(41.31736819043914 -87.21944402698921) bank134841 +134842 POINT(41.16397414684293 -87.2470514228674) bank134842 +134843 POINT(41.85238330375004 -88.12916832045167) bank134843 +134844 POINT(41.929149842186476 -87.20879212456508) bank134844 +134845 POINT(40.94810358004769 -88.41364251538552) bank134845 +134846 POINT(41.052578066974505 -87.03678794097755) bank134846 +134847 POINT(41.22149777474658 -88.0024930659323) bank134847 +134848 POINT(41.49906003823486 -88.14802429637618) bank134848 +134849 POINT(41.44488828359367 -87.99798165009929) bank134849 +134850 POINT(41.80969551404167 -87.59133982148438) bank134850 +134851 POINT(41.386077648209074 -86.85554039737464) bank134851 +134852 POINT(42.86865040188294 -86.7424563452862) bank134852 +134853 POINT(42.31797942198186 -87.74694736167305) bank134853 +134854 POINT(40.882619833088604 -87.277692652334) bank134854 +134855 POINT(41.317596859824235 -87.63833224490153) bank134855 +134856 POINT(41.102743857558835 -87.60672427319862) bank134856 +134857 POINT(41.20166380262191 -88.24048759993798) bank134857 +134858 POINT(41.19831708099643 -87.07413481284766) bank134858 +134859 POINT(41.70568584737321 -87.58071119633433) bank134859 +134860 POINT(41.45096472907649 -87.88154797912279) bank134860 +134861 POINT(40.993471066806684 -87.04266718016473) bank134861 +134862 POINT(40.89158828311068 -87.81702641543706) bank134862 +134863 POINT(41.4614659335876 -88.02780223789406) bank134863 +134864 POINT(41.09804337405186 -88.10876567589884) bank134864 +134865 POINT(42.85889849103669 -87.16321290823491) bank134865 +134866 POINT(42.72891638602001 -87.6684720377695) bank134866 +134867 POINT(41.731061714426914 -87.01882795229086) bank134867 +134868 POINT(41.879007033279215 -87.31679436341018) bank134868 +134869 POINT(41.963520902131876 -86.68434958623128) bank134869 +134870 POINT(41.203005848587935 -88.52259177453504) bank134870 +134871 POINT(42.56892275216874 -87.11088951259693) bank134871 +134872 POINT(42.200431183889386 -87.99200288656537) bank134872 +134873 POINT(42.71318292409313 -86.89254287352468) bank134873 +134874 POINT(41.4869614080821 -87.8226339082226) bank134874 +134875 POINT(42.134540453683336 -87.35801087462856) bank134875 +134876 POINT(41.22823908536822 -88.23847890362694) bank134876 +134877 POINT(42.18518403798428 -87.60033324032021) bank134877 +134878 POINT(41.62527993977247 -87.02213563884784) bank134878 +134879 POINT(42.3890071180911 -87.65956407294397) bank134879 +134880 POINT(41.11712773104481 -87.68929558520594) bank134880 +134881 POINT(42.317524519323094 -87.03388223674666) bank134881 +134882 POINT(42.87802103531537 -88.22873492446675) bank134882 +134883 POINT(42.35062225016387 -86.93543992441097) bank134883 +134884 POINT(41.37242695636878 -88.45794187098383) bank134884 +134885 POINT(42.848071362591185 -86.75372254925608) bank134885 +134886 POINT(42.4969015745458 -87.32638856107421) bank134886 +134887 POINT(40.946506706226145 -86.64101517282131) bank134887 +134888 POINT(41.03357870109394 -87.4272979961285) bank134888 +134889 POINT(41.242075787059655 -88.08799709206639) bank134889 +134890 POINT(42.74238258161273 -88.28368485427531) bank134890 +134891 POINT(41.550743740048176 -88.15949730809479) bank134891 +134892 POINT(42.43463574178387 -87.24094666032117) bank134892 +134893 POINT(41.22949524369559 -87.01841115940911) bank134893 +134894 POINT(41.07413883867115 -87.39377792663876) bank134894 +134895 POINT(41.11430735071842 -86.79056650299229) bank134895 +134896 POINT(42.61617093010656 -88.19793001522895) bank134896 +134897 POINT(41.33627297730912 -87.24667192419071) bank134897 +134898 POINT(41.29839431950892 -87.75938065929228) bank134898 +134899 POINT(42.640930583505956 -88.19617415663504) bank134899 +134900 POINT(41.73603652477181 -87.87720487102658) bank134900 +134901 POINT(41.79451005444951 -88.36981755366958) bank134901 +134902 POINT(42.578415266236945 -86.92625798307228) bank134902 +134903 POINT(41.35032158169706 -88.44076794271008) bank134903 +134904 POINT(41.70929593486617 -88.43482289197783) bank134904 +134905 POINT(41.00878302907937 -87.36512268087648) bank134905 +134906 POINT(41.210328377352006 -87.57424824471572) bank134906 +134907 POINT(41.67314140290679 -87.89347153713182) bank134907 +134908 POINT(41.43648884932643 -86.83500983715265) bank134908 +134909 POINT(40.95654958999544 -87.07953677769706) bank134909 +134910 POINT(42.743357133288065 -87.45887376066196) bank134910 +134911 POINT(42.43571475193917 -87.8396205485874) bank134911 +134912 POINT(41.12602232474729 -88.21560127939486) bank134912 +134913 POINT(41.159488183313826 -87.06702167514693) bank134913 +134914 POINT(42.809209828682704 -88.41800914649545) bank134914 +134915 POINT(41.141500239658 -87.5737225881477) bank134915 +134916 POINT(41.068030055301804 -87.52971242602904) bank134916 +134917 POINT(41.15189465944257 -87.07402771657252) bank134917 +134918 POINT(41.19609350897162 -87.49895033558222) bank134918 +134919 POINT(42.85928756976307 -87.17797473051378) bank134919 +134920 POINT(41.76479132133919 -88.37451576122018) bank134920 +134921 POINT(42.11077040349521 -87.07521735353556) bank134921 +134922 POINT(41.67668502532981 -86.73127537064356) bank134922 +134923 POINT(41.13156572026751 -86.82499783717158) bank134923 +134924 POINT(41.18889908527113 -87.02359219500907) bank134924 +134925 POINT(42.467076577648186 -87.1112688039049) bank134925 +134926 POINT(42.372062987151374 -87.6598567010391) bank134926 +134927 POINT(42.11317200197043 -88.61881083784874) bank134927 +134928 POINT(42.17515995376313 -88.62929915083744) bank134928 +134929 POINT(42.63092179096556 -87.97682538571242) bank134929 +134930 POINT(41.438294121133254 -86.92782504714536) bank134930 +134931 POINT(42.6853645651835 -86.84312281817293) bank134931 +134932 POINT(41.41386176253265 -87.419788551019) bank134932 +134933 POINT(42.16461682928174 -86.66612095621706) bank134933 +134934 POINT(41.36048488466963 -87.79315948056566) bank134934 +134935 POINT(42.61609318140464 -88.55148711688567) bank134935 +134936 POINT(42.627139213240675 -87.86734138882039) bank134936 +134937 POINT(40.89798912532958 -87.96856839030295) bank134937 +134938 POINT(41.68196599880731 -86.9682752826861) bank134938 +134939 POINT(41.488762546404764 -86.67205996163169) bank134939 +134940 POINT(41.33083300810643 -87.48368592565919) bank134940 +134941 POINT(40.97402563003263 -87.00297314167241) bank134941 +134942 POINT(42.73359887499554 -88.1290734662744) bank134942 +134943 POINT(42.110894284714206 -88.47547007000018) bank134943 +134944 POINT(42.50711061092744 -86.82983085006212) bank134944 +134945 POINT(41.18424412771489 -86.6394254213502) bank134945 +134946 POINT(41.35964239597436 -86.97042881561238) bank134946 +134947 POINT(41.46582340291258 -87.41564952975422) bank134947 +134948 POINT(41.860482956558045 -87.60520649752088) bank134948 +134949 POINT(41.03636421122799 -86.9077687227815) bank134949 +134950 POINT(41.421122954908576 -88.3121514314247) bank134950 +134951 POINT(40.95029595519769 -86.96524105424471) bank134951 +134952 POINT(41.32633643318243 -87.6045739208949) bank134952 +134953 POINT(41.47595989989542 -87.8510663074747) bank134953 +134954 POINT(41.855357485889016 -86.74110344248476) bank134954 +134955 POINT(41.95428859264136 -88.39712920033466) bank134955 +134956 POINT(42.727439750404024 -88.31551583829285) bank134956 +134957 POINT(41.11769050974401 -87.54995081163209) bank134957 +134958 POINT(41.866306595673386 -86.68222460318819) bank134958 +134959 POINT(42.51358562888681 -86.7472455236131) bank134959 +134960 POINT(41.559604614432324 -87.76006905420715) bank134960 +134961 POINT(41.27527753814528 -86.84315735131901) bank134961 +134962 POINT(42.600214300287746 -88.14463301183099) bank134962 +134963 POINT(41.62315196642387 -87.26203351407679) bank134963 +134964 POINT(42.203147217125725 -86.68246623179469) bank134964 +134965 POINT(41.078865865968986 -87.3810796870999) bank134965 +134966 POINT(42.13079744956623 -87.8659469931001) bank134966 +134967 POINT(42.381460129044726 -88.36047111920227) bank134967 +134968 POINT(42.29422028346391 -88.58084010650278) bank134968 +134969 POINT(42.77048583201737 -87.79619709075058) bank134969 +134970 POINT(42.86713162261821 -87.74224015503454) bank134970 +134971 POINT(41.29608022246483 -86.68358836990234) bank134971 +134972 POINT(41.97861236440449 -88.58518725551788) bank134972 +134973 POINT(41.44048112759476 -88.56940995072341) bank134973 +134974 POINT(41.493388689056125 -87.89577259164028) bank134974 +134975 POINT(41.16458076126271 -87.25207040603811) bank134975 +134976 POINT(42.249029576935406 -86.67746964892048) bank134976 +134977 POINT(41.92708348514253 -88.34985751293075) bank134977 +134978 POINT(41.466157733098456 -87.17948585706641) bank134978 +134979 POINT(41.56807787445103 -86.91068822087715) bank134979 +134980 POINT(42.76704795942469 -88.00824098638859) bank134980 +134981 POINT(41.47969343171613 -88.19002629641447) bank134981 +134982 POINT(41.31159403295134 -88.60382810512297) bank134982 +134983 POINT(42.753691958191965 -88.38000448617942) bank134983 +134984 POINT(41.57960978105803 -87.02461160637044) bank134984 +134985 POINT(41.161735370558986 -87.6207359572362) bank134985 +134986 POINT(42.76734500027104 -87.09174122712685) bank134986 +134987 POINT(41.02637389141004 -87.94015867776808) bank134987 +134988 POINT(41.80907102273037 -86.96779831942048) bank134988 +134989 POINT(42.67501866502816 -86.9323655606104) bank134989 +134990 POINT(41.29046798366997 -87.13778431068309) bank134990 +134991 POINT(41.19275696437593 -87.8956129780691) bank134991 +134992 POINT(40.91490029864124 -86.65821447176438) bank134992 +134993 POINT(40.88748982998825 -87.1886476988674) bank134993 +134994 POINT(42.64610871392857 -87.36987361531139) bank134994 +134995 POINT(42.68678118969934 -88.34755755363273) bank134995 +134996 POINT(42.67555510785226 -87.9293554941367) bank134996 +134997 POINT(42.83697679688636 -86.7136987042407) bank134997 +134998 POINT(42.86807846567781 -87.01281081865326) bank134998 +134999 POINT(41.93108100469085 -87.66439642924998) bank134999 +135000 POINT(42.48196104134513 -86.81369964195392) bank135000 +135001 POINT(42.01052066983431 -87.1963958836913) bank135001 +135002 POINT(41.00662381505086 -88.34808004233629) bank135002 +135003 POINT(42.11951808500762 -88.2783524482687) bank135003 +135004 POINT(42.36895755942856 -88.10441002771364) bank135004 +135005 POINT(42.01777388602661 -87.01701904703418) bank135005 +135006 POINT(42.85760311939972 -86.75058093957072) bank135006 +135007 POINT(41.797245760921946 -86.82015515690019) bank135007 +135008 POINT(42.30976609197413 -88.35546072456175) bank135008 +135009 POINT(42.78642019136364 -86.86205307209332) bank135009 +135010 POINT(41.06135036257623 -88.27587304638375) bank135010 +135011 POINT(42.547609403412665 -87.56888581767821) bank135011 +135012 POINT(42.28299120444125 -87.95714384611236) bank135012 +135013 POINT(42.371549744926085 -87.63953873183145) bank135013 +135014 POINT(42.57036009488142 -87.9509510827919) bank135014 +135015 POINT(41.280884001201606 -87.83002777715757) bank135015 +135016 POINT(42.84788020899375 -88.01406576527035) bank135016 +135017 POINT(41.903998479447594 -87.70017323177515) bank135017 +135018 POINT(41.457090480468786 -88.00943288113538) bank135018 +135019 POINT(42.12173806061073 -87.40140161157731) bank135019 +135020 POINT(42.328585753047484 -87.00329625467982) bank135020 +135021 POINT(42.68335389355336 -86.7913631983296) bank135021 +135022 POINT(42.84531394744532 -87.20078058633135) bank135022 +135023 POINT(41.24454399487278 -88.38511068946565) bank135023 +135024 POINT(41.39904752653664 -87.2415373451396) bank135024 +135025 POINT(42.71179369623349 -87.56943168087778) bank135025 +135026 POINT(41.34582765903348 -86.74598835099184) bank135026 +135027 POINT(41.98334415967111 -86.85875225500925) bank135027 +135028 POINT(42.821353568561136 -87.05201046226654) bank135028 +135029 POINT(42.55904055744342 -87.90890072129869) bank135029 +135030 POINT(41.18887239403809 -87.55402446146125) bank135030 +135031 POINT(41.33520073880589 -88.30532790271926) bank135031 +135032 POINT(41.11371239990575 -88.05763117025283) bank135032 +135033 POINT(41.82453156078124 -86.9486559354635) bank135033 +135034 POINT(42.31407189838066 -87.6355639395328) bank135034 +135035 POINT(41.273484897035694 -87.62809360068516) bank135035 +135036 POINT(42.469890351466844 -88.16045529261915) bank135036 +135037 POINT(42.09781103934809 -87.53803500560139) bank135037 +135038 POINT(42.55117500388306 -88.42193772213105) bank135038 +135039 POINT(41.46843061465684 -87.89733875806398) bank135039 +135040 POINT(40.99162427688154 -87.31165674759119) bank135040 +135041 POINT(42.158914638474876 -88.54724166847446) bank135041 +135042 POINT(41.9062634741612 -87.7427149048601) bank135042 +135043 POINT(41.461032100546724 -88.38739252763776) bank135043 +135044 POINT(41.9258445048397 -87.23720791264724) bank135044 +135045 POINT(42.82182412347907 -87.92463712433079) bank135045 +135046 POINT(41.67715328950644 -86.74320966897534) bank135046 +135047 POINT(42.02502175741369 -88.55258285360304) bank135047 +135048 POINT(41.79355086654785 -87.0817947770714) bank135048 +135049 POINT(42.11058877346619 -86.81220995649845) bank135049 +135050 POINT(42.84948473737982 -88.31355741277069) bank135050 +135051 POINT(42.25415351146766 -88.4192912947246) bank135051 +135052 POINT(41.529657965773204 -86.68339249187633) bank135052 +135053 POINT(42.587805787995165 -88.34468512504925) bank135053 +135054 POINT(41.47781596904718 -88.46726068932773) bank135054 +135055 POINT(42.828808630720296 -86.7651494487388) bank135055 +135056 POINT(42.081526553854474 -87.53908834330893) bank135056 +135057 POINT(42.09388707567408 -86.89422358106964) bank135057 +135058 POINT(41.31680978488851 -87.71994531048394) bank135058 +135059 POINT(41.5604705077984 -87.23210223430057) bank135059 +135060 POINT(42.41973804065054 -88.40109623190818) bank135060 +135061 POINT(42.17378954283278 -87.80624359553275) bank135061 +135062 POINT(42.14307706524712 -88.24342651392011) bank135062 +135063 POINT(41.71309332316248 -88.1942623297173) bank135063 +135064 POINT(41.54616208521843 -87.21851052360896) bank135064 +135065 POINT(42.181653754367545 -87.30974217585604) bank135065 +135066 POINT(42.031802212150176 -87.41050661830761) bank135066 +135067 POINT(41.514605424515445 -88.10937234290226) bank135067 +135068 POINT(41.93492742194774 -88.56619676442172) bank135068 +135069 POINT(42.32274386855068 -87.05871746226856) bank135069 +135070 POINT(42.06688010666751 -86.87089312756287) bank135070 +135071 POINT(41.58882693867235 -87.32005015046684) bank135071 +135072 POINT(42.18841129907158 -87.74896019751658) bank135072 +135073 POINT(41.37029167067775 -87.52657481046435) bank135073 +135074 POINT(41.46500991373195 -86.66526803198157) bank135074 +135075 POINT(41.49027634006035 -88.42983570362614) bank135075 +135076 POINT(42.5693744258991 -88.29541972833137) bank135076 +135077 POINT(41.30956771814503 -87.01844490075344) bank135077 +135078 POINT(42.590314653229406 -86.8619221185128) bank135078 +135079 POINT(41.26845704131261 -87.50725294652557) bank135079 +135080 POINT(41.05128307445316 -87.9301761891659) bank135080 +135081 POINT(41.47617119057879 -87.21642002455695) bank135081 +135082 POINT(41.561659020828586 -87.25467844714605) bank135082 +135083 POINT(41.90372072013962 -87.43386826849586) bank135083 +135084 POINT(41.880104944762095 -87.25810807011541) bank135084 +135085 POINT(40.976610251141224 -87.76628234589016) bank135085 +135086 POINT(41.99094389140328 -88.49494165357387) bank135086 +135087 POINT(42.650581872256886 -88.43600709425544) bank135087 +135088 POINT(42.50534003837662 -87.04776806911357) bank135088 +135089 POINT(42.6861705601568 -87.58566666964506) bank135089 +135090 POINT(42.2916522967573 -87.99333176079303) bank135090 +135091 POINT(41.78209544869294 -87.28486467131759) bank135091 +135092 POINT(40.93645660193253 -87.8237037842437) bank135092 +135093 POINT(41.517574193082766 -88.43293982301512) bank135093 +135094 POINT(40.955522226586474 -87.9465793322359) bank135094 +135095 POINT(42.597430448284165 -86.87025899202766) bank135095 +135096 POINT(42.242868738812 -87.81334300771218) bank135096 +135097 POINT(41.451508022343035 -87.12527354115056) bank135097 +135098 POINT(42.57804466251877 -86.67704515885555) bank135098 +135099 POINT(41.30106412039245 -86.93230693717283) bank135099 +135100 POINT(41.07659577666365 -87.2445812529053) bank135100 +135101 POINT(41.22237415406287 -88.08680038506355) bank135101 +135102 POINT(41.90909570183179 -87.0996587881659) bank135102 +135103 POINT(41.11576582227532 -87.46303907115389) bank135103 +135104 POINT(42.08387484466301 -87.66555977679518) bank135104 +135105 POINT(40.92280972679927 -87.50242862062936) bank135105 +135106 POINT(42.712203519113125 -87.08264664183943) bank135106 +135107 POINT(42.19947217715444 -88.2905245293699) bank135107 +135108 POINT(41.650337296732786 -87.40141337904711) bank135108 +135109 POINT(42.38547717968089 -87.02585046223214) bank135109 +135110 POINT(42.65048025179765 -88.1764087967486) bank135110 +135111 POINT(41.90673632239579 -87.35878186502353) bank135111 +135112 POINT(42.795993539436566 -86.84551782755638) bank135112 +135113 POINT(41.05579098639999 -88.03774861273781) bank135113 +135114 POINT(42.07726183233331 -87.58121709219778) bank135114 +135115 POINT(42.59584541434517 -87.06934448997947) bank135115 +135116 POINT(41.58962891996131 -86.72481970190472) bank135116 +135117 POINT(41.908918684873086 -87.96048603240824) bank135117 +135118 POINT(41.94882562603767 -88.34781120181329) bank135118 +135119 POINT(41.1980537385686 -88.06063487814333) bank135119 +135120 POINT(41.50666361093327 -88.29643016900167) bank135120 +135121 POINT(42.44961671871431 -87.78055340935111) bank135121 +135122 POINT(42.7353673492154 -87.10542481519785) bank135122 +135123 POINT(41.411091289077405 -88.2666188169734) bank135123 +135124 POINT(42.37756129159653 -87.73513684956751) bank135124 +135125 POINT(40.89310680912772 -86.70453740247412) bank135125 +135126 POINT(41.8839607787265 -86.76320134971562) bank135126 +135127 POINT(41.39168799573374 -87.70123424867275) bank135127 +135128 POINT(40.884204215978336 -87.8049703662428) bank135128 +135129 POINT(41.75335772058447 -86.65791458176871) bank135129 +135130 POINT(42.518118900444236 -87.13264360684242) bank135130 +135131 POINT(42.729826409763206 -88.17364979066595) bank135131 +135132 POINT(42.678300217452424 -88.3899247706546) bank135132 +135133 POINT(42.8777058867048 -88.45059199732023) bank135133 +135134 POINT(42.34264807128409 -88.02803028953404) bank135134 +135135 POINT(41.96942228169091 -87.58215927602856) bank135135 +135136 POINT(41.88954475517491 -88.30509233497554) bank135136 +135137 POINT(41.67084613845464 -87.68427782941183) bank135137 +135138 POINT(42.07106970949726 -88.17121598121506) bank135138 +135139 POINT(41.53182220317108 -88.14839482972262) bank135139 +135140 POINT(42.48115337033384 -87.83295936713601) bank135140 +135141 POINT(40.99727815348128 -88.20533952141258) bank135141 +135142 POINT(40.921422853068826 -88.20519786746488) bank135142 +135143 POINT(41.81855944193246 -88.29933521580077) bank135143 +135144 POINT(42.29811971476946 -86.75975764156478) bank135144 +135145 POINT(41.99131615128011 -86.82601323828983) bank135145 +135146 POINT(42.58682135776753 -88.49130659267976) bank135146 +135147 POINT(42.12054996449173 -87.72145002536453) bank135147 +135148 POINT(41.04798222002039 -88.22130036354724) bank135148 +135149 POINT(42.489190896333206 -87.84808269821559) bank135149 +135150 POINT(41.32502499347326 -87.37007202325111) bank135150 +135151 POINT(42.15079395787775 -87.34424377869807) bank135151 +135152 POINT(40.89419324258979 -87.84642726100563) bank135152 +135153 POINT(42.76224139119543 -86.97146798160287) bank135153 +135154 POINT(41.20442252986897 -87.05963253910433) bank135154 +135155 POINT(41.375984656492044 -86.79768793354269) bank135155 +135156 POINT(41.389525119837884 -88.2880947020012) bank135156 +135157 POINT(42.37450793714164 -88.01128290056911) bank135157 +135158 POINT(42.575062850024224 -88.01165599030227) bank135158 +135159 POINT(41.7153861874012 -88.52072768386418) bank135159 +135160 POINT(42.192252514556195 -87.9772331649034) bank135160 +135161 POINT(41.8566709316968 -87.87211607565068) bank135161 +135162 POINT(41.16465700675508 -86.82051512909803) bank135162 +135163 POINT(42.25305930105663 -87.20628117396683) bank135163 +135164 POINT(42.10192583527369 -88.3612657749053) bank135164 +135165 POINT(42.460454093218836 -88.46631491629755) bank135165 +135166 POINT(42.82423719425415 -88.24636622129262) bank135166 +135167 POINT(41.1734553717529 -88.45341948206026) bank135167 +135168 POINT(41.73534725043182 -88.48059845452407) bank135168 +135169 POINT(41.40672166744153 -87.8275630775658) bank135169 +135170 POINT(41.277873997627786 -86.78367296081362) bank135170 +135171 POINT(41.73564541717762 -86.70228134327999) bank135171 +135172 POINT(41.37721206202187 -88.43343492397602) bank135172 +135173 POINT(42.42544283290923 -86.87816673429874) bank135173 +135174 POINT(42.096557229670154 -87.4196834935752) bank135174 +135175 POINT(42.38645350762637 -87.08540304866558) bank135175 +135176 POINT(41.802467993406104 -88.59337983377273) bank135176 +135177 POINT(40.93709497550824 -87.59858659625093) bank135177 +135178 POINT(41.18760744690658 -88.23714530859897) bank135178 +135179 POINT(41.298373014595064 -87.31154162843757) bank135179 +135180 POINT(42.571677914415375 -87.10834891837479) bank135180 +135181 POINT(42.14017874784999 -87.307461501584) bank135181 +135182 POINT(41.924925659831025 -87.3545010765056) bank135182 +135183 POINT(41.124823847516076 -87.57116935486788) bank135183 +135184 POINT(41.398087542288444 -87.04929932215329) bank135184 +135185 POINT(41.55817138781199 -86.66360482776406) bank135185 +135186 POINT(42.69002556657919 -86.77355686758153) bank135186 +135187 POINT(42.391592303882334 -88.3025639188292) bank135187 +135188 POINT(41.10856898424571 -88.08796143147725) bank135188 +135189 POINT(41.20294706285063 -88.54976076759259) bank135189 +135190 POINT(40.91967570575272 -88.19665633285007) bank135190 +135191 POINT(41.6614610336934 -88.039601106747) bank135191 +135192 POINT(41.360880151003855 -87.43566627104941) bank135192 +135193 POINT(41.274314831035305 -88.20699855654668) bank135193 +135194 POINT(41.515285822325566 -87.74066614429087) bank135194 +135195 POINT(42.4668910652561 -87.10229756514167) bank135195 +135196 POINT(41.83589587873827 -88.54861044475406) bank135196 +135197 POINT(42.11964666517601 -86.8485008071752) bank135197 +135198 POINT(41.09797017840751 -87.27286185524643) bank135198 +135199 POINT(42.46679094918124 -88.06643103321841) bank135199 +135200 POINT(41.89885992170171 -88.2801485589066) bank135200 +135201 POINT(42.56756511000694 -86.96187514684001) bank135201 +135202 POINT(41.453855885924185 -86.8787220289751) bank135202 +135203 POINT(42.39150770830807 -88.01562255151086) bank135203 +135204 POINT(41.746126624493755 -87.74192415971123) bank135204 +135205 POINT(41.01840530406466 -88.4264216487594) bank135205 +135206 POINT(41.07232300853232 -87.62600088235858) bank135206 +135207 POINT(41.54282336898767 -87.26244318727433) bank135207 +135208 POINT(41.64028868840473 -88.03495355219624) bank135208 +135209 POINT(41.56833340435508 -87.24535086119268) bank135209 +135210 POINT(42.632491152346084 -88.28457083646775) bank135210 +135211 POINT(40.95966390148202 -87.53364037002443) bank135211 +135212 POINT(42.72687128613048 -86.76926132922463) bank135212 +135213 POINT(41.995979791768264 -88.21882017989651) bank135213 +135214 POINT(40.87936709719442 -88.30061348463228) bank135214 +135215 POINT(41.43390452997369 -88.11610760944751) bank135215 +135216 POINT(41.03024623163605 -88.38425300028138) bank135216 +135217 POINT(40.99324034616042 -87.26019952537393) bank135217 +135218 POINT(42.65919994304141 -86.94771457331268) bank135218 +135219 POINT(41.33958729216405 -88.42736775918662) bank135219 +135220 POINT(42.65206863781566 -88.16274607077189) bank135220 +135221 POINT(41.25375317865729 -88.33470714389887) bank135221 +135222 POINT(41.76202139379184 -88.21902952000477) bank135222 +135223 POINT(41.3032523781751 -88.03985571392194) bank135223 +135224 POINT(41.490983684218456 -88.12228911862019) bank135224 +135225 POINT(41.3586247904521 -87.51671198568204) bank135225 +135226 POINT(42.31743588180552 -87.60123620121007) bank135226 +135227 POINT(42.064251555452785 -86.72031895743372) bank135227 +135228 POINT(41.80271101888805 -88.54772805545552) bank135228 +135229 POINT(40.931791680126054 -87.2994237780413) bank135229 +135230 POINT(40.9304210302012 -86.98601974797671) bank135230 +135231 POINT(42.50591226641669 -86.82891765694218) bank135231 +135232 POINT(42.02390612385408 -86.76399093271579) bank135232 +135233 POINT(42.256090583016906 -87.58838359543614) bank135233 +135234 POINT(42.348797961772156 -87.0205663759923) bank135234 +135235 POINT(42.616445304431664 -87.43931004834957) bank135235 +135236 POINT(41.95745840853551 -88.59595654184402) bank135236 +135237 POINT(41.99487165713961 -87.72526069592402) bank135237 +135238 POINT(42.8549202092891 -87.33679591558818) bank135238 +135239 POINT(41.89515933043335 -86.81402654488554) bank135239 +135240 POINT(42.567999755563825 -88.38711316624753) bank135240 +135241 POINT(41.81226201092096 -87.02932924230929) bank135241 +135242 POINT(40.941259881867595 -86.82752748844482) bank135242 +135243 POINT(41.625665037106664 -87.93477310754533) bank135243 +135244 POINT(42.58424296424413 -87.38276354684125) bank135244 +135245 POINT(41.04906020954165 -88.11350559332142) bank135245 +135246 POINT(41.34043236785941 -88.05761829159141) bank135246 +135247 POINT(40.90043249026517 -87.1506215574699) bank135247 +135248 POINT(41.83021815964974 -88.15147062593904) bank135248 +135249 POINT(42.08454667315446 -87.53728379578334) bank135249 +135250 POINT(42.207889216463485 -87.65306428839268) bank135250 +135251 POINT(42.132309832275574 -87.49038983341283) bank135251 +135252 POINT(41.022814440237134 -86.67598454324481) bank135252 +135253 POINT(42.30412305319959 -88.51255136856525) bank135253 +135254 POINT(42.75400338993928 -87.42951012435705) bank135254 +135255 POINT(42.191068239839446 -87.99378009774266) bank135255 +135256 POINT(41.67512050697703 -88.52935780068577) bank135256 +135257 POINT(41.37104501174208 -87.17353192492622) bank135257 +135258 POINT(41.15885034399657 -88.28138023105862) bank135258 +135259 POINT(41.639082241590046 -87.79931285308483) bank135259 +135260 POINT(40.937185810168224 -88.32538052425545) bank135260 +135261 POINT(41.889278311211754 -87.89901576047386) bank135261 +135262 POINT(41.76663542302633 -87.57571431550188) bank135262 +135263 POINT(41.82903899034556 -86.97296177052041) bank135263 +135264 POINT(42.10692659244658 -88.59644450484461) bank135264 +135265 POINT(42.52629230397935 -86.89920108736146) bank135265 +135266 POINT(41.090056308483554 -88.08484469418073) bank135266 +135267 POINT(41.64275071082998 -87.26475278137286) bank135267 +135268 POINT(41.52416698394838 -88.4822845344667) bank135268 +135269 POINT(42.62338478386749 -88.41117619840722) bank135269 +135270 POINT(40.960558264252334 -88.30437208869333) bank135270 +135271 POINT(41.59556598495423 -88.28052591160049) bank135271 +135272 POINT(42.34207338075309 -87.77083610447018) bank135272 +135273 POINT(42.71618385891362 -86.8877352336824) bank135273 +135274 POINT(41.46508132960788 -86.63551311423275) bank135274 +135275 POINT(41.30529730505718 -87.34175918160561) bank135275 +135276 POINT(41.23737848991052 -87.46441693931958) bank135276 +135277 POINT(42.17747264330188 -88.10888214918631) bank135277 +135278 POINT(42.85439681823234 -87.63749987066187) bank135278 +135279 POINT(41.166933794872094 -86.79261068672197) bank135279 +135280 POINT(41.79399406474015 -87.54470946880755) bank135280 +135281 POINT(42.64775256356529 -86.92432357273688) bank135281 +135282 POINT(42.745932067394016 -88.01941708836938) bank135282 +135283 POINT(41.83460467372506 -88.57072845170045) bank135283 +135284 POINT(42.69260241410996 -87.19340685336493) bank135284 +135285 POINT(41.80380298199484 -88.58336370851436) bank135285 +135286 POINT(42.797957897932115 -87.10973988709382) bank135286 +135287 POINT(42.229474381249865 -87.675503315344) bank135287 +135288 POINT(41.02879674308376 -88.3892969788714) bank135288 +135289 POINT(41.32743855674646 -87.43627828010327) bank135289 +135290 POINT(42.109611211409955 -87.89980540996984) bank135290 +135291 POINT(42.40598281274052 -87.13491125872237) bank135291 +135292 POINT(41.36434770462567 -87.18565726910288) bank135292 +135293 POINT(41.246719005721424 -88.54503134149647) bank135293 +135294 POINT(42.83333665568653 -87.07621038131212) bank135294 +135295 POINT(42.63253197834118 -87.83228399898515) bank135295 +135296 POINT(41.38057701976943 -88.02331003023934) bank135296 +135297 POINT(41.24690002898564 -87.95710519514398) bank135297 +135298 POINT(41.036216154443416 -88.31159397567798) bank135298 +135299 POINT(41.53113421107156 -87.26910103661737) bank135299 +135300 POINT(42.22745967275061 -88.08365999407657) bank135300 +135301 POINT(42.434544617252385 -88.20265280580001) bank135301 +135302 POINT(41.08370952902773 -87.85947645212343) bank135302 +135303 POINT(42.735396613313526 -87.8615366449341) bank135303 +135304 POINT(42.64491060424507 -87.79984048312988) bank135304 +135305 POINT(41.650012995608805 -86.83373054261821) bank135305 +135306 POINT(41.25023126514168 -87.8500674473443) bank135306 +135307 POINT(42.76587993090072 -87.9787148265748) bank135307 +135308 POINT(41.69544152047315 -88.02108153975807) bank135308 +135309 POINT(41.5959702202485 -87.27004906073589) bank135309 +135310 POINT(42.51758500631928 -87.02211617037013) bank135310 +135311 POINT(40.9161836504676 -87.80166885208838) bank135311 +135312 POINT(41.23866153499744 -88.3405667313635) bank135312 +135313 POINT(42.228836517475614 -86.79489131988214) bank135313 +135314 POINT(41.761971632744455 -87.9562237037119) bank135314 +135315 POINT(41.30681756764945 -87.82316564086982) bank135315 +135316 POINT(41.31934061788248 -88.14623806815312) bank135316 +135317 POINT(41.30887825426765 -87.50833129364617) bank135317 +135318 POINT(42.10101708151005 -87.44998805122209) bank135318 +135319 POINT(42.39126045039251 -87.95215912124709) bank135319 +135320 POINT(42.381098446668894 -88.379998227962) bank135320 +135321 POINT(41.39668808777844 -87.20064968797305) bank135321 +135322 POINT(42.33177712406599 -87.95293508994226) bank135322 +135323 POINT(41.67159853198843 -87.3233685642106) bank135323 +135324 POINT(41.58860319553115 -88.43024765423867) bank135324 +135325 POINT(41.59659811483999 -87.2585491337203) bank135325 +135326 POINT(41.84433147370231 -87.0493283211481) bank135326 +135327 POINT(41.03180371925556 -87.42272223409054) bank135327 +135328 POINT(41.757504652894525 -87.01364740841397) bank135328 +135329 POINT(42.278996717603114 -87.96298213346945) bank135329 +135330 POINT(42.20658041480839 -87.84836668708779) bank135330 +135331 POINT(42.05671225788462 -88.23196932683236) bank135331 +135332 POINT(41.37939414961035 -87.35571791801135) bank135332 +135333 POINT(41.693498788155644 -86.74571733641409) bank135333 +135334 POINT(41.967526011956494 -86.8668008664249) bank135334 +135335 POINT(41.77383626208744 -87.29210175672686) bank135335 +135336 POINT(42.21946038073334 -88.24935623416557) bank135336 +135337 POINT(41.16195370244102 -86.78662030176794) bank135337 +135338 POINT(42.15680306961376 -87.5553646264379) bank135338 +135339 POINT(41.22779733095221 -87.64531434555752) bank135339 +135340 POINT(42.4561765390431 -88.53789976772114) bank135340 +135341 POINT(42.54518725515298 -88.18201611293743) bank135341 +135342 POINT(42.26577198800747 -87.9414910218322) bank135342 +135343 POINT(41.891062993307784 -87.4503417501251) bank135343 +135344 POINT(40.97672309973001 -87.29137348977217) bank135344 +135345 POINT(41.69321761185101 -87.09694143309656) bank135345 +135346 POINT(41.484573329727716 -87.92569171723656) bank135346 +135347 POINT(42.77599744160817 -88.2910442996096) bank135347 +135348 POINT(41.176506299000955 -86.69246716329991) bank135348 +135349 POINT(41.15212371240904 -88.58003982441143) bank135349 +135350 POINT(42.72198234409346 -86.74873827910724) bank135350 +135351 POINT(42.73026850557185 -88.25768460488096) bank135351 +135352 POINT(42.57735011789576 -88.26532066590202) bank135352 +135353 POINT(40.91186489123505 -87.47969018194418) bank135353 +135354 POINT(41.61075435269317 -87.8990751071971) bank135354 +135355 POINT(41.06644875883265 -88.55018511903289) bank135355 +135356 POINT(41.63637747271742 -86.74926233635834) bank135356 +135357 POINT(41.810743162542195 -88.16213009667597) bank135357 +135358 POINT(42.5060248549612 -88.04033465266166) bank135358 +135359 POINT(41.57277073119275 -87.55950781233287) bank135359 +135360 POINT(41.20424378274426 -88.18537207082869) bank135360 +135361 POINT(42.645436108553184 -87.42568544369675) bank135361 +135362 POINT(41.60990570757586 -88.30573180962122) bank135362 +135363 POINT(42.378212801063334 -88.41830033035649) bank135363 +135364 POINT(42.70990677108183 -87.45694675366653) bank135364 +135365 POINT(42.59569261895472 -88.2361837289006) bank135365 +135366 POINT(40.911599103173245 -88.514180268862) bank135366 +135367 POINT(41.34355063740901 -86.72890867498329) bank135367 +135368 POINT(41.44560146454036 -88.55039325058935) bank135368 +135369 POINT(42.12014066287039 -87.90284134089684) bank135369 +135370 POINT(42.497580489490154 -88.42195424276967) bank135370 +135371 POINT(41.918615555474 -88.1787290776451) bank135371 +135372 POINT(41.77479844599078 -87.92555145524722) bank135372 +135373 POINT(41.623740808549776 -86.76293711626789) bank135373 +135374 POINT(42.55978502538134 -88.30329416932214) bank135374 +135375 POINT(41.32981153212476 -86.78379458307951) bank135375 +135376 POINT(41.00583782499289 -87.53391241088174) bank135376 +135377 POINT(40.98730909813398 -87.5641024732259) bank135377 +135378 POINT(42.08680750258129 -88.48444205642662) bank135378 +135379 POINT(42.01651513410597 -88.07048234269395) bank135379 +135380 POINT(41.04459819656655 -87.07047608976472) bank135380 +135381 POINT(41.079108794957236 -86.66558908791417) bank135381 +135382 POINT(41.375647264383 -87.85648331049207) bank135382 +135383 POINT(41.75049556958871 -88.01572598722441) bank135383 +135384 POINT(41.6371732901991 -87.17040133985824) bank135384 +135385 POINT(41.66361823168691 -87.82036990090323) bank135385 +135386 POINT(41.80934984465291 -88.26312133311016) bank135386 +135387 POINT(42.252008769043044 -87.85761810337078) bank135387 +135388 POINT(42.392866750724906 -87.82551068542055) bank135388 +135389 POINT(41.98180517379636 -88.41110708813802) bank135389 +135390 POINT(42.248682242294784 -86.96341767168306) bank135390 +135391 POINT(41.44157202675858 -87.90653838501721) bank135391 +135392 POINT(41.84628356891277 -87.01232730457528) bank135392 +135393 POINT(42.20383617119734 -88.21699799833564) bank135393 +135394 POINT(41.01193429706378 -86.94707262553935) bank135394 +135395 POINT(41.36755521215611 -88.3802646141184) bank135395 +135396 POINT(42.85613735439308 -87.44916234486888) bank135396 +135397 POINT(41.51857412398538 -86.82808184326646) bank135397 +135398 POINT(41.64742047575121 -86.9559572917012) bank135398 +135399 POINT(41.48001584122869 -87.64320409126954) bank135399 +135400 POINT(41.78505811521756 -88.37419080768478) bank135400 +135401 POINT(42.47134101106148 -87.45221834326333) bank135401 +135402 POINT(42.324436865191835 -88.29613751556333) bank135402 +135403 POINT(42.42621519715978 -88.34860873929902) bank135403 +135404 POINT(42.72794014874764 -88.31104777383638) bank135404 +135405 POINT(41.93790602361131 -88.60316921254253) bank135405 +135406 POINT(42.707133981993564 -87.38624456549789) bank135406 +135407 POINT(42.15231392165521 -88.45596548455028) bank135407 +135408 POINT(41.38427441596218 -86.96604577654183) bank135408 +135409 POINT(42.441712225548464 -87.33609227624417) bank135409 +135410 POINT(42.51106333407811 -86.75006169409167) bank135410 +135411 POINT(42.463134342632145 -88.21013909811884) bank135411 +135412 POINT(42.48667849443123 -87.4005046027474) bank135412 +135413 POINT(41.031933542458844 -88.15463804840022) bank135413 +135414 POINT(41.642979307267964 -88.26144283137769) bank135414 +135415 POINT(41.73612541257924 -87.40954795059848) bank135415 +135416 POINT(42.477689019928576 -87.26098980648192) bank135416 +135417 POINT(42.222137774825555 -87.19851163073166) bank135417 +135418 POINT(42.52167404516052 -86.7836214491645) bank135418 +135419 POINT(42.10997477828077 -86.97504914767309) bank135419 +135420 POINT(42.840597119603025 -87.24482141434044) bank135420 +135421 POINT(42.14227503452514 -86.97384556287436) bank135421 +135422 POINT(42.765098539834355 -88.04085168599734) bank135422 +135423 POINT(42.628565258069465 -88.0326022254458) bank135423 +135424 POINT(41.534699461840454 -86.63527141376423) bank135424 +135425 POINT(41.85102796606836 -88.04352266148477) bank135425 +135426 POINT(41.36016769507657 -88.3644105822878) bank135426 +135427 POINT(40.97550159639317 -87.11090735946968) bank135427 +135428 POINT(41.24490534315389 -87.95757349599948) bank135428 +135429 POINT(42.132912996151866 -87.19475374214112) bank135429 +135430 POINT(42.48331687967153 -87.43035830710238) bank135430 +135431 POINT(42.06046768313667 -88.54771484959812) bank135431 +135432 POINT(42.81564561419395 -87.75254754822882) bank135432 +135433 POINT(42.82606097520727 -87.49717880931445) bank135433 +135434 POINT(41.55204726546394 -87.02274467223694) bank135434 +135435 POINT(42.48166950491368 -87.14921542392955) bank135435 +135436 POINT(42.51933354993379 -87.44634315344547) bank135436 +135437 POINT(42.38612289648571 -87.87090130956278) bank135437 +135438 POINT(42.0649044108836 -87.11316480319763) bank135438 +135439 POINT(41.99317226021113 -87.3773585874638) bank135439 +135440 POINT(41.157671807036515 -88.052413099802) bank135440 +135441 POINT(41.79548958662371 -87.8855127876655) bank135441 +135442 POINT(42.42969720434652 -88.01658754053511) bank135442 +135443 POINT(42.74102916522998 -87.04516566344928) bank135443 +135444 POINT(42.82694977366148 -88.22802869058589) bank135444 +135445 POINT(41.73504904427421 -87.01261191461133) bank135445 +135446 POINT(41.24144308015976 -87.90341608042871) bank135446 +135447 POINT(41.618147617017065 -88.40130306709851) bank135447 +135448 POINT(42.302879444671746 -88.30362027248697) bank135448 +135449 POINT(42.032392260864 -87.99961149261567) bank135449 +135450 POINT(41.448306332049754 -87.61037910606942) bank135450 +135451 POINT(41.15481994176268 -86.81466129768258) bank135451 +135452 POINT(42.7770158738768 -87.44339096399439) bank135452 +135453 POINT(42.3542505091052 -88.57762425815764) bank135453 +135454 POINT(41.079169978608434 -88.54568396352369) bank135454 +135455 POINT(41.1193673140344 -88.432783856834) bank135455 +135456 POINT(42.3626734941194 -88.35323088836779) bank135456 +135457 POINT(42.253173867815704 -88.40835221300877) bank135457 +135458 POINT(42.24415169404173 -87.34832709253577) bank135458 +135459 POINT(42.486448339382825 -87.80161223716638) bank135459 +135460 POINT(42.23928889056077 -87.42875012939282) bank135460 +135461 POINT(42.427613101137396 -87.30308760626531) bank135461 +135462 POINT(41.70288342195566 -86.93475757948484) bank135462 +135463 POINT(41.37832909579306 -87.57630113091858) bank135463 +135464 POINT(41.81339997745749 -88.10933666246201) bank135464 +135465 POINT(42.69673467924856 -88.16022406746144) bank135465 +135466 POINT(42.02903009354506 -87.92103669679516) bank135466 +135467 POINT(41.602704899981426 -87.27030317363814) bank135467 +135468 POINT(42.119887871387505 -88.19028388549484) bank135468 +135469 POINT(42.38341338946595 -88.04108799355932) bank135469 +135470 POINT(42.08657134128784 -87.61051660704697) bank135470 +135471 POINT(42.133263887060146 -86.93489011356245) bank135471 +135472 POINT(41.26996236600232 -88.28337897225194) bank135472 +135473 POINT(42.49477367606853 -86.9670248554673) bank135473 +135474 POINT(42.07905065662228 -86.73211118679654) bank135474 +135475 POINT(41.45209713028006 -88.01139795975682) bank135475 +135476 POINT(41.1050323306757 -88.11497828265163) bank135476 +135477 POINT(42.44254731242204 -86.80932009047305) bank135477 +135478 POINT(41.622416093646116 -88.3852418669829) bank135478 +135479 POINT(41.68970499945898 -87.06587080686458) bank135479 +135480 POINT(41.71998628054778 -87.17378600218724) bank135480 +135481 POINT(41.64487680885018 -88.55996010001044) bank135481 +135482 POINT(42.43034080645072 -88.06726630376681) bank135482 +135483 POINT(42.640077027285116 -87.09833603417566) bank135483 +135484 POINT(41.61984250854927 -88.28252724451816) bank135484 +135485 POINT(42.083581281639724 -87.66045128336292) bank135485 +135486 POINT(41.67789770014019 -87.85434624667592) bank135486 +135487 POINT(40.900983717738285 -87.87797917986346) bank135487 +135488 POINT(41.52517860478256 -86.98117807016854) bank135488 +135489 POINT(41.66580958902052 -88.03929918447898) bank135489 +135490 POINT(41.78673323084669 -87.62492364218126) bank135490 +135491 POINT(41.76916877556996 -86.6387521549073) bank135491 +135492 POINT(42.62149363814047 -86.8935724981527) bank135492 +135493 POINT(42.79434815836451 -87.58550632634996) bank135493 +135494 POINT(42.18009603344698 -88.18852741093833) bank135494 +135495 POINT(42.682352766129995 -87.61651305241801) bank135495 +135496 POINT(42.1935405851003 -87.45543133776725) bank135496 +135497 POINT(41.33330675954436 -86.70339349227649) bank135497 +135498 POINT(41.37123094539009 -87.42129797941783) bank135498 +135499 POINT(42.297098717061175 -87.44657913928121) bank135499 +135500 POINT(41.64268361693357 -88.00395596783757) bank135500 +135501 POINT(40.897171340208416 -88.34845527533739) bank135501 +135502 POINT(41.608124155805235 -87.12082930007455) bank135502 +135503 POINT(42.00901809391884 -88.30698746659851) bank135503 +135504 POINT(41.21678127001525 -88.47791063085796) bank135504 +135505 POINT(41.474856937320546 -88.13828513642328) bank135505 +135506 POINT(42.12506676297134 -87.57968151286603) bank135506 +135507 POINT(41.78210557296252 -87.78984416239463) bank135507 +135508 POINT(42.40947992183778 -86.94131241202425) bank135508 +135509 POINT(41.24944457614045 -87.70377117577787) bank135509 +135510 POINT(42.4145219442185 -87.81732126782894) bank135510 +135511 POINT(42.39567449226074 -87.44855885430002) bank135511 +135512 POINT(42.104126670564135 -87.17437862664552) bank135512 +135513 POINT(42.15614696397458 -87.50750669509745) bank135513 +135514 POINT(41.006060570405516 -87.28007493983016) bank135514 +135515 POINT(42.27226961558693 -88.20571253679925) bank135515 +135516 POINT(42.56124959738183 -87.69896127368524) bank135516 +135517 POINT(41.9353429627884 -87.65194883199862) bank135517 +135518 POINT(40.97195842436422 -87.27556382875514) bank135518 +135519 POINT(41.34519577501941 -86.85977027029536) bank135519 +135520 POINT(40.98953835315255 -86.83722643954022) bank135520 +135521 POINT(42.348825589396526 -87.42179135980396) bank135521 +135522 POINT(41.30999484611749 -88.17536544243279) bank135522 +135523 POINT(41.345076974215445 -88.06941389303981) bank135523 +135524 POINT(41.292273163049735 -86.95423978202697) bank135524 +135525 POINT(41.85366511285933 -87.19451085686258) bank135525 +135526 POINT(42.59717743140572 -88.25228431704754) bank135526 +135527 POINT(41.61790624953921 -86.86200689049) bank135527 +135528 POINT(42.16780134365124 -86.66638022418961) bank135528 +135529 POINT(41.03407685442086 -87.27122505574883) bank135529 +135530 POINT(42.24885883391909 -86.80017466853704) bank135530 +135531 POINT(42.61975852896787 -87.82652141515179) bank135531 +135532 POINT(41.44441288098557 -88.42678977150747) bank135532 +135533 POINT(41.85694009882296 -88.18114040961913) bank135533 +135534 POINT(42.56559075259298 -87.71259318180613) bank135534 +135535 POINT(42.47508043922163 -87.03537010072252) bank135535 +135536 POINT(41.85467770836142 -87.36424967567781) bank135536 +135537 POINT(42.42797192180842 -86.69125475196172) bank135537 +135538 POINT(42.17008492182053 -87.50738923632102) bank135538 +135539 POINT(42.402673613077006 -88.30743220512394) bank135539 +135540 POINT(41.511472601392114 -87.00167844460168) bank135540 +135541 POINT(42.294105898134035 -87.19982871621747) bank135541 +135542 POINT(41.67825153183121 -88.46133337781495) bank135542 +135543 POINT(41.60885666797531 -86.92579607155429) bank135543 +135544 POINT(42.608171912184716 -87.53724873785869) bank135544 +135545 POINT(41.28973990234904 -88.4307145444915) bank135545 +135546 POINT(42.800233878813835 -87.31242918654243) bank135546 +135547 POINT(41.81317279680778 -88.51097437335062) bank135547 +135548 POINT(40.8930429952863 -86.75728693832822) bank135548 +135549 POINT(41.939872707717086 -86.9251816912565) bank135549 +135550 POINT(42.3760423508405 -87.19401699884325) bank135550 +135551 POINT(42.4593493726195 -87.22234143263235) bank135551 +135552 POINT(41.48645975400499 -87.3900687356951) bank135552 +135553 POINT(41.84438966553562 -87.32891116434654) bank135553 +135554 POINT(41.021827121482914 -87.05579846353153) bank135554 +135555 POINT(40.89621761553451 -87.30340486618661) bank135555 +135556 POINT(41.99989559842201 -87.06919540734336) bank135556 +135557 POINT(40.997028590509196 -86.82537639598546) bank135557 +135558 POINT(41.89378660555285 -87.68870271786214) bank135558 +135559 POINT(41.54237824702949 -87.49460795002943) bank135559 +135560 POINT(41.23557564719801 -88.44767835029106) bank135560 +135561 POINT(41.89573609862971 -88.61206443435147) bank135561 +135562 POINT(42.182163619107506 -86.78121299578281) bank135562 +135563 POINT(41.37592860161613 -87.60314372311966) bank135563 +135564 POINT(41.1685793337069 -87.69546137894622) bank135564 +135565 POINT(41.67314694081628 -87.85476162275141) bank135565 +135566 POINT(42.75796154974651 -86.99997178227987) bank135566 +135567 POINT(41.35907847144793 -88.36631760416913) bank135567 +135568 POINT(41.86154561160507 -88.33778719359064) bank135568 +135569 POINT(41.187454946436965 -87.36884010496905) bank135569 +135570 POINT(42.81700064125061 -87.47788760002045) bank135570 +135571 POINT(41.05948526731035 -86.64953396832374) bank135571 +135572 POINT(42.723089643941215 -87.14372134583529) bank135572 +135573 POINT(41.0097303704265 -88.09475178553024) bank135573 +135574 POINT(41.34551950458967 -88.2154682500484) bank135574 +135575 POINT(42.0421914137945 -87.92486577999924) bank135575 +135576 POINT(40.99712247016086 -88.10436608427356) bank135576 +135577 POINT(42.230978085616535 -88.5705128665752) bank135577 +135578 POINT(42.393274416082505 -88.57887659842737) bank135578 +135579 POINT(41.05746538102684 -87.45908794237789) bank135579 +135580 POINT(42.26591817065924 -87.11210203784373) bank135580 +135581 POINT(42.341351676190634 -87.75073402534792) bank135581 +135582 POINT(42.65082827434038 -88.5689600806176) bank135582 +135583 POINT(42.77616731746584 -88.5718912741398) bank135583 +135584 POINT(41.19612796077221 -88.45029527737064) bank135584 +135585 POINT(42.39511983833731 -88.20065142155552) bank135585 +135586 POINT(41.14241150794874 -88.18882568164375) bank135586 +135587 POINT(42.6713122027658 -87.44569566758501) bank135587 +135588 POINT(41.08897054541299 -86.88428653208511) bank135588 +135589 POINT(41.62328769405413 -88.61212831027792) bank135589 +135590 POINT(41.67367823388964 -87.54092087558926) bank135590 +135591 POINT(41.06202437811237 -87.03934373374759) bank135591 +135592 POINT(41.932696052511346 -88.01045834503122) bank135592 +135593 POINT(41.70664204369914 -88.09027582465662) bank135593 +135594 POINT(41.86230808493709 -88.37301373288508) bank135594 +135595 POINT(42.51609374137646 -87.29078137091206) bank135595 +135596 POINT(41.43196821990069 -87.39120818580164) bank135596 +135597 POINT(42.08103409305807 -87.87795926268471) bank135597 +135598 POINT(42.356311051092554 -88.45492059923329) bank135598 +135599 POINT(41.32082189763777 -88.12524504951533) bank135599 +135600 POINT(41.52758944707376 -87.85383938399256) bank135600 +135601 POINT(41.827544636896675 -88.60065917392333) bank135601 +135602 POINT(42.381236150073164 -88.14598032733484) bank135602 +135603 POINT(42.1156717713649 -87.20801047233859) bank135603 +135604 POINT(41.511908114982475 -87.01416665594331) bank135604 +135605 POINT(41.24005831656762 -87.41029324636732) bank135605 +135606 POINT(41.70414455222735 -86.88555010462011) bank135606 +135607 POINT(41.279093799790076 -88.2469295451053) bank135607 +135608 POINT(41.4280043277312 -88.01067393018934) bank135608 +135609 POINT(42.73956770573791 -88.11752749063014) bank135609 +135610 POINT(41.89990656770555 -87.13946755362538) bank135610 +135611 POINT(42.87574653701678 -86.73272351634371) bank135611 +135612 POINT(41.20306546671888 -88.30209473888314) bank135612 +135613 POINT(41.67701401979295 -88.60953770439485) bank135613 +135614 POINT(42.178246895463005 -86.84716269747865) bank135614 +135615 POINT(42.16283436969806 -87.64416419588646) bank135615 +135616 POINT(41.62900326461755 -88.16254274286354) bank135616 +135617 POINT(41.41713499760874 -88.28591009143679) bank135617 +135618 POINT(40.94923172397104 -88.19804789597063) bank135618 +135619 POINT(41.40405719925417 -88.08887418438051) bank135619 +135620 POINT(41.792626769269965 -88.19725167642511) bank135620 +135621 POINT(41.33174984889227 -88.2455732636678) bank135621 +135622 POINT(42.14641546458442 -86.66100620368712) bank135622 +135623 POINT(42.86171103469075 -87.95505673932087) bank135623 +135624 POINT(42.32031729561473 -88.38219809500406) bank135624 +135625 POINT(41.65071489499144 -88.60712057846352) bank135625 +135626 POINT(42.68931243643865 -87.02348231458976) bank135626 +135627 POINT(41.986474787222896 -86.94568418182044) bank135627 +135628 POINT(41.868057814858375 -87.50914519150649) bank135628 +135629 POINT(41.9822741124699 -87.13245244058577) bank135629 +135630 POINT(41.22669274105673 -88.59021646723984) bank135630 +135631 POINT(42.48770118030596 -88.15518721612234) bank135631 +135632 POINT(41.47334122803214 -87.99132570959402) bank135632 +135633 POINT(42.70601329033052 -87.30494936464797) bank135633 +135634 POINT(41.985626904405684 -86.9367721038007) bank135634 +135635 POINT(42.64776244938716 -88.04707478851803) bank135635 +135636 POINT(41.35185608393021 -88.12614501480792) bank135636 +135637 POINT(42.68099031359489 -87.68213310514354) bank135637 +135638 POINT(41.42496868957439 -87.79697561487166) bank135638 +135639 POINT(41.8287482431699 -87.8758924164346) bank135639 +135640 POINT(41.484656030111935 -87.51603148014901) bank135640 +135641 POINT(41.9980440214738 -87.64948497913267) bank135641 +135642 POINT(41.218032415868116 -87.17329387602429) bank135642 +135643 POINT(41.478371027202066 -87.58944736791382) bank135643 +135644 POINT(42.15415351004766 -87.22699614968246) bank135644 +135645 POINT(41.93517920156457 -88.00696223062891) bank135645 +135646 POINT(42.78414791641988 -87.89585510174709) bank135646 +135647 POINT(41.199139003634244 -87.56189591211394) bank135647 +135648 POINT(42.44970666852075 -87.00450654210272) bank135648 +135649 POINT(42.8714116212841 -88.22055852289954) bank135649 +135650 POINT(41.06885436656908 -87.47942960973977) bank135650 +135651 POINT(41.00099644931844 -88.41002380403958) bank135651 +135652 POINT(41.48164795234674 -88.09436851349383) bank135652 +135653 POINT(42.811720376249546 -87.49781522804848) bank135653 +135654 POINT(42.74430135654267 -87.39040359246029) bank135654 +135655 POINT(42.324003104756926 -88.17589251304946) bank135655 +135656 POINT(41.607353201360624 -86.65146163641043) bank135656 +135657 POINT(41.75424782934528 -88.49479352011434) bank135657 +135658 POINT(41.260013184593845 -88.62296242489413) bank135658 +135659 POINT(42.680776768348956 -88.26158914917536) bank135659 +135660 POINT(41.159917667514655 -87.30791394497575) bank135660 +135661 POINT(42.13527895706659 -88.22012482622955) bank135661 +135662 POINT(42.757141327853816 -87.40088729060159) bank135662 +135663 POINT(41.06858604689545 -87.45477077937862) bank135663 +135664 POINT(41.78523590839458 -87.54857306045916) bank135664 +135665 POINT(41.20684671178954 -87.81826681808079) bank135665 +135666 POINT(41.921033546422194 -88.05227633872167) bank135666 +135667 POINT(42.14314483437652 -87.95897287877123) bank135667 +135668 POINT(42.724121242849606 -87.82666037219865) bank135668 +135669 POINT(41.25641974594885 -88.21061243699921) bank135669 +135670 POINT(41.08173468626498 -87.54142066596867) bank135670 +135671 POINT(42.1901818410904 -88.1165786126485) bank135671 +135672 POINT(41.77807797213428 -87.66748715398548) bank135672 +135673 POINT(41.97572983542731 -87.33315094145117) bank135673 +135674 POINT(41.60786242030999 -87.90746739745987) bank135674 +135675 POINT(41.94932415319077 -88.10261426481829) bank135675 +135676 POINT(42.78445402659175 -87.74848133633856) bank135676 +135677 POINT(42.36183043508163 -87.42506589156451) bank135677 +135678 POINT(42.283876872716554 -86.77322202903773) bank135678 +135679 POINT(41.85338794399317 -88.0268172695789) bank135679 +135680 POINT(42.366783234946254 -86.90267032780395) bank135680 +135681 POINT(41.64853064964559 -87.06129017870272) bank135681 +135682 POINT(41.621189039723966 -87.20715026319932) bank135682 +135683 POINT(42.18261525216696 -88.45097217420192) bank135683 +135684 POINT(41.79298806325021 -87.31215521149421) bank135684 +135685 POINT(42.0244037665976 -88.19726044493584) bank135685 +135686 POINT(41.524516182221596 -88.35239697639551) bank135686 +135687 POINT(42.25547773903907 -87.56377836035182) bank135687 +135688 POINT(41.88610751932399 -87.56576542281314) bank135688 +135689 POINT(41.98318802218758 -87.95661185587231) bank135689 +135690 POINT(41.21650571547591 -87.96389533108857) bank135690 +135691 POINT(42.36365479468954 -87.73553756852168) bank135691 +135692 POINT(41.64677793506061 -87.29471811365238) bank135692 +135693 POINT(41.619900761193385 -88.29068676150848) bank135693 +135694 POINT(41.62155707125614 -86.683604886173) bank135694 +135695 POINT(42.18228054869853 -86.9188436837092) bank135695 +135696 POINT(41.567727804723106 -87.48117288356237) bank135696 +135697 POINT(41.475957603721625 -86.81330034578701) bank135697 +135698 POINT(41.34365555995535 -86.71494583985049) bank135698 +135699 POINT(41.93534384611046 -88.40894262799523) bank135699 +135700 POINT(42.459323066446 -86.87192826622393) bank135700 +135701 POINT(42.19477071459657 -88.2159479441606) bank135701 +135702 POINT(41.9527520418949 -88.32847787822675) bank135702 +135703 POINT(41.43677494435283 -88.25970464817287) bank135703 +135704 POINT(42.1137775746423 -87.55042125363948) bank135704 +135705 POINT(42.35506343983902 -88.53470998037386) bank135705 +135706 POINT(42.64261134167711 -88.47230154793174) bank135706 +135707 POINT(42.84773500055744 -87.44642928241882) bank135707 +135708 POINT(42.59313082488871 -88.62289111492784) bank135708 +135709 POINT(41.65196301942892 -87.34287787334554) bank135709 +135710 POINT(41.89045839706307 -86.99889478769758) bank135710 +135711 POINT(41.16931057929204 -87.80448487270638) bank135711 +135712 POINT(41.97905948951133 -88.62439469030345) bank135712 +135713 POINT(41.48466650005854 -87.64398838206559) bank135713 +135714 POINT(42.71172047908508 -87.1046645580242) bank135714 +135715 POINT(41.44116231961676 -88.59272773612776) bank135715 +135716 POINT(41.2413811382002 -86.94375174547181) bank135716 +135717 POINT(41.75025828584726 -87.08616369238976) bank135717 +135718 POINT(42.00488190696491 -88.60661537863528) bank135718 +135719 POINT(41.0140705571383 -87.28456501986837) bank135719 +135720 POINT(41.06047362774876 -87.67252950668309) bank135720 +135721 POINT(42.272290890645856 -87.22828545905527) bank135721 +135722 POINT(42.576354379612255 -88.2693345855166) bank135722 +135723 POINT(42.82693773612071 -87.56574426545018) bank135723 +135724 POINT(41.18834866299194 -88.34153463300353) bank135724 +135725 POINT(41.43087503483852 -87.03951455661486) bank135725 +135726 POINT(41.14464671696514 -87.01139780747772) bank135726 +135727 POINT(42.449856120707864 -87.40551215332553) bank135727 +135728 POINT(42.20938935295897 -87.11994561189901) bank135728 +135729 POINT(41.647099826599415 -87.84300671749394) bank135729 +135730 POINT(41.52488004119096 -87.75220317487413) bank135730 +135731 POINT(42.57851274173945 -87.19956560312558) bank135731 +135732 POINT(41.40907813851014 -88.18973640438828) bank135732 +135733 POINT(41.29408597066832 -87.06390896913915) bank135733 +135734 POINT(42.35393508020798 -87.4999832967045) bank135734 +135735 POINT(42.028592178614296 -87.20231847484762) bank135735 +135736 POINT(41.646850397960485 -87.4271213441835) bank135736 +135737 POINT(42.1397594407285 -87.10848055273514) bank135737 +135738 POINT(41.0154485086593 -87.06543003578851) bank135738 +135739 POINT(41.61442604871903 -87.04876033331696) bank135739 +135740 POINT(42.643943702729075 -88.40809875253316) bank135740 +135741 POINT(42.25096128264678 -87.77471707149066) bank135741 +135742 POINT(42.149729594713754 -88.39095187071156) bank135742 +135743 POINT(41.249271555911044 -88.37133302026909) bank135743 +135744 POINT(42.452069485216256 -87.88116828413575) bank135744 +135745 POINT(42.849124503163516 -86.73329749816978) bank135745 +135746 POINT(41.33170223520875 -88.25492892891103) bank135746 +135747 POINT(42.297618140907844 -87.75390916323816) bank135747 +135748 POINT(41.436775312425034 -88.21648499110353) bank135748 +135749 POINT(41.37341853990808 -87.22187253390679) bank135749 +135750 POINT(41.937209834401735 -86.91728530962642) bank135750 +135751 POINT(42.63326568870589 -87.8241333818395) bank135751 +135752 POINT(42.75528704347084 -87.34353749674386) bank135752 +135753 POINT(42.266882112775654 -87.24254884371862) bank135753 +135754 POINT(42.52907948240856 -88.42946880621032) bank135754 +135755 POINT(41.316374671260114 -86.68570328797207) bank135755 +135756 POINT(41.783539369204 -86.92764611635565) bank135756 +135757 POINT(41.35949045060983 -87.4567126270182) bank135757 +135758 POINT(42.19233337311 -87.38119156443989) bank135758 +135759 POINT(40.999893625799885 -87.20687529717196) bank135759 +135760 POINT(41.224288571037945 -88.56090765758314) bank135760 +135761 POINT(42.27202315574747 -87.89324585901106) bank135761 +135762 POINT(42.4860974335267 -88.45278894777802) bank135762 +135763 POINT(41.29188950917123 -88.21824501256602) bank135763 +135764 POINT(42.74372703475975 -87.47970740381142) bank135764 +135765 POINT(41.684649999114804 -86.67756559398677) bank135765 +135766 POINT(42.44821105202202 -88.27284005525547) bank135766 +135767 POINT(41.54062101029365 -87.32359092541117) bank135767 +135768 POINT(41.41340265469004 -88.30710828593126) bank135768 +135769 POINT(42.76310270968714 -87.59601267771868) bank135769 +135770 POINT(42.358636837850796 -87.46453512656356) bank135770 +135771 POINT(42.31786931396362 -86.87793375102764) bank135771 +135772 POINT(41.90561763430276 -88.4344518835905) bank135772 +135773 POINT(42.74693920303261 -86.76576327164616) bank135773 +135774 POINT(40.92233927300268 -88.57385656823479) bank135774 +135775 POINT(42.10976767649569 -86.84680249380288) bank135775 +135776 POINT(41.2102402650331 -87.2550555554062) bank135776 +135777 POINT(41.00047489487449 -88.06529352583648) bank135777 +135778 POINT(41.46232002458058 -87.47346462201186) bank135778 +135779 POINT(41.82090184914374 -88.48257257313064) bank135779 +135780 POINT(42.29223112802739 -87.07857224438965) bank135780 +135781 POINT(41.50876247937234 -87.19329969120486) bank135781 +135782 POINT(41.15745510573764 -86.97075115702835) bank135782 +135783 POINT(42.63319269882429 -87.27316658480296) bank135783 +135784 POINT(42.74367682251046 -87.73253130055463) bank135784 +135785 POINT(41.6490986507983 -87.2709827972529) bank135785 +135786 POINT(41.21090438782084 -86.70789890783874) bank135786 +135787 POINT(41.61558989451302 -86.9914503236511) bank135787 +135788 POINT(40.92921046468822 -87.08060883634921) bank135788 +135789 POINT(42.68516771162013 -88.62290567641361) bank135789 +135790 POINT(41.813788772135304 -87.29977192685286) bank135790 +135791 POINT(42.04748821610846 -88.01767245613189) bank135791 +135792 POINT(41.56715909265927 -88.14736678115943) bank135792 +135793 POINT(41.07774269988741 -87.45922830847122) bank135793 +135794 POINT(42.83082202303224 -87.10394091807753) bank135794 +135795 POINT(41.12171749971283 -87.14190150475291) bank135795 +135796 POINT(41.3024087866006 -87.57422462560004) bank135796 +135797 POINT(41.75242606465456 -87.12980632149296) bank135797 +135798 POINT(42.309057752682165 -87.59494277970427) bank135798 +135799 POINT(42.65194576721421 -88.42725374470997) bank135799 +135800 POINT(42.710629815351204 -88.16390637555385) bank135800 +135801 POINT(42.06093959913433 -88.15704277951959) bank135801 +135802 POINT(42.69504792225903 -86.63845447153314) bank135802 +135803 POINT(41.68632633971039 -88.06344457424397) bank135803 +135804 POINT(41.01047539095403 -86.93581990006383) bank135804 +135805 POINT(41.28434746053458 -86.73362655070831) bank135805 +135806 POINT(42.81196935133243 -86.63239199322797) bank135806 +135807 POINT(41.487622621162394 -86.72617043796807) bank135807 +135808 POINT(42.33679519160029 -87.677590504784) bank135808 +135809 POINT(42.17719873010744 -86.83080009318586) bank135809 +135810 POINT(41.22626109624529 -88.19393374942803) bank135810 +135811 POINT(42.22507536928186 -87.78601996636235) bank135811 +135812 POINT(42.02802599096648 -86.77525000939694) bank135812 +135813 POINT(40.941379891309076 -87.56956494207704) bank135813 +135814 POINT(41.07125774428608 -86.80849343633909) bank135814 +135815 POINT(41.894780978503505 -88.25084051217799) bank135815 +135816 POINT(41.91378451315828 -87.9062338480554) bank135816 +135817 POINT(42.801899106748166 -88.56202286885663) bank135817 +135818 POINT(41.86447978752197 -87.8737649574396) bank135818 +135819 POINT(41.16464802691332 -87.07153874123854) bank135819 +135820 POINT(42.395175536040675 -86.66267663562525) bank135820 +135821 POINT(42.246858716858796 -86.87418664555418) bank135821 +135822 POINT(42.09908140563535 -88.49487888963858) bank135822 +135823 POINT(42.32671277269141 -88.59011765546374) bank135823 +135824 POINT(41.24533244806006 -87.8760630104225) bank135824 +135825 POINT(42.74081838485511 -87.21889086238872) bank135825 +135826 POINT(41.51915806186433 -88.07201583567092) bank135826 +135827 POINT(42.45116465981714 -88.52348778154911) bank135827 +135828 POINT(41.36138062726748 -87.64765151276987) bank135828 +135829 POINT(41.2158402806752 -86.96529852603233) bank135829 +135830 POINT(41.81019590453886 -87.26397154327178) bank135830 +135831 POINT(41.114326280034895 -87.59579389275947) bank135831 +135832 POINT(41.029853771085214 -88.52979992377543) bank135832 +135833 POINT(42.63232376453847 -88.11183670507103) bank135833 +135834 POINT(42.84945440019515 -87.25617589793534) bank135834 +135835 POINT(42.814856135519655 -87.2572252208949) bank135835 +135836 POINT(41.89235791756563 -88.34884925414985) bank135836 +135837 POINT(42.68924391444678 -87.68153248164491) bank135837 +135838 POINT(41.697539812221166 -87.07568304047241) bank135838 +135839 POINT(42.83099490030289 -87.90747870351623) bank135839 +135840 POINT(41.16068941311716 -88.23144033789828) bank135840 +135841 POINT(42.682483687859055 -88.37641850250716) bank135841 +135842 POINT(41.94487963409355 -88.53164197265237) bank135842 +135843 POINT(42.65960853870166 -88.53138709490185) bank135843 +135844 POINT(42.85672912814942 -88.05564954487379) bank135844 +135845 POINT(41.7935179201547 -86.84684021133553) bank135845 +135846 POINT(41.57356863516439 -87.07327335054947) bank135846 +135847 POINT(41.93426051741774 -86.79332882435496) bank135847 +135848 POINT(40.96873733934432 -88.4451703581019) bank135848 +135849 POINT(42.47949367549513 -87.74033957343829) bank135849 +135850 POINT(41.5396210503682 -86.92848090215325) bank135850 +135851 POINT(42.46748904147121 -88.40768585417798) bank135851 +135852 POINT(40.977539710726916 -86.71776978868547) bank135852 +135853 POINT(42.81527372727294 -88.48628758517167) bank135853 +135854 POINT(41.62159037342462 -87.20587270429372) bank135854 +135855 POINT(41.0247586257572 -87.7416457992033) bank135855 +135856 POINT(42.089870945512416 -87.52951617548442) bank135856 +135857 POINT(41.633854638922465 -87.22300032300944) bank135857 +135858 POINT(42.597985098911295 -86.89393455084368) bank135858 +135859 POINT(42.178343928171174 -87.86550878849681) bank135859 +135860 POINT(42.19174558940155 -86.66946987444955) bank135860 +135861 POINT(41.46706714809905 -87.57720459204342) bank135861 +135862 POINT(42.87532620665221 -87.83137370689609) bank135862 +135863 POINT(41.1103598952486 -86.87101617248298) bank135863 +135864 POINT(41.09745826148286 -87.73559223225928) bank135864 +135865 POINT(41.22980689673419 -86.84353655698787) bank135865 +135866 POINT(42.29549150522252 -88.61790402370131) bank135866 +135867 POINT(42.78570652283801 -88.06593253643057) bank135867 +135868 POINT(42.81962831593439 -88.53712184531665) bank135868 +135869 POINT(41.39451138657881 -87.52340548477432) bank135869 +135870 POINT(41.43023862553517 -86.838651076666) bank135870 +135871 POINT(42.6202529843392 -88.53755124666985) bank135871 +135872 POINT(42.215535739227754 -87.59429434123425) bank135872 +135873 POINT(41.83468925267864 -88.40933161834273) bank135873 +135874 POINT(42.28413223922836 -87.26147173036766) bank135874 +135875 POINT(41.50981844103318 -86.846099255969) bank135875 +135876 POINT(42.11458031623859 -88.04707398544258) bank135876 +135877 POINT(42.032912437230515 -88.10117492506447) bank135877 +135878 POINT(42.297607275301445 -88.075800057594) bank135878 +135879 POINT(41.66276242832422 -88.19660321793246) bank135879 +135880 POINT(40.914794517815324 -87.4496192064133) bank135880 +135881 POINT(42.07812788946275 -87.84095884040914) bank135881 +135882 POINT(42.182671174356265 -86.83092791130854) bank135882 +135883 POINT(41.49111725525216 -87.7488055990564) bank135883 +135884 POINT(42.83253133951197 -86.8761207460575) bank135884 +135885 POINT(41.27443294070396 -87.55331834427793) bank135885 +135886 POINT(41.90652483006307 -87.57284052211061) bank135886 +135887 POINT(41.91055069573319 -88.46918069128036) bank135887 +135888 POINT(41.329843533214174 -87.64284665972316) bank135888 +135889 POINT(42.06417811722624 -87.36815261244601) bank135889 +135890 POINT(41.235117226674085 -88.29866198322976) bank135890 +135891 POINT(41.11840311203372 -88.61595465446487) bank135891 +135892 POINT(42.874092557813995 -87.51680464311178) bank135892 +135893 POINT(42.81720206749265 -87.17652855443683) bank135893 +135894 POINT(42.2835750285082 -88.60333719536816) bank135894 +135895 POINT(42.0440088222948 -88.58002754087671) bank135895 +135896 POINT(42.08923268348994 -87.630999584083) bank135896 +135897 POINT(41.78933413712485 -87.54523451190103) bank135897 +135898 POINT(42.49006723509958 -88.28831043080875) bank135898 +135899 POINT(42.07734988515279 -88.51590273017119) bank135899 +135900 POINT(41.842680323711576 -87.45316495831827) bank135900 +135901 POINT(42.64450887974993 -88.43444442656696) bank135901 +135902 POINT(42.59260769931098 -87.69082083505646) bank135902 +135903 POINT(42.30138667021117 -87.11108997032962) bank135903 +135904 POINT(42.571267942630115 -88.22572841763576) bank135904 +135905 POINT(41.18387060877005 -88.59674990595552) bank135905 +135906 POINT(41.587856982138774 -88.18621042226806) bank135906 +135907 POINT(41.04325561172306 -87.82202357609127) bank135907 +135908 POINT(41.85843041978945 -86.88523732453422) bank135908 +135909 POINT(42.713335462274166 -86.91353228784618) bank135909 +135910 POINT(41.918896534068956 -87.01203902030494) bank135910 +135911 POINT(41.418542563706325 -88.2677480246998) bank135911 +135912 POINT(42.618153358791844 -86.84013485030657) bank135912 +135913 POINT(42.12148648398942 -87.37849760868045) bank135913 +135914 POINT(41.95806934161603 -87.51008322323017) bank135914 +135915 POINT(42.54948663061698 -88.49393949901769) bank135915 +135916 POINT(42.162133515205475 -87.75143258488892) bank135916 +135917 POINT(42.726451178184114 -87.0077756134764) bank135917 +135918 POINT(41.15763341005351 -87.46134580862116) bank135918 +135919 POINT(41.26611696729461 -86.79099960222858) bank135919 +135920 POINT(41.08355341386525 -87.35787207815038) bank135920 +135921 POINT(41.58205035021483 -87.27358366909178) bank135921 +135922 POINT(41.09194010560266 -87.5909826824629) bank135922 +135923 POINT(42.59145405807959 -87.73332139691317) bank135923 +135924 POINT(42.29070783471652 -87.10407218593971) bank135924 +135925 POINT(42.56683303498795 -87.93970602210682) bank135925 +135926 POINT(42.186898093092616 -87.74818454227533) bank135926 +135927 POINT(41.63139998439805 -87.325450664414) bank135927 +135928 POINT(41.46880547812561 -86.777662177268) bank135928 +135929 POINT(41.65127315269014 -88.45841253027629) bank135929 +135930 POINT(41.88559377384889 -87.82161446572961) bank135930 +135931 POINT(41.69038541955011 -88.17091666091127) bank135931 +135932 POINT(41.787787968257945 -87.4776176279709) bank135932 +135933 POINT(41.390814512497876 -87.85095702784072) bank135933 +135934 POINT(42.832008224577635 -86.65789866666103) bank135934 +135935 POINT(41.13478202689827 -87.97896442447566) bank135935 +135936 POINT(42.2687782777693 -87.6757585357108) bank135936 +135937 POINT(41.85972868783007 -87.21833201811161) bank135937 +135938 POINT(41.25377941908104 -88.17563742203828) bank135938 +135939 POINT(41.08144740138713 -88.23661721873616) bank135939 +135940 POINT(42.41552642451226 -87.15431236969303) bank135940 +135941 POINT(41.48404682870556 -87.36152356817284) bank135941 +135942 POINT(41.42712175486296 -87.75160204350433) bank135942 +135943 POINT(42.17204315314145 -86.97998415882884) bank135943 +135944 POINT(42.28742624534824 -88.36914513502967) bank135944 +135945 POINT(41.67337413153845 -88.22644521407383) bank135945 +135946 POINT(41.10595369636395 -87.72747815588275) bank135946 +135947 POINT(42.74027104413325 -88.19662621134837) bank135947 +135948 POINT(41.88598264977073 -88.25936183528617) bank135948 +135949 POINT(41.182555294917236 -87.33695270886679) bank135949 +135950 POINT(41.64690691533487 -87.48313524112662) bank135950 +135951 POINT(42.169451138622144 -88.58009220744411) bank135951 +135952 POINT(42.34580991816765 -87.65534206121595) bank135952 +135953 POINT(41.53420349264313 -87.16957215180555) bank135953 +135954 POINT(42.62373301814878 -87.74425372044647) bank135954 +135955 POINT(42.79516792057088 -88.12183038345717) bank135955 +135956 POINT(40.91198657843108 -88.39158463611815) bank135956 +135957 POINT(40.977689834210224 -88.6159216125085) bank135957 +135958 POINT(41.84469839060666 -86.80943456906745) bank135958 +135959 POINT(42.644534579019485 -87.29631375674924) bank135959 +135960 POINT(41.335323441263895 -88.31577925076495) bank135960 +135961 POINT(41.170713479823384 -86.72507254343402) bank135961 +135962 POINT(41.92802757236775 -88.09745011647841) bank135962 +135963 POINT(41.550714095728146 -88.24373998288887) bank135963 +135964 POINT(41.33262047518279 -87.16282776702198) bank135964 +135965 POINT(42.15912646143157 -87.90770786370267) bank135965 +135966 POINT(42.68724504550982 -87.12668512958905) bank135966 +135967 POINT(41.125702306699324 -86.6540228510234) bank135967 +135968 POINT(41.998359734682836 -86.73551520247004) bank135968 +135969 POINT(41.236136796927255 -88.32849210802091) bank135969 +135970 POINT(41.94620024058558 -87.38016156298531) bank135970 +135971 POINT(41.26981130133446 -88.51786215646572) bank135971 +135972 POINT(41.37238130195691 -87.19119612890319) bank135972 +135973 POINT(42.195062352951275 -87.54520986196442) bank135973 +135974 POINT(42.55659982823196 -87.93828038887987) bank135974 +135975 POINT(41.20853729939941 -86.69323644399775) bank135975 +135976 POINT(42.17847023118927 -86.94356604100643) bank135976 +135977 POINT(42.863010640687676 -88.33541086601828) bank135977 +135978 POINT(41.13878105606807 -87.11193533001834) bank135978 +135979 POINT(41.283804155075 -87.08634162814106) bank135979 +135980 POINT(42.63949769041605 -87.88168755389722) bank135980 +135981 POINT(41.04051125077213 -87.43630872286775) bank135981 +135982 POINT(41.97640565190172 -88.31355045057916) bank135982 +135983 POINT(42.86619831747162 -87.04379966204245) bank135983 +135984 POINT(40.929442872569396 -86.85211190068935) bank135984 +135985 POINT(40.967689651094155 -87.93087734060703) bank135985 +135986 POINT(42.077125370590686 -86.80601496325157) bank135986 +135987 POINT(41.13994827893542 -88.27175631500171) bank135987 +135988 POINT(40.93900827959312 -88.00812422751939) bank135988 +135989 POINT(42.06687968090791 -87.03638727295132) bank135989 +135990 POINT(42.431487828684446 -87.90842423543799) bank135990 +135991 POINT(41.2030315565211 -87.98213166088362) bank135991 +135992 POINT(41.03741166764978 -88.50666816842099) bank135992 +135993 POINT(41.73366380423526 -88.5084832252934) bank135993 +135994 POINT(40.95400550670635 -88.10051329480333) bank135994 +135995 POINT(41.57267614215409 -88.06978695095964) bank135995 +135996 POINT(42.291897778157036 -87.65436356872978) bank135996 +135997 POINT(42.35081235520172 -87.89927588187948) bank135997 +135998 POINT(41.21975343998847 -87.58662209216507) bank135998 +135999 POINT(42.436324731519065 -88.42316539888445) bank135999 +136000 POINT(42.46082462862072 -87.69392491910274) bank136000 +136001 POINT(41.97961838409326 -87.30725520945768) bank136001 +136002 POINT(42.52378461916817 -87.18161576801192) bank136002 +136003 POINT(42.1974195376173 -87.52788443075163) bank136003 +136004 POINT(41.70320461149442 -88.06619999212688) bank136004 +136005 POINT(41.24736675626393 -87.89556996558593) bank136005 +136006 POINT(42.082378034207004 -88.16190339607262) bank136006 +136007 POINT(42.5575254733872 -86.98771778946431) bank136007 +136008 POINT(41.40582324819884 -87.9409538301017) bank136008 +136009 POINT(42.79807084235868 -86.87772832654785) bank136009 +136010 POINT(42.781952313850354 -87.69980902422276) bank136010 +136011 POINT(42.45219976163797 -88.02373904905335) bank136011 +136012 POINT(40.98885015421674 -86.7076689367293) bank136012 +136013 POINT(41.584189985202165 -87.83354997699058) bank136013 +136014 POINT(41.75939955714428 -86.89201303949656) bank136014 +136015 POINT(42.23529481812768 -88.071993158332) bank136015 +136016 POINT(41.15267811338553 -88.44003653361149) bank136016 +136017 POINT(41.77795155703132 -87.55228347629087) bank136017 +136018 POINT(42.12346377944431 -86.65599380421736) bank136018 +136019 POINT(41.95562016744308 -87.59534628663637) bank136019 +136020 POINT(41.11179615371511 -87.47762802467727) bank136020 +136021 POINT(42.27706206272366 -87.07867650356879) bank136021 +136022 POINT(41.046636977183134 -86.71472387836715) bank136022 +136023 POINT(42.21263737086391 -88.56532747657465) bank136023 +136024 POINT(41.81182916401862 -86.73978897622206) bank136024 +136025 POINT(41.86173302246725 -87.76716059982753) bank136025 +136026 POINT(40.910988102359276 -86.91196936574212) bank136026 +136027 POINT(40.88635440379656 -88.31521173713602) bank136027 +136028 POINT(41.58411067505998 -86.80152836807927) bank136028 +136029 POINT(41.20913332069444 -87.12102777298158) bank136029 +136030 POINT(41.619957761233934 -87.1818767172604) bank136030 +136031 POINT(40.997353703389365 -88.11407322891958) bank136031 +136032 POINT(41.00368522212093 -86.8982830104711) bank136032 +136033 POINT(41.19385201673728 -88.61522428969316) bank136033 +136034 POINT(42.00488266404754 -88.35971991747672) bank136034 +136035 POINT(41.10626443320308 -88.35290441452663) bank136035 +136036 POINT(41.13832302822834 -87.67434518262637) bank136036 +136037 POINT(41.945853192155724 -88.544123209125) bank136037 +136038 POINT(41.33682795621074 -86.7296248727215) bank136038 +136039 POINT(42.04631291002199 -87.05142319227659) bank136039 +136040 POINT(41.54461845497155 -87.5583287148347) bank136040 +136041 POINT(42.44137397487452 -88.33641600099199) bank136041 +136042 POINT(40.908041841034596 -86.65397346281192) bank136042 +136043 POINT(41.50628661311391 -87.02405520572307) bank136043 +136044 POINT(41.037404469747905 -86.73785933156559) bank136044 +136045 POINT(42.61631435009398 -88.30081431436042) bank136045 +136046 POINT(41.387847426438896 -87.42993058945795) bank136046 +136047 POINT(42.40384073037967 -88.41365017292927) bank136047 +136048 POINT(41.10319434387904 -87.86384157344203) bank136048 +136049 POINT(42.81399331093663 -88.57097109587139) bank136049 +136050 POINT(41.25430879263277 -87.69954207411863) bank136050 +136051 POINT(42.06100826768337 -86.69996841257712) bank136051 +136052 POINT(41.64738076143711 -87.92094695998988) bank136052 +136053 POINT(41.826945686038655 -87.03203648083965) bank136053 +136054 POINT(41.512265902509085 -88.0615390362302) bank136054 +136055 POINT(41.22253726091373 -86.87962113212751) bank136055 +136056 POINT(41.892053364684685 -87.7533587578269) bank136056 +136057 POINT(42.76251555605394 -88.4873727283859) bank136057 +136058 POINT(41.65080752487034 -87.28859043015643) bank136058 +136059 POINT(41.63172828064545 -87.50101552239241) bank136059 +136060 POINT(41.87780759130221 -88.248381892962) bank136060 +136061 POINT(41.08768093139578 -87.24373296626874) bank136061 +136062 POINT(42.854792605205375 -87.07962885388157) bank136062 +136063 POINT(42.21370391639679 -88.61151252025401) bank136063 +136064 POINT(40.909183214927985 -87.27403162459116) bank136064 +136065 POINT(41.07129485796651 -87.4637691945213) bank136065 +136066 POINT(42.05396418534505 -87.5883713359494) bank136066 +136067 POINT(41.54233219724418 -88.54377667049567) bank136067 +136068 POINT(41.679425402920764 -88.62808136485616) bank136068 +136069 POINT(42.30587773440991 -87.74162844210555) bank136069 +136070 POINT(41.05986623812475 -88.07951042283477) bank136070 +136071 POINT(42.30246025896699 -87.89901602659039) bank136071 +136072 POINT(41.07080414475629 -87.60831365388393) bank136072 +136073 POINT(41.759730536518816 -88.52491221642504) bank136073 +136074 POINT(42.42327795887765 -88.12097032493392) bank136074 +136075 POINT(41.120335896288054 -87.75312627823503) bank136075 +136076 POINT(42.02387125070962 -87.28434699209912) bank136076 +136077 POINT(42.227777095621164 -88.05625577587531) bank136077 +136078 POINT(41.70481183437147 -88.2139746064847) bank136078 +136079 POINT(41.07716265300335 -88.39701009895053) bank136079 +136080 POINT(42.22799548459412 -87.48054775857746) bank136080 +136081 POINT(42.48958321355569 -87.01107310671706) bank136081 +136082 POINT(42.40115453058137 -87.72259683028452) bank136082 +136083 POINT(41.06180392648433 -88.22522763807495) bank136083 +136084 POINT(42.50747880812105 -87.07712108996915) bank136084 +136085 POINT(41.37160850049374 -88.02792483994855) bank136085 +136086 POINT(42.32062700221071 -88.08357093367424) bank136086 +136087 POINT(42.2988936719527 -87.2871492071789) bank136087 +136088 POINT(41.63046532890518 -87.91488151804958) bank136088 +136089 POINT(42.335998025273206 -87.607509979246) bank136089 +136090 POINT(42.80637323090541 -88.29639492523171) bank136090 +136091 POINT(42.437873178044114 -88.13985735199839) bank136091 +136092 POINT(42.26901508529573 -87.84736961609673) bank136092 +136093 POINT(42.44340287287182 -87.27848693696109) bank136093 +136094 POINT(42.119340879076084 -86.92471647734685) bank136094 +136095 POINT(42.46293478424071 -87.3864218949357) bank136095 +136096 POINT(41.106026923476584 -88.56405768045512) bank136096 +136097 POINT(41.53550349433253 -88.53621206364646) bank136097 +136098 POINT(42.67098181037325 -87.72228506181396) bank136098 +136099 POINT(41.964622329586966 -88.11302656838309) bank136099 +136100 POINT(41.521910136326895 -87.46876217865625) bank136100 +136101 POINT(40.934008886529774 -87.5150915909722) bank136101 +136102 POINT(41.75361385104812 -87.64707303852752) bank136102 +136103 POINT(42.7158734717275 -87.62518546257799) bank136103 +136104 POINT(42.2123201416513 -86.78938935093463) bank136104 +136105 POINT(41.028843091272535 -86.93823285589686) bank136105 +136106 POINT(41.927788648298396 -87.8427472795265) bank136106 +136107 POINT(42.14404292567296 -86.89214919490482) bank136107 +136108 POINT(41.57357959603124 -88.09850565900689) bank136108 +136109 POINT(42.1927771396333 -87.56376244662097) bank136109 +136110 POINT(42.09563367453668 -88.17606695301711) bank136110 +136111 POINT(42.27831241580465 -87.75011287890854) bank136111 +136112 POINT(42.584203011220424 -87.29377339775948) bank136112 +136113 POINT(42.594978452029 -87.28799231924458) bank136113 +136114 POINT(42.07843025852722 -87.46318489497007) bank136114 +136115 POINT(41.732959020878916 -88.45279447817656) bank136115 +136116 POINT(41.34656551394995 -87.96128481711933) bank136116 +136117 POINT(41.89694243567024 -88.33429524093675) bank136117 +136118 POINT(42.27409287877626 -87.29372554620569) bank136118 +136119 POINT(41.677481365538505 -86.86902894044309) bank136119 +136120 POINT(41.82408759657986 -88.33872374175517) bank136120 +136121 POINT(41.13559717349713 -87.98743316527828) bank136121 +136122 POINT(42.65937848397027 -87.9847702378906) bank136122 +136123 POINT(41.861817148484946 -86.72881095577104) bank136123 +136124 POINT(41.05472946809202 -88.17983263367834) bank136124 +136125 POINT(41.030888224963206 -87.2235810834902) bank136125 +136126 POINT(41.50945988726705 -87.16082258642452) bank136126 +136127 POINT(42.370312766269315 -87.69149482652799) bank136127 +136128 POINT(42.78028148020994 -86.82257829056925) bank136128 +136129 POINT(41.18118847610113 -87.7324521201989) bank136129 +136130 POINT(40.962750664627826 -86.63529963670427) bank136130 +136131 POINT(42.60795912306659 -86.75628812876836) bank136131 +136132 POINT(41.76040549473753 -88.06999706549229) bank136132 +136133 POINT(41.13642155803855 -87.15906113981757) bank136133 +136134 POINT(42.412897941044456 -87.18671779231256) bank136134 +136135 POINT(42.05108124828901 -88.45767763364456) bank136135 +136136 POINT(41.44027961661594 -87.24638605546) bank136136 +136137 POINT(42.75087934499083 -88.08190440797685) bank136137 +136138 POINT(42.15655669264222 -86.7059988761866) bank136138 +136139 POINT(41.860936691977464 -87.9356869414684) bank136139 +136140 POINT(42.50644417321289 -87.779786697057) bank136140 +136141 POINT(41.55761783389937 -88.47757141488435) bank136141 +136142 POINT(42.4192004563924 -88.61473574910178) bank136142 +136143 POINT(41.07471912999316 -87.68454010494884) bank136143 +136144 POINT(42.05516938486436 -87.86505376864733) bank136144 +136145 POINT(41.7032250988032 -87.91503425211347) bank136145 +136146 POINT(41.412342552651275 -87.09546660231965) bank136146 +136147 POINT(42.06301411444148 -88.467771299199) bank136147 +136148 POINT(40.87929457904122 -86.95407242814288) bank136148 +136149 POINT(42.61510937865317 -86.981458309055) bank136149 +136150 POINT(42.44850993955686 -88.30112329076253) bank136150 +136151 POINT(40.905647225702765 -87.24406305491712) bank136151 +136152 POINT(41.15391200258091 -88.58819224651425) bank136152 +136153 POINT(42.16727651869967 -88.06860060219785) bank136153 +136154 POINT(42.68558484024397 -87.53490942357094) bank136154 +136155 POINT(42.134083581071444 -87.0500578915716) bank136155 +136156 POINT(41.094239992952744 -86.81759474263376) bank136156 +136157 POINT(41.01418982948467 -88.08579848777092) bank136157 +136158 POINT(42.37221847779941 -86.77554881919917) bank136158 +136159 POINT(42.56045110631461 -87.80230279046545) bank136159 +136160 POINT(42.60726365903787 -88.15843745605217) bank136160 +136161 POINT(40.89101563647167 -87.46998806143455) bank136161 +136162 POINT(42.785545947191984 -87.67502223612199) bank136162 +136163 POINT(41.24288856789918 -88.16007820491618) bank136163 +136164 POINT(40.95880906603359 -88.07224575953165) bank136164 +136165 POINT(41.2090220336433 -87.771367129931) bank136165 +136166 POINT(41.09412611805795 -88.17189372277244) bank136166 +136167 POINT(41.76716694745844 -87.23916867524386) bank136167 +136168 POINT(40.901936052670706 -87.93410837364223) bank136168 +136169 POINT(40.99555977850298 -87.14275662278192) bank136169 +136170 POINT(41.41956743122203 -88.42029747785848) bank136170 +136171 POINT(41.368128859084536 -86.87886872200532) bank136171 +136172 POINT(41.313791201391034 -88.62627074873039) bank136172 +136173 POINT(41.33322379992778 -86.7396400916787) bank136173 +136174 POINT(42.47846725852948 -87.13793927650873) bank136174 +136175 POINT(42.87358583679295 -86.93125504438864) bank136175 +136176 POINT(41.35187728507315 -86.80045491101598) bank136176 +136177 POINT(42.148746447827875 -87.96471923407559) bank136177 +136178 POINT(42.33949361119274 -87.16292145705039) bank136178 +136179 POINT(42.15562271499203 -88.21966112266037) bank136179 +136180 POINT(42.56292023906653 -87.55493273460073) bank136180 +136181 POINT(41.559405824812885 -87.0957518311881) bank136181 +136182 POINT(41.413526516724914 -87.0517814929396) bank136182 +136183 POINT(42.64293923664207 -87.77290123686903) bank136183 +136184 POINT(42.80628474731935 -87.92097746173418) bank136184 +136185 POINT(42.084593679354576 -87.6042025748284) bank136185 +136186 POINT(40.920790733470305 -87.66941052770197) bank136186 +136187 POINT(41.263062345346874 -87.79491894414731) bank136187 +136188 POINT(42.359312383492664 -87.75954779373109) bank136188 +136189 POINT(42.16113039595196 -87.8606545970728) bank136189 +136190 POINT(40.90642626717022 -87.37858571583949) bank136190 +136191 POINT(41.13813420893417 -86.81145254062879) bank136191 +136192 POINT(41.49045830194742 -86.83658946948951) bank136192 +136193 POINT(41.5441759714872 -87.18292554167756) bank136193 +136194 POINT(41.55411987388864 -88.41192128893003) bank136194 +136195 POINT(42.67556689144258 -86.84745818537394) bank136195 +136196 POINT(40.997158010462435 -87.67825831534184) bank136196 +136197 POINT(42.83779320133286 -87.8860251970426) bank136197 +136198 POINT(42.62261400697569 -87.75208281789237) bank136198 +136199 POINT(42.19073210332643 -86.80786964063911) bank136199 +136200 POINT(42.34127727190828 -88.5593231305629) bank136200 +136201 POINT(42.73395151498925 -87.13529815683293) bank136201 +136202 POINT(41.9909003154251 -88.53626097954012) bank136202 +136203 POINT(41.338263592884026 -87.63509694116675) bank136203 +136204 POINT(42.733678611970504 -86.71204609800462) bank136204 +136205 POINT(42.52332549571746 -87.0506628759854) bank136205 +136206 POINT(41.63526371833613 -86.74951567733437) bank136206 +136207 POINT(42.70426182442867 -86.88259365028392) bank136207 +136208 POINT(41.85809504493801 -88.24063554135238) bank136208 +136209 POINT(41.575776245769184 -88.4325230547398) bank136209 +136210 POINT(40.98125774346218 -87.81560550015605) bank136210 +136211 POINT(41.13236150966573 -88.36077875448434) bank136211 +136212 POINT(41.76484376147589 -87.74833208676385) bank136212 +136213 POINT(41.21576821135901 -87.65147781357577) bank136213 +136214 POINT(41.69642461447499 -87.74147652889) bank136214 +136215 POINT(40.96421115502977 -87.34479329461756) bank136215 +136216 POINT(41.13538175175882 -87.93328703448724) bank136216 +136217 POINT(41.60375981147969 -87.87332762253533) bank136217 +136218 POINT(42.58066024808443 -88.5361572266684) bank136218 +136219 POINT(41.45392610438012 -88.593213308891) bank136219 +136220 POINT(42.80670055282586 -88.21680851343105) bank136220 +136221 POINT(41.237680996645764 -86.83418708583581) bank136221 +136222 POINT(40.96583935237281 -87.14040218088982) bank136222 +136223 POINT(41.41393326254637 -87.41919289172338) bank136223 +136224 POINT(41.036158729366285 -87.45997338952544) bank136224 +136225 POINT(42.85451231192299 -87.10249121364613) bank136225 +136226 POINT(41.3458930087954 -88.18304066529701) bank136226 +136227 POINT(41.306349243545554 -87.77268787934435) bank136227 +136228 POINT(41.32633513915702 -88.54724349409007) bank136228 +136229 POINT(41.68027190027397 -87.98279312748791) bank136229 +136230 POINT(42.39341836198807 -87.79248375513298) bank136230 +136231 POINT(42.4929014876421 -87.5439639714887) bank136231 +136232 POINT(41.45806174372375 -87.19106825701589) bank136232 +136233 POINT(42.080351888556784 -87.49856771674652) bank136233 +136234 POINT(42.298066991501265 -86.99536163623996) bank136234 +136235 POINT(41.5733815214539 -87.94564109834745) bank136235 +136236 POINT(41.256749502242755 -87.67364625335838) bank136236 +136237 POINT(41.10677103778085 -87.5795201039114) bank136237 +136238 POINT(42.762807230141675 -87.40683338434513) bank136238 +136239 POINT(41.35549915519605 -88.46315404673499) bank136239 +136240 POINT(40.913584201180846 -86.96241081818441) bank136240 +136241 POINT(42.03267455023789 -87.9379296227186) bank136241 +136242 POINT(41.91563509779779 -87.00860429945685) bank136242 +136243 POINT(41.726471409843526 -87.17462259332513) bank136243 +136244 POINT(41.30052527909103 -86.93447383914378) bank136244 +136245 POINT(41.52009285551979 -86.74540864822174) bank136245 +136246 POINT(41.03801359833216 -88.54016655588832) bank136246 +136247 POINT(41.294898916278726 -88.11174670228475) bank136247 +136248 POINT(42.74473694378078 -87.06599763686869) bank136248 +136249 POINT(42.475987839083125 -87.61286507693328) bank136249 +136250 POINT(41.92041363540559 -86.69159165321706) bank136250 +136251 POINT(41.819515061458866 -88.15069353253182) bank136251 +136252 POINT(42.48117743276508 -86.91841109404491) bank136252 +136253 POINT(42.66114999049749 -87.45982187472836) bank136253 +136254 POINT(41.450984045653236 -88.0189428894228) bank136254 +136255 POINT(41.7405186365277 -88.22179975341237) bank136255 +136256 POINT(41.449829385150444 -87.24464954205993) bank136256 +136257 POINT(41.35506288836031 -87.21818879013634) bank136257 +136258 POINT(41.59211095669828 -88.18050335042376) bank136258 +136259 POINT(42.2633727227834 -87.39372475018106) bank136259 +136260 POINT(42.430725876721894 -86.93523355979836) bank136260 +136261 POINT(41.48612597868545 -87.42650693199134) bank136261 +136262 POINT(41.07739258131168 -87.6414245982091) bank136262 +136263 POINT(42.83813599777564 -86.76380954586892) bank136263 +136264 POINT(42.73969164073294 -86.99964004096249) bank136264 +136265 POINT(40.96194005253024 -86.8299243377462) bank136265 +136266 POINT(42.2613595027608 -88.26902851413196) bank136266 +136267 POINT(42.71806671579676 -87.55413339113404) bank136267 +136268 POINT(41.93944763838656 -88.5866187542819) bank136268 +136269 POINT(41.92826406104436 -87.23715129179935) bank136269 +136270 POINT(41.52950053523984 -87.68218288281335) bank136270 +136271 POINT(41.4759207169664 -87.52605224693433) bank136271 +136272 POINT(40.90021291364344 -88.605967668222) bank136272 +136273 POINT(41.18408875133481 -87.16347685714669) bank136273 +136274 POINT(41.85640356884656 -86.97838631134266) bank136274 +136275 POINT(42.51209472913919 -87.90601520470985) bank136275 +136276 POINT(41.33319448841115 -86.81142656750072) bank136276 +136277 POINT(42.8465846175815 -87.74222768998821) bank136277 +136278 POINT(41.24854496774704 -86.74074591573304) bank136278 +136279 POINT(42.60423120805116 -88.13728123632656) bank136279 +136280 POINT(41.355688120057174 -88.551394274545) bank136280 +136281 POINT(42.4554887667328 -88.60685865971644) bank136281 +136282 POINT(41.540741441584245 -87.48512501373024) bank136282 +136283 POINT(42.475382850572196 -87.04650118789844) bank136283 +136284 POINT(41.55441975277046 -88.29853581006698) bank136284 +136285 POINT(42.14840656236443 -86.75245736370755) bank136285 +136286 POINT(41.3798559243932 -88.0293274474656) bank136286 +136287 POINT(42.084820197578516 -88.44203210474281) bank136287 +136288 POINT(42.58683844592743 -88.34851620686753) bank136288 +136289 POINT(42.53765564851132 -87.41639187083575) bank136289 +136290 POINT(42.147272329805425 -88.45822396637436) bank136290 +136291 POINT(41.302356771077484 -87.88404005287086) bank136291 +136292 POINT(40.97521710860665 -87.12654503846356) bank136292 +136293 POINT(41.99362009145419 -87.49398350013671) bank136293 +136294 POINT(42.86668350626501 -88.24952746249649) bank136294 +136295 POINT(41.26838417849333 -87.83353920859142) bank136295 +136296 POINT(41.612542328401965 -87.74785468423904) bank136296 +136297 POINT(42.76844775058522 -88.30187021869047) bank136297 +136298 POINT(41.398548840605045 -86.75490196035041) bank136298 +136299 POINT(42.648410090857105 -86.72591540866614) bank136299 +136300 POINT(41.566593450491006 -87.71557387242044) bank136300 +136301 POINT(41.81400692917923 -87.81242003984454) bank136301 +136302 POINT(42.25395547578102 -88.31276414617433) bank136302 +136303 POINT(41.7484742980093 -88.21080132819274) bank136303 +136304 POINT(41.666372809541315 -87.03674654206134) bank136304 +136305 POINT(42.0494291024805 -87.17914269495076) bank136305 +136306 POINT(40.97464029469511 -87.85701492179767) bank136306 +136307 POINT(42.72923597450528 -87.42538711970126) bank136307 +136308 POINT(41.233413342164916 -87.24673294751962) bank136308 +136309 POINT(42.40845429072983 -86.72769593147143) bank136309 +136310 POINT(41.887599094287914 -87.3096856186033) bank136310 +136311 POINT(42.52385890049364 -88.34987841723212) bank136311 +136312 POINT(42.48540946412321 -86.77976477214395) bank136312 +136313 POINT(41.19879371630124 -86.70130031980156) bank136313 +136314 POINT(41.46188320947396 -88.09906929817546) bank136314 +136315 POINT(42.59510825170357 -87.30346355798905) bank136315 +136316 POINT(41.09568271735114 -86.8887254491479) bank136316 +136317 POINT(42.42219076512324 -87.19345437258086) bank136317 +136318 POINT(41.376071414702 -87.8524933041697) bank136318 +136319 POINT(40.91690111743443 -86.68203467377731) bank136319 +136320 POINT(42.60651532167791 -88.38443277748301) bank136320 +136321 POINT(40.920411861526574 -87.23637407937217) bank136321 +136322 POINT(42.35856089089893 -86.84903760257056) bank136322 +136323 POINT(41.245732888246664 -88.00593435657181) bank136323 +136324 POINT(42.454713482476556 -88.23411884017611) bank136324 +136325 POINT(42.274196466614534 -88.40684459865618) bank136325 +136326 POINT(41.364604341653965 -86.68239733016466) bank136326 +136327 POINT(42.014951489657356 -87.98750351980607) bank136327 +136328 POINT(42.0005279908736 -87.60241509867562) bank136328 +136329 POINT(42.186688034162216 -87.41816132732971) bank136329 +136330 POINT(41.782640516089 -86.82821691974274) bank136330 +136331 POINT(42.38866102581923 -87.80636083636622) bank136331 +136332 POINT(41.676133557076234 -88.2654068489122) bank136332 +136333 POINT(41.97086296541639 -87.2317560758) bank136333 +136334 POINT(41.96089759074112 -86.64360250482245) bank136334 +136335 POINT(41.526140161319724 -88.32341039081342) bank136335 +136336 POINT(42.72564098851802 -87.63092750742892) bank136336 +136337 POINT(40.90633610672059 -86.8233641062151) bank136337 +136338 POINT(41.91710341129384 -86.74273577100959) bank136338 +136339 POINT(42.07983894043661 -88.55436484279417) bank136339 +136340 POINT(41.08765738736597 -87.94995835310937) bank136340 +136341 POINT(42.301108589490056 -86.96348045489279) bank136341 +136342 POINT(41.30973524666666 -88.05558226119662) bank136342 +136343 POINT(41.05663144340499 -88.554211830032) bank136343 +136344 POINT(41.52253083198934 -87.42145157041935) bank136344 +136345 POINT(41.408517239909386 -87.44684365431598) bank136345 +136346 POINT(42.68503680672216 -87.21696774887877) bank136346 +136347 POINT(40.948894089291855 -86.7818973309607) bank136347 +136348 POINT(41.30606886610729 -87.65641496795415) bank136348 +136349 POINT(41.78236077302754 -88.37871863085338) bank136349 +136350 POINT(41.65389558382909 -88.36158036073516) bank136350 +136351 POINT(41.60724538699583 -87.08590528402172) bank136351 +136352 POINT(41.43437164614976 -88.12357580207527) bank136352 +136353 POINT(41.8619871848955 -86.73489358253683) bank136353 +136354 POINT(41.024114721499934 -87.42321794098696) bank136354 +136355 POINT(42.276174398593135 -87.79979492874969) bank136355 +136356 POINT(42.73129564507748 -86.98994856162992) bank136356 +136357 POINT(41.65392195182819 -88.0141805026556) bank136357 +136358 POINT(41.80455493000384 -87.01193680108224) bank136358 +136359 POINT(41.63374254589475 -87.01127407692196) bank136359 +136360 POINT(42.859585268728004 -87.95490389905154) bank136360 +136361 POINT(40.97840787508014 -88.55754499645725) bank136361 +136362 POINT(42.24104134733832 -88.28753665912224) bank136362 +136363 POINT(42.33227332276714 -88.18730694483673) bank136363 +136364 POINT(41.61615165429374 -87.03912885080445) bank136364 +136365 POINT(41.68212355014523 -86.77370582934525) bank136365 +136366 POINT(41.75502298275172 -87.93881457922157) bank136366 +136367 POINT(42.07948407976227 -86.89287872237995) bank136367 +136368 POINT(42.19487885418585 -87.25455936083067) bank136368 +136369 POINT(42.56353955476802 -86.95493333828524) bank136369 +136370 POINT(41.60394750606435 -88.41445544859859) bank136370 +136371 POINT(40.891848934325644 -87.55702213797225) bank136371 +136372 POINT(40.92902738017474 -88.4991450703068) bank136372 +136373 POINT(42.640485879132434 -88.18102946667499) bank136373 +136374 POINT(41.44027695846632 -87.52223536001176) bank136374 +136375 POINT(42.1549811325244 -88.53176959743281) bank136375 +136376 POINT(41.240157690036625 -86.64726790594514) bank136376 +136377 POINT(42.187788024820414 -87.0375502790013) bank136377 +136378 POINT(42.54553456402428 -88.18367946699072) bank136378 +136379 POINT(42.59148558378247 -86.91653284895911) bank136379 +136380 POINT(41.90994960598193 -88.38399034135159) bank136380 +136381 POINT(42.339679526682524 -87.71776559767505) bank136381 +136382 POINT(42.19532946780259 -88.1323112637564) bank136382 +136383 POINT(41.114272233170624 -87.0628201355777) bank136383 +136384 POINT(41.83658442565182 -88.40656511570293) bank136384 +136385 POINT(42.74628657150675 -87.13924889374468) bank136385 +136386 POINT(42.84057471457511 -88.24474425555213) bank136386 +136387 POINT(41.75372217943217 -88.02448748801316) bank136387 +136388 POINT(42.65221480213219 -86.95025062398629) bank136388 +136389 POINT(42.51830372265321 -87.47802704218448) bank136389 +136390 POINT(42.65363610706365 -86.72121616799035) bank136390 +136391 POINT(41.99954069665182 -87.52439657142965) bank136391 +136392 POINT(42.17736924339581 -87.80678776585775) bank136392 +136393 POINT(41.316623186600715 -86.75295479678431) bank136393 +136394 POINT(42.317319228561175 -87.47000127924008) bank136394 +136395 POINT(41.72795071026066 -87.4667963815969) bank136395 +136396 POINT(41.875150805690375 -88.21384147628763) bank136396 +136397 POINT(41.36356430617259 -88.33953126302606) bank136397 +136398 POINT(41.1273439739402 -87.16930207903644) bank136398 +136399 POINT(42.53191356081465 -87.60337832262387) bank136399 +136400 POINT(42.71373152092828 -86.88657316355179) bank136400 +136401 POINT(42.21919402624406 -87.00845382006018) bank136401 +136402 POINT(41.47786466226317 -87.58372686306356) bank136402 +136403 POINT(41.481891237891844 -87.6575494785519) bank136403 +136404 POINT(42.243193658338804 -86.8345487114451) bank136404 +136405 POINT(42.05729953446874 -88.01324846534692) bank136405 +136406 POINT(41.15848184567169 -88.18922166545721) bank136406 +136407 POINT(41.19910285787493 -87.63434235701727) bank136407 +136408 POINT(41.70394504014705 -88.58296535135011) bank136408 +136409 POINT(41.27594721786464 -88.2658711513149) bank136409 +136410 POINT(41.22889228176521 -88.16654798143122) bank136410 +136411 POINT(41.936145607916544 -87.62907816475801) bank136411 +136412 POINT(42.80133538832195 -86.85775017006264) bank136412 +136413 POINT(41.54299198348939 -88.47719026878958) bank136413 +136414 POINT(41.35205256996592 -88.44656284199624) bank136414 +136415 POINT(41.105501575037785 -88.17906769658165) bank136415 +136416 POINT(42.45569589238152 -87.56436344707525) bank136416 +136417 POINT(41.71875876879443 -86.67091024757174) bank136417 +136418 POINT(41.347361963044804 -88.0774566691959) bank136418 +136419 POINT(42.75055036412484 -87.42679341618552) bank136419 +136420 POINT(41.585811651765944 -87.39300299555171) bank136420 +136421 POINT(41.69302433953558 -87.46762141238244) bank136421 +136422 POINT(42.8018033532934 -86.99736975000701) bank136422 +136423 POINT(41.02201824882861 -86.97355852576321) bank136423 +136424 POINT(41.29496289021244 -87.8757429092058) bank136424 +136425 POINT(41.967588293500356 -87.17116589554414) bank136425 +136426 POINT(41.5419705342807 -86.92513012277648) bank136426 +136427 POINT(42.14921100107521 -86.67071779778361) bank136427 +136428 POINT(41.930228113226086 -86.81961436531283) bank136428 +136429 POINT(41.72746100948007 -88.01632062676255) bank136429 +136430 POINT(41.539048025883574 -87.35684941487825) bank136430 +136431 POINT(41.5317023873676 -86.77108117129399) bank136431 +136432 POINT(41.471529256339544 -86.7535965148803) bank136432 +136433 POINT(40.884286271929575 -86.81140830211797) bank136433 +136434 POINT(41.22300763221486 -86.75414254693511) bank136434 +136435 POINT(42.16131584184066 -87.20782298858545) bank136435 +136436 POINT(41.060743404492875 -87.70904198829179) bank136436 +136437 POINT(42.79100431599963 -87.00764131887416) bank136437 +136438 POINT(42.707201765389115 -87.28038299271142) bank136438 +136439 POINT(42.58993645535662 -88.60918916235805) bank136439 +136440 POINT(41.0568285037522 -87.7285460176451) bank136440 +136441 POINT(40.88339529493478 -88.61417850493588) bank136441 +136442 POINT(42.0835957621056 -86.98068834198732) bank136442 +136443 POINT(41.78232300959346 -88.39344943394191) bank136443 +136444 POINT(42.62618249871097 -88.17224182436222) bank136444 +136445 POINT(41.439787951914624 -88.30278976907033) bank136445 +136446 POINT(42.54649490560546 -87.54297216417797) bank136446 +136447 POINT(42.126362201318734 -88.56387126514734) bank136447 +136448 POINT(41.784842701875284 -87.3394229456376) bank136448 +136449 POINT(42.845184337052736 -87.85773257508431) bank136449 +136450 POINT(42.2831207447893 -87.91024016737472) bank136450 +136451 POINT(42.834398738543754 -87.59506711230044) bank136451 +136452 POINT(41.92868372322767 -86.658303335983) bank136452 +136453 POINT(41.52841001646796 -88.38997359821504) bank136453 +136454 POINT(42.11595551542112 -87.52665482326915) bank136454 +136455 POINT(40.89547987977525 -87.5675096883838) bank136455 +136456 POINT(41.81349887568246 -88.38511236973973) bank136456 +136457 POINT(42.13163789766523 -87.26417368796922) bank136457 +136458 POINT(40.90442142617668 -87.05092583555161) bank136458 +136459 POINT(42.1246903899837 -86.78648490901546) bank136459 +136460 POINT(42.264494630204446 -87.88327845619054) bank136460 +136461 POINT(41.41126777751187 -88.22339950338332) bank136461 +136462 POINT(41.31993239903987 -87.28499666809589) bank136462 +136463 POINT(41.45360118755792 -87.19854839111342) bank136463 +136464 POINT(41.308575713547235 -87.74816839084205) bank136464 +136465 POINT(41.54699105927455 -88.09554152412744) bank136465 +136466 POINT(42.54835800646024 -86.75134605756085) bank136466 +136467 POINT(42.50736444695241 -87.96305844542381) bank136467 +136468 POINT(41.64958220101737 -87.92441752934305) bank136468 +136469 POINT(42.08968018617803 -87.6577315724034) bank136469 +136470 POINT(41.764669553914494 -88.43462096714804) bank136470 +136471 POINT(42.740150048661135 -88.13144751031905) bank136471 +136472 POINT(42.73564479847138 -87.99233395874303) bank136472 +136473 POINT(41.32930529550963 -87.72059942615408) bank136473 +136474 POINT(42.080586865981225 -86.70120248280632) bank136474 +136475 POINT(41.83147706057433 -86.73421969367404) bank136475 +136476 POINT(42.83267736946816 -87.19518536307442) bank136476 +136477 POINT(42.354364545902484 -88.25058671470025) bank136477 +136478 POINT(42.00966657015434 -88.52618419602442) bank136478 +136479 POINT(41.84910159175886 -88.27130743094405) bank136479 +136480 POINT(42.464528212708515 -87.30650946768448) bank136480 +136481 POINT(41.59489773619098 -87.25826967758334) bank136481 +136482 POINT(42.429982296899084 -86.73525717557447) bank136482 +136483 POINT(42.85596332449725 -88.03040510720984) bank136483 +136484 POINT(41.10167984419383 -87.8843599355477) bank136484 +136485 POINT(42.78024275016295 -88.31653070477995) bank136485 +136486 POINT(41.721767010406516 -87.74776860640469) bank136486 +136487 POINT(42.24996744274181 -87.92590192187116) bank136487 +136488 POINT(42.43029415616346 -87.71172083956382) bank136488 +136489 POINT(41.71447307118257 -87.647872116444) bank136489 +136490 POINT(42.43127516824942 -86.9846135196955) bank136490 +136491 POINT(41.6791160030854 -87.38603387477191) bank136491 +136492 POINT(41.8117056986492 -88.38406926787195) bank136492 +136493 POINT(42.23562585952843 -88.25726705852705) bank136493 +136494 POINT(41.25267370959881 -86.95210951137297) bank136494 +136495 POINT(41.64892506991727 -87.655993542225) bank136495 +136496 POINT(41.329295739899294 -88.0362418879834) bank136496 +136497 POINT(41.238386654471704 -88.31007685317405) bank136497 +136498 POINT(41.842437726211955 -87.80087846580528) bank136498 +136499 POINT(42.25071120998319 -87.3158881023173) bank136499 +136500 POINT(41.528422890413545 -88.08576343371031) bank136500 +136501 POINT(41.352182433518806 -86.86247659088653) bank136501 +136502 POINT(41.26229993250306 -87.51091587290352) bank136502 +136503 POINT(42.721277485280254 -86.81050459449106) bank136503 +136504 POINT(42.829358587043785 -86.73184658122895) bank136504 +136505 POINT(42.22906153175348 -86.97698034501732) bank136505 +136506 POINT(42.255330004906746 -88.2676375633866) bank136506 +136507 POINT(41.62506900661676 -86.77054480713672) bank136507 +136508 POINT(42.801410483149155 -86.8353871743866) bank136508 +136509 POINT(42.812228510729646 -87.57667886908699) bank136509 +136510 POINT(42.05976599722788 -88.20852968610994) bank136510 +136511 POINT(41.42063587552499 -86.66025445578765) bank136511 +136512 POINT(42.504488895200396 -87.01772252517618) bank136512 +136513 POINT(41.55540980785947 -88.40824083981296) bank136513 +136514 POINT(41.70737392748472 -87.4042440728926) bank136514 +136515 POINT(41.895749095966984 -87.59847045302607) bank136515 +136516 POINT(41.53917782194434 -86.99452025809194) bank136516 +136517 POINT(41.567391853900126 -87.66090661719025) bank136517 +136518 POINT(41.94473025356118 -87.02480443905111) bank136518 +136519 POINT(42.1242424680067 -86.90588796014471) bank136519 +136520 POINT(42.4665144594827 -87.38973615570212) bank136520 +136521 POINT(42.61075093620393 -88.1376475107691) bank136521 +136522 POINT(42.44819532985857 -88.0901633792255) bank136522 +136523 POINT(41.08338854572688 -86.73185622301507) bank136523 +136524 POINT(41.63234878970739 -87.4424255484861) bank136524 +136525 POINT(42.77380633848213 -88.20894663821679) bank136525 +136526 POINT(42.27785737911659 -86.89895204007662) bank136526 +136527 POINT(42.3518721528377 -87.12844571621768) bank136527 +136528 POINT(41.660968351872675 -86.86692804160008) bank136528 +136529 POINT(42.093971132924565 -87.96012121027299) bank136529 +136530 POINT(41.53812772374595 -87.26874068128113) bank136530 +136531 POINT(42.62798569086013 -88.15012879097442) bank136531 +136532 POINT(42.77572016863465 -86.806969164342) bank136532 +136533 POINT(42.4848117142015 -87.11193388216738) bank136533 +136534 POINT(42.06117466136069 -88.2879965304304) bank136534 +136535 POINT(40.88134189430911 -87.19558374806758) bank136535 +136536 POINT(41.968992666873525 -86.65405883055095) bank136536 +136537 POINT(42.155934224388595 -88.08351609788917) bank136537 +136538 POINT(42.43727677678656 -86.67460272409843) bank136538 +136539 POINT(42.5047699777629 -87.05860591651458) bank136539 +136540 POINT(42.65395683347647 -87.37084435669819) bank136540 +136541 POINT(42.52431462424705 -87.37153446860432) bank136541 +136542 POINT(42.447266837200914 -86.98797666215212) bank136542 +136543 POINT(42.60474512094049 -87.76743756731327) bank136543 +136544 POINT(41.16825220424213 -86.96667337439607) bank136544 +136545 POINT(41.470899751723934 -86.99338634408936) bank136545 +136546 POINT(42.0081401291621 -88.07688052910976) bank136546 +136547 POINT(41.003756495983275 -87.17589133020468) bank136547 +136548 POINT(41.10164764043371 -86.93157351745559) bank136548 +136549 POINT(41.94605447162678 -86.81611212676765) bank136549 +136550 POINT(41.459495602748824 -88.28227418686446) bank136550 +136551 POINT(42.61166765347235 -88.50569839845208) bank136551 +136552 POINT(41.165233032020836 -88.37485387781142) bank136552 +136553 POINT(40.96095507408372 -87.88207478174829) bank136553 +136554 POINT(42.14856817057394 -86.95051745811024) bank136554 +136555 POINT(42.32992065438453 -87.77355948747011) bank136555 +136556 POINT(42.52038176925854 -87.39230878055193) bank136556 +136557 POINT(42.257431513685425 -87.14299165123376) bank136557 +136558 POINT(42.68087113911288 -87.20123446897439) bank136558 +136559 POINT(42.198389058975366 -87.81965902253312) bank136559 +136560 POINT(41.10466031228627 -88.25213493384457) bank136560 +136561 POINT(42.20972936049516 -87.03688280920586) bank136561 +136562 POINT(41.145426256854655 -88.30206172189793) bank136562 +136563 POINT(41.94209467160716 -86.8244854925732) bank136563 +136564 POINT(41.63233968482949 -86.99080416740823) bank136564 +136565 POINT(42.34602681498083 -88.18903636097838) bank136565 +136566 POINT(41.77281582405392 -87.71566198392873) bank136566 +136567 POINT(41.126481809698845 -88.0364952874502) bank136567 +136568 POINT(41.82968645415907 -87.96051704799854) bank136568 +136569 POINT(42.50207107778014 -87.75646264164874) bank136569 +136570 POINT(40.920865024855345 -88.26770488603938) bank136570 +136571 POINT(42.25692944048241 -87.63055229689773) bank136571 +136572 POINT(42.282814861947664 -87.2201636877084) bank136572 +136573 POINT(42.02402354095984 -88.03498720218174) bank136573 +136574 POINT(41.545332430368354 -88.53052586132875) bank136574 +136575 POINT(41.82635300551897 -86.96432610307677) bank136575 +136576 POINT(41.08737940256535 -87.41259234935836) bank136576 +136577 POINT(42.517880743713846 -87.52629811869836) bank136577 +136578 POINT(42.41055129090551 -88.54685664462698) bank136578 +136579 POINT(42.75013715604192 -86.65950569610628) bank136579 +136580 POINT(42.808599586540225 -87.66038098946308) bank136580 +136581 POINT(41.32626739675999 -87.52038497042894) bank136581 +136582 POINT(42.7601023770437 -86.71746057823245) bank136582 +136583 POINT(41.573605127987456 -86.8913409415094) bank136583 +136584 POINT(41.12477587157219 -87.5357063672063) bank136584 +136585 POINT(40.96471679432753 -88.26078474656148) bank136585 +136586 POINT(42.2142858371333 -88.22848764262675) bank136586 +136587 POINT(42.585715524508764 -87.94588515622895) bank136587 +136588 POINT(42.44517248250446 -87.79135242786147) bank136588 +136589 POINT(42.800557305403785 -88.53051804850979) bank136589 +136590 POINT(42.68342174836392 -86.73999750001227) bank136590 +136591 POINT(40.915941004654286 -86.89581237855296) bank136591 +136592 POINT(41.49749382666091 -87.67857791211202) bank136592 +136593 POINT(41.99667221469918 -88.42377071159785) bank136593 +136594 POINT(41.09982922599068 -87.14125606160556) bank136594 +136595 POINT(41.61265276429939 -88.08168734202411) bank136595 +136596 POINT(41.929820312191666 -87.72824866606216) bank136596 +136597 POINT(41.18232401502993 -87.15132977587967) bank136597 +136598 POINT(41.2899591621908 -87.01609548462235) bank136598 +136599 POINT(41.751896242350504 -87.92180354184167) bank136599 +136600 POINT(42.609997543202496 -87.12077178563774) bank136600 +136601 POINT(42.87413410678856 -88.32910307602279) bank136601 +136602 POINT(41.881316223857056 -88.2175294809366) bank136602 +136603 POINT(41.587398442491654 -88.44986010606699) bank136603 +136604 POINT(41.976209928765094 -87.77554555438161) bank136604 +136605 POINT(41.42700787306891 -88.20738066676425) bank136605 +136606 POINT(41.36000942834953 -87.65144468929678) bank136606 +136607 POINT(40.946070861562525 -87.06442453650038) bank136607 +136608 POINT(41.66630186994059 -86.95934417153838) bank136608 +136609 POINT(41.080853043054674 -88.1097092191293) bank136609 +136610 POINT(41.20915597064696 -87.06494803542816) bank136610 +136611 POINT(41.02602441003151 -88.59791484190431) bank136611 +136612 POINT(42.791884109363366 -86.94200952404337) bank136612 +136613 POINT(41.75176487885958 -87.57770113301213) bank136613 +136614 POINT(40.90427772230998 -87.86765730374488) bank136614 +136615 POINT(41.56826013622412 -87.6124404736323) bank136615 +136616 POINT(41.13600936697578 -87.80056654910723) bank136616 +136617 POINT(42.10413247827744 -88.45310659093214) bank136617 +136618 POINT(40.92343977736086 -87.60297894629511) bank136618 +136619 POINT(42.59164898849076 -86.76715442643648) bank136619 +136620 POINT(41.44971805592966 -88.26489121472056) bank136620 +136621 POINT(42.32287264796306 -88.29681664890674) bank136621 +136622 POINT(42.84431418622866 -87.39522568242641) bank136622 +136623 POINT(41.928687177361105 -88.52149640296943) bank136623 +136624 POINT(41.69691323628836 -88.19579227730598) bank136624 +136625 POINT(42.090208023961274 -86.71459046680506) bank136625 +136626 POINT(42.81655892274123 -87.54633407459998) bank136626 +136627 POINT(42.740671018702336 -86.76844355424619) bank136627 +136628 POINT(41.08217480484876 -87.4355708772833) bank136628 +136629 POINT(41.238922138574495 -88.27936425755256) bank136629 +136630 POINT(42.751855773620015 -88.09924674248477) bank136630 +136631 POINT(41.78645398124381 -88.45815959182418) bank136631 +136632 POINT(41.48939986273446 -87.66464371493595) bank136632 +136633 POINT(42.37802457692224 -88.51426066541312) bank136633 +136634 POINT(42.532164441684884 -87.66599108480949) bank136634 +136635 POINT(42.84071153622201 -88.15142894339081) bank136635 +136636 POINT(42.41464517696128 -86.97038201745121) bank136636 +136637 POINT(41.991890568788016 -87.47106660100145) bank136637 +136638 POINT(42.10030350706673 -87.38390386004373) bank136638 +136639 POINT(41.047683909742744 -87.5570054982754) bank136639 +136640 POINT(42.51067446379478 -88.28038544265507) bank136640 +136641 POINT(41.363974708297306 -87.3936840324956) bank136641 +136642 POINT(42.44656820427553 -87.63341360080184) bank136642 +136643 POINT(41.6753266006782 -88.45595897985471) bank136643 +136644 POINT(41.14923397769449 -87.16732961847491) bank136644 +136645 POINT(42.408485936503745 -88.15921514062758) bank136645 +136646 POINT(42.815695589716 -87.81141783378698) bank136646 +136647 POINT(42.42393332246158 -86.92850223796881) bank136647 +136648 POINT(42.75168116959673 -87.39765689993227) bank136648 +136649 POINT(42.36027408368383 -88.12427246933872) bank136649 +136650 POINT(41.7392980052341 -86.9012331182299) bank136650 +136651 POINT(42.1282060665852 -88.51996684442857) bank136651 +136652 POINT(41.077412781289915 -88.50068671281957) bank136652 +136653 POINT(42.666600908659056 -86.95710374219337) bank136653 +136654 POINT(41.0080328149883 -86.75190886471853) bank136654 +136655 POINT(42.092477448122466 -87.4438849985082) bank136655 +136656 POINT(41.432908320865266 -87.80628497737824) bank136656 +136657 POINT(42.258725049605786 -88.5131607486707) bank136657 +136658 POINT(42.634713669581785 -87.88071591723786) bank136658 +136659 POINT(41.9135016694884 -88.33029509871213) bank136659 +136660 POINT(41.37046112749026 -88.31151074058967) bank136660 +136661 POINT(42.33045375437231 -88.24238952936618) bank136661 +136662 POINT(42.579513567942136 -87.687801516504) bank136662 +136663 POINT(42.3550883282791 -87.8567275483761) bank136663 +136664 POINT(41.75454480999195 -88.51855721862862) bank136664 +136665 POINT(41.867969732654075 -87.01393976832885) bank136665 +136666 POINT(42.325195760922206 -87.00226364373601) bank136666 +136667 POINT(42.74651509396596 -87.40832291111795) bank136667 +136668 POINT(42.60318342281708 -86.89312813652342) bank136668 +136669 POINT(42.18503363736049 -86.7657077502743) bank136669 +136670 POINT(42.478048350067496 -88.37342558192803) bank136670 +136671 POINT(42.36089182835441 -86.80240904322153) bank136671 +136672 POINT(42.016439325802104 -87.34146662358204) bank136672 +136673 POINT(42.42185158088723 -87.6063861909849) bank136673 +136674 POINT(41.14874060111719 -87.03974938215904) bank136674 +136675 POINT(41.77787766209588 -87.92756107820503) bank136675 +136676 POINT(41.44534104421095 -86.96597699358117) bank136676 +136677 POINT(42.387067992750175 -87.04846772466209) bank136677 +136678 POINT(42.796374606731334 -87.00397080175875) bank136678 +136679 POINT(41.583610220588476 -88.00946795363274) bank136679 +136680 POINT(41.89231156367299 -88.09862125525578) bank136680 +136681 POINT(41.70490918147819 -87.02596982110866) bank136681 +136682 POINT(41.40676690944379 -88.40064144815321) bank136682 +136683 POINT(42.088315670481336 -88.59566985212007) bank136683 +136684 POINT(41.461875338545056 -87.49362714693466) bank136684 +136685 POINT(40.909640799827045 -88.38666548111831) bank136685 +136686 POINT(40.972473422513566 -87.69001571047131) bank136686 +136687 POINT(40.89561169520507 -87.84958311950831) bank136687 +136688 POINT(41.83043106463104 -87.10297124782667) bank136688 +136689 POINT(41.99501423211465 -88.13352593879233) bank136689 +136690 POINT(42.81271348023597 -88.41438686781905) bank136690 +136691 POINT(42.30628471097346 -87.38905712442114) bank136691 +136692 POINT(42.09852814595796 -88.19547108548926) bank136692 +136693 POINT(42.35887821412957 -88.6013636448808) bank136693 +136694 POINT(42.47479115710136 -87.29111231059454) bank136694 +136695 POINT(41.40122019746943 -86.82519442913002) bank136695 +136696 POINT(41.35515458369495 -88.32214624936731) bank136696 +136697 POINT(41.102638101320565 -88.14221290520771) bank136697 +136698 POINT(42.09156883150462 -87.86540716679552) bank136698 +136699 POINT(41.644855543354346 -88.61477629858875) bank136699 +136700 POINT(42.49264271565781 -88.26592889552404) bank136700 +136701 POINT(41.76197614009842 -88.62597001787705) bank136701 +136702 POINT(41.53125581005676 -88.15493471860476) bank136702 +136703 POINT(42.16040520143579 -88.62970339319982) bank136703 +136704 POINT(41.25873008789106 -88.2950049198029) bank136704 +136705 POINT(40.99446853340005 -87.41510408189976) bank136705 +136706 POINT(41.01536567742516 -87.81356122271582) bank136706 +136707 POINT(40.885246422013395 -88.47616673934705) bank136707 +136708 POINT(42.229382879842895 -87.33746409120312) bank136708 +136709 POINT(42.42853980953714 -87.35173635543464) bank136709 +136710 POINT(42.80948945850863 -87.04868521647347) bank136710 +136711 POINT(42.85447690589416 -86.92888414156538) bank136711 +136712 POINT(41.226776135951 -88.56309695437875) bank136712 +136713 POINT(42.64887532731858 -87.20829819478331) bank136713 +136714 POINT(42.27369096936051 -88.2669710939155) bank136714 +136715 POINT(42.162545915569545 -87.57354406070367) bank136715 +136716 POINT(42.114614963231766 -86.79163527756648) bank136716 +136717 POINT(41.04541476042976 -87.29230950737046) bank136717 +136718 POINT(42.73086557390615 -86.75120650238286) bank136718 +136719 POINT(42.38810180517372 -88.30109014131527) bank136719 +136720 POINT(41.630420191799764 -87.38534294157454) bank136720 +136721 POINT(41.312282108100895 -88.3902558147973) bank136721 +136722 POINT(41.564926841282364 -88.00527218712647) bank136722 +136723 POINT(41.66612466310953 -87.65824305007294) bank136723 +136724 POINT(42.42739315577065 -86.72234888720446) bank136724 +136725 POINT(42.09897428873189 -88.3484957197298) bank136725 +136726 POINT(42.43250001800622 -88.06079167785133) bank136726 +136727 POINT(41.43560701667041 -86.91176444918695) bank136727 +136728 POINT(42.52374661478739 -87.58824496657097) bank136728 +136729 POINT(40.977535606755026 -86.66391876291839) bank136729 +136730 POINT(41.73391277138028 -86.99697453578938) bank136730 +136731 POINT(41.3491211234133 -88.46084012637769) bank136731 +136732 POINT(41.172112502406044 -86.84507796435089) bank136732 +136733 POINT(42.01481021707396 -88.5429506234425) bank136733 +136734 POINT(42.248985071502084 -87.21876065771198) bank136734 +136735 POINT(42.43694526424376 -87.8038984008613) bank136735 +136736 POINT(41.69730115715676 -88.43793405532186) bank136736 +136737 POINT(41.49167224221633 -86.84002017805092) bank136737 +136738 POINT(41.29260405043398 -86.67094651215716) bank136738 +136739 POINT(42.69488348546125 -86.70942230136018) bank136739 +136740 POINT(41.37194761352132 -88.29784108344897) bank136740 +136741 POINT(41.779325802937564 -88.42601305583166) bank136741 +136742 POINT(41.23691290954828 -87.42959390156574) bank136742 +136743 POINT(42.32555407088555 -87.41994123765815) bank136743 +136744 POINT(42.32554272822663 -87.40440500723211) bank136744 +136745 POINT(42.65686962941887 -87.95043355420998) bank136745 +136746 POINT(41.33871130436322 -86.84983904412904) bank136746 +136747 POINT(42.576162060486254 -88.01842859927267) bank136747 +136748 POINT(42.27514026045142 -88.52796463760924) bank136748 +136749 POINT(42.362348335194426 -88.51621377975323) bank136749 +136750 POINT(41.814358933276225 -87.13180084770049) bank136750 +136751 POINT(42.05568815001342 -87.18179337414975) bank136751 +136752 POINT(42.37369958624483 -87.71692596213606) bank136752 +136753 POINT(41.014277983018914 -88.0330399514798) bank136753 +136754 POINT(42.329149657338256 -88.05915928368091) bank136754 +136755 POINT(41.655887416631906 -87.98538473263987) bank136755 +136756 POINT(41.70715615918602 -87.06354194437422) bank136756 +136757 POINT(41.24079084359704 -87.49531427839776) bank136757 +136758 POINT(42.168183170133865 -87.39083497672725) bank136758 +136759 POINT(42.263161439592075 -87.90356457812103) bank136759 +136760 POINT(42.87478429736928 -87.12177338635092) bank136760 +136761 POINT(40.89125620288737 -87.6684250407229) bank136761 +136762 POINT(42.15282331213501 -87.92839488689174) bank136762 +136763 POINT(42.22616984660735 -87.9043557536619) bank136763 +136764 POINT(42.293153621507926 -88.15754791280597) bank136764 +136765 POINT(42.50858687294891 -87.12205124414848) bank136765 +136766 POINT(42.219708575285225 -87.81269764909284) bank136766 +136767 POINT(42.091034382351985 -87.85931673021709) bank136767 +136768 POINT(41.35080863002597 -88.19535274680587) bank136768 +136769 POINT(41.545584001767146 -88.05009280714677) bank136769 +136770 POINT(42.12437484589787 -88.57720476677952) bank136770 +136771 POINT(41.292610867164 -88.23487765601146) bank136771 +136772 POINT(42.23090690748291 -87.94166351006176) bank136772 +136773 POINT(42.26211465830957 -86.92975835688382) bank136773 +136774 POINT(41.12427169902921 -86.9391966636083) bank136774 +136775 POINT(41.79452649351403 -88.090297327175) bank136775 +136776 POINT(41.936228424108606 -88.60486581068174) bank136776 +136777 POINT(41.31529913752653 -87.78737558159592) bank136777 +136778 POINT(42.61362134160354 -87.89672482156492) bank136778 +136779 POINT(41.20592898663998 -86.7266855619795) bank136779 +136780 POINT(40.93169101978722 -87.73850955402956) bank136780 +136781 POINT(42.67665061043239 -87.96357405942409) bank136781 +136782 POINT(41.52750780592457 -86.9349923078553) bank136782 +136783 POINT(42.18654379935846 -86.97025339648958) bank136783 +136784 POINT(42.547627204154104 -87.20846414656542) bank136784 +136785 POINT(42.694464245283186 -88.43922112540457) bank136785 +136786 POINT(41.77908320630183 -87.98091757837403) bank136786 +136787 POINT(42.60643896007615 -87.297711882798) bank136787 +136788 POINT(42.23426663011819 -88.58784632403811) bank136788 +136789 POINT(41.5712216255647 -86.80969380745994) bank136789 +136790 POINT(42.793830597718326 -87.99840324879725) bank136790 +136791 POINT(42.71491807691792 -86.69477189680794) bank136791 +136792 POINT(41.86492207923291 -87.20139379250757) bank136792 +136793 POINT(41.27975273099748 -87.86051061380378) bank136793 +136794 POINT(41.810012631203676 -87.05321203476133) bank136794 +136795 POINT(42.393371867185714 -88.34638243544977) bank136795 +136796 POINT(42.75918150660065 -87.31974645308122) bank136796 +136797 POINT(41.17526341090296 -86.67074360263032) bank136797 +136798 POINT(41.553481411080256 -88.42374388570055) bank136798 +136799 POINT(40.949955782355495 -87.79815333681124) bank136799 +136800 POINT(42.5426136246824 -88.23617596045344) bank136800 +136801 POINT(41.81299135964286 -87.26834093328004) bank136801 +136802 POINT(41.30496174966647 -86.63137486995143) bank136802 +136803 POINT(41.70334359132458 -88.10154790825484) bank136803 +136804 POINT(41.40848717238633 -88.54057668573866) bank136804 +136805 POINT(41.71320721750449 -88.518308833617) bank136805 +136806 POINT(42.83136319523956 -88.53325538348389) bank136806 +136807 POINT(42.668643315909215 -87.70984275268022) bank136807 +136808 POINT(42.38319287046588 -88.09852035290807) bank136808 +136809 POINT(42.63006058888574 -86.69929030202941) bank136809 +136810 POINT(40.95758731193593 -87.853765053197) bank136810 +136811 POINT(41.985298396251856 -87.20675730643279) bank136811 +136812 POINT(41.514488272592004 -88.61829776948775) bank136812 +136813 POINT(42.806728329945614 -87.71345786664116) bank136813 +136814 POINT(42.85758770021172 -86.79938638071948) bank136814 +136815 POINT(42.69933634555581 -87.55013020515584) bank136815 +136816 POINT(42.81735021559535 -88.22080707926976) bank136816 +136817 POINT(40.93672699903206 -87.60234830542382) bank136817 +136818 POINT(41.8326602851358 -87.83025404860805) bank136818 +136819 POINT(41.1544548429725 -86.95767007065905) bank136819 +136820 POINT(41.02702700846564 -87.62537730757364) bank136820 +136821 POINT(42.657817054714236 -86.87849883114305) bank136821 +136822 POINT(42.59888964085618 -86.7866716074739) bank136822 +136823 POINT(40.94918695849734 -87.1756183233482) bank136823 +136824 POINT(41.171891887835734 -87.06985917850238) bank136824 +136825 POINT(42.47648437117433 -88.30506872457536) bank136825 +136826 POINT(42.584332396925404 -87.3125050245777) bank136826 +136827 POINT(41.342367637039196 -86.80691658717282) bank136827 +136828 POINT(41.054173654176864 -88.04412497991478) bank136828 +136829 POINT(41.242134813713 -86.64712633788838) bank136829 +136830 POINT(42.549515439461075 -87.60421571003417) bank136830 +136831 POINT(41.65572630831932 -88.07915437113445) bank136831 +136832 POINT(41.528420954258166 -88.28051399043453) bank136832 +136833 POINT(42.03290628276449 -88.14938772011766) bank136833 +136834 POINT(42.8152074850598 -87.0054308281428) bank136834 +136835 POINT(41.86235558377056 -88.21969154429834) bank136835 +136836 POINT(42.01329325994405 -88.3735588452877) bank136836 +136837 POINT(41.197576378953706 -86.66781945436686) bank136837 +136838 POINT(42.44830720386629 -86.7445695542202) bank136838 +136839 POINT(42.64514207772857 -88.5477937268483) bank136839 +136840 POINT(42.81220108031757 -88.284699487208) bank136840 +136841 POINT(41.89073010089051 -88.03704070970704) bank136841 +136842 POINT(41.13307475095668 -86.77769061431891) bank136842 +136843 POINT(42.118048202863804 -88.14841431967554) bank136843 +136844 POINT(42.673573926669725 -87.81725294523669) bank136844 +136845 POINT(42.35130907433006 -87.59941265884622) bank136845 +136846 POINT(42.221548065040864 -88.04873500603719) bank136846 +136847 POINT(41.291114087015934 -88.42989890016469) bank136847 +136848 POINT(42.82110471307997 -86.78688384600731) bank136848 +136849 POINT(42.43201487358574 -87.37560728567709) bank136849 +136850 POINT(42.78735035641337 -87.87043932729924) bank136850 +136851 POINT(42.507895388547205 -86.9149450025736) bank136851 +136852 POINT(41.18278254553805 -87.5592799591337) bank136852 +136853 POINT(42.271367278683826 -87.18198172407173) bank136853 +136854 POINT(42.7041729695019 -87.30125926914604) bank136854 +136855 POINT(41.160189659023715 -87.01093239049602) bank136855 +136856 POINT(41.75589629085673 -86.89891021057855) bank136856 +136857 POINT(42.3107379502823 -88.50949341919366) bank136857 +136858 POINT(42.08210734690111 -87.59524622268171) bank136858 +136859 POINT(41.66829936915457 -88.10612532981295) bank136859 +136860 POINT(42.41408290311485 -87.72905838288032) bank136860 +136861 POINT(41.27991903088693 -87.84968364024387) bank136861 +136862 POINT(42.04454686740476 -86.89955333406365) bank136862 +136863 POINT(41.88763659170412 -87.8559389705396) bank136863 +136864 POINT(40.979930658643 -88.27106085637426) bank136864 +136865 POINT(41.65483809762746 -86.81385167935385) bank136865 +136866 POINT(41.173751990510766 -87.42271714802449) bank136866 +136867 POINT(42.01523838501805 -87.12361533947961) bank136867 +136868 POINT(42.309678044732706 -86.92078356395008) bank136868 +136869 POINT(42.33096334441044 -87.0219618432572) bank136869 +136870 POINT(41.0729121843757 -87.46646978850224) bank136870 +136871 POINT(41.14483253465093 -88.62773338195868) bank136871 +136872 POINT(41.18932018678167 -87.66571336259707) bank136872 +136873 POINT(41.52475829738107 -86.68811256453372) bank136873 +136874 POINT(42.72598560535721 -87.7182948649076) bank136874 +136875 POINT(42.74781419937419 -86.74113973846336) bank136875 +136876 POINT(41.9333115648211 -87.88321121345366) bank136876 +136877 POINT(41.37040653916197 -86.79848884330684) bank136877 +136878 POINT(41.62891877495084 -87.08519123542314) bank136878 +136879 POINT(41.78016306140544 -86.63352409916817) bank136879 +136880 POINT(41.401950490019 -87.11336932147199) bank136880 +136881 POINT(41.63921092869904 -88.2831068762103) bank136881 +136882 POINT(42.02003204341384 -88.21349089722447) bank136882 +136883 POINT(41.161160236227026 -87.45764928728478) bank136883 +136884 POINT(41.905608660522766 -87.20358570094743) bank136884 +136885 POINT(42.745936309158516 -87.61835917711788) bank136885 +136886 POINT(42.637252983324245 -87.01521678947859) bank136886 +136887 POINT(41.13925615707358 -88.04382726785522) bank136887 +136888 POINT(41.31135868501061 -87.88685907427318) bank136888 +136889 POINT(42.3147366453557 -88.44961686136809) bank136889 +136890 POINT(41.34226922016276 -88.00068439868691) bank136890 +136891 POINT(41.871485238456565 -88.28543869796543) bank136891 +136892 POINT(40.881007145476886 -88.4613006032603) bank136892 +136893 POINT(41.003725004973795 -88.58164791454557) bank136893 +136894 POINT(42.017132212182844 -88.06514589837407) bank136894 +136895 POINT(40.933827371039555 -87.01819998244271) bank136895 +136896 POINT(41.32581885070138 -87.22185370296246) bank136896 +136897 POINT(41.991622434522874 -86.69076365955101) bank136897 +136898 POINT(41.12060510170607 -87.75625950877053) bank136898 +136899 POINT(41.94381121859081 -87.99630660703252) bank136899 +136900 POINT(41.96144916434781 -88.2298360298492) bank136900 +136901 POINT(41.86737716693442 -87.3691971908518) bank136901 +136902 POINT(41.586791797289116 -88.31153959771316) bank136902 +136903 POINT(42.54145198736968 -87.19961882168565) bank136903 +136904 POINT(42.53143841661399 -87.27378007201686) bank136904 +136905 POINT(40.919150414678654 -88.55754721821641) bank136905 +136906 POINT(42.645699588160376 -87.43006973202827) bank136906 +136907 POINT(42.33234940458224 -86.89868636736806) bank136907 +136908 POINT(42.40848517614057 -87.8777250420771) bank136908 +136909 POINT(42.569516000296105 -87.8741813357924) bank136909 +136910 POINT(42.66044576337195 -87.71420990461647) bank136910 +136911 POINT(41.9573756426827 -87.77787668536163) bank136911 +136912 POINT(41.89366707350945 -87.06847863546324) bank136912 +136913 POINT(42.67752383606351 -88.08658739782386) bank136913 +136914 POINT(42.756661200209564 -88.11014570276886) bank136914 +136915 POINT(41.90303404621821 -87.6138504936151) bank136915 +136916 POINT(41.57011124248565 -86.7645118326027) bank136916 +136917 POINT(42.40974156068187 -87.14910035240295) bank136917 +136918 POINT(42.100449521698806 -86.70629270780239) bank136918 +136919 POINT(42.07455392191932 -86.77830988464632) bank136919 +136920 POINT(42.15284813772336 -87.71514768763808) bank136920 +136921 POINT(41.67800086502982 -86.94216534912296) bank136921 +136922 POINT(41.1495892490767 -88.48016088645305) bank136922 +136923 POINT(42.015384059759576 -88.62268689767288) bank136923 +136924 POINT(41.80541121464083 -87.75117545140677) bank136924 +136925 POINT(42.704267377649636 -87.21500638530958) bank136925 +136926 POINT(41.182790912265034 -86.81750572199265) bank136926 +136927 POINT(41.71271886800101 -86.9089365626477) bank136927 +136928 POINT(41.86028755164445 -88.4997780379678) bank136928 +136929 POINT(42.451844648572404 -87.42803430783377) bank136929 +136930 POINT(42.04207746152532 -86.96543149521992) bank136930 +136931 POINT(42.509753035619894 -87.583166226544) bank136931 +136932 POINT(41.50833490885653 -87.3660706963885) bank136932 +136933 POINT(41.46529363609242 -87.63318250891574) bank136933 +136934 POINT(41.480676592244876 -87.87374177856195) bank136934 +136935 POINT(42.073008874485495 -88.16600540982918) bank136935 +136936 POINT(41.950222793694266 -87.98521690033107) bank136936 +136937 POINT(41.17904039826176 -87.86200978982515) bank136937 +136938 POINT(41.74498435350213 -86.73281413176329) bank136938 +136939 POINT(41.24762176881034 -88.50847802716757) bank136939 +136940 POINT(42.552976492727254 -86.93847300024639) bank136940 +136941 POINT(41.69652053526612 -88.57143312237882) bank136941 +136942 POINT(40.882362827625855 -86.96601271498949) bank136942 +136943 POINT(41.22989784233173 -86.75426975304224) bank136943 +136944 POINT(41.320670386653894 -87.99327443454668) bank136944 +136945 POINT(41.71701186752492 -86.82224070739157) bank136945 +136946 POINT(42.12130295565114 -87.6247871995717) bank136946 +136947 POINT(41.75657137284597 -88.5802571580139) bank136947 +136948 POINT(41.63756307288335 -86.6310249213872) bank136948 +136949 POINT(41.2504136817018 -88.1968196884112) bank136949 +136950 POINT(41.409622970061385 -88.47560437022335) bank136950 +136951 POINT(41.12043353184756 -87.28526883086361) bank136951 +136952 POINT(42.26163880171713 -87.95406313916865) bank136952 +136953 POINT(42.72446158363475 -87.87132138623605) bank136953 +136954 POINT(41.10946076011088 -87.39161310909414) bank136954 +136955 POINT(41.66064850975579 -87.25316615680919) bank136955 +136956 POINT(41.41863072046259 -87.03156469582903) bank136956 +136957 POINT(42.57939261804654 -86.9306171661363) bank136957 +136958 POINT(42.50903861202977 -87.99670422524645) bank136958 +136959 POINT(42.27952040270791 -86.64329722162705) bank136959 +136960 POINT(41.267302521670686 -87.90532519599826) bank136960 +136961 POINT(42.415307474013325 -87.78560847794807) bank136961 +136962 POINT(41.94098365432123 -88.05446551725923) bank136962 +136963 POINT(41.762990902906616 -86.75343420484772) bank136963 +136964 POINT(40.974971453284844 -88.09069375851634) bank136964 +136965 POINT(41.51074696553309 -88.52880295039193) bank136965 +136966 POINT(40.92895900084114 -86.68799076446356) bank136966 +136967 POINT(42.39709013041223 -88.52732192727217) bank136967 +136968 POINT(41.64310104139267 -87.12185268327349) bank136968 +136969 POINT(41.86710400590796 -86.97798456746209) bank136969 +136970 POINT(42.31325500306297 -87.83480577889786) bank136970 +136971 POINT(41.37078329202606 -87.15597404389501) bank136971 +136972 POINT(40.99141505163038 -87.57168632856389) bank136972 +136973 POINT(42.846333998252014 -87.00748913532105) bank136973 +136974 POINT(41.66794976378265 -87.93348421293364) bank136974 +136975 POINT(41.509106473556635 -88.04251860135857) bank136975 +136976 POINT(41.58397462907765 -88.37357985544698) bank136976 +136977 POINT(41.86016384300169 -88.00524670201546) bank136977 +136978 POINT(42.53288891625363 -87.18235750709408) bank136978 +136979 POINT(41.46398024779693 -88.57949051603187) bank136979 +136980 POINT(42.30362763753778 -88.09274112780292) bank136980 +136981 POINT(41.468269527888935 -87.2160664053054) bank136981 +136982 POINT(41.40157923727094 -87.25648911626408) bank136982 +136983 POINT(41.12667741958423 -88.04565994449443) bank136983 +136984 POINT(41.53124962568641 -87.9471523284545) bank136984 +136985 POINT(41.150009612454944 -88.00157776281542) bank136985 +136986 POINT(42.759720945098685 -87.70435368508211) bank136986 +136987 POINT(41.19576404423873 -87.58425900191148) bank136987 +136988 POINT(41.39382046083417 -88.14850627633768) bank136988 +136989 POINT(41.7386623509906 -86.67699031881601) bank136989 +136990 POINT(42.50426043808632 -87.92174641141688) bank136990 +136991 POINT(40.955449882063036 -88.39237608335927) bank136991 +136992 POINT(41.23675744707965 -87.32986204721176) bank136992 +136993 POINT(42.18920205869917 -86.89065207554815) bank136993 +136994 POINT(41.255962946930666 -88.41817333326544) bank136994 +136995 POINT(41.44887477792447 -88.13597396687967) bank136995 +136996 POINT(41.255451823363636 -87.99942232906709) bank136996 +136997 POINT(42.49489158907484 -86.76680112486504) bank136997 +136998 POINT(41.95645721627082 -87.9014260500004) bank136998 +136999 POINT(40.981474189003755 -88.51709982016818) bank136999 +137000 POINT(42.40030841970522 -88.36002940196626) bank137000 +137001 POINT(41.70994091093651 -87.39550252560986) bank137001 +137002 POINT(40.973516077894274 -86.72456129226806) bank137002 +137003 POINT(41.14066896082783 -88.03675808090043) bank137003 +137004 POINT(42.329460264368414 -87.81387816000905) bank137004 +137005 POINT(41.75912656440421 -86.72099742195452) bank137005 +137006 POINT(42.40685796506602 -86.9401791178359) bank137006 +137007 POINT(42.4916071335367 -87.99241073819093) bank137007 +137008 POINT(40.901792692051735 -87.69789538940552) bank137008 +137009 POINT(41.089785491430256 -87.56039975644113) bank137009 +137010 POINT(40.8830922240327 -86.89579958679889) bank137010 +137011 POINT(42.425288642447846 -88.60945591917098) bank137011 +137012 POINT(41.76543509057663 -87.67515282331497) bank137012 +137013 POINT(41.71039013502972 -86.7434464495555) bank137013 +137014 POINT(41.15293534117583 -88.01681583491805) bank137014 +137015 POINT(41.11543113767137 -87.44545657949739) bank137015 +137016 POINT(42.202769368936934 -87.52185187550923) bank137016 +137017 POINT(42.50115296192764 -87.05973340613156) bank137017 +137018 POINT(42.65248246676022 -87.49749691900092) bank137018 +137019 POINT(41.65941920322024 -87.44525743578009) bank137019 +137020 POINT(41.90629268890039 -88.01338916847443) bank137020 +137021 POINT(42.52907972184593 -87.64213033900911) bank137021 +137022 POINT(41.65778116544111 -87.77741250640743) bank137022 +137023 POINT(42.50944809035563 -87.13252654135361) bank137023 +137024 POINT(41.49760470048411 -87.88003963422527) bank137024 +137025 POINT(42.03461735500822 -87.65647679998237) bank137025 +137026 POINT(41.72136950858021 -87.55288238214354) bank137026 +137027 POINT(40.9932808912708 -87.81015698700453) bank137027 +137028 POINT(41.18137389200828 -86.81504223484177) bank137028 +137029 POINT(42.5989952063541 -87.70733581748902) bank137029 +137030 POINT(41.52311555347966 -87.11378182475976) bank137030 +137031 POINT(42.61029322338859 -87.67824483085751) bank137031 +137032 POINT(42.868668943612604 -87.65930759530488) bank137032 +137033 POINT(41.60055337328096 -86.81160680884186) bank137033 +137034 POINT(41.6297692629775 -87.61331556956897) bank137034 +137035 POINT(41.22362648015415 -88.04237762532563) bank137035 +137036 POINT(42.0658118132769 -87.40024709731125) bank137036 +137037 POINT(41.57371599088434 -88.22303116020731) bank137037 +137038 POINT(42.17002064078579 -87.53444641950684) bank137038 +137039 POINT(41.214232340038045 -87.3897450612662) bank137039 +137040 POINT(41.91614397663489 -88.55075151058593) bank137040 +137041 POINT(42.11400481564253 -88.16284103505438) bank137041 +137042 POINT(42.20407222599933 -87.57144296733537) bank137042 +137043 POINT(42.852355815976715 -87.16407414682176) bank137043 +137044 POINT(41.13573914562857 -87.71397218521277) bank137044 +137045 POINT(41.05772875876699 -87.9059210338264) bank137045 +137046 POINT(41.400528531511824 -87.77036751076116) bank137046 +137047 POINT(42.62522646264349 -88.56165202517566) bank137047 +137048 POINT(42.70292786023809 -88.35048517815173) bank137048 +137049 POINT(41.29779889964732 -88.53598993322026) bank137049 +137050 POINT(42.82968926124862 -88.0240814331299) bank137050 +137051 POINT(42.66503215435724 -87.89420088985575) bank137051 +137052 POINT(42.58566709949357 -86.73609023174326) bank137052 +137053 POINT(42.09532151638929 -88.48744276046637) bank137053 +137054 POINT(40.99699911974145 -88.43294405376857) bank137054 +137055 POINT(42.54485773087464 -87.01842164259823) bank137055 +137056 POINT(41.555588570475315 -88.56271889955748) bank137056 +137057 POINT(42.00560784876958 -88.39749559098995) bank137057 +137058 POINT(41.90259422147228 -86.67168988977238) bank137058 +137059 POINT(42.01578794770229 -87.74485463570115) bank137059 +137060 POINT(41.207135650182515 -86.76658126857129) bank137060 +137061 POINT(42.435462652093605 -87.2911335232446) bank137061 +137062 POINT(42.656414742641644 -88.29678723040854) bank137062 +137063 POINT(42.21223985270703 -87.81773958339716) bank137063 +137064 POINT(42.37578243231508 -88.24735307306976) bank137064 +137065 POINT(42.23313775102265 -87.00800597897842) bank137065 +137066 POINT(41.38849036128974 -88.42270411404542) bank137066 +137067 POINT(41.321665823586414 -87.20331773131852) bank137067 +137068 POINT(41.25822063118839 -87.8710363955373) bank137068 +137069 POINT(41.64846592944816 -88.60405595238869) bank137069 +137070 POINT(41.68964143065243 -88.10718709020317) bank137070 +137071 POINT(41.8435534475383 -86.8554439249258) bank137071 +137072 POINT(42.36708890205041 -88.44164797948977) bank137072 +137073 POINT(42.26391791419816 -87.50483017496882) bank137073 +137074 POINT(41.70791004209097 -88.51425473567356) bank137074 +137075 POINT(42.86967892969831 -86.99858830658312) bank137075 +137076 POINT(41.99878486405443 -88.5216162954403) bank137076 +137077 POINT(41.126080881366214 -87.77380616422687) bank137077 +137078 POINT(41.28201666846588 -88.43414074748047) bank137078 +137079 POINT(41.74383628506172 -87.48561854521648) bank137079 +137080 POINT(42.525899798755574 -87.78634063403048) bank137080 +137081 POINT(42.64120084971876 -87.01558641923947) bank137081 +137082 POINT(41.35818334184649 -86.8583312321796) bank137082 +137083 POINT(41.38294343355461 -87.74553115381664) bank137083 +137084 POINT(42.244998656752294 -88.31427891878697) bank137084 +137085 POINT(40.92188609390733 -87.27785506475152) bank137085 +137086 POINT(42.420677291341825 -87.05799710916402) bank137086 +137087 POINT(41.759791582570145 -88.29282049922683) bank137087 +137088 POINT(41.31666595321039 -87.08391161437365) bank137088 +137089 POINT(40.975693860501 -88.16806173723162) bank137089 +137090 POINT(41.18893701245184 -87.66357995443957) bank137090 +137091 POINT(41.91679464691831 -87.54751407078699) bank137091 +137092 POINT(40.88300654475774 -88.39711882521121) bank137092 +137093 POINT(42.13840867425266 -88.58397973292249) bank137093 +137094 POINT(41.12053514210499 -87.50236361041718) bank137094 +137095 POINT(41.59527864853245 -88.11974952008003) bank137095 +137096 POINT(42.3869868540502 -88.10447448559727) bank137096 +137097 POINT(42.161891064901 -88.11862469619149) bank137097 +137098 POINT(42.13050861097763 -87.3318095192697) bank137098 +137099 POINT(42.84984219146502 -87.5254804577631) bank137099 +137100 POINT(42.043118292902996 -87.0494152894106) bank137100 +137101 POINT(42.00709290277772 -87.06623229894953) bank137101 +137102 POINT(41.3660122084781 -86.90174456165566) bank137102 +137103 POINT(41.728810872537174 -87.12424432359317) bank137103 +137104 POINT(42.312876747014386 -86.71630392387425) bank137104 +137105 POINT(41.535727186980296 -88.47510633454276) bank137105 +137106 POINT(41.60819550592828 -88.50224312476514) bank137106 +137107 POINT(42.82281406686523 -87.63766494877368) bank137107 +137108 POINT(41.19477169398711 -88.06474749238077) bank137108 +137109 POINT(41.250509833372206 -88.37745778889169) bank137109 +137110 POINT(41.499565762777486 -87.93961220218267) bank137110 +137111 POINT(42.21435543134349 -87.42356572244883) bank137111 +137112 POINT(41.305425802966795 -87.39906069588149) bank137112 +137113 POINT(41.96364880380317 -88.11990947800857) bank137113 +137114 POINT(41.73455527981856 -88.43936310592461) bank137114 +137115 POINT(42.66524852790874 -87.57854218689012) bank137115 +137116 POINT(42.206445159053274 -87.0117937909409) bank137116 +137117 POINT(42.343948395911525 -87.21457529492103) bank137117 +137118 POINT(42.72666414151608 -87.06106140750225) bank137118 +137119 POINT(41.47742765842492 -87.05775015292025) bank137119 +137120 POINT(41.83522702446706 -88.23099306198709) bank137120 +137121 POINT(41.02985640515901 -87.13273466230959) bank137121 +137122 POINT(41.91301503374462 -88.38980580010156) bank137122 +137123 POINT(41.72823998707555 -87.35310642017808) bank137123 +137124 POINT(41.370988680425775 -86.97860701964443) bank137124 +137125 POINT(41.08490384708787 -87.34234753591996) bank137125 +137126 POINT(41.963479808704854 -88.01770254025445) bank137126 +137127 POINT(42.1432809952791 -86.93211771600312) bank137127 +137128 POINT(41.43668505231377 -88.13362037716735) bank137128 +137129 POINT(42.10896414619897 -88.1729700349091) bank137129 +137130 POINT(41.53512467853146 -88.50063504364452) bank137130 +137131 POINT(41.30684477584757 -86.7013515151511) bank137131 +137132 POINT(42.03277826708243 -86.78755130472297) bank137132 +137133 POINT(42.47555525257269 -86.63330563930539) bank137133 +137134 POINT(41.949140377397974 -86.83885912462264) bank137134 +137135 POINT(42.19216053269792 -87.99346300322352) bank137135 +137136 POINT(41.544207277468246 -88.19916393647114) bank137136 +137137 POINT(41.071113174588106 -87.84114568805947) bank137137 +137138 POINT(41.55940332042948 -87.74605184233155) bank137138 +137139 POINT(41.28405336967534 -87.41998875103987) bank137139 +137140 POINT(41.811948412512095 -86.71876729739651) bank137140 +137141 POINT(41.34193238862605 -88.34494811124381) bank137141 +137142 POINT(42.414223115733655 -88.57659271812813) bank137142 +137143 POINT(42.166934435931225 -86.88219232282522) bank137143 +137144 POINT(41.87872558929103 -87.99491108866353) bank137144 +137145 POINT(41.700053087956974 -87.21827125850001) bank137145 +137146 POINT(42.01129201959848 -88.47846258839685) bank137146 +137147 POINT(42.006780725483644 -87.85254155026382) bank137147 +137148 POINT(42.50905328729729 -86.76823124136202) bank137148 +137149 POINT(41.6271888227596 -87.97957771703899) bank137149 +137150 POINT(42.10353788767183 -87.1474738278894) bank137150 +137151 POINT(42.21122900508602 -86.6300102923599) bank137151 +137152 POINT(42.8391830432275 -87.91197912581657) bank137152 +137153 POINT(41.86728164191704 -87.27946760225771) bank137153 +137154 POINT(41.28521865357829 -88.56069331944347) bank137154 +137155 POINT(42.761086372930635 -86.95713227423624) bank137155 +137156 POINT(42.37280521329087 -87.64396676672054) bank137156 +137157 POINT(41.26314581302497 -88.48615311957256) bank137157 +137158 POINT(41.324513031859404 -86.76873696591257) bank137158 +137159 POINT(42.207245370215574 -87.58277068511285) bank137159 +137160 POINT(42.582132892405866 -88.19726469376857) bank137160 +137161 POINT(42.66885264808762 -87.67264578416189) bank137161 +137162 POINT(42.09733202082345 -86.84289399277928) bank137162 +137163 POINT(41.64664498642643 -88.15450692803599) bank137163 +137164 POINT(42.48006319697562 -86.74478114947568) bank137164 +137165 POINT(42.15071996684755 -86.89617570081342) bank137165 +137166 POINT(41.96223434776464 -87.52371818108766) bank137166 +137167 POINT(41.14449600704942 -88.36445985168203) bank137167 +137168 POINT(41.3473583935901 -88.410093066365) bank137168 +137169 POINT(42.79962100755902 -88.52329059498256) bank137169 +137170 POINT(42.71879641917955 -87.49570162404763) bank137170 +137171 POINT(41.31077216858229 -87.00892878941914) bank137171 +137172 POINT(40.947117996461195 -87.48805371747594) bank137172 +137173 POINT(41.79972395183176 -88.00775087178492) bank137173 +137174 POINT(41.655563393454024 -87.23351630220105) bank137174 +137175 POINT(41.15116962523352 -88.00740073488284) bank137175 +137176 POINT(42.56187815318714 -87.43158587594175) bank137176 +137177 POINT(42.78630357542569 -87.1625795837282) bank137177 +137178 POINT(41.9867989410665 -88.30267821682833) bank137178 +137179 POINT(42.39195048416907 -87.56644719189498) bank137179 +137180 POINT(40.968211217010484 -87.11374381831081) bank137180 +137181 POINT(42.32366501544339 -87.49476513491398) bank137181 +137182 POINT(42.6784262078634 -88.25241847962367) bank137182 +137183 POINT(42.128492665610956 -88.52233795348226) bank137183 +137184 POINT(42.21213373712913 -87.1516373509382) bank137184 +137185 POINT(42.58498181486662 -88.44354739316202) bank137185 +137186 POINT(42.741645061621625 -86.69098532079359) bank137186 +137187 POINT(41.18066919385683 -88.0440863462796) bank137187 +137188 POINT(41.876474143937024 -87.76201390329265) bank137188 +137189 POINT(41.123543084312956 -88.12208361112391) bank137189 +137190 POINT(42.34279533025782 -86.8370518586889) bank137190 +137191 POINT(42.733582320198465 -87.91772008478888) bank137191 +137192 POINT(41.24436808873498 -87.51417452648387) bank137192 +137193 POINT(42.27289923254406 -88.2176654040825) bank137193 +137194 POINT(41.80929051243388 -88.5074169414582) bank137194 +137195 POINT(42.5926189215269 -87.46569503870889) bank137195 +137196 POINT(42.5422152014815 -86.98090909724822) bank137196 +137197 POINT(40.89767127432925 -86.74130689910237) bank137197 +137198 POINT(41.865422722017854 -87.8197830504058) bank137198 +137199 POINT(40.96332694013173 -87.25805125016575) bank137199 +137200 POINT(42.25423594604616 -88.04921881231634) bank137200 +137201 POINT(41.28029062568061 -87.87473915348346) bank137201 +137202 POINT(41.016624600838895 -87.34667791483159) bank137202 +137203 POINT(41.846230655581586 -87.38119010371959) bank137203 +137204 POINT(40.98180728629447 -87.96776526693813) bank137204 +137205 POINT(42.0050425365679 -88.3870339908546) bank137205 +137206 POINT(41.00246989095562 -87.56291244064016) bank137206 +137207 POINT(41.634922985215276 -87.01264927209509) bank137207 +137208 POINT(40.89639582714784 -88.46994967348985) bank137208 +137209 POINT(42.384553190647196 -87.59653826475085) bank137209 +137210 POINT(41.81006963101978 -87.20129499467242) bank137210 +137211 POINT(42.514217739510904 -88.54527811320699) bank137211 +137212 POINT(42.30970586118833 -86.92604361607624) bank137212 +137213 POINT(41.1057294645773 -87.88534156492133) bank137213 +137214 POINT(40.878355579524175 -88.344254578216) bank137214 +137215 POINT(41.24975410944753 -88.61964505376416) bank137215 +137216 POINT(42.30352146456384 -87.69745710609945) bank137216 +137217 POINT(41.796725679456024 -87.09464636853322) bank137217 +137218 POINT(41.682931844219844 -88.19206221141093) bank137218 +137219 POINT(41.09153280513046 -87.35596463275199) bank137219 +137220 POINT(41.467560575771934 -88.39594890050324) bank137220 +137221 POINT(42.8248146590361 -87.27051221779341) bank137221 +137222 POINT(41.40893476753717 -88.25093647023535) bank137222 +137223 POINT(42.48013201096055 -86.7434760071785) bank137223 +137224 POINT(40.93294122238404 -87.09529620391794) bank137224 +137225 POINT(40.90444884459342 -88.31547269013637) bank137225 +137226 POINT(42.006521171156855 -86.85991402824297) bank137226 +137227 POINT(42.68300254573731 -87.20333182286286) bank137227 +137228 POINT(42.598631961085125 -88.32292128422321) bank137228 +137229 POINT(42.53624661857607 -87.24429227114871) bank137229 +137230 POINT(42.37029618150903 -87.96493626574822) bank137230 +137231 POINT(41.64454349236204 -86.81829601570269) bank137231 +137232 POINT(42.08675347268684 -87.06196978058406) bank137232 +137233 POINT(41.182088842478734 -88.21331240845652) bank137233 +137234 POINT(41.18883865985181 -86.71165376305565) bank137234 +137235 POINT(42.69207456080668 -88.19408881058052) bank137235 +137236 POINT(41.79397241626335 -87.00637480809576) bank137236 +137237 POINT(41.35597423037173 -86.86045817971097) bank137237 +137238 POINT(42.485256969762 -86.96685524372838) bank137238 +137239 POINT(41.64964737196499 -88.44618094057267) bank137239 +137240 POINT(41.5838898451894 -87.73930967102682) bank137240 +137241 POINT(42.71745297572057 -87.34086484123313) bank137241 +137242 POINT(42.270594068581225 -87.71220329405983) bank137242 +137243 POINT(42.714611722077386 -87.5745328728723) bank137243 +137244 POINT(41.52322126343208 -86.90176719830747) bank137244 +137245 POINT(42.48062968298937 -88.40650014450934) bank137245 +137246 POINT(40.883059777013514 -87.87549750433702) bank137246 +137247 POINT(40.94207134642315 -87.79391742405754) bank137247 +137248 POINT(41.50671003989032 -86.82526021278537) bank137248 +137249 POINT(41.02174159640577 -87.32067393077276) bank137249 +137250 POINT(41.25728421536633 -88.23058700620624) bank137250 +137251 POINT(41.03878398307261 -88.02724929953688) bank137251 +137252 POINT(41.847937683907425 -87.88024082002885) bank137252 +137253 POINT(42.789603393068305 -87.89944804091085) bank137253 +137254 POINT(41.790907960785304 -87.23950175202191) bank137254 +137255 POINT(42.582749881119916 -88.2498841844206) bank137255 +137256 POINT(41.92932812077529 -87.45557306745323) bank137256 +137257 POINT(41.543030819446926 -88.58603557058558) bank137257 +137258 POINT(42.33494029564713 -87.80556903741511) bank137258 +137259 POINT(41.98630684244238 -87.49815536657232) bank137259 +137260 POINT(42.727544088161274 -88.0874796688358) bank137260 +137261 POINT(42.01333061943439 -87.82186109928709) bank137261 +137262 POINT(42.697475224698024 -86.92137593187829) bank137262 +137263 POINT(42.524643472820536 -88.29069046255862) bank137263 +137264 POINT(40.98328656368605 -87.94173622395255) bank137264 +137265 POINT(41.273412053896564 -87.7318001212685) bank137265 +137266 POINT(41.092202478383896 -86.79050554494462) bank137266 +137267 POINT(40.957244600403804 -88.45052471177678) bank137267 +137268 POINT(42.57920252114501 -88.40147646246999) bank137268 +137269 POINT(41.52831447951108 -86.76092650975431) bank137269 +137270 POINT(41.2870462046801 -88.48278756242917) bank137270 +137271 POINT(41.189409548747946 -87.59353106985817) bank137271 +137272 POINT(41.666560452210376 -87.62501024415455) bank137272 +137273 POINT(41.99449713717842 -86.81091058891593) bank137273 +137274 POINT(41.217004954059526 -87.6129485982978) bank137274 +137275 POINT(42.60351390700481 -88.01049288198817) bank137275 +137276 POINT(42.729840271253785 -87.5350375331849) bank137276 +137277 POINT(41.1635671491163 -86.99345128924386) bank137277 +137278 POINT(42.84022265207125 -88.11964190808229) bank137278 +137279 POINT(41.78004741508083 -88.07118623484585) bank137279 +137280 POINT(41.797195000151206 -87.25508841942896) bank137280 +137281 POINT(41.63263208227361 -87.77495960308022) bank137281 +137282 POINT(41.59626538150702 -88.51269619385164) bank137282 +137283 POINT(42.828072841036494 -88.32370747698111) bank137283 +137284 POINT(41.21882392153434 -88.020936161434) bank137284 +137285 POINT(41.68704136882858 -87.80720099455574) bank137285 +137286 POINT(42.45679291162101 -87.49942746045012) bank137286 +137287 POINT(41.86619272290994 -86.79116699374698) bank137287 +137288 POINT(41.47761612753417 -87.22922336933465) bank137288 +137289 POINT(42.03333918870857 -86.95180345448603) bank137289 +137290 POINT(41.70437159715433 -88.27745288529833) bank137290 +137291 POINT(42.00962506828447 -87.0393319716909) bank137291 +137292 POINT(42.66230423782951 -86.8030879398106) bank137292 +137293 POINT(42.40623703624542 -87.02981115159652) bank137293 +137294 POINT(42.04964866093066 -87.04763734003166) bank137294 +137295 POINT(41.745010664517295 -86.72403792000179) bank137295 +137296 POINT(41.6577463490183 -87.86343208600583) bank137296 +137297 POINT(41.37065948831729 -87.29609882944395) bank137297 +137298 POINT(42.277306232723724 -88.53776777941837) bank137298 +137299 POINT(40.87860057986418 -87.92622130392525) bank137299 +137300 POINT(41.96683332339798 -88.24576790339474) bank137300 +137301 POINT(41.975174022478974 -88.24356441133392) bank137301 +137302 POINT(42.17814526487201 -87.70024908909596) bank137302 +137303 POINT(41.82675324578669 -87.24036811696249) bank137303 +137304 POINT(41.614028039847454 -86.7754384165087) bank137304 +137305 POINT(41.148370907284516 -88.10099180241903) bank137305 +137306 POINT(42.437387608889644 -87.18283598855197) bank137306 +137307 POINT(42.09223463595563 -86.87949617564261) bank137307 +137308 POINT(41.65608102532388 -87.57380458280997) bank137308 +137309 POINT(41.82449262759022 -87.5267098394863) bank137309 +137310 POINT(42.292578097820176 -86.74300765824239) bank137310 +137311 POINT(40.99057706422607 -87.51619480069901) bank137311 +137312 POINT(41.140705853422524 -88.23717337562276) bank137312 +137313 POINT(42.11701240293074 -87.29707279490944) bank137313 +137314 POINT(41.40739209416588 -87.10420348962307) bank137314 +137315 POINT(42.063879118938466 -88.36005859444779) bank137315 +137316 POINT(41.95814109683775 -86.91121722622765) bank137316 +137317 POINT(41.622384136452844 -87.71505835401814) bank137317 +137318 POINT(41.14335361703454 -87.659614723897) bank137318 +137319 POINT(42.16919588279169 -87.54866352095402) bank137319 +137320 POINT(41.2422906130815 -88.25660925976437) bank137320 +137321 POINT(41.11892139956627 -88.12816289296924) bank137321 +137322 POINT(40.883947465789845 -87.90796532151187) bank137322 +137323 POINT(41.566415475062755 -88.4049144903217) bank137323 +137324 POINT(41.11983976524341 -88.24702657830568) bank137324 +137325 POINT(42.57183765989886 -88.37468466228145) bank137325 +137326 POINT(41.359332874301174 -88.60451567184258) bank137326 +137327 POINT(42.6966626674231 -87.25301901191035) bank137327 +137328 POINT(42.384078235743836 -86.78147372252218) bank137328 +137329 POINT(41.522903441799016 -87.6458228391577) bank137329 +137330 POINT(41.35476540899701 -86.74474706424033) bank137330 +137331 POINT(41.09012281262585 -88.10696216203048) bank137331 +137332 POINT(41.98188520204263 -87.76590633209277) bank137332 +137333 POINT(42.3845375648206 -87.05050947045326) bank137333 +137334 POINT(41.154941292440085 -86.99148280850821) bank137334 +137335 POINT(41.6599825777703 -86.79870267570801) bank137335 +137336 POINT(41.43035457345205 -88.59476854791444) bank137336 +137337 POINT(41.00543001726547 -88.17158476552315) bank137337 +137338 POINT(41.58415178075399 -87.41448072544027) bank137338 +137339 POINT(42.77624212953104 -87.1941878227339) bank137339 +137340 POINT(41.5707756739949 -88.4084357107394) bank137340 +137341 POINT(41.962064230535844 -86.69399898554552) bank137341 +137342 POINT(41.01489887456427 -88.03396188017946) bank137342 +137343 POINT(41.80708160217813 -87.606091325551) bank137343 +137344 POINT(42.32987374323941 -86.76151436803956) bank137344 +137345 POINT(42.148790386223105 -86.82253907463445) bank137345 +137346 POINT(41.73304393608138 -87.84212384230014) bank137346 +137347 POINT(42.11237325019504 -88.17797261342307) bank137347 +137348 POINT(42.00264161928908 -86.94577930514765) bank137348 +137349 POINT(42.304433094070006 -87.01770083432793) bank137349 +137350 POINT(42.06445126891893 -86.87090365740798) bank137350 +137351 POINT(42.327711656025265 -87.97801067996052) bank137351 +137352 POINT(41.508523356458724 -86.76662829461947) bank137352 +137353 POINT(41.76530768457991 -88.5279056354552) bank137353 +137354 POINT(41.569782648225086 -88.02457519969091) bank137354 +137355 POINT(42.10436289524775 -87.32604414651269) bank137355 +137356 POINT(41.93786890185994 -86.90618228186692) bank137356 +137357 POINT(41.35287155150102 -88.16884048827517) bank137357 +137358 POINT(41.5369208025753 -87.05394761400325) bank137358 +137359 POINT(42.160697074638044 -86.92131757200376) bank137359 +137360 POINT(41.631933334562596 -87.79338168100023) bank137360 +137361 POINT(41.49777687851203 -88.30882661975856) bank137361 +137362 POINT(41.02218355481135 -88.39829364803025) bank137362 +137363 POINT(42.3646866218726 -87.42963457486672) bank137363 +137364 POINT(42.855291492122085 -87.4740303043558) bank137364 +137365 POINT(41.56680875002543 -87.41259723985203) bank137365 +137366 POINT(42.38320444128559 -87.05095296291637) bank137366 +137367 POINT(41.379542222493264 -87.17447897517958) bank137367 +137368 POINT(41.22026022155627 -87.91135957666081) bank137368 +137369 POINT(42.644739636029065 -86.87286278759042) bank137369 +137370 POINT(41.00386019601514 -87.56110752188525) bank137370 +137371 POINT(41.650949462149136 -87.42445942931865) bank137371 +137372 POINT(41.5316584489344 -86.73408314706093) bank137372 +137373 POINT(42.830009305248396 -88.51728759797332) bank137373 +137374 POINT(42.51068306586171 -86.71816663794662) bank137374 +137375 POINT(41.34405602546085 -86.63086708859869) bank137375 +137376 POINT(42.83102417399307 -87.39518690463893) bank137376 +137377 POINT(41.31593675832483 -87.08660564955343) bank137377 +137378 POINT(41.855416942260895 -86.7300702258526) bank137378 +137379 POINT(42.304175244205396 -88.41436536416781) bank137379 +137380 POINT(42.49698982919957 -88.53806527314498) bank137380 +137381 POINT(41.507727214214135 -88.32156673254136) bank137381 +137382 POINT(40.882060996955815 -87.99480364943065) bank137382 +137383 POINT(41.1631073972735 -87.03945498189323) bank137383 +137384 POINT(41.96442638243345 -87.44775158694404) bank137384 +137385 POINT(41.22632329743029 -87.64431538186928) bank137385 +137386 POINT(42.87356522189602 -87.40723302212729) bank137386 +137387 POINT(42.58789887511442 -86.7460200830133) bank137387 +137388 POINT(41.45130189084833 -86.88415685558198) bank137388 +137389 POINT(41.415306790564756 -87.43397697786484) bank137389 +137390 POINT(41.92572947769641 -87.57534754379583) bank137390 +137391 POINT(41.114512573091375 -87.7294478452001) bank137391 +137392 POINT(41.26263679112916 -88.60138480994739) bank137392 +137393 POINT(42.5713109923296 -88.06406540845369) bank137393 +137394 POINT(41.64663858758053 -88.25037008381412) bank137394 +137395 POINT(42.80128447232983 -88.0294524921263) bank137395 +137396 POINT(42.26354582413197 -88.0111342030264) bank137396 +137397 POINT(40.95568183621275 -86.63122819650184) bank137397 +137398 POINT(42.72407154299702 -87.75384853129539) bank137398 +137399 POINT(41.34956286573856 -87.73364079756665) bank137399 +137400 POINT(41.1551165574002 -87.16721095376819) bank137400 +137401 POINT(41.11367675043597 -87.70157498440575) bank137401 +137402 POINT(41.582596934254724 -87.22366324081786) bank137402 +137403 POINT(42.702311653875874 -88.10383208059727) bank137403 +137404 POINT(42.26872759568254 -87.7338180245436) bank137404 +137405 POINT(41.367154735335376 -86.80097857373157) bank137405 +137406 POINT(41.801070585594005 -87.85056714321169) bank137406 +137407 POINT(42.800643268683444 -87.1165854551472) bank137407 +137408 POINT(42.847907778926746 -88.41447581254678) bank137408 +137409 POINT(42.162769890696985 -87.41462095686009) bank137409 +137410 POINT(41.05705411645828 -87.94136962361148) bank137410 +137411 POINT(41.26114317280207 -87.65405772296039) bank137411 +137412 POINT(42.61771954063157 -87.6956024715327) bank137412 +137413 POINT(41.237510179499075 -87.6105512291198) bank137413 +137414 POINT(42.07488229224875 -87.89391848953407) bank137414 +137415 POINT(42.30660580492472 -87.00543013039342) bank137415 +137416 POINT(42.14784897486042 -88.24918785600188) bank137416 +137417 POINT(41.44181354844514 -88.06375645208881) bank137417 +137418 POINT(41.166534782524046 -88.1805898499382) bank137418 +137419 POINT(42.41015170809274 -86.83441989128889) bank137419 +137420 POINT(41.97332233116867 -86.89975823218435) bank137420 +137421 POINT(42.72992974146743 -86.99524436331836) bank137421 +137422 POINT(42.191282708827906 -86.96075333608289) bank137422 +137423 POINT(41.447171881901035 -86.99027489715505) bank137423 +137424 POINT(41.890223629734194 -87.96576356417346) bank137424 +137425 POINT(42.470989756852326 -87.32371238261325) bank137425 +137426 POINT(41.85028538392152 -86.78792796286974) bank137426 +137427 POINT(41.296345759072715 -87.02951351260114) bank137427 +137428 POINT(42.181941559462565 -86.81099983038374) bank137428 +137429 POINT(40.99533026214479 -88.39227221896518) bank137429 +137430 POINT(42.119075885151226 -88.20702898121982) bank137430 +137431 POINT(42.0593470175263 -86.94679376826765) bank137431 +137432 POINT(42.0227664108579 -86.65168507460619) bank137432 +137433 POINT(41.635329485341096 -86.6771566318137) bank137433 +137434 POINT(41.28200721113858 -87.09101652623386) bank137434 +137435 POINT(42.141051003636264 -87.86396248791405) bank137435 +137436 POINT(41.64750216533412 -88.45079866458728) bank137436 +137437 POINT(41.32889050595179 -87.5883163534788) bank137437 +137438 POINT(41.813217347634925 -88.62267964825084) bank137438 +137439 POINT(41.33684743180734 -87.3485486006158) bank137439 +137440 POINT(41.01196394440783 -88.02288246011649) bank137440 +137441 POINT(42.52583686315397 -87.60229889099436) bank137441 +137442 POINT(42.189888522154 -86.68935364327069) bank137442 +137443 POINT(42.676809780606106 -86.68433567976066) bank137443 +137444 POINT(42.240763729131224 -87.96487501381564) bank137444 +137445 POINT(42.67566560882385 -86.66178306163123) bank137445 +137446 POINT(41.073858932491866 -88.62560838904768) bank137446 +137447 POINT(42.06169117514849 -86.6467385227203) bank137447 +137448 POINT(42.47959923647629 -87.43720477797658) bank137448 +137449 POINT(41.78938462079338 -87.65383512602453) bank137449 +137450 POINT(41.862217178770436 -88.22746086758163) bank137450 +137451 POINT(41.21015708727709 -86.96894146766827) bank137451 +137452 POINT(42.29252073485364 -86.78627463378922) bank137452 +137453 POINT(41.77145988844122 -88.2739890838336) bank137453 +137454 POINT(41.725655574037226 -88.01315853193294) bank137454 +137455 POINT(41.605570902271744 -88.01472178739708) bank137455 +137456 POINT(41.22116294566575 -87.13943278118523) bank137456 +137457 POINT(41.2940903553059 -86.8794139369168) bank137457 +137458 POINT(41.9742190140734 -88.3188035553175) bank137458 +137459 POINT(42.84952082399763 -87.1528578866275) bank137459 +137460 POINT(42.2710008729048 -86.97035701896347) bank137460 +137461 POINT(42.612963646058724 -87.61216169616968) bank137461 +137462 POINT(42.1640745812679 -88.43873470552319) bank137462 +137463 POINT(42.49239800964813 -87.0759822693588) bank137463 +137464 POINT(41.966532153076486 -87.30297971534134) bank137464 +137465 POINT(41.44646627961688 -87.67962870582322) bank137465 +137466 POINT(42.42150115050524 -87.0058217693313) bank137466 +137467 POINT(42.5065960217181 -87.99846789125036) bank137467 +137468 POINT(42.27711173140428 -87.10090767904357) bank137468 +137469 POINT(42.23153694960594 -86.74909228924085) bank137469 +137470 POINT(40.90149584309753 -86.993700438862) bank137470 +137471 POINT(41.42849081139331 -86.87331666029537) bank137471 +137472 POINT(42.101025297567986 -86.72768535298901) bank137472 +137473 POINT(40.88597044803924 -88.02689834752918) bank137473 +137474 POINT(42.207819365580406 -88.45531872156154) bank137474 +137475 POINT(42.56862142245391 -86.83593336529385) bank137475 +137476 POINT(41.22548389101454 -87.44061450943379) bank137476 +137477 POINT(42.53566499866545 -87.11512996958626) bank137477 +137478 POINT(41.16114183767256 -87.43155881984022) bank137478 +137479 POINT(41.40565354893505 -88.17336828261998) bank137479 +137480 POINT(42.756141156704196 -87.98445877096272) bank137480 +137481 POINT(42.340201833204766 -87.47999270362374) bank137481 +137482 POINT(42.52862419835132 -88.62592506526168) bank137482 +137483 POINT(41.67020869709202 -86.98928467936958) bank137483 +137484 POINT(41.72204608220186 -87.5950272921707) bank137484 +137485 POINT(41.10304134508575 -87.62635032128117) bank137485 +137486 POINT(42.8083377079878 -86.81628917939535) bank137486 +137487 POINT(41.74827628257763 -86.83317761366739) bank137487 +137488 POINT(42.73988512881867 -88.10045065485254) bank137488 +137489 POINT(41.570735542640506 -87.97313281035576) bank137489 +137490 POINT(41.54912762152283 -87.38160664467253) bank137490 +137491 POINT(42.307276802636636 -87.51577225732214) bank137491 +137492 POINT(42.08405614878678 -88.43057150243408) bank137492 +137493 POINT(42.715579352196755 -88.23268415888563) bank137493 +137494 POINT(41.97685792991881 -86.97252735419268) bank137494 +137495 POINT(42.37552273526693 -86.76358843991203) bank137495 +137496 POINT(41.40528597525551 -87.50298020344432) bank137496 +137497 POINT(41.77194837847951 -88.44037783044244) bank137497 +137498 POINT(41.05372489556158 -87.37208301480926) bank137498 +137499 POINT(42.216433661470184 -88.17100029916205) bank137499 +137500 POINT(40.92664901361589 -87.37677501130858) bank137500 +137501 POINT(41.27230057473824 -87.94543892658272) bank137501 +137502 POINT(41.108733900836036 -88.18049145352907) bank137502 +137503 POINT(41.028850096451045 -87.20260545242562) bank137503 +137504 POINT(40.93368363814481 -87.34631083658793) bank137504 +137505 POINT(42.1231318152145 -87.27922956757322) bank137505 +137506 POINT(40.99873127329291 -87.82120323202543) bank137506 +137507 POINT(41.82644539111105 -88.47589607962557) bank137507 +137508 POINT(42.76968416971691 -87.80671661907297) bank137508 +137509 POINT(40.89734841744466 -86.84841505810238) bank137509 +137510 POINT(41.894639305259545 -87.67337909737587) bank137510 +137511 POINT(40.97813904543531 -86.72211363106517) bank137511 +137512 POINT(41.35795546825381 -88.36562265377134) bank137512 +137513 POINT(40.88786950096897 -87.10766765967026) bank137513 +137514 POINT(40.888474926094716 -87.97505782965307) bank137514 +137515 POINT(41.09217365844377 -87.57522625038017) bank137515 +137516 POINT(42.06058505633054 -88.00911666444871) bank137516 +137517 POINT(40.934672482294275 -87.72146230552329) bank137517 +137518 POINT(41.47270435981406 -87.9078491035886) bank137518 +137519 POINT(42.80033951633535 -88.06439966609716) bank137519 +137520 POINT(42.73951012206315 -86.81100052426208) bank137520 +137521 POINT(41.03517460550386 -87.35197491513496) bank137521 +137522 POINT(42.18506175503022 -88.60268690828862) bank137522 +137523 POINT(41.80058181533415 -87.74254486905946) bank137523 +137524 POINT(41.75083018422302 -86.73501370218109) bank137524 +137525 POINT(42.07835408215377 -87.1919565000915) bank137525 +137526 POINT(41.32113571478931 -87.67795824173658) bank137526 +137527 POINT(42.654202585348635 -87.58620096576213) bank137527 +137528 POINT(41.380667121393586 -87.99064725374669) bank137528 +137529 POINT(41.71507732096324 -87.16840984762166) bank137529 +137530 POINT(41.82017970638545 -87.15600415035631) bank137530 +137531 POINT(41.25893780023897 -87.80195264605486) bank137531 +137532 POINT(42.703032104578135 -87.72257173014714) bank137532 +137533 POINT(42.376615263746864 -86.85902115251228) bank137533 +137534 POINT(42.451584405324176 -86.65113085116448) bank137534 +137535 POINT(42.76416464778844 -88.03319672238823) bank137535 +137536 POINT(41.8853314095768 -88.49986353378843) bank137536 +137537 POINT(42.87306565903363 -87.90559651020354) bank137537 +137538 POINT(41.178205775759814 -87.31583250935049) bank137538 +137539 POINT(42.014453061314825 -88.32193514408544) bank137539 +137540 POINT(42.73318159123325 -87.51362798514373) bank137540 +137541 POINT(41.84537291286907 -87.15704887151878) bank137541 +137542 POINT(42.17468441375696 -88.31444293636333) bank137542 +137543 POINT(41.69696406329419 -87.65065480249528) bank137543 +137544 POINT(42.79176038694935 -87.05014807158197) bank137544 +137545 POINT(41.01942122453155 -87.53109224657771) bank137545 +137546 POINT(41.6864511423047 -86.89303636069471) bank137546 +137547 POINT(42.54881199646217 -88.03708782612128) bank137547 +137548 POINT(42.713491956183844 -86.97978293408421) bank137548 +137549 POINT(41.12327083880369 -87.00220616032252) bank137549 +137550 POINT(41.95805402476694 -87.01642239539375) bank137550 +137551 POINT(42.07427919319273 -88.42814136924862) bank137551 +137552 POINT(41.73590043836927 -87.09485074018517) bank137552 +137553 POINT(41.25233122573457 -86.87904902063191) bank137553 +137554 POINT(42.78552291472269 -87.64401795004495) bank137554 +137555 POINT(41.48642103461563 -87.9364679307507) bank137555 +137556 POINT(40.91692423367826 -86.85078083416575) bank137556 +137557 POINT(40.99081909604029 -87.05645325858508) bank137557 +137558 POINT(42.75737457459092 -88.21390365304542) bank137558 +137559 POINT(41.90748725619321 -87.56429339917527) bank137559 +137560 POINT(41.52624879510209 -86.6962713429213) bank137560 +137561 POINT(41.518443541529315 -86.72399527268449) bank137561 +137562 POINT(41.00535867337634 -87.4191282335653) bank137562 +137563 POINT(41.395410448863814 -88.1609966865804) bank137563 +137564 POINT(42.192783508865475 -87.63864893334866) bank137564 +137565 POINT(42.854091003011035 -88.06695950026665) bank137565 +137566 POINT(41.95437520715423 -88.07264392068367) bank137566 +137567 POINT(42.074969564898325 -88.60791054810507) bank137567 +137568 POINT(41.241825382725196 -87.9336164497654) bank137568 +137569 POINT(41.872771600661785 -87.78122462101938) bank137569 +137570 POINT(42.24889886727141 -86.87329228996829) bank137570 +137571 POINT(40.915299609303645 -87.22330218720883) bank137571 +137572 POINT(42.48350422797515 -87.43982775373875) bank137572 +137573 POINT(42.69725572006323 -87.19724369426112) bank137573 +137574 POINT(41.94626473621915 -86.80768895137868) bank137574 +137575 POINT(41.2415988047351 -87.34104621043407) bank137575 +137576 POINT(40.89699839273409 -87.75900180085303) bank137576 +137577 POINT(41.0351357783226 -87.09449835555031) bank137577 +137578 POINT(42.48275687494757 -87.20229399864961) bank137578 +137579 POINT(41.42410261104934 -88.29290417753779) bank137579 +137580 POINT(41.35739429351326 -87.23955511896092) bank137580 +137581 POINT(41.83155994567276 -88.50683257671096) bank137581 +137582 POINT(40.97881624790445 -87.28759655039454) bank137582 +137583 POINT(41.56038698642186 -87.05005405512013) bank137583 +137584 POINT(41.35070046346017 -88.36269706097896) bank137584 +137585 POINT(42.16055961299748 -87.36565932926075) bank137585 +137586 POINT(41.319967049012185 -87.29688332675644) bank137586 +137587 POINT(41.43447286539306 -87.11916862481259) bank137587 +137588 POINT(40.9906093580774 -87.95873591361956) bank137588 +137589 POINT(41.48579373428465 -87.9551490757364) bank137589 +137590 POINT(41.79846156015377 -87.16874593368136) bank137590 +137591 POINT(41.9990264461901 -87.764761418092) bank137591 +137592 POINT(41.69574934422743 -86.8816068819698) bank137592 +137593 POINT(42.49514019762764 -87.87366610843257) bank137593 +137594 POINT(42.28488284492734 -87.62970790944908) bank137594 +137595 POINT(41.81702147546989 -87.60276574410666) bank137595 +137596 POINT(41.16682418586628 -87.9914150855373) bank137596 +137597 POINT(41.66740615876152 -88.18934519590884) bank137597 +137598 POINT(42.19182477656578 -87.77550427732457) bank137598 +137599 POINT(42.36338415328717 -87.95810046879112) bank137599 +137600 POINT(41.38900943886436 -87.66373290055745) bank137600 +137601 POINT(41.296744815895075 -87.87290048683415) bank137601 +137602 POINT(41.053553661390914 -87.80546788764961) bank137602 +137603 POINT(42.707987496460085 -88.1500904156559) bank137603 +137604 POINT(42.58695442100346 -87.36891384849628) bank137604 +137605 POINT(42.38699442972441 -87.0109608889723) bank137605 +137606 POINT(41.48696557807172 -88.42272247727797) bank137606 +137607 POINT(41.30229191723472 -87.36088608725593) bank137607 +137608 POINT(41.680460324922834 -87.48493714199851) bank137608 +137609 POINT(40.996259741215326 -87.30793288871811) bank137609 +137610 POINT(40.9740214185237 -86.64184193188078) bank137610 +137611 POINT(41.754361726512386 -88.47557788558586) bank137611 +137612 POINT(42.76406139766389 -88.10161379408538) bank137612 +137613 POINT(42.54214735287515 -87.70984843208936) bank137613 +137614 POINT(42.50700376427545 -87.66149449818651) bank137614 +137615 POINT(41.381427516779894 -87.82934597623158) bank137615 +137616 POINT(41.27992847144308 -88.56335282926801) bank137616 +137617 POINT(40.90761087409362 -87.61657011464426) bank137617 +137618 POINT(41.21697936946359 -88.00500851944926) bank137618 +137619 POINT(42.3153213265649 -86.87326087350706) bank137619 +137620 POINT(40.98640427930084 -88.49890412221782) bank137620 +137621 POINT(42.21525440585553 -88.6246977258206) bank137621 +137622 POINT(41.50664135911406 -88.11454849220863) bank137622 +137623 POINT(41.99712994162096 -86.87835287917649) bank137623 +137624 POINT(41.46552332547488 -87.51507391850825) bank137624 +137625 POINT(41.4642073336988 -87.61943393056464) bank137625 +137626 POINT(41.767420275206675 -88.04798126694953) bank137626 +137627 POINT(42.75804682283351 -87.48331569447875) bank137627 +137628 POINT(42.53153514933672 -88.04539734790345) bank137628 +137629 POINT(41.680315177467776 -87.60439246976242) bank137629 +137630 POINT(41.572736134435665 -86.81344596803471) bank137630 +137631 POINT(41.86118944176212 -88.15481572862174) bank137631 +137632 POINT(42.006209932794256 -86.77771234368863) bank137632 +137633 POINT(40.88800684012885 -87.7787886522186) bank137633 +137634 POINT(41.934402073626146 -87.68337570343523) bank137634 +137635 POINT(42.03265685787486 -87.11026664946145) bank137635 +137636 POINT(41.116993036126935 -86.80547225746533) bank137636 +137637 POINT(41.97059450495209 -87.92069994010927) bank137637 +137638 POINT(40.978633121728095 -86.84754290464684) bank137638 +137639 POINT(42.19235725990883 -86.76111965664018) bank137639 +137640 POINT(41.43446125389695 -87.26800696259224) bank137640 +137641 POINT(42.191091023399274 -87.49967566985171) bank137641 +137642 POINT(41.33947265197932 -87.79351507264165) bank137642 +137643 POINT(41.43454613513061 -87.43095222622762) bank137643 +137644 POINT(41.81747682898592 -87.22085493189769) bank137644 +137645 POINT(41.10756731527721 -88.0862921274976) bank137645 +137646 POINT(41.38895205092571 -87.16538640652591) bank137646 +137647 POINT(42.120452323954126 -88.43446925260601) bank137647 +137648 POINT(41.5804853883888 -87.38981514796887) bank137648 +137649 POINT(41.80088311897148 -87.90757121716172) bank137649 +137650 POINT(42.80231900539891 -87.50739076560676) bank137650 +137651 POINT(42.03223856112562 -88.59906496833686) bank137651 +137652 POINT(41.1149811886357 -87.9324683057427) bank137652 +137653 POINT(42.81697297383051 -87.2055072253182) bank137653 +137654 POINT(41.13034588477349 -87.77441093699818) bank137654 +137655 POINT(41.808559948381976 -87.2705816568139) bank137655 +137656 POINT(41.68354731107267 -87.25305151541724) bank137656 +137657 POINT(42.359614457287144 -87.37554304570857) bank137657 +137658 POINT(41.86834315546919 -88.5001035204042) bank137658 +137659 POINT(42.7088109634503 -88.1355766089261) bank137659 +137660 POINT(42.07558544582482 -87.40265867550832) bank137660 +137661 POINT(42.062061862138364 -88.22302481513421) bank137661 +137662 POINT(41.398769444222026 -87.55570001181071) bank137662 +137663 POINT(41.66028494341201 -87.8279153422706) bank137663 +137664 POINT(42.233207495432374 -87.68224296820969) bank137664 +137665 POINT(41.6088209137784 -87.18055062800198) bank137665 +137666 POINT(42.46209178222404 -88.24746448563444) bank137666 +137667 POINT(42.371845479223225 -87.28931481647005) bank137667 +137668 POINT(41.287010234937085 -87.99336531395315) bank137668 +137669 POINT(40.90233028160266 -87.92455520242903) bank137669 +137670 POINT(42.50876964561492 -87.20481567120001) bank137670 +137671 POINT(42.680577344682824 -86.7336989933925) bank137671 +137672 POINT(42.07429823666431 -87.05567906633654) bank137672 +137673 POINT(42.05429007497141 -86.75030729243454) bank137673 +137674 POINT(41.46838597016931 -86.96722057475237) bank137674 +137675 POINT(42.222879193281784 -87.4425327767182) bank137675 +137676 POINT(41.336618016055944 -86.936298575195) bank137676 +137677 POINT(42.58040332816628 -86.83137342218333) bank137677 +137678 POINT(42.62297648612961 -87.77514188869826) bank137678 +137679 POINT(42.28650016151464 -87.11687213863206) bank137679 +137680 POINT(41.56969230973238 -88.42932568571558) bank137680 +137681 POINT(41.37227232193212 -86.71304391661221) bank137681 +137682 POINT(41.69125555170368 -87.56720411997635) bank137682 +137683 POINT(41.89769450920239 -88.49589611532845) bank137683 +137684 POINT(42.70220029619462 -87.86835070640838) bank137684 +137685 POINT(42.552109297640236 -86.80661246319455) bank137685 +137686 POINT(42.79374972066774 -88.41965660974783) bank137686 +137687 POINT(42.40172959435344 -86.8527501818173) bank137687 +137688 POINT(41.441131313247816 -87.9539592698092) bank137688 +137689 POINT(42.24556605754482 -88.45154911598097) bank137689 +137690 POINT(41.086232600142885 -86.87658392021393) bank137690 +137691 POINT(41.746994626567876 -87.97096684681189) bank137691 +137692 POINT(41.05171658284314 -87.79911107485084) bank137692 +137693 POINT(42.26566821020523 -87.19705218315801) bank137693 +137694 POINT(42.05078120507784 -88.1780420679497) bank137694 +137695 POINT(42.50635135402118 -88.48543211602326) bank137695 +137696 POINT(42.72220970125931 -86.64555686258518) bank137696 +137697 POINT(41.483886843497 -87.45923675837905) bank137697 +137698 POINT(41.74138706442869 -87.32689495340392) bank137698 +137699 POINT(42.17276866741293 -87.45430005348923) bank137699 +137700 POINT(41.79247173593972 -87.24420230903601) bank137700 +137701 POINT(42.67957265002417 -87.49212985400528) bank137701 +137702 POINT(40.94521352600966 -87.74435240476747) bank137702 +137703 POINT(42.136369488289425 -88.00627938824785) bank137703 +137704 POINT(41.87714323708354 -88.59852839061887) bank137704 +137705 POINT(42.60756458490544 -87.87263058681452) bank137705 +137706 POINT(41.55712160424477 -87.95209926710143) bank137706 +137707 POINT(41.3860461878569 -86.97316933556644) bank137707 +137708 POINT(41.42252371610923 -87.61562735752744) bank137708 +137709 POINT(40.9638079872817 -86.75883144337884) bank137709 +137710 POINT(41.41538847062844 -88.20934903616211) bank137710 +137711 POINT(41.84640955359251 -87.31351907105332) bank137711 +137712 POINT(42.53309783536697 -87.7921703208914) bank137712 +137713 POINT(41.47365917228034 -86.83275136818685) bank137713 +137714 POINT(41.66370010364201 -88.50692836045103) bank137714 +137715 POINT(41.0365924968376 -88.22032162639657) bank137715 +137716 POINT(41.380236477025555 -87.78784417652825) bank137716 +137717 POINT(41.4017881510151 -87.7927963715401) bank137717 +137718 POINT(41.87738279564338 -87.7584873320564) bank137718 +137719 POINT(41.2544034850615 -88.57567780144237) bank137719 +137720 POINT(41.22015448652893 -87.3835990697896) bank137720 +137721 POINT(42.86897341307751 -87.96802330764068) bank137721 +137722 POINT(41.542563247030344 -88.60642703397652) bank137722 +137723 POINT(41.967134518559575 -88.09903640487754) bank137723 +137724 POINT(41.75137356387698 -88.03599524961203) bank137724 +137725 POINT(41.88754709049981 -88.56274330927452) bank137725 +137726 POINT(41.69651467448071 -88.21849300559354) bank137726 +137727 POINT(41.102531557291094 -87.17615625780516) bank137727 +137728 POINT(41.01676609032999 -86.67422475434186) bank137728 +137729 POINT(41.78696927565971 -87.69471666599027) bank137729 +137730 POINT(42.87704935583762 -88.2548654872639) bank137730 +137731 POINT(42.73427948600242 -87.98820617303653) bank137731 +137732 POINT(41.4259588461895 -88.28913126821584) bank137732 +137733 POINT(42.30446117633717 -87.2741673106941) bank137733 +137734 POINT(41.389109051869006 -88.21215150720461) bank137734 +137735 POINT(41.74682883718342 -87.08535251953882) bank137735 +137736 POINT(41.74647845504812 -87.3441659347371) bank137736 +137737 POINT(41.01633661257165 -88.41475735701425) bank137737 +137738 POINT(41.44415187248553 -86.89197894853483) bank137738 +137739 POINT(42.57051747090772 -88.17374581352759) bank137739 +137740 POINT(41.25406473188872 -88.5788996713586) bank137740 +137741 POINT(42.765580342831086 -86.946744753486) bank137741 +137742 POINT(42.39889916077104 -88.10242685524872) bank137742 +137743 POINT(41.213743402380096 -88.36035855624141) bank137743 +137744 POINT(40.93730018105464 -87.64812659342435) bank137744 +137745 POINT(42.33603906587052 -88.20535230679997) bank137745 +137746 POINT(42.781787886122146 -88.24226033135625) bank137746 +137747 POINT(42.41345389041653 -87.93785355562822) bank137747 +137748 POINT(42.81221206924105 -86.81612612929622) bank137748 +137749 POINT(41.08398604059532 -87.00119105961492) bank137749 +137750 POINT(42.20074122604124 -86.87147543158143) bank137750 +137751 POINT(42.86598274333342 -87.55565332824442) bank137751 +137752 POINT(41.10456187356196 -87.38111689995121) bank137752 +137753 POINT(42.23445255561328 -87.4093482146149) bank137753 +137754 POINT(42.252019540647446 -87.88041098404791) bank137754 +137755 POINT(41.90243393346945 -87.95061706623594) bank137755 +137756 POINT(41.57356430255701 -87.87280965121964) bank137756 +137757 POINT(42.01377063194479 -87.82096838825535) bank137757 +137758 POINT(41.647569313498614 -87.91246726156815) bank137758 +137759 POINT(42.25820092979656 -87.11200785476287) bank137759 +137760 POINT(41.87196477793236 -88.56987938721673) bank137760 +137761 POINT(42.15576916486193 -87.44326096808126) bank137761 +137762 POINT(42.241797904840425 -87.17894799352146) bank137762 +137763 POINT(41.64096810721347 -88.29165707318909) bank137763 +137764 POINT(42.71679906691996 -88.5943442242292) bank137764 +137765 POINT(40.977113010658954 -87.89235814477709) bank137765 +137766 POINT(41.30651077628234 -86.94266344180907) bank137766 +137767 POINT(41.16045430970325 -87.56040519517303) bank137767 +137768 POINT(41.93472625576414 -87.30582017296517) bank137768 +137769 POINT(42.779379612398394 -86.87344783286157) bank137769 +137770 POINT(42.100671158672185 -87.7823917418627) bank137770 +137771 POINT(42.36583750425452 -87.98384438965303) bank137771 +137772 POINT(40.921667304396706 -86.81063485119174) bank137772 +137773 POINT(42.36196248033624 -87.76887128205644) bank137773 +137774 POINT(41.18995830472326 -87.10785506947785) bank137774 +137775 POINT(42.82460430339861 -86.9123000744512) bank137775 +137776 POINT(42.05766827094147 -88.45719394620292) bank137776 +137777 POINT(42.68133361309826 -87.9850315698076) bank137777 +137778 POINT(42.29876391777392 -86.87226060502907) bank137778 +137779 POINT(41.20248366174645 -87.98778034840332) bank137779 +137780 POINT(42.21541082965143 -87.90225494349774) bank137780 +137781 POINT(42.85967879763632 -87.11655057975847) bank137781 +137782 POINT(41.09426632840626 -87.5531680292968) bank137782 +137783 POINT(42.568205189990444 -88.49052439844692) bank137783 +137784 POINT(42.69784482216403 -86.73259282357918) bank137784 +137785 POINT(42.56974324175509 -87.51438053248062) bank137785 +137786 POINT(42.53873702025978 -86.71266540073586) bank137786 +137787 POINT(41.01623195447249 -87.76768101309449) bank137787 +137788 POINT(42.82585981242195 -87.82636895494937) bank137788 +137789 POINT(41.11256734092398 -88.12825816024498) bank137789 +137790 POINT(41.82962207143982 -87.07644264342441) bank137790 +137791 POINT(42.412464119564575 -87.96423962660049) bank137791 +137792 POINT(42.59509211366916 -88.10258494789055) bank137792 +137793 POINT(42.243086868019624 -87.03301528654048) bank137793 +137794 POINT(41.03034460167802 -88.34453103316396) bank137794 +137795 POINT(42.40593527052772 -87.0013097107522) bank137795 +137796 POINT(42.5953739961631 -88.28823050609529) bank137796 +137797 POINT(42.83664682641894 -87.08804177433878) bank137797 +137798 POINT(42.46470336349888 -87.05167172884222) bank137798 +137799 POINT(42.06783591469804 -88.20497194836203) bank137799 +137800 POINT(42.5411839790857 -88.62097513252839) bank137800 +137801 POINT(42.76866346113508 -87.97831241537236) bank137801 +137802 POINT(42.56099634967275 -88.0845319600258) bank137802 +137803 POINT(42.415194538401515 -87.34503903802175) bank137803 +137804 POINT(41.750966550881145 -87.47954761781934) bank137804 +137805 POINT(41.26399830221665 -87.2788807867903) bank137805 +137806 POINT(42.77596754940651 -87.57095706313513) bank137806 +137807 POINT(42.071795513509194 -87.45938683718173) bank137807 +137808 POINT(42.455126031043626 -88.62687716846904) bank137808 +137809 POINT(41.82767915461676 -87.76549418099505) bank137809 +137810 POINT(42.315374363266976 -86.78427614752273) bank137810 +137811 POINT(41.403753240494545 -87.44365935600106) bank137811 +137812 POINT(42.78237647517476 -86.75579446568733) bank137812 +137813 POINT(41.89263422601163 -87.10131007250395) bank137813 +137814 POINT(41.713848692164504 -87.89908918404548) bank137814 +137815 POINT(42.77880764247405 -87.40647025296414) bank137815 +137816 POINT(42.495279851837495 -87.23399692073511) bank137816 +137817 POINT(42.301423504514695 -86.87590200548627) bank137817 +137818 POINT(42.66885004157421 -88.45140301603344) bank137818 +137819 POINT(40.89789661036082 -88.08521708583854) bank137819 +137820 POINT(42.54519135055668 -87.07083338707633) bank137820 +137821 POINT(41.17377919659568 -88.18356306076403) bank137821 +137822 POINT(41.12846824325584 -87.38294050227833) bank137822 +137823 POINT(40.955442252997 -87.27919020645702) bank137823 +137824 POINT(41.69992910190864 -87.62967303144312) bank137824 +137825 POINT(42.47952359717985 -88.07433555025912) bank137825 +137826 POINT(42.75424088607928 -87.83953517123804) bank137826 +137827 POINT(42.15857711521513 -88.08939604970232) bank137827 +137828 POINT(41.53948912218897 -88.16127600935219) bank137828 +137829 POINT(42.099077416141824 -86.65070189871417) bank137829 +137830 POINT(42.093100413246944 -88.50547655748737) bank137830 +137831 POINT(42.25587147181647 -86.8260307059703) bank137831 +137832 POINT(41.72045922509919 -87.30278825817945) bank137832 +137833 POINT(42.59626815083057 -86.92110571278941) bank137833 +137834 POINT(41.73561286116044 -87.52704607529661) bank137834 +137835 POINT(42.38150498027615 -87.91385611796338) bank137835 +137836 POINT(42.23051307208958 -86.91841038619215) bank137836 +137837 POINT(41.104517721328726 -87.8848513698667) bank137837 +137838 POINT(41.19818840801073 -88.59143248383513) bank137838 +137839 POINT(42.46555576837861 -87.37492741394883) bank137839 +137840 POINT(42.29342074654965 -87.10346305383992) bank137840 +137841 POINT(42.175571401419 -88.30201039433761) bank137841 +137842 POINT(42.533061970388964 -88.16314555589152) bank137842 +137843 POINT(41.72169530151223 -88.19348372300907) bank137843 +137844 POINT(41.926750357298666 -86.69656353914073) bank137844 +137845 POINT(42.665291462384005 -88.21449145536496) bank137845 +137846 POINT(41.970925919119864 -88.12755657253548) bank137846 +137847 POINT(41.049789062831806 -88.153402539102) bank137847 +137848 POINT(41.249467795122165 -87.9657957994512) bank137848 +137849 POINT(41.788994532838714 -87.6692916623485) bank137849 +137850 POINT(42.48383569714641 -87.12035223673927) bank137850 +137851 POINT(41.34579837172278 -87.70240495616764) bank137851 +137852 POINT(41.41958084796097 -86.73497778964692) bank137852 +137853 POINT(41.44106257178604 -88.53062436551622) bank137853 +137854 POINT(41.58945883104983 -88.0757550310933) bank137854 +137855 POINT(42.199724394981615 -87.6966468588515) bank137855 +137856 POINT(41.94142738590959 -88.50987557129307) bank137856 +137857 POINT(42.20543828042444 -87.66898261288318) bank137857 +137858 POINT(40.998361606055276 -88.0028264484989) bank137858 +137859 POINT(41.31705641637885 -87.42643564063792) bank137859 +137860 POINT(42.12152161273633 -87.39659091547726) bank137860 +137861 POINT(41.8757067465281 -88.43704664343927) bank137861 +137862 POINT(41.7365400703429 -87.16932483579677) bank137862 +137863 POINT(41.25594776860277 -87.82191955722584) bank137863 +137864 POINT(42.28335154455242 -87.25947384566018) bank137864 +137865 POINT(41.87542767124803 -87.34387858951338) bank137865 +137866 POINT(41.951823607819904 -88.07666558597842) bank137866 +137867 POINT(41.470529618166964 -88.26200275503635) bank137867 +137868 POINT(40.95016742841834 -86.63270172421059) bank137868 +137869 POINT(42.85234437882095 -88.0587867807267) bank137869 +137870 POINT(41.253918650166575 -87.01138215747915) bank137870 +137871 POINT(41.561052380709334 -87.26906309728443) bank137871 +137872 POINT(42.65694082489349 -87.09933319287029) bank137872 +137873 POINT(41.13641385832485 -86.64671412638374) bank137873 +137874 POINT(42.844664123917745 -87.76753733371116) bank137874 +137875 POINT(42.27042387526073 -88.25600147123585) bank137875 +137876 POINT(42.18752747549675 -87.04353355023353) bank137876 +137877 POINT(41.10023805641683 -87.68374637352183) bank137877 +137878 POINT(41.75813536978992 -88.49915489159785) bank137878 +137879 POINT(41.490417338567134 -88.31794533614004) bank137879 +137880 POINT(41.67073424019197 -87.56206326136456) bank137880 +137881 POINT(42.741118324544935 -88.37249147246013) bank137881 +137882 POINT(41.347132374354516 -87.8972979210631) bank137882 +137883 POINT(41.46393328824096 -87.5635609654833) bank137883 +137884 POINT(40.97722377024742 -87.34417954963787) bank137884 +137885 POINT(42.65514230108745 -88.5516293341818) bank137885 +137886 POINT(42.04790162629973 -88.20381685951726) bank137886 +137887 POINT(41.89446386109717 -87.70004716273235) bank137887 +137888 POINT(41.42887821924682 -86.81736887192908) bank137888 +137889 POINT(42.18387578532731 -88.22774908349693) bank137889 +137890 POINT(41.131068992086014 -88.15684940106753) bank137890 +137891 POINT(41.3462634772531 -86.9861767555735) bank137891 +137892 POINT(41.94311526642017 -88.06446075510785) bank137892 +137893 POINT(42.1470082176668 -87.79946368107818) bank137893 +137894 POINT(42.0373876060665 -86.6394729968764) bank137894 +137895 POINT(42.47992109097573 -88.0411911010238) bank137895 +137896 POINT(42.05209107019495 -87.83099106063051) bank137896 +137897 POINT(41.30412667531097 -88.3987195668599) bank137897 +137898 POINT(41.56483127358551 -86.80163383854378) bank137898 +137899 POINT(41.00122283763327 -87.3686643972988) bank137899 +137900 POINT(42.79248859052589 -86.80666454349667) bank137900 +137901 POINT(41.105527345567246 -86.77112845196903) bank137901 +137902 POINT(41.2805784459908 -86.97950507206124) bank137902 +137903 POINT(42.68131507581539 -86.87875559986036) bank137903 +137904 POINT(42.052792724358746 -87.49775018742457) bank137904 +137905 POINT(41.47464724313452 -87.5993449318629) bank137905 +137906 POINT(42.74360629751285 -88.38139804006829) bank137906 +137907 POINT(42.648519794483605 -87.91933557004816) bank137907 +137908 POINT(42.37084999178319 -87.40145564425958) bank137908 +137909 POINT(42.209748729001106 -87.54584666836948) bank137909 +137910 POINT(41.92654212914748 -86.70020205422983) bank137910 +137911 POINT(41.62644891778035 -87.6915082228982) bank137911 +137912 POINT(42.72460517293305 -87.6020270210972) bank137912 +137913 POINT(41.903168279443435 -87.87613726397204) bank137913 +137914 POINT(42.72604456515107 -87.14862654611623) bank137914 +137915 POINT(42.35403198990801 -87.23506701664088) bank137915 +137916 POINT(41.53008146070844 -87.56496246837347) bank137916 +137917 POINT(40.89168697306528 -88.32877389069107) bank137917 +137918 POINT(40.89746099027188 -88.14438222521072) bank137918 +137919 POINT(42.615076832262645 -86.74910448699033) bank137919 +137920 POINT(41.807145036678904 -86.95597373211096) bank137920 +137921 POINT(42.18391548539552 -88.15657860393623) bank137921 +137922 POINT(41.377987690549055 -87.83659466756048) bank137922 +137923 POINT(42.44383023978299 -88.46375738056895) bank137923 +137924 POINT(42.40561293627634 -87.58454661611941) bank137924 +137925 POINT(40.92510296145993 -86.70311301275824) bank137925 +137926 POINT(42.310464745570364 -88.06627204883964) bank137926 +137927 POINT(41.84999896230041 -87.1424913250387) bank137927 +137928 POINT(41.11935398611699 -88.27355000927656) bank137928 +137929 POINT(41.37488258712923 -87.33585793373189) bank137929 +137930 POINT(41.47183524005105 -87.56215686854985) bank137930 +137931 POINT(41.438969187814365 -88.6132287117339) bank137931 +137932 POINT(41.24061605826741 -88.22165635413779) bank137932 +137933 POINT(41.44053852292943 -87.83651826885071) bank137933 +137934 POINT(42.03439822706973 -87.17144449316679) bank137934 +137935 POINT(41.964339669364804 -86.81457505117899) bank137935 +137936 POINT(41.49983318658831 -87.50636857696901) bank137936 +137937 POINT(42.13856743067232 -87.87304226791333) bank137937 +137938 POINT(42.08670731189858 -88.19833346371752) bank137938 +137939 POINT(42.6041713043724 -88.39962138294526) bank137939 +137940 POINT(41.91431319537683 -88.00201090144637) bank137940 +137941 POINT(41.48522484101431 -88.51087253741107) bank137941 +137942 POINT(42.72229706979915 -88.59261696662394) bank137942 +137943 POINT(41.06442489113424 -87.43222573950892) bank137943 +137944 POINT(41.56118589892678 -87.83703456617563) bank137944 +137945 POINT(42.8125410111309 -88.34738118150008) bank137945 +137946 POINT(42.77920193418825 -86.7343591286778) bank137946 +137947 POINT(42.71299633938711 -88.33403403080725) bank137947 +137948 POINT(41.777662811406685 -87.41418041777544) bank137948 +137949 POINT(42.451704485757084 -86.70194959003119) bank137949 +137950 POINT(41.80021028737742 -88.52788161573592) bank137950 +137951 POINT(42.43718819211229 -88.47721716290674) bank137951 +137952 POINT(41.90320143181528 -86.77307115853738) bank137952 +137953 POINT(42.57368310193576 -87.27405248858213) bank137953 +137954 POINT(40.961005002499114 -87.22293569705302) bank137954 +137955 POINT(41.53548591169138 -86.95782377732978) bank137955 +137956 POINT(42.28693525933481 -88.60394428379917) bank137956 +137957 POINT(42.49662857330999 -88.20041342713427) bank137957 +137958 POINT(41.271984046025864 -88.23602231580358) bank137958 +137959 POINT(41.053895622290895 -88.30633689873243) bank137959 +137960 POINT(40.96985786615323 -88.61890263744975) bank137960 +137961 POINT(41.37020299313367 -87.66335215414333) bank137961 +137962 POINT(42.711297160688204 -86.6832850164349) bank137962 +137963 POINT(41.505316447022494 -88.23564888732056) bank137963 +137964 POINT(42.17064434376948 -87.09002439838635) bank137964 +137965 POINT(42.03240021947169 -88.52681621316839) bank137965 +137966 POINT(42.26748278480548 -88.55666079730835) bank137966 +137967 POINT(40.89262534829622 -88.51981639946572) bank137967 +137968 POINT(41.84983743629276 -88.40657823352137) bank137968 +137969 POINT(41.777789012478145 -87.46813022601481) bank137969 +137970 POINT(41.83898468179926 -86.97991818039515) bank137970 +137971 POINT(40.99570604503612 -88.40377228741242) bank137971 +137972 POINT(42.27383400633202 -88.53487839159295) bank137972 +137973 POINT(41.327865429201225 -86.7606064984341) bank137973 +137974 POINT(41.735460103092684 -88.19555091785678) bank137974 +137975 POINT(42.77264985390537 -87.11893696694736) bank137975 +137976 POINT(41.897318401121545 -87.53205548533707) bank137976 +137977 POINT(42.265452774882455 -88.53782266702176) bank137977 +137978 POINT(42.217577631270494 -88.12579492991095) bank137978 +137979 POINT(41.436134705308554 -86.89648385791662) bank137979 +137980 POINT(42.078237930986006 -86.99087692293634) bank137980 +137981 POINT(41.18700434528248 -87.16442097599901) bank137981 +137982 POINT(41.91463179028487 -87.90768174857705) bank137982 +137983 POINT(42.24367373801226 -88.26909646577813) bank137983 +137984 POINT(42.72289258468772 -87.46624682671856) bank137984 +137985 POINT(41.186192033327224 -88.52001666582453) bank137985 +137986 POINT(42.7867460851421 -88.38005874035737) bank137986 +137987 POINT(41.49332498764375 -87.69193776825439) bank137987 +137988 POINT(41.78356250482167 -87.19922876448652) bank137988 +137989 POINT(42.1689672699066 -88.24230495100228) bank137989 +137990 POINT(42.541204611683675 -87.12058394920483) bank137990 +137991 POINT(42.14223221444325 -86.63694529962143) bank137991 +137992 POINT(41.8160163583003 -87.53431026952053) bank137992 +137993 POINT(41.00948021270309 -87.55485046213927) bank137993 +137994 POINT(41.96145878543796 -86.64368713721954) bank137994 +137995 POINT(42.390289628095815 -86.81279129891311) bank137995 +137996 POINT(40.973756249113244 -86.85617067293978) bank137996 +137997 POINT(41.174706917853754 -88.04180906512379) bank137997 +137998 POINT(40.98806631193853 -88.26453691766582) bank137998 +137999 POINT(41.96105664219152 -88.05726259610594) bank137999 +138000 POINT(41.304908268412376 -88.10125736212923) bank138000 +138001 POINT(42.01881492717827 -87.46590345078769) bank138001 +138002 POINT(41.980445372577435 -88.55358330442787) bank138002 +138003 POINT(41.43237594247918 -88.2418217740304) bank138003 +138004 POINT(41.394988680001525 -88.48812244865132) bank138004 +138005 POINT(41.72512650453418 -86.8949072580285) bank138005 +138006 POINT(42.8236692190012 -88.48980805211676) bank138006 +138007 POINT(42.86651728723006 -86.88438541491645) bank138007 +138008 POINT(42.406702021435095 -88.584068963605) bank138008 +138009 POINT(41.091924589820515 -87.55344169236616) bank138009 +138010 POINT(41.327027167249426 -87.28897888842111) bank138010 +138011 POINT(41.36321139824993 -88.41762541746832) bank138011 +138012 POINT(41.517976215394775 -87.50343463903229) bank138012 +138013 POINT(41.16347868644781 -88.21854988602102) bank138013 +138014 POINT(42.20360968846527 -88.0414904033957) bank138014 +138015 POINT(42.47304906715393 -88.43509526219928) bank138015 +138016 POINT(42.8267976779055 -88.01179987326056) bank138016 +138017 POINT(42.66014687230434 -87.3254465281811) bank138017 +138018 POINT(41.56718734934904 -87.70079350287271) bank138018 +138019 POINT(42.27291986087587 -88.36808967187567) bank138019 +138020 POINT(42.05166175845691 -87.11856606868075) bank138020 +138021 POINT(41.16510555758443 -87.7831770506751) bank138021 +138022 POINT(40.91811686665358 -88.42837083284499) bank138022 +138023 POINT(41.52136467147657 -87.00560240305799) bank138023 +138024 POINT(41.77773266328863 -88.16175362066768) bank138024 +138025 POINT(41.143636813743186 -86.74159602621863) bank138025 +138026 POINT(42.04629217381865 -88.49999326686579) bank138026 +138027 POINT(42.102398659560045 -88.49548980844612) bank138027 +138028 POINT(42.68551355090697 -86.70320987014432) bank138028 +138029 POINT(41.38855300182882 -86.99560627853334) bank138029 +138030 POINT(41.415321175246454 -87.86175468621855) bank138030 +138031 POINT(42.86734030124861 -87.81582960544971) bank138031 +138032 POINT(41.041736538474765 -87.42155651278114) bank138032 +138033 POINT(41.783140186371384 -88.22047134762242) bank138033 +138034 POINT(42.18040721407075 -87.93477389359737) bank138034 +138035 POINT(42.10824508597245 -87.60096231633685) bank138035 +138036 POINT(42.17708434936117 -88.06959428578341) bank138036 +138037 POINT(42.659564654761404 -88.30352548064168) bank138037 +138038 POINT(42.72189312022901 -87.43847533974034) bank138038 +138039 POINT(42.13410439558443 -88.38819357429814) bank138039 +138040 POINT(41.00899726289534 -87.25058469177353) bank138040 +138041 POINT(41.11907961199329 -86.92184745742698) bank138041 +138042 POINT(42.57187237306051 -87.08015069257293) bank138042 +138043 POINT(41.62749447515116 -86.94813028836015) bank138043 +138044 POINT(41.566528265123246 -88.55632642316601) bank138044 +138045 POINT(41.4298884151021 -87.24634104633455) bank138045 +138046 POINT(42.18403336911541 -87.26009538741572) bank138046 +138047 POINT(42.80603142360378 -87.19347169456836) bank138047 +138048 POINT(42.81807991241913 -86.67629317509981) bank138048 +138049 POINT(41.8948676865705 -88.10886699716741) bank138049 +138050 POINT(41.8973857382361 -86.71024772905797) bank138050 +138051 POINT(42.089662417904044 -88.05531538980335) bank138051 +138052 POINT(41.82546370273766 -88.00521452754658) bank138052 +138053 POINT(42.54539130010043 -86.85725493615652) bank138053 +138054 POINT(41.38855271563534 -88.10345497212411) bank138054 +138055 POINT(42.19788420453395 -87.40321163819418) bank138055 +138056 POINT(42.7616707043574 -86.82605408756355) bank138056 +138057 POINT(41.80127750698905 -87.40390284363886) bank138057 +138058 POINT(40.95775622262589 -88.09695058849738) bank138058 +138059 POINT(41.88431958589571 -86.97921173444855) bank138059 +138060 POINT(41.14594964499596 -87.63311850359813) bank138060 +138061 POINT(41.932325858617915 -87.10182606570277) bank138061 +138062 POINT(42.278400582801616 -86.91814073028621) bank138062 +138063 POINT(41.373197673283194 -87.48520825723067) bank138063 +138064 POINT(42.31127210699972 -88.57390048275765) bank138064 +138065 POINT(42.01210595353901 -88.34537349021066) bank138065 +138066 POINT(42.59906741532484 -88.23922200823056) bank138066 +138067 POINT(41.9759836462237 -87.82669023676681) bank138067 +138068 POINT(41.11319262312625 -86.95753959079593) bank138068 +138069 POINT(41.155298800605195 -87.62668448093805) bank138069 +138070 POINT(41.18306346058487 -87.2952003040359) bank138070 +138071 POINT(41.30076734648729 -88.32535111999209) bank138071 +138072 POINT(42.76894283748558 -87.46586853368953) bank138072 +138073 POINT(42.2476139711306 -88.48371129165048) bank138073 +138074 POINT(42.137654753970644 -88.09007768094253) bank138074 +138075 POINT(41.25381079428048 -88.04511220283612) bank138075 +138076 POINT(40.92198991175711 -87.07305239831858) bank138076 +138077 POINT(42.35182926631356 -87.47016932508062) bank138077 +138078 POINT(41.06895589126016 -88.23866419510078) bank138078 +138079 POINT(41.12433548938151 -86.97012640243638) bank138079 +138080 POINT(42.505543063120996 -88.2185999629761) bank138080 +138081 POINT(41.04376979957933 -87.39316395561254) bank138081 +138082 POINT(40.996459505757116 -87.09704807839697) bank138082 +138083 POINT(41.52828559292429 -88.57387153805072) bank138083 +138084 POINT(42.080462465046296 -87.12588290989939) bank138084 +138085 POINT(41.79448265505708 -87.17643592698005) bank138085 +138086 POINT(42.34919232235987 -86.88542696609977) bank138086 +138087 POINT(42.02741339090575 -87.55796116236287) bank138087 +138088 POINT(42.456367832136785 -88.54378724613184) bank138088 +138089 POINT(41.14222890522064 -87.30503350226952) bank138089 +138090 POINT(42.6389305400789 -88.37477414465991) bank138090 +138091 POINT(41.94366503541399 -88.22765831550815) bank138091 +138092 POINT(42.65786282142992 -87.67076456469916) bank138092 +138093 POINT(42.381246611486546 -86.96149401645205) bank138093 +138094 POINT(42.22830155589262 -87.6787905240266) bank138094 +138095 POINT(40.95774656191119 -86.91855518890212) bank138095 +138096 POINT(42.783656933411976 -88.57643666565396) bank138096 +138097 POINT(41.409953227956095 -87.38538137105681) bank138097 +138098 POINT(42.53313491401772 -87.33774856651208) bank138098 +138099 POINT(40.95383143782755 -87.32427763559416) bank138099 +138100 POINT(41.50044371929075 -88.40565905516102) bank138100 +138101 POINT(42.85382513965265 -88.44840371962822) bank138101 +138102 POINT(42.06765805944368 -87.85757361770112) bank138102 +138103 POINT(41.152351745106074 -87.49764219413485) bank138103 +138104 POINT(41.104371058459606 -87.23140256143081) bank138104 +138105 POINT(42.094660630732534 -87.716828156073) bank138105 +138106 POINT(42.30661065166793 -87.49737762469829) bank138106 +138107 POINT(41.369993626301024 -87.5764196828994) bank138107 +138108 POINT(41.690833102388545 -87.44641103281572) bank138108 +138109 POINT(42.80994309485589 -88.5786829224981) bank138109 +138110 POINT(41.727324403644296 -88.26354440856826) bank138110 +138111 POINT(41.08770027572801 -87.33081421973695) bank138111 +138112 POINT(41.24336897267661 -88.42162964621673) bank138112 +138113 POINT(41.45656178637259 -87.04199260744674) bank138113 +138114 POINT(42.687110796443946 -88.52733043244432) bank138114 +138115 POINT(41.17925340439633 -87.03503655775717) bank138115 +138116 POINT(41.60766486749369 -86.91792476558852) bank138116 +138117 POINT(41.42180096129836 -86.68403506245596) bank138117 +138118 POINT(40.91826674697151 -88.24025597685633) bank138118 +138119 POINT(42.62850966097527 -88.51700872849564) bank138119 +138120 POINT(42.009166098876484 -86.97209911893061) bank138120 +138121 POINT(42.795322611283645 -87.20137532898855) bank138121 +138122 POINT(40.95412743734589 -88.05199149644032) bank138122 +138123 POINT(41.07265702539897 -87.73533813994901) bank138123 +138124 POINT(41.284509308942546 -88.58153707692081) bank138124 +138125 POINT(42.04094307102021 -88.30418946177609) bank138125 +138126 POINT(42.34173948180398 -88.23549575115739) bank138126 +138127 POINT(42.531616557872525 -87.74736593351533) bank138127 +138128 POINT(42.187898291301764 -87.5413522910825) bank138128 +138129 POINT(41.977438240770404 -86.98293755160134) bank138129 +138130 POINT(42.4339616448699 -87.5349080752712) bank138130 +138131 POINT(42.53392963678205 -88.23194745256208) bank138131 +138132 POINT(41.693438038834394 -88.47886787577517) bank138132 +138133 POINT(42.34573639388224 -88.24492798138706) bank138133 +138134 POINT(41.25663005754375 -87.65293719455114) bank138134 +138135 POINT(42.37740391178351 -87.52638189531527) bank138135 +138136 POINT(41.95562325735637 -88.40470395497077) bank138136 +138137 POINT(42.57347095692972 -86.7228240243164) bank138137 +138138 POINT(41.74111840790851 -88.62881787451555) bank138138 +138139 POINT(41.234862732643954 -87.34436876738955) bank138139 +138140 POINT(42.39668574958469 -87.38869899136157) bank138140 +138141 POINT(42.21299176286093 -88.06734679262102) bank138141 +138142 POINT(41.48666954506261 -88.08958668399995) bank138142 +138143 POINT(41.38265607264702 -88.61033264371775) bank138143 +138144 POINT(42.05667374515565 -88.18788498006698) bank138144 +138145 POINT(42.22464387108659 -87.86118552367482) bank138145 +138146 POINT(42.77941049872718 -87.29049444510912) bank138146 +138147 POINT(41.22477067407689 -86.84635341341429) bank138147 +138148 POINT(41.42026637245623 -87.98035345078732) bank138148 +138149 POINT(42.427774774213255 -88.16064979733353) bank138149 +138150 POINT(41.04012612290276 -87.56144264135249) bank138150 +138151 POINT(41.08106432061561 -87.65553172876884) bank138151 +138152 POINT(41.01702114824346 -88.5134106030599) bank138152 +138153 POINT(41.35260009262264 -88.53356941638937) bank138153 +138154 POINT(41.265697670070594 -88.6008365604835) bank138154 +138155 POINT(41.20644782793564 -87.7661072020487) bank138155 +138156 POINT(41.39303317771575 -87.94299306117111) bank138156 +138157 POINT(41.810334556648584 -88.33876859747798) bank138157 +138158 POINT(42.036160626446254 -87.15239777742326) bank138158 +138159 POINT(42.643994115862135 -86.81735611973482) bank138159 +138160 POINT(42.322206943518125 -87.82125024417772) bank138160 +138161 POINT(41.052098454274585 -88.51187957982128) bank138161 +138162 POINT(41.41281731794724 -86.72259612163968) bank138162 +138163 POINT(41.00030170035505 -87.39803651563116) bank138163 +138164 POINT(41.19254501571998 -87.2728291941649) bank138164 +138165 POINT(42.334550097668476 -86.7073688190225) bank138165 +138166 POINT(41.759982761855476 -88.18432558917559) bank138166 +138167 POINT(40.912039211272955 -88.05901926283238) bank138167 +138168 POINT(42.550747546550234 -86.77658441807348) bank138168 +138169 POINT(42.75091268907727 -86.68721195140253) bank138169 +138170 POINT(41.58594445115425 -87.60564710143846) bank138170 +138171 POINT(42.14171358004645 -87.54981061793431) bank138171 +138172 POINT(40.914805132385304 -88.15303890976236) bank138172 +138173 POINT(41.63820945242527 -87.09676206747386) bank138173 +138174 POINT(42.53586381660161 -88.203936715788) bank138174 +138175 POINT(41.13814781363695 -87.97641175240557) bank138175 +138176 POINT(41.64463964588003 -87.14011710197603) bank138176 +138177 POINT(41.99672655206849 -86.78202123209336) bank138177 +138178 POINT(41.74502617783608 -87.18755063631136) bank138178 +138179 POINT(42.31457042082445 -88.40502362338052) bank138179 +138180 POINT(41.367945539877105 -86.79191179685284) bank138180 +138181 POINT(42.1869082959696 -86.71879423008549) bank138181 +138182 POINT(42.742074617425374 -88.50007037470453) bank138182 +138183 POINT(42.860788791832455 -86.89344344846712) bank138183 +138184 POINT(42.04338053722184 -86.83775140948124) bank138184 +138185 POINT(41.903543253642844 -87.0162760222448) bank138185 +138186 POINT(42.46382640003957 -87.80879578461564) bank138186 +138187 POINT(41.63517822082762 -87.37564141305727) bank138187 +138188 POINT(42.41385281334748 -87.92157623384279) bank138188 +138189 POINT(41.18240632350797 -87.11676144255043) bank138189 +138190 POINT(41.72461372203085 -87.41861088960555) bank138190 +138191 POINT(41.61471992627732 -88.35621781242423) bank138191 +138192 POINT(41.32558146793816 -88.20765882809246) bank138192 +138193 POINT(41.39214735280883 -87.85605889525814) bank138193 +138194 POINT(42.55500712982041 -88.58133505132426) bank138194 +138195 POINT(41.52021439728258 -88.25094027848712) bank138195 +138196 POINT(42.79232720758281 -88.07957448161243) bank138196 +138197 POINT(41.357176488553435 -88.47084920292892) bank138197 +138198 POINT(41.94966903272872 -87.095562906316) bank138198 +138199 POINT(41.41933405989249 -86.67924207049549) bank138199 +138200 POINT(42.51689824781896 -88.58246174251431) bank138200 +138201 POINT(42.832617259521506 -87.05499641648545) bank138201 +138202 POINT(42.63496643742284 -88.23544175779854) bank138202 +138203 POINT(41.20479935084811 -87.04016409899963) bank138203 +138204 POINT(41.978567195849955 -87.55788491793987) bank138204 +138205 POINT(41.54743385878497 -87.17509657069964) bank138205 +138206 POINT(42.50529074334628 -88.23707140053776) bank138206 +138207 POINT(40.91468351180519 -87.29226902398625) bank138207 +138208 POINT(42.04378858144904 -87.59961410326981) bank138208 +138209 POINT(41.7515556597444 -86.82119162402701) bank138209 +138210 POINT(42.13401043670079 -87.488459152043) bank138210 +138211 POINT(42.461053618911706 -88.42510701322097) bank138211 +138212 POINT(41.56317284053248 -87.9976957819074) bank138212 +138213 POINT(41.54218706851597 -87.64669376634781) bank138213 +138214 POINT(41.147671596827955 -87.71833039779796) bank138214 +138215 POINT(42.125128019897936 -87.32729050741092) bank138215 +138216 POINT(42.40996478835801 -87.31013805216165) bank138216 +138217 POINT(41.9421228996601 -88.40773786997289) bank138217 +138218 POINT(41.06559658524577 -87.09568533865036) bank138218 +138219 POINT(41.063587623127134 -88.50901693742613) bank138219 +138220 POINT(41.03612816365264 -87.0704455635209) bank138220 +138221 POINT(41.47685904362929 -87.77414329432686) bank138221 +138222 POINT(42.67227492234606 -88.52187810603736) bank138222 +138223 POINT(41.063536887689814 -87.30351952120564) bank138223 +138224 POINT(42.862720898207186 -87.78450969513449) bank138224 +138225 POINT(42.863518266543124 -88.61303295034163) bank138225 +138226 POINT(41.85046867795367 -88.41986030409188) bank138226 +138227 POINT(41.959631923895 -88.3325433570304) bank138227 +138228 POINT(40.91941338111312 -87.56478809251334) bank138228 +138229 POINT(41.31395273977356 -87.68245341453144) bank138229 +138230 POINT(41.39454900561452 -87.11067046747652) bank138230 +138231 POINT(41.47765118249559 -87.91448169019743) bank138231 +138232 POINT(42.29210247831852 -86.88348190959572) bank138232 +138233 POINT(42.619917955332525 -87.6724285698353) bank138233 +138234 POINT(41.01044988883172 -86.9643744018671) bank138234 +138235 POINT(42.39147229065722 -86.66908077601722) bank138235 +138236 POINT(42.071365208182186 -87.00830049064744) bank138236 +138237 POINT(42.773941075974854 -88.17455819194983) bank138237 +138238 POINT(41.29942118086974 -86.90624787787567) bank138238 +138239 POINT(42.68978081470228 -88.55690140061341) bank138239 +138240 POINT(41.199026681011524 -88.58241693058062) bank138240 +138241 POINT(42.1546510365222 -87.9565550777399) bank138241 +138242 POINT(42.54734752212396 -88.29712731357642) bank138242 +138243 POINT(42.074317802364085 -88.41086131886547) bank138243 +138244 POINT(42.155793749387506 -87.96783607257491) bank138244 +138245 POINT(41.883505452881174 -86.88193788457188) bank138245 +138246 POINT(42.553966590911266 -87.44282104480074) bank138246 +138247 POINT(40.97384757080566 -87.62610325178649) bank138247 +138248 POINT(42.696491950421155 -86.83081879856702) bank138248 +138249 POINT(42.55457622686871 -88.19329042389167) bank138249 +138250 POINT(41.78344602752404 -86.7191249186442) bank138250 +138251 POINT(42.30833049813736 -88.13585492680154) bank138251 +138252 POINT(41.281584909494576 -86.69003802778558) bank138252 +138253 POINT(41.626037339353964 -87.68624818755721) bank138253 +138254 POINT(42.216735731966004 -88.27481135660445) bank138254 +138255 POINT(42.68427869804617 -88.00533324495537) bank138255 +138256 POINT(42.42865316968264 -87.20178975448192) bank138256 +138257 POINT(42.48611314017379 -87.8461552160052) bank138257 +138258 POINT(42.254057625380845 -88.26828692685572) bank138258 +138259 POINT(41.829641194292954 -87.50362183727952) bank138259 +138260 POINT(42.811467961331196 -88.28260882978161) bank138260 +138261 POINT(41.351898223446014 -86.70019660965897) bank138261 +138262 POINT(41.66137845851147 -87.88257560373123) bank138262 +138263 POINT(41.80277396314256 -87.04377856659802) bank138263 +138264 POINT(42.44359620489755 -87.36281090957164) bank138264 +138265 POINT(41.973158723293345 -87.58740355580122) bank138265 +138266 POINT(41.669560992792356 -87.46176023177138) bank138266 +138267 POINT(41.29510349821787 -87.61864375481478) bank138267 +138268 POINT(41.06045031897038 -88.20699562540216) bank138268 +138269 POINT(41.18508110594726 -88.49020760301383) bank138269 +138270 POINT(40.94865966925204 -87.78281854487311) bank138270 +138271 POINT(42.47323812340093 -88.4638224489358) bank138271 +138272 POINT(42.0562955754937 -86.91780080207555) bank138272 +138273 POINT(41.95504295309253 -88.17060141421786) bank138273 +138274 POINT(42.308900540179444 -86.94995089659157) bank138274 +138275 POINT(42.34320656889809 -87.4986991342809) bank138275 +138276 POINT(41.458519405156984 -86.88356257697026) bank138276 +138277 POINT(41.70057580804754 -86.85938163748772) bank138277 +138278 POINT(42.04736939445314 -87.8742487341422) bank138278 +138279 POINT(41.46505737243946 -87.60836801258121) bank138279 +138280 POINT(42.46408676065282 -88.04800909651523) bank138280 +138281 POINT(41.61881188001968 -86.95473507344244) bank138281 +138282 POINT(42.47853733417939 -87.2215615385539) bank138282 +138283 POINT(42.75231348521393 -87.58696604246687) bank138283 +138284 POINT(41.74736608799106 -86.9675627786446) bank138284 +138285 POINT(41.7278099954666 -87.47256970745144) bank138285 +138286 POINT(41.265056394973996 -88.00106500660458) bank138286 +138287 POINT(41.2559309757229 -87.41865030512268) bank138287 +138288 POINT(41.91438428451528 -87.9106888719399) bank138288 +138289 POINT(42.28251685629226 -86.6876764812994) bank138289 +138290 POINT(41.02836268222804 -88.17064815857594) bank138290 +138291 POINT(42.556590353835304 -88.05685544042971) bank138291 +138292 POINT(42.340797652241186 -87.60495804120686) bank138292 +138293 POINT(41.153522140104336 -88.23870784295487) bank138293 +138294 POINT(41.03148245183662 -87.89143577682373) bank138294 +138295 POINT(42.38797811831339 -87.32545183079809) bank138295 +138296 POINT(41.23190340050317 -88.1751725933888) bank138296 +138297 POINT(41.01649782406318 -86.85468237807014) bank138297 +138298 POINT(41.7360102064518 -88.41729308724192) bank138298 +138299 POINT(41.22666098831956 -86.8222486323588) bank138299 +138300 POINT(42.42104505433356 -88.32734117568756) bank138300 +138301 POINT(40.8925011892388 -87.89047936419078) bank138301 +138302 POINT(42.14545303467343 -88.25042491160613) bank138302 +138303 POINT(41.15203921890426 -87.21807185600048) bank138303 +138304 POINT(41.05834622808463 -86.740190233696) bank138304 +138305 POINT(42.82425691854041 -87.27915863917532) bank138305 +138306 POINT(41.5766632445656 -87.41528048164699) bank138306 +138307 POINT(42.10986502330379 -88.21530018536689) bank138307 +138308 POINT(41.15465300387989 -88.41900660932374) bank138308 +138309 POINT(42.22405548679274 -88.30302441204103) bank138309 +138310 POINT(41.50602726200811 -88.20288477325379) bank138310 +138311 POINT(42.65934194614372 -87.30399289611704) bank138311 +138312 POINT(41.57308551044044 -87.40325461023573) bank138312 +138313 POINT(41.831283495689256 -87.84241768299306) bank138313 +138314 POINT(41.632279241586424 -87.84088915994597) bank138314 +138315 POINT(42.14343145593115 -88.49892191320616) bank138315 +138316 POINT(41.540751916335005 -87.6273357580886) bank138316 +138317 POINT(41.17628656391983 -87.0085225210209) bank138317 +138318 POINT(42.70099244222156 -87.78267279777369) bank138318 +138319 POINT(42.38514267548027 -87.02460400184337) bank138319 +138320 POINT(41.21574993338973 -87.65858138231414) bank138320 +138321 POINT(41.77482108571801 -87.01099253178269) bank138321 +138322 POINT(41.398662834874074 -86.74261987995386) bank138322 +138323 POINT(41.27264300231394 -87.91152234294665) bank138323 +138324 POINT(42.69095876069077 -87.87910327829219) bank138324 +138325 POINT(42.11782197580074 -88.4326644412029) bank138325 +138326 POINT(41.332900117842584 -87.33944384433136) bank138326 +138327 POINT(41.13603961784461 -87.23052828259391) bank138327 +138328 POINT(42.66037544758796 -87.11168595586165) bank138328 +138329 POINT(41.25819049166317 -88.02507128532797) bank138329 +138330 POINT(41.039703747262 -87.47881035438326) bank138330 +138331 POINT(41.62610774183023 -88.2593629751263) bank138331 +138332 POINT(41.250534155368236 -88.61203176219524) bank138332 +138333 POINT(42.21320395536505 -87.38428055731256) bank138333 +138334 POINT(40.90538502955896 -87.51335821634498) bank138334 +138335 POINT(41.60770891767793 -88.31035020567761) bank138335 +138336 POINT(42.86570158476085 -88.62061744766595) bank138336 +138337 POINT(40.91426614227153 -86.88132542188619) bank138337 +138338 POINT(41.024791190002226 -88.28011717459843) bank138338 +138339 POINT(41.89412225180199 -87.0181544566222) bank138339 +138340 POINT(41.94389640556642 -87.61716967203779) bank138340 +138341 POINT(41.67133608913332 -87.00443062074575) bank138341 +138342 POINT(41.16410903917195 -87.77986794812466) bank138342 +138343 POINT(41.8147194122186 -88.37027058767391) bank138343 +138344 POINT(42.5864325261692 -87.05253899951681) bank138344 +138345 POINT(41.92628252705717 -86.87448222554457) bank138345 +138346 POINT(41.58929249638432 -87.2142960975116) bank138346 +138347 POINT(41.4674490915995 -88.13767276813515) bank138347 +138348 POINT(42.38998851825808 -86.92016212515148) bank138348 +138349 POINT(41.104775980858804 -87.71792592927117) bank138349 +138350 POINT(42.12229445658752 -88.5882356326741) bank138350 +138351 POINT(40.959276150245024 -87.30077034393054) bank138351 +138352 POINT(42.45263235316455 -86.86705935887663) bank138352 +138353 POINT(41.59265354398604 -87.70257647453285) bank138353 +138354 POINT(42.15012476312501 -87.97345309454806) bank138354 +138355 POINT(41.08684823988699 -86.77747661464217) bank138355 +138356 POINT(42.66152670742951 -87.81478862564333) bank138356 +138357 POINT(42.379352780874264 -87.70625677861727) bank138357 +138358 POINT(41.8998538852012 -87.24363842161856) bank138358 +138359 POINT(41.28906469665473 -87.97454278786536) bank138359 +138360 POINT(42.42548163379115 -88.60940765341412) bank138360 +138361 POINT(41.7466972258467 -88.13461944839479) bank138361 +138362 POINT(42.25980302122483 -87.86825092914889) bank138362 +138363 POINT(41.6397107492841 -86.81232278548003) bank138363 +138364 POINT(42.72186838494623 -87.12600086342078) bank138364 +138365 POINT(42.377551866933864 -88.13593212809019) bank138365 +138366 POINT(41.26058981167537 -88.1000037616268) bank138366 +138367 POINT(41.95552615309306 -87.62499794597353) bank138367 +138368 POINT(42.38933756776103 -87.06067977857377) bank138368 +138369 POINT(42.533325185980225 -87.60175342327557) bank138369 +138370 POINT(42.183764712173954 -86.90232288905713) bank138370 +138371 POINT(41.522712730093865 -87.7599532792867) bank138371 +138372 POINT(42.86824832742593 -87.5849298858868) bank138372 +138373 POINT(41.96663094288734 -88.1544847800399) bank138373 +138374 POINT(42.727634912747305 -87.9169643528419) bank138374 +138375 POINT(41.51453454138755 -87.01442448916661) bank138375 +138376 POINT(41.34912900809506 -88.18039096991085) bank138376 +138377 POINT(42.38606637711605 -87.06857966897668) bank138377 +138378 POINT(42.478989941464306 -87.70035521725676) bank138378 +138379 POINT(42.2191710977833 -87.74748383308571) bank138379 +138380 POINT(41.39996086116114 -86.66472745286625) bank138380 +138381 POINT(41.84652850498105 -87.96237679857602) bank138381 +138382 POINT(41.51879138074761 -86.8063289837859) bank138382 +138383 POINT(41.59664118516492 -86.73869540918672) bank138383 +138384 POINT(41.87161048714923 -87.74566733651109) bank138384 +138385 POINT(41.60969859021568 -87.44259410002496) bank138385 +138386 POINT(40.9131606524199 -87.79561835105179) bank138386 +138387 POINT(42.30475499438007 -87.6719346117093) bank138387 +138388 POINT(41.812176832821145 -86.84250621873089) bank138388 +138389 POINT(41.301308984276076 -87.42532708480388) bank138389 +138390 POINT(41.39206175577522 -87.33908305821916) bank138390 +138391 POINT(41.92878976504507 -87.44585335287994) bank138391 +138392 POINT(40.992179124158895 -88.36358477295029) bank138392 +138393 POINT(41.38179144099459 -88.41170618666202) bank138393 +138394 POINT(42.682974575378275 -87.13360654331463) bank138394 +138395 POINT(40.90754964929393 -86.66014488741378) bank138395 +138396 POINT(42.07702830105688 -88.40940818841867) bank138396 +138397 POINT(41.07034587526525 -87.12074984041952) bank138397 +138398 POINT(40.92447098029267 -88.24016871963693) bank138398 +138399 POINT(42.30028921433774 -88.41340785425582) bank138399 +138400 POINT(42.462232255571585 -86.69405175242638) bank138400 +138401 POINT(41.38245299847847 -88.00221484950883) bank138401 +138402 POINT(42.29998480813776 -87.52270776976651) bank138402 +138403 POINT(41.84531291249138 -88.5587365254633) bank138403 +138404 POINT(41.58981168463298 -87.9483665680803) bank138404 +138405 POINT(42.465002396378424 -88.6005771439324) bank138405 +138406 POINT(41.90514354251447 -87.65812810189404) bank138406 +138407 POINT(41.29612563048191 -87.59271656390749) bank138407 +138408 POINT(40.982655419488474 -88.533440998819) bank138408 +138409 POINT(42.541863066115795 -87.86852478863888) bank138409 +138410 POINT(41.72813166132875 -88.41972693611801) bank138410 +138411 POINT(42.581844221173995 -87.41608926354766) bank138411 +138412 POINT(42.85254650699734 -87.48257592249236) bank138412 +138413 POINT(41.30673572666038 -87.26449059156025) bank138413 +138414 POINT(42.300313909576225 -88.05896749956206) bank138414 +138415 POINT(41.236718532412425 -88.11924301790515) bank138415 +138416 POINT(42.60414507267528 -86.86660718732207) bank138416 +138417 POINT(42.51804893678165 -86.66504258452035) bank138417 +138418 POINT(40.88554884827299 -87.832038372557) bank138418 +138419 POINT(41.14559924943902 -87.15272254328877) bank138419 +138420 POINT(42.26896880456382 -87.17409584099046) bank138420 +138421 POINT(41.806873047273555 -88.0571142733021) bank138421 +138422 POINT(41.15707197324232 -88.01940609736846) bank138422 +138423 POINT(42.0725499278664 -86.66329935129981) bank138423 +138424 POINT(41.93208913031812 -88.23756224795476) bank138424 +138425 POINT(42.30695605351134 -87.17759657642671) bank138425 +138426 POINT(41.80817486013043 -87.32702647602953) bank138426 +138427 POINT(41.776947244278816 -87.5625159028222) bank138427 +138428 POINT(42.30470104818303 -87.8632237689721) bank138428 +138429 POINT(42.01271502643619 -87.26907499402296) bank138429 +138430 POINT(41.862343590391035 -87.3721474781201) bank138430 +138431 POINT(41.507922562378354 -87.33460426615568) bank138431 +138432 POINT(42.751434110046034 -86.81834736266993) bank138432 +138433 POINT(42.338495687920044 -87.66737937214575) bank138433 +138434 POINT(42.786347407796725 -87.0944260477552) bank138434 +138435 POINT(42.847312799239404 -87.41687332166967) bank138435 +138436 POINT(42.39210180633007 -87.46345611681865) bank138436 +138437 POINT(41.622697150210435 -87.31557653530409) bank138437 +138438 POINT(40.93618814528426 -87.06597306573985) bank138438 +138439 POINT(42.12525863826888 -86.83405176193038) bank138439 +138440 POINT(42.264121474010295 -87.10979473388707) bank138440 +138441 POINT(41.29277151051154 -87.40921641476413) bank138441 +138442 POINT(41.59103526374346 -87.45981199953692) bank138442 +138443 POINT(42.74622048677899 -87.6735927085089) bank138443 +138444 POINT(41.11551846466904 -87.48564475141441) bank138444 +138445 POINT(41.76117631311821 -88.42780300858522) bank138445 +138446 POINT(42.552880182673135 -87.45769790455654) bank138446 +138447 POINT(42.825762815617324 -88.40322815176795) bank138447 +138448 POINT(41.21408532652542 -88.16283567162888) bank138448 +138449 POINT(41.515079592921126 -86.68844959243205) bank138449 +138450 POINT(42.81004191850186 -87.1053890643354) bank138450 +138451 POINT(41.12817560921266 -87.51758815333284) bank138451 +138452 POINT(42.71153866608091 -88.23355068664704) bank138452 +138453 POINT(41.94264328067091 -88.5251758119753) bank138453 +138454 POINT(42.55076639076765 -86.69495682741461) bank138454 +138455 POINT(41.067233617300616 -87.86260108052309) bank138455 +138456 POINT(41.67812470738405 -87.13036493960854) bank138456 +138457 POINT(41.75747813672345 -87.88110340521276) bank138457 +138458 POINT(41.302303899044 -87.82350843525138) bank138458 +138459 POINT(41.043689824715734 -87.73445615239541) bank138459 +138460 POINT(42.08903727839738 -87.85382407246614) bank138460 +138461 POINT(42.21214371074692 -87.13378953040159) bank138461 +138462 POINT(42.460020581631944 -88.5134275555511) bank138462 +138463 POINT(42.46979041246538 -86.78285006473791) bank138463 +138464 POINT(42.396318688328456 -87.55298460207217) bank138464 +138465 POINT(42.333083921815096 -88.2445837598513) bank138465 +138466 POINT(41.73531766659177 -87.80358315472687) bank138466 +138467 POINT(41.09196880730267 -87.3265423906346) bank138467 +138468 POINT(42.37013011413257 -88.03891568535299) bank138468 +138469 POINT(41.77067789807304 -88.41916258475369) bank138469 +138470 POINT(41.363626335735354 -87.60454179198581) bank138470 +138471 POINT(41.15004753721127 -87.19103321530115) bank138471 +138472 POINT(41.10272902367923 -87.65609648341717) bank138472 +138473 POINT(42.26454704840353 -86.84665976254419) bank138473 +138474 POINT(41.24038726851934 -88.08425398716237) bank138474 +138475 POINT(41.3154261401639 -87.53834549117616) bank138475 +138476 POINT(41.20105661274469 -87.03325319226215) bank138476 +138477 POINT(41.69326186164893 -87.21552634902649) bank138477 +138478 POINT(41.768063097421816 -88.56728059349362) bank138478 +138479 POINT(41.79230245015877 -88.4491468491709) bank138479 +138480 POINT(41.66265705515761 -86.88086795580713) bank138480 +138481 POINT(41.90397040669413 -86.6326671345588) bank138481 +138482 POINT(42.275819907374924 -87.42509932787377) bank138482 +138483 POINT(42.72890333470515 -86.6455584461435) bank138483 +138484 POINT(40.997299446248235 -87.43694102505846) bank138484 +138485 POINT(42.11205410442155 -88.17127689280917) bank138485 +138486 POINT(42.79447102595773 -87.89347166169507) bank138486 +138487 POINT(40.98052892199486 -87.90392130543567) bank138487 +138488 POINT(42.25887458315137 -86.73836543188844) bank138488 +138489 POINT(41.77895437473558 -88.10105591133077) bank138489 +138490 POINT(41.82924401166829 -88.59491967724186) bank138490 +138491 POINT(42.33341051840397 -87.69893683122424) bank138491 +138492 POINT(42.21105637235385 -86.97921936901339) bank138492 +138493 POINT(41.87136678905301 -87.40394513546333) bank138493 +138494 POINT(41.51260375337949 -88.44729399479253) bank138494 +138495 POINT(41.93978444894368 -88.46173176922012) bank138495 +138496 POINT(42.44406639518745 -87.3364653953441) bank138496 +138497 POINT(41.93791635729957 -88.54090728190057) bank138497 +138498 POINT(42.54988421811535 -87.73915273961033) bank138498 +138499 POINT(41.907496388139485 -86.69115535729604) bank138499 +138500 POINT(41.355150021721414 -86.7785400347258) bank138500 +138501 POINT(42.466642940782336 -88.34596543006502) bank138501 +138502 POINT(41.608246073242775 -87.36020352733436) bank138502 +138503 POINT(41.7051851092161 -88.28973548023475) bank138503 +138504 POINT(41.486159299773576 -87.02139936125899) bank138504 +138505 POINT(41.58153045577734 -87.50056169110961) bank138505 +138506 POINT(41.55582718134301 -87.80443618080322) bank138506 +138507 POINT(41.11690648447947 -88.47582244907126) bank138507 +138508 POINT(42.22108484967833 -87.17275566497497) bank138508 +138509 POINT(40.9241668544216 -87.72813199869088) bank138509 +138510 POINT(41.09763519908755 -88.47115279397208) bank138510 +138511 POINT(41.65346875227746 -87.56168390051734) bank138511 +138512 POINT(41.28130350315315 -86.87414091011742) bank138512 +138513 POINT(42.64784562217599 -87.3551062920322) bank138513 +138514 POINT(41.75590533534955 -86.85666502884942) bank138514 +138515 POINT(41.37444183793898 -88.35906397186177) bank138515 +138516 POINT(42.282541978328396 -87.95823499351319) bank138516 +138517 POINT(41.20829166618283 -87.93284192032472) bank138517 +138518 POINT(41.739283168045105 -87.22430012087104) bank138518 +138519 POINT(42.78063942548761 -88.43694927367096) bank138519 +138520 POINT(42.551373718293 -87.67565926636534) bank138520 +138521 POINT(40.90193742750871 -87.70361781983695) bank138521 +138522 POINT(41.78642464874006 -87.77215612299318) bank138522 +138523 POINT(42.40805439765122 -88.54350215368973) bank138523 +138524 POINT(42.29950189104375 -87.47223090947378) bank138524 +138525 POINT(42.07461179862174 -88.22961216531904) bank138525 +138526 POINT(41.16547685826884 -87.34415383366073) bank138526 +138527 POINT(41.65977702321434 -88.5864895912153) bank138527 +138528 POINT(41.15428126638828 -88.03087377012083) bank138528 +138529 POINT(41.747358638414745 -88.08507634763839) bank138529 +138530 POINT(41.31428307634666 -88.48094463515324) bank138530 +138531 POINT(42.850430349532104 -87.48851475749973) bank138531 +138532 POINT(42.607930431719424 -88.39812855229945) bank138532 +138533 POINT(41.238922448508895 -87.67078580639122) bank138533 +138534 POINT(42.73734561085512 -87.02340823913585) bank138534 +138535 POINT(42.12307237274851 -87.95775334247628) bank138535 +138536 POINT(41.72844913132556 -87.84938066049153) bank138536 +138537 POINT(41.057167809049275 -87.99219540820988) bank138537 +138538 POINT(42.36640143446394 -88.28902439743034) bank138538 +138539 POINT(41.98146688661101 -86.8524106298474) bank138539 +138540 POINT(41.142221705136905 -88.15222517919553) bank138540 +138541 POINT(42.11752282752592 -86.70448074535773) bank138541 +138542 POINT(42.5882880990235 -87.43338785481808) bank138542 +138543 POINT(41.35187760765899 -88.27388453447215) bank138543 +138544 POINT(42.14448065218689 -87.00196836004672) bank138544 +138545 POINT(41.67164228873739 -86.71104209347607) bank138545 +138546 POINT(41.951792922038045 -88.02030850604922) bank138546 +138547 POINT(41.2940393041915 -88.25877863379142) bank138547 +138548 POINT(41.90657160377396 -88.4908870203824) bank138548 +138549 POINT(41.613302375909136 -87.46697455874364) bank138549 +138550 POINT(41.04831867130986 -87.93964536280319) bank138550 +138551 POINT(41.97460035427175 -86.74341529354322) bank138551 +138552 POINT(42.83817115368318 -87.73395373239906) bank138552 +138553 POINT(42.836203519292425 -86.89270082787874) bank138553 +138554 POINT(42.36252325469601 -88.37342846615736) bank138554 +138555 POINT(41.14969737373036 -87.4898734073802) bank138555 +138556 POINT(40.878536927705106 -87.64763203142877) bank138556 +138557 POINT(42.804836874841534 -86.74347194794294) bank138557 +138558 POINT(42.050916974616435 -87.0786654006298) bank138558 +138559 POINT(42.13437624446199 -88.59561886337832) bank138559 +138560 POINT(41.39272173389641 -88.25028339514694) bank138560 +138561 POINT(42.640155236291136 -87.79980991327574) bank138561 +138562 POINT(42.83702617448215 -87.64438264035908) bank138562 +138563 POINT(41.377526100822884 -87.26840129293142) bank138563 +138564 POINT(41.32722846401717 -87.72731932391245) bank138564 +138565 POINT(41.976866403077274 -87.15035120279083) bank138565 +138566 POINT(42.28927691762199 -88.50691460211867) bank138566 +138567 POINT(42.35880020262696 -86.6935885671354) bank138567 +138568 POINT(42.63081328001655 -88.18646065321477) bank138568 +138569 POINT(42.29844651436021 -87.40778171927774) bank138569 +138570 POINT(42.25723975554663 -87.81591476412412) bank138570 +138571 POINT(41.42029700721618 -87.2113322531085) bank138571 +138572 POINT(42.49317597330951 -86.65839080211015) bank138572 +138573 POINT(42.060223733907456 -88.61358451818055) bank138573 +138574 POINT(42.21534384877056 -88.13606634088582) bank138574 +138575 POINT(41.258390044353845 -87.70382396126033) bank138575 +138576 POINT(41.98313706671609 -87.22420193432127) bank138576 +138577 POINT(42.14078946111182 -88.5353441474599) bank138577 +138578 POINT(41.75611989007458 -88.32930081314167) bank138578 +138579 POINT(42.0772915335036 -87.51403466498213) bank138579 +138580 POINT(42.16233314170089 -87.17703417366953) bank138580 +138581 POINT(42.33809586046576 -87.23591360855161) bank138581 +138582 POINT(41.98341015078544 -86.99940526155942) bank138582 +138583 POINT(42.46879012125998 -87.69650394487701) bank138583 +138584 POINT(41.6906445939666 -87.1358947725912) bank138584 +138585 POINT(42.25648259686846 -87.58869112842272) bank138585 +138586 POINT(41.02136516758507 -88.62061935070328) bank138586 +138587 POINT(42.46425477415468 -88.28730359344729) bank138587 +138588 POINT(42.5852662126793 -87.92401235080985) bank138588 +138589 POINT(41.71907945212505 -86.73984181023853) bank138589 +138590 POINT(42.31705195429215 -88.1506314477065) bank138590 +138591 POINT(41.328791941926035 -87.74770615627695) bank138591 +138592 POINT(41.33910280344449 -88.25918058007748) bank138592 +138593 POINT(41.20141964796521 -88.53019824037959) bank138593 +138594 POINT(41.00372257466375 -87.57654897238032) bank138594 +138595 POINT(41.39250573211486 -88.44190074682112) bank138595 +138596 POINT(41.4388623373988 -86.89297526263297) bank138596 +138597 POINT(41.899782307180025 -87.16435616773089) bank138597 +138598 POINT(42.31151333916476 -88.5601307222591) bank138598 +138599 POINT(42.846550675048306 -87.02177185758576) bank138599 +138600 POINT(42.10310933908026 -87.17454088929807) bank138600 +138601 POINT(42.724735409563074 -88.39708176146208) bank138601 +138602 POINT(42.61529857599087 -86.77754473588135) bank138602 +138603 POINT(41.53740590690132 -88.61162202314361) bank138603 +138604 POINT(41.324783109337034 -87.23417399378489) bank138604 +138605 POINT(41.586337420289 -88.11182024397412) bank138605 +138606 POINT(41.39698212565268 -87.54444185088546) bank138606 +138607 POINT(41.53366608425376 -86.65437930242962) bank138607 +138608 POINT(40.89414656616538 -86.8648227008064) bank138608 +138609 POINT(41.33846133479289 -86.8632492267086) bank138609 +138610 POINT(42.3544178165684 -87.99747509278052) bank138610 +138611 POINT(42.70416683474888 -88.54256823662656) bank138611 +138612 POINT(42.31128882462854 -87.24454802215503) bank138612 +138613 POINT(41.45460755981179 -88.52074399132336) bank138613 +138614 POINT(41.43050594201558 -87.11525115906346) bank138614 +138615 POINT(42.36908675251136 -88.46162577754369) bank138615 +138616 POINT(42.47290306439929 -87.7616277485981) bank138616 +138617 POINT(42.49874115403848 -87.9480571442354) bank138617 +138618 POINT(42.05179398473948 -88.3823613519071) bank138618 +138619 POINT(42.05257576281679 -88.31081988190941) bank138619 +138620 POINT(42.19458715664031 -87.58399869352988) bank138620 +138621 POINT(41.19336107481381 -87.07558937983845) bank138621 +138622 POINT(42.63906036931137 -86.83433435183555) bank138622 +138623 POINT(42.4924159026797 -87.91224184144976) bank138623 +138624 POINT(42.681030225651796 -87.30659523188439) bank138624 +138625 POINT(42.76251418379336 -88.49079009851307) bank138625 +138626 POINT(42.74519331576495 -87.73566587670777) bank138626 +138627 POINT(42.578227145982744 -87.29463448950212) bank138627 +138628 POINT(41.17494781478173 -87.91640211883865) bank138628 +138629 POINT(42.47340158030276 -86.98447632863434) bank138629 +138630 POINT(42.144979896959036 -87.78487506222098) bank138630 +138631 POINT(42.453977901305876 -87.65766775348588) bank138631 +138632 POINT(41.40214527678152 -86.7857803044061) bank138632 +138633 POINT(41.630793243681104 -87.0303688376473) bank138633 +138634 POINT(41.44828588641117 -87.88739712456884) bank138634 +138635 POINT(41.58861264081136 -86.92731200232451) bank138635 +138636 POINT(42.00076781667249 -87.74151037694045) bank138636 +138637 POINT(40.97593733784196 -86.68985604566686) bank138637 +138638 POINT(41.25495405691915 -87.5470574733399) bank138638 +138639 POINT(41.194300427434655 -86.9294733774552) bank138639 +138640 POINT(41.59201371643951 -87.4485212727268) bank138640 +138641 POINT(41.91172128866392 -86.65119155773966) bank138641 +138642 POINT(41.97517663259037 -87.59102416329968) bank138642 +138643 POINT(42.02269723406228 -87.87753631971184) bank138643 +138644 POINT(42.87304240706668 -87.7977440461271) bank138644 +138645 POINT(41.423000523395466 -87.72166681990616) bank138645 +138646 POINT(41.9894549464029 -87.87920315239288) bank138646 +138647 POINT(41.198069508686956 -87.11926674815157) bank138647 +138648 POINT(42.01596424229535 -88.57299526319953) bank138648 +138649 POINT(42.08008737153735 -86.64446702331108) bank138649 +138650 POINT(41.37984223537966 -87.1438901460299) bank138650 +138651 POINT(42.67492658025829 -88.29379922251867) bank138651 +138652 POINT(41.094865264732945 -87.69556078451878) bank138652 +138653 POINT(42.34647554504875 -86.92718774267405) bank138653 +138654 POINT(42.22267021263097 -88.20566386399265) bank138654 +138655 POINT(42.759023189055846 -87.36045583177211) bank138655 +138656 POINT(41.15926695985584 -86.7442652720535) bank138656 +138657 POINT(42.533765234279876 -87.94560694747614) bank138657 +138658 POINT(41.61005439537538 -87.29970410849474) bank138658 +138659 POINT(42.65672771849858 -87.67958749977126) bank138659 +138660 POINT(41.80769700320964 -86.77000858987037) bank138660 +138661 POINT(41.78727135628026 -87.60021756883764) bank138661 +138662 POINT(42.752616189667826 -87.34820610650925) bank138662 +138663 POINT(41.5325468655724 -87.09732839617885) bank138663 +138664 POINT(40.96619462034086 -86.87933381569309) bank138664 +138665 POINT(41.802995213151455 -86.84713407583392) bank138665 +138666 POINT(41.61594845651655 -88.03594084400443) bank138666 +138667 POINT(42.547736598941285 -88.35231296217393) bank138667 +138668 POINT(41.32208813965479 -87.90553602745318) bank138668 +138669 POINT(41.82470248268863 -88.5559573233875) bank138669 +138670 POINT(41.55856824180981 -88.38013159878463) bank138670 +138671 POINT(41.23782198733041 -88.26723311673751) bank138671 +138672 POINT(41.561043603456916 -87.84182039525982) bank138672 +138673 POINT(42.10686381398767 -86.94769389042816) bank138673 +138674 POINT(42.00353958247641 -87.02202687542426) bank138674 +138675 POINT(42.67973197364256 -87.33314796636172) bank138675 +138676 POINT(41.25771269551892 -87.20656416394903) bank138676 +138677 POINT(41.18095381356438 -87.41968254913627) bank138677 +138678 POINT(41.576873051396376 -86.9668461173093) bank138678 +138679 POINT(40.92995682754243 -87.73608196461294) bank138679 +138680 POINT(42.70025014289742 -87.96325268589136) bank138680 +138681 POINT(41.489578639379246 -87.12353893100456) bank138681 +138682 POINT(41.276847739902145 -88.28910282471199) bank138682 +138683 POINT(41.12933256556757 -87.36934782137477) bank138683 +138684 POINT(41.561248507058025 -88.37163363960921) bank138684 +138685 POINT(41.37501470765709 -88.02129203954983) bank138685 +138686 POINT(42.19609782600811 -87.91736367334833) bank138686 +138687 POINT(42.39526864280904 -87.24266226697125) bank138687 +138688 POINT(42.867749826753695 -88.47601978562949) bank138688 +138689 POINT(42.011944829335285 -87.6052271562852) bank138689 +138690 POINT(42.759934903719056 -87.24368395220472) bank138690 +138691 POINT(41.60706951087686 -87.49582693571077) bank138691 +138692 POINT(41.76009970960775 -86.91943868477388) bank138692 +138693 POINT(41.64193340643021 -87.29325941940252) bank138693 +138694 POINT(41.70437343271326 -88.37830922321545) bank138694 +138695 POINT(41.93268656858833 -87.0162567351158) bank138695 +138696 POINT(41.30793237452176 -87.58681802837273) bank138696 +138697 POINT(41.56870257475303 -88.58233735316549) bank138697 +138698 POINT(42.36073012232542 -87.72486934783785) bank138698 +138699 POINT(41.03623393001763 -86.73397044048511) bank138699 +138700 POINT(41.85314801596964 -88.51486782179302) bank138700 +138701 POINT(42.24107897824965 -88.2178937292292) bank138701 +138702 POINT(42.626401261449665 -87.71007720233497) bank138702 +138703 POINT(42.77668600794127 -86.8933931221032) bank138703 +138704 POINT(42.254606687968426 -87.52272062238437) bank138704 +138705 POINT(41.585817821776274 -87.7921560028692) bank138705 +138706 POINT(42.81441167400669 -86.63754651844725) bank138706 +138707 POINT(42.18957797289228 -87.16364251537675) bank138707 +138708 POINT(41.794946124404916 -88.54542842443828) bank138708 +138709 POINT(42.84692671637878 -88.0294191731555) bank138709 +138710 POINT(42.69287635376712 -87.99127717477646) bank138710 +138711 POINT(42.36794590152191 -86.80519267218278) bank138711 +138712 POINT(42.00825671119786 -87.45280512831302) bank138712 +138713 POINT(42.13900188727802 -88.05980950748632) bank138713 +138714 POINT(40.982976038333284 -86.79076859782957) bank138714 +138715 POINT(41.237568700463385 -87.08078004655235) bank138715 +138716 POINT(41.626896003403566 -87.4062821227439) bank138716 +138717 POINT(41.14916848948241 -87.60270449476779) bank138717 +138718 POINT(41.044773954852886 -88.44519519247402) bank138718 +138719 POINT(42.08300369148839 -88.35795110937478) bank138719 +138720 POINT(40.95631568006922 -87.45646161764657) bank138720 +138721 POINT(42.593977070528624 -86.86921015303203) bank138721 +138722 POINT(42.04809027105208 -88.07559698016307) bank138722 +138723 POINT(41.755557099929604 -86.70019595695888) bank138723 +138724 POINT(42.72508401865637 -86.86059906858928) bank138724 +138725 POINT(42.03730276337526 -87.04395672766772) bank138725 +138726 POINT(42.58859932937103 -87.549851425502) bank138726 +138727 POINT(42.32127833423094 -87.67343493954219) bank138727 +138728 POINT(42.87581017961815 -87.01293749783926) bank138728 +138729 POINT(41.988805337820416 -88.0771677385246) bank138729 +138730 POINT(42.46746764310523 -87.45124308260213) bank138730 +138731 POINT(42.68834402177351 -87.3079169899626) bank138731 +138732 POINT(41.171309235009346 -87.9130776856207) bank138732 +138733 POINT(42.65848243620016 -88.23576307402661) bank138733 +138734 POINT(41.362726055137934 -88.14565137936455) bank138734 +138735 POINT(42.44726212437899 -88.14748158689758) bank138735 +138736 POINT(42.58805816483947 -88.57958455346215) bank138736 +138737 POINT(42.06316403605235 -87.39993937946058) bank138737 +138738 POINT(42.123316928678854 -87.26396292821345) bank138738 +138739 POINT(42.02699886365549 -87.8003128135266) bank138739 +138740 POINT(41.43505195814575 -88.05175970067711) bank138740 +138741 POINT(41.49019472758524 -86.8639369369137) bank138741 +138742 POINT(41.57357235304385 -88.18465120851279) bank138742 +138743 POINT(42.52598072398043 -86.91394913886255) bank138743 +138744 POINT(42.18007292805891 -88.62107201350956) bank138744 +138745 POINT(42.205800987536435 -87.9859938218833) bank138745 +138746 POINT(42.67943421904537 -87.49473899748784) bank138746 +138747 POINT(41.248798002272046 -87.73194270016226) bank138747 +138748 POINT(41.46144090688099 -88.29506816067845) bank138748 +138749 POINT(42.63788035714153 -88.45297629631362) bank138749 +138750 POINT(41.69568821210237 -88.33492091243802) bank138750 +138751 POINT(42.29882318577222 -86.90173034989226) bank138751 +138752 POINT(42.30886182725091 -88.25325322842191) bank138752 +138753 POINT(42.346918743081815 -87.70442743961786) bank138753 +138754 POINT(41.97680008736197 -87.76132430684666) bank138754 +138755 POINT(42.82697844807201 -88.49252377485307) bank138755 +138756 POINT(42.46431801809992 -88.01718420420951) bank138756 +138757 POINT(42.1268220308484 -87.18657651738036) bank138757 +138758 POINT(42.848167243633576 -87.99951846865727) bank138758 +138759 POINT(41.099121108585074 -87.95466765402338) bank138759 +138760 POINT(41.4774295512328 -86.85912863303129) bank138760 +138761 POINT(42.55625892393901 -87.5773411237911) bank138761 +138762 POINT(41.216579479773856 -88.50571404040976) bank138762 +138763 POINT(41.016242567528224 -88.60601772567702) bank138763 +138764 POINT(40.97871432604864 -87.13719073684999) bank138764 +138765 POINT(41.87653652239402 -86.77145026790113) bank138765 +138766 POINT(41.583975225995445 -87.77370151547154) bank138766 +138767 POINT(41.778627342499455 -87.10258142832102) bank138767 +138768 POINT(42.71721528801602 -88.4411423519344) bank138768 +138769 POINT(42.30859784115227 -87.6702577942452) bank138769 +138770 POINT(41.62025364673551 -88.58865056833257) bank138770 +138771 POINT(41.22938994608868 -88.14878662259387) bank138771 +138772 POINT(40.95610809614741 -87.51527686119042) bank138772 +138773 POINT(41.76429462359286 -88.33327478238688) bank138773 +138774 POINT(42.13672840619217 -88.36840927525706) bank138774 +138775 POINT(41.170704474631265 -87.43075450001434) bank138775 +138776 POINT(41.18420354423954 -87.87895020755602) bank138776 +138777 POINT(40.96016304525306 -87.8907949364497) bank138777 +138778 POINT(41.95169532088006 -86.69672940700845) bank138778 +138779 POINT(41.09401488306048 -86.67566450499669) bank138779 +138780 POINT(41.268674479384394 -87.72537450255341) bank138780 +138781 POINT(41.08708338066081 -87.63953824313009) bank138781 +138782 POINT(41.73324340565245 -88.00283101844542) bank138782 +138783 POINT(41.366595891610615 -88.3456468415908) bank138783 +138784 POINT(41.28980155591682 -86.66233454585647) bank138784 +138785 POINT(42.76718508340731 -86.94863796474976) bank138785 +138786 POINT(41.6901456251687 -87.51671090083562) bank138786 +138787 POINT(42.63242491308946 -88.1315795221827) bank138787 +138788 POINT(41.382719280696826 -86.68280498750492) bank138788 +138789 POINT(41.57616360076375 -87.31188156498283) bank138789 +138790 POINT(41.735141654102875 -88.3224256841389) bank138790 +138791 POINT(41.041054554055705 -87.86425211321792) bank138791 +138792 POINT(41.012523397856775 -86.88215891013427) bank138792 +138793 POINT(42.279106013813276 -86.74036837038626) bank138793 +138794 POINT(42.244791172408924 -87.36771153891725) bank138794 +138795 POINT(42.273329039301984 -88.61698651749796) bank138795 +138796 POINT(42.199896456722094 -88.57676881182928) bank138796 +138797 POINT(42.05920237882662 -86.80873329179067) bank138797 +138798 POINT(41.16433404774895 -88.22309401418781) bank138798 +138799 POINT(41.122384746093424 -88.34294748553691) bank138799 +138800 POINT(42.444630831738706 -88.33370594604573) bank138800 +138801 POINT(41.87220915409588 -87.06354602077569) bank138801 +138802 POINT(41.948184813264184 -86.92364054135933) bank138802 +138803 POINT(42.27329804376967 -87.8929435618759) bank138803 +138804 POINT(42.07984518488281 -87.61120034852091) bank138804 +138805 POINT(41.196051138390665 -88.53038023255847) bank138805 +138806 POINT(42.152387569499616 -88.48872444188487) bank138806 +138807 POINT(41.893999808040405 -86.66364279301104) bank138807 +138808 POINT(42.80593637732683 -87.23753705339311) bank138808 +138809 POINT(41.07962545139651 -87.6999317538372) bank138809 +138810 POINT(41.52705326704262 -88.37886298544316) bank138810 +138811 POINT(40.963919459664645 -88.21819876388267) bank138811 +138812 POINT(41.43896740640319 -88.61758052792057) bank138812 +138813 POINT(42.65827298176864 -88.36617887259249) bank138813 +138814 POINT(41.7075376325686 -86.94927702453818) bank138814 +138815 POINT(41.11402460384191 -88.11245652143747) bank138815 +138816 POINT(41.535889147603214 -87.20180241537642) bank138816 +138817 POINT(42.066964933776454 -87.32045967937306) bank138817 +138818 POINT(41.53888961943941 -88.01753479192558) bank138818 +138819 POINT(42.25513675320315 -87.48520956703892) bank138819 +138820 POINT(41.49492442173579 -86.94759801640015) bank138820 +138821 POINT(42.3161935362262 -87.32830249288925) bank138821 +138822 POINT(42.700708908977944 -86.82955089655714) bank138822 +138823 POINT(42.02679482189002 -87.89199321334293) bank138823 +138824 POINT(42.106149257083466 -88.14311478833473) bank138824 +138825 POINT(41.7090952372039 -87.37534087536778) bank138825 +138826 POINT(42.408092708710214 -88.20131532556344) bank138826 +138827 POINT(41.823434666902244 -87.00080934775045) bank138827 +138828 POINT(42.328334245572364 -87.76585080947244) bank138828 +138829 POINT(41.58003850690214 -87.17332636222106) bank138829 +138830 POINT(42.81668945701054 -88.2058089195474) bank138830 +138831 POINT(41.91697396346654 -87.35853451482805) bank138831 +138832 POINT(42.22398147754086 -88.17000998692984) bank138832 +138833 POINT(42.82746727068569 -86.74863721163075) bank138833 +138834 POINT(41.141476448497464 -86.79368193321895) bank138834 +138835 POINT(41.22601728955713 -87.92758143308558) bank138835 +138836 POINT(42.07839861019311 -88.23629677706975) bank138836 +138837 POINT(42.348715030107655 -88.5904799913545) bank138837 +138838 POINT(42.00939373944267 -87.7787632849821) bank138838 +138839 POINT(41.746128299221176 -88.34207611241663) bank138839 +138840 POINT(42.07902475862409 -88.09092615447133) bank138840 +138841 POINT(42.560355904305815 -87.80256329207789) bank138841 +138842 POINT(42.851601787030404 -87.76775672341715) bank138842 +138843 POINT(41.71116901983324 -87.00586133112871) bank138843 +138844 POINT(41.394429244804975 -88.46466311704539) bank138844 +138845 POINT(42.057600041334204 -87.73359426485185) bank138845 +138846 POINT(41.0710866625043 -88.41040527521754) bank138846 +138847 POINT(41.7021496799648 -86.98592360520293) bank138847 +138848 POINT(42.828778256677246 -87.66505523146688) bank138848 +138849 POINT(42.486350749384705 -86.65001780560584) bank138849 +138850 POINT(42.76188139191424 -88.40019807497772) bank138850 +138851 POINT(41.999881594593724 -88.29233604274984) bank138851 +138852 POINT(42.234971371728285 -87.15643041165936) bank138852 +138853 POINT(42.109443188632596 -87.82012309657853) bank138853 +138854 POINT(42.285406062072624 -87.15218178312027) bank138854 +138855 POINT(42.44875441903732 -88.36376735355809) bank138855 +138856 POINT(42.50165449340004 -88.19827380851052) bank138856 +138857 POINT(41.22472973811743 -88.31445902428939) bank138857 +138858 POINT(41.21145946483932 -87.79829714194442) bank138858 +138859 POINT(41.771457005883974 -87.09217299098783) bank138859 +138860 POINT(42.04417910105579 -87.54120066943075) bank138860 +138861 POINT(41.799423782569086 -87.21575502407093) bank138861 +138862 POINT(41.22063774758401 -87.61876892766968) bank138862 +138863 POINT(42.40521569785437 -87.91411633330095) bank138863 +138864 POINT(41.00818265279216 -87.13566347003356) bank138864 +138865 POINT(41.5723680723484 -87.14829052090111) bank138865 +138866 POINT(41.726347179769306 -87.55880157007428) bank138866 +138867 POINT(42.36569438198576 -88.18209014745396) bank138867 +138868 POINT(42.65680601767929 -87.72197286455905) bank138868 +138869 POINT(42.46752226976975 -88.58112626814571) bank138869 +138870 POINT(41.42551973756303 -87.44371933996402) bank138870 +138871 POINT(41.66081135049288 -87.9593718193483) bank138871 +138872 POINT(41.46232340316126 -88.44147152388003) bank138872 +138873 POINT(41.5282616300693 -87.29343040181797) bank138873 +138874 POINT(42.68948134691943 -86.89120743096701) bank138874 +138875 POINT(41.65094451927885 -88.3497840047874) bank138875 +138876 POINT(41.82401874713094 -88.12265994991557) bank138876 +138877 POINT(42.32148233074496 -88.60809568369373) bank138877 +138878 POINT(42.47484716890126 -87.35607897877038) bank138878 +138879 POINT(42.4700126212398 -86.71419317521922) bank138879 +138880 POINT(41.52023327773899 -88.07846579463855) bank138880 +138881 POINT(42.40834727139846 -88.29418472335354) bank138881 +138882 POINT(41.35623301936995 -88.4718533131526) bank138882 +138883 POINT(42.528479807495955 -86.83263614643492) bank138883 +138884 POINT(41.475151467989676 -87.12561001979198) bank138884 +138885 POINT(42.14691253518571 -87.16935869942574) bank138885 +138886 POINT(41.74688887216931 -88.27820595520208) bank138886 +138887 POINT(42.299111115721736 -86.83917243036583) bank138887 +138888 POINT(42.39808387314053 -87.85429089463335) bank138888 +138889 POINT(41.362243467454036 -88.25869786249834) bank138889 +138890 POINT(42.23427991094211 -87.1665380895971) bank138890 +138891 POINT(40.886018925606365 -88.23516068678911) bank138891 +138892 POINT(42.45257400590605 -87.48284366203883) bank138892 +138893 POINT(42.54830850771681 -87.84117345718875) bank138893 +138894 POINT(41.126966674248735 -87.12086828811665) bank138894 +138895 POINT(42.320071542009856 -86.80326268355716) bank138895 +138896 POINT(42.20842099570193 -86.83477715037525) bank138896 +138897 POINT(41.604798508219545 -87.22983743123422) bank138897 +138898 POINT(42.81612008389722 -88.61011914032434) bank138898 +138899 POINT(41.6391103953011 -86.80246255137274) bank138899 +138900 POINT(41.65460523283671 -87.04751785024577) bank138900 +138901 POINT(41.878255415304956 -87.85997946918918) bank138901 +138902 POINT(41.3157768151618 -86.8682689095356) bank138902 +138903 POINT(41.57952953303058 -88.47858936448354) bank138903 +138904 POINT(41.49221467780301 -87.91416743765352) bank138904 +138905 POINT(42.50569835496797 -88.38040045475243) bank138905 +138906 POINT(41.46659197073682 -86.90579458346885) bank138906 +138907 POINT(42.86068099951128 -88.01715947640191) bank138907 +138908 POINT(41.34943332610567 -88.18793164489165) bank138908 +138909 POINT(41.14912576084822 -87.33887937313062) bank138909 +138910 POINT(41.28264108096335 -88.26896941211315) bank138910 +138911 POINT(42.68200193141455 -87.12086001234732) bank138911 +138912 POINT(41.00758674444679 -87.1101396859378) bank138912 +138913 POINT(41.41417075843394 -86.67527183898582) bank138913 +138914 POINT(41.735659570122195 -87.33849192704427) bank138914 +138915 POINT(42.356261010640566 -88.04966054476877) bank138915 +138916 POINT(42.287600733168475 -87.56042077278201) bank138916 +138917 POINT(41.618199962083644 -87.21626556666368) bank138917 +138918 POINT(41.46302460216374 -88.24975019548123) bank138918 +138919 POINT(42.5077032546989 -87.37822913208828) bank138919 +138920 POINT(41.756554303607 -88.50089456223304) bank138920 +138921 POINT(41.04447167991057 -86.81517301402899) bank138921 +138922 POINT(41.62713921005786 -87.31353627806675) bank138922 +138923 POINT(41.6943214760832 -86.73493317387289) bank138923 +138924 POINT(41.772001567424525 -87.7387849284585) bank138924 +138925 POINT(41.45220396341198 -88.29982050795302) bank138925 +138926 POINT(42.2347422487385 -87.96670073474834) bank138926 +138927 POINT(41.802774192196615 -88.0930521815466) bank138927 +138928 POINT(42.685219975341745 -87.46358713813387) bank138928 +138929 POINT(41.378841407973084 -87.73173501822619) bank138929 +138930 POINT(42.86791085071145 -87.89724052484519) bank138930 +138931 POINT(42.35920045497516 -86.95777691208617) bank138931 +138932 POINT(41.378521601160614 -88.23460340603397) bank138932 +138933 POINT(41.93002490556561 -88.37797526871351) bank138933 +138934 POINT(42.627682724686075 -88.35112959303918) bank138934 +138935 POINT(41.812332686885014 -87.18363344905771) bank138935 +138936 POINT(42.73187958922587 -88.24903733908863) bank138936 +138937 POINT(42.15937631858058 -88.55648191759252) bank138937 +138938 POINT(41.152529626176545 -86.83733889497779) bank138938 +138939 POINT(41.50603185322404 -88.32467287446136) bank138939 +138940 POINT(42.84823298738725 -87.99042504949587) bank138940 +138941 POINT(41.16549993090436 -87.49387936423129) bank138941 +138942 POINT(42.265419086530166 -88.34554393464165) bank138942 +138943 POINT(42.47637686324623 -86.69361841256087) bank138943 +138944 POINT(42.30557056381601 -88.6283370564342) bank138944 +138945 POINT(41.433731293140696 -87.31317403503775) bank138945 +138946 POINT(41.929972465509366 -86.92840526161031) bank138946 +138947 POINT(42.11886641462764 -86.65587494284202) bank138947 +138948 POINT(41.71270582585335 -88.01927432765704) bank138948 +138949 POINT(41.93488992416152 -86.80007875476583) bank138949 +138950 POINT(42.07382340900531 -87.9759337776812) bank138950 +138951 POINT(41.11990446987634 -87.73880128769895) bank138951 +138952 POINT(41.063418377163366 -88.5877270686626) bank138952 +138953 POINT(42.46584717538404 -87.39451146010867) bank138953 +138954 POINT(41.99860125796449 -87.23854360855907) bank138954 +138955 POINT(41.02169205020651 -86.7718994723657) bank138955 +138956 POINT(42.80680045036844 -87.0113474543568) bank138956 +138957 POINT(41.880971742489706 -87.53463620615432) bank138957 +138958 POINT(41.73190116497499 -87.36687066133703) bank138958 +138959 POINT(42.676498390327886 -86.73023830077611) bank138959 +138960 POINT(42.83702501449661 -87.72815514512736) bank138960 +138961 POINT(41.03019735207787 -87.60680980892643) bank138961 +138962 POINT(42.12648188892282 -87.36340466177163) bank138962 +138963 POINT(41.31396510868759 -87.7149599579354) bank138963 +138964 POINT(42.097594275339056 -87.96685727505542) bank138964 +138965 POINT(42.02791837782046 -87.3471578312921) bank138965 +138966 POINT(42.07063666723996 -88.44458851626811) bank138966 +138967 POINT(41.24164142895537 -88.16778864693438) bank138967 +138968 POINT(41.69532828907001 -87.76940432676005) bank138968 +138969 POINT(41.81739222932522 -87.21665695013586) bank138969 +138970 POINT(42.1890473800933 -88.59486913975392) bank138970 +138971 POINT(42.736782908691026 -88.14781709803675) bank138971 +138972 POINT(41.7746559374532 -87.6691255988235) bank138972 +138973 POINT(41.36645312347387 -88.00030684315597) bank138973 +138974 POINT(42.136530322324646 -87.8698118530825) bank138974 +138975 POINT(41.23502682195494 -87.30204738206834) bank138975 +138976 POINT(42.48757196813997 -86.9606201377894) bank138976 +138977 POINT(41.48979707300932 -88.46436737626131) bank138977 +138978 POINT(41.26006641295495 -87.55398340298892) bank138978 +138979 POINT(41.338847092643434 -88.56141094483131) bank138979 +138980 POINT(41.663482983676474 -88.49697974643492) bank138980 +138981 POINT(42.79180536128776 -86.98103134580161) bank138981 +138982 POINT(41.4566518129848 -88.25315648856645) bank138982 +138983 POINT(42.85808525035592 -88.30612305756523) bank138983 +138984 POINT(42.28746408523539 -87.5729226172298) bank138984 +138985 POINT(41.15218619956607 -86.88074954885053) bank138985 +138986 POINT(41.30717727891249 -88.24372841332905) bank138986 +138987 POINT(42.38602120428864 -88.40913615096437) bank138987 +138988 POINT(42.066167768733806 -87.10488980088945) bank138988 +138989 POINT(42.357925931297096 -87.01460734748852) bank138989 +138990 POINT(41.0054520261459 -88.15764200859769) bank138990 +138991 POINT(42.87153417279497 -86.66617135347741) bank138991 +138992 POINT(42.69529302354099 -87.54561995756998) bank138992 +138993 POINT(41.8638923649928 -86.9783633702789) bank138993 +138994 POINT(41.27265082344122 -87.47912869360395) bank138994 +138995 POINT(41.39760690043349 -87.4627603767169) bank138995 +138996 POINT(42.77962103008956 -87.26307785465453) bank138996 +138997 POINT(42.37597393490906 -86.67211871480765) bank138997 +138998 POINT(41.274817920981384 -87.5756428506045) bank138998 +138999 POINT(41.8891933402046 -88.27590162059482) bank138999 +139000 POINT(41.27219987181328 -88.10211328610414) bank139000 +139001 POINT(41.469178272761596 -87.8646541052091) bank139001 +139002 POINT(41.32930156162264 -88.2792653191095) bank139002 +139003 POINT(41.80854286692019 -87.2977511605673) bank139003 +139004 POINT(41.501346421392306 -87.76432180721135) bank139004 +139005 POINT(40.94947887707317 -87.3127499382571) bank139005 +139006 POINT(42.71440606465366 -88.02534095407306) bank139006 +139007 POINT(41.199679169523584 -88.49714018160319) bank139007 +139008 POINT(42.03082968134073 -86.95301327331826) bank139008 +139009 POINT(40.99064315711373 -86.96156935316426) bank139009 +139010 POINT(42.53099835399734 -87.06926851189499) bank139010 +139011 POINT(42.443240330248905 -87.19667817522564) bank139011 +139012 POINT(42.32488528467127 -87.06830005266157) bank139012 +139013 POINT(41.43999210255269 -86.98778664017344) bank139013 +139014 POINT(41.53858016376673 -86.82249649517078) bank139014 +139015 POINT(42.68987977325015 -87.3638105236932) bank139015 +139016 POINT(42.854745989499044 -88.5832486776264) bank139016 +139017 POINT(42.35256519251203 -88.03265961693712) bank139017 +139018 POINT(41.276485329421334 -88.4072520122897) bank139018 +139019 POINT(40.98427920905941 -87.33998134007025) bank139019 +139020 POINT(41.64137429378245 -88.17291552072169) bank139020 +139021 POINT(42.18364099165643 -87.46907399243239) bank139021 +139022 POINT(41.8481663575952 -86.70780906572132) bank139022 +139023 POINT(42.09939089563584 -86.79006769663302) bank139023 +139024 POINT(41.89332593451705 -88.11517819143619) bank139024 +139025 POINT(42.3810726494119 -88.18919805447602) bank139025 +139026 POINT(42.16961053267251 -86.78045508752999) bank139026 +139027 POINT(40.98973369495829 -87.73434663302359) bank139027 +139028 POINT(41.43508073641264 -87.34412525263146) bank139028 +139029 POINT(42.163061512091865 -86.66595823029769) bank139029 +139030 POINT(42.514817360747045 -87.89376132490142) bank139030 +139031 POINT(42.08727735156639 -87.64940281279829) bank139031 +139032 POINT(41.12485986353431 -86.74063372758013) bank139032 +139033 POINT(42.233530033025474 -87.38825289396905) bank139033 +139034 POINT(41.8727175112039 -87.21092601719374) bank139034 +139035 POINT(41.51305908681499 -86.63889215847438) bank139035 +139036 POINT(41.82466968435912 -87.15825311691678) bank139036 +139037 POINT(41.87020190797156 -87.1818394865213) bank139037 +139038 POINT(42.37676504814769 -87.0967852653691) bank139038 +139039 POINT(41.48487478493618 -87.75879863420892) bank139039 +139040 POINT(42.67915869354939 -87.8733366946286) bank139040 +139041 POINT(41.13845400964759 -87.52836084241419) bank139041 +139042 POINT(41.58570834297198 -87.92990865195137) bank139042 +139043 POINT(42.1676277205976 -86.70609726513169) bank139043 +139044 POINT(41.30089538926438 -86.84217791524284) bank139044 +139045 POINT(41.08348618504746 -87.28173446970841) bank139045 +139046 POINT(42.26624625463041 -87.30482186642773) bank139046 +139047 POINT(41.07131860031885 -88.19149673332346) bank139047 +139048 POINT(40.97184108895034 -87.83626814105843) bank139048 +139049 POINT(42.75297502224281 -87.78466688420212) bank139049 +139050 POINT(42.24915586267128 -88.4468344964037) bank139050 +139051 POINT(42.852296630060415 -88.3787128535509) bank139051 +139052 POINT(41.080887486531395 -86.82000357043648) bank139052 +139053 POINT(42.55841020267164 -88.4360525290231) bank139053 +139054 POINT(41.84854881312665 -87.37070251038533) bank139054 +139055 POINT(41.62190206858015 -87.32793702981263) bank139055 +139056 POINT(41.20674058696517 -86.78484329953591) bank139056 +139057 POINT(41.4157096191097 -87.98589480561897) bank139057 +139058 POINT(42.66058349817339 -88.39089366151545) bank139058 +139059 POINT(42.1782133044909 -87.65543933114574) bank139059 +139060 POINT(41.117302493251685 -87.78366012044593) bank139060 +139061 POINT(41.09900090398635 -87.13656552763116) bank139061 +139062 POINT(42.34172990405232 -87.3320329946908) bank139062 +139063 POINT(41.68511846368123 -87.69867751225271) bank139063 +139064 POINT(41.647663875255866 -87.97493305378418) bank139064 +139065 POINT(42.21506431830156 -87.4902053625311) bank139065 +139066 POINT(41.68342714270638 -88.08926176948945) bank139066 +139067 POINT(42.05905914973333 -87.87906900796764) bank139067 +139068 POINT(41.732082689550886 -87.96959218508458) bank139068 +139069 POINT(42.84445623377782 -88.33341789448991) bank139069 +139070 POINT(42.81703163020665 -87.5367184346373) bank139070 +139071 POINT(41.79462760945873 -87.37220097980037) bank139071 +139072 POINT(41.815738234282506 -88.48266772860607) bank139072 +139073 POINT(41.08868823390479 -87.0154865775125) bank139073 +139074 POINT(41.754110314562496 -87.2684583722381) bank139074 +139075 POINT(42.756825730652004 -87.37727276589665) bank139075 +139076 POINT(42.14217641589055 -87.26717670370947) bank139076 +139077 POINT(42.494022818818095 -87.62075189670362) bank139077 +139078 POINT(42.01318804907839 -87.18931744276404) bank139078 +139079 POINT(41.875700539404384 -88.03334381783107) bank139079 +139080 POINT(41.96236096901003 -88.46623402791866) bank139080 +139081 POINT(42.39805895763065 -86.6980410534216) bank139081 +139082 POINT(42.70075847219576 -86.8792353041443) bank139082 +139083 POINT(42.59629264470295 -88.49271740513505) bank139083 +139084 POINT(42.82244469922902 -87.90822534919879) bank139084 +139085 POINT(42.31452008787168 -87.27697669752881) bank139085 +139086 POINT(41.184699983554154 -87.48108077861379) bank139086 +139087 POINT(41.70838382168836 -88.39948272415509) bank139087 +139088 POINT(41.360731816871116 -88.37593760319963) bank139088 +139089 POINT(42.352499612552364 -88.04480983797563) bank139089 +139090 POINT(42.60010253727705 -86.84474358555488) bank139090 +139091 POINT(41.761557388969806 -87.10863714260812) bank139091 +139092 POINT(42.02549235281138 -86.75971981297187) bank139092 +139093 POINT(42.36810274872161 -86.69655448089173) bank139093 +139094 POINT(41.2352600035094 -88.418253318269) bank139094 +139095 POINT(42.00464914252699 -86.64944143100705) bank139095 +139096 POINT(41.53259429904314 -87.94926913023237) bank139096 +139097 POINT(42.484630257381106 -86.9632571427638) bank139097 +139098 POINT(41.93610594789849 -87.14156104326388) bank139098 +139099 POINT(42.366216351723935 -88.61696682507034) bank139099 +139100 POINT(41.01070742407146 -87.84820840306946) bank139100 +139101 POINT(42.617392548157454 -88.51609655460501) bank139101 +139102 POINT(42.21578720462775 -88.34597397569965) bank139102 +139103 POINT(41.65533307218254 -88.02447999151389) bank139103 +139104 POINT(41.53922678945843 -88.53837874208484) bank139104 +139105 POINT(42.360850898707014 -88.61830867380321) bank139105 +139106 POINT(42.64888722553575 -88.52055243446789) bank139106 +139107 POINT(41.40685679869034 -88.2186725211303) bank139107 +139108 POINT(41.86442800619273 -87.94230173462431) bank139108 +139109 POINT(41.51282364653177 -88.53855606318403) bank139109 +139110 POINT(40.94865211627953 -87.79393839410865) bank139110 +139111 POINT(41.20818678936725 -88.58369641806598) bank139111 +139112 POINT(41.2398085839435 -87.04838218152976) bank139112 +139113 POINT(41.407321548626456 -88.42654562518244) bank139113 +139114 POINT(41.6544736908714 -87.39767138078739) bank139114 +139115 POINT(42.18859799281822 -87.16467514743056) bank139115 +139116 POINT(41.42525092731731 -88.00798407101796) bank139116 +139117 POINT(42.76743225680203 -88.28617288410011) bank139117 +139118 POINT(42.76851853716759 -86.74220019102118) bank139118 +139119 POINT(41.04409908996592 -87.25180695573972) bank139119 +139120 POINT(42.41398534276018 -86.68115275069597) bank139120 +139121 POINT(41.162319296499895 -87.49076662621844) bank139121 +139122 POINT(42.23396079140328 -86.8506135155279) bank139122 +139123 POINT(41.02260032702549 -86.87605285699209) bank139123 +139124 POINT(41.21801380020803 -87.16003642801013) bank139124 +139125 POINT(42.23145562367154 -88.61741281015912) bank139125 +139126 POINT(41.81450386673853 -87.41816722159399) bank139126 +139127 POINT(42.29342642104447 -87.40034922030351) bank139127 +139128 POINT(41.101600186304964 -86.95285116373344) bank139128 +139129 POINT(41.77430556143932 -86.7451759832899) bank139129 +139130 POINT(42.65343663296603 -87.25406366299997) bank139130 +139131 POINT(42.62446923916736 -86.68307049942445) bank139131 +139132 POINT(41.78858268322588 -87.72477034555826) bank139132 +139133 POINT(42.82530687001423 -88.01565166294947) bank139133 +139134 POINT(42.10399071997818 -88.35651895771026) bank139134 +139135 POINT(41.055502055376415 -88.6214533239427) bank139135 +139136 POINT(41.479710847009876 -88.45938235363418) bank139136 +139137 POINT(42.81129023060251 -88.32441402101752) bank139137 +139138 POINT(42.07716317257875 -87.53199786255819) bank139138 +139139 POINT(42.70250739147372 -88.30676059315489) bank139139 +139140 POINT(42.645555855117934 -86.96495526590563) bank139140 +139141 POINT(42.236223717510754 -86.68628296461513) bank139141 +139142 POINT(42.01094001334695 -87.64299621831344) bank139142 +139143 POINT(42.379923449849144 -87.97741205026786) bank139143 +139144 POINT(42.14854378175599 -88.42907622179193) bank139144 +139145 POINT(41.60091259587217 -87.4371959706652) bank139145 +139146 POINT(42.12591391975727 -88.50929634673687) bank139146 +139147 POINT(41.505782659138 -88.58509281492994) bank139147 +139148 POINT(42.165603916489474 -87.18018517426853) bank139148 +139149 POINT(42.018176000427815 -87.64907421200301) bank139149 +139150 POINT(42.50563378458752 -87.09308836588727) bank139150 +139151 POINT(42.23302008407231 -87.01587141044548) bank139151 +139152 POINT(42.66793001257205 -87.13459287657355) bank139152 +139153 POINT(42.435411158570254 -87.40681131650011) bank139153 +139154 POINT(40.92541764476373 -88.20670831459941) bank139154 +139155 POINT(41.79439146874782 -88.11273656702684) bank139155 +139156 POINT(42.381318742885625 -88.32120199259074) bank139156 +139157 POINT(42.02491551278459 -87.83197621524926) bank139157 +139158 POINT(41.20916900345425 -86.8128840376373) bank139158 +139159 POINT(42.12552944014101 -87.08333922732766) bank139159 +139160 POINT(40.89823989065517 -87.29819999088579) bank139160 +139161 POINT(41.64826165464624 -87.4364278757839) bank139161 +139162 POINT(42.32440214599921 -87.95483085642287) bank139162 +139163 POINT(41.91837277720049 -87.75405201389732) bank139163 +139164 POINT(42.18664367354003 -88.32215565124808) bank139164 +139165 POINT(41.50653330565308 -86.85872210868047) bank139165 +139166 POINT(42.78810403265846 -86.83731445929948) bank139166 +139167 POINT(41.374948798785766 -87.07272682319208) bank139167 +139168 POINT(41.78417921143382 -88.57651184138003) bank139168 +139169 POINT(42.42089265046791 -86.7521096014855) bank139169 +139170 POINT(41.66656840284375 -86.9488387283001) bank139170 +139171 POINT(42.65464905768535 -87.18573263350133) bank139171 +139172 POINT(41.54357778246242 -86.93348164133752) bank139172 +139173 POINT(41.308332302010044 -88.24911965179618) bank139173 +139174 POINT(41.51361609639719 -88.5671847275855) bank139174 +139175 POINT(42.4167575492727 -87.92392171502445) bank139175 +139176 POINT(42.38016961151602 -87.5666524319877) bank139176 +139177 POINT(41.681589711258596 -87.50107089151848) bank139177 +139178 POINT(41.49081626608287 -88.04196674979387) bank139178 +139179 POINT(42.32622821012895 -87.75594233307487) bank139179 +139180 POINT(42.07514626142002 -87.57694495183551) bank139180 +139181 POINT(41.82563296459334 -88.00293387476226) bank139181 +139182 POINT(42.750018834303056 -87.73918196272672) bank139182 +139183 POINT(41.3321633025682 -88.08803281163898) bank139183 +139184 POINT(42.2715026559234 -88.41877482864845) bank139184 +139185 POINT(41.76288020566022 -86.85109904262423) bank139185 +139186 POINT(41.74175251350804 -87.1627018893929) bank139186 +139187 POINT(41.91829600973082 -86.67258494819549) bank139187 +139188 POINT(42.856885027534055 -86.86500221914754) bank139188 +139189 POINT(41.174666238088044 -88.53740974464836) bank139189 +139190 POINT(41.46283408643471 -86.7552587450151) bank139190 +139191 POINT(41.02517765959362 -86.72049817402873) bank139191 +139192 POINT(42.54076402712776 -88.00453168431795) bank139192 +139193 POINT(42.1525025117308 -88.52859222682726) bank139193 +139194 POINT(41.16034263623129 -87.9413309182802) bank139194 +139195 POINT(42.570572053209126 -86.70795547968844) bank139195 +139196 POINT(42.86779520094904 -88.59640393919436) bank139196 +139197 POINT(41.90053695227837 -88.55507896660046) bank139197 +139198 POINT(41.721029256881714 -87.45062609837036) bank139198 +139199 POINT(41.01020178347523 -87.28382513351312) bank139199 +139200 POINT(42.069752496556454 -88.05407902157282) bank139200 +139201 POINT(42.36288438236023 -86.69907434057649) bank139201 +139202 POINT(42.15367885924824 -88.26566240902204) bank139202 +139203 POINT(41.75318673875266 -88.47259057823359) bank139203 +139204 POINT(40.88000159371987 -87.55719677206241) bank139204 +139205 POINT(41.22950960236118 -87.49027698126619) bank139205 +139206 POINT(41.597767941020805 -86.81593864001374) bank139206 +139207 POINT(42.65467111176139 -88.57361449064138) bank139207 +139208 POINT(41.66499490338 -88.34801204683288) bank139208 +139209 POINT(42.64173439282359 -86.82808753892068) bank139209 +139210 POINT(40.96337658341944 -87.75608172018367) bank139210 +139211 POINT(42.21475036069943 -87.00668239736464) bank139211 +139212 POINT(41.61050622791909 -87.57021919344365) bank139212 +139213 POINT(41.23380706584339 -87.41820459886056) bank139213 +139214 POINT(41.421689449915775 -87.63817626258425) bank139214 +139215 POINT(42.48979520275702 -86.67697251041776) bank139215 +139216 POINT(41.80448440155032 -88.29827151520028) bank139216 +139217 POINT(42.14214095405985 -87.46221002231516) bank139217 +139218 POINT(40.88119511154832 -87.11096055943808) bank139218 +139219 POINT(41.31353612763075 -86.74938232995069) bank139219 +139220 POINT(41.77531270081642 -87.20110866664575) bank139220 +139221 POINT(42.7129260395826 -88.01645197132815) bank139221 +139222 POINT(41.785665103039264 -88.52471784322654) bank139222 +139223 POINT(42.744636420514865 -88.4285495713191) bank139223 +139224 POINT(41.291501951097196 -87.11051995492255) bank139224 +139225 POINT(41.29410135089741 -87.10112114281041) bank139225 +139226 POINT(42.33966567088514 -87.46448707043369) bank139226 +139227 POINT(42.22507868307484 -88.20157529230971) bank139227 +139228 POINT(41.70025881933975 -86.71410132429453) bank139228 +139229 POINT(42.49775369997408 -86.65642574768806) bank139229 +139230 POINT(42.642528231436145 -86.79173326207463) bank139230 +139231 POINT(42.464422040728245 -87.04416361360668) bank139231 +139232 POINT(41.38667840951077 -87.54741900390424) bank139232 +139233 POINT(42.230645942538374 -88.36339736675609) bank139233 +139234 POINT(42.1869713301441 -87.62692760937496) bank139234 +139235 POINT(41.95671233819778 -88.52536492035286) bank139235 +139236 POINT(41.60920865431297 -87.37322171312339) bank139236 +139237 POINT(42.859488838138724 -87.3697447164619) bank139237 +139238 POINT(41.85687796439641 -86.82620421672169) bank139238 +139239 POINT(41.61151889423839 -87.8436091942224) bank139239 +139240 POINT(42.2188681512448 -88.06635902425722) bank139240 +139241 POINT(41.69107714948988 -88.57724125690501) bank139241 +139242 POINT(42.07804891978209 -88.4159974854846) bank139242 +139243 POINT(42.06266745374522 -87.00020510942583) bank139243 +139244 POINT(42.10929598271222 -87.26448771878778) bank139244 +139245 POINT(41.01815763455336 -87.16043171761511) bank139245 +139246 POINT(41.49989767527836 -87.06968087003976) bank139246 +139247 POINT(41.30633350185025 -87.9676601385415) bank139247 +139248 POINT(42.87610317760294 -86.77675507365961) bank139248 +139249 POINT(42.851913844194634 -88.28622130249148) bank139249 +139250 POINT(41.5590959485207 -88.26826073202045) bank139250 +139251 POINT(41.70773412248477 -87.35764582298599) bank139251 +139252 POINT(41.8695851300181 -87.47800207214547) bank139252 +139253 POINT(41.686777053384716 -86.97013824552135) bank139253 +139254 POINT(40.931131439365636 -86.90736850852736) bank139254 +139255 POINT(40.97642723532266 -86.63649436645936) bank139255 +139256 POINT(41.04073535267952 -86.76414035808459) bank139256 +139257 POINT(42.854040400432346 -87.3613334209642) bank139257 +139258 POINT(41.79419975551638 -87.19455880371241) bank139258 +139259 POINT(41.041502773975395 -87.41094279722422) bank139259 +139260 POINT(41.40623298092564 -88.49450434096134) bank139260 +139261 POINT(41.53476647519529 -86.96924223755794) bank139261 +139262 POINT(42.81448754819659 -88.3210492513042) bank139262 +139263 POINT(42.59292093136305 -87.13132906671147) bank139263 +139264 POINT(42.60487613501037 -88.31756822126665) bank139264 +139265 POINT(41.81010537821144 -88.33713768479183) bank139265 +139266 POINT(42.27889818642314 -87.05513488422956) bank139266 +139267 POINT(40.99128205623889 -86.6943966038763) bank139267 +139268 POINT(41.585073530146516 -87.40340027488874) bank139268 +139269 POINT(42.7352572939461 -86.73840747394397) bank139269 +139270 POINT(42.37753226441889 -86.80128626557597) bank139270 +139271 POINT(40.90871473354207 -87.50185242081358) bank139271 +139272 POINT(41.88856500462259 -87.10775401705901) bank139272 +139273 POINT(42.04209884288377 -87.46709166573099) bank139273 +139274 POINT(42.479101212819465 -87.48135228790888) bank139274 +139275 POINT(42.01979828463449 -87.6919312506962) bank139275 +139276 POINT(42.86394948027505 -88.5117960958704) bank139276 +139277 POINT(42.18995414719548 -86.65697244765988) bank139277 +139278 POINT(42.3692442071967 -87.28844977690868) bank139278 +139279 POINT(41.34351602368973 -88.542234600501) bank139279 +139280 POINT(42.185912153349314 -87.19453742537736) bank139280 +139281 POINT(41.6000461684311 -86.68459843914141) bank139281 +139282 POINT(42.65043662051098 -87.02084697405185) bank139282 +139283 POINT(41.048264615487255 -87.28291592249504) bank139283 +139284 POINT(41.1993900717161 -88.02613731123527) bank139284 +139285 POINT(42.46464771147056 -87.3140102256381) bank139285 +139286 POINT(42.59582629724856 -87.11182514454349) bank139286 +139287 POINT(42.48824278457974 -87.96388276856305) bank139287 +139288 POINT(42.712521119132695 -87.75667471722318) bank139288 +139289 POINT(41.01425029240085 -87.57444517893875) bank139289 +139290 POINT(42.83493947473562 -87.89876158858067) bank139290 +139291 POINT(41.479828149724945 -87.46339207869197) bank139291 +139292 POINT(41.86626983645349 -86.96226479998464) bank139292 +139293 POINT(40.887168250223844 -86.69394224782737) bank139293 +139294 POINT(41.49086706119449 -87.42979094384978) bank139294 +139295 POINT(42.36698498028571 -87.72850180283082) bank139295 +139296 POINT(41.48748252301481 -88.59402924876329) bank139296 +139297 POINT(41.73971400433221 -88.49600399953516) bank139297 +139298 POINT(40.91149848899404 -88.02673410543514) bank139298 +139299 POINT(42.757365523255814 -87.45121931937368) bank139299 +139300 POINT(42.6522500194881 -87.92528222723537) bank139300 +139301 POINT(42.81427674803757 -87.25390369105273) bank139301 +139302 POINT(42.06659265726994 -87.9635464505201) bank139302 +139303 POINT(41.36754497958759 -86.79282429583488) bank139303 +139304 POINT(42.28346968379919 -87.44582866279796) bank139304 +139305 POINT(42.73152710317624 -88.51564936646103) bank139305 +139306 POINT(41.6490607513384 -88.23164525101247) bank139306 +139307 POINT(41.33084720032077 -88.24243131681145) bank139307 +139308 POINT(41.92073549970193 -86.94877779221062) bank139308 +139309 POINT(41.431991107073955 -87.18747881062406) bank139309 +139310 POINT(42.63233801891894 -86.89703871737545) bank139310 +139311 POINT(42.432769237508744 -88.5932790797965) bank139311 +139312 POINT(41.277942719488244 -86.6838128093539) bank139312 +139313 POINT(42.43202946245344 -87.74597747758291) bank139313 +139314 POINT(41.272399076201836 -87.73034072683046) bank139314 +139315 POINT(42.23403863347185 -87.82106282505268) bank139315 +139316 POINT(42.20633197367218 -87.45258205212183) bank139316 +139317 POINT(42.83876421660691 -87.97241747560827) bank139317 +139318 POINT(42.774423792167696 -86.96684613579406) bank139318 +139319 POINT(41.006989542996024 -88.36771778062574) bank139319 +139320 POINT(42.56340864101459 -86.86472896873255) bank139320 +139321 POINT(42.7201524112227 -87.96396781676656) bank139321 +139322 POINT(41.27019626574046 -87.87341184759617) bank139322 +139323 POINT(40.90761892716376 -87.99866311639776) bank139323 +139324 POINT(41.5009520566027 -87.72995984032919) bank139324 +139325 POINT(41.522399341695746 -87.95077412835182) bank139325 +139326 POINT(41.76472812136042 -87.79795130557329) bank139326 +139327 POINT(40.88377839634815 -86.87348824964526) bank139327 +139328 POINT(41.43625474105036 -87.24092805138366) bank139328 +139329 POINT(41.588920469049086 -87.05774802489923) bank139329 +139330 POINT(42.24965567713708 -87.11123416676888) bank139330 +139331 POINT(41.961837849477604 -88.2316514841772) bank139331 +139332 POINT(41.85568417187211 -86.72898806717579) bank139332 +139333 POINT(42.06124029533783 -87.08250053184939) bank139333 +139334 POINT(41.53838691220294 -88.17102510362879) bank139334 +139335 POINT(41.938707952872036 -86.87585315754778) bank139335 +139336 POINT(42.75094476627632 -88.55636574748614) bank139336 +139337 POINT(42.13402806832876 -87.6270993352833) bank139337 +139338 POINT(42.40643157746339 -86.7298067142691) bank139338 +139339 POINT(41.76188145835326 -86.74401613506454) bank139339 +139340 POINT(41.382774483520045 -87.25249862965127) bank139340 +139341 POINT(41.18995380242399 -88.15134818242927) bank139341 +139342 POINT(41.00691474568786 -87.96757047141247) bank139342 +139343 POINT(41.967276380691345 -87.6204008030722) bank139343 +139344 POINT(42.60227660478626 -87.12989989197557) bank139344 +139345 POINT(41.76717971928087 -86.84326809963316) bank139345 +139346 POINT(41.55400972708961 -87.937391504572) bank139346 +139347 POINT(41.99012252580848 -87.6861888965376) bank139347 +139348 POINT(41.577053094951026 -88.31539878100017) bank139348 +139349 POINT(41.2792945901463 -88.0291023420694) bank139349 +139350 POINT(41.21815894025648 -87.43474338798045) bank139350 +139351 POINT(42.81167448459781 -87.68947481523563) bank139351 +139352 POINT(41.04731731245812 -88.58095799352067) bank139352 +139353 POINT(41.561848554601745 -87.36376420775646) bank139353 +139354 POINT(40.98144028004253 -87.17975510236606) bank139354 +139355 POINT(42.74988778730099 -88.10385896295766) bank139355 +139356 POINT(41.253804631075624 -87.72200735564837) bank139356 +139357 POINT(41.51226062254081 -88.4383531748936) bank139357 +139358 POINT(42.42693543923591 -87.25180887094248) bank139358 +139359 POINT(41.797975746265195 -87.41695417724301) bank139359 +139360 POINT(41.4144874992533 -88.43854134788053) bank139360 +139361 POINT(42.37057242982577 -88.2001275518619) bank139361 +139362 POINT(41.11710232165906 -88.56976282728715) bank139362 +139363 POINT(42.26463435095617 -87.28138986576778) bank139363 +139364 POINT(41.454064303600816 -88.43169848928942) bank139364 +139365 POINT(41.255785005404334 -88.03809133512597) bank139365 +139366 POINT(40.991540269755454 -87.14876202129956) bank139366 +139367 POINT(41.504061201528124 -87.90102957695692) bank139367 +139368 POINT(41.78495787940152 -87.02276387497639) bank139368 +139369 POINT(41.24374063294669 -88.06466063561462) bank139369 +139370 POINT(41.41679790125205 -87.95005352381347) bank139370 +139371 POINT(41.53364926145949 -87.36567353119078) bank139371 +139372 POINT(42.03254666109336 -87.38101238349074) bank139372 +139373 POINT(42.35323922296698 -87.95451870248446) bank139373 +139374 POINT(41.4981349817426 -87.91967985324398) bank139374 +139375 POINT(41.52545425563806 -87.51956147695282) bank139375 +139376 POINT(41.25440611199083 -87.71293558033321) bank139376 +139377 POINT(42.80969153344696 -88.2795856312969) bank139377 +139378 POINT(41.01438124264019 -88.00485766625236) bank139378 +139379 POINT(41.03840376297664 -86.76998098842259) bank139379 +139380 POINT(40.91887066475996 -87.57831856632116) bank139380 +139381 POINT(42.02981461184258 -86.67258332455296) bank139381 +139382 POINT(41.65455875584063 -88.23850877368521) bank139382 +139383 POINT(41.70878938610061 -87.02966065019443) bank139383 +139384 POINT(41.01363793892724 -86.68101614155752) bank139384 +139385 POINT(41.41425057054971 -86.77245717481098) bank139385 +139386 POINT(41.28075186119604 -86.63992765447719) bank139386 +139387 POINT(41.835657184399224 -88.33574402737277) bank139387 +139388 POINT(41.591181349351885 -86.82603533334553) bank139388 +139389 POINT(41.74236679789903 -86.75072748093851) bank139389 +139390 POINT(40.891466508349836 -86.97299791098789) bank139390 +139391 POINT(41.289195072974444 -86.98033915314518) bank139391 +139392 POINT(41.173480495107455 -88.04295264176685) bank139392 +139393 POINT(42.621312523749104 -86.96890298437981) bank139393 +139394 POINT(41.94117381864694 -87.57026154498372) bank139394 +139395 POINT(42.85713405822589 -87.2203459250982) bank139395 +139396 POINT(41.14806302475854 -88.4126596632746) bank139396 +139397 POINT(41.05687981276497 -87.86970673101538) bank139397 +139398 POINT(41.98760301288411 -86.68206857512037) bank139398 +139399 POINT(42.636785765776914 -86.82239468122242) bank139399 +139400 POINT(41.29985306472351 -86.6918525741834) bank139400 +139401 POINT(41.226869112495365 -87.04445079334549) bank139401 +139402 POINT(41.99985068767737 -86.70636338236748) bank139402 +139403 POINT(41.16407718595763 -88.55113344582786) bank139403 +139404 POINT(42.68726403192568 -86.87174093757322) bank139404 +139405 POINT(41.65335503471838 -86.8301788133101) bank139405 +139406 POINT(41.52228318910178 -87.65656916119566) bank139406 +139407 POINT(42.26363598065495 -87.02004774272966) bank139407 +139408 POINT(41.75165694321783 -87.47351195643009) bank139408 +139409 POINT(42.7627391400977 -88.1658301101318) bank139409 +139410 POINT(42.09860328070336 -88.26584829604634) bank139410 +139411 POINT(42.20872405937796 -87.15914879952234) bank139411 +139412 POINT(42.18741529133319 -86.91613758609972) bank139412 +139413 POINT(41.79702899130678 -87.59360417134369) bank139413 +139414 POINT(41.144959420934136 -87.22021967821668) bank139414 +139415 POINT(42.15710245826462 -87.00332114975069) bank139415 +139416 POINT(41.815042571585 -86.72449641426891) bank139416 +139417 POINT(42.29482996295533 -87.54848721532454) bank139417 +139418 POINT(41.388095665912566 -88.61104283238278) bank139418 +139419 POINT(41.83339188913231 -87.33207471723303) bank139419 +139420 POINT(42.36485270148187 -87.65452744173979) bank139420 +139421 POINT(41.68748643604589 -87.80458680836514) bank139421 +139422 POINT(42.8375444581795 -87.12798917147816) bank139422 +139423 POINT(41.22923281113093 -86.86062405729636) bank139423 +139424 POINT(41.59043507565704 -87.10388126829326) bank139424 +139425 POINT(41.160463520062784 -88.60063896604838) bank139425 +139426 POINT(40.91358967890556 -87.6902372529452) bank139426 +139427 POINT(41.68642982472512 -87.4574117739839) bank139427 +139428 POINT(42.582961600563486 -88.6117885122987) bank139428 +139429 POINT(42.67302116105715 -88.3118534331238) bank139429 +139430 POINT(42.63852050361568 -87.57093581266845) bank139430 +139431 POINT(41.16993362339668 -87.28697096472445) bank139431 +139432 POINT(41.12829390977166 -86.63512281653207) bank139432 +139433 POINT(42.223358675113055 -86.91233915285926) bank139433 +139434 POINT(42.375440215528535 -88.51844723468787) bank139434 +139435 POINT(42.43212367161536 -88.36179688001428) bank139435 +139436 POINT(42.39025098161611 -88.31100163634689) bank139436 +139437 POINT(42.85636601376724 -87.65695921900812) bank139437 +139438 POINT(41.00711622205266 -86.85692416567774) bank139438 +139439 POINT(41.806868515495474 -87.76916658992224) bank139439 +139440 POINT(41.7313120913847 -87.08085625532108) bank139440 +139441 POINT(41.84400972985915 -87.83638622486927) bank139441 +139442 POINT(42.30261662920399 -87.81243512393809) bank139442 +139443 POINT(41.835501029501685 -88.06789022580156) bank139443 +139444 POINT(40.91181935801484 -87.43233753498895) bank139444 +139445 POINT(42.314378477705596 -86.9620704733248) bank139445 +139446 POINT(41.635470588562626 -88.0801627620085) bank139446 +139447 POINT(41.60837567706071 -87.3909464509535) bank139447 +139448 POINT(42.682071919864896 -87.52777229137914) bank139448 +139449 POINT(41.97225856198142 -87.66628599942646) bank139449 +139450 POINT(41.11177232886224 -87.96424416103756) bank139450 +139451 POINT(42.20406248648102 -87.91871402084813) bank139451 +139452 POINT(41.85676040453854 -87.22048246705106) bank139452 +139453 POINT(41.38659833812693 -88.1557444472088) bank139453 +139454 POINT(41.299727989507424 -88.34710167666022) bank139454 +139455 POINT(41.81904508527802 -87.64188003311071) bank139455 +139456 POINT(41.29612140353262 -88.62819766504113) bank139456 +139457 POINT(41.966028151970384 -87.8128633772707) bank139457 +139458 POINT(42.40973428916388 -88.1878399322384) bank139458 +139459 POINT(41.43310885178382 -88.37037623889282) bank139459 +139460 POINT(42.66253166935048 -87.38566522179792) bank139460 +139461 POINT(41.15378681265992 -88.50446392862428) bank139461 +139462 POINT(42.05717052789692 -88.21463767568495) bank139462 +139463 POINT(42.49865267064982 -87.38204587021669) bank139463 +139464 POINT(41.051823497679 -87.56219738191605) bank139464 +139465 POINT(40.89570089791727 -87.1960784949082) bank139465 +139466 POINT(41.815662033841875 -87.55146232208465) bank139466 +139467 POINT(42.84465810333421 -86.68371907455932) bank139467 +139468 POINT(41.34033241022951 -86.7328877683923) bank139468 +139469 POINT(41.31372337402139 -87.33677863742783) bank139469 +139470 POINT(42.50533136437395 -88.60625525984668) bank139470 +139471 POINT(42.72526524655084 -88.28914873380178) bank139471 +139472 POINT(42.77632046621335 -86.89657151345745) bank139472 +139473 POINT(42.80215965847958 -88.20120872971309) bank139473 +139474 POINT(41.37834049229368 -88.05262778421452) bank139474 +139475 POINT(42.787396946415676 -88.18608312573411) bank139475 +139476 POINT(42.561571794307476 -87.30937739666288) bank139476 +139477 POINT(42.095172322884785 -87.63419712933046) bank139477 +139478 POINT(42.3075679079727 -87.10310493029324) bank139478 +139479 POINT(42.8231372491193 -86.76981377285738) bank139479 +139480 POINT(41.78544958770206 -88.56692944865681) bank139480 +139481 POINT(42.847655178696584 -88.546018031433) bank139481 +139482 POINT(42.17305706191105 -88.34037414285437) bank139482 +139483 POINT(42.51933736106259 -86.8724183109172) bank139483 +139484 POINT(42.22937682368482 -86.72180787837152) bank139484 +139485 POINT(42.614875864357046 -88.564516618523) bank139485 +139486 POINT(40.92332843316769 -88.32970974459981) bank139486 +139487 POINT(41.68213444008558 -87.72518361576643) bank139487 +139488 POINT(41.114676207232726 -87.32269035739955) bank139488 +139489 POINT(42.63904989960252 -87.38319823881406) bank139489 +139490 POINT(41.081419493525814 -87.29431420754439) bank139490 +139491 POINT(41.963116168095375 -87.35573935123816) bank139491 +139492 POINT(41.171286775681416 -88.09210672109236) bank139492 +139493 POINT(41.3276524214198 -87.83451305532236) bank139493 +139494 POINT(42.19819271287153 -86.94745386785775) bank139494 +139495 POINT(41.806645421832606 -88.33297812732593) bank139495 +139496 POINT(41.40899630051163 -87.89321020207505) bank139496 +139497 POINT(41.357925276941415 -87.44037212373557) bank139497 +139498 POINT(40.91082008009539 -86.90828323014043) bank139498 +139499 POINT(42.78400293416867 -88.0800892810977) bank139499 +139500 POINT(41.39923757947785 -88.25751307201296) bank139500 +139501 POINT(41.556946229022635 -86.63725499002511) bank139501 +139502 POINT(41.73360000487087 -88.01222131064995) bank139502 +139503 POINT(41.150287056647485 -88.45725682388819) bank139503 +139504 POINT(41.81969826713185 -86.73011072948627) bank139504 +139505 POINT(41.91636334374732 -87.09309921323187) bank139505 +139506 POINT(41.89010131360517 -87.62691728641956) bank139506 +139507 POINT(42.67806550190797 -87.03768673808) bank139507 +139508 POINT(41.62358591439163 -86.9299208084892) bank139508 +139509 POINT(42.78849759061221 -88.19018726714397) bank139509 +139510 POINT(42.85887922144112 -87.16426147975092) bank139510 +139511 POINT(41.33950621728566 -87.88134504906014) bank139511 +139512 POINT(42.67171228952717 -88.2055988838519) bank139512 +139513 POINT(41.25721900753807 -87.2280840114568) bank139513 +139514 POINT(42.38231114290252 -87.68590458060677) bank139514 +139515 POINT(42.59697913121407 -88.2860604363073) bank139515 +139516 POINT(41.18921196881971 -86.92000176269016) bank139516 +139517 POINT(41.82239151988353 -88.41006524986993) bank139517 +139518 POINT(41.723247255020446 -87.32679840788202) bank139518 +139519 POINT(42.15319745251838 -87.71345058815591) bank139519 +139520 POINT(41.83536752158855 -86.73101705483674) bank139520 +139521 POINT(42.53733208829497 -87.16518953476276) bank139521 +139522 POINT(42.72187728584276 -87.23590947514582) bank139522 +139523 POINT(41.100795918532285 -87.1711076491673) bank139523 +139524 POINT(40.92996350402349 -86.83901031919511) bank139524 +139525 POINT(42.731544193203845 -87.36259659932625) bank139525 +139526 POINT(41.18062163079653 -86.81859345512024) bank139526 +139527 POINT(41.867494673077516 -87.64991651846526) bank139527 +139528 POINT(42.2144884812408 -88.03433683841382) bank139528 +139529 POINT(41.02052304738788 -88.14574829755378) bank139529 +139530 POINT(42.00627986895788 -86.88321365557796) bank139530 +139531 POINT(41.60073887314532 -88.15196488052992) bank139531 +139532 POINT(42.766436633571026 -88.18962405294769) bank139532 +139533 POINT(42.397341805704386 -88.21553098098137) bank139533 +139534 POINT(42.24774690290611 -88.44682252702655) bank139534 +139535 POINT(41.70650575835452 -87.27651708544165) bank139535 +139536 POINT(41.29882299148565 -88.36327238334843) bank139536 +139537 POINT(42.098213217946224 -87.34228472857083) bank139537 +139538 POINT(42.05661797162815 -88.43821771665102) bank139538 +139539 POINT(41.93782730251202 -87.98572430496459) bank139539 +139540 POINT(41.81478335672493 -87.53688588014958) bank139540 +139541 POINT(41.03963922903295 -86.83312161799856) bank139541 +139542 POINT(41.39941858258346 -87.55827366803993) bank139542 +139543 POINT(41.23700659492244 -87.16688732928323) bank139543 +139544 POINT(42.6370284157298 -88.18578122402893) bank139544 +139545 POINT(41.43714307470342 -87.81003600472539) bank139545 +139546 POINT(42.838802546498016 -87.56467484974725) bank139546 +139547 POINT(41.01192492806457 -86.93609949287625) bank139547 +139548 POINT(41.33801510131293 -88.34388676482709) bank139548 +139549 POINT(41.58346498819318 -88.13246388567748) bank139549 +139550 POINT(42.80583710668322 -87.72061044606458) bank139550 +139551 POINT(42.51225890344989 -87.92452759719377) bank139551 +139552 POINT(42.01913561908823 -87.45793455880359) bank139552 +139553 POINT(40.94266577715853 -86.64854995163802) bank139553 +139554 POINT(42.177146638743004 -86.8804262194152) bank139554 +139555 POINT(42.132471987591074 -87.93329388628145) bank139555 +139556 POINT(41.99463836954492 -86.74714272182018) bank139556 +139557 POINT(42.69205969272512 -86.72104607186616) bank139557 +139558 POINT(41.21435579310275 -88.49880280664595) bank139558 +139559 POINT(41.42104353275281 -86.76652596479899) bank139559 +139560 POINT(41.21749122915151 -87.89517788647791) bank139560 +139561 POINT(42.475176850080324 -87.08649012369048) bank139561 +139562 POINT(41.454771742782434 -88.27828222943039) bank139562 +139563 POINT(42.848250085390276 -88.19401457112252) bank139563 +139564 POINT(42.171934376635285 -88.61646735263255) bank139564 +139565 POINT(41.161254668907915 -86.76542088044074) bank139565 +139566 POINT(41.92111217955701 -86.68123227370772) bank139566 +139567 POINT(42.70030032688163 -88.42154093132343) bank139567 +139568 POINT(41.673521319262434 -87.28949372826824) bank139568 +139569 POINT(41.85898534103492 -87.3496413131702) bank139569 +139570 POINT(41.50106761919911 -87.8923323964212) bank139570 +139571 POINT(42.593469821706876 -87.53056503510949) bank139571 +139572 POINT(42.70203841912084 -88.28827926387538) bank139572 +139573 POINT(42.799582649648194 -86.78693601636841) bank139573 +139574 POINT(41.468284584546296 -87.643472179651) bank139574 +139575 POINT(42.41100679235755 -87.25263053237698) bank139575 +139576 POINT(41.59909496683947 -87.01574830951738) bank139576 +139577 POINT(42.30461477875545 -88.03449861712032) bank139577 +139578 POINT(41.59297064339406 -87.0934556899068) bank139578 +139579 POINT(40.920039324827925 -87.47068410111164) bank139579 +139580 POINT(41.120767920111454 -88.01437159644102) bank139580 +139581 POINT(41.0078184668507 -86.86582086241016) bank139581 +139582 POINT(42.35387474285658 -87.99891718293077) bank139582 +139583 POINT(42.09773030235525 -87.8637782694168) bank139583 +139584 POINT(41.71403030046781 -87.53388078402504) bank139584 +139585 POINT(41.246015531530944 -88.61924314007967) bank139585 +139586 POINT(42.77209524705858 -88.49686598575903) bank139586 +139587 POINT(41.35145213665431 -87.53115052868242) bank139587 +139588 POINT(41.83657987346548 -87.48551591024486) bank139588 +139589 POINT(42.085061333874414 -87.2107202123347) bank139589 +139590 POINT(40.952966256537934 -87.55963547209255) bank139590 +139591 POINT(42.786220502240255 -86.87579404248882) bank139591 +139592 POINT(42.381885001226166 -88.47710684358938) bank139592 +139593 POINT(41.42175945640795 -87.8428521831008) bank139593 +139594 POINT(42.26043462240475 -88.08346080792906) bank139594 +139595 POINT(41.63953579762719 -88.50823929067595) bank139595 +139596 POINT(41.42928577106074 -88.0571515164162) bank139596 +139597 POINT(41.371094227334545 -88.3928143412684) bank139597 +139598 POINT(41.66670142492466 -88.58918595854678) bank139598 +139599 POINT(42.72964510772344 -87.20640271302) bank139599 +139600 POINT(42.51543732164318 -88.41921912755193) bank139600 +139601 POINT(42.59573272128951 -88.18160003830275) bank139601 +139602 POINT(41.22159166248728 -88.41804074705088) bank139602 +139603 POINT(42.641780729471854 -88.57074738185146) bank139603 +139604 POINT(42.67113343240832 -87.80825419239825) bank139604 +139605 POINT(42.61035093805293 -88.51389778058531) bank139605 +139606 POINT(41.10110304852772 -87.2757913087056) bank139606 +139607 POINT(41.87165333511095 -86.95273368640287) bank139607 +139608 POINT(42.57246810227162 -87.33271512063405) bank139608 +139609 POINT(42.62981656863588 -88.47357302554494) bank139609 +139610 POINT(42.487680132953216 -88.35212739276633) bank139610 +139611 POINT(41.62840459688434 -87.31986808205208) bank139611 +139612 POINT(42.15097321458266 -86.66530593409271) bank139612 +139613 POINT(40.91448155394282 -86.83722110028356) bank139613 +139614 POINT(41.49755982829668 -86.80123950597368) bank139614 +139615 POINT(42.18483627764486 -87.66245977829936) bank139615 +139616 POINT(42.05861575687641 -88.58669268910911) bank139616 +139617 POINT(40.944915740518425 -88.21844749158846) bank139617 +139618 POINT(42.60703169474239 -88.19740902696793) bank139618 +139619 POINT(41.86322207689059 -87.38796902973995) bank139619 +139620 POINT(42.32193570234464 -87.70768772071983) bank139620 +139621 POINT(41.809995816013554 -86.80991969566709) bank139621 +139622 POINT(41.31003323417805 -88.30001073420192) bank139622 +139623 POINT(41.535291236092135 -86.63702115509558) bank139623 +139624 POINT(42.63388473941494 -87.872200338707) bank139624 +139625 POINT(42.54043676643696 -87.75695266322407) bank139625 +139626 POINT(41.19664853102335 -87.5988897431338) bank139626 +139627 POINT(41.65203153625123 -87.36268462843793) bank139627 +139628 POINT(42.20320858000253 -88.52153579251853) bank139628 +139629 POINT(41.04934030245516 -87.18843170120286) bank139629 +139630 POINT(41.24598899583851 -88.04127310792126) bank139630 +139631 POINT(41.092728541802764 -87.79951462690892) bank139631 +139632 POINT(41.88375309915753 -88.47057988608809) bank139632 +139633 POINT(41.22846668899656 -88.12559169713059) bank139633 +139634 POINT(42.30963456860765 -87.23119638950362) bank139634 +139635 POINT(42.78932515284774 -87.40674399117601) bank139635 +139636 POINT(41.26921644655831 -88.07921341715347) bank139636 +139637 POINT(41.77672216351274 -86.72576689275435) bank139637 +139638 POINT(42.01524168620907 -87.43674118612635) bank139638 +139639 POINT(42.14185075978421 -88.08616778989008) bank139639 +139640 POINT(41.676000009124834 -88.62794209499249) bank139640 +139641 POINT(42.630816571128875 -88.13971960464441) bank139641 +139642 POINT(41.27978488901083 -88.1831641414581) bank139642 +139643 POINT(41.71536698943342 -88.17175377217457) bank139643 +139644 POINT(41.532243644651935 -88.4906569317442) bank139644 +139645 POINT(42.05453386973094 -86.83624254035485) bank139645 +139646 POINT(42.168668560613824 -87.60170831246836) bank139646 +139647 POINT(41.98960072505106 -87.23175071830788) bank139647 +139648 POINT(42.45417882504484 -87.87741508291342) bank139648 +139649 POINT(40.96277808789666 -87.14693166884906) bank139649 +139650 POINT(42.23605638473008 -87.94309482274598) bank139650 +139651 POINT(41.2712317881077 -88.25096777219821) bank139651 +139652 POINT(41.97745486835953 -87.98930739656376) bank139652 +139653 POINT(42.780851442440834 -87.9307892502608) bank139653 +139654 POINT(42.6265641643862 -88.24694761716343) bank139654 +139655 POINT(42.274545157654785 -87.59298632427964) bank139655 +139656 POINT(40.96204902953335 -87.68628403436107) bank139656 +139657 POINT(41.66447042306409 -87.38305571747016) bank139657 +139658 POINT(42.13727860869274 -87.55195006908394) bank139658 +139659 POINT(41.52895918602023 -87.62489995334953) bank139659 +139660 POINT(42.47395640211441 -88.11854752156059) bank139660 +139661 POINT(40.90523552045463 -86.93224092991441) bank139661 +139662 POINT(41.322753488015934 -87.9920925660593) bank139662 +139663 POINT(42.24795634108826 -87.41424107455134) bank139663 +139664 POINT(41.332966730477565 -87.3700342888848) bank139664 +139665 POINT(42.683150674715954 -88.23318411128206) bank139665 +139666 POINT(42.55689181405882 -86.77357108011891) bank139666 +139667 POINT(42.83953390721942 -87.34763556030576) bank139667 +139668 POINT(42.77094522034507 -87.07308278985813) bank139668 +139669 POINT(41.5056412481774 -87.38032679351181) bank139669 +139670 POINT(42.13597989245558 -86.83170184725668) bank139670 +139671 POINT(41.52642353506829 -87.66581221123815) bank139671 +139672 POINT(41.13825480314919 -88.31257300271079) bank139672 +139673 POINT(42.02735237839071 -87.84556339638516) bank139673 +139674 POINT(41.486951811729526 -88.510772957873) bank139674 +139675 POINT(42.437322773678204 -88.11513022967236) bank139675 +139676 POINT(41.681828984927456 -88.59553741258675) bank139676 +139677 POINT(41.63329678519584 -87.52981488965678) bank139677 +139678 POINT(40.94631011581107 -86.90811730856761) bank139678 +139679 POINT(41.51746949162312 -87.71988110219432) bank139679 +139680 POINT(42.31150077664506 -88.11654184972285) bank139680 +139681 POINT(42.75880093417101 -86.8302048209551) bank139681 +139682 POINT(42.58561241155846 -86.66653387299147) bank139682 +139683 POINT(41.19218338079908 -88.49191976682054) bank139683 +139684 POINT(41.42651075525674 -88.48554881791812) bank139684 +139685 POINT(42.82226406065877 -87.8128751279256) bank139685 +139686 POINT(42.470464152154996 -86.72891039749801) bank139686 +139687 POINT(42.22136155900617 -88.52063777179622) bank139687 +139688 POINT(42.471864483542994 -87.1916567380902) bank139688 +139689 POINT(42.74345265672378 -87.17525269527651) bank139689 +139690 POINT(41.64000229877181 -86.7364676747547) bank139690 +139691 POINT(41.48170493776813 -87.9883947351488) bank139691 +139692 POINT(41.43788038404754 -88.13949454147809) bank139692 +139693 POINT(41.64975581819751 -86.68964313872593) bank139693 +139694 POINT(42.004285682045456 -86.79811386145555) bank139694 +139695 POINT(41.84075818708736 -86.8643154746127) bank139695 +139696 POINT(41.281188845567435 -87.46843434878414) bank139696 +139697 POINT(41.19969642009802 -86.98559854919276) bank139697 +139698 POINT(42.78060419306381 -87.02189278305708) bank139698 +139699 POINT(41.42817203384545 -87.60049742919642) bank139699 +139700 POINT(42.456575927933805 -87.67173746895692) bank139700 +139701 POINT(42.78284800146278 -87.43111677801251) bank139701 +139702 POINT(41.87228173854999 -88.0492521354175) bank139702 +139703 POINT(42.100613976599114 -88.32358077081453) bank139703 +139704 POINT(42.596584451401164 -87.40182824208111) bank139704 +139705 POINT(41.75749309353136 -88.5603315020351) bank139705 +139706 POINT(41.08781122775839 -86.83182836339819) bank139706 +139707 POINT(42.1838676874365 -88.6012613322398) bank139707 +139708 POINT(42.35645005095169 -86.93439518809002) bank139708 +139709 POINT(41.009202275589686 -87.14181003994447) bank139709 +139710 POINT(41.239087531811855 -87.12113678213281) bank139710 +139711 POINT(40.93143212714665 -88.448902883156) bank139711 +139712 POINT(40.89006753196032 -86.69425525552184) bank139712 +139713 POINT(41.74345557788434 -86.91488357265285) bank139713 +139714 POINT(42.13267673496679 -88.22456611394415) bank139714 +139715 POINT(41.20291182503896 -87.51042834325494) bank139715 +139716 POINT(42.0999408811011 -87.29053639511787) bank139716 +139717 POINT(41.00953120714948 -87.93300368678955) bank139717 +139718 POINT(41.4184235383222 -86.63743952607281) bank139718 +139719 POINT(41.308030957927 -87.8630717982766) bank139719 +139720 POINT(41.30007487529235 -88.07974434891258) bank139720 +139721 POINT(41.631651803263274 -87.51157469510837) bank139721 +139722 POINT(42.72295167724079 -86.88387917855654) bank139722 +139723 POINT(42.74911030044564 -87.79234714744962) bank139723 +139724 POINT(41.72955153234777 -87.00869726050531) bank139724 +139725 POINT(41.47054415614086 -88.22564402374597) bank139725 +139726 POINT(41.59893826837849 -88.03009938565889) bank139726 +139727 POINT(41.40524081267396 -86.63202254048278) bank139727 +139728 POINT(41.386989259864215 -87.29255792626537) bank139728 +139729 POINT(42.30620510022399 -87.86169605754021) bank139729 +139730 POINT(41.407599789621294 -87.60320132178519) bank139730 +139731 POINT(42.1265356923198 -86.79100090558069) bank139731 +139732 POINT(42.15648906572871 -87.09297391950001) bank139732 +139733 POINT(41.82360537554526 -87.4386592886578) bank139733 +139734 POINT(41.66631877960574 -88.37795406331769) bank139734 +139735 POINT(41.61074939606218 -87.8820747093461) bank139735 +139736 POINT(41.075182678453366 -87.8850025664469) bank139736 +139737 POINT(41.54827853878378 -87.83510530242972) bank139737 +139738 POINT(41.134408714333404 -87.12704757703216) bank139738 +139739 POINT(41.28678426656879 -88.06958761912188) bank139739 +139740 POINT(42.06428284522062 -88.44785966946502) bank139740 +139741 POINT(42.51128939899535 -87.65391628302592) bank139741 +139742 POINT(41.43401531675272 -88.24036465029788) bank139742 +139743 POINT(42.05061359165382 -87.70802582280228) bank139743 +139744 POINT(41.8687996336024 -87.83725233360295) bank139744 +139745 POINT(42.8276654166298 -87.45665759121319) bank139745 +139746 POINT(41.392900146639775 -87.52070036947639) bank139746 +139747 POINT(42.203355899475866 -87.74022820055426) bank139747 +139748 POINT(41.7326490430485 -87.06334305952159) bank139748 +139749 POINT(42.41068097186597 -88.12907640104302) bank139749 +139750 POINT(42.169337055303075 -87.44846349667674) bank139750 +139751 POINT(41.796378904927565 -87.61852367913741) bank139751 +139752 POINT(41.753378533542026 -87.28377198645668) bank139752 +139753 POINT(42.10147164735188 -88.03682414817341) bank139753 +139754 POINT(41.707969613463746 -87.38185223089204) bank139754 +139755 POINT(42.76988981705872 -88.43291553383554) bank139755 +139756 POINT(42.68457731302156 -87.59328973073026) bank139756 +139757 POINT(41.59528794886575 -87.78330237910635) bank139757 +139758 POINT(41.445126987564876 -88.26818559795782) bank139758 +139759 POINT(42.71615541554218 -88.18265535234289) bank139759 +139760 POINT(41.04754063152658 -88.23073438567172) bank139760 +139761 POINT(41.672217077139564 -87.72340613423626) bank139761 +139762 POINT(42.17919908604208 -87.51730921774585) bank139762 +139763 POINT(42.219281677747446 -86.64325354077319) bank139763 +139764 POINT(41.70358890040489 -87.75295577253956) bank139764 +139765 POINT(42.538554779338675 -88.19099037223445) bank139765 +139766 POINT(41.145754418034045 -87.40602494245557) bank139766 +139767 POINT(41.12980109585953 -86.91236197005233) bank139767 +139768 POINT(42.27013787987345 -87.1860454986654) bank139768 +139769 POINT(41.31725252703041 -86.89980062104254) bank139769 +139770 POINT(41.59770106506118 -86.66340388897412) bank139770 +139771 POINT(41.13761664400674 -86.67915348546889) bank139771 +139772 POINT(41.02279679562565 -87.40790465393592) bank139772 +139773 POINT(41.79531143551759 -88.36125919294533) bank139773 +139774 POINT(42.169515630292864 -88.03993560359805) bank139774 +139775 POINT(40.96914028820558 -87.06038197549692) bank139775 +139776 POINT(41.88412005500317 -88.23981778161625) bank139776 +139777 POINT(41.65913861516315 -86.66370692531257) bank139777 +139778 POINT(42.835646578070026 -86.95602661857984) bank139778 +139779 POINT(41.26048905573855 -88.01551009910722) bank139779 +139780 POINT(41.04110532791793 -86.9253037959118) bank139780 +139781 POINT(41.443275524269154 -87.79887622811418) bank139781 +139782 POINT(41.432902378383545 -87.0637369080704) bank139782 +139783 POINT(40.95391566140913 -87.01316913159769) bank139783 +139784 POINT(41.616898879105854 -87.7571222681093) bank139784 +139785 POINT(42.2462200080181 -86.76988028323606) bank139785 +139786 POINT(42.65661778949118 -87.08972911180207) bank139786 +139787 POINT(41.285163829997686 -87.10259569991784) bank139787 +139788 POINT(41.30569541278916 -87.43117312864315) bank139788 +139789 POINT(41.66608798827802 -87.64327360193707) bank139789 +139790 POINT(41.33896299950919 -88.38909513803534) bank139790 +139791 POINT(42.6042064152783 -87.61536555046382) bank139791 +139792 POINT(41.16978336490813 -87.31975276325667) bank139792 +139793 POINT(42.70358650781754 -87.93264840687684) bank139793 +139794 POINT(42.72531032891315 -87.85545545388158) bank139794 +139795 POINT(41.4694526893589 -87.47829676857842) bank139795 +139796 POINT(40.98245123333219 -87.90833884222275) bank139796 +139797 POINT(41.70676039600619 -88.05447890676201) bank139797 +139798 POINT(41.482501187614986 -87.44660467752202) bank139798 +139799 POINT(42.0814947127878 -87.27449558108641) bank139799 +139800 POINT(42.792614849876934 -87.75619095227655) bank139800 +139801 POINT(40.982154155592816 -87.36435403566637) bank139801 +139802 POINT(42.726736493312806 -88.5714361616773) bank139802 +139803 POINT(42.30811937316082 -87.83341187305909) bank139803 +139804 POINT(41.95044923384379 -88.20487128940762) bank139804 +139805 POINT(41.051324642116576 -87.43912963800966) bank139805 +139806 POINT(42.147157054933054 -88.21807457211457) bank139806 +139807 POINT(42.34147286367969 -87.63567782941033) bank139807 +139808 POINT(41.80239260792618 -88.4372460693542) bank139808 +139809 POINT(41.835225629411255 -86.69088727566884) bank139809 +139810 POINT(40.90569349164849 -87.58351772004298) bank139810 +139811 POINT(42.56683768347243 -88.33852485219909) bank139811 +139812 POINT(41.66463014535473 -86.67104625777613) bank139812 +139813 POINT(42.83127435960651 -87.78740466269765) bank139813 +139814 POINT(42.4342676798669 -88.11522328451386) bank139814 +139815 POINT(41.90053200764291 -86.90137908986038) bank139815 +139816 POINT(42.732326721801186 -87.12577182422348) bank139816 +139817 POINT(42.484618250393005 -86.96044232105034) bank139817 +139818 POINT(42.544832609339814 -88.02333528901457) bank139818 +139819 POINT(40.98849215896811 -88.53130171582778) bank139819 +139820 POINT(42.44335586071103 -87.93615134350314) bank139820 +139821 POINT(41.48013459254947 -86.6459341813111) bank139821 +139822 POINT(42.6469872156116 -88.49451666183762) bank139822 +139823 POINT(42.82689039945434 -88.26143556294558) bank139823 +139824 POINT(42.536347069916125 -86.65233398400113) bank139824 +139825 POINT(42.686283476478195 -88.20566396024716) bank139825 +139826 POINT(42.50182903898021 -88.44211246692748) bank139826 +139827 POINT(41.9802434053365 -87.34736487074832) bank139827 +139828 POINT(42.52619250331782 -88.4482345633705) bank139828 +139829 POINT(41.24182830576055 -86.77670620628751) bank139829 +139830 POINT(42.21458735185774 -86.88297776020009) bank139830 +139831 POINT(41.80936302735779 -87.28884701068334) bank139831 +139832 POINT(42.308984680223254 -87.27445592220154) bank139832 +139833 POINT(42.388407872784086 -87.61811559485619) bank139833 +139834 POINT(42.12036747877192 -88.46483803466701) bank139834 +139835 POINT(41.91646403354802 -86.95410979019502) bank139835 +139836 POINT(41.77833345397902 -88.59075936494655) bank139836 +139837 POINT(41.21612877953008 -86.8791531465144) bank139837 +139838 POINT(40.93189063977713 -88.56730183323599) bank139838 +139839 POINT(42.62218399806777 -88.3727163781071) bank139839 +139840 POINT(42.82185209139242 -86.9813994894274) bank139840 +139841 POINT(41.39539197319432 -86.6659501962304) bank139841 +139842 POINT(42.454528942876074 -87.89003551407082) bank139842 +139843 POINT(41.541418084847436 -86.96719950970034) bank139843 +139844 POINT(42.82374610968067 -86.87423655896065) bank139844 +139845 POINT(42.40026006073206 -87.09975676935277) bank139845 +139846 POINT(41.63575792925334 -87.87274060668403) bank139846 +139847 POINT(42.31876185561198 -87.65896697223718) bank139847 +139848 POINT(42.51394722732785 -88.18338885837582) bank139848 +139849 POINT(41.9478125657231 -88.34755135014764) bank139849 +139850 POINT(41.767806417066964 -87.24954905534281) bank139850 +139851 POINT(41.13190910830232 -87.81420334651438) bank139851 +139852 POINT(42.190188551121736 -87.63019488607614) bank139852 +139853 POINT(42.19626172453226 -88.08125258034853) bank139853 +139854 POINT(42.84290611209058 -87.22388893887145) bank139854 +139855 POINT(42.51481893089977 -87.3182935613894) bank139855 +139856 POINT(42.08303658695995 -88.26614643058267) bank139856 +139857 POINT(42.08493527594061 -88.35006176261773) bank139857 +139858 POINT(41.11095358867377 -87.20810748919743) bank139858 +139859 POINT(41.16235458810346 -87.027959235962) bank139859 +139860 POINT(42.52261947753448 -88.1353153730902) bank139860 +139861 POINT(41.917181704757894 -87.17780500861741) bank139861 +139862 POINT(40.88983608915404 -86.79615783475) bank139862 +139863 POINT(41.7673179477047 -86.83843751127749) bank139863 +139864 POINT(41.46016387066301 -87.68769832736407) bank139864 +139865 POINT(41.95301789389387 -88.17836359315501) bank139865 +139866 POINT(42.483664582617585 -87.78469776191122) bank139866 +139867 POINT(41.96154608767892 -88.4721061188045) bank139867 +139868 POINT(41.21449770091688 -87.0012368528055) bank139868 +139869 POINT(41.174564835361764 -87.58873104821166) bank139869 +139870 POINT(40.97535243309826 -87.45938431702687) bank139870 +139871 POINT(42.132696514956386 -86.68349080773586) bank139871 +139872 POINT(41.27232450578673 -87.10149699000642) bank139872 +139873 POINT(41.0344602127819 -87.77495772266047) bank139873 +139874 POINT(42.547411865243646 -87.02588912717043) bank139874 +139875 POINT(41.397478999874274 -87.39909090452022) bank139875 +139876 POINT(42.19088706518228 -88.55184596302678) bank139876 +139877 POINT(41.88515187928299 -87.02721536413536) bank139877 +139878 POINT(42.164573729645504 -86.90967067935244) bank139878 +139879 POINT(41.34410012520465 -88.27891860316079) bank139879 +139880 POINT(41.02297561617023 -88.56985553509) bank139880 +139881 POINT(40.9594137239536 -87.59123210803406) bank139881 +139882 POINT(42.2280279790757 -88.17599773435441) bank139882 +139883 POINT(41.92515475125811 -87.07747141892644) bank139883 +139884 POINT(42.35100027350325 -87.11584603592793) bank139884 +139885 POINT(42.67224867552993 -87.81350098303533) bank139885 +139886 POINT(42.85938341645991 -87.44412981459546) bank139886 +139887 POINT(41.54549065715543 -86.84235496211654) bank139887 +139888 POINT(42.876523757263655 -86.91010280157344) bank139888 +139889 POINT(41.92462622349034 -87.0816596813917) bank139889 +139890 POINT(40.98819544021252 -86.78975121410772) bank139890 +139891 POINT(41.39750064372241 -87.4625466559303) bank139891 +139892 POINT(41.68442854833198 -87.69013620382046) bank139892 +139893 POINT(41.45515034569079 -87.03840221077319) bank139893 +139894 POINT(42.218969537674475 -88.62879244286898) bank139894 +139895 POINT(41.62139348090988 -87.92716636980917) bank139895 +139896 POINT(41.9107397208277 -87.65784142311344) bank139896 +139897 POINT(42.419638451642484 -86.83308195962283) bank139897 +139898 POINT(42.29652312071509 -86.96872290665893) bank139898 +139899 POINT(41.09266460016923 -86.82222960024744) bank139899 +139900 POINT(41.83308839607427 -87.93143388077365) bank139900 +139901 POINT(42.749779139441195 -87.0957472709631) bank139901 +139902 POINT(42.556415003598886 -86.92906426954153) bank139902 +139903 POINT(42.269756253751396 -88.52088879099674) bank139903 +139904 POINT(41.204529866659314 -87.64164732876988) bank139904 +139905 POINT(40.906761345153846 -86.81983012271017) bank139905 +139906 POINT(42.79131967206795 -86.76295942270538) bank139906 +139907 POINT(41.32497266567892 -86.92511317576688) bank139907 +139908 POINT(42.78887061925487 -88.048648812192) bank139908 +139909 POINT(42.76691979675988 -86.95835529859448) bank139909 +139910 POINT(42.749798222915466 -87.14539632969763) bank139910 +139911 POINT(41.94931401842623 -88.40066625799668) bank139911 +139912 POINT(42.18722497033074 -88.15517824983387) bank139912 +139913 POINT(42.58716676733802 -87.51222412779684) bank139913 +139914 POINT(41.189638951315736 -87.80736725125524) bank139914 +139915 POINT(41.60860758573813 -87.31812949418662) bank139915 +139916 POINT(42.418574274493764 -88.14866768783293) bank139916 +139917 POINT(41.796944778647685 -86.84388393305478) bank139917 +139918 POINT(41.155407034091844 -88.46476591784257) bank139918 +139919 POINT(41.60667201850851 -87.74214867325156) bank139919 +139920 POINT(42.158386791554015 -87.90049319970058) bank139920 +139921 POINT(41.917301483440596 -88.41997471176062) bank139921 +139922 POINT(42.78662593679622 -87.68591470467724) bank139922 +139923 POINT(41.18410981115728 -88.5484823569754) bank139923 +139924 POINT(41.89250894780404 -87.20825714599313) bank139924 +139925 POINT(41.08348605960724 -87.34519367912723) bank139925 +139926 POINT(42.82538616647221 -86.85732365734675) bank139926 +139927 POINT(42.038180084409326 -87.37927810456209) bank139927 +139928 POINT(42.7665796453383 -86.67445417733614) bank139928 +139929 POINT(41.907128380700044 -87.22441533834296) bank139929 +139930 POINT(42.53814317285629 -87.13878896270727) bank139930 +139931 POINT(41.84266083560328 -88.24391300961072) bank139931 +139932 POINT(41.67712878362735 -87.72145797698373) bank139932 +139933 POINT(41.18922490599635 -86.94948003291505) bank139933 +139934 POINT(42.84186472793034 -87.17743090136511) bank139934 +139935 POINT(40.962678276757465 -88.37687880876767) bank139935 +139936 POINT(42.5908487838293 -88.54511093158547) bank139936 +139937 POINT(42.16840319718912 -86.9980037548498) bank139937 +139938 POINT(42.38830517160897 -86.70941735201231) bank139938 +139939 POINT(41.165085903399145 -87.9496092169867) bank139939 +139940 POINT(41.905257902538516 -86.831732663578) bank139940 +139941 POINT(42.283251006304134 -87.33461817331604) bank139941 +139942 POINT(42.433582692295886 -88.48449566546824) bank139942 +139943 POINT(42.798007956067316 -88.13778210637848) bank139943 +139944 POINT(41.44225486536539 -88.25473002968987) bank139944 +139945 POINT(41.682438119943846 -87.26728614861973) bank139945 +139946 POINT(42.34892946662559 -87.00791953393238) bank139946 +139947 POINT(40.95546649409235 -87.50993336652456) bank139947 +139948 POINT(41.781301180468404 -88.61730367414961) bank139948 +139949 POINT(42.63428173321878 -86.96085915478008) bank139949 +139950 POINT(42.35896224056104 -88.06283990059092) bank139950 +139951 POINT(42.142400852300824 -88.55900100257453) bank139951 +139952 POINT(42.366593059676326 -87.47887762370681) bank139952 +139953 POINT(42.54107204484899 -87.61597509673597) bank139953 +139954 POINT(41.0314350219051 -87.72940481677364) bank139954 +139955 POINT(41.73650623976113 -88.14022449077254) bank139955 +139956 POINT(40.967900005499075 -88.48516669175554) bank139956 +139957 POINT(41.512308581108606 -87.13705641195293) bank139957 +139958 POINT(42.798936331238885 -87.6055966050003) bank139958 +139959 POINT(41.56273497487772 -87.33980042569901) bank139959 +139960 POINT(42.49763646281576 -88.03107457033387) bank139960 +139961 POINT(41.13430217384308 -88.57610786449672) bank139961 +139962 POINT(41.2456053998833 -87.69079831948568) bank139962 +139963 POINT(41.07833425421188 -88.24674168441274) bank139963 +139964 POINT(41.63726357733269 -87.32973141197209) bank139964 +139965 POINT(41.12553609419968 -87.37923117758969) bank139965 +139966 POINT(41.88183632263626 -87.63175167973485) bank139966 +139967 POINT(42.2676008569383 -87.76864457172688) bank139967 +139968 POINT(42.47465665151716 -88.02898385773112) bank139968 +139969 POINT(41.66670672258829 -88.38944278550427) bank139969 +139970 POINT(42.21137933447839 -87.42681126721095) bank139970 +139971 POINT(41.2570693404935 -88.51929809357956) bank139971 +139972 POINT(42.778998735252216 -88.48479817693097) bank139972 +139973 POINT(41.91494712467058 -88.1080276358143) bank139973 +139974 POINT(41.35200264731239 -88.42274235892526) bank139974 +139975 POINT(41.988945686678214 -87.44861240066618) bank139975 +139976 POINT(41.90522561584695 -87.28418872125027) bank139976 +139977 POINT(42.52915695772881 -88.25285778930414) bank139977 +139978 POINT(42.43437037159503 -87.33121767613395) bank139978 +139979 POINT(42.16364343488785 -88.05180789182032) bank139979 +139980 POINT(42.181582949124014 -88.5229146691039) bank139980 +139981 POINT(41.268165646545185 -86.69717088075335) bank139981 +139982 POINT(41.56720055208209 -88.0841363782872) bank139982 +139983 POINT(41.78113727217293 -88.5447815516438) bank139983 +139984 POINT(41.516036223376126 -87.40374464012962) bank139984 +139985 POINT(42.183217632473216 -88.55427388991178) bank139985 +139986 POINT(41.26577482191192 -88.1947672908512) bank139986 +139987 POINT(42.8539509182622 -88.27519281229797) bank139987 +139988 POINT(41.03803329215896 -86.70750129414402) bank139988 +139989 POINT(40.961286979676274 -87.89127851332512) bank139989 +139990 POINT(41.2834691400984 -87.17003170652964) bank139990 +139991 POINT(42.35423999314014 -87.59396227444041) bank139991 +139992 POINT(42.03703427390858 -87.53354111227016) bank139992 +139993 POINT(41.00206153787598 -87.9566231177529) bank139993 +139994 POINT(40.91219153287406 -87.3041125138694) bank139994 +139995 POINT(41.866914726297395 -87.51342361346715) bank139995 +139996 POINT(41.63216101585092 -86.8959521096095) bank139996 +139997 POINT(41.44640274650802 -87.12134441152355) bank139997 +139998 POINT(41.18337583058929 -87.3204103348711) bank139998 +139999 POINT(41.24591705200126 -87.01463545174431) bank139999 +140000 POINT(41.41018907126736 -88.1171529496881) bank140000 +140001 POINT(42.43546563108129 -86.71845901877728) bank140001 +140002 POINT(41.363666555056845 -86.97188906499126) bank140002 +140003 POINT(41.896779054790905 -87.14741546410217) bank140003 +140004 POINT(42.618094155537975 -86.882226540966) bank140004 +140005 POINT(41.2105856448678 -88.22682440617245) bank140005 +140006 POINT(42.57633252436595 -87.32360567516739) bank140006 +140007 POINT(41.290812888401604 -88.57790096898897) bank140007 +140008 POINT(42.16372438183909 -87.46144000739605) bank140008 +140009 POINT(41.3864900469071 -87.56867328415075) bank140009 +140010 POINT(40.940593519744134 -87.10150698318138) bank140010 +140011 POINT(41.693757853213604 -87.551836707447) bank140011 +140012 POINT(41.530478418442954 -88.35081014798149) bank140012 +140013 POINT(42.25577564319474 -86.65774692702777) bank140013 +140014 POINT(42.739565093558504 -88.59253464684998) bank140014 +140015 POINT(41.60820088085995 -88.27139674086325) bank140015 +140016 POINT(41.3013269556624 -87.81117898285096) bank140016 +140017 POINT(42.50634156516659 -88.39615576821606) bank140017 +140018 POINT(41.346177213726314 -88.35772926522576) bank140018 +140019 POINT(41.82617981925066 -87.9735743911403) bank140019 +140020 POINT(41.32917314261694 -87.9538986745586) bank140020 +140021 POINT(42.14926378837618 -88.07896668067036) bank140021 +140022 POINT(42.03417898982748 -86.8572864971472) bank140022 +140023 POINT(42.22502481563522 -88.05267788118816) bank140023 +140024 POINT(41.85713207739241 -88.51610549184296) bank140024 +140025 POINT(42.517997628086974 -88.2075608092433) bank140025 +140026 POINT(41.2748765100464 -87.80989725998705) bank140026 +140027 POINT(42.55279293566955 -87.83177716256098) bank140027 +140028 POINT(41.11042334490687 -88.09758450070768) bank140028 +140029 POINT(41.055990481402695 -87.8754698801516) bank140029 +140030 POINT(42.412773858034505 -87.29108055783277) bank140030 +140031 POINT(42.005128380781116 -86.71170943589387) bank140031 +140032 POINT(42.867370598796725 -86.85312933490682) bank140032 +140033 POINT(41.56042871622079 -88.25965838421891) bank140033 +140034 POINT(42.4512223137701 -86.79363279224515) bank140034 +140035 POINT(42.40623198754771 -88.51225570591109) bank140035 +140036 POINT(41.05293474378202 -87.3768108365384) bank140036 +140037 POINT(42.84858890177664 -87.44465273801329) bank140037 +140038 POINT(42.00055786368231 -86.7994648336941) bank140038 +140039 POINT(41.307506789169395 -87.12404086887088) bank140039 +140040 POINT(41.73672319871614 -88.5896312907643) bank140040 +140041 POINT(41.03367857112919 -86.67524596089575) bank140041 +140042 POINT(42.450394643738726 -88.45119143776961) bank140042 +140043 POINT(42.524488060095095 -87.4460279277868) bank140043 +140044 POINT(42.12315119241724 -88.4341000858325) bank140044 +140045 POINT(41.04708986708449 -88.27424679653546) bank140045 +140046 POINT(41.94169514521231 -87.21122574398566) bank140046 +140047 POINT(41.03891579148307 -87.25518320414473) bank140047 +140048 POINT(41.96617436631665 -88.07083580534126) bank140048 +140049 POINT(42.041330760214095 -87.26716235243414) bank140049 +140050 POINT(42.10652126599901 -87.17261251971766) bank140050 +140051 POINT(42.78797140646783 -88.0922272683815) bank140051 +140052 POINT(41.49031581538097 -87.22660595663136) bank140052 +140053 POINT(41.69634081904508 -86.98038825288472) bank140053 +140054 POINT(42.7415985292393 -86.96890184692792) bank140054 +140055 POINT(41.17957738960794 -88.21352107771433) bank140055 +140056 POINT(41.56207075357934 -88.09640025981996) bank140056 +140057 POINT(42.53963499366015 -87.82700852879236) bank140057 +140058 POINT(42.69638503276264 -87.02357477817132) bank140058 +140059 POINT(41.8175510561512 -88.4346260794835) bank140059 +140060 POINT(42.05991552511196 -88.2013687745836) bank140060 +140061 POINT(41.28957891954238 -87.80768465757531) bank140061 +140062 POINT(42.7741204630455 -87.41280304666194) bank140062 +140063 POINT(41.19052539835158 -87.77456408256819) bank140063 +140064 POINT(40.914769859696456 -88.51317899036292) bank140064 +140065 POINT(42.82656885113373 -88.3639077142681) bank140065 +140066 POINT(41.077406667912165 -86.75809577497743) bank140066 +140067 POINT(42.431845892276314 -88.11628610452973) bank140067 +140068 POINT(41.44528092060457 -86.77619226916528) bank140068 +140069 POINT(41.05517016530351 -87.41986970243777) bank140069 +140070 POINT(41.07963339657765 -87.79551369371515) bank140070 +140071 POINT(41.85703572481067 -86.66058576454901) bank140071 +140072 POINT(41.863155858415176 -87.27494920840255) bank140072 +140073 POINT(42.45708808030673 -86.95968825972587) bank140073 +140074 POINT(41.29687992586894 -88.31121812306039) bank140074 +140075 POINT(41.461729608707245 -87.60906541712801) bank140075 +140076 POINT(41.493517119115744 -87.83122618421815) bank140076 +140077 POINT(41.4516546964816 -87.25420692854055) bank140077 +140078 POINT(41.33895928105945 -87.32206332895015) bank140078 +140079 POINT(41.370313970033514 -88.14486548880602) bank140079 +140080 POINT(42.215780271048025 -87.35703634414398) bank140080 +140081 POINT(41.92235185365522 -88.2368715228424) bank140081 +140082 POINT(42.18982267583175 -87.24278018530747) bank140082 +140083 POINT(41.02111923593136 -86.66253950804034) bank140083 +140084 POINT(41.23408732239976 -86.92648691820504) bank140084 +140085 POINT(42.811042013554804 -88.48869298121741) bank140085 +140086 POINT(42.43390121036215 -87.72083993229184) bank140086 +140087 POINT(42.04165835333218 -87.86067831811404) bank140087 +140088 POINT(41.684800935462846 -87.20048134803866) bank140088 +140089 POINT(42.14299263422753 -88.6255318707545) bank140089 +140090 POINT(41.75003455842519 -88.33280359410809) bank140090 +140091 POINT(42.11476602828568 -87.80120396577321) bank140091 +140092 POINT(41.44280960202943 -87.4956497241025) bank140092 +140093 POINT(42.56676450730826 -87.8300458563301) bank140093 +140094 POINT(42.47128301434005 -87.36002212238739) bank140094 +140095 POINT(42.111223878031986 -88.49388693777296) bank140095 +140096 POINT(41.29257247984261 -88.28416323808386) bank140096 +140097 POINT(41.26205995958356 -87.93214637224987) bank140097 +140098 POINT(41.266352782437345 -88.62629522007126) bank140098 +140099 POINT(41.88095212447208 -87.45078732865434) bank140099 +140100 POINT(40.92402958850502 -87.09543680710156) bank140100 +140101 POINT(42.72669865690315 -87.53039111612654) bank140101 +140102 POINT(41.429949775328055 -88.06517625309246) bank140102 +140103 POINT(41.10082885607855 -86.84215119402346) bank140103 +140104 POINT(41.31704904273273 -86.71876395395016) bank140104 +140105 POINT(41.37065771241056 -87.33754280159498) bank140105 +140106 POINT(42.82278205072252 -87.64347211142943) bank140106 +140107 POINT(42.11123428778429 -87.61617720311821) bank140107 +140108 POINT(41.824302270395236 -87.29653439528002) bank140108 +140109 POINT(42.6524221978317 -88.6217492274174) bank140109 +140110 POINT(41.60768291948383 -87.84537527621097) bank140110 +140111 POINT(42.237999598987734 -88.22615226513133) bank140111 +140112 POINT(41.85852778300758 -87.38668941244859) bank140112 +140113 POINT(41.682545665768515 -87.45764254307853) bank140113 +140114 POINT(41.17961155849 -87.42197789021667) bank140114 +140115 POINT(41.37451088994621 -88.24913234043618) bank140115 +140116 POINT(42.30432820263919 -88.43094938201709) bank140116 +140117 POINT(40.95788830094088 -88.00831361453201) bank140117 +140118 POINT(41.59624184669033 -87.17013159007065) bank140118 +140119 POINT(41.08095692004867 -86.79186404184) bank140119 +140120 POINT(42.785853975751095 -86.76359924915032) bank140120 +140121 POINT(40.96723917719895 -88.47352988959271) bank140121 +140122 POINT(42.50802620930149 -87.5734747264273) bank140122 +140123 POINT(41.810824593959595 -87.29687658412321) bank140123 +140124 POINT(41.949903067791034 -88.50061581156024) bank140124 +140125 POINT(41.793777777817034 -86.66711691908047) bank140125 +140126 POINT(41.43744258033243 -88.0603021926226) bank140126 +140127 POINT(41.42751292994009 -88.41520467087177) bank140127 +140128 POINT(41.43410568153775 -88.4842028841537) bank140128 +140129 POINT(41.10891423576878 -88.51871511280915) bank140129 +140130 POINT(41.92854935937716 -86.9268280891092) bank140130 +140131 POINT(42.15727684498968 -88.12209492467609) bank140131 +140132 POINT(42.36657567552798 -88.59547847976881) bank140132 +140133 POINT(42.44515647346658 -87.64144411314093) bank140133 +140134 POINT(41.36826623909237 -87.04138798028704) bank140134 +140135 POINT(41.03411565366741 -88.20507855532323) bank140135 +140136 POINT(41.99946205044515 -87.44676466935238) bank140136 +140137 POINT(41.31756460779363 -87.53428539313099) bank140137 +140138 POINT(41.43134911357999 -88.02652033495923) bank140138 +140139 POINT(42.42581973735384 -87.65564162801176) bank140139 +140140 POINT(41.78341724949361 -87.6397382034829) bank140140 +140141 POINT(42.83132102822284 -86.86357243338453) bank140141 +140142 POINT(41.82439079271017 -86.8781419688505) bank140142 +140143 POINT(42.843218790450265 -86.81780640002629) bank140143 +140144 POINT(41.71617122391541 -87.23689819213844) bank140144 +140145 POINT(42.22848325427511 -86.687942701451) bank140145 +140146 POINT(42.16837995240578 -87.22652921197181) bank140146 +140147 POINT(41.05641462336147 -88.02309262984075) bank140147 +140148 POINT(41.55768623320604 -87.22141653625341) bank140148 +140149 POINT(41.263887511157556 -86.73967962803303) bank140149 +140150 POINT(42.379422751005 -87.32980003235372) bank140150 +140151 POINT(41.87319433034894 -88.32097862162156) bank140151 +140152 POINT(42.12338511559459 -86.88752823387651) bank140152 +140153 POINT(42.817284767708514 -88.56771082702028) bank140153 +140154 POINT(41.53359076445156 -87.50362012176981) bank140154 +140155 POINT(42.84996295524615 -87.99089303968361) bank140155 +140156 POINT(41.83313805159645 -87.80157113239297) bank140156 +140157 POINT(41.35044894692176 -88.36139569397474) bank140157 +140158 POINT(41.85269803833201 -88.38327853243027) bank140158 +140159 POINT(41.941376268269316 -87.62539293122347) bank140159 +140160 POINT(42.084910694990974 -88.17638372202393) bank140160 +140161 POINT(41.44066580793763 -88.38905092264403) bank140161 +140162 POINT(42.58445328963259 -88.48476795053114) bank140162 +140163 POINT(42.26013988112539 -87.74519656915072) bank140163 +140164 POINT(41.44837157366493 -87.62551642025953) bank140164 +140165 POINT(41.213857327076305 -88.19209017440177) bank140165 +140166 POINT(42.76414717085903 -88.29708815619719) bank140166 +140167 POINT(41.693372017086496 -86.9679467023084) bank140167 +140168 POINT(42.34760769486137 -88.55610019907542) bank140168 +140169 POINT(41.94483569108526 -87.70582419686157) bank140169 +140170 POINT(41.019318790196586 -88.41650259671015) bank140170 +140171 POINT(41.420489587440606 -87.13978896184871) bank140171 +140172 POINT(41.4674469570935 -87.05570678435281) bank140172 +140173 POINT(41.51159357741617 -88.07995896175127) bank140173 +140174 POINT(42.441134370843145 -87.96114478250209) bank140174 +140175 POINT(41.26061552393725 -88.41160784191274) bank140175 +140176 POINT(42.87472431351468 -87.0515370753477) bank140176 +140177 POINT(40.92065603508274 -86.7522919998787) bank140177 +140178 POINT(42.49217535551065 -87.29559005177278) bank140178 +140179 POINT(41.94328312289301 -88.09646731521593) bank140179 +140180 POINT(41.682804998264615 -88.62143193564071) bank140180 +140181 POINT(40.93264743696323 -88.48697633398893) bank140181 +140182 POINT(42.85759385344862 -86.64929579251438) bank140182 +140183 POINT(42.09702115724705 -86.7636348221058) bank140183 +140184 POINT(42.19786643128115 -88.04959266150004) bank140184 +140185 POINT(42.45133356662087 -87.35326732588523) bank140185 +140186 POINT(42.76656402657704 -87.21151741841275) bank140186 +140187 POINT(41.31841507313088 -88.59753158631818) bank140187 +140188 POINT(41.695577497270655 -87.40762179773894) bank140188 +140189 POINT(41.88046217623424 -88.1444976305888) bank140189 +140190 POINT(41.5655984240968 -88.00444714241375) bank140190 +140191 POINT(41.85333448807584 -86.9617995829444) bank140191 +140192 POINT(41.318341072542765 -86.9892012316603) bank140192 +140193 POINT(41.29946298913121 -87.08612019084181) bank140193 +140194 POINT(41.45240375636565 -87.21545642366641) bank140194 +140195 POINT(41.130175691550846 -87.19463065255934) bank140195 +140196 POINT(42.179271972321125 -87.81937374775161) bank140196 +140197 POINT(42.786661240594164 -87.18251502937537) bank140197 +140198 POINT(41.169772816304324 -87.03261003532066) bank140198 +140199 POINT(42.76733820426902 -87.91873761768076) bank140199 +140200 POINT(40.98390499605051 -87.79807646168929) bank140200 +140201 POINT(41.789135499515126 -87.36892894629065) bank140201 +140202 POINT(40.89573656003073 -88.31925181937414) bank140202 +140203 POINT(41.38699206990946 -87.81753766986817) bank140203 +140204 POINT(41.718129931477975 -87.94737878428937) bank140204 +140205 POINT(42.69788369965288 -87.68194477628387) bank140205 +140206 POINT(42.48903322335684 -87.50329230238478) bank140206 +140207 POINT(41.10771058242991 -88.31041200839128) bank140207 +140208 POINT(41.99453297029979 -87.46928383115144) bank140208 +140209 POINT(41.359549662367385 -87.48934395901394) bank140209 +140210 POINT(41.783808768949655 -87.86601709383598) bank140210 +140211 POINT(42.21385445294303 -87.22313866586452) bank140211 +140212 POINT(41.67766148626828 -87.10335776243257) bank140212 +140213 POINT(41.87697622456544 -86.835419909415) bank140213 +140214 POINT(41.62834539633098 -87.47007017977758) bank140214 +140215 POINT(42.38137543786805 -86.75213157878217) bank140215 +140216 POINT(42.38695030463085 -88.31487682692834) bank140216 +140217 POINT(41.981307239110066 -87.50697282114152) bank140217 +140218 POINT(42.38823642344077 -88.50676252574046) bank140218 +140219 POINT(42.385985952612295 -87.37758542354572) bank140219 +140220 POINT(41.9919307311114 -87.87897546419428) bank140220 +140221 POINT(42.266407949995774 -87.71685901358057) bank140221 +140222 POINT(42.67056703000927 -88.4855537729688) bank140222 +140223 POINT(41.941347340810644 -86.91116978830921) bank140223 +140224 POINT(42.835160962653184 -88.48872730137866) bank140224 +140225 POINT(41.346515432200945 -86.6619048032656) bank140225 +140226 POINT(42.70736813110609 -87.78300510834745) bank140226 +140227 POINT(42.51936452486848 -87.76343997963782) bank140227 +140228 POINT(42.10889802891836 -87.8317323314299) bank140228 +140229 POINT(41.92697179219746 -87.80232427762985) bank140229 +140230 POINT(42.58985961713189 -87.67012990534683) bank140230 +140231 POINT(41.9202928264489 -87.89971936281346) bank140231 +140232 POINT(42.51324273627221 -87.67610942235008) bank140232 +140233 POINT(41.4643340165687 -88.36318316770864) bank140233 +140234 POINT(42.28404371283998 -86.66132456841576) bank140234 +140235 POINT(41.135419221029004 -87.13866730640854) bank140235 +140236 POINT(41.57199458276822 -86.64097078844702) bank140236 +140237 POINT(42.11070706086776 -87.04454526942803) bank140237 +140238 POINT(41.648302166159084 -87.50928208149966) bank140238 +140239 POINT(42.0275113420924 -88.56569846410316) bank140239 +140240 POINT(41.374024609032006 -86.83880647138656) bank140240 +140241 POINT(40.925950861011636 -87.61309029529514) bank140241 +140242 POINT(42.212391338252985 -87.86122488270561) bank140242 +140243 POINT(41.80439787788364 -87.26009672178756) bank140243 +140244 POINT(41.938398704817956 -88.51308345330438) bank140244 +140245 POINT(41.899200331474624 -88.59744048287867) bank140245 +140246 POINT(41.68037289778282 -88.3257795637081) bank140246 +140247 POINT(41.62679083218791 -86.93487155176291) bank140247 +140248 POINT(41.012169855523894 -87.42869615241206) bank140248 +140249 POINT(41.35978902951613 -87.85371549819163) bank140249 +140250 POINT(41.778017791541956 -87.08159334790372) bank140250 +140251 POINT(41.14014469886343 -88.10264784647521) bank140251 +140252 POINT(41.782178427499915 -88.16816988687714) bank140252 +140253 POINT(41.596142183975395 -88.29606040597511) bank140253 +140254 POINT(42.50445785299127 -87.175024241435) bank140254 +140255 POINT(40.89276531808557 -87.52601348133528) bank140255 +140256 POINT(41.37408366287732 -86.67186338007276) bank140256 +140257 POINT(42.643337706125116 -87.33676719651996) bank140257 +140258 POINT(42.49032006738324 -88.42884446114253) bank140258 +140259 POINT(41.929429284679514 -88.22191547430263) bank140259 +140260 POINT(42.45134890657699 -87.11012311292674) bank140260 +140261 POINT(41.07176988390339 -86.8793218434045) bank140261 +140262 POINT(41.79156990554981 -87.19464502722676) bank140262 +140263 POINT(41.06409442028596 -87.13926288351661) bank140263 +140264 POINT(41.975531952889476 -87.69309910646639) bank140264 +140265 POINT(42.57140836189376 -87.97676321050358) bank140265 +140266 POINT(41.93691813399273 -86.88746997048358) bank140266 +140267 POINT(42.746840741308354 -87.20681572681279) bank140267 +140268 POINT(41.73622688741013 -86.70249330926508) bank140268 +140269 POINT(42.70457765345542 -87.98835569601682) bank140269 +140270 POINT(42.05971028512555 -86.91145418418017) bank140270 +140271 POINT(41.635383013719256 -87.57483249571558) bank140271 +140272 POINT(41.593000434321176 -86.88942899905487) bank140272 +140273 POINT(41.68927448685909 -86.8934227776845) bank140273 +140274 POINT(41.52177703701833 -88.31409785244101) bank140274 +140275 POINT(41.772238199375664 -88.41927552005197) bank140275 +140276 POINT(41.6759836243754 -87.79108076746162) bank140276 +140277 POINT(42.27102899028962 -88.01968849434014) bank140277 +140278 POINT(41.41094920238122 -86.82146600458329) bank140278 +140279 POINT(41.47734195973755 -86.7486718997656) bank140279 +140280 POINT(41.219558192052446 -86.9788028376519) bank140280 +140281 POINT(42.221912229622035 -87.60366417053001) bank140281 +140282 POINT(41.47608796285053 -87.1296588761082) bank140282 +140283 POINT(41.94621631869958 -88.10423192028469) bank140283 +140284 POINT(42.482003617066006 -87.18979793373717) bank140284 +140285 POINT(41.57608647018388 -88.48715170292942) bank140285 +140286 POINT(41.20617831761637 -88.18520198471873) bank140286 +140287 POINT(42.38621966571824 -88.16249162164435) bank140287 +140288 POINT(41.91272692119233 -86.79949063717669) bank140288 +140289 POINT(41.387014490535556 -86.64057124437899) bank140289 +140290 POINT(42.51980771880039 -88.0866239094713) bank140290 +140291 POINT(42.07353490783298 -87.6181502582927) bank140291 +140292 POINT(41.40640621738947 -87.71412018045726) bank140292 +140293 POINT(41.025299946185214 -88.19672433269389) bank140293 +140294 POINT(41.88181723952554 -86.73780442191101) bank140294 +140295 POINT(42.45988055569242 -88.38852440933694) bank140295 +140296 POINT(41.7538189416556 -88.2734040125913) bank140296 +140297 POINT(41.891067703098074 -87.73113729294921) bank140297 +140298 POINT(41.15766994952333 -87.82655527334056) bank140298 +140299 POINT(41.281045202991 -87.86252028872042) bank140299 +140300 POINT(41.16691119776216 -87.39458181043227) bank140300 +140301 POINT(40.87843220468032 -87.08249108948381) bank140301 +140302 POINT(42.20150647639165 -88.57511777712483) bank140302 +140303 POINT(42.496136990859256 -87.30416568141901) bank140303 +140304 POINT(42.19823603707721 -87.84626583740481) bank140304 +140305 POINT(42.09094904631953 -86.75075022561327) bank140305 +140306 POINT(41.62923200012771 -87.72275937040985) bank140306 +140307 POINT(42.129978047522464 -86.67590632562043) bank140307 +140308 POINT(42.58286776483921 -87.56565542334322) bank140308 +140309 POINT(41.31220084623629 -88.42624792677029) bank140309 +140310 POINT(42.27276029501338 -87.6420458114439) bank140310 +140311 POINT(41.456307403838366 -88.54575912888774) bank140311 +140312 POINT(42.1250454161417 -86.86452191469472) bank140312 +140313 POINT(41.777956291869764 -88.46758959774915) bank140313 +140314 POINT(41.62929466240203 -88.38653484988781) bank140314 +140315 POINT(41.539766764863096 -86.80957585033693) bank140315 +140316 POINT(42.7799543911513 -87.59488068444453) bank140316 +140317 POINT(41.664576432409774 -88.54440733773168) bank140317 +140318 POINT(42.83384860604625 -87.53547581126948) bank140318 +140319 POINT(41.7739805522739 -87.79678797403997) bank140319 +140320 POINT(42.43641171994341 -87.19729947832862) bank140320 +140321 POINT(41.96246099750128 -86.67140323546157) bank140321 +140322 POINT(41.715913870744465 -88.27923798489721) bank140322 +140323 POINT(42.30651584765477 -87.20398373178604) bank140323 +140324 POINT(42.764626690913374 -88.55124427455243) bank140324 +140325 POINT(42.69293113218039 -87.11784017988981) bank140325 +140326 POINT(41.74719509759122 -88.58076456588466) bank140326 +140327 POINT(41.60863244287442 -87.66592317987573) bank140327 +140328 POINT(41.499829043238485 -86.85532373667343) bank140328 +140329 POINT(42.601852822228444 -87.49630374720434) bank140329 +140330 POINT(41.63041740867804 -87.50650563671056) bank140330 +140331 POINT(42.676434253026045 -87.53887153919467) bank140331 +140332 POINT(42.02947228038565 -88.13748242021674) bank140332 +140333 POINT(41.09092242336364 -88.53455744018441) bank140333 +140334 POINT(41.28969540116624 -88.1525146589718) bank140334 +140335 POINT(41.8115521016715 -86.99334381451068) bank140335 +140336 POINT(42.134536094498415 -86.67030700645972) bank140336 +140337 POINT(41.298861992381475 -88.12778914194935) bank140337 +140338 POINT(42.54772232087693 -88.24883926261148) bank140338 +140339 POINT(41.82198394623238 -87.76022777622067) bank140339 +140340 POINT(42.74554048690329 -87.79925270484193) bank140340 +140341 POINT(41.709689401484226 -88.43138546749657) bank140341 +140342 POINT(41.60687667964846 -88.09557090801195) bank140342 +140343 POINT(41.927172033606325 -88.35868001756808) bank140343 +140344 POINT(42.20957138936531 -88.05559545687397) bank140344 +140345 POINT(40.923547568622766 -86.94612903614416) bank140345 +140346 POINT(40.95821529829166 -87.00036986196636) bank140346 +140347 POINT(41.856291804184174 -86.80540467451645) bank140347 +140348 POINT(41.65542189072309 -86.96899998943053) bank140348 +140349 POINT(41.181885134215065 -87.77933825547304) bank140349 +140350 POINT(42.413948453515566 -87.94412997850809) bank140350 +140351 POINT(41.27442928692462 -86.83960589106255) bank140351 +140352 POINT(41.153041342857975 -88.47282607905991) bank140352 +140353 POINT(42.30635107438418 -87.6267805077348) bank140353 +140354 POINT(41.60957000453708 -87.10077324022852) bank140354 +140355 POINT(41.6486780996691 -88.00135107844034) bank140355 +140356 POINT(42.51919026475833 -86.74116275317964) bank140356 +140357 POINT(41.7151326302806 -87.82542984115413) bank140357 +140358 POINT(41.589112610829524 -88.32822680372637) bank140358 +140359 POINT(41.320536335342055 -88.04538176504245) bank140359 +140360 POINT(41.57983290875848 -86.76106256404518) bank140360 +140361 POINT(41.42945819201407 -86.71162589218909) bank140361 +140362 POINT(41.82557888713307 -87.68198887465935) bank140362 +140363 POINT(42.72811708055803 -87.52810184039564) bank140363 +140364 POINT(42.230093409012746 -88.10655409290547) bank140364 +140365 POINT(41.8414357135986 -87.27834599778774) bank140365 +140366 POINT(42.001652208085915 -87.0442793504749) bank140366 +140367 POINT(42.187493249587725 -87.38205327753691) bank140367 +140368 POINT(41.65587493717336 -88.55393191351588) bank140368 +140369 POINT(42.020060514471005 -87.44007502094745) bank140369 +140370 POINT(41.61205274997388 -87.84844112449939) bank140370 +140371 POINT(41.61993429872848 -87.66799558753183) bank140371 +140372 POINT(42.86959301380587 -87.87663133768488) bank140372 +140373 POINT(41.43693400909953 -87.62958622322114) bank140373 +140374 POINT(41.842710041815366 -86.86100284823473) bank140374 +140375 POINT(42.19955406494099 -87.23185826001496) bank140375 +140376 POINT(42.8048253498441 -88.46188002937352) bank140376 +140377 POINT(41.245223351673424 -86.77165221634041) bank140377 +140378 POINT(42.4866508146272 -87.08096125267144) bank140378 +140379 POINT(42.302758162584 -87.42870294984112) bank140379 +140380 POINT(40.9580441998649 -87.32967575267476) bank140380 +140381 POINT(42.65946999180214 -87.00444645957866) bank140381 +140382 POINT(41.12097297361422 -88.42510812334136) bank140382 +140383 POINT(42.81823048855595 -87.35208961578341) bank140383 +140384 POINT(42.5982761359689 -86.65730017429892) bank140384 +140385 POINT(41.48737816239607 -86.98506895961093) bank140385 +140386 POINT(42.73229914185051 -87.29357559384168) bank140386 +140387 POINT(42.8207741938184 -87.18940082109408) bank140387 +140388 POINT(41.44808624538549 -87.70989932139868) bank140388 +140389 POINT(41.912981737419926 -87.91094135631951) bank140389 +140390 POINT(41.09785033987868 -88.26240259317007) bank140390 +140391 POINT(41.15934122128063 -88.26658305635021) bank140391 +140392 POINT(42.63622830578747 -87.67677658801544) bank140392 +140393 POINT(42.2191513470341 -87.85770930854909) bank140393 +140394 POINT(41.56735200181171 -87.22787974388326) bank140394 +140395 POINT(41.43181282965426 -86.74757983682551) bank140395 +140396 POINT(41.362374036789404 -88.15124596376293) bank140396 +140397 POINT(42.50717622508779 -88.12216986921332) bank140397 +140398 POINT(42.522556111361624 -87.64952171943766) bank140398 +140399 POINT(41.933584500786566 -87.02976433097517) bank140399 +140400 POINT(41.86807376008389 -86.74233186885235) bank140400 +140401 POINT(42.56635558709129 -87.95810162741314) bank140401 +140402 POINT(41.501872429163875 -86.87544497112096) bank140402 +140403 POINT(40.92670338484164 -87.07094042978996) bank140403 +140404 POINT(40.96300508990706 -87.41084180866854) bank140404 +140405 POINT(41.51502653243487 -87.4383189321279) bank140405 +140406 POINT(42.31271001400297 -88.55802540701396) bank140406 +140407 POINT(41.8682256834619 -87.31647996607697) bank140407 +140408 POINT(42.26367762023964 -86.76215488000688) bank140408 +140409 POINT(41.180832018738755 -86.74272431146967) bank140409 +140410 POINT(41.311787534906635 -86.8393196328983) bank140410 +140411 POINT(42.55822935464162 -88.09354036547764) bank140411 +140412 POINT(42.634669185982716 -86.71554715236415) bank140412 +140413 POINT(42.151499500930015 -87.06025813841018) bank140413 +140414 POINT(41.36309250955014 -88.23884926605479) bank140414 +140415 POINT(41.79405934012296 -86.81575963781445) bank140415 +140416 POINT(42.731434688857384 -88.33490138762681) bank140416 +140417 POINT(41.146156567327296 -86.64211758794774) bank140417 +140418 POINT(42.50148614658659 -86.73486730336485) bank140418 +140419 POINT(41.181867131632416 -87.64527007745501) bank140419 +140420 POINT(42.21967699898288 -87.48692541418436) bank140420 +140421 POINT(41.89899268775934 -88.54231194215565) bank140421 +140422 POINT(42.26657867711206 -87.04203146608533) bank140422 +140423 POINT(42.72823999289816 -86.81860128971685) bank140423 +140424 POINT(42.283139566638226 -87.66727071473203) bank140424 +140425 POINT(42.18355643365321 -88.53310631678296) bank140425 +140426 POINT(41.55373359363569 -87.25626866212146) bank140426 +140427 POINT(41.93169230163311 -87.92059063928663) bank140427 +140428 POINT(41.41488751585631 -87.91174295245882) bank140428 +140429 POINT(42.38996256280404 -87.97234398379973) bank140429 +140430 POINT(42.854392925448266 -87.44970365511638) bank140430 +140431 POINT(42.00308044079743 -87.0112118029421) bank140431 +140432 POINT(42.27229318540815 -86.86400758434394) bank140432 +140433 POINT(42.67865829913708 -88.43721254508331) bank140433 +140434 POINT(42.293790179799586 -87.38320397787635) bank140434 +140435 POINT(41.757008795002406 -87.8589000042061) bank140435 +140436 POINT(41.89220677400864 -87.00622258315526) bank140436 +140437 POINT(42.21728423417798 -87.15295252837089) bank140437 +140438 POINT(41.57156696162291 -87.81299662230538) bank140438 +140439 POINT(41.743951163147855 -88.09751856108419) bank140439 +140440 POINT(42.592603878270566 -87.09414297872218) bank140440 +140441 POINT(42.12227710611415 -87.16246368572074) bank140441 +140442 POINT(41.926245512998015 -88.4209357447309) bank140442 +140443 POINT(41.955528609835085 -86.99056979311447) bank140443 +140444 POINT(42.37455938316091 -87.16022349391095) bank140444 +140445 POINT(41.43685942133535 -86.83110565387666) bank140445 +140446 POINT(41.205698928367234 -86.77773649023223) bank140446 +140447 POINT(40.99365395149336 -88.1898396749765) bank140447 +140448 POINT(41.11344563943819 -88.2231671836294) bank140448 +140449 POINT(41.573462195934965 -87.38339323010041) bank140449 +140450 POINT(41.20483717669017 -88.54367401476988) bank140450 +140451 POINT(41.469969825987434 -87.34815545990809) bank140451 +140452 POINT(42.38846071171548 -86.78284968500728) bank140452 +140453 POINT(42.67437701093993 -88.33076847752052) bank140453 +140454 POINT(42.35533237059961 -87.4393266043431) bank140454 +140455 POINT(41.61434188791316 -87.33551569402884) bank140455 +140456 POINT(40.972894423741415 -86.91772802553609) bank140456 +140457 POINT(42.84563036088677 -87.72234583725249) bank140457 +140458 POINT(41.278204242241266 -87.05823569900726) bank140458 +140459 POINT(42.23912152989688 -87.9177636506313) bank140459 +140460 POINT(40.936179082048476 -87.40669303837335) bank140460 +140461 POINT(42.08392009248754 -86.98048529265714) bank140461 +140462 POINT(42.73449536645249 -86.75823856476741) bank140462 +140463 POINT(40.93847609129646 -87.46114383427017) bank140463 +140464 POINT(41.414276860875766 -87.82977736574465) bank140464 +140465 POINT(40.97057770114735 -86.95619664448087) bank140465 +140466 POINT(42.107751871210255 -87.41710319652782) bank140466 +140467 POINT(41.658769073732316 -86.92283585761184) bank140467 +140468 POINT(42.16630463382569 -86.89703197972273) bank140468 +140469 POINT(42.01976039140959 -88.02237115366813) bank140469 +140470 POINT(42.358709038319276 -87.95111524455022) bank140470 +140471 POINT(41.59393778391025 -88.2276443690549) bank140471 +140472 POINT(40.92018388788172 -88.21490545093164) bank140472 +140473 POINT(40.92582294105018 -87.31271675340525) bank140473 +140474 POINT(41.58762419684826 -87.47214155632886) bank140474 +140475 POINT(41.297615874887285 -87.97655356065538) bank140475 +140476 POINT(41.07621541066751 -87.32090047960193) bank140476 +140477 POINT(41.452858319486815 -86.7301363424384) bank140477 +140478 POINT(42.09959887240461 -87.01110814623966) bank140478 +140479 POINT(42.02701211506228 -86.90961755030247) bank140479 +140480 POINT(42.86046559432735 -86.8271925735641) bank140480 +140481 POINT(42.70825507806186 -87.07961817644357) bank140481 +140482 POINT(42.45993147181014 -88.35328169010714) bank140482 +140483 POINT(42.36617706405404 -87.22422518519046) bank140483 +140484 POINT(41.626639567102416 -87.65724530815498) bank140484 +140485 POINT(40.9794317257352 -88.31722527787227) bank140485 +140486 POINT(41.63225813472614 -87.18216315697136) bank140486 +140487 POINT(42.09869313368213 -88.39728477460848) bank140487 +140488 POINT(41.38243887001761 -87.72299151870399) bank140488 +140489 POINT(41.45137175626196 -87.92641336391785) bank140489 +140490 POINT(42.539418241221966 -86.79902780210506) bank140490 +140491 POINT(41.69409843251175 -87.28933612167796) bank140491 +140492 POINT(42.33397530788559 -87.88838719213155) bank140492 +140493 POINT(42.42034412265909 -88.21090724851288) bank140493 +140494 POINT(41.733602015619965 -86.8650726986346) bank140494 +140495 POINT(42.14955866565945 -87.85665483459314) bank140495 +140496 POINT(41.989274133653126 -86.73402927283055) bank140496 +140497 POINT(40.99177454254376 -87.270785093523) bank140497 +140498 POINT(42.360981324456475 -86.84086054959118) bank140498 +140499 POINT(40.952036448260785 -88.15207445566965) bank140499 +140500 POINT(42.206445581230355 -87.84589088705494) bank140500 +140501 POINT(41.90417735381533 -87.59923320658686) bank140501 +140502 POINT(41.449668367714516 -87.6986961397929) bank140502 +140503 POINT(42.01865392647632 -87.03713197034891) bank140503 +140504 POINT(42.326815296255326 -87.50879651739248) bank140504 +140505 POINT(40.99119009180941 -86.66008883475979) bank140505 +140506 POINT(41.931437861013826 -87.5004950708962) bank140506 +140507 POINT(41.84193467243141 -87.29252336169223) bank140507 +140508 POINT(42.671109311984964 -87.70314226921411) bank140508 +140509 POINT(41.939476814133364 -86.72593406787212) bank140509 +140510 POINT(41.05655612967876 -88.00744445649659) bank140510 +140511 POINT(41.32764865176482 -88.1852075167382) bank140511 +140512 POINT(41.68449505789054 -86.7157014493419) bank140512 +140513 POINT(41.90661032043052 -87.04330171978957) bank140513 +140514 POINT(41.01220726766704 -86.94281750086103) bank140514 +140515 POINT(41.752998922627704 -87.03788260903167) bank140515 +140516 POINT(41.55716612965607 -87.50680892239251) bank140516 +140517 POINT(41.989457620061 -87.56277449882104) bank140517 +140518 POINT(42.74156042691391 -88.51295946290423) bank140518 +140519 POINT(41.70310339924843 -87.17884017455853) bank140519 +140520 POINT(41.231798918977276 -87.75287805166954) bank140520 +140521 POINT(41.421549370737615 -88.3764064845958) bank140521 +140522 POINT(41.763945639446625 -87.33863750324731) bank140522 +140523 POINT(41.66806299085652 -87.92549198356406) bank140523 +140524 POINT(42.033517042034184 -87.83714927467638) bank140524 +140525 POINT(41.69075050010344 -88.01205622417187) bank140525 +140526 POINT(41.425287953270605 -87.59565024453802) bank140526 +140527 POINT(41.59465505040895 -88.44893089730793) bank140527 +140528 POINT(41.66204731014097 -87.62203470909692) bank140528 +140529 POINT(41.345808884643404 -87.222099955783) bank140529 +140530 POINT(41.309443394337706 -86.6960292031425) bank140530 +140531 POINT(42.624820031151515 -87.47670530825154) bank140531 +140532 POINT(42.66965726503906 -88.20251471312066) bank140532 +140533 POINT(41.30005430591778 -88.49635810140146) bank140533 +140534 POINT(41.96007199850065 -87.55893450648617) bank140534 +140535 POINT(42.58300087630527 -87.277526037689) bank140535 +140536 POINT(40.95996324412609 -86.88036537503217) bank140536 +140537 POINT(42.231164733979064 -87.24871865371169) bank140537 +140538 POINT(42.77043131410842 -86.66288091866683) bank140538 +140539 POINT(42.51199315348274 -87.12236270935713) bank140539 +140540 POINT(42.397131314214604 -87.23360258256062) bank140540 +140541 POINT(42.36246995661344 -88.45439957778981) bank140541 +140542 POINT(40.94612474492225 -88.10197797699237) bank140542 +140543 POINT(41.819471209491205 -87.00153812671391) bank140543 +140544 POINT(41.22418384788095 -88.49561169967248) bank140544 +140545 POINT(41.33206808264939 -88.22319478409347) bank140545 +140546 POINT(41.31972190012216 -86.83089181932068) bank140546 +140547 POINT(41.522767695032904 -86.74052897429945) bank140547 +140548 POINT(42.805723383988614 -86.71953808203241) bank140548 +140549 POINT(41.10935367939142 -87.9817793463575) bank140549 +140550 POINT(42.664703204445296 -87.59778683078807) bank140550 +140551 POINT(41.75577719296782 -88.00435461798943) bank140551 +140552 POINT(41.99191997405521 -88.41696708482657) bank140552 +140553 POINT(41.44242016595696 -87.30706036720969) bank140553 +140554 POINT(42.348440028024015 -86.67415398820714) bank140554 +140555 POINT(40.91572132483201 -86.68295099070522) bank140555 +140556 POINT(40.92324770294424 -87.94336576954169) bank140556 +140557 POINT(41.96407147007604 -87.53814727238877) bank140557 +140558 POINT(41.2522520776068 -87.45875418095723) bank140558 +140559 POINT(41.01084028421754 -87.83239639368354) bank140559 +140560 POINT(42.655874470881606 -87.7932957891919) bank140560 +140561 POINT(41.0884089073213 -88.59668285956957) bank140561 +140562 POINT(42.33475009263204 -87.4986085239188) bank140562 +140563 POINT(42.017395652150086 -87.14822148209566) bank140563 +140564 POINT(42.387353992310246 -87.31777353496211) bank140564 +140565 POINT(42.5545251331868 -87.71972436439391) bank140565 +140566 POINT(41.2471164090826 -88.25487984045982) bank140566 +140567 POINT(42.20117062321743 -88.19367272914032) bank140567 +140568 POINT(41.86325304846185 -87.44756172227548) bank140568 +140569 POINT(41.158117898713584 -87.30717453277924) bank140569 +140570 POINT(42.266446761411835 -88.24955575702437) bank140570 +140571 POINT(41.49442559223979 -88.52813719680819) bank140571 +140572 POINT(41.780352471045354 -88.36207575259081) bank140572 +140573 POINT(42.64480185590735 -86.75396258774325) bank140573 +140574 POINT(42.38248261157355 -88.51689211140348) bank140574 +140575 POINT(41.78949978968928 -87.8658605381722) bank140575 +140576 POINT(42.54705219176536 -88.36048470707662) bank140576 +140577 POINT(41.529463318504284 -87.0224823210236) bank140577 +140578 POINT(42.59378698349385 -88.19120209361508) bank140578 +140579 POINT(41.23643120568272 -86.72733524789805) bank140579 +140580 POINT(41.65307564018883 -88.55026496575728) bank140580 +140581 POINT(41.68083777779403 -88.2864369946612) bank140581 +140582 POINT(42.43861258504562 -88.1482813134454) bank140582 +140583 POINT(42.688612191558974 -88.12022501667465) bank140583 +140584 POINT(42.388752896069924 -86.9091482390718) bank140584 +140585 POINT(42.157894277063455 -88.53645966488683) bank140585 +140586 POINT(42.08278372278263 -87.78606471884844) bank140586 +140587 POINT(41.47731487083966 -88.5632422634022) bank140587 +140588 POINT(42.41591864922459 -88.55781166782438) bank140588 +140589 POINT(41.845944141408225 -87.96154115621984) bank140589 +140590 POINT(42.81789978009211 -87.20953733702088) bank140590 +140591 POINT(40.938317447781074 -87.01139423836169) bank140591 +140592 POINT(42.69414543902897 -86.97665185644317) bank140592 +140593 POINT(42.297654648936295 -87.23898812431625) bank140593 +140594 POINT(41.910217967905034 -87.04272226923817) bank140594 +140595 POINT(42.206841253949456 -88.3582707830309) bank140595 +140596 POINT(42.12329942237363 -86.86992986767305) bank140596 +140597 POINT(41.916412043750434 -87.79348337058474) bank140597 +140598 POINT(42.119583021878945 -87.01171147350615) bank140598 +140599 POINT(42.47630189887776 -86.79981945793598) bank140599 +140600 POINT(42.763225044514925 -87.73229226266587) bank140600 +140601 POINT(42.80803819133563 -87.79919621824992) bank140601 +140602 POINT(40.91870818090306 -86.83014015031877) bank140602 +140603 POINT(42.00469822005365 -87.55592672889502) bank140603 +140604 POINT(42.6149122954917 -87.48916439036034) bank140604 +140605 POINT(42.80485097096565 -86.86877743087881) bank140605 +140606 POINT(42.53968368268548 -86.96051951811035) bank140606 +140607 POINT(42.303872720077116 -87.74553386714432) bank140607 +140608 POINT(41.0320350904276 -88.48939718922966) bank140608 +140609 POINT(42.154605319068494 -88.2148006155173) bank140609 +140610 POINT(41.772413208060364 -86.84915637317121) bank140610 +140611 POINT(40.95429843960447 -87.69898849777871) bank140611 +140612 POINT(42.437564903806695 -88.25980446785177) bank140612 +140613 POINT(41.943000438301425 -86.95548421191695) bank140613 +140614 POINT(41.50877868426357 -86.88070919583465) bank140614 +140615 POINT(42.49258122769008 -87.87973321812795) bank140615 +140616 POINT(41.4709207419816 -88.43228538321736) bank140616 +140617 POINT(40.999440893829906 -88.14859182500183) bank140617 +140618 POINT(41.5135708456546 -87.50135405482496) bank140618 +140619 POINT(41.80773547325827 -87.5683485092069) bank140619 +140620 POINT(41.40383439004313 -88.01142023569983) bank140620 +140621 POINT(41.89428226317343 -87.43522593172722) bank140621 +140622 POINT(42.83857655639615 -87.43219818106958) bank140622 +140623 POINT(42.8151607922388 -88.2553156795816) bank140623 +140624 POINT(41.379404399840304 -87.7630295367165) bank140624 +140625 POINT(41.115431914976064 -87.07699579034026) bank140625 +140626 POINT(41.200094318079366 -86.92022141235371) bank140626 +140627 POINT(41.145351194229285 -87.51423202663314) bank140627 +140628 POINT(42.696355869519344 -87.5602405828164) bank140628 +140629 POINT(41.09136010907167 -87.38840680202674) bank140629 +140630 POINT(41.428946752108814 -87.68552342886336) bank140630 +140631 POINT(42.86570230922176 -88.069379380774) bank140631 +140632 POINT(41.92092423257782 -88.25339016451979) bank140632 +140633 POINT(42.15043612141129 -87.68160098851637) bank140633 +140634 POINT(41.87817852378147 -87.65203583795076) bank140634 +140635 POINT(41.06959458661759 -88.12692803242551) bank140635 +140636 POINT(42.42231562393509 -87.0858485545954) bank140636 +140637 POINT(41.4956280643543 -86.83228989154917) bank140637 +140638 POINT(42.512813249743196 -88.3389997978185) bank140638 +140639 POINT(41.42984229254966 -88.09257635460595) bank140639 +140640 POINT(41.697976145568795 -87.22138708021947) bank140640 +140641 POINT(41.02980721400235 -88.61398968319548) bank140641 +140642 POINT(41.60381140461115 -88.12235135200018) bank140642 +140643 POINT(41.23940046079441 -87.3734014045596) bank140643 +140644 POINT(42.71343737227675 -88.62187031205366) bank140644 +140645 POINT(41.19355568130119 -87.77455741490938) bank140645 +140646 POINT(40.902879015277136 -87.38725604616558) bank140646 +140647 POINT(42.58329202396876 -87.04659467996083) bank140647 +140648 POINT(42.35173908344785 -88.32753915033123) bank140648 +140649 POINT(41.339877331893604 -88.20261570075853) bank140649 +140650 POINT(41.506208309217804 -87.77909946195365) bank140650 +140651 POINT(42.47524516888676 -87.72042021169754) bank140651 +140652 POINT(41.90575214698423 -87.347969045635) bank140652 +140653 POINT(42.322514877810576 -86.89480127340217) bank140653 +140654 POINT(42.79001063093327 -87.07117734459513) bank140654 +140655 POINT(42.45901726818015 -88.54358204526325) bank140655 +140656 POINT(40.9360484280791 -87.21360378983059) bank140656 +140657 POINT(41.08372502868182 -87.28091238968021) bank140657 +140658 POINT(41.66628042346352 -86.80048368445031) bank140658 +140659 POINT(40.91016998854045 -87.7919451632062) bank140659 +140660 POINT(42.009434733742644 -87.17563733176759) bank140660 +140661 POINT(41.009325455275274 -87.91071293615633) bank140661 +140662 POINT(41.788524665599176 -87.7197099946823) bank140662 +140663 POINT(40.94543766961214 -87.02970275344585) bank140663 +140664 POINT(42.034898936770986 -88.51993414853939) bank140664 +140665 POINT(42.685742914255314 -86.89668049989773) bank140665 +140666 POINT(42.22081237755082 -87.51254208314994) bank140666 +140667 POINT(41.25135061437723 -88.20199143391568) bank140667 +140668 POINT(42.6825634149931 -87.26013180136266) bank140668 +140669 POINT(41.76258480932221 -87.5865628947063) bank140669 +140670 POINT(42.54612389912433 -88.02062262317133) bank140670 +140671 POINT(42.82500168690456 -88.43533716837682) bank140671 +140672 POINT(41.89422481245247 -88.44096269799509) bank140672 +140673 POINT(41.81941416182069 -88.0359644779249) bank140673 +140674 POINT(41.69468818575409 -88.30501049239943) bank140674 +140675 POINT(41.50678422170236 -88.20608209988173) bank140675 +140676 POINT(42.011144953122134 -88.15132476523164) bank140676 +140677 POINT(42.13792091358756 -87.67473840058528) bank140677 +140678 POINT(41.81514507740039 -88.1895488782562) bank140678 +140679 POINT(40.93040168344343 -87.42330111415558) bank140679 +140680 POINT(41.43586922927008 -88.41908440989309) bank140680 +140681 POINT(41.063286711040604 -88.09614099675751) bank140681 +140682 POINT(42.140907572004764 -86.9000304117689) bank140682 +140683 POINT(42.68700858031202 -86.97144167459635) bank140683 +140684 POINT(42.108029600424715 -86.81123207317633) bank140684 +140685 POINT(41.928581375433865 -88.48102573012898) bank140685 +140686 POINT(42.560405069231415 -88.52917654750914) bank140686 +140687 POINT(41.19706638946185 -88.09307604681842) bank140687 +140688 POINT(42.31921379822381 -86.79361282320092) bank140688 +140689 POINT(41.51058223904502 -88.386769153027) bank140689 +140690 POINT(42.273219858551535 -87.83738640597524) bank140690 +140691 POINT(41.062994265382294 -86.72139823382426) bank140691 +140692 POINT(42.48115118933781 -87.67746952944249) bank140692 +140693 POINT(40.966665879854425 -87.12611896286805) bank140693 +140694 POINT(41.14245531266102 -87.06493090204619) bank140694 +140695 POINT(41.853022188082974 -87.91458479879354) bank140695 +140696 POINT(41.29638231734396 -86.73733053212436) bank140696 +140697 POINT(41.87644216337378 -86.65364724178312) bank140697 +140698 POINT(42.815518360895226 -86.90068944433632) bank140698 +140699 POINT(41.540700692682314 -88.39650999240263) bank140699 +140700 POINT(42.46720261869084 -88.32409484591902) bank140700 +140701 POINT(42.04387563716236 -86.73138210121334) bank140701 +140702 POINT(42.84824269243959 -86.78704681622574) bank140702 +140703 POINT(41.5085592190455 -88.49306943948807) bank140703 +140704 POINT(42.15893227514579 -87.17405880171343) bank140704 +140705 POINT(41.03741510207313 -87.24650223606233) bank140705 +140706 POINT(41.48442039381079 -88.33133504328832) bank140706 +140707 POINT(42.77542793749854 -87.92403063029357) bank140707 +140708 POINT(41.655548060160186 -87.90650551475174) bank140708 +140709 POINT(41.56504192292108 -87.12239049880627) bank140709 +140710 POINT(42.4584839428439 -88.09934831068493) bank140710 +140711 POINT(41.66193691166789 -88.48007077949948) bank140711 +140712 POINT(42.79994131612801 -88.31556838911611) bank140712 +140713 POINT(42.78956865009917 -88.49868649767046) bank140713 +140714 POINT(41.31980921963648 -87.51982073105549) bank140714 +140715 POINT(41.884247109468255 -87.78464749154729) bank140715 +140716 POINT(41.09629572878621 -87.76999463062913) bank140716 +140717 POINT(41.601534846967375 -87.54582189738088) bank140717 +140718 POINT(40.983386405502955 -87.90930378278571) bank140718 +140719 POINT(41.50364866766639 -87.14108563143226) bank140719 +140720 POINT(41.497719178219455 -88.3738597291198) bank140720 +140721 POINT(42.30934098966899 -88.62866564785782) bank140721 +140722 POINT(42.418501123237206 -86.82990032977575) bank140722 +140723 POINT(40.8945030653328 -87.47330155056125) bank140723 +140724 POINT(42.35281267818498 -88.53353376410175) bank140724 +140725 POINT(41.85948212533365 -87.98699350625853) bank140725 +140726 POINT(41.34394982343926 -88.42211246595164) bank140726 +140727 POINT(41.36348399295627 -86.8265880766373) bank140727 +140728 POINT(41.35226225617914 -87.1564748232854) bank140728 +140729 POINT(41.895231382805356 -87.3636130834541) bank140729 +140730 POINT(42.58500272707712 -86.82685700949396) bank140730 +140731 POINT(42.400495912363034 -87.98199037728703) bank140731 +140732 POINT(42.8087565032032 -88.44626262374985) bank140732 +140733 POINT(42.504567309551824 -86.89926675254236) bank140733 +140734 POINT(41.08437453969764 -87.60632948186542) bank140734 +140735 POINT(42.8398486099634 -88.41938710538219) bank140735 +140736 POINT(41.91663858594685 -88.1347780718217) bank140736 +140737 POINT(41.160291952404364 -86.80383901626062) bank140737 +140738 POINT(42.557492074829824 -87.24831175752922) bank140738 +140739 POINT(41.040490693668836 -87.83483983599498) bank140739 +140740 POINT(41.97947883024052 -86.98078763486568) bank140740 +140741 POINT(42.51939996593243 -88.53882191198896) bank140741 +140742 POINT(41.305823374557015 -86.65863411934177) bank140742 +140743 POINT(42.61086098744752 -88.57826152929846) bank140743 +140744 POINT(42.53291480821801 -87.90056498078803) bank140744 +140745 POINT(41.30768493936131 -88.18496070658742) bank140745 +140746 POINT(41.30314608727374 -87.34539433390069) bank140746 +140747 POINT(41.20654997392927 -87.60631603216785) bank140747 +140748 POINT(41.33332888650928 -87.54784266346981) bank140748 +140749 POINT(42.52688481538778 -88.31370422084126) bank140749 +140750 POINT(41.17085533160402 -86.74580929242514) bank140750 +140751 POINT(41.03199603346165 -87.14699846797532) bank140751 +140752 POINT(42.5891446459705 -88.24522552702152) bank140752 +140753 POINT(41.67747307455128 -86.64865321277051) bank140753 +140754 POINT(41.02947921876094 -88.30794901450663) bank140754 +140755 POINT(41.541437733123566 -88.14055993056141) bank140755 +140756 POINT(41.48539312824553 -87.51191150936009) bank140756 +140757 POINT(41.74999213686165 -87.11807563886794) bank140757 +140758 POINT(42.3914724714628 -87.64793435622315) bank140758 +140759 POINT(42.04965621626587 -87.24071182289947) bank140759 +140760 POINT(41.84776971505863 -87.46385981028838) bank140760 +140761 POINT(41.65534537121908 -88.07860707951835) bank140761 +140762 POINT(42.86080487673798 -86.8354437615705) bank140762 +140763 POINT(42.84996056680004 -87.73406329904859) bank140763 +140764 POINT(40.87819016358145 -88.42206806543848) bank140764 +140765 POINT(42.11191873974759 -87.64028046937054) bank140765 +140766 POINT(41.1862886421363 -88.40600091821912) bank140766 +140767 POINT(42.51750167846632 -87.48170844622176) bank140767 +140768 POINT(42.51718880971653 -87.76827311848407) bank140768 +140769 POINT(41.78340294416659 -87.19260906765027) bank140769 +140770 POINT(41.25701491950966 -87.11656593922442) bank140770 +140771 POINT(41.242804724940356 -87.1477625568511) bank140771 +140772 POINT(41.96721868360456 -87.67154601660178) bank140772 +140773 POINT(41.729500673659395 -87.68339823354985) bank140773 +140774 POINT(42.177220335393656 -87.71638108439346) bank140774 +140775 POINT(41.97109642962348 -88.10208120084306) bank140775 +140776 POINT(41.178090757440216 -88.02861517497512) bank140776 +140777 POINT(41.8048021428773 -87.98053616068489) bank140777 +140778 POINT(41.80977698976218 -88.55355139761839) bank140778 +140779 POINT(42.69613766123702 -87.3231874202825) bank140779 +140780 POINT(42.5507785432925 -86.86223810654262) bank140780 +140781 POINT(40.983536584905266 -88.29996201319126) bank140781 +140782 POINT(42.60629834589291 -87.0675258777426) bank140782 +140783 POINT(42.56995607308038 -88.18202519561652) bank140783 +140784 POINT(41.71452186239625 -88.43961300148597) bank140784 +140785 POINT(41.25905794791946 -86.80719051268552) bank140785 +140786 POINT(40.90998608124039 -88.07135553651366) bank140786 +140787 POINT(42.476908388689644 -87.57419830504206) bank140787 +140788 POINT(40.878991066070085 -87.1675432343159) bank140788 +140789 POINT(41.337706674053045 -88.35208489529617) bank140789 +140790 POINT(42.12492020184239 -87.3049738141543) bank140790 +140791 POINT(41.77474673341207 -88.06169005325005) bank140791 +140792 POINT(41.25378007815484 -86.99392576161337) bank140792 +140793 POINT(41.35024251146301 -87.03259310658544) bank140793 +140794 POINT(42.084535841517095 -88.13977256263537) bank140794 +140795 POINT(41.527401725990465 -87.13312810264203) bank140795 +140796 POINT(42.692459593096714 -86.72058162959517) bank140796 +140797 POINT(42.306668179449765 -88.42890343177345) bank140797 +140798 POINT(40.95795546772683 -86.85072764571558) bank140798 +140799 POINT(41.700917754936214 -88.07730356589015) bank140799 +140800 POINT(41.00544204554809 -87.36114935657085) bank140800 +140801 POINT(42.64535273635603 -87.40251528025811) bank140801 +140802 POINT(41.439684196483896 -88.14851453315556) bank140802 +140803 POINT(41.2463386558759 -87.71174796368885) bank140803 +140804 POINT(41.73508321089347 -87.79617052369568) bank140804 +140805 POINT(42.73979597428614 -87.60265143716371) bank140805 +140806 POINT(42.749688272449596 -87.32721004703129) bank140806 +140807 POINT(41.30004073884563 -86.94249144793312) bank140807 +140808 POINT(42.55798892374142 -87.05741695951826) bank140808 +140809 POINT(41.7204607643351 -88.45511379071684) bank140809 +140810 POINT(41.442162062491334 -88.36910055290264) bank140810 +140811 POINT(41.18723171805149 -87.21430719378392) bank140811 +140812 POINT(41.13667844813418 -87.71209938301513) bank140812 +140813 POINT(42.26199391007264 -88.54254185117372) bank140813 +140814 POINT(41.94191931509373 -87.59899674196735) bank140814 +140815 POINT(41.42267274781198 -87.06904656591614) bank140815 +140816 POINT(42.447995746 -86.76553596833654) bank140816 +140817 POINT(41.53798098539809 -88.22006216488964) bank140817 +140818 POINT(42.26416190955287 -87.80419755931885) bank140818 +140819 POINT(42.371540519500115 -88.00843201843485) bank140819 +140820 POINT(42.62691782294769 -88.23929499551394) bank140820 +140821 POINT(42.616477193805046 -87.1888528148379) bank140821 +140822 POINT(41.23837483964614 -88.47779925313547) bank140822 +140823 POINT(41.472632574840986 -88.57965155764329) bank140823 +140824 POINT(41.459400851022494 -87.91157612653622) bank140824 +140825 POINT(42.21780358179904 -88.34113226133393) bank140825 +140826 POINT(41.260017939562246 -88.14707277901927) bank140826 +140827 POINT(41.24092223629169 -87.64520289065536) bank140827 +140828 POINT(42.388172900002026 -88.05889256314506) bank140828 +140829 POINT(41.460320539072434 -86.81465137420898) bank140829 +140830 POINT(40.99166180296496 -87.1430134480181) bank140830 +140831 POINT(42.84886550989486 -88.35279796690979) bank140831 +140832 POINT(42.303545003257476 -88.4530851221691) bank140832 +140833 POINT(41.477239096902196 -87.67692860433792) bank140833 +140834 POINT(42.12548732853353 -88.25494768706585) bank140834 +140835 POINT(42.441385070862815 -87.69987589427824) bank140835 +140836 POINT(42.75965758117585 -87.53530607841579) bank140836 +140837 POINT(41.90757690202389 -87.17664049863556) bank140837 +140838 POINT(42.83662251987399 -86.92291242076868) bank140838 +140839 POINT(41.386973617910265 -88.21842455416315) bank140839 +140840 POINT(41.186139226688915 -88.44323419083743) bank140840 +140841 POINT(41.97321609983267 -87.35357237210435) bank140841 +140842 POINT(42.69773640831798 -87.47784005734582) bank140842 +140843 POINT(42.32554915825301 -88.29960716616759) bank140843 +140844 POINT(42.47289666282964 -86.70555361039155) bank140844 +140845 POINT(41.65004432784958 -87.08717319264989) bank140845 +140846 POINT(42.33168570879583 -87.10752150641153) bank140846 +140847 POINT(42.595707991762 -86.77334694802038) bank140847 +140848 POINT(41.592844555748805 -86.88232214084043) bank140848 +140849 POINT(41.54162296539532 -87.42466868091783) bank140849 +140850 POINT(41.64039007286838 -87.64233296779236) bank140850 +140851 POINT(41.59462546540639 -86.78186509423368) bank140851 +140852 POINT(42.415648993803245 -87.11354417562173) bank140852 +140853 POINT(41.77987616381477 -87.75792629935252) bank140853 +140854 POINT(42.49439174339242 -88.52854028031219) bank140854 +140855 POINT(42.761726464324205 -87.6092916718343) bank140855 +140856 POINT(42.52353762912016 -86.95188308104711) bank140856 +140857 POINT(42.52662405291931 -87.4307742624586) bank140857 +140858 POINT(41.35700092038212 -88.22977150396177) bank140858 +140859 POINT(42.21742472372647 -88.30847379358586) bank140859 +140860 POINT(42.861970981362404 -86.84466137387105) bank140860 +140861 POINT(42.508100360745566 -87.67572243549169) bank140861 +140862 POINT(42.875736082575195 -88.48181371986566) bank140862 +140863 POINT(42.05324937134662 -88.52585135658504) bank140863 +140864 POINT(42.67437991454387 -87.92656418682786) bank140864 +140865 POINT(41.993870273771414 -87.57884324527143) bank140865 +140866 POINT(41.37699398339303 -86.7642490206225) bank140866 +140867 POINT(42.46438644692727 -88.17392944783097) bank140867 +140868 POINT(41.675002117739545 -86.80396592436406) bank140868 +140869 POINT(40.966524369270275 -86.80233310809) bank140869 +140870 POINT(42.30325798293043 -88.17125500842472) bank140870 +140871 POINT(42.68256397624076 -86.99656349259402) bank140871 +140872 POINT(42.38597450743844 -88.55053248594136) bank140872 +140873 POINT(42.49990927455133 -87.03500796385948) bank140873 +140874 POINT(42.169768359306474 -88.29739983722828) bank140874 +140875 POINT(41.31893374115023 -87.81495092806516) bank140875 +140876 POINT(42.47519600969847 -88.47465575369722) bank140876 +140877 POINT(41.01962131310079 -87.8914943208273) bank140877 +140878 POINT(42.06146640629359 -87.71646431536585) bank140878 +140879 POINT(42.577131960684646 -87.01498959081414) bank140879 +140880 POINT(41.713369176918356 -87.11598109547812) bank140880 +140881 POINT(41.22730972760714 -87.3272884834454) bank140881 +140882 POINT(41.34078117331472 -87.28261047492921) bank140882 +140883 POINT(40.99575916028889 -86.99415022845496) bank140883 +140884 POINT(42.34684298549625 -88.51367579799799) bank140884 +140885 POINT(41.2539392620228 -87.33317576805868) bank140885 +140886 POINT(41.61978907627143 -86.67928728758173) bank140886 +140887 POINT(42.42238691319014 -87.49262141456438) bank140887 +140888 POINT(42.50045234072073 -86.96226088803319) bank140888 +140889 POINT(41.921537678367436 -86.84943980829131) bank140889 +140890 POINT(41.98888116006434 -87.65361036867424) bank140890 +140891 POINT(42.206180626543144 -88.05803863423564) bank140891 +140892 POINT(40.936627502993716 -88.15591515093179) bank140892 +140893 POINT(41.148942654687794 -87.5741929467555) bank140893 +140894 POINT(41.50905119839041 -87.16884274195509) bank140894 +140895 POINT(42.08805104081681 -87.2175265960787) bank140895 +140896 POINT(42.81522028627937 -87.72042633187162) bank140896 +140897 POINT(41.027553774671766 -87.62314731171143) bank140897 +140898 POINT(40.97902060163476 -87.52262727771773) bank140898 +140899 POINT(41.09703445494896 -87.18153610356987) bank140899 +140900 POINT(42.00981434150711 -87.985910968412) bank140900 +140901 POINT(42.19665194907726 -87.01492411172468) bank140901 +140902 POINT(42.02763361479192 -88.36181360873222) bank140902 +140903 POINT(41.618992274971575 -87.17075142993633) bank140903 +140904 POINT(42.1249625254353 -87.6488996500008) bank140904 +140905 POINT(42.036263261018455 -86.8100324920013) bank140905 +140906 POINT(41.85520303827361 -86.83931080036882) bank140906 +140907 POINT(41.25760853499751 -88.1449367857261) bank140907 +140908 POINT(42.07378103744424 -87.72982780073325) bank140908 +140909 POINT(41.95546123611553 -87.61694994321533) bank140909 +140910 POINT(41.4835008501228 -87.60179491022276) bank140910 +140911 POINT(42.45344545807008 -88.37616600656493) bank140911 +140912 POINT(41.975536433217535 -86.66278207712955) bank140912 +140913 POINT(42.35947052503165 -87.22916066786259) bank140913 +140914 POINT(42.1777803092771 -87.52710631573447) bank140914 +140915 POINT(41.15675449788515 -87.47821184869828) bank140915 +140916 POINT(42.62821262468457 -88.17303499314025) bank140916 +140917 POINT(42.345240987593414 -87.05694892418339) bank140917 +140918 POINT(41.97725486967962 -87.37080829971568) bank140918 +140919 POINT(41.27277661105736 -86.88807804175356) bank140919 +140920 POINT(41.69302152348534 -87.90474787476747) bank140920 +140921 POINT(42.444211009832195 -88.00927052437194) bank140921 +140922 POINT(41.54522586860628 -86.73884243367242) bank140922 +140923 POINT(41.25300489849694 -87.63419211776393) bank140923 +140924 POINT(41.98227025148999 -87.35291592489064) bank140924 +140925 POINT(41.02367840186218 -86.68727317025511) bank140925 +140926 POINT(41.69571895229742 -86.99177276391462) bank140926 +140927 POINT(42.23023187709065 -86.8041752265156) bank140927 +140928 POINT(41.88268680984857 -88.3826603396522) bank140928 +140929 POINT(40.931390562602985 -88.25624269923077) bank140929 +140930 POINT(42.14629134095838 -88.30781707783908) bank140930 +140931 POINT(42.60298488215613 -88.14576668241693) bank140931 +140932 POINT(42.80951696986051 -87.89922551998592) bank140932 +140933 POINT(41.6885354048626 -86.95698019292978) bank140933 +140934 POINT(42.66898979871945 -86.80391513094422) bank140934 +140935 POINT(42.27445182884448 -87.55111057064867) bank140935 +140936 POINT(42.282762500885966 -88.50616310319343) bank140936 +140937 POINT(41.24601500856827 -86.92865875159596) bank140937 +140938 POINT(41.77318074900432 -87.7158579573325) bank140938 +140939 POINT(41.521598149388645 -86.79055443340539) bank140939 +140940 POINT(42.28595046349352 -86.83190882083338) bank140940 +140941 POINT(41.092165185721356 -87.72997689321537) bank140941 +140942 POINT(41.53879008554348 -86.9375951057546) bank140942 +140943 POINT(42.51601311455842 -86.94268006288779) bank140943 +140944 POINT(41.660736784809174 -88.36598003667861) bank140944 +140945 POINT(40.888016718517775 -88.00005034914213) bank140945 +140946 POINT(42.10328642414809 -87.09427749655114) bank140946 +140947 POINT(42.38914938616907 -86.9331973239934) bank140947 +140948 POINT(42.71193191245658 -88.04709475321468) bank140948 +140949 POINT(42.43533644584812 -87.48963561087773) bank140949 +140950 POINT(41.26862250830967 -86.66638420183084) bank140950 +140951 POINT(42.84045214980791 -87.73260347516877) bank140951 +140952 POINT(41.11179059438857 -87.83971936911526) bank140952 +140953 POINT(40.96662286683259 -87.69165619965464) bank140953 +140954 POINT(42.87093851729623 -87.1662422748731) bank140954 +140955 POINT(42.213689470883025 -86.86696470439688) bank140955 +140956 POINT(40.98172096585457 -88.51888366334015) bank140956 +140957 POINT(42.03184161275814 -87.12609362750979) bank140957 +140958 POINT(42.686201426291156 -88.26593232354985) bank140958 +140959 POINT(42.399104090567036 -87.86298769474624) bank140959 +140960 POINT(41.356208274577604 -86.66976606857119) bank140960 +140961 POINT(42.3399425159047 -88.31466703807003) bank140961 +140962 POINT(41.97545034400048 -87.97927683744153) bank140962 +140963 POINT(42.58114481825579 -87.72676487559181) bank140963 +140964 POINT(41.74678477577625 -88.33201741469715) bank140964 +140965 POINT(40.889652235343966 -87.7326974230719) bank140965 +140966 POINT(41.989910135072606 -88.56290111312609) bank140966 +140967 POINT(42.81299663512418 -88.56838715202599) bank140967 +140968 POINT(40.89053585327187 -86.89716110848556) bank140968 +140969 POINT(42.60014085408219 -88.04219686492974) bank140969 +140970 POINT(41.00493486844038 -88.36082451491528) bank140970 +140971 POINT(41.83351204192865 -87.77680008480559) bank140971 +140972 POINT(42.65356131723114 -87.95913953768944) bank140972 +140973 POINT(40.91907606947583 -88.2707490505772) bank140973 +140974 POINT(41.52865590479644 -88.41994530063357) bank140974 +140975 POINT(41.85818784019801 -88.4938741328542) bank140975 +140976 POINT(42.27605976056595 -87.56583894594623) bank140976 +140977 POINT(42.61560161964779 -87.71089087459018) bank140977 +140978 POINT(41.48566383024026 -88.37032271295337) bank140978 +140979 POINT(41.765434861470105 -88.29677124133724) bank140979 +140980 POINT(42.79928321952437 -87.91424930135244) bank140980 +140981 POINT(41.629513012515716 -87.29887629262994) bank140981 +140982 POINT(41.63533083753938 -88.14596109966452) bank140982 +140983 POINT(42.31069267392738 -88.04498580072257) bank140983 +140984 POINT(42.474545504055364 -87.34435176945374) bank140984 +140985 POINT(41.22363166967448 -87.58397887952053) bank140985 +140986 POINT(40.886890364982186 -88.21329616763641) bank140986 +140987 POINT(42.51320767799779 -87.45216295042201) bank140987 +140988 POINT(41.530623085815435 -88.26207668145234) bank140988 +140989 POINT(42.19215775017389 -87.79313486619414) bank140989 +140990 POINT(42.45301070447543 -88.48829326539654) bank140990 +140991 POINT(42.78034481978523 -88.22596502652263) bank140991 +140992 POINT(42.745124664681995 -86.92497589630786) bank140992 +140993 POINT(41.56018368021547 -88.41926718829748) bank140993 +140994 POINT(42.843822583749706 -86.7645651496776) bank140994 +140995 POINT(42.20075998023035 -87.24005620557746) bank140995 +140996 POINT(42.548544484725724 -87.65809241052877) bank140996 +140997 POINT(42.26952467113123 -87.04521479950967) bank140997 +140998 POINT(42.70010990507589 -86.96236992941462) bank140998 +140999 POINT(41.63960943504148 -87.95364109047476) bank140999 +141000 POINT(42.21110560868229 -86.70791163181647) bank141000 +141001 POINT(41.75287221228763 -88.14289028366593) bank141001 +141002 POINT(41.95812520813511 -88.61602890651253) bank141002 +141003 POINT(42.31168511247595 -88.08419017696735) bank141003 +141004 POINT(42.129993259758294 -87.05785155187611) bank141004 +141005 POINT(41.016400072370416 -87.61677856230962) bank141005 +141006 POINT(41.934627519764206 -88.54441332409392) bank141006 +141007 POINT(42.71726392697657 -88.43710134259562) bank141007 +141008 POINT(41.13796684184357 -87.58728538337623) bank141008 +141009 POINT(40.8946727550798 -88.36284890180089) bank141009 +141010 POINT(41.53775311923277 -88.15564492090957) bank141010 +141011 POINT(42.56360061560545 -88.3613333112208) bank141011 +141012 POINT(41.01156067305785 -86.93048308326786) bank141012 +141013 POINT(42.24590302644465 -87.86175323358941) bank141013 +141014 POINT(42.86780849721073 -87.95802388644229) bank141014 +141015 POINT(42.5444686601373 -87.55140846536418) bank141015 +141016 POINT(42.668845195893354 -86.6299328261552) bank141016 +141017 POINT(42.00521119954386 -87.61592141293404) bank141017 +141018 POINT(41.80487242371343 -88.25079609633228) bank141018 +141019 POINT(41.17456208388911 -86.77798307586383) bank141019 +141020 POINT(41.6054735656179 -87.32707630678523) bank141020 +141021 POINT(42.53511237236631 -87.61817122881199) bank141021 +141022 POINT(42.44367754814216 -88.48231792496067) bank141022 +141023 POINT(40.9987562467202 -86.63047066433664) bank141023 +141024 POINT(42.68215010810998 -88.3444751653758) bank141024 +141025 POINT(41.376414946161525 -86.77420966997332) bank141025 +141026 POINT(41.0963359022501 -88.3202243028584) bank141026 +141027 POINT(42.31998020098664 -87.63254259078333) bank141027 +141028 POINT(40.99697109844262 -86.94769568631318) bank141028 +141029 POINT(42.2902656761144 -87.40455878275141) bank141029 +141030 POINT(42.124742690465375 -87.00538885262652) bank141030 +141031 POINT(42.581612745930414 -86.63869750408534) bank141031 +141032 POINT(41.0597149099777 -87.52501695136635) bank141032 +141033 POINT(42.43768371413222 -87.55807292275856) bank141033 +141034 POINT(42.144171631483374 -86.75997961666171) bank141034 +141035 POINT(42.137437276901025 -88.0901461666583) bank141035 +141036 POINT(41.31005446382023 -87.61817633562242) bank141036 +141037 POINT(42.19425226586019 -86.81202475353578) bank141037 +141038 POINT(41.13739667528125 -88.62627314743243) bank141038 +141039 POINT(42.20196152842607 -88.5101410399304) bank141039 +141040 POINT(42.81755357339273 -87.57592889225124) bank141040 +141041 POINT(41.699105666547325 -88.60078953323826) bank141041 +141042 POINT(42.767109759609674 -88.42760969034853) bank141042 +141043 POINT(42.58116669068211 -86.93477335937746) bank141043 +141044 POINT(42.00905377115628 -87.10463609116889) bank141044 +141045 POINT(41.59482119526731 -88.01901264040097) bank141045 +141046 POINT(41.90618391351063 -87.43065447849784) bank141046 +141047 POINT(42.34232792423616 -88.03274851697076) bank141047 +141048 POINT(41.526331735330125 -87.8725316258645) bank141048 +141049 POINT(41.79280676463739 -88.10512554507824) bank141049 +141050 POINT(42.40112603301467 -87.91801498804156) bank141050 +141051 POINT(42.57963650854258 -87.84179365188484) bank141051 +141052 POINT(42.14158389636381 -88.2762476815636) bank141052 +141053 POINT(41.77999444812694 -87.17320046361851) bank141053 +141054 POINT(42.330478127321825 -87.95719434278386) bank141054 +141055 POINT(41.47831271790605 -88.46993255757494) bank141055 +141056 POINT(41.479523763240515 -88.30652048277757) bank141056 +141057 POINT(41.57816181881814 -88.3507829730838) bank141057 +141058 POINT(41.47501091512986 -88.02687090173312) bank141058 +141059 POINT(42.21233238230026 -88.30123314921343) bank141059 +141060 POINT(42.27981030828335 -87.23876015870736) bank141060 +141061 POINT(41.710911711809096 -86.67326350244151) bank141061 +141062 POINT(42.62436191727808 -88.07521555438825) bank141062 +141063 POINT(41.1248278851823 -87.8024737407723) bank141063 +141064 POINT(42.561549524289426 -88.53732408707215) bank141064 +141065 POINT(42.80495566821671 -87.22203526011593) bank141065 +141066 POINT(42.759764524966506 -86.69979590067548) bank141066 +141067 POINT(41.44862764647247 -88.43839893573751) bank141067 +141068 POINT(41.03049275392733 -87.7964871568838) bank141068 +141069 POINT(42.760053239237834 -87.5487204667442) bank141069 +141070 POINT(41.10485061366027 -88.2477336461066) bank141070 +141071 POINT(42.365077020380475 -87.13327544588486) bank141071 +141072 POINT(42.17716313970722 -87.1196548460036) bank141072 +141073 POINT(42.03763328641413 -87.97377497679977) bank141073 +141074 POINT(41.76912351847423 -87.32727145502248) bank141074 +141075 POINT(42.182446079404485 -87.18489842496705) bank141075 +141076 POINT(41.403625818978036 -88.42308654146908) bank141076 +141077 POINT(41.84189961077487 -87.40515711517678) bank141077 +141078 POINT(41.22303644929152 -87.16683929211662) bank141078 +141079 POINT(41.20449972129228 -86.99448644158736) bank141079 +141080 POINT(40.93272214651875 -86.69264204733541) bank141080 +141081 POINT(42.45312513545931 -88.21862507829894) bank141081 +141082 POINT(42.616789744372674 -87.43268763824364) bank141082 +141083 POINT(40.95568349224151 -88.24311571281162) bank141083 +141084 POINT(41.76068727148855 -87.77163664591997) bank141084 +141085 POINT(42.22369106975745 -88.09481123775096) bank141085 +141086 POINT(40.88918845918613 -87.91396531770778) bank141086 +141087 POINT(41.58673775170322 -87.60847268893103) bank141087 +141088 POINT(42.013119351154224 -88.41642654890116) bank141088 +141089 POINT(41.308925889888464 -87.99456900767143) bank141089 +141090 POINT(42.12142518807844 -87.07284223636107) bank141090 +141091 POINT(42.21569921728515 -87.00658469741707) bank141091 +141092 POINT(42.26101815408591 -88.40284675168738) bank141092 +141093 POINT(41.69622968979964 -87.37063975028) bank141093 +141094 POINT(42.73732798635961 -87.35125507791601) bank141094 +141095 POINT(42.49631060372979 -88.56025632466907) bank141095 +141096 POINT(41.4689553061306 -88.01183406354997) bank141096 +141097 POINT(41.042001460669184 -87.7391202195128) bank141097 +141098 POINT(41.381158987017315 -87.83251898150853) bank141098 +141099 POINT(42.147282456045275 -88.23996002236981) bank141099 +141100 POINT(41.67257103746881 -88.20012771069403) bank141100 +141101 POINT(41.973729412692364 -87.22628989370784) bank141101 +141102 POINT(42.8424107288378 -87.41085687238927) bank141102 +141103 POINT(42.15392696457521 -88.29347290115003) bank141103 +141104 POINT(41.058677274508184 -88.4208494444652) bank141104 +141105 POINT(41.69487572590155 -87.14549523686854) bank141105 +141106 POINT(41.636594774192645 -86.71654631836778) bank141106 +141107 POINT(41.22703912897292 -86.64634282893999) bank141107 +141108 POINT(41.5082648240325 -88.59917233830159) bank141108 +141109 POINT(42.64684479236026 -86.69449078081905) bank141109 +141110 POINT(41.81927745325322 -86.87865657849315) bank141110 +141111 POINT(41.28560469361693 -87.40528429536714) bank141111 +141112 POINT(42.465077147586115 -86.80801298925869) bank141112 +141113 POINT(41.309964982096844 -87.22602095748546) bank141113 +141114 POINT(42.61682613377474 -87.14532790768443) bank141114 +141115 POINT(41.39033538518069 -88.43741709363853) bank141115 +141116 POINT(42.69087285957489 -86.64263586453964) bank141116 +141117 POINT(42.58933814973439 -87.7456219059031) bank141117 +141118 POINT(41.601798705915755 -88.22037393512097) bank141118 +141119 POINT(42.078868104057555 -88.57529555219554) bank141119 +141120 POINT(42.52434623109719 -88.07375222406098) bank141120 +141121 POINT(41.2687051420868 -87.03523661461175) bank141121 +141122 POINT(41.43088132987482 -87.18394363898332) bank141122 +141123 POINT(41.75390229278319 -88.26321991689142) bank141123 +141124 POINT(42.81363603871947 -86.68451912560724) bank141124 +141125 POINT(41.22898531828561 -88.07750955684354) bank141125 +141126 POINT(42.00974298553384 -88.31919584677178) bank141126 +141127 POINT(42.75004886695006 -88.28204471327042) bank141127 +141128 POINT(41.82892447410506 -88.33277053481174) bank141128 +141129 POINT(41.607104639786364 -88.59255848301702) bank141129 +141130 POINT(41.16296187379987 -86.70337353922513) bank141130 +141131 POINT(42.04354238733772 -88.44504670223716) bank141131 +141132 POINT(41.97788666643169 -86.94742667531546) bank141132 +141133 POINT(41.02211352671085 -87.19445967448371) bank141133 +141134 POINT(42.5991865860102 -86.98751644271516) bank141134 +141135 POINT(41.977092258292295 -87.72843508607633) bank141135 +141136 POINT(41.05111282449228 -88.42814262278846) bank141136 +141137 POINT(41.22159761855825 -87.64340188254651) bank141137 +141138 POINT(41.47777234455372 -87.26571547141313) bank141138 +141139 POINT(42.87398530609231 -86.67169913064251) bank141139 +141140 POINT(42.346584016209974 -87.99895670467451) bank141140 +141141 POINT(42.13042365246915 -87.20848060877934) bank141141 +141142 POINT(41.51639333962248 -87.56865496802514) bank141142 +141143 POINT(41.736702312309816 -88.17653906004392) bank141143 +141144 POINT(41.02024827435159 -87.27015565613738) bank141144 +141145 POINT(42.68816673119085 -87.45336864283793) bank141145 +141146 POINT(41.54753305361143 -87.66517790171595) bank141146 +141147 POINT(41.00275224822557 -87.15980424822148) bank141147 +141148 POINT(41.635988189432375 -87.41547162318533) bank141148 +141149 POINT(41.9472002162769 -87.58702765358046) bank141149 +141150 POINT(41.15527129116375 -88.36395577491767) bank141150 +141151 POINT(40.91855464710462 -87.94400048213342) bank141151 +141152 POINT(41.2073948102153 -87.5576129080863) bank141152 +141153 POINT(42.26768774413306 -88.40656117048273) bank141153 +141154 POINT(42.61717266367537 -87.98277658080015) bank141154 +141155 POINT(42.597814472256346 -87.70483730452736) bank141155 +141156 POINT(42.06489161854446 -87.2966892189563) bank141156 +141157 POINT(41.093575747319775 -88.39121725520818) bank141157 +141158 POINT(42.18626114392963 -87.00274468323232) bank141158 +141159 POINT(41.683978555571684 -87.88541471299558) bank141159 +141160 POINT(41.21861281060449 -86.82461477320695) bank141160 +141161 POINT(41.83833901711807 -87.8482403788997) bank141161 +141162 POINT(41.02928848977867 -87.55916028947888) bank141162 +141163 POINT(42.17267715930636 -87.89811105976501) bank141163 +141164 POINT(41.76879072121446 -88.45291906755162) bank141164 +141165 POINT(40.97851830879956 -88.12699637381303) bank141165 +141166 POINT(41.88351753056482 -87.17984017057142) bank141166 +141167 POINT(42.56927063350102 -86.81036514583008) bank141167 +141168 POINT(42.080728704339066 -88.37059167036483) bank141168 +141169 POINT(41.41007477143691 -87.45929090814546) bank141169 +141170 POINT(40.96102028279996 -87.84893107018335) bank141170 +141171 POINT(41.71789373525293 -87.64640890812612) bank141171 +141172 POINT(42.5347612019199 -87.38624534918492) bank141172 +141173 POINT(41.26918830650007 -87.72398854680515) bank141173 +141174 POINT(42.270070612281174 -88.49852560667088) bank141174 +141175 POINT(42.64241307194672 -86.8482840043784) bank141175 +141176 POINT(41.508691244549645 -87.62945295451021) bank141176 +141177 POINT(41.006790284523724 -87.71634593785517) bank141177 +141178 POINT(42.27806737058086 -87.90750888280922) bank141178 +141179 POINT(40.934165471919215 -88.50170254852793) bank141179 +141180 POINT(42.24928198932659 -88.40996321763032) bank141180 +141181 POINT(41.76383704567736 -88.57944004606797) bank141181 +141182 POINT(41.01488116352828 -87.27115163690567) bank141182 +141183 POINT(42.569579395806265 -87.20333669367275) bank141183 +141184 POINT(41.89023590222885 -87.18201196652838) bank141184 +141185 POINT(41.81542903643786 -87.19934477206513) bank141185 +141186 POINT(41.62434144310095 -88.07849343877574) bank141186 +141187 POINT(41.61590320164527 -88.27651648107428) bank141187 +141188 POINT(42.371550319601596 -86.8806152048011) bank141188 +141189 POINT(41.38506078386892 -88.555746496537) bank141189 +141190 POINT(41.1136118139782 -88.61369046316932) bank141190 +141191 POINT(41.18069453421728 -88.38566133096577) bank141191 +141192 POINT(41.5322768900653 -88.48398427370772) bank141192 +141193 POINT(41.79251880097001 -87.29407192864063) bank141193 +141194 POINT(40.88817932463902 -87.08854347487016) bank141194 +141195 POINT(41.454130159043615 -87.16015576828083) bank141195 +141196 POINT(42.850678180026506 -86.78536949202822) bank141196 +141197 POINT(41.81612267494669 -87.58416549571486) bank141197 +141198 POINT(41.53698561935684 -88.16720243667731) bank141198 +141199 POINT(41.87968387800167 -88.10423182791503) bank141199 +141200 POINT(41.93044406459816 -88.57009471696338) bank141200 +141201 POINT(42.35854491354353 -87.5658248258666) bank141201 +141202 POINT(41.372849676386345 -87.50135599718767) bank141202 +141203 POINT(42.83494291829436 -86.84876415667681) bank141203 +141204 POINT(41.48389188004493 -86.68315719097468) bank141204 +141205 POINT(40.973992454186586 -88.32093886098943) bank141205 +141206 POINT(42.19814786235848 -88.3929745358179) bank141206 +141207 POINT(42.11152813632831 -87.46979421678414) bank141207 +141208 POINT(42.48164908254859 -87.45445346268096) bank141208 +141209 POINT(42.751623144018154 -87.83605862632487) bank141209 +141210 POINT(42.108272083058985 -87.19952383686781) bank141210 +141211 POINT(42.34228193685578 -86.65653065932088) bank141211 +141212 POINT(41.75076306694206 -87.12355783244253) bank141212 +141213 POINT(42.369949493554124 -87.71910790817353) bank141213 +141214 POINT(42.303888335459085 -87.12347871150027) bank141214 +141215 POINT(42.09028711703108 -87.11907060492102) bank141215 +141216 POINT(42.29148709885858 -87.65560184622434) bank141216 +141217 POINT(42.78367489110868 -88.49182971545356) bank141217 +141218 POINT(42.107025528911656 -88.43490248238594) bank141218 +141219 POINT(42.741727794791196 -87.66294108787702) bank141219 +141220 POINT(41.11736642527914 -87.4436339201801) bank141220 +141221 POINT(41.53112350391915 -88.56663574778386) bank141221 +141222 POINT(41.79713649113733 -86.94281606015704) bank141222 +141223 POINT(41.99807152130252 -88.05106825557102) bank141223 +141224 POINT(40.959499447300104 -87.87243071758952) bank141224 +141225 POINT(42.718035292349334 -87.47809318031366) bank141225 +141226 POINT(42.01807807351963 -88.15771872977089) bank141226 +141227 POINT(41.244865265133484 -87.08661088718506) bank141227 +141228 POINT(42.389031605402565 -88.30373096322262) bank141228 +141229 POINT(42.76547941634516 -88.33019044654296) bank141229 +141230 POINT(42.595166483342204 -87.04043818885634) bank141230 +141231 POINT(41.90271133962083 -87.08647053264333) bank141231 +141232 POINT(42.58310351963723 -87.21430896033209) bank141232 +141233 POINT(41.18357736690686 -87.28990559219851) bank141233 +141234 POINT(42.628591399102135 -87.19884354621497) bank141234 +141235 POINT(41.63160355647834 -87.71139297048809) bank141235 +141236 POINT(41.585528087070955 -86.9454517279711) bank141236 +141237 POINT(41.22086367150533 -87.59038548493426) bank141237 +141238 POINT(42.45513565231384 -87.61597495472203) bank141238 +141239 POINT(41.11537384764556 -88.46613386545039) bank141239 +141240 POINT(41.768058188933516 -88.03828667292044) bank141240 +141241 POINT(42.029884158374244 -86.6956289577139) bank141241 +141242 POINT(41.30164582299823 -88.28433769469781) bank141242 +141243 POINT(40.941825110073786 -86.76602016261376) bank141243 +141244 POINT(41.52328293015805 -86.791623000574) bank141244 +141245 POINT(42.48796961258082 -87.9556956722774) bank141245 +141246 POINT(42.04725693324359 -86.79183012201736) bank141246 +141247 POINT(41.52807354336803 -87.39452552603974) bank141247 +141248 POINT(40.92873573919879 -88.17589460853804) bank141248 +141249 POINT(41.30633183129563 -87.69459625981285) bank141249 +141250 POINT(42.19605746459799 -87.66694954280617) bank141250 +141251 POINT(41.332987596419784 -86.75175889212862) bank141251 +141252 POINT(41.25876556454408 -86.99366114849707) bank141252 +141253 POINT(41.621934620044534 -88.55215882108703) bank141253 +141254 POINT(42.306392560203946 -88.22241262717289) bank141254 +141255 POINT(41.54438493301493 -88.05987312215147) bank141255 +141256 POINT(41.57103362930978 -86.88932431381171) bank141256 +141257 POINT(41.13744657871368 -88.57054960289044) bank141257 +141258 POINT(40.919905461421514 -88.44762811557926) bank141258 +141259 POINT(42.19552944619066 -88.36450493052483) bank141259 +141260 POINT(41.49688793789617 -87.9102343687747) bank141260 +141261 POINT(41.44199064351017 -86.92753955739315) bank141261 +141262 POINT(42.52670018194046 -88.5328929739214) bank141262 +141263 POINT(41.55594748448391 -87.71223722046561) bank141263 +141264 POINT(41.82364104357834 -87.02175233677077) bank141264 +141265 POINT(41.65422532892488 -86.87466701826465) bank141265 +141266 POINT(42.249447005749545 -88.25341001204983) bank141266 +141267 POINT(42.15998446613114 -87.35616019299765) bank141267 +141268 POINT(41.404453508019 -88.29431835564074) bank141268 +141269 POINT(42.43485104347557 -86.99721008859291) bank141269 +141270 POINT(41.873786241859726 -87.38300088451534) bank141270 +141271 POINT(42.58107151530263 -86.65059085054388) bank141271 +141272 POINT(42.164729540200106 -87.01997227806336) bank141272 +141273 POINT(41.02178055490361 -87.89387545560331) bank141273 +141274 POINT(42.23986578103585 -87.43209457643718) bank141274 +141275 POINT(41.46124821790442 -88.55866450345522) bank141275 +141276 POINT(42.590708620069705 -88.0409034892648) bank141276 +141277 POINT(41.71949638327093 -88.62551679060422) bank141277 +141278 POINT(41.73939852394314 -87.65294289860081) bank141278 +141279 POINT(41.58050939329992 -87.68234515524642) bank141279 +141280 POINT(42.648877290524496 -88.62167885129081) bank141280 +141281 POINT(41.3798765099952 -87.90969127331047) bank141281 +141282 POINT(42.141311883186546 -88.33848459033322) bank141282 +141283 POINT(40.90366634489178 -88.32924944399227) bank141283 +141284 POINT(42.16023430465334 -87.20465056400113) bank141284 +141285 POINT(40.88564910325308 -88.62292354753667) bank141285 +141286 POINT(42.55940184499834 -87.25949324110222) bank141286 +141287 POINT(41.19972891687871 -88.22663265116665) bank141287 +141288 POINT(42.08001242926096 -86.91478481291598) bank141288 +141289 POINT(42.0085807099849 -87.56424885481314) bank141289 +141290 POINT(41.99659971357571 -88.16700970096836) bank141290 +141291 POINT(42.29080515665419 -87.7172443638354) bank141291 +141292 POINT(40.998014590029705 -87.9963557070487) bank141292 +141293 POINT(41.68152205641918 -87.06093607551392) bank141293 +141294 POINT(42.273765683735895 -88.46665971509435) bank141294 +141295 POINT(41.97188990812496 -88.17026474336784) bank141295 +141296 POINT(41.131471808875006 -87.58089560647466) bank141296 +141297 POINT(42.05679788283922 -88.31092925296632) bank141297 +141298 POINT(42.45445164790986 -88.6194482756327) bank141298 +141299 POINT(41.715035372009055 -87.32755206512807) bank141299 +141300 POINT(42.0346386170473 -87.83229053511864) bank141300 +141301 POINT(42.510497999690344 -87.56754222164051) bank141301 +141302 POINT(40.94784116753728 -87.70812493849942) bank141302 +141303 POINT(41.42390678855964 -87.03244797686193) bank141303 +141304 POINT(42.59083552999831 -86.92234311090499) bank141304 +141305 POINT(41.53659650654492 -88.14656521341047) bank141305 +141306 POINT(40.98849928893969 -87.3990887743846) bank141306 +141307 POINT(41.10384041751625 -87.60510480052213) bank141307 +141308 POINT(42.200771554251816 -88.01617470985197) bank141308 +141309 POINT(42.69902996096738 -87.65095699205787) bank141309 +141310 POINT(41.00911716827048 -88.53686057630931) bank141310 +141311 POINT(41.67578085911541 -87.22539314518913) bank141311 +141312 POINT(41.64146477463984 -87.57485040272908) bank141312 +141313 POINT(41.806562548535645 -87.96105042316584) bank141313 +141314 POINT(40.885948873918764 -88.04964148862221) bank141314 +141315 POINT(41.992294279173734 -87.47479800639164) bank141315 +141316 POINT(42.87729852508421 -87.94583654269817) bank141316 +141317 POINT(40.91384922567367 -88.18318686136513) bank141317 +141318 POINT(41.70851534275728 -87.81251330188816) bank141318 +141319 POINT(41.173421038295395 -86.68132639960807) bank141319 +141320 POINT(42.660422095197475 -87.81024932870889) bank141320 +141321 POINT(42.51581109260766 -87.34198113039243) bank141321 +141322 POINT(40.98806915023521 -87.02206976001378) bank141322 +141323 POINT(41.97429813878902 -87.04312181670231) bank141323 +141324 POINT(41.12419910089567 -88.12273579113042) bank141324 +141325 POINT(41.29582267046296 -87.33696449739115) bank141325 +141326 POINT(41.70897435373682 -86.69714286816628) bank141326 +141327 POINT(42.53420532129509 -87.19355312693203) bank141327 +141328 POINT(41.24471354612528 -88.1993539574181) bank141328 +141329 POINT(42.23297246610625 -87.44628692762822) bank141329 +141330 POINT(41.20628715544802 -87.49895153860018) bank141330 +141331 POINT(41.844897033627035 -88.46305391336391) bank141331 +141332 POINT(41.49453342595838 -87.771201094401) bank141332 +141333 POINT(41.66121514889758 -88.50402445003984) bank141333 +141334 POINT(41.40804766812217 -88.05052548651723) bank141334 +141335 POINT(41.28332820012932 -87.85573691718555) bank141335 +141336 POINT(42.54184877818781 -86.82719198391564) bank141336 +141337 POINT(41.353948058729046 -87.80354819448476) bank141337 +141338 POINT(41.99795358058699 -88.06812649559537) bank141338 +141339 POINT(41.29196777320102 -88.39781314753598) bank141339 +141340 POINT(42.1147054488698 -88.17527780496339) bank141340 +141341 POINT(41.849770522022 -87.46266883907376) bank141341 +141342 POINT(42.51618941040121 -88.2052941173502) bank141342 +141343 POINT(42.66168263022318 -86.9224926804447) bank141343 +141344 POINT(42.58085144469467 -87.13609678725382) bank141344 +141345 POINT(41.85148007004434 -87.59961415217545) bank141345 +141346 POINT(41.653244005796154 -87.96172989479676) bank141346 +141347 POINT(41.19588117436777 -87.7627118361378) bank141347 +141348 POINT(41.555217739373326 -87.33382163928864) bank141348 +141349 POINT(41.47872087548665 -87.1624380341069) bank141349 +141350 POINT(41.152131328225835 -87.63949779987827) bank141350 +141351 POINT(41.084857268009266 -86.91062163919088) bank141351 +141352 POINT(41.432260853550375 -87.62530192433024) bank141352 +141353 POINT(41.284956746885726 -87.29578264515538) bank141353 +141354 POINT(41.37195674282036 -86.81064998500058) bank141354 +141355 POINT(41.507713137541664 -87.77486026533852) bank141355 +141356 POINT(42.31852524736447 -87.38048160225608) bank141356 +141357 POINT(42.180896152695496 -87.16678833852706) bank141357 +141358 POINT(42.53689298282143 -86.96594961538054) bank141358 +141359 POINT(41.49599803309842 -87.25363309163346) bank141359 +141360 POINT(42.02853762162493 -88.3209536813542) bank141360 +141361 POINT(41.26914336485624 -87.7974834702573) bank141361 +141362 POINT(41.932285897237044 -86.84777842109484) bank141362 +141363 POINT(41.3428144587665 -87.77318425020766) bank141363 +141364 POINT(41.970142261578616 -87.44640617723822) bank141364 +141365 POINT(41.91181520196492 -87.62987272621247) bank141365 +141366 POINT(41.385101828779604 -88.58075787846298) bank141366 +141367 POINT(41.72642986478164 -86.74143067411771) bank141367 +141368 POINT(41.600253459793905 -87.13005073971938) bank141368 +141369 POINT(41.286172553433104 -87.41996516302726) bank141369 +141370 POINT(42.04094344300474 -86.63281528227908) bank141370 +141371 POINT(41.56289166651073 -87.48093381878121) bank141371 +141372 POINT(41.64280450336482 -88.04897226504835) bank141372 +141373 POINT(42.45382590958277 -87.95473318189211) bank141373 +141374 POINT(42.432899033618945 -86.98445795041918) bank141374 +141375 POINT(41.275849996456245 -87.26377938695596) bank141375 +141376 POINT(42.1095486549555 -87.97169983420402) bank141376 +141377 POINT(42.23244437501519 -87.33737175972504) bank141377 +141378 POINT(41.74166787678732 -87.17826251943076) bank141378 +141379 POINT(42.86018935803095 -86.7749512113809) bank141379 +141380 POINT(41.49337865316974 -87.55682864680969) bank141380 +141381 POINT(41.21775148173898 -86.95937539482183) bank141381 +141382 POINT(42.31588514968232 -87.21000595323926) bank141382 +141383 POINT(41.20992110367453 -87.68727801137014) bank141383 +141384 POINT(42.59566005354016 -86.74782305463926) bank141384 +141385 POINT(41.155595729220664 -88.42570222777155) bank141385 +141386 POINT(42.68574119159444 -88.0376478441806) bank141386 +141387 POINT(40.981704994417406 -88.13541719338184) bank141387 +141388 POINT(42.77899742968521 -87.19608302888965) bank141388 +141389 POINT(41.27296595270969 -88.57167924856755) bank141389 +141390 POINT(41.17604994566756 -87.2327625938308) bank141390 +141391 POINT(42.266102404541776 -87.1386717315614) bank141391 +141392 POINT(42.86891352935467 -86.94197266981291) bank141392 +141393 POINT(41.12708226018821 -88.01504957595151) bank141393 +141394 POINT(42.74230520539534 -87.11150855395849) bank141394 +141395 POINT(42.320262135894836 -88.16922287107833) bank141395 +141396 POINT(42.80351199137697 -86.87203514065271) bank141396 +141397 POINT(42.5508342603901 -87.12201941980754) bank141397 +141398 POINT(40.96263883394161 -87.07137259175902) bank141398 +141399 POINT(41.50636404925393 -86.63951465381714) bank141399 +141400 POINT(41.35785913840136 -87.66132730955663) bank141400 +141401 POINT(42.272625132090916 -88.17226757422945) bank141401 +141402 POINT(42.624508541863186 -87.89159506869166) bank141402 +141403 POINT(41.65348171278868 -86.85219837903395) bank141403 +141404 POINT(40.88222975495655 -87.48229792613982) bank141404 +141405 POINT(40.91295905037632 -87.28186940228004) bank141405 +141406 POINT(41.83597649283063 -88.39824359851977) bank141406 +141407 POINT(41.50084463499419 -88.58444905676957) bank141407 +141408 POINT(41.0133274464905 -87.90412218800965) bank141408 +141409 POINT(42.1454500853691 -88.29226375243252) bank141409 +141410 POINT(42.87129334106936 -87.47912949909283) bank141410 +141411 POINT(42.78823015183691 -87.05954939689315) bank141411 +141412 POINT(42.876144300582034 -87.9530859101458) bank141412 +141413 POINT(41.80197370123057 -86.7325144184552) bank141413 +141414 POINT(42.50594968218874 -87.18255968719998) bank141414 +141415 POINT(42.297455182007035 -87.79164720971895) bank141415 +141416 POINT(41.92681286763595 -87.17421027718123) bank141416 +141417 POINT(42.220001311198864 -88.24616344424528) bank141417 +141418 POINT(41.61404973809552 -88.10073660560633) bank141418 +141419 POINT(42.302696546523926 -87.19935069589302) bank141419 +141420 POINT(42.529346418492494 -86.95300682449687) bank141420 +141421 POINT(42.29851430713236 -87.17206004324159) bank141421 +141422 POINT(41.96484330831911 -88.41350424873008) bank141422 +141423 POINT(41.03134458601226 -88.61581034814972) bank141423 +141424 POINT(42.65157012388835 -87.38111183211853) bank141424 +141425 POINT(42.82169089279108 -86.77620192094292) bank141425 +141426 POINT(41.21414177504604 -86.86230454646115) bank141426 +141427 POINT(41.256849095466066 -86.82030655740398) bank141427 +141428 POINT(41.51493774570848 -88.12775599534318) bank141428 +141429 POINT(41.76689480831074 -88.61446315032116) bank141429 +141430 POINT(42.718040816198375 -88.2345031064706) bank141430 +141431 POINT(41.64350739435761 -86.8701358681384) bank141431 +141432 POINT(41.10456038401809 -88.36358440902448) bank141432 +141433 POINT(41.36194882131163 -88.40836782635593) bank141433 +141434 POINT(41.288573034809374 -87.25564515666198) bank141434 +141435 POINT(42.407561327703654 -88.4311912242135) bank141435 +141436 POINT(41.46696026217059 -87.30153734038889) bank141436 +141437 POINT(42.24407549558874 -87.37426446008423) bank141437 +141438 POINT(42.43367120871772 -88.50615123894538) bank141438 +141439 POINT(42.82801100563935 -87.76238495271707) bank141439 +141440 POINT(42.420940545683024 -87.62794244240573) bank141440 +141441 POINT(42.84241499789321 -87.208707647795) bank141441 +141442 POINT(42.33562568007597 -87.02691937183126) bank141442 +141443 POINT(41.93340963375284 -87.36528680150111) bank141443 +141444 POINT(40.942769843304234 -88.25190766927429) bank141444 +141445 POINT(41.39998873038634 -86.90906587047444) bank141445 +141446 POINT(41.370540209066895 -86.92921210505553) bank141446 +141447 POINT(41.282641823192556 -88.02610630635792) bank141447 +141448 POINT(42.784293790192606 -88.45289017319875) bank141448 +141449 POINT(41.3174735985248 -86.80025735760594) bank141449 +141450 POINT(41.60749695217106 -86.86603207261257) bank141450 +141451 POINT(41.18482474987043 -87.87900921362329) bank141451 +141452 POINT(41.753199100801396 -87.96530653049287) bank141452 +141453 POINT(41.93144999077638 -88.14236358976457) bank141453 +141454 POINT(42.51358332802943 -87.36921069891035) bank141454 +141455 POINT(42.031246151091565 -87.24073531997414) bank141455 +141456 POINT(42.37182548085367 -88.45209333089247) bank141456 +141457 POINT(42.832257625935185 -87.77967024519059) bank141457 +141458 POINT(41.92373155737439 -87.08344676742844) bank141458 +141459 POINT(41.31267473789464 -88.20285457285729) bank141459 +141460 POINT(42.59456096012895 -88.35262874627838) bank141460 +141461 POINT(42.86601143069185 -86.78010852347651) bank141461 +141462 POINT(42.77847724966272 -88.03482908462003) bank141462 +141463 POINT(41.58603728595053 -87.21266976468985) bank141463 +141464 POINT(41.7937388606934 -88.38944041197433) bank141464 +141465 POINT(41.560480182491915 -87.91026123647244) bank141465 +141466 POINT(42.746510364630346 -88.53193685256785) bank141466 +141467 POINT(42.30878312999772 -87.23128888330166) bank141467 +141468 POINT(41.65624638861237 -87.42532528667674) bank141468 +141469 POINT(41.32582438582682 -88.61042016344635) bank141469 +141470 POINT(42.020233890471516 -88.31411228506454) bank141470 +141471 POINT(42.06884945402794 -88.04689536766965) bank141471 +141472 POINT(42.072071234226264 -88.53585604311196) bank141472 +141473 POINT(41.77042126951007 -88.39943885470745) bank141473 +141474 POINT(42.1519748140555 -87.73728892632168) bank141474 +141475 POINT(41.21053995252585 -87.68227603803095) bank141475 +141476 POINT(41.83561130178952 -88.12489600620208) bank141476 +141477 POINT(41.28727671809358 -87.71113632914115) bank141477 +141478 POINT(41.149668900954744 -88.42228991550506) bank141478 +141479 POINT(42.86966043922181 -88.57560547953777) bank141479 +141480 POINT(41.4916098105885 -86.90064040911243) bank141480 +141481 POINT(41.93827489297747 -86.82432773980308) bank141481 +141482 POINT(41.04690869820894 -87.48415236054305) bank141482 +141483 POINT(41.885911740332794 -86.88142486208554) bank141483 +141484 POINT(42.23940657133252 -87.4554537753831) bank141484 +141485 POINT(41.5783432215091 -88.57523266497739) bank141485 +141486 POINT(41.696456819642194 -87.4276005230979) bank141486 +141487 POINT(41.168688115467276 -87.56672031108644) bank141487 +141488 POINT(42.06590630623427 -88.44989826526988) bank141488 +141489 POINT(41.91380138969901 -88.50261294361063) bank141489 +141490 POINT(41.313316237148996 -88.592306993717) bank141490 +141491 POINT(41.36104565280938 -86.82295699592173) bank141491 +141492 POINT(41.97899455139937 -88.48757168388155) bank141492 +141493 POINT(41.935614558355034 -88.61051343519327) bank141493 +141494 POINT(41.11080663343891 -88.25995228520263) bank141494 +141495 POINT(41.7438324539142 -86.69239938432764) bank141495 +141496 POINT(41.49204250987668 -88.56241524809643) bank141496 +141497 POINT(41.79061417611931 -87.32315231502433) bank141497 +141498 POINT(41.68586728942294 -87.41525764635196) bank141498 +141499 POINT(41.24339304107465 -86.88127216202331) bank141499 +141500 POINT(41.11348627374474 -87.53951080750296) bank141500 +141501 POINT(41.33418630241052 -86.77202537107406) bank141501 +141502 POINT(42.576140910898474 -87.65890911094198) bank141502 +141503 POINT(41.778185187364585 -87.65372022033289) bank141503 +141504 POINT(41.31991701366364 -86.8700978623076) bank141504 +141505 POINT(41.22517549175174 -87.01090675325962) bank141505 +141506 POINT(42.44788826034741 -86.9998769602754) bank141506 +141507 POINT(41.43444766510548 -88.37308122400158) bank141507 +141508 POINT(42.359394808802804 -88.23473797708132) bank141508 +141509 POINT(42.198192259203964 -87.95641316549701) bank141509 +141510 POINT(42.57106985132079 -86.97891814789563) bank141510 +141511 POINT(42.01750559058806 -87.64667014256017) bank141511 +141512 POINT(41.54569159648624 -87.88634904527139) bank141512 +141513 POINT(41.98689968099506 -87.4159634492122) bank141513 +141514 POINT(42.12847317256096 -88.14447654857213) bank141514 +141515 POINT(41.566961566803826 -86.71203324645793) bank141515 +141516 POINT(41.08906174584193 -88.57935682465653) bank141516 +141517 POINT(41.14450798579351 -87.80128559361059) bank141517 +141518 POINT(41.63420257062835 -87.72275663420203) bank141518 +141519 POINT(42.23758684640809 -86.94158820009093) bank141519 +141520 POINT(42.7538693673 -87.04826397783566) bank141520 +141521 POINT(42.53698933165392 -87.330902777422) bank141521 +141522 POINT(41.04937566554938 -87.30992688420429) bank141522 +141523 POINT(42.63576927063799 -86.98996316142035) bank141523 +141524 POINT(41.414423367783115 -88.0864383028984) bank141524 +141525 POINT(42.50882461561207 -88.00559045387637) bank141525 +141526 POINT(42.72978984141695 -88.37364702045146) bank141526 +141527 POINT(41.015460952057246 -88.41421974012042) bank141527 +141528 POINT(41.25058983712428 -86.66467322463518) bank141528 +141529 POINT(42.198548214350275 -87.71323333948159) bank141529 +141530 POINT(42.28577054627634 -87.52175268595718) bank141530 +141531 POINT(41.37298429632199 -88.4937191462275) bank141531 +141532 POINT(41.88777217695066 -87.15400093599436) bank141532 +141533 POINT(42.60001755939245 -87.23714045726739) bank141533 +141534 POINT(40.95867658406003 -88.61809430822295) bank141534 +141535 POINT(41.989033452758086 -87.45486894252426) bank141535 +141536 POINT(41.6861604689661 -86.93759977480525) bank141536 +141537 POINT(42.50450469891954 -88.62088517888034) bank141537 +141538 POINT(41.11467218003667 -88.00619916830279) bank141538 +141539 POINT(42.298609006482565 -87.14140265406985) bank141539 +141540 POINT(41.36411811610131 -88.53734947648047) bank141540 +141541 POINT(41.404875465726846 -87.88284075261578) bank141541 +141542 POINT(41.92115470490684 -86.99670586415114) bank141542 +141543 POINT(42.43661938541529 -88.14803036314198) bank141543 +141544 POINT(41.75200821523809 -87.11863782742516) bank141544 +141545 POINT(42.04833047769431 -86.85849677804251) bank141545 +141546 POINT(42.507070945463155 -88.3521376134505) bank141546 +141547 POINT(41.2320580958121 -87.47998648533355) bank141547 +141548 POINT(40.96937175134643 -88.49641787414387) bank141548 +141549 POINT(42.71445262044865 -88.61673799141143) bank141549 +141550 POINT(41.63162346529011 -87.59559130326923) bank141550 +141551 POINT(42.585296189463975 -88.14839930998947) bank141551 +141552 POINT(41.156901314042386 -87.76391727074247) bank141552 +141553 POINT(41.67154197094581 -88.415144527581) bank141553 +141554 POINT(42.63514973855185 -88.51616505748775) bank141554 +141555 POINT(42.39142470855531 -86.67962900894393) bank141555 +141556 POINT(42.773758899670014 -87.17344909901145) bank141556 +141557 POINT(41.30654096771963 -87.73045554610803) bank141557 +141558 POINT(41.055185686529384 -88.20550659172072) bank141558 +141559 POINT(41.206602734854634 -88.14985637425269) bank141559 +141560 POINT(42.2028879810197 -87.26728976587542) bank141560 +141561 POINT(42.56077906188323 -87.52246135214943) bank141561 +141562 POINT(42.3576286709648 -87.47203973332553) bank141562 +141563 POINT(42.8555094855119 -87.30749968717565) bank141563 +141564 POINT(41.93572810518283 -87.00988560344503) bank141564 +141565 POINT(41.05180419884875 -86.66387698558196) bank141565 +141566 POINT(41.64052869464399 -88.50931956626684) bank141566 +141567 POINT(41.145361253086676 -88.60650345672973) bank141567 +141568 POINT(42.35381868829627 -86.7498681754443) bank141568 +141569 POINT(42.21870311890469 -87.19512846416335) bank141569 +141570 POINT(42.052504553961754 -87.41702633405053) bank141570 +141571 POINT(41.36345337457607 -87.4291387585415) bank141571 +141572 POINT(40.958984322270695 -87.20784356387071) bank141572 +141573 POINT(42.0140998881691 -88.014684629795) bank141573 +141574 POINT(41.19476650487467 -87.70479110183636) bank141574 +141575 POINT(41.17029774820949 -87.08266299866355) bank141575 +141576 POINT(41.00141113467246 -88.04964617430313) bank141576 +141577 POINT(41.72131588486006 -87.45570882347499) bank141577 +141578 POINT(41.10263100205728 -86.85725571456892) bank141578 +141579 POINT(42.57753242692625 -87.32748699321742) bank141579 +141580 POINT(42.46985089326602 -87.82677265469954) bank141580 +141581 POINT(40.913538373330084 -86.90549570260444) bank141581 +141582 POINT(41.34811008541714 -88.336178433898) bank141582 +141583 POINT(42.78728013313465 -88.29486074120527) bank141583 +141584 POINT(41.51083687639218 -88.3853606093642) bank141584 +141585 POINT(41.339851882056855 -87.2758406153976) bank141585 +141586 POINT(41.22203469128137 -87.2623590108984) bank141586 +141587 POINT(42.29808091123555 -88.21469222228896) bank141587 +141588 POINT(41.051832988629954 -88.62487843266766) bank141588 +141589 POINT(42.80071671154303 -87.15265747121505) bank141589 +141590 POINT(41.206408837419595 -87.83618240478103) bank141590 +141591 POINT(41.194118827802335 -87.41397736344557) bank141591 +141592 POINT(41.78726068087121 -87.31230467143413) bank141592 +141593 POINT(42.48258720257046 -88.54103840541912) bank141593 +141594 POINT(42.218123730046 -86.7034960848501) bank141594 +141595 POINT(42.042648758072694 -88.39070501035249) bank141595 +141596 POINT(41.02839012662093 -86.87407385401569) bank141596 +141597 POINT(42.38367765554054 -86.80090519070642) bank141597 +141598 POINT(42.814827680136034 -87.12851979219587) bank141598 +141599 POINT(42.30109869817646 -88.51766329794735) bank141599 +141600 POINT(42.323817300366585 -88.1667464641428) bank141600 +141601 POINT(42.085881391776496 -88.52010160936571) bank141601 +141602 POINT(42.60484713694344 -88.29010051369126) bank141602 +141603 POINT(42.733665705826965 -88.4008351299583) bank141603 +141604 POINT(42.33773450393049 -87.83502818203166) bank141604 +141605 POINT(42.61730768658818 -87.32397567182858) bank141605 +141606 POINT(41.56552099999081 -88.10472897538605) bank141606 +141607 POINT(40.96108879299771 -86.74971392841208) bank141607 +141608 POINT(41.89499332172223 -87.18466950232181) bank141608 +141609 POINT(41.23925192477786 -88.00354855882348) bank141609 +141610 POINT(42.76844041281911 -87.39123430845666) bank141610 +141611 POINT(42.23793996751146 -88.3592551584388) bank141611 +141612 POINT(42.48193580933592 -88.3071184992349) bank141612 +141613 POINT(41.43141396932136 -88.25865552691647) bank141613 +141614 POINT(41.22171233725573 -88.3642078399748) bank141614 +141615 POINT(41.18544726893136 -87.68567829970499) bank141615 +141616 POINT(41.00480916552265 -86.67034299246755) bank141616 +141617 POINT(41.18166662749688 -87.94927963330562) bank141617 +141618 POINT(42.48260299509253 -88.36437574069198) bank141618 +141619 POINT(42.60341747463919 -88.1225370630351) bank141619 +141620 POINT(41.00927640914002 -87.14755883560044) bank141620 +141621 POINT(42.19992973242567 -86.85499394255766) bank141621 +141622 POINT(42.62538549300719 -86.71898517902767) bank141622 +141623 POINT(41.20577273749041 -88.09120538116582) bank141623 +141624 POINT(41.986817514061585 -88.09741755947476) bank141624 +141625 POINT(41.872989297135696 -87.41938740532386) bank141625 +141626 POINT(42.610634883704016 -87.41858429415797) bank141626 +141627 POINT(42.59546776228561 -87.42951305985899) bank141627 +141628 POINT(41.780920193116735 -87.41880259303377) bank141628 +141629 POINT(41.821244041339725 -86.78422848151514) bank141629 +141630 POINT(41.42605227555507 -87.15627325005661) bank141630 +141631 POINT(41.905917426881935 -87.32626451444685) bank141631 +141632 POINT(42.144962662717205 -86.89397991678601) bank141632 +141633 POINT(41.318907478642956 -87.55550621953412) bank141633 +141634 POINT(40.93622561319233 -86.72683064972719) bank141634 +141635 POINT(42.25189549425758 -87.06534810817962) bank141635 +141636 POINT(42.302967913067214 -88.13961588083609) bank141636 +141637 POINT(41.69565693539562 -87.7240308329657) bank141637 +141638 POINT(42.56984997754784 -87.23464976609542) bank141638 +141639 POINT(41.96383274611714 -87.77835011681843) bank141639 +141640 POINT(41.97298865864874 -87.6878048927729) bank141640 +141641 POINT(42.68580358430029 -87.68776323316003) bank141641 +141642 POINT(41.3568472006489 -88.48109234509072) bank141642 +141643 POINT(41.32350124177022 -87.89951663337524) bank141643 +141644 POINT(42.532100466939085 -87.71895562089789) bank141644 +141645 POINT(41.966730302222395 -87.31435836309831) bank141645 +141646 POINT(42.840323037248766 -88.3647899298214) bank141646 +141647 POINT(42.8499668157056 -87.1198401614034) bank141647 +141648 POINT(42.85264378825541 -87.99699468204433) bank141648 +141649 POINT(41.94779764968634 -88.16988962182779) bank141649 +141650 POINT(42.28535641572632 -86.96479258644011) bank141650 +141651 POINT(41.93416443539475 -88.05610026609752) bank141651 +141652 POINT(42.76085858146053 -88.5049851358456) bank141652 +141653 POINT(41.248662940933585 -86.80882823375713) bank141653 +141654 POINT(42.13444943959781 -88.41853532487727) bank141654 +141655 POINT(41.98344375954882 -87.60587506997511) bank141655 +141656 POINT(41.10566791788152 -88.02459651644057) bank141656 +141657 POINT(40.91281006633862 -87.67960171394338) bank141657 +141658 POINT(42.71289469157069 -88.0574534488368) bank141658 +141659 POINT(41.825955279022814 -86.69889401187048) bank141659 +141660 POINT(41.45441132186546 -88.18503729743307) bank141660 +141661 POINT(42.42099690723012 -86.82106486049089) bank141661 +141662 POINT(42.329211555427705 -87.77697201684427) bank141662 +141663 POINT(41.97561100567704 -87.0907541762013) bank141663 +141664 POINT(42.36905776185306 -87.06671983558755) bank141664 +141665 POINT(42.51445069351348 -88.51626224071624) bank141665 +141666 POINT(41.38658659991014 -88.384864877989) bank141666 +141667 POINT(42.31343674690401 -88.42958635099511) bank141667 +141668 POINT(42.68523681147042 -87.27072110389679) bank141668 +141669 POINT(42.47788863023246 -86.8229263297988) bank141669 +141670 POINT(40.95039117707026 -88.38654032497062) bank141670 +141671 POINT(42.05309906930391 -86.85417542354695) bank141671 +141672 POINT(41.585328396145364 -88.25074542609954) bank141672 +141673 POINT(40.96172722962508 -86.8260111088443) bank141673 +141674 POINT(41.34585173849447 -88.12923171541102) bank141674 +141675 POINT(42.838472406490624 -88.45938908199248) bank141675 +141676 POINT(41.24646327690969 -86.74978852435841) bank141676 +141677 POINT(42.575038103357784 -87.52349729172376) bank141677 +141678 POINT(42.035360177011015 -87.0679628775897) bank141678 +141679 POINT(42.0030932935542 -88.34219065984385) bank141679 +141680 POINT(41.91860288166136 -87.6662404377499) bank141680 +141681 POINT(42.321642184960304 -88.01542144943897) bank141681 +141682 POINT(42.35186352926177 -86.88701912052143) bank141682 +141683 POINT(42.09092386849183 -87.57778134836119) bank141683 +141684 POINT(42.87468329927829 -87.53406705106447) bank141684 +141685 POINT(41.16103825947601 -87.7105260386621) bank141685 +141686 POINT(42.17629618269903 -87.61944115877496) bank141686 +141687 POINT(41.395476250634296 -88.27011846623809) bank141687 +141688 POINT(42.67445244434145 -87.006462187837) bank141688 +141689 POINT(42.247540997205355 -88.57081003955194) bank141689 +141690 POINT(41.11626188235961 -87.59852420464723) bank141690 +141691 POINT(42.04371070641051 -88.34134214271354) bank141691 +141692 POINT(42.04235146681744 -87.28617228246789) bank141692 +141693 POINT(41.06221568301089 -86.63924605159104) bank141693 +141694 POINT(41.369790696276844 -86.86350855836034) bank141694 +141695 POINT(41.69842864151837 -87.82442976004288) bank141695 +141696 POINT(42.34859450269983 -87.65779153560312) bank141696 +141697 POINT(42.5200312438788 -87.07871135309502) bank141697 +141698 POINT(42.195904538376126 -87.02243302797102) bank141698 +141699 POINT(42.83574142657598 -87.27937320679291) bank141699 +141700 POINT(42.78359687445636 -87.02811352562784) bank141700 +141701 POINT(41.23615325322001 -87.2391206477733) bank141701 +141702 POINT(41.78985862216512 -88.21526477211981) bank141702 +141703 POINT(41.67868233254873 -86.65723383772833) bank141703 +141704 POINT(41.87272883566959 -88.26291177368469) bank141704 +141705 POINT(42.82858730984352 -88.52966755223468) bank141705 +141706 POINT(41.07658038687016 -87.98332751863094) bank141706 +141707 POINT(42.38004933538099 -87.76002752070583) bank141707 +141708 POINT(40.994036509675276 -88.0992206774623) bank141708 +141709 POINT(42.66919182149907 -88.20305219086046) bank141709 +141710 POINT(41.412102586760824 -87.61908186269065) bank141710 +141711 POINT(42.65526282117657 -86.8771449316931) bank141711 +141712 POINT(42.62144314806674 -88.58586177267283) bank141712 +141713 POINT(42.612827209345575 -87.50865787313607) bank141713 +141714 POINT(42.68251999165862 -86.73175429792849) bank141714 +141715 POINT(42.08664076959806 -88.12214319852745) bank141715 +141716 POINT(42.74130317817699 -86.64404499791499) bank141716 +141717 POINT(42.45117333962162 -88.46408919462257) bank141717 +141718 POINT(42.49440522466129 -87.70946149845085) bank141718 +141719 POINT(42.63889916573754 -87.37107863658305) bank141719 +141720 POINT(41.25382088913332 -87.53603617638952) bank141720 +141721 POINT(42.59440754668514 -87.81099469781554) bank141721 +141722 POINT(41.884100073989146 -88.32155908358426) bank141722 +141723 POINT(41.14185459187719 -86.96544534736029) bank141723 +141724 POINT(41.3896591207941 -87.65362281452649) bank141724 +141725 POINT(41.15641734041916 -86.89051368005599) bank141725 +141726 POINT(40.93716667425158 -88.54794457220768) bank141726 +141727 POINT(42.21855119671682 -88.4122409466076) bank141727 +141728 POINT(41.616359280729824 -88.58266065454592) bank141728 +141729 POINT(42.862734469221 -88.54086695310595) bank141729 +141730 POINT(42.33755496734116 -87.1295054140057) bank141730 +141731 POINT(42.76333283395558 -87.69854910022167) bank141731 +141732 POINT(41.49658440843539 -87.80815023672778) bank141732 +141733 POINT(42.362553655337564 -87.34645874079864) bank141733 +141734 POINT(42.72544185631936 -86.97325929013769) bank141734 +141735 POINT(41.42832159873868 -87.81361538738075) bank141735 +141736 POINT(42.226773048912015 -86.66779610197209) bank141736 +141737 POINT(42.467765868810105 -88.40047983685743) bank141737 +141738 POINT(41.831622869598135 -87.96099400155573) bank141738 +141739 POINT(42.147764386642514 -88.22687251456726) bank141739 +141740 POINT(41.72786363118726 -88.00121056379369) bank141740 +141741 POINT(41.62322605347054 -88.25293622192807) bank141741 +141742 POINT(41.41926900572816 -87.9361244476579) bank141742 +141743 POINT(42.71834216354281 -86.76174862582208) bank141743 +141744 POINT(41.910998982699404 -87.24895931172708) bank141744 +141745 POINT(42.44391758692983 -87.09760077988248) bank141745 +141746 POINT(41.265817322686566 -87.139222282949) bank141746 +141747 POINT(42.700758619886564 -88.12530588291314) bank141747 +141748 POINT(40.94237585026306 -86.7207388559059) bank141748 +141749 POINT(41.85673344289787 -88.15643108780765) bank141749 +141750 POINT(41.57498043024492 -86.84267033974443) bank141750 +141751 POINT(41.683182218183745 -87.50212287481492) bank141751 +141752 POINT(41.13447284861282 -88.48654983434986) bank141752 +141753 POINT(41.36584162446579 -87.37576853822813) bank141753 +141754 POINT(42.506059052705666 -86.7184276538518) bank141754 +141755 POINT(41.92946504280478 -87.74051512848187) bank141755 +141756 POINT(42.731862537147165 -86.89317946159655) bank141756 +141757 POINT(42.3043163877424 -87.65076035668135) bank141757 +141758 POINT(42.030186936239176 -87.63526699758135) bank141758 +141759 POINT(40.95639273718418 -87.71764543479364) bank141759 +141760 POINT(41.31282066185284 -88.41011122262384) bank141760 +141761 POINT(42.06523414029088 -88.01783026019739) bank141761 +141762 POINT(42.55713621357501 -87.65556943647762) bank141762 +141763 POINT(42.23201394126275 -87.89716234958108) bank141763 +141764 POINT(42.6215913072479 -88.58932625782066) bank141764 +141765 POINT(41.424308056584245 -87.56122344523884) bank141765 +141766 POINT(41.527402707093586 -87.13650028377519) bank141766 +141767 POINT(41.408162837428335 -88.5558959686869) bank141767 +141768 POINT(42.6948039053042 -87.96397043453908) bank141768 +141769 POINT(42.190124905454084 -88.32330734681264) bank141769 +141770 POINT(42.792478096100325 -87.71644168316583) bank141770 +141771 POINT(42.66249098391992 -87.77633671307309) bank141771 +141772 POINT(41.78672786368697 -87.42752569471124) bank141772 +141773 POINT(40.894228470651036 -87.77833730245585) bank141773 +141774 POINT(41.39428135386152 -88.61028658913597) bank141774 +141775 POINT(42.150087954382265 -87.45600292242716) bank141775 +141776 POINT(42.46533001335569 -88.52654341344135) bank141776 +141777 POINT(42.846905673869195 -88.5714346051554) bank141777 +141778 POINT(41.5992944303697 -86.8153903456663) bank141778 +141779 POINT(42.28371232184174 -88.53245293971693) bank141779 +141780 POINT(42.245995932861526 -87.2186433583732) bank141780 +141781 POINT(41.65142381458542 -87.42934320514497) bank141781 +141782 POINT(42.75468925689043 -88.31862174766238) bank141782 +141783 POINT(41.34175143102743 -87.64707758163335) bank141783 +141784 POINT(41.82861366414737 -86.91765789297276) bank141784 +141785 POINT(41.46882448589108 -87.3831005795502) bank141785 +141786 POINT(41.297538318049504 -87.8635417598581) bank141786 +141787 POINT(41.37644342538814 -88.3872923626624) bank141787 +141788 POINT(41.44371810823105 -88.31163345760665) bank141788 +141789 POINT(41.78537966109814 -87.73097391486222) bank141789 +141790 POINT(42.059168560915914 -88.37495098479152) bank141790 +141791 POINT(41.54423103339042 -87.34554866024091) bank141791 +141792 POINT(40.97936348000605 -87.45575920956139) bank141792 +141793 POINT(42.17901046238589 -87.2687818056411) bank141793 +141794 POINT(41.17339920930022 -88.28666261534569) bank141794 +141795 POINT(42.014686108604145 -86.83263312129371) bank141795 +141796 POINT(41.82878905759671 -87.00248434863425) bank141796 +141797 POINT(41.31927687660732 -87.99635327035786) bank141797 +141798 POINT(41.3284320820921 -86.95089721390869) bank141798 +141799 POINT(41.19905291411181 -88.51487370200692) bank141799 +141800 POINT(42.19382664486782 -86.89788515088644) bank141800 +141801 POINT(42.050893285870195 -87.0812245336066) bank141801 +141802 POINT(42.56543807412545 -87.73420997340229) bank141802 +141803 POINT(41.56895786095617 -87.3211915228192) bank141803 +141804 POINT(41.859310106294814 -86.91060688134783) bank141804 +141805 POINT(42.069343440256674 -88.52767909167989) bank141805 +141806 POINT(42.32291638114839 -86.74598826908273) bank141806 +141807 POINT(42.01407417637399 -88.18152692649404) bank141807 +141808 POINT(42.787808928061494 -87.61760324611441) bank141808 +141809 POINT(42.01795177773928 -86.9925672244051) bank141809 +141810 POINT(40.92073486448275 -88.19373894108715) bank141810 +141811 POINT(41.23142742141621 -86.66082542870375) bank141811 +141812 POINT(41.26190234530587 -88.13548099835332) bank141812 +141813 POINT(41.388469543872745 -87.03019447501512) bank141813 +141814 POINT(42.139927172194966 -87.9433958154153) bank141814 +141815 POINT(41.547914944139244 -88.19805867747999) bank141815 +141816 POINT(42.298990351580734 -87.6689981524203) bank141816 +141817 POINT(42.22544137091773 -86.84670707367297) bank141817 +141818 POINT(42.374104875672884 -87.30643299179158) bank141818 +141819 POINT(41.71131141422775 -88.37565446823622) bank141819 +141820 POINT(41.89315110316421 -88.14152130458382) bank141820 +141821 POINT(41.2613482123373 -86.7211800330584) bank141821 +141822 POINT(41.9253927133775 -87.06475364355252) bank141822 +141823 POINT(41.2117604011803 -87.62813289072484) bank141823 +141824 POINT(42.43062482719129 -87.54059334271584) bank141824 +141825 POINT(41.8798832565055 -86.70790076125563) bank141825 +141826 POINT(41.964570549461406 -88.14441742559856) bank141826 +141827 POINT(41.017221839659804 -87.97963319752986) bank141827 +141828 POINT(42.48966600184247 -88.45307549618076) bank141828 +141829 POINT(42.29455446030555 -87.1272118213023) bank141829 +141830 POINT(41.467924685041176 -87.5395131023368) bank141830 +141831 POINT(42.740746475741254 -88.026354063009) bank141831 +141832 POINT(41.75548182036167 -86.76682897552703) bank141832 +141833 POINT(41.832115861471124 -87.31778440898978) bank141833 +141834 POINT(42.80514939809702 -88.5393741378385) bank141834 +141835 POINT(42.41788479185097 -88.24597624060681) bank141835 +141836 POINT(41.06989714545827 -87.54796087482147) bank141836 +141837 POINT(42.523892210403346 -87.75379280577097) bank141837 +141838 POINT(41.96716754212487 -87.86934241351955) bank141838 +141839 POINT(41.17952512738924 -87.8854511923982) bank141839 +141840 POINT(41.47597504190813 -86.88565247394641) bank141840 +141841 POINT(40.89652397794797 -87.23254984039549) bank141841 +141842 POINT(42.64981373920246 -87.89838692553818) bank141842 +141843 POINT(42.28489441523189 -86.72760843462446) bank141843 +141844 POINT(42.50666297101876 -88.13089561748177) bank141844 +141845 POINT(40.92529133323651 -88.62410193699485) bank141845 +141846 POINT(42.81246668061811 -87.97372878554064) bank141846 +141847 POINT(41.49127198782447 -87.85248746184679) bank141847 +141848 POINT(42.470835691772436 -87.89938693862001) bank141848 +141849 POINT(40.89200025381567 -87.8854281800768) bank141849 +141850 POINT(42.68922290136362 -87.68626946188412) bank141850 +141851 POINT(42.05675069078609 -86.7987832816035) bank141851 +141852 POINT(41.29955300695484 -87.62152119905278) bank141852 +141853 POINT(41.24062366534691 -87.02970971639735) bank141853 +141854 POINT(41.80465195208588 -88.06298199242259) bank141854 +141855 POINT(42.57046157150181 -87.7043440058748) bank141855 +141856 POINT(42.11873938431436 -88.50571138888472) bank141856 +141857 POINT(42.48292180773437 -88.05434770230248) bank141857 +141858 POINT(42.22460352370252 -87.06977451980755) bank141858 +141859 POINT(42.41286228150528 -88.48586077676902) bank141859 +141860 POINT(41.69845415127673 -88.0185745876725) bank141860 +141861 POINT(41.75698518862525 -87.0250229940191) bank141861 +141862 POINT(42.561599860786394 -86.96119231891079) bank141862 +141863 POINT(41.81572019760095 -87.99713793386698) bank141863 +141864 POINT(40.99728117987869 -87.55334894730093) bank141864 +141865 POINT(42.7357976453333 -87.54203517470094) bank141865 +141866 POINT(41.13070811692977 -88.29359732605542) bank141866 +141867 POINT(41.793808464656166 -87.70808726381436) bank141867 +141868 POINT(41.92664519777879 -87.85613283526442) bank141868 +141869 POINT(42.45319967006145 -86.71224645117098) bank141869 +141870 POINT(41.552282953268005 -88.11172763382254) bank141870 +141871 POINT(41.732967097247794 -87.79128935696563) bank141871 +141872 POINT(42.017065371208005 -87.67148840593141) bank141872 +141873 POINT(41.531508869810644 -88.11632768471956) bank141873 +141874 POINT(41.331828604470104 -88.5865394809539) bank141874 +141875 POINT(41.83822451623848 -87.17315984730008) bank141875 +141876 POINT(42.049424654467494 -88.11044194963478) bank141876 +141877 POINT(42.35732509443091 -87.36903760107984) bank141877 +141878 POINT(42.2833869108889 -88.20999145561403) bank141878 +141879 POINT(41.902197027554756 -87.9888229057436) bank141879 +141880 POINT(41.557176262759015 -87.8995076061173) bank141880 +141881 POINT(42.32776974686765 -88.13932056014654) bank141881 +141882 POINT(40.95747294828691 -87.17311234282472) bank141882 +141883 POINT(42.76076859090022 -87.89586004640259) bank141883 +141884 POINT(42.37752528058481 -87.75452961984189) bank141884 +141885 POINT(41.48135152598781 -87.29175722217329) bank141885 +141886 POINT(41.50205665824928 -87.36313834763058) bank141886 +141887 POINT(42.280624060299935 -87.40598436142437) bank141887 +141888 POINT(42.86250865652287 -86.99287443175005) bank141888 +141889 POINT(41.922628504737624 -87.18911647259408) bank141889 +141890 POINT(41.64042541270872 -86.6356407423595) bank141890 +141891 POINT(42.76035250656216 -87.79791148632157) bank141891 +141892 POINT(41.60283973477233 -88.41716427115075) bank141892 +141893 POINT(42.766945947202125 -86.79852462315125) bank141893 +141894 POINT(42.08814639565405 -87.88575763615351) bank141894 +141895 POINT(41.58766410127288 -86.94933477303279) bank141895 +141896 POINT(41.414014772588004 -87.0534765530076) bank141896 +141897 POINT(41.301062511825855 -87.48689223821837) bank141897 +141898 POINT(42.202079376693796 -86.796781086546) bank141898 +141899 POINT(41.146471151294676 -87.35669885566215) bank141899 +141900 POINT(41.69015215154503 -87.6439002736152) bank141900 +141901 POINT(41.59158535126276 -88.54713088281076) bank141901 +141902 POINT(41.614570011682176 -87.799363347682) bank141902 +141903 POINT(41.8476270276225 -86.69116254781166) bank141903 +141904 POINT(42.22926415726294 -88.56380574207392) bank141904 +141905 POINT(41.54901434498025 -87.22452887730773) bank141905 +141906 POINT(42.36644099029038 -87.47919230866127) bank141906 +141907 POINT(42.56481954742235 -88.20679031282147) bank141907 +141908 POINT(41.9880830054167 -87.7694333770128) bank141908 +141909 POINT(42.04954058544629 -86.63459496932725) bank141909 +141910 POINT(41.385017612432506 -87.65094625797778) bank141910 +141911 POINT(41.480717626422134 -87.28738163858993) bank141911 +141912 POINT(42.20860778234137 -87.73304742375062) bank141912 +141913 POINT(41.47927471600768 -88.08128209918254) bank141913 +141914 POINT(41.27967686590847 -88.44166828780003) bank141914 +141915 POINT(42.420696615301225 -87.95457343249899) bank141915 +141916 POINT(40.899148639702695 -88.14718672877176) bank141916 +141917 POINT(41.47419404549263 -88.58637006785739) bank141917 +141918 POINT(41.83657428032724 -87.63710829497896) bank141918 +141919 POINT(42.40999389363387 -87.019063599645) bank141919 +141920 POINT(42.25695732206489 -87.09863149364054) bank141920 +141921 POINT(40.96449613440343 -87.26888423426288) bank141921 +141922 POINT(42.7935434056797 -86.68794425415184) bank141922 +141923 POINT(41.476579755553864 -87.47748734128788) bank141923 +141924 POINT(41.70483615895982 -87.95689024460812) bank141924 +141925 POINT(41.370198825914485 -88.09859152475934) bank141925 +141926 POINT(41.96569596226458 -88.15428040540513) bank141926 +141927 POINT(40.997324690977585 -88.41924777519701) bank141927 +141928 POINT(41.466649184628146 -87.28946693612424) bank141928 +141929 POINT(42.5177705072405 -86.97396223641402) bank141929 +141930 POINT(41.065595487434315 -86.98090497966578) bank141930 +141931 POINT(41.83543382711408 -87.20168901917391) bank141931 +141932 POINT(42.38913996405545 -88.18773504862091) bank141932 +141933 POINT(41.591361558024744 -88.54727981746565) bank141933 +141934 POINT(41.20657830208187 -88.62795062362035) bank141934 +141935 POINT(42.75548904177927 -87.13391531403246) bank141935 +141936 POINT(41.265271317434376 -88.52895458510864) bank141936 +141937 POINT(42.01162312149386 -86.99320238135464) bank141937 +141938 POINT(42.68067754785276 -87.6006861376686) bank141938 +141939 POINT(41.28268356538027 -87.2014444263031) bank141939 +141940 POINT(41.30932064841166 -87.24833227869807) bank141940 +141941 POINT(41.28439158629951 -88.49322152552573) bank141941 +141942 POINT(42.75442630540195 -87.98623304152086) bank141942 +141943 POINT(42.5018667244207 -86.67241038623482) bank141943 +141944 POINT(41.173798954961796 -87.93358638205072) bank141944 +141945 POINT(42.71474166501834 -87.8728052728845) bank141945 +141946 POINT(41.461549374680885 -88.61808678409255) bank141946 +141947 POINT(41.671653908951114 -87.46980145260203) bank141947 +141948 POINT(41.75397128674089 -86.94321209618873) bank141948 +141949 POINT(41.142496169919525 -88.50668299468359) bank141949 +141950 POINT(41.43272883478925 -88.15834949028296) bank141950 +141951 POINT(42.70728510733779 -86.89527878077608) bank141951 +141952 POINT(41.018042751999516 -88.2842658420584) bank141952 +141953 POINT(42.45300722920522 -87.85440842259646) bank141953 +141954 POINT(41.933591608743086 -87.60535139193706) bank141954 +141955 POINT(41.31692813393708 -87.00496419575937) bank141955 +141956 POINT(42.56436277029831 -87.60970315716817) bank141956 +141957 POINT(42.51692452996669 -87.144827700832) bank141957 +141958 POINT(41.15433591743485 -86.84972609084987) bank141958 +141959 POINT(42.700482559554274 -88.61376800311936) bank141959 +141960 POINT(41.493746457475304 -87.33850346186546) bank141960 +141961 POINT(42.15857517702974 -87.32458708990715) bank141961 +141962 POINT(40.93786301051152 -86.74198036124204) bank141962 +141963 POINT(42.78682199917775 -87.3187908616502) bank141963 +141964 POINT(42.66501918444945 -86.95453027596253) bank141964 +141965 POINT(41.56882171773396 -88.01381872948329) bank141965 +141966 POINT(42.69616156871887 -88.38733574122186) bank141966 +141967 POINT(42.57928470446309 -88.2651917060226) bank141967 +141968 POINT(41.47998784474732 -88.56580575968664) bank141968 +141969 POINT(41.03548955358848 -87.47893898526061) bank141969 +141970 POINT(42.70527014174094 -88.11315428819309) bank141970 +141971 POINT(41.55326586459201 -88.48977829792118) bank141971 +141972 POINT(42.3568074228695 -87.24623277422312) bank141972 +141973 POINT(41.734978864766425 -87.26768877445946) bank141973 +141974 POINT(41.821672042101 -88.54731084965192) bank141974 +141975 POINT(42.050480747578234 -87.25364415264544) bank141975 +141976 POINT(42.05940950294858 -87.33625993096827) bank141976 +141977 POINT(42.31901018854216 -87.99896822168621) bank141977 +141978 POINT(41.54492563315319 -87.14275260911423) bank141978 +141979 POINT(41.040269749809646 -88.50717616703774) bank141979 +141980 POINT(41.00714956781316 -87.32813464780615) bank141980 +141981 POINT(42.392905568975294 -88.07338329477061) bank141981 +141982 POINT(40.92730242459562 -86.75066225342641) bank141982 +141983 POINT(42.210795011826164 -87.59717430989785) bank141983 +141984 POINT(41.29359854603987 -88.00013580062308) bank141984 +141985 POINT(42.08436848025315 -87.71594471735217) bank141985 +141986 POINT(42.30953967646632 -88.42529666293918) bank141986 +141987 POINT(42.21224721526763 -86.73362533338569) bank141987 +141988 POINT(41.353106251961144 -87.07163479657753) bank141988 +141989 POINT(42.15784480478017 -87.64768023329613) bank141989 +141990 POINT(41.349269286069664 -88.50490998167467) bank141990 +141991 POINT(42.32364838558396 -88.26784501757099) bank141991 +141992 POINT(41.28185147809983 -86.68754159871838) bank141992 +141993 POINT(41.00748632571919 -87.97086501978596) bank141993 +141994 POINT(41.98134189961865 -87.89240124312255) bank141994 +141995 POINT(42.105739412802954 -86.9879203509189) bank141995 +141996 POINT(42.50804522282576 -87.74366358290415) bank141996 +141997 POINT(41.32583233537852 -87.89977032326891) bank141997 +141998 POINT(41.625811285855356 -87.59810355440622) bank141998 +141999 POINT(42.3089459533923 -88.2963657736726) bank141999 +142000 POINT(42.698187397216294 -86.86818791698873) bank142000 +142001 POINT(41.08040037282594 -87.67957661309461) bank142001 +142002 POINT(41.750150349171356 -88.2855810975863) bank142002 +142003 POINT(42.37024456013908 -86.87024430970482) bank142003 +142004 POINT(40.94936637275627 -87.38763910675033) bank142004 +142005 POINT(42.10105822966091 -87.78926228329759) bank142005 +142006 POINT(41.963181757626984 -87.26606488750437) bank142006 +142007 POINT(42.84977082101178 -87.16729490910353) bank142007 +142008 POINT(42.29005044272037 -86.99782784783885) bank142008 +142009 POINT(42.35892640609797 -87.52383225935893) bank142009 +142010 POINT(42.474976992249765 -88.54268388944276) bank142010 +142011 POINT(41.978904022981204 -88.6214059208374) bank142011 +142012 POINT(41.787873648021716 -87.98426687464264) bank142012 +142013 POINT(41.97150784712658 -88.01180647474325) bank142013 +142014 POINT(42.67397101855091 -86.66849595895573) bank142014 +142015 POINT(41.26552646389062 -87.55106016939692) bank142015 +142016 POINT(40.88516457853525 -87.44959407955204) bank142016 +142017 POINT(42.48085277908421 -87.22700543805394) bank142017 +142018 POINT(42.01975428607492 -87.95078645991859) bank142018 +142019 POINT(42.548046703350096 -87.61129196116129) bank142019 +142020 POINT(41.994975316749475 -87.65972212329892) bank142020 +142021 POINT(41.09496089830811 -86.6370186731804) bank142021 +142022 POINT(42.83906909008448 -87.01586198947786) bank142022 +142023 POINT(41.280803359238995 -87.92481868751537) bank142023 +142024 POINT(42.60191116569786 -88.3458364696173) bank142024 +142025 POINT(42.56490207775441 -87.05387731372917) bank142025 +142026 POINT(42.313461228880044 -87.3514125107098) bank142026 +142027 POINT(42.17273425198534 -86.76441363246438) bank142027 +142028 POINT(41.92159538934003 -88.62120379997495) bank142028 +142029 POINT(40.98551008812087 -87.8506817690445) bank142029 +142030 POINT(41.221559191822934 -86.7459877226335) bank142030 +142031 POINT(42.130595207463024 -87.78713646395968) bank142031 +142032 POINT(41.29024781445084 -87.49155034716135) bank142032 +142033 POINT(42.00419716315449 -88.08133110279734) bank142033 +142034 POINT(42.426433020487046 -86.95661982713037) bank142034 +142035 POINT(40.98637462092398 -88.47906860903086) bank142035 +142036 POINT(41.743273407110955 -86.86849385607289) bank142036 +142037 POINT(42.475789980433895 -88.2676126682806) bank142037 +142038 POINT(42.6858184546543 -88.40320818457944) bank142038 +142039 POINT(42.16799890574216 -86.73020178876214) bank142039 +142040 POINT(41.1644585438962 -86.74070723832061) bank142040 +142041 POINT(42.64514418175845 -88.22772157628087) bank142041 +142042 POINT(42.115340197192666 -87.48944324403571) bank142042 +142043 POINT(42.06843039456935 -86.72362065555096) bank142043 +142044 POINT(41.891406068904146 -87.59360993185109) bank142044 +142045 POINT(41.13169511380188 -87.06953668132486) bank142045 +142046 POINT(42.83399050157608 -86.83787163613094) bank142046 +142047 POINT(41.067842702720604 -87.36103544095087) bank142047 +142048 POINT(40.924277473685514 -86.65422693448939) bank142048 +142049 POINT(42.793464968029554 -87.3681508622574) bank142049 +142050 POINT(42.208191176770484 -88.44496625954622) bank142050 +142051 POINT(40.96686449470842 -86.72810356501913) bank142051 +142052 POINT(42.56161696930386 -87.48985875647226) bank142052 +142053 POINT(41.0384064539158 -87.81430992504559) bank142053 +142054 POINT(41.591406129157434 -88.22772198774402) bank142054 +142055 POINT(42.39807694289891 -87.00821209596458) bank142055 +142056 POINT(42.690221271212 -88.10982295115882) bank142056 +142057 POINT(42.00755321631588 -87.34362214656659) bank142057 +142058 POINT(42.56027154133534 -87.41839183826028) bank142058 +142059 POINT(41.44428958362021 -88.35454886060865) bank142059 +142060 POINT(41.16373264655479 -87.80573933329974) bank142060 +142061 POINT(41.68618843225396 -87.98157462204483) bank142061 +142062 POINT(41.553556210912234 -87.62747515427809) bank142062 +142063 POINT(41.34620668735821 -87.24503412785909) bank142063 +142064 POINT(41.28963473911023 -88.12683183219825) bank142064 +142065 POINT(41.9370230814911 -87.5717511982511) bank142065 +142066 POINT(41.46815295069011 -87.44591299736851) bank142066 +142067 POINT(42.41777919770685 -87.01381538630346) bank142067 +142068 POINT(41.89416821547393 -87.0698799898092) bank142068 +142069 POINT(42.70551040155926 -88.43429169020368) bank142069 +142070 POINT(41.366170620936096 -87.74972725835521) bank142070 +142071 POINT(42.63650099483768 -88.57663522665877) bank142071 +142072 POINT(41.23363394546418 -87.57734419208239) bank142072 +142073 POINT(41.68398755245394 -87.39734688958723) bank142073 +142074 POINT(42.30059671017556 -87.63047360405497) bank142074 +142075 POINT(42.81974195317193 -87.9428282686885) bank142075 +142076 POINT(42.87762901891196 -87.61151621518377) bank142076 +142077 POINT(40.88521842139869 -86.9058441907745) bank142077 +142078 POINT(41.14757795472277 -87.29156840858658) bank142078 +142079 POINT(41.46068285025531 -87.414673436436) bank142079 +142080 POINT(41.34903154271247 -87.17280745063535) bank142080 +142081 POINT(42.00506117449918 -88.21835567012138) bank142081 +142082 POINT(41.6167877366012 -88.3398774744132) bank142082 +142083 POINT(41.69384532092424 -88.50604384055079) bank142083 +142084 POINT(42.16933282466076 -88.31162225781442) bank142084 +142085 POINT(42.3568105015114 -87.98756581278377) bank142085 +142086 POINT(41.954291216509425 -87.85983790682461) bank142086 +142087 POINT(42.6559198226283 -86.9070119602781) bank142087 +142088 POINT(42.363872978124036 -88.60331542234648) bank142088 +142089 POINT(42.05791604002747 -88.21317368670013) bank142089 +142090 POINT(42.07917626593104 -88.4848009487994) bank142090 +142091 POINT(41.93729927950623 -87.23483005613618) bank142091 +142092 POINT(42.17464606885033 -86.86422746588357) bank142092 +142093 POINT(41.90193818247195 -88.03176570614079) bank142093 +142094 POINT(40.903617826829816 -87.65867980959392) bank142094 +142095 POINT(41.50550375250731 -86.91865947412298) bank142095 +142096 POINT(41.82819358171512 -87.81251450566964) bank142096 +142097 POINT(41.784764478432244 -87.67852123062552) bank142097 +142098 POINT(41.19306935110825 -87.15043007594406) bank142098 +142099 POINT(42.19098381688865 -87.62915295764485) bank142099 +142100 POINT(42.7324454552053 -86.85886707575672) bank142100 +142101 POINT(41.249036931539216 -87.29565693067147) bank142101 +142102 POINT(41.272955990743995 -88.59750481419063) bank142102 +142103 POINT(41.89638657675937 -88.55346410013142) bank142103 +142104 POINT(42.05390977918062 -86.91501900978616) bank142104 +142105 POINT(40.96742199007651 -86.82543754586636) bank142105 +142106 POINT(42.17653927708756 -87.63706280488289) bank142106 +142107 POINT(41.122769768496866 -88.17952371756017) bank142107 +142108 POINT(41.94035837114325 -87.1247310965582) bank142108 +142109 POINT(40.96283411697445 -87.67046781916964) bank142109 +142110 POINT(42.29557663610775 -87.38613017895277) bank142110 +142111 POINT(41.830209233980014 -87.89806809311415) bank142111 +142112 POINT(41.84054187288615 -86.8306779503588) bank142112 +142113 POINT(41.90637425754795 -87.23793284869824) bank142113 +142114 POINT(41.192251170705404 -88.42563301049422) bank142114 +142115 POINT(41.08695983628637 -88.08896628966266) bank142115 +142116 POINT(42.25688721596407 -87.16047863733077) bank142116 +142117 POINT(42.46316911003736 -87.29435558301705) bank142117 +142118 POINT(41.94868086338384 -87.3409963443334) bank142118 +142119 POINT(42.85791302139685 -86.72153730156992) bank142119 +142120 POINT(42.270410505474764 -88.20690530572345) bank142120 +142121 POINT(42.284647204531424 -87.12732769162535) bank142121 +142122 POINT(41.86364085227889 -88.58866284253574) bank142122 +142123 POINT(41.92197822861616 -87.00815784101172) bank142123 +142124 POINT(42.127896923140604 -87.85940180918655) bank142124 +142125 POINT(42.326571112853344 -88.37281694309021) bank142125 +142126 POINT(42.678368921734595 -87.2213827695906) bank142126 +142127 POINT(41.09339175789771 -88.22899513391656) bank142127 +142128 POINT(42.56924312226174 -88.42887225224307) bank142128 +142129 POINT(41.90321560297154 -86.8231634219687) bank142129 +142130 POINT(42.66132465442219 -86.81825675270416) bank142130 +142131 POINT(42.517470832803994 -88.02832082726583) bank142131 +142132 POINT(41.95050025700146 -87.52302415516236) bank142132 +142133 POINT(41.88335903000022 -87.89848138921948) bank142133 +142134 POINT(41.67285582544599 -87.46653304762164) bank142134 +142135 POINT(42.66740902178056 -87.63146672044877) bank142135 +142136 POINT(42.71273553520063 -87.34127957801397) bank142136 +142137 POINT(41.143256364439026 -87.52433077907745) bank142137 +142138 POINT(41.03348745389251 -86.97441698282083) bank142138 +142139 POINT(42.40104108554021 -88.24225844028834) bank142139 +142140 POINT(40.969888016851876 -87.58560933685366) bank142140 +142141 POINT(42.45950094796145 -88.0428179209809) bank142141 +142142 POINT(41.36970239774912 -87.52791975163527) bank142142 +142143 POINT(41.86185754904634 -88.09289838686244) bank142143 +142144 POINT(41.79004171260289 -87.36510350761097) bank142144 +142145 POINT(42.864529868256675 -88.27889522065247) bank142145 +142146 POINT(42.8580323742368 -88.0335642096548) bank142146 +142147 POINT(41.16720520064341 -87.21943956799831) bank142147 +142148 POINT(42.830939525924286 -87.3551541986105) bank142148 +142149 POINT(41.42561242733131 -87.25469291128016) bank142149 +142150 POINT(42.459140250546234 -88.11989969443712) bank142150 +142151 POINT(42.10713096173964 -87.66109015528808) bank142151 +142152 POINT(42.183530881888636 -87.72260696578465) bank142152 +142153 POINT(42.46533503934069 -88.1794732218115) bank142153 +142154 POINT(41.30129006117287 -87.42072613771236) bank142154 +142155 POINT(40.901644430928584 -87.5177377365737) bank142155 +142156 POINT(42.305344808687 -88.39881133418453) bank142156 +142157 POINT(40.930710401278596 -87.99572928859111) bank142157 +142158 POINT(41.42913658288803 -87.79165589468747) bank142158 +142159 POINT(41.4723750894955 -88.16732036650913) bank142159 +142160 POINT(42.59202816345391 -86.93438730533454) bank142160 +142161 POINT(41.12612820254834 -87.49417316722293) bank142161 +142162 POINT(41.02058846173855 -86.66137369257707) bank142162 +142163 POINT(42.657679421611974 -87.19680327424452) bank142163 +142164 POINT(41.5492803447384 -87.31830444575414) bank142164 +142165 POINT(42.18303055205783 -88.51658553424333) bank142165 +142166 POINT(42.52075550704611 -87.30664849859295) bank142166 +142167 POINT(41.87706024887969 -87.03097728371927) bank142167 +142168 POINT(41.929541468423004 -87.53542563130796) bank142168 +142169 POINT(42.25212003547512 -87.06266740421766) bank142169 +142170 POINT(41.96066105729301 -87.49021543788187) bank142170 +142171 POINT(42.01975862616949 -88.11115010274061) bank142171 +142172 POINT(41.57056349869873 -87.0504140013985) bank142172 +142173 POINT(41.927829500908715 -88.30172124407089) bank142173 +142174 POINT(42.77965339925157 -86.81224320069677) bank142174 +142175 POINT(40.90806336846454 -87.66684697862314) bank142175 +142176 POINT(42.82199313373015 -88.18497101410088) bank142176 +142177 POINT(41.16251104575451 -87.2721029826076) bank142177 +142178 POINT(42.80444106743842 -88.22691972974215) bank142178 +142179 POINT(41.477759254039995 -86.97674426019898) bank142179 +142180 POINT(42.639733591463155 -88.27893387160182) bank142180 +142181 POINT(41.394525015570316 -87.08935778803595) bank142181 +142182 POINT(42.055251656195125 -86.65858364114129) bank142182 +142183 POINT(42.79833786808811 -86.73038512805199) bank142183 +142184 POINT(41.07643561347317 -88.04951392461808) bank142184 +142185 POINT(41.100558435707654 -88.44320568991388) bank142185 +142186 POINT(41.2522405926647 -87.19532173433963) bank142186 +142187 POINT(42.14318542107862 -86.88058853373083) bank142187 +142188 POINT(42.17582540870037 -87.11005559114304) bank142188 +142189 POINT(41.587460327130835 -87.19944711716595) bank142189 +142190 POINT(41.151024661052745 -86.77859308769207) bank142190 +142191 POINT(41.30461415174003 -87.48924046909292) bank142191 +142192 POINT(42.00502230343882 -88.25661122472593) bank142192 +142193 POINT(41.02111032094296 -87.26548963042256) bank142193 +142194 POINT(42.12238819935485 -87.66761131968077) bank142194 +142195 POINT(41.79611634256115 -87.60501357469074) bank142195 +142196 POINT(42.200316586861305 -88.38661152576935) bank142196 +142197 POINT(41.490773718990376 -87.18073517018996) bank142197 +142198 POINT(41.93523723792833 -87.51392368258814) bank142198 +142199 POINT(42.568542678862805 -87.74659803474074) bank142199 +142200 POINT(42.87097784842216 -86.67122702934412) bank142200 +142201 POINT(42.265111834885886 -87.25160576685228) bank142201 +142202 POINT(41.597297099885154 -87.40491531651124) bank142202 +142203 POINT(42.090935082920694 -87.41144458246907) bank142203 +142204 POINT(41.2824340342037 -87.95078841420404) bank142204 +142205 POINT(41.32357278688184 -87.28874777069719) bank142205 +142206 POINT(41.62167725237311 -86.66020838775586) bank142206 +142207 POINT(41.68448529081383 -87.28022366885122) bank142207 +142208 POINT(40.952603159408184 -88.31853900291632) bank142208 +142209 POINT(41.41297942582568 -86.91561759562643) bank142209 +142210 POINT(42.4102170221489 -88.29940447096351) bank142210 +142211 POINT(41.55017004995295 -87.07054212297798) bank142211 +142212 POINT(40.977912566602804 -88.28692743022387) bank142212 +142213 POINT(41.13877261371519 -87.63488461664903) bank142213 +142214 POINT(41.49636256769054 -87.89924484035748) bank142214 +142215 POINT(42.777528032208174 -88.56848648568052) bank142215 +142216 POINT(41.497118275324 -87.86615871320413) bank142216 +142217 POINT(41.799463389497376 -87.00121947937312) bank142217 +142218 POINT(42.44896733376219 -87.9658315252403) bank142218 +142219 POINT(42.72940362131858 -87.74922076134597) bank142219 +142220 POINT(41.10470352982057 -86.92899385313356) bank142220 +142221 POINT(42.00351757336055 -87.50105671543464) bank142221 +142222 POINT(42.10902664715864 -87.95690541393084) bank142222 +142223 POINT(42.22829215875298 -87.16258991792768) bank142223 +142224 POINT(42.38878078609017 -87.00798587699863) bank142224 +142225 POINT(41.8874590692553 -86.7411803604763) bank142225 +142226 POINT(42.61202741907668 -88.19190546942731) bank142226 +142227 POINT(42.79947238025733 -88.46158832714738) bank142227 +142228 POINT(42.66040501018481 -88.29608956217173) bank142228 +142229 POINT(42.7281716618208 -87.90146062764165) bank142229 +142230 POINT(42.79880915122379 -88.5819821523802) bank142230 +142231 POINT(41.254056642537556 -86.95486409637145) bank142231 +142232 POINT(41.859949256621945 -87.32623984069662) bank142232 +142233 POINT(42.80242885522488 -86.65274205850707) bank142233 +142234 POINT(41.34033497704264 -86.9975056824825) bank142234 +142235 POINT(42.650183946424576 -88.43743254713762) bank142235 +142236 POINT(42.327941646677985 -87.48111043070638) bank142236 +142237 POINT(42.14021715455771 -87.44301646347132) bank142237 +142238 POINT(41.36107463717776 -87.01609284370593) bank142238 +142239 POINT(42.74028901940273 -87.15463879888442) bank142239 +142240 POINT(42.03129191054706 -87.30946597555132) bank142240 +142241 POINT(42.52761947157707 -87.46759691528418) bank142241 +142242 POINT(42.69323357237024 -86.71925372687444) bank142242 +142243 POINT(41.59149184887889 -88.01661984645442) bank142243 +142244 POINT(42.551900280792886 -86.67176512860871) bank142244 +142245 POINT(40.91062683601761 -88.10911215879554) bank142245 +142246 POINT(41.68636767498714 -87.68226991202509) bank142246 +142247 POINT(40.916050068019274 -87.20077596724086) bank142247 +142248 POINT(41.67522401932908 -87.81256253839823) bank142248 +142249 POINT(42.81796765853236 -87.19946635638348) bank142249 +142250 POINT(42.521744422662394 -88.45943810916077) bank142250 +142251 POINT(41.50425644673775 -88.45118392548589) bank142251 +142252 POINT(41.873469006908316 -87.48160740687608) bank142252 +142253 POINT(42.508863187018186 -87.21891043265009) bank142253 +142254 POINT(41.896662458583464 -87.15105629277642) bank142254 +142255 POINT(41.18520757475638 -88.01470682549224) bank142255 +142256 POINT(41.15010144931298 -87.62252396457643) bank142256 +142257 POINT(42.832105371211156 -87.86712648475113) bank142257 +142258 POINT(41.902809557314136 -87.80410153150167) bank142258 +142259 POINT(41.27048861087255 -86.84815310931883) bank142259 +142260 POINT(41.04944685978377 -87.84203209770979) bank142260 +142261 POINT(42.557954863278155 -86.69601105640253) bank142261 +142262 POINT(41.5348834621302 -88.05984312786516) bank142262 +142263 POINT(41.07039419137704 -87.54519765357686) bank142263 +142264 POINT(41.766496455176586 -87.61373678067157) bank142264 +142265 POINT(41.62233621537736 -86.95515400638634) bank142265 +142266 POINT(41.67365136993219 -87.39164398232262) bank142266 +142267 POINT(42.72022107287325 -87.7136222851741) bank142267 +142268 POINT(41.06698522831953 -86.93275668088273) bank142268 +142269 POINT(42.651213480146986 -87.62703553524928) bank142269 +142270 POINT(42.72314427323111 -86.72743710407552) bank142270 +142271 POINT(41.86109837957329 -86.77741230557471) bank142271 +142272 POINT(41.582049606355326 -86.66200721786451) bank142272 +142273 POINT(42.07267994236782 -86.7372332522822) bank142273 +142274 POINT(42.18890829848411 -88.25126295475971) bank142274 +142275 POINT(41.87844308781576 -86.74273681002013) bank142275 +142276 POINT(41.01321780264944 -88.50156442526783) bank142276 +142277 POINT(42.70401026631738 -86.67548460855458) bank142277 +142278 POINT(42.63575267562416 -88.00512529248434) bank142278 +142279 POINT(41.7035111010426 -86.79807701967194) bank142279 +142280 POINT(41.48928460361954 -87.3676385774616) bank142280 +142281 POINT(42.292016265508956 -88.20851442058957) bank142281 +142282 POINT(42.2529917197566 -88.10916522248166) bank142282 +142283 POINT(41.87853700683792 -87.01823225391182) bank142283 +142284 POINT(41.16735089424982 -87.86845013110567) bank142284 +142285 POINT(41.085250716753706 -88.0576853351186) bank142285 +142286 POINT(41.8735663518311 -86.89140464093303) bank142286 +142287 POINT(41.483030033772586 -87.30408416254414) bank142287 +142288 POINT(42.3883025463654 -88.40345218258445) bank142288 +142289 POINT(42.762581793763175 -87.517865594401) bank142289 +142290 POINT(41.27942508212119 -88.32187306825648) bank142290 +142291 POINT(42.03715328166829 -87.0603669847056) bank142291 +142292 POINT(41.373733636823864 -86.63369512083123) bank142292 +142293 POINT(42.26839609503003 -87.77321686523561) bank142293 +142294 POINT(41.73157160123755 -88.43757558132799) bank142294 +142295 POINT(42.497857368879004 -87.99520178889584) bank142295 +142296 POINT(41.21464974229889 -87.5817171121964) bank142296 +142297 POINT(41.81981797050519 -87.46461094243189) bank142297 +142298 POINT(41.919724341381105 -88.18252336615188) bank142298 +142299 POINT(41.680421562897926 -87.43578051906552) bank142299 +142300 POINT(42.54677168733299 -88.55156561014287) bank142300 +142301 POINT(40.91846152109225 -87.50878156783315) bank142301 +142302 POINT(41.19442343303018 -88.52061914317227) bank142302 +142303 POINT(41.3670705721358 -87.87604157983462) bank142303 +142304 POINT(41.21050718106658 -87.1783830673064) bank142304 +142305 POINT(42.87226278731454 -86.91101501889213) bank142305 +142306 POINT(42.34782680719054 -87.40721770351617) bank142306 +142307 POINT(42.17264022968634 -88.06936126109771) bank142307 +142308 POINT(41.500057596310306 -87.68515667598633) bank142308 +142309 POINT(41.45201232310714 -88.3631966241009) bank142309 +142310 POINT(42.5114943342475 -87.783501983734) bank142310 +142311 POINT(41.29491651567459 -87.24107609412697) bank142311 +142312 POINT(42.320945679684534 -87.62322911005931) bank142312 +142313 POINT(41.3114713126282 -87.56970851578632) bank142313 +142314 POINT(42.2861381397142 -87.54751043938786) bank142314 +142315 POINT(42.67755269051703 -87.84090144678201) bank142315 +142316 POINT(41.22038203557963 -88.32878254563694) bank142316 +142317 POINT(41.628787926978056 -87.2026621040879) bank142317 +142318 POINT(41.99071351409096 -87.54810127013873) bank142318 +142319 POINT(41.22669216082453 -87.73213521609733) bank142319 +142320 POINT(41.81629374249853 -87.61278035672142) bank142320 +142321 POINT(42.21746227401235 -88.44343612642325) bank142321 +142322 POINT(42.45768098544486 -87.89207844033058) bank142322 +142323 POINT(41.992618687503764 -87.76196863848787) bank142323 +142324 POINT(41.05244708682397 -87.5169631457522) bank142324 +142325 POINT(42.6215158699828 -88.11214737089118) bank142325 +142326 POINT(42.543476558799284 -87.45205494455742) bank142326 +142327 POINT(42.75285666422668 -86.76435791742064) bank142327 +142328 POINT(41.12271863160774 -87.32016155339649) bank142328 +142329 POINT(41.15958601683492 -87.80866271102866) bank142329 +142330 POINT(42.33347973630098 -86.65134111675657) bank142330 +142331 POINT(42.08536191115036 -88.05592681053652) bank142331 +142332 POINT(42.13340230315533 -87.22354711194282) bank142332 +142333 POINT(41.91442062590645 -87.82878026025978) bank142333 +142334 POINT(41.87948091018543 -87.57550637067214) bank142334 +142335 POINT(41.173684540155634 -88.62625891717316) bank142335 +142336 POINT(41.32251431694372 -87.27870201642305) bank142336 +142337 POINT(42.81124001065851 -88.3717896556322) bank142337 +142338 POINT(41.35185277121315 -88.47043197496404) bank142338 +142339 POINT(41.21793857465095 -87.40550230369327) bank142339 +142340 POINT(41.29573534313525 -87.71849167476032) bank142340 +142341 POINT(41.184319075182465 -87.21217454408472) bank142341 +142342 POINT(42.20863604925233 -87.74491431774561) bank142342 +142343 POINT(42.79379765602403 -87.4625707308584) bank142343 +142344 POINT(41.545020845060414 -87.18514078245389) bank142344 +142345 POINT(42.211606456564915 -88.5642296169571) bank142345 +142346 POINT(41.052447461008 -87.83423278777184) bank142346 +142347 POINT(42.50337156296829 -86.90560459612973) bank142347 +142348 POINT(41.79564671614075 -87.07226171856698) bank142348 +142349 POINT(42.85690712490452 -86.83897820109922) bank142349 +142350 POINT(42.736105845557816 -88.09486742202577) bank142350 +142351 POINT(41.04778501302507 -87.98084236851409) bank142351 +142352 POINT(41.234206745868065 -88.43413833458251) bank142352 +142353 POINT(42.02441070863157 -86.85863145591753) bank142353 +142354 POINT(41.50680323450748 -86.83644852766685) bank142354 +142355 POINT(42.373430582991396 -88.5098660253064) bank142355 +142356 POINT(41.129488756132695 -86.95703403919501) bank142356 +142357 POINT(40.956775346247184 -87.23406623236397) bank142357 +142358 POINT(40.91163738659911 -87.39460968269641) bank142358 +142359 POINT(41.70232071663286 -87.60749525913819) bank142359 +142360 POINT(42.400659534910744 -86.92299560483254) bank142360 +142361 POINT(41.11668167116794 -87.32558964145173) bank142361 +142362 POINT(42.23354261400834 -87.38843918300353) bank142362 +142363 POINT(41.680152395692616 -87.97910916913428) bank142363 +142364 POINT(41.895745678419416 -87.86800373816395) bank142364 +142365 POINT(42.097617517949296 -86.88439671348058) bank142365 +142366 POINT(41.74891442070892 -87.63349081165255) bank142366 +142367 POINT(41.4295894803504 -88.0917186674031) bank142367 +142368 POINT(41.90712526885221 -88.42014720451597) bank142368 +142369 POINT(41.12231777471725 -86.90792671540193) bank142369 +142370 POINT(41.246021893362126 -87.2261844640032) bank142370 +142371 POINT(42.135025429770515 -87.38074337505891) bank142371 +142372 POINT(41.71311450089224 -88.50911732199626) bank142372 +142373 POINT(41.03827713848024 -87.9336664086333) bank142373 +142374 POINT(41.028982372380035 -87.88447919105823) bank142374 +142375 POINT(41.44853696419177 -87.89392969681737) bank142375 +142376 POINT(42.860063501978665 -86.7813545274866) bank142376 +142377 POINT(41.72748243816281 -88.3517431158898) bank142377 +142378 POINT(41.656903110749326 -88.56212941691639) bank142378 +142379 POINT(41.404401502794144 -87.37036328495037) bank142379 +142380 POINT(41.194042602428716 -88.37856065419432) bank142380 +142381 POINT(42.38808470762575 -87.21069981966917) bank142381 +142382 POINT(42.63368317537515 -88.4715814659424) bank142382 +142383 POINT(40.91884589402088 -87.25069598242479) bank142383 +142384 POINT(41.404914139179645 -86.99726151592417) bank142384 +142385 POINT(41.853286668375745 -87.5988511042265) bank142385 +142386 POINT(41.61537345996656 -87.28332542208038) bank142386 +142387 POINT(41.082237909871495 -87.88949339061857) bank142387 +142388 POINT(41.00139065933138 -87.34229394654707) bank142388 +142389 POINT(41.35529913556839 -88.02517765484882) bank142389 +142390 POINT(40.91595195631985 -87.47326275781526) bank142390 +142391 POINT(41.376197398125825 -87.98186016905227) bank142391 +142392 POINT(41.11088660878012 -87.10560860290757) bank142392 +142393 POINT(42.31431043821479 -87.03245678194254) bank142393 +142394 POINT(41.92458112862148 -87.3424247564854) bank142394 +142395 POINT(41.94239475304437 -88.09386446815851) bank142395 +142396 POINT(42.83260955494934 -87.093025745781) bank142396 +142397 POINT(41.304418626594355 -86.96057560898826) bank142397 +142398 POINT(42.5927999942339 -87.78136323478915) bank142398 +142399 POINT(41.329802032431886 -87.06154014364115) bank142399 +142400 POINT(42.154003656326935 -87.00536611006831) bank142400 +142401 POINT(40.98298630225346 -87.65736938039589) bank142401 +142402 POINT(42.42033149158996 -88.05549933240279) bank142402 +142403 POINT(42.86572749597502 -87.36382555279323) bank142403 +142404 POINT(42.14465857179593 -88.30361399507787) bank142404 +142405 POINT(41.99534504598357 -86.96163951460112) bank142405 +142406 POINT(42.67227198395927 -88.11064551459174) bank142406 +142407 POINT(41.69387855811732 -87.59970045721887) bank142407 +142408 POINT(40.902214091302945 -87.91980350086924) bank142408 +142409 POINT(42.53731248699299 -86.91843383579705) bank142409 +142410 POINT(41.68274497190092 -87.66231605299306) bank142410 +142411 POINT(41.54163723535286 -86.75880590121132) bank142411 +142412 POINT(41.478364397398465 -88.56986961640969) bank142412 +142413 POINT(41.7027638530561 -87.55012967215481) bank142413 +142414 POINT(41.57869232863773 -86.99956818228003) bank142414 +142415 POINT(41.8060468418279 -88.17086320671403) bank142415 +142416 POINT(42.04947020847219 -87.35321558363866) bank142416 +142417 POINT(41.26298472457391 -87.825712481414) bank142417 +142418 POINT(41.19309158448234 -86.90503980755004) bank142418 +142419 POINT(41.896776908791715 -87.49739810918408) bank142419 +142420 POINT(42.80293794905346 -86.75546785090955) bank142420 +142421 POINT(41.78862698578655 -86.70547543113028) bank142421 +142422 POINT(42.6484187531059 -87.98724294053164) bank142422 +142423 POINT(42.762155970021276 -86.86283129415929) bank142423 +142424 POINT(42.81452272841839 -86.7338035467401) bank142424 +142425 POINT(42.829381708042575 -87.62588223501005) bank142425 +142426 POINT(42.402439067332466 -88.29645437762076) bank142426 +142427 POINT(40.894494639354896 -87.61181454753576) bank142427 +142428 POINT(42.795316507535546 -88.61716628514239) bank142428 +142429 POINT(40.9541433964659 -87.45733011539052) bank142429 +142430 POINT(41.35286333140939 -87.84907672487444) bank142430 +142431 POINT(41.16223250120903 -88.35413486841261) bank142431 +142432 POINT(42.06757032797963 -86.79954809741149) bank142432 +142433 POINT(42.84953107235599 -86.92941141152365) bank142433 +142434 POINT(41.13624641779401 -88.30265247681852) bank142434 +142435 POINT(42.09212248267274 -87.29375721945922) bank142435 +142436 POINT(41.827193809616624 -86.9395549255459) bank142436 +142437 POINT(41.50686774005445 -88.34541425347095) bank142437 +142438 POINT(41.09333378546648 -87.00563495690925) bank142438 +142439 POINT(41.812365357207554 -88.17282418943833) bank142439 +142440 POINT(41.898005664622474 -88.44150370085426) bank142440 +142441 POINT(41.57529738061104 -86.93075132944765) bank142441 +142442 POINT(42.240690577564145 -87.70977749805115) bank142442 +142443 POINT(42.3427120587581 -87.12603318182133) bank142443 +142444 POINT(42.58011473869788 -88.48165504543735) bank142444 +142445 POINT(42.43701272280543 -88.39516657713231) bank142445 +142446 POINT(41.06496190689843 -88.25768924095101) bank142446 +142447 POINT(42.4691551385575 -87.72957538278547) bank142447 +142448 POINT(41.74626217948756 -87.26395088530685) bank142448 +142449 POINT(42.11254048838918 -87.82034720965333) bank142449 +142450 POINT(41.48647804912384 -88.59314261438283) bank142450 +142451 POINT(42.751579657347015 -86.98854408501319) bank142451 +142452 POINT(42.74226068374931 -87.24651715847791) bank142452 +142453 POINT(42.00511582817047 -88.56373993646437) bank142453 +142454 POINT(41.449992839451994 -86.84552303608498) bank142454 +142455 POINT(42.73204498122585 -87.32972981637718) bank142455 +142456 POINT(42.42203627633428 -88.25365702984215) bank142456 +142457 POINT(41.41314878396763 -87.08385410280478) bank142457 +142458 POINT(42.206584067428615 -87.03082352347757) bank142458 +142459 POINT(42.649746437838246 -88.02344958341203) bank142459 +142460 POINT(41.736026560932316 -87.89448729917274) bank142460 +142461 POINT(42.1197183916807 -87.67847486652774) bank142461 +142462 POINT(41.8540004067408 -87.32428409050381) bank142462 +142463 POINT(41.52277335679958 -87.64724682094112) bank142463 +142464 POINT(41.98115531294157 -87.95420156960017) bank142464 +142465 POINT(42.63316458804437 -88.38135982201722) bank142465 +142466 POINT(41.98069858225958 -87.88848637837835) bank142466 +142467 POINT(42.726167417164525 -86.72042899916434) bank142467 +142468 POINT(41.262137829119524 -87.41201835144254) bank142468 +142469 POINT(41.051163385343244 -88.34159098867299) bank142469 +142470 POINT(42.20498060137355 -87.3608580389046) bank142470 +142471 POINT(42.8478648331822 -88.37548595060063) bank142471 +142472 POINT(41.812754939597006 -87.50550971927659) bank142472 +142473 POINT(41.312246600708484 -87.99189508849733) bank142473 +142474 POINT(42.61755698139526 -88.02542938419994) bank142474 +142475 POINT(41.43301405216371 -87.07568555051341) bank142475 +142476 POINT(42.04159015166152 -87.7422084449742) bank142476 +142477 POINT(42.76710785466977 -87.98036510647893) bank142477 +142478 POINT(41.75878758959908 -88.45530250121378) bank142478 +142479 POINT(42.820812366512286 -87.03929160672672) bank142479 +142480 POINT(42.02337534072044 -87.05070151028374) bank142480 +142481 POINT(41.34021791900557 -86.66757788815086) bank142481 +142482 POINT(41.7452193830343 -88.2422878232856) bank142482 +142483 POINT(42.839282237926525 -87.25675963556645) bank142483 +142484 POINT(42.70411104450177 -88.48013588411729) bank142484 +142485 POINT(42.51012947518599 -88.52246891767302) bank142485 +142486 POINT(41.772523127239445 -87.4219609869968) bank142486 +142487 POINT(41.736513193880846 -88.1572409786927) bank142487 +142488 POINT(42.47936756368922 -88.13963784996274) bank142488 +142489 POINT(42.567670407201284 -86.71481648872356) bank142489 +142490 POINT(41.56865497598311 -87.40148884724064) bank142490 +142491 POINT(42.14242511028699 -86.95340988109099) bank142491 +142492 POINT(41.010162557869464 -86.97568988251592) bank142492 +142493 POINT(40.89618231965142 -87.06367258755226) bank142493 +142494 POINT(41.920612734562745 -88.32747745942041) bank142494 +142495 POINT(42.06527959660543 -87.9432909490681) bank142495 +142496 POINT(42.052006378407015 -88.17644469292796) bank142496 +142497 POINT(40.927128979400145 -88.54945996590624) bank142497 +142498 POINT(41.1615581733459 -87.54469351730474) bank142498 +142499 POINT(41.67381142375019 -88.38834475545511) bank142499 +142500 POINT(41.739064003821284 -88.33621456384314) bank142500 +142501 POINT(41.91153302373238 -87.7508133915171) bank142501 +142502 POINT(42.58155950174285 -88.37888304109032) bank142502 +142503 POINT(42.51753766217365 -88.18605512764499) bank142503 +142504 POINT(42.498401815603714 -87.90281863874641) bank142504 +142505 POINT(41.780415146535944 -86.64615817078614) bank142505 +142506 POINT(42.82978627493953 -87.03735369473021) bank142506 +142507 POINT(41.93492365488794 -88.58635852276507) bank142507 +142508 POINT(42.35432179515232 -87.30383955084312) bank142508 +142509 POINT(41.60246908662519 -86.79644633712077) bank142509 +142510 POINT(41.26445101243461 -87.86435008761035) bank142510 +142511 POINT(41.0567408902028 -87.11640434470878) bank142511 +142512 POINT(41.997692037176115 -87.90279750232159) bank142512 +142513 POINT(42.78388050571585 -88.15591480543688) bank142513 +142514 POINT(42.54115978840298 -88.12332638578626) bank142514 +142515 POINT(41.78242287248641 -88.09394234640096) bank142515 +142516 POINT(40.97813917716629 -87.60728327996688) bank142516 +142517 POINT(41.495696914599456 -87.31752925943164) bank142517 +142518 POINT(41.72466000410392 -86.90077751836537) bank142518 +142519 POINT(42.8452370964486 -87.51886936743496) bank142519 +142520 POINT(41.47630521668852 -88.48957618298878) bank142520 +142521 POINT(41.47030726852513 -88.04836738238225) bank142521 +142522 POINT(42.57416905450307 -86.85238486523023) bank142522 +142523 POINT(41.892294942282234 -88.18802018071182) bank142523 +142524 POINT(41.434262543231505 -88.3322316370456) bank142524 +142525 POINT(40.981701070284394 -87.07215897170786) bank142525 +142526 POINT(42.30903202950624 -87.95169858606296) bank142526 +142527 POINT(40.896045315583294 -86.93318261953029) bank142527 +142528 POINT(41.41990879487832 -87.81925379099243) bank142528 +142529 POINT(41.90592546802712 -87.58935759158157) bank142529 +142530 POINT(42.351264478568865 -87.249233206264) bank142530 +142531 POINT(41.60895517431341 -88.09227106951761) bank142531 +142532 POINT(41.07053010155891 -87.84326011063524) bank142532 +142533 POINT(41.42674838216669 -87.14261701848505) bank142533 +142534 POINT(41.546060428202765 -87.5811088474686) bank142534 +142535 POINT(41.17828112172171 -87.39205382499443) bank142535 +142536 POINT(42.30084846744188 -87.16735679612447) bank142536 +142537 POINT(42.29500308738143 -87.87226932572815) bank142537 +142538 POINT(41.74889074865038 -88.36880279964738) bank142538 +142539 POINT(41.733454210387045 -86.89000355207241) bank142539 +142540 POINT(42.393000141255534 -87.81339388272337) bank142540 +142541 POINT(41.98611005210051 -87.55549188022152) bank142541 +142542 POINT(42.36766171583349 -88.03883969063081) bank142542 +142543 POINT(41.922707240877344 -88.50687335906076) bank142543 +142544 POINT(41.60792711012389 -87.28622515371919) bank142544 +142545 POINT(41.7112810265457 -87.76264032639449) bank142545 +142546 POINT(42.812277398841346 -86.91469887118275) bank142546 +142547 POINT(42.81673219173246 -87.81031428720594) bank142547 +142548 POINT(42.77859781157691 -87.17707139794778) bank142548 +142549 POINT(41.188211310124416 -88.54260628801384) bank142549 +142550 POINT(42.617907264703376 -87.38478833820126) bank142550 +142551 POINT(41.96421888399231 -87.57433251124128) bank142551 +142552 POINT(42.39587762213177 -88.54741536914221) bank142552 +142553 POINT(42.56077841464351 -86.81211792704255) bank142553 +142554 POINT(42.72191290824952 -87.66626551633445) bank142554 +142555 POINT(42.01430988939035 -87.38044673895537) bank142555 +142556 POINT(42.467221406475325 -86.94693048288906) bank142556 +142557 POINT(42.432742461086505 -86.87509753012088) bank142557 +142558 POINT(42.04932352412607 -87.145438779145) bank142558 +142559 POINT(42.22266155642649 -87.6433191187867) bank142559 +142560 POINT(42.29140460479921 -88.40138550114352) bank142560 +142561 POINT(41.37079327088023 -88.59943453551926) bank142561 +142562 POINT(42.487298067673 -87.81590565991245) bank142562 +142563 POINT(41.721310671856685 -88.3233392003457) bank142563 +142564 POINT(41.86748769000615 -86.94640178761567) bank142564 +142565 POINT(41.58999225601293 -88.58957330441552) bank142565 +142566 POINT(42.03215173102848 -88.575564645768) bank142566 +142567 POINT(42.53152980607724 -88.1385841197315) bank142567 +142568 POINT(42.163703789229416 -87.85506947721353) bank142568 +142569 POINT(40.90427173021134 -88.01809864386067) bank142569 +142570 POINT(41.29112994346652 -87.45556006069353) bank142570 +142571 POINT(42.06953955726822 -86.7209196268233) bank142571 +142572 POINT(40.95683763240703 -86.7726057732211) bank142572 +142573 POINT(42.19201294846449 -88.62667623042395) bank142573 +142574 POINT(41.84179863394448 -86.89592558912179) bank142574 +142575 POINT(41.543010525093614 -87.58756015078681) bank142575 +142576 POINT(42.00465559812718 -86.70378960508432) bank142576 +142577 POINT(42.07639897049531 -87.89636631705659) bank142577 +142578 POINT(40.93654858970324 -87.18250334647246) bank142578 +142579 POINT(42.81910514350045 -87.963211035056) bank142579 +142580 POINT(41.26748721310878 -88.17184534364526) bank142580 +142581 POINT(42.295296204763176 -87.79951619629817) bank142581 +142582 POINT(42.43491478202544 -86.97856552459318) bank142582 +142583 POINT(41.28131498267971 -87.11871471577548) bank142583 +142584 POINT(41.653007288726506 -87.38363888037249) bank142584 +142585 POINT(41.49445225979807 -87.08208980515218) bank142585 +142586 POINT(41.26528169524098 -86.94649197272685) bank142586 +142587 POINT(42.28235758581794 -87.6861019621846) bank142587 +142588 POINT(42.40097876152056 -88.0959647118548) bank142588 +142589 POINT(42.728700779588095 -88.2820098125797) bank142589 +142590 POINT(42.78911562810008 -87.37606604016038) bank142590 +142591 POINT(42.341452520416176 -86.68341421212664) bank142591 +142592 POINT(41.248982140525335 -87.50610290027754) bank142592 +142593 POINT(42.077347928138074 -87.64677531394055) bank142593 +142594 POINT(41.237341267098174 -87.57747666358486) bank142594 +142595 POINT(41.55149662326035 -87.98335078071482) bank142595 +142596 POINT(42.299286364085056 -88.49318467503569) bank142596 +142597 POINT(40.938677066197656 -88.27901991673792) bank142597 +142598 POINT(42.275048671399965 -88.5560314147086) bank142598 +142599 POINT(41.260523656137515 -87.34503028677632) bank142599 +142600 POINT(41.05241614367899 -86.72719688561355) bank142600 +142601 POINT(42.4113206443371 -86.83321725872099) bank142601 +142602 POINT(42.6825973834932 -88.54656713467146) bank142602 +142603 POINT(42.6383911303903 -87.24751346531546) bank142603 +142604 POINT(41.58324352224187 -88.08764482884426) bank142604 +142605 POINT(42.64398608308782 -88.4264959819461) bank142605 +142606 POINT(41.06600236598285 -86.64412406581361) bank142606 +142607 POINT(41.45221467518295 -88.14871687203771) bank142607 +142608 POINT(41.505785335226626 -87.54319587313766) bank142608 +142609 POINT(41.72499129274544 -87.36206930275343) bank142609 +142610 POINT(41.65740050480652 -88.17080652613622) bank142610 +142611 POINT(42.697690693945965 -87.61880104143633) bank142611 +142612 POINT(42.56035786314592 -87.05728897787965) bank142612 +142613 POINT(41.612913734740154 -88.3451005955351) bank142613 +142614 POINT(42.52822147104149 -88.46869665689609) bank142614 +142615 POINT(42.33518839494434 -88.5480176081583) bank142615 +142616 POINT(41.79926137357239 -87.80282524269019) bank142616 +142617 POINT(41.785986649683785 -86.96627614133806) bank142617 +142618 POINT(40.97793351403626 -88.24886192053705) bank142618 +142619 POINT(41.01349365558457 -87.42378336634162) bank142619 +142620 POINT(41.16893233534568 -87.97817598710475) bank142620 +142621 POINT(42.0136438978569 -87.9406089857288) bank142621 +142622 POINT(42.77064101822151 -87.76800993227174) bank142622 +142623 POINT(41.46961761546485 -88.57578426794926) bank142623 +142624 POINT(41.3018224499129 -87.08070853202362) bank142624 +142625 POINT(41.53115331048172 -87.0856879946779) bank142625 +142626 POINT(42.00885576024683 -87.73587216782099) bank142626 +142627 POINT(41.44426605495102 -87.42795641860674) bank142627 +142628 POINT(42.13820775303444 -86.7472462311846) bank142628 +142629 POINT(40.92221759804092 -86.73468898815756) bank142629 +142630 POINT(41.998144300077115 -87.86687496673335) bank142630 +142631 POINT(40.898016907352925 -86.83036697665086) bank142631 +142632 POINT(41.21382547879176 -87.82273594423891) bank142632 +142633 POINT(41.95489847858845 -86.6663463333996) bank142633 +142634 POINT(41.79989050973916 -87.24712490478097) bank142634 +142635 POINT(42.37824008503688 -87.11864271775508) bank142635 +142636 POINT(41.97653596158497 -86.96870058877293) bank142636 +142637 POINT(42.86443580776446 -88.48282552008051) bank142637 +142638 POINT(42.84330351888145 -87.4404462837325) bank142638 +142639 POINT(42.52941200547884 -86.97906198697804) bank142639 +142640 POINT(41.299824313372035 -87.97482695783756) bank142640 +142641 POINT(40.954223464638126 -88.34527680658528) bank142641 +142642 POINT(42.48827130607492 -88.59557919432689) bank142642 +142643 POINT(42.64885375844894 -87.18137166744646) bank142643 +142644 POINT(41.45594312423802 -87.69445282084358) bank142644 +142645 POINT(41.22698696919917 -87.62025775515092) bank142645 +142646 POINT(42.38233394214767 -86.82855948644512) bank142646 +142647 POINT(42.8512999280936 -88.35230073607418) bank142647 +142648 POINT(42.4253006765434 -88.4658131194183) bank142648 +142649 POINT(41.96881134016 -87.70170396986948) bank142649 +142650 POINT(42.376780918556065 -86.66731685392853) bank142650 +142651 POINT(42.1158644304678 -86.98362958504353) bank142651 +142652 POINT(41.02703228375066 -87.86352105048037) bank142652 +142653 POINT(42.35594770744499 -88.13237740130432) bank142653 +142654 POINT(41.440496841805846 -87.37606440467297) bank142654 +142655 POINT(42.53201693559351 -87.1065748339247) bank142655 +142656 POINT(40.95985648448573 -88.40072627833659) bank142656 +142657 POINT(41.01316979502542 -87.83233945046828) bank142657 +142658 POINT(41.95047622221661 -86.72274287258489) bank142658 +142659 POINT(42.290411606706115 -86.94710137803939) bank142659 +142660 POINT(42.323737161944855 -87.33478764032886) bank142660 +142661 POINT(42.45400336880033 -88.03971383783642) bank142661 +142662 POINT(41.86435524842171 -88.53950881970205) bank142662 +142663 POINT(41.671821706054594 -87.11524579329041) bank142663 +142664 POINT(42.278941748580785 -88.09304117651662) bank142664 +142665 POINT(42.82416663892973 -87.51878112494452) bank142665 +142666 POINT(41.496055320262904 -87.18909674155967) bank142666 +142667 POINT(41.815793516086714 -87.27824070082502) bank142667 +142668 POINT(40.95617239514745 -87.99280753640149) bank142668 +142669 POINT(42.42306383302241 -87.1659187224276) bank142669 +142670 POINT(41.7552216824491 -88.0265060232175) bank142670 +142671 POINT(41.09131435324119 -86.67032241669277) bank142671 +142672 POINT(41.472555236040066 -87.97817735033703) bank142672 +142673 POINT(41.635213689448314 -86.70437179651172) bank142673 +142674 POINT(41.66725660536106 -87.51452540314833) bank142674 +142675 POINT(42.06819446508381 -88.58635481390947) bank142675 +142676 POINT(40.96063787770591 -88.34014828911926) bank142676 +142677 POINT(41.43468233388192 -86.76376184731662) bank142677 +142678 POINT(42.28458886102651 -87.72229651334924) bank142678 +142679 POINT(41.38697497438352 -87.29743675287172) bank142679 +142680 POINT(42.23686310462093 -87.17413039217554) bank142680 +142681 POINT(41.760621980679055 -86.67743858138817) bank142681 +142682 POINT(42.49043794562963 -86.9927510451457) bank142682 +142683 POINT(41.70472661685111 -87.11592412041712) bank142683 +142684 POINT(40.95906262444853 -86.66070689315738) bank142684 +142685 POINT(42.809698184077966 -88.18275244117885) bank142685 +142686 POINT(42.08891179835544 -86.63077324787618) bank142686 +142687 POINT(41.122132465964555 -88.62437250320103) bank142687 +142688 POINT(41.96073022091293 -87.11122200713794) bank142688 +142689 POINT(41.724852776897336 -86.63483174480214) bank142689 +142690 POINT(42.34739403910689 -87.31607399077652) bank142690 +142691 POINT(42.67127020793282 -87.37866502325286) bank142691 +142692 POINT(41.13931121649048 -87.92899535345896) bank142692 +142693 POINT(42.52113441549262 -88.59435844318008) bank142693 +142694 POINT(42.861140657936645 -88.27043097323035) bank142694 +142695 POINT(41.84097328822044 -86.69702777663429) bank142695 +142696 POINT(41.06422949856869 -87.16710655556443) bank142696 +142697 POINT(41.44629383407492 -86.73458326144637) bank142697 +142698 POINT(41.13339738612936 -88.2354444301477) bank142698 +142699 POINT(42.2218820423154 -87.27956037487071) bank142699 +142700 POINT(41.92242829218034 -87.67414372506886) bank142700 +142701 POINT(42.25054805992161 -88.42980840595415) bank142701 +142702 POINT(41.05252307292886 -88.51405578887439) bank142702 +142703 POINT(41.2015111690614 -86.9513294684207) bank142703 +142704 POINT(42.24439391034388 -87.84513387374963) bank142704 +142705 POINT(41.61870599105436 -86.82099621071575) bank142705 +142706 POINT(42.11950788955383 -86.85950212622608) bank142706 +142707 POINT(42.49894400554438 -87.59579649599328) bank142707 +142708 POINT(42.1027415807286 -87.85592093865219) bank142708 +142709 POINT(42.05831258698522 -87.66325306722486) bank142709 +142710 POINT(41.223534320506396 -87.32916343058305) bank142710 +142711 POINT(41.476023126214784 -87.05040788659514) bank142711 +142712 POINT(41.75902761049018 -87.09203984309057) bank142712 +142713 POINT(41.17121525116011 -88.61675888339519) bank142713 +142714 POINT(41.507066577110855 -87.10488609953617) bank142714 +142715 POINT(41.78640291086582 -87.1634700554978) bank142715 +142716 POINT(42.51412218686109 -87.82905104906831) bank142716 +142717 POINT(41.093500191569824 -87.52726665705886) bank142717 +142718 POINT(41.811745205968585 -86.86067985325536) bank142718 +142719 POINT(42.0748540843097 -86.75099162965195) bank142719 +142720 POINT(41.82658786290665 -87.97226804260191) bank142720 +142721 POINT(42.173319926076864 -87.0448805503334) bank142721 +142722 POINT(41.76736191663687 -87.65691878683155) bank142722 +142723 POINT(42.38947013895711 -87.03315459210441) bank142723 +142724 POINT(41.85496425580289 -87.0499217522142) bank142724 +142725 POINT(41.22760936253465 -87.74609232294357) bank142725 +142726 POINT(41.79863741487741 -87.3115349624295) bank142726 +142727 POINT(41.57314916441529 -87.83023009255146) bank142727 +142728 POINT(42.319214225929336 -88.14356003881458) bank142728 +142729 POINT(40.91120005865966 -88.27509504323517) bank142729 +142730 POINT(42.30915435434296 -88.42461034047716) bank142730 +142731 POINT(42.68492807362282 -86.75185701576525) bank142731 +142732 POINT(41.47491951242188 -88.2687615639131) bank142732 +142733 POINT(41.54087204889775 -88.48091729325348) bank142733 +142734 POINT(41.81656661351361 -87.50139780294583) bank142734 +142735 POINT(41.45855094807077 -87.97701100046646) bank142735 +142736 POINT(41.82917874324667 -87.19357079556698) bank142736 +142737 POINT(41.3812874058569 -87.70838514592664) bank142737 +142738 POINT(41.81557303687066 -86.86159841809052) bank142738 +142739 POINT(42.02200398810712 -88.02842260409054) bank142739 +142740 POINT(41.41064134316641 -88.52250003758606) bank142740 +142741 POINT(41.823134087253166 -87.02707398044792) bank142741 +142742 POINT(42.21890370306554 -86.78112740372823) bank142742 +142743 POINT(40.93382943437178 -87.4928915227578) bank142743 +142744 POINT(41.44493743388906 -88.480550063355) bank142744 +142745 POINT(42.215390560667956 -86.69182472513515) bank142745 +142746 POINT(41.394599533480566 -86.6475597735893) bank142746 +142747 POINT(42.5323323799074 -87.69883641321702) bank142747 +142748 POINT(42.224057023614094 -86.66307151642638) bank142748 +142749 POINT(42.800705732879564 -87.382145685247) bank142749 +142750 POINT(41.04780526786253 -86.76406816806869) bank142750 +142751 POINT(41.30402042161753 -87.45835628218114) bank142751 +142752 POINT(40.89198922068684 -87.01638903111868) bank142752 +142753 POINT(41.421369866650636 -86.7087578229974) bank142753 +142754 POINT(42.571957574271 -87.13823013850613) bank142754 +142755 POINT(42.874861785478025 -87.94769855121606) bank142755 +142756 POINT(42.29905871141869 -88.15421412250521) bank142756 +142757 POINT(41.641593192611 -87.42997363992836) bank142757 +142758 POINT(41.3912812563808 -86.73958542587415) bank142758 +142759 POINT(42.30997027964231 -88.01336267450868) bank142759 +142760 POINT(40.936630677059654 -86.69400660802334) bank142760 +142761 POINT(42.767576893732915 -86.77084775766536) bank142761 +142762 POINT(41.283971617928664 -88.0389694166539) bank142762 +142763 POINT(42.07417616420585 -87.00536040009575) bank142763 +142764 POINT(41.75764246241685 -87.20835621870307) bank142764 +142765 POINT(42.03193394588035 -88.41946507145043) bank142765 +142766 POINT(41.656823642040855 -88.59924484779654) bank142766 +142767 POINT(42.15083701618744 -87.13951722694941) bank142767 +142768 POINT(42.010164183542834 -87.79532972003601) bank142768 +142769 POINT(41.47113647393516 -87.68996240401923) bank142769 +142770 POINT(42.36044302600429 -87.47959633479846) bank142770 +142771 POINT(41.16974298040829 -87.7449925700147) bank142771 +142772 POINT(42.47994660694136 -86.75143909374471) bank142772 +142773 POINT(40.955401350755736 -87.28125133402709) bank142773 +142774 POINT(42.15826464811496 -88.26109705088223) bank142774 +142775 POINT(42.032400292269706 -87.48579350107066) bank142775 +142776 POINT(41.55371906605074 -86.79911393298202) bank142776 +142777 POINT(41.854141163738056 -87.6288023861962) bank142777 +142778 POINT(42.65244181118336 -87.30748419954195) bank142778 +142779 POINT(41.17686546387235 -86.75315179894304) bank142779 +142780 POINT(42.353896097402384 -87.33211401326653) bank142780 +142781 POINT(41.953605739452925 -88.4423218904486) bank142781 +142782 POINT(41.120290592114266 -87.59317076886477) bank142782 +142783 POINT(41.71961721221472 -86.78465097734488) bank142783 +142784 POINT(42.088844623627345 -88.13224131320402) bank142784 +142785 POINT(41.015205782699745 -87.28933656361852) bank142785 +142786 POINT(42.83980653080038 -88.23100773362422) bank142786 +142787 POINT(42.25590510791098 -87.34063158853917) bank142787 +142788 POINT(42.558944396622806 -88.32647165467877) bank142788 +142789 POINT(42.09445735900216 -87.77588641753282) bank142789 +142790 POINT(41.70243803166839 -87.31885557610781) bank142790 +142791 POINT(42.34867857218772 -86.77762667227731) bank142791 +142792 POINT(42.56924945537131 -87.56218489384028) bank142792 +142793 POINT(41.1750625080507 -87.22743096602866) bank142793 +142794 POINT(42.77156348091763 -86.7017351128556) bank142794 +142795 POINT(41.18576724517659 -88.1309432661192) bank142795 +142796 POINT(41.11892588184481 -87.90581900434408) bank142796 +142797 POINT(40.92228529281965 -87.33899690799564) bank142797 +142798 POINT(41.09503038512167 -87.85699834163533) bank142798 +142799 POINT(42.34692774731834 -87.84709933840283) bank142799 +142800 POINT(42.719467653212305 -87.19596467853778) bank142800 +142801 POINT(42.04544172204107 -87.10882725887542) bank142801 +142802 POINT(41.91841707180849 -88.14545025263261) bank142802 +142803 POINT(41.70116786397801 -87.96640282923474) bank142803 +142804 POINT(41.111955631598306 -88.07411875446667) bank142804 +142805 POINT(41.619925294578366 -87.73105421677212) bank142805 +142806 POINT(41.914216518290345 -88.36498679080844) bank142806 +142807 POINT(42.69225668317649 -87.34934595319277) bank142807 +142808 POINT(41.43623479609379 -87.73484035795364) bank142808 +142809 POINT(41.7078148600883 -87.36785330835738) bank142809 +142810 POINT(41.71062172795564 -88.36641269217147) bank142810 +142811 POINT(41.16617361877022 -86.89796066024202) bank142811 +142812 POINT(41.205910198480666 -88.56333119800999) bank142812 +142813 POINT(41.79370318627141 -87.46721111239385) bank142813 +142814 POINT(41.764449894132 -88.60946514305725) bank142814 +142815 POINT(41.684389640651105 -88.43325299419479) bank142815 +142816 POINT(42.33372706637366 -86.8155237428744) bank142816 +142817 POINT(41.450001535160666 -88.51600177767534) bank142817 +142818 POINT(41.57431678567979 -87.04071923301692) bank142818 +142819 POINT(41.38525191566718 -87.06456005485188) bank142819 +142820 POINT(41.66165838577381 -87.63080948290916) bank142820 +142821 POINT(41.0543042657535 -87.11769394450936) bank142821 +142822 POINT(42.459453554809976 -88.0950808886659) bank142822 +142823 POINT(41.41036983281955 -88.34943293102707) bank142823 +142824 POINT(41.02255491860741 -88.50805714313881) bank142824 +142825 POINT(41.265304183806265 -88.57011073382822) bank142825 +142826 POINT(42.604695866133646 -88.5696917415243) bank142826 +142827 POINT(42.63346676860869 -88.4975680545134) bank142827 +142828 POINT(42.410931585393186 -87.12462037611522) bank142828 +142829 POINT(42.35565120610088 -87.76226646107466) bank142829 +142830 POINT(42.065948337016096 -87.31312539785397) bank142830 +142831 POINT(42.65655604250155 -87.43858026050184) bank142831 +142832 POINT(41.08253766311495 -88.31603736248705) bank142832 +142833 POINT(42.64543628234994 -87.8183842334055) bank142833 +142834 POINT(42.68746059755296 -88.06781161202352) bank142834 +142835 POINT(42.05365100288423 -88.06862630346663) bank142835 +142836 POINT(41.08879072507435 -87.10865222839273) bank142836 +142837 POINT(40.92250193635805 -87.63773211386126) bank142837 +142838 POINT(42.063913083090064 -87.03323917871462) bank142838 +142839 POINT(42.0282323032836 -88.4530706118631) bank142839 +142840 POINT(41.011451598484015 -87.40455583454086) bank142840 +142841 POINT(42.04894448551478 -87.55151207824953) bank142841 +142842 POINT(42.14551772079452 -88.38926171193948) bank142842 +142843 POINT(40.96352060482073 -86.83275864432633) bank142843 +142844 POINT(42.01689704767882 -87.72613393457328) bank142844 +142845 POINT(41.078375300706305 -88.02719476064382) bank142845 +142846 POINT(41.77952574271608 -88.53997966708467) bank142846 +142847 POINT(42.213782085577115 -87.25855485477517) bank142847 +142848 POINT(42.32350939514772 -86.99267321283013) bank142848 +142849 POINT(42.68250557869612 -87.29243440735212) bank142849 +142850 POINT(40.90741928190608 -87.24222337355371) bank142850 +142851 POINT(41.14422972485372 -87.24928472008588) bank142851 +142852 POINT(41.21768107281844 -87.17691292034209) bank142852 +142853 POINT(41.813000285022326 -87.82512244084) bank142853 +142854 POINT(41.48436662925433 -88.18098272100374) bank142854 +142855 POINT(42.263453023056044 -86.95186675846706) bank142855 +142856 POINT(42.40848251014109 -87.6657999867422) bank142856 +142857 POINT(42.34022484717264 -87.33960723227956) bank142857 +142858 POINT(41.52941521550624 -88.52452316206782) bank142858 +142859 POINT(40.99398420213748 -88.12384679027849) bank142859 +142860 POINT(42.58232667479134 -88.41933731715315) bank142860 +142861 POINT(41.38908143110339 -88.37016431112303) bank142861 +142862 POINT(42.22955382558433 -87.54575287571413) bank142862 +142863 POINT(42.61500192507892 -87.7366564562537) bank142863 +142864 POINT(42.39024238371876 -86.96177213884158) bank142864 +142865 POINT(41.962938771626526 -88.47950708498634) bank142865 +142866 POINT(41.07528119097853 -88.60430959730168) bank142866 +142867 POINT(40.991295583772015 -87.65169106180312) bank142867 +142868 POINT(42.05167054800997 -88.15550652813972) bank142868 +142869 POINT(41.78300128191043 -86.85438317527634) bank142869 +142870 POINT(42.53111917308683 -88.51687679069073) bank142870 +142871 POINT(41.40730092059406 -88.13524291797698) bank142871 +142872 POINT(42.477898407887366 -87.67075319861911) bank142872 +142873 POINT(41.426910960376986 -87.44023656590535) bank142873 +142874 POINT(42.46339285524397 -87.34929018165562) bank142874 +142875 POINT(41.93266483169822 -87.26690347579434) bank142875 +142876 POINT(42.331296368742734 -88.46721943665796) bank142876 +142877 POINT(41.57872562983622 -88.10867019307581) bank142877 +142878 POINT(41.65331164097219 -87.05136392015655) bank142878 +142879 POINT(42.37194833556329 -87.64225665697033) bank142879 +142880 POINT(41.169708645496094 -87.2164717539022) bank142880 +142881 POINT(41.71924833528736 -88.48657990629712) bank142881 +142882 POINT(42.557682481601205 -88.27604662459923) bank142882 +142883 POINT(40.97147211521707 -87.16991627229395) bank142883 +142884 POINT(41.919587931424424 -88.10123288990913) bank142884 +142885 POINT(42.436943775453095 -87.74974519441862) bank142885 +142886 POINT(42.223400926306944 -87.85442583618241) bank142886 +142887 POINT(41.479611356906126 -87.8731639752561) bank142887 +142888 POINT(41.99247116534909 -87.5654641930739) bank142888 +142889 POINT(41.78959050237483 -87.13354320561419) bank142889 +142890 POINT(41.204433461044836 -88.45277457897055) bank142890 +142891 POINT(41.3999713185532 -86.90954951430834) bank142891 +142892 POINT(41.320460928761534 -88.20361295107814) bank142892 +142893 POINT(42.458998812208755 -88.1362860331021) bank142893 +142894 POINT(41.70372331336509 -88.50503016255975) bank142894 +142895 POINT(42.24932033387003 -87.90463003482668) bank142895 +142896 POINT(42.11285700138324 -87.82022310196031) bank142896 +142897 POINT(41.075426409106925 -87.48335423601353) bank142897 +142898 POINT(42.0810437902051 -87.77300345755276) bank142898 +142899 POINT(41.84297232534459 -87.35030416438074) bank142899 +142900 POINT(42.87385272445908 -87.26086857551883) bank142900 +142901 POINT(41.23263295877839 -87.16566187004412) bank142901 +142902 POINT(41.20129388662184 -87.31735799223212) bank142902 +142903 POINT(42.4321389092495 -87.82375036739919) bank142903 +142904 POINT(41.437743583044146 -87.23576395098311) bank142904 +142905 POINT(41.920007067911214 -88.31494075911935) bank142905 +142906 POINT(41.007002044106905 -87.32560796340417) bank142906 +142907 POINT(42.43088557306853 -88.43375546347436) bank142907 +142908 POINT(41.05880274186989 -87.61024711034474) bank142908 +142909 POINT(42.73774171579841 -87.50728409747987) bank142909 +142910 POINT(42.473185462903615 -87.56609518311451) bank142910 +142911 POINT(41.65467275351939 -87.1783296392505) bank142911 +142912 POINT(41.5882046936035 -87.48055729953639) bank142912 +142913 POINT(42.5564955139726 -87.26335774024531) bank142913 +142914 POINT(41.711843697520756 -87.61058414991415) bank142914 +142915 POINT(41.91385466136435 -88.34440826873572) bank142915 +142916 POINT(42.63237375548593 -87.53466053083551) bank142916 +142917 POINT(40.97861329593481 -88.2631389831194) bank142917 +142918 POINT(40.880793000618254 -88.17605685657428) bank142918 +142919 POINT(42.3166518004503 -87.00529243993961) bank142919 +142920 POINT(41.93794468547903 -88.30048398523171) bank142920 +142921 POINT(42.84887551660742 -88.51291028077756) bank142921 +142922 POINT(41.80545450413008 -87.69032051891631) bank142922 +142923 POINT(41.465663239797806 -87.00052790660101) bank142923 +142924 POINT(42.55058604944901 -87.76218354138462) bank142924 +142925 POINT(42.03087044427961 -88.02070770890654) bank142925 +142926 POINT(42.09732737186507 -87.1109487895834) bank142926 +142927 POINT(41.584678147857524 -86.78029096540459) bank142927 +142928 POINT(42.27594117029872 -88.4470527405817) bank142928 +142929 POINT(42.851154964826094 -88.01788910780361) bank142929 +142930 POINT(42.314267118166164 -88.2290600149161) bank142930 +142931 POINT(42.276003296226094 -88.28477438180697) bank142931 +142932 POINT(41.86953401451779 -88.23201207155033) bank142932 +142933 POINT(41.525934483319865 -88.01357257111451) bank142933 +142934 POINT(42.39255205392348 -86.84031731569522) bank142934 +142935 POINT(41.36102284071845 -87.96755061957856) bank142935 +142936 POINT(42.843063347386924 -87.99067165870018) bank142936 +142937 POINT(42.55486975670544 -86.81867398202931) bank142937 +142938 POINT(42.473930215377955 -87.16041327094158) bank142938 +142939 POINT(40.98251531021916 -87.4114908807109) bank142939 +142940 POINT(41.051074128338456 -86.88003159950362) bank142940 +142941 POINT(42.55465568417669 -86.72363711626375) bank142941 +142942 POINT(42.053584550098435 -87.1393440846597) bank142942 +142943 POINT(40.99626064233003 -86.97219268582133) bank142943 +142944 POINT(42.44252649148091 -88.51878542986564) bank142944 +142945 POINT(41.213330003533116 -88.32013610825214) bank142945 +142946 POINT(42.54556472290214 -87.8140320471252) bank142946 +142947 POINT(40.97107291143251 -87.33491460351422) bank142947 +142948 POINT(42.86011721923896 -88.35211835198167) bank142948 +142949 POINT(40.89049305471558 -87.74450642467626) bank142949 +142950 POINT(41.484672626723736 -87.35701051251618) bank142950 +142951 POINT(41.262873791586685 -87.02296690930459) bank142951 +142952 POINT(41.218621792585225 -87.59497176596022) bank142952 +142953 POINT(41.881066101423585 -87.1240151411964) bank142953 +142954 POINT(41.23642871234605 -87.38699067185873) bank142954 +142955 POINT(42.57126278618219 -88.49716441668137) bank142955 +142956 POINT(41.16349982600002 -88.09238350621523) bank142956 +142957 POINT(42.49775331597382 -87.44900077223106) bank142957 +142958 POINT(42.79267161923103 -87.85408358130148) bank142958 +142959 POINT(41.21663080760932 -87.70358134436208) bank142959 +142960 POINT(41.72642318121076 -86.99080353361656) bank142960 +142961 POINT(42.14021739430707 -87.98880199191989) bank142961 +142962 POINT(41.22284245111915 -88.40578721275298) bank142962 +142963 POINT(41.292668483047954 -87.86603098190311) bank142963 +142964 POINT(42.13421332018672 -87.80383992792373) bank142964 +142965 POINT(42.30231757871355 -87.33464171076069) bank142965 +142966 POINT(42.36768299697321 -87.8386733500369) bank142966 +142967 POINT(41.697607666766366 -88.20002745478638) bank142967 +142968 POINT(42.32317345797042 -88.54731167664664) bank142968 +142969 POINT(41.19061309269483 -88.16329782051076) bank142969 +142970 POINT(41.18706874232594 -86.7202373012034) bank142970 +142971 POINT(42.62806640375274 -86.82795571714647) bank142971 +142972 POINT(40.976046284281026 -88.30959924324266) bank142972 +142973 POINT(41.42498115360129 -88.3631368818177) bank142973 +142974 POINT(42.866334454657085 -88.0021237038025) bank142974 +142975 POINT(42.39855183482012 -87.47174858828281) bank142975 +142976 POINT(42.40695210177272 -88.47402981770755) bank142976 +142977 POINT(42.648303506715834 -88.11694190553115) bank142977 +142978 POINT(41.01825641891071 -87.14346652472379) bank142978 +142979 POINT(41.27384320898782 -86.72607953654455) bank142979 +142980 POINT(41.94547752417131 -88.4595309225657) bank142980 +142981 POINT(42.30353169047962 -87.25491773185263) bank142981 +142982 POINT(41.91332133220417 -87.61368354491636) bank142982 +142983 POINT(41.62327559599293 -87.54967069448328) bank142983 +142984 POINT(42.70086550977215 -88.59735933485945) bank142984 +142985 POINT(41.46980472491981 -87.21745514388314) bank142985 +142986 POINT(42.06362790425976 -87.97773111891183) bank142986 +142987 POINT(42.081567038464584 -87.9233291978319) bank142987 +142988 POINT(41.01148008646775 -87.14829499351987) bank142988 +142989 POINT(41.586130799928746 -88.20983921063366) bank142989 +142990 POINT(42.81034422233189 -87.96397750760256) bank142990 +142991 POINT(42.05802213119173 -86.873927665773) bank142991 +142992 POINT(42.16246914490049 -87.23957377731324) bank142992 +142993 POINT(41.09149985917378 -87.54947616100416) bank142993 +142994 POINT(41.92528134337269 -87.04597877318382) bank142994 +142995 POINT(41.421707276402486 -86.6442611977502) bank142995 +142996 POINT(42.184052487435636 -86.92076039319025) bank142996 +142997 POINT(42.748240418638574 -88.61078772216871) bank142997 +142998 POINT(41.891335708181856 -88.56383504689713) bank142998 +142999 POINT(41.59971231931483 -86.77471631703283) bank142999 +143000 POINT(42.72218539801979 -87.1354103081885) bank143000 +143001 POINT(41.63232577807048 -87.38700828611096) bank143001 +143002 POINT(42.78892643962661 -87.0979015450421) bank143002 +143003 POINT(41.88127492813622 -86.91934454477862) bank143003 +143004 POINT(41.69812050471019 -86.89268188526295) bank143004 +143005 POINT(42.27753543454141 -87.67654316948052) bank143005 +143006 POINT(41.62052863289585 -86.97986746732504) bank143006 +143007 POINT(41.34384874155321 -87.87801986401234) bank143007 +143008 POINT(41.17651099975542 -87.25748217897728) bank143008 +143009 POINT(42.14071114651903 -88.26893547820087) bank143009 +143010 POINT(41.70050700594642 -88.26392561256598) bank143010 +143011 POINT(42.547422355032296 -88.37367894773405) bank143011 +143012 POINT(40.93476249674139 -86.71947977472423) bank143012 +143013 POINT(42.326945344121405 -87.91719311886554) bank143013 +143014 POINT(41.06473463227425 -88.32757022879501) bank143014 +143015 POINT(40.93857951105318 -87.0541705770892) bank143015 +143016 POINT(42.333833831896264 -87.41906370820388) bank143016 +143017 POINT(40.91885650831273 -87.09557231407439) bank143017 +143018 POINT(41.32742429366006 -87.90670178763742) bank143018 +143019 POINT(42.81122068669534 -87.62716347803193) bank143019 +143020 POINT(42.311277672968856 -87.79291382000575) bank143020 +143021 POINT(42.65814404357614 -87.14238167568448) bank143021 +143022 POINT(42.8501891922284 -86.87102357748707) bank143022 +143023 POINT(42.26148879050472 -87.73194498712472) bank143023 +143024 POINT(41.32167308415858 -88.30620568167079) bank143024 +143025 POINT(42.170033628527 -88.50392065807279) bank143025 +143026 POINT(41.38650795261386 -86.93465798682762) bank143026 +143027 POINT(41.165632454007884 -87.91300925677551) bank143027 +143028 POINT(41.98242567557208 -87.83630673621747) bank143028 +143029 POINT(42.636011200502594 -88.15503585372669) bank143029 +143030 POINT(41.77084107076519 -87.3640935304221) bank143030 +143031 POINT(42.66735825513954 -87.14098986315311) bank143031 +143032 POINT(40.929340020911376 -87.98231709634823) bank143032 +143033 POINT(42.65615074613955 -87.90113649821346) bank143033 +143034 POINT(41.62835555161874 -88.00667663947326) bank143034 +143035 POINT(42.81175343715706 -88.18168362161187) bank143035 +143036 POINT(42.04865370833365 -86.91102417443162) bank143036 +143037 POINT(42.61961478316674 -87.81795024818155) bank143037 +143038 POINT(42.75079210281019 -88.55290004985717) bank143038 +143039 POINT(41.75767615423388 -87.43681664528476) bank143039 +143040 POINT(41.5573169288063 -87.98226016625216) bank143040 +143041 POINT(42.654056192221475 -87.8150058486375) bank143041 +143042 POINT(41.830009187763316 -88.25704417519806) bank143042 +143043 POINT(40.93931883078753 -86.91646225365368) bank143043 +143044 POINT(42.51656292574267 -88.2685253393907) bank143044 +143045 POINT(41.65712001937821 -88.41503630769404) bank143045 +143046 POINT(41.189834017328785 -86.97119760396924) bank143046 +143047 POINT(41.07288348605222 -87.75505696800188) bank143047 +143048 POINT(42.80670463802633 -87.86026371119524) bank143048 +143049 POINT(41.978833234460105 -86.90815434028531) bank143049 +143050 POINT(41.73494134536705 -87.23295208258708) bank143050 +143051 POINT(41.132544210667824 -87.37624811817503) bank143051 +143052 POINT(42.027021431961124 -88.21615593331929) bank143052 +143053 POINT(42.82710645358066 -87.45802737150483) bank143053 +143054 POINT(41.350246668662734 -87.99813117069714) bank143054 +143055 POINT(42.11577013136747 -87.6383863208973) bank143055 +143056 POINT(42.21061727175095 -87.95541523949454) bank143056 +143057 POINT(41.98568446676608 -87.0737680030479) bank143057 +143058 POINT(42.55178334317621 -87.57434188510132) bank143058 +143059 POINT(42.383572078124274 -87.3665192493154) bank143059 +143060 POINT(42.685850491729525 -87.92421901417754) bank143060 +143061 POINT(42.09523501062563 -87.6506306337002) bank143061 +143062 POINT(41.22208176637535 -86.62997354195576) bank143062 +143063 POINT(41.238524733720716 -87.39398801509438) bank143063 +143064 POINT(41.84866358289061 -88.20482795441885) bank143064 +143065 POINT(42.46779424126586 -87.05566200186193) bank143065 +143066 POINT(41.983323404813916 -88.17729582068321) bank143066 +143067 POINT(42.67757877242574 -86.84288402402049) bank143067 +143068 POINT(42.622349164511895 -88.06763898975176) bank143068 +143069 POINT(42.0875562984262 -86.7972228088653) bank143069 +143070 POINT(42.646641878410705 -87.08121642582992) bank143070 +143071 POINT(42.797240980379875 -86.89138524886313) bank143071 +143072 POINT(42.06861514836296 -87.61441072317194) bank143072 +143073 POINT(42.12031611265802 -86.86944463971824) bank143073 +143074 POINT(41.117292343319 -87.62986916652974) bank143074 +143075 POINT(41.37286852145258 -87.92031482802466) bank143075 +143076 POINT(41.90201546430958 -88.56681438302525) bank143076 +143077 POINT(42.39472589998417 -87.2868648703029) bank143077 +143078 POINT(42.77365094937654 -87.31397063140653) bank143078 +143079 POINT(41.93530842279525 -87.79871427141076) bank143079 +143080 POINT(41.61668857219154 -88.44954429509076) bank143080 +143081 POINT(41.08642689542851 -88.32331756973026) bank143081 +143082 POINT(41.78229245724928 -88.07681226097331) bank143082 +143083 POINT(40.90330199899193 -87.26568031553755) bank143083 +143084 POINT(42.47017380251107 -88.03159907819168) bank143084 +143085 POINT(42.298802040716616 -87.60906588601074) bank143085 +143086 POINT(41.880993390030596 -87.62377257122033) bank143086 +143087 POINT(41.477555022442964 -87.29139818277314) bank143087 +143088 POINT(41.9155053984708 -87.30303138821252) bank143088 +143089 POINT(42.452759215713215 -88.10023919188242) bank143089 +143090 POINT(42.024089738564925 -87.1427937169595) bank143090 +143091 POINT(41.391453159352096 -87.54229993710797) bank143091 +143092 POINT(41.22757933347464 -88.16232016580021) bank143092 +143093 POINT(42.53633437150794 -87.96559359650266) bank143093 +143094 POINT(42.48212210937789 -88.18943727219164) bank143094 +143095 POINT(42.48189849666887 -87.42503351490926) bank143095 +143096 POINT(42.538480675369556 -86.69551478901276) bank143096 +143097 POINT(42.63201047484084 -87.69432846072938) bank143097 +143098 POINT(42.276525247787355 -87.25321505987296) bank143098 +143099 POINT(41.78206490742646 -88.40985587004609) bank143099 +143100 POINT(41.23921284893886 -86.81754322870758) bank143100 +143101 POINT(42.57810062357258 -87.43682100141405) bank143101 +143102 POINT(41.10542797357791 -86.81984734273307) bank143102 +143103 POINT(40.88253327888553 -87.1007708243618) bank143103 +143104 POINT(40.91472198284497 -88.47860850530918) bank143104 +143105 POINT(40.91411686597073 -87.4845068358717) bank143105 +143106 POINT(41.256142808570914 -87.51996809560686) bank143106 +143107 POINT(42.293382024264574 -88.37265241005373) bank143107 +143108 POINT(42.75809508927321 -87.88851846691601) bank143108 +143109 POINT(42.25730308904417 -88.45761888196905) bank143109 +143110 POINT(41.22743316378832 -87.13679133423574) bank143110 +143111 POINT(42.79186992341888 -87.64165798496221) bank143111 +143112 POINT(41.896343567659784 -87.61901392285664) bank143112 +143113 POINT(42.16858752502561 -87.86126734783711) bank143113 +143114 POINT(41.34177972483646 -87.02315563539503) bank143114 +143115 POINT(40.95888386611066 -87.67562472402084) bank143115 +143116 POINT(41.155228636155044 -87.52343003474454) bank143116 +143117 POINT(42.790145850586725 -88.1425424711746) bank143117 +143118 POINT(41.537877449376786 -87.84113936452734) bank143118 +143119 POINT(41.90192727323315 -86.95322709668741) bank143119 +143120 POINT(41.70903416243814 -88.32224000663948) bank143120 +143121 POINT(40.93219385269388 -87.40143808987433) bank143121 +143122 POINT(40.98447257789255 -86.89101908370694) bank143122 +143123 POINT(41.352378302656156 -88.32200550420222) bank143123 +143124 POINT(41.463769300195565 -87.1165467755266) bank143124 +143125 POINT(42.39702376045892 -87.5746091600006) bank143125 +143126 POINT(41.165641809708035 -87.16095559672567) bank143126 +143127 POINT(41.16132450372382 -87.41001275705095) bank143127 +143128 POINT(42.117490691308554 -88.29260820656381) bank143128 +143129 POINT(41.361856463325466 -87.90562113788067) bank143129 +143130 POINT(42.38889062582321 -87.59373731838016) bank143130 +143131 POINT(41.775898188651674 -87.48711820068009) bank143131 +143132 POINT(42.201018289640345 -86.91101682288566) bank143132 +143133 POINT(42.3235061016807 -87.16295194743348) bank143133 +143134 POINT(42.83778841325246 -87.29849545511134) bank143134 +143135 POINT(41.73701012944733 -86.90379711430093) bank143135 +143136 POINT(41.15595140554674 -86.904312862362) bank143136 +143137 POINT(41.192249888553135 -88.14617677512176) bank143137 +143138 POINT(42.63738845868495 -87.61987453503382) bank143138 +143139 POINT(41.91203676009246 -88.14618937093675) bank143139 +143140 POINT(42.00175056868644 -86.85762942956958) bank143140 +143141 POINT(40.96417534000954 -88.51208972679154) bank143141 +143142 POINT(41.13435808944682 -87.02330480946068) bank143142 +143143 POINT(41.751440814579794 -87.31375305646046) bank143143 +143144 POINT(42.239700099262606 -86.69129471970709) bank143144 +143145 POINT(42.56042759950758 -87.07049829328676) bank143145 +143146 POINT(42.182271181357414 -88.44219773135794) bank143146 +143147 POINT(41.63157870430366 -87.43593665585095) bank143147 +143148 POINT(42.757586888402784 -86.90632455291757) bank143148 +143149 POINT(41.28202867716399 -88.0155287590886) bank143149 +143150 POINT(41.853178593085396 -88.2603641780062) bank143150 +143151 POINT(42.458760184894835 -86.85925492886695) bank143151 +143152 POINT(40.88117111472724 -86.76728732521971) bank143152 +143153 POINT(41.61509963438137 -87.86693686630241) bank143153 +143154 POINT(41.46303447114065 -86.94538850560346) bank143154 +143155 POINT(41.605293308426056 -88.58959396261588) bank143155 +143156 POINT(42.21737220062305 -87.69832202794608) bank143156 +143157 POINT(41.55873710058496 -87.44792966857555) bank143157 +143158 POINT(42.72833652117752 -87.39380995810983) bank143158 +143159 POINT(41.71703648928863 -88.12331534867276) bank143159 +143160 POINT(42.43684208357829 -87.1542318818736) bank143160 +143161 POINT(42.33326384860625 -87.801192714129) bank143161 +143162 POINT(41.10802642346135 -87.68198208289584) bank143162 +143163 POINT(40.965057196879194 -88.31267801908645) bank143163 +143164 POINT(42.32948195451559 -88.1300641018037) bank143164 +143165 POINT(40.93209278501428 -86.98717234089708) bank143165 +143166 POINT(41.37694617880341 -88.20821757185534) bank143166 +143167 POINT(42.79693225519836 -88.13940053548394) bank143167 +143168 POINT(42.76789933870926 -88.02640646267555) bank143168 +143169 POINT(41.703826143993204 -86.84872835335577) bank143169 +143170 POINT(42.58160468562838 -87.89827276533146) bank143170 +143171 POINT(41.133409337256886 -87.9903373622517) bank143171 +143172 POINT(40.88552083260729 -87.16396634008736) bank143172 +143173 POINT(42.67207941799679 -87.93307782876136) bank143173 +143174 POINT(41.97247895972912 -86.88934501172136) bank143174 +143175 POINT(41.074671514575954 -86.83914722796916) bank143175 +143176 POINT(41.323084805038015 -87.12141552198382) bank143176 +143177 POINT(41.61561081597405 -86.84171344758929) bank143177 +143178 POINT(41.62287237960093 -88.04688504040594) bank143178 +143179 POINT(41.991463611572264 -87.4515895418364) bank143179 +143180 POINT(41.11305089522082 -88.1485661342096) bank143180 +143181 POINT(42.26235826147679 -87.56519071532486) bank143181 +143182 POINT(41.87370169234407 -88.30028197667329) bank143182 +143183 POINT(41.7203362735998 -86.6667834946296) bank143183 +143184 POINT(41.51493412919738 -86.86708520946968) bank143184 +143185 POINT(42.29572364709801 -87.49340305083096) bank143185 +143186 POINT(42.86307621020926 -87.1443764640477) bank143186 +143187 POINT(42.21843253208704 -86.96582348024643) bank143187 +143188 POINT(42.83666073123222 -87.09059465280588) bank143188 +143189 POINT(41.94798822039354 -87.65313530300855) bank143189 +143190 POINT(41.488603427356686 -88.22780046698445) bank143190 +143191 POINT(41.70100810159548 -87.06586595848515) bank143191 +143192 POINT(42.0151507387404 -88.52176574841934) bank143192 +143193 POINT(41.10971611848206 -87.06404664391474) bank143193 +143194 POINT(41.299089562283456 -87.13234131389508) bank143194 +143195 POINT(42.08302280293762 -86.94473558565282) bank143195 +143196 POINT(41.745970030163434 -87.75325863553319) bank143196 +143197 POINT(42.121386172089295 -88.1513063658263) bank143197 +143198 POINT(41.429491825759214 -88.34114216807114) bank143198 +143199 POINT(41.04365973156669 -87.95131739195534) bank143199 +143200 POINT(42.862005515918895 -86.73273940552856) bank143200 +143201 POINT(42.03810473737754 -88.5771643521814) bank143201 +143202 POINT(42.111275706533846 -86.89140965440941) bank143202 +143203 POINT(42.25075798372212 -87.50825520280416) bank143203 +143204 POINT(42.86960022827635 -87.36945142040642) bank143204 +143205 POINT(42.06243343240533 -87.1843415660977) bank143205 +143206 POINT(42.50297061619434 -87.01242610059803) bank143206 +143207 POINT(42.640934855380095 -86.89647835127104) bank143207 +143208 POINT(42.15119287153183 -87.46213240528888) bank143208 +143209 POINT(42.04648622881239 -87.24383424201635) bank143209 +143210 POINT(41.39515053236134 -87.01603550949842) bank143210 +143211 POINT(41.003335217478835 -86.91632554622093) bank143211 +143212 POINT(42.758422366016866 -87.67030352732064) bank143212 +143213 POINT(42.50719461687837 -87.14803119087392) bank143213 +143214 POINT(41.0052884998265 -88.55212390785546) bank143214 +143215 POINT(42.27435976762269 -87.80503674559314) bank143215 +143216 POINT(42.57248619574284 -87.94098759145587) bank143216 +143217 POINT(42.72569453520366 -87.30840167909302) bank143217 +143218 POINT(41.77598878181156 -88.00965095971337) bank143218 +143219 POINT(42.046886000179825 -87.95189175672705) bank143219 +143220 POINT(41.16086009768178 -87.98041779446575) bank143220 +143221 POINT(41.31810346651382 -87.76185163049338) bank143221 +143222 POINT(41.97034379617293 -87.92723341290757) bank143222 +143223 POINT(42.77152608063697 -87.32481076560299) bank143223 +143224 POINT(41.40565195816022 -88.30529366819441) bank143224 +143225 POINT(41.61346221392563 -88.20791308266797) bank143225 +143226 POINT(42.06290858669625 -87.01582859969197) bank143226 +143227 POINT(41.76217538690866 -88.33817046172496) bank143227 +143228 POINT(41.97702095222567 -86.92225262369517) bank143228 +143229 POINT(42.35465261573494 -88.17637111425914) bank143229 +143230 POINT(42.111500478706674 -88.00433448573091) bank143230 +143231 POINT(42.198997483162294 -88.2896383313856) bank143231 +143232 POINT(41.49914880150506 -88.47012803499534) bank143232 +143233 POINT(42.76944844839376 -86.68303541390611) bank143233 +143234 POINT(41.786428988320196 -87.90340222286639) bank143234 +143235 POINT(41.37419285510344 -86.87845255025113) bank143235 +143236 POINT(42.598902507239 -86.87580515937411) bank143236 +143237 POINT(41.24008413943407 -88.55283085821077) bank143237 +143238 POINT(41.75154537956825 -87.10425501010171) bank143238 +143239 POINT(40.97062132030479 -87.87819148058739) bank143239 +143240 POINT(42.68176596507614 -87.53268484708174) bank143240 +143241 POINT(42.208360856723104 -88.10328329333166) bank143241 +143242 POINT(41.748798883740236 -87.30297056689417) bank143242 +143243 POINT(42.635727024947656 -86.79666146732494) bank143243 +143244 POINT(41.42324213402655 -88.49954971219863) bank143244 +143245 POINT(41.03938462756614 -88.41987277096324) bank143245 +143246 POINT(41.70476842435357 -87.86278798601712) bank143246 +143247 POINT(41.2318978363244 -88.4380803557538) bank143247 +143248 POINT(40.902488087984935 -88.30006262318585) bank143248 +143249 POINT(42.455212277718864 -87.98642982020304) bank143249 +143250 POINT(41.641498313555715 -88.62154450949774) bank143250 +143251 POINT(41.14231182201556 -87.57163224974454) bank143251 +143252 POINT(41.38299368078413 -88.61554977436417) bank143252 +143253 POINT(42.8554046739065 -87.52455870411744) bank143253 +143254 POINT(41.196019602478444 -87.45134048517173) bank143254 +143255 POINT(41.44313962881901 -88.32409748246894) bank143255 +143256 POINT(41.56989256193084 -87.00992181505418) bank143256 +143257 POINT(41.86389460993427 -88.38812106378322) bank143257 +143258 POINT(41.20206140185065 -87.08509204743679) bank143258 +143259 POINT(41.643187163884136 -87.53003320314154) bank143259 +143260 POINT(42.1932610304065 -87.70501070941961) bank143260 +143261 POINT(42.067602131312626 -86.91774056609968) bank143261 +143262 POINT(41.74348087164218 -87.11999462107352) bank143262 +143263 POINT(41.12126600430489 -88.012721284315) bank143263 +143264 POINT(41.54118352386206 -88.27833239846156) bank143264 +143265 POINT(41.156845797854565 -87.92157706516218) bank143265 +143266 POINT(42.45369263767197 -87.94152240773155) bank143266 +143267 POINT(41.93291186798873 -87.40319198498666) bank143267 +143268 POINT(42.25496523318766 -87.42481524518632) bank143268 +143269 POINT(42.34247291387817 -87.60296310938105) bank143269 +143270 POINT(42.30591604408612 -88.4701396918821) bank143270 +143271 POINT(41.72890699943352 -87.48568976136562) bank143271 +143272 POINT(41.465635954145554 -87.94997925464463) bank143272 +143273 POINT(41.73005624368484 -86.65980142816827) bank143273 +143274 POINT(41.74589352814808 -88.46464617284013) bank143274 +143275 POINT(42.85036429131789 -86.81264293638729) bank143275 +143276 POINT(42.41188024544732 -87.50971522633577) bank143276 +143277 POINT(42.44568756252136 -87.44289403214074) bank143277 +143278 POINT(42.3322415856111 -88.41078908214308) bank143278 +143279 POINT(41.68433186221636 -88.15597947571884) bank143279 +143280 POINT(41.08711861911053 -87.90032438520603) bank143280 +143281 POINT(41.05313896445469 -87.02975771640715) bank143281 +143282 POINT(41.712200611375756 -86.69420048778989) bank143282 +143283 POINT(42.474764824926005 -87.37708910667479) bank143283 +143284 POINT(41.863067173355894 -88.26504178328942) bank143284 +143285 POINT(42.47181952519641 -87.65226107180037) bank143285 +143286 POINT(42.46352890605189 -88.29973897017395) bank143286 +143287 POINT(41.61579780619107 -86.79030622742872) bank143287 +143288 POINT(41.37549134902546 -86.70977987367648) bank143288 +143289 POINT(41.3215958783273 -87.14544450195514) bank143289 +143290 POINT(42.28089311077184 -87.98415715513974) bank143290 +143291 POINT(41.03223376232173 -86.68072024308067) bank143291 +143292 POINT(41.003274086842765 -87.06819960561022) bank143292 +143293 POINT(41.378625665997546 -86.85590564033153) bank143293 +143294 POINT(41.42136149744529 -87.87466999447106) bank143294 +143295 POINT(41.005901596808826 -87.39157728173777) bank143295 +143296 POINT(41.61292919880089 -87.92427073545683) bank143296 +143297 POINT(42.41950098489256 -88.56448314822407) bank143297 +143298 POINT(42.39594643892427 -86.71325301906434) bank143298 +143299 POINT(42.3336378420525 -87.07890925361512) bank143299 +143300 POINT(42.02970893498632 -87.18867706824702) bank143300 +143301 POINT(41.00928927318369 -87.7954317827828) bank143301 +143302 POINT(41.46954226104942 -88.08381573182587) bank143302 +143303 POINT(41.76716378659148 -86.9646297617632) bank143303 +143304 POINT(42.645012826022686 -87.37773819394233) bank143304 +143305 POINT(42.64372352542427 -87.26111202193268) bank143305 +143306 POINT(41.13703870480098 -88.41179336128828) bank143306 +143307 POINT(42.32201134060261 -87.25759964938115) bank143307 +143308 POINT(41.351088663066456 -88.30511636866913) bank143308 +143309 POINT(41.971174003088116 -87.18106789454306) bank143309 +143310 POINT(41.283884309469045 -88.04479287369261) bank143310 +143311 POINT(41.841138305993695 -87.38378891279018) bank143311 +143312 POINT(41.466616110109634 -88.40444022055964) bank143312 +143313 POINT(41.23561220915851 -87.01613793471053) bank143313 +143314 POINT(42.3104577763348 -88.45337135517195) bank143314 +143315 POINT(42.82581323775093 -88.53765900004684) bank143315 +143316 POINT(41.0806941739994 -88.2523328562755) bank143316 +143317 POINT(42.588780763273775 -88.2304784336576) bank143317 +143318 POINT(42.8400521851269 -88.04916562464547) bank143318 +143319 POINT(41.50516650740474 -87.76724543479754) bank143319 +143320 POINT(41.52965028959485 -88.39617585785398) bank143320 +143321 POINT(41.280138451092796 -87.18958054571281) bank143321 +143322 POINT(42.59376788972908 -87.05168344231748) bank143322 +143323 POINT(41.663073489190566 -88.23606929677463) bank143323 +143324 POINT(41.001886627613004 -87.68714102515081) bank143324 +143325 POINT(41.540518918728104 -87.22995651670645) bank143325 +143326 POINT(42.006828548590754 -87.93971513666054) bank143326 +143327 POINT(40.932443172506645 -88.43420589386143) bank143327 +143328 POINT(42.433779108598124 -88.47274313342834) bank143328 +143329 POINT(41.90184490570694 -87.02089209881625) bank143329 +143330 POINT(41.116450963440336 -87.8932243208117) bank143330 +143331 POINT(42.2868082709288 -87.51161471103757) bank143331 +143332 POINT(42.86395492383492 -86.85722265793696) bank143332 +143333 POINT(41.30275980146769 -86.73598579821657) bank143333 +143334 POINT(41.87666588839282 -86.92940467416035) bank143334 +143335 POINT(41.506535439430806 -86.76969535609547) bank143335 +143336 POINT(41.27480781060452 -87.52546035330612) bank143336 +143337 POINT(41.493315640679114 -87.04410334852865) bank143337 +143338 POINT(41.075845145016906 -87.59516103960686) bank143338 +143339 POINT(41.819996758303276 -88.21077528319881) bank143339 +143340 POINT(42.55438486544553 -86.77008056748652) bank143340 +143341 POINT(41.14666558034525 -87.11205731512392) bank143341 +143342 POINT(41.92178020303447 -87.7168883730334) bank143342 +143343 POINT(41.97858030913087 -87.51317383767908) bank143343 +143344 POINT(42.844670696664465 -88.0800154025833) bank143344 +143345 POINT(42.471269146775505 -88.54872652151154) bank143345 +143346 POINT(42.50861552818065 -87.3860526919525) bank143346 +143347 POINT(41.67214646277214 -87.54021210889333) bank143347 +143348 POINT(42.67822771288627 -88.25263964568424) bank143348 +143349 POINT(42.56690771242804 -86.78835395358635) bank143349 +143350 POINT(41.99892263863511 -88.20518420466456) bank143350 +143351 POINT(41.04036596649753 -87.17240080867083) bank143351 +143352 POINT(41.26804976722183 -86.92663839427902) bank143352 +143353 POINT(41.92053640176574 -87.0635295901906) bank143353 +143354 POINT(42.558016597226064 -87.17523408884534) bank143354 +143355 POINT(41.705097878568694 -88.1130897742812) bank143355 +143356 POINT(41.343247638598335 -87.51306122262004) bank143356 +143357 POINT(41.487527453881405 -87.68310364890952) bank143357 +143358 POINT(41.80903392423279 -86.92127424268645) bank143358 +143359 POINT(41.529041527410925 -88.6008107645403) bank143359 +143360 POINT(42.348989731010654 -88.05067236853182) bank143360 +143361 POINT(40.98403951743634 -88.27179555879677) bank143361 +143362 POINT(41.83540409321091 -86.95825495657891) bank143362 +143363 POINT(42.34091563671025 -87.9127298034978) bank143363 +143364 POINT(42.83930266870602 -88.62165309020973) bank143364 +143365 POINT(41.55191520053666 -87.6822882407805) bank143365 +143366 POINT(41.3364773466156 -87.90448064563483) bank143366 +143367 POINT(42.51345269494249 -87.8351991660163) bank143367 +143368 POINT(41.91199147608907 -87.86505462526263) bank143368 +143369 POINT(42.541605086139825 -87.32714083432835) bank143369 +143370 POINT(42.49038635967834 -87.28084019256421) bank143370 +143371 POINT(42.22152846339102 -87.92535218977434) bank143371 +143372 POINT(41.75964052618905 -87.98104282541807) bank143372 +143373 POINT(42.00013895012905 -86.68314289385606) bank143373 +143374 POINT(42.48818549666491 -88.55476674563508) bank143374 +143375 POINT(40.90985737866007 -87.55676701434471) bank143375 +143376 POINT(42.672735286720616 -87.95535936723678) bank143376 +143377 POINT(42.66920822623337 -88.38826544867003) bank143377 +143378 POINT(41.2463109969065 -88.52775737914422) bank143378 +143379 POINT(42.06308415613077 -87.86940698598116) bank143379 +143380 POINT(42.840928588102585 -87.60894010007523) bank143380 +143381 POINT(42.512227820938875 -87.23307279420895) bank143381 +143382 POINT(41.771096690889436 -88.52813872410076) bank143382 +143383 POINT(42.86911890674581 -87.91113733809566) bank143383 +143384 POINT(41.434058878388555 -86.82612980656013) bank143384 +143385 POINT(42.71561375023599 -88.12670700483167) bank143385 +143386 POINT(42.56602748941053 -86.76860376293341) bank143386 +143387 POINT(40.969378572745335 -87.32111326898873) bank143387 +143388 POINT(41.86810950241422 -86.74075243287838) bank143388 +143389 POINT(41.03327404331402 -86.64773098443852) bank143389 +143390 POINT(42.416085398025636 -88.61557429498704) bank143390 +143391 POINT(42.8008754242675 -87.92161283126194) bank143391 +143392 POINT(41.955350893094284 -87.78675843498976) bank143392 +143393 POINT(41.01981607889139 -87.30693804988165) bank143393 +143394 POINT(41.35260966066921 -88.01691997533774) bank143394 +143395 POINT(42.607485307088254 -88.60467505812957) bank143395 +143396 POINT(41.850547078861695 -87.03603813421655) bank143396 +143397 POINT(42.648501335160105 -87.57783723746665) bank143397 +143398 POINT(42.663375827802206 -87.37272950393134) bank143398 +143399 POINT(41.88293622552389 -86.90975635835264) bank143399 +143400 POINT(40.98651727393599 -86.7915262845295) bank143400 +143401 POINT(42.637925024168176 -86.87427272192113) bank143401 +143402 POINT(41.86363602670193 -88.53970251466883) bank143402 +143403 POINT(41.5228426622785 -86.87042177611337) bank143403 +143404 POINT(41.1234734935422 -87.41753373843838) bank143404 +143405 POINT(41.93698505982192 -88.37155130587722) bank143405 +143406 POINT(42.45473374540272 -87.82934035234751) bank143406 +143407 POINT(41.1584471593362 -87.10014808282807) bank143407 +143408 POINT(42.65311251984451 -88.48309074048761) bank143408 +143409 POINT(42.67560740785417 -88.16830260683126) bank143409 +143410 POINT(42.26318611399109 -87.30998564313533) bank143410 +143411 POINT(42.79810473152238 -88.07148603936723) bank143411 +143412 POINT(41.944236935036265 -86.90278858532393) bank143412 +143413 POINT(42.03046520322018 -88.44993553902833) bank143413 +143414 POINT(41.5563518742737 -86.80397261071224) bank143414 +143415 POINT(41.16016669905744 -87.32081553625875) bank143415 +143416 POINT(42.646027562765696 -88.36116723733988) bank143416 +143417 POINT(41.00643007481456 -88.33516393769462) bank143417 +143418 POINT(41.599475560369335 -87.95355657539874) bank143418 +143419 POINT(40.87867517956817 -87.34282119975524) bank143419 +143420 POINT(42.473169912822904 -88.21720036133895) bank143420 +143421 POINT(41.709298119748595 -87.22604620393757) bank143421 +143422 POINT(42.473801587585314 -87.63190583377198) bank143422 +143423 POINT(40.99559904119441 -87.92355983657505) bank143423 +143424 POINT(41.649386547711934 -88.54400885782692) bank143424 +143425 POINT(42.01640172277597 -86.67849992588148) bank143425 +143426 POINT(41.572371774948564 -88.02585754179101) bank143426 +143427 POINT(42.320337264368206 -87.08263270010517) bank143427 +143428 POINT(41.370640930596245 -87.86388584801054) bank143428 +143429 POINT(41.806941372865005 -86.65958994549587) bank143429 +143430 POINT(42.629133691290505 -87.33353554853936) bank143430 +143431 POINT(42.24243344671965 -87.286456461171) bank143431 +143432 POINT(41.26660195625813 -86.84344942627372) bank143432 +143433 POINT(41.85371815550009 -87.3764448767657) bank143433 +143434 POINT(41.647232258527126 -86.78332355339586) bank143434 +143435 POINT(41.86264508651985 -87.13114845526775) bank143435 +143436 POINT(42.75905082292317 -87.51048429665784) bank143436 +143437 POINT(42.26907263130762 -86.70450575954204) bank143437 +143438 POINT(42.374688165556655 -88.42163695828685) bank143438 +143439 POINT(41.16325629506371 -88.52425963299251) bank143439 +143440 POINT(41.43133602024375 -87.17290192402731) bank143440 +143441 POINT(42.262474447927026 -88.28818760737016) bank143441 +143442 POINT(41.334330719395716 -87.165510126068) bank143442 +143443 POINT(41.51404138021764 -86.77637003884202) bank143443 +143444 POINT(40.90823882963478 -88.03816623367247) bank143444 +143445 POINT(41.49498769269671 -87.26841461526733) bank143445 +143446 POINT(42.49576960178836 -86.81188140990905) bank143446 +143447 POINT(41.0925862980618 -86.76065904365505) bank143447 +143448 POINT(41.09131381380547 -87.65262659588115) bank143448 +143449 POINT(42.29966818316984 -88.08029730930511) bank143449 +143450 POINT(42.23055021541078 -88.50242002182756) bank143450 +143451 POINT(42.34224808991221 -86.779750874547) bank143451 +143452 POINT(41.265746517818926 -87.36400649324227) bank143452 +143453 POINT(42.68493713590861 -87.01200931918207) bank143453 +143454 POINT(42.481783005626234 -87.74624817182917) bank143454 +143455 POINT(42.672937730472434 -88.38697114901157) bank143455 +143456 POINT(42.065091866383874 -87.55189815349209) bank143456 +143457 POINT(42.32161451285199 -86.83599855725606) bank143457 +143458 POINT(42.6806327012939 -87.89583179389668) bank143458 +143459 POINT(42.33338425072944 -87.80377914317972) bank143459 +143460 POINT(42.32698717151598 -88.53213930449068) bank143460 +143461 POINT(42.682987878357345 -88.24342762142435) bank143461 +143462 POINT(41.17297363866989 -88.37537686175803) bank143462 +143463 POINT(41.81579579691523 -87.28432815545715) bank143463 +143464 POINT(42.1645097416096 -87.59028733015323) bank143464 +143465 POINT(40.99883435164604 -87.03427330536985) bank143465 +143466 POINT(41.02349921725851 -88.06678740464764) bank143466 +143467 POINT(42.01237022112738 -87.7732577242181) bank143467 +143468 POINT(41.04740977415385 -88.36769791403135) bank143468 +143469 POINT(41.50643269625939 -86.94302573187166) bank143469 +143470 POINT(42.673294473670985 -88.1287188526869) bank143470 +143471 POINT(42.35781803475684 -86.89083603000161) bank143471 +143472 POINT(40.98291461886681 -88.1544518747075) bank143472 +143473 POINT(41.57702217226018 -88.29078882910001) bank143473 +143474 POINT(41.5559417683376 -86.87505881497458) bank143474 +143475 POINT(41.951346084943395 -88.10103387049227) bank143475 +143476 POINT(41.9675849021297 -87.89190353299428) bank143476 +143477 POINT(41.18540452057796 -87.14544120175015) bank143477 +143478 POINT(42.36779303343427 -87.25263449125752) bank143478 +143479 POINT(41.8349491636135 -88.52144222762152) bank143479 +143480 POINT(42.79514033395592 -86.74785132247307) bank143480 +143481 POINT(42.69914725423192 -87.54478229803877) bank143481 +143482 POINT(42.057894033950376 -87.84279867708413) bank143482 +143483 POINT(42.06426250844678 -87.47030327222662) bank143483 +143484 POINT(41.82913518266773 -88.28492628092044) bank143484 +143485 POINT(42.35796451090581 -88.01707106110901) bank143485 +143486 POINT(40.97950004591372 -87.69177835624387) bank143486 +143487 POINT(42.19697139954532 -88.21883010030082) bank143487 +143488 POINT(42.54437327659074 -88.60069222382018) bank143488 +143489 POINT(40.95261067950396 -86.69634737630619) bank143489 +143490 POINT(42.33628438702049 -87.53021405399909) bank143490 +143491 POINT(41.052810785105436 -87.7621264988705) bank143491 +143492 POINT(42.42411451783084 -86.67752608527832) bank143492 +143493 POINT(42.02259765055844 -86.89128072961991) bank143493 +143494 POINT(42.02140814616895 -88.60766219687645) bank143494 +143495 POINT(41.790142376940864 -87.36782261640055) bank143495 +143496 POINT(42.87194063065983 -87.70134388098543) bank143496 +143497 POINT(42.60622056567548 -88.19023998878684) bank143497 +143498 POINT(41.6963161383706 -88.33898057854408) bank143498 +143499 POINT(42.19721305642644 -86.89649874894906) bank143499 +143500 POINT(41.08390927896603 -87.37246244226978) bank143500 +143501 POINT(42.40343960733179 -87.65344819244012) bank143501 +143502 POINT(41.465047183968956 -88.52547075425592) bank143502 +143503 POINT(42.18909192982497 -88.28095361211751) bank143503 +143504 POINT(41.12872278633652 -87.98159604710031) bank143504 +143505 POINT(41.17382570355227 -88.18368427804823) bank143505 +143506 POINT(40.92031114890146 -86.88083733420595) bank143506 +143507 POINT(41.41150591125103 -86.81517080494169) bank143507 +143508 POINT(41.43473883230199 -87.93978666131667) bank143508 +143509 POINT(42.59841059585115 -88.34833255889905) bank143509 +143510 POINT(41.75174942772507 -86.70124035373254) bank143510 +143511 POINT(42.727634366815636 -87.40969595524801) bank143511 +143512 POINT(42.11189526502537 -88.04951273794616) bank143512 +143513 POINT(41.340454935316906 -87.56620868478163) bank143513 +143514 POINT(41.29071523728488 -87.36224610535353) bank143514 +143515 POINT(41.85023167635692 -86.94768956834093) bank143515 +143516 POINT(40.978511278766256 -86.96269871981846) bank143516 +143517 POINT(41.88732346089256 -87.23379903421645) bank143517 +143518 POINT(42.69256890368953 -87.73855622465247) bank143518 +143519 POINT(41.170466370127286 -87.95762353557745) bank143519 +143520 POINT(42.84003744034311 -87.30681751690041) bank143520 +143521 POINT(41.69582909607522 -87.71896857494423) bank143521 +143522 POINT(42.65955889275569 -87.68149712822796) bank143522 +143523 POINT(41.8078427285315 -87.9855698928701) bank143523 +143524 POINT(42.7789181626589 -87.25619114947652) bank143524 +143525 POINT(41.86690413269892 -88.47223790730216) bank143525 +143526 POINT(42.38280396528803 -86.94743202410903) bank143526 +143527 POINT(41.95352584849376 -87.29369083497161) bank143527 +143528 POINT(41.241228792702834 -87.25153315544907) bank143528 +143529 POINT(41.99531475295858 -87.12420190437174) bank143529 +143530 POINT(41.488391081953836 -88.1389878731388) bank143530 +143531 POINT(42.80289458243966 -87.62523977744323) bank143531 +143532 POINT(41.71575004896533 -87.28044493672002) bank143532 +143533 POINT(41.0801468398651 -87.63815258149044) bank143533 +143534 POINT(41.19772329279493 -88.25372118264448) bank143534 +143535 POINT(42.81950641720112 -87.57118251237623) bank143535 +143536 POINT(41.630899307519115 -87.24243138326365) bank143536 +143537 POINT(41.485257936050324 -87.69430224694148) bank143537 +143538 POINT(42.19641287251179 -88.2155181054954) bank143538 +143539 POINT(41.810071425187225 -88.28937097896419) bank143539 +143540 POINT(42.19107125689828 -87.32722349911494) bank143540 +143541 POINT(41.127209857303654 -87.34599719460815) bank143541 +143542 POINT(41.32047317529493 -87.53859878506168) bank143542 +143543 POINT(42.07403832441717 -87.5717804680255) bank143543 +143544 POINT(41.58095394784102 -88.17192473871762) bank143544 +143545 POINT(42.55080050161062 -87.20692237611362) bank143545 +143546 POINT(42.02556978654377 -87.2600477357979) bank143546 +143547 POINT(40.94752056099566 -87.76640754297162) bank143547 +143548 POINT(41.1601609790933 -87.94684655509313) bank143548 +143549 POINT(41.12187154138025 -87.9051028969081) bank143549 +143550 POINT(42.69282449736228 -87.32956617035651) bank143550 +143551 POINT(40.98702557144196 -88.07731948695773) bank143551 +143552 POINT(42.69844900458446 -87.56046268469451) bank143552 +143553 POINT(41.449684405351825 -87.72932144368257) bank143553 +143554 POINT(41.81584966959067 -88.32975922786005) bank143554 +143555 POINT(41.640726103609886 -87.61539767646391) bank143555 +143556 POINT(42.40826725363337 -87.07762057896277) bank143556 +143557 POINT(41.93974640043542 -87.02734772699954) bank143557 +143558 POINT(41.45861000911083 -87.94653683588969) bank143558 +143559 POINT(42.26740295882573 -86.83535697974763) bank143559 +143560 POINT(41.880881357362696 -87.99595195284603) bank143560 +143561 POINT(41.05861248019996 -87.12374995964284) bank143561 +143562 POINT(42.3130398085239 -88.42971493455673) bank143562 +143563 POINT(41.548679585868584 -88.57774080238678) bank143563 +143564 POINT(41.17539626328358 -88.53906823919546) bank143564 +143565 POINT(41.3303684126006 -87.50604426375612) bank143565 +143566 POINT(41.15297294834385 -86.93467150263416) bank143566 +143567 POINT(42.55386785432511 -86.77599181489133) bank143567 +143568 POINT(40.9127825622041 -87.36975594358437) bank143568 +143569 POINT(42.63527050362565 -88.47355397032682) bank143569 +143570 POINT(42.66329585053443 -88.61197281415043) bank143570 +143571 POINT(42.06701928125381 -87.17961569625825) bank143571 +143572 POINT(41.16505588787604 -87.25856172330226) bank143572 +143573 POINT(42.472523218514674 -87.16675521175414) bank143573 +143574 POINT(42.84038140567049 -86.68470650394977) bank143574 +143575 POINT(42.0449571730117 -87.28095704418426) bank143575 +143576 POINT(41.803021882484686 -88.02454173834015) bank143576 +143577 POINT(41.218503488093994 -87.69787980838873) bank143577 +143578 POINT(42.17617351836536 -88.61312311201898) bank143578 +143579 POINT(41.93576037975361 -87.50900216111958) bank143579 +143580 POINT(41.59858612823704 -87.27698182442792) bank143580 +143581 POINT(42.30926086194622 -87.18812730085483) bank143581 +143582 POINT(42.04064683578557 -87.35749135565135) bank143582 +143583 POINT(41.24872338311606 -87.9585955589406) bank143583 +143584 POINT(42.468525497215936 -88.25444345767639) bank143584 +143585 POINT(41.07269542925691 -87.16502445269126) bank143585 +143586 POINT(41.51685192470123 -88.112401244877) bank143586 +143587 POINT(41.0267913910022 -87.14767353190925) bank143587 +143588 POINT(42.62164418349465 -86.69123051484337) bank143588 +143589 POINT(41.91911738101275 -87.99129368914396) bank143589 +143590 POINT(42.45912292167748 -86.86699646739073) bank143590 +143591 POINT(42.845307931976365 -87.93598330980359) bank143591 +143592 POINT(41.04845735064192 -86.75151762026539) bank143592 +143593 POINT(41.51412227404567 -88.14535355640913) bank143593 +143594 POINT(41.09301703509287 -87.09717039942404) bank143594 +143595 POINT(41.49717256289387 -88.52026894249434) bank143595 +143596 POINT(41.4308403289921 -87.31620511739246) bank143596 +143597 POINT(41.8316503159371 -86.65926636047926) bank143597 +143598 POINT(42.837654338232284 -86.78568114680081) bank143598 +143599 POINT(41.6081293292139 -87.99702301253748) bank143599 +143600 POINT(41.05462172204521 -87.25107489917248) bank143600 +143601 POINT(41.74333309854511 -88.56507600073986) bank143601 +143602 POINT(41.315937027192724 -87.37689272121958) bank143602 +143603 POINT(41.57533000947744 -87.79962078273851) bank143603 +143604 POINT(42.51588230795101 -87.57808518986063) bank143604 +143605 POINT(42.26715324417426 -87.68960348617249) bank143605 +143606 POINT(42.22826850835815 -86.83226827672978) bank143606 +143607 POINT(42.11479651037933 -87.44556985969885) bank143607 +143608 POINT(41.58334939875991 -87.88451687947932) bank143608 +143609 POINT(42.64572249408763 -87.0208735805047) bank143609 +143610 POINT(42.412710378667754 -88.53384097887975) bank143610 +143611 POINT(41.6885222815054 -86.92208619185956) bank143611 +143612 POINT(42.62468579183294 -88.4595827491634) bank143612 +143613 POINT(42.87257563840446 -87.5775892643133) bank143613 +143614 POINT(42.049724770571565 -87.99661176965118) bank143614 +143615 POINT(41.935173279614816 -88.29881249260906) bank143615 +143616 POINT(42.615663552624234 -88.19818717881164) bank143616 +143617 POINT(40.990904549643304 -87.44715513156606) bank143617 +143618 POINT(41.766351744683206 -88.26698262828529) bank143618 +143619 POINT(42.70527706657066 -87.91550512016703) bank143619 +143620 POINT(42.59980418860784 -88.20002501719917) bank143620 +143621 POINT(42.516176956640635 -88.51555726952317) bank143621 +143622 POINT(42.093754872030175 -88.1726560533317) bank143622 +143623 POINT(42.14803712591834 -87.56056955501637) bank143623 +143624 POINT(42.07852334144968 -88.40396910408647) bank143624 +143625 POINT(41.99181083031446 -87.9190459135173) bank143625 +143626 POINT(42.30636534833423 -88.21743715224346) bank143626 +143627 POINT(42.050716591947456 -88.00867847768336) bank143627 +143628 POINT(41.56375086690055 -88.5319794384537) bank143628 +143629 POINT(42.33268385712742 -86.64908930552012) bank143629 +143630 POINT(40.89080055878517 -86.93849969413357) bank143630 +143631 POINT(42.82782060230019 -88.29167750005946) bank143631 +143632 POINT(42.27718529088775 -87.9544163761901) bank143632 +143633 POINT(42.550816822417886 -87.43813346788112) bank143633 +143634 POINT(42.834799030487446 -86.85619489483585) bank143634 +143635 POINT(40.885292832696464 -87.7940033760539) bank143635 +143636 POINT(41.423036442093284 -88.31821387866785) bank143636 +143637 POINT(42.47560707904461 -86.72640848374213) bank143637 +143638 POINT(40.98243667168892 -86.75580240499545) bank143638 +143639 POINT(40.95024275340429 -87.70254501844538) bank143639 +143640 POINT(42.113926280321316 -87.53010348670959) bank143640 +143641 POINT(41.872352972806624 -87.0246617951998) bank143641 +143642 POINT(41.33576236627379 -87.92978849570291) bank143642 +143643 POINT(41.2443783182723 -86.68806678513373) bank143643 +143644 POINT(41.18441365272029 -88.17601550757003) bank143644 +143645 POINT(41.38037961649327 -88.29171665524012) bank143645 +143646 POINT(42.65448491013071 -88.34133894015059) bank143646 +143647 POINT(42.07226983531096 -87.76498597541168) bank143647 +143648 POINT(40.954237347002035 -87.20053666078371) bank143648 +143649 POINT(42.66073402148374 -87.77976792700679) bank143649 +143650 POINT(41.131670147460454 -87.56529624552505) bank143650 +143651 POINT(40.9981259702354 -88.51383607378105) bank143651 +143652 POINT(42.688444024279256 -86.9954693966886) bank143652 +143653 POINT(42.58478110309907 -87.4952642785526) bank143653 +143654 POINT(41.26889152819542 -87.08329032348303) bank143654 +143655 POINT(41.82344044045779 -88.41174111038082) bank143655 +143656 POINT(42.684671747972786 -88.41499333436623) bank143656 +143657 POINT(41.23324003915637 -86.78346004309138) bank143657 +143658 POINT(42.46944703653754 -86.86385356183281) bank143658 +143659 POINT(41.91855663042753 -88.55914295021839) bank143659 +143660 POINT(42.876396537227436 -87.5643604771759) bank143660 +143661 POINT(41.258122190204936 -87.44563450912983) bank143661 +143662 POINT(40.94093222057552 -87.95860649208956) bank143662 +143663 POINT(42.06917658447513 -87.01090172717221) bank143663 +143664 POINT(42.47936735147272 -87.1235861706078) bank143664 +143665 POINT(41.48945897221982 -87.69564273850077) bank143665 +143666 POINT(42.332773927208045 -87.90274417139037) bank143666 +143667 POINT(42.10445484614601 -88.20958672882288) bank143667 +143668 POINT(41.880091441176184 -87.3372910184347) bank143668 +143669 POINT(41.05131301036402 -88.26290760281329) bank143669 +143670 POINT(41.76892505683074 -87.11604971920673) bank143670 +143671 POINT(41.01985149977592 -86.9576471195512) bank143671 +143672 POINT(42.54192528604779 -88.32341768553702) bank143672 +143673 POINT(42.73617137042372 -88.08842104106662) bank143673 +143674 POINT(41.75205132293314 -88.26794665246304) bank143674 +143675 POINT(42.58177725851348 -88.51614956005415) bank143675 +143676 POINT(42.85249255443301 -87.81042701224668) bank143676 +143677 POINT(42.65317180836482 -87.95734495605342) bank143677 +143678 POINT(42.752876696222195 -87.26631136850006) bank143678 +143679 POINT(41.84804392779876 -87.02161970708676) bank143679 +143680 POINT(41.813503978239744 -88.62495354513443) bank143680 +143681 POINT(41.79318753961401 -88.12200321925907) bank143681 +143682 POINT(42.24665233704208 -88.26944004534705) bank143682 +143683 POINT(42.592034269245886 -86.9186770590068) bank143683 +143684 POINT(42.07901277398214 -88.44823546279184) bank143684 +143685 POINT(42.301727734635655 -87.95214415020997) bank143685 +143686 POINT(41.61163482829766 -88.21947307565921) bank143686 +143687 POINT(41.177718907774796 -87.45647646287878) bank143687 +143688 POINT(41.338762062460596 -87.57188286919545) bank143688 +143689 POINT(42.784176819311014 -88.55540095559425) bank143689 +143690 POINT(41.02207364614243 -86.71739698105002) bank143690 +143691 POINT(41.68839560942714 -88.14540863274806) bank143691 +143692 POINT(41.944268256768304 -87.00977688132126) bank143692 +143693 POINT(42.47391714542292 -86.88272208708865) bank143693 +143694 POINT(42.20057593959467 -87.28943064257676) bank143694 +143695 POINT(41.52350103983472 -87.40431666537874) bank143695 +143696 POINT(40.98417294025541 -87.39820015957812) bank143696 +143697 POINT(42.7534632435427 -87.65443688255101) bank143697 +143698 POINT(41.32736652887201 -87.10775306957565) bank143698 +143699 POINT(41.860703723867815 -87.52623515830781) bank143699 +143700 POINT(41.33310537347345 -86.7505699190919) bank143700 +143701 POINT(41.374572968136555 -88.07843901020942) bank143701 +143702 POINT(41.83261766274191 -87.18832848453935) bank143702 +143703 POINT(42.40311035538835 -87.91526597728239) bank143703 +143704 POINT(41.99833598664832 -87.2395241695195) bank143704 +143705 POINT(42.52982978861815 -88.50941650456551) bank143705 +143706 POINT(41.39294792444559 -88.45489683356905) bank143706 +143707 POINT(41.621365566911095 -88.25905262094186) bank143707 +143708 POINT(42.33284975696162 -88.5150781833866) bank143708 +143709 POINT(41.573828522631636 -87.89325058679233) bank143709 +143710 POINT(41.040253167496445 -87.93013274466797) bank143710 +143711 POINT(42.85330183257635 -87.63096736388349) bank143711 +143712 POINT(42.66839641633889 -87.31829173316277) bank143712 +143713 POINT(41.37090914392843 -86.64088028182123) bank143713 +143714 POINT(41.50655425321662 -87.0777789577425) bank143714 +143715 POINT(42.02485946914997 -88.50730641463515) bank143715 +143716 POINT(41.92468318926992 -88.15527753618522) bank143716 +143717 POINT(42.45420314584235 -87.8648724166348) bank143717 +143718 POINT(41.80798260359766 -88.47480079140483) bank143718 +143719 POINT(41.30815321210782 -88.51045237829803) bank143719 +143720 POINT(41.48279701069926 -88.60593523304395) bank143720 +143721 POINT(42.09259531473862 -88.0098600817067) bank143721 +143722 POINT(42.30010112954549 -87.80260948081265) bank143722 +143723 POINT(42.31504212058191 -86.63866481256329) bank143723 +143724 POINT(42.82637876569519 -88.22573641195191) bank143724 +143725 POINT(41.98550548517761 -87.56540191811727) bank143725 +143726 POINT(42.56462200382716 -87.60018733265993) bank143726 +143727 POINT(42.754678846501115 -86.8198758862892) bank143727 +143728 POINT(42.164966278038555 -87.82562931021427) bank143728 +143729 POINT(42.765698211202135 -87.63642829830748) bank143729 +143730 POINT(41.661890888927026 -86.74771495615029) bank143730 +143731 POINT(42.587543370477626 -86.94818061199436) bank143731 +143732 POINT(42.62616154734438 -87.22835401230074) bank143732 +143733 POINT(41.26489641243488 -86.64607574659411) bank143733 +143734 POINT(41.663825256883655 -88.39056794957122) bank143734 +143735 POINT(41.591863591972576 -87.66545899826104) bank143735 +143736 POINT(42.74757769078559 -88.30313759585664) bank143736 +143737 POINT(41.88876939541334 -86.8195327728403) bank143737 +143738 POINT(42.24347292695651 -88.10792699346861) bank143738 +143739 POINT(41.665210948800535 -88.3299825077149) bank143739 +143740 POINT(42.67781205693624 -87.50980900155318) bank143740 +143741 POINT(42.78272330497118 -88.503211302273) bank143741 +143742 POINT(42.396125706847954 -87.60900967408844) bank143742 +143743 POINT(41.96294267297731 -87.69277482873984) bank143743 +143744 POINT(42.485014049971575 -86.74316889681167) bank143744 +143745 POINT(41.03284628292531 -88.06274878625814) bank143745 +143746 POINT(41.32524226191434 -88.30563252784222) bank143746 +143747 POINT(41.582746869322264 -88.62620571556776) bank143747 +143748 POINT(41.48158265668321 -87.33311377562771) bank143748 +143749 POINT(40.953499374893134 -87.46097440147241) bank143749 +143750 POINT(42.40563405255979 -87.67623286770375) bank143750 +143751 POINT(42.184949490975654 -87.88795747502856) bank143751 +143752 POINT(41.339057620820256 -88.01941174744267) bank143752 +143753 POINT(42.203915873333706 -88.0648499944137) bank143753 +143754 POINT(42.24220314053088 -87.38753014636127) bank143754 +143755 POINT(41.53911252184605 -86.76785641044069) bank143755 +143756 POINT(41.644879687092 -87.96968433400107) bank143756 +143757 POINT(42.197146898966665 -88.15198779513857) bank143757 +143758 POINT(42.325725121313134 -86.79813891084781) bank143758 +143759 POINT(41.20609105192575 -86.87362947383092) bank143759 +143760 POINT(41.132801520386415 -86.71293196188635) bank143760 +143761 POINT(41.36922609839158 -88.1577048289027) bank143761 +143762 POINT(41.43350759678567 -86.77007034264524) bank143762 +143763 POINT(41.73619625769112 -87.95662390325215) bank143763 +143764 POINT(42.66734889438048 -87.90110000494627) bank143764 +143765 POINT(41.75546652692768 -87.9795176227634) bank143765 +143766 POINT(42.55032455701492 -88.34001355872311) bank143766 +143767 POINT(42.056444228029086 -87.67986631708115) bank143767 +143768 POINT(42.23197336869934 -88.17422028221009) bank143768 +143769 POINT(42.22066423041055 -87.4705767594204) bank143769 +143770 POINT(41.74231912978117 -87.65194505014307) bank143770 +143771 POINT(41.79692029833542 -88.32719890813758) bank143771 +143772 POINT(41.79734728158755 -88.59991717214501) bank143772 +143773 POINT(41.62399307288219 -87.16210568011056) bank143773 +143774 POINT(41.8781730593581 -88.0019131637121) bank143774 +143775 POINT(41.97250609486689 -87.76671015175495) bank143775 +143776 POINT(42.76521457706689 -87.32895816990595) bank143776 +143777 POINT(42.27401412413707 -87.15136788831283) bank143777 +143778 POINT(41.59743233173589 -88.13428935506037) bank143778 +143779 POINT(42.152674426112924 -87.3937704606636) bank143779 +143780 POINT(41.98377458168094 -86.70745319257146) bank143780 +143781 POINT(41.0937166907298 -86.94465963803805) bank143781 +143782 POINT(41.63771319683629 -88.06616705987787) bank143782 +143783 POINT(40.898112648681675 -88.46763225113247) bank143783 +143784 POINT(41.2472532747015 -87.98974793168709) bank143784 +143785 POINT(42.49930028683896 -87.6570837326411) bank143785 +143786 POINT(42.50613553639444 -87.21378299681811) bank143786 +143787 POINT(41.388293482671344 -88.23888026400141) bank143787 +143788 POINT(41.96958713631143 -87.13553737884989) bank143788 +143789 POINT(41.74126944840922 -88.3171163077095) bank143789 +143790 POINT(41.13527298303433 -88.48036433412156) bank143790 +143791 POINT(42.5978578692616 -86.68639236187578) bank143791 +143792 POINT(41.108410835264486 -86.65132459910707) bank143792 +143793 POINT(41.747220239841894 -87.19765710852347) bank143793 +143794 POINT(42.608384465243645 -87.48406556936017) bank143794 +143795 POINT(42.342044214527654 -87.65838765164199) bank143795 +143796 POINT(42.877826404471946 -87.6847358879577) bank143796 +143797 POINT(42.163039997371506 -86.96646567788098) bank143797 +143798 POINT(42.06465811889673 -88.43003370112226) bank143798 +143799 POINT(42.425065503830005 -87.83314945718121) bank143799 +143800 POINT(41.60227968891325 -86.72659613366778) bank143800 +143801 POINT(42.65734839881756 -87.45713156092145) bank143801 +143802 POINT(41.783706755930595 -87.99313913726519) bank143802 +143803 POINT(42.196213496535655 -87.07992063989852) bank143803 +143804 POINT(42.39246105071409 -86.9622556201569) bank143804 +143805 POINT(42.10864583289876 -88.43747322742871) bank143805 +143806 POINT(42.77575662820602 -87.64099520215758) bank143806 +143807 POINT(42.41953162113959 -88.3240734534) bank143807 +143808 POINT(41.33165203520592 -87.28292968537842) bank143808 +143809 POINT(41.43155721115746 -88.08306775152488) bank143809 +143810 POINT(42.22932474962427 -86.78979490844107) bank143810 +143811 POINT(42.43783535592214 -88.39998775265522) bank143811 +143812 POINT(40.974384428427385 -87.86243736200876) bank143812 +143813 POINT(42.03825667895877 -87.07303186246381) bank143813 +143814 POINT(42.23058954049494 -87.2250306010562) bank143814 +143815 POINT(42.3415077325881 -87.2536851481942) bank143815 +143816 POINT(40.95567704702523 -88.21727465734105) bank143816 +143817 POINT(42.53527263860531 -87.66275038595215) bank143817 +143818 POINT(41.11952421377144 -87.66720256730387) bank143818 +143819 POINT(42.581855067963005 -87.23409227922974) bank143819 +143820 POINT(41.844527910865565 -87.76338274096477) bank143820 +143821 POINT(41.48162266014613 -87.12224628015124) bank143821 +143822 POINT(41.933942199648904 -86.66047815035145) bank143822 +143823 POINT(42.11656824014727 -88.47450660292071) bank143823 +143824 POINT(42.84046243326235 -88.59509324331911) bank143824 +143825 POINT(42.115905405157186 -88.42856314103074) bank143825 +143826 POINT(41.44825219425749 -88.2938022047853) bank143826 +143827 POINT(41.796147156787406 -86.74353026922878) bank143827 +143828 POINT(42.548391079769715 -88.50990490039368) bank143828 +143829 POINT(42.50508868699801 -87.18074076576227) bank143829 +143830 POINT(42.07443668281038 -87.46302220893034) bank143830 +143831 POINT(41.71374941924215 -87.21007035225546) bank143831 +143832 POINT(41.65372085697334 -87.970942462495) bank143832 +143833 POINT(41.22758743102965 -88.01988010608625) bank143833 +143834 POINT(41.013544194663694 -87.12539364729624) bank143834 +143835 POINT(41.620216894585944 -88.26837906165782) bank143835 +143836 POINT(42.63601269645945 -87.65321226553658) bank143836 +143837 POINT(42.47961298905319 -88.6087185548836) bank143837 +143838 POINT(42.49771487207778 -87.77861388192002) bank143838 +143839 POINT(42.62338120812595 -87.12183546928193) bank143839 +143840 POINT(41.32270030144223 -87.02712190885771) bank143840 +143841 POINT(41.87839812752418 -88.36833425650232) bank143841 +143842 POINT(41.81161515466175 -87.66626687100256) bank143842 +143843 POINT(41.99848984136202 -86.94904523523932) bank143843 +143844 POINT(41.613464530203665 -86.70898589503979) bank143844 +143845 POINT(41.60838197669959 -86.83668084368627) bank143845 +143846 POINT(41.3238682743414 -87.28892678471078) bank143846 +143847 POINT(42.516574152550064 -87.1554901015427) bank143847 +143848 POINT(41.70945870504326 -87.73115647298292) bank143848 +143849 POINT(40.970738426794476 -88.26881652486514) bank143849 +143850 POINT(42.75656983826313 -87.05084507499689) bank143850 +143851 POINT(41.755310181045246 -87.46748245903478) bank143851 +143852 POINT(42.73167482508738 -87.12565634155285) bank143852 +143853 POINT(41.13675372428123 -87.1436234319788) bank143853 +143854 POINT(41.38231082621103 -87.88891199506799) bank143854 +143855 POINT(42.289743922836585 -88.46247750263862) bank143855 +143856 POINT(41.1573362176326 -87.55277169180376) bank143856 +143857 POINT(42.59458368830886 -88.56737705624316) bank143857 +143858 POINT(41.92335656776978 -87.33560335307018) bank143858 +143859 POINT(42.17762785934171 -87.1713749553273) bank143859 +143860 POINT(41.14307144125548 -86.99392715178224) bank143860 +143861 POINT(42.44353070738157 -88.45821115749858) bank143861 +143862 POINT(41.0880450232222 -87.43599001581754) bank143862 +143863 POINT(40.87982580033081 -87.67748933187059) bank143863 +143864 POINT(42.56895023418897 -86.68847052321044) bank143864 +143865 POINT(42.00362386559474 -87.59622195720665) bank143865 +143866 POINT(41.820013197142 -88.36342391743551) bank143866 +143867 POINT(42.309192048339995 -87.80367665674206) bank143867 +143868 POINT(42.08936000543308 -87.38069635577577) bank143868 +143869 POINT(42.19520634754647 -88.40353449387085) bank143869 +143870 POINT(41.38060051447797 -88.02091877403745) bank143870 +143871 POINT(41.024634469866314 -87.26154115215128) bank143871 +143872 POINT(41.90786143935114 -87.7017280062737) bank143872 +143873 POINT(41.15649073322449 -87.3530754214587) bank143873 +143874 POINT(42.253798630681366 -87.86485206923668) bank143874 +143875 POINT(41.338409401620936 -87.3879749008638) bank143875 +143876 POINT(41.853728984512585 -88.46602922494601) bank143876 +143877 POINT(42.230198434556556 -87.01051393376133) bank143877 +143878 POINT(41.87549041658754 -88.5954315923181) bank143878 +143879 POINT(42.63700995601933 -86.94311637681757) bank143879 +143880 POINT(41.68193817278358 -88.26664374562499) bank143880 +143881 POINT(41.81493630905746 -87.79621985703801) bank143881 +143882 POINT(41.45996256648845 -88.59825270304479) bank143882 +143883 POINT(42.17100314272794 -87.46875929414031) bank143883 +143884 POINT(42.28835590228819 -87.12499285670796) bank143884 +143885 POINT(41.09968795434892 -86.72662365248746) bank143885 +143886 POINT(41.63316672481209 -87.6738334983475) bank143886 +143887 POINT(41.5245059330491 -87.4862953374636) bank143887 +143888 POINT(41.9014346194597 -86.94829854230375) bank143888 +143889 POINT(42.52116966674187 -86.85666924300746) bank143889 +143890 POINT(42.801133450256835 -88.36203146287716) bank143890 +143891 POINT(42.209057619561044 -88.26696185856719) bank143891 +143892 POINT(42.468494649219224 -87.62301619614604) bank143892 +143893 POINT(42.630742189536846 -88.33109046920679) bank143893 +143894 POINT(41.20472328582041 -87.78297569911979) bank143894 +143895 POINT(42.06565990165836 -87.47452740048719) bank143895 +143896 POINT(42.82109360511334 -87.63768347011008) bank143896 +143897 POINT(42.75868318723361 -87.30150958305074) bank143897 +143898 POINT(42.09070278927765 -87.66983063198796) bank143898 +143899 POINT(42.696125560261414 -86.77284664499764) bank143899 +143900 POINT(41.912905906263845 -87.2630282633576) bank143900 +143901 POINT(41.24419980602828 -87.77270269001397) bank143901 +143902 POINT(41.41356568754467 -86.64310005554522) bank143902 +143903 POINT(42.47586446912884 -87.85108985325662) bank143903 +143904 POINT(42.38463006155193 -86.85550698802014) bank143904 +143905 POINT(42.13122367968721 -86.7495433805545) bank143905 +143906 POINT(41.24420695698808 -88.43494704944983) bank143906 +143907 POINT(42.34889955845882 -87.29237700855907) bank143907 +143908 POINT(41.364706817203064 -88.12585186438059) bank143908 +143909 POINT(42.25150812604563 -86.97487753332481) bank143909 +143910 POINT(42.77733725548442 -87.64440416344539) bank143910 +143911 POINT(41.53847456949675 -88.61993271216707) bank143911 +143912 POINT(42.588429673359315 -86.7255383888922) bank143912 +143913 POINT(42.64336366486646 -87.36063253117311) bank143913 +143914 POINT(41.45901220317222 -88.4534800656712) bank143914 +143915 POINT(41.27103775675788 -88.1206963898585) bank143915 +143916 POINT(41.59009604527007 -87.02721517418941) bank143916 +143917 POINT(41.75059228247226 -87.63867477807753) bank143917 +143918 POINT(42.257376722506685 -88.24351725489761) bank143918 +143919 POINT(41.61246877168385 -88.31994920854139) bank143919 +143920 POINT(41.66176525808363 -87.18143804364146) bank143920 +143921 POINT(42.217858244429046 -88.14421306346485) bank143921 +143922 POINT(42.34198189769914 -87.80858706012324) bank143922 +143923 POINT(41.79204892135888 -86.81650754103379) bank143923 +143924 POINT(40.961245493559204 -87.89809083428213) bank143924 +143925 POINT(41.24238835464516 -88.61865714144908) bank143925 +143926 POINT(42.45720411629279 -88.50441428139179) bank143926 +143927 POINT(41.5856081747489 -86.69488649136397) bank143927 +143928 POINT(41.97305294865312 -86.63962262625571) bank143928 +143929 POINT(41.9688480638738 -87.91132963651012) bank143929 +143930 POINT(42.7888811753947 -86.89934303360475) bank143930 +143931 POINT(42.24474382422253 -88.09955406152498) bank143931 +143932 POINT(41.90529658878815 -86.63205502032164) bank143932 +143933 POINT(41.16245893263125 -88.48914112979311) bank143933 +143934 POINT(41.39488222723532 -87.76580198257399) bank143934 +143935 POINT(41.351833949322156 -88.08056773144256) bank143935 +143936 POINT(42.42430356800311 -87.77697669210536) bank143936 +143937 POINT(42.23056733618661 -87.25361244247499) bank143937 +143938 POINT(42.66262104637635 -87.53256570573484) bank143938 +143939 POINT(42.36651751484529 -87.89681684627995) bank143939 +143940 POINT(42.527210806177585 -87.1715937780428) bank143940 +143941 POINT(41.181865444882476 -88.61503244567035) bank143941 +143942 POINT(42.52988340794265 -87.11534244289122) bank143942 +143943 POINT(42.032325795710086 -86.88205798952993) bank143943 +143944 POINT(41.10791161033266 -86.80598199848333) bank143944 +143945 POINT(42.84985956360333 -88.5474526294937) bank143945 +143946 POINT(40.897658088688836 -87.57694120588623) bank143946 +143947 POINT(42.815698732776184 -87.22785279082426) bank143947 +143948 POINT(41.49233671496855 -86.8907138889957) bank143948 +143949 POINT(41.38620580415608 -88.57477318425924) bank143949 +143950 POINT(42.544359487072654 -87.30210927635831) bank143950 +143951 POINT(42.5661172792494 -86.86633598715122) bank143951 +143952 POINT(40.916188875066034 -87.13393551849316) bank143952 +143953 POINT(42.09944253481558 -87.33158465623102) bank143953 +143954 POINT(42.57479157014616 -87.08412762426909) bank143954 +143955 POINT(41.70183906123047 -88.22050721697111) bank143955 +143956 POINT(41.70670466332021 -86.78851814390605) bank143956 +143957 POINT(41.296627506544546 -87.36786099572424) bank143957 +143958 POINT(42.648673155126914 -88.43492230355966) bank143958 +143959 POINT(42.55268013269614 -86.8514832213688) bank143959 +143960 POINT(41.57237623930522 -86.83154348011806) bank143960 +143961 POINT(42.57748798428765 -86.78935667558162) bank143961 +143962 POINT(41.367972812377474 -87.4559656276935) bank143962 +143963 POINT(41.14774820832872 -87.18355900596164) bank143963 +143964 POINT(41.843537135576106 -87.33083064909634) bank143964 +143965 POINT(42.1084948196706 -88.3735813141009) bank143965 +143966 POINT(42.748995483248116 -87.5362064219174) bank143966 +143967 POINT(41.41558236294772 -87.30509437943222) bank143967 +143968 POINT(42.65285502764625 -87.06655349704792) bank143968 +143969 POINT(42.41184862376194 -87.99839733076826) bank143969 +143970 POINT(42.23683955651641 -88.47606970741111) bank143970 +143971 POINT(41.922963097275115 -86.92435136475767) bank143971 +143972 POINT(42.40471047817045 -87.34034503546559) bank143972 +143973 POINT(42.595365286566434 -87.87755845146826) bank143973 +143974 POINT(41.69136485664067 -88.5230831228708) bank143974 +143975 POINT(42.443018292107276 -87.93094757226763) bank143975 +143976 POINT(42.48755720926275 -86.76237855676446) bank143976 +143977 POINT(41.34247850056835 -88.35606236798635) bank143977 +143978 POINT(41.647738713100026 -88.10060823645703) bank143978 +143979 POINT(40.993678326771885 -86.86575802167337) bank143979 +143980 POINT(42.49403643015286 -87.12649720244497) bank143980 +143981 POINT(41.88153386647385 -87.87542484365157) bank143981 +143982 POINT(41.63975417304275 -87.78449425670964) bank143982 +143983 POINT(41.33175344021081 -87.6382236278691) bank143983 +143984 POINT(42.1359159006607 -87.64607452677237) bank143984 +143985 POINT(41.74861551194353 -86.880323230588) bank143985 +143986 POINT(42.69374120497351 -88.2795792886372) bank143986 +143987 POINT(41.37950490386466 -86.98496308614037) bank143987 +143988 POINT(42.494454290924956 -87.87070354770329) bank143988 +143989 POINT(42.69995731303471 -87.85314673707934) bank143989 +143990 POINT(42.42401942957456 -86.94554109144576) bank143990 +143991 POINT(41.901063275167644 -88.32606842525844) bank143991 +143992 POINT(42.693183787558375 -88.46874521498738) bank143992 +143993 POINT(41.826291618057155 -87.71386695910098) bank143993 +143994 POINT(41.437517032637764 -88.19609595877836) bank143994 +143995 POINT(42.053064299053304 -87.85516162229305) bank143995 +143996 POINT(41.55627920706733 -87.9326954687888) bank143996 +143997 POINT(42.58542760367398 -86.78742138700899) bank143997 +143998 POINT(41.589031126106526 -88.04740156991701) bank143998 +143999 POINT(42.66520321854663 -86.89619277580371) bank143999 +144000 POINT(41.33300020827383 -87.19322789608185) bank144000 +144001 POINT(42.04018643190488 -88.06199167177031) bank144001 +144002 POINT(40.89394490734853 -86.75354404091959) bank144002 +144003 POINT(41.53739376744273 -87.11218412138614) bank144003 +144004 POINT(41.89014471741095 -87.46454297320207) bank144004 +144005 POINT(42.06002336839106 -88.40100847172792) bank144005 +144006 POINT(42.63038305097177 -88.34761005841993) bank144006 +144007 POINT(42.00226252439931 -87.57143445622613) bank144007 +144008 POINT(42.78887443705047 -88.17009114256626) bank144008 +144009 POINT(42.335116115071756 -87.88461999993964) bank144009 +144010 POINT(41.56072950467746 -87.07088991074819) bank144010 +144011 POINT(42.028125311442885 -88.55611478211452) bank144011 +144012 POINT(40.92726961006908 -86.63735086874883) bank144012 +144013 POINT(41.19450274202899 -87.04917343297606) bank144013 +144014 POINT(41.914223328848244 -87.84335737046331) bank144014 +144015 POINT(41.04802668684144 -87.29013362695393) bank144015 +144016 POINT(41.99413057474621 -87.76896450049053) bank144016 +144017 POINT(41.67720710981376 -86.78008865408682) bank144017 +144018 POINT(41.7781312733382 -87.33675120257352) bank144018 +144019 POINT(42.33071746608721 -87.17368623275951) bank144019 +144020 POINT(41.77559104195073 -88.33138452309946) bank144020 +144021 POINT(41.282627833143984 -87.56830466670444) bank144021 +144022 POINT(41.05318351371695 -87.07701680164475) bank144022 +144023 POINT(42.79528504116985 -88.19692888877977) bank144023 +144024 POINT(41.23254031919453 -86.73235191116098) bank144024 +144025 POINT(41.895352486973295 -87.5870428291063) bank144025 +144026 POINT(41.87249294459765 -87.55449360035675) bank144026 +144027 POINT(42.2101531186659 -88.12869065070717) bank144027 +144028 POINT(42.728430069708395 -88.27561632673044) bank144028 +144029 POINT(42.432625215451985 -88.45173342616648) bank144029 +144030 POINT(41.81803425851013 -87.27671721212582) bank144030 +144031 POINT(42.35057130657029 -86.72312019819788) bank144031 +144032 POINT(41.575410013254405 -88.3631918368073) bank144032 +144033 POINT(41.54054336182183 -88.26323385258527) bank144033 +144034 POINT(42.75353373541686 -88.28458580146928) bank144034 +144035 POINT(41.3790777165417 -87.37772519943374) bank144035 +144036 POINT(41.51251732357997 -87.3961591037138) bank144036 +144037 POINT(41.69221104788895 -87.49506231327378) bank144037 +144038 POINT(41.57971852913514 -87.63182227926475) bank144038 +144039 POINT(41.64814621155167 -86.97418919800168) bank144039 +144040 POINT(42.173939942499786 -87.71076173105243) bank144040 +144041 POINT(41.18623313734488 -87.92986358750393) bank144041 +144042 POINT(42.64916137793187 -87.55852067111742) bank144042 +144043 POINT(42.03348651067008 -87.7820825003541) bank144043 +144044 POINT(41.989853579448976 -88.4888248342286) bank144044 +144045 POINT(41.28650826030333 -86.93185271826604) bank144045 +144046 POINT(41.03774474805263 -87.55486264119715) bank144046 +144047 POINT(41.547958211310146 -86.78983876696007) bank144047 +144048 POINT(40.95545313894102 -87.23563144359025) bank144048 +144049 POINT(41.91888458151451 -87.91652271134842) bank144049 +144050 POINT(41.60675944582509 -87.5652491319307) bank144050 +144051 POINT(41.56668437221804 -87.95522564619411) bank144051 +144052 POINT(42.85946323889343 -88.17677774862953) bank144052 +144053 POINT(42.55784153537482 -87.3692638906634) bank144053 +144054 POINT(41.36383694245441 -87.22233042413728) bank144054 +144055 POINT(41.944607662297756 -88.08577755138026) bank144055 +144056 POINT(41.91168915025677 -87.25857426476526) bank144056 +144057 POINT(42.08466167095493 -88.00582790985919) bank144057 +144058 POINT(42.26772169069879 -87.01524689569403) bank144058 +144059 POINT(42.86741440395188 -88.60517139480156) bank144059 +144060 POINT(41.29953263969187 -88.39822871618419) bank144060 +144061 POINT(42.20351090017854 -87.32088049105757) bank144061 +144062 POINT(41.13359703815185 -88.56523498848904) bank144062 +144063 POINT(42.10551017575209 -86.99047184089876) bank144063 +144064 POINT(42.132451867214456 -86.8544420474764) bank144064 +144065 POINT(42.61164075177601 -88.3517490189647) bank144065 +144066 POINT(41.14487540116871 -88.0679757896551) bank144066 +144067 POINT(41.155179397101215 -86.64543796884476) bank144067 +144068 POINT(41.35495448538213 -88.41103023065287) bank144068 +144069 POINT(41.03895972871873 -86.63975203067888) bank144069 +144070 POINT(41.25818484816086 -87.7900876533958) bank144070 +144071 POINT(41.1262183006642 -87.77205411609044) bank144071 +144072 POINT(42.65635396794818 -88.16930302532005) bank144072 +144073 POINT(41.01120630030192 -87.97030297812893) bank144073 +144074 POINT(42.876436571586304 -88.40733503320418) bank144074 +144075 POINT(42.73349623495729 -87.51234990608135) bank144075 +144076 POINT(41.2955380156662 -87.4203846868019) bank144076 +144077 POINT(42.13825467633053 -86.7241954941979) bank144077 +144078 POINT(41.16220129863925 -87.7784191289414) bank144078 +144079 POINT(41.93947043879558 -87.58691557562054) bank144079 +144080 POINT(41.87227049694508 -87.23347348116613) bank144080 +144081 POINT(42.636703915324674 -88.11907356186227) bank144081 +144082 POINT(41.1243028757413 -88.09417571397597) bank144082 +144083 POINT(42.424160389618486 -87.49079917835576) bank144083 +144084 POINT(41.78483734565746 -87.70702504496657) bank144084 +144085 POINT(41.679159068786966 -87.74764905229179) bank144085 +144086 POINT(42.32403577726441 -88.45443883378641) bank144086 +144087 POINT(41.678776827769134 -86.70813797864147) bank144087 +144088 POINT(41.907874609173696 -87.1118338619032) bank144088 +144089 POINT(41.75493005653467 -87.32435205349718) bank144089 +144090 POINT(41.20597741142842 -87.71711756456422) bank144090 +144091 POINT(42.744989363089694 -87.8249000465187) bank144091 +144092 POINT(41.63424952503138 -88.55133060582948) bank144092 +144093 POINT(42.19223277105502 -86.8732448396822) bank144093 +144094 POINT(41.49719812216078 -87.3142177433867) bank144094 +144095 POINT(41.15938676074739 -88.59650649778418) bank144095 +144096 POINT(42.40585733355574 -86.93781464475188) bank144096 +144097 POINT(42.135079798065576 -87.8851587265576) bank144097 +144098 POINT(42.78869941013768 -88.10372236839703) bank144098 +144099 POINT(42.246094301096115 -88.3554711528347) bank144099 +144100 POINT(41.024193485154434 -88.47366408272147) bank144100 +144101 POINT(41.035858575865944 -86.78940011813935) bank144101 +144102 POINT(41.44695418253105 -87.57392862025355) bank144102 +144103 POINT(41.609659270769086 -87.8267413204751) bank144103 +144104 POINT(41.576993962633125 -88.46643692802172) bank144104 +144105 POINT(42.47699252469569 -87.50905550290601) bank144105 +144106 POINT(42.36188114141834 -87.43604152635902) bank144106 +144107 POINT(41.336039773536406 -86.7745359067131) bank144107 +144108 POINT(41.927508315501775 -86.70339737630896) bank144108 +144109 POINT(40.93510004513339 -87.13679021292877) bank144109 +144110 POINT(42.19915939852234 -87.31425309205855) bank144110 +144111 POINT(41.60513069776315 -87.60239209583015) bank144111 +144112 POINT(42.81625998263007 -87.53510971400846) bank144112 +144113 POINT(42.49280005268313 -88.40307618761875) bank144113 +144114 POINT(41.23703216219218 -87.95468378053657) bank144114 +144115 POINT(40.97215358172892 -87.23904082062946) bank144115 +144116 POINT(41.384435205943184 -88.26691450115536) bank144116 +144117 POINT(41.2518811147042 -87.8005975331268) bank144117 +144118 POINT(41.86066823093647 -88.28554589619255) bank144118 +144119 POINT(42.19434310937712 -88.53268188799814) bank144119 +144120 POINT(41.10937208602466 -87.04249227285368) bank144120 +144121 POINT(41.0057913427821 -87.45105379765077) bank144121 +144122 POINT(42.283909962007016 -87.92777413385633) bank144122 +144123 POINT(41.69654672651007 -87.26189518702756) bank144123 +144124 POINT(41.16127317057435 -87.74707207988185) bank144124 +144125 POINT(41.74273342398183 -87.8120225383884) bank144125 +144126 POINT(42.625874061865176 -87.20248530812464) bank144126 +144127 POINT(41.5514531211022 -86.71878992805615) bank144127 +144128 POINT(41.136553886241686 -87.71650828207568) bank144128 +144129 POINT(41.59031413361841 -87.20564665517082) bank144129 +144130 POINT(41.728664015871125 -87.13796456511488) bank144130 +144131 POINT(41.66949027134604 -86.75800495591908) bank144131 +144132 POINT(42.6443039704324 -88.35761616581702) bank144132 +144133 POINT(41.95661728138632 -86.84735377327664) bank144133 +144134 POINT(42.286492179503036 -87.96948253104333) bank144134 +144135 POINT(42.47524146237478 -86.88364841710671) bank144135 +144136 POINT(42.728612734899585 -87.0984642503946) bank144136 +144137 POINT(42.360444671743984 -88.42290155455734) bank144137 +144138 POINT(41.6548871712041 -87.4518461216533) bank144138 +144139 POINT(42.15094320174508 -88.55812895743605) bank144139 +144140 POINT(41.83828950203702 -87.3142285619421) bank144140 +144141 POINT(42.81243684189852 -87.29216151151755) bank144141 +144142 POINT(42.13714756205843 -87.47319678317517) bank144142 +144143 POINT(42.82149710669718 -87.84158458928253) bank144143 +144144 POINT(42.08501254536252 -86.66680355316323) bank144144 +144145 POINT(41.58392149045088 -87.05531288244703) bank144145 +144146 POINT(41.51017223909532 -86.72629718568622) bank144146 +144147 POINT(42.84872175598666 -88.5899272379577) bank144147 +144148 POINT(41.70714862958013 -87.51564513087013) bank144148 +144149 POINT(41.230516319266755 -87.85953092150996) bank144149 +144150 POINT(41.963651006679896 -87.55507051626493) bank144150 +144151 POINT(42.171638497946006 -88.60632857590073) bank144151 +144152 POINT(42.19606938556533 -88.42619050775338) bank144152 +144153 POINT(42.3963623553194 -87.69059246193106) bank144153 +144154 POINT(41.21931870371077 -88.54841665157582) bank144154 +144155 POINT(42.02977470442389 -88.39027363677891) bank144155 +144156 POINT(42.55090199547366 -87.36791865147704) bank144156 +144157 POINT(42.86201921401008 -87.91412615257221) bank144157 +144158 POINT(41.47914496207947 -88.55672324110641) bank144158 +144159 POINT(41.272508308583184 -87.17068049202484) bank144159 +144160 POINT(42.558763213938825 -87.13252422084027) bank144160 +144161 POINT(41.1437819584824 -86.87655037676899) bank144161 +144162 POINT(41.73179539935403 -88.53607204183135) bank144162 +144163 POINT(40.90715590014659 -88.10794150979906) bank144163 +144164 POINT(42.13297861145209 -86.99381323316962) bank144164 +144165 POINT(40.98820540513577 -87.08756703864427) bank144165 +144166 POINT(41.90852660204438 -87.72107607548408) bank144166 +144167 POINT(42.3917129376411 -87.89772993999236) bank144167 +144168 POINT(41.2344040890137 -87.57262082034904) bank144168 +144169 POINT(41.13998833144536 -87.63895865010716) bank144169 +144170 POINT(42.851254900590796 -87.0544452840851) bank144170 +144171 POINT(42.75035499839343 -88.03121991142697) bank144171 +144172 POINT(42.28415727699121 -87.5348434131626) bank144172 +144173 POINT(41.621980474868444 -87.0241413576175) bank144173 +144174 POINT(42.24646104747229 -86.71391685624909) bank144174 +144175 POINT(41.97213881932966 -88.06528456035754) bank144175 +144176 POINT(42.592890426006214 -88.47611635778352) bank144176 +144177 POINT(42.56700404066518 -87.04260470311489) bank144177 +144178 POINT(41.68602196592239 -86.69900933109646) bank144178 +144179 POINT(41.84853425261642 -87.45685711493311) bank144179 +144180 POINT(41.31826991327096 -87.61218246347629) bank144180 +144181 POINT(41.938710147125654 -86.7615515559881) bank144181 +144182 POINT(41.015574460714596 -87.28780886689155) bank144182 +144183 POINT(42.0992288176642 -88.29879747673382) bank144183 +144184 POINT(42.698825105611505 -87.15641503557214) bank144184 +144185 POINT(41.29434326537691 -87.7478688944753) bank144185 +144186 POINT(41.547905415525534 -87.91585909634166) bank144186 +144187 POINT(40.95223153760112 -86.94044001875844) bank144187 +144188 POINT(42.83804632766737 -87.43889893055255) bank144188 +144189 POINT(42.653091440295995 -87.0661040103858) bank144189 +144190 POINT(42.03138493498453 -87.03188272784907) bank144190 +144191 POINT(41.39298882180532 -87.70192139192537) bank144191 +144192 POINT(42.5135703943796 -87.53239113941622) bank144192 +144193 POINT(42.868212661875184 -88.39592983072077) bank144193 +144194 POINT(42.49380381998997 -87.88417171641828) bank144194 +144195 POINT(41.92674472870339 -87.97117541665327) bank144195 +144196 POINT(41.600547408339295 -87.27662195121219) bank144196 +144197 POINT(41.52922814365588 -87.50297579786077) bank144197 +144198 POINT(42.71984639484299 -87.9985494166144) bank144198 +144199 POINT(42.48156094860545 -87.98694956636287) bank144199 +144200 POINT(41.797662064768744 -88.14043485268859) bank144200 +144201 POINT(42.64414197808127 -87.9905654973237) bank144201 +144202 POINT(41.921790436226196 -87.54202462716141) bank144202 +144203 POINT(42.779687572827946 -87.96371069669104) bank144203 +144204 POINT(42.82565366590085 -87.97371295660287) bank144204 +144205 POINT(41.12323669920984 -88.24843227520051) bank144205 +144206 POINT(41.98525775515069 -87.58947287306471) bank144206 +144207 POINT(41.46623707341014 -87.13061143347052) bank144207 +144208 POINT(41.44139824682933 -87.01940588037114) bank144208 +144209 POINT(42.33644542480266 -87.29718507195481) bank144209 +144210 POINT(41.6600611242241 -87.29346788923267) bank144210 +144211 POINT(40.968983998014835 -87.29431644344784) bank144211 +144212 POINT(41.49990529823292 -87.16584914703196) bank144212 +144213 POINT(42.263196365106246 -87.17439127077185) bank144213 +144214 POINT(42.28081397551121 -88.04472983251834) bank144214 +144215 POINT(42.64340757708195 -87.69963080086258) bank144215 +144216 POINT(41.18821908141638 -87.97629168759174) bank144216 +144217 POINT(41.98161717823085 -87.90684655701767) bank144217 +144218 POINT(41.30716832587339 -86.77839423462837) bank144218 +144219 POINT(42.49770584872195 -88.4211149797465) bank144219 +144220 POINT(42.69206068995958 -87.88359374693773) bank144220 +144221 POINT(42.061918641752925 -87.440800875585) bank144221 +144222 POINT(42.863193474360855 -86.65937238967574) bank144222 +144223 POINT(41.61437075014772 -88.11868313629031) bank144223 +144224 POINT(42.11980616253012 -88.06582744688973) bank144224 +144225 POINT(42.05448483520133 -87.09530751688092) bank144225 +144226 POINT(42.39110433279597 -87.67483726835117) bank144226 +144227 POINT(42.61654828864792 -87.14068851621073) bank144227 +144228 POINT(41.22538660034362 -86.95288412593747) bank144228 +144229 POINT(42.35677216111345 -87.36836463466739) bank144229 +144230 POINT(41.22029939387229 -86.87271152291439) bank144230 +144231 POINT(42.35536828058133 -87.21735604827111) bank144231 +144232 POINT(41.22605292700729 -87.48336493887875) bank144232 +144233 POINT(41.99093304356699 -87.85481849252076) bank144233 +144234 POINT(41.948919578893765 -88.53321394713602) bank144234 +144235 POINT(41.34916773977863 -87.2869275949716) bank144235 +144236 POINT(40.97060165150255 -86.71544128049845) bank144236 +144237 POINT(41.944027440509615 -88.26821070017839) bank144237 +144238 POINT(41.397259748972274 -86.65180249935568) bank144238 +144239 POINT(42.01823013305143 -87.8619730278367) bank144239 +144240 POINT(42.08608311283103 -87.15194861317762) bank144240 +144241 POINT(42.14144715215376 -86.64191601470559) bank144241 +144242 POINT(41.155812488037085 -86.84630758030347) bank144242 +144243 POINT(42.12099496652319 -88.20091297981949) bank144243 +144244 POINT(42.557566175216415 -87.40221332347086) bank144244 +144245 POINT(42.322341471629095 -87.03109851289734) bank144245 +144246 POINT(41.1359735098647 -88.59461463524853) bank144246 +144247 POINT(42.64004403512873 -87.3355382070919) bank144247 +144248 POINT(41.41983418745951 -87.62484627128254) bank144248 +144249 POINT(42.63305313638191 -87.26905603124071) bank144249 +144250 POINT(41.40083493922833 -88.241470985447) bank144250 +144251 POINT(41.65869082775066 -87.15559498508264) bank144251 +144252 POINT(42.008558171031275 -87.97110041299604) bank144252 +144253 POINT(41.96805918222854 -86.6889911907731) bank144253 +144254 POINT(42.661036310486374 -88.62534855877722) bank144254 +144255 POINT(42.0966363291637 -88.10529834459469) bank144255 +144256 POINT(41.26728324791838 -87.96050612471484) bank144256 +144257 POINT(41.681023225448705 -87.66033839048569) bank144257 +144258 POINT(42.09732091131527 -88.14147882924631) bank144258 +144259 POINT(42.373995868363046 -87.0740261830691) bank144259 +144260 POINT(42.4752949928647 -87.40241589696727) bank144260 +144261 POINT(41.936339586755345 -88.46173557311361) bank144261 +144262 POINT(41.88866179609661 -88.57951185478855) bank144262 +144263 POINT(41.45498614300286 -87.2104188504534) bank144263 +144264 POINT(42.3539878849147 -87.04152568244291) bank144264 +144265 POINT(41.173972952762185 -88.08578438233938) bank144265 +144266 POINT(41.94567905919641 -87.47501765610181) bank144266 +144267 POINT(40.962315123653866 -87.95850577617355) bank144267 +144268 POINT(42.78741451871887 -88.53561835371137) bank144268 +144269 POINT(41.120251615625016 -86.73597267944497) bank144269 +144270 POINT(41.17250624404015 -88.25347890959321) bank144270 +144271 POINT(41.224871696416244 -86.71031663098515) bank144271 +144272 POINT(42.81901680624342 -87.07724414875962) bank144272 +144273 POINT(42.778597884520636 -88.14292139363052) bank144273 +144274 POINT(42.720062098820605 -87.45410950015588) bank144274 +144275 POINT(42.170938107122396 -87.00250993906818) bank144275 +144276 POINT(41.249275687322985 -87.03401854835683) bank144276 +144277 POINT(41.15629580882576 -87.97354843367057) bank144277 +144278 POINT(41.98316188572952 -88.16825499707885) bank144278 +144279 POINT(41.467317174006595 -88.56705712554691) bank144279 +144280 POINT(40.9139539768674 -88.08108402215855) bank144280 +144281 POINT(41.02070463851446 -88.22963783752539) bank144281 +144282 POINT(42.26090164405682 -87.71956706240718) bank144282 +144283 POINT(41.94116012726015 -87.03918857494055) bank144283 +144284 POINT(42.092687964385824 -86.8262259641419) bank144284 +144285 POINT(41.304662031241435 -88.53719394451186) bank144285 +144286 POINT(42.66612924413205 -88.37945198228378) bank144286 +144287 POINT(42.79097408940041 -87.19817396047888) bank144287 +144288 POINT(41.428381670941775 -88.3830216972215) bank144288 +144289 POINT(41.38695453919629 -88.08599724161465) bank144289 +144290 POINT(41.223652961589075 -87.05029123459626) bank144290 +144291 POINT(41.64712015980339 -86.86180112515926) bank144291 +144292 POINT(42.77167743586891 -88.45771096177988) bank144292 +144293 POINT(42.54502851184327 -88.13345908082627) bank144293 +144294 POINT(40.941008154916375 -87.21411654098051) bank144294 +144295 POINT(42.492060206572205 -87.50515425507277) bank144295 +144296 POINT(42.6842203845039 -87.89650709327869) bank144296 +144297 POINT(41.094795881106215 -87.76269509195814) bank144297 +144298 POINT(41.68957797372013 -87.20853310689606) bank144298 +144299 POINT(41.42907323777632 -87.37138504006893) bank144299 +144300 POINT(40.992922236972724 -88.23657955053504) bank144300 +144301 POINT(41.67950703836547 -86.95062487692178) bank144301 +144302 POINT(41.48747914841669 -87.20714992794133) bank144302 +144303 POINT(42.86692322096987 -87.94609156088136) bank144303 +144304 POINT(41.33143650496499 -87.2751836238854) bank144304 +144305 POINT(41.229750509810906 -86.87240293301848) bank144305 +144306 POINT(40.974244973002016 -87.4396045536579) bank144306 +144307 POINT(41.46558300023892 -87.55838052481523) bank144307 +144308 POINT(42.70509464272801 -87.3921110051053) bank144308 +144309 POINT(41.16003745686787 -88.11596746108086) bank144309 +144310 POINT(42.64518979068638 -88.42776220165261) bank144310 +144311 POINT(41.50248411585389 -88.03298396824286) bank144311 +144312 POINT(42.55740723395281 -88.30581434037879) bank144312 +144313 POINT(42.038454375434604 -88.28670950882797) bank144313 +144314 POINT(41.87089119544389 -87.28549229858437) bank144314 +144315 POINT(42.442654344594814 -87.384336899236) bank144315 +144316 POINT(41.12386168395881 -87.36486234025685) bank144316 +144317 POINT(42.49027683823029 -86.87859337576975) bank144317 +144318 POINT(42.244286535894005 -88.36629534303263) bank144318 +144319 POINT(42.54567077013113 -87.01747057805413) bank144319 +144320 POINT(42.4063241700441 -88.17971335495106) bank144320 +144321 POINT(42.24367751531307 -88.42908670445327) bank144321 +144322 POINT(42.72106982241618 -87.47454113428655) bank144322 +144323 POINT(42.76230267663456 -88.43934684265494) bank144323 +144324 POINT(41.79825566227086 -86.73034650896007) bank144324 +144325 POINT(41.59932138707207 -87.32958766521051) bank144325 +144326 POINT(41.756314448334834 -88.57576144240495) bank144326 +144327 POINT(41.9783743493945 -87.65368643720618) bank144327 +144328 POINT(42.339603552752315 -87.27373999395739) bank144328 +144329 POINT(41.04673350702794 -87.1979760010059) bank144329 +144330 POINT(41.91805208462707 -87.66723106154653) bank144330 +144331 POINT(42.09557586043492 -87.89894974622185) bank144331 +144332 POINT(40.8790437222724 -88.31387544419664) bank144332 +144333 POINT(41.10031086065055 -88.04741924040853) bank144333 +144334 POINT(41.18457282198667 -87.21617990339155) bank144334 +144335 POINT(42.639362745976854 -87.61049691625053) bank144335 +144336 POINT(41.93230070538581 -87.75513565975143) bank144336 +144337 POINT(41.54345970315474 -88.36339423752413) bank144337 +144338 POINT(41.1961299213175 -88.42402483589764) bank144338 +144339 POINT(41.72921611308858 -87.69013333174651) bank144339 +144340 POINT(42.39423402693183 -86.96105720295971) bank144340 +144341 POINT(41.1506430391724 -86.80047742501367) bank144341 +144342 POINT(41.24044947678792 -87.99665372922748) bank144342 +144343 POINT(41.65690419464901 -87.02016181465979) bank144343 +144344 POINT(40.95823623561602 -87.47937531223123) bank144344 +144345 POINT(42.44889595945311 -87.24162325807907) bank144345 +144346 POINT(42.0010721386362 -86.93166203109548) bank144346 +144347 POINT(42.5866525484276 -88.54617335871723) bank144347 +144348 POINT(42.25356949228261 -87.21278503710059) bank144348 +144349 POINT(41.00502637179605 -86.90234959574491) bank144349 +144350 POINT(41.52314628074013 -87.18205415350388) bank144350 +144351 POINT(42.2970206162653 -88.25483678545619) bank144351 +144352 POINT(41.176988618257596 -87.67522308399613) bank144352 +144353 POINT(42.05616242185574 -86.83588008298071) bank144353 +144354 POINT(42.061407624587154 -88.30543594112272) bank144354 +144355 POINT(42.32960539720596 -87.69575187394503) bank144355 +144356 POINT(41.64800703827118 -88.2074309643936) bank144356 +144357 POINT(41.98808294001988 -86.64341938636836) bank144357 +144358 POINT(41.176496629310606 -88.60520490827496) bank144358 +144359 POINT(42.079192134764185 -88.58779748189636) bank144359 +144360 POINT(41.279022594182 -87.50069599510262) bank144360 +144361 POINT(42.41445112735801 -87.86546001789868) bank144361 +144362 POINT(42.55431169742473 -86.68942452738806) bank144362 +144363 POINT(42.75764120511493 -88.58131659637473) bank144363 +144364 POINT(42.75841809156601 -87.97030440305154) bank144364 +144365 POINT(41.037069490109 -88.62966982407995) bank144365 +144366 POINT(41.751068693909396 -87.70935410750064) bank144366 +144367 POINT(41.778335009368526 -86.95613552879388) bank144367 +144368 POINT(41.272885536767916 -87.33246655817024) bank144368 +144369 POINT(41.67933985513765 -88.33857779046305) bank144369 +144370 POINT(42.01584610676348 -86.95442465456925) bank144370 +144371 POINT(41.02600422265177 -87.91719484141983) bank144371 +144372 POINT(42.28713169312833 -86.66661681006453) bank144372 +144373 POINT(42.17185252905346 -88.0565494267815) bank144373 +144374 POINT(41.01580109192293 -86.96299016119889) bank144374 +144375 POINT(42.64180232419336 -87.48101305846556) bank144375 +144376 POINT(42.574749523632484 -88.13860382557107) bank144376 +144377 POINT(42.289173655454526 -87.56474326712231) bank144377 +144378 POINT(41.61873936326999 -88.264916999973) bank144378 +144379 POINT(42.01744422461191 -87.98302605220582) bank144379 +144380 POINT(41.781663428437554 -87.40796928464094) bank144380 +144381 POINT(41.25478780832443 -88.46222272259628) bank144381 +144382 POINT(41.65688120894289 -87.69882871462464) bank144382 +144383 POINT(41.276161755723884 -87.43544860136129) bank144383 +144384 POINT(41.29656360477077 -87.79670160309537) bank144384 +144385 POINT(42.617392528895245 -87.15354351515128) bank144385 +144386 POINT(41.59137769787878 -86.92464636132001) bank144386 +144387 POINT(41.15192427201029 -88.03151220832476) bank144387 +144388 POINT(42.303600421009335 -87.55194035625348) bank144388 +144389 POINT(42.56658154664182 -87.32984745691077) bank144389 +144390 POINT(41.154837534234034 -88.43221198505614) bank144390 +144391 POINT(42.184219846935896 -86.88464874581487) bank144391 +144392 POINT(41.41867510477332 -87.52267816875222) bank144392 +144393 POINT(42.459514092123825 -88.51900968661863) bank144393 +144394 POINT(42.30284328775646 -87.82181315187933) bank144394 +144395 POINT(40.99252459799984 -87.73808467361171) bank144395 +144396 POINT(42.107093981171445 -86.74346210165925) bank144396 +144397 POINT(42.49551422013644 -88.20341609802529) bank144397 +144398 POINT(41.959154763731156 -87.05863862614888) bank144398 +144399 POINT(40.94940712868229 -88.40776224833026) bank144399 +144400 POINT(42.306085091383395 -88.15924819485419) bank144400 +144401 POINT(42.61808964899865 -87.28097867959985) bank144401 +144402 POINT(41.525531150651716 -87.78296879917563) bank144402 +144403 POINT(41.99027697888765 -86.84086422614959) bank144403 +144404 POINT(41.44887487130905 -87.28170732709542) bank144404 +144405 POINT(42.42742214696874 -87.16226029311511) bank144405 +144406 POINT(42.82851071780792 -88.45081120057088) bank144406 +144407 POINT(42.734935163830244 -87.87625475774438) bank144407 +144408 POINT(42.073549431090406 -88.325276279707) bank144408 +144409 POINT(41.80292066817309 -86.89498016898038) bank144409 +144410 POINT(42.53831758528847 -87.73340020333644) bank144410 +144411 POINT(41.598911979551076 -88.50823313288139) bank144411 +144412 POINT(42.53351406103544 -86.67704426174382) bank144412 +144413 POINT(42.267866806012755 -86.72337875622956) bank144413 +144414 POINT(41.50144941932013 -87.43522586893589) bank144414 +144415 POINT(42.58748138249258 -87.09960835430344) bank144415 +144416 POINT(42.771339433895236 -88.04238814286097) bank144416 +144417 POINT(41.01907237078562 -88.22827157505334) bank144417 +144418 POINT(41.25472367696016 -87.19157721518795) bank144418 +144419 POINT(42.491896590746386 -87.55644727098706) bank144419 +144420 POINT(42.01847378552467 -87.13371543795844) bank144420 +144421 POINT(41.29811422918624 -87.24328080230526) bank144421 +144422 POINT(42.39790438287035 -88.41200842185079) bank144422 +144423 POINT(41.85127798028899 -87.5029907313026) bank144423 +144424 POINT(41.37775319618622 -87.69635078275904) bank144424 +144425 POINT(42.01470034806417 -87.93380934137308) bank144425 +144426 POINT(41.9261091581716 -88.0973985431109) bank144426 +144427 POINT(42.6944698607644 -87.63309766559698) bank144427 +144428 POINT(41.75479127418668 -87.75362540417746) bank144428 +144429 POINT(42.77579986603474 -87.17628324022205) bank144429 +144430 POINT(42.423632565768656 -86.71187841258771) bank144430 +144431 POINT(41.6046212675551 -87.07116091440726) bank144431 +144432 POINT(41.98885898874586 -87.94267084865731) bank144432 +144433 POINT(42.06421150722623 -88.5883105452898) bank144433 +144434 POINT(41.1102017035724 -87.29990167493101) bank144434 +144435 POINT(41.521359661042034 -86.89869106602977) bank144435 +144436 POINT(42.54065565419139 -87.051436710273) bank144436 +144437 POINT(42.53924881888005 -87.82894358299652) bank144437 +144438 POINT(42.66518106114969 -87.42767434155698) bank144438 +144439 POINT(42.47494503480826 -88.41255126307304) bank144439 +144440 POINT(40.939251164999156 -88.02347318631162) bank144440 +144441 POINT(42.4884906987394 -86.73927067913064) bank144441 +144442 POINT(42.483761696288276 -86.82471797338772) bank144442 +144443 POINT(42.375328141619754 -87.26459527509988) bank144443 +144444 POINT(41.63962158291167 -87.19673944379197) bank144444 +144445 POINT(41.8262516604216 -88.5352972475642) bank144445 +144446 POINT(41.572359528748706 -86.79135640511547) bank144446 +144447 POINT(42.766434449569864 -87.83547524052544) bank144447 +144448 POINT(42.04785298448449 -87.94893077471646) bank144448 +144449 POINT(42.711236182691806 -87.66830196818212) bank144449 +144450 POINT(42.08579130039073 -88.01356960039) bank144450 +144451 POINT(41.97121878211101 -88.30940421984936) bank144451 +144452 POINT(41.6571448436023 -87.65675975973936) bank144452 +144453 POINT(41.13909581147352 -88.5420291204241) bank144453 +144454 POINT(42.40133825657615 -88.30914701423627) bank144454 +144455 POINT(42.42118932745259 -87.26040715858866) bank144455 +144456 POINT(41.72392967303572 -88.54607397949552) bank144456 +144457 POINT(41.236582784654 -86.6964530787133) bank144457 +144458 POINT(41.26768940383838 -86.74693875755376) bank144458 +144459 POINT(41.83702512700753 -87.90509991256309) bank144459 +144460 POINT(42.08973783616367 -87.77881375210838) bank144460 +144461 POINT(41.86069894187112 -88.57368519446179) bank144461 +144462 POINT(42.74809064522114 -86.84623586990249) bank144462 +144463 POINT(41.18682110317016 -86.89952085103258) bank144463 +144464 POINT(42.823461305259606 -87.34395176153102) bank144464 +144465 POINT(41.41031565482029 -86.98743324495697) bank144465 +144466 POINT(42.08046221536921 -88.24500905824004) bank144466 +144467 POINT(42.69479784765905 -87.35269168249881) bank144467 +144468 POINT(41.02283310477397 -88.17575059636576) bank144468 +144469 POINT(42.51523261570178 -87.51594242003183) bank144469 +144470 POINT(41.786250778313914 -86.68428700140282) bank144470 +144471 POINT(42.580881030143864 -87.24440659969474) bank144471 +144472 POINT(40.945606874382875 -87.21619930344143) bank144472 +144473 POINT(41.563192742984356 -88.13978327080676) bank144473 +144474 POINT(41.01424773708415 -88.22824877945953) bank144474 +144475 POINT(42.624371594893 -88.2219545475656) bank144475 +144476 POINT(42.15878416498293 -87.52518071933909) bank144476 +144477 POINT(41.82287294033474 -87.5567008756052) bank144477 +144478 POINT(42.62052047219765 -88.50640013937564) bank144478 +144479 POINT(42.85028284823382 -88.19432355660167) bank144479 +144480 POINT(41.79300998936664 -88.06469762262314) bank144480 +144481 POINT(41.569404319933604 -87.15381310966626) bank144481 +144482 POINT(41.71769607028651 -87.86947519642895) bank144482 +144483 POINT(42.70514836321377 -88.49226494105183) bank144483 +144484 POINT(40.9449083524614 -88.42975600190248) bank144484 +144485 POINT(41.418394908403464 -88.35310216825233) bank144485 +144486 POINT(41.52558639348245 -86.88436238969119) bank144486 +144487 POINT(42.588126710851135 -86.73410880138329) bank144487 +144488 POINT(41.62155509556457 -88.01193326675899) bank144488 +144489 POINT(41.824955586091065 -87.56343523555542) bank144489 +144490 POINT(41.13286978691431 -87.24080704824212) bank144490 +144491 POINT(41.668537204446906 -86.70453149597353) bank144491 +144492 POINT(41.14388972187602 -88.35274904461166) bank144492 +144493 POINT(42.320891398393975 -88.34461220395798) bank144493 +144494 POINT(41.91913563534781 -88.47373467208189) bank144494 +144495 POINT(41.06748403240837 -88.15666384671007) bank144495 +144496 POINT(41.815743678716714 -86.91150614612243) bank144496 +144497 POINT(41.82990161185297 -88.30914605598271) bank144497 +144498 POINT(41.597070842193794 -87.48050860631596) bank144498 +144499 POINT(41.21361695204883 -87.72911718639328) bank144499 +144500 POINT(41.711423144767934 -87.25223953569079) bank144500 +144501 POINT(42.48118914889135 -87.94230395354782) bank144501 +144502 POINT(41.23349318528784 -86.8923198110381) bank144502 +144503 POINT(42.05753269683557 -87.4403968640469) bank144503 +144504 POINT(42.3177297690337 -87.884526766401) bank144504 +144505 POINT(42.17090446956985 -87.96787132009217) bank144505 +144506 POINT(42.19900997012244 -87.92854929840108) bank144506 +144507 POINT(41.15730600692329 -87.38944425969859) bank144507 +144508 POINT(41.261543097798416 -86.96910697577442) bank144508 +144509 POINT(42.802293326685025 -87.50364933667268) bank144509 +144510 POINT(42.23871737872602 -88.0959070268154) bank144510 +144511 POINT(42.234039969850485 -88.05271314180244) bank144511 +144512 POINT(41.392469912765264 -88.04097288479923) bank144512 +144513 POINT(42.7627521841204 -88.08084249309914) bank144513 +144514 POINT(42.082877872351325 -86.76254522168782) bank144514 +144515 POINT(41.88298229038325 -87.41309638778826) bank144515 +144516 POINT(41.7092263569431 -87.25689088576806) bank144516 +144517 POINT(42.10614656134375 -88.20309808108186) bank144517 +144518 POINT(40.9278404752306 -87.62883951926314) bank144518 +144519 POINT(41.67359246187299 -88.23365455328917) bank144519 +144520 POINT(41.00831249816636 -86.86598239408568) bank144520 +144521 POINT(41.499840034908516 -87.66050479727882) bank144521 +144522 POINT(42.29355748192146 -86.73304210722378) bank144522 +144523 POINT(42.316282570655446 -86.91821827712951) bank144523 +144524 POINT(42.79415348114758 -86.94973541798467) bank144524 +144525 POINT(42.87681236617935 -86.88643424170344) bank144525 +144526 POINT(41.96104699892312 -87.44545844911079) bank144526 +144527 POINT(40.94166965647608 -87.7822881958923) bank144527 +144528 POINT(41.170623867461714 -87.55039857266524) bank144528 +144529 POINT(42.7637035424258 -86.69995166554288) bank144529 +144530 POINT(42.36880860189555 -86.82578960097086) bank144530 +144531 POINT(42.438661207958894 -87.75481223597171) bank144531 +144532 POINT(41.748025233964576 -86.7134716380267) bank144532 +144533 POINT(42.541911358195875 -88.45298527314529) bank144533 +144534 POINT(41.249480343988964 -86.78927748761839) bank144534 +144535 POINT(41.35700483370167 -88.18673764497524) bank144535 +144536 POINT(42.607932693645544 -86.80597690616582) bank144536 +144537 POINT(42.1659987036617 -87.84635566639419) bank144537 +144538 POINT(42.651834068469405 -88.11924950841474) bank144538 +144539 POINT(41.990843217582515 -87.02784571427492) bank144539 +144540 POINT(41.341621770124846 -87.7102015688356) bank144540 +144541 POINT(42.63738995056492 -86.65827928502789) bank144541 +144542 POINT(41.21452223040478 -87.68141666688614) bank144542 +144543 POINT(41.43501906930041 -87.773711886853) bank144543 +144544 POINT(41.492657375141995 -88.55388958385572) bank144544 +144545 POINT(41.58648804417244 -86.85541847793723) bank144545 +144546 POINT(41.98653983907491 -87.36304599164467) bank144546 +144547 POINT(42.66611510146518 -86.9782149757309) bank144547 +144548 POINT(41.13274128749033 -88.48928064193616) bank144548 +144549 POINT(40.87866097801741 -88.21147263548669) bank144549 +144550 POINT(41.898623687964545 -88.24891421186732) bank144550 +144551 POINT(41.49470603395403 -87.70914838889495) bank144551 +144552 POINT(42.835409390800244 -88.45791435974142) bank144552 +144553 POINT(41.38400477003405 -87.4023457382477) bank144553 +144554 POINT(41.17486971211057 -87.39310507345321) bank144554 +144555 POINT(42.578211354377174 -87.4233211762979) bank144555 +144556 POINT(42.2393520374421 -87.36410585858512) bank144556 +144557 POINT(42.84771473567576 -86.67387136723437) bank144557 +144558 POINT(41.073146049941045 -87.80520716705934) bank144558 +144559 POINT(42.305463322690436 -87.66418465894697) bank144559 +144560 POINT(42.27480736213339 -87.59619495352575) bank144560 +144561 POINT(41.6327652258135 -87.88640102834633) bank144561 +144562 POINT(41.132778463287345 -87.1604650751212) bank144562 +144563 POINT(41.63970178738104 -86.83166362211185) bank144563 +144564 POINT(41.522425079089686 -88.34677954352355) bank144564 +144565 POINT(42.62954433659198 -87.51298381667307) bank144565 +144566 POINT(42.79665806257468 -87.11003049027134) bank144566 +144567 POINT(41.79124429584597 -87.178922908073) bank144567 +144568 POINT(41.63272781329905 -88.42764907090839) bank144568 +144569 POINT(40.94404628226326 -87.59602136787362) bank144569 +144570 POINT(40.91485863174722 -87.92571450169834) bank144570 +144571 POINT(41.937380985697814 -88.14441194731079) bank144571 +144572 POINT(41.711644931704974 -87.40335512364187) bank144572 +144573 POINT(41.29962159987153 -88.40367949059176) bank144573 +144574 POINT(42.54189159381759 -87.52689698599909) bank144574 +144575 POINT(40.9565615089244 -87.01935822965972) bank144575 +144576 POINT(42.748747965431015 -87.12529047135793) bank144576 +144577 POINT(41.9695618378087 -88.2434498382514) bank144577 +144578 POINT(42.6295676004897 -88.09018699478719) bank144578 +144579 POINT(41.92082177779646 -87.81082437486087) bank144579 +144580 POINT(42.049246868407884 -87.22104203108579) bank144580 +144581 POINT(41.59175391650613 -88.31521411595993) bank144581 +144582 POINT(41.564367174158114 -87.87497296806667) bank144582 +144583 POINT(41.933537388703336 -88.52360790319976) bank144583 +144584 POINT(41.18043693402106 -87.27400884189386) bank144584 +144585 POINT(41.25738628040466 -88.339661666385) bank144585 +144586 POINT(41.00153226471761 -87.50720775295048) bank144586 +144587 POINT(40.8958726727741 -87.4787757201992) bank144587 +144588 POINT(42.10045163818194 -86.88700516297264) bank144588 +144589 POINT(42.63163128598215 -87.32576675473433) bank144589 +144590 POINT(41.65263288322197 -88.52499458556589) bank144590 +144591 POINT(41.53453538856012 -86.89010423274387) bank144591 +144592 POINT(41.21521328423571 -86.78124065118107) bank144592 +144593 POINT(41.35951685309898 -87.66722253087147) bank144593 +144594 POINT(41.26852646529646 -87.24880088561264) bank144594 +144595 POINT(41.06330384873479 -88.11786539810342) bank144595 +144596 POINT(40.95481931756995 -88.353801161787) bank144596 +144597 POINT(42.301706118617375 -87.41812058408826) bank144597 +144598 POINT(41.117567511835375 -87.88226588821128) bank144598 +144599 POINT(41.19019387265312 -86.80772954035375) bank144599 +144600 POINT(41.05276914720079 -88.13930384166407) bank144600 +144601 POINT(41.41758974295932 -87.07503007644681) bank144601 +144602 POINT(42.27166723841864 -87.95021142155787) bank144602 +144603 POINT(41.83030588486155 -87.91822498658209) bank144603 +144604 POINT(41.396149900880346 -87.82499825851126) bank144604 +144605 POINT(41.5931125966444 -87.68580297419172) bank144605 +144606 POINT(42.08279760999156 -87.15504158308728) bank144606 +144607 POINT(40.891032869635865 -87.2729040081317) bank144607 +144608 POINT(42.27227230629904 -87.33189658996473) bank144608 +144609 POINT(42.469572238658294 -88.39269260168352) bank144609 +144610 POINT(42.2015340702385 -87.70772236422853) bank144610 +144611 POINT(42.70804823233141 -88.59994232922176) bank144611 +144612 POINT(42.15785409740041 -87.18240528077993) bank144612 +144613 POINT(41.125721986489175 -88.4476002447458) bank144613 +144614 POINT(42.12678473193011 -87.3052338545191) bank144614 +144615 POINT(42.333645476896855 -88.62397457985455) bank144615 +144616 POINT(41.992177102813436 -87.08282968159206) bank144616 +144617 POINT(41.74169203151563 -87.56855876318029) bank144617 +144618 POINT(40.92654831963416 -87.64151767048392) bank144618 +144619 POINT(42.04560944824818 -88.08693008616494) bank144619 +144620 POINT(41.67665908522665 -87.6691221950889) bank144620 +144621 POINT(42.75219027377057 -88.23505337370693) bank144621 +144622 POINT(40.960486410732955 -87.7253293071919) bank144622 +144623 POINT(42.82131825534745 -87.2848750286135) bank144623 +144624 POINT(41.314794746302155 -88.53991038355419) bank144624 +144625 POINT(40.96830050346314 -86.71473462365998) bank144625 +144626 POINT(41.02625947413598 -87.3711777093157) bank144626 +144627 POINT(42.69418694523018 -87.2097282243719) bank144627 +144628 POINT(42.34252940777136 -88.27791060888576) bank144628 +144629 POINT(42.545046743861185 -86.7027148283139) bank144629 +144630 POINT(42.46228391556415 -87.62383823695028) bank144630 +144631 POINT(42.01260289640512 -87.95313000849578) bank144631 +144632 POINT(41.068387764408534 -88.00071174442567) bank144632 +144633 POINT(42.35990182285796 -88.50091513810182) bank144633 +144634 POINT(41.378383468966014 -87.12787210482442) bank144634 +144635 POINT(42.74789044539321 -86.88346819179186) bank144635 +144636 POINT(42.01761846629755 -88.00801789050425) bank144636 +144637 POINT(42.09313584156207 -87.77318581176932) bank144637 +144638 POINT(41.183155070687214 -88.57270731742899) bank144638 +144639 POINT(41.26392546486937 -87.91744260292397) bank144639 +144640 POINT(40.9077243557189 -88.08899402234874) bank144640 +144641 POINT(41.9973743916941 -87.53166176447) bank144641 +144642 POINT(42.501925450385194 -88.19223849066027) bank144642 +144643 POINT(42.78874084515465 -88.08808856930584) bank144643 +144644 POINT(41.19478002740752 -88.06797113861919) bank144644 +144645 POINT(41.58002117552364 -88.06015527849723) bank144645 +144646 POINT(42.795870683731415 -88.0050605987501) bank144646 +144647 POINT(41.34349891963031 -86.6994850836387) bank144647 +144648 POINT(42.530328260297495 -87.7445321492639) bank144648 +144649 POINT(41.559656095119166 -86.66126421892312) bank144649 +144650 POINT(41.81403018489661 -87.64061316159449) bank144650 +144651 POINT(41.70352552333471 -86.7303448145408) bank144651 +144652 POINT(41.4032682497308 -86.9713847527371) bank144652 +144653 POINT(42.18196619566918 -88.57265762507784) bank144653 +144654 POINT(42.441079255001604 -86.63669556797808) bank144654 +144655 POINT(42.588058423819824 -86.95054216077723) bank144655 +144656 POINT(41.87736707001657 -88.15945121202489) bank144656 +144657 POINT(41.72414481187335 -87.823367133154) bank144657 +144658 POINT(41.352543537618786 -88.16200213520968) bank144658 +144659 POINT(41.25689992747957 -88.09124468441968) bank144659 +144660 POINT(42.061528660095064 -87.83519796951035) bank144660 +144661 POINT(42.1982335769033 -86.64438367050073) bank144661 +144662 POINT(41.323280014957774 -88.42888877413293) bank144662 +144663 POINT(42.177846643561026 -86.88073959873287) bank144663 +144664 POINT(41.003859433077814 -87.82342552017654) bank144664 +144665 POINT(41.609366991967626 -87.9166099672883) bank144665 +144666 POINT(40.9613358649326 -87.83592869559926) bank144666 +144667 POINT(41.37724478920023 -86.92385315898375) bank144667 +144668 POINT(41.979572521059666 -88.28542711276268) bank144668 +144669 POINT(41.59670124339607 -88.55709879682192) bank144669 +144670 POINT(41.80517951118997 -86.9359421739773) bank144670 +144671 POINT(41.74807127343085 -87.20518702681682) bank144671 +144672 POINT(42.214462074526544 -87.52432914581203) bank144672 +144673 POINT(42.2803380334297 -87.07184690315088) bank144673 +144674 POINT(42.14357622985508 -87.8747505187273) bank144674 +144675 POINT(40.96714317109948 -86.82103200705967) bank144675 +144676 POINT(42.04512118169142 -87.55233089908454) bank144676 +144677 POINT(41.635345158967056 -87.43669701249415) bank144677 +144678 POINT(41.258321900922446 -87.50551851462708) bank144678 +144679 POINT(41.586621244085485 -86.79686736338189) bank144679 +144680 POINT(41.244318275915646 -86.79380858117366) bank144680 +144681 POINT(41.24144490538371 -88.12914977390382) bank144681 +144682 POINT(41.39267760590313 -87.64375843598138) bank144682 +144683 POINT(42.64956083490214 -87.12653778359677) bank144683 +144684 POINT(41.989845196177846 -88.04520194432787) bank144684 +144685 POINT(41.87869717246907 -88.37092596910414) bank144685 +144686 POINT(41.73718036312659 -87.26066205845434) bank144686 +144687 POINT(40.912679240434656 -88.30060311433566) bank144687 +144688 POINT(41.47142107744677 -86.84949205437782) bank144688 +144689 POINT(41.57476270039143 -88.31269650663549) bank144689 +144690 POINT(42.33602587143592 -87.79214658191694) bank144690 +144691 POINT(41.593331977134156 -88.38001770882347) bank144691 +144692 POINT(41.18344360410964 -87.46916554178505) bank144692 +144693 POINT(42.81382579338833 -86.75545890414543) bank144693 +144694 POINT(41.91919575755865 -88.54653397696774) bank144694 +144695 POINT(42.38258980171203 -86.72923541297303) bank144695 +144696 POINT(41.74690675415758 -88.56608924983868) bank144696 +144697 POINT(42.51769251505371 -87.79578295935491) bank144697 +144698 POINT(42.00213235366884 -88.59182576876707) bank144698 +144699 POINT(42.65484367283114 -87.63277426921694) bank144699 +144700 POINT(42.27499132944643 -88.2704840201945) bank144700 +144701 POINT(41.16621725079112 -88.56746802299233) bank144701 +144702 POINT(41.16263517292077 -88.46452498934096) bank144702 +144703 POINT(40.93307658636833 -87.82469164415342) bank144703 +144704 POINT(42.09796344300591 -88.39477217328836) bank144704 +144705 POINT(41.76951040094273 -87.4457995178767) bank144705 +144706 POINT(41.81558156250925 -87.91054640265905) bank144706 +144707 POINT(42.58156066666038 -86.7052178264759) bank144707 +144708 POINT(42.11081648664481 -87.86522651470644) bank144708 +144709 POINT(41.355820091174685 -86.90159714193103) bank144709 +144710 POINT(41.718336235464086 -87.94002173698928) bank144710 +144711 POINT(41.4471551533788 -87.8493398691715) bank144711 +144712 POINT(41.50434803249919 -87.63066700549285) bank144712 +144713 POINT(42.22592022186538 -86.99911183146354) bank144713 +144714 POINT(41.670450690241374 -87.69377701272006) bank144714 +144715 POINT(42.56146787962391 -86.9545927592824) bank144715 +144716 POINT(42.29923914002673 -87.3814828637936) bank144716 +144717 POINT(42.203708809654565 -87.88455486987695) bank144717 +144718 POINT(42.29255965944357 -87.75472406777386) bank144718 +144719 POINT(41.30169867887123 -88.50335850525532) bank144719 +144720 POINT(42.151006040425884 -87.06761077120122) bank144720 +144721 POINT(42.28881025618355 -87.96321914605049) bank144721 +144722 POINT(41.025404041654994 -87.37555332056942) bank144722 +144723 POINT(42.1931442270715 -87.11808203439506) bank144723 +144724 POINT(41.3541146076779 -87.58237381273477) bank144724 +144725 POINT(42.86910390515673 -86.89566143656972) bank144725 +144726 POINT(41.4858242502885 -87.15560390653691) bank144726 +144727 POINT(41.306484519688624 -86.63156834690987) bank144727 +144728 POINT(41.05846168285279 -88.54055900953915) bank144728 +144729 POINT(41.39203524905248 -87.13628486190467) bank144729 +144730 POINT(42.74769738088398 -87.4591704045517) bank144730 +144731 POINT(42.31056347159031 -87.37051143849095) bank144731 +144732 POINT(42.137021165491625 -88.18278601139245) bank144732 +144733 POINT(42.69000157803628 -87.79158054332662) bank144733 +144734 POINT(42.43390841283076 -86.83583985483894) bank144734 +144735 POINT(41.596552463859375 -87.08867950179209) bank144735 +144736 POINT(41.14140400679716 -87.34559208479023) bank144736 +144737 POINT(42.79769744769309 -86.73989296945584) bank144737 +144738 POINT(42.25684053416168 -88.6001350571364) bank144738 +144739 POINT(42.832283766443645 -87.06879478181425) bank144739 +144740 POINT(42.20005512018804 -87.76215900601242) bank144740 +144741 POINT(41.39746929388745 -87.7262730162219) bank144741 +144742 POINT(41.76511603333292 -87.1095670841203) bank144742 +144743 POINT(41.212782164298545 -86.72551689586103) bank144743 +144744 POINT(42.19342163512738 -87.93023210466032) bank144744 +144745 POINT(41.519388352485315 -87.23212474310857) bank144745 +144746 POINT(40.94556454446966 -87.9117405059816) bank144746 +144747 POINT(42.76241640817113 -87.0097727900319) bank144747 +144748 POINT(42.35667993539467 -87.72051199930553) bank144748 +144749 POINT(41.4399719850224 -87.01108233339437) bank144749 +144750 POINT(40.892848784476755 -87.62018399886817) bank144750 +144751 POINT(40.99070147090369 -88.58835093659341) bank144751 +144752 POINT(42.46570483269806 -88.42113813778606) bank144752 +144753 POINT(42.580877766455984 -88.07300998879161) bank144753 +144754 POINT(42.02103444726078 -86.75694521430596) bank144754 +144755 POINT(42.84872777109715 -87.45030614527282) bank144755 +144756 POINT(42.590543966715494 -88.5291269523488) bank144756 +144757 POINT(42.48400724496993 -87.00747633077063) bank144757 +144758 POINT(42.193663982354124 -88.02679658196993) bank144758 +144759 POINT(41.622203751906426 -86.92923617890847) bank144759 +144760 POINT(42.36120622698403 -88.22004962634539) bank144760 +144761 POINT(41.59848976906637 -88.02116060908877) bank144761 +144762 POINT(42.35956028985951 -87.7315540125521) bank144762 +144763 POINT(41.476998981861456 -86.817770222679) bank144763 +144764 POINT(41.88473499250164 -88.25888078169403) bank144764 +144765 POINT(41.32884887461361 -86.70036049777264) bank144765 +144766 POINT(41.33816027072157 -87.9518289759586) bank144766 +144767 POINT(41.332161616578595 -88.2379659170018) bank144767 +144768 POINT(42.64281368928836 -87.54554819232098) bank144768 +144769 POINT(41.07339626545748 -88.16892706005086) bank144769 +144770 POINT(42.604691209383994 -88.09806499576993) bank144770 +144771 POINT(41.92103637777314 -87.53152374137204) bank144771 +144772 POINT(42.0294720936461 -87.68353579410547) bank144772 +144773 POINT(41.657498586297216 -87.96152411699963) bank144773 +144774 POINT(42.46498671050464 -88.52795200445638) bank144774 +144775 POINT(41.12077108575109 -88.15779737255461) bank144775 +144776 POINT(41.78593249970124 -88.47197797870969) bank144776 +144777 POINT(40.94652152171754 -87.33625802461704) bank144777 +144778 POINT(42.26881273703017 -86.64609249376045) bank144778 +144779 POINT(41.83386682197178 -88.05538274819895) bank144779 +144780 POINT(41.87009037292215 -87.72569793906713) bank144780 +144781 POINT(41.01205610650014 -88.00546549604297) bank144781 +144782 POINT(42.39347120070742 -86.99496705104366) bank144782 +144783 POINT(42.40967990239447 -88.56151592376538) bank144783 +144784 POINT(42.738687931613505 -88.24498189192403) bank144784 +144785 POINT(41.858087833761736 -88.59485372848866) bank144785 +144786 POINT(40.90455546315648 -87.91978207645518) bank144786 +144787 POINT(42.11938917775954 -88.231071665211) bank144787 +144788 POINT(42.17331384913451 -86.74399951211724) bank144788 +144789 POINT(41.77964493380068 -87.96947833351683) bank144789 +144790 POINT(41.29880065099923 -87.22380923319162) bank144790 +144791 POINT(41.632693715889815 -88.14208485215072) bank144791 +144792 POINT(42.06459763891101 -88.1409391881327) bank144792 +144793 POINT(41.89538424703454 -87.09911051749162) bank144793 +144794 POINT(41.23998338409088 -87.1942215629545) bank144794 +144795 POINT(41.62172141744965 -88.57326730624926) bank144795 +144796 POINT(42.100495392619244 -87.69132172254413) bank144796 +144797 POINT(42.832508664862274 -88.5989807721825) bank144797 +144798 POINT(42.762916064358905 -88.3833418405677) bank144798 +144799 POINT(42.08532195998745 -87.8964506165396) bank144799 +144800 POINT(42.34060103745656 -86.80241356391906) bank144800 +144801 POINT(41.54703666789013 -88.48050853542058) bank144801 +144802 POINT(42.30073848534354 -87.95007637054145) bank144802 +144803 POINT(41.20443524438503 -86.68031275774916) bank144803 +144804 POINT(42.6116307088974 -87.24552833351756) bank144804 +144805 POINT(41.88282050483828 -86.70628611487338) bank144805 +144806 POINT(41.38028096605806 -87.65078506918336) bank144806 +144807 POINT(41.73332920479494 -87.41448827281498) bank144807 +144808 POINT(41.19389564737363 -87.90630273990382) bank144808 +144809 POINT(40.89525372950249 -87.61777169040371) bank144809 +144810 POINT(41.62038071719876 -88.19870730111069) bank144810 +144811 POINT(42.02678326524138 -86.89742238481404) bank144811 +144812 POINT(42.089007459086766 -87.43778902580347) bank144812 +144813 POINT(41.5433889746355 -88.48376513080916) bank144813 +144814 POINT(41.59285824236402 -86.84627641272584) bank144814 +144815 POINT(41.547972398477434 -86.96503796243725) bank144815 +144816 POINT(42.56708845667249 -87.82243142946143) bank144816 +144817 POINT(41.74796141057054 -88.06813303739088) bank144817 +144818 POINT(42.25994258855653 -87.9052148595588) bank144818 +144819 POINT(41.08954962988644 -87.71442280091998) bank144819 +144820 POINT(41.48138587614825 -88.62856635064401) bank144820 +144821 POINT(42.85858747597952 -87.93868411893008) bank144821 +144822 POINT(42.23296769238328 -88.17493163355627) bank144822 +144823 POINT(41.958052277012676 -86.95610749934943) bank144823 +144824 POINT(42.8463300728726 -87.20728612846524) bank144824 +144825 POINT(41.69993861384907 -87.69983114763903) bank144825 +144826 POINT(40.931128092566674 -88.02500184006523) bank144826 +144827 POINT(42.34596063940185 -88.54053835156485) bank144827 +144828 POINT(42.22520486074589 -86.85573139235366) bank144828 +144829 POINT(42.59030285820704 -87.83058481087744) bank144829 +144830 POINT(41.940815001699946 -86.76166108886) bank144830 +144831 POINT(42.03562071076636 -86.81465585015941) bank144831 +144832 POINT(41.41967517968512 -88.36839733822369) bank144832 +144833 POINT(41.824429016359595 -86.70538753933278) bank144833 +144834 POINT(41.115178662244226 -88.14461369539934) bank144834 +144835 POINT(41.02422679553844 -87.0128434127468) bank144835 +144836 POINT(41.102367219122044 -87.4505326995083) bank144836 +144837 POINT(41.15023137678091 -88.35682575687231) bank144837 +144838 POINT(41.53302610633258 -87.46555572154128) bank144838 +144839 POINT(41.439432518488104 -88.37653770820496) bank144839 +144840 POINT(42.81191517242738 -87.1981229892422) bank144840 +144841 POINT(41.54740174276081 -87.77085573395424) bank144841 +144842 POINT(41.81922655386226 -87.00889326708548) bank144842 +144843 POINT(41.316135597382605 -88.29110163763724) bank144843 +144844 POINT(42.33985915917265 -88.55434531775626) bank144844 +144845 POINT(40.95050593315228 -86.82356318442424) bank144845 +144846 POINT(41.62730810840434 -86.9425340879626) bank144846 +144847 POINT(42.050110459876684 -87.94981988091935) bank144847 +144848 POINT(42.155999393971385 -88.34900902269797) bank144848 +144849 POINT(41.87426813824955 -87.81915239354812) bank144849 +144850 POINT(42.80810914211749 -88.0503484521055) bank144850 +144851 POINT(41.041222740742306 -87.44286000780758) bank144851 +144852 POINT(40.96931003531362 -87.68947649724569) bank144852 +144853 POINT(41.44964241654997 -86.86837212909144) bank144853 +144854 POINT(42.2061595352825 -86.87833265785818) bank144854 +144855 POINT(42.55864909065676 -88.19884236776178) bank144855 +144856 POINT(41.63276464966322 -88.57875602329649) bank144856 +144857 POINT(41.17030078793955 -88.27629876614289) bank144857 +144858 POINT(42.36789783302571 -88.28026347961644) bank144858 +144859 POINT(42.3012034664593 -87.25992387358349) bank144859 +144860 POINT(42.52522728308203 -87.2682341425197) bank144860 +144861 POINT(42.03422535765021 -87.84509022450843) bank144861 +144862 POINT(40.97416923987083 -87.13889788028337) bank144862 +144863 POINT(41.40206993056532 -87.44834279027066) bank144863 +144864 POINT(40.95047902124214 -86.74629861828907) bank144864 +144865 POINT(40.93129615910164 -88.01527613904564) bank144865 +144866 POINT(42.1277122883811 -86.88973882247912) bank144866 +144867 POINT(42.80515418826673 -87.24919701365714) bank144867 +144868 POINT(42.264541757393765 -87.48840331934949) bank144868 +144869 POINT(41.225427770722014 -88.50578794721159) bank144869 +144870 POINT(42.32700397390704 -88.50508863568007) bank144870 +144871 POINT(41.18579001240725 -87.90732659080064) bank144871 +144872 POINT(41.72612969850043 -86.81139988328061) bank144872 +144873 POINT(41.71221575123884 -87.58314150789626) bank144873 +144874 POINT(41.16741230717591 -86.66060530588248) bank144874 +144875 POINT(41.65762226109881 -87.6919471366007) bank144875 +144876 POINT(42.53954756255787 -86.80489260953156) bank144876 +144877 POINT(41.68819028814088 -87.3220407171181) bank144877 +144878 POINT(41.17247812771645 -88.17661934558045) bank144878 +144879 POINT(42.87504351447322 -87.56557530084058) bank144879 +144880 POINT(41.37933609753349 -88.0963039525472) bank144880 +144881 POINT(40.921311608182386 -88.04492565681308) bank144881 +144882 POINT(42.439243010977016 -86.74266446324494) bank144882 +144883 POINT(42.782199709223285 -87.50508026250634) bank144883 +144884 POINT(41.35657681833571 -87.28515070286133) bank144884 +144885 POINT(41.48941550118914 -87.93663705049188) bank144885 +144886 POINT(41.241145816214086 -88.09431482593779) bank144886 +144887 POINT(40.89410427045723 -87.82044660271727) bank144887 +144888 POINT(42.479952855858144 -86.83709893152911) bank144888 +144889 POINT(42.461094650919335 -87.66313184166485) bank144889 +144890 POINT(41.266462053003096 -88.19127078906212) bank144890 +144891 POINT(42.20802735502533 -87.57276899187511) bank144891 +144892 POINT(41.71300056348244 -88.61277750618875) bank144892 +144893 POINT(41.80327390505407 -87.64682398219504) bank144893 +144894 POINT(41.60951041562861 -87.4430495911269) bank144894 +144895 POINT(41.15042741500321 -88.05851098831918) bank144895 +144896 POINT(41.65918743160394 -87.09368034074933) bank144896 +144897 POINT(42.85710385461681 -88.50570860711184) bank144897 +144898 POINT(41.327050666059165 -87.61575938947568) bank144898 +144899 POINT(41.35077878897604 -88.27260207687856) bank144899 +144900 POINT(41.74244606149302 -88.22126589604574) bank144900 +144901 POINT(41.452417837256014 -87.06417301060418) bank144901 +144902 POINT(42.55313176788682 -87.8520504168565) bank144902 +144903 POINT(41.702178918242566 -88.30588776439862) bank144903 +144904 POINT(40.93887916603775 -88.3532080741047) bank144904 +144905 POINT(42.009149841970256 -87.73521293849782) bank144905 +144906 POINT(41.889511071256386 -86.93530435438097) bank144906 +144907 POINT(42.25510473125635 -86.82092442447856) bank144907 +144908 POINT(42.72219288684828 -87.6311239134016) bank144908 +144909 POINT(42.70179565288536 -86.94305355013424) bank144909 +144910 POINT(42.759690288544945 -87.28740064115055) bank144910 +144911 POINT(41.34849927154991 -87.94065269325353) bank144911 +144912 POINT(41.60135515047183 -87.69840374677638) bank144912 +144913 POINT(40.965895046433104 -88.11270084481988) bank144913 +144914 POINT(42.739760208560625 -87.11787142409885) bank144914 +144915 POINT(41.60308640167238 -86.84291892776292) bank144915 +144916 POINT(42.18257810706557 -88.03265740021816) bank144916 +144917 POINT(42.183258667401 -86.89232159145314) bank144917 +144918 POINT(41.10470156708043 -88.05829100186094) bank144918 +144919 POINT(41.421197995290676 -87.32710149274254) bank144919 +144920 POINT(41.761484052685624 -86.70478018271004) bank144920 +144921 POINT(42.2569576868282 -88.06576070042692) bank144921 +144922 POINT(41.041016223928565 -88.6089737757009) bank144922 +144923 POINT(41.200190450312846 -87.15634747339995) bank144923 +144924 POINT(42.428877982649475 -88.15327670586123) bank144924 +144925 POINT(41.796485037991744 -86.93373421846934) bank144925 +144926 POINT(42.42880656947011 -88.17166441566353) bank144926 +144927 POINT(41.869324619291255 -87.28543032809613) bank144927 +144928 POINT(41.78939084996444 -87.80787991950487) bank144928 +144929 POINT(42.265783297029074 -88.49391715362303) bank144929 +144930 POINT(42.38517126936158 -87.75352711040865) bank144930 +144931 POINT(42.51680627734851 -86.93894497669415) bank144931 +144932 POINT(41.55866403549887 -87.45207657483573) bank144932 +144933 POINT(41.193941762408194 -88.47512078645131) bank144933 +144934 POINT(41.57092736348128 -88.01674277060538) bank144934 +144935 POINT(42.736680488138575 -87.37475950463748) bank144935 +144936 POINT(42.32326376640421 -88.47993605719805) bank144936 +144937 POINT(42.10649085636987 -87.1674039686557) bank144937 +144938 POINT(42.07993829014695 -88.06959025913015) bank144938 +144939 POINT(42.05648783960236 -87.50293372738976) bank144939 +144940 POINT(42.64897723386123 -88.20533859365005) bank144940 +144941 POINT(41.71006502604156 -87.0794808243757) bank144941 +144942 POINT(42.53430093566134 -88.33109128895578) bank144942 +144943 POINT(41.727410539034715 -86.7091609440297) bank144943 +144944 POINT(41.30631895257055 -88.39820427076148) bank144944 +144945 POINT(41.00929554891396 -88.55325981206771) bank144945 +144946 POINT(41.1964734983628 -88.174177496297) bank144946 +144947 POINT(41.74168482797704 -88.28345159840204) bank144947 +144948 POINT(42.77576241650536 -87.41066359449523) bank144948 +144949 POINT(42.44293562178869 -87.09982576959293) bank144949 +144950 POINT(42.02551675745614 -87.17095664247795) bank144950 +144951 POINT(40.917953072048626 -87.85756797099505) bank144951 +144952 POINT(41.348788674993905 -88.30025959861588) bank144952 +144953 POINT(41.93500529406472 -88.60533877146521) bank144953 +144954 POINT(42.34520179781632 -86.94237103599558) bank144954 +144955 POINT(42.093727001716154 -88.41166285461733) bank144955 +144956 POINT(42.875714615869306 -86.68068785743102) bank144956 +144957 POINT(42.41558582988345 -86.78367457205583) bank144957 +144958 POINT(41.37265694191312 -86.98597265065757) bank144958 +144959 POINT(42.665451746037505 -86.93401681197608) bank144959 +144960 POINT(41.734521599880615 -88.2999323832274) bank144960 +144961 POINT(42.659978990414636 -87.66891127015434) bank144961 +144962 POINT(41.54525378962561 -87.26894592065531) bank144962 +144963 POINT(41.38579179845433 -86.75260637494434) bank144963 +144964 POINT(41.572842887358995 -87.57434815600642) bank144964 +144965 POINT(42.307146498099414 -86.78603112779697) bank144965 +144966 POINT(42.33320668653872 -87.04544689429615) bank144966 +144967 POINT(42.63393890128935 -87.4255478684944) bank144967 +144968 POINT(42.669133658090544 -88.29532401889927) bank144968 +144969 POINT(42.17285163030159 -88.28930864098874) bank144969 +144970 POINT(41.16797031752822 -86.77203368368805) bank144970 +144971 POINT(42.38503906603168 -86.77741119595487) bank144971 +144972 POINT(42.742520054210765 -86.81620892527643) bank144972 +144973 POINT(41.85877181647266 -87.91998084219658) bank144973 +144974 POINT(42.179729546278644 -87.43301038326638) bank144974 +144975 POINT(42.02143728090465 -87.53302361470625) bank144975 +144976 POINT(42.231057033904946 -86.78585900345149) bank144976 +144977 POINT(42.478888096505734 -88.15902320001192) bank144977 +144978 POINT(41.51408917436702 -86.86624291067588) bank144978 +144979 POINT(41.00657871180183 -88.40520723075934) bank144979 +144980 POINT(41.70243078669726 -86.70592199350243) bank144980 +144981 POINT(42.57487309025921 -86.68581476759951) bank144981 +144982 POINT(42.008489871810184 -87.68247669283024) bank144982 +144983 POINT(41.82858051011116 -88.44733003785733) bank144983 +144984 POINT(42.53562125510013 -88.05941679801116) bank144984 +144985 POINT(42.16362843727275 -86.84626764790734) bank144985 +144986 POINT(41.67034298517765 -87.36065477537386) bank144986 +144987 POINT(42.60839528548231 -86.65725941500715) bank144987 +144988 POINT(42.04855492416698 -87.99916976147364) bank144988 +144989 POINT(41.204407448681536 -87.9794204651321) bank144989 +144990 POINT(42.35458198247652 -88.19848109311481) bank144990 +144991 POINT(42.06792931574854 -87.81103185366912) bank144991 +144992 POINT(41.17733711850755 -87.1422860099341) bank144992 +144993 POINT(42.67147600859542 -87.15647951283648) bank144993 +144994 POINT(42.25114860492902 -87.76427803903836) bank144994 +144995 POINT(42.28062607517966 -87.20701296927908) bank144995 +144996 POINT(42.03358682991259 -88.29245044618017) bank144996 +144997 POINT(42.62756462652299 -87.03097359368967) bank144997 +144998 POINT(41.56802240266338 -87.46247486743201) bank144998 +144999 POINT(41.652262313408066 -86.83252028558103) bank144999 +145000 POINT(42.14638638312305 -86.75078383439904) bank145000 +145001 POINT(42.68409805892119 -86.99393528711323) bank145001 +145002 POINT(42.5019238420846 -87.9055531842949) bank145002 +145003 POINT(42.30876672645989 -86.92280979535369) bank145003 +145004 POINT(41.28605457780564 -88.05138360914418) bank145004 +145005 POINT(41.79181096233493 -86.8627324673323) bank145005 +145006 POINT(42.14964284087528 -87.09363969178092) bank145006 +145007 POINT(41.17287756593332 -88.53201943095937) bank145007 +145008 POINT(41.564070319046586 -87.33604131472643) bank145008 +145009 POINT(41.67764401283865 -88.31699516642355) bank145009 +145010 POINT(41.60469962807972 -88.38996003106865) bank145010 +145011 POINT(41.493099534969915 -87.0724722940811) bank145011 +145012 POINT(41.473115567821445 -87.21831255835492) bank145012 +145013 POINT(41.14591996975874 -86.97023235225453) bank145013 +145014 POINT(42.33381468383983 -87.59022515210927) bank145014 +145015 POINT(42.34764394359347 -88.1634816366432) bank145015 +145016 POINT(41.41936873901392 -86.71487712857726) bank145016 +145017 POINT(41.65864591737762 -87.55645923664557) bank145017 +145018 POINT(42.551061771170716 -87.15340320451699) bank145018 +145019 POINT(40.906031111104085 -87.99929418543816) bank145019 +145020 POINT(41.58279369095183 -86.70734956167834) bank145020 +145021 POINT(41.73634454430997 -88.42321300546125) bank145021 +145022 POINT(41.582931656645336 -87.10760778191789) bank145022 +145023 POINT(41.42265273884823 -86.63236793207402) bank145023 +145024 POINT(42.24638836718774 -88.57527704075936) bank145024 +145025 POINT(42.3242470286414 -86.93116606143921) bank145025 +145026 POINT(42.60230901902774 -86.80802090321288) bank145026 +145027 POINT(41.28782967881725 -86.99673457960519) bank145027 +145028 POINT(42.64572557155217 -87.38590052134252) bank145028 +145029 POINT(42.22312518579059 -87.89685904710103) bank145029 +145030 POINT(40.90766942139428 -88.39121837107405) bank145030 +145031 POINT(40.88566886802599 -88.0306360685626) bank145031 +145032 POINT(41.86580905018149 -87.63920971977527) bank145032 +145033 POINT(41.30578925327119 -87.50036309115319) bank145033 +145034 POINT(41.732004708936515 -87.91030057236527) bank145034 +145035 POINT(41.19601111551234 -87.7535717818408) bank145035 +145036 POINT(41.88513515368371 -88.37715712235371) bank145036 +145037 POINT(41.80419960458716 -87.08731151045774) bank145037 +145038 POINT(42.42097040642213 -86.65083419612735) bank145038 +145039 POINT(42.85983954201413 -88.03289422142456) bank145039 +145040 POINT(41.41804719749305 -88.28587373826839) bank145040 +145041 POINT(41.380863750872784 -87.80557868859528) bank145041 +145042 POINT(41.51108611825816 -87.30363614328363) bank145042 +145043 POINT(41.84403830021235 -87.23745447601449) bank145043 +145044 POINT(41.43938756515597 -86.80434873550831) bank145044 +145045 POINT(41.736467210527955 -87.35639594884866) bank145045 +145046 POINT(41.244848023162064 -87.80901372603309) bank145046 +145047 POINT(42.609466729605415 -87.77929236960752) bank145047 +145048 POINT(41.432535893039955 -87.88960818430058) bank145048 +145049 POINT(41.492390718213954 -88.02506403838414) bank145049 +145050 POINT(41.33494710841791 -87.39735155965823) bank145050 +145051 POINT(42.847936608508114 -87.76740813263491) bank145051 +145052 POINT(42.75200893257311 -88.36532050495322) bank145052 +145053 POINT(42.67915604292136 -87.93913182236544) bank145053 +145054 POINT(41.981662354769846 -86.72850563771209) bank145054 +145055 POINT(42.466867298314156 -88.03708501176499) bank145055 +145056 POINT(42.19565607298998 -88.51396975215407) bank145056 +145057 POINT(42.54538238692687 -87.30305033388257) bank145057 +145058 POINT(42.1591297921154 -87.3091246547817) bank145058 +145059 POINT(41.86093695897062 -87.24953347076023) bank145059 +145060 POINT(41.144372413610135 -86.90592899885785) bank145060 +145061 POINT(42.438869827092525 -87.44892853794603) bank145061 +145062 POINT(42.021227086528626 -88.44568647828132) bank145062 +145063 POINT(42.60602714898357 -87.94526459384879) bank145063 +145064 POINT(41.76714660014346 -88.48378782244193) bank145064 +145065 POINT(42.65802674166674 -86.85810597264579) bank145065 +145066 POINT(41.32322295443355 -87.43736179631206) bank145066 +145067 POINT(42.20583634777913 -87.73562557719875) bank145067 +145068 POINT(42.836253406207156 -88.23109399203352) bank145068 +145069 POINT(40.96699759229222 -87.29245790415544) bank145069 +145070 POINT(41.20707144050067 -87.29400957290339) bank145070 +145071 POINT(41.61347189871901 -86.77743374066101) bank145071 +145072 POINT(41.336441323145294 -87.3482427985238) bank145072 +145073 POINT(41.6562367887938 -87.63321954231765) bank145073 +145074 POINT(41.58355889145394 -86.76339274818751) bank145074 +145075 POINT(42.87071383518232 -87.75467413599918) bank145075 +145076 POINT(42.49666046047314 -87.08457359838455) bank145076 +145077 POINT(42.859937170270676 -87.73490850366235) bank145077 +145078 POINT(40.97807658347176 -87.50288350590998) bank145078 +145079 POINT(41.70684581584083 -87.26794779010804) bank145079 +145080 POINT(41.39556403453249 -88.33881065627834) bank145080 +145081 POINT(42.024052966175006 -88.12936485231752) bank145081 +145082 POINT(41.02311698472222 -87.7404571217775) bank145082 +145083 POINT(42.16758321585801 -86.96686324807906) bank145083 +145084 POINT(41.92426628908493 -88.3495458950445) bank145084 +145085 POINT(42.683070513491096 -88.17170170208242) bank145085 +145086 POINT(42.22380412837539 -88.58725072724191) bank145086 +145087 POINT(42.204584794912805 -87.72212094886649) bank145087 +145088 POINT(42.018032132569566 -87.3281343484404) bank145088 +145089 POINT(42.417278458070115 -87.61608928516574) bank145089 +145090 POINT(41.522057331866506 -86.79879466771875) bank145090 +145091 POINT(42.843263736000665 -88.35386848995824) bank145091 +145092 POINT(41.99690102280765 -88.08922047783325) bank145092 +145093 POINT(41.61526528896153 -87.09766684600903) bank145093 +145094 POINT(42.40262717336363 -86.9106614456581) bank145094 +145095 POINT(41.89482203276304 -87.51690099915983) bank145095 +145096 POINT(41.77293672519185 -88.09730275278488) bank145096 +145097 POINT(42.667676250567816 -88.4887272235535) bank145097 +145098 POINT(42.83817015884326 -86.81272670597947) bank145098 +145099 POINT(41.1928612978935 -86.87509642977099) bank145099 +145100 POINT(42.20598766861908 -87.09471547521558) bank145100 +145101 POINT(41.96680629574563 -87.50969691658509) bank145101 +145102 POINT(42.79217509518471 -87.61973106363396) bank145102 +145103 POINT(41.600646444988975 -86.86189949095646) bank145103 +145104 POINT(41.909737209447584 -87.67437217342463) bank145104 +145105 POINT(42.398538957314976 -88.60647758361519) bank145105 +145106 POINT(41.12180982174054 -86.97706071677055) bank145106 +145107 POINT(41.46981736302272 -86.80024689680165) bank145107 +145108 POINT(41.76796830509831 -88.13030004192004) bank145108 +145109 POINT(41.51257488521092 -87.6470215904744) bank145109 +145110 POINT(41.32662908490208 -87.32511302796637) bank145110 +145111 POINT(40.92965062746736 -88.37542121437959) bank145111 +145112 POINT(41.32086159523926 -87.25321240936498) bank145112 +145113 POINT(42.416443012226964 -87.32016247057128) bank145113 +145114 POINT(42.26899340731043 -87.51939765706928) bank145114 +145115 POINT(42.758836558672094 -87.77274343351272) bank145115 +145116 POINT(42.13851005988984 -87.28738806043667) bank145116 +145117 POINT(42.75405528909906 -86.65187342213503) bank145117 +145118 POINT(41.29319522793672 -87.32872615026572) bank145118 +145119 POINT(40.89106176784982 -88.56232124656336) bank145119 +145120 POINT(41.061789160535945 -88.47856858052475) bank145120 +145121 POINT(40.96031814791992 -86.6682519041776) bank145121 +145122 POINT(41.487338052046354 -86.95635880820667) bank145122 +145123 POINT(42.37537566967774 -87.4619593926667) bank145123 +145124 POINT(41.20485703065248 -86.74991250351242) bank145124 +145125 POINT(42.80173845758028 -87.24709912659318) bank145125 +145126 POINT(42.84595530691007 -87.65155903805292) bank145126 +145127 POINT(42.10545668408986 -88.172933007029) bank145127 +145128 POINT(41.34480692274697 -87.872375452308) bank145128 +145129 POINT(42.573368287468554 -86.7739375367707) bank145129 +145130 POINT(41.52132255264597 -87.66639035206414) bank145130 +145131 POINT(41.91469760227812 -88.58853568980193) bank145131 +145132 POINT(42.24592081760075 -87.57412427250485) bank145132 +145133 POINT(41.694816109207736 -86.70795307802851) bank145133 +145134 POINT(40.9501892003079 -88.29419019541403) bank145134 +145135 POINT(41.839748392832355 -86.9435384296677) bank145135 +145136 POINT(42.03836590352425 -87.92097848649655) bank145136 +145137 POINT(42.12492821309983 -86.6509049034304) bank145137 +145138 POINT(41.39166778639961 -88.01822448925299) bank145138 +145139 POINT(42.42558352656023 -87.94432343355741) bank145139 +145140 POINT(42.25795835913004 -87.07742602153299) bank145140 +145141 POINT(40.88259489099244 -86.67965791288358) bank145141 +145142 POINT(41.003906101443704 -88.43431132362366) bank145142 +145143 POINT(41.006836075125186 -88.45889760591879) bank145143 +145144 POINT(42.27202884859248 -88.4911248070421) bank145144 +145145 POINT(42.39072179779182 -86.68238828966999) bank145145 +145146 POINT(41.653873090862646 -87.2054642576367) bank145146 +145147 POINT(42.307702053868944 -87.91956192033214) bank145147 +145148 POINT(41.748098600133574 -88.16075817182224) bank145148 +145149 POINT(41.09785029812142 -87.53525048565331) bank145149 +145150 POINT(42.81071144675918 -87.99377401256723) bank145150 +145151 POINT(41.45878330298619 -88.33461922005671) bank145151 +145152 POINT(42.762149334244505 -87.86617592749045) bank145152 +145153 POINT(42.08760938985605 -88.07314379444915) bank145153 +145154 POINT(41.81155618494767 -87.96641786061241) bank145154 +145155 POINT(42.439102393391344 -87.23272263817859) bank145155 +145156 POINT(41.649913726674775 -87.27201679499298) bank145156 +145157 POINT(42.77482395915502 -88.13346743862243) bank145157 +145158 POINT(41.417654567396134 -88.03641029656397) bank145158 +145159 POINT(42.44485191401518 -87.06309629193713) bank145159 +145160 POINT(42.736324438828 -87.54177579274072) bank145160 +145161 POINT(41.60500389893618 -87.56191918386904) bank145161 +145162 POINT(42.656892820769016 -87.79576753376452) bank145162 +145163 POINT(42.598901503655235 -87.5875204797909) bank145163 +145164 POINT(42.76902796741449 -87.88920258544928) bank145164 +145165 POINT(41.122752102488995 -87.06829070665665) bank145165 +145166 POINT(42.30690296431962 -86.74069250111602) bank145166 +145167 POINT(42.01203486177287 -86.9604602115311) bank145167 +145168 POINT(41.307445465532865 -86.77916728632158) bank145168 +145169 POINT(42.02607220161972 -87.39745065303005) bank145169 +145170 POINT(41.890636647521596 -88.49424250534408) bank145170 +145171 POINT(41.81153667118395 -88.29574068205336) bank145171 +145172 POINT(41.179154997932386 -86.64211940885097) bank145172 +145173 POINT(41.88279978412992 -88.40304295552605) bank145173 +145174 POINT(42.74101211288334 -88.15628320035358) bank145174 +145175 POINT(41.27074701787619 -87.58210088017375) bank145175 +145176 POINT(41.76311681800647 -88.41179478786786) bank145176 +145177 POINT(42.34229897909875 -88.20295812043932) bank145177 +145178 POINT(42.33155971128977 -87.9375121890779) bank145178 +145179 POINT(41.0786938460698 -87.05034695329599) bank145179 +145180 POINT(41.153022274322424 -86.7139185457012) bank145180 +145181 POINT(41.567621265884284 -87.4202085014767) bank145181 +145182 POINT(41.4940879388318 -87.92119226819266) bank145182 +145183 POINT(41.934437732114496 -88.15568905988431) bank145183 +145184 POINT(42.46726133444734 -87.49162122962167) bank145184 +145185 POINT(41.497861060533985 -88.12258487241787) bank145185 +145186 POINT(40.88964405253938 -87.90850233776322) bank145186 +145187 POINT(42.766060790995624 -87.75840307147429) bank145187 +145188 POINT(42.41243254963984 -87.29043977873418) bank145188 +145189 POINT(42.368657874738815 -87.76849897405866) bank145189 +145190 POINT(42.09426688103855 -86.895547528968) bank145190 +145191 POINT(42.422323565968426 -87.57533055997708) bank145191 +145192 POINT(41.60477288157688 -87.75522111928152) bank145192 +145193 POINT(41.56096511741469 -86.95140570884152) bank145193 +145194 POINT(40.90466880669212 -87.62242258652856) bank145194 +145195 POINT(42.360211937092245 -87.42072428423238) bank145195 +145196 POINT(41.54739255628024 -87.58166088944081) bank145196 +145197 POINT(41.136546143383114 -87.78106263161307) bank145197 +145198 POINT(41.47516579809625 -87.94807048357964) bank145198 +145199 POINT(41.15558139411078 -88.56731340572259) bank145199 +145200 POINT(41.99218947605755 -87.55549817075003) bank145200 +145201 POINT(41.58844166612089 -87.54148294251937) bank145201 +145202 POINT(42.423084574777974 -88.54854019677047) bank145202 +145203 POINT(42.20502525228733 -86.94032869607811) bank145203 +145204 POINT(40.93605891000892 -88.32588085220283) bank145204 +145205 POINT(42.727222504027274 -87.62643897181457) bank145205 +145206 POINT(41.822738289489656 -87.9426700856976) bank145206 +145207 POINT(41.89532992949936 -88.18632176595732) bank145207 +145208 POINT(41.64549206804325 -88.30923854470878) bank145208 +145209 POINT(41.97532765737625 -88.57141088267583) bank145209 +145210 POINT(41.563059465330866 -87.79012878899883) bank145210 +145211 POINT(41.91320182402237 -88.10609449137013) bank145211 +145212 POINT(41.80187829941938 -88.12114736582268) bank145212 +145213 POINT(41.01083159178273 -87.13770489959134) bank145213 +145214 POINT(41.911592506221 -87.50129245995508) bank145214 +145215 POINT(42.80191792493408 -88.30559711040318) bank145215 +145216 POINT(42.349143763637045 -86.83151804673432) bank145216 +145217 POINT(42.77420285831353 -87.26390838843616) bank145217 +145218 POINT(42.80381277154085 -87.17226155786858) bank145218 +145219 POINT(42.059759545170536 -87.97002699251674) bank145219 +145220 POINT(42.16867403983161 -87.0142935337236) bank145220 +145221 POINT(42.56262005687908 -87.47769564871304) bank145221 +145222 POINT(42.02470762296155 -87.68199533392617) bank145222 +145223 POINT(41.00690868403559 -88.1636950818728) bank145223 +145224 POINT(42.26329661956289 -88.39325741412205) bank145224 +145225 POINT(42.63134463459441 -88.42480325283515) bank145225 +145226 POINT(42.138398717911635 -86.97906962836521) bank145226 +145227 POINT(41.915867133026204 -87.94949773535888) bank145227 +145228 POINT(42.243661556562444 -87.11475834503435) bank145228 +145229 POINT(42.289556693008194 -87.47179273888753) bank145229 +145230 POINT(42.79063304900331 -88.37234893795873) bank145230 +145231 POINT(41.87786247206853 -88.05730785270941) bank145231 +145232 POINT(40.92960436782554 -87.01834179710539) bank145232 +145233 POINT(42.74731062550801 -88.4056438289292) bank145233 +145234 POINT(41.664068455804916 -87.79981949326125) bank145234 +145235 POINT(41.49760623468265 -86.72718122842598) bank145235 +145236 POINT(42.72736558201573 -86.70690665512996) bank145236 +145237 POINT(41.185672624786754 -88.35521771880488) bank145237 +145238 POINT(41.16400662378638 -88.59546245655972) bank145238 +145239 POINT(41.80410739194122 -86.86892964376888) bank145239 +145240 POINT(42.5960631100168 -88.0810444674618) bank145240 +145241 POINT(41.86891920993791 -87.18272973763877) bank145241 +145242 POINT(42.00288047315866 -87.32692517488219) bank145242 +145243 POINT(42.08806366667127 -87.6194055833286) bank145243 +145244 POINT(41.11397580363897 -86.92636805428965) bank145244 +145245 POINT(42.25747660263421 -87.28357381935005) bank145245 +145246 POINT(42.339953399379674 -88.13095185324937) bank145246 +145247 POINT(41.593567801187874 -86.65407602961085) bank145247 +145248 POINT(42.21584838166579 -86.7426743905753) bank145248 +145249 POINT(41.96497036040831 -87.40836721275952) bank145249 +145250 POINT(41.22237057273004 -87.38394875391056) bank145250 +145251 POINT(42.24365386934035 -87.27871053742733) bank145251 +145252 POINT(42.638151185124656 -86.9656677160034) bank145252 +145253 POINT(41.68155254627058 -87.45147914785392) bank145253 +145254 POINT(41.29166866719385 -88.45378125250782) bank145254 +145255 POINT(42.163124271425346 -87.42128415789384) bank145255 +145256 POINT(41.034128604014896 -86.7806919680357) bank145256 +145257 POINT(42.702924245706775 -87.61745345180559) bank145257 +145258 POINT(41.38580504094753 -87.97238699233363) bank145258 +145259 POINT(41.831742633105634 -87.1230548395019) bank145259 +145260 POINT(41.39046693984227 -87.71704344555324) bank145260 +145261 POINT(41.068003825830985 -88.43496990658467) bank145261 +145262 POINT(41.89419432552412 -88.07911938538251) bank145262 +145263 POINT(41.16994988023716 -87.22628439089934) bank145263 +145264 POINT(42.351045617172396 -87.7457309963888) bank145264 +145265 POINT(41.63225939070756 -88.1600183177131) bank145265 +145266 POINT(41.82747421242714 -86.80498274616731) bank145266 +145267 POINT(40.994852368907694 -87.01584127848827) bank145267 +145268 POINT(41.94202533504628 -87.32307653208959) bank145268 +145269 POINT(41.783485115597166 -86.95412523797428) bank145269 +145270 POINT(41.398480557054725 -87.93228779269168) bank145270 +145271 POINT(41.81435193424665 -87.03126777475212) bank145271 +145272 POINT(42.04597143462699 -86.97529260759578) bank145272 +145273 POINT(41.05019952111739 -86.82977258949803) bank145273 +145274 POINT(41.719891664463475 -87.1587527093046) bank145274 +145275 POINT(41.40704620328971 -87.75138710358775) bank145275 +145276 POINT(42.80471098333268 -88.19382932570859) bank145276 +145277 POINT(42.087336850668976 -88.24629428415088) bank145277 +145278 POINT(41.30382727683506 -87.22299479597758) bank145278 +145279 POINT(41.06025972642468 -87.70316382171036) bank145279 +145280 POINT(42.79759033636276 -88.32745058949189) bank145280 +145281 POINT(42.705521132696504 -88.20100215206467) bank145281 +145282 POINT(41.7981361977532 -88.21494666510107) bank145282 +145283 POINT(41.47391384887187 -87.67180149437215) bank145283 +145284 POINT(41.272909511869834 -88.5284451217573) bank145284 +145285 POINT(41.651298774312316 -86.99428988466705) bank145285 +145286 POINT(41.0290058312144 -88.05921788903314) bank145286 +145287 POINT(41.22208987249287 -86.835950622583) bank145287 +145288 POINT(41.476996847674585 -87.8365365934925) bank145288 +145289 POINT(42.46094829994812 -88.25350567506686) bank145289 +145290 POINT(42.642084483169015 -87.3196589838869) bank145290 +145291 POINT(42.66150888277366 -87.39942849360662) bank145291 +145292 POINT(41.14033321254 -86.91767637181343) bank145292 +145293 POINT(42.375808890750434 -88.43309802002072) bank145293 +145294 POINT(41.97055709317484 -87.94475818924029) bank145294 +145295 POINT(41.33977214022504 -87.17218401055072) bank145295 +145296 POINT(42.10834983600668 -87.25173914814107) bank145296 +145297 POINT(42.539413886048244 -88.31177568409416) bank145297 +145298 POINT(42.10997609515526 -87.50499093610964) bank145298 +145299 POINT(42.34708026042147 -87.72681296234354) bank145299 +145300 POINT(42.82912009793457 -86.95414381660608) bank145300 +145301 POINT(42.557649652444354 -87.70442337949525) bank145301 +145302 POINT(41.70071782028785 -88.05744616156305) bank145302 +145303 POINT(40.904996438805384 -87.21550040923874) bank145303 +145304 POINT(42.46938660330701 -87.35958831534646) bank145304 +145305 POINT(41.61809952601414 -87.92027901572986) bank145305 +145306 POINT(42.28820305156426 -88.3495351821458) bank145306 +145307 POINT(41.408679318450055 -87.6081840624933) bank145307 +145308 POINT(41.020123165781406 -86.9377341364735) bank145308 +145309 POINT(41.79394370589899 -88.58066924241044) bank145309 +145310 POINT(41.347465077992815 -87.94053042463018) bank145310 +145311 POINT(41.36701949009327 -88.44595679866963) bank145311 +145312 POINT(41.502180509509984 -86.95797798534306) bank145312 +145313 POINT(42.348234980331405 -88.35917445866818) bank145313 +145314 POINT(40.958591920320764 -87.23977754419639) bank145314 +145315 POINT(41.78031050929305 -87.41756015858225) bank145315 +145316 POINT(42.590468710999026 -87.26998449138226) bank145316 +145317 POINT(41.843702903015355 -87.99797642781175) bank145317 +145318 POINT(41.96468086168446 -87.98768103282407) bank145318 +145319 POINT(42.761079372346565 -87.64715432004988) bank145319 +145320 POINT(41.562067094158415 -88.32367318756859) bank145320 +145321 POINT(42.52598001944837 -87.49130990528496) bank145321 +145322 POINT(40.9631954135005 -88.25903832251251) bank145322 +145323 POINT(41.55470501000861 -87.54010944107196) bank145323 +145324 POINT(42.58822124476595 -88.24361604119494) bank145324 +145325 POINT(41.91554003150821 -87.48116875894037) bank145325 +145326 POINT(41.37060413918989 -87.53440416250339) bank145326 +145327 POINT(42.03096237046218 -87.8230981609975) bank145327 +145328 POINT(41.292476331381934 -87.0227577201714) bank145328 +145329 POINT(42.040309136406755 -87.61720238331618) bank145329 +145330 POINT(42.5095143085394 -87.86362568218092) bank145330 +145331 POINT(41.22947825276887 -86.833982788028) bank145331 +145332 POINT(41.680563367730976 -86.85683632081493) bank145332 +145333 POINT(42.667078689805045 -87.81327830302499) bank145333 +145334 POINT(41.212461979557425 -87.87634164778105) bank145334 +145335 POINT(42.40487203007966 -87.77208433282782) bank145335 +145336 POINT(41.5589862514086 -88.55111771198398) bank145336 +145337 POINT(42.26049426359351 -87.80569963142618) bank145337 +145338 POINT(42.2905820091775 -88.53677765136504) bank145338 +145339 POINT(41.05411109495575 -86.73713795074929) bank145339 +145340 POINT(40.89305630184124 -87.36609165828531) bank145340 +145341 POINT(42.514592688409586 -87.04840649722118) bank145341 +145342 POINT(41.431198630097306 -87.51484665337034) bank145342 +145343 POINT(41.21438602338078 -87.66188746421295) bank145343 +145344 POINT(41.72534435618675 -88.43816325457838) bank145344 +145345 POINT(42.29778816586212 -86.97602573637872) bank145345 +145346 POINT(42.6929159192865 -88.50687861729496) bank145346 +145347 POINT(42.719282317815214 -87.65768802078605) bank145347 +145348 POINT(42.57081136663972 -88.30962645930012) bank145348 +145349 POINT(42.47938899123406 -88.10316061815189) bank145349 +145350 POINT(42.66964652287196 -88.451067368025) bank145350 +145351 POINT(41.11688076582873 -87.68738795530783) bank145351 +145352 POINT(41.31661543441522 -86.68552764205273) bank145352 +145353 POINT(41.37542284769761 -87.92659901026) bank145353 +145354 POINT(42.85057318134248 -87.25193286409117) bank145354 +145355 POINT(42.77738439398688 -86.97995058671604) bank145355 +145356 POINT(41.82835256423232 -87.16996144622541) bank145356 +145357 POINT(42.34681925790273 -87.20717805685827) bank145357 +145358 POINT(42.25748446830478 -87.2831772258118) bank145358 +145359 POINT(42.81431030375846 -87.05198967606627) bank145359 +145360 POINT(41.12516768956367 -88.0887558453823) bank145360 +145361 POINT(42.51241168382215 -87.18031783692682) bank145361 +145362 POINT(42.06619194277622 -87.66270429403488) bank145362 +145363 POINT(41.31461415151822 -87.33322473169409) bank145363 +145364 POINT(42.663746414037135 -88.54095959539696) bank145364 +145365 POINT(42.49877560016975 -86.77665355835724) bank145365 +145366 POINT(41.20860892139411 -86.92848853661246) bank145366 +145367 POINT(41.79878789616804 -87.96379506453437) bank145367 +145368 POINT(42.01783590679564 -86.65408195887413) bank145368 +145369 POINT(42.02237357742542 -88.21572042081297) bank145369 +145370 POINT(41.799007978093506 -86.80833638437666) bank145370 +145371 POINT(42.39209090555612 -87.54825341103653) bank145371 +145372 POINT(40.94929386413907 -87.47324900260405) bank145372 +145373 POINT(42.46452334829836 -87.84966357558513) bank145373 +145374 POINT(42.412777255691395 -87.77204470867538) bank145374 +145375 POINT(41.90491706172341 -88.55915147948296) bank145375 +145376 POINT(41.87960990699978 -87.60438357856283) bank145376 +145377 POINT(41.57358619383037 -86.92300449146283) bank145377 +145378 POINT(41.38343424659998 -88.20307161640234) bank145378 +145379 POINT(42.257073530127926 -87.63899673272218) bank145379 +145380 POINT(40.92981204725949 -87.70264354216751) bank145380 +145381 POINT(42.388795668432955 -87.68000434596578) bank145381 +145382 POINT(42.08526794275203 -87.61773123345611) bank145382 +145383 POINT(41.41269773531789 -86.82976535859406) bank145383 +145384 POINT(41.52785914502891 -88.13215009350002) bank145384 +145385 POINT(41.827682879221655 -88.4089909692275) bank145385 +145386 POINT(42.38089860658334 -88.13653908562479) bank145386 +145387 POINT(41.899204503664826 -86.8365834562435) bank145387 +145388 POINT(41.10595539726154 -88.40751100006383) bank145388 +145389 POINT(41.60416514060423 -87.60844150476717) bank145389 +145390 POINT(41.4434846036016 -87.02825009674292) bank145390 +145391 POINT(42.77496802651152 -87.6280753819437) bank145391 +145392 POINT(42.22652531683634 -88.58707541802072) bank145392 +145393 POINT(42.36154164510949 -88.45920667255419) bank145393 +145394 POINT(41.93614129180777 -87.04485299894552) bank145394 +145395 POINT(42.64322900347428 -88.34349797982223) bank145395 +145396 POINT(41.272643323668376 -88.03223077754723) bank145396 +145397 POINT(42.736786516466665 -88.58501985858351) bank145397 +145398 POINT(41.90478768236524 -88.44063850768212) bank145398 +145399 POINT(41.242342106127765 -88.56725573110593) bank145399 +145400 POINT(42.48334928079222 -87.21494343838026) bank145400 +145401 POINT(41.625436461210306 -87.17548405264465) bank145401 +145402 POINT(42.08654574251218 -87.94656813738385) bank145402 +145403 POINT(41.38864658297008 -88.1534912987021) bank145403 +145404 POINT(42.7277333030379 -88.23709187196421) bank145404 +145405 POINT(41.72059481332154 -86.92272168327898) bank145405 +145406 POINT(42.32367923605951 -88.05116679239778) bank145406 +145407 POINT(42.1823816125648 -88.32258442472647) bank145407 +145408 POINT(41.286701458425476 -87.60067751876919) bank145408 +145409 POINT(42.58104443313642 -88.32167658790048) bank145409 +145410 POINT(42.46611228906848 -87.90674365019386) bank145410 +145411 POINT(41.68846513840921 -87.85731797927721) bank145411 +145412 POINT(41.25448727603348 -87.80614557874188) bank145412 +145413 POINT(41.74745831664561 -87.9436513859167) bank145413 +145414 POINT(41.95633251229088 -87.45720152599559) bank145414 +145415 POINT(42.837745908006546 -87.69444400525022) bank145415 +145416 POINT(42.46268278203004 -87.36031320696036) bank145416 +145417 POINT(42.048504419574755 -88.07814836262898) bank145417 +145418 POINT(41.60203901730605 -87.93169424592135) bank145418 +145419 POINT(42.61461951753456 -88.25154478966196) bank145419 +145420 POINT(40.95620251166908 -87.58554297485283) bank145420 +145421 POINT(42.06524747321548 -87.1488617737694) bank145421 +145422 POINT(41.83246417768046 -88.61562894700708) bank145422 +145423 POINT(42.184446439924145 -87.20224824463814) bank145423 +145424 POINT(41.78794162408824 -86.65464602765482) bank145424 +145425 POINT(42.709553955611156 -88.10213902127357) bank145425 +145426 POINT(42.076525005611046 -87.87755886057062) bank145426 +145427 POINT(41.81974615222152 -87.49398159530828) bank145427 +145428 POINT(41.750586229635324 -86.7914185261527) bank145428 +145429 POINT(42.452219678603186 -87.07685711687238) bank145429 +145430 POINT(41.52985616437931 -86.9712170248914) bank145430 +145431 POINT(41.286257874293625 -87.90059311389417) bank145431 +145432 POINT(42.039939443853086 -87.80364016249865) bank145432 +145433 POINT(41.44437809992573 -88.50272917497409) bank145433 +145434 POINT(42.449330088051 -87.73652838663638) bank145434 +145435 POINT(41.7485324332962 -86.78229474389995) bank145435 +145436 POINT(42.70352376112784 -86.72526668164478) bank145436 +145437 POINT(41.880632623249255 -87.48771833797643) bank145437 +145438 POINT(42.2143322798592 -88.49389072746796) bank145438 +145439 POINT(41.73972582566406 -87.17318804923185) bank145439 +145440 POINT(42.72404611947394 -87.36759194787902) bank145440 +145441 POINT(41.0823432018107 -87.12387553819488) bank145441 +145442 POINT(41.775413254774584 -87.60895435026997) bank145442 +145443 POINT(42.830266741704186 -87.07072217730065) bank145443 +145444 POINT(41.14451119205039 -88.2062314300133) bank145444 +145445 POINT(42.77197798546945 -87.57165883177458) bank145445 +145446 POINT(40.99502152150301 -87.14054794004433) bank145446 +145447 POINT(41.7882285104056 -87.65867130846794) bank145447 +145448 POINT(41.12369660904977 -86.91419168233601) bank145448 +145449 POINT(42.13409905846508 -88.28026281406292) bank145449 +145450 POINT(42.65865467830459 -87.18910523469783) bank145450 +145451 POINT(41.326165331394535 -87.85546167270365) bank145451 +145452 POINT(42.40346155630231 -88.3582895323487) bank145452 +145453 POINT(41.828583742373034 -87.35021363346114) bank145453 +145454 POINT(41.26696087971444 -87.09014178544331) bank145454 +145455 POINT(42.47875796136127 -88.39888235337895) bank145455 +145456 POINT(42.26022091159062 -88.45146248328138) bank145456 +145457 POINT(41.69226858475845 -88.38435671956539) bank145457 +145458 POINT(40.95093492975743 -88.28838595354884) bank145458 +145459 POINT(41.92725550545882 -87.71356256642272) bank145459 +145460 POINT(41.409946833891446 -87.84892086561106) bank145460 +145461 POINT(42.662044375024905 -88.33462412611723) bank145461 +145462 POINT(42.109241455532434 -86.87169168844119) bank145462 +145463 POINT(41.38092709866153 -87.15041410957988) bank145463 +145464 POINT(41.031150293734065 -87.04915161571569) bank145464 +145465 POINT(42.86348207374146 -88.45384916210858) bank145465 +145466 POINT(42.07802105479777 -87.03692467555169) bank145466 +145467 POINT(41.88225366820366 -87.02483830899787) bank145467 +145468 POINT(42.518872683864366 -87.45381327970833) bank145468 +145469 POINT(42.250368400346645 -87.07028790188944) bank145469 +145470 POINT(41.57797036833213 -87.42479427561163) bank145470 +145471 POINT(41.54994082271504 -87.59572041295813) bank145471 +145472 POINT(41.68229460064363 -88.47684523975576) bank145472 +145473 POINT(42.389985563682345 -86.90684531130083) bank145473 +145474 POINT(42.46527149991278 -87.29735718023045) bank145474 +145475 POINT(42.37529207516542 -88.02916099611186) bank145475 +145476 POINT(41.813432772654636 -86.64421370070552) bank145476 +145477 POINT(42.0575984837858 -87.25607365873992) bank145477 +145478 POINT(41.24064245361542 -88.04021418500085) bank145478 +145479 POINT(41.568346161515024 -88.0816974170464) bank145479 +145480 POINT(42.391243335913906 -87.84089152308194) bank145480 +145481 POINT(40.90111131073805 -88.57502527734613) bank145481 +145482 POINT(42.49533022926622 -88.53523805887652) bank145482 +145483 POINT(42.438636909070965 -87.83900820253552) bank145483 +145484 POINT(42.69909691574202 -88.37257279781169) bank145484 +145485 POINT(42.62939139363646 -87.58294892645647) bank145485 +145486 POINT(41.96350662161117 -87.83578516791435) bank145486 +145487 POINT(41.73688112778924 -87.86576401303525) bank145487 +145488 POINT(42.45723128379978 -86.78090760681636) bank145488 +145489 POINT(42.28245989543347 -87.72014753929257) bank145489 +145490 POINT(42.85037351729939 -87.89219196231252) bank145490 +145491 POINT(42.536399619171874 -88.34870603128618) bank145491 +145492 POINT(41.702263410328555 -87.05169131233612) bank145492 +145493 POINT(41.778638198883755 -86.72751469140384) bank145493 +145494 POINT(41.01603005147945 -87.209039101362) bank145494 +145495 POINT(41.98755642041451 -88.53595899674302) bank145495 +145496 POINT(41.6075133945665 -87.30894277857021) bank145496 +145497 POINT(41.68379987690441 -87.74887800942194) bank145497 +145498 POINT(41.89992121913712 -88.22931555597609) bank145498 +145499 POINT(41.215695031693045 -87.73854911560049) bank145499 +145500 POINT(41.78844454847121 -87.17196080165138) bank145500 +145501 POINT(42.80192313038621 -87.03573315020829) bank145501 +145502 POINT(42.29043595543255 -86.8160850460112) bank145502 +145503 POINT(42.85772202418092 -88.3629543039285) bank145503 +145504 POINT(42.0875686711454 -87.71158277734487) bank145504 +145505 POINT(41.67290798201853 -87.02757873711246) bank145505 +145506 POINT(41.7676046603989 -87.09243004008452) bank145506 +145507 POINT(42.27901024744321 -87.67576617874325) bank145507 +145508 POINT(42.05722551454979 -87.96283955183975) bank145508 +145509 POINT(42.79000203885586 -87.36503938713354) bank145509 +145510 POINT(41.40980802191607 -88.62871822757269) bank145510 +145511 POINT(41.01247413701448 -88.52335810221798) bank145511 +145512 POINT(40.94825605225345 -88.46182622138943) bank145512 +145513 POINT(41.296659639541 -87.7707851273208) bank145513 +145514 POINT(41.49950144519347 -88.42684711922848) bank145514 +145515 POINT(41.73852937441122 -88.08143341108708) bank145515 +145516 POINT(41.35131390588058 -87.41638650279866) bank145516 +145517 POINT(42.286725208155595 -88.58565622729152) bank145517 +145518 POINT(41.36396243622558 -88.39475724362273) bank145518 +145519 POINT(42.80296363626415 -87.5120461382823) bank145519 +145520 POINT(41.159844500526994 -87.10283406222825) bank145520 +145521 POINT(41.38641685366201 -87.10296629536606) bank145521 +145522 POINT(42.69151706154939 -87.27565837200561) bank145522 +145523 POINT(42.24931144240291 -88.23191104940958) bank145523 +145524 POINT(41.60380658273122 -88.23160581161969) bank145524 +145525 POINT(42.56550575690054 -87.04122924837004) bank145525 +145526 POINT(40.90036748757816 -87.57674021059233) bank145526 +145527 POINT(41.59382699612564 -88.2066054623018) bank145527 +145528 POINT(41.808871528471755 -88.38651927295882) bank145528 +145529 POINT(41.41659982440346 -87.276173167168) bank145529 +145530 POINT(41.71971552547741 -87.15532289108404) bank145530 +145531 POINT(41.14818186571329 -87.24668074599411) bank145531 +145532 POINT(42.13989810127909 -86.88627512466132) bank145532 +145533 POINT(41.256263200130704 -87.8417044958576) bank145533 +145534 POINT(41.82928689333098 -86.96622440291036) bank145534 +145535 POINT(42.874015124211795 -86.75808249230374) bank145535 +145536 POINT(41.9063861023739 -87.29192884982997) bank145536 +145537 POINT(41.70841441858223 -86.75797820761257) bank145537 +145538 POINT(42.19319604858126 -87.20641449247641) bank145538 +145539 POINT(42.8552581447315 -87.75342785300386) bank145539 +145540 POINT(42.46190071612784 -87.232223055736) bank145540 +145541 POINT(40.91360659787239 -87.50718169211696) bank145541 +145542 POINT(42.35612068513006 -88.08706166625174) bank145542 +145543 POINT(41.862243053693966 -87.79982750069574) bank145543 +145544 POINT(42.1481102204888 -88.4720919458215) bank145544 +145545 POINT(42.07697899509384 -88.25799189927567) bank145545 +145546 POINT(42.10511501242025 -87.55387006017357) bank145546 +145547 POINT(41.61354531996977 -87.1165505459648) bank145547 +145548 POINT(41.195013757390264 -86.92385825487565) bank145548 +145549 POINT(42.479677421092745 -87.59456584702778) bank145549 +145550 POINT(41.71175771138677 -86.84141999655694) bank145550 +145551 POINT(42.27231561621852 -87.35193758819749) bank145551 +145552 POINT(42.64004576312504 -88.51574546674779) bank145552 +145553 POINT(41.30256395314239 -87.43613329319847) bank145553 +145554 POINT(41.930250863102636 -88.09782537542272) bank145554 +145555 POINT(41.727493125920596 -88.53407545891058) bank145555 +145556 POINT(42.73823143032567 -86.65840895534473) bank145556 +145557 POINT(42.462084873269156 -87.20675241239645) bank145557 +145558 POINT(41.60814096454158 -86.69705756945554) bank145558 +145559 POINT(41.81826959253628 -88.44129676018515) bank145559 +145560 POINT(41.13666461715535 -88.15603171150902) bank145560 +145561 POINT(42.57190665360923 -87.54714965864785) bank145561 +145562 POINT(42.03067527051033 -87.24332363750185) bank145562 +145563 POINT(42.79079723637199 -88.43306029949781) bank145563 +145564 POINT(42.55462579176559 -87.31034578437252) bank145564 +145565 POINT(41.852087367428716 -88.57600378903713) bank145565 +145566 POINT(42.660656342191686 -87.06241872019729) bank145566 +145567 POINT(42.026959092731964 -87.59631494125286) bank145567 +145568 POINT(42.469856811440785 -88.13981473321304) bank145568 +145569 POINT(42.63272839569487 -87.23878427416552) bank145569 +145570 POINT(41.523517592770474 -87.02092898122417) bank145570 +145571 POINT(41.57369223884427 -87.58939144800601) bank145571 +145572 POINT(41.510659992785925 -88.21175817232475) bank145572 +145573 POINT(42.65667595494915 -87.87702107031896) bank145573 +145574 POINT(41.48697823952771 -87.70278616175136) bank145574 +145575 POINT(41.756536018651104 -86.68206702727285) bank145575 +145576 POINT(41.19017579389869 -87.93158977473986) bank145576 +145577 POINT(42.193788704414594 -87.84944372390795) bank145577 +145578 POINT(42.8374824167869 -86.73049132059462) bank145578 +145579 POINT(42.33392561493033 -88.43672946177455) bank145579 +145580 POINT(41.537282914542814 -87.21382621824895) bank145580 +145581 POINT(41.22606567342952 -87.66699293843791) bank145581 +145582 POINT(40.9815325441365 -88.54396626990356) bank145582 +145583 POINT(41.103901130066205 -87.46351677651494) bank145583 +145584 POINT(41.191506557500354 -87.1076663596737) bank145584 +145585 POINT(42.692191311376504 -86.95635253920753) bank145585 +145586 POINT(42.24005330970391 -87.67379408246669) bank145586 +145587 POINT(42.058776612409275 -88.07017625159162) bank145587 +145588 POINT(41.43974985046636 -88.00285874939824) bank145588 +145589 POINT(42.68293176938892 -87.10417799304811) bank145589 +145590 POINT(41.70811773842664 -88.02720068075166) bank145590 +145591 POINT(41.83638681009075 -87.36174509151236) bank145591 +145592 POINT(41.865567699533514 -88.31020160662848) bank145592 +145593 POINT(41.24569469119356 -86.9892130422324) bank145593 +145594 POINT(41.245229641169864 -86.74414935397508) bank145594 +145595 POINT(40.971253414858914 -88.42415482834998) bank145595 +145596 POINT(41.834913414190495 -88.34926557425356) bank145596 +145597 POINT(41.76827387068875 -86.65659075778989) bank145597 +145598 POINT(42.84291511467846 -88.53755977661449) bank145598 +145599 POINT(41.66358094902956 -86.83904603869208) bank145599 +145600 POINT(41.24357754045621 -86.70214115791495) bank145600 +145601 POINT(41.488594086051734 -88.0443050003938) bank145601 +145602 POINT(41.494946737911 -88.43208931114873) bank145602 +145603 POINT(41.066335414814496 -87.46198220207032) bank145603 +145604 POINT(41.555751166095355 -86.7306447120995) bank145604 +145605 POINT(42.20171994941262 -87.20656131525215) bank145605 +145606 POINT(40.88006899471547 -86.87720965775834) bank145606 +145607 POINT(42.062137832448066 -87.09147835222903) bank145607 +145608 POINT(41.290217518241896 -86.82841257041443) bank145608 +145609 POINT(42.135090746543554 -87.93865497981228) bank145609 +145610 POINT(41.81740953338801 -86.7391833343008) bank145610 +145611 POINT(41.450376027543456 -87.93514404410067) bank145611 +145612 POINT(42.73447120518723 -87.83354080099917) bank145612 +145613 POINT(42.536333754053295 -88.28473622304645) bank145613 +145614 POINT(41.48155399459657 -87.7840132341956) bank145614 +145615 POINT(41.28281472150191 -87.19075426996746) bank145615 +145616 POINT(41.429948971383794 -88.4291252945764) bank145616 +145617 POINT(41.32200272958613 -86.76288440388494) bank145617 +145618 POINT(42.34907663739527 -88.44003344009816) bank145618 +145619 POINT(42.58208322230566 -87.37298475210243) bank145619 +145620 POINT(40.891979015251195 -87.91315925487254) bank145620 +145621 POINT(42.44818723514918 -86.80618144859659) bank145621 +145622 POINT(41.76556244684885 -87.64855514896675) bank145622 +145623 POINT(42.62576329956126 -87.47315185110253) bank145623 +145624 POINT(42.850660844819465 -87.79727299548468) bank145624 +145625 POINT(41.25581644249411 -87.00905539444608) bank145625 +145626 POINT(40.96930124177981 -88.0906036833641) bank145626 +145627 POINT(42.628214115998794 -86.93230960262517) bank145627 +145628 POINT(42.13852707112615 -87.9141860984468) bank145628 +145629 POINT(41.1581775426143 -87.6119192806856) bank145629 +145630 POINT(42.46163446080001 -88.04088059039485) bank145630 +145631 POINT(41.43176841381777 -88.41421628338875) bank145631 +145632 POINT(42.564211715945056 -86.83950059360312) bank145632 +145633 POINT(42.52199392740483 -87.95334214263825) bank145633 +145634 POINT(41.876831512479164 -88.49705165969283) bank145634 +145635 POINT(42.16838172931307 -87.15956882993336) bank145635 +145636 POINT(41.29393158480763 -87.30049553497591) bank145636 +145637 POINT(41.78953084940947 -88.45860578069555) bank145637 +145638 POINT(42.021380919510776 -87.79678478353263) bank145638 +145639 POINT(41.89364327964772 -87.22921710687515) bank145639 +145640 POINT(41.09065414151853 -87.13609193868888) bank145640 +145641 POINT(42.7106848942298 -88.56374870222811) bank145641 +145642 POINT(42.522636074214674 -87.71496719740433) bank145642 +145643 POINT(42.44740289076257 -87.21883578491402) bank145643 +145644 POINT(42.35992451849456 -87.81913004203292) bank145644 +145645 POINT(42.79801854106252 -86.83097867243742) bank145645 +145646 POINT(41.132311052995284 -86.63905111554729) bank145646 +145647 POINT(41.79189417233225 -86.78570408235441) bank145647 +145648 POINT(41.221047368800086 -86.7160048142445) bank145648 +145649 POINT(41.773228736845645 -88.08769313707859) bank145649 +145650 POINT(42.572314911672564 -88.23628137292185) bank145650 +145651 POINT(42.320520849124044 -87.10721372338641) bank145651 +145652 POINT(41.89003665801155 -86.76658233770469) bank145652 +145653 POINT(42.07488601083112 -86.94965921099269) bank145653 +145654 POINT(41.10030738311216 -88.59777290170452) bank145654 +145655 POINT(42.151211951616844 -88.17008770285875) bank145655 +145656 POINT(41.862623426225205 -87.96885327652754) bank145656 +145657 POINT(42.0500363876335 -87.20926009385873) bank145657 +145658 POINT(42.571003300906625 -88.15609527163892) bank145658 +145659 POINT(41.445551093107646 -88.54056530299255) bank145659 +145660 POINT(42.42773135320112 -86.63450867835616) bank145660 +145661 POINT(41.710571276921776 -87.59300211726233) bank145661 +145662 POINT(42.562028730636854 -88.6271572420172) bank145662 +145663 POINT(42.541847554119876 -88.33393871883494) bank145663 +145664 POINT(42.818829363925566 -88.26209156287464) bank145664 +145665 POINT(42.28228118162648 -88.34298640044018) bank145665 +145666 POINT(42.81259510363893 -88.54410832018779) bank145666 +145667 POINT(41.85101350980513 -87.90316278930891) bank145667 +145668 POINT(41.40288372976116 -88.34563366548666) bank145668 +145669 POINT(42.829236380790505 -88.41558664224851) bank145669 +145670 POINT(41.341065567759024 -86.99363113432064) bank145670 +145671 POINT(41.96560933682484 -88.19038972879336) bank145671 +145672 POINT(41.39014754414489 -88.03731157890972) bank145672 +145673 POINT(42.75619210382566 -86.8961171280059) bank145673 +145674 POINT(41.009249634518326 -88.61190398122585) bank145674 +145675 POINT(42.50895519059506 -87.51120728892161) bank145675 +145676 POINT(41.385758901292945 -88.46130529551414) bank145676 +145677 POINT(42.29735846802637 -88.34629870523348) bank145677 +145678 POINT(42.667865982301116 -88.4559528674606) bank145678 +145679 POINT(42.012929838981776 -88.26085527831789) bank145679 +145680 POINT(41.130862386077425 -87.12390377053718) bank145680 +145681 POINT(41.34989893084188 -87.12516276834384) bank145681 +145682 POINT(41.24057235280276 -88.22569071545762) bank145682 +145683 POINT(40.97762933358381 -87.06198217369271) bank145683 +145684 POINT(41.62327653388647 -87.97269443787283) bank145684 +145685 POINT(41.55951276151988 -87.43601011710666) bank145685 +145686 POINT(41.62899559770789 -88.51668951452888) bank145686 +145687 POINT(42.22127001413142 -87.24057055516222) bank145687 +145688 POINT(42.14790404973268 -88.3194161729011) bank145688 +145689 POINT(41.812410402679085 -87.84685803024077) bank145689 +145690 POINT(41.7955947073368 -87.74500211218762) bank145690 +145691 POINT(42.76763234641593 -86.91670439787333) bank145691 +145692 POINT(41.323945190783505 -88.39573699368508) bank145692 +145693 POINT(41.264421793062894 -88.154679181623) bank145693 +145694 POINT(42.871315575118444 -86.6852307081751) bank145694 +145695 POINT(42.03811152237517 -88.18588763098033) bank145695 +145696 POINT(41.907148870097814 -86.75517194079183) bank145696 +145697 POINT(42.25831457131973 -88.61290478008519) bank145697 +145698 POINT(41.51525117294039 -88.20460992755775) bank145698 +145699 POINT(41.32813419983074 -87.34150202574884) bank145699 +145700 POINT(41.56939992139408 -86.87976894680618) bank145700 +145701 POINT(42.02385413409329 -88.02437515950851) bank145701 +145702 POINT(41.066850836404086 -86.86845443754791) bank145702 +145703 POINT(42.36406063072001 -86.7294278485079) bank145703 +145704 POINT(41.335569231913304 -87.35826730922622) bank145704 +145705 POINT(41.83306838820698 -88.15518672397359) bank145705 +145706 POINT(41.019478053948845 -87.76579437399539) bank145706 +145707 POINT(42.078941951295555 -86.77000810076107) bank145707 +145708 POINT(42.42367350651686 -87.49200968304753) bank145708 +145709 POINT(42.812126256336235 -86.82796355199307) bank145709 +145710 POINT(40.914349979521376 -87.24065969815237) bank145710 +145711 POINT(42.00539799669678 -87.21353387260444) bank145711 +145712 POINT(41.87837261547117 -87.68673034709362) bank145712 +145713 POINT(42.0781632638211 -87.48445021353999) bank145713 +145714 POINT(41.24555169052776 -88.04286716306403) bank145714 +145715 POINT(41.73389560846697 -87.16856204450511) bank145715 +145716 POINT(42.68672201540515 -88.3141141092045) bank145716 +145717 POINT(41.34989025624411 -88.01556657036798) bank145717 +145718 POINT(41.51447038139727 -87.1777864935948) bank145718 +145719 POINT(42.454512053034094 -87.86925562038641) bank145719 +145720 POINT(42.24374357497835 -87.03707009441895) bank145720 +145721 POINT(42.81879268868755 -87.33546501897334) bank145721 +145722 POINT(42.79994772262612 -87.32649096067703) bank145722 +145723 POINT(41.27250306803347 -87.59771222321001) bank145723 +145724 POINT(41.0621788859915 -86.82391751328781) bank145724 +145725 POINT(42.33034681478509 -88.4644776757958) bank145725 +145726 POINT(41.682800551106986 -86.82458149755875) bank145726 +145727 POINT(42.10464816163728 -87.88390159326363) bank145727 +145728 POINT(41.40787817196791 -88.03531744829624) bank145728 +145729 POINT(41.86013270174492 -88.36214023126907) bank145729 +145730 POINT(42.02730034260688 -86.83057836771937) bank145730 +145731 POINT(42.774217404900256 -87.28798244752336) bank145731 +145732 POINT(41.54387267859403 -86.71759108443196) bank145732 +145733 POINT(42.46852383008654 -86.99036401048191) bank145733 +145734 POINT(41.79327618239662 -88.09390474032068) bank145734 +145735 POINT(42.4587979386199 -88.21549482236635) bank145735 +145736 POINT(42.14860563735082 -87.49424282631385) bank145736 +145737 POINT(42.02967627788493 -87.83366215074507) bank145737 +145738 POINT(42.08546710810371 -87.26663915444695) bank145738 +145739 POINT(41.02980946352395 -87.36674159760597) bank145739 +145740 POINT(42.12244128740767 -87.62685503428273) bank145740 +145741 POINT(41.033025534858716 -87.20424603773185) bank145741 +145742 POINT(42.01974339969084 -88.44299647032469) bank145742 +145743 POINT(42.05309556815778 -87.96532570472935) bank145743 +145744 POINT(42.146118489193206 -87.18524959864008) bank145744 +145745 POINT(41.48320464962489 -87.30326656338619) bank145745 +145746 POINT(41.5921610014345 -86.64995339041306) bank145746 +145747 POINT(42.65673444581938 -88.24144988102323) bank145747 +145748 POINT(42.1548608587702 -87.04994253882687) bank145748 +145749 POINT(41.040342525272024 -87.24955975591743) bank145749 +145750 POINT(42.08344283424452 -86.87709133051277) bank145750 +145751 POINT(41.71994183368051 -87.59334158348321) bank145751 +145752 POINT(41.436074328254705 -88.14024199034006) bank145752 +145753 POINT(42.65562274442817 -87.5870358772991) bank145753 +145754 POINT(41.86414461039089 -87.37803668134224) bank145754 +145755 POINT(41.44246727367014 -87.54544634321444) bank145755 +145756 POINT(42.26095914324808 -87.63746845713264) bank145756 +145757 POINT(41.52648429919677 -88.00289677786344) bank145757 +145758 POINT(41.74450678788044 -86.85616029942179) bank145758 +145759 POINT(41.31476143441162 -87.62993653237248) bank145759 +145760 POINT(41.71283004727511 -88.23481005894766) bank145760 +145761 POINT(41.256145408322325 -88.060837983907) bank145761 +145762 POINT(41.12273321059553 -87.31245625035544) bank145762 +145763 POINT(42.49542546103522 -88.10008398980837) bank145763 +145764 POINT(41.27994479481097 -87.01309693881457) bank145764 +145765 POINT(41.633828194675736 -88.1364509701392) bank145765 +145766 POINT(41.20125710903794 -88.43939336063586) bank145766 +145767 POINT(40.90486649393113 -88.1395103074807) bank145767 +145768 POINT(42.038208315250856 -88.13391922561335) bank145768 +145769 POINT(40.953165324709744 -87.670695757314) bank145769 +145770 POINT(42.56284854029129 -87.94414469423222) bank145770 +145771 POINT(41.09971069856704 -87.07582438131082) bank145771 +145772 POINT(41.75544863310344 -87.20865516791946) bank145772 +145773 POINT(41.57976087798092 -87.27905870359038) bank145773 +145774 POINT(42.40483647817263 -86.79068284240343) bank145774 +145775 POINT(41.61734268270986 -86.67084717496557) bank145775 +145776 POINT(42.40246830090896 -87.91993355622901) bank145776 +145777 POINT(42.464178470673815 -87.65509231155535) bank145777 +145778 POINT(41.593645857743624 -87.70406333758892) bank145778 +145779 POINT(42.72005527061172 -87.23356810189047) bank145779 +145780 POINT(41.26205942756704 -87.65327793417448) bank145780 +145781 POINT(42.83810132714823 -87.60329550824598) bank145781 +145782 POINT(42.213315760922654 -88.35138740419148) bank145782 +145783 POINT(41.51169465111316 -88.57878844491208) bank145783 +145784 POINT(41.69449891727878 -88.61468634885718) bank145784 +145785 POINT(42.29449832674269 -86.67775982812383) bank145785 +145786 POINT(41.07836216278364 -88.30824320068541) bank145786 +145787 POINT(42.42685927148959 -86.96024060344334) bank145787 +145788 POINT(41.838267102198316 -87.89629357322916) bank145788 +145789 POINT(42.8320048205687 -88.15753874444486) bank145789 +145790 POINT(42.75945434065166 -87.0769603694839) bank145790 +145791 POINT(42.220697090990015 -88.26638631415568) bank145791 +145792 POINT(41.029136401105895 -86.7124848869546) bank145792 +145793 POINT(42.868222608146475 -87.04458149590174) bank145793 +145794 POINT(42.79257823201357 -87.47873347061137) bank145794 +145795 POINT(41.37200814836487 -87.10120383064398) bank145795 +145796 POINT(41.80903550865797 -87.6132111325646) bank145796 +145797 POINT(41.23575651328359 -87.72202223427027) bank145797 +145798 POINT(41.10636541473181 -87.75353671071755) bank145798 +145799 POINT(42.5637197595232 -87.17368836277203) bank145799 +145800 POINT(41.60024584824522 -86.66110371701329) bank145800 +145801 POINT(42.27476179211515 -87.20250361268113) bank145801 +145802 POINT(42.573808018984124 -86.7172002054295) bank145802 +145803 POINT(42.83400839471147 -88.35043505756768) bank145803 +145804 POINT(41.23022148660233 -87.37179515281706) bank145804 +145805 POINT(41.3947335292624 -87.69776933752495) bank145805 +145806 POINT(42.12758614037939 -88.26236819018996) bank145806 +145807 POINT(41.20985314018498 -87.40971553562395) bank145807 +145808 POINT(41.45881820919306 -88.4662888995233) bank145808 +145809 POINT(42.01937794616203 -87.51543391037698) bank145809 +145810 POINT(42.80764414547619 -87.76749011092011) bank145810 +145811 POINT(41.666047734764405 -88.49711068112335) bank145811 +145812 POINT(42.618421036156796 -87.01906235787018) bank145812 +145813 POINT(41.637968127290456 -87.71977289373973) bank145813 +145814 POINT(42.31159351500898 -87.38265783258701) bank145814 +145815 POINT(41.834586809181715 -86.97863370488807) bank145815 +145816 POINT(42.82410186493565 -86.98256890831712) bank145816 +145817 POINT(42.1768929681965 -87.58838069051289) bank145817 +145818 POINT(42.550195850274065 -88.43311613104213) bank145818 +145819 POINT(41.66258787145825 -87.17549411626219) bank145819 +145820 POINT(42.486038344276096 -87.25844713960065) bank145820 +145821 POINT(41.101691085003885 -86.8920808301386) bank145821 +145822 POINT(42.31824085679062 -86.73916142269901) bank145822 +145823 POINT(42.602334039642756 -87.26498151443596) bank145823 +145824 POINT(41.538322509625324 -87.72696122530228) bank145824 +145825 POINT(42.016442546811646 -88.51989594280596) bank145825 +145826 POINT(42.87377322043081 -86.93617105860672) bank145826 +145827 POINT(41.80771888206314 -87.64003293832877) bank145827 +145828 POINT(41.036166391989596 -87.67278367807411) bank145828 +145829 POINT(42.1096672677137 -87.66198886095194) bank145829 +145830 POINT(42.68555628325309 -87.89156778091798) bank145830 +145831 POINT(41.71770931187512 -87.28995630619212) bank145831 +145832 POINT(42.563814340473115 -87.19008170460604) bank145832 +145833 POINT(41.132907536068956 -87.87473085188125) bank145833 +145834 POINT(42.6245828270112 -87.62164597760383) bank145834 +145835 POINT(42.576533001341126 -88.59665563010994) bank145835 +145836 POINT(40.989295595661574 -87.63132629321409) bank145836 +145837 POINT(42.58535145260163 -88.591491850813) bank145837 +145838 POINT(42.044581202172154 -87.96007710017278) bank145838 +145839 POINT(42.5077831708104 -88.14992508817701) bank145839 +145840 POINT(42.16192144226632 -87.9565052387168) bank145840 +145841 POINT(42.46237660504878 -88.29956907020167) bank145841 +145842 POINT(41.013557848061325 -87.65196501864646) bank145842 +145843 POINT(41.36486997327798 -88.6062872252447) bank145843 +145844 POINT(42.784467559126654 -87.2989202981602) bank145844 +145845 POINT(42.14199836168937 -86.99273801229053) bank145845 +145846 POINT(41.486153128317 -87.8710652473322) bank145846 +145847 POINT(42.61538329968861 -88.04200843866435) bank145847 +145848 POINT(41.48045446183493 -86.63126811875956) bank145848 +145849 POINT(42.16510137141109 -88.27264333563747) bank145849 +145850 POINT(42.22860887484048 -88.330147481813) bank145850 +145851 POINT(42.81428916314286 -87.9816218951474) bank145851 +145852 POINT(41.953431203215 -87.59657871692299) bank145852 +145853 POINT(41.502623762952524 -88.50815383881613) bank145853 +145854 POINT(42.58671210482918 -87.44070812297076) bank145854 +145855 POINT(42.47081108461154 -86.79071965761486) bank145855 +145856 POINT(42.82025232803111 -87.81545202096343) bank145856 +145857 POINT(42.2548660769683 -87.42874194377325) bank145857 +145858 POINT(41.61855262207753 -87.43377278793155) bank145858 +145859 POINT(40.968094384143626 -87.89558046571105) bank145859 +145860 POINT(41.86856098574253 -87.29705834408844) bank145860 +145861 POINT(41.5448411265246 -88.094948297068) bank145861 +145862 POINT(41.07953933317229 -87.51926968570311) bank145862 +145863 POINT(42.622419219843394 -87.5658285123526) bank145863 +145864 POINT(41.33134477239253 -88.32385894287263) bank145864 +145865 POINT(41.64117113593986 -87.24014562432403) bank145865 +145866 POINT(40.91579405165993 -87.73267135329276) bank145866 +145867 POINT(41.69081571097223 -87.36906706115462) bank145867 +145868 POINT(41.851477061860436 -88.55098979376486) bank145868 +145869 POINT(42.62476644245512 -87.07271898245577) bank145869 +145870 POINT(42.13036722861089 -87.61378808182329) bank145870 +145871 POINT(42.620510040864666 -86.83839903507571) bank145871 +145872 POINT(41.520384168539415 -88.28042725397236) bank145872 +145873 POINT(42.7826609850552 -88.16026390207197) bank145873 +145874 POINT(41.649373485821485 -87.27711905711325) bank145874 +145875 POINT(42.09357616076938 -86.9044475701422) bank145875 +145876 POINT(41.51368624693289 -87.59070932446024) bank145876 +145877 POINT(40.9361017951646 -86.7954288140982) bank145877 +145878 POINT(41.49010538460717 -87.54280250054325) bank145878 +145879 POINT(41.531307860710136 -87.3126780990026) bank145879 +145880 POINT(41.228839477989354 -87.63042151578186) bank145880 +145881 POINT(42.22472646040948 -87.21380955539942) bank145881 +145882 POINT(41.91339277995303 -87.23680909799998) bank145882 +145883 POINT(41.19497568432043 -87.45842984907948) bank145883 +145884 POINT(42.261564878406965 -86.68315793842507) bank145884 +145885 POINT(40.91109237763959 -87.20184575467637) bank145885 +145886 POINT(42.40888822565723 -88.46426089273201) bank145886 +145887 POINT(42.13884234134392 -87.70808001812094) bank145887 +145888 POINT(41.161240877628 -88.51426558124949) bank145888 +145889 POINT(41.28998475902997 -87.1023569773834) bank145889 +145890 POINT(41.62357168201498 -87.44074158755924) bank145890 +145891 POINT(41.44673668767646 -87.93061033410767) bank145891 +145892 POINT(41.255059930741936 -87.93273021974161) bank145892 +145893 POINT(41.77189468690405 -87.54135278904421) bank145893 +145894 POINT(42.619171470229205 -87.92641667729427) bank145894 +145895 POINT(42.40350847134692 -88.3887002686504) bank145895 +145896 POINT(41.88530027244416 -86.64196092081005) bank145896 +145897 POINT(41.99297718600464 -86.72301742258522) bank145897 +145898 POINT(41.03604838292308 -87.58089546536662) bank145898 +145899 POINT(42.039511022267114 -87.94570644601816) bank145899 +145900 POINT(41.95848485624973 -87.48716742803103) bank145900 +145901 POINT(41.60199398743502 -88.37971554151731) bank145901 +145902 POINT(41.34618446190038 -87.40688118592381) bank145902 +145903 POINT(42.1504628700727 -86.75481754425483) bank145903 +145904 POINT(41.43628855399126 -87.1352276886502) bank145904 +145905 POINT(41.567688384101366 -88.48293639790138) bank145905 +145906 POINT(41.646173576301834 -87.1877601180008) bank145906 +145907 POINT(41.94397421260762 -87.29539047568583) bank145907 +145908 POINT(42.21086474210467 -86.91193898790546) bank145908 +145909 POINT(40.95055617273519 -87.05448803372529) bank145909 +145910 POINT(41.04655930369307 -88.17805589364234) bank145910 +145911 POINT(41.25922982855081 -87.16872501787765) bank145911 +145912 POINT(42.050781962511714 -88.51655037313789) bank145912 +145913 POINT(42.711387853755376 -88.06215288007114) bank145913 +145914 POINT(42.797328047623374 -87.65147664987299) bank145914 +145915 POINT(42.50675979060262 -87.16349489707633) bank145915 +145916 POINT(42.60948076571421 -88.46496316178441) bank145916 +145917 POINT(41.019444789128634 -87.66737647778268) bank145917 +145918 POINT(41.77531178447357 -88.47151618705759) bank145918 +145919 POINT(41.50278543000439 -86.83350626743807) bank145919 +145920 POINT(42.317530881768306 -86.97392182064112) bank145920 +145921 POINT(42.02125655031307 -87.6800939773407) bank145921 +145922 POINT(41.4257612181177 -88.3273617032317) bank145922 +145923 POINT(41.02158276103002 -87.08459588224815) bank145923 +145924 POINT(41.001894566769046 -86.6306050678441) bank145924 +145925 POINT(41.603355487173594 -88.16952341406282) bank145925 +145926 POINT(41.62790904236474 -87.90083765710342) bank145926 +145927 POINT(42.58784751100525 -87.77124364248309) bank145927 +145928 POINT(41.55458530475682 -88.4104370424334) bank145928 +145929 POINT(41.14286513044364 -87.12723994959994) bank145929 +145930 POINT(42.44180934639926 -87.96214772298806) bank145930 +145931 POINT(42.03326285977957 -87.6542539912167) bank145931 +145932 POINT(41.83268277276896 -87.35903861527405) bank145932 +145933 POINT(42.58078727699839 -86.74228034252513) bank145933 +145934 POINT(41.775685697955666 -87.13694648721957) bank145934 +145935 POINT(42.69101252396144 -88.46127104311611) bank145935 +145936 POINT(42.166006750292055 -87.00015391227699) bank145936 +145937 POINT(41.79941278895342 -87.12782341046282) bank145937 +145938 POINT(41.62859354180061 -88.2755324185942) bank145938 +145939 POINT(42.02755650455532 -88.1270623982877) bank145939 +145940 POINT(42.2888291577423 -87.79102631964247) bank145940 +145941 POINT(42.74693197062346 -86.66646051005502) bank145941 +145942 POINT(42.848515442808846 -88.48945918183533) bank145942 +145943 POINT(41.3134981310845 -88.14323670330155) bank145943 +145944 POINT(42.59307949406396 -87.65939325961516) bank145944 +145945 POINT(42.7284381950895 -87.17242119687194) bank145945 +145946 POINT(41.4897233213342 -86.74685558238475) bank145946 +145947 POINT(41.58613688386498 -88.47234732890692) bank145947 +145948 POINT(42.19978070906819 -87.05084646454675) bank145948 +145949 POINT(41.52531360089683 -86.86382891293421) bank145949 +145950 POINT(41.82307577174563 -86.66753720403563) bank145950 +145951 POINT(41.62661327666759 -88.32368487972178) bank145951 +145952 POINT(41.843465531925006 -88.53748305330804) bank145952 +145953 POINT(42.24078413293928 -87.5278458430573) bank145953 +145954 POINT(41.86553538117644 -86.91947649375034) bank145954 +145955 POINT(41.42924763320714 -87.75342071364737) bank145955 +145956 POINT(41.41263749124814 -87.04885992111605) bank145956 +145957 POINT(42.08045322457464 -87.00562691639827) bank145957 +145958 POINT(42.828368448927506 -86.68338204105558) bank145958 +145959 POINT(41.89824659483041 -86.70528664262142) bank145959 +145960 POINT(41.46381367882544 -86.80609316759944) bank145960 +145961 POINT(40.96164144904792 -88.03428776372907) bank145961 +145962 POINT(41.597372731509715 -87.46663922198634) bank145962 +145963 POINT(41.07812177590338 -88.34439166902976) bank145963 +145964 POINT(41.532836826497274 -87.38419993749474) bank145964 +145965 POINT(41.245764886095216 -88.33143369015308) bank145965 +145966 POINT(41.628486567336395 -88.33423323051899) bank145966 +145967 POINT(41.1038812740822 -88.32512513974945) bank145967 +145968 POINT(42.47739071327283 -87.88986777089072) bank145968 +145969 POINT(42.231522734673284 -87.15139225228448) bank145969 +145970 POINT(42.450453947776474 -86.63159986088172) bank145970 +145971 POINT(42.27566189761445 -88.37526402761745) bank145971 +145972 POINT(41.95624173425173 -87.35331593512154) bank145972 +145973 POINT(41.6605638319991 -88.22379932455506) bank145973 +145974 POINT(42.61698225295456 -87.58310806584747) bank145974 +145975 POINT(42.72657469074424 -88.1525999494276) bank145975 +145976 POINT(41.40088771510301 -87.92978803339847) bank145976 +145977 POINT(42.299738055846994 -87.47421682648184) bank145977 +145978 POINT(42.660565126499684 -88.12441162304584) bank145978 +145979 POINT(42.09453755636381 -87.26237323471015) bank145979 +145980 POINT(41.05598453762479 -88.25580840856577) bank145980 +145981 POINT(42.87322211433275 -86.6713275955248) bank145981 +145982 POINT(42.70130197346615 -87.86511916521462) bank145982 +145983 POINT(42.55573774277252 -87.46266474024178) bank145983 +145984 POINT(41.58001619238131 -88.46768523477368) bank145984 +145985 POINT(41.08197638253163 -88.19655280227137) bank145985 +145986 POINT(41.63932914967241 -87.81382462911695) bank145986 +145987 POINT(42.346481921421955 -88.38909288762981) bank145987 +145988 POINT(42.26642496515968 -87.88453676667476) bank145988 +145989 POINT(41.063373730179 -86.74855795660264) bank145989 +145990 POINT(42.543589686928314 -88.18156260769305) bank145990 +145991 POINT(41.3513181991597 -87.29087504593566) bank145991 +145992 POINT(41.20358125762679 -86.71645164225525) bank145992 +145993 POINT(42.65353779177154 -87.45292880957508) bank145993 +145994 POINT(42.15863921855149 -87.38598730032734) bank145994 +145995 POINT(41.42583986901033 -87.75160571709426) bank145995 +145996 POINT(40.92801086418952 -87.17627900109541) bank145996 +145997 POINT(41.090650274736674 -86.82382634478398) bank145997 +145998 POINT(42.75985436239815 -87.65234817902895) bank145998 +145999 POINT(42.01863298533241 -87.07471083983674) bank145999 +146000 POINT(41.1459642131549 -87.04563682232532) bank146000 +146001 POINT(42.459750555986545 -87.22601290228975) bank146001 +146002 POINT(42.69324423701618 -86.77533815785954) bank146002 +146003 POINT(41.29623730049056 -88.42204950969547) bank146003 +146004 POINT(42.55501774690749 -87.91738372455481) bank146004 +146005 POINT(41.19353386029574 -87.65811179762879) bank146005 +146006 POINT(41.4475800711155 -88.20882367559982) bank146006 +146007 POINT(41.73384507224725 -87.2724985079076) bank146007 +146008 POINT(42.75320562044389 -87.19227039069217) bank146008 +146009 POINT(41.47746961853256 -88.49168197271261) bank146009 +146010 POINT(41.880858183380894 -88.43432482973404) bank146010 +146011 POINT(42.704836826936756 -87.82025045244843) bank146011 +146012 POINT(41.41583354017013 -87.70568235434796) bank146012 +146013 POINT(41.947702713881434 -86.96191520729954) bank146013 +146014 POINT(41.918952778324694 -87.69404238989583) bank146014 +146015 POINT(41.15205841762238 -86.73206908183842) bank146015 +146016 POINT(42.11637757659852 -88.35045189794609) bank146016 +146017 POINT(42.6374354745498 -86.72673385626757) bank146017 +146018 POINT(42.23051781637171 -86.98257213172906) bank146018 +146019 POINT(41.2897367309821 -87.85760128391365) bank146019 +146020 POINT(41.619101207485414 -88.15918905988646) bank146020 +146021 POINT(41.01916573368138 -87.97535498904693) bank146021 +146022 POINT(42.333791422789794 -88.19958148379189) bank146022 +146023 POINT(41.205148213880314 -86.70026223544467) bank146023 +146024 POINT(42.56779305741072 -86.96249909774039) bank146024 +146025 POINT(42.21050404301199 -88.03933404573024) bank146025 +146026 POINT(42.50742240052205 -86.92903778174137) bank146026 +146027 POINT(41.344807554166536 -87.9253607564296) bank146027 +146028 POINT(41.747490051507384 -88.25686189894955) bank146028 +146029 POINT(42.4536960199334 -87.29617835509657) bank146029 +146030 POINT(42.011966519455044 -88.33806306470274) bank146030 +146031 POINT(41.79797011044878 -86.74778398911351) bank146031 +146032 POINT(41.20002238862601 -87.2284887566354) bank146032 +146033 POINT(41.20964084958378 -86.6639228073874) bank146033 +146034 POINT(41.58573414033144 -87.67394528115085) bank146034 +146035 POINT(42.55752152276817 -86.9341277487499) bank146035 +146036 POINT(40.898806418870855 -87.26165864343763) bank146036 +146037 POINT(42.239183343284 -86.66659248900545) bank146037 +146038 POINT(42.3173882101911 -87.17807183335162) bank146038 +146039 POINT(42.673825686526285 -87.41835199325578) bank146039 +146040 POINT(42.029284330915814 -87.9836989460582) bank146040 +146041 POINT(41.43005141389029 -87.15786492314452) bank146041 +146042 POINT(41.02139332636085 -87.28977312882297) bank146042 +146043 POINT(42.38325925926465 -88.29182169172677) bank146043 +146044 POINT(42.46466581070862 -87.62601116355748) bank146044 +146045 POINT(42.25734966396522 -88.27869438647173) bank146045 +146046 POINT(42.11398648342071 -87.10619016320393) bank146046 +146047 POINT(42.587718401606 -88.55549039985252) bank146047 +146048 POINT(40.93681144800666 -86.66665169190772) bank146048 +146049 POINT(42.24393809719226 -88.0020274634573) bank146049 +146050 POINT(42.04416603674821 -88.12989620865095) bank146050 +146051 POINT(42.238916049906756 -87.47673170779555) bank146051 +146052 POINT(40.971638146143846 -87.56635785698853) bank146052 +146053 POINT(42.87206271594323 -86.76830397395395) bank146053 +146054 POINT(41.769731485004435 -88.03652864149475) bank146054 +146055 POINT(41.11759599480269 -87.4922339441179) bank146055 +146056 POINT(42.552158038484905 -87.56771788146986) bank146056 +146057 POINT(42.040518812524205 -86.70812575911606) bank146057 +146058 POINT(42.58524708062252 -86.82908906775282) bank146058 +146059 POINT(41.301144547782876 -87.94614954267855) bank146059 +146060 POINT(42.467743135162365 -86.66947099043234) bank146060 +146061 POINT(42.098824799766405 -88.5231931898819) bank146061 +146062 POINT(42.219342698197764 -88.55858140426119) bank146062 +146063 POINT(41.84947803540688 -87.71141571665342) bank146063 +146064 POINT(41.56312041285629 -86.70097170950295) bank146064 +146065 POINT(42.77786975131141 -87.2769088445851) bank146065 +146066 POINT(42.0652523672443 -88.15988834223188) bank146066 +146067 POINT(42.182422555344786 -87.19416799563227) bank146067 +146068 POINT(41.94426303780386 -87.57395522722449) bank146068 +146069 POINT(42.723674706525074 -88.28393074590672) bank146069 +146070 POINT(42.23848511735937 -88.15931343360056) bank146070 +146071 POINT(41.99286248963772 -86.67416592910416) bank146071 +146072 POINT(42.50639341279746 -86.92498706018489) bank146072 +146073 POINT(41.59207283346694 -87.41506887954314) bank146073 +146074 POINT(41.62502222708297 -86.88577562721211) bank146074 +146075 POINT(41.33579090505113 -87.38268620421537) bank146075 +146076 POINT(40.92427528955291 -87.08804418985382) bank146076 +146077 POINT(40.98795686858233 -86.72481405784332) bank146077 +146078 POINT(41.08532184434485 -88.30440085709148) bank146078 +146079 POINT(42.85948036372959 -87.93983182735695) bank146079 +146080 POINT(42.52183909665792 -87.62536941080468) bank146080 +146081 POINT(41.36495401074444 -87.25816050798284) bank146081 +146082 POINT(41.36115263388031 -88.00083602629364) bank146082 +146083 POINT(41.07612517368195 -88.25519073290346) bank146083 +146084 POINT(42.199191718436026 -88.3777101247075) bank146084 +146085 POINT(42.73615635733252 -88.32236691245024) bank146085 +146086 POINT(42.26672828036969 -87.34920535735156) bank146086 +146087 POINT(40.98601468575673 -88.46338985399477) bank146087 +146088 POINT(41.91226150887111 -88.25465196634156) bank146088 +146089 POINT(42.68541208310094 -87.4507117387087) bank146089 +146090 POINT(42.620431920454436 -87.02953663621591) bank146090 +146091 POINT(41.79386318660063 -87.22675671222558) bank146091 +146092 POINT(41.519775948344964 -88.15104564015793) bank146092 +146093 POINT(41.722002410546246 -87.26118309984125) bank146093 +146094 POINT(41.13176151496655 -87.28013600071235) bank146094 +146095 POINT(41.375244186166185 -87.44650147562727) bank146095 +146096 POINT(40.90186800493736 -88.53154528709752) bank146096 +146097 POINT(42.30638047332987 -86.855225489603) bank146097 +146098 POINT(42.81068629889918 -87.835580651581) bank146098 +146099 POINT(41.01741713998378 -88.0395499502848) bank146099 +146100 POINT(41.08973213351473 -86.82908622436906) bank146100 +146101 POINT(42.44008340192145 -87.28424455992064) bank146101 +146102 POINT(41.9083330004088 -88.15049580569077) bank146102 +146103 POINT(42.587376785680156 -86.93654429175767) bank146103 +146104 POINT(42.122449899826016 -87.70044252476933) bank146104 +146105 POINT(42.08422227763256 -87.12151268638048) bank146105 +146106 POINT(42.74111497311656 -88.54938102145407) bank146106 +146107 POINT(42.62772340974903 -88.57909851840128) bank146107 +146108 POINT(40.89899277938532 -86.96067584681056) bank146108 +146109 POINT(42.57749540969208 -87.31184722591752) bank146109 +146110 POINT(42.03046461120119 -87.16698572790877) bank146110 +146111 POINT(42.59490775730034 -87.34759564142429) bank146111 +146112 POINT(42.591291038112885 -86.81879319654828) bank146112 +146113 POINT(41.31220122835662 -87.42698746288686) bank146113 +146114 POINT(41.094742497597196 -88.59370104978846) bank146114 +146115 POINT(42.579715433028746 -88.0840129461852) bank146115 +146116 POINT(42.60373449344536 -88.02542163791226) bank146116 +146117 POINT(41.661294525437405 -88.62162676480963) bank146117 +146118 POINT(41.106935730042096 -87.89386761199222) bank146118 +146119 POINT(41.5952689459349 -88.58216918579598) bank146119 +146120 POINT(41.38514801294205 -88.53074621337699) bank146120 +146121 POINT(42.80141444426513 -88.62225910831745) bank146121 +146122 POINT(41.98941703937572 -87.06609228473947) bank146122 +146123 POINT(41.53243264098711 -88.00557974368688) bank146123 +146124 POINT(41.544106750929515 -87.42036247321478) bank146124 +146125 POINT(41.110464993933036 -87.12183869103318) bank146125 +146126 POINT(41.960643697262235 -87.12150140253735) bank146126 +146127 POINT(41.51151861710144 -87.38013375775819) bank146127 +146128 POINT(42.779897720039706 -88.29500739535388) bank146128 +146129 POINT(42.79932787321559 -88.34324427439826) bank146129 +146130 POINT(41.52070163314583 -87.76762383668506) bank146130 +146131 POINT(41.185768387284185 -88.37581515691228) bank146131 +146132 POINT(42.45030646147562 -87.65012087994879) bank146132 +146133 POINT(41.899316169601846 -86.88143426781475) bank146133 +146134 POINT(42.0417765183232 -87.93681302455886) bank146134 +146135 POINT(41.92571255919077 -86.77968563912303) bank146135 +146136 POINT(41.43414246046528 -87.95946213062669) bank146136 +146137 POINT(42.263838227539786 -86.72328909576385) bank146137 +146138 POINT(41.08772673081316 -87.59061324168633) bank146138 +146139 POINT(42.56431667997036 -88.57297057677252) bank146139 +146140 POINT(42.85117572888416 -88.433641716592) bank146140 +146141 POINT(41.496875606595914 -88.04113746525253) bank146141 +146142 POINT(41.18779758436225 -87.66105528254322) bank146142 +146143 POINT(41.29402716953116 -86.70756366358661) bank146143 +146144 POINT(40.985140880345654 -86.80951458852907) bank146144 +146145 POINT(42.064453684408385 -86.66605999797345) bank146145 +146146 POINT(42.61745589535219 -87.68633382420298) bank146146 +146147 POINT(42.65224906492567 -87.45063102953836) bank146147 +146148 POINT(42.606676118001815 -88.23843042490496) bank146148 +146149 POINT(42.017331132684035 -88.07277655314832) bank146149 +146150 POINT(42.04528565880831 -87.19071420394611) bank146150 +146151 POINT(42.759351510662455 -88.57822732574019) bank146151 +146152 POINT(41.32795169457716 -88.40871358583507) bank146152 +146153 POINT(41.982871362949055 -88.38541143517462) bank146153 +146154 POINT(42.480200871517745 -87.92836065949722) bank146154 +146155 POINT(42.08631471997382 -87.76764654566625) bank146155 +146156 POINT(41.1839282467023 -87.2385849121921) bank146156 +146157 POINT(42.164055440790335 -87.30704052726998) bank146157 +146158 POINT(41.36336302731437 -86.83256285252396) bank146158 +146159 POINT(42.226703606973324 -88.42411579935501) bank146159 +146160 POINT(41.11778318753589 -88.24118314065726) bank146160 +146161 POINT(42.22054034821273 -88.1203604349938) bank146161 +146162 POINT(42.3290851490163 -86.77911302466293) bank146162 +146163 POINT(41.32641565047631 -88.13274195418522) bank146163 +146164 POINT(40.95485712580595 -87.75628514526859) bank146164 +146165 POINT(41.179768018216244 -88.14867787214021) bank146165 +146166 POINT(42.21662719003635 -88.57386446930764) bank146166 +146167 POINT(41.84663199375458 -87.44370162078317) bank146167 +146168 POINT(41.422822926799405 -87.45281747518341) bank146168 +146169 POINT(41.286277468626565 -88.20743834487719) bank146169 +146170 POINT(41.97783342822545 -87.67711971501765) bank146170 +146171 POINT(41.83429114462322 -86.73388663346667) bank146171 +146172 POINT(42.079026928142014 -87.7536426457892) bank146172 +146173 POINT(41.48044393433806 -87.15812479341243) bank146173 +146174 POINT(41.84304348846093 -88.48964415559955) bank146174 +146175 POINT(42.76153642609599 -88.24663547421288) bank146175 +146176 POINT(41.120348221319375 -87.14702275622238) bank146176 +146177 POINT(42.40444836471282 -87.21467420249343) bank146177 +146178 POINT(41.47030046499488 -88.49323028945257) bank146178 +146179 POINT(41.920196206748045 -88.48222425443791) bank146179 +146180 POINT(42.27353855409656 -86.78954452278076) bank146180 +146181 POINT(42.482425881795024 -87.62043502015226) bank146181 +146182 POINT(42.850530324579864 -88.00047539093086) bank146182 +146183 POINT(41.35911942340523 -88.49211995075368) bank146183 +146184 POINT(42.61647347848031 -87.54462742941345) bank146184 +146185 POINT(41.94260059254138 -87.08909019564139) bank146185 +146186 POINT(42.257943441582235 -87.57652424982582) bank146186 +146187 POINT(41.813774253981045 -87.06267203851061) bank146187 +146188 POINT(42.86906808045231 -87.62284168331986) bank146188 +146189 POINT(42.80334296277488 -87.28746145325748) bank146189 +146190 POINT(42.31968072302182 -87.80429941681967) bank146190 +146191 POINT(42.13633521830688 -87.37854762944711) bank146191 +146192 POINT(41.175135387592626 -87.6831830766205) bank146192 +146193 POINT(42.0456298320976 -86.68855851663439) bank146193 +146194 POINT(42.70907312381651 -88.15196680593773) bank146194 +146195 POINT(41.77293660247479 -88.13440127915584) bank146195 +146196 POINT(42.54405079625736 -87.00333966038953) bank146196 +146197 POINT(41.9347260754495 -87.8432144072765) bank146197 +146198 POINT(41.115921507554724 -86.85089700112565) bank146198 +146199 POINT(42.862116388202715 -87.1474274912748) bank146199 +146200 POINT(41.60724292314945 -86.71905657069728) bank146200 +146201 POINT(41.002939109837584 -87.76614056943951) bank146201 +146202 POINT(42.581207497811384 -88.3101845653975) bank146202 +146203 POINT(41.9980911658284 -87.29588709985629) bank146203 +146204 POINT(40.93403713336993 -87.4458324991385) bank146204 +146205 POINT(41.91880038443765 -88.14544981443905) bank146205 +146206 POINT(42.717414449815095 -88.04156406625204) bank146206 +146207 POINT(42.6354432633834 -87.78017479064216) bank146207 +146208 POINT(42.84771823191625 -87.0444780924346) bank146208 +146209 POINT(41.363975014979935 -88.50992881379773) bank146209 +146210 POINT(41.4106894393727 -87.99426290244926) bank146210 +146211 POINT(41.70990757184652 -86.87604801888841) bank146211 +146212 POINT(41.82304966500142 -88.11912624004432) bank146212 +146213 POINT(41.05800747615475 -87.50725231481933) bank146213 +146214 POINT(42.07655501378026 -87.57511475969204) bank146214 +146215 POINT(42.85582030055966 -87.16088016459084) bank146215 +146216 POINT(41.73482947991278 -88.19335572690845) bank146216 +146217 POINT(42.81654053724297 -87.12096921995084) bank146217 +146218 POINT(41.54148446202015 -87.81033723485105) bank146218 +146219 POINT(42.507499777108265 -88.44330103139976) bank146219 +146220 POINT(41.24345905595045 -86.74579953999502) bank146220 +146221 POINT(41.60204722222058 -86.99109998336799) bank146221 +146222 POINT(42.75965805809611 -87.76084186921861) bank146222 +146223 POINT(42.76578861675766 -88.11398857723471) bank146223 +146224 POINT(42.519063762687814 -88.2786227457619) bank146224 +146225 POINT(42.64625733475059 -86.84644855572421) bank146225 +146226 POINT(41.45859900876251 -87.54284238100513) bank146226 +146227 POINT(42.76938695353226 -88.19771524912842) bank146227 +146228 POINT(42.250265620785214 -87.93013865346028) bank146228 +146229 POINT(41.92739508591304 -87.63568724630784) bank146229 +146230 POINT(42.05791961665027 -87.51050188416876) bank146230 +146231 POINT(41.60395188335087 -88.58466057171626) bank146231 +146232 POINT(41.41568026046277 -87.94648714031919) bank146232 +146233 POINT(40.890894098955336 -88.26268797519346) bank146233 +146234 POINT(42.00514308122871 -88.27839227239599) bank146234 +146235 POINT(41.59224490600379 -88.08579683411403) bank146235 +146236 POINT(41.290455670282824 -87.69874100941912) bank146236 +146237 POINT(42.72244777255568 -87.95930235203176) bank146237 +146238 POINT(42.008774024085675 -88.34030631648669) bank146238 +146239 POINT(42.344917123078204 -87.21174393319174) bank146239 +146240 POINT(42.02394939063755 -86.68166451606214) bank146240 +146241 POINT(42.11621959994694 -87.4367764169697) bank146241 +146242 POINT(41.886107632961476 -87.32396525939315) bank146242 +146243 POINT(41.50858445046619 -87.82546596492921) bank146243 +146244 POINT(42.784979883334664 -87.34141683662004) bank146244 +146245 POINT(40.998449313026 -88.2306406921499) bank146245 +146246 POINT(42.25344199766549 -87.08863805776542) bank146246 +146247 POINT(41.478792954334295 -88.17090885403762) bank146247 +146248 POINT(41.86313184704749 -88.38291471110931) bank146248 +146249 POINT(42.521087722855995 -87.57035658311705) bank146249 +146250 POINT(41.95162143507092 -88.0801481924002) bank146250 +146251 POINT(41.20591285144786 -86.89602682348156) bank146251 +146252 POINT(41.7998857246885 -87.72241683972562) bank146252 +146253 POINT(41.58027451242071 -87.81066488390925) bank146253 +146254 POINT(41.95123375154259 -86.7760351098795) bank146254 +146255 POINT(42.454547408998486 -87.12871369951777) bank146255 +146256 POINT(40.99254418940975 -88.45005427849622) bank146256 +146257 POINT(41.51673939995818 -87.9123580296349) bank146257 +146258 POINT(42.14715066880795 -88.04084269914567) bank146258 +146259 POINT(41.358246456046615 -87.00144088754516) bank146259 +146260 POINT(42.846317551202766 -87.16269140893002) bank146260 +146261 POINT(41.73685512775799 -87.68790030925508) bank146261 +146262 POINT(41.0123052466187 -88.60648821680692) bank146262 +146263 POINT(42.410183199531936 -87.08899466203796) bank146263 +146264 POINT(41.37089354233019 -87.61720370908694) bank146264 +146265 POINT(42.541704274834395 -88.5523173994608) bank146265 +146266 POINT(41.143485097408416 -86.63357904495948) bank146266 +146267 POINT(42.85934316781306 -87.83773459667567) bank146267 +146268 POINT(41.33143267656479 -87.30535375719401) bank146268 +146269 POINT(41.39966005531543 -88.49429767294706) bank146269 +146270 POINT(42.430014171111296 -87.43505375125277) bank146270 +146271 POINT(41.903420694017875 -88.21749316268836) bank146271 +146272 POINT(42.062657697568675 -88.04116991515473) bank146272 +146273 POINT(42.07428271025919 -87.30217043305721) bank146273 +146274 POINT(41.363746323330595 -87.37765468189053) bank146274 +146275 POINT(42.3859880682695 -87.79719300441694) bank146275 +146276 POINT(42.077681523231234 -87.27345661104597) bank146276 +146277 POINT(41.325715462092 -88.06397223477792) bank146277 +146278 POINT(41.7490813448064 -87.63537452248936) bank146278 +146279 POINT(42.11663757830031 -87.58070030570012) bank146279 +146280 POINT(41.991178605409544 -88.6143888853621) bank146280 +146281 POINT(42.42151968296197 -87.65501331592598) bank146281 +146282 POINT(41.24869518070774 -88.13975938937236) bank146282 +146283 POINT(42.177841918056835 -87.31006750923608) bank146283 +146284 POINT(42.665184579140366 -86.93867012938237) bank146284 +146285 POINT(41.49075653050516 -86.98797360725573) bank146285 +146286 POINT(41.134333010599036 -88.17730064324327) bank146286 +146287 POINT(41.03616681982919 -88.09303626179899) bank146287 +146288 POINT(42.31955807705137 -87.20649094820703) bank146288 +146289 POINT(42.83471689481371 -88.55405279885196) bank146289 +146290 POINT(42.069621140603466 -87.58033200410782) bank146290 +146291 POINT(41.82855710881189 -88.36326690622349) bank146291 +146292 POINT(42.62679896975565 -87.61655604460618) bank146292 +146293 POINT(41.86642852691291 -86.6380219903377) bank146293 +146294 POINT(41.19145524091915 -87.56368546735773) bank146294 +146295 POINT(42.31131136003356 -87.7281066336596) bank146295 +146296 POINT(42.28260407723389 -87.79004244282729) bank146296 +146297 POINT(40.985164003520325 -87.95391129343916) bank146297 +146298 POINT(42.44347791794025 -88.0883254294805) bank146298 +146299 POINT(42.11449417708198 -87.68593892942037) bank146299 +146300 POINT(42.64256750506669 -86.79717303502464) bank146300 +146301 POINT(41.607933358205926 -88.50368296000556) bank146301 +146302 POINT(42.27236198191774 -87.83022764173401) bank146302 +146303 POINT(42.82659692409068 -88.46437857927867) bank146303 +146304 POINT(42.072034169754936 -87.50665667129721) bank146304 +146305 POINT(42.47535060677592 -88.55804894376968) bank146305 +146306 POINT(42.50949506948962 -87.96876799268155) bank146306 +146307 POINT(41.344394567302 -88.22525189851021) bank146307 +146308 POINT(41.78270266476563 -88.60021587953719) bank146308 +146309 POINT(42.76593591121368 -86.99473125687263) bank146309 +146310 POINT(41.568031522908704 -88.27680309896022) bank146310 +146311 POINT(42.10940599146424 -87.4914086034455) bank146311 +146312 POINT(41.23270755699141 -87.26262670491087) bank146312 +146313 POINT(41.76762757883016 -87.72587803682949) bank146313 +146314 POINT(42.63885642596107 -87.04053104391672) bank146314 +146315 POINT(42.782279328156754 -88.07680570671036) bank146315 +146316 POINT(42.688298366882584 -86.64997878640385) bank146316 +146317 POINT(42.68656312866999 -87.45194515425042) bank146317 +146318 POINT(41.976742059720195 -87.43684543715784) bank146318 +146319 POINT(42.517337202369234 -88.55191134710138) bank146319 +146320 POINT(41.25521261556447 -87.82153733131352) bank146320 +146321 POINT(41.03489041222895 -88.40750812166483) bank146321 +146322 POINT(42.845167074623554 -87.09607874871502) bank146322 +146323 POINT(41.618758091388386 -88.40927554648383) bank146323 +146324 POINT(41.004338750633984 -87.09927654239463) bank146324 +146325 POINT(42.053529011332444 -87.65835850875104) bank146325 +146326 POINT(42.50348193559907 -88.53386907657762) bank146326 +146327 POINT(40.91357392667949 -86.77404611915938) bank146327 +146328 POINT(41.30590798646328 -88.03942376106896) bank146328 +146329 POINT(41.02078566991624 -88.43761246305085) bank146329 +146330 POINT(42.32749501935177 -86.78085055468777) bank146330 +146331 POINT(41.21154950421545 -87.55698664951043) bank146331 +146332 POINT(41.44340970588576 -87.80459371445342) bank146332 +146333 POINT(42.708602222324615 -86.790950545876) bank146333 +146334 POINT(40.90306434934191 -87.73213331605643) bank146334 +146335 POINT(41.72712254103401 -88.02721058955952) bank146335 +146336 POINT(41.33898734601514 -87.70495017696537) bank146336 +146337 POINT(41.86996150788161 -87.21765429008089) bank146337 +146338 POINT(42.38217168695113 -88.62551787256365) bank146338 +146339 POINT(42.27299958987457 -87.33459900766265) bank146339 +146340 POINT(42.55414482674642 -87.4139270901149) bank146340 +146341 POINT(42.272226398976514 -87.60078086543163) bank146341 +146342 POINT(41.06206705595804 -88.62136556157857) bank146342 +146343 POINT(41.82869064933085 -86.93226339170549) bank146343 +146344 POINT(41.804026296997506 -88.14591029825569) bank146344 +146345 POINT(41.59987411506545 -87.45549468259073) bank146345 +146346 POINT(41.599665860080826 -88.59619360713079) bank146346 +146347 POINT(40.96290045073707 -86.81884872091392) bank146347 +146348 POINT(42.66778875618865 -87.31265029593881) bank146348 +146349 POINT(42.34020645304063 -87.63345506603308) bank146349 +146350 POINT(41.756525070235504 -88.44722246199099) bank146350 +146351 POINT(41.22227422161459 -86.72671160403168) bank146351 +146352 POINT(42.5110899204633 -86.68045960796677) bank146352 +146353 POINT(42.85672585165427 -87.77771683229447) bank146353 +146354 POINT(41.50972037034914 -88.46556863284604) bank146354 +146355 POINT(42.11781785019768 -87.12359353913486) bank146355 +146356 POINT(42.18290986367712 -88.58656000725301) bank146356 +146357 POINT(42.230283282735506 -87.56239157088966) bank146357 +146358 POINT(42.541461202441596 -87.94939723511645) bank146358 +146359 POINT(41.79284349664499 -87.08061989932632) bank146359 +146360 POINT(41.689475841039645 -88.00818973814675) bank146360 +146361 POINT(41.454020784542536 -86.8736251853376) bank146361 +146362 POINT(41.20930139489083 -88.3885104368605) bank146362 +146363 POINT(41.95918959789172 -86.71010767938735) bank146363 +146364 POINT(42.31575079651754 -88.0464778182897) bank146364 +146365 POINT(42.68396695651523 -87.87070692509955) bank146365 +146366 POINT(41.644952390239645 -87.85956218275716) bank146366 +146367 POINT(41.27210541928166 -87.46852178208792) bank146367 +146368 POINT(41.598926164243004 -87.0715474229261) bank146368 +146369 POINT(41.7956248351764 -87.52814512542626) bank146369 +146370 POINT(41.47909095656589 -87.3047947716103) bank146370 +146371 POINT(42.26203091290748 -87.09868575323884) bank146371 +146372 POINT(42.84740796387943 -87.2865322718933) bank146372 +146373 POINT(40.92210560312908 -87.47729634808601) bank146373 +146374 POINT(42.852748002918815 -87.55003895515155) bank146374 +146375 POINT(41.05077700465315 -87.36496864149348) bank146375 +146376 POINT(42.20872554916851 -87.39947154793371) bank146376 +146377 POINT(42.02658767559618 -88.54697664596905) bank146377 +146378 POINT(41.98540218835648 -87.0599027608418) bank146378 +146379 POINT(41.295154164282096 -88.47107371967063) bank146379 +146380 POINT(41.75384669835572 -87.2324197013621) bank146380 +146381 POINT(42.003903193320504 -87.94164377446076) bank146381 +146382 POINT(41.7854831869412 -87.434702827443) bank146382 +146383 POINT(41.760042254437764 -87.92197258329576) bank146383 +146384 POINT(41.73966841606338 -87.54152861441038) bank146384 +146385 POINT(41.1418326339742 -87.52526734039549) bank146385 +146386 POINT(41.76391503058025 -88.36462403461024) bank146386 +146387 POINT(41.79978229119501 -86.81518480769631) bank146387 +146388 POINT(41.137999617555295 -87.75339866302147) bank146388 +146389 POINT(42.75950224260637 -87.03757190335887) bank146389 +146390 POINT(42.8352942145926 -87.37955097799099) bank146390 +146391 POINT(42.019367537590604 -87.52254759264632) bank146391 +146392 POINT(41.27142809976495 -88.55563910994039) bank146392 +146393 POINT(41.955538812154664 -87.22205603983032) bank146393 +146394 POINT(41.04286776641982 -86.8461184294847) bank146394 +146395 POINT(41.718532217808594 -88.58455401716503) bank146395 +146396 POINT(42.298778972143516 -88.32198581913222) bank146396 +146397 POINT(41.90944494806712 -88.24391336134154) bank146397 +146398 POINT(41.51058847657036 -88.11648649931966) bank146398 +146399 POINT(41.546069870315996 -87.14856769393869) bank146399 +146400 POINT(42.06730806293656 -88.56473382963179) bank146400 +146401 POINT(42.471357648000456 -88.45444962636839) bank146401 +146402 POINT(42.50451107271314 -87.92563791228754) bank146402 +146403 POINT(42.50663580609682 -87.58009937432215) bank146403 +146404 POINT(42.480419973531774 -87.78631868470683) bank146404 +146405 POINT(41.95326646365836 -88.19187749365332) bank146405 +146406 POINT(41.703942821395664 -86.87860406839185) bank146406 +146407 POINT(40.898065269853724 -88.49365633728394) bank146407 +146408 POINT(41.43479981344384 -87.43497558601392) bank146408 +146409 POINT(41.631639771969596 -86.6677793461951) bank146409 +146410 POINT(41.518368072066714 -87.41417214765838) bank146410 +146411 POINT(42.17536380364525 -86.67385249654996) bank146411 +146412 POINT(42.07867161740281 -87.51062904855557) bank146412 +146413 POINT(42.25245359911415 -88.50131013748567) bank146413 +146414 POINT(42.570503852660195 -86.89628125258953) bank146414 +146415 POINT(42.17415184768941 -88.45154055440184) bank146415 +146416 POINT(40.914352599799024 -87.23238985914442) bank146416 +146417 POINT(41.87795211618205 -87.97176829231218) bank146417 +146418 POINT(41.901344691340526 -88.37642934059524) bank146418 +146419 POINT(42.80634145233322 -87.72397574481195) bank146419 +146420 POINT(41.76568379534136 -88.03873885916585) bank146420 +146421 POINT(42.264809998128825 -87.98704018007892) bank146421 +146422 POINT(41.08757176951656 -87.90947107030146) bank146422 +146423 POINT(41.06358670086593 -87.58609329510955) bank146423 +146424 POINT(41.03436587969831 -87.09799692080479) bank146424 +146425 POINT(42.21322670349649 -87.9833334891669) bank146425 +146426 POINT(42.83176504026558 -87.08031353029156) bank146426 +146427 POINT(42.211470065882565 -86.8948593190972) bank146427 +146428 POINT(41.22224048834765 -87.25982607921561) bank146428 +146429 POINT(41.05609342798826 -86.81293406496228) bank146429 +146430 POINT(41.254579833715354 -88.51731642019433) bank146430 +146431 POINT(42.573193543198855 -87.8402466244989) bank146431 +146432 POINT(40.88299024422305 -86.90334144878051) bank146432 +146433 POINT(42.039360590365284 -87.78259984884532) bank146433 +146434 POINT(42.467179108534296 -88.04464433639481) bank146434 +146435 POINT(42.34474899544485 -87.52029107653088) bank146435 +146436 POINT(41.44162735412515 -87.02789101000707) bank146436 +146437 POINT(41.90666846130849 -86.75379892776246) bank146437 +146438 POINT(41.865090398624176 -87.51443349629768) bank146438 +146439 POINT(42.14773855126898 -87.29650215350364) bank146439 +146440 POINT(41.8069856651047 -88.14737777864036) bank146440 +146441 POINT(42.23609232777137 -86.8256188989095) bank146441 +146442 POINT(41.54948941754782 -87.78892038925144) bank146442 +146443 POINT(42.862942773889735 -88.51840574647572) bank146443 +146444 POINT(41.46869215508222 -87.75132361571053) bank146444 +146445 POINT(42.22278076093451 -87.10818708972076) bank146445 +146446 POINT(42.84728738694705 -87.67449819047496) bank146446 +146447 POINT(42.54574913462413 -87.5053134274663) bank146447 +146448 POINT(42.853250680114336 -87.53044090628764) bank146448 +146449 POINT(42.57620260469243 -87.25454931542524) bank146449 +146450 POINT(42.56641771748347 -87.01294794409894) bank146450 +146451 POINT(41.43677991027409 -88.57151197508851) bank146451 +146452 POINT(41.56645305170133 -86.86767499890291) bank146452 +146453 POINT(41.622739619205475 -86.74310530636077) bank146453 +146454 POINT(41.16528410927835 -88.26978227182653) bank146454 +146455 POINT(41.57325690432659 -88.38409494624383) bank146455 +146456 POINT(42.30015274562195 -87.8920479427718) bank146456 +146457 POINT(41.08879008808889 -88.09549652917146) bank146457 +146458 POINT(42.06576013204919 -87.3648107091204) bank146458 +146459 POINT(42.72657190099147 -87.19504494138648) bank146459 +146460 POINT(41.8381890991467 -87.62452486246292) bank146460 +146461 POINT(42.4332505985154 -87.70905707170319) bank146461 +146462 POINT(42.81701901843927 -86.64349001483936) bank146462 +146463 POINT(41.80367034504366 -87.36894706203472) bank146463 +146464 POINT(41.97295417155518 -88.45797886155769) bank146464 +146465 POINT(41.17568473008584 -87.33357512228207) bank146465 +146466 POINT(41.62232855934342 -87.25090982157259) bank146466 +146467 POINT(41.20728727579327 -88.30106026905878) bank146467 +146468 POINT(42.05549734949187 -86.9662708462988) bank146468 +146469 POINT(40.95462842209987 -87.35877261364998) bank146469 +146470 POINT(41.70474055426085 -86.74443802721709) bank146470 +146471 POINT(41.93368654775518 -86.7968546150554) bank146471 +146472 POINT(42.73342583616686 -87.04390362835325) bank146472 +146473 POINT(42.35342259146579 -88.52764097354273) bank146473 +146474 POINT(41.913767797788616 -87.04059739061042) bank146474 +146475 POINT(41.89141938457385 -87.5696511937174) bank146475 +146476 POINT(41.073260220625436 -87.14090095567207) bank146476 +146477 POINT(41.255185199281286 -87.10331013547699) bank146477 +146478 POINT(42.05945111993526 -88.29733433300447) bank146478 +146479 POINT(41.674986777268735 -88.36490130573094) bank146479 +146480 POINT(42.087351906306175 -86.64245990673813) bank146480 +146481 POINT(42.03504900965014 -87.83633153499606) bank146481 +146482 POINT(42.203203765788444 -87.86024732397897) bank146482 +146483 POINT(41.44905854732164 -87.35263087892511) bank146483 +146484 POINT(41.2320734536482 -87.53714503966971) bank146484 +146485 POINT(42.675996412974506 -86.98562572546138) bank146485 +146486 POINT(41.667320081096264 -86.89617825462314) bank146486 +146487 POINT(41.119475516112125 -88.43040392768512) bank146487 +146488 POINT(42.52620941689096 -88.38459816547939) bank146488 +146489 POINT(41.488731754023064 -87.74162235996037) bank146489 +146490 POINT(42.253432547016565 -86.82602956389056) bank146490 +146491 POINT(42.83219783366423 -87.42541136826407) bank146491 +146492 POINT(41.56162621208412 -86.91891217433664) bank146492 +146493 POINT(41.450373701140265 -87.52349278486646) bank146493 +146494 POINT(42.485968619466604 -86.97410443122779) bank146494 +146495 POINT(41.196301939977644 -86.70004892947628) bank146495 +146496 POINT(42.50445858149841 -87.03816646892982) bank146496 +146497 POINT(42.16256212328055 -87.11181673665395) bank146497 +146498 POINT(42.4459582442333 -88.01162378871473) bank146498 +146499 POINT(41.830593040730086 -87.5957104413209) bank146499 +146500 POINT(42.4885570544071 -87.95981756159155) bank146500 +146501 POINT(41.10546518857037 -87.72363871423876) bank146501 +146502 POINT(42.442922165521 -87.48810014214762) bank146502 +146503 POINT(42.876830186959694 -87.61575148604136) bank146503 +146504 POINT(41.01368136160141 -86.93306363618402) bank146504 +146505 POINT(42.50393350575396 -86.80658345427844) bank146505 +146506 POINT(41.05381824880398 -88.57593073195285) bank146506 +146507 POINT(42.80425261542827 -87.81769111731465) bank146507 +146508 POINT(41.386199111673136 -87.32111124794203) bank146508 +146509 POINT(42.47588969037186 -88.36624123670862) bank146509 +146510 POINT(42.39635159010453 -87.97583155857238) bank146510 +146511 POINT(41.917911029420836 -88.22673775641758) bank146511 +146512 POINT(41.1886702287889 -88.16826992742043) bank146512 +146513 POINT(41.76836347333215 -87.91595473329718) bank146513 +146514 POINT(41.02820108863496 -88.44107904060675) bank146514 +146515 POINT(42.30339011895621 -87.50693309043331) bank146515 +146516 POINT(41.89747316112502 -87.9924694795722) bank146516 +146517 POINT(41.38202077162475 -87.64232428949404) bank146517 +146518 POINT(42.50273464319364 -87.32364104947652) bank146518 +146519 POINT(42.55792532521952 -86.9603347468437) bank146519 +146520 POINT(40.98315361645916 -87.07318184183117) bank146520 +146521 POINT(42.75424087829184 -87.77345193565677) bank146521 +146522 POINT(42.84483769766182 -87.23991528152904) bank146522 +146523 POINT(41.37595444563799 -87.50271740173527) bank146523 +146524 POINT(41.5267170961105 -87.01169088252765) bank146524 +146525 POINT(41.665503269411545 -87.94853225440627) bank146525 +146526 POINT(41.233072743375274 -88.26695084977969) bank146526 +146527 POINT(42.47940554736684 -87.17093007813071) bank146527 +146528 POINT(41.99492873581661 -87.04472732817133) bank146528 +146529 POINT(42.87529410085813 -87.978379051488) bank146529 +146530 POINT(41.778543733979845 -88.42879203192096) bank146530 +146531 POINT(42.670719614249776 -86.76364564588717) bank146531 +146532 POINT(41.175824090562585 -87.54280950837767) bank146532 +146533 POINT(42.496334331133134 -87.71082466276808) bank146533 +146534 POINT(41.136917606444584 -86.80583827220579) bank146534 +146535 POINT(41.978298387422335 -87.61542593752904) bank146535 +146536 POINT(42.18862656722297 -87.76681632652699) bank146536 +146537 POINT(42.54229528881121 -87.62767585030105) bank146537 +146538 POINT(42.160886475141005 -87.12990603852369) bank146538 +146539 POINT(42.31928964388523 -87.13740707265845) bank146539 +146540 POINT(41.15836592367169 -87.26225336618855) bank146540 +146541 POINT(41.129808293553396 -87.77034751435423) bank146541 +146542 POINT(40.954882089775964 -87.73984382789405) bank146542 +146543 POINT(42.20531045516848 -87.22879629570367) bank146543 +146544 POINT(42.638606649961474 -87.33877367057534) bank146544 +146545 POINT(42.534746975795535 -86.70973538128973) bank146545 +146546 POINT(41.248371891547706 -86.91690963062825) bank146546 +146547 POINT(40.92915267196941 -87.2697775820923) bank146547 +146548 POINT(42.27598804457829 -87.42837504038286) bank146548 +146549 POINT(41.8158354575562 -87.14205326084581) bank146549 +146550 POINT(41.16133875594689 -87.50560607109928) bank146550 +146551 POINT(42.05359110864795 -86.80206344423739) bank146551 +146552 POINT(42.612303417097735 -86.8465388394156) bank146552 +146553 POINT(41.589428524359654 -87.39191314976144) bank146553 +146554 POINT(41.58761748708539 -88.23901511238924) bank146554 +146555 POINT(41.25378981676029 -88.4890538657789) bank146555 +146556 POINT(41.288532591830794 -87.23270797769725) bank146556 +146557 POINT(42.46470886671507 -88.50388157692777) bank146557 +146558 POINT(42.65894244046832 -87.84142702657992) bank146558 +146559 POINT(40.8865361304686 -86.98112051019665) bank146559 +146560 POINT(41.163773108612 -88.10354819493222) bank146560 +146561 POINT(41.19574129306621 -86.73217902890725) bank146561 +146562 POINT(41.19003121350142 -88.14052116056894) bank146562 +146563 POINT(41.22243773376345 -87.61942281696693) bank146563 +146564 POINT(42.34493698525691 -88.04510072331215) bank146564 +146565 POINT(42.523791952634795 -87.50437015461135) bank146565 +146566 POINT(41.488653916898855 -88.16211035806214) bank146566 +146567 POINT(42.67774632753925 -87.34547503275704) bank146567 +146568 POINT(41.8593150411391 -87.2362105624661) bank146568 +146569 POINT(42.255089674869325 -87.79960226578008) bank146569 +146570 POINT(42.619516581007204 -87.74834001040672) bank146570 +146571 POINT(42.18785822864689 -86.9558191478918) bank146571 +146572 POINT(41.94686948905317 -88.51814541748557) bank146572 +146573 POINT(42.765325691422255 -88.17347685935734) bank146573 +146574 POINT(41.732555750524895 -87.76264149664681) bank146574 +146575 POINT(42.51662149590521 -88.08740380334346) bank146575 +146576 POINT(41.19142252235394 -87.15006101728456) bank146576 +146577 POINT(42.67027054413023 -88.15536255014536) bank146577 +146578 POINT(42.72760397974132 -86.87450701908078) bank146578 +146579 POINT(41.13358806538808 -86.95279848969909) bank146579 +146580 POINT(42.76696838145772 -87.14794271093567) bank146580 +146581 POINT(41.59954044818621 -87.7739863546719) bank146581 +146582 POINT(42.52075640770794 -88.3213176076267) bank146582 +146583 POINT(42.6053272845222 -86.73443603024823) bank146583 +146584 POINT(42.73787287422932 -86.72201385636667) bank146584 +146585 POINT(41.234641375932256 -87.15194101410604) bank146585 +146586 POINT(42.71017488145298 -87.54913278327126) bank146586 +146587 POINT(42.66979546038628 -87.81313375976642) bank146587 +146588 POINT(42.651952013025614 -87.04218696052818) bank146588 +146589 POINT(42.55350570067217 -88.12239888014064) bank146589 +146590 POINT(41.56427560187514 -88.16316041715851) bank146590 +146591 POINT(41.64590292222991 -87.61335851120283) bank146591 +146592 POINT(41.357786293719656 -88.51654559734428) bank146592 +146593 POINT(41.700697000717476 -88.22432587069116) bank146593 +146594 POINT(41.40546426603252 -87.01013679382754) bank146594 +146595 POINT(41.245096060118115 -88.3016325077413) bank146595 +146596 POINT(42.2816553934984 -87.22328326340443) bank146596 +146597 POINT(42.62257675154621 -87.6903895012287) bank146597 +146598 POINT(41.3724661681528 -87.47928592460737) bank146598 +146599 POINT(40.90824695200877 -87.40610636189571) bank146599 +146600 POINT(42.61721780081453 -87.39847312602295) bank146600 +146601 POINT(40.99180386537458 -87.03559984384835) bank146601 +146602 POINT(41.377472226554694 -87.0149584877985) bank146602 +146603 POINT(42.41990488232685 -86.6387678888647) bank146603 +146604 POINT(40.92055737734472 -86.71394379809328) bank146604 +146605 POINT(41.86828787767147 -87.89269130779024) bank146605 +146606 POINT(42.33954978617068 -86.80657574884353) bank146606 +146607 POINT(42.12898071319956 -88.41698616650899) bank146607 +146608 POINT(42.426577892343595 -86.97022702540876) bank146608 +146609 POINT(41.03543398111471 -86.75265383375515) bank146609 +146610 POINT(42.16165186746817 -88.20295861622105) bank146610 +146611 POINT(42.539997408679056 -87.75483281440205) bank146611 +146612 POINT(41.79522285514683 -87.50189568975325) bank146612 +146613 POINT(41.76261514268752 -86.966692825956) bank146613 +146614 POINT(41.5475922515005 -87.93154156593569) bank146614 +146615 POINT(41.85121318611867 -86.90593179119116) bank146615 +146616 POINT(41.364100812275694 -87.70556288864678) bank146616 +146617 POINT(42.057920484498645 -87.5651793384357) bank146617 +146618 POINT(41.922899991952626 -88.19618848207635) bank146618 +146619 POINT(42.674730853029246 -87.79409246569939) bank146619 +146620 POINT(42.500028595104396 -87.66361413999843) bank146620 +146621 POINT(42.079583054639606 -88.13211005219813) bank146621 +146622 POINT(41.488456447534105 -87.15769166571536) bank146622 +146623 POINT(41.058699776578386 -88.18070753622318) bank146623 +146624 POINT(41.93112815057992 -88.10510517650366) bank146624 +146625 POINT(42.01324909009508 -86.68793116591642) bank146625 +146626 POINT(42.205105586394296 -87.0117476039071) bank146626 +146627 POINT(42.6675389226484 -87.31912975352579) bank146627 +146628 POINT(42.70789113365779 -88.44930609231525) bank146628 +146629 POINT(40.975634513026385 -86.65319676660404) bank146629 +146630 POINT(42.007533497110714 -86.666203500861) bank146630 +146631 POINT(42.06148609270431 -87.69274581116628) bank146631 +146632 POINT(41.36427693210091 -88.17127266916616) bank146632 +146633 POINT(42.09789057459443 -87.34659777570775) bank146633 +146634 POINT(41.35010277516703 -86.73422299880585) bank146634 +146635 POINT(41.19150894398838 -88.62575595968465) bank146635 +146636 POINT(40.92320637271926 -87.32585744248136) bank146636 +146637 POINT(42.51968479572877 -88.05116158348567) bank146637 +146638 POINT(42.07615836600366 -87.72820556089547) bank146638 +146639 POINT(42.003168090432446 -88.2201710625505) bank146639 +146640 POINT(41.00346767429866 -87.0710572757488) bank146640 +146641 POINT(40.90370298998967 -88.0213187641804) bank146641 +146642 POINT(42.53844364457889 -87.85612694180458) bank146642 +146643 POINT(41.686117975766905 -87.27172887107957) bank146643 +146644 POINT(41.561502525680204 -87.19909237072942) bank146644 +146645 POINT(40.90611626644643 -87.80619618857526) bank146645 +146646 POINT(41.0071054596654 -87.46726562864991) bank146646 +146647 POINT(41.573055057668334 -88.45343219031602) bank146647 +146648 POINT(41.455664256124464 -87.88911589124027) bank146648 +146649 POINT(41.02374390494196 -87.84100499812297) bank146649 +146650 POINT(41.49104983809693 -87.68205413193955) bank146650 +146651 POINT(41.47129017179424 -88.15776915746085) bank146651 +146652 POINT(42.0936911652047 -88.38850664312633) bank146652 +146653 POINT(41.7980056717412 -88.27211081749658) bank146653 +146654 POINT(41.68160876238359 -88.36508880454983) bank146654 +146655 POINT(41.04984934723983 -88.38827614727859) bank146655 +146656 POINT(41.91676438071183 -87.60488212591704) bank146656 +146657 POINT(41.418028180246026 -87.94108942590324) bank146657 +146658 POINT(42.29263768321693 -86.7785815190201) bank146658 +146659 POINT(41.950563019175405 -88.11979802034966) bank146659 +146660 POINT(41.74656405442303 -87.18807959701059) bank146660 +146661 POINT(41.533115595018934 -87.60650856147609) bank146661 +146662 POINT(42.049919112386036 -87.63688609335107) bank146662 +146663 POINT(42.286452980846114 -87.67204291264557) bank146663 +146664 POINT(41.74693685516552 -88.06255505004732) bank146664 +146665 POINT(42.10491025800567 -87.41241302136842) bank146665 +146666 POINT(42.70005653129085 -87.91843401529772) bank146666 +146667 POINT(42.216609450442874 -86.74438330220775) bank146667 +146668 POINT(42.69310205280816 -88.5383760348489) bank146668 +146669 POINT(41.86241752105708 -88.06896956666047) bank146669 +146670 POINT(41.110982836830345 -88.31105002486106) bank146670 +146671 POINT(41.55346652274609 -88.59265915160455) bank146671 +146672 POINT(42.58973109314991 -88.06602459444352) bank146672 +146673 POINT(41.77535552122741 -87.55970958371607) bank146673 +146674 POINT(41.56065123197271 -87.93130865164116) bank146674 +146675 POINT(42.269608232257575 -87.2470164930965) bank146675 +146676 POINT(41.84325564086352 -88.08488875822728) bank146676 +146677 POINT(42.47464918026631 -86.74074288115686) bank146677 +146678 POINT(41.603873617828114 -86.84687215359428) bank146678 +146679 POINT(42.06624725267549 -86.66357445147494) bank146679 +146680 POINT(41.271843489290696 -87.41718319310597) bank146680 +146681 POINT(41.912488266809085 -87.877710997815) bank146681 +146682 POINT(42.54897063855001 -88.2177458433503) bank146682 +146683 POINT(40.98674637835785 -87.8374717941618) bank146683 +146684 POINT(42.20226870698208 -88.0390418339258) bank146684 +146685 POINT(42.11320904585683 -87.83442508538039) bank146685 +146686 POINT(42.47870046519964 -87.9485562944297) bank146686 +146687 POINT(41.83377220453042 -87.7886767612499) bank146687 +146688 POINT(41.82041596220226 -88.38738869669154) bank146688 +146689 POINT(41.63527305320601 -88.01847377808382) bank146689 +146690 POINT(41.32084047778473 -88.44427776859852) bank146690 +146691 POINT(42.017151845927 -87.50308306477807) bank146691 +146692 POINT(42.816726163751326 -87.38140530761463) bank146692 +146693 POINT(42.390064121209925 -86.67460919845857) bank146693 +146694 POINT(41.780388068007206 -88.58519408708547) bank146694 +146695 POINT(41.77412251660408 -87.503260527386) bank146695 +146696 POINT(42.666152195174064 -87.83892600482369) bank146696 +146697 POINT(41.794738881136006 -87.21838993873942) bank146697 +146698 POINT(41.94440193074511 -88.37155205585793) bank146698 +146699 POINT(41.70483257003748 -86.75472669885893) bank146699 +146700 POINT(41.448605970775155 -86.95841676177785) bank146700 +146701 POINT(42.368416929657826 -86.76322732965852) bank146701 +146702 POINT(42.504028656072734 -87.13230339733778) bank146702 +146703 POINT(41.63786088022343 -86.83281460153671) bank146703 +146704 POINT(42.661338399505624 -88.2919110676879) bank146704 +146705 POINT(42.0247901235886 -88.2131101623153) bank146705 +146706 POINT(41.24379796472447 -87.44588906844089) bank146706 +146707 POINT(41.936155213035974 -88.60066961020591) bank146707 +146708 POINT(41.09629207874703 -88.33940583864923) bank146708 +146709 POINT(41.15095894072047 -88.08971371967057) bank146709 +146710 POINT(42.46689709932943 -87.62619807779134) bank146710 +146711 POINT(42.664918145275436 -87.25770090714425) bank146711 +146712 POINT(41.69641344966672 -87.2850266078296) bank146712 +146713 POINT(41.3189533204491 -88.57757625219178) bank146713 +146714 POINT(41.04257680044168 -87.59309478084073) bank146714 +146715 POINT(41.139478560977146 -87.72267808597746) bank146715 +146716 POINT(42.14450803158905 -86.81700154348486) bank146716 +146717 POINT(41.83436474263689 -88.45808022672755) bank146717 +146718 POINT(42.702207452176815 -88.29108912571074) bank146718 +146719 POINT(41.73816440317969 -86.65245262904422) bank146719 +146720 POINT(42.302046613681 -87.37678840156046) bank146720 +146721 POINT(41.77223440662515 -87.71506143195231) bank146721 +146722 POINT(42.6153380684702 -87.2850261027542) bank146722 +146723 POINT(41.09513902689523 -87.91715206742585) bank146723 +146724 POINT(42.49606955983399 -87.70275212466203) bank146724 +146725 POINT(42.25185250393632 -87.10785696970079) bank146725 +146726 POINT(41.71547917720813 -88.4386528365512) bank146726 +146727 POINT(42.424470339266854 -86.73865097624414) bank146727 +146728 POINT(42.58765433306886 -86.75977345164488) bank146728 +146729 POINT(42.388620141308216 -86.97020470611363) bank146729 +146730 POINT(41.89929496110081 -86.92655526642672) bank146730 +146731 POINT(41.23821601705798 -87.68676483374867) bank146731 +146732 POINT(42.283503738259434 -88.5855746774452) bank146732 +146733 POINT(42.721619722117794 -87.34526487619299) bank146733 +146734 POINT(41.58133487355195 -86.90378027835477) bank146734 +146735 POINT(41.148641800339604 -87.59392028900974) bank146735 +146736 POINT(41.908200730759184 -88.59450028927165) bank146736 +146737 POINT(41.27966562179225 -88.10452572031268) bank146737 +146738 POINT(41.11145547794468 -87.6213241097288) bank146738 +146739 POINT(40.970979240680364 -87.1616528518325) bank146739 +146740 POINT(41.70527609597359 -88.04799891522322) bank146740 +146741 POINT(41.94265333743319 -86.94087121447406) bank146741 +146742 POINT(41.18848961843204 -87.42205699978999) bank146742 +146743 POINT(42.493986144624316 -88.41565089773538) bank146743 +146744 POINT(42.70108569110119 -88.04664352607358) bank146744 +146745 POINT(42.54917473398356 -87.83577190616403) bank146745 +146746 POINT(42.56484361255132 -87.6796082761011) bank146746 +146747 POINT(40.963122184542605 -88.17066388781659) bank146747 +146748 POINT(42.82035397841713 -88.02928062221646) bank146748 +146749 POINT(41.262673098407056 -87.62395577134174) bank146749 +146750 POINT(41.61493103608373 -87.48017005265652) bank146750 +146751 POINT(41.923785955663895 -88.56517263637971) bank146751 +146752 POINT(42.84789746786973 -86.71598074475646) bank146752 +146753 POINT(42.45566928085566 -87.57302473377472) bank146753 +146754 POINT(41.635758190533814 -86.85552030793994) bank146754 +146755 POINT(41.232635510559085 -87.48320548388327) bank146755 +146756 POINT(42.87272689590817 -87.10075904061036) bank146756 +146757 POINT(42.56216176266258 -86.97991028746047) bank146757 +146758 POINT(41.7792263503394 -88.29187969622811) bank146758 +146759 POINT(41.33717869702295 -86.68424306671922) bank146759 +146760 POINT(41.62390209312823 -87.99065429100767) bank146760 +146761 POINT(41.80825738434994 -86.75199403145143) bank146761 +146762 POINT(41.26268245579976 -88.28537797409683) bank146762 +146763 POINT(41.28043023644337 -86.98389586271483) bank146763 +146764 POINT(42.276785249071104 -88.52220512150777) bank146764 +146765 POINT(41.25835814320105 -86.70011613188099) bank146765 +146766 POINT(42.237240652944514 -87.62408145536857) bank146766 +146767 POINT(42.477218488706114 -87.80484680828648) bank146767 +146768 POINT(41.37921250450206 -87.56744012168761) bank146768 +146769 POINT(41.232099919326636 -86.98366947396349) bank146769 +146770 POINT(41.611349701281256 -87.52062080568284) bank146770 +146771 POINT(42.41331802441816 -87.26530516454847) bank146771 +146772 POINT(42.4381233499969 -88.19647879962348) bank146772 +146773 POINT(41.6957688070053 -87.06273211509738) bank146773 +146774 POINT(41.88916490099717 -86.7507289923105) bank146774 +146775 POINT(40.92839163290985 -88.18057874210903) bank146775 +146776 POINT(42.54927162469741 -87.11841083497677) bank146776 +146777 POINT(41.261459155204 -87.87932155402827) bank146777 +146778 POINT(41.37733430907709 -87.83746396995659) bank146778 +146779 POINT(41.813181274690336 -87.88566630343136) bank146779 +146780 POINT(42.28889154187493 -86.98407892536001) bank146780 +146781 POINT(41.75207689025285 -87.77515954526199) bank146781 +146782 POINT(42.125407176363666 -87.12259271956476) bank146782 +146783 POINT(40.94684819009905 -88.06365075017708) bank146783 +146784 POINT(41.2889951659333 -86.92484993611117) bank146784 +146785 POINT(41.334678596392656 -87.91047840937455) bank146785 +146786 POINT(41.030998524487785 -86.99929523327657) bank146786 +146787 POINT(41.41045354113253 -87.5762453504302) bank146787 +146788 POINT(41.89987681448307 -88.25154772181475) bank146788 +146789 POINT(42.23877337681023 -88.12234514702612) bank146789 +146790 POINT(41.6960681912219 -87.58326192623733) bank146790 +146791 POINT(42.71698570409368 -87.42434238244674) bank146791 +146792 POINT(42.42347622425548 -86.67906848792039) bank146792 +146793 POINT(41.07817089177651 -86.76740919818897) bank146793 +146794 POINT(42.44556474136813 -87.41302464544768) bank146794 +146795 POINT(41.65986564410053 -87.75786790247888) bank146795 +146796 POINT(41.6407316064487 -87.52862100270103) bank146796 +146797 POINT(42.648902356961315 -88.34970516024048) bank146797 +146798 POINT(42.86307285332605 -86.79224919306085) bank146798 +146799 POINT(41.94812344198797 -88.22667712047556) bank146799 +146800 POINT(41.25502440272315 -87.95950582895182) bank146800 +146801 POINT(40.95476166543374 -86.9276411795065) bank146801 +146802 POINT(42.5675486723291 -88.56421013563201) bank146802 +146803 POINT(41.92139911631981 -87.58795415261903) bank146803 +146804 POINT(42.17660071398281 -87.312877947056) bank146804 +146805 POINT(40.95568181195599 -88.2460828358461) bank146805 +146806 POINT(42.14242404678862 -88.3196869012439) bank146806 +146807 POINT(40.97155611493749 -86.89414592856876) bank146807 +146808 POINT(42.38156985119835 -88.45468961799226) bank146808 +146809 POINT(42.52710686560895 -87.6993063393218) bank146809 +146810 POINT(41.232798557547746 -86.87572455773066) bank146810 +146811 POINT(41.01915863152778 -88.29118587541387) bank146811 +146812 POINT(42.03253762709406 -87.81355141740312) bank146812 +146813 POINT(41.82085883920564 -88.51357220448612) bank146813 +146814 POINT(40.957102889611384 -87.40754364487631) bank146814 +146815 POINT(42.14761095149799 -87.14601003505973) bank146815 +146816 POINT(41.60792525200438 -87.51246318835189) bank146816 +146817 POINT(42.58147825621412 -88.3648473085402) bank146817 +146818 POINT(41.78666161104696 -88.56233211543373) bank146818 +146819 POINT(41.66062444129087 -88.16357024872242) bank146819 +146820 POINT(42.09130165137986 -86.88472845975883) bank146820 +146821 POINT(42.02880951986218 -86.81957210590487) bank146821 +146822 POINT(42.35463494469883 -88.46936271829476) bank146822 +146823 POINT(41.80179029725786 -87.70801661702332) bank146823 +146824 POINT(40.9747792121009 -87.95078117833685) bank146824 +146825 POINT(41.8847369034061 -88.36181589262816) bank146825 +146826 POINT(42.04587251390685 -86.96866218926067) bank146826 +146827 POINT(42.250960152585556 -88.13097535859463) bank146827 +146828 POINT(41.13100345360983 -88.5685124772123) bank146828 +146829 POINT(41.15355637239743 -87.07341902503288) bank146829 +146830 POINT(42.50546049958938 -87.43129148432918) bank146830 +146831 POINT(42.17189337378513 -87.48515221732791) bank146831 +146832 POINT(41.17928403362972 -87.98345119792441) bank146832 +146833 POINT(41.99249769932213 -87.46895886187028) bank146833 +146834 POINT(41.16019536148951 -88.34339022674618) bank146834 +146835 POINT(42.15035535452393 -86.66928526226667) bank146835 +146836 POINT(41.2111987675029 -87.542904238825) bank146836 +146837 POINT(41.172103357294915 -88.44124802916895) bank146837 +146838 POINT(41.294999029429654 -86.71348123181917) bank146838 +146839 POINT(41.53815456635167 -87.79269592349799) bank146839 +146840 POINT(41.909650176793356 -87.94857560251108) bank146840 +146841 POINT(42.16481641205328 -87.79655739387739) bank146841 +146842 POINT(41.119998638734835 -88.35212254619958) bank146842 +146843 POINT(42.64453407692587 -88.56282791241938) bank146843 +146844 POINT(42.740469172777395 -88.54798806669847) bank146844 +146845 POINT(41.48477578856648 -87.67178202059695) bank146845 +146846 POINT(42.85705434979338 -86.75350610612115) bank146846 +146847 POINT(41.16267181556428 -88.11688800377992) bank146847 +146848 POINT(42.811761664132845 -87.16144286643143) bank146848 +146849 POINT(41.96434544155327 -87.47137297002135) bank146849 +146850 POINT(42.6832974288958 -88.24619409101746) bank146850 +146851 POINT(42.74097439962158 -87.38341583648575) bank146851 +146852 POINT(42.207111542547736 -88.04790346445989) bank146852 +146853 POINT(42.0171164167155 -87.31586896547111) bank146853 +146854 POINT(40.89137357381325 -87.42411566348743) bank146854 +146855 POINT(41.11891318267078 -86.9598588439575) bank146855 +146856 POINT(41.22517421567258 -87.21054879834267) bank146856 +146857 POINT(41.21377956214975 -88.24833347255444) bank146857 +146858 POINT(41.12065791926914 -88.16808292409524) bank146858 +146859 POINT(42.32773417242356 -86.9785440718446) bank146859 +146860 POINT(42.19907956660016 -88.54488028890863) bank146860 +146861 POINT(42.38022208161924 -88.34579631866868) bank146861 +146862 POINT(42.41138912308055 -88.15279661772372) bank146862 +146863 POINT(42.26137220181029 -87.29103746673243) bank146863 +146864 POINT(42.0039135370976 -87.99350841800411) bank146864 +146865 POINT(42.37152347859277 -87.92826239835537) bank146865 +146866 POINT(41.16752542202755 -86.83389577630437) bank146866 +146867 POINT(42.868076183094104 -87.47525590936978) bank146867 +146868 POINT(42.682852853527606 -87.5782332347986) bank146868 +146869 POINT(41.210799599164446 -87.75681013375511) bank146869 +146870 POINT(41.045263889981975 -87.61668177992706) bank146870 +146871 POINT(41.61116444322383 -88.28721032058675) bank146871 +146872 POINT(41.15125510628599 -87.8699724420755) bank146872 +146873 POINT(40.99410225559315 -87.59610768623565) bank146873 +146874 POINT(41.93949751596432 -87.50404918503715) bank146874 +146875 POINT(41.87156169302249 -87.76781026633954) bank146875 +146876 POINT(41.9396966599952 -88.20220734036944) bank146876 +146877 POINT(42.04338889272293 -88.15256392491278) bank146877 +146878 POINT(41.11885160139489 -88.28099535918133) bank146878 +146879 POINT(42.614368485323126 -87.2434840641736) bank146879 +146880 POINT(41.95879399923151 -87.6299050368707) bank146880 +146881 POINT(41.28201192349239 -86.95376109156061) bank146881 +146882 POINT(42.713047704086435 -88.27751678347154) bank146882 +146883 POINT(41.32969972468648 -86.86296934520945) bank146883 +146884 POINT(42.35945022226525 -88.05545119349) bank146884 +146885 POINT(41.06409846057127 -87.02130617297483) bank146885 +146886 POINT(40.93520927251255 -87.82725385501574) bank146886 +146887 POINT(41.54849820488962 -87.91940167700332) bank146887 +146888 POINT(41.60998296958237 -87.23762960778893) bank146888 +146889 POINT(42.69228478145529 -88.13565337373997) bank146889 +146890 POINT(41.66340343350514 -88.24304090179537) bank146890 +146891 POINT(41.225338434036566 -88.13482076121693) bank146891 +146892 POINT(40.971988234090745 -87.75227506663181) bank146892 +146893 POINT(41.469363055941386 -86.65496423318169) bank146893 +146894 POINT(41.007868683994445 -86.77369984036581) bank146894 +146895 POINT(41.41274534896945 -87.33755281868416) bank146895 +146896 POINT(42.0369425159402 -86.84529945846512) bank146896 +146897 POINT(41.239771863952036 -88.30678832249953) bank146897 +146898 POINT(41.40758519632444 -88.54476376439247) bank146898 +146899 POINT(41.79311912176199 -88.35198183849971) bank146899 +146900 POINT(41.27816677857457 -87.99971849916545) bank146900 +146901 POINT(42.804630578295296 -88.39004253049251) bank146901 +146902 POINT(40.90513545378927 -88.21474813004545) bank146902 +146903 POINT(41.464373132599704 -87.41288025127548) bank146903 +146904 POINT(41.60116399879884 -88.54781107990294) bank146904 +146905 POINT(42.70808804994731 -87.68751708920689) bank146905 +146906 POINT(42.66151264744037 -88.13634985596502) bank146906 +146907 POINT(41.20290552610048 -87.30199326189319) bank146907 +146908 POINT(41.26771916282157 -87.39441539434553) bank146908 +146909 POINT(41.154216808448645 -87.18376899722755) bank146909 +146910 POINT(42.268013634473064 -87.23799051995294) bank146910 +146911 POINT(42.24085167622564 -86.73802757320627) bank146911 +146912 POINT(42.207812316060746 -87.81792426710936) bank146912 +146913 POINT(41.58261214595093 -88.60434790153919) bank146913 +146914 POINT(42.781642651624665 -88.37280572430211) bank146914 +146915 POINT(41.677291915009846 -88.36567182089462) bank146915 +146916 POINT(42.16555236575984 -87.3033120903107) bank146916 +146917 POINT(41.55287191704041 -87.65139822940691) bank146917 +146918 POINT(41.36648853798367 -87.96063853868874) bank146918 +146919 POINT(40.919006723353355 -88.12697630127653) bank146919 +146920 POINT(42.252134474988615 -86.6701362617222) bank146920 +146921 POINT(42.38538527585696 -86.8356043846885) bank146921 +146922 POINT(42.47716213882787 -88.36070842300973) bank146922 +146923 POINT(41.563090469603516 -87.982140084066) bank146923 +146924 POINT(41.91660127384908 -87.51326262222956) bank146924 +146925 POINT(42.57631906679707 -88.037343740751) bank146925 +146926 POINT(41.36237400296801 -87.32510139708491) bank146926 +146927 POINT(42.13098456730744 -88.36544045756823) bank146927 +146928 POINT(42.35272944568096 -87.61324888012966) bank146928 +146929 POINT(42.074751447430906 -88.18907406209959) bank146929 +146930 POINT(42.20868394749825 -87.62095910593479) bank146930 +146931 POINT(40.915878029060096 -86.65838685988108) bank146931 +146932 POINT(41.64085607648053 -86.84825391402899) bank146932 +146933 POINT(41.65392337191253 -86.68117963441713) bank146933 +146934 POINT(41.50891204994319 -88.22138664045372) bank146934 +146935 POINT(42.137030886690354 -88.23773821703571) bank146935 +146936 POINT(41.069182876454754 -87.1007557746275) bank146936 +146937 POINT(41.61401459737623 -86.8720074225847) bank146937 +146938 POINT(41.04737458357464 -88.25166149112164) bank146938 +146939 POINT(41.179675468669515 -88.18574516528953) bank146939 +146940 POINT(41.92908390626424 -88.59439588101667) bank146940 +146941 POINT(41.304397315416914 -87.88168736707138) bank146941 +146942 POINT(42.04055095202802 -88.00470085628464) bank146942 +146943 POINT(41.68182695164693 -88.39830376669899) bank146943 +146944 POINT(42.722475570448644 -87.88936182562622) bank146944 +146945 POINT(42.64731742152859 -86.89136936924629) bank146945 +146946 POINT(42.449270826922195 -88.12191868316754) bank146946 +146947 POINT(42.781111090594834 -86.99043143539741) bank146947 +146948 POINT(42.543966035711776 -88.44267558186873) bank146948 +146949 POINT(40.88229325299881 -87.29651213545432) bank146949 +146950 POINT(42.81163699308921 -87.29193057320443) bank146950 +146951 POINT(41.74716501531697 -88.26221929089492) bank146951 +146952 POINT(41.11995040777468 -88.5495134177253) bank146952 +146953 POINT(42.72909607628995 -87.45171161370051) bank146953 +146954 POINT(42.78346553110647 -87.28452967539883) bank146954 +146955 POINT(40.98815221606089 -86.76787568377289) bank146955 +146956 POINT(42.601116267493396 -88.20120730712961) bank146956 +146957 POINT(42.735890646963604 -87.63230499808988) bank146957 +146958 POINT(42.64667307463905 -87.15610939510357) bank146958 +146959 POINT(42.54096310178711 -87.31931385682466) bank146959 +146960 POINT(41.16352965689905 -86.64466979255384) bank146960 +146961 POINT(41.52134440496751 -88.04697504911111) bank146961 +146962 POINT(41.49638535584904 -88.41769387900085) bank146962 +146963 POINT(41.71436028919288 -87.10274012083958) bank146963 +146964 POINT(41.526271704856626 -87.04062876170997) bank146964 +146965 POINT(41.727589615486245 -87.54440264508541) bank146965 +146966 POINT(42.36682858046978 -86.98227874165936) bank146966 +146967 POINT(42.399551614855866 -88.33781542747995) bank146967 +146968 POINT(41.885032577970364 -87.38291084754297) bank146968 +146969 POINT(42.110058542969355 -87.91406896513138) bank146969 +146970 POINT(41.25586520068672 -88.18000523120463) bank146970 +146971 POINT(41.71896014962939 -87.84440553152847) bank146971 +146972 POINT(42.51763130093814 -87.84001269176593) bank146972 +146973 POINT(42.813352636049565 -87.70648500712115) bank146973 +146974 POINT(42.318954115159585 -88.2102102659556) bank146974 +146975 POINT(42.611054321220095 -86.83145784494592) bank146975 +146976 POINT(42.085217016086126 -87.90034850590476) bank146976 +146977 POINT(41.1330480572657 -88.05171344143785) bank146977 +146978 POINT(42.00473656974824 -87.825998606489) bank146978 +146979 POINT(41.0578698874807 -87.2274595800758) bank146979 +146980 POINT(41.25310651978345 -87.9564226766798) bank146980 +146981 POINT(42.21184026769919 -87.37473555902422) bank146981 +146982 POINT(42.810771542961454 -87.90254650409383) bank146982 +146983 POINT(42.569445561064356 -87.56901920170591) bank146983 +146984 POINT(41.76814089331695 -88.11424130470039) bank146984 +146985 POINT(41.194477899843235 -86.8175236413082) bank146985 +146986 POINT(41.37833919593944 -86.94638957215903) bank146986 +146987 POINT(42.6352594745878 -88.34327070793697) bank146987 +146988 POINT(42.08551081112609 -87.70224961400409) bank146988 +146989 POINT(41.153263364542205 -86.6897234803553) bank146989 +146990 POINT(40.92114208390683 -86.72827560815018) bank146990 +146991 POINT(41.001394040251625 -88.5143015820536) bank146991 +146992 POINT(41.71340973575345 -87.50238404274258) bank146992 +146993 POINT(42.320401664676396 -88.31919193266565) bank146993 +146994 POINT(41.81703449141344 -87.67651061958935) bank146994 +146995 POINT(41.16888004990658 -87.9727117556433) bank146995 +146996 POINT(42.35641249687611 -87.8635859838324) bank146996 +146997 POINT(41.18161570418262 -87.62388125513537) bank146997 +146998 POINT(41.69370184208888 -86.69430000998665) bank146998 +146999 POINT(42.006129472914864 -87.222625369996) bank146999 +147000 POINT(42.81266084230775 -86.94741319474284) bank147000 +147001 POINT(42.15535453990459 -87.03139403921236) bank147001 +147002 POINT(42.27227957064814 -88.41633498123264) bank147002 +147003 POINT(40.99744513435251 -86.64601850914106) bank147003 +147004 POINT(41.7857345258524 -87.73301005961615) bank147004 +147005 POINT(42.87757031560109 -86.84742881416436) bank147005 +147006 POINT(42.37251986271921 -88.39168070666959) bank147006 +147007 POINT(42.62147537690006 -87.86495653202913) bank147007 +147008 POINT(42.582237421591515 -88.60967979519182) bank147008 +147009 POINT(41.04580114039527 -87.63046633699021) bank147009 +147010 POINT(41.3694314942644 -87.67597225647836) bank147010 +147011 POINT(41.426476169783456 -87.08096269986403) bank147011 +147012 POINT(42.799713703027486 -86.68788910017938) bank147012 +147013 POINT(41.8037500002551 -87.11728568828542) bank147013 +147014 POINT(41.138610138267715 -88.57756660046012) bank147014 +147015 POINT(42.045763584207556 -88.3435556982757) bank147015 +147016 POINT(41.908089921810415 -87.95320666575726) bank147016 +147017 POINT(42.017222093491235 -87.8802020399596) bank147017 +147018 POINT(42.786478320563155 -88.62369668011387) bank147018 +147019 POINT(41.42931869294251 -87.89454670579977) bank147019 +147020 POINT(42.673252315809314 -88.19100619215828) bank147020 +147021 POINT(42.650590007589635 -87.04468620261419) bank147021 +147022 POINT(41.16981563412456 -86.79627646599063) bank147022 +147023 POINT(42.41601611453937 -87.96795632073601) bank147023 +147024 POINT(41.01439342998498 -86.66230054575557) bank147024 +147025 POINT(42.781811518101975 -88.4180143754688) bank147025 +147026 POINT(41.80787526378233 -86.78031506197816) bank147026 +147027 POINT(42.40109563833165 -87.98051308794264) bank147027 +147028 POINT(41.48110486571563 -86.78407503626603) bank147028 +147029 POINT(41.56939991026689 -87.0085563412958) bank147029 +147030 POINT(42.825096268051325 -87.02968722267536) bank147030 +147031 POINT(42.07377463066257 -87.2636178942942) bank147031 +147032 POINT(42.488653343029874 -87.17277214456001) bank147032 +147033 POINT(41.172323507779225 -87.78671960180104) bank147033 +147034 POINT(42.3423846900875 -86.73005547161405) bank147034 +147035 POINT(41.57693277793212 -88.49822817251331) bank147035 +147036 POINT(41.27755098379139 -87.99744178001127) bank147036 +147037 POINT(41.4309198300469 -87.99415899438269) bank147037 +147038 POINT(42.13091688419231 -86.8167437222762) bank147038 +147039 POINT(41.88095892493819 -87.08050940579945) bank147039 +147040 POINT(42.22297982287571 -87.5830249120198) bank147040 +147041 POINT(42.440358286319785 -87.02878585182263) bank147041 +147042 POINT(42.305528140195975 -87.56150707823127) bank147042 +147043 POINT(41.2430514308873 -88.18341519566863) bank147043 +147044 POINT(42.72276968708242 -87.22423663736349) bank147044 +147045 POINT(41.88159355141218 -87.69609144102097) bank147045 +147046 POINT(41.19737008991538 -86.65320693138887) bank147046 +147047 POINT(41.65832439000919 -87.4531709730677) bank147047 +147048 POINT(42.26389348825123 -88.04458991262855) bank147048 +147049 POINT(41.705075068771755 -87.1854668831481) bank147049 +147050 POINT(41.92858374331797 -86.74365733191173) bank147050 +147051 POINT(41.731583227503315 -87.89627459016516) bank147051 +147052 POINT(40.96986054461023 -87.67465996392576) bank147052 +147053 POINT(42.68482038376605 -87.8196302434433) bank147053 +147054 POINT(41.2429228968051 -87.28002243905208) bank147054 +147055 POINT(41.00363037770451 -87.79581038142938) bank147055 +147056 POINT(41.17136724540714 -88.43374171636998) bank147056 +147057 POINT(41.54532320860742 -87.4926623530882) bank147057 +147058 POINT(42.33594600373699 -88.45130182454085) bank147058 +147059 POINT(41.84836634566443 -86.83320484546394) bank147059 +147060 POINT(41.502741036989946 -87.86567650167882) bank147060 +147061 POINT(42.58535208899603 -87.35073006489539) bank147061 +147062 POINT(40.98372695691689 -88.06346417760133) bank147062 +147063 POINT(42.53125449797358 -88.21470517634698) bank147063 +147064 POINT(41.37630551713244 -87.53920689873937) bank147064 +147065 POINT(41.49568940692814 -87.95915249016618) bank147065 +147066 POINT(41.59054045077686 -87.79331214433995) bank147066 +147067 POINT(42.721481182265016 -86.9060027142545) bank147067 +147068 POINT(42.435800398737456 -88.01100645537294) bank147068 +147069 POINT(41.57725332306956 -87.34533712579324) bank147069 +147070 POINT(42.08585258218931 -87.85309132446235) bank147070 +147071 POINT(41.09273511749303 -87.47811633180027) bank147071 +147072 POINT(42.479009768257264 -88.20576258390366) bank147072 +147073 POINT(40.887194286064336 -86.95991692353535) bank147073 +147074 POINT(42.721863142963926 -88.14189308287933) bank147074 +147075 POINT(41.67154814389865 -86.97018491230193) bank147075 +147076 POINT(41.04170902271886 -87.58751263889823) bank147076 +147077 POINT(42.29249854966475 -88.48016403281379) bank147077 +147078 POINT(41.215305202810846 -87.71040660146325) bank147078 +147079 POINT(42.29436250166545 -87.55048808880682) bank147079 +147080 POINT(41.32708360566442 -88.05561227584111) bank147080 +147081 POINT(41.81550387588326 -87.42895703868956) bank147081 +147082 POINT(40.976246128651276 -87.44306958474985) bank147082 +147083 POINT(41.2277987738037 -87.58209761444155) bank147083 +147084 POINT(41.137706428392576 -86.99575426751638) bank147084 +147085 POINT(41.002916433065266 -87.43020569760087) bank147085 +147086 POINT(42.20107200878843 -88.41273524985903) bank147086 +147087 POINT(41.64631541210861 -87.47887442613293) bank147087 +147088 POINT(42.339270944575816 -87.87323273805026) bank147088 +147089 POINT(42.8741320589801 -86.98796440267589) bank147089 +147090 POINT(41.19044524261242 -88.18147845126367) bank147090 +147091 POINT(41.80517185334518 -86.66232359020955) bank147091 +147092 POINT(42.01916677447663 -86.81606389377123) bank147092 +147093 POINT(42.818113781997894 -87.64555365207119) bank147093 +147094 POINT(42.71326151085327 -87.06808170069154) bank147094 +147095 POINT(42.4551352440431 -87.69106656684038) bank147095 +147096 POINT(41.73151052644774 -87.18831508320567) bank147096 +147097 POINT(41.596634964750855 -87.52636100541322) bank147097 +147098 POINT(42.31328770253692 -87.85325416365626) bank147098 +147099 POINT(41.43924161016628 -87.01858474529897) bank147099 +147100 POINT(42.606335182114826 -87.09585053824073) bank147100 +147101 POINT(41.97658208159177 -86.84237290378152) bank147101 +147102 POINT(41.99268542800905 -88.60237309916113) bank147102 +147103 POINT(41.03856257158579 -87.32561042321213) bank147103 +147104 POINT(41.28331198024858 -88.5073260173524) bank147104 +147105 POINT(42.0473575675602 -88.05361745187359) bank147105 +147106 POINT(42.35460085288963 -88.08825767122966) bank147106 +147107 POINT(41.588168684762536 -87.87160827764052) bank147107 +147108 POINT(41.594331437349624 -87.90845923808116) bank147108 +147109 POINT(41.4559412867226 -87.35744949937119) bank147109 +147110 POINT(41.282594697504635 -87.13241512682733) bank147110 +147111 POINT(41.08846428090669 -87.53160762478878) bank147111 +147112 POINT(41.112925669092895 -87.37186437626367) bank147112 +147113 POINT(40.95649150860556 -86.89527681426628) bank147113 +147114 POINT(42.1183945811684 -87.15214945759566) bank147114 +147115 POINT(41.27171394421241 -88.50623885790027) bank147115 +147116 POINT(41.13567513643466 -87.16772197734007) bank147116 +147117 POINT(41.78444373339248 -88.50653858847957) bank147117 +147118 POINT(41.29787624567747 -88.06957291314585) bank147118 +147119 POINT(41.393988217036004 -86.6401043931817) bank147119 +147120 POINT(41.2421822342977 -88.49544719272312) bank147120 +147121 POINT(41.9660060580024 -87.05172516353385) bank147121 +147122 POINT(42.76490494442477 -87.38137147087767) bank147122 +147123 POINT(40.97757372104314 -88.2284599636159) bank147123 +147124 POINT(42.711445649258216 -87.9516323017491) bank147124 +147125 POINT(42.09180562009282 -87.52809940102154) bank147125 +147126 POINT(41.34077766701922 -88.11600658180174) bank147126 +147127 POINT(41.69837539132689 -87.6610011246361) bank147127 +147128 POINT(41.16157575488446 -87.46199767937237) bank147128 +147129 POINT(42.42460513439687 -88.0767388028533) bank147129 +147130 POINT(41.53312091283386 -87.17447274017938) bank147130 +147131 POINT(41.174802133715914 -87.32084105913685) bank147131 +147132 POINT(41.387927090387095 -87.55401282684025) bank147132 +147133 POINT(41.34907500817397 -88.0288198756169) bank147133 +147134 POINT(42.45068757169973 -86.83198691671178) bank147134 +147135 POINT(41.33119065943352 -87.70450434604534) bank147135 +147136 POINT(42.551223826347645 -88.33908775864796) bank147136 +147137 POINT(41.127951197404634 -87.84861158302272) bank147137 +147138 POINT(41.55330629769369 -87.13437927814184) bank147138 +147139 POINT(41.946627109092084 -87.4862903793109) bank147139 +147140 POINT(42.14312642896357 -87.28328800497108) bank147140 +147141 POINT(42.09434400147512 -87.3341234269124) bank147141 +147142 POINT(42.73190628657938 -88.0811686556833) bank147142 +147143 POINT(42.59233553022298 -88.25536926025251) bank147143 +147144 POINT(41.87586249131266 -86.70434174406476) bank147144 +147145 POINT(42.40544541232228 -87.26104372705939) bank147145 +147146 POINT(42.09699152459991 -88.53863395964966) bank147146 +147147 POINT(41.08504235767396 -87.54238015273441) bank147147 +147148 POINT(41.83799671654273 -87.05751818858423) bank147148 +147149 POINT(42.67377489439167 -86.65384388140018) bank147149 +147150 POINT(42.69330428388962 -87.81277066572434) bank147150 +147151 POINT(42.33985452838911 -87.11462694293051) bank147151 +147152 POINT(41.824206049107715 -88.27435351879588) bank147152 +147153 POINT(42.735018986800874 -87.54372097201994) bank147153 +147154 POINT(40.96077753495054 -88.01578445910539) bank147154 +147155 POINT(42.62324393205449 -87.46270292444142) bank147155 +147156 POINT(41.81813341630386 -86.6809501732683) bank147156 +147157 POINT(41.49028152830955 -88.03656524086217) bank147157 +147158 POINT(41.55536003408571 -87.9636276535246) bank147158 +147159 POINT(42.25931022854758 -87.37627633926229) bank147159 +147160 POINT(41.63912401756503 -87.93522325517898) bank147160 +147161 POINT(42.71408389478395 -87.9129368385594) bank147161 +147162 POINT(41.55723885048736 -87.11554489232779) bank147162 +147163 POINT(42.848590445207044 -87.76663568661964) bank147163 +147164 POINT(42.455902736777844 -86.79979480473945) bank147164 +147165 POINT(42.57122067419299 -87.66238563231276) bank147165 +147166 POINT(41.58331361863289 -87.06917077359704) bank147166 +147167 POINT(42.47205576634329 -86.75824083171443) bank147167 +147168 POINT(41.92992125151479 -87.57171219034304) bank147168 +147169 POINT(42.144165486526425 -87.49653098476608) bank147169 +147170 POINT(41.51443293460572 -87.20148206340255) bank147170 +147171 POINT(42.745002282914776 -88.143199262697) bank147171 +147172 POINT(41.47419856596552 -87.38302243585235) bank147172 +147173 POINT(41.471512548601055 -88.43753583715274) bank147173 +147174 POINT(42.62719956091682 -87.52460768859352) bank147174 +147175 POINT(42.414406478432646 -86.8682267173728) bank147175 +147176 POINT(41.91565326717447 -86.96835507085746) bank147176 +147177 POINT(41.4843588550838 -87.05289159435613) bank147177 +147178 POINT(41.481811452270556 -86.7862658860029) bank147178 +147179 POINT(42.34985335289375 -88.41776552276949) bank147179 +147180 POINT(41.387018747145774 -86.8544910055642) bank147180 +147181 POINT(41.99513385006312 -87.25879643206984) bank147181 +147182 POINT(42.272917695656474 -86.63308402232376) bank147182 +147183 POINT(42.365653641873614 -87.91842272047344) bank147183 +147184 POINT(42.46538844080515 -88.49179286746366) bank147184 +147185 POINT(42.25272795431813 -88.07935404138422) bank147185 +147186 POINT(42.83855061399173 -87.86546815306002) bank147186 +147187 POINT(41.76618881890754 -87.28557164898476) bank147187 +147188 POINT(40.9719863810617 -87.48390980964838) bank147188 +147189 POINT(41.47486346277879 -87.2376756584816) bank147189 +147190 POINT(41.95932317932369 -86.84645662050171) bank147190 +147191 POINT(40.97595847535678 -87.46677782120912) bank147191 +147192 POINT(42.547789377000434 -88.49737901888932) bank147192 +147193 POINT(41.076452557036234 -88.47561768211585) bank147193 +147194 POINT(42.59884836268185 -86.70986641717401) bank147194 +147195 POINT(41.66608522686532 -88.04653659487407) bank147195 +147196 POINT(41.857230731353496 -87.40090436319225) bank147196 +147197 POINT(41.8953518648999 -87.42455717585861) bank147197 +147198 POINT(42.8237976326384 -87.13012213758904) bank147198 +147199 POINT(42.21339004421127 -87.70978512197418) bank147199 +147200 POINT(42.04422607933974 -86.77336051339873) bank147200 +147201 POINT(42.48529651693863 -88.58478892481774) bank147201 +147202 POINT(41.89880814708025 -86.86744390601861) bank147202 +147203 POINT(42.42583444106528 -88.18732432218607) bank147203 +147204 POINT(42.77751442914312 -87.52278888230308) bank147204 +147205 POINT(42.852810548547076 -87.2320885864631) bank147205 +147206 POINT(41.72743494435996 -88.29610721562877) bank147206 +147207 POINT(42.329431929503215 -87.07138255924322) bank147207 +147208 POINT(41.547884405586714 -88.35550883547347) bank147208 +147209 POINT(42.82801474642989 -87.49283251989482) bank147209 +147210 POINT(42.275104734499244 -87.64885164164465) bank147210 +147211 POINT(40.989669690773546 -86.86886835243429) bank147211 +147212 POINT(41.114320781690346 -86.85232767403784) bank147212 +147213 POINT(42.19741554883446 -87.27542528339359) bank147213 +147214 POINT(40.946376028874035 -88.31077479259729) bank147214 +147215 POINT(41.59153483119654 -88.09895244188998) bank147215 +147216 POINT(41.84173070353894 -87.13161957414626) bank147216 +147217 POINT(42.600835141414116 -87.91231298651935) bank147217 +147218 POINT(42.66267796979833 -88.31003369518575) bank147218 +147219 POINT(41.56269291235337 -86.717919843796) bank147219 +147220 POINT(42.46031304375352 -88.5828509033634) bank147220 +147221 POINT(41.451680709254035 -86.95574303289287) bank147221 +147222 POINT(42.205339681418444 -87.09663875177756) bank147222 +147223 POINT(41.021700131206096 -88.15506899424408) bank147223 +147224 POINT(42.81922965034407 -88.59373091568129) bank147224 +147225 POINT(42.25292621844768 -86.99889781609431) bank147225 +147226 POINT(42.45666872989132 -87.26865723250533) bank147226 +147227 POINT(41.89391416735799 -87.71825410155989) bank147227 +147228 POINT(42.124938955471414 -87.54111044989126) bank147228 +147229 POINT(42.26537367991616 -87.24554795491939) bank147229 +147230 POINT(41.91963054096016 -87.35212383264812) bank147230 +147231 POINT(42.053813885462624 -87.69301856807722) bank147231 +147232 POINT(42.572158773858305 -87.46462455981894) bank147232 +147233 POINT(41.5907669212887 -87.31171287980605) bank147233 +147234 POINT(42.454978248773216 -87.04383961206082) bank147234 +147235 POINT(41.70319034893003 -87.4810827754999) bank147235 +147236 POINT(41.442944132185474 -88.0394208135339) bank147236 +147237 POINT(42.19946698447611 -88.03149999035807) bank147237 +147238 POINT(41.227091384385055 -87.01942209517676) bank147238 +147239 POINT(42.433983983645 -87.9543416298086) bank147239 +147240 POINT(42.402853186175044 -88.35433320040703) bank147240 +147241 POINT(41.12704485660003 -87.3140991436738) bank147241 +147242 POINT(42.781157526132475 -88.20935445602788) bank147242 +147243 POINT(42.07957809387163 -86.8135471425388) bank147243 +147244 POINT(42.36694781434868 -87.39576936256191) bank147244 +147245 POINT(41.97121382519832 -88.52840443273327) bank147245 +147246 POINT(41.75672015130481 -87.06791387353002) bank147246 +147247 POINT(41.343856181210604 -87.2492972435324) bank147247 +147248 POINT(42.25970832886662 -86.87198237398121) bank147248 +147249 POINT(40.935217158244086 -87.89882122712814) bank147249 +147250 POINT(41.27287294092701 -86.68627311671179) bank147250 +147251 POINT(41.363429537719675 -87.46182495854228) bank147251 +147252 POINT(42.21514129457857 -87.99184849533526) bank147252 +147253 POINT(42.43465024227278 -87.93964481402189) bank147253 +147254 POINT(41.33995209743823 -86.94487612698919) bank147254 +147255 POINT(41.57747809322446 -86.68671402615784) bank147255 +147256 POINT(40.90640206432879 -87.86981258672915) bank147256 +147257 POINT(40.99553764439262 -86.76679941664612) bank147257 +147258 POINT(41.779402045616344 -87.30035123853371) bank147258 +147259 POINT(42.70965836234477 -87.25647892490781) bank147259 +147260 POINT(42.03315428522218 -86.70374532123498) bank147260 +147261 POINT(41.81340674781545 -87.31700001877522) bank147261 +147262 POINT(41.47757939679914 -88.05581366708063) bank147262 +147263 POINT(41.09580905458198 -87.4290739221471) bank147263 +147264 POINT(41.7031895008871 -87.56236060930497) bank147264 +147265 POINT(42.53329974476314 -86.83126470721699) bank147265 +147266 POINT(41.71054314436244 -86.85568011101019) bank147266 +147267 POINT(42.31827216594549 -87.05583654533086) bank147267 +147268 POINT(41.17084079670904 -88.58182444574933) bank147268 +147269 POINT(42.589251372671434 -88.01740633442768) bank147269 +147270 POINT(42.293643474053255 -87.05189076752873) bank147270 +147271 POINT(41.87211239419454 -87.27760802190564) bank147271 +147272 POINT(41.4952898236562 -86.82381533394982) bank147272 +147273 POINT(42.22224488669425 -86.96427586271412) bank147273 +147274 POINT(41.85173888735741 -87.76595302599432) bank147274 +147275 POINT(41.54478118219325 -88.45861905315674) bank147275 +147276 POINT(40.98756721130516 -87.29756189796242) bank147276 +147277 POINT(42.28762669083385 -86.91453482305643) bank147277 +147278 POINT(41.14098737300454 -88.06538756699838) bank147278 +147279 POINT(41.17648780946192 -87.0454466503381) bank147279 +147280 POINT(42.79351244277087 -87.5230167427892) bank147280 +147281 POINT(41.44275614311767 -87.51825078494693) bank147281 +147282 POINT(41.80069120987615 -88.42036544351761) bank147282 +147283 POINT(41.89535206085937 -87.96870304498091) bank147283 +147284 POINT(41.23119875064292 -87.34485074984417) bank147284 +147285 POINT(41.65769977173521 -87.11581037903511) bank147285 +147286 POINT(42.05465112321391 -87.03134007449295) bank147286 +147287 POINT(40.96744916839941 -87.62598326691196) bank147287 +147288 POINT(41.68512812916047 -88.52430601884282) bank147288 +147289 POINT(41.170820966625044 -86.63392090913781) bank147289 +147290 POINT(41.468391346813426 -87.50800700018259) bank147290 +147291 POINT(41.52635316483261 -87.53150771111208) bank147291 +147292 POINT(42.34098734727276 -88.20669508676521) bank147292 +147293 POINT(42.488990671734996 -88.37020179749966) bank147293 +147294 POINT(40.97818446237508 -88.33853662401158) bank147294 +147295 POINT(42.12103216774573 -86.65579082921145) bank147295 +147296 POINT(42.68579349497791 -87.66113322342568) bank147296 +147297 POINT(42.70489688668381 -88.32446995271346) bank147297 +147298 POINT(41.14807583620708 -86.66461451215497) bank147298 +147299 POINT(41.59758797303136 -88.34928327690983) bank147299 +147300 POINT(41.21184151513367 -88.41387955456214) bank147300 +147301 POINT(41.45123094433135 -86.85544413756269) bank147301 +147302 POINT(42.070599041781556 -87.58796430391371) bank147302 +147303 POINT(42.28401767353641 -88.4338085724536) bank147303 +147304 POINT(41.38505648387896 -87.62744181530044) bank147304 +147305 POINT(41.69517179521523 -87.8705877214065) bank147305 +147306 POINT(41.268204568367544 -86.63987192787506) bank147306 +147307 POINT(41.94585195720801 -86.86500075789556) bank147307 +147308 POINT(41.94669581601037 -88.61880282655045) bank147308 +147309 POINT(41.18835098906763 -87.39339802554319) bank147309 +147310 POINT(41.07923558299207 -87.66254680470942) bank147310 +147311 POINT(41.8407486121354 -87.19589467478117) bank147311 +147312 POINT(41.551067693573756 -88.07572569011303) bank147312 +147313 POINT(41.38897695819785 -87.43238126050166) bank147313 +147314 POINT(40.92830325467377 -88.2833257926735) bank147314 +147315 POINT(42.52414820445836 -88.29073693233688) bank147315 +147316 POINT(41.932137439786494 -87.17092626002149) bank147316 +147317 POINT(41.96075793879465 -88.22386575609391) bank147317 +147318 POINT(41.89698088202448 -87.47257487021727) bank147318 +147319 POINT(42.47524426037555 -87.59712509335364) bank147319 +147320 POINT(42.07411701253405 -88.23745614307066) bank147320 +147321 POINT(41.68417738279478 -88.42182141931812) bank147321 +147322 POINT(42.85028396453046 -87.09738976440772) bank147322 +147323 POINT(42.57897148707269 -87.14473597731227) bank147323 +147324 POINT(42.79261951921572 -87.83183147578012) bank147324 +147325 POINT(42.13206394575162 -87.45309276890667) bank147325 +147326 POINT(41.607444656225134 -88.31547145691324) bank147326 +147327 POINT(41.33365837542501 -88.44996782555837) bank147327 +147328 POINT(42.35741697520608 -86.8744492028419) bank147328 +147329 POINT(41.05679442332274 -88.19446862823223) bank147329 +147330 POINT(42.27901299834586 -87.8566376260478) bank147330 +147331 POINT(40.885125127027315 -87.26228450140403) bank147331 +147332 POINT(42.4493337359087 -88.31295528548004) bank147332 +147333 POINT(42.025093676373345 -87.95061652605983) bank147333 +147334 POINT(42.44584107638722 -88.32550122886293) bank147334 +147335 POINT(42.258308428831235 -87.32038584408157) bank147335 +147336 POINT(42.493890432766975 -87.42624053575854) bank147336 +147337 POINT(41.4543721275334 -86.816176799551) bank147337 +147338 POINT(42.31314001441898 -87.8166037432225) bank147338 +147339 POINT(42.51290394963001 -87.86034861703843) bank147339 +147340 POINT(41.72896794434267 -88.06147300619651) bank147340 +147341 POINT(42.688917041388535 -87.07132157918744) bank147341 +147342 POINT(42.334930383832635 -87.3363047339271) bank147342 +147343 POINT(41.358996419618364 -87.5278798581518) bank147343 +147344 POINT(42.85803635590275 -87.10994068411621) bank147344 +147345 POINT(41.46716408151525 -87.33020741904258) bank147345 +147346 POINT(42.5390966177666 -86.82981829180238) bank147346 +147347 POINT(42.77303708317017 -87.54092635731024) bank147347 +147348 POINT(42.543067055331335 -87.92221546362512) bank147348 +147349 POINT(42.230389545351564 -87.0653472166349) bank147349 +147350 POINT(41.20075548838929 -86.69425678471576) bank147350 +147351 POINT(41.64980091957059 -87.46764341699365) bank147351 +147352 POINT(42.22215904059088 -88.00690453624716) bank147352 +147353 POINT(42.019430142596995 -88.34504587676213) bank147353 +147354 POINT(41.756322663784836 -88.46026572248894) bank147354 +147355 POINT(42.56306575536844 -87.56357134146062) bank147355 +147356 POINT(41.10552580583576 -87.56398637132109) bank147356 +147357 POINT(42.654368091721196 -87.58232406241028) bank147357 +147358 POINT(42.301461219507175 -87.77163032531878) bank147358 +147359 POINT(42.21181229829533 -87.08653575953136) bank147359 +147360 POINT(41.129269003049686 -87.86242114441878) bank147360 +147361 POINT(42.786445240943884 -86.9401857640675) bank147361 +147362 POINT(42.67255577453536 -88.62804690247161) bank147362 +147363 POINT(40.97743399660259 -88.26202153031531) bank147363 +147364 POINT(41.058909332840834 -87.95449381869628) bank147364 +147365 POINT(41.16018855477958 -88.09028763652594) bank147365 +147366 POINT(41.49568564940586 -87.00771348375243) bank147366 +147367 POINT(41.99240756418768 -87.89229562341282) bank147367 +147368 POINT(41.194327062990254 -87.90002985123093) bank147368 +147369 POINT(41.118123847446434 -88.2990943007213) bank147369 +147370 POINT(40.95591426830402 -88.31076864121331) bank147370 +147371 POINT(41.283909988640616 -88.25321188334367) bank147371 +147372 POINT(41.15959824574596 -87.20363428480962) bank147372 +147373 POINT(42.30701729172345 -87.14132734305905) bank147373 +147374 POINT(41.99610386457558 -87.90091487570669) bank147374 +147375 POINT(42.67542511231058 -87.25741461815677) bank147375 +147376 POINT(42.11505133787567 -88.08463010363634) bank147376 +147377 POINT(41.358505656855115 -87.45587511170517) bank147377 +147378 POINT(40.99293512375044 -88.2234491149345) bank147378 +147379 POINT(42.02617806933124 -87.17561692164085) bank147379 +147380 POINT(42.17275382803536 -88.12956317696012) bank147380 +147381 POINT(42.379691020584985 -87.36521662208517) bank147381 +147382 POINT(41.48194745862948 -88.60072746065207) bank147382 +147383 POINT(41.927732808996375 -87.12633177745182) bank147383 +147384 POINT(41.64955054359634 -87.13303048696079) bank147384 +147385 POINT(40.93789012410211 -88.11964928283335) bank147385 +147386 POINT(41.473163455465034 -88.0097983020528) bank147386 +147387 POINT(41.18598546302771 -88.51023982455187) bank147387 +147388 POINT(42.82423197644326 -87.17599081140455) bank147388 +147389 POINT(42.203890348639185 -87.18688215003613) bank147389 +147390 POINT(42.54541173059628 -87.48636550172203) bank147390 +147391 POINT(41.955414723125244 -88.5463612297416) bank147391 +147392 POINT(42.43505026975747 -86.74179768503006) bank147392 +147393 POINT(42.27834807851724 -86.6612881013868) bank147393 +147394 POINT(41.89929436099563 -87.68003594880291) bank147394 +147395 POINT(42.68625589556515 -86.88498940716279) bank147395 +147396 POINT(42.622377555225704 -88.19339071349603) bank147396 +147397 POINT(41.18452973208853 -87.5505101667125) bank147397 +147398 POINT(42.20335095420216 -87.00420805505489) bank147398 +147399 POINT(42.603604111049606 -87.92596085361589) bank147399 +147400 POINT(40.96351005241611 -88.27353059017015) bank147400 +147401 POINT(40.90694677773648 -88.13660411872011) bank147401 +147402 POINT(42.318082138001536 -88.0059785507017) bank147402 +147403 POINT(42.199738206615 -87.33729447311418) bank147403 +147404 POINT(42.28697682651678 -87.55486249714653) bank147404 +147405 POINT(41.86574913371834 -86.94242681363944) bank147405 +147406 POINT(41.51010644296532 -87.35129016631589) bank147406 +147407 POINT(41.232250782373214 -86.96258614306083) bank147407 +147408 POINT(42.47195265475721 -86.79010859218145) bank147408 +147409 POINT(40.937438586568206 -87.25837033013686) bank147409 +147410 POINT(42.41289254253194 -87.50999630033094) bank147410 +147411 POINT(41.58350046387922 -87.58264969806436) bank147411 +147412 POINT(42.5998687663523 -87.3765191944622) bank147412 +147413 POINT(42.32283358550721 -87.54290015746439) bank147413 +147414 POINT(41.53612529096257 -88.54488417090488) bank147414 +147415 POINT(42.01033731921985 -87.7747479986366) bank147415 +147416 POINT(41.01174347193512 -88.32040557545879) bank147416 +147417 POINT(41.0462666243873 -87.82947634667396) bank147417 +147418 POINT(42.24882026819775 -86.6564643729424) bank147418 +147419 POINT(41.059391992046244 -87.53242404039862) bank147419 +147420 POINT(41.28659871331337 -88.46329355720623) bank147420 +147421 POINT(42.57283961807209 -87.38514997852005) bank147421 +147422 POINT(42.044707994320795 -86.67631251730855) bank147422 +147423 POINT(42.619587153289324 -88.30668308491148) bank147423 +147424 POINT(42.1740444384076 -88.36383946199108) bank147424 +147425 POINT(42.347644935041274 -87.28614696246329) bank147425 +147426 POINT(42.435350879798364 -86.72484068520679) bank147426 +147427 POINT(41.49552519745417 -87.40293523001354) bank147427 +147428 POINT(41.83937496084826 -87.59617941388959) bank147428 +147429 POINT(42.24053797469854 -88.47771293703654) bank147429 +147430 POINT(42.58690767620882 -87.41024259785182) bank147430 +147431 POINT(41.11729597222046 -88.47580055358846) bank147431 +147432 POINT(42.35305010218117 -88.28666180509293) bank147432 +147433 POINT(40.93674009326654 -87.03862989863251) bank147433 +147434 POINT(42.59997354510372 -86.77652788435026) bank147434 +147435 POINT(41.74529847847848 -87.3622748556479) bank147435 +147436 POINT(41.07918727738772 -88.4242121476955) bank147436 +147437 POINT(42.405776743419956 -87.53376422646603) bank147437 +147438 POINT(42.67522867065733 -86.68909560739432) bank147438 +147439 POINT(41.784566037355404 -88.49910824533588) bank147439 +147440 POINT(41.36715539631303 -88.13173444821416) bank147440 +147441 POINT(41.46658111662825 -87.45752142065503) bank147441 +147442 POINT(41.05665557905553 -86.77857713151246) bank147442 +147443 POINT(41.023617331240885 -88.03483113174555) bank147443 +147444 POINT(42.832675855013726 -88.01019110788194) bank147444 +147445 POINT(41.194739167706196 -88.2616976962762) bank147445 +147446 POINT(42.500730840494356 -88.46844113799403) bank147446 +147447 POINT(42.46722894910813 -86.72773658875681) bank147447 +147448 POINT(42.23159672125034 -87.02660646530009) bank147448 +147449 POINT(42.30112809512485 -87.1101552825102) bank147449 +147450 POINT(42.21174968892721 -86.64819745343526) bank147450 +147451 POINT(42.14714823854012 -88.50555513416862) bank147451 +147452 POINT(42.6511891816864 -86.88192358325317) bank147452 +147453 POINT(41.154620131008436 -87.58934158581143) bank147453 +147454 POINT(41.39863251518976 -87.47628561858383) bank147454 +147455 POINT(42.62498674083299 -88.5910352094388) bank147455 +147456 POINT(41.90585189480567 -88.02902208756738) bank147456 +147457 POINT(42.63926027233509 -88.04845525481723) bank147457 +147458 POINT(41.657867342768924 -87.0388711570404) bank147458 +147459 POINT(42.53009746674806 -87.32310164966506) bank147459 +147460 POINT(42.54064188606266 -87.56748384380242) bank147460 +147461 POINT(41.808947307667076 -87.92117436557159) bank147461 +147462 POINT(42.086567161591574 -87.37405774894133) bank147462 +147463 POINT(41.46671614429973 -88.18201488813828) bank147463 +147464 POINT(41.20104680962001 -87.67839328336727) bank147464 +147465 POINT(41.881174318105344 -86.63627257795177) bank147465 +147466 POINT(41.96465521602988 -87.91548913196874) bank147466 +147467 POINT(42.12900446476282 -88.10148391064855) bank147467 +147468 POINT(42.34247925696443 -88.24778467249017) bank147468 +147469 POINT(41.73512078122061 -87.24111516033624) bank147469 +147470 POINT(41.55560474123171 -87.43961384873153) bank147470 +147471 POINT(41.984537532900816 -87.82688832864554) bank147471 +147472 POINT(41.058778624411445 -87.43422267772577) bank147472 +147473 POINT(41.43991357643175 -87.19499674361832) bank147473 +147474 POINT(42.62964351216771 -87.67232490923553) bank147474 +147475 POINT(42.415645971587026 -86.67628162940625) bank147475 +147476 POINT(40.881327633376095 -87.51759221281117) bank147476 +147477 POINT(41.34049588310761 -88.0335447889743) bank147477 +147478 POINT(42.66129430293802 -88.48656390001975) bank147478 +147479 POINT(41.70434530237708 -86.93416081298273) bank147479 +147480 POINT(41.15400707918148 -87.34720059403732) bank147480 +147481 POINT(41.48561115240309 -86.9664592661655) bank147481 +147482 POINT(42.656534112709856 -87.36045828515971) bank147482 +147483 POINT(41.478103529494426 -87.400398948499) bank147483 +147484 POINT(41.16833126540055 -88.28597699334587) bank147484 +147485 POINT(41.98082886578301 -87.8745759426519) bank147485 +147486 POINT(41.252850379824764 -87.63959487782108) bank147486 +147487 POINT(42.76000639939289 -87.51294297130855) bank147487 +147488 POINT(42.55449730403033 -88.17495277591563) bank147488 +147489 POINT(42.11152416082035 -86.68104134287283) bank147489 +147490 POINT(41.47676499600432 -87.31118612552993) bank147490 +147491 POINT(41.56381202197725 -87.4349746825872) bank147491 +147492 POINT(41.022448849971866 -87.81329026256086) bank147492 +147493 POINT(42.80428402895306 -87.86196693036625) bank147493 +147494 POINT(42.528151619554386 -88.36775894325996) bank147494 +147495 POINT(41.51728129112462 -88.36928522722451) bank147495 +147496 POINT(41.752388783923045 -88.2675088981247) bank147496 +147497 POINT(42.55062822399114 -87.52664999150298) bank147497 +147498 POINT(42.52446593880457 -88.20523913544766) bank147498 +147499 POINT(42.40821657258607 -87.6271924584469) bank147499 +147500 POINT(42.16366672653878 -88.2659477264807) bank147500 +147501 POINT(41.85068687927124 -87.38898097959412) bank147501 +147502 POINT(41.08026354365691 -86.63802418620901) bank147502 +147503 POINT(40.97767890743211 -87.83871286497423) bank147503 +147504 POINT(41.29798168768088 -86.72100872107478) bank147504 +147505 POINT(41.37873641925542 -88.10822841014956) bank147505 +147506 POINT(42.37422480811371 -86.91407269633912) bank147506 +147507 POINT(42.29618279908886 -88.14463859644927) bank147507 +147508 POINT(42.42050165184679 -87.93995659631439) bank147508 +147509 POINT(40.88206931889313 -87.18662092522187) bank147509 +147510 POINT(41.680196535967546 -88.08944263906842) bank147510 +147511 POINT(41.58803328158976 -88.03361268547408) bank147511 +147512 POINT(42.231214846458926 -86.67478615397643) bank147512 +147513 POINT(40.90484323179972 -88.29647799466086) bank147513 +147514 POINT(41.4555697794869 -88.25951937497706) bank147514 +147515 POINT(42.003500839472345 -87.15111631058103) bank147515 +147516 POINT(41.64027152668175 -86.8029209286748) bank147516 +147517 POINT(41.202226060490425 -86.96055458295498) bank147517 +147518 POINT(41.12426663167867 -88.59744900694312) bank147518 +147519 POINT(41.11869735297108 -87.72117500835712) bank147519 +147520 POINT(41.62448296365121 -86.79672225201345) bank147520 +147521 POINT(41.83298449861774 -87.35495707291588) bank147521 +147522 POINT(42.00971254907694 -87.9127797091676) bank147522 +147523 POINT(41.99779212767872 -87.33509718762198) bank147523 +147524 POINT(41.39447464539172 -88.35710748767424) bank147524 +147525 POINT(41.834192258173694 -87.84464891379564) bank147525 +147526 POINT(41.9878307789541 -86.95047185493824) bank147526 +147527 POINT(41.406904652189326 -88.20466839200782) bank147527 +147528 POINT(41.836512745792966 -88.00362683652598) bank147528 +147529 POINT(42.692166959458085 -88.59073257740759) bank147529 +147530 POINT(42.46073545212129 -87.36689872221889) bank147530 +147531 POINT(41.52188594350225 -87.32953283593133) bank147531 +147532 POINT(42.86146331888861 -87.26440697126954) bank147532 +147533 POINT(41.26014844767357 -87.6872944175844) bank147533 +147534 POINT(42.253987608711974 -87.06423677270176) bank147534 +147535 POINT(41.95278637299478 -86.7350462833723) bank147535 +147536 POINT(41.84720776178 -87.5275721887043) bank147536 +147537 POINT(41.5347607126103 -86.74263227357685) bank147537 +147538 POINT(41.81244185547541 -87.60785558735904) bank147538 +147539 POINT(41.952377946701525 -87.58523922043766) bank147539 +147540 POINT(42.798692570184116 -87.59526552835439) bank147540 +147541 POINT(42.27874926240623 -88.391473274629) bank147541 +147542 POINT(41.48208748530068 -88.5880185922591) bank147542 +147543 POINT(42.535796738160435 -87.28842975017311) bank147543 +147544 POINT(42.81388982343281 -86.65931112644515) bank147544 +147545 POINT(41.01817897433532 -87.892273805887) bank147545 +147546 POINT(41.078585818223964 -87.40766136961517) bank147546 +147547 POINT(42.59900603966558 -87.80460269749136) bank147547 +147548 POINT(41.22278275235456 -87.4891133605726) bank147548 +147549 POINT(40.94474235061806 -87.83707166782548) bank147549 +147550 POINT(41.81272457084168 -87.84950810612837) bank147550 +147551 POINT(41.592449407128136 -87.81064967264939) bank147551 +147552 POINT(42.07112438260659 -87.114100901771) bank147552 +147553 POINT(41.54616568481753 -88.09122310960419) bank147553 +147554 POINT(42.51133077530686 -86.82696750620873) bank147554 +147555 POINT(41.93816038178758 -86.96174319758623) bank147555 +147556 POINT(41.63630453479991 -88.46941317493939) bank147556 +147557 POINT(41.19944452350459 -86.65044518086644) bank147557 +147558 POINT(42.01639958461918 -87.73487677439479) bank147558 +147559 POINT(41.25679680223216 -87.87215442654075) bank147559 +147560 POINT(42.85868565794482 -88.38258160248786) bank147560 +147561 POINT(41.440977091473826 -87.11235754945677) bank147561 +147562 POINT(42.095932512601195 -88.3160654718968) bank147562 +147563 POINT(41.75272629006168 -88.18899012138931) bank147563 +147564 POINT(41.31319249478081 -87.26551373555101) bank147564 +147565 POINT(42.39444754254325 -87.22827590717708) bank147565 +147566 POINT(42.45795012440356 -88.52162731570755) bank147566 +147567 POINT(41.37891947935062 -88.26164328699761) bank147567 +147568 POINT(41.10083176268904 -88.4173414922348) bank147568 +147569 POINT(41.88924302496719 -86.77035984830198) bank147569 +147570 POINT(41.93327293704784 -87.39834382543997) bank147570 +147571 POINT(41.28553735874001 -88.59934304405142) bank147571 +147572 POINT(42.225600820588944 -87.16041833374254) bank147572 +147573 POINT(42.10988340993981 -88.2744493590145) bank147573 +147574 POINT(42.587310566427355 -87.00369766815732) bank147574 +147575 POINT(41.46851424177507 -86.82214700048696) bank147575 +147576 POINT(42.267294072410444 -88.37768874860727) bank147576 +147577 POINT(41.64563574103945 -88.5917885085556) bank147577 +147578 POINT(41.0072243908934 -88.52777427062743) bank147578 +147579 POINT(42.48678216028713 -87.54167952727323) bank147579 +147580 POINT(41.90712608862548 -88.14422615284073) bank147580 +147581 POINT(41.00151299975718 -88.14311305850933) bank147581 +147582 POINT(41.0187567273782 -87.15016743570975) bank147582 +147583 POINT(41.4092933657599 -86.77191849494754) bank147583 +147584 POINT(41.35473157415979 -87.21802413647569) bank147584 +147585 POINT(41.11046678222946 -87.58978977332036) bank147585 +147586 POINT(42.845594825057994 -87.20264468350688) bank147586 +147587 POINT(42.607421364452364 -87.410955318393) bank147587 +147588 POINT(42.730420639702714 -86.85571994116431) bank147588 +147589 POINT(42.793730031530956 -88.2054659999009) bank147589 +147590 POINT(42.069390497790565 -87.32311279640584) bank147590 +147591 POINT(41.67220883246992 -86.81483705347874) bank147591 +147592 POINT(42.57456286499337 -87.31434197427365) bank147592 +147593 POINT(42.322463446795936 -88.41900091242898) bank147593 +147594 POINT(42.68771003711701 -87.0163343429619) bank147594 +147595 POINT(42.06713222760533 -88.21288903589267) bank147595 +147596 POINT(41.021050994772025 -88.40156368768915) bank147596 +147597 POINT(41.868989855898576 -87.51273545488873) bank147597 +147598 POINT(41.6306316679954 -88.5133400826747) bank147598 +147599 POINT(41.58403125836481 -88.08536106559148) bank147599 +147600 POINT(42.79534451543513 -87.65601918655754) bank147600 +147601 POINT(40.9402331341692 -88.14325940799667) bank147601 +147602 POINT(41.0880103380725 -87.2336438421157) bank147602 +147603 POINT(42.10130664684538 -88.4966925479531) bank147603 +147604 POINT(41.02382608041177 -87.2607000472854) bank147604 +147605 POINT(42.12958504816217 -86.90801913310703) bank147605 +147606 POINT(41.152884841871966 -87.43001789472585) bank147606 +147607 POINT(42.812408278838 -87.21073686996374) bank147607 +147608 POINT(42.500527898454514 -87.1088854505958) bank147608 +147609 POINT(42.693122767786726 -88.11772831839455) bank147609 +147610 POINT(41.66086912928656 -88.41750892934466) bank147610 +147611 POINT(42.69092750978844 -86.79706476681966) bank147611 +147612 POINT(42.506698498581905 -88.27777581223894) bank147612 +147613 POINT(41.99295217650675 -87.20672568920614) bank147613 +147614 POINT(41.25617714535197 -86.6606424595221) bank147614 +147615 POINT(41.245960855939515 -87.51053841726157) bank147615 +147616 POINT(42.46748092143025 -88.10944723220955) bank147616 +147617 POINT(41.55457983147961 -87.3000279938671) bank147617 +147618 POINT(42.395699660796105 -86.92071901503715) bank147618 +147619 POINT(42.57454947991317 -87.893034476776) bank147619 +147620 POINT(42.16912894524405 -87.21310129497704) bank147620 +147621 POINT(41.15557404551344 -88.43864004164185) bank147621 +147622 POINT(42.13022406986822 -87.27582251701425) bank147622 +147623 POINT(41.08157396214138 -88.34355556999778) bank147623 +147624 POINT(41.38689043899131 -87.24696709615371) bank147624 +147625 POINT(42.408111679001756 -86.95107201871804) bank147625 +147626 POINT(41.00967691286724 -87.81064206447488) bank147626 +147627 POINT(41.46665794451697 -86.79049440687001) bank147627 +147628 POINT(41.67533546178688 -87.05492929618762) bank147628 +147629 POINT(41.1013986673241 -87.46147861926273) bank147629 +147630 POINT(42.05790534810137 -88.02869132078723) bank147630 +147631 POINT(42.066565514525905 -87.82441243453819) bank147631 +147632 POINT(42.49633913049339 -88.36589886149923) bank147632 +147633 POINT(41.592587637290364 -87.53817438465845) bank147633 +147634 POINT(41.93646538801422 -86.66421579435489) bank147634 +147635 POINT(41.863553239070804 -87.71642440255259) bank147635 +147636 POINT(42.624921738250535 -86.83899246003749) bank147636 +147637 POINT(41.81542840305904 -87.26355868724275) bank147637 +147638 POINT(42.4807263157387 -86.77733135802315) bank147638 +147639 POINT(41.96828861632509 -88.59555833307095) bank147639 +147640 POINT(42.85949171602168 -87.4976218064304) bank147640 +147641 POINT(42.095925679197265 -87.22647424036336) bank147641 +147642 POINT(42.262644229004174 -87.36694386810669) bank147642 +147643 POINT(42.7793893240762 -87.82480032767208) bank147643 +147644 POINT(41.66094706652123 -87.29813445014086) bank147644 +147645 POINT(42.467711808131774 -87.62372195500832) bank147645 +147646 POINT(41.53867181060216 -88.53281974396371) bank147646 +147647 POINT(41.79711029748078 -86.69532664072635) bank147647 +147648 POINT(41.54114923609235 -86.75427964717366) bank147648 +147649 POINT(42.651483936074 -88.10409940311587) bank147649 +147650 POINT(41.781056637886394 -87.86383335815482) bank147650 +147651 POINT(41.25475735487022 -87.18866446755167) bank147651 +147652 POINT(42.22458664652947 -86.88972379434603) bank147652 +147653 POINT(41.00481665284378 -86.69581732624614) bank147653 +147654 POINT(42.077354082564185 -88.62241960287265) bank147654 +147655 POINT(42.182565543619056 -86.66011505034014) bank147655 +147656 POINT(42.68180672174763 -87.09445682153004) bank147656 +147657 POINT(42.439595995318406 -86.94289922230392) bank147657 +147658 POINT(42.03324584623838 -87.71621459490044) bank147658 +147659 POINT(41.32845898409833 -86.94546417944784) bank147659 +147660 POINT(41.33148073364615 -86.6504813743754) bank147660 +147661 POINT(42.35180387342549 -87.62904896062737) bank147661 +147662 POINT(41.42291864709366 -88.03517481418064) bank147662 +147663 POINT(42.603991567911066 -87.30415722580841) bank147663 +147664 POINT(41.64506348385935 -87.09411744349389) bank147664 +147665 POINT(40.895282057798454 -87.16260212735241) bank147665 +147666 POINT(41.23444956742321 -86.9784048211808) bank147666 +147667 POINT(41.20135136780021 -86.6572136630532) bank147667 +147668 POINT(41.79337992950703 -88.41985131526356) bank147668 +147669 POINT(41.74979491956672 -87.44669163253343) bank147669 +147670 POINT(42.68842105578701 -87.12965795926948) bank147670 +147671 POINT(42.76307169179768 -86.86112810008417) bank147671 +147672 POINT(41.418827599316664 -88.12667937004197) bank147672 +147673 POINT(41.70862082681344 -87.2737907390789) bank147673 +147674 POINT(41.6796154196501 -88.12314751565648) bank147674 +147675 POINT(42.60638902849772 -86.893096221439) bank147675 +147676 POINT(41.15701422347019 -87.26342195255897) bank147676 +147677 POINT(42.24155545231874 -86.64416928697374) bank147677 +147678 POINT(41.07273217013953 -86.97577726971717) bank147678 +147679 POINT(41.21188422967211 -87.9017880992074) bank147679 +147680 POINT(41.6595426146359 -86.71122274381541) bank147680 +147681 POINT(42.7289257563183 -87.9204549495087) bank147681 +147682 POINT(41.16965242175749 -88.17763653682316) bank147682 +147683 POINT(41.62953734247191 -88.14340829437553) bank147683 +147684 POINT(41.682238693981944 -87.427639949145) bank147684 +147685 POINT(41.09718166256906 -87.13390166715574) bank147685 +147686 POINT(42.79031483714474 -87.97035235198744) bank147686 +147687 POINT(42.1115372058691 -88.06726083338333) bank147687 +147688 POINT(41.91822371890572 -86.68923564246761) bank147688 +147689 POINT(41.54226409385698 -86.92335030192312) bank147689 +147690 POINT(41.352785014241846 -86.90312404995055) bank147690 +147691 POINT(42.76606290814384 -87.26641005653227) bank147691 +147692 POINT(42.83652599724748 -88.37244842124495) bank147692 +147693 POINT(42.45328434706446 -88.59763705097647) bank147693 +147694 POINT(41.96232378824596 -87.18516504446396) bank147694 +147695 POINT(42.48085310433574 -87.38352768414538) bank147695 +147696 POINT(41.24015901266212 -88.06038778274386) bank147696 +147697 POINT(41.140531996295934 -88.58626328956225) bank147697 +147698 POINT(41.87736721970987 -88.45922959937563) bank147698 +147699 POINT(40.98483016263304 -86.77518777673187) bank147699 +147700 POINT(42.68340568611284 -88.40501217656978) bank147700 +147701 POINT(42.70798402194561 -87.05602184890607) bank147701 +147702 POINT(41.68621889924089 -87.0722939014215) bank147702 +147703 POINT(42.077387058831924 -88.28355424619721) bank147703 +147704 POINT(41.12162999763377 -88.59991219632849) bank147704 +147705 POINT(42.027335723974645 -87.4446281936642) bank147705 +147706 POINT(42.71864610687729 -86.69890568670935) bank147706 +147707 POINT(41.26409690424663 -86.7319549433195) bank147707 +147708 POINT(41.63012758132797 -87.8633635500038) bank147708 +147709 POINT(41.74542800265994 -87.24818908634772) bank147709 +147710 POINT(42.19934092804093 -87.92195524809696) bank147710 +147711 POINT(41.63140575900062 -88.27717634910121) bank147711 +147712 POINT(41.432831356758975 -87.36602602908897) bank147712 +147713 POINT(41.620804501202166 -88.22371038867017) bank147713 +147714 POINT(41.24737369498767 -87.59026286158225) bank147714 +147715 POINT(41.649670352897125 -88.42734626587212) bank147715 +147716 POINT(41.81769649950747 -88.09874330359466) bank147716 +147717 POINT(41.78142935738501 -88.04727169392285) bank147717 +147718 POINT(41.93632451463088 -87.412117629645) bank147718 +147719 POINT(42.123348819947395 -88.22211163250958) bank147719 +147720 POINT(41.42297773860811 -87.58027810991204) bank147720 +147721 POINT(42.715083116900104 -86.78866285662828) bank147721 +147722 POINT(41.34976508784106 -87.48585390474727) bank147722 +147723 POINT(41.812731748499075 -87.51258729167942) bank147723 +147724 POINT(41.3340291996545 -88.60718519897064) bank147724 +147725 POINT(41.833166676689785 -87.98241443262744) bank147725 +147726 POINT(41.75460449281024 -88.57698610063872) bank147726 +147727 POINT(42.526175056948354 -87.39154436513424) bank147727 +147728 POINT(41.019119754182476 -88.30408281059859) bank147728 +147729 POINT(41.049260235672136 -88.48363604124526) bank147729 +147730 POINT(41.25074049183207 -88.27507051326964) bank147730 +147731 POINT(42.50832862269071 -86.87606889474621) bank147731 +147732 POINT(40.95721931197763 -88.55802612390474) bank147732 +147733 POINT(41.99937744642401 -87.51761932983368) bank147733 +147734 POINT(42.167234649798274 -87.59150098922396) bank147734 +147735 POINT(40.904066579854906 -87.52390476758775) bank147735 +147736 POINT(42.74115255004426 -87.60590186055099) bank147736 +147737 POINT(41.386745678289465 -87.43356132866725) bank147737 +147738 POINT(41.688292734027776 -86.83788304699321) bank147738 +147739 POINT(42.563674877509946 -87.0061377491623) bank147739 +147740 POINT(42.856239594505595 -87.96288055324209) bank147740 +147741 POINT(41.784902889104124 -88.30568587575104) bank147741 +147742 POINT(42.00307901868176 -88.54907710798986) bank147742 +147743 POINT(41.17523156398476 -87.70991170237069) bank147743 +147744 POINT(41.33853706205906 -88.27773495978339) bank147744 +147745 POINT(42.77338879455751 -87.42240458557565) bank147745 +147746 POINT(42.689346305946444 -86.93740236419839) bank147746 +147747 POINT(42.01362082949195 -86.6965086307827) bank147747 +147748 POINT(42.51903776489602 -87.02876077288605) bank147748 +147749 POINT(41.67029106074321 -87.937106993913) bank147749 +147750 POINT(42.31388962977411 -86.87767002165279) bank147750 +147751 POINT(41.540910416628435 -87.73870326729958) bank147751 +147752 POINT(42.086575306141 -87.75418313679376) bank147752 +147753 POINT(42.81385005366693 -87.06998347180041) bank147753 +147754 POINT(41.60588273405177 -87.1229797324887) bank147754 +147755 POINT(41.725531600570925 -87.11571284441428) bank147755 +147756 POINT(40.904505023971915 -87.9140866668459) bank147756 +147757 POINT(40.91223610819483 -88.57622335906542) bank147757 +147758 POINT(41.250831036487014 -88.29621308920623) bank147758 +147759 POINT(41.53611739016834 -86.83010434186211) bank147759 +147760 POINT(42.81546680027424 -86.93525971009358) bank147760 +147761 POINT(41.19865774482584 -87.62970253335341) bank147761 +147762 POINT(42.18702255902564 -87.52832463500518) bank147762 +147763 POINT(41.016823163679476 -88.52546127353045) bank147763 +147764 POINT(41.41547562351476 -87.51861881414318) bank147764 +147765 POINT(41.34357550152733 -88.42213137516748) bank147765 +147766 POINT(41.415557091315065 -87.60162439584946) bank147766 +147767 POINT(42.536775277934105 -87.11162119055544) bank147767 +147768 POINT(42.366418115887114 -87.44999633600038) bank147768 +147769 POINT(41.71282868409043 -87.90336313359778) bank147769 +147770 POINT(42.635013499748354 -88.29067886057454) bank147770 +147771 POINT(41.33532360769355 -88.04865455180703) bank147771 +147772 POINT(41.841103383552195 -88.23107747083182) bank147772 +147773 POINT(42.78330609795695 -88.30033287063276) bank147773 +147774 POINT(41.97225749220993 -88.430259454849) bank147774 +147775 POINT(42.576947232755295 -87.89282332687817) bank147775 +147776 POINT(42.67878741701361 -88.28682547167955) bank147776 +147777 POINT(42.174335099417135 -87.53328944643552) bank147777 +147778 POINT(41.36556199223541 -87.31671013106158) bank147778 +147779 POINT(41.55076819850655 -87.06111958087693) bank147779 +147780 POINT(42.660290744088904 -87.25344929555004) bank147780 +147781 POINT(42.42461464317867 -87.78048193931464) bank147781 +147782 POINT(41.182978769079206 -87.56374551858957) bank147782 +147783 POINT(41.25316970637767 -87.34273329782039) bank147783 +147784 POINT(42.147077872321454 -87.78247620072977) bank147784 +147785 POINT(41.73464004701397 -86.98783661774799) bank147785 +147786 POINT(40.93428169287184 -87.06291671978904) bank147786 +147787 POINT(42.73473653398204 -88.04826684128281) bank147787 +147788 POINT(41.51539911013615 -86.80999940569585) bank147788 +147789 POINT(42.06885203215801 -86.64368754533383) bank147789 +147790 POINT(41.19763377067033 -88.04591879874276) bank147790 +147791 POINT(42.655011282400885 -87.15560433461283) bank147791 +147792 POINT(42.69117239659483 -87.55887727789924) bank147792 +147793 POINT(41.505927424227266 -87.68830500374796) bank147793 +147794 POINT(42.26008259361585 -87.3972116793351) bank147794 +147795 POINT(42.27932151858382 -86.64654136072608) bank147795 +147796 POINT(41.40865199701118 -88.4468361818706) bank147796 +147797 POINT(41.39082698682577 -87.8944360546176) bank147797 +147798 POINT(41.1282080365136 -87.30273778733958) bank147798 +147799 POINT(41.32902550469337 -87.77371031524439) bank147799 +147800 POINT(42.095946388110015 -87.1735955627279) bank147800 +147801 POINT(41.81089224203384 -88.49861536268384) bank147801 +147802 POINT(42.435812872716404 -87.70886046322887) bank147802 +147803 POINT(42.45162934304858 -86.90391368673836) bank147803 +147804 POINT(42.4415225739329 -87.4475250842457) bank147804 +147805 POINT(42.37192028988788 -87.75295687437854) bank147805 +147806 POINT(41.506246336251536 -86.86190090226029) bank147806 +147807 POINT(42.77951838472938 -87.33193512924018) bank147807 +147808 POINT(42.01725248833881 -87.6352330199269) bank147808 +147809 POINT(41.158278653031225 -87.62180337385351) bank147809 +147810 POINT(42.58988732676835 -86.72357840281742) bank147810 +147811 POINT(42.034333348319436 -87.17797719804236) bank147811 +147812 POINT(42.34271623130291 -86.8188171655101) bank147812 +147813 POINT(41.23295172007398 -87.21779645047974) bank147813 +147814 POINT(41.64999629365231 -87.01367568575473) bank147814 +147815 POINT(42.541959617410086 -88.24307850279548) bank147815 +147816 POINT(41.40686143885946 -87.30513589058941) bank147816 +147817 POINT(41.66043336921385 -88.2551522685938) bank147817 +147818 POINT(41.15147631611445 -88.07072794580408) bank147818 +147819 POINT(41.64207566283001 -88.47132225183508) bank147819 +147820 POINT(41.36687258096191 -87.29527529573352) bank147820 +147821 POINT(41.27419245792053 -87.12604605565525) bank147821 +147822 POINT(42.317772481885584 -88.26909515743935) bank147822 +147823 POINT(41.69852422555547 -88.33428582335186) bank147823 +147824 POINT(40.945524687263635 -88.62383286192764) bank147824 +147825 POINT(41.10622801688867 -87.14484104388667) bank147825 +147826 POINT(42.75967136646606 -86.88391779371734) bank147826 +147827 POINT(41.23614183218289 -86.97711959235502) bank147827 +147828 POINT(41.874312465994045 -87.27525972646475) bank147828 +147829 POINT(42.72310949532752 -87.40604454172669) bank147829 +147830 POINT(42.2103235921486 -87.95584431556233) bank147830 +147831 POINT(41.26962233946417 -88.1086816975023) bank147831 +147832 POINT(41.24863316080169 -87.82073016780068) bank147832 +147833 POINT(41.48584097086319 -88.21869325968444) bank147833 +147834 POINT(42.11555346409415 -88.17333669310031) bank147834 +147835 POINT(42.50729714304769 -88.05279096049672) bank147835 +147836 POINT(42.561683086284724 -86.83240980030257) bank147836 +147837 POINT(40.963055706955245 -87.5069704055486) bank147837 +147838 POINT(41.78375794396609 -87.04921541031099) bank147838 +147839 POINT(41.882367153954846 -88.36760037724241) bank147839 +147840 POINT(41.57241594101547 -86.65462309036445) bank147840 +147841 POINT(41.92343116142522 -87.24766008031393) bank147841 +147842 POINT(41.07674478483702 -87.0284586711811) bank147842 +147843 POINT(41.056943544105565 -88.34567359339066) bank147843 +147844 POINT(42.75075535114897 -88.58235915439066) bank147844 +147845 POINT(41.222719115652424 -86.8609109600041) bank147845 +147846 POINT(41.0567245562537 -86.72086385194442) bank147846 +147847 POINT(42.79657927314566 -88.54441582775091) bank147847 +147848 POINT(41.79252842895422 -87.44915958167373) bank147848 +147849 POINT(41.03945400507196 -88.54866690958008) bank147849 +147850 POINT(41.90851325288769 -86.79556190304099) bank147850 +147851 POINT(41.9723395581895 -87.38502639193966) bank147851 +147852 POINT(41.27223653055793 -87.60293944876436) bank147852 +147853 POINT(42.49906348348526 -88.28342393610293) bank147853 +147854 POINT(41.31542497425019 -86.96605120346392) bank147854 +147855 POINT(41.45274126597241 -87.62137884575951) bank147855 +147856 POINT(41.64270356116219 -87.56992864629309) bank147856 +147857 POINT(41.98460949474584 -88.27011372805488) bank147857 +147858 POINT(40.989312350840095 -88.44758679271736) bank147858 +147859 POINT(41.850621394628604 -87.70962173816011) bank147859 +147860 POINT(42.63745001812306 -87.13145247378195) bank147860 +147861 POINT(42.11897526542345 -86.93823758793194) bank147861 +147862 POINT(41.108266127173735 -88.53127693184716) bank147862 +147863 POINT(41.642888915531195 -88.41330549374857) bank147863 +147864 POINT(41.92038289984573 -88.20824645446024) bank147864 +147865 POINT(41.03375112211687 -87.30902470606391) bank147865 +147866 POINT(42.81609830157453 -86.7097522324321) bank147866 +147867 POINT(42.0017702662517 -86.83065367318898) bank147867 +147868 POINT(41.23767089299084 -87.9096064659863) bank147868 +147869 POINT(41.579369895769865 -87.26754942182302) bank147869 +147870 POINT(42.002573037657555 -86.81475286473157) bank147870 +147871 POINT(42.30233342143238 -88.27507820231978) bank147871 +147872 POINT(42.248895672321 -87.73553054897263) bank147872 +147873 POINT(42.247317761210894 -86.9436487703275) bank147873 +147874 POINT(41.71950340821771 -87.49781047092658) bank147874 +147875 POINT(41.772496281854146 -87.09474734551885) bank147875 +147876 POINT(42.48485255219499 -86.75339635385616) bank147876 +147877 POINT(42.12870405711703 -88.32691301734128) bank147877 +147878 POINT(42.47634646286414 -87.15332199826264) bank147878 +147879 POINT(41.17173374231521 -87.48614447983554) bank147879 +147880 POINT(42.7861624832231 -87.56891332416676) bank147880 +147881 POINT(42.70490460587224 -88.2789492711584) bank147881 +147882 POINT(41.33308446718293 -86.90994753955792) bank147882 +147883 POINT(40.939004822031784 -88.38040939785333) bank147883 +147884 POINT(42.01134904362975 -86.68800726892492) bank147884 +147885 POINT(42.001973998001866 -87.56391709132846) bank147885 +147886 POINT(40.97833086949135 -88.56314838368722) bank147886 +147887 POINT(41.393732529408545 -87.78048401801271) bank147887 +147888 POINT(41.18690746566802 -88.3331585234806) bank147888 +147889 POINT(41.105324723949714 -87.04776317765953) bank147889 +147890 POINT(41.89269383954954 -86.66243064944805) bank147890 +147891 POINT(42.16958428805221 -87.38369093899317) bank147891 +147892 POINT(42.058673838202736 -87.75831417719586) bank147892 +147893 POINT(40.889118836178135 -87.31509178128547) bank147893 +147894 POINT(41.28590931242913 -88.26257357477668) bank147894 +147895 POINT(41.156174670866065 -87.53174742218118) bank147895 +147896 POINT(42.8234886569932 -87.72628887858238) bank147896 +147897 POINT(41.30444375409424 -87.95732683472745) bank147897 +147898 POINT(40.91519604839112 -88.50948427543283) bank147898 +147899 POINT(41.38722291274334 -87.85141588865577) bank147899 +147900 POINT(42.08866288902575 -88.55675016866066) bank147900 +147901 POINT(41.90480826653678 -87.80632705985664) bank147901 +147902 POINT(40.97855454013694 -87.67708770846467) bank147902 +147903 POINT(41.772369745419475 -87.30728395010046) bank147903 +147904 POINT(42.37425532101193 -86.94311162245901) bank147904 +147905 POINT(42.615904566415395 -87.224576369219) bank147905 +147906 POINT(42.12196403740236 -88.59307319209455) bank147906 +147907 POINT(42.57001752402782 -87.02144369533772) bank147907 +147908 POINT(41.24847565514987 -87.92910691560638) bank147908 +147909 POINT(41.17118143288903 -88.05036000120509) bank147909 +147910 POINT(41.85751031506445 -87.31382662929751) bank147910 +147911 POINT(42.13004037450185 -86.95018891460568) bank147911 +147912 POINT(41.9122142225568 -88.43650651598432) bank147912 +147913 POINT(41.82259502660683 -88.56229462896977) bank147913 +147914 POINT(42.83798285616821 -87.17803837623347) bank147914 +147915 POINT(41.58871766865678 -86.94781713635534) bank147915 +147916 POINT(42.82539156875272 -88.19049238610762) bank147916 +147917 POINT(42.36352284526863 -87.23176262000906) bank147917 +147918 POINT(42.36584118293122 -87.74246354887671) bank147918 +147919 POINT(41.78373458192162 -88.57891312816106) bank147919 +147920 POINT(42.531887877213165 -88.14206900932268) bank147920 +147921 POINT(41.8458469858445 -88.41963559110873) bank147921 +147922 POINT(42.578466950389185 -87.60825035945608) bank147922 +147923 POINT(42.284871409720296 -87.25501658414723) bank147923 +147924 POINT(42.13320592344269 -88.43608816713672) bank147924 +147925 POINT(42.168713283982314 -88.60464357093726) bank147925 +147926 POINT(41.832460445328095 -87.00523603981813) bank147926 +147927 POINT(42.385713111169444 -88.13494100194892) bank147927 +147928 POINT(41.61720987158429 -86.63830870685203) bank147928 +147929 POINT(41.88287087505635 -87.29793121510276) bank147929 +147930 POINT(41.28597412194934 -86.76669413795427) bank147930 +147931 POINT(41.402801256589406 -86.96438616848397) bank147931 +147932 POINT(41.767562683227766 -86.9012441370742) bank147932 +147933 POINT(40.99804431978729 -86.77416060340579) bank147933 +147934 POINT(42.85510907530329 -87.7324784586449) bank147934 +147935 POINT(40.90760474457705 -88.22011416294839) bank147935 +147936 POINT(42.67807557773312 -87.04078613378687) bank147936 +147937 POINT(41.70042945436372 -86.95058350503) bank147937 +147938 POINT(41.800909001918264 -88.59148575595634) bank147938 +147939 POINT(41.354769485192854 -87.35994890687321) bank147939 +147940 POINT(42.06267340073807 -86.91797351759367) bank147940 +147941 POINT(41.679935969299194 -87.92532303764759) bank147941 +147942 POINT(41.5890963798233 -88.28192860417981) bank147942 +147943 POINT(42.14575287945984 -88.4283399749594) bank147943 +147944 POINT(41.65921298457826 -88.50564263740937) bank147944 +147945 POINT(41.9692750959539 -87.48746078579516) bank147945 +147946 POINT(42.31265360344533 -87.57681886375282) bank147946 +147947 POINT(40.93403837642704 -88.33510795641786) bank147947 +147948 POINT(41.505111196675635 -87.07321142649879) bank147948 +147949 POINT(42.33920981673573 -87.07942159305746) bank147949 +147950 POINT(41.54534347000939 -87.64654444084792) bank147950 +147951 POINT(42.765015320017106 -87.99324064866862) bank147951 +147952 POINT(41.90756720938158 -87.82488972982783) bank147952 +147953 POINT(42.31450724660562 -88.24925590243153) bank147953 +147954 POINT(41.544831245294354 -88.22653886921856) bank147954 +147955 POINT(42.85301993976819 -87.92965085255803) bank147955 +147956 POINT(41.08264917358198 -87.42687410618178) bank147956 +147957 POINT(41.07362025685491 -88.05466768040934) bank147957 +147958 POINT(40.901374695177125 -87.92448826965146) bank147958 +147959 POINT(42.05086343616265 -87.3607266857394) bank147959 +147960 POINT(41.34161534568231 -87.30859849352781) bank147960 +147961 POINT(40.893710428523384 -87.82732872803602) bank147961 +147962 POINT(41.84196072583056 -87.5337529186786) bank147962 +147963 POINT(41.33669137193278 -87.27002367411536) bank147963 +147964 POINT(42.77904642837931 -87.54583817741275) bank147964 +147965 POINT(40.97486233747754 -87.67567302964196) bank147965 +147966 POINT(42.644337411669824 -88.24323726000355) bank147966 +147967 POINT(41.64950600593916 -87.51171952458246) bank147967 +147968 POINT(42.37464143376134 -87.34992569730619) bank147968 +147969 POINT(41.41961917851684 -87.87050947246884) bank147969 +147970 POINT(42.7339909838859 -88.59817209501509) bank147970 +147971 POINT(42.32443762992547 -88.10831735264037) bank147971 +147972 POINT(41.5279961412492 -88.30261221546695) bank147972 +147973 POINT(42.60268413571297 -87.11045059899833) bank147973 +147974 POINT(42.7367786677293 -87.3191979533366) bank147974 +147975 POINT(42.717803930931005 -88.35540898308079) bank147975 +147976 POINT(41.44148444006099 -87.23382143965917) bank147976 +147977 POINT(41.407933736963074 -87.79354880395655) bank147977 +147978 POINT(41.57174947027101 -86.86936914178958) bank147978 +147979 POINT(41.1595161980671 -87.97792007423625) bank147979 +147980 POINT(42.14096551367969 -87.39725449411841) bank147980 +147981 POINT(41.51325747120699 -87.50250472953229) bank147981 +147982 POINT(41.669069548423884 -88.2409579710275) bank147982 +147983 POINT(41.703699639824556 -88.14108411846918) bank147983 +147984 POINT(41.026730597280896 -86.9116703393683) bank147984 +147985 POINT(41.97056439887453 -88.50629989202824) bank147985 +147986 POINT(41.076718126203 -88.22196245950188) bank147986 +147987 POINT(41.386864751080935 -86.93736626811709) bank147987 +147988 POINT(42.815608756951406 -88.00779185265144) bank147988 +147989 POINT(41.61430986943225 -88.26842230325288) bank147989 +147990 POINT(41.1279411334317 -88.33638309548375) bank147990 +147991 POINT(41.83700842021753 -87.85781216441288) bank147991 +147992 POINT(41.081625898868744 -87.14196741710721) bank147992 +147993 POINT(41.40357825936485 -87.87936781860144) bank147993 +147994 POINT(41.25580193554598 -87.33329296932321) bank147994 +147995 POINT(41.64511283779039 -88.58020168823059) bank147995 +147996 POINT(41.98297984509032 -87.88779383351878) bank147996 +147997 POINT(41.419435683530956 -87.23159876677202) bank147997 +147998 POINT(41.609566081114714 -87.26870001684088) bank147998 +147999 POINT(41.309358589188896 -87.55866005532002) bank147999 +148000 POINT(41.31920951195542 -88.35409762246293) bank148000 +148001 POINT(41.955870821249505 -86.72587229036058) bank148001 +148002 POINT(42.46640281404261 -87.45430944116188) bank148002 +148003 POINT(41.12014971808215 -87.74088707962855) bank148003 +148004 POINT(41.142437482142036 -87.75080117069837) bank148004 +148005 POINT(41.608829114933094 -88.25005853393087) bank148005 +148006 POINT(42.225368397362764 -87.86687460137085) bank148006 +148007 POINT(42.262558376348274 -87.53224832726764) bank148007 +148008 POINT(41.932710389651135 -88.44599570510806) bank148008 +148009 POINT(42.1265015360072 -86.64079800784158) bank148009 +148010 POINT(41.39766295074437 -87.13161422522481) bank148010 +148011 POINT(42.30119131339704 -87.67480345660607) bank148011 +148012 POINT(42.106067495608805 -87.6500530706063) bank148012 +148013 POINT(42.27965950466615 -86.82148267800147) bank148013 +148014 POINT(42.75301829122621 -87.42391672551763) bank148014 +148015 POINT(42.26601850336244 -86.63012964775763) bank148015 +148016 POINT(41.139123171804584 -86.89405826342882) bank148016 +148017 POINT(41.04410623043706 -87.14880409378401) bank148017 +148018 POINT(42.19105033230952 -87.78372810708169) bank148018 +148019 POINT(42.78743073049197 -86.68998405554271) bank148019 +148020 POINT(42.50181795210509 -87.43302592450645) bank148020 +148021 POINT(41.28820530527047 -87.32471357850241) bank148021 +148022 POINT(41.41178059918339 -87.10845801628724) bank148022 +148023 POINT(41.82998720340621 -86.77870540673317) bank148023 +148024 POINT(42.43268860761542 -87.54577935986809) bank148024 +148025 POINT(42.258412335736935 -87.16522304642614) bank148025 +148026 POINT(42.11903766933581 -88.55075686073579) bank148026 +148027 POINT(41.60179440140619 -88.29366494102818) bank148027 +148028 POINT(41.50198625515726 -87.15510981614511) bank148028 +148029 POINT(42.39863817377408 -87.69750982809764) bank148029 +148030 POINT(41.733237544030246 -88.2838470595516) bank148030 +148031 POINT(41.449926310449555 -88.1450324582936) bank148031 +148032 POINT(41.71422436570316 -88.50529298804507) bank148032 +148033 POINT(41.3285382538668 -86.97187942364444) bank148033 +148034 POINT(40.92085580640527 -87.59573191479244) bank148034 +148035 POINT(42.590149827561504 -88.49398644984558) bank148035 +148036 POINT(42.415511357356195 -88.56167673718392) bank148036 +148037 POINT(41.62306980865967 -87.69243095064432) bank148037 +148038 POINT(40.930704077229365 -86.63150644478954) bank148038 +148039 POINT(42.68238127672825 -88.56513223032037) bank148039 +148040 POINT(41.74555021893336 -88.43319378145875) bank148040 +148041 POINT(41.46567719259473 -87.16395778678434) bank148041 +148042 POINT(41.51325278370184 -87.44363149344177) bank148042 +148043 POINT(42.755907250191534 -88.43499831993688) bank148043 +148044 POINT(42.28954451512458 -87.87477508920462) bank148044 +148045 POINT(42.801041268730316 -87.42899945404059) bank148045 +148046 POINT(41.512423843418944 -87.30586988581985) bank148046 +148047 POINT(42.1542441929212 -88.42699754830575) bank148047 +148048 POINT(41.59504756826867 -86.91990094805418) bank148048 +148049 POINT(42.13879418128104 -87.3351890707264) bank148049 +148050 POINT(41.0034601320079 -87.89103893177621) bank148050 +148051 POINT(41.89950816261664 -87.31649051485753) bank148051 +148052 POINT(41.931322140155366 -87.79907854599224) bank148052 +148053 POINT(42.778058092328436 -88.05073892999692) bank148053 +148054 POINT(42.533462577024096 -87.06589001457593) bank148054 +148055 POINT(41.78965187605703 -86.76873377488897) bank148055 +148056 POINT(41.235294232523444 -87.93046900085436) bank148056 +148057 POINT(41.33228741186755 -87.68361171082095) bank148057 +148058 POINT(42.52333515531015 -87.61642213015858) bank148058 +148059 POINT(41.31182413206032 -87.2044856518351) bank148059 +148060 POINT(41.69599588835082 -87.06533187440421) bank148060 +148061 POINT(41.645570851398034 -87.3090993545855) bank148061 +148062 POINT(42.09287134655168 -88.19205598222813) bank148062 +148063 POINT(42.325953984152164 -86.77442987975788) bank148063 +148064 POINT(40.987864593669094 -86.85196721028932) bank148064 +148065 POINT(42.68454008876354 -88.25977980926635) bank148065 +148066 POINT(41.41423442645263 -87.13570423167852) bank148066 +148067 POINT(41.79659742427473 -87.42462736414787) bank148067 +148068 POINT(42.52833583971626 -86.77608586647663) bank148068 +148069 POINT(42.167519624501374 -88.40722591849885) bank148069 +148070 POINT(41.13391839887556 -87.2797385812304) bank148070 +148071 POINT(42.30066990850355 -87.38684209106117) bank148071 +148072 POINT(42.63738240277537 -87.81463803166895) bank148072 +148073 POINT(41.61487396939574 -88.49441274293638) bank148073 +148074 POINT(41.71389591571339 -88.47684132956185) bank148074 +148075 POINT(41.687445051876566 -86.65460140898608) bank148075 +148076 POINT(41.42164641068466 -88.43506949034023) bank148076 +148077 POINT(41.21222640179145 -88.07653553266606) bank148077 +148078 POINT(42.28761799296212 -87.06225456279233) bank148078 +148079 POINT(42.498364221468464 -88.53934663195643) bank148079 +148080 POINT(41.40793981928758 -87.98719455730777) bank148080 +148081 POINT(41.43201082791208 -87.90554428254516) bank148081 +148082 POINT(41.80529341525324 -87.24561786836153) bank148082 +148083 POINT(41.733675944843064 -88.3483485351711) bank148083 +148084 POINT(41.271195697373635 -87.98029332733573) bank148084 +148085 POINT(42.36199918285313 -87.38105406024971) bank148085 +148086 POINT(42.869337488775805 -87.42304891182118) bank148086 +148087 POINT(41.09789518693981 -88.1611886918263) bank148087 +148088 POINT(41.87247932684699 -88.26725737573955) bank148088 +148089 POINT(41.49636846655963 -87.54089454727122) bank148089 +148090 POINT(42.0950488761927 -87.12334294453949) bank148090 +148091 POINT(42.16211509165494 -87.42028045780594) bank148091 +148092 POINT(41.370232031491106 -88.00785364685997) bank148092 +148093 POINT(42.51222095467895 -88.21398294329177) bank148093 +148094 POINT(42.61349310343423 -88.50853781287421) bank148094 +148095 POINT(42.509626678041045 -88.14098165404647) bank148095 +148096 POINT(41.663005328365784 -87.38092910388434) bank148096 +148097 POINT(42.08031343753637 -88.1108200834568) bank148097 +148098 POINT(42.01567639873426 -88.54066783934745) bank148098 +148099 POINT(41.465347193193956 -88.11632375710639) bank148099 +148100 POINT(42.457846715548655 -86.91390627810836) bank148100 +148101 POINT(42.119822166254714 -86.97541575243895) bank148101 +148102 POINT(42.41072723311582 -88.21195009187552) bank148102 +148103 POINT(41.12581941770362 -86.74138708290256) bank148103 +148104 POINT(41.658659242587184 -88.21082420426067) bank148104 +148105 POINT(41.40397177853847 -86.87753677070073) bank148105 +148106 POINT(41.71673755116257 -87.80319968796934) bank148106 +148107 POINT(41.41997644421205 -87.93703492660529) bank148107 +148108 POINT(40.95365668633752 -88.13775131392015) bank148108 +148109 POINT(41.75597608186128 -87.99009856877133) bank148109 +148110 POINT(41.3605629125844 -87.07715864156606) bank148110 +148111 POINT(42.6539367523088 -88.48727734081497) bank148111 +148112 POINT(42.04989052669911 -87.99474395717567) bank148112 +148113 POINT(41.59252684694427 -86.99769010467585) bank148113 +148114 POINT(42.14229167331321 -88.11984112508463) bank148114 +148115 POINT(41.411918506688956 -87.46979606318449) bank148115 +148116 POINT(41.07584247154571 -87.7836385218677) bank148116 +148117 POINT(42.344379383172004 -87.7236313231775) bank148117 +148118 POINT(42.25161311482232 -86.89085165813202) bank148118 +148119 POINT(41.67502985938578 -87.82390492460489) bank148119 +148120 POINT(40.94584103227951 -86.77257468081633) bank148120 +148121 POINT(42.596540312966965 -88.11361952395544) bank148121 +148122 POINT(42.17490645325155 -87.06418660458681) bank148122 +148123 POINT(42.489628120593615 -88.21976850956081) bank148123 +148124 POINT(41.70121000771208 -86.78186175968709) bank148124 +148125 POINT(41.38739119954063 -87.86963978624128) bank148125 +148126 POINT(42.15931179547071 -87.80759591580164) bank148126 +148127 POINT(41.9596866729339 -88.40544313015904) bank148127 +148128 POINT(41.28039519808131 -86.88935125158658) bank148128 +148129 POINT(41.112690156842376 -88.60164446037696) bank148129 +148130 POINT(42.529463865710255 -86.92118391283475) bank148130 +148131 POINT(42.20503476629074 -87.47228253728092) bank148131 +148132 POINT(41.45305040113989 -87.8508657614911) bank148132 +148133 POINT(41.681408527221244 -87.52189131488407) bank148133 +148134 POINT(42.43405588307817 -86.99005734794812) bank148134 +148135 POINT(42.448825722458665 -87.3568571496814) bank148135 +148136 POINT(41.05064642893176 -87.89378764897772) bank148136 +148137 POINT(41.46632798437756 -87.70580488258219) bank148137 +148138 POINT(40.91219835621659 -86.68715518692602) bank148138 +148139 POINT(41.72724177401084 -88.59793771623006) bank148139 +148140 POINT(41.43361590435729 -87.6799778002505) bank148140 +148141 POINT(40.89074098936932 -87.45504194146955) bank148141 +148142 POINT(42.0698563483203 -87.48944351698098) bank148142 +148143 POINT(41.033266850620016 -87.77887039967001) bank148143 +148144 POINT(41.73997842757542 -88.39004344441535) bank148144 +148145 POINT(41.865063336936664 -86.81429214237288) bank148145 +148146 POINT(40.93901466817038 -86.70721836696009) bank148146 +148147 POINT(42.459564901455806 -87.41876760803132) bank148147 +148148 POINT(42.24410986945513 -88.16571939281546) bank148148 +148149 POINT(41.68327351095261 -88.4533734023653) bank148149 +148150 POINT(41.411719308260395 -87.49295437934177) bank148150 +148151 POINT(42.32046402917417 -87.03662695184742) bank148151 +148152 POINT(42.72931797146717 -87.36745586650383) bank148152 +148153 POINT(42.24400146812186 -88.51908962386761) bank148153 +148154 POINT(42.07673826174269 -88.04030923976637) bank148154 +148155 POINT(41.686765892762146 -87.61132039485948) bank148155 +148156 POINT(40.96899042335045 -88.41513766268034) bank148156 +148157 POINT(42.07701590333666 -88.5747376607576) bank148157 +148158 POINT(42.4527767239852 -88.45896433689273) bank148158 +148159 POINT(42.136994971726956 -87.2590402737417) bank148159 +148160 POINT(41.996005506636386 -87.45208345982128) bank148160 +148161 POINT(41.985465035319976 -86.80619472015658) bank148161 +148162 POINT(40.99246487141802 -86.91074844484795) bank148162 +148163 POINT(42.1513083280364 -87.523103497514) bank148163 +148164 POINT(42.75481780296397 -88.6039055036841) bank148164 +148165 POINT(41.73365168579999 -87.04435312027775) bank148165 +148166 POINT(42.12573105497277 -87.13858462671186) bank148166 +148167 POINT(42.366202975426035 -86.67945894223247) bank148167 +148168 POINT(42.39072942931061 -88.52865742637051) bank148168 +148169 POINT(42.304392013589585 -88.4416349508667) bank148169 +148170 POINT(41.99139375266674 -88.60425231824246) bank148170 +148171 POINT(42.16197011956976 -86.74851053108894) bank148171 +148172 POINT(42.09166635247517 -88.27745103042183) bank148172 +148173 POINT(41.45179586035549 -88.33557716075624) bank148173 +148174 POINT(41.93743329264039 -87.77384200487461) bank148174 +148175 POINT(42.693979016176165 -87.16220636874675) bank148175 +148176 POINT(42.70705170735205 -87.56149202061043) bank148176 +148177 POINT(41.64737144838585 -87.5238304341288) bank148177 +148178 POINT(41.80307213712314 -86.73363189538614) bank148178 +148179 POINT(41.559588479184015 -87.18607010914134) bank148179 +148180 POINT(42.532697937444006 -87.41943530957931) bank148180 +148181 POINT(41.57528851707352 -87.93642042543604) bank148181 +148182 POINT(41.993109016432925 -87.1291618915744) bank148182 +148183 POINT(42.088505273640685 -87.12097802640726) bank148183 +148184 POINT(42.253210455459616 -88.44517488812933) bank148184 +148185 POINT(41.637903321234774 -88.25599616327872) bank148185 +148186 POINT(42.71769560666819 -87.7281741194896) bank148186 +148187 POINT(41.84692279242053 -86.81053388626464) bank148187 +148188 POINT(42.05532196087676 -86.73508305204791) bank148188 +148189 POINT(41.0477462193787 -87.1743596636508) bank148189 +148190 POINT(41.53261712317031 -87.76888937572515) bank148190 +148191 POINT(42.134803399222506 -88.41283710097673) bank148191 +148192 POINT(42.05253337068256 -87.83269605218827) bank148192 +148193 POINT(41.951539103241466 -87.31266525575043) bank148193 +148194 POINT(42.29348538439145 -87.62651599431042) bank148194 +148195 POINT(42.86796065414735 -87.03931714090756) bank148195 +148196 POINT(42.761797514484144 -88.44268853807202) bank148196 +148197 POINT(41.59863319906855 -87.78973731023831) bank148197 +148198 POINT(41.87273148830748 -86.79598441041546) bank148198 +148199 POINT(42.49452406549816 -87.52158976348376) bank148199 +148200 POINT(41.30674913230485 -87.44251400493667) bank148200 +148201 POINT(42.15467604420369 -87.60489318049288) bank148201 +148202 POINT(41.943088158690095 -86.72250236343443) bank148202 +148203 POINT(41.78999846467668 -87.96000780679317) bank148203 +148204 POINT(41.0433333253635 -86.83428805563847) bank148204 +148205 POINT(41.344032269940826 -86.63173521951398) bank148205 +148206 POINT(42.794865211325785 -87.85570787655136) bank148206 +148207 POINT(42.71813887325265 -88.30501150800018) bank148207 +148208 POINT(40.89990604363443 -88.3454534478686) bank148208 +148209 POINT(41.860328697525276 -86.9837633513226) bank148209 +148210 POINT(41.45031396518149 -88.54544344758398) bank148210 +148211 POINT(40.98467230356171 -87.05908476742383) bank148211 +148212 POINT(41.47879627093756 -88.16472973494692) bank148212 +148213 POINT(41.33181146933156 -88.26404913123905) bank148213 +148214 POINT(40.92696096890367 -87.64149757988714) bank148214 +148215 POINT(41.431495323610534 -87.61053656847245) bank148215 +148216 POINT(41.67947906166404 -87.04487754163792) bank148216 +148217 POINT(41.044251341627636 -88.48876562563855) bank148217 +148218 POINT(41.630164022333844 -87.09728182378574) bank148218 +148219 POINT(41.385117245276504 -88.29664323912016) bank148219 +148220 POINT(41.04971861240263 -87.72646156156273) bank148220 +148221 POINT(42.769425538927806 -87.44162461821992) bank148221 +148222 POINT(42.64530845057812 -86.6350431899916) bank148222 +148223 POINT(41.75739384198021 -87.889275928774) bank148223 +148224 POINT(41.192817425231375 -88.27195387780637) bank148224 +148225 POINT(41.6385887454793 -87.22432275786835) bank148225 +148226 POINT(42.03498918726163 -86.84246037125132) bank148226 +148227 POINT(42.40872956604818 -88.17224844878628) bank148227 +148228 POINT(41.31598194810568 -87.71369501896656) bank148228 +148229 POINT(42.39377801863781 -86.67215073318744) bank148229 +148230 POINT(42.36525117258094 -87.23381532456519) bank148230 +148231 POINT(41.66578450864917 -88.14884495439361) bank148231 +148232 POINT(41.998301099707405 -87.26150103197757) bank148232 +148233 POINT(41.77488690881153 -88.10187565619172) bank148233 +148234 POINT(42.80886671737618 -86.8323282168559) bank148234 +148235 POINT(42.53569879277102 -87.88512688547375) bank148235 +148236 POINT(41.50368127070407 -88.03344735638603) bank148236 +148237 POINT(42.249425651138885 -88.35088831591044) bank148237 +148238 POINT(41.94676826799379 -88.38263303927916) bank148238 +148239 POINT(42.06425395494085 -87.2818493863535) bank148239 +148240 POINT(42.54295958058514 -87.52480617338558) bank148240 +148241 POINT(42.54985873173235 -86.94712813573881) bank148241 +148242 POINT(41.59567138866931 -87.69744842544405) bank148242 +148243 POINT(41.665729229396774 -88.11303305616373) bank148243 +148244 POINT(40.94514731749999 -88.38865549813703) bank148244 +148245 POINT(42.29399343594212 -88.30654808745234) bank148245 +148246 POINT(41.623899322523826 -87.39328522956995) bank148246 +148247 POINT(42.45035156612319 -87.14920356163006) bank148247 +148248 POINT(40.97439269516569 -86.74096650910383) bank148248 +148249 POINT(41.222623847324904 -88.23380403622394) bank148249 +148250 POINT(41.68872116185521 -88.50368700023466) bank148250 +148251 POINT(41.52839902009259 -88.46029405318384) bank148251 +148252 POINT(41.04299227966006 -87.96492799753308) bank148252 +148253 POINT(41.994452605884696 -87.75575791183904) bank148253 +148254 POINT(41.34206880521695 -86.93938518676052) bank148254 +148255 POINT(41.82734974650542 -87.54129608035966) bank148255 +148256 POINT(42.85766836457057 -87.1842800939577) bank148256 +148257 POINT(42.86371815083274 -86.9668757888782) bank148257 +148258 POINT(41.893801264178975 -86.825865631259) bank148258 +148259 POINT(42.67530445327412 -88.13876478938879) bank148259 +148260 POINT(42.28195126252417 -88.22993640420184) bank148260 +148261 POINT(41.91805491329484 -87.630149120277) bank148261 +148262 POINT(41.16067731455144 -87.32158583883776) bank148262 +148263 POINT(42.24664383886548 -87.30282806481074) bank148263 +148264 POINT(41.35114813530018 -86.8982810784432) bank148264 +148265 POINT(42.713028498418716 -87.02350151697529) bank148265 +148266 POINT(42.04047316883558 -87.95153361922952) bank148266 +148267 POINT(41.9069814276019 -87.75394867319956) bank148267 +148268 POINT(41.2573324735898 -88.07397051055067) bank148268 +148269 POINT(42.26469474870722 -88.44957202087576) bank148269 +148270 POINT(41.12503094743846 -87.46558294722783) bank148270 +148271 POINT(42.757738932771844 -87.53353813554327) bank148271 +148272 POINT(41.172522900054055 -88.5646558043275) bank148272 +148273 POINT(41.047705855844214 -86.83813620913095) bank148273 +148274 POINT(42.65121340226917 -88.44274852787031) bank148274 +148275 POINT(42.19226746680283 -88.2940040190556) bank148275 +148276 POINT(41.55653048120879 -87.18715532704616) bank148276 +148277 POINT(42.34447304087401 -87.02671897301364) bank148277 +148278 POINT(42.65952870234783 -88.16015258673745) bank148278 +148279 POINT(41.339062483820804 -88.10449253027069) bank148279 +148280 POINT(40.943365856269665 -87.84232520971742) bank148280 +148281 POINT(42.239223798869226 -86.94691727086182) bank148281 +148282 POINT(42.85354427714726 -88.58078357409264) bank148282 +148283 POINT(41.41252202091049 -87.8815957332833) bank148283 +148284 POINT(41.218949387520155 -87.00015821179609) bank148284 +148285 POINT(41.752745755987746 -88.38424527405888) bank148285 +148286 POINT(42.59900436500217 -87.59026041438305) bank148286 +148287 POINT(42.71842143537729 -88.19275993033072) bank148287 +148288 POINT(41.246554591654736 -87.89600551959072) bank148288 +148289 POINT(41.396204220931196 -88.12374209505977) bank148289 +148290 POINT(41.2437926685627 -86.84223889474895) bank148290 +148291 POINT(40.91362862226512 -87.40971040205179) bank148291 +148292 POINT(42.80067667944239 -87.24249001619124) bank148292 +148293 POINT(42.219300142164855 -88.23083094541143) bank148293 +148294 POINT(41.04624907734502 -86.7635500366664) bank148294 +148295 POINT(40.89382138444398 -86.87864432755391) bank148295 +148296 POINT(42.531032244433185 -86.65500790499651) bank148296 +148297 POINT(42.75113769080609 -88.61313263492865) bank148297 +148298 POINT(41.15597150351974 -88.55885775313273) bank148298 +148299 POINT(42.19809318650053 -87.34581647154573) bank148299 +148300 POINT(42.276604296951525 -87.54347743411475) bank148300 +148301 POINT(41.11835585861711 -87.98017443733357) bank148301 +148302 POINT(42.34124889451401 -88.53342392299595) bank148302 +148303 POINT(41.38499326562643 -88.01581025912814) bank148303 +148304 POINT(41.779218114791796 -87.7622415729225) bank148304 +148305 POINT(42.80802008784649 -87.99306987408272) bank148305 +148306 POINT(42.12528528653855 -87.31901795644409) bank148306 +148307 POINT(41.6990367312197 -88.07502543871098) bank148307 +148308 POINT(42.14765000837469 -87.87646744874614) bank148308 +148309 POINT(41.508915436000265 -88.19336468638092) bank148309 +148310 POINT(42.27983688193352 -86.91856259536821) bank148310 +148311 POINT(40.92193619407363 -87.27595923598783) bank148311 +148312 POINT(42.5576692699693 -87.58821409382868) bank148312 +148313 POINT(42.26669485647908 -87.56302802186492) bank148313 +148314 POINT(42.828339570191204 -87.89544897231379) bank148314 +148315 POINT(41.80236271229836 -86.97879273440266) bank148315 +148316 POINT(41.54215392993507 -88.48341734367925) bank148316 +148317 POINT(42.540636070422885 -87.34105310353509) bank148317 +148318 POINT(41.14808410985988 -87.8058301841234) bank148318 +148319 POINT(42.87127853773767 -87.86340476209018) bank148319 +148320 POINT(42.365683666589504 -88.30451687656861) bank148320 +148321 POINT(41.80553701709209 -87.812703983118) bank148321 +148322 POINT(42.46443872630894 -87.0610369455497) bank148322 +148323 POINT(42.16540276863215 -87.43762343289752) bank148323 +148324 POINT(42.632311751853706 -88.04493924219732) bank148324 +148325 POINT(42.577159821353845 -88.1516655508974) bank148325 +148326 POINT(42.13844735844845 -87.1527760367907) bank148326 +148327 POINT(41.62031543307279 -87.45054354210342) bank148327 +148328 POINT(41.709414626353194 -86.98959071596983) bank148328 +148329 POINT(41.26627637726683 -86.86705701269713) bank148329 +148330 POINT(40.93550649278312 -87.79275409370211) bank148330 +148331 POINT(41.94761560110698 -86.7920238732289) bank148331 +148332 POINT(42.58387921449685 -88.12898534124038) bank148332 +148333 POINT(42.36603896809467 -86.90087174954763) bank148333 +148334 POINT(41.889871783122004 -87.86487625651414) bank148334 +148335 POINT(41.957340668069676 -88.38560050310585) bank148335 +148336 POINT(42.02092135528948 -87.90390335198295) bank148336 +148337 POINT(42.50896166195103 -88.26625291930294) bank148337 +148338 POINT(42.82150665592811 -88.17467181183306) bank148338 +148339 POINT(42.59392507896993 -88.41487579461912) bank148339 +148340 POINT(41.48891703315656 -87.30872954546011) bank148340 +148341 POINT(42.81781005801712 -88.34638809894746) bank148341 +148342 POINT(42.4148284262146 -88.25602840765762) bank148342 +148343 POINT(40.95645473370854 -88.01341917981452) bank148343 +148344 POINT(41.954908122622044 -86.98620587473604) bank148344 +148345 POINT(40.9449130454733 -86.97552130650215) bank148345 +148346 POINT(41.05748390121396 -88.34542330191321) bank148346 +148347 POINT(42.606764408287006 -87.46646542661962) bank148347 +148348 POINT(42.57583960028229 -86.70182487108264) bank148348 +148349 POINT(41.730976486310205 -86.8568340566515) bank148349 +148350 POINT(41.82643685197442 -87.87474742069033) bank148350 +148351 POINT(42.12716167923147 -88.18687958695047) bank148351 +148352 POINT(41.409165162408605 -88.03090979845483) bank148352 +148353 POINT(42.696272295142705 -88.34282606230293) bank148353 +148354 POINT(40.93301144669958 -87.88597279059492) bank148354 +148355 POINT(42.0906507373631 -87.37632606222833) bank148355 +148356 POINT(41.244965459266005 -87.85825594144231) bank148356 +148357 POINT(41.05208102017392 -88.00147663905584) bank148357 +148358 POINT(40.90705809573134 -88.13874240333952) bank148358 +148359 POINT(42.29862322094706 -88.45178623957378) bank148359 +148360 POINT(41.897248106077804 -86.64437816954498) bank148360 +148361 POINT(41.164522428450574 -87.81285254704594) bank148361 +148362 POINT(41.015186793075095 -87.99081594580944) bank148362 +148363 POINT(42.04287044517079 -86.74298451768946) bank148363 +148364 POINT(42.613627522602194 -88.31668197014481) bank148364 +148365 POINT(42.35791588932648 -86.90783231758083) bank148365 +148366 POINT(42.866586167336735 -87.27081202376152) bank148366 +148367 POINT(41.64026467122243 -87.14664247107673) bank148367 +148368 POINT(41.5570264014069 -87.76089018441633) bank148368 +148369 POINT(40.99860000941115 -86.8673721091792) bank148369 +148370 POINT(41.81344085497702 -86.82010167517167) bank148370 +148371 POINT(41.49218058982789 -88.18348282385605) bank148371 +148372 POINT(42.66527053178025 -88.05124239246699) bank148372 +148373 POINT(42.42752038522039 -87.74323462324183) bank148373 +148374 POINT(42.372536458937745 -87.08193377206736) bank148374 +148375 POINT(41.40245263776806 -87.99281342962578) bank148375 +148376 POINT(41.11600367334855 -87.49553200547618) bank148376 +148377 POINT(42.16962224851768 -87.23938251503729) bank148377 +148378 POINT(41.586877410777895 -87.1882808276505) bank148378 +148379 POINT(42.60142633886895 -88.35913981270386) bank148379 +148380 POINT(42.512017975539976 -87.56349934693864) bank148380 +148381 POINT(41.63967125179095 -88.08785223116519) bank148381 +148382 POINT(42.70749654577301 -87.65737407573896) bank148382 +148383 POINT(42.82389689805861 -88.59216625321324) bank148383 +148384 POINT(42.592337164288914 -86.68499989913853) bank148384 +148385 POINT(42.743192505764945 -87.98748934529554) bank148385 +148386 POINT(41.20309437545782 -88.0646947775588) bank148386 +148387 POINT(42.566445881826255 -86.8528431385136) bank148387 +148388 POINT(42.6689566052931 -86.782828882963) bank148388 +148389 POINT(41.2215376523626 -86.87854930064191) bank148389 +148390 POINT(41.585556161530036 -87.1800016094643) bank148390 +148391 POINT(40.902995148717814 -86.88008316106202) bank148391 +148392 POINT(41.98144720776129 -87.20447493240232) bank148392 +148393 POINT(40.90506443843629 -88.08214065250331) bank148393 +148394 POINT(41.951531294807836 -87.21200134892233) bank148394 +148395 POINT(42.38287694913377 -87.83318190514633) bank148395 +148396 POINT(41.987386311109596 -87.06294065468894) bank148396 +148397 POINT(41.043795985553096 -87.73320674385664) bank148397 +148398 POINT(41.32941007399384 -87.140928952806) bank148398 +148399 POINT(41.99197532339033 -87.89582856480865) bank148399 +148400 POINT(41.78126633300447 -87.57422969106852) bank148400 +148401 POINT(41.89538339091259 -86.85863327483968) bank148401 +148402 POINT(42.12339208279893 -87.9928117816147) bank148402 +148403 POINT(40.969795691862075 -86.74495238943625) bank148403 +148404 POINT(41.73616455989496 -88.34327529387294) bank148404 +148405 POINT(41.819112233608436 -88.08231087016954) bank148405 +148406 POINT(41.926766423073495 -87.06486389269303) bank148406 +148407 POINT(41.205951261309686 -87.77522886383113) bank148407 +148408 POINT(40.969238741740355 -86.95783246497703) bank148408 +148409 POINT(41.21871775136986 -87.59488575816059) bank148409 +148410 POINT(41.18342493368905 -88.05192244199664) bank148410 +148411 POINT(42.60717763414642 -88.32633927265591) bank148411 +148412 POINT(40.924195890235616 -87.39773708786763) bank148412 +148413 POINT(42.407279556970884 -86.8327046018264) bank148413 +148414 POINT(41.52477423322041 -87.36241066269771) bank148414 +148415 POINT(41.28699287809115 -87.5461158476277) bank148415 +148416 POINT(41.51960734947126 -87.17002239249695) bank148416 +148417 POINT(42.214283611508215 -88.0225815261033) bank148417 +148418 POINT(42.217932296209284 -87.20099622194479) bank148418 +148419 POINT(41.9616971466983 -88.5328591111674) bank148419 +148420 POINT(41.0310318846772 -88.6181676323004) bank148420 +148421 POINT(42.19332724657239 -86.83615978324035) bank148421 +148422 POINT(41.808261381654305 -87.90436699938887) bank148422 +148423 POINT(41.50975434636581 -88.20267805360872) bank148423 +148424 POINT(42.78210657880255 -87.82149119007015) bank148424 +148425 POINT(41.04699596990143 -87.6147014576226) bank148425 +148426 POINT(41.7318403792911 -88.3770526882098) bank148426 +148427 POINT(40.94872616084981 -88.08213272700164) bank148427 +148428 POINT(41.8036517462006 -87.97497911184394) bank148428 +148429 POINT(41.336978842791424 -87.01407361177645) bank148429 +148430 POINT(42.852136852592785 -87.3427505891486) bank148430 +148431 POINT(42.205799808030264 -88.28141955432777) bank148431 +148432 POINT(42.003706211791894 -87.75552785070198) bank148432 +148433 POINT(41.12963021582859 -88.62176032873312) bank148433 +148434 POINT(41.860763655053056 -87.9088222875424) bank148434 +148435 POINT(42.66711938531933 -87.11685196466978) bank148435 +148436 POINT(42.26097937512632 -87.03521073786735) bank148436 +148437 POINT(41.580314695320844 -87.81002039522689) bank148437 +148438 POINT(42.16315620584417 -87.99396971342485) bank148438 +148439 POINT(41.26387805417242 -88.4034141431237) bank148439 +148440 POINT(42.541677718605186 -88.31045874511823) bank148440 +148441 POINT(42.666898793814426 -87.06527520403975) bank148441 +148442 POINT(41.38252220224534 -88.0592299017179) bank148442 +148443 POINT(41.66534242185154 -88.51828926028902) bank148443 +148444 POINT(40.99805758383893 -87.41047447495438) bank148444 +148445 POINT(40.987175670039704 -86.88181893714531) bank148445 +148446 POINT(42.278262956279114 -88.11177773076211) bank148446 +148447 POINT(41.57990854850043 -87.12100379076664) bank148447 +148448 POINT(42.70086264797982 -87.37617571861861) bank148448 +148449 POINT(41.336844532502845 -87.49094629860228) bank148449 +148450 POINT(41.63991082276667 -87.28078145966532) bank148450 +148451 POINT(42.105052267058014 -87.90683404385152) bank148451 +148452 POINT(42.681848393871554 -87.81602208914454) bank148452 +148453 POINT(41.168656917845276 -86.72096741921695) bank148453 +148454 POINT(42.2581204902686 -87.32524169118926) bank148454 +148455 POINT(40.92067548042234 -87.55870694326242) bank148455 +148456 POINT(42.62278382016501 -86.87863732711077) bank148456 +148457 POINT(42.467587497740816 -86.90230423906564) bank148457 +148458 POINT(42.05563062712031 -87.4592843905852) bank148458 +148459 POINT(42.65897344914838 -87.04800804036222) bank148459 +148460 POINT(41.63284788611365 -87.6631409907141) bank148460 +148461 POINT(41.85616592263434 -88.50221354021619) bank148461 +148462 POINT(42.82809467845304 -86.78828048639842) bank148462 +148463 POINT(41.38552904764915 -88.3915819523452) bank148463 +148464 POINT(40.919635812101376 -87.88479450858918) bank148464 +148465 POINT(42.75510764492777 -87.28876127621315) bank148465 +148466 POINT(40.961457450874406 -87.81499049148125) bank148466 +148467 POINT(41.949602889651 -88.42891773533421) bank148467 +148468 POINT(41.13990014556813 -87.74195500929194) bank148468 +148469 POINT(42.687273855083724 -87.50360109655195) bank148469 +148470 POINT(42.30014046099728 -86.88917695113294) bank148470 +148471 POINT(42.72968833283981 -87.17141148311639) bank148471 +148472 POINT(41.00077152305102 -87.3477669359023) bank148472 +148473 POINT(42.02236395583462 -87.17822003615944) bank148473 +148474 POINT(41.50868586333391 -88.56972950296623) bank148474 +148475 POINT(42.8679178984749 -87.45119109157183) bank148475 +148476 POINT(41.21692197548246 -88.07290870405404) bank148476 +148477 POINT(42.820072002987686 -87.75648724529076) bank148477 +148478 POINT(42.81541558884277 -88.28514974985626) bank148478 +148479 POINT(41.96874569070482 -86.93917104332382) bank148479 +148480 POINT(42.25933627124859 -86.91677724662927) bank148480 +148481 POINT(42.83974572459108 -87.99272213071706) bank148481 +148482 POINT(42.592761923741946 -86.90456476972487) bank148482 +148483 POINT(42.40592948906779 -87.73635525081893) bank148483 +148484 POINT(42.08452921992406 -88.60417674322231) bank148484 +148485 POINT(41.706734155944076 -86.80301676016842) bank148485 +148486 POINT(42.052905269185615 -87.29406007254893) bank148486 +148487 POINT(42.23842263235453 -88.41026073466016) bank148487 +148488 POINT(42.37925722440052 -87.0024392262299) bank148488 +148489 POINT(42.84542152077225 -87.88732444635579) bank148489 +148490 POINT(41.5125742070052 -87.45138549129021) bank148490 +148491 POINT(42.395874375466136 -87.58735125146805) bank148491 +148492 POINT(41.90887319796266 -86.68294668823977) bank148492 +148493 POINT(41.52991289010116 -87.00700121330762) bank148493 +148494 POINT(41.43204157733459 -86.95310431280227) bank148494 +148495 POINT(41.99923909167501 -87.58652484333567) bank148495 +148496 POINT(40.93287357136796 -87.98874929994116) bank148496 +148497 POINT(41.423169639148036 -87.7281099399718) bank148497 +148498 POINT(41.335109429910226 -87.98153343942315) bank148498 +148499 POINT(41.36023433613554 -87.36208921704201) bank148499 +148500 POINT(41.03802302279558 -87.69973946327987) bank148500 +148501 POINT(40.93100241444615 -88.60348071192294) bank148501 +148502 POINT(42.255414071182976 -87.69669053088516) bank148502 +148503 POINT(41.92446110677081 -88.2741913746394) bank148503 +148504 POINT(41.916958368916596 -87.96253749982421) bank148504 +148505 POINT(41.11153574522649 -87.59500260511392) bank148505 +148506 POINT(41.217222629179695 -86.81288701324193) bank148506 +148507 POINT(42.74457614437945 -87.69882751337026) bank148507 +148508 POINT(42.83341269303206 -88.31024643654544) bank148508 +148509 POINT(41.799174884054196 -87.84979555655156) bank148509 +148510 POINT(41.057800878403974 -86.70915013459994) bank148510 +148511 POINT(41.663668133685356 -87.34752569073918) bank148511 +148512 POINT(41.13527455979738 -87.94268434764932) bank148512 +148513 POINT(40.965702081866766 -87.04558920265949) bank148513 +148514 POINT(40.970753039104565 -88.24492068639032) bank148514 +148515 POINT(41.881727836595154 -87.31338082632692) bank148515 +148516 POINT(42.66979333809658 -87.57060540729422) bank148516 +148517 POINT(41.83635180758855 -86.91066535512257) bank148517 +148518 POINT(41.427494245732156 -87.98075110610193) bank148518 +148519 POINT(42.41032977417995 -88.41006640037187) bank148519 +148520 POINT(42.390073222104164 -87.89824910502465) bank148520 +148521 POINT(42.81706439947162 -87.05914367597788) bank148521 +148522 POINT(41.628266092768804 -87.4375502899166) bank148522 +148523 POINT(41.27625956940369 -87.25850909762875) bank148523 +148524 POINT(41.8766095978476 -87.92660412862466) bank148524 +148525 POINT(41.2051381070648 -87.22741736803837) bank148525 +148526 POINT(42.435397213273724 -87.38056103194867) bank148526 +148527 POINT(42.20807525752889 -86.82742026514609) bank148527 +148528 POINT(42.18154908860487 -88.5358799872313) bank148528 +148529 POINT(42.56933591165552 -88.20175963578262) bank148529 +148530 POINT(42.002688662769536 -87.58503267823839) bank148530 +148531 POINT(41.36766877916858 -86.8715619592045) bank148531 +148532 POINT(41.07359065603084 -86.63046476621486) bank148532 +148533 POINT(41.494740446619446 -86.89874088137232) bank148533 +148534 POINT(42.86983294918688 -88.37708152957487) bank148534 +148535 POINT(41.23577060425729 -88.2583870781671) bank148535 +148536 POINT(42.39560599221578 -87.55004892200353) bank148536 +148537 POINT(42.60668375426127 -86.69438680941091) bank148537 +148538 POINT(41.40939580995349 -87.79173878870293) bank148538 +148539 POINT(42.58042109069859 -87.03066827231056) bank148539 +148540 POINT(42.272800272518445 -86.70711374315994) bank148540 +148541 POINT(42.399616963645585 -86.92382456640335) bank148541 +148542 POINT(41.527622946137186 -88.51313492680237) bank148542 +148543 POINT(42.86680814369709 -87.10845023939858) bank148543 +148544 POINT(41.71676724774736 -86.69365248478101) bank148544 +148545 POINT(41.06614622434407 -87.88376301085927) bank148545 +148546 POINT(42.606208071923135 -88.12914390285742) bank148546 +148547 POINT(41.02095198986416 -87.0074847625944) bank148547 +148548 POINT(42.645511427914464 -87.93674549249452) bank148548 +148549 POINT(42.41711813433034 -86.89293323137548) bank148549 +148550 POINT(41.242859674645054 -87.09952677592219) bank148550 +148551 POINT(41.939732353866425 -87.18506785970285) bank148551 +148552 POINT(41.058167082619995 -87.58260913216456) bank148552 +148553 POINT(42.661476228232175 -87.42271457557449) bank148553 +148554 POINT(42.587027090563794 -87.36988612412974) bank148554 +148555 POINT(42.67479469216827 -86.80449017523742) bank148555 +148556 POINT(41.71242780616108 -88.62634686715943) bank148556 +148557 POINT(41.04734009380741 -87.79907814869301) bank148557 +148558 POINT(41.441186401361755 -88.05668571645522) bank148558 +148559 POINT(41.1288678366558 -88.558495732127) bank148559 +148560 POINT(42.33844038609631 -87.14595287815189) bank148560 +148561 POINT(41.69497054845662 -87.91582800342985) bank148561 +148562 POINT(42.02911227314871 -86.97710673486851) bank148562 +148563 POINT(42.13532150993813 -87.34638549525702) bank148563 +148564 POINT(41.5565281712939 -87.0066478819504) bank148564 +148565 POINT(41.282743676238056 -87.05819853981203) bank148565 +148566 POINT(41.006828453975096 -87.48972350225642) bank148566 +148567 POINT(42.09727602494043 -87.6852134167232) bank148567 +148568 POINT(41.880280783096055 -88.05492599606114) bank148568 +148569 POINT(42.703177633472464 -86.79981091707369) bank148569 +148570 POINT(41.40129774182015 -87.59865691405491) bank148570 +148571 POINT(41.94846872844315 -88.06810174703543) bank148571 +148572 POINT(42.52784274683702 -87.75948125851721) bank148572 +148573 POINT(41.55065122118314 -87.82388361848483) bank148573 +148574 POINT(41.73662975843012 -86.98067150360018) bank148574 +148575 POINT(41.208026081178765 -87.3169106409115) bank148575 +148576 POINT(41.947127682815655 -86.87777298911729) bank148576 +148577 POINT(42.5112570878603 -88.26632939529996) bank148577 +148578 POINT(41.167614246269395 -86.81915300329682) bank148578 +148579 POINT(41.53954318137615 -86.71059008902449) bank148579 +148580 POINT(42.81890592978907 -86.90554306743292) bank148580 +148581 POINT(42.01704985239004 -87.87976267463057) bank148581 +148582 POINT(42.38891918241871 -87.5626596593539) bank148582 +148583 POINT(41.9380902284281 -88.01449256173682) bank148583 +148584 POINT(42.19953010556129 -86.96706744361656) bank148584 +148585 POINT(42.5609822220149 -86.9952924158805) bank148585 +148586 POINT(41.03303288357936 -86.94793658532295) bank148586 +148587 POINT(41.732447208338954 -86.65669322054868) bank148587 +148588 POINT(42.83307853645513 -86.65120802207369) bank148588 +148589 POINT(41.0990436576579 -86.82034915647446) bank148589 +148590 POINT(41.11671604472067 -87.7836287812052) bank148590 +148591 POINT(41.21517888891479 -88.19764286178462) bank148591 +148592 POINT(41.19041803970359 -88.32449662796992) bank148592 +148593 POINT(42.47133008255651 -88.32421411085997) bank148593 +148594 POINT(42.6955476210069 -86.7446040693649) bank148594 +148595 POINT(42.437797849263276 -86.6667128793206) bank148595 +148596 POINT(42.08398694374593 -87.17042913098624) bank148596 +148597 POINT(42.1697446513195 -87.33912101499456) bank148597 +148598 POINT(42.576124679581646 -88.05251392079352) bank148598 +148599 POINT(41.99626930118633 -87.84171859813205) bank148599 +148600 POINT(42.333008196534685 -87.77279998759869) bank148600 +148601 POINT(42.744853641151174 -87.78873990684544) bank148601 +148602 POINT(41.134668938593364 -87.3490867376398) bank148602 +148603 POINT(42.81530217904733 -88.60217144862095) bank148603 +148604 POINT(41.28936217716383 -86.75397075622412) bank148604 +148605 POINT(41.271329867765004 -86.6706700188491) bank148605 +148606 POINT(41.53391642609707 -87.05861355256081) bank148606 +148607 POINT(41.15437912566062 -86.87747662107951) bank148607 +148608 POINT(41.432267317061594 -87.8620010389092) bank148608 +148609 POINT(41.26189462065095 -86.64125325464245) bank148609 +148610 POINT(42.502062328049114 -87.7441595429013) bank148610 +148611 POINT(41.292992310394574 -88.36829125316027) bank148611 +148612 POINT(42.56224007815426 -88.46535667004864) bank148612 +148613 POINT(41.965844590854985 -88.47613463739546) bank148613 +148614 POINT(40.915368099903695 -87.02056984825433) bank148614 +148615 POINT(41.45928220063221 -86.69243890993954) bank148615 +148616 POINT(42.72458331858973 -88.58496259313077) bank148616 +148617 POINT(41.656290671322125 -87.6224613730667) bank148617 +148618 POINT(41.59147605332735 -86.67380394747823) bank148618 +148619 POINT(42.32660453906375 -87.84353103437194) bank148619 +148620 POINT(42.73001564388534 -86.81529165795457) bank148620 +148621 POINT(42.54765330177717 -86.82595496489498) bank148621 +148622 POINT(42.621734129778325 -86.63872922066557) bank148622 +148623 POINT(41.285784530973636 -87.37154931397683) bank148623 +148624 POINT(42.64761270394425 -87.93888392692969) bank148624 +148625 POINT(42.41175071499373 -87.87908109851168) bank148625 +148626 POINT(41.67587179170108 -88.34504463096738) bank148626 +148627 POINT(42.00149561899375 -87.88789046496008) bank148627 +148628 POINT(42.0221117089026 -86.92833471148204) bank148628 +148629 POINT(42.52093852851412 -87.98028741169338) bank148629 +148630 POINT(41.393152721278064 -86.95095708962012) bank148630 +148631 POINT(42.11689281422695 -87.60240534179911) bank148631 +148632 POINT(41.25066531874107 -87.26644752976253) bank148632 +148633 POINT(42.122056126573405 -87.57293778679445) bank148633 +148634 POINT(41.45038064896335 -88.60898577490813) bank148634 +148635 POINT(41.62425239885487 -88.38430248319942) bank148635 +148636 POINT(41.03953628955673 -87.19766726053894) bank148636 +148637 POINT(41.82500484529603 -87.61485327376506) bank148637 +148638 POINT(41.50607141533905 -87.69897468469819) bank148638 +148639 POINT(41.37082489289648 -86.88894902499852) bank148639 +148640 POINT(42.17767151508552 -87.80242499940223) bank148640 +148641 POINT(40.93199941100644 -87.94107055242378) bank148641 +148642 POINT(41.40079976061839 -87.65953938621394) bank148642 +148643 POINT(41.914069681240434 -87.8809744922941) bank148643 +148644 POINT(41.88759221008752 -87.12076738068497) bank148644 +148645 POINT(41.290557096972336 -87.01452510578541) bank148645 +148646 POINT(41.04437110584614 -88.1036553479474) bank148646 +148647 POINT(42.577903981608976 -87.30009941045927) bank148647 +148648 POINT(41.9178462176987 -87.46289165928751) bank148648 +148649 POINT(42.13729287250889 -87.32471283803962) bank148649 +148650 POINT(42.69266417876415 -87.70797548946918) bank148650 +148651 POINT(41.468212423310824 -87.60926942862683) bank148651 +148652 POINT(41.23729057923385 -88.62686962140297) bank148652 +148653 POINT(41.189382388544026 -87.95948085335434) bank148653 +148654 POINT(42.83897271621995 -87.81073543593604) bank148654 +148655 POINT(41.77601515300847 -87.7919969093153) bank148655 +148656 POINT(41.064348220488604 -88.2798210639971) bank148656 +148657 POINT(42.18863420913871 -87.36115855428572) bank148657 +148658 POINT(42.25816452084374 -88.57604050698438) bank148658 +148659 POINT(41.717652537529204 -88.51584266669502) bank148659 +148660 POINT(41.54610085891143 -88.25016232519587) bank148660 +148661 POINT(42.71595046332943 -86.7588888467535) bank148661 +148662 POINT(42.412814525245736 -87.79546530120965) bank148662 +148663 POINT(41.70854320075312 -87.81145579321607) bank148663 +148664 POINT(41.67364979318938 -86.80940362231163) bank148664 +148665 POINT(42.379490861536 -87.68958638090356) bank148665 +148666 POINT(42.52200529361341 -87.04405841110724) bank148666 +148667 POINT(41.51495725770985 -88.13151739750124) bank148667 +148668 POINT(42.76078842577546 -87.33019225361184) bank148668 +148669 POINT(41.415699175432266 -86.6334141087198) bank148669 +148670 POINT(41.840816231740106 -87.36026926083846) bank148670 +148671 POINT(42.019845999114516 -86.73874621703276) bank148671 +148672 POINT(41.66882834231167 -86.68262264595786) bank148672 +148673 POINT(42.062056765681874 -87.1440462512833) bank148673 +148674 POINT(41.39852600661926 -87.9757811204708) bank148674 +148675 POINT(41.36230713803198 -88.0629182259422) bank148675 +148676 POINT(42.6405504050941 -87.77722548460103) bank148676 +148677 POINT(41.564639510312865 -88.18393789204903) bank148677 +148678 POINT(41.62393657860102 -86.65189915500814) bank148678 +148679 POINT(41.429339848373516 -87.72197262071177) bank148679 +148680 POINT(42.81460207836583 -87.29245073077958) bank148680 +148681 POINT(41.58306346095943 -88.50017476109534) bank148681 +148682 POINT(42.86385630116473 -87.88612194399953) bank148682 +148683 POINT(41.02342555904889 -87.60821348517723) bank148683 +148684 POINT(41.041959166224856 -87.47099177021433) bank148684 +148685 POINT(41.9983008389319 -87.89328630519826) bank148685 +148686 POINT(42.78097644599296 -88.21301933149107) bank148686 +148687 POINT(41.871895375494795 -88.09482135735391) bank148687 +148688 POINT(40.91093636864207 -86.99050840880908) bank148688 +148689 POINT(42.28664829206715 -88.56996707560035) bank148689 +148690 POINT(40.893317248766465 -87.59128780169324) bank148690 +148691 POINT(42.29925693378196 -88.17300914740808) bank148691 +148692 POINT(41.62491137895049 -87.97186633925142) bank148692 +148693 POINT(41.38240080919628 -88.08676913873857) bank148693 +148694 POINT(41.16617994539536 -88.28971055102784) bank148694 +148695 POINT(40.93882091635374 -87.00516361086088) bank148695 +148696 POINT(40.943574781615744 -87.99805904095567) bank148696 +148697 POINT(41.22122830405125 -86.74077600539718) bank148697 +148698 POINT(41.19386618300329 -88.4701804824347) bank148698 +148699 POINT(41.14585910849496 -87.30197928948681) bank148699 +148700 POINT(42.019316760240294 -88.18793561113334) bank148700 +148701 POINT(42.53248745106915 -87.88765072651596) bank148701 +148702 POINT(41.53771029419409 -87.39906576580614) bank148702 +148703 POINT(41.00148437099497 -88.3409454793092) bank148703 +148704 POINT(41.06824132647212 -87.93306768607985) bank148704 +148705 POINT(42.714395816112216 -86.78291475987369) bank148705 +148706 POINT(42.118124151321055 -87.63934513257459) bank148706 +148707 POINT(41.44442336062005 -88.17042035895425) bank148707 +148708 POINT(41.03442127006569 -88.22722742154633) bank148708 +148709 POINT(41.97356966418253 -88.02106424444942) bank148709 +148710 POINT(41.93683052712608 -87.91082219203538) bank148710 +148711 POINT(41.910024950209774 -88.21395852536142) bank148711 +148712 POINT(41.66587961566502 -86.84887418382367) bank148712 +148713 POINT(42.35435938626353 -88.02666052622021) bank148713 +148714 POINT(42.67900829242396 -87.53430489096745) bank148714 +148715 POINT(42.44117222464608 -88.17576464378507) bank148715 +148716 POINT(40.996559672706965 -88.51798325926316) bank148716 +148717 POINT(41.99905571935632 -86.85538065399028) bank148717 +148718 POINT(41.579551466353585 -88.41305449002132) bank148718 +148719 POINT(40.97635902794752 -87.92841309507277) bank148719 +148720 POINT(42.33374721390097 -86.63311880135552) bank148720 +148721 POINT(40.8998617017999 -86.8006846467408) bank148721 +148722 POINT(42.63031132097585 -87.3217608200582) bank148722 +148723 POINT(42.2527598771494 -86.89671579998843) bank148723 +148724 POINT(40.90301754147308 -86.86041821356255) bank148724 +148725 POINT(41.55627860733827 -86.84279857930197) bank148725 +148726 POINT(41.601786292945064 -87.31029583789619) bank148726 +148727 POINT(42.6973211215511 -88.45587061275448) bank148727 +148728 POINT(42.617728716665994 -87.63279430784732) bank148728 +148729 POINT(41.41126052193424 -86.94356202035657) bank148729 +148730 POINT(40.91054533433062 -87.93141262580897) bank148730 +148731 POINT(41.873412361459906 -88.60846148488575) bank148731 +148732 POINT(42.13621888302357 -87.6610689722058) bank148732 +148733 POINT(42.80462977338378 -88.27953099212577) bank148733 +148734 POINT(42.25856652453537 -86.97688646928759) bank148734 +148735 POINT(42.164759499439256 -87.7978425486613) bank148735 +148736 POINT(41.57840086884564 -87.72933875331174) bank148736 +148737 POINT(42.74006059824679 -86.78934935793248) bank148737 +148738 POINT(42.1693677954544 -87.44693328430239) bank148738 +148739 POINT(41.837555965399524 -88.34153362984605) bank148739 +148740 POINT(42.666184812874455 -88.43455629499095) bank148740 +148741 POINT(41.11540438817996 -88.60084544151641) bank148741 +148742 POINT(42.83992616286229 -88.06517277412222) bank148742 +148743 POINT(41.149380595783875 -87.60102363764896) bank148743 +148744 POINT(42.699856799207204 -87.29270145110816) bank148744 +148745 POINT(42.37290359369857 -86.95324262854629) bank148745 +148746 POINT(41.36673567822113 -88.60061947657645) bank148746 +148747 POINT(41.377815077874 -86.94490710362446) bank148747 +148748 POINT(41.72820370515319 -87.48547935243197) bank148748 +148749 POINT(41.9942795200499 -86.87029707205576) bank148749 +148750 POINT(40.99657541679588 -88.2209115520955) bank148750 +148751 POINT(41.97368965972219 -88.12678262479139) bank148751 +148752 POINT(41.72882675215101 -87.46230592978729) bank148752 +148753 POINT(42.738946493855245 -88.46028847185288) bank148753 +148754 POINT(42.77930058654654 -87.27190185519889) bank148754 +148755 POINT(41.42968078299508 -88.53145712039426) bank148755 +148756 POINT(42.87596250460014 -87.43625848143894) bank148756 +148757 POINT(41.21982778712025 -88.30690048771152) bank148757 +148758 POINT(41.64810080935389 -87.13382404679176) bank148758 +148759 POINT(41.103813942896515 -87.26803288077492) bank148759 +148760 POINT(40.90362175332598 -86.77619435793892) bank148760 +148761 POINT(41.27690957353247 -87.45561486508996) bank148761 +148762 POINT(41.48147100384658 -87.12573841411506) bank148762 +148763 POINT(42.26291370921415 -88.03649634411295) bank148763 +148764 POINT(41.157714140715775 -88.17031585649778) bank148764 +148765 POINT(42.23794108417142 -87.27909523362537) bank148765 +148766 POINT(41.12982329016516 -87.30714819068876) bank148766 +148767 POINT(41.57143461882367 -87.82992719356407) bank148767 +148768 POINT(41.165102662564394 -87.58745588208264) bank148768 +148769 POINT(41.94121486933915 -88.01148634213489) bank148769 +148770 POINT(41.24827173070751 -87.48020460918562) bank148770 +148771 POINT(41.90968049077861 -87.29369330725508) bank148771 +148772 POINT(40.94770309236126 -87.36623133449166) bank148772 +148773 POINT(42.18099174362202 -86.9716384311338) bank148773 +148774 POINT(42.005508417607274 -87.24650228183205) bank148774 +148775 POINT(42.80983930164699 -87.10251393478597) bank148775 +148776 POINT(41.34298949994102 -86.87774033065521) bank148776 +148777 POINT(42.84101344765306 -88.52597911934869) bank148777 +148778 POINT(42.30291594810049 -88.36493166860286) bank148778 +148779 POINT(42.290274409582 -86.73170517846883) bank148779 +148780 POINT(42.23872458489673 -88.1480863100869) bank148780 +148781 POINT(41.008773199990706 -87.60003878013521) bank148781 +148782 POINT(41.83536874787488 -87.20654266849229) bank148782 +148783 POINT(41.46981759423509 -88.31370982923457) bank148783 +148784 POINT(41.34655731843749 -86.95106705042643) bank148784 +148785 POINT(41.76769248656323 -87.34677440944117) bank148785 +148786 POINT(42.28963310843346 -86.90763547803378) bank148786 +148787 POINT(41.77214806660459 -87.05687741523417) bank148787 +148788 POINT(41.92295595095843 -86.89796819660297) bank148788 +148789 POINT(42.14680332171336 -87.11674455126348) bank148789 +148790 POINT(42.80593449279918 -88.59345022617902) bank148790 +148791 POINT(41.488059612092535 -86.73745379422867) bank148791 +148792 POINT(42.21719880544228 -88.31875355356951) bank148792 +148793 POINT(42.55155343482583 -87.97182138708546) bank148793 +148794 POINT(41.35307640428562 -87.03621987337482) bank148794 +148795 POINT(42.52963168420978 -86.88240589741497) bank148795 +148796 POINT(42.3379029590235 -87.64575605049602) bank148796 +148797 POINT(41.678573733946365 -86.78113726630674) bank148797 +148798 POINT(42.512352954295 -87.00446541773269) bank148798 +148799 POINT(42.55076109603854 -87.4688028562294) bank148799 +148800 POINT(41.71628932796258 -86.68500600779035) bank148800 +148801 POINT(41.049554425175735 -87.43694308844957) bank148801 +148802 POINT(41.07772800922252 -86.8830553233413) bank148802 +148803 POINT(41.27788431766021 -87.73138661261578) bank148803 +148804 POINT(41.061344350141745 -86.66031539686385) bank148804 +148805 POINT(41.16365177750148 -88.30374459809647) bank148805 +148806 POINT(41.31260972059605 -87.04888224115992) bank148806 +148807 POINT(42.67137128849195 -87.27394931230867) bank148807 +148808 POINT(41.94459151582072 -86.8040497037714) bank148808 +148809 POINT(41.406177603370466 -86.69620596935248) bank148809 +148810 POINT(40.97787393237215 -88.51106897560206) bank148810 +148811 POINT(41.96739523846238 -87.96881970729234) bank148811 +148812 POINT(41.914206047403994 -87.26480321888867) bank148812 +148813 POINT(42.46102757597858 -88.0934951709384) bank148813 +148814 POINT(41.96377743170598 -87.25175948340815) bank148814 +148815 POINT(40.96154298339599 -87.11108888474008) bank148815 +148816 POINT(41.068734874629754 -87.6924210451375) bank148816 +148817 POINT(40.906228661961606 -88.42244539067313) bank148817 +148818 POINT(42.224606251655736 -87.19608055030741) bank148818 +148819 POINT(41.82026194135039 -86.72497131134405) bank148819 +148820 POINT(41.00205866181477 -88.55425499241926) bank148820 +148821 POINT(40.97760861486769 -87.03150604274174) bank148821 +148822 POINT(42.17956583799388 -87.87608530833333) bank148822 +148823 POINT(41.22955516654263 -87.9147645091104) bank148823 +148824 POINT(42.72794781048565 -86.88751620677556) bank148824 +148825 POINT(41.56693616249243 -87.21222761942131) bank148825 +148826 POINT(42.76627180736749 -87.83253410549021) bank148826 +148827 POINT(41.053558909267295 -88.00396028012078) bank148827 +148828 POINT(42.554182609494724 -88.44844675952534) bank148828 +148829 POINT(42.618214395471384 -86.74153475899952) bank148829 +148830 POINT(41.3505599977612 -88.18559277262924) bank148830 +148831 POINT(41.346545069174766 -88.220835689194) bank148831 +148832 POINT(41.70808665820299 -88.14698292536582) bank148832 +148833 POINT(42.75960639438124 -87.81522105583471) bank148833 +148834 POINT(42.49897483183142 -87.58510525629576) bank148834 +148835 POINT(42.265823349777705 -87.85064790643433) bank148835 +148836 POINT(41.74902276304203 -88.12687265017495) bank148836 +148837 POINT(42.22400044420323 -87.30418093409723) bank148837 +148838 POINT(41.130713578785155 -86.77887558292058) bank148838 +148839 POINT(41.401043858949734 -87.42987388307185) bank148839 +148840 POINT(42.249453088985724 -88.12153411868627) bank148840 +148841 POINT(42.30573904038659 -87.61627050080087) bank148841 +148842 POINT(41.014324144048196 -86.97231711147393) bank148842 +148843 POINT(42.15298933849133 -86.69693197895283) bank148843 +148844 POINT(41.659421757197315 -87.67538738108071) bank148844 +148845 POINT(41.22710374383175 -87.54253134945053) bank148845 +148846 POINT(42.18171256472524 -88.04128490536175) bank148846 +148847 POINT(42.13671121803624 -87.80347111746214) bank148847 +148848 POINT(42.7839675912891 -86.94738819782852) bank148848 +148849 POINT(40.91017072186472 -86.97962624534557) bank148849 +148850 POINT(42.238507285457594 -87.21861182770074) bank148850 +148851 POINT(42.29435612669001 -87.39702742132485) bank148851 +148852 POINT(42.24448226221254 -86.76031543078994) bank148852 +148853 POINT(41.873781443441615 -88.58221603923622) bank148853 +148854 POINT(42.69711325251279 -87.20707233852845) bank148854 +148855 POINT(42.018275394770555 -87.24327921008535) bank148855 +148856 POINT(41.71007343831254 -86.9853738350544) bank148856 +148857 POINT(41.64111806158777 -86.7268994227522) bank148857 +148858 POINT(41.33809566630118 -87.92374845184018) bank148858 +148859 POINT(41.08641659648393 -88.59243417664617) bank148859 +148860 POINT(41.46526201948423 -88.53443027326949) bank148860 +148861 POINT(41.394168800677924 -87.04463845608085) bank148861 +148862 POINT(42.587121259580954 -87.58054219650444) bank148862 +148863 POINT(42.1229987329711 -88.5658842868196) bank148863 +148864 POINT(42.714465736620035 -87.90781950680665) bank148864 +148865 POINT(41.99027651172061 -88.21487216384288) bank148865 +148866 POINT(42.646014473922 -88.55428074206337) bank148866 +148867 POINT(42.07598703997558 -88.50893710404306) bank148867 +148868 POINT(42.28486431491473 -87.382495109046) bank148868 +148869 POINT(42.53111085261733 -88.36884814974971) bank148869 +148870 POINT(42.37629554782471 -86.7158288245738) bank148870 +148871 POINT(42.8102122544226 -86.77713759344327) bank148871 +148872 POINT(41.294989708321 -87.91727843733153) bank148872 +148873 POINT(42.53104527131823 -86.81382988225313) bank148873 +148874 POINT(42.31777215691415 -88.48069081801403) bank148874 +148875 POINT(41.82567610533584 -87.94482673548765) bank148875 +148876 POINT(41.92026793631207 -87.43499684060471) bank148876 +148877 POINT(42.297118265556826 -87.87086894838677) bank148877 +148878 POINT(42.49288685464472 -86.75208428035329) bank148878 +148879 POINT(41.82619114249333 -87.30046625880924) bank148879 +148880 POINT(42.87724218256985 -88.18711318622475) bank148880 +148881 POINT(42.56864535539893 -88.29667504336537) bank148881 +148882 POINT(42.68577898488109 -87.43386825151349) bank148882 +148883 POINT(41.00867211352959 -87.02232935194223) bank148883 +148884 POINT(42.36670338757951 -87.40942369846906) bank148884 +148885 POINT(41.08087104113528 -87.9066144744783) bank148885 +148886 POINT(41.012858264921036 -86.99718163858772) bank148886 +148887 POINT(42.744038087270155 -86.89576596600074) bank148887 +148888 POINT(42.6807180343853 -88.51292994629739) bank148888 +148889 POINT(41.8369062606386 -86.74965951293908) bank148889 +148890 POINT(40.88680073484615 -88.48447309305975) bank148890 +148891 POINT(42.85250627322874 -87.13059895726866) bank148891 +148892 POINT(42.638020914360325 -88.2569401909817) bank148892 +148893 POINT(41.33396869673545 -86.89321209259676) bank148893 +148894 POINT(41.96797816676086 -87.88728142600596) bank148894 +148895 POINT(42.85648385579736 -86.70851188950246) bank148895 +148896 POINT(42.75638327441365 -88.5148764937548) bank148896 +148897 POINT(41.384039227040184 -86.77949787077426) bank148897 +148898 POINT(41.911699589915756 -88.52943852519635) bank148898 +148899 POINT(42.63229902702728 -87.44223328806265) bank148899 +148900 POINT(42.048076707370925 -86.86835037284631) bank148900 +148901 POINT(41.817764484694536 -87.91866348201692) bank148901 +148902 POINT(42.45612320803627 -88.31524310906957) bank148902 +148903 POINT(41.396649629604724 -86.64045603664817) bank148903 +148904 POINT(41.770337152458666 -86.74196105834994) bank148904 +148905 POINT(42.872781268176645 -87.11175523337772) bank148905 +148906 POINT(42.507056496739935 -87.31012266815465) bank148906 +148907 POINT(42.720713583737094 -88.08603944050584) bank148907 +148908 POINT(42.8733334757823 -88.18701512936694) bank148908 +148909 POINT(41.75288779578126 -87.83272574222507) bank148909 +148910 POINT(41.49605227599559 -86.6366019044656) bank148910 +148911 POINT(42.70087001747476 -87.44100464325736) bank148911 +148912 POINT(41.08054697071685 -87.98516142597704) bank148912 +148913 POINT(42.45485605060068 -87.06996313172164) bank148913 +148914 POINT(41.401871692512564 -87.23386412273479) bank148914 +148915 POINT(42.71118557520952 -87.3757084316159) bank148915 +148916 POINT(42.13940853623111 -86.69257258140593) bank148916 +148917 POINT(40.89381974879556 -87.70614095871487) bank148917 +148918 POINT(42.47573265790419 -87.17651201144479) bank148918 +148919 POINT(40.92359833037386 -86.94928078333328) bank148919 +148920 POINT(42.08426525081039 -86.6754876692743) bank148920 +148921 POINT(42.136009343172404 -86.93041565736172) bank148921 +148922 POINT(42.847034268687295 -87.47140714683381) bank148922 +148923 POINT(41.79489376769238 -88.02014703081022) bank148923 +148924 POINT(40.98739176600804 -88.45932306365961) bank148924 +148925 POINT(41.70449675947017 -87.84982431089463) bank148925 +148926 POINT(41.337764947886846 -87.63584509987395) bank148926 +148927 POINT(41.31031828537875 -87.95241208687253) bank148927 +148928 POINT(42.575163572347 -87.1845378283076) bank148928 +148929 POINT(41.54618519904213 -87.67450453120243) bank148929 +148930 POINT(41.26712653525919 -86.85506153300798) bank148930 +148931 POINT(40.89995147907659 -87.80737053073247) bank148931 +148932 POINT(41.11555821984861 -88.12565857403256) bank148932 +148933 POINT(41.35068807399836 -88.20959193101507) bank148933 +148934 POINT(42.460948199220944 -87.0641535813827) bank148934 +148935 POINT(41.07754277379405 -88.18577604397642) bank148935 +148936 POINT(41.29466537495355 -88.44164154891183) bank148936 +148937 POINT(41.82881857922954 -86.85320143955262) bank148937 +148938 POINT(42.80389815690412 -87.02020704606251) bank148938 +148939 POINT(41.003818876787854 -88.57286141129333) bank148939 +148940 POINT(42.7364562322941 -87.4834760619862) bank148940 +148941 POINT(42.70447716803768 -86.9898154307178) bank148941 +148942 POINT(41.72400996148594 -86.72834958588588) bank148942 +148943 POINT(42.274487141227176 -87.89799550254018) bank148943 +148944 POINT(41.73791292797667 -87.7796585277042) bank148944 +148945 POINT(40.95372244111662 -87.00523039775858) bank148945 +148946 POINT(41.392709140526236 -86.65890306861029) bank148946 +148947 POINT(42.346238687051816 -86.65725174705872) bank148947 +148948 POINT(41.59369064871998 -86.99127616584582) bank148948 +148949 POINT(41.96340432023309 -86.94477934010494) bank148949 +148950 POINT(42.2688264360854 -87.76496368001341) bank148950 +148951 POINT(41.57743373072378 -87.34241315249213) bank148951 +148952 POINT(42.32619163895546 -88.48200827586703) bank148952 +148953 POINT(41.15946775546297 -87.74626394368444) bank148953 +148954 POINT(42.46822401505878 -87.1134106741383) bank148954 +148955 POINT(40.927465636952526 -88.20006555767398) bank148955 +148956 POINT(40.88927367748277 -88.28947827055941) bank148956 +148957 POINT(42.15157511446709 -87.9498960237878) bank148957 +148958 POINT(42.725127467431285 -86.69203805587931) bank148958 +148959 POINT(41.29514639297486 -87.59670306460976) bank148959 +148960 POINT(41.34774106857089 -88.43490374007592) bank148960 +148961 POINT(42.218479543332016 -88.52828666397994) bank148961 +148962 POINT(41.09191200896481 -87.90322006392006) bank148962 +148963 POINT(42.44975639697324 -87.69801119974558) bank148963 +148964 POINT(41.09284319272589 -87.61141503634384) bank148964 +148965 POINT(42.52224205807238 -87.8414606972052) bank148965 +148966 POINT(42.81109142501869 -88.40210562522934) bank148966 +148967 POINT(42.00623148477607 -88.48970982017761) bank148967 +148968 POINT(42.75822673532221 -88.36957554684523) bank148968 +148969 POINT(40.968448042203 -87.93835786671698) bank148969 +148970 POINT(42.30290183042538 -87.27319787827936) bank148970 +148971 POINT(41.74931722216502 -88.4232667631795) bank148971 +148972 POINT(42.2278617526683 -87.78932630725919) bank148972 +148973 POINT(41.554520569014564 -88.15901940179934) bank148973 +148974 POINT(41.248484299578315 -87.45545971462097) bank148974 +148975 POINT(41.91552724283087 -88.47510161154644) bank148975 +148976 POINT(40.99143794615901 -88.21562288301944) bank148976 +148977 POINT(41.935967407599456 -87.21975779236381) bank148977 +148978 POINT(42.36461968561399 -87.64859881637732) bank148978 +148979 POINT(42.58778779905028 -86.91778180719261) bank148979 +148980 POINT(41.88818625922273 -87.47351036102437) bank148980 +148981 POINT(41.33736251820505 -86.99967036399951) bank148981 +148982 POINT(42.676944674110445 -87.94538965097051) bank148982 +148983 POINT(41.079325896689994 -87.08443638664023) bank148983 +148984 POINT(41.45968710975475 -87.58543227276363) bank148984 +148985 POINT(42.18377198858829 -87.57622601441773) bank148985 +148986 POINT(42.14013663746223 -87.50065860418633) bank148986 +148987 POINT(41.5324465631983 -88.13878308615087) bank148987 +148988 POINT(41.055880523114304 -88.26031482648533) bank148988 +148989 POINT(42.4151566280449 -86.82476283120606) bank148989 +148990 POINT(41.003492287550664 -87.18568556177398) bank148990 +148991 POINT(41.593396746820495 -88.57436675885006) bank148991 +148992 POINT(41.13698418998872 -88.38514700797793) bank148992 +148993 POINT(41.852711040048014 -86.73383574256744) bank148993 +148994 POINT(42.650270797025215 -87.30740177154878) bank148994 +148995 POINT(41.300542239691296 -86.70663238341187) bank148995 +148996 POINT(42.75112132092961 -88.21785280718451) bank148996 +148997 POINT(42.57173594493656 -87.80106499543355) bank148997 +148998 POINT(42.30182466905651 -87.21311368548463) bank148998 +148999 POINT(41.98451120954367 -88.39886101730107) bank148999 +149000 POINT(41.23482916978675 -88.29541159346712) bank149000 +149001 POINT(41.97844867613256 -88.49921845712156) bank149001 +149002 POINT(42.50631343626916 -88.17443311500884) bank149002 +149003 POINT(41.7721459265529 -87.0076607605083) bank149003 +149004 POINT(42.056450850792196 -87.24082489680168) bank149004 +149005 POINT(40.893388350338036 -87.46714325879832) bank149005 +149006 POINT(41.80839077716812 -87.23331046805923) bank149006 +149007 POINT(41.452727279356154 -88.41940267847929) bank149007 +149008 POINT(42.79780333130023 -86.89017421640395) bank149008 +149009 POINT(42.85599612695342 -87.58713549513351) bank149009 +149010 POINT(42.49521038817729 -88.15301996621541) bank149010 +149011 POINT(41.27606476280291 -87.01307033696465) bank149011 +149012 POINT(41.32146401829466 -86.77839047425671) bank149012 +149013 POINT(42.734739024882785 -86.6535555564172) bank149013 +149014 POINT(41.82152712787549 -87.46119023526686) bank149014 +149015 POINT(41.27400766225061 -87.59711549883518) bank149015 +149016 POINT(41.249468474771284 -86.86004651533716) bank149016 +149017 POINT(41.77100704681273 -87.4776243586848) bank149017 +149018 POINT(42.055800722060724 -87.00968419531357) bank149018 +149019 POINT(41.244730975034 -88.08249734934705) bank149019 +149020 POINT(41.88344460043291 -87.1887592684032) bank149020 +149021 POINT(41.9082063281177 -87.35832046370261) bank149021 +149022 POINT(41.19944697619912 -87.13380832836921) bank149022 +149023 POINT(42.73458116831222 -87.70501159534452) bank149023 +149024 POINT(41.351606962621176 -87.93516963583181) bank149024 +149025 POINT(42.17073526758557 -88.01367909524504) bank149025 +149026 POINT(41.80699552740039 -88.19887894840443) bank149026 +149027 POINT(42.619527210440076 -87.25245059052084) bank149027 +149028 POINT(41.2478168595503 -86.91945507673648) bank149028 +149029 POINT(42.494661954376674 -88.42164089915866) bank149029 +149030 POINT(41.881519765948674 -86.70654157192448) bank149030 +149031 POINT(41.335297755059926 -86.97395999666952) bank149031 +149032 POINT(41.70499566901013 -86.75731578643015) bank149032 +149033 POINT(42.43411547493367 -87.47840733987223) bank149033 +149034 POINT(40.96363140472155 -87.25019703149276) bank149034 +149035 POINT(42.025157849734086 -87.68232146876814) bank149035 +149036 POINT(42.267473047225884 -88.26408927324906) bank149036 +149037 POINT(42.26138456626905 -87.1129892051694) bank149037 +149038 POINT(42.648027559465916 -88.34534377832425) bank149038 +149039 POINT(42.678577271644684 -87.2929125254194) bank149039 +149040 POINT(41.32967761463444 -86.78579439088405) bank149040 +149041 POINT(41.33546095462751 -86.85043609540045) bank149041 +149042 POINT(41.51590521871303 -87.00648086765321) bank149042 +149043 POINT(41.83636299657148 -86.69215803153486) bank149043 +149044 POINT(41.906913228136325 -87.96209110306619) bank149044 +149045 POINT(42.700864606142076 -87.03244316165751) bank149045 +149046 POINT(42.42707147495118 -86.70514999547082) bank149046 +149047 POINT(41.25782555566291 -87.56808286867926) bank149047 +149048 POINT(42.262356882708076 -87.09905932008965) bank149048 +149049 POINT(41.167145214980955 -86.75807386622523) bank149049 +149050 POINT(42.3279435269883 -86.70938883573724) bank149050 +149051 POINT(41.714230744715174 -86.8273044806005) bank149051 +149052 POINT(42.21020210767861 -86.66166436160009) bank149052 +149053 POINT(41.442396892710455 -88.04149145284464) bank149053 +149054 POINT(41.64562175706738 -88.28320286441622) bank149054 +149055 POINT(42.87390097862971 -87.00467338342722) bank149055 +149056 POINT(40.9790929551321 -87.51761824291033) bank149056 +149057 POINT(42.54215453969628 -87.73033362033513) bank149057 +149058 POINT(42.04736821327261 -88.56374291292339) bank149058 +149059 POINT(42.83100644436281 -87.39843752723122) bank149059 +149060 POINT(41.03594835999647 -87.74140600737188) bank149060 +149061 POINT(42.185372658120706 -87.3621342241874) bank149061 +149062 POINT(41.53752640351545 -86.67041836674028) bank149062 +149063 POINT(42.15593001398858 -87.37118089879345) bank149063 +149064 POINT(41.015078024489966 -88.11681150218578) bank149064 +149065 POINT(42.29901055867725 -86.98530729471646) bank149065 +149066 POINT(42.75372853304614 -87.98322255553713) bank149066 +149067 POINT(40.94194143782109 -86.66810485463861) bank149067 +149068 POINT(42.76338165232831 -87.35213270784259) bank149068 +149069 POINT(41.700529661345975 -87.77690531970453) bank149069 +149070 POINT(41.78956681646766 -87.0185108282748) bank149070 +149071 POINT(42.87449512067001 -87.29643604875461) bank149071 +149072 POINT(42.11987982000135 -88.34300837879476) bank149072 +149073 POINT(42.028221726100284 -87.25346886590027) bank149073 +149074 POINT(42.48035343549239 -88.00008386776466) bank149074 +149075 POINT(41.548110274218416 -87.53111377416414) bank149075 +149076 POINT(42.12803672195896 -87.85811056164371) bank149076 +149077 POINT(42.22606227265684 -87.99415280475675) bank149077 +149078 POINT(42.225408223597384 -87.80732553121375) bank149078 +149079 POINT(41.32361986360829 -88.59312969526714) bank149079 +149080 POINT(42.747623956768216 -87.05890654899615) bank149080 +149081 POINT(41.348734715847044 -86.67764389492163) bank149081 +149082 POINT(41.97340784461078 -88.46532715364603) bank149082 +149083 POINT(40.88517624237705 -88.31307351374956) bank149083 +149084 POINT(41.90377343203456 -88.3511025408986) bank149084 +149085 POINT(42.106766767169105 -87.38264296911882) bank149085 +149086 POINT(40.94665030958132 -87.6063586900652) bank149086 +149087 POINT(41.46233225552728 -86.6418269982051) bank149087 +149088 POINT(41.094755803987425 -88.08985368879874) bank149088 +149089 POINT(42.671291739270345 -86.7313050588331) bank149089 +149090 POINT(41.03399277174846 -87.1212715068516) bank149090 +149091 POINT(42.087045111321096 -87.35513004012911) bank149091 +149092 POINT(42.44690399804131 -87.15448327595462) bank149092 +149093 POINT(41.14055195462853 -87.94538209591114) bank149093 +149094 POINT(42.075481803448504 -88.33494565879603) bank149094 +149095 POINT(41.194872819926466 -87.0232536187268) bank149095 +149096 POINT(40.922731410862966 -88.56555720213318) bank149096 +149097 POINT(42.52067009892314 -88.59584509718137) bank149097 +149098 POINT(42.25829973302506 -88.45993494014695) bank149098 +149099 POINT(41.65038722386094 -87.38111187378503) bank149099 +149100 POINT(42.48022920139198 -87.10186968889609) bank149100 +149101 POINT(40.91232557899525 -88.23333680830919) bank149101 +149102 POINT(41.00472776364571 -86.77611357774286) bank149102 +149103 POINT(42.68914182929095 -88.61242975437942) bank149103 +149104 POINT(42.62049944485999 -87.2761312170024) bank149104 +149105 POINT(42.64817937485336 -87.0464826818841) bank149105 +149106 POINT(41.20524429257164 -88.04671291144176) bank149106 +149107 POINT(42.42502419144401 -87.56881793656244) bank149107 +149108 POINT(42.265102347120305 -86.70374250178943) bank149108 +149109 POINT(42.74186615165279 -86.76854303287324) bank149109 +149110 POINT(42.099885083769266 -87.60157195012302) bank149110 +149111 POINT(42.87675425318276 -86.92297014120352) bank149111 +149112 POINT(42.20411148119434 -87.38893425826674) bank149112 +149113 POINT(41.7546505978878 -87.28879865976214) bank149113 +149114 POINT(42.053538957948135 -88.00393452101102) bank149114 +149115 POINT(41.59764825850962 -87.9024209660141) bank149115 +149116 POINT(40.97265098403592 -87.30193087550175) bank149116 +149117 POINT(42.17337352848847 -87.3117691129488) bank149117 +149118 POINT(42.63792188338251 -87.45944358700511) bank149118 +149119 POINT(42.2253553358617 -88.12517083678739) bank149119 +149120 POINT(41.684466160576704 -87.19113080878506) bank149120 +149121 POINT(42.23996858280218 -87.41266470595441) bank149121 +149122 POINT(42.69952394135248 -87.0231446298077) bank149122 +149123 POINT(41.337242254141756 -87.65273363390887) bank149123 +149124 POINT(42.790662115701465 -88.28275282549944) bank149124 +149125 POINT(42.108455719372806 -87.7839092692687) bank149125 +149126 POINT(40.92386849949852 -88.0531946053205) bank149126 +149127 POINT(42.197123590492545 -88.48833315058855) bank149127 +149128 POINT(41.677117984125395 -87.43300395134602) bank149128 +149129 POINT(42.8477023010342 -87.49195996605646) bank149129 +149130 POINT(41.6539832606256 -87.14109120813082) bank149130 +149131 POINT(42.55208639578416 -87.51928813113462) bank149131 +149132 POINT(41.02491438361051 -87.7839917385988) bank149132 +149133 POINT(42.52091560867162 -88.00860384412931) bank149133 +149134 POINT(42.25789544348644 -87.39283795545458) bank149134 +149135 POINT(41.9819753057273 -87.74585756883) bank149135 +149136 POINT(42.0562231890965 -88.18230607696657) bank149136 +149137 POINT(41.5472652607056 -88.21718472719212) bank149137 +149138 POINT(41.31749333423409 -87.52519877459478) bank149138 +149139 POINT(42.52961512502608 -86.64546299510926) bank149139 +149140 POINT(41.69602738941311 -87.09492334775231) bank149140 +149141 POINT(42.57741181398573 -88.52683839289745) bank149141 +149142 POINT(41.167332950680354 -88.45277738285715) bank149142 +149143 POINT(40.98248051424798 -88.15159763461988) bank149143 +149144 POINT(41.26739306541103 -88.08372862421761) bank149144 +149145 POINT(41.746842272313415 -88.09727243282022) bank149145 +149146 POINT(41.75577085824202 -88.13032485171368) bank149146 +149147 POINT(41.33123248175451 -87.42063059337866) bank149147 +149148 POINT(40.917934088477075 -88.04197334905733) bank149148 +149149 POINT(41.61595707496776 -87.15039585270176) bank149149 +149150 POINT(42.73645649535809 -88.46742482167593) bank149150 +149151 POINT(41.79940216352798 -87.74317881035647) bank149151 +149152 POINT(41.29650292596275 -88.54321224602275) bank149152 +149153 POINT(42.16921956565924 -88.31403469053652) bank149153 +149154 POINT(41.95057023584643 -87.90925917169719) bank149154 +149155 POINT(41.76475792730559 -88.58500414142085) bank149155 +149156 POINT(42.28350057153272 -88.44284916685038) bank149156 +149157 POINT(41.99264254007391 -88.21161257058236) bank149157 +149158 POINT(42.333454727682096 -87.4401004910766) bank149158 +149159 POINT(41.37065386506665 -87.81295027831477) bank149159 +149160 POINT(42.380698410420266 -86.63144656865174) bank149160 +149161 POINT(42.583733663148806 -88.39887837171273) bank149161 +149162 POINT(40.91203033165721 -87.49182519006284) bank149162 +149163 POINT(42.06621639733132 -87.06214963504976) bank149163 +149164 POINT(41.000915419731996 -88.2057472987312) bank149164 +149165 POINT(42.48274308751538 -87.3916851978303) bank149165 +149166 POINT(41.25999255769311 -87.22730015240181) bank149166 +149167 POINT(41.769099082852726 -86.97152444237013) bank149167 +149168 POINT(42.46672684178316 -87.55428400990657) bank149168 +149169 POINT(41.15148452063294 -87.15833710426645) bank149169 +149170 POINT(41.224519559858834 -86.71031326675795) bank149170 +149171 POINT(42.79649904949417 -87.76365817964424) bank149171 +149172 POINT(41.44622308931908 -87.3266563277542) bank149172 +149173 POINT(41.601477227746386 -88.07827245909367) bank149173 +149174 POINT(41.17467439844588 -88.31431231662495) bank149174 +149175 POINT(41.830819078553624 -87.8544675702745) bank149175 +149176 POINT(42.28986306457878 -87.04329638059676) bank149176 +149177 POINT(42.775688548235685 -87.90492718463324) bank149177 +149178 POINT(41.78351649096521 -87.51459952276598) bank149178 +149179 POINT(41.03141435404061 -87.13280044094101) bank149179 +149180 POINT(41.290665708683086 -87.45730872175689) bank149180 +149181 POINT(41.60968301119363 -87.36860485122531) bank149181 +149182 POINT(42.27447336613963 -87.62542738464313) bank149182 +149183 POINT(41.76415164141529 -87.94640321415925) bank149183 +149184 POINT(41.688766906355966 -87.35576419640296) bank149184 +149185 POINT(41.38429498669252 -88.44561006950585) bank149185 +149186 POINT(42.757919681917095 -87.8375329887208) bank149186 +149187 POINT(42.52798736107226 -87.34538086615406) bank149187 +149188 POINT(41.865873086071254 -87.635332968429) bank149188 +149189 POINT(42.00931520683138 -88.1435599899947) bank149189 +149190 POINT(42.76672402407681 -88.29523773861199) bank149190 +149191 POINT(41.38708836743636 -87.14986070488169) bank149191 +149192 POINT(41.57602003577121 -88.03079355341742) bank149192 +149193 POINT(41.766142676413175 -88.15655710524099) bank149193 +149194 POINT(42.61410294282941 -87.63285303790275) bank149194 +149195 POINT(42.32356371478332 -88.58507431072645) bank149195 +149196 POINT(41.8805352187462 -88.25502844273143) bank149196 +149197 POINT(41.893316571086785 -87.2235381088439) bank149197 +149198 POINT(41.03344208580801 -87.50166747838782) bank149198 +149199 POINT(42.75683758464098 -88.12973042344825) bank149199 +149200 POINT(41.939703977584216 -87.55868781455412) bank149200 +149201 POINT(40.96041552476716 -88.31448663073708) bank149201 +149202 POINT(42.130588091618236 -86.94026284505036) bank149202 +149203 POINT(42.71992908642207 -88.04593610427261) bank149203 +149204 POINT(41.07521190295769 -87.6734643643955) bank149204 +149205 POINT(42.671191507765904 -86.91749511357489) bank149205 +149206 POINT(41.30220261941672 -87.21363119176257) bank149206 +149207 POINT(42.639971816734615 -87.57071997878857) bank149207 +149208 POINT(40.93863110330145 -87.27996159209287) bank149208 +149209 POINT(41.152593330437476 -87.7966543839246) bank149209 +149210 POINT(42.66165538943436 -87.03046189383853) bank149210 +149211 POINT(40.91794641365137 -87.96667185661822) bank149211 +149212 POINT(41.432869930658875 -87.26765391229866) bank149212 +149213 POINT(42.0937880367436 -88.53619266449006) bank149213 +149214 POINT(42.61436557720782 -88.17086162253432) bank149214 +149215 POINT(42.569164548067874 -86.76765221356048) bank149215 +149216 POINT(41.85319816846806 -88.00489955137769) bank149216 +149217 POINT(41.52322568876017 -87.97672592992585) bank149217 +149218 POINT(42.324245681390735 -87.49780857244482) bank149218 +149219 POINT(41.03043307602261 -87.68617433529822) bank149219 +149220 POINT(41.07332918098811 -87.61164840819488) bank149220 +149221 POINT(41.148227925445404 -87.19870556714376) bank149221 +149222 POINT(40.885797494358314 -87.4131228527788) bank149222 +149223 POINT(40.943266276351636 -87.69457376241401) bank149223 +149224 POINT(42.62981774122327 -88.55487034542251) bank149224 +149225 POINT(40.95519462575793 -87.79400840647244) bank149225 +149226 POINT(41.316897839544694 -88.14582957807474) bank149226 +149227 POINT(41.91845653137698 -87.81454491117154) bank149227 +149228 POINT(41.09443482472317 -87.17901464874862) bank149228 +149229 POINT(41.18923518381372 -88.62345266678088) bank149229 +149230 POINT(41.38005119783361 -86.9308648322037) bank149230 +149231 POINT(41.25110896148521 -87.10010817975753) bank149231 +149232 POINT(42.19285276687948 -88.41675983165116) bank149232 +149233 POINT(41.192680969953024 -87.54224870710804) bank149233 +149234 POINT(40.904075229536794 -87.5113478066623) bank149234 +149235 POINT(41.444810577241455 -88.32527696089883) bank149235 +149236 POINT(42.68922649734445 -88.60508866087743) bank149236 +149237 POINT(42.061531690848525 -87.44189968856014) bank149237 +149238 POINT(42.570423812966894 -88.46797221542587) bank149238 +149239 POINT(41.87358483975299 -88.38834842458294) bank149239 +149240 POINT(42.33995311279073 -86.87461552505206) bank149240 +149241 POINT(42.60842337587785 -87.97675292688852) bank149241 +149242 POINT(41.059697422345664 -87.43776178429924) bank149242 +149243 POINT(41.17669447600355 -88.55752997601859) bank149243 +149244 POINT(41.575488388737796 -88.0279571247383) bank149244 +149245 POINT(42.3201472489385 -87.33878173618297) bank149245 +149246 POINT(42.45048354502812 -86.76865411636122) bank149246 +149247 POINT(41.600462929695496 -87.63567324069913) bank149247 +149248 POINT(42.164320884005434 -87.01268110614359) bank149248 +149249 POINT(41.757553648954584 -87.74312747474976) bank149249 +149250 POINT(42.02507505862338 -87.16041740971131) bank149250 +149251 POINT(41.55275071247073 -88.37531690574501) bank149251 +149252 POINT(42.15132056232439 -87.33890685249642) bank149252 +149253 POINT(41.730384619784644 -87.80152368774155) bank149253 +149254 POINT(42.80228565319366 -88.11800197451258) bank149254 +149255 POINT(42.38276393500115 -86.92133782790077) bank149255 +149256 POINT(42.595273933181765 -87.09355114595796) bank149256 +149257 POINT(41.41845432066065 -87.99628845687609) bank149257 +149258 POINT(42.13075585938988 -87.15010463166095) bank149258 +149259 POINT(42.37961700562658 -88.28575135734668) bank149259 +149260 POINT(40.932377627858244 -88.13270197030909) bank149260 +149261 POINT(41.505573834630056 -87.47905864047357) bank149261 +149262 POINT(42.31564350691622 -88.37775781854793) bank149262 +149263 POINT(41.09513864361243 -88.11128116034567) bank149263 +149264 POINT(40.92990559436135 -87.41646271785378) bank149264 +149265 POINT(41.90096054112908 -86.76957617342167) bank149265 +149266 POINT(41.19333270699155 -87.00267035601945) bank149266 +149267 POINT(41.238185574811006 -86.85442626125848) bank149267 +149268 POINT(42.11086143581293 -86.70168661068168) bank149268 +149269 POINT(41.04774729338112 -87.9255750410652) bank149269 +149270 POINT(41.44275140050334 -86.86011920296936) bank149270 +149271 POINT(40.93964110817766 -88.25721439887666) bank149271 +149272 POINT(41.1327599745086 -88.31376341231896) bank149272 +149273 POINT(40.89821933808649 -88.00324193616376) bank149273 +149274 POINT(42.44966806443227 -87.07411545407082) bank149274 +149275 POINT(42.691363211358144 -88.00660859961548) bank149275 +149276 POINT(42.80782527074198 -86.63490661074316) bank149276 +149277 POINT(42.058864461454036 -87.4514157615611) bank149277 +149278 POINT(41.47744566445194 -86.82156471044799) bank149278 +149279 POINT(40.89182499239502 -86.78091190667611) bank149279 +149280 POINT(42.13969055552832 -87.38790716251737) bank149280 +149281 POINT(41.08081937526547 -87.17445011103764) bank149281 +149282 POINT(40.943329035414386 -88.44787342690582) bank149282 +149283 POINT(41.49594093596252 -88.08587713695181) bank149283 +149284 POINT(41.668427016458566 -87.2755710060558) bank149284 +149285 POINT(41.517613291686374 -88.26314840937964) bank149285 +149286 POINT(41.41755769130868 -87.2922989170476) bank149286 +149287 POINT(42.72788492413604 -87.93629522324719) bank149287 +149288 POINT(42.267235694193886 -87.41149249903411) bank149288 +149289 POINT(42.61510505570049 -86.93780807737504) bank149289 +149290 POINT(41.95051985663098 -86.8474378932474) bank149290 +149291 POINT(41.88193147473754 -88.02242838163706) bank149291 +149292 POINT(41.2050612326591 -88.43085491677832) bank149292 +149293 POINT(42.2836526155314 -87.34295003163598) bank149293 +149294 POINT(41.44019535180172 -87.8203368041124) bank149294 +149295 POINT(42.380369501084495 -88.6075248031737) bank149295 +149296 POINT(42.8377783517407 -87.34510669181525) bank149296 +149297 POINT(42.53587620750129 -88.53383246279196) bank149297 +149298 POINT(42.50760006816177 -87.60814114537739) bank149298 +149299 POINT(41.79547904573546 -88.46682971157655) bank149299 +149300 POINT(42.089173436699184 -87.17961148968678) bank149300 +149301 POINT(41.98879675489646 -88.40086548915816) bank149301 +149302 POINT(41.15184290446381 -88.24466855776122) bank149302 +149303 POINT(41.18423188580355 -86.86606063824631) bank149303 +149304 POINT(41.75170638297016 -88.49538999996687) bank149304 +149305 POINT(41.411666704997764 -86.69532669398045) bank149305 +149306 POINT(42.717830267210374 -87.05225011860718) bank149306 +149307 POINT(42.81431911463095 -87.70416493302102) bank149307 +149308 POINT(41.932725481473646 -88.56200910942653) bank149308 +149309 POINT(42.789319219453425 -86.82914144227541) bank149309 +149310 POINT(41.69772293722856 -87.19752712859012) bank149310 +149311 POINT(40.890043085067916 -88.27918716786334) bank149311 +149312 POINT(41.639333610313145 -87.6250124882453) bank149312 +149313 POINT(42.05018987398046 -88.14162446090798) bank149313 +149314 POINT(40.92853160006181 -88.60327684079955) bank149314 +149315 POINT(42.22379592699746 -88.27484384723624) bank149315 +149316 POINT(40.88623645471933 -86.84963651942572) bank149316 +149317 POINT(41.19116429343303 -86.68506206246) bank149317 +149318 POINT(42.703146341641215 -88.49273214336017) bank149318 +149319 POINT(42.62542402835405 -87.96997860819897) bank149319 +149320 POINT(42.82618372993255 -88.24321044239154) bank149320 +149321 POINT(41.385945424979 -88.17862333432873) bank149321 +149322 POINT(42.869838076450684 -88.05546277735826) bank149322 +149323 POINT(42.8656199924784 -87.1745140513882) bank149323 +149324 POINT(41.92005108053885 -88.45669927403917) bank149324 +149325 POINT(42.527464998337166 -88.34112264757773) bank149325 +149326 POINT(41.359381456374535 -87.24723405623284) bank149326 +149327 POINT(41.96580392455023 -88.49824633801823) bank149327 +149328 POINT(42.08819960208828 -88.58520333032807) bank149328 +149329 POINT(42.75174989261648 -87.62096903785537) bank149329 +149330 POINT(41.340741021634685 -87.04610172017372) bank149330 +149331 POINT(42.4652151958956 -87.49200475356939) bank149331 +149332 POINT(41.5241933183398 -87.28752370634437) bank149332 +149333 POINT(41.967912370811625 -88.12519501829058) bank149333 +149334 POINT(42.52362777980515 -88.25148703135292) bank149334 +149335 POINT(42.44457864660683 -88.07271638147245) bank149335 +149336 POINT(42.145060100920546 -87.45599705036437) bank149336 +149337 POINT(42.28816528556278 -88.57671524929806) bank149337 +149338 POINT(42.09601755261285 -86.92220120013086) bank149338 +149339 POINT(42.13581844209946 -87.48381900007358) bank149339 +149340 POINT(41.34362461902288 -87.25581431261477) bank149340 +149341 POINT(41.47345194594707 -86.64198030902064) bank149341 +149342 POINT(42.273206334726034 -87.82367769394548) bank149342 +149343 POINT(41.33292192927995 -88.20671709698117) bank149343 +149344 POINT(41.63969659763749 -87.70868156243168) bank149344 +149345 POINT(42.21648827258584 -88.18018130174401) bank149345 +149346 POINT(41.03570345195981 -87.93804729619345) bank149346 +149347 POINT(42.54605621706538 -88.07659697747381) bank149347 +149348 POINT(42.52308327373942 -88.15618398841876) bank149348 +149349 POINT(42.16385363848053 -87.77932427472803) bank149349 +149350 POINT(41.51764924292635 -88.57792518069647) bank149350 +149351 POINT(41.51615161379769 -88.23174457042388) bank149351 +149352 POINT(41.36796081854735 -87.09844214046934) bank149352 +149353 POINT(41.95196903779546 -88.52064301740455) bank149353 +149354 POINT(42.637987781125446 -86.80610225130826) bank149354 +149355 POINT(41.74701727514718 -86.86809063153748) bank149355 +149356 POINT(42.77700920931588 -86.7430466448869) bank149356 +149357 POINT(41.33427635144841 -87.88367059959714) bank149357 +149358 POINT(41.72957180193647 -87.84094071734458) bank149358 +149359 POINT(42.27381381109304 -87.42024801431656) bank149359 +149360 POINT(41.52956416003503 -88.49536269998856) bank149360 +149361 POINT(42.52313023352177 -87.4853774719841) bank149361 +149362 POINT(42.601170416575826 -87.05567230174697) bank149362 +149363 POINT(41.188958621911176 -86.84455799973874) bank149363 +149364 POINT(41.099032003996534 -87.1107287142831) bank149364 +149365 POINT(42.587479380076935 -87.59541483440974) bank149365 +149366 POINT(42.52395088046217 -88.42655517670818) bank149366 +149367 POINT(41.58830951421301 -86.86009590412304) bank149367 +149368 POINT(41.22754681730975 -88.09372741831918) bank149368 +149369 POINT(42.72336180474586 -88.5702902897772) bank149369 +149370 POINT(42.444215112121526 -87.69101896484786) bank149370 +149371 POINT(42.14046002821867 -86.69651017912742) bank149371 +149372 POINT(41.46849055881795 -87.78541019564943) bank149372 +149373 POINT(41.58467302039748 -87.47764818845029) bank149373 +149374 POINT(41.460786169455126 -87.1621461604601) bank149374 +149375 POINT(41.82676334732694 -87.5898020826273) bank149375 +149376 POINT(41.66438084263633 -86.9045647451562) bank149376 +149377 POINT(41.77024918231652 -87.61422529209194) bank149377 +149378 POINT(41.447714366924366 -88.08248990042605) bank149378 +149379 POINT(42.7316582141558 -87.0714162114859) bank149379 +149380 POINT(42.044617215870474 -88.43145553836023) bank149380 +149381 POINT(41.973341474724755 -86.78544346579807) bank149381 +149382 POINT(42.113041706875194 -86.70166716653145) bank149382 +149383 POINT(41.416207534319206 -86.68726742760643) bank149383 +149384 POINT(41.89738125417521 -87.20161809342115) bank149384 +149385 POINT(42.440886186312646 -88.02135338097511) bank149385 +149386 POINT(41.18888687587632 -87.69626281519906) bank149386 +149387 POINT(41.81881777424653 -88.0551638856683) bank149387 +149388 POINT(41.58622216731605 -87.5211796791705) bank149388 +149389 POINT(41.46330014661502 -87.39044165626451) bank149389 +149390 POINT(41.368567916356604 -87.68730724390069) bank149390 +149391 POINT(41.44665014797693 -88.5172392207573) bank149391 +149392 POINT(41.354714581210644 -87.44668239159725) bank149392 +149393 POINT(42.54844470933902 -87.495208736602) bank149393 +149394 POINT(41.95996385399404 -88.06653444593604) bank149394 +149395 POINT(42.624801870144495 -87.78229849825816) bank149395 +149396 POINT(42.593316794545025 -86.82504001740396) bank149396 +149397 POINT(41.208467745790216 -87.08163304779906) bank149397 +149398 POINT(41.93187811028121 -87.93143208060793) bank149398 +149399 POINT(42.77268225945104 -87.26853686717628) bank149399 +149400 POINT(42.87205824173252 -87.32803151547988) bank149400 +149401 POINT(41.91152484410871 -86.69712446828055) bank149401 +149402 POINT(42.448057559079565 -87.8899420057989) bank149402 +149403 POINT(41.29576392700009 -87.20805773704629) bank149403 +149404 POINT(42.318648504321466 -86.69893648987471) bank149404 +149405 POINT(41.247793283205695 -87.11882023436578) bank149405 +149406 POINT(42.723149060841486 -87.3232137448439) bank149406 +149407 POINT(42.26341222695785 -88.29850250414354) bank149407 +149408 POINT(42.127233688206715 -87.58183598782404) bank149408 +149409 POINT(41.948480589781134 -88.54951466388457) bank149409 +149410 POINT(41.69192388982666 -87.47991715297178) bank149410 +149411 POINT(41.072542494401496 -88.1782800828846) bank149411 +149412 POINT(42.73047650686788 -88.05006248535472) bank149412 +149413 POINT(42.64389581869697 -87.24836258927478) bank149413 +149414 POINT(42.60877871166881 -87.56566451920362) bank149414 +149415 POINT(42.23472241500033 -87.58441704432789) bank149415 +149416 POINT(42.76573618933296 -88.4725980107891) bank149416 +149417 POINT(41.28668571974431 -87.16448938576815) bank149417 +149418 POINT(42.16603630482682 -87.50970128976587) bank149418 +149419 POINT(41.3316896490468 -86.86753579207962) bank149419 +149420 POINT(42.550253703728686 -88.02185592244616) bank149420 +149421 POINT(41.453908031978706 -87.32344189988311) bank149421 +149422 POINT(42.60406932277469 -87.95732920263761) bank149422 +149423 POINT(41.66871135173544 -87.44608667015547) bank149423 +149424 POINT(41.39532620026279 -87.13342834130566) bank149424 +149425 POINT(41.90838777427756 -87.51437930195706) bank149425 +149426 POINT(41.22680388382605 -87.70582998393863) bank149426 +149427 POINT(41.55509117001734 -86.82631327005352) bank149427 +149428 POINT(42.13079497094164 -87.91386522285268) bank149428 +149429 POINT(42.5170568412819 -86.68138215057476) bank149429 +149430 POINT(42.635735957597795 -86.63454908238907) bank149430 +149431 POINT(41.71900840162912 -87.52506855420833) bank149431 +149432 POINT(41.92875323822956 -86.968454331) bank149432 +149433 POINT(41.287169020026425 -87.31099806251974) bank149433 +149434 POINT(42.563571857385774 -87.36434557226512) bank149434 +149435 POINT(41.27086900688235 -87.93331702978153) bank149435 +149436 POINT(42.777813040110935 -86.66286428115586) bank149436 +149437 POINT(41.84166639794125 -86.64525524804273) bank149437 +149438 POINT(40.99722278757765 -88.2559680281634) bank149438 +149439 POINT(41.35967744639289 -87.65700242291891) bank149439 +149440 POINT(42.28099430287223 -86.99996785555757) bank149440 +149441 POINT(41.45650687668068 -87.7756394411343) bank149441 +149442 POINT(42.68491609437037 -88.03151424782615) bank149442 +149443 POINT(42.202176999959256 -88.17379438408219) bank149443 +149444 POINT(41.755971413024255 -86.96213986018974) bank149444 +149445 POINT(42.589735235235004 -86.82084487303918) bank149445 +149446 POINT(41.94655290245695 -88.26880297181808) bank149446 +149447 POINT(42.54545144960397 -88.1349734389181) bank149447 +149448 POINT(41.59724729725429 -88.0610909444885) bank149448 +149449 POINT(42.730670898310684 -87.96986221373632) bank149449 +149450 POINT(40.95853386296989 -87.4070393171388) bank149450 +149451 POINT(41.05150893400284 -88.60410532749104) bank149451 +149452 POINT(42.030725892287506 -86.97182811146871) bank149452 +149453 POINT(41.817196229591026 -87.49025674650234) bank149453 +149454 POINT(42.32800950520106 -88.39071944924395) bank149454 +149455 POINT(42.72188909082221 -86.80180931896848) bank149455 +149456 POINT(42.074175800598965 -87.79572519949262) bank149456 +149457 POINT(41.028496226150764 -86.95870403722829) bank149457 +149458 POINT(42.58245999354297 -87.39532288570055) bank149458 +149459 POINT(41.70844057854676 -87.76078655362073) bank149459 +149460 POINT(42.63666917360314 -86.69950747136322) bank149460 +149461 POINT(41.817952567676016 -87.20058029271934) bank149461 +149462 POINT(40.9251846960752 -88.5769361960651) bank149462 +149463 POINT(42.197907328736186 -87.32376960703196) bank149463 +149464 POINT(41.82686370047787 -87.08207623694847) bank149464 +149465 POINT(42.266031419463296 -88.09057299477159) bank149465 +149466 POINT(42.687707875521276 -87.31932580079746) bank149466 +149467 POINT(41.692132694414646 -86.83132001807479) bank149467 +149468 POINT(42.1560360262121 -86.88887759470904) bank149468 +149469 POINT(40.947405893985895 -88.18108371917259) bank149469 +149470 POINT(42.72921709610266 -87.60038524937394) bank149470 +149471 POINT(42.62106989922284 -87.18943182712755) bank149471 +149472 POINT(41.31753820703032 -88.15140012396584) bank149472 +149473 POINT(41.35910632091391 -86.68972375640195) bank149473 +149474 POINT(41.01946334815406 -88.36754072299642) bank149474 +149475 POINT(41.16573565886722 -88.2413975696201) bank149475 +149476 POINT(42.56427253775889 -87.66337325451737) bank149476 +149477 POINT(41.675198480405356 -88.28651429092923) bank149477 +149478 POINT(42.33110109811839 -87.75447661192099) bank149478 +149479 POINT(41.248126398711726 -88.33963064886731) bank149479 +149480 POINT(41.675519512202264 -86.93185383783012) bank149480 +149481 POINT(42.45157452807164 -86.83919322062916) bank149481 +149482 POINT(41.27369566751893 -88.56107503429399) bank149482 +149483 POINT(41.25807278596163 -88.54627280680779) bank149483 +149484 POINT(41.63639943414781 -88.44007815978932) bank149484 +149485 POINT(41.02349811978402 -88.31429007020638) bank149485 +149486 POINT(41.5604455123934 -87.39907356125792) bank149486 +149487 POINT(41.07410171470165 -86.85262578176778) bank149487 +149488 POINT(41.41521171255547 -88.0049210574573) bank149488 +149489 POINT(41.69098119068381 -86.85063693214161) bank149489 +149490 POINT(41.8241223768723 -87.42717143565064) bank149490 +149491 POINT(42.71979558707898 -87.18760719175737) bank149491 +149492 POINT(42.87407622869928 -88.37237201020557) bank149492 +149493 POINT(41.50745875157423 -87.24944648398534) bank149493 +149494 POINT(41.19192913789756 -87.1305835778654) bank149494 +149495 POINT(41.8487357876019 -87.16199921297891) bank149495 +149496 POINT(41.94382695738529 -87.98186995031021) bank149496 +149497 POINT(41.814339289554525 -86.68559423339356) bank149497 +149498 POINT(40.92652413404479 -86.87622871241743) bank149498 +149499 POINT(41.8337018308931 -88.18989449145157) bank149499 +149500 POINT(41.950065628908376 -86.93336669944124) bank149500 +149501 POINT(42.713787344937266 -87.92892747926545) bank149501 +149502 POINT(41.603451652700656 -88.44956614949888) bank149502 +149503 POINT(42.499153925332145 -87.1164315530517) bank149503 +149504 POINT(42.20722141115892 -88.57084725409335) bank149504 +149505 POINT(41.94643257879643 -87.50751461401673) bank149505 +149506 POINT(42.39410110184998 -87.51936064369986) bank149506 +149507 POINT(40.93864150320413 -88.22638665069515) bank149507 +149508 POINT(41.5554116712353 -88.28343336300075) bank149508 +149509 POINT(42.3116132555996 -87.19640587425023) bank149509 +149510 POINT(41.26696944391775 -86.98311364665574) bank149510 +149511 POINT(42.25446761540642 -87.96765438512543) bank149511 +149512 POINT(42.39453972342845 -88.34438292465693) bank149512 +149513 POINT(41.262541590457445 -87.0387672872095) bank149513 +149514 POINT(41.13822791041447 -86.9729759446112) bank149514 +149515 POINT(41.74018558905574 -87.36108730689848) bank149515 +149516 POINT(42.05360273061192 -87.45860412015463) bank149516 +149517 POINT(41.62725733799216 -87.82999133853755) bank149517 +149518 POINT(41.03516516795625 -87.84990790687496) bank149518 +149519 POINT(40.98441795799265 -87.64513678494315) bank149519 +149520 POINT(42.837787055664734 -86.78142369150815) bank149520 +149521 POINT(41.16438743474192 -86.67513555426282) bank149521 +149522 POINT(42.7710819666837 -88.55841475636012) bank149522 +149523 POINT(41.227092492980425 -87.91351218436085) bank149523 +149524 POINT(42.85379017574006 -88.49718229181961) bank149524 +149525 POINT(41.89941777005633 -87.61984022516268) bank149525 +149526 POINT(42.096213587880555 -87.48578060583667) bank149526 +149527 POINT(41.015227113849086 -88.27355000215879) bank149527 +149528 POINT(41.8680853851213 -87.0066589079778) bank149528 +149529 POINT(42.143118581892566 -87.45075735799394) bank149529 +149530 POINT(41.032958638327536 -87.17710738318812) bank149530 +149531 POINT(41.534353496770976 -87.69368242326973) bank149531 +149532 POINT(42.24315418378143 -88.05561851948376) bank149532 +149533 POINT(41.28650896966008 -87.89650660673077) bank149533 +149534 POINT(41.81193146711318 -88.11691024669713) bank149534 +149535 POINT(42.53460880485071 -86.7371142290229) bank149535 +149536 POINT(41.23389776543696 -86.96881592405833) bank149536 +149537 POINT(42.380652042529654 -88.30398412887466) bank149537 +149538 POINT(41.459102624400444 -87.3269748689005) bank149538 +149539 POINT(42.47078274637891 -87.62216090802019) bank149539 +149540 POINT(41.26938689546529 -88.20618640658404) bank149540 +149541 POINT(42.58838416569143 -87.13153329694455) bank149541 +149542 POINT(42.10587151860431 -88.10442199555088) bank149542 +149543 POINT(41.17937830289693 -87.24077502368836) bank149543 +149544 POINT(41.019424539831256 -87.02266733605965) bank149544 +149545 POINT(42.640329550368925 -87.58438903165636) bank149545 +149546 POINT(41.131346272712356 -88.51631529944096) bank149546 +149547 POINT(42.31215577806399 -87.86284720162489) bank149547 +149548 POINT(40.91769544426086 -86.8520712365748) bank149548 +149549 POINT(41.84402006321107 -86.90441234641305) bank149549 +149550 POINT(42.20843421902191 -87.7110616013057) bank149550 +149551 POINT(41.11854050418121 -86.89917569910422) bank149551 +149552 POINT(42.12733643305458 -88.59892199711155) bank149552 +149553 POINT(42.7966651182153 -88.41367322622588) bank149553 +149554 POINT(41.08639930313897 -88.18202549176699) bank149554 +149555 POINT(41.7902850169523 -86.92477885257371) bank149555 +149556 POINT(42.026606250844594 -87.13629726234008) bank149556 +149557 POINT(42.53786028096803 -88.24232834521317) bank149557 +149558 POINT(41.834805404772794 -86.9406809610706) bank149558 +149559 POINT(41.488820640174666 -87.40644500864406) bank149559 +149560 POINT(42.012124766590006 -87.77508335363342) bank149560 +149561 POINT(41.52357515256782 -86.99029222917912) bank149561 +149562 POINT(42.0538860276055 -88.14327271796637) bank149562 +149563 POINT(42.77485193075601 -88.14373086764833) bank149563 +149564 POINT(42.169253245834305 -87.55668411142892) bank149564 +149565 POINT(41.54301701101182 -87.62200874783251) bank149565 +149566 POINT(42.50769572238035 -87.23110692591781) bank149566 +149567 POINT(41.92394226009694 -87.0664175516302) bank149567 +149568 POINT(42.48839732723039 -87.36944355582206) bank149568 +149569 POINT(41.555842997182616 -87.31500982871832) bank149569 +149570 POINT(42.31375912799848 -88.45670412633834) bank149570 +149571 POINT(41.267463190500614 -88.50132915204884) bank149571 +149572 POINT(41.66839933382398 -88.54843764315063) bank149572 +149573 POINT(42.54749932824284 -87.91036392318301) bank149573 +149574 POINT(41.95127277040573 -86.92636150316797) bank149574 +149575 POINT(41.37646518180705 -88.23607873744706) bank149575 +149576 POINT(42.57978022137964 -88.18813428106203) bank149576 +149577 POINT(41.22804751625543 -86.72264041384473) bank149577 +149578 POINT(41.0989800294665 -87.94949136482275) bank149578 +149579 POINT(42.02226593625607 -86.98203458371873) bank149579 +149580 POINT(42.215825056425395 -87.31978487397487) bank149580 +149581 POINT(41.300873557093226 -88.47388435038383) bank149581 +149582 POINT(41.460413763029685 -88.40049427063836) bank149582 +149583 POINT(41.88918927080412 -87.20936588166991) bank149583 +149584 POINT(42.85181564346183 -86.74097172077607) bank149584 +149585 POINT(41.95002209375707 -87.2696502871851) bank149585 +149586 POINT(42.3582159698839 -88.37422028992513) bank149586 +149587 POINT(41.62585515893312 -87.08579251376491) bank149587 +149588 POINT(41.96961254791076 -87.60748068494405) bank149588 +149589 POINT(41.34884876073468 -88.33613786880878) bank149589 +149590 POINT(41.39146821328076 -87.69984899683914) bank149590 +149591 POINT(41.393383813967304 -86.81866970226336) bank149591 +149592 POINT(41.25612288463485 -88.20782786729525) bank149592 +149593 POINT(41.651292944759525 -86.65021665699435) bank149593 +149594 POINT(41.37395498292492 -87.51415290961893) bank149594 +149595 POINT(42.29410950039388 -88.36266777684094) bank149595 +149596 POINT(41.067521522374086 -88.24202774700994) bank149596 +149597 POINT(41.16805609946199 -88.0835391927176) bank149597 +149598 POINT(41.185948358078406 -88.57015243365562) bank149598 +149599 POINT(41.253706533701006 -87.04611660729853) bank149599 +149600 POINT(41.931796812252706 -87.79224220538259) bank149600 +149601 POINT(42.54560831091338 -88.44866787078084) bank149601 +149602 POINT(41.02532690772442 -87.5948199432318) bank149602 +149603 POINT(42.559557729678666 -87.37822794430537) bank149603 +149604 POINT(41.50241335608123 -88.50127933541332) bank149604 +149605 POINT(42.26277756318686 -87.62370950494841) bank149605 +149606 POINT(41.3221133815568 -87.50361819191735) bank149606 +149607 POINT(41.46557719905431 -87.870835911119) bank149607 +149608 POINT(41.76221771922788 -87.80589999015058) bank149608 +149609 POINT(42.25893888825162 -87.59218761258883) bank149609 +149610 POINT(41.71788836314102 -88.45559741738799) bank149610 +149611 POINT(41.04144122104305 -88.0298896308972) bank149611 +149612 POINT(41.088918281857765 -88.11376461863411) bank149612 +149613 POINT(41.75673076279649 -87.00035237210021) bank149613 +149614 POINT(42.133013505462195 -88.39518896216235) bank149614 +149615 POINT(41.18531970504306 -86.76249421090837) bank149615 +149616 POINT(40.92679793856777 -88.51405462125834) bank149616 +149617 POINT(41.602260107618264 -87.73326724709426) bank149617 +149618 POINT(42.24806849908464 -88.40289792039783) bank149618 +149619 POINT(41.60749879454054 -87.02448427067262) bank149619 +149620 POINT(42.20540534355972 -87.4091165203437) bank149620 +149621 POINT(41.11575226238963 -87.56066066439368) bank149621 +149622 POINT(41.335371189599414 -86.63924187942946) bank149622 +149623 POINT(42.57501654751027 -87.85646276125995) bank149623 +149624 POINT(41.15689385647718 -87.31639122470109) bank149624 +149625 POINT(40.94867270753685 -87.23533408112654) bank149625 +149626 POINT(42.13719990251925 -87.83351006831975) bank149626 +149627 POINT(41.6529879336447 -88.06350435076249) bank149627 +149628 POINT(41.14610206383305 -88.33804211477694) bank149628 +149629 POINT(41.862545965779674 -87.42500159267767) bank149629 +149630 POINT(42.48339959603026 -87.66949872722185) bank149630 +149631 POINT(42.60623640598594 -86.6475790375607) bank149631 +149632 POINT(41.25334379940674 -88.58096819624227) bank149632 +149633 POINT(41.55314575954755 -87.34583847571942) bank149633 +149634 POINT(42.859291559481534 -86.99611232028234) bank149634 +149635 POINT(41.502227784955856 -87.06539845617894) bank149635 +149636 POINT(42.789905583321854 -86.8054016089553) bank149636 +149637 POINT(42.77471174608466 -87.51370218436486) bank149637 +149638 POINT(42.265811979814146 -87.85581020064919) bank149638 +149639 POINT(42.74594932660694 -87.4825065544812) bank149639 +149640 POINT(41.69311900401863 -87.10655343997477) bank149640 +149641 POINT(42.31815203815252 -88.51350840004132) bank149641 +149642 POINT(42.28311274063088 -87.41594722482088) bank149642 +149643 POINT(41.08088020136641 -87.08335009703002) bank149643 +149644 POINT(41.78596931682654 -87.72997626109692) bank149644 +149645 POINT(42.044877564991495 -86.68549009153031) bank149645 +149646 POINT(42.44088298810757 -88.40656453248263) bank149646 +149647 POINT(42.19599683103361 -87.17656453142577) bank149647 +149648 POINT(41.70515997000163 -87.54262662429508) bank149648 +149649 POINT(41.933988890094305 -87.17730631735479) bank149649 +149650 POINT(41.86655456622342 -87.75799271279566) bank149650 +149651 POINT(41.7460725206967 -87.92724524622048) bank149651 +149652 POINT(41.04190983932698 -87.25810751708795) bank149652 +149653 POINT(42.43805128494163 -88.50472022461878) bank149653 +149654 POINT(42.72361606594038 -87.85892541708758) bank149654 +149655 POINT(42.453913461157825 -87.03811938664666) bank149655 +149656 POINT(41.43405006924594 -87.3208491122971) bank149656 +149657 POINT(42.83619263937049 -87.36437008882868) bank149657 +149658 POINT(41.03453860939569 -86.80636413308011) bank149658 +149659 POINT(42.33238731349918 -88.41139404132376) bank149659 +149660 POINT(42.19794943686477 -87.40886721251647) bank149660 +149661 POINT(41.17770053980688 -87.17958945099487) bank149661 +149662 POINT(41.07611333785461 -86.90579079819929) bank149662 +149663 POINT(42.26732323103858 -86.63709814737723) bank149663 +149664 POINT(42.810690707458875 -88.60950817960664) bank149664 +149665 POINT(41.71916697002405 -88.07076047820499) bank149665 +149666 POINT(41.82718104970702 -87.34653841961895) bank149666 +149667 POINT(42.20198367496409 -88.1082994115653) bank149667 +149668 POINT(42.000476717410464 -88.14928989677162) bank149668 +149669 POINT(41.72056215737295 -87.23837964010288) bank149669 +149670 POINT(41.153563350749536 -88.51889823244782) bank149670 +149671 POINT(42.66874666672012 -87.59818572451873) bank149671 +149672 POINT(41.67909349460582 -88.32259959809943) bank149672 +149673 POINT(41.938682600725045 -87.5586940239422) bank149673 +149674 POINT(42.21309623969713 -86.96755919476398) bank149674 +149675 POINT(42.30895758727414 -86.79726062014329) bank149675 +149676 POINT(40.87859289472109 -88.43083054683933) bank149676 +149677 POINT(41.29318333975225 -87.00482111005196) bank149677 +149678 POINT(42.652793039474204 -87.8486462356339) bank149678 +149679 POINT(42.23036070400779 -86.94994717455484) bank149679 +149680 POINT(41.2040933408433 -87.41594651336574) bank149680 +149681 POINT(41.1198561632487 -87.79875768970392) bank149681 +149682 POINT(42.297478909857304 -88.57456790783776) bank149682 +149683 POINT(41.90660488677445 -86.9356510930861) bank149683 +149684 POINT(41.373367868473224 -88.24406192444307) bank149684 +149685 POINT(41.83917863217857 -87.51864988317601) bank149685 +149686 POINT(42.337029011910836 -88.38429391316883) bank149686 +149687 POINT(41.556017786453765 -88.36136179090414) bank149687 +149688 POINT(42.231065463151516 -88.38378051819092) bank149688 +149689 POINT(41.44216170577431 -87.99108943773263) bank149689 +149690 POINT(41.02428880668457 -87.62743789788395) bank149690 +149691 POINT(41.36963669135544 -88.11916147092147) bank149691 +149692 POINT(42.118606243890106 -86.89816418740888) bank149692 +149693 POINT(42.4527404761732 -86.63694286158535) bank149693 +149694 POINT(41.98467067628659 -86.96403756857808) bank149694 +149695 POINT(41.302964917710135 -87.25643869354481) bank149695 +149696 POINT(42.42450769708574 -86.83443473201554) bank149696 +149697 POINT(41.665898064978535 -88.53750879868994) bank149697 +149698 POINT(41.237577269695265 -87.53269556530444) bank149698 +149699 POINT(41.69544900070903 -88.22636135512363) bank149699 +149700 POINT(41.841013703589894 -88.4219488261057) bank149700 +149701 POINT(42.65958165563175 -86.66161271196664) bank149701 +149702 POINT(41.32139234341559 -87.53267563298829) bank149702 +149703 POINT(42.77632645560071 -86.81980890041724) bank149703 +149704 POINT(41.1858746342247 -87.083159495585) bank149704 +149705 POINT(41.989460740636396 -87.65242235479643) bank149705 +149706 POINT(41.628158981264306 -87.3854709587707) bank149706 +149707 POINT(41.76811096829554 -88.38483033471925) bank149707 +149708 POINT(42.74913367713376 -88.6128017406523) bank149708 +149709 POINT(41.021370054867056 -88.45094428622818) bank149709 +149710 POINT(41.93613440171891 -87.98604989612787) bank149710 +149711 POINT(41.26948637685777 -87.82916110708557) bank149711 +149712 POINT(42.863541647077874 -87.50094174892382) bank149712 +149713 POINT(41.85675864626063 -88.5810801544466) bank149713 +149714 POINT(41.01006993744248 -88.04988729980843) bank149714 +149715 POINT(42.73198281928867 -88.45558516189796) bank149715 +149716 POINT(41.593552028376436 -87.79228488484522) bank149716 +149717 POINT(41.91602446072707 -87.84433169159193) bank149717 +149718 POINT(42.62582371624718 -88.28239966941534) bank149718 +149719 POINT(41.55571538747581 -87.32313662611712) bank149719 +149720 POINT(42.2252774893349 -86.93595014128431) bank149720 +149721 POINT(42.02856435283847 -87.98633578833876) bank149721 +149722 POINT(42.13818261529847 -87.31312377312027) bank149722 +149723 POINT(41.50415305387353 -87.72958205164765) bank149723 +149724 POINT(42.05684310069145 -87.07618578010297) bank149724 +149725 POINT(42.57207013487455 -87.27750334114657) bank149725 +149726 POINT(41.67891461554114 -86.98221183686557) bank149726 +149727 POINT(41.17263556851002 -87.75543678124168) bank149727 +149728 POINT(42.00972331877631 -88.47935315885371) bank149728 +149729 POINT(41.15074453568916 -87.05954485186811) bank149729 +149730 POINT(41.00074805784969 -86.82345996127304) bank149730 +149731 POINT(42.71698592929219 -88.3938217587796) bank149731 +149732 POINT(40.98205772240369 -87.37550313168377) bank149732 +149733 POINT(41.251364686961516 -86.68833032899303) bank149733 +149734 POINT(41.118871960109196 -87.05869021988633) bank149734 +149735 POINT(41.39090497199835 -86.86233161171414) bank149735 +149736 POINT(41.77585792312895 -87.95766020907281) bank149736 +149737 POINT(42.230945580275375 -87.30839816540292) bank149737 +149738 POINT(41.74947102367559 -86.65980457952072) bank149738 +149739 POINT(41.45403099239405 -88.56412382158544) bank149739 +149740 POINT(42.615163009349104 -86.7901472602987) bank149740 +149741 POINT(41.69983179266557 -88.017420801424) bank149741 +149742 POINT(41.05337974711301 -87.42217820847024) bank149742 +149743 POINT(41.37380613865248 -88.03638027581701) bank149743 +149744 POINT(40.92652570090181 -88.27713361454934) bank149744 +149745 POINT(42.345371994406186 -88.47515170260056) bank149745 +149746 POINT(41.13566451978656 -88.2632854368607) bank149746 +149747 POINT(42.368618265481466 -88.32844904201569) bank149747 +149748 POINT(42.181990090499994 -88.3851129030953) bank149748 +149749 POINT(41.09717819645204 -87.73658866412451) bank149749 +149750 POINT(40.94825024030391 -88.1064205481202) bank149750 +149751 POINT(41.85495248782514 -87.85579900905888) bank149751 +149752 POINT(42.284170823730854 -87.58270557198024) bank149752 +149753 POINT(40.976373920384304 -86.70042975726801) bank149753 +149754 POINT(41.009765422906334 -87.9707081714744) bank149754 +149755 POINT(42.400974375563244 -88.25499782104227) bank149755 +149756 POINT(41.46175864832406 -87.87274877115325) bank149756 +149757 POINT(41.260075884896764 -88.07921522036122) bank149757 +149758 POINT(42.53515903687592 -88.23938461619565) bank149758 +149759 POINT(41.92159509638513 -88.18699142024046) bank149759 +149760 POINT(40.931268829286246 -88.2296795317164) bank149760 +149761 POINT(41.97029254566586 -86.67119463083114) bank149761 +149762 POINT(41.53138368926467 -86.89454491169174) bank149762 +149763 POINT(41.91812031571644 -87.26039650274278) bank149763 +149764 POINT(41.06305937360257 -87.73466872743685) bank149764 +149765 POINT(41.29782987598004 -88.14956644554343) bank149765 +149766 POINT(41.781745400533524 -88.29560057915135) bank149766 +149767 POINT(41.737617081982194 -87.86290466462893) bank149767 +149768 POINT(41.29877450133236 -87.87767971138794) bank149768 +149769 POINT(42.1341716430894 -88.11804134391582) bank149769 +149770 POINT(42.786676116745106 -88.55685230792149) bank149770 +149771 POINT(42.38411786957029 -87.5418427471125) bank149771 +149772 POINT(41.010342315711995 -88.13280192727679) bank149772 +149773 POINT(42.10307470579152 -88.0421296041104) bank149773 +149774 POINT(42.66999480289193 -87.21749264064393) bank149774 +149775 POINT(42.36141239344899 -86.67263283461624) bank149775 +149776 POINT(41.08735429551402 -88.46826192528727) bank149776 +149777 POINT(41.89376322708004 -87.20103840862535) bank149777 +149778 POINT(41.007383874790484 -87.39553397591695) bank149778 +149779 POINT(42.312398933125706 -87.29436293580031) bank149779 +149780 POINT(42.581029752679946 -88.0215468299984) bank149780 +149781 POINT(42.68819180884884 -88.40925935809717) bank149781 +149782 POINT(41.457452330886625 -88.4417744497646) bank149782 +149783 POINT(41.58562736211964 -88.45407043651032) bank149783 +149784 POINT(42.65109332599763 -88.45090145170082) bank149784 +149785 POINT(41.37656206079468 -87.47049758366289) bank149785 +149786 POINT(41.52890085996414 -88.20343039031553) bank149786 +149787 POINT(42.54853522437418 -86.8104079404289) bank149787 +149788 POINT(41.01762127271299 -87.4244293305154) bank149788 +149789 POINT(42.48005277081961 -88.56578044553783) bank149789 +149790 POINT(41.069817042258954 -87.52621512008757) bank149790 +149791 POINT(41.33541744336407 -87.56098888357565) bank149791 +149792 POINT(41.30537105800839 -87.45300797416046) bank149792 +149793 POINT(41.31162274543266 -86.91940619773847) bank149793 +149794 POINT(41.06615906754639 -87.37798406106195) bank149794 +149795 POINT(42.37397895134742 -88.25735138335168) bank149795 +149796 POINT(42.10316901411204 -87.63694419311614) bank149796 +149797 POINT(40.91019095315524 -87.4552456491146) bank149797 +149798 POINT(41.49462648263157 -86.77114838103164) bank149798 +149799 POINT(42.77912202129447 -87.6257497584928) bank149799 +149800 POINT(42.74043589993202 -88.55130868367331) bank149800 +149801 POINT(41.06195378938861 -87.14773059743378) bank149801 +149802 POINT(41.30979140414413 -88.43024937764297) bank149802 +149803 POINT(41.34908240956534 -88.0935129517602) bank149803 +149804 POINT(40.97903910576944 -88.44851555041369) bank149804 +149805 POINT(42.45676656005824 -87.35625361193647) bank149805 +149806 POINT(42.83289306483024 -86.64465754807331) bank149806 +149807 POINT(40.87990512923032 -87.52004659434287) bank149807 +149808 POINT(42.61863793506569 -87.11257375822747) bank149808 +149809 POINT(42.12586372409424 -88.4237252164295) bank149809 +149810 POINT(41.31556142435395 -86.98410933984121) bank149810 +149811 POINT(40.95638742625454 -88.41776901787684) bank149811 +149812 POINT(40.985605779716494 -88.52902313310528) bank149812 +149813 POINT(42.08499696735127 -86.74235203448268) bank149813 +149814 POINT(42.16820122792541 -87.95477907935238) bank149814 +149815 POINT(42.04182680999843 -87.83118631493635) bank149815 +149816 POINT(41.16019143308559 -87.73447591791198) bank149816 +149817 POINT(41.030707969072424 -88.52654573881568) bank149817 +149818 POINT(41.79099057225836 -87.17763145379372) bank149818 +149819 POINT(42.278301193022244 -87.39079018269811) bank149819 +149820 POINT(41.0810714345872 -86.90404582648104) bank149820 +149821 POINT(41.547851741446806 -87.00840040843157) bank149821 +149822 POINT(41.02405136395523 -88.24393334181465) bank149822 +149823 POINT(42.49697665764668 -88.4809082147518) bank149823 +149824 POINT(40.982794074156836 -88.59435702558939) bank149824 +149825 POINT(41.62162229529383 -86.83847739794682) bank149825 +149826 POINT(41.41609156653724 -88.11669831992616) bank149826 +149827 POINT(42.86597853499053 -88.00067964144277) bank149827 +149828 POINT(41.82117436677624 -87.2595370708617) bank149828 +149829 POINT(41.50248705606186 -88.41011785126915) bank149829 +149830 POINT(42.340317236047646 -87.70149584648343) bank149830 +149831 POINT(41.62560302698408 -88.58795706021925) bank149831 +149832 POINT(42.41224320306645 -87.29974048868253) bank149832 +149833 POINT(41.5114880667813 -86.94655628305493) bank149833 +149834 POINT(41.77958765304088 -88.10180851361878) bank149834 +149835 POINT(42.12991261036766 -86.7064406369166) bank149835 +149836 POINT(42.13565794752585 -87.42125044328688) bank149836 +149837 POINT(41.80769715080702 -86.99074285154856) bank149837 +149838 POINT(42.676877941539175 -88.2871131767939) bank149838 +149839 POINT(42.801575427511835 -86.73041637363279) bank149839 +149840 POINT(42.409354523176134 -87.10297882642634) bank149840 +149841 POINT(41.050603858522855 -88.0784925419869) bank149841 +149842 POINT(41.887384979757506 -87.4441479184539) bank149842 +149843 POINT(41.02945998975447 -88.53965948814577) bank149843 +149844 POINT(41.6158060105442 -87.42866199033568) bank149844 +149845 POINT(40.885286682140595 -88.55384049743981) bank149845 +149846 POINT(42.71088960910397 -87.67176213225028) bank149846 +149847 POINT(41.60461893974717 -87.41001847204568) bank149847 +149848 POINT(41.59647651435786 -87.7589464930451) bank149848 +149849 POINT(42.43028418074539 -87.94601846348083) bank149849 +149850 POINT(42.34394731185209 -87.58199854124852) bank149850 +149851 POINT(42.15026987008102 -87.10308881769168) bank149851 +149852 POINT(42.66467479966413 -88.04712297799612) bank149852 +149853 POINT(41.64782362907741 -88.0934421571033) bank149853 +149854 POINT(41.593863604943486 -88.44514318765484) bank149854 +149855 POINT(41.75903157644229 -88.2349121296742) bank149855 +149856 POINT(42.06649197322149 -86.94254082318987) bank149856 +149857 POINT(42.70124574714301 -87.87444976883357) bank149857 +149858 POINT(42.11569980537566 -86.94055199399598) bank149858 +149859 POINT(41.773345525194955 -87.61460247099973) bank149859 +149860 POINT(41.938690165186955 -86.7988031712431) bank149860 +149861 POINT(42.33039214107575 -87.02417707504779) bank149861 +149862 POINT(41.383737123172196 -88.42136648907697) bank149862 +149863 POINT(42.81037683545384 -88.30302954285222) bank149863 +149864 POINT(42.21343052292326 -87.41151254198661) bank149864 +149865 POINT(41.2135475412635 -87.9481757507699) bank149865 +149866 POINT(41.33558022194225 -86.75391205106287) bank149866 +149867 POINT(41.505739240483955 -87.96899900085607) bank149867 +149868 POINT(41.451039961960994 -86.95280333751803) bank149868 +149869 POINT(42.685307575543455 -88.13033321050438) bank149869 +149870 POINT(41.380368302517326 -87.42395897475983) bank149870 +149871 POINT(42.400549172256 -87.24145923364233) bank149871 +149872 POINT(42.588634476009474 -87.37713649969517) bank149872 +149873 POINT(41.72285419524623 -87.03810748992534) bank149873 +149874 POINT(41.043202583180154 -87.9441111515632) bank149874 +149875 POINT(41.262836178788994 -88.55664195506922) bank149875 +149876 POINT(42.73910448881458 -88.41074337865312) bank149876 +149877 POINT(41.56029637228353 -86.85998959785898) bank149877 +149878 POINT(41.29803460682662 -88.55805516751659) bank149878 +149879 POINT(42.53480355311142 -87.76293628462149) bank149879 +149880 POINT(42.5521843601754 -87.16199268719252) bank149880 +149881 POINT(42.785321363559724 -87.2534283834332) bank149881 +149882 POINT(40.97377195675245 -87.20404127326212) bank149882 +149883 POINT(42.688846265575215 -88.3636337279324) bank149883 +149884 POINT(41.35646657664661 -88.55392766321552) bank149884 +149885 POINT(41.63677760273035 -88.25493926800038) bank149885 +149886 POINT(42.330747963099434 -87.0245762186997) bank149886 +149887 POINT(42.72489130202554 -88.17826102127648) bank149887 +149888 POINT(41.22283181367833 -88.2750757690239) bank149888 +149889 POINT(42.38598201223889 -87.30551660794396) bank149889 +149890 POINT(41.13513716257444 -88.56790279494469) bank149890 +149891 POINT(41.15485581538179 -87.76995532330837) bank149891 +149892 POINT(42.14853461426665 -88.18955358229542) bank149892 +149893 POINT(41.03401347234252 -87.01983645275524) bank149893 +149894 POINT(42.32021205883294 -88.0267890458684) bank149894 +149895 POINT(42.635703221548475 -87.48505736046594) bank149895 +149896 POINT(42.17875509280113 -87.22032065051151) bank149896 +149897 POINT(41.5393886189805 -87.123537107434) bank149897 +149898 POINT(42.43035895690726 -87.25748580660407) bank149898 +149899 POINT(41.93205526975093 -87.55957299594645) bank149899 +149900 POINT(42.6804061790482 -86.74201972937854) bank149900 +149901 POINT(41.84286357265317 -88.36725489620393) bank149901 +149902 POINT(42.54741895869394 -86.7494069400411) bank149902 +149903 POINT(41.064242559534485 -87.66340841677139) bank149903 +149904 POINT(42.7176875937175 -87.7747938118288) bank149904 +149905 POINT(41.23049701160188 -88.13374872924527) bank149905 +149906 POINT(42.86396699479244 -87.50142806115718) bank149906 +149907 POINT(41.83562922312329 -87.89457335900372) bank149907 +149908 POINT(41.64962627560566 -88.14539309067888) bank149908 +149909 POINT(41.12243136410326 -88.1691571978439) bank149909 +149910 POINT(42.60733226287427 -88.15733529024544) bank149910 +149911 POINT(40.98972368944349 -87.62401543475126) bank149911 +149912 POINT(40.90177074896908 -88.00754202731608) bank149912 +149913 POINT(42.57197700160612 -88.45800263481289) bank149913 +149914 POINT(40.94960649481613 -88.36675766408054) bank149914 +149915 POINT(41.47211454915637 -86.71353555418366) bank149915 +149916 POINT(42.35589608217838 -87.33291592529847) bank149916 +149917 POINT(41.8929489750977 -86.89761500465386) bank149917 +149918 POINT(41.64248089250475 -87.016552289733) bank149918 +149919 POINT(42.74663544698521 -87.64219018183294) bank149919 +149920 POINT(41.36359472340391 -87.62163375378195) bank149920 +149921 POINT(42.35894698928917 -87.2766975939128) bank149921 +149922 POINT(42.8304757394367 -87.14948659178597) bank149922 +149923 POINT(42.85408173740428 -87.36828605289622) bank149923 +149924 POINT(41.602504604711996 -88.428775459656) bank149924 +149925 POINT(42.85287106446209 -87.88000725330778) bank149925 +149926 POINT(41.148477889228545 -86.80940718603863) bank149926 +149927 POINT(42.42631870537102 -88.02638125594454) bank149927 +149928 POINT(42.015071326275816 -87.01808132829615) bank149928 +149929 POINT(42.655862418971154 -88.3007055326104) bank149929 +149930 POINT(41.455611319190346 -88.32413216060475) bank149930 +149931 POINT(41.24774447679306 -86.67710338418208) bank149931 +149932 POINT(42.599900587721535 -87.36325913291675) bank149932 +149933 POINT(41.66035786950578 -87.7238369629856) bank149933 +149934 POINT(42.46880380388863 -88.4777652991491) bank149934 +149935 POINT(41.68197342961666 -88.22386654034074) bank149935 +149936 POINT(41.323746828527064 -86.71651696897172) bank149936 +149937 POINT(41.990548565351084 -87.65423657087528) bank149937 +149938 POINT(42.33233022141459 -86.76390409438719) bank149938 +149939 POINT(42.40482859631658 -88.14968039908368) bank149939 +149940 POINT(41.08261452577129 -88.2342957253224) bank149940 +149941 POINT(42.243206828001206 -87.17601044864406) bank149941 +149942 POINT(41.44410788430408 -87.98254153186953) bank149942 +149943 POINT(42.48593989113928 -88.06385223625105) bank149943 +149944 POINT(42.25945760309391 -87.04045335239535) bank149944 +149945 POINT(42.618825489593675 -87.4395766733654) bank149945 +149946 POINT(42.470828917635785 -87.02129730007076) bank149946 +149947 POINT(42.064311818827086 -88.5172380781063) bank149947 +149948 POINT(42.48816112362565 -87.3108108040522) bank149948 +149949 POINT(41.566976717790105 -86.76853201292664) bank149949 +149950 POINT(41.61097044910515 -86.67421664880148) bank149950 +149951 POINT(41.13593633934641 -88.03734526647095) bank149951 +149952 POINT(41.09876078658136 -87.85238892748175) bank149952 +149953 POINT(42.77227714592757 -88.2977513654766) bank149953 +149954 POINT(42.656839942824625 -88.09981264838036) bank149954 +149955 POINT(42.065867322141116 -88.06639560936283) bank149955 +149956 POINT(42.03984234122224 -88.38363337185706) bank149956 +149957 POINT(41.02184480347009 -86.72184652491113) bank149957 +149958 POINT(42.668154403994016 -88.267140572901) bank149958 +149959 POINT(42.77552609547671 -87.98628542584709) bank149959 +149960 POINT(41.340066109946136 -86.64642005459267) bank149960 +149961 POINT(41.49215169250119 -88.36748875473279) bank149961 +149962 POINT(42.18349212327307 -87.68145263442625) bank149962 +149963 POINT(40.90554194079815 -88.4549968523996) bank149963 +149964 POINT(41.91079591526802 -87.32160612586745) bank149964 +149965 POINT(41.35244456910364 -88.22174304931376) bank149965 +149966 POINT(41.431466401541115 -87.53451194382939) bank149966 +149967 POINT(42.640061626184604 -87.0746991938007) bank149967 +149968 POINT(40.91607094760565 -86.84390898917748) bank149968 +149969 POINT(40.915936931124804 -88.20453596597012) bank149969 +149970 POINT(41.20093449396045 -88.45834717987223) bank149970 +149971 POINT(41.10757639377576 -88.32763082302047) bank149971 +149972 POINT(41.627781455981946 -87.40930098613069) bank149972 +149973 POINT(41.17529846523699 -88.62456184422577) bank149973 +149974 POINT(42.73541539574418 -86.66913228402794) bank149974 +149975 POINT(41.75744917360554 -88.0576870358055) bank149975 +149976 POINT(41.017413936171174 -87.62479926339986) bank149976 +149977 POINT(41.74292184156474 -87.44282446527568) bank149977 +149978 POINT(42.86472264780563 -87.41501474359266) bank149978 +149979 POINT(42.826745549435635 -88.20929845943365) bank149979 +149980 POINT(41.06214922623672 -86.63006960834214) bank149980 +149981 POINT(41.00419639026274 -88.13135498424475) bank149981 +149982 POINT(42.16245044730386 -87.96449188373302) bank149982 +149983 POINT(41.14827831053307 -88.09195299384061) bank149983 +149984 POINT(41.182712223420474 -88.13753230635126) bank149984 +149985 POINT(42.00150318834906 -86.67471254822485) bank149985 +149986 POINT(42.71529713904531 -88.5598877625708) bank149986 +149987 POINT(41.30555472667134 -86.6358911383384) bank149987 +149988 POINT(41.10558714206017 -86.65054738838963) bank149988 +149989 POINT(42.208857765337584 -88.0920539297779) bank149989 +149990 POINT(42.04536029006023 -87.97502665521759) bank149990 +149991 POINT(41.57528375697869 -87.72382033744515) bank149991 +149992 POINT(42.03006112208644 -88.43979444245869) bank149992 +149993 POINT(41.604159361420635 -87.53789095940307) bank149993 +149994 POINT(42.04596622647706 -86.7324893340841) bank149994 +149995 POINT(40.938912121725316 -86.84974796411063) bank149995 +149996 POINT(42.35331746986762 -88.14796948116337) bank149996 +149997 POINT(42.809490124293355 -88.04926236449012) bank149997 +149998 POINT(42.017014143229204 -87.18429176716674) bank149998 +149999 POINT(42.30962098964741 -88.13932690243831) bank149999 +150000 POINT(41.33876156959299 -87.71407877431886) bank150000 +150001 POINT(41.61176779748968 -88.42544901200417) bank150001 +150002 POINT(42.30285839825572 -87.52775926543647) bank150002 +150003 POINT(41.9040847023903 -87.17577934322284) bank150003 +150004 POINT(42.74286362778034 -87.95475617280526) bank150004 +150005 POINT(42.534783003949286 -87.10281162775163) bank150005 +150006 POINT(42.469827600094796 -88.2581178078699) bank150006 +150007 POINT(41.81774099171868 -87.24794653379065) bank150007 +150008 POINT(42.67913762957889 -87.69910597993005) bank150008 +150009 POINT(42.54636608391068 -87.98284613742494) bank150009 +150010 POINT(41.7469674826355 -86.73841662873399) bank150010 +150011 POINT(42.28033129771748 -88.2930326788776) bank150011 +150012 POINT(41.07385335067534 -87.50757880247119) bank150012 +150013 POINT(41.20732933455754 -87.30293225650395) bank150013 +150014 POINT(41.376981024482625 -86.98237231349995) bank150014 +150015 POINT(42.15154811561681 -86.91648284277714) bank150015 +150016 POINT(42.43171824097907 -87.93439592961431) bank150016 +150017 POINT(41.331411233534844 -88.51148257710527) bank150017 +150018 POINT(42.81118750917479 -87.80611647302918) bank150018 +150019 POINT(41.0064128835048 -88.47964437125904) bank150019 +150020 POINT(41.48842497439278 -88.19602322217023) bank150020 +150021 POINT(42.21097739420209 -87.8670846044232) bank150021 +150022 POINT(41.59307606741184 -86.92027100747947) bank150022 +150023 POINT(42.06250186397441 -88.4475727193943) bank150023 +150024 POINT(41.929186994929005 -87.08446798421022) bank150024 +150025 POINT(42.689358553856835 -87.0853863869392) bank150025 +150026 POINT(42.34881705234612 -88.12333559568457) bank150026 +150027 POINT(42.06427836464447 -88.39802221363851) bank150027 +150028 POINT(42.18567448297356 -86.85294001736597) bank150028 +150029 POINT(41.04091317960283 -86.75944720071045) bank150029 +150030 POINT(41.226881147241826 -86.65168853738955) bank150030 +150031 POINT(41.68248177046479 -86.70104200428504) bank150031 +150032 POINT(41.903521002553994 -86.79049878095395) bank150032 +150033 POINT(41.647338776249725 -88.4918940103885) bank150033 +150034 POINT(40.90795615519442 -87.87712446316635) bank150034 +150035 POINT(41.50378083916733 -87.12697914761644) bank150035 +150036 POINT(42.2227838024591 -88.33902849167636) bank150036 +150037 POINT(42.7488004455287 -86.74342807454343) bank150037 +150038 POINT(41.474532384135934 -88.57170242358549) bank150038 +150039 POINT(41.6365293551506 -87.4302501034642) bank150039 +150040 POINT(41.1358816112361 -87.62821763721631) bank150040 +150041 POINT(42.054109411645065 -86.70358563430727) bank150041 +150042 POINT(41.69447969889732 -87.85471770060809) bank150042 +150043 POINT(40.996773303677834 -87.37906508642024) bank150043 +150044 POINT(42.6589398380205 -87.70991556724906) bank150044 +150045 POINT(42.053812341693984 -87.24530773428995) bank150045 +150046 POINT(42.481945097899 -88.44897714337546) bank150046 +150047 POINT(42.51816798770012 -88.34897768434716) bank150047 +150048 POINT(41.58532891094105 -87.80414396476219) bank150048 +150049 POINT(41.22432984320996 -86.98007033683604) bank150049 +150050 POINT(42.83927620629537 -86.88531664081245) bank150050 +150051 POINT(42.46988645069685 -87.0271409067506) bank150051 +150052 POINT(42.409939640531995 -87.79653783709229) bank150052 +150053 POINT(42.46972618714165 -88.5454976743002) bank150053 +150054 POINT(42.55557565288371 -87.44922495634486) bank150054 +150055 POINT(41.03740839949598 -88.44158220878141) bank150055 +150056 POINT(41.70514202472894 -88.36673238273517) bank150056 +150057 POINT(42.71271438627563 -87.2900130653959) bank150057 +150058 POINT(42.85728019914203 -88.09216670342848) bank150058 +150059 POINT(41.37365166638194 -88.16261192203781) bank150059 +150060 POINT(42.19195138073274 -88.59000128900242) bank150060 +150061 POINT(41.32410820407107 -86.99577429536082) bank150061 +150062 POINT(41.1596807357092 -87.26075777699647) bank150062 +150063 POINT(41.49281942572536 -87.26084852594369) bank150063 +150064 POINT(42.53830483405127 -87.99944164176536) bank150064 +150065 POINT(41.693916827438706 -88.61955953240798) bank150065 +150066 POINT(42.59169432750918 -88.15597866683481) bank150066 +150067 POINT(42.62924714422394 -87.27664886400383) bank150067 +150068 POINT(41.08596008567245 -87.13668985426516) bank150068 +150069 POINT(41.853282500068694 -88.46383107744995) bank150069 +150070 POINT(41.989993057477996 -88.61691484035181) bank150070 +150071 POINT(41.10426101785046 -86.87489256337116) bank150071 +150072 POINT(41.75343127348812 -87.67651946282261) bank150072 +150073 POINT(41.01529983053585 -87.44729761232145) bank150073 +150074 POINT(41.02585684414527 -87.00993898459839) bank150074 +150075 POINT(41.59292121432118 -88.44107131870001) bank150075 +150076 POINT(41.92024245785938 -88.5639121060601) bank150076 +150077 POINT(42.023809252665934 -86.64508575329171) bank150077 +150078 POINT(42.36534996354409 -88.56299725596773) bank150078 +150079 POINT(42.29551578739649 -88.61775069013105) bank150079 +150080 POINT(41.14549031675231 -88.45644766361224) bank150080 +150081 POINT(41.00106180264111 -87.03244759074002) bank150081 +150082 POINT(41.76157564701334 -87.71816993349049) bank150082 +150083 POINT(41.18393540258289 -88.09914807028917) bank150083 +150084 POINT(41.08367674437166 -87.49009146542741) bank150084 +150085 POINT(41.107364859035044 -87.15977451222902) bank150085 +150086 POINT(41.21018376464304 -87.65384487498758) bank150086 +150087 POINT(41.64820916945806 -87.28621831857384) bank150087 +150088 POINT(42.14959839898485 -87.47121595331748) bank150088 +150089 POINT(41.568021766544426 -87.4049619450772) bank150089 +150090 POINT(41.79260355293195 -87.72106524773757) bank150090 +150091 POINT(41.32153725496116 -88.36238549803836) bank150091 +150092 POINT(40.955132674642286 -86.90922734477493) bank150092 +150093 POINT(41.85491766942227 -88.51304111027085) bank150093 +150094 POINT(41.193156719702245 -88.16542481378015) bank150094 +150095 POINT(41.4407713187034 -86.97396054529288) bank150095 +150096 POINT(42.22830145437967 -86.76076789441453) bank150096 +150097 POINT(42.09204645014468 -87.93417450431922) bank150097 +150098 POINT(42.372874490724485 -87.08294451838077) bank150098 +150099 POINT(41.582974123686526 -88.24444637958786) bank150099 +150100 POINT(41.61622128297664 -87.43920243758839) bank150100 +150101 POINT(42.601695891640496 -88.4232079155104) bank150101 +150102 POINT(41.46968528120244 -87.79342076320596) bank150102 +150103 POINT(42.59201607175206 -86.72693296815555) bank150103 +150104 POINT(42.71662368518777 -88.22971826704003) bank150104 +150105 POINT(41.08838702817469 -87.2973206446285) bank150105 +150106 POINT(41.55967103862043 -86.79602661508103) bank150106 +150107 POINT(42.15101588281812 -87.35082649537564) bank150107 +150108 POINT(40.90095144117061 -87.62475184623851) bank150108 +150109 POINT(42.16279238704398 -86.65940541798338) bank150109 +150110 POINT(42.48220785912087 -87.25478823486317) bank150110 +150111 POINT(41.028335048094824 -87.92725166823953) bank150111 +150112 POINT(42.114821476942275 -86.87810851398798) bank150112 +150113 POINT(42.67997817137517 -86.96014763004423) bank150113 +150114 POINT(42.54753567473183 -87.10524892021321) bank150114 +150115 POINT(41.779976588647656 -88.5921058232524) bank150115 +150116 POINT(41.55374489374191 -87.67929204495974) bank150116 +150117 POINT(41.38101787128952 -87.65098325960984) bank150117 +150118 POINT(42.53992703008447 -87.38976655569017) bank150118 +150119 POINT(42.59717204036619 -87.19792250761543) bank150119 +150120 POINT(42.24201222564189 -87.7247679998163) bank150120 +150121 POINT(42.26400033956548 -88.39917996772263) bank150121 +150122 POINT(40.98540291098639 -87.6957300702554) bank150122 +150123 POINT(42.17021154498536 -88.05309679746414) bank150123 +150124 POINT(41.337472240616606 -87.58527026248349) bank150124 +150125 POINT(41.392329423108364 -88.2534370167245) bank150125 +150126 POINT(41.70048370012876 -88.6172747645658) bank150126 +150127 POINT(41.96810382781346 -88.45907760704914) bank150127 +150128 POINT(42.528552463472515 -86.80115683483331) bank150128 +150129 POINT(41.734792773297045 -87.7233830012868) bank150129 +150130 POINT(40.9521244157468 -86.89256896373281) bank150130 +150131 POINT(42.00032605304318 -88.1107577011919) bank150131 +150132 POINT(42.45472172824071 -87.79907084026864) bank150132 +150133 POINT(42.20748050077453 -86.99162565458144) bank150133 +150134 POINT(41.051513747688865 -87.1709944006746) bank150134 +150135 POINT(41.535612499305486 -87.02419130220716) bank150135 +150136 POINT(42.540677195342234 -86.85288788723345) bank150136 +150137 POINT(42.85635640998876 -88.234274057853) bank150137 +150138 POINT(42.48030761998745 -87.07183964609064) bank150138 +150139 POINT(41.41535908882696 -87.54303058654165) bank150139 +150140 POINT(41.90564212200343 -87.79578548854681) bank150140 +150141 POINT(40.97887710389327 -88.59521267598492) bank150141 +150142 POINT(42.25818181372277 -87.80525860498997) bank150142 +150143 POINT(42.002602225717325 -87.54762176289047) bank150143 +150144 POINT(41.36460139758617 -87.41266819088109) bank150144 +150145 POINT(42.653938677435896 -87.3968956376226) bank150145 +150146 POINT(41.93230379871827 -87.93558046488167) bank150146 +150147 POINT(42.42069152035363 -87.1959226569753) bank150147 +150148 POINT(41.05769661789777 -87.97340443046508) bank150148 +150149 POINT(41.640709329857124 -88.36615302029156) bank150149 +150150 POINT(41.28257535255177 -87.6609675529848) bank150150 +150151 POINT(42.753045147495754 -87.21564235128153) bank150151 +150152 POINT(42.10253310225228 -88.33401333780137) bank150152 +150153 POINT(42.57860303817648 -86.6576999085919) bank150153 +150154 POINT(42.47973479110539 -88.35007753832043) bank150154 +150155 POINT(42.73130606409457 -86.9316248086552) bank150155 +150156 POINT(40.88859825583023 -86.77649459935851) bank150156 +150157 POINT(42.71135302582041 -86.75471643843666) bank150157 +150158 POINT(42.647082321047 -87.28492795753944) bank150158 +150159 POINT(41.8984625770515 -86.75322317322939) bank150159 +150160 POINT(42.28023120630859 -88.46251313363435) bank150160 +150161 POINT(41.48883903917368 -88.18878074833148) bank150161 +150162 POINT(41.257818386626184 -87.5554225681024) bank150162 +150163 POINT(41.6733549709091 -87.34320938126962) bank150163 +150164 POINT(42.32064234670161 -87.98830520091106) bank150164 +150165 POINT(42.081626153953124 -86.97586009760181) bank150165 +150166 POINT(41.79899691752651 -87.80628247393808) bank150166 +150167 POINT(41.30878868934585 -88.53598497216883) bank150167 +150168 POINT(41.609627146069144 -87.57943879373487) bank150168 +150169 POINT(41.575709564259746 -88.0628248100486) bank150169 +150170 POINT(42.12819416848732 -87.25023901371098) bank150170 +150171 POINT(40.92433392272772 -86.94885171011325) bank150171 +150172 POINT(42.495908150075756 -88.23878319798538) bank150172 +150173 POINT(40.901373200443544 -88.5773999685349) bank150173 +150174 POINT(42.61542273233581 -87.01415092951748) bank150174 +150175 POINT(42.388586715147284 -88.58433308000187) bank150175 +150176 POINT(41.775951897366056 -88.34922003171812) bank150176 +150177 POINT(42.73424382546406 -88.48839468313187) bank150177 +150178 POINT(42.35986299443661 -87.5094741635455) bank150178 +150179 POINT(42.84481291069831 -86.74662661240508) bank150179 +150180 POINT(42.725256857495005 -87.68430932646885) bank150180 +150181 POINT(41.573573456881036 -87.8822057160608) bank150181 +150182 POINT(41.280732594468475 -86.84841179369754) bank150182 +150183 POINT(41.99707612877274 -88.49181216698993) bank150183 +150184 POINT(41.47426836650213 -87.8330721983677) bank150184 +150185 POINT(41.54545287752692 -86.97721273155628) bank150185 +150186 POINT(42.69690267272046 -86.98244092427443) bank150186 +150187 POINT(41.368388170860136 -86.65518188480061) bank150187 +150188 POINT(42.07021063339257 -86.892666136415) bank150188 +150189 POINT(42.37396139082218 -88.02136290156469) bank150189 +150190 POINT(41.962346810262886 -86.78744643230291) bank150190 +150191 POINT(41.59456731352887 -86.71632520213109) bank150191 +150192 POINT(42.30815744085741 -88.49122284898223) bank150192 +150193 POINT(42.139728472522854 -87.00674208846401) bank150193 +150194 POINT(41.069068758061974 -88.25214970076614) bank150194 +150195 POINT(41.22877219120743 -88.5198856748375) bank150195 +150196 POINT(41.538822962567195 -87.69825611063051) bank150196 +150197 POINT(42.05752934109084 -86.96866841751147) bank150197 +150198 POINT(42.581072814546644 -88.26728969117133) bank150198 +150199 POINT(41.124696808354415 -86.71722176271072) bank150199 +150200 POINT(41.309945511633714 -86.85855760250634) bank150200 +150201 POINT(41.01347200713322 -86.63065080645067) bank150201 +150202 POINT(41.116045696583946 -87.58375340616765) bank150202 +150203 POINT(41.42726537453438 -87.51635154253673) bank150203 +150204 POINT(41.776578745261205 -88.28867688196227) bank150204 +150205 POINT(41.42375244675282 -87.67855429457339) bank150205 +150206 POINT(42.23709090421856 -88.33746448024915) bank150206 +150207 POINT(41.482320569168294 -87.39314986459274) bank150207 +150208 POINT(40.89749557791513 -88.24547681211277) bank150208 +150209 POINT(41.97880986836538 -87.50882152872283) bank150209 +150210 POINT(41.09101251602942 -87.68618164713708) bank150210 +150211 POINT(41.777387640853945 -87.68926280705412) bank150211 +150212 POINT(41.538470712422836 -87.5602542253191) bank150212 +150213 POINT(41.47340014354857 -87.21179423041801) bank150213 +150214 POINT(41.100300122227736 -86.82001154851172) bank150214 +150215 POINT(42.76349410153721 -86.701415584757) bank150215 +150216 POINT(41.85234363141439 -88.42683030687911) bank150216 +150217 POINT(41.10631670824807 -87.4351865192168) bank150217 +150218 POINT(42.67885360312379 -88.00693112125435) bank150218 +150219 POINT(41.02060623333239 -87.9685099678632) bank150219 +150220 POINT(42.16220419190812 -87.29099581309298) bank150220 +150221 POINT(41.18809958713736 -88.45356482433685) bank150221 +150222 POINT(42.00874868091405 -87.56109030132896) bank150222 +150223 POINT(40.8919846735621 -88.11387217128569) bank150223 +150224 POINT(41.759441161253385 -88.29172331841237) bank150224 +150225 POINT(42.375418822667314 -87.14105411589952) bank150225 +150226 POINT(42.02380341816571 -87.59419781345338) bank150226 +150227 POINT(41.63500470921833 -87.49991367155475) bank150227 +150228 POINT(42.31034265536566 -87.63627648544629) bank150228 +150229 POINT(42.65407903747808 -86.70557554908547) bank150229 +150230 POINT(42.87325223282539 -87.95562492443197) bank150230 +150231 POINT(42.13805209876456 -87.89060489164483) bank150231 +150232 POINT(42.62907814345412 -87.61796326773904) bank150232 +150233 POINT(42.81441058021252 -87.88815996558077) bank150233 +150234 POINT(42.62572133460585 -88.56594515488108) bank150234 +150235 POINT(41.5788734671039 -87.6128052816765) bank150235 +150236 POINT(41.90494654601724 -87.48837486225169) bank150236 +150237 POINT(42.80608938574542 -88.31805317137513) bank150237 +150238 POINT(42.55390685013427 -87.46785404920249) bank150238 +150239 POINT(42.49874958818343 -87.0066039012902) bank150239 +150240 POINT(42.48272092112734 -86.8346473229642) bank150240 +150241 POINT(42.41858880118055 -88.0342791317115) bank150241 +150242 POINT(42.84714539721539 -87.22910892013844) bank150242 +150243 POINT(41.380908557811836 -88.52533461138201) bank150243 +150244 POINT(42.47011999363639 -88.26183627079772) bank150244 +150245 POINT(40.993780247306645 -87.7307490856601) bank150245 +150246 POINT(42.86408599557095 -86.90174286029628) bank150246 +150247 POINT(42.2288711172476 -87.59250261232393) bank150247 +150248 POINT(41.88942823964291 -88.38124469512786) bank150248 +150249 POINT(42.81464679537114 -87.57233193436119) bank150249 +150250 POINT(42.385245188585685 -88.59645610214405) bank150250 +150251 POINT(41.09985274630316 -87.944199793149) bank150251 +150252 POINT(41.135591944680975 -88.02007158175162) bank150252 +150253 POINT(42.726173824485386 -88.51867005572728) bank150253 +150254 POINT(41.739767899296226 -86.80130576987722) bank150254 +150255 POINT(42.7533093696739 -87.31601708054608) bank150255 +150256 POINT(41.514678770196916 -87.89820944310887) bank150256 +150257 POINT(42.24542533065056 -87.65697986994522) bank150257 +150258 POINT(42.30354386910993 -86.69283033321206) bank150258 +150259 POINT(42.52709507426821 -87.48823149891581) bank150259 +150260 POINT(42.480690854613336 -87.63654171559644) bank150260 +150261 POINT(42.286885042338525 -87.266311172089) bank150261 +150262 POINT(42.846903649123604 -87.17209306959018) bank150262 +150263 POINT(42.84883800013856 -87.66129607560175) bank150263 +150264 POINT(42.78478516132832 -87.03571627877265) bank150264 +150265 POINT(42.71935407778782 -86.77585703600019) bank150265 +150266 POINT(41.311449461431614 -86.70816303776164) bank150266 +150267 POINT(40.964052891244734 -86.76337622587558) bank150267 +150268 POINT(42.437002330619364 -88.39191557089231) bank150268 +150269 POINT(42.667703027392896 -88.25783142652051) bank150269 +150270 POINT(42.66804523311675 -87.48705751546652) bank150270 +150271 POINT(42.256444462363774 -88.43638492049367) bank150271 +150272 POINT(42.7488904613356 -88.55328950828472) bank150272 +150273 POINT(42.74876533869663 -87.55605291119095) bank150273 +150274 POINT(41.81561358199347 -87.00870269144522) bank150274 +150275 POINT(41.26038244341311 -88.2168029417581) bank150275 +150276 POINT(41.59095813741495 -88.13464506088332) bank150276 +150277 POINT(41.35980920379351 -86.92943591753458) bank150277 +150278 POINT(41.686514032319955 -88.27733863284496) bank150278 +150279 POINT(42.67097360899304 -88.0377005844069) bank150279 +150280 POINT(41.251017700658664 -86.89965886734116) bank150280 +150281 POINT(41.891192297280305 -86.95783612499793) bank150281 +150282 POINT(42.23497975246504 -87.31929284516907) bank150282 +150283 POINT(42.12979377214677 -87.93887926515815) bank150283 +150284 POINT(41.90706010810705 -86.76530688801269) bank150284 +150285 POINT(41.36717779343887 -87.75222084299331) bank150285 +150286 POINT(41.48599807811452 -87.65547458894629) bank150286 +150287 POINT(42.44070189646462 -88.43935549833421) bank150287 +150288 POINT(42.6609891563847 -88.41907298110557) bank150288 +150289 POINT(40.98755033922144 -88.61845638169609) bank150289 +150290 POINT(42.77085954480557 -86.93913006715967) bank150290 +150291 POINT(42.44155674089357 -87.48043664365616) bank150291 +150292 POINT(40.98340820082103 -88.58570423294181) bank150292 +150293 POINT(41.42887831661166 -87.02634458764447) bank150293 +150294 POINT(42.671061470540295 -88.47601114880875) bank150294 +150295 POINT(42.57000265255341 -87.64436256411933) bank150295 +150296 POINT(41.23759766611171 -88.04105183408481) bank150296 +150297 POINT(41.29593386546307 -87.27750992118915) bank150297 +150298 POINT(42.45443948884235 -87.6087711664967) bank150298 +150299 POINT(41.96317185959455 -86.9068764119753) bank150299 +150300 POINT(41.783534558752876 -87.10922655699078) bank150300 +150301 POINT(42.09515292037329 -87.12459269054217) bank150301 +150302 POINT(42.8211772319514 -87.13862223360105) bank150302 +150303 POINT(42.43011477203351 -88.20705722768815) bank150303 +150304 POINT(41.64509092997287 -87.59224140144958) bank150304 +150305 POINT(42.410540910974916 -88.34327775439755) bank150305 +150306 POINT(41.21974344378527 -88.12912482269242) bank150306 +150307 POINT(41.35718310229914 -87.32516944278422) bank150307 +150308 POINT(41.58668454500446 -86.80386341256633) bank150308 +150309 POINT(42.49902736055685 -88.07454132271506) bank150309 +150310 POINT(42.344552948167575 -87.83740712691551) bank150310 +150311 POINT(41.926129463490085 -87.58310289648732) bank150311 +150312 POINT(41.846574013225464 -87.73895789951547) bank150312 +150313 POINT(41.16839611855587 -86.76337247411522) bank150313 +150314 POINT(41.65409396741184 -87.40113977275408) bank150314 +150315 POINT(42.494015978200494 -88.49727274647597) bank150315 +150316 POINT(41.752986507645204 -87.82877857522512) bank150316 +150317 POINT(41.71812855180867 -88.57703073412787) bank150317 +150318 POINT(40.99188342687387 -87.5081238629835) bank150318 +150319 POINT(41.47726902381947 -88.21316592699256) bank150319 +150320 POINT(42.08724791834079 -87.45378974612301) bank150320 +150321 POINT(41.388725847440604 -88.49960881265636) bank150321 +150322 POINT(41.21073363382732 -86.99666606446978) bank150322 +150323 POINT(42.84520049676499 -86.90960032876723) bank150323 +150324 POINT(41.586230292681 -87.97936090106153) bank150324 +150325 POINT(42.55532353481562 -88.48831020031972) bank150325 +150326 POINT(42.45953709492857 -86.90578832126684) bank150326 +150327 POINT(41.78053301803626 -87.87663521688985) bank150327 +150328 POINT(42.53965903902421 -86.77438197084972) bank150328 +150329 POINT(41.472098076867226 -88.09685589786467) bank150329 +150330 POINT(41.23765488930129 -88.62561770030082) bank150330 +150331 POINT(42.86542991671396 -87.32153364613416) bank150331 +150332 POINT(41.042873878151504 -87.22017336945207) bank150332 +150333 POINT(41.472057653604615 -88.48438423549221) bank150333 +150334 POINT(41.28382118190573 -86.83764898798275) bank150334 +150335 POINT(42.29459269446345 -87.80027962901359) bank150335 +150336 POINT(41.15305830482831 -88.0804164883975) bank150336 +150337 POINT(42.802096089775745 -87.75551423894953) bank150337 +150338 POINT(42.62244956720176 -86.94091124379548) bank150338 +150339 POINT(41.09020915418532 -87.62114518845299) bank150339 +150340 POINT(41.85605721475643 -88.31087224564143) bank150340 +150341 POINT(41.79599444643732 -87.56576339639213) bank150341 +150342 POINT(42.37078570273838 -87.16014478362283) bank150342 +150343 POINT(42.05843814395958 -88.6084244620407) bank150343 +150344 POINT(42.588066716831946 -87.80720158586209) bank150344 +150345 POINT(42.4659132045505 -87.20986246897814) bank150345 +150346 POINT(41.686236965717505 -87.75113187653994) bank150346 +150347 POINT(41.71535305785067 -87.90908749390496) bank150347 +150348 POINT(40.98509973830117 -86.85154547816641) bank150348 +150349 POINT(42.55169980952882 -87.25356286596464) bank150349 +150350 POINT(41.29588321025589 -87.1657709951116) bank150350 +150351 POINT(41.9302412035425 -88.34995841816823) bank150351 +150352 POINT(40.924537438379815 -86.89460178150891) bank150352 +150353 POINT(41.994308134161116 -87.27173036594395) bank150353 +150354 POINT(42.83158865482588 -87.55091863409551) bank150354 +150355 POINT(42.71547610944506 -88.27578428282527) bank150355 +150356 POINT(42.36392378314377 -87.58511406466823) bank150356 +150357 POINT(42.75217482756311 -86.73669800472483) bank150357 +150358 POINT(42.228467493957595 -87.43310510596315) bank150358 +150359 POINT(42.427958969436915 -88.00140845016713) bank150359 +150360 POINT(42.33717525164459 -88.0738178263074) bank150360 +150361 POINT(41.98137738622046 -87.32573462678506) bank150361 +150362 POINT(42.22948794939991 -86.83252888202672) bank150362 +150363 POINT(41.85773132663966 -87.18162170095096) bank150363 +150364 POINT(42.085349717728256 -88.54455745548543) bank150364 +150365 POINT(41.04051347635696 -86.7382902838237) bank150365 +150366 POINT(42.61453329011644 -87.31196983797088) bank150366 +150367 POINT(42.29449710542735 -87.71459152140159) bank150367 +150368 POINT(42.1837906644278 -87.16938926524995) bank150368 +150369 POINT(41.75183877421415 -87.95174961137792) bank150369 +150370 POINT(42.03595053907498 -88.32592134321999) bank150370 +150371 POINT(41.492263037588565 -87.32868663402813) bank150371 +150372 POINT(42.60541124200745 -87.80393140594705) bank150372 +150373 POINT(40.93761519015062 -87.41733285413902) bank150373 +150374 POINT(42.03189499185671 -88.54550546894735) bank150374 +150375 POINT(42.15973837876564 -86.91684940976876) bank150375 +150376 POINT(41.58184735518089 -87.3965962301254) bank150376 +150377 POINT(41.9442821032515 -87.00514173825418) bank150377 +150378 POINT(41.805003992976054 -86.71150142322718) bank150378 +150379 POINT(42.71869223439643 -86.6891469263169) bank150379 +150380 POINT(42.57030419752618 -87.273406160876) bank150380 +150381 POINT(41.60119977138001 -87.2989600832306) bank150381 +150382 POINT(40.938691267016104 -87.32806457085249) bank150382 +150383 POINT(42.66775487278201 -88.14062649086569) bank150383 +150384 POINT(41.31071014479151 -87.98874650904196) bank150384 +150385 POINT(42.81582539435152 -88.51165087748578) bank150385 +150386 POINT(42.312260654644184 -87.69806733236268) bank150386 +150387 POINT(41.581209145409396 -88.35482897570705) bank150387 +150388 POINT(42.324521723809234 -88.59429718614948) bank150388 +150389 POINT(41.21464032211446 -87.49899131711291) bank150389 +150390 POINT(41.930541051279754 -88.47427256155937) bank150390 +150391 POINT(42.24337171542745 -87.8266156973656) bank150391 +150392 POINT(42.34583733274801 -87.70140614828398) bank150392 +150393 POINT(41.47914891161253 -88.44259201599284) bank150393 +150394 POINT(41.450545565477924 -86.99155799397053) bank150394 +150395 POINT(41.64692191406391 -87.37105831187517) bank150395 +150396 POINT(41.29679126240499 -87.6481996462643) bank150396 +150397 POINT(42.53909099158936 -86.80873682520877) bank150397 +150398 POINT(40.96357537349417 -88.51422261071465) bank150398 +150399 POINT(42.73700046278382 -88.453971490011) bank150399 +150400 POINT(42.765940883749245 -87.92849190581602) bank150400 +150401 POINT(41.442695252276906 -88.2608564221502) bank150401 +150402 POINT(41.12545393551959 -88.42902387230752) bank150402 +150403 POINT(41.11198927289129 -87.93093209124496) bank150403 +150404 POINT(41.21249147976743 -87.21405874667856) bank150404 +150405 POINT(41.501170681806734 -87.69847924547297) bank150405 +150406 POINT(41.0642673645482 -86.99942506254112) bank150406 +150407 POINT(41.0928833988439 -86.81281154682151) bank150407 +150408 POINT(42.79170143379728 -87.10296854892762) bank150408 +150409 POINT(41.28712221435067 -87.52544467121903) bank150409 +150410 POINT(41.71745887868406 -87.41556253830683) bank150410 +150411 POINT(41.03485789607559 -87.35652212162141) bank150411 +150412 POINT(41.60580192012543 -88.50732576773451) bank150412 +150413 POINT(42.55745034820527 -87.2582044679837) bank150413 +150414 POINT(42.69187185948064 -87.40763832835016) bank150414 +150415 POINT(42.61576255739949 -88.5144949305538) bank150415 +150416 POINT(41.12072112214708 -87.74363268671455) bank150416 +150417 POINT(42.607368847861906 -87.09915021795103) bank150417 +150418 POINT(42.86446551086807 -87.78170552239423) bank150418 +150419 POINT(41.5685377423092 -88.44293273747924) bank150419 +150420 POINT(41.603794965961946 -87.69261464773909) bank150420 +150421 POINT(42.09834789562634 -87.69580550331513) bank150421 +150422 POINT(42.27052081235832 -88.35534458110078) bank150422 +150423 POINT(41.369780689029234 -87.12853098788635) bank150423 +150424 POINT(41.2425987963214 -87.14506800236855) bank150424 +150425 POINT(41.851931900004764 -88.35416562514979) bank150425 +150426 POINT(41.13307611394823 -88.62040027316479) bank150426 +150427 POINT(41.07752366358777 -87.05034692864072) bank150427 +150428 POINT(41.167294890359464 -87.97739158750377) bank150428 +150429 POINT(41.334918570390194 -88.25895087645534) bank150429 +150430 POINT(42.01785970015695 -86.98835536008211) bank150430 +150431 POINT(41.322500607488166 -88.21883251788633) bank150431 +150432 POINT(42.03324713219612 -87.42650115089741) bank150432 +150433 POINT(42.55744495583317 -88.34076067734634) bank150433 +150434 POINT(42.276168768263325 -87.00066527535225) bank150434 +150435 POINT(41.942148163189984 -87.6785055577225) bank150435 +150436 POINT(40.89023682439789 -88.13729922331254) bank150436 +150437 POINT(41.93599002532948 -87.93976084698724) bank150437 +150438 POINT(42.462755390731544 -86.66403307203697) bank150438 +150439 POINT(41.95175407075781 -87.73825975281332) bank150439 +150440 POINT(42.04928971430724 -87.20937380056382) bank150440 +150441 POINT(42.66124379543872 -87.29194476097226) bank150441 +150442 POINT(41.908738108311866 -87.65996803508742) bank150442 +150443 POINT(41.864821325282186 -86.97482614232045) bank150443 +150444 POINT(41.78387056096772 -87.11688007721878) bank150444 +150445 POINT(41.97521367322692 -87.03290120232984) bank150445 +150446 POINT(42.09621342074582 -88.13727399086861) bank150446 +150447 POINT(42.45039572099201 -87.2978053246472) bank150447 +150448 POINT(41.88073540546886 -88.01023983707692) bank150448 +150449 POINT(41.471821332880445 -87.66782147915023) bank150449 +150450 POINT(42.21029243001344 -87.9394207048298) bank150450 +150451 POINT(42.293028734239094 -87.77510832137428) bank150451 +150452 POINT(41.546283944769705 -87.56690131232097) bank150452 +150453 POINT(41.64238767353605 -87.64350056355268) bank150453 +150454 POINT(41.67178222672563 -88.61235332161623) bank150454 +150455 POINT(41.80692854802016 -87.52324916886496) bank150455 +150456 POINT(42.35931742996306 -87.69606743226514) bank150456 +150457 POINT(41.69974179556729 -87.43802803170612) bank150457 +150458 POINT(41.84703866798135 -87.19707568537775) bank150458 +150459 POINT(42.32573212243717 -86.67727227787819) bank150459 +150460 POINT(41.51144941290563 -86.7300153888784) bank150460 +150461 POINT(42.70399690338546 -88.51142010495268) bank150461 +150462 POINT(42.16739538540473 -88.56470146820598) bank150462 +150463 POINT(41.691039743419836 -87.41156009616546) bank150463 +150464 POINT(41.60556192278666 -87.00236364241698) bank150464 +150465 POINT(42.10534290603327 -87.64828178243853) bank150465 +150466 POINT(40.94956411494841 -87.15795109436081) bank150466 +150467 POINT(42.397869021615094 -87.7279103235197) bank150467 +150468 POINT(41.00858281281017 -86.86466470489839) bank150468 +150469 POINT(42.440629596647476 -88.55363313363809) bank150469 +150470 POINT(42.2574609295144 -87.44929972358243) bank150470 +150471 POINT(41.70387365370051 -88.47854115996) bank150471 +150472 POINT(42.19756377390597 -86.64871684588167) bank150472 +150473 POINT(41.283914419956844 -86.90548723460577) bank150473 +150474 POINT(41.868912526697024 -86.80522856921534) bank150474 +150475 POINT(41.79461808686544 -88.02214292049824) bank150475 +150476 POINT(41.44239059845177 -87.33021227597759) bank150476 +150477 POINT(42.103317028018886 -87.52638188529106) bank150477 +150478 POINT(42.663694915505125 -87.92850441872388) bank150478 +150479 POINT(41.539167929713 -87.15101680501482) bank150479 +150480 POINT(40.886677569544375 -88.14297737915452) bank150480 +150481 POINT(41.27208181089073 -88.2722943817152) bank150481 +150482 POINT(41.67229312965901 -87.7348543072708) bank150482 +150483 POINT(41.12909391619234 -87.19021299272751) bank150483 +150484 POINT(41.615865126711334 -87.21265062092101) bank150484 +150485 POINT(41.436162390000995 -87.84420566119033) bank150485 +150486 POINT(41.36236020708667 -87.38169994073732) bank150486 +150487 POINT(42.616013616275005 -87.81323241666034) bank150487 +150488 POINT(42.609294928493135 -86.76819096543404) bank150488 +150489 POINT(41.428766274801326 -87.60747400298082) bank150489 +150490 POINT(41.08954390622978 -86.74907412561859) bank150490 +150491 POINT(41.20234959395533 -87.0279444650169) bank150491 +150492 POINT(41.78024875811753 -86.90717851744769) bank150492 +150493 POINT(42.46912707804521 -86.67734065277193) bank150493 +150494 POINT(41.63580820015296 -88.51351602577695) bank150494 +150495 POINT(42.27492161792326 -87.67096273500259) bank150495 +150496 POINT(42.66168014362512 -86.86527316330495) bank150496 +150497 POINT(41.26309770549975 -87.2371297000187) bank150497 +150498 POINT(42.35855495009525 -87.31288021634681) bank150498 +150499 POINT(42.51950007406858 -87.49452273056211) bank150499 +150500 POINT(41.57716105593695 -87.06114915373345) bank150500 +150501 POINT(41.10972086692927 -88.22169581635836) bank150501 +150502 POINT(41.78629921561893 -88.31464403927983) bank150502 +150503 POINT(41.52366240823703 -88.35890238746198) bank150503 +150504 POINT(42.614139971562636 -87.14154534773363) bank150504 +150505 POINT(42.57032367015678 -87.65842485534475) bank150505 +150506 POINT(42.86410858622783 -86.68783216122915) bank150506 +150507 POINT(42.38378831298728 -87.19854156092485) bank150507 +150508 POINT(41.54421252994384 -87.990986201454) bank150508 +150509 POINT(41.86570755203361 -88.23441616587985) bank150509 +150510 POINT(41.88515845900583 -87.6385365241449) bank150510 +150511 POINT(42.2790249182856 -87.46617657906906) bank150511 +150512 POINT(41.28366905645964 -88.12056433108992) bank150512 +150513 POINT(42.41518390319597 -88.26780344804759) bank150513 +150514 POINT(41.94961490539179 -87.69013401245414) bank150514 +150515 POINT(42.56376764214669 -86.81561559511191) bank150515 +150516 POINT(42.086254971359864 -87.95275536553157) bank150516 +150517 POINT(42.51674648247838 -88.23124033013927) bank150517 +150518 POINT(41.84678762405033 -87.83511154837181) bank150518 +150519 POINT(40.91970028508972 -87.43056351598379) bank150519 +150520 POINT(41.850478341113856 -87.48910720916099) bank150520 +150521 POINT(41.792221115534936 -87.08901947810864) bank150521 +150522 POINT(41.66105471137885 -87.57977948254461) bank150522 +150523 POINT(42.53109077274306 -87.79301517623117) bank150523 +150524 POINT(41.76242907623654 -87.69115338902806) bank150524 +150525 POINT(41.43898963727886 -86.93695404856537) bank150525 +150526 POINT(41.85259338075486 -87.04903316764883) bank150526 +150527 POINT(41.85777094393657 -87.2016180087908) bank150527 +150528 POINT(41.13963918250517 -88.36225786602475) bank150528 +150529 POINT(42.76445966054374 -87.54582826747996) bank150529 +150530 POINT(41.173027439365924 -88.00072083404044) bank150530 +150531 POINT(42.38541904460575 -87.75815270888116) bank150531 +150532 POINT(41.984509809248266 -88.44969250566007) bank150532 +150533 POINT(41.530768064762874 -88.32318985252418) bank150533 +150534 POINT(41.42450699698232 -88.6189569228957) bank150534 +150535 POINT(40.918584239697864 -87.1293377131589) bank150535 +150536 POINT(42.623391238797865 -87.53812919093168) bank150536 +150537 POINT(41.86295253813231 -86.83512096227368) bank150537 +150538 POINT(41.178495975508255 -87.29693884387157) bank150538 +150539 POINT(42.82410869427165 -86.79932489108376) bank150539 +150540 POINT(41.425368436228815 -87.5922603785168) bank150540 +150541 POINT(41.91935050637475 -88.05837786872699) bank150541 +150542 POINT(41.30603523574399 -87.1401937418248) bank150542 +150543 POINT(42.63415726984155 -87.1576633081548) bank150543 +150544 POINT(42.29670052565129 -87.08252290527982) bank150544 +150545 POINT(41.844937888184454 -86.99028629077453) bank150545 +150546 POINT(41.72914646526003 -86.8431131590936) bank150546 +150547 POINT(42.850813926503136 -87.7246670845703) bank150547 +150548 POINT(42.090760138565095 -88.13445368305499) bank150548 +150549 POINT(41.52365601913626 -88.16446581198615) bank150549 +150550 POINT(41.10318857829946 -87.81919171062364) bank150550 +150551 POINT(40.929256489891976 -88.56997755733279) bank150551 +150552 POINT(41.88102874161735 -87.96091089827402) bank150552 +150553 POINT(42.823879588871804 -87.44470460699262) bank150553 +150554 POINT(41.810000805838435 -87.82718152571596) bank150554 +150555 POINT(41.06909252068698 -88.57114234448314) bank150555 +150556 POINT(41.396134690471776 -87.7005921883878) bank150556 +150557 POINT(42.84756711806655 -87.1906816095002) bank150557 +150558 POINT(41.77779115270639 -88.59756251594683) bank150558 +150559 POINT(41.79857922352149 -88.26840738335734) bank150559 +150560 POINT(41.12756477007442 -88.15175001830207) bank150560 +150561 POINT(41.234419879477585 -87.03391125418408) bank150561 +150562 POINT(42.2210758818825 -86.90520070142483) bank150562 +150563 POINT(41.495673018848045 -88.28997969823143) bank150563 +150564 POINT(42.51013682305587 -87.94786511364529) bank150564 +150565 POINT(42.850079783787976 -86.8813903776064) bank150565 +150566 POINT(41.00545583204491 -88.44697627419092) bank150566 +150567 POINT(41.561304925222494 -88.40706861230024) bank150567 +150568 POINT(41.09415195741639 -88.46870784639583) bank150568 +150569 POINT(42.39306815429904 -88.25943867078279) bank150569 +150570 POINT(40.963144213807986 -87.4049091049217) bank150570 +150571 POINT(42.22100676444835 -86.9990270110955) bank150571 +150572 POINT(41.21419501386717 -87.67490524911797) bank150572 +150573 POINT(41.11430462798572 -88.0433460775566) bank150573 +150574 POINT(42.11052678812638 -87.83143423398312) bank150574 +150575 POINT(41.20786502071832 -87.73748509580803) bank150575 +150576 POINT(42.50735532944 -87.65658258774283) bank150576 +150577 POINT(42.04647386656665 -87.36555049900629) bank150577 +150578 POINT(41.056421521641695 -87.20096275910397) bank150578 +150579 POINT(41.434161550192776 -87.32517250520696) bank150579 +150580 POINT(42.480918583466035 -87.35545027445272) bank150580 +150581 POINT(41.99106950377749 -86.64572507970296) bank150581 +150582 POINT(41.943132694901706 -88.05466692714954) bank150582 +150583 POINT(41.05417560949807 -87.59101253412146) bank150583 +150584 POINT(41.373316730078564 -88.30155706841094) bank150584 +150585 POINT(42.11349639620234 -88.04618815692827) bank150585 +150586 POINT(42.592497655512744 -87.71334801133742) bank150586 +150587 POINT(41.341226414250606 -87.56891229703457) bank150587 +150588 POINT(41.34066029946856 -88.52460351359652) bank150588 +150589 POINT(42.81895352563944 -86.9361316755805) bank150589 +150590 POINT(41.83489674744832 -88.3100447123785) bank150590 +150591 POINT(42.15770151281296 -87.47184268285092) bank150591 +150592 POINT(41.77335888789009 -86.66546141484847) bank150592 +150593 POINT(41.90811146004882 -86.97156766393117) bank150593 +150594 POINT(42.59321840619952 -86.98171721905076) bank150594 +150595 POINT(40.91780872361316 -87.57112770721331) bank150595 +150596 POINT(42.4407787664885 -86.82631174780335) bank150596 +150597 POINT(42.67505255250389 -86.93138851684267) bank150597 +150598 POINT(42.11466698030812 -87.85330940472154) bank150598 +150599 POINT(42.53874408071145 -86.85654069518178) bank150599 +150600 POINT(42.220383871750485 -87.60596734243617) bank150600 +150601 POINT(41.93470308031457 -86.73434124613114) bank150601 +150602 POINT(41.79580782684188 -87.00227355217088) bank150602 +150603 POINT(40.8837744595668 -87.42866320451861) bank150603 +150604 POINT(41.83628891524563 -87.27011896367563) bank150604 +150605 POINT(42.665182199776375 -87.44768395660373) bank150605 +150606 POINT(42.66262359858089 -87.7421542557004) bank150606 +150607 POINT(42.00995371858718 -87.41794219129909) bank150607 +150608 POINT(41.93727443362131 -88.56948076769595) bank150608 +150609 POINT(41.599797564552546 -86.79914845066686) bank150609 +150610 POINT(41.22314550918918 -86.70583357247287) bank150610 +150611 POINT(42.85073911008266 -87.48062194310545) bank150611 +150612 POINT(41.195412062616356 -87.34873604327832) bank150612 +150613 POINT(41.370547065931 -87.79934307627843) bank150613 +150614 POINT(41.55986541377768 -87.34965831172605) bank150614 +150615 POINT(41.81392388377078 -87.26365550185281) bank150615 +150616 POINT(41.51594494262065 -88.53336861249883) bank150616 +150617 POINT(40.95275176011519 -88.56712994231528) bank150617 +150618 POINT(42.04827112922495 -88.36144786924679) bank150618 +150619 POINT(41.452566543110365 -86.86873215958752) bank150619 +150620 POINT(41.76551343882303 -86.99620755231196) bank150620 +150621 POINT(42.64857831307308 -88.17836740022476) bank150621 +150622 POINT(42.42181908376314 -87.45021430710999) bank150622 +150623 POINT(41.61943683754227 -88.49227736884271) bank150623 +150624 POINT(40.919303683273036 -87.01654069019588) bank150624 +150625 POINT(42.64501653294799 -87.03189990422274) bank150625 +150626 POINT(42.113342238437184 -86.81704885215618) bank150626 +150627 POINT(41.721645941491715 -86.70176877664741) bank150627 +150628 POINT(40.968854646706326 -87.74700719872956) bank150628 +150629 POINT(41.17411502676661 -88.14521169541253) bank150629 +150630 POINT(40.96599025722032 -87.48942659633772) bank150630 +150631 POINT(41.92780292497776 -88.21256725697393) bank150631 +150632 POINT(41.941799198170074 -88.58712332890511) bank150632 +150633 POINT(42.37958082602717 -88.23058263470847) bank150633 +150634 POINT(41.23019441379257 -88.10368980129739) bank150634 +150635 POINT(42.33557007581558 -87.3475918002824) bank150635 +150636 POINT(41.29030221800753 -87.14896560434896) bank150636 +150637 POINT(41.38511981337765 -87.04561911679275) bank150637 +150638 POINT(42.74268249772907 -86.84297377079187) bank150638 +150639 POINT(42.434208414406875 -86.95086797302405) bank150639 +150640 POINT(42.18275915135048 -88.25007633364044) bank150640 +150641 POINT(42.72042576653978 -86.70744210425299) bank150641 +150642 POINT(41.74249837399427 -88.47998978612567) bank150642 +150643 POINT(41.704606518152985 -87.86113314655515) bank150643 +150644 POINT(41.14145299566763 -88.29306128356379) bank150644 +150645 POINT(42.40750020353798 -87.07329278017939) bank150645 +150646 POINT(42.56334676716475 -87.00303877529497) bank150646 +150647 POINT(40.99002086477648 -87.40357361957591) bank150647 +150648 POINT(41.008076189183285 -86.96979486044484) bank150648 +150649 POINT(42.3576178010169 -88.36717496470621) bank150649 +150650 POINT(42.75072713537856 -88.07122940746869) bank150650 +150651 POINT(42.46243397043776 -87.09144974521539) bank150651 +150652 POINT(41.314247919378765 -86.69203591677757) bank150652 +150653 POINT(42.47424858647382 -86.64617743090126) bank150653 +150654 POINT(41.46913765573845 -87.69164810345643) bank150654 +150655 POINT(42.14928674425608 -87.74673198125433) bank150655 +150656 POINT(41.4153365898212 -87.51072413841233) bank150656 +150657 POINT(41.64801114519256 -88.13387235088256) bank150657 +150658 POINT(40.993677932933714 -88.5491932474295) bank150658 +150659 POINT(41.06255322293907 -88.0839989742361) bank150659 +150660 POINT(42.533383131041184 -86.75733935297863) bank150660 +150661 POINT(41.4734470215134 -87.13657326833375) bank150661 +150662 POINT(40.892758613168105 -87.0014260387641) bank150662 +150663 POINT(41.590331877775334 -87.02432436313659) bank150663 +150664 POINT(42.39687596289859 -88.56060277099004) bank150664 +150665 POINT(41.64268188024145 -88.24666136178355) bank150665 +150666 POINT(41.61858702298535 -88.44693354525508) bank150666 +150667 POINT(41.227284532894295 -86.9638386728239) bank150667 +150668 POINT(41.95344232762868 -87.93237313532036) bank150668 +150669 POINT(42.46727937556087 -87.00002008673081) bank150669 +150670 POINT(40.993344976049215 -86.65446275320764) bank150670 +150671 POINT(41.392190641138704 -88.32641302490352) bank150671 +150672 POINT(42.41542953464809 -87.43832547785334) bank150672 +150673 POINT(40.89595967376807 -87.9415626197345) bank150673 +150674 POINT(42.08339003794258 -86.7451642748759) bank150674 +150675 POINT(42.56709937418425 -87.31835499564079) bank150675 +150676 POINT(42.26029966646231 -87.09024348268858) bank150676 +150677 POINT(42.26158156988865 -86.73874498756155) bank150677 +150678 POINT(42.50241234151416 -87.32588616455857) bank150678 +150679 POINT(42.304332031433965 -86.92161068938182) bank150679 +150680 POINT(42.55679180581815 -86.84557951422556) bank150680 +150681 POINT(41.162147327976555 -87.47461923156978) bank150681 +150682 POINT(40.99385306519585 -86.92590260912694) bank150682 +150683 POINT(42.3581672745964 -87.0639836807319) bank150683 +150684 POINT(41.55330177201754 -87.65122645939023) bank150684 +150685 POINT(42.33651064937168 -88.3722364675274) bank150685 +150686 POINT(42.26212637521226 -87.99915175281355) bank150686 +150687 POINT(40.93799870775598 -87.72881115205587) bank150687 +150688 POINT(41.00563742861609 -88.039983007042) bank150688 +150689 POINT(41.934695558232484 -87.0190042982533) bank150689 +150690 POINT(41.61053234388018 -87.95018701982438) bank150690 +150691 POINT(42.764901587839425 -87.74225083041031) bank150691 +150692 POINT(42.84491880681794 -86.80774702136877) bank150692 +150693 POINT(42.38034823596122 -87.45029765335019) bank150693 +150694 POINT(41.62674381872357 -86.99232474078988) bank150694 +150695 POINT(42.74366910177591 -87.2395244728886) bank150695 +150696 POINT(42.766836411435705 -86.67070058942271) bank150696 +150697 POINT(41.88979180465788 -86.79726359367525) bank150697 +150698 POINT(41.65863482115613 -87.56735341162553) bank150698 +150699 POINT(41.454630786091045 -87.33112571727474) bank150699 +150700 POINT(41.1169941507277 -87.61301842736981) bank150700 +150701 POINT(41.32419080512123 -87.62940108622531) bank150701 +150702 POINT(41.998955057835026 -87.35210804886276) bank150702 +150703 POINT(41.267819577225026 -87.658635045487) bank150703 +150704 POINT(42.65370226495654 -87.64460588089935) bank150704 +150705 POINT(42.4546822169199 -87.19441848269898) bank150705 +150706 POINT(42.244596892447376 -87.52252179063352) bank150706 +150707 POINT(42.06521585476308 -87.70372294403046) bank150707 +150708 POINT(41.16645354700168 -88.02984992659424) bank150708 +150709 POINT(40.88470184192209 -86.94193656012445) bank150709 +150710 POINT(42.49964810966841 -88.14827133953781) bank150710 +150711 POINT(42.87252107267694 -88.2026569983633) bank150711 +150712 POINT(42.160798754328276 -88.16598117827897) bank150712 +150713 POINT(42.0218624488391 -87.00392345474057) bank150713 +150714 POINT(41.4843802788518 -86.91866929963746) bank150714 +150715 POINT(41.4525392072602 -87.9140426534163) bank150715 +150716 POINT(41.26306954330443 -87.391392698215) bank150716 +150717 POINT(42.56755720655689 -87.73865817807571) bank150717 +150718 POINT(41.19945414121395 -87.35150705837037) bank150718 +150719 POINT(42.86196592290726 -87.18017960359816) bank150719 +150720 POINT(41.26950306479948 -87.07363343720061) bank150720 +150721 POINT(41.28731016777181 -88.235126549052) bank150721 +150722 POINT(42.86950575805252 -88.31030598742448) bank150722 +150723 POINT(41.807398678920855 -87.78895069544812) bank150723 +150724 POINT(42.154513890411764 -86.86114270104278) bank150724 +150725 POINT(41.08079470843001 -86.70305020310319) bank150725 +150726 POINT(42.682398462350044 -88.05249464919302) bank150726 +150727 POINT(41.68472288863693 -87.40349100833843) bank150727 +150728 POINT(42.68805218682457 -88.03909017777642) bank150728 +150729 POINT(41.417167208492366 -87.19113062884765) bank150729 +150730 POINT(42.22805123172725 -88.26256811320327) bank150730 +150731 POINT(42.52935187059709 -88.23602747141148) bank150731 +150732 POINT(41.16255304120162 -88.55250658561239) bank150732 +150733 POINT(41.301715030324104 -88.36087321964234) bank150733 +150734 POINT(41.31627838671314 -88.41115760721769) bank150734 +150735 POINT(42.730801146913414 -88.54904690225911) bank150735 +150736 POINT(40.90063685566602 -87.50871680292796) bank150736 +150737 POINT(42.372905871749616 -88.2672839187978) bank150737 +150738 POINT(40.92024378303836 -87.81058260983023) bank150738 +150739 POINT(42.26050364342067 -87.79074967376891) bank150739 +150740 POINT(42.095313995288436 -88.13661217721375) bank150740 +150741 POINT(40.93845558100502 -87.05200716861651) bank150741 +150742 POINT(42.42960004482546 -87.80830380667453) bank150742 +150743 POINT(41.24881364559227 -88.4527117642259) bank150743 +150744 POINT(42.03704929638026 -88.55813165466063) bank150744 +150745 POINT(42.286267940547276 -87.26180650679314) bank150745 +150746 POINT(42.23495978038419 -88.32712513977687) bank150746 +150747 POINT(42.21205317453375 -86.82682164513498) bank150747 +150748 POINT(41.55123561401525 -86.81606884134914) bank150748 +150749 POINT(41.33461322633523 -87.20567269177687) bank150749 +150750 POINT(41.404743643253006 -87.09625396388192) bank150750 +150751 POINT(42.47168298077024 -87.94130212671143) bank150751 +150752 POINT(41.61610655660788 -87.05931927088528) bank150752 +150753 POINT(41.48590147456372 -86.72652730443754) bank150753 +150754 POINT(41.878356705234914 -86.7110694170588) bank150754 +150755 POINT(42.32924203424251 -88.34245062842615) bank150755 +150756 POINT(42.27775917081317 -87.1259835313539) bank150756 +150757 POINT(41.31395804925853 -87.71809215570586) bank150757 +150758 POINT(41.882737241863545 -87.77161580086047) bank150758 +150759 POINT(42.246126982242686 -88.06339165496482) bank150759 +150760 POINT(42.05070928499854 -87.28438407277127) bank150760 +150761 POINT(42.40138997721785 -86.85852421069687) bank150761 +150762 POINT(42.60478075344507 -86.6517694726885) bank150762 +150763 POINT(41.14960225862094 -88.5471402454871) bank150763 +150764 POINT(41.16973094962344 -88.29209831754467) bank150764 +150765 POINT(42.50565328222685 -87.90629818879273) bank150765 +150766 POINT(41.19082629572459 -88.19805691962834) bank150766 +150767 POINT(42.00797082370536 -88.4428068808241) bank150767 +150768 POINT(42.655940903887604 -87.63777644014797) bank150768 +150769 POINT(41.767142457506516 -86.9088279641275) bank150769 +150770 POINT(41.495322026938496 -88.16869548905473) bank150770 +150771 POINT(42.63058603300522 -86.71027435693583) bank150771 +150772 POINT(41.116848723598565 -87.59626359952738) bank150772 +150773 POINT(42.0197014984725 -86.68031617254398) bank150773 +150774 POINT(42.816699595202444 -87.86732430942206) bank150774 +150775 POINT(41.43816309467124 -88.27143608777527) bank150775 +150776 POINT(41.12843801066906 -87.19747821599447) bank150776 +150777 POINT(41.000042242559864 -87.79882710871074) bank150777 +150778 POINT(42.27230138518222 -87.96589097831387) bank150778 +150779 POINT(41.14467693197133 -86.69740301020572) bank150779 +150780 POINT(41.611837942476946 -86.76777581390134) bank150780 +150781 POINT(41.770408600299334 -87.61470998249588) bank150781 +150782 POINT(42.80052745862851 -87.4339427538715) bank150782 +150783 POINT(41.39645636325047 -86.90540675815043) bank150783 +150784 POINT(41.352089069426405 -87.25745619737043) bank150784 +150785 POINT(41.25140316501972 -87.50745374241623) bank150785 +150786 POINT(41.258057934957485 -87.48466487287968) bank150786 +150787 POINT(41.853399631341475 -88.48754543144231) bank150787 +150788 POINT(40.936714098621955 -86.75245730719917) bank150788 +150789 POINT(41.895261977160544 -87.241542599697) bank150789 +150790 POINT(42.55085047752419 -87.54440322230758) bank150790 +150791 POINT(41.859562359679856 -88.26736979783843) bank150791 +150792 POINT(41.86250897427923 -87.40198181997532) bank150792 +150793 POINT(42.78062210501003 -87.18300629556167) bank150793 +150794 POINT(41.40140899085468 -87.83575208222729) bank150794 +150795 POINT(42.36605245306565 -86.7272476999583) bank150795 +150796 POINT(42.67860960575808 -86.8811438191386) bank150796 +150797 POINT(40.93501597680643 -88.43816358848869) bank150797 +150798 POINT(41.71893607252274 -86.99141656774896) bank150798 +150799 POINT(41.16037412643218 -87.73008465355272) bank150799 +150800 POINT(41.33903162615177 -88.07117301204342) bank150800 +150801 POINT(42.267673077894564 -88.02519154669919) bank150801 +150802 POINT(42.54473442595371 -86.71527180670016) bank150802 +150803 POINT(41.65401718333086 -87.19991932051838) bank150803 +150804 POINT(42.46150560678149 -86.97143510345354) bank150804 +150805 POINT(42.05014555567112 -87.18371632122646) bank150805 +150806 POINT(42.07403849781005 -87.30066915124323) bank150806 +150807 POINT(42.00431857943856 -87.05609556798814) bank150807 +150808 POINT(42.71565716242678 -88.0656388614012) bank150808 +150809 POINT(42.282764297717186 -87.53680856722049) bank150809 +150810 POINT(41.78657926500351 -87.10668730296867) bank150810 +150811 POINT(42.20003464801878 -88.42183080720024) bank150811 +150812 POINT(42.81068206355162 -86.73541788353309) bank150812 +150813 POINT(42.727703524024115 -87.50936752618618) bank150813 +150814 POINT(41.592182845803336 -86.65469123890699) bank150814 +150815 POINT(41.40539409893186 -88.38125269577957) bank150815 +150816 POINT(42.309296900873555 -87.00518136370259) bank150816 +150817 POINT(41.02263606737797 -87.76651162185944) bank150817 +150818 POINT(42.832617413703865 -87.64122016882727) bank150818 +150819 POINT(41.907743502773585 -86.76035893960047) bank150819 +150820 POINT(42.35842981442337 -87.58451058319199) bank150820 +150821 POINT(42.06132463027673 -87.99591173825179) bank150821 +150822 POINT(41.5416381403204 -87.61371345321949) bank150822 +150823 POINT(42.006899779480484 -87.84221953727702) bank150823 +150824 POINT(41.656048097890704 -87.06293940865338) bank150824 +150825 POINT(42.0125070776654 -87.7439716444707) bank150825 +150826 POINT(41.38373219260238 -87.1010855124015) bank150826 +150827 POINT(42.4609602221369 -88.60606102827697) bank150827 +150828 POINT(41.32638506460364 -86.90496008273858) bank150828 +150829 POINT(41.80774905023097 -87.83903306408278) bank150829 +150830 POINT(42.58092371382867 -87.35454868440941) bank150830 +150831 POINT(41.40434513292314 -87.00676360462437) bank150831 +150832 POINT(42.67767749514747 -87.89746587866114) bank150832 +150833 POINT(41.659292152873526 -87.73284917973452) bank150833 +150834 POINT(42.41313969083988 -87.32291217867842) bank150834 +150835 POINT(42.21308769005176 -87.98296661021323) bank150835 +150836 POINT(41.43809149988704 -86.98758380555806) bank150836 +150837 POINT(41.6088958926896 -88.47849439277068) bank150837 +150838 POINT(41.30494469298149 -87.05810873216394) bank150838 +150839 POINT(40.904718189909204 -88.09911801090487) bank150839 +150840 POINT(41.33692572993582 -88.20485213975545) bank150840 +150841 POINT(41.51984301483027 -87.96784487733454) bank150841 +150842 POINT(42.69501551939472 -87.70973084593805) bank150842 +150843 POINT(42.10329164414055 -86.82710139543379) bank150843 +150844 POINT(41.05572895975067 -88.43625266118724) bank150844 +150845 POINT(41.41003358500191 -87.29331678925605) bank150845 +150846 POINT(40.88395296608367 -87.31711493147543) bank150846 +150847 POINT(42.764889132447834 -87.50350314913923) bank150847 +150848 POINT(42.70301951963986 -87.43378805467515) bank150848 +150849 POINT(41.93440445807122 -88.32648948639829) bank150849 +150850 POINT(42.177779860260124 -87.73735547859677) bank150850 +150851 POINT(42.20998042578146 -87.14856368718182) bank150851 +150852 POINT(42.20896940332241 -87.90678917390511) bank150852 +150853 POINT(42.64430461782847 -87.79802327176745) bank150853 +150854 POINT(42.04525697852221 -88.57862485842084) bank150854 +150855 POINT(42.30967822929293 -87.33839788061022) bank150855 +150856 POINT(41.24926351774008 -87.36656378364958) bank150856 +150857 POINT(42.29453718975236 -87.2270985429152) bank150857 +150858 POINT(42.763148194887016 -88.43298249249303) bank150858 +150859 POINT(41.75652603471047 -87.80459111433512) bank150859 +150860 POINT(41.02974073601797 -87.30586961872308) bank150860 +150861 POINT(41.22676694133363 -87.54298212298245) bank150861 +150862 POINT(42.87496438851709 -87.99546133252949) bank150862 +150863 POINT(42.5366935614414 -87.1827291329724) bank150863 +150864 POINT(41.52558531703263 -87.72006529795094) bank150864 +150865 POINT(42.87260282645935 -87.21892783230685) bank150865 +150866 POINT(41.1782770166362 -87.04266166094165) bank150866 +150867 POINT(41.364520345078404 -87.16030902356692) bank150867 +150868 POINT(42.047211879285236 -86.9843881285642) bank150868 +150869 POINT(42.03138109240467 -86.93528485975585) bank150869 +150870 POINT(41.56234046132683 -88.50549456051488) bank150870 +150871 POINT(42.34962593463835 -87.93053410449862) bank150871 +150872 POINT(42.65889155534087 -87.21775329094567) bank150872 +150873 POINT(41.455472311578156 -88.4365457599451) bank150873 +150874 POINT(42.317330786851045 -87.66561275667783) bank150874 +150875 POINT(42.67430178441338 -86.7664143047296) bank150875 +150876 POINT(40.95211381369961 -87.66768188822867) bank150876 +150877 POINT(40.92308173483663 -88.20401987239072) bank150877 +150878 POINT(42.14497693447711 -88.23763030371363) bank150878 +150879 POINT(41.591666986064524 -87.45422086910902) bank150879 +150880 POINT(42.44332551823425 -86.67522171026496) bank150880 +150881 POINT(41.77814629794054 -88.3122314936918) bank150881 +150882 POINT(42.060169984020085 -88.55029725030047) bank150882 +150883 POINT(41.74269798231245 -88.44072376111413) bank150883 +150884 POINT(42.653756603625 -87.52529910913763) bank150884 +150885 POINT(42.36190762772856 -86.67269574734893) bank150885 +150886 POINT(42.58031726410611 -88.53140735215686) bank150886 +150887 POINT(41.030772404128605 -88.07805050388667) bank150887 +150888 POINT(42.522851056952156 -88.1556962733525) bank150888 +150889 POINT(41.593026070281844 -88.19804187766418) bank150889 +150890 POINT(42.31827377165576 -88.15048871177555) bank150890 +150891 POINT(41.723958955640995 -86.82712803593462) bank150891 +150892 POINT(42.429298740541974 -86.75560477297671) bank150892 +150893 POINT(41.53647684164686 -86.9372165985313) bank150893 +150894 POINT(41.10918736388088 -86.63126559671528) bank150894 +150895 POINT(41.72434392518732 -87.64774676300453) bank150895 +150896 POINT(41.84499853988916 -86.68317122254695) bank150896 +150897 POINT(42.774725016597586 -88.50211690795336) bank150897 +150898 POINT(41.50803813970822 -88.30311592761973) bank150898 +150899 POINT(42.838223552192616 -87.59019482044806) bank150899 +150900 POINT(42.74827201594525 -87.2453523072533) bank150900 +150901 POINT(40.92478920459198 -88.1927141343627) bank150901 +150902 POINT(42.78817177428432 -88.5896685935369) bank150902 +150903 POINT(40.95552771135475 -87.1554700085689) bank150903 +150904 POINT(42.45679332597144 -88.26145173113898) bank150904 +150905 POINT(42.64527732787572 -86.89894161758437) bank150905 +150906 POINT(41.08913105880866 -87.72722294427601) bank150906 +150907 POINT(42.550339619359114 -87.39807164062785) bank150907 +150908 POINT(40.982971220999794 -88.4108617855736) bank150908 +150909 POINT(41.436956424019805 -87.93315766997736) bank150909 +150910 POINT(41.559574916722084 -86.94080223615052) bank150910 +150911 POINT(41.42605845355312 -86.76191744992937) bank150911 +150912 POINT(41.129596162518666 -87.34419834879039) bank150912 +150913 POINT(41.756703396555245 -88.48734184978073) bank150913 +150914 POINT(40.90662401899176 -87.1184346521455) bank150914 +150915 POINT(42.46640602761643 -88.31441472247548) bank150915 +150916 POINT(40.98487243113705 -87.42705766951937) bank150916 +150917 POINT(42.056907411344255 -88.07391589290212) bank150917 +150918 POINT(41.38173581966109 -88.51847890574552) bank150918 +150919 POINT(41.543593659205264 -86.90923145461292) bank150919 +150920 POINT(41.786821223092424 -86.81542998450105) bank150920 +150921 POINT(42.13411483628722 -87.81519553142356) bank150921 +150922 POINT(40.97115176132328 -88.23728657262151) bank150922 +150923 POINT(41.83849100824097 -87.49519364609124) bank150923 +150924 POINT(41.10237461385103 -88.61660206863075) bank150924 +150925 POINT(40.90033712081804 -88.1443670317095) bank150925 +150926 POINT(41.787481600445304 -88.31207393308858) bank150926 +150927 POINT(42.36792991411591 -86.66369680421052) bank150927 +150928 POINT(41.42418213451597 -88.51830892297491) bank150928 +150929 POINT(41.392117060978 -88.03002225323075) bank150929 +150930 POINT(40.97687012657728 -86.8521100042212) bank150930 +150931 POINT(41.23418678613123 -88.28720424563639) bank150931 +150932 POINT(41.538132000319756 -87.53365934586274) bank150932 +150933 POINT(41.93423503039552 -86.88999255440191) bank150933 +150934 POINT(41.49907461306781 -87.4252182068156) bank150934 +150935 POINT(42.72949999771392 -87.49469260500742) bank150935 +150936 POINT(41.096633157981614 -88.09743843388785) bank150936 +150937 POINT(40.909474339510076 -87.52894128170628) bank150937 +150938 POINT(42.636072538464475 -88.60710630646676) bank150938 +150939 POINT(41.57861226102584 -86.93221049920268) bank150939 +150940 POINT(42.772284122017716 -87.11815651978185) bank150940 +150941 POINT(40.997600188501586 -86.98835226917248) bank150941 +150942 POINT(41.85088036106809 -88.03959698881337) bank150942 +150943 POINT(41.11035094455511 -87.5737720593241) bank150943 +150944 POINT(40.92364069880025 -86.87006907261936) bank150944 +150945 POINT(41.16583979258658 -87.89291546914495) bank150945 +150946 POINT(41.13813904503974 -87.6758128008053) bank150946 +150947 POINT(40.96896059305687 -88.27082012669884) bank150947 +150948 POINT(42.1996471382219 -87.93343182012698) bank150948 +150949 POINT(41.7137214003048 -87.15024505066947) bank150949 +150950 POINT(41.963740289212154 -87.55122602241346) bank150950 +150951 POINT(41.75954455732593 -86.92815739192866) bank150951 +150952 POINT(41.2873765121957 -87.15303900809455) bank150952 +150953 POINT(41.87433263414767 -86.70728313835805) bank150953 +150954 POINT(42.83183498226117 -86.79362693896678) bank150954 +150955 POINT(41.74878538052578 -87.44273338364381) bank150955 +150956 POINT(41.42175794321983 -87.70875275051003) bank150956 +150957 POINT(41.74361990118171 -86.66350257967862) bank150957 +150958 POINT(41.61768895609087 -87.5444955312529) bank150958 +150959 POINT(42.47931947280313 -88.36920232514504) bank150959 +150960 POINT(42.598240060716776 -88.12727266121541) bank150960 +150961 POINT(42.368670437434595 -87.76216989487924) bank150961 +150962 POINT(42.85237146852441 -87.19528547398663) bank150962 +150963 POINT(41.08632263905201 -88.39379524701886) bank150963 +150964 POINT(42.05686583393707 -88.32292059272747) bank150964 +150965 POINT(41.50674888055865 -86.96528977930593) bank150965 +150966 POINT(41.81010046888563 -86.66732289383022) bank150966 +150967 POINT(41.68442408421236 -88.57501784576914) bank150967 +150968 POINT(41.82594776150469 -86.92391430591692) bank150968 +150969 POINT(42.87044989602362 -88.21408137077277) bank150969 +150970 POINT(41.422916566317376 -86.65447291195784) bank150970 +150971 POINT(41.54026929624323 -88.08439865769544) bank150971 +150972 POINT(42.08521773944623 -87.93225683226582) bank150972 +150973 POINT(41.349684516154355 -87.84841439517577) bank150973 +150974 POINT(41.977386082524816 -87.70607584201649) bank150974 +150975 POINT(41.674139956742955 -88.00440412762201) bank150975 +150976 POINT(41.05284384153718 -88.08612700921694) bank150976 +150977 POINT(41.816004937089346 -88.54904753022015) bank150977 +150978 POINT(41.47134645639962 -88.48192014955885) bank150978 +150979 POINT(41.42484898931974 -86.75169553748896) bank150979 +150980 POINT(42.184713004899024 -87.14131419128611) bank150980 +150981 POINT(41.362424214229875 -87.3404132348733) bank150981 +150982 POINT(42.511894665166594 -88.18793975302012) bank150982 +150983 POINT(41.9816038746256 -88.08135459169804) bank150983 +150984 POINT(41.99943973592461 -86.91435451108869) bank150984 +150985 POINT(41.75134578731215 -87.02138284867526) bank150985 +150986 POINT(41.382303263257626 -87.62735185648991) bank150986 +150987 POINT(41.73196167087176 -88.53583899871927) bank150987 +150988 POINT(42.10467097638779 -87.70138203066121) bank150988 +150989 POINT(42.252633363060035 -88.33071117290716) bank150989 +150990 POINT(40.90493466141975 -87.91281750833126) bank150990 +150991 POINT(41.14246920668394 -88.23114986752054) bank150991 +150992 POINT(42.69634969257749 -86.88422614256304) bank150992 +150993 POINT(41.0777524080801 -87.03216064614719) bank150993 +150994 POINT(42.23388478309914 -87.55897981853123) bank150994 +150995 POINT(40.97519051488394 -86.64163046927997) bank150995 +150996 POINT(41.41241172723958 -86.98790085273741) bank150996 +150997 POINT(42.1862481033222 -86.80911336158943) bank150997 +150998 POINT(40.93446353678793 -87.98844470525373) bank150998 +150999 POINT(42.75452758602755 -87.33002438965273) bank150999 +151000 POINT(41.8720710432461 -87.38882224072776) bank151000 +151001 POINT(41.17181789837627 -86.74278526388908) bank151001 +151002 POINT(42.06458187150011 -88.29405914335783) bank151002 +151003 POINT(41.42872112622014 -87.62343267292145) bank151003 +151004 POINT(41.838460615169936 -88.16102210840401) bank151004 +151005 POINT(42.569743969553755 -87.33453413051782) bank151005 +151006 POINT(42.099942839889074 -88.59957066910228) bank151006 +151007 POINT(42.63891018304093 -87.475979586191) bank151007 +151008 POINT(41.76899037281471 -86.66322331173407) bank151008 +151009 POINT(42.26732663976054 -87.49497310385821) bank151009 +151010 POINT(42.68845546032213 -86.64293095700296) bank151010 +151011 POINT(41.90489065647353 -87.20564435886938) bank151011 +151012 POINT(41.494079949261895 -87.74499597774691) bank151012 +151013 POINT(42.595251635968545 -88.09982499025261) bank151013 +151014 POINT(40.94712900405426 -86.85405608571743) bank151014 +151015 POINT(42.71659656819684 -88.20593618641956) bank151015 +151016 POINT(42.365627728371514 -88.07754158886895) bank151016 +151017 POINT(41.817889833769236 -86.98244448766017) bank151017 +151018 POINT(40.90911381262884 -86.92957415309205) bank151018 +151019 POINT(40.946055358253794 -87.19081907873169) bank151019 +151020 POINT(41.46756743418314 -86.70285205486765) bank151020 +151021 POINT(41.198650154642316 -87.98376364366779) bank151021 +151022 POINT(42.196382807288956 -87.64501271453234) bank151022 +151023 POINT(42.17735125658894 -86.91903818454634) bank151023 +151024 POINT(41.76449501115884 -86.91236147068227) bank151024 +151025 POINT(41.42052979126137 -87.60344482323401) bank151025 +151026 POINT(41.31495898753776 -86.68621378439661) bank151026 +151027 POINT(42.77830149133695 -86.67261211352066) bank151027 +151028 POINT(42.66618495585435 -88.50850050538274) bank151028 +151029 POINT(41.835451063560384 -87.67502063177925) bank151029 +151030 POINT(41.79582723706923 -87.34875077810791) bank151030 +151031 POINT(42.26964423613249 -88.26662596323612) bank151031 +151032 POINT(42.29826264004293 -87.4885601674029) bank151032 +151033 POINT(40.998271100765294 -88.03626416628111) bank151033 +151034 POINT(41.866608526652804 -87.8939999188604) bank151034 +151035 POINT(42.50538746943081 -87.71346687985472) bank151035 +151036 POINT(40.93790477566136 -88.56106695988142) bank151036 +151037 POINT(42.869287612053455 -87.21187721050501) bank151037 +151038 POINT(41.70922860577066 -88.49759195680527) bank151038 +151039 POINT(41.940643545556036 -87.63051000788867) bank151039 +151040 POINT(41.25235866017757 -88.46083034919712) bank151040 +151041 POINT(41.118027037224486 -88.41653368219616) bank151041 +151042 POINT(42.18575340392175 -87.37833570938581) bank151042 +151043 POINT(42.22081636088748 -87.82465965999451) bank151043 +151044 POINT(41.78277855191883 -88.13154646745132) bank151044 +151045 POINT(41.55540705615039 -87.04304509356268) bank151045 +151046 POINT(41.81050739304749 -87.25130862645034) bank151046 +151047 POINT(41.88593893823165 -87.73309306924305) bank151047 +151048 POINT(41.01364129056665 -87.89538792368988) bank151048 +151049 POINT(42.46337427793653 -88.00506532232795) bank151049 +151050 POINT(42.825719565179426 -88.07136117447715) bank151050 +151051 POINT(42.05879293033171 -87.55683871342652) bank151051 +151052 POINT(42.762581273188275 -86.77876530723661) bank151052 +151053 POINT(41.235558461017206 -86.63526252952687) bank151053 +151054 POINT(41.4871645738832 -88.47418547678643) bank151054 +151055 POINT(42.44746083840165 -87.69725823168923) bank151055 +151056 POINT(40.958361314323255 -86.87916088852737) bank151056 +151057 POINT(41.85080387863148 -88.23785081953145) bank151057 +151058 POINT(42.24998217112028 -86.83548286012851) bank151058 +151059 POINT(41.41605667751536 -87.42204087780242) bank151059 +151060 POINT(42.10286133032882 -87.02931692483378) bank151060 +151061 POINT(40.90697539304836 -87.95448847075886) bank151061 +151062 POINT(42.828501281360694 -87.2263988674245) bank151062 +151063 POINT(41.80478335064612 -86.93774110084186) bank151063 +151064 POINT(42.6513024987642 -86.7692947397591) bank151064 +151065 POINT(41.23226594580025 -87.61669151411908) bank151065 +151066 POINT(41.12609365250996 -88.38181035880129) bank151066 +151067 POINT(42.532883848393105 -87.89050344430368) bank151067 +151068 POINT(41.18625325265083 -87.65582371655418) bank151068 +151069 POINT(41.09538651822786 -87.09954922984831) bank151069 +151070 POINT(42.82752104436395 -86.70947342459233) bank151070 +151071 POINT(42.37060783087804 -87.81961070767206) bank151071 +151072 POINT(42.75520275956962 -87.51867544712744) bank151072 +151073 POINT(41.59930848187021 -87.05823648282563) bank151073 +151074 POINT(42.62711425346992 -88.47643238837078) bank151074 +151075 POINT(42.650206033275836 -86.81393649269833) bank151075 +151076 POINT(42.22959634190358 -87.21423871470621) bank151076 +151077 POINT(42.21000545782378 -87.93838909385524) bank151077 +151078 POINT(42.25875995169313 -88.43703186762656) bank151078 +151079 POINT(41.19736882814189 -88.50192884145393) bank151079 +151080 POINT(40.91804535275794 -88.51595515158333) bank151080 +151081 POINT(42.3712292964277 -88.55087060462724) bank151081 +151082 POINT(41.95830632021488 -86.98623425539063) bank151082 +151083 POINT(41.45882915160075 -88.58899425711385) bank151083 +151084 POINT(42.70021824977457 -86.83662581521097) bank151084 +151085 POINT(41.1591995628736 -87.91639829845478) bank151085 +151086 POINT(41.37191794995233 -87.34255269750943) bank151086 +151087 POINT(42.49469632062426 -87.00384238387697) bank151087 +151088 POINT(42.78776549565639 -88.3784646284892) bank151088 +151089 POINT(41.502447971141414 -88.54796463820068) bank151089 +151090 POINT(40.90315918383873 -87.00162089841022) bank151090 +151091 POINT(40.928612538653276 -86.73663834054487) bank151091 +151092 POINT(41.93265692635539 -87.65836220309731) bank151092 +151093 POINT(42.206053422112845 -87.66301347686378) bank151093 +151094 POINT(41.98695803346572 -87.63679216920255) bank151094 +151095 POINT(42.45489170067733 -87.96594589696272) bank151095 +151096 POINT(42.46462256576544 -88.42429818924514) bank151096 +151097 POINT(42.4137709203102 -87.97710540448004) bank151097 +151098 POINT(41.061730373326014 -88.47140812752549) bank151098 +151099 POINT(41.921636450364524 -86.75460935226174) bank151099 +151100 POINT(41.08090100636849 -87.56204637817922) bank151100 +151101 POINT(41.325918852003866 -87.85661883665585) bank151101 +151102 POINT(41.92058833879591 -87.25938154861775) bank151102 +151103 POINT(42.2476795270239 -86.69851081034663) bank151103 +151104 POINT(41.94017904420882 -87.46331737049051) bank151104 +151105 POINT(42.370291873456544 -88.57779197389117) bank151105 +151106 POINT(41.51307380405871 -86.94140428386171) bank151106 +151107 POINT(41.77633006084302 -88.52071724465767) bank151107 +151108 POINT(42.81845528333364 -88.16936731900729) bank151108 +151109 POINT(42.51723402130304 -88.48087006954918) bank151109 +151110 POINT(42.17347017537408 -88.06996273860547) bank151110 +151111 POINT(41.854175764653085 -88.60204387249694) bank151111 +151112 POINT(41.40805900255827 -88.47074226849114) bank151112 +151113 POINT(42.223016771390434 -88.43102568902398) bank151113 +151114 POINT(41.160021142513884 -88.54715615688801) bank151114 +151115 POINT(42.05670338764081 -87.49827887062544) bank151115 +151116 POINT(40.98151132599422 -88.10174043222393) bank151116 +151117 POINT(42.39594444749811 -86.63845063633964) bank151117 +151118 POINT(42.81103568779333 -86.69555812921483) bank151118 +151119 POINT(41.57105316940471 -87.86866975879597) bank151119 +151120 POINT(42.39337169101897 -86.6871980213667) bank151120 +151121 POINT(41.80270876461438 -87.4768517394846) bank151121 +151122 POINT(41.73514606134637 -88.3887622356266) bank151122 +151123 POINT(41.60835862676289 -87.95498824762497) bank151123 +151124 POINT(42.4534019278426 -86.65975117430229) bank151124 +151125 POINT(40.98643951738204 -86.94812509488781) bank151125 +151126 POINT(42.74545375842274 -88.36124556853349) bank151126 +151127 POINT(42.11278191379357 -88.07924873022941) bank151127 +151128 POINT(42.463830197523215 -87.78434402858646) bank151128 +151129 POINT(41.861840039683585 -88.40955930027506) bank151129 +151130 POINT(41.312907099172 -88.24935970938746) bank151130 +151131 POINT(42.306868735503365 -88.3480978841496) bank151131 +151132 POINT(42.42070531191666 -86.98621872330861) bank151132 +151133 POINT(42.199094087793 -88.45761823913118) bank151133 +151134 POINT(42.319458918758684 -88.08980869397152) bank151134 +151135 POINT(42.664567528941866 -88.32457355146117) bank151135 +151136 POINT(41.17841362187002 -87.33562218930233) bank151136 +151137 POINT(40.94849543559435 -88.4295640036485) bank151137 +151138 POINT(42.76109454365751 -86.79300243441968) bank151138 +151139 POINT(42.19792416382636 -86.77123918562951) bank151139 +151140 POINT(41.51774824697778 -88.22407322259625) bank151140 +151141 POINT(41.06656745142657 -88.38757410709215) bank151141 +151142 POINT(42.361491003399145 -87.55526066985841) bank151142 +151143 POINT(42.2807071261984 -87.7400612814433) bank151143 +151144 POINT(42.22135908347313 -88.21442554552551) bank151144 +151145 POINT(41.79112331845172 -88.12348039125996) bank151145 +151146 POINT(42.023995284507755 -88.13119061109211) bank151146 +151147 POINT(42.705920029388906 -87.06833247734585) bank151147 +151148 POINT(41.90824643151503 -86.97712452466645) bank151148 +151149 POINT(42.01992229269589 -88.62527305443062) bank151149 +151150 POINT(41.35283527646642 -86.6584713641774) bank151150 +151151 POINT(42.631668301005256 -87.88372692934577) bank151151 +151152 POINT(42.30937868360762 -86.69538878250037) bank151152 +151153 POINT(41.15401281850759 -87.97801887204255) bank151153 +151154 POINT(42.362581822004366 -88.44597314531744) bank151154 +151155 POINT(41.63912932821004 -86.8706209881725) bank151155 +151156 POINT(41.236277766480335 -86.8148446698917) bank151156 +151157 POINT(42.49078863627538 -86.99395059423993) bank151157 +151158 POINT(41.991959325229516 -86.63011823343861) bank151158 +151159 POINT(41.86041499093417 -87.2598662797507) bank151159 +151160 POINT(42.46914977450195 -87.96634646627294) bank151160 +151161 POINT(42.10184825813203 -87.20009149983899) bank151161 +151162 POINT(42.48606180309436 -87.03911534096696) bank151162 +151163 POINT(42.752390459219036 -87.68849073895846) bank151163 +151164 POINT(41.16972079719164 -86.71997653301798) bank151164 +151165 POINT(42.01966906084082 -87.9756707731853) bank151165 +151166 POINT(42.46223804648488 -87.8317939726321) bank151166 +151167 POINT(41.86812855766274 -87.33393420609099) bank151167 +151168 POINT(40.94854396836115 -88.0588209944991) bank151168 +151169 POINT(42.66254767961268 -87.17111482623635) bank151169 +151170 POINT(42.021607698344326 -87.28424264744555) bank151170 +151171 POINT(41.61672949997679 -87.52938299223352) bank151171 +151172 POINT(41.0656194343567 -86.74302328742722) bank151172 +151173 POINT(41.81286478892888 -88.17012869933924) bank151173 +151174 POINT(41.83274537938732 -88.25395126318844) bank151174 +151175 POINT(42.015095438361385 -88.09376951564838) bank151175 +151176 POINT(41.05277650321483 -86.98494719368597) bank151176 +151177 POINT(41.75555017010797 -88.54497985281968) bank151177 +151178 POINT(42.12898035388931 -87.99609148471761) bank151178 +151179 POINT(42.87560829815648 -87.29128728232098) bank151179 +151180 POINT(41.71595026998164 -87.71442618062913) bank151180 +151181 POINT(42.04499475442419 -88.27819152994358) bank151181 +151182 POINT(42.58060543134322 -87.33784261807418) bank151182 +151183 POINT(41.87966275605259 -87.31810027742839) bank151183 +151184 POINT(41.96418822436141 -87.20026557062495) bank151184 +151185 POINT(42.648555484072155 -87.33511288523701) bank151185 +151186 POINT(42.426815080061864 -87.21585906102129) bank151186 +151187 POINT(41.34188565274064 -86.88694199666526) bank151187 +151188 POINT(41.774270408348926 -87.36647435064233) bank151188 +151189 POINT(41.21464427446916 -87.0425023468397) bank151189 +151190 POINT(42.58503897519076 -86.86178746184952) bank151190 +151191 POINT(41.14434807806609 -86.93838092469665) bank151191 +151192 POINT(42.712761043138265 -87.94429184885838) bank151192 +151193 POINT(41.309206680980026 -87.2343654546127) bank151193 +151194 POINT(41.7962027249149 -87.28870201867771) bank151194 +151195 POINT(42.659556682306324 -88.45569919492125) bank151195 +151196 POINT(40.928061699630724 -87.61755685881761) bank151196 +151197 POINT(41.575885804896856 -87.3163413689391) bank151197 +151198 POINT(42.262910939977125 -86.69102261614051) bank151198 +151199 POINT(41.76119545102916 -88.26283526800714) bank151199 +151200 POINT(41.9690832248254 -88.18348996047881) bank151200 +151201 POINT(41.41548913376713 -87.26646181364667) bank151201 +151202 POINT(41.4013868846223 -88.43086578389395) bank151202 +151203 POINT(42.197606272135005 -88.54773749104925) bank151203 +151204 POINT(41.795678545020046 -86.7984334580702) bank151204 +151205 POINT(41.55169296228231 -88.27204735437975) bank151205 +151206 POINT(41.76252762340001 -87.01141388231375) bank151206 +151207 POINT(42.708327922164045 -86.78709181486579) bank151207 +151208 POINT(42.356908170673066 -88.27883961178877) bank151208 +151209 POINT(41.88965688935025 -87.33480706628535) bank151209 +151210 POINT(42.02005830702089 -87.71075109042211) bank151210 +151211 POINT(42.267882684514646 -88.25833818463322) bank151211 +151212 POINT(41.35977757196661 -87.29454316985931) bank151212 +151213 POINT(42.57639468429821 -87.7141746478191) bank151213 +151214 POINT(42.79205501515726 -88.28647849373358) bank151214 +151215 POINT(41.38513366843636 -87.67949602785393) bank151215 +151216 POINT(40.948235863282925 -88.54112390258614) bank151216 +151217 POINT(42.70632584858277 -86.83593374308158) bank151217 +151218 POINT(41.71049216049939 -86.9719703403084) bank151218 +151219 POINT(41.33187955863118 -88.22637165900757) bank151219 +151220 POINT(41.40582289716853 -87.48962832733466) bank151220 +151221 POINT(41.87486613601163 -87.89091538372901) bank151221 +151222 POINT(41.76274756882999 -88.13408143366695) bank151222 +151223 POINT(41.53573391608487 -87.11043161821146) bank151223 +151224 POINT(42.390413489397545 -87.12633094475595) bank151224 +151225 POINT(42.503432973837654 -87.90210708366567) bank151225 +151226 POINT(41.416157155815064 -87.58207484770674) bank151226 +151227 POINT(40.93739914463865 -87.06327174311373) bank151227 +151228 POINT(42.501046432485985 -87.36059917424157) bank151228 +151229 POINT(41.34634951830636 -88.61007029125797) bank151229 +151230 POINT(42.36248686447876 -88.332498103308) bank151230 +151231 POINT(41.08420843515526 -86.92138571052193) bank151231 +151232 POINT(42.1827931116659 -88.28305539862893) bank151232 +151233 POINT(42.34102048074584 -86.67523592972584) bank151233 +151234 POINT(41.135133146476434 -87.38510300550817) bank151234 +151235 POINT(42.611617211943965 -87.96259009856735) bank151235 +151236 POINT(42.77105135033905 -88.3718199455411) bank151236 +151237 POINT(42.32546967665916 -88.17589131481117) bank151237 +151238 POINT(41.0630940047331 -87.20331399528025) bank151238 +151239 POINT(42.35400638521465 -88.61824863769684) bank151239 +151240 POINT(41.32959782792094 -88.03701330283135) bank151240 +151241 POINT(42.05056836077973 -86.75480965962247) bank151241 +151242 POINT(42.12243289970122 -87.54292213382082) bank151242 +151243 POINT(41.545878046258764 -88.4693704597341) bank151243 +151244 POINT(41.01729127872521 -87.90752190818618) bank151244 +151245 POINT(41.79040153324129 -87.06413211800222) bank151245 +151246 POINT(41.050146410870205 -87.83612167481577) bank151246 +151247 POINT(42.52784161307899 -87.29307359051255) bank151247 +151248 POINT(42.14231594304139 -88.58447914850022) bank151248 +151249 POINT(42.33025995015611 -88.34813895393665) bank151249 +151250 POINT(41.36633628174799 -88.15672683796406) bank151250 +151251 POINT(42.58515661193454 -86.63322324548885) bank151251 +151252 POINT(42.80722493977219 -88.50948771069324) bank151252 +151253 POINT(42.83807587535775 -88.41633077050797) bank151253 +151254 POINT(42.721539779651444 -88.18876853891354) bank151254 +151255 POINT(42.31553903461464 -87.1130288142673) bank151255 +151256 POINT(42.54340270097603 -88.42051912686487) bank151256 +151257 POINT(41.58413097981847 -88.113146214376) bank151257 +151258 POINT(42.772190219136405 -88.1937519122917) bank151258 +151259 POINT(41.51147139863653 -87.9993347959258) bank151259 +151260 POINT(41.6142376110704 -87.96189215109318) bank151260 +151261 POINT(41.96167685870995 -88.16369338791043) bank151261 +151262 POINT(41.725422406589935 -86.93235905573401) bank151262 +151263 POINT(42.683336188333946 -88.443133622489) bank151263 +151264 POINT(42.10693136217302 -87.69898471211356) bank151264 +151265 POINT(42.10102219710023 -88.11490319383816) bank151265 +151266 POINT(41.70719405428018 -88.346163873192) bank151266 +151267 POINT(42.441204376219815 -87.81403956033886) bank151267 +151268 POINT(40.97774927228193 -88.45792644173974) bank151268 +151269 POINT(41.810461277047814 -86.89256940775499) bank151269 +151270 POINT(41.60419964017347 -87.07729718237894) bank151270 +151271 POINT(42.73980263853263 -87.24500180537834) bank151271 +151272 POINT(42.68124391921029 -87.6136540169088) bank151272 +151273 POINT(41.498368508586154 -87.28252078785493) bank151273 +151274 POINT(42.81525311583097 -87.08140318019869) bank151274 +151275 POINT(42.38247835500113 -86.76700418645436) bank151275 +151276 POINT(41.80159528558867 -87.55031497730819) bank151276 +151277 POINT(41.037712279761934 -86.7183070750155) bank151277 +151278 POINT(42.03457340282881 -87.05269450100991) bank151278 +151279 POINT(41.2395961961361 -88.01202788061948) bank151279 +151280 POINT(42.526603318111334 -86.93732746836157) bank151280 +151281 POINT(41.31001554410823 -88.10844107988353) bank151281 +151282 POINT(42.134573000155605 -87.64187423359726) bank151282 +151283 POINT(42.280919834715874 -86.88130916714692) bank151283 +151284 POINT(41.52408438262314 -87.15755895665782) bank151284 +151285 POINT(41.07680203403893 -86.90993575464485) bank151285 +151286 POINT(41.08133925078866 -87.31858079649402) bank151286 +151287 POINT(42.733111226059556 -86.7569856040091) bank151287 +151288 POINT(41.39053663521149 -86.90811988680522) bank151288 +151289 POINT(41.70134834552582 -88.4435941213486) bank151289 +151290 POINT(41.807210422753876 -88.2072707779353) bank151290 +151291 POINT(41.20755376479463 -88.56569208331642) bank151291 +151292 POINT(41.410476701498276 -87.60133374181494) bank151292 +151293 POINT(42.6590697559586 -87.52345781078664) bank151293 +151294 POINT(41.41094457760304 -86.64049847208707) bank151294 +151295 POINT(40.91212308226187 -88.39622536564048) bank151295 +151296 POINT(42.63864257703819 -88.15951901655653) bank151296 +151297 POINT(41.37690273541696 -87.61126782638294) bank151297 +151298 POINT(42.04603704835157 -86.98228258384825) bank151298 +151299 POINT(42.07843887225673 -86.83399269639283) bank151299 +151300 POINT(40.99029596227639 -88.42663267801686) bank151300 +151301 POINT(41.34343138340965 -87.5685886844707) bank151301 +151302 POINT(42.05356566146659 -88.52040848199985) bank151302 +151303 POINT(41.68767386967094 -88.04859909366652) bank151303 +151304 POINT(41.82231208544859 -86.95156036836724) bank151304 +151305 POINT(42.20825252327984 -87.03683054691882) bank151305 +151306 POINT(42.74366799940662 -87.03120584225339) bank151306 +151307 POINT(41.01987431160313 -88.1198821073068) bank151307 +151308 POINT(41.10380884909157 -87.15599663091905) bank151308 +151309 POINT(42.841983107484985 -88.00675770162128) bank151309 +151310 POINT(41.2074295755775 -87.9246081225974) bank151310 +151311 POINT(41.477718863389306 -86.67352480233657) bank151311 +151312 POINT(42.131984392336555 -86.97271531933255) bank151312 +151313 POINT(41.890526928428805 -86.9856677674135) bank151313 +151314 POINT(41.51418048581774 -86.91807278939125) bank151314 +151315 POINT(42.755013491509665 -88.37747108593749) bank151315 +151316 POINT(42.758836775834936 -87.382321014129) bank151316 +151317 POINT(41.70587505658511 -87.81747752405879) bank151317 +151318 POINT(41.19776244149672 -87.0799209266312) bank151318 +151319 POINT(41.67683065391581 -87.04681812747272) bank151319 +151320 POINT(41.55229796971228 -87.3849074104962) bank151320 +151321 POINT(41.39096067373371 -86.87874848339312) bank151321 +151322 POINT(42.69928484828706 -86.78752093657023) bank151322 +151323 POINT(41.10892312049878 -87.57856275261116) bank151323 +151324 POINT(42.37634766518458 -86.66133155234094) bank151324 +151325 POINT(42.33817437094915 -86.88260503429603) bank151325 +151326 POINT(41.387322538065874 -88.33779278187964) bank151326 +151327 POINT(42.82715165984341 -88.16226107993405) bank151327 +151328 POINT(41.61694964975452 -88.09096022993232) bank151328 +151329 POINT(41.313614791546584 -87.98326395090909) bank151329 +151330 POINT(41.44805328766834 -87.10164975636141) bank151330 +151331 POINT(41.99419821037503 -86.88169738606737) bank151331 +151332 POINT(40.95217244602886 -87.40193476612872) bank151332 +151333 POINT(41.13808304088741 -87.10956651744459) bank151333 +151334 POINT(41.990931786867556 -88.35779745154677) bank151334 +151335 POINT(42.39090116607208 -87.65273671451763) bank151335 +151336 POINT(41.30802047574031 -87.53641768146906) bank151336 +151337 POINT(41.890384777010766 -87.74068765318394) bank151337 +151338 POINT(41.00069773106527 -87.85780518869147) bank151338 +151339 POINT(41.69009956292267 -88.09147152060095) bank151339 +151340 POINT(41.632840518479334 -87.19581276166524) bank151340 +151341 POINT(42.08771541763041 -86.78081925128015) bank151341 +151342 POINT(42.87510903073652 -87.20649144658917) bank151342 +151343 POINT(42.691979232283074 -87.28543930984398) bank151343 +151344 POINT(41.249129895102165 -88.11060605741481) bank151344 +151345 POINT(41.38646216891593 -88.4361199170913) bank151345 +151346 POINT(42.471923263713286 -88.58307265269904) bank151346 +151347 POINT(40.99792320232778 -87.26094311408013) bank151347 +151348 POINT(41.44608587459868 -87.09726121742445) bank151348 +151349 POINT(42.355054804058 -87.47880183484594) bank151349 +151350 POINT(40.945466784809675 -87.96709277976268) bank151350 +151351 POINT(41.48942963894236 -88.32838390784855) bank151351 +151352 POINT(41.15917751833743 -88.08119857577455) bank151352 +151353 POINT(41.63371600252738 -88.1000148574291) bank151353 +151354 POINT(41.93093969346761 -87.89492137450782) bank151354 +151355 POINT(41.81244811392639 -87.38382981475488) bank151355 +151356 POINT(42.44260400836274 -88.21483193355328) bank151356 +151357 POINT(42.03444657165916 -87.59639905884539) bank151357 +151358 POINT(41.357901029217494 -87.40625596820792) bank151358 +151359 POINT(41.776720126427264 -87.96179926428253) bank151359 +151360 POINT(41.634455956000515 -88.04125070113163) bank151360 +151361 POINT(40.91730502625289 -86.81354931938822) bank151361 +151362 POINT(41.94626430179801 -88.50232817531213) bank151362 +151363 POINT(41.66152574867286 -88.60941020481805) bank151363 +151364 POINT(42.61983471587238 -87.53748393847147) bank151364 +151365 POINT(40.896020411371325 -88.15819315965877) bank151365 +151366 POINT(41.464052633536774 -87.05537391526896) bank151366 +151367 POINT(42.21720852653355 -86.63600216425225) bank151367 +151368 POINT(42.43564753688645 -86.83776356199377) bank151368 +151369 POINT(42.40342366947421 -87.65323214033259) bank151369 +151370 POINT(42.70502001967768 -87.02653023931805) bank151370 +151371 POINT(40.94446332176339 -86.87404783331954) bank151371 +151372 POINT(42.53624655277881 -88.0141917407705) bank151372 +151373 POINT(41.591018974930186 -86.68620210091544) bank151373 +151374 POINT(42.308903137823485 -86.93709406518349) bank151374 +151375 POINT(41.81162967838052 -88.04443437050737) bank151375 +151376 POINT(42.71456064952788 -87.43868747997728) bank151376 +151377 POINT(42.46682070921884 -86.81120248907455) bank151377 +151378 POINT(41.07893314872731 -87.85472418688212) bank151378 +151379 POINT(41.03969724121805 -87.95350780070534) bank151379 +151380 POINT(41.49371564588583 -87.47445506511735) bank151380 +151381 POINT(42.4439777257697 -88.58466012072759) bank151381 +151382 POINT(42.44517328556814 -87.2910271007124) bank151382 +151383 POINT(41.94101397680166 -87.48347232101955) bank151383 +151384 POINT(40.88976747752609 -88.60459996451243) bank151384 +151385 POINT(41.55108127265898 -87.50812541359282) bank151385 +151386 POINT(41.68756481652048 -87.3089281150565) bank151386 +151387 POINT(42.71229689753357 -86.9868618442342) bank151387 +151388 POINT(41.04256645714047 -87.48954532002784) bank151388 +151389 POINT(41.612206593412274 -87.4103715172621) bank151389 +151390 POINT(41.23293684436281 -87.58789227991184) bank151390 +151391 POINT(41.21385902938582 -87.74185711356242) bank151391 +151392 POINT(42.43984063738754 -87.71741930133867) bank151392 +151393 POINT(42.41843012258716 -88.11701208658482) bank151393 +151394 POINT(41.910936637347774 -88.23524387876039) bank151394 +151395 POINT(42.13643485287095 -86.84070539341212) bank151395 +151396 POINT(42.64658178990855 -86.98448099787888) bank151396 +151397 POINT(42.00151817178948 -86.78504608982877) bank151397 +151398 POINT(41.06859801135979 -88.28595221536308) bank151398 +151399 POINT(41.839278506532686 -87.09257773951765) bank151399 +151400 POINT(42.1042873252801 -87.24392261352348) bank151400 +151401 POINT(41.2040269581586 -87.9540548849585) bank151401 +151402 POINT(42.86772693065147 -87.85971974573656) bank151402 +151403 POINT(42.14862916819562 -87.47920000775419) bank151403 +151404 POINT(41.12415477653049 -86.70472182089682) bank151404 +151405 POINT(41.30854346067311 -86.91222119919239) bank151405 +151406 POINT(42.1759333613992 -87.91565418029178) bank151406 +151407 POINT(42.24210147918551 -88.5852571773264) bank151407 +151408 POINT(42.20364157045826 -88.05937356731445) bank151408 +151409 POINT(41.62989581242099 -86.93695374183129) bank151409 +151410 POINT(41.97151727197309 -87.2756723721303) bank151410 +151411 POINT(41.045296333202195 -87.22389476819515) bank151411 +151412 POINT(42.47237243075568 -88.51191549525778) bank151412 +151413 POINT(42.186866797653664 -88.0360744129099) bank151413 +151414 POINT(42.123733388318115 -88.17780659020858) bank151414 +151415 POINT(42.852337708755755 -88.57061828033568) bank151415 +151416 POINT(41.75039274800717 -87.39615033997258) bank151416 +151417 POINT(42.49463583722692 -88.04093721493257) bank151417 +151418 POINT(41.16800915112677 -87.89981130215286) bank151418 +151419 POINT(42.571833079599614 -88.48222032119342) bank151419 +151420 POINT(42.57637154940499 -87.7281667980915) bank151420 +151421 POINT(41.93307793615847 -87.33578423344046) bank151421 +151422 POINT(42.16482431243151 -87.68650299466579) bank151422 +151423 POINT(42.44929139897569 -88.15506707055313) bank151423 +151424 POINT(41.30091912020051 -86.9675690542585) bank151424 +151425 POINT(40.96256591250481 -87.56637947138819) bank151425 +151426 POINT(42.833578043067895 -88.08704430181562) bank151426 +151427 POINT(42.20566163338945 -87.58655281804235) bank151427 +151428 POINT(42.14608064698757 -87.26235974533884) bank151428 +151429 POINT(42.633932143657866 -87.42025162807005) bank151429 +151430 POINT(42.59364973405661 -86.78738261920205) bank151430 +151431 POINT(42.6964261529408 -88.07728214277488) bank151431 +151432 POINT(41.9497469585363 -88.331569429969) bank151432 +151433 POINT(42.21071204874458 -88.22200451136645) bank151433 +151434 POINT(41.537678474689116 -88.06806584419247) bank151434 +151435 POINT(41.71958525756962 -87.54726938070719) bank151435 +151436 POINT(41.39194073486156 -88.52973983140295) bank151436 +151437 POINT(40.88620525850219 -88.5024752016905) bank151437 +151438 POINT(42.18068123493423 -87.7014216931938) bank151438 +151439 POINT(40.99967238124674 -88.29489329579097) bank151439 +151440 POINT(41.264644048289746 -87.47391378188378) bank151440 +151441 POINT(41.19988362277959 -88.50023584080253) bank151441 +151442 POINT(41.39303637986842 -87.83152724288492) bank151442 +151443 POINT(41.30766115013051 -87.2313599745772) bank151443 +151444 POINT(41.31262290371622 -88.59484516878499) bank151444 +151445 POINT(41.81343724477296 -87.88628850799884) bank151445 +151446 POINT(42.5498961752001 -88.257387249766) bank151446 +151447 POINT(42.2500555246429 -88.03986273241888) bank151447 +151448 POINT(42.6294562327052 -87.53619281686144) bank151448 +151449 POINT(41.5918504117031 -88.54481557313795) bank151449 +151450 POINT(41.95369626990037 -87.61687806946844) bank151450 +151451 POINT(41.10012235943822 -88.49173974960769) bank151451 +151452 POINT(42.325164966835 -88.54808356213147) bank151452 +151453 POINT(41.87017857384085 -88.44782685280329) bank151453 +151454 POINT(42.341634651623906 -86.76830930205979) bank151454 +151455 POINT(41.22626613598617 -87.44923194198542) bank151455 +151456 POINT(41.540016075507026 -87.43490417258363) bank151456 +151457 POINT(42.69240949590494 -87.40946503895717) bank151457 +151458 POINT(42.829098381511244 -87.14853944999895) bank151458 +151459 POINT(42.15641022046197 -87.40897928167473) bank151459 +151460 POINT(41.76644643938347 -88.11691346295645) bank151460 +151461 POINT(41.99993322013287 -87.21524219884284) bank151461 +151462 POINT(42.50793570143225 -88.03816106147003) bank151462 +151463 POINT(41.34495996520678 -88.58044597121396) bank151463 +151464 POINT(42.4039586612752 -87.62939912372478) bank151464 +151465 POINT(40.9003470171849 -87.36865914423011) bank151465 +151466 POINT(42.03497582630491 -86.64675263361627) bank151466 +151467 POINT(41.085613512241615 -86.66360510409054) bank151467 +151468 POINT(42.43727243140397 -86.78374796701947) bank151468 +151469 POINT(41.604623543464406 -86.89127980493022) bank151469 +151470 POINT(42.067314404325636 -87.15387131067885) bank151470 +151471 POINT(42.16478563233355 -87.04575966546462) bank151471 +151472 POINT(42.066089609458956 -88.58659385280896) bank151472 +151473 POINT(41.21919926084208 -87.40586177905705) bank151473 +151474 POINT(41.191193894771146 -88.55513039324518) bank151474 +151475 POINT(41.77710930744357 -87.17862275086048) bank151475 +151476 POINT(41.81385080046082 -88.21660045289022) bank151476 +151477 POINT(42.37656609612663 -88.28333063797523) bank151477 +151478 POINT(42.48200006930553 -88.15873159652692) bank151478 +151479 POINT(42.20927906380478 -87.12619644476771) bank151479 +151480 POINT(41.65913988727623 -88.34644005324155) bank151480 +151481 POINT(42.036092244165886 -87.36252921671151) bank151481 +151482 POINT(42.52748883520534 -87.70409004302972) bank151482 +151483 POINT(41.82285939737751 -88.54986477668878) bank151483 +151484 POINT(41.75562385556331 -87.77227536294285) bank151484 +151485 POINT(41.031898805433144 -88.03229805439567) bank151485 +151486 POINT(41.45605459175876 -86.64212006495356) bank151486 +151487 POINT(42.85696468111315 -87.33088172911714) bank151487 +151488 POINT(42.04380765699959 -87.10425550904407) bank151488 +151489 POINT(41.05566807758273 -86.65317289009677) bank151489 +151490 POINT(41.95906202307114 -86.89699121013365) bank151490 +151491 POINT(42.08499566302925 -87.52204441641175) bank151491 +151492 POINT(40.91649910176113 -88.47636271695262) bank151492 +151493 POINT(40.90953535647591 -87.10951628050692) bank151493 +151494 POINT(41.73967233139146 -88.08925379469147) bank151494 +151495 POINT(41.74945446932468 -87.30867475491257) bank151495 +151496 POINT(41.53399008095816 -88.3405180426705) bank151496 +151497 POINT(42.75110292748959 -88.19756903946593) bank151497 +151498 POINT(41.585002680830094 -87.4137662797904) bank151498 +151499 POINT(42.34151571209507 -87.46656055249736) bank151499 +151500 POINT(42.38259308051415 -87.07888067606191) bank151500 +151501 POINT(42.5955350085795 -87.95790006377706) bank151501 +151502 POINT(41.31290305409273 -87.7022595496859) bank151502 +151503 POINT(41.60926442931003 -88.62175807078872) bank151503 +151504 POINT(42.09137370679073 -87.60753211192143) bank151504 +151505 POINT(42.42850832355747 -87.83224112177396) bank151505 +151506 POINT(42.30177081642081 -88.12010045214622) bank151506 +151507 POINT(40.96285256166794 -86.92061089495085) bank151507 +151508 POINT(41.20596728119932 -88.38428697078047) bank151508 +151509 POINT(41.509743761247925 -87.89870835321847) bank151509 +151510 POINT(40.95104804427512 -88.54215289030988) bank151510 +151511 POINT(42.61918333582886 -86.74998303905022) bank151511 +151512 POINT(41.47045753710971 -88.25949250576026) bank151512 +151513 POINT(41.54282617772628 -86.95745966036984) bank151513 +151514 POINT(41.1899035559161 -87.14831943554061) bank151514 +151515 POINT(42.44180443025913 -88.21564683586037) bank151515 +151516 POINT(42.39387779061279 -87.0681878054111) bank151516 +151517 POINT(42.31520097554234 -88.10184223724418) bank151517 +151518 POINT(41.70805930206401 -87.261994217988) bank151518 +151519 POINT(41.71696946264481 -86.8830651171661) bank151519 +151520 POINT(42.19599671906289 -87.5100373246225) bank151520 +151521 POINT(40.88028423778997 -86.65018747093414) bank151521 +151522 POINT(42.082716169736514 -88.46941976454146) bank151522 +151523 POINT(41.610438363889884 -88.24333202330882) bank151523 +151524 POINT(41.8557024073226 -87.87380022487363) bank151524 +151525 POINT(42.07047258037467 -87.88059492473161) bank151525 +151526 POINT(42.37132560364208 -86.83271191250253) bank151526 +151527 POINT(41.7057221906729 -86.89563156355354) bank151527 +151528 POINT(41.90767148224473 -87.05804306123525) bank151528 +151529 POINT(41.24912663791716 -88.11872475019402) bank151529 +151530 POINT(41.6727843477377 -86.90755723908443) bank151530 +151531 POINT(41.76054580998221 -86.84590122299832) bank151531 +151532 POINT(41.050160691691026 -86.78128293895763) bank151532 +151533 POINT(42.0016636621134 -86.84614290288667) bank151533 +151534 POINT(42.44485657746596 -88.3038670749743) bank151534 +151535 POINT(42.114229334840566 -88.0308613967558) bank151535 +151536 POINT(42.62689361533734 -87.60995884095013) bank151536 +151537 POINT(41.48106201657027 -87.87528633384332) bank151537 +151538 POINT(41.06642323980133 -86.71815923003105) bank151538 +151539 POINT(42.21480921229566 -86.82920956359088) bank151539 +151540 POINT(41.50182676453225 -87.37279203558097) bank151540 +151541 POINT(41.37486104016469 -87.52489765079746) bank151541 +151542 POINT(40.87916190125785 -87.14225440824056) bank151542 +151543 POINT(42.234086342450375 -88.17656487311639) bank151543 +151544 POINT(42.76160720184886 -87.02609047722925) bank151544 +151545 POINT(42.80311374118479 -88.6042392103432) bank151545 +151546 POINT(42.60459439182198 -86.84040277578997) bank151546 +151547 POINT(42.60494599012353 -87.17553080469396) bank151547 +151548 POINT(42.022335206643504 -88.12493809452002) bank151548 +151549 POINT(42.845070866096655 -87.64685498288647) bank151549 +151550 POINT(42.268798268323856 -87.53303413946816) bank151550 +151551 POINT(42.225396525213085 -87.17202104509572) bank151551 +151552 POINT(42.34216485073583 -87.17565990579865) bank151552 +151553 POINT(41.458227660909166 -87.24146536650538) bank151553 +151554 POINT(41.23866585252153 -87.80187257171308) bank151554 +151555 POINT(41.941723220972534 -87.50041154318673) bank151555 +151556 POINT(41.18808689705192 -86.92196949139793) bank151556 +151557 POINT(42.57494417910261 -87.86042884573725) bank151557 +151558 POINT(42.3673079383927 -88.43405864573424) bank151558 +151559 POINT(42.20001352691784 -87.51190253291287) bank151559 +151560 POINT(42.001576791598104 -88.01432609957155) bank151560 +151561 POINT(41.94913488968145 -88.09316743037031) bank151561 +151562 POINT(42.84411702810504 -87.81347443740951) bank151562 +151563 POINT(41.50900353309335 -86.95289491752095) bank151563 +151564 POINT(42.08677668527927 -88.18843305933048) bank151564 +151565 POINT(42.6112232650929 -88.32529567546797) bank151565 +151566 POINT(41.11644307515319 -87.18017545067438) bank151566 +151567 POINT(42.69630338757456 -87.44491902635679) bank151567 +151568 POINT(41.653003733495844 -88.47435112976193) bank151568 +151569 POINT(42.87556430161074 -86.66614074521317) bank151569 +151570 POINT(42.31546386704538 -86.91328281450865) bank151570 +151571 POINT(41.50067408962597 -87.57580791225713) bank151571 +151572 POINT(42.65563862461992 -88.5591379214124) bank151572 +151573 POINT(40.956009784819116 -86.97627510154915) bank151573 +151574 POINT(42.43477193426782 -86.7434379833051) bank151574 +151575 POINT(41.98948369535237 -88.40373933358202) bank151575 +151576 POINT(42.02419620051651 -87.92466176246668) bank151576 +151577 POINT(42.13269427700018 -87.02786484636276) bank151577 +151578 POINT(41.5013323605195 -87.6201282971942) bank151578 +151579 POINT(41.036008472624786 -86.69278691645155) bank151579 +151580 POINT(42.12008958262059 -88.21235311459549) bank151580 +151581 POINT(42.490462163095216 -87.8703557550701) bank151581 +151582 POINT(41.24643301044857 -86.71381959048004) bank151582 +151583 POINT(41.01406037693434 -87.1510812025822) bank151583 +151584 POINT(42.77448482622408 -87.16070889306735) bank151584 +151585 POINT(42.36994154772681 -86.97403353359341) bank151585 +151586 POINT(42.43002727177755 -88.22072799676955) bank151586 +151587 POINT(41.19113358283895 -87.73861863552818) bank151587 +151588 POINT(41.8326876579337 -87.11231786972645) bank151588 +151589 POINT(41.60727016726251 -88.15247568900593) bank151589 +151590 POINT(41.4969690436901 -88.31572884313586) bank151590 +151591 POINT(41.89007793374543 -87.74113846745504) bank151591 +151592 POINT(42.686693439318354 -88.53037570999106) bank151592 +151593 POINT(42.64524996365004 -86.88511169887758) bank151593 +151594 POINT(42.5156292881224 -87.73189483198944) bank151594 +151595 POINT(41.86553096799803 -88.31327937159853) bank151595 +151596 POINT(42.76717808669311 -87.58026964725892) bank151596 +151597 POINT(41.717704600563984 -88.4583118720673) bank151597 +151598 POINT(41.41133584010166 -87.02684092330831) bank151598 +151599 POINT(42.59106973623105 -87.8718490103263) bank151599 +151600 POINT(42.561467843084294 -87.19482033643546) bank151600 +151601 POINT(42.53949385773146 -86.83266627237984) bank151601 +151602 POINT(42.78516169909853 -87.87061673952785) bank151602 +151603 POINT(40.972859809545554 -87.76592530566877) bank151603 +151604 POINT(42.00664850192395 -87.54170360911439) bank151604 +151605 POINT(41.9936038633255 -86.82115480526333) bank151605 +151606 POINT(41.984015781213714 -87.75081095073247) bank151606 +151607 POINT(41.330733690352865 -88.13010407604243) bank151607 +151608 POINT(41.00718209011582 -86.92393358569215) bank151608 +151609 POINT(42.590585958854774 -86.68244549914428) bank151609 +151610 POINT(41.22365947774964 -88.52140688514956) bank151610 +151611 POINT(42.575966333029356 -87.2362172459875) bank151611 +151612 POINT(41.86758019005205 -87.28968615021896) bank151612 +151613 POINT(41.377904173961326 -88.04850492118776) bank151613 +151614 POINT(41.373090314960564 -87.97557071132277) bank151614 +151615 POINT(41.852669835240455 -87.4854699648511) bank151615 +151616 POINT(41.7930955218226 -88.22670001500984) bank151616 +151617 POINT(42.82108495501528 -87.07952571995835) bank151617 +151618 POINT(41.172837245319684 -88.46013580270963) bank151618 +151619 POINT(41.845051155080334 -88.55845964551678) bank151619 +151620 POINT(41.4453737315928 -87.8989328816345) bank151620 +151621 POINT(42.25659651358666 -86.82431508254366) bank151621 +151622 POINT(41.683423700223244 -88.44749015825295) bank151622 +151623 POINT(41.41705187707976 -86.90221827310809) bank151623 +151624 POINT(41.241300517582346 -87.11606502126526) bank151624 +151625 POINT(41.089422000104264 -88.25877125841579) bank151625 +151626 POINT(41.47010486600351 -87.35794231516266) bank151626 +151627 POINT(41.55190718600869 -87.88152158229539) bank151627 +151628 POINT(42.304490246756444 -88.15368116907659) bank151628 +151629 POINT(41.322633766363396 -87.02682052273376) bank151629 +151630 POINT(42.17114994185057 -88.33448914553198) bank151630 +151631 POINT(42.51273432449594 -87.50876819375695) bank151631 +151632 POINT(41.94438119776308 -87.91573157146425) bank151632 +151633 POINT(40.9908929345736 -87.14735590977143) bank151633 +151634 POINT(42.59727822958549 -87.42385867699296) bank151634 +151635 POINT(41.014754460860836 -88.08689934478777) bank151635 +151636 POINT(41.05351456353304 -87.24707703884327) bank151636 +151637 POINT(41.47710118556388 -86.7238345829403) bank151637 +151638 POINT(42.003638210556524 -86.90486003698305) bank151638 +151639 POINT(41.971515489557845 -87.3153048241978) bank151639 +151640 POINT(41.96280995208414 -88.31061985908067) bank151640 +151641 POINT(41.39898106379705 -88.10701006066776) bank151641 +151642 POINT(41.03218194910852 -88.51877334905923) bank151642 +151643 POINT(42.62930327726005 -88.37028003645858) bank151643 +151644 POINT(41.37928997276797 -87.30695231503594) bank151644 +151645 POINT(41.83058984745305 -86.84727113954324) bank151645 +151646 POINT(42.49054213765636 -87.70563303544786) bank151646 +151647 POINT(41.870412268568394 -88.30476008405842) bank151647 +151648 POINT(42.21772091929836 -88.60032294543407) bank151648 +151649 POINT(41.3349034071089 -87.72523285685605) bank151649 +151650 POINT(41.34222445838838 -86.6672193047101) bank151650 +151651 POINT(42.12623625451914 -87.98626141985149) bank151651 +151652 POINT(42.276988945751256 -87.81636005115965) bank151652 +151653 POINT(41.75438440770527 -88.19430806901075) bank151653 +151654 POINT(42.60588432920476 -86.90872629797076) bank151654 +151655 POINT(42.356369222243565 -87.52971619164717) bank151655 +151656 POINT(42.35961761302244 -88.03417227119142) bank151656 +151657 POINT(41.01376349626948 -86.63424515729695) bank151657 +151658 POINT(42.05062425139568 -88.17195541285064) bank151658 +151659 POINT(42.63450415012622 -88.37746263081071) bank151659 +151660 POINT(41.00739642579258 -88.39460503405479) bank151660 +151661 POINT(42.47208099120105 -86.91549460208881) bank151661 +151662 POINT(42.62053435515727 -88.2684017021114) bank151662 +151663 POINT(41.581485033629804 -87.58197386699186) bank151663 +151664 POINT(41.700712536167785 -86.93888568760794) bank151664 +151665 POINT(42.05323857255762 -86.75534245057084) bank151665 +151666 POINT(41.71920395595696 -86.8126731962035) bank151666 +151667 POINT(42.365413301920796 -88.31114650990779) bank151667 +151668 POINT(42.14742109821894 -87.607682193003) bank151668 +151669 POINT(41.59833611906421 -86.67938168504044) bank151669 +151670 POINT(41.26796348433637 -88.12619366019466) bank151670 +151671 POINT(42.027261717483576 -88.59677276775386) bank151671 +151672 POINT(42.196734385647936 -87.47061797688663) bank151672 +151673 POINT(41.175771908456746 -87.95013392767989) bank151673 +151674 POINT(40.88478117978308 -88.26728897283708) bank151674 +151675 POINT(41.10575066900057 -87.72886621519767) bank151675 +151676 POINT(41.557994908659616 -87.58577647414714) bank151676 +151677 POINT(41.83616486383932 -86.7026149888002) bank151677 +151678 POINT(42.641196980353804 -87.441900650884) bank151678 +151679 POINT(41.02877445166693 -87.2490379486323) bank151679 +151680 POINT(41.801985572575404 -87.15538836132704) bank151680 +151681 POINT(41.01643148300746 -86.99191823602659) bank151681 +151682 POINT(41.663562216212796 -86.83509085595061) bank151682 +151683 POINT(40.932538425923156 -88.37244766624158) bank151683 +151684 POINT(41.2020592991549 -88.26460639819231) bank151684 +151685 POINT(41.32840596704384 -88.02037245098948) bank151685 +151686 POINT(42.03100476286179 -88.39212796258634) bank151686 +151687 POINT(41.82043895756341 -86.69558576720715) bank151687 +151688 POINT(41.17549813913582 -86.96987163581247) bank151688 +151689 POINT(41.94313340883963 -87.84464959291402) bank151689 +151690 POINT(42.02392564541961 -87.39728557733773) bank151690 +151691 POINT(42.17767666245457 -86.8275102198652) bank151691 +151692 POINT(41.52836296839446 -87.34392277917878) bank151692 +151693 POINT(42.309505176327946 -87.12183569235803) bank151693 +151694 POINT(42.85784353204225 -88.49145195409253) bank151694 +151695 POINT(41.30837629513607 -86.8980045545387) bank151695 +151696 POINT(42.41613005358148 -88.3550977945066) bank151696 +151697 POINT(41.76826957444437 -88.17606511156698) bank151697 +151698 POINT(42.25634727291733 -88.48362873930219) bank151698 +151699 POINT(42.2279174395981 -87.13693285765103) bank151699 +151700 POINT(40.92549613887282 -87.7457397743357) bank151700 +151701 POINT(42.13660255860896 -88.07880823886164) bank151701 +151702 POINT(40.94195676812566 -88.06506004019793) bank151702 +151703 POINT(41.068743050863276 -87.91513770490799) bank151703 +151704 POINT(41.27832552303861 -87.0265666377002) bank151704 +151705 POINT(42.34910147262292 -87.85469490431426) bank151705 +151706 POINT(41.41483011406186 -87.54520101283613) bank151706 +151707 POINT(41.15788834520772 -86.87349374110464) bank151707 +151708 POINT(42.309993533527205 -88.02006980301364) bank151708 +151709 POINT(41.81815724340243 -87.49166825666512) bank151709 +151710 POINT(40.907876743703916 -88.24424551408264) bank151710 +151711 POINT(42.788982237187604 -88.28829398003265) bank151711 +151712 POINT(41.22959103962038 -88.39681088913994) bank151712 +151713 POINT(41.64206840374796 -87.38680214608478) bank151713 +151714 POINT(42.53493106446207 -86.64126159431584) bank151714 +151715 POINT(41.51993635010338 -86.63924871002601) bank151715 +151716 POINT(42.35384503942352 -87.16928838088496) bank151716 +151717 POINT(42.04738695449867 -88.44490383657617) bank151717 +151718 POINT(42.72676163961682 -88.08092513327891) bank151718 +151719 POINT(42.28558204314336 -88.42023976283022) bank151719 +151720 POINT(41.71484157082312 -87.80854739092214) bank151720 +151721 POINT(42.14318916315635 -87.22712221108159) bank151721 +151722 POINT(42.302851771685944 -87.53333184710893) bank151722 +151723 POINT(41.760268258029654 -88.56540665313571) bank151723 +151724 POINT(41.271921441746926 -86.70901521250723) bank151724 +151725 POINT(41.06843394740379 -88.37820902321168) bank151725 +151726 POINT(41.19417232739101 -88.47943228801681) bank151726 +151727 POINT(41.91501406664261 -87.24819666546044) bank151727 +151728 POINT(42.033997325014255 -88.49982510542705) bank151728 +151729 POINT(42.43733710914444 -87.37577559549534) bank151729 +151730 POINT(42.513582376886106 -87.8830478608786) bank151730 +151731 POINT(42.326000088501765 -87.99900833007699) bank151731 +151732 POINT(40.97456234601084 -88.45524755155617) bank151732 +151733 POINT(42.265795688223655 -86.91866852492996) bank151733 +151734 POINT(41.95406918453436 -87.82511850268057) bank151734 +151735 POINT(42.831765109863326 -87.42966978652285) bank151735 +151736 POINT(41.147350244367374 -88.56505432957168) bank151736 +151737 POINT(41.6127670084203 -87.14989798383142) bank151737 +151738 POINT(41.93379831936277 -87.93955681242566) bank151738 +151739 POINT(42.551102154270566 -88.26268728941967) bank151739 +151740 POINT(41.048000838808335 -87.08850991154601) bank151740 +151741 POINT(42.162903438194974 -88.14124186884594) bank151741 +151742 POINT(42.612706837026174 -88.24755811227203) bank151742 +151743 POINT(41.880940751580496 -86.70663267825788) bank151743 +151744 POINT(41.34619189095352 -87.37971648253829) bank151744 +151745 POINT(42.37255442359106 -88.22041204155472) bank151745 +151746 POINT(42.103803467708445 -87.89649957169007) bank151746 +151747 POINT(41.28905028823421 -87.84179781467168) bank151747 +151748 POINT(42.57432632552917 -87.83106211739374) bank151748 +151749 POINT(42.54981316652048 -87.78606973715478) bank151749 +151750 POINT(40.98722082664063 -88.4333211266427) bank151750 +151751 POINT(42.371918690584266 -88.05188759116658) bank151751 +151752 POINT(41.50026171771144 -88.05145602305649) bank151752 +151753 POINT(42.18932624630953 -88.5555832852568) bank151753 +151754 POINT(41.53223231047231 -87.18011768737755) bank151754 +151755 POINT(41.52732520671091 -87.00973403480091) bank151755 +151756 POINT(41.96454606592661 -87.44110460988207) bank151756 +151757 POINT(42.24548099792765 -86.98123886036828) bank151757 +151758 POINT(41.32659934827819 -87.56349105402523) bank151758 +151759 POINT(41.97062636611352 -88.59058700163577) bank151759 +151760 POINT(41.33781065498258 -87.27025149672993) bank151760 +151761 POINT(40.89113470791464 -87.05584210704949) bank151761 +151762 POINT(41.67991686196809 -87.80410662821481) bank151762 +151763 POINT(42.570429662273966 -88.12479716050186) bank151763 +151764 POINT(40.91150936862684 -88.11204558950445) bank151764 +151765 POINT(41.93838515016838 -88.61449557616096) bank151765 +151766 POINT(42.38865718855933 -86.65586198318178) bank151766 +151767 POINT(41.956666522769005 -87.92442346398389) bank151767 +151768 POINT(41.6996890418522 -87.2977541873308) bank151768 +151769 POINT(41.242557038045334 -88.5691386945547) bank151769 +151770 POINT(41.76465548349452 -87.42902517479622) bank151770 +151771 POINT(42.548761026219964 -87.65353617673664) bank151771 +151772 POINT(41.622994082535115 -86.68717748176819) bank151772 +151773 POINT(41.2047482941634 -87.33679359983815) bank151773 +151774 POINT(42.525950632479095 -86.85429891049733) bank151774 +151775 POINT(41.74824662537952 -86.79068063898394) bank151775 +151776 POINT(41.64656395950466 -86.91941661516672) bank151776 +151777 POINT(41.32633515023251 -88.31716421911432) bank151777 +151778 POINT(42.40097975985997 -87.12170270904365) bank151778 +151779 POINT(42.20413250998056 -86.84297764066885) bank151779 +151780 POINT(41.35195257600455 -86.93608429036132) bank151780 +151781 POINT(42.10990818232023 -87.56723118995455) bank151781 +151782 POINT(42.70992716381527 -87.57060250983437) bank151782 +151783 POINT(42.089386543201684 -87.61513757874448) bank151783 +151784 POINT(41.84030483293192 -87.65537269180325) bank151784 +151785 POINT(41.89247391686904 -86.685943631955) bank151785 +151786 POINT(41.95586396887151 -88.48882634452023) bank151786 +151787 POINT(42.71183927831142 -87.62156391817246) bank151787 +151788 POINT(42.51324298855662 -87.95883844842231) bank151788 +151789 POINT(41.4174002590153 -88.1429770598554) bank151789 +151790 POINT(41.097316233605724 -88.34626029397299) bank151790 +151791 POINT(42.085000686931174 -87.59113424421874) bank151791 +151792 POINT(41.49383461433355 -88.46137788787883) bank151792 +151793 POINT(41.33286108674844 -86.77445307121019) bank151793 +151794 POINT(41.86553475789794 -87.93193240491452) bank151794 +151795 POINT(40.97640156488517 -87.84798139224294) bank151795 +151796 POINT(42.78062132241617 -87.2238461898094) bank151796 +151797 POINT(42.1366272570634 -87.97600049675967) bank151797 +151798 POINT(42.57764893622644 -87.1428144583273) bank151798 +151799 POINT(42.82295335292831 -87.098594468475) bank151799 +151800 POINT(42.438354415564035 -87.25515482769099) bank151800 +151801 POINT(42.4154020972539 -87.88947986621825) bank151801 +151802 POINT(41.10890562270438 -88.03945764175155) bank151802 +151803 POINT(41.25258785965571 -88.53939587636106) bank151803 +151804 POINT(42.1668860892627 -86.91673325338355) bank151804 +151805 POINT(42.84685941238891 -87.50789865372316) bank151805 +151806 POINT(41.30276765404893 -87.0941955226855) bank151806 +151807 POINT(41.16996192468174 -87.0534701539843) bank151807 +151808 POINT(41.63522415141773 -87.60006624129579) bank151808 +151809 POINT(40.906865779114604 -88.5503585902488) bank151809 +151810 POINT(41.77482149347661 -88.46363795740709) bank151810 +151811 POINT(42.46395576686254 -87.42291749962234) bank151811 +151812 POINT(42.636412445238385 -88.06430850249671) bank151812 +151813 POINT(42.05622325777212 -87.44778502772549) bank151813 +151814 POINT(41.07672387380077 -87.1511404516946) bank151814 +151815 POINT(41.91465626090426 -87.57456324993682) bank151815 +151816 POINT(40.94030401670881 -88.52624426113982) bank151816 +151817 POINT(42.241487183556835 -88.00722675137165) bank151817 +151818 POINT(41.48311923792567 -87.81591515241281) bank151818 +151819 POINT(42.41380438851203 -87.86690717118319) bank151819 +151820 POINT(42.63910236931172 -88.62719443724315) bank151820 +151821 POINT(42.41249917282298 -87.31442186550647) bank151821 +151822 POINT(42.474178676047345 -87.61798058395061) bank151822 +151823 POINT(41.47620148684068 -87.95692882678303) bank151823 +151824 POINT(40.999880510471556 -87.73843163177713) bank151824 +151825 POINT(41.86900371521928 -87.89795831775216) bank151825 +151826 POINT(42.75963021054867 -87.61316743153613) bank151826 +151827 POINT(42.09020336500772 -86.76746431048346) bank151827 +151828 POINT(42.7744168484595 -87.62327894443375) bank151828 +151829 POINT(42.82255434688032 -88.39950608762678) bank151829 +151830 POINT(41.64839848618468 -87.63359083654827) bank151830 +151831 POINT(42.48824060017767 -86.80763864351634) bank151831 +151832 POINT(42.81287487848294 -88.08830302740492) bank151832 +151833 POINT(41.926450970597806 -87.4192439596857) bank151833 +151834 POINT(41.84565419394675 -87.38974877587961) bank151834 +151835 POINT(41.867843012369754 -87.95780561452096) bank151835 +151836 POINT(42.41901554166339 -86.74579303403897) bank151836 +151837 POINT(40.95615342911333 -86.83881938358462) bank151837 +151838 POINT(41.1868260580101 -88.49895714201136) bank151838 +151839 POINT(41.02742635951391 -87.75272577703996) bank151839 +151840 POINT(41.53016859141858 -88.40760652965649) bank151840 +151841 POINT(42.41297427940202 -88.11390474096109) bank151841 +151842 POINT(42.27203361498638 -86.78020900643897) bank151842 +151843 POINT(41.22112813450833 -86.78890415064653) bank151843 +151844 POINT(42.01947970087627 -87.90150522583386) bank151844 +151845 POINT(42.38328054301791 -87.83264552419848) bank151845 +151846 POINT(41.23344537638677 -88.31074882015797) bank151846 +151847 POINT(42.35970601753236 -86.82879378124846) bank151847 +151848 POINT(42.85678026062433 -88.49270326101424) bank151848 +151849 POINT(41.78113818167831 -86.70932599533786) bank151849 +151850 POINT(41.94123702070316 -87.40441460491233) bank151850 +151851 POINT(42.48537308925111 -87.65136895409461) bank151851 +151852 POINT(41.700536192533214 -87.16879093879609) bank151852 +151853 POINT(41.153546845198846 -87.6471750559983) bank151853 +151854 POINT(40.93656868206159 -87.1741394041021) bank151854 +151855 POINT(41.51704452412154 -86.95563132744286) bank151855 +151856 POINT(42.05126871498175 -88.41988147861504) bank151856 +151857 POINT(42.55334590179118 -87.16677387420042) bank151857 +151858 POINT(41.881635862133145 -86.85842090392008) bank151858 +151859 POINT(41.022473686456465 -86.72486953432603) bank151859 +151860 POINT(40.95214333187144 -88.0245544661315) bank151860 +151861 POINT(41.71386517378255 -86.7876679206673) bank151861 +151862 POINT(41.637474443436226 -87.89160703996589) bank151862 +151863 POINT(41.50034198217998 -88.19119686305021) bank151863 +151864 POINT(40.95800701324184 -87.14064520564217) bank151864 +151865 POINT(42.023457403474744 -87.08641580811322) bank151865 +151866 POINT(41.55891180817543 -87.05156938508209) bank151866 +151867 POINT(42.82424131357963 -88.0429941466142) bank151867 +151868 POINT(41.428548486999595 -86.801396283596) bank151868 +151869 POINT(41.216007980942564 -87.2346010571334) bank151869 +151870 POINT(41.23964232438671 -86.68449186063465) bank151870 +151871 POINT(41.45303544785033 -87.38620740954373) bank151871 +151872 POINT(41.90983647441726 -87.11111398296364) bank151872 +151873 POINT(42.72375042148754 -87.65389937484024) bank151873 +151874 POINT(42.757471396620865 -86.73714691411375) bank151874 +151875 POINT(40.99034160174563 -87.92996741758105) bank151875 +151876 POINT(42.599957358401625 -87.69811094425907) bank151876 +151877 POINT(42.4887551615289 -87.39882194180332) bank151877 +151878 POINT(41.827435532175876 -88.04230130975604) bank151878 +151879 POINT(41.27541555916103 -88.04322751799617) bank151879 +151880 POINT(41.86869619966989 -87.81976651298062) bank151880 +151881 POINT(42.63535937011123 -87.22735597837814) bank151881 +151882 POINT(41.00032117179322 -88.37104288297795) bank151882 +151883 POINT(42.81831707401821 -88.40558921229733) bank151883 +151884 POINT(41.086904589897095 -86.99245985705716) bank151884 +151885 POINT(41.836622266483026 -87.73088308804061) bank151885 +151886 POINT(41.060836278838266 -87.47095277161786) bank151886 +151887 POINT(41.62193976276681 -86.96929700715404) bank151887 +151888 POINT(41.3801093016401 -88.61382274452974) bank151888 +151889 POINT(41.05973903732181 -87.31486067777077) bank151889 +151890 POINT(41.80354114481401 -87.9911272650878) bank151890 +151891 POINT(42.652996323242846 -87.7863252375589) bank151891 +151892 POINT(42.14130518186984 -87.3876479884484) bank151892 +151893 POINT(41.42155682671976 -88.49963221284321) bank151893 +151894 POINT(41.79810421209961 -87.74819079138919) bank151894 +151895 POINT(42.24880954306983 -86.7564280161257) bank151895 +151896 POINT(42.73041881516321 -87.09146825800363) bank151896 +151897 POINT(42.623977040733614 -87.96880661451897) bank151897 +151898 POINT(42.35385119827481 -87.78055994159372) bank151898 +151899 POINT(41.625526273242315 -86.97954658064378) bank151899 +151900 POINT(41.714735471605636 -87.74070019024647) bank151900 +151901 POINT(41.61571709808437 -88.62779493902468) bank151901 +151902 POINT(42.11970501194811 -87.25791518777434) bank151902 +151903 POINT(42.504721248909235 -88.02359313543279) bank151903 +151904 POINT(41.37181993584539 -86.82642432222423) bank151904 +151905 POINT(41.546858650156864 -87.52183734347255) bank151905 +151906 POINT(41.4474411716113 -88.41873760321475) bank151906 +151907 POINT(41.55145824611927 -87.65227507263317) bank151907 +151908 POINT(42.60461263897622 -87.47402002098607) bank151908 +151909 POINT(41.94016677692074 -87.33673746369696) bank151909 +151910 POINT(41.04302054282341 -86.8451084597891) bank151910 +151911 POINT(42.82928441967286 -88.58000810803645) bank151911 +151912 POINT(42.56863802165945 -88.12660535137083) bank151912 +151913 POINT(42.316296252675 -88.03442871173331) bank151913 +151914 POINT(41.24356617290234 -88.13966662947405) bank151914 +151915 POINT(42.254441937568075 -86.63548664915865) bank151915 +151916 POINT(42.601666942211835 -87.04278230593512) bank151916 +151917 POINT(41.23367264286957 -88.00024577479869) bank151917 +151918 POINT(41.96066165718873 -87.55949067906056) bank151918 +151919 POINT(41.78095717305477 -87.35740602327405) bank151919 +151920 POINT(40.98739798944045 -86.82053050578055) bank151920 +151921 POINT(42.19903959842218 -88.26044591954091) bank151921 +151922 POINT(41.965936754860046 -87.62547200379) bank151922 +151923 POINT(41.17510770392879 -88.04735559073272) bank151923 +151924 POINT(41.03702504735794 -87.40103490319437) bank151924 +151925 POINT(42.76817922576546 -86.75053392871749) bank151925 +151926 POINT(41.40808947444756 -87.64435066677984) bank151926 +151927 POINT(41.26687572967009 -88.53650057846417) bank151927 +151928 POINT(41.81928432902378 -88.56727289843485) bank151928 +151929 POINT(41.19672802824116 -86.94584669645303) bank151929 +151930 POINT(42.11945869524334 -87.61898382714082) bank151930 +151931 POINT(42.357231277840675 -87.91218553025614) bank151931 +151932 POINT(42.06791430088835 -87.36783442480245) bank151932 +151933 POINT(42.44690475398842 -87.66755661090401) bank151933 +151934 POINT(41.39357612325132 -87.58353815563342) bank151934 +151935 POINT(41.55995389247169 -87.60729255664627) bank151935 +151936 POINT(42.39582352168809 -87.85359669635595) bank151936 +151937 POINT(41.03016166181944 -87.50595623405657) bank151937 +151938 POINT(42.47466214320744 -88.50500035303057) bank151938 +151939 POINT(41.643430917851624 -87.76304242794683) bank151939 +151940 POINT(42.404300541543094 -88.47057270453924) bank151940 +151941 POINT(42.52647871448252 -87.1149565464031) bank151941 +151942 POINT(42.14961080050867 -86.98563272770913) bank151942 +151943 POINT(42.397716352492125 -87.98462013941977) bank151943 +151944 POINT(41.39620295613008 -87.2979471976689) bank151944 +151945 POINT(41.88589712842834 -88.30114549761858) bank151945 +151946 POINT(41.04732880912201 -87.2566014482844) bank151946 +151947 POINT(40.97433075590315 -88.08189736083516) bank151947 +151948 POINT(42.120661157363216 -88.53399386480712) bank151948 +151949 POINT(42.373625395049224 -88.33502530998004) bank151949 +151950 POINT(41.28931372407385 -88.1614489628131) bank151950 +151951 POINT(42.78838164960336 -87.76964720257116) bank151951 +151952 POINT(42.832017567435805 -87.24053799967453) bank151952 +151953 POINT(41.653492740110075 -88.22881201121511) bank151953 +151954 POINT(42.713990907375376 -87.37779832786676) bank151954 +151955 POINT(42.474928425777286 -87.2310942154796) bank151955 +151956 POINT(42.14069547644729 -88.25607391439347) bank151956 +151957 POINT(42.03460996192989 -87.51913183655392) bank151957 +151958 POINT(41.0437365021403 -87.00642491324169) bank151958 +151959 POINT(41.95768745913493 -86.82225374138964) bank151959 +151960 POINT(41.655628634205684 -87.28239124505807) bank151960 +151961 POINT(42.26555645992811 -88.283597360257) bank151961 +151962 POINT(41.719235099047594 -87.39829231094986) bank151962 +151963 POINT(42.5023374256792 -87.0621970959398) bank151963 +151964 POINT(40.9314040733397 -87.01449480923456) bank151964 +151965 POINT(41.45668194167263 -86.88946768033539) bank151965 +151966 POINT(42.682608041726695 -87.36534119630474) bank151966 +151967 POINT(41.83728424687929 -88.42017649076035) bank151967 +151968 POINT(42.05787542416947 -88.43192550311839) bank151968 +151969 POINT(40.94485040799863 -88.4489974597892) bank151969 +151970 POINT(41.44963185442248 -87.07333406348637) bank151970 +151971 POINT(42.67423751529507 -88.24621406508442) bank151971 +151972 POINT(42.00774457105491 -87.43347973436478) bank151972 +151973 POINT(41.90024949304314 -87.74789970446093) bank151973 +151974 POINT(42.66292349815788 -87.7125519239158) bank151974 +151975 POINT(42.3498699578518 -88.20077046830008) bank151975 +151976 POINT(41.58902925879612 -86.69121792643143) bank151976 +151977 POINT(42.58979622567475 -88.07261139332888) bank151977 +151978 POINT(42.84826649623617 -87.23481704090261) bank151978 +151979 POINT(41.48676558196378 -88.48280386298914) bank151979 +151980 POINT(41.96796039918773 -87.06677115466901) bank151980 +151981 POINT(41.06138699637575 -87.99192705629314) bank151981 +151982 POINT(42.46324901013144 -87.64304601204921) bank151982 +151983 POINT(42.272439682284 -87.76384706900838) bank151983 +151984 POINT(41.89890131231063 -86.69233554560209) bank151984 +151985 POINT(42.72323384483329 -87.48873082240674) bank151985 +151986 POINT(41.84492046003741 -87.29876386512804) bank151986 +151987 POINT(41.48641080928766 -88.51891503948542) bank151987 +151988 POINT(40.93604478083977 -88.01749503889832) bank151988 +151989 POINT(41.58364268905452 -86.91985399048997) bank151989 +151990 POINT(41.97628738471104 -87.35843361493282) bank151990 +151991 POINT(42.53681591757905 -86.86294307880993) bank151991 +151992 POINT(42.8236148660064 -88.1899023233956) bank151992 +151993 POINT(42.2449745471947 -88.05115393865556) bank151993 +151994 POINT(41.77158354140334 -87.3465870387367) bank151994 +151995 POINT(42.803528178237556 -87.45682524379728) bank151995 +151996 POINT(40.980350736399224 -87.48671150438578) bank151996 +151997 POINT(41.470578698988184 -88.38883724570368) bank151997 +151998 POINT(41.630790222147304 -87.66058689056706) bank151998 +151999 POINT(41.555648772189485 -88.11740579854008) bank151999 +152000 POINT(42.29972815372106 -88.2687100522148) bank152000 +152001 POINT(41.50957331264097 -86.89564221176852) bank152001 +152002 POINT(41.47547595463517 -88.30870019687288) bank152002 +152003 POINT(42.83681519765714 -87.02618141585558) bank152003 +152004 POINT(42.45242114286722 -88.15482777764743) bank152004 +152005 POINT(42.130356025474605 -88.06991405736626) bank152005 +152006 POINT(42.50302613463126 -87.91056503374809) bank152006 +152007 POINT(41.56287880105847 -87.12877622358229) bank152007 +152008 POINT(41.3395573528762 -88.49624681199559) bank152008 +152009 POINT(42.382299849971176 -87.9475295298896) bank152009 +152010 POINT(41.135114048887814 -87.92729632842605) bank152010 +152011 POINT(41.216776378254735 -87.61655000034278) bank152011 +152012 POINT(41.30942351713628 -87.31155591682545) bank152012 +152013 POINT(42.178804192277255 -87.35202179220227) bank152013 +152014 POINT(42.50287010459952 -87.3982430744102) bank152014 +152015 POINT(41.275886799970486 -88.4300036094331) bank152015 +152016 POINT(41.38632854063515 -88.17402055256946) bank152016 +152017 POINT(40.89423057401752 -88.24483335329312) bank152017 +152018 POINT(41.73955424795064 -88.6110105413617) bank152018 +152019 POINT(42.78503232488632 -87.98097414831275) bank152019 +152020 POINT(41.14723842046875 -88.51954369109339) bank152020 +152021 POINT(42.39396658762905 -88.52676928705611) bank152021 +152022 POINT(41.958899665104035 -87.44387519052583) bank152022 +152023 POINT(42.25585619267657 -86.98203179048842) bank152023 +152024 POINT(42.102213099635485 -86.82484429773635) bank152024 +152025 POINT(41.30174226114711 -87.34500871116802) bank152025 +152026 POINT(42.827646818548494 -88.48523737971549) bank152026 +152027 POINT(42.702649189215556 -86.67223031976722) bank152027 +152028 POINT(41.520461041565646 -88.55865464957243) bank152028 +152029 POINT(42.77514422667155 -87.79846165763644) bank152029 +152030 POINT(41.41189586674324 -88.1621132496902) bank152030 +152031 POINT(42.09998551785053 -88.21417251356456) bank152031 +152032 POINT(41.834268373337075 -87.48779717888173) bank152032 +152033 POINT(41.17541763777376 -87.81019818031719) bank152033 +152034 POINT(41.60284043139316 -86.86816607254639) bank152034 +152035 POINT(41.185617842180406 -88.00833606584912) bank152035 +152036 POINT(41.37127075331799 -87.77378454838133) bank152036 +152037 POINT(40.887332912729754 -86.82491157088197) bank152037 +152038 POINT(42.28525782222453 -88.11688226735099) bank152038 +152039 POINT(42.5185656652681 -87.7237669506949) bank152039 +152040 POINT(42.28912719350502 -87.32794936519505) bank152040 +152041 POINT(40.974248400554615 -88.24279355751077) bank152041 +152042 POINT(41.44778881564676 -88.15761946053522) bank152042 +152043 POINT(41.61591823977826 -87.89207504095927) bank152043 +152044 POINT(41.40565470217556 -86.79251977887664) bank152044 +152045 POINT(42.00014463084503 -87.00936139747621) bank152045 +152046 POINT(41.94108799144314 -88.11795111339471) bank152046 +152047 POINT(41.10317415528599 -87.30862249386614) bank152047 +152048 POINT(42.63442504199634 -87.50366556859335) bank152048 +152049 POINT(41.48299625414952 -88.25339143846819) bank152049 +152050 POINT(41.953468220283824 -88.62894558982704) bank152050 +152051 POINT(41.166082394636675 -87.56351082884271) bank152051 +152052 POINT(41.032422810979 -87.70394072161612) bank152052 +152053 POINT(42.08948093712818 -88.3006499042908) bank152053 +152054 POINT(42.739739733327426 -87.34288872852512) bank152054 +152055 POINT(41.56344298775861 -87.20116277916198) bank152055 +152056 POINT(42.35325494527019 -88.06962266094682) bank152056 +152057 POINT(41.15173325325353 -88.11910957770235) bank152057 +152058 POINT(41.59050866703832 -88.10327872147508) bank152058 +152059 POINT(41.55148935052314 -87.9556698499924) bank152059 +152060 POINT(42.11981404011642 -87.35104948074904) bank152060 +152061 POINT(42.245565593503834 -88.42465449716876) bank152061 +152062 POINT(42.75498794539345 -87.17113799106319) bank152062 +152063 POINT(41.119767186764705 -87.73107759141692) bank152063 +152064 POINT(42.80140528145917 -87.9520747527407) bank152064 +152065 POINT(42.66830538254588 -87.68365676997308) bank152065 +152066 POINT(41.89691501856862 -88.00773062164238) bank152066 +152067 POINT(42.55633551119005 -88.31365315329019) bank152067 +152068 POINT(40.92309560345803 -87.24067578749106) bank152068 +152069 POINT(42.865024822202805 -86.77968723603827) bank152069 +152070 POINT(41.76488370885538 -87.39193431215804) bank152070 +152071 POINT(40.99640492560485 -87.85399141543891) bank152071 +152072 POINT(40.90376868613714 -88.24320173056627) bank152072 +152073 POINT(41.68439785972612 -88.20012024993433) bank152073 +152074 POINT(41.669909277730866 -86.7152424648486) bank152074 +152075 POINT(42.76527149930371 -86.85452851618112) bank152075 +152076 POINT(41.26086856676908 -87.09760174941103) bank152076 +152077 POINT(42.83906037744884 -88.28600318281258) bank152077 +152078 POINT(41.29642389213385 -88.43117686595062) bank152078 +152079 POINT(41.4838326998409 -88.05568470883932) bank152079 +152080 POINT(41.51901421594737 -86.64777302422851) bank152080 +152081 POINT(42.452822220128944 -86.9972769203271) bank152081 +152082 POINT(41.44910885591989 -87.31604461769717) bank152082 +152083 POINT(41.987924714779886 -87.93141242233469) bank152083 +152084 POINT(40.961614659749074 -87.84860129065066) bank152084 +152085 POINT(42.83969271180497 -87.94957233391602) bank152085 +152086 POINT(42.32422567194704 -87.9674045946113) bank152086 +152087 POINT(42.59220234293837 -86.97618868432326) bank152087 +152088 POINT(40.9214676847057 -87.92286177762101) bank152088 +152089 POINT(42.7704057662038 -87.88961886728724) bank152089 +152090 POINT(41.088201150655465 -87.79072772572547) bank152090 +152091 POINT(41.97548058127299 -86.76107499554783) bank152091 +152092 POINT(41.93679020705887 -87.85096178438822) bank152092 +152093 POINT(41.380894305748576 -86.72736947055836) bank152093 +152094 POINT(42.11689627508091 -87.8913817388053) bank152094 +152095 POINT(42.5974144262111 -88.61368857081101) bank152095 +152096 POINT(41.04021656340743 -87.8302832966649) bank152096 +152097 POINT(42.37958599683593 -88.24761638421134) bank152097 +152098 POINT(41.78099878467605 -88.59252064937291) bank152098 +152099 POINT(42.30788537170959 -86.63421745109576) bank152099 +152100 POINT(41.07589255508792 -86.85403041795924) bank152100 +152101 POINT(42.13851539055387 -88.61568005848846) bank152101 +152102 POINT(42.79459571928507 -88.40689929173878) bank152102 +152103 POINT(41.92714710975911 -88.44159455439282) bank152103 +152104 POINT(41.35279763082302 -87.84351398197961) bank152104 +152105 POINT(42.652287098414206 -87.40751338714821) bank152105 +152106 POINT(41.94199036526207 -86.87943636166526) bank152106 +152107 POINT(41.730705436523635 -86.885159430772) bank152107 +152108 POINT(41.26767122553378 -87.16432645411646) bank152108 +152109 POINT(42.53340896196289 -87.09916417143315) bank152109 +152110 POINT(42.80619427866765 -87.83433989371107) bank152110 +152111 POINT(42.128573843043945 -87.78400084653329) bank152111 +152112 POINT(41.082972029520974 -86.68439355649822) bank152112 +152113 POINT(42.56224822333807 -88.25385311366554) bank152113 +152114 POINT(41.99818766396099 -88.06976285056153) bank152114 +152115 POINT(41.19520148578714 -87.46117445686164) bank152115 +152116 POINT(41.83205366280995 -87.65489091451255) bank152116 +152117 POINT(41.02814585946839 -87.3182240582827) bank152117 +152118 POINT(41.361951677072355 -86.81216480008871) bank152118 +152119 POINT(41.43872366536385 -88.05337686047513) bank152119 +152120 POINT(41.16585023539585 -87.56066787728612) bank152120 +152121 POINT(42.10718069078658 -86.77990774976504) bank152121 +152122 POINT(41.93628232972971 -87.89462519744853) bank152122 +152123 POINT(42.55940531115139 -88.080987010816) bank152123 +152124 POINT(41.07084530836928 -88.42224735549593) bank152124 +152125 POINT(41.50473565267037 -87.49803070985763) bank152125 +152126 POINT(42.388644795126666 -86.69560010295537) bank152126 +152127 POINT(42.07322987676238 -88.14968183828465) bank152127 +152128 POINT(41.579363683408694 -86.89623057991336) bank152128 +152129 POINT(40.978653954133 -88.3268783736792) bank152129 +152130 POINT(42.77650731775327 -87.09373445871225) bank152130 +152131 POINT(41.29786338604508 -86.8260599151668) bank152131 +152132 POINT(41.93517642547647 -86.98147633670405) bank152132 +152133 POINT(42.788814116461246 -87.20124765930937) bank152133 +152134 POINT(41.921459575974474 -88.55606561410808) bank152134 +152135 POINT(41.68871516885011 -86.6327724751966) bank152135 +152136 POINT(42.85384357270151 -87.80099348108308) bank152136 +152137 POINT(41.68981636566414 -87.12767718599605) bank152137 +152138 POINT(42.25922957935894 -87.8854952753244) bank152138 +152139 POINT(41.01288523063446 -88.10225478759337) bank152139 +152140 POINT(41.76004726907734 -86.71918063559744) bank152140 +152141 POINT(41.56006468485482 -86.71056402758602) bank152141 +152142 POINT(41.29191509415961 -87.42343050636049) bank152142 +152143 POINT(41.47276520401211 -87.63474126557205) bank152143 +152144 POINT(41.37782139455389 -88.27681549746359) bank152144 +152145 POINT(42.25967976645179 -87.51013432765826) bank152145 +152146 POINT(42.52287870391316 -88.12416511200477) bank152146 +152147 POINT(42.63101937340891 -87.25939040022543) bank152147 +152148 POINT(41.91626675796212 -88.02426236092538) bank152148 +152149 POINT(41.458928239193 -87.82493010499097) bank152149 +152150 POINT(40.998985843400334 -88.21922109443585) bank152150 +152151 POINT(42.38066166951467 -86.79048836076512) bank152151 +152152 POINT(41.24780237059976 -87.8623954829188) bank152152 +152153 POINT(40.96934949041256 -87.53823642822557) bank152153 +152154 POINT(41.43956199622994 -87.41110669765213) bank152154 +152155 POINT(42.7524918895806 -87.63041114615051) bank152155 +152156 POINT(42.65454347292053 -87.52438978777143) bank152156 +152157 POINT(41.41470891417434 -88.31510268282338) bank152157 +152158 POINT(42.36522461423967 -86.92451831964662) bank152158 +152159 POINT(41.70997794442993 -87.91526526041972) bank152159 +152160 POINT(42.07010797505004 -88.59231309717373) bank152160 +152161 POINT(41.32948454993986 -87.72601242912746) bank152161 +152162 POINT(42.13130253061694 -88.3944190907814) bank152162 +152163 POINT(42.399409554104594 -87.60341528412596) bank152163 +152164 POINT(41.39397526316964 -88.38205294030558) bank152164 +152165 POINT(41.639458272534114 -88.10507363502515) bank152165 +152166 POINT(42.87040876537436 -87.80458833062396) bank152166 +152167 POINT(41.60303770770206 -86.98571391113457) bank152167 +152168 POINT(41.871542276142335 -87.34990346069794) bank152168 +152169 POINT(41.75533393766801 -88.22307111953145) bank152169 +152170 POINT(42.35148734409992 -87.76291740112225) bank152170 +152171 POINT(41.196960017763324 -87.20087188120489) bank152171 +152172 POINT(41.156640694079634 -88.62069573017754) bank152172 +152173 POINT(41.20648054439603 -87.95326063566417) bank152173 +152174 POINT(41.48345239768818 -87.59290485495772) bank152174 +152175 POINT(41.77266670960147 -88.00726464372408) bank152175 +152176 POINT(41.451592367198444 -87.83587819159548) bank152176 +152177 POINT(42.36343870176974 -87.83659379536674) bank152177 +152178 POINT(42.253756460589685 -87.7663963107133) bank152178 +152179 POINT(42.6627408739273 -87.47409672957154) bank152179 +152180 POINT(41.15239684902196 -86.78647473831818) bank152180 +152181 POINT(42.86250457167378 -86.78088114842836) bank152181 +152182 POINT(41.238599820263886 -86.63471087314336) bank152182 +152183 POINT(41.37403367491165 -88.59906025827377) bank152183 +152184 POINT(41.23534237326707 -87.12685058250268) bank152184 +152185 POINT(41.69882442666566 -87.27122200657075) bank152185 +152186 POINT(40.889873435817684 -88.22191750121887) bank152186 +152187 POINT(42.15634431888189 -87.36937448521324) bank152187 +152188 POINT(41.48539021164986 -87.82678315891724) bank152188 +152189 POINT(40.98378337213244 -88.52373936951076) bank152189 +152190 POINT(41.06572063141722 -87.64281505218024) bank152190 +152191 POINT(42.47204152136996 -87.50667688603218) bank152191 +152192 POINT(41.359275002053714 -88.43666585399899) bank152192 +152193 POINT(42.81156862293564 -87.85522959020611) bank152193 +152194 POINT(41.92778884985747 -88.14890137330782) bank152194 +152195 POINT(40.88667797589154 -87.31168600075307) bank152195 +152196 POINT(42.857103407021825 -87.9267675937673) bank152196 +152197 POINT(41.870011680691135 -88.33646532400007) bank152197 +152198 POINT(42.05271276928006 -87.27161724424032) bank152198 +152199 POINT(41.793946777313934 -88.08973205384252) bank152199 +152200 POINT(41.66265184450656 -87.10486425643911) bank152200 +152201 POINT(42.7810660793354 -87.80230261957412) bank152201 +152202 POINT(42.31744196601472 -88.5736533482721) bank152202 +152203 POINT(41.484733507014454 -86.6741482522749) bank152203 +152204 POINT(41.114565633944814 -87.55879928808199) bank152204 +152205 POINT(41.97751801266792 -88.24252510935949) bank152205 +152206 POINT(41.95684951639117 -88.48591686831398) bank152206 +152207 POINT(42.47705302922074 -88.43889986195799) bank152207 +152208 POINT(42.34519667091172 -88.56251204408726) bank152208 +152209 POINT(42.004336930253814 -87.98054185796681) bank152209 +152210 POINT(40.98256561818184 -86.99835168573227) bank152210 +152211 POINT(41.96843939470469 -88.43759899250352) bank152211 +152212 POINT(41.829425755925726 -87.07305478828403) bank152212 +152213 POINT(41.09425354941893 -87.2786706411437) bank152213 +152214 POINT(41.70656757642696 -87.52626802025848) bank152214 +152215 POINT(41.283670432384476 -88.4700912674663) bank152215 +152216 POINT(41.37675030863407 -88.41637580751389) bank152216 +152217 POINT(41.83306245989282 -88.29932720620985) bank152217 +152218 POINT(40.979283878907786 -88.10771790603525) bank152218 +152219 POINT(42.46336205151744 -86.8114312647) bank152219 +152220 POINT(42.06119046659567 -87.68523435127275) bank152220 +152221 POINT(41.88083007072215 -87.83422232966893) bank152221 +152222 POINT(41.50853202123268 -88.33415003795298) bank152222 +152223 POINT(42.73513111351035 -88.46964318297046) bank152223 +152224 POINT(41.88385042143098 -87.96027879466409) bank152224 +152225 POINT(41.775132936633234 -88.14312789416553) bank152225 +152226 POINT(42.415444523463364 -87.10498606203207) bank152226 +152227 POINT(42.67624311258519 -88.37936324037545) bank152227 +152228 POINT(42.021715117691016 -87.66936264904334) bank152228 +152229 POINT(41.74841386549239 -87.05781521830335) bank152229 +152230 POINT(40.91786911029637 -88.5207418341825) bank152230 +152231 POINT(41.72413607191183 -87.31718637603792) bank152231 +152232 POINT(42.19909529232326 -87.83330161458007) bank152232 +152233 POINT(41.78827978615612 -88.44568318787461) bank152233 +152234 POINT(41.5062463115466 -87.82331577061895) bank152234 +152235 POINT(41.22100514390672 -87.20917957353454) bank152235 +152236 POINT(42.52963514785892 -88.00481759388636) bank152236 +152237 POINT(42.61817745290866 -87.69532009008013) bank152237 +152238 POINT(40.92785299339301 -88.01209900464993) bank152238 +152239 POINT(41.673605664289795 -87.37097650262675) bank152239 +152240 POINT(41.45653840337609 -86.82934165574511) bank152240 +152241 POINT(42.798244591723275 -86.64430062304672) bank152241 +152242 POINT(40.931526745541525 -86.81939452502851) bank152242 +152243 POINT(41.55141814642124 -88.26723253199535) bank152243 +152244 POINT(41.74214073798683 -88.02024272117497) bank152244 +152245 POINT(40.946860249334804 -87.14774294272493) bank152245 +152246 POINT(42.21225738548497 -87.46516078459489) bank152246 +152247 POINT(41.69105105944594 -87.32570221445854) bank152247 +152248 POINT(41.474239014613275 -87.70569645652846) bank152248 +152249 POINT(41.812858910901724 -87.70027710846043) bank152249 +152250 POINT(41.661512360180865 -88.50568341544675) bank152250 +152251 POINT(42.754752374811055 -88.56062244288287) bank152251 +152252 POINT(41.65920672932717 -87.41209251818422) bank152252 +152253 POINT(41.69807779948541 -88.09205580461368) bank152253 +152254 POINT(41.24497601457899 -87.5214317288881) bank152254 +152255 POINT(41.78190712535639 -88.24408129590968) bank152255 +152256 POINT(41.57045890434362 -87.32601834349951) bank152256 +152257 POINT(41.14626048875216 -86.8404880041519) bank152257 +152258 POINT(41.72629622204259 -86.98723338605504) bank152258 +152259 POINT(42.44894247495226 -88.0322591302602) bank152259 +152260 POINT(42.67175073440401 -87.66889209022096) bank152260 +152261 POINT(42.290576039493544 -88.50671875010958) bank152261 +152262 POINT(40.96299222300184 -86.74044152454952) bank152262 +152263 POINT(42.71805853750098 -88.15963520705081) bank152263 +152264 POINT(41.149221468380105 -86.91808492578159) bank152264 +152265 POINT(41.88703927003706 -87.48762466139031) bank152265 +152266 POINT(41.32438824720595 -88.28491658701535) bank152266 +152267 POINT(41.22216340380382 -86.88598226439291) bank152267 +152268 POINT(42.793107958582816 -88.31941866533255) bank152268 +152269 POINT(42.82648021354132 -87.10004476209863) bank152269 +152270 POINT(41.42152033871487 -87.82008849247904) bank152270 +152271 POINT(42.598576043864966 -88.56393874896318) bank152271 +152272 POINT(41.60257769771109 -86.99725267880108) bank152272 +152273 POINT(42.756866761248276 -87.701462157052) bank152273 +152274 POINT(41.943116793020124 -88.3470395180668) bank152274 +152275 POINT(42.76193282407379 -87.37941608333766) bank152275 +152276 POINT(41.77649591357623 -87.08235530018773) bank152276 +152277 POINT(40.983262054226024 -87.74424193146113) bank152277 +152278 POINT(41.55258636021859 -87.53414629609458) bank152278 +152279 POINT(42.442086257839165 -88.50011084136264) bank152279 +152280 POINT(41.98597445974843 -87.92172893508491) bank152280 +152281 POINT(42.14433681652225 -88.22085056126483) bank152281 +152282 POINT(42.14337310111827 -88.03214320721895) bank152282 +152283 POINT(41.09227021413534 -86.67010989612083) bank152283 +152284 POINT(41.79632332972153 -88.37469472154427) bank152284 +152285 POINT(41.221338050018026 -88.52691088282005) bank152285 +152286 POINT(41.02630904843166 -88.6073336285697) bank152286 +152287 POINT(42.4854039303725 -87.75346381833776) bank152287 +152288 POINT(42.46260726776594 -87.27946761158205) bank152288 +152289 POINT(41.518488599798594 -86.94164864504438) bank152289 +152290 POINT(41.79338695662211 -86.89767175812516) bank152290 +152291 POINT(42.49516901293345 -88.1577837685363) bank152291 +152292 POINT(40.914929503405254 -88.46634651180084) bank152292 +152293 POINT(41.39416113280107 -87.14983274281279) bank152293 +152294 POINT(41.46348279656017 -88.24371586185846) bank152294 +152295 POINT(42.82200542344769 -87.35942772880406) bank152295 +152296 POINT(42.23570139109429 -88.4274432667402) bank152296 +152297 POINT(42.3898818154595 -88.46484732828166) bank152297 +152298 POINT(42.024483182093604 -87.39884816835634) bank152298 +152299 POINT(42.24949730144388 -86.6370987315097) bank152299 +152300 POINT(42.698796845519084 -87.3449694996857) bank152300 +152301 POINT(42.1011376961418 -87.35986334555483) bank152301 +152302 POINT(41.99272934265084 -87.01841756586497) bank152302 +152303 POINT(41.579543735497694 -87.2561843531682) bank152303 +152304 POINT(42.07055465145416 -87.56920776436993) bank152304 +152305 POINT(41.411062230562024 -87.148965425338) bank152305 +152306 POINT(41.374349913690985 -88.35490419600211) bank152306 +152307 POINT(41.73999451524698 -87.30612082245894) bank152307 +152308 POINT(42.33169175579967 -86.82225422296777) bank152308 +152309 POINT(42.356237888204134 -87.34142542969103) bank152309 +152310 POINT(41.8797661516772 -86.88797219709176) bank152310 +152311 POINT(42.323107328999086 -88.26789974338226) bank152311 +152312 POINT(42.85297405878647 -87.62231473551793) bank152312 +152313 POINT(42.785332581434155 -87.51172182437179) bank152313 +152314 POINT(42.34055366326188 -87.9616932429973) bank152314 +152315 POINT(42.41868284872389 -86.9783306011794) bank152315 +152316 POINT(41.284348237307526 -87.99855547511838) bank152316 +152317 POINT(42.83692706013689 -87.6616163562544) bank152317 +152318 POINT(41.38138233601063 -86.70583885659909) bank152318 +152319 POINT(42.75124368293794 -88.40086140928605) bank152319 +152320 POINT(40.91355662924516 -88.50840359983343) bank152320 +152321 POINT(41.02052629068442 -86.83859654161404) bank152321 +152322 POINT(41.28845600310922 -88.15474530387961) bank152322 +152323 POINT(42.25369100834254 -88.02327041672594) bank152323 +152324 POINT(42.625056433510906 -87.3080413370313) bank152324 +152325 POINT(41.054735853132996 -88.18999374280703) bank152325 +152326 POINT(42.75347701953703 -87.24181515557386) bank152326 +152327 POINT(42.490210177538444 -86.74521298462955) bank152327 +152328 POINT(41.59161019919732 -88.10145495133204) bank152328 +152329 POINT(41.409693171066756 -86.81331749516167) bank152329 +152330 POINT(41.13722732424645 -86.77900681379103) bank152330 +152331 POINT(41.668902161024214 -87.54158154084415) bank152331 +152332 POINT(42.35886219376123 -87.74735260345511) bank152332 +152333 POINT(41.06892525447371 -87.53934152518612) bank152333 +152334 POINT(41.405102060284285 -88.00080088505611) bank152334 +152335 POINT(41.105935143922004 -86.76124125638071) bank152335 +152336 POINT(42.73045113639124 -86.65318397624678) bank152336 +152337 POINT(40.92744147392789 -86.88419112421022) bank152337 +152338 POINT(41.183400566920255 -86.96070378891984) bank152338 +152339 POINT(41.40201808700939 -87.14869800466727) bank152339 +152340 POINT(41.07113705902669 -86.94457839509451) bank152340 +152341 POINT(42.76074107867962 -86.93848659006088) bank152341 +152342 POINT(40.96389359726863 -87.22588672524031) bank152342 +152343 POINT(41.72490560305821 -87.20354948982371) bank152343 +152344 POINT(41.41919341821124 -86.85102518092216) bank152344 +152345 POINT(41.11323488825285 -87.40629577729382) bank152345 +152346 POINT(41.311064906132685 -86.81402389042957) bank152346 +152347 POINT(42.59923090104728 -86.93828306038225) bank152347 +152348 POINT(42.03765126469717 -88.25556544974118) bank152348 +152349 POINT(41.39987815825876 -86.94632320669214) bank152349 +152350 POINT(41.23248441167636 -87.76770035511652) bank152350 +152351 POINT(41.52370433516172 -87.4367883646271) bank152351 +152352 POINT(41.155299188597844 -87.87145887920255) bank152352 +152353 POINT(41.062250752596455 -87.71689966148408) bank152353 +152354 POINT(41.92539284366449 -88.07095367639309) bank152354 +152355 POINT(41.69040267478648 -87.58005082832803) bank152355 +152356 POINT(41.43510030624473 -87.80371092214403) bank152356 +152357 POINT(41.609910529647046 -88.42475176571782) bank152357 +152358 POINT(41.52825236992623 -88.2444031113277) bank152358 +152359 POINT(42.04013683351563 -87.97231321476255) bank152359 +152360 POINT(42.759726642448356 -88.42046345957704) bank152360 +152361 POINT(42.73401548637789 -87.9973116166337) bank152361 +152362 POINT(41.71662106341484 -87.74810002191268) bank152362 +152363 POINT(42.07650771574092 -86.82295005504076) bank152363 +152364 POINT(42.184183876965875 -87.95717280791234) bank152364 +152365 POINT(42.50886239894463 -88.25578225737479) bank152365 +152366 POINT(41.83430936084589 -88.54327975853315) bank152366 +152367 POINT(42.43592632813516 -86.95506326925236) bank152367 +152368 POINT(41.35732850630397 -87.52711587998895) bank152368 +152369 POINT(42.55061757277861 -87.7619032318563) bank152369 +152370 POINT(41.41628919366237 -87.30439613536723) bank152370 +152371 POINT(42.77212012137631 -87.9773573228423) bank152371 +152372 POINT(41.7955027674788 -88.28082709230573) bank152372 +152373 POINT(42.41238965377794 -87.1416202537792) bank152373 +152374 POINT(40.981005983489645 -88.02636581686629) bank152374 +152375 POINT(41.40449685363313 -87.63914594825214) bank152375 +152376 POINT(42.49940290518623 -88.4761387107377) bank152376 +152377 POINT(41.45097727713369 -87.00941352593067) bank152377 +152378 POINT(42.194408216231665 -86.76784992934125) bank152378 +152379 POINT(41.96264306875656 -87.98997953779006) bank152379 +152380 POINT(41.97531154381913 -88.5972948832055) bank152380 +152381 POINT(41.06583727048139 -88.26988171366646) bank152381 +152382 POINT(42.3028482154253 -87.75946860771174) bank152382 +152383 POINT(40.969045788272176 -87.30195905049904) bank152383 +152384 POINT(41.258873317453954 -86.92939243312047) bank152384 +152385 POINT(41.88911374910505 -88.3280903189029) bank152385 +152386 POINT(41.72260420499341 -88.49756940294407) bank152386 +152387 POINT(42.2824550807554 -88.12214316219972) bank152387 +152388 POINT(42.01178497568672 -88.28753082337337) bank152388 +152389 POINT(41.859821845439846 -87.66417577961454) bank152389 +152390 POINT(41.28526755183786 -87.76478596949872) bank152390 +152391 POINT(42.82051098909464 -87.49844587793332) bank152391 +152392 POINT(42.17459043890254 -88.35993794601742) bank152392 +152393 POINT(42.71615651504594 -87.8294829514097) bank152393 +152394 POINT(41.579059647394814 -87.30841456454819) bank152394 +152395 POINT(41.60574754052385 -87.97094922621028) bank152395 +152396 POINT(42.16426718316411 -86.99166942160626) bank152396 +152397 POINT(40.97070479707246 -86.65309069151834) bank152397 +152398 POINT(42.02071562659256 -88.1202753345673) bank152398 +152399 POINT(41.339253008661174 -88.41755430749113) bank152399 +152400 POINT(41.11701674174533 -88.51729366279197) bank152400 +152401 POINT(41.59842319864117 -87.56476552648721) bank152401 +152402 POINT(42.07439231706594 -88.33803833972608) bank152402 +152403 POINT(41.870577159716404 -87.4539147816112) bank152403 +152404 POINT(41.471213613126764 -86.88492921255032) bank152404 +152405 POINT(42.76192119927278 -87.32542979106223) bank152405 +152406 POINT(42.161384657818225 -88.25044996059869) bank152406 +152407 POINT(42.14252457373363 -86.85750150358238) bank152407 +152408 POINT(41.53809650470478 -87.99102500086984) bank152408 +152409 POINT(41.11623284650011 -86.70095541295808) bank152409 +152410 POINT(41.93336331588387 -87.13635760319391) bank152410 +152411 POINT(41.33002525407484 -87.91975271644169) bank152411 +152412 POINT(42.77202336378453 -87.38295338434027) bank152412 +152413 POINT(42.622063995341755 -88.48681621625322) bank152413 +152414 POINT(42.54465778528085 -88.33981209156659) bank152414 +152415 POINT(42.38727342940396 -87.18479723106667) bank152415 +152416 POINT(42.1585797973723 -87.67020414573314) bank152416 +152417 POINT(41.95094798914628 -88.37319838459565) bank152417 +152418 POINT(42.20648746766922 -88.61305768148138) bank152418 +152419 POINT(42.131944240466794 -87.06360260731175) bank152419 +152420 POINT(42.73084655197453 -87.76385507093885) bank152420 +152421 POINT(41.9099033910916 -87.86177128581306) bank152421 +152422 POINT(41.50434959765288 -87.73637293337093) bank152422 +152423 POINT(42.14228872380954 -87.07437130294959) bank152423 +152424 POINT(42.59506222674444 -88.30700895135958) bank152424 +152425 POINT(42.791966508035884 -87.22483554871636) bank152425 +152426 POINT(41.42680411578012 -87.18408841875888) bank152426 +152427 POINT(41.591803728282976 -88.37792871039909) bank152427 +152428 POINT(42.19380015481314 -87.81739761328707) bank152428 +152429 POINT(41.54583438590882 -88.61859100102332) bank152429 +152430 POINT(42.75464562239724 -87.17587501160787) bank152430 +152431 POINT(41.31518753938945 -87.70818601480084) bank152431 +152432 POINT(42.5382898677591 -87.0813972630709) bank152432 +152433 POINT(42.600699960803446 -86.72057969103237) bank152433 +152434 POINT(41.27618532723143 -88.3114397383422) bank152434 +152435 POINT(41.035331649096925 -88.33577912362732) bank152435 +152436 POINT(42.109590143048734 -88.09606876589574) bank152436 +152437 POINT(41.70941279269476 -86.99374786489946) bank152437 +152438 POINT(40.93686764650515 -87.29930230048262) bank152438 +152439 POINT(42.282412962196574 -86.87451769864475) bank152439 +152440 POINT(42.342464995365006 -88.49284771836174) bank152440 +152441 POINT(42.0041135381565 -88.6019678795675) bank152441 +152442 POINT(40.897827343877985 -88.09790624701057) bank152442 +152443 POINT(42.02083883157302 -88.47998634974614) bank152443 +152444 POINT(41.12548039813767 -86.82092440070434) bank152444 +152445 POINT(41.77363595153791 -88.48181261588557) bank152445 +152446 POINT(42.24727788613602 -87.95494239803989) bank152446 +152447 POINT(41.62658590958679 -87.75378936973475) bank152447 +152448 POINT(41.553616677069286 -86.84842698103618) bank152448 +152449 POINT(41.271484640183736 -87.62131886607715) bank152449 +152450 POINT(42.192587112805064 -86.8845502661307) bank152450 +152451 POINT(41.663649108244634 -88.48990354975172) bank152451 +152452 POINT(41.865483358137375 -86.73508086346233) bank152452 +152453 POINT(41.956673169882535 -88.10249388364947) bank152453 +152454 POINT(41.11079278231587 -87.51142103132312) bank152454 +152455 POINT(41.224148405851274 -87.36850825308102) bank152455 +152456 POINT(41.285447176704764 -88.34432966120018) bank152456 +152457 POINT(41.93109122571864 -87.01656701152963) bank152457 +152458 POINT(42.358119425773225 -86.80446291954784) bank152458 +152459 POINT(42.112784492953885 -87.72525613428404) bank152459 +152460 POINT(42.85696233399838 -87.35657708280871) bank152460 +152461 POINT(41.03340849457886 -88.6088711118452) bank152461 +152462 POINT(42.07714769067623 -87.5876053144803) bank152462 +152463 POINT(41.92908289893068 -88.00205621718118) bank152463 +152464 POINT(42.33726493062693 -86.80667967046865) bank152464 +152465 POINT(41.55408367851474 -88.22368879682264) bank152465 +152466 POINT(42.54605335069809 -87.38286708370498) bank152466 +152467 POINT(42.51408517825167 -86.85364985518801) bank152467 +152468 POINT(41.05223424278215 -87.38742766785114) bank152468 +152469 POINT(41.39178310791623 -87.01807803111687) bank152469 +152470 POINT(41.55812720024428 -87.95585410449559) bank152470 +152471 POINT(42.686749509290635 -86.75106323317328) bank152471 +152472 POINT(41.497246512942816 -87.70376384097949) bank152472 +152473 POINT(41.97898322753 -87.312759992452) bank152473 +152474 POINT(42.25571772237589 -87.22774803243892) bank152474 +152475 POINT(42.288246357110275 -86.6868427242385) bank152475 +152476 POINT(41.20852924390327 -87.48983280405801) bank152476 +152477 POINT(42.14696025198434 -86.92173463468521) bank152477 +152478 POINT(41.571439641784345 -87.37411064877145) bank152478 +152479 POINT(42.58609796947267 -88.1312318414452) bank152479 +152480 POINT(42.0963257526882 -86.97063999303381) bank152480 +152481 POINT(42.7195960700502 -88.19842662003425) bank152481 +152482 POINT(40.99679637180309 -88.37359754511537) bank152482 +152483 POINT(40.88276898812154 -87.59986017736358) bank152483 +152484 POINT(41.735549610194106 -88.52118207904377) bank152484 +152485 POINT(41.83769610192479 -88.3442098838559) bank152485 +152486 POINT(42.70331713468844 -87.2142018868099) bank152486 +152487 POINT(41.24008588409692 -87.64206580096676) bank152487 +152488 POINT(42.642439494554424 -87.35467101170661) bank152488 +152489 POINT(42.25844191039243 -88.10823575953789) bank152489 +152490 POINT(42.788807789979714 -87.38358018743311) bank152490 +152491 POINT(41.40915135376355 -86.82913853896939) bank152491 +152492 POINT(41.227155252234844 -86.84996410366549) bank152492 +152493 POINT(41.36673126158127 -88.02703246394962) bank152493 +152494 POINT(41.62139875435127 -87.67389097285799) bank152494 +152495 POINT(42.68405180726309 -88.37680819933766) bank152495 +152496 POINT(41.917866222373064 -88.42519979238521) bank152496 +152497 POINT(41.41799209079609 -87.11346015554138) bank152497 +152498 POINT(41.60742528577516 -87.54077841072815) bank152498 +152499 POINT(42.051260723628324 -87.49203623676674) bank152499 +152500 POINT(42.42880058674632 -88.46237292641834) bank152500 +152501 POINT(42.215359296381756 -88.38171387903618) bank152501 +152502 POINT(42.240195809127684 -87.28175620751328) bank152502 +152503 POINT(41.174114857545675 -86.82104956399937) bank152503 +152504 POINT(41.110077812187555 -87.35726097978771) bank152504 +152505 POINT(41.06747063615786 -86.80745353651739) bank152505 +152506 POINT(41.277498997462544 -87.67947228459673) bank152506 +152507 POINT(41.47430294684621 -87.52414506039722) bank152507 +152508 POINT(42.67457852938258 -88.01889456984766) bank152508 +152509 POINT(42.24195872199303 -87.57563711180724) bank152509 +152510 POINT(42.03093741947031 -88.18035457472769) bank152510 +152511 POINT(41.44282060291764 -86.7140470142403) bank152511 +152512 POINT(42.390471653312154 -87.18274890098368) bank152512 +152513 POINT(41.303218011623606 -88.05165946057812) bank152513 +152514 POINT(42.075755507372186 -87.35607585846373) bank152514 +152515 POINT(41.901465870496374 -86.92955168788752) bank152515 +152516 POINT(42.86674384744528 -87.93524959669551) bank152516 +152517 POINT(41.42970584893365 -87.28954231492342) bank152517 +152518 POINT(42.42246660183695 -86.94069771074795) bank152518 +152519 POINT(42.80829133021444 -88.1568034631134) bank152519 +152520 POINT(40.988098759215944 -88.03855360558799) bank152520 +152521 POINT(42.69868163690964 -88.15595980764257) bank152521 +152522 POINT(41.12398638384328 -87.41636393958265) bank152522 +152523 POINT(42.037301406764236 -87.95489985794244) bank152523 +152524 POINT(41.004852686373695 -87.90560982499419) bank152524 +152525 POINT(41.612051738178266 -88.36039878077911) bank152525 +152526 POINT(41.58271225689774 -88.43532222538693) bank152526 +152527 POINT(42.738251084770866 -88.54389933550246) bank152527 +152528 POINT(41.1118448997049 -88.30350122482946) bank152528 +152529 POINT(42.2184166437373 -87.35694538353751) bank152529 +152530 POINT(42.5589486144721 -87.8978035214095) bank152530 +152531 POINT(40.969485797940585 -88.00404163669796) bank152531 +152532 POINT(42.82071230267047 -87.16272863368944) bank152532 +152533 POINT(41.75972594819539 -87.25650423782582) bank152533 +152534 POINT(42.12755997890386 -87.57292512619684) bank152534 +152535 POINT(41.794178809602585 -88.14879934125867) bank152535 +152536 POINT(41.85093592109418 -87.15266191231883) bank152536 +152537 POINT(41.874014737738825 -87.16771603236808) bank152537 +152538 POINT(41.97085341859306 -88.18778161361953) bank152538 +152539 POINT(41.39485590424952 -87.91396388350633) bank152539 +152540 POINT(41.434971117810434 -88.52224342440579) bank152540 +152541 POINT(42.66480097234831 -87.46036370229788) bank152541 +152542 POINT(41.12228536524953 -86.74101933709173) bank152542 +152543 POINT(41.7547829059118 -87.41134222337621) bank152543 +152544 POINT(42.467175412240636 -87.29825064584655) bank152544 +152545 POINT(41.106895798403116 -87.1844994758708) bank152545 +152546 POINT(42.79236803709923 -87.96778611811779) bank152546 +152547 POINT(42.83623729952965 -88.12338925570475) bank152547 +152548 POINT(41.64448097942176 -88.38218431189773) bank152548 +152549 POINT(41.26371132771594 -87.00673067020753) bank152549 +152550 POINT(42.37661473569169 -86.77353755261873) bank152550 +152551 POINT(41.71057609345469 -87.78339040195937) bank152551 +152552 POINT(42.45804512187169 -88.61365832791707) bank152552 +152553 POINT(41.014727810348326 -86.98308049799462) bank152553 +152554 POINT(42.16970374397029 -88.13303883271756) bank152554 +152555 POINT(42.20031387750765 -87.55257844399542) bank152555 +152556 POINT(41.51566303870339 -87.1299971931224) bank152556 +152557 POINT(42.4071432109837 -87.67577396472402) bank152557 +152558 POINT(42.15251643304743 -88.13648257423928) bank152558 +152559 POINT(41.33410618757673 -87.95321423691605) bank152559 +152560 POINT(41.094116566930886 -88.5046271997021) bank152560 +152561 POINT(41.46323367420687 -86.7378761964377) bank152561 +152562 POINT(42.56411329115653 -87.93863200425683) bank152562 +152563 POINT(42.43734597600017 -86.68476012038774) bank152563 +152564 POINT(42.77924498874211 -87.38920108831864) bank152564 +152565 POINT(42.786790849129424 -87.81556227858444) bank152565 +152566 POINT(41.033773052445106 -88.18163528179447) bank152566 +152567 POINT(42.491373221413866 -87.77398255635569) bank152567 +152568 POINT(41.01998138301782 -87.6958243277132) bank152568 +152569 POINT(40.88302762435341 -87.08604567937678) bank152569 +152570 POINT(41.11696676861203 -88.26784313492816) bank152570 +152571 POINT(41.403613014855544 -88.5054121150908) bank152571 +152572 POINT(40.90584962117834 -87.4384404285737) bank152572 +152573 POINT(40.956974227000615 -87.34197363656777) bank152573 +152574 POINT(42.61129662268768 -87.80511223477006) bank152574 +152575 POINT(42.15791245407126 -86.72691805953238) bank152575 +152576 POINT(41.29165206188067 -87.0565754469711) bank152576 +152577 POINT(42.62555023375473 -87.05958592460675) bank152577 +152578 POINT(42.739430505348935 -87.64817507547409) bank152578 +152579 POINT(42.51551436724854 -88.16511756584075) bank152579 +152580 POINT(41.609657166168866 -86.71970879857186) bank152580 +152581 POINT(41.77917189295002 -87.1957377593831) bank152581 +152582 POINT(41.67328997381352 -87.37560284453005) bank152582 +152583 POINT(42.11840101948766 -88.3191977511245) bank152583 +152584 POINT(41.594853134694446 -88.5309947120985) bank152584 +152585 POINT(41.126014152191885 -88.57359661070441) bank152585 +152586 POINT(41.88957111058413 -86.64382652295946) bank152586 +152587 POINT(41.8538165504634 -87.14493167040587) bank152587 +152588 POINT(41.58638643839015 -86.69851646857288) bank152588 +152589 POINT(41.23974919741099 -86.6705605239331) bank152589 +152590 POINT(42.712005686036434 -87.16130943790296) bank152590 +152591 POINT(42.86845160066921 -88.01764110964707) bank152591 +152592 POINT(42.4324725489951 -87.18815748392558) bank152592 +152593 POINT(41.592143245600106 -88.23394460500295) bank152593 +152594 POINT(41.25602783048485 -88.13100068396251) bank152594 +152595 POINT(41.84265904614919 -86.76022903231247) bank152595 +152596 POINT(40.926475171552354 -88.49196333941758) bank152596 +152597 POINT(42.791020269812165 -87.05340055378292) bank152597 +152598 POINT(41.58529542972212 -87.34904968354601) bank152598 +152599 POINT(41.08932788050648 -88.48253701794427) bank152599 +152600 POINT(41.28835728201782 -88.52570316254698) bank152600 +152601 POINT(41.02218305941778 -88.60895391186185) bank152601 +152602 POINT(41.634745739537024 -86.91832569604732) bank152602 +152603 POINT(41.34135839442062 -86.89829744543987) bank152603 +152604 POINT(41.11193572155628 -88.02389970738288) bank152604 +152605 POINT(41.75629706459164 -88.53069478942213) bank152605 +152606 POINT(41.60462031715338 -88.35886299363995) bank152606 +152607 POINT(42.43131265761109 -87.46849451196624) bank152607 +152608 POINT(41.1269786427691 -88.05699988993221) bank152608 +152609 POINT(41.680324370949876 -88.12722749440402) bank152609 +152610 POINT(41.69133107703301 -88.3028711738268) bank152610 +152611 POINT(42.79219477218417 -86.81339120502022) bank152611 +152612 POINT(42.771040008058975 -87.03057266557592) bank152612 +152613 POINT(42.054824600953886 -87.09100904603147) bank152613 +152614 POINT(42.62917728146774 -87.32025330027976) bank152614 +152615 POINT(41.781884171505496 -87.25040838822883) bank152615 +152616 POINT(41.86350583890654 -86.94583366752578) bank152616 +152617 POINT(41.04583713498612 -86.69106840518282) bank152617 +152618 POINT(41.1446947320658 -87.51428297017276) bank152618 +152619 POINT(42.730340116774094 -87.38001382954666) bank152619 +152620 POINT(42.48930214866189 -88.24545165391251) bank152620 +152621 POINT(41.35268592238555 -87.62826241004363) bank152621 +152622 POINT(41.72866703724565 -87.91875568070765) bank152622 +152623 POINT(41.622448250514466 -86.75485801569104) bank152623 +152624 POINT(42.26848411750468 -88.1434857068242) bank152624 +152625 POINT(41.01772135277693 -88.29887673812922) bank152625 +152626 POINT(42.62476816534787 -86.93971293504626) bank152626 +152627 POINT(41.54309442433935 -87.92670961421241) bank152627 +152628 POINT(41.47433555632942 -87.77853770674048) bank152628 +152629 POINT(41.85648638880742 -87.45434334532395) bank152629 +152630 POINT(41.98745722770879 -87.08085177441303) bank152630 +152631 POINT(42.38560416788356 -87.52160155695353) bank152631 +152632 POINT(41.2130538026403 -87.190063167692) bank152632 +152633 POINT(42.1359973065675 -87.49916557017978) bank152633 +152634 POINT(41.16936842275348 -88.16211140599641) bank152634 +152635 POINT(42.70181137196715 -88.01283413625288) bank152635 +152636 POINT(42.834264700948296 -88.0546717374791) bank152636 +152637 POINT(42.14133324966125 -88.37560718211837) bank152637 +152638 POINT(42.307634166807645 -86.69771297636359) bank152638 +152639 POINT(42.873121141283754 -88.34327892487252) bank152639 +152640 POINT(41.07388666897321 -87.10894262349927) bank152640 +152641 POINT(42.65214578426297 -88.56068486803315) bank152641 +152642 POINT(42.178276596531006 -86.79193229664274) bank152642 +152643 POINT(42.241246551355495 -87.54951654728772) bank152643 +152644 POINT(41.97123957786069 -87.76864094490098) bank152644 +152645 POINT(41.3897767089199 -87.14609095785049) bank152645 +152646 POINT(41.46710502840186 -88.03887314690016) bank152646 +152647 POINT(42.284389775036246 -87.12273749307325) bank152647 +152648 POINT(42.2597945628377 -88.31073919457161) bank152648 +152649 POINT(41.25168632330065 -88.15523216446086) bank152649 +152650 POINT(42.41522984945739 -86.89651832295978) bank152650 +152651 POINT(42.86108520814367 -86.83381530017756) bank152651 +152652 POINT(41.95484973462458 -87.87333592285997) bank152652 +152653 POINT(42.190396232335075 -88.09164343644562) bank152653 +152654 POINT(41.63891654676057 -87.89558676617887) bank152654 +152655 POINT(42.499796958777104 -88.54939993805907) bank152655 +152656 POINT(42.81452171154356 -87.1015438387029) bank152656 +152657 POINT(41.665381667786754 -88.13993307809714) bank152657 +152658 POINT(42.841264174026364 -87.80314925072176) bank152658 +152659 POINT(42.65744874033427 -86.92961350314414) bank152659 +152660 POINT(41.85539127625782 -86.79945294227706) bank152660 +152661 POINT(41.76772056706084 -88.24085022225098) bank152661 +152662 POINT(42.01048224036847 -88.14439721045105) bank152662 +152663 POINT(40.904222160609244 -88.34749770975873) bank152663 +152664 POINT(41.485635022696115 -88.61443430360073) bank152664 +152665 POINT(41.084623622297535 -87.95101315835205) bank152665 +152666 POINT(42.78094056370944 -87.23425971487207) bank152666 +152667 POINT(41.62185921565729 -87.67546741086085) bank152667 +152668 POINT(41.356837624371266 -88.0217140981294) bank152668 +152669 POINT(41.76836195484327 -87.13438997411043) bank152669 +152670 POINT(42.35987247181427 -88.61831238907038) bank152670 +152671 POINT(42.82202544669018 -88.39409213020059) bank152671 +152672 POINT(42.11096688059487 -87.88624658963971) bank152672 +152673 POINT(41.31151243103151 -88.59532046361903) bank152673 +152674 POINT(42.47815853313323 -87.69961234051593) bank152674 +152675 POINT(41.565405318671544 -87.47190323860123) bank152675 +152676 POINT(42.16559630591372 -88.14613164075745) bank152676 +152677 POINT(42.113694033028985 -87.60824100567457) bank152677 +152678 POINT(41.98071225415325 -87.88004424022665) bank152678 +152679 POINT(41.85834808534992 -86.68799754510924) bank152679 +152680 POINT(41.058948052270914 -87.16598961545907) bank152680 +152681 POINT(42.49324580272324 -87.9634961027772) bank152681 +152682 POINT(41.7764825406727 -86.68907096683246) bank152682 +152683 POINT(40.987560209874275 -88.42161109341436) bank152683 +152684 POINT(42.6948871079011 -88.618570556474) bank152684 +152685 POINT(41.765233726403956 -88.02145917572255) bank152685 +152686 POINT(41.25319446418388 -87.15720192615474) bank152686 +152687 POINT(41.64384363647001 -87.16224734685937) bank152687 +152688 POINT(41.786205002620925 -88.20114942028607) bank152688 +152689 POINT(41.05167538641241 -88.35343514613636) bank152689 +152690 POINT(41.547763241431355 -87.58123440564316) bank152690 +152691 POINT(42.58977660377287 -86.79670799280758) bank152691 +152692 POINT(41.88570908703082 -88.15373075175113) bank152692 +152693 POINT(41.21333118231612 -87.93649720919741) bank152693 +152694 POINT(41.700943825293976 -88.16577807391103) bank152694 +152695 POINT(42.438219027926046 -87.59385238993676) bank152695 +152696 POINT(41.61318068178383 -88.10879813548463) bank152696 +152697 POINT(41.172932658244974 -88.00209373229897) bank152697 +152698 POINT(42.44068922118033 -88.02097165221838) bank152698 +152699 POINT(41.17869317018169 -86.64025003449555) bank152699 +152700 POINT(42.655221276999235 -86.87488822322064) bank152700 +152701 POINT(42.11668175238134 -88.23109820436903) bank152701 +152702 POINT(42.01615931025105 -88.25177992037696) bank152702 +152703 POINT(41.42388915439606 -86.98140140399913) bank152703 +152704 POINT(41.28298946793582 -87.84946791206397) bank152704 +152705 POINT(41.19390538774125 -87.12521482226498) bank152705 +152706 POINT(41.7818797015149 -87.7187386297673) bank152706 +152707 POINT(42.4971417803589 -87.34906073364525) bank152707 +152708 POINT(41.017809023026274 -87.84042487724727) bank152708 +152709 POINT(41.55796651502152 -88.25400761554377) bank152709 +152710 POINT(42.217846078278754 -88.47217197635226) bank152710 +152711 POINT(41.214163686834006 -88.4925448146938) bank152711 +152712 POINT(42.85444322126683 -87.93042449063974) bank152712 +152713 POINT(42.78290559850741 -87.15449068941605) bank152713 +152714 POINT(42.421615433513246 -86.78518164452039) bank152714 +152715 POINT(41.9596657607489 -88.13076539255395) bank152715 +152716 POINT(42.669554384773285 -88.36680578437169) bank152716 +152717 POINT(42.57488526650064 -87.13499339349443) bank152717 +152718 POINT(42.48722770518249 -86.98338211815187) bank152718 +152719 POINT(40.938101183480974 -86.86957560485119) bank152719 +152720 POINT(41.888954156652474 -87.07213256584237) bank152720 +152721 POINT(41.568807142539015 -88.50344228273715) bank152721 +152722 POINT(41.80241652622118 -87.87500329230242) bank152722 +152723 POINT(40.979880744981486 -86.89722340847734) bank152723 +152724 POINT(42.346815329106086 -86.92675799042213) bank152724 +152725 POINT(42.36880355641247 -87.03848411641538) bank152725 +152726 POINT(41.13461557735188 -86.72580044794947) bank152726 +152727 POINT(41.08086777552809 -88.40193926035091) bank152727 +152728 POINT(42.33203293952141 -87.299109320411) bank152728 +152729 POINT(41.65929752197607 -87.00517283685572) bank152729 +152730 POINT(41.82604245654758 -87.38516661025622) bank152730 +152731 POINT(41.68542612339 -87.22139788988012) bank152731 +152732 POINT(40.89029089747236 -87.34615682893147) bank152732 +152733 POINT(41.78486547780309 -87.21232929111605) bank152733 +152734 POINT(41.21396551479315 -87.68614043524316) bank152734 +152735 POINT(42.52928981370493 -87.08734884511719) bank152735 +152736 POINT(42.09870216786361 -86.83442661132706) bank152736 +152737 POINT(41.391565970786296 -88.4755996094912) bank152737 +152738 POINT(41.45145988824357 -88.57903471986846) bank152738 +152739 POINT(42.68544287596428 -87.66773209186773) bank152739 +152740 POINT(42.604110265626076 -88.14320920827606) bank152740 +152741 POINT(41.770169197647576 -87.13564564228562) bank152741 +152742 POINT(42.38561561384966 -86.72557964077397) bank152742 +152743 POINT(41.618948982552574 -87.31910843082811) bank152743 +152744 POINT(42.253053175631095 -88.00958087602159) bank152744 +152745 POINT(41.12738320849709 -87.22858035463284) bank152745 +152746 POINT(42.1378838220342 -87.19054395042188) bank152746 +152747 POINT(42.838622654429834 -87.43369404849734) bank152747 +152748 POINT(42.49821026136398 -88.5740244422459) bank152748 +152749 POINT(41.45418560655721 -88.54732526970295) bank152749 +152750 POINT(40.96739843633002 -87.36891810827285) bank152750 +152751 POINT(41.82759787960608 -87.93301244474503) bank152751 +152752 POINT(41.82233355129015 -87.89068185351842) bank152752 +152753 POINT(41.731276717346276 -87.48234672010773) bank152753 +152754 POINT(42.11322835465414 -88.1483444972502) bank152754 +152755 POINT(41.56644916975663 -87.57853117693045) bank152755 +152756 POINT(41.133501800481895 -88.62031483670093) bank152756 +152757 POINT(41.07344450772085 -87.15056936510348) bank152757 +152758 POINT(41.57210857134472 -88.36540839346024) bank152758 +152759 POINT(40.926175249567635 -87.88061546817777) bank152759 +152760 POINT(41.61806631680235 -87.43936893156695) bank152760 +152761 POINT(42.84352578202846 -87.43362361351977) bank152761 +152762 POINT(41.02296428432863 -86.63738525289587) bank152762 +152763 POINT(41.84261447817341 -87.15088729352799) bank152763 +152764 POINT(42.1561221114272 -88.09128872701962) bank152764 +152765 POINT(41.84853878527583 -86.84063511278019) bank152765 +152766 POINT(42.696793117528934 -86.7581462175149) bank152766 +152767 POINT(41.07259922436559 -88.01122914288543) bank152767 +152768 POINT(41.599168786269345 -88.18380118565138) bank152768 +152769 POINT(40.88180294638215 -86.76094404642143) bank152769 +152770 POINT(42.195035884684486 -87.9191475214548) bank152770 +152771 POINT(41.24178316527609 -87.68257001458021) bank152771 +152772 POINT(42.58763095311635 -87.2812130559263) bank152772 +152773 POINT(41.95908692701186 -86.63566636924149) bank152773 +152774 POINT(42.00062706088899 -88.05138557317824) bank152774 +152775 POINT(41.4451230250028 -88.31272366450034) bank152775 +152776 POINT(42.733121501771876 -88.240210729613) bank152776 +152777 POINT(40.89933128485308 -87.74636957759274) bank152777 +152778 POINT(41.65701078033317 -88.55254767403333) bank152778 +152779 POINT(42.734560383589574 -87.11451007383125) bank152779 +152780 POINT(41.55302446015145 -87.77456560511813) bank152780 +152781 POINT(41.76539857574617 -88.05813455426457) bank152781 +152782 POINT(41.12816716806778 -87.4879604273782) bank152782 +152783 POINT(41.2227107005344 -88.03961011260937) bank152783 +152784 POINT(40.95815276877847 -86.88208177346853) bank152784 +152785 POINT(42.80403150979503 -88.08582921527727) bank152785 +152786 POINT(41.264882574540124 -87.62781598958388) bank152786 +152787 POINT(40.96504126445195 -86.85632853854186) bank152787 +152788 POINT(42.44748453593752 -88.2427883589202) bank152788 +152789 POINT(42.181355848001 -87.36598343658214) bank152789 +152790 POINT(40.87918542069184 -88.23454441762641) bank152790 +152791 POINT(41.85689091749567 -87.96407783821456) bank152791 +152792 POINT(40.900603838564194 -87.24516686591247) bank152792 +152793 POINT(42.248022960724334 -87.67189130460162) bank152793 +152794 POINT(41.0310421870478 -87.56351774306933) bank152794 +152795 POINT(41.45282859875624 -88.60903227677447) bank152795 +152796 POINT(42.57254618258041 -87.1386153338372) bank152796 +152797 POINT(41.970721416327486 -88.57562415748995) bank152797 +152798 POINT(42.098948386676476 -86.87788922082278) bank152798 +152799 POINT(42.692999750101805 -86.82620030461877) bank152799 +152800 POINT(42.26826956391184 -86.99932744436636) bank152800 +152801 POINT(40.95483185460916 -87.40855631719988) bank152801 +152802 POINT(42.23573480290328 -87.56020242327153) bank152802 +152803 POINT(41.399627618865786 -87.53346986898177) bank152803 +152804 POINT(41.63436283551327 -87.43501521121185) bank152804 +152805 POINT(42.640490449798406 -88.0241264686472) bank152805 +152806 POINT(42.83400907761807 -87.94579215892583) bank152806 +152807 POINT(41.81529880312343 -87.76100336240697) bank152807 +152808 POINT(40.88277770528092 -87.21183278185427) bank152808 +152809 POINT(41.22673353278689 -88.34556183822397) bank152809 +152810 POINT(41.036140835439994 -86.94847307588644) bank152810 +152811 POINT(42.34011419716343 -88.36728003493208) bank152811 +152812 POINT(41.795910893542676 -87.1680851403267) bank152812 +152813 POINT(42.06545674386151 -88.1908184010833) bank152813 +152814 POINT(41.56051520558212 -86.83282075149152) bank152814 +152815 POINT(41.39566127579969 -87.85511866843737) bank152815 +152816 POINT(40.94863508920275 -87.378166888534) bank152816 +152817 POINT(41.71956578635186 -87.81803027793046) bank152817 +152818 POINT(42.6356458130267 -87.30533876833225) bank152818 +152819 POINT(42.38257064607746 -88.38333071513313) bank152819 +152820 POINT(42.32552766783318 -87.63227606917407) bank152820 +152821 POINT(41.606525026363435 -87.38206249461687) bank152821 +152822 POINT(41.5078041599553 -88.42505222380508) bank152822 +152823 POINT(42.23336414560442 -88.13099209108395) bank152823 +152824 POINT(42.48300113074232 -87.06545018713638) bank152824 +152825 POINT(41.14068643577517 -87.51275886998668) bank152825 +152826 POINT(40.96981868430694 -86.92352135390324) bank152826 +152827 POINT(40.9319781316175 -86.8744561446561) bank152827 +152828 POINT(41.44144933279031 -88.38313179816097) bank152828 +152829 POINT(41.34956322387594 -87.59338417900513) bank152829 +152830 POINT(41.480574813638064 -86.99415022481686) bank152830 +152831 POINT(40.89540010218666 -86.74489080132615) bank152831 +152832 POINT(40.98470934192351 -87.05051230351062) bank152832 +152833 POINT(42.068673146782686 -87.5384471461732) bank152833 +152834 POINT(41.931027080076014 -88.01430224433804) bank152834 +152835 POINT(41.84780200040859 -88.10843723999271) bank152835 +152836 POINT(41.87981704377036 -87.59085860684124) bank152836 +152837 POINT(41.7709553181521 -88.48329190082914) bank152837 +152838 POINT(42.796081351128 -88.39171789462452) bank152838 +152839 POINT(41.91367803202185 -87.01019530192761) bank152839 +152840 POINT(41.69239163969413 -88.60971914520012) bank152840 +152841 POINT(42.491305396728634 -86.70071830094479) bank152841 +152842 POINT(42.43213286021873 -87.73079846387714) bank152842 +152843 POINT(41.53868728537359 -87.7739332196601) bank152843 +152844 POINT(41.609976506273725 -86.86905607663897) bank152844 +152845 POINT(41.321578889903556 -88.6225149849172) bank152845 +152846 POINT(42.74388624254962 -87.00261547049514) bank152846 +152847 POINT(41.12148655571066 -88.49410202406683) bank152847 +152848 POINT(42.71162949628649 -87.41542375329993) bank152848 +152849 POINT(42.8294978178245 -88.4565259162064) bank152849 +152850 POINT(42.04480411314531 -87.54931324053302) bank152850 +152851 POINT(41.13762617140742 -87.64326149078742) bank152851 +152852 POINT(41.70926007168827 -86.95746113153736) bank152852 +152853 POINT(41.50587963714209 -86.92358869451706) bank152853 +152854 POINT(41.829216455227154 -87.68820982807227) bank152854 +152855 POINT(41.85199306105383 -88.49134572412089) bank152855 +152856 POINT(41.06933226925221 -87.88181747514811) bank152856 +152857 POINT(40.880012843593725 -88.4515171176416) bank152857 +152858 POINT(42.66087006009554 -87.85902291190507) bank152858 +152859 POINT(42.73268808256848 -88.34694474515229) bank152859 +152860 POINT(42.78240379152777 -87.19560311027247) bank152860 +152861 POINT(41.91462463610157 -87.45042994369749) bank152861 +152862 POINT(42.876621537317455 -88.4353327961517) bank152862 +152863 POINT(42.54908966039604 -87.58115527487017) bank152863 +152864 POINT(42.122934791196094 -87.64982235061096) bank152864 +152865 POINT(40.97438225377392 -87.89121591413773) bank152865 +152866 POINT(41.15634779318271 -88.32620428177655) bank152866 +152867 POINT(40.94028778952922 -86.69896547928639) bank152867 +152868 POINT(41.39994862739454 -87.6130347622132) bank152868 +152869 POINT(42.18532231667441 -87.9387589432961) bank152869 +152870 POINT(42.66837217197131 -87.92298011260094) bank152870 +152871 POINT(42.01801488965751 -88.43182482925093) bank152871 +152872 POINT(41.51966976668313 -87.87653510007976) bank152872 +152873 POINT(41.757325617515406 -87.90346184936544) bank152873 +152874 POINT(41.99663125324155 -87.76749615561774) bank152874 +152875 POINT(41.77554394301609 -87.1834353990894) bank152875 +152876 POINT(40.99241003384576 -87.6194644662118) bank152876 +152877 POINT(41.15973976550908 -88.56686497755265) bank152877 +152878 POINT(41.61654566802039 -87.62979038997501) bank152878 +152879 POINT(42.045750818530024 -87.24929413269433) bank152879 +152880 POINT(42.52620753130909 -86.74822365092622) bank152880 +152881 POINT(42.23851585589283 -87.80938867100372) bank152881 +152882 POINT(42.29099948735616 -88.5723852917148) bank152882 +152883 POINT(42.80124955168977 -86.94169466559865) bank152883 +152884 POINT(41.28248831657979 -87.66866802646038) bank152884 +152885 POINT(41.201581972867444 -87.80690809968328) bank152885 +152886 POINT(42.437539538736715 -87.70697048737468) bank152886 +152887 POINT(41.127552426002424 -88.217488542926) bank152887 +152888 POINT(41.454291200518824 -88.36199614774021) bank152888 +152889 POINT(41.016783037245624 -87.59217876259775) bank152889 +152890 POINT(42.27275448958133 -88.42435546075168) bank152890 +152891 POINT(42.162470193620734 -87.67368467669914) bank152891 +152892 POINT(41.412755778120264 -87.83680032784457) bank152892 +152893 POINT(41.680940887194545 -88.38059205694452) bank152893 +152894 POINT(42.86775569074908 -86.93635346424229) bank152894 +152895 POINT(41.56841219083419 -87.52825653665857) bank152895 +152896 POINT(41.72460711195393 -87.93473123009294) bank152896 +152897 POINT(41.09102096728639 -87.40537675814505) bank152897 +152898 POINT(41.44458592981226 -88.56949507225175) bank152898 +152899 POINT(41.718341461164904 -87.71935319037777) bank152899 +152900 POINT(41.56868967514608 -87.74108964056717) bank152900 +152901 POINT(41.790999300569595 -88.25979960156057) bank152901 +152902 POINT(41.78483901834489 -87.61398581454844) bank152902 +152903 POINT(41.233311209815 -87.09339046496802) bank152903 +152904 POINT(42.13046183470551 -88.05900870321162) bank152904 +152905 POINT(41.938863519878005 -86.75410891085211) bank152905 +152906 POINT(41.21664523803693 -87.86642850472454) bank152906 +152907 POINT(42.44469409122428 -88.20239134402674) bank152907 +152908 POINT(42.5177813250139 -87.72004466873437) bank152908 +152909 POINT(40.93705265796032 -88.32529866166661) bank152909 +152910 POINT(41.69865308407827 -86.73771153588534) bank152910 +152911 POINT(42.0432503055931 -88.31089938697052) bank152911 +152912 POINT(41.16971638564329 -87.71688607083665) bank152912 +152913 POINT(42.34814287959058 -88.50776559444168) bank152913 +152914 POINT(42.52933607690988 -86.9843392124031) bank152914 +152915 POINT(41.385016423807386 -88.15773810283349) bank152915 +152916 POINT(41.423567593032054 -88.29864234668749) bank152916 +152917 POINT(42.65688751325992 -87.3026696235771) bank152917 +152918 POINT(41.226522899033654 -88.13347391525781) bank152918 +152919 POINT(41.237818653264654 -88.54614979189935) bank152919 +152920 POINT(42.188015549417656 -87.61974358633861) bank152920 +152921 POINT(42.268848699827316 -87.0885221883876) bank152921 +152922 POINT(41.03376231770895 -87.47629070130681) bank152922 +152923 POINT(42.333387830597665 -87.3776214863263) bank152923 +152924 POINT(42.28117351044433 -88.46026665907802) bank152924 +152925 POINT(41.89324310438944 -87.91605284822707) bank152925 +152926 POINT(42.41583153406176 -88.40626061822356) bank152926 +152927 POINT(42.02900213934671 -88.50618685192521) bank152927 +152928 POINT(41.492322956883086 -87.9574325201369) bank152928 +152929 POINT(41.531329019356754 -88.23807000517628) bank152929 +152930 POINT(41.95056749917848 -88.15494973113957) bank152930 +152931 POINT(42.23413662959767 -87.4455518558795) bank152931 +152932 POINT(41.87851592802059 -87.32466238494014) bank152932 +152933 POINT(42.33058210355886 -87.32964106728953) bank152933 +152934 POINT(42.73785772287689 -87.44711997968477) bank152934 +152935 POINT(40.977213012382364 -87.52851761815673) bank152935 +152936 POINT(41.57764793419202 -87.83559018109585) bank152936 +152937 POINT(42.3375341017255 -87.48338089057324) bank152937 +152938 POINT(42.5742030852681 -87.22098728266874) bank152938 +152939 POINT(42.0162177180318 -86.65460028237736) bank152939 +152940 POINT(42.47922967235598 -88.39700056114825) bank152940 +152941 POINT(41.07729471922984 -88.27925923062975) bank152941 +152942 POINT(42.085108865976004 -86.86889039825478) bank152942 +152943 POINT(41.800164964416645 -86.88650640778512) bank152943 +152944 POINT(42.506035226543396 -88.04408913530814) bank152944 +152945 POINT(41.862085247220215 -87.25336660956339) bank152945 +152946 POINT(42.48422043007021 -88.4753788807095) bank152946 +152947 POINT(41.685067791941655 -87.41855143856019) bank152947 +152948 POINT(42.42691926214063 -87.09479820314698) bank152948 +152949 POINT(42.43795685505687 -88.40626543902835) bank152949 +152950 POINT(42.54726321011616 -88.54930500391642) bank152950 +152951 POINT(42.09991015872468 -87.58473292141171) bank152951 +152952 POINT(42.77879698902451 -87.04869333060616) bank152952 +152953 POINT(41.622519160506236 -87.44464611938407) bank152953 +152954 POINT(42.24680427091351 -87.82724052015267) bank152954 +152955 POINT(42.39005803156966 -87.22149297423226) bank152955 +152956 POINT(42.299093683659535 -86.75360976591666) bank152956 +152957 POINT(41.94631147570422 -88.61846178902402) bank152957 +152958 POINT(42.4482008503357 -88.57895931942986) bank152958 +152959 POINT(42.62768059138692 -87.44006202512773) bank152959 +152960 POINT(42.25052510888136 -87.02177242673532) bank152960 +152961 POINT(41.715835984972394 -86.84655367784437) bank152961 +152962 POINT(42.46971373343987 -88.54997788653779) bank152962 +152963 POINT(42.24965314673941 -88.22402003934039) bank152963 +152964 POINT(41.79425057585039 -86.65533516846534) bank152964 +152965 POINT(41.89630488454777 -86.93176149366339) bank152965 +152966 POINT(41.23911222799211 -87.60941954458995) bank152966 +152967 POINT(41.344352857725 -87.87903694118955) bank152967 +152968 POINT(42.87464490326363 -88.28157442245032) bank152968 +152969 POINT(42.349690366145204 -87.36825581809104) bank152969 +152970 POINT(41.433721805187425 -86.77854479791094) bank152970 +152971 POINT(41.315936631042305 -87.73414392480619) bank152971 +152972 POINT(42.28820329365572 -87.2617218100786) bank152972 +152973 POINT(41.86595486753777 -87.97113525217905) bank152973 +152974 POINT(41.47857305571783 -87.0873663534135) bank152974 +152975 POINT(41.62391038792645 -88.13238303607292) bank152975 +152976 POINT(40.88267461136086 -88.01495302706884) bank152976 +152977 POINT(40.89331521134394 -87.14007226117516) bank152977 +152978 POINT(42.19739895346094 -88.07864167351238) bank152978 +152979 POINT(42.423278704245455 -87.2668198650848) bank152979 +152980 POINT(42.53692100742135 -87.94498211792649) bank152980 +152981 POINT(40.97029464368481 -87.31749090959572) bank152981 +152982 POINT(42.07884015495819 -87.5952543834926) bank152982 +152983 POINT(42.67251371821323 -87.22804848109406) bank152983 +152984 POINT(42.12417401797526 -88.431532253102) bank152984 +152985 POINT(41.58574646065692 -87.66772448129868) bank152985 +152986 POINT(42.42490173172196 -87.16903364002967) bank152986 +152987 POINT(42.16641417362665 -88.06880011209594) bank152987 +152988 POINT(42.40625093846314 -87.39007956663107) bank152988 +152989 POINT(41.51739918380148 -87.88364542235212) bank152989 +152990 POINT(42.29185757020865 -87.18544601690455) bank152990 +152991 POINT(42.76224784306779 -88.31219775855753) bank152991 +152992 POINT(42.759885172242335 -87.35675065967558) bank152992 +152993 POINT(41.577465733265605 -86.75238301225967) bank152993 +152994 POINT(42.07101180693668 -86.73749835298486) bank152994 +152995 POINT(42.70249933176945 -88.4438105081077) bank152995 +152996 POINT(41.78504929108995 -86.64147657510225) bank152996 +152997 POINT(41.807906903212256 -87.38444436929841) bank152997 +152998 POINT(41.813763390536096 -87.33063513353785) bank152998 +152999 POINT(42.330464342115405 -88.28615122183605) bank152999 +153000 POINT(42.135422634214926 -87.19245943756243) bank153000 +153001 POINT(41.04872005711971 -87.2642176709399) bank153001 +153002 POINT(42.1259732750579 -88.19496705656046) bank153002 +153003 POINT(40.98529477605316 -88.00902709505395) bank153003 +153004 POINT(41.726296099308456 -87.35860335739882) bank153004 +153005 POINT(42.0013175622073 -87.1171781067791) bank153005 +153006 POINT(42.750373444356825 -87.37967340821065) bank153006 +153007 POINT(42.80950566211225 -86.78964919583363) bank153007 +153008 POINT(41.305808827417685 -87.51919491438238) bank153008 +153009 POINT(41.66036383634605 -88.00988436909189) bank153009 +153010 POINT(42.81733157171453 -88.50148654534071) bank153010 +153011 POINT(41.05066704939909 -87.8899170932583) bank153011 +153012 POINT(42.19305242919682 -87.86268322951483) bank153012 +153013 POINT(42.78715258227354 -87.52601028523284) bank153013 +153014 POINT(42.63470253802269 -87.14049492567234) bank153014 +153015 POINT(42.833031832344595 -87.87964475873278) bank153015 +153016 POINT(42.65407968918425 -88.23849274728536) bank153016 +153017 POINT(41.22539740097424 -88.34890591345454) bank153017 +153018 POINT(40.967823554348215 -88.55387777265484) bank153018 +153019 POINT(41.49001899482502 -87.61392644223314) bank153019 +153020 POINT(42.27472950632484 -86.78009832548355) bank153020 +153021 POINT(42.0973725210594 -87.86855350463075) bank153021 +153022 POINT(40.90536031047078 -88.48276314300337) bank153022 +153023 POINT(42.400253717041146 -86.87836919731446) bank153023 +153024 POINT(42.35828368634305 -86.89302832366182) bank153024 +153025 POINT(42.30379695427929 -87.16943010468874) bank153025 +153026 POINT(41.14695367457363 -88.18990124519236) bank153026 +153027 POINT(42.86100636863851 -88.41116976388761) bank153027 +153028 POINT(40.994071989075856 -88.46846367547462) bank153028 +153029 POINT(41.853516496050695 -88.45798274138403) bank153029 +153030 POINT(42.75497894690395 -87.25440116628593) bank153030 +153031 POINT(41.64547999452032 -88.62464454695315) bank153031 +153032 POINT(42.089199051828544 -87.6175879257496) bank153032 +153033 POINT(42.12772380165447 -87.40907395933968) bank153033 +153034 POINT(41.72199482379432 -88.05276658508926) bank153034 +153035 POINT(42.42920539719354 -87.46220613620451) bank153035 +153036 POINT(40.9037433544199 -87.02552526611032) bank153036 +153037 POINT(41.317411932331765 -87.9092520216578) bank153037 +153038 POINT(42.04515127483348 -87.28557799813537) bank153038 +153039 POINT(42.35831053608576 -86.74884489553048) bank153039 +153040 POINT(42.51476802771698 -87.38777727589496) bank153040 +153041 POINT(41.79244882563909 -88.33103144050209) bank153041 +153042 POINT(41.87156745041859 -87.43103408956526) bank153042 +153043 POINT(42.425714288323285 -87.99518927728863) bank153043 +153044 POINT(41.82627781788838 -88.44469452067555) bank153044 +153045 POINT(41.52012024725312 -88.44918265369628) bank153045 +153046 POINT(41.45823544036599 -87.50731349558977) bank153046 +153047 POINT(41.24407987207519 -88.36518855355058) bank153047 +153048 POINT(42.71211272611734 -86.82739909319055) bank153048 +153049 POINT(42.0415803049996 -86.66084340788052) bank153049 +153050 POINT(42.02559251239574 -87.57412250263218) bank153050 +153051 POINT(41.949156054485115 -87.04498138049911) bank153051 +153052 POINT(41.49897185293152 -86.70500920809168) bank153052 +153053 POINT(41.44966178630295 -87.83558952734886) bank153053 +153054 POINT(41.1373102411055 -86.78946586548105) bank153054 +153055 POINT(42.815127414576004 -87.65999283389942) bank153055 +153056 POINT(42.7247333709735 -87.25050623031662) bank153056 +153057 POINT(42.73903878319259 -87.47987419891042) bank153057 +153058 POINT(42.617793527700435 -88.25400673712875) bank153058 +153059 POINT(42.36146315024059 -88.37197129944377) bank153059 +153060 POINT(40.9931357704356 -88.0805296550662) bank153060 +153061 POINT(41.240684341813406 -87.09878798489662) bank153061 +153062 POINT(41.30107929975218 -86.96282581355248) bank153062 +153063 POINT(42.75280978624321 -87.97299405202622) bank153063 +153064 POINT(42.05443949459996 -87.72245114678337) bank153064 +153065 POINT(42.364196405642474 -86.8957397271905) bank153065 +153066 POINT(42.71675340841646 -87.24633922854416) bank153066 +153067 POINT(41.00673227917014 -87.5534076736748) bank153067 +153068 POINT(42.586110403866265 -87.21920037931743) bank153068 +153069 POINT(42.36093929484365 -88.37393521723617) bank153069 +153070 POINT(42.797729187780334 -87.51015601382754) bank153070 +153071 POINT(41.15123555158379 -87.41418558047766) bank153071 +153072 POINT(42.03562749632199 -88.16464591395079) bank153072 +153073 POINT(41.049516158234475 -88.12796239206338) bank153073 +153074 POINT(41.29760930437426 -88.25285002872171) bank153074 +153075 POINT(41.17381582416053 -87.25542767815405) bank153075 +153076 POINT(41.1683064883018 -86.73968523471828) bank153076 +153077 POINT(41.98049803162822 -88.59781525493021) bank153077 +153078 POINT(41.73891514331459 -88.38153287679411) bank153078 +153079 POINT(41.3092261076397 -87.59736216351683) bank153079 +153080 POINT(40.92151521828904 -87.897912952532) bank153080 +153081 POINT(41.460130428258125 -86.79209898387447) bank153081 +153082 POINT(41.235333357373456 -88.11333121283201) bank153082 +153083 POINT(41.18815385795617 -87.5745399942052) bank153083 +153084 POINT(41.49080488554596 -88.48957374026162) bank153084 +153085 POINT(41.73097277735274 -88.6194222537722) bank153085 +153086 POINT(40.94143531404642 -87.14970790006198) bank153086 +153087 POINT(41.236620846505545 -87.31517332531816) bank153087 +153088 POINT(42.255698415015544 -86.82465551744923) bank153088 +153089 POINT(40.92801385795185 -86.94319263887877) bank153089 +153090 POINT(41.868123470111755 -86.63306830192926) bank153090 +153091 POINT(42.36755318316891 -87.2115201305823) bank153091 +153092 POINT(41.65725956259733 -87.3586188181794) bank153092 +153093 POINT(42.27102533146452 -86.77628466243424) bank153093 +153094 POINT(42.39710009575776 -87.05568417090599) bank153094 +153095 POINT(41.250353570845135 -88.58638305152428) bank153095 +153096 POINT(42.529795016029 -87.13243443563977) bank153096 +153097 POINT(42.47061262400243 -87.45036833266234) bank153097 +153098 POINT(42.87565697061608 -86.88410300881722) bank153098 +153099 POINT(42.68099150587858 -87.56834134834646) bank153099 +153100 POINT(42.36778508971699 -88.26518204704679) bank153100 +153101 POINT(41.83816509807715 -87.85564553934314) bank153101 +153102 POINT(41.151026197782066 -87.07895906214785) bank153102 +153103 POINT(42.86188861238888 -86.65029800333618) bank153103 +153104 POINT(41.79022668395043 -88.21711707303139) bank153104 +153105 POINT(42.54023144214622 -87.59606359834827) bank153105 +153106 POINT(41.071546123479116 -87.91486607203797) bank153106 +153107 POINT(41.35832871689374 -87.54986747118014) bank153107 +153108 POINT(41.425021298867456 -87.71700312165557) bank153108 +153109 POINT(41.75433907097172 -88.23827172629818) bank153109 +153110 POINT(41.60825533892252 -88.09987468049869) bank153110 +153111 POINT(41.27883171423898 -87.6567613564336) bank153111 +153112 POINT(40.99803274438417 -88.01637478258928) bank153112 +153113 POINT(41.538274461904436 -87.77401939097619) bank153113 +153114 POINT(42.00565970422758 -87.85557002703787) bank153114 +153115 POINT(41.67485145307351 -86.91258931127865) bank153115 +153116 POINT(41.55108708102002 -87.54831860847264) bank153116 +153117 POINT(41.79154489725642 -88.15572031237934) bank153117 +153118 POINT(41.69356303427531 -87.6749295398149) bank153118 +153119 POINT(42.267075919703764 -87.95315493656254) bank153119 +153120 POINT(40.942016947812334 -88.39686123849195) bank153120 +153121 POINT(42.14418709931338 -87.68808633786907) bank153121 +153122 POINT(41.765506820781525 -88.38731796123612) bank153122 +153123 POINT(41.8704991059804 -87.03375969677525) bank153123 +153124 POINT(42.179384699967876 -88.04928150571673) bank153124 +153125 POINT(41.75233198430244 -86.78157537255157) bank153125 +153126 POINT(42.86121988956234 -86.91087312216852) bank153126 +153127 POINT(41.474489632655846 -86.77193974659066) bank153127 +153128 POINT(42.06636198534889 -87.77492521299452) bank153128 +153129 POINT(40.91583006648247 -87.94713090503217) bank153129 +153130 POINT(41.15000356189549 -87.217359098407) bank153130 +153131 POINT(41.45271599251733 -86.65307875928022) bank153131 +153132 POINT(42.841078362067975 -87.76781366399665) bank153132 +153133 POINT(41.05045233203185 -87.87569614433654) bank153133 +153134 POINT(41.561448218179216 -87.28791843674883) bank153134 +153135 POINT(42.65185312618564 -88.45153627166522) bank153135 +153136 POINT(41.92879986267662 -87.72331323030225) bank153136 +153137 POINT(41.13493742321688 -86.88189131664902) bank153137 +153138 POINT(41.091118320244156 -87.16961080647205) bank153138 +153139 POINT(41.244931129734056 -87.29396079819301) bank153139 +153140 POINT(41.475066828570775 -87.10590938697561) bank153140 +153141 POINT(41.89080718253082 -87.7862389082489) bank153141 +153142 POINT(41.773737717422236 -88.1711023503621) bank153142 +153143 POINT(42.294284583913175 -86.79481579615093) bank153143 +153144 POINT(42.08890994588822 -87.44728776872122) bank153144 +153145 POINT(40.97356299497022 -87.20352710323556) bank153145 +153146 POINT(42.034334916501095 -87.26432388933152) bank153146 +153147 POINT(42.00143704078454 -88.02091409046368) bank153147 +153148 POINT(41.10167754049263 -87.24824376529142) bank153148 +153149 POINT(41.134560359850944 -86.85082664292317) bank153149 +153150 POINT(41.30024036021564 -87.20390911144953) bank153150 +153151 POINT(41.20614516789757 -87.8170490544856) bank153151 +153152 POINT(42.80777621829508 -87.70870173819799) bank153152 +153153 POINT(42.0583187447513 -87.47297966757812) bank153153 +153154 POINT(42.65621528301719 -86.64680264088163) bank153154 +153155 POINT(41.785624981707095 -88.15704678273488) bank153155 +153156 POINT(41.67093474387583 -87.31281081572914) bank153156 +153157 POINT(42.44380878129308 -88.26145505771818) bank153157 +153158 POINT(41.00025128365234 -87.94791063246889) bank153158 +153159 POINT(42.09797850256415 -86.75350177669944) bank153159 +153160 POINT(42.831409055164286 -87.6086049782606) bank153160 +153161 POINT(41.77226976768877 -87.04248114580383) bank153161 +153162 POINT(41.14914646477614 -88.29753888311318) bank153162 +153163 POINT(42.25326478433381 -86.94864603106748) bank153163 +153164 POINT(41.082821297438805 -86.67967853841465) bank153164 +153165 POINT(41.8444449602711 -88.44647699849175) bank153165 +153166 POINT(40.99020617423755 -87.52046430175946) bank153166 +153167 POINT(41.26517824470622 -87.1730454255829) bank153167 +153168 POINT(42.15300202702455 -87.38925307901276) bank153168 +153169 POINT(40.92398422886905 -87.07035691254235) bank153169 +153170 POINT(42.027430985009005 -87.45331176598779) bank153170 +153171 POINT(41.0818752963114 -88.56575903118011) bank153171 +153172 POINT(42.387070880631086 -88.16126311971466) bank153172 +153173 POINT(42.57602957677714 -88.35846666235052) bank153173 +153174 POINT(41.532884845255325 -87.05721507564729) bank153174 +153175 POINT(41.79312895496312 -88.36108324596087) bank153175 +153176 POINT(42.42602118948234 -87.4044644871384) bank153176 +153177 POINT(42.63067620583754 -87.81439971641421) bank153177 +153178 POINT(42.24664218089934 -87.12993485780635) bank153178 +153179 POINT(41.04599009482867 -87.16035260402127) bank153179 +153180 POINT(42.034140710602394 -86.7714807994546) bank153180 +153181 POINT(41.68346845209283 -87.45102089452898) bank153181 +153182 POINT(41.7344741385449 -87.42252201523402) bank153182 +153183 POINT(42.80629272408371 -88.29647203729168) bank153183 +153184 POINT(42.72504004682707 -87.67118660576124) bank153184 +153185 POINT(41.94541369982042 -87.46959786521423) bank153185 +153186 POINT(42.71166465709685 -88.26008299064033) bank153186 +153187 POINT(41.22264875382336 -87.79448509808014) bank153187 +153188 POINT(41.32930363339647 -87.00711146281559) bank153188 +153189 POINT(41.14940349270611 -88.50268276366583) bank153189 +153190 POINT(42.60833437243731 -87.26693483756094) bank153190 +153191 POINT(41.98619875243354 -87.44944859714944) bank153191 +153192 POINT(41.65291705442933 -87.21296544786283) bank153192 +153193 POINT(41.548605452076394 -86.99440294519175) bank153193 +153194 POINT(41.12161953100993 -87.85664963715917) bank153194 +153195 POINT(42.453796498841214 -86.74738873140674) bank153195 +153196 POINT(41.138039725535904 -88.4186889792782) bank153196 +153197 POINT(41.1189401940753 -87.2044445246344) bank153197 +153198 POINT(40.89404511553486 -88.1540789958462) bank153198 +153199 POINT(41.595467445329916 -87.80668310445522) bank153199 +153200 POINT(42.03033478973458 -87.0524096135828) bank153200 +153201 POINT(41.75963628994698 -87.3984494304533) bank153201 +153202 POINT(41.578289465834196 -87.45417396852393) bank153202 +153203 POINT(42.22134397528707 -88.08634507854252) bank153203 +153204 POINT(41.30052654527382 -88.53256274777131) bank153204 +153205 POINT(42.49810175443093 -86.97600775240825) bank153205 +153206 POINT(41.22510122623989 -88.50172420227712) bank153206 +153207 POINT(42.115180332547325 -87.61324866152144) bank153207 +153208 POINT(42.20879450162266 -86.76117493681971) bank153208 +153209 POINT(42.11126602157489 -86.87755463373345) bank153209 +153210 POINT(42.74170750763874 -87.31028255068283) bank153210 +153211 POINT(41.99721734688103 -87.66972191380255) bank153211 +153212 POINT(41.22691633539696 -88.17067432647659) bank153212 +153213 POINT(41.882041088533455 -88.2883104006795) bank153213 +153214 POINT(42.702599042638305 -87.60306871299474) bank153214 +153215 POINT(42.56893656286833 -87.51929989101312) bank153215 +153216 POINT(42.61308041381217 -88.52311149672934) bank153216 +153217 POINT(42.85626391464377 -87.27293841784788) bank153217 +153218 POINT(41.919509210315944 -88.22315884630753) bank153218 +153219 POINT(42.121964161495974 -87.69540396461171) bank153219 +153220 POINT(41.11593876431637 -88.20400300469197) bank153220 +153221 POINT(41.75872716176013 -87.53282935205058) bank153221 +153222 POINT(42.75538285085009 -87.6416954082221) bank153222 +153223 POINT(41.81499717755972 -87.82534835647532) bank153223 +153224 POINT(41.387225275625276 -87.85461032406631) bank153224 +153225 POINT(42.73209908933218 -86.72932837745569) bank153225 +153226 POINT(42.82352790485359 -87.32864145429427) bank153226 +153227 POINT(42.23940281551132 -87.5706387482154) bank153227 +153228 POINT(42.70696025934436 -86.97025101407883) bank153228 +153229 POINT(42.00916864603692 -87.50017527826832) bank153229 +153230 POINT(40.99981680351582 -87.03991352837598) bank153230 +153231 POINT(42.30581349384355 -88.19412980484634) bank153231 +153232 POINT(41.22637437812795 -86.7384843444015) bank153232 +153233 POINT(42.33546295602275 -87.28084099286978) bank153233 +153234 POINT(41.266191263066474 -87.56696306571718) bank153234 +153235 POINT(42.04324494232462 -86.79291576473089) bank153235 +153236 POINT(42.48964357078407 -87.82254233233098) bank153236 +153237 POINT(42.70373101206848 -88.5272608830606) bank153237 +153238 POINT(41.33152419800729 -87.51020621384768) bank153238 +153239 POINT(42.15850742260049 -87.4870927576421) bank153239 +153240 POINT(42.17857631296665 -88.43947770553092) bank153240 +153241 POINT(42.56751073448486 -87.97822193142267) bank153241 +153242 POINT(42.44935246492893 -87.66116620522709) bank153242 +153243 POINT(41.77785645263251 -87.4091656516178) bank153243 +153244 POINT(40.95531889609593 -87.42711509296034) bank153244 +153245 POINT(42.30127494125845 -88.37654171865941) bank153245 +153246 POINT(42.4931789272667 -87.0942577946504) bank153246 +153247 POINT(41.31725667930641 -88.54054360726134) bank153247 +153248 POINT(42.54804136262496 -88.11855110604614) bank153248 +153249 POINT(41.54392269895929 -87.1307483814851) bank153249 +153250 POINT(41.000503253304835 -87.04425067757565) bank153250 +153251 POINT(41.00876475489929 -87.65692688708333) bank153251 +153252 POINT(41.138675681270364 -88.60033241950877) bank153252 +153253 POINT(42.7434249700655 -88.45598975234543) bank153253 +153254 POINT(41.48081430270101 -87.92855108363734) bank153254 +153255 POINT(41.5209173881532 -87.99859704061878) bank153255 +153256 POINT(42.18977690840583 -87.45728347842409) bank153256 +153257 POINT(41.3013864459263 -86.68646773444917) bank153257 +153258 POINT(41.145583286059264 -87.49038250989197) bank153258 +153259 POINT(40.984910115545524 -86.97066017667683) bank153259 +153260 POINT(41.96842815558327 -87.25752317673931) bank153260 +153261 POINT(41.37645251306671 -87.21443199346534) bank153261 +153262 POINT(42.499238836588695 -88.62633739018756) bank153262 +153263 POINT(41.61937787616035 -88.23070960521005) bank153263 +153264 POINT(41.22083908914976 -87.15716300100961) bank153264 +153265 POINT(42.483791265337004 -87.01100832085065) bank153265 +153266 POINT(40.966223223780396 -88.20600288239093) bank153266 +153267 POINT(41.18490889241883 -86.9985279545008) bank153267 +153268 POINT(41.382628074468535 -87.29622794410652) bank153268 +153269 POINT(42.84209398993599 -88.3922707482733) bank153269 +153270 POINT(42.681154304011685 -87.12524612254798) bank153270 +153271 POINT(41.75141165015404 -88.19178022893546) bank153271 +153272 POINT(42.76999016903737 -86.79719372763005) bank153272 +153273 POINT(42.03492559701484 -88.34184200861948) bank153273 +153274 POINT(41.66807953690953 -87.98715598228416) bank153274 +153275 POINT(42.25014580416062 -88.18854028079322) bank153275 +153276 POINT(41.90675278521038 -88.41518062912053) bank153276 +153277 POINT(42.139410231885215 -88.61422943046678) bank153277 +153278 POINT(41.041209381917795 -88.05722285563981) bank153278 +153279 POINT(42.10421323092568 -86.83308138237007) bank153279 +153280 POINT(42.055821762629805 -88.45503866182457) bank153280 +153281 POINT(41.91414059999542 -86.87167992000848) bank153281 +153282 POINT(42.83508006243388 -88.52739471527545) bank153282 +153283 POINT(41.043142428165034 -88.15726790345445) bank153283 +153284 POINT(42.372796640966605 -87.3566451467631) bank153284 +153285 POINT(42.86804743911388 -86.94404900138696) bank153285 +153286 POINT(41.581106192683734 -87.36963226836599) bank153286 +153287 POINT(41.36205143197426 -86.83081180102222) bank153287 +153288 POINT(42.510467066524065 -88.11738528918161) bank153288 +153289 POINT(42.11555685685772 -87.63536118282873) bank153289 +153290 POINT(41.99517427171177 -88.32341874377221) bank153290 +153291 POINT(41.74922514841971 -87.00724354469891) bank153291 +153292 POINT(42.21848651571329 -86.97006221912449) bank153292 +153293 POINT(41.968926299204576 -88.44246088375239) bank153293 +153294 POINT(41.06948863123037 -87.2095524609999) bank153294 +153295 POINT(42.765627672089195 -88.17775411260182) bank153295 +153296 POINT(41.36587822892124 -87.20271861326108) bank153296 +153297 POINT(41.6024249208918 -88.02515926387721) bank153297 +153298 POINT(41.425914637863826 -87.61658844366778) bank153298 +153299 POINT(41.54225438481407 -87.00708286327992) bank153299 +153300 POINT(42.80614278028735 -87.77455859462421) bank153300 +153301 POINT(40.9099291819335 -87.41900165927957) bank153301 +153302 POINT(42.14454821855915 -88.22921250206355) bank153302 +153303 POINT(41.10685918244307 -86.8843754710461) bank153303 +153304 POINT(42.81449450040541 -87.80273797879889) bank153304 +153305 POINT(42.27283063869837 -86.96768836662) bank153305 +153306 POINT(41.58193815370475 -87.35214071294719) bank153306 +153307 POINT(40.95371448709707 -88.12572413904581) bank153307 +153308 POINT(41.30530064519382 -86.86541568648722) bank153308 +153309 POINT(42.19279708092649 -88.61064220464861) bank153309 +153310 POINT(41.7984999116649 -87.33127159593273) bank153310 +153311 POINT(41.026844490349404 -88.25536142932548) bank153311 +153312 POINT(41.515146817022845 -88.28529185037932) bank153312 +153313 POINT(41.74545307115914 -87.60968274910454) bank153313 +153314 POINT(42.778782559354056 -87.06699099744776) bank153314 +153315 POINT(41.016890006715045 -88.31862131091994) bank153315 +153316 POINT(41.24977351104332 -86.63546522720608) bank153316 +153317 POINT(41.77893418075753 -87.02357370702933) bank153317 +153318 POINT(41.818896207443125 -86.86427009124766) bank153318 +153319 POINT(42.03321026495176 -88.27536735409183) bank153319 +153320 POINT(40.93501028769995 -87.66375699156772) bank153320 +153321 POINT(41.95373229716235 -87.81010647082304) bank153321 +153322 POINT(42.68174223163328 -87.57273059437674) bank153322 +153323 POINT(41.637894338098405 -88.51097792966526) bank153323 +153324 POINT(41.23604768072841 -87.09474892379411) bank153324 +153325 POINT(41.11639534222086 -88.49177328842086) bank153325 +153326 POINT(42.81115762808989 -88.43142013271738) bank153326 +153327 POINT(42.47677424434242 -87.11393147708087) bank153327 +153328 POINT(41.4712477796284 -87.00486770070403) bank153328 +153329 POINT(41.00332685822793 -88.42669949763236) bank153329 +153330 POINT(41.04606329254976 -86.79128740265706) bank153330 +153331 POINT(40.96738758502428 -88.07050505194246) bank153331 +153332 POINT(41.68055326022214 -87.11788311799764) bank153332 +153333 POINT(42.28449858057293 -88.40632530196694) bank153333 +153334 POINT(41.80019221931339 -86.76735175878646) bank153334 +153335 POINT(42.667200633030085 -87.31831597346675) bank153335 +153336 POINT(42.62278670283515 -88.13953825236085) bank153336 +153337 POINT(41.75899872906415 -87.42858195446107) bank153337 +153338 POINT(41.036685713131845 -88.04682825126018) bank153338 +153339 POINT(41.225416177787416 -88.42275451030316) bank153339 +153340 POINT(41.57498772445253 -87.77616067310338) bank153340 +153341 POINT(42.018621118168916 -87.06860074131839) bank153341 +153342 POINT(41.836516984457674 -88.214959043375) bank153342 +153343 POINT(42.39142838419525 -87.50584466368693) bank153343 +153344 POINT(41.601018859037126 -87.83011687986959) bank153344 +153345 POINT(42.68312428602388 -87.6697246279578) bank153345 +153346 POINT(40.95850117297739 -87.60387883087994) bank153346 +153347 POINT(41.31261526523283 -88.25795470018487) bank153347 +153348 POINT(42.473154108930686 -86.87693385946312) bank153348 +153349 POINT(42.73709118327053 -87.3693501650267) bank153349 +153350 POINT(42.021800368223175 -86.7448462096303) bank153350 +153351 POINT(42.61040497514968 -86.81317620816162) bank153351 +153352 POINT(42.4092250489137 -86.70608041913623) bank153352 +153353 POINT(42.083390657142516 -88.14479069503349) bank153353 +153354 POINT(42.38225992139277 -87.00611195932508) bank153354 +153355 POINT(42.656433625569335 -87.5212830383627) bank153355 +153356 POINT(42.265069840873316 -87.91818061400254) bank153356 +153357 POINT(41.81828723417024 -88.57634995828072) bank153357 +153358 POINT(41.98347425978353 -88.57152793207598) bank153358 +153359 POINT(41.58444204269416 -86.98646379179215) bank153359 +153360 POINT(41.59396722664375 -87.08839170638645) bank153360 +153361 POINT(41.39010749833418 -87.22426747250152) bank153361 +153362 POINT(41.753878620387546 -87.42646615962602) bank153362 +153363 POINT(41.88588743987266 -87.48170615500064) bank153363 +153364 POINT(42.39241772632099 -87.79542669543626) bank153364 +153365 POINT(42.74316204154872 -87.45204264669842) bank153365 +153366 POINT(40.88068775425706 -86.9386579455494) bank153366 +153367 POINT(40.97725453425267 -88.5805234831876) bank153367 +153368 POINT(41.57806550576123 -88.21692760629459) bank153368 +153369 POINT(42.85332788779202 -86.8097602024974) bank153369 +153370 POINT(41.52283012709507 -88.13534700257107) bank153370 +153371 POINT(41.65027264639921 -87.06238389855655) bank153371 +153372 POINT(42.31534663789516 -87.76460417948766) bank153372 +153373 POINT(41.546802040044405 -87.67765965121625) bank153373 +153374 POINT(41.0342329735119 -88.61611819331443) bank153374 +153375 POINT(42.54523734284023 -87.0601597363937) bank153375 +153376 POINT(42.7091475762701 -86.7232337804586) bank153376 +153377 POINT(41.3286098816806 -88.01252191925938) bank153377 +153378 POINT(40.985930196400254 -88.36562772690193) bank153378 +153379 POINT(41.92220892603032 -86.82242006921811) bank153379 +153380 POINT(42.593551619969865 -86.70245903134905) bank153380 +153381 POINT(42.28515358840057 -87.55574771176104) bank153381 +153382 POINT(42.627076490515066 -87.24573808807196) bank153382 +153383 POINT(42.238647944575646 -88.07315403833317) bank153383 +153384 POINT(41.36102851173574 -87.40124833802746) bank153384 +153385 POINT(41.57908392571682 -87.024067266905) bank153385 +153386 POINT(41.520290866332445 -87.0568214737248) bank153386 +153387 POINT(42.578851041501274 -86.67715803549244) bank153387 +153388 POINT(42.79201880590163 -87.23810722933983) bank153388 +153389 POINT(42.58607207611821 -88.23652570650631) bank153389 +153390 POINT(41.6172115519716 -87.44283161364831) bank153390 +153391 POINT(41.33832883895481 -87.76211193680112) bank153391 +153392 POINT(41.64537727105072 -87.7008198565559) bank153392 +153393 POINT(42.24202983190979 -87.65129699525197) bank153393 +153394 POINT(42.86060113756312 -88.0118284291771) bank153394 +153395 POINT(42.82437052330573 -88.19306485172613) bank153395 +153396 POINT(42.52887407634731 -87.3437585232762) bank153396 +153397 POINT(41.14307616912905 -87.71158608106707) bank153397 +153398 POINT(42.15062567382198 -87.76345446517898) bank153398 +153399 POINT(42.17688280278115 -88.29962661051862) bank153399 +153400 POINT(42.16681985046839 -88.14179447982289) bank153400 +153401 POINT(41.68698029964208 -87.23192099349218) bank153401 +153402 POINT(42.75347547133389 -88.4946122420413) bank153402 +153403 POINT(41.79444551763328 -87.10581498226703) bank153403 +153404 POINT(41.103289517990014 -88.51465536281164) bank153404 +153405 POINT(42.29343848357651 -87.70974057377651) bank153405 +153406 POINT(41.71252094643405 -87.06876617791251) bank153406 +153407 POINT(41.38544824250923 -87.93267576400248) bank153407 +153408 POINT(42.51762214784911 -88.01993937443648) bank153408 +153409 POINT(41.72932924932656 -87.18779620776529) bank153409 +153410 POINT(41.673214528580054 -87.83918138108456) bank153410 +153411 POINT(41.03239330438148 -87.59545486479377) bank153411 +153412 POINT(41.8477705216956 -86.90987005065199) bank153412 +153413 POINT(42.40934712805725 -86.82045643376627) bank153413 +153414 POINT(41.52006001856371 -86.9565991914982) bank153414 +153415 POINT(42.82547841077203 -87.02851712837294) bank153415 +153416 POINT(41.75578695628761 -88.17440570083583) bank153416 +153417 POINT(41.7394810692613 -88.17161495012556) bank153417 +153418 POINT(42.80690145394083 -86.64993332063632) bank153418 +153419 POINT(42.71886947189456 -87.35813929879039) bank153419 +153420 POINT(42.504730491586955 -87.92502342274454) bank153420 +153421 POINT(41.89860328449508 -87.18776742795366) bank153421 +153422 POINT(41.009846607243595 -87.253803317942) bank153422 +153423 POINT(41.48929765629933 -88.10916741546943) bank153423 +153424 POINT(42.04208855245888 -88.38666105543692) bank153424 +153425 POINT(41.70887422473927 -88.00678474549787) bank153425 +153426 POINT(42.1935985588056 -87.47146994833781) bank153426 +153427 POINT(42.387485949120695 -88.2390184524822) bank153427 +153428 POINT(42.46898295293757 -87.8762621283512) bank153428 +153429 POINT(42.00708102337862 -88.41658461710075) bank153429 +153430 POINT(41.22338124781847 -88.17170171575422) bank153430 +153431 POINT(41.81472071458519 -86.63559407848855) bank153431 +153432 POINT(42.86572740240796 -86.69801001387351) bank153432 +153433 POINT(41.53811983306407 -88.588076361061) bank153433 +153434 POINT(42.1815247886954 -87.80077323731332) bank153434 +153435 POINT(42.56601518353608 -86.9634259060743) bank153435 +153436 POINT(41.7572709208697 -87.36851182868968) bank153436 +153437 POINT(42.66091611232548 -87.18513889713351) bank153437 +153438 POINT(41.504038633015135 -88.0861381410792) bank153438 +153439 POINT(42.03102749686897 -86.84339054939376) bank153439 +153440 POINT(41.4215112481033 -87.748378089516) bank153440 +153441 POINT(42.07004154317619 -87.64570135664161) bank153441 +153442 POINT(42.4947970302853 -87.77577659453627) bank153442 +153443 POINT(42.53789179038135 -88.15255215648024) bank153443 +153444 POINT(42.84700386561326 -88.10952423972766) bank153444 +153445 POINT(41.44152970766762 -88.01836648825036) bank153445 +153446 POINT(41.51641190142588 -87.17851783766989) bank153446 +153447 POINT(41.77495150436549 -87.25254452386474) bank153447 +153448 POINT(42.30335766611482 -86.76151672050955) bank153448 +153449 POINT(42.235603249140134 -87.02262487069684) bank153449 +153450 POINT(41.53047387796541 -87.25544382033596) bank153450 +153451 POINT(41.5889772130852 -87.9476854704499) bank153451 +153452 POINT(42.504343645424726 -87.121837401679) bank153452 +153453 POINT(42.77003632214301 -87.09772708966393) bank153453 +153454 POINT(41.124151955623056 -88.11972841691987) bank153454 +153455 POINT(42.20414570396933 -87.82909367550262) bank153455 +153456 POINT(42.7504533003738 -88.4767146371229) bank153456 +153457 POINT(41.44654999695151 -87.21888783657747) bank153457 +153458 POINT(41.36401382526362 -87.29232457023855) bank153458 +153459 POINT(41.08935239695888 -87.51256751039908) bank153459 +153460 POINT(41.4386765137548 -88.37076383224547) bank153460 +153461 POINT(41.27716747650376 -87.44704380462886) bank153461 +153462 POINT(41.026710807577516 -87.93701746583723) bank153462 +153463 POINT(41.21576958862928 -87.53269621399478) bank153463 +153464 POINT(42.470151500824024 -87.6564078794039) bank153464 +153465 POINT(41.261492270288684 -88.384934412749) bank153465 +153466 POINT(41.00004719015695 -87.98704905395455) bank153466 +153467 POINT(42.87268747246982 -86.99419040449848) bank153467 +153468 POINT(41.52996900852878 -86.78354905663957) bank153468 +153469 POINT(41.39189390031239 -87.36792559036287) bank153469 +153470 POINT(41.77175002110356 -87.30592155208949) bank153470 +153471 POINT(42.542238467575125 -87.90700510771292) bank153471 +153472 POINT(42.16383494501087 -87.10008106610795) bank153472 +153473 POINT(41.99461059204142 -87.51002296627618) bank153473 +153474 POINT(42.56998455582037 -86.70250442075397) bank153474 +153475 POINT(41.668605048323954 -88.5052956970544) bank153475 +153476 POINT(41.57047811961725 -87.50512287921498) bank153476 +153477 POINT(42.028227458927134 -87.07511299327784) bank153477 +153478 POINT(40.979787349435654 -86.63752288089879) bank153478 +153479 POINT(42.66600672778393 -86.8356224262781) bank153479 +153480 POINT(42.689746397084704 -87.83636718403804) bank153480 +153481 POINT(41.1293667952697 -88.18383369330871) bank153481 +153482 POINT(41.73675787453637 -87.36796653733342) bank153482 +153483 POINT(42.41957707027146 -87.39956675498476) bank153483 +153484 POINT(41.3163601135536 -88.48653743842195) bank153484 +153485 POINT(41.01013202273024 -87.3143168466029) bank153485 +153486 POINT(42.78776153373694 -86.83379277126508) bank153486 +153487 POINT(41.314300510515594 -87.89432653250827) bank153487 +153488 POINT(41.02109701155733 -88.25765594064518) bank153488 +153489 POINT(40.979191045963205 -86.91357252631882) bank153489 +153490 POINT(41.1244766473595 -87.16003902696755) bank153490 +153491 POINT(42.65764810159055 -86.83528729838628) bank153491 +153492 POINT(42.11993093736681 -87.83134772003757) bank153492 +153493 POINT(40.916790878917524 -88.39520580068849) bank153493 +153494 POINT(42.49267971179115 -87.4122672714331) bank153494 +153495 POINT(42.405368149911865 -87.55833398186782) bank153495 +153496 POINT(42.180629426739486 -88.00331570355856) bank153496 +153497 POINT(41.84488277499322 -87.27195116061748) bank153497 +153498 POINT(41.867074685884056 -88.39241855364199) bank153498 +153499 POINT(42.654834974923574 -88.36975097329788) bank153499 +153500 POINT(41.738138386111416 -87.89713312596194) bank153500 +153501 POINT(41.074100582981295 -86.87697559010255) bank153501 +153502 POINT(42.445853825724804 -87.05694609610235) bank153502 +153503 POINT(41.70037241519157 -86.64920626314861) bank153503 +153504 POINT(41.811229062709224 -87.24823150649789) bank153504 +153505 POINT(41.22863512820071 -87.9651424071988) bank153505 +153506 POINT(42.158107643393734 -87.75236858481712) bank153506 +153507 POINT(41.50663757333377 -88.22413553596807) bank153507 +153508 POINT(42.81485142721339 -86.78714851232658) bank153508 +153509 POINT(41.48385796097514 -88.49362554155432) bank153509 +153510 POINT(41.20587195474981 -88.21747482024355) bank153510 +153511 POINT(41.64734581663199 -87.60693353787616) bank153511 +153512 POINT(41.55930171172576 -88.13689540748533) bank153512 +153513 POINT(42.50458098523578 -87.15880916769649) bank153513 +153514 POINT(41.0847147729143 -86.94249706211829) bank153514 +153515 POINT(42.725468926600385 -88.28992126752247) bank153515 +153516 POINT(41.14607821324391 -87.23105247304254) bank153516 +153517 POINT(42.07964983855867 -86.97293877315295) bank153517 +153518 POINT(41.797279578317294 -87.05575063515917) bank153518 +153519 POINT(42.447417746375436 -86.78607448000578) bank153519 +153520 POINT(42.013367576461654 -88.0349560326671) bank153520 +153521 POINT(42.20339568928725 -86.91112174126677) bank153521 +153522 POINT(42.3445359393521 -87.82893106764035) bank153522 +153523 POINT(41.98878569984945 -87.30225585827628) bank153523 +153524 POINT(41.63398518994239 -87.84856537133292) bank153524 +153525 POINT(42.11574051558926 -87.85087979202466) bank153525 +153526 POINT(42.44945845973146 -86.97360538826007) bank153526 +153527 POINT(42.025499818840366 -87.68387704131634) bank153527 +153528 POINT(41.84674803510802 -88.23788220205556) bank153528 +153529 POINT(42.716025591801156 -87.41293955225235) bank153529 +153530 POINT(41.5811028115549 -86.95802569235829) bank153530 +153531 POINT(41.559064014745935 -88.52044822713508) bank153531 +153532 POINT(42.7834863157663 -87.87432454551858) bank153532 +153533 POINT(41.36611473846858 -88.52734811033639) bank153533 +153534 POINT(41.31234407200093 -86.71765990684605) bank153534 +153535 POINT(41.020669812622266 -87.4262256309868) bank153535 +153536 POINT(41.84708103707507 -88.13822092754931) bank153536 +153537 POINT(41.9229092481954 -87.59842038894774) bank153537 +153538 POINT(41.48402495468205 -87.60555594766899) bank153538 +153539 POINT(42.1781935239326 -87.84348786871686) bank153539 +153540 POINT(42.090820623013705 -87.65389784274727) bank153540 +153541 POINT(42.64711854312708 -88.2444042095601) bank153541 +153542 POINT(41.79875265704026 -87.6734223830869) bank153542 +153543 POINT(41.03255394092506 -88.46517460806514) bank153543 +153544 POINT(41.6192381959034 -86.8223129364042) bank153544 +153545 POINT(42.662687176235956 -88.35509277606458) bank153545 +153546 POINT(42.78578958041022 -87.88848162548358) bank153546 +153547 POINT(42.56272438570151 -87.80140269637432) bank153547 +153548 POINT(42.134598048290435 -87.89280839269347) bank153548 +153549 POINT(41.935813669410265 -87.3852811152641) bank153549 +153550 POINT(41.44454980749389 -86.91793909898438) bank153550 +153551 POINT(42.55927380606748 -87.01188912703431) bank153551 +153552 POINT(41.90703979925017 -87.8103578284985) bank153552 +153553 POINT(41.984268632428765 -88.52078982513038) bank153553 +153554 POINT(42.55715427916135 -87.97693756526046) bank153554 +153555 POINT(41.77471412041201 -88.62513597272158) bank153555 +153556 POINT(41.17433482157669 -87.89935873546447) bank153556 +153557 POINT(42.47894923887719 -88.496406944071) bank153557 +153558 POINT(41.34519537465742 -87.25003904499627) bank153558 +153559 POINT(42.51825872384669 -87.45702600888113) bank153559 +153560 POINT(42.385628124939366 -87.15631955670692) bank153560 +153561 POINT(41.54898955701896 -88.13509433463119) bank153561 +153562 POINT(42.3015230433849 -87.38263039295765) bank153562 +153563 POINT(42.40595342575197 -87.22062800402784) bank153563 +153564 POINT(41.69487679061435 -88.27920808750574) bank153564 +153565 POINT(41.48315685678827 -86.82300689652823) bank153565 +153566 POINT(41.91846074318866 -86.8237527377134) bank153566 +153567 POINT(42.44879824765525 -87.85763896236212) bank153567 +153568 POINT(42.001731355309154 -86.64718441641598) bank153568 +153569 POINT(42.07598976382801 -87.6838463220333) bank153569 +153570 POINT(40.88758646263151 -86.67080958968059) bank153570 +153571 POINT(41.27353243987613 -88.16606944107689) bank153571 +153572 POINT(42.57546011594165 -87.7318549947003) bank153572 +153573 POINT(41.72241257929419 -88.32660808505314) bank153573 +153574 POINT(41.503124723119406 -87.15900269505259) bank153574 +153575 POINT(41.10587347444322 -87.15995924951282) bank153575 +153576 POINT(41.62779683777404 -88.54213537903043) bank153576 +153577 POINT(42.408328970206774 -87.27428831425507) bank153577 +153578 POINT(41.35400637062559 -87.92201980890275) bank153578 +153579 POINT(40.88999878536913 -87.30858409025136) bank153579 +153580 POINT(41.31037079492348 -87.81825936522844) bank153580 +153581 POINT(41.14327806621962 -88.23814102042093) bank153581 +153582 POINT(42.150834903612186 -86.87239386898466) bank153582 +153583 POINT(41.94379876437788 -87.31001592796332) bank153583 +153584 POINT(41.630992821729954 -88.23785572549501) bank153584 +153585 POINT(41.851564664994235 -87.08273602022389) bank153585 +153586 POINT(40.92337279085263 -86.7597724986936) bank153586 +153587 POINT(41.850898335648715 -87.68229989941615) bank153587 +153588 POINT(42.338266482947404 -88.09890179231434) bank153588 +153589 POINT(42.66400931779903 -88.02753790237787) bank153589 +153590 POINT(41.88320342416211 -86.97159713147533) bank153590 +153591 POINT(40.89124499990507 -87.72588767722513) bank153591 +153592 POINT(41.32635663461536 -87.20228389676053) bank153592 +153593 POINT(41.350081294634876 -86.98078927838175) bank153593 +153594 POINT(41.652858041218344 -87.5087631686226) bank153594 +153595 POINT(42.01568823352457 -87.99272914626907) bank153595 +153596 POINT(41.9284669423292 -86.96222219929399) bank153596 +153597 POINT(41.07809169006992 -87.37943223893818) bank153597 +153598 POINT(41.67525550737177 -88.60351280254208) bank153598 +153599 POINT(41.5608139714079 -86.98989680353714) bank153599 +153600 POINT(41.987351098133004 -88.01036797489812) bank153600 +153601 POINT(41.211980752185774 -87.2714826243534) bank153601 +153602 POINT(40.97558079189644 -86.7503297004184) bank153602 +153603 POINT(42.760491323213216 -88.46145885561123) bank153603 +153604 POINT(41.46404080507613 -87.32115631452491) bank153604 +153605 POINT(41.05092454706702 -88.02371601830389) bank153605 +153606 POINT(42.24906847121778 -88.5991453061465) bank153606 +153607 POINT(41.32405648274882 -87.34428721841057) bank153607 +153608 POINT(41.047636127562726 -87.87313681566528) bank153608 +153609 POINT(42.47512728903289 -88.5695322057486) bank153609 +153610 POINT(41.16606695687285 -87.4862158551644) bank153610 +153611 POINT(41.31245822550644 -87.7729464119903) bank153611 +153612 POINT(40.93318044591705 -87.23691814901456) bank153612 +153613 POINT(42.06739009703237 -87.34197681600368) bank153613 +153614 POINT(41.66469316268353 -87.93657148785014) bank153614 +153615 POINT(42.644323900760796 -88.07506035816391) bank153615 +153616 POINT(41.111559302261846 -88.61784259131363) bank153616 +153617 POINT(42.22397155555569 -87.96404141705243) bank153617 +153618 POINT(42.69249841824483 -87.25346273392252) bank153618 +153619 POINT(42.69380662621145 -87.89353873945048) bank153619 +153620 POINT(41.05267360724776 -87.85723104855185) bank153620 +153621 POINT(42.545537724057645 -87.45542875428164) bank153621 +153622 POINT(41.33544800324418 -87.46600289522209) bank153622 +153623 POINT(41.518574555715354 -87.57856397033066) bank153623 +153624 POINT(42.343929277587335 -87.5575797287428) bank153624 +153625 POINT(41.11324538382837 -86.98883650625564) bank153625 +153626 POINT(42.31306195222859 -88.46520780452353) bank153626 +153627 POINT(41.18574724785343 -87.61945205453087) bank153627 +153628 POINT(42.71286353110594 -88.48253709627994) bank153628 +153629 POINT(41.82939011821612 -87.3290273675202) bank153629 +153630 POINT(42.58257756695417 -87.76722394746923) bank153630 +153631 POINT(41.51979900825662 -87.05100701766413) bank153631 +153632 POINT(41.49251093205351 -87.19175988776091) bank153632 +153633 POINT(41.37560917558839 -87.60439929852032) bank153633 +153634 POINT(41.294986903810354 -88.33770646335078) bank153634 +153635 POINT(40.97057423315103 -86.78460884492829) bank153635 +153636 POINT(40.90959165411953 -87.16169187423651) bank153636 +153637 POINT(41.53783626445934 -87.39225432184612) bank153637 +153638 POINT(41.00878400699054 -88.03949432049863) bank153638 +153639 POINT(42.574800308900514 -87.51736268086817) bank153639 +153640 POINT(40.9750977254854 -86.65780911652776) bank153640 +153641 POINT(41.868220247037165 -88.2543546988046) bank153641 +153642 POINT(42.3674399414397 -88.0103787609012) bank153642 +153643 POINT(42.51577220867896 -88.40012212524314) bank153643 +153644 POINT(41.96832178335277 -87.59197235083191) bank153644 +153645 POINT(41.41247067416879 -87.19464601105969) bank153645 +153646 POINT(42.08512145062295 -87.46876729472493) bank153646 +153647 POINT(41.60872420586615 -87.99602758706975) bank153647 +153648 POINT(41.735821565400514 -88.31061882540243) bank153648 +153649 POINT(42.83523858154428 -87.58589874028495) bank153649 +153650 POINT(42.22784776225521 -87.79613360501673) bank153650 +153651 POINT(42.24691865450712 -86.96868383494481) bank153651 +153652 POINT(42.14682798289545 -87.6377303544612) bank153652 +153653 POINT(41.89284017049302 -86.93232168356816) bank153653 +153654 POINT(40.97267182210794 -88.14840376530219) bank153654 +153655 POINT(41.030031763289976 -86.8951433416125) bank153655 +153656 POINT(41.28739733400654 -87.4275871455338) bank153656 +153657 POINT(42.028608973847554 -87.51091566373992) bank153657 +153658 POINT(42.53415804065783 -86.84886800885933) bank153658 +153659 POINT(41.94876605054298 -87.38102688799465) bank153659 +153660 POINT(42.46469406088431 -87.60372198099448) bank153660 +153661 POINT(41.95694326752626 -87.29465271076961) bank153661 +153662 POINT(42.53797670915187 -87.831786314616) bank153662 +153663 POINT(42.65788250724914 -88.5754164826467) bank153663 +153664 POINT(42.55462965025926 -88.06629615333071) bank153664 +153665 POINT(42.823695603459086 -87.3350060900415) bank153665 +153666 POINT(41.408103650249046 -87.25567752824425) bank153666 +153667 POINT(41.72820433105368 -87.5193173840798) bank153667 +153668 POINT(42.21625987065124 -87.1017035543854) bank153668 +153669 POINT(42.27218105061317 -86.86006934322968) bank153669 +153670 POINT(41.23032043038855 -87.70058433899264) bank153670 +153671 POINT(42.68745985916688 -87.59978565725332) bank153671 +153672 POINT(41.7086009777285 -88.50013718363752) bank153672 +153673 POINT(41.61626060003066 -87.55206248138667) bank153673 +153674 POINT(41.921312568840605 -86.77985218764034) bank153674 +153675 POINT(41.53064721571535 -88.38523169268716) bank153675 +153676 POINT(42.78280595401223 -87.97823959050635) bank153676 +153677 POINT(41.78026771920981 -88.49746471924132) bank153677 +153678 POINT(41.31123548780391 -87.63010469184965) bank153678 +153679 POINT(42.770757214410196 -87.87772736484432) bank153679 +153680 POINT(42.05086070973248 -87.69039161240843) bank153680 +153681 POINT(41.57425166796794 -88.2776223997935) bank153681 +153682 POINT(41.67504117802629 -87.38322280745909) bank153682 +153683 POINT(42.266243148151666 -88.31645659886546) bank153683 +153684 POINT(40.9473425666104 -87.02022060110131) bank153684 +153685 POINT(40.965402271919295 -86.96206710995672) bank153685 +153686 POINT(41.86903848996334 -87.98095696302589) bank153686 +153687 POINT(42.49707950208311 -86.90614957386475) bank153687 +153688 POINT(41.18504259008058 -88.47318636597976) bank153688 +153689 POINT(41.61961590914591 -87.14091632424746) bank153689 +153690 POINT(41.38853036120604 -87.15763594169165) bank153690 +153691 POINT(41.63889806633442 -86.71514324281655) bank153691 +153692 POINT(42.290015107379524 -87.53307282128716) bank153692 +153693 POINT(41.95327634416402 -86.93517412906958) bank153693 +153694 POINT(41.81515409124489 -86.85997456630844) bank153694 +153695 POINT(42.33986310266031 -86.95826643985298) bank153695 +153696 POINT(41.38467784929785 -87.2071169226474) bank153696 +153697 POINT(40.96137114144923 -87.75572262244056) bank153697 +153698 POINT(41.739243079693864 -87.16409100670505) bank153698 +153699 POINT(42.818550451684914 -87.16853930664398) bank153699 +153700 POINT(41.01656780312303 -88.42664313942619) bank153700 +153701 POINT(41.118037531356016 -88.22773582363855) bank153701 +153702 POINT(41.86853310291406 -87.86770247183352) bank153702 +153703 POINT(41.93523157259762 -87.5359285256537) bank153703 +153704 POINT(41.11036123885786 -88.22748991673824) bank153704 +153705 POINT(41.04628455639217 -88.55370302325649) bank153705 +153706 POINT(40.91507437309201 -88.59649070405955) bank153706 +153707 POINT(42.783677904600935 -87.04016133132541) bank153707 +153708 POINT(42.82465504774013 -87.3799641813258) bank153708 +153709 POINT(42.6403493363394 -88.00476657052906) bank153709 +153710 POINT(41.591762664013814 -87.825387353006) bank153710 +153711 POINT(42.67291774960139 -88.31511771742433) bank153711 +153712 POINT(41.840642800115965 -88.42067067712473) bank153712 +153713 POINT(41.313556728208866 -86.97359685570669) bank153713 +153714 POINT(40.938332507301375 -88.02008382985179) bank153714 +153715 POINT(41.741315794438925 -88.52761175850199) bank153715 +153716 POINT(42.70272309950999 -88.16250949244764) bank153716 +153717 POINT(41.47880831240296 -87.92579710408914) bank153717 +153718 POINT(42.4590575564572 -87.26525113768513) bank153718 +153719 POINT(41.66435244828992 -88.069080303903) bank153719 +153720 POINT(41.38405298686138 -87.81407114450954) bank153720 +153721 POINT(41.420866911750146 -88.4259958411535) bank153721 +153722 POINT(42.432818881279275 -87.1843795275332) bank153722 +153723 POINT(41.1846064569234 -88.01212090874813) bank153723 +153724 POINT(42.082095254713636 -88.187059341772) bank153724 +153725 POINT(42.55497952306913 -88.176932359177) bank153725 +153726 POINT(41.36147706613086 -86.83915490970963) bank153726 +153727 POINT(42.050860167723535 -88.45988532518156) bank153727 +153728 POINT(41.97882360171978 -86.90641947471893) bank153728 +153729 POINT(40.976871605377546 -88.14466770147656) bank153729 +153730 POINT(41.98398762438346 -87.01114654402454) bank153730 +153731 POINT(42.02604425930426 -87.9348734966441) bank153731 +153732 POINT(42.0542197823508 -86.89725802818911) bank153732 +153733 POINT(42.4582116414338 -87.16587787156014) bank153733 +153734 POINT(42.05077023400947 -87.72318061707949) bank153734 +153735 POINT(41.733720974591854 -88.39702652379897) bank153735 +153736 POINT(41.60679747191525 -87.72822279679664) bank153736 +153737 POINT(42.82123987856941 -88.61589167907057) bank153737 +153738 POINT(42.40584872897684 -87.3147827734657) bank153738 +153739 POINT(42.07156176456153 -87.22415857359181) bank153739 +153740 POINT(42.54257475624088 -86.86319526161707) bank153740 +153741 POINT(42.80175973158541 -88.60746952748116) bank153741 +153742 POINT(42.17609718492724 -87.11661715990607) bank153742 +153743 POINT(42.50630985739895 -87.7498235422525) bank153743 +153744 POINT(41.12567668054447 -88.5428983627023) bank153744 +153745 POINT(42.367235319864825 -88.20777602623404) bank153745 +153746 POINT(41.518311009886894 -86.79337681203513) bank153746 +153747 POINT(40.9717535821721 -87.9779925631442) bank153747 +153748 POINT(41.0290681625828 -87.4037546333937) bank153748 +153749 POINT(41.811167302642446 -87.84362677149073) bank153749 +153750 POINT(42.35984641505532 -87.39820665049619) bank153750 +153751 POINT(42.71890357045315 -88.58031459054381) bank153751 +153752 POINT(42.38332215376649 -87.90796942567604) bank153752 +153753 POINT(41.0295546307069 -88.31223371488137) bank153753 +153754 POINT(42.74129140144057 -88.15449265644125) bank153754 +153755 POINT(41.08054925893846 -87.68257093892348) bank153755 +153756 POINT(41.04079591821156 -88.22572221308938) bank153756 +153757 POINT(41.61587360003073 -88.19863796295509) bank153757 +153758 POINT(42.691162916497674 -87.71397373102187) bank153758 +153759 POINT(41.93500338760503 -87.00004516904733) bank153759 +153760 POINT(41.92792227824471 -88.57878430323386) bank153760 +153761 POINT(42.50668228644714 -88.40865669998563) bank153761 +153762 POINT(42.32775244288289 -88.62408800364122) bank153762 +153763 POINT(42.11769270610067 -87.27598795740604) bank153763 +153764 POINT(42.11621429971267 -87.24149983068315) bank153764 +153765 POINT(42.236714684083545 -87.98675996136781) bank153765 +153766 POINT(41.712716924417144 -87.00961755104026) bank153766 +153767 POINT(40.9529795496497 -87.15206035728171) bank153767 +153768 POINT(41.691193786846625 -86.77316801716947) bank153768 +153769 POINT(42.007106614112786 -87.56335762338017) bank153769 +153770 POINT(42.521756112995874 -88.17403309974142) bank153770 +153771 POINT(41.876116248896146 -87.16471793092066) bank153771 +153772 POINT(41.098695472735876 -87.23283652119602) bank153772 +153773 POINT(41.38225390775195 -87.01950427446924) bank153773 +153774 POINT(42.46340433322048 -87.65487643438436) bank153774 +153775 POINT(42.15407018293443 -87.51914779715102) bank153775 +153776 POINT(41.19386668378699 -88.47877927478125) bank153776 +153777 POINT(42.47202157788456 -86.91172512997682) bank153777 +153778 POINT(42.24255312115331 -88.15662150817195) bank153778 +153779 POINT(41.94255878778973 -87.81689175505075) bank153779 +153780 POINT(42.008045587076495 -88.20693129938005) bank153780 +153781 POINT(41.00554007421172 -86.76105279937407) bank153781 +153782 POINT(42.68887649661011 -87.3205595447909) bank153782 +153783 POINT(42.26612281153987 -86.80547842326344) bank153783 +153784 POINT(40.934555455495286 -87.4445977929159) bank153784 +153785 POINT(42.791031153436016 -88.573406217461) bank153785 +153786 POINT(42.40847429213851 -86.6452896541072) bank153786 +153787 POINT(42.075088973513836 -86.82800861426065) bank153787 +153788 POINT(41.181761449238344 -88.18089769431896) bank153788 +153789 POINT(41.845962078593494 -87.78840616698116) bank153789 +153790 POINT(42.57670092157817 -87.97444616092378) bank153790 +153791 POINT(42.571127057312914 -86.6320929093029) bank153791 +153792 POINT(41.98781665498853 -87.4343691799153) bank153792 +153793 POINT(41.63371371032292 -86.97242367720003) bank153793 +153794 POINT(42.00571137391653 -88.58178262938463) bank153794 +153795 POINT(42.50619030682499 -87.87335241218013) bank153795 +153796 POINT(41.062369388379224 -86.84715464188733) bank153796 +153797 POINT(42.23391106931723 -88.07653817572303) bank153797 +153798 POINT(42.81917854381786 -86.79722457474207) bank153798 +153799 POINT(42.18348860675872 -87.23293990365299) bank153799 +153800 POINT(41.17687476611814 -88.1107759351328) bank153800 +153801 POINT(41.79574622284944 -88.57179525145668) bank153801 +153802 POINT(40.9287123096686 -88.40467532285533) bank153802 +153803 POINT(42.72876231497996 -88.0281967464121) bank153803 +153804 POINT(41.125994714792526 -87.1979276579817) bank153804 +153805 POINT(42.35396600913882 -88.3411114709731) bank153805 +153806 POINT(41.249506352963124 -87.78525240233253) bank153806 +153807 POINT(42.51048383379595 -88.07760016771391) bank153807 +153808 POINT(42.288668505556394 -87.27586076709493) bank153808 +153809 POINT(41.6683833295633 -87.44324289458241) bank153809 +153810 POINT(41.809363837345956 -87.57698747316314) bank153810 +153811 POINT(41.409657139899636 -88.22748483405798) bank153811 +153812 POINT(41.49205821444197 -87.55522145972675) bank153812 +153813 POINT(42.21400518655447 -87.52627999325031) bank153813 +153814 POINT(42.207567113132384 -86.86738967351089) bank153814 +153815 POINT(41.80886614987688 -88.43460309598242) bank153815 +153816 POINT(42.0852325235103 -87.84829197065174) bank153816 +153817 POINT(41.42468135041314 -87.82926381744336) bank153817 +153818 POINT(42.55701431885299 -88.54137017028297) bank153818 +153819 POINT(42.842182421423104 -88.2399732895247) bank153819 +153820 POINT(41.60214426383082 -87.42583776222943) bank153820 +153821 POINT(42.40544647280535 -87.9468461530793) bank153821 +153822 POINT(41.34141636550234 -87.94588665952179) bank153822 +153823 POINT(41.10852517575122 -86.63578797764907) bank153823 +153824 POINT(41.84866333749534 -87.52991711943561) bank153824 +153825 POINT(42.764008244747885 -87.63667922075084) bank153825 +153826 POINT(41.750486002349156 -87.92027741034347) bank153826 +153827 POINT(41.75897735457763 -87.76785583827645) bank153827 +153828 POINT(42.39696262430601 -87.64679684055017) bank153828 +153829 POINT(41.63311566368885 -87.79505561856288) bank153829 +153830 POINT(42.73432460998769 -87.49817375258182) bank153830 +153831 POINT(41.63469522321015 -87.77250118575088) bank153831 +153832 POINT(41.01073213447141 -86.84367886260904) bank153832 +153833 POINT(42.05781726903389 -87.54643478838128) bank153833 +153834 POINT(42.295170812907564 -87.92937983965719) bank153834 +153835 POINT(42.69190691243957 -88.09660431447719) bank153835 +153836 POINT(40.93109811459967 -87.39111711921193) bank153836 +153837 POINT(41.07093222773007 -87.80265456121928) bank153837 +153838 POINT(41.75575892734252 -86.6680968094711) bank153838 +153839 POINT(41.77800440967264 -87.48654221554342) bank153839 +153840 POINT(41.48217226061976 -87.67813227785562) bank153840 +153841 POINT(41.198826136808094 -86.84179387296157) bank153841 +153842 POINT(42.82298589280648 -88.00628388125587) bank153842 +153843 POINT(41.10534958509599 -86.64943428228827) bank153843 +153844 POINT(41.29272457398606 -88.24256615404084) bank153844 +153845 POINT(41.3367979173374 -87.58519530087585) bank153845 +153846 POINT(41.76137888030538 -87.01223945990509) bank153846 +153847 POINT(41.32082806439078 -87.69493673311194) bank153847 +153848 POINT(41.64173631797344 -88.09678850837233) bank153848 +153849 POINT(42.35398197392226 -87.27804206379389) bank153849 +153850 POINT(42.39084963403296 -87.97658660058912) bank153850 +153851 POINT(42.455368722905746 -87.95196926273954) bank153851 +153852 POINT(41.53389353223461 -87.05011084877782) bank153852 +153853 POINT(41.082311396996914 -87.47868603150378) bank153853 +153854 POINT(40.879687113083456 -86.83323870557889) bank153854 +153855 POINT(42.78391003933314 -87.61679251846927) bank153855 +153856 POINT(42.414006434275514 -88.26998930564993) bank153856 +153857 POINT(42.04579578110918 -88.30542652471955) bank153857 +153858 POINT(41.82138955652325 -87.75177156384919) bank153858 +153859 POINT(42.257700233729615 -88.54704165621885) bank153859 +153860 POINT(42.348131754087945 -87.01176109289356) bank153860 +153861 POINT(42.78854015865447 -87.8822732153556) bank153861 +153862 POINT(42.417026119372856 -86.81184097010335) bank153862 +153863 POINT(41.858086748522354 -87.04758275378938) bank153863 +153864 POINT(42.74292317770324 -86.72226611258257) bank153864 +153865 POINT(42.777267913352716 -87.01290826757615) bank153865 +153866 POINT(42.454512397367864 -88.4224622675314) bank153866 +153867 POINT(41.11169547270463 -86.94458855556056) bank153867 +153868 POINT(42.841658207380135 -87.4127660463611) bank153868 +153869 POINT(41.37109195641604 -87.40842099185372) bank153869 +153870 POINT(41.785442394308504 -86.69669127838861) bank153870 +153871 POINT(42.434396324834395 -87.69401176397803) bank153871 +153872 POINT(41.34383221823315 -87.70859419432472) bank153872 +153873 POINT(42.16075900885728 -88.06748088319677) bank153873 +153874 POINT(41.39197768270841 -87.39026548886318) bank153874 +153875 POINT(42.76101908048643 -87.2321338529706) bank153875 +153876 POINT(42.39885474408655 -88.2129563324922) bank153876 +153877 POINT(42.14282089120461 -87.64464400728568) bank153877 +153878 POINT(42.05193975898878 -87.94652428210854) bank153878 +153879 POINT(42.65623864069259 -87.86002800624544) bank153879 +153880 POINT(42.249169529263504 -88.07365663921891) bank153880 +153881 POINT(40.97756858988194 -86.87779466257234) bank153881 +153882 POINT(42.246113177842496 -87.74935185541251) bank153882 +153883 POINT(42.060003412206875 -87.3738866330938) bank153883 +153884 POINT(41.71213656895417 -88.35659265473521) bank153884 +153885 POINT(42.65580708025191 -88.4803398824069) bank153885 +153886 POINT(42.11676653542604 -87.93949541570035) bank153886 +153887 POINT(42.072484780185015 -87.23056823716658) bank153887 +153888 POINT(41.8668381717945 -86.65009351360719) bank153888 +153889 POINT(42.83793925518557 -88.02851962327374) bank153889 +153890 POINT(42.57168062003652 -88.20444339310863) bank153890 +153891 POINT(41.831191815069545 -87.63669164267284) bank153891 +153892 POINT(42.51058818575634 -87.95287955557049) bank153892 +153893 POINT(41.89179900438528 -87.82036502905125) bank153893 +153894 POINT(41.469267692442486 -88.38117888540957) bank153894 +153895 POINT(41.60220072722212 -87.45314226311255) bank153895 +153896 POINT(41.706746396622535 -87.6608800451566) bank153896 +153897 POINT(41.71927705381381 -88.48240241273751) bank153897 +153898 POINT(42.61205986747663 -86.83049529840552) bank153898 +153899 POINT(42.48034054773621 -88.55262092798657) bank153899 +153900 POINT(41.00788639227078 -87.40404529223844) bank153900 +153901 POINT(41.366329113964 -88.11833479403948) bank153901 +153902 POINT(42.06015962287523 -86.8862094892451) bank153902 +153903 POINT(41.81347997360058 -87.05003423762632) bank153903 +153904 POINT(41.03680840135605 -86.88856944539793) bank153904 +153905 POINT(41.686001499875275 -87.91028650421657) bank153905 +153906 POINT(41.17272598021698 -88.46781506540411) bank153906 +153907 POINT(41.40777042459098 -87.29051160560925) bank153907 +153908 POINT(41.30142337278488 -87.81996391867212) bank153908 +153909 POINT(42.435165058688064 -87.60928291074765) bank153909 +153910 POINT(41.265935957589086 -87.77018020483291) bank153910 +153911 POINT(42.12358332860657 -86.77852264795862) bank153911 +153912 POINT(40.90586671088752 -87.71757926610672) bank153912 +153913 POINT(41.53213502884045 -88.36462919166031) bank153913 +153914 POINT(42.01455248289707 -87.38885370138452) bank153914 +153915 POINT(41.59346001476082 -87.34374776293352) bank153915 +153916 POINT(41.84410493999164 -87.06946429517022) bank153916 +153917 POINT(41.99748452523679 -88.44125519171845) bank153917 +153918 POINT(41.10355399690074 -87.54713721279087) bank153918 +153919 POINT(42.49320207255568 -87.52391598946289) bank153919 +153920 POINT(41.31700194607099 -87.18052234262468) bank153920 +153921 POINT(41.54030323419037 -88.00927159687261) bank153921 +153922 POINT(42.72503989281292 -88.04440950562677) bank153922 +153923 POINT(42.11577977526633 -87.62397312177517) bank153923 +153924 POINT(41.89640213108387 -87.49885574901975) bank153924 +153925 POINT(41.27576211393456 -88.46221372812091) bank153925 +153926 POINT(42.15331945199189 -86.71887812880729) bank153926 +153927 POINT(41.83444142262325 -86.65477643123157) bank153927 +153928 POINT(40.8905251573974 -88.15578947771564) bank153928 +153929 POINT(42.802132150416895 -86.94500805817252) bank153929 +153930 POINT(41.41775016592448 -87.58096415229669) bank153930 +153931 POINT(41.29976417745835 -87.60320570148505) bank153931 +153932 POINT(41.97212383546639 -86.70536199101258) bank153932 +153933 POINT(41.70311098403013 -88.11978540493922) bank153933 +153934 POINT(41.064626612870995 -87.5017176882282) bank153934 +153935 POINT(42.648755234567126 -88.14660998193092) bank153935 +153936 POINT(41.41438566198711 -87.59633336214556) bank153936 +153937 POINT(41.26875271313256 -88.112585659224) bank153937 +153938 POINT(41.00862567210167 -87.40772953426774) bank153938 +153939 POINT(42.82612493223911 -87.69419467714658) bank153939 +153940 POINT(42.54121551989771 -87.4183810698101) bank153940 +153941 POINT(41.06857054404983 -87.64756458599186) bank153941 +153942 POINT(42.3501640018976 -86.98115074905989) bank153942 +153943 POINT(41.05683631579473 -86.73203298657506) bank153943 +153944 POINT(42.365091998785815 -87.80369487255402) bank153944 +153945 POINT(41.84655177394496 -88.16530283591851) bank153945 +153946 POINT(41.71942845576632 -88.15660004074965) bank153946 +153947 POINT(41.048268077769535 -88.4161744600265) bank153947 +153948 POINT(41.288301754016025 -86.75402834160475) bank153948 +153949 POINT(42.37064022561135 -88.36084368770379) bank153949 +153950 POINT(41.824832430675315 -86.98631392877793) bank153950 +153951 POINT(41.70295848864085 -87.80529400135464) bank153951 +153952 POINT(41.25783766591478 -87.75777989856822) bank153952 +153953 POINT(42.3649222884523 -87.19748332795412) bank153953 +153954 POINT(41.57980418846695 -86.74703427531799) bank153954 +153955 POINT(41.43662772088403 -88.45267030364855) bank153955 +153956 POINT(40.91180209975245 -86.74674436873084) bank153956 +153957 POINT(42.33427935965196 -88.56845761323792) bank153957 +153958 POINT(42.65239010811406 -86.78145978961257) bank153958 +153959 POINT(41.05258861916355 -87.00847469624851) bank153959 +153960 POINT(40.984542130226195 -88.26015194303967) bank153960 +153961 POINT(42.845602367786434 -87.09067325305865) bank153961 +153962 POINT(41.320478660005364 -88.18847391131042) bank153962 +153963 POINT(42.12901480263928 -88.42069754465132) bank153963 +153964 POINT(42.58952695113483 -87.31624001757463) bank153964 +153965 POINT(41.86149307699516 -88.51249847041846) bank153965 +153966 POINT(42.396319256812916 -87.53266672124741) bank153966 +153967 POINT(42.20725335499949 -87.45777642609663) bank153967 +153968 POINT(41.8917149522642 -87.54356563358047) bank153968 +153969 POINT(42.554348266047214 -86.7181202333961) bank153969 +153970 POINT(42.50424567128691 -88.11527049190809) bank153970 +153971 POINT(42.857013049808295 -88.22522008740779) bank153971 +153972 POINT(40.92802026071389 -87.69087371920673) bank153972 +153973 POINT(41.12126665818453 -87.53556994030862) bank153973 +153974 POINT(42.70533765820592 -87.0173538010917) bank153974 +153975 POINT(41.407938371460126 -87.74214916267816) bank153975 +153976 POINT(42.85223032261137 -87.50575188731707) bank153976 +153977 POINT(42.3476404209981 -88.0972565671611) bank153977 +153978 POINT(42.76951344832798 -87.86884908336869) bank153978 +153979 POINT(41.62606746599438 -86.89491092940067) bank153979 +153980 POINT(42.623508465472845 -87.65040251427746) bank153980 +153981 POINT(42.25646363968263 -88.15594144496305) bank153981 +153982 POINT(41.3137771024285 -86.92276488416185) bank153982 +153983 POINT(41.55919572828126 -87.547437151613) bank153983 +153984 POINT(41.830250769471775 -87.66468871562236) bank153984 +153985 POINT(42.05396719952567 -86.8786307560821) bank153985 +153986 POINT(40.949574195791726 -88.21725956110704) bank153986 +153987 POINT(41.27218429276107 -86.99036275225116) bank153987 +153988 POINT(41.17585891675993 -88.38061487172423) bank153988 +153989 POINT(41.08984840432872 -87.00223012765642) bank153989 +153990 POINT(41.03189749903985 -87.82157899302352) bank153990 +153991 POINT(41.06232453417743 -88.01556314987604) bank153991 +153992 POINT(42.48532378451634 -87.31756337091274) bank153992 +153993 POINT(42.06773638068511 -87.37733676756457) bank153993 +153994 POINT(41.530894126991456 -88.5524577861647) bank153994 +153995 POINT(42.01477315465198 -87.4584923167646) bank153995 +153996 POINT(42.57768794559248 -87.84322207170204) bank153996 +153997 POINT(42.42134600674248 -86.9259118915921) bank153997 +153998 POINT(42.15018267367454 -88.13478463333227) bank153998 +153999 POINT(42.78247366306325 -87.18878989159525) bank153999 +154000 POINT(42.40459571969989 -87.8846661503822) bank154000 +154001 POINT(41.61926816571498 -87.04737411623817) bank154001 +154002 POINT(41.26422426020231 -88.5242165686591) bank154002 +154003 POINT(42.26383119533363 -86.68359188856024) bank154003 +154004 POINT(42.6571029884317 -87.92224252216508) bank154004 +154005 POINT(42.119163848637065 -86.63399484465917) bank154005 +154006 POINT(41.887584386568264 -88.3593736266262) bank154006 +154007 POINT(42.014599477555386 -86.85567177777693) bank154007 +154008 POINT(41.589300056229995 -86.64208180525742) bank154008 +154009 POINT(41.35335012631398 -88.60875991879963) bank154009 +154010 POINT(41.194432902154624 -87.70269818001273) bank154010 +154011 POINT(40.994121330169214 -88.46926069805357) bank154011 +154012 POINT(41.066427811448385 -87.10123715269872) bank154012 +154013 POINT(42.870067289268796 -87.26567124316492) bank154013 +154014 POINT(41.58899914165838 -87.53348929442211) bank154014 +154015 POINT(41.224842868235925 -87.82211559171381) bank154015 +154016 POINT(42.82912160751816 -87.27585286362817) bank154016 +154017 POINT(42.77103477281654 -87.89407038973091) bank154017 +154018 POINT(42.01184771795221 -87.13625943954999) bank154018 +154019 POINT(41.716799735366976 -87.63806612512452) bank154019 +154020 POINT(42.55867241343412 -86.63812225080912) bank154020 +154021 POINT(41.834181623898814 -87.40240221731591) bank154021 +154022 POINT(41.275342222889336 -88.59389005207315) bank154022 +154023 POINT(41.741457604749854 -87.71392409516649) bank154023 +154024 POINT(41.95749513522757 -88.17493514818736) bank154024 +154025 POINT(41.57006330350504 -88.0068765599312) bank154025 +154026 POINT(41.066354974416406 -86.92985751862581) bank154026 +154027 POINT(41.76831818568665 -87.33064273964128) bank154027 +154028 POINT(42.25667095194292 -86.78262106745485) bank154028 +154029 POINT(42.813454573770144 -87.74728595368961) bank154029 +154030 POINT(41.769079395741414 -87.88511870566539) bank154030 +154031 POINT(41.87173313946007 -88.0973147073208) bank154031 +154032 POINT(41.45939798825907 -88.56274946951575) bank154032 +154033 POINT(41.25157205598554 -87.38067433589529) bank154033 +154034 POINT(42.08368310278356 -88.11428964816795) bank154034 +154035 POINT(41.592203418155755 -87.21396044365602) bank154035 +154036 POINT(41.605508440835834 -87.14509848529572) bank154036 +154037 POINT(41.7214301391762 -87.75466713214938) bank154037 +154038 POINT(42.52417787043652 -88.39749094405553) bank154038 +154039 POINT(41.91695546105443 -87.75292404333155) bank154039 +154040 POINT(41.52808056129921 -88.22659123057628) bank154040 +154041 POINT(41.21085203985068 -88.47299838639614) bank154041 +154042 POINT(42.38349622960935 -87.69394965431796) bank154042 +154043 POINT(41.34578033362426 -87.4622389495796) bank154043 +154044 POINT(41.407772977532616 -87.44855740375164) bank154044 +154045 POINT(41.742662678468044 -88.34411465663331) bank154045 +154046 POINT(42.8052086390898 -87.003832096801) bank154046 +154047 POINT(41.51414779891844 -87.86317698186845) bank154047 +154048 POINT(42.17066922328166 -87.20253430095723) bank154048 +154049 POINT(41.38425702915862 -86.68875500285722) bank154049 +154050 POINT(42.14131846562172 -87.41705210709571) bank154050 +154051 POINT(41.05593472460556 -88.39150522274961) bank154051 +154052 POINT(41.8801705905446 -88.49051438098499) bank154052 +154053 POINT(42.225540833631115 -87.77319681271669) bank154053 +154054 POINT(41.80775084320352 -88.13939624957236) bank154054 +154055 POINT(41.474205546130044 -86.89900561210422) bank154055 +154056 POINT(42.73029762856267 -88.42718814031171) bank154056 +154057 POINT(41.51453647040206 -86.83447998316741) bank154057 +154058 POINT(42.71250492145679 -86.99693695236242) bank154058 +154059 POINT(42.09479138499049 -87.46244504159469) bank154059 +154060 POINT(42.54130609112873 -87.0092173182934) bank154060 +154061 POINT(41.05094454396535 -87.94426073136249) bank154061 +154062 POINT(42.08921172498904 -87.90653579123409) bank154062 +154063 POINT(42.571274172972224 -88.04862824747143) bank154063 +154064 POINT(41.716242846354 -87.91322901254772) bank154064 +154065 POINT(42.600459658115916 -87.82277949119873) bank154065 +154066 POINT(41.52832221821786 -87.17497897516222) bank154066 +154067 POINT(41.29440539979847 -87.27077341565688) bank154067 +154068 POINT(42.05301153159265 -86.67845536648774) bank154068 +154069 POINT(41.50376218457669 -87.56885685427) bank154069 +154070 POINT(41.441880789864996 -87.67532265077219) bank154070 +154071 POINT(40.96919891795652 -87.12322343819712) bank154071 +154072 POINT(41.93378840942575 -88.5311225174774) bank154072 +154073 POINT(41.21422764455238 -86.7044270917713) bank154073 +154074 POINT(41.59345443919237 -87.95940976298755) bank154074 +154075 POINT(42.26746517457895 -87.88701827727726) bank154075 +154076 POINT(41.01676853717246 -87.32310803519402) bank154076 +154077 POINT(41.322842077512036 -86.90291170240346) bank154077 +154078 POINT(42.872353515111904 -88.06638662265365) bank154078 +154079 POINT(41.52586824026935 -88.46832757482076) bank154079 +154080 POINT(42.542317882780196 -87.40803149982139) bank154080 +154081 POINT(41.78738336799987 -87.60560919771729) bank154081 +154082 POINT(42.71867193004471 -88.42858106583344) bank154082 +154083 POINT(42.431978813485614 -86.72634795091258) bank154083 +154084 POINT(40.89601035449959 -86.84580306362815) bank154084 +154085 POINT(40.904813318809275 -87.48223550906212) bank154085 +154086 POINT(42.28158858999293 -87.57957468959704) bank154086 +154087 POINT(41.8333959112949 -88.183944245805) bank154087 +154088 POINT(41.38098128802144 -86.66871874661291) bank154088 +154089 POINT(41.988637937441005 -88.11832095520388) bank154089 +154090 POINT(41.79421144651307 -87.04230681062907) bank154090 +154091 POINT(41.77443676622986 -86.7632503315483) bank154091 +154092 POINT(41.43532635357553 -88.51652360025365) bank154092 +154093 POINT(41.106903161834914 -86.9481548790538) bank154093 +154094 POINT(42.31536578998181 -86.69336013071273) bank154094 +154095 POINT(42.58579846212759 -87.78641167640176) bank154095 +154096 POINT(41.483064266199136 -88.06595541156415) bank154096 +154097 POINT(42.50580592390767 -87.18721890059196) bank154097 +154098 POINT(41.27300023251104 -87.05887763871107) bank154098 +154099 POINT(41.533143737027956 -87.51098245791697) bank154099 +154100 POINT(41.21227035290145 -87.2456743867414) bank154100 +154101 POINT(42.457008989565495 -86.85663065244778) bank154101 +154102 POINT(42.0136388251586 -87.8484182164813) bank154102 +154103 POINT(41.55229841594133 -86.73619828070572) bank154103 +154104 POINT(41.56565215227347 -88.32957441684127) bank154104 +154105 POINT(42.143214887704325 -88.00237304511533) bank154105 +154106 POINT(41.300645540996264 -87.22203765992109) bank154106 +154107 POINT(41.35145278293934 -87.89452409003697) bank154107 +154108 POINT(41.386245412783495 -86.68102244459763) bank154108 +154109 POINT(42.71260717134499 -88.58281354812816) bank154109 +154110 POINT(41.6115712870003 -87.27313470814573) bank154110 +154111 POINT(41.09534462173358 -87.26072426786281) bank154111 +154112 POINT(41.08872934708652 -86.75005164819027) bank154112 +154113 POINT(42.03375383159063 -88.40541880040206) bank154113 +154114 POINT(41.743514384087895 -86.75215649086857) bank154114 +154115 POINT(42.505896604388504 -87.11774890224127) bank154115 +154116 POINT(41.657525810245104 -88.56806818216042) bank154116 +154117 POINT(41.829343842825494 -87.59754935012822) bank154117 +154118 POINT(40.89130489871042 -86.84537325683101) bank154118 +154119 POINT(41.41456693100929 -87.68573534393495) bank154119 +154120 POINT(41.54042681501238 -87.70007045497205) bank154120 +154121 POINT(42.40146568737679 -87.32199629931334) bank154121 +154122 POINT(41.30357030117758 -87.35127980162353) bank154122 +154123 POINT(41.01901200631547 -88.31902880716346) bank154123 +154124 POINT(41.50644061716436 -87.01929395942452) bank154124 +154125 POINT(42.580201040043136 -87.50627508277596) bank154125 +154126 POINT(42.2706664307412 -87.1240320400802) bank154126 +154127 POINT(42.31189459962128 -88.32186392505928) bank154127 +154128 POINT(42.7783371002582 -87.17633312712725) bank154128 +154129 POINT(42.40805839016893 -86.6806434155596) bank154129 +154130 POINT(41.55565043399462 -88.05023284239806) bank154130 +154131 POINT(42.79629479658743 -86.83919868539434) bank154131 +154132 POINT(41.68727880986009 -87.75255665352593) bank154132 +154133 POINT(41.61905810955943 -87.58176977305388) bank154133 +154134 POINT(42.24848480517666 -87.45507189267875) bank154134 +154135 POINT(40.89525300567415 -87.99946627016523) bank154135 +154136 POINT(41.9186630988724 -87.77247670260364) bank154136 +154137 POINT(42.52869563575037 -87.93393407659416) bank154137 +154138 POINT(42.3250175756063 -87.86921912380524) bank154138 +154139 POINT(42.375224517271064 -86.68254777238444) bank154139 +154140 POINT(42.70376045634352 -87.90256017559261) bank154140 +154141 POINT(41.23753850463102 -87.50148232471116) bank154141 +154142 POINT(41.58483267090802 -87.1118964512701) bank154142 +154143 POINT(42.33308355424849 -87.6367860793506) bank154143 +154144 POINT(40.96441946875966 -88.04925691583851) bank154144 +154145 POINT(41.542645736645774 -87.30991563181863) bank154145 +154146 POINT(41.248034602192384 -88.21146365844537) bank154146 +154147 POINT(42.54682370678271 -88.54981012759016) bank154147 +154148 POINT(41.61480623252769 -87.3780473215708) bank154148 +154149 POINT(40.94729690318843 -88.43033198731499) bank154149 +154150 POINT(42.61333335656854 -87.47657627494316) bank154150 +154151 POINT(42.40264151677356 -86.75572588092696) bank154151 +154152 POINT(41.981897085902546 -87.10910819475525) bank154152 +154153 POINT(41.967937246635856 -87.32802304806107) bank154153 +154154 POINT(41.13367726240096 -88.31663140261183) bank154154 +154155 POINT(41.074143992999794 -88.54720775087314) bank154155 +154156 POINT(42.29598425739691 -87.99734579822393) bank154156 +154157 POINT(41.975164888389614 -87.9245126241026) bank154157 +154158 POINT(41.19487264937057 -88.24782705514448) bank154158 +154159 POINT(42.54808410832846 -88.0040450481479) bank154159 +154160 POINT(41.56764199256272 -87.48258170881509) bank154160 +154161 POINT(41.23587806698853 -87.80716359448483) bank154161 +154162 POINT(40.94582041533577 -86.71654510839629) bank154162 +154163 POINT(41.7837255499829 -88.24115645926662) bank154163 +154164 POINT(42.22272132955877 -88.3504435007218) bank154164 +154165 POINT(42.50123741543038 -87.16597715480711) bank154165 +154166 POINT(42.01995689518373 -86.86225788375165) bank154166 +154167 POINT(41.743364463041615 -86.89118761066207) bank154167 +154168 POINT(41.16621195008822 -87.77406515911191) bank154168 +154169 POINT(42.33766221383553 -87.79847689477839) bank154169 +154170 POINT(41.22751694306435 -86.85504157109249) bank154170 +154171 POINT(41.401975670238514 -86.76189512810377) bank154171 +154172 POINT(41.97553724642797 -86.84280928829517) bank154172 +154173 POINT(42.537753599185244 -88.57603699622017) bank154173 +154174 POINT(41.51097689585272 -87.8699695723023) bank154174 +154175 POINT(41.15125059510146 -87.73130481616954) bank154175 +154176 POINT(41.540175057388005 -87.97091928658605) bank154176 +154177 POINT(41.474791074354876 -88.55717693559201) bank154177 +154178 POINT(41.941269092349025 -87.89065870120554) bank154178 +154179 POINT(41.39666653631078 -87.63101212174955) bank154179 +154180 POINT(42.213861501791435 -87.62414572528904) bank154180 +154181 POINT(41.77305655369213 -87.94755599015659) bank154181 +154182 POINT(41.6083711262109 -86.72752833486487) bank154182 +154183 POINT(41.82388379341329 -88.53023780807278) bank154183 +154184 POINT(42.01679928232325 -86.73972321253443) bank154184 +154185 POINT(42.628752157986256 -87.59349435695275) bank154185 +154186 POINT(41.20162901766991 -86.80308360819697) bank154186 +154187 POINT(41.96199720918784 -87.6141513621009) bank154187 +154188 POINT(41.156979806910485 -86.8259993661918) bank154188 +154189 POINT(42.836297109367386 -87.36131918668052) bank154189 +154190 POINT(42.037382061959384 -88.36569862826644) bank154190 +154191 POINT(41.74513212560889 -88.01173200131316) bank154191 +154192 POINT(42.356169441188555 -87.94074024005333) bank154192 +154193 POINT(41.52732676883002 -86.92051694119814) bank154193 +154194 POINT(42.39516718863943 -86.76176654372767) bank154194 +154195 POINT(41.54023877021592 -87.56205330181406) bank154195 +154196 POINT(41.477459316594775 -87.53953191119659) bank154196 +154197 POINT(42.79419637979836 -87.43392614789073) bank154197 +154198 POINT(42.42083087633024 -87.00910979647065) bank154198 +154199 POINT(41.426640041688046 -87.93930156076233) bank154199 +154200 POINT(41.180716160397004 -88.17969111379597) bank154200 +154201 POINT(42.622182251186956 -88.20219638508713) bank154201 +154202 POINT(41.184948966959844 -87.40074485850248) bank154202 +154203 POINT(41.48225386627802 -88.514782470226) bank154203 +154204 POINT(42.694351524781815 -86.92760128776348) bank154204 +154205 POINT(41.885916419046744 -87.69723762310565) bank154205 +154206 POINT(42.51694750793061 -86.83820969452245) bank154206 +154207 POINT(42.11606416296273 -87.58770000412886) bank154207 +154208 POINT(42.012277670801005 -87.12917618301852) bank154208 +154209 POINT(40.9166519100062 -88.19452818799992) bank154209 +154210 POINT(41.05298086996107 -88.02606924286147) bank154210 +154211 POINT(41.78708898781019 -87.16126193756801) bank154211 +154212 POINT(41.250839877981406 -88.18876294228103) bank154212 +154213 POINT(41.457824157717425 -88.35249768815207) bank154213 +154214 POINT(41.874797090564606 -87.22296784646853) bank154214 +154215 POINT(41.48170510700943 -88.30923319527622) bank154215 +154216 POINT(42.00066892252061 -87.05243483738501) bank154216 +154217 POINT(42.38608976519391 -87.01108641555317) bank154217 +154218 POINT(41.74229554802861 -88.07300014492749) bank154218 +154219 POINT(42.61252431496337 -88.34350837270985) bank154219 +154220 POINT(41.363316210086055 -88.45068843619862) bank154220 +154221 POINT(41.6693454038416 -88.24806871770647) bank154221 +154222 POINT(41.71714776229473 -86.85415469871941) bank154222 +154223 POINT(42.35873584150086 -87.96523970807652) bank154223 +154224 POINT(42.531444577387255 -87.38939446430034) bank154224 +154225 POINT(41.59103492342777 -87.21277556610045) bank154225 +154226 POINT(42.50483363253197 -87.57606350847563) bank154226 +154227 POINT(41.23200491610253 -88.33573413533819) bank154227 +154228 POINT(40.94054419546858 -87.75411496123493) bank154228 +154229 POINT(41.849377890710514 -88.03735139430455) bank154229 +154230 POINT(42.45494514901474 -88.26417216252128) bank154230 +154231 POINT(41.90364275866785 -88.41316115934863) bank154231 +154232 POINT(41.46840452893725 -87.05870062806592) bank154232 +154233 POINT(41.40064620478265 -87.20814203553442) bank154233 +154234 POINT(41.838989361028624 -87.55039667257293) bank154234 +154235 POINT(41.76003053553894 -86.96027257223095) bank154235 +154236 POINT(42.307068845945295 -87.10852391402906) bank154236 +154237 POINT(42.67054884443584 -87.20224519455184) bank154237 +154238 POINT(41.2574098346205 -88.31295796441353) bank154238 +154239 POINT(41.12829675225555 -88.0986369896298) bank154239 +154240 POINT(41.7991629180589 -87.12693971439842) bank154240 +154241 POINT(42.79830675213131 -86.77429589532298) bank154241 +154242 POINT(42.175721533756644 -87.04002025448588) bank154242 +154243 POINT(42.19423799486594 -88.16754256069034) bank154243 +154244 POINT(41.968731035970094 -87.96094980838839) bank154244 +154245 POINT(41.72113527639975 -87.97177456327509) bank154245 +154246 POINT(42.10755155622711 -86.69031156611929) bank154246 +154247 POINT(42.089374756862725 -86.63370350785726) bank154247 +154248 POINT(41.784049237300074 -87.97831078923302) bank154248 +154249 POINT(40.97136390661636 -87.09663777222164) bank154249 +154250 POINT(41.152932512658545 -87.0484701402469) bank154250 +154251 POINT(41.96935488438016 -86.99977408151095) bank154251 +154252 POINT(41.98107781291221 -88.33009787602141) bank154252 +154253 POINT(41.72616215894338 -87.22533161794675) bank154253 +154254 POINT(41.960967076176146 -88.30452306466283) bank154254 +154255 POINT(41.91802174709367 -88.39203390800404) bank154255 +154256 POINT(42.633749486738914 -88.27717644447472) bank154256 +154257 POINT(42.23853746239249 -88.48382113467932) bank154257 +154258 POINT(41.34258039069667 -87.48506490439419) bank154258 +154259 POINT(42.627938672351966 -87.13915396587107) bank154259 +154260 POINT(42.39057401150755 -87.66829829582583) bank154260 +154261 POINT(42.1927123590324 -88.59823589366717) bank154261 +154262 POINT(41.64750106667087 -88.14887014658949) bank154262 +154263 POINT(41.71755220831725 -86.64522083152409) bank154263 +154264 POINT(42.5120677228558 -86.71097955302972) bank154264 +154265 POINT(42.138948292629394 -87.26532865109016) bank154265 +154266 POINT(41.353036002660915 -88.56911338748752) bank154266 +154267 POINT(42.769782251657176 -88.19089546161734) bank154267 +154268 POINT(42.42692579598738 -86.7625872709071) bank154268 +154269 POINT(42.06914010336318 -87.55271040376536) bank154269 +154270 POINT(42.687829416877065 -88.35961937292677) bank154270 +154271 POINT(42.78485393410722 -88.06024330265767) bank154271 +154272 POINT(40.958271519336414 -87.82026069239626) bank154272 +154273 POINT(41.900501352024385 -87.8799741510306) bank154273 +154274 POINT(42.746839758289696 -86.99469226783445) bank154274 +154275 POINT(42.10473021662163 -88.22484622513795) bank154275 +154276 POINT(41.81455666029871 -87.36263197255684) bank154276 +154277 POINT(42.81351134941955 -87.34517509178717) bank154277 +154278 POINT(41.83488816642067 -86.94079435161831) bank154278 +154279 POINT(41.30401194041026 -87.39489736136393) bank154279 +154280 POINT(41.81898096228806 -88.0463819253242) bank154280 +154281 POINT(42.01776349228208 -88.20107421495102) bank154281 +154282 POINT(41.65316161857792 -87.37427950186682) bank154282 +154283 POINT(42.550362065061705 -87.65763058359494) bank154283 +154284 POINT(41.08846642673307 -88.59364918252565) bank154284 +154285 POINT(40.939945128888894 -88.08687384724513) bank154285 +154286 POINT(42.35302602260073 -86.82687566208439) bank154286 +154287 POINT(41.495450154288555 -88.43898658864933) bank154287 +154288 POINT(42.84556068215452 -88.57597077971742) bank154288 +154289 POINT(41.52471973790126 -86.75697536469386) bank154289 +154290 POINT(41.1084025838518 -86.72362717210083) bank154290 +154291 POINT(42.765563620158716 -87.21931527433226) bank154291 +154292 POINT(41.02514144328596 -87.40998122070405) bank154292 +154293 POINT(42.448899337611785 -87.11270420376985) bank154293 +154294 POINT(41.2858647111738 -87.53930430387179) bank154294 +154295 POINT(42.613683142849055 -87.36160753141412) bank154295 +154296 POINT(40.8860927937515 -87.97679049244458) bank154296 +154297 POINT(41.73764978257087 -88.36280756525574) bank154297 +154298 POINT(42.79754897844476 -87.9027212206802) bank154298 +154299 POINT(41.50081323568304 -88.31793305636499) bank154299 +154300 POINT(41.45889389298772 -87.316353367776) bank154300 +154301 POINT(41.096574282791245 -87.907816416114) bank154301 +154302 POINT(41.47224445361863 -86.96480437034512) bank154302 +154303 POINT(42.87345593975705 -88.29725585288001) bank154303 +154304 POINT(40.909304770140366 -86.79589027177134) bank154304 +154305 POINT(42.730022594453295 -88.5982531709719) bank154305 +154306 POINT(42.417631969250564 -87.20902878081306) bank154306 +154307 POINT(42.852010878616355 -86.99850306978911) bank154307 +154308 POINT(41.87150787943763 -87.6828419121039) bank154308 +154309 POINT(42.12739656200391 -87.21521642131) bank154309 +154310 POINT(41.8918743849829 -88.48364692874718) bank154310 +154311 POINT(42.26556602676807 -86.76615441327817) bank154311 +154312 POINT(42.171179450876345 -87.44533356402485) bank154312 +154313 POINT(42.851735096283974 -88.0679025353144) bank154313 +154314 POINT(42.01307160876428 -87.51710721720949) bank154314 +154315 POINT(41.06736736066856 -87.72792039271323) bank154315 +154316 POINT(42.26125367190407 -87.56674279154292) bank154316 +154317 POINT(42.433493510963395 -88.31569967961596) bank154317 +154318 POINT(42.177346626571776 -88.28026784475342) bank154318 +154319 POINT(42.463005116268505 -87.11982241274235) bank154319 +154320 POINT(42.03354871944903 -87.34572796584096) bank154320 +154321 POINT(41.478484910671746 -88.27205943226733) bank154321 +154322 POINT(41.975601640604665 -87.1718809684346) bank154322 +154323 POINT(41.23560751915629 -87.31176610379039) bank154323 +154324 POINT(41.93641935018362 -86.76367139722998) bank154324 +154325 POINT(41.421771201701226 -87.3827688236248) bank154325 +154326 POINT(42.31819207385955 -87.38712887143873) bank154326 +154327 POINT(41.70394881966721 -88.43122179174198) bank154327 +154328 POINT(42.58885630904366 -87.58086699228785) bank154328 +154329 POINT(41.69692624453972 -87.5646692879976) bank154329 +154330 POINT(42.342867821286404 -87.85247668086917) bank154330 +154331 POINT(41.12256796008137 -88.62848685831217) bank154331 +154332 POINT(40.91498474901177 -88.56676413162438) bank154332 +154333 POINT(41.939091182714 -88.37015694418383) bank154333 +154334 POINT(41.364193645999705 -87.24832260781568) bank154334 +154335 POINT(41.07043060791172 -88.15979747450668) bank154335 +154336 POINT(41.50469082441532 -88.00768332213887) bank154336 +154337 POINT(41.75582841658023 -87.95166052574687) bank154337 +154338 POINT(42.8668383306825 -87.11406265118379) bank154338 +154339 POINT(41.2890371399794 -87.59931068446572) bank154339 +154340 POINT(41.34245165750869 -87.10827743935629) bank154340 +154341 POINT(42.602787868984734 -88.2911158224755) bank154341 +154342 POINT(42.2154786469026 -87.98351145323055) bank154342 +154343 POINT(41.54428452019282 -87.65355610961666) bank154343 +154344 POINT(42.307684144501046 -87.27018055664732) bank154344 +154345 POINT(40.88513550617939 -87.88459326495962) bank154345 +154346 POINT(42.15875788739986 -88.4105331197297) bank154346 +154347 POINT(42.410545900097915 -87.29780106255389) bank154347 +154348 POINT(41.57105345441964 -87.41450943741572) bank154348 +154349 POINT(41.163626899909055 -88.29783283015657) bank154349 +154350 POINT(42.36747926834191 -87.84525325376416) bank154350 +154351 POINT(40.91433032333567 -87.12681200062877) bank154351 +154352 POINT(42.37694577695806 -87.16847639326026) bank154352 +154353 POINT(42.629649976378346 -87.97225709133257) bank154353 +154354 POINT(41.7155448429527 -88.25409808370121) bank154354 +154355 POINT(41.63069388787123 -86.71802177226407) bank154355 +154356 POINT(41.26118612116962 -87.86921959259533) bank154356 +154357 POINT(42.48933363537283 -86.9796066920066) bank154357 +154358 POINT(41.828448413216904 -86.79446783780276) bank154358 +154359 POINT(42.703033397072424 -88.0671660676657) bank154359 +154360 POINT(41.54920213398695 -87.00225205163498) bank154360 +154361 POINT(41.22267118354859 -86.87543297890959) bank154361 +154362 POINT(41.73256074929142 -87.03139654077498) bank154362 +154363 POINT(41.871676458729546 -88.5067681114852) bank154363 +154364 POINT(41.518259427710476 -86.82073863654776) bank154364 +154365 POINT(41.99269945638916 -87.78629613690427) bank154365 +154366 POINT(42.28927547461707 -86.64240690115322) bank154366 +154367 POINT(41.952294450772804 -87.96030105615121) bank154367 +154368 POINT(42.816755382219675 -87.79664104347661) bank154368 +154369 POINT(41.783671977746664 -88.56800917171641) bank154369 +154370 POINT(41.77338313506004 -87.78200560864018) bank154370 +154371 POINT(41.00426931380041 -88.3009429480507) bank154371 +154372 POINT(41.0789756473682 -87.84741494715396) bank154372 +154373 POINT(42.29584112478118 -87.6472444826723) bank154373 +154374 POINT(41.64388587888804 -87.43440933181387) bank154374 +154375 POINT(40.900038761212514 -87.86769413553131) bank154375 +154376 POINT(41.28510525130883 -87.07390651284639) bank154376 +154377 POINT(42.33660292346381 -88.58352081828822) bank154377 +154378 POINT(40.99518733064023 -87.10745510747277) bank154378 +154379 POINT(42.79149228799803 -87.92827683459276) bank154379 +154380 POINT(41.08559581161288 -86.81819184617625) bank154380 +154381 POINT(42.44187581974758 -87.62342106366305) bank154381 +154382 POINT(42.405577196163904 -87.4449149223335) bank154382 +154383 POINT(41.30321885334995 -87.51693664111774) bank154383 +154384 POINT(42.77052475578835 -87.56797687132642) bank154384 +154385 POINT(41.34473179048384 -87.7181571862792) bank154385 +154386 POINT(42.05633949158195 -88.07607875604467) bank154386 +154387 POINT(41.583835238863585 -86.77045874650277) bank154387 +154388 POINT(41.99961596067433 -87.44175840783305) bank154388 +154389 POINT(41.44483903512179 -86.8878096409169) bank154389 +154390 POINT(42.614434671118666 -87.9582621781069) bank154390 +154391 POINT(41.02366844705861 -88.46822213224249) bank154391 +154392 POINT(41.90646444053657 -87.79385940991146) bank154392 +154393 POINT(41.46751028432802 -86.78032542575144) bank154393 +154394 POINT(41.3093470795066 -88.49692836442165) bank154394 +154395 POINT(42.6559635558825 -88.58295393322011) bank154395 +154396 POINT(41.706983432638474 -86.71639384524899) bank154396 +154397 POINT(42.08969116840396 -87.07082969601613) bank154397 +154398 POINT(40.87854914624089 -87.76542568872571) bank154398 +154399 POINT(42.30821802011752 -88.42431572632988) bank154399 +154400 POINT(41.68240390113913 -87.38625055621617) bank154400 +154401 POINT(42.56036413388056 -86.90407454047548) bank154401 +154402 POINT(42.76626745527275 -88.44231087226923) bank154402 +154403 POINT(42.36260933474697 -88.25619515671015) bank154403 +154404 POINT(42.64726697037806 -87.79165485797279) bank154404 +154405 POINT(42.20621270261583 -87.94963918475229) bank154405 +154406 POINT(41.28772968696534 -87.57437822366978) bank154406 +154407 POINT(41.018008503818905 -87.59770753416724) bank154407 +154408 POINT(41.80915148842332 -87.06871218162011) bank154408 +154409 POINT(42.195999621241505 -87.83731388651498) bank154409 +154410 POINT(41.003951283533134 -86.89723894050887) bank154410 +154411 POINT(42.076316431753256 -87.21743090612566) bank154411 +154412 POINT(42.36639835987497 -87.22027688765104) bank154412 +154413 POINT(41.862760613535556 -88.33947279143119) bank154413 +154414 POINT(42.2180280799366 -87.1358109953681) bank154414 +154415 POINT(42.35921373777886 -87.33723728136489) bank154415 +154416 POINT(42.492157778108975 -87.06902853190829) bank154416 +154417 POINT(42.751848820653386 -87.05483910233373) bank154417 +154418 POINT(42.42483562065048 -87.15663464317105) bank154418 +154419 POINT(41.809170921385146 -86.80963082407146) bank154419 +154420 POINT(41.15460559850079 -87.3078294261819) bank154420 +154421 POINT(41.50272103320403 -87.52037821898048) bank154421 +154422 POINT(41.17759636738503 -88.61443452998526) bank154422 +154423 POINT(42.44145181193753 -86.97140076037222) bank154423 +154424 POINT(41.974556845281775 -88.31694648156677) bank154424 +154425 POINT(41.18367184215897 -87.71126983637315) bank154425 +154426 POINT(42.6281905132347 -87.25729098310175) bank154426 +154427 POINT(40.99512407255405 -88.12800410825977) bank154427 +154428 POINT(41.62322383005134 -88.57920956794274) bank154428 +154429 POINT(42.30959117481567 -88.392886140681) bank154429 +154430 POINT(41.022584828623195 -86.7502602716393) bank154430 +154431 POINT(41.699667837476895 -88.08562313791697) bank154431 +154432 POINT(41.87977256122118 -87.62974591854888) bank154432 +154433 POINT(40.96834075352401 -88.08502669553974) bank154433 +154434 POINT(42.77583750323416 -86.93382874617477) bank154434 +154435 POINT(41.57800484275125 -88.0897295106821) bank154435 +154436 POINT(42.5638928898265 -88.37431850976579) bank154436 +154437 POINT(41.87022941056554 -87.23391962388989) bank154437 +154438 POINT(42.56820405882299 -88.61118540276189) bank154438 +154439 POINT(42.486565568949004 -87.33536377985153) bank154439 +154440 POINT(41.631318960893516 -87.75248479113735) bank154440 +154441 POINT(41.24613166643152 -88.39566885031087) bank154441 +154442 POINT(42.83802855963629 -88.25470584944969) bank154442 +154443 POINT(41.54704087160495 -88.00023229531685) bank154443 +154444 POINT(42.46164993778645 -86.99313693849231) bank154444 +154445 POINT(41.83573118772877 -87.41050354448163) bank154445 +154446 POINT(40.90767941079854 -87.71475427586084) bank154446 +154447 POINT(42.43174631714959 -88.49651156310277) bank154447 +154448 POINT(41.03156663098037 -87.83643649177009) bank154448 +154449 POINT(42.22808262113906 -86.98219497858388) bank154449 +154450 POINT(41.08304619672494 -86.96670994162876) bank154450 +154451 POINT(41.59818946794373 -88.03249205355486) bank154451 +154452 POINT(42.864312159972826 -87.21566924374036) bank154452 +154453 POINT(41.01896257250193 -88.14513768129639) bank154453 +154454 POINT(42.37413979297047 -87.17222937997539) bank154454 +154455 POINT(41.90406035138535 -87.47300667737927) bank154455 +154456 POINT(42.133676444786545 -87.18481524181425) bank154456 +154457 POINT(41.76540280532028 -87.74960817578508) bank154457 +154458 POINT(40.914614150190424 -86.95093570120969) bank154458 +154459 POINT(42.25594147916892 -86.9074610683033) bank154459 +154460 POINT(42.4127942244737 -87.17382246015397) bank154460 +154461 POINT(41.214680022768334 -88.03730423895125) bank154461 +154462 POINT(42.301862858479645 -88.5624875775536) bank154462 +154463 POINT(41.51833162559758 -87.21317406295196) bank154463 +154464 POINT(42.59876949071987 -88.3444344278376) bank154464 +154465 POINT(41.9069333872609 -87.8431499619445) bank154465 +154466 POINT(42.4941309152416 -86.75847340822867) bank154466 +154467 POINT(42.31208797921014 -87.70054227464584) bank154467 +154468 POINT(41.89266347277908 -88.38736646780033) bank154468 +154469 POINT(41.1845949870667 -86.94960750467332) bank154469 +154470 POINT(42.389082438406746 -86.77859709574695) bank154470 +154471 POINT(41.100994164285346 -87.61052437092518) bank154471 +154472 POINT(42.01176662681341 -87.35657495530035) bank154472 +154473 POINT(42.86492415896257 -87.06788595384356) bank154473 +154474 POINT(42.08965048716637 -87.8133244883507) bank154474 +154475 POINT(41.083433853961765 -88.21214070645321) bank154475 +154476 POINT(41.38088093277276 -87.84074734418371) bank154476 +154477 POINT(42.52518798829964 -87.66298788801113) bank154477 +154478 POINT(41.35500673345636 -88.41819481745081) bank154478 +154479 POINT(41.188862491429475 -88.37733773315114) bank154479 +154480 POINT(42.753186452014546 -87.54343426798563) bank154480 +154481 POINT(41.93513412999763 -88.43675229054618) bank154481 +154482 POINT(41.724481426192 -87.47814345780712) bank154482 +154483 POINT(41.04111681735398 -87.03189207432243) bank154483 +154484 POINT(42.09270470481627 -87.5717150338487) bank154484 +154485 POINT(42.23519812607268 -88.23116996283802) bank154485 +154486 POINT(41.94670474347096 -87.50477983458337) bank154486 +154487 POINT(42.84545885145905 -87.39363872752553) bank154487 +154488 POINT(41.53302054601053 -88.0100425230621) bank154488 +154489 POINT(41.16830763922231 -88.55341145328546) bank154489 +154490 POINT(41.9421349932532 -87.62740445453927) bank154490 +154491 POINT(40.924699049565795 -87.00125802553997) bank154491 +154492 POINT(41.889611891737964 -88.07158504979544) bank154492 +154493 POINT(41.27024087987453 -87.32253530565899) bank154493 +154494 POINT(42.1096672603697 -87.3190446769598) bank154494 +154495 POINT(42.62536128351437 -87.5303065788367) bank154495 +154496 POINT(41.606073339596605 -88.09497662312849) bank154496 +154497 POINT(41.07421406642907 -87.21866206106935) bank154497 +154498 POINT(42.74256895003142 -87.25872298841611) bank154498 +154499 POINT(41.02936344838315 -88.05078317287328) bank154499 +154500 POINT(42.79483342758621 -88.58883704145858) bank154500 +154501 POINT(41.95248750686255 -88.09581323107807) bank154501 +154502 POINT(42.41693929283613 -87.97435667102134) bank154502 +154503 POINT(42.31637721972133 -88.23293360382785) bank154503 +154504 POINT(41.12993756269759 -87.25338856498286) bank154504 +154505 POINT(42.63728868709027 -87.43179750819027) bank154505 +154506 POINT(41.13632411761705 -86.76461577767853) bank154506 +154507 POINT(41.517897161122356 -88.52714878928897) bank154507 +154508 POINT(41.93706726118275 -87.65437834517657) bank154508 +154509 POINT(42.023171925462876 -87.0599165424055) bank154509 +154510 POINT(41.31290670641948 -87.50060979161171) bank154510 +154511 POINT(42.03966154718677 -88.59872870821754) bank154511 +154512 POINT(41.567243328723634 -86.8938218278407) bank154512 +154513 POINT(41.578018901828024 -88.06378221264482) bank154513 +154514 POINT(42.54198274762019 -88.2032508404064) bank154514 +154515 POINT(42.55189753140848 -88.51148629262325) bank154515 +154516 POINT(41.756579375977246 -88.02683110264215) bank154516 +154517 POINT(42.86971194087805 -88.54081683991546) bank154517 +154518 POINT(41.759102722546885 -88.23688266701906) bank154518 +154519 POINT(42.87199957994915 -87.35992807426045) bank154519 +154520 POINT(42.644573843779334 -88.51592277323174) bank154520 +154521 POINT(42.61699779405444 -87.23000096503934) bank154521 +154522 POINT(41.41882623146691 -87.58994023845861) bank154522 +154523 POINT(41.93200283668359 -87.56146353817489) bank154523 +154524 POINT(41.38314503706312 -87.31638569723287) bank154524 +154525 POINT(41.25500032295188 -87.96762710038034) bank154525 +154526 POINT(40.879883126908254 -88.47577270903646) bank154526 +154527 POINT(41.3735978907927 -88.26426064014747) bank154527 +154528 POINT(41.047317309513915 -88.02437779850911) bank154528 +154529 POINT(42.74562717427043 -87.28071367839394) bank154529 +154530 POINT(41.167311574880905 -88.53050114586871) bank154530 +154531 POINT(41.920757707797776 -86.72527302292687) bank154531 +154532 POINT(41.96289748362348 -87.41687761131848) bank154532 +154533 POINT(42.752472497300445 -88.08306360251927) bank154533 +154534 POINT(41.45829955159273 -88.4019072625114) bank154534 +154535 POINT(42.00556755485655 -88.34982111486288) bank154535 +154536 POINT(41.23893532552977 -87.3588282772904) bank154536 +154537 POINT(42.56223357579142 -88.1000457308796) bank154537 +154538 POINT(42.68180492079914 -86.86099495343124) bank154538 +154539 POINT(42.32035794278696 -88.16249733457421) bank154539 +154540 POINT(42.82399886466946 -88.0056077844573) bank154540 +154541 POINT(42.4584065920797 -88.0476725170582) bank154541 +154542 POINT(42.74029597468654 -87.927353702075) bank154542 +154543 POINT(41.73151071472375 -88.5339749260563) bank154543 +154544 POINT(42.07175685517797 -86.8171952433795) bank154544 +154545 POINT(40.935410864031354 -88.02281366783525) bank154545 +154546 POINT(42.29068771631634 -87.5529741265365) bank154546 +154547 POINT(40.90533629024602 -88.54068483176451) bank154547 +154548 POINT(41.38711743737623 -87.89016488647667) bank154548 +154549 POINT(42.0851765869793 -88.03400788490578) bank154549 +154550 POINT(41.68716766310533 -88.27593002606928) bank154550 +154551 POINT(41.46258916007038 -88.24025259119671) bank154551 +154552 POINT(41.53763079739403 -87.42203805972807) bank154552 +154553 POINT(42.2257443462663 -88.30446896546344) bank154553 +154554 POINT(41.341957361070044 -88.06320682718422) bank154554 +154555 POINT(42.17455387549921 -86.69478500187984) bank154555 +154556 POINT(41.53160057398227 -87.95855294121823) bank154556 +154557 POINT(41.820556131858204 -86.80497400562452) bank154557 +154558 POINT(41.6111893396042 -87.66223383211018) bank154558 +154559 POINT(41.417943673746414 -87.11133181366678) bank154559 +154560 POINT(42.77696858291604 -86.7353554187294) bank154560 +154561 POINT(42.74612987279515 -87.07038026891394) bank154561 +154562 POINT(42.032435925371246 -86.82554848608054) bank154562 +154563 POINT(40.89963559171971 -88.58641980852258) bank154563 +154564 POINT(41.07778316673032 -88.56152600999671) bank154564 +154565 POINT(41.813628063402355 -88.20173549549813) bank154565 +154566 POINT(41.72818786802665 -86.82599049647315) bank154566 +154567 POINT(40.890349423081986 -87.33979286358696) bank154567 +154568 POINT(41.68157155671777 -87.98580414504518) bank154568 +154569 POINT(41.14656197452 -88.47115223783256) bank154569 +154570 POINT(42.31403377425202 -86.90481186642346) bank154570 +154571 POINT(41.96898269389248 -86.96463144737437) bank154571 +154572 POINT(40.95754223443248 -86.80083412484) bank154572 +154573 POINT(42.87794892909456 -87.8059082645426) bank154573 +154574 POINT(42.54899071281501 -87.79861427925148) bank154574 +154575 POINT(42.21709440986714 -87.66164336383991) bank154575 +154576 POINT(42.248664956099574 -87.99564276412774) bank154576 +154577 POINT(41.52176751571741 -87.78497695283001) bank154577 +154578 POINT(41.211617494155796 -88.004462071803) bank154578 +154579 POINT(41.260079194842845 -87.00053014121741) bank154579 +154580 POINT(42.87189928655384 -88.05487059153322) bank154580 +154581 POINT(42.03881440339756 -86.86187452523245) bank154581 +154582 POINT(40.93769490078332 -88.52744347960065) bank154582 +154583 POINT(42.66179348531216 -87.28356928416632) bank154583 +154584 POINT(41.05861732823119 -87.71521617993155) bank154584 +154585 POINT(42.07972511622128 -88.51269080688921) bank154585 +154586 POINT(41.32283975592964 -87.94326204487311) bank154586 +154587 POINT(41.984728141483444 -88.56206247762428) bank154587 +154588 POINT(41.57628677195098 -88.53312660683062) bank154588 +154589 POINT(42.3222167962208 -87.54122272348447) bank154589 +154590 POINT(42.795597142120876 -86.77942591035685) bank154590 +154591 POINT(41.56797943078338 -87.60630097405317) bank154591 +154592 POINT(42.054981444285836 -87.66908906268252) bank154592 +154593 POINT(41.114365606904805 -88.42828756297745) bank154593 +154594 POINT(41.25722258951796 -87.55683067709435) bank154594 +154595 POINT(41.685306134144895 -87.79397974504217) bank154595 +154596 POINT(40.98030898583549 -86.98079848440773) bank154596 +154597 POINT(41.3714433635774 -86.91069314798149) bank154597 +154598 POINT(42.145358370947484 -86.65526824262554) bank154598 +154599 POINT(41.16066847963894 -87.01966394268614) bank154599 +154600 POINT(42.58198953226911 -88.07465904781927) bank154600 +154601 POINT(41.675534618081244 -87.37104910911894) bank154601 +154602 POINT(42.05853577556516 -88.243230546245) bank154602 +154603 POINT(41.593690611154244 -87.71737213230153) bank154603 +154604 POINT(42.01029889783627 -86.91349296377886) bank154604 +154605 POINT(41.56317889132707 -87.50786580371219) bank154605 +154606 POINT(41.55764440811072 -87.64508623304744) bank154606 +154607 POINT(42.602363865212546 -87.78376932364289) bank154607 +154608 POINT(41.040645481169626 -86.97707046616662) bank154608 +154609 POINT(40.971070402060214 -87.85981537403815) bank154609 +154610 POINT(41.425798742814116 -87.25168076730166) bank154610 +154611 POINT(41.17098070196392 -87.92644233550224) bank154611 +154612 POINT(41.2634087382684 -87.07041844065668) bank154612 +154613 POINT(42.66559667675138 -88.51030461348918) bank154613 +154614 POINT(41.24075247151395 -86.64293388143045) bank154614 +154615 POINT(41.626790349109775 -87.74436422796812) bank154615 +154616 POINT(42.39072059174671 -86.97296929012018) bank154616 +154617 POINT(41.258470660566225 -87.92171831565005) bank154617 +154618 POINT(42.20962353372808 -87.27430587156353) bank154618 +154619 POINT(41.460811165937386 -87.32426574002788) bank154619 +154620 POINT(42.68633510737192 -88.30382717621369) bank154620 +154621 POINT(41.32404481432545 -87.10590735850403) bank154621 +154622 POINT(41.747967687846455 -86.89466251853494) bank154622 +154623 POINT(42.470065179480144 -87.07736401250895) bank154623 +154624 POINT(41.18289581144978 -88.36462041359675) bank154624 +154625 POINT(41.01655000868525 -87.40154884218757) bank154625 +154626 POINT(41.37390386676766 -88.54424182718725) bank154626 +154627 POINT(42.40411729854887 -88.39365550617529) bank154627 +154628 POINT(42.228114136027784 -87.34862788336554) bank154628 +154629 POINT(42.358523117965014 -88.25370233804331) bank154629 +154630 POINT(41.62917907260485 -87.42726495774703) bank154630 +154631 POINT(41.319856965359406 -87.25478035402146) bank154631 +154632 POINT(42.05859405369237 -88.13604973610663) bank154632 +154633 POINT(41.04094955643699 -87.67253907482099) bank154633 +154634 POINT(41.991402453922404 -87.28495840058955) bank154634 +154635 POINT(42.30933940765222 -88.55836493500487) bank154635 +154636 POINT(40.953146952260035 -87.69058601106428) bank154636 +154637 POINT(42.693049268507 -88.47590034355567) bank154637 +154638 POINT(41.26681675556755 -87.8197991365409) bank154638 +154639 POINT(42.05892120875172 -87.87336884581285) bank154639 +154640 POINT(41.385824920494514 -87.6137691033676) bank154640 +154641 POINT(42.10670722630468 -87.10772891126797) bank154641 +154642 POINT(41.132636592984 -87.29548615077019) bank154642 +154643 POINT(42.330625476525675 -86.73991786020405) bank154643 +154644 POINT(40.9725796367744 -87.8762517599898) bank154644 +154645 POINT(42.53904937698582 -87.25031645562926) bank154645 +154646 POINT(42.02878423572845 -86.87617813402385) bank154646 +154647 POINT(42.697239809929556 -87.45927498822743) bank154647 +154648 POINT(41.14134315723593 -86.66717585377822) bank154648 +154649 POINT(41.45144912139999 -88.05270741492326) bank154649 +154650 POINT(41.26493029500639 -88.5894670681913) bank154650 +154651 POINT(42.343693248724925 -86.85434917503936) bank154651 +154652 POINT(41.652744297314506 -87.52455248985123) bank154652 +154653 POINT(42.37271189237098 -87.54732089256613) bank154653 +154654 POINT(41.12991289047288 -87.3418333626964) bank154654 +154655 POINT(41.141186824025 -88.43893601478617) bank154655 +154656 POINT(42.33022263706013 -88.1682391969056) bank154656 +154657 POINT(41.75502469107762 -87.0678622168462) bank154657 +154658 POINT(42.23761461422969 -87.64403839329967) bank154658 +154659 POINT(41.93141238014167 -88.44772020930456) bank154659 +154660 POINT(42.40184336970229 -87.58870731598986) bank154660 +154661 POINT(41.522115276430185 -88.59609965389622) bank154661 +154662 POINT(42.2620097247688 -87.14915793808245) bank154662 +154663 POINT(41.25675759423755 -88.02793960758162) bank154663 +154664 POINT(41.27796814002556 -87.13658192079546) bank154664 +154665 POINT(42.56248118589632 -86.72671223151656) bank154665 +154666 POINT(41.01863566072916 -88.42342277798198) bank154666 +154667 POINT(42.45313058249294 -87.20141201546714) bank154667 +154668 POINT(42.14611124173207 -88.03070977994713) bank154668 +154669 POINT(41.29723857387592 -88.58229302391499) bank154669 +154670 POINT(42.11649110305086 -87.61374760811174) bank154670 +154671 POINT(42.1480795581791 -87.68190930735133) bank154671 +154672 POINT(42.07305439984414 -88.33970822634936) bank154672 +154673 POINT(42.084710220343055 -86.93032602869192) bank154673 +154674 POINT(42.40828756315339 -87.52485303115598) bank154674 +154675 POINT(41.346271668663015 -87.77770982930899) bank154675 +154676 POINT(42.34545310969703 -88.49085573915161) bank154676 +154677 POINT(40.97611847565616 -87.3105086878435) bank154677 +154678 POINT(41.493442905305855 -86.63562830117674) bank154678 +154679 POINT(41.8769588151746 -87.10897187434318) bank154679 +154680 POINT(41.31169023627785 -88.57600967604922) bank154680 +154681 POINT(41.377564561925375 -87.0952777751109) bank154681 +154682 POINT(42.742828425655546 -87.25408051026369) bank154682 +154683 POINT(42.556729071588585 -87.05286648144973) bank154683 +154684 POINT(42.28185291666489 -86.88100139568843) bank154684 +154685 POINT(42.28038699935559 -86.93105842672776) bank154685 +154686 POINT(41.41820181699529 -87.10403140518093) bank154686 +154687 POINT(42.64261491410887 -87.40157249795226) bank154687 +154688 POINT(42.63490676564161 -88.01945445948508) bank154688 +154689 POINT(42.7940314324355 -87.65108949043807) bank154689 +154690 POINT(41.913552358532485 -87.59097347705479) bank154690 +154691 POINT(42.380897481044705 -87.34124780927) bank154691 +154692 POINT(41.51654929592872 -88.20067434941322) bank154692 +154693 POINT(42.54628864530878 -87.43351495084173) bank154693 +154694 POINT(41.59559209844844 -86.8275603969225) bank154694 +154695 POINT(42.45384894777816 -88.24611170992854) bank154695 +154696 POINT(40.88803258252046 -87.23966403158751) bank154696 +154697 POINT(41.066950597807846 -87.99633706218255) bank154697 +154698 POINT(41.612676626936114 -87.35542423758787) bank154698 +154699 POINT(42.155220912562996 -87.86717762775638) bank154699 +154700 POINT(41.53255798545646 -87.84880993910306) bank154700 +154701 POINT(41.23296989220045 -86.7302706780424) bank154701 +154702 POINT(41.59482515972036 -87.60018388153816) bank154702 +154703 POINT(41.21081273407387 -87.70346368281388) bank154703 +154704 POINT(42.57182745238822 -87.5927248732328) bank154704 +154705 POINT(40.91480535406936 -87.70424301872222) bank154705 +154706 POINT(42.54156840682887 -86.92919884880676) bank154706 +154707 POINT(41.928715897534516 -87.49160635452513) bank154707 +154708 POINT(41.65815814598112 -87.8448999314381) bank154708 +154709 POINT(42.42115540223302 -87.86884874809569) bank154709 +154710 POINT(42.09541729210531 -88.16711733376944) bank154710 +154711 POINT(41.96236553609595 -87.40623323995563) bank154711 +154712 POINT(42.384319564111024 -88.10511462495676) bank154712 +154713 POINT(42.36630624018115 -88.41659871064745) bank154713 +154714 POINT(42.84949045087821 -88.16022502738522) bank154714 +154715 POINT(42.41523110959001 -87.03139295645325) bank154715 +154716 POINT(42.09368027635293 -87.27688271935352) bank154716 +154717 POINT(42.22255933948284 -88.0716505247134) bank154717 +154718 POINT(41.43153849209024 -87.40094247081738) bank154718 +154719 POINT(41.57226148371506 -86.88303128319791) bank154719 +154720 POINT(41.58550946095474 -87.66945363044084) bank154720 +154721 POINT(42.835858031543225 -86.73221481474643) bank154721 +154722 POINT(42.41338096120514 -87.41264335967286) bank154722 +154723 POINT(41.08840297587555 -88.31282650240384) bank154723 +154724 POINT(42.770225560067274 -87.84627532089276) bank154724 +154725 POINT(41.23035126211972 -87.18095139416107) bank154725 +154726 POINT(41.205803034135855 -87.67257473528437) bank154726 +154727 POINT(41.711639493411646 -87.49508593656537) bank154727 +154728 POINT(41.99707743989429 -88.60271395166502) bank154728 +154729 POINT(41.23166072376865 -86.80222352428825) bank154729 +154730 POINT(41.13119775582516 -87.8232299956569) bank154730 +154731 POINT(41.320390612565426 -87.61394711988743) bank154731 +154732 POINT(41.556204862223694 -88.32592902520835) bank154732 +154733 POINT(42.55565177026804 -88.04014293594254) bank154733 +154734 POINT(41.39292914660047 -87.67296502405337) bank154734 +154735 POINT(42.61427203518558 -88.26989106209149) bank154735 +154736 POINT(41.077522025292424 -87.05175934053126) bank154736 +154737 POINT(41.4415315520655 -87.37236779483557) bank154737 +154738 POINT(41.56825961930873 -86.76802259433448) bank154738 +154739 POINT(42.259241082200155 -87.61883215000175) bank154739 +154740 POINT(41.32900226868651 -88.46961504303063) bank154740 +154741 POINT(42.08767434906991 -86.84970058035094) bank154741 +154742 POINT(41.17928490413966 -87.0201253295014) bank154742 +154743 POINT(41.7621980037821 -87.22727142735002) bank154743 +154744 POINT(41.298956544873604 -87.57447688017658) bank154744 +154745 POINT(42.173127851339785 -87.61498575393985) bank154745 +154746 POINT(42.08210120925286 -88.37239773849474) bank154746 +154747 POINT(42.18247352274509 -87.35074093740623) bank154747 +154748 POINT(42.824365661118485 -88.57592982513704) bank154748 +154749 POINT(41.52306895741762 -88.49960443168763) bank154749 +154750 POINT(42.41332437543074 -87.34910477438726) bank154750 +154751 POINT(42.429058259754704 -87.81423420599909) bank154751 +154752 POINT(41.20537575323772 -87.18990243649547) bank154752 +154753 POINT(41.47620496370502 -87.42813512591377) bank154753 +154754 POINT(41.279120616358796 -88.03484222939136) bank154754 +154755 POINT(42.069552147250015 -88.58494468070141) bank154755 +154756 POINT(42.677707715232515 -87.8456200450993) bank154756 +154757 POINT(41.018011841440995 -88.17981169725432) bank154757 +154758 POINT(41.264203894816696 -87.21211009443536) bank154758 +154759 POINT(41.38969512467312 -86.71345228100445) bank154759 +154760 POINT(40.89868098013964 -88.13166217263381) bank154760 +154761 POINT(41.724836431902766 -88.02324748586214) bank154761 +154762 POINT(41.71763564948478 -88.38173181758347) bank154762 +154763 POINT(41.969810669895246 -86.84079744962588) bank154763 +154764 POINT(42.04667938300816 -88.2398602448833) bank154764 +154765 POINT(42.536187338779655 -87.5377990916209) bank154765 +154766 POINT(42.629846271973435 -86.90086240170874) bank154766 +154767 POINT(41.48785378308232 -87.61300931754081) bank154767 +154768 POINT(41.42932005633357 -87.16154155360194) bank154768 +154769 POINT(41.238006883164104 -87.74381170485238) bank154769 +154770 POINT(41.262093817239375 -88.27250726257023) bank154770 +154771 POINT(42.62287057969044 -88.01522854765724) bank154771 +154772 POINT(41.36312863031655 -87.67570436555802) bank154772 +154773 POINT(41.74972318124064 -88.20643627353446) bank154773 +154774 POINT(42.07700375790203 -87.4045880395463) bank154774 +154775 POINT(42.83472734420576 -87.58291725745714) bank154775 +154776 POINT(42.32612887939147 -88.17720263918099) bank154776 +154777 POINT(42.74513588201679 -86.63098700316529) bank154777 +154778 POINT(42.73126326855671 -86.73694956574555) bank154778 +154779 POINT(41.458014872608146 -87.64053177007372) bank154779 +154780 POINT(42.3263567824206 -88.37722975306038) bank154780 +154781 POINT(42.70357979377164 -86.86785078000356) bank154781 +154782 POINT(42.7795164692175 -88.33862645738569) bank154782 +154783 POINT(42.52612957427749 -87.37729841150741) bank154783 +154784 POINT(41.581406657254846 -88.6242981039223) bank154784 +154785 POINT(41.01501282686612 -86.75273101396621) bank154785 +154786 POINT(41.13056846853182 -87.02066485078042) bank154786 +154787 POINT(41.869036430604005 -88.10931360987894) bank154787 +154788 POINT(41.87702206898448 -87.82514567153393) bank154788 +154789 POINT(41.61540517030759 -88.10973136725185) bank154789 +154790 POINT(42.83876199618067 -87.19793101048849) bank154790 +154791 POINT(41.0567136069666 -87.56088271893309) bank154791 +154792 POINT(42.7998402284942 -88.23796275913003) bank154792 +154793 POINT(41.62087990724973 -87.05486384650234) bank154793 +154794 POINT(40.941118808224665 -87.89053265311898) bank154794 +154795 POINT(42.153494857546164 -88.6009436497219) bank154795 +154796 POINT(42.393321987917105 -87.67377734546938) bank154796 +154797 POINT(42.3541203029241 -88.4489982142445) bank154797 +154798 POINT(41.118880954663574 -87.09558531857307) bank154798 +154799 POINT(42.01511330329219 -87.41601674856359) bank154799 +154800 POINT(41.96951491507993 -87.48203611358622) bank154800 +154801 POINT(41.09082106275448 -87.12004936242369) bank154801 +154802 POINT(42.529627402160585 -87.33317365084885) bank154802 +154803 POINT(42.85285013657771 -87.36273337124459) bank154803 +154804 POINT(41.27548688332285 -88.48134951175638) bank154804 +154805 POINT(41.26467004601708 -87.00318468426013) bank154805 +154806 POINT(41.647486136385915 -88.31453483442884) bank154806 +154807 POINT(42.06467859302217 -88.21083692175847) bank154807 +154808 POINT(42.74623724120667 -88.2027945618228) bank154808 +154809 POINT(41.51873721097679 -86.97658221780641) bank154809 +154810 POINT(41.49925567658956 -88.5328781390682) bank154810 +154811 POINT(42.18960666998953 -88.60040600787327) bank154811 +154812 POINT(41.296925150082515 -87.32840674495512) bank154812 +154813 POINT(42.1520745696538 -87.08970678438394) bank154813 +154814 POINT(42.64716224189234 -87.07680593267567) bank154814 +154815 POINT(41.772459386591066 -87.66096667347958) bank154815 +154816 POINT(41.19158602532464 -87.23866737100023) bank154816 +154817 POINT(41.2398322710092 -88.36378201425953) bank154817 +154818 POINT(42.84288019280369 -86.675466345341) bank154818 +154819 POINT(42.8158352013453 -87.73203671716017) bank154819 +154820 POINT(42.838828835599 -87.76031257570075) bank154820 +154821 POINT(42.66491895032218 -88.07959689368111) bank154821 +154822 POINT(42.50795869067863 -88.11592981304165) bank154822 +154823 POINT(41.915232247082585 -88.04135870618082) bank154823 +154824 POINT(42.569451432723575 -87.9933417001168) bank154824 +154825 POINT(41.5404365902716 -87.0946265340717) bank154825 +154826 POINT(41.18859794415668 -87.76299849002551) bank154826 +154827 POINT(42.66975405819805 -88.39244250447997) bank154827 +154828 POINT(40.901866283328815 -88.36331920443466) bank154828 +154829 POINT(41.69519432786506 -87.75771574836922) bank154829 +154830 POINT(41.72051067586144 -87.51826872902264) bank154830 +154831 POINT(41.50717061500446 -87.23789304839377) bank154831 +154832 POINT(41.797951892857476 -87.06825426477009) bank154832 +154833 POINT(42.08437225972397 -88.55442668711581) bank154833 +154834 POINT(42.48852272722628 -87.93955561792431) bank154834 +154835 POINT(41.11501357351321 -86.69499252526158) bank154835 +154836 POINT(41.130054039499186 -87.09853260509796) bank154836 +154837 POINT(41.82535355288656 -87.72376003907591) bank154837 +154838 POINT(41.782824732470424 -88.03794688477785) bank154838 +154839 POINT(42.57640953606314 -86.63685080536807) bank154839 +154840 POINT(40.99656217346089 -88.44118065140258) bank154840 +154841 POINT(41.53774789989013 -88.21775934806102) bank154841 +154842 POINT(42.60783965703183 -87.50556437023282) bank154842 +154843 POINT(42.787514108892694 -88.56090152666324) bank154843 +154844 POINT(41.12131628636122 -86.92922258589182) bank154844 +154845 POINT(42.77601823764488 -87.17878439750035) bank154845 +154846 POINT(42.31508623563593 -88.10639070146074) bank154846 +154847 POINT(42.6668683460519 -87.07848771172775) bank154847 +154848 POINT(42.23180516532777 -86.97241756933927) bank154848 +154849 POINT(41.98022370175867 -88.21048638910017) bank154849 +154850 POINT(42.793900174884996 -88.05740590264489) bank154850 +154851 POINT(42.4163441421359 -86.81108772778018) bank154851 +154852 POINT(42.788923023093616 -87.79991665814188) bank154852 +154853 POINT(41.12829344303441 -87.30021565553776) bank154853 +154854 POINT(42.574072665242404 -87.83503806525952) bank154854 +154855 POINT(42.62977599735259 -86.9305926114209) bank154855 +154856 POINT(42.657839526102954 -87.28143128542939) bank154856 +154857 POINT(42.309992383180436 -88.23229835743787) bank154857 +154858 POINT(42.846293890974444 -86.71918915031198) bank154858 +154859 POINT(42.068329380082055 -86.80698049164663) bank154859 +154860 POINT(41.709294568274174 -87.81762716408366) bank154860 +154861 POINT(42.29781076545284 -86.94440282451693) bank154861 +154862 POINT(41.87654383229712 -88.18477765052336) bank154862 +154863 POINT(41.58637053077382 -88.02857339520271) bank154863 +154864 POINT(42.33172515122042 -87.68723393854613) bank154864 +154865 POINT(42.59266487039523 -88.31567415789634) bank154865 +154866 POINT(42.37641480281198 -86.83447886133968) bank154866 +154867 POINT(41.57188628613689 -87.01249353111942) bank154867 +154868 POINT(41.98854153713149 -87.31918986439874) bank154868 +154869 POINT(41.66751338632148 -87.66837816114258) bank154869 +154870 POINT(41.58072484174785 -87.29600097725975) bank154870 +154871 POINT(42.01581995852228 -86.87825381719534) bank154871 +154872 POINT(41.42713102705222 -88.46631544630169) bank154872 +154873 POINT(40.900020878283485 -87.4657281347378) bank154873 +154874 POINT(41.39995951285084 -87.26824841011971) bank154874 +154875 POINT(42.396416572332235 -87.39093878268405) bank154875 +154876 POINT(42.72811536978647 -88.32846022669204) bank154876 +154877 POINT(41.28063813326819 -87.97335035305206) bank154877 +154878 POINT(40.880656698336175 -88.38553388773111) bank154878 +154879 POINT(41.33606122964996 -87.92673357496096) bank154879 +154880 POINT(42.641592232465634 -88.29138759796818) bank154880 +154881 POINT(41.803016309590156 -87.35151386902429) bank154881 +154882 POINT(42.746413875926784 -87.0838931490119) bank154882 +154883 POINT(42.01116909241709 -86.90367019658419) bank154883 +154884 POINT(41.51381734619351 -87.51818528855422) bank154884 +154885 POINT(41.05392770560789 -87.59824804047282) bank154885 +154886 POINT(42.54985910095748 -86.74925372789308) bank154886 +154887 POINT(42.38726426275661 -87.27211595673651) bank154887 +154888 POINT(42.481186646292954 -87.3374049873574) bank154888 +154889 POINT(40.931670129687895 -87.87689777868621) bank154889 +154890 POINT(41.119385577325616 -88.2856118747909) bank154890 +154891 POINT(41.962051421420085 -86.75419614086613) bank154891 +154892 POINT(41.360502569015345 -86.7745958061252) bank154892 +154893 POINT(42.1189716672026 -88.5871700931114) bank154893 +154894 POINT(41.68741524937678 -87.89447355160576) bank154894 +154895 POINT(42.70061132430143 -87.33188624287861) bank154895 +154896 POINT(42.376446673953645 -88.33233879212186) bank154896 +154897 POINT(42.01632060069213 -88.34935209960052) bank154897 +154898 POINT(42.48656634932449 -86.88107098145579) bank154898 +154899 POINT(42.79734986185201 -87.72749632248056) bank154899 +154900 POINT(42.791438477588024 -87.68741115665239) bank154900 +154901 POINT(41.19786162718757 -88.01529020648485) bank154901 +154902 POINT(41.61312849914473 -86.95825215137867) bank154902 +154903 POINT(41.245502507477966 -86.82018614098864) bank154903 +154904 POINT(41.87712951952756 -86.85749674243534) bank154904 +154905 POINT(41.389162556968564 -87.59645699187601) bank154905 +154906 POINT(42.78464925938016 -86.93400917817893) bank154906 +154907 POINT(41.364511675564934 -86.77381501064963) bank154907 +154908 POINT(42.518302170894536 -87.12209130137313) bank154908 +154909 POINT(42.66671440042348 -88.2578208362573) bank154909 +154910 POINT(41.018333756665164 -88.12959959028225) bank154910 +154911 POINT(41.23911210616352 -87.77650699161688) bank154911 +154912 POINT(41.587679350806546 -87.52582688025448) bank154912 +154913 POINT(41.846925368590675 -88.48680589244557) bank154913 +154914 POINT(42.565529384609306 -88.18097896923526) bank154914 +154915 POINT(40.97257260768465 -87.7825111207342) bank154915 +154916 POINT(41.273472887978535 -87.70598591144365) bank154916 +154917 POINT(41.17020597236086 -88.35172892421424) bank154917 +154918 POINT(41.814455094142346 -87.05706204921161) bank154918 +154919 POINT(42.87486005264916 -86.99221756425285) bank154919 +154920 POINT(41.498909726859665 -87.91633091419693) bank154920 +154921 POINT(41.71517882845968 -88.29137967505281) bank154921 +154922 POINT(41.948892535137055 -88.26423911998866) bank154922 +154923 POINT(42.081660549603754 -88.16803102182023) bank154923 +154924 POINT(41.76373885381377 -88.01462826131933) bank154924 +154925 POINT(41.06718873477513 -88.34262919561472) bank154925 +154926 POINT(42.7632759656024 -88.54648196146944) bank154926 +154927 POINT(41.83156165395268 -87.61710978737185) bank154927 +154928 POINT(41.28572916813731 -88.0984543498105) bank154928 +154929 POINT(41.48883897237204 -88.46159331371776) bank154929 +154930 POINT(41.87837694853698 -87.62496814362036) bank154930 +154931 POINT(42.28585722862554 -88.08414199479775) bank154931 +154932 POINT(42.64382718737205 -87.34123824900732) bank154932 +154933 POINT(41.881150568231334 -86.77582539703923) bank154933 +154934 POINT(42.483882619976676 -86.79562997770904) bank154934 +154935 POINT(41.75308694554829 -87.57564465976151) bank154935 +154936 POINT(42.71995303467646 -86.99667331184556) bank154936 +154937 POINT(42.508002421639546 -86.74805945461148) bank154937 +154938 POINT(41.77216080270625 -88.3356469319574) bank154938 +154939 POINT(42.72083244136435 -87.79900847872165) bank154939 +154940 POINT(41.580263974340134 -88.50523628761482) bank154940 +154941 POINT(42.84162082892141 -87.44107171864037) bank154941 +154942 POINT(41.579735880005416 -87.48269208794841) bank154942 +154943 POINT(41.17227611060972 -87.05188903789512) bank154943 +154944 POINT(41.71177509127651 -87.53906778731354) bank154944 +154945 POINT(41.81039200067375 -87.70217143139055) bank154945 +154946 POINT(42.62454582031656 -87.35124710907125) bank154946 +154947 POINT(41.810377797182305 -87.72286640312275) bank154947 +154948 POINT(42.4054228684827 -88.02522540720976) bank154948 +154949 POINT(42.67343848845273 -88.21586631103098) bank154949 +154950 POINT(42.52729569428462 -87.63489387915604) bank154950 +154951 POINT(42.62280631658486 -87.69292226282576) bank154951 +154952 POINT(41.46929582080762 -88.18511880836952) bank154952 +154953 POINT(42.625755354137674 -87.89784412116694) bank154953 +154954 POINT(41.039660397704566 -88.04146504337731) bank154954 +154955 POINT(42.092925397720855 -86.74798853781964) bank154955 +154956 POINT(42.71392066403258 -88.18650258719053) bank154956 +154957 POINT(41.79007402666632 -87.39712105847255) bank154957 +154958 POINT(41.1342589736706 -88.00403871054961) bank154958 +154959 POINT(42.44901641765479 -88.33411705094233) bank154959 +154960 POINT(41.83974723945107 -87.83521970622607) bank154960 +154961 POINT(41.09402765439313 -88.40652843013652) bank154961 +154962 POINT(41.089627354898326 -87.15421069317699) bank154962 +154963 POINT(42.71723750890659 -88.16641592164277) bank154963 +154964 POINT(42.415192110469704 -87.54961015025451) bank154964 +154965 POINT(41.211359868528646 -88.18088650888302) bank154965 +154966 POINT(42.11077143493782 -87.70030687705403) bank154966 +154967 POINT(42.215290789527366 -87.21798102811934) bank154967 +154968 POINT(41.69216732299098 -86.99753857051775) bank154968 +154969 POINT(41.63039627878565 -88.26548677791921) bank154969 +154970 POINT(42.47774966910697 -88.06378516912216) bank154970 +154971 POINT(41.83472270947584 -88.13815778335204) bank154971 +154972 POINT(42.650564665199 -87.96865125351356) bank154972 +154973 POINT(42.12942591244593 -87.19284753749474) bank154973 +154974 POINT(41.89191746821703 -87.68176299152614) bank154974 +154975 POINT(42.81528555413755 -87.5845109002317) bank154975 +154976 POINT(42.038229554486506 -86.869131870975) bank154976 +154977 POINT(41.08825140232383 -86.73581881939337) bank154977 +154978 POINT(41.052874106551265 -88.22020034560146) bank154978 +154979 POINT(41.99151819290897 -88.15448305679584) bank154979 +154980 POINT(42.705054175595144 -87.37559443124859) bank154980 +154981 POINT(42.49348276151029 -88.28196277759983) bank154981 +154982 POINT(41.37954830518212 -88.02683151390565) bank154982 +154983 POINT(42.376808532145965 -87.34477711075938) bank154983 +154984 POINT(42.396486972651765 -88.28605168727836) bank154984 +154985 POINT(41.247721421624 -88.07378607579268) bank154985 +154986 POINT(41.9441733029727 -88.24714469875586) bank154986 +154987 POINT(41.6835256554794 -87.33422288079407) bank154987 +154988 POINT(42.51633321353321 -87.322257061827) bank154988 +154989 POINT(41.31608316460808 -87.78887166050337) bank154989 +154990 POINT(41.6260268568699 -86.96396552924448) bank154990 +154991 POINT(42.008155463297165 -86.68578204360396) bank154991 +154992 POINT(41.725914067479444 -87.28771047444012) bank154992 +154993 POINT(41.6598030905587 -87.54432565009651) bank154993 +154994 POINT(42.81415296376873 -88.04707616112859) bank154994 +154995 POINT(40.97028534430108 -87.98585273153103) bank154995 +154996 POINT(42.10263089194445 -88.3959556210956) bank154996 +154997 POINT(41.49297217168466 -87.251174110004) bank154997 +154998 POINT(41.30178392823808 -87.6064505829019) bank154998 +154999 POINT(41.29028949825823 -87.10706044612004) bank154999 +155000 POINT(41.856956242538594 -88.3026287039417) bank155000 +155001 POINT(42.356344197680606 -86.86581484256058) bank155001 +155002 POINT(41.54034988145392 -86.83589577679969) bank155002 +155003 POINT(41.242122131135126 -87.86746233201042) bank155003 +155004 POINT(41.17696454381627 -87.56641544041074) bank155004 +155005 POINT(41.0893888018022 -87.8002051787025) bank155005 +155006 POINT(41.0495578351183 -87.92109535127747) bank155006 +155007 POINT(42.772934166454796 -86.93221855687193) bank155007 +155008 POINT(42.14540050622231 -87.95538717377504) bank155008 +155009 POINT(41.53774484061194 -88.54477610528441) bank155009 +155010 POINT(42.42629923464707 -86.82592714825691) bank155010 +155011 POINT(41.437189900223544 -86.8304471300221) bank155011 +155012 POINT(42.41682001631793 -88.11171683924138) bank155012 +155013 POINT(42.84361362569043 -86.95270748119205) bank155013 +155014 POINT(41.555510155442924 -87.28634622547551) bank155014 +155015 POINT(42.84768896587264 -86.97924693488612) bank155015 +155016 POINT(41.29674781295378 -87.26546306836806) bank155016 +155017 POINT(41.547822292156226 -87.77236008794222) bank155017 +155018 POINT(41.62720488434467 -87.50260427845065) bank155018 +155019 POINT(41.94434199468363 -86.98952834726825) bank155019 +155020 POINT(41.069344275954826 -88.52220685588674) bank155020 +155021 POINT(42.286129506445214 -88.44010658350436) bank155021 +155022 POINT(41.17510913766677 -87.89324641496833) bank155022 +155023 POINT(42.07625123420623 -87.61248184566158) bank155023 +155024 POINT(41.708176477147255 -88.51115466310273) bank155024 +155025 POINT(41.51117917862204 -86.78500635424307) bank155025 +155026 POINT(42.464670223713675 -88.25400548912769) bank155026 +155027 POINT(42.69426913603209 -87.24726287785228) bank155027 +155028 POINT(41.20667137932758 -88.51526289511884) bank155028 +155029 POINT(40.91463667448271 -87.81000613928894) bank155029 +155030 POINT(41.228712051119295 -87.88018911485217) bank155030 +155031 POINT(41.67460881615499 -87.75026095528287) bank155031 +155032 POINT(41.32144103068076 -87.83220657291106) bank155032 +155033 POINT(41.07298428448964 -87.33480797016365) bank155033 +155034 POINT(41.62830928713038 -88.28094201029596) bank155034 +155035 POINT(42.317876311317505 -88.07684259703146) bank155035 +155036 POINT(41.990404445322405 -87.53722796952238) bank155036 +155037 POINT(41.383725118649274 -87.3915577199044) bank155037 +155038 POINT(42.28983128027475 -88.43543892229599) bank155038 +155039 POINT(41.81003335819258 -88.0332732090034) bank155039 +155040 POINT(41.645620712002426 -88.56463604207346) bank155040 +155041 POINT(42.709902822650804 -87.41101556359038) bank155041 +155042 POINT(42.466462309482154 -86.74756924384019) bank155042 +155043 POINT(42.87010183078862 -87.05668512313343) bank155043 +155044 POINT(41.22255685389092 -88.23932437375899) bank155044 +155045 POINT(42.857067410296175 -87.89577221506592) bank155045 +155046 POINT(42.710495045173275 -87.01198620298358) bank155046 +155047 POINT(42.793163347443105 -87.39402343467803) bank155047 +155048 POINT(41.99630476499847 -87.502441799053) bank155048 +155049 POINT(41.86096762856629 -86.83521990428554) bank155049 +155050 POINT(41.439252114724205 -87.97289367022637) bank155050 +155051 POINT(41.99238894609337 -88.28622429183342) bank155051 +155052 POINT(41.65176408199981 -88.45850123028126) bank155052 +155053 POINT(42.13762297825939 -86.69036986224377) bank155053 +155054 POINT(41.22377612548868 -87.64503246424317) bank155054 +155055 POINT(41.44626422532958 -87.18184958716738) bank155055 +155056 POINT(42.161036630438815 -87.77422962428828) bank155056 +155057 POINT(41.8623531706762 -87.95673592855042) bank155057 +155058 POINT(41.63837287717943 -87.81660180169598) bank155058 +155059 POINT(41.243229028764205 -88.12277709268142) bank155059 +155060 POINT(41.61916469599959 -87.15405225577112) bank155060 +155061 POINT(41.156196458030706 -87.6529194894006) bank155061 +155062 POINT(42.375914793448196 -87.12644869136798) bank155062 +155063 POINT(41.13527404143683 -88.17026184049199) bank155063 +155064 POINT(42.20433462065075 -88.1456459079636) bank155064 +155065 POINT(42.47959060659263 -86.65260193135865) bank155065 +155066 POINT(42.769381314779295 -86.95656522722442) bank155066 +155067 POINT(41.03250448579522 -88.56618148666621) bank155067 +155068 POINT(42.534400041613914 -86.99475776437174) bank155068 +155069 POINT(41.18314699705084 -88.09278310106589) bank155069 +155070 POINT(41.20470917546601 -88.45121295748405) bank155070 +155071 POINT(40.969582341242685 -87.84445009751809) bank155071 +155072 POINT(41.4290120162387 -87.08308637740204) bank155072 +155073 POINT(41.80355609321297 -87.68313102319158) bank155073 +155074 POINT(42.619426827829685 -87.2434025028005) bank155074 +155075 POINT(41.37342084625446 -87.64655326538296) bank155075 +155076 POINT(42.082742356977036 -87.62608915390187) bank155076 +155077 POINT(40.882136520943995 -88.02245406677928) bank155077 +155078 POINT(41.278312533775406 -88.51628661695653) bank155078 +155079 POINT(41.749048207219005 -87.56505834915414) bank155079 +155080 POINT(42.46493622113839 -87.11405060455438) bank155080 +155081 POINT(41.20509638258804 -88.52520204215402) bank155081 +155082 POINT(41.6155248019644 -86.76206447029224) bank155082 +155083 POINT(41.42103905585149 -87.48622111621668) bank155083 +155084 POINT(41.19399798330899 -88.33923273519835) bank155084 +155085 POINT(42.18279163553375 -87.93061127111429) bank155085 +155086 POINT(42.593777788835865 -86.97167984388936) bank155086 +155087 POINT(41.897028779483996 -88.28268088425264) bank155087 +155088 POINT(41.57383072280161 -87.25508395289549) bank155088 +155089 POINT(41.711403390267314 -87.07668605277942) bank155089 +155090 POINT(41.307864536136854 -86.84399862607023) bank155090 +155091 POINT(41.2122078717026 -86.99011324441477) bank155091 +155092 POINT(41.62971166041922 -88.17014809144591) bank155092 +155093 POINT(40.95762430487592 -88.49334279575504) bank155093 +155094 POINT(42.84725669268984 -88.35192068427979) bank155094 +155095 POINT(40.97356748560266 -86.69410886026328) bank155095 +155096 POINT(41.54783343597527 -88.50048605094337) bank155096 +155097 POINT(41.18968960296029 -88.11807338756817) bank155097 +155098 POINT(41.79321307048133 -88.42694911494036) bank155098 +155099 POINT(42.269962385851606 -87.91070505266379) bank155099 +155100 POINT(42.65784136964156 -88.17056599074488) bank155100 +155101 POINT(41.93535229076803 -87.42187372363546) bank155101 +155102 POINT(41.324066430147084 -88.02910189049578) bank155102 +155103 POINT(41.98584963352488 -88.44439814848) bank155103 +155104 POINT(41.542170820794375 -86.7009249210431) bank155104 +155105 POINT(42.694419041291475 -86.7071161066602) bank155105 +155106 POINT(41.06459688584004 -87.25117818160695) bank155106 +155107 POINT(40.94541587678864 -87.406908730778) bank155107 +155108 POINT(42.234702200239255 -87.50442595124044) bank155108 +155109 POINT(42.25420858280958 -88.15526725544095) bank155109 +155110 POINT(42.060508817147436 -87.68858138557142) bank155110 +155111 POINT(42.23474594212841 -87.27112768689521) bank155111 +155112 POINT(42.808357812534894 -87.3898934480593) bank155112 +155113 POINT(41.588736876115995 -87.96683947903078) bank155113 +155114 POINT(41.41920137184432 -87.97946153768241) bank155114 +155115 POINT(42.0076354429772 -86.70221661219534) bank155115 +155116 POINT(42.83638105690749 -87.58422999120684) bank155116 +155117 POINT(42.14436617934985 -87.54079499002032) bank155117 +155118 POINT(41.59205049488042 -87.65020208246483) bank155118 +155119 POINT(41.15886997725034 -88.41408538396938) bank155119 +155120 POINT(42.784904634300545 -88.24886190877201) bank155120 +155121 POINT(41.37747783464809 -86.80447771726215) bank155121 +155122 POINT(41.06556458330994 -87.24901644010141) bank155122 +155123 POINT(41.28024824653442 -88.23865410720077) bank155123 +155124 POINT(41.39747503681822 -87.37081087729767) bank155124 +155125 POINT(41.81531586849764 -88.20364910704288) bank155125 +155126 POINT(41.92943286724854 -87.3742858583035) bank155126 +155127 POINT(41.163110403074256 -87.23485339669315) bank155127 +155128 POINT(41.36168299395889 -87.10547004467668) bank155128 +155129 POINT(42.03909347743806 -87.95968100330306) bank155129 +155130 POINT(41.13397036191311 -87.53623683749038) bank155130 +155131 POINT(42.490101987899685 -88.49186223838788) bank155131 +155132 POINT(42.514594603371606 -87.29081459521227) bank155132 +155133 POINT(42.54661710301993 -87.31892912697309) bank155133 +155134 POINT(41.06967412388175 -87.54538254088267) bank155134 +155135 POINT(41.177656130869636 -87.91120638487934) bank155135 +155136 POINT(41.254701455882895 -88.6265505154476) bank155136 +155137 POINT(41.18435846718371 -87.78990317215892) bank155137 +155138 POINT(42.55291945034227 -88.62428950797623) bank155138 +155139 POINT(41.148156428417344 -87.80966520306771) bank155139 +155140 POINT(41.115115104759916 -87.79948016104815) bank155140 +155141 POINT(40.966572985193764 -88.26973919066405) bank155141 +155142 POINT(40.94364012615701 -88.03319128530991) bank155142 +155143 POINT(41.29146130352135 -86.99282632618082) bank155143 +155144 POINT(42.00748640290133 -86.70349996925015) bank155144 +155145 POINT(42.07362581051046 -87.58651012448482) bank155145 +155146 POINT(42.476039934195846 -88.40520896907742) bank155146 +155147 POINT(42.71147187125151 -87.68049675014844) bank155147 +155148 POINT(42.419007333074184 -87.40644471802315) bank155148 +155149 POINT(42.05463454466512 -87.5337098660115) bank155149 +155150 POINT(42.845515762298824 -87.19459864308311) bank155150 +155151 POINT(41.26966274884369 -87.72880597514703) bank155151 +155152 POINT(42.7280242497606 -87.36022552918678) bank155152 +155153 POINT(42.044105399444 -87.0483818258195) bank155153 +155154 POINT(42.14903773571543 -86.87028022414229) bank155154 +155155 POINT(40.965468073317545 -88.59670516646146) bank155155 +155156 POINT(41.27861759582586 -88.19184990694365) bank155156 +155157 POINT(42.248482782769 -88.16581921786349) bank155157 +155158 POINT(41.6442911225393 -86.71550850752179) bank155158 +155159 POINT(42.01699870584602 -87.38861487897029) bank155159 +155160 POINT(40.97940832737816 -88.43462079297379) bank155160 +155161 POINT(42.711609197181225 -88.1806403465846) bank155161 +155162 POINT(41.98022792894764 -87.74221557891289) bank155162 +155163 POINT(41.44133284712906 -87.1855685087522) bank155163 +155164 POINT(42.743001817965514 -87.00472255823225) bank155164 +155165 POINT(40.92361366600542 -88.1208263421187) bank155165 +155166 POINT(42.34128360714452 -87.59551366712299) bank155166 +155167 POINT(40.93807227623137 -88.17360380021776) bank155167 +155168 POINT(41.09286284240668 -87.31043672515182) bank155168 +155169 POINT(41.29213285239929 -86.79418481039386) bank155169 +155170 POINT(40.99302714952343 -87.02759988747741) bank155170 +155171 POINT(42.565840285247454 -87.88314423761022) bank155171 +155172 POINT(41.99057493590696 -86.72304860009301) bank155172 +155173 POINT(42.12468557952208 -87.94189742727957) bank155173 +155174 POINT(41.94901893054301 -88.12797532645902) bank155174 +155175 POINT(42.294061401574815 -88.57637575198102) bank155175 +155176 POINT(41.374329420851765 -86.98596990702893) bank155176 +155177 POINT(41.16024232684473 -88.30011614280052) bank155177 +155178 POINT(41.07086435927382 -88.40266564076896) bank155178 +155179 POINT(42.66951790927688 -87.53795279949122) bank155179 +155180 POINT(42.44667400035938 -87.21777804708452) bank155180 +155181 POINT(42.57980461245513 -86.96822302442612) bank155181 +155182 POINT(41.36954888737075 -86.99109615082773) bank155182 +155183 POINT(41.29459034581527 -88.60073528535163) bank155183 +155184 POINT(42.678654848179356 -87.43759165537125) bank155184 +155185 POINT(42.18019784117394 -87.74883769563358) bank155185 +155186 POINT(42.53288412616227 -86.72577207093636) bank155186 +155187 POINT(41.41590148846811 -88.00279622159943) bank155187 +155188 POINT(41.07327705631764 -88.3307127485789) bank155188 +155189 POINT(41.86437343485227 -88.50993795367081) bank155189 +155190 POINT(42.59299821480451 -87.37914563802993) bank155190 +155191 POINT(41.7354631753507 -88.52560666410456) bank155191 +155192 POINT(41.71786800432612 -88.48242097948734) bank155192 +155193 POINT(42.089953262827166 -86.96003805393808) bank155193 +155194 POINT(42.453275504883614 -88.47427237411848) bank155194 +155195 POINT(42.385436328064145 -86.94000819834478) bank155195 +155196 POINT(41.34947655262025 -87.3421099133882) bank155196 +155197 POINT(41.89680396140544 -86.99156549901133) bank155197 +155198 POINT(42.328952041478026 -87.90325014658316) bank155198 +155199 POINT(42.86290260257685 -87.1376890657459) bank155199 +155200 POINT(41.511269152338706 -86.70001868535162) bank155200 +155201 POINT(42.181203953062024 -87.09208480126762) bank155201 +155202 POINT(41.66087849664319 -87.88131698726127) bank155202 +155203 POINT(42.465471909390544 -88.39040950544833) bank155203 +155204 POINT(41.52469163353769 -87.93144493357413) bank155204 +155205 POINT(41.324199865923774 -87.68284977763615) bank155205 +155206 POINT(42.1023641194345 -86.7803061819673) bank155206 +155207 POINT(42.14501013205208 -87.24369412481917) bank155207 +155208 POINT(42.411429453224926 -86.8455996160551) bank155208 +155209 POINT(41.10509389595587 -87.77502345237197) bank155209 +155210 POINT(41.92206757773681 -88.60153860112392) bank155210 +155211 POINT(41.28132605540865 -86.88738087012419) bank155211 +155212 POINT(42.36744146037177 -87.82905443142951) bank155212 +155213 POINT(42.59137593312398 -87.238869498477) bank155213 +155214 POINT(42.103131987571125 -87.23224534467242) bank155214 +155215 POINT(41.378866992113174 -88.30256896243131) bank155215 +155216 POINT(41.90657759430928 -88.0889496504027) bank155216 +155217 POINT(41.01550876908165 -86.70699355454576) bank155217 +155218 POINT(41.99192437804583 -88.00130096819728) bank155218 +155219 POINT(42.062164429782264 -87.4873808029036) bank155219 +155220 POINT(40.92065667596417 -88.49050172641061) bank155220 +155221 POINT(41.55039720695769 -86.7278386463382) bank155221 +155222 POINT(42.84255243420336 -86.67238011154168) bank155222 +155223 POINT(42.77730621229703 -87.82027311037825) bank155223 +155224 POINT(42.458109565430625 -87.10201144555998) bank155224 +155225 POINT(41.12713605493112 -88.28386400941429) bank155225 +155226 POINT(41.52907191263202 -87.35799691443043) bank155226 +155227 POINT(41.65962746691406 -88.17918556589821) bank155227 +155228 POINT(42.13713513350078 -87.7940985485263) bank155228 +155229 POINT(42.63984889161064 -87.17870757745341) bank155229 +155230 POINT(42.371268038386376 -87.4486879897518) bank155230 +155231 POINT(41.190283885606014 -88.09689431295514) bank155231 +155232 POINT(41.604666150562494 -87.29886185494861) bank155232 +155233 POINT(40.93426042791908 -86.70792909897347) bank155233 +155234 POINT(40.93244194267858 -87.55832207017767) bank155234 +155235 POINT(41.833997295044 -86.63183691527739) bank155235 +155236 POINT(41.461269493667174 -88.10750560613315) bank155236 +155237 POINT(42.57338948477945 -87.0644689174357) bank155237 +155238 POINT(42.08772631463287 -87.13123215057897) bank155238 +155239 POINT(42.375935820789245 -87.76767882530774) bank155239 +155240 POINT(41.38675215865414 -87.93316451373778) bank155240 +155241 POINT(42.38423848356828 -87.31136451332685) bank155241 +155242 POINT(42.18634067262131 -87.8105236590637) bank155242 +155243 POINT(42.42240440156089 -87.91966090999394) bank155243 +155244 POINT(42.01645525387403 -87.73818145972706) bank155244 +155245 POINT(42.555347579573784 -86.78781463794562) bank155245 +155246 POINT(42.25198144329776 -87.90813007954533) bank155246 +155247 POINT(40.989194964517694 -87.31581189895824) bank155247 +155248 POINT(41.14452965600204 -87.72568139395611) bank155248 +155249 POINT(41.71709708683696 -86.85531850840859) bank155249 +155250 POINT(41.00231772918992 -87.59531599994669) bank155250 +155251 POINT(42.480617072307474 -87.47843278302749) bank155251 +155252 POINT(41.038039087723675 -87.4369153621954) bank155252 +155253 POINT(41.72959093117178 -87.51855151176618) bank155253 +155254 POINT(41.289764796163475 -87.98527196508506) bank155254 +155255 POINT(41.62205241513165 -86.87055136153093) bank155255 +155256 POINT(42.186977912547064 -86.8761846253792) bank155256 +155257 POINT(42.558504257829284 -87.50701878881009) bank155257 +155258 POINT(41.15200249432474 -87.76664094410918) bank155258 +155259 POINT(42.03973006127518 -87.03592433626164) bank155259 +155260 POINT(42.305282041909315 -87.33076276742715) bank155260 +155261 POINT(41.86716607878984 -87.79371040339532) bank155261 +155262 POINT(41.86465155968932 -88.52127439414313) bank155262 +155263 POINT(41.635133246298956 -86.9772179225666) bank155263 +155264 POINT(41.667180434774544 -87.56629705248538) bank155264 +155265 POINT(40.915010971430604 -88.13465504427208) bank155265 +155266 POINT(42.494024162257915 -86.76455194678127) bank155266 +155267 POINT(41.37567392591018 -86.80319153641481) bank155267 +155268 POINT(41.768920432564116 -88.28144816148756) bank155268 +155269 POINT(41.5450226423448 -87.90116882062136) bank155269 +155270 POINT(41.834469331059985 -88.01151624938277) bank155270 +155271 POINT(42.226305039732516 -88.25281316439595) bank155271 +155272 POINT(42.51940365050678 -87.14466114213221) bank155272 +155273 POINT(41.066516003836014 -88.34133479221975) bank155273 +155274 POINT(41.913974097613604 -86.82640904156233) bank155274 +155275 POINT(42.64265487199739 -87.99261764708766) bank155275 +155276 POINT(41.06335675055247 -88.09609736325326) bank155276 +155277 POINT(41.32440683532183 -87.25612407263574) bank155277 +155278 POINT(41.2606551331444 -86.94501523870937) bank155278 +155279 POINT(42.31168607945756 -88.12944506924065) bank155279 +155280 POINT(42.47927399759002 -87.12225880896938) bank155280 +155281 POINT(40.881480027483946 -87.915089921345) bank155281 +155282 POINT(41.15640866755485 -88.48996301315499) bank155282 +155283 POINT(42.576744513104984 -87.19751629701655) bank155283 +155284 POINT(41.516219283458746 -87.89673121313919) bank155284 +155285 POINT(41.66590292616119 -87.70220075690042) bank155285 +155286 POINT(42.053147495104604 -87.29865671641996) bank155286 +155287 POINT(41.930476079629585 -88.55568211043779) bank155287 +155288 POINT(41.28459148144645 -87.38272237930514) bank155288 +155289 POINT(42.69114235526721 -87.22775759563906) bank155289 +155290 POINT(41.34515140442779 -88.59996483922515) bank155290 +155291 POINT(41.77518142423973 -88.47545974747291) bank155291 +155292 POINT(40.92860060718766 -87.2538219395087) bank155292 +155293 POINT(42.48825349638776 -87.70536393717214) bank155293 +155294 POINT(41.94167233014573 -86.6451025517614) bank155294 +155295 POINT(42.181549802068204 -87.48257197816183) bank155295 +155296 POINT(42.082681062461184 -87.01706765531358) bank155296 +155297 POINT(42.60903469845453 -87.09013872500545) bank155297 +155298 POINT(41.501354155515536 -88.09210820244023) bank155298 +155299 POINT(41.5852640919367 -87.91240372640523) bank155299 +155300 POINT(41.19429821092081 -87.76849365045013) bank155300 +155301 POINT(42.187355963781165 -88.5078786020151) bank155301 +155302 POINT(40.88031713455063 -86.86668808661402) bank155302 +155303 POINT(41.69259207489294 -88.57962921184743) bank155303 +155304 POINT(41.93981560790279 -86.65686055737416) bank155304 +155305 POINT(40.89317688636615 -87.03994660083626) bank155305 +155306 POINT(42.77517197172207 -87.75724062023382) bank155306 +155307 POINT(41.89346535737631 -87.75693843948413) bank155307 +155308 POINT(42.83564305776558 -88.13364478357947) bank155308 +155309 POINT(42.29810319123848 -88.04243498126645) bank155309 +155310 POINT(41.57842883546856 -87.28484233001574) bank155310 +155311 POINT(42.08432850017886 -88.2103163108661) bank155311 +155312 POINT(42.40204934002317 -88.55582844387288) bank155312 +155313 POINT(41.87419576699399 -88.25359293621344) bank155313 +155314 POINT(42.52319399681443 -87.73939744425294) bank155314 +155315 POINT(42.6930589906931 -88.58557689012714) bank155315 +155316 POINT(40.962150467290726 -88.32450025244374) bank155316 +155317 POINT(41.2817031167761 -88.04890472865968) bank155317 +155318 POINT(41.74739359719063 -86.8187111338862) bank155318 +155319 POINT(41.02364426900827 -87.174650641712) bank155319 +155320 POINT(41.361017927006195 -87.19770695996158) bank155320 +155321 POINT(41.3270823515694 -87.22510648112485) bank155321 +155322 POINT(41.894419115509564 -87.83195871215057) bank155322 +155323 POINT(42.628501185576575 -88.39199256581742) bank155323 +155324 POINT(41.315553185297354 -88.27567211163962) bank155324 +155325 POINT(41.238470013824085 -87.67639943449221) bank155325 +155326 POINT(42.5373623134682 -87.70868647876729) bank155326 +155327 POINT(40.95904570296255 -88.55795736724957) bank155327 +155328 POINT(41.06961550913094 -86.9641790609651) bank155328 +155329 POINT(42.34059122145398 -87.9995542633614) bank155329 +155330 POINT(42.2830855866218 -88.4495584548312) bank155330 +155331 POINT(42.3393073062576 -86.67883669130238) bank155331 +155332 POINT(41.108791187144064 -86.76421482141343) bank155332 +155333 POINT(42.46926335695375 -88.1727765042072) bank155333 +155334 POINT(41.18742771934934 -86.78844838407278) bank155334 +155335 POINT(41.195783482650214 -87.39503434781919) bank155335 +155336 POINT(42.380773248611135 -86.67857993911217) bank155336 +155337 POINT(41.84641519552177 -87.1632046866871) bank155337 +155338 POINT(42.08260718852953 -87.33063269450489) bank155338 +155339 POINT(41.69989389368816 -86.94782906292633) bank155339 +155340 POINT(41.12835983800132 -88.1780541022625) bank155340 +155341 POINT(41.751137501467994 -87.39635892256827) bank155341 +155342 POINT(41.3475771533187 -86.80645217132536) bank155342 +155343 POINT(41.84720616380134 -87.42670079924721) bank155343 +155344 POINT(42.778247342304425 -86.98652304165813) bank155344 +155345 POINT(42.73545928956848 -88.5655536974001) bank155345 +155346 POINT(41.47684129941207 -88.12158360643207) bank155346 +155347 POINT(42.19745951617985 -87.12716818824008) bank155347 +155348 POINT(41.3993485517352 -87.71136990817301) bank155348 +155349 POINT(42.74906031960509 -87.46179026464752) bank155349 +155350 POINT(41.09201283968202 -87.18337767860041) bank155350 +155351 POINT(41.88417089792506 -87.02130161421292) bank155351 +155352 POINT(40.886896052453125 -87.55062436304989) bank155352 +155353 POINT(42.7555367318721 -87.1433584791494) bank155353 +155354 POINT(41.538423062969265 -87.78289226376968) bank155354 +155355 POINT(41.17272130151972 -86.86314716175602) bank155355 +155356 POINT(40.97808380082866 -86.96718884584945) bank155356 +155357 POINT(42.85970746547475 -88.1634654191059) bank155357 +155358 POINT(41.56092838617859 -88.06167711170457) bank155358 +155359 POINT(41.221360866667005 -88.14196251715344) bank155359 +155360 POINT(42.50814548635349 -87.83018014791855) bank155360 +155361 POINT(41.506008684397564 -87.95925427429928) bank155361 +155362 POINT(41.070698171017135 -86.78958532944415) bank155362 +155363 POINT(42.074486171725155 -88.4428228499257) bank155363 +155364 POINT(41.52594302376903 -86.96114866567338) bank155364 +155365 POINT(42.87556255144044 -86.99861757559347) bank155365 +155366 POINT(40.92648528387462 -86.74185663664571) bank155366 +155367 POINT(41.29305003793595 -86.73998932601451) bank155367 +155368 POINT(41.98034928129071 -87.54272889201224) bank155368 +155369 POINT(42.46592052099724 -87.96099070194562) bank155369 +155370 POINT(41.092535788030894 -86.82871151831228) bank155370 +155371 POINT(41.1450514279844 -88.35210646529973) bank155371 +155372 POINT(42.39521596145319 -88.56446749068634) bank155372 +155373 POINT(41.91753695759053 -87.8184137334134) bank155373 +155374 POINT(42.35557556890215 -86.7225954355759) bank155374 +155375 POINT(42.57432908949722 -87.81192511950468) bank155375 +155376 POINT(42.00353943493534 -87.41835974223495) bank155376 +155377 POINT(41.278012713785095 -88.13943808401466) bank155377 +155378 POINT(42.82286942537798 -87.22487033100005) bank155378 +155379 POINT(42.41965429310963 -87.15358976227738) bank155379 +155380 POINT(41.05356900737087 -88.09968115018224) bank155380 +155381 POINT(41.747281467849994 -88.41849185169283) bank155381 +155382 POINT(42.30884996098571 -88.30344279874373) bank155382 +155383 POINT(42.84453044327777 -87.24941388788325) bank155383 +155384 POINT(42.13939013205777 -88.1587735185297) bank155384 +155385 POINT(42.39647128236452 -87.46807763743033) bank155385 +155386 POINT(42.72896426011123 -87.5610285554374) bank155386 +155387 POINT(42.119717603605764 -87.04557183337714) bank155387 +155388 POINT(41.49570058828421 -88.4079750702983) bank155388 +155389 POINT(42.81912476516838 -88.49968197695937) bank155389 +155390 POINT(40.9924476290412 -86.66607074873616) bank155390 +155391 POINT(42.838558095172516 -87.3177286917923) bank155391 +155392 POINT(42.305133977541274 -88.32485286731622) bank155392 +155393 POINT(41.26394759893545 -88.32079416034564) bank155393 +155394 POINT(42.114083803109175 -86.8621256698207) bank155394 +155395 POINT(41.89782300201403 -86.661266395161) bank155395 +155396 POINT(42.683500261951345 -87.59456799616477) bank155396 +155397 POINT(41.2115181700045 -88.42879799053037) bank155397 +155398 POINT(41.37516164330278 -87.61728456493303) bank155398 +155399 POINT(41.63218347492738 -86.71273541980358) bank155399 +155400 POINT(42.20391589373433 -87.37155586825989) bank155400 +155401 POINT(41.08121446540601 -86.74645692443306) bank155401 +155402 POINT(42.31950470228726 -87.96602273548876) bank155402 +155403 POINT(41.7094847533395 -87.23677769765389) bank155403 +155404 POINT(42.22854697660557 -87.05330550950023) bank155404 +155405 POINT(41.34348351582593 -86.99366168881595) bank155405 +155406 POINT(42.61870629568038 -87.96145357251233) bank155406 +155407 POINT(42.37857122632834 -87.36637059948322) bank155407 +155408 POINT(40.971220744003034 -87.41710885357938) bank155408 +155409 POINT(41.51842616200914 -87.0877028675849) bank155409 +155410 POINT(42.168674946877054 -86.99467680215157) bank155410 +155411 POINT(41.64168677814658 -88.17318076909305) bank155411 +155412 POINT(42.585944114843734 -87.34892740393404) bank155412 +155413 POINT(41.85634056651238 -87.36250406817572) bank155413 +155414 POINT(41.985316673460325 -87.19641129345848) bank155414 +155415 POINT(42.2403031513908 -87.82313657962324) bank155415 +155416 POINT(42.71998511089373 -87.32570284839353) bank155416 +155417 POINT(41.41515627748605 -87.05113000223719) bank155417 +155418 POINT(41.7186228739312 -87.23226232562222) bank155418 +155419 POINT(40.95260191877448 -87.81879033270286) bank155419 +155420 POINT(42.704978791520176 -86.69141505513028) bank155420 +155421 POINT(42.45053618519295 -87.4575728039973) bank155421 +155422 POINT(42.31717512329841 -88.62926904501929) bank155422 +155423 POINT(42.19212818168935 -88.47327482871266) bank155423 +155424 POINT(42.4131126317424 -87.16163966462025) bank155424 +155425 POINT(42.656411701066084 -88.00845996073306) bank155425 +155426 POINT(41.025780903625495 -87.68998383006918) bank155426 +155427 POINT(41.77153402824756 -88.12767146180656) bank155427 +155428 POINT(40.951453040183246 -87.93830309074862) bank155428 +155429 POINT(41.05812111440027 -86.6412544354866) bank155429 +155430 POINT(42.45476553138222 -88.59971449024977) bank155430 +155431 POINT(41.52090346067743 -86.97407653293021) bank155431 +155432 POINT(41.60635997932576 -87.11950391038887) bank155432 +155433 POINT(42.174137249734756 -88.44170002541257) bank155433 +155434 POINT(41.60473158881639 -87.879098218992) bank155434 +155435 POINT(42.563309003275045 -86.85153057022704) bank155435 +155436 POINT(41.89122150807687 -86.68227149054619) bank155436 +155437 POINT(42.781050253878526 -87.97577250194666) bank155437 +155438 POINT(41.17371313314928 -87.96177587155483) bank155438 +155439 POINT(42.096812832713454 -88.19550937614042) bank155439 +155440 POINT(41.96585062482974 -86.84914395891487) bank155440 +155441 POINT(41.02589494743808 -87.70675587175646) bank155441 +155442 POINT(42.573556781300795 -87.73787669474106) bank155442 +155443 POINT(42.70491148679178 -88.34371156368458) bank155443 +155444 POINT(42.28537720273913 -87.13159460739087) bank155444 +155445 POINT(41.90150186317229 -86.8731393687143) bank155445 +155446 POINT(41.21729111673227 -88.0537069479657) bank155446 +155447 POINT(42.51417681716839 -88.2514174058509) bank155447 +155448 POINT(42.46756982593735 -87.34619509913517) bank155448 +155449 POINT(41.06943809766398 -88.10847913394392) bank155449 +155450 POINT(41.51054886706192 -87.99624682179663) bank155450 +155451 POINT(42.28614248382187 -86.70124320626286) bank155451 +155452 POINT(41.259875448258214 -87.19170852710859) bank155452 +155453 POINT(42.546908127132085 -88.62835125589521) bank155453 +155454 POINT(42.525632433383564 -86.73753214875452) bank155454 +155455 POINT(42.40220851904375 -88.58733569847) bank155455 +155456 POINT(40.943130540354886 -87.12621727885082) bank155456 +155457 POINT(42.216916337047095 -88.44214048546868) bank155457 +155458 POINT(42.02749784791581 -87.82734038597387) bank155458 +155459 POINT(41.41777412621238 -88.4569690702298) bank155459 +155460 POINT(41.69319012686363 -88.10362764836096) bank155460 +155461 POINT(41.24362418310652 -86.73697219882479) bank155461 +155462 POINT(41.086165613182146 -88.41058709065402) bank155462 +155463 POINT(41.82875570141117 -88.31785243250823) bank155463 +155464 POINT(42.83510964409705 -87.40658336326803) bank155464 +155465 POINT(41.623585783823096 -87.21109593861026) bank155465 +155466 POINT(41.331085880220975 -88.32208180594785) bank155466 +155467 POINT(42.53694246031939 -87.27815029028226) bank155467 +155468 POINT(42.764853733687445 -88.52562762159756) bank155468 +155469 POINT(41.54382434519939 -88.1506439978794) bank155469 +155470 POINT(42.34351044178086 -87.97865722124533) bank155470 +155471 POINT(42.25522749370197 -87.72514999598226) bank155471 +155472 POINT(42.175212361193374 -87.05767147405209) bank155472 +155473 POINT(41.17486476890039 -87.36577812258297) bank155473 +155474 POINT(42.20984510882456 -87.93127410952457) bank155474 +155475 POINT(42.279525756372706 -88.4829089887261) bank155475 +155476 POINT(41.05052972309409 -87.23050807286774) bank155476 +155477 POINT(41.90770657427136 -86.79380036512688) bank155477 +155478 POINT(42.84517438079092 -87.21704037702465) bank155478 +155479 POINT(41.082393733409795 -87.20238524499887) bank155479 +155480 POINT(41.351846068917204 -88.44035242902642) bank155480 +155481 POINT(41.902180673274344 -86.68060743660138) bank155481 +155482 POINT(42.272143821617775 -86.80109345533054) bank155482 +155483 POINT(42.34228384180007 -88.6019749323312) bank155483 +155484 POINT(42.23357052817828 -88.03324375252576) bank155484 +155485 POINT(41.66345225415131 -87.85462130838606) bank155485 +155486 POINT(41.71080717696308 -88.01037891685047) bank155486 +155487 POINT(41.61545022510241 -86.89111665915279) bank155487 +155488 POINT(42.132953154913814 -86.73171178055732) bank155488 +155489 POINT(41.16787311866004 -88.59419575126573) bank155489 +155490 POINT(42.74910422746745 -87.67516422878113) bank155490 +155491 POINT(41.10570390311752 -87.9885102102565) bank155491 +155492 POINT(42.75626408937653 -87.14537940186904) bank155492 +155493 POINT(41.92437379094003 -88.1998176641693) bank155493 +155494 POINT(42.040121085702594 -88.26956595959682) bank155494 +155495 POINT(41.61282260420426 -88.37104558282051) bank155495 +155496 POINT(42.604028112344515 -88.12832392604439) bank155496 +155497 POINT(40.88919644335431 -87.6639110635556) bank155497 +155498 POINT(42.3581085247991 -86.7594599620545) bank155498 +155499 POINT(41.7880004839903 -87.31861315542979) bank155499 +155500 POINT(42.55020726632021 -87.47117477888905) bank155500 +155501 POINT(41.98587383177084 -88.40191822587569) bank155501 +155502 POINT(41.8939741850355 -88.43605071322706) bank155502 +155503 POINT(42.610164714616204 -88.5384516567953) bank155503 +155504 POINT(41.66262524703857 -87.46169211070386) bank155504 +155505 POINT(41.33854886295346 -87.51881511784244) bank155505 +155506 POINT(42.19824088384 -88.3317759542519) bank155506 +155507 POINT(42.453986884247826 -87.08520002220641) bank155507 +155508 POINT(42.49756720843168 -87.73305566503322) bank155508 +155509 POINT(42.155536854608265 -88.4201418180349) bank155509 +155510 POINT(41.96461129942862 -86.88682427400138) bank155510 +155511 POINT(42.05560706245175 -87.57267221149202) bank155511 +155512 POINT(41.47833148633046 -87.45127490256336) bank155512 +155513 POINT(42.12856212344471 -86.92034023313185) bank155513 +155514 POINT(41.392156918532066 -87.51591437332259) bank155514 +155515 POINT(42.499541009161874 -87.04782895261268) bank155515 +155516 POINT(41.19966748417389 -88.62856508081461) bank155516 +155517 POINT(41.91670425323213 -86.87183923280966) bank155517 +155518 POINT(42.10056892282062 -86.89209091784404) bank155518 +155519 POINT(42.80483753477859 -88.21517995905292) bank155519 +155520 POINT(41.361643969779216 -88.59633661367792) bank155520 +155521 POINT(41.77365960049022 -87.53005569621835) bank155521 +155522 POINT(42.571241942797876 -87.10516745541283) bank155522 +155523 POINT(41.2955550513669 -87.03710447191466) bank155523 +155524 POINT(42.110746035027134 -88.15225129180814) bank155524 +155525 POINT(41.56453816073802 -88.54136811072335) bank155525 +155526 POINT(42.424338097023266 -87.72739764390492) bank155526 +155527 POINT(41.723597734856305 -88.1265594330182) bank155527 +155528 POINT(41.10954713659208 -86.95367246624764) bank155528 +155529 POINT(41.305667597019415 -87.19855479360707) bank155529 +155530 POINT(40.945466898599825 -87.20023802947472) bank155530 +155531 POINT(42.327239413501424 -88.5943532751622) bank155531 +155532 POINT(41.76136465365408 -87.05439054103351) bank155532 +155533 POINT(42.34719596354317 -86.6666196831291) bank155533 +155534 POINT(41.458366433924496 -88.13512714190233) bank155534 +155535 POINT(41.031413188520204 -87.14851837594372) bank155535 +155536 POINT(42.23559487397328 -87.98011333422382) bank155536 +155537 POINT(42.35012522820484 -87.27388861204561) bank155537 +155538 POINT(41.892874328343744 -88.53522566750154) bank155538 +155539 POINT(42.1053590599301 -88.37604217780988) bank155539 +155540 POINT(41.983372855651766 -87.6465398831128) bank155540 +155541 POINT(41.625307309481734 -87.24855895452713) bank155541 +155542 POINT(41.733043995467014 -87.03264131899408) bank155542 +155543 POINT(42.71225888817295 -87.61379670569288) bank155543 +155544 POINT(42.501753974376385 -87.49064721442328) bank155544 +155545 POINT(42.22611497415311 -87.59332963668619) bank155545 +155546 POINT(42.1050485067614 -87.26939654428372) bank155546 +155547 POINT(42.080669837506264 -87.52902525263772) bank155547 +155548 POINT(42.37884731942266 -86.80447636099811) bank155548 +155549 POINT(41.14815324835458 -88.32268342390269) bank155549 +155550 POINT(42.246150744962485 -88.03618282474875) bank155550 +155551 POINT(42.103637644717075 -86.84574556930703) bank155551 +155552 POINT(42.07881087801585 -87.90335348244177) bank155552 +155553 POINT(40.910435165941934 -88.03970852222929) bank155553 +155554 POINT(42.18137217803195 -88.52341200920102) bank155554 +155555 POINT(41.114212622869246 -88.15485071766521) bank155555 +155556 POINT(41.44648002948689 -86.7281383130579) bank155556 +155557 POINT(40.87857530358497 -86.71690026071569) bank155557 +155558 POINT(41.39752352381105 -87.41292615312226) bank155558 +155559 POINT(41.560071595577966 -88.05016070526038) bank155559 +155560 POINT(42.32381109406599 -88.31099693433619) bank155560 +155561 POINT(42.32066398180237 -87.01365797609158) bank155561 +155562 POINT(41.4095529003318 -88.33937757919537) bank155562 +155563 POINT(41.32096043884477 -88.61648795137603) bank155563 +155564 POINT(41.32464313594147 -87.20491070986749) bank155564 +155565 POINT(42.02243977073138 -87.56443933824474) bank155565 +155566 POINT(42.50606154802678 -87.35950951350856) bank155566 +155567 POINT(40.94185016515944 -87.26063542424394) bank155567 +155568 POINT(41.828200792425534 -87.63541251966596) bank155568 +155569 POINT(42.59764497973717 -86.64302654649971) bank155569 +155570 POINT(41.96790094965381 -86.8260651053501) bank155570 +155571 POINT(41.667108274523386 -87.0420829310236) bank155571 +155572 POINT(42.20974730319785 -87.91054934823701) bank155572 +155573 POINT(41.03339603777399 -87.81777028272842) bank155573 +155574 POINT(41.66382454692648 -87.16931739034372) bank155574 +155575 POINT(42.12578947120075 -86.65741268428683) bank155575 +155576 POINT(41.38490725706811 -87.43837129266514) bank155576 +155577 POINT(41.41298505632277 -88.37266646021392) bank155577 +155578 POINT(41.2566145444095 -86.90073196052226) bank155578 +155579 POINT(41.51996858168437 -88.1097070820267) bank155579 +155580 POINT(41.462363467321914 -86.8283597842467) bank155580 +155581 POINT(41.68946374405584 -87.61719986791837) bank155581 +155582 POINT(42.05300444440278 -88.21647445973913) bank155582 +155583 POINT(41.516379476788664 -88.45361630732037) bank155583 +155584 POINT(42.42234623569655 -86.9830901090216) bank155584 +155585 POINT(42.23757093391152 -88.20182561309784) bank155585 +155586 POINT(41.08130932760395 -86.86307204270072) bank155586 +155587 POINT(41.018772054724764 -87.13777818324228) bank155587 +155588 POINT(40.97407313883054 -87.15673018162836) bank155588 +155589 POINT(41.18463123446657 -87.83582156038712) bank155589 +155590 POINT(41.188473850783 -87.49709376780852) bank155590 +155591 POINT(42.86758608229421 -88.55259131956463) bank155591 +155592 POINT(40.91809311768225 -87.11932902326981) bank155592 +155593 POINT(41.096234553973815 -88.13410853363435) bank155593 +155594 POINT(42.10542946160776 -86.9727609186975) bank155594 +155595 POINT(41.54879425771425 -87.74832942569996) bank155595 +155596 POINT(42.622074020440635 -86.89490555870884) bank155596 +155597 POINT(42.84297700886601 -88.39208762469758) bank155597 +155598 POINT(42.68922563573657 -87.76005020459034) bank155598 +155599 POINT(40.905386901498666 -87.41282829639228) bank155599 +155600 POINT(41.285118676134175 -86.79141482688698) bank155600 +155601 POINT(41.04149178391912 -87.78649328167043) bank155601 +155602 POINT(42.33226943122597 -87.63180969585679) bank155602 +155603 POINT(42.31469380595835 -88.26950580468036) bank155603 +155604 POINT(41.89337796856568 -87.45047047458162) bank155604 +155605 POINT(42.29358320803606 -87.57988236041179) bank155605 +155606 POINT(41.125137954444725 -88.59511122777982) bank155606 +155607 POINT(42.334081498126125 -88.51807969481044) bank155607 +155608 POINT(42.112349012922174 -87.59386325621797) bank155608 +155609 POINT(42.086107881127184 -86.70715998642869) bank155609 +155610 POINT(41.43168177212944 -87.24567809279779) bank155610 +155611 POINT(40.98967484927184 -87.7530897170242) bank155611 +155612 POINT(42.84454675893018 -88.56280909701229) bank155612 +155613 POINT(42.32560108891187 -86.68365871644163) bank155613 +155614 POINT(42.344925469722014 -88.45456871990028) bank155614 +155615 POINT(41.30257981506788 -87.33092669373934) bank155615 +155616 POINT(41.11787693813341 -88.36502380660794) bank155616 +155617 POINT(42.734085334279776 -88.61531909051885) bank155617 +155618 POINT(42.31523549820107 -87.43343173714318) bank155618 +155619 POINT(42.61552415791723 -88.45314929344782) bank155619 +155620 POINT(41.407295924499756 -86.65013918362959) bank155620 +155621 POINT(41.35873519304005 -86.77326602694583) bank155621 +155622 POINT(41.54188696184434 -87.2439151711819) bank155622 +155623 POINT(41.8333660664209 -87.43752601335794) bank155623 +155624 POINT(42.17817760281227 -88.30674939590082) bank155624 +155625 POINT(42.15812535899089 -87.24267659544174) bank155625 +155626 POINT(42.33875101607078 -86.90090168238426) bank155626 +155627 POINT(42.374463343976956 -88.55657971309238) bank155627 +155628 POINT(41.38972414016352 -88.38880215330619) bank155628 +155629 POINT(41.104496821305126 -88.28034066117353) bank155629 +155630 POINT(41.65495661245611 -88.36640216031495) bank155630 +155631 POINT(41.06265904678313 -88.56598943189778) bank155631 +155632 POINT(42.18519758402001 -88.37164742113502) bank155632 +155633 POINT(40.97661032311487 -87.25953691538376) bank155633 +155634 POINT(41.22020122770541 -87.87414973696428) bank155634 +155635 POINT(42.30253617967303 -86.82824984433303) bank155635 +155636 POINT(42.56361467824617 -87.66156033800146) bank155636 +155637 POINT(42.11260365222743 -87.7632980588703) bank155637 +155638 POINT(42.72930299229915 -87.28962715572165) bank155638 +155639 POINT(41.31300529140978 -88.24587578765743) bank155639 +155640 POINT(42.12841455315927 -87.91935210678406) bank155640 +155641 POINT(41.253300993931425 -88.15330626901945) bank155641 +155642 POINT(41.50721067731126 -87.45961523442696) bank155642 +155643 POINT(41.096679029908366 -86.80677857948413) bank155643 +155644 POINT(42.166572875434994 -87.55004379967951) bank155644 +155645 POINT(42.213231438972755 -87.11852528477525) bank155645 +155646 POINT(41.340722655521155 -86.80790906121567) bank155646 +155647 POINT(42.61284415278668 -87.52809495720109) bank155647 +155648 POINT(42.557579455617194 -87.07012688012112) bank155648 +155649 POINT(41.184481579813166 -88.3015370326716) bank155649 +155650 POINT(41.612898633685276 -87.14877666456184) bank155650 +155651 POINT(42.29603022007486 -88.09578999244579) bank155651 +155652 POINT(42.40303467903716 -87.81683391353324) bank155652 +155653 POINT(41.45344759110685 -87.10976078259448) bank155653 +155654 POINT(42.49254979213141 -87.90658478335324) bank155654 +155655 POINT(42.499088068053936 -88.22133723884714) bank155655 +155656 POINT(42.20656163733506 -87.2980101855789) bank155656 +155657 POINT(40.948107988788315 -87.81270855448255) bank155657 +155658 POINT(41.892910212251394 -86.95763063350526) bank155658 +155659 POINT(41.786498354761434 -87.46484055790256) bank155659 +155660 POINT(41.28354274268525 -87.56672141627142) bank155660 +155661 POINT(42.74752758653356 -87.5466109456203) bank155661 +155662 POINT(42.37770767533183 -86.78789621841248) bank155662 +155663 POINT(42.8526085286188 -88.42671613820573) bank155663 +155664 POINT(41.76780570848677 -87.0444859498071) bank155664 +155665 POINT(42.16273792818719 -87.8490434490491) bank155665 +155666 POINT(41.29960835716427 -87.40178137074666) bank155666 +155667 POINT(41.96366471217219 -87.37903560284981) bank155667 +155668 POINT(42.18189925399057 -88.4255566675316) bank155668 +155669 POINT(41.6258036503871 -87.42076399248133) bank155669 +155670 POINT(42.027558304993036 -87.68637507237753) bank155670 +155671 POINT(42.648788327249406 -88.50402259379543) bank155671 +155672 POINT(42.25195513750089 -88.21653708175214) bank155672 +155673 POINT(41.123892291606914 -88.10854716470035) bank155673 +155674 POINT(41.02698028986814 -88.20421442628935) bank155674 +155675 POINT(41.33383283569156 -88.12935720171788) bank155675 +155676 POINT(42.007492355901725 -87.09259354864977) bank155676 +155677 POINT(42.595145655834365 -87.77493111832338) bank155677 +155678 POINT(42.554395717225546 -87.39741951088243) bank155678 +155679 POINT(41.37241948690767 -88.42512538494368) bank155679 +155680 POINT(40.929395930149056 -86.81860808024138) bank155680 +155681 POINT(42.567780410664184 -87.24194471847365) bank155681 +155682 POINT(41.87485834207989 -88.2178916996602) bank155682 +155683 POINT(41.2167304767118 -88.24557363965626) bank155683 +155684 POINT(42.02979169116996 -86.63431424765609) bank155684 +155685 POINT(41.25494748444327 -87.45731188928683) bank155685 +155686 POINT(42.18629031405204 -87.21263819979364) bank155686 +155687 POINT(42.27558012894353 -87.84567198321801) bank155687 +155688 POINT(42.60844972115139 -88.46907448127297) bank155688 +155689 POINT(42.798775109666586 -86.89554744022394) bank155689 +155690 POINT(42.30695050285873 -87.20543403721106) bank155690 +155691 POINT(41.29868035443988 -86.891061679587) bank155691 +155692 POINT(40.90943478134316 -88.00190953515205) bank155692 +155693 POINT(42.177465515812976 -87.10137370698328) bank155693 +155694 POINT(41.82524989375668 -86.98509014764834) bank155694 +155695 POINT(42.60753015765427 -86.66840487173174) bank155695 +155696 POINT(41.67059249558752 -87.177362565131) bank155696 +155697 POINT(40.98114030886892 -87.17059307357698) bank155697 +155698 POINT(42.260181297324664 -88.60757502445229) bank155698 +155699 POINT(41.99425134378165 -87.00571494356548) bank155699 +155700 POINT(42.532764522393016 -88.62589901990205) bank155700 +155701 POINT(42.709113952591096 -88.60682357834997) bank155701 +155702 POINT(41.02575581417043 -87.24789956932275) bank155702 +155703 POINT(41.07292152339457 -87.51488816865711) bank155703 +155704 POINT(42.485176760018916 -87.36875290815819) bank155704 +155705 POINT(42.29943128372145 -86.7484018102588) bank155705 +155706 POINT(41.22851455229201 -88.22094613639614) bank155706 +155707 POINT(42.25477961004153 -88.22174109186042) bank155707 +155708 POINT(40.89041769043019 -87.0787451943173) bank155708 +155709 POINT(41.039877219456685 -87.1667902234454) bank155709 +155710 POINT(41.177077117361456 -86.74959124079916) bank155710 +155711 POINT(41.4688000254779 -87.58287321680895) bank155711 +155712 POINT(41.79409251393911 -87.77715061364458) bank155712 +155713 POINT(42.222520682080166 -88.00819330022888) bank155713 +155714 POINT(42.20681316872829 -88.52100791658462) bank155714 +155715 POINT(41.199765424950996 -86.63586065417051) bank155715 +155716 POINT(40.91534116931827 -87.051423766704) bank155716 +155717 POINT(41.260238070625114 -87.02041103991742) bank155717 +155718 POINT(42.10905215030423 -87.7301072444616) bank155718 +155719 POINT(42.140762159335154 -88.43486388334316) bank155719 +155720 POINT(41.372725461255264 -88.37964873819645) bank155720 +155721 POINT(41.35213854159699 -86.93123248245482) bank155721 +155722 POINT(42.54475325063571 -86.99002616098583) bank155722 +155723 POINT(41.15283393246635 -87.23041553120092) bank155723 +155724 POINT(41.918705874045834 -88.59419727056337) bank155724 +155725 POINT(42.257091768064925 -88.31347312496878) bank155725 +155726 POINT(41.03921700924272 -87.47229537842462) bank155726 +155727 POINT(42.79897565212133 -88.37962768303134) bank155727 +155728 POINT(42.31024993168214 -87.81139929776295) bank155728 +155729 POINT(41.47538139582125 -86.77303731383398) bank155729 +155730 POINT(42.79174123174258 -87.78904019743024) bank155730 +155731 POINT(42.35255298718444 -87.31485060489535) bank155731 +155732 POINT(41.63724954731598 -87.59954144691427) bank155732 +155733 POINT(41.86963407445124 -87.9626307770636) bank155733 +155734 POINT(41.64621644495396 -86.63960033536664) bank155734 +155735 POINT(42.07158567712138 -87.71775528951369) bank155735 +155736 POINT(42.42785264771452 -88.01782969459329) bank155736 +155737 POINT(42.655739458932295 -87.93875065580544) bank155737 +155738 POINT(42.581475921443975 -87.84067033823831) bank155738 +155739 POINT(41.0510372590712 -88.57119612927765) bank155739 +155740 POINT(41.598898280747505 -87.88345285090821) bank155740 +155741 POINT(42.86685545674284 -86.85879267587414) bank155741 +155742 POINT(42.41177983510477 -86.73288440660838) bank155742 +155743 POINT(42.22269094284771 -87.40011377263845) bank155743 +155744 POINT(41.32666287873622 -87.16038723940542) bank155744 +155745 POINT(42.46702502922304 -87.89873402456796) bank155745 +155746 POINT(41.67774621158923 -87.59739054888767) bank155746 +155747 POINT(41.577055563527125 -88.09789843686264) bank155747 +155748 POINT(42.059937539351246 -88.40272198330123) bank155748 +155749 POINT(40.98074933312971 -87.52950144929454) bank155749 +155750 POINT(42.771313849531616 -87.36618664782455) bank155750 +155751 POINT(41.91443492633059 -88.57208285982873) bank155751 +155752 POINT(42.74679575835212 -86.95945237509622) bank155752 +155753 POINT(42.17183261696331 -88.61397603817454) bank155753 +155754 POINT(41.28271945521884 -88.00653409168605) bank155754 +155755 POINT(42.83821230454874 -87.4590805141948) bank155755 +155756 POINT(41.316705805977485 -87.5295068756865) bank155756 +155757 POINT(41.37445111839822 -87.21487227804971) bank155757 +155758 POINT(42.39590396167782 -88.055114883923) bank155758 +155759 POINT(42.36687874678408 -86.80076832180518) bank155759 +155760 POINT(42.85311060162413 -87.16087487869211) bank155760 +155761 POINT(40.975083163845845 -88.59182803488363) bank155761 +155762 POINT(41.63018673813457 -87.27170588909567) bank155762 +155763 POINT(42.691348277104055 -88.13961345390251) bank155763 +155764 POINT(42.27833624460961 -87.57029743909098) bank155764 +155765 POINT(41.02063038057139 -88.09185214477426) bank155765 +155766 POINT(41.390120532254926 -88.46316481874315) bank155766 +155767 POINT(41.13488512316184 -87.00385743414544) bank155767 +155768 POINT(41.40060704531998 -87.81719602950618) bank155768 +155769 POINT(41.809133623762065 -87.0997517362849) bank155769 +155770 POINT(42.3539130708263 -86.65159114738977) bank155770 +155771 POINT(41.126787130086264 -87.93831648021907) bank155771 +155772 POINT(41.53174563155027 -86.99321507916314) bank155772 +155773 POINT(42.06109635541743 -87.54040461867153) bank155773 +155774 POINT(41.712614231328786 -87.3900243461498) bank155774 +155775 POINT(41.21856122997102 -86.73936874422796) bank155775 +155776 POINT(41.4011385885966 -88.41815927834512) bank155776 +155777 POINT(42.697451630568544 -86.95506104771758) bank155777 +155778 POINT(41.18424102349636 -88.01681472851065) bank155778 +155779 POINT(41.015427187995364 -88.45579953311633) bank155779 +155780 POINT(41.1710347257559 -88.08963938689791) bank155780 +155781 POINT(42.080103288637076 -87.14433244304631) bank155781 +155782 POINT(42.86465384046734 -88.1900444538402) bank155782 +155783 POINT(41.90490387448749 -87.15181681070436) bank155783 +155784 POINT(42.67187513013977 -86.76440093673514) bank155784 +155785 POINT(41.096449587958354 -87.00547039913238) bank155785 +155786 POINT(42.807907900168594 -86.80652657776034) bank155786 +155787 POINT(42.43077400360642 -87.86110505573849) bank155787 +155788 POINT(41.01216092144642 -87.4840701550877) bank155788 +155789 POINT(42.34703277551042 -88.60359008456282) bank155789 +155790 POINT(42.71925570689073 -88.50193078297828) bank155790 +155791 POINT(42.312817237879614 -86.7842503627136) bank155791 +155792 POINT(42.39987540533729 -87.1752254879013) bank155792 +155793 POINT(42.51464290101789 -88.1100233889996) bank155793 +155794 POINT(42.774719984970794 -86.73606372144548) bank155794 +155795 POINT(41.89192466600072 -86.67753853806177) bank155795 +155796 POINT(42.59197458700708 -87.05705971006675) bank155796 +155797 POINT(42.18744768375606 -87.81002579309745) bank155797 +155798 POINT(42.293720181039056 -88.04480590548367) bank155798 +155799 POINT(41.6690293285101 -87.7706111795761) bank155799 +155800 POINT(40.96893033112585 -87.35937196929942) bank155800 +155801 POINT(42.2917061152175 -87.83411527832456) bank155801 +155802 POINT(41.38530375066172 -87.86719290416062) bank155802 +155803 POINT(41.136084248556934 -87.2236209374712) bank155803 +155804 POINT(41.52680929279284 -87.04465658888242) bank155804 +155805 POINT(42.79516246600511 -86.977212530501) bank155805 +155806 POINT(42.62923899268404 -86.99028092604532) bank155806 +155807 POINT(41.70532462297403 -87.32210352958938) bank155807 +155808 POINT(42.122212541915566 -86.6476297078795) bank155808 +155809 POINT(42.64283736758976 -87.26948150874273) bank155809 +155810 POINT(42.850709643064945 -88.08589849543057) bank155810 +155811 POINT(41.12817178434509 -87.77572564401618) bank155811 +155812 POINT(42.82458105490433 -87.74379553163772) bank155812 +155813 POINT(41.46346512992414 -87.12287955269487) bank155813 +155814 POINT(41.32877251448069 -87.6072345958969) bank155814 +155815 POINT(41.04587773978958 -87.63158683434541) bank155815 +155816 POINT(41.24788972995769 -88.071851685338) bank155816 +155817 POINT(41.28661383564939 -88.07067734991193) bank155817 +155818 POINT(41.98175386047132 -87.0043942527809) bank155818 +155819 POINT(40.91109007292657 -87.39872143699947) bank155819 +155820 POINT(41.640997931992196 -86.74472686874388) bank155820 +155821 POINT(41.13680571242196 -87.47981619736952) bank155821 +155822 POINT(42.08465857790786 -86.88042208952976) bank155822 +155823 POINT(41.27818291283832 -88.52381601408652) bank155823 +155824 POINT(42.69739274016924 -87.7425759456546) bank155824 +155825 POINT(41.80320060219257 -87.50754007285458) bank155825 +155826 POINT(41.56046240896656 -88.40732542138312) bank155826 +155827 POINT(42.77289631539007 -87.43225799150285) bank155827 +155828 POINT(42.64462804715288 -88.04863435983792) bank155828 +155829 POINT(41.716363794074525 -87.13005033152312) bank155829 +155830 POINT(42.227202603643576 -87.08789022066401) bank155830 +155831 POINT(42.108422317209694 -87.18178720163768) bank155831 +155832 POINT(41.221806765247734 -87.26011510224119) bank155832 +155833 POINT(41.70690867674655 -88.0131453709212) bank155833 +155834 POINT(41.74504969877294 -88.14164587237279) bank155834 +155835 POINT(42.569983398452386 -86.84387366731237) bank155835 +155836 POINT(41.93665601777511 -86.6466754366486) bank155836 +155837 POINT(41.90699686698293 -88.31224975546992) bank155837 +155838 POINT(41.26960579709751 -88.56211310898671) bank155838 +155839 POINT(41.02655102199478 -86.65116807102692) bank155839 +155840 POINT(41.12477182822971 -86.96569935314022) bank155840 +155841 POINT(42.58245439752494 -87.71613231089532) bank155841 +155842 POINT(40.91039189648154 -88.22383370761227) bank155842 +155843 POINT(41.746314746013475 -88.49788108028119) bank155843 +155844 POINT(41.7816342285965 -87.85585690004318) bank155844 +155845 POINT(41.98013801181541 -87.99379064107583) bank155845 +155846 POINT(42.06381029999549 -87.46028636805211) bank155846 +155847 POINT(41.12145988789225 -88.14389428050936) bank155847 +155848 POINT(42.12179716977693 -88.10893916404255) bank155848 +155849 POINT(42.40974485554303 -87.58801903950257) bank155849 +155850 POINT(41.20991977051096 -87.69903074611962) bank155850 +155851 POINT(42.30464397869015 -87.37630525006188) bank155851 +155852 POINT(41.810533566434856 -87.8578770420708) bank155852 +155853 POINT(42.13173454846293 -87.75142899029557) bank155853 +155854 POINT(42.58809158272789 -87.86094468722102) bank155854 +155855 POINT(42.36294328508801 -87.031176367171) bank155855 +155856 POINT(41.378180232685715 -87.40785430361673) bank155856 +155857 POINT(42.09608026549394 -87.12884531911264) bank155857 +155858 POINT(42.25977157403624 -87.52799818974263) bank155858 +155859 POINT(42.500160497271786 -88.6114569159101) bank155859 +155860 POINT(42.2578194828985 -87.65606090997032) bank155860 +155861 POINT(41.91831707404369 -88.44740307396502) bank155861 +155862 POINT(42.76512910459436 -88.52194077601709) bank155862 +155863 POINT(41.78922752999565 -87.64121742024459) bank155863 +155864 POINT(41.22413861385073 -87.5198707701784) bank155864 +155865 POINT(41.446216839285405 -88.34762928244095) bank155865 +155866 POINT(41.58399770652474 -87.72579008193603) bank155866 +155867 POINT(42.53611623487503 -87.58078828552807) bank155867 +155868 POINT(42.43574505462899 -87.59217145902115) bank155868 +155869 POINT(41.94816156980252 -88.28687768769696) bank155869 +155870 POINT(42.27099867321847 -87.46496018577277) bank155870 +155871 POINT(41.20818771210502 -87.99539090433267) bank155871 +155872 POINT(41.05492323025585 -87.27232829111563) bank155872 +155873 POINT(41.446068538008255 -88.23458348423274) bank155873 +155874 POINT(42.69797231330064 -88.03707035657223) bank155874 +155875 POINT(41.746570972956235 -87.08522793413773) bank155875 +155876 POINT(41.40814667471974 -87.07883827446734) bank155876 +155877 POINT(41.71063054165767 -87.13149696256023) bank155877 +155878 POINT(42.73636013127649 -87.46505253632324) bank155878 +155879 POINT(41.574008990716955 -87.77411489306138) bank155879 +155880 POINT(42.01879252881646 -87.78298924223988) bank155880 +155881 POINT(41.029928868936736 -87.38877174818234) bank155881 +155882 POINT(40.95393880063935 -87.00674009151582) bank155882 +155883 POINT(41.090469019632785 -88.37174745768175) bank155883 +155884 POINT(41.596447601361696 -86.93667191447312) bank155884 +155885 POINT(42.87098964330958 -87.05433143197403) bank155885 +155886 POINT(42.454876390577574 -86.85116093289889) bank155886 +155887 POINT(42.03800812371681 -87.76986534265434) bank155887 +155888 POINT(40.93986668924557 -87.30095543180774) bank155888 +155889 POINT(41.75068614518665 -87.17109510132296) bank155889 +155890 POINT(42.29610646869653 -87.89816028166615) bank155890 +155891 POINT(41.115846686522325 -87.5509939708427) bank155891 +155892 POINT(42.54791511967562 -86.66739076081015) bank155892 +155893 POINT(42.059137214744524 -87.31748166236027) bank155893 +155894 POINT(42.690866564543036 -87.41326500934703) bank155894 +155895 POINT(42.504566203756426 -87.36450727278991) bank155895 +155896 POINT(42.62421800609474 -87.85186170656314) bank155896 +155897 POINT(42.00833981550278 -87.76839128013265) bank155897 +155898 POINT(40.913387550155605 -87.00170248763081) bank155898 +155899 POINT(41.80989546556885 -86.77280978055869) bank155899 +155900 POINT(41.11204071528981 -88.46252738272412) bank155900 +155901 POINT(42.64555529643661 -88.11625810961775) bank155901 +155902 POINT(41.53845070090341 -88.09909699175574) bank155902 +155903 POINT(41.653037883561915 -87.00558152381745) bank155903 +155904 POINT(41.68653549273032 -87.84657941704582) bank155904 +155905 POINT(41.2511910472916 -86.79129587218735) bank155905 +155906 POINT(41.822800085242974 -88.11823097178484) bank155906 +155907 POINT(42.54784483456571 -87.78902922393746) bank155907 +155908 POINT(41.964529662490115 -87.96910473799905) bank155908 +155909 POINT(42.70271361349975 -88.32780103371447) bank155909 +155910 POINT(41.741878488313745 -87.70809176571154) bank155910 +155911 POINT(42.597971763865 -88.50272081984276) bank155911 +155912 POINT(41.131345746766755 -88.05027376759975) bank155912 +155913 POINT(42.508548735726286 -86.90834463636133) bank155913 +155914 POINT(41.1585047202287 -87.18968936966458) bank155914 +155915 POINT(41.10561612122935 -87.69038693221034) bank155915 +155916 POINT(42.81159890614471 -87.79662162041694) bank155916 +155917 POINT(42.511760158238275 -88.61276085516678) bank155917 +155918 POINT(42.0516163638636 -86.98086701301993) bank155918 +155919 POINT(42.47572821922778 -87.56241952295701) bank155919 +155920 POINT(41.05071107884206 -88.12339374886325) bank155920 +155921 POINT(41.75981097713605 -87.32405563970359) bank155921 +155922 POINT(42.63087486211732 -87.3000729794769) bank155922 +155923 POINT(42.83046590392192 -86.83116967545958) bank155923 +155924 POINT(41.799709872447394 -87.103546449217) bank155924 +155925 POINT(40.988658267869035 -86.99578518643209) bank155925 +155926 POINT(42.67618324541598 -87.81809472106305) bank155926 +155927 POINT(42.44276108146881 -87.92921933542819) bank155927 +155928 POINT(42.2569218907214 -86.73502024797537) bank155928 +155929 POINT(41.59795932628084 -87.16958287879098) bank155929 +155930 POINT(41.49715292239388 -87.59942200595678) bank155930 +155931 POINT(41.76849660938704 -88.60456714328996) bank155931 +155932 POINT(41.73789562535349 -87.51672051760524) bank155932 +155933 POINT(41.78495971864544 -87.69158830675694) bank155933 +155934 POINT(41.77125857790833 -87.05700339385642) bank155934 +155935 POINT(42.78129751194589 -87.06767722298419) bank155935 +155936 POINT(42.566100627478896 -87.9606622045921) bank155936 +155937 POINT(42.73880439611814 -87.20157328021595) bank155937 +155938 POINT(42.70835738087263 -88.153961632915) bank155938 +155939 POINT(42.353098309761045 -88.34983709370833) bank155939 +155940 POINT(41.359080469842674 -87.0196993973698) bank155940 +155941 POINT(41.342850154723735 -88.17088603920172) bank155941 +155942 POINT(40.8931196427199 -87.64310526366401) bank155942 +155943 POINT(41.46455161256778 -87.22037367760868) bank155943 +155944 POINT(42.11819778754276 -87.98904007811265) bank155944 +155945 POINT(40.99539219026367 -88.5153840636836) bank155945 +155946 POINT(41.325820360877174 -87.56598139990912) bank155946 +155947 POINT(41.648076532758076 -87.40131095721593) bank155947 +155948 POINT(41.32343069016727 -88.02416833926347) bank155948 +155949 POINT(42.0819882827042 -88.20100717246555) bank155949 +155950 POINT(42.054689729487116 -86.73879782117682) bank155950 +155951 POINT(41.36006342613082 -87.98030885290983) bank155951 +155952 POINT(42.202781358925236 -87.80045211780576) bank155952 +155953 POINT(41.81719014535468 -87.4387315413037) bank155953 +155954 POINT(42.65269838301076 -88.0638932602296) bank155954 +155955 POINT(41.75170081799422 -86.94376048886294) bank155955 +155956 POINT(41.43527905332757 -86.88779606819062) bank155956 +155957 POINT(41.41744324373687 -87.2013487415283) bank155957 +155958 POINT(42.15197524025505 -87.54474381533416) bank155958 +155959 POINT(42.50420138886571 -87.19968196381741) bank155959 +155960 POINT(41.984555837561125 -87.5390899642637) bank155960 +155961 POINT(42.82854342363227 -87.90461968229631) bank155961 +155962 POINT(41.96214450712126 -87.7472184950881) bank155962 +155963 POINT(41.68237427318213 -86.66721490616611) bank155963 +155964 POINT(41.462157384261225 -87.47168584562046) bank155964 +155965 POINT(42.537010847453296 -88.48842233366335) bank155965 +155966 POINT(42.83576196435901 -87.37629937927976) bank155966 +155967 POINT(40.98498917772159 -87.4154201733048) bank155967 +155968 POINT(41.10513642308108 -87.09567139581516) bank155968 +155969 POINT(42.656952078924675 -86.72605983367023) bank155969 +155970 POINT(41.2685001487021 -87.6495918314171) bank155970 +155971 POINT(41.528022038175074 -86.82570330021957) bank155971 +155972 POINT(42.415771330367456 -88.24729978175537) bank155972 +155973 POINT(41.271654347769505 -86.78653391423899) bank155973 +155974 POINT(42.29825574507484 -87.72388989433455) bank155974 +155975 POINT(41.80257328507594 -87.168547367277) bank155975 +155976 POINT(42.86861705850176 -86.9208922875759) bank155976 +155977 POINT(40.98818268620053 -87.16997671808257) bank155977 +155978 POINT(42.633426757323846 -86.87651581285661) bank155978 +155979 POINT(42.258152918688005 -87.75744778554038) bank155979 +155980 POINT(42.8168675718019 -87.04544587627231) bank155980 +155981 POINT(41.26290651239255 -86.88015928031196) bank155981 +155982 POINT(42.66762060894974 -87.60765978720538) bank155982 +155983 POINT(41.68935273107398 -88.58301781260425) bank155983 +155984 POINT(42.28178152188231 -88.21028676413341) bank155984 +155985 POINT(41.571856422112646 -87.00879193682172) bank155985 +155986 POINT(41.970737812192674 -87.11753018328974) bank155986 +155987 POINT(42.53004539137174 -86.79224704693728) bank155987 +155988 POINT(42.73453595211901 -88.11422544704213) bank155988 +155989 POINT(41.86521928676497 -87.79574163240898) bank155989 +155990 POINT(42.413527080573914 -87.4196909777364) bank155990 +155991 POINT(41.73661922805846 -87.60336112808378) bank155991 +155992 POINT(42.16645946500257 -87.75700987769261) bank155992 +155993 POINT(40.95404438556749 -87.5764467556299) bank155993 +155994 POINT(42.19330616960776 -88.09209654086426) bank155994 +155995 POINT(41.030945966353755 -86.64759838291423) bank155995 +155996 POINT(41.93572433095643 -87.17678261279278) bank155996 +155997 POINT(42.78451606192544 -87.80730589682362) bank155997 +155998 POINT(42.750272681996236 -86.8743391841325) bank155998 +155999 POINT(41.402837560195856 -86.82954470292951) bank155999 +156000 POINT(42.66522757016113 -88.16444396827983) bank156000 +156001 POINT(42.500805111764556 -88.53308193355612) bank156001 +156002 POINT(42.186348341653634 -86.66268530544808) bank156002 +156003 POINT(42.7914134335563 -86.83385552938094) bank156003 +156004 POINT(42.64229316566477 -88.0255745190743) bank156004 +156005 POINT(41.7363140094051 -87.08057214108304) bank156005 +156006 POINT(42.60732605093419 -87.72713006979204) bank156006 +156007 POINT(42.521699420370055 -87.57108031241482) bank156007 +156008 POINT(41.53421562372264 -88.52683392260693) bank156008 +156009 POINT(42.772739445877136 -88.50496894925148) bank156009 +156010 POINT(41.43538708566463 -88.25723334640298) bank156010 +156011 POINT(41.2416386888745 -87.59246988125311) bank156011 +156012 POINT(41.23335156566869 -86.84095520955668) bank156012 +156013 POINT(41.28850076810138 -87.13399706331039) bank156013 +156014 POINT(41.543678368114826 -87.68738713865392) bank156014 +156015 POINT(40.98402638089192 -87.93687846126599) bank156015 +156016 POINT(41.99168815398973 -88.24753927391676) bank156016 +156017 POINT(42.660656888466534 -87.32213033268039) bank156017 +156018 POINT(41.62550834364383 -88.61279429479475) bank156018 +156019 POINT(42.03912867015057 -88.11829873511192) bank156019 +156020 POINT(41.13679501625374 -87.75007548960482) bank156020 +156021 POINT(41.44962597084269 -86.92141558759124) bank156021 +156022 POINT(42.55226013904769 -87.52232583214266) bank156022 +156023 POINT(41.493830642584406 -88.61560644791084) bank156023 +156024 POINT(41.385017033593094 -87.01559601871152) bank156024 +156025 POINT(41.016018441039314 -86.9068492607547) bank156025 +156026 POINT(42.246538488600486 -87.25508697798927) bank156026 +156027 POINT(42.63740765371297 -86.98017237391701) bank156027 +156028 POINT(41.44920538641447 -86.94100496241721) bank156028 +156029 POINT(42.39801403377106 -88.01929174899574) bank156029 +156030 POINT(42.14900963457235 -87.64738369669183) bank156030 +156031 POINT(42.67931187494606 -88.55377141676252) bank156031 +156032 POINT(41.325081947733885 -87.66278366091723) bank156032 +156033 POINT(41.80117150965081 -87.66494132546586) bank156033 +156034 POINT(41.512834987740625 -87.37446538139949) bank156034 +156035 POINT(41.60470221315748 -88.00865441831547) bank156035 +156036 POINT(41.25896406617777 -87.58441670780468) bank156036 +156037 POINT(41.574179986065595 -88.39789358239088) bank156037 +156038 POINT(41.389747196977524 -87.2667479617666) bank156038 +156039 POINT(41.14912343757203 -87.97748463491321) bank156039 +156040 POINT(41.53202256884945 -86.84578320474048) bank156040 +156041 POINT(42.144763561667936 -88.40822389612771) bank156041 +156042 POINT(41.80158504113836 -87.38567286417663) bank156042 +156043 POINT(41.467519588627944 -87.91462718306157) bank156043 +156044 POINT(42.28441222928031 -86.81178140119717) bank156044 +156045 POINT(41.51064486595071 -87.57877355162017) bank156045 +156046 POINT(42.680422806642405 -87.37696632528495) bank156046 +156047 POINT(41.493544603766296 -88.09506182930006) bank156047 +156048 POINT(42.62570178326365 -86.7153543602789) bank156048 +156049 POINT(41.59235100639777 -87.71437922959748) bank156049 +156050 POINT(40.9535262266377 -88.5369620050629) bank156050 +156051 POINT(41.70346459991277 -87.9226497148785) bank156051 +156052 POINT(41.368542363327876 -87.85822698062914) bank156052 +156053 POINT(41.889947826017234 -88.27847558433497) bank156053 +156054 POINT(41.440919163840405 -86.81102864297414) bank156054 +156055 POINT(40.937913182936995 -87.56534234467661) bank156055 +156056 POINT(42.78853202470014 -88.09933166841309) bank156056 +156057 POINT(42.064971624253275 -88.28465326250522) bank156057 +156058 POINT(41.02112232215576 -87.45510808428328) bank156058 +156059 POINT(42.502520353916054 -88.54926665264425) bank156059 +156060 POINT(41.304749701908314 -86.97226465068103) bank156060 +156061 POINT(40.99035082814411 -88.17595386263164) bank156061 +156062 POINT(42.32488978594301 -88.4159455938457) bank156062 +156063 POINT(42.35237496754418 -87.20190149070753) bank156063 +156064 POINT(42.745171193429464 -87.7926981461345) bank156064 +156065 POINT(42.14207121791221 -87.40835675634084) bank156065 +156066 POINT(42.4868174700047 -87.38439062578874) bank156066 +156067 POINT(41.87383020497358 -88.02519560776213) bank156067 +156068 POINT(41.349923474441354 -87.92354713269587) bank156068 +156069 POINT(41.473326124172594 -87.23152099731502) bank156069 +156070 POINT(42.64710300815873 -88.49157102055806) bank156070 +156071 POINT(41.25817202808423 -87.72562850723382) bank156071 +156072 POINT(41.299007544428996 -87.43263493695156) bank156072 +156073 POINT(40.910735679712225 -87.76109667705981) bank156073 +156074 POINT(41.141591301104405 -87.11509512611957) bank156074 +156075 POINT(42.751271416681064 -87.50702771573931) bank156075 +156076 POINT(42.55972514470454 -88.32751719221137) bank156076 +156077 POINT(41.830045954029366 -87.31867422027705) bank156077 +156078 POINT(42.12427356644778 -88.17292064567368) bank156078 +156079 POINT(41.57850374111995 -88.35706050217145) bank156079 +156080 POINT(40.91646971337891 -88.44520642466229) bank156080 +156081 POINT(42.648651637454904 -87.2086969164786) bank156081 +156082 POINT(41.757160524641414 -87.84408694998454) bank156082 +156083 POINT(40.91557863227321 -88.33789795186036) bank156083 +156084 POINT(41.407573678322144 -88.2481324066271) bank156084 +156085 POINT(41.2422600691444 -88.17403328305728) bank156085 +156086 POINT(40.95658841770559 -88.6187007547914) bank156086 +156087 POINT(41.463525874108605 -86.88011795016054) bank156087 +156088 POINT(41.54799176587418 -86.65259885013424) bank156088 +156089 POINT(41.86683304481191 -88.22234293610008) bank156089 +156090 POINT(42.76236849737286 -88.21279938407268) bank156090 +156091 POINT(41.17505625045174 -87.5203202795461) bank156091 +156092 POINT(41.52476325508531 -87.70646968528207) bank156092 +156093 POINT(42.1783695697663 -86.82853499504654) bank156093 +156094 POINT(41.01138650329641 -87.67579554113439) bank156094 +156095 POINT(41.28128752845653 -88.36906723511495) bank156095 +156096 POINT(42.636947609616556 -87.03489824434935) bank156096 +156097 POINT(42.44671929901466 -88.11524133374746) bank156097 +156098 POINT(42.853600953710995 -88.62333514331654) bank156098 +156099 POINT(42.46543571538636 -87.23020622186255) bank156099 +156100 POINT(41.25062439913583 -87.23608282768106) bank156100 +156101 POINT(42.262081572338836 -88.47165966675811) bank156101 +156102 POINT(41.85675620807398 -88.0125249309173) bank156102 +156103 POINT(42.4549119374283 -87.37264628094657) bank156103 +156104 POINT(41.27400277815444 -88.34778185899552) bank156104 +156105 POINT(41.25550760141117 -87.25105004781292) bank156105 +156106 POINT(42.48300716585366 -88.566628536271) bank156106 +156107 POINT(40.995204814700216 -88.33951642293309) bank156107 +156108 POINT(41.51927775916434 -88.12575567426344) bank156108 +156109 POINT(41.87050020106903 -88.26161895537237) bank156109 +156110 POINT(41.84458525319019 -87.08883044926486) bank156110 +156111 POINT(42.19891318247884 -87.29839224184153) bank156111 +156112 POINT(42.167177391157374 -87.93440198701998) bank156112 +156113 POINT(42.747566343445506 -86.82312023360159) bank156113 +156114 POINT(40.94761228858507 -88.62797344279606) bank156114 +156115 POINT(41.49206512104474 -86.81622246649039) bank156115 +156116 POINT(41.17102183556074 -86.69548467123316) bank156116 +156117 POINT(42.43873700230009 -86.7203141580192) bank156117 +156118 POINT(42.13699701420296 -88.36928145262732) bank156118 +156119 POINT(42.39072291972332 -88.41407358283145) bank156119 +156120 POINT(40.93177503102576 -88.09472927268723) bank156120 +156121 POINT(41.264556943907195 -88.1127926392173) bank156121 +156122 POINT(42.32762368049455 -87.17071488632234) bank156122 +156123 POINT(42.432620622798865 -86.74101757802995) bank156123 +156124 POINT(41.250189708836174 -87.91260084765563) bank156124 +156125 POINT(40.993195173640586 -87.55732217109761) bank156125 +156126 POINT(41.464149416514665 -87.61281655046987) bank156126 +156127 POINT(42.22594410180973 -87.1816026941066) bank156127 +156128 POINT(42.542688531331436 -87.83080016413022) bank156128 +156129 POINT(42.7089058762369 -87.18616109594588) bank156129 +156130 POINT(42.85035269553865 -88.08648895606166) bank156130 +156131 POINT(42.65349132045866 -87.83672462429845) bank156131 +156132 POINT(41.047019485362604 -88.30765675799056) bank156132 +156133 POINT(42.746403198787775 -87.35061195586975) bank156133 +156134 POINT(42.262797840430025 -87.70296792139008) bank156134 +156135 POINT(42.26753387749194 -87.14235689102544) bank156135 +156136 POINT(41.11632663359323 -88.56486827282795) bank156136 +156137 POINT(42.67953424254259 -87.62992083102304) bank156137 +156138 POINT(41.55335810261521 -87.65510921967704) bank156138 +156139 POINT(41.2723211819386 -87.56899154858489) bank156139 +156140 POINT(42.16820580616457 -86.92187190598698) bank156140 +156141 POINT(41.471361072257736 -87.46417232812978) bank156141 +156142 POINT(42.87643320530471 -88.58288997396558) bank156142 +156143 POINT(42.27756216962915 -88.41298858917287) bank156143 +156144 POINT(41.27317773624484 -87.0724631331038) bank156144 +156145 POINT(41.52866607467505 -87.49448233701855) bank156145 +156146 POINT(41.71957661357114 -87.53027648837748) bank156146 +156147 POINT(41.1508096559042 -87.58127102306405) bank156147 +156148 POINT(41.56588197837219 -88.31003788653592) bank156148 +156149 POINT(42.08098123012247 -86.75770699654927) bank156149 +156150 POINT(42.45691775651538 -87.83579504898869) bank156150 +156151 POINT(40.975288096085315 -87.52421925436096) bank156151 +156152 POINT(41.48530204736159 -87.5039512699541) bank156152 +156153 POINT(42.33878242289175 -86.65600469493387) bank156153 +156154 POINT(42.7391768399541 -86.88757642867203) bank156154 +156155 POINT(41.04422786146659 -87.4878477583548) bank156155 +156156 POINT(40.97520153260534 -87.86482378627076) bank156156 +156157 POINT(42.610836716773115 -87.61413413883784) bank156157 +156158 POINT(42.243880879667685 -87.61125780575951) bank156158 +156159 POINT(42.41610572173201 -87.66260879234937) bank156159 +156160 POINT(42.81033209349579 -88.12929277182972) bank156160 +156161 POINT(41.13588388321433 -88.5093594361785) bank156161 +156162 POINT(40.99562274474621 -88.535152304501) bank156162 +156163 POINT(41.06883199138522 -87.22350189850292) bank156163 +156164 POINT(42.47440418701923 -87.15837564739232) bank156164 +156165 POINT(42.54535367059985 -87.9863583364005) bank156165 +156166 POINT(42.45147028725095 -87.30808718855548) bank156166 +156167 POINT(41.366094279506015 -86.94032722950028) bank156167 +156168 POINT(41.440673037858105 -88.27153805740681) bank156168 +156169 POINT(42.128603502266046 -88.32923889810206) bank156169 +156170 POINT(41.79623092286679 -86.6825416969476) bank156170 +156171 POINT(41.18365740735183 -87.62829496827366) bank156171 +156172 POINT(41.500938721799315 -87.92269700781019) bank156172 +156173 POINT(42.05059874849203 -88.51490429344985) bank156173 +156174 POINT(41.03344154186108 -87.10470866904095) bank156174 +156175 POINT(41.143642790438626 -87.33604478327692) bank156175 +156176 POINT(41.53087938105068 -87.45284144764236) bank156176 +156177 POINT(41.260426364872444 -87.42478959419611) bank156177 +156178 POINT(42.74891668775268 -87.20496772223372) bank156178 +156179 POINT(42.27463179633772 -86.92364380144367) bank156179 +156180 POINT(42.12161667138282 -87.18009155389191) bank156180 +156181 POINT(41.79935177944565 -88.52965551615596) bank156181 +156182 POINT(42.870701128951936 -86.68155182318547) bank156182 +156183 POINT(42.26870261205977 -88.60619941800172) bank156183 +156184 POINT(42.07730948214328 -86.67589341222026) bank156184 +156185 POINT(41.6692834388346 -87.25193083750762) bank156185 +156186 POINT(42.11722089793543 -87.53856477547151) bank156186 +156187 POINT(41.73843840270537 -87.4905914715464) bank156187 +156188 POINT(41.26094404953717 -86.70945670083913) bank156188 +156189 POINT(41.56128775206255 -87.17010816712516) bank156189 +156190 POINT(40.97705878765792 -86.88689450972514) bank156190 +156191 POINT(41.753453630460896 -87.55814268343185) bank156191 +156192 POINT(41.309375242993184 -86.71121431135658) bank156192 +156193 POINT(41.116950296871 -86.69671791747649) bank156193 +156194 POINT(41.25726141356374 -87.1003559182071) bank156194 +156195 POINT(42.24177901256842 -88.09521754210355) bank156195 +156196 POINT(42.35976705990703 -87.37720626556064) bank156196 +156197 POINT(42.22233346398826 -88.24990968005352) bank156197 +156198 POINT(41.726618333753365 -88.0655316817348) bank156198 +156199 POINT(41.055805051843855 -86.736346156376) bank156199 +156200 POINT(41.745312573383465 -87.37874068989571) bank156200 +156201 POINT(41.39528327935815 -87.1826899589763) bank156201 +156202 POINT(42.11244944621055 -87.01520072330514) bank156202 +156203 POINT(41.253215492700015 -87.23025503933476) bank156203 +156204 POINT(41.1905483058255 -87.1248941172988) bank156204 +156205 POINT(42.62946259571458 -87.86135541263684) bank156205 +156206 POINT(41.6132809130096 -87.39271869899898) bank156206 +156207 POINT(42.651479768103904 -88.49887255698003) bank156207 +156208 POINT(41.30457827287564 -87.62339211466454) bank156208 +156209 POINT(42.660562494085475 -88.04685231121708) bank156209 +156210 POINT(41.19393672336514 -88.23873448269056) bank156210 +156211 POINT(41.078093484434376 -86.98891526021438) bank156211 +156212 POINT(41.822775386350294 -87.80093404509881) bank156212 +156213 POINT(41.15580918728201 -87.2129372203313) bank156213 +156214 POINT(41.00818678245047 -87.14723828340972) bank156214 +156215 POINT(42.10708225543209 -87.65164316262913) bank156215 +156216 POINT(42.874767221686135 -88.54253238684042) bank156216 +156217 POINT(41.10100305284173 -87.7323828656579) bank156217 +156218 POINT(41.17497483367077 -87.26579382911551) bank156218 +156219 POINT(41.743849083790046 -88.20435633909614) bank156219 +156220 POINT(41.53346137674078 -87.21875608745798) bank156220 +156221 POINT(42.57658215899155 -86.78259577441969) bank156221 +156222 POINT(41.72875439116295 -88.0525891314541) bank156222 +156223 POINT(41.230914925380304 -88.48943320549252) bank156223 +156224 POINT(41.23781765793708 -87.96839371826208) bank156224 +156225 POINT(41.949753479400336 -88.53275063247023) bank156225 +156226 POINT(40.99981612442889 -87.84899986304356) bank156226 +156227 POINT(41.48568998276289 -86.9801534628615) bank156227 +156228 POINT(41.33664853029823 -87.66736545557704) bank156228 +156229 POINT(41.778053251163755 -87.90676622782495) bank156229 +156230 POINT(42.563277285235635 -87.8200143684412) bank156230 +156231 POINT(42.74109625218959 -86.69320777722598) bank156231 +156232 POINT(41.098836254653655 -88.2411071559429) bank156232 +156233 POINT(40.90894787712114 -86.79502011503986) bank156233 +156234 POINT(42.061948973292274 -87.73034817294494) bank156234 +156235 POINT(41.579721562466396 -86.78415973387993) bank156235 +156236 POINT(41.423629896199515 -88.01032102769729) bank156236 +156237 POINT(42.57607518568068 -87.74004283675126) bank156237 +156238 POINT(42.3999718977381 -87.384892379053) bank156238 +156239 POINT(42.55058028755413 -87.02742607858575) bank156239 +156240 POINT(42.70728636601311 -88.02290435705541) bank156240 +156241 POINT(42.032252655244434 -87.60183975213445) bank156241 +156242 POINT(42.35334357628269 -87.07830236973203) bank156242 +156243 POINT(41.50048249346479 -87.05038599332696) bank156243 +156244 POINT(42.388288765788104 -87.3004500267184) bank156244 +156245 POINT(42.439288738390026 -87.41413184544083) bank156245 +156246 POINT(41.706825679129025 -87.1720137556564) bank156246 +156247 POINT(42.18722067788898 -88.60264863706719) bank156247 +156248 POINT(42.71287839902697 -88.62098240232325) bank156248 +156249 POINT(42.22163969485156 -88.31104093325459) bank156249 +156250 POINT(40.95384193365348 -87.57506518509707) bank156250 +156251 POINT(41.88652321142768 -86.65479994451174) bank156251 +156252 POINT(42.23190980694953 -86.93083263202296) bank156252 +156253 POINT(40.94474477987641 -87.41523109447296) bank156253 +156254 POINT(42.834433623408174 -87.75659743875225) bank156254 +156255 POINT(40.98452659012288 -87.90479604614296) bank156255 +156256 POINT(42.34760819100889 -86.77384242482098) bank156256 +156257 POINT(41.544081676750494 -87.30690213113323) bank156257 +156258 POINT(41.49041143629105 -88.06497139649295) bank156258 +156259 POINT(42.2125141530223 -87.82241415310938) bank156259 +156260 POINT(42.4445025914334 -87.32109787358432) bank156260 +156261 POINT(42.69144107765355 -86.81390383670141) bank156261 +156262 POINT(42.08005311598729 -87.39239863965325) bank156262 +156263 POINT(41.25767423959409 -88.14595656515695) bank156263 +156264 POINT(41.97400191419801 -86.92545982702165) bank156264 +156265 POINT(42.79660369573764 -87.65586102953517) bank156265 +156266 POINT(41.57348203473693 -87.45996328689785) bank156266 +156267 POINT(42.36169900608788 -88.21727052125337) bank156267 +156268 POINT(42.6374312889033 -88.09824761690113) bank156268 +156269 POINT(41.836367925456486 -88.19786005446824) bank156269 +156270 POINT(41.383003500341886 -87.18996988614741) bank156270 +156271 POINT(41.208405374419456 -87.72444116916066) bank156271 +156272 POINT(41.55052181086769 -87.62658738584344) bank156272 +156273 POINT(41.84720264392767 -87.95840870658934) bank156273 +156274 POINT(41.89708799013915 -88.33077836320992) bank156274 +156275 POINT(42.61826167107551 -87.77707877104721) bank156275 +156276 POINT(42.32586233693069 -87.09300260639192) bank156276 +156277 POINT(42.30009333630159 -88.26456155496066) bank156277 +156278 POINT(41.86273202353581 -87.8137426063634) bank156278 +156279 POINT(42.09705881090191 -88.20536375644234) bank156279 +156280 POINT(42.272150211972395 -88.46883136758117) bank156280 +156281 POINT(42.03550319553434 -86.9742762431709) bank156281 +156282 POINT(42.77806785892121 -87.36561360610014) bank156282 +156283 POINT(42.265821980053936 -87.49982725734978) bank156283 +156284 POINT(42.33540527464407 -88.22508586207432) bank156284 +156285 POINT(42.13137376178318 -87.10750805740885) bank156285 +156286 POINT(42.48644013007187 -88.42420087251004) bank156286 +156287 POINT(41.51383235379411 -87.07540189778194) bank156287 +156288 POINT(41.106839848419575 -87.78764851424894) bank156288 +156289 POINT(42.52576254560824 -86.74447558661949) bank156289 +156290 POINT(42.206650034776885 -88.46168829167897) bank156290 +156291 POINT(42.16296960558998 -87.21008963917409) bank156291 +156292 POINT(40.97279925487454 -87.13362186620157) bank156292 +156293 POINT(42.24460274951708 -88.39380335186209) bank156293 +156294 POINT(42.532453958603995 -87.88845724442643) bank156294 +156295 POINT(41.15668712613827 -86.87968126850073) bank156295 +156296 POINT(42.852328187794015 -87.02722905345934) bank156296 +156297 POINT(40.95587551917083 -87.64947223854426) bank156297 +156298 POINT(41.75427317986329 -86.64938793058346) bank156298 +156299 POINT(42.339975593589 -88.14716101030956) bank156299 +156300 POINT(42.05945085016119 -86.86922233387436) bank156300 +156301 POINT(41.73057741507011 -87.8087475974898) bank156301 +156302 POINT(42.526884679010244 -88.36859740378326) bank156302 +156303 POINT(42.22062718660269 -88.21067635135968) bank156303 +156304 POINT(41.58904111392172 -88.19420893282762) bank156304 +156305 POINT(42.665919832414964 -87.24103532484952) bank156305 +156306 POINT(42.523192869060445 -88.55364347083501) bank156306 +156307 POINT(42.340360134398246 -87.08091262584047) bank156307 +156308 POINT(42.60098020535905 -86.69518989368744) bank156308 +156309 POINT(41.383590139514595 -87.68280488422799) bank156309 +156310 POINT(41.7028360051134 -87.83116806822667) bank156310 +156311 POINT(42.73950350145074 -86.66377374980388) bank156311 +156312 POINT(41.85785328124496 -87.74512189064129) bank156312 +156313 POINT(41.244410451152824 -87.977152620062) bank156313 +156314 POINT(42.50674536403091 -87.19282735387984) bank156314 +156315 POINT(41.85088504667626 -88.23742663706774) bank156315 +156316 POINT(41.49758736525735 -87.40508346280475) bank156316 +156317 POINT(42.17888189280705 -87.12806347502988) bank156317 +156318 POINT(41.24831241534362 -87.63967943457521) bank156318 +156319 POINT(41.21354704526026 -88.33350312234387) bank156319 +156320 POINT(42.682973877447395 -87.55086099818013) bank156320 +156321 POINT(41.15477700680461 -87.17741841662811) bank156321 +156322 POINT(41.6024002750465 -87.12802548830697) bank156322 +156323 POINT(41.387127403504586 -88.47442949372655) bank156323 +156324 POINT(41.3007332755439 -87.88071092567876) bank156324 +156325 POINT(41.795898478694006 -88.49820386377996) bank156325 +156326 POINT(41.743378568953105 -87.40053164626451) bank156326 +156327 POINT(41.71608458462222 -86.84132327565919) bank156327 +156328 POINT(41.06084915818906 -86.97699303937883) bank156328 +156329 POINT(41.53817440855487 -86.65545530534301) bank156329 +156330 POINT(41.8531234854877 -86.99507186734944) bank156330 +156331 POINT(41.130676724811536 -87.0119489857049) bank156331 +156332 POINT(42.016428550213476 -87.08181691911629) bank156332 +156333 POINT(41.19570214627578 -87.95582546399916) bank156333 +156334 POINT(41.82672616671328 -88.42634822598441) bank156334 +156335 POINT(41.44414902946918 -88.24894457160153) bank156335 +156336 POINT(41.219195873501135 -88.33509913077636) bank156336 +156337 POINT(41.534339163552474 -88.5664268649315) bank156337 +156338 POINT(41.91319292069542 -87.35332458405264) bank156338 +156339 POINT(42.40037879449214 -87.12942687220749) bank156339 +156340 POINT(42.05160480333994 -87.38076645432898) bank156340 +156341 POINT(41.68289263558992 -88.36226175770203) bank156341 +156342 POINT(42.18848016862193 -88.40425361046375) bank156342 +156343 POINT(42.22013330396343 -88.51089657677842) bank156343 +156344 POINT(41.096012666542066 -86.91391200135193) bank156344 +156345 POINT(41.094102414146036 -87.72591720913053) bank156345 +156346 POINT(41.85491016051341 -88.05167461423011) bank156346 +156347 POINT(41.94213777815859 -87.31491789085801) bank156347 +156348 POINT(41.69216310524629 -87.05519815677593) bank156348 +156349 POINT(42.245073419785015 -87.0210145989603) bank156349 +156350 POINT(42.574036347747956 -87.46895729175735) bank156350 +156351 POINT(42.622201864514864 -88.07206192252399) bank156351 +156352 POINT(42.21819890205427 -88.47806015464516) bank156352 +156353 POINT(41.541401359562855 -87.68575895855541) bank156353 +156354 POINT(41.284018482021615 -87.58527319864034) bank156354 +156355 POINT(41.65107724818786 -88.13930826839342) bank156355 +156356 POINT(42.03970912301834 -87.81407327857706) bank156356 +156357 POINT(41.95636188911213 -87.36840974353545) bank156357 +156358 POINT(42.79156857608867 -88.42949776771857) bank156358 +156359 POINT(41.11237478962004 -87.30191568035272) bank156359 +156360 POINT(42.61183747652892 -87.35776885857527) bank156360 +156361 POINT(41.854559467193425 -88.60016483636562) bank156361 +156362 POINT(40.96305159657647 -88.51443299574755) bank156362 +156363 POINT(41.885622859042485 -87.83308296555595) bank156363 +156364 POINT(42.83058044822476 -87.40678230605475) bank156364 +156365 POINT(42.21970176614527 -87.49728096899246) bank156365 +156366 POINT(41.437470584614665 -86.6617407421094) bank156366 +156367 POINT(41.03951124394909 -88.45206383632193) bank156367 +156368 POINT(41.565799224925854 -87.12087353915612) bank156368 +156369 POINT(40.91352689277211 -87.44773508997103) bank156369 +156370 POINT(42.672519648564005 -87.83924346557224) bank156370 +156371 POINT(42.69307319215832 -87.60713758443947) bank156371 +156372 POINT(41.052534578891844 -88.37105796074486) bank156372 +156373 POINT(42.10912311485882 -88.3332825498012) bank156373 +156374 POINT(41.139211786179786 -88.28572420619656) bank156374 +156375 POINT(41.7818915633281 -87.09568752600839) bank156375 +156376 POINT(40.90266048696764 -88.38097895885805) bank156376 +156377 POINT(41.983072354421914 -87.5350698737625) bank156377 +156378 POINT(41.562473319626086 -88.60230482625529) bank156378 +156379 POINT(42.80466030636225 -88.33931458635388) bank156379 +156380 POINT(41.65734574492791 -87.54643345271836) bank156380 +156381 POINT(41.368537377744275 -88.61861917521456) bank156381 +156382 POINT(41.594810521903845 -87.55286598739563) bank156382 +156383 POINT(41.42363751859122 -87.0907795858461) bank156383 +156384 POINT(42.05961028812704 -87.15495399977097) bank156384 +156385 POINT(41.174608225229605 -88.60306000252668) bank156385 +156386 POINT(42.058181910601235 -88.24290543896068) bank156386 +156387 POINT(42.286030259054236 -88.30656327199866) bank156387 +156388 POINT(42.31896129965815 -87.51728907477516) bank156388 +156389 POINT(41.06271046924838 -88.41437314306079) bank156389 +156390 POINT(41.795104855455456 -86.67124784110253) bank156390 +156391 POINT(41.424333701800876 -87.45045505546665) bank156391 +156392 POINT(42.093860308369514 -87.16502146461008) bank156392 +156393 POINT(42.20719179855372 -86.87779713396755) bank156393 +156394 POINT(42.57694978729773 -87.08481388893013) bank156394 +156395 POINT(41.32194117516924 -88.03502931996228) bank156395 +156396 POINT(41.29706550856599 -86.92326837261388) bank156396 +156397 POINT(42.05853084385207 -88.22758735159404) bank156397 +156398 POINT(41.370982087205746 -87.90140194392532) bank156398 +156399 POINT(42.71117930155283 -86.74260701725736) bank156399 +156400 POINT(42.283298759644765 -87.70871006672132) bank156400 +156401 POINT(42.697175770206805 -87.73705509244118) bank156401 +156402 POINT(42.480560296052275 -87.01340700320392) bank156402 +156403 POINT(41.045608543273914 -87.2327251428357) bank156403 +156404 POINT(42.412232028594275 -87.11988851532648) bank156404 +156405 POINT(42.76319300833428 -87.90365939174308) bank156405 +156406 POINT(42.29808269955114 -88.23597563836768) bank156406 +156407 POINT(42.125243233986254 -87.00561005032743) bank156407 +156408 POINT(42.46056626364245 -86.8560202594526) bank156408 +156409 POINT(41.569423273349656 -88.07265175326104) bank156409 +156410 POINT(42.45428461732044 -87.20265782550392) bank156410 +156411 POINT(42.01750242657608 -87.56006801166505) bank156411 +156412 POINT(41.63004919894894 -88.13248219293249) bank156412 +156413 POINT(42.21421670025937 -87.26091206014692) bank156413 +156414 POINT(41.774456157223966 -86.89184928442097) bank156414 +156415 POINT(41.83548948422598 -87.18906377799806) bank156415 +156416 POINT(40.90607950703662 -86.84506990404442) bank156416 +156417 POINT(41.518317985852185 -88.29552139548272) bank156417 +156418 POINT(42.63803571712681 -87.16268094339266) bank156418 +156419 POINT(41.92782271922188 -88.37903399530188) bank156419 +156420 POINT(41.43405469022713 -87.281451949471) bank156420 +156421 POINT(42.87435631439963 -87.04030138252398) bank156421 +156422 POINT(42.23139572855111 -88.48387365682419) bank156422 +156423 POINT(41.67158070964461 -87.15382269654337) bank156423 +156424 POINT(40.969162692739715 -87.74067375565437) bank156424 +156425 POINT(41.80686453729305 -87.10953398349041) bank156425 +156426 POINT(42.6827846732113 -86.67546441630729) bank156426 +156427 POINT(42.60880690466687 -88.4085693619763) bank156427 +156428 POINT(40.88395016828392 -87.34984181066874) bank156428 +156429 POINT(41.722453553793954 -87.25681177561026) bank156429 +156430 POINT(41.26892639400354 -87.40072211641719) bank156430 +156431 POINT(41.562446924342474 -86.74807801184234) bank156431 +156432 POINT(41.06806605635531 -88.4635057744353) bank156432 +156433 POINT(42.31755000485516 -86.7692582111764) bank156433 +156434 POINT(41.83827036395396 -88.33779896755152) bank156434 +156435 POINT(41.60948263738269 -87.4421612695364) bank156435 +156436 POINT(42.218276885070956 -86.98677530797092) bank156436 +156437 POINT(42.55014713945342 -87.21021141799115) bank156437 +156438 POINT(41.84139587084853 -87.36209631247615) bank156438 +156439 POINT(40.93120131762068 -87.13755695305957) bank156439 +156440 POINT(41.89356012859388 -88.21447313319312) bank156440 +156441 POINT(40.87968357227546 -86.72284788082213) bank156441 +156442 POINT(42.519914775414435 -88.26066528906269) bank156442 +156443 POINT(41.13352148874566 -87.30058517294982) bank156443 +156444 POINT(41.1536748779753 -88.17670434040956) bank156444 +156445 POINT(42.778358978715936 -87.91323301010347) bank156445 +156446 POINT(42.63475541093156 -88.28916755814389) bank156446 +156447 POINT(42.235003400961645 -87.00123862983298) bank156447 +156448 POINT(41.69171421330771 -87.35877661700702) bank156448 +156449 POINT(42.78256183618237 -87.32349253970158) bank156449 +156450 POINT(42.753377347889014 -86.64166473094993) bank156450 +156451 POINT(41.078979713289776 -87.36968368146063) bank156451 +156452 POINT(41.83512197635069 -87.15397882676083) bank156452 +156453 POINT(42.290352732584196 -87.74983946490406) bank156453 +156454 POINT(41.864676756034314 -87.46029975224418) bank156454 +156455 POINT(41.222953895644615 -86.86692538241213) bank156455 +156456 POINT(42.5730131572895 -86.6942881416927) bank156456 +156457 POINT(42.0608627250492 -88.41792680439114) bank156457 +156458 POINT(42.85800155582124 -88.4268600121074) bank156458 +156459 POINT(41.9187204249296 -86.63345557794949) bank156459 +156460 POINT(42.686933315147996 -87.59833822953931) bank156460 +156461 POINT(42.252877790361225 -87.54506925959903) bank156461 +156462 POINT(42.3377376023476 -86.72158649353527) bank156462 +156463 POINT(41.71224414523914 -87.83964911704106) bank156463 +156464 POINT(41.89333760574809 -87.7051354112934) bank156464 +156465 POINT(41.56426194630896 -87.91702330010658) bank156465 +156466 POINT(42.2846553431178 -87.17207342064923) bank156466 +156467 POINT(42.05117167334791 -88.54070226074019) bank156467 +156468 POINT(42.28257670905514 -88.31414203610737) bank156468 +156469 POINT(42.62957446176394 -87.0209697945543) bank156469 +156470 POINT(41.765753315269194 -86.91865939586363) bank156470 +156471 POINT(42.143949471576235 -88.56965047882787) bank156471 +156472 POINT(41.23283575925328 -88.48339398686613) bank156472 +156473 POINT(42.59836577499291 -87.7709970340054) bank156473 +156474 POINT(41.225149458386994 -87.1219409887697) bank156474 +156475 POINT(41.478034117981025 -88.06574933775617) bank156475 +156476 POINT(42.74038190759937 -87.45402714367196) bank156476 +156477 POINT(42.04110751521453 -86.97090584086978) bank156477 +156478 POINT(41.7973970077006 -87.79735485544643) bank156478 +156479 POINT(41.228156177541045 -86.85481222756434) bank156479 +156480 POINT(41.153904025894875 -87.65509094930106) bank156480 +156481 POINT(41.729994596095246 -87.58575905828987) bank156481 +156482 POINT(41.84118201518675 -87.44734010716516) bank156482 +156483 POINT(42.87677106751169 -86.91106950129242) bank156483 +156484 POINT(41.442010859144546 -88.52740086106047) bank156484 +156485 POINT(41.756428344545476 -87.9435203549736) bank156485 +156486 POINT(41.511656991937436 -88.50308075155506) bank156486 +156487 POINT(42.767055246360314 -87.47903729557538) bank156487 +156488 POINT(41.4816158547119 -87.08917301945085) bank156488 +156489 POINT(42.130836613980236 -88.18246661190102) bank156489 +156490 POINT(41.65217664950516 -88.18670321638999) bank156490 +156491 POINT(41.10483586432492 -88.16295869891435) bank156491 +156492 POINT(41.435593123340404 -87.44785731564791) bank156492 +156493 POINT(41.74888470927218 -88.20469306726172) bank156493 +156494 POINT(42.82620538589068 -87.22813323073339) bank156494 +156495 POINT(42.23325528581804 -87.90818610613243) bank156495 +156496 POINT(41.95062599857697 -87.85493065747836) bank156496 +156497 POINT(40.933260385884275 -88.12900142684852) bank156497 +156498 POINT(42.17003097325601 -86.69981553642742) bank156498 +156499 POINT(41.261289169711716 -87.70545903077688) bank156499 +156500 POINT(42.61562701873214 -87.1551311858154) bank156500 +156501 POINT(41.40032045568376 -87.0493851620133) bank156501 +156502 POINT(42.5944080243545 -88.61628900455953) bank156502 +156503 POINT(40.95283122247279 -87.4895910313275) bank156503 +156504 POINT(42.28581836447783 -88.41519123103126) bank156504 +156505 POINT(42.35364327158513 -88.15220368531075) bank156505 +156506 POINT(41.816554126458165 -88.13253609170809) bank156506 +156507 POINT(42.62817387891122 -87.48638635223773) bank156507 +156508 POINT(42.2493669527034 -87.71854657002024) bank156508 +156509 POINT(41.679879833001905 -87.29651631947345) bank156509 +156510 POINT(41.43238293210665 -87.65979451842695) bank156510 +156511 POINT(42.683854649269676 -87.78500796887404) bank156511 +156512 POINT(40.88765455340055 -87.4443494139705) bank156512 +156513 POINT(41.58515468504944 -86.83029733416552) bank156513 +156514 POINT(40.98823320101653 -88.36073611257312) bank156514 +156515 POINT(41.02946597712156 -88.32353000254433) bank156515 +156516 POINT(42.15639885209463 -87.67808455486119) bank156516 +156517 POINT(41.476266569336644 -87.73745587260949) bank156517 +156518 POINT(41.86516895616097 -87.39885085278142) bank156518 +156519 POINT(41.270889550977024 -87.70119198484602) bank156519 +156520 POINT(41.55718355719104 -86.9467388943997) bank156520 +156521 POINT(42.64138078328029 -88.45006102723903) bank156521 +156522 POINT(41.55264834686976 -87.61973832377149) bank156522 +156523 POINT(41.861934803808836 -88.06653653738823) bank156523 +156524 POINT(41.383905753153684 -86.73525798684148) bank156524 +156525 POINT(41.54751482963817 -86.99928099633271) bank156525 +156526 POINT(40.93977449300006 -88.50855437278078) bank156526 +156527 POINT(41.320783390020694 -86.63986111809913) bank156527 +156528 POINT(40.89418146272216 -87.64793110310546) bank156528 +156529 POINT(42.71761263548122 -87.93201884517991) bank156529 +156530 POINT(42.63900857445714 -87.16707183954541) bank156530 +156531 POINT(41.88907697107276 -87.97730088365033) bank156531 +156532 POINT(41.4942556844108 -87.59988555243487) bank156532 +156533 POINT(41.381047889448624 -87.9659805224021) bank156533 +156534 POINT(42.340699077481375 -87.6759378348008) bank156534 +156535 POINT(41.92051953296389 -86.86543818585695) bank156535 +156536 POINT(42.064057759354455 -87.3653939480154) bank156536 +156537 POINT(41.80941725517882 -88.2566316265619) bank156537 +156538 POINT(40.91781056869982 -88.44151447964809) bank156538 +156539 POINT(41.29959585720655 -88.37292648760796) bank156539 +156540 POINT(41.79915906387038 -88.22240298204231) bank156540 +156541 POINT(42.81752808874064 -87.56325119460543) bank156541 +156542 POINT(41.52971328466315 -87.13844870092124) bank156542 +156543 POINT(42.602102272715705 -88.24079171760151) bank156543 +156544 POINT(41.61966118077875 -86.65906879821439) bank156544 +156545 POINT(41.30801369610613 -87.9091393799552) bank156545 +156546 POINT(41.47880008021062 -87.34914277701809) bank156546 +156547 POINT(41.09365864470642 -88.50541949082958) bank156547 +156548 POINT(41.92617693018362 -88.09514216413717) bank156548 +156549 POINT(42.78283620764041 -87.98633120008172) bank156549 +156550 POINT(41.68417878806233 -88.50577943721784) bank156550 +156551 POINT(41.284283646346786 -87.48717456730522) bank156551 +156552 POINT(42.17733717062705 -87.43855005096947) bank156552 +156553 POINT(42.60482422264163 -87.71752211721827) bank156553 +156554 POINT(42.831838142412195 -87.0533813068345) bank156554 +156555 POINT(42.82268923163878 -86.6571942812882) bank156555 +156556 POINT(41.97010876310739 -87.30537144369187) bank156556 +156557 POINT(42.038916736642804 -88.30131687747706) bank156557 +156558 POINT(41.80535868141811 -87.00457910189607) bank156558 +156559 POINT(41.703282757179025 -87.37380866692313) bank156559 +156560 POINT(42.534771833658326 -88.52634550892641) bank156560 +156561 POINT(41.25836752562606 -87.94213354351822) bank156561 +156562 POINT(42.107014019506494 -87.89076966214404) bank156562 +156563 POINT(41.50353729275617 -87.55127996783294) bank156563 +156564 POINT(41.65098581646483 -86.81447958307442) bank156564 +156565 POINT(42.706920980466585 -88.33195810586605) bank156565 +156566 POINT(41.62783442396659 -87.36275685959048) bank156566 +156567 POINT(41.85340749131296 -87.23229610821308) bank156567 +156568 POINT(41.14180502172212 -87.26031120499283) bank156568 +156569 POINT(42.06127681094854 -87.28157409603827) bank156569 +156570 POINT(40.96450766119162 -87.8515183480042) bank156570 +156571 POINT(42.12620825339814 -87.37310671175062) bank156571 +156572 POINT(41.649525545949544 -87.07789267420164) bank156572 +156573 POINT(40.93680061197291 -87.06663776716687) bank156573 +156574 POINT(41.11498014564168 -88.18426139789423) bank156574 +156575 POINT(42.710271961863576 -87.52853196801841) bank156575 +156576 POINT(41.10908790441628 -87.79585793352084) bank156576 +156577 POINT(41.08837960469127 -88.31957614619633) bank156577 +156578 POINT(42.301457410588704 -87.34138054380145) bank156578 +156579 POINT(42.76765830273679 -87.816963727597) bank156579 +156580 POINT(42.684813197564964 -87.1551062287825) bank156580 +156581 POINT(42.46885415499871 -87.0867470682143) bank156581 +156582 POINT(42.791293421841615 -88.26011787192971) bank156582 +156583 POINT(42.536565105821424 -86.68341772838225) bank156583 +156584 POINT(41.85472394257368 -87.93808059241987) bank156584 +156585 POINT(40.906327882652434 -87.38852741553984) bank156585 +156586 POINT(41.878216596581325 -86.76489340248234) bank156586 +156587 POINT(40.95599504451114 -86.64197185577258) bank156587 +156588 POINT(42.49235409774517 -87.62448247596461) bank156588 +156589 POINT(41.90641059743969 -87.05830244719284) bank156589 +156590 POINT(41.01103228780265 -87.77006074307657) bank156590 +156591 POINT(42.794515229500924 -88.45308032390399) bank156591 +156592 POINT(41.54700415608346 -88.46312711389633) bank156592 +156593 POINT(41.37992185895893 -88.31493993159847) bank156593 +156594 POINT(40.92215797788438 -86.82962334389278) bank156594 +156595 POINT(42.41716327680979 -87.51172725190288) bank156595 +156596 POINT(42.616265873899394 -86.66659008516268) bank156596 +156597 POINT(41.18989788901291 -88.15571939267238) bank156597 +156598 POINT(42.018802050165164 -87.35295112721575) bank156598 +156599 POINT(41.25364054746136 -86.65507467114453) bank156599 +156600 POINT(41.60056377593659 -88.00274666389812) bank156600 +156601 POINT(42.514726871565735 -88.21530316591274) bank156601 +156602 POINT(42.13100491070687 -86.97914662544389) bank156602 +156603 POINT(42.29543119657418 -87.37612135796968) bank156603 +156604 POINT(42.20463606654345 -87.3850446259152) bank156604 +156605 POINT(42.31228937181064 -88.39214897472161) bank156605 +156606 POINT(42.146942270685756 -87.29831502057418) bank156606 +156607 POINT(40.89520087533066 -88.46205798192119) bank156607 +156608 POINT(42.61929771374202 -87.6312352775225) bank156608 +156609 POINT(41.782999707321466 -88.34811085494233) bank156609 +156610 POINT(41.49048854662282 -86.85066678079801) bank156610 +156611 POINT(42.28343571492122 -86.91857115350636) bank156611 +156612 POINT(42.450785922217825 -87.0888595675768) bank156612 +156613 POINT(42.5742129443571 -87.65799155571786) bank156613 +156614 POINT(41.43579690297112 -87.96331943338667) bank156614 +156615 POINT(42.264909061693096 -88.58599698681218) bank156615 +156616 POINT(42.34648446153236 -87.65868138862372) bank156616 +156617 POINT(42.16657800975854 -86.79850500829275) bank156617 +156618 POINT(42.08897373685132 -87.29828467892185) bank156618 +156619 POINT(42.188562157323815 -88.43192232251438) bank156619 +156620 POINT(41.77577762549645 -87.84833978736268) bank156620 +156621 POINT(42.17735479351199 -86.941457403508) bank156621 +156622 POINT(41.82181374228871 -86.72884177355873) bank156622 +156623 POINT(41.643034028154744 -87.7097457346564) bank156623 +156624 POINT(42.083942555205944 -87.33404758053632) bank156624 +156625 POINT(41.67363563081197 -86.8623899164019) bank156625 +156626 POINT(42.20981681895003 -87.42286173637557) bank156626 +156627 POINT(41.879498494708926 -88.3840110374275) bank156627 +156628 POINT(41.71841691331255 -88.11198698358278) bank156628 +156629 POINT(42.501284248049316 -88.28712624978301) bank156629 +156630 POINT(42.42757603618862 -88.1695913790334) bank156630 +156631 POINT(42.706072236544266 -87.9964819821698) bank156631 +156632 POINT(42.1652030521138 -88.55229591599488) bank156632 +156633 POINT(40.976573132864985 -86.84945281208375) bank156633 +156634 POINT(41.57427236814889 -87.53875576229453) bank156634 +156635 POINT(42.43798416030605 -88.5680499706527) bank156635 +156636 POINT(42.84265328118391 -87.58078984028664) bank156636 +156637 POINT(42.72470063195699 -87.23017215784255) bank156637 +156638 POINT(41.121829048764646 -86.75296037656429) bank156638 +156639 POINT(42.251756207542165 -88.56826446092315) bank156639 +156640 POINT(41.116728734551856 -86.68438290763191) bank156640 +156641 POINT(42.413829780559894 -88.36128240800637) bank156641 +156642 POINT(42.238610248918015 -88.076103876093) bank156642 +156643 POINT(40.93385655851365 -88.13881126232104) bank156643 +156644 POINT(41.68871917264687 -87.93894293641105) bank156644 +156645 POINT(41.647079716876135 -87.68039762633965) bank156645 +156646 POINT(41.98490613405971 -88.31153074438726) bank156646 +156647 POINT(41.30230167537047 -87.54569605718153) bank156647 +156648 POINT(41.085386158626825 -87.34910732682529) bank156648 +156649 POINT(41.543910813979636 -88.45873387800243) bank156649 +156650 POINT(41.84065081875285 -88.52018462726305) bank156650 +156651 POINT(41.3685708664684 -87.355527116206) bank156651 +156652 POINT(41.02076587562866 -87.94901022186066) bank156652 +156653 POINT(41.1068459939217 -88.42054267174785) bank156653 +156654 POINT(42.234111359422556 -87.58038941675666) bank156654 +156655 POINT(40.993518755901434 -87.52174913602413) bank156655 +156656 POINT(42.02412749285215 -87.2865954955172) bank156656 +156657 POINT(41.94200015515604 -87.70177582081979) bank156657 +156658 POINT(41.81144366377836 -87.800174347242) bank156658 +156659 POINT(41.2804146054618 -87.47139053925508) bank156659 +156660 POINT(41.44211889132859 -88.39596406007955) bank156660 +156661 POINT(42.62754190655781 -87.18952568869895) bank156661 +156662 POINT(41.902797998041 -87.07013717813359) bank156662 +156663 POINT(41.42553390268715 -86.9602598190671) bank156663 +156664 POINT(41.75710970628121 -87.09675740524399) bank156664 +156665 POINT(42.52631815424609 -87.30320222729078) bank156665 +156666 POINT(42.648881319729604 -87.05885043300854) bank156666 +156667 POINT(40.946469595069644 -86.73389162323139) bank156667 +156668 POINT(42.53070988875942 -88.30535598113035) bank156668 +156669 POINT(42.36854186014449 -87.20682420192824) bank156669 +156670 POINT(42.32770572646441 -87.40080156780938) bank156670 +156671 POINT(41.25281514853762 -87.47465455830503) bank156671 +156672 POINT(41.026099382808226 -87.26326635444468) bank156672 +156673 POINT(41.723632017073946 -86.78345114838196) bank156673 +156674 POINT(41.405562194816326 -87.34829775553253) bank156674 +156675 POINT(41.97954008929778 -87.89456944761322) bank156675 +156676 POINT(41.78379819613453 -88.12901741495317) bank156676 +156677 POINT(41.489782532644455 -88.25069121536401) bank156677 +156678 POINT(40.88580161400075 -87.95762293152609) bank156678 +156679 POINT(42.01968960224225 -87.02339128551411) bank156679 +156680 POINT(41.471801999139686 -87.31453410854063) bank156680 +156681 POINT(41.57890360082249 -86.65474647783641) bank156681 +156682 POINT(42.15592803770732 -87.2523960547242) bank156682 +156683 POINT(42.72636050224568 -88.43630182161482) bank156683 +156684 POINT(41.74111242560422 -87.1392745328474) bank156684 +156685 POINT(42.7893198021267 -88.4365897007853) bank156685 +156686 POINT(41.876359294269555 -88.53295822029119) bank156686 +156687 POINT(42.47667488272605 -88.11297213437679) bank156687 +156688 POINT(40.8942586671433 -87.1038833529469) bank156688 +156689 POINT(42.817251294111216 -87.74569217448546) bank156689 +156690 POINT(41.59938981310949 -87.73422128318649) bank156690 +156691 POINT(42.46207324545247 -88.6257652627582) bank156691 +156692 POINT(42.114367386685885 -87.33080541718151) bank156692 +156693 POINT(42.84819158893622 -88.52557177578701) bank156693 +156694 POINT(42.66180309457351 -86.74686257659177) bank156694 +156695 POINT(40.93774692541407 -86.98011370338507) bank156695 +156696 POINT(42.29182326378187 -87.05511642488337) bank156696 +156697 POINT(40.984999845973334 -86.84975243694436) bank156697 +156698 POINT(41.57001566522893 -87.52438268862537) bank156698 +156699 POINT(42.241999137582056 -88.51218699859854) bank156699 +156700 POINT(42.230155897279104 -87.81250171752599) bank156700 +156701 POINT(41.22736143673 -87.21885265233217) bank156701 +156702 POINT(41.261498424657766 -87.53964847919444) bank156702 +156703 POINT(42.17300297129654 -88.39085748128609) bank156703 +156704 POINT(41.84483825672604 -87.92620621433761) bank156704 +156705 POINT(41.14581288435308 -86.96493144753718) bank156705 +156706 POINT(42.83848678082792 -87.63679437678202) bank156706 +156707 POINT(42.49643738038306 -87.55457025748831) bank156707 +156708 POINT(42.73157913649043 -86.65041457743528) bank156708 +156709 POINT(42.79288463731233 -87.4447412883392) bank156709 +156710 POINT(41.35187104524388 -87.35647464635672) bank156710 +156711 POINT(41.59276247639335 -88.41253728465685) bank156711 +156712 POINT(41.95429027490938 -87.31181426664284) bank156712 +156713 POINT(42.00097952602123 -88.31669056595555) bank156713 +156714 POINT(41.18083893058363 -87.69183828927348) bank156714 +156715 POINT(41.391678350271974 -88.4600812082906) bank156715 +156716 POINT(42.1082604974025 -87.49136627232804) bank156716 +156717 POINT(42.03511516887729 -87.9416668362777) bank156717 +156718 POINT(41.322678657312935 -87.16971657956807) bank156718 +156719 POINT(42.58527427498973 -86.86076585914498) bank156719 +156720 POINT(41.528754728894505 -87.32712274846497) bank156720 +156721 POINT(41.93919909653975 -88.18000463053733) bank156721 +156722 POINT(42.50486085564568 -87.35184333232435) bank156722 +156723 POINT(41.96287482132969 -88.47326748486046) bank156723 +156724 POINT(41.16729381354396 -87.00003291151266) bank156724 +156725 POINT(41.8750487270169 -86.63798924932715) bank156725 +156726 POINT(42.036257670859044 -87.39041228741512) bank156726 +156727 POINT(41.71473067115167 -87.9385028059696) bank156727 +156728 POINT(41.65027065278657 -87.06667201144441) bank156728 +156729 POINT(42.54260854626785 -87.77207117121867) bank156729 +156730 POINT(42.39310849340721 -87.81656851103062) bank156730 +156731 POINT(41.675217830196786 -86.67419280268912) bank156731 +156732 POINT(41.11043336005232 -87.95285008209588) bank156732 +156733 POINT(42.49216103131464 -87.04865989135773) bank156733 +156734 POINT(42.08700782422435 -88.16258363365678) bank156734 +156735 POINT(41.12070844643556 -88.36231451553776) bank156735 +156736 POINT(42.13819064516604 -87.2103098122257) bank156736 +156737 POINT(41.649617817145035 -87.78861146724522) bank156737 +156738 POINT(41.53159472657474 -88.38154728430436) bank156738 +156739 POINT(41.045614503756546 -87.38413279819734) bank156739 +156740 POINT(42.56264244339606 -87.5477258087044) bank156740 +156741 POINT(40.886569306154044 -88.03644130953742) bank156741 +156742 POINT(42.06091053292052 -87.62882376304908) bank156742 +156743 POINT(41.477737009768475 -87.80320210702202) bank156743 +156744 POINT(41.024462931744615 -87.20217077083103) bank156744 +156745 POINT(41.51208338721034 -87.30764229999346) bank156745 +156746 POINT(41.658789543248744 -86.69609007429588) bank156746 +156747 POINT(41.023071346714225 -86.7778945848472) bank156747 +156748 POINT(41.96347871055295 -88.39380439932177) bank156748 +156749 POINT(42.23489101162971 -87.29762487923843) bank156749 +156750 POINT(42.63374475107313 -88.47523579000735) bank156750 +156751 POINT(40.955257445181616 -88.16052646547026) bank156751 +156752 POINT(41.78282261246424 -87.62641654059176) bank156752 +156753 POINT(42.00880185912941 -88.4227057057821) bank156753 +156754 POINT(42.1810975302799 -88.18971233919359) bank156754 +156755 POINT(42.00525479444043 -86.83554401062322) bank156755 +156756 POINT(42.495230319740564 -86.92919025035918) bank156756 +156757 POINT(41.75317339525341 -87.40472785728286) bank156757 +156758 POINT(41.154626929604085 -88.38826211439957) bank156758 +156759 POINT(42.364751315318465 -88.46820883295032) bank156759 +156760 POINT(42.4306317765953 -88.0734278209654) bank156760 +156761 POINT(41.47955075698028 -86.74779643453148) bank156761 +156762 POINT(42.33554246958572 -87.75630094286099) bank156762 +156763 POINT(42.19300689074331 -87.52402231292008) bank156763 +156764 POINT(41.80819339999183 -88.60729175759232) bank156764 +156765 POINT(41.817240698752826 -87.39442518687895) bank156765 +156766 POINT(41.42127803135149 -87.08082945808853) bank156766 +156767 POINT(42.573381339506454 -86.7486282737956) bank156767 +156768 POINT(42.59370442432715 -86.64627623670884) bank156768 +156769 POINT(41.16730796647877 -88.01990099074895) bank156769 +156770 POINT(40.90594519693362 -87.97866107462555) bank156770 +156771 POINT(41.033958121324 -88.50236964188181) bank156771 +156772 POINT(41.741343454028744 -87.5546404055515) bank156772 +156773 POINT(41.27762586621157 -88.02815865673412) bank156773 +156774 POINT(41.808825269052846 -88.21363600446786) bank156774 +156775 POINT(42.66295209688371 -87.43761430265636) bank156775 +156776 POINT(42.671565510101395 -87.80510644068558) bank156776 +156777 POINT(42.31455460894964 -88.35364150360384) bank156777 +156778 POINT(42.81173552783194 -87.75037719975376) bank156778 +156779 POINT(42.625438669434914 -87.07240132839814) bank156779 +156780 POINT(42.36072566870443 -87.8395746803154) bank156780 +156781 POINT(42.72168767931985 -87.01337347101938) bank156781 +156782 POINT(42.62535263764671 -87.70889630460955) bank156782 +156783 POINT(40.92123969342512 -88.29195638271798) bank156783 +156784 POINT(42.47031996342574 -88.21540602096955) bank156784 +156785 POINT(41.9688908211378 -87.85126874759095) bank156785 +156786 POINT(41.99449967595136 -87.15150833668159) bank156786 +156787 POINT(41.765909241545586 -86.69023935689295) bank156787 +156788 POINT(42.1324003890094 -87.43099633653459) bank156788 +156789 POINT(41.972369250698215 -87.36745765840519) bank156789 +156790 POINT(41.9628295412835 -87.14681226519755) bank156790 +156791 POINT(41.916719815324306 -88.42515868137524) bank156791 +156792 POINT(41.662782948772765 -88.29200774764777) bank156792 +156793 POINT(41.81407951025637 -87.66259461008141) bank156793 +156794 POINT(42.605404385005194 -86.6430112453804) bank156794 +156795 POINT(41.850816128191376 -88.16923967175249) bank156795 +156796 POINT(42.1418361090556 -87.60286782854493) bank156796 +156797 POINT(40.90926890514825 -87.52174210661053) bank156797 +156798 POINT(42.64860377059802 -88.3823346750338) bank156798 +156799 POINT(42.195054265647784 -87.47689595453535) bank156799 +156800 POINT(41.550494712042756 -88.33808058456763) bank156800 +156801 POINT(42.83735203636203 -86.905505165817) bank156801 +156802 POINT(42.74196011124068 -87.12654437037236) bank156802 +156803 POINT(42.571636106641954 -87.83764402386025) bank156803 +156804 POINT(41.20996138155134 -88.50626820946997) bank156804 +156805 POINT(40.93266848218323 -88.1181474552972) bank156805 +156806 POINT(42.744570444239166 -88.60711082411021) bank156806 +156807 POINT(40.9996970039684 -87.59685281593129) bank156807 +156808 POINT(41.637585032326086 -88.5556412526366) bank156808 +156809 POINT(42.47749216622502 -86.98792985171131) bank156809 +156810 POINT(42.69900870348412 -88.31989529556861) bank156810 +156811 POINT(40.92892989817903 -87.95946219357438) bank156811 +156812 POINT(41.28024008873997 -88.12835720203358) bank156812 +156813 POINT(42.53495508786623 -87.32888134340996) bank156813 +156814 POINT(42.246993463148776 -88.08575801659484) bank156814 +156815 POINT(41.74539032396527 -87.95788985705877) bank156815 +156816 POINT(41.57646406693898 -87.51097003748927) bank156816 +156817 POINT(41.78518271132245 -86.91981123763) bank156817 +156818 POINT(41.09266329322255 -87.15606653669018) bank156818 +156819 POINT(42.33354357566295 -88.08484055616579) bank156819 +156820 POINT(41.48850887758434 -86.82297371066261) bank156820 +156821 POINT(42.64118150156448 -87.75597906141381) bank156821 +156822 POINT(42.79604145308804 -87.8627116680193) bank156822 +156823 POINT(41.53353079503909 -87.21718931991337) bank156823 +156824 POINT(41.09919510394144 -88.27253075501142) bank156824 +156825 POINT(42.29553118126881 -88.37730912112985) bank156825 +156826 POINT(42.3296604097466 -87.12294396496756) bank156826 +156827 POINT(41.67743500998085 -87.55459441221988) bank156827 +156828 POINT(41.74784705823854 -86.78559340282224) bank156828 +156829 POINT(42.05231446946172 -87.93839362850014) bank156829 +156830 POINT(41.36975334700448 -87.25266435493278) bank156830 +156831 POINT(41.8648356487868 -87.71998840161696) bank156831 +156832 POINT(42.26965447617568 -86.83566683606416) bank156832 +156833 POINT(42.22280695406254 -88.43865032656919) bank156833 +156834 POINT(41.251467039758495 -87.82865627818057) bank156834 +156835 POINT(40.950058262477675 -87.78091980608177) bank156835 +156836 POINT(41.40370209507469 -86.81340274412416) bank156836 +156837 POINT(42.01781863849028 -88.58973111707927) bank156837 +156838 POINT(42.504817207957714 -87.12824012336955) bank156838 +156839 POINT(41.88101320835154 -88.35187589838955) bank156839 +156840 POINT(41.16557692878105 -88.48743014459839) bank156840 +156841 POINT(41.61105715323454 -86.99201916254471) bank156841 +156842 POINT(41.050578702639775 -88.01285452806839) bank156842 +156843 POINT(42.74456196017983 -87.11292948886111) bank156843 +156844 POINT(41.55901439500246 -87.36317557420752) bank156844 +156845 POINT(42.46187960774216 -86.67952191591822) bank156845 +156846 POINT(42.3761693200667 -88.12362291487233) bank156846 +156847 POINT(42.63524681377253 -87.7036657486632) bank156847 +156848 POINT(41.6282444103203 -88.2264252224607) bank156848 +156849 POINT(41.03181165488152 -86.69692636175537) bank156849 +156850 POINT(42.739906411760856 -87.27888322251482) bank156850 +156851 POINT(42.72168667459951 -86.77905062600905) bank156851 +156852 POINT(40.97064530745495 -88.47352677000288) bank156852 +156853 POINT(42.33184696581176 -87.57909973729532) bank156853 +156854 POINT(42.421169664522 -87.6518556466831) bank156854 +156855 POINT(41.567590863323545 -88.36692243948956) bank156855 +156856 POINT(42.490255196211976 -87.48907499541646) bank156856 +156857 POINT(41.05582731510317 -87.9885359896269) bank156857 +156858 POINT(42.744118046303505 -87.13899523333012) bank156858 +156859 POINT(42.2581135430487 -87.8066646918395) bank156859 +156860 POINT(42.32698275509901 -86.79080023583849) bank156860 +156861 POINT(42.64716339553153 -87.02654868565047) bank156861 +156862 POINT(42.45415004399668 -88.31450536634902) bank156862 +156863 POINT(42.6930263607538 -87.30169234033187) bank156863 +156864 POINT(41.576996054148765 -87.36352944814377) bank156864 +156865 POINT(42.3773119979817 -87.98643860652456) bank156865 +156866 POINT(41.45674060648155 -87.88178720948429) bank156866 +156867 POINT(42.17349245055753 -87.96640707075365) bank156867 +156868 POINT(42.502125420449886 -87.54696294360949) bank156868 +156869 POINT(41.27161515126203 -88.59803716289838) bank156869 +156870 POINT(41.92853295863187 -88.14228615151963) bank156870 +156871 POINT(41.160344719607465 -88.34448249990135) bank156871 +156872 POINT(41.044721940011335 -87.11008028990464) bank156872 +156873 POINT(41.90064106135134 -87.83694868758964) bank156873 +156874 POINT(42.05972512608629 -88.15350897856393) bank156874 +156875 POINT(42.306112181654626 -86.89550654629991) bank156875 +156876 POINT(40.95868249928614 -87.03136623136619) bank156876 +156877 POINT(42.324135054461415 -88.32968830145482) bank156877 +156878 POINT(42.68918799783484 -87.33136513086293) bank156878 +156879 POINT(41.901836910536325 -87.51221625126847) bank156879 +156880 POINT(41.54670564449278 -87.490704520512) bank156880 +156881 POINT(41.08926493299724 -87.55311311127063) bank156881 +156882 POINT(41.728391413013206 -86.71821075064398) bank156882 +156883 POINT(41.70334103314927 -87.20572604068) bank156883 +156884 POINT(42.4139201577709 -87.5030407641765) bank156884 +156885 POINT(41.86534517981951 -87.23501081434814) bank156885 +156886 POINT(41.50470751429189 -87.14120721878994) bank156886 +156887 POINT(41.90025440317249 -88.38085749749872) bank156887 +156888 POINT(41.737505136567684 -87.57045590234767) bank156888 +156889 POINT(41.89132956839555 -87.44785537472073) bank156889 +156890 POINT(42.16360974068014 -87.06572266991495) bank156890 +156891 POINT(41.4577016439996 -87.81322930497659) bank156891 +156892 POINT(42.62691412970275 -88.31627758812894) bank156892 +156893 POINT(42.43621905339863 -87.88599630425516) bank156893 +156894 POINT(42.44652108837148 -87.79486958642849) bank156894 +156895 POINT(41.22474871803324 -87.16697991906419) bank156895 +156896 POINT(42.63833977820244 -88.43471735622974) bank156896 +156897 POINT(42.18968657549167 -86.64210813339307) bank156897 +156898 POINT(42.83754174488289 -87.4131307107391) bank156898 +156899 POINT(42.4147833391316 -88.24639619380281) bank156899 +156900 POINT(41.095433091748845 -87.70267277796579) bank156900 +156901 POINT(41.525113634813096 -88.57472446887958) bank156901 +156902 POINT(40.97895916199704 -86.94834033472745) bank156902 +156903 POINT(42.59048269369764 -87.88956601303369) bank156903 +156904 POINT(41.037457900149356 -87.54154053038525) bank156904 +156905 POINT(41.410325413880166 -88.0919654303346) bank156905 +156906 POINT(41.42594760318032 -87.4238135797655) bank156906 +156907 POINT(42.086270303164156 -86.64558443619309) bank156907 +156908 POINT(42.62791855903939 -87.37551232179779) bank156908 +156909 POINT(41.83915387696551 -87.36176733426993) bank156909 +156910 POINT(42.05134505739564 -87.33154801901652) bank156910 +156911 POINT(42.111818147267364 -87.51216379515493) bank156911 +156912 POINT(41.21824648792659 -87.2340783460899) bank156912 +156913 POINT(41.38235541887305 -86.74074063519035) bank156913 +156914 POINT(41.5814058830489 -87.48700728381627) bank156914 +156915 POINT(42.81960892350475 -87.10176931343378) bank156915 +156916 POINT(41.60257421034309 -87.99667290060887) bank156916 +156917 POINT(42.822038413074374 -86.65807367859247) bank156917 +156918 POINT(41.604478554537 -86.89416843913817) bank156918 +156919 POINT(41.25853615207005 -87.34690423555823) bank156919 +156920 POINT(42.71991617626257 -87.36137600780314) bank156920 +156921 POINT(41.63888783914758 -87.92723226720037) bank156921 +156922 POINT(42.51373023134143 -88.00647467870289) bank156922 +156923 POINT(42.16926504358631 -88.08441144152064) bank156923 +156924 POINT(42.64001433088506 -87.83856829819166) bank156924 +156925 POINT(42.271579568829026 -88.54671882866181) bank156925 +156926 POINT(41.800312526110844 -88.62579248006344) bank156926 +156927 POINT(41.882258883951906 -87.26807622701189) bank156927 +156928 POINT(42.86163335636622 -87.47410610560699) bank156928 +156929 POINT(41.3332718144008 -87.52288531848511) bank156929 +156930 POINT(42.62063416790925 -87.60788231131174) bank156930 +156931 POINT(42.028367850014156 -88.26020898249547) bank156931 +156932 POINT(42.002654928827916 -88.18594065124107) bank156932 +156933 POINT(41.67302190931419 -87.04192565701209) bank156933 +156934 POINT(42.81900948440569 -88.35945788906436) bank156934 +156935 POINT(42.81419376121983 -87.10443028393495) bank156935 +156936 POINT(41.26263293377087 -86.7265236773755) bank156936 +156937 POINT(42.65078046602948 -86.64463796240888) bank156937 +156938 POINT(41.864102165590296 -87.5488104006265) bank156938 +156939 POINT(42.63658190962089 -86.87377213027492) bank156939 +156940 POINT(42.33369822010429 -86.6434487856197) bank156940 +156941 POINT(42.433066506459674 -88.14783456294529) bank156941 +156942 POINT(40.958766823981385 -87.04255637082161) bank156942 +156943 POINT(40.95527082093167 -87.77444428698368) bank156943 +156944 POINT(41.5035091247771 -87.15216674752809) bank156944 +156945 POINT(41.04549308215645 -88.2354485243292) bank156945 +156946 POINT(42.410605618000666 -88.42708873365811) bank156946 +156947 POINT(42.79927861881436 -87.82088890946164) bank156947 +156948 POINT(41.949217806044004 -88.39326766001142) bank156948 +156949 POINT(42.599425597223735 -87.86586656945902) bank156949 +156950 POINT(42.045625614913256 -87.07359520003584) bank156950 +156951 POINT(40.89971296129177 -86.9845249915107) bank156951 +156952 POINT(41.79847703001243 -87.03106621263414) bank156952 +156953 POINT(42.516940909114375 -88.44229835482068) bank156953 +156954 POINT(42.808763773675444 -87.07791341768187) bank156954 +156955 POINT(41.524563996633226 -86.96152637873453) bank156955 +156956 POINT(42.08706682708117 -87.30767892256921) bank156956 +156957 POINT(42.305453745798715 -87.9568184473679) bank156957 +156958 POINT(41.83443795916116 -87.47266900958344) bank156958 +156959 POINT(41.28500245924257 -88.1292793729304) bank156959 +156960 POINT(41.663061333990136 -87.17180977851855) bank156960 +156961 POINT(42.52309630106641 -87.75665301179363) bank156961 +156962 POINT(42.81654163712871 -87.25636790518075) bank156962 +156963 POINT(41.735598870621644 -86.88920403654824) bank156963 +156964 POINT(42.80840823719957 -87.88998546437588) bank156964 +156965 POINT(41.52350310417084 -87.99526644015887) bank156965 +156966 POINT(42.1783082601232 -87.03609734631709) bank156966 +156967 POINT(42.331694645802976 -87.74764406483627) bank156967 +156968 POINT(41.17615866429807 -88.39230408919687) bank156968 +156969 POINT(42.05252322579842 -87.50212285236198) bank156969 +156970 POINT(41.326661542677606 -88.42392703458724) bank156970 +156971 POINT(41.72495710669443 -88.17396894312944) bank156971 +156972 POINT(42.01118504382082 -87.95356642546228) bank156972 +156973 POINT(42.503729737778514 -87.44998648127424) bank156973 +156974 POINT(41.783981361565736 -86.9447189971758) bank156974 +156975 POINT(41.41359802917645 -86.93804712467013) bank156975 +156976 POINT(42.018094005032125 -86.84998433546767) bank156976 +156977 POINT(42.847300480270256 -87.9531629358144) bank156977 +156978 POINT(42.573850634590684 -87.58017928188434) bank156978 +156979 POINT(41.058959344322 -87.91399950918081) bank156979 +156980 POINT(40.894326862719616 -87.80714635287843) bank156980 +156981 POINT(41.92179229476167 -88.14376151097682) bank156981 +156982 POINT(41.636358940148796 -87.04304193257929) bank156982 +156983 POINT(41.60984840712692 -87.7236084994619) bank156983 +156984 POINT(40.99216504973318 -87.16494229575203) bank156984 +156985 POINT(42.56167040166431 -87.46201394921682) bank156985 +156986 POINT(42.17755903193717 -87.76304181021602) bank156986 +156987 POINT(42.1185951471224 -88.1227510978199) bank156987 +156988 POINT(41.28353171445799 -88.02736687093058) bank156988 +156989 POINT(41.188251496347306 -87.48230150696472) bank156989 +156990 POINT(41.195128708964475 -87.13874151440618) bank156990 +156991 POINT(41.28674510700552 -87.30637533296691) bank156991 +156992 POINT(41.00082063180255 -88.5897421490153) bank156992 +156993 POINT(40.996969915249544 -87.29302220924261) bank156993 +156994 POINT(42.330723509412856 -88.41902088440082) bank156994 +156995 POINT(41.93885583825765 -87.44637016912476) bank156995 +156996 POINT(42.20260183740053 -88.40170382536544) bank156996 +156997 POINT(41.98030451396802 -88.23188830113597) bank156997 +156998 POINT(42.726806152965196 -87.25615948239967) bank156998 +156999 POINT(42.21772743156837 -87.72356897187136) bank156999 +157000 POINT(42.373449938605525 -87.87580782536526) bank157000 +157001 POINT(42.1214208596064 -86.97360040448375) bank157001 +157002 POINT(40.99905354455262 -87.53875241697574) bank157002 +157003 POINT(42.84383272633501 -87.33248783510183) bank157003 +157004 POINT(41.71925653245692 -87.97278553563868) bank157004 +157005 POINT(40.944619516797054 -88.44025200223827) bank157005 +157006 POINT(41.81081854457608 -87.95328101841348) bank157006 +157007 POINT(41.253300122930646 -86.66832134190732) bank157007 +157008 POINT(42.82523907025623 -87.72451764069771) bank157008 +157009 POINT(41.04100002542034 -87.36783120222111) bank157009 +157010 POINT(41.283470293402885 -88.45434287356296) bank157010 +157011 POINT(42.382849161955896 -88.29101478961779) bank157011 +157012 POINT(41.378866970760264 -86.69504849430642) bank157012 +157013 POINT(42.24975606437522 -87.1723372252865) bank157013 +157014 POINT(41.210659175188724 -88.55574005787267) bank157014 +157015 POINT(42.79132867119686 -86.69998734453517) bank157015 +157016 POINT(41.971890834874664 -88.27569088016794) bank157016 +157017 POINT(42.833820451223545 -87.0033122523733) bank157017 +157018 POINT(41.95617787815046 -88.06865698802714) bank157018 +157019 POINT(41.129017399249825 -87.07375889912866) bank157019 +157020 POINT(42.539055630955374 -88.36401294202909) bank157020 +157021 POINT(42.343722490794185 -87.19224001725176) bank157021 +157022 POINT(42.21335790816709 -87.91390740660736) bank157022 +157023 POINT(42.47570082076565 -88.40652846147339) bank157023 +157024 POINT(41.56349461090264 -88.13073270113404) bank157024 +157025 POINT(42.44797932201602 -87.11415662599639) bank157025 +157026 POINT(41.57157923361131 -87.10881252270838) bank157026 +157027 POINT(41.20297063345091 -87.94328808405476) bank157027 +157028 POINT(41.13107940029268 -88.36066584857667) bank157028 +157029 POINT(41.27605951111962 -88.14242795465027) bank157029 +157030 POINT(41.58913760728422 -87.66714158071784) bank157030 +157031 POINT(42.44109419851915 -86.73594178851538) bank157031 +157032 POINT(42.78945859517288 -86.69853726258052) bank157032 +157033 POINT(41.40465706959883 -86.6469959815523) bank157033 +157034 POINT(42.301079517311265 -86.7734856712439) bank157034 +157035 POINT(41.70764933603213 -88.00063760539024) bank157035 +157036 POINT(42.14541614855049 -88.50020533040825) bank157036 +157037 POINT(42.04498095186011 -86.95941989588592) bank157037 +157038 POINT(42.222497050410716 -87.13194454219273) bank157038 +157039 POINT(42.349138974910126 -88.2916938845163) bank157039 +157040 POINT(41.86218943872679 -88.479169446911) bank157040 +157041 POINT(41.0060916566129 -86.63541225527796) bank157041 +157042 POINT(41.63088713043711 -88.16312773434055) bank157042 +157043 POINT(42.743859476751126 -87.04090164491768) bank157043 +157044 POINT(41.45898072434048 -87.21896261387784) bank157044 +157045 POINT(41.38652752272572 -87.9672446586954) bank157045 +157046 POINT(41.37647648466483 -88.62741491112794) bank157046 +157047 POINT(41.72091023627164 -87.97735716011198) bank157047 +157048 POINT(40.963928839635514 -87.7058028572514) bank157048 +157049 POINT(41.727515074889034 -88.42852858552664) bank157049 +157050 POINT(42.31540153480026 -88.09820319074491) bank157050 +157051 POINT(42.442080287287645 -86.87561946507749) bank157051 +157052 POINT(41.74812977601582 -87.47091723557348) bank157052 +157053 POINT(41.96974514782309 -87.80585033599323) bank157053 +157054 POINT(41.10571033165519 -87.51570911961154) bank157054 +157055 POINT(42.22289834281118 -87.37470434180031) bank157055 +157056 POINT(42.09284434204967 -87.82146093354322) bank157056 +157057 POINT(42.725913407950905 -87.36819760259681) bank157057 +157058 POINT(41.01609598356908 -86.74096480859247) bank157058 +157059 POINT(41.50635020571993 -87.20847307656024) bank157059 +157060 POINT(41.803984416229504 -88.18515608547109) bank157060 +157061 POINT(41.22522431592018 -87.019040908195) bank157061 +157062 POINT(41.05676078223906 -88.09739457046469) bank157062 +157063 POINT(41.16592852945683 -87.81890484056908) bank157063 +157064 POINT(41.56136992861062 -86.70561381817066) bank157064 +157065 POINT(42.3646126808133 -88.26718190589624) bank157065 +157066 POINT(42.20894620259591 -87.29207175917087) bank157066 +157067 POINT(41.64108697556092 -87.84958398188863) bank157067 +157068 POINT(42.145999169418914 -87.20672979488434) bank157068 +157069 POINT(42.64417346892876 -88.27982904680563) bank157069 +157070 POINT(41.734165944274956 -87.4744668334356) bank157070 +157071 POINT(41.41552429907871 -88.41201048430324) bank157071 +157072 POINT(42.86966456260594 -88.13365570725203) bank157072 +157073 POINT(41.87822778911437 -87.47118865201395) bank157073 +157074 POINT(42.17267380369106 -88.37926621305574) bank157074 +157075 POINT(41.29009090605998 -88.53765503955911) bank157075 +157076 POINT(41.066809030199515 -88.25353362069764) bank157076 +157077 POINT(42.87652037957974 -87.8013436882911) bank157077 +157078 POINT(42.429892637133605 -86.85660489707294) bank157078 +157079 POINT(41.25814777500197 -87.2020489422899) bank157079 +157080 POINT(41.95056879855624 -88.06173111233163) bank157080 +157081 POINT(41.117919457785405 -88.43473872888576) bank157081 +157082 POINT(41.02055414198203 -87.02212254192037) bank157082 +157083 POINT(40.909272998611826 -86.73483697003736) bank157083 +157084 POINT(41.13421736150496 -86.84318329833738) bank157084 +157085 POINT(40.9015479390182 -88.1305003211732) bank157085 +157086 POINT(41.63854933307976 -87.38869879313091) bank157086 +157087 POINT(41.05001797250427 -88.07082676958687) bank157087 +157088 POINT(40.91335375051504 -87.5249738147125) bank157088 +157089 POINT(42.44006007578244 -88.08911566210926) bank157089 +157090 POINT(42.80784327095393 -88.3496615114098) bank157090 +157091 POINT(41.9826002993987 -88.56340567647491) bank157091 +157092 POINT(41.971826450747 -88.5438084961665) bank157092 +157093 POINT(42.13719971454601 -87.58040763886386) bank157093 +157094 POINT(41.60457656792565 -87.97399516224203) bank157094 +157095 POINT(42.8234653361704 -87.17678267859992) bank157095 +157096 POINT(41.966200070168064 -86.69817143119737) bank157096 +157097 POINT(42.23730093548103 -88.34949747995935) bank157097 +157098 POINT(42.851368635030205 -87.4810185930077) bank157098 +157099 POINT(41.211733574071225 -86.76589814094342) bank157099 +157100 POINT(41.43099236548584 -88.53077220347784) bank157100 +157101 POINT(41.49373705772055 -88.51572696421783) bank157101 +157102 POINT(42.50234469827603 -87.23639257407446) bank157102 +157103 POINT(42.527834863265 -87.77914895337508) bank157103 +157104 POINT(42.354964417110224 -88.35651075849357) bank157104 +157105 POINT(40.96473439570501 -86.66503452788079) bank157105 +157106 POINT(41.49737368511029 -87.36528271643576) bank157106 +157107 POINT(42.34983035516856 -86.83356356205373) bank157107 +157108 POINT(41.158545964560915 -87.55985206690617) bank157108 +157109 POINT(42.66109099380999 -87.16136621506654) bank157109 +157110 POINT(41.584779101572146 -87.7159889111389) bank157110 +157111 POINT(41.19254598041588 -88.42359965366484) bank157111 +157112 POINT(41.67075639994723 -88.62217832258351) bank157112 +157113 POINT(42.565635187469134 -87.14652295423814) bank157113 +157114 POINT(42.283542716514035 -87.0223533118657) bank157114 +157115 POINT(41.54916319219245 -86.94336709006602) bank157115 +157116 POINT(42.33122639486548 -88.37761698831373) bank157116 +157117 POINT(42.642028548407595 -86.81452820403273) bank157117 +157118 POINT(41.95549245267913 -87.61360986796112) bank157118 +157119 POINT(42.717120747084586 -86.68244687701372) bank157119 +157120 POINT(41.20054571230697 -87.53359923441937) bank157120 +157121 POINT(42.656680172192914 -87.84562672577478) bank157121 +157122 POINT(41.232696827881625 -87.30849932938793) bank157122 +157123 POINT(42.608083487443565 -88.14435896364337) bank157123 +157124 POINT(41.98261559574052 -87.4717755546058) bank157124 +157125 POINT(41.554663894349694 -87.98579986657158) bank157125 +157126 POINT(40.99113039156415 -87.3672958030052) bank157126 +157127 POINT(42.340272333548036 -87.87943415266368) bank157127 +157128 POINT(41.1092382831812 -88.51424462055142) bank157128 +157129 POINT(41.95983369147147 -87.73467855439225) bank157129 +157130 POINT(42.6884899878293 -88.33103263537431) bank157130 +157131 POINT(41.57466860285421 -88.12285159030229) bank157131 +157132 POINT(41.3834194297672 -87.642902099391) bank157132 +157133 POINT(42.7540472585279 -87.43970914000597) bank157133 +157134 POINT(42.52026057223174 -86.67111304342976) bank157134 +157135 POINT(41.288692402489964 -87.09781515227043) bank157135 +157136 POINT(42.632375531122634 -88.29049691125871) bank157136 +157137 POINT(41.622297781168506 -87.17517821853356) bank157137 +157138 POINT(42.71700200977502 -86.8023335620343) bank157138 +157139 POINT(42.71188891543997 -86.96640169092517) bank157139 +157140 POINT(42.329732749371466 -87.61579815972138) bank157140 +157141 POINT(42.6388104969805 -87.92135060609978) bank157141 +157142 POINT(41.17808111889277 -87.49650803948828) bank157142 +157143 POINT(42.33596013486826 -87.27964993983551) bank157143 +157144 POINT(41.58331306058205 -88.22676580795763) bank157144 +157145 POINT(41.44990581420807 -86.92725526253506) bank157145 +157146 POINT(41.627009990095964 -87.54705402253994) bank157146 +157147 POINT(40.98051592407139 -88.54531575746671) bank157147 +157148 POINT(41.41865756433738 -87.36617266650424) bank157148 +157149 POINT(41.498173454972715 -87.60969048890568) bank157149 +157150 POINT(41.803759771539646 -86.82808600732243) bank157150 +157151 POINT(41.10680333610436 -87.10566379472844) bank157151 +157152 POINT(42.32622605304437 -88.01512853292769) bank157152 +157153 POINT(42.786205081767974 -87.47726662834812) bank157153 +157154 POINT(42.62378701103711 -88.02193510817258) bank157154 +157155 POINT(42.23264475717778 -87.76730258221721) bank157155 +157156 POINT(42.05373626786383 -88.60499561438975) bank157156 +157157 POINT(41.994897524107465 -88.22080475543635) bank157157 +157158 POINT(40.98188681214715 -87.65452129484127) bank157158 +157159 POINT(42.59496656874633 -86.82216527843164) bank157159 +157160 POINT(42.44191944921167 -88.04958320047658) bank157160 +157161 POINT(41.757905110029824 -88.48935591937183) bank157161 +157162 POINT(42.51859889180156 -87.98750819020087) bank157162 +157163 POINT(41.13717450841605 -86.95513211482009) bank157163 +157164 POINT(40.976256453771974 -88.5790896227759) bank157164 +157165 POINT(42.83785959833635 -87.5273588874035) bank157165 +157166 POINT(42.35844823212449 -87.85338762944998) bank157166 +157167 POINT(42.191588125011485 -87.88643742047212) bank157167 +157168 POINT(42.50294888821333 -87.59185257659267) bank157168 +157169 POINT(42.70967573553784 -88.5085186979984) bank157169 +157170 POINT(41.979187446468416 -86.90240089402693) bank157170 +157171 POINT(42.33893055470286 -87.04580628198246) bank157171 +157172 POINT(41.712264206653494 -88.26707914104956) bank157172 +157173 POINT(41.176942823928705 -88.03134475750153) bank157173 +157174 POINT(41.18704392023669 -88.05485921173015) bank157174 +157175 POINT(42.69520989245248 -88.08316546119893) bank157175 +157176 POINT(41.72991752549942 -86.7290359859085) bank157176 +157177 POINT(41.47564358783472 -88.25635635057125) bank157177 +157178 POINT(42.136126876278105 -86.7969663836838) bank157178 +157179 POINT(41.18762464944579 -87.04230715246752) bank157179 +157180 POINT(41.99395785692193 -87.3762057781949) bank157180 +157181 POINT(41.10372926874645 -88.41435983262375) bank157181 +157182 POINT(42.73875006020846 -87.76311717945828) bank157182 +157183 POINT(41.53374762135577 -87.60233048887432) bank157183 +157184 POINT(41.80720214466241 -87.36572156730112) bank157184 +157185 POINT(42.70804007510531 -87.94218795196257) bank157185 +157186 POINT(42.194995826771446 -87.5258655301008) bank157186 +157187 POINT(41.28533545796099 -86.85181107953194) bank157187 +157188 POINT(41.990406841283296 -87.67882080489608) bank157188 +157189 POINT(41.85432574515022 -86.6586075027961) bank157189 +157190 POINT(42.8075386228517 -87.3865969339486) bank157190 +157191 POINT(42.687635285328675 -88.03425811422845) bank157191 +157192 POINT(42.39370122032508 -87.65070758723593) bank157192 +157193 POINT(41.38595162739085 -87.45296084300811) bank157193 +157194 POINT(41.7044732020645 -87.05512992533083) bank157194 +157195 POINT(41.125193342207176 -87.58597880228125) bank157195 +157196 POINT(42.396204476082914 -87.6100971318108) bank157196 +157197 POINT(41.62553657895131 -87.96755099241001) bank157197 +157198 POINT(41.45352351362704 -88.23006142258428) bank157198 +157199 POINT(42.240648673601335 -86.83356849749732) bank157199 +157200 POINT(42.11620648086999 -86.94300487480551) bank157200 +157201 POINT(41.89798743678494 -86.80638263496179) bank157201 +157202 POINT(42.210148636857845 -87.28243003125895) bank157202 +157203 POINT(42.1180929341735 -88.40459931138759) bank157203 +157204 POINT(41.50274881789067 -86.81300454881918) bank157204 +157205 POINT(41.55898172407916 -88.57701659696055) bank157205 +157206 POINT(41.50183128596361 -86.88839791918353) bank157206 +157207 POINT(42.72669629531338 -87.19914461165561) bank157207 +157208 POINT(41.30340627570473 -87.7388818113787) bank157208 +157209 POINT(41.591272908853966 -87.83168137495569) bank157209 +157210 POINT(41.652331316689505 -87.92506593990434) bank157210 +157211 POINT(41.48096470695462 -88.3182216791446) bank157211 +157212 POINT(41.25656010463213 -88.03936512542194) bank157212 +157213 POINT(42.191492218791794 -88.58877372653124) bank157213 +157214 POINT(41.596290740260756 -88.34108358178925) bank157214 +157215 POINT(42.00626560041113 -87.50039989769883) bank157215 +157216 POINT(41.49432085082397 -87.6361162357766) bank157216 +157217 POINT(41.979910748447544 -88.49803553406826) bank157217 +157218 POINT(42.15146075243118 -86.66911777767919) bank157218 +157219 POINT(42.72393583660948 -86.89927097928042) bank157219 +157220 POINT(41.918244909464455 -87.63282406391511) bank157220 +157221 POINT(41.20866213696396 -87.82770285025087) bank157221 +157222 POINT(41.248400021466885 -87.52129308675244) bank157222 +157223 POINT(42.48815587603415 -88.13138660228577) bank157223 +157224 POINT(41.56710616029475 -87.17813278702141) bank157224 +157225 POINT(40.93779190491881 -87.84185297069199) bank157225 +157226 POINT(42.83231689392162 -87.40662848341819) bank157226 +157227 POINT(42.8268662724607 -87.56260889278323) bank157227 +157228 POINT(41.72521237417527 -87.29644141668533) bank157228 +157229 POINT(41.04712400246529 -87.8428488667182) bank157229 +157230 POINT(41.349394268131476 -87.06944552318544) bank157230 +157231 POINT(42.64269270142112 -87.28941355528363) bank157231 +157232 POINT(41.61696108990539 -87.77671954329479) bank157232 +157233 POINT(40.980886833253045 -87.89418228987954) bank157233 +157234 POINT(42.075918472526475 -87.72462395472904) bank157234 +157235 POINT(42.792952308067086 -88.2766391352767) bank157235 +157236 POINT(41.88018112346596 -87.87051378651057) bank157236 +157237 POINT(41.49962530984946 -86.822149819273) bank157237 +157238 POINT(41.101094127808175 -87.06336140312185) bank157238 +157239 POINT(42.76850742026206 -87.66676113164972) bank157239 +157240 POINT(42.047938932510185 -87.5224624162576) bank157240 +157241 POINT(41.027702103515885 -88.55879395475938) bank157241 +157242 POINT(42.30290368280248 -87.04686702122153) bank157242 +157243 POINT(41.10054990453432 -86.82637791602227) bank157243 +157244 POINT(41.530627715466366 -88.56162422635377) bank157244 +157245 POINT(41.81401278611152 -87.7586100497991) bank157245 +157246 POINT(41.91101079181225 -87.61123078997907) bank157246 +157247 POINT(41.71466640640879 -86.68472379708436) bank157247 +157248 POINT(41.38851428885239 -87.6678690974849) bank157248 +157249 POINT(40.99776878417589 -87.1540349534542) bank157249 +157250 POINT(42.414912303793045 -88.12359940659017) bank157250 +157251 POINT(42.873276103741375 -88.51670010698506) bank157251 +157252 POINT(41.738197424413734 -87.60504935823208) bank157252 +157253 POINT(41.983941021927215 -87.90514728012712) bank157253 +157254 POINT(41.080755177116615 -87.3084718496218) bank157254 +157255 POINT(42.8261218897088 -88.29299891371444) bank157255 +157256 POINT(41.79176738151175 -87.00595233401975) bank157256 +157257 POINT(42.87726526258128 -88.27601268087074) bank157257 +157258 POINT(42.131614325352 -87.34967822368971) bank157258 +157259 POINT(41.70573821043843 -88.28933483677247) bank157259 +157260 POINT(42.07021598063773 -87.31791707944808) bank157260 +157261 POINT(41.17367132203425 -87.05652359483746) bank157261 +157262 POINT(42.41680060733626 -86.69900772000133) bank157262 +157263 POINT(42.35799607525914 -86.98618976001714) bank157263 +157264 POINT(41.116485091506654 -88.18214228071474) bank157264 +157265 POINT(41.57970217893135 -86.98764601520651) bank157265 +157266 POINT(41.27066508821046 -88.29577956615985) bank157266 +157267 POINT(41.939840204250444 -86.72208859358594) bank157267 +157268 POINT(41.46108607669675 -87.45883932023433) bank157268 +157269 POINT(42.40441178379409 -87.02935940013256) bank157269 +157270 POINT(40.91131712446966 -86.8940754046032) bank157270 +157271 POINT(41.03195056290591 -86.97908250134267) bank157271 +157272 POINT(42.47719188516366 -88.15575850548485) bank157272 +157273 POINT(41.86782966976019 -87.75488744227356) bank157273 +157274 POINT(41.1412145174523 -87.75801034838338) bank157274 +157275 POINT(42.151301469585825 -87.16016279100539) bank157275 +157276 POINT(42.79991134016311 -87.10528643276633) bank157276 +157277 POINT(42.738734134533146 -87.42437735109792) bank157277 +157278 POINT(42.751330595167424 -87.87126836449679) bank157278 +157279 POINT(41.22696850181109 -86.77324272226716) bank157279 +157280 POINT(42.49876059135306 -88.26326360082541) bank157280 +157281 POINT(42.1020863871679 -88.12311922081454) bank157281 +157282 POINT(42.86074906778775 -86.86069738932636) bank157282 +157283 POINT(42.519166612009776 -88.14548769573202) bank157283 +157284 POINT(41.25553025216439 -87.20947336006586) bank157284 +157285 POINT(41.851144109380904 -88.48232163934549) bank157285 +157286 POINT(42.57940125860273 -87.33150496714359) bank157286 +157287 POINT(41.94645369299661 -88.45590563554636) bank157287 +157288 POINT(42.54755487572557 -88.59410140969023) bank157288 +157289 POINT(42.756669795681056 -86.74908748378465) bank157289 +157290 POINT(41.5801076317929 -87.99360651819136) bank157290 +157291 POINT(42.6174928550942 -87.5272704825399) bank157291 +157292 POINT(41.09521869846682 -88.03167082090692) bank157292 +157293 POINT(42.71227846420358 -88.30420798793669) bank157293 +157294 POINT(41.89556258757894 -87.55614619007889) bank157294 +157295 POINT(42.86368803845339 -87.0840023985385) bank157295 +157296 POINT(42.73704051525635 -87.53940786368607) bank157296 +157297 POINT(40.956177083630905 -88.5986604952611) bank157297 +157298 POINT(41.642634686412954 -88.12466881306965) bank157298 +157299 POINT(42.41795380209487 -87.94725310383839) bank157299 +157300 POINT(42.73290109386769 -88.10672612555179) bank157300 +157301 POINT(41.85092535367401 -88.2476680595286) bank157301 +157302 POINT(41.66186705724981 -88.03984431898012) bank157302 +157303 POINT(42.32771200700595 -86.96804674349949) bank157303 +157304 POINT(42.14527177202359 -87.75734969178494) bank157304 +157305 POINT(41.78585409126085 -87.19882860027182) bank157305 +157306 POINT(42.55429806573183 -88.40950196189382) bank157306 +157307 POINT(42.081211675136885 -88.09719886717608) bank157307 +157308 POINT(42.55773626434264 -87.77126892946929) bank157308 +157309 POINT(41.43379860297759 -88.0596923173779) bank157309 +157310 POINT(41.05853288679163 -87.70589642044894) bank157310 +157311 POINT(41.550785261993774 -86.68370998826367) bank157311 +157312 POINT(42.57657572723275 -88.17548240456797) bank157312 +157313 POINT(41.55399625161157 -87.77653083240345) bank157313 +157314 POINT(42.771774397122066 -87.06500507518835) bank157314 +157315 POINT(41.03249069023168 -87.69400249491991) bank157315 +157316 POINT(42.64860885045547 -87.90658454332093) bank157316 +157317 POINT(41.239243514921704 -87.82249607295763) bank157317 +157318 POINT(42.62966438668849 -87.18215392948143) bank157318 +157319 POINT(40.886802952112944 -88.07717767818343) bank157319 +157320 POINT(41.87669589576018 -87.7765814642048) bank157320 +157321 POINT(40.9503297837572 -87.25140261999083) bank157321 +157322 POINT(41.74482439536319 -88.13754084892928) bank157322 +157323 POINT(42.267510672103256 -87.69862383971403) bank157323 +157324 POINT(42.406151720382596 -88.42793611040794) bank157324 +157325 POINT(41.44173951855838 -88.45779649242559) bank157325 +157326 POINT(41.2477724303996 -87.15310722793943) bank157326 +157327 POINT(41.068859633722745 -88.08298743491058) bank157327 +157328 POINT(41.39994684526528 -86.81719589355167) bank157328 +157329 POINT(42.52631183330464 -87.41485113550088) bank157329 +157330 POINT(41.50449460604605 -87.87852414450309) bank157330 +157331 POINT(42.739901474081556 -86.78087089323772) bank157331 +157332 POINT(42.74559815882941 -87.61761077687531) bank157332 +157333 POINT(41.72544882119857 -88.12885519845331) bank157333 +157334 POINT(41.46398796044099 -87.70420663279319) bank157334 +157335 POINT(42.45220267877922 -87.35967946267012) bank157335 +157336 POINT(41.88520342305099 -86.80494250055763) bank157336 +157337 POINT(42.69070055870802 -88.55968622898074) bank157337 +157338 POINT(41.10236422304691 -88.58310110441468) bank157338 +157339 POINT(40.97233401686381 -86.64448015732694) bank157339 +157340 POINT(41.5468462837011 -88.18119848218497) bank157340 +157341 POINT(41.43064555022561 -87.25851327221287) bank157341 +157342 POINT(42.1383135781507 -87.65076368191119) bank157342 +157343 POINT(41.80295548985565 -87.87889914515105) bank157343 +157344 POINT(41.26420522982133 -87.1472655341429) bank157344 +157345 POINT(41.762570382990425 -88.42036296864163) bank157345 +157346 POINT(42.502585235317156 -88.46649617481003) bank157346 +157347 POINT(42.78298588005782 -86.86639426043901) bank157347 +157348 POINT(41.82357937279354 -88.57336535191698) bank157348 +157349 POINT(41.311144132049556 -87.91849475645748) bank157349 +157350 POINT(41.220718448612296 -88.27830718132577) bank157350 +157351 POINT(42.469906783001385 -88.25499100257973) bank157351 +157352 POINT(41.246541452632364 -88.59796632067106) bank157352 +157353 POINT(42.577894900037975 -87.82274279363384) bank157353 +157354 POINT(41.17067772176042 -87.40805093535286) bank157354 +157355 POINT(42.0452485327871 -87.26816966518236) bank157355 +157356 POINT(42.18074643777215 -86.84712291558407) bank157356 +157357 POINT(41.3658280966872 -88.16687937615484) bank157357 +157358 POINT(41.854615638578444 -87.7208582503986) bank157358 +157359 POINT(42.28334337608932 -87.73937125300762) bank157359 +157360 POINT(41.68052134416651 -87.10666402113209) bank157360 +157361 POINT(40.95687212992837 -86.66129786655952) bank157361 +157362 POINT(41.766529910509604 -87.02439913706652) bank157362 +157363 POINT(42.05617599677656 -87.38890691854286) bank157363 +157364 POINT(42.01501216335656 -87.97660791579845) bank157364 +157365 POINT(42.55092742707124 -88.62848095991716) bank157365 +157366 POINT(40.89668031405398 -87.53240366499796) bank157366 +157367 POINT(41.59373819526522 -88.19512643872372) bank157367 +157368 POINT(41.219456635158 -88.60150506792216) bank157368 +157369 POINT(42.497354257287164 -87.5487127882666) bank157369 +157370 POINT(41.72684276986326 -88.13942315782982) bank157370 +157371 POINT(42.050374156296215 -88.62762494999133) bank157371 +157372 POINT(42.08569123604776 -87.81025066315092) bank157372 +157373 POINT(41.44607107727259 -87.01399038912174) bank157373 +157374 POINT(42.77991384694286 -88.11994400431233) bank157374 +157375 POINT(41.23323224602974 -87.38832324825556) bank157375 +157376 POINT(41.03001482624064 -86.93858815760822) bank157376 +157377 POINT(41.35672656073388 -87.08091641138645) bank157377 +157378 POINT(42.01489636619429 -88.23189924477498) bank157378 +157379 POINT(41.85948264752269 -87.58572169647687) bank157379 +157380 POINT(42.87497528511605 -87.60545840370179) bank157380 +157381 POINT(41.01939516712518 -86.8914944228794) bank157381 +157382 POINT(41.672140030055175 -87.14604394563308) bank157382 +157383 POINT(40.89573041308805 -87.5028948656062) bank157383 +157384 POINT(41.26779145928324 -87.4962295650815) bank157384 +157385 POINT(42.2166614712946 -87.69496165715914) bank157385 +157386 POINT(41.53972005853287 -86.74688715117938) bank157386 +157387 POINT(41.87011795489054 -87.06661436345637) bank157387 +157388 POINT(42.5547048898272 -87.77543092296554) bank157388 +157389 POINT(42.50150987124035 -87.53582269060189) bank157389 +157390 POINT(42.588527161202954 -88.26399608221118) bank157390 +157391 POINT(42.83209873447187 -87.76618858369687) bank157391 +157392 POINT(41.00764205275088 -88.25771665409158) bank157392 +157393 POINT(41.02220248194526 -87.81561028717643) bank157393 +157394 POINT(41.555624653025035 -87.21507201045652) bank157394 +157395 POINT(41.15960703493071 -87.03125954062769) bank157395 +157396 POINT(42.62699701146321 -86.93100935853032) bank157396 +157397 POINT(41.90312048735667 -87.81683768187335) bank157397 +157398 POINT(41.47933968245888 -87.09291638583188) bank157398 +157399 POINT(41.337847858345945 -87.1619456960216) bank157399 +157400 POINT(41.24120684644669 -88.00249181107651) bank157400 +157401 POINT(41.14396123402023 -87.86307907127951) bank157401 +157402 POINT(41.953954259770875 -87.51409298743633) bank157402 +157403 POINT(42.260600445479014 -86.80780698743706) bank157403 +157404 POINT(41.59375664675396 -86.71663269573034) bank157404 +157405 POINT(42.76125399763181 -88.523276517228) bank157405 +157406 POINT(41.485817356566244 -87.25353506748384) bank157406 +157407 POINT(41.79217731383539 -88.15384320277668) bank157407 +157408 POINT(42.3033886636999 -87.831785077755) bank157408 +157409 POINT(42.00908463123773 -87.09492290207318) bank157409 +157410 POINT(41.96665365033143 -88.15974495412246) bank157410 +157411 POINT(41.53255574442046 -87.87129216214335) bank157411 +157412 POINT(41.18592678479679 -87.7896876717238) bank157412 +157413 POINT(41.27043580291021 -87.59410807241898) bank157413 +157414 POINT(41.61536737226641 -87.11654642690785) bank157414 +157415 POINT(42.14449594935156 -88.59944153001882) bank157415 +157416 POINT(42.49293934307504 -87.06747954253896) bank157416 +157417 POINT(42.079113634551426 -87.16656800983124) bank157417 +157418 POINT(41.28558898265547 -87.3327001628935) bank157418 +157419 POINT(41.69986130592712 -86.96416323216398) bank157419 +157420 POINT(41.65363206577946 -87.17623853651601) bank157420 +157421 POINT(41.8265692530066 -87.38652871929625) bank157421 +157422 POINT(42.33299072322204 -86.84296666034118) bank157422 +157423 POINT(42.153918092850994 -87.67773987352902) bank157423 +157424 POINT(41.47723927389975 -88.47200863675324) bank157424 +157425 POINT(41.08138592828246 -87.53731385398439) bank157425 +157426 POINT(41.98991048222352 -87.21470109623971) bank157426 +157427 POINT(41.692154263621276 -88.03088263073926) bank157427 +157428 POINT(41.581078658320614 -88.35076784845748) bank157428 +157429 POINT(41.73749985955744 -88.16718038567062) bank157429 +157430 POINT(40.99227179005676 -87.96971317826973) bank157430 +157431 POINT(42.65354883965088 -88.13991377985995) bank157431 +157432 POINT(41.80830822724161 -87.69885908901308) bank157432 +157433 POINT(41.792603678048714 -86.74210892382253) bank157433 +157434 POINT(42.73173452388632 -87.74984760740284) bank157434 +157435 POINT(42.637757363775975 -87.8376493962055) bank157435 +157436 POINT(42.604788805576554 -87.51936269104925) bank157436 +157437 POINT(41.914113568213956 -86.88066119445034) bank157437 +157438 POINT(42.87446668778314 -88.14884079920994) bank157438 +157439 POINT(42.804779134507875 -87.36886266390657) bank157439 +157440 POINT(41.60302325633539 -88.37712881142701) bank157440 +157441 POINT(41.2540972805541 -86.68515250829634) bank157441 +157442 POINT(41.84469270865714 -88.20348563835641) bank157442 +157443 POINT(41.02515366783432 -86.85213977799408) bank157443 +157444 POINT(41.13484920800826 -87.01004032289764) bank157444 +157445 POINT(41.72500826661691 -87.88719047647118) bank157445 +157446 POINT(42.71349214199953 -87.94554264849906) bank157446 +157447 POINT(42.60867068926613 -87.59313885252165) bank157447 +157448 POINT(41.049124053226706 -86.99066080262092) bank157448 +157449 POINT(42.085986817627166 -88.41783331233876) bank157449 +157450 POINT(42.800630968160576 -86.8201095941652) bank157450 +157451 POINT(41.03652858824204 -87.51838772610355) bank157451 +157452 POINT(41.0446226894027 -87.76461332549442) bank157452 +157453 POINT(41.003062383251006 -88.08162758921085) bank157453 +157454 POINT(42.76964891211326 -88.43482952772179) bank157454 +157455 POINT(41.18770989384109 -86.79192828186719) bank157455 +157456 POINT(41.10490359571241 -88.39091317996693) bank157456 +157457 POINT(42.33752974371978 -87.66701720086131) bank157457 +157458 POINT(41.65105715793053 -87.34010887321124) bank157458 +157459 POINT(42.838180745640614 -87.41559419252263) bank157459 +157460 POINT(42.166931099004 -88.55219043063776) bank157460 +157461 POINT(41.57856441846696 -87.33367829083541) bank157461 +157462 POINT(42.093539188054706 -87.63792006248285) bank157462 +157463 POINT(42.42258296689479 -87.70508285486518) bank157463 +157464 POINT(42.33703743696628 -86.72545086793811) bank157464 +157465 POINT(42.39280355673031 -87.12792990998868) bank157465 +157466 POINT(42.48768732882076 -87.31724506635118) bank157466 +157467 POINT(41.1206199463613 -87.74627283668359) bank157467 +157468 POINT(42.59333012328397 -86.68478934549829) bank157468 +157469 POINT(42.37673535158744 -88.17075942672503) bank157469 +157470 POINT(40.975119069838414 -86.99256320038141) bank157470 +157471 POINT(42.02462959145287 -88.22213332201105) bank157471 +157472 POINT(41.61170615193754 -87.65841295784938) bank157472 +157473 POINT(41.20747147208067 -87.3575688932706) bank157473 +157474 POINT(42.1936563737217 -87.67953844056888) bank157474 +157475 POINT(42.69407235242435 -87.30124943657816) bank157475 +157476 POINT(42.08663063314452 -88.24452301658766) bank157476 +157477 POINT(41.91926508001783 -87.01801834912115) bank157477 +157478 POINT(42.3748707053632 -88.30110894472423) bank157478 +157479 POINT(40.95805188181014 -87.94778392576198) bank157479 +157480 POINT(41.57209406355478 -88.4706472296376) bank157480 +157481 POINT(41.81369799190539 -87.20863624899795) bank157481 +157482 POINT(42.3392750094625 -86.88702673126842) bank157482 +157483 POINT(41.19974737601072 -87.9143169380767) bank157483 +157484 POINT(42.386352054158785 -87.64432158233416) bank157484 +157485 POINT(42.717600135065766 -87.9638992937554) bank157485 +157486 POINT(42.25401532683288 -87.19345252300059) bank157486 +157487 POINT(41.429108096784766 -88.52409191003504) bank157487 +157488 POINT(41.56890861942102 -88.55505812174337) bank157488 +157489 POINT(42.21391406238914 -87.88028236171428) bank157489 +157490 POINT(42.22030023058124 -87.15520813932703) bank157490 +157491 POINT(41.911455243857745 -87.71817964491711) bank157491 +157492 POINT(42.28100509828373 -87.93088845266156) bank157492 +157493 POINT(42.158647844139686 -87.51253722240367) bank157493 +157494 POINT(42.18309993245519 -86.66969859350814) bank157494 +157495 POINT(42.24750813987085 -87.29089207862785) bank157495 +157496 POINT(42.55126875639847 -88.25848699581208) bank157496 +157497 POINT(41.21929986297905 -86.84312377520624) bank157497 +157498 POINT(42.81580305438022 -88.12701933398564) bank157498 +157499 POINT(40.992526413961016 -87.855334548598) bank157499 +157500 POINT(42.71714925074806 -86.7556480152598) bank157500 +157501 POINT(42.39860587968383 -87.93163429868868) bank157501 +157502 POINT(41.58091078935789 -86.69298315671422) bank157502 +157503 POINT(42.77513597685739 -86.69113111884023) bank157503 +157504 POINT(42.16421695657689 -86.79881652622073) bank157504 +157505 POINT(42.1978189983231 -87.2515031086787) bank157505 +157506 POINT(42.58550145230804 -88.33242611889486) bank157506 +157507 POINT(40.953207646284305 -87.20891665212137) bank157507 +157508 POINT(41.638546156230056 -88.38194399726687) bank157508 +157509 POINT(40.97044293058375 -88.40925955000372) bank157509 +157510 POINT(41.55369527790303 -87.1378278705266) bank157510 +157511 POINT(42.46787300861487 -86.8795629713561) bank157511 +157512 POINT(41.509624148688395 -87.01681304366528) bank157512 +157513 POINT(41.19251119836318 -87.00399112324914) bank157513 +157514 POINT(41.511838353693626 -87.14659965113762) bank157514 +157515 POINT(41.45793033659597 -87.3955305143369) bank157515 +157516 POINT(40.895759882293405 -88.13913842477506) bank157516 +157517 POINT(41.487379218707325 -87.66836856542281) bank157517 +157518 POINT(42.445267277701824 -87.64048168864684) bank157518 +157519 POINT(41.415428482918614 -87.49094235720531) bank157519 +157520 POINT(41.13166194585675 -86.66701459849067) bank157520 +157521 POINT(41.2737410342882 -87.96289473542808) bank157521 +157522 POINT(41.97829293904424 -87.87172455726868) bank157522 +157523 POINT(41.07788669547364 -88.35379558572022) bank157523 +157524 POINT(41.07057741260454 -88.16475180684343) bank157524 +157525 POINT(40.940753149280695 -87.15922401747689) bank157525 +157526 POINT(41.00343514336823 -88.185245802165) bank157526 +157527 POINT(41.409943611654455 -88.11296113794609) bank157527 +157528 POINT(42.56614303265305 -87.2402698959651) bank157528 +157529 POINT(41.81481931023076 -88.54255833215642) bank157529 +157530 POINT(41.55909579633236 -87.74606475346854) bank157530 +157531 POINT(42.0834833169279 -87.6555792206882) bank157531 +157532 POINT(42.818704541962695 -88.2318963287962) bank157532 +157533 POINT(41.7059008442251 -88.49138735437609) bank157533 +157534 POINT(41.67592195236853 -87.0631006926773) bank157534 +157535 POINT(41.93777907484768 -86.9073005321553) bank157535 +157536 POINT(41.126764897795326 -87.06803271098322) bank157536 +157537 POINT(42.68169648778242 -86.84298510588293) bank157537 +157538 POINT(42.393397508500065 -87.2125716833553) bank157538 +157539 POINT(41.0619471355473 -87.26585158876493) bank157539 +157540 POINT(42.108794692778744 -86.65724516154762) bank157540 +157541 POINT(41.969776788688364 -87.3943868458859) bank157541 +157542 POINT(41.70270427620463 -87.80478605617041) bank157542 +157543 POINT(41.140472634381894 -88.52945370784624) bank157543 +157544 POINT(41.49383814547506 -86.80117516842726) bank157544 +157545 POINT(40.943200101735 -87.8264170889208) bank157545 +157546 POINT(42.8258269524313 -87.89537720571414) bank157546 +157547 POINT(42.28347265534566 -88.15316731294965) bank157547 +157548 POINT(41.750317999104844 -87.61521525745918) bank157548 +157549 POINT(41.5892199652565 -88.62304450055697) bank157549 +157550 POINT(41.437377191150794 -88.13397293289125) bank157550 +157551 POINT(41.51915266743126 -87.11350634909999) bank157551 +157552 POINT(42.73818997873011 -86.76824387661154) bank157552 +157553 POINT(41.76979128504373 -86.99095832836042) bank157553 +157554 POINT(42.85861038496856 -87.85042266171729) bank157554 +157555 POINT(41.341515237430755 -87.83072497302761) bank157555 +157556 POINT(42.76745758391533 -87.24963565143521) bank157556 +157557 POINT(41.180543232294895 -86.78874868896985) bank157557 +157558 POINT(42.7698411448132 -87.49554642348721) bank157558 +157559 POINT(41.573161276182375 -88.08421373184031) bank157559 +157560 POINT(42.136385289950965 -86.63817705667458) bank157560 +157561 POINT(40.94824072611219 -88.34732145747977) bank157561 +157562 POINT(41.351699312565025 -86.84392254672768) bank157562 +157563 POINT(41.06680430604175 -87.39778963309531) bank157563 +157564 POINT(41.84680430672392 -87.41517403232749) bank157564 +157565 POINT(42.69782855506993 -87.9728193590856) bank157565 +157566 POINT(42.40042577629526 -87.5416957284951) bank157566 +157567 POINT(42.678789215781954 -87.86117927989582) bank157567 +157568 POINT(42.25334049374008 -88.46797308319354) bank157568 +157569 POINT(41.59923591172948 -88.41534823716748) bank157569 +157570 POINT(42.657387833707496 -87.8066903980875) bank157570 +157571 POINT(42.438328831569805 -88.04861445685518) bank157571 +157572 POINT(41.36819819509349 -87.16997244586362) bank157572 +157573 POINT(41.009872512278335 -87.67054049164538) bank157573 +157574 POINT(42.71852095030211 -87.78803041980294) bank157574 +157575 POINT(42.63809360481815 -88.17196290622559) bank157575 +157576 POINT(42.60690434123733 -88.03499227440012) bank157576 +157577 POINT(41.27625410341152 -88.61515201842467) bank157577 +157578 POINT(41.4535481725414 -87.6286468720554) bank157578 +157579 POINT(41.986098213145816 -87.6327128015107) bank157579 +157580 POINT(42.37455515595663 -88.40375988142344) bank157580 +157581 POINT(41.17196605715223 -88.56198834727563) bank157581 +157582 POINT(41.39866000980543 -86.63907466545939) bank157582 +157583 POINT(41.16411515964907 -88.06683940442421) bank157583 +157584 POINT(42.39080115425562 -87.12513279068601) bank157584 +157585 POINT(41.3131710869491 -86.94051459524799) bank157585 +157586 POINT(41.1628453962062 -86.80112110587885) bank157586 +157587 POINT(42.622947417492945 -88.53150428690954) bank157587 +157588 POINT(40.94256568387639 -87.01429809212601) bank157588 +157589 POINT(41.483707128255226 -88.01135093187278) bank157589 +157590 POINT(41.53341087342285 -88.03867066052909) bank157590 +157591 POINT(41.518031601492495 -87.24892447289784) bank157591 +157592 POINT(42.04307169236678 -87.91212091661008) bank157592 +157593 POINT(41.452119013014126 -88.53362029474349) bank157593 +157594 POINT(41.837883465626085 -87.18757413959378) bank157594 +157595 POINT(41.59313104192384 -88.50971058042354) bank157595 +157596 POINT(42.74432606358684 -87.09328946229422) bank157596 +157597 POINT(41.37008306521431 -87.8666391429413) bank157597 +157598 POINT(40.910778657535154 -87.47233590724997) bank157598 +157599 POINT(41.894473779946466 -86.88654086633781) bank157599 +157600 POINT(41.4923646299778 -87.18610566567239) bank157600 +157601 POINT(40.898990782167 -88.1859533697595) bank157601 +157602 POINT(41.678973680820086 -87.36308517752148) bank157602 +157603 POINT(40.93982553602645 -86.78855908451779) bank157603 +157604 POINT(41.08595576005575 -87.96613352965365) bank157604 +157605 POINT(42.85471309602481 -88.0495071357135) bank157605 +157606 POINT(42.142887175596755 -87.48378516719742) bank157606 +157607 POINT(41.162810474515545 -88.03306052641406) bank157607 +157608 POINT(42.736582172253414 -88.19631389456686) bank157608 +157609 POINT(42.73191371056836 -87.72570749122319) bank157609 +157610 POINT(42.22811628841183 -87.3769242878133) bank157610 +157611 POINT(41.30156395995938 -86.9956855266621) bank157611 +157612 POINT(42.173600244478 -87.27441288029814) bank157612 +157613 POINT(42.192633745803526 -87.34463643615729) bank157613 +157614 POINT(42.64942563377518 -88.36995058596864) bank157614 +157615 POINT(42.602314050093554 -88.42535907041314) bank157615 +157616 POINT(42.682323982144645 -87.97708575872316) bank157616 +157617 POINT(41.954204055053204 -86.63326281328082) bank157617 +157618 POINT(41.02751820272264 -86.89426055952882) bank157618 +157619 POINT(41.259157590097445 -87.44556555656725) bank157619 +157620 POINT(42.56918347034938 -87.36395995642756) bank157620 +157621 POINT(42.621617917536156 -88.48264948403018) bank157621 +157622 POINT(41.412943068149225 -88.35530598986445) bank157622 +157623 POINT(41.629019993537106 -88.1995531334786) bank157623 +157624 POINT(42.35988648607759 -87.79323857832323) bank157624 +157625 POINT(42.501250724382196 -88.11381766194314) bank157625 +157626 POINT(42.13146465047099 -88.55083172712621) bank157626 +157627 POINT(42.706152066933946 -87.1546114808084) bank157627 +157628 POINT(41.3912673625152 -87.58720127295746) bank157628 +157629 POINT(41.178486127396525 -87.2304535101467) bank157629 +157630 POINT(42.47352327066963 -87.73009300536302) bank157630 +157631 POINT(42.58191965161289 -87.65698433659936) bank157631 +157632 POINT(41.491156482348245 -87.8776089633867) bank157632 +157633 POINT(41.90988595216387 -86.896714997035) bank157633 +157634 POINT(41.32554713302375 -88.24913572862023) bank157634 +157635 POINT(42.624129872590764 -86.70844199775021) bank157635 +157636 POINT(42.65852927403575 -86.82509221588909) bank157636 +157637 POINT(42.574644750626916 -88.24965576213587) bank157637 +157638 POINT(42.02205417797742 -87.95929268544413) bank157638 +157639 POINT(41.487142753023306 -87.23201452974436) bank157639 +157640 POINT(41.518753654289654 -87.89501988267519) bank157640 +157641 POINT(41.62807277633204 -86.74672198284316) bank157641 +157642 POINT(41.541554237091496 -88.08598211790937) bank157642 +157643 POINT(41.71774751082004 -87.02494174409142) bank157643 +157644 POINT(41.94086160119805 -88.33732643198347) bank157644 +157645 POINT(42.42722791090628 -86.94213413322929) bank157645 +157646 POINT(42.53584739157703 -86.99652711058144) bank157646 +157647 POINT(41.03264401133483 -86.70027229055444) bank157647 +157648 POINT(42.49226283963984 -87.83604002625097) bank157648 +157649 POINT(42.52193179116086 -87.82387159868634) bank157649 +157650 POINT(42.83272529791327 -86.91760986177232) bank157650 +157651 POINT(42.23156905947997 -86.70014931277976) bank157651 +157652 POINT(41.42652875816356 -88.3285384554006) bank157652 +157653 POINT(42.590496864127395 -86.99359766190534) bank157653 +157654 POINT(40.91876645824861 -88.02809975369652) bank157654 +157655 POINT(41.656597987702234 -87.94609902120288) bank157655 +157656 POINT(42.131962191500605 -88.46260483740639) bank157656 +157657 POINT(41.66650730547034 -88.11606619566298) bank157657 +157658 POINT(41.16855379979088 -86.71800751744398) bank157658 +157659 POINT(42.68602719765508 -88.41606261661035) bank157659 +157660 POINT(41.831495721484906 -87.01549689516862) bank157660 +157661 POINT(41.70737005147672 -87.6436424771348) bank157661 +157662 POINT(41.59161329415159 -87.69643830893901) bank157662 +157663 POINT(41.16036117888345 -87.17709375691622) bank157663 +157664 POINT(41.479611872229434 -87.02177457077433) bank157664 +157665 POINT(41.72454247242633 -87.63623682066194) bank157665 +157666 POINT(42.52737616656363 -88.05304381588954) bank157666 +157667 POINT(42.00385514943277 -87.16221397513053) bank157667 +157668 POINT(42.23891599236885 -88.30452811340474) bank157668 +157669 POINT(41.343331256724454 -87.27471467466256) bank157669 +157670 POINT(41.52158701789362 -87.87249209595687) bank157670 +157671 POINT(41.0956935280505 -87.26740725157293) bank157671 +157672 POINT(41.63987540841329 -88.36216037023273) bank157672 +157673 POINT(41.70681811743452 -88.29649125865242) bank157673 +157674 POINT(41.96312606141922 -86.97230842327112) bank157674 +157675 POINT(42.47849669351344 -88.20183799898147) bank157675 +157676 POINT(42.67271205478738 -87.00673744609733) bank157676 +157677 POINT(41.252975443323805 -86.9354846479012) bank157677 +157678 POINT(42.63630205519856 -87.42153920019547) bank157678 +157679 POINT(42.76094629425142 -87.81165287117486) bank157679 +157680 POINT(41.593971179399524 -87.1315820233603) bank157680 +157681 POINT(41.449142606703376 -88.41838349410382) bank157681 +157682 POINT(41.640857117063526 -86.92623763291253) bank157682 +157683 POINT(41.51123027844982 -87.15200437391543) bank157683 +157684 POINT(41.58250866634802 -86.96512605596986) bank157684 +157685 POINT(41.62237692852313 -88.08432135828663) bank157685 +157686 POINT(41.050301460006125 -87.88869366571639) bank157686 +157687 POINT(41.511887680837866 -87.3847936797023) bank157687 +157688 POINT(42.484533270956355 -86.78894781573764) bank157688 +157689 POINT(40.89314568077359 -87.95723327994745) bank157689 +157690 POINT(41.617941617726075 -88.34248105147043) bank157690 +157691 POINT(41.9051563933792 -88.5511717742842) bank157691 +157692 POINT(41.09152622667088 -86.7977729514376) bank157692 +157693 POINT(41.517577023834505 -87.76095306388022) bank157693 +157694 POINT(41.452098429784996 -88.32627160161476) bank157694 +157695 POINT(41.61465839223209 -87.41757771580524) bank157695 +157696 POINT(42.22822035938526 -87.80294897380949) bank157696 +157697 POINT(42.56394647409894 -87.00488054697338) bank157697 +157698 POINT(42.668303282765855 -87.72738361365126) bank157698 +157699 POINT(41.04234191690933 -87.09610609130551) bank157699 +157700 POINT(41.289972567605126 -87.95747728623081) bank157700 +157701 POINT(40.944222039016374 -88.1054273428752) bank157701 +157702 POINT(41.79248482910832 -87.18482111947945) bank157702 +157703 POINT(41.95811252505061 -87.4449193846015) bank157703 +157704 POINT(41.60039455488725 -87.73614037175874) bank157704 +157705 POINT(41.44490890523053 -86.9411325209665) bank157705 +157706 POINT(41.648212311805565 -87.49347786384939) bank157706 +157707 POINT(41.06661189461605 -86.70845965296925) bank157707 +157708 POINT(42.83017637007686 -88.01454858743979) bank157708 +157709 POINT(41.24575643097685 -86.72366447083776) bank157709 +157710 POINT(42.07229322911825 -87.26645923057897) bank157710 +157711 POINT(41.46146488573727 -86.76314016393881) bank157711 +157712 POINT(42.60417992199198 -88.21178931546038) bank157712 +157713 POINT(41.745037600600575 -87.53284847658085) bank157713 +157714 POINT(41.201536518370666 -86.63653315145231) bank157714 +157715 POINT(42.71465908208783 -87.29095243634104) bank157715 +157716 POINT(41.50491497711473 -87.70225303113239) bank157716 +157717 POINT(42.841401672211305 -86.85011105543248) bank157717 +157718 POINT(41.735060104291506 -87.04706859307686) bank157718 +157719 POINT(42.659591579387985 -88.2509075971399) bank157719 +157720 POINT(41.06365312640487 -86.9902710737103) bank157720 +157721 POINT(42.52540595085347 -88.59112255360789) bank157721 +157722 POINT(42.83587771210094 -88.08496975471064) bank157722 +157723 POINT(42.50939552142124 -86.642732334577) bank157723 +157724 POINT(41.37375630725354 -87.41327908032916) bank157724 +157725 POINT(40.91627400136957 -88.57359506737308) bank157725 +157726 POINT(41.91130409305727 -86.63995629232134) bank157726 +157727 POINT(41.459947334385916 -88.33405706748754) bank157727 +157728 POINT(42.216671259098995 -86.71334959475645) bank157728 +157729 POINT(42.53380004123627 -88.58443944724763) bank157729 +157730 POINT(41.902064891478304 -88.43727439308964) bank157730 +157731 POINT(41.20839438561793 -87.42231222399369) bank157731 +157732 POINT(42.19539668156245 -88.54248434555022) bank157732 +157733 POINT(41.974124594523154 -88.50627492388296) bank157733 +157734 POINT(41.98283377417047 -86.99004901472391) bank157734 +157735 POINT(42.68576068501516 -86.95134988508048) bank157735 +157736 POINT(41.64416398534423 -87.95214551232863) bank157736 +157737 POINT(41.279051473946865 -87.38068915884082) bank157737 +157738 POINT(42.13188052926885 -87.40821645124427) bank157738 +157739 POINT(42.63699831785041 -87.23479570685515) bank157739 +157740 POINT(42.45236025884651 -86.72452902471822) bank157740 +157741 POINT(42.78599557856168 -86.69908652202557) bank157741 +157742 POINT(41.074690843444955 -88.50265213230392) bank157742 +157743 POINT(42.80975264071499 -88.51813649778684) bank157743 +157744 POINT(41.98002628968532 -87.67451080897102) bank157744 +157745 POINT(42.60068447834629 -87.97371415611057) bank157745 +157746 POINT(41.22017449963592 -87.32188243004599) bank157746 +157747 POINT(42.237848038466524 -86.90882668493376) bank157747 +157748 POINT(40.99247761587237 -87.62954221825686) bank157748 +157749 POINT(42.510063627011334 -87.35191650837426) bank157749 +157750 POINT(42.7558309433077 -88.28943623554224) bank157750 +157751 POINT(41.92285274190227 -87.32216749825449) bank157751 +157752 POINT(41.13025231780123 -86.69746327313021) bank157752 +157753 POINT(40.97388988829373 -88.60374054651518) bank157753 +157754 POINT(41.07391056134101 -87.38690118042791) bank157754 +157755 POINT(42.65735803307376 -88.15489138233085) bank157755 +157756 POINT(42.526186229802434 -88.34979965891256) bank157756 +157757 POINT(41.47508428108077 -88.52863518029636) bank157757 +157758 POINT(42.81633419855167 -87.6170889601259) bank157758 +157759 POINT(41.89822269585882 -86.84412970321043) bank157759 +157760 POINT(41.11215132734629 -88.38218692043982) bank157760 +157761 POINT(42.09987777067562 -88.34691505123476) bank157761 +157762 POINT(41.04317329517723 -88.12388735398466) bank157762 +157763 POINT(41.82032450134176 -88.02953544933763) bank157763 +157764 POINT(41.23741834907144 -88.1787597727033) bank157764 +157765 POINT(41.095188249178314 -88.00249851568277) bank157765 +157766 POINT(41.60133954348431 -88.3847227039859) bank157766 +157767 POINT(41.970235649481 -86.83234234568761) bank157767 +157768 POINT(42.3005048782346 -87.98062534091055) bank157768 +157769 POINT(42.80284106804266 -87.90860254990336) bank157769 +157770 POINT(40.99370303989141 -88.2460433536636) bank157770 +157771 POINT(41.01289825842135 -88.01443522760047) bank157771 +157772 POINT(42.21317701245731 -86.66301705593449) bank157772 +157773 POINT(41.05439944170964 -87.176277445192) bank157773 +157774 POINT(41.12403957841732 -87.72550641315344) bank157774 +157775 POINT(41.73105056928524 -87.52000932441706) bank157775 +157776 POINT(42.54830069419112 -87.68269356015816) bank157776 +157777 POINT(41.61438617373984 -87.18943653955054) bank157777 +157778 POINT(41.122563326700906 -86.72233077690369) bank157778 +157779 POINT(42.5149401941069 -86.8555073388209) bank157779 +157780 POINT(41.287925482310115 -87.3621792762354) bank157780 +157781 POINT(42.87522107838933 -87.10579333820655) bank157781 +157782 POINT(42.8264738301124 -86.84108773847615) bank157782 +157783 POINT(42.442620668426265 -88.43702160246706) bank157783 +157784 POINT(42.0418675307142 -87.72662070002143) bank157784 +157785 POINT(41.49316690920682 -87.49439526793346) bank157785 +157786 POINT(42.76478462580633 -87.4660892716672) bank157786 +157787 POINT(42.173999680285235 -87.50406168008509) bank157787 +157788 POINT(40.89032973163471 -86.68883884355188) bank157788 +157789 POINT(42.869034635629724 -87.80192756707446) bank157789 +157790 POINT(41.582087766202065 -87.84422878040841) bank157790 +157791 POINT(41.19216052795005 -86.65241596179422) bank157791 +157792 POINT(41.64135061341742 -88.2079473897419) bank157792 +157793 POINT(41.424562780705244 -87.02749856733203) bank157793 +157794 POINT(42.78893580395469 -87.21120049304733) bank157794 +157795 POINT(42.24249184008788 -87.16707989945336) bank157795 +157796 POINT(42.14992872545301 -87.34191446478607) bank157796 +157797 POINT(42.57550928468996 -88.5913883997033) bank157797 +157798 POINT(42.847438696995106 -87.25649951985376) bank157798 +157799 POINT(42.25235735194986 -88.59065603832606) bank157799 +157800 POINT(41.99207350895605 -86.75520032402909) bank157800 +157801 POINT(41.03272263570549 -88.29462836370898) bank157801 +157802 POINT(41.73021049363352 -87.52378741043769) bank157802 +157803 POINT(42.58101100917771 -87.11463104223414) bank157803 +157804 POINT(41.1380253313213 -86.72420658267653) bank157804 +157805 POINT(41.34196543401306 -87.90988875353544) bank157805 +157806 POINT(42.04993515994803 -87.41427132271883) bank157806 +157807 POINT(41.32089961605972 -87.23311806855996) bank157807 +157808 POINT(41.71761797212271 -86.75987618559377) bank157808 +157809 POINT(40.9923526227028 -86.7713369403233) bank157809 +157810 POINT(41.9991368929141 -87.73030784678555) bank157810 +157811 POINT(42.836123052705695 -87.87402765366143) bank157811 +157812 POINT(42.47974066655403 -87.44990094329303) bank157812 +157813 POINT(42.687602671342525 -88.52621552693502) bank157813 +157814 POINT(42.791010162174985 -86.97976583225447) bank157814 +157815 POINT(42.28381798266571 -87.72778900250874) bank157815 +157816 POINT(42.648591430523936 -88.15303074753847) bank157816 +157817 POINT(42.73696108378838 -87.43756637369732) bank157817 +157818 POINT(41.77247610295106 -86.9658509930262) bank157818 +157819 POINT(42.38523299946161 -88.32206829807167) bank157819 +157820 POINT(41.161959585760385 -87.21322357493361) bank157820 +157821 POINT(41.08762301120831 -86.80572457502032) bank157821 +157822 POINT(42.85254948441111 -87.8314320231472) bank157822 +157823 POINT(41.6163031878853 -87.00160383993716) bank157823 +157824 POINT(41.61756566456048 -87.02737649848346) bank157824 +157825 POINT(41.07542356663992 -87.86313525688853) bank157825 +157826 POINT(42.14173606736097 -87.98198079289688) bank157826 +157827 POINT(41.146369423836 -88.38920146553474) bank157827 +157828 POINT(41.36207263312038 -87.71057182960804) bank157828 +157829 POINT(41.76348136515072 -87.82224240817423) bank157829 +157830 POINT(41.30229401657648 -87.07910355960688) bank157830 +157831 POINT(41.338888737891445 -88.10841079442018) bank157831 +157832 POINT(42.17480156525865 -87.73477866059046) bank157832 +157833 POINT(42.784211574882974 -88.26966348721693) bank157833 +157834 POINT(40.918291653906486 -87.81495974937727) bank157834 +157835 POINT(41.90085136943033 -88.25991815750116) bank157835 +157836 POINT(41.99312747675176 -88.39199253881077) bank157836 +157837 POINT(41.649481519677245 -87.09724098433496) bank157837 +157838 POINT(42.169707916209966 -88.59733065604982) bank157838 +157839 POINT(41.90518309498829 -88.49657357998585) bank157839 +157840 POINT(41.56589937235968 -88.10872028694338) bank157840 +157841 POINT(42.58438119875461 -87.34343344176676) bank157841 +157842 POINT(42.40403167551457 -88.2972647367763) bank157842 +157843 POINT(42.14649214054137 -87.30308700018892) bank157843 +157844 POINT(41.80206839192366 -88.42783239666262) bank157844 +157845 POINT(42.31163377296049 -87.2382223127449) bank157845 +157846 POINT(41.35503142153717 -88.46683314443172) bank157846 +157847 POINT(42.66795281272072 -87.78078534571328) bank157847 +157848 POINT(40.981229626007355 -88.12495477878612) bank157848 +157849 POINT(41.506428171574555 -87.58958959153438) bank157849 +157850 POINT(42.08661763919709 -87.40575362740282) bank157850 +157851 POINT(42.82438704134539 -87.44530165280473) bank157851 +157852 POINT(42.4354577562775 -87.26981887853769) bank157852 +157853 POINT(41.13221875952395 -88.44990299169841) bank157853 +157854 POINT(42.20106614475694 -88.38374198083504) bank157854 +157855 POINT(42.077303868651 -86.89407525777678) bank157855 +157856 POINT(41.81534457778169 -88.00157861597353) bank157856 +157857 POINT(41.99099552547247 -86.90642272732138) bank157857 +157858 POINT(41.97305306020088 -87.91291722998287) bank157858 +157859 POINT(42.67145423381238 -86.83640077709266) bank157859 +157860 POINT(42.497779911731435 -86.70254627651235) bank157860 +157861 POINT(42.47792365295878 -87.91847520517324) bank157861 +157862 POINT(41.760791239715246 -88.54136645336081) bank157862 +157863 POINT(41.60902356578406 -87.69879669337809) bank157863 +157864 POINT(42.70438361242101 -87.816462824581) bank157864 +157865 POINT(40.89191068210775 -88.55159597033385) bank157865 +157866 POINT(42.81634536435821 -87.53049669690826) bank157866 +157867 POINT(41.114221001623406 -87.16746111824605) bank157867 +157868 POINT(42.41117476293466 -87.57658007447047) bank157868 +157869 POINT(41.48123212521084 -87.16700682840934) bank157869 +157870 POINT(42.74861297458825 -87.81969347306185) bank157870 +157871 POINT(41.032856922665076 -86.99033326503488) bank157871 +157872 POINT(41.74155183357894 -87.04045232468394) bank157872 +157873 POINT(41.12909585959234 -88.28293484410288) bank157873 +157874 POINT(42.7741266289514 -88.62008753287498) bank157874 +157875 POINT(42.51831376325949 -86.9216313987477) bank157875 +157876 POINT(41.89309025199666 -88.2372528193564) bank157876 +157877 POINT(41.493886830285035 -86.87967213486266) bank157877 +157878 POINT(41.89018070785152 -86.93676623476226) bank157878 +157879 POINT(42.71507888555119 -86.97071790287536) bank157879 +157880 POINT(41.38923366656855 -87.6382306471624) bank157880 +157881 POINT(41.864740071428834 -86.6611755033466) bank157881 +157882 POINT(41.0343823572251 -86.8216466090968) bank157882 +157883 POINT(41.725382080525264 -87.07568878129662) bank157883 +157884 POINT(41.696139860269206 -86.64576783988116) bank157884 +157885 POINT(41.60660852278871 -88.39964371940572) bank157885 +157886 POINT(40.989693732281815 -87.58089294197539) bank157886 +157887 POINT(41.28166888986162 -88.30479008779233) bank157887 +157888 POINT(41.533878541402146 -87.96916913760619) bank157888 +157889 POINT(41.139691184970516 -88.11076333162721) bank157889 +157890 POINT(42.15565844203 -87.298228489291) bank157890 +157891 POINT(40.92560891828842 -87.04699674010425) bank157891 +157892 POINT(41.269903787154206 -87.29944734207271) bank157892 +157893 POINT(42.674902408634445 -87.51622944192157) bank157893 +157894 POINT(42.112830130911945 -88.36984097324486) bank157894 +157895 POINT(42.77737666680754 -86.80524400763542) bank157895 +157896 POINT(42.10111119792472 -88.35301918499633) bank157896 +157897 POINT(41.22893955343019 -87.74861981470289) bank157897 +157898 POINT(42.10947771231432 -87.37177637446081) bank157898 +157899 POINT(42.344005282801035 -87.78891805137154) bank157899 +157900 POINT(41.47492902462918 -87.97113572249798) bank157900 +157901 POINT(42.77314177330047 -87.66493357837743) bank157901 +157902 POINT(41.78343064434342 -87.5740143855624) bank157902 +157903 POINT(42.18539444737864 -88.51359705799874) bank157903 +157904 POINT(41.57394491181002 -86.96144126927888) bank157904 +157905 POINT(41.78841299570402 -87.6039807623403) bank157905 +157906 POINT(41.056930138658615 -87.58704211227463) bank157906 +157907 POINT(40.98157755787947 -87.39576030751655) bank157907 +157908 POINT(41.694736111121394 -88.52548883001651) bank157908 +157909 POINT(41.35976117692268 -87.86018970771725) bank157909 +157910 POINT(41.31166299205624 -87.19470268540582) bank157910 +157911 POINT(41.95090996965453 -86.73876757557954) bank157911 +157912 POINT(40.99993461145858 -88.50284956694485) bank157912 +157913 POINT(42.19373282776199 -86.75034057190416) bank157913 +157914 POINT(42.08826715651796 -88.48799873006683) bank157914 +157915 POINT(41.817242150031284 -88.21541284721526) bank157915 +157916 POINT(42.214857248347094 -86.68999236762502) bank157916 +157917 POINT(42.15214139340637 -86.90390996623493) bank157917 +157918 POINT(42.228628638658556 -87.8719908140159) bank157918 +157919 POINT(40.892297196732066 -87.30583197473372) bank157919 +157920 POINT(42.714263374113656 -87.57661121167894) bank157920 +157921 POINT(41.22719199489135 -88.18108929117493) bank157921 +157922 POINT(42.3549715431123 -87.95623759602987) bank157922 +157923 POINT(41.75246156284421 -88.58590949487811) bank157923 +157924 POINT(42.19396960402715 -87.0292817730817) bank157924 +157925 POINT(40.93409088210633 -87.6228735122734) bank157925 +157926 POINT(41.04370912448418 -86.7227233944796) bank157926 +157927 POINT(40.992526292994434 -87.103423606329) bank157927 +157928 POINT(41.57286040614259 -87.81817683923367) bank157928 +157929 POINT(41.55993622628522 -86.82038259269709) bank157929 +157930 POINT(42.56324931366093 -88.01695908021065) bank157930 +157931 POINT(42.497115498395104 -86.95899279320516) bank157931 +157932 POINT(42.73019972293095 -88.11682561737626) bank157932 +157933 POINT(42.32603175286961 -88.53574603958457) bank157933 +157934 POINT(41.700095979587466 -86.72189970176471) bank157934 +157935 POINT(41.12907618168613 -86.7878545620505) bank157935 +157936 POINT(41.96159847057545 -86.68874296263769) bank157936 +157937 POINT(42.42038453028982 -87.7048924786746) bank157937 +157938 POINT(41.27859607603585 -86.7792996836939) bank157938 +157939 POINT(42.31528159160107 -87.35573311622728) bank157939 +157940 POINT(42.329979271818104 -88.30286554146198) bank157940 +157941 POINT(40.93691578700638 -87.98811475044958) bank157941 +157942 POINT(41.14051019492521 -87.78475060182438) bank157942 +157943 POINT(40.98438845742299 -88.54909567633649) bank157943 +157944 POINT(42.12516277767392 -87.34527045717255) bank157944 +157945 POINT(41.432630164025255 -86.88086994992494) bank157945 +157946 POINT(42.454901293227344 -88.3225509317662) bank157946 +157947 POINT(40.96135711797116 -87.19006784772554) bank157947 +157948 POINT(42.61211342319123 -86.90466044094003) bank157948 +157949 POINT(42.76561481160018 -87.86797422321806) bank157949 +157950 POINT(41.79804347252465 -88.06118144326832) bank157950 +157951 POINT(41.81528387905591 -87.55522716249688) bank157951 +157952 POINT(41.99129747955803 -86.96230562398084) bank157952 +157953 POINT(42.26953616152336 -88.57911605641783) bank157953 +157954 POINT(42.51832554850099 -87.01780584743382) bank157954 +157955 POINT(42.69719264354601 -87.28702655761387) bank157955 +157956 POINT(41.01351169260141 -86.63248242987538) bank157956 +157957 POINT(42.513053304466325 -87.12960450294474) bank157957 +157958 POINT(41.72744291928652 -87.48437265087158) bank157958 +157959 POINT(41.116051807740185 -87.09074170091905) bank157959 +157960 POINT(41.76678440215072 -88.1541728348785) bank157960 +157961 POINT(42.44712657110233 -87.78210983525194) bank157961 +157962 POINT(41.319826758163074 -88.1006924884553) bank157962 +157963 POINT(42.125627780767644 -87.07298956164604) bank157963 +157964 POINT(41.4044828747352 -87.47744606909544) bank157964 +157965 POINT(41.28712830205975 -87.48347244793999) bank157965 +157966 POINT(41.306171258418715 -88.49700075538607) bank157966 +157967 POINT(42.073171053361584 -88.08909965350357) bank157967 +157968 POINT(41.87768781140676 -87.08686457710706) bank157968 +157969 POINT(41.79966244892885 -87.463581120056) bank157969 +157970 POINT(42.4385555658564 -87.75541875416917) bank157970 +157971 POINT(41.405680101627524 -86.90660138299218) bank157971 +157972 POINT(42.144263891526116 -87.80646901242956) bank157972 +157973 POINT(41.44675565344769 -88.50359942329092) bank157973 +157974 POINT(42.22766497880674 -87.28841942772296) bank157974 +157975 POINT(41.84165850089337 -87.32160506114442) bank157975 +157976 POINT(42.494983197100986 -87.07175951688778) bank157976 +157977 POINT(42.546913262475414 -88.56867326856388) bank157977 +157978 POINT(41.42697835156158 -87.36693501229524) bank157978 +157979 POINT(40.888274725228435 -87.27805600635334) bank157979 +157980 POINT(42.356837054625224 -87.5057247831112) bank157980 +157981 POINT(42.30523561399858 -86.69131584450132) bank157981 +157982 POINT(42.56462311995255 -86.74100524534991) bank157982 +157983 POINT(42.579436672443535 -88.35614937484385) bank157983 +157984 POINT(42.55897448408852 -88.0348495522933) bank157984 +157985 POINT(42.6982216011494 -87.3545557658742) bank157985 +157986 POINT(41.20960869543243 -88.59050582768124) bank157986 +157987 POINT(42.29912570966171 -87.30584258075378) bank157987 +157988 POINT(41.12887667653084 -88.17888663299415) bank157988 +157989 POINT(41.43946237087273 -88.04729081272994) bank157989 +157990 POINT(42.45436672911737 -87.90293993378965) bank157990 +157991 POINT(42.03884195277586 -87.19482840147067) bank157991 +157992 POINT(42.4719857818655 -87.35366331852248) bank157992 +157993 POINT(42.04972780764569 -88.0219482035185) bank157993 +157994 POINT(41.78226550804661 -88.52224963993604) bank157994 +157995 POINT(41.84611346060033 -88.28900984524606) bank157995 +157996 POINT(42.79288758234543 -86.86592071181566) bank157996 +157997 POINT(41.6565093654192 -86.80883017609149) bank157997 +157998 POINT(41.50068514663533 -87.61978651290096) bank157998 +157999 POINT(42.54030760031027 -87.84967559673878) bank157999 +158000 POINT(42.21813812103289 -87.7933462154967) bank158000 +158001 POINT(40.92334312702201 -88.3806601530689) bank158001 +158002 POINT(42.87129824916762 -88.18149796634653) bank158002 +158003 POINT(41.637188820762695 -86.66125630220023) bank158003 +158004 POINT(41.5349827683003 -87.78095312053733) bank158004 +158005 POINT(41.01341226198925 -87.83077349625226) bank158005 +158006 POINT(42.278686927596645 -86.79548817652314) bank158006 +158007 POINT(42.14538564036849 -87.2684773899988) bank158007 +158008 POINT(41.74635163474481 -87.07847573575238) bank158008 +158009 POINT(42.65225014261985 -87.72528164753956) bank158009 +158010 POINT(41.734975092012434 -87.85947676326731) bank158010 +158011 POINT(42.72432620271657 -88.5193605709602) bank158011 +158012 POINT(42.63796000200621 -87.12896337198652) bank158012 +158013 POINT(42.6949585408147 -88.1999804808432) bank158013 +158014 POINT(41.03586868356693 -87.05708842812791) bank158014 +158015 POINT(42.57383529216462 -87.32291563162862) bank158015 +158016 POINT(42.83528357985956 -86.90872667389849) bank158016 +158017 POINT(42.13735562353266 -88.11613610860249) bank158017 +158018 POINT(42.22367431525805 -87.450758868456) bank158018 +158019 POINT(42.8245056493686 -87.81410094937762) bank158019 +158020 POINT(42.290840886647935 -86.8992788738049) bank158020 +158021 POINT(41.25870758115257 -87.4107246512247) bank158021 +158022 POINT(42.492718131018854 -87.27988430823946) bank158022 +158023 POINT(42.511790088076054 -87.05803648976382) bank158023 +158024 POINT(41.10921598061578 -87.81833586232543) bank158024 +158025 POINT(40.93686475647273 -88.19637203260667) bank158025 +158026 POINT(41.02568151986798 -86.92960579960825) bank158026 +158027 POINT(42.48330656672566 -88.61062995558962) bank158027 +158028 POINT(41.73589534470934 -88.60652729970595) bank158028 +158029 POINT(41.38381491521054 -88.33114456584171) bank158029 +158030 POINT(41.75681836059973 -88.06441376272994) bank158030 +158031 POINT(41.22444872055771 -88.2724568373002) bank158031 +158032 POINT(42.33358274367883 -86.78674008759228) bank158032 +158033 POINT(42.64305618925198 -87.02247920256123) bank158033 +158034 POINT(41.75607247443669 -87.47356666548707) bank158034 +158035 POINT(40.982471640217824 -87.49952478674163) bank158035 +158036 POINT(41.49631692198363 -88.42647172515704) bank158036 +158037 POINT(42.6628202053406 -86.96790736156079) bank158037 +158038 POINT(40.99452988148844 -88.22733850453923) bank158038 +158039 POINT(41.99646519048652 -86.88013464470154) bank158039 +158040 POINT(40.88243069473642 -88.30350015306792) bank158040 +158041 POINT(41.814126160378365 -87.56195822041076) bank158041 +158042 POINT(42.567271149703245 -87.65118548377494) bank158042 +158043 POINT(42.25600279736888 -87.37475189930237) bank158043 +158044 POINT(41.23521237329848 -87.73806321666832) bank158044 +158045 POINT(42.03412540352075 -87.51868051510098) bank158045 +158046 POINT(41.47656968419118 -88.43699099048625) bank158046 +158047 POINT(41.668061271564746 -87.3213871630871) bank158047 +158048 POINT(42.472966742810776 -88.07292321494847) bank158048 +158049 POINT(42.73178285510677 -87.03210061058856) bank158049 +158050 POINT(41.45492868315645 -86.884592639243) bank158050 +158051 POINT(41.97314298702372 -87.75979196163537) bank158051 +158052 POINT(41.10118215202443 -87.6928489674859) bank158052 +158053 POINT(41.61994629453659 -86.78517769690049) bank158053 +158054 POINT(41.85713174542593 -87.06830015177316) bank158054 +158055 POINT(41.12849003499262 -88.34791394483693) bank158055 +158056 POINT(41.01619588127946 -88.31679155140034) bank158056 +158057 POINT(41.23907647270474 -88.52204602067725) bank158057 +158058 POINT(42.28819039387312 -87.58541440268398) bank158058 +158059 POINT(42.781781576756174 -87.12544613569517) bank158059 +158060 POINT(41.852004016474204 -87.19959081615728) bank158060 +158061 POINT(42.686916112631444 -87.08201863849659) bank158061 +158062 POINT(41.726867676339985 -87.9350751103661) bank158062 +158063 POINT(41.48207284428023 -88.20320810942255) bank158063 +158064 POINT(42.15384829908013 -86.84504179000496) bank158064 +158065 POINT(42.06555878717227 -87.20858709297526) bank158065 +158066 POINT(41.335279157562525 -87.67488741393964) bank158066 +158067 POINT(42.79124240708215 -87.67924560875984) bank158067 +158068 POINT(40.91977108302446 -87.90731070409308) bank158068 +158069 POINT(41.47597876477828 -88.34437171464451) bank158069 +158070 POINT(41.31599834194569 -88.1265111250397) bank158070 +158071 POINT(42.24350306190414 -87.73089537297272) bank158071 +158072 POINT(42.089363273171564 -86.69399289748253) bank158072 +158073 POINT(42.33077310058913 -87.01482261184738) bank158073 +158074 POINT(41.19363535887777 -86.81213570179034) bank158074 +158075 POINT(42.79899000200825 -87.01958174860364) bank158075 +158076 POINT(41.053169378917126 -87.9080003915243) bank158076 +158077 POINT(42.58590399728072 -87.57010153689508) bank158077 +158078 POINT(41.10131489775315 -88.39244770886613) bank158078 +158079 POINT(42.0289641098918 -88.52194009115581) bank158079 +158080 POINT(42.723175454313214 -88.35859613718226) bank158080 +158081 POINT(42.66492930489098 -88.26461967038358) bank158081 +158082 POINT(42.87601252459758 -87.36843369660309) bank158082 +158083 POINT(41.88304165281259 -87.96638529442227) bank158083 +158084 POINT(42.73810712751929 -87.74690167917892) bank158084 +158085 POINT(42.705266269747035 -87.25848316668419) bank158085 +158086 POINT(41.34279156264329 -87.88307475660137) bank158086 +158087 POINT(41.02974107112047 -87.4105234143196) bank158087 +158088 POINT(42.70926116484859 -87.64759642929775) bank158088 +158089 POINT(41.23733746123184 -87.57102354175632) bank158089 +158090 POINT(41.353750670883784 -87.48457156404695) bank158090 +158091 POINT(42.52877957769929 -87.87039609220363) bank158091 +158092 POINT(41.15628108158076 -88.0880438343975) bank158092 +158093 POINT(41.433403153114064 -87.71374661897708) bank158093 +158094 POINT(41.71165445659081 -87.88513044594603) bank158094 +158095 POINT(40.91795342024883 -88.09502467152312) bank158095 +158096 POINT(42.57868523972488 -87.83915257873) bank158096 +158097 POINT(42.55033315476607 -87.31589351936015) bank158097 +158098 POINT(41.485959162702606 -88.56205712868642) bank158098 +158099 POINT(42.21245481998398 -86.69761972945209) bank158099 +158100 POINT(42.240709728582466 -87.9047869303009) bank158100 +158101 POINT(41.69460403231757 -88.44452822815103) bank158101 +158102 POINT(41.173769036565915 -86.68457007136666) bank158102 +158103 POINT(42.288377117010434 -88.61316399442109) bank158103 +158104 POINT(42.38930413957268 -86.79646618044382) bank158104 +158105 POINT(42.414399482064596 -87.9122161540307) bank158105 +158106 POINT(41.06171476223563 -86.69491138621655) bank158106 +158107 POINT(40.999914207587075 -88.61302435296953) bank158107 +158108 POINT(42.18574932916125 -88.13524048757034) bank158108 +158109 POINT(41.50425611853146 -87.19139061727792) bank158109 +158110 POINT(41.34330215539033 -87.57534153767503) bank158110 +158111 POINT(42.15841116852095 -87.93491607516377) bank158111 +158112 POINT(41.74247298121368 -87.68171262327826) bank158112 +158113 POINT(41.8281314127172 -88.4170672559437) bank158113 +158114 POINT(41.868999448363475 -87.7620514436783) bank158114 +158115 POINT(42.517518287713216 -86.90564676559342) bank158115 +158116 POINT(41.50312282084209 -87.77724766348392) bank158116 +158117 POINT(41.20116929041058 -87.06427393796696) bank158117 +158118 POINT(42.31129592074339 -87.87196361680763) bank158118 +158119 POINT(42.7457209646144 -88.37041334266841) bank158119 +158120 POINT(41.786759735121265 -86.83519809853819) bank158120 +158121 POINT(42.154317224885055 -87.28788245274689) bank158121 +158122 POINT(41.69320279778817 -87.11196756102879) bank158122 +158123 POINT(42.172034200999136 -88.57254209883666) bank158123 +158124 POINT(41.15008928926008 -88.35767572557921) bank158124 +158125 POINT(42.77232905375069 -88.40683423594395) bank158125 +158126 POINT(41.069751440750295 -87.25381739710232) bank158126 +158127 POINT(41.880919689119665 -87.41217435964303) bank158127 +158128 POINT(42.85522471878781 -88.18741972863802) bank158128 +158129 POINT(42.368936080863676 -87.18961244426823) bank158129 +158130 POINT(42.13859602239798 -88.59615432816433) bank158130 +158131 POINT(40.88997200001436 -88.5772854331594) bank158131 +158132 POINT(40.91574009831341 -87.89987076363424) bank158132 +158133 POINT(42.864199205342295 -87.79393988131478) bank158133 +158134 POINT(41.98540950518876 -88.17563224119426) bank158134 +158135 POINT(41.91704206887435 -87.10558625717184) bank158135 +158136 POINT(42.36261267818289 -87.26838060830772) bank158136 +158137 POINT(41.415641503583196 -88.26549298921769) bank158137 +158138 POINT(42.097134870822316 -86.94953159424618) bank158138 +158139 POINT(41.978061041912525 -88.50832158562336) bank158139 +158140 POINT(41.97588361003168 -86.74737979683476) bank158140 +158141 POINT(41.77499337430533 -86.75904311929642) bank158141 +158142 POINT(42.69176388978024 -88.60312656145041) bank158142 +158143 POINT(42.119473240630846 -87.85312514849335) bank158143 +158144 POINT(42.593993791929876 -87.04143445578885) bank158144 +158145 POINT(42.19982950874855 -87.24616866511543) bank158145 +158146 POINT(42.321958207462984 -87.83466384687351) bank158146 +158147 POINT(42.3004847208627 -87.57668273773808) bank158147 +158148 POINT(41.83616324318304 -88.54950438205996) bank158148 +158149 POINT(41.811762823381805 -87.35043842691636) bank158149 +158150 POINT(41.275144814749744 -87.02820226791911) bank158150 +158151 POINT(41.89059213601985 -87.15196468742779) bank158151 +158152 POINT(41.60742189282854 -87.60178214301035) bank158152 +158153 POINT(41.21325311290623 -88.46135801084883) bank158153 +158154 POINT(41.49796899520263 -88.28565311770882) bank158154 +158155 POINT(41.816411018782325 -86.78486885833068) bank158155 +158156 POINT(41.79975781028922 -88.150636740086) bank158156 +158157 POINT(42.33464045015283 -87.89217178922381) bank158157 +158158 POINT(41.81309430932385 -86.7156518570448) bank158158 +158159 POINT(42.29760533351234 -88.3494302792893) bank158159 +158160 POINT(41.42563358502164 -87.25868195346332) bank158160 +158161 POINT(41.55956068569571 -88.0222540218837) bank158161 +158162 POINT(42.58219067800471 -87.49108035987045) bank158162 +158163 POINT(42.77232655873144 -88.02181048360005) bank158163 +158164 POINT(40.98214058791193 -88.56262296976543) bank158164 +158165 POINT(41.19799361512926 -87.05205102113072) bank158165 +158166 POINT(42.20605131257206 -87.0314321036879) bank158166 +158167 POINT(42.14977111547803 -86.91095134625567) bank158167 +158168 POINT(42.57670249626489 -88.21947218534812) bank158168 +158169 POINT(41.63543571359773 -87.15756169859262) bank158169 +158170 POINT(40.94998506373921 -88.37821117209441) bank158170 +158171 POINT(41.64205912935937 -88.07151883653744) bank158171 +158172 POINT(41.97615260361663 -87.6051017076335) bank158172 +158173 POINT(42.40494312079583 -87.95162514189312) bank158173 +158174 POINT(42.83033018544184 -88.46175279763887) bank158174 +158175 POINT(40.88630329991598 -87.29784804957595) bank158175 +158176 POINT(42.814685380990575 -87.09779826771266) bank158176 +158177 POINT(41.33442119235417 -86.7781789386899) bank158177 +158178 POINT(42.74435077855565 -88.57929931859161) bank158178 +158179 POINT(41.37370142820813 -88.57732511208505) bank158179 +158180 POINT(42.15981066124211 -87.07182047898307) bank158180 +158181 POINT(42.385174523535895 -87.87950635261352) bank158181 +158182 POINT(40.891504771937115 -88.48437368155562) bank158182 +158183 POINT(41.26879015172396 -87.98295137194651) bank158183 +158184 POINT(41.01719581206104 -86.99686589620073) bank158184 +158185 POINT(42.10512611839685 -88.10789555765795) bank158185 +158186 POINT(41.45973351572166 -87.57682716933552) bank158186 +158187 POINT(41.018975114616374 -86.87608909309868) bank158187 +158188 POINT(42.37200634101244 -87.83402787122901) bank158188 +158189 POINT(42.66472558019741 -87.44701674023898) bank158189 +158190 POINT(41.20958917524609 -87.21403578871929) bank158190 +158191 POINT(42.56390133309157 -86.70288764421058) bank158191 +158192 POINT(41.6547786782485 -87.76040814114015) bank158192 +158193 POINT(41.129119044926675 -86.77515647584396) bank158193 +158194 POINT(42.818210363639196 -87.16132830243451) bank158194 +158195 POINT(41.9312541626042 -86.63352319307607) bank158195 +158196 POINT(40.8824178798383 -88.29585691956518) bank158196 +158197 POINT(41.7906107826513 -88.19516181166257) bank158197 +158198 POINT(41.20675814369204 -87.31070564172133) bank158198 +158199 POINT(41.29714041306694 -88.32081910284859) bank158199 +158200 POINT(42.03935967128055 -87.32926422223204) bank158200 +158201 POINT(41.30896478511512 -87.06470014701526) bank158201 +158202 POINT(41.99750778812125 -86.87654509974598) bank158202 +158203 POINT(42.24508765089849 -87.47717951640028) bank158203 +158204 POINT(41.38903320773393 -87.84411127566618) bank158204 +158205 POINT(41.90811516124263 -86.65394991343067) bank158205 +158206 POINT(42.09471098711431 -86.89489694414293) bank158206 +158207 POINT(42.694765783233834 -88.33767490747074) bank158207 +158208 POINT(42.16963804046519 -88.27631014317927) bank158208 +158209 POINT(41.12291893787143 -87.16874198294114) bank158209 +158210 POINT(41.45314530821659 -87.5742314023661) bank158210 +158211 POINT(41.092584960045116 -87.88328775143808) bank158211 +158212 POINT(42.61549394644772 -87.76992924524498) bank158212 +158213 POINT(41.170105928043085 -88.26662664520164) bank158213 +158214 POINT(41.78297809396115 -86.63686472031972) bank158214 +158215 POINT(42.796785477857874 -87.74639439916544) bank158215 +158216 POINT(40.95224477772701 -87.86619496425263) bank158216 +158217 POINT(42.44718570023622 -87.27554881337808) bank158217 +158218 POINT(42.14363058832093 -87.45291654413725) bank158218 +158219 POINT(41.4055205989605 -87.31411449531612) bank158219 +158220 POINT(42.69148904419646 -87.48672701343537) bank158220 +158221 POINT(41.178740073284835 -86.75945525510417) bank158221 +158222 POINT(41.2960910316281 -87.0323914887444) bank158222 +158223 POINT(41.471022958130824 -86.91857759640753) bank158223 +158224 POINT(42.86293555886325 -87.54856970382647) bank158224 +158225 POINT(41.76956331507597 -88.53960813194811) bank158225 +158226 POINT(41.74910581838607 -88.1381757555052) bank158226 +158227 POINT(40.90039769148937 -87.65897832373582) bank158227 +158228 POINT(42.764718237489774 -87.11959599664814) bank158228 +158229 POINT(41.83196782510926 -87.87222431063368) bank158229 +158230 POINT(41.877782173821075 -86.87153757266809) bank158230 +158231 POINT(40.958084399557215 -88.28435271182937) bank158231 +158232 POINT(41.7333313919691 -88.18267617662575) bank158232 +158233 POINT(42.10420857526283 -88.00570857668511) bank158233 +158234 POINT(42.84915786309066 -88.43378158915523) bank158234 +158235 POINT(42.781007522762394 -88.21187688243258) bank158235 +158236 POINT(41.96260739752877 -86.75570553992365) bank158236 +158237 POINT(42.52517107444534 -88.4265489003166) bank158237 +158238 POINT(42.691219479493434 -87.6570368178043) bank158238 +158239 POINT(42.431338058165984 -88.50285522887661) bank158239 +158240 POINT(41.142657075255414 -88.4847021332167) bank158240 +158241 POINT(40.9827674951039 -87.82490992454275) bank158241 +158242 POINT(40.941195688180215 -88.43839708614429) bank158242 +158243 POINT(40.99501135734821 -87.28633092902494) bank158243 +158244 POINT(41.21108613426305 -87.06328567609532) bank158244 +158245 POINT(41.65410070281017 -87.40752340797698) bank158245 +158246 POINT(41.01779778820346 -86.77119215800634) bank158246 +158247 POINT(41.76835351717271 -86.70378065901842) bank158247 +158248 POINT(42.77497127959141 -87.35060311185724) bank158248 +158249 POINT(41.59760696710355 -87.62965122914788) bank158249 +158250 POINT(41.650005796806106 -86.80746936725114) bank158250 +158251 POINT(41.38204200544217 -88.22539140167956) bank158251 +158252 POINT(42.6022279455866 -87.98043382853088) bank158252 +158253 POINT(41.325477391216005 -87.35809937202131) bank158253 +158254 POINT(41.53967153499488 -88.15239544198816) bank158254 +158255 POINT(41.948026278304226 -88.32154504571248) bank158255 +158256 POINT(41.59296880604823 -87.08127283897687) bank158256 +158257 POINT(41.21846789487777 -87.4165722778677) bank158257 +158258 POINT(41.938697771206385 -87.57751037089183) bank158258 +158259 POINT(42.1594525962258 -86.97698103617516) bank158259 +158260 POINT(40.921145697369 -88.13003396933635) bank158260 +158261 POINT(42.82870397511564 -87.20883233990814) bank158261 +158262 POINT(41.634858819146345 -86.64987895611742) bank158262 +158263 POINT(41.33772531065842 -87.55816007354164) bank158263 +158264 POINT(41.54783955697754 -86.87771817117029) bank158264 +158265 POINT(41.599251637163036 -86.98034321549251) bank158265 +158266 POINT(41.10889776572593 -87.34220457520132) bank158266 +158267 POINT(41.49479522535153 -86.91050222151938) bank158267 +158268 POINT(41.41470174779598 -87.87866987323001) bank158268 +158269 POINT(42.42268442912232 -87.61869708441522) bank158269 +158270 POINT(41.80406239888053 -87.03878963470152) bank158270 +158271 POINT(41.84807666283204 -87.1125242232196) bank158271 +158272 POINT(42.29012631437443 -88.09661628903413) bank158272 +158273 POINT(42.61190305062295 -87.16221648985616) bank158273 +158274 POINT(42.06415038230736 -88.25950772634106) bank158274 +158275 POINT(41.85303603584915 -87.85026780738458) bank158275 +158276 POINT(42.601629427904044 -87.34834260975039) bank158276 +158277 POINT(41.32723278791525 -88.39793727892425) bank158277 +158278 POINT(41.221635415604965 -87.8640741614514) bank158278 +158279 POINT(42.21460501321144 -87.78662403268892) bank158279 +158280 POINT(42.02947303001182 -86.9185044745485) bank158280 +158281 POINT(41.12727280126228 -88.39601293087887) bank158281 +158282 POINT(41.18054181339672 -87.50308435466809) bank158282 +158283 POINT(42.16247679302551 -87.06404686023902) bank158283 +158284 POINT(41.49708677292993 -87.95964152278718) bank158284 +158285 POINT(42.05359554056506 -87.55503808807345) bank158285 +158286 POINT(41.29603689301405 -88.61142600735046) bank158286 +158287 POINT(41.298628289356465 -86.90001192915864) bank158287 +158288 POINT(41.23513242691287 -87.9095266058624) bank158288 +158289 POINT(42.232123750263916 -88.23411632773681) bank158289 +158290 POINT(42.42600078648433 -87.72104797615401) bank158290 +158291 POINT(41.604486748361815 -87.27558959052561) bank158291 +158292 POINT(41.0055848640721 -87.76674742753443) bank158292 +158293 POINT(42.16525795446621 -87.45286704034785) bank158293 +158294 POINT(41.79281590688803 -87.25083023473616) bank158294 +158295 POINT(42.222308756415266 -87.06023611103915) bank158295 +158296 POINT(41.957504541858896 -88.300267799505) bank158296 +158297 POINT(42.72502753692739 -87.98959435827861) bank158297 +158298 POINT(42.12291268168032 -87.128620318591) bank158298 +158299 POINT(41.37904691623554 -87.51121459447542) bank158299 +158300 POINT(41.17398573143646 -87.6848201839821) bank158300 +158301 POINT(42.209177335593175 -87.22831570221804) bank158301 +158302 POINT(42.52747675337551 -86.73367716775041) bank158302 +158303 POINT(42.31123231339141 -88.57324476506916) bank158303 +158304 POINT(42.170558192146046 -87.48296529921413) bank158304 +158305 POINT(42.518819262971405 -87.96690367585458) bank158305 +158306 POINT(41.43649281570927 -88.25397531355708) bank158306 +158307 POINT(41.98312519845117 -87.080140849829) bank158307 +158308 POINT(41.9543820874101 -88.57620771102312) bank158308 +158309 POINT(41.94532281411077 -87.88890775164377) bank158309 +158310 POINT(41.528581127456576 -87.0442964254191) bank158310 +158311 POINT(42.14066901150561 -88.55157179320757) bank158311 +158312 POINT(42.67090493682855 -86.9114216973981) bank158312 +158313 POINT(42.64503719543447 -86.63328252965235) bank158313 +158314 POINT(41.96390939675196 -87.5250939037037) bank158314 +158315 POINT(41.49143059671083 -87.4334445586524) bank158315 +158316 POINT(42.32090044539458 -87.30293923337334) bank158316 +158317 POINT(41.53622904367443 -86.63727849888252) bank158317 +158318 POINT(41.27700631632163 -87.5973898838176) bank158318 +158319 POINT(41.66857856344813 -88.34765090869331) bank158319 +158320 POINT(41.58993860490207 -86.63869530412137) bank158320 +158321 POINT(42.5439679817186 -87.7494596798854) bank158321 +158322 POINT(40.895534616070215 -87.0736009219382) bank158322 +158323 POINT(42.615545795587586 -86.92141963500362) bank158323 +158324 POINT(41.25656068262045 -86.8402268214881) bank158324 +158325 POINT(42.36884845123594 -86.86915477478212) bank158325 +158326 POINT(42.31376752835911 -88.24005216139386) bank158326 +158327 POINT(42.46502882106444 -87.94037996203069) bank158327 +158328 POINT(42.14300116412251 -86.6613272716815) bank158328 +158329 POINT(41.773268077807685 -87.55061390679201) bank158329 +158330 POINT(41.95428086220418 -87.8216603065894) bank158330 +158331 POINT(42.06529955820602 -87.4587702450446) bank158331 +158332 POINT(40.91752129219855 -86.71703850518723) bank158332 +158333 POINT(42.09296931593718 -86.75705456132268) bank158333 +158334 POINT(42.45601619941526 -88.43146733346217) bank158334 +158335 POINT(42.04378804331319 -88.36676414903054) bank158335 +158336 POINT(42.33933137060563 -87.87125170173354) bank158336 +158337 POINT(42.08390958228108 -87.57536561798463) bank158337 +158338 POINT(42.308295523068104 -87.72870188969905) bank158338 +158339 POINT(42.76653892790746 -88.00543217565787) bank158339 +158340 POINT(41.875533942148465 -88.39051249237059) bank158340 +158341 POINT(41.37111577930136 -86.99196408970916) bank158341 +158342 POINT(42.45884000872002 -87.47983208941815) bank158342 +158343 POINT(40.9979790147665 -87.47185819858457) bank158343 +158344 POINT(40.88666358097553 -88.53277087232074) bank158344 +158345 POINT(42.5258992318474 -88.29081903276591) bank158345 +158346 POINT(42.47180266624199 -86.94790327184012) bank158346 +158347 POINT(41.97582430716773 -87.52328628312021) bank158347 +158348 POINT(42.01879968369609 -87.58575969509944) bank158348 +158349 POINT(42.79576484108347 -87.86600478684691) bank158349 +158350 POINT(42.73610788569201 -86.98532654575561) bank158350 +158351 POINT(42.40231576401552 -87.88588575616053) bank158351 +158352 POINT(41.69439256965528 -87.40736744480513) bank158352 +158353 POINT(41.15813381698111 -87.06725351560497) bank158353 +158354 POINT(41.55215011202936 -87.04502970700143) bank158354 +158355 POINT(41.19101083715243 -87.98472120952002) bank158355 +158356 POINT(42.77066564108027 -87.29226256225054) bank158356 +158357 POINT(40.97071736888128 -86.81642726093065) bank158357 +158358 POINT(41.42493731239667 -88.35625657484886) bank158358 +158359 POINT(41.64663913062088 -88.59545570067924) bank158359 +158360 POINT(42.02943648572924 -88.16152837116017) bank158360 +158361 POINT(41.90661238656145 -88.05794805023973) bank158361 +158362 POINT(42.70927940468024 -87.54901272294676) bank158362 +158363 POINT(42.692820487936864 -87.07463685286523) bank158363 +158364 POINT(41.9058989460702 -87.68480515019112) bank158364 +158365 POINT(42.137458484818865 -86.7555137930841) bank158365 +158366 POINT(41.335268040524475 -87.1553670497527) bank158366 +158367 POINT(41.7585873150988 -87.62650059767374) bank158367 +158368 POINT(41.04174170497513 -88.10735030506996) bank158368 +158369 POINT(41.661137704827915 -87.88087930042718) bank158369 +158370 POINT(41.27232877237305 -86.95050303324572) bank158370 +158371 POINT(42.85795467266633 -88.57159120682792) bank158371 +158372 POINT(41.004558151122644 -88.3953189737949) bank158372 +158373 POINT(42.862092863441305 -86.75123882053352) bank158373 +158374 POINT(41.97839048178968 -87.95522643146684) bank158374 +158375 POINT(42.57085448195634 -87.36206546370613) bank158375 +158376 POINT(42.07688867705121 -87.45287363364453) bank158376 +158377 POINT(42.76115609045794 -86.6631251548022) bank158377 +158378 POINT(41.9884622918809 -88.3446105651104) bank158378 +158379 POINT(40.98484743607154 -86.90094125051203) bank158379 +158380 POINT(41.56090708991338 -87.54055264222923) bank158380 +158381 POINT(42.81564885518725 -86.95433241680217) bank158381 +158382 POINT(41.65158924027935 -87.4976710098987) bank158382 +158383 POINT(41.64952929413804 -87.87285529208026) bank158383 +158384 POINT(41.45876333948308 -88.32594647986285) bank158384 +158385 POINT(42.7131200169574 -87.05179600268058) bank158385 +158386 POINT(42.086059573324285 -87.22227826019609) bank158386 +158387 POINT(41.118038033185606 -87.55533366665482) bank158387 +158388 POINT(42.47320459949776 -88.25226727593125) bank158388 +158389 POINT(42.09579209564427 -86.8513179746445) bank158389 +158390 POINT(42.01899953665776 -88.24572815948649) bank158390 +158391 POINT(41.78398029632421 -87.69260088776059) bank158391 +158392 POINT(42.30463713531155 -88.02596848668641) bank158392 +158393 POINT(42.36031947552992 -88.1262255557632) bank158393 +158394 POINT(42.562700345821 -88.44087032512006) bank158394 +158395 POINT(42.313301303078106 -88.34435031952142) bank158395 +158396 POINT(42.66749395081436 -86.99741203212598) bank158396 +158397 POINT(42.62116836802667 -87.05315962302608) bank158397 +158398 POINT(40.98721515621354 -88.58908307266057) bank158398 +158399 POINT(42.26904063277477 -88.58832211701943) bank158399 +158400 POINT(40.925048308758505 -87.03727981986648) bank158400 +158401 POINT(41.776465387838215 -88.08881708660871) bank158401 +158402 POINT(41.0307343775371 -86.8441966066758) bank158402 +158403 POINT(41.45848416032642 -88.45802558323916) bank158403 +158404 POINT(40.917067421217176 -87.5608588160436) bank158404 +158405 POINT(41.90997637616864 -87.85020520168426) bank158405 +158406 POINT(41.57104102642122 -87.96951318945098) bank158406 +158407 POINT(42.337934671129005 -87.76324373342564) bank158407 +158408 POINT(41.12985024578389 -87.51025601411386) bank158408 +158409 POINT(41.67718042763208 -87.50742733199216) bank158409 +158410 POINT(41.4554530224598 -87.96650339172065) bank158410 +158411 POINT(41.06630108574177 -87.82052430152568) bank158411 +158412 POINT(42.67326418158432 -87.4715227666685) bank158412 +158413 POINT(41.96121905881299 -86.84338707159573) bank158413 +158414 POINT(42.72208274222118 -87.21252261934823) bank158414 +158415 POINT(41.229211178362256 -88.1717269410345) bank158415 +158416 POINT(42.7003082956585 -87.40124859448635) bank158416 +158417 POINT(41.99928223425892 -87.1810489769591) bank158417 +158418 POINT(42.857085049628054 -87.53345787013329) bank158418 +158419 POINT(41.55848690788985 -87.59929653317475) bank158419 +158420 POINT(42.405704746269635 -87.98597221354348) bank158420 +158421 POINT(41.334833307543526 -88.11212263583484) bank158421 +158422 POINT(41.97402960688662 -86.65004051219614) bank158422 +158423 POINT(42.289408422352075 -88.24515312341457) bank158423 +158424 POINT(42.66691037911916 -86.81634990007956) bank158424 +158425 POINT(42.23974538150766 -86.67686361547608) bank158425 +158426 POINT(41.339303294689216 -86.82998836635522) bank158426 +158427 POINT(42.02835868077199 -87.61902206097763) bank158427 +158428 POINT(42.191198390479364 -87.0540327526809) bank158428 +158429 POINT(42.60606521262136 -86.91610048436655) bank158429 +158430 POINT(41.41419080005618 -87.03250305850581) bank158430 +158431 POINT(40.98522185684336 -86.6823785157988) bank158431 +158432 POINT(41.03323041650877 -87.51047403366167) bank158432 +158433 POINT(42.489234035003804 -87.6965112798224) bank158433 +158434 POINT(41.08300700452573 -87.18070724506474) bank158434 +158435 POINT(42.0684190014253 -87.79903591395984) bank158435 +158436 POINT(41.25906069579969 -86.69402954867776) bank158436 +158437 POINT(42.18068044542198 -87.25653937537885) bank158437 +158438 POINT(42.83174926566598 -88.41718508726235) bank158438 +158439 POINT(41.51045935215188 -87.38443367700543) bank158439 +158440 POINT(42.70760895639195 -87.1799767503462) bank158440 +158441 POINT(41.4572460466302 -87.43428206578012) bank158441 +158442 POINT(41.774634946595285 -87.11825232188158) bank158442 +158443 POINT(42.74663724126624 -88.07346805187478) bank158443 +158444 POINT(41.4496978298231 -88.05086329027334) bank158444 +158445 POINT(41.460358202660196 -88.24101240424358) bank158445 +158446 POINT(41.33014674782368 -88.48539340282206) bank158446 +158447 POINT(41.53154845557169 -87.6083603231049) bank158447 +158448 POINT(42.21769732336561 -87.89131415959203) bank158448 +158449 POINT(41.12272137493777 -87.49619631439563) bank158449 +158450 POINT(42.572670644834325 -86.86205902615357) bank158450 +158451 POINT(42.4612401944189 -87.75935856357876) bank158451 +158452 POINT(41.58699921737934 -86.9928042380016) bank158452 +158453 POINT(41.70586602433613 -87.25047819717712) bank158453 +158454 POINT(42.50679326863927 -87.06253873961936) bank158454 +158455 POINT(41.834250622117175 -87.6620642209036) bank158455 +158456 POINT(42.72583414498663 -87.20702151181847) bank158456 +158457 POINT(42.180873188112656 -87.68069960594059) bank158457 +158458 POINT(42.69228792464508 -86.70436432750584) bank158458 +158459 POINT(41.324407530443615 -86.74516979882509) bank158459 +158460 POINT(42.46280583925414 -88.225058937903) bank158460 +158461 POINT(40.9038195854087 -87.34896509605382) bank158461 +158462 POINT(42.34489086629549 -87.34785316304199) bank158462 +158463 POINT(41.14066413068132 -87.45979601550096) bank158463 +158464 POINT(42.41026769731129 -88.58282785220621) bank158464 +158465 POINT(42.05598305983541 -88.39773190096639) bank158465 +158466 POINT(40.97484104985506 -87.333849663418) bank158466 +158467 POINT(42.282705886040155 -88.2542465794651) bank158467 +158468 POINT(41.31324009424298 -87.08745931169105) bank158468 +158469 POINT(41.07737230911159 -87.16424727916026) bank158469 +158470 POINT(41.05970693180224 -87.66056231821766) bank158470 +158471 POINT(41.19281945205963 -87.31107187943546) bank158471 +158472 POINT(41.47690392773189 -88.11524506653654) bank158472 +158473 POINT(42.754155586290544 -87.27228361129474) bank158473 +158474 POINT(42.59644636951195 -87.57913866158118) bank158474 +158475 POINT(42.43398522727317 -88.00234681449223) bank158475 +158476 POINT(42.34835178056506 -87.8984111414937) bank158476 +158477 POINT(42.65572987894948 -86.92752471294529) bank158477 +158478 POINT(41.03832671260252 -87.4065438403903) bank158478 +158479 POINT(42.78484883685196 -87.99610948001234) bank158479 +158480 POINT(41.290808920527944 -86.9526568141746) bank158480 +158481 POINT(42.021488613150304 -88.3566049076554) bank158481 +158482 POINT(41.70597973702599 -87.6604989873597) bank158482 +158483 POINT(42.20772789218133 -86.94661676628037) bank158483 +158484 POINT(41.38361687721435 -87.18234808943956) bank158484 +158485 POINT(42.409322207893396 -86.91163521418588) bank158485 +158486 POINT(42.25589633864237 -87.73611591580969) bank158486 +158487 POINT(41.00343895556384 -86.98347967238267) bank158487 +158488 POINT(42.86184117022809 -87.37643315980817) bank158488 +158489 POINT(41.72994455818467 -88.07437224721423) bank158489 +158490 POINT(41.497777068958996 -87.16188846028571) bank158490 +158491 POINT(41.66341276413069 -88.45161040595974) bank158491 +158492 POINT(41.13540705068198 -86.89138945879299) bank158492 +158493 POINT(42.673390078533785 -88.4723829554466) bank158493 +158494 POINT(42.467299575608166 -88.49383812212292) bank158494 +158495 POINT(42.131680933280556 -86.76575076267741) bank158495 +158496 POINT(42.462029466196796 -88.3734095588015) bank158496 +158497 POINT(42.30702809203271 -88.20268986666451) bank158497 +158498 POINT(42.815019805561896 -87.68601218408621) bank158498 +158499 POINT(42.41215540774296 -87.11259952169249) bank158499 +158500 POINT(42.78019452053013 -87.32454147872762) bank158500 +158501 POINT(42.274227977479036 -88.04238856146519) bank158501 +158502 POINT(42.44416918308837 -88.15051538939164) bank158502 +158503 POINT(42.52736264815449 -88.35447000279994) bank158503 +158504 POINT(42.67579530190141 -86.83862302946471) bank158504 +158505 POINT(42.17807078880009 -87.14764081139765) bank158505 +158506 POINT(41.04347914669864 -88.32974871302021) bank158506 +158507 POINT(41.4174635420184 -87.95568863432433) bank158507 +158508 POINT(41.89214746814318 -88.55305548399423) bank158508 +158509 POINT(42.469909175996 -87.08813394992565) bank158509 +158510 POINT(41.03456182865119 -87.54773665784954) bank158510 +158511 POINT(41.66057367659167 -87.22919979087276) bank158511 +158512 POINT(42.854362469395966 -86.99332868916316) bank158512 +158513 POINT(42.05589051765673 -86.795863918386) bank158513 +158514 POINT(41.29485535923435 -86.82776525556025) bank158514 +158515 POINT(41.48683355673829 -86.76276630417928) bank158515 +158516 POINT(41.27547156651658 -86.64811024411486) bank158516 +158517 POINT(42.2097313064231 -87.62919999395703) bank158517 +158518 POINT(41.600518907700526 -87.49178766002585) bank158518 +158519 POINT(42.38680346719574 -86.96983601436686) bank158519 +158520 POINT(42.49181291556092 -86.96927675238453) bank158520 +158521 POINT(41.29217622179444 -88.17727685512402) bank158521 +158522 POINT(41.50980622467621 -87.68581959271616) bank158522 +158523 POINT(42.45154942654094 -88.31500177925969) bank158523 +158524 POINT(42.149511591501934 -88.32947178867424) bank158524 +158525 POINT(41.445231110168095 -87.30710952917212) bank158525 +158526 POINT(42.73181456362089 -88.46749024659076) bank158526 +158527 POINT(42.489859590376284 -86.67634433262161) bank158527 +158528 POINT(42.662231946050305 -87.82979522086539) bank158528 +158529 POINT(42.26580298999113 -87.96423287855116) bank158529 +158530 POINT(41.45431156947157 -88.17525829725349) bank158530 +158531 POINT(41.8283718547436 -88.32397999232928) bank158531 +158532 POINT(40.9181850422403 -86.76196803509538) bank158532 +158533 POINT(41.38882514080283 -88.55937242645254) bank158533 +158534 POINT(42.77860292912774 -87.44499634006627) bank158534 +158535 POINT(40.91457051001886 -88.08938626812837) bank158535 +158536 POINT(42.38976163887056 -87.18641603026283) bank158536 +158537 POINT(42.53896478176117 -88.05194700882151) bank158537 +158538 POINT(42.18921346144076 -86.9137880254927) bank158538 +158539 POINT(41.47022038184575 -87.96566738795866) bank158539 +158540 POINT(42.79043645850102 -88.02070663807805) bank158540 +158541 POINT(42.28113694950538 -87.68147130194075) bank158541 +158542 POINT(41.09046393058402 -86.89594131560902) bank158542 +158543 POINT(42.294708915960285 -87.02243282489238) bank158543 +158544 POINT(40.88196121534184 -87.13095770213279) bank158544 +158545 POINT(42.53876146248685 -88.57080344562881) bank158545 +158546 POINT(42.283208120688045 -87.57223070203189) bank158546 +158547 POINT(41.37939508766089 -87.91127323502845) bank158547 +158548 POINT(41.12039998293302 -87.14974298320288) bank158548 +158549 POINT(41.35736290018961 -87.64109010872849) bank158549 +158550 POINT(41.47705370253033 -87.74023783332788) bank158550 +158551 POINT(41.88261369885015 -87.75122025038873) bank158551 +158552 POINT(42.64009210391925 -87.27997014843865) bank158552 +158553 POINT(41.39197016842892 -88.05925022688827) bank158553 +158554 POINT(41.58371983232547 -87.40020287219996) bank158554 +158555 POINT(41.08695232127951 -87.14727552743229) bank158555 +158556 POINT(42.41287832840172 -86.8543620987798) bank158556 +158557 POINT(41.12346960900993 -87.82532646741086) bank158557 +158558 POINT(42.341090869282624 -88.08734660343588) bank158558 +158559 POINT(42.390460770698944 -87.65967621408734) bank158559 +158560 POINT(42.671332933992005 -86.63711781857168) bank158560 +158561 POINT(41.288868413171386 -87.33105571346931) bank158561 +158562 POINT(41.31843665577711 -87.36128409717394) bank158562 +158563 POINT(42.30515847694286 -87.10545069793167) bank158563 +158564 POINT(41.3309777107617 -87.1631188208419) bank158564 +158565 POINT(42.305857003551644 -86.79617337244014) bank158565 +158566 POINT(42.39032732864022 -87.46147208579083) bank158566 +158567 POINT(42.0695896588491 -87.04563409281961) bank158567 +158568 POINT(42.062001562272904 -86.92797841272382) bank158568 +158569 POINT(41.84361823636531 -86.79768093697864) bank158569 +158570 POINT(42.72500146337572 -88.48259943622692) bank158570 +158571 POINT(42.84064008576434 -87.8058454342888) bank158571 +158572 POINT(41.761899413370706 -88.55776085889609) bank158572 +158573 POINT(41.65477706776276 -87.85176653493082) bank158573 +158574 POINT(41.70540291922562 -87.19724670054475) bank158574 +158575 POINT(41.96122018033484 -87.89460904793174) bank158575 +158576 POINT(41.69542144955783 -87.62829989319619) bank158576 +158577 POINT(40.9783531972055 -88.51435077497516) bank158577 +158578 POINT(42.4393772381666 -87.57199252730926) bank158578 +158579 POINT(41.100680193362045 -87.3380402619763) bank158579 +158580 POINT(40.906261356418476 -87.88980985742548) bank158580 +158581 POINT(41.19843578503519 -87.54489620422447) bank158581 +158582 POINT(42.04836247759388 -88.04798156630166) bank158582 +158583 POINT(41.073163750439086 -88.30169939523809) bank158583 +158584 POINT(42.75952447745686 -86.95320978926597) bank158584 +158585 POINT(41.52574753140215 -87.03202232029464) bank158585 +158586 POINT(41.89790442141241 -87.48651240441927) bank158586 +158587 POINT(42.031692269435815 -86.81353235919514) bank158587 +158588 POINT(42.452943155253024 -88.47286836776806) bank158588 +158589 POINT(42.23004820001289 -86.77121500873012) bank158589 +158590 POINT(42.51059333661808 -88.34414281251485) bank158590 +158591 POINT(42.66718158919591 -88.61578982194725) bank158591 +158592 POINT(42.43124976126512 -87.72108293976964) bank158592 +158593 POINT(41.76773186119775 -87.71643248970155) bank158593 +158594 POINT(42.05887022062611 -88.45263375825091) bank158594 +158595 POINT(42.102802335709384 -88.08644610545458) bank158595 +158596 POINT(41.93189708869977 -86.67899308459567) bank158596 +158597 POINT(40.94297461399476 -88.53811903730954) bank158597 +158598 POINT(42.61489415804419 -86.90539706754568) bank158598 +158599 POINT(42.126602395380324 -87.56070417437874) bank158599 +158600 POINT(42.45207092646831 -88.51919931121436) bank158600 +158601 POINT(40.92699977376753 -87.20527581187764) bank158601 +158602 POINT(41.55751544374729 -88.47980642278452) bank158602 +158603 POINT(42.83078280084512 -87.70559614860933) bank158603 +158604 POINT(42.41255951327709 -87.55250875263627) bank158604 +158605 POINT(41.34555380808524 -86.92442249727289) bank158605 +158606 POINT(42.75905720616337 -86.86848277218246) bank158606 +158607 POINT(42.62837084086297 -88.52971026755526) bank158607 +158608 POINT(42.56135900409383 -87.94587115009682) bank158608 +158609 POINT(42.62085173200553 -87.5799596707766) bank158609 +158610 POINT(42.07269308809159 -88.45888790115434) bank158610 +158611 POINT(41.42715363511854 -87.46682528963663) bank158611 +158612 POINT(41.30022924690299 -87.9004870715622) bank158612 +158613 POINT(42.094784715495884 -88.23152767360081) bank158613 +158614 POINT(41.96173200512275 -88.02351415305796) bank158614 +158615 POINT(41.19420116364018 -88.47663075354069) bank158615 +158616 POINT(41.84447643242632 -86.86227360885955) bank158616 +158617 POINT(42.40347849124581 -87.86855344333819) bank158617 +158618 POINT(42.47985577368437 -88.26073948534298) bank158618 +158619 POINT(40.996762062313955 -87.55296043732685) bank158619 +158620 POINT(41.727671624320095 -87.15560246500846) bank158620 +158621 POINT(41.63887189716522 -87.51263123957648) bank158621 +158622 POINT(41.59584755184935 -87.94495685956005) bank158622 +158623 POINT(42.81248009227178 -87.92067753937371) bank158623 +158624 POINT(40.90142589509763 -87.99030721733806) bank158624 +158625 POINT(41.963082319836744 -88.02763495255243) bank158625 +158626 POINT(42.749673607574046 -86.87345475594893) bank158626 +158627 POINT(41.866155254550804 -87.49358094068793) bank158627 +158628 POINT(41.72120978680164 -86.81108989419236) bank158628 +158629 POINT(40.885736900126574 -88.09646449460539) bank158629 +158630 POINT(42.51540940668618 -87.00936304632751) bank158630 +158631 POINT(41.38304485631454 -88.42417252042202) bank158631 +158632 POINT(42.29972415160369 -87.03284925422594) bank158632 +158633 POINT(41.51232110617289 -88.13821911002611) bank158633 +158634 POINT(42.5142661043919 -88.34739584310955) bank158634 +158635 POINT(41.93933171146194 -86.98613793204177) bank158635 +158636 POINT(42.13875375787689 -87.03726394324417) bank158636 +158637 POINT(42.406613477685866 -87.66628026960309) bank158637 +158638 POINT(40.994910562706124 -87.25621311151768) bank158638 +158639 POINT(42.475079915603196 -86.69142232964181) bank158639 +158640 POINT(41.52509469063662 -86.92768042660339) bank158640 +158641 POINT(42.500482069939885 -87.27032112823066) bank158641 +158642 POINT(41.107929051752066 -87.2205044246076) bank158642 +158643 POINT(42.33457835210433 -87.24439826233848) bank158643 +158644 POINT(41.06015020443494 -87.45516221082697) bank158644 +158645 POINT(41.31870879295301 -88.52540354677426) bank158645 +158646 POINT(41.43085335891004 -88.48652025883024) bank158646 +158647 POINT(41.320055827527966 -87.04573672375312) bank158647 +158648 POINT(41.44791107169796 -87.30765480151067) bank158648 +158649 POINT(42.62569279808426 -86.89462036670879) bank158649 +158650 POINT(41.007376805291116 -87.5746556629649) bank158650 +158651 POINT(41.72848339575373 -87.3768573857153) bank158651 +158652 POINT(41.209193981764045 -87.1248411950555) bank158652 +158653 POINT(41.757065098260306 -87.96378008586119) bank158653 +158654 POINT(41.3191036598743 -87.43033728007754) bank158654 +158655 POINT(41.02222737883161 -88.59003503463093) bank158655 +158656 POINT(42.45178508584094 -88.45430782615064) bank158656 +158657 POINT(42.52129192346574 -87.19667291088287) bank158657 +158658 POINT(42.61709132667221 -88.11951649500119) bank158658 +158659 POINT(42.35417751111142 -88.11535261317594) bank158659 +158660 POINT(41.2948177907593 -87.297628124777) bank158660 +158661 POINT(41.55746556956252 -88.45582883956075) bank158661 +158662 POINT(41.385247923024494 -87.89083738310462) bank158662 +158663 POINT(41.78884077810265 -88.57576647531742) bank158663 +158664 POINT(41.876600172828546 -88.57985552775152) bank158664 +158665 POINT(42.29003446045054 -87.13413423458698) bank158665 +158666 POINT(42.29700722874517 -88.52037427913574) bank158666 +158667 POINT(41.10571528793532 -86.72872752489297) bank158667 +158668 POINT(42.51921006596335 -87.55463807874415) bank158668 +158669 POINT(41.885003782959856 -88.20504587074083) bank158669 +158670 POINT(41.00857720646708 -87.97153344187774) bank158670 +158671 POINT(42.4506711663589 -87.95522962931791) bank158671 +158672 POINT(41.02944653242271 -87.02877789971407) bank158672 +158673 POINT(41.295063178507824 -86.83815217772144) bank158673 +158674 POINT(41.69423929747209 -87.59653484691418) bank158674 +158675 POINT(42.130774865288615 -87.74862652740326) bank158675 +158676 POINT(42.18592648711275 -87.66967534114598) bank158676 +158677 POINT(42.230221550655244 -87.8780613372378) bank158677 +158678 POINT(42.56415619664476 -88.33535229935634) bank158678 +158679 POINT(42.52478365775348 -88.46011890290711) bank158679 +158680 POINT(41.889982906080725 -86.67790940287198) bank158680 +158681 POINT(41.24961812446122 -87.54631615392753) bank158681 +158682 POINT(41.20127757345467 -87.67133060522694) bank158682 +158683 POINT(40.99241310595822 -87.0979196939374) bank158683 +158684 POINT(41.39943692885689 -87.53690962827258) bank158684 +158685 POINT(42.36387434997376 -88.59654833768715) bank158685 +158686 POINT(41.58901830620588 -87.57763596173602) bank158686 +158687 POINT(40.96273610933304 -86.9726020456949) bank158687 +158688 POINT(41.70150157966225 -87.20192714898536) bank158688 +158689 POINT(41.61847682226309 -88.23878800416544) bank158689 +158690 POINT(41.69809215997285 -87.88988711868207) bank158690 +158691 POINT(42.163247725010464 -86.95609488556534) bank158691 +158692 POINT(41.28951115429582 -86.78791740012) bank158692 +158693 POINT(41.87258260977595 -87.36013682144078) bank158693 +158694 POINT(41.01975527115008 -88.48330730449449) bank158694 +158695 POINT(41.61572313258222 -87.04336217064) bank158695 +158696 POINT(41.4407031926832 -88.33989193640818) bank158696 +158697 POINT(42.0619843878186 -87.06830546804284) bank158697 +158698 POINT(42.615195282633685 -87.61969972006847) bank158698 +158699 POINT(42.48378176045178 -87.9693442950902) bank158699 +158700 POINT(41.419457524699965 -87.33540590820984) bank158700 +158701 POINT(41.36858810110949 -87.82480771923045) bank158701 +158702 POINT(41.406420957122194 -87.38508508810474) bank158702 +158703 POINT(41.49608785370158 -88.04127423240892) bank158703 +158704 POINT(41.56340246073022 -87.3025318335236) bank158704 +158705 POINT(41.05589634785004 -86.81261815367382) bank158705 +158706 POINT(41.247269752618514 -87.02335594574174) bank158706 +158707 POINT(41.5735589692159 -87.46903072145548) bank158707 +158708 POINT(41.78749079353781 -87.76019090677312) bank158708 +158709 POINT(41.03755794889363 -87.71447900093004) bank158709 +158710 POINT(41.04616542100447 -87.76770078663648) bank158710 +158711 POINT(41.21164699923396 -88.54262166617652) bank158711 +158712 POINT(41.21287647065747 -86.79813379731117) bank158712 +158713 POINT(41.908641174335 -88.32566310170543) bank158713 +158714 POINT(41.20984958521967 -88.37938643754207) bank158714 +158715 POINT(42.30831290217234 -87.30434303217137) bank158715 +158716 POINT(40.93348715738256 -87.3947169280157) bank158716 +158717 POINT(42.12269380700621 -86.69500866662044) bank158717 +158718 POINT(41.70554152971005 -87.92165570727524) bank158718 +158719 POINT(42.19228633959568 -88.58299232896411) bank158719 +158720 POINT(41.74471253737174 -88.52717094580238) bank158720 +158721 POINT(41.31242157666832 -86.77994985814647) bank158721 +158722 POINT(42.0027094314662 -86.66990565874298) bank158722 +158723 POINT(42.17595111005535 -86.72338890553699) bank158723 +158724 POINT(41.01015359666153 -86.68397735565135) bank158724 +158725 POINT(41.756319141405164 -88.55750410448428) bank158725 +158726 POINT(42.02033520687651 -87.86741121822661) bank158726 +158727 POINT(42.786721487513944 -87.852811256081) bank158727 +158728 POINT(41.52597608963888 -86.81446006910419) bank158728 +158729 POINT(40.93475306271379 -87.91391375885095) bank158729 +158730 POINT(42.21350940678882 -87.69541956661752) bank158730 +158731 POINT(41.18534053870409 -87.91664866590445) bank158731 +158732 POINT(42.85754238838398 -86.74210670979986) bank158732 +158733 POINT(41.68968758085388 -86.76066969578402) bank158733 +158734 POINT(41.002483383396346 -87.2751644013244) bank158734 +158735 POINT(41.89560667966756 -87.78478128074046) bank158735 +158736 POINT(42.463252868977996 -87.90520137085419) bank158736 +158737 POINT(41.312903583872576 -88.52089417833078) bank158737 +158738 POINT(42.10846474066907 -87.13757961765664) bank158738 +158739 POINT(41.65579883240969 -87.7537551922244) bank158739 +158740 POINT(41.6913299273049 -87.0952230024793) bank158740 +158741 POINT(41.66435239360002 -86.75193113856847) bank158741 +158742 POINT(42.40431114406945 -88.2996788226701) bank158742 +158743 POINT(41.898552467332394 -87.6576955592726) bank158743 +158744 POINT(42.74859682174373 -88.59363193837696) bank158744 +158745 POINT(41.16508958722973 -86.76650260484071) bank158745 +158746 POINT(41.94934605176022 -87.94772787766487) bank158746 +158747 POINT(40.90782635969685 -87.39899157413228) bank158747 +158748 POINT(42.302786466788056 -88.399530239003) bank158748 +158749 POINT(42.231624367012856 -86.97913958144011) bank158749 +158750 POINT(42.68213288786427 -88.16244863951029) bank158750 +158751 POINT(41.46008905344024 -88.55223241588688) bank158751 +158752 POINT(42.25961841603539 -87.84885823800018) bank158752 +158753 POINT(42.873969491700635 -86.72352233868321) bank158753 +158754 POINT(41.719566627090586 -88.48220021412513) bank158754 +158755 POINT(41.14726386084362 -88.21401079480748) bank158755 +158756 POINT(41.280469676684696 -87.95329181410308) bank158756 +158757 POINT(41.64876275698889 -87.12407410528644) bank158757 +158758 POINT(41.133747871515986 -88.3634649636202) bank158758 +158759 POINT(42.6963602460709 -86.97785179076338) bank158759 +158760 POINT(42.59919672677057 -88.02243509426677) bank158760 +158761 POINT(41.75597837589684 -88.62337800821524) bank158761 +158762 POINT(42.042928335529254 -88.3276846603702) bank158762 +158763 POINT(41.57777234900065 -86.84432982593164) bank158763 +158764 POINT(42.165376219769584 -87.39143813337783) bank158764 +158765 POINT(41.00668460955536 -88.13339177551273) bank158765 +158766 POINT(41.123958371822674 -86.94217737868435) bank158766 +158767 POINT(41.736951826547575 -87.86127680299255) bank158767 +158768 POINT(41.47139659290183 -87.60599205381638) bank158768 +158769 POINT(42.37115436389217 -86.8794624477644) bank158769 +158770 POINT(41.07721598833803 -86.89472515903896) bank158770 +158771 POINT(41.91465510256663 -88.51777509280842) bank158771 +158772 POINT(42.195352321780035 -87.72054383161515) bank158772 +158773 POINT(42.40645874378335 -86.97718745726172) bank158773 +158774 POINT(42.29609584845545 -87.09138971723875) bank158774 +158775 POINT(41.07137635252564 -87.82580253151951) bank158775 +158776 POINT(41.707912840927555 -87.53895829269511) bank158776 +158777 POINT(41.67812055108066 -87.0591463822759) bank158777 +158778 POINT(41.17982358538495 -88.52665987870381) bank158778 +158779 POINT(42.53987886902906 -88.3373773025789) bank158779 +158780 POINT(42.483614952347466 -88.29559433134625) bank158780 +158781 POINT(41.03189079911552 -88.02249599567988) bank158781 +158782 POINT(42.39237780137748 -87.30551057835766) bank158782 +158783 POINT(42.01723569246346 -86.93930195729422) bank158783 +158784 POINT(41.807176629536954 -87.45106799096602) bank158784 +158785 POINT(40.90071696918723 -86.91084926829774) bank158785 +158786 POINT(41.093033738803975 -88.05518309140072) bank158786 +158787 POINT(42.45334648607112 -88.02603078056516) bank158787 +158788 POINT(41.22052185772519 -87.59139735817637) bank158788 +158789 POINT(42.166557399676634 -87.22917119075294) bank158789 +158790 POINT(41.54606040230419 -88.52035705578321) bank158790 +158791 POINT(42.27910031467803 -88.54037922286312) bank158791 +158792 POINT(41.12898639676929 -87.05982469313622) bank158792 +158793 POINT(41.34771995957903 -87.69021247952045) bank158793 +158794 POINT(41.46817766902492 -87.58639287125904) bank158794 +158795 POINT(42.3264835983953 -86.81157183133907) bank158795 +158796 POINT(41.04525882643798 -88.08427368430088) bank158796 +158797 POINT(42.42071127280248 -88.0933608261161) bank158797 +158798 POINT(42.82410920785336 -86.77040011585427) bank158798 +158799 POINT(41.6024316651316 -88.10379649750818) bank158799 +158800 POINT(41.30772320675417 -88.61418913328839) bank158800 +158801 POINT(42.44039700823954 -87.34863439317192) bank158801 +158802 POINT(41.14746565398422 -88.26756750623801) bank158802 +158803 POINT(42.759163867801014 -88.11660773737454) bank158803 +158804 POINT(41.52308048710598 -87.3258251489291) bank158804 +158805 POINT(41.33636889421534 -87.97809131742083) bank158805 +158806 POINT(41.289087250538515 -86.6950382720338) bank158806 +158807 POINT(41.94879605803627 -86.74394559908448) bank158807 +158808 POINT(42.23278046629014 -87.78801288757347) bank158808 +158809 POINT(42.759695391861285 -88.4869300503665) bank158809 +158810 POINT(42.32100161442793 -87.0770378669562) bank158810 +158811 POINT(41.05958922416788 -87.14218087898148) bank158811 +158812 POINT(42.361919220132314 -87.91623499607985) bank158812 +158813 POINT(41.27571575967242 -88.56442121908765) bank158813 +158814 POINT(42.579386221588315 -87.32832682664439) bank158814 +158815 POINT(41.217034908308285 -86.8340007441132) bank158815 +158816 POINT(42.46700846875687 -86.82065333347346) bank158816 +158817 POINT(41.32365831070453 -87.47298427984913) bank158817 +158818 POINT(41.43117072799921 -88.02831520850508) bank158818 +158819 POINT(42.262115202496204 -87.42174620313885) bank158819 +158820 POINT(42.0808523343878 -86.74814265172682) bank158820 +158821 POINT(42.59009860075089 -87.90056363802286) bank158821 +158822 POINT(41.67273392597049 -87.74814665087247) bank158822 +158823 POINT(41.531306530431536 -86.747213694462) bank158823 +158824 POINT(42.09683024434972 -88.19684600921981) bank158824 +158825 POINT(41.10828524668871 -86.72477514839768) bank158825 +158826 POINT(41.24272214631153 -87.93361651512153) bank158826 +158827 POINT(42.27063143889534 -86.7519412454144) bank158827 +158828 POINT(41.552679364863245 -88.1367124119892) bank158828 +158829 POINT(41.9974277736274 -87.87799338544897) bank158829 +158830 POINT(41.15981070857993 -87.21791068568112) bank158830 +158831 POINT(42.85809665731246 -87.08236175399773) bank158831 +158832 POINT(42.820964766446004 -87.12825055506073) bank158832 +158833 POINT(40.93608634928557 -88.45727537921437) bank158833 +158834 POINT(42.759143283214826 -86.851921455357) bank158834 +158835 POINT(41.61141657530779 -88.07130145592605) bank158835 +158836 POINT(42.686450787887836 -87.83821429374258) bank158836 +158837 POINT(41.42484026956674 -88.21491898663255) bank158837 +158838 POINT(42.81310889951666 -87.09921891169259) bank158838 +158839 POINT(41.70014559839016 -87.53229404014888) bank158839 +158840 POINT(42.6042122492312 -88.53003699908886) bank158840 +158841 POINT(42.32351160380861 -88.5413040430905) bank158841 +158842 POINT(42.03799733134283 -88.18420220722217) bank158842 +158843 POINT(41.969671246000715 -87.31203988314172) bank158843 +158844 POINT(42.73158216882791 -87.94942677511264) bank158844 +158845 POINT(41.183708798728844 -87.0515002708572) bank158845 +158846 POINT(41.10195937086924 -87.0175805318207) bank158846 +158847 POINT(41.14606912790941 -87.33117185360777) bank158847 +158848 POINT(42.0082206135729 -88.45724565810002) bank158848 +158849 POINT(42.623473893643954 -88.58050351137871) bank158849 +158850 POINT(42.75324104150793 -87.66055991108053) bank158850 +158851 POINT(41.52113456696395 -87.91478432122518) bank158851 +158852 POINT(42.82187839906097 -86.95046368926234) bank158852 +158853 POINT(42.28505713525786 -87.04386472542875) bank158853 +158854 POINT(41.30705250915462 -86.88339209269691) bank158854 +158855 POINT(42.79114346454614 -86.85653746888107) bank158855 +158856 POINT(42.07025728107315 -87.88329017494966) bank158856 +158857 POINT(41.934598236707245 -87.79110217036403) bank158857 +158858 POINT(42.444125835351755 -86.7025017178329) bank158858 +158859 POINT(41.263698252499374 -87.63787179052245) bank158859 +158860 POINT(42.30732039197183 -87.86834806232005) bank158860 +158861 POINT(42.19898933479076 -87.94445417998782) bank158861 +158862 POINT(42.37563181449319 -86.73629245723525) bank158862 +158863 POINT(42.512838215924724 -87.24114381997057) bank158863 +158864 POINT(41.77825259756454 -87.98322083381834) bank158864 +158865 POINT(42.463370167511044 -87.99818375413912) bank158865 +158866 POINT(42.18628436119205 -87.77547894356684) bank158866 +158867 POINT(42.121460300166106 -88.30878309164548) bank158867 +158868 POINT(41.27553525123232 -87.91750664594919) bank158868 +158869 POINT(41.467551883947486 -87.76434901527169) bank158869 +158870 POINT(40.976841017179645 -86.9075658933947) bank158870 +158871 POINT(42.26098001211842 -86.81156965648815) bank158871 +158872 POINT(41.13213027878733 -88.28407701894143) bank158872 +158873 POINT(42.23544727009143 -87.21051853203616) bank158873 +158874 POINT(42.84590134977863 -87.92387020369664) bank158874 +158875 POINT(42.65403144761213 -87.70371107399545) bank158875 +158876 POINT(42.15513317568415 -86.8609778517806) bank158876 +158877 POINT(42.481944630511286 -87.44516485247345) bank158877 +158878 POINT(42.608896971613476 -87.49533518735011) bank158878 +158879 POINT(41.11265430393309 -87.0663849454413) bank158879 +158880 POINT(40.96841583489684 -87.69768531826102) bank158880 +158881 POINT(42.437975293874935 -86.95539768563339) bank158881 +158882 POINT(41.70506105130003 -88.46583811173018) bank158882 +158883 POINT(41.30259916465036 -87.2732059808874) bank158883 +158884 POINT(42.45787580325059 -87.45335398418376) bank158884 +158885 POINT(41.67070890037181 -87.5233276027074) bank158885 +158886 POINT(42.619649535447856 -86.87462982780586) bank158886 +158887 POINT(41.67805838866682 -88.31754673209893) bank158887 +158888 POINT(41.36416475038902 -88.14684310088) bank158888 +158889 POINT(41.141846386750466 -87.89493251769662) bank158889 +158890 POINT(42.8315960183401 -87.52062276359018) bank158890 +158891 POINT(41.595843067978336 -86.91958824081001) bank158891 +158892 POINT(42.49794135224538 -87.16110031134804) bank158892 +158893 POINT(42.856393065961804 -87.99437387038577) bank158893 +158894 POINT(41.698514790537985 -87.04601579218996) bank158894 +158895 POINT(42.3113770478357 -87.55863559159015) bank158895 +158896 POINT(41.858740326361435 -87.84456569119185) bank158896 +158897 POINT(42.228887718914116 -86.65752598184656) bank158897 +158898 POINT(41.16759751154006 -87.13178319132776) bank158898 +158899 POINT(41.5680758695144 -87.35679467211394) bank158899 +158900 POINT(42.14437780053736 -87.95755303675021) bank158900 +158901 POINT(42.69544401793979 -87.160509773159) bank158901 +158902 POINT(41.29789985392853 -87.15200669097126) bank158902 +158903 POINT(41.70613064949155 -86.83107260809855) bank158903 +158904 POINT(41.987259861734884 -87.82114521196065) bank158904 +158905 POINT(42.386463601329275 -88.53982619760485) bank158905 +158906 POINT(41.63144493120976 -88.26853792950492) bank158906 +158907 POINT(42.26841007274889 -87.73657041699269) bank158907 +158908 POINT(42.44790147523437 -88.34705840774309) bank158908 +158909 POINT(42.06302789043006 -88.40340075175189) bank158909 +158910 POINT(41.74957635851067 -87.43121802465122) bank158910 +158911 POINT(42.41854848674959 -87.37836029510551) bank158911 +158912 POINT(41.30962246861663 -87.65934931562555) bank158912 +158913 POINT(42.09966087373121 -87.40183197513232) bank158913 +158914 POINT(41.252410057870655 -86.63998590246898) bank158914 +158915 POINT(41.804301483615454 -88.38339033422348) bank158915 +158916 POINT(42.14592119260427 -88.2422303155573) bank158916 +158917 POINT(41.4817215181603 -87.34510779158276) bank158917 +158918 POINT(42.860028235369526 -88.28732582377103) bank158918 +158919 POINT(41.719661110038 -88.54353820507498) bank158919 +158920 POINT(42.260085109063084 -87.76640638352124) bank158920 +158921 POINT(41.26980957181851 -86.6470461242901) bank158921 +158922 POINT(41.79322717124715 -88.02114445544026) bank158922 +158923 POINT(42.516173921002476 -87.87026399935836) bank158923 +158924 POINT(42.764086037267376 -86.87324935258934) bank158924 +158925 POINT(41.79533671290688 -87.66586790772092) bank158925 +158926 POINT(41.63716149229152 -87.91854009474868) bank158926 +158927 POINT(42.411438618396645 -87.02853773145365) bank158927 +158928 POINT(41.54319519639697 -87.33505636240871) bank158928 +158929 POINT(42.49308910333739 -88.07719653025565) bank158929 +158930 POINT(42.494069862731436 -86.69218796423574) bank158930 +158931 POINT(41.76463130998017 -87.83699360594721) bank158931 +158932 POINT(42.64057271913577 -87.17154460192528) bank158932 +158933 POINT(40.9789904914381 -88.25516654120023) bank158933 +158934 POINT(40.96347061943783 -88.58077812095503) bank158934 +158935 POINT(41.5141301462155 -87.45832409519247) bank158935 +158936 POINT(41.75480678257099 -86.91349720011938) bank158936 +158937 POINT(42.231865762008034 -88.55420446864517) bank158937 +158938 POINT(40.919684669901166 -86.88821030308208) bank158938 +158939 POINT(42.719908836718254 -87.0748497967157) bank158939 +158940 POINT(42.420975526771706 -88.37312971009094) bank158940 +158941 POINT(40.92785307095842 -88.07852771632675) bank158941 +158942 POINT(41.68416621345735 -86.74018339730279) bank158942 +158943 POINT(41.867415111252704 -86.74837601345264) bank158943 +158944 POINT(42.48294963027705 -87.72520845995801) bank158944 +158945 POINT(41.65265293311286 -87.71700042139099) bank158945 +158946 POINT(42.04710613325066 -87.9839396119888) bank158946 +158947 POINT(41.139772944476036 -86.70252201066423) bank158947 +158948 POINT(41.48870116579443 -88.49095985004257) bank158948 +158949 POINT(40.910696582766526 -88.44628942453275) bank158949 +158950 POINT(42.16931347799616 -87.38035406137249) bank158950 +158951 POINT(42.439855797815774 -86.72421451152675) bank158951 +158952 POINT(41.95483225475073 -87.58061501578459) bank158952 +158953 POINT(42.53492998073535 -86.66305884853053) bank158953 +158954 POINT(42.48060111607304 -88.38275576482827) bank158954 +158955 POINT(40.93532041343084 -87.68028693464402) bank158955 +158956 POINT(42.401349472034795 -87.57320887556156) bank158956 +158957 POINT(42.30855717190026 -86.65138044840076) bank158957 +158958 POINT(42.06691879051156 -86.83644030747226) bank158958 +158959 POINT(41.87454224484368 -88.4129173289507) bank158959 +158960 POINT(41.868693916398165 -86.72436038802962) bank158960 +158961 POINT(41.230688573275074 -87.13015017480173) bank158961 +158962 POINT(42.41412027679666 -86.8563408587601) bank158962 +158963 POINT(42.065828172150766 -87.86228438596066) bank158963 +158964 POINT(42.69795112533138 -87.52603439565969) bank158964 +158965 POINT(42.49165719575934 -88.12069055869962) bank158965 +158966 POINT(40.90148903879914 -87.6596798925185) bank158966 +158967 POINT(41.98934162690746 -87.31147206317105) bank158967 +158968 POINT(41.72815261172334 -86.72091715892377) bank158968 +158969 POINT(41.851431337516885 -88.31159513803202) bank158969 +158970 POINT(41.98971749359627 -88.16600270980841) bank158970 +158971 POINT(42.04932884335622 -87.61655391137622) bank158971 +158972 POINT(42.23042027808747 -88.22403705871353) bank158972 +158973 POINT(42.406848055499474 -87.25848042307697) bank158973 +158974 POINT(42.67074581459898 -88.09897629011903) bank158974 +158975 POINT(42.14855155001605 -87.67979477638286) bank158975 +158976 POINT(42.84209911341862 -86.78256010327804) bank158976 +158977 POINT(42.871059514108566 -88.39252977860087) bank158977 +158978 POINT(41.768861153372704 -88.0298923092843) bank158978 +158979 POINT(41.45297095891409 -87.66486286436871) bank158979 +158980 POINT(42.72168120719641 -87.7298648208102) bank158980 +158981 POINT(42.29007699130103 -87.11739341602302) bank158981 +158982 POINT(42.772283913834364 -86.87355756783113) bank158982 +158983 POINT(41.97408297746553 -88.43228866695863) bank158983 +158984 POINT(41.21588493094856 -86.79839517583567) bank158984 +158985 POINT(42.7425225036622 -87.07032307055607) bank158985 +158986 POINT(42.49830878284055 -87.63522926038074) bank158986 +158987 POINT(42.84691188682595 -87.01078600958212) bank158987 +158988 POINT(40.91866315745393 -87.43767877642962) bank158988 +158989 POINT(42.00662638984972 -86.9264803164234) bank158989 +158990 POINT(41.70064498736501 -87.1251867807599) bank158990 +158991 POINT(41.494116663989026 -87.82260395620362) bank158991 +158992 POINT(41.21000646273788 -87.53710546674895) bank158992 +158993 POINT(42.52504375526005 -88.19052991577772) bank158993 +158994 POINT(42.21704307237892 -88.59148558158655) bank158994 +158995 POINT(41.01770335456269 -87.0294213500247) bank158995 +158996 POINT(41.63474045060776 -87.64749882485856) bank158996 +158997 POINT(41.50250654657761 -88.14978335556032) bank158997 +158998 POINT(42.353039113297086 -87.83431698236096) bank158998 +158999 POINT(42.498224073675644 -88.42811758699779) bank158999 +159000 POINT(42.33730591501387 -88.30575805816743) bank159000 +159001 POINT(40.98351870769277 -88.31936026419703) bank159001 +159002 POINT(42.28675927669828 -87.89991814069309) bank159002 +159003 POINT(41.673139972908025 -87.41418040850313) bank159003 +159004 POINT(42.826462480189996 -88.112626228347) bank159004 +159005 POINT(41.61111889335392 -87.33462438702266) bank159005 +159006 POINT(41.80805875466899 -86.79110205674225) bank159006 +159007 POINT(41.8501527212271 -86.64893563470747) bank159007 +159008 POINT(41.220345555346036 -88.49735895908714) bank159008 +159009 POINT(41.40848170003238 -87.05445348498758) bank159009 +159010 POINT(41.47828646170531 -87.40191858001089) bank159010 +159011 POINT(41.32414415119827 -87.73910516204569) bank159011 +159012 POINT(41.46635723339715 -87.4825137647335) bank159012 +159013 POINT(42.640008010426904 -86.83797090698141) bank159013 +159014 POINT(41.14675508112634 -88.10213395108663) bank159014 +159015 POINT(40.88327031332735 -88.15766487411776) bank159015 +159016 POINT(42.36886834915031 -86.76392132994434) bank159016 +159017 POINT(42.003419349280335 -88.17941597284316) bank159017 +159018 POINT(40.944228596721814 -86.96586104440101) bank159018 +159019 POINT(41.47215884248869 -88.35456244878482) bank159019 +159020 POINT(42.182487203415725 -87.56222183646685) bank159020 +159021 POINT(40.996845751755814 -88.08333123169697) bank159021 +159022 POINT(41.05810697864686 -86.85001438757455) bank159022 +159023 POINT(41.21174113041716 -87.75082048896296) bank159023 +159024 POINT(41.35714285713677 -86.68087532703638) bank159024 +159025 POINT(41.52814677314078 -87.34721564830174) bank159025 +159026 POINT(42.46717706159008 -87.16860918677457) bank159026 +159027 POINT(42.61519709366312 -87.00200080499833) bank159027 +159028 POINT(41.87696682627244 -86.69406815459224) bank159028 +159029 POINT(41.50195035967476 -88.61346039042179) bank159029 +159030 POINT(40.98651795897901 -87.37938002854588) bank159030 +159031 POINT(41.21605974273642 -87.54430942972368) bank159031 +159032 POINT(41.37917839494493 -88.48255631762939) bank159032 +159033 POINT(41.002912337467194 -88.31150900203835) bank159033 +159034 POINT(42.3971275282007 -88.5173228103379) bank159034 +159035 POINT(41.59381655956699 -87.03621672989317) bank159035 +159036 POINT(41.7413212615825 -87.61942652935596) bank159036 +159037 POINT(41.06882244754053 -86.86593497686516) bank159037 +159038 POINT(42.77974993638262 -88.04176526141343) bank159038 +159039 POINT(42.31946369462258 -87.35916529344188) bank159039 +159040 POINT(42.162391941163364 -88.03801487485585) bank159040 +159041 POINT(41.057263108624184 -88.38031169674015) bank159041 +159042 POINT(41.73245913421506 -87.8359637780472) bank159042 +159043 POINT(42.45069879397787 -86.79969681240942) bank159043 +159044 POINT(42.30798098524087 -88.56084970901587) bank159044 +159045 POINT(42.67131010908114 -88.31751582556002) bank159045 +159046 POINT(42.77779608507915 -88.38056974683275) bank159046 +159047 POINT(40.93249011102402 -87.69448541651127) bank159047 +159048 POINT(42.38931635628164 -86.77097609893868) bank159048 +159049 POINT(41.03210541388288 -87.3671222108389) bank159049 +159050 POINT(42.262316442732335 -86.96954449752589) bank159050 +159051 POINT(41.50604497340213 -87.10102525656717) bank159051 +159052 POINT(42.38185233218714 -87.27759303447115) bank159052 +159053 POINT(41.22521219981079 -88.45403650077961) bank159053 +159054 POINT(41.335659240091175 -88.08503740959611) bank159054 +159055 POINT(41.67233175781959 -86.84101138182899) bank159055 +159056 POINT(41.52326472104694 -88.1300446338449) bank159056 +159057 POINT(42.65181816221443 -87.5650407002912) bank159057 +159058 POINT(42.49883118126154 -87.8358007574502) bank159058 +159059 POINT(41.934671066191775 -87.2160815731601) bank159059 +159060 POINT(41.65458539951207 -88.42766971180465) bank159060 +159061 POINT(41.15819858463538 -87.04709041699277) bank159061 +159062 POINT(42.57644530619611 -86.88471277051018) bank159062 +159063 POINT(42.15632213817808 -88.154918864253) bank159063 +159064 POINT(41.11607054029704 -87.0623804533593) bank159064 +159065 POINT(41.70737540757288 -87.71686801792033) bank159065 +159066 POINT(40.97044371163013 -87.00993045258396) bank159066 +159067 POINT(42.43180544381713 -88.32522675774766) bank159067 +159068 POINT(41.2697190574644 -88.6176328677769) bank159068 +159069 POINT(42.49471421222788 -87.37957762323326) bank159069 +159070 POINT(40.888896873229754 -88.29223341044751) bank159070 +159071 POINT(40.981462566246876 -88.36536129810588) bank159071 +159072 POINT(41.796355475162116 -87.26868990824482) bank159072 +159073 POINT(40.9021105426502 -88.05634200998448) bank159073 +159074 POINT(42.28310885764947 -88.00551095346043) bank159074 +159075 POINT(42.22196576770331 -86.82900269763547) bank159075 +159076 POINT(42.58098088893525 -86.78757664498181) bank159076 +159077 POINT(42.17232977213107 -87.14286437540653) bank159077 +159078 POINT(41.02313771585936 -88.32477536265958) bank159078 +159079 POINT(41.77515021683761 -87.01335746397957) bank159079 +159080 POINT(42.441639537697995 -87.08047018103547) bank159080 +159081 POINT(41.22318010077942 -86.95227673869381) bank159081 +159082 POINT(41.669110733609514 -87.23702733507976) bank159082 +159083 POINT(40.898730806524135 -86.7321557732213) bank159083 +159084 POINT(42.446619010581436 -87.66023573978661) bank159084 +159085 POINT(41.503695686401265 -87.17456364246544) bank159085 +159086 POINT(42.20157486523877 -86.76205193616653) bank159086 +159087 POINT(42.29546435356255 -88.14759724348583) bank159087 +159088 POINT(42.59986475944649 -86.91035916663947) bank159088 +159089 POINT(42.444052358223765 -87.91427539364709) bank159089 +159090 POINT(41.813986058454574 -87.29004575984088) bank159090 +159091 POINT(41.59321198683161 -87.94389382531529) bank159091 +159092 POINT(40.95216736257849 -88.32952028533232) bank159092 +159093 POINT(42.41184100257447 -88.1493629332898) bank159093 +159094 POINT(42.08860181721658 -87.91869158620054) bank159094 +159095 POINT(42.036816301418945 -87.07900527483784) bank159095 +159096 POINT(41.60035533298716 -87.04576490598942) bank159096 +159097 POINT(42.486891302893284 -87.73130252639972) bank159097 +159098 POINT(41.035917006770916 -86.73446561207976) bank159098 +159099 POINT(42.154350101021535 -87.1624917976319) bank159099 +159100 POINT(42.37442970870954 -86.65929934486246) bank159100 +159101 POINT(42.05075181240058 -88.11157830248037) bank159101 +159102 POINT(41.370967821222855 -87.5080351298461) bank159102 +159103 POINT(41.60684524546149 -88.38885689438727) bank159103 +159104 POINT(42.05088611394044 -86.63649988020575) bank159104 +159105 POINT(41.096541848902135 -88.26895785587084) bank159105 +159106 POINT(41.636172020657 -87.37401932659948) bank159106 +159107 POINT(42.76050147273812 -86.998264104164) bank159107 +159108 POINT(42.708160253601434 -87.03852304696143) bank159108 +159109 POINT(42.36422891712624 -86.94673244808533) bank159109 +159110 POINT(42.398558138831156 -88.4639388140532) bank159110 +159111 POINT(42.167585612805034 -86.96824839097893) bank159111 +159112 POINT(41.140760740785254 -88.20607763594798) bank159112 +159113 POINT(41.17757500762206 -86.9840725513409) bank159113 +159114 POINT(41.87536455862162 -87.02511517046256) bank159114 +159115 POINT(41.479744609164165 -86.69606055328133) bank159115 +159116 POINT(40.93318113152462 -86.82742832736386) bank159116 +159117 POINT(41.57122316838806 -86.92024470579615) bank159117 +159118 POINT(41.7640594437754 -88.05762061952287) bank159118 +159119 POINT(42.48856987901129 -87.01806450192092) bank159119 +159120 POINT(42.588421304858954 -86.83772099308844) bank159120 +159121 POINT(42.30968977163588 -87.69777237844717) bank159121 +159122 POINT(41.44014667129566 -87.4170859545757) bank159122 +159123 POINT(41.28466640365758 -87.6559436965336) bank159123 +159124 POINT(41.93463359043217 -87.02603998563063) bank159124 +159125 POINT(41.194476621851926 -87.6442660372488) bank159125 +159126 POINT(42.38234012236246 -88.26137398072744) bank159126 +159127 POINT(42.68887125057109 -87.74197128526417) bank159127 +159128 POINT(42.45759562008953 -87.28989966671173) bank159128 +159129 POINT(41.42961405782521 -86.70000091606347) bank159129 +159130 POINT(42.52980420893722 -87.26701511190116) bank159130 +159131 POINT(41.00367541182033 -87.58049374035602) bank159131 +159132 POINT(42.29045737597111 -87.77992649817782) bank159132 +159133 POINT(41.32859492316794 -87.12970726881939) bank159133 +159134 POINT(41.548296737590796 -86.65729204575292) bank159134 +159135 POINT(41.16349523727858 -86.91027782767347) bank159135 +159136 POINT(41.29025463751119 -86.83447158208676) bank159136 +159137 POINT(41.10370222628943 -87.76340063241068) bank159137 +159138 POINT(42.15562305457507 -87.20947198861577) bank159138 +159139 POINT(42.03809185278811 -88.4236800356646) bank159139 +159140 POINT(41.468857199559714 -88.0450220064954) bank159140 +159141 POINT(41.17910518678354 -87.51611747442911) bank159141 +159142 POINT(42.49452225218214 -88.35052663747611) bank159142 +159143 POINT(41.12322216969305 -88.3925333531054) bank159143 +159144 POINT(41.34667766699846 -86.77678370916732) bank159144 +159145 POINT(42.08666515455963 -87.23250866035235) bank159145 +159146 POINT(41.881134979673455 -88.0700207799027) bank159146 +159147 POINT(42.243930077076 -87.54770962276253) bank159147 +159148 POINT(42.40483198360263 -88.28331433090996) bank159148 +159149 POINT(41.890419351417684 -87.0164202325502) bank159149 +159150 POINT(42.283186867246144 -86.76584510274135) bank159150 +159151 POINT(42.19424057996016 -87.14473699922772) bank159151 +159152 POINT(42.5546081589021 -88.15224850515116) bank159152 +159153 POINT(41.54905078090824 -87.29232827260745) bank159153 +159154 POINT(42.718307714179375 -88.29713829080289) bank159154 +159155 POINT(42.54537708697517 -86.84309708499147) bank159155 +159156 POINT(41.71424157371789 -87.1348311136219) bank159156 +159157 POINT(42.408317765340115 -86.93677356340343) bank159157 +159158 POINT(41.575076801505546 -87.12206915154036) bank159158 +159159 POINT(41.9616196346867 -87.91427860189434) bank159159 +159160 POINT(42.56124869325593 -86.71120721129702) bank159160 +159161 POINT(41.71317388258069 -88.57630173302249) bank159161 +159162 POINT(42.011231139630475 -88.09407721517682) bank159162 +159163 POINT(41.13559764649562 -88.19915861543784) bank159163 +159164 POINT(41.097764363068954 -87.2778127905366) bank159164 +159165 POINT(41.33109933494594 -88.35498999632702) bank159165 +159166 POINT(41.56822270434648 -87.802689504159) bank159166 +159167 POINT(41.07693502390523 -87.11569099363756) bank159167 +159168 POINT(40.99176191566812 -87.10333963826632) bank159168 +159169 POINT(41.44110257171547 -88.23084586777485) bank159169 +159170 POINT(41.13178094588799 -87.02896472607927) bank159170 +159171 POINT(41.774732981338985 -87.48086732767375) bank159171 +159172 POINT(41.74374770305773 -88.20072615972235) bank159172 +159173 POINT(41.33328100116911 -86.96621835373375) bank159173 +159174 POINT(41.0447224881418 -87.96108991101048) bank159174 +159175 POINT(40.98852260565344 -88.62283209629959) bank159175 +159176 POINT(41.46780302038835 -87.38835776059993) bank159176 +159177 POINT(41.954490623837806 -86.73552591257261) bank159177 +159178 POINT(40.95131482904833 -87.96986626521014) bank159178 +159179 POINT(42.2256682261248 -86.68338035760021) bank159179 +159180 POINT(41.060797185917934 -86.87408514821387) bank159180 +159181 POINT(42.52103522515636 -87.10429479623896) bank159181 +159182 POINT(42.08472757723446 -87.11285673324244) bank159182 +159183 POINT(42.58560864987368 -87.59976385054611) bank159183 +159184 POINT(42.08337488575988 -87.75565408874168) bank159184 +159185 POINT(41.21876008856614 -87.22001876681335) bank159185 +159186 POINT(41.49804550594033 -86.86697121997592) bank159186 +159187 POINT(41.71855684599669 -87.59427025464552) bank159187 +159188 POINT(41.92407339594661 -87.0181156061781) bank159188 +159189 POINT(42.55331741050372 -87.75474628479628) bank159189 +159190 POINT(41.384762528475115 -88.51575271576401) bank159190 +159191 POINT(41.335875388714236 -88.05217732059235) bank159191 +159192 POINT(41.80257616581342 -86.91850260973759) bank159192 +159193 POINT(41.719119925819754 -87.18397938477912) bank159193 +159194 POINT(41.07559202473884 -87.53054751485308) bank159194 +159195 POINT(42.35744578383627 -87.13653989562387) bank159195 +159196 POINT(42.56477336861231 -87.7663861681458) bank159196 +159197 POINT(41.51512694081679 -86.66272407683952) bank159197 +159198 POINT(41.148225503228126 -88.44976565714832) bank159198 +159199 POINT(40.922486900033405 -88.24695039281629) bank159199 +159200 POINT(42.66002721323319 -88.21602588406907) bank159200 +159201 POINT(41.673489197532774 -87.55001648181647) bank159201 +159202 POINT(41.506302719177675 -88.60231878489722) bank159202 +159203 POINT(41.151831154309434 -88.36011214134929) bank159203 +159204 POINT(42.45840871617873 -86.77451296472624) bank159204 +159205 POINT(41.71012195540214 -87.52114315016597) bank159205 +159206 POINT(42.07883628118526 -87.3917217379637) bank159206 +159207 POINT(41.94313812251878 -88.34401272921653) bank159207 +159208 POINT(42.87307502754487 -87.79303817908561) bank159208 +159209 POINT(41.86841322710789 -86.79472604565603) bank159209 +159210 POINT(42.475570811899466 -86.66857027773771) bank159210 +159211 POINT(42.49243129689372 -87.7832422073702) bank159211 +159212 POINT(41.62326904774092 -87.02155808859209) bank159212 +159213 POINT(42.463540893741836 -87.47256842425568) bank159213 +159214 POINT(41.01441315682226 -88.51567306934868) bank159214 +159215 POINT(40.931137041862996 -88.03451001473574) bank159215 +159216 POINT(42.37638203158224 -86.85185647866052) bank159216 +159217 POINT(42.442373473773316 -88.10106511579917) bank159217 +159218 POINT(41.57220697923373 -87.18624360242151) bank159218 +159219 POINT(42.688341646972404 -87.14275639457259) bank159219 +159220 POINT(42.757084914512056 -87.25959326786533) bank159220 +159221 POINT(40.891739322175304 -87.75827188416129) bank159221 +159222 POINT(42.11277893405958 -87.67383629111268) bank159222 +159223 POINT(42.49741221547821 -88.15719219669545) bank159223 +159224 POINT(42.72827390420679 -88.02047236826658) bank159224 +159225 POINT(42.18008528758661 -88.29696430474743) bank159225 +159226 POINT(40.89493897007293 -87.05868131459646) bank159226 +159227 POINT(42.43604384884272 -87.46243041627616) bank159227 +159228 POINT(42.60163347088676 -87.82723440261925) bank159228 +159229 POINT(41.88094987647467 -87.86056511731802) bank159229 +159230 POINT(41.67622474070491 -88.07052661355841) bank159230 +159231 POINT(41.61715682104709 -86.92252343501981) bank159231 +159232 POINT(41.65418588875862 -87.87881890114834) bank159232 +159233 POINT(42.67723846115434 -87.44581764624307) bank159233 +159234 POINT(41.2759409151958 -88.51225506047643) bank159234 +159235 POINT(41.50132863207355 -87.87975205862067) bank159235 +159236 POINT(42.846034321555074 -86.9145224349461) bank159236 +159237 POINT(41.42821987208951 -87.135878157933) bank159237 +159238 POINT(42.76254900897851 -87.20367829940507) bank159238 +159239 POINT(41.02424075015232 -88.41046393652117) bank159239 +159240 POINT(41.975298237031325 -88.43952753594625) bank159240 +159241 POINT(41.42639358098323 -86.81925483598602) bank159241 +159242 POINT(42.18402159312829 -87.93582391168657) bank159242 +159243 POINT(42.34399678904202 -87.88824503707924) bank159243 +159244 POINT(42.252097521975436 -86.93814673381404) bank159244 +159245 POINT(41.46611711392493 -86.75881415937995) bank159245 +159246 POINT(42.69072429030172 -87.85853508012895) bank159246 +159247 POINT(40.889109989332546 -87.18151080463883) bank159247 +159248 POINT(42.05213434423479 -87.55457637300601) bank159248 +159249 POINT(42.08400244089724 -86.76169250941713) bank159249 +159250 POINT(41.40450324427196 -88.05884329111333) bank159250 +159251 POINT(42.23886213686018 -87.33664579480599) bank159251 +159252 POINT(42.705696097326175 -88.4425232177207) bank159252 +159253 POINT(42.03617668400588 -88.39142170706916) bank159253 +159254 POINT(40.9682160935417 -87.0209873434324) bank159254 +159255 POINT(41.67714932221678 -87.74747192872466) bank159255 +159256 POINT(42.09770993848447 -87.77560193131794) bank159256 +159257 POINT(41.58612768336149 -87.49393472468185) bank159257 +159258 POINT(41.78234145198327 -86.91658589699924) bank159258 +159259 POINT(41.937830180262964 -87.064381706583) bank159259 +159260 POINT(41.97441684257781 -88.60918699936191) bank159260 +159261 POINT(41.404226055358244 -87.47280317153741) bank159261 +159262 POINT(42.341849726085854 -88.24148814459463) bank159262 +159263 POINT(41.66841734183473 -87.10859372501045) bank159263 +159264 POINT(41.05194566567219 -86.87558057431343) bank159264 +159265 POINT(42.556499578321954 -87.1343871806285) bank159265 +159266 POINT(42.62466200203812 -87.24063995578388) bank159266 +159267 POINT(41.56965295069017 -88.21990402855472) bank159267 +159268 POINT(41.82037488908675 -87.06871717591456) bank159268 +159269 POINT(42.85834865981558 -87.42243006222014) bank159269 +159270 POINT(42.568539698384576 -87.34442521766346) bank159270 +159271 POINT(42.34863243354157 -87.31720584881111) bank159271 +159272 POINT(42.29810938458871 -87.51606967248735) bank159272 +159273 POINT(41.491437817318946 -87.57927279449312) bank159273 +159274 POINT(42.20223561193995 -87.77915349606347) bank159274 +159275 POINT(41.607383559156105 -88.57158984598257) bank159275 +159276 POINT(41.78215575752254 -87.69955683360482) bank159276 +159277 POINT(42.865722554928254 -88.16330849989615) bank159277 +159278 POINT(42.32688751655669 -87.09139006372347) bank159278 +159279 POINT(42.76862460077213 -87.91587766481815) bank159279 +159280 POINT(42.533211699060224 -87.29287662781681) bank159280 +159281 POINT(41.970047244493486 -88.2363845493754) bank159281 +159282 POINT(41.07664743651309 -87.52480485719119) bank159282 +159283 POINT(41.92377951405288 -88.08844192656186) bank159283 +159284 POINT(41.81098791803172 -87.09845228313398) bank159284 +159285 POINT(42.20908321627764 -88.08246399846638) bank159285 +159286 POINT(41.208687537470134 -88.45679576167704) bank159286 +159287 POINT(41.64514902563481 -87.89676780293767) bank159287 +159288 POINT(42.09194311315489 -87.2629666642555) bank159288 +159289 POINT(42.19147898147985 -88.12241695875167) bank159289 +159290 POINT(42.78484501829093 -88.11050189801134) bank159290 +159291 POINT(41.642243035313236 -88.36640277668147) bank159291 +159292 POINT(41.2233298490284 -87.1609442513274) bank159292 +159293 POINT(41.05552371063801 -88.60450380216427) bank159293 +159294 POINT(41.380110127775474 -87.46364017875968) bank159294 +159295 POINT(42.39576350824203 -87.63741041596313) bank159295 +159296 POINT(42.275353411372905 -87.21439649121623) bank159296 +159297 POINT(42.36858080571656 -88.43138919483583) bank159297 +159298 POINT(42.33505429675893 -88.0495696514128) bank159298 +159299 POINT(41.26564057538563 -87.32008853299403) bank159299 +159300 POINT(42.23312702487005 -88.3017450834509) bank159300 +159301 POINT(41.74290943512848 -87.64922348264928) bank159301 +159302 POINT(42.02261417064528 -88.0638129762486) bank159302 +159303 POINT(41.516300515580006 -88.56454631361554) bank159303 +159304 POINT(42.136323334728644 -87.17997825417054) bank159304 +159305 POINT(42.48712888575626 -87.45848830153571) bank159305 +159306 POINT(41.38840691714679 -87.42729028817722) bank159306 +159307 POINT(41.243871226416125 -88.28739197869497) bank159307 +159308 POINT(42.0882365547716 -88.10199275795634) bank159308 +159309 POINT(41.5965465419138 -87.66492701074016) bank159309 +159310 POINT(41.30607460435699 -86.76945747167245) bank159310 +159311 POINT(42.36263526448413 -88.078997805958) bank159311 +159312 POINT(41.13873678034342 -86.65565506318521) bank159312 +159313 POINT(41.65499335745839 -86.84135499467514) bank159313 +159314 POINT(40.89951362196443 -88.45450220944488) bank159314 +159315 POINT(42.808965093499424 -88.58032936283016) bank159315 +159316 POINT(41.55434558647819 -88.53842374574418) bank159316 +159317 POINT(41.759660949584806 -86.96766391531436) bank159317 +159318 POINT(42.03821875329118 -87.56775519051587) bank159318 +159319 POINT(42.784772381228294 -88.20804206613748) bank159319 +159320 POINT(42.06274282128214 -88.61017782745769) bank159320 +159321 POINT(41.800839090603205 -86.88704367668095) bank159321 +159322 POINT(42.673098123681015 -87.3840026033956) bank159322 +159323 POINT(41.097697891584446 -87.75788956005594) bank159323 +159324 POINT(41.78195407769683 -86.92569946194668) bank159324 +159325 POINT(41.40791329903847 -87.7824372541706) bank159325 +159326 POINT(41.30604233596187 -87.85056209456054) bank159326 +159327 POINT(41.226754066102544 -87.31259002283491) bank159327 +159328 POINT(41.34034052173094 -88.21443334815169) bank159328 +159329 POINT(41.075659935781324 -87.46004471192504) bank159329 +159330 POINT(41.51342175794971 -88.185938396977) bank159330 +159331 POINT(41.55664254607512 -86.91964767908382) bank159331 +159332 POINT(42.09807758864432 -87.94087725280487) bank159332 +159333 POINT(41.54619492390895 -87.04788910723582) bank159333 +159334 POINT(41.568042240260816 -88.52941671618511) bank159334 +159335 POINT(40.96078446073385 -87.60396298920534) bank159335 +159336 POINT(41.42367386802257 -87.40102583921048) bank159336 +159337 POINT(41.11868752895761 -87.32268304214823) bank159337 +159338 POINT(41.60579097661599 -87.7485650829375) bank159338 +159339 POINT(40.996781977252716 -87.54361675449154) bank159339 +159340 POINT(41.194926791230785 -87.97451857991435) bank159340 +159341 POINT(41.77405475909993 -88.59551446264527) bank159341 +159342 POINT(42.31182974039417 -87.81475861959564) bank159342 +159343 POINT(41.334154796841325 -88.30401729606734) bank159343 +159344 POINT(42.10275629573533 -87.06170903753853) bank159344 +159345 POINT(42.52379475635372 -88.55834070065605) bank159345 +159346 POINT(42.20033342669766 -87.60694897107088) bank159346 +159347 POINT(41.02183584875449 -88.48566113559463) bank159347 +159348 POINT(41.85207173676857 -87.70323950820195) bank159348 +159349 POINT(42.12168757726439 -86.87093361129938) bank159349 +159350 POINT(41.04783766695793 -88.10944492598533) bank159350 +159351 POINT(41.71933902014333 -88.44576567708462) bank159351 +159352 POINT(42.613017634571094 -86.72978584173119) bank159352 +159353 POINT(42.64979882679493 -87.59189492200757) bank159353 +159354 POINT(42.54963116752961 -87.92493183961328) bank159354 +159355 POINT(41.73211735833644 -88.18027526566239) bank159355 +159356 POINT(42.03090989291561 -86.87836761758302) bank159356 +159357 POINT(41.86145898965377 -88.07433848185876) bank159357 +159358 POINT(42.183450248120835 -87.68991567943274) bank159358 +159359 POINT(41.23660549811734 -87.17742429068127) bank159359 +159360 POINT(41.04151806218084 -87.26372883017022) bank159360 +159361 POINT(41.70130347645236 -87.49028054785464) bank159361 +159362 POINT(41.72028956123681 -87.02629990220045) bank159362 +159363 POINT(41.48340634688658 -88.37990333118485) bank159363 +159364 POINT(41.7141690496805 -87.82109622618485) bank159364 +159365 POINT(42.85427472760411 -86.97710146566305) bank159365 +159366 POINT(41.5319701656729 -88.61994652186051) bank159366 +159367 POINT(41.393227897349576 -88.19959337832961) bank159367 +159368 POINT(42.310137198943444 -88.02794073997428) bank159368 +159369 POINT(41.3743047090354 -88.2231964094545) bank159369 +159370 POINT(41.87132431302573 -86.7509881137913) bank159370 +159371 POINT(41.129848614126075 -88.31897618580459) bank159371 +159372 POINT(41.589524845707665 -87.80612031658018) bank159372 +159373 POINT(40.87944483737964 -88.40047616573675) bank159373 +159374 POINT(41.01811262328156 -86.66604681013203) bank159374 +159375 POINT(42.58186712342897 -87.68590103140859) bank159375 +159376 POINT(42.78025346575416 -87.06847147142331) bank159376 +159377 POINT(41.73733608303753 -88.43646368570343) bank159377 +159378 POINT(42.568330760895456 -87.20049748563399) bank159378 +159379 POINT(42.562111644052266 -86.82669976386939) bank159379 +159380 POINT(42.83599620305242 -88.14896191000638) bank159380 +159381 POINT(42.19710394995437 -87.52161104382729) bank159381 +159382 POINT(41.092859326866126 -87.82988503625714) bank159382 +159383 POINT(42.391780431321116 -87.4911119753883) bank159383 +159384 POINT(42.74946301308037 -88.23893902590957) bank159384 +159385 POINT(42.53134614676459 -88.44013358772517) bank159385 +159386 POINT(41.272585294615304 -87.75609231712656) bank159386 +159387 POINT(42.51526452112069 -88.41316583161033) bank159387 +159388 POINT(41.98992264812884 -87.74939106273092) bank159388 +159389 POINT(41.76122682750077 -86.75862088112777) bank159389 +159390 POINT(41.43096317038993 -87.52460473317517) bank159390 +159391 POINT(41.220866336859245 -87.96863736163588) bank159391 +159392 POINT(41.76130024926503 -88.57288701043096) bank159392 +159393 POINT(42.53878228382154 -88.1728385793057) bank159393 +159394 POINT(41.88734169361379 -88.003157077993) bank159394 +159395 POINT(42.15066361964449 -87.68429503560805) bank159395 +159396 POINT(42.429829315547586 -88.54826186527303) bank159396 +159397 POINT(42.87427271878068 -88.18242849827298) bank159397 +159398 POINT(42.12239138919101 -88.32353060073812) bank159398 +159399 POINT(42.568003802303934 -88.45265125288441) bank159399 +159400 POINT(42.45522158846857 -87.67610528215359) bank159400 +159401 POINT(42.618963581716095 -87.72582834621436) bank159401 +159402 POINT(41.554584709718206 -88.035361467057) bank159402 +159403 POINT(42.08629491901267 -88.11243510844086) bank159403 +159404 POINT(42.68405614408568 -86.92398231346797) bank159404 +159405 POINT(41.88436636808114 -88.33071846535965) bank159405 +159406 POINT(41.96224761417753 -87.24411831281242) bank159406 +159407 POINT(41.491591601859135 -87.29001698888503) bank159407 +159408 POINT(41.94989415303498 -86.71529079698404) bank159408 +159409 POINT(41.21299069297653 -87.00832587873055) bank159409 +159410 POINT(41.10186518683257 -88.53036623900863) bank159410 +159411 POINT(42.86706167129099 -88.61047018671111) bank159411 +159412 POINT(42.16567683539979 -87.51222863443407) bank159412 +159413 POINT(41.43890781185275 -87.76651262271649) bank159413 +159414 POINT(42.281556019577266 -88.04497047569622) bank159414 +159415 POINT(41.3105878976672 -87.75542126191553) bank159415 +159416 POINT(41.1329375498918 -87.92363933685985) bank159416 +159417 POINT(42.52745662684371 -87.32431568890024) bank159417 +159418 POINT(41.173898308649754 -86.84207721848846) bank159418 +159419 POINT(41.502786713517516 -87.04520457712813) bank159419 +159420 POINT(41.535879152271605 -88.31512308914311) bank159420 +159421 POINT(41.14795082535442 -87.16481557620276) bank159421 +159422 POINT(42.34088049575879 -87.61240142689326) bank159422 +159423 POINT(41.47105673877051 -87.33534096560015) bank159423 +159424 POINT(41.640335552981995 -88.62960519160625) bank159424 +159425 POINT(41.99589384290239 -87.07211012249044) bank159425 +159426 POINT(41.78477672646057 -87.43316186048625) bank159426 +159427 POINT(41.63949397299337 -88.04563934584957) bank159427 +159428 POINT(41.718528631012965 -86.66405580918583) bank159428 +159429 POINT(41.07182544181414 -87.4342115418217) bank159429 +159430 POINT(41.701157867779145 -88.11141228465749) bank159430 +159431 POINT(42.11105263255266 -88.0013783181027) bank159431 +159432 POINT(41.19522560944389 -87.03378212438243) bank159432 +159433 POINT(42.72858512979769 -87.51077411941031) bank159433 +159434 POINT(42.84741711935243 -86.71624250411189) bank159434 +159435 POINT(41.27137311760173 -87.83973446793259) bank159435 +159436 POINT(41.50513582552237 -86.69730781510171) bank159436 +159437 POINT(42.688588603638806 -87.46019169881679) bank159437 +159438 POINT(42.202327832079675 -88.16691976421976) bank159438 +159439 POINT(40.95925350298734 -88.12272814880184) bank159439 +159440 POINT(41.28276186592883 -88.06872894755136) bank159440 +159441 POINT(42.06568361013666 -87.13232957482158) bank159441 +159442 POINT(41.11347662171864 -87.09984261352109) bank159442 +159443 POINT(42.3559099636693 -87.46065422835912) bank159443 +159444 POINT(42.19027910926908 -88.03333931217983) bank159444 +159445 POINT(41.8112173194506 -86.85875955639844) bank159445 +159446 POINT(41.936021832238616 -86.88719369509681) bank159446 +159447 POINT(41.9127542509916 -87.17144344089898) bank159447 +159448 POINT(41.103792851870395 -88.18102178949036) bank159448 +159449 POINT(41.291730638396224 -88.11967027322801) bank159449 +159450 POINT(42.26680069799988 -87.57123496806959) bank159450 +159451 POINT(41.39575125152964 -87.47948141688099) bank159451 +159452 POINT(41.87599523307064 -87.58140413802292) bank159452 +159453 POINT(41.59610020474649 -87.85671436485364) bank159453 +159454 POINT(42.70435487243562 -87.92805630285746) bank159454 +159455 POINT(42.22361891978302 -87.71968102631874) bank159455 +159456 POINT(41.8235928996265 -87.69653313110035) bank159456 +159457 POINT(41.398582079589296 -86.87715220800102) bank159457 +159458 POINT(41.413408822582966 -88.38850564880552) bank159458 +159459 POINT(41.594903045498235 -87.53042494368421) bank159459 +159460 POINT(41.327791872742985 -87.71443766324998) bank159460 +159461 POINT(42.453372697250636 -87.84417429407193) bank159461 +159462 POINT(41.263952675621596 -87.46277942694448) bank159462 +159463 POINT(42.71829238539461 -87.43110067875295) bank159463 +159464 POINT(42.650011726878134 -87.035898654135) bank159464 +159465 POINT(41.02120354513927 -88.10572592828969) bank159465 +159466 POINT(42.659688066913624 -88.30974760541403) bank159466 +159467 POINT(41.25280294331696 -87.75890823601519) bank159467 +159468 POINT(41.13871512565237 -87.31923444168388) bank159468 +159469 POINT(41.534327151761836 -87.92796071442963) bank159469 +159470 POINT(41.88201319010344 -87.09424301367471) bank159470 +159471 POINT(41.5709183870316 -87.53194793349506) bank159471 +159472 POINT(41.21084524350144 -87.8639163484463) bank159472 +159473 POINT(42.39536307477805 -88.33048465030605) bank159473 +159474 POINT(42.779728520889144 -86.93825143318591) bank159474 +159475 POINT(41.13816191522884 -87.24248585454144) bank159475 +159476 POINT(41.79972949113039 -87.97288406856474) bank159476 +159477 POINT(41.87174859173738 -86.79370801531009) bank159477 +159478 POINT(41.4973215722262 -88.3928188698849) bank159478 +159479 POINT(40.981237299439414 -86.80610203464794) bank159479 +159480 POINT(41.976604876975856 -87.45298627596387) bank159480 +159481 POINT(41.430138287569456 -88.1304344721041) bank159481 +159482 POINT(42.04099586597193 -88.2135825167907) bank159482 +159483 POINT(42.648986062478556 -88.37562440078257) bank159483 +159484 POINT(42.697125762973535 -87.15272915336601) bank159484 +159485 POINT(42.85751258943127 -87.15406517777204) bank159485 +159486 POINT(42.804628597997706 -87.39571605832684) bank159486 +159487 POINT(41.768066590462354 -87.46458328131709) bank159487 +159488 POINT(41.94504537232336 -87.03138698833855) bank159488 +159489 POINT(42.83347525530782 -86.80539549512122) bank159489 +159490 POINT(40.97688918780554 -88.11562577676159) bank159490 +159491 POINT(41.52814453915638 -87.0237713623182) bank159491 +159492 POINT(42.63233930759909 -87.54496062899659) bank159492 +159493 POINT(42.19418954789618 -87.5305062940068) bank159493 +159494 POINT(41.91494901219741 -87.88704739427112) bank159494 +159495 POINT(41.54036267781397 -88.0371191479457) bank159495 +159496 POINT(42.22914970890727 -88.09893831325766) bank159496 +159497 POINT(41.03201099214448 -87.98373640458463) bank159497 +159498 POINT(41.31702451151522 -88.57704596878978) bank159498 +159499 POINT(41.3132774116386 -87.81096912951206) bank159499 +159500 POINT(42.71467740452026 -88.3938674166156) bank159500 +159501 POINT(42.134863960824354 -86.71565441497243) bank159501 +159502 POINT(42.412113413712945 -88.12513077497347) bank159502 +159503 POINT(41.71918051020486 -87.81300553016426) bank159503 +159504 POINT(42.21690595992654 -87.57555161224928) bank159504 +159505 POINT(41.98581818267971 -87.72211934046268) bank159505 +159506 POINT(41.56751282493472 -88.57299107988167) bank159506 +159507 POINT(42.49075647678779 -88.06984466350083) bank159507 +159508 POINT(42.81636693343181 -87.70475378538018) bank159508 +159509 POINT(40.90972385297295 -87.35813243729156) bank159509 +159510 POINT(42.468155614140514 -88.34189210012808) bank159510 +159511 POINT(41.17430089521124 -88.40065000162843) bank159511 +159512 POINT(41.18228785098188 -86.98510531880741) bank159512 +159513 POINT(42.20024837381424 -87.62873381243325) bank159513 +159514 POINT(42.328285509500134 -86.87616914338786) bank159514 +159515 POINT(41.66352509640394 -86.75565126599176) bank159515 +159516 POINT(42.0089419727061 -87.97735768572394) bank159516 +159517 POINT(42.50563878533889 -87.64456078659335) bank159517 +159518 POINT(42.83773949430438 -86.93764579106897) bank159518 +159519 POINT(41.95562397553836 -88.565447292072) bank159519 +159520 POINT(42.05218049165133 -87.54887588900532) bank159520 +159521 POINT(42.74733175271955 -87.74088038495697) bank159521 +159522 POINT(41.90509435277931 -86.71727663620902) bank159522 +159523 POINT(41.39219978204939 -86.9235520578586) bank159523 +159524 POINT(42.802201546439385 -86.98593447905718) bank159524 +159525 POINT(41.446638689883606 -88.40919530641295) bank159525 +159526 POINT(41.05361780384714 -88.62961948303942) bank159526 +159527 POINT(42.76648501431827 -87.59198581054552) bank159527 +159528 POINT(42.448738350445915 -88.2907318109679) bank159528 +159529 POINT(41.993433373898114 -88.35877112254742) bank159529 +159530 POINT(41.38626853760387 -86.8069809817737) bank159530 +159531 POINT(40.927171698905276 -87.88987810891115) bank159531 +159532 POINT(41.093533879702925 -86.64167482770483) bank159532 +159533 POINT(42.58813324584679 -87.17838389096195) bank159533 +159534 POINT(42.181139303828026 -87.83555677361075) bank159534 +159535 POINT(41.56285025104675 -87.30830517487563) bank159535 +159536 POINT(41.22422675545955 -88.5380150369989) bank159536 +159537 POINT(42.70770088150803 -88.08437952928838) bank159537 +159538 POINT(41.42807731058552 -88.3675973674237) bank159538 +159539 POINT(41.14870791060565 -87.6457221309988) bank159539 +159540 POINT(41.209910683480025 -88.25012221531588) bank159540 +159541 POINT(41.3140439742426 -88.10384183743933) bank159541 +159542 POINT(41.45977032170155 -87.43910774735124) bank159542 +159543 POINT(42.206508571408875 -87.57451032372119) bank159543 +159544 POINT(41.593860893302754 -88.42657328165402) bank159544 +159545 POINT(41.5438449060453 -87.41765911197811) bank159545 +159546 POINT(41.436259718614345 -87.89558680725153) bank159546 +159547 POINT(42.827710618228295 -87.54076243253967) bank159547 +159548 POINT(42.71884198242428 -88.46398036210358) bank159548 +159549 POINT(42.52045707956942 -87.09392716264851) bank159549 +159550 POINT(42.78956571997938 -88.43708082876073) bank159550 +159551 POINT(41.08319650668452 -87.1589443118461) bank159551 +159552 POINT(41.73322626623145 -88.34110422613506) bank159552 +159553 POINT(41.6324962426498 -87.68516219780366) bank159553 +159554 POINT(42.62869071308553 -88.57001098397043) bank159554 +159555 POINT(41.247275367267605 -86.66514878560537) bank159555 +159556 POINT(42.42181537377336 -88.12136096065548) bank159556 +159557 POINT(42.491488220688 -88.53715428210545) bank159557 +159558 POINT(41.26455064022242 -87.79090732169072) bank159558 +159559 POINT(41.011244719521166 -88.13086914978076) bank159559 +159560 POINT(41.11377831895528 -87.73296702885337) bank159560 +159561 POINT(42.297487383028525 -88.52518373051176) bank159561 +159562 POINT(42.2246409991625 -87.6478423558906) bank159562 +159563 POINT(41.74932796317984 -88.52989537587723) bank159563 +159564 POINT(41.47791052883585 -87.30095982936977) bank159564 +159565 POINT(42.111504358409526 -88.61313208182594) bank159565 +159566 POINT(41.84003878241491 -86.76398440371628) bank159566 +159567 POINT(41.42228626829564 -87.3902776814659) bank159567 +159568 POINT(42.24633366310751 -87.17282124978077) bank159568 +159569 POINT(41.74263624864148 -86.97306630540977) bank159569 +159570 POINT(42.81354954979447 -87.1716458162795) bank159570 +159571 POINT(41.032169461307376 -87.93672421287029) bank159571 +159572 POINT(41.072533674348044 -88.16070050945994) bank159572 +159573 POINT(42.77112368757112 -88.1868039309948) bank159573 +159574 POINT(41.28034571567034 -86.840581928076) bank159574 +159575 POINT(41.03770781598851 -88.62948148528243) bank159575 +159576 POINT(41.222255010435894 -87.48275867823419) bank159576 +159577 POINT(41.549331060834106 -87.1317985218126) bank159577 +159578 POINT(41.32781165686741 -88.19251748847941) bank159578 +159579 POINT(41.14579382876111 -87.84389783973698) bank159579 +159580 POINT(42.05108795982781 -88.40595723936339) bank159580 +159581 POINT(41.37081192693306 -86.68302491732092) bank159581 +159582 POINT(42.1737936848035 -88.51875055838367) bank159582 +159583 POINT(41.753818643905944 -87.47160913288745) bank159583 +159584 POINT(41.29122978447506 -87.58601232585438) bank159584 +159585 POINT(42.17971291001852 -88.0703595135048) bank159585 +159586 POINT(40.87994187395003 -88.05322125013913) bank159586 +159587 POINT(41.53870474956879 -87.86914397261546) bank159587 +159588 POINT(40.957912595962334 -87.26827028968538) bank159588 +159589 POINT(42.24320369825371 -88.27541849273172) bank159589 +159590 POINT(41.59002725493271 -87.79489136694446) bank159590 +159591 POINT(41.67381575809752 -87.48256319334912) bank159591 +159592 POINT(41.15610440378535 -87.63692800847052) bank159592 +159593 POINT(41.41746197406222 -88.29514721952269) bank159593 +159594 POINT(41.08975379022579 -87.88280998138434) bank159594 +159595 POINT(41.66111957640793 -87.40819791299154) bank159595 +159596 POINT(41.801282645768524 -87.64342187869107) bank159596 +159597 POINT(42.07047960225559 -86.8342104019124) bank159597 +159598 POINT(42.778702084426214 -88.37421558672649) bank159598 +159599 POINT(42.73659174114989 -87.95629026771225) bank159599 +159600 POINT(40.913022880772616 -87.34971088959684) bank159600 +159601 POINT(42.83882915059288 -88.29535495347753) bank159601 +159602 POINT(41.37682218208936 -87.87257345293041) bank159602 +159603 POINT(41.73159961993744 -88.23418208818151) bank159603 +159604 POINT(42.76109591476841 -88.04847652156921) bank159604 +159605 POINT(40.99782904939522 -87.52668849120128) bank159605 +159606 POINT(41.09656374157848 -88.27556389789373) bank159606 +159607 POINT(42.297712519345154 -88.25597184073003) bank159607 +159608 POINT(41.85605648637998 -88.25767375074177) bank159608 +159609 POINT(41.53940289389894 -87.63121849832957) bank159609 +159610 POINT(42.69593542214626 -87.38311732812397) bank159610 +159611 POINT(41.1557356645558 -87.22044007674202) bank159611 +159612 POINT(41.651477850741344 -87.28143170562736) bank159612 +159613 POINT(41.099482769482776 -87.91517421046105) bank159613 +159614 POINT(42.55222370623823 -87.71681670022818) bank159614 +159615 POINT(42.167642605659495 -87.78510665390162) bank159615 +159616 POINT(41.7797028743402 -87.99801327212926) bank159616 +159617 POINT(42.40924102361964 -86.69310596556463) bank159617 +159618 POINT(41.88946430081551 -86.91500861999828) bank159618 +159619 POINT(42.62719297614626 -87.75991613439325) bank159619 +159620 POINT(41.534511201813224 -88.25202530700288) bank159620 +159621 POINT(41.66608546601123 -88.53870833317158) bank159621 +159622 POINT(41.82846677859558 -87.31002926405556) bank159622 +159623 POINT(42.02696410726454 -86.87382608553894) bank159623 +159624 POINT(42.72626448362667 -88.07432271852379) bank159624 +159625 POINT(41.64426342073309 -87.9481258783706) bank159625 +159626 POINT(42.3416017231864 -87.57846596077431) bank159626 +159627 POINT(42.04914771567448 -88.46372185157085) bank159627 +159628 POINT(41.33248662659758 -86.67722819001523) bank159628 +159629 POINT(42.85579898627062 -88.46228494852505) bank159629 +159630 POINT(42.19411153218984 -87.8289451402013) bank159630 +159631 POINT(42.62986206103583 -88.30766793228818) bank159631 +159632 POINT(41.89788003797969 -86.89777403580236) bank159632 +159633 POINT(41.05196933860428 -87.4158536645109) bank159633 +159634 POINT(42.37007922323036 -87.09901177009417) bank159634 +159635 POINT(41.17981690332167 -87.08148178194955) bank159635 +159636 POINT(41.05224240698314 -87.6997635831437) bank159636 +159637 POINT(41.60343852112692 -87.1795173605568) bank159637 +159638 POINT(42.092315289014095 -88.48601616673515) bank159638 +159639 POINT(41.63179640991881 -87.95108270663013) bank159639 +159640 POINT(41.69493432554259 -86.65201695969104) bank159640 +159641 POINT(42.112878182812715 -88.3633425412231) bank159641 +159642 POINT(42.72543471781579 -87.52614742736955) bank159642 +159643 POINT(41.49561116038861 -87.2469255984685) bank159643 +159644 POINT(42.183171305088635 -87.03322495366906) bank159644 +159645 POINT(41.5017061133158 -86.74684283290165) bank159645 +159646 POINT(41.03254362853298 -88.38349387480558) bank159646 +159647 POINT(42.37361926448412 -86.95490420324657) bank159647 +159648 POINT(42.202828203516894 -86.6417733849356) bank159648 +159649 POINT(40.93505133462979 -88.37718808262025) bank159649 +159650 POINT(42.5378196763228 -87.00055106834414) bank159650 +159651 POINT(41.341161704938955 -87.76413789760139) bank159651 +159652 POINT(41.22925228649697 -87.077551776636) bank159652 +159653 POINT(40.9352240402756 -87.82517323681239) bank159653 +159654 POINT(41.562931691414235 -87.29939551190999) bank159654 +159655 POINT(42.70984106878981 -88.23870694582351) bank159655 +159656 POINT(41.85697343288931 -87.10159241446756) bank159656 +159657 POINT(41.72023749453602 -87.41881438801026) bank159657 +159658 POINT(42.087323594779065 -87.82750033621346) bank159658 +159659 POINT(42.39630086406556 -87.48032808640029) bank159659 +159660 POINT(41.94559513268432 -87.5898239284919) bank159660 +159661 POINT(41.541211916312484 -88.29294165568511) bank159661 +159662 POINT(42.40928946258722 -87.37659700348622) bank159662 +159663 POINT(42.544824708673154 -88.29214793513995) bank159663 +159664 POINT(42.86034053417814 -87.89084682238753) bank159664 +159665 POINT(42.615791936595876 -87.12470314895377) bank159665 +159666 POINT(41.49077646070564 -88.06648312613063) bank159666 +159667 POINT(41.76715931174712 -88.14602206663497) bank159667 +159668 POINT(41.20568632965457 -88.17381904768538) bank159668 +159669 POINT(41.829233873605695 -87.12549352658839) bank159669 +159670 POINT(42.648779949565665 -86.6791031955152) bank159670 +159671 POINT(41.180628776423674 -88.02999885786274) bank159671 +159672 POINT(41.630397841493014 -86.64136618916483) bank159672 +159673 POINT(42.0550470203648 -87.37566731039729) bank159673 +159674 POINT(41.96753950227945 -88.25284181248516) bank159674 +159675 POINT(42.69383009286778 -87.82218926550732) bank159675 +159676 POINT(42.78680798763723 -87.96076385840787) bank159676 +159677 POINT(42.048706246866615 -86.78966246284241) bank159677 +159678 POINT(41.58394623765981 -88.0853227637472) bank159678 +159679 POINT(41.18260251664785 -88.59041093470151) bank159679 +159680 POINT(41.703966794508716 -87.87825757346482) bank159680 +159681 POINT(42.56278838104852 -88.52545016862626) bank159681 +159682 POINT(42.68454080001685 -87.09841587538864) bank159682 +159683 POINT(41.28325740272282 -87.64991689433974) bank159683 +159684 POINT(41.21362378701204 -88.1579737545348) bank159684 +159685 POINT(42.18965413575607 -88.13822127522633) bank159685 +159686 POINT(42.107412706702156 -88.102945268259) bank159686 +159687 POINT(41.72329580404919 -88.38902553379494) bank159687 +159688 POINT(41.29532310245443 -88.05638761720624) bank159688 +159689 POINT(40.92700299637647 -87.14211394659829) bank159689 +159690 POINT(41.7887045434504 -88.62640273817675) bank159690 +159691 POINT(42.18070340310666 -87.87236321904628) bank159691 +159692 POINT(42.84846471358562 -86.6607755918537) bank159692 +159693 POINT(41.62715154237078 -87.7863539701156) bank159693 +159694 POINT(41.21305651087254 -88.22350728589771) bank159694 +159695 POINT(42.638715604713774 -87.51014907952644) bank159695 +159696 POINT(41.28619347996963 -87.06106600948644) bank159696 +159697 POINT(42.36781433861387 -87.4147804870441) bank159697 +159698 POINT(41.14974533505216 -86.84990888807421) bank159698 +159699 POINT(42.16411002858039 -88.53981015211235) bank159699 +159700 POINT(41.203151407006196 -87.0900964885937) bank159700 +159701 POINT(42.82628516719051 -87.6640097985166) bank159701 +159702 POINT(42.48859790324783 -88.098947192523) bank159702 +159703 POINT(41.313920282067365 -87.59644466851866) bank159703 +159704 POINT(42.39966861567303 -87.92429191324474) bank159704 +159705 POINT(41.75730353871363 -87.12591108901401) bank159705 +159706 POINT(41.41535627598152 -86.98745840288854) bank159706 +159707 POINT(41.15907525123386 -88.02458606746661) bank159707 +159708 POINT(40.90266493904147 -87.90746997240845) bank159708 +159709 POINT(41.17255858251504 -88.359049231556) bank159709 +159710 POINT(41.136935148165136 -87.9112804756683) bank159710 +159711 POINT(42.776179324795905 -88.05605358190654) bank159711 +159712 POINT(41.26203588217365 -88.47493594135788) bank159712 +159713 POINT(41.7061591922429 -88.46878641500483) bank159713 +159714 POINT(41.95960486199914 -87.43081882150605) bank159714 +159715 POINT(42.345230622200944 -87.96004238538842) bank159715 +159716 POINT(42.557007267263025 -87.28643898521467) bank159716 +159717 POINT(41.02041879039115 -86.94988969491308) bank159717 +159718 POINT(40.94722383292287 -86.81571823932566) bank159718 +159719 POINT(41.880841045522416 -87.42312542679578) bank159719 +159720 POINT(41.17395234316006 -88.01203245710992) bank159720 +159721 POINT(41.395010352808974 -88.1410321454721) bank159721 +159722 POINT(41.408254534942174 -88.57137137647469) bank159722 +159723 POINT(41.403585968941435 -87.11094499823179) bank159723 +159724 POINT(42.46496773418205 -88.26489606791496) bank159724 +159725 POINT(42.77804488098445 -87.19401920155897) bank159725 +159726 POINT(42.84661342407971 -88.43409898609082) bank159726 +159727 POINT(41.2548383467792 -86.90382316712198) bank159727 +159728 POINT(42.51261477710124 -88.16424404668304) bank159728 +159729 POINT(42.389432751432786 -88.36614305389874) bank159729 +159730 POINT(41.87020770292918 -86.89444836114204) bank159730 +159731 POINT(42.25741891800707 -87.98177961536081) bank159731 +159732 POINT(41.23568901446028 -88.40058250400917) bank159732 +159733 POINT(41.30242958476374 -86.74029823597506) bank159733 +159734 POINT(41.93537621177042 -87.38949524772718) bank159734 +159735 POINT(42.87123175803346 -86.80879952406659) bank159735 +159736 POINT(41.2695277415392 -87.4134804114858) bank159736 +159737 POINT(41.611024776447394 -86.974774928636) bank159737 +159738 POINT(42.41515874587689 -87.7036634940454) bank159738 +159739 POINT(41.19732724071371 -87.87828341912497) bank159739 +159740 POINT(41.647083364074426 -86.70682614948525) bank159740 +159741 POINT(42.84386359558587 -88.35130683911943) bank159741 +159742 POINT(42.03805581360923 -87.02038804992512) bank159742 +159743 POINT(41.6414420788664 -88.12526574808577) bank159743 +159744 POINT(42.551960926596465 -88.33667975841942) bank159744 +159745 POINT(42.58708254238332 -87.99054294925097) bank159745 +159746 POINT(42.28594623866281 -88.05940357959747) bank159746 +159747 POINT(41.31707287633368 -86.97354102767966) bank159747 +159748 POINT(42.053330026297104 -88.17004756181419) bank159748 +159749 POINT(42.481613656713435 -88.53291200215943) bank159749 +159750 POINT(42.005714998774344 -88.30782045370681) bank159750 +159751 POINT(41.445621139581654 -88.45956130533905) bank159751 +159752 POINT(41.217511518900146 -86.82144236938767) bank159752 +159753 POINT(41.96052342655155 -87.36343175840175) bank159753 +159754 POINT(41.8197186102274 -87.2860616854782) bank159754 +159755 POINT(42.71128651623306 -86.68225355567644) bank159755 +159756 POINT(41.357664979765346 -87.76663315240819) bank159756 +159757 POINT(41.3143094196728 -87.37183752188021) bank159757 +159758 POINT(41.84971329365253 -87.71829392131121) bank159758 +159759 POINT(42.27824610479051 -87.6218329360764) bank159759 +159760 POINT(41.50810006108081 -88.08085240134008) bank159760 +159761 POINT(41.142097246724276 -87.26415920480223) bank159761 +159762 POINT(42.65160597195817 -87.58177864674211) bank159762 +159763 POINT(41.10470632321985 -86.82279415824203) bank159763 +159764 POINT(42.68687906335345 -87.83206228083188) bank159764 +159765 POINT(41.841771537594596 -86.92533851586334) bank159765 +159766 POINT(42.80248393396513 -86.82364722735738) bank159766 +159767 POINT(41.12665516695774 -87.4563826150111) bank159767 +159768 POINT(40.90449335811388 -88.16933319108159) bank159768 +159769 POINT(41.913992235963356 -88.37418233781307) bank159769 +159770 POINT(42.22316923651984 -88.5152761143857) bank159770 +159771 POINT(42.58527224471771 -88.14769008679437) bank159771 +159772 POINT(41.42901667287748 -88.34575185132125) bank159772 +159773 POINT(41.497402594527316 -86.99398077671881) bank159773 +159774 POINT(40.950576092269955 -87.89043347955483) bank159774 +159775 POINT(41.39569619596635 -87.86403381249475) bank159775 +159776 POINT(42.714924205361996 -87.74249978348765) bank159776 +159777 POINT(41.438908580098804 -88.31010060803854) bank159777 +159778 POINT(42.54995854278198 -87.06364726227831) bank159778 +159779 POINT(42.018337974383236 -88.07100644433194) bank159779 +159780 POINT(41.32930112758012 -87.81462314744907) bank159780 +159781 POINT(40.941914527477 -88.58207011834658) bank159781 +159782 POINT(42.77008215798987 -86.86178201326018) bank159782 +159783 POINT(41.99306060565239 -87.85358224550335) bank159783 +159784 POINT(41.71212313167352 -88.569467321085) bank159784 +159785 POINT(41.29843214903376 -88.28182479546514) bank159785 +159786 POINT(40.99920736842657 -87.69231833114603) bank159786 +159787 POINT(42.34706580894589 -86.83622447119356) bank159787 +159788 POINT(42.18582110441587 -87.70455894519641) bank159788 +159789 POINT(41.30044129348795 -86.77125877453994) bank159789 +159790 POINT(41.819489550016634 -87.22177685915595) bank159790 +159791 POINT(41.181938740950926 -86.9005905442798) bank159791 +159792 POINT(41.44659784679422 -87.94239191609684) bank159792 +159793 POINT(42.805448237104244 -87.04623331630164) bank159793 +159794 POINT(42.256391240177294 -87.42165475511139) bank159794 +159795 POINT(41.79407424676011 -86.81571723977358) bank159795 +159796 POINT(41.945599742595356 -87.81160094060989) bank159796 +159797 POINT(41.39679812641416 -87.7367142923161) bank159797 +159798 POINT(41.749560126083736 -86.99611520206584) bank159798 +159799 POINT(41.885638933034656 -88.56825099876863) bank159799 +159800 POINT(42.858278827611194 -87.21153742764945) bank159800 +159801 POINT(42.471174170495395 -87.7515933730511) bank159801 +159802 POINT(42.18630944957222 -87.58929921596375) bank159802 +159803 POINT(42.83032108649056 -87.02038694718242) bank159803 +159804 POINT(41.040457184431276 -88.44146135897036) bank159804 +159805 POINT(41.94101030269262 -87.83165563581686) bank159805 +159806 POINT(42.4178128754758 -88.37794924444107) bank159806 +159807 POINT(42.466830159956615 -87.23532431574266) bank159807 +159808 POINT(42.6143234286771 -88.45150266733032) bank159808 +159809 POINT(41.13594343283339 -86.78228925895188) bank159809 +159810 POINT(42.01730506043828 -87.84839033932616) bank159810 +159811 POINT(42.30422802282951 -87.88733188401991) bank159811 +159812 POINT(41.330671241833514 -88.07272033239647) bank159812 +159813 POINT(42.082173477193145 -87.6033231838075) bank159813 +159814 POINT(42.82928323536485 -87.85483267279658) bank159814 +159815 POINT(42.4769850802904 -86.95084857550464) bank159815 +159816 POINT(40.98868971192737 -88.2870645403616) bank159816 +159817 POINT(42.804599368057644 -86.84321114464902) bank159817 +159818 POINT(41.358984795436854 -87.69509043303822) bank159818 +159819 POINT(42.009618786699576 -88.17144055892038) bank159819 +159820 POINT(42.23931727785247 -86.70115460715292) bank159820 +159821 POINT(41.10004324597067 -88.03328714733117) bank159821 +159822 POINT(41.228702733063976 -88.58572073513565) bank159822 +159823 POINT(41.98235221090809 -87.99635523752166) bank159823 +159824 POINT(42.41746805004669 -88.41904632080679) bank159824 +159825 POINT(42.87512630919098 -86.63627529753167) bank159825 +159826 POINT(40.97247473660186 -86.75730858389466) bank159826 +159827 POINT(41.88151565742271 -86.82035854003537) bank159827 +159828 POINT(41.104223789691304 -88.23452066926119) bank159828 +159829 POINT(41.56405929559737 -86.85373181565735) bank159829 +159830 POINT(42.34246131209166 -87.94859361630753) bank159830 +159831 POINT(41.62668406068994 -86.8036677006165) bank159831 +159832 POINT(41.7658811285838 -87.89491988546928) bank159832 +159833 POINT(41.5825189659077 -86.8537076527459) bank159833 +159834 POINT(41.575583133857506 -88.0568679465055) bank159834 +159835 POINT(41.90116595486126 -87.90448942554708) bank159835 +159836 POINT(41.42785931243734 -87.93263401181272) bank159836 +159837 POINT(42.3297960085377 -87.64779441654646) bank159837 +159838 POINT(41.389216489130035 -87.76071291529135) bank159838 +159839 POINT(41.932815002151806 -86.9812165983889) bank159839 +159840 POINT(42.14670523510853 -87.69559959884675) bank159840 +159841 POINT(41.666503918906585 -87.4359608782802) bank159841 +159842 POINT(42.48983034635706 -88.05056745553169) bank159842 +159843 POINT(41.17264569509108 -88.43877576010675) bank159843 +159844 POINT(42.764991408559986 -88.59657410441724) bank159844 +159845 POINT(41.19718439494048 -87.45548484231286) bank159845 +159846 POINT(41.66206884189422 -88.22200545047978) bank159846 +159847 POINT(42.26817676975647 -88.1475301681373) bank159847 +159848 POINT(41.304766612404656 -87.15706958222046) bank159848 +159849 POINT(41.66434596720123 -87.85448767424342) bank159849 +159850 POINT(42.15896842744672 -86.99147009918993) bank159850 +159851 POINT(42.70221413122526 -87.88682427067171) bank159851 +159852 POINT(41.70262212013522 -86.74921153931155) bank159852 +159853 POINT(41.21813748848852 -87.98499963142292) bank159853 +159854 POINT(41.886016540328825 -88.22065606831961) bank159854 +159855 POINT(41.329875884895294 -88.33814744098709) bank159855 +159856 POINT(42.78810264696177 -88.55939009718067) bank159856 +159857 POINT(42.15418932518101 -87.54692210027058) bank159857 +159858 POINT(42.09086838313623 -86.765059203312) bank159858 +159859 POINT(40.949834729867604 -86.6605443166732) bank159859 +159860 POINT(40.943968188070876 -88.2213097728274) bank159860 +159861 POINT(42.27776892930376 -87.73859045718905) bank159861 +159862 POINT(41.25290845902696 -87.87182156624702) bank159862 +159863 POINT(42.028539411866944 -86.99759620162344) bank159863 +159864 POINT(41.93077219889023 -87.93747470252407) bank159864 +159865 POINT(42.575509785675585 -87.1152454261643) bank159865 +159866 POINT(42.57389523067186 -86.63597659264481) bank159866 +159867 POINT(41.8563821930642 -87.86434393048899) bank159867 +159868 POINT(41.96062408914842 -88.21991051770003) bank159868 +159869 POINT(41.26072733189487 -88.47441219687938) bank159869 +159870 POINT(42.066953121802555 -87.96291442573805) bank159870 +159871 POINT(42.7784858286203 -88.40144706264581) bank159871 +159872 POINT(41.42653138468359 -87.38780670355331) bank159872 +159873 POINT(41.452964484220985 -86.69276565058708) bank159873 +159874 POINT(42.531203125584774 -87.01325193144417) bank159874 +159875 POINT(41.90028604230755 -88.07705862596796) bank159875 +159876 POINT(41.204421302340556 -86.71681744542504) bank159876 +159877 POINT(41.32825328412137 -87.7520428098098) bank159877 +159878 POINT(42.43583850130908 -87.49970003274589) bank159878 +159879 POINT(42.801903815608576 -87.89666004837477) bank159879 +159880 POINT(40.94255080081916 -87.21672012965287) bank159880 +159881 POINT(42.83647219644802 -87.93215450520832) bank159881 +159882 POINT(41.42063281926986 -88.21811708852744) bank159882 +159883 POINT(42.83440257277767 -88.21495072933267) bank159883 +159884 POINT(41.062827076088965 -87.50678727102017) bank159884 +159885 POINT(42.09398594904998 -87.42559519858007) bank159885 +159886 POINT(41.59736085470804 -87.90286022319668) bank159886 +159887 POINT(41.275627183436505 -87.68355221996278) bank159887 +159888 POINT(42.0616226072314 -88.21051207656542) bank159888 +159889 POINT(41.35689983804108 -87.38952703315256) bank159889 +159890 POINT(42.234179166770154 -86.7801515663609) bank159890 +159891 POINT(40.94000881316338 -87.81137042772957) bank159891 +159892 POINT(42.74584458120224 -88.49990551815776) bank159892 +159893 POINT(42.829755301580406 -86.64191196484971) bank159893 +159894 POINT(42.63819308904142 -87.93747936113596) bank159894 +159895 POINT(42.75396333489577 -86.75385597740721) bank159895 +159896 POINT(42.812724305594706 -87.31723626058147) bank159896 +159897 POINT(42.6384697560947 -88.11923777932998) bank159897 +159898 POINT(41.64653701971169 -86.76111328420077) bank159898 +159899 POINT(42.739918229111574 -88.04669445428024) bank159899 +159900 POINT(40.946393189906054 -88.0430120652999) bank159900 +159901 POINT(42.054277939458785 -86.70845367538323) bank159901 +159902 POINT(41.50383593038975 -87.90199822251586) bank159902 +159903 POINT(42.80077800803218 -87.5760920136215) bank159903 +159904 POINT(42.72764292277865 -88.53526364308695) bank159904 +159905 POINT(41.57948244628314 -87.19948995208529) bank159905 +159906 POINT(41.7192249227256 -88.45754077300062) bank159906 +159907 POINT(41.09960878913681 -87.00834056970827) bank159907 +159908 POINT(41.70896462619098 -88.14108940823056) bank159908 +159909 POINT(41.63475372935669 -86.68861054493367) bank159909 +159910 POINT(42.800607892872854 -87.90968023859347) bank159910 +159911 POINT(42.690148176034036 -86.79571698921895) bank159911 +159912 POINT(42.403940551861474 -86.9858240984093) bank159912 +159913 POINT(41.07092032156176 -88.07329504414896) bank159913 +159914 POINT(41.282975986584184 -88.14293802327245) bank159914 +159915 POINT(42.69184457468368 -88.061790221236) bank159915 +159916 POINT(42.25999852840924 -87.95447506975916) bank159916 +159917 POINT(42.50959432155913 -88.39943130876186) bank159917 +159918 POINT(41.117763333050135 -87.37260346379469) bank159918 +159919 POINT(41.652030194078606 -87.07961719547154) bank159919 +159920 POINT(41.07240772769623 -87.5741223091692) bank159920 +159921 POINT(41.729279493913 -87.80842366449757) bank159921 +159922 POINT(41.640329182398446 -87.02884714449553) bank159922 +159923 POINT(42.7900440357305 -88.58284540642575) bank159923 +159924 POINT(42.15819800188913 -88.37968572099192) bank159924 +159925 POINT(41.3490345052964 -88.45793478017967) bank159925 +159926 POINT(41.00020178495297 -88.36044077282807) bank159926 +159927 POINT(41.04208186512643 -88.14353954444391) bank159927 +159928 POINT(42.428484985740546 -87.03286006671672) bank159928 +159929 POINT(41.76469554133374 -88.4872659378503) bank159929 +159930 POINT(41.310109917938405 -86.7974447805787) bank159930 +159931 POINT(41.17195575529851 -87.56743489090121) bank159931 +159932 POINT(41.932752455796276 -87.68487316137974) bank159932 +159933 POINT(42.211247608494695 -86.97075842357575) bank159933 +159934 POINT(41.748077766128425 -86.70714745212157) bank159934 +159935 POINT(42.46209230317128 -86.79516569182606) bank159935 +159936 POINT(41.53741776661411 -87.81854565749735) bank159936 +159937 POINT(41.32182355099539 -86.94761471394567) bank159937 +159938 POINT(42.61796164491592 -88.10439190585888) bank159938 +159939 POINT(41.608864020332504 -87.84282110728087) bank159939 +159940 POINT(41.13538941661114 -86.9794512078156) bank159940 +159941 POINT(40.93754451078925 -88.00933815588259) bank159941 +159942 POINT(41.44152487544583 -87.23562482673552) bank159942 +159943 POINT(41.907696292832135 -86.80101412494378) bank159943 +159944 POINT(41.04755161895293 -88.44377939640832) bank159944 +159945 POINT(40.90566166071221 -88.45911267020298) bank159945 +159946 POINT(42.473317137193696 -86.64374837831083) bank159946 +159947 POINT(41.93852891284172 -87.23280377802689) bank159947 +159948 POINT(42.41998283671117 -87.39832727510681) bank159948 +159949 POINT(42.62058464437816 -87.3437375591507) bank159949 +159950 POINT(41.52579184284153 -87.73738912395858) bank159950 +159951 POINT(40.92808786849145 -88.17938077586696) bank159951 +159952 POINT(42.40427384358928 -86.68869439643605) bank159952 +159953 POINT(42.86850331150473 -86.76060766791703) bank159953 +159954 POINT(42.17263297353667 -88.04026872950804) bank159954 +159955 POINT(41.91638305521438 -86.98395311499354) bank159955 +159956 POINT(42.534254169014766 -87.87202684139524) bank159956 +159957 POINT(41.94386239286709 -86.89166001745528) bank159957 +159958 POINT(42.4279790778929 -87.6270842478193) bank159958 +159959 POINT(41.397758229027815 -87.50513499934304) bank159959 +159960 POINT(41.788276355891966 -87.63495280263744) bank159960 +159961 POINT(41.24184351463845 -88.04159845113075) bank159961 +159962 POINT(42.007590685093824 -87.72286026411949) bank159962 +159963 POINT(41.394723250496185 -87.34829594232392) bank159963 +159964 POINT(40.952780574736096 -86.86416924263317) bank159964 +159965 POINT(42.52622422660987 -87.0590226203519) bank159965 +159966 POINT(42.05526895470024 -88.25521093460375) bank159966 +159967 POINT(42.3026652968638 -86.99512555354617) bank159967 +159968 POINT(41.13661446194579 -88.39791666190432) bank159968 +159969 POINT(42.39948728877741 -88.12123498686087) bank159969 +159970 POINT(42.86397444058325 -86.81631184917721) bank159970 +159971 POINT(41.156801652344285 -88.56920090417957) bank159971 +159972 POINT(42.715339964722475 -88.17739986240713) bank159972 +159973 POINT(42.0489221566888 -88.54939156340828) bank159973 +159974 POINT(41.98271937448341 -87.35101324765503) bank159974 +159975 POINT(41.37123340928406 -87.59768561117394) bank159975 +159976 POINT(42.00350702901785 -88.09039132083814) bank159976 +159977 POINT(41.61616549788771 -88.44749453476102) bank159977 +159978 POINT(41.816488959636104 -87.98891564908405) bank159978 +159979 POINT(42.27947232252938 -87.1425298486205) bank159979 +159980 POINT(42.233725749772745 -86.9945426507972) bank159980 +159981 POINT(42.009448113172574 -87.82842710939836) bank159981 +159982 POINT(42.538330521720454 -87.160544464849) bank159982 +159983 POINT(41.832048906199574 -87.96463273799002) bank159983 +159984 POINT(41.29442547455814 -87.74049051607666) bank159984 +159985 POINT(41.57359305658603 -88.42639168799607) bank159985 +159986 POINT(42.11475472530152 -87.78200491648148) bank159986 +159987 POINT(41.54614914670747 -86.70986999617945) bank159987 +159988 POINT(41.31421257535805 -88.12004147472095) bank159988 +159989 POINT(42.74183558814308 -88.05538442028575) bank159989 +159990 POINT(41.10079328983945 -87.93837968875326) bank159990 +159991 POINT(41.114564698510556 -87.6097521582183) bank159991 +159992 POINT(42.78028768654112 -87.14525335070233) bank159992 +159993 POINT(41.6536912823745 -86.81306373644995) bank159993 +159994 POINT(42.364460310608116 -87.52187053730117) bank159994 +159995 POINT(42.875848111781146 -87.33860779946278) bank159995 +159996 POINT(42.6025141675244 -86.71056443246296) bank159996 +159997 POINT(42.291207324682276 -88.21742529452652) bank159997 +159998 POINT(41.48611408886472 -87.46834726751909) bank159998 +159999 POINT(42.17865554750165 -87.55819252232007) bank159999 +160000 POINT(41.79490561286098 -87.9023074959135) bank160000 +160001 POINT(42.52632419150922 -87.31535119385366) bank160001 +160002 POINT(42.61108892015074 -87.09026849154809) bank160002 +160003 POINT(41.27177080633112 -88.62106086099276) bank160003 +160004 POINT(41.51725398191837 -86.85522840987656) bank160004 +160005 POINT(41.16429706208815 -88.54360916610855) bank160005 +160006 POINT(42.5333436792055 -87.16325496935227) bank160006 +160007 POINT(41.19507372281252 -87.09013034504927) bank160007 +160008 POINT(41.83873658769514 -88.09419001002662) bank160008 +160009 POINT(41.86745466427268 -87.97265163158025) bank160009 +160010 POINT(41.468523158272255 -88.1388083802169) bank160010 +160011 POINT(41.9800152341944 -87.27244571218887) bank160011 +160012 POINT(41.43770189984152 -86.90908810207029) bank160012 +160013 POINT(41.08800392498095 -88.00509824467063) bank160013 +160014 POINT(41.84941180715282 -87.70315240545904) bank160014 +160015 POINT(41.26685310254213 -86.97360824419142) bank160015 +160016 POINT(42.64447827160401 -87.70270346082076) bank160016 +160017 POINT(41.46998991146291 -88.58347137302377) bank160017 +160018 POINT(42.01611474991256 -86.82868409908471) bank160018 +160019 POINT(42.03627432840479 -86.80124329238045) bank160019 +160020 POINT(42.46279925801641 -87.88941753964406) bank160020 +160021 POINT(42.24311392144359 -88.41586494904357) bank160021 +160022 POINT(42.06139248384919 -88.1265386858326) bank160022 +160023 POINT(41.05798360286092 -86.9450457792924) bank160023 +160024 POINT(42.651496699605865 -87.76870907735184) bank160024 +160025 POINT(42.238740712508516 -88.4725074020831) bank160025 +160026 POINT(41.29066402609934 -87.26663941592138) bank160026 +160027 POINT(41.64244283565646 -87.21231177040278) bank160027 +160028 POINT(42.79404226332444 -87.41743781389737) bank160028 +160029 POINT(42.254082349179434 -87.13602245951603) bank160029 +160030 POINT(41.26911203142923 -87.69645789112715) bank160030 +160031 POINT(41.4212657880272 -87.73797703265338) bank160031 +160032 POINT(42.54319447449801 -87.84567560795014) bank160032 +160033 POINT(41.200411313983466 -87.30778853360223) bank160033 +160034 POINT(41.69696057390877 -87.36779594569101) bank160034 +160035 POINT(40.93238794090048 -86.9736393548006) bank160035 +160036 POINT(40.97555499970991 -87.54629574794062) bank160036 +160037 POINT(40.88176742417623 -87.16210199756993) bank160037 +160038 POINT(42.84932709389052 -86.82855742416157) bank160038 +160039 POINT(41.21447611415596 -86.96010673262683) bank160039 +160040 POINT(42.301361493969985 -88.28977161783908) bank160040 +160041 POINT(40.89820837609004 -87.02461127566515) bank160041 +160042 POINT(42.74729439859931 -86.87361309755634) bank160042 +160043 POINT(41.10871746890434 -88.04950545673682) bank160043 +160044 POINT(41.36460616687298 -87.48716125727027) bank160044 +160045 POINT(42.36204779897883 -88.60873321969777) bank160045 +160046 POINT(42.634254780763435 -88.04799415818987) bank160046 +160047 POINT(41.77772514047727 -88.28979190613315) bank160047 +160048 POINT(40.99256691192139 -86.8382265371885) bank160048 +160049 POINT(41.01678854557559 -86.78126938605273) bank160049 +160050 POINT(41.14690093561883 -87.56170426556294) bank160050 +160051 POINT(41.92718485346246 -87.09799694385302) bank160051 +160052 POINT(42.335570397290084 -87.15061922930147) bank160052 +160053 POINT(41.188741135965266 -86.9500996338206) bank160053 +160054 POINT(42.09503861044571 -86.64652964177523) bank160054 +160055 POINT(41.82454146686035 -87.18840223346385) bank160055 +160056 POINT(41.89710922365967 -86.73597141540716) bank160056 +160057 POINT(41.52505199978265 -87.25682242892412) bank160057 +160058 POINT(42.80413603109855 -88.12790895382001) bank160058 +160059 POINT(42.83458167546721 -86.81256349208044) bank160059 +160060 POINT(41.957495975709946 -87.4009252338899) bank160060 +160061 POINT(42.7026057259352 -88.17521179120831) bank160061 +160062 POINT(42.619259443817 -87.2003309882356) bank160062 +160063 POINT(42.298867441338864 -87.4905253467617) bank160063 +160064 POINT(41.30128317851525 -86.76774603823327) bank160064 +160065 POINT(41.76997933778385 -88.52574609417358) bank160065 +160066 POINT(42.0604201855576 -88.14684079526037) bank160066 +160067 POINT(42.58294737847049 -87.89451268860431) bank160067 +160068 POINT(41.90331475332136 -88.31964385065095) bank160068 +160069 POINT(42.48759554328824 -88.55185546902028) bank160069 +160070 POINT(41.72363213209077 -88.05800976572975) bank160070 +160071 POINT(41.06191160878623 -88.54994574136192) bank160071 +160072 POINT(41.251253214859716 -86.73433255558645) bank160072 +160073 POINT(42.501223859539664 -87.58914884827179) bank160073 +160074 POINT(42.273019402593164 -88.12732061577809) bank160074 +160075 POINT(42.03856069482197 -87.73716816169608) bank160075 +160076 POINT(42.68388455792566 -88.3312667733179) bank160076 +160077 POINT(41.87766528926289 -87.34941101715101) bank160077 +160078 POINT(41.10609228471584 -88.34311319727408) bank160078 +160079 POINT(42.61149882087623 -87.84719207807231) bank160079 +160080 POINT(41.37883194283683 -87.48280643986143) bank160080 +160081 POINT(42.765119322299704 -88.05209264853457) bank160081 +160082 POINT(42.61278869992172 -88.5417606588776) bank160082 +160083 POINT(41.454188229251656 -87.07268099817725) bank160083 +160084 POINT(41.32511187751981 -88.2065144482544) bank160084 +160085 POINT(41.0010783773406 -86.77193317284356) bank160085 +160086 POINT(41.54396544193643 -86.95726140826213) bank160086 +160087 POINT(41.015213137452776 -87.83174010408122) bank160087 +160088 POINT(41.76827127180681 -87.51374406372194) bank160088 +160089 POINT(41.543339382268606 -88.30898439366335) bank160089 +160090 POINT(42.112035581222926 -88.00676914040416) bank160090 +160091 POINT(42.09130515194743 -86.96493225350221) bank160091 +160092 POINT(42.39630955670089 -87.88878424562927) bank160092 +160093 POINT(42.755131120758406 -87.76954539457557) bank160093 +160094 POINT(41.04928760223116 -87.98255494046721) bank160094 +160095 POINT(42.398733687451426 -87.21984347001158) bank160095 +160096 POINT(41.531697578851 -86.84867602315968) bank160096 +160097 POINT(42.17980639132923 -86.87677081327915) bank160097 +160098 POINT(41.465829307791644 -88.35714405163884) bank160098 +160099 POINT(42.24705840219113 -88.6199567232754) bank160099 +160100 POINT(42.23506145412458 -87.16078828812542) bank160100 +160101 POINT(41.26178584964731 -87.58377338522116) bank160101 +160102 POINT(42.62694570443907 -87.93658391343376) bank160102 +160103 POINT(42.26461809885969 -88.3390036542775) bank160103 +160104 POINT(41.77632768274486 -88.59243914879241) bank160104 +160105 POINT(42.0058690959411 -88.22159014868042) bank160105 +160106 POINT(41.18337861601792 -87.50198185767066) bank160106 +160107 POINT(41.00189527103053 -87.53792178962269) bank160107 +160108 POINT(41.38286765156888 -86.96725567190742) bank160108 +160109 POINT(42.72700393395221 -87.84219787285153) bank160109 +160110 POINT(41.68509338019542 -87.12956524646025) bank160110 +160111 POINT(41.66142734115006 -88.04368307901564) bank160111 +160112 POINT(42.419788637170726 -88.4382842246129) bank160112 +160113 POINT(42.278526838301914 -87.99488407041602) bank160113 +160114 POINT(42.65550767986136 -88.59150413965838) bank160114 +160115 POINT(40.94113115739577 -87.86191183425753) bank160115 +160116 POINT(42.00407843747671 -87.45692082713536) bank160116 +160117 POINT(41.75529347660434 -88.54618883112069) bank160117 +160118 POINT(41.2281045411889 -87.27245271858962) bank160118 +160119 POINT(40.896657604487835 -88.30426520257544) bank160119 +160120 POINT(42.6828249507743 -87.96285277882868) bank160120 +160121 POINT(42.244949263135226 -86.70365703901085) bank160121 +160122 POINT(42.678848026819885 -88.44319422506032) bank160122 +160123 POINT(41.92973085178483 -87.41937527431294) bank160123 +160124 POINT(41.53283358189172 -88.14538301806832) bank160124 +160125 POINT(41.346676936803085 -87.0132261185027) bank160125 +160126 POINT(41.368067267154395 -86.80282695098681) bank160126 +160127 POINT(42.6350867993445 -88.51852172661967) bank160127 +160128 POINT(42.548256261756535 -87.8168765920125) bank160128 +160129 POINT(40.893833737707745 -87.82860173222369) bank160129 +160130 POINT(42.06541728865271 -86.81958536620911) bank160130 +160131 POINT(42.58231868365114 -86.92170199002312) bank160131 +160132 POINT(41.05345076195659 -86.81330417104235) bank160132 +160133 POINT(42.03037864972409 -87.00100613206023) bank160133 +160134 POINT(41.93342598561769 -88.16347683054816) bank160134 +160135 POINT(42.41024697231763 -86.93905991788026) bank160135 +160136 POINT(42.39690814863593 -87.66561993782071) bank160136 +160137 POINT(42.75462363744333 -87.17078852406577) bank160137 +160138 POINT(42.70049236191561 -88.132723470205) bank160138 +160139 POINT(42.677849380887054 -88.49495669535753) bank160139 +160140 POINT(41.81598974104701 -87.86164807109863) bank160140 +160141 POINT(42.1940496872066 -87.55779847449794) bank160141 +160142 POINT(40.99087614812131 -88.24278517397562) bank160142 +160143 POINT(41.177725135398866 -87.77287366592454) bank160143 +160144 POINT(42.34306763949135 -88.2748043401004) bank160144 +160145 POINT(41.1901331521954 -86.65054394559571) bank160145 +160146 POINT(41.225071882311624 -86.9168969789048) bank160146 +160147 POINT(42.867061437614915 -87.01558385283376) bank160147 +160148 POINT(40.89192779775068 -88.41826105774742) bank160148 +160149 POINT(41.4853363219116 -87.77867382874577) bank160149 +160150 POINT(42.103762968000666 -87.63344412100308) bank160150 +160151 POINT(41.04542073955769 -86.84081983322085) bank160151 +160152 POINT(41.06228340067642 -87.71729096712757) bank160152 +160153 POINT(42.11357222062864 -88.04982084633986) bank160153 +160154 POINT(40.90308212953369 -87.82801747218151) bank160154 +160155 POINT(41.54157990385632 -87.80856197647743) bank160155 +160156 POINT(42.43086621933658 -86.83487686405827) bank160156 +160157 POINT(42.05949136489617 -87.77671129298393) bank160157 +160158 POINT(41.26549491074607 -86.9088284518187) bank160158 +160159 POINT(42.10331636092234 -88.54869698897025) bank160159 +160160 POINT(42.13849677643095 -88.01521222194806) bank160160 +160161 POINT(42.45583197173823 -88.11589552210542) bank160161 +160162 POINT(40.93555697991197 -88.21396936179988) bank160162 +160163 POINT(42.265101064701355 -87.21781082124821) bank160163 +160164 POINT(42.7231166047488 -87.37321715554076) bank160164 +160165 POINT(41.113621808110956 -87.2024410221608) bank160165 +160166 POINT(40.921583414667026 -88.21224518415535) bank160166 +160167 POINT(42.07583367080335 -88.4288974018585) bank160167 +160168 POINT(42.23914768145584 -87.06069401265668) bank160168 +160169 POINT(40.98935649715909 -88.52633020892159) bank160169 +160170 POINT(41.05186493043549 -87.55199239106068) bank160170 +160171 POINT(41.292878843966264 -88.58057046813275) bank160171 +160172 POINT(41.243101999016275 -87.89677169151673) bank160172 +160173 POINT(42.35996893349264 -86.93973425356165) bank160173 +160174 POINT(42.3998239564498 -87.55425268451293) bank160174 +160175 POINT(41.73875400181047 -87.8799048100916) bank160175 +160176 POINT(40.96333497032679 -87.50315098600647) bank160176 +160177 POINT(41.940392449904614 -86.66596037092468) bank160177 +160178 POINT(41.69672145392418 -87.06857433740363) bank160178 +160179 POINT(41.57932925049216 -88.46287152206067) bank160179 +160180 POINT(40.99528834424769 -87.29541599209485) bank160180 +160181 POINT(41.26677565441966 -87.77679992624742) bank160181 +160182 POINT(41.16812535789957 -86.70952655084558) bank160182 +160183 POINT(41.429971626941054 -86.75892750855557) bank160183 +160184 POINT(41.25280572792479 -87.59442634722811) bank160184 +160185 POINT(42.14444877185157 -87.2464544045768) bank160185 +160186 POINT(42.66007442719367 -87.70505399505913) bank160186 +160187 POINT(41.93621543451347 -86.89433918392403) bank160187 +160188 POINT(40.99896068571343 -87.19933404713595) bank160188 +160189 POINT(41.98364305039742 -87.96362474109698) bank160189 +160190 POINT(42.26754756570999 -87.60477814976944) bank160190 +160191 POINT(42.44119449855934 -87.86844309620646) bank160191 +160192 POINT(41.430883524269944 -86.64847055602166) bank160192 +160193 POINT(41.442353981924605 -88.22202876897242) bank160193 +160194 POINT(42.78951318611482 -86.91782922948983) bank160194 +160195 POINT(42.62435465240845 -88.34972665889734) bank160195 +160196 POINT(42.76447624320148 -88.12570785996068) bank160196 +160197 POINT(41.693065829307365 -88.1080228874092) bank160197 +160198 POINT(41.40664103963946 -86.78870686867852) bank160198 +160199 POINT(41.433004497946925 -86.82423223224966) bank160199 +160200 POINT(40.892146937028315 -88.14629486471652) bank160200 +160201 POINT(41.684172183433255 -88.54328325811093) bank160201 +160202 POINT(42.5940400024042 -87.30862945420273) bank160202 +160203 POINT(41.186469929044385 -86.94470294850409) bank160203 +160204 POINT(42.59863758362871 -87.66055906879308) bank160204 +160205 POINT(41.52090021783008 -88.54516292299581) bank160205 +160206 POINT(41.71179332225355 -88.26332243025601) bank160206 +160207 POINT(41.5088650523003 -88.07224322471029) bank160207 +160208 POINT(41.03854469412548 -87.8517842857907) bank160208 +160209 POINT(42.757919340234 -88.13561975614648) bank160209 +160210 POINT(41.92621874226201 -88.59228217067708) bank160210 +160211 POINT(41.77378991666804 -86.65358153946542) bank160211 +160212 POINT(42.7825416763514 -87.53920916676857) bank160212 +160213 POINT(42.01739195911358 -87.99458404064775) bank160213 +160214 POINT(40.95452465371932 -86.95784437501167) bank160214 +160215 POINT(42.181019697044135 -86.63234471219417) bank160215 +160216 POINT(41.93021234916688 -87.51747945937718) bank160216 +160217 POINT(41.05486058895951 -88.04040944463304) bank160217 +160218 POINT(40.929317891621004 -88.02620686280811) bank160218 +160219 POINT(41.42810448524974 -88.30597339589134) bank160219 +160220 POINT(41.53943029224682 -87.76161050086492) bank160220 +160221 POINT(42.591618190722976 -87.93847572214267) bank160221 +160222 POINT(41.68966362992933 -87.00992868334822) bank160222 +160223 POINT(41.87280434474205 -87.15059588274966) bank160223 +160224 POINT(42.46071417909352 -87.56191644939271) bank160224 +160225 POINT(42.77120895176972 -87.7252402098115) bank160225 +160226 POINT(41.88306559944721 -87.15481443591649) bank160226 +160227 POINT(41.2450848167604 -87.44018360553191) bank160227 +160228 POINT(41.90088187085377 -88.02382690425593) bank160228 +160229 POINT(42.678229420091874 -88.43060664069635) bank160229 +160230 POINT(41.624397981028785 -88.02782758429075) bank160230 +160231 POINT(42.70106425800822 -87.39405761459555) bank160231 +160232 POINT(42.036420060831304 -86.68500873683917) bank160232 +160233 POINT(41.109222828672294 -88.62028778487881) bank160233 +160234 POINT(42.734635209451106 -87.47066452174793) bank160234 +160235 POINT(41.8561373005106 -88.32467520881978) bank160235 +160236 POINT(42.652121507007095 -87.56260963152992) bank160236 +160237 POINT(42.59728724728195 -87.5700866439296) bank160237 +160238 POINT(42.57008760021983 -86.9448502529998) bank160238 +160239 POINT(42.244176411885434 -87.13488786804493) bank160239 +160240 POINT(42.84958867859028 -87.57048708509546) bank160240 +160241 POINT(42.21482627536862 -88.24822878720121) bank160241 +160242 POINT(41.10731706342569 -87.14131371658556) bank160242 +160243 POINT(42.4960161917045 -88.14535318653694) bank160243 +160244 POINT(40.971841332236686 -87.75271498886758) bank160244 +160245 POINT(42.17077508169948 -87.13388709338862) bank160245 +160246 POINT(42.569162579287415 -86.81951045112916) bank160246 +160247 POINT(41.90485050178428 -88.62128096059439) bank160247 +160248 POINT(42.44399728634306 -87.24964851830788) bank160248 +160249 POINT(41.67750483717294 -86.6664088139138) bank160249 +160250 POINT(41.27644408249889 -87.13920472550073) bank160250 +160251 POINT(41.66762761429103 -87.77363173841012) bank160251 +160252 POINT(42.37906661770852 -87.24025785092924) bank160252 +160253 POINT(41.35355951364815 -87.22391832731637) bank160253 +160254 POINT(42.10032681773611 -88.26466887229103) bank160254 +160255 POINT(42.51841718504976 -86.78183311094942) bank160255 +160256 POINT(42.82686032362876 -87.1074623333071) bank160256 +160257 POINT(41.98297082806078 -86.75001615397572) bank160257 +160258 POINT(41.50775631064466 -86.97955904321037) bank160258 +160259 POINT(41.79578932233502 -86.7462199388983) bank160259 +160260 POINT(41.54393906642324 -86.69619145152576) bank160260 +160261 POINT(40.948655287335896 -87.40054837729228) bank160261 +160262 POINT(40.97601719924494 -88.03740065987448) bank160262 +160263 POINT(42.47903995461986 -87.87442661870126) bank160263 +160264 POINT(42.22624970453493 -87.53985447357216) bank160264 +160265 POINT(42.658555445556324 -87.89158471830373) bank160265 +160266 POINT(41.68699593497732 -87.00387246207076) bank160266 +160267 POINT(41.39878445466714 -86.70832898412127) bank160267 +160268 POINT(42.77724131066699 -88.43490346035671) bank160268 +160269 POINT(41.635921149958826 -88.3532685231109) bank160269 +160270 POINT(42.25624329717324 -87.21604960639542) bank160270 +160271 POINT(41.717603959648095 -87.25476517110201) bank160271 +160272 POINT(41.373824694949064 -88.43503586346262) bank160272 +160273 POINT(42.84991337157349 -86.91167077547406) bank160273 +160274 POINT(42.282514671562026 -87.94169534864832) bank160274 +160275 POINT(41.81548598994932 -86.75195521845446) bank160275 +160276 POINT(41.835998905851184 -88.35005207248254) bank160276 +160277 POINT(41.41382037882681 -88.20730801458794) bank160277 +160278 POINT(40.994086915682104 -87.39935516741915) bank160278 +160279 POINT(41.45851532228428 -88.00717871179482) bank160279 +160280 POINT(41.19947794354427 -86.67443316569607) bank160280 +160281 POINT(41.72668153329108 -87.76548947844401) bank160281 +160282 POINT(42.32772909697584 -88.2859308542704) bank160282 +160283 POINT(42.07226702328352 -86.78543301717538) bank160283 +160284 POINT(42.540127954796944 -86.96874673841756) bank160284 +160285 POINT(42.35063862814209 -87.85324981712094) bank160285 +160286 POINT(41.36770321424985 -87.24582720900543) bank160286 +160287 POINT(41.59369696331019 -86.64771801303546) bank160287 +160288 POINT(41.996709235961426 -87.58702567609991) bank160288 +160289 POINT(42.304218467589386 -88.39741166688094) bank160289 +160290 POINT(42.79405668964699 -88.61775988829892) bank160290 +160291 POINT(41.77260931409448 -87.20289036707817) bank160291 +160292 POINT(42.53588853100628 -86.91289534250214) bank160292 +160293 POINT(40.98155455163832 -86.97282772467295) bank160293 +160294 POINT(41.996805732197934 -88.37729084987855) bank160294 +160295 POINT(41.10190255091581 -88.09333063488107) bank160295 +160296 POINT(41.04468969163774 -88.06263009477028) bank160296 +160297 POINT(42.72332131100726 -86.74198895068191) bank160297 +160298 POINT(42.82746197788088 -88.15712344830925) bank160298 +160299 POINT(42.37916575449045 -87.54221371495119) bank160299 +160300 POINT(42.244738881762935 -87.3001679005197) bank160300 +160301 POINT(42.0245169406532 -88.14295967576115) bank160301 +160302 POINT(41.9513818513265 -87.68724922077307) bank160302 +160303 POINT(41.00079676486594 -86.98098797267852) bank160303 +160304 POINT(41.93884726673895 -88.04054582651602) bank160304 +160305 POINT(42.37483182929612 -88.36821702330899) bank160305 +160306 POINT(41.15088795698096 -86.91479466657123) bank160306 +160307 POINT(41.83597498434617 -87.208694005057) bank160307 +160308 POINT(41.868068054595646 -87.91712299194346) bank160308 +160309 POINT(40.96398650183566 -87.97291921074803) bank160309 +160310 POINT(41.59771558661032 -87.50189885609105) bank160310 +160311 POINT(42.764840188671414 -88.18050482596183) bank160311 +160312 POINT(42.140450820329775 -87.78125575179804) bank160312 +160313 POINT(42.33118677485392 -88.16721801418451) bank160313 +160314 POINT(42.67948011693967 -88.0841700483201) bank160314 +160315 POINT(42.40677257984245 -87.08046742138096) bank160315 +160316 POINT(42.47521004708733 -88.43293975011673) bank160316 +160317 POINT(41.47491467067436 -88.14358935861132) bank160317 +160318 POINT(42.442988919743584 -88.09704101548672) bank160318 +160319 POINT(42.22868260044974 -87.41566490872533) bank160319 +160320 POINT(41.46610685388745 -87.18461522806744) bank160320 +160321 POINT(41.16499149033159 -88.48616545259299) bank160321 +160322 POINT(42.63009219449906 -87.13733568660678) bank160322 +160323 POINT(42.704923018830605 -86.9538565035009) bank160323 +160324 POINT(42.30318837272759 -88.42403552269553) bank160324 +160325 POINT(41.2051681185542 -88.12710237532679) bank160325 +160326 POINT(41.28110057801463 -88.62097229844103) bank160326 +160327 POINT(42.685111339460214 -87.9376862575113) bank160327 +160328 POINT(41.81026849006878 -86.63767167357908) bank160328 +160329 POINT(41.08445643674847 -87.72060849201063) bank160329 +160330 POINT(42.17040321176204 -87.37675741820189) bank160330 +160331 POINT(42.278256401190724 -88.46062353026883) bank160331 +160332 POINT(41.99851702600229 -87.64130100732243) bank160332 +160333 POINT(42.370000391599895 -87.99485355586384) bank160333 +160334 POINT(41.38194169111924 -87.02186390024055) bank160334 +160335 POINT(42.28264625372032 -86.87037519561467) bank160335 +160336 POINT(41.741603517404236 -87.76829203498383) bank160336 +160337 POINT(42.513105626849025 -86.90810683899957) bank160337 +160338 POINT(41.44173897732388 -86.8356330916959) bank160338 +160339 POINT(42.75119922576548 -87.98443137367235) bank160339 +160340 POINT(42.56349906234507 -88.00403517143837) bank160340 +160341 POINT(42.77569351422081 -88.13744941719798) bank160341 +160342 POINT(41.57949244994807 -86.84242510117308) bank160342 +160343 POINT(41.40012285358807 -88.3232417933517) bank160343 +160344 POINT(40.96794270816639 -88.34474231910441) bank160344 +160345 POINT(42.40964625773432 -87.74258784613833) bank160345 +160346 POINT(42.745940466767145 -88.26014507779394) bank160346 +160347 POINT(42.60018733180386 -87.70414830042913) bank160347 +160348 POINT(42.31336578112491 -87.34043577419239) bank160348 +160349 POINT(41.642706734019995 -87.77260476186734) bank160349 +160350 POINT(42.124358764669594 -87.71665388337907) bank160350 +160351 POINT(40.91264253534661 -88.11939421773589) bank160351 +160352 POINT(41.452396221959305 -88.50245750054405) bank160352 +160353 POINT(42.015298751647165 -88.44852649233516) bank160353 +160354 POINT(41.82887540529604 -87.16709286162947) bank160354 +160355 POINT(40.950626230308245 -87.53779845295988) bank160355 +160356 POINT(41.2113183310132 -87.29557302707096) bank160356 +160357 POINT(42.312201532429086 -86.67144353330976) bank160357 +160358 POINT(42.08178284480255 -87.5353159643004) bank160358 +160359 POINT(42.173657940242975 -87.6592812748903) bank160359 +160360 POINT(41.49286718706268 -86.8125858961309) bank160360 +160361 POINT(41.916635136289166 -87.44077888248606) bank160361 +160362 POINT(42.1633419172695 -88.01347769436654) bank160362 +160363 POINT(42.52282813219473 -88.42987323994193) bank160363 +160364 POINT(41.82922036159664 -87.91470515573371) bank160364 +160365 POINT(42.17793558423937 -87.79672655110319) bank160365 +160366 POINT(42.81212851305801 -86.69745526361919) bank160366 +160367 POINT(41.041252439212016 -87.09027781916579) bank160367 +160368 POINT(42.87055010568156 -87.81738383968909) bank160368 +160369 POINT(42.74814910762837 -86.66064908380623) bank160369 +160370 POINT(41.21208848643669 -87.63375522728262) bank160370 +160371 POINT(42.21754250486122 -88.0330296390421) bank160371 +160372 POINT(41.78495469363768 -86.79113414778799) bank160372 +160373 POINT(42.825207154060635 -88.37433134136815) bank160373 +160374 POINT(41.907176682327474 -86.75799956563219) bank160374 +160375 POINT(42.02088853968544 -86.79799582402663) bank160375 +160376 POINT(42.36260425540515 -87.60588595075313) bank160376 +160377 POINT(42.57388795692484 -88.19314438482294) bank160377 +160378 POINT(41.71789249601573 -86.86671719098675) bank160378 +160379 POINT(41.95131293323583 -87.88000941953483) bank160379 +160380 POINT(42.725724112827905 -86.77047711674786) bank160380 +160381 POINT(41.032341807419954 -87.66053408628278) bank160381 +160382 POINT(42.050981115592954 -87.29700107310423) bank160382 +160383 POINT(41.380415315488136 -87.62724495281549) bank160383 +160384 POINT(41.22642132299884 -88.01947370766311) bank160384 +160385 POINT(41.016232438109775 -88.33528938781294) bank160385 +160386 POINT(42.745560551948564 -88.28957678514433) bank160386 +160387 POINT(42.39017122295264 -88.01259486397531) bank160387 +160388 POINT(42.75463286553306 -88.47252090211653) bank160388 +160389 POINT(42.72337703406324 -87.3818698228458) bank160389 +160390 POINT(41.598011925453584 -87.51994136719742) bank160390 +160391 POINT(41.218749638077064 -87.70973138549837) bank160391 +160392 POINT(41.76500604919783 -86.65728727089908) bank160392 +160393 POINT(41.03536532319277 -86.91300086543738) bank160393 +160394 POINT(42.01852979439268 -86.70466915597738) bank160394 +160395 POINT(41.05547584848166 -87.74687576101077) bank160395 +160396 POINT(41.758005589344876 -87.03622930684949) bank160396 +160397 POINT(41.67973547982397 -87.05611453070688) bank160397 +160398 POINT(42.42764540252062 -88.40133241678313) bank160398 +160399 POINT(41.295041480041256 -87.66242791893232) bank160399 +160400 POINT(41.60158932064839 -88.29891177542619) bank160400 +160401 POINT(42.763471535654794 -87.95998206976012) bank160401 +160402 POINT(42.62133206922076 -86.79097573122633) bank160402 +160403 POINT(41.06176097572729 -87.84392345198249) bank160403 +160404 POINT(41.23050797746245 -88.27936850899725) bank160404 +160405 POINT(41.24937454749618 -88.12827958336005) bank160405 +160406 POINT(41.35786184424181 -87.43766765741327) bank160406 +160407 POINT(41.22176936328268 -88.41106485184636) bank160407 +160408 POINT(41.205075568785986 -86.7727986157447) bank160408 +160409 POINT(41.86188898814554 -87.79856317646899) bank160409 +160410 POINT(41.05379815043265 -87.2762779432673) bank160410 +160411 POINT(41.35461678936818 -88.42965401358181) bank160411 +160412 POINT(42.68590544878759 -87.4243013190441) bank160412 +160413 POINT(40.95102811655797 -87.5172644435067) bank160413 +160414 POINT(41.545164402922964 -88.31781204052741) bank160414 +160415 POINT(42.09732464115384 -88.37554465989024) bank160415 +160416 POINT(41.202764390384104 -88.15326231840018) bank160416 +160417 POINT(41.9004927597018 -88.08009056479071) bank160417 +160418 POINT(42.082219119878324 -87.01587541164699) bank160418 +160419 POINT(42.85100208750522 -86.65930144125444) bank160419 +160420 POINT(42.01073813608749 -87.0433920298892) bank160420 +160421 POINT(41.5931034320993 -88.55632318016745) bank160421 +160422 POINT(41.468721554063805 -88.50849743128312) bank160422 +160423 POINT(42.214050302997784 -87.2891648265305) bank160423 +160424 POINT(42.38884105712245 -87.16683430271944) bank160424 +160425 POINT(41.933505372389746 -87.91628983980316) bank160425 +160426 POINT(40.93776068854605 -87.12662672022795) bank160426 +160427 POINT(41.71836756694473 -88.38046649877147) bank160427 +160428 POINT(42.51364054007675 -87.8969652063891) bank160428 +160429 POINT(42.59017262757583 -86.73587411431032) bank160429 +160430 POINT(41.86253075789325 -86.73825836573135) bank160430 +160431 POINT(41.57192423528414 -88.30662669792228) bank160431 +160432 POINT(42.45474878571321 -86.93123537101334) bank160432 +160433 POINT(41.877821101113526 -87.59636703320203) bank160433 +160434 POINT(41.4145610415436 -88.27103734659444) bank160434 +160435 POINT(42.224564100883484 -86.78632131707657) bank160435 +160436 POINT(42.446163482019415 -87.99820961267689) bank160436 +160437 POINT(42.682959762785785 -88.40770136522079) bank160437 +160438 POINT(41.802656952172896 -88.4194748596946) bank160438 +160439 POINT(41.88743390557023 -88.13563708949124) bank160439 +160440 POINT(41.23175151543525 -88.174007056191) bank160440 +160441 POINT(42.68858496737806 -87.64533970935811) bank160441 +160442 POINT(42.33364371341016 -86.70799420664792) bank160442 +160443 POINT(41.86028135135891 -87.75721925579833) bank160443 +160444 POINT(41.25686011156215 -87.35738024490239) bank160444 +160445 POINT(42.16835779297362 -87.14043898564573) bank160445 +160446 POINT(42.25006553764561 -86.84937375533926) bank160446 +160447 POINT(41.308036867405946 -87.24799934553633) bank160447 +160448 POINT(42.58158474526921 -88.48864469282857) bank160448 +160449 POINT(42.373894431389466 -86.69291800210038) bank160449 +160450 POINT(42.31089465478437 -86.86917636267721) bank160450 +160451 POINT(41.321601817556555 -88.03525823435221) bank160451 +160452 POINT(41.33537578415264 -86.96147314043904) bank160452 +160453 POINT(41.39144336814586 -88.48629770130776) bank160453 +160454 POINT(41.916218849568395 -87.21673428776113) bank160454 +160455 POINT(41.911768623629385 -87.13970855250423) bank160455 +160456 POINT(42.3798087107218 -86.89492489736718) bank160456 +160457 POINT(41.233157639439966 -86.69125617200679) bank160457 +160458 POINT(41.24431977693536 -87.19367402827467) bank160458 +160459 POINT(41.67860740348925 -88.0007103153257) bank160459 +160460 POINT(41.90880116096476 -87.56034347060918) bank160460 +160461 POINT(41.99515284561996 -87.06705903942076) bank160461 +160462 POINT(41.75376774683973 -88.45125666521191) bank160462 +160463 POINT(41.04381123528538 -88.21489676950459) bank160463 +160464 POINT(41.98289343915681 -88.01662477510621) bank160464 +160465 POINT(42.007834203380035 -87.6122487679784) bank160465 +160466 POINT(41.092314357372466 -87.25379799172234) bank160466 +160467 POINT(41.73050565032088 -88.32193708670306) bank160467 +160468 POINT(41.18558903657374 -87.20124163127643) bank160468 +160469 POINT(41.215955542002476 -86.75478911781597) bank160469 +160470 POINT(42.728041199773656 -87.98553731711318) bank160470 +160471 POINT(42.86085212011943 -87.73336706363587) bank160471 +160472 POINT(40.91367862009402 -88.6096742917092) bank160472 +160473 POINT(42.46592824266278 -88.2361747197874) bank160473 +160474 POINT(42.43760312794085 -87.07264851729778) bank160474 +160475 POINT(41.96054600191731 -86.9639015295409) bank160475 +160476 POINT(41.09652219700109 -88.43383450845243) bank160476 +160477 POINT(42.158125549035 -87.11491202394181) bank160477 +160478 POINT(42.79343131684141 -87.55265633109397) bank160478 +160479 POINT(41.28807210647005 -87.85479931593528) bank160479 +160480 POINT(40.99249982513132 -87.45260983547423) bank160480 +160481 POINT(41.30684137303035 -86.70308997193531) bank160481 +160482 POINT(41.56160154572226 -86.63760267784946) bank160482 +160483 POINT(41.90105343981507 -87.91034189952403) bank160483 +160484 POINT(42.43492026554156 -86.81611059442623) bank160484 +160485 POINT(41.16381066520332 -86.86337345276922) bank160485 +160486 POINT(42.36626960940811 -87.89046565503617) bank160486 +160487 POINT(41.94553557328669 -87.281991385135) bank160487 +160488 POINT(42.596823740416966 -87.33049475407223) bank160488 +160489 POINT(41.590279193845824 -87.73112937288204) bank160489 +160490 POINT(41.75838663307749 -86.94854346474916) bank160490 +160491 POINT(41.884409761241365 -88.16632839835307) bank160491 +160492 POINT(42.170633312525965 -87.41268656157902) bank160492 +160493 POINT(41.368645190502185 -87.80717121689314) bank160493 +160494 POINT(41.67954638532319 -87.81239302067051) bank160494 +160495 POINT(41.90930286465049 -88.21709968462419) bank160495 +160496 POINT(42.77673579612446 -87.0987404224151) bank160496 +160497 POINT(42.81084422070718 -87.66740821256754) bank160497 +160498 POINT(42.154101165243404 -87.88692512347941) bank160498 +160499 POINT(42.0960873379326 -86.73333443733681) bank160499 +160500 POINT(41.78045204269677 -87.20192496047584) bank160500 +160501 POINT(42.531272205709165 -88.05092393031029) bank160501 +160502 POINT(41.50545156217525 -87.69477448866873) bank160502 +160503 POINT(42.87629082029235 -87.19089400012018) bank160503 +160504 POINT(42.52010106225147 -86.75898528192616) bank160504 +160505 POINT(41.24439261197762 -87.74448832405376) bank160505 +160506 POINT(42.207002512606465 -86.78134551190153) bank160506 +160507 POINT(41.24505560795819 -88.62076421997281) bank160507 +160508 POINT(41.18011281552343 -88.61502492018805) bank160508 +160509 POINT(41.64495656660059 -87.6159513742805) bank160509 +160510 POINT(41.28899841551376 -87.99607495141093) bank160510 +160511 POINT(41.339663268387135 -88.34076806752046) bank160511 +160512 POINT(41.509628752187744 -88.55659369948486) bank160512 +160513 POINT(41.44709702596139 -88.07387179800965) bank160513 +160514 POINT(42.67069917729818 -87.0905484611731) bank160514 +160515 POINT(41.65736857643679 -88.38829507961314) bank160515 +160516 POINT(41.330628766094065 -87.78452525643488) bank160516 +160517 POINT(42.666953077338626 -87.05352706600083) bank160517 +160518 POINT(42.08319936645351 -86.99531127472098) bank160518 +160519 POINT(42.064470283066136 -88.12340511019374) bank160519 +160520 POINT(41.20070682779507 -87.98238027401882) bank160520 +160521 POINT(41.416987657774804 -87.79140068709064) bank160521 +160522 POINT(42.374800593216676 -86.79704344456464) bank160522 +160523 POINT(41.52751429437953 -87.9386539429176) bank160523 +160524 POINT(42.60380772876751 -87.41536930286608) bank160524 +160525 POINT(41.02388292591857 -87.56784390344595) bank160525 +160526 POINT(42.35379230405742 -88.27915552582112) bank160526 +160527 POINT(41.932884674755186 -88.20157751729556) bank160527 +160528 POINT(42.23536655258001 -87.9117236923089) bank160528 +160529 POINT(41.29918858116758 -87.78584867518134) bank160529 +160530 POINT(41.93023127703207 -86.6923382583266) bank160530 +160531 POINT(41.35925486049158 -86.64904546685035) bank160531 +160532 POINT(42.777284723241735 -86.94218762637861) bank160532 +160533 POINT(42.62829422315813 -86.92695633329215) bank160533 +160534 POINT(41.258659982189066 -88.09347825482256) bank160534 +160535 POINT(42.06623009432767 -87.1720572070097) bank160535 +160536 POINT(41.627590862226505 -86.82568562234451) bank160536 +160537 POINT(42.07080476964478 -87.61649406996982) bank160537 +160538 POINT(41.36788793587924 -87.19584386523235) bank160538 +160539 POINT(40.9412260908715 -87.14948429216986) bank160539 +160540 POINT(41.029772202984894 -87.20656088141283) bank160540 +160541 POINT(42.8246273293476 -87.7605381812664) bank160541 +160542 POINT(41.03334401045239 -88.5294964877934) bank160542 +160543 POINT(40.93498425173868 -86.83991003678099) bank160543 +160544 POINT(41.32812148107016 -88.4651221075511) bank160544 +160545 POINT(41.33791747705819 -88.40935877322076) bank160545 +160546 POINT(41.18143302922567 -88.49446665099875) bank160546 +160547 POINT(42.50254613688231 -86.94193602989017) bank160547 +160548 POINT(41.49259013024231 -87.69498259653308) bank160548 +160549 POINT(41.514895816345636 -87.3937537354069) bank160549 +160550 POINT(41.917041991315315 -87.82141405069633) bank160550 +160551 POINT(42.68479544644656 -87.11962605807929) bank160551 +160552 POINT(41.44159326167678 -88.4365212951292) bank160552 +160553 POINT(41.18791425343277 -86.84589799356094) bank160553 +160554 POINT(41.50682551986153 -87.2288368575725) bank160554 +160555 POINT(42.27588844488169 -87.75322702667908) bank160555 +160556 POINT(40.88235130033834 -86.69122783743401) bank160556 +160557 POINT(41.11392306046812 -87.77332876009078) bank160557 +160558 POINT(42.04236102485767 -87.6386003003602) bank160558 +160559 POINT(41.75376692283428 -88.110798382061) bank160559 +160560 POINT(41.69355402195322 -87.99034807956231) bank160560 +160561 POINT(42.82788413506013 -87.4682431023638) bank160561 +160562 POINT(41.3249094818051 -86.71500544594477) bank160562 +160563 POINT(41.43932372316635 -86.82777737066206) bank160563 +160564 POINT(42.43482166578402 -87.07875291252834) bank160564 +160565 POINT(42.84803660280115 -87.7894836667116) bank160565 +160566 POINT(41.07096869347256 -87.82241888859008) bank160566 +160567 POINT(42.25942470402549 -88.36718578287463) bank160567 +160568 POINT(41.88391352706457 -88.14386113174419) bank160568 +160569 POINT(41.35497009035862 -87.53962576920513) bank160569 +160570 POINT(42.83706785489498 -88.35947732752246) bank160570 +160571 POINT(41.4938873119833 -88.16547372688386) bank160571 +160572 POINT(42.82225082012183 -86.8293822469756) bank160572 +160573 POINT(41.24399117209766 -87.14358962230682) bank160573 +160574 POINT(41.87250621953078 -88.35406282548965) bank160574 +160575 POINT(41.20597529877209 -86.81559800009359) bank160575 +160576 POINT(41.76457511191748 -88.41414810273119) bank160576 +160577 POINT(41.529282431877775 -88.09461864209865) bank160577 +160578 POINT(42.497054247921206 -86.99824255835392) bank160578 +160579 POINT(42.25622694042214 -88.50354337126517) bank160579 +160580 POINT(41.342819558543304 -87.16058429905604) bank160580 +160581 POINT(42.0126737903268 -88.51477335918393) bank160581 +160582 POINT(42.32514830942676 -86.89302673874346) bank160582 +160583 POINT(42.657956098552994 -87.11977722833126) bank160583 +160584 POINT(41.2862014303365 -88.50339784384867) bank160584 +160585 POINT(42.69987208928657 -86.77796053390514) bank160585 +160586 POINT(41.65330946232308 -86.80813883295455) bank160586 +160587 POINT(42.07877537668541 -88.19058412257793) bank160587 +160588 POINT(42.196157397258645 -88.43231578237913) bank160588 +160589 POINT(41.12858330650362 -88.31805853033086) bank160589 +160590 POINT(41.45912331354963 -86.90773181765643) bank160590 +160591 POINT(41.88915751422541 -87.94546584321947) bank160591 +160592 POINT(42.7210419338985 -87.76878504829618) bank160592 +160593 POINT(42.62577875834221 -87.21841936071534) bank160593 +160594 POINT(42.02414561850621 -88.12840717574296) bank160594 +160595 POINT(42.56845159536698 -88.52431700782104) bank160595 +160596 POINT(42.08364577138418 -87.60527524982331) bank160596 +160597 POINT(42.16674835915625 -88.29756723849195) bank160597 +160598 POINT(41.712672668713886 -87.1869628914751) bank160598 +160599 POINT(41.07423856759133 -88.20951658944743) bank160599 +160600 POINT(41.116530557879685 -87.84032435940182) bank160600 +160601 POINT(42.74641212994308 -86.69655851683898) bank160601 +160602 POINT(42.09164091033696 -86.65263960259966) bank160602 +160603 POINT(41.67272497313804 -88.2832065096174) bank160603 +160604 POINT(42.66298282396978 -86.6585102662723) bank160604 +160605 POINT(41.389244065933156 -86.9668971616791) bank160605 +160606 POINT(42.18582590712162 -88.47172311542559) bank160606 +160607 POINT(40.97901809520283 -86.99727559863067) bank160607 +160608 POINT(41.95439920891306 -88.46851081640789) bank160608 +160609 POINT(42.16076481438377 -87.85574681160786) bank160609 +160610 POINT(41.67634351335488 -87.28454342345309) bank160610 +160611 POINT(41.43644125134779 -87.25612671806523) bank160611 +160612 POINT(41.863905434643804 -87.71222770288169) bank160612 +160613 POINT(41.06391909991462 -88.25284446805641) bank160613 +160614 POINT(42.644332211967324 -87.66586645589764) bank160614 +160615 POINT(40.99890603046303 -87.99788391351343) bank160615 +160616 POINT(41.252765048735995 -87.65842704229128) bank160616 +160617 POINT(41.581852788812576 -87.26356142895456) bank160617 +160618 POINT(41.29512775995166 -88.35903755197388) bank160618 +160619 POINT(41.989198193978126 -87.81768072882625) bank160619 +160620 POINT(42.261918738638286 -88.0854837960243) bank160620 +160621 POINT(41.31714497593122 -87.23488830375564) bank160621 +160622 POINT(42.10140027441757 -88.60000025417594) bank160622 +160623 POINT(42.662276762321596 -87.41734055540655) bank160623 +160624 POINT(42.85322983257812 -86.96119660096737) bank160624 +160625 POINT(42.359289963386345 -87.49383711375023) bank160625 +160626 POINT(40.962583585466824 -88.6170862426467) bank160626 +160627 POINT(41.45023722972956 -88.29339423273068) bank160627 +160628 POINT(42.629216612542166 -87.52214294722268) bank160628 +160629 POINT(42.29833753883878 -88.11436661955601) bank160629 +160630 POINT(42.45817111971451 -87.71255839322487) bank160630 +160631 POINT(41.9625418739454 -88.25428222173166) bank160631 +160632 POINT(42.08000719170387 -86.78054808022878) bank160632 +160633 POINT(41.0302044809681 -88.09730814503637) bank160633 +160634 POINT(42.874119337394895 -88.12073291275723) bank160634 +160635 POINT(41.9321638616832 -87.92656490476415) bank160635 +160636 POINT(41.70403310204842 -88.18802745016312) bank160636 +160637 POINT(42.80590477923102 -88.57863247620638) bank160637 +160638 POINT(42.369064459347555 -87.07553368242995) bank160638 +160639 POINT(42.492195074831294 -88.56252259976452) bank160639 +160640 POINT(41.80411477648003 -88.34429318654342) bank160640 +160641 POINT(41.31583531437487 -87.5340053572396) bank160641 +160642 POINT(41.123087908102335 -86.9340130998099) bank160642 +160643 POINT(42.55482022492185 -88.46135379164879) bank160643 +160644 POINT(42.05011952235302 -88.58131339205642) bank160644 +160645 POINT(41.67847915304738 -88.56851483546723) bank160645 +160646 POINT(41.80382521643089 -86.83807878295259) bank160646 +160647 POINT(41.40987962862963 -86.74075412432617) bank160647 +160648 POINT(42.021246674726754 -87.33551775670593) bank160648 +160649 POINT(41.088037957196285 -87.7257997700117) bank160649 +160650 POINT(42.230790793627634 -88.3317955643763) bank160650 +160651 POINT(42.04855053254699 -87.59459177089913) bank160651 +160652 POINT(41.316514805662074 -87.13144802413805) bank160652 +160653 POINT(42.49635447943602 -87.68847989324406) bank160653 +160654 POINT(42.318884156674336 -87.8685455281964) bank160654 +160655 POINT(41.207062080184826 -87.70151844481045) bank160655 +160656 POINT(40.91685736525936 -87.32988242297468) bank160656 +160657 POINT(41.57424246662785 -87.20324946834192) bank160657 +160658 POINT(42.85926300422059 -87.28129953214248) bank160658 +160659 POINT(41.450529189361255 -87.74302963671394) bank160659 +160660 POINT(40.942116170926916 -87.0805786740231) bank160660 +160661 POINT(41.18474501987738 -87.67437613346664) bank160661 +160662 POINT(42.59140379290557 -88.3757493487245) bank160662 +160663 POINT(42.25920095155201 -87.01703846719863) bank160663 +160664 POINT(42.64231884369448 -88.04615537297778) bank160664 +160665 POINT(42.21769556707808 -88.62373440800367) bank160665 +160666 POINT(41.381796467737736 -87.43649574861732) bank160666 +160667 POINT(41.0037850488737 -87.72019446320571) bank160667 +160668 POINT(40.946963149365544 -87.58129679917609) bank160668 +160669 POINT(41.0936035420002 -88.26660913618626) bank160669 +160670 POINT(41.750646542240546 -88.2579102173338) bank160670 +160671 POINT(41.13709246151795 -88.00680454302189) bank160671 +160672 POINT(42.3421049090879 -86.98380069970422) bank160672 +160673 POINT(41.477307593786385 -88.10136019774279) bank160673 +160674 POINT(42.52342781772644 -87.97443678493453) bank160674 +160675 POINT(41.51304424039534 -88.55830996879638) bank160675 +160676 POINT(42.164070233920555 -87.20851097153913) bank160676 +160677 POINT(41.81525007325685 -87.40612451209593) bank160677 +160678 POINT(41.868365429267676 -87.78083661901947) bank160678 +160679 POINT(41.49855763870846 -88.10202645807937) bank160679 +160680 POINT(42.29422954552314 -87.92625010312265) bank160680 +160681 POINT(41.67027803236788 -88.4115142688062) bank160681 +160682 POINT(42.352879014082475 -87.09140723768816) bank160682 +160683 POINT(41.5708445376155 -87.69556814471316) bank160683 +160684 POINT(42.22086115188587 -88.5506627108716) bank160684 +160685 POINT(41.91247523958851 -87.11589662266739) bank160685 +160686 POINT(42.55507214389433 -87.75041512566058) bank160686 +160687 POINT(42.56782849593814 -87.94978590045264) bank160687 +160688 POINT(41.077203168517684 -87.24961009154728) bank160688 +160689 POINT(42.30459973193994 -88.53042578630199) bank160689 +160690 POINT(41.50405438693003 -87.0532445054029) bank160690 +160691 POINT(42.61892737873795 -88.12238498770567) bank160691 +160692 POINT(41.203841313690134 -87.72224457925053) bank160692 +160693 POINT(42.463158521150845 -87.4672866459172) bank160693 +160694 POINT(40.99970141018097 -87.93997433465098) bank160694 +160695 POINT(42.67328774069633 -87.72197425436815) bank160695 +160696 POINT(40.9828784836612 -87.16880045218994) bank160696 +160697 POINT(40.88470464461884 -88.00787388018394) bank160697 +160698 POINT(41.56255984036413 -87.46771725148129) bank160698 +160699 POINT(41.49083995980674 -88.24654921485875) bank160699 +160700 POINT(42.38348127552686 -88.48325926823264) bank160700 +160701 POINT(41.67207460944469 -86.90072508492005) bank160701 +160702 POINT(42.31572824118936 -87.96868886295856) bank160702 +160703 POINT(42.265580977789135 -87.70535482219364) bank160703 +160704 POINT(42.40567805373925 -86.91627542356429) bank160704 +160705 POINT(41.84911151135673 -87.11635314397618) bank160705 +160706 POINT(42.0440073639468 -86.96364797115928) bank160706 +160707 POINT(41.48383079702111 -88.61797861754178) bank160707 +160708 POINT(41.18019033238401 -87.6802315052202) bank160708 +160709 POINT(42.24259162324112 -87.12343968956999) bank160709 +160710 POINT(41.09675160596881 -86.83964174764877) bank160710 +160711 POINT(42.079021271348246 -88.13307723426219) bank160711 +160712 POINT(42.831195439323544 -87.55644809146285) bank160712 +160713 POINT(42.05223549096391 -87.24818265684523) bank160713 +160714 POINT(42.79768699252113 -87.38358302500609) bank160714 +160715 POINT(42.32225364259945 -87.70037158232353) bank160715 +160716 POINT(42.4780570180434 -87.33467643142866) bank160716 +160717 POINT(41.00798352875417 -86.90424011677611) bank160717 +160718 POINT(41.970592254312464 -88.33590350576148) bank160718 +160719 POINT(42.37972175984998 -87.26543457141177) bank160719 +160720 POINT(42.622056958984466 -87.75785656250461) bank160720 +160721 POINT(42.57127284898953 -87.97483063543991) bank160721 +160722 POINT(42.34333787126148 -87.84973983018818) bank160722 +160723 POINT(41.310693466645866 -87.37237795005956) bank160723 +160724 POINT(41.329891353754086 -88.53680457540332) bank160724 +160725 POINT(41.83679678341389 -88.2632044402944) bank160725 +160726 POINT(41.26813047817347 -87.47603069532082) bank160726 +160727 POINT(40.97494334173544 -88.22606357975187) bank160727 +160728 POINT(41.85549917502851 -87.55192547322935) bank160728 +160729 POINT(41.798334702999455 -87.42584908198496) bank160729 +160730 POINT(42.70090048242935 -88.59368117766321) bank160730 +160731 POINT(42.14106860843246 -87.66629193102436) bank160731 +160732 POINT(42.25543203372337 -86.82015605895859) bank160732 +160733 POINT(41.9375490538319 -87.71063886143142) bank160733 +160734 POINT(41.68115046223395 -87.71991270037176) bank160734 +160735 POINT(42.171895227854364 -87.3414487545218) bank160735 +160736 POINT(42.46567148001314 -87.7265805000215) bank160736 +160737 POINT(41.388848177123734 -88.39069860466411) bank160737 +160738 POINT(41.522504227248774 -88.39513738853569) bank160738 +160739 POINT(41.401007012031606 -88.51177321894659) bank160739 +160740 POINT(41.443052900733974 -88.19619620597506) bank160740 +160741 POINT(41.23799462508394 -87.7906161984596) bank160741 +160742 POINT(42.517908158376834 -87.89785270524264) bank160742 +160743 POINT(42.241125764003364 -88.1787694608436) bank160743 +160744 POINT(40.97963350255988 -87.60308679762218) bank160744 +160745 POINT(42.15812697807665 -86.77545519503781) bank160745 +160746 POINT(40.96918085844733 -88.04212896520939) bank160746 +160747 POINT(42.22523026993673 -88.44041557308269) bank160747 +160748 POINT(41.84468846632811 -86.67916540988335) bank160748 +160749 POINT(41.99562579171083 -88.00573063393851) bank160749 +160750 POINT(41.51757846755758 -87.6426137358929) bank160750 +160751 POINT(41.41506711767885 -88.51051307412384) bank160751 +160752 POINT(42.26685209501076 -86.91377243480262) bank160752 +160753 POINT(42.209224999462116 -87.34490884162032) bank160753 +160754 POINT(42.80773246113708 -88.59230229180476) bank160754 +160755 POINT(42.7681232725028 -88.4089492882267) bank160755 +160756 POINT(42.59094980121905 -87.90732205461985) bank160756 +160757 POINT(41.24465814489846 -87.77999562275181) bank160757 +160758 POINT(41.802958770641915 -87.24590460874518) bank160758 +160759 POINT(41.66452545476068 -86.66855019103042) bank160759 +160760 POINT(41.85701014638593 -87.69489063653577) bank160760 +160761 POINT(40.886709744858976 -87.78509741630855) bank160761 +160762 POINT(41.21626930077103 -87.49038246663567) bank160762 +160763 POINT(41.643364996303916 -87.99360553730617) bank160763 +160764 POINT(41.82925607082815 -87.47662940331223) bank160764 +160765 POINT(41.462636257931074 -87.5023884707926) bank160765 +160766 POINT(42.59764747921373 -86.64203346009533) bank160766 +160767 POINT(41.48402347368335 -86.90789481224098) bank160767 +160768 POINT(41.57472681294032 -88.38237547865123) bank160768 +160769 POINT(40.979316781800364 -87.74858923578897) bank160769 +160770 POINT(41.34433005129842 -87.64308309899985) bank160770 +160771 POINT(41.16601352521685 -87.69883838360111) bank160771 +160772 POINT(41.41944861710443 -88.11325194024839) bank160772 +160773 POINT(41.150569774063946 -86.78497464962291) bank160773 +160774 POINT(42.26131296070214 -87.8237122236663) bank160774 +160775 POINT(42.42889799175636 -88.42660499596026) bank160775 +160776 POINT(41.68312347288634 -88.52148406340073) bank160776 +160777 POINT(42.587401782639986 -87.18622924884588) bank160777 +160778 POINT(41.43234419489582 -88.12832389140324) bank160778 +160779 POINT(42.446801024597406 -88.58285323145122) bank160779 +160780 POINT(41.20240312040363 -87.11398446205088) bank160780 +160781 POINT(41.3187098504763 -88.27484565130895) bank160781 +160782 POINT(40.884468144659024 -88.33122673437896) bank160782 +160783 POINT(42.58560279633646 -88.25396982417104) bank160783 +160784 POINT(41.86015655188012 -86.88872806048711) bank160784 +160785 POINT(42.10871367604712 -87.76798785930613) bank160785 +160786 POINT(41.767452073877124 -86.73796017210634) bank160786 +160787 POINT(42.76504312187623 -86.92954276498044) bank160787 +160788 POINT(41.26036024236591 -88.45225988978011) bank160788 +160789 POINT(42.61366015968388 -88.37838406707974) bank160789 +160790 POINT(41.59659155229154 -88.09534921043392) bank160790 +160791 POINT(42.15087310556602 -88.53978884003732) bank160791 +160792 POINT(42.53218855054453 -88.60415626176069) bank160792 +160793 POINT(42.58734128197894 -88.5452827202568) bank160793 +160794 POINT(42.645780137941266 -87.67972333240552) bank160794 +160795 POINT(41.82532250734716 -86.84603422382415) bank160795 +160796 POINT(41.885657100945274 -86.92416005482919) bank160796 +160797 POINT(40.925017659853765 -86.85177438284019) bank160797 +160798 POINT(42.1356576608472 -88.50491639296482) bank160798 +160799 POINT(42.13681846922917 -87.46629867613356) bank160799 +160800 POINT(40.93466008030251 -88.30219003389843) bank160800 +160801 POINT(41.047732129429036 -87.6224677876524) bank160801 +160802 POINT(42.668483192712834 -88.62515897714408) bank160802 +160803 POINT(42.157936652990166 -87.5757734082736) bank160803 +160804 POINT(41.40238996006353 -87.09206572097253) bank160804 +160805 POINT(41.5976459382751 -88.48756158058407) bank160805 +160806 POINT(42.15235448826589 -87.96604733720632) bank160806 +160807 POINT(42.10122124977012 -87.23928841309394) bank160807 +160808 POINT(41.074498660579934 -87.47276615110096) bank160808 +160809 POINT(41.82907649940004 -86.84423083310142) bank160809 +160810 POINT(41.683298899639084 -87.98551485248967) bank160810 +160811 POINT(42.3083362950214 -88.04004281890398) bank160811 +160812 POINT(41.4988950608272 -86.8508421709467) bank160812 +160813 POINT(40.897750167422586 -86.91100924914151) bank160813 +160814 POINT(41.80927209735273 -87.94445892721704) bank160814 +160815 POINT(42.30963761201903 -88.25012780425777) bank160815 +160816 POINT(41.93870261846168 -88.37001243578075) bank160816 +160817 POINT(41.85130893543309 -88.35279178964326) bank160817 +160818 POINT(42.855716219182646 -87.67810923311171) bank160818 +160819 POINT(41.82541260542115 -86.88095541469427) bank160819 +160820 POINT(41.06195478048646 -87.27959881080113) bank160820 +160821 POINT(41.891279736805835 -88.5181381827637) bank160821 +160822 POINT(42.54003173691599 -87.75340566686125) bank160822 +160823 POINT(42.75352976997256 -87.54901860493405) bank160823 +160824 POINT(42.74901888527208 -87.26848920726968) bank160824 +160825 POINT(41.84488100255693 -88.27708484098738) bank160825 +160826 POINT(42.26141766699268 -87.60382155512326) bank160826 +160827 POINT(41.93758379440818 -87.65355160264602) bank160827 +160828 POINT(41.18488175933643 -88.36436817670112) bank160828 +160829 POINT(42.3825155654928 -87.09777663445668) bank160829 +160830 POINT(41.8038968212085 -87.48009497886123) bank160830 +160831 POINT(42.42818269568493 -88.48551464077579) bank160831 +160832 POINT(41.528043291673825 -86.79338999021509) bank160832 +160833 POINT(42.27079041959738 -87.358489360321) bank160833 +160834 POINT(42.13077192546727 -86.68353615442274) bank160834 +160835 POINT(42.624275945974304 -87.27803554973187) bank160835 +160836 POINT(41.19459145622518 -86.72981268251935) bank160836 +160837 POINT(41.53493848079966 -87.17274950542793) bank160837 +160838 POINT(41.4776464057457 -87.77654087538774) bank160838 +160839 POINT(42.16139686960556 -86.67855894520578) bank160839 +160840 POINT(41.205817712416 -88.43092290648292) bank160840 +160841 POINT(41.58728421337734 -86.82091286912129) bank160841 +160842 POINT(41.782078116856745 -88.38165296659461) bank160842 +160843 POINT(42.50776458694615 -87.87733907544447) bank160843 +160844 POINT(41.95658642850423 -87.22847141688896) bank160844 +160845 POINT(41.68979924580438 -87.18497735537046) bank160845 +160846 POINT(42.41658998747528 -86.74349526433382) bank160846 +160847 POINT(41.97495346266087 -86.63015465687215) bank160847 +160848 POINT(41.950955207013514 -87.2586331618065) bank160848 +160849 POINT(41.455182162346986 -87.10773234489594) bank160849 +160850 POINT(42.41840382471065 -88.53903910493196) bank160850 +160851 POINT(41.13567442266026 -87.25740778639447) bank160851 +160852 POINT(41.546306894462994 -88.61856431276566) bank160852 +160853 POINT(42.82384046697685 -88.53909697667967) bank160853 +160854 POINT(42.06754126143052 -86.98121373273457) bank160854 +160855 POINT(42.71304231649412 -86.96892830519766) bank160855 +160856 POINT(42.24686290567888 -87.16196659997249) bank160856 +160857 POINT(42.349738338109674 -88.26037031537395) bank160857 +160858 POINT(42.85852852981362 -87.1813962262678) bank160858 +160859 POINT(42.86710159545959 -87.525054120898) bank160859 +160860 POINT(42.200471803597274 -88.35476663534517) bank160860 +160861 POINT(42.05308617086154 -88.11467210543132) bank160861 +160862 POINT(42.09291943743204 -88.46148788727203) bank160862 +160863 POINT(42.576522175925646 -86.78100205217036) bank160863 +160864 POINT(42.37703609966159 -87.9723077782603) bank160864 +160865 POINT(42.00741104091157 -86.81051832469262) bank160865 +160866 POINT(41.19609492361449 -87.53348877333521) bank160866 +160867 POINT(41.58200556239404 -86.90888614052432) bank160867 +160868 POINT(42.34705660182847 -88.59642182514176) bank160868 +160869 POINT(42.3938102049646 -88.24299867174352) bank160869 +160870 POINT(42.76955312342559 -87.55966662945136) bank160870 +160871 POINT(42.145347362230154 -86.74240853625605) bank160871 +160872 POINT(42.3319762446417 -88.2484239750928) bank160872 +160873 POINT(41.923762307338535 -87.9480693124161) bank160873 +160874 POINT(41.63882538578381 -87.26813190966672) bank160874 +160875 POINT(42.0777907517135 -88.600706498735) bank160875 +160876 POINT(41.64538817364731 -87.9287468720686) bank160876 +160877 POINT(42.20358165516937 -86.67979682195545) bank160877 +160878 POINT(41.9183470691005 -87.80122468368381) bank160878 +160879 POINT(42.81219175958321 -86.86504574847949) bank160879 +160880 POINT(42.0768574832027 -87.52142074484318) bank160880 +160881 POINT(42.81067426341452 -88.23522117619457) bank160881 +160882 POINT(41.21861934715474 -88.15055993554438) bank160882 +160883 POINT(42.049912666956 -87.41917782553094) bank160883 +160884 POINT(41.18181748820984 -86.88596344745751) bank160884 +160885 POINT(42.44051476891527 -88.08198615549823) bank160885 +160886 POINT(41.84564166198913 -86.7325592432613) bank160886 +160887 POINT(41.24722987047771 -86.64905287473347) bank160887 +160888 POINT(41.79132131105632 -87.12955040566867) bank160888 +160889 POINT(41.30275054926438 -87.19505893493846) bank160889 +160890 POINT(41.25747642809943 -87.5637017673962) bank160890 +160891 POINT(41.78196511253779 -86.83680848171436) bank160891 +160892 POINT(41.70310505366534 -88.5592839449342) bank160892 +160893 POINT(41.41843790325581 -88.6287479348497) bank160893 +160894 POINT(42.28477932543408 -88.16015162999531) bank160894 +160895 POINT(41.08348809365595 -86.85513464457094) bank160895 +160896 POINT(42.27524046971609 -87.53450069142515) bank160896 +160897 POINT(41.76429163210555 -88.46121853416861) bank160897 +160898 POINT(41.749546987912765 -87.48953172070594) bank160898 +160899 POINT(40.888944765886805 -87.13220682026044) bank160899 +160900 POINT(41.67150294189268 -87.81335368700553) bank160900 +160901 POINT(42.395076270113876 -86.84921395543098) bank160901 +160902 POINT(41.13371040016124 -88.14993397968983) bank160902 +160903 POINT(41.55912797158617 -87.76929074107463) bank160903 +160904 POINT(42.679481688521946 -86.75466153082323) bank160904 +160905 POINT(42.00724301029684 -86.73441836131157) bank160905 +160906 POINT(41.911717020155656 -87.95824362430875) bank160906 +160907 POINT(41.439911009706776 -86.69620620127436) bank160907 +160908 POINT(42.03188039978354 -87.61056243119533) bank160908 +160909 POINT(41.573913421979924 -88.4160510781057) bank160909 +160910 POINT(42.65240273254624 -86.62997492200968) bank160910 +160911 POINT(42.163568913951316 -87.12013133211458) bank160911 +160912 POINT(41.50397083060769 -87.80701044392337) bank160912 +160913 POINT(42.55573573127423 -87.96906332777935) bank160913 +160914 POINT(41.701156056101304 -88.37156643489224) bank160914 +160915 POINT(41.40467119022575 -86.85089882623062) bank160915 +160916 POINT(42.311775273647044 -88.50203498747425) bank160916 +160917 POINT(42.579892301642325 -88.49112785371977) bank160917 +160918 POINT(42.52257909810811 -87.43845261301986) bank160918 +160919 POINT(42.85584617305538 -87.06836581526434) bank160919 +160920 POINT(41.77123810991457 -87.84027885984271) bank160920 +160921 POINT(42.45686594973513 -87.3011785465596) bank160921 +160922 POINT(41.53278865964973 -87.28514756758753) bank160922 +160923 POINT(40.994558010803985 -87.12455839610108) bank160923 +160924 POINT(42.759176123246874 -88.58414151940653) bank160924 +160925 POINT(42.24185575730424 -87.7491298212495) bank160925 +160926 POINT(41.76041760939908 -86.82591407018289) bank160926 +160927 POINT(42.83352871404229 -87.63093215730187) bank160927 +160928 POINT(42.21944004156734 -86.91555458045507) bank160928 +160929 POINT(41.224364190048085 -87.72987505937492) bank160929 +160930 POINT(41.49752232077659 -87.4998131730412) bank160930 +160931 POINT(41.30706061051041 -86.69971610848052) bank160931 +160932 POINT(41.10062144540269 -88.37988529339553) bank160932 +160933 POINT(42.837374181789144 -86.72644990897612) bank160933 +160934 POINT(40.91234127005845 -86.68152818599522) bank160934 +160935 POINT(41.83646022409694 -86.82013885420851) bank160935 +160936 POINT(41.45914749605186 -87.96595047273809) bank160936 +160937 POINT(41.69642797167782 -86.93270845502293) bank160937 +160938 POINT(42.42905899691603 -86.70258579537206) bank160938 +160939 POINT(41.00413193018189 -87.53702681329113) bank160939 +160940 POINT(41.21153030873101 -88.11574538585432) bank160940 +160941 POINT(41.764950805654316 -88.58640414478461) bank160941 +160942 POINT(41.16943293632122 -87.6639201211556) bank160942 +160943 POINT(42.261178969173244 -87.79299839532685) bank160943 +160944 POINT(41.333490736682755 -87.97530500371533) bank160944 +160945 POINT(40.985653749969494 -86.69987393115336) bank160945 +160946 POINT(41.3857834223404 -88.29238539924965) bank160946 +160947 POINT(42.29403672289118 -87.65033199259842) bank160947 +160948 POINT(42.23991281245805 -88.50382116188456) bank160948 +160949 POINT(41.004679748917916 -87.26461748272331) bank160949 +160950 POINT(42.45276475096766 -88.01527804639402) bank160950 +160951 POINT(41.89819738723276 -88.62428288212277) bank160951 +160952 POINT(42.47415184554444 -86.68531443059497) bank160952 +160953 POINT(41.51859038626793 -87.50696086611947) bank160953 +160954 POINT(42.64574769121047 -87.86139307561281) bank160954 +160955 POINT(42.81650499539828 -87.1763209426112) bank160955 +160956 POINT(41.86657085650503 -86.68371165489083) bank160956 +160957 POINT(41.701577804762 -87.11094701952211) bank160957 +160958 POINT(42.6480648076965 -87.20270837920083) bank160958 +160959 POINT(42.561982925634005 -87.07402064227915) bank160959 +160960 POINT(42.099687831801226 -88.19335961498165) bank160960 +160961 POINT(42.61861485669751 -88.30033058258759) bank160961 +160962 POINT(41.13048995095298 -88.29401303866092) bank160962 +160963 POINT(42.30377152721524 -87.3782881368687) bank160963 +160964 POINT(42.62423523058063 -88.11917038673472) bank160964 +160965 POINT(42.719822687626376 -87.86423963774772) bank160965 +160966 POINT(42.059831486508514 -87.11604969815646) bank160966 +160967 POINT(41.385722623629114 -88.44790306894444) bank160967 +160968 POINT(42.72523679182502 -88.13300382888343) bank160968 +160969 POINT(41.67816417998162 -88.14407424087167) bank160969 +160970 POINT(42.387274158379505 -88.57375132356287) bank160970 +160971 POINT(42.87015747766248 -87.12779240346002) bank160971 +160972 POINT(41.23396078692867 -88.00106397926268) bank160972 +160973 POINT(42.222646979572275 -87.23467919842147) bank160973 +160974 POINT(41.87677727846383 -87.7376955228194) bank160974 +160975 POINT(42.72245178207382 -86.81504859142221) bank160975 +160976 POINT(41.001046947605424 -87.13111737334256) bank160976 +160977 POINT(42.83064116919992 -87.86699489611011) bank160977 +160978 POINT(41.68595128928632 -88.16796321119014) bank160978 +160979 POINT(41.609940860787155 -86.853149141575) bank160979 +160980 POINT(42.725524180999706 -86.88490124002045) bank160980 +160981 POINT(41.07304107414403 -86.81447725855467) bank160981 +160982 POINT(41.82074685344304 -88.34394772474491) bank160982 +160983 POINT(42.61623082257302 -88.60997976463138) bank160983 +160984 POINT(41.01721771478602 -87.13554117480928) bank160984 +160985 POINT(41.27650076005041 -88.4715924559536) bank160985 +160986 POINT(41.210170264591106 -87.28397860484372) bank160986 +160987 POINT(41.07493136142663 -87.41304244547963) bank160987 +160988 POINT(41.47866166323626 -87.84326642423716) bank160988 +160989 POINT(40.89635065817565 -87.5058578259128) bank160989 +160990 POINT(41.436654317229355 -87.30938148792951) bank160990 +160991 POINT(41.6382570121651 -88.40964490060857) bank160991 +160992 POINT(42.08722344011836 -88.13960757948152) bank160992 +160993 POINT(42.29443707597138 -87.63438702761306) bank160993 +160994 POINT(42.77294433036957 -87.58695694729599) bank160994 +160995 POINT(42.86179900526378 -88.28963403506133) bank160995 +160996 POINT(42.5079984891455 -88.22673615800515) bank160996 +160997 POINT(41.67427165214101 -87.92689047925612) bank160997 +160998 POINT(41.28811240475865 -87.87523620642098) bank160998 +160999 POINT(40.93472185489498 -87.73425569476292) bank160999 +161000 POINT(42.50640665350453 -87.7383405921468) bank161000 +161001 POINT(42.27805693906773 -87.175132734431) bank161001 +161002 POINT(41.88526411474418 -88.17840177296637) bank161002 +161003 POINT(41.7563673783769 -87.67837590727702) bank161003 +161004 POINT(41.6513976547116 -87.18206310332087) bank161004 +161005 POINT(41.37475089526538 -88.0798296377532) bank161005 +161006 POINT(42.11059946135202 -87.87137002356712) bank161006 +161007 POINT(42.664994901113914 -86.94281686692351) bank161007 +161008 POINT(41.16279695025061 -87.43819309015835) bank161008 +161009 POINT(41.336977685369995 -87.09973417855461) bank161009 +161010 POINT(42.40001417890363 -87.82770125618207) bank161010 +161011 POINT(41.005244544802686 -87.81859363364127) bank161011 +161012 POINT(41.650211269131525 -88.54056155306618) bank161012 +161013 POINT(42.01602119899751 -86.6966381937265) bank161013 +161014 POINT(41.377179007064925 -88.06612393273986) bank161014 +161015 POINT(42.80480145364961 -86.95139128268431) bank161015 +161016 POINT(41.09142929274868 -88.31930659310927) bank161016 +161017 POINT(41.92955665605956 -88.20502270234667) bank161017 +161018 POINT(41.81633660438425 -87.05473041584408) bank161018 +161019 POINT(41.00135953348448 -86.67145997248718) bank161019 +161020 POINT(42.067229847458556 -88.33531021771219) bank161020 +161021 POINT(42.40361546704776 -87.67288284661755) bank161021 +161022 POINT(41.15581743604882 -88.62855572571763) bank161022 +161023 POINT(41.82773714559959 -86.93852834143011) bank161023 +161024 POINT(42.415511561150076 -88.0421272171824) bank161024 +161025 POINT(41.06925710174182 -88.61411293166091) bank161025 +161026 POINT(42.41911623920879 -88.22040731656354) bank161026 +161027 POINT(41.813797941278736 -88.52151924835921) bank161027 +161028 POINT(42.053395950315114 -88.30267824566707) bank161028 +161029 POINT(42.463382138029985 -87.81797164771194) bank161029 +161030 POINT(41.13535601579981 -87.7154792704861) bank161030 +161031 POINT(42.03702098133592 -87.13157349375025) bank161031 +161032 POINT(41.332350201052826 -87.42821946041094) bank161032 +161033 POINT(41.013769913020326 -87.99754816093838) bank161033 +161034 POINT(41.578547486789596 -87.88738691573666) bank161034 +161035 POINT(41.73887869614078 -86.79172045494443) bank161035 +161036 POINT(41.39275511098334 -87.07562260475933) bank161036 +161037 POINT(42.37073423754624 -88.5765382201525) bank161037 +161038 POINT(42.472320893212284 -87.68791546702339) bank161038 +161039 POINT(41.35358443764618 -88.49326284310266) bank161039 +161040 POINT(41.8535613594224 -88.00143368572967) bank161040 +161041 POINT(42.814975387340105 -87.17941754988415) bank161041 +161042 POINT(40.92071522465244 -88.4447247138277) bank161042 +161043 POINT(42.86321751121403 -87.86099423553367) bank161043 +161044 POINT(41.61163088554225 -87.03646798440934) bank161044 +161045 POINT(41.9572131931743 -87.94032880023926) bank161045 +161046 POINT(42.66775826023198 -88.18673782191105) bank161046 +161047 POINT(41.9141439209618 -86.92413340142589) bank161047 +161048 POINT(42.239301783057556 -87.82148843133369) bank161048 +161049 POINT(42.150622728405246 -87.23611270817982) bank161049 +161050 POINT(41.35188764455273 -87.16899290323151) bank161050 +161051 POINT(42.763277097075346 -87.38188256050225) bank161051 +161052 POINT(42.1956929527678 -87.95248275279053) bank161052 +161053 POINT(42.44258837273468 -87.59475093489903) bank161053 +161054 POINT(41.175199986912 -88.05049406715062) bank161054 +161055 POINT(42.35601626640373 -87.59258873239068) bank161055 +161056 POINT(42.02625694592374 -87.30346504047506) bank161056 +161057 POINT(40.940722282775795 -88.30457373178928) bank161057 +161058 POINT(41.64806952927983 -86.80392201137239) bank161058 +161059 POINT(42.136263328273785 -87.6839529629537) bank161059 +161060 POINT(41.45528468831729 -87.56525560601304) bank161060 +161061 POINT(41.24884658318058 -87.83398268457996) bank161061 +161062 POINT(42.28878175725779 -88.54258682641556) bank161062 +161063 POINT(41.361346201615106 -88.3402990919581) bank161063 +161064 POINT(42.06891742995853 -88.46043671916964) bank161064 +161065 POINT(42.05426759452517 -87.70655613769638) bank161065 +161066 POINT(42.39432972421763 -87.78903103929488) bank161066 +161067 POINT(41.67217935216651 -88.27182546558329) bank161067 +161068 POINT(41.25161752300498 -87.42619443843948) bank161068 +161069 POINT(41.266970029012484 -88.34431275336621) bank161069 +161070 POINT(42.48275780242682 -88.279594220277) bank161070 +161071 POINT(41.7043157369738 -86.67372231132249) bank161071 +161072 POINT(41.828648069724984 -88.05785188345422) bank161072 +161073 POINT(41.857651863143765 -87.96500284500333) bank161073 +161074 POINT(42.32381580929575 -87.35790634054204) bank161074 +161075 POINT(41.91325180012791 -86.8980355127908) bank161075 +161076 POINT(41.539385472099575 -86.75205396638955) bank161076 +161077 POINT(42.153439664218624 -88.3319979034893) bank161077 +161078 POINT(42.51107982909831 -86.98383550903776) bank161078 +161079 POINT(42.52059429507628 -86.9908406762168) bank161079 +161080 POINT(41.50111057858177 -87.41586136323482) bank161080 +161081 POINT(41.66995685916737 -87.01500195390446) bank161081 +161082 POINT(41.74465284720699 -87.1815355231906) bank161082 +161083 POINT(41.53644446028431 -87.06289083528264) bank161083 +161084 POINT(42.53260107084366 -86.85559472535851) bank161084 +161085 POINT(41.32733806470937 -87.97312167123441) bank161085 +161086 POINT(42.37411702667642 -88.42507068690108) bank161086 +161087 POINT(41.27221050785975 -86.89439890684838) bank161087 +161088 POINT(41.82956640794112 -87.07187312085748) bank161088 +161089 POINT(42.376008825735 -87.30495362480008) bank161089 +161090 POINT(41.80510141588691 -88.2414675095401) bank161090 +161091 POINT(42.406699743260425 -87.54366614640202) bank161091 +161092 POINT(41.819145069973956 -88.41710429664559) bank161092 +161093 POINT(41.102851402498715 -87.1160505880968) bank161093 +161094 POINT(41.397603547549814 -87.29312483472698) bank161094 +161095 POINT(41.14935588575178 -88.26958168485518) bank161095 +161096 POINT(41.99726263444269 -86.79712999502158) bank161096 +161097 POINT(41.177785317959255 -88.49815186598349) bank161097 +161098 POINT(42.200173434295685 -88.39292235772696) bank161098 +161099 POINT(42.31841788081119 -87.41302364916554) bank161099 +161100 POINT(41.55187331996241 -87.09033263625001) bank161100 +161101 POINT(41.49333747786115 -88.62782842289845) bank161101 +161102 POINT(41.655856775619604 -87.48675053242276) bank161102 +161103 POINT(41.012170346792466 -86.95442309441734) bank161103 +161104 POINT(41.93435024730563 -87.73117915892153) bank161104 +161105 POINT(41.48357164838991 -88.09715579757003) bank161105 +161106 POINT(41.36752486703527 -88.10655778300662) bank161106 +161107 POINT(41.34292512668781 -86.88110761209764) bank161107 +161108 POINT(41.09688940990336 -88.58013514503918) bank161108 +161109 POINT(41.015887252840955 -88.20622070323698) bank161109 +161110 POINT(41.79755138999429 -88.17920588289968) bank161110 +161111 POINT(41.73043497913044 -88.210067657833) bank161111 +161112 POINT(41.72162282160131 -87.87038510893699) bank161112 +161113 POINT(41.856384130133804 -88.01902261150921) bank161113 +161114 POINT(40.933637258301026 -87.89199585392373) bank161114 +161115 POINT(42.18326273609706 -88.11302032181791) bank161115 +161116 POINT(41.39436300745035 -88.3328921263154) bank161116 +161117 POINT(42.5783356046625 -88.58742400023418) bank161117 +161118 POINT(41.13347695245578 -87.54289679422166) bank161118 +161119 POINT(41.85213365570699 -87.61745590114012) bank161119 +161120 POINT(42.809351402701054 -88.44812282176761) bank161120 +161121 POINT(41.92743013230129 -87.96247830140601) bank161121 +161122 POINT(42.425493101870735 -87.64134484948995) bank161122 +161123 POINT(41.50668102222506 -88.32201051021656) bank161123 +161124 POINT(41.23086960920149 -86.88973787639246) bank161124 +161125 POINT(41.42772162315974 -88.0218767030117) bank161125 +161126 POINT(40.97582198551425 -86.99630586880347) bank161126 +161127 POINT(40.90577057917108 -88.23377567718846) bank161127 +161128 POINT(41.65647277726279 -86.70858367156072) bank161128 +161129 POINT(41.770934660667315 -86.75260902842393) bank161129 +161130 POINT(41.79930267849959 -87.37769167815961) bank161130 +161131 POINT(40.87964183620737 -88.06821999368499) bank161131 +161132 POINT(42.55380879925141 -86.69465866943112) bank161132 +161133 POINT(42.23595160704136 -86.73916849634199) bank161133 +161134 POINT(41.38205669753592 -88.46998723374888) bank161134 +161135 POINT(42.75436641987704 -87.08505095708779) bank161135 +161136 POINT(41.93930974242003 -87.51785300683107) bank161136 +161137 POINT(41.325839762233066 -86.83211260648307) bank161137 +161138 POINT(41.6282618713671 -86.9822324433135) bank161138 +161139 POINT(41.274445715446646 -87.4592765095096) bank161139 +161140 POINT(42.04887414806875 -87.38578188711604) bank161140 +161141 POINT(41.885143165017396 -88.3367359312787) bank161141 +161142 POINT(42.149088295021556 -88.26632241689325) bank161142 +161143 POINT(41.58178961317193 -87.29016368737534) bank161143 +161144 POINT(40.91784714486712 -86.75906603529786) bank161144 +161145 POINT(42.63620112060099 -86.76283090781212) bank161145 +161146 POINT(42.11695142509253 -87.93799323560054) bank161146 +161147 POINT(42.84496259999621 -86.63796328715036) bank161147 +161148 POINT(41.21693106945802 -88.41688735523041) bank161148 +161149 POINT(41.3261713255931 -87.90716488178712) bank161149 +161150 POINT(41.388392688570356 -87.24978341304417) bank161150 +161151 POINT(41.368168074431864 -88.18003251550105) bank161151 +161152 POINT(41.82766271767742 -87.50179527216486) bank161152 +161153 POINT(42.163685525768166 -88.48930902530007) bank161153 +161154 POINT(40.974477996704735 -87.79688804518578) bank161154 +161155 POINT(40.92608056547331 -87.18610590212481) bank161155 +161156 POINT(42.02502019938648 -88.58880607783613) bank161156 +161157 POINT(41.947778139893714 -86.76343879405634) bank161157 +161158 POINT(41.262481141531374 -87.81928522109122) bank161158 +161159 POINT(41.020170332228304 -87.289502347382) bank161159 +161160 POINT(41.96991173007599 -87.35547066202669) bank161160 +161161 POINT(41.7482087785697 -87.78026626955865) bank161161 +161162 POINT(41.68350798287413 -87.43342356282909) bank161162 +161163 POINT(42.41324660094278 -88.09488100412368) bank161163 +161164 POINT(40.95113227943838 -88.00963657844409) bank161164 +161165 POINT(42.430853185696066 -86.90700433596035) bank161165 +161166 POINT(42.40726163601328 -88.18385909501409) bank161166 +161167 POINT(41.54502591975708 -88.55653720457016) bank161167 +161168 POINT(41.01004409389194 -88.52964773325841) bank161168 +161169 POINT(42.05756863181236 -88.29135176810142) bank161169 +161170 POINT(42.093738873860595 -88.61791492453756) bank161170 +161171 POINT(40.98163102574872 -87.94442625732826) bank161171 +161172 POINT(41.638536068686435 -88.3397246045191) bank161172 +161173 POINT(42.07297262366535 -87.44663175722975) bank161173 +161174 POINT(42.69480167041469 -88.57285650032819) bank161174 +161175 POINT(42.74008044835373 -87.3137857491594) bank161175 +161176 POINT(41.9857301970394 -88.29605872284174) bank161176 +161177 POINT(42.30070627303955 -87.30601206738947) bank161177 +161178 POINT(41.05400585063777 -87.91031193446966) bank161178 +161179 POINT(41.220536437693625 -87.3608869726157) bank161179 +161180 POINT(42.108181236125695 -87.1651554315907) bank161180 +161181 POINT(42.815579136359226 -88.01655242223376) bank161181 +161182 POINT(42.73073604543695 -88.57850177059886) bank161182 +161183 POINT(42.59204925222717 -88.08367333676095) bank161183 +161184 POINT(42.278248183751884 -87.02779297997907) bank161184 +161185 POINT(42.836137786853016 -87.59742111184076) bank161185 +161186 POINT(41.74364949767092 -87.87401963806781) bank161186 +161187 POINT(42.22323652118714 -86.73489016396394) bank161187 +161188 POINT(41.66167873152948 -87.8668788267431) bank161188 +161189 POINT(42.63435649279764 -86.93079399066094) bank161189 +161190 POINT(41.53959722470348 -87.98025656592354) bank161190 +161191 POINT(42.42247607244892 -88.58558295006088) bank161191 +161192 POINT(42.45694086332882 -87.45972143695127) bank161192 +161193 POINT(41.89244413381534 -86.99054270172397) bank161193 +161194 POINT(42.47144434443707 -87.51119275927425) bank161194 +161195 POINT(42.65281467353452 -87.76075908963622) bank161195 +161196 POINT(42.38961756708474 -87.7244826335279) bank161196 +161197 POINT(41.799186977664945 -87.67923744593502) bank161197 +161198 POINT(42.51510112801057 -87.33753547296226) bank161198 +161199 POINT(41.38671550247794 -87.52601042459126) bank161199 +161200 POINT(41.805188425587616 -88.04589656879924) bank161200 +161201 POINT(41.765087400984015 -88.34168507627327) bank161201 +161202 POINT(42.661571334611246 -87.70433389509068) bank161202 +161203 POINT(42.69174363586576 -88.44473966995673) bank161203 +161204 POINT(41.389547413460164 -86.89452457125623) bank161204 +161205 POINT(41.13082609039246 -88.49150214583756) bank161205 +161206 POINT(42.43824146650554 -87.91556960031211) bank161206 +161207 POINT(42.77703478310431 -86.87877155370103) bank161207 +161208 POINT(42.15873429393517 -88.07270432316022) bank161208 +161209 POINT(41.70789976254369 -87.57776496644341) bank161209 +161210 POINT(40.95027678198847 -88.49438540934231) bank161210 +161211 POINT(42.536492085552545 -88.01628051763349) bank161211 +161212 POINT(40.993784800951964 -88.53942496676845) bank161212 +161213 POINT(42.73166468061748 -88.48023747489086) bank161213 +161214 POINT(41.99924259011465 -87.92466426495652) bank161214 +161215 POINT(42.462264397109834 -88.27314899780622) bank161215 +161216 POINT(42.32539159964439 -87.28831712919401) bank161216 +161217 POINT(40.90468886216106 -87.04963560687678) bank161217 +161218 POINT(42.02875875241473 -87.60867020790306) bank161218 +161219 POINT(42.503622591599736 -88.0320448874262) bank161219 +161220 POINT(42.86655415692187 -87.47122771621488) bank161220 +161221 POINT(41.39860839801443 -88.24067618451342) bank161221 +161222 POINT(41.51913343748906 -87.22084261256741) bank161222 +161223 POINT(41.89628855243026 -87.34988423414946) bank161223 +161224 POINT(41.42614488325201 -86.7582085577498) bank161224 +161225 POINT(41.00228687681245 -87.93516817319362) bank161225 +161226 POINT(41.657828555194534 -87.80985043373859) bank161226 +161227 POINT(41.39200458680041 -87.29097763347663) bank161227 +161228 POINT(41.224695653606844 -88.32670887883542) bank161228 +161229 POINT(42.67043382575361 -86.67928266343003) bank161229 +161230 POINT(41.99221926512597 -87.73190339606144) bank161230 +161231 POINT(41.601580329064674 -88.4604918636584) bank161231 +161232 POINT(41.782156359052365 -87.05918215973209) bank161232 +161233 POINT(42.70028617659964 -88.21712116665074) bank161233 +161234 POINT(42.09103614586979 -87.03074798669263) bank161234 +161235 POINT(42.56812737575916 -86.935749994527) bank161235 +161236 POINT(41.410156572553575 -88.07306081068316) bank161236 +161237 POINT(42.4937126957372 -88.25874677237559) bank161237 +161238 POINT(42.46744769155794 -88.24281150857857) bank161238 +161239 POINT(42.2741155460943 -87.88765280962234) bank161239 +161240 POINT(42.2465852164379 -88.08710863617532) bank161240 +161241 POINT(41.44133554930525 -87.50773370212896) bank161241 +161242 POINT(41.27401575612278 -87.8642772356474) bank161242 +161243 POINT(41.50692350969914 -87.52454578093938) bank161243 +161244 POINT(42.77187149978315 -87.16503615863813) bank161244 +161245 POINT(42.46451257784398 -87.68424731020272) bank161245 +161246 POINT(42.57331503594475 -88.33582049403408) bank161246 +161247 POINT(41.7415942064979 -88.18881853214641) bank161247 +161248 POINT(41.88151914285996 -86.63243087695724) bank161248 +161249 POINT(42.560559919629625 -87.83712984566965) bank161249 +161250 POINT(41.7997468796698 -87.55514533374522) bank161250 +161251 POINT(42.33427664808861 -87.3990699079253) bank161251 +161252 POINT(42.184019564134516 -87.65654947512448) bank161252 +161253 POINT(42.530338806015806 -88.5727467235355) bank161253 +161254 POINT(41.215194145030324 -88.30725125314298) bank161254 +161255 POINT(42.35781186652509 -88.06397261758255) bank161255 +161256 POINT(41.203401637748954 -86.72319932061887) bank161256 +161257 POINT(41.7633588017367 -88.05056736009452) bank161257 +161258 POINT(41.677495356503336 -87.41271400671974) bank161258 +161259 POINT(41.63981710860272 -87.56109692944825) bank161259 +161260 POINT(42.48471284539303 -86.75408004317018) bank161260 +161261 POINT(41.94556130849628 -87.72619354405403) bank161261 +161262 POINT(42.053435372809055 -86.64862449106218) bank161262 +161263 POINT(41.78083147481394 -87.84393144681731) bank161263 +161264 POINT(42.22674892105348 -86.69805819898268) bank161264 +161265 POINT(41.37856181181857 -88.31903929964747) bank161265 +161266 POINT(42.46730924319884 -86.75745763660906) bank161266 +161267 POINT(42.33073864792489 -86.95344988509508) bank161267 +161268 POINT(42.519380367186535 -88.02230905793968) bank161268 +161269 POINT(41.10294537558104 -87.34677650158473) bank161269 +161270 POINT(42.2197370682981 -87.97951534095706) bank161270 +161271 POINT(41.3471079542065 -87.77906834552961) bank161271 +161272 POINT(42.1899599770366 -87.20390779202863) bank161272 +161273 POINT(42.26205988755652 -87.29913799513687) bank161273 +161274 POINT(42.040216765154426 -87.4437339078252) bank161274 +161275 POINT(42.226887354543685 -88.62590877848976) bank161275 +161276 POINT(42.872502255061434 -87.98873378541788) bank161276 +161277 POINT(41.754466501093596 -88.48687441316159) bank161277 +161278 POINT(41.46507256203998 -87.30208103403739) bank161278 +161279 POINT(41.445885206011944 -86.79604758734293) bank161279 +161280 POINT(41.82578135241719 -88.13584252951215) bank161280 +161281 POINT(41.98323498016483 -88.47224145889909) bank161281 +161282 POINT(41.134088741169634 -86.70867349611471) bank161282 +161283 POINT(42.04053301483874 -86.77911521420074) bank161283 +161284 POINT(42.40964886387686 -88.44517573118762) bank161284 +161285 POINT(42.1124414431559 -88.0321734967926) bank161285 +161286 POINT(41.640672039423144 -87.92235559720257) bank161286 +161287 POINT(42.719717837587154 -86.95422934192764) bank161287 +161288 POINT(42.26362679378384 -87.08228798348172) bank161288 +161289 POINT(42.06703572383054 -87.95758879471173) bank161289 +161290 POINT(41.444598132971066 -87.01149505257895) bank161290 +161291 POINT(41.20049606776622 -87.32818181302036) bank161291 +161292 POINT(41.748815039963084 -87.35632023550308) bank161292 +161293 POINT(42.83934664211899 -88.49993729049399) bank161293 +161294 POINT(41.27519163630146 -87.56523100360177) bank161294 +161295 POINT(41.28182930347796 -87.40384531850282) bank161295 +161296 POINT(41.03730378316782 -88.45377491236613) bank161296 +161297 POINT(42.27476009009475 -87.78638109834957) bank161297 +161298 POINT(42.46275990521982 -86.8931177934092) bank161298 +161299 POINT(41.73612097982413 -86.94942058288687) bank161299 +161300 POINT(42.33683353883803 -87.30617975166646) bank161300 +161301 POINT(42.44543640371909 -87.40944503571288) bank161301 +161302 POINT(42.5882782211362 -87.21505869999896) bank161302 +161303 POINT(42.08094589902814 -87.08670417304376) bank161303 +161304 POINT(42.600241297372825 -88.03272183988572) bank161304 +161305 POINT(41.49938035885119 -87.0180461237813) bank161305 +161306 POINT(41.401674492612905 -86.97898483717647) bank161306 +161307 POINT(41.589562562976255 -87.56806915758082) bank161307 +161308 POINT(42.87297879834325 -87.17341192533058) bank161308 +161309 POINT(41.4873918014854 -86.83074481120318) bank161309 +161310 POINT(42.21444843991272 -87.33178801290478) bank161310 +161311 POINT(42.0985962344236 -86.78327776458555) bank161311 +161312 POINT(41.29685741571846 -88.24566071803298) bank161312 +161313 POINT(42.44345629767257 -87.81633873859542) bank161313 +161314 POINT(41.25244564156107 -87.19236349611049) bank161314 +161315 POINT(41.59209599673133 -86.67825375845452) bank161315 +161316 POINT(42.420552496920855 -87.79630330737326) bank161316 +161317 POINT(41.99061619386726 -87.45325226443128) bank161317 +161318 POINT(42.68082900980756 -87.57383596777451) bank161318 +161319 POINT(41.1358480733778 -87.74555614972948) bank161319 +161320 POINT(41.70475633516889 -87.85823001931774) bank161320 +161321 POINT(41.851423011372496 -87.77360199832097) bank161321 +161322 POINT(41.968075943963 -88.3461172592758) bank161322 +161323 POINT(41.27449563319081 -87.08491476377388) bank161323 +161324 POINT(41.728780477016144 -87.95066019074457) bank161324 +161325 POINT(42.319073452451605 -88.49595910751466) bank161325 +161326 POINT(41.46377145971069 -87.2164891139946) bank161326 +161327 POINT(41.225736336385104 -86.96589458868127) bank161327 +161328 POINT(41.30597867041057 -86.83131575578233) bank161328 +161329 POINT(41.02277424992221 -87.21545374079346) bank161329 +161330 POINT(41.95881259872158 -87.55184190893262) bank161330 +161331 POINT(42.49108952479019 -87.31575099408309) bank161331 +161332 POINT(41.8072302549321 -86.72009028846996) bank161332 +161333 POINT(41.15364797293908 -87.60138102043527) bank161333 +161334 POINT(42.107178344713645 -87.11447003068933) bank161334 +161335 POINT(42.54557656423488 -87.20672183984811) bank161335 +161336 POINT(41.99669951103711 -87.68113758977618) bank161336 +161337 POINT(42.656140469895064 -88.62543506824632) bank161337 +161338 POINT(41.93259405564828 -88.5646386348332) bank161338 +161339 POINT(41.31971130857873 -86.88211030678211) bank161339 +161340 POINT(42.433640631913846 -87.3027589079318) bank161340 +161341 POINT(41.63249733467338 -86.799609122212) bank161341 +161342 POINT(41.04269394217214 -87.80376685314087) bank161342 +161343 POINT(41.25548325222765 -88.46592066316794) bank161343 +161344 POINT(41.7837778013632 -87.84543968868918) bank161344 +161345 POINT(41.11810098292824 -88.37204496771572) bank161345 +161346 POINT(40.94033175016532 -86.74445288896355) bank161346 +161347 POINT(40.9377309242851 -86.71307532676352) bank161347 +161348 POINT(41.22034109137099 -87.57928585971592) bank161348 +161349 POINT(40.87845220832862 -88.10661020250136) bank161349 +161350 POINT(41.690533594117525 -87.94692486532398) bank161350 +161351 POINT(42.03069715584272 -87.42902943816568) bank161351 +161352 POINT(41.84441995815886 -87.59047524054438) bank161352 +161353 POINT(41.90536912957997 -87.5085303317398) bank161353 +161354 POINT(41.881607550937034 -88.58926280971001) bank161354 +161355 POINT(40.94494498559443 -86.65552936351881) bank161355 +161356 POINT(41.11071241849333 -87.15888427015429) bank161356 +161357 POINT(42.61597200879766 -87.25301487919256) bank161357 +161358 POINT(42.51735141071782 -87.97755772778017) bank161358 +161359 POINT(41.30443404343008 -87.07919013408063) bank161359 +161360 POINT(41.80076039573705 -88.02937548010914) bank161360 +161361 POINT(41.461321609852504 -86.85343094679206) bank161361 +161362 POINT(42.530649313924535 -87.13313184326401) bank161362 +161363 POINT(41.40825345721415 -87.80397158946077) bank161363 +161364 POINT(41.83153447074052 -87.68057930310573) bank161364 +161365 POINT(41.24649779729016 -87.58213111953708) bank161365 +161366 POINT(42.45383543681235 -87.73790962444869) bank161366 +161367 POINT(42.86144984376056 -86.84015973526273) bank161367 +161368 POINT(41.164289042402785 -87.86038724050321) bank161368 +161369 POINT(41.45769213251601 -86.86994748930348) bank161369 +161370 POINT(42.54008510360506 -87.50173216550634) bank161370 +161371 POINT(42.4271510677081 -88.30219096358847) bank161371 +161372 POINT(42.60670812307846 -87.21040071874319) bank161372 +161373 POINT(41.49197290601758 -87.81877049195474) bank161373 +161374 POINT(42.08641775519001 -88.17616636016767) bank161374 +161375 POINT(42.48485199153739 -87.61420528092253) bank161375 +161376 POINT(41.02097401623197 -88.39567652814735) bank161376 +161377 POINT(42.1061302768081 -87.76622708813508) bank161377 +161378 POINT(42.36098783210375 -87.93697057487674) bank161378 +161379 POINT(41.900436610941924 -87.4210305261721) bank161379 +161380 POINT(41.366853150327366 -87.72854969812892) bank161380 +161381 POINT(42.14808241470663 -86.94534747269743) bank161381 +161382 POINT(41.98136501201783 -88.11208720165698) bank161382 +161383 POINT(41.936730177653594 -87.58417852471948) bank161383 +161384 POINT(41.41490214991355 -87.82795763108707) bank161384 +161385 POINT(41.64432095404362 -87.06938887512513) bank161385 +161386 POINT(40.91118229090966 -88.23615293460044) bank161386 +161387 POINT(42.69906771924678 -88.06539115165876) bank161387 +161388 POINT(41.130117570848356 -88.40911318671776) bank161388 +161389 POINT(42.687105246252415 -87.56526929260079) bank161389 +161390 POINT(41.82302666154986 -86.8592340860348) bank161390 +161391 POINT(40.971596334819395 -86.91172490413983) bank161391 +161392 POINT(42.66452086076486 -87.73197269242087) bank161392 +161393 POINT(41.08762235226469 -88.0769890132665) bank161393 +161394 POINT(41.62978031257828 -87.80570260249685) bank161394 +161395 POINT(41.925188883282054 -87.3248089465803) bank161395 +161396 POINT(42.2881154454488 -86.89211736890394) bank161396 +161397 POINT(41.84821150465318 -88.07844164327861) bank161397 +161398 POINT(40.94503860921128 -86.88369477269565) bank161398 +161399 POINT(41.110345176561836 -88.37581389305447) bank161399 +161400 POINT(42.68749134768296 -87.22375185694189) bank161400 +161401 POINT(41.72857295146633 -88.36995239200073) bank161401 +161402 POINT(41.15871905484359 -88.54451130299461) bank161402 +161403 POINT(42.19316924820379 -87.93261624005882) bank161403 +161404 POINT(41.59525144799864 -86.81747181186614) bank161404 +161405 POINT(41.6194478178219 -87.42285305625313) bank161405 +161406 POINT(41.083363886634366 -86.77842471741702) bank161406 +161407 POINT(42.73998755231038 -86.83126516788106) bank161407 +161408 POINT(42.15384094394495 -87.14208906035711) bank161408 +161409 POINT(42.831557275104124 -86.75576059246886) bank161409 +161410 POINT(41.86344833343532 -87.03292777454507) bank161410 +161411 POINT(40.95990909890642 -88.39171592305722) bank161411 +161412 POINT(41.38646808937646 -87.83840390832319) bank161412 +161413 POINT(41.17751427456319 -88.11350311798788) bank161413 +161414 POINT(41.76439925302887 -87.96433991440928) bank161414 +161415 POINT(40.95980572275786 -87.24327668630755) bank161415 +161416 POINT(41.10163469045107 -88.3011585265159) bank161416 +161417 POINT(42.41153569886874 -86.99294545220708) bank161417 +161418 POINT(42.85013974977311 -86.63775165387781) bank161418 +161419 POINT(42.69180279413034 -87.67780749375393) bank161419 +161420 POINT(42.55215409825769 -86.87847846206013) bank161420 +161421 POINT(41.703662239357946 -87.88858343976328) bank161421 +161422 POINT(41.544117839559405 -86.88640239541132) bank161422 +161423 POINT(42.0806946998801 -88.55949743310379) bank161423 +161424 POINT(42.74648461078606 -88.5515196760847) bank161424 +161425 POINT(42.80914373663142 -87.87200366318106) bank161425 +161426 POINT(40.97244471362947 -86.87335333372224) bank161426 +161427 POINT(42.58995069092518 -88.42732655581575) bank161427 +161428 POINT(41.8387788736779 -87.12408206866601) bank161428 +161429 POINT(41.23396832265953 -88.30159165098799) bank161429 +161430 POINT(42.37416444270362 -87.93190631962538) bank161430 +161431 POINT(42.65436811215235 -88.44406585464759) bank161431 +161432 POINT(40.88346872109034 -87.18145802466346) bank161432 +161433 POINT(41.984253184947406 -87.27111792144298) bank161433 +161434 POINT(42.550609176304334 -88.41596104097994) bank161434 +161435 POINT(41.75043277534584 -87.07184890042454) bank161435 +161436 POINT(41.5209069410097 -88.24124877515254) bank161436 +161437 POINT(42.82974575221794 -88.33729485951109) bank161437 +161438 POINT(41.36136267509459 -87.73800437424468) bank161438 +161439 POINT(41.17084885208745 -88.16476089191136) bank161439 +161440 POINT(41.94195121781008 -88.45536558466968) bank161440 +161441 POINT(42.63604034832755 -87.02524955295762) bank161441 +161442 POINT(41.37950497806044 -88.60910543742749) bank161442 +161443 POINT(41.07432222127272 -86.93885516230142) bank161443 +161444 POINT(41.00965555047549 -87.42053659565529) bank161444 +161445 POINT(41.97001224295145 -87.45084666323399) bank161445 +161446 POINT(42.55374054396093 -88.04863960582851) bank161446 +161447 POINT(41.96206011549939 -87.90735126405002) bank161447 +161448 POINT(42.37444137643001 -86.69766541740732) bank161448 +161449 POINT(41.27058401670103 -87.9714928220173) bank161449 +161450 POINT(41.897188517129855 -88.07970011895868) bank161450 +161451 POINT(42.550426586971 -87.56215459053475) bank161451 +161452 POINT(42.346609999235454 -88.03872694445113) bank161452 +161453 POINT(41.9885082205191 -88.55070972129552) bank161453 +161454 POINT(41.07079586782168 -86.77823518250645) bank161454 +161455 POINT(42.41275849956257 -87.31505567954737) bank161455 +161456 POINT(41.580438317986946 -87.3597582209964) bank161456 +161457 POINT(42.41163723534606 -87.65459013975652) bank161457 +161458 POINT(42.409581391375134 -86.88148958642327) bank161458 +161459 POINT(41.57798026113798 -88.04605088099888) bank161459 +161460 POINT(42.46898768634738 -86.72179359546044) bank161460 +161461 POINT(42.008857817581436 -86.87203492087824) bank161461 +161462 POINT(42.17150974230089 -87.60384101419457) bank161462 +161463 POINT(42.37847892370656 -88.15645509658434) bank161463 +161464 POINT(40.99810104052317 -87.25098170883388) bank161464 +161465 POINT(42.25297531203702 -87.3066451999155) bank161465 +161466 POINT(40.98109270442653 -87.38841759649257) bank161466 +161467 POINT(41.99777856519819 -87.97950616806858) bank161467 +161468 POINT(41.24864657310993 -88.37009902684197) bank161468 +161469 POINT(42.61380551682213 -87.67501201542979) bank161469 +161470 POINT(42.19604061189449 -88.5204790794264) bank161470 +161471 POINT(42.74326812652325 -88.46121810155307) bank161471 +161472 POINT(42.41531128324649 -87.84538481940933) bank161472 +161473 POINT(42.589979185359816 -88.57801660993563) bank161473 +161474 POINT(41.30593615319376 -87.83415556888663) bank161474 +161475 POINT(41.2497573782036 -87.7655700291087) bank161475 +161476 POINT(41.04935318057977 -88.61919064178727) bank161476 +161477 POINT(41.13444209503338 -88.5771868376822) bank161477 +161478 POINT(42.54803008773149 -87.47280424416003) bank161478 +161479 POINT(42.456512929386726 -87.4959003335863) bank161479 +161480 POINT(42.49400300448089 -87.5401371661089) bank161480 +161481 POINT(41.25642012250762 -87.83666819707477) bank161481 +161482 POINT(41.931657741648095 -87.76974728548085) bank161482 +161483 POINT(42.10327725272794 -87.78040542378888) bank161483 +161484 POINT(41.42492121498403 -88.00491125358255) bank161484 +161485 POINT(41.0645568608945 -87.16021303916226) bank161485 +161486 POINT(41.6860673720425 -88.40396420669519) bank161486 +161487 POINT(42.38252269927107 -86.81970980251833) bank161487 +161488 POINT(42.67956845403853 -87.75374699358225) bank161488 +161489 POINT(42.21205274233304 -87.76855077839767) bank161489 +161490 POINT(42.23701367972768 -87.13739466764498) bank161490 +161491 POINT(41.21980154128924 -87.54955792250442) bank161491 +161492 POINT(41.36723223937705 -88.04783524581316) bank161492 +161493 POINT(42.23146510067672 -87.2312903789791) bank161493 +161494 POINT(42.37050724908377 -87.33366671371108) bank161494 +161495 POINT(41.96112860780974 -87.29386054011631) bank161495 +161496 POINT(40.90262151738061 -88.12202805274913) bank161496 +161497 POINT(42.42654759593077 -88.22114785229887) bank161497 +161498 POINT(40.934681491179845 -87.72159473347766) bank161498 +161499 POINT(41.14296099991823 -88.52734129609595) bank161499 +161500 POINT(42.453560543488116 -87.85678175187529) bank161500 +161501 POINT(41.62571418024937 -86.77184997299031) bank161501 +161502 POINT(41.750093057048886 -88.14665978845447) bank161502 +161503 POINT(41.2054041516493 -88.25505081056752) bank161503 +161504 POINT(41.28464807055654 -86.92504674020218) bank161504 +161505 POINT(41.65924684597863 -87.88383868920846) bank161505 +161506 POINT(41.84255452882851 -88.11266159728237) bank161506 +161507 POINT(42.490620619030764 -87.91653294879634) bank161507 +161508 POINT(41.69946107359243 -87.96020050818596) bank161508 +161509 POINT(42.374437575943574 -87.43629803065873) bank161509 +161510 POINT(42.73041156440207 -87.24422732864824) bank161510 +161511 POINT(41.983674166939096 -88.40279889086838) bank161511 +161512 POINT(41.67160390298783 -87.90802654326009) bank161512 +161513 POINT(42.34143421007096 -86.71726169946375) bank161513 +161514 POINT(41.23041621588422 -87.23960555725778) bank161514 +161515 POINT(41.109093773545176 -87.89387315317921) bank161515 +161516 POINT(41.64546652983705 -87.9592421893754) bank161516 +161517 POINT(41.67111407992129 -86.87810421933852) bank161517 +161518 POINT(41.55472352444252 -87.00239645768856) bank161518 +161519 POINT(40.93464852967001 -87.35717041347463) bank161519 +161520 POINT(41.305231931865556 -86.90967164988635) bank161520 +161521 POINT(41.84434872467959 -88.03746557447911) bank161521 +161522 POINT(41.19484616500266 -88.57523366991015) bank161522 +161523 POINT(42.09500282254475 -87.70788664221064) bank161523 +161524 POINT(41.908222028661044 -87.67150955377882) bank161524 +161525 POINT(41.19152783877865 -87.16073908413021) bank161525 +161526 POINT(41.94837411765376 -86.71403044470503) bank161526 +161527 POINT(42.201985374085005 -88.62642981251125) bank161527 +161528 POINT(41.774330016553066 -87.94989026919265) bank161528 +161529 POINT(41.219023253894505 -87.09915806459746) bank161529 +161530 POINT(42.528577068848534 -86.792996822519) bank161530 +161531 POINT(42.23647385894056 -88.0865085479124) bank161531 +161532 POINT(41.75300873703939 -88.53113521098828) bank161532 +161533 POINT(40.985920274874445 -87.53609774394053) bank161533 +161534 POINT(41.973818354950296 -86.68600134428343) bank161534 +161535 POINT(41.151033807723294 -88.4151128264534) bank161535 +161536 POINT(42.18113930531725 -86.7142110396475) bank161536 +161537 POINT(41.523823702542735 -86.97922214850256) bank161537 +161538 POINT(40.916532325174906 -86.76988315735177) bank161538 +161539 POINT(41.64598686452662 -86.73139003139397) bank161539 +161540 POINT(41.54903617368126 -87.57240755968972) bank161540 +161541 POINT(41.026714991206774 -86.961748668021) bank161541 +161542 POINT(42.39313962264654 -88.50406690374699) bank161542 +161543 POINT(42.59923531594426 -87.40102459295625) bank161543 +161544 POINT(41.88690361434186 -87.89961484859332) bank161544 +161545 POINT(42.30671075037378 -87.09059967527274) bank161545 +161546 POINT(41.22601074004449 -86.7921658676206) bank161546 +161547 POINT(41.379857473159795 -87.71288683154502) bank161547 +161548 POINT(42.815019340912016 -88.62519675355252) bank161548 +161549 POINT(41.249678315705665 -87.43074365710486) bank161549 +161550 POINT(42.21883824262226 -87.46546526369323) bank161550 +161551 POINT(41.323447360139745 -87.42259812693844) bank161551 +161552 POINT(42.06811062609403 -88.50408665528613) bank161552 +161553 POINT(42.074401380766446 -87.10137013944355) bank161553 +161554 POINT(40.89118305520522 -87.5718500419039) bank161554 +161555 POINT(41.846361400546634 -88.46499480368925) bank161555 +161556 POINT(41.94885121770296 -87.32563658689071) bank161556 +161557 POINT(41.84243667975713 -87.05532609676459) bank161557 +161558 POINT(42.613281157022094 -87.98327407078636) bank161558 +161559 POINT(41.83169277885713 -87.58791471693561) bank161559 +161560 POINT(42.38616143849576 -88.51949513606428) bank161560 +161561 POINT(41.967205299706514 -87.64654540185711) bank161561 +161562 POINT(41.06333697485565 -87.90753995060093) bank161562 +161563 POINT(42.85260658131167 -86.83064759492518) bank161563 +161564 POINT(42.50400406092634 -87.80853860492938) bank161564 +161565 POINT(41.73976620294758 -88.39017618426708) bank161565 +161566 POINT(41.58762716016781 -88.0097627115568) bank161566 +161567 POINT(42.506252998136894 -86.63313720409671) bank161567 +161568 POINT(41.182932279251574 -87.07728239593524) bank161568 +161569 POINT(41.04286405173669 -87.65566760136775) bank161569 +161570 POINT(41.13778356199697 -88.36068244969805) bank161570 +161571 POINT(41.30329079796904 -87.40565243841982) bank161571 +161572 POINT(40.918609001199705 -87.38782004873693) bank161572 +161573 POINT(41.74945388360358 -88.16368714746255) bank161573 +161574 POINT(41.12408685304652 -87.135819316077) bank161574 +161575 POINT(41.24795143551808 -86.7229166882909) bank161575 +161576 POINT(42.831036851277304 -88.0608753614601) bank161576 +161577 POINT(41.480702891344556 -87.98032731256531) bank161577 +161578 POINT(42.23327029477357 -86.8845262388112) bank161578 +161579 POINT(42.17313009551376 -86.79458085654984) bank161579 +161580 POINT(42.50888333782591 -86.80952390289598) bank161580 +161581 POINT(42.57327456787412 -87.6741453683132) bank161581 +161582 POINT(41.12974170877913 -87.90628968844808) bank161582 +161583 POINT(40.925916505231896 -87.50575173921527) bank161583 +161584 POINT(42.508744579764915 -88.34272802369632) bank161584 +161585 POINT(40.892216488136626 -88.47113170430829) bank161585 +161586 POINT(42.410700583076064 -86.78795458540907) bank161586 +161587 POINT(42.632054728901366 -88.12402073076771) bank161587 +161588 POINT(42.30177075054236 -88.06512257184025) bank161588 +161589 POINT(42.4075492678864 -87.9782371274857) bank161589 +161590 POINT(41.4077486105177 -87.8909970447199) bank161590 +161591 POINT(42.32299897761145 -88.6027449785688) bank161591 +161592 POINT(42.55922449041164 -86.783830277274) bank161592 +161593 POINT(41.111071991357306 -87.49916191600464) bank161593 +161594 POINT(41.000296356301796 -87.627371560179) bank161594 +161595 POINT(41.07368310612662 -87.9382797508837) bank161595 +161596 POINT(41.14098639449268 -87.78553633261734) bank161596 +161597 POINT(41.40936951859987 -87.10980663970506) bank161597 +161598 POINT(42.80531909520546 -87.05487401875503) bank161598 +161599 POINT(42.2934424377182 -87.59689658678765) bank161599 +161600 POINT(41.3665574560992 -88.61561681261338) bank161600 +161601 POINT(41.73984666599284 -87.20020868535543) bank161601 +161602 POINT(41.696428210528985 -87.7839274978954) bank161602 +161603 POINT(42.86353760351348 -87.63953884298114) bank161603 +161604 POINT(41.269137994523106 -87.4514514928175) bank161604 +161605 POINT(42.779180409460274 -87.26938147949294) bank161605 +161606 POINT(41.06291162393073 -86.76998754186155) bank161606 +161607 POINT(40.95415718822983 -87.52591772423429) bank161607 +161608 POINT(41.80046201008167 -88.39664586128653) bank161608 +161609 POINT(41.789237470807954 -87.49283118277745) bank161609 +161610 POINT(42.555419302833364 -87.17599054471503) bank161610 +161611 POINT(42.76390188681207 -88.38222871602015) bank161611 +161612 POINT(42.39618833289941 -86.73101096813953) bank161612 +161613 POINT(42.82124817670621 -88.40305412498415) bank161613 +161614 POINT(42.144914401087064 -87.8876746317083) bank161614 +161615 POINT(41.87852867458923 -88.38318836237077) bank161615 +161616 POINT(42.664345977972964 -86.82717403911776) bank161616 +161617 POINT(41.01874443514343 -86.68361887459044) bank161617 +161618 POINT(41.46103348504924 -87.56028448174665) bank161618 +161619 POINT(42.31204991408556 -87.55538004804264) bank161619 +161620 POINT(41.50911894570786 -87.94936858575421) bank161620 +161621 POINT(41.1237374387738 -87.34543918865361) bank161621 +161622 POINT(41.187151313283294 -88.01927700151592) bank161622 +161623 POINT(42.434085969920076 -88.46694794645052) bank161623 +161624 POINT(40.88301644139632 -87.84774910087087) bank161624 +161625 POINT(42.10733732600458 -87.48716589108834) bank161625 +161626 POINT(41.98142515069217 -87.39335402854776) bank161626 +161627 POINT(41.81341636640539 -87.29797306902488) bank161627 +161628 POINT(42.6162723486842 -87.37658082799604) bank161628 +161629 POINT(41.444393881689216 -86.95256042917833) bank161629 +161630 POINT(42.606685707343104 -87.39142295170068) bank161630 +161631 POINT(41.06655579489561 -87.57505693561303) bank161631 +161632 POINT(41.98566627351992 -88.33738579647182) bank161632 +161633 POINT(41.13566260976287 -87.66729809937185) bank161633 +161634 POINT(41.2834405063169 -88.49575770199456) bank161634 +161635 POINT(41.63399477162764 -87.75696151879472) bank161635 +161636 POINT(41.49749476605915 -87.34323906844403) bank161636 +161637 POINT(42.7942304250207 -88.18223998462322) bank161637 +161638 POINT(41.46953282227978 -87.99824080118351) bank161638 +161639 POINT(42.81028531930638 -87.7538398877236) bank161639 +161640 POINT(41.73036915023268 -87.43591631780885) bank161640 +161641 POINT(42.102817450635364 -88.32553160709317) bank161641 +161642 POINT(41.06729337377152 -87.14593945504916) bank161642 +161643 POINT(42.338410614766914 -87.96769617913444) bank161643 +161644 POINT(42.52045769471832 -88.3587858714618) bank161644 +161645 POINT(41.87695081238805 -86.69413253403366) bank161645 +161646 POINT(42.55251501285525 -86.83711953702759) bank161646 +161647 POINT(41.812208127898764 -87.33792597663479) bank161647 +161648 POINT(41.49303378423812 -87.3486592709919) bank161648 +161649 POINT(42.70387317850463 -88.6294809648213) bank161649 +161650 POINT(41.48306986627574 -88.35751563282653) bank161650 +161651 POINT(41.73723137289468 -87.78344870297649) bank161651 +161652 POINT(42.600155935897185 -86.71513550836336) bank161652 +161653 POINT(41.42112368145414 -87.91008374537631) bank161653 +161654 POINT(42.29520567901008 -87.80851013243371) bank161654 +161655 POINT(41.302186067007334 -86.70614379380324) bank161655 +161656 POINT(42.00928028252205 -88.50669605784265) bank161656 +161657 POINT(42.6980565018415 -88.11964411709351) bank161657 +161658 POINT(41.45439213616333 -88.53889115025228) bank161658 +161659 POINT(42.17370318247224 -88.18746315203587) bank161659 +161660 POINT(41.305205553832316 -87.85187119143905) bank161660 +161661 POINT(42.111578807225804 -87.51426798423) bank161661 +161662 POINT(42.57094483481722 -87.54373809456227) bank161662 +161663 POINT(41.87579576916359 -88.08983911856771) bank161663 +161664 POINT(41.44534361098879 -88.49896878337185) bank161664 +161665 POINT(41.11154686632294 -87.5608608107108) bank161665 +161666 POINT(42.71045732846191 -87.70804606619063) bank161666 +161667 POINT(41.244718605640564 -87.00624737529445) bank161667 +161668 POINT(41.95830396780909 -88.13814593509345) bank161668 +161669 POINT(42.23224039427976 -88.1463801218156) bank161669 +161670 POINT(41.20619799107829 -87.97717466337338) bank161670 +161671 POINT(42.79854174796439 -87.78090902116381) bank161671 +161672 POINT(42.40198876743383 -88.08394674532533) bank161672 +161673 POINT(42.63404143981904 -87.57391292312765) bank161673 +161674 POINT(42.41771571420021 -87.79438473337072) bank161674 +161675 POINT(42.47428649989711 -87.36584158655717) bank161675 +161676 POINT(42.44966146253708 -88.1971939742645) bank161676 +161677 POINT(40.92026871893727 -88.56240751535647) bank161677 +161678 POINT(42.45756620197646 -87.08749039800537) bank161678 +161679 POINT(41.30974308489438 -87.66945218711354) bank161679 +161680 POINT(42.537809510178135 -87.64872942382628) bank161680 +161681 POINT(42.53406192895067 -87.31880308065348) bank161681 +161682 POINT(41.73495559253171 -88.49921595199307) bank161682 +161683 POINT(42.561485097212596 -87.06610356176265) bank161683 +161684 POINT(41.08768211512007 -86.91693217663098) bank161684 +161685 POINT(41.60970610523448 -87.82388417996154) bank161685 +161686 POINT(41.35291536675827 -87.04158993426458) bank161686 +161687 POINT(42.073762471911955 -87.99080517643871) bank161687 +161688 POINT(41.64474325015753 -87.73807421087619) bank161688 +161689 POINT(41.33607130354333 -88.2315107369467) bank161689 +161690 POINT(42.6247228916845 -87.37135301348275) bank161690 +161691 POINT(42.24410443478574 -87.5714667972253) bank161691 +161692 POINT(41.79226252889497 -86.94662643407993) bank161692 +161693 POINT(41.228159854396914 -88.45208010680402) bank161693 +161694 POINT(41.419389516508936 -88.44970044539149) bank161694 +161695 POINT(42.48744859129245 -86.78896080755395) bank161695 +161696 POINT(42.02703212259709 -87.26883515111432) bank161696 +161697 POINT(41.54386524163795 -88.3499452155206) bank161697 +161698 POINT(42.591774238145575 -87.18324605344894) bank161698 +161699 POINT(42.35812339347984 -87.80739082248417) bank161699 +161700 POINT(41.50957454963395 -88.45528739161587) bank161700 +161701 POINT(41.3145791047016 -87.35124781440449) bank161701 +161702 POINT(41.55031515408097 -86.65555449543356) bank161702 +161703 POINT(41.64283427747922 -88.33807438586827) bank161703 +161704 POINT(40.92666309565604 -88.04589341480569) bank161704 +161705 POINT(41.009757175430906 -88.44204876593359) bank161705 +161706 POINT(41.59846156025181 -88.41127107934997) bank161706 +161707 POINT(41.75188621328828 -88.00785401132397) bank161707 +161708 POINT(41.32789544293871 -88.03214681078282) bank161708 +161709 POINT(41.32844973885976 -86.81652379715494) bank161709 +161710 POINT(42.19538057120203 -86.94076527799285) bank161710 +161711 POINT(42.23103058100717 -87.95540005658509) bank161711 +161712 POINT(42.83537358893622 -86.92538055175444) bank161712 +161713 POINT(41.77678733195544 -88.5664816290162) bank161713 +161714 POINT(41.47978189806936 -87.75056156517158) bank161714 +161715 POINT(41.44281966165375 -87.61471793580986) bank161715 +161716 POINT(41.88801001524337 -87.98636946066435) bank161716 +161717 POINT(42.06042816630888 -87.29008922722082) bank161717 +161718 POINT(42.21675838718607 -87.70085624746095) bank161718 +161719 POINT(41.71580025253085 -87.19762283403165) bank161719 +161720 POINT(42.812971632469925 -88.12003948089833) bank161720 +161721 POINT(41.13636378177826 -87.2887998004003) bank161721 +161722 POINT(42.3476010796392 -88.38288634514262) bank161722 +161723 POINT(41.8023980779853 -87.97857003364778) bank161723 +161724 POINT(42.658285856302996 -87.11327669734234) bank161724 +161725 POINT(42.159452323480224 -88.12665667213717) bank161725 +161726 POINT(41.39087263081376 -87.72599281857775) bank161726 +161727 POINT(42.57339835709501 -88.16518742634831) bank161727 +161728 POINT(41.990518582679876 -88.5119230263227) bank161728 +161729 POINT(42.76529252538255 -87.956986609762) bank161729 +161730 POINT(42.73312411477468 -88.47101055909829) bank161730 +161731 POINT(41.00879646764325 -88.03143280413857) bank161731 +161732 POINT(42.48440676046919 -87.82429374549959) bank161732 +161733 POINT(42.26253776740473 -87.45982234088497) bank161733 +161734 POINT(42.31962526011109 -87.19618278345803) bank161734 +161735 POINT(42.670140663221964 -88.62786425251997) bank161735 +161736 POINT(42.02318729911606 -87.49214702772707) bank161736 +161737 POINT(42.30030014509278 -87.40252864985818) bank161737 +161738 POINT(41.743125237605526 -88.2547121197725) bank161738 +161739 POINT(42.09932666456926 -88.5174090398923) bank161739 +161740 POINT(42.53196245202217 -86.92093645786507) bank161740 +161741 POINT(42.55446599114218 -86.82744957151786) bank161741 +161742 POINT(42.42349863446858 -87.54284437558034) bank161742 +161743 POINT(40.92210266769767 -87.56834674958944) bank161743 +161744 POINT(41.5588562930518 -86.77871095591455) bank161744 +161745 POINT(42.719069029466674 -88.24625523462974) bank161745 +161746 POINT(42.738434040760296 -87.36704322591697) bank161746 +161747 POINT(41.42135714205588 -86.77165102325178) bank161747 +161748 POINT(41.052457361548925 -88.36171657172919) bank161748 +161749 POINT(41.844135929682885 -88.51101121308677) bank161749 +161750 POINT(41.94161523960959 -87.10369267116334) bank161750 +161751 POINT(41.07797417734099 -87.91211018247562) bank161751 +161752 POINT(42.739601331770245 -87.46348399465023) bank161752 +161753 POINT(41.57193593551749 -86.86991666842118) bank161753 +161754 POINT(41.768949905229746 -87.09971045170239) bank161754 +161755 POINT(42.37143346115799 -87.26110569076887) bank161755 +161756 POINT(40.92569430533376 -88.59680354781243) bank161756 +161757 POINT(42.53819598699226 -87.45125450329198) bank161757 +161758 POINT(42.395441110918085 -87.10599661280088) bank161758 +161759 POINT(41.49515266040506 -86.66543192663269) bank161759 +161760 POINT(41.98515631282816 -88.48870433119225) bank161760 +161761 POINT(41.92086804411972 -87.56665909312593) bank161761 +161762 POINT(42.1945761394592 -87.74592166257199) bank161762 +161763 POINT(41.297304755140786 -88.07993710382911) bank161763 +161764 POINT(42.705387878180524 -86.9744026787799) bank161764 +161765 POINT(41.64275000225934 -87.5426862365448) bank161765 +161766 POINT(42.70868789033704 -86.71919349987796) bank161766 +161767 POINT(41.31870670980592 -87.44871216653594) bank161767 +161768 POINT(42.323232462739924 -88.56897565466478) bank161768 +161769 POINT(41.679348499114035 -87.24987511695014) bank161769 +161770 POINT(41.16606959674412 -86.7783934782114) bank161770 +161771 POINT(41.372738035993194 -86.70059267075968) bank161771 +161772 POINT(41.45126125206023 -87.11326111868014) bank161772 +161773 POINT(42.12894761688419 -87.96408479847105) bank161773 +161774 POINT(42.44092677307234 -87.66571628333422) bank161774 +161775 POINT(41.085088434680046 -88.01735453660535) bank161775 +161776 POINT(42.800327856668325 -88.4282161365759) bank161776 +161777 POINT(41.487358776815974 -87.83969989274786) bank161777 +161778 POINT(41.855819716794976 -87.74225829757) bank161778 +161779 POINT(42.35628913234881 -86.80296722073837) bank161779 +161780 POINT(41.30142100467162 -88.13817005555362) bank161780 +161781 POINT(42.38766782549224 -86.65949112876261) bank161781 +161782 POINT(42.73364339140034 -86.98471735364079) bank161782 +161783 POINT(42.47103568489272 -87.18379659327213) bank161783 +161784 POINT(42.52555808931417 -88.05711422160597) bank161784 +161785 POINT(42.39617837039892 -87.98162545685766) bank161785 +161786 POINT(41.733352911835624 -86.64104614546726) bank161786 +161787 POINT(42.47520931322564 -87.167852775275) bank161787 +161788 POINT(42.80837921680617 -88.14755742104362) bank161788 +161789 POINT(41.41919436323937 -88.56829797593264) bank161789 +161790 POINT(42.40695477174033 -86.71525121984986) bank161790 +161791 POINT(41.16102726044958 -87.43620328275318) bank161791 +161792 POINT(41.37082448083429 -87.97669656001824) bank161792 +161793 POINT(42.390803465951116 -86.96971266718737) bank161793 +161794 POINT(40.94265787005883 -88.56684941489753) bank161794 +161795 POINT(41.73981617864223 -88.61621303486181) bank161795 +161796 POINT(42.56066620428507 -86.69196244478985) bank161796 +161797 POINT(41.646756783604594 -88.01702691156432) bank161797 +161798 POINT(42.52778194348522 -88.20798168590834) bank161798 +161799 POINT(42.16606706334188 -88.0537652942775) bank161799 +161800 POINT(41.976620246703035 -88.03710359278044) bank161800 +161801 POINT(41.45628173401203 -87.97375186993773) bank161801 +161802 POINT(42.014740683019454 -87.62928642605601) bank161802 +161803 POINT(41.11467198000873 -88.58046045697674) bank161803 +161804 POINT(42.66876971605137 -87.36346773767399) bank161804 +161805 POINT(41.490919082252304 -86.81410492602743) bank161805 +161806 POINT(42.39575736502822 -88.18339042717756) bank161806 +161807 POINT(42.01718996627579 -86.9448498921093) bank161807 +161808 POINT(42.251652934038894 -87.2416738180505) bank161808 +161809 POINT(41.188062326870885 -87.28785257526194) bank161809 +161810 POINT(41.94107654647895 -86.8686962036072) bank161810 +161811 POINT(41.737874117406015 -87.34426122858729) bank161811 +161812 POINT(41.938927095406896 -86.79517178762698) bank161812 +161813 POINT(41.404557494374856 -87.15059674870398) bank161813 +161814 POINT(42.38931380520455 -86.83877703723681) bank161814 +161815 POINT(41.41209709549969 -88.6232284710168) bank161815 +161816 POINT(41.05212344758687 -86.69375468452142) bank161816 +161817 POINT(42.63268585562711 -87.52959003564743) bank161817 +161818 POINT(42.71107612634868 -88.32159896038232) bank161818 +161819 POINT(41.8018272685853 -87.57024649184976) bank161819 +161820 POINT(40.99008689704866 -87.55219596193116) bank161820 +161821 POINT(41.637210120583 -86.64834283459122) bank161821 +161822 POINT(40.955307442708175 -88.08827859531182) bank161822 +161823 POINT(41.249950194414694 -88.2778113898277) bank161823 +161824 POINT(42.16530413572537 -87.86110551907534) bank161824 +161825 POINT(42.07692374162187 -87.61508813603245) bank161825 +161826 POINT(41.33001892663384 -88.19802471244195) bank161826 +161827 POINT(42.70425696398918 -87.71466444616726) bank161827 +161828 POINT(42.7362353938658 -86.74102653865687) bank161828 +161829 POINT(41.6125878649483 -88.06084574471555) bank161829 +161830 POINT(42.667983700772915 -88.56300097550626) bank161830 +161831 POINT(41.298789250007445 -88.2541200156436) bank161831 +161832 POINT(41.51313154783663 -87.4651460430585) bank161832 +161833 POINT(41.47901015216512 -88.32723561440058) bank161833 +161834 POINT(41.38384032979643 -87.38386152196014) bank161834 +161835 POINT(41.06272465965281 -88.20379968712334) bank161835 +161836 POINT(42.0497976945602 -88.47063078147741) bank161836 +161837 POINT(42.47862396533621 -88.54377419637245) bank161837 +161838 POINT(42.83668793914029 -86.78878782968229) bank161838 +161839 POINT(41.933867733372864 -86.7826367805673) bank161839 +161840 POINT(41.08488630063025 -86.73747179873166) bank161840 +161841 POINT(41.54831012378228 -87.16561755457569) bank161841 +161842 POINT(42.1472314500473 -87.6954138584284) bank161842 +161843 POINT(42.09790996335771 -88.01924862332486) bank161843 +161844 POINT(41.03104415025139 -87.12919616949097) bank161844 +161845 POINT(42.7038650225956 -87.76965307984044) bank161845 +161846 POINT(40.97739982260863 -87.16498535947571) bank161846 +161847 POINT(41.47647539512049 -87.86174270214396) bank161847 +161848 POINT(41.96442791050288 -88.3553350997333) bank161848 +161849 POINT(41.32753216403369 -87.85752784067776) bank161849 +161850 POINT(42.45803707361085 -87.77519248643827) bank161850 +161851 POINT(42.075655383440996 -88.13203130920905) bank161851 +161852 POINT(42.87793812910082 -86.97814786796903) bank161852 +161853 POINT(41.93664244988809 -88.41667843419793) bank161853 +161854 POINT(41.61417344396277 -88.30260475582443) bank161854 +161855 POINT(41.57090047602494 -86.88666334607875) bank161855 +161856 POINT(42.39545381060452 -87.11053773872459) bank161856 +161857 POINT(41.801489802325754 -88.24618443987515) bank161857 +161858 POINT(41.23357347083583 -88.33528769716769) bank161858 +161859 POINT(42.57513862809087 -88.22978578430991) bank161859 +161860 POINT(42.00343344713739 -86.67043430026524) bank161860 +161861 POINT(42.73608362135745 -86.8726542190532) bank161861 +161862 POINT(41.39135714833171 -87.97306709544524) bank161862 +161863 POINT(41.4990192596088 -88.2759813140185) bank161863 +161864 POINT(41.90888346073444 -87.99064105938572) bank161864 +161865 POINT(41.045563928038156 -87.43923201074222) bank161865 +161866 POINT(41.33877810956992 -86.73036142305355) bank161866 +161867 POINT(42.19228725139025 -87.04011075324664) bank161867 +161868 POINT(41.15772174057478 -87.9161618935338) bank161868 +161869 POINT(41.20104013866422 -86.819283774644) bank161869 +161870 POINT(42.72680368168487 -87.93876052681777) bank161870 +161871 POINT(41.10206026634998 -87.55538650702285) bank161871 +161872 POINT(41.4183247179023 -88.47800770917122) bank161872 +161873 POINT(42.38552050634392 -87.05801733573325) bank161873 +161874 POINT(42.590846039208934 -88.00086329135654) bank161874 +161875 POINT(42.74637594459116 -88.27612147400032) bank161875 +161876 POINT(42.21429935277744 -87.14008820556266) bank161876 +161877 POINT(42.65281017758909 -87.56105185772502) bank161877 +161878 POINT(42.21597041316323 -86.94488857010522) bank161878 +161879 POINT(42.550050791119006 -86.71874899069404) bank161879 +161880 POINT(42.32397458312321 -86.79071083386778) bank161880 +161881 POINT(40.93951048566813 -87.00823913056904) bank161881 +161882 POINT(42.49339409899686 -88.40874730301115) bank161882 +161883 POINT(42.36617649462731 -87.30129004856668) bank161883 +161884 POINT(41.16254851156416 -87.1491088895224) bank161884 +161885 POINT(42.71162475450235 -87.72605933687804) bank161885 +161886 POINT(41.84757960274071 -88.51563614390628) bank161886 +161887 POINT(42.184951176669955 -87.91515491449891) bank161887 +161888 POINT(42.867628643567684 -86.9820342767404) bank161888 +161889 POINT(42.38486884125037 -87.10556528704478) bank161889 +161890 POINT(41.490414261413214 -87.13615691969284) bank161890 +161891 POINT(41.231800313873165 -87.96802301029705) bank161891 +161892 POINT(42.48526691808634 -87.32672726724105) bank161892 +161893 POINT(41.089426416672254 -86.76311766869776) bank161893 +161894 POINT(42.090241232064606 -87.44947952517823) bank161894 +161895 POINT(41.54345463827201 -88.58491091018767) bank161895 +161896 POINT(42.312936183867194 -87.137324429354) bank161896 +161897 POINT(42.84665774548462 -86.79653638806955) bank161897 +161898 POINT(41.10322689103059 -88.28024489064398) bank161898 +161899 POINT(41.249329045260815 -87.73746349737351) bank161899 +161900 POINT(41.32058282655348 -87.32541107728971) bank161900 +161901 POINT(41.271545034978885 -87.54843208496916) bank161901 +161902 POINT(41.875135485296255 -87.42170890090021) bank161902 +161903 POINT(40.94303260307711 -87.24137985386962) bank161903 +161904 POINT(41.6837013001381 -86.64242410363133) bank161904 +161905 POINT(42.572809488594935 -88.16619608361043) bank161905 +161906 POINT(42.83428354323992 -87.56077119909607) bank161906 +161907 POINT(42.406941222942585 -87.1496493872558) bank161907 +161908 POINT(42.354246462102005 -87.36325439078776) bank161908 +161909 POINT(41.53311149306739 -88.12966495951163) bank161909 +161910 POINT(42.54386679126233 -86.81934283485444) bank161910 +161911 POINT(41.623515936199055 -87.37920006570214) bank161911 +161912 POINT(41.49457901694848 -87.23581234620293) bank161912 +161913 POINT(41.8488228331847 -87.43429198896779) bank161913 +161914 POINT(41.71708478837322 -86.67129581040508) bank161914 +161915 POINT(41.25342382509588 -87.83643678003615) bank161915 +161916 POINT(41.88680019194988 -87.25679988056947) bank161916 +161917 POINT(40.97474215511653 -87.58140609959332) bank161917 +161918 POINT(42.35927646465304 -88.34918174636458) bank161918 +161919 POINT(42.50596811925766 -87.08111294980513) bank161919 +161920 POINT(40.983996296376304 -87.85945850993456) bank161920 +161921 POINT(42.132010886008324 -87.7137017275976) bank161921 +161922 POINT(42.80257007036534 -87.54964533903875) bank161922 +161923 POINT(42.15723442535138 -86.77785845009727) bank161923 +161924 POINT(42.78093676121942 -88.35494840974606) bank161924 +161925 POINT(42.05398410201641 -88.18343833159284) bank161925 +161926 POINT(41.65090444998599 -87.88901689872921) bank161926 +161927 POINT(42.86627963362382 -87.36325919214819) bank161927 +161928 POINT(41.913817404072944 -86.95530745874886) bank161928 +161929 POINT(41.01810991876313 -87.96930017434181) bank161929 +161930 POINT(42.35602618413072 -88.52269773548757) bank161930 +161931 POINT(41.55530557084837 -86.75607423298351) bank161931 +161932 POINT(41.44589465459483 -88.31743264045076) bank161932 +161933 POINT(41.72850925602864 -88.11814362649343) bank161933 +161934 POINT(42.02137448898291 -88.41037240118553) bank161934 +161935 POINT(41.77028395604967 -87.58202411887613) bank161935 +161936 POINT(41.720622913523655 -88.10986598565405) bank161936 +161937 POINT(42.41998360119716 -87.66235447814495) bank161937 +161938 POINT(42.55792622490261 -87.46197599095538) bank161938 +161939 POINT(42.064367606801795 -86.8246827800951) bank161939 +161940 POINT(42.69675000083947 -87.37769388252745) bank161940 +161941 POINT(41.97959861954165 -86.89471694276426) bank161941 +161942 POINT(42.857515932863734 -88.3566256038471) bank161942 +161943 POINT(41.667201783185675 -86.72097800017372) bank161943 +161944 POINT(41.23049815473331 -88.49984711562968) bank161944 +161945 POINT(41.767903511876355 -87.33889057394296) bank161945 +161946 POINT(42.50260904615456 -88.13488857339534) bank161946 +161947 POINT(42.221392279315715 -88.62427464029624) bank161947 +161948 POINT(42.105720822832176 -88.31325213991477) bank161948 +161949 POINT(42.42968032890062 -86.86362627230196) bank161949 +161950 POINT(42.43367939410203 -87.65596949937985) bank161950 +161951 POINT(42.72171499406155 -86.75332735190352) bank161951 +161952 POINT(41.232964061293075 -88.29510027764498) bank161952 +161953 POINT(41.06820078087993 -87.00617990764403) bank161953 +161954 POINT(41.45811472353316 -87.09011791444426) bank161954 +161955 POINT(42.70556722420022 -87.63051768992794) bank161955 +161956 POINT(41.66816357109867 -87.90718913318881) bank161956 +161957 POINT(42.41047276848911 -87.38342365223316) bank161957 +161958 POINT(42.76130116792376 -87.32842916367058) bank161958 +161959 POINT(41.7522641292317 -88.04521182466934) bank161959 +161960 POINT(42.5429893865323 -88.4165384752614) bank161960 +161961 POINT(41.39382114777564 -86.87595515545011) bank161961 +161962 POINT(41.014794150645 -86.94778303063562) bank161962 +161963 POINT(42.52767292445527 -87.63024705962808) bank161963 +161964 POINT(42.49136116215324 -88.55060767902565) bank161964 +161965 POINT(41.556027921925505 -86.88833359188475) bank161965 +161966 POINT(41.370315143958244 -87.57968829537597) bank161966 +161967 POINT(41.09652276295976 -87.2589675361856) bank161967 +161968 POINT(41.98563650993164 -88.13478542864944) bank161968 +161969 POINT(42.53647491152224 -88.49452488322811) bank161969 +161970 POINT(41.61348172765089 -88.17084992829537) bank161970 +161971 POINT(41.05614860682952 -87.85981156462053) bank161971 +161972 POINT(42.16349843767615 -88.33291633567391) bank161972 +161973 POINT(41.44965880551595 -87.49006008797552) bank161973 +161974 POINT(42.31032600191272 -87.26830537325523) bank161974 +161975 POINT(42.581145540728265 -87.8430498217872) bank161975 +161976 POINT(41.33477417812129 -88.00079709612696) bank161976 +161977 POINT(41.85588568480879 -88.4150743532633) bank161977 +161978 POINT(41.8214878221572 -87.14699608944395) bank161978 +161979 POINT(41.29385742676876 -87.37945168298933) bank161979 +161980 POINT(42.152488845243894 -87.79632256763003) bank161980 +161981 POINT(41.420991579800074 -87.09991552958924) bank161981 +161982 POINT(42.05791814024247 -87.21322285764808) bank161982 +161983 POINT(41.386112766529955 -87.90284810570512) bank161983 +161984 POINT(42.84452205310751 -86.8156295062002) bank161984 +161985 POINT(41.14839489690253 -87.75610170295846) bank161985 +161986 POINT(42.40908871340765 -87.06713758780539) bank161986 +161987 POINT(41.61355315087002 -88.19310830221917) bank161987 +161988 POINT(41.88764981994741 -87.26898964821689) bank161988 +161989 POINT(41.00780286404536 -88.52883837961369) bank161989 +161990 POINT(40.87961306945819 -88.22395122206042) bank161990 +161991 POINT(41.186944695995166 -88.13109990894054) bank161991 +161992 POINT(41.86880526227663 -86.7969659473933) bank161992 +161993 POINT(41.32204274307609 -87.87703581875489) bank161993 +161994 POINT(41.73358699428746 -87.15388163197083) bank161994 +161995 POINT(41.90699354842364 -87.99420651341723) bank161995 +161996 POINT(41.97976447262109 -87.0971428664579) bank161996 +161997 POINT(42.30301262401362 -87.99801253680104) bank161997 +161998 POINT(41.804512606659344 -88.49468340998129) bank161998 +161999 POINT(41.67410424405012 -87.45892428184037) bank161999 +162000 POINT(42.592151110567244 -88.16579056906279) bank162000 +162001 POINT(42.63444792689106 -88.24701102628103) bank162001 +162002 POINT(41.80935844738681 -88.47037903821497) bank162002 +162003 POINT(41.54691344539412 -87.02865224690258) bank162003 +162004 POINT(41.976126857311385 -87.23793128451493) bank162004 +162005 POINT(40.93328912111931 -87.9499304113251) bank162005 +162006 POINT(42.181069460629175 -88.08005771221838) bank162006 +162007 POINT(41.1798852040622 -86.83623112044347) bank162007 +162008 POINT(41.29849753913673 -88.34547710971722) bank162008 +162009 POINT(41.3932101722393 -87.14760612796304) bank162009 +162010 POINT(41.63701099244431 -87.96783210988269) bank162010 +162011 POINT(41.559852650468834 -88.55501083630206) bank162011 +162012 POINT(41.17620743166999 -87.00623792888393) bank162012 +162013 POINT(41.60737544176205 -86.91268561439789) bank162013 +162014 POINT(42.57337201976766 -87.93557706375324) bank162014 +162015 POINT(40.92423235029807 -87.69485547961358) bank162015 +162016 POINT(41.90407505562536 -86.8093184874362) bank162016 +162017 POINT(42.742617597551785 -88.39513502825653) bank162017 +162018 POINT(42.2556476977129 -86.83654484968287) bank162018 +162019 POINT(42.17599348189775 -86.72137419067914) bank162019 +162020 POINT(41.247948045207586 -86.73004817680659) bank162020 +162021 POINT(41.79766200987319 -88.24897782633364) bank162021 +162022 POINT(42.30277315020703 -87.61292516476243) bank162022 +162023 POINT(41.41918564141799 -88.50773678156902) bank162023 +162024 POINT(41.22134622347313 -86.9743057604263) bank162024 +162025 POINT(42.32062812486676 -87.34017131295721) bank162025 +162026 POINT(42.570674320731385 -87.968161693448) bank162026 +162027 POINT(41.15797177418382 -87.90510969258268) bank162027 +162028 POINT(41.75274411895214 -88.51501949786494) bank162028 +162029 POINT(41.50139203784755 -86.69390133880765) bank162029 +162030 POINT(42.548794504902226 -87.11258661932648) bank162030 +162031 POINT(41.340672242953346 -87.63370053630672) bank162031 +162032 POINT(41.286604833781126 -86.74950485251435) bank162032 +162033 POINT(42.009364543207674 -86.7792503187665) bank162033 +162034 POINT(42.648180652419974 -87.7325417288126) bank162034 +162035 POINT(42.585829373239136 -87.97934349819572) bank162035 +162036 POINT(41.53224191779772 -88.12170430246964) bank162036 +162037 POINT(42.469963004391694 -88.26828350507868) bank162037 +162038 POINT(41.735611786043116 -86.87980365089624) bank162038 +162039 POINT(41.40164189526896 -88.28905973187682) bank162039 +162040 POINT(41.01098422082302 -87.57907654810292) bank162040 +162041 POINT(42.08237422399607 -87.41809520842276) bank162041 +162042 POINT(42.69197677781214 -87.3287129568232) bank162042 +162043 POINT(41.361362831768176 -87.46979769332859) bank162043 +162044 POINT(40.9161595421979 -88.17062084501474) bank162044 +162045 POINT(41.49697951707255 -87.18654306749787) bank162045 +162046 POINT(40.880506279083924 -87.54934984131474) bank162046 +162047 POINT(41.68915513897625 -87.98174554590611) bank162047 +162048 POINT(42.3068549957263 -87.99783708117309) bank162048 +162049 POINT(41.56144768544348 -86.90908966965812) bank162049 +162050 POINT(42.50117564074217 -88.62794600720113) bank162050 +162051 POINT(41.09343898505664 -87.49013155275912) bank162051 +162052 POINT(42.42589975175168 -87.07169435735572) bank162052 +162053 POINT(42.342959154567716 -87.26438517267196) bank162053 +162054 POINT(41.575181309607714 -86.73695472468107) bank162054 +162055 POINT(41.14688818678548 -88.02831663175292) bank162055 +162056 POINT(42.72553761595876 -87.75862394562411) bank162056 +162057 POINT(42.40095662538612 -87.8175993820439) bank162057 +162058 POINT(42.79001770519703 -86.72335283274596) bank162058 +162059 POINT(41.99217390473159 -87.86518711757482) bank162059 +162060 POINT(42.530278736052196 -87.33511446142592) bank162060 +162061 POINT(41.47015906458665 -86.92034561474424) bank162061 +162062 POINT(41.662782227368986 -87.94254228375138) bank162062 +162063 POINT(42.516372922649104 -87.26303072977875) bank162063 +162064 POINT(41.94684012109564 -88.27639585231925) bank162064 +162065 POINT(42.585665274104244 -86.75978643088425) bank162065 +162066 POINT(42.771963794701705 -87.54357491067246) bank162066 +162067 POINT(40.87997359879591 -86.69035468540268) bank162067 +162068 POINT(41.05159798122946 -86.93128357447273) bank162068 +162069 POINT(42.44074160429852 -88.43265299774284) bank162069 +162070 POINT(41.128251621895366 -87.30402757237661) bank162070 +162071 POINT(42.636378533550314 -88.62367915750298) bank162071 +162072 POINT(41.250244065212584 -88.29748411537568) bank162072 +162073 POINT(42.508133885217646 -88.2142826518972) bank162073 +162074 POINT(41.87199977406467 -86.92664807120521) bank162074 +162075 POINT(41.46421738217783 -88.16993689761343) bank162075 +162076 POINT(42.40772862414329 -86.85647710295898) bank162076 +162077 POINT(42.54907495615065 -87.65498965338064) bank162077 +162078 POINT(42.00746951215727 -86.98567844010833) bank162078 +162079 POINT(42.41576332565803 -87.48310171269748) bank162079 +162080 POINT(42.13433476408785 -88.44530470615058) bank162080 +162081 POINT(41.1650594988125 -87.29036483106289) bank162081 +162082 POINT(42.102967029852834 -86.87923086581331) bank162082 +162083 POINT(42.21397176463619 -88.5348007980829) bank162083 +162084 POINT(40.96074278210332 -88.52815590343945) bank162084 +162085 POINT(41.59863179754089 -87.31090646917298) bank162085 +162086 POINT(42.64434924725096 -87.51331929941115) bank162086 +162087 POINT(42.038730296828575 -87.08643240037993) bank162087 +162088 POINT(41.381547452469405 -87.53834755029183) bank162088 +162089 POINT(42.23158010441963 -87.7908288104852) bank162089 +162090 POINT(41.410917076301956 -87.25238159580276) bank162090 +162091 POINT(41.74145018382521 -87.97250517117989) bank162091 +162092 POINT(41.29149478259071 -87.61932886094847) bank162092 +162093 POINT(42.67189885677658 -88.3317020596364) bank162093 +162094 POINT(41.75406002949242 -87.22881085142264) bank162094 +162095 POINT(41.577015660782095 -87.59230271151664) bank162095 +162096 POINT(41.80356476705851 -88.27667951361616) bank162096 +162097 POINT(41.39230744242099 -87.12493134899307) bank162097 +162098 POINT(41.624247178911205 -86.79276539185823) bank162098 +162099 POINT(41.4503817014153 -87.65939230754667) bank162099 +162100 POINT(41.098677410642495 -86.96299809677696) bank162100 +162101 POINT(41.70790475299017 -88.04148368041697) bank162101 +162102 POINT(41.89993391370981 -87.16008569191057) bank162102 +162103 POINT(42.463099893574764 -88.5537669886701) bank162103 +162104 POINT(41.082518628688135 -86.81881659950787) bank162104 +162105 POINT(42.40264161281564 -87.90299631743686) bank162105 +162106 POINT(40.91723486989262 -87.52198727998544) bank162106 +162107 POINT(40.994380180252655 -88.29537211741223) bank162107 +162108 POINT(42.59639070881786 -87.6170030607587) bank162108 +162109 POINT(42.5756973342082 -87.50324552082563) bank162109 +162110 POINT(41.65229681953367 -88.11231547886476) bank162110 +162111 POINT(41.76246971691998 -87.4679874844411) bank162111 +162112 POINT(41.449955284054916 -88.05402155642747) bank162112 +162113 POINT(40.91124104582059 -88.36660352764372) bank162113 +162114 POINT(42.057586288077125 -86.84307465531823) bank162114 +162115 POINT(41.45244860091528 -87.47913644465966) bank162115 +162116 POINT(42.33150113343576 -87.25299952957002) bank162116 +162117 POINT(41.707404031445726 -86.64125848807792) bank162117 +162118 POINT(42.628623885698325 -87.88052131332647) bank162118 +162119 POINT(42.28069605778792 -87.46223523394512) bank162119 +162120 POINT(42.41075533140133 -86.94545987096576) bank162120 +162121 POINT(42.080898990632036 -86.86705875270866) bank162121 +162122 POINT(42.49959778263293 -88.14612941845435) bank162122 +162123 POINT(40.981679750198914 -88.52143539802628) bank162123 +162124 POINT(41.75832965766751 -86.9624140615748) bank162124 +162125 POINT(42.564169607872486 -88.17103155508687) bank162125 +162126 POINT(42.809487431071624 -86.70461260047912) bank162126 +162127 POINT(42.4375244717302 -86.94489450006175) bank162127 +162128 POINT(41.22602393637007 -88.06834075609784) bank162128 +162129 POINT(42.26234294057802 -88.53543285457096) bank162129 +162130 POINT(42.28828214082996 -88.36766127279101) bank162130 +162131 POINT(41.9170606483074 -87.49045403484189) bank162131 +162132 POINT(41.283655475527645 -88.41883704071748) bank162132 +162133 POINT(41.17301353066885 -87.4774048529383) bank162133 +162134 POINT(41.6313668929708 -87.11397709948612) bank162134 +162135 POINT(42.663279379020366 -88.34425961653247) bank162135 +162136 POINT(42.60088753680866 -88.56185819715206) bank162136 +162137 POINT(42.05908239995661 -88.20201526715688) bank162137 +162138 POINT(42.23374280946304 -87.78043169151464) bank162138 +162139 POINT(41.81910091594297 -87.39942273880196) bank162139 +162140 POINT(41.51515646036215 -88.00555026145777) bank162140 +162141 POINT(41.298804564613434 -87.44085711516762) bank162141 +162142 POINT(41.66532537502606 -87.40717633635373) bank162142 +162143 POINT(42.066983642235606 -86.99724267131948) bank162143 +162144 POINT(41.05969170086988 -87.02269601913034) bank162144 +162145 POINT(42.42735681316344 -86.90638653969914) bank162145 +162146 POINT(42.09659047547424 -88.26066459770524) bank162146 +162147 POINT(42.014437449162365 -86.89323944943567) bank162147 +162148 POINT(40.906160046901775 -86.80768563584442) bank162148 +162149 POINT(41.66054854372951 -88.5606858573669) bank162149 +162150 POINT(42.44167475651064 -87.49356245487712) bank162150 +162151 POINT(42.57911760912848 -87.70050854664596) bank162151 +162152 POINT(42.27528726740906 -88.34169130210739) bank162152 +162153 POINT(41.94656116269609 -87.29633072106255) bank162153 +162154 POINT(41.79936052964942 -87.63170035242084) bank162154 +162155 POINT(41.5917380533074 -87.75868941185189) bank162155 +162156 POINT(41.53939662073665 -87.50112697189627) bank162156 +162157 POINT(42.08219255837526 -87.39302492511348) bank162157 +162158 POINT(41.460068007141345 -88.44069673617028) bank162158 +162159 POINT(41.226235439845915 -86.71263124184453) bank162159 +162160 POINT(42.139056506761506 -87.53262794084365) bank162160 +162161 POINT(41.03085169871786 -88.07778257658532) bank162161 +162162 POINT(41.98393463031989 -88.5366638790368) bank162162 +162163 POINT(42.28187253131797 -88.35690590553926) bank162163 +162164 POINT(42.79115230674704 -88.29689906636382) bank162164 +162165 POINT(41.97268289406148 -87.19118113357568) bank162165 +162166 POINT(40.94618676818955 -88.29285182642174) bank162166 +162167 POINT(41.81667872327304 -88.19481627751033) bank162167 +162168 POINT(42.18980990249535 -88.41196421989329) bank162168 +162169 POINT(42.25576417620593 -87.37711880131603) bank162169 +162170 POINT(42.798325334150356 -87.13976018419979) bank162170 +162171 POINT(41.95808441774105 -87.97008689447021) bank162171 +162172 POINT(42.02997825057421 -87.42550169848545) bank162172 +162173 POINT(41.509982421247074 -86.87707264090855) bank162173 +162174 POINT(42.155985276534 -87.47284095918978) bank162174 +162175 POINT(40.894121567556994 -87.14718904507686) bank162175 +162176 POINT(41.19886991112923 -88.39629168525678) bank162176 +162177 POINT(41.46000301245751 -88.12261493251606) bank162177 +162178 POINT(41.96510580657366 -87.25013347506118) bank162178 +162179 POINT(42.5579929295739 -87.7470028400411) bank162179 +162180 POINT(42.092430041040885 -87.6299859132049) bank162180 +162181 POINT(42.730871726459114 -87.80271404918844) bank162181 +162182 POINT(41.384653120533216 -88.22864042016943) bank162182 +162183 POINT(41.23140455742167 -87.47242083293322) bank162183 +162184 POINT(41.570457651900654 -87.05464974768708) bank162184 +162185 POINT(41.32804672053656 -86.91092737966567) bank162185 +162186 POINT(41.356958467642535 -88.21576118405413) bank162186 +162187 POINT(41.49176240200347 -86.88730533740103) bank162187 +162188 POINT(41.80001103436081 -88.23385687359534) bank162188 +162189 POINT(41.58194621578766 -87.57169251056249) bank162189 +162190 POINT(41.679499629675284 -87.59139277884029) bank162190 +162191 POINT(42.661768473211154 -88.19758428778573) bank162191 +162192 POINT(41.47361848363215 -86.93640560189415) bank162192 +162193 POINT(42.47446133090958 -86.78217785094444) bank162193 +162194 POINT(42.44165970280073 -87.40942066198014) bank162194 +162195 POINT(42.22795193353949 -88.55863627347492) bank162195 +162196 POINT(41.863424171468175 -88.08337901234636) bank162196 +162197 POINT(42.07307639993852 -86.7316731605847) bank162197 +162198 POINT(40.95343520195771 -86.7424486761597) bank162198 +162199 POINT(42.45226077488169 -87.65842718250516) bank162199 +162200 POINT(41.02140602935052 -87.0904193137617) bank162200 +162201 POINT(42.32021720341646 -86.647903926363) bank162201 +162202 POINT(41.94158625580356 -88.04474950646419) bank162202 +162203 POINT(41.1728611178257 -88.21492191775181) bank162203 +162204 POINT(41.68349054845367 -88.49325954783072) bank162204 +162205 POINT(40.993730429081936 -87.67684868529804) bank162205 +162206 POINT(41.28336852145257 -88.20899049450061) bank162206 +162207 POINT(42.82398562142483 -88.59346627908016) bank162207 +162208 POINT(41.22731134465503 -86.97171759106493) bank162208 +162209 POINT(41.6204424786483 -88.08802083638444) bank162209 +162210 POINT(42.77124220373334 -87.76094100298981) bank162210 +162211 POINT(41.14508922420804 -88.48219997172924) bank162211 +162212 POINT(40.98931385335566 -87.42961237221654) bank162212 +162213 POINT(42.08087395584094 -87.36906565789448) bank162213 +162214 POINT(41.525099216632945 -86.71173933194001) bank162214 +162215 POINT(41.866001105533286 -87.19465486024474) bank162215 +162216 POINT(42.8166714719665 -87.60365871562638) bank162216 +162217 POINT(42.82155114911841 -87.77165664399391) bank162217 +162218 POINT(42.81487455473962 -86.7498272377932) bank162218 +162219 POINT(41.80688240978828 -86.91840822985039) bank162219 +162220 POINT(41.357201930761384 -87.6430356967888) bank162220 +162221 POINT(42.23165781908143 -87.173080901944) bank162221 +162222 POINT(41.936460427299295 -87.99729956469717) bank162222 +162223 POINT(42.43317948400076 -88.20336759312725) bank162223 +162224 POINT(42.524860234459815 -87.9468580937624) bank162224 +162225 POINT(41.44475574734524 -87.709969752381) bank162225 +162226 POINT(41.21999805814613 -87.4403559162632) bank162226 +162227 POINT(42.83016893232044 -87.96508632045055) bank162227 +162228 POINT(41.67710495409433 -87.25221940965207) bank162228 +162229 POINT(42.195425552916205 -87.87748225992904) bank162229 +162230 POINT(40.91841372747711 -87.42924821677454) bank162230 +162231 POINT(42.65024088962984 -87.2450893129414) bank162231 +162232 POINT(40.92155508472242 -88.40786907597337) bank162232 +162233 POINT(41.96369364313528 -88.13810404482969) bank162233 +162234 POINT(41.8351886125608 -88.0454915660429) bank162234 +162235 POINT(42.224942229194035 -87.76852953013703) bank162235 +162236 POINT(41.849427652399804 -87.73561808912368) bank162236 +162237 POINT(41.81532932004144 -86.89230673350175) bank162237 +162238 POINT(42.134198831460196 -87.45596944772447) bank162238 +162239 POINT(41.566790022851144 -87.13425639715177) bank162239 +162240 POINT(42.8239680826694 -87.26715018545451) bank162240 +162241 POINT(40.90480310661768 -86.73898984705649) bank162241 +162242 POINT(41.59630852567177 -87.8212049135318) bank162242 +162243 POINT(42.56895958923536 -87.13507143546364) bank162243 +162244 POINT(41.54796025684825 -87.45366197248981) bank162244 +162245 POINT(42.69965287092521 -88.33260984046257) bank162245 +162246 POINT(42.00399908496104 -87.6566210542279) bank162246 +162247 POINT(42.058040288131046 -88.32033185834204) bank162247 +162248 POINT(41.65393692705697 -86.88508361980614) bank162248 +162249 POINT(41.653201447917745 -87.68803409935062) bank162249 +162250 POINT(41.507136941676976 -87.0817270145879) bank162250 +162251 POINT(41.174516827705965 -87.85425097731594) bank162251 +162252 POINT(41.56981998814438 -87.84454442159571) bank162252 +162253 POINT(41.23136281989105 -88.6163206502481) bank162253 +162254 POINT(41.07123508366338 -87.66399712685791) bank162254 +162255 POINT(41.958090371518594 -87.36544921055453) bank162255 +162256 POINT(41.78757810852684 -88.4587732175602) bank162256 +162257 POINT(42.76719310203018 -88.0560991524113) bank162257 +162258 POINT(42.4839127717845 -87.59797740157214) bank162258 +162259 POINT(41.25325665604699 -87.92200944388722) bank162259 +162260 POINT(41.11168773003346 -86.88426134579409) bank162260 +162261 POINT(42.72830849383927 -87.04642244993217) bank162261 +162262 POINT(41.72399369672346 -87.97776344335004) bank162262 +162263 POINT(41.15122261118877 -86.67865391382298) bank162263 +162264 POINT(41.642138108247835 -87.92963920371086) bank162264 +162265 POINT(42.477750793442596 -87.84311510482517) bank162265 +162266 POINT(41.27381074488835 -87.75814479518898) bank162266 +162267 POINT(41.51918828222973 -87.46225619318763) bank162267 +162268 POINT(42.27736716381327 -86.75079010745633) bank162268 +162269 POINT(41.97985979701157 -87.18037160781952) bank162269 +162270 POINT(41.25289904979101 -88.22265771120534) bank162270 +162271 POINT(42.38570083379515 -87.28449771556015) bank162271 +162272 POINT(41.76517131022603 -88.43025584810903) bank162272 +162273 POINT(42.08830426745351 -86.77644269028404) bank162273 +162274 POINT(42.18305406834611 -88.35591857545396) bank162274 +162275 POINT(42.408126674196 -86.68252258291149) bank162275 +162276 POINT(42.38959653643165 -87.8848286177925) bank162276 +162277 POINT(41.41656464488907 -87.26666147921595) bank162277 +162278 POINT(41.540468321794506 -87.96639492792724) bank162278 +162279 POINT(40.88052754492162 -86.78619289288808) bank162279 +162280 POINT(41.22731653060902 -87.00202793233413) bank162280 +162281 POINT(41.51005473811605 -87.97795812769905) bank162281 +162282 POINT(41.86596781840959 -88.06488197190697) bank162282 +162283 POINT(42.66436771415616 -87.76346183390574) bank162283 +162284 POINT(40.9079578632704 -88.38605802160905) bank162284 +162285 POINT(40.971850435795126 -87.30966475557572) bank162285 +162286 POINT(42.306886396362906 -88.55876980794658) bank162286 +162287 POINT(41.55193503408735 -86.82186536187348) bank162287 +162288 POINT(41.61056825502589 -87.11434121817769) bank162288 +162289 POINT(42.81131193259696 -87.05585812831816) bank162289 +162290 POINT(42.17892755359968 -87.59106207992252) bank162290 +162291 POINT(40.96914590557339 -88.31094728226292) bank162291 +162292 POINT(41.6007210519697 -87.54316437575883) bank162292 +162293 POINT(42.40015069788739 -88.590790712713) bank162293 +162294 POINT(40.98087518915679 -86.75649590849166) bank162294 +162295 POINT(42.33680808138327 -87.23096069942719) bank162295 +162296 POINT(42.70790060410958 -87.4351525813744) bank162296 +162297 POINT(41.09393541449455 -87.37856982658019) bank162297 +162298 POINT(41.08179531654776 -87.45375545289595) bank162298 +162299 POINT(42.76812867494157 -87.48142729793798) bank162299 +162300 POINT(42.292226926372 -88.18613559987782) bank162300 +162301 POINT(41.91673004587783 -87.19793948378815) bank162301 +162302 POINT(42.32096493371326 -87.29830054829633) bank162302 +162303 POINT(42.76918460704012 -87.36405656187326) bank162303 +162304 POINT(41.39157179053294 -87.81396303960189) bank162304 +162305 POINT(41.778655425358615 -86.77224285275108) bank162305 +162306 POINT(41.09841886907567 -88.26090916546332) bank162306 +162307 POINT(40.89114163518196 -87.28764483786176) bank162307 +162308 POINT(41.845449550610134 -88.46637948925968) bank162308 +162309 POINT(41.20009839436803 -87.61238962612418) bank162309 +162310 POINT(41.90816309319906 -88.21491869022614) bank162310 +162311 POINT(41.5235600481152 -87.70445306058765) bank162311 +162312 POINT(42.40982510758115 -87.22295893981465) bank162312 +162313 POINT(41.65700411155369 -86.96061294048073) bank162313 +162314 POINT(40.99135682469987 -88.3248221991811) bank162314 +162315 POINT(42.47291417373268 -88.44083864236025) bank162315 +162316 POINT(41.749868767231895 -86.92945871401882) bank162316 +162317 POINT(42.43724142063523 -88.34143434085918) bank162317 +162318 POINT(42.53982648832872 -86.82581530745495) bank162318 +162319 POINT(41.2311178587245 -87.60342146540873) bank162319 +162320 POINT(42.62784238178432 -88.39088590097649) bank162320 +162321 POINT(42.78329224533709 -86.72965471385302) bank162321 +162322 POINT(41.30559437799022 -88.2285784415627) bank162322 +162323 POINT(41.73572732011261 -88.40643170714975) bank162323 +162324 POINT(41.27890705224023 -87.58176099887122) bank162324 +162325 POINT(41.63733971288537 -87.77367031142828) bank162325 +162326 POINT(41.25609077629232 -88.61894610498278) bank162326 +162327 POINT(41.61917693341529 -88.57363664964379) bank162327 +162328 POINT(42.658012276751236 -88.31604371751912) bank162328 +162329 POINT(42.189371340369284 -86.9022667638809) bank162329 +162330 POINT(41.84292682258846 -87.76676483170931) bank162330 +162331 POINT(42.0262335167418 -88.46871204896328) bank162331 +162332 POINT(42.74269102130639 -88.4383856142074) bank162332 +162333 POINT(42.77167251093618 -87.4207478075613) bank162333 +162334 POINT(41.06864924047291 -87.7170955260238) bank162334 +162335 POINT(41.6101557601384 -87.32586475264438) bank162335 +162336 POINT(41.499202983614005 -88.37899116031812) bank162336 +162337 POINT(41.15968485696678 -87.99487652503382) bank162337 +162338 POINT(42.20795420888326 -87.46637917990283) bank162338 +162339 POINT(42.443824498552516 -88.13512476408376) bank162339 +162340 POINT(42.32714094499559 -87.89556944952572) bank162340 +162341 POINT(42.46953176981122 -87.4545716523434) bank162341 +162342 POINT(41.73879297628217 -87.19340139434298) bank162342 +162343 POINT(41.718238166769076 -87.19948393695499) bank162343 +162344 POINT(41.73551388333481 -88.06100129642782) bank162344 +162345 POINT(42.061833094533675 -87.9598970987874) bank162345 +162346 POINT(40.9591297677657 -87.6990392950818) bank162346 +162347 POINT(40.93076244512306 -87.67066070941239) bank162347 +162348 POINT(41.40237801967456 -88.14802679999801) bank162348 +162349 POINT(42.241684974619794 -87.87513500822291) bank162349 +162350 POINT(41.323668848129145 -87.73856023154764) bank162350 +162351 POINT(41.57825253736999 -86.85928324215396) bank162351 +162352 POINT(41.24993009507035 -86.938891388314) bank162352 +162353 POINT(41.42293922899099 -87.23465306420871) bank162353 +162354 POINT(42.86900793560343 -88.52805578198061) bank162354 +162355 POINT(41.24397869756923 -87.40187538217097) bank162355 +162356 POINT(42.62171267965193 -88.45032005133523) bank162356 +162357 POINT(41.84418941792186 -88.11048926057218) bank162357 +162358 POINT(40.89606782224215 -86.66332879135776) bank162358 +162359 POINT(42.19721965664401 -87.34845283267002) bank162359 +162360 POINT(40.915573905277206 -87.3295700809144) bank162360 +162361 POINT(41.024133901887666 -88.29524525831303) bank162361 +162362 POINT(41.80214387082519 -88.59210237161548) bank162362 +162363 POINT(40.94380207925909 -88.24290261813259) bank162363 +162364 POINT(41.29036698815019 -88.20768749909428) bank162364 +162365 POINT(41.42200855395907 -88.16760633936897) bank162365 +162366 POINT(41.89926616191349 -87.64830686683678) bank162366 +162367 POINT(40.90116788574795 -87.71219657977544) bank162367 +162368 POINT(42.3481527119813 -86.76867141130357) bank162368 +162369 POINT(41.56018441142712 -87.84794867496807) bank162369 +162370 POINT(42.586359703023845 -88.10028938358356) bank162370 +162371 POINT(42.828015463210654 -86.73444571007798) bank162371 +162372 POINT(41.37481737076514 -87.30277512746406) bank162372 +162373 POINT(42.295560152946585 -87.60785518674652) bank162373 +162374 POINT(41.97021855240404 -88.54790716928224) bank162374 +162375 POINT(42.846661315142 -87.5415331810697) bank162375 +162376 POINT(42.39804689277564 -87.73633963538606) bank162376 +162377 POINT(41.030221890814666 -87.70834903754489) bank162377 +162378 POINT(42.22960620518107 -88.15486400321882) bank162378 +162379 POINT(41.20112562868908 -88.3926507472682) bank162379 +162380 POINT(42.659501483857305 -86.73503429853682) bank162380 +162381 POINT(42.14423898716722 -86.77188847581932) bank162381 +162382 POINT(41.72193244613962 -86.88010704412412) bank162382 +162383 POINT(42.11557986728115 -87.58695776771351) bank162383 +162384 POINT(42.36336205568045 -88.0960141493058) bank162384 +162385 POINT(41.414120893769045 -86.72694559142376) bank162385 +162386 POINT(41.79572770240707 -88.07233855054078) bank162386 +162387 POINT(41.71120045503925 -88.61498438734911) bank162387 +162388 POINT(41.92082875434513 -87.47455576709001) bank162388 +162389 POINT(42.15090707015648 -88.05129066036453) bank162389 +162390 POINT(42.77764177599529 -88.31455430152909) bank162390 +162391 POINT(42.11623128817205 -87.48770229726114) bank162391 +162392 POINT(42.17517054681339 -87.18905893391437) bank162392 +162393 POINT(42.59613878481419 -86.78456089230588) bank162393 +162394 POINT(41.84674346080718 -88.01064979779387) bank162394 +162395 POINT(41.58955574126062 -88.59039020344831) bank162395 +162396 POINT(41.513792103795964 -87.73307086170767) bank162396 +162397 POINT(42.32040578290011 -86.65726309319457) bank162397 +162398 POINT(41.638272817712796 -87.04772012241581) bank162398 +162399 POINT(42.12486434175702 -86.692952452176) bank162399 +162400 POINT(41.8022695651724 -86.67242644719032) bank162400 +162401 POINT(41.653295106195216 -87.10091669244878) bank162401 +162402 POINT(42.01294259075007 -87.88869437959369) bank162402 +162403 POINT(41.09344787427526 -88.04837511187019) bank162403 +162404 POINT(42.219080844258066 -88.08443769987238) bank162404 +162405 POINT(42.71511421769896 -86.77996526140653) bank162405 +162406 POINT(42.347318656565186 -88.60953936610073) bank162406 +162407 POINT(42.14840210690228 -88.40563763168487) bank162407 +162408 POINT(41.61030336071399 -87.31374267979943) bank162408 +162409 POINT(41.5453970543146 -87.36586099462627) bank162409 +162410 POINT(42.49212270216788 -87.34877485250857) bank162410 +162411 POINT(41.53682121081305 -88.2968647506935) bank162411 +162412 POINT(41.251825742575775 -86.77677463909242) bank162412 +162413 POINT(41.39549906692545 -88.369670394547) bank162413 +162414 POINT(42.37604468151311 -88.0385379412191) bank162414 +162415 POINT(41.94895165493815 -87.71030051208737) bank162415 +162416 POINT(41.001078136513655 -87.98212159911992) bank162416 +162417 POINT(40.89299119365086 -87.67066620409165) bank162417 +162418 POINT(40.92462178953781 -88.43754909286974) bank162418 +162419 POINT(41.933995417883324 -87.10617550686936) bank162419 +162420 POINT(41.050138082886846 -88.1601532158456) bank162420 +162421 POINT(41.91978272753351 -87.09008659546087) bank162421 +162422 POINT(42.77666469732586 -88.34590475371995) bank162422 +162423 POINT(42.85205678874492 -88.37508180009092) bank162423 +162424 POINT(42.81355379188257 -88.39224084369812) bank162424 +162425 POINT(41.781491164299574 -86.8640050817493) bank162425 +162426 POINT(41.678209445298414 -86.99832997052138) bank162426 +162427 POINT(40.89808854568485 -88.48351575679449) bank162427 +162428 POINT(41.94077642302686 -87.00365909259615) bank162428 +162429 POINT(42.252687160686726 -87.18413755946378) bank162429 +162430 POINT(42.36294086168927 -86.84621737543833) bank162430 +162431 POINT(42.13109521725845 -87.9079825660597) bank162431 +162432 POINT(42.22636587342225 -86.8429646708729) bank162432 +162433 POINT(41.50238601882088 -86.72234363452692) bank162433 +162434 POINT(41.086559300237155 -86.7938165978712) bank162434 +162435 POINT(42.00954975242963 -88.43744993980346) bank162435 +162436 POINT(42.41707853190964 -87.24312620072246) bank162436 +162437 POINT(42.71531234015455 -87.34792015571205) bank162437 +162438 POINT(41.16490212692979 -88.55782205840887) bank162438 +162439 POINT(42.68318313429805 -87.22869757203856) bank162439 +162440 POINT(42.1636969456897 -86.77697524608018) bank162440 +162441 POINT(42.11972817900256 -87.25051822640948) bank162441 +162442 POINT(40.993284515232496 -88.59565280893119) bank162442 +162443 POINT(42.49024026696584 -87.84198301952769) bank162443 +162444 POINT(41.552126507917556 -86.98005093115708) bank162444 +162445 POINT(42.034054809254364 -86.94890006344285) bank162445 +162446 POINT(42.871405377468434 -88.59776877282003) bank162446 +162447 POINT(40.93806341901049 -87.63431799869964) bank162447 +162448 POINT(41.512561334653896 -87.42596647408669) bank162448 +162449 POINT(42.87430487654256 -87.0175021798598) bank162449 +162450 POINT(41.39296503831094 -87.40705550393228) bank162450 +162451 POINT(41.585801014471286 -88.43257798457836) bank162451 +162452 POINT(42.56837353043241 -87.64560868594164) bank162452 +162453 POINT(41.41057379767952 -87.22490547930703) bank162453 +162454 POINT(42.605420342139794 -88.17028164725645) bank162454 +162455 POINT(41.39891271489298 -87.70517848018835) bank162455 +162456 POINT(41.65807473578203 -87.46712298494728) bank162456 +162457 POINT(42.221190394928385 -88.49596638190081) bank162457 +162458 POINT(42.73237806185319 -86.90130870691358) bank162458 +162459 POINT(41.414919759915946 -88.14751844373696) bank162459 +162460 POINT(41.33545732002473 -87.7929258882121) bank162460 +162461 POINT(42.436305655922354 -86.81718897974999) bank162461 +162462 POINT(41.81599666496653 -87.43210418184565) bank162462 +162463 POINT(41.32491562594216 -87.07504171101228) bank162463 +162464 POINT(41.249034302644034 -87.41814000569477) bank162464 +162465 POINT(40.89140999539683 -88.62209162784448) bank162465 +162466 POINT(41.11705016788809 -87.27305872417406) bank162466 +162467 POINT(41.375704265952045 -86.79888655375613) bank162467 +162468 POINT(42.86879377087387 -87.2726432595878) bank162468 +162469 POINT(41.25216686285928 -87.24018652347799) bank162469 +162470 POINT(41.284014734109775 -88.35897970280682) bank162470 +162471 POINT(41.64395973756353 -86.84577629496042) bank162471 +162472 POINT(41.07980530101549 -87.70960211465115) bank162472 +162473 POINT(40.958923662853586 -86.90351218642586) bank162473 +162474 POINT(41.5879891242469 -87.43198706653322) bank162474 +162475 POINT(41.20635457804439 -87.44361760270192) bank162475 +162476 POINT(42.08393058670717 -88.30021297831856) bank162476 +162477 POINT(41.80669478947525 -88.46737791310933) bank162477 +162478 POINT(41.51731540850585 -88.06641215682605) bank162478 +162479 POINT(41.77986700092138 -86.78642328901775) bank162479 +162480 POINT(42.1901647054691 -88.02046466047315) bank162480 +162481 POINT(42.17316721758297 -87.33664024629927) bank162481 +162482 POINT(42.34917698867946 -88.28409870135862) bank162482 +162483 POINT(42.10275597929311 -87.10177832632243) bank162483 +162484 POINT(41.27094132053599 -86.68865464522018) bank162484 +162485 POINT(41.52109073550578 -87.51704027216334) bank162485 +162486 POINT(41.35341462816535 -87.6337427429391) bank162486 +162487 POINT(41.884617849791425 -87.59718536759638) bank162487 +162488 POINT(42.304299078368764 -88.12353255428468) bank162488 +162489 POINT(42.299785307422454 -87.99264808341329) bank162489 +162490 POINT(41.08785005268133 -87.87411395860467) bank162490 +162491 POINT(42.68209287030722 -87.07769426113282) bank162491 +162492 POINT(42.293342730356564 -87.00964588047125) bank162492 +162493 POINT(42.55927165646054 -87.67864462578999) bank162493 +162494 POINT(42.40816410124813 -87.75605936874898) bank162494 +162495 POINT(41.109354118303024 -86.87594620683842) bank162495 +162496 POINT(42.76604270564821 -86.87745461637415) bank162496 +162497 POINT(41.51457996005336 -87.50327918260209) bank162497 +162498 POINT(41.980217951159524 -86.78966662533803) bank162498 +162499 POINT(41.96824546208781 -86.80113467647027) bank162499 +162500 POINT(41.51819074699091 -88.36431597308109) bank162500 +162501 POINT(41.59447658892911 -87.43661627947434) bank162501 +162502 POINT(42.12860005985774 -87.3951853805318) bank162502 +162503 POINT(41.87283341619534 -86.6997362437083) bank162503 +162504 POINT(40.919282602696654 -87.7753433129984) bank162504 +162505 POINT(41.65980078413785 -87.49782600670416) bank162505 +162506 POINT(42.00880559193938 -88.5926085640365) bank162506 +162507 POINT(42.29056475684662 -88.38715450740912) bank162507 +162508 POINT(42.87046029490322 -87.66248084192536) bank162508 +162509 POINT(42.40640344258016 -88.1529951126115) bank162509 +162510 POINT(42.31429089111962 -87.65647667435113) bank162510 +162511 POINT(42.512400221929695 -88.19474260944753) bank162511 +162512 POINT(42.8451043630297 -88.391317107105) bank162512 +162513 POINT(42.47977477510705 -86.86994056630776) bank162513 +162514 POINT(42.81799664571222 -88.23323998982241) bank162514 +162515 POINT(42.0119434250323 -86.7510600817895) bank162515 +162516 POINT(42.50014618744922 -87.52674918777183) bank162516 +162517 POINT(41.86696500840925 -86.7282508677893) bank162517 +162518 POINT(41.34517022749018 -86.86976346683147) bank162518 +162519 POINT(41.02250721378034 -87.97231823079709) bank162519 +162520 POINT(42.793834771163866 -87.4884319948844) bank162520 +162521 POINT(41.14417748852426 -87.09080810097142) bank162521 +162522 POINT(42.3647919844843 -88.14690394134686) bank162522 +162523 POINT(42.45118598242993 -87.85689149093501) bank162523 +162524 POINT(42.1426439464335 -86.64635996798926) bank162524 +162525 POINT(41.86054344825845 -86.79291061999233) bank162525 +162526 POINT(41.864917557038794 -88.26386778368033) bank162526 +162527 POINT(42.4768669658347 -87.08721554431503) bank162527 +162528 POINT(41.71837328105217 -88.55426702834556) bank162528 +162529 POINT(42.493033531226274 -87.92275503378994) bank162529 +162530 POINT(42.82092395587848 -87.88649112315723) bank162530 +162531 POINT(41.323824792776584 -88.00579964799348) bank162531 +162532 POINT(41.86977282022782 -88.30511030704932) bank162532 +162533 POINT(41.0145558144932 -87.943009178403) bank162533 +162534 POINT(41.40362934796503 -88.57154746167761) bank162534 +162535 POINT(41.93715266508531 -87.22835123014559) bank162535 +162536 POINT(41.03320868003574 -87.05548861624304) bank162536 +162537 POINT(41.91730057564493 -86.83848005928888) bank162537 +162538 POINT(41.246680545950134 -87.92942115962775) bank162538 +162539 POINT(41.34149078761805 -87.48928267721782) bank162539 +162540 POINT(42.59154594121594 -86.68810817254207) bank162540 +162541 POINT(42.382278455716616 -87.35279182579403) bank162541 +162542 POINT(42.20112301508851 -86.88459797642578) bank162542 +162543 POINT(42.066125430316106 -88.19860142384967) bank162543 +162544 POINT(40.93393971257885 -87.43079312888833) bank162544 +162545 POINT(41.959752403654264 -86.79735494082252) bank162545 +162546 POINT(41.1906764447575 -86.77642280260699) bank162546 +162547 POINT(41.210509808161866 -86.94520733406395) bank162547 +162548 POINT(40.93576429449803 -87.17493478144479) bank162548 +162549 POINT(42.073031656627556 -86.73317785023036) bank162549 +162550 POINT(41.387718313211124 -87.1292529484352) bank162550 +162551 POINT(41.084970402370274 -88.49793721430146) bank162551 +162552 POINT(42.15014531409031 -88.38697047252305) bank162552 +162553 POINT(42.52274137670295 -88.1177060823628) bank162553 +162554 POINT(41.95479105882025 -86.86426726827567) bank162554 +162555 POINT(41.549560783606886 -86.85435383273952) bank162555 +162556 POINT(42.48971686574032 -87.69934919565621) bank162556 +162557 POINT(42.73714741142931 -88.25620329758577) bank162557 +162558 POINT(41.681447795865616 -86.90995627588596) bank162558 +162559 POINT(41.45438407493956 -86.74967913804228) bank162559 +162560 POINT(41.80051384232307 -87.5871699689104) bank162560 +162561 POINT(42.56293717829276 -88.26312353415273) bank162561 +162562 POINT(41.60823484909445 -88.6254237561652) bank162562 +162563 POINT(42.417028341314825 -87.91798288101988) bank162563 +162564 POINT(40.988329671830726 -87.9001845517668) bank162564 +162565 POINT(40.94215403532988 -87.71832766165426) bank162565 +162566 POINT(42.13740130647997 -87.54489525187961) bank162566 +162567 POINT(41.71683605623754 -86.85836985049686) bank162567 +162568 POINT(42.174245615547434 -88.03675663520697) bank162568 +162569 POINT(41.749473554736035 -87.05706105866712) bank162569 +162570 POINT(42.57469410336097 -86.93777148980006) bank162570 +162571 POINT(41.45910322339892 -87.84815027548422) bank162571 +162572 POINT(42.56368660767937 -88.15762521758904) bank162572 +162573 POINT(41.048034701104264 -87.28703992482778) bank162573 +162574 POINT(42.06631547380269 -88.62834814415618) bank162574 +162575 POINT(41.10554794616639 -87.65564155242292) bank162575 +162576 POINT(40.98482934484581 -88.53117925633954) bank162576 +162577 POINT(41.450424066891586 -88.24141340217714) bank162577 +162578 POINT(42.69369688702938 -88.07835732615804) bank162578 +162579 POINT(42.01266463862794 -87.33493540110429) bank162579 +162580 POINT(42.004502075580746 -87.40463043619295) bank162580 +162581 POINT(42.78116613612896 -87.55289515184272) bank162581 +162582 POINT(41.84920399651894 -86.66479257878423) bank162582 +162583 POINT(40.91923114694953 -87.0884773951023) bank162583 +162584 POINT(42.39870638501167 -88.20941166089888) bank162584 +162585 POINT(42.33392953882743 -87.68926260274515) bank162585 +162586 POINT(42.0743910457736 -87.87252120418212) bank162586 +162587 POINT(42.35976355563472 -87.72717375599373) bank162587 +162588 POINT(41.76781007563636 -88.16276210032089) bank162588 +162589 POINT(41.90378940012349 -88.10989401848578) bank162589 +162590 POINT(42.09903339902894 -87.7375334242311) bank162590 +162591 POINT(41.479941970906644 -87.75509416591488) bank162591 +162592 POINT(41.4234532097235 -87.3524398936888) bank162592 +162593 POINT(41.272831316211764 -87.56642630390876) bank162593 +162594 POINT(41.71006114398481 -87.84381235699831) bank162594 +162595 POINT(42.037067919167235 -87.69842064778666) bank162595 +162596 POINT(41.43024005348479 -87.31328339651775) bank162596 +162597 POINT(41.43979233122563 -87.23840708615727) bank162597 +162598 POINT(42.27885951617803 -87.99778082341037) bank162598 +162599 POINT(41.53766199046683 -87.14673574981296) bank162599 +162600 POINT(42.68964082579636 -88.62645189012915) bank162600 +162601 POINT(41.10718459073081 -88.10366179264895) bank162601 +162602 POINT(41.364077800952764 -88.04965451467145) bank162602 +162603 POINT(40.91706889590195 -87.213463658904) bank162603 +162604 POINT(41.18159689695493 -87.07074686918946) bank162604 +162605 POINT(41.79323854903013 -86.84137804472708) bank162605 +162606 POINT(42.5580527703375 -86.80779875615016) bank162606 +162607 POINT(41.78820053458275 -87.674016007462) bank162607 +162608 POINT(41.103227221185136 -87.5420839365028) bank162608 +162609 POINT(41.86202910620561 -86.78352723450128) bank162609 +162610 POINT(42.267831214676306 -87.19287765607919) bank162610 +162611 POINT(40.979755651556964 -86.99023705926786) bank162611 +162612 POINT(41.89207266776723 -88.21897504012527) bank162612 +162613 POINT(41.1729547290403 -88.5881550102043) bank162613 +162614 POINT(41.23344191641855 -87.87791767333864) bank162614 +162615 POINT(42.708341898675265 -87.34504240671593) bank162615 +162616 POINT(41.30895368299796 -87.09965568953861) bank162616 +162617 POINT(41.98481296942872 -88.39943823144013) bank162617 +162618 POINT(42.63567151298953 -86.83375082633151) bank162618 +162619 POINT(42.43857593466536 -86.65249981636472) bank162619 +162620 POINT(42.784699478573586 -87.35073251358983) bank162620 +162621 POINT(42.45444309129333 -86.97265839273365) bank162621 +162622 POINT(41.651063674631125 -88.07620033295683) bank162622 +162623 POINT(42.21980453658311 -86.94247828809087) bank162623 +162624 POINT(42.829384654460185 -87.10132173836529) bank162624 +162625 POINT(42.259300563107175 -86.8842618169084) bank162625 +162626 POINT(41.51788573185065 -87.45643721847588) bank162626 +162627 POINT(42.637946501205384 -87.78261796191833) bank162627 +162628 POINT(42.19628047625455 -87.63780765019868) bank162628 +162629 POINT(41.50198598609448 -88.1049947261766) bank162629 +162630 POINT(42.17280784276811 -88.51808975382158) bank162630 +162631 POINT(41.827312023728474 -86.94935770154477) bank162631 +162632 POINT(42.497234365359965 -88.54717381120028) bank162632 +162633 POINT(42.215062034820676 -88.41331612717543) bank162633 +162634 POINT(41.60721029421691 -87.28968666706683) bank162634 +162635 POINT(41.24770759440641 -88.30167732672776) bank162635 +162636 POINT(41.33923668216152 -86.91370136364226) bank162636 +162637 POINT(41.98968733875095 -87.62857133632957) bank162637 +162638 POINT(40.98874494463845 -87.13858385690213) bank162638 +162639 POINT(42.04833896605208 -86.96014062601992) bank162639 +162640 POINT(41.17266582017517 -86.82202134042518) bank162640 +162641 POINT(41.13378292141047 -87.0237384063189) bank162641 +162642 POINT(41.05908896376567 -86.68873758759622) bank162642 +162643 POINT(41.28790662539988 -86.7087395228579) bank162643 +162644 POINT(42.48690496441574 -87.9451563462932) bank162644 +162645 POINT(42.51421202533518 -86.98459273747484) bank162645 +162646 POINT(41.53645883321164 -88.05850478458571) bank162646 +162647 POINT(42.35768972524523 -87.73389495537421) bank162647 +162648 POINT(41.562081260493976 -87.14599181893404) bank162648 +162649 POINT(40.99803566158244 -88.59059432446514) bank162649 +162650 POINT(41.83843555745271 -86.66819886173796) bank162650 +162651 POINT(41.13100420850031 -87.00490344768755) bank162651 +162652 POINT(41.9191149750896 -86.68654526995003) bank162652 +162653 POINT(42.80178673972743 -86.92232783168869) bank162653 +162654 POINT(41.49800723398188 -87.01030500478477) bank162654 +162655 POINT(41.98337764450421 -87.98853271066378) bank162655 +162656 POINT(42.38950950592187 -88.02933067761698) bank162656 +162657 POINT(42.2320040140912 -88.5098037618061) bank162657 +162658 POINT(41.47017824097052 -86.68592540839715) bank162658 +162659 POINT(42.10072784615538 -88.50744978352539) bank162659 +162660 POINT(42.581638722188245 -87.625642914535) bank162660 +162661 POINT(41.162986916562836 -87.5319023225849) bank162661 +162662 POINT(41.65404487505623 -87.89962631328869) bank162662 +162663 POINT(41.071922948389854 -88.19619998012682) bank162663 +162664 POINT(41.31454548088352 -87.84028885223461) bank162664 +162665 POINT(41.46235792631747 -88.36582680791294) bank162665 +162666 POINT(41.33077952035931 -87.80819377241708) bank162666 +162667 POINT(41.40290051210158 -86.92862900108163) bank162667 +162668 POINT(42.75990983620361 -87.81288645918427) bank162668 +162669 POINT(42.3729852142912 -87.21903095931701) bank162669 +162670 POINT(42.77699450432852 -88.286848371414) bank162670 +162671 POINT(42.829363406406024 -87.07723081190699) bank162671 +162672 POINT(41.587963729161075 -86.85353769302999) bank162672 +162673 POINT(42.71717068789682 -87.65452232432956) bank162673 +162674 POINT(42.40934953757271 -87.64284223046707) bank162674 +162675 POINT(41.86029945484714 -87.59898489469194) bank162675 +162676 POINT(41.1306822985085 -87.84702285719204) bank162676 +162677 POINT(42.44267480878556 -87.72504104251364) bank162677 +162678 POINT(41.43183927564692 -87.59051234309909) bank162678 +162679 POINT(42.84801966160251 -86.81244103337741) bank162679 +162680 POINT(41.54465259312572 -86.81458483231243) bank162680 +162681 POINT(42.64599436934598 -88.62973488029893) bank162681 +162682 POINT(41.65250065185844 -87.3016763871535) bank162682 +162683 POINT(41.91724415221384 -87.30859559597356) bank162683 +162684 POINT(41.953524662507654 -86.9049725460076) bank162684 +162685 POINT(40.966641984793455 -86.71727228167052) bank162685 +162686 POINT(40.963629826902725 -87.58134918190866) bank162686 +162687 POINT(41.71988051352509 -88.30671931899593) bank162687 +162688 POINT(42.8303254613489 -86.90524930121497) bank162688 +162689 POINT(42.65151046914915 -88.21063051893289) bank162689 +162690 POINT(42.5786640072096 -87.31556298067544) bank162690 +162691 POINT(41.786114960056 -87.8539821410981) bank162691 +162692 POINT(41.15935049062788 -88.12162263114692) bank162692 +162693 POINT(42.87024278387406 -87.0290429695262) bank162693 +162694 POINT(41.31753046677234 -87.78574632623288) bank162694 +162695 POINT(41.70714859001765 -87.60796118595948) bank162695 +162696 POINT(42.4653149542853 -86.86097424955958) bank162696 +162697 POINT(41.489597788492866 -88.37624626842025) bank162697 +162698 POINT(41.67249345184117 -87.80544527060165) bank162698 +162699 POINT(42.66807486994776 -87.49335998672808) bank162699 +162700 POINT(41.101519595857226 -88.40077879013982) bank162700 +162701 POINT(41.146760658076865 -86.7101155834085) bank162701 +162702 POINT(42.647049031997376 -88.5228156120158) bank162702 +162703 POINT(42.0062331230586 -88.52457554763338) bank162703 +162704 POINT(41.24885197984874 -87.60087269725256) bank162704 +162705 POINT(42.0115554442671 -88.40201447385988) bank162705 +162706 POINT(42.38204746916886 -88.00340455706997) bank162706 +162707 POINT(41.73549915602916 -88.23101534450385) bank162707 +162708 POINT(42.681110660378266 -87.74278442323376) bank162708 +162709 POINT(41.583779098402665 -87.99916897391304) bank162709 +162710 POINT(42.293967549902135 -86.75058496940422) bank162710 +162711 POINT(41.81404934681186 -87.92137273672323) bank162711 +162712 POINT(41.43243252140311 -86.75865091707024) bank162712 +162713 POINT(42.1133083996133 -87.65524201625635) bank162713 +162714 POINT(42.72653299887677 -87.85011898212838) bank162714 +162715 POINT(41.39164831431331 -87.92545964441636) bank162715 +162716 POINT(42.33263799602499 -87.23509723602405) bank162716 +162717 POINT(41.57573571420042 -87.70009037654675) bank162717 +162718 POINT(41.88825158041025 -87.07527381860812) bank162718 +162719 POINT(42.83894187325504 -88.16430091705237) bank162719 +162720 POINT(42.016349343207736 -88.60859724478529) bank162720 +162721 POINT(41.977996669855216 -88.18870118801453) bank162721 +162722 POINT(41.824590339375675 -86.6543689967712) bank162722 +162723 POINT(41.09698493652322 -88.25730103004342) bank162723 +162724 POINT(40.93369876462681 -88.0023556258611) bank162724 +162725 POINT(41.76701264544584 -86.82630390605885) bank162725 +162726 POINT(40.92568451222314 -86.652952143621) bank162726 +162727 POINT(42.27178894503569 -88.59136342989132) bank162727 +162728 POINT(41.0873571404713 -88.60955590450824) bank162728 +162729 POINT(41.207115056879324 -87.19876183937431) bank162729 +162730 POINT(42.30253579612324 -88.02980909466078) bank162730 +162731 POINT(42.05103727750344 -88.58224656024183) bank162731 +162732 POINT(42.29280789558114 -88.44232457162136) bank162732 +162733 POINT(42.422356901744685 -86.74032170256909) bank162733 +162734 POINT(42.07697914685302 -86.89093404953542) bank162734 +162735 POINT(40.960746038746244 -88.20842758960848) bank162735 +162736 POINT(40.93525407322988 -87.79058147753778) bank162736 +162737 POINT(42.432936264608855 -88.59808603716577) bank162737 +162738 POINT(42.36301399130476 -86.65226552058171) bank162738 +162739 POINT(42.76559283401514 -88.16204876800869) bank162739 +162740 POINT(40.96155532753838 -86.87784949682732) bank162740 +162741 POINT(41.87272883187258 -87.33045380735115) bank162741 +162742 POINT(41.99175992139012 -87.73057926326852) bank162742 +162743 POINT(40.92315919258887 -87.36600381072239) bank162743 +162744 POINT(42.386110102206345 -87.52661821473271) bank162744 +162745 POINT(41.94890996150157 -88.54484586705625) bank162745 +162746 POINT(41.070903865418536 -87.03661457282583) bank162746 +162747 POINT(41.28729780994125 -87.67509000703558) bank162747 +162748 POINT(42.76094782659667 -87.05989067869494) bank162748 +162749 POINT(42.58153528873834 -86.70585722516637) bank162749 +162750 POINT(42.316002076745264 -87.69380559100672) bank162750 +162751 POINT(41.61562350032462 -87.0421542115834) bank162751 +162752 POINT(41.100324456462964 -86.85524496668263) bank162752 +162753 POINT(41.45605759459991 -88.22268705706318) bank162753 +162754 POINT(40.90568868763864 -88.32057199172026) bank162754 +162755 POINT(42.15803691991227 -88.14640573764578) bank162755 +162756 POINT(42.29847972149127 -87.63522845589253) bank162756 +162757 POINT(41.11325651459727 -88.2863304938911) bank162757 +162758 POINT(42.38032467303128 -87.25586245037523) bank162758 +162759 POINT(42.311198902253814 -86.77457074198414) bank162759 +162760 POINT(42.17870139929339 -87.62557141966187) bank162760 +162761 POINT(42.81932477761567 -88.3059737369913) bank162761 +162762 POINT(42.22017197648191 -86.78143100776907) bank162762 +162763 POINT(41.20875910622434 -88.56000684565079) bank162763 +162764 POINT(41.95766739905381 -88.56957581776528) bank162764 +162765 POINT(41.786235332203134 -88.39245137891233) bank162765 +162766 POINT(42.25677041503531 -87.30614584833883) bank162766 +162767 POINT(42.29726857073958 -88.59302564656487) bank162767 +162768 POINT(40.98391515438285 -86.6723127634956) bank162768 +162769 POINT(42.53621868728218 -88.20388564716595) bank162769 +162770 POINT(42.25623769302865 -88.07742456757933) bank162770 +162771 POINT(42.00029813346403 -87.31794378273996) bank162771 +162772 POINT(41.00894906763879 -87.91919862709123) bank162772 +162773 POINT(42.01685438567042 -87.38183121109161) bank162773 +162774 POINT(41.32597008883071 -87.61299213420288) bank162774 +162775 POINT(41.82448318687498 -86.70953669877682) bank162775 +162776 POINT(42.278951334129545 -87.78963596953449) bank162776 +162777 POINT(41.540631158417625 -87.09790513063521) bank162777 +162778 POINT(41.55738601022881 -86.87982775254403) bank162778 +162779 POINT(41.27750908088664 -87.34296222520278) bank162779 +162780 POINT(41.09745952395967 -87.03398780307512) bank162780 +162781 POINT(42.451381969852775 -87.98194813329995) bank162781 +162782 POINT(41.55440111879153 -87.54925622668927) bank162782 +162783 POINT(41.594813069436576 -87.35123050363914) bank162783 +162784 POINT(42.545427348034 -86.89625790923955) bank162784 +162785 POINT(42.473862448426885 -87.20186348450132) bank162785 +162786 POINT(41.20640264758883 -87.8425115568736) bank162786 +162787 POINT(42.38111013045714 -87.27421578502123) bank162787 +162788 POINT(41.23667578178382 -87.8652834716055) bank162788 +162789 POINT(40.884761157920714 -87.5013339376611) bank162789 +162790 POINT(42.55470283209071 -87.97518113585144) bank162790 +162791 POINT(41.39741143546316 -87.6083395514379) bank162791 +162792 POINT(41.149757651561046 -86.68099492452437) bank162792 +162793 POINT(41.61948058464643 -87.6582647370549) bank162793 +162794 POINT(41.724809766027334 -88.04029830098487) bank162794 +162795 POINT(42.15452917757937 -87.9065511568064) bank162795 +162796 POINT(42.27427819596099 -86.71056316618716) bank162796 +162797 POINT(41.38048469434947 -87.48329016756877) bank162797 +162798 POINT(41.18312264591545 -86.64763652636779) bank162798 +162799 POINT(42.37598088372907 -88.30946081194874) bank162799 +162800 POINT(41.59425178431325 -88.51445086820557) bank162800 +162801 POINT(42.51701641156464 -86.6352838569946) bank162801 +162802 POINT(41.74929182008054 -87.74364049473527) bank162802 +162803 POINT(42.09575725461772 -87.99168348718851) bank162803 +162804 POINT(41.55725521840164 -88.20825524009936) bank162804 +162805 POINT(42.0213803393301 -88.41414505846822) bank162805 +162806 POINT(41.54877747864536 -87.96941930118916) bank162806 +162807 POINT(41.31623092325249 -87.91851542525386) bank162807 +162808 POINT(41.37205948653212 -88.10799295453099) bank162808 +162809 POINT(42.72456554870174 -86.75765886025388) bank162809 +162810 POINT(41.14737335032631 -87.785959563168) bank162810 +162811 POINT(42.02100341519699 -88.31345521384375) bank162811 +162812 POINT(41.01313984145582 -87.72966514811499) bank162812 +162813 POINT(41.442787688138736 -86.66766623772754) bank162813 +162814 POINT(41.693084126512346 -86.89928280600337) bank162814 +162815 POINT(41.79448187906782 -88.56250797187364) bank162815 +162816 POINT(42.18559949324993 -87.69125790428991) bank162816 +162817 POINT(41.409663556442396 -86.75055561835731) bank162817 +162818 POINT(41.34240617634442 -87.00338153353427) bank162818 +162819 POINT(41.66605157271425 -87.04649490199705) bank162819 +162820 POINT(40.976166103903275 -87.57767881305749) bank162820 +162821 POINT(42.76943862303082 -88.47950362186918) bank162821 +162822 POINT(41.13645798068822 -88.37572330638127) bank162822 +162823 POINT(42.362276640351865 -88.0708736743281) bank162823 +162824 POINT(41.98321614915414 -88.32607861947085) bank162824 +162825 POINT(42.57026165696583 -87.69776438879651) bank162825 +162826 POINT(41.27787964175509 -87.49582381222197) bank162826 +162827 POINT(41.53044234291098 -87.2068135654944) bank162827 +162828 POINT(42.21887529881608 -87.60710245367855) bank162828 +162829 POINT(41.44148315524806 -87.6712095308553) bank162829 +162830 POINT(41.370753928096846 -87.30420271574148) bank162830 +162831 POINT(42.640866969010645 -88.37890018996917) bank162831 +162832 POINT(42.386559552394814 -87.77384754247568) bank162832 +162833 POINT(41.94546588615239 -87.62778218947797) bank162833 +162834 POINT(42.81205482543021 -88.57047529728368) bank162834 +162835 POINT(42.442796869378455 -87.01009445614243) bank162835 +162836 POINT(42.64754542862459 -88.52834293415559) bank162836 +162837 POINT(42.19237467533201 -87.4518980289743) bank162837 +162838 POINT(42.2198062687013 -88.44325585576267) bank162838 +162839 POINT(42.81246283860809 -87.5509945553443) bank162839 +162840 POINT(41.21239476731266 -88.05085780448891) bank162840 +162841 POINT(42.83381929210027 -86.82869638217059) bank162841 +162842 POINT(40.96713975222872 -86.84968039451745) bank162842 +162843 POINT(42.32202761631815 -87.12677967726711) bank162843 +162844 POINT(42.619956201434114 -86.8579381120476) bank162844 +162845 POINT(42.04778245884718 -87.47982518885169) bank162845 +162846 POINT(41.44866542039768 -87.76949891997089) bank162846 +162847 POINT(41.561997043666516 -86.85202911249843) bank162847 +162848 POINT(41.80099272114728 -88.5139831395824) bank162848 +162849 POINT(42.02148909810828 -87.81997900646907) bank162849 +162850 POINT(42.25397492393492 -87.4853656759632) bank162850 +162851 POINT(41.19733349301193 -86.99943672906907) bank162851 +162852 POINT(42.717066999657796 -87.53793812374134) bank162852 +162853 POINT(41.60777028690037 -88.30007995005256) bank162853 +162854 POINT(41.25151261126425 -87.1074430969634) bank162854 +162855 POINT(41.181130713640805 -88.48827038055566) bank162855 +162856 POINT(41.027037415452206 -86.75452671960895) bank162856 +162857 POINT(42.07455379666022 -88.29469760626013) bank162857 +162858 POINT(40.97915103405019 -87.08037598699428) bank162858 +162859 POINT(41.92140563587521 -87.56126400414607) bank162859 +162860 POINT(42.500018933614435 -86.81465165329853) bank162860 +162861 POINT(41.66581330395128 -88.35724553795542) bank162861 +162862 POINT(41.867267544661956 -87.89636585276936) bank162862 +162863 POINT(41.54021000755156 -87.0520357894093) bank162863 +162864 POINT(42.19312770053678 -86.83314230808395) bank162864 +162865 POINT(42.64312838583835 -88.30072031463366) bank162865 +162866 POINT(41.53399422580651 -88.18171531422513) bank162866 +162867 POINT(41.66247620318002 -88.36761248146797) bank162867 +162868 POINT(41.76500954043691 -88.54428729984915) bank162868 +162869 POINT(41.49048269304274 -88.52214542410765) bank162869 +162870 POINT(41.58627224742639 -87.51955172568694) bank162870 +162871 POINT(41.67385884809631 -87.23737268559621) bank162871 +162872 POINT(42.78046747161385 -87.5616101806834) bank162872 +162873 POINT(42.02249137290504 -87.87613367601321) bank162873 +162874 POINT(41.10948275242291 -86.84125834177506) bank162874 +162875 POINT(42.19339713959719 -88.24335852581987) bank162875 +162876 POINT(42.08591743126635 -87.91918403239741) bank162876 +162877 POINT(42.462070431789286 -88.34261990106674) bank162877 +162878 POINT(41.091346420125184 -87.67025448922516) bank162878 +162879 POINT(41.277916132534976 -86.84147677753148) bank162879 +162880 POINT(41.38081992202052 -86.95126110015876) bank162880 +162881 POINT(41.29674628117724 -88.33097544609609) bank162881 +162882 POINT(42.82291388388969 -87.8772481053978) bank162882 +162883 POINT(42.00664414931135 -86.7438632997634) bank162883 +162884 POINT(41.34944797699448 -86.83587287674129) bank162884 +162885 POINT(42.80942055162883 -88.53089308179946) bank162885 +162886 POINT(41.842900357382064 -87.10913319591874) bank162886 +162887 POINT(42.48586168032772 -87.97811935264107) bank162887 +162888 POINT(41.66833178752508 -86.70141576352495) bank162888 +162889 POINT(41.941784769179165 -88.4572480942872) bank162889 +162890 POINT(42.583618748950684 -87.45270827413599) bank162890 +162891 POINT(41.55109777453518 -86.69845617885191) bank162891 +162892 POINT(42.397770424466565 -88.07493292455752) bank162892 +162893 POINT(42.00967237470762 -88.52685662532676) bank162893 +162894 POINT(41.82295221701928 -88.51259432316469) bank162894 +162895 POINT(41.99017356829678 -88.45423955678348) bank162895 +162896 POINT(42.57828901191124 -87.97961638650467) bank162896 +162897 POINT(41.15902101458548 -87.83643181379071) bank162897 +162898 POINT(42.38692105858399 -86.99985190413297) bank162898 +162899 POINT(42.286837886368694 -88.42508788897734) bank162899 +162900 POINT(42.20348616152435 -87.01174885378614) bank162900 +162901 POINT(42.5408578746132 -87.34082006660344) bank162901 +162902 POINT(41.21687033059672 -86.90411635392452) bank162902 +162903 POINT(42.702299417836734 -87.99743656037269) bank162903 +162904 POINT(42.688519156648056 -87.86005075844236) bank162904 +162905 POINT(41.84818952422784 -88.44234382939392) bank162905 +162906 POINT(41.926462504550635 -87.36150686179066) bank162906 +162907 POINT(42.09044870334929 -87.17447362145406) bank162907 +162908 POINT(42.56682361450231 -87.76886790911061) bank162908 +162909 POINT(42.10877633441966 -87.91348374093242) bank162909 +162910 POINT(42.483478050445186 -87.54917283780841) bank162910 +162911 POINT(41.3886787654734 -86.80823750664733) bank162911 +162912 POINT(41.60714465489589 -86.91624594300377) bank162912 +162913 POINT(41.00864228726537 -86.78723912043314) bank162913 +162914 POINT(41.69857384394304 -87.60696646640073) bank162914 +162915 POINT(41.9113401444052 -87.79700965374793) bank162915 +162916 POINT(42.78958411478658 -87.08391976647611) bank162916 +162917 POINT(41.944772059171605 -87.39394109766486) bank162917 +162918 POINT(41.48415342250002 -87.40544832376582) bank162918 +162919 POINT(41.98285769623831 -86.72667776738312) bank162919 +162920 POINT(42.1333184493287 -87.24315657985798) bank162920 +162921 POINT(42.74789840650993 -88.08429852954221) bank162921 +162922 POINT(42.03582997679816 -87.55016014258169) bank162922 +162923 POINT(40.94565545832486 -87.1336578232754) bank162923 +162924 POINT(42.207193482784774 -87.89441068988077) bank162924 +162925 POINT(41.020177376783984 -87.26838948270719) bank162925 +162926 POINT(42.56805227595069 -87.9702162169713) bank162926 +162927 POINT(41.168176109440914 -86.76723991151181) bank162927 +162928 POINT(41.18668757664751 -88.30950324526701) bank162928 +162929 POINT(41.39791415893456 -86.66320358782944) bank162929 +162930 POINT(42.15645707017597 -88.42701425635289) bank162930 +162931 POINT(40.897531503926174 -87.76472624665836) bank162931 +162932 POINT(41.91571942193731 -86.63558800691493) bank162932 +162933 POINT(42.152436148988606 -87.03453033731778) bank162933 +162934 POINT(41.33092986317901 -88.3964326545003) bank162934 +162935 POINT(41.37347497472992 -87.56660562292566) bank162935 +162936 POINT(42.87017806408764 -86.7346091785854) bank162936 +162937 POINT(41.44250272526778 -86.81914646190378) bank162937 +162938 POINT(41.6930566018373 -87.18011621125028) bank162938 +162939 POINT(41.55559443827245 -88.38522711633388) bank162939 +162940 POINT(41.24644356918734 -88.24135792270512) bank162940 +162941 POINT(41.32918691825591 -88.58134323573175) bank162941 +162942 POINT(41.79160173756162 -86.80049587753945) bank162942 +162943 POINT(42.685977448006234 -87.19426567190914) bank162943 +162944 POINT(40.926646960852274 -87.28595248102995) bank162944 +162945 POINT(42.2054748948463 -88.27972291133774) bank162945 +162946 POINT(41.89097290553983 -88.147954657276) bank162946 +162947 POINT(42.371444241277295 -88.23082608816138) bank162947 +162948 POINT(41.879626325488424 -88.42645307208993) bank162948 +162949 POINT(41.31720696242299 -87.3224322353078) bank162949 +162950 POINT(42.831422055203966 -88.40338230314308) bank162950 +162951 POINT(42.35721477425328 -86.71969335396456) bank162951 +162952 POINT(42.48110656243817 -88.04682968878163) bank162952 +162953 POINT(41.54920146864348 -87.53389896524753) bank162953 +162954 POINT(42.4839436811269 -87.19458164492919) bank162954 +162955 POINT(40.925599613920596 -88.00028625572428) bank162955 +162956 POINT(41.50149315967945 -88.13093251037833) bank162956 +162957 POINT(42.574959840468765 -86.95667684921135) bank162957 +162958 POINT(41.4733601715447 -87.89585932063551) bank162958 +162959 POINT(41.30426027550532 -88.5252949786781) bank162959 +162960 POINT(40.939212296444396 -88.30673395982193) bank162960 +162961 POINT(41.48607676561391 -88.26610518979764) bank162961 +162962 POINT(42.63079622490088 -86.7911374316391) bank162962 +162963 POINT(40.89787115974284 -88.39019350673757) bank162963 +162964 POINT(42.209846820263685 -86.73862355690403) bank162964 +162965 POINT(41.53359724130149 -87.57028226461469) bank162965 +162966 POINT(41.49771149077783 -87.85359471831718) bank162966 +162967 POINT(41.94722073321492 -86.64897411410091) bank162967 +162968 POINT(42.17683691907826 -87.3685255046656) bank162968 +162969 POINT(41.703530574015666 -86.79871911430634) bank162969 +162970 POINT(42.42950248549122 -87.07822779793973) bank162970 +162971 POINT(41.12535827281594 -88.49215766018882) bank162971 +162972 POINT(42.8034707663428 -87.6706634331998) bank162972 +162973 POINT(42.2976549484743 -87.88234927822232) bank162973 +162974 POINT(41.734383031421174 -87.8093783553865) bank162974 +162975 POINT(41.00149943372925 -88.53774314753122) bank162975 +162976 POINT(42.14987279317535 -88.21665479048187) bank162976 +162977 POINT(41.97020541983628 -87.70709486959554) bank162977 +162978 POINT(42.31211192612094 -86.96857434947495) bank162978 +162979 POINT(42.69715724926109 -88.08371684236049) bank162979 +162980 POINT(41.456316208837976 -87.85062730346147) bank162980 +162981 POINT(41.261416189363466 -87.20599970445885) bank162981 +162982 POINT(41.69694430136187 -87.67180749658561) bank162982 +162983 POINT(42.35514167693316 -86.68929497360043) bank162983 +162984 POINT(41.881820862161895 -88.42490252279525) bank162984 +162985 POINT(41.774966264766825 -87.16422541308448) bank162985 +162986 POINT(41.52215045509932 -86.87760037823655) bank162986 +162987 POINT(42.38776290132221 -88.48651050376131) bank162987 +162988 POINT(42.68153348790019 -88.29398525820746) bank162988 +162989 POINT(42.658641050291614 -87.89549062669568) bank162989 +162990 POINT(41.431169392001415 -87.82154408480294) bank162990 +162991 POINT(42.708420011375104 -88.41555081838267) bank162991 +162992 POINT(42.591027929571666 -87.39818563430875) bank162992 +162993 POINT(40.88592553762114 -87.37289727839466) bank162993 +162994 POINT(42.484453256624306 -88.57621665153881) bank162994 +162995 POINT(41.2370843988606 -86.86029551318391) bank162995 +162996 POINT(40.99934640716452 -87.29749623650602) bank162996 +162997 POINT(42.615675798885356 -87.4394779518552) bank162997 +162998 POINT(41.92747354994237 -88.08537775835234) bank162998 +162999 POINT(41.97646442380816 -87.77594194718628) bank162999 +163000 POINT(41.53755716175236 -87.44693132451049) bank163000 +163001 POINT(41.837469099165915 -86.71360408039217) bank163001 +163002 POINT(41.59070300404238 -87.71755224957238) bank163002 +163003 POINT(41.65265331595858 -88.40715655057502) bank163003 +163004 POINT(41.34287432680182 -87.45407735067138) bank163004 +163005 POINT(42.80279428874146 -86.9571362224817) bank163005 +163006 POINT(41.52357940595494 -87.92902554808263) bank163006 +163007 POINT(42.15693040486327 -88.20163790500138) bank163007 +163008 POINT(40.885355027923254 -87.47702127826669) bank163008 +163009 POINT(41.936814842346216 -88.0910805852223) bank163009 +163010 POINT(42.4899657431204 -87.39278650061372) bank163010 +163011 POINT(41.329819568471436 -87.00139619493467) bank163011 +163012 POINT(42.27854622958396 -87.00500391288023) bank163012 +163013 POINT(41.66508304583935 -88.3785385693146) bank163013 +163014 POINT(41.6260478722656 -87.37019027329576) bank163014 +163015 POINT(41.84228268079593 -87.02822832245934) bank163015 +163016 POINT(41.458350595338246 -87.39086743396258) bank163016 +163017 POINT(42.278199511253895 -88.17037316270563) bank163017 +163018 POINT(42.670340231971345 -87.24758233093345) bank163018 +163019 POINT(41.769756134154164 -87.15745143072049) bank163019 +163020 POINT(41.100367089594464 -87.45816149722725) bank163020 +163021 POINT(42.47285267945566 -87.6300923709644) bank163021 +163022 POINT(40.93231634672891 -87.66543190829776) bank163022 +163023 POINT(41.453095784016845 -88.16500459883333) bank163023 +163024 POINT(41.70356992731941 -88.5552390248505) bank163024 +163025 POINT(42.32810179354248 -87.43899081491892) bank163025 +163026 POINT(42.151202128739605 -87.81509705413929) bank163026 +163027 POINT(41.9883970302781 -88.41896748580955) bank163027 +163028 POINT(41.795021070922 -86.9849807965939) bank163028 +163029 POINT(41.352990239666255 -88.24000552241759) bank163029 +163030 POINT(42.212400565510045 -87.03138496700846) bank163030 +163031 POINT(40.917014817521064 -86.68589663076976) bank163031 +163032 POINT(42.29326533281284 -86.85670936222763) bank163032 +163033 POINT(41.72919935448146 -86.74047892732939) bank163033 +163034 POINT(41.471677034003214 -86.66426905515175) bank163034 +163035 POINT(42.385318595602044 -88.58400450122663) bank163035 +163036 POINT(42.180269677705084 -87.34713422235657) bank163036 +163037 POINT(42.30532918612218 -86.69709298375516) bank163037 +163038 POINT(42.45017026417175 -87.13654656808113) bank163038 +163039 POINT(41.53118598724551 -86.63572275639594) bank163039 +163040 POINT(41.27868731464703 -87.6404535320838) bank163040 +163041 POINT(42.8456523037894 -88.5106982893608) bank163041 +163042 POINT(42.03960359076342 -88.3905598899611) bank163042 +163043 POINT(41.75995156482116 -87.86535696957604) bank163043 +163044 POINT(42.39995836095715 -88.49212996598972) bank163044 +163045 POINT(42.41374478729377 -88.55343925840464) bank163045 +163046 POINT(41.76859967632819 -88.25833710520877) bank163046 +163047 POINT(40.92069050609873 -86.79417277165186) bank163047 +163048 POINT(41.89650883592359 -87.92326256330932) bank163048 +163049 POINT(41.35224331226924 -87.00215765764695) bank163049 +163050 POINT(41.48518224266998 -87.72369242014369) bank163050 +163051 POINT(41.398191958970344 -86.85126729308102) bank163051 +163052 POINT(41.69824222147796 -88.18090629777292) bank163052 +163053 POINT(41.95011020935891 -87.70109182744336) bank163053 +163054 POINT(41.52003236042843 -86.78117892391138) bank163054 +163055 POINT(41.36730696714555 -88.15048747305543) bank163055 +163056 POINT(41.10184286825796 -87.4567163025364) bank163056 +163057 POINT(41.08717855392247 -86.67359837234628) bank163057 +163058 POINT(42.67474651832815 -87.95688004613811) bank163058 +163059 POINT(41.28652927951197 -87.43308247529706) bank163059 +163060 POINT(41.8994276089728 -87.31723904431857) bank163060 +163061 POINT(42.52175648457102 -87.3319079733908) bank163061 +163062 POINT(41.06201932819173 -87.17899304342414) bank163062 +163063 POINT(42.71799471059321 -87.79572450387664) bank163063 +163064 POINT(41.04216009802029 -87.11285246355945) bank163064 +163065 POINT(41.18984864997786 -88.18523630135647) bank163065 +163066 POINT(41.538811495243806 -87.04046110660668) bank163066 +163067 POINT(41.5394843326956 -87.37775511335163) bank163067 +163068 POINT(41.54436822048007 -87.78591212722888) bank163068 +163069 POINT(41.443304536775535 -86.63135348552386) bank163069 +163070 POINT(41.49861832786693 -88.43411655457504) bank163070 +163071 POINT(41.08735346520405 -87.10070679826742) bank163071 +163072 POINT(41.123863139154416 -86.83309987723365) bank163072 +163073 POINT(41.7555382448169 -87.09032120087384) bank163073 +163074 POINT(41.84819173876268 -86.85858892355007) bank163074 +163075 POINT(42.75960083402944 -87.98337800887548) bank163075 +163076 POINT(42.75635187208347 -87.27779201045318) bank163076 +163077 POINT(41.58927948480446 -87.70826528928703) bank163077 +163078 POINT(42.52472478346422 -87.9504841236785) bank163078 +163079 POINT(41.32672968372063 -87.04449303199381) bank163079 +163080 POINT(41.114056661577415 -88.23107936082755) bank163080 +163081 POINT(41.50297096925537 -87.6339644983203) bank163081 +163082 POINT(42.23558178419859 -87.89805200329066) bank163082 +163083 POINT(40.971767262341956 -87.43426659563117) bank163083 +163084 POINT(42.06038528976929 -87.81592290225127) bank163084 +163085 POINT(42.463082621085604 -87.50159843741261) bank163085 +163086 POINT(41.213077257674726 -87.73102460550392) bank163086 +163087 POINT(41.755134532773695 -88.48645274229499) bank163087 +163088 POINT(42.18685789551199 -87.65863375613316) bank163088 +163089 POINT(41.21618213115445 -87.58872216285079) bank163089 +163090 POINT(41.51493230050108 -86.90827613627275) bank163090 +163091 POINT(42.670351903842786 -88.16051949329147) bank163091 +163092 POINT(41.18413399893007 -88.50991990790531) bank163092 +163093 POINT(41.61321341122838 -87.36368494393797) bank163093 +163094 POINT(41.6728901784025 -87.96390393503239) bank163094 +163095 POINT(41.90684001755306 -88.03193450269005) bank163095 +163096 POINT(42.64008198238716 -87.09502414356686) bank163096 +163097 POINT(42.84778379976277 -87.48683665303521) bank163097 +163098 POINT(41.66807599994008 -87.23507525674441) bank163098 +163099 POINT(42.7046310689149 -88.18949825407533) bank163099 +163100 POINT(42.65405969786001 -87.19020537973867) bank163100 +163101 POINT(42.85278092345815 -88.50952438094686) bank163101 +163102 POINT(42.53736259832979 -88.11611242049054) bank163102 +163103 POINT(42.72951736050527 -86.70241679303882) bank163103 +163104 POINT(42.79060020154367 -86.64604022186212) bank163104 +163105 POINT(42.595407130863904 -87.19522045827813) bank163105 +163106 POINT(41.2547886503665 -87.02793715349138) bank163106 +163107 POINT(41.125813627738395 -88.05819905014603) bank163107 +163108 POINT(42.38223064790769 -87.28724563087691) bank163108 +163109 POINT(42.87522829808417 -88.34851535075742) bank163109 +163110 POINT(42.110265611169595 -86.69895751537891) bank163110 +163111 POINT(41.867402272441524 -87.24915803252544) bank163111 +163112 POINT(42.129477622541735 -87.76471672649615) bank163112 +163113 POINT(41.0791173185782 -87.25771492457494) bank163113 +163114 POINT(41.8363577023172 -87.8353559273832) bank163114 +163115 POINT(42.4568084434512 -86.78540665391989) bank163115 +163116 POINT(42.466118181478656 -87.81660282021811) bank163116 +163117 POINT(41.646742789294535 -87.05083605930558) bank163117 +163118 POINT(42.267370208167016 -87.31978625086458) bank163118 +163119 POINT(40.93661555224948 -88.537168563265) bank163119 +163120 POINT(40.91846193391206 -88.00691336005825) bank163120 +163121 POINT(41.37627830725691 -87.70128668847663) bank163121 +163122 POINT(41.34233420386568 -86.84751196621046) bank163122 +163123 POINT(41.26620668725286 -87.06379241053632) bank163123 +163124 POINT(42.68070844998267 -87.46741213472394) bank163124 +163125 POINT(42.720241028586045 -87.18730742455816) bank163125 +163126 POINT(41.05605124129921 -88.38873928681615) bank163126 +163127 POINT(41.995078913588735 -88.2769474601981) bank163127 +163128 POINT(41.399586417052475 -87.67034997860277) bank163128 +163129 POINT(41.136263192025105 -88.52378406154874) bank163129 +163130 POINT(41.58077418448161 -86.93572174039089) bank163130 +163131 POINT(42.432320917773446 -87.90521814155093) bank163131 +163132 POINT(42.483445603236675 -86.83912370775373) bank163132 +163133 POINT(42.560046785878846 -87.3865395681722) bank163133 +163134 POINT(41.61797890899885 -87.82995455879735) bank163134 +163135 POINT(42.76621439263644 -88.58527216066109) bank163135 +163136 POINT(41.0448769706929 -87.7665772725491) bank163136 +163137 POINT(41.72602436330497 -87.25918712986928) bank163137 +163138 POINT(41.25892072547208 -88.39462741429485) bank163138 +163139 POINT(42.10827104934215 -87.64657596070779) bank163139 +163140 POINT(42.11383000111224 -88.00183359048813) bank163140 +163141 POINT(40.894618952968415 -88.28720405325295) bank163141 +163142 POINT(41.79306621115574 -88.38412846748388) bank163142 +163143 POINT(41.87879227109449 -88.41484482102234) bank163143 +163144 POINT(42.63001975585386 -88.26337877950989) bank163144 +163145 POINT(42.14583442424083 -87.35076783704513) bank163145 +163146 POINT(41.63979565303893 -88.46937921961583) bank163146 +163147 POINT(42.483644624075616 -88.01156397420151) bank163147 +163148 POINT(42.86901316140992 -87.81950718462258) bank163148 +163149 POINT(41.933603905512115 -87.51909290732713) bank163149 +163150 POINT(41.78257042294983 -87.32806565937298) bank163150 +163151 POINT(41.358296714019524 -88.30060944141273) bank163151 +163152 POINT(41.603958634634346 -87.51114260457004) bank163152 +163153 POINT(41.92297023982804 -86.85106374141218) bank163153 +163154 POINT(42.1100468438404 -87.19264333896793) bank163154 +163155 POINT(41.92902468454647 -87.05698627709823) bank163155 +163156 POINT(42.66021987664163 -86.7155782669253) bank163156 +163157 POINT(41.690869880739804 -88.30284339455598) bank163157 +163158 POINT(40.93130577836167 -86.81690570378488) bank163158 +163159 POINT(41.20396771352562 -87.08763194160542) bank163159 +163160 POINT(41.457263139582814 -88.37928293730559) bank163160 +163161 POINT(41.472960230807765 -87.19413550752316) bank163161 +163162 POINT(41.64331691967284 -87.75864290303527) bank163162 +163163 POINT(41.124371558959474 -87.85755837834235) bank163163 +163164 POINT(42.794662771333705 -87.58126747913018) bank163164 +163165 POINT(42.81671277648315 -87.27228055682907) bank163165 +163166 POINT(41.92334446703771 -86.80525919404519) bank163166 +163167 POINT(42.19264261619101 -88.33485329199289) bank163167 +163168 POINT(42.09596474873839 -87.59029019416347) bank163168 +163169 POINT(42.121687425110096 -88.50417903628949) bank163169 +163170 POINT(41.45582842581219 -87.08401901804804) bank163170 +163171 POINT(41.28297855535097 -87.57821758494883) bank163171 +163172 POINT(41.00985274043504 -87.52488344065264) bank163172 +163173 POINT(42.19600188843678 -88.13572242939804) bank163173 +163174 POINT(41.78813522572507 -88.16607257864308) bank163174 +163175 POINT(42.36635296048405 -87.66956295696042) bank163175 +163176 POINT(42.710616671908916 -86.66214890453428) bank163176 +163177 POINT(41.7510379970542 -87.21599613703421) bank163177 +163178 POINT(42.386841310167775 -87.42845232539523) bank163178 +163179 POINT(41.87471229378947 -88.39657905234748) bank163179 +163180 POINT(42.195945133315014 -87.85307250037341) bank163180 +163181 POINT(41.622875668451115 -87.74207335604957) bank163181 +163182 POINT(40.93835651273312 -87.54759496951372) bank163182 +163183 POINT(41.267023138026076 -87.69867656938425) bank163183 +163184 POINT(40.95649606759888 -87.76122276340568) bank163184 +163185 POINT(42.643131450260135 -87.63496266963642) bank163185 +163186 POINT(41.22642327741302 -88.13150603774305) bank163186 +163187 POINT(42.69455223085316 -87.57548959504304) bank163187 +163188 POINT(40.99941343182718 -87.07932542689204) bank163188 +163189 POINT(42.150866763731855 -87.21837803852299) bank163189 +163190 POINT(42.23796539888054 -88.30500275190957) bank163190 +163191 POINT(42.00744493429165 -87.22549523677728) bank163191 +163192 POINT(41.73660608482105 -87.99390800112093) bank163192 +163193 POINT(41.05134986906618 -87.40405975981702) bank163193 +163194 POINT(41.10777304169137 -88.16316936761997) bank163194 +163195 POINT(41.44293587627499 -87.9458538608637) bank163195 +163196 POINT(42.69250789839752 -86.78473697405553) bank163196 +163197 POINT(41.08725926856185 -87.4082365579541) bank163197 +163198 POINT(42.351982950949115 -87.27037923447118) bank163198 +163199 POINT(42.490335795906205 -87.99497891892574) bank163199 +163200 POINT(42.08057443298157 -88.29021231405773) bank163200 +163201 POINT(41.22586848505815 -86.75810471541774) bank163201 +163202 POINT(41.539226142163365 -88.17650178868877) bank163202 +163203 POINT(42.080554359716295 -87.14577729910967) bank163203 +163204 POINT(42.34544115125992 -86.95977828332597) bank163204 +163205 POINT(41.81075530682437 -87.1404393357985) bank163205 +163206 POINT(40.887950103473926 -87.34265136195121) bank163206 +163207 POINT(40.90044663915666 -87.63455759141061) bank163207 +163208 POINT(41.63250587608983 -86.90692994461251) bank163208 +163209 POINT(41.068395136312084 -87.00482469958553) bank163209 +163210 POINT(41.82585298675435 -87.8576349169434) bank163210 +163211 POINT(41.425171870992294 -87.7815113749929) bank163211 +163212 POINT(42.20000256090663 -86.77303246000027) bank163212 +163213 POINT(42.20764135940337 -88.32874488922263) bank163213 +163214 POINT(42.8746042128209 -87.995692360677) bank163214 +163215 POINT(41.216073954390914 -86.87039912975614) bank163215 +163216 POINT(42.30583435528771 -86.91371869723625) bank163216 +163217 POINT(42.86778673943562 -88.09139900824071) bank163217 +163218 POINT(41.989370029838824 -86.9681894617159) bank163218 +163219 POINT(41.50560009363768 -88.60452953776645) bank163219 +163220 POINT(41.70981610228539 -88.35518953560717) bank163220 +163221 POINT(41.88191533780821 -88.3210113307741) bank163221 +163222 POINT(42.651940870024134 -87.62421348753175) bank163222 +163223 POINT(41.52904272609229 -87.4789709152003) bank163223 +163224 POINT(41.93512957313548 -87.59887856330515) bank163224 +163225 POINT(42.51405467921311 -87.72066512803917) bank163225 +163226 POINT(41.06727095033834 -87.23035261881866) bank163226 +163227 POINT(42.06816454248796 -88.39462321032809) bank163227 +163228 POINT(40.95849110478181 -87.7211824966667) bank163228 +163229 POINT(42.36134377044667 -87.45154881934678) bank163229 +163230 POINT(42.31932364372333 -88.07972536997922) bank163230 +163231 POINT(42.36065322791905 -88.11682869242208) bank163231 +163232 POINT(41.036875458740475 -86.65540630422211) bank163232 +163233 POINT(41.17514716099074 -87.42497406136744) bank163233 +163234 POINT(42.54635742637662 -88.32033764468798) bank163234 +163235 POINT(42.37680569688203 -88.25699730545384) bank163235 +163236 POINT(41.324227249638454 -88.080657587045) bank163236 +163237 POINT(42.22386778078438 -87.30161887766275) bank163237 +163238 POINT(42.340516243526594 -88.34624786212247) bank163238 +163239 POINT(41.475388503242755 -88.15292842973895) bank163239 +163240 POINT(41.05967372277851 -86.80662830139485) bank163240 +163241 POINT(42.2721392984841 -88.44940279742495) bank163241 +163242 POINT(42.19399829679645 -88.35408759323376) bank163242 +163243 POINT(42.38180565385448 -86.7952917441016) bank163243 +163244 POINT(42.41181007154903 -88.61778913433933) bank163244 +163245 POINT(42.510951295825265 -87.19024778792651) bank163245 +163246 POINT(42.552730032432734 -87.81508366058813) bank163246 +163247 POINT(42.650243037050494 -88.33989001601608) bank163247 +163248 POINT(42.6560640115797 -88.4604339565288) bank163248 +163249 POINT(40.99026948217176 -87.76372690353205) bank163249 +163250 POINT(42.11768986218853 -87.40385741139754) bank163250 +163251 POINT(42.40263241870533 -88.4969687400229) bank163251 +163252 POINT(41.58818250219887 -87.4639013106488) bank163252 +163253 POINT(42.87770399832663 -87.82378588644545) bank163253 +163254 POINT(41.35786012801252 -87.5419210565508) bank163254 +163255 POINT(41.037122303190436 -87.25037353544) bank163255 +163256 POINT(42.414747282679436 -87.24881093588431) bank163256 +163257 POINT(42.77881461978119 -88.16082453383987) bank163257 +163258 POINT(42.569394705432565 -88.03856732241526) bank163258 +163259 POINT(41.48526415008395 -87.91359434001185) bank163259 +163260 POINT(41.56906820389342 -87.24206425442411) bank163260 +163261 POINT(41.92795379802531 -87.60949061606405) bank163261 +163262 POINT(41.83930077712691 -88.43195238255487) bank163262 +163263 POINT(41.46083332254937 -88.30402820702861) bank163263 +163264 POINT(42.54171603968032 -88.41986505413807) bank163264 +163265 POINT(41.45851227313474 -88.09082992117834) bank163265 +163266 POINT(41.06361725870163 -87.40881164127501) bank163266 +163267 POINT(41.53639958192807 -86.89748834340152) bank163267 +163268 POINT(41.80196492565159 -87.15534884730377) bank163268 +163269 POINT(40.99586710416081 -88.12427905715907) bank163269 +163270 POINT(42.7088353267728 -87.82179837059023) bank163270 +163271 POINT(41.46192962520961 -86.66316407974358) bank163271 +163272 POINT(41.23549279612936 -86.82873722276247) bank163272 +163273 POINT(41.57062991237878 -87.38479725478646) bank163273 +163274 POINT(41.39222098654196 -86.73177669226045) bank163274 +163275 POINT(40.994040405819625 -88.60920654760693) bank163275 +163276 POINT(41.36525256772231 -87.54829082522846) bank163276 +163277 POINT(42.36563945184812 -88.24718306736266) bank163277 +163278 POINT(42.24676746394015 -88.57841883007013) bank163278 +163279 POINT(41.91145743606267 -88.33512245518097) bank163279 +163280 POINT(41.41331428727012 -87.29923024113468) bank163280 +163281 POINT(42.81418146847193 -88.2529665105357) bank163281 +163282 POINT(41.612223094119045 -86.90710206096654) bank163282 +163283 POINT(42.34431619405578 -86.8437183041269) bank163283 +163284 POINT(42.73621907123027 -86.71345526503083) bank163284 +163285 POINT(41.949865337250806 -88.54098706313584) bank163285 +163286 POINT(42.09257754554472 -88.00043392651202) bank163286 +163287 POINT(42.01092444806434 -86.82991257256755) bank163287 +163288 POINT(41.41187903407512 -87.78066551660676) bank163288 +163289 POINT(41.8863008164231 -86.89977950540792) bank163289 +163290 POINT(41.6092654410651 -87.16257938763793) bank163290 +163291 POINT(41.04310569352649 -87.59100217808026) bank163291 +163292 POINT(41.256513537120384 -87.51046314352092) bank163292 +163293 POINT(42.18929685276753 -87.06780063398699) bank163293 +163294 POINT(42.87524376832834 -86.80946863330252) bank163294 +163295 POINT(41.63197935606263 -86.75665763005281) bank163295 +163296 POINT(41.74358773649786 -87.77023071814672) bank163296 +163297 POINT(41.920910870496215 -86.99350760904184) bank163297 +163298 POINT(41.61131089174988 -88.32091631374851) bank163298 +163299 POINT(42.770966461429204 -88.51160563416566) bank163299 +163300 POINT(41.029481306560925 -87.90343364353058) bank163300 +163301 POINT(41.92090120899503 -87.7480553051676) bank163301 +163302 POINT(42.57523801186109 -88.59889910420887) bank163302 +163303 POINT(41.652603561367364 -87.51205309579525) bank163303 +163304 POINT(41.865940094118514 -87.6169365065695) bank163304 +163305 POINT(41.854048535938766 -87.50386786842617) bank163305 +163306 POINT(41.621241869852895 -88.15322934555924) bank163306 +163307 POINT(42.77305143554837 -87.38355944668227) bank163307 +163308 POINT(42.32865439514792 -88.39139074093053) bank163308 +163309 POINT(41.48992491999004 -87.6446127550092) bank163309 +163310 POINT(41.66888885807604 -86.78405706160744) bank163310 +163311 POINT(41.612667003621524 -87.7354118261967) bank163311 +163312 POINT(42.64850586714651 -88.59218161783612) bank163312 +163313 POINT(42.616848905403785 -87.62209961991628) bank163313 +163314 POINT(41.14550989702043 -87.02243370142467) bank163314 +163315 POINT(42.46979008004795 -87.46850540421654) bank163315 +163316 POINT(41.80953171034252 -88.50435764916679) bank163316 +163317 POINT(41.49588743881907 -87.02755650579492) bank163317 +163318 POINT(42.13214265333162 -88.36321054356078) bank163318 +163319 POINT(40.956818479877285 -88.5499356228307) bank163319 +163320 POINT(42.516132678039845 -87.3226446507064) bank163320 +163321 POINT(40.97718037664405 -87.30908706881529) bank163321 +163322 POINT(41.35601916147842 -86.8616493699648) bank163322 +163323 POINT(41.770035824059924 -88.08480741530666) bank163323 +163324 POINT(40.927738475859044 -86.94862641382454) bank163324 +163325 POINT(41.655280031230895 -86.85408391347374) bank163325 +163326 POINT(41.48235860381871 -87.11214012089694) bank163326 +163327 POINT(41.948399049853734 -87.29965262536847) bank163327 +163328 POINT(40.88142560876349 -87.75462816717364) bank163328 +163329 POINT(41.1880787162801 -86.669994439954) bank163329 +163330 POINT(41.114231364738174 -87.35716580862018) bank163330 +163331 POINT(42.555500763509734 -88.32403231523024) bank163331 +163332 POINT(41.9174341679381 -87.44838845054568) bank163332 +163333 POINT(41.741199897176784 -87.66632587268793) bank163333 +163334 POINT(40.900323474987715 -88.12146204848399) bank163334 +163335 POINT(41.98982704254441 -87.08334084791092) bank163335 +163336 POINT(41.66506243296983 -88.48480648696649) bank163336 +163337 POINT(42.02284520076883 -86.74536418039662) bank163337 +163338 POINT(41.98327667024914 -87.91610756772262) bank163338 +163339 POINT(42.392276029010716 -86.87901356835198) bank163339 +163340 POINT(41.07311935940558 -87.1278766264056) bank163340 +163341 POINT(42.00976838858158 -88.45951180615306) bank163341 +163342 POINT(41.883381628236094 -86.67869749612295) bank163342 +163343 POINT(42.02678351610503 -87.29467640506118) bank163343 +163344 POINT(42.15111148183205 -87.28899729618999) bank163344 +163345 POINT(41.91611565322446 -87.83303666292362) bank163345 +163346 POINT(41.62335154345287 -88.35201277941087) bank163346 +163347 POINT(41.89107987563061 -87.93505182593464) bank163347 +163348 POINT(41.868842996237305 -88.39053739179862) bank163348 +163349 POINT(41.71821929803515 -86.80958014551916) bank163349 +163350 POINT(41.6893297979532 -87.86149271905357) bank163350 +163351 POINT(40.886274609485156 -88.21915184695418) bank163351 +163352 POINT(41.126277372371675 -86.65306538327137) bank163352 +163353 POINT(42.56144127913817 -88.05617446240666) bank163353 +163354 POINT(42.32137236902189 -88.56321387964245) bank163354 +163355 POINT(41.827145674891774 -87.90583964455263) bank163355 +163356 POINT(42.307114510091125 -88.224945494019) bank163356 +163357 POINT(41.99135578997681 -88.58486218652759) bank163357 +163358 POINT(40.97019719024505 -88.45940172440812) bank163358 +163359 POINT(42.04158455642292 -87.14806189997007) bank163359 +163360 POINT(42.527277092437636 -86.66203439884639) bank163360 +163361 POINT(42.44201146875915 -88.04156493251836) bank163361 +163362 POINT(41.96232849688145 -87.07106437713176) bank163362 +163363 POINT(42.324623984486884 -86.65160639194049) bank163363 +163364 POINT(42.06464425716761 -87.40725800541362) bank163364 +163365 POINT(42.31453406722668 -87.64201436657427) bank163365 +163366 POINT(42.34067391749999 -88.54755739935054) bank163366 +163367 POINT(42.64227665406426 -88.34788870488224) bank163367 +163368 POINT(42.06103528629987 -88.3486561169615) bank163368 +163369 POINT(40.97475811198689 -86.72416403094606) bank163369 +163370 POINT(42.87400821654391 -87.31627689094076) bank163370 +163371 POINT(40.915848705706665 -87.87973560174571) bank163371 +163372 POINT(41.88521139638782 -88.37483919560464) bank163372 +163373 POINT(42.66853938161748 -87.25183430904839) bank163373 +163374 POINT(42.34602092812496 -87.87492409470724) bank163374 +163375 POINT(41.18684264621126 -88.43235107609341) bank163375 +163376 POINT(42.621765607176904 -88.02525929267225) bank163376 +163377 POINT(41.2696293173172 -87.33994034997203) bank163377 +163378 POINT(41.33592289297628 -86.67831229922766) bank163378 +163379 POINT(42.34943112342368 -88.55247281878528) bank163379 +163380 POINT(40.99019870571935 -86.78359361114703) bank163380 +163381 POINT(40.933569294946935 -87.09538110979321) bank163381 +163382 POINT(42.31744885000059 -87.39651035630749) bank163382 +163383 POINT(41.390792303548345 -88.20286877608154) bank163383 +163384 POINT(41.13075766897335 -86.8714544049463) bank163384 +163385 POINT(42.45892678353482 -87.9038964905028) bank163385 +163386 POINT(40.99155718287198 -86.94689300558062) bank163386 +163387 POINT(42.065044040262116 -88.20447547638311) bank163387 +163388 POINT(42.52578592590882 -88.15561689646073) bank163388 +163389 POINT(42.53508360824026 -87.27468184740103) bank163389 +163390 POINT(40.911904643505764 -87.13992286800904) bank163390 +163391 POINT(42.84141494716082 -87.37083464222104) bank163391 +163392 POINT(41.19341970356471 -87.85610032512682) bank163392 +163393 POINT(41.832835001003886 -88.14533082579109) bank163393 +163394 POINT(41.43695470268496 -86.64512565979062) bank163394 +163395 POINT(42.830606025026164 -87.0558908405756) bank163395 +163396 POINT(41.39312850123324 -87.04048822232139) bank163396 +163397 POINT(41.51141011971042 -88.24024267820917) bank163397 +163398 POINT(42.34604278327319 -86.66624085218207) bank163398 +163399 POINT(41.264250749196805 -86.69540382038043) bank163399 +163400 POINT(41.16812638050896 -86.9147457719051) bank163400 +163401 POINT(42.57140273972507 -86.85374422513225) bank163401 +163402 POINT(41.600628740149276 -88.43020223944453) bank163402 +163403 POINT(42.12169350396383 -87.07559367825397) bank163403 +163404 POINT(41.83221744712774 -86.65475539761398) bank163404 +163405 POINT(41.707747252205735 -87.46398914550484) bank163405 +163406 POINT(41.185513381627345 -87.39992738446057) bank163406 +163407 POINT(40.99985918804881 -87.6839845605879) bank163407 +163408 POINT(42.15208598250144 -86.94882903329494) bank163408 +163409 POINT(41.04785069234773 -88.32484646687027) bank163409 +163410 POINT(42.738005498547636 -87.60502115474104) bank163410 +163411 POINT(41.45425777618409 -88.25942708881786) bank163411 +163412 POINT(41.15597667028269 -87.58437557292122) bank163412 +163413 POINT(41.27129331611206 -88.4842435888709) bank163413 +163414 POINT(42.11141806639043 -87.75792255722025) bank163414 +163415 POINT(40.926037884983 -88.35764816164328) bank163415 +163416 POINT(41.040427948024885 -87.45282332705874) bank163416 +163417 POINT(41.343939826966086 -87.17643447000526) bank163417 +163418 POINT(41.93591945006761 -87.10070837968242) bank163418 +163419 POINT(41.84732858407823 -86.83838315540581) bank163419 +163420 POINT(42.008991178198855 -88.49718477187452) bank163420 +163421 POINT(42.76022821668196 -87.0219522676682) bank163421 +163422 POINT(42.07527847095827 -88.49343847628833) bank163422 +163423 POINT(42.26738313116979 -87.20157910799561) bank163423 +163424 POINT(41.406884700873896 -87.00921559851395) bank163424 +163425 POINT(41.56072077098292 -87.83849906494733) bank163425 +163426 POINT(42.862225661965205 -86.6509755420426) bank163426 +163427 POINT(41.12284953792122 -88.46016181355373) bank163427 +163428 POINT(42.090406699396645 -86.97457306539204) bank163428 +163429 POINT(41.7535422926441 -88.11701694972615) bank163429 +163430 POINT(41.5726883248612 -87.32183945596518) bank163430 +163431 POINT(42.45782065970823 -87.59351955624624) bank163431 +163432 POINT(42.447622584169956 -88.02818566339805) bank163432 +163433 POINT(42.391457525048835 -88.05538592426446) bank163433 +163434 POINT(42.53185763658097 -87.82756235456989) bank163434 +163435 POINT(42.82939069055429 -86.91050904714396) bank163435 +163436 POINT(41.50755197288101 -87.00949720928324) bank163436 +163437 POINT(41.35192895688486 -88.05827949636685) bank163437 +163438 POINT(42.49475170092885 -86.849839036788) bank163438 +163439 POINT(42.16419196932477 -87.68770429607645) bank163439 +163440 POINT(41.20551879363639 -87.82936247590189) bank163440 +163441 POINT(42.13561239046631 -88.30409408062495) bank163441 +163442 POINT(41.278964837308436 -87.83775217710154) bank163442 +163443 POINT(42.342757258013535 -88.54684893920006) bank163443 +163444 POINT(41.06056395806875 -87.98823693171788) bank163444 +163445 POINT(41.570379928202335 -87.42891310274156) bank163445 +163446 POINT(41.60353475777696 -88.07218576655575) bank163446 +163447 POINT(41.9883881193601 -88.34108672740038) bank163447 +163448 POINT(41.32624793553035 -86.72215450610169) bank163448 +163449 POINT(41.61049590474738 -87.69516813946952) bank163449 +163450 POINT(42.683865345351016 -87.60924606215441) bank163450 +163451 POINT(42.504433866459266 -88.08360399279026) bank163451 +163452 POINT(42.86007016717467 -86.83924744458396) bank163452 +163453 POINT(42.28377291723166 -87.7501745374417) bank163453 +163454 POINT(41.459456188122914 -86.70045676173858) bank163454 +163455 POINT(42.23429588784453 -87.01227148502059) bank163455 +163456 POINT(41.59619224008548 -87.76323347204598) bank163456 +163457 POINT(42.692164493228375 -87.7550671195258) bank163457 +163458 POINT(40.96093882358341 -87.53600553773795) bank163458 +163459 POINT(42.53327260602039 -86.65564507963701) bank163459 +163460 POINT(42.7258189859198 -87.40746016691516) bank163460 +163461 POINT(41.72590512248052 -86.9234573874826) bank163461 +163462 POINT(42.51980953968692 -88.15631670894064) bank163462 +163463 POINT(41.196678997564405 -88.2404242392487) bank163463 +163464 POINT(40.92284392608167 -88.25902404119441) bank163464 +163465 POINT(41.24491124811059 -88.5435559680387) bank163465 +163466 POINT(41.16274179858749 -88.419573441029) bank163466 +163467 POINT(42.819987928702446 -87.81703456278605) bank163467 +163468 POINT(40.9359671445141 -88.54252329946343) bank163468 +163469 POINT(42.53830005428486 -87.28529485463608) bank163469 +163470 POINT(41.87384941926547 -87.6194164585775) bank163470 +163471 POINT(41.480301692501065 -87.18144626087582) bank163471 +163472 POINT(40.91504726695249 -87.4017339816571) bank163472 +163473 POINT(40.953915460374205 -87.30578572776668) bank163473 +163474 POINT(41.07562109133877 -88.52003993943785) bank163474 +163475 POINT(42.298025345882344 -88.17311390833156) bank163475 +163476 POINT(42.08014055724085 -86.7543959559371) bank163476 +163477 POINT(41.919059054864164 -86.71865778918301) bank163477 +163478 POINT(41.11548250831222 -87.30793156671633) bank163478 +163479 POINT(41.78945046602455 -86.83941009550395) bank163479 +163480 POINT(42.50239043837319 -88.39522443964343) bank163480 +163481 POINT(41.953730188736344 -88.25800343385454) bank163481 +163482 POINT(41.75973151922492 -88.60168331962899) bank163482 +163483 POINT(42.46495217691259 -87.12994514940999) bank163483 +163484 POINT(42.1135241617011 -88.57367920671132) bank163484 +163485 POINT(41.49559689533258 -87.50663105251618) bank163485 +163486 POINT(41.21340433457175 -86.74428496378697) bank163486 +163487 POINT(41.6599092337464 -87.22814386910882) bank163487 +163488 POINT(41.59467365285898 -87.16957920170007) bank163488 +163489 POINT(41.794451301956926 -86.82289580641888) bank163489 +163490 POINT(41.04997277711551 -88.49883783625474) bank163490 +163491 POINT(42.077341759919115 -87.14247644341731) bank163491 +163492 POINT(42.31806135854522 -87.60201449485443) bank163492 +163493 POINT(40.98443911060836 -86.87678660799877) bank163493 +163494 POINT(42.41272292983989 -88.24650565456952) bank163494 +163495 POINT(41.2417549013603 -88.08873031026627) bank163495 +163496 POINT(42.283155349345535 -86.8266252962423) bank163496 +163497 POINT(41.460138935852086 -88.19513296611159) bank163497 +163498 POINT(42.4357898285668 -87.95446631923625) bank163498 +163499 POINT(41.92659003802145 -87.28179305725624) bank163499 +163500 POINT(41.959820875644915 -87.09210777227548) bank163500 +163501 POINT(41.40202114931525 -87.88170991687065) bank163501 +163502 POINT(41.322662825212916 -87.56762646305775) bank163502 +163503 POINT(42.03898355721873 -88.05468612283997) bank163503 +163504 POINT(41.34448768050437 -87.30359017131202) bank163504 +163505 POINT(42.21500024387264 -87.7844448911817) bank163505 +163506 POINT(41.19278034452338 -86.69068934538114) bank163506 +163507 POINT(42.3086888534389 -87.17818449791642) bank163507 +163508 POINT(42.54552938116384 -88.49631723812718) bank163508 +163509 POINT(42.436085665379636 -87.2513636255282) bank163509 +163510 POINT(42.41830275229472 -87.89245307850436) bank163510 +163511 POINT(42.8142981227394 -87.3526503065061) bank163511 +163512 POINT(41.618400294183424 -87.33812439088634) bank163512 +163513 POINT(41.55608145283092 -87.40764431972602) bank163513 +163514 POINT(41.3556377462048 -87.14570037313715) bank163514 +163515 POINT(42.29908311990888 -88.02690401961193) bank163515 +163516 POINT(41.64444178354537 -88.50076626903596) bank163516 +163517 POINT(42.72964622865398 -88.24112983130696) bank163517 +163518 POINT(41.52970341576264 -88.28022491375596) bank163518 +163519 POINT(40.8852072344078 -87.65135914349072) bank163519 +163520 POINT(41.06554223422347 -87.85723475746) bank163520 +163521 POINT(42.13635772919483 -87.16506843026978) bank163521 +163522 POINT(41.4254910294915 -87.48748018610831) bank163522 +163523 POINT(41.81767100882891 -88.61342478618167) bank163523 +163524 POINT(41.43080063193746 -87.88528210339355) bank163524 +163525 POINT(42.761019070037236 -87.88180023674903) bank163525 +163526 POINT(42.290162790709054 -87.93360190107163) bank163526 +163527 POINT(42.19451172315607 -87.8081947982725) bank163527 +163528 POINT(42.637208850037624 -87.50762498560056) bank163528 +163529 POINT(42.45340888288282 -88.13180737082408) bank163529 +163530 POINT(42.85254196102735 -87.74349802494055) bank163530 +163531 POINT(41.99661959441925 -87.00023586603362) bank163531 +163532 POINT(42.14286429741609 -87.68325427102789) bank163532 +163533 POINT(42.5456090887526 -87.690287379053) bank163533 +163534 POINT(41.4466120352018 -88.23512728577226) bank163534 +163535 POINT(41.48105333662216 -87.00985042108735) bank163535 +163536 POINT(42.336420226661836 -86.79352813972864) bank163536 +163537 POINT(41.86587594806966 -87.09825514772412) bank163537 +163538 POINT(42.66358488932597 -87.56083254430582) bank163538 +163539 POINT(41.38817888531174 -87.54105951413604) bank163539 +163540 POINT(41.276548138402475 -88.07808547223003) bank163540 +163541 POINT(41.559604942403745 -87.9192854161748) bank163541 +163542 POINT(41.855866785322874 -88.29633594966674) bank163542 +163543 POINT(42.53447963496523 -87.9331712837877) bank163543 +163544 POINT(42.3952410246664 -88.50556826516117) bank163544 +163545 POINT(41.99018035436576 -88.34108217083133) bank163545 +163546 POINT(40.995215507843334 -88.20845207128464) bank163546 +163547 POINT(41.84368384536758 -88.36342385736472) bank163547 +163548 POINT(41.81843808253598 -87.25701272867033) bank163548 +163549 POINT(42.10573085128506 -88.15533693504705) bank163549 +163550 POINT(42.65755773547811 -87.79961987395414) bank163550 +163551 POINT(41.88734337580497 -87.48144596824369) bank163551 +163552 POINT(41.25009528203288 -86.73302249729755) bank163552 +163553 POINT(42.2789156855713 -88.18778842038975) bank163553 +163554 POINT(42.53948426116836 -87.7925029358057) bank163554 +163555 POINT(41.27443737690781 -87.91205063821171) bank163555 +163556 POINT(42.664441343146784 -86.88597518629405) bank163556 +163557 POINT(41.86091768472901 -88.09963374688317) bank163557 +163558 POINT(41.50613015753439 -87.52214346823285) bank163558 +163559 POINT(42.64525426217052 -88.54450760334849) bank163559 +163560 POINT(41.39542785998591 -88.05462911104065) bank163560 +163561 POINT(41.88413933455478 -86.80682075375988) bank163561 +163562 POINT(42.0543650218167 -87.58264431010014) bank163562 +163563 POINT(42.195391703643146 -86.81363934658572) bank163563 +163564 POINT(42.870793274044985 -86.73529199088247) bank163564 +163565 POINT(41.375294484206286 -86.82451398119613) bank163565 +163566 POINT(42.39718816394109 -87.16702490913627) bank163566 +163567 POINT(42.30030897371033 -88.24034939650123) bank163567 +163568 POINT(41.62212113252851 -88.08116744335655) bank163568 +163569 POINT(41.87488292557205 -86.6890700953352) bank163569 +163570 POINT(41.127476354677015 -88.52860282605982) bank163570 +163571 POINT(42.14891101976333 -87.03440494955768) bank163571 +163572 POINT(41.63011533069371 -87.29542340954266) bank163572 +163573 POINT(42.79997674305061 -88.35800050893883) bank163573 +163574 POINT(41.207121134478 -86.98941052292778) bank163574 +163575 POINT(41.51402931380533 -86.83865100302133) bank163575 +163576 POINT(42.75814724644404 -87.66158812128283) bank163576 +163577 POINT(42.78310168359217 -87.45319956208604) bank163577 +163578 POINT(41.60527460881264 -88.31559340318151) bank163578 +163579 POINT(42.760453519819464 -86.89539088746582) bank163579 +163580 POINT(42.50290909417264 -88.03862701791141) bank163580 +163581 POINT(41.25771765858451 -87.4233723751864) bank163581 +163582 POINT(41.880334217008496 -87.01379466198652) bank163582 +163583 POINT(42.116458460928065 -88.48862793602896) bank163583 +163584 POINT(41.695997418446126 -87.33183003309173) bank163584 +163585 POINT(42.50398159057314 -86.81606738011764) bank163585 +163586 POINT(41.26258886471316 -88.11770606137233) bank163586 +163587 POINT(40.91745493175876 -87.93973406885044) bank163587 +163588 POINT(40.97111391704417 -87.54535242854072) bank163588 +163589 POINT(41.30348748506027 -88.62432911924317) bank163589 +163590 POINT(41.016252492283336 -86.85663309130962) bank163590 +163591 POINT(41.539203874552136 -86.80910310195038) bank163591 +163592 POINT(42.42062381889074 -88.1665321438437) bank163592 +163593 POINT(42.544824284512984 -88.4589409874513) bank163593 +163594 POINT(41.966165370676386 -86.64859146368137) bank163594 +163595 POINT(40.91992594627792 -87.07008215255927) bank163595 +163596 POINT(40.917418270856295 -87.44478562910463) bank163596 +163597 POINT(42.694176676966926 -87.62905570145212) bank163597 +163598 POINT(42.27689930887166 -86.96383486092674) bank163598 +163599 POINT(42.44739762199671 -86.6305150170964) bank163599 +163600 POINT(41.44530674341909 -87.16185610020177) bank163600 +163601 POINT(41.35323192288174 -87.11771614671808) bank163601 +163602 POINT(40.925045864981726 -88.56244491964698) bank163602 +163603 POINT(42.492018470299605 -87.73094812767386) bank163603 +163604 POINT(41.90660828600144 -86.98595413929809) bank163604 +163605 POINT(41.59533334637495 -87.25825713312241) bank163605 +163606 POINT(42.53412983563623 -88.16556563515545) bank163606 +163607 POINT(41.03993708022255 -87.7130774038126) bank163607 +163608 POINT(41.29096643254009 -86.85854470315459) bank163608 +163609 POINT(41.003384639245084 -86.9798152964999) bank163609 +163610 POINT(41.455681567729975 -87.88703108639574) bank163610 +163611 POINT(42.176332769853374 -87.68079401255464) bank163611 +163612 POINT(41.231920612538985 -86.83808678155795) bank163612 +163613 POINT(42.85258394568117 -87.05304974060606) bank163613 +163614 POINT(41.63476127851181 -88.26150153204671) bank163614 +163615 POINT(42.241688707753276 -88.29992771887117) bank163615 +163616 POINT(42.24336351020707 -87.26964213266217) bank163616 +163617 POINT(42.7866119718713 -86.71137644711025) bank163617 +163618 POINT(41.94805970243494 -86.77398889682982) bank163618 +163619 POINT(40.93683017491867 -88.40338137268634) bank163619 +163620 POINT(42.24332731659587 -87.36063475574834) bank163620 +163621 POINT(42.716857876179354 -87.78562816494387) bank163621 +163622 POINT(41.46187037360061 -87.58371666054447) bank163622 +163623 POINT(41.48032617441504 -87.92442839595121) bank163623 +163624 POINT(42.07066235978274 -88.17372511907223) bank163624 +163625 POINT(41.97880612865041 -86.71953331272049) bank163625 +163626 POINT(42.02546634785535 -88.23977073289875) bank163626 +163627 POINT(42.44395190510881 -87.3012697636446) bank163627 +163628 POINT(42.47058216825445 -86.86600135294823) bank163628 +163629 POINT(41.34010476075259 -87.36486743191944) bank163629 +163630 POINT(41.9059387045035 -88.62458813865256) bank163630 +163631 POINT(41.17160511237385 -86.70480930456272) bank163631 +163632 POINT(41.58154102823614 -86.9741988007506) bank163632 +163633 POINT(41.19016776366454 -87.1357851256071) bank163633 +163634 POINT(41.08829353573238 -86.90671917409573) bank163634 +163635 POINT(42.47660436095244 -87.0692790940047) bank163635 +163636 POINT(42.27342358670886 -87.92419593640737) bank163636 +163637 POINT(42.009938874365524 -88.07511465930106) bank163637 +163638 POINT(41.17671149814608 -88.22712310550023) bank163638 +163639 POINT(42.09805135319992 -87.66063941103762) bank163639 +163640 POINT(41.14252583448614 -86.75720580870966) bank163640 +163641 POINT(41.34964904117617 -86.93166209091387) bank163641 +163642 POINT(41.94328846883073 -88.40576835173745) bank163642 +163643 POINT(42.13586774403851 -86.99688926289986) bank163643 +163644 POINT(41.29177695789157 -88.46438577221369) bank163644 +163645 POINT(42.670321304669564 -87.9578570402597) bank163645 +163646 POINT(42.19784839810864 -86.93609891019882) bank163646 +163647 POINT(41.38874054480913 -87.60421113290033) bank163647 +163648 POINT(42.78350619430295 -88.33031396931489) bank163648 +163649 POINT(42.623294806194146 -87.67662853738967) bank163649 +163650 POINT(41.37075538431876 -88.46817033338746) bank163650 +163651 POINT(41.64132770093954 -87.99762247428438) bank163651 +163652 POINT(41.33272951349598 -87.78612988981442) bank163652 +163653 POINT(42.296099352114055 -87.87833993347553) bank163653 +163654 POINT(40.911430925910174 -87.57329910055653) bank163654 +163655 POINT(41.67004449335292 -87.07585114478972) bank163655 +163656 POINT(42.34177991052174 -87.04100843952592) bank163656 +163657 POINT(42.34344837679053 -87.51714565650887) bank163657 +163658 POINT(41.25086883190845 -87.97256292978159) bank163658 +163659 POINT(40.89248419417116 -88.2641121915809) bank163659 +163660 POINT(42.38808732773409 -87.9866874857788) bank163660 +163661 POINT(40.91532895924791 -88.20930890423767) bank163661 +163662 POINT(42.48103580697851 -87.78783436721177) bank163662 +163663 POINT(40.952438993332706 -87.95255523683231) bank163663 +163664 POINT(41.48419073966297 -88.53320625393611) bank163664 +163665 POINT(42.081551630963155 -87.98279421671558) bank163665 +163666 POINT(42.18776111206491 -88.39800849397459) bank163666 +163667 POINT(42.32399448319476 -86.64325558122397) bank163667 +163668 POINT(42.057991241524434 -87.07028200084187) bank163668 +163669 POINT(42.665620096398655 -86.98873468643247) bank163669 +163670 POINT(42.07124169864945 -88.23275995942934) bank163670 +163671 POINT(41.89706526785987 -86.72312496247483) bank163671 +163672 POINT(40.98999733465317 -87.20802154776631) bank163672 +163673 POINT(41.281918568829575 -87.1085189411519) bank163673 +163674 POINT(41.13629574390877 -87.31502595266304) bank163674 +163675 POINT(41.052511142498155 -86.78936416818614) bank163675 +163676 POINT(41.068235351093634 -87.69219416235468) bank163676 +163677 POINT(41.530976336845114 -86.63042096101722) bank163677 +163678 POINT(42.49042056668239 -87.70573630115327) bank163678 +163679 POINT(41.10269551740891 -87.97402052458368) bank163679 +163680 POINT(42.691691274097714 -87.13969118935444) bank163680 +163681 POINT(42.15860266819805 -88.61715411819398) bank163681 +163682 POINT(42.67792303006908 -88.49871418675114) bank163682 +163683 POINT(42.595570289806204 -88.24411516053271) bank163683 +163684 POINT(41.66510699332574 -87.24632579484069) bank163684 +163685 POINT(42.050803487207055 -87.92154418151266) bank163685 +163686 POINT(42.267036192855215 -87.52623395977248) bank163686 +163687 POINT(41.96297571097474 -88.55496043712998) bank163687 +163688 POINT(42.34207889797699 -86.86524557237816) bank163688 +163689 POINT(40.95461218372937 -87.32060261530019) bank163689 +163690 POINT(42.47447298905124 -88.55469533779223) bank163690 +163691 POINT(42.27320787467965 -87.48027492268315) bank163691 +163692 POINT(42.76049472524482 -87.59796246832231) bank163692 +163693 POINT(41.24148339803506 -88.10931683772247) bank163693 +163694 POINT(41.30580706652848 -87.51831587794594) bank163694 +163695 POINT(41.14684976277426 -87.83045090315852) bank163695 +163696 POINT(41.70441629284295 -88.16862571021846) bank163696 +163697 POINT(42.73885882951077 -87.51079417101822) bank163697 +163698 POINT(41.55216614728785 -88.37823132249838) bank163698 +163699 POINT(42.61700554032966 -87.83521472359128) bank163699 +163700 POINT(41.69383011335962 -87.11676450986502) bank163700 +163701 POINT(41.72127487002966 -86.92723538737057) bank163701 +163702 POINT(42.420121176684354 -86.69899746586813) bank163702 +163703 POINT(42.51015187053359 -88.13647859215202) bank163703 +163704 POINT(41.02869657325867 -87.30698910213039) bank163704 +163705 POINT(42.10810579306644 -88.08115869331843) bank163705 +163706 POINT(41.6072072783845 -87.09240641912916) bank163706 +163707 POINT(42.793068453205485 -87.15999768182611) bank163707 +163708 POINT(42.245226811237494 -88.06044806523217) bank163708 +163709 POINT(41.70048732847168 -87.85645273271305) bank163709 +163710 POINT(41.82441441199897 -87.07232242682741) bank163710 +163711 POINT(42.07300602846098 -87.36814292196917) bank163711 +163712 POINT(41.31422836223264 -87.12949089497728) bank163712 +163713 POINT(41.769865097616744 -87.54228440670595) bank163713 +163714 POINT(41.190225906084876 -87.35322802873486) bank163714 +163715 POINT(41.51521914116062 -87.7471122852978) bank163715 +163716 POINT(42.80944107545753 -88.27789593608408) bank163716 +163717 POINT(42.33702156482296 -88.43140204838842) bank163717 +163718 POINT(42.6763261043288 -87.07160736162093) bank163718 +163719 POINT(42.6133124543911 -87.04607735395787) bank163719 +163720 POINT(41.981805909753724 -86.95322901618661) bank163720 +163721 POINT(41.44512545833508 -87.50372685137337) bank163721 +163722 POINT(41.392556372116 -88.600114120217) bank163722 +163723 POINT(41.07316389530749 -86.84050762508159) bank163723 +163724 POINT(41.05142063646312 -88.3654015601909) bank163724 +163725 POINT(41.77654372494934 -87.15959689898003) bank163725 +163726 POINT(42.37579478977752 -86.71076933138103) bank163726 +163727 POINT(41.231781874671775 -88.04967417439241) bank163727 +163728 POINT(42.84245008194739 -87.99752548584189) bank163728 +163729 POINT(42.09227351976098 -86.92922744909053) bank163729 +163730 POINT(42.58609873240287 -86.6471676399827) bank163730 +163731 POINT(41.8073703441199 -88.10211980142509) bank163731 +163732 POINT(41.77330747269281 -87.75536342684501) bank163732 +163733 POINT(41.30413879952771 -87.82564088046875) bank163733 +163734 POINT(42.08036363418076 -86.89205626168766) bank163734 +163735 POINT(41.87712214638104 -86.84516026514218) bank163735 +163736 POINT(41.23733630921701 -88.56555503069865) bank163736 +163737 POINT(42.01691536572689 -87.68362138432575) bank163737 +163738 POINT(41.46287526465568 -88.24420628942869) bank163738 +163739 POINT(42.825235668905336 -87.21981710051429) bank163739 +163740 POINT(41.794251685366035 -87.46764452394952) bank163740 +163741 POINT(41.81997418101779 -87.48995521478938) bank163741 +163742 POINT(41.43036426489722 -87.97694554649385) bank163742 +163743 POINT(42.04525200052509 -88.01345924440434) bank163743 +163744 POINT(41.891057099370535 -87.57218997094832) bank163744 +163745 POINT(41.56716296529828 -87.61712061381493) bank163745 +163746 POINT(41.67856749734936 -87.9388366954847) bank163746 +163747 POINT(41.97862051130734 -88.01453091723438) bank163747 +163748 POINT(42.554658518792806 -88.48051589856064) bank163748 +163749 POINT(41.74772524317071 -88.5182727266086) bank163749 +163750 POINT(41.31047417455976 -87.13728607308462) bank163750 +163751 POINT(41.188884824610334 -87.42877328662067) bank163751 +163752 POINT(42.66577190919338 -86.99174599254131) bank163752 +163753 POINT(41.65976557659155 -87.3860494364133) bank163753 +163754 POINT(41.69973032313993 -87.86028207093135) bank163754 +163755 POINT(41.327036193922325 -88.56920383438847) bank163755 +163756 POINT(42.15900845007017 -88.32714084484763) bank163756 +163757 POINT(41.41256643893772 -88.50766098605315) bank163757 +163758 POINT(42.31283884525377 -88.2863761576959) bank163758 +163759 POINT(41.381694129024254 -87.5217618935383) bank163759 +163760 POINT(41.61921611721677 -87.21101479741449) bank163760 +163761 POINT(42.301880086118445 -87.79167576598233) bank163761 +163762 POINT(41.570603486818136 -86.99841727443965) bank163762 +163763 POINT(41.47602167103104 -88.29813272429527) bank163763 +163764 POINT(40.91372818442894 -87.27969287416117) bank163764 +163765 POINT(41.59003052981702 -88.2078945127261) bank163765 +163766 POINT(42.6298893741914 -87.79736855232844) bank163766 +163767 POINT(42.17813768568951 -87.08055208865592) bank163767 +163768 POINT(41.0937588739814 -88.38499618057065) bank163768 +163769 POINT(41.296845356546854 -87.0699436761578) bank163769 +163770 POINT(42.29121162594077 -88.31252892289511) bank163770 +163771 POINT(41.76008337502226 -86.71870511474008) bank163771 +163772 POINT(42.34248787789524 -88.31921853444868) bank163772 +163773 POINT(41.30051282088304 -87.54326985712038) bank163773 +163774 POINT(41.07982130276266 -87.31018133223036) bank163774 +163775 POINT(41.54232803004876 -88.14211579892013) bank163775 +163776 POINT(40.903955159607804 -88.02447401392078) bank163776 +163777 POINT(42.814839023386405 -87.93026297234891) bank163777 +163778 POINT(42.00316581843332 -86.72826166406817) bank163778 +163779 POINT(41.508246811367236 -87.63514796760805) bank163779 +163780 POINT(42.4021964498754 -87.15256520359894) bank163780 +163781 POINT(41.87392141250413 -87.22683681229152) bank163781 +163782 POINT(42.0677137974969 -88.19484193204248) bank163782 +163783 POINT(41.10957283833664 -87.22326913954379) bank163783 +163784 POINT(42.65536759699775 -87.98210102014491) bank163784 +163785 POINT(42.855981015429705 -87.0159069604662) bank163785 +163786 POINT(40.88285424154614 -86.90351480766246) bank163786 +163787 POINT(42.51882848670197 -88.23771565839468) bank163787 +163788 POINT(42.41060962942275 -88.24446760756126) bank163788 +163789 POINT(41.36758180657736 -86.72430497708307) bank163789 +163790 POINT(41.920310430818944 -88.2030045130543) bank163790 +163791 POINT(41.4328747243592 -88.45801546933133) bank163791 +163792 POINT(41.1113341970646 -86.74164983463743) bank163792 +163793 POINT(42.297090268021165 -86.63597120875095) bank163793 +163794 POINT(40.93477284430368 -88.56887637983407) bank163794 +163795 POINT(41.237030252641645 -86.69975384879227) bank163795 +163796 POINT(42.7030332021879 -86.893507888713) bank163796 +163797 POINT(40.98066476622119 -88.27537906897798) bank163797 +163798 POINT(42.113300367115386 -88.28033521644741) bank163798 +163799 POINT(41.762046657953384 -88.1799513268404) bank163799 +163800 POINT(42.67114550397147 -87.14697899704191) bank163800 +163801 POINT(42.82069027650317 -87.04082298984758) bank163801 +163802 POINT(41.39447295766253 -87.33294861080478) bank163802 +163803 POINT(41.94417439153899 -86.95723520761682) bank163803 +163804 POINT(42.811202114335885 -87.95424015883498) bank163804 +163805 POINT(41.4722537246329 -87.55739728817194) bank163805 +163806 POINT(41.39797602644194 -88.27436790430203) bank163806 +163807 POINT(42.675675941086645 -86.71538396831893) bank163807 +163808 POINT(41.543609484959966 -88.56482098996088) bank163808 +163809 POINT(40.94645359620109 -86.92918965893234) bank163809 +163810 POINT(41.88466572267494 -87.82719166231533) bank163810 +163811 POINT(42.64552741132042 -87.071074794352) bank163811 +163812 POINT(40.96723133404487 -88.36184115859722) bank163812 +163813 POINT(41.72421332721694 -88.21360111638049) bank163813 +163814 POINT(42.02975426468409 -87.73274906295815) bank163814 +163815 POINT(41.24273465131101 -87.05050748846365) bank163815 +163816 POINT(41.16094388779431 -87.14303067887256) bank163816 +163817 POINT(40.971417121308704 -86.96795533595446) bank163817 +163818 POINT(41.04281369777154 -88.53675153393921) bank163818 +163819 POINT(41.7070966168207 -87.08642310821543) bank163819 +163820 POINT(41.66727818447135 -87.59562098820857) bank163820 +163821 POINT(41.01724337046464 -86.92313898698569) bank163821 +163822 POINT(41.53528317031063 -87.39881090110761) bank163822 +163823 POINT(42.10249758015432 -87.1890958396464) bank163823 +163824 POINT(41.64179907134427 -87.39270473544742) bank163824 +163825 POINT(42.330683804280774 -88.06574737908173) bank163825 +163826 POINT(41.71020473536315 -88.57149660088977) bank163826 +163827 POINT(41.15571780384214 -88.18578067888193) bank163827 +163828 POINT(42.68998006273351 -87.64701898683572) bank163828 +163829 POINT(42.10238522670106 -88.21405513951437) bank163829 +163830 POINT(42.76680237811923 -88.04207071521327) bank163830 +163831 POINT(42.31272402751533 -87.71445594431174) bank163831 +163832 POINT(41.43768267515497 -88.29344646875691) bank163832 +163833 POINT(41.789173732905745 -88.62691065168741) bank163833 +163834 POINT(42.34987991596942 -87.9027586360622) bank163834 +163835 POINT(42.37415159778349 -88.40187741593263) bank163835 +163836 POINT(41.55588637138797 -88.10477422537586) bank163836 +163837 POINT(41.75320547405865 -88.27583362040414) bank163837 +163838 POINT(41.577701836184794 -87.99551828406321) bank163838 +163839 POINT(41.6428233916915 -88.0151577691169) bank163839 +163840 POINT(42.58974212406644 -86.95012154675379) bank163840 +163841 POINT(41.39795523611466 -88.24556068634485) bank163841 +163842 POINT(41.39332899527646 -87.51055596290425) bank163842 +163843 POINT(41.32551899480974 -87.06761108563985) bank163843 +163844 POINT(40.98215836780639 -87.95513328226741) bank163844 +163845 POINT(41.018692605457346 -87.45996035911499) bank163845 +163846 POINT(41.150642423657644 -87.50435330795543) bank163846 +163847 POINT(42.22177196715668 -86.93177064428774) bank163847 +163848 POINT(41.78865725187613 -87.84502041095743) bank163848 +163849 POINT(42.866045082495575 -87.73374186526802) bank163849 +163850 POINT(41.061092912528515 -88.47113566566473) bank163850 +163851 POINT(41.394030983005415 -87.6450077879078) bank163851 +163852 POINT(42.23382652025691 -87.41514392302444) bank163852 +163853 POINT(42.51860672630539 -87.92458014665175) bank163853 +163854 POINT(42.68934533328732 -88.45548036345755) bank163854 +163855 POINT(42.528907706455165 -87.92861643467596) bank163855 +163856 POINT(41.51382923251984 -87.76458296643929) bank163856 +163857 POINT(42.71362752554749 -87.71768280088637) bank163857 +163858 POINT(42.33785118804788 -88.18121804949666) bank163858 +163859 POINT(42.35750415437817 -87.61602335774444) bank163859 +163860 POINT(42.57381785301999 -88.6109875697942) bank163860 +163861 POINT(41.013352796248135 -86.75180993115258) bank163861 +163862 POINT(41.53446400537476 -88.04928662855414) bank163862 +163863 POINT(41.38171799195187 -86.99671687211449) bank163863 +163864 POINT(42.71457346147495 -88.43017228362739) bank163864 +163865 POINT(41.77055567318363 -86.87612466947436) bank163865 +163866 POINT(41.591200204105874 -86.63267870076056) bank163866 +163867 POINT(41.38514556073715 -87.21367367939243) bank163867 +163868 POINT(41.06039503924803 -87.13052778339785) bank163868 +163869 POINT(42.02786395303334 -87.72490263199502) bank163869 +163870 POINT(42.57948003404147 -87.22593566260174) bank163870 +163871 POINT(42.19647810040684 -88.49796383848253) bank163871 +163872 POINT(42.8776397909511 -86.71880917540436) bank163872 +163873 POINT(42.69036082114698 -87.17669122600232) bank163873 +163874 POINT(41.0834522565166 -88.03020334327888) bank163874 +163875 POINT(42.55110814173868 -86.8898803161633) bank163875 +163876 POINT(41.888320101988626 -86.98708344091212) bank163876 +163877 POINT(40.99269027535502 -88.43184901109399) bank163877 +163878 POINT(41.89162225202479 -86.819991253265) bank163878 +163879 POINT(42.53825589216511 -88.50823075670247) bank163879 +163880 POINT(41.87731185568605 -88.52613945812485) bank163880 +163881 POINT(41.13583665312835 -87.19771823369786) bank163881 +163882 POINT(42.14778549173514 -88.36470817052704) bank163882 +163883 POINT(41.54258519205995 -87.1831129360813) bank163883 +163884 POINT(41.6571064058047 -87.43737990262052) bank163884 +163885 POINT(42.17698437936851 -87.00940743013082) bank163885 +163886 POINT(42.7374387169129 -86.78498376333702) bank163886 +163887 POINT(41.659137766637556 -87.3817308254923) bank163887 +163888 POINT(41.88017768951917 -88.09077011906227) bank163888 +163889 POINT(41.40149979588669 -87.76750339579674) bank163889 +163890 POINT(42.77549362156603 -87.19194391186396) bank163890 +163891 POINT(42.276309282034475 -88.07573327645481) bank163891 +163892 POINT(41.95501144690323 -87.78487627329714) bank163892 +163893 POINT(42.67533256653657 -87.274327882747) bank163893 +163894 POINT(40.89407988868192 -87.15654728002627) bank163894 +163895 POINT(42.06668045146002 -87.98804339185784) bank163895 +163896 POINT(42.188188192425585 -87.97208326903733) bank163896 +163897 POINT(41.27924121954621 -87.14841681260822) bank163897 +163898 POINT(42.56652172909248 -87.20339812992644) bank163898 +163899 POINT(41.64121980212566 -86.95251514856588) bank163899 +163900 POINT(41.55200545898299 -86.83593330545868) bank163900 +163901 POINT(42.286581728698195 -88.51479848832261) bank163901 +163902 POINT(42.142763375933455 -88.11038024894584) bank163902 +163903 POINT(41.96814900570157 -87.9467976420985) bank163903 +163904 POINT(41.929523102350494 -87.87388530615752) bank163904 +163905 POINT(42.61964068651423 -87.53161594170776) bank163905 +163906 POINT(42.03518916461946 -87.4958578210668) bank163906 +163907 POINT(42.037566670871385 -88.50352993248426) bank163907 +163908 POINT(42.75186626801042 -86.77404053169907) bank163908 +163909 POINT(41.78375921857556 -88.5545221541715) bank163909 +163910 POINT(42.10534465507515 -87.07923882724674) bank163910 +163911 POINT(41.67757886738045 -86.97328975781998) bank163911 +163912 POINT(42.59617990794252 -87.09396908043173) bank163912 +163913 POINT(41.5831633201918 -88.40841968779492) bank163913 +163914 POINT(42.40826703917576 -87.41267634159517) bank163914 +163915 POINT(41.24096460471935 -87.56602115593078) bank163915 +163916 POINT(42.02801263352898 -86.63508883955109) bank163916 +163917 POINT(41.715678687883454 -87.79124668439597) bank163917 +163918 POINT(41.9260772433365 -86.69236758498481) bank163918 +163919 POINT(41.07438171282023 -87.57248501404197) bank163919 +163920 POINT(41.04210491049947 -87.13237317894442) bank163920 +163921 POINT(41.533427355735384 -87.76475837078662) bank163921 +163922 POINT(41.516235390608934 -87.33391955210912) bank163922 +163923 POINT(42.56637121028502 -88.41029750790459) bank163923 +163924 POINT(42.48122055862904 -86.73689675294005) bank163924 +163925 POINT(42.18400036319853 -88.61469640227395) bank163925 +163926 POINT(40.91097336806322 -88.33842937375599) bank163926 +163927 POINT(41.122448100427334 -87.44431314611326) bank163927 +163928 POINT(41.3214277045707 -87.61106599755328) bank163928 +163929 POINT(41.3040504517091 -87.26933378696562) bank163929 +163930 POINT(42.07197013363173 -86.65031584561196) bank163930 +163931 POINT(41.4865869706848 -87.78052021316103) bank163931 +163932 POINT(42.18129085932632 -86.67760847680604) bank163932 +163933 POINT(42.480841406716614 -87.78933827613227) bank163933 +163934 POINT(42.15734125158874 -86.75460251436486) bank163934 +163935 POINT(42.13428691151287 -86.73747904163608) bank163935 +163936 POINT(41.724019493699444 -87.15519108311187) bank163936 +163937 POINT(42.24228202784203 -88.24094358372416) bank163937 +163938 POINT(42.3216801175958 -88.01258669093869) bank163938 +163939 POINT(41.16615322055872 -86.93992427828694) bank163939 +163940 POINT(42.20486276364803 -87.60268860522856) bank163940 +163941 POINT(42.15037246415655 -87.08656492383624) bank163941 +163942 POINT(42.42787690012523 -87.03577190344) bank163942 +163943 POINT(42.105555240649394 -87.66791359649554) bank163943 +163944 POINT(42.7659152605921 -88.45142376317973) bank163944 +163945 POINT(42.33557880713787 -88.5412906058703) bank163945 +163946 POINT(42.739602085590114 -88.2908054080692) bank163946 +163947 POINT(40.883936268039726 -88.55965450183996) bank163947 +163948 POINT(42.252323170087735 -87.79875602210917) bank163948 +163949 POINT(41.659058593423346 -88.35475187779434) bank163949 +163950 POINT(42.24025337902045 -87.41207134919216) bank163950 +163951 POINT(41.07919896801268 -87.97643771238424) bank163951 +163952 POINT(42.77576759294513 -87.83983689381417) bank163952 +163953 POINT(42.56546432119738 -87.89001909774703) bank163953 +163954 POINT(41.77925599722793 -87.44329001261684) bank163954 +163955 POINT(41.4107315488294 -87.11760850389598) bank163955 +163956 POINT(41.535321123521946 -87.01388351324829) bank163956 +163957 POINT(41.490549425018855 -88.55749836562993) bank163957 +163958 POINT(42.21898259414265 -87.97001741218595) bank163958 +163959 POINT(42.18231474016777 -88.44376038410968) bank163959 +163960 POINT(41.23587971191955 -86.77737298498127) bank163960 +163961 POINT(41.07953940927651 -87.04834842407239) bank163961 +163962 POINT(41.41407732085824 -87.45696653875876) bank163962 +163963 POINT(42.7113670703082 -86.92557746325244) bank163963 +163964 POINT(41.393475769057595 -88.077414715157) bank163964 +163965 POINT(42.488991097154944 -88.29662294828532) bank163965 +163966 POINT(41.07112203182043 -88.24161876831569) bank163966 +163967 POINT(41.85271003942239 -87.30079588177675) bank163967 +163968 POINT(41.286979662479155 -86.90004692811605) bank163968 +163969 POINT(41.1709160159756 -87.39581883452081) bank163969 +163970 POINT(42.74125239131101 -87.12184100179701) bank163970 +163971 POINT(42.453358202373806 -86.67220649333792) bank163971 +163972 POINT(41.985643814606995 -88.29029801081374) bank163972 +163973 POINT(42.388697638324444 -87.08293105647573) bank163973 +163974 POINT(42.61290118778289 -88.08480060042511) bank163974 +163975 POINT(42.47143058725194 -87.90521709128657) bank163975 +163976 POINT(41.351763674144856 -87.57852747708858) bank163976 +163977 POINT(42.2441309904232 -86.67743304286977) bank163977 +163978 POINT(41.38446799417609 -87.11904819181342) bank163978 +163979 POINT(41.041630328209564 -87.5716618742473) bank163979 +163980 POINT(42.4595043791331 -86.9205671419896) bank163980 +163981 POINT(40.96304073571272 -87.00743577453342) bank163981 +163982 POINT(41.21086408904287 -87.56263240818676) bank163982 +163983 POINT(42.13403186758108 -86.76040838300999) bank163983 +163984 POINT(41.6269739256695 -88.18168073390427) bank163984 +163985 POINT(41.42226208971816 -87.51828729477269) bank163985 +163986 POINT(42.65200928259857 -86.8982741271719) bank163986 +163987 POINT(41.96116066356863 -87.33731799971069) bank163987 +163988 POINT(41.76855980316806 -88.0956120937753) bank163988 +163989 POINT(41.672639679788006 -86.85301181536256) bank163989 +163990 POINT(41.99067719959944 -87.35477777131136) bank163990 +163991 POINT(41.578197172854196 -86.76011050366631) bank163991 +163992 POINT(42.1611350599891 -87.26894898920135) bank163992 +163993 POINT(42.77430163115062 -86.87770396545439) bank163993 +163994 POINT(42.52517530573421 -88.21317157868637) bank163994 +163995 POINT(41.43061345848338 -88.33131274282519) bank163995 +163996 POINT(41.52821257786918 -86.94717586651247) bank163996 +163997 POINT(42.203655632047294 -86.81573029862894) bank163997 +163998 POINT(42.171590814876126 -87.18776525021617) bank163998 +163999 POINT(42.32865719585292 -87.07912857764683) bank163999 +164000 POINT(41.45705523681828 -87.54642806223316) bank164000 +164001 POINT(41.74679657892433 -87.95488766150507) bank164001 +164002 POINT(42.48484554336567 -88.21841001666638) bank164002 +164003 POINT(41.40562168121281 -88.59232701295878) bank164003 +164004 POINT(42.44054521293826 -88.07672288540697) bank164004 +164005 POINT(42.57236040065686 -88.6095129346356) bank164005 +164006 POINT(41.810652840892914 -87.21795130203807) bank164006 +164007 POINT(42.15475789228892 -88.37518411232087) bank164007 +164008 POINT(41.53793091249085 -88.1283467238897) bank164008 +164009 POINT(42.786380355567466 -87.1269129332407) bank164009 +164010 POINT(41.760968158252176 -87.19196440711664) bank164010 +164011 POINT(41.44980834029633 -87.61826379119192) bank164011 +164012 POINT(42.45827212824924 -88.1175096422402) bank164012 +164013 POINT(42.099822843273785 -88.58857164083273) bank164013 +164014 POINT(42.556249234267376 -88.31545177986132) bank164014 +164015 POINT(41.83720250811205 -87.80191994272866) bank164015 +164016 POINT(41.31395156332147 -87.439337350293) bank164016 +164017 POINT(41.170458011030206 -87.62142763847704) bank164017 +164018 POINT(42.60428548789741 -87.9046098142086) bank164018 +164019 POINT(42.100155940652634 -87.04110928604727) bank164019 +164020 POINT(41.505267613279294 -87.58309807560101) bank164020 +164021 POINT(41.252491420331616 -86.81584335222317) bank164021 +164022 POINT(41.1540590467812 -87.37273548670684) bank164022 +164023 POINT(42.770133197075154 -87.46121158571205) bank164023 +164024 POINT(41.87348059150689 -87.45961375709797) bank164024 +164025 POINT(41.054151997408916 -88.19052627687796) bank164025 +164026 POINT(42.34284330927854 -88.31341548003212) bank164026 +164027 POINT(42.174729542355585 -87.22143355282691) bank164027 +164028 POINT(41.71490073712491 -87.17431619287007) bank164028 +164029 POINT(42.78919213971931 -86.75449896898084) bank164029 +164030 POINT(41.90756496266058 -86.957719890624) bank164030 +164031 POINT(42.14495430293222 -87.08157387365013) bank164031 +164032 POINT(42.01278147953046 -87.08245434595031) bank164032 +164033 POINT(41.42751377523611 -86.82360829406184) bank164033 +164034 POINT(42.49445948216889 -88.05756862644247) bank164034 +164035 POINT(41.31034500020887 -87.59645740241386) bank164035 +164036 POINT(40.967506471864155 -88.55421166497423) bank164036 +164037 POINT(42.600142386782636 -87.45504086560611) bank164037 +164038 POINT(40.88184317424976 -88.15180258193942) bank164038 +164039 POINT(41.10154524066609 -87.57955441687558) bank164039 +164040 POINT(42.401953171728735 -87.63933480160252) bank164040 +164041 POINT(42.64725762610516 -88.4063237032526) bank164041 +164042 POINT(41.10478283713278 -87.3853266443124) bank164042 +164043 POINT(41.57930885766049 -87.0630684233669) bank164043 +164044 POINT(42.713366595820055 -86.98802171638431) bank164044 +164045 POINT(42.59992196192833 -87.56347408844229) bank164045 +164046 POINT(41.04671842367696 -86.74664248717238) bank164046 +164047 POINT(42.84122851800133 -88.21253773289244) bank164047 +164048 POINT(42.203974315585505 -88.01002007723781) bank164048 +164049 POINT(41.52189079467925 -88.10657813042674) bank164049 +164050 POINT(41.612942064077444 -87.63036325849063) bank164050 +164051 POINT(41.13067899007184 -87.19311062408917) bank164051 +164052 POINT(41.16617534177512 -88.37768823650794) bank164052 +164053 POINT(41.258796388874075 -88.35403190954466) bank164053 +164054 POINT(42.8189203267574 -86.98731887197943) bank164054 +164055 POINT(40.88600519801312 -88.33118502289005) bank164055 +164056 POINT(42.27454549203593 -88.10237488996742) bank164056 +164057 POINT(41.484134982816826 -88.54655643846858) bank164057 +164058 POINT(42.23971472338526 -86.98463402515435) bank164058 +164059 POINT(41.53248208249357 -87.01643774369428) bank164059 +164060 POINT(41.229166724244436 -87.49185572379552) bank164060 +164061 POINT(42.322012175348604 -87.09416211029546) bank164061 +164062 POINT(41.47226413649266 -87.66439836096329) bank164062 +164063 POINT(41.696775596081615 -87.04942240779411) bank164063 +164064 POINT(41.31740282659444 -87.3813006428412) bank164064 +164065 POINT(42.020840771833555 -88.59195187043409) bank164065 +164066 POINT(42.73907730507641 -86.89476502132975) bank164066 +164067 POINT(41.91470263706874 -87.25341550308713) bank164067 +164068 POINT(41.24508316796101 -87.07589184647665) bank164068 +164069 POINT(41.21174289073638 -87.34943404611946) bank164069 +164070 POINT(41.76370210775172 -87.81875533788856) bank164070 +164071 POINT(42.75033868023987 -87.57630589279326) bank164071 +164072 POINT(42.36137027282941 -88.34051111225531) bank164072 +164073 POINT(42.4096624283429 -88.34573994741547) bank164073 +164074 POINT(42.07205000220615 -87.9857066873567) bank164074 +164075 POINT(41.7564260085464 -86.97675217744161) bank164075 +164076 POINT(42.155875365750724 -86.84607764881305) bank164076 +164077 POINT(41.60331644214714 -87.99789898021815) bank164077 +164078 POINT(42.75546176979097 -87.9642043936948) bank164078 +164079 POINT(41.765343748226286 -87.70093455181828) bank164079 +164080 POINT(42.67021208951724 -87.37572173961247) bank164080 +164081 POINT(41.57598320477845 -88.29980526565572) bank164081 +164082 POINT(41.057993205296576 -88.35397914154213) bank164082 +164083 POINT(42.29585176654978 -87.17143809367734) bank164083 +164084 POINT(41.1977406796484 -87.20497466059024) bank164084 +164085 POINT(42.07128138417832 -87.66078750478465) bank164085 +164086 POINT(41.67296419861185 -87.55232851404209) bank164086 +164087 POINT(41.176079114097185 -88.23437277908637) bank164087 +164088 POINT(41.52195453783816 -87.03287539261886) bank164088 +164089 POINT(42.81746198782644 -86.8195390762821) bank164089 +164090 POINT(41.31499804709322 -86.69999111126302) bank164090 +164091 POINT(40.90271891985857 -87.70732812755901) bank164091 +164092 POINT(41.38738408410482 -88.31328684053244) bank164092 +164093 POINT(40.97143507395512 -87.82321838161991) bank164093 +164094 POINT(42.435009259917464 -87.20952704333357) bank164094 +164095 POINT(41.7728927284807 -86.7392664131769) bank164095 +164096 POINT(42.25935747848659 -87.17816452479671) bank164096 +164097 POINT(42.4544141503668 -88.55603149682003) bank164097 +164098 POINT(41.77168637425991 -87.60296142189172) bank164098 +164099 POINT(41.15776526152754 -88.01111420651141) bank164099 +164100 POINT(41.53540875238988 -88.09522295674508) bank164100 +164101 POINT(41.917437577545684 -87.34663077316169) bank164101 +164102 POINT(41.16807252962161 -88.59803752897925) bank164102 +164103 POINT(41.15545021370209 -88.02408045035203) bank164103 +164104 POINT(41.43028901021248 -88.3769462639034) bank164104 +164105 POINT(42.344120523753155 -88.49938088702125) bank164105 +164106 POINT(42.54354347227826 -87.11401628641252) bank164106 +164107 POINT(41.61177775714815 -86.9965135620004) bank164107 +164108 POINT(40.99357753136025 -86.69104550345583) bank164108 +164109 POINT(42.17623753860721 -86.70111936447792) bank164109 +164110 POINT(41.5040541760748 -87.97110866438992) bank164110 +164111 POINT(41.66807474317227 -88.00624949044973) bank164111 +164112 POINT(41.615882996482824 -87.927749377766) bank164112 +164113 POINT(40.996413954350295 -88.17028627635888) bank164113 +164114 POINT(41.70718965516056 -88.36404113654217) bank164114 +164115 POINT(41.98422035790362 -88.56180581055337) bank164115 +164116 POINT(41.95903001987733 -86.69985636519657) bank164116 +164117 POINT(42.78050049242156 -86.71213693497926) bank164117 +164118 POINT(41.96929339894964 -88.03557619343675) bank164118 +164119 POINT(42.57902160344633 -88.31229122669767) bank164119 +164120 POINT(41.27048928503904 -87.593638270626) bank164120 +164121 POINT(41.99925040287456 -88.38648731672926) bank164121 +164122 POINT(41.71800167024538 -86.63641253828736) bank164122 +164123 POINT(41.7493837929715 -88.12159898238284) bank164123 +164124 POINT(41.201891985893084 -87.8445071917688) bank164124 +164125 POINT(41.917410146886795 -86.96900887826682) bank164125 +164126 POINT(42.73540454249477 -88.47154206665554) bank164126 +164127 POINT(41.603487290995396 -88.53373917063968) bank164127 +164128 POINT(42.10476081517948 -87.77579681597932) bank164128 +164129 POINT(42.70915002751168 -87.1977368574443) bank164129 +164130 POINT(42.13371038719553 -87.5803264632794) bank164130 +164131 POINT(41.55187418515712 -88.12304256312484) bank164131 +164132 POINT(42.769970445873795 -87.51546565625259) bank164132 +164133 POINT(42.011350193929545 -88.62678671276673) bank164133 +164134 POINT(41.20849020202392 -88.17628289001786) bank164134 +164135 POINT(42.413562468356936 -87.69425089861383) bank164135 +164136 POINT(41.38143782046622 -87.09270073363173) bank164136 +164137 POINT(40.993647425519185 -87.10715238739448) bank164137 +164138 POINT(41.18608629544124 -86.8885412926383) bank164138 +164139 POINT(41.855826918752996 -87.25637759623305) bank164139 +164140 POINT(41.682717650429346 -87.77132372723742) bank164140 +164141 POINT(41.14621873677911 -87.15698794206342) bank164141 +164142 POINT(41.91798582634046 -88.20362403164464) bank164142 +164143 POINT(42.31530919191646 -88.17781183597089) bank164143 +164144 POINT(42.349277334811994 -88.08028548401694) bank164144 +164145 POINT(42.58961688883863 -86.74752406392322) bank164145 +164146 POINT(41.34192801555452 -88.24553551470495) bank164146 +164147 POINT(41.254896056424755 -88.54940080157877) bank164147 +164148 POINT(41.17832898644431 -87.1466006630018) bank164148 +164149 POINT(41.97210823226115 -87.37259345753114) bank164149 +164150 POINT(41.7809448697644 -86.74284640984463) bank164150 +164151 POINT(41.25145504329521 -88.54269754839603) bank164151 +164152 POINT(42.630938920404276 -86.81908331655914) bank164152 +164153 POINT(42.69686642355204 -88.03023993176073) bank164153 +164154 POINT(41.42573815031598 -87.30372588088674) bank164154 +164155 POINT(41.627915117266696 -88.57531731964708) bank164155 +164156 POINT(42.4663108975074 -86.98828388986387) bank164156 +164157 POINT(41.26575718510288 -86.88019260061714) bank164157 +164158 POINT(40.98649213321717 -88.53824657832915) bank164158 +164159 POINT(42.580027821067176 -86.74182565811965) bank164159 +164160 POINT(41.14849618951979 -88.58258586092334) bank164160 +164161 POINT(41.97595898562656 -87.66518328110288) bank164161 +164162 POINT(40.89030517356275 -87.08163216533373) bank164162 +164163 POINT(42.057184030732536 -88.00472061857822) bank164163 +164164 POINT(42.45148109505019 -88.22660759448944) bank164164 +164165 POINT(41.569330147344786 -86.83484982979434) bank164165 +164166 POINT(42.10075109579306 -86.93638861121723) bank164166 +164167 POINT(41.94545842797296 -86.90990746631942) bank164167 +164168 POINT(41.99102299918181 -88.46793042269134) bank164168 +164169 POINT(41.96889725395331 -88.22235031953159) bank164169 +164170 POINT(41.16587735067849 -87.1736106112366) bank164170 +164171 POINT(41.662863921261426 -88.41077357405216) bank164171 +164172 POINT(41.31460561662723 -88.14862548419613) bank164172 +164173 POINT(42.352476549217066 -88.0745711553639) bank164173 +164174 POINT(42.1960012286323 -87.29778403368246) bank164174 +164175 POINT(41.8841620221168 -86.9725656284739) bank164175 +164176 POINT(42.667436367705506 -87.65643646167645) bank164176 +164177 POINT(41.99395256712389 -87.0935057472756) bank164177 +164178 POINT(42.33300101198608 -88.04204435876889) bank164178 +164179 POINT(42.55430713681005 -87.67216038370769) bank164179 +164180 POINT(42.438552696092124 -87.33275848914256) bank164180 +164181 POINT(41.11261530346143 -86.78919857096305) bank164181 +164182 POINT(41.89548989651659 -86.90107006711828) bank164182 +164183 POINT(42.60922052162088 -88.52964567019973) bank164183 +164184 POINT(41.46861733589211 -88.45301325495859) bank164184 +164185 POINT(42.78457547967057 -87.72527126560146) bank164185 +164186 POINT(41.04252925874324 -86.67888863461609) bank164186 +164187 POINT(41.736899916428165 -87.91220809457208) bank164187 +164188 POINT(42.56781237919724 -87.43878570466549) bank164188 +164189 POINT(42.261808348759835 -86.93081240503936) bank164189 +164190 POINT(42.338586421779475 -86.79787159351969) bank164190 +164191 POINT(42.828195552629765 -87.69188486887828) bank164191 +164192 POINT(42.438584416393226 -86.75621423023209) bank164192 +164193 POINT(41.395405436058006 -86.9535078029133) bank164193 +164194 POINT(42.54855955477834 -86.76166865588716) bank164194 +164195 POINT(41.47798662976436 -88.6221101116826) bank164195 +164196 POINT(42.18225931349377 -88.61576340841432) bank164196 +164197 POINT(41.15220088405348 -87.24462548267425) bank164197 +164198 POINT(42.22519960831548 -88.00647551593642) bank164198 +164199 POINT(41.381530214345155 -87.14017769462377) bank164199 +164200 POINT(42.10012993452394 -88.53530859362151) bank164200 +164201 POINT(42.495153005129765 -87.64821752362144) bank164201 +164202 POINT(41.31718864818238 -87.21796739370404) bank164202 +164203 POINT(42.005771092027416 -87.02462357838657) bank164203 +164204 POINT(42.37555146603087 -88.08210799810264) bank164204 +164205 POINT(42.61029121082786 -86.77316443348128) bank164205 +164206 POINT(41.553923225569775 -87.35214909899454) bank164206 +164207 POINT(42.73568685578006 -87.76609065639931) bank164207 +164208 POINT(41.75455654205094 -87.03377310393739) bank164208 +164209 POINT(41.16568263774276 -87.84818388251628) bank164209 +164210 POINT(42.14179467547713 -87.43288227651976) bank164210 +164211 POINT(42.39889986728183 -87.96330538347256) bank164211 +164212 POINT(42.243676878622225 -86.93995358103743) bank164212 +164213 POINT(41.51805719285998 -86.7556738349825) bank164213 +164214 POINT(42.83591340019024 -86.94983963587283) bank164214 +164215 POINT(41.593399999812576 -88.46575231241842) bank164215 +164216 POINT(41.04719483793623 -87.13379389832907) bank164216 +164217 POINT(41.79868205371329 -87.4241010436354) bank164217 +164218 POINT(42.624300053980576 -86.81592579857018) bank164218 +164219 POINT(41.88058162035304 -87.53991288493452) bank164219 +164220 POINT(41.41284421645002 -87.37060086398155) bank164220 +164221 POINT(42.69217763198605 -87.37327533945081) bank164221 +164222 POINT(42.70059232687227 -86.88067665337552) bank164222 +164223 POINT(42.05171255535939 -87.92468165189501) bank164223 +164224 POINT(42.58401897478083 -87.80029259010254) bank164224 +164225 POINT(40.922336410465064 -88.3538122096797) bank164225 +164226 POINT(42.458789606056335 -88.09647550749712) bank164226 +164227 POINT(42.748967113234485 -88.11897954074013) bank164227 +164228 POINT(41.40048027238319 -88.07736111618036) bank164228 +164229 POINT(41.72007873016603 -86.81473034196559) bank164229 +164230 POINT(42.059566958336475 -88.15910279769984) bank164230 +164231 POINT(42.71030507091644 -87.45307399619234) bank164231 +164232 POINT(41.22534158152702 -87.30595836250268) bank164232 +164233 POINT(41.932876645387005 -88.287803252683) bank164233 +164234 POINT(42.69747216408845 -86.79244271359941) bank164234 +164235 POINT(41.378755027726164 -87.78301363590552) bank164235 +164236 POINT(42.40648542749141 -87.28640998321416) bank164236 +164237 POINT(41.23132125245739 -87.19049543834157) bank164237 +164238 POINT(41.9072593137091 -86.83086701265617) bank164238 +164239 POINT(42.553846635756464 -88.37793268992655) bank164239 +164240 POINT(40.88305896963947 -88.3596787966608) bank164240 +164241 POINT(42.03358395786901 -87.64898806398344) bank164241 +164242 POINT(41.220556811860675 -87.30771979417375) bank164242 +164243 POINT(42.34220497056255 -87.6990415051395) bank164243 +164244 POINT(42.291277284244195 -87.25844899653974) bank164244 +164245 POINT(41.680098272654064 -88.04049127764084) bank164245 +164246 POINT(41.374018218129635 -87.98575558785156) bank164246 +164247 POINT(42.75371031393764 -86.86004673843146) bank164247 +164248 POINT(42.07996733140636 -88.49816093628462) bank164248 +164249 POINT(41.43259257445993 -88.01028230147881) bank164249 +164250 POINT(41.777215102443094 -87.88872250107276) bank164250 +164251 POINT(41.126719964618694 -87.12618624104589) bank164251 +164252 POINT(41.53629962491479 -86.83877056838288) bank164252 +164253 POINT(41.00990819216241 -86.70225485298614) bank164253 +164254 POINT(42.85469083641249 -87.3852244472762) bank164254 +164255 POINT(41.54445526715373 -87.56999291082555) bank164255 +164256 POINT(42.53534453722975 -88.26491833251684) bank164256 +164257 POINT(41.760184632266274 -87.27087563488038) bank164257 +164258 POINT(41.34554505063689 -87.19420001385315) bank164258 +164259 POINT(42.727550650964204 -87.01722168731447) bank164259 +164260 POINT(41.649496680550484 -87.43296189080132) bank164260 +164261 POINT(42.71774541429505 -87.22472259790054) bank164261 +164262 POINT(41.402022011899724 -86.70134519739945) bank164262 +164263 POINT(42.30297814623941 -86.77548637479994) bank164263 +164264 POINT(41.185614385282314 -87.03980827267127) bank164264 +164265 POINT(42.24045154211631 -88.1196335612625) bank164265 +164266 POINT(42.625113534424884 -87.5677029249303) bank164266 +164267 POINT(41.29651665373899 -86.76803553492756) bank164267 +164268 POINT(41.8360831233342 -88.52937242637336) bank164268 +164269 POINT(42.7713851197461 -87.65252749004489) bank164269 +164270 POINT(41.68474051431193 -87.00366715363565) bank164270 +164271 POINT(42.31494364496878 -87.85753785089062) bank164271 +164272 POINT(41.14542784615544 -87.72445415165352) bank164272 +164273 POINT(42.525064354442414 -86.65388985280981) bank164273 +164274 POINT(41.18129428351168 -87.64677452290034) bank164274 +164275 POINT(40.90759404199925 -87.78946005808402) bank164275 +164276 POINT(41.99413532258716 -88.48880912038129) bank164276 +164277 POINT(42.799968159047026 -88.22694938881527) bank164277 +164278 POINT(41.5944225153451 -86.73076467521182) bank164278 +164279 POINT(42.18538347776184 -87.8401673531817) bank164279 +164280 POINT(41.07541598894322 -87.18350810742665) bank164280 +164281 POINT(41.74829714767873 -87.10548708396087) bank164281 +164282 POINT(41.64443578115009 -86.71162734284249) bank164282 +164283 POINT(41.40035719285623 -88.05545120244346) bank164283 +164284 POINT(41.281889716612746 -88.46119289108992) bank164284 +164285 POINT(41.104880629214044 -88.57709571470824) bank164285 +164286 POINT(41.5345900607648 -86.86693417801118) bank164286 +164287 POINT(41.99373155382513 -87.99903048395497) bank164287 +164288 POINT(41.47162923050312 -86.77325025289846) bank164288 +164289 POINT(41.5676736659002 -87.8049031331952) bank164289 +164290 POINT(41.30238432945342 -86.74301436895456) bank164290 +164291 POINT(40.99874901850004 -87.65911747337623) bank164291 +164292 POINT(41.78269601636475 -88.01469921068451) bank164292 +164293 POINT(41.694493461532836 -86.82863047559532) bank164293 +164294 POINT(40.99691740253 -88.28617152446304) bank164294 +164295 POINT(40.98974836223387 -88.53896575414225) bank164295 +164296 POINT(42.61441841756919 -88.31387751250601) bank164296 +164297 POINT(41.33476110946835 -88.07435693288836) bank164297 +164298 POINT(40.98115900386208 -88.00158881995257) bank164298 +164299 POINT(42.19571638540999 -87.6549861863762) bank164299 +164300 POINT(42.401742807172326 -88.29023311542959) bank164300 +164301 POINT(42.79392010775855 -87.40810280976018) bank164301 +164302 POINT(41.98199167668575 -86.95254962569645) bank164302 +164303 POINT(41.87639807094926 -87.1619769596226) bank164303 +164304 POINT(41.08031190629062 -86.76644195053305) bank164304 +164305 POINT(42.251002460950566 -86.75756460209483) bank164305 +164306 POINT(42.49054821522954 -87.61818549383482) bank164306 +164307 POINT(41.31708593223946 -86.68214085609378) bank164307 +164308 POINT(41.25302709384853 -87.24478965294396) bank164308 +164309 POINT(41.83979260227312 -86.84883315180032) bank164309 +164310 POINT(41.781017855199096 -87.24073164634261) bank164310 +164311 POINT(41.61925965379547 -87.16097965194174) bank164311 +164312 POINT(40.95073878942831 -87.74500555481254) bank164312 +164313 POINT(42.12349238952744 -88.3168126689344) bank164313 +164314 POINT(42.457223090249485 -87.84345058005364) bank164314 +164315 POINT(41.87636270443241 -87.97126565858404) bank164315 +164316 POINT(40.997417019035616 -86.95529484713238) bank164316 +164317 POINT(42.685870790107934 -86.87274783623022) bank164317 +164318 POINT(42.57005449907756 -88.52484343808491) bank164318 +164319 POINT(42.62578511801282 -86.99561304142769) bank164319 +164320 POINT(42.764130526616796 -88.17319790329603) bank164320 +164321 POINT(41.12174464360549 -86.99642629486202) bank164321 +164322 POINT(41.075023810308764 -87.5624984019367) bank164322 +164323 POINT(42.649663368628204 -88.3422832596317) bank164323 +164324 POINT(41.180914464831716 -88.15365721071853) bank164324 +164325 POINT(41.259334879429886 -88.45936121719053) bank164325 +164326 POINT(42.3209743452836 -87.17833526177589) bank164326 +164327 POINT(42.75812551158485 -88.39225135764185) bank164327 +164328 POINT(41.49699815569394 -88.03333029050617) bank164328 +164329 POINT(42.224100581614 -87.73219989378873) bank164329 +164330 POINT(41.60447407812716 -86.74227020658634) bank164330 +164331 POINT(42.28063534368759 -86.94336843348493) bank164331 +164332 POINT(41.68526334112027 -87.90239227931856) bank164332 +164333 POINT(42.037518903225255 -87.12067863118122) bank164333 +164334 POINT(41.17398635595634 -87.21303746914907) bank164334 +164335 POINT(40.90616842215392 -87.76386707471978) bank164335 +164336 POINT(42.12451312225725 -87.5869894108232) bank164336 +164337 POINT(42.65663301451391 -88.53279086999241) bank164337 +164338 POINT(41.48713956092393 -87.8943816122604) bank164338 +164339 POINT(41.736049305504814 -86.7200624807226) bank164339 +164340 POINT(41.91892842909712 -86.70325606273816) bank164340 +164341 POINT(42.680463213575926 -88.15024253683411) bank164341 +164342 POINT(42.22684037604085 -87.01362627325663) bank164342 +164343 POINT(41.66271624162057 -87.2226188190105) bank164343 +164344 POINT(42.51597487948497 -88.09684744730652) bank164344 +164345 POINT(41.01392060965684 -87.00088964877422) bank164345 +164346 POINT(41.877847387302516 -88.04738285608812) bank164346 +164347 POINT(40.923982854997405 -87.58699565454235) bank164347 +164348 POINT(40.892527802868095 -87.317004370894) bank164348 +164349 POINT(41.690641057466834 -88.60100136412919) bank164349 +164350 POINT(42.43150958325203 -87.5140143579882) bank164350 +164351 POINT(41.13698964114563 -87.43710844149592) bank164351 +164352 POINT(41.2734083508036 -88.13025587478566) bank164352 +164353 POINT(42.13449786961605 -87.98027729046673) bank164353 +164354 POINT(42.45841566574826 -87.68982641772307) bank164354 +164355 POINT(41.373631117855766 -87.39340481964153) bank164355 +164356 POINT(42.16271583520599 -88.49258947525637) bank164356 +164357 POINT(41.42424341317722 -88.21754532345697) bank164357 +164358 POINT(42.55912816883863 -88.54965190968271) bank164358 +164359 POINT(40.946017747219095 -87.0918348573133) bank164359 +164360 POINT(41.007676037906926 -87.60273494005062) bank164360 +164361 POINT(41.845843939748136 -86.78420185089789) bank164361 +164362 POINT(42.504147856858225 -88.21142005855593) bank164362 +164363 POINT(42.8730537163804 -87.50054586337089) bank164363 +164364 POINT(40.94474921530624 -88.42953031289686) bank164364 +164365 POINT(42.55703006467472 -87.37567443633793) bank164365 +164366 POINT(41.006356985466475 -87.8033599641901) bank164366 +164367 POINT(42.32810801346706 -88.61087362622955) bank164367 +164368 POINT(41.65055455284804 -87.06077769140444) bank164368 +164369 POINT(40.95681556251385 -86.79929268925318) bank164369 +164370 POINT(41.23822590604964 -88.52446607498167) bank164370 +164371 POINT(41.83407591354709 -86.65411776434699) bank164371 +164372 POINT(41.610409397120655 -86.78477573117746) bank164372 +164373 POINT(42.74940740288983 -88.54089284858216) bank164373 +164374 POINT(42.1706917941345 -88.09705063813482) bank164374 +164375 POINT(41.54409494950262 -87.64127706255307) bank164375 +164376 POINT(42.348938451860576 -88.23478556523602) bank164376 +164377 POINT(42.19128567696526 -87.8995945785393) bank164377 +164378 POINT(41.19063244419589 -86.83531684871164) bank164378 +164379 POINT(41.81325469706251 -87.26044812044951) bank164379 +164380 POINT(41.944950980754015 -88.5940158342746) bank164380 +164381 POINT(42.56021474964449 -87.6668899224333) bank164381 +164382 POINT(41.54565962585848 -88.47133293130587) bank164382 +164383 POINT(41.340757970414955 -88.58958343993528) bank164383 +164384 POINT(41.82872127379132 -87.13092182130909) bank164384 +164385 POINT(42.53940311887932 -88.45504784069696) bank164385 +164386 POINT(41.265112758543054 -86.8652630234838) bank164386 +164387 POINT(42.03095005832426 -86.88820213194184) bank164387 +164388 POINT(41.85095511063412 -86.80002866301847) bank164388 +164389 POINT(42.34572027082223 -88.36034946668482) bank164389 +164390 POINT(41.93693392475528 -88.11482469791105) bank164390 +164391 POINT(42.46866809621834 -87.4804564055043) bank164391 +164392 POINT(42.48701621248448 -87.4573961331292) bank164392 +164393 POINT(42.09478516702542 -88.1975847140854) bank164393 +164394 POINT(41.34672779937617 -86.99275792563913) bank164394 +164395 POINT(42.13796804044291 -86.89482841440962) bank164395 +164396 POINT(42.44085954830855 -87.6753361852871) bank164396 +164397 POINT(42.78967552167447 -87.98046338230503) bank164397 +164398 POINT(42.677537628908254 -88.04027765397767) bank164398 +164399 POINT(41.77474054353108 -88.59839822537393) bank164399 +164400 POINT(42.178273056256494 -88.35511163079906) bank164400 +164401 POINT(41.569320003460206 -87.32266462874094) bank164401 +164402 POINT(41.15297852081618 -88.41754250465107) bank164402 +164403 POINT(42.269531130578876 -86.95291100559984) bank164403 +164404 POINT(41.313440401548334 -87.46722173538178) bank164404 +164405 POINT(41.50468849856909 -87.80671390575934) bank164405 +164406 POINT(42.798512925273506 -87.45247425810317) bank164406 +164407 POINT(42.79582290153202 -87.61042588042353) bank164407 +164408 POINT(41.74831725220099 -86.91368554981725) bank164408 +164409 POINT(41.808603409056104 -88.38323318779526) bank164409 +164410 POINT(40.983168536500266 -87.92723016052895) bank164410 +164411 POINT(41.68640599175978 -87.28134836121822) bank164411 +164412 POINT(42.524875663392436 -86.84308269292637) bank164412 +164413 POINT(41.29620683501977 -88.25417712693987) bank164413 +164414 POINT(40.88010144287895 -87.13489267012781) bank164414 +164415 POINT(42.72676445821244 -88.13663148855298) bank164415 +164416 POINT(41.559242757137994 -87.72446207550324) bank164416 +164417 POINT(42.251239983609096 -88.44107067783244) bank164417 +164418 POINT(42.59828501397035 -87.59264917343452) bank164418 +164419 POINT(41.94764330560578 -87.87242530728716) bank164419 +164420 POINT(42.46453622169723 -86.64226500632735) bank164420 +164421 POINT(41.867916924924714 -86.89957777391245) bank164421 +164422 POINT(41.359294437578825 -87.41647647427891) bank164422 +164423 POINT(41.50493541212813 -86.92260493780965) bank164423 +164424 POINT(41.43183297770695 -87.67185451449403) bank164424 +164425 POINT(42.62082397078863 -87.97631646344935) bank164425 +164426 POINT(42.566976620381006 -86.9243084972876) bank164426 +164427 POINT(42.547854846865135 -87.38335315562131) bank164427 +164428 POINT(41.81233831167288 -88.02733217578289) bank164428 +164429 POINT(42.41183721434365 -88.2285974276849) bank164429 +164430 POINT(42.699865433486366 -88.59656618929888) bank164430 +164431 POINT(41.485754866651035 -87.53398164166815) bank164431 +164432 POINT(41.76849225307471 -87.25896362210602) bank164432 +164433 POINT(41.69988869045611 -87.41494580650806) bank164433 +164434 POINT(41.657997758068596 -88.48330110084187) bank164434 +164435 POINT(42.11369723574599 -87.39931708478008) bank164435 +164436 POINT(42.192609909694745 -88.15736111025214) bank164436 +164437 POINT(41.11627189488381 -87.18889251319943) bank164437 +164438 POINT(42.27494980145292 -86.96371010043237) bank164438 +164439 POINT(41.30471549978797 -88.31823132889707) bank164439 +164440 POINT(42.704031830384885 -88.30059370729207) bank164440 +164441 POINT(41.696162897977686 -88.03849183906004) bank164441 +164442 POINT(41.38323695063694 -87.66252992547426) bank164442 +164443 POINT(41.76896214385023 -88.39506888934997) bank164443 +164444 POINT(41.550548539212684 -87.54511200684814) bank164444 +164445 POINT(41.58823490600761 -87.96442885755243) bank164445 +164446 POINT(41.1135073813856 -87.05499862436066) bank164446 +164447 POINT(42.01350441813484 -86.91961862547629) bank164447 +164448 POINT(40.939232504363595 -86.63752000066266) bank164448 +164449 POINT(42.302989504333354 -88.57688086453142) bank164449 +164450 POINT(41.76744562324149 -87.25939456149034) bank164450 +164451 POINT(42.85250249778727 -87.83225706024186) bank164451 +164452 POINT(42.743997157683815 -87.92465490330532) bank164452 +164453 POINT(41.50708038280942 -87.01525591648655) bank164453 +164454 POINT(42.28162120174346 -86.88685707595702) bank164454 +164455 POINT(42.58270897448932 -88.30698936572063) bank164455 +164456 POINT(41.539267214172526 -87.31440566927814) bank164456 +164457 POINT(42.321171938443435 -87.93641996954685) bank164457 +164458 POINT(41.93987738723843 -87.25404426341336) bank164458 +164459 POINT(41.051384790849596 -87.10245725856561) bank164459 +164460 POINT(42.475485359533856 -87.78106346562917) bank164460 +164461 POINT(42.11111994687116 -87.66119893098171) bank164461 +164462 POINT(41.3021463408194 -87.78143964813967) bank164462 +164463 POINT(42.05226538312126 -87.15813820167449) bank164463 +164464 POINT(41.851841745093765 -87.49450207857673) bank164464 +164465 POINT(42.56453568787501 -87.14306388838598) bank164465 +164466 POINT(41.79780649676876 -86.78694230999082) bank164466 +164467 POINT(41.518765255151756 -87.28768012700242) bank164467 +164468 POINT(41.73349340330521 -86.71174532382703) bank164468 +164469 POINT(42.860002518575044 -86.9801206696073) bank164469 +164470 POINT(41.96324271343389 -87.23493953533253) bank164470 +164471 POINT(41.720065358705426 -87.44118116654535) bank164471 +164472 POINT(42.09197303416421 -88.56167131617437) bank164472 +164473 POINT(42.03569979710701 -88.00205727080964) bank164473 +164474 POINT(42.33626939353583 -88.32506631182177) bank164474 +164475 POINT(42.79814888605985 -87.54424496265315) bank164475 +164476 POINT(41.43623851036404 -86.79303343840066) bank164476 +164477 POINT(42.87250813798534 -87.31700603323031) bank164477 +164478 POINT(42.8206241152382 -88.55995855787401) bank164478 +164479 POINT(42.360055137809724 -87.44778297359332) bank164479 +164480 POINT(42.467141152570974 -87.19969144764427) bank164480 +164481 POINT(42.15336246477486 -87.12094217425582) bank164481 +164482 POINT(42.622628628120474 -88.125366480219) bank164482 +164483 POINT(41.04005575313187 -87.97370142275625) bank164483 +164484 POINT(42.86945381131815 -87.37115327421803) bank164484 +164485 POINT(42.19377648199837 -88.60878128791613) bank164485 +164486 POINT(41.64978966422428 -88.0851060855681) bank164486 +164487 POINT(41.81038422515644 -88.55172102954677) bank164487 +164488 POINT(41.23613027453481 -86.77228966045492) bank164488 +164489 POINT(41.03921380403081 -86.64679131844971) bank164489 +164490 POINT(41.18319019123806 -87.47449257407672) bank164490 +164491 POINT(41.5699220760751 -87.14522362369244) bank164491 +164492 POINT(42.399601253087724 -87.25751686393181) bank164492 +164493 POINT(42.07820563245697 -87.19019529993028) bank164493 +164494 POINT(41.73404274502187 -88.56253345531589) bank164494 +164495 POINT(41.55953533636968 -86.93603705369259) bank164495 +164496 POINT(42.11537965743912 -86.64722350320922) bank164496 +164497 POINT(41.28518629115845 -87.66752790931798) bank164497 +164498 POINT(42.87060011267927 -87.86547253593723) bank164498 +164499 POINT(41.935248797582105 -88.4395022257628) bank164499 +164500 POINT(41.2800463762636 -87.99680025160167) bank164500 +164501 POINT(42.50493649496871 -87.87074042336654) bank164501 +164502 POINT(41.95038412798923 -87.28198997875614) bank164502 +164503 POINT(41.69322944548281 -86.70002138710619) bank164503 +164504 POINT(42.41961752507612 -88.43850692315296) bank164504 +164505 POINT(42.06770784559028 -88.39658467974478) bank164505 +164506 POINT(40.88698164519566 -86.82372022559049) bank164506 +164507 POINT(42.31257196243847 -86.95518944398461) bank164507 +164508 POINT(41.559902024278735 -86.67136077189733) bank164508 +164509 POINT(42.86287362070169 -88.61293686585691) bank164509 +164510 POINT(42.86671448623771 -88.07274998498914) bank164510 +164511 POINT(42.14487526674199 -88.32242555295906) bank164511 +164512 POINT(41.28742254496232 -88.25794696002103) bank164512 +164513 POINT(41.06543946293285 -87.40695108106078) bank164513 +164514 POINT(42.65213095636362 -87.250843867178) bank164514 +164515 POINT(41.69972049001468 -88.04075063700303) bank164515 +164516 POINT(41.898168893899744 -87.3123206103366) bank164516 +164517 POINT(42.047824157099804 -88.5377996747404) bank164517 +164518 POINT(41.73082778237037 -88.36780531087705) bank164518 +164519 POINT(42.76824695227081 -87.5700629799604) bank164519 +164520 POINT(40.99344496927223 -86.89172996216071) bank164520 +164521 POINT(42.04140473041732 -87.89489223969014) bank164521 +164522 POINT(42.48362565491611 -87.06204552850647) bank164522 +164523 POINT(41.07468357494731 -87.03187343517038) bank164523 +164524 POINT(41.6491092742502 -86.70912646475131) bank164524 +164525 POINT(42.469157848491776 -87.81291138097886) bank164525 +164526 POINT(41.044944625790976 -88.37574341901522) bank164526 +164527 POINT(41.53197827176107 -86.72255682931157) bank164527 +164528 POINT(41.989758218975965 -87.6930552308323) bank164528 +164529 POINT(42.35202539622972 -88.04132319653613) bank164529 +164530 POINT(42.04392240668861 -88.00029333404602) bank164530 +164531 POINT(41.16795622314812 -87.75386220565618) bank164531 +164532 POINT(40.96252661093225 -87.42306229524407) bank164532 +164533 POINT(42.28464468074751 -87.16452769121761) bank164533 +164534 POINT(41.28122011619001 -86.98808043072148) bank164534 +164535 POINT(41.875087892758565 -88.12855093789983) bank164535 +164536 POINT(40.94311066525491 -88.07360673995704) bank164536 +164537 POINT(42.045525194241606 -87.8286144957179) bank164537 +164538 POINT(41.67643437038143 -88.2452915534572) bank164538 +164539 POINT(41.70585543419489 -88.49823308232787) bank164539 +164540 POINT(41.7084672289704 -87.48098332627775) bank164540 +164541 POINT(40.985075964478234 -88.10168973407534) bank164541 +164542 POINT(42.091440034008144 -87.31079437861779) bank164542 +164543 POINT(42.42548221401251 -88.39158965091) bank164543 +164544 POINT(41.64563796714942 -87.36258346724779) bank164544 +164545 POINT(42.034637725632344 -88.47051193824817) bank164545 +164546 POINT(41.80483585492393 -87.48870263928356) bank164546 +164547 POINT(42.293600594129245 -87.04259241464104) bank164547 +164548 POINT(42.01553782510352 -86.78720145127338) bank164548 +164549 POINT(42.19921276134565 -87.22947518524882) bank164549 +164550 POINT(42.307424173686755 -88.20877635405732) bank164550 +164551 POINT(42.02798603875889 -87.28812107324454) bank164551 +164552 POINT(41.54098925363329 -88.29461094099565) bank164552 +164553 POINT(41.753844799105345 -88.09987072095043) bank164553 +164554 POINT(42.38692463102432 -88.58787684254709) bank164554 +164555 POINT(42.46184360235479 -86.7662513449933) bank164555 +164556 POINT(41.12105882769337 -87.18011993842282) bank164556 +164557 POINT(41.22015446877233 -87.48329414757853) bank164557 +164558 POINT(41.57690381632142 -87.02073347087584) bank164558 +164559 POINT(41.00582245186904 -87.71009529043408) bank164559 +164560 POINT(41.746000587864074 -87.59761110415128) bank164560 +164561 POINT(41.231038510640786 -88.51164585357206) bank164561 +164562 POINT(40.986841874520984 -88.55483966198933) bank164562 +164563 POINT(41.0663283503034 -88.54353591004974) bank164563 +164564 POINT(40.94444868617672 -88.05790444812915) bank164564 +164565 POINT(42.196083500391964 -88.19463548536497) bank164565 +164566 POINT(41.53233560873215 -86.70395688238848) bank164566 +164567 POINT(41.53292329906676 -88.25613980116924) bank164567 +164568 POINT(41.707253312855954 -87.34772159320504) bank164568 +164569 POINT(41.71314528471019 -86.86251795627491) bank164569 +164570 POINT(42.05941066759477 -87.60743346615713) bank164570 +164571 POINT(42.41422281845593 -87.09573196698703) bank164571 +164572 POINT(41.40801089367657 -87.17065640883997) bank164572 +164573 POINT(41.26714248564904 -87.61494805235009) bank164573 +164574 POINT(42.30588444025979 -87.37706496511493) bank164574 +164575 POINT(40.88857707580885 -88.41368853226261) bank164575 +164576 POINT(41.60616303300155 -87.4854227843669) bank164576 +164577 POINT(41.66827079846819 -87.90969874734094) bank164577 +164578 POINT(40.89628652519953 -87.88666465905321) bank164578 +164579 POINT(41.787384838827116 -87.68874522457733) bank164579 +164580 POINT(42.54273198345596 -87.0931729757341) bank164580 +164581 POINT(41.87399401782697 -86.76315331398362) bank164581 +164582 POINT(41.76683996582956 -88.5459444749181) bank164582 +164583 POINT(41.11180691175907 -88.00344450763909) bank164583 +164584 POINT(42.254582542535125 -86.64238559335165) bank164584 +164585 POINT(41.58078984855501 -88.51952347174004) bank164585 +164586 POINT(42.07459168576477 -87.6826373970196) bank164586 +164587 POINT(41.718440732565696 -88.55430073954899) bank164587 +164588 POINT(42.24201037498213 -87.39311072338191) bank164588 +164589 POINT(41.24164732344242 -86.85765487517791) bank164589 +164590 POINT(42.36472898815755 -88.28440510781232) bank164590 +164591 POINT(41.089563112404186 -88.35873068834053) bank164591 +164592 POINT(42.870489788926605 -88.5028865555297) bank164592 +164593 POINT(42.37199431711359 -88.35890855404655) bank164593 +164594 POINT(42.33841802859144 -87.45011924457921) bank164594 +164595 POINT(41.01622786327054 -88.45495371442583) bank164595 +164596 POINT(42.114856367455815 -87.91178363515947) bank164596 +164597 POINT(41.8703501927571 -87.81505906667675) bank164597 +164598 POINT(41.92734413784272 -87.45025509960016) bank164598 +164599 POINT(41.884569995686554 -88.5519875419101) bank164599 +164600 POINT(42.585870077596 -87.52703101144161) bank164600 +164601 POINT(40.91021845393336 -86.81394675433855) bank164601 +164602 POINT(42.46496752440771 -87.04263122629575) bank164602 +164603 POINT(42.58812063299661 -86.75966881298429) bank164603 +164604 POINT(41.26011185277718 -88.29103749516625) bank164604 +164605 POINT(41.537602573889934 -87.69492798064289) bank164605 +164606 POINT(41.84155709449066 -88.26181445340882) bank164606 +164607 POINT(42.24571344676165 -88.0724361658735) bank164607 +164608 POINT(41.79475635699034 -87.59355808322161) bank164608 +164609 POINT(42.18447577192438 -86.77160030491288) bank164609 +164610 POINT(41.711418711325 -87.53718092642288) bank164610 +164611 POINT(41.51231160925284 -88.02507167939972) bank164611 +164612 POINT(40.975652763144645 -87.63511030722296) bank164612 +164613 POINT(41.62852170888814 -88.59019087826925) bank164613 +164614 POINT(41.44836546594291 -87.63700807048312) bank164614 +164615 POINT(41.53711153202375 -88.30713835760686) bank164615 +164616 POINT(41.21065059616432 -88.13553207863113) bank164616 +164617 POINT(42.78140961192363 -88.46911133449323) bank164617 +164618 POINT(42.09293956001031 -87.77954183300115) bank164618 +164619 POINT(41.76452970411849 -87.33105484129945) bank164619 +164620 POINT(42.71514640616713 -87.7371954667155) bank164620 +164621 POINT(40.87956379128158 -87.56968746352037) bank164621 +164622 POINT(41.150778438061806 -87.52388694259264) bank164622 +164623 POINT(41.85862461186309 -87.00090256998082) bank164623 +164624 POINT(42.84495487440071 -86.82573826297116) bank164624 +164625 POINT(42.607258454475 -87.48016787536383) bank164625 +164626 POINT(41.435357693581615 -87.88959222720489) bank164626 +164627 POINT(42.388565176992735 -88.40384063027071) bank164627 +164628 POINT(41.67709063707609 -88.2360522227604) bank164628 +164629 POINT(42.06310858693649 -88.18540820269487) bank164629 +164630 POINT(40.939502890522604 -88.10941789560519) bank164630 +164631 POINT(41.18544823976821 -88.21651757707843) bank164631 +164632 POINT(42.07457540000305 -88.51151011072562) bank164632 +164633 POINT(41.95480214071885 -86.97750957095036) bank164633 +164634 POINT(41.323908340106 -88.13348531635604) bank164634 +164635 POINT(41.21441091998933 -87.98612952958844) bank164635 +164636 POINT(42.179084897164856 -87.99689628856821) bank164636 +164637 POINT(42.30462063337237 -87.29893495431693) bank164637 +164638 POINT(41.38920618915021 -88.39675003227143) bank164638 +164639 POINT(40.967194287367505 -87.42836614635146) bank164639 +164640 POINT(41.08609142739354 -87.13694852001521) bank164640 +164641 POINT(42.08065170506464 -88.52409980337443) bank164641 +164642 POINT(41.96505997989089 -86.89742868342887) bank164642 +164643 POINT(41.10330578640685 -87.9799863615094) bank164643 +164644 POINT(42.16599459296758 -88.16680458541099) bank164644 +164645 POINT(42.03141940844659 -88.20964742065645) bank164645 +164646 POINT(42.241868458832826 -87.45380501672925) bank164646 +164647 POINT(41.46071317906464 -87.20156096322036) bank164647 +164648 POINT(41.028890705458394 -86.76343315953903) bank164648 +164649 POINT(41.202122519883005 -87.49573165441657) bank164649 +164650 POINT(42.39335551421277 -88.42035571537555) bank164650 +164651 POINT(41.506997521593405 -87.82502939607008) bank164651 +164652 POINT(41.74398922540391 -87.2230394181426) bank164652 +164653 POINT(41.41844396807999 -87.28648590304957) bank164653 +164654 POINT(41.4684773821737 -88.20344892499158) bank164654 +164655 POINT(41.56639945336277 -87.53671941075686) bank164655 +164656 POINT(42.16850766126647 -87.35155693112026) bank164656 +164657 POINT(42.87387465308781 -88.06988369690971) bank164657 +164658 POINT(41.80278868688064 -88.26890581847175) bank164658 +164659 POINT(42.020085901761796 -87.56140138155196) bank164659 +164660 POINT(41.997699596533266 -86.74129374945926) bank164660 +164661 POINT(41.56580973360538 -88.35965915516375) bank164661 +164662 POINT(42.297260975897515 -87.94123781368258) bank164662 +164663 POINT(42.60602847325151 -86.81063836838528) bank164663 +164664 POINT(42.36861524987399 -86.86921695772632) bank164664 +164665 POINT(40.98646427161054 -88.33401191947344) bank164665 +164666 POINT(40.89896536998179 -87.42291748875361) bank164666 +164667 POINT(41.402230128464964 -87.8989928704196) bank164667 +164668 POINT(41.43329606076433 -87.43662337592248) bank164668 +164669 POINT(42.282653369185724 -88.15947966609086) bank164669 +164670 POINT(41.23302414129336 -87.87121506681328) bank164670 +164671 POINT(42.119194552545096 -87.196835521727) bank164671 +164672 POINT(41.92150425164776 -88.29494840584405) bank164672 +164673 POINT(42.07217436484686 -87.58377336403915) bank164673 +164674 POINT(42.580762309237436 -87.90451394257863) bank164674 +164675 POINT(42.83892638506443 -88.507707099101) bank164675 +164676 POINT(41.431293005106 -87.7389290596889) bank164676 +164677 POINT(41.002678758128056 -88.05882754596254) bank164677 +164678 POINT(40.99772546913802 -88.05548374420984) bank164678 +164679 POINT(41.76049056786145 -88.45361947080944) bank164679 +164680 POINT(42.139963514868136 -87.60179677756273) bank164680 +164681 POINT(41.70653272315732 -87.87844914074925) bank164681 +164682 POINT(41.94037459785235 -88.32170792826453) bank164682 +164683 POINT(41.225565021085046 -88.19032309045106) bank164683 +164684 POINT(40.95418657628842 -87.72407475753639) bank164684 +164685 POINT(41.52555732459264 -88.60330374112792) bank164685 +164686 POINT(42.11845609424705 -88.12965551455562) bank164686 +164687 POINT(41.30172926023558 -87.7887115718151) bank164687 +164688 POINT(41.00802161354486 -86.76011942909692) bank164688 +164689 POINT(42.35009214050596 -87.05242977368569) bank164689 +164690 POINT(42.633403136309084 -87.56410612572203) bank164690 +164691 POINT(42.22634713293988 -88.02319216738631) bank164691 +164692 POINT(42.60170463471104 -88.50351459430848) bank164692 +164693 POINT(42.64519491398074 -88.26060797556576) bank164693 +164694 POINT(42.280949299109864 -87.8761608091606) bank164694 +164695 POINT(42.829439161049024 -86.66857309374242) bank164695 +164696 POINT(41.812609761137246 -88.25627519603347) bank164696 +164697 POINT(42.27364604512013 -87.89655814553683) bank164697 +164698 POINT(41.04752983394196 -88.319777648719) bank164698 +164699 POINT(41.534735315961186 -87.02837418350458) bank164699 +164700 POINT(41.07257116535932 -88.27694411160235) bank164700 +164701 POINT(41.755508174583795 -87.14061892160726) bank164701 +164702 POINT(41.331273396040444 -87.47693076634319) bank164702 +164703 POINT(41.01283477443265 -86.93741288438545) bank164703 +164704 POINT(42.70400855727349 -87.01400730354673) bank164704 +164705 POINT(41.61422223506336 -88.12169304772883) bank164705 +164706 POINT(41.91787194434574 -86.79659676036799) bank164706 +164707 POINT(42.196155283191366 -88.03087775153615) bank164707 +164708 POINT(42.18781404681356 -86.87712229780126) bank164708 +164709 POINT(42.02084408759 -87.20203840607039) bank164709 +164710 POINT(42.451604962453665 -86.80757582631986) bank164710 +164711 POINT(41.67844976372988 -87.62207226476704) bank164711 +164712 POINT(41.05615666208983 -87.01720139480204) bank164712 +164713 POINT(42.22486755697796 -87.094829441212) bank164713 +164714 POINT(42.50223217492537 -88.44175940568908) bank164714 +164715 POINT(41.211591510393006 -87.74729229528032) bank164715 +164716 POINT(42.70255533109205 -87.32750429152867) bank164716 +164717 POINT(42.14578541157363 -88.47984126016722) bank164717 +164718 POINT(42.74791509806038 -88.36528320042541) bank164718 +164719 POINT(41.853605523837466 -87.844937704024) bank164719 +164720 POINT(42.661583004142464 -87.39562098671722) bank164720 +164721 POINT(41.860896431073236 -87.2917569375748) bank164721 +164722 POINT(41.487444620288066 -87.35901033068261) bank164722 +164723 POINT(42.61294558219329 -87.18237445672113) bank164723 +164724 POINT(41.08833934393215 -86.7054494785252) bank164724 +164725 POINT(41.53188509948367 -88.16410962941268) bank164725 +164726 POINT(40.912245777143966 -86.7654005134666) bank164726 +164727 POINT(41.819481469179216 -87.52251708709086) bank164727 +164728 POINT(42.61260433988245 -86.80084813201842) bank164728 +164729 POINT(42.71744795006977 -87.517703249251) bank164729 +164730 POINT(40.98784038498432 -87.37492354788988) bank164730 +164731 POINT(42.47748808916603 -88.41524649256397) bank164731 +164732 POINT(42.51564421748961 -88.14479803100181) bank164732 +164733 POINT(42.09497047484848 -88.46444648051192) bank164733 +164734 POINT(40.911458875219175 -88.08660560360026) bank164734 +164735 POINT(42.806713244692254 -87.04909873946116) bank164735 +164736 POINT(42.5269704473274 -86.63132450082932) bank164736 +164737 POINT(41.31461709543077 -87.24611795234426) bank164737 +164738 POINT(40.90089635184844 -87.77375812107918) bank164738 +164739 POINT(41.49400412662793 -87.93676875889471) bank164739 +164740 POINT(42.2449042104363 -87.66351569128567) bank164740 +164741 POINT(41.5509613478537 -88.5562865981218) bank164741 +164742 POINT(41.58192070225311 -86.68355190119348) bank164742 +164743 POINT(41.63552121595429 -86.86084335666062) bank164743 +164744 POINT(41.141836204340606 -87.29137478862039) bank164744 +164745 POINT(41.475524275741726 -87.78710804377225) bank164745 +164746 POINT(42.8166044885778 -87.36884530162189) bank164746 +164747 POINT(41.396417847460896 -86.80141776575552) bank164747 +164748 POINT(40.95229925669674 -86.69927959690496) bank164748 +164749 POINT(41.67005085056853 -88.591175608019) bank164749 +164750 POINT(41.692354312876844 -87.25595509627396) bank164750 +164751 POINT(41.983966450997556 -88.56134213318398) bank164751 +164752 POINT(41.30926085582638 -86.75085415072758) bank164752 +164753 POINT(42.681542338112074 -87.08163447492521) bank164753 +164754 POINT(41.22035638454625 -87.19700360702687) bank164754 +164755 POINT(41.02941030110662 -87.38622718928835) bank164755 +164756 POINT(41.02086511300474 -87.11584253684204) bank164756 +164757 POINT(42.874952289791594 -88.0030303892747) bank164757 +164758 POINT(42.04682423219467 -87.34572664198754) bank164758 +164759 POINT(40.92985820799784 -86.88912574316576) bank164759 +164760 POINT(42.06541258482964 -87.94764291367005) bank164760 +164761 POINT(41.04885898953245 -88.20732217624938) bank164761 +164762 POINT(41.254273994523885 -88.40419051325955) bank164762 +164763 POINT(42.05077139287879 -86.79403510621306) bank164763 +164764 POINT(42.493719942624494 -88.56601078680198) bank164764 +164765 POINT(42.063724098132774 -87.09412817762629) bank164765 +164766 POINT(41.63194340510527 -87.81279000448649) bank164766 +164767 POINT(41.401208875685036 -88.4781004434964) bank164767 +164768 POINT(42.801375287566174 -88.14807924710213) bank164768 +164769 POINT(42.586214185417674 -87.33643987213046) bank164769 +164770 POINT(41.12701297677219 -86.94041510536154) bank164770 +164771 POINT(41.40258117072956 -88.07435951465438) bank164771 +164772 POINT(42.49559895948765 -88.34908365069674) bank164772 +164773 POINT(42.03283468096177 -87.13098078465711) bank164773 +164774 POINT(42.27460205828525 -87.1093078728635) bank164774 +164775 POINT(41.212822733152095 -87.57557807236454) bank164775 +164776 POINT(41.93045071469781 -87.44824788211008) bank164776 +164777 POINT(41.55155067955169 -87.39353858308964) bank164777 +164778 POINT(41.986311559577175 -88.45273791499993) bank164778 +164779 POINT(42.47281532183697 -86.74913475856027) bank164779 +164780 POINT(41.84520151713271 -88.07196676003296) bank164780 +164781 POINT(40.91220273455275 -87.05358281259714) bank164781 +164782 POINT(42.08634222581121 -88.08107351781223) bank164782 +164783 POINT(41.13596662756562 -86.64011331085496) bank164783 +164784 POINT(41.18272904264893 -87.86024437008076) bank164784 +164785 POINT(42.58189725655815 -86.89700349701293) bank164785 +164786 POINT(42.13270486188459 -87.89626117095052) bank164786 +164787 POINT(42.47751656007083 -86.72133643401854) bank164787 +164788 POINT(42.332408214392025 -88.08811933063375) bank164788 +164789 POINT(42.699297385530784 -87.73646742078029) bank164789 +164790 POINT(41.17227909587439 -87.39332214532791) bank164790 +164791 POINT(40.974769723406624 -86.90408669640517) bank164791 +164792 POINT(41.70485965231995 -88.04902436225795) bank164792 +164793 POINT(42.51528973364401 -87.89483846073563) bank164793 +164794 POINT(42.29201826271993 -87.4824199503285) bank164794 +164795 POINT(41.873830590100674 -87.39384816104165) bank164795 +164796 POINT(41.998543186235004 -88.26919042555427) bank164796 +164797 POINT(41.50591234672841 -87.96199693391817) bank164797 +164798 POINT(41.95477053021098 -88.53915892650578) bank164798 +164799 POINT(41.68894114991433 -88.5286133431097) bank164799 +164800 POINT(40.898305612697754 -87.10936741502161) bank164800 +164801 POINT(40.88051181781181 -87.89416442340548) bank164801 +164802 POINT(41.96180641228008 -88.46318291984248) bank164802 +164803 POINT(41.33185976732531 -87.56568440855989) bank164803 +164804 POINT(42.54617169377992 -87.64461831117306) bank164804 +164805 POINT(42.81653450670371 -87.98850956024869) bank164805 +164806 POINT(42.519021424472456 -88.01061353552991) bank164806 +164807 POINT(42.212511986787796 -86.81763219419965) bank164807 +164808 POINT(40.89966714612867 -88.05558570648577) bank164808 +164809 POINT(41.03073325355566 -87.29318384476292) bank164809 +164810 POINT(42.680173215915815 -88.33178679686038) bank164810 +164811 POINT(41.24210583191564 -86.68146021544884) bank164811 +164812 POINT(41.62605870839832 -87.09166550439281) bank164812 +164813 POINT(42.24375314602432 -86.98733888938068) bank164813 +164814 POINT(41.841781155918696 -88.49738459380521) bank164814 +164815 POINT(42.758472147418836 -87.24744852105039) bank164815 +164816 POINT(41.78207010280892 -88.09969480814453) bank164816 +164817 POINT(41.14938443531902 -88.37983277963875) bank164817 +164818 POINT(41.79887641166597 -86.92145489997095) bank164818 +164819 POINT(41.505382550902794 -87.8999304752068) bank164819 +164820 POINT(42.139977257584995 -88.23367499998275) bank164820 +164821 POINT(42.784942301765085 -88.04938814714738) bank164821 +164822 POINT(42.41102693983992 -87.53562220374258) bank164822 +164823 POINT(42.49362973477181 -87.27827694838457) bank164823 +164824 POINT(42.284144938358445 -87.89144507825895) bank164824 +164825 POINT(42.348291631706225 -87.36453502191627) bank164825 +164826 POINT(41.82204338463917 -88.43565656642656) bank164826 +164827 POINT(42.35078994252826 -87.91949411732423) bank164827 +164828 POINT(41.06667518472609 -86.9476097645362) bank164828 +164829 POINT(41.392487387142204 -87.94380680064829) bank164829 +164830 POINT(42.31085925710535 -87.56748712616476) bank164830 +164831 POINT(41.671049659710995 -86.87558892771055) bank164831 +164832 POINT(42.412268026414196 -88.1671454677479) bank164832 +164833 POINT(41.59981240726794 -86.7300716571046) bank164833 +164834 POINT(41.36232722476879 -88.57255946498282) bank164834 +164835 POINT(41.29471748293118 -87.0989724909367) bank164835 +164836 POINT(42.50662061371914 -87.53280447934723) bank164836 +164837 POINT(41.10632343400133 -86.66982440315208) bank164837 +164838 POINT(42.78533924749424 -86.79717880484942) bank164838 +164839 POINT(41.734584701365044 -86.69258300150902) bank164839 +164840 POINT(41.41065725112618 -87.56835660576432) bank164840 +164841 POINT(41.69144221415436 -87.30433722311389) bank164841 +164842 POINT(42.00095251627231 -88.00349668361008) bank164842 +164843 POINT(41.554526567220144 -86.66179526669112) bank164843 +164844 POINT(42.32180336108463 -87.3527372901186) bank164844 +164845 POINT(41.59710929412538 -87.26700421837853) bank164845 +164846 POINT(42.461713144525525 -86.89651949192825) bank164846 +164847 POINT(42.77108595600248 -88.12984442799034) bank164847 +164848 POINT(42.626941405441265 -87.50731492709178) bank164848 +164849 POINT(42.748406400345516 -86.86255641350732) bank164849 +164850 POINT(41.73646237334977 -88.59248560249283) bank164850 +164851 POINT(41.61738981541016 -87.80739269480296) bank164851 +164852 POINT(41.77602975109601 -88.1369803102482) bank164852 +164853 POINT(40.97437204188672 -86.64482715975502) bank164853 +164854 POINT(42.167523855942484 -87.6759151939704) bank164854 +164855 POINT(42.075607679729195 -87.95233439973777) bank164855 +164856 POINT(41.95569788799959 -88.23170146900551) bank164856 +164857 POINT(41.16845825961511 -87.5184596083483) bank164857 +164858 POINT(42.679044731562584 -87.22578994717568) bank164858 +164859 POINT(42.67220146159411 -88.34503618322277) bank164859 +164860 POINT(42.34495740893009 -86.66400098509803) bank164860 +164861 POINT(41.66081202921855 -87.34884067363221) bank164861 +164862 POINT(42.52960044795207 -86.9087882783943) bank164862 +164863 POINT(41.180045072756286 -87.57566234837114) bank164863 +164864 POINT(41.553501992916964 -86.96756675382774) bank164864 +164865 POINT(42.8112643408812 -86.66094636854434) bank164865 +164866 POINT(42.23039888716697 -88.46593387382681) bank164866 +164867 POINT(41.21327494151558 -86.8119629174594) bank164867 +164868 POINT(41.95936297653869 -87.11706547046991) bank164868 +164869 POINT(42.115430730430994 -87.39186590122189) bank164869 +164870 POINT(41.20109216891737 -87.27066602455282) bank164870 +164871 POINT(41.41815652117585 -88.52930771595206) bank164871 +164872 POINT(42.18834893872186 -87.02026566660159) bank164872 +164873 POINT(42.64139056023973 -88.50204056479969) bank164873 +164874 POINT(42.4584531791345 -88.07531043922646) bank164874 +164875 POINT(41.96254382322102 -87.56552900565812) bank164875 +164876 POINT(41.12436084249834 -88.30206114054216) bank164876 +164877 POINT(42.86855837562755 -86.96111062919053) bank164877 +164878 POINT(41.51719148461046 -88.55172321599903) bank164878 +164879 POINT(42.087645230272656 -88.35805499741171) bank164879 +164880 POINT(41.574788007498626 -87.62546968024148) bank164880 +164881 POINT(41.14070905217531 -86.97900847960042) bank164881 +164882 POINT(42.3421365826763 -87.57731694945868) bank164882 +164883 POINT(41.66451407326078 -87.00525732028535) bank164883 +164884 POINT(42.63086550237504 -87.16380825136125) bank164884 +164885 POINT(42.421011965281586 -86.79312659449346) bank164885 +164886 POINT(41.08940134998325 -86.68547006393199) bank164886 +164887 POINT(41.04444455749304 -88.07878251982451) bank164887 +164888 POINT(42.27035791985758 -86.94934603598317) bank164888 +164889 POINT(42.263689516841346 -88.20528619034557) bank164889 +164890 POINT(42.24943258948337 -87.70261670807784) bank164890 +164891 POINT(41.23089587153041 -86.76890588611812) bank164891 +164892 POINT(41.85046539167602 -88.23657103589159) bank164892 +164893 POINT(41.11506969404196 -88.23991726114032) bank164893 +164894 POINT(41.89225261937132 -87.56593336379196) bank164894 +164895 POINT(42.265795038915726 -87.36781168653637) bank164895 +164896 POINT(41.181157326881106 -87.84540673486751) bank164896 +164897 POINT(42.67167595535192 -86.69948403264392) bank164897 +164898 POINT(42.7702520571918 -88.233783650714) bank164898 +164899 POINT(41.900285241630115 -87.09250631016222) bank164899 +164900 POINT(42.39544486061044 -87.32743923861337) bank164900 +164901 POINT(42.18471406744241 -88.2270806809796) bank164901 +164902 POINT(41.69431970386885 -87.00785403386467) bank164902 +164903 POINT(41.42314267263208 -88.6270739486007) bank164903 +164904 POINT(42.56778362018781 -88.53958042811202) bank164904 +164905 POINT(42.681926426390504 -86.98363508720074) bank164905 +164906 POINT(41.52081388837227 -87.13102990879645) bank164906 +164907 POINT(41.1538473164199 -88.15006722960888) bank164907 +164908 POINT(42.426588735679665 -88.61931431740567) bank164908 +164909 POINT(42.48008599888885 -88.56290604610452) bank164909 +164910 POINT(42.71420775296289 -86.71620821185185) bank164910 +164911 POINT(41.85975621226279 -87.32561383037509) bank164911 +164912 POINT(41.64311604956451 -88.42109275026209) bank164912 +164913 POINT(42.67264683753509 -88.35384936749921) bank164913 +164914 POINT(41.04677501833157 -86.64097600831123) bank164914 +164915 POINT(41.14208799438934 -87.4569539643956) bank164915 +164916 POINT(41.005305852902765 -88.2515431489515) bank164916 +164917 POINT(42.08330361694657 -88.3885015160896) bank164917 +164918 POINT(41.93633073121498 -87.54653395846942) bank164918 +164919 POINT(42.197825468046055 -86.97872992427442) bank164919 +164920 POINT(42.712412962470246 -87.82187345693215) bank164920 +164921 POINT(42.66076916263031 -87.91546016493619) bank164921 +164922 POINT(42.439985073144705 -86.89380758934072) bank164922 +164923 POINT(42.357357661035756 -86.81383921240126) bank164923 +164924 POINT(41.15641103599142 -88.33496191895303) bank164924 +164925 POINT(42.261417560031234 -87.61118595051474) bank164925 +164926 POINT(41.24242823764266 -87.76637323650748) bank164926 +164927 POINT(41.590903169337864 -87.18055833881392) bank164927 +164928 POINT(42.05705785311109 -88.35311708055318) bank164928 +164929 POINT(41.44205075099936 -87.75867736435599) bank164929 +164930 POINT(41.48055988175669 -87.4957398342053) bank164930 +164931 POINT(41.97212566123878 -88.180694992523) bank164931 +164932 POINT(40.96843328256563 -87.49757134898913) bank164932 +164933 POINT(42.740879628163974 -88.5366491883954) bank164933 +164934 POINT(41.750476531176346 -87.44778989927993) bank164934 +164935 POINT(41.41984318705029 -88.56557153572736) bank164935 +164936 POINT(41.9006907027376 -87.15898438598157) bank164936 +164937 POINT(42.76406728336103 -87.52031548685025) bank164937 +164938 POINT(42.26285899564482 -87.19721102839819) bank164938 +164939 POINT(41.10887745821117 -87.84776638333331) bank164939 +164940 POINT(42.526179066116555 -88.10264046179284) bank164940 +164941 POINT(41.24211886697692 -87.48563191048645) bank164941 +164942 POINT(41.81856293509804 -87.32197679944805) bank164942 +164943 POINT(42.72900209266492 -87.4238991289097) bank164943 +164944 POINT(42.39069066787268 -87.68932332904059) bank164944 +164945 POINT(41.41440896628609 -87.43382004946453) bank164945 +164946 POINT(41.942933999227265 -87.13684217111144) bank164946 +164947 POINT(42.683893730925085 -87.48062468421182) bank164947 +164948 POINT(42.280329193161904 -88.06544810504272) bank164948 +164949 POINT(41.25919366442464 -88.22951466192085) bank164949 +164950 POINT(41.9356393280725 -87.70841346172696) bank164950 +164951 POINT(42.55736135876904 -86.63188630735577) bank164951 +164952 POINT(42.700667230977565 -88.38858709017023) bank164952 +164953 POINT(41.647171546204056 -87.96470207087347) bank164953 +164954 POINT(42.71541468906431 -88.39197936399994) bank164954 +164955 POINT(41.98664476219683 -87.01064386744477) bank164955 +164956 POINT(41.741449331874804 -87.82748299037686) bank164956 +164957 POINT(41.38416057993994 -86.6640822795509) bank164957 +164958 POINT(42.682651555167915 -87.44260190992073) bank164958 +164959 POINT(41.643613011173784 -87.76011856839818) bank164959 +164960 POINT(42.59587937864316 -87.88527567693014) bank164960 +164961 POINT(42.47625612643469 -86.95259372667815) bank164961 +164962 POINT(42.567922715265155 -87.70057175645032) bank164962 +164963 POINT(42.65689735864466 -87.3372390552764) bank164963 +164964 POINT(42.42080729379023 -87.73541673114792) bank164964 +164965 POINT(41.53740359904377 -87.36674937043914) bank164965 +164966 POINT(42.44968567404266 -88.57660088017484) bank164966 +164967 POINT(42.136342446822084 -87.61048169722548) bank164967 +164968 POINT(42.69385940659408 -87.75139812891595) bank164968 +164969 POINT(40.98893762956395 -86.75368199796192) bank164969 +164970 POINT(42.708032484350184 -87.06888124788578) bank164970 +164971 POINT(41.66606748745383 -88.50068969421001) bank164971 +164972 POINT(41.59366425547947 -87.23615750846606) bank164972 +164973 POINT(41.922091049216235 -87.32592379269427) bank164973 +164974 POINT(41.58111940628608 -88.41596965194537) bank164974 +164975 POINT(41.58799999342499 -87.85955501766993) bank164975 +164976 POINT(41.20098498813499 -87.76755500214412) bank164976 +164977 POINT(41.37354077130642 -87.96000971447623) bank164977 +164978 POINT(41.28747692066809 -88.10558753952233) bank164978 +164979 POINT(41.82578640944236 -88.02546120585703) bank164979 +164980 POINT(42.70469590000408 -86.7919585369833) bank164980 +164981 POINT(42.645141408814325 -88.45281615101813) bank164981 +164982 POINT(42.56897363090938 -87.68018572910137) bank164982 +164983 POINT(41.31042437782945 -88.33267776654162) bank164983 +164984 POINT(42.649229522462065 -86.92193905245678) bank164984 +164985 POINT(41.75434251205663 -87.56210428133576) bank164985 +164986 POINT(41.97802047264875 -88.28630177105094) bank164986 +164987 POINT(42.37139169715077 -87.9906183502212) bank164987 +164988 POINT(40.93582554297602 -87.99546243469426) bank164988 +164989 POINT(41.7758840677694 -86.98333380130217) bank164989 +164990 POINT(41.69620186650796 -87.99696722175547) bank164990 +164991 POINT(41.025499615491526 -87.40989294706952) bank164991 +164992 POINT(42.146804925481554 -87.1854513460567) bank164992 +164993 POINT(41.3256503964771 -87.91837445280896) bank164993 +164994 POINT(41.45764789454015 -88.58162455401254) bank164994 +164995 POINT(41.07943910889518 -88.29965333662021) bank164995 +164996 POINT(42.3429022288455 -86.97346006368582) bank164996 +164997 POINT(41.14604214262376 -86.6579255559301) bank164997 +164998 POINT(42.130571608503615 -87.51097332025732) bank164998 +164999 POINT(41.525517283070535 -88.34661523761112) bank164999 +165000 POINT(42.2093051025365 -87.3071001284845) bank165000 +165001 POINT(41.97446923607166 -87.95470940872079) bank165001 +165002 POINT(41.940917903412334 -88.45939539734144) bank165002 +165003 POINT(41.54975085835962 -87.08045284231422) bank165003 +165004 POINT(41.33855357061937 -88.48582163435279) bank165004 +165005 POINT(41.74542809356987 -87.48824640105796) bank165005 +165006 POINT(42.829917951327936 -87.73636712814132) bank165006 +165007 POINT(40.910311167848235 -88.0668307303005) bank165007 +165008 POINT(42.59839528693563 -87.95391860495158) bank165008 +165009 POINT(41.233473255703856 -87.22412112676636) bank165009 +165010 POINT(42.80551074170203 -87.99425626967925) bank165010 +165011 POINT(40.95940989442345 -86.70419048358463) bank165011 +165012 POINT(41.994960629935406 -87.81635442800177) bank165012 +165013 POINT(42.867480740131136 -87.19323186585788) bank165013 +165014 POINT(42.14739708282923 -88.13142742611309) bank165014 +165015 POINT(42.74916804960287 -86.99325482847608) bank165015 +165016 POINT(42.359396650841234 -87.14172237273526) bank165016 +165017 POINT(42.61938302708871 -88.58905412250085) bank165017 +165018 POINT(41.57303220990333 -88.51842193512333) bank165018 +165019 POINT(41.35708688103148 -86.88941530203203) bank165019 +165020 POINT(41.82578452893582 -88.43259555543224) bank165020 +165021 POINT(42.6695958794567 -86.80323589246089) bank165021 +165022 POINT(41.28045591976966 -88.23786382799645) bank165022 +165023 POINT(41.92472537265193 -86.9029564481082) bank165023 +165024 POINT(41.8926110848113 -86.73966532154179) bank165024 +165025 POINT(41.8302115134523 -87.82000933676464) bank165025 +165026 POINT(41.67526794661679 -86.66642031782528) bank165026 +165027 POINT(42.46397391126573 -86.6796822422813) bank165027 +165028 POINT(42.69754038070545 -87.12615854373652) bank165028 +165029 POINT(42.6070952160194 -88.12142793503172) bank165029 +165030 POINT(41.04769647955938 -86.63300953615305) bank165030 +165031 POINT(42.39998668273859 -87.13892272514903) bank165031 +165032 POINT(42.714208648918564 -88.59456920270361) bank165032 +165033 POINT(41.58597079967705 -87.9012623844218) bank165033 +165034 POINT(42.309483503148954 -87.55214025960821) bank165034 +165035 POINT(42.70106150008527 -88.58452165620942) bank165035 +165036 POINT(41.03792135088045 -86.93766127276898) bank165036 +165037 POINT(42.49355063284594 -87.74638892926592) bank165037 +165038 POINT(42.18379508983499 -87.83659727323906) bank165038 +165039 POINT(41.071163442326394 -88.52098621436451) bank165039 +165040 POINT(41.633390380495314 -88.0684730538398) bank165040 +165041 POINT(42.759570887162674 -87.70555898628159) bank165041 +165042 POINT(41.613609044537846 -87.04736092362576) bank165042 +165043 POINT(41.62268998031438 -87.07175333476805) bank165043 +165044 POINT(41.61936869477958 -87.96190331955715) bank165044 +165045 POINT(41.75453499976871 -87.6730271837925) bank165045 +165046 POINT(42.30391520798006 -88.34728757710275) bank165046 +165047 POINT(41.03999607054421 -88.24145365245408) bank165047 +165048 POINT(41.296958198382605 -86.95551090833078) bank165048 +165049 POINT(41.82177686843335 -88.41328225131696) bank165049 +165050 POINT(41.52667277306139 -88.00587172920524) bank165050 +165051 POINT(42.36781008272279 -86.74893122531358) bank165051 +165052 POINT(42.21757973658804 -86.79934850803026) bank165052 +165053 POINT(41.29069928651531 -86.85148051804482) bank165053 +165054 POINT(41.9577779952126 -88.54252058153143) bank165054 +165055 POINT(42.05097795342072 -87.79603640234092) bank165055 +165056 POINT(41.67165040424267 -87.86556710194) bank165056 +165057 POINT(41.722309549023606 -87.04010152643598) bank165057 +165058 POINT(41.12505334957664 -87.35028327844647) bank165058 +165059 POINT(42.72689304219994 -87.37051826680641) bank165059 +165060 POINT(41.24830239021745 -86.84282589665371) bank165060 +165061 POINT(41.62895845805849 -87.51779527257413) bank165061 +165062 POINT(41.169746726884235 -87.7388414722014) bank165062 +165063 POINT(40.98389939617802 -87.87664125409442) bank165063 +165064 POINT(41.104910516500276 -88.59071467381209) bank165064 +165065 POINT(42.4578407040739 -87.12570312976848) bank165065 +165066 POINT(42.57432154311176 -87.03365945737352) bank165066 +165067 POINT(42.07001025912208 -86.6498355930178) bank165067 +165068 POINT(41.26512946385563 -87.9007345380584) bank165068 +165069 POINT(41.12033348403586 -87.81208893366082) bank165069 +165070 POINT(41.615930292306594 -87.42980379309337) bank165070 +165071 POINT(41.46293363791007 -87.72477478436885) bank165071 +165072 POINT(42.424629227748405 -87.47279010133188) bank165072 +165073 POINT(41.12240092715571 -87.03420387451368) bank165073 +165074 POINT(41.684465285131296 -86.87677263829322) bank165074 +165075 POINT(40.89261251669704 -87.17228397609745) bank165075 +165076 POINT(42.6502277223955 -88.03706847208645) bank165076 +165077 POINT(42.505725297657094 -87.51776943457905) bank165077 +165078 POINT(42.10346910103933 -87.1906689620078) bank165078 +165079 POINT(41.761078590017945 -86.8620757695727) bank165079 +165080 POINT(42.17383940382908 -87.2528364999287) bank165080 +165081 POINT(42.548036621267514 -87.9293030891722) bank165081 +165082 POINT(42.06369870714926 -87.67666353830087) bank165082 +165083 POINT(41.96286468601969 -86.777658228072) bank165083 +165084 POINT(40.916160518092916 -87.40373360520732) bank165084 +165085 POINT(42.54492809441288 -88.45619653187859) bank165085 +165086 POINT(41.8575801165392 -87.37439133581664) bank165086 +165087 POINT(41.509002641689925 -86.74770775998321) bank165087 +165088 POINT(41.436880369019505 -87.43004649633649) bank165088 +165089 POINT(42.74061521361601 -88.26385727573174) bank165089 +165090 POINT(42.47964852811063 -88.45021403113148) bank165090 +165091 POINT(41.593510747814136 -88.56131610867497) bank165091 +165092 POINT(41.4041437295927 -87.35289180883969) bank165092 +165093 POINT(42.17726601205916 -87.14790845050769) bank165093 +165094 POINT(42.617130583024526 -88.59181432932182) bank165094 +165095 POINT(42.57051499796903 -86.76849139476991) bank165095 +165096 POINT(41.6294051915149 -87.62930304969761) bank165096 +165097 POINT(41.997911489590805 -86.69713995275596) bank165097 +165098 POINT(40.95128875092631 -87.1858263809542) bank165098 +165099 POINT(41.295148698467045 -88.1504708050584) bank165099 +165100 POINT(41.59760674308182 -87.59144631490545) bank165100 +165101 POINT(41.804288436377824 -88.01645982844542) bank165101 +165102 POINT(41.94079573602043 -88.58668854481907) bank165102 +165103 POINT(40.98655453021972 -88.09766377394308) bank165103 +165104 POINT(41.918625004180385 -87.89321329667769) bank165104 +165105 POINT(42.21314415360047 -86.63046597578241) bank165105 +165106 POINT(41.83234229576807 -87.81266119756118) bank165106 +165107 POINT(42.81766678832169 -87.84781616982154) bank165107 +165108 POINT(42.47129280775617 -88.41292880969927) bank165108 +165109 POINT(42.45667234080871 -88.05469445299305) bank165109 +165110 POINT(41.48437070333932 -88.14816187426003) bank165110 +165111 POINT(41.506926577949166 -88.20710028804486) bank165111 +165112 POINT(41.83089383442092 -87.93536116390558) bank165112 +165113 POINT(42.76466848915033 -87.10944401125343) bank165113 +165114 POINT(41.23689908933794 -88.50868842539789) bank165114 +165115 POINT(42.799481639554905 -86.87894831985304) bank165115 +165116 POINT(42.40050615273586 -87.2330371509075) bank165116 +165117 POINT(40.949534922083 -87.8353858061512) bank165117 +165118 POINT(41.77579847462382 -87.66692932714554) bank165118 +165119 POINT(41.437030121135 -88.31992011690917) bank165119 +165120 POINT(41.029122396709525 -88.50963171377249) bank165120 +165121 POINT(42.085627865264485 -88.04767599240503) bank165121 +165122 POINT(42.38189698396961 -88.15999914201967) bank165122 +165123 POINT(42.52805940338931 -87.29639808016798) bank165123 +165124 POINT(42.556708200467156 -88.43451333550784) bank165124 +165125 POINT(41.69368970872463 -87.02190772075173) bank165125 +165126 POINT(41.989358936365626 -88.57988938500233) bank165126 +165127 POINT(42.860532178005286 -87.69384914689167) bank165127 +165128 POINT(42.51376080376717 -87.30138781784908) bank165128 +165129 POINT(42.01063746142574 -87.29147818144524) bank165129 +165130 POINT(41.16625869995222 -86.90225651662655) bank165130 +165131 POINT(41.593278344094024 -87.10562266381822) bank165131 +165132 POINT(42.314625147717656 -88.42285405025908) bank165132 +165133 POINT(42.11464364089525 -86.63626390678034) bank165133 +165134 POINT(42.8582790200466 -88.16249403310164) bank165134 +165135 POINT(42.70201344201024 -88.00394551274756) bank165135 +165136 POINT(42.28059687193578 -87.50711788507645) bank165136 +165137 POINT(41.06189220848004 -86.70968878184853) bank165137 +165138 POINT(42.72738506897611 -87.91267264399313) bank165138 +165139 POINT(41.40555662840639 -87.71080030744477) bank165139 +165140 POINT(42.600688029966236 -88.38887007722222) bank165140 +165141 POINT(42.43014335416792 -86.82407954641116) bank165141 +165142 POINT(40.94187624765396 -86.64746279060073) bank165142 +165143 POINT(42.522983970667184 -88.49229695136941) bank165143 +165144 POINT(41.3167559377499 -87.05375154863486) bank165144 +165145 POINT(41.48990701113026 -88.04881734305731) bank165145 +165146 POINT(40.89590232011032 -88.21696699215279) bank165146 +165147 POINT(41.84804725721084 -86.82155611544617) bank165147 +165148 POINT(42.573202643546345 -88.25162213671373) bank165148 +165149 POINT(41.43571600983957 -86.64978317769477) bank165149 +165150 POINT(42.26628685076255 -87.79787600944418) bank165150 +165151 POINT(41.09159016224315 -88.20358948065255) bank165151 +165152 POINT(41.78765101735303 -87.3665552388497) bank165152 +165153 POINT(40.911422660925815 -87.86501480659155) bank165153 +165154 POINT(41.725213877927516 -86.89396518000743) bank165154 +165155 POINT(41.472848893152275 -88.25357339146046) bank165155 +165156 POINT(42.21754538390242 -87.02107001917567) bank165156 +165157 POINT(41.67683524710413 -87.42154178504506) bank165157 +165158 POINT(42.36684096972717 -87.85478913346931) bank165158 +165159 POINT(41.44776137968812 -87.2832850068938) bank165159 +165160 POINT(41.45787014296469 -88.53817019883333) bank165160 +165161 POINT(42.41228566839714 -86.91991131117649) bank165161 +165162 POINT(41.72768059696619 -87.17757470754776) bank165162 +165163 POINT(42.4078868959409 -86.77809966687185) bank165163 +165164 POINT(41.136645214834154 -88.20526696542099) bank165164 +165165 POINT(41.54404095212973 -88.48545634885414) bank165165 +165166 POINT(42.270024611855106 -87.18794919958619) bank165166 +165167 POINT(42.7359854251071 -86.67500166022447) bank165167 +165168 POINT(42.722673033677374 -86.98435207376345) bank165168 +165169 POINT(42.790246335381816 -88.39382227533586) bank165169 +165170 POINT(42.869033857997515 -86.98695506200151) bank165170 +165171 POINT(41.68555154807212 -86.69230943100507) bank165171 +165172 POINT(41.06540221705903 -87.96657478904733) bank165172 +165173 POINT(40.90391797859073 -87.90122796974856) bank165173 +165174 POINT(42.18654351522345 -86.9458150027328) bank165174 +165175 POINT(42.4254571161288 -87.12434556911454) bank165175 +165176 POINT(42.42331968388716 -88.25253747906649) bank165176 +165177 POINT(42.33756913502914 -87.50606435209055) bank165177 +165178 POINT(42.44324416617383 -87.38634887589794) bank165178 +165179 POINT(42.80253422662104 -87.63591526702982) bank165179 +165180 POINT(41.8723991655739 -87.92278995542469) bank165180 +165181 POINT(41.1678028389448 -86.73080132568055) bank165181 +165182 POINT(42.32576082087184 -87.29757553672918) bank165182 +165183 POINT(40.990586381647546 -86.71878768887208) bank165183 +165184 POINT(42.16940200231824 -87.12701524268905) bank165184 +165185 POINT(42.08650192690662 -88.17924452384615) bank165185 +165186 POINT(41.23765611638424 -86.81617315382991) bank165186 +165187 POINT(41.69511721551587 -87.76703148085966) bank165187 +165188 POINT(41.90419833445151 -88.24874883819132) bank165188 +165189 POINT(42.553648209894696 -88.60390241247937) bank165189 +165190 POINT(41.29550907190134 -87.72598644975228) bank165190 +165191 POINT(42.85115940460178 -87.10809696127286) bank165191 +165192 POINT(41.30310084225863 -88.46487668626433) bank165192 +165193 POINT(42.61808296818573 -86.98771030548414) bank165193 +165194 POINT(41.20019078918293 -87.351861663866) bank165194 +165195 POINT(42.29766589544601 -87.90089804023779) bank165195 +165196 POINT(40.90166019298219 -88.18690193426431) bank165196 +165197 POINT(42.45435990272826 -87.61419329911845) bank165197 +165198 POINT(41.53967262125008 -88.27410010439431) bank165198 +165199 POINT(40.939137768732046 -88.31128196355513) bank165199 +165200 POINT(42.77022017729629 -86.70632393032909) bank165200 +165201 POINT(41.2734803523079 -87.98549243065935) bank165201 +165202 POINT(41.479839151985345 -88.28727518114438) bank165202 +165203 POINT(41.74822782889486 -86.80417699974137) bank165203 +165204 POINT(41.34542310540856 -88.05583992695226) bank165204 +165205 POINT(42.860247654536764 -88.43834425655824) bank165205 +165206 POINT(41.28825329757522 -87.76130761260143) bank165206 +165207 POINT(41.58567561904513 -87.19623096335675) bank165207 +165208 POINT(41.98149998793459 -88.26376113977095) bank165208 +165209 POINT(42.865432068740965 -86.80045576334403) bank165209 +165210 POINT(40.886944274830086 -86.93937499357496) bank165210 +165211 POINT(41.96943022781152 -87.32309570560264) bank165211 +165212 POINT(40.94875232501692 -88.22381674758742) bank165212 +165213 POINT(41.37250623907075 -88.52295832817137) bank165213 +165214 POINT(41.13802877392362 -88.50079059352028) bank165214 +165215 POINT(41.40128432750271 -87.34592257082953) bank165215 +165216 POINT(41.19218366003291 -87.72633520785364) bank165216 +165217 POINT(41.61648469261418 -88.40541741091198) bank165217 +165218 POINT(41.03864575395067 -88.4868105828611) bank165218 +165219 POINT(42.39686974197462 -87.06685661928294) bank165219 +165220 POINT(41.44795417168204 -87.15322882825222) bank165220 +165221 POINT(41.98004401489856 -87.55876951144326) bank165221 +165222 POINT(42.523764558518586 -87.29591929190396) bank165222 +165223 POINT(40.9405509941536 -88.53788792014718) bank165223 +165224 POINT(41.112495669528904 -87.43850379785646) bank165224 +165225 POINT(42.38799404699275 -88.06541776984585) bank165225 +165226 POINT(42.46677101306407 -87.90849511720315) bank165226 +165227 POINT(42.288850883675295 -87.16286247061022) bank165227 +165228 POINT(42.33757551327423 -86.92117191245573) bank165228 +165229 POINT(41.78035626590515 -87.17931793919983) bank165229 +165230 POINT(42.80314697545992 -88.23308936956116) bank165230 +165231 POINT(41.76522512878739 -86.73375484243472) bank165231 +165232 POINT(42.60310446572895 -87.20614464067089) bank165232 +165233 POINT(42.40750583727123 -87.71992258371303) bank165233 +165234 POINT(42.49829890568768 -88.57319033498129) bank165234 +165235 POINT(42.014060247435474 -87.68563495600209) bank165235 +165236 POINT(42.85921747653774 -88.09336006997984) bank165236 +165237 POINT(41.58054477856406 -87.85918335525272) bank165237 +165238 POINT(41.276365030233826 -86.79670463201127) bank165238 +165239 POINT(41.93109672174128 -86.66398227048677) bank165239 +165240 POINT(42.39525429895751 -87.35376395119681) bank165240 +165241 POINT(42.01865322251857 -87.19753777592042) bank165241 +165242 POINT(41.643378454140766 -86.68240077000314) bank165242 +165243 POINT(42.44996115626583 -88.40936489679305) bank165243 +165244 POINT(42.20753079782912 -88.58465584819156) bank165244 +165245 POINT(41.228024090770504 -87.19468444187542) bank165245 +165246 POINT(42.34475424951932 -87.75302141673049) bank165246 +165247 POINT(41.39176373441994 -88.58864243911668) bank165247 +165248 POINT(42.37368935558256 -87.56867625667357) bank165248 +165249 POINT(42.78310572366873 -88.23268779365888) bank165249 +165250 POINT(41.13291145375623 -86.98387596877085) bank165250 +165251 POINT(41.38878054220975 -88.6166260699639) bank165251 +165252 POINT(41.459246213207 -88.4145606844111) bank165252 +165253 POINT(42.47045118791532 -87.9031545625221) bank165253 +165254 POINT(42.80842304354583 -87.49452134864285) bank165254 +165255 POINT(42.50353384988301 -88.42563308695891) bank165255 +165256 POINT(41.63672677391432 -87.01764028035382) bank165256 +165257 POINT(42.53215008851481 -86.65997301314968) bank165257 +165258 POINT(41.41693501021556 -86.95393237860904) bank165258 +165259 POINT(42.03768680844396 -88.1055229667544) bank165259 +165260 POINT(41.68663307465349 -88.05562634341862) bank165260 +165261 POINT(42.43991972021175 -87.43088460371925) bank165261 +165262 POINT(42.012473747313216 -86.6647471875444) bank165262 +165263 POINT(41.96996130725836 -88.22940019889595) bank165263 +165264 POINT(41.589753840170665 -87.02682842652182) bank165264 +165265 POINT(42.1159625743379 -87.75771616025929) bank165265 +165266 POINT(41.365681329361365 -88.22124364162282) bank165266 +165267 POINT(41.778752159773 -87.46330833303699) bank165267 +165268 POINT(42.40952111612528 -86.97395696832871) bank165268 +165269 POINT(42.51830307003079 -87.8234728138029) bank165269 +165270 POINT(41.36443364941713 -88.52362194977914) bank165270 +165271 POINT(41.71745929796705 -88.38647685390075) bank165271 +165272 POINT(41.406136281062786 -88.27616529935777) bank165272 +165273 POINT(40.93692597594743 -87.99583854030053) bank165273 +165274 POINT(41.54004759706487 -87.19259668969814) bank165274 +165275 POINT(42.448581135163906 -86.71351118106944) bank165275 +165276 POINT(42.297366429051046 -88.08812074343744) bank165276 +165277 POINT(41.45281647177588 -88.18205619281515) bank165277 +165278 POINT(42.415877516237934 -87.00794616291961) bank165278 +165279 POINT(42.80188990766707 -88.27815955502315) bank165279 +165280 POINT(42.33983171979198 -87.72402497434526) bank165280 +165281 POINT(42.03099162955301 -87.92831356479552) bank165281 +165282 POINT(41.13757267335793 -86.81778316816442) bank165282 +165283 POINT(41.43197799615718 -87.15164791065808) bank165283 +165284 POINT(42.60304826414885 -87.88639740837334) bank165284 +165285 POINT(41.876681057326834 -86.96911667747183) bank165285 +165286 POINT(42.72551371689497 -86.72341385195634) bank165286 +165287 POINT(42.717536930223304 -86.87981464002684) bank165287 +165288 POINT(41.71209872824954 -88.53279693866556) bank165288 +165289 POINT(42.60888934204769 -87.64720537217396) bank165289 +165290 POINT(42.65169205443158 -87.88892537525521) bank165290 +165291 POINT(41.65852740444501 -86.97581408357492) bank165291 +165292 POINT(42.20007563433013 -87.08487675655262) bank165292 +165293 POINT(42.52075207521694 -88.29457911315289) bank165293 +165294 POINT(42.56594495626822 -87.85313784605901) bank165294 +165295 POINT(41.89202907374538 -88.54996256666229) bank165295 +165296 POINT(41.101059222767724 -87.2088587425255) bank165296 +165297 POINT(42.00976060171057 -88.48802925730004) bank165297 +165298 POINT(41.716632568633806 -88.08966121971315) bank165298 +165299 POINT(42.61196597507831 -87.58005955678249) bank165299 +165300 POINT(41.86367881323839 -88.57941067272715) bank165300 +165301 POINT(42.15243040096771 -88.61196456448785) bank165301 +165302 POINT(41.24884066545478 -86.77212829554367) bank165302 +165303 POINT(42.393466282421386 -88.56684060510854) bank165303 +165304 POINT(41.2809011486435 -88.23789876511894) bank165304 +165305 POINT(41.56366162152008 -86.92979139152405) bank165305 +165306 POINT(42.46521847079477 -87.277624030478) bank165306 +165307 POINT(42.54782928704894 -86.7316486753395) bank165307 +165308 POINT(41.8025660914796 -87.74385593108097) bank165308 +165309 POINT(42.42916647367637 -87.33797564497732) bank165309 +165310 POINT(42.28930613942087 -87.93431351510505) bank165310 +165311 POINT(41.856434057217264 -88.47672259641368) bank165311 +165312 POINT(42.40476351194341 -87.95129836226684) bank165312 +165313 POINT(41.779372125778224 -87.57741988871007) bank165313 +165314 POINT(41.87555571020767 -86.89334840009931) bank165314 +165315 POINT(42.53223026596734 -88.07409533860367) bank165315 +165316 POINT(42.586085551389324 -87.45166761807914) bank165316 +165317 POINT(42.37760749251793 -86.68333941725699) bank165317 +165318 POINT(41.03329892027342 -86.67973213933817) bank165318 +165319 POINT(41.43532982705355 -87.81623013570636) bank165319 +165320 POINT(42.564294586369975 -87.77695627468371) bank165320 +165321 POINT(41.416494289487574 -87.71028872490629) bank165321 +165322 POINT(42.83296449267349 -86.96580705695263) bank165322 +165323 POINT(41.96760496665626 -88.59047530210816) bank165323 +165324 POINT(41.96623902266164 -88.20510207104464) bank165324 +165325 POINT(41.18531212581905 -87.92486162449377) bank165325 +165326 POINT(42.05601932148243 -86.73381600679187) bank165326 +165327 POINT(42.29976921451473 -88.1280296054242) bank165327 +165328 POINT(42.50980105308701 -87.85760889388054) bank165328 +165329 POINT(42.11195733062541 -87.9518355998929) bank165329 +165330 POINT(41.43310235565404 -86.93656654991152) bank165330 +165331 POINT(40.99773319320703 -87.20418612787273) bank165331 +165332 POINT(41.25207976363139 -87.41670684127727) bank165332 +165333 POINT(40.91837110112928 -86.6587520268048) bank165333 +165334 POINT(42.63707905792126 -88.05060445657698) bank165334 +165335 POINT(42.47748076656657 -88.46737725988875) bank165335 +165336 POINT(42.472299649016854 -87.27548540560038) bank165336 +165337 POINT(41.33373835780601 -88.34729631159533) bank165337 +165338 POINT(42.02820824974488 -88.11112549434336) bank165338 +165339 POINT(42.00952496961288 -87.52204727899395) bank165339 +165340 POINT(41.993990880557746 -87.888271896284) bank165340 +165341 POINT(41.298950866227 -86.64948911031067) bank165341 +165342 POINT(42.331436028169556 -87.66943781920143) bank165342 +165343 POINT(41.369128755186956 -87.30344262473506) bank165343 +165344 POINT(41.10072300355592 -87.17189704103284) bank165344 +165345 POINT(42.526368679877926 -87.0159766260215) bank165345 +165346 POINT(42.75161643846028 -88.45157716524454) bank165346 +165347 POINT(41.80751201155819 -86.81935421183044) bank165347 +165348 POINT(41.823793260331634 -87.19171404821664) bank165348 +165349 POINT(41.35665377908512 -88.48849918323556) bank165349 +165350 POINT(42.8371307877076 -88.4680122098275) bank165350 +165351 POINT(41.408410032598994 -87.95068270788286) bank165351 +165352 POINT(41.78609288066241 -88.58035000137434) bank165352 +165353 POINT(42.63194121447044 -87.10563698585767) bank165353 +165354 POINT(42.0660474848095 -86.98534903353298) bank165354 +165355 POINT(40.878800948200336 -88.53961273453915) bank165355 +165356 POINT(42.703497256093186 -86.9776188928835) bank165356 +165357 POINT(41.760693265713655 -87.9348853424742) bank165357 +165358 POINT(42.84137741835187 -88.10872662087446) bank165358 +165359 POINT(41.86216612755941 -86.87007109040549) bank165359 +165360 POINT(41.566912852459254 -88.47912213798183) bank165360 +165361 POINT(40.88731077133299 -87.97013060488234) bank165361 +165362 POINT(42.12362530055002 -87.58055737792776) bank165362 +165363 POINT(41.39020371458201 -87.8106867003266) bank165363 +165364 POINT(40.95455593559214 -87.09649506336334) bank165364 +165365 POINT(41.35221721128971 -88.40778193336912) bank165365 +165366 POINT(41.95355244933424 -88.45476721326932) bank165366 +165367 POINT(42.466866219693735 -87.65748160661404) bank165367 +165368 POINT(41.30113405495877 -88.2276523303044) bank165368 +165369 POINT(42.165189835637065 -87.59489498383344) bank165369 +165370 POINT(41.80169330484139 -87.3049122782722) bank165370 +165371 POINT(41.34652905469895 -88.4328890072022) bank165371 +165372 POINT(42.051878536689884 -88.12133204526987) bank165372 +165373 POINT(41.76638321619537 -88.09636392035326) bank165373 +165374 POINT(42.248501023506286 -88.27045149832453) bank165374 +165375 POINT(41.30077767294788 -87.74899909013804) bank165375 +165376 POINT(41.744426643741036 -88.42346173585146) bank165376 +165377 POINT(41.938035917066664 -88.60127273119589) bank165377 +165378 POINT(41.279454047872214 -88.14556030006858) bank165378 +165379 POINT(41.17732199210512 -87.64572179493122) bank165379 +165380 POINT(41.35725796823902 -88.62568139583664) bank165380 +165381 POINT(41.95255499984652 -86.99389512964709) bank165381 +165382 POINT(42.62031037261457 -87.8865621766445) bank165382 +165383 POINT(41.59660687129867 -87.92240642628245) bank165383 +165384 POINT(42.26530491907235 -86.78644582454905) bank165384 +165385 POINT(42.665882515663405 -88.26104025412224) bank165385 +165386 POINT(41.19207231932543 -87.78750241275372) bank165386 +165387 POINT(42.6252226147058 -88.0667369417214) bank165387 +165388 POINT(41.426715952409126 -88.34405174635518) bank165388 +165389 POINT(41.369230465830405 -88.18648454944973) bank165389 +165390 POINT(42.45212930527208 -88.1272645142627) bank165390 +165391 POINT(41.7354076753294 -88.21561016992024) bank165391 +165392 POINT(42.3402175512943 -88.23828617858527) bank165392 +165393 POINT(42.32083402428055 -87.36295753908425) bank165393 +165394 POINT(42.17327713824398 -87.71693814009556) bank165394 +165395 POINT(42.515066545044085 -86.65479218275821) bank165395 +165396 POINT(41.92771413765193 -87.23524188146357) bank165396 +165397 POINT(41.861997391746684 -87.42025910442669) bank165397 +165398 POINT(41.049375888840416 -87.6679051605114) bank165398 +165399 POINT(42.51216581179091 -88.09807991477201) bank165399 +165400 POINT(41.98520187732908 -87.79476443885224) bank165400 +165401 POINT(41.77638419664933 -87.54482616546352) bank165401 +165402 POINT(41.17376062450919 -88.58827150685956) bank165402 +165403 POINT(42.01054647612305 -87.93757824952105) bank165403 +165404 POINT(41.16616593294258 -87.42047566227606) bank165404 +165405 POINT(40.93895970189231 -87.65133038888905) bank165405 +165406 POINT(41.528559312211215 -88.48809842431453) bank165406 +165407 POINT(42.78470541927848 -87.86275288501365) bank165407 +165408 POINT(41.28267640381627 -88.18374190700567) bank165408 +165409 POINT(42.86262620964145 -87.73472076116782) bank165409 +165410 POINT(42.69068973493417 -87.7922824201113) bank165410 +165411 POINT(40.9147489739257 -87.12863252605821) bank165411 +165412 POINT(41.17914565299874 -87.7515832350032) bank165412 +165413 POINT(41.23455872566614 -87.6250280470428) bank165413 +165414 POINT(42.186501458680915 -87.46101870040626) bank165414 +165415 POINT(41.41525445225537 -87.7166722382052) bank165415 +165416 POINT(41.11768781097349 -87.22014878796594) bank165416 +165417 POINT(41.9067841264745 -86.86345803678296) bank165417 +165418 POINT(42.046027829143775 -86.92035984417899) bank165418 +165419 POINT(41.26054087105962 -88.09008280703932) bank165419 +165420 POINT(42.26492020258312 -86.70307976814298) bank165420 +165421 POINT(42.61477284486571 -87.52885984956838) bank165421 +165422 POINT(42.254998753422704 -86.67031613434256) bank165422 +165423 POINT(41.28379263716467 -87.72752221251216) bank165423 +165424 POINT(41.178835110244094 -87.95053034458331) bank165424 +165425 POINT(42.15835800160069 -88.03495834652293) bank165425 +165426 POINT(41.04102710396573 -87.93762994644493) bank165426 +165427 POINT(41.56408773171334 -86.68644026161084) bank165427 +165428 POINT(41.17718654374955 -87.65685919879763) bank165428 +165429 POINT(42.765586298593405 -88.49629837992885) bank165429 +165430 POINT(42.74165947624645 -87.66770536750543) bank165430 +165431 POINT(42.24693061307952 -86.82334123655006) bank165431 +165432 POINT(41.55685791002495 -87.07937199032602) bank165432 +165433 POINT(41.080676004302774 -86.67937148950631) bank165433 +165434 POINT(42.18877035276719 -88.3260699453545) bank165434 +165435 POINT(42.55924601063824 -86.84917463269751) bank165435 +165436 POINT(42.43563746857388 -88.18335519846961) bank165436 +165437 POINT(41.84653308635702 -88.14403146047758) bank165437 +165438 POINT(42.512649548181244 -87.54032952175957) bank165438 +165439 POINT(41.95664710390784 -88.32386248150614) bank165439 +165440 POINT(42.124563708487585 -87.80524305655312) bank165440 +165441 POINT(42.63889264151789 -87.0089192508663) bank165441 +165442 POINT(41.96534522735821 -86.96991309181668) bank165442 +165443 POINT(41.06019220452908 -86.71287242002698) bank165443 +165444 POINT(41.186731374130844 -87.87640644468864) bank165444 +165445 POINT(41.679872563762665 -86.68832130895755) bank165445 +165446 POINT(41.664244665828654 -86.73255244724) bank165446 +165447 POINT(42.81584836916775 -88.00858158394921) bank165447 +165448 POINT(41.905029483451166 -87.68070923307711) bank165448 +165449 POINT(41.11568825715178 -87.73889710419887) bank165449 +165450 POINT(41.66856556570706 -87.51209020886726) bank165450 +165451 POINT(42.197560879581054 -88.55932886699546) bank165451 +165452 POINT(42.83995788983176 -88.18341538159272) bank165452 +165453 POINT(42.734360002460065 -88.36791743265005) bank165453 +165454 POINT(42.364317289470286 -87.3369109248679) bank165454 +165455 POINT(41.51029022280661 -88.06066361976221) bank165455 +165456 POINT(42.65608959758963 -88.40953488866161) bank165456 +165457 POINT(41.76382100879934 -88.26696136757869) bank165457 +165458 POINT(40.94870368309279 -86.94616287454087) bank165458 +165459 POINT(41.923631780132446 -87.0841247585217) bank165459 +165460 POINT(41.46660950405368 -86.92540881781039) bank165460 +165461 POINT(41.20788078480542 -87.31262832068691) bank165461 +165462 POINT(41.618342849117745 -88.43498408684499) bank165462 +165463 POINT(42.120874970247996 -86.94892198656211) bank165463 +165464 POINT(42.87761231056312 -86.900311499604) bank165464 +165465 POINT(41.18459079595529 -87.44870235162522) bank165465 +165466 POINT(42.838547467062845 -87.42840225512123) bank165466 +165467 POINT(42.70000224243236 -87.33944690829199) bank165467 +165468 POINT(42.19873243107233 -88.12461070243337) bank165468 +165469 POINT(41.90505315804298 -87.26088497824335) bank165469 +165470 POINT(40.999870531125445 -87.54447741781077) bank165470 +165471 POINT(42.355676670083696 -87.20734188079062) bank165471 +165472 POINT(41.462509039193435 -87.51893530660091) bank165472 +165473 POINT(41.813319028089175 -86.94796961781218) bank165473 +165474 POINT(41.93135805904265 -87.22046035397639) bank165474 +165475 POINT(41.53631674871967 -87.5489789642217) bank165475 +165476 POINT(42.033930505572684 -88.21156685571012) bank165476 +165477 POINT(42.17146526490931 -87.37771514690714) bank165477 +165478 POINT(42.39147313680306 -87.32172843607735) bank165478 +165479 POINT(41.0403762420272 -87.74375344558081) bank165479 +165480 POINT(41.5916825006176 -87.88039247514298) bank165480 +165481 POINT(42.430221657858716 -88.35478101117835) bank165481 +165482 POINT(42.77225776138748 -88.2717818607764) bank165482 +165483 POINT(41.11246127438768 -86.94848617677727) bank165483 +165484 POINT(41.33784664089582 -86.95748688045491) bank165484 +165485 POINT(41.23245604675384 -87.29405535672807) bank165485 +165486 POINT(41.5545548071678 -88.23753558456549) bank165486 +165487 POINT(40.89975418579455 -87.40390311102303) bank165487 +165488 POINT(42.654277954099385 -87.19055990141476) bank165488 +165489 POINT(41.84920654099212 -88.21870674069162) bank165489 +165490 POINT(40.921618907948265 -86.90487632390342) bank165490 +165491 POINT(40.94056797141871 -88.42501879079576) bank165491 +165492 POINT(41.3049430778529 -87.71913867860654) bank165492 +165493 POINT(41.34208471075003 -87.34213129829088) bank165493 +165494 POINT(40.922998627704736 -88.28795977626369) bank165494 +165495 POINT(42.863841935303256 -86.80071748098324) bank165495 +165496 POINT(42.1383702918881 -88.01280507918295) bank165496 +165497 POINT(42.378529368892266 -86.69106959223456) bank165497 +165498 POINT(41.73365390249991 -87.69151615084031) bank165498 +165499 POINT(41.69790505720165 -88.58265224658136) bank165499 +165500 POINT(41.97325665087511 -87.81134601180949) bank165500 +165501 POINT(40.946369004934965 -87.95202340310692) bank165501 +165502 POINT(42.09455716168117 -87.82142694004945) bank165502 +165503 POINT(42.10724071774447 -87.41741086721515) bank165503 +165504 POINT(42.69046743937667 -88.30009634775476) bank165504 +165505 POINT(42.78930681855373 -88.43872582941944) bank165505 +165506 POINT(42.43513651608448 -87.28728300950448) bank165506 +165507 POINT(41.54145826408792 -87.85121351514479) bank165507 +165508 POINT(42.43111505149312 -87.9900105560117) bank165508 +165509 POINT(40.9285142809354 -87.57164506915692) bank165509 +165510 POINT(42.34224377536759 -87.83188698306563) bank165510 +165511 POINT(41.12888175431609 -87.44831786556445) bank165511 +165512 POINT(42.54678546410258 -87.63241526904082) bank165512 +165513 POINT(41.667576188570465 -86.65919153731582) bank165513 +165514 POINT(42.43994930245148 -87.72606814270993) bank165514 +165515 POINT(41.828970108625384 -86.85997505478875) bank165515 +165516 POINT(42.47808365589869 -86.79125744737512) bank165516 +165517 POINT(41.54217244276874 -88.22697575998549) bank165517 +165518 POINT(40.953949583106024 -88.37775727626854) bank165518 +165519 POINT(42.760746377875485 -86.8718044684673) bank165519 +165520 POINT(42.02715304994383 -86.70422359299789) bank165520 +165521 POINT(42.85006358146129 -87.22738815373575) bank165521 +165522 POINT(42.43311270012849 -88.10861610747828) bank165522 +165523 POINT(41.787590221077224 -87.85168556218673) bank165523 +165524 POINT(42.465461392832644 -87.4585296443294) bank165524 +165525 POINT(42.09264337444212 -87.0113912327795) bank165525 +165526 POINT(40.99408915344146 -87.13547950365579) bank165526 +165527 POINT(42.06294779397239 -87.77579042650608) bank165527 +165528 POINT(41.65463430333099 -87.82196275058304) bank165528 +165529 POINT(41.89391021259373 -87.36640231825136) bank165529 +165530 POINT(42.72080254902883 -88.31092642429246) bank165530 +165531 POINT(41.536005403818095 -87.42860415998334) bank165531 +165532 POINT(42.05425246826421 -87.72565722719091) bank165532 +165533 POINT(42.07756157973642 -86.75900990679355) bank165533 +165534 POINT(42.81487276972965 -87.24339956586203) bank165534 +165535 POINT(42.503868758068194 -88.30124381617586) bank165535 +165536 POINT(41.59064682868681 -87.13888318775946) bank165536 +165537 POINT(40.93041180577398 -87.85246777742479) bank165537 +165538 POINT(42.22225614902572 -88.4809952888125) bank165538 +165539 POINT(41.66552786369277 -87.0016970740146) bank165539 +165540 POINT(42.36305334864463 -88.22981553757896) bank165540 +165541 POINT(42.224300020672594 -88.42371272960283) bank165541 +165542 POINT(42.65472589009548 -87.04693178052224) bank165542 +165543 POINT(42.41362340552729 -87.09102005455631) bank165543 +165544 POINT(41.38246338713234 -87.08865731751493) bank165544 +165545 POINT(41.098592967500615 -87.1878505320595) bank165545 +165546 POINT(41.44106386046946 -88.14024115360075) bank165546 +165547 POINT(42.398062636944914 -87.4324413262691) bank165547 +165548 POINT(42.734137059213644 -88.54120730479464) bank165548 +165549 POINT(40.92354436942586 -87.19316502933096) bank165549 +165550 POINT(41.047286336410714 -88.07603595438044) bank165550 +165551 POINT(41.22162441724322 -87.15876243024312) bank165551 +165552 POINT(42.12723214671605 -88.18793588615198) bank165552 +165553 POINT(41.32439529787843 -88.13696152693129) bank165553 +165554 POINT(42.02087631965433 -87.75590021290353) bank165554 +165555 POINT(42.14702991456 -86.92504325991553) bank165555 +165556 POINT(41.29478016211396 -87.5002764644501) bank165556 +165557 POINT(41.38419292848252 -86.79183903466138) bank165557 +165558 POINT(42.43240086679333 -87.80289372469089) bank165558 +165559 POINT(42.59887627672963 -88.06431166784684) bank165559 +165560 POINT(42.706460789590444 -88.06655666531185) bank165560 +165561 POINT(42.55691051663085 -86.85417179822683) bank165561 +165562 POINT(42.46070548539527 -87.07433235804415) bank165562 +165563 POINT(41.62277394485912 -87.37743052438597) bank165563 +165564 POINT(42.83482361018335 -88.253114748065) bank165564 +165565 POINT(40.934672310368065 -87.85270087497238) bank165565 +165566 POINT(41.63762157641914 -87.47831363707292) bank165566 +165567 POINT(41.26483495950565 -87.4400984494783) bank165567 +165568 POINT(42.67666073687626 -88.45186772827161) bank165568 +165569 POINT(41.940386695022895 -87.99905083353025) bank165569 +165570 POINT(42.57521084884198 -86.66077815502507) bank165570 +165571 POINT(41.03341486226167 -88.31060708851768) bank165571 +165572 POINT(41.18327324890568 -88.26632385400315) bank165572 +165573 POINT(41.430714626689294 -86.83928991464715) bank165573 +165574 POINT(42.25449233650327 -87.39581504537018) bank165574 +165575 POINT(42.45537888920607 -87.80248756925509) bank165575 +165576 POINT(42.38584433598737 -87.31061647067963) bank165576 +165577 POINT(42.53605140149846 -88.55433030304481) bank165577 +165578 POINT(42.02717391122216 -87.19999597603329) bank165578 +165579 POINT(42.79222603201391 -87.12795037116818) bank165579 +165580 POINT(42.264708036116936 -88.21866237544022) bank165580 +165581 POINT(41.390706167383456 -88.45100939311943) bank165581 +165582 POINT(41.58703245936213 -87.66446485675931) bank165582 +165583 POINT(42.517125173931156 -87.34419590710705) bank165583 +165584 POINT(41.342939432619794 -88.34597945251116) bank165584 +165585 POINT(42.05667073652875 -88.61651620650355) bank165585 +165586 POINT(42.79778701106587 -87.60547693206557) bank165586 +165587 POINT(41.8658919906538 -87.14772618089332) bank165587 +165588 POINT(41.65372642607643 -86.8773728703271) bank165588 +165589 POINT(42.55186376354495 -87.21007194172813) bank165589 +165590 POINT(41.67599810412608 -88.17922426600232) bank165590 +165591 POINT(41.01928552983938 -86.86993736329802) bank165591 +165592 POINT(42.545744857595544 -88.07096067668121) bank165592 +165593 POINT(42.281474595519036 -88.40478215224117) bank165593 +165594 POINT(42.55513700942468 -86.85527157814793) bank165594 +165595 POINT(41.582049255519415 -86.89127963197163) bank165595 +165596 POINT(41.33701250583328 -87.62801327303465) bank165596 +165597 POINT(41.0155661908762 -88.28963728055858) bank165597 +165598 POINT(40.92185286645953 -87.21129498968186) bank165598 +165599 POINT(41.88622601461412 -87.10996702657319) bank165599 +165600 POINT(42.2515641740188 -87.54989633289098) bank165600 +165601 POINT(42.29236411024904 -87.06047123028273) bank165601 +165602 POINT(42.00432729748409 -87.97936435344879) bank165602 +165603 POINT(41.59228990793636 -87.07879975101633) bank165603 +165604 POINT(42.75880810809399 -87.03369726992332) bank165604 +165605 POINT(41.32870610369185 -88.18173941329644) bank165605 +165606 POINT(41.184991807565055 -86.88236505813448) bank165606 +165607 POINT(42.87419634398421 -87.21913548897138) bank165607 +165608 POINT(40.88924620174666 -88.51747294765003) bank165608 +165609 POINT(41.09973146069838 -86.78038073858096) bank165609 +165610 POINT(42.04850617309085 -86.81995964017628) bank165610 +165611 POINT(41.75126199354933 -88.24569699167748) bank165611 +165612 POINT(42.463856570823054 -88.28848950309491) bank165612 +165613 POINT(41.65786809980591 -86.96643138570745) bank165613 +165614 POINT(42.60439258642044 -86.74479623390185) bank165614 +165615 POINT(41.059023361097864 -87.44546234569432) bank165615 +165616 POINT(42.5132951147806 -87.74176600352922) bank165616 +165617 POINT(42.47589814186394 -87.8731029276621) bank165617 +165618 POINT(42.27449402137872 -87.2714378486369) bank165618 +165619 POINT(41.51963252549625 -87.08406883697636) bank165619 +165620 POINT(41.56541780765013 -86.66916540621715) bank165620 +165621 POINT(42.413092663309165 -87.09298935964759) bank165621 +165622 POINT(42.14677776812167 -87.65917550641366) bank165622 +165623 POINT(41.57650837681419 -87.9031530666187) bank165623 +165624 POINT(41.219616619279414 -87.10211761018606) bank165624 +165625 POINT(41.908977627458285 -86.63391192912722) bank165625 +165626 POINT(40.923991620493446 -88.62282878641797) bank165626 +165627 POINT(42.600504444009644 -88.58565473095264) bank165627 +165628 POINT(41.62763918073645 -86.99437490112975) bank165628 +165629 POINT(42.63169989056516 -87.67948990361587) bank165629 +165630 POINT(42.872626266201735 -88.41113239259367) bank165630 +165631 POINT(41.4140494346693 -88.24682122101474) bank165631 +165632 POINT(42.817018092574514 -88.62826034760386) bank165632 +165633 POINT(42.441015473020805 -87.25687269921379) bank165633 +165634 POINT(42.755759962725726 -86.77703264485166) bank165634 +165635 POINT(42.440053594390974 -88.35413256542554) bank165635 +165636 POINT(42.706094119678134 -88.1327207249236) bank165636 +165637 POINT(41.57096725079043 -87.36543598830602) bank165637 +165638 POINT(40.890642203352144 -87.09692057736113) bank165638 +165639 POINT(42.534023338404644 -86.65240060757245) bank165639 +165640 POINT(40.98820217291921 -88.52086837150644) bank165640 +165641 POINT(42.53574595511108 -88.37655985885847) bank165641 +165642 POINT(41.243319696608225 -87.50515820511211) bank165642 +165643 POINT(42.063822253515745 -87.01831601190207) bank165643 +165644 POINT(42.70384540674836 -88.54939376453028) bank165644 +165645 POINT(42.552340951773445 -88.4844807072148) bank165645 +165646 POINT(42.28428303941394 -87.43628504927442) bank165646 +165647 POINT(41.75101717351872 -87.10996245301713) bank165647 +165648 POINT(42.093799827429976 -87.89097568968748) bank165648 +165649 POINT(42.66327181800795 -88.295012534891) bank165649 +165650 POINT(41.46488014331403 -86.6649060418614) bank165650 +165651 POINT(42.73486897845547 -86.95313566332771) bank165651 +165652 POINT(40.94604114754431 -87.71906740794715) bank165652 +165653 POINT(42.60668579901662 -87.27094025468702) bank165653 +165654 POINT(42.41831779124107 -87.82264963375673) bank165654 +165655 POINT(42.05821233404365 -87.8868508243956) bank165655 +165656 POINT(41.46541470729004 -86.89259237980549) bank165656 +165657 POINT(41.7901844359558 -87.4835259842078) bank165657 +165658 POINT(41.563000083947905 -88.4315222921344) bank165658 +165659 POINT(41.338383862958004 -86.97405853573773) bank165659 +165660 POINT(42.399904442845035 -86.80025714477094) bank165660 +165661 POINT(41.35488009218195 -88.06966200184793) bank165661 +165662 POINT(42.30525925671062 -87.98571724021356) bank165662 +165663 POINT(42.00917805481502 -88.36743572881016) bank165663 +165664 POINT(41.40780718018646 -87.48460951803428) bank165664 +165665 POINT(42.805517117460695 -88.5022807579406) bank165665 +165666 POINT(40.92782984484294 -87.72573916336273) bank165666 +165667 POINT(41.68704085398173 -87.4485941835024) bank165667 +165668 POINT(42.01384728849458 -88.57219714943487) bank165668 +165669 POINT(41.71886491538878 -87.73330206345419) bank165669 +165670 POINT(42.71647428086829 -87.30224306219591) bank165670 +165671 POINT(41.16269489975069 -87.59101204506888) bank165671 +165672 POINT(42.03548481162292 -87.01540178031233) bank165672 +165673 POINT(41.13583251902481 -87.64411779304362) bank165673 +165674 POINT(42.653927522838345 -87.95294298080756) bank165674 +165675 POINT(42.20547405323941 -88.4005303830507) bank165675 +165676 POINT(42.01740646258342 -88.25404969556551) bank165676 +165677 POINT(42.05702355626517 -88.10131928599552) bank165677 +165678 POINT(42.25383301317812 -87.31682333402085) bank165678 +165679 POINT(42.76299297957674 -88.52749020068977) bank165679 +165680 POINT(41.64934528115952 -86.87401755371833) bank165680 +165681 POINT(41.26951679568804 -88.61977295826955) bank165681 +165682 POINT(41.90141272557172 -88.60789629953995) bank165682 +165683 POINT(40.923912252352636 -88.39647457779482) bank165683 +165684 POINT(41.51336416674436 -88.6271817549481) bank165684 +165685 POINT(41.797493818444735 -86.70799792321098) bank165685 +165686 POINT(42.80602087755062 -87.14066569300043) bank165686 +165687 POINT(41.029208861835514 -87.05721004413037) bank165687 +165688 POINT(40.98046685908528 -88.11357155963249) bank165688 +165689 POINT(41.13968600698481 -87.66034024751016) bank165689 +165690 POINT(41.30075592712246 -87.36602902506363) bank165690 +165691 POINT(41.32922298829716 -86.68102954787607) bank165691 +165692 POINT(41.93996590986216 -86.70957298223583) bank165692 +165693 POINT(42.79391218215536 -87.53499243748138) bank165693 +165694 POINT(41.06009155435553 -88.39505738348642) bank165694 +165695 POINT(41.82000844703778 -87.44432019737776) bank165695 +165696 POINT(41.51283649558458 -87.95937731942689) bank165696 +165697 POINT(42.5596069030407 -88.1193851763584) bank165697 +165698 POINT(40.9528746658021 -87.79828196647621) bank165698 +165699 POINT(41.9178397188381 -88.16372860643747) bank165699 +165700 POINT(42.802914975444004 -86.84722115027286) bank165700 +165701 POINT(41.484721274088656 -88.40077820267734) bank165701 +165702 POINT(41.20700269931909 -87.24777373103124) bank165702 +165703 POINT(41.52114385591945 -88.10769337210608) bank165703 +165704 POINT(42.359205616466454 -87.32287539257048) bank165704 +165705 POINT(42.35668634006188 -87.38590699670156) bank165705 +165706 POINT(41.39338110080394 -87.83377306803438) bank165706 +165707 POINT(41.51820356820566 -87.28552326599986) bank165707 +165708 POINT(41.70151718539457 -87.8093937252763) bank165708 +165709 POINT(42.75161296277986 -88.48104571523959) bank165709 +165710 POINT(42.43286696790433 -86.81873844680231) bank165710 +165711 POINT(41.33818043681895 -86.94763362789733) bank165711 +165712 POINT(41.10361604455132 -87.33011853868923) bank165712 +165713 POINT(41.966873417771765 -87.68939908308889) bank165713 +165714 POINT(42.36324176293747 -88.55182925809218) bank165714 +165715 POINT(41.0072589335323 -88.02303130028673) bank165715 +165716 POINT(42.85710527852566 -87.66352842442694) bank165716 +165717 POINT(41.98892892213941 -88.41957371073732) bank165717 +165718 POINT(41.81326350684382 -87.00994302143043) bank165718 +165719 POINT(42.62329348491467 -87.94349995211078) bank165719 +165720 POINT(42.22746143589775 -88.53414696611674) bank165720 +165721 POINT(41.49001214535323 -86.81775120226455) bank165721 +165722 POINT(40.90749769682551 -88.3096540194847) bank165722 +165723 POINT(42.20192425804943 -87.90517202069054) bank165723 +165724 POINT(41.30633627065098 -88.45667263043451) bank165724 +165725 POINT(41.68597973875382 -87.31412829624811) bank165725 +165726 POINT(42.3958564948511 -87.87952077941881) bank165726 +165727 POINT(41.80828890139479 -87.17996233080143) bank165727 +165728 POINT(41.73952545050632 -87.85410208204198) bank165728 +165729 POINT(42.84474423710518 -87.2387419715627) bank165729 +165730 POINT(41.44332162702847 -87.64943196406848) bank165730 +165731 POINT(42.413891052460286 -86.63022373330818) bank165731 +165732 POINT(40.90318564067049 -87.02606729812983) bank165732 +165733 POINT(41.20732481115647 -87.3919538966617) bank165733 +165734 POINT(41.471917090087125 -88.33774048216173) bank165734 +165735 POINT(41.931911654405695 -87.98956598059993) bank165735 +165736 POINT(42.61119700133088 -87.22331892348976) bank165736 +165737 POINT(42.83336145031963 -88.03341852480943) bank165737 +165738 POINT(41.521799110976396 -86.98169807494848) bank165738 +165739 POINT(42.27252549761142 -88.13175774034336) bank165739 +165740 POINT(41.2743911240172 -88.16186586952772) bank165740 +165741 POINT(41.88817902567758 -87.016472131539) bank165741 +165742 POINT(41.64098145495299 -88.46340416123262) bank165742 +165743 POINT(41.364222860974785 -86.91573264290876) bank165743 +165744 POINT(42.71487505133519 -88.22683891732372) bank165744 +165745 POINT(41.784932154357364 -88.2240535353388) bank165745 +165746 POINT(42.65296280603735 -87.57642008475403) bank165746 +165747 POINT(41.980704412202755 -86.68806754717342) bank165747 +165748 POINT(41.35749661346255 -87.81740581773298) bank165748 +165749 POINT(42.853146836219516 -87.66127927799508) bank165749 +165750 POINT(42.65732407851858 -87.75743340974195) bank165750 +165751 POINT(40.90778393864103 -87.83700733175) bank165751 +165752 POINT(42.57900378227785 -88.4014203958056) bank165752 +165753 POINT(40.94898543930737 -88.26576514573254) bank165753 +165754 POINT(41.2089772773414 -87.96161235708381) bank165754 +165755 POINT(41.56153429003957 -86.95433293098468) bank165755 +165756 POINT(42.23770713235254 -88.38750969930138) bank165756 +165757 POINT(42.31844942190324 -88.58797183435779) bank165757 +165758 POINT(41.68898588277193 -87.17931946699508) bank165758 +165759 POINT(42.74761088873786 -86.89788074072358) bank165759 +165760 POINT(42.34862444317924 -87.24005711983465) bank165760 +165761 POINT(42.34979048045697 -87.37044866336596) bank165761 +165762 POINT(40.98211503653651 -87.314163332277) bank165762 +165763 POINT(40.91838561446137 -87.81682620111486) bank165763 +165764 POINT(41.72353823381027 -88.32408880305702) bank165764 +165765 POINT(42.549103941196925 -87.89835154094025) bank165765 +165766 POINT(41.37596792825261 -87.65220664918978) bank165766 +165767 POINT(41.827744239092006 -88.59278588264134) bank165767 +165768 POINT(41.1215158936382 -87.878881438441) bank165768 +165769 POINT(41.72882910479504 -87.74017286483785) bank165769 +165770 POINT(41.096649568887166 -87.24036559975828) bank165770 +165771 POINT(41.172557549246484 -87.08481197182616) bank165771 +165772 POINT(42.1275538704028 -88.32370550576148) bank165772 +165773 POINT(41.17589029121531 -87.92477234084737) bank165773 +165774 POINT(42.36353114760987 -87.87704020996324) bank165774 +165775 POINT(41.82868620006228 -88.43054905347655) bank165775 +165776 POINT(41.89042455339364 -87.0573448143698) bank165776 +165777 POINT(42.231989721471194 -86.95420363068118) bank165777 +165778 POINT(41.34407099056662 -87.70785050804785) bank165778 +165779 POINT(42.53078424827234 -87.74675498854486) bank165779 +165780 POINT(42.334366157565775 -86.74165764019564) bank165780 +165781 POINT(41.26015160279448 -87.31449450156994) bank165781 +165782 POINT(40.90814334528673 -88.6191443385675) bank165782 +165783 POINT(41.099184134574685 -88.3603874913642) bank165783 +165784 POINT(42.17467836605369 -87.2201015865445) bank165784 +165785 POINT(41.1573354581541 -87.41501845805855) bank165785 +165786 POINT(42.4018693228656 -87.98068523332769) bank165786 +165787 POINT(41.573567746561274 -88.11709026566875) bank165787 +165788 POINT(41.951428292613436 -88.30076743685792) bank165788 +165789 POINT(42.196882449295366 -87.44366345008925) bank165789 +165790 POINT(40.90963735716436 -87.35210533152) bank165790 +165791 POINT(40.98587746866501 -87.97967590223668) bank165791 +165792 POINT(42.480140753180315 -87.90561973960001) bank165792 +165793 POINT(42.86267866127816 -87.37281176309861) bank165793 +165794 POINT(41.52174265368064 -88.44115415893997) bank165794 +165795 POINT(42.329216407475414 -87.72027266363435) bank165795 +165796 POINT(42.62775782771955 -86.81921560424895) bank165796 +165797 POINT(42.74558723335215 -86.80030531114063) bank165797 +165798 POINT(42.70295432757596 -87.24293632671888) bank165798 +165799 POINT(42.170713363842495 -87.50129895508888) bank165799 +165800 POINT(41.682431634077275 -88.21826833864627) bank165800 +165801 POINT(41.030847564958876 -86.84442487352824) bank165801 +165802 POINT(42.2213519623171 -87.15573419870535) bank165802 +165803 POINT(41.378436504173344 -87.6958420881708) bank165803 +165804 POINT(41.85103758793544 -87.62833036094166) bank165804 +165805 POINT(42.402410767362895 -86.69499649298675) bank165805 +165806 POINT(41.062327891101674 -86.74383946067435) bank165806 +165807 POINT(41.32225627580134 -87.73940208790518) bank165807 +165808 POINT(42.725895498394365 -87.51676050611171) bank165808 +165809 POINT(42.655638289154155 -87.17473651111857) bank165809 +165810 POINT(41.5611303039136 -88.12662031246109) bank165810 +165811 POINT(42.48561360586971 -88.32770168853494) bank165811 +165812 POINT(40.99820588582713 -87.52798074011608) bank165812 +165813 POINT(41.755667460639366 -87.17505331121059) bank165813 +165814 POINT(41.19271623103286 -87.48823939514047) bank165814 +165815 POINT(41.78056205037693 -87.88169789729592) bank165815 +165816 POINT(42.30353372022584 -88.59970920464298) bank165816 +165817 POINT(42.71519104862099 -88.33378603207237) bank165817 +165818 POINT(42.09920689990894 -88.26731910047924) bank165818 +165819 POINT(40.905282230492865 -87.79146982348593) bank165819 +165820 POINT(42.786476704748274 -87.7969724132313) bank165820 +165821 POINT(42.32609009477748 -86.85890150591298) bank165821 +165822 POINT(41.46167446386034 -87.06272483452197) bank165822 +165823 POINT(42.39889720809922 -87.0401281951921) bank165823 +165824 POINT(41.56498806742326 -87.84934036633247) bank165824 +165825 POINT(41.2324591687136 -87.13971279011803) bank165825 +165826 POINT(41.072948596777515 -87.74936735538965) bank165826 +165827 POINT(42.766520951574066 -86.93940865127044) bank165827 +165828 POINT(42.57353033642679 -87.88223277616305) bank165828 +165829 POINT(42.62289262824017 -87.563590022063) bank165829 +165830 POINT(42.192531545663776 -87.85141135357287) bank165830 +165831 POINT(41.580000414972595 -88.61653359901615) bank165831 +165832 POINT(41.938315048535216 -87.55586127352191) bank165832 +165833 POINT(41.972774519048144 -88.2272522960765) bank165833 +165834 POINT(41.27653785985202 -86.89717128298169) bank165834 +165835 POINT(42.47351056131545 -87.81437942533428) bank165835 +165836 POINT(42.1505805738617 -88.52240191207446) bank165836 +165837 POINT(41.121348126724186 -86.88931421692946) bank165837 +165838 POINT(42.755015207631274 -86.80298787165502) bank165838 +165839 POINT(41.745066955828925 -87.43408293701614) bank165839 +165840 POINT(41.971928865102655 -88.43112525108106) bank165840 +165841 POINT(42.170698646627244 -86.68103436106746) bank165841 +165842 POINT(41.27458193288714 -86.99305098363536) bank165842 +165843 POINT(41.05904415585427 -86.89296457679559) bank165843 +165844 POINT(42.58361250273805 -88.47675151110855) bank165844 +165845 POINT(42.872296264202376 -87.6777568406307) bank165845 +165846 POINT(41.438037686204055 -88.51524305102913) bank165846 +165847 POINT(41.95432345456789 -87.16267651718836) bank165847 +165848 POINT(41.36475266544238 -88.36112658644595) bank165848 +165849 POINT(41.75409371987798 -88.45019016450588) bank165849 +165850 POINT(42.3939874523897 -87.19597452783049) bank165850 +165851 POINT(41.51717913713675 -87.50131581659575) bank165851 +165852 POINT(42.20603260203441 -86.9355520529432) bank165852 +165853 POINT(40.95282347386825 -87.46083122960637) bank165853 +165854 POINT(41.100849057299904 -87.28313987181993) bank165854 +165855 POINT(41.078411880294226 -87.82765529277741) bank165855 +165856 POINT(41.14182495529524 -87.62855250475663) bank165856 +165857 POINT(42.62606956808067 -87.47617700826788) bank165857 +165858 POINT(42.04646982981721 -87.03780577084636) bank165858 +165859 POINT(42.49704144135355 -87.62606827037966) bank165859 +165860 POINT(42.05840949562896 -86.95332667876764) bank165860 +165861 POINT(42.15358411522489 -88.00593132854753) bank165861 +165862 POINT(42.17467036746844 -86.98623864418924) bank165862 +165863 POINT(41.83153099479556 -88.23048357909389) bank165863 +165864 POINT(42.14963137180947 -86.78184315765097) bank165864 +165865 POINT(42.36009750969426 -86.72247700281825) bank165865 +165866 POINT(41.89723785286158 -87.68343419478241) bank165866 +165867 POINT(41.55430525416989 -87.1694890967545) bank165867 +165868 POINT(42.40264557714116 -88.62827579753906) bank165868 +165869 POINT(41.025982011667246 -87.55062638812187) bank165869 +165870 POINT(41.986428458113004 -87.01558550143822) bank165870 +165871 POINT(41.27134397587172 -86.85937455047593) bank165871 +165872 POINT(42.75595667508639 -88.29447035482445) bank165872 +165873 POINT(41.0993078050402 -88.20868821053394) bank165873 +165874 POINT(42.29343364188165 -88.13224374313324) bank165874 +165875 POINT(42.350712754983 -88.28782123922862) bank165875 +165876 POINT(41.043494497830146 -87.1907704315102) bank165876 +165877 POINT(41.36785969382702 -87.87008744012292) bank165877 +165878 POINT(42.28860922539691 -87.39131911414489) bank165878 +165879 POINT(42.26381804279242 -87.29962134944141) bank165879 +165880 POINT(41.40115633161433 -88.48637565711766) bank165880 +165881 POINT(41.3675928627466 -86.65898709607077) bank165881 +165882 POINT(41.59247483968802 -88.0290242923692) bank165882 +165883 POINT(42.61330367237924 -87.72692711943034) bank165883 +165884 POINT(40.915434310239704 -88.0467068983956) bank165884 +165885 POINT(41.190274040512804 -87.66907454644321) bank165885 +165886 POINT(41.46081873139235 -87.69359773224195) bank165886 +165887 POINT(41.3684800138347 -86.6706712508926) bank165887 +165888 POINT(41.69132476590812 -88.00531143380512) bank165888 +165889 POINT(42.31600612842191 -87.77218641981611) bank165889 +165890 POINT(41.406786280541965 -86.69709024532914) bank165890 +165891 POINT(40.978157156414696 -87.47443256190847) bank165891 +165892 POINT(42.08535268973438 -87.71556942843338) bank165892 +165893 POINT(42.12672168124453 -86.6333962843694) bank165893 +165894 POINT(42.79204002132631 -88.21235345377312) bank165894 +165895 POINT(42.39527691419598 -88.57010458186144) bank165895 +165896 POINT(42.17821255091231 -87.60259887393201) bank165896 +165897 POINT(41.65044905815198 -87.3504239484702) bank165897 +165898 POINT(41.81364273595919 -88.13796190174189) bank165898 +165899 POINT(41.57957392671626 -88.25738493304799) bank165899 +165900 POINT(41.451077833892434 -86.68903380069327) bank165900 +165901 POINT(42.37896775022648 -86.8228426747098) bank165901 +165902 POINT(42.16908836002341 -88.36349871711508) bank165902 +165903 POINT(41.142846661886324 -87.15552252316084) bank165903 +165904 POINT(41.37479227250996 -88.34929734334402) bank165904 +165905 POINT(42.21236552085894 -88.04250900101658) bank165905 +165906 POINT(41.271503016437414 -87.65104700869648) bank165906 +165907 POINT(42.45404419518317 -88.60547411013856) bank165907 +165908 POINT(42.79191183382836 -88.44991369179006) bank165908 +165909 POINT(42.41995020546005 -88.03823689717368) bank165909 +165910 POINT(41.178673406564634 -86.98540531918832) bank165910 +165911 POINT(42.35610616036462 -87.87216996262724) bank165911 +165912 POINT(42.21302142259036 -87.43156119653013) bank165912 +165913 POINT(41.21021511075163 -87.31720213699995) bank165913 +165914 POINT(41.57991406731999 -88.37156604248833) bank165914 +165915 POINT(42.20022466386124 -87.22324016285027) bank165915 +165916 POINT(42.45537669486945 -87.68605598958887) bank165916 +165917 POINT(41.87140003137436 -87.01856062075998) bank165917 +165918 POINT(42.35376531389329 -86.7805840000462) bank165918 +165919 POINT(41.2644351204341 -88.22178585931326) bank165919 +165920 POINT(41.78740148656544 -87.99263520952834) bank165920 +165921 POINT(41.97627769605495 -87.16100801298187) bank165921 +165922 POINT(40.916681991484985 -87.02032686072691) bank165922 +165923 POINT(41.044883569158166 -86.69974148531263) bank165923 +165924 POINT(41.27551150517629 -86.97991084992758) bank165924 +165925 POINT(41.967726262189295 -87.00048085203409) bank165925 +165926 POINT(41.46006782744338 -87.8519165412737) bank165926 +165927 POINT(41.00834452780383 -88.1114780941834) bank165927 +165928 POINT(42.50604279512514 -87.49389290927783) bank165928 +165929 POINT(41.064694466487765 -87.09732062737463) bank165929 +165930 POINT(42.28546418490546 -87.55739715844145) bank165930 +165931 POINT(41.688027802372034 -87.97398340672072) bank165931 +165932 POINT(40.954258859924344 -87.33588106035349) bank165932 +165933 POINT(42.59812637383599 -87.14735805887878) bank165933 +165934 POINT(42.39920162128516 -87.15422412152782) bank165934 +165935 POINT(41.536603437364455 -88.62437927455504) bank165935 +165936 POINT(42.487161593635896 -88.38681232896027) bank165936 +165937 POINT(42.76658425468824 -86.64476929174417) bank165937 +165938 POINT(41.52369435200818 -86.80890230486837) bank165938 +165939 POINT(42.61402527748036 -86.68845357225005) bank165939 +165940 POINT(42.67259516737193 -86.78315408543511) bank165940 +165941 POINT(41.29588534871263 -87.78912318490703) bank165941 +165942 POINT(42.3495638716015 -87.91893739539898) bank165942 +165943 POINT(41.97390411214337 -87.21482845538313) bank165943 +165944 POINT(42.57300301120114 -87.76783167019367) bank165944 +165945 POINT(41.199035329758864 -87.87287647080416) bank165945 +165946 POINT(41.53313588357176 -87.43090589693851) bank165946 +165947 POINT(42.12560548505343 -87.94226417758831) bank165947 +165948 POINT(41.50408051222713 -88.57497761617255) bank165948 +165949 POINT(41.28612972863997 -86.8601762686179) bank165949 +165950 POINT(42.158694429972535 -87.11542892472296) bank165950 +165951 POINT(41.77403803786584 -87.87807442589109) bank165951 +165952 POINT(40.94903893802649 -86.84345882785735) bank165952 +165953 POINT(41.64481526272719 -87.76012505237254) bank165953 +165954 POINT(42.03014914633415 -87.88257505300584) bank165954 +165955 POINT(42.50311359329687 -88.02253202447515) bank165955 +165956 POINT(42.57519049673386 -87.05869761931145) bank165956 +165957 POINT(41.14899986109985 -87.9998113206754) bank165957 +165958 POINT(41.39745137994697 -88.36093009906352) bank165958 +165959 POINT(41.13113828909265 -87.00893881330045) bank165959 +165960 POINT(40.91696981645931 -86.78454319647082) bank165960 +165961 POINT(42.541442686831644 -87.42077161469793) bank165961 +165962 POINT(41.08341825124312 -86.80149148329393) bank165962 +165963 POINT(41.17176492870035 -86.75433995147154) bank165963 +165964 POINT(42.75874195833192 -87.30202799397729) bank165964 +165965 POINT(42.78750589149292 -88.55575740153513) bank165965 +165966 POINT(42.8297432781566 -87.15743090515383) bank165966 +165967 POINT(41.35828127246322 -87.05036420846697) bank165967 +165968 POINT(42.58887542547369 -88.29191563917999) bank165968 +165969 POINT(42.20411301570461 -87.70583334298503) bank165969 +165970 POINT(41.20119898020087 -86.65865103356062) bank165970 +165971 POINT(42.56503967320682 -88.35317932577841) bank165971 +165972 POINT(41.127119369917374 -87.0947227251782) bank165972 +165973 POINT(41.534863249664326 -87.49195434037131) bank165973 +165974 POINT(42.07958217934742 -88.6286035470272) bank165974 +165975 POINT(42.4206116967334 -88.35939394358805) bank165975 +165976 POINT(41.8950402448141 -88.46935580137388) bank165976 +165977 POINT(41.641371637365864 -88.38300965182552) bank165977 +165978 POINT(41.52475614223808 -87.74030029001956) bank165978 +165979 POINT(42.20209221191837 -86.86251937486387) bank165979 +165980 POINT(41.07090142244735 -88.15774662599031) bank165980 +165981 POINT(41.53675645798224 -87.36060529560986) bank165981 +165982 POINT(41.364166399565 -87.581977200494) bank165982 +165983 POINT(41.3045731880892 -87.31379832234563) bank165983 +165984 POINT(42.07205270994165 -87.50365901287917) bank165984 +165985 POINT(42.307211976220636 -87.12673807946165) bank165985 +165986 POINT(42.4762196759883 -86.64616732719182) bank165986 +165987 POINT(41.50825196996756 -88.24946192845475) bank165987 +165988 POINT(42.59262792796542 -88.51875349369334) bank165988 +165989 POINT(42.87043406127191 -87.55333009769694) bank165989 +165990 POINT(42.03998672298535 -87.99213109344554) bank165990 +165991 POINT(40.93835053345044 -87.94828448035645) bank165991 +165992 POINT(42.07211117531983 -88.61096623157927) bank165992 +165993 POINT(41.638637617577096 -87.04480492656312) bank165993 +165994 POINT(42.26178840310415 -87.416654105918) bank165994 +165995 POINT(41.81583996113416 -88.34639960239834) bank165995 +165996 POINT(42.34816753550746 -87.53571260375095) bank165996 +165997 POINT(42.24814177910629 -86.82806586683746) bank165997 +165998 POINT(42.35838062511397 -87.66958121368724) bank165998 +165999 POINT(42.33103752601471 -87.04030224943725) bank165999 +166000 POINT(41.350894099104465 -88.31260190231264) bank166000 +166001 POINT(41.966834638928994 -88.1065446212606) bank166001 +166002 POINT(42.78541337797273 -88.3985761354075) bank166002 +166003 POINT(41.53276494411233 -87.28826688444312) bank166003 +166004 POINT(42.489845798090236 -87.7309141175989) bank166004 +166005 POINT(41.913436675384254 -87.02233159651493) bank166005 +166006 POINT(41.69146412667318 -87.17236844827087) bank166006 +166007 POINT(42.53782913881034 -87.89310458748342) bank166007 +166008 POINT(42.09249506955984 -86.67735420275461) bank166008 +166009 POINT(42.36621974003392 -87.82879902326201) bank166009 +166010 POINT(41.95778359310213 -87.61159917363698) bank166010 +166011 POINT(41.789771152227814 -87.83539530326246) bank166011 +166012 POINT(42.23008404939396 -87.67164426478016) bank166012 +166013 POINT(42.67742726796038 -87.00391696967117) bank166013 +166014 POINT(41.38716269769011 -87.11141427944199) bank166014 +166015 POINT(41.10421539336513 -87.39557255946413) bank166015 +166016 POINT(41.162760334104306 -87.3973083931565) bank166016 +166017 POINT(41.39767316028238 -87.75979827589555) bank166017 +166018 POINT(41.711344603934904 -87.5437935547075) bank166018 +166019 POINT(42.65958339541654 -88.20747550995665) bank166019 +166020 POINT(41.04849724154774 -88.00392280124099) bank166020 +166021 POINT(42.370316440399854 -86.8844800597582) bank166021 +166022 POINT(41.47525810767282 -88.54897978435363) bank166022 +166023 POINT(42.559923439740295 -87.10663251980738) bank166023 +166024 POINT(42.68406655870237 -87.67841299655332) bank166024 +166025 POINT(41.89698149853153 -88.13813789343537) bank166025 +166026 POINT(42.52669315356527 -87.32063226336561) bank166026 +166027 POINT(41.17658268790821 -86.92531193107475) bank166027 +166028 POINT(41.81869401503628 -86.90565489163318) bank166028 +166029 POINT(42.22175354417557 -87.14373357264374) bank166029 +166030 POINT(41.30471536295159 -88.53083574304716) bank166030 +166031 POINT(41.97663961677952 -87.91614552883756) bank166031 +166032 POINT(42.64188828787216 -87.6211770154023) bank166032 +166033 POINT(41.62487444770137 -87.60245562343061) bank166033 +166034 POINT(42.116506214812794 -87.84116067061647) bank166034 +166035 POINT(41.94964872025943 -88.49597854007668) bank166035 +166036 POINT(41.50135451439887 -88.5355589131377) bank166036 +166037 POINT(41.87544283787856 -86.94935791566625) bank166037 +166038 POINT(42.71715138798312 -87.44332017746419) bank166038 +166039 POINT(41.03687317808063 -87.08793438367519) bank166039 +166040 POINT(42.17718842424097 -87.60176193777318) bank166040 +166041 POINT(41.49117568914473 -86.9211688622559) bank166041 +166042 POINT(41.54674487233676 -87.86695464147482) bank166042 +166043 POINT(41.56556679051091 -88.57567047864374) bank166043 +166044 POINT(42.22129004093944 -87.60876553936721) bank166044 +166045 POINT(41.04673775744254 -88.07279195104407) bank166045 +166046 POINT(41.67704470440312 -88.51257718618352) bank166046 +166047 POINT(42.77418496337072 -86.71407449035534) bank166047 +166048 POINT(41.50557148754243 -87.19936541704884) bank166048 +166049 POINT(40.95067573666164 -88.2978479855754) bank166049 +166050 POINT(42.29313991335897 -87.07562392751066) bank166050 +166051 POINT(42.331647946609145 -87.60291803559168) bank166051 +166052 POINT(41.24967723688771 -86.88675633796647) bank166052 +166053 POINT(42.65262480628212 -86.86288342813913) bank166053 +166054 POINT(42.63402717522258 -87.59722979550244) bank166054 +166055 POINT(42.10841993949792 -87.28758578986086) bank166055 +166056 POINT(42.83129299111017 -87.53445989448291) bank166056 +166057 POINT(41.933967497527384 -87.43996562784224) bank166057 +166058 POINT(41.690095052955044 -86.88898432609065) bank166058 +166059 POINT(41.87233140906784 -87.77268771388911) bank166059 +166060 POINT(41.72711642189117 -87.71631781205804) bank166060 +166061 POINT(41.3164226573737 -87.1943777829964) bank166061 +166062 POINT(41.512707262710315 -87.57324036207118) bank166062 +166063 POINT(42.19365540011153 -87.0294113006626) bank166063 +166064 POINT(41.02450658896848 -87.18771963282254) bank166064 +166065 POINT(41.779793080399294 -88.18993953636823) bank166065 +166066 POINT(41.50206659755539 -86.75513143042261) bank166066 +166067 POINT(41.17732930243702 -88.16333678377012) bank166067 +166068 POINT(42.08554258776868 -86.90775633754812) bank166068 +166069 POINT(41.92796461715196 -87.85682110257851) bank166069 +166070 POINT(42.34313088309978 -87.7422550998595) bank166070 +166071 POINT(41.4959525968159 -87.54391868677843) bank166071 +166072 POINT(41.03114752970648 -86.88161048108974) bank166072 +166073 POINT(42.05679293404925 -88.5240065631781) bank166073 +166074 POINT(42.67489708482808 -88.09063713992471) bank166074 +166075 POINT(41.14994934094926 -88.00624180632327) bank166075 +166076 POINT(41.83413528018395 -87.1991993272845) bank166076 +166077 POINT(42.10945738452163 -86.86188231481891) bank166077 +166078 POINT(41.92570957710146 -86.89132493736822) bank166078 +166079 POINT(41.477527003702214 -87.25419771729364) bank166079 +166080 POINT(42.69646211945722 -86.69273307611945) bank166080 +166081 POINT(42.645178435662736 -88.43839965083478) bank166081 +166082 POINT(41.84533153882558 -87.3391525792468) bank166082 +166083 POINT(40.92709058514417 -87.44141583638485) bank166083 +166084 POINT(42.59168805220201 -88.29764886327028) bank166084 +166085 POINT(41.191242965506014 -88.42656367587993) bank166085 +166086 POINT(42.59181938781867 -87.37574720173791) bank166086 +166087 POINT(41.610446565332545 -86.85077635792834) bank166087 +166088 POINT(41.34253097706176 -88.27369195554995) bank166088 +166089 POINT(41.36291225853988 -87.43152293786375) bank166089 +166090 POINT(41.68524099714354 -87.94332560150777) bank166090 +166091 POINT(42.021251556855646 -87.75412843894499) bank166091 +166092 POINT(41.19533920067397 -88.44250696428487) bank166092 +166093 POINT(41.27481377997182 -87.30815797644193) bank166093 +166094 POINT(42.27056540603833 -88.03931213649054) bank166094 +166095 POINT(41.29326150820474 -86.88038710445733) bank166095 +166096 POINT(41.20718717504441 -86.92978281858693) bank166096 +166097 POINT(42.85445733767778 -87.06263003915892) bank166097 +166098 POINT(42.16043093399598 -88.12257491313623) bank166098 +166099 POINT(41.56281957367688 -87.23079605425119) bank166099 +166100 POINT(42.40391823570844 -87.15810176840536) bank166100 +166101 POINT(41.887875067578655 -87.16016784648328) bank166101 +166102 POINT(42.57547086301174 -87.30947482289933) bank166102 +166103 POINT(42.03316915739177 -88.58340616658977) bank166103 +166104 POINT(41.75862605874544 -87.45863255305747) bank166104 +166105 POINT(42.54338787292274 -86.87839703763956) bank166105 +166106 POINT(41.54615474185879 -88.01949233982833) bank166106 +166107 POINT(41.015860462828044 -88.4456366392139) bank166107 +166108 POINT(42.2159699614794 -87.78824015442547) bank166108 +166109 POINT(42.38581871536297 -87.33245612282549) bank166109 +166110 POINT(42.35634061404888 -87.2885202569583) bank166110 +166111 POINT(40.95846615777529 -88.28967487160006) bank166111 +166112 POINT(41.71500374081898 -88.22149890001583) bank166112 +166113 POINT(42.55075076560751 -88.20561617275948) bank166113 +166114 POINT(40.93833115135156 -88.05397043849558) bank166114 +166115 POINT(42.00900029885683 -87.29747057866854) bank166115 +166116 POINT(42.337857772905814 -87.72503386662993) bank166116 +166117 POINT(42.2101536955535 -87.52714472091935) bank166117 +166118 POINT(42.362597889021124 -88.10353744492342) bank166118 +166119 POINT(41.26779614110411 -87.32234184516545) bank166119 +166120 POINT(41.238557330449325 -87.53427807351763) bank166120 +166121 POINT(41.62929411407924 -88.58671360972232) bank166121 +166122 POINT(41.786870378423515 -86.97236350797336) bank166122 +166123 POINT(41.86174746218862 -88.09032976308319) bank166123 +166124 POINT(41.03229362022418 -87.27824236583112) bank166124 +166125 POINT(42.15917955270983 -86.96653696160865) bank166125 +166126 POINT(42.3314088521729 -88.1940830945415) bank166126 +166127 POINT(41.57779740600253 -88.10088454228122) bank166127 +166128 POINT(41.44488256723982 -88.5288730775912) bank166128 +166129 POINT(42.84900467548476 -88.2449988633944) bank166129 +166130 POINT(42.03068731042496 -87.419020613314) bank166130 +166131 POINT(41.117682303768404 -88.58445191634618) bank166131 +166132 POINT(42.785495643508476 -88.0928906875243) bank166132 +166133 POINT(41.31447916405068 -87.42868812995003) bank166133 +166134 POINT(41.26772681604188 -88.29468560224187) bank166134 +166135 POINT(42.81036481848781 -88.1726124257074) bank166135 +166136 POINT(41.89060908536025 -88.28488590678175) bank166136 +166137 POINT(42.51884630166027 -87.26174166280784) bank166137 +166138 POINT(42.725381542266724 -87.20043172490965) bank166138 +166139 POINT(41.954333970217725 -88.25465503973439) bank166139 +166140 POINT(41.42389052466417 -88.44645417019194) bank166140 +166141 POINT(41.38456741220807 -87.82695398222471) bank166141 +166142 POINT(42.052847362539794 -86.65757863090955) bank166142 +166143 POINT(42.28839388796862 -87.57621974459578) bank166143 +166144 POINT(42.35219969477063 -88.27466400719466) bank166144 +166145 POINT(41.27152302956059 -87.74805013341206) bank166145 +166146 POINT(42.16111810477121 -88.60626182727906) bank166146 +166147 POINT(41.121487334627865 -86.91730131850919) bank166147 +166148 POINT(41.338128270701795 -87.9852834734926) bank166148 +166149 POINT(41.24291510397783 -86.85587933080292) bank166149 +166150 POINT(41.24083139148782 -88.46638410965775) bank166150 +166151 POINT(42.062262771914206 -88.50829962349216) bank166151 +166152 POINT(42.799611885226625 -87.66030926243359) bank166152 +166153 POINT(42.087988077543976 -88.18811144410638) bank166153 +166154 POINT(42.51741880958009 -88.50617782949759) bank166154 +166155 POINT(42.70325795371592 -88.61813018328498) bank166155 +166156 POINT(41.6567929824764 -88.09025525656386) bank166156 +166157 POINT(42.114525979949995 -88.51812736443611) bank166157 +166158 POINT(42.483728182873236 -88.47946066157041) bank166158 +166159 POINT(42.29665834301565 -88.20598022569943) bank166159 +166160 POINT(41.00431636698169 -86.64768306892694) bank166160 +166161 POINT(42.12269787150962 -88.58927260645156) bank166161 +166162 POINT(42.422053826558006 -88.16259453305368) bank166162 +166163 POINT(42.04509415972572 -87.26350698366556) bank166163 +166164 POINT(41.3092086687132 -87.44027370591571) bank166164 +166165 POINT(42.122255269350966 -86.83903173322618) bank166165 +166166 POINT(41.44950755249746 -88.14742457686985) bank166166 +166167 POINT(42.41908441423306 -86.99812426587789) bank166167 +166168 POINT(42.68520291175147 -88.14940317008627) bank166168 +166169 POINT(42.22307934786104 -87.5574758104389) bank166169 +166170 POINT(41.452034273148485 -88.58368518992258) bank166170 +166171 POINT(40.98209904431009 -88.11675568739469) bank166171 +166172 POINT(41.735477016892936 -86.80377995192181) bank166172 +166173 POINT(42.71684733283318 -88.1033164333651) bank166173 +166174 POINT(42.06615697200126 -86.94107855079267) bank166174 +166175 POINT(42.78159937887815 -88.35207596912889) bank166175 +166176 POINT(41.2413619646475 -86.67319390210037) bank166176 +166177 POINT(41.056488236772616 -87.56016340068217) bank166177 +166178 POINT(42.53490386558244 -87.78426152040242) bank166178 +166179 POINT(41.91166705492668 -88.15653547448602) bank166179 +166180 POINT(42.08675876302381 -87.64665811540884) bank166180 +166181 POINT(40.96776336647589 -88.25217604823084) bank166181 +166182 POINT(41.157120231090246 -87.87825963868782) bank166182 +166183 POINT(41.754419032427286 -87.41398487834661) bank166183 +166184 POINT(41.322253949808015 -87.66423448897154) bank166184 +166185 POINT(41.60966509387647 -87.55003344107197) bank166185 +166186 POINT(41.02688570964857 -86.82496437701849) bank166186 +166187 POINT(41.1723356123535 -87.83217161256559) bank166187 +166188 POINT(41.41122282443825 -87.82615093882922) bank166188 +166189 POINT(41.655308358215784 -87.71783096498721) bank166189 +166190 POINT(42.37423418665983 -86.9915774626366) bank166190 +166191 POINT(41.40233179420409 -88.32334815539377) bank166191 +166192 POINT(42.59206272358356 -87.18330492729534) bank166192 +166193 POINT(42.254361967954345 -87.28505975804265) bank166193 +166194 POINT(42.29169891750532 -87.67404270390227) bank166194 +166195 POINT(41.56451138774867 -87.38944243328476) bank166195 +166196 POINT(41.71906270845414 -86.70902456242291) bank166196 +166197 POINT(41.59004786350863 -87.72527172884303) bank166197 +166198 POINT(42.73335091487439 -87.74234967592011) bank166198 +166199 POINT(40.97138643272459 -87.72005130044785) bank166199 +166200 POINT(42.0033628123815 -87.37896929019375) bank166200 +166201 POINT(41.72175598295508 -87.17009459792757) bank166201 +166202 POINT(40.98188903547435 -88.38430922240522) bank166202 +166203 POINT(41.13665618022799 -87.12007010865756) bank166203 +166204 POINT(42.30105014557788 -88.33424981763656) bank166204 +166205 POINT(42.61237228330928 -88.60346606364092) bank166205 +166206 POINT(42.17607758305778 -87.82792305289983) bank166206 +166207 POINT(41.01048159992159 -87.67998150553457) bank166207 +166208 POINT(41.042454370292184 -87.5414316773395) bank166208 +166209 POINT(42.814722372954336 -88.35859089276502) bank166209 +166210 POINT(41.03782071406814 -86.91640000304477) bank166210 +166211 POINT(42.462269135037936 -87.82770571105868) bank166211 +166212 POINT(42.83429869316966 -87.99868311724644) bank166212 +166213 POINT(42.530668109869865 -88.45034333212388) bank166213 +166214 POINT(41.745218679383036 -88.5267988865562) bank166214 +166215 POINT(41.45356253525492 -88.08678797859456) bank166215 +166216 POINT(41.97723950994161 -88.55175236590269) bank166216 +166217 POINT(42.35584635997336 -88.60593262911199) bank166217 +166218 POINT(42.235796387316036 -87.34244380783318) bank166218 +166219 POINT(41.00711588590615 -87.85674486730322) bank166219 +166220 POINT(42.34383869362296 -86.78124991050046) bank166220 +166221 POINT(41.80581535521828 -88.04270429941327) bank166221 +166222 POINT(42.30393359766391 -87.88867187028133) bank166222 +166223 POINT(42.25599534625254 -87.65789333264318) bank166223 +166224 POINT(42.7418301960598 -87.32591408518819) bank166224 +166225 POINT(41.03053488242456 -87.72641376352664) bank166225 +166226 POINT(41.65701915702566 -86.85978173676244) bank166226 +166227 POINT(42.41707373122689 -87.51345698501959) bank166227 +166228 POINT(42.05739903913486 -87.16748833786114) bank166228 +166229 POINT(42.58543010428325 -86.99538128520989) bank166229 +166230 POINT(42.71934156675546 -87.01769649509953) bank166230 +166231 POINT(42.517839793743434 -88.48216086277816) bank166231 +166232 POINT(41.76459573312953 -88.45982204007457) bank166232 +166233 POINT(42.348353594335556 -88.01449213371058) bank166233 +166234 POINT(42.69156900813015 -87.10434033653631) bank166234 +166235 POINT(41.13172668931955 -87.64189743271885) bank166235 +166236 POINT(42.22652869366709 -88.05125394446871) bank166236 +166237 POINT(41.55502984791703 -88.21389899793903) bank166237 +166238 POINT(42.00231905564889 -86.69033091535768) bank166238 +166239 POINT(41.80311628989411 -88.59315090354441) bank166239 +166240 POINT(42.212053637129145 -88.62214989441604) bank166240 +166241 POINT(41.016650403335284 -86.84132231109402) bank166241 +166242 POINT(41.97193563445682 -88.29411203668413) bank166242 +166243 POINT(41.439893752356134 -88.08733788914729) bank166243 +166244 POINT(41.676779696329746 -88.12102375258809) bank166244 +166245 POINT(42.7716063311716 -87.00675905863608) bank166245 +166246 POINT(42.30039877374872 -87.12537431947551) bank166246 +166247 POINT(42.11306194171521 -86.7227487507356) bank166247 +166248 POINT(42.730843718283005 -88.02646047412601) bank166248 +166249 POINT(41.88024284005933 -87.39404814502768) bank166249 +166250 POINT(41.31740009840539 -87.18101505445598) bank166250 +166251 POINT(42.16792769688695 -86.63067903417853) bank166251 +166252 POINT(42.24891567141121 -87.55875573926684) bank166252 +166253 POINT(42.752312023910896 -88.42970155920457) bank166253 +166254 POINT(41.362606400497405 -87.74927848420916) bank166254 +166255 POINT(42.211114105386386 -86.66079433425057) bank166255 +166256 POINT(42.12971865503861 -88.00424660355844) bank166256 +166257 POINT(41.34487913840788 -87.54283117097746) bank166257 +166258 POINT(42.30887626434468 -87.93139130549162) bank166258 +166259 POINT(41.969313179765415 -87.34013084980481) bank166259 +166260 POINT(42.54745491947607 -87.24260933130506) bank166260 +166261 POINT(42.36090073900892 -87.3397008986386) bank166261 +166262 POINT(41.86065575679188 -88.36328990044294) bank166262 +166263 POINT(41.14658770449418 -86.69250801539985) bank166263 +166264 POINT(40.91891462088226 -88.04503822227976) bank166264 +166265 POINT(42.77178676664798 -88.3789154603321) bank166265 +166266 POINT(40.945278624108894 -88.25434382996835) bank166266 +166267 POINT(42.011845496218584 -87.95110903746257) bank166267 +166268 POINT(42.275998882287496 -87.09097463309858) bank166268 +166269 POINT(41.844361537574535 -86.8889537249556) bank166269 +166270 POINT(42.44786735008719 -86.67316287048651) bank166270 +166271 POINT(41.37976324593974 -86.78970907333026) bank166271 +166272 POINT(42.405909140817684 -88.00193427234606) bank166272 +166273 POINT(41.206301086808836 -86.65040678991407) bank166273 +166274 POINT(42.61862481127608 -87.35393508360706) bank166274 +166275 POINT(42.69250577296747 -87.5928756969425) bank166275 +166276 POINT(41.361139695192996 -87.22480559242088) bank166276 +166277 POINT(41.324451546276684 -88.24427585064336) bank166277 +166278 POINT(42.30803405521258 -88.09330864899817) bank166278 +166279 POINT(42.78395437356443 -88.40866700057201) bank166279 +166280 POINT(42.567832631706324 -88.0740531222813) bank166280 +166281 POINT(42.780897905178286 -87.01129276173656) bank166281 +166282 POINT(41.19203010079055 -87.75960548079802) bank166282 +166283 POINT(41.88334560230359 -88.42130062694363) bank166283 +166284 POINT(42.472472525903115 -88.26153038165894) bank166284 +166285 POINT(42.01711329362015 -87.68475929725172) bank166285 +166286 POINT(41.115317166355176 -87.82483937434864) bank166286 +166287 POINT(41.11762867856648 -86.88007273308234) bank166287 +166288 POINT(42.272236458141535 -87.17056711289099) bank166288 +166289 POINT(41.04920751116324 -86.95652738250763) bank166289 +166290 POINT(41.35204349490547 -88.41809633687441) bank166290 +166291 POINT(41.12919033838869 -88.04481846503994) bank166291 +166292 POINT(42.47225923272598 -86.7418601327427) bank166292 +166293 POINT(42.718063485344594 -87.46673679803222) bank166293 +166294 POINT(41.73928331526025 -87.16096469453916) bank166294 +166295 POINT(40.88821850777449 -87.54059613447724) bank166295 +166296 POINT(41.54680869010783 -88.60480863204182) bank166296 +166297 POINT(42.69990845690364 -88.12472266295578) bank166297 +166298 POINT(41.46444958898078 -87.29188879404465) bank166298 +166299 POINT(42.037184339619266 -87.84808083803597) bank166299 +166300 POINT(42.47634248001258 -87.35089637374895) bank166300 +166301 POINT(41.77220769034556 -88.314883546787) bank166301 +166302 POINT(42.369518714314864 -88.47756187251484) bank166302 +166303 POINT(40.90928975515473 -88.58998429198675) bank166303 +166304 POINT(41.92395290189299 -86.78547086248557) bank166304 +166305 POINT(41.66122787262107 -87.90155153791925) bank166305 +166306 POINT(41.23773347088582 -86.80915434951838) bank166306 +166307 POINT(42.517006889537 -86.76405413623512) bank166307 +166308 POINT(42.53160875692276 -86.74291053198117) bank166308 +166309 POINT(42.357134614012665 -86.94997610623773) bank166309 +166310 POINT(41.30791842732146 -88.46625683472946) bank166310 +166311 POINT(42.75701739996204 -87.19261519632919) bank166311 +166312 POINT(42.34945814326651 -88.12361537863472) bank166312 +166313 POINT(42.52413654451746 -88.05740343630534) bank166313 +166314 POINT(42.753916355817665 -87.6996562283781) bank166314 +166315 POINT(42.08330565992422 -88.57237226401747) bank166315 +166316 POINT(42.70687837092298 -87.27529010744695) bank166316 +166317 POINT(41.2082548385982 -86.90875363072375) bank166317 +166318 POINT(41.29156089363651 -87.3239162633418) bank166318 +166319 POINT(41.768264606057826 -87.3631180336458) bank166319 +166320 POINT(41.45836132384574 -86.80290698976489) bank166320 +166321 POINT(41.3159659446167 -87.8280013208385) bank166321 +166322 POINT(42.263708739850216 -87.383575621363) bank166322 +166323 POINT(41.07605198091164 -88.58388371371993) bank166323 +166324 POINT(42.39149814200155 -87.2950953482524) bank166324 +166325 POINT(41.242349018772266 -87.30068174488996) bank166325 +166326 POINT(40.994050469084655 -87.96568463858856) bank166326 +166327 POINT(42.667788747581334 -87.05325636594267) bank166327 +166328 POINT(41.54775404663973 -88.34902251052867) bank166328 +166329 POINT(41.14565399841319 -88.59701336259488) bank166329 +166330 POINT(41.6782698242029 -88.3247428302756) bank166330 +166331 POINT(42.240490705058136 -88.06496885562363) bank166331 +166332 POINT(42.653717387016904 -87.63649918076052) bank166332 +166333 POINT(41.40132697998102 -88.54819528918209) bank166333 +166334 POINT(41.57996346365016 -87.59186739855012) bank166334 +166335 POINT(42.832424378818956 -87.9244887111049) bank166335 +166336 POINT(41.75243792795677 -86.96280864029232) bank166336 +166337 POINT(42.04060471393233 -88.46903832773619) bank166337 +166338 POINT(42.6803543535859 -86.7973978414225) bank166338 +166339 POINT(42.35066467973027 -87.79337146371626) bank166339 +166340 POINT(41.732222819162295 -88.61610213057887) bank166340 +166341 POINT(41.05900501938339 -87.17299595524938) bank166341 +166342 POINT(42.800606138829444 -87.53676868328266) bank166342 +166343 POINT(41.81594583868764 -86.80772101622061) bank166343 +166344 POINT(41.786384836445215 -88.44389177928872) bank166344 +166345 POINT(42.83207143301668 -87.8915057645387) bank166345 +166346 POINT(41.322447195314616 -88.00509528716967) bank166346 +166347 POINT(42.441722436052025 -87.22710625049778) bank166347 +166348 POINT(41.26607617692742 -87.82019972824867) bank166348 +166349 POINT(42.39806099111274 -87.60681146806586) bank166349 +166350 POINT(42.18568856349124 -87.12646791440174) bank166350 +166351 POINT(42.24649678826866 -88.124485418352) bank166351 +166352 POINT(41.44269396918228 -86.71259139785218) bank166352 +166353 POINT(42.123608806203244 -88.34442783466201) bank166353 +166354 POINT(41.28843508196363 -88.10468529215288) bank166354 +166355 POINT(41.115016866712566 -87.99378070654477) bank166355 +166356 POINT(41.06462476405286 -86.718214550198) bank166356 +166357 POINT(42.1408947559734 -86.68170629335616) bank166357 +166358 POINT(42.16007318680719 -88.039341226825) bank166358 +166359 POINT(42.085524150580916 -88.60020140949388) bank166359 +166360 POINT(41.88016938366967 -88.14086763483549) bank166360 +166361 POINT(41.80301031580262 -88.33639369014432) bank166361 +166362 POINT(42.584796672927325 -88.05067168643252) bank166362 +166363 POINT(42.699442069041176 -87.19558239020546) bank166363 +166364 POINT(42.24393986122735 -86.70376129134361) bank166364 +166365 POINT(41.49204096613368 -88.42955101326643) bank166365 +166366 POINT(41.97181715912581 -86.99035824325532) bank166366 +166367 POINT(40.884451921774705 -87.31665391464465) bank166367 +166368 POINT(42.19549979512856 -87.46494243092255) bank166368 +166369 POINT(42.27924299981487 -88.07104642214141) bank166369 +166370 POINT(41.83252785949494 -88.35683533961132) bank166370 +166371 POINT(40.97073898396808 -88.39422238332602) bank166371 +166372 POINT(40.91628545660167 -87.37100345120321) bank166372 +166373 POINT(41.05397709712454 -86.76460186753924) bank166373 +166374 POINT(41.84767587867986 -88.0587456015897) bank166374 +166375 POINT(41.35851501603082 -87.56504786859782) bank166375 +166376 POINT(42.2039890348489 -86.77774549466818) bank166376 +166377 POINT(41.66333325525387 -87.13623087500818) bank166377 +166378 POINT(41.01467811039429 -87.38737567421794) bank166378 +166379 POINT(41.11538859003571 -88.16624288561049) bank166379 +166380 POINT(41.08412672236498 -88.35165163562418) bank166380 +166381 POINT(41.592934866720874 -86.89998694308773) bank166381 +166382 POINT(42.490126576836765 -88.06519860485344) bank166382 +166383 POINT(41.6121080653067 -88.55063628145241) bank166383 +166384 POINT(42.211830855136135 -88.49542969808734) bank166384 +166385 POINT(41.412127371595 -87.55038825295735) bank166385 +166386 POINT(42.28016501853629 -88.59046995799551) bank166386 +166387 POINT(41.70482261987137 -87.23400710800416) bank166387 +166388 POINT(41.120379367057126 -86.95761536734042) bank166388 +166389 POINT(42.85353985876583 -87.43068650480372) bank166389 +166390 POINT(41.5035999429917 -86.9560740399689) bank166390 +166391 POINT(41.71274655349393 -87.37503617171117) bank166391 +166392 POINT(42.615577261472005 -87.0224873153558) bank166392 +166393 POINT(41.81025004513498 -88.2852621426838) bank166393 +166394 POINT(41.68147835153864 -86.75607585579706) bank166394 +166395 POINT(42.725064822025054 -86.95514545015378) bank166395 +166396 POINT(42.01112803030111 -86.74029244468439) bank166396 +166397 POINT(42.579877609125 -87.07406867965213) bank166397 +166398 POINT(41.310132103402296 -87.26186210132467) bank166398 +166399 POINT(42.65110708459247 -87.68717473207283) bank166399 +166400 POINT(41.529429322810145 -87.00259898920389) bank166400 +166401 POINT(41.06940655246175 -87.555588772817) bank166401 +166402 POINT(40.882931945102904 -88.19071315235308) bank166402 +166403 POINT(41.14738661942651 -87.70948352446138) bank166403 +166404 POINT(42.37818778373986 -87.80909637481086) bank166404 +166405 POINT(42.152894404753184 -88.29703516122616) bank166405 +166406 POINT(42.477460059875476 -88.11010786663661) bank166406 +166407 POINT(41.354993847078575 -87.75836364370718) bank166407 +166408 POINT(41.88175003697875 -88.40570846403138) bank166408 +166409 POINT(41.01666395426141 -87.01473000216176) bank166409 +166410 POINT(42.8714996463676 -87.42123713520742) bank166410 +166411 POINT(41.00796639721823 -87.51608690234305) bank166411 +166412 POINT(42.285391364763264 -88.28400386461274) bank166412 +166413 POINT(41.72794477438926 -88.3707047281705) bank166413 +166414 POINT(42.557048179615094 -87.37617008929122) bank166414 +166415 POINT(41.1305468567766 -86.69683735835136) bank166415 +166416 POINT(42.32765643380953 -88.44565348687546) bank166416 +166417 POINT(42.831878991784855 -87.76123552784043) bank166417 +166418 POINT(41.06397394823049 -88.17726096039439) bank166418 +166419 POINT(41.098713501947216 -87.68667238122644) bank166419 +166420 POINT(41.79913911338322 -87.87770966067112) bank166420 +166421 POINT(42.298518493449286 -87.99896580759209) bank166421 +166422 POINT(41.868751228182674 -87.67124492607509) bank166422 +166423 POINT(41.17394599615536 -86.93412029135972) bank166423 +166424 POINT(41.156295859981306 -86.92697002773531) bank166424 +166425 POINT(42.06611484186874 -87.88217296720423) bank166425 +166426 POINT(41.016842622713796 -87.91866618357005) bank166426 +166427 POINT(41.58998137434894 -87.25677221057772) bank166427 +166428 POINT(41.38779474868289 -88.60398587405612) bank166428 +166429 POINT(42.30734705835017 -87.2954116224357) bank166429 +166430 POINT(42.0469680155151 -87.26164263896732) bank166430 +166431 POINT(41.46975136082636 -86.95501285840415) bank166431 +166432 POINT(42.0027853830023 -87.16208747063887) bank166432 +166433 POINT(42.37609480485519 -87.94047221613397) bank166433 +166434 POINT(41.77405783120089 -86.64978746713827) bank166434 +166435 POINT(41.599612502028066 -87.56968578108435) bank166435 +166436 POINT(42.12684052774564 -88.57927314804067) bank166436 +166437 POINT(42.291637012128994 -87.65063786502819) bank166437 +166438 POINT(41.53126334468083 -87.70281019941237) bank166438 +166439 POINT(41.212207118077245 -88.1346937207095) bank166439 +166440 POINT(41.636109215122126 -86.66988637729455) bank166440 +166441 POINT(41.24329834053864 -88.20531573758919) bank166441 +166442 POINT(41.66204029326855 -88.40870663103632) bank166442 +166443 POINT(41.53234814536598 -88.61541687784928) bank166443 +166444 POINT(41.704869552748235 -87.68191084684324) bank166444 +166445 POINT(41.03514804414867 -86.8607436454065) bank166445 +166446 POINT(42.28757156865163 -87.22615250296936) bank166446 +166447 POINT(41.97890645388462 -87.64588973892957) bank166447 +166448 POINT(41.292339305485875 -87.76245967880095) bank166448 +166449 POINT(40.99013353875861 -88.3108700190015) bank166449 +166450 POINT(41.707183438834804 -88.43289928206904) bank166450 +166451 POINT(41.31091983041665 -87.7625378450082) bank166451 +166452 POINT(42.55935497627905 -86.96253162657081) bank166452 +166453 POINT(41.206635549801206 -87.00465301161424) bank166453 +166454 POINT(42.78030091779716 -86.85441871240893) bank166454 +166455 POINT(42.75189304134287 -87.40335395518943) bank166455 +166456 POINT(41.723394406205365 -86.92200327646337) bank166456 +166457 POINT(41.449573556765436 -87.07924161712589) bank166457 +166458 POINT(41.0581115026233 -86.70163112425507) bank166458 +166459 POINT(40.89462975518864 -86.99985157267281) bank166459 +166460 POINT(41.77183661629865 -88.38176976215654) bank166460 +166461 POINT(41.07085369840877 -87.95795174583262) bank166461 +166462 POINT(42.17990743955194 -87.8049166606763) bank166462 +166463 POINT(42.353122377015744 -86.73903409327917) bank166463 +166464 POINT(41.03613697090764 -87.42754196588575) bank166464 +166465 POINT(41.91689010828935 -87.25311589574639) bank166465 +166466 POINT(41.40462774892769 -86.74501460998509) bank166466 +166467 POINT(42.77012105017965 -87.76708840241551) bank166467 +166468 POINT(42.60881130020336 -87.14382559519561) bank166468 +166469 POINT(42.36895973875249 -88.51570975938812) bank166469 +166470 POINT(41.93128024103588 -86.98069546674672) bank166470 +166471 POINT(41.850829146591636 -87.31197103999418) bank166471 +166472 POINT(41.26162879514158 -87.37439115190101) bank166472 +166473 POINT(41.462991965799354 -88.09225655279614) bank166473 +166474 POINT(42.09897094735912 -88.3227417686625) bank166474 +166475 POINT(41.94368498649305 -86.85370831104576) bank166475 +166476 POINT(42.829733393353806 -86.81238976616226) bank166476 +166477 POINT(41.07880456946987 -86.96842534385173) bank166477 +166478 POINT(42.16125458409974 -87.07211082539271) bank166478 +166479 POINT(41.59704756997464 -87.36657258940699) bank166479 +166480 POINT(42.18783851268688 -86.98718880844761) bank166480 +166481 POINT(42.678273257530144 -86.65276544655914) bank166481 +166482 POINT(42.77908665681197 -88.56693696576176) bank166482 +166483 POINT(42.33994596330621 -87.49181090009981) bank166483 +166484 POINT(41.98920360885528 -86.92170104828983) bank166484 +166485 POINT(42.36474682292104 -86.91911515806788) bank166485 +166486 POINT(42.8078025889177 -86.68498584770613) bank166486 +166487 POINT(42.639333826966045 -87.92370363625169) bank166487 +166488 POINT(42.77748052254649 -87.06456683763334) bank166488 +166489 POINT(41.15640074725731 -88.2113946097886) bank166489 +166490 POINT(41.83029723055549 -87.13484814305109) bank166490 +166491 POINT(41.75201787915304 -87.84570433406525) bank166491 +166492 POINT(42.608187237762074 -86.77438280422709) bank166492 +166493 POINT(42.85313208463077 -87.55620463657993) bank166493 +166494 POINT(42.55718272829148 -87.96901327538463) bank166494 +166495 POINT(42.74530266550057 -88.25398256394605) bank166495 +166496 POINT(42.01781999984773 -88.52470154952766) bank166496 +166497 POINT(41.03395123617761 -87.54140201925976) bank166497 +166498 POINT(41.02373732277128 -87.27086722810057) bank166498 +166499 POINT(41.88107618639755 -86.76809553855573) bank166499 +166500 POINT(41.35016582766114 -87.8960904593091) bank166500 +166501 POINT(42.388409746917105 -87.45450853990911) bank166501 +166502 POINT(42.10917625331927 -87.28344325386138) bank166502 +166503 POINT(42.17472729275107 -87.4857174003236) bank166503 +166504 POINT(42.526623748006855 -87.60058430543394) bank166504 +166505 POINT(42.70504439512335 -86.93685971288915) bank166505 +166506 POINT(42.669686148540656 -88.08404084740145) bank166506 +166507 POINT(40.92226711915149 -87.11397440669974) bank166507 +166508 POINT(41.81039396322562 -86.78879378136627) bank166508 +166509 POINT(42.09684710554804 -87.02405708344884) bank166509 +166510 POINT(41.43885175247949 -87.55950603138055) bank166510 +166511 POINT(41.86362323912423 -86.97331118166237) bank166511 +166512 POINT(41.589407801432706 -87.50979419751965) bank166512 +166513 POINT(42.79222102010411 -88.6164350170811) bank166513 +166514 POINT(41.665283118436356 -87.45541249548293) bank166514 +166515 POINT(41.25513992471783 -88.4268106536809) bank166515 +166516 POINT(41.59988171826373 -87.2431138989164) bank166516 +166517 POINT(42.02584878878069 -86.63517195160223) bank166517 +166518 POINT(42.111233364960974 -88.407243412081) bank166518 +166519 POINT(41.92646435317387 -87.22709342957457) bank166519 +166520 POINT(40.883163621362584 -87.57014640821905) bank166520 +166521 POINT(42.86898856202446 -87.49737206228762) bank166521 +166522 POINT(41.29664776468735 -87.01329750865223) bank166522 +166523 POINT(41.81233339492495 -87.7550162590492) bank166523 +166524 POINT(40.98711682760583 -87.10873263226905) bank166524 +166525 POINT(42.79563040973748 -88.01610318223624) bank166525 +166526 POINT(42.32736998639257 -88.3252563586684) bank166526 +166527 POINT(40.901935249109705 -87.90928406358668) bank166527 +166528 POINT(41.05540876528765 -86.6600757734634) bank166528 +166529 POINT(41.5965676234225 -87.17795822335061) bank166529 +166530 POINT(41.52174178372822 -88.32310903007918) bank166530 +166531 POINT(41.4156852571086 -87.22497370948057) bank166531 +166532 POINT(41.83404279708717 -87.01741507625975) bank166532 +166533 POINT(41.51198104598414 -86.76243468714716) bank166533 +166534 POINT(41.113310454649046 -86.83544350395225) bank166534 +166535 POINT(42.2707035631731 -87.60794512594269) bank166535 +166536 POINT(41.670911606771504 -88.48147923800946) bank166536 +166537 POINT(42.455861765664615 -88.4151594263137) bank166537 +166538 POINT(40.903249987820566 -87.47937627182657) bank166538 +166539 POINT(42.084745827253336 -88.08083875667708) bank166539 +166540 POINT(40.94629671871506 -87.05756695020378) bank166540 +166541 POINT(40.99291064274309 -87.4989269649672) bank166541 +166542 POINT(42.68970219872447 -87.50317792844638) bank166542 +166543 POINT(41.655272092571465 -88.4081832324665) bank166543 +166544 POINT(41.274845277048385 -88.36064600369562) bank166544 +166545 POINT(41.924616318459535 -88.24417361431784) bank166545 +166546 POINT(42.37206095130662 -87.39397117387395) bank166546 +166547 POINT(41.789800060151336 -87.07420988702277) bank166547 +166548 POINT(40.9549988320066 -88.41622504035972) bank166548 +166549 POINT(41.11225792499586 -87.46547324417023) bank166549 +166550 POINT(41.67536546590304 -88.4409855530214) bank166550 +166551 POINT(42.42499734043034 -86.70471240807683) bank166551 +166552 POINT(42.84344318520034 -88.47855145422285) bank166552 +166553 POINT(42.37992126734705 -87.08269691484239) bank166553 +166554 POINT(42.87073076573909 -86.98594682902785) bank166554 +166555 POINT(41.128589975756874 -87.4636470268336) bank166555 +166556 POINT(42.614832698404754 -88.5540433175199) bank166556 +166557 POINT(42.77350880184129 -87.67158070259235) bank166557 +166558 POINT(40.98903093412713 -87.00191482840789) bank166558 +166559 POINT(41.144009311191894 -87.11234515793551) bank166559 +166560 POINT(42.87576471046104 -87.73928649938904) bank166560 +166561 POINT(42.37934087527729 -88.2285584087789) bank166561 +166562 POINT(41.77265102470802 -87.60063324269447) bank166562 +166563 POINT(42.48264140983986 -87.78192988806353) bank166563 +166564 POINT(42.30797096545455 -87.75428994846037) bank166564 +166565 POINT(41.335148586333204 -88.09525592508403) bank166565 +166566 POINT(41.05841618128357 -88.26291372159841) bank166566 +166567 POINT(42.52944047104106 -86.92973975493781) bank166567 +166568 POINT(40.9916875926834 -87.51829142942233) bank166568 +166569 POINT(41.19139224234732 -86.6812594367709) bank166569 +166570 POINT(42.55104639984127 -86.66881665856413) bank166570 +166571 POINT(42.388082118208 -87.40162015420171) bank166571 +166572 POINT(40.900517952040545 -88.38787982834117) bank166572 +166573 POINT(42.59113634871673 -88.54142452936992) bank166573 +166574 POINT(41.512422661789294 -87.91026179510185) bank166574 +166575 POINT(41.616315674138455 -87.81542852556376) bank166575 +166576 POINT(42.35330299363064 -86.910338238266) bank166576 +166577 POINT(41.72201839018079 -88.28676380228994) bank166577 +166578 POINT(42.48740608236625 -87.39435007199768) bank166578 +166579 POINT(41.10989947848091 -86.8195899270371) bank166579 +166580 POINT(42.679007602912655 -86.91666131870217) bank166580 +166581 POINT(42.80660951693795 -87.84310850033954) bank166581 +166582 POINT(41.38389189117994 -87.35199742219443) bank166582 +166583 POINT(41.35177462635136 -88.45372465390824) bank166583 +166584 POINT(42.27162149316667 -86.64174667891491) bank166584 +166585 POINT(42.61532674035712 -86.73670351429975) bank166585 +166586 POINT(42.257420270830266 -86.66676439355462) bank166586 +166587 POINT(41.96383564673033 -88.01766871444423) bank166587 +166588 POINT(41.325528259377094 -88.58257638149891) bank166588 +166589 POINT(41.09948792371916 -87.23327565016223) bank166589 +166590 POINT(41.26148888794652 -87.41060491655192) bank166590 +166591 POINT(42.74517276384166 -87.03181948299296) bank166591 +166592 POINT(41.05001401975786 -87.76879825378933) bank166592 +166593 POINT(40.90041179476226 -87.99922950618914) bank166593 +166594 POINT(41.58628399385821 -86.98091470687449) bank166594 +166595 POINT(42.569521008096245 -88.2010896439092) bank166595 +166596 POINT(42.252812348791075 -88.1903565487479) bank166596 +166597 POINT(41.03299470534487 -88.43461635398091) bank166597 +166598 POINT(41.628219100352084 -87.34201780126396) bank166598 +166599 POINT(41.966788501274316 -88.10263056017071) bank166599 +166600 POINT(42.21263328080121 -88.12340628990464) bank166600 +166601 POINT(42.676647738011106 -88.17791485737962) bank166601 +166602 POINT(42.25812203961573 -88.41991121090123) bank166602 +166603 POINT(41.54221458537299 -87.55412714920062) bank166603 +166604 POINT(41.92551938844652 -88.43034049926585) bank166604 +166605 POINT(42.63277114506745 -87.94652607880501) bank166605 +166606 POINT(41.67066472827581 -88.13951975777918) bank166606 +166607 POINT(42.706706114155324 -88.50604282898544) bank166607 +166608 POINT(42.437379169014086 -87.6297650995982) bank166608 +166609 POINT(41.5225873619423 -87.94964550623236) bank166609 +166610 POINT(41.636467015259925 -87.44088407119871) bank166610 +166611 POINT(40.88882269755901 -87.12270880711134) bank166611 +166612 POINT(42.49139899043933 -87.30915913807564) bank166612 +166613 POINT(42.642077639659625 -87.74005710044884) bank166613 +166614 POINT(41.19679803697044 -86.84089640766614) bank166614 +166615 POINT(42.505367260643126 -86.89682800729855) bank166615 +166616 POINT(41.46991783137953 -88.30469877132563) bank166616 +166617 POINT(42.84238255599565 -88.42559445621293) bank166617 +166618 POINT(42.41143762259488 -86.66498291951378) bank166618 +166619 POINT(42.56900939861463 -87.37610289354325) bank166619 +166620 POINT(41.85167307144708 -86.95218121236528) bank166620 +166621 POINT(41.95999593228001 -86.94133511046779) bank166621 +166622 POINT(42.761460991315886 -88.46000684333121) bank166622 +166623 POINT(42.01133535029363 -87.14202153509702) bank166623 +166624 POINT(41.789619557372724 -88.38557653771755) bank166624 +166625 POINT(42.54943471057198 -88.49363555796195) bank166625 +166626 POINT(41.54476457810418 -87.50257222973326) bank166626 +166627 POINT(41.00688649221236 -87.94187550880537) bank166627 +166628 POINT(42.569503655001846 -87.52815027221418) bank166628 +166629 POINT(42.39573104543151 -87.27869142963117) bank166629 +166630 POINT(41.44805661352636 -88.56756311511624) bank166630 +166631 POINT(41.258966984726634 -87.57218141692657) bank166631 +166632 POINT(42.849173772092 -87.91443640868276) bank166632 +166633 POINT(41.760907492063225 -88.17535204029566) bank166633 +166634 POINT(41.25447599938957 -87.51395662260524) bank166634 +166635 POINT(42.79659833162547 -88.2694349576794) bank166635 +166636 POINT(41.59778945743927 -87.10123688380492) bank166636 +166637 POINT(41.3243992939894 -86.89645680186968) bank166637 +166638 POINT(41.14527718803021 -87.34464122100087) bank166638 +166639 POINT(41.27645575361146 -87.61946985375779) bank166639 +166640 POINT(42.3081718355653 -87.9419926603194) bank166640 +166641 POINT(42.74580338152645 -88.1069537237523) bank166641 +166642 POINT(41.436367018641 -86.77192148731724) bank166642 +166643 POINT(42.47275743539548 -86.75537273471133) bank166643 +166644 POINT(41.96234382447943 -86.65441455181212) bank166644 +166645 POINT(40.95845818906382 -87.98885638611554) bank166645 +166646 POINT(42.03399296700866 -86.98822449989085) bank166646 +166647 POINT(42.514739793214034 -87.34106552112615) bank166647 +166648 POINT(41.47067170818833 -86.96821822624246) bank166648 +166649 POINT(41.84764182421034 -88.07375962459987) bank166649 +166650 POINT(42.86634366628164 -87.76064269515388) bank166650 +166651 POINT(42.11454144095065 -88.29221607214632) bank166651 +166652 POINT(41.094446750833775 -87.65147277143191) bank166652 +166653 POINT(41.130960105802664 -88.41570936120792) bank166653 +166654 POINT(41.25443750141989 -86.67047241417896) bank166654 +166655 POINT(41.89933504077721 -88.10171497383537) bank166655 +166656 POINT(41.007980157354076 -88.35044009693831) bank166656 +166657 POINT(41.492128026489766 -88.37023664200225) bank166657 +166658 POINT(41.9121447600073 -87.86638860542311) bank166658 +166659 POINT(42.84382201719828 -88.39995135117327) bank166659 +166660 POINT(42.046184817838565 -88.27020785663241) bank166660 +166661 POINT(42.76955781243889 -87.26639923258962) bank166661 +166662 POINT(41.58591614071911 -87.59781257073249) bank166662 +166663 POINT(40.983365883170414 -87.90235277027865) bank166663 +166664 POINT(41.52300855513989 -87.78807546860826) bank166664 +166665 POINT(41.74867945312897 -86.90180422082948) bank166665 +166666 POINT(42.55370619161323 -86.85755345840761) bank166666 +166667 POINT(42.727652938982374 -88.20928388951141) bank166667 +166668 POINT(41.77219157269199 -87.39548208375398) bank166668 +166669 POINT(40.87961448237278 -87.57796595071203) bank166669 +166670 POINT(38.610365413121585 -122.16166530956896) bank166670 +166671 POINT(38.378673697376726 -121.89292712053631) bank166671 +166672 POINT(37.84093442790482 -122.28961055026474) bank166672 +166673 POINT(37.220809675971424 -123.40735236121635) bank166673 +166674 POINT(37.283273929702865 -123.14546057069862) bank166674 +166675 POINT(37.77354696535916 -123.13535465882418) bank166675 +166676 POINT(37.38815863465644 -122.90691465225615) bank166676 +166677 POINT(38.15548585870347 -122.41892843053024) bank166677 +166678 POINT(37.52296951140693 -122.86881590160318) bank166678 +166679 POINT(38.43489089675653 -122.45007185454776) bank166679 +166680 POINT(38.26100215876485 -122.14178390406052) bank166680 +166681 POINT(37.209682664574814 -121.6164113852042) bank166681 +166682 POINT(37.506230526468215 -122.063429600136) bank166682 +166683 POINT(38.21763676317769 -121.84533369091764) bank166683 +166684 POINT(37.02599056221519 -123.35638944715859) bank166684 +166685 POINT(37.27772145534298 -121.9873351251711) bank166685 +166686 POINT(37.01865733621612 -121.65749585136528) bank166686 +166687 POINT(38.61032532360201 -122.73437042681046) bank166687 +166688 POINT(38.06349687454141 -122.58326561741931) bank166688 +166689 POINT(36.966588995850906 -123.10310486193322) bank166689 +166690 POINT(38.486620848086055 -122.5891142668931) bank166690 +166691 POINT(38.05725701891536 -123.11619864971047) bank166691 +166692 POINT(38.15126267794703 -122.35652464063357) bank166692 +166693 POINT(38.53710724513905 -122.14961628783026) bank166693 +166694 POINT(37.37648334527929 -122.19042441091203) bank166694 +166695 POINT(37.76343444505596 -122.83547271200412) bank166695 +166696 POINT(38.54744734073406 -123.24684475188278) bank166696 +166697 POINT(38.58199208168375 -122.00701435141701) bank166697 +166698 POINT(37.28984857767061 -122.90121709459493) bank166698 +166699 POINT(36.95772614727113 -121.72898983345722) bank166699 +166700 POINT(36.79442776135847 -122.32364465110405) bank166700 +166701 POINT(37.715039254968076 -123.34974146716706) bank166701 +166702 POINT(38.36762650071815 -122.56942361348422) bank166702 +166703 POINT(37.16255357127662 -122.21314922256316) bank166703 +166704 POINT(38.37375558006054 -122.20312591001634) bank166704 +166705 POINT(36.94528643572339 -122.4858888994187) bank166705 +166706 POINT(36.830277003138555 -122.36621055468923) bank166706 +166707 POINT(38.241731538041755 -122.61766963429659) bank166707 +166708 POINT(37.90252178047476 -122.05694928441) bank166708 +166709 POINT(37.05704090185764 -122.90790137568585) bank166709 +166710 POINT(38.10836301961688 -122.64969476620261) bank166710 +166711 POINT(37.26346990747112 -122.7225136440192) bank166711 +166712 POINT(37.92106001722336 -122.54557734514964) bank166712 +166713 POINT(38.597320085709995 -122.52737650218464) bank166713 +166714 POINT(38.206916935596325 -123.40545141700954) bank166714 +166715 POINT(38.22135230139398 -121.75000065905739) bank166715 +166716 POINT(36.825638789460285 -122.18767137075433) bank166716 +166717 POINT(37.74318614652601 -122.8010340889501) bank166717 +166718 POINT(37.871833570007404 -122.9855143387299) bank166718 +166719 POINT(37.79274375807938 -122.51413693786853) bank166719 +166720 POINT(36.97746935764485 -122.51136556648736) bank166720 +166721 POINT(37.87725631041088 -123.25217952458152) bank166721 +166722 POINT(38.509151524813106 -123.38455573580987) bank166722 +166723 POINT(37.93443495604386 -122.88519842827371) bank166723 +166724 POINT(37.15385213351535 -122.50524150658907) bank166724 +166725 POINT(37.5588866611971 -122.00573781771661) bank166725 +166726 POINT(37.9759933603798 -121.7151839384119) bank166726 +166727 POINT(37.28061727362813 -122.70948342924028) bank166727 +166728 POINT(38.2145310030973 -122.41204394121824) bank166728 +166729 POINT(37.594606694891624 -122.12176566256676) bank166729 +166730 POINT(38.74895164095785 -121.78994826151313) bank166730 +166731 POINT(37.64368697801346 -121.93079811862866) bank166731 +166732 POINT(37.567405698564215 -121.69382306970354) bank166732 +166733 POINT(37.65551652162187 -121.87974182704075) bank166733 +166734 POINT(37.85375796414714 -122.5636983611609) bank166734 +166735 POINT(38.64660783362203 -121.77354658241494) bank166735 +166736 POINT(37.88351096863962 -122.8463386525328) bank166736 +166737 POINT(36.96452611699823 -122.97222099914674) bank166737 +166738 POINT(38.09744080431079 -121.6260812170537) bank166738 +166739 POINT(38.11882508099372 -121.43574859634745) bank166739 +166740 POINT(37.09876524007042 -122.6691502600948) bank166740 +166741 POINT(37.17267275687042 -123.19708398539458) bank166741 +166742 POINT(36.897606225519844 -122.21076184422445) bank166742 +166743 POINT(38.57420103388435 -123.33866711494336) bank166743 +166744 POINT(36.90594633677314 -122.42969562109307) bank166744 +166745 POINT(36.858634344676595 -123.21629225109709) bank166745 +166746 POINT(37.120932727654555 -122.80799743614799) bank166746 +166747 POINT(37.42678452855701 -123.0946340400085) bank166747 +166748 POINT(37.612545008557184 -121.6920993499375) bank166748 +166749 POINT(37.28860090321938 -121.98773618559481) bank166749 +166750 POINT(36.97059276812037 -122.70486719697618) bank166750 +166751 POINT(37.396105518147515 -122.65865773905826) bank166751 +166752 POINT(37.22028534397228 -122.20210534052313) bank166752 +166753 POINT(38.40590777573986 -122.74571274064414) bank166753 +166754 POINT(38.19256636239194 -123.03067317228546) bank166754 +166755 POINT(37.79363284604168 -122.8307488313358) bank166755 +166756 POINT(37.10375886523886 -122.91605814102095) bank166756 +166757 POINT(37.39189945387279 -122.58192713149849) bank166757 +166758 POINT(37.18198552203335 -122.36799698951313) bank166758 +166759 POINT(37.08123109619218 -122.08752374494362) bank166759 +166760 POINT(37.59009599711419 -122.80971639735961) bank166760 +166761 POINT(37.852612870858735 -121.89050671663904) bank166761 +166762 POINT(37.32285896878751 -122.49379162950618) bank166762 +166763 POINT(37.77959960709141 -122.39982887777613) bank166763 +166764 POINT(36.856957695763015 -121.67759523671768) bank166764 +166765 POINT(37.76511981269647 -123.11004035451778) bank166765 +166766 POINT(38.47422809967494 -122.746262265256) bank166766 +166767 POINT(37.21280590491724 -121.72597730988434) bank166767 +166768 POINT(38.05010414554442 -122.10352012879218) bank166768 +166769 POINT(36.89715137655079 -122.38144644357965) bank166769 +166770 POINT(38.62293656919894 -123.3501630500723) bank166770 +166771 POINT(38.309750272429035 -121.51396193256883) bank166771 +166772 POINT(37.59852220348915 -122.4391002582091) bank166772 +166773 POINT(37.28169675613988 -123.13333259234822) bank166773 +166774 POINT(37.53582421207844 -122.01887991068548) bank166774 +166775 POINT(38.156790430643944 -123.40362212541476) bank166775 +166776 POINT(37.561861914877355 -122.85597561090731) bank166776 +166777 POINT(38.496635314555775 -122.05730329476258) bank166777 +166778 POINT(37.19863786656371 -122.18549176919045) bank166778 +166779 POINT(37.041008807104824 -121.9707934166014) bank166779 +166780 POINT(37.068199318273834 -122.86979954112) bank166780 +166781 POINT(37.17890213831668 -123.39608757829052) bank166781 +166782 POINT(37.71721189452985 -122.96935293283192) bank166782 +166783 POINT(37.228235869325026 -123.2352736404745) bank166783 +166784 POINT(38.198207774565255 -123.26945214655994) bank166784 +166785 POINT(37.86368094148688 -123.29422563994326) bank166785 +166786 POINT(38.455681944880865 -121.6612089136068) bank166786 +166787 POINT(38.22614215613056 -121.49316541556158) bank166787 +166788 POINT(38.58382860681892 -122.39486714734386) bank166788 +166789 POINT(36.85122408630873 -122.40130411529087) bank166789 +166790 POINT(36.7969872997347 -121.61675650288731) bank166790 +166791 POINT(38.72436896373482 -123.11462632100717) bank166791 +166792 POINT(38.006585993919956 -123.12310840579119) bank166792 +166793 POINT(37.61687220626844 -123.0269819789619) bank166793 +166794 POINT(38.28294292955715 -122.25929232682452) bank166794 +166795 POINT(37.47602870818803 -122.22704134597673) bank166795 +166796 POINT(38.734763374529884 -121.5642193116905) bank166796 +166797 POINT(38.6154503702765 -123.3476686628206) bank166797 +166798 POINT(37.998668213181936 -121.49636421164847) bank166798 +166799 POINT(37.29082612553628 -122.79514690723019) bank166799 +166800 POINT(38.00392326014822 -122.63113572113593) bank166800 +166801 POINT(38.45512616629391 -123.14178966904997) bank166801 +166802 POINT(38.38108245158535 -123.3976802539541) bank166802 +166803 POINT(37.26254182232473 -122.75590784387464) bank166803 +166804 POINT(38.16945098880962 -123.1589912238816) bank166804 +166805 POINT(37.63582067110201 -122.59424860750552) bank166805 +166806 POINT(37.216477037306305 -122.98246463253854) bank166806 +166807 POINT(38.18006655046245 -122.59097845522838) bank166807 +166808 POINT(38.67094703578427 -122.534765915154) bank166808 +166809 POINT(37.76869143485592 -122.12800697507593) bank166809 +166810 POINT(37.32480268572413 -122.9528753349609) bank166810 +166811 POINT(38.127904987099924 -122.81387935810733) bank166811 +166812 POINT(38.26343793772012 -121.4705470375815) bank166812 +166813 POINT(37.176321317565034 -121.84483137737547) bank166813 +166814 POINT(38.14194638695113 -123.0773125694512) bank166814 +166815 POINT(36.866782224748334 -122.16738263500409) bank166815 +166816 POINT(38.5794340698994 -123.32865198650052) bank166816 +166817 POINT(38.571588211453424 -121.96753403257084) bank166817 +166818 POINT(37.57247534938943 -122.40068975066295) bank166818 +166819 POINT(38.12205314433219 -122.42633466191607) bank166819 +166820 POINT(38.61705332182296 -122.72751632631443) bank166820 +166821 POINT(37.545297831940246 -122.13843973270319) bank166821 +166822 POINT(38.76453313241961 -122.98565050855561) bank166822 +166823 POINT(38.037421237813035 -122.07643766369164) bank166823 +166824 POINT(38.1426709818542 -122.54867102863456) bank166824 +166825 POINT(38.08099994735114 -121.79877669919115) bank166825 +166826 POINT(36.977602675316824 -122.64089411226448) bank166826 +166827 POINT(36.981715181643246 -121.42456211861303) bank166827 +166828 POINT(37.17323719194285 -122.66401368354265) bank166828 +166829 POINT(37.79516703963429 -122.30191061393266) bank166829 +166830 POINT(38.12646838579678 -122.10428754323401) bank166830 +166831 POINT(37.37425399435036 -123.15907432878411) bank166831 +166832 POINT(38.24640431938434 -122.63937546040547) bank166832 +166833 POINT(37.70844678012239 -123.2656545027763) bank166833 +166834 POINT(37.39449092823359 -122.5967815902088) bank166834 +166835 POINT(38.49277971482147 -122.43315573928268) bank166835 +166836 POINT(37.7120187062537 -122.07931725837385) bank166836 +166837 POINT(37.412797024319715 -122.6499081432909) bank166837 +166838 POINT(37.11483921545026 -123.13133229359572) bank166838 +166839 POINT(36.834406869583816 -122.83254389319495) bank166839 +166840 POINT(36.79860776646764 -121.67995724394862) bank166840 +166841 POINT(37.465754241208 -122.83111120051373) bank166841 +166842 POINT(37.87315051177613 -122.9398367663466) bank166842 +166843 POINT(37.20767927287325 -122.54934848380653) bank166843 +166844 POINT(37.67818437775559 -122.23133936582823) bank166844 +166845 POINT(38.39478573761431 -123.38955043141037) bank166845 +166846 POINT(36.894338865330035 -122.39785108967072) bank166846 +166847 POINT(37.96235768198103 -122.13641372040509) bank166847 +166848 POINT(36.923884657114215 -122.28550803853128) bank166848 +166849 POINT(38.39862908324513 -121.98395517245201) bank166849 +166850 POINT(38.68327707772335 -121.49799655658276) bank166850 +166851 POINT(37.38515840715114 -121.7614740691392) bank166851 +166852 POINT(37.62128000259165 -123.3089486763232) bank166852 +166853 POINT(37.590676839628195 -122.27110799514737) bank166853 +166854 POINT(36.900934523143114 -122.3418695631313) bank166854 +166855 POINT(36.94588706821441 -122.58039658672058) bank166855 +166856 POINT(37.86308132038737 -122.35841295065734) bank166856 +166857 POINT(37.0707586767428 -122.59275008247127) bank166857 +166858 POINT(37.94827687390842 -122.37549778289365) bank166858 +166859 POINT(37.52077412911306 -123.21395934531145) bank166859 +166860 POINT(36.99589056146481 -122.42802583275738) bank166860 +166861 POINT(38.7432895066207 -122.08541005460887) bank166861 +166862 POINT(37.73748885120392 -122.23377826952992) bank166862 +166863 POINT(38.63864130425365 -123.2137121511318) bank166863 +166864 POINT(38.63992679235826 -122.76230944254281) bank166864 +166865 POINT(37.757155735215036 -122.03809400704176) bank166865 +166866 POINT(37.084646337987934 -122.66182437479334) bank166866 +166867 POINT(37.5164489912974 -122.83125876413573) bank166867 +166868 POINT(37.134947050163305 -123.13334806216416) bank166868 +166869 POINT(36.83221301650798 -122.2293877865412) bank166869 +166870 POINT(37.08650604644935 -121.73151265115676) bank166870 +166871 POINT(37.80425903647663 -122.12607120909811) bank166871 +166872 POINT(37.17537996572636 -122.56954735055234) bank166872 +166873 POINT(37.1120046213783 -122.42580266362845) bank166873 +166874 POINT(37.77440319214989 -123.37934169741547) bank166874 +166875 POINT(38.33553357586365 -122.12190921269949) bank166875 +166876 POINT(38.43841590848227 -121.98440819226884) bank166876 +166877 POINT(37.17149405254816 -121.47955726692547) bank166877 +166878 POINT(37.32729141882638 -121.78654182445413) bank166878 +166879 POINT(38.514450145337584 -121.49778300841048) bank166879 +166880 POINT(37.20667403282439 -122.62037531796302) bank166880 +166881 POINT(38.17671598923498 -122.81392291124449) bank166881 +166882 POINT(37.49850309422594 -122.81471030755738) bank166882 +166883 POINT(37.90834143014589 -122.21138944628846) bank166883 +166884 POINT(37.75926864434457 -123.18560801372234) bank166884 +166885 POINT(37.4312166764747 -122.71236420958968) bank166885 +166886 POINT(37.90132357529317 -122.49477758470383) bank166886 +166887 POINT(37.955028399409144 -122.18779854379866) bank166887 +166888 POINT(38.73250967291472 -122.15032921665207) bank166888 +166889 POINT(38.628599542763375 -123.05964869311886) bank166889 +166890 POINT(38.25047059999785 -122.11788913149893) bank166890 +166891 POINT(38.288361897827784 -122.46577570578148) bank166891 +166892 POINT(37.0858521359172 -122.11412105462114) bank166892 +166893 POINT(37.16013203324163 -122.08455786410012) bank166893 +166894 POINT(38.027018358019646 -122.00019605617916) bank166894 +166895 POINT(36.936852674791986 -122.169975218058) bank166895 +166896 POINT(38.21732703681739 -121.95522750451727) bank166896 +166897 POINT(38.74945262459776 -122.74881600682467) bank166897 +166898 POINT(38.30259421230052 -122.27668527366227) bank166898 +166899 POINT(38.233793206483256 -121.50457422042093) bank166899 +166900 POINT(37.727651315072855 -122.32441069877497) bank166900 +166901 POINT(37.767415034769805 -123.23908072604162) bank166901 +166902 POINT(38.33353467858957 -122.35028460777124) bank166902 +166903 POINT(36.9802265353579 -122.80514204088784) bank166903 +166904 POINT(37.08659530608023 -121.97203690994051) bank166904 +166905 POINT(36.96334244516828 -121.62954066918701) bank166905 +166906 POINT(38.44813095260967 -122.54888225537806) bank166906 +166907 POINT(38.748161500804365 -122.09994288059198) bank166907 +166908 POINT(37.063114262693944 -123.31333129696493) bank166908 +166909 POINT(38.53702319012997 -122.19275395801687) bank166909 +166910 POINT(38.36000757028644 -122.21180024638673) bank166910 +166911 POINT(37.076593171012036 -122.9382244628195) bank166911 +166912 POINT(38.697160315363064 -121.5668131836676) bank166912 +166913 POINT(37.11337930710318 -122.35293117437901) bank166913 +166914 POINT(37.42742523796783 -122.29081958577767) bank166914 +166915 POINT(37.75926861095586 -123.30809595344847) bank166915 +166916 POINT(38.49456175207452 -122.33002197550455) bank166916 +166917 POINT(38.673398805844556 -122.01445861333818) bank166917 +166918 POINT(37.38129624065766 -121.55698311350359) bank166918 +166919 POINT(38.70882847742671 -122.4117952879669) bank166919 +166920 POINT(37.4923422231796 -122.73891365834585) bank166920 +166921 POINT(37.19763479479482 -123.2803126396282) bank166921 +166922 POINT(38.64327363816527 -121.90363375628263) bank166922 +166923 POINT(37.33257674319316 -122.13588435352162) bank166923 +166924 POINT(37.776777223694545 -123.16215364101747) bank166924 +166925 POINT(36.919483219747505 -122.15454724456653) bank166925 +166926 POINT(37.59558884115113 -122.4274089115858) bank166926 +166927 POINT(37.829576631264864 -121.46272560740192) bank166927 +166928 POINT(38.69490666698061 -123.37821734629273) bank166928 +166929 POINT(37.43446605080034 -123.30076910309366) bank166929 +166930 POINT(37.95818108539414 -122.58142924853297) bank166930 +166931 POINT(36.805952878231516 -123.24218617847635) bank166931 +166932 POINT(37.22469655926389 -122.79921112689217) bank166932 +166933 POINT(38.23357485832275 -121.78850777560213) bank166933 +166934 POINT(37.39021946841682 -122.97786115059785) bank166934 +166935 POINT(37.176277881406115 -121.76867516292218) bank166935 +166936 POINT(38.04074230388261 -122.13342588014619) bank166936 +166937 POINT(36.9267767791177 -121.71946261364569) bank166937 +166938 POINT(37.14887527110256 -123.39959555684399) bank166938 +166939 POINT(38.762157020750834 -123.0058774166006) bank166939 +166940 POINT(37.29951122092703 -123.15137562029999) bank166940 +166941 POINT(36.81554787111676 -122.85269842362489) bank166941 +166942 POINT(37.893423358420755 -123.0441661896962) bank166942 +166943 POINT(37.04418298183674 -122.91707007566986) bank166943 +166944 POINT(37.592242447199325 -122.39998407157725) bank166944 +166945 POINT(38.01199524986454 -122.14313031543671) bank166945 +166946 POINT(37.05868886846117 -123.10227720395157) bank166946 +166947 POINT(37.94341178950695 -122.07326155128601) bank166947 +166948 POINT(38.07327071958546 -122.77156514442588) bank166948 +166949 POINT(37.385940061377944 -122.06991249331291) bank166949 +166950 POINT(38.41969404203838 -122.30370326082028) bank166950 +166951 POINT(38.6032486377585 -122.79854768945623) bank166951 +166952 POINT(37.42125113733056 -121.75918301445641) bank166952 +166953 POINT(37.27692866759855 -122.33739226433744) bank166953 +166954 POINT(36.86442665361933 -122.3564911922862) bank166954 +166955 POINT(38.337931884470784 -123.29842983987417) bank166955 +166956 POINT(37.12340345738477 -122.16353366950067) bank166956 +166957 POINT(37.98067219925618 -122.85099145105193) bank166957 +166958 POINT(37.350824080892814 -121.48763259936885) bank166958 +166959 POINT(37.7687142889215 -122.57423592299415) bank166959 +166960 POINT(37.96313771745084 -122.50433690839344) bank166960 +166961 POINT(37.03443056736281 -122.14852605235731) bank166961 +166962 POINT(37.38340839714983 -122.42277911795075) bank166962 +166963 POINT(37.70791125334015 -123.10020645067621) bank166963 +166964 POINT(37.135992224378576 -122.47675563070062) bank166964 +166965 POINT(38.44831325806445 -123.35642686343712) bank166965 +166966 POINT(38.43934223438317 -123.36010849781775) bank166966 +166967 POINT(38.74856887423998 -122.50524377108874) bank166967 +166968 POINT(38.30434251733683 -122.76968905481648) bank166968 +166969 POINT(37.888969524388465 -122.37649376010455) bank166969 +166970 POINT(37.73419322718543 -122.3868988232236) bank166970 +166971 POINT(37.82649705881041 -122.91693683610951) bank166971 +166972 POINT(38.15718749749341 -122.2719861370974) bank166972 +166973 POINT(37.5160289144181 -122.90813745721853) bank166973 +166974 POINT(37.101500836365126 -122.1397993226777) bank166974 +166975 POINT(37.042086757570914 -122.9014067295666) bank166975 +166976 POINT(38.422294970012985 -122.6398299597087) bank166976 +166977 POINT(36.97369385530862 -123.33963463276581) bank166977 +166978 POINT(38.12209203445585 -123.32353503994415) bank166978 +166979 POINT(37.015258316178034 -122.50858281358546) bank166979 +166980 POINT(38.476554791907375 -123.26912689333614) bank166980 +166981 POINT(37.06740874493487 -121.53297646265577) bank166981 +166982 POINT(38.22271954741568 -122.62684287127053) bank166982 +166983 POINT(37.8131928201034 -123.31317324327503) bank166983 +166984 POINT(37.6236696887241 -122.04808052574158) bank166984 +166985 POINT(38.08796124624598 -122.22127947636602) bank166985 +166986 POINT(38.60764920053751 -123.16212089357465) bank166986 +166987 POINT(37.70085690539027 -121.67129604731463) bank166987 +166988 POINT(37.389659825042074 -122.42518348258844) bank166988 +166989 POINT(37.51914804264444 -121.58490626532294) bank166989 +166990 POINT(38.47852852994586 -122.37535959280581) bank166990 +166991 POINT(37.722955227723304 -122.64296433374085) bank166991 +166992 POINT(38.718354695773236 -123.38397532780654) bank166992 +166993 POINT(38.50985000739173 -121.5234909447691) bank166993 +166994 POINT(36.942852463272516 -121.81543346120625) bank166994 +166995 POINT(37.06661969708557 -123.18383692412552) bank166995 +166996 POINT(36.89125032360287 -123.05609799693613) bank166996 +166997 POINT(37.22889759392858 -122.5228898508536) bank166997 +166998 POINT(37.77925103870644 -122.62072046387483) bank166998 +166999 POINT(38.644355913004766 -123.11086180499984) bank166999 +167000 POINT(38.006845559887736 -122.19720495924479) bank167000 +167001 POINT(38.271507478649085 -122.48031327771304) bank167001 +167002 POINT(37.68355805399212 -121.54103228678983) bank167002 +167003 POINT(38.099062574006645 -121.49396924945485) bank167003 +167004 POINT(38.121889835241056 -121.60225117430484) bank167004 +167005 POINT(38.3699591633891 -122.15262716186848) bank167005 +167006 POINT(38.30473503926926 -121.68075257208359) bank167006 +167007 POINT(37.94980402571956 -121.80211280602477) bank167007 +167008 POINT(37.61753979187949 -122.90267984539472) bank167008 +167009 POINT(38.30134285094521 -121.64153709104643) bank167009 +167010 POINT(38.71120025662067 -122.77953327472532) bank167010 +167011 POINT(37.37068562547898 -123.23115441450561) bank167011 +167012 POINT(38.54095496883775 -121.8431574490771) bank167012 +167013 POINT(38.528484612559254 -121.65333620867554) bank167013 +167014 POINT(37.17935475861783 -122.24743381336529) bank167014 +167015 POINT(38.00448701809018 -121.89477626745094) bank167015 +167016 POINT(37.39787166198231 -123.14897969724701) bank167016 +167017 POINT(38.16660046448021 -123.08087511919969) bank167017 +167018 POINT(38.29144264568397 -122.25670057031357) bank167018 +167019 POINT(37.393809373431644 -122.6901933463291) bank167019 +167020 POINT(37.92768453044735 -122.31777504831331) bank167020 +167021 POINT(38.20822910308119 -122.3401165093651) bank167021 +167022 POINT(37.12475550459777 -122.98915058661848) bank167022 +167023 POINT(37.7205859887748 -121.70921489561559) bank167023 +167024 POINT(38.28244986131471 -121.99219142584268) bank167024 +167025 POINT(37.422058576753514 -123.38856157928014) bank167025 +167026 POINT(38.02879438507294 -121.53757526548027) bank167026 +167027 POINT(37.52832098437896 -121.76594301617473) bank167027 +167028 POINT(37.53922423587375 -122.10472447297505) bank167028 +167029 POINT(37.26308385176694 -122.65633477419487) bank167029 +167030 POINT(37.34152737112759 -121.93829412599086) bank167030 +167031 POINT(37.730835426699635 -122.04301446512287) bank167031 +167032 POINT(37.707559104519746 -121.80833308555204) bank167032 +167033 POINT(38.128740472547406 -122.0277242815242) bank167033 +167034 POINT(37.136505508615144 -121.71314160544017) bank167034 +167035 POINT(37.64297203957386 -121.83285430753084) bank167035 +167036 POINT(37.0636649550635 -122.97688747530228) bank167036 +167037 POINT(37.10513724651437 -122.13178403769753) bank167037 +167038 POINT(38.390873210439516 -121.84148588548193) bank167038 +167039 POINT(38.09595413573605 -122.21338185403997) bank167039 +167040 POINT(37.158721911632455 -122.87418522013867) bank167040 +167041 POINT(37.26918577537232 -123.08696998225163) bank167041 +167042 POINT(38.61082166918683 -122.1591365551273) bank167042 +167043 POINT(37.78576031571462 -122.44285939654247) bank167043 +167044 POINT(38.222332979459516 -121.4223939773403) bank167044 +167045 POINT(37.84794802346495 -122.15069845715333) bank167045 +167046 POINT(37.755276421345215 -122.67603921502084) bank167046 +167047 POINT(38.37422092116448 -122.68759208465832) bank167047 +167048 POINT(38.404500037806834 -122.48138931857686) bank167048 +167049 POINT(37.615737084227234 -122.62878412555742) bank167049 +167050 POINT(37.50629894524133 -123.30941872594755) bank167050 +167051 POINT(37.1121258316904 -122.65131407762858) bank167051 +167052 POINT(37.206476913411095 -122.16189592183113) bank167052 +167053 POINT(38.275930855496725 -122.40653990026807) bank167053 +167054 POINT(37.849714264705455 -122.97904888088655) bank167054 +167055 POINT(37.06677589719413 -122.24560795507549) bank167055 +167056 POINT(38.091343245310156 -122.69431802509298) bank167056 +167057 POINT(37.45371232249027 -121.60718999758788) bank167057 +167058 POINT(37.27018546720623 -122.92645001822132) bank167058 +167059 POINT(37.69105566641086 -123.12032413326166) bank167059 +167060 POINT(37.70193947439132 -123.29813823913466) bank167060 +167061 POINT(37.70370546006902 -122.44651005235588) bank167061 +167062 POINT(37.90496057296016 -122.39164046834175) bank167062 +167063 POINT(38.61286987525168 -121.81274072241807) bank167063 +167064 POINT(37.055709079215525 -121.70454949072492) bank167064 +167065 POINT(37.05717433547475 -123.25601659761652) bank167065 +167066 POINT(37.58009758381625 -123.18808334748917) bank167066 +167067 POINT(37.62604870904405 -122.53571590051851) bank167067 +167068 POINT(37.21339050079374 -122.38032710133201) bank167068 +167069 POINT(38.72421982599889 -121.67399317512735) bank167069 +167070 POINT(38.42265018486176 -121.521221097364) bank167070 +167071 POINT(38.42648418431863 -122.408932336063) bank167071 +167072 POINT(38.13520557856954 -121.49838816365802) bank167072 +167073 POINT(38.43084859305832 -123.3189687837772) bank167073 +167074 POINT(38.22322608266373 -123.02793612296374) bank167074 +167075 POINT(37.78511237086841 -122.20901618382611) bank167075 +167076 POINT(38.27247379509757 -121.58435794844377) bank167076 +167077 POINT(37.04147971656882 -122.34760359909424) bank167077 +167078 POINT(37.80208805347185 -121.54579064768993) bank167078 +167079 POINT(37.92800435918414 -123.40894346574092) bank167079 +167080 POINT(38.435203289270156 -122.29278241318622) bank167080 +167081 POINT(38.20656384517734 -122.18652759440195) bank167081 +167082 POINT(37.720710028674084 -121.46073822985929) bank167082 +167083 POINT(37.08567875051649 -121.80905926206508) bank167083 +167084 POINT(38.557771990454206 -121.77533273021798) bank167084 +167085 POINT(37.14100731480994 -122.18775239125334) bank167085 +167086 POINT(37.5238855189668 -121.82926536893179) bank167086 +167087 POINT(37.452337774102084 -122.61847936566234) bank167087 +167088 POINT(36.88133358487338 -122.29604193565096) bank167088 +167089 POINT(37.66506771014725 -121.8182788057254) bank167089 +167090 POINT(38.75999812288041 -122.268198638517) bank167090 +167091 POINT(38.647145797444225 -122.42096790800554) bank167091 +167092 POINT(37.49490131798453 -123.2492386849506) bank167092 +167093 POINT(38.49104467917958 -122.05329807622977) bank167093 +167094 POINT(36.897662044886395 -121.95753410179039) bank167094 +167095 POINT(38.46269859084419 -121.58109603616568) bank167095 +167096 POINT(38.70164664609626 -123.09227675870397) bank167096 +167097 POINT(38.615699100912366 -121.73081169202538) bank167097 +167098 POINT(38.355724108163734 -122.22568519141122) bank167098 +167099 POINT(37.38764766714098 -121.692595336295) bank167099 +167100 POINT(37.305475131139076 -122.80835732817238) bank167100 +167101 POINT(38.58630634073776 -121.53713824570579) bank167101 +167102 POINT(37.639039997303165 -121.4390401227963) bank167102 +167103 POINT(37.49663283771541 -121.96353304823899) bank167103 +167104 POINT(37.1138467716957 -123.28871950677808) bank167104 +167105 POINT(37.07736858722022 -122.28487353735842) bank167105 +167106 POINT(36.94912564086272 -122.76465022471253) bank167106 +167107 POINT(36.94908207502882 -122.80453050467123) bank167107 +167108 POINT(38.31323593323035 -122.35940509854716) bank167108 +167109 POINT(37.4666486257756 -122.28766623294683) bank167109 +167110 POINT(38.73347391885976 -121.81310464538215) bank167110 +167111 POINT(38.15213805940992 -121.52537303739186) bank167111 +167112 POINT(38.550677904521436 -122.64837733116829) bank167112 +167113 POINT(38.04879919026961 -122.90447345541949) bank167113 +167114 POINT(37.54188737077476 -121.45532547989463) bank167114 +167115 POINT(38.75334812236247 -122.49561689438714) bank167115 +167116 POINT(37.83511908394303 -122.86287540327726) bank167116 +167117 POINT(38.34776534142891 -121.75296881344246) bank167117 +167118 POINT(37.004733268982726 -121.4665980858137) bank167118 +167119 POINT(37.30292280136206 -121.9104498822888) bank167119 +167120 POINT(37.248180639274814 -123.08596280936405) bank167120 +167121 POINT(38.081792570036335 -122.33059609184161) bank167121 +167122 POINT(37.26844460268769 -121.69245638681568) bank167122 +167123 POINT(37.60412433722985 -121.92057841225015) bank167123 +167124 POINT(36.82659438908836 -121.53534642717159) bank167124 +167125 POINT(37.91888243078715 -123.02885454763974) bank167125 +167126 POINT(38.20603478048943 -122.1548631098647) bank167126 +167127 POINT(37.65665554361237 -122.59123325769296) bank167127 +167128 POINT(38.58290765360534 -122.8358627111641) bank167128 +167129 POINT(38.57432913451661 -123.25855278802764) bank167129 +167130 POINT(37.61505362758626 -122.27587463922347) bank167130 +167131 POINT(37.47412510687128 -123.36524556482699) bank167131 +167132 POINT(37.61706367587917 -122.4397324648758) bank167132 +167133 POINT(38.14311494092156 -122.45229794715624) bank167133 +167134 POINT(38.08985281912335 -121.76188901312453) bank167134 +167135 POINT(37.47506556931613 -123.05715502294684) bank167135 +167136 POINT(36.77534025257572 -121.7441494959027) bank167136 +167137 POINT(38.12555401126879 -122.99267662936339) bank167137 +167138 POINT(38.31870963814749 -122.57427013384564) bank167138 +167139 POINT(37.89105005642704 -122.48011371094387) bank167139 +167140 POINT(37.2454968427068 -122.45722529186033) bank167140 +167141 POINT(37.50228501588457 -122.30144592739173) bank167141 +167142 POINT(37.866307080288365 -121.69799241087803) bank167142 +167143 POINT(37.72221152743542 -122.08352211917814) bank167143 +167144 POINT(36.80304461688903 -123.27205326777025) bank167144 +167145 POINT(38.155973699871815 -121.7794920759369) bank167145 +167146 POINT(37.33735811128267 -122.60074204299848) bank167146 +167147 POINT(37.34169640590271 -121.54227076925011) bank167147 +167148 POINT(36.905114903795635 -121.662215661065) bank167148 +167149 POINT(38.55504386783115 -121.63791455965224) bank167149 +167150 POINT(37.65493155968743 -123.34970941277665) bank167150 +167151 POINT(36.99181825846336 -123.23110712678186) bank167151 +167152 POINT(36.79851015975491 -123.35104581388315) bank167152 +167153 POINT(37.41869664117532 -123.37399523778471) bank167153 +167154 POINT(37.152700723503195 -121.88298543534479) bank167154 +167155 POINT(38.32470761459736 -122.83115953965921) bank167155 +167156 POINT(37.926140224531714 -122.41449989943901) bank167156 +167157 POINT(37.15126017602054 -121.7177668081392) bank167157 +167158 POINT(37.26390135310119 -122.05617907733067) bank167158 +167159 POINT(38.73983526813874 -122.21085134539395) bank167159 +167160 POINT(37.16057289760188 -122.05266718615816) bank167160 +167161 POINT(38.0892296096698 -122.83477649593094) bank167161 +167162 POINT(38.27016972945477 -122.65515959073278) bank167162 +167163 POINT(38.53031648095303 -122.94264050608932) bank167163 +167164 POINT(37.25227143587942 -121.86525845122901) bank167164 +167165 POINT(36.78228487796164 -122.64014545106042) bank167165 +167166 POINT(36.902759602998955 -122.85397415147614) bank167166 +167167 POINT(38.12859825122168 -122.02454847588669) bank167167 +167168 POINT(36.8241600728993 -121.84947166786962) bank167168 +167169 POINT(37.06103481618032 -123.15770062199783) bank167169 +167170 POINT(37.11714085806683 -122.63496876887531) bank167170 +167171 POINT(37.991806212408164 -122.58519907060375) bank167171 +167172 POINT(37.144254113641004 -122.46226872555384) bank167172 +167173 POINT(38.41122604696285 -121.50471698648172) bank167173 +167174 POINT(38.67884315906658 -121.79088363323204) bank167174 +167175 POINT(37.020340695420806 -122.8427581914011) bank167175 +167176 POINT(37.23567757157415 -122.44517711151703) bank167176 +167177 POINT(37.815257081911184 -121.42666948956115) bank167177 +167178 POINT(38.61465707367084 -122.24899169747985) bank167178 +167179 POINT(37.499278725124334 -123.03169903111264) bank167179 +167180 POINT(37.15380187060383 -122.41041205022304) bank167180 +167181 POINT(37.05769175708339 -123.02343259970594) bank167181 +167182 POINT(38.20788272946974 -123.18772146897214) bank167182 +167183 POINT(37.24843093279043 -121.95124478812112) bank167183 +167184 POINT(37.97786446595155 -123.19736535803831) bank167184 +167185 POINT(36.903924946155186 -123.17087363308364) bank167185 +167186 POINT(37.728702133056935 -123.18571603554884) bank167186 +167187 POINT(37.82936139806998 -122.5155128710056) bank167187 +167188 POINT(38.53299908427742 -122.49593929086444) bank167188 +167189 POINT(36.969170683889445 -121.45670515378913) bank167189 +167190 POINT(38.660304937453674 -121.60098613883031) bank167190 +167191 POINT(38.42907955040267 -121.80622901612004) bank167191 +167192 POINT(37.16245193195862 -122.87232698599676) bank167192 +167193 POINT(38.14151108896715 -121.98473507900458) bank167193 +167194 POINT(37.7016236231886 -122.96740149547742) bank167194 +167195 POINT(38.63828153601356 -122.00049079574953) bank167195 +167196 POINT(37.36449073558495 -121.8170523452913) bank167196 +167197 POINT(36.81628197194505 -122.06770515305061) bank167197 +167198 POINT(37.658022069464465 -121.45186653590072) bank167198 +167199 POINT(36.825667141579636 -121.78701381751824) bank167199 +167200 POINT(37.03967860051735 -122.65152208616072) bank167200 +167201 POINT(38.30190757754918 -122.39918231319149) bank167201 +167202 POINT(37.96312074130211 -121.90879789089092) bank167202 +167203 POINT(37.72332408990648 -121.58598824451882) bank167203 +167204 POINT(36.93323232169416 -121.89692359612161) bank167204 +167205 POINT(37.88034230483259 -121.89265228009869) bank167205 +167206 POINT(37.239905103376 -121.75857748038865) bank167206 +167207 POINT(38.52248588166984 -123.15218407120143) bank167207 +167208 POINT(37.23527555994945 -121.64292344057316) bank167208 +167209 POINT(38.283657112445276 -123.22078814299846) bank167209 +167210 POINT(37.656863580207094 -122.29155184599394) bank167210 +167211 POINT(37.813089765715134 -123.29528638952173) bank167211 +167212 POINT(38.73039107354479 -123.15655388358546) bank167212 +167213 POINT(37.86708089141326 -122.0856979094571) bank167213 +167214 POINT(37.23550188324623 -122.44747567063547) bank167214 +167215 POINT(38.270006154843244 -122.0317224289115) bank167215 +167216 POINT(37.827274423318585 -122.27982160587626) bank167216 +167217 POINT(37.61511478954339 -123.05274395133581) bank167217 +167218 POINT(38.02353704109892 -122.17965047039786) bank167218 +167219 POINT(37.73449440282116 -123.07155870839458) bank167219 +167220 POINT(36.937127818995144 -122.60101607368777) bank167220 +167221 POINT(38.50982798003937 -121.84910257406496) bank167221 +167222 POINT(36.82770058424704 -122.5139796769884) bank167222 +167223 POINT(37.43233601744584 -122.50828632517664) bank167223 +167224 POINT(37.016547957016385 -122.88939484920525) bank167224 +167225 POINT(37.84003745447551 -121.67379084045196) bank167225 +167226 POINT(37.52662748784402 -122.11226163591805) bank167226 +167227 POINT(37.827121915617376 -123.10502824471469) bank167227 +167228 POINT(37.993014509775215 -122.48323022951547) bank167228 +167229 POINT(38.57109361182752 -122.7665392068272) bank167229 +167230 POINT(36.8460450051391 -122.80180869145065) bank167230 +167231 POINT(36.79684013693213 -122.71494002137311) bank167231 +167232 POINT(37.75100711742478 -123.30851257294796) bank167232 +167233 POINT(38.74612717258539 -122.28704707999971) bank167233 +167234 POINT(38.69460318172061 -123.05341003035105) bank167234 +167235 POINT(38.50867075040321 -121.97011525477689) bank167235 +167236 POINT(37.2647631446097 -121.95257662339235) bank167236 +167237 POINT(36.79323831477554 -122.04485424009957) bank167237 +167238 POINT(38.726477323877475 -122.77059625656325) bank167238 +167239 POINT(37.86946780284624 -122.62171170336379) bank167239 +167240 POINT(37.1268134217583 -121.4798758630981) bank167240 +167241 POINT(38.609971852064696 -121.83770569135763) bank167241 +167242 POINT(38.53068649243594 -122.67995329963468) bank167242 +167243 POINT(36.798117670502194 -122.48856036680006) bank167243 +167244 POINT(38.27848833619273 -121.96269219820822) bank167244 +167245 POINT(37.91944324813538 -121.58133797939168) bank167245 +167246 POINT(38.21315679145102 -122.70759994459382) bank167246 +167247 POINT(38.221641710299714 -123.1229156075849) bank167247 +167248 POINT(38.578206445031434 -122.4451515781181) bank167248 +167249 POINT(37.31302260418903 -122.28103886580921) bank167249 +167250 POINT(38.30544433144251 -121.4315245398725) bank167250 +167251 POINT(37.27487955387008 -121.65802104973093) bank167251 +167252 POINT(37.03674125770004 -121.68807093563483) bank167252 +167253 POINT(36.78306149700905 -121.74830963253619) bank167253 +167254 POINT(36.844866386607585 -121.56266773899381) bank167254 +167255 POINT(37.73254757670852 -123.10927310553312) bank167255 +167256 POINT(38.66657788063848 -121.86183226524136) bank167256 +167257 POINT(38.40042514494994 -122.02553905995568) bank167257 +167258 POINT(36.997276209058946 -123.02867988520812) bank167258 +167259 POINT(37.02234166606802 -121.65946633969041) bank167259 +167260 POINT(38.28364092461536 -123.35087314456841) bank167260 +167261 POINT(37.23807108337891 -122.52270888915493) bank167261 +167262 POINT(37.64671200222548 -122.30554470436452) bank167262 +167263 POINT(37.45285053706364 -122.53147899340046) bank167263 +167264 POINT(37.80311379535049 -122.2109570530029) bank167264 +167265 POINT(38.635864520098124 -121.7784048102559) bank167265 +167266 POINT(37.27362824367188 -122.66319033747422) bank167266 +167267 POINT(37.15388962995032 -122.9102399457552) bank167267 +167268 POINT(37.10445265253317 -123.2363772838667) bank167268 +167269 POINT(38.00452110870646 -122.23507528378146) bank167269 +167270 POINT(38.46479877950277 -123.05388172793438) bank167270 +167271 POINT(38.42258367974741 -121.93125954637335) bank167271 +167272 POINT(38.60604289582656 -122.76311873205188) bank167272 +167273 POINT(37.90540116684343 -121.91662560061548) bank167273 +167274 POINT(37.57742818992934 -122.85291969493083) bank167274 +167275 POINT(38.54711171605625 -121.67918227856194) bank167275 +167276 POINT(37.98049384909774 -122.24381049612904) bank167276 +167277 POINT(38.25188135065395 -121.59334266795662) bank167277 +167278 POINT(37.87535559658911 -121.52169875720517) bank167278 +167279 POINT(37.61242800962934 -122.9453137748328) bank167279 +167280 POINT(37.57020010957427 -123.12189456314863) bank167280 +167281 POINT(37.81532136001832 -122.18462662961707) bank167281 +167282 POINT(37.65329363157811 -122.47257157025622) bank167282 +167283 POINT(36.97146926902232 -123.0218198520691) bank167283 +167284 POINT(38.04957641188643 -122.34884886166495) bank167284 +167285 POINT(38.74087147856466 -122.61227485157174) bank167285 +167286 POINT(38.62307514802695 -122.67058882650943) bank167286 +167287 POINT(37.64729137456226 -122.09211328550732) bank167287 +167288 POINT(38.00498965779723 -121.80484630170288) bank167288 +167289 POINT(37.66114104147227 -121.89213186499106) bank167289 +167290 POINT(37.5395804638413 -121.9604690806928) bank167290 +167291 POINT(38.613910845477506 -122.40433719273177) bank167291 +167292 POINT(37.017777285788526 -123.25671585701092) bank167292 +167293 POINT(37.98419293538725 -122.12147493018774) bank167293 +167294 POINT(37.07629396075938 -122.99361975915491) bank167294 +167295 POINT(37.446285714680265 -122.56930834894912) bank167295 +167296 POINT(38.47378504537814 -122.33887913116791) bank167296 +167297 POINT(37.225191503188825 -122.21091846043265) bank167297 +167298 POINT(36.81208903369843 -122.26348654866435) bank167298 +167299 POINT(38.412340357634584 -121.99128405371293) bank167299 +167300 POINT(37.77092516496444 -121.67597939544534) bank167300 +167301 POINT(36.83473609939517 -121.63774778801492) bank167301 +167302 POINT(38.64603242295993 -121.63533251558913) bank167302 +167303 POINT(37.750834472066416 -122.75531866371652) bank167303 +167304 POINT(38.28139767754079 -122.85668772018678) bank167304 +167305 POINT(38.12550695611377 -122.46610385137927) bank167305 +167306 POINT(38.41376991670952 -121.94481006940754) bank167306 +167307 POINT(36.969932221228106 -123.00181779507562) bank167307 +167308 POINT(38.443481403491326 -122.86924375536697) bank167308 +167309 POINT(37.923349434607466 -122.70274469554339) bank167309 +167310 POINT(38.33786464630598 -121.89215060531497) bank167310 +167311 POINT(37.18244577833518 -121.51003570928489) bank167311 +167312 POINT(37.55354623350726 -122.22019312723472) bank167312 +167313 POINT(37.972034901076334 -121.55410923055288) bank167313 +167314 POINT(38.19949088249983 -122.33054991739962) bank167314 +167315 POINT(38.77293190379386 -123.27798938560066) bank167315 +167316 POINT(38.33665973782475 -121.53301155690933) bank167316 +167317 POINT(38.69696844093365 -121.44294707847632) bank167317 +167318 POINT(37.8860973515667 -121.49616967750266) bank167318 +167319 POINT(37.58995107213081 -122.79445562063133) bank167319 +167320 POINT(36.93187484882478 -122.65361444237477) bank167320 +167321 POINT(36.912103420314125 -122.06989576764364) bank167321 +167322 POINT(38.275887862058696 -122.62977333486481) bank167322 +167323 POINT(38.579493993823675 -122.99677813608751) bank167323 +167324 POINT(38.34254410344772 -123.08239851998192) bank167324 +167325 POINT(37.6312831282135 -122.61653890125616) bank167325 +167326 POINT(37.77726309136319 -121.67721226811572) bank167326 +167327 POINT(36.8880232423038 -121.94011255581509) bank167327 +167328 POINT(38.34887241732467 -122.18777515253687) bank167328 +167329 POINT(37.26118139129389 -121.455191851349) bank167329 +167330 POINT(37.33216992843147 -123.13930207607918) bank167330 +167331 POINT(37.23017637886293 -121.92438058118697) bank167331 +167332 POINT(37.400052534340674 -121.64931821039525) bank167332 +167333 POINT(38.18073825016103 -122.17127616602376) bank167333 +167334 POINT(36.79248722566531 -122.2626048298372) bank167334 +167335 POINT(37.6352337294763 -122.46343827489808) bank167335 +167336 POINT(37.598268455506314 -122.75619848486328) bank167336 +167337 POINT(36.99406031481721 -123.34166758118528) bank167337 +167338 POINT(38.36152098996836 -121.79390511657367) bank167338 +167339 POINT(38.589041993203324 -123.14387036045244) bank167339 +167340 POINT(36.82685442866608 -122.47569949597796) bank167340 +167341 POINT(37.864039246847156 -123.0539361599899) bank167341 +167342 POINT(38.248196681313416 -121.43454448094835) bank167342 +167343 POINT(36.84701685407663 -122.61668558906717) bank167343 +167344 POINT(38.18405682568978 -122.57736629743711) bank167344 +167345 POINT(37.03782117417489 -122.55308758714776) bank167345 +167346 POINT(37.97950056691013 -122.84511622373599) bank167346 +167347 POINT(38.48821337754304 -122.81214604186415) bank167347 +167348 POINT(38.68504557813819 -121.92360411727361) bank167348 +167349 POINT(38.587937555098506 -121.4954073163816) bank167349 +167350 POINT(37.27132067155381 -122.66410558145928) bank167350 +167351 POINT(38.11562580948494 -121.98650600879829) bank167351 +167352 POINT(38.66069622523353 -122.37457155161158) bank167352 +167353 POINT(37.80837788116872 -122.82897689472631) bank167353 +167354 POINT(36.98679777627623 -123.17399357245598) bank167354 +167355 POINT(37.30966100612588 -122.17159485599521) bank167355 +167356 POINT(38.20401991173411 -123.07813291716138) bank167356 +167357 POINT(38.629692022701185 -122.67039087146473) bank167357 +167358 POINT(37.6182921387829 -122.34920928976999) bank167358 +167359 POINT(38.43762528717244 -122.48634704216157) bank167359 +167360 POINT(38.61224976576229 -122.36430074763707) bank167360 +167361 POINT(37.166969046344946 -122.21919914479356) bank167361 +167362 POINT(38.13301411896263 -122.35081618792248) bank167362 +167363 POINT(36.968794691428315 -122.94155610773188) bank167363 +167364 POINT(37.43877692935089 -122.54453760375209) bank167364 +167365 POINT(37.8934612523593 -121.46730446747104) bank167365 +167366 POINT(37.89411504700587 -123.31970119651879) bank167366 +167367 POINT(37.903255290277095 -122.56968438680164) bank167367 +167368 POINT(37.878604775471885 -122.58383956566999) bank167368 +167369 POINT(38.206719811560646 -123.15521412179807) bank167369 +167370 POINT(37.795531378061874 -121.52775920038583) bank167370 +167371 POINT(38.68788044001762 -122.9581640463061) bank167371 +167372 POINT(38.50424052042692 -121.51294341838262) bank167372 +167373 POINT(37.93754315730395 -122.33104982921975) bank167373 +167374 POINT(37.29216030427219 -122.22149522360547) bank167374 +167375 POINT(37.17752886128036 -123.32716087349526) bank167375 +167376 POINT(38.33798038927709 -121.99274789869456) bank167376 +167377 POINT(37.08485274974949 -122.19772893028625) bank167377 +167378 POINT(37.331315378340285 -122.23815403611866) bank167378 +167379 POINT(37.51243963799408 -122.63919351564367) bank167379 +167380 POINT(37.81243244121858 -122.65226597587973) bank167380 +167381 POINT(37.26088450837458 -121.55412134599983) bank167381 +167382 POINT(38.269300235592404 -122.77302063847762) bank167382 +167383 POINT(36.92722348503789 -122.7472004874364) bank167383 +167384 POINT(37.0630586564241 -122.23928997772154) bank167384 +167385 POINT(37.22365863047292 -122.93238060397462) bank167385 +167386 POINT(37.0461820135545 -122.78509864261231) bank167386 +167387 POINT(37.875678743827 -122.70875246103677) bank167387 +167388 POINT(36.95071258078755 -121.8207874324312) bank167388 +167389 POINT(37.459924588641535 -123.04946728408665) bank167389 +167390 POINT(38.366329420351065 -122.26803747572363) bank167390 +167391 POINT(37.931153807869336 -122.7293998882459) bank167391 +167392 POINT(37.77181705033106 -122.25558646583832) bank167392 +167393 POINT(36.87570351401472 -122.12478328949543) bank167393 +167394 POINT(36.80952618284424 -123.06446389248767) bank167394 +167395 POINT(38.45751370928804 -121.67974968325836) bank167395 +167396 POINT(38.083704145422956 -122.07315525092112) bank167396 +167397 POINT(36.92170651220802 -122.38218564426981) bank167397 +167398 POINT(38.589389419520444 -122.94048536385273) bank167398 +167399 POINT(38.674928736108875 -122.95981997621503) bank167399 +167400 POINT(37.25911636259266 -122.32760431238951) bank167400 +167401 POINT(37.81673596924658 -121.75256406364177) bank167401 +167402 POINT(37.409214872657095 -122.34720032702751) bank167402 +167403 POINT(38.07488592260797 -122.25804970030282) bank167403 +167404 POINT(37.43911967558041 -123.36203472620828) bank167404 +167405 POINT(38.354081338724875 -123.30544112875168) bank167405 +167406 POINT(38.12143319676385 -122.51051777803497) bank167406 +167407 POINT(37.58715238144309 -122.92747506945109) bank167407 +167408 POINT(37.857718246302724 -121.4979823417802) bank167408 +167409 POINT(36.98987985433178 -121.55074749929545) bank167409 +167410 POINT(38.54334661116946 -122.57041590370484) bank167410 +167411 POINT(38.258661293999154 -122.40261381375801) bank167411 +167412 POINT(38.730026641096714 -122.09717525620947) bank167412 +167413 POINT(38.72160673307705 -122.84221190155635) bank167413 +167414 POINT(37.67545339526331 -121.42733686051778) bank167414 +167415 POINT(38.20819743143551 -122.10053958327127) bank167415 +167416 POINT(38.076277614972796 -123.37996541196279) bank167416 +167417 POINT(38.403055997894555 -122.48063739640058) bank167417 +167418 POINT(38.45740140730454 -121.94939774444951) bank167418 +167419 POINT(38.20219719424522 -121.69216938444443) bank167419 +167420 POINT(38.46906582938261 -122.65719931804509) bank167420 +167421 POINT(38.45358211534254 -121.57472179023554) bank167421 +167422 POINT(38.416805129857146 -123.02727000951533) bank167422 +167423 POINT(37.3197493671584 -123.02372713511332) bank167423 +167424 POINT(37.25432058971325 -122.21905886214792) bank167424 +167425 POINT(37.11259775470147 -122.10832944164677) bank167425 +167426 POINT(38.55198142545941 -122.1963381402817) bank167426 +167427 POINT(37.83298089291016 -122.86706082736472) bank167427 +167428 POINT(38.17139911642754 -122.41621971486288) bank167428 +167429 POINT(38.34920056122374 -123.2326763815003) bank167429 +167430 POINT(37.96054465533171 -122.11427471521918) bank167430 +167431 POINT(37.46405905983591 -122.92297756256384) bank167431 +167432 POINT(38.32274956097235 -121.92010314188857) bank167432 +167433 POINT(38.13796438870448 -121.60945187627023) bank167433 +167434 POINT(37.11733288992818 -122.18181210713914) bank167434 +167435 POINT(37.62084494792685 -122.72450471995788) bank167435 +167436 POINT(38.290018710744896 -122.69836063046526) bank167436 +167437 POINT(37.619350679704795 -121.7984499956873) bank167437 +167438 POINT(37.82788583189581 -123.32536952141348) bank167438 +167439 POINT(38.36200612852982 -122.23783245746786) bank167439 +167440 POINT(36.857942680769995 -122.02002281909972) bank167440 +167441 POINT(37.38897086773883 -122.42697316515634) bank167441 +167442 POINT(37.0444012961524 -122.51480906192118) bank167442 +167443 POINT(38.68557505761887 -122.93217044850363) bank167443 +167444 POINT(38.72679324714853 -123.0827970334039) bank167444 +167445 POINT(36.945538936097535 -121.98984052590716) bank167445 +167446 POINT(38.47830057720328 -121.52789579679626) bank167446 +167447 POINT(38.679650541271535 -122.57381596983413) bank167447 +167448 POINT(38.55005887796435 -122.69660735538717) bank167448 +167449 POINT(37.333130715992894 -122.48427933791095) bank167449 +167450 POINT(38.13637297845944 -122.81392494347179) bank167450 +167451 POINT(37.011366911419586 -121.98065450135181) bank167451 +167452 POINT(37.46168001780511 -121.62888931397661) bank167452 +167453 POINT(37.24344741882523 -122.91308542989835) bank167453 +167454 POINT(37.48790903761837 -121.97554140070189) bank167454 +167455 POINT(36.932448869757835 -123.18744197358694) bank167455 +167456 POINT(37.89637207898167 -122.71022751295578) bank167456 +167457 POINT(37.13168588825077 -121.66569308842207) bank167457 +167458 POINT(37.38878886819045 -122.38508395144044) bank167458 +167459 POINT(37.069124973071574 -121.69873056202967) bank167459 +167460 POINT(38.24225728949506 -122.7884074529964) bank167460 +167461 POINT(38.03335777464541 -123.28935904177727) bank167461 +167462 POINT(38.53968069338169 -123.3290573964404) bank167462 +167463 POINT(38.41579418088791 -123.12087416746702) bank167463 +167464 POINT(37.487601915054896 -123.26984301379021) bank167464 +167465 POINT(37.07996170330402 -122.96800488242553) bank167465 +167466 POINT(37.94867129389721 -123.2726523291116) bank167466 +167467 POINT(37.35359495455158 -121.6782590144315) bank167467 +167468 POINT(38.156520815765276 -122.15638321245476) bank167468 +167469 POINT(37.450817979030035 -122.02909374849457) bank167469 +167470 POINT(38.193204603693275 -122.61165300161498) bank167470 +167471 POINT(37.02483617583434 -123.26125236335724) bank167471 +167472 POINT(38.49778527947661 -121.59054231170154) bank167472 +167473 POINT(38.134209855665205 -122.36446366551833) bank167473 +167474 POINT(38.7397773824473 -123.30803347792364) bank167474 +167475 POINT(36.868293308412724 -123.19044539805219) bank167475 +167476 POINT(38.0462725463763 -121.63803174511105) bank167476 +167477 POINT(37.30451775447314 -122.51048422980739) bank167477 +167478 POINT(36.99332430931448 -121.83075588431103) bank167478 +167479 POINT(37.57175863696803 -121.79594355059324) bank167479 +167480 POINT(37.388239182317875 -121.49857142237421) bank167480 +167481 POINT(37.240307552469055 -122.39645615636088) bank167481 +167482 POINT(37.903573218123135 -121.62819369951414) bank167482 +167483 POINT(38.47499568075567 -122.17884118019568) bank167483 +167484 POINT(37.269329976262455 -122.59473880499344) bank167484 +167485 POINT(38.27622687453296 -122.77436776015256) bank167485 +167486 POINT(36.77697316040882 -123.04376994098092) bank167486 +167487 POINT(36.819742804987655 -123.08194729574177) bank167487 +167488 POINT(38.74781509636125 -121.65742473218367) bank167488 +167489 POINT(37.635742911438214 -122.14143572015017) bank167489 +167490 POINT(38.18584819024318 -121.86949026716272) bank167490 +167491 POINT(37.68854640751881 -122.20566810640244) bank167491 +167492 POINT(38.583017421042726 -122.92652536929566) bank167492 +167493 POINT(37.22775955199772 -123.31844572369157) bank167493 +167494 POINT(37.11102398353534 -121.56854916360429) bank167494 +167495 POINT(37.447018646251536 -122.61276595093511) bank167495 +167496 POINT(38.72836400897903 -122.8329749553526) bank167496 +167497 POINT(38.26385215025263 -122.8018390423758) bank167497 +167498 POINT(36.92651536509855 -122.67147472425964) bank167498 +167499 POINT(38.24088653425631 -122.38670273964065) bank167499 +167500 POINT(37.23483538909684 -121.852843161636) bank167500 +167501 POINT(37.073462337518535 -121.97103521076336) bank167501 +167502 POINT(38.18693695538395 -122.17067877193377) bank167502 +167503 POINT(38.52793267753129 -122.38747227714009) bank167503 +167504 POINT(38.65750158592151 -122.27306609779387) bank167504 +167505 POINT(36.78251181302992 -122.98706214749149) bank167505 +167506 POINT(37.01582847119612 -123.29736334705322) bank167506 +167507 POINT(38.63592163875756 -122.47163690255411) bank167507 +167508 POINT(37.24492958586538 -123.13077778910686) bank167508 +167509 POINT(36.97966487612481 -123.27791231433945) bank167509 +167510 POINT(38.68007808050417 -121.56016010922237) bank167510 +167511 POINT(36.957883685090856 -121.71179710322582) bank167511 +167512 POINT(36.96392348545588 -123.35660492337858) bank167512 +167513 POINT(37.231909242633634 -122.73462507549948) bank167513 +167514 POINT(37.47485362593165 -121.91829307509207) bank167514 +167515 POINT(37.27989083938751 -121.51034931866364) bank167515 +167516 POINT(38.05715552355272 -123.15439244321652) bank167516 +167517 POINT(36.98667268565372 -123.12397005377575) bank167517 +167518 POINT(38.440623030235656 -121.58729316132427) bank167518 +167519 POINT(38.38853251882463 -122.44849226700718) bank167519 +167520 POINT(38.10289617838586 -122.69259745242624) bank167520 +167521 POINT(37.06594307924458 -121.6100502385807) bank167521 +167522 POINT(37.313748453439935 -122.15619011119313) bank167522 +167523 POINT(37.551504766515706 -123.09323858779452) bank167523 +167524 POINT(37.40194221203094 -122.52915164202793) bank167524 +167525 POINT(36.83041390318381 -122.57787267421273) bank167525 +167526 POINT(36.91517589407429 -121.71140553629934) bank167526 +167527 POINT(38.32181285618199 -122.82599122831797) bank167527 +167528 POINT(37.8564888187776 -122.61462385650971) bank167528 +167529 POINT(37.85928478129174 -121.51127076761409) bank167529 +167530 POINT(38.53793450709001 -121.44941388699044) bank167530 +167531 POINT(37.46053207432183 -123.34286417068118) bank167531 +167532 POINT(37.17631636712698 -122.9929543004747) bank167532 +167533 POINT(38.22406742406982 -123.01858773451919) bank167533 +167534 POINT(37.49965197707083 -122.77632236197182) bank167534 +167535 POINT(38.46633301746173 -122.34333015824174) bank167535 +167536 POINT(38.06251243978016 -123.12752837594833) bank167536 +167537 POINT(38.27257605804867 -122.39906163193089) bank167537 +167538 POINT(37.1223855139893 -123.10219571123001) bank167538 +167539 POINT(38.266885619607 -122.62939083511193) bank167539 +167540 POINT(37.17451962074651 -123.1460572656915) bank167540 +167541 POINT(38.449570714567955 -121.54106705545429) bank167541 +167542 POINT(37.3161397451968 -123.11477724947753) bank167542 +167543 POINT(37.90012208389376 -122.45429516083466) bank167543 +167544 POINT(38.73283032328256 -122.87164550218662) bank167544 +167545 POINT(37.46924115528408 -121.72225420178019) bank167545 +167546 POINT(38.22117498982354 -121.45087172822171) bank167546 +167547 POINT(37.771361284623225 -122.8934684273469) bank167547 +167548 POINT(38.18546646462685 -121.81759602675042) bank167548 +167549 POINT(38.448443494679466 -121.59891458788773) bank167549 +167550 POINT(38.242748775719924 -122.81795892746696) bank167550 +167551 POINT(38.718638002509785 -122.71421960020342) bank167551 +167552 POINT(37.04759377455429 -122.58682577744943) bank167552 +167553 POINT(37.933023656790425 -122.73039405378645) bank167553 +167554 POINT(37.265255062554786 -122.80425317329374) bank167554 +167555 POINT(36.97182896064928 -122.44165588825246) bank167555 +167556 POINT(37.19743046766005 -122.93091488884663) bank167556 +167557 POINT(37.842674316196494 -122.37363069487948) bank167557 +167558 POINT(37.90788714610032 -122.43231633585944) bank167558 +167559 POINT(37.568324312114 -121.86632334680222) bank167559 +167560 POINT(37.1171991033937 -122.31317250630103) bank167560 +167561 POINT(37.53760011251439 -123.31466692008806) bank167561 +167562 POINT(38.7509944843007 -122.04124938822775) bank167562 +167563 POINT(37.34906996351652 -122.03842056776476) bank167563 +167564 POINT(37.60936225110767 -121.86392243286839) bank167564 +167565 POINT(38.01814962441948 -122.32296786041734) bank167565 +167566 POINT(37.138228138443914 -122.67418846555191) bank167566 +167567 POINT(36.94185094665566 -123.16529456627224) bank167567 +167568 POINT(38.657173132441855 -122.7355766770405) bank167568 +167569 POINT(37.27200102094147 -122.0567641820152) bank167569 +167570 POINT(36.822954140366285 -122.99018577604001) bank167570 +167571 POINT(36.83552113566689 -121.86916608782558) bank167571 +167572 POINT(36.952010411041435 -123.2276506137838) bank167572 +167573 POINT(38.06856487454883 -122.32850538704093) bank167573 +167574 POINT(37.553288921019444 -122.17588048410671) bank167574 +167575 POINT(38.42788728889728 -123.00587844237768) bank167575 +167576 POINT(36.85472915994096 -121.82722549500322) bank167576 +167577 POINT(37.66544192131662 -122.30564732710381) bank167577 +167578 POINT(38.458189439941656 -123.38364276833089) bank167578 +167579 POINT(37.127866716191456 -122.18027612445323) bank167579 +167580 POINT(38.260765896603054 -122.20848347832731) bank167580 +167581 POINT(38.58199392328714 -122.71199073759925) bank167581 +167582 POINT(37.669383327128266 -122.82415657555381) bank167582 +167583 POINT(38.07030284994336 -122.8981204450314) bank167583 +167584 POINT(38.07809745248488 -121.89791448770754) bank167584 +167585 POINT(38.696475336205616 -123.30541986597788) bank167585 +167586 POINT(38.297738042019404 -123.22893167009411) bank167586 +167587 POINT(37.26307739502896 -123.34001451135506) bank167587 +167588 POINT(38.39043491077342 -121.61879831957715) bank167588 +167589 POINT(37.55833956937214 -122.01512813858919) bank167589 +167590 POINT(37.51724096018963 -121.73976624724261) bank167590 +167591 POINT(38.34773184663547 -122.90199404064323) bank167591 +167592 POINT(38.629653777743414 -121.48705216053096) bank167592 +167593 POINT(36.92626262854891 -122.36250386773142) bank167593 +167594 POINT(37.115266770468224 -122.79932452330742) bank167594 +167595 POINT(37.63762632253359 -123.3202076126609) bank167595 +167596 POINT(38.160131259056456 -122.33118846425927) bank167596 +167597 POINT(37.794785536390584 -122.28351947876767) bank167597 +167598 POINT(36.951175609549765 -123.07266207020061) bank167598 +167599 POINT(37.09872273544004 -123.05323401003312) bank167599 +167600 POINT(37.48032207854227 -122.15433900210428) bank167600 +167601 POINT(38.39262902118869 -122.25430767119424) bank167601 +167602 POINT(38.377666212506874 -122.83305379622662) bank167602 +167603 POINT(37.17778595394099 -122.61863511115106) bank167603 +167604 POINT(38.029798836275276 -122.35931219756093) bank167604 +167605 POINT(37.57958728917633 -122.18822954290869) bank167605 +167606 POINT(38.62191720950692 -121.73837437103765) bank167606 +167607 POINT(37.34901578345337 -122.91777954211084) bank167607 +167608 POINT(38.22946167830428 -121.91392822976381) bank167608 +167609 POINT(36.857066343377696 -122.8990350414476) bank167609 +167610 POINT(38.321528650961746 -121.48040284121822) bank167610 +167611 POINT(38.73948819719175 -123.36309314615754) bank167611 +167612 POINT(38.659772166647215 -121.96616446898418) bank167612 +167613 POINT(37.86716225227767 -123.32423928658505) bank167613 +167614 POINT(37.334549249524656 -123.35052872385334) bank167614 +167615 POINT(37.88153431831867 -122.8077617253299) bank167615 +167616 POINT(37.40870552392976 -122.5260810574171) bank167616 +167617 POINT(37.81308056216999 -122.81746541755031) bank167617 +167618 POINT(37.34116001635825 -122.65880248979924) bank167618 +167619 POINT(37.44387075015671 -123.2213888169171) bank167619 +167620 POINT(37.64859564492376 -122.94801439368975) bank167620 +167621 POINT(38.69333373686668 -123.27143866593156) bank167621 +167622 POINT(38.247557921128376 -121.65808454907796) bank167622 +167623 POINT(37.56800914432103 -122.61352509911866) bank167623 +167624 POINT(38.65320753825865 -123.02373431069076) bank167624 +167625 POINT(38.73097808829291 -122.24591063309542) bank167625 +167626 POINT(36.786496527082775 -121.63758437664912) bank167626 +167627 POINT(38.348909493071574 -123.40235713804617) bank167627 +167628 POINT(37.4968222347255 -122.35484376908803) bank167628 +167629 POINT(36.924646465534416 -122.93346401815057) bank167629 +167630 POINT(37.14544495872201 -122.93663079023743) bank167630 +167631 POINT(38.772253583897886 -121.85266339184993) bank167631 +167632 POINT(38.588220350337245 -122.24177459241082) bank167632 +167633 POINT(38.46532005626704 -123.03624774064573) bank167633 +167634 POINT(37.27370296277823 -122.06864934014772) bank167634 +167635 POINT(37.49254764001497 -122.51166101844255) bank167635 +167636 POINT(37.548645770069434 -121.49953621588891) bank167636 +167637 POINT(37.566807140936156 -121.57097809884695) bank167637 +167638 POINT(38.222519473350104 -123.30122435459519) bank167638 +167639 POINT(36.77575900484776 -123.41363230583251) bank167639 +167640 POINT(36.87311498461911 -122.57737319966428) bank167640 +167641 POINT(37.647002816240345 -122.50106924702364) bank167641 +167642 POINT(38.26132104408189 -122.78225607367804) bank167642 +167643 POINT(37.039749712237196 -123.14672246897754) bank167643 +167644 POINT(38.06146272794091 -121.87451273679814) bank167644 +167645 POINT(37.82825983106139 -122.7746841973327) bank167645 +167646 POINT(38.52424648391709 -123.21909322627131) bank167646 +167647 POINT(38.28044956087184 -123.21355380490041) bank167647 +167648 POINT(37.006412823878954 -123.3497017543672) bank167648 +167649 POINT(37.47785955629689 -121.956413812545) bank167649 +167650 POINT(38.04321702924399 -121.48419077896915) bank167650 +167651 POINT(36.810366193569585 -123.2579717691342) bank167651 +167652 POINT(38.70227925173873 -123.06223855712125) bank167652 +167653 POINT(37.880072593544874 -122.30591285005201) bank167653 +167654 POINT(38.46993698884919 -122.02229510015458) bank167654 +167655 POINT(38.664394257816014 -121.60189074462731) bank167655 +167656 POINT(37.90583228290296 -121.68013426900968) bank167656 +167657 POINT(37.10981863482465 -122.75622003430358) bank167657 +167658 POINT(38.38551698381814 -122.20919580912133) bank167658 +167659 POINT(37.29456790785394 -121.55403641726308) bank167659 +167660 POINT(37.70512923137516 -121.69780835575312) bank167660 +167661 POINT(37.366558405550286 -122.86383086812887) bank167661 +167662 POINT(38.37516406942974 -122.7194808563384) bank167662 +167663 POINT(38.211855049401485 -122.25997365864293) bank167663 +167664 POINT(38.522674829070624 -122.59885036479066) bank167664 +167665 POINT(36.9822425813954 -123.41023282670749) bank167665 +167666 POINT(37.755897247018645 -121.90707138381738) bank167666 +167667 POINT(38.65622928813341 -123.15553190887223) bank167667 +167668 POINT(36.85834456380622 -123.16369902562396) bank167668 +167669 POINT(38.30707060978007 -122.1234519972277) bank167669 +167670 POINT(38.74893752411142 -122.38744436819994) bank167670 +167671 POINT(37.18842544507707 -122.50470300915475) bank167671 +167672 POINT(37.5070266493332 -121.44057575984307) bank167672 +167673 POINT(37.40097022472877 -122.56387565920498) bank167673 +167674 POINT(37.02499904843864 -122.658336133299) bank167674 +167675 POINT(36.951135162775934 -122.69039006739914) bank167675 +167676 POINT(37.859163529457895 -123.25313463288862) bank167676 +167677 POINT(38.4406996113493 -123.15030456142354) bank167677 +167678 POINT(37.2516651963609 -121.47092367347277) bank167678 +167679 POINT(37.36105238510814 -122.4466747922512) bank167679 +167680 POINT(37.402035000767775 -122.74051763923295) bank167680 +167681 POINT(38.01843840282029 -122.38013827926305) bank167681 +167682 POINT(36.84549043127497 -122.32009279986136) bank167682 +167683 POINT(37.28103518905072 -123.26961353034498) bank167683 +167684 POINT(38.21390854350275 -122.0806314539799) bank167684 +167685 POINT(38.589331293031066 -122.68127384188394) bank167685 +167686 POINT(37.225171822388155 -123.0926594211947) bank167686 +167687 POINT(38.565569822924864 -121.95653643561498) bank167687 +167688 POINT(38.25328957142855 -122.11170832714278) bank167688 +167689 POINT(37.98540335149099 -123.2552322684741) bank167689 +167690 POINT(38.30892053251013 -122.79691353057578) bank167690 +167691 POINT(38.10491751644795 -123.1936378127442) bank167691 +167692 POINT(37.98278057063337 -122.08372638546034) bank167692 +167693 POINT(36.8564015867218 -122.34484112567954) bank167693 +167694 POINT(38.653109627701625 -121.45772939970436) bank167694 +167695 POINT(37.332114929925275 -121.7414819461536) bank167695 +167696 POINT(37.51599214809705 -121.83338751800031) bank167696 +167697 POINT(38.05543319648325 -122.28567305506519) bank167697 +167698 POINT(37.183365802701 -122.80497368198496) bank167698 +167699 POINT(37.94807999211342 -122.64182129436004) bank167699 +167700 POINT(37.67308684227788 -122.42858019681003) bank167700 +167701 POINT(38.107503640056024 -123.00253445161391) bank167701 +167702 POINT(38.49891306195425 -122.7659060438304) bank167702 +167703 POINT(38.51428944228145 -121.42064118905213) bank167703 +167704 POINT(37.89725302410361 -121.73746171991613) bank167704 +167705 POINT(38.41650584252684 -122.08981018459005) bank167705 +167706 POINT(38.34379371176179 -121.85666778622858) bank167706 +167707 POINT(37.5281125294263 -121.76550118695391) bank167707 +167708 POINT(37.27236990534739 -122.25163697706422) bank167708 +167709 POINT(37.847947886734275 -121.89804920816007) bank167709 +167710 POINT(36.84773016147632 -121.58146909327796) bank167710 +167711 POINT(38.14926498206156 -122.74547165637271) bank167711 +167712 POINT(37.7401795721884 -121.70881247165924) bank167712 +167713 POINT(38.241341525426 -121.88034795971949) bank167713 +167714 POINT(38.081035760342736 -122.61001648877871) bank167714 +167715 POINT(37.1086131659624 -122.5685903236903) bank167715 +167716 POINT(37.566648422882146 -123.34990736843703) bank167716 +167717 POINT(36.857205422060844 -121.95427170140688) bank167717 +167718 POINT(38.38130559073021 -121.46059379043992) bank167718 +167719 POINT(38.487126258407244 -122.87065344919706) bank167719 +167720 POINT(36.972554588824195 -123.25780207362538) bank167720 +167721 POINT(37.640708950769195 -122.75902305869408) bank167721 +167722 POINT(38.75134549367625 -122.20262696702966) bank167722 +167723 POINT(38.25092576816611 -123.22944639949927) bank167723 +167724 POINT(38.410524832789136 -121.49702804473601) bank167724 +167725 POINT(37.28916111878472 -122.62080019312351) bank167725 +167726 POINT(37.81119138717007 -123.30266809061544) bank167726 +167727 POINT(37.954602850493345 -122.12092914213994) bank167727 +167728 POINT(37.452933347797135 -122.50765641962565) bank167728 +167729 POINT(37.73072969668463 -121.92180588179637) bank167729 +167730 POINT(37.97974092250806 -123.40811356760094) bank167730 +167731 POINT(37.685751822747406 -122.29243778334796) bank167731 +167732 POINT(37.940131948232825 -121.68718516735753) bank167732 +167733 POINT(38.65861607434876 -122.08261240314754) bank167733 +167734 POINT(37.66627284628476 -123.01289894271632) bank167734 +167735 POINT(38.18543315290995 -122.56632479870107) bank167735 +167736 POINT(38.11893759000248 -122.77976236193855) bank167736 +167737 POINT(38.7398231074419 -122.96039689433545) bank167737 +167738 POINT(38.74542363156967 -122.49489662779017) bank167738 +167739 POINT(37.80794283832426 -123.18245746870707) bank167739 +167740 POINT(37.455204519164916 -121.70674984228005) bank167740 +167741 POINT(37.82389471008316 -122.72690510507135) bank167741 +167742 POINT(36.99920748224869 -121.5611278739731) bank167742 +167743 POINT(38.22220234667178 -122.43479769548877) bank167743 +167744 POINT(37.51098586438717 -122.43053572544065) bank167744 +167745 POINT(38.68051277648383 -123.33733891987693) bank167745 +167746 POINT(37.00244847995277 -123.36385359257476) bank167746 +167747 POINT(37.25696145795117 -121.67086000305927) bank167747 +167748 POINT(37.8934580210045 -121.42874000602737) bank167748 +167749 POINT(38.15852093343557 -121.77052283971838) bank167749 +167750 POINT(38.691750418416554 -122.58345633933625) bank167750 +167751 POINT(37.63785345333101 -121.43851679472453) bank167751 +167752 POINT(38.18409368098935 -122.24783276445572) bank167752 +167753 POINT(37.187225689589525 -123.33517657231283) bank167753 +167754 POINT(38.05712980732324 -123.2939955002181) bank167754 +167755 POINT(38.313689792227535 -121.45098589853302) bank167755 +167756 POINT(37.80880324131591 -121.6543585708422) bank167756 +167757 POINT(38.325676318011595 -121.93838729963646) bank167757 +167758 POINT(38.12520131230533 -121.8625814330791) bank167758 +167759 POINT(36.96018338185782 -121.7504365332404) bank167759 +167760 POINT(37.665401104794974 -122.24873475823208) bank167760 +167761 POINT(37.06058696362689 -122.46869003563266) bank167761 +167762 POINT(38.71238203658902 -122.62444203490692) bank167762 +167763 POINT(37.5078855637671 -121.84971357853566) bank167763 +167764 POINT(37.57086115663359 -123.0034867882917) bank167764 +167765 POINT(37.12219079142095 -121.91452604892616) bank167765 +167766 POINT(37.06070328210577 -121.53919013766368) bank167766 +167767 POINT(37.10110815325631 -122.7600441273298) bank167767 +167768 POINT(37.34141748018637 -121.76380793005919) bank167768 +167769 POINT(38.54193510499481 -123.06419308771035) bank167769 +167770 POINT(36.986818789571686 -121.68297518670886) bank167770 +167771 POINT(37.68913618646449 -121.85755063010797) bank167771 +167772 POINT(38.003854999894024 -123.41381791896232) bank167772 +167773 POINT(37.873544791530676 -121.69888781682491) bank167773 +167774 POINT(37.42962884758314 -122.64071288857872) bank167774 +167775 POINT(38.68226510901754 -122.56127802224775) bank167775 +167776 POINT(37.4253610799215 -121.99285025071518) bank167776 +167777 POINT(36.89025607715463 -123.04107554199739) bank167777 +167778 POINT(38.765180922251794 -121.89930577187955) bank167778 +167779 POINT(38.61656521055791 -122.29057536318582) bank167779 +167780 POINT(37.0397215266615 -121.9244666663015) bank167780 +167781 POINT(38.22527636254398 -122.98950421293956) bank167781 +167782 POINT(37.05737875123598 -122.14735018308495) bank167782 +167783 POINT(37.28892216037731 -121.8312735345851) bank167783 +167784 POINT(37.1547949237385 -123.18952442293617) bank167784 +167785 POINT(38.57559395696564 -122.68306409577873) bank167785 +167786 POINT(37.437641008326445 -122.4752680152121) bank167786 +167787 POINT(38.08945121009163 -121.78183299922698) bank167787 +167788 POINT(37.76904469631086 -122.21166383493006) bank167788 +167789 POINT(37.026828526742946 -122.14215630352334) bank167789 +167790 POINT(37.821828142870835 -123.00319001005381) bank167790 +167791 POINT(38.701393241459925 -123.30378926068008) bank167791 +167792 POINT(37.495578700441555 -122.46731835733212) bank167792 +167793 POINT(37.437587996485064 -121.53456425027471) bank167793 +167794 POINT(37.61706418161905 -121.55006928414234) bank167794 +167795 POINT(37.945942076662035 -121.94480594579713) bank167795 +167796 POINT(37.34644539559538 -122.4450171108588) bank167796 +167797 POINT(37.941917587004184 -123.10303466274921) bank167797 +167798 POINT(37.729148056377205 -122.19490493342374) bank167798 +167799 POINT(37.611037724863564 -122.8227936327459) bank167799 +167800 POINT(37.24701968721738 -123.39464960293719) bank167800 +167801 POINT(38.55516106579226 -121.76820334888708) bank167801 +167802 POINT(37.87508137641478 -121.58410269095909) bank167802 +167803 POINT(37.18704242061503 -122.05260838472744) bank167803 +167804 POINT(37.137868650579534 -122.0908606951049) bank167804 +167805 POINT(38.16347698669766 -122.14339793433585) bank167805 +167806 POINT(38.621592744155045 -121.54916850025063) bank167806 +167807 POINT(38.37220690765232 -122.91201886400432) bank167807 +167808 POINT(37.33793603633546 -122.83094043576291) bank167808 +167809 POINT(37.86862868014233 -121.44103931356958) bank167809 +167810 POINT(38.64059611575841 -122.54981178884542) bank167810 +167811 POINT(38.2862346918912 -122.05006170335861) bank167811 +167812 POINT(38.61885630579561 -121.67383783331614) bank167812 +167813 POINT(37.51445448903963 -122.40943317244817) bank167813 +167814 POINT(37.59485895205815 -123.18782568727657) bank167814 +167815 POINT(37.12158248398497 -122.85051899169362) bank167815 +167816 POINT(36.91104518070545 -122.74476630867605) bank167816 +167817 POINT(37.58300752344658 -122.73993058384278) bank167817 +167818 POINT(38.66698807557614 -122.23531782106244) bank167818 +167819 POINT(38.188172864353774 -122.43090362964793) bank167819 +167820 POINT(37.204674671385554 -123.14073762803316) bank167820 +167821 POINT(37.86572502313416 -121.90638326939677) bank167821 +167822 POINT(36.996352027173614 -123.14285374356992) bank167822 +167823 POINT(37.18379122927253 -123.13415830871615) bank167823 +167824 POINT(37.58779212108987 -121.52472584217497) bank167824 +167825 POINT(37.76490371307547 -122.77424040910732) bank167825 +167826 POINT(37.629947949771946 -121.51183856790058) bank167826 +167827 POINT(38.409112461704936 -123.19250242558897) bank167827 +167828 POINT(36.90674377414112 -122.4794344914591) bank167828 +167829 POINT(38.261198646637745 -121.42499418582682) bank167829 +167830 POINT(38.60472863458116 -121.98818130931419) bank167830 +167831 POINT(37.13298640597321 -122.14805246237356) bank167831 +167832 POINT(37.41163985092117 -121.42727155661444) bank167832 +167833 POINT(37.21324496383855 -122.99497506901618) bank167833 +167834 POINT(37.10252151851099 -122.00955594319112) bank167834 +167835 POINT(38.29319948244698 -122.40963320549984) bank167835 +167836 POINT(37.67808075397479 -122.86872359510653) bank167836 +167837 POINT(37.189767646553726 -122.5331045786768) bank167837 +167838 POINT(37.638334720786254 -122.00942105326212) bank167838 +167839 POINT(37.365030444675384 -122.63997484991611) bank167839 +167840 POINT(38.46969552477788 -123.35623110923844) bank167840 +167841 POINT(38.46296137391501 -122.53191009543886) bank167841 +167842 POINT(38.07003801328065 -121.82399580472475) bank167842 +167843 POINT(38.52363705102161 -121.75438066042275) bank167843 +167844 POINT(37.85775861265891 -122.12986501258409) bank167844 +167845 POINT(37.01055072476953 -121.90763076912681) bank167845 +167846 POINT(36.91065015831411 -123.28878464239777) bank167846 +167847 POINT(38.399858237072145 -122.0129469174213) bank167847 +167848 POINT(38.60907773535466 -121.55571750592566) bank167848 +167849 POINT(38.49265535631449 -123.1775990554619) bank167849 +167850 POINT(37.62255270103977 -122.41427286845065) bank167850 +167851 POINT(36.9877075405168 -123.32562442717595) bank167851 +167852 POINT(38.12506412954013 -121.77813861585479) bank167852 +167853 POINT(37.745990986518756 -122.4118434578508) bank167853 +167854 POINT(38.3515184585263 -123.07779320414527) bank167854 +167855 POINT(37.90686453230997 -122.21091482641826) bank167855 +167856 POINT(37.114454548476466 -122.50650104503545) bank167856 +167857 POINT(38.499616916972016 -121.87687105781644) bank167857 +167858 POINT(37.212435080879935 -122.00553364833623) bank167858 +167859 POINT(37.94551479959824 -123.21534164498651) bank167859 +167860 POINT(36.962018620827344 -121.94007940785912) bank167860 +167861 POINT(38.43280718058433 -122.07479019165405) bank167861 +167862 POINT(36.86907030510884 -121.48241967728838) bank167862 +167863 POINT(38.54272618591277 -122.32283617009004) bank167863 +167864 POINT(37.10187335118611 -122.76504875686003) bank167864 +167865 POINT(37.27563082945049 -123.30794525214017) bank167865 +167866 POINT(38.64151306943551 -123.22540832468992) bank167866 +167867 POINT(36.89193250392572 -122.31259314873182) bank167867 +167868 POINT(38.555137854254454 -121.62192555849568) bank167868 +167869 POINT(38.41759543078061 -122.65513541622508) bank167869 +167870 POINT(38.56682517399068 -122.56507313636506) bank167870 +167871 POINT(37.800520778159886 -121.93569010250285) bank167871 +167872 POINT(37.01500538117721 -123.17270880458591) bank167872 +167873 POINT(36.957725328250355 -122.27662437575458) bank167873 +167874 POINT(37.03183239526628 -122.25382587241603) bank167874 +167875 POINT(37.84255106882781 -121.88419560342831) bank167875 +167876 POINT(38.29375840933658 -122.04016805234588) bank167876 +167877 POINT(38.144513707950196 -122.90203843680449) bank167877 +167878 POINT(38.58330427930427 -121.8616743669692) bank167878 +167879 POINT(38.321022224172715 -122.30842112404727) bank167879 +167880 POINT(38.26000161519978 -122.77858652780911) bank167880 +167881 POINT(36.79629841600085 -123.01033275893802) bank167881 +167882 POINT(37.788263194367836 -122.55242443456764) bank167882 +167883 POINT(37.77080229516841 -121.62285539305816) bank167883 +167884 POINT(38.41601928751054 -122.52218889212756) bank167884 +167885 POINT(38.430262753381214 -121.68436363629965) bank167885 +167886 POINT(36.87945412617644 -123.19833766022184) bank167886 +167887 POINT(37.30021040119533 -123.3151575536123) bank167887 +167888 POINT(38.32473145718252 -121.47206453166878) bank167888 +167889 POINT(37.90207704154142 -122.13323195397776) bank167889 +167890 POINT(37.9055554075484 -122.65757001314783) bank167890 +167891 POINT(37.958851948280554 -122.84345656473278) bank167891 +167892 POINT(37.84055667187449 -122.17283973071798) bank167892 +167893 POINT(38.41728812777568 -121.94529055269841) bank167893 +167894 POINT(37.925437443724206 -122.65432504841503) bank167894 +167895 POINT(36.81519950983601 -122.65606258781901) bank167895 +167896 POINT(38.76821732830932 -122.09405841754786) bank167896 +167897 POINT(37.275447955685365 -122.8770148800119) bank167897 +167898 POINT(38.699604597684974 -122.65074641537595) bank167898 +167899 POINT(38.69929817604945 -121.94756036356269) bank167899 +167900 POINT(38.401410333551674 -122.76795009094793) bank167900 +167901 POINT(38.07196822215035 -122.67519444974904) bank167901 +167902 POINT(36.99546495285608 -123.29617731706759) bank167902 +167903 POINT(37.5152339708761 -123.04150443227367) bank167903 +167904 POINT(37.8823641244472 -122.47887570328245) bank167904 +167905 POINT(38.66135850054577 -122.86690376618533) bank167905 +167906 POINT(36.84676102182575 -123.14392224674886) bank167906 +167907 POINT(37.151324896746246 -123.30464508531173) bank167907 +167908 POINT(37.017267086819 -121.70664620243335) bank167908 +167909 POINT(37.51007230762522 -122.05108131747298) bank167909 +167910 POINT(37.6662561779433 -122.85587342706947) bank167910 +167911 POINT(37.8274196119862 -123.39919596368246) bank167911 +167912 POINT(36.93592812720416 -122.13177047571565) bank167912 +167913 POINT(38.263771335960904 -121.83995175888036) bank167913 +167914 POINT(37.82487911338866 -123.20655971961612) bank167914 +167915 POINT(36.9061258610923 -123.21334089843924) bank167915 +167916 POINT(37.81977463407042 -122.37960351345373) bank167916 +167917 POINT(37.30623456001213 -123.19656026105172) bank167917 +167918 POINT(37.29891861957561 -123.19358113628749) bank167918 +167919 POINT(38.237930944319075 -122.76981639467007) bank167919 +167920 POINT(38.487794595159286 -123.10840402590901) bank167920 +167921 POINT(38.42680721316582 -122.76379161868601) bank167921 +167922 POINT(37.46841904683048 -121.97991062959554) bank167922 +167923 POINT(37.995561768073514 -122.89073616005234) bank167923 +167924 POINT(37.918796950297235 -122.00858307202934) bank167924 +167925 POINT(37.781659451385494 -122.71107157760267) bank167925 +167926 POINT(37.84064245481786 -123.02622865457427) bank167926 +167927 POINT(37.0595141474398 -122.2665759794169) bank167927 +167928 POINT(38.57992460757016 -122.9679757830914) bank167928 +167929 POINT(38.28083317879041 -121.9721468467431) bank167929 +167930 POINT(37.019755744605206 -122.56157510015535) bank167930 +167931 POINT(38.307681777847435 -121.53019551423759) bank167931 +167932 POINT(38.370132009452135 -121.96188171452708) bank167932 +167933 POINT(38.196675454057 -123.39349723039135) bank167933 +167934 POINT(37.83876193307893 -123.21370735461855) bank167934 +167935 POINT(38.75456396534292 -122.55007457498846) bank167935 +167936 POINT(37.919239734393805 -122.41416408767168) bank167936 +167937 POINT(38.444221712109965 -122.4089301993643) bank167937 +167938 POINT(38.560685287836534 -122.33449131477177) bank167938 +167939 POINT(38.326496452355805 -122.18424173608783) bank167939 +167940 POINT(38.62423889923308 -122.5956532337212) bank167940 +167941 POINT(38.58827891564723 -123.29377200413458) bank167941 +167942 POINT(37.46274522637013 -123.06477317556515) bank167942 +167943 POINT(36.82551257729663 -123.13372459197534) bank167943 +167944 POINT(37.561531054473036 -123.30550855584346) bank167944 +167945 POINT(38.16080309709437 -122.37962170935083) bank167945 +167946 POINT(38.56656760083225 -121.66133657442953) bank167946 +167947 POINT(37.45404449339371 -121.67244687315046) bank167947 +167948 POINT(38.30345116807698 -122.48924497318306) bank167948 +167949 POINT(38.65267796168977 -121.96474461422432) bank167949 +167950 POINT(37.4279838092481 -122.54093860906683) bank167950 +167951 POINT(38.735921172312786 -122.66200692627957) bank167951 +167952 POINT(37.759013711608006 -121.82966110496328) bank167952 +167953 POINT(37.50667689718737 -121.49643899411961) bank167953 +167954 POINT(38.43449876307899 -121.44260628853488) bank167954 +167955 POINT(37.02797648020891 -121.85251289104097) bank167955 +167956 POINT(37.87627390766293 -121.75614427655668) bank167956 +167957 POINT(37.4627172340044 -122.28459099201302) bank167957 +167958 POINT(37.675694924641775 -122.47972008937703) bank167958 +167959 POINT(37.15197738478561 -121.76868443576457) bank167959 +167960 POINT(37.363789413763904 -122.24618822882215) bank167960 +167961 POINT(38.471454363643474 -121.67946125410627) bank167961 +167962 POINT(38.184515252616414 -121.44418343061831) bank167962 +167963 POINT(38.1831491817259 -123.33023584359682) bank167963 +167964 POINT(37.580424252496975 -122.66352353222943) bank167964 +167965 POINT(37.21459562146714 -123.12217592758876) bank167965 +167966 POINT(38.29873699909619 -122.22939915910425) bank167966 +167967 POINT(37.463945174230126 -122.99200852515222) bank167967 +167968 POINT(38.38977445355243 -121.60297592694653) bank167968 +167969 POINT(38.66221128033558 -122.8575042829108) bank167969 +167970 POINT(38.52877577481996 -122.30700036312571) bank167970 +167971 POINT(36.82575448803775 -122.04869771649753) bank167971 +167972 POINT(37.92820572010228 -121.94371521526077) bank167972 +167973 POINT(38.748382445352384 -121.81748168779913) bank167973 +167974 POINT(37.33649734334003 -122.55143650831305) bank167974 +167975 POINT(37.48641079568577 -122.27035980518376) bank167975 +167976 POINT(38.676076798800274 -121.70873929209175) bank167976 +167977 POINT(37.16162178674597 -122.67906341678906) bank167977 +167978 POINT(37.55267808362496 -122.44617308453117) bank167978 +167979 POINT(38.716277558606315 -122.32443996505647) bank167979 +167980 POINT(37.75473147871858 -121.90473544467838) bank167980 +167981 POINT(38.065312468004194 -123.04856772856762) bank167981 +167982 POINT(37.478430703659704 -122.8832995899589) bank167982 +167983 POINT(37.28961036926401 -122.21387000303518) bank167983 +167984 POINT(37.58040438804762 -123.298369052103) bank167984 +167985 POINT(38.62606947072965 -122.905009771722) bank167985 +167986 POINT(37.01942855361893 -121.84726816609377) bank167986 +167987 POINT(38.093248901480905 -122.04099712149466) bank167987 +167988 POINT(37.73004455345282 -122.10812261997278) bank167988 +167989 POINT(36.78919749339566 -122.34673622780966) bank167989 +167990 POINT(37.70527077524628 -123.36317208959011) bank167990 +167991 POINT(38.333619378859154 -121.60559830660068) bank167991 +167992 POINT(38.25513217328023 -122.63321648267767) bank167992 +167993 POINT(38.313058681736244 -122.3103951375777) bank167993 +167994 POINT(38.57209036468367 -122.49311644108117) bank167994 +167995 POINT(38.526674955867755 -122.91046282943664) bank167995 +167996 POINT(38.31286621941453 -121.54072917039379) bank167996 +167997 POINT(37.973145021462045 -122.40560528989919) bank167997 +167998 POINT(37.73579080168285 -123.19959007813279) bank167998 +167999 POINT(36.92693370314142 -122.27644593623498) bank167999 +168000 POINT(37.94717924249195 -122.39725800282112) bank168000 +168001 POINT(37.3575502361871 -122.75886305766797) bank168001 +168002 POINT(37.27868783292014 -122.85481109786788) bank168002 +168003 POINT(37.03282358348925 -121.78661833028765) bank168003 +168004 POINT(38.24694847437715 -121.57471806649431) bank168004 +168005 POINT(38.617518337168185 -123.21966778442868) bank168005 +168006 POINT(37.91954163920987 -121.89333050497437) bank168006 +168007 POINT(36.810839083806705 -122.4065561019175) bank168007 +168008 POINT(37.13123753190492 -122.47001663029077) bank168008 +168009 POINT(38.7363211874595 -122.9712457867036) bank168009 +168010 POINT(37.7341364490579 -122.89818239444155) bank168010 +168011 POINT(37.5746375549964 -121.47059158360646) bank168011 +168012 POINT(38.087768610202 -121.53426034314168) bank168012 +168013 POINT(38.54265707167753 -123.41373616507532) bank168013 +168014 POINT(36.79370022660501 -123.1213458327872) bank168014 +168015 POINT(38.74540570563683 -122.30435560039803) bank168015 +168016 POINT(37.37200345858711 -121.70912927780834) bank168016 +168017 POINT(37.51979739822494 -122.33511500050683) bank168017 +168018 POINT(38.08504338049289 -122.52679596775752) bank168018 +168019 POINT(37.94178402690838 -122.25670183914642) bank168019 +168020 POINT(37.55443391514059 -122.59435659452053) bank168020 +168021 POINT(38.565099660383176 -122.54351767615401) bank168021 +168022 POINT(38.29918680588835 -122.18477304008967) bank168022 +168023 POINT(37.3352712191986 -122.82214183144075) bank168023 +168024 POINT(38.042993563903146 -121.93526309420231) bank168024 +168025 POINT(37.66897807761021 -122.84128152860109) bank168025 +168026 POINT(36.93408913762458 -122.7187880257566) bank168026 +168027 POINT(36.90972584814476 -121.76069346799828) bank168027 +168028 POINT(36.81103536690472 -123.19033778702658) bank168028 +168029 POINT(38.510941839473524 -122.0151326638761) bank168029 +168030 POINT(38.397659803191935 -121.59644839369551) bank168030 +168031 POINT(37.19048236029816 -121.511270920004) bank168031 +168032 POINT(38.0677009860916 -122.78238772821767) bank168032 +168033 POINT(37.37599986900077 -121.84072944837169) bank168033 +168034 POINT(37.46907980804325 -123.01384939998735) bank168034 +168035 POINT(37.5309542568605 -122.6771165685111) bank168035 +168036 POINT(37.62129248419898 -121.68037166542999) bank168036 +168037 POINT(36.79328337909294 -122.60616434706863) bank168037 +168038 POINT(38.36124760588453 -122.42575569151055) bank168038 +168039 POINT(38.25885171383045 -121.78310474892963) bank168039 +168040 POINT(37.325693215210016 -121.92807820611392) bank168040 +168041 POINT(37.70905430428583 -123.07599640894178) bank168041 +168042 POINT(37.68080607740809 -122.43650060105224) bank168042 +168043 POINT(37.35949460778714 -122.2920161393771) bank168043 +168044 POINT(37.919310102944394 -122.8474035859448) bank168044 +168045 POINT(36.89493761354944 -121.93457617398626) bank168045 +168046 POINT(37.970830031723665 -122.38199526756287) bank168046 +168047 POINT(37.61453615218328 -121.99058251899655) bank168047 +168048 POINT(37.38887065343363 -122.99538480739507) bank168048 +168049 POINT(38.356925172136854 -122.63210420654387) bank168049 +168050 POINT(37.36799283258667 -123.39461156341815) bank168050 +168051 POINT(36.93310290478679 -122.49393404204291) bank168051 +168052 POINT(37.739374217573314 -122.46685938528162) bank168052 +168053 POINT(38.62521431217173 -122.25907683772263) bank168053 +168054 POINT(37.5985210984282 -122.2877976218201) bank168054 +168055 POINT(38.53600836138315 -123.23686491587411) bank168055 +168056 POINT(38.10575974335096 -122.71948961917712) bank168056 +168057 POINT(38.13765857881406 -121.45522014708801) bank168057 +168058 POINT(37.02269503843734 -123.24890665740753) bank168058 +168059 POINT(38.29928991359608 -122.40321247382133) bank168059 +168060 POINT(37.60503745686594 -121.67885009054243) bank168060 +168061 POINT(38.5515772695841 -121.81261792366858) bank168061 +168062 POINT(38.719555711766766 -122.97253158940289) bank168062 +168063 POINT(37.24507544336537 -122.06011200966779) bank168063 +168064 POINT(37.901850278818415 -121.82478079365974) bank168064 +168065 POINT(37.630433648321905 -122.58930020094621) bank168065 +168066 POINT(37.28306896010943 -122.61885480061513) bank168066 +168067 POINT(37.11215114321823 -122.7994449044596) bank168067 +168068 POINT(37.0279995386498 -122.625341708493) bank168068 +168069 POINT(36.996329968455655 -122.89995780867233) bank168069 +168070 POINT(37.128258191388454 -121.78987488147389) bank168070 +168071 POINT(37.39683685405758 -121.85689519128603) bank168071 +168072 POINT(37.18953270717487 -121.8932402135355) bank168072 +168073 POINT(36.787143860701626 -121.88279265651397) bank168073 +168074 POINT(36.86018871854267 -123.35256567096688) bank168074 +168075 POINT(37.136852215041806 -121.8778725294978) bank168075 +168076 POINT(38.7427666846871 -122.12650690338981) bank168076 +168077 POINT(38.765483455633245 -121.54851891148304) bank168077 +168078 POINT(38.17312633201965 -122.55751764556624) bank168078 +168079 POINT(37.24628369496965 -121.96536194732698) bank168079 +168080 POINT(37.13870046690481 -122.42773141321196) bank168080 +168081 POINT(38.057513934464275 -122.4802357214998) bank168081 +168082 POINT(38.56472882592695 -123.17758733734792) bank168082 +168083 POINT(37.040870887068834 -122.11182561494873) bank168083 +168084 POINT(37.914433424316655 -122.15564059193649) bank168084 +168085 POINT(37.279026689076034 -122.2584007128599) bank168085 +168086 POINT(38.41261003624175 -122.42679061291611) bank168086 +168087 POINT(37.015880593039114 -122.96633053332712) bank168087 +168088 POINT(36.98502550821338 -122.70108862497399) bank168088 +168089 POINT(37.311434011902165 -121.71637751053724) bank168089 +168090 POINT(37.46595623757855 -122.22969349173073) bank168090 +168091 POINT(38.27642538822786 -121.420156673358) bank168091 +168092 POINT(37.67707031292351 -123.17802027159912) bank168092 +168093 POINT(37.72187349021764 -121.43639180162825) bank168093 +168094 POINT(37.03616867573054 -122.24219998936911) bank168094 +168095 POINT(38.094043221564704 -121.9550156052591) bank168095 +168096 POINT(37.638578519793604 -122.81353198013136) bank168096 +168097 POINT(37.20713252080699 -122.16874045157277) bank168097 +168098 POINT(38.74100343542404 -121.94255959976287) bank168098 +168099 POINT(38.190583565026216 -122.53540853132178) bank168099 +168100 POINT(37.48263707981646 -122.40421055052502) bank168100 +168101 POINT(38.053911208965644 -122.00268374498548) bank168101 +168102 POINT(36.87224400511268 -121.63508402642276) bank168102 +168103 POINT(37.31779937337083 -121.89328119706008) bank168103 +168104 POINT(38.69206725585343 -122.79099716396428) bank168104 +168105 POINT(38.24629639762702 -123.36004176929377) bank168105 +168106 POINT(37.07343467416154 -122.33095705711355) bank168106 +168107 POINT(38.109863627315804 -121.57971590529797) bank168107 +168108 POINT(37.26757627856599 -121.62190376887688) bank168108 +168109 POINT(38.54731762377834 -121.82606983438016) bank168109 +168110 POINT(37.0241265517418 -122.90081878216176) bank168110 +168111 POINT(38.4462383659407 -122.63248165844693) bank168111 +168112 POINT(37.81004445994093 -121.52758289953498) bank168112 +168113 POINT(38.61177091039875 -122.83962077965798) bank168113 +168114 POINT(37.18841124660321 -122.96209213445634) bank168114 +168115 POINT(37.62987734602665 -121.75885332945303) bank168115 +168116 POINT(37.16378032832724 -123.32937272355386) bank168116 +168117 POINT(37.886995197220024 -122.44591231256165) bank168117 +168118 POINT(37.9919625978975 -123.25891772515541) bank168118 +168119 POINT(36.84195234391909 -122.4264534765496) bank168119 +168120 POINT(38.40348118631535 -123.40897353472957) bank168120 +168121 POINT(37.29828694350205 -122.12074611034107) bank168121 +168122 POINT(38.366693897627535 -123.1677373640208) bank168122 +168123 POINT(37.7489771416864 -122.32001146219008) bank168123 +168124 POINT(37.326581108906616 -122.42012609738919) bank168124 +168125 POINT(37.32605801813389 -121.73644693990884) bank168125 +168126 POINT(37.93022834259649 -123.20728258944396) bank168126 +168127 POINT(37.159558872124585 -122.40276049712175) bank168127 +168128 POINT(37.472161985660705 -122.36813040235386) bank168128 +168129 POINT(37.81333547296251 -121.63969234965177) bank168129 +168130 POINT(37.9849659799082 -121.93274484586969) bank168130 +168131 POINT(37.68552072434472 -122.06222519581169) bank168131 +168132 POINT(37.7025684593025 -122.23815064981416) bank168132 +168133 POINT(37.540367928005395 -122.32685462045652) bank168133 +168134 POINT(37.11503129219524 -122.97182895669194) bank168134 +168135 POINT(37.03110449189032 -121.61133340503291) bank168135 +168136 POINT(37.25143507677496 -122.14567652951433) bank168136 +168137 POINT(37.76007915550381 -122.65366016232214) bank168137 +168138 POINT(38.654379551514 -122.83602367672695) bank168138 +168139 POINT(36.80154926072791 -122.28727437170676) bank168139 +168140 POINT(38.3228853965736 -121.77707212514801) bank168140 +168141 POINT(37.252093350257574 -122.57646973768944) bank168141 +168142 POINT(38.216521508598504 -122.83697069908901) bank168142 +168143 POINT(38.354700979560846 -121.7987411583019) bank168143 +168144 POINT(37.30100926958482 -121.59427859355199) bank168144 +168145 POINT(37.03191347459097 -121.97845303465986) bank168145 +168146 POINT(37.7467315639699 -122.63185702679554) bank168146 +168147 POINT(37.01383068685029 -123.007437460411) bank168147 +168148 POINT(38.77445855613285 -121.67402057020792) bank168148 +168149 POINT(37.01468297250453 -122.29830135050899) bank168149 +168150 POINT(38.52299080144591 -122.41586672943188) bank168150 +168151 POINT(37.48218506365211 -121.54924046380609) bank168151 +168152 POINT(38.53336636167482 -121.46960464524535) bank168152 +168153 POINT(37.373376498313625 -121.51751031969395) bank168153 +168154 POINT(37.038412904389915 -122.78105117033786) bank168154 +168155 POINT(38.74250635671229 -122.24963529029597) bank168155 +168156 POINT(37.97087086200597 -121.64945185938726) bank168156 +168157 POINT(38.35959805092151 -123.35094189007206) bank168157 +168158 POINT(37.85852186363178 -121.81401051488601) bank168158 +168159 POINT(37.16178290706646 -121.60290353204564) bank168159 +168160 POINT(38.40372049481101 -122.08967779668953) bank168160 +168161 POINT(36.83067910211986 -121.95913201929999) bank168161 +168162 POINT(37.592013340802275 -121.59174755809725) bank168162 +168163 POINT(37.60885183218236 -121.76866369128511) bank168163 +168164 POINT(37.87913333341039 -122.45392019639338) bank168164 +168165 POINT(38.634639267990345 -123.18052959335152) bank168165 +168166 POINT(37.56718076589713 -122.98110842091252) bank168166 +168167 POINT(38.7395177912223 -122.75973896854441) bank168167 +168168 POINT(36.964323176235496 -121.58053813901914) bank168168 +168169 POINT(38.27004808169345 -122.12868676629095) bank168169 +168170 POINT(37.964566143407005 -122.72087478561181) bank168170 +168171 POINT(37.49095443064149 -121.65639244840202) bank168171 +168172 POINT(37.59390555148169 -123.30511683214147) bank168172 +168173 POINT(38.37404921330877 -121.55120670674737) bank168173 +168174 POINT(37.72959158857078 -122.56261603896652) bank168174 +168175 POINT(37.68307972540272 -121.64194544767702) bank168175 +168176 POINT(37.68021279066045 -121.84998057413709) bank168176 +168177 POINT(38.286714496691026 -122.45113550489455) bank168177 +168178 POINT(37.315250525882426 -122.42104598500906) bank168178 +168179 POINT(37.11221260859494 -121.85703195359622) bank168179 +168180 POINT(37.89669796020418 -121.80900800882341) bank168180 +168181 POINT(37.88060107088628 -121.79631986165681) bank168181 +168182 POINT(38.02241960262454 -122.38496511847653) bank168182 +168183 POINT(36.84989218779821 -122.52161428091318) bank168183 +168184 POINT(36.888524897651514 -123.24846099050956) bank168184 +168185 POINT(37.513625512546 -121.79741867967395) bank168185 +168186 POINT(37.98822920481914 -121.48160266897355) bank168186 +168187 POINT(37.47605702305358 -123.02237065590629) bank168187 +168188 POINT(37.010975174705784 -122.31854958293064) bank168188 +168189 POINT(38.52198543299649 -121.9837788246831) bank168189 +168190 POINT(37.17120152328767 -122.56178430241772) bank168190 +168191 POINT(37.55047414700618 -122.19093266237876) bank168191 +168192 POINT(37.75905254107002 -123.25892799044595) bank168192 +168193 POINT(38.51620083195658 -122.65473688979338) bank168193 +168194 POINT(37.302715367630285 -123.2113626154382) bank168194 +168195 POINT(37.88743879791765 -123.12552411587478) bank168195 +168196 POINT(37.78236562451234 -123.14547797220911) bank168196 +168197 POINT(37.04430531305526 -122.600725907259) bank168197 +168198 POINT(37.31426306931539 -121.78058008413191) bank168198 +168199 POINT(38.370091766270484 -122.32496804874413) bank168199 +168200 POINT(36.89378964700383 -122.56333659300674) bank168200 +168201 POINT(37.679110241497646 -122.1847097865554) bank168201 +168202 POINT(37.672308447771805 -122.80135546170065) bank168202 +168203 POINT(37.65098829079239 -121.73567801690392) bank168203 +168204 POINT(37.305988953414335 -122.74765689533703) bank168204 +168205 POINT(37.988579162519926 -122.65995054422034) bank168205 +168206 POINT(37.34092938918041 -122.5342892518397) bank168206 +168207 POINT(37.17102900552215 -122.4191955403857) bank168207 +168208 POINT(38.63265759646604 -122.69995860146531) bank168208 +168209 POINT(37.29161329547254 -121.79442753868335) bank168209 +168210 POINT(37.62920383792162 -121.79025044875802) bank168210 +168211 POINT(36.914903343004724 -123.03580941291975) bank168211 +168212 POINT(37.98694943577438 -123.02358923172643) bank168212 +168213 POINT(37.511083058972645 -121.7223838262813) bank168213 +168214 POINT(38.23314005505286 -122.40871432708289) bank168214 +168215 POINT(37.31144923187985 -122.11367779194117) bank168215 +168216 POINT(37.11122263312331 -121.4569712544664) bank168216 +168217 POINT(37.33701019735753 -122.22127125432255) bank168217 +168218 POINT(37.263582222415984 -122.2439125144184) bank168218 +168219 POINT(38.45724081950788 -121.73387754485651) bank168219 +168220 POINT(38.72385893059169 -121.85809006396423) bank168220 +168221 POINT(38.12174027029042 -122.96317919015198) bank168221 +168222 POINT(37.57446094341279 -122.86323172842401) bank168222 +168223 POINT(37.37727176808783 -122.25047347953495) bank168223 +168224 POINT(38.64835716442719 -122.99208985493415) bank168224 +168225 POINT(38.02244367650612 -122.28865812082554) bank168225 +168226 POINT(37.54164594714889 -122.57408977001494) bank168226 +168227 POINT(37.47486578567822 -121.54424393309543) bank168227 +168228 POINT(38.756732892486575 -123.1048905716926) bank168228 +168229 POINT(37.505208198436506 -121.72018930837685) bank168229 +168230 POINT(36.84952796113576 -123.38719762632302) bank168230 +168231 POINT(38.14463992204293 -122.43947945715914) bank168231 +168232 POINT(37.04702552836084 -122.32898590743108) bank168232 +168233 POINT(37.629284148779384 -122.32849978006689) bank168233 +168234 POINT(37.54680806007576 -123.35491882182941) bank168234 +168235 POINT(38.672186773183775 -122.16831861556018) bank168235 +168236 POINT(36.815209554272855 -122.62155526304183) bank168236 +168237 POINT(38.550363892903206 -121.7969997360479) bank168237 +168238 POINT(37.9097592885629 -123.15489631270715) bank168238 +168239 POINT(38.18060050600143 -121.53884337215726) bank168239 +168240 POINT(38.28902354665351 -123.10654009184175) bank168240 +168241 POINT(38.153494513457844 -122.70188738350592) bank168241 +168242 POINT(37.21494800904482 -123.10627990875736) bank168242 +168243 POINT(36.828185692463826 -123.35268225418851) bank168243 +168244 POINT(38.23456494916322 -122.75758486879839) bank168244 +168245 POINT(38.31145534264731 -121.84292252238092) bank168245 +168246 POINT(38.52407490814306 -123.38104122944645) bank168246 +168247 POINT(36.989878339659775 -121.56567135694799) bank168247 +168248 POINT(37.970594752820055 -123.20664519559391) bank168248 +168249 POINT(37.220029627183116 -123.3794999047167) bank168249 +168250 POINT(37.11335987771117 -122.25210711711789) bank168250 +168251 POINT(37.15787673746541 -121.44331530764285) bank168251 +168252 POINT(38.32994779109421 -121.93679877394489) bank168252 +168253 POINT(36.79173804129789 -123.0965902035968) bank168253 +168254 POINT(38.69524210551533 -122.41429319252508) bank168254 +168255 POINT(38.38480200814811 -121.95271377149568) bank168255 +168256 POINT(38.73503968250229 -121.59989853982847) bank168256 +168257 POINT(38.54256731134627 -121.587645877909) bank168257 +168258 POINT(38.29703390010991 -123.21640965249297) bank168258 +168259 POINT(38.10193130767531 -122.35450535219321) bank168259 +168260 POINT(37.99771679048444 -121.648259470935) bank168260 +168261 POINT(37.9285738769891 -123.08835308475575) bank168261 +168262 POINT(38.41954948729433 -121.66770007560048) bank168262 +168263 POINT(37.11359208798595 -121.50900950509916) bank168263 +168264 POINT(37.76612672580311 -121.66990084301491) bank168264 +168265 POINT(38.01005477844974 -123.3451145439374) bank168265 +168266 POINT(38.56843420187262 -123.31963510351798) bank168266 +168267 POINT(38.5529593953055 -121.46126769864681) bank168267 +168268 POINT(38.7244901442126 -123.3664164326058) bank168268 +168269 POINT(37.48210554351105 -122.89407419244478) bank168269 +168270 POINT(37.438606848545085 -122.58313272379667) bank168270 +168271 POINT(38.15658127370489 -123.3553181259731) bank168271 +168272 POINT(37.21900328501007 -123.08277219454172) bank168272 +168273 POINT(37.43368298121178 -122.35882405884313) bank168273 +168274 POINT(37.6210111504883 -122.9384602646358) bank168274 +168275 POINT(38.25555887574306 -123.13738185375682) bank168275 +168276 POINT(37.94944427044335 -121.72657023374478) bank168276 +168277 POINT(36.793133404186435 -122.67946434322874) bank168277 +168278 POINT(37.614725582904775 -122.26525976503824) bank168278 +168279 POINT(36.8579364028141 -122.78629342675765) bank168279 +168280 POINT(38.26747564802367 -123.35825055192998) bank168280 +168281 POINT(37.8449216321921 -123.24952303815337) bank168281 +168282 POINT(38.72207440689173 -122.57355540667844) bank168282 +168283 POINT(38.72760580128259 -121.58828247108792) bank168283 +168284 POINT(38.1568863367247 -121.76096534306532) bank168284 +168285 POINT(37.26584243718383 -123.21435322844664) bank168285 +168286 POINT(37.01099432663075 -123.03249428779087) bank168286 +168287 POINT(38.087402953742306 -122.47190821765963) bank168287 +168288 POINT(38.001930154646494 -121.62217882998623) bank168288 +168289 POINT(38.26604810797773 -122.70032113168213) bank168289 +168290 POINT(38.68649864078835 -123.1538190391493) bank168290 +168291 POINT(36.865396727470134 -122.63860031865786) bank168291 +168292 POINT(38.7113203949071 -123.06624242643326) bank168292 +168293 POINT(37.460722807259586 -122.95767699426744) bank168293 +168294 POINT(38.63267989133582 -122.64992450746256) bank168294 +168295 POINT(36.86683530939089 -123.16819530302577) bank168295 +168296 POINT(37.44321451318371 -122.56193113389588) bank168296 +168297 POINT(37.939076085163 -121.5675685664869) bank168297 +168298 POINT(37.87218560315456 -122.81993090828419) bank168298 +168299 POINT(37.14024934731148 -122.3338742653607) bank168299 +168300 POINT(38.492575138436166 -122.59944982891876) bank168300 +168301 POINT(38.48312600548679 -122.63261236953372) bank168301 +168302 POINT(37.635611821581826 -122.71328269111301) bank168302 +168303 POINT(38.16638804566546 -121.82378282637862) bank168303 +168304 POINT(38.1552285904754 -121.62504356818795) bank168304 +168305 POINT(37.43356833567487 -121.73782587293185) bank168305 +168306 POINT(38.31742275923593 -122.3084401255233) bank168306 +168307 POINT(38.56701158646703 -122.88786321247244) bank168307 +168308 POINT(37.00393913989963 -122.1101812909997) bank168308 +168309 POINT(38.578376272204245 -121.52380307459707) bank168309 +168310 POINT(38.35972748888644 -121.63063948666272) bank168310 +168311 POINT(37.15462303938023 -122.73343865527987) bank168311 +168312 POINT(37.806598461214506 -121.654020047701) bank168312 +168313 POINT(37.84507535629071 -122.97555018821106) bank168313 +168314 POINT(37.77369848178867 -123.07125214309309) bank168314 +168315 POINT(37.616972995143605 -121.65036805455308) bank168315 +168316 POINT(38.68879033033842 -123.3725979922436) bank168316 +168317 POINT(37.32988931563166 -122.05402183410234) bank168317 +168318 POINT(38.39244815605892 -122.41154528189483) bank168318 +168319 POINT(38.31289319427724 -121.87175296393207) bank168319 +168320 POINT(38.67798952635213 -123.1261036348263) bank168320 +168321 POINT(38.02338920056147 -122.3035756371499) bank168321 +168322 POINT(37.17736327874299 -121.75242674123388) bank168322 +168323 POINT(38.37776619847441 -123.36117711806406) bank168323 +168324 POINT(37.49886907138265 -123.31851317647387) bank168324 +168325 POINT(37.3933469811135 -122.36605811672298) bank168325 +168326 POINT(37.39420100809182 -121.45218431038187) bank168326 +168327 POINT(36.991668868547954 -121.51630861702228) bank168327 +168328 POINT(38.36024064140074 -123.00620472272546) bank168328 +168329 POINT(38.293807411624016 -123.07317854882334) bank168329 +168330 POINT(38.71828926260907 -122.54911748835275) bank168330 +168331 POINT(38.215988464480134 -122.97215404654851) bank168331 +168332 POINT(37.8218699295098 -121.59045941677624) bank168332 +168333 POINT(37.67673781075464 -123.2775498313788) bank168333 +168334 POINT(37.320568966862325 -123.02690050436634) bank168334 +168335 POINT(37.191575567250446 -123.20169124002919) bank168335 +168336 POINT(38.635782825866634 -123.38543883393613) bank168336 +168337 POINT(37.798836944333196 -121.47170884112336) bank168337 +168338 POINT(38.646095447948184 -122.26649461241676) bank168338 +168339 POINT(37.92192488343579 -122.31315870752691) bank168339 +168340 POINT(38.408046875048356 -122.38770770012073) bank168340 +168341 POINT(37.07529899559934 -122.76719375836532) bank168341 +168342 POINT(37.32965687768602 -122.53960646315001) bank168342 +168343 POINT(38.664428651521405 -122.10781034775788) bank168343 +168344 POINT(36.906524644680324 -121.99121395636168) bank168344 +168345 POINT(37.18581701256263 -122.708432459873) bank168345 +168346 POINT(37.24576576026373 -123.1708654375329) bank168346 +168347 POINT(37.62978619585638 -122.21497844577861) bank168347 +168348 POINT(38.04219157174706 -122.46914608003172) bank168348 +168349 POINT(37.927997715596184 -121.70096615481027) bank168349 +168350 POINT(38.57788901315927 -122.31178927971033) bank168350 +168351 POINT(37.53074281400169 -122.27779072437197) bank168351 +168352 POINT(38.08154080567993 -122.9061913965224) bank168352 +168353 POINT(38.504215540286125 -123.23789468670219) bank168353 +168354 POINT(37.59181394392579 -122.83443136215074) bank168354 +168355 POINT(38.55441974610051 -121.71853032938574) bank168355 +168356 POINT(38.277966729371656 -122.59574972348291) bank168356 +168357 POINT(37.81889828671315 -122.42749819630524) bank168357 +168358 POINT(37.163484374192905 -122.24869931244363) bank168358 +168359 POINT(37.7539106492255 -121.89403854345967) bank168359 +168360 POINT(38.347373650610905 -122.57952433126695) bank168360 +168361 POINT(37.558271189020545 -122.78294428537473) bank168361 +168362 POINT(37.41654125538534 -122.87814249935477) bank168362 +168363 POINT(37.40395205294781 -122.64702106575649) bank168363 +168364 POINT(37.488777692629924 -121.99974289709203) bank168364 +168365 POINT(37.83159534616082 -122.3912677957571) bank168365 +168366 POINT(37.18793471712624 -122.67410410588128) bank168366 +168367 POINT(38.11844077581638 -123.18272498690781) bank168367 +168368 POINT(38.595852457449475 -121.61592872579803) bank168368 +168369 POINT(38.63146111776704 -123.26720703408489) bank168369 +168370 POINT(37.185571575228835 -122.02642750939945) bank168370 +168371 POINT(37.173657959607795 -123.05044167866356) bank168371 +168372 POINT(37.29092949497002 -123.12647534193061) bank168372 +168373 POINT(37.345380938988 -123.1784433535887) bank168373 +168374 POINT(37.32980645532778 -122.24487107573111) bank168374 +168375 POINT(36.90945665198675 -122.12315977064266) bank168375 +168376 POINT(37.36511087560035 -123.08788754244304) bank168376 +168377 POINT(38.7456194071465 -122.4661858334124) bank168377 +168378 POINT(36.917493263090726 -122.7131564226579) bank168378 +168379 POINT(36.798834646160465 -122.87139216492875) bank168379 +168380 POINT(38.22595059806092 -121.90504888189923) bank168380 +168381 POINT(37.010614179097836 -122.41926229083437) bank168381 +168382 POINT(36.86570168090438 -122.81057308920668) bank168382 +168383 POINT(37.69651301257164 -122.34050855648424) bank168383 +168384 POINT(38.75254348351445 -122.50660316423884) bank168384 +168385 POINT(38.53670287542103 -122.61460253281938) bank168385 +168386 POINT(37.61461737514989 -122.94402696395056) bank168386 +168387 POINT(36.82123971260244 -121.6387018362477) bank168387 +168388 POINT(37.85941674977208 -122.86431430575045) bank168388 +168389 POINT(37.666760526917805 -122.92762080874792) bank168389 +168390 POINT(38.26671069497376 -122.98139830428005) bank168390 +168391 POINT(38.72922833004101 -122.35583091123245) bank168391 +168392 POINT(36.91980737180891 -122.43055235952252) bank168392 +168393 POINT(38.355122107815035 -122.79304729249345) bank168393 +168394 POINT(37.10403210342465 -122.08792717841574) bank168394 +168395 POINT(37.17262728950464 -122.34919261752663) bank168395 +168396 POINT(38.094767985970826 -123.01508224261082) bank168396 +168397 POINT(38.318813211222846 -123.06992493745315) bank168397 +168398 POINT(36.96025874961932 -122.72448800229516) bank168398 +168399 POINT(37.89739939746533 -122.67052706292783) bank168399 +168400 POINT(37.95877427103083 -121.65712350348592) bank168400 +168401 POINT(36.7754579074096 -122.57540381843914) bank168401 +168402 POINT(37.35687622145008 -122.21464139218115) bank168402 +168403 POINT(37.690171213039406 -122.65887479663759) bank168403 +168404 POINT(38.16329315824214 -122.22504739308118) bank168404 +168405 POINT(37.67565624328164 -121.94310866240009) bank168405 +168406 POINT(37.90018182167127 -122.19664950967768) bank168406 +168407 POINT(37.1149972470272 -123.02567611230525) bank168407 +168408 POINT(38.732130266729115 -122.15656035434753) bank168408 +168409 POINT(36.95224549897725 -122.85735590255403) bank168409 +168410 POINT(38.07477202484185 -122.00185234584285) bank168410 +168411 POINT(36.82216670430066 -122.2402535029003) bank168411 +168412 POINT(38.00327477929315 -122.54480527293302) bank168412 +168413 POINT(37.00637035926232 -121.63812185065152) bank168413 +168414 POINT(38.581396350268264 -122.65701801912029) bank168414 +168415 POINT(37.744779834800575 -122.77666991497202) bank168415 +168416 POINT(38.470678784533106 -121.54453179576657) bank168416 +168417 POINT(38.15707129660556 -122.89108776007926) bank168417 +168418 POINT(36.89878916891294 -123.21986301361329) bank168418 +168419 POINT(36.954003829114015 -122.78544690562126) bank168419 +168420 POINT(37.46450871540889 -122.7673929264741) bank168420 +168421 POINT(37.31989781837785 -122.76994406526646) bank168421 +168422 POINT(37.357981318217654 -121.45345687421393) bank168422 +168423 POINT(38.38172836998767 -122.5351721923404) bank168423 +168424 POINT(37.61163527845597 -123.33947688292614) bank168424 +168425 POINT(37.50695033245088 -122.98409234883056) bank168425 +168426 POINT(38.69886949430427 -122.8611012666237) bank168426 +168427 POINT(37.95948011624527 -123.02120572321714) bank168427 +168428 POINT(38.6864029765432 -121.8548342765308) bank168428 +168429 POINT(37.849138353885465 -122.11796416848762) bank168429 +168430 POINT(37.74885123434498 -122.25783111854219) bank168430 +168431 POINT(38.10851410937153 -121.96514288077212) bank168431 +168432 POINT(37.49451722733094 -122.4158147401355) bank168432 +168433 POINT(38.18145741437884 -122.20834726661899) bank168433 +168434 POINT(38.27426752708565 -122.41982864852018) bank168434 +168435 POINT(37.35396422486966 -123.41710678055559) bank168435 +168436 POINT(38.23770517373262 -123.36106380157548) bank168436 +168437 POINT(37.729202113555985 -121.59108825145603) bank168437 +168438 POINT(36.85925720003775 -122.57678251075642) bank168438 +168439 POINT(37.66040924248795 -123.17395648230611) bank168439 +168440 POINT(37.97221660744641 -121.77679290673422) bank168440 +168441 POINT(38.00127561463768 -121.60262976202678) bank168441 +168442 POINT(37.11779431118659 -123.1237024105621) bank168442 +168443 POINT(38.563822285381924 -123.02940170699353) bank168443 +168444 POINT(37.67092303890167 -122.75438244374328) bank168444 +168445 POINT(37.1912953966675 -121.7820153212758) bank168445 +168446 POINT(38.70417374495027 -122.6047052780424) bank168446 +168447 POINT(37.847499295429934 -122.61840105759978) bank168447 +168448 POINT(38.27384148583951 -122.3198752894319) bank168448 +168449 POINT(38.442629580926976 -121.96136672015959) bank168449 +168450 POINT(37.28527796001143 -122.8598303105523) bank168450 +168451 POINT(38.234101884502216 -122.30762932268058) bank168451 +168452 POINT(37.60602922659619 -121.89390894227219) bank168452 +168453 POINT(36.98020330832501 -121.52313784668893) bank168453 +168454 POINT(37.239494480957916 -121.70244452067809) bank168454 +168455 POINT(38.30967336937732 -122.66800666550826) bank168455 +168456 POINT(38.20950928813255 -122.6624950121955) bank168456 +168457 POINT(37.32422109352553 -122.14300144795133) bank168457 +168458 POINT(36.9552960116011 -122.20733002852839) bank168458 +168459 POINT(37.113512441168446 -123.13970021690731) bank168459 +168460 POINT(38.11781657029691 -121.53975047498241) bank168460 +168461 POINT(38.77331163419072 -121.53367020141195) bank168461 +168462 POINT(38.12475072177404 -122.59887819054937) bank168462 +168463 POINT(37.511556282690414 -122.5560078204087) bank168463 +168464 POINT(38.714917840138384 -122.98628577880876) bank168464 +168465 POINT(38.534476560263506 -122.84804863761465) bank168465 +168466 POINT(37.49394081751964 -122.87843025344094) bank168466 +168467 POINT(38.755585052589126 -122.03935770205946) bank168467 +168468 POINT(38.43911764564815 -122.2522006820893) bank168468 +168469 POINT(37.326885109069956 -122.10442232658556) bank168469 +168470 POINT(37.659248837174474 -121.60073565314046) bank168470 +168471 POINT(38.34268389732308 -122.72286793910027) bank168471 +168472 POINT(38.39565819268348 -121.4295626939878) bank168472 +168473 POINT(37.98956832447106 -122.8701006275094) bank168473 +168474 POINT(38.046386972998675 -123.41370889586308) bank168474 +168475 POINT(36.97230895759047 -122.60639979192413) bank168475 +168476 POINT(38.587612522457405 -123.16651206771743) bank168476 +168477 POINT(37.405556436666075 -123.02855365361265) bank168477 +168478 POINT(37.703857238287675 -122.36291262644006) bank168478 +168479 POINT(38.337100682117374 -122.19152576314478) bank168479 +168480 POINT(38.74727764056738 -121.65814205294338) bank168480 +168481 POINT(37.140631384970455 -123.1349471356392) bank168481 +168482 POINT(37.99509744395412 -122.7558175778562) bank168482 +168483 POINT(37.3555124766925 -122.72945223801297) bank168483 +168484 POINT(37.5118219882182 -123.00332428964731) bank168484 +168485 POINT(38.1662000556221 -123.11984567649026) bank168485 +168486 POINT(37.670759222403184 -123.38771277427837) bank168486 +168487 POINT(38.47829636193467 -122.53018250640703) bank168487 +168488 POINT(37.208163198592565 -122.20633902030347) bank168488 +168489 POINT(36.99592413354213 -123.15105747047619) bank168489 +168490 POINT(36.88868543471377 -123.07894611639098) bank168490 +168491 POINT(37.124749142778654 -122.57865830308202) bank168491 +168492 POINT(37.01319459354924 -121.93590893021799) bank168492 +168493 POINT(38.73176907299679 -122.50114553065326) bank168493 +168494 POINT(36.855310107620426 -122.25276327258638) bank168494 +168495 POINT(38.32664077598481 -122.73940978855443) bank168495 +168496 POINT(37.06473390923166 -122.60366906466723) bank168496 +168497 POINT(38.21433296552165 -121.75928363785262) bank168497 +168498 POINT(37.76411175253463 -122.16631873200603) bank168498 +168499 POINT(38.36219001939962 -122.14876942709573) bank168499 +168500 POINT(37.93836762280924 -121.60012318799829) bank168500 +168501 POINT(38.48782762077033 -121.94334118861764) bank168501 +168502 POINT(38.710988365364685 -123.17515603935367) bank168502 +168503 POINT(36.828621477699144 -122.10895776471779) bank168503 +168504 POINT(37.35472503666637 -123.1982471686809) bank168504 +168505 POINT(37.21294611012354 -122.13214721768178) bank168505 +168506 POINT(36.85489348595792 -123.25949918130736) bank168506 +168507 POINT(37.94767778180779 -123.17869344856263) bank168507 +168508 POINT(37.70679977568082 -122.50257499220315) bank168508 +168509 POINT(37.86815500676486 -121.60040559667256) bank168509 +168510 POINT(37.13381323404358 -121.6469934208965) bank168510 +168511 POINT(36.89497451559108 -122.76431927080807) bank168511 +168512 POINT(38.012625102057456 -122.97619450284306) bank168512 +168513 POINT(37.81918015495657 -123.15971596845017) bank168513 +168514 POINT(38.33508831962025 -121.4674470500883) bank168514 +168515 POINT(38.16601822884519 -121.98732713467038) bank168515 +168516 POINT(37.877018221100506 -121.73241087612368) bank168516 +168517 POINT(38.17082397663372 -122.59081516433426) bank168517 +168518 POINT(37.29062859905333 -122.34683578482121) bank168518 +168519 POINT(38.29265972622896 -122.47168363473502) bank168519 +168520 POINT(38.44157675395531 -122.26706102092189) bank168520 +168521 POINT(38.66288487132081 -122.30233642813879) bank168521 +168522 POINT(37.046533938028595 -122.80865476929743) bank168522 +168523 POINT(37.824107412160515 -122.83480410905447) bank168523 +168524 POINT(37.07117110308796 -123.13280930428819) bank168524 +168525 POINT(36.965026866843175 -122.71483342776592) bank168525 +168526 POINT(38.22605916636027 -122.60075886559687) bank168526 +168527 POINT(37.25662497609846 -123.12249085107324) bank168527 +168528 POINT(38.3566642179716 -122.53699140751473) bank168528 +168529 POINT(37.74145100971006 -122.14755481005083) bank168529 +168530 POINT(38.4864462188259 -122.66112329992184) bank168530 +168531 POINT(38.59650338071339 -121.92496970137286) bank168531 +168532 POINT(38.53935876410025 -122.53218199420509) bank168532 +168533 POINT(38.21454254206671 -121.61718683552873) bank168533 +168534 POINT(37.35802999117548 -121.94705944921593) bank168534 +168535 POINT(38.35764052973069 -123.23761049082695) bank168535 +168536 POINT(38.382293789914456 -123.21015422333747) bank168536 +168537 POINT(38.30077275628135 -121.87842415185753) bank168537 +168538 POINT(38.24410863986259 -121.83194791557057) bank168538 +168539 POINT(37.031222292570405 -121.82252787459815) bank168539 +168540 POINT(38.63796050501212 -122.3850170464253) bank168540 +168541 POINT(37.519154741135694 -123.30651498523473) bank168541 +168542 POINT(36.8893198396332 -123.03342388832708) bank168542 +168543 POINT(38.61639652022968 -121.90076875722323) bank168543 +168544 POINT(37.33068874251776 -121.9832474257973) bank168544 +168545 POINT(37.40347312299038 -123.12962689338377) bank168545 +168546 POINT(37.65746666014766 -122.30876311754534) bank168546 +168547 POINT(37.46882976686224 -121.75255593024725) bank168547 +168548 POINT(38.51249460070139 -121.70223796058544) bank168548 +168549 POINT(38.62924485847792 -122.25389917644091) bank168549 +168550 POINT(36.87315470910233 -123.18022919814878) bank168550 +168551 POINT(37.80627481140127 -121.67780026949762) bank168551 +168552 POINT(37.56054671303156 -121.58491674663436) bank168552 +168553 POINT(37.766452785801086 -121.61942769498017) bank168553 +168554 POINT(38.71872140450037 -123.24412174431643) bank168554 +168555 POINT(37.35990120511541 -121.85757627281305) bank168555 +168556 POINT(37.39841152439308 -122.42222395966412) bank168556 +168557 POINT(37.29835261926867 -122.6112025908931) bank168557 +168558 POINT(37.48318158606993 -122.88210370408108) bank168558 +168559 POINT(38.356676344434845 -122.09373774530141) bank168559 +168560 POINT(37.995944282545665 -122.21088403957745) bank168560 +168561 POINT(37.614650419204345 -121.73508990715857) bank168561 +168562 POINT(38.522825588009916 -122.4104154513927) bank168562 +168563 POINT(38.63113338645932 -122.8924184877821) bank168563 +168564 POINT(38.225077827565336 -122.67679431168476) bank168564 +168565 POINT(37.34609866906879 -121.91415255795562) bank168565 +168566 POINT(38.50884076150811 -121.95807860080109) bank168566 +168567 POINT(37.80814075236108 -122.33534413540116) bank168567 +168568 POINT(38.61436697482145 -121.8190778862049) bank168568 +168569 POINT(37.00672975835674 -122.87235201466282) bank168569 +168570 POINT(37.9497447325635 -121.94307906923837) bank168570 +168571 POINT(38.138708428738774 -122.64441690147031) bank168571 +168572 POINT(37.13815848259307 -122.16146098454078) bank168572 +168573 POINT(37.45164558137935 -122.29701359903795) bank168573 +168574 POINT(37.541601411771566 -122.18927679296644) bank168574 +168575 POINT(38.147633671551986 -121.97835936636744) bank168575 +168576 POINT(37.05398743892316 -123.18771259261902) bank168576 +168577 POINT(38.540115144772166 -121.64619385997058) bank168577 +168578 POINT(36.8943028836514 -122.24063246728369) bank168578 +168579 POINT(38.65880763982675 -121.61451627164357) bank168579 +168580 POINT(38.76941707583809 -122.19843935128374) bank168580 +168581 POINT(38.53209801078183 -122.22543026602548) bank168581 +168582 POINT(38.67903345956613 -122.49016984543009) bank168582 +168583 POINT(37.01495566205582 -122.46433695529896) bank168583 +168584 POINT(38.193962480109974 -123.364840239689) bank168584 +168585 POINT(37.19276993098907 -122.93477686059695) bank168585 +168586 POINT(37.4666437178826 -121.86598293917187) bank168586 +168587 POINT(36.82037711079651 -122.96187546731242) bank168587 +168588 POINT(36.83518311829995 -121.98322391288201) bank168588 +168589 POINT(38.38696276063751 -122.31706838955755) bank168589 +168590 POINT(37.899273306973804 -122.36127674349194) bank168590 +168591 POINT(38.74037769736964 -122.30475221935669) bank168591 +168592 POINT(37.574658032842926 -122.90584116995558) bank168592 +168593 POINT(37.51165298484461 -123.2450607965255) bank168593 +168594 POINT(37.52367600957697 -121.56489540777477) bank168594 +168595 POINT(37.86302215250914 -122.4555890528353) bank168595 +168596 POINT(37.20642255469103 -121.96135658916894) bank168596 +168597 POINT(37.21186958255127 -121.75779927831859) bank168597 +168598 POINT(37.41721833282073 -122.05417016828982) bank168598 +168599 POINT(37.98934593686673 -122.02374808768938) bank168599 +168600 POINT(37.71559911382555 -122.75426496628208) bank168600 +168601 POINT(38.434946647398945 -123.07308654937134) bank168601 +168602 POINT(37.41003564255652 -121.90027312454957) bank168602 +168603 POINT(36.881935379410805 -123.19747862876928) bank168603 +168604 POINT(37.370915062952484 -121.5035394703042) bank168604 +168605 POINT(38.41524680275422 -122.7383981953322) bank168605 +168606 POINT(38.50035849443246 -122.7450937453414) bank168606 +168607 POINT(38.48219793574337 -122.37487411574547) bank168607 +168608 POINT(36.85166218975322 -123.08205990711994) bank168608 +168609 POINT(36.894378944819266 -121.9620774039579) bank168609 +168610 POINT(38.62957187022617 -121.64218842352012) bank168610 +168611 POINT(38.45513202824963 -122.40927861957074) bank168611 +168612 POINT(38.1790122656214 -123.12283989743086) bank168612 +168613 POINT(38.09037919969091 -122.69153827030873) bank168613 +168614 POINT(37.36279016351327 -121.70944247764177) bank168614 +168615 POINT(37.94250859374358 -122.32998869277552) bank168615 +168616 POINT(38.362364048512205 -123.03261032227309) bank168616 +168617 POINT(38.54404772830254 -122.57494209333838) bank168617 +168618 POINT(38.29310180573879 -123.08134624683856) bank168618 +168619 POINT(38.046823297874724 -122.55251168786711) bank168619 +168620 POINT(36.788716524871845 -122.71648155792074) bank168620 +168621 POINT(38.202115566604334 -122.80620102224542) bank168621 +168622 POINT(37.93417799341256 -123.1938586052765) bank168622 +168623 POINT(38.06607271874396 -122.08670234186894) bank168623 +168624 POINT(37.411653914043576 -122.17247297015628) bank168624 +168625 POINT(38.49612252063058 -122.9587597716265) bank168625 +168626 POINT(37.623738795079824 -123.33248225852326) bank168626 +168627 POINT(37.409686795132664 -122.06898151340603) bank168627 +168628 POINT(37.27630718210269 -122.31442530533798) bank168628 +168629 POINT(37.70997207717987 -121.95012953903799) bank168629 +168630 POINT(37.17740920279878 -123.14489010883332) bank168630 +168631 POINT(38.024053013470215 -122.67005147226469) bank168631 +168632 POINT(38.695141557517346 -123.20693281147334) bank168632 +168633 POINT(37.711066445195485 -122.69232019165705) bank168633 +168634 POINT(37.416323083735556 -122.64693487716264) bank168634 +168635 POINT(36.8150273792792 -121.86059754780715) bank168635 +168636 POINT(37.68181722648106 -122.64087881229108) bank168636 +168637 POINT(38.07389777589234 -122.32020210406765) bank168637 +168638 POINT(37.64896646888127 -122.29132678253437) bank168638 +168639 POINT(38.316283870996166 -121.79108005124147) bank168639 +168640 POINT(36.92147083486803 -122.79187859921375) bank168640 +168641 POINT(38.04590393011456 -122.49787643852514) bank168641 +168642 POINT(38.41868225817677 -122.07713254619294) bank168642 +168643 POINT(37.32535312853431 -121.86059357500439) bank168643 +168644 POINT(38.716654410953836 -123.2405678359495) bank168644 +168645 POINT(37.22027647192282 -122.00187183735723) bank168645 +168646 POINT(37.95560192860842 -121.87040163024305) bank168646 +168647 POINT(37.955046737136556 -121.6860781633973) bank168647 +168648 POINT(37.036767574940576 -123.11821186764739) bank168648 +168649 POINT(37.423140664683174 -122.00153713971564) bank168649 +168650 POINT(37.157317950861795 -121.65647655094736) bank168650 +168651 POINT(38.61975590749988 -121.78580381406933) bank168651 +168652 POINT(37.56220869555415 -121.78409949131135) bank168652 +168653 POINT(37.22919820994612 -122.91951001284399) bank168653 +168654 POINT(37.533521928239466 -123.06523849087802) bank168654 +168655 POINT(38.38346555963903 -122.57533487525635) bank168655 +168656 POINT(38.275493988445476 -123.3253705053841) bank168656 +168657 POINT(36.90689092497031 -121.53511733337722) bank168657 +168658 POINT(38.40469116605061 -122.57101412466805) bank168658 +168659 POINT(38.653535930127234 -121.97133694578922) bank168659 +168660 POINT(38.43025709859614 -122.41533563612518) bank168660 +168661 POINT(37.29240785802622 -123.21947601990877) bank168661 +168662 POINT(37.66340433694626 -122.35158310600266) bank168662 +168663 POINT(38.6449688557284 -123.34299657327219) bank168663 +168664 POINT(38.44292675702542 -121.85238728503353) bank168664 +168665 POINT(36.99798188391189 -122.01277418527275) bank168665 +168666 POINT(36.81992646570387 -122.82780229026542) bank168666 +168667 POINT(37.1045241149556 -122.40193686658937) bank168667 +168668 POINT(38.185716854080745 -121.45702030331469) bank168668 +168669 POINT(37.05791399054353 -121.7768186239985) bank168669 +168670 POINT(37.64823933180969 -123.33621237799902) bank168670 +168671 POINT(37.12763339696183 -122.95614133955179) bank168671 +168672 POINT(37.75934214237295 -122.93980243688709) bank168672 +168673 POINT(37.75734965116859 -121.7321823705566) bank168673 +168674 POINT(37.61467181920867 -123.25122251683321) bank168674 +168675 POINT(38.47701524476269 -122.09745376943403) bank168675 +168676 POINT(37.97613020490504 -121.4382090244671) bank168676 +168677 POINT(37.41903640469335 -121.92689300312843) bank168677 +168678 POINT(37.726303312876745 -122.3980845086144) bank168678 +168679 POINT(36.90408942412789 -123.00673889415606) bank168679 +168680 POINT(38.55431260477116 -122.4511801826257) bank168680 +168681 POINT(38.46347861219108 -121.45027609127683) bank168681 +168682 POINT(37.945722085648136 -121.59431554193336) bank168682 +168683 POINT(37.230090384993765 -123.2153098324235) bank168683 +168684 POINT(38.2804433413196 -123.16950372993263) bank168684 +168685 POINT(37.781694278727514 -121.75506329471575) bank168685 +168686 POINT(38.15694974756401 -121.95922076749403) bank168686 +168687 POINT(37.77251617212798 -123.28653328157064) bank168687 +168688 POINT(37.0956349333644 -123.26257032420783) bank168688 +168689 POINT(38.077417862022706 -123.30786048919038) bank168689 +168690 POINT(37.54595661846325 -122.01255947299093) bank168690 +168691 POINT(38.07852356447604 -122.43245838958225) bank168691 +168692 POINT(38.2886949834653 -122.9706419075012) bank168692 +168693 POINT(38.04357170044428 -122.66772950147836) bank168693 +168694 POINT(38.5870242690327 -122.52024088493779) bank168694 +168695 POINT(37.30735938821309 -121.71462814642999) bank168695 +168696 POINT(37.07879107015784 -122.18106112281865) bank168696 +168697 POINT(36.86612440501834 -122.45234829843454) bank168697 +168698 POINT(37.50515611361564 -122.70609500803144) bank168698 +168699 POINT(38.364513284911354 -121.8077225638772) bank168699 +168700 POINT(38.111265078359835 -121.83056925004152) bank168700 +168701 POINT(37.364783902967915 -122.99803203045136) bank168701 +168702 POINT(38.33990353383351 -122.97295927005496) bank168702 +168703 POINT(37.64300591790555 -123.19258452738637) bank168703 +168704 POINT(38.095104659565685 -122.63488445420924) bank168704 +168705 POINT(38.14048134559642 -121.95206108839257) bank168705 +168706 POINT(37.72508570454625 -122.99647628206928) bank168706 +168707 POINT(38.01340045679519 -123.20926664947311) bank168707 +168708 POINT(36.895968613629535 -122.4349544315091) bank168708 +168709 POINT(37.41906875926274 -122.79385586681778) bank168709 +168710 POINT(38.50675747142974 -122.84953541143425) bank168710 +168711 POINT(37.57309278226225 -121.53118528572558) bank168711 +168712 POINT(37.63241207070113 -122.94490337860239) bank168712 +168713 POINT(37.61903422617209 -123.1046778218407) bank168713 +168714 POINT(38.191340679564036 -121.89953735220031) bank168714 +168715 POINT(36.87159843765702 -122.6117925609173) bank168715 +168716 POINT(36.86101789780222 -122.83527216532201) bank168716 +168717 POINT(38.62344139557283 -121.82056504145139) bank168717 +168718 POINT(37.56451431600177 -121.96725202983691) bank168718 +168719 POINT(37.30966373105845 -121.45738124559112) bank168719 +168720 POINT(37.125906236931684 -121.82550745850584) bank168720 +168721 POINT(37.690493599062265 -122.61964027660134) bank168721 +168722 POINT(37.56506540512865 -122.13456355160315) bank168722 +168723 POINT(37.49469477455082 -122.78193261893391) bank168723 +168724 POINT(38.40684601714484 -122.90781515129115) bank168724 +168725 POINT(37.051980990795784 -123.06821667002542) bank168725 +168726 POINT(38.49424312953585 -122.90444078182082) bank168726 +168727 POINT(37.61025606803682 -122.5672137683628) bank168727 +168728 POINT(37.08971039965672 -121.71566144009698) bank168728 +168729 POINT(38.39596263883117 -122.71725821187442) bank168729 +168730 POINT(38.52521795635917 -122.83724853591211) bank168730 +168731 POINT(38.240354346928754 -123.31290066312184) bank168731 +168732 POINT(37.90569982535598 -122.51009537361483) bank168732 +168733 POINT(37.29565705689186 -122.39205296402238) bank168733 +168734 POINT(38.03930789042932 -122.65754706963651) bank168734 +168735 POINT(38.4591172536126 -121.95312445886242) bank168735 +168736 POINT(38.30367244108696 -123.29555475373769) bank168736 +168737 POINT(38.57456015108662 -122.51250810470825) bank168737 +168738 POINT(37.96074565943365 -121.57683238483058) bank168738 +168739 POINT(37.07918538231791 -121.92547291893604) bank168739 +168740 POINT(37.66931486629356 -122.49416474598279) bank168740 +168741 POINT(37.31939119882602 -122.78577684009176) bank168741 +168742 POINT(37.68072997230906 -122.49821917750687) bank168742 +168743 POINT(37.350367511553486 -121.45484548929484) bank168743 +168744 POINT(37.96820131524379 -121.50398267082497) bank168744 +168745 POINT(36.84252123893516 -122.68099454499132) bank168745 +168746 POINT(36.84834617254522 -122.76629518276215) bank168746 +168747 POINT(38.08456396950591 -122.12137242537814) bank168747 +168748 POINT(36.82203629978859 -122.44932834027937) bank168748 +168749 POINT(38.750853549049 -121.5164779055052) bank168749 +168750 POINT(38.32444838738371 -123.0606812896423) bank168750 +168751 POINT(36.81302750120541 -123.31652309658908) bank168751 +168752 POINT(38.241345990089016 -121.68808485246542) bank168752 +168753 POINT(38.27311908532107 -121.62755889491079) bank168753 +168754 POINT(37.247337285287124 -121.84226430056933) bank168754 +168755 POINT(38.632291762603856 -122.96611789027651) bank168755 +168756 POINT(36.90352632974272 -121.54683998411988) bank168756 +168757 POINT(37.40519474076443 -122.56696543562909) bank168757 +168758 POINT(37.57523354718522 -123.09277377788652) bank168758 +168759 POINT(37.28776220875797 -123.23222501502674) bank168759 +168760 POINT(37.788826278424764 -123.38381857154948) bank168760 +168761 POINT(37.96849655400938 -122.2382985319202) bank168761 +168762 POINT(38.31220172118786 -121.70362123011513) bank168762 +168763 POINT(37.93004050672697 -123.2078292129584) bank168763 +168764 POINT(37.695900827851425 -123.23176113029085) bank168764 +168765 POINT(37.237597829811314 -123.00753719971162) bank168765 +168766 POINT(36.87478104470417 -121.82652621699647) bank168766 +168767 POINT(37.130524435279135 -122.73962523277571) bank168767 +168768 POINT(38.32244833364503 -121.96616242093972) bank168768 +168769 POINT(38.643741999567524 -123.29882398044842) bank168769 +168770 POINT(36.820789351322844 -123.13959991423977) bank168770 +168771 POINT(36.893836712328124 -121.5257669839955) bank168771 +168772 POINT(36.960978734372276 -122.15240159066359) bank168772 +168773 POINT(37.23237212833463 -121.57540540279832) bank168773 +168774 POINT(37.633319786318324 -123.36219438468622) bank168774 +168775 POINT(37.633763809888215 -123.41228430773938) bank168775 +168776 POINT(38.24746315996181 -121.45806963215033) bank168776 +168777 POINT(38.171220981549915 -121.95583752461768) bank168777 +168778 POINT(37.316043773651195 -123.19316766814848) bank168778 +168779 POINT(38.64613271496778 -123.01201605387986) bank168779 +168780 POINT(37.31112696958988 -121.46568172669181) bank168780 +168781 POINT(37.07778193390574 -121.73769961765406) bank168781 +168782 POINT(37.70307292129333 -122.33397720743292) bank168782 +168783 POINT(36.89680564697963 -123.40790773988647) bank168783 +168784 POINT(37.2432914558087 -121.82534803786515) bank168784 +168785 POINT(38.3972726597353 -122.95871563832857) bank168785 +168786 POINT(38.10928231636596 -122.61807694943985) bank168786 +168787 POINT(37.39785615525539 -121.76841569627612) bank168787 +168788 POINT(37.22905358087748 -122.85782727705846) bank168788 +168789 POINT(38.02862350632493 -121.70170507769966) bank168789 +168790 POINT(38.37127802627261 -122.26165026066333) bank168790 +168791 POINT(38.259369775777316 -122.45192005163054) bank168791 +168792 POINT(37.67850007353045 -122.76442273612699) bank168792 +168793 POINT(37.11928132002178 -123.04775718283139) bank168793 +168794 POINT(36.84843727421066 -122.36710359459194) bank168794 +168795 POINT(38.605413045334494 -121.59612037066493) bank168795 +168796 POINT(36.92490382212494 -121.75980040368967) bank168796 +168797 POINT(37.62257421227116 -122.26561127865719) bank168797 +168798 POINT(38.42162563323769 -123.28085820029025) bank168798 +168799 POINT(38.648268814292344 -122.40610852991534) bank168799 +168800 POINT(37.584033778003985 -122.19207547511586) bank168800 +168801 POINT(37.28554675884598 -122.67684185879638) bank168801 +168802 POINT(38.60884450184922 -122.6693169671011) bank168802 +168803 POINT(37.326104987200345 -123.00826825929025) bank168803 +168804 POINT(38.293556532630596 -123.3123296835227) bank168804 +168805 POINT(38.645404172350126 -122.62811574410706) bank168805 +168806 POINT(36.82495435883986 -123.34542750591791) bank168806 +168807 POINT(37.175194646016685 -122.56724214676329) bank168807 +168808 POINT(38.56607413493077 -121.42317576261672) bank168808 +168809 POINT(38.66065329904244 -122.5034255170439) bank168809 +168810 POINT(37.92159971202348 -121.98294912437414) bank168810 +168811 POINT(37.20577293413389 -122.48016980843285) bank168811 +168812 POINT(36.78887745309443 -122.44194890223746) bank168812 +168813 POINT(37.26961563554862 -122.70572918293888) bank168813 +168814 POINT(38.441931582927396 -122.75161261187057) bank168814 +168815 POINT(37.04995635447944 -123.06689870226711) bank168815 +168816 POINT(37.923978140465294 -122.72862310915804) bank168816 +168817 POINT(37.77451792312272 -121.47041206502708) bank168817 +168818 POINT(36.77791346518314 -121.50439935857776) bank168818 +168819 POINT(36.89663446334813 -123.319587122342) bank168819 +168820 POINT(38.21451371390845 -121.67389162664314) bank168820 +168821 POINT(37.81244248361788 -122.68191839152175) bank168821 +168822 POINT(37.40229903072638 -123.06274355847833) bank168822 +168823 POINT(37.91714384045721 -121.81757968157495) bank168823 +168824 POINT(37.64722825418739 -123.37990865136203) bank168824 +168825 POINT(38.464981487601214 -122.51623461680357) bank168825 +168826 POINT(38.63172890137357 -123.27266955224037) bank168826 +168827 POINT(37.131148931858114 -121.64082182024086) bank168827 +168828 POINT(37.35279439036672 -122.69815761447282) bank168828 +168829 POINT(37.27589179201638 -123.40096102197492) bank168829 +168830 POINT(38.468601349275716 -121.47924629655189) bank168830 +168831 POINT(37.81614215260079 -122.14829673339013) bank168831 +168832 POINT(38.599083622362286 -122.20500414438814) bank168832 +168833 POINT(38.394065991096795 -122.07415370381361) bank168833 +168834 POINT(38.46195178533638 -122.8894170873937) bank168834 +168835 POINT(38.49073421860748 -121.92771416874979) bank168835 +168836 POINT(38.49546143963408 -123.10378750224763) bank168836 +168837 POINT(38.00638096959407 -123.23387459813686) bank168837 +168838 POINT(37.51298392184225 -123.33194542873589) bank168838 +168839 POINT(37.335220978366884 -123.33271311405306) bank168839 +168840 POINT(37.90254384403471 -121.96738377639946) bank168840 +168841 POINT(37.34152056782104 -121.96307949791803) bank168841 +168842 POINT(38.483320920863434 -122.730839355162) bank168842 +168843 POINT(38.53944878713479 -122.40772693408245) bank168843 +168844 POINT(37.02711657830722 -122.29147633714778) bank168844 +168845 POINT(37.68833583685145 -123.02436885082837) bank168845 +168846 POINT(38.61965038651291 -123.10432415344727) bank168846 +168847 POINT(37.15214346257995 -122.79502629737614) bank168847 +168848 POINT(38.63477759500801 -121.57448151379555) bank168848 +168849 POINT(37.59244773157727 -122.10356300701825) bank168849 +168850 POINT(38.2045917667995 -122.4449364643621) bank168850 +168851 POINT(37.31292745367727 -122.00088340712581) bank168851 +168852 POINT(37.37030630022569 -123.20934824017112) bank168852 +168853 POINT(38.18644764603121 -122.49300241878231) bank168853 +168854 POINT(37.47626819455317 -121.4613960987792) bank168854 +168855 POINT(37.89531568742864 -122.40091872390948) bank168855 +168856 POINT(38.0873031947375 -121.55503527038695) bank168856 +168857 POINT(37.863267453797256 -121.7379602925675) bank168857 +168858 POINT(37.8190866125719 -122.6923816678) bank168858 +168859 POINT(38.067843260283915 -122.49672725894933) bank168859 +168860 POINT(38.40839835422802 -121.86407655905184) bank168860 +168861 POINT(38.64164660669097 -122.81436057446811) bank168861 +168862 POINT(37.155402503972404 -121.43083447480312) bank168862 +168863 POINT(37.26143984045627 -122.1922255063893) bank168863 +168864 POINT(38.46074606492612 -122.39047798876376) bank168864 +168865 POINT(38.7364384315022 -121.62391923969989) bank168865 +168866 POINT(37.361705291540964 -123.41777189652495) bank168866 +168867 POINT(38.63623224997811 -122.48481308669896) bank168867 +168868 POINT(37.19868814626018 -122.1526041965434) bank168868 +168869 POINT(38.309243705260094 -123.18746967314227) bank168869 +168870 POINT(36.95588386451303 -122.23957086752101) bank168870 +168871 POINT(37.55784457885402 -122.71290013358669) bank168871 +168872 POINT(38.3435533204214 -123.40000254123959) bank168872 +168873 POINT(37.46834349730111 -122.0866203691839) bank168873 +168874 POINT(37.55142571820187 -122.65634723537252) bank168874 +168875 POINT(38.16470212033331 -122.3166153447403) bank168875 +168876 POINT(38.265936340236856 -121.88334710810447) bank168876 +168877 POINT(37.00146244302448 -122.24163282874245) bank168877 +168878 POINT(38.658072900895284 -122.41823897506403) bank168878 +168879 POINT(38.48899023981895 -121.80225092939993) bank168879 +168880 POINT(37.72847500517103 -123.0931774287441) bank168880 +168881 POINT(37.78218835074257 -122.58136522032581) bank168881 +168882 POINT(38.25423038117101 -121.53854348866096) bank168882 +168883 POINT(37.258257804627 -122.23638374432335) bank168883 +168884 POINT(37.286113738993656 -121.80436132142066) bank168884 +168885 POINT(38.01567633067162 -122.21466491879646) bank168885 +168886 POINT(38.280849528540244 -122.43640462505735) bank168886 +168887 POINT(37.97963782555178 -122.33352032344321) bank168887 +168888 POINT(38.244040774504434 -122.3632650429542) bank168888 +168889 POINT(37.28347947429923 -122.22424661380232) bank168889 +168890 POINT(37.964801828945866 -121.77146375062466) bank168890 +168891 POINT(37.23294480797683 -121.73900106764512) bank168891 +168892 POINT(36.79255431078924 -122.28851338047976) bank168892 +168893 POINT(37.0990268451404 -121.86704492798489) bank168893 +168894 POINT(37.07819398235462 -123.35855316242905) bank168894 +168895 POINT(38.50676845048298 -122.39800346006267) bank168895 +168896 POINT(36.794423094102534 -123.12179376068951) bank168896 +168897 POINT(37.93252135248043 -123.28904241628337) bank168897 +168898 POINT(37.840532023567945 -122.90035565997239) bank168898 +168899 POINT(38.38239222215876 -122.78391310264523) bank168899 +168900 POINT(38.58179527277111 -121.68519611050523) bank168900 +168901 POINT(37.76256653909591 -123.05176412414842) bank168901 +168902 POINT(37.138979310357705 -122.23347933265164) bank168902 +168903 POINT(38.354559039055005 -121.7498034289058) bank168903 +168904 POINT(38.646450371757524 -123.02993869341519) bank168904 +168905 POINT(37.12530573666636 -122.32183860163046) bank168905 +168906 POINT(38.43426273998337 -122.17137074978402) bank168906 +168907 POINT(37.66835993211827 -123.14496567723548) bank168907 +168908 POINT(38.33477930345341 -121.63001110023829) bank168908 +168909 POINT(37.42985326806517 -121.46321588964685) bank168909 +168910 POINT(38.24984496173938 -122.76167115880637) bank168910 +168911 POINT(38.28051214613987 -122.78986935050902) bank168911 +168912 POINT(36.9853262817012 -123.32224320877978) bank168912 +168913 POINT(38.47508408270613 -123.11200499656805) bank168913 +168914 POINT(37.299708528566704 -122.61508448049597) bank168914 +168915 POINT(38.28640461082088 -122.06410834882423) bank168915 +168916 POINT(38.179864616613436 -121.71765839564797) bank168916 +168917 POINT(37.948477419816 -122.2222807055138) bank168917 +168918 POINT(37.16097312282516 -122.88758582097017) bank168918 +168919 POINT(37.335785294695086 -122.122217547274) bank168919 +168920 POINT(37.24372086572044 -122.6266819217167) bank168920 +168921 POINT(38.34342884312602 -123.11862739590703) bank168921 +168922 POINT(37.72980414170322 -122.85107801270073) bank168922 +168923 POINT(37.94743618712025 -121.66871372305383) bank168923 +168924 POINT(37.73082609730158 -123.30289384279239) bank168924 +168925 POINT(38.75532254776291 -121.82306168632506) bank168925 +168926 POINT(36.99902694652313 -121.84630417675476) bank168926 +168927 POINT(36.865655578537634 -122.50723938972855) bank168927 +168928 POINT(36.98437619959273 -122.8308893535208) bank168928 +168929 POINT(37.058653765619276 -123.14421339447756) bank168929 +168930 POINT(38.172813848408765 -122.62655400056121) bank168930 +168931 POINT(37.53367681923061 -121.6942643767626) bank168931 +168932 POINT(37.86869428218331 -122.22682708025454) bank168932 +168933 POINT(37.3552874338278 -123.02802916199185) bank168933 +168934 POINT(38.543359840255164 -122.31349903408486) bank168934 +168935 POINT(38.26216892410146 -122.69317177573363) bank168935 +168936 POINT(38.2786712845794 -122.2857993011526) bank168936 +168937 POINT(38.16463981994088 -121.8616456683802) bank168937 +168938 POINT(38.55558282263166 -122.02551906783482) bank168938 +168939 POINT(38.45104636834936 -122.44173983628146) bank168939 +168940 POINT(38.74037281147333 -122.54097336210889) bank168940 +168941 POINT(37.75473009056074 -123.16260399417736) bank168941 +168942 POINT(38.0276197907981 -122.83975770414416) bank168942 +168943 POINT(38.69077515839311 -122.32761397202897) bank168943 +168944 POINT(37.0890641105481 -122.28089083334328) bank168944 +168945 POINT(36.823676643669344 -122.90850015742994) bank168945 +168946 POINT(36.971529735148046 -123.1422292691787) bank168946 +168947 POINT(38.460650652044265 -122.86357444298177) bank168947 +168948 POINT(36.97271821470812 -122.6514647909646) bank168948 +168949 POINT(38.36588757956172 -121.57647162132146) bank168949 +168950 POINT(38.6392185338549 -123.38330954368453) bank168950 +168951 POINT(37.54635983026609 -121.58057724710626) bank168951 +168952 POINT(37.68426480027003 -122.2346348737396) bank168952 +168953 POINT(37.72837669018658 -123.28106263853815) bank168953 +168954 POINT(37.236166030013905 -123.07262862245435) bank168954 +168955 POINT(37.452298504529026 -121.89270310868874) bank168955 +168956 POINT(38.2685002839373 -121.4752879879928) bank168956 +168957 POINT(38.465320670252844 -122.2010201944625) bank168957 +168958 POINT(38.02405057080304 -122.77156098231232) bank168958 +168959 POINT(38.56194952685213 -122.88679790947297) bank168959 +168960 POINT(37.328152820247254 -121.7500098584342) bank168960 +168961 POINT(36.91202656473839 -122.96680706379892) bank168961 +168962 POINT(37.00236752940176 -121.99742443373351) bank168962 +168963 POINT(37.688566449387636 -122.85180490245384) bank168963 +168964 POINT(37.785000853769056 -122.96261036664767) bank168964 +168965 POINT(38.38110664248193 -122.88037757147228) bank168965 +168966 POINT(37.688344100695375 -121.85051920660685) bank168966 +168967 POINT(38.21091275054736 -122.72952896023423) bank168967 +168968 POINT(36.81730261042166 -123.07053110759188) bank168968 +168969 POINT(37.687020882244234 -121.71131111930514) bank168969 +168970 POINT(38.6880685741631 -123.18918345167532) bank168970 +168971 POINT(37.745543525322304 -122.0150102055208) bank168971 +168972 POINT(38.62388696245537 -122.13949240237068) bank168972 +168973 POINT(38.56876317079637 -123.15172341684085) bank168973 +168974 POINT(37.02300637578747 -121.83114119300221) bank168974 +168975 POINT(36.99739718233543 -121.46797530877075) bank168975 +168976 POINT(38.28725717982341 -122.10175921616894) bank168976 +168977 POINT(38.17675731281912 -122.84042233470885) bank168977 +168978 POINT(37.97803763035438 -122.03018031832734) bank168978 +168979 POINT(38.47810159233295 -123.31167687580306) bank168979 +168980 POINT(38.460674011290436 -122.00791549654942) bank168980 +168981 POINT(38.45864366326021 -123.32196897126053) bank168981 +168982 POINT(37.204734268989206 -121.88763538398065) bank168982 +168983 POINT(36.994411368698586 -123.12462318211277) bank168983 +168984 POINT(36.99602275538568 -121.98961924293674) bank168984 +168985 POINT(37.16618036906388 -122.06831549291307) bank168985 +168986 POINT(37.28843226190508 -121.65024780298248) bank168986 +168987 POINT(38.02397327013958 -122.18921684562284) bank168987 +168988 POINT(38.3827713961255 -123.18233184264713) bank168988 +168989 POINT(37.274938137528615 -121.8627616821143) bank168989 +168990 POINT(37.11112142075465 -122.14006988533625) bank168990 +168991 POINT(37.68781616165878 -122.13334675259611) bank168991 +168992 POINT(37.19828772774625 -122.81381107594785) bank168992 +168993 POINT(37.880802020776834 -121.84907931067994) bank168993 +168994 POINT(37.98439781570913 -122.44826852030275) bank168994 +168995 POINT(38.37359187124403 -122.68798582395124) bank168995 +168996 POINT(37.79700375466388 -122.12432171273522) bank168996 +168997 POINT(37.97563472201459 -123.27690916754872) bank168997 +168998 POINT(37.2314396986837 -123.30872006141101) bank168998 +168999 POINT(38.04187981700116 -121.86592524935467) bank168999 +169000 POINT(38.20078415175169 -121.7838875042892) bank169000 +169001 POINT(38.00131467433234 -122.3104850314626) bank169001 +169002 POINT(37.308942822441814 -123.17948827451589) bank169002 +169003 POINT(38.151406061000166 -121.48174790777736) bank169003 +169004 POINT(37.51550946022125 -122.42721713568719) bank169004 +169005 POINT(37.209830295849834 -121.4848929046024) bank169005 +169006 POINT(37.008409914474555 -123.01685014203595) bank169006 +169007 POINT(37.82629007530497 -121.83935844555106) bank169007 +169008 POINT(37.95500225169308 -121.8875469950365) bank169008 +169009 POINT(37.86652471707462 -121.5103788207581) bank169009 +169010 POINT(36.97221470601426 -123.0209137626311) bank169010 +169011 POINT(37.62347360823778 -121.72634066046399) bank169011 +169012 POINT(37.97426537117006 -121.78801246043399) bank169012 +169013 POINT(37.71816625655469 -123.08642279269955) bank169013 +169014 POINT(38.719389244746424 -121.4263586047587) bank169014 +169015 POINT(37.00168244217164 -122.79626400489761) bank169015 +169016 POINT(37.04133517108953 -121.84198450632655) bank169016 +169017 POINT(37.109421949105645 -122.22914468083708) bank169017 +169018 POINT(38.41482035036235 -122.00392891587148) bank169018 +169019 POINT(37.60579986242376 -122.5906554868914) bank169019 +169020 POINT(37.262479834034856 -123.27996842253516) bank169020 +169021 POINT(38.592279964810096 -122.75324848528027) bank169021 +169022 POINT(38.06610471908011 -121.5277528630668) bank169022 +169023 POINT(38.72961171871881 -122.59558595032254) bank169023 +169024 POINT(37.718028240317494 -123.3347008007756) bank169024 +169025 POINT(37.65125897138742 -122.38897585898891) bank169025 +169026 POINT(37.09074368134683 -121.96747185039642) bank169026 +169027 POINT(38.22503241512748 -123.23948190029313) bank169027 +169028 POINT(37.986160676388664 -121.77734382858027) bank169028 +169029 POINT(37.20214315949796 -123.08152994844426) bank169029 +169030 POINT(37.49482924089363 -123.01637049935421) bank169030 +169031 POINT(37.13733717627218 -122.71220356415823) bank169031 +169032 POINT(38.35918123048726 -122.30000781626242) bank169032 +169033 POINT(38.695637500125315 -122.27667297051599) bank169033 +169034 POINT(37.841933300922655 -121.77925310333202) bank169034 +169035 POINT(37.19963151577218 -122.99706100549612) bank169035 +169036 POINT(38.14988561491346 -123.38310533074636) bank169036 +169037 POINT(38.31313197128476 -123.37220889580708) bank169037 +169038 POINT(38.34353017548404 -122.04344603910825) bank169038 +169039 POINT(38.0608433014068 -121.92383397139298) bank169039 +169040 POINT(38.75603848317855 -122.05822037402588) bank169040 +169041 POINT(37.457535806548115 -122.06385743559258) bank169041 +169042 POINT(37.66286221238559 -121.53611641724281) bank169042 +169043 POINT(37.06190660830048 -122.87802918707546) bank169043 +169044 POINT(38.288749389609414 -122.15680409593247) bank169044 +169045 POINT(36.797698004783705 -122.20494618135382) bank169045 +169046 POINT(37.35181315148681 -123.00543094300293) bank169046 +169047 POINT(37.04459244965821 -122.09398045572681) bank169047 +169048 POINT(37.72542685576596 -122.27627149338616) bank169048 +169049 POINT(38.5134547096563 -123.03386127127702) bank169049 +169050 POINT(38.59493227256953 -121.6479731328884) bank169050 +169051 POINT(37.03752709126692 -122.62461168479433) bank169051 +169052 POINT(37.80006130183777 -123.13586237827212) bank169052 +169053 POINT(38.50961520483217 -123.39533218621033) bank169053 +169054 POINT(37.676551699023335 -122.95919515660637) bank169054 +169055 POINT(37.396574846521304 -121.64906613791229) bank169055 +169056 POINT(38.337880795921016 -122.52508392535887) bank169056 +169057 POINT(38.44887781359677 -122.94703697331043) bank169057 +169058 POINT(37.88525325959417 -123.1801493420245) bank169058 +169059 POINT(37.53431788634803 -121.73389322596115) bank169059 +169060 POINT(38.28214711008843 -121.46916154920926) bank169060 +169061 POINT(38.20429989531582 -121.77157242366229) bank169061 +169062 POINT(38.61979707143413 -121.53113469417015) bank169062 +169063 POINT(38.29207482074765 -122.4682589162459) bank169063 +169064 POINT(38.48203881508185 -121.83152568371301) bank169064 +169065 POINT(37.51328783752452 -123.38463358790959) bank169065 +169066 POINT(37.56709768990057 -123.26767284649435) bank169066 +169067 POINT(37.89986632705551 -122.6109295957785) bank169067 +169068 POINT(38.277309593944175 -123.04357118584485) bank169068 +169069 POINT(38.63630411880538 -123.33875803725591) bank169069 +169070 POINT(38.65270055177041 -122.44513781298683) bank169070 +169071 POINT(37.74152602731832 -122.69556017731006) bank169071 +169072 POINT(38.358423480632304 -122.49666041827707) bank169072 +169073 POINT(37.92149596218271 -122.43872128154706) bank169073 +169074 POINT(37.85356457575805 -122.88404553099551) bank169074 +169075 POINT(38.26416063585444 -122.67896983044659) bank169075 +169076 POINT(38.69798672009961 -122.767569819376) bank169076 +169077 POINT(37.15284544206832 -123.01695959343627) bank169077 +169078 POINT(37.27909107345422 -122.17584980843378) bank169078 +169079 POINT(37.29510400320559 -122.65864294078717) bank169079 +169080 POINT(38.520214315836405 -121.6790275938993) bank169080 +169081 POINT(37.46757235008654 -122.85979750407903) bank169081 +169082 POINT(38.07992607708962 -123.23140540142451) bank169082 +169083 POINT(37.29488571279635 -122.53852528031129) bank169083 +169084 POINT(37.44149227825903 -123.03869922351639) bank169084 +169085 POINT(38.02671883033556 -122.92435848231071) bank169085 +169086 POINT(38.432932364569375 -122.2901791759789) bank169086 +169087 POINT(38.43033625350909 -122.01269276984661) bank169087 +169088 POINT(38.695058233254265 -122.5793981867217) bank169088 +169089 POINT(37.91767159499992 -123.40397129148664) bank169089 +169090 POINT(38.22851110266567 -122.07496667480196) bank169090 +169091 POINT(37.63255520424522 -123.26622326324424) bank169091 +169092 POINT(38.202975754467424 -121.92151560968881) bank169092 +169093 POINT(38.36967607273584 -122.6965808902197) bank169093 +169094 POINT(37.49636436284337 -123.1135175334247) bank169094 +169095 POINT(37.92264670453554 -121.83201640325308) bank169095 +169096 POINT(37.836099349538515 -121.525571046395) bank169096 +169097 POINT(38.01896949062497 -122.85773099958313) bank169097 +169098 POINT(38.71766188060974 -123.1169407858358) bank169098 +169099 POINT(37.27807964390453 -122.51701620326222) bank169099 +169100 POINT(37.678175834158765 -122.56128906028508) bank169100 +169101 POINT(37.45577917907176 -122.55468965704353) bank169101 +169102 POINT(37.00995899713569 -122.93888165300176) bank169102 +169103 POINT(38.14301550460238 -123.19912371658484) bank169103 +169104 POINT(37.54942810496086 -121.66307120804666) bank169104 +169105 POINT(37.59205882826512 -122.89133430927224) bank169105 +169106 POINT(36.84658849188028 -123.27440993944282) bank169106 +169107 POINT(37.97883509062956 -122.88478820926761) bank169107 +169108 POINT(37.53059646027594 -122.80554757477364) bank169108 +169109 POINT(37.4011861288953 -123.3080330506557) bank169109 +169110 POINT(38.61142074595236 -121.42718156719216) bank169110 +169111 POINT(38.08733749039623 -121.47341615203305) bank169111 +169112 POINT(38.362275387632316 -123.11347748388134) bank169112 +169113 POINT(38.55591490478593 -122.1337349535998) bank169113 +169114 POINT(38.650109264911045 -121.97410932139157) bank169114 +169115 POINT(36.83596317094566 -122.50841026985923) bank169115 +169116 POINT(38.349638898157906 -121.54585205561278) bank169116 +169117 POINT(38.074898155610654 -121.76152621901234) bank169117 +169118 POINT(37.549458986187545 -122.35480067335088) bank169118 +169119 POINT(37.662981288660504 -122.12534585023825) bank169119 +169120 POINT(37.17719238602788 -122.69614671022653) bank169120 +169121 POINT(38.50564807727287 -122.37219372601064) bank169121 +169122 POINT(37.35784860677686 -122.01040471122708) bank169122 +169123 POINT(38.0563684387164 -121.54726359253306) bank169123 +169124 POINT(38.527348095477954 -123.40312022021304) bank169124 +169125 POINT(37.85549051942013 -123.32645953188535) bank169125 +169126 POINT(37.2084842993908 -121.97773262069467) bank169126 +169127 POINT(37.46199707173906 -123.35821730701014) bank169127 +169128 POINT(37.68019074398536 -122.98669297061956) bank169128 +169129 POINT(38.54967524917882 -122.59419707985253) bank169129 +169130 POINT(37.891024174949436 -123.13380234298735) bank169130 +169131 POINT(37.590102812041735 -123.39809044823643) bank169131 +169132 POINT(38.56519309562345 -121.59950342533394) bank169132 +169133 POINT(38.053610124261354 -121.91401973787885) bank169133 +169134 POINT(37.511780295507414 -122.16229734438183) bank169134 +169135 POINT(36.77989386202059 -121.8088591590455) bank169135 +169136 POINT(37.38092928056687 -122.21641228918241) bank169136 +169137 POINT(38.48299055479177 -121.75931598390525) bank169137 +169138 POINT(38.266668509480176 -123.12286299855214) bank169138 +169139 POINT(38.68098213545251 -121.46973415307082) bank169139 +169140 POINT(38.215238098613305 -122.55728402733739) bank169140 +169141 POINT(36.98672400393211 -123.37104496364032) bank169141 +169142 POINT(38.27778749559932 -122.8347536661713) bank169142 +169143 POINT(38.22730842798566 -121.59130797517732) bank169143 +169144 POINT(37.774082616513795 -123.2849555783795) bank169144 +169145 POINT(36.77617833023618 -121.48184800577009) bank169145 +169146 POINT(38.22918450854231 -122.7407697201399) bank169146 +169147 POINT(38.14247888503608 -121.62215853887719) bank169147 +169148 POINT(37.011124234067815 -122.66597162203733) bank169148 +169149 POINT(38.06440526741472 -123.26498613980935) bank169149 +169150 POINT(38.29154316237823 -122.62913792450077) bank169150 +169151 POINT(37.026311141530485 -122.09178118491721) bank169151 +169152 POINT(37.08269374989282 -121.88450280515244) bank169152 +169153 POINT(38.08451899348423 -121.514000765981) bank169153 +169154 POINT(38.757271524705516 -121.91564597941486) bank169154 +169155 POINT(38.235329800054146 -121.9618984405203) bank169155 +169156 POINT(38.53762170184418 -122.95194114029896) bank169156 +169157 POINT(38.74679619166459 -123.15024854700822) bank169157 +169158 POINT(38.2237947972328 -122.08722813002913) bank169158 +169159 POINT(38.37636134972993 -122.57157458063739) bank169159 +169160 POINT(37.994154500496585 -121.73170340647488) bank169160 +169161 POINT(37.46429097970436 -121.79856882277508) bank169161 +169162 POINT(36.96493059429814 -122.00246229801613) bank169162 +169163 POINT(38.60008048359502 -121.98635181836646) bank169163 +169164 POINT(37.07716970223032 -122.1824968446378) bank169164 +169165 POINT(38.0920515066694 -121.66355202390734) bank169165 +169166 POINT(37.07164717899924 -122.67228667799915) bank169166 +169167 POINT(37.66242889952078 -123.08627872595574) bank169167 +169168 POINT(37.384903899116615 -121.70953540314956) bank169168 +169169 POINT(37.78149950566166 -122.39517904541634) bank169169 +169170 POINT(36.96465991127897 -123.20424327686429) bank169170 +169171 POINT(36.82822166552578 -122.77021525923526) bank169171 +169172 POINT(38.12707464833437 -121.53054397728539) bank169172 +169173 POINT(37.04114934750138 -122.20998084788016) bank169173 +169174 POINT(37.33684394311172 -122.41505171722262) bank169174 +169175 POINT(38.02160453928862 -123.33825297464782) bank169175 +169176 POINT(37.85837471733919 -123.07577841207846) bank169176 +169177 POINT(38.19056509128334 -122.37549277936968) bank169177 +169178 POINT(38.395603221002425 -122.9471515684374) bank169178 +169179 POINT(37.85753131305788 -122.2509125718938) bank169179 +169180 POINT(36.80013923694125 -121.66019380406605) bank169180 +169181 POINT(38.21666428988475 -122.28734318135771) bank169181 +169182 POINT(37.800204948237166 -121.70771934170156) bank169182 +169183 POINT(37.37818900475571 -123.06559916542207) bank169183 +169184 POINT(37.203928907454916 -121.44582331518389) bank169184 +169185 POINT(36.78580884545506 -122.5027721742906) bank169185 +169186 POINT(37.003963454757596 -121.82427246921995) bank169186 +169187 POINT(37.14516179881822 -122.31564868994032) bank169187 +169188 POINT(38.43632288337664 -121.71513970085316) bank169188 +169189 POINT(37.90241796588971 -123.33508951546757) bank169189 +169190 POINT(38.72467173996723 -122.09751829373259) bank169190 +169191 POINT(37.11663730329042 -122.3020278761215) bank169191 +169192 POINT(37.20497489701988 -123.17777023757822) bank169192 +169193 POINT(36.98620510330825 -122.8444221214123) bank169193 +169194 POINT(37.46999346266541 -121.6104344977198) bank169194 +169195 POINT(38.40113312246016 -123.2280280086218) bank169195 +169196 POINT(37.61550058636045 -122.85530427970595) bank169196 +169197 POINT(38.59911160677241 -122.56095253025518) bank169197 +169198 POINT(36.823948872143106 -121.76334193813996) bank169198 +169199 POINT(38.328976114053404 -123.12718869119155) bank169199 +169200 POINT(36.92271799844553 -121.60934893561074) bank169200 +169201 POINT(37.84030099984714 -121.54100992776011) bank169201 +169202 POINT(38.117060805277326 -122.70827203083479) bank169202 +169203 POINT(36.95023349687864 -121.7999096272186) bank169203 +169204 POINT(38.602766364263914 -122.4952189138855) bank169204 +169205 POINT(38.299916931383926 -121.61692849549571) bank169205 +169206 POINT(37.15514029447739 -122.02512142863155) bank169206 +169207 POINT(38.62204910070755 -122.49724021803851) bank169207 +169208 POINT(38.569681365831784 -122.25699669997844) bank169208 +169209 POINT(38.04923060080275 -122.84211587381907) bank169209 +169210 POINT(38.22180542231978 -121.92141661571176) bank169210 +169211 POINT(37.376971179246006 -121.75694018526382) bank169211 +169212 POINT(38.62876101178325 -122.33510457351129) bank169212 +169213 POINT(37.571294113710856 -122.90443069950551) bank169213 +169214 POINT(37.11747672401388 -122.5535089441762) bank169214 +169215 POINT(38.41708008462263 -122.1937413680365) bank169215 +169216 POINT(38.27627011425516 -122.3391801315797) bank169216 +169217 POINT(37.85672056176406 -121.63899818946935) bank169217 +169218 POINT(37.590315716585614 -121.8519740273967) bank169218 +169219 POINT(37.16400106094072 -122.06001835502418) bank169219 +169220 POINT(36.91512811857396 -122.76778641284369) bank169220 +169221 POINT(38.213685051739034 -122.466318073002) bank169221 +169222 POINT(38.55064364846718 -122.12107769733635) bank169222 +169223 POINT(37.94547705375406 -122.6282420866161) bank169223 +169224 POINT(37.117835650438245 -121.57728967738026) bank169224 +169225 POINT(38.59935054173159 -122.07435422325753) bank169225 +169226 POINT(38.157618125999406 -121.83597293187134) bank169226 +169227 POINT(38.36844087988999 -122.08023550716618) bank169227 +169228 POINT(37.41168910730869 -121.70607627477621) bank169228 +169229 POINT(36.908278094828134 -121.43656763505524) bank169229 +169230 POINT(37.52789295941257 -121.63585917859132) bank169230 +169231 POINT(38.65109613664427 -122.40319458823234) bank169231 +169232 POINT(37.86174245319847 -121.82111967864444) bank169232 +169233 POINT(37.9823262797083 -121.76182496020182) bank169233 +169234 POINT(37.43056836203277 -122.8478765226322) bank169234 +169235 POINT(38.15347432910397 -122.06878167966156) bank169235 +169236 POINT(38.047997197380504 -122.41657970296916) bank169236 +169237 POINT(37.37061637540029 -122.24895358387442) bank169237 +169238 POINT(37.81061607842081 -122.30538776466787) bank169238 +169239 POINT(38.12314959722525 -121.70321189427527) bank169239 +169240 POINT(38.04791189204507 -123.080796622974) bank169240 +169241 POINT(37.37930170514563 -123.30792175425535) bank169241 +169242 POINT(37.760566193445875 -122.2315814111726) bank169242 +169243 POINT(38.10704104789604 -121.99457559864953) bank169243 +169244 POINT(38.31400912339176 -123.26895582549436) bank169244 +169245 POINT(38.70991858300569 -122.77882500104009) bank169245 +169246 POINT(38.58830160092895 -122.85480030473012) bank169246 +169247 POINT(38.76811953403689 -122.34729255238162) bank169247 +169248 POINT(38.15891264256167 -121.77510728287363) bank169248 +169249 POINT(38.650799408301694 -123.20135718950071) bank169249 +169250 POINT(37.08868283107957 -122.93306449018293) bank169250 +169251 POINT(36.8379027524904 -121.86499728308196) bank169251 +169252 POINT(38.28380611322658 -122.72022449753146) bank169252 +169253 POINT(37.66015386479297 -121.7905426446051) bank169253 +169254 POINT(38.32315035733024 -121.77139003480325) bank169254 +169255 POINT(37.787508649101234 -122.63783556701776) bank169255 +169256 POINT(38.55169346690414 -122.97855864075825) bank169256 +169257 POINT(38.25892133302925 -122.43490456755781) bank169257 +169258 POINT(38.33889720986463 -122.07877092151838) bank169258 +169259 POINT(38.04595664628154 -121.69158822984936) bank169259 +169260 POINT(37.94609024732227 -121.80185131141603) bank169260 +169261 POINT(38.64211855748706 -122.51656044527734) bank169261 +169262 POINT(37.02661547533983 -122.22381059794414) bank169262 +169263 POINT(36.94375998086556 -123.03973219982603) bank169263 +169264 POINT(38.69705624812616 -121.85800241252372) bank169264 +169265 POINT(37.46739265750959 -122.07335024902973) bank169265 +169266 POINT(38.29169542048737 -123.38004545438477) bank169266 +169267 POINT(38.386081819047874 -122.16285071240797) bank169267 +169268 POINT(38.171718971403095 -122.77519307837403) bank169268 +169269 POINT(38.11975233172975 -122.39282625393908) bank169269 +169270 POINT(37.08076078067068 -121.66626089545508) bank169270 +169271 POINT(38.072483925125724 -121.50014881701576) bank169271 +169272 POINT(37.26333536858645 -123.07697572196541) bank169272 +169273 POINT(37.54384166590187 -122.16117663246177) bank169273 +169274 POINT(37.3654916407045 -121.76414819029209) bank169274 +169275 POINT(37.14989717055018 -122.9236765764048) bank169275 +169276 POINT(38.19788598831361 -121.43326015942296) bank169276 +169277 POINT(38.51671056949893 -121.9413340232387) bank169277 +169278 POINT(38.153074009070885 -122.93071881546403) bank169278 +169279 POINT(38.17617655679548 -122.76236899005737) bank169279 +169280 POINT(36.81497833845976 -122.10020459895509) bank169280 +169281 POINT(38.358186880038346 -121.42521978974082) bank169281 +169282 POINT(38.453920142847345 -122.01684201394066) bank169282 +169283 POINT(37.6117004785243 -122.70176603914278) bank169283 +169284 POINT(36.90785853958178 -121.76286613883236) bank169284 +169285 POINT(37.05225663675217 -122.257026445248) bank169285 +169286 POINT(36.946232071764214 -122.07559812374545) bank169286 +169287 POINT(37.11622677390207 -121.81302206409889) bank169287 +169288 POINT(37.772308682131424 -122.67720594574932) bank169288 +169289 POINT(38.65346436284634 -121.72783987423738) bank169289 +169290 POINT(37.54523150981167 -121.51913167868278) bank169290 +169291 POINT(37.946446573282635 -123.05525607982788) bank169291 +169292 POINT(36.9861986478564 -122.91980593540092) bank169292 +169293 POINT(38.4465627684573 -122.09432297674185) bank169293 +169294 POINT(37.2738439128656 -122.7497301983292) bank169294 +169295 POINT(36.936750712058995 -121.56845234787504) bank169295 +169296 POINT(37.40633593068283 -122.77614062948118) bank169296 +169297 POINT(37.427649923388586 -121.47440698155869) bank169297 +169298 POINT(38.3794866123964 -121.81434575087908) bank169298 +169299 POINT(36.82360317942399 -121.95167288116748) bank169299 +169300 POINT(38.56474058459636 -123.38855473099918) bank169300 +169301 POINT(38.260643807043465 -122.05962541829025) bank169301 +169302 POINT(37.68443859894374 -122.46165934259963) bank169302 +169303 POINT(37.9254534758295 -123.36959915645369) bank169303 +169304 POINT(37.639668331132974 -121.95096348676879) bank169304 +169305 POINT(38.142096724974394 -121.9036621643759) bank169305 +169306 POINT(38.62229723875992 -121.84321153607463) bank169306 +169307 POINT(37.31902885168108 -122.76684068628667) bank169307 +169308 POINT(37.50062597854445 -122.19901425732321) bank169308 +169309 POINT(37.91391574966303 -121.47696931332099) bank169309 +169310 POINT(37.43830459945052 -122.3910842751377) bank169310 +169311 POINT(37.67201293085089 -121.77505083408526) bank169311 +169312 POINT(38.57630658256524 -123.06532730052979) bank169312 +169313 POINT(37.69608966081779 -123.0839100062119) bank169313 +169314 POINT(37.56140473428616 -123.41561611374289) bank169314 +169315 POINT(37.222493765258335 -122.63832463011254) bank169315 +169316 POINT(37.97102642760138 -122.41433962230136) bank169316 +169317 POINT(38.40522147675915 -123.19585510310851) bank169317 +169318 POINT(37.0578630630494 -123.32429108681903) bank169318 +169319 POINT(38.7612771355183 -122.76682452429486) bank169319 +169320 POINT(37.48486592125941 -122.77405646889122) bank169320 +169321 POINT(38.565711165846906 -122.69242923982789) bank169321 +169322 POINT(38.17877274289188 -121.79668513491966) bank169322 +169323 POINT(36.97329863785529 -122.94129662445664) bank169323 +169324 POINT(38.30256125524537 -121.650092182696) bank169324 +169325 POINT(37.81646691897797 -123.03348545434045) bank169325 +169326 POINT(38.62607614515901 -122.90002471140257) bank169326 +169327 POINT(38.55728980827292 -122.86094424893534) bank169327 +169328 POINT(36.90713738138708 -122.0513640458886) bank169328 +169329 POINT(38.26167538709987 -122.80225858452708) bank169329 +169330 POINT(37.700104436689564 -122.07257911696354) bank169330 +169331 POINT(38.510668813709636 -122.48224068451316) bank169331 +169332 POINT(38.154762839902496 -122.0173374651258) bank169332 +169333 POINT(36.98717581773135 -123.21163073783563) bank169333 +169334 POINT(37.58692204496474 -123.1132626069256) bank169334 +169335 POINT(38.20436661026669 -122.37392865826031) bank169335 +169336 POINT(38.454723144581145 -121.91520187302149) bank169336 +169337 POINT(37.51376451257753 -122.54395322267057) bank169337 +169338 POINT(38.688418584066106 -122.87326157948887) bank169338 +169339 POINT(38.48690939780369 -121.71310028721707) bank169339 +169340 POINT(37.307073076651214 -122.55178745280303) bank169340 +169341 POINT(38.742281293164964 -122.40136997105641) bank169341 +169342 POINT(37.674392664304996 -123.15761187729265) bank169342 +169343 POINT(38.61192578859982 -122.24040517286643) bank169343 +169344 POINT(37.927253733501956 -121.83993919555097) bank169344 +169345 POINT(37.2963240006919 -123.08491228363137) bank169345 +169346 POINT(37.70688473402551 -121.79571746564143) bank169346 +169347 POINT(37.98229667595688 -122.93306121398892) bank169347 +169348 POINT(37.39144400306762 -123.35181628638688) bank169348 +169349 POINT(37.87860739874112 -122.66159291029832) bank169349 +169350 POINT(38.193279074688206 -122.07208782144735) bank169350 +169351 POINT(38.66375794616567 -122.76958778679477) bank169351 +169352 POINT(37.74089055250896 -122.93741415757881) bank169352 +169353 POINT(37.029191955911024 -122.48573538398385) bank169353 +169354 POINT(37.93584858379684 -123.03187403830475) bank169354 +169355 POINT(38.362796280972795 -121.77205766259672) bank169355 +169356 POINT(37.955680987295274 -121.87183535068934) bank169356 +169357 POINT(38.08155540854681 -122.85505555783399) bank169357 +169358 POINT(37.47911238363354 -123.22283440461352) bank169358 +169359 POINT(38.465946381251584 -121.94635518032904) bank169359 +169360 POINT(37.58045132071306 -122.5979623418282) bank169360 +169361 POINT(37.78446929797494 -123.28009182537781) bank169361 +169362 POINT(38.097963178883724 -123.39178396186962) bank169362 +169363 POINT(38.382154699384245 -123.3963567637728) bank169363 +169364 POINT(37.46420303829162 -121.92268709265694) bank169364 +169365 POINT(38.587454526655435 -122.08400192156881) bank169365 +169366 POINT(37.590770532847706 -122.12530795290809) bank169366 +169367 POINT(38.2306232108219 -121.79396680909008) bank169367 +169368 POINT(37.65703152091628 -123.24289620212635) bank169368 +169369 POINT(37.111953409106924 -121.44421133355829) bank169369 +169370 POINT(36.962457048032846 -121.9515111504564) bank169370 +169371 POINT(38.630533950568534 -121.50235329804926) bank169371 +169372 POINT(38.46862284421217 -122.51798160095522) bank169372 +169373 POINT(38.03746999186426 -122.51051589307924) bank169373 +169374 POINT(38.31909468592177 -121.49473506917391) bank169374 +169375 POINT(38.09570509704099 -122.9212760733726) bank169375 +169376 POINT(37.85539965919838 -122.58332777046016) bank169376 +169377 POINT(37.62597363601308 -122.88648716435543) bank169377 +169378 POINT(37.386547124431935 -122.75368353912218) bank169378 +169379 POINT(36.7907407685393 -122.4568337584182) bank169379 +169380 POINT(37.3039079706695 -122.51234684963917) bank169380 +169381 POINT(37.25353687542933 -121.90320751791309) bank169381 +169382 POINT(38.22291542327454 -122.70065151610028) bank169382 +169383 POINT(37.92378678284424 -121.44319901835918) bank169383 +169384 POINT(38.695680177919996 -122.1443981840003) bank169384 +169385 POINT(37.645300195294524 -121.97582665174292) bank169385 +169386 POINT(38.343884082374814 -121.62085231036846) bank169386 +169387 POINT(37.42949721406957 -121.99501148837632) bank169387 +169388 POINT(38.73304011267243 -121.95258136472528) bank169388 +169389 POINT(38.57920228585732 -122.22151884931543) bank169389 +169390 POINT(38.024945032193884 -122.4161737956182) bank169390 +169391 POINT(38.72805592386234 -123.0546171177987) bank169391 +169392 POINT(37.52396865084766 -123.1100989289724) bank169392 +169393 POINT(36.94452731715252 -122.74760664898542) bank169393 +169394 POINT(38.287890011584516 -121.49575318532047) bank169394 +169395 POINT(37.32216774882999 -122.22645876600116) bank169395 +169396 POINT(36.77846332124201 -121.46328419872907) bank169396 +169397 POINT(38.72638198690696 -122.26308412859066) bank169397 +169398 POINT(36.88317579664717 -121.6576726165156) bank169398 +169399 POINT(38.23501657936837 -122.64726367972655) bank169399 +169400 POINT(38.703936172108904 -123.1862832264152) bank169400 +169401 POINT(38.45384487472317 -123.04642465206545) bank169401 +169402 POINT(37.25137579801164 -122.96947583619756) bank169402 +169403 POINT(38.55892275219418 -123.10614500855814) bank169403 +169404 POINT(36.91077010960923 -122.07530514201272) bank169404 +169405 POINT(37.05345727214694 -123.38043199186725) bank169405 +169406 POINT(37.136446424395146 -122.5056074569302) bank169406 +169407 POINT(37.312093107726625 -122.56106602455851) bank169407 +169408 POINT(38.69587142668669 -122.8526827287549) bank169408 +169409 POINT(38.04883064143869 -121.89589875012807) bank169409 +169410 POINT(37.801650474979255 -123.19014408887439) bank169410 +169411 POINT(38.52143026101496 -121.74514554327438) bank169411 +169412 POINT(38.12565572771348 -121.96679068072514) bank169412 +169413 POINT(37.33703296005927 -122.4561331016668) bank169413 +169414 POINT(38.57185070181947 -121.43832685804045) bank169414 +169415 POINT(38.43187076886009 -123.1992870705128) bank169415 +169416 POINT(38.333456169487334 -121.46004515929862) bank169416 +169417 POINT(37.246775155172344 -122.38673499200611) bank169417 +169418 POINT(36.93338124796876 -123.28685837694049) bank169418 +169419 POINT(38.07892075177124 -121.88928699478701) bank169419 +169420 POINT(37.90266726752519 -121.6427444020186) bank169420 +169421 POINT(36.83061776678676 -122.37509758094235) bank169421 +169422 POINT(38.159286229818974 -122.84182303596876) bank169422 +169423 POINT(37.62895331068277 -122.24611995367819) bank169423 +169424 POINT(38.42281304580561 -122.91744580850707) bank169424 +169425 POINT(37.88000993273309 -122.40659645084338) bank169425 +169426 POINT(36.934051663313554 -121.59460139935345) bank169426 +169427 POINT(37.34753978375995 -122.26106374996067) bank169427 +169428 POINT(37.41756711150958 -122.82106141299882) bank169428 +169429 POINT(38.160403855536615 -122.93184390598819) bank169429 +169430 POINT(38.51572633999363 -121.75060976152291) bank169430 +169431 POINT(37.591327961438736 -121.66124185845392) bank169431 +169432 POINT(37.166203952357755 -121.72003344572167) bank169432 +169433 POINT(38.16972995840479 -122.65126572818886) bank169433 +169434 POINT(38.518739007452936 -123.19787540232292) bank169434 +169435 POINT(37.411716212366564 -122.20270177331449) bank169435 +169436 POINT(36.9187058756788 -122.59285051584305) bank169436 +169437 POINT(38.276031400268316 -122.46399546940685) bank169437 +169438 POINT(37.56809469522476 -121.452247946103) bank169438 +169439 POINT(37.55631223098166 -121.708372115353) bank169439 +169440 POINT(37.34594717635948 -121.61277587274303) bank169440 +169441 POINT(38.39572737454246 -122.2831889642045) bank169441 +169442 POINT(38.618111324198416 -123.24514660219478) bank169442 +169443 POINT(37.53795829095909 -122.34681678089605) bank169443 +169444 POINT(38.011066994370935 -122.11618298848207) bank169444 +169445 POINT(37.83732281238889 -121.60680696701073) bank169445 +169446 POINT(37.07572156383126 -122.37082188245687) bank169446 +169447 POINT(38.708692220508816 -123.24094840289278) bank169447 +169448 POINT(38.41454888505397 -123.34613216077878) bank169448 +169449 POINT(38.57176584694439 -123.08333869893735) bank169449 +169450 POINT(36.980454574181536 -122.74565537486046) bank169450 +169451 POINT(36.82348689523221 -121.43881377055351) bank169451 +169452 POINT(38.61091521792723 -122.65992440732587) bank169452 +169453 POINT(38.18947698709987 -122.5360744444522) bank169453 +169454 POINT(37.42412466165633 -122.57934578802517) bank169454 +169455 POINT(38.25740223259786 -121.69576115117968) bank169455 +169456 POINT(37.030905827251814 -122.04258975055484) bank169456 +169457 POINT(37.21966671051455 -122.85582957627732) bank169457 +169458 POINT(37.70218227242739 -122.26270489691812) bank169458 +169459 POINT(36.941852471310376 -122.67529270551269) bank169459 +169460 POINT(36.84041235230234 -121.56240990019909) bank169460 +169461 POINT(38.63467193788396 -122.95448226943218) bank169461 +169462 POINT(38.59311376953935 -122.94912653643814) bank169462 +169463 POINT(37.400185512905836 -123.34470155467608) bank169463 +169464 POINT(38.23081868067758 -123.37762687540592) bank169464 +169465 POINT(37.90183459839153 -123.38579165982806) bank169465 +169466 POINT(38.34096490107577 -122.80868547232359) bank169466 +169467 POINT(37.13530539801652 -122.47728554982733) bank169467 +169468 POINT(37.37988852710402 -122.02473359968239) bank169468 +169469 POINT(38.45569895140105 -122.48999033034555) bank169469 +169470 POINT(36.96981900085363 -122.38598727346248) bank169470 +169471 POINT(37.70043256014473 -123.40481904799603) bank169471 +169472 POINT(38.23394416292001 -121.79756244308311) bank169472 +169473 POINT(37.16993517736426 -123.37300935893064) bank169473 +169474 POINT(37.20175978806248 -122.6545418007315) bank169474 +169475 POINT(38.04855683578376 -123.18365496463375) bank169475 +169476 POINT(38.28480949228743 -122.77118703130947) bank169476 +169477 POINT(37.79753722691361 -121.6044654175883) bank169477 +169478 POINT(36.95769008141911 -121.77345316783727) bank169478 +169479 POINT(38.13701122000874 -123.23200351946439) bank169479 +169480 POINT(38.670536082810386 -123.2363013901091) bank169480 +169481 POINT(37.69327168803378 -121.44416953662211) bank169481 +169482 POINT(37.30312100075439 -122.95636293671524) bank169482 +169483 POINT(36.79966566739598 -122.78339506297188) bank169483 +169484 POINT(37.08914515399558 -122.87718551055961) bank169484 +169485 POINT(37.45212773213284 -122.23909892599596) bank169485 +169486 POINT(37.50200864724291 -121.5106283617163) bank169486 +169487 POINT(36.81952760427757 -123.23534513585952) bank169487 +169488 POINT(38.0997169577916 -122.66490144200154) bank169488 +169489 POINT(36.83839697198791 -123.29497491029144) bank169489 +169490 POINT(38.22675796702359 -123.19924284800976) bank169490 +169491 POINT(38.54049331648876 -121.87572070814731) bank169491 +169492 POINT(36.86949247069657 -122.95054665774381) bank169492 +169493 POINT(38.14889162563665 -121.69963792968834) bank169493 +169494 POINT(38.67898754838046 -123.13331202248727) bank169494 +169495 POINT(38.00828632975627 -123.20235334756956) bank169495 +169496 POINT(37.1720839624844 -121.95769304564908) bank169496 +169497 POINT(38.46370434119772 -121.88332162906823) bank169497 +169498 POINT(37.50540628049807 -123.03359687199216) bank169498 +169499 POINT(38.212672125778404 -122.10846080881689) bank169499 +169500 POINT(37.02145904387416 -123.09730309567367) bank169500 +169501 POINT(37.37964629096488 -121.93113922672511) bank169501 +169502 POINT(37.21795533488936 -121.81320686772138) bank169502 +169503 POINT(37.64180872169854 -122.89806418862922) bank169503 +169504 POINT(37.72600164989693 -121.74365328875585) bank169504 +169505 POINT(37.819064338225516 -122.29389850505926) bank169505 +169506 POINT(37.0958988322586 -121.67185178724655) bank169506 +169507 POINT(38.26846905143797 -122.72952035716004) bank169507 +169508 POINT(38.7682472157533 -122.15316442894452) bank169508 +169509 POINT(37.15251032181631 -122.23976876854394) bank169509 +169510 POINT(37.18886459073289 -122.57421896217635) bank169510 +169511 POINT(37.28066193163637 -121.61719261253168) bank169511 +169512 POINT(37.12782138358374 -122.86321920166557) bank169512 +169513 POINT(38.359662181952906 -122.8340151875845) bank169513 +169514 POINT(38.00978484347924 -122.14105131320996) bank169514 +169515 POINT(38.150054702313746 -122.10585815578132) bank169515 +169516 POINT(38.201529150386506 -122.20924431680369) bank169516 +169517 POINT(36.95834562353814 -122.78186213189569) bank169517 +169518 POINT(37.45111025053291 -121.85894040309961) bank169518 +169519 POINT(37.095174020228775 -123.21346222174402) bank169519 +169520 POINT(38.19675855344578 -121.93070353315166) bank169520 +169521 POINT(38.227989968363964 -123.35895853705954) bank169521 +169522 POINT(38.45701981294865 -122.36999017095054) bank169522 +169523 POINT(37.82377629894152 -122.5055071873202) bank169523 +169524 POINT(38.02484616686496 -122.06020011678352) bank169524 +169525 POINT(37.37729173485027 -122.1439592945022) bank169525 +169526 POINT(38.33584231821933 -122.20834189878937) bank169526 +169527 POINT(36.947631760241315 -123.31431619176138) bank169527 +169528 POINT(37.76087873423236 -122.69084906132598) bank169528 +169529 POINT(37.29736667963573 -122.07702390580732) bank169529 +169530 POINT(38.30498050460283 -123.24439984771391) bank169530 +169531 POINT(36.90671119634876 -121.77846078998886) bank169531 +169532 POINT(37.09043843002532 -123.30679621597922) bank169532 +169533 POINT(38.49353105040554 -121.84353968381862) bank169533 +169534 POINT(37.56153154421503 -121.56005674666888) bank169534 +169535 POINT(37.8406231695452 -122.13968034382265) bank169535 +169536 POINT(38.60278564344753 -122.06679091559609) bank169536 +169537 POINT(38.68849787977113 -123.2876917168812) bank169537 +169538 POINT(37.67944550727097 -122.55263061258982) bank169538 +169539 POINT(36.83023104250076 -122.08344318346774) bank169539 +169540 POINT(37.138903523635086 -122.5173701526682) bank169540 +169541 POINT(36.83785764247805 -122.5340480553072) bank169541 +169542 POINT(37.06973247146693 -122.03516342238036) bank169542 +169543 POINT(37.4667006361793 -122.27840596957664) bank169543 +169544 POINT(37.9349392262807 -121.70616830577238) bank169544 +169545 POINT(37.609598991528564 -122.28689330609348) bank169545 +169546 POINT(38.06456276058857 -121.76713019636432) bank169546 +169547 POINT(37.24932592502235 -121.9632785874739) bank169547 +169548 POINT(37.62032560633985 -121.52955974139326) bank169548 +169549 POINT(38.64766691799301 -123.04324987976173) bank169549 +169550 POINT(38.67588898877269 -121.97495422829186) bank169550 +169551 POINT(38.73709852182586 -122.97874252661047) bank169551 +169552 POINT(37.05645698216647 -122.95984832659467) bank169552 +169553 POINT(38.59455173435363 -121.92127000479732) bank169553 +169554 POINT(38.70743059072888 -122.97303310146373) bank169554 +169555 POINT(36.954359963693136 -121.50867448974874) bank169555 +169556 POINT(38.34392288053455 -121.66993414536623) bank169556 +169557 POINT(37.69177878529039 -123.18671175547219) bank169557 +169558 POINT(37.945186234291654 -121.63211956840956) bank169558 +169559 POINT(38.12011123743406 -122.56285352978203) bank169559 +169560 POINT(37.98993749415658 -123.1667603724923) bank169560 +169561 POINT(37.17502215355418 -122.05299748395039) bank169561 +169562 POINT(37.63713918536457 -122.55869728014893) bank169562 +169563 POINT(36.80275273425811 -122.47682814577796) bank169563 +169564 POINT(37.9425287899787 -123.1705524958443) bank169564 +169565 POINT(37.06971307493191 -121.76832759226606) bank169565 +169566 POINT(37.912592120650906 -122.24093945851514) bank169566 +169567 POINT(38.53667191327392 -122.47034922936889) bank169567 +169568 POINT(36.792690393048666 -121.57003763934867) bank169568 +169569 POINT(38.368430299891855 -122.84552417813929) bank169569 +169570 POINT(37.67050795974566 -122.363411995327) bank169570 +169571 POINT(36.82369628298074 -121.82358554863592) bank169571 +169572 POINT(38.670719658342726 -122.45067320913637) bank169572 +169573 POINT(37.16824743646655 -122.39476920696062) bank169573 +169574 POINT(37.05067659154527 -122.02509609733978) bank169574 +169575 POINT(38.444132503925 -121.72888831533365) bank169575 +169576 POINT(37.78371147541072 -121.56315006856492) bank169576 +169577 POINT(38.753929709303115 -122.36649413441401) bank169577 +169578 POINT(37.32933429835706 -121.83791370174461) bank169578 +169579 POINT(38.13168281592809 -121.8226364092655) bank169579 +169580 POINT(37.58748097887589 -122.9617806740144) bank169580 +169581 POINT(37.77052878426378 -122.71889908959515) bank169581 +169582 POINT(37.01733890382622 -122.24070590809993) bank169582 +169583 POINT(38.59797742651672 -121.57540005069207) bank169583 +169584 POINT(38.5528904023695 -123.1241062976647) bank169584 +169585 POINT(37.510137981624204 -123.38467249578336) bank169585 +169586 POINT(37.51574318996558 -122.30236987451572) bank169586 +169587 POINT(37.22604154906459 -122.45234201978396) bank169587 +169588 POINT(38.47749958248527 -121.44020314661049) bank169588 +169589 POINT(37.21681663190994 -121.44673896658288) bank169589 +169590 POINT(37.71409185461013 -121.998551738735) bank169590 +169591 POINT(37.247575071446775 -121.74120677828823) bank169591 +169592 POINT(38.600207876948815 -123.10603643853317) bank169592 +169593 POINT(38.718007323532156 -122.90435276065965) bank169593 +169594 POINT(38.453526832535985 -122.32623630582275) bank169594 +169595 POINT(37.5311478823773 -122.73160175253295) bank169595 +169596 POINT(38.1944145031469 -123.23590523726212) bank169596 +169597 POINT(37.67085447665305 -121.94990180132673) bank169597 +169598 POINT(38.268080713465984 -121.90712574445074) bank169598 +169599 POINT(37.52575702143056 -122.50049632266496) bank169599 +169600 POINT(37.419286739651625 -122.85855009119652) bank169600 +169601 POINT(37.47076963087284 -122.17600632744312) bank169601 +169602 POINT(37.793611155150124 -121.80178183357101) bank169602 +169603 POINT(37.3310421556797 -123.20865352131972) bank169603 +169604 POINT(38.1340116863513 -122.15016911301588) bank169604 +169605 POINT(38.27056162991241 -122.49604391701338) bank169605 +169606 POINT(37.43961181113916 -123.19886567084433) bank169606 +169607 POINT(37.56736359492354 -122.73307809159412) bank169607 +169608 POINT(38.3119524859418 -123.18124914544373) bank169608 +169609 POINT(36.85292031410566 -123.11311989992711) bank169609 +169610 POINT(38.516888254318204 -121.83904679851138) bank169610 +169611 POINT(37.20000451154249 -122.69289861447325) bank169611 +169612 POINT(37.81406650456347 -122.42922996884708) bank169612 +169613 POINT(38.34176672828651 -122.64285578528487) bank169613 +169614 POINT(38.029692347524296 -123.23998449414167) bank169614 +169615 POINT(38.232883584685794 -122.59002721924388) bank169615 +169616 POINT(38.03055626373464 -122.8558453133246) bank169616 +169617 POINT(37.24406326273925 -122.7218797651718) bank169617 +169618 POINT(37.80569059718031 -121.48869064090542) bank169618 +169619 POINT(37.246644061747816 -122.51103305249669) bank169619 +169620 POINT(37.79127472731414 -122.83042976191484) bank169620 +169621 POINT(37.484480713895415 -121.99470081044915) bank169621 +169622 POINT(37.14549923651191 -121.44495672193467) bank169622 +169623 POINT(38.455540101528456 -122.7127469061762) bank169623 +169624 POINT(38.630241755794174 -121.7738768834781) bank169624 +169625 POINT(37.063864461277014 -121.98285718445524) bank169625 +169626 POINT(38.279742145931614 -123.33981164289014) bank169626 +169627 POINT(37.19414770831953 -123.27178770750955) bank169627 +169628 POINT(37.02276049485569 -122.23360637149375) bank169628 +169629 POINT(37.01851486846358 -121.4973079289262) bank169629 +169630 POINT(36.925807319857675 -123.14080324732498) bank169630 +169631 POINT(38.404479189792916 -121.56246003527214) bank169631 +169632 POINT(37.31758277142639 -123.29184786686133) bank169632 +169633 POINT(36.89191931114051 -121.72354713746417) bank169633 +169634 POINT(38.44150392599045 -121.57023965167143) bank169634 +169635 POINT(38.7454119824786 -122.24942822141753) bank169635 +169636 POINT(37.86267604933704 -121.5584661901761) bank169636 +169637 POINT(38.2588146746351 -123.13747037788598) bank169637 +169638 POINT(37.621109743482286 -123.29812576207921) bank169638 +169639 POINT(38.23192219159534 -121.90333554788968) bank169639 +169640 POINT(37.8588021576271 -123.38122459756791) bank169640 +169641 POINT(37.58913864146903 -122.69061524263186) bank169641 +169642 POINT(37.03211751042773 -122.85782282065843) bank169642 +169643 POINT(37.031352057921 -122.46637325317349) bank169643 +169644 POINT(38.55885780946816 -121.71994527095964) bank169644 +169645 POINT(36.80456097073023 -121.66629022082107) bank169645 +169646 POINT(37.834578745763515 -122.07929935734542) bank169646 +169647 POINT(38.341965040514204 -121.50745445627003) bank169647 +169648 POINT(37.75598296228362 -122.39020797854256) bank169648 +169649 POINT(38.206180287444916 -122.81193913014113) bank169649 +169650 POINT(37.159379223851424 -123.06466603568096) bank169650 +169651 POINT(37.926999576890914 -122.91153007911569) bank169651 +169652 POINT(37.487239639086546 -122.79393998360578) bank169652 +169653 POINT(36.86996975102588 -123.17720885487107) bank169653 +169654 POINT(38.1803284808108 -123.12635411293331) bank169654 +169655 POINT(37.78367611667058 -121.6848039003463) bank169655 +169656 POINT(37.57473914430897 -122.5576021108644) bank169656 +169657 POINT(37.65862002331376 -122.5760363341165) bank169657 +169658 POINT(37.74178065635544 -121.74943516589191) bank169658 +169659 POINT(37.27067390865849 -121.6838183579445) bank169659 +169660 POINT(38.31100721973347 -122.25113063268232) bank169660 +169661 POINT(37.81017192560951 -123.20406899798508) bank169661 +169662 POINT(37.706360782439226 -122.93753082062966) bank169662 +169663 POINT(38.40290801557272 -121.70163709617988) bank169663 +169664 POINT(37.48554760383434 -122.55144145186726) bank169664 +169665 POINT(38.63398703040772 -123.41150367894383) bank169665 +169666 POINT(38.145105255490236 -121.79386545040256) bank169666 +169667 POINT(37.675788966445424 -122.01518332399323) bank169667 +169668 POINT(37.88159299663342 -123.29212526874) bank169668 +169669 POINT(38.27225672699976 -122.16044239910066) bank169669 +169670 POINT(38.06870731298519 -121.4750592720597) bank169670 +169671 POINT(38.52898716803065 -121.82366655266915) bank169671 +169672 POINT(37.799060507852744 -121.75969296926796) bank169672 +169673 POINT(37.273226611840535 -121.45697907893629) bank169673 +169674 POINT(38.17683061554537 -121.50475079518421) bank169674 +169675 POINT(36.85432190979679 -123.31524191446015) bank169675 +169676 POINT(37.167716088235935 -121.9422958298067) bank169676 +169677 POINT(37.882844049321115 -121.86158221573041) bank169677 +169678 POINT(36.858124461557125 -123.1152852333295) bank169678 +169679 POINT(38.76132813753347 -121.62154802762879) bank169679 +169680 POINT(36.967537554651656 -122.15710545905151) bank169680 +169681 POINT(37.711174912805475 -122.19705976779744) bank169681 +169682 POINT(37.54609408191904 -122.9879975682587) bank169682 +169683 POINT(38.20157636368102 -122.39203075555625) bank169683 +169684 POINT(37.20430896058601 -123.2114364857571) bank169684 +169685 POINT(37.60656087768397 -122.875300073015) bank169685 +169686 POINT(38.18006826083788 -121.70563788741825) bank169686 +169687 POINT(37.55095615217921 -123.29970641633507) bank169687 +169688 POINT(38.525048370795346 -122.69847563638831) bank169688 +169689 POINT(36.94855609971891 -122.63098491862084) bank169689 +169690 POINT(38.63695398005838 -122.81809170578228) bank169690 +169691 POINT(38.764213312090504 -121.61794253295528) bank169691 +169692 POINT(38.40014163579097 -121.70891376551704) bank169692 +169693 POINT(37.24961288253921 -121.67162943035747) bank169693 +169694 POINT(37.73387451113728 -121.75726752302005) bank169694 +169695 POINT(36.8034220780694 -122.26085936065125) bank169695 +169696 POINT(38.10777567842392 -121.47267555981081) bank169696 +169697 POINT(36.915525501331075 -121.58235228393937) bank169697 +169698 POINT(36.88851232845127 -121.96142757592229) bank169698 +169699 POINT(38.446785551997216 -122.3824117728143) bank169699 +169700 POINT(36.84272634182268 -122.29901939175348) bank169700 +169701 POINT(38.473560012111015 -122.73450402022608) bank169701 +169702 POINT(36.868582080807535 -123.40983370444164) bank169702 +169703 POINT(37.907071612039196 -122.92763296603661) bank169703 +169704 POINT(37.83158981881314 -122.29179272358934) bank169704 +169705 POINT(37.22386644625333 -123.39837399055126) bank169705 +169706 POINT(38.2903485085257 -121.99251018998487) bank169706 +169707 POINT(38.681513075823005 -123.3590843312425) bank169707 +169708 POINT(38.49755075135816 -122.56294870063185) bank169708 +169709 POINT(37.02403051399797 -123.11710333442596) bank169709 +169710 POINT(38.315195203411974 -123.25830633339629) bank169710 +169711 POINT(37.060762808476156 -121.59942516728347) bank169711 +169712 POINT(37.95945044340339 -123.21518457202787) bank169712 +169713 POINT(37.04479091746774 -122.90439271053178) bank169713 +169714 POINT(38.658232729267944 -122.49593293880447) bank169714 +169715 POINT(37.12508354373272 -122.96154444501411) bank169715 +169716 POINT(37.54842048978665 -122.92216571210386) bank169716 +169717 POINT(37.610359669661044 -123.39329250004137) bank169717 +169718 POINT(37.81901867026188 -122.09245088673288) bank169718 +169719 POINT(38.330022818000586 -122.68017185570342) bank169719 +169720 POINT(38.66366405067779 -123.28952093413965) bank169720 +169721 POINT(37.29136179785025 -122.87847751259648) bank169721 +169722 POINT(37.28969831291762 -122.78351012976132) bank169722 +169723 POINT(38.35248915487156 -121.52115099622164) bank169723 +169724 POINT(37.54970070347739 -122.70184781659741) bank169724 +169725 POINT(38.05210072371162 -123.25090114846425) bank169725 +169726 POINT(38.20938378875718 -123.16192204111135) bank169726 +169727 POINT(37.93333995194111 -123.38041258232346) bank169727 +169728 POINT(38.31620532409292 -121.58119085911031) bank169728 +169729 POINT(38.552071891114 -122.02605729477268) bank169729 +169730 POINT(37.606087351749125 -122.68641915253107) bank169730 +169731 POINT(38.687957412058424 -121.96521254345042) bank169731 +169732 POINT(37.80784023127593 -123.19248699297945) bank169732 +169733 POINT(38.22171742024089 -121.9972914579953) bank169733 +169734 POINT(37.093655813411914 -122.29059780318184) bank169734 +169735 POINT(37.91316338640395 -122.23328513476454) bank169735 +169736 POINT(37.4546538790133 -123.33116265238523) bank169736 +169737 POINT(37.32307770871893 -121.7096364866439) bank169737 +169738 POINT(38.00946634664415 -123.06370803803826) bank169738 +169739 POINT(38.757226719214685 -122.87300770756048) bank169739 +169740 POINT(37.96340835412382 -122.40413847489803) bank169740 +169741 POINT(37.230051228468255 -122.51466992751718) bank169741 +169742 POINT(36.81205888429643 -122.70499278947152) bank169742 +169743 POINT(37.239796014405165 -121.92160132589366) bank169743 +169744 POINT(37.91869281531765 -122.24337731961715) bank169744 +169745 POINT(38.05190704315923 -122.79824061790015) bank169745 +169746 POINT(38.6335955664516 -122.60593195994002) bank169746 +169747 POINT(38.14106111649732 -122.16207295306727) bank169747 +169748 POINT(38.16824384130935 -122.66839925880493) bank169748 +169749 POINT(37.557092674355395 -122.7255569853286) bank169749 +169750 POINT(38.240176758911346 -122.86689231328626) bank169750 +169751 POINT(36.931735109961615 -122.32805532840584) bank169751 +169752 POINT(37.19508669236071 -122.2589323445177) bank169752 +169753 POINT(37.304288763072464 -123.29901941304306) bank169753 +169754 POINT(37.10372035484568 -122.11669982839898) bank169754 +169755 POINT(38.630558802410945 -122.02872662570903) bank169755 +169756 POINT(36.87955642573006 -122.03598997764608) bank169756 +169757 POINT(38.503521273430515 -122.423783125312) bank169757 +169758 POINT(37.02430243118724 -123.08600074465824) bank169758 +169759 POINT(38.67084563561994 -122.77680090945282) bank169759 +169760 POINT(38.7661295953245 -122.88230298458679) bank169760 +169761 POINT(37.23167412832572 -123.40179865105857) bank169761 +169762 POINT(38.441505270608616 -122.77014312012236) bank169762 +169763 POINT(37.20675080985881 -121.80839752746708) bank169763 +169764 POINT(37.726742012133734 -122.70171829392775) bank169764 +169765 POINT(37.12397833109859 -122.8854154557606) bank169765 +169766 POINT(37.84065293495817 -121.62302250045225) bank169766 +169767 POINT(38.5184493875863 -122.42393777874263) bank169767 +169768 POINT(37.559713716986145 -122.30933568890309) bank169768 +169769 POINT(36.836981112081375 -121.45723802899029) bank169769 +169770 POINT(38.600382106192846 -122.01610313968546) bank169770 +169771 POINT(37.17275460853657 -121.51028396849425) bank169771 +169772 POINT(38.47549407560527 -122.90612316662673) bank169772 +169773 POINT(37.77822233531961 -122.93233912598335) bank169773 +169774 POINT(37.30976129505357 -123.21058493676429) bank169774 +169775 POINT(38.56189204644332 -121.65444593239283) bank169775 +169776 POINT(37.42585709839411 -122.41698131926246) bank169776 +169777 POINT(38.3437552070729 -122.87830975970253) bank169777 +169778 POINT(38.256194561506035 -122.73600187184624) bank169778 +169779 POINT(38.493026259365074 -123.2014099861551) bank169779 +169780 POINT(38.74418768535924 -121.96887675237323) bank169780 +169781 POINT(38.35267851419896 -122.68487320347577) bank169781 +169782 POINT(37.5555243572816 -123.39065409218384) bank169782 +169783 POINT(37.877009056308836 -121.59664394704288) bank169783 +169784 POINT(38.71871523752289 -121.51820725945477) bank169784 +169785 POINT(37.93594785779484 -122.26503808451552) bank169785 +169786 POINT(38.47377435593462 -122.44813445781364) bank169786 +169787 POINT(37.999193132755416 -123.0658388628455) bank169787 +169788 POINT(38.50553876538659 -123.35911924700933) bank169788 +169789 POINT(38.333280225396926 -121.7765742872389) bank169789 +169790 POINT(37.075181031903504 -121.55077220059623) bank169790 +169791 POINT(38.67448079248095 -122.79831936568789) bank169791 +169792 POINT(37.71607099199349 -122.94289335777688) bank169792 +169793 POINT(37.42625747870971 -122.55097017496627) bank169793 +169794 POINT(38.26957957407521 -121.42200916703072) bank169794 +169795 POINT(36.88698892655675 -122.63422163337319) bank169795 +169796 POINT(37.37393479211767 -121.82247308221629) bank169796 +169797 POINT(38.376334882052966 -123.32530739039493) bank169797 +169798 POINT(37.08420142426852 -121.87026777776248) bank169798 +169799 POINT(38.66887324505951 -123.37533242145025) bank169799 +169800 POINT(37.05115952164326 -121.91890126263961) bank169800 +169801 POINT(38.58091889189874 -122.95549648947329) bank169801 +169802 POINT(37.65355880958866 -123.38618400480492) bank169802 +169803 POINT(37.415886665639285 -122.41983348201009) bank169803 +169804 POINT(36.991971168222385 -122.77443982409471) bank169804 +169805 POINT(37.79703974738 -123.32055851718675) bank169805 +169806 POINT(38.47321240809353 -122.38765409866863) bank169806 +169807 POINT(38.540760620763955 -122.86975386044911) bank169807 +169808 POINT(37.541414755993806 -122.44789166126259) bank169808 +169809 POINT(38.348899137302645 -122.28046206056354) bank169809 +169810 POINT(37.251257786672475 -122.67777962731213) bank169810 +169811 POINT(37.789725963468406 -122.04631755328924) bank169811 +169812 POINT(38.2622526518485 -121.95477387295355) bank169812 +169813 POINT(37.002016435918726 -122.07501000980373) bank169813 +169814 POINT(38.475563738610376 -122.65952227797702) bank169814 +169815 POINT(38.23084914229454 -121.92384777879171) bank169815 +169816 POINT(38.59463967810479 -121.68449454031337) bank169816 +169817 POINT(37.390805268069585 -123.12380991724802) bank169817 +169818 POINT(38.38339137804326 -121.70904233010828) bank169818 +169819 POINT(38.16329529479479 -123.10954408211401) bank169819 +169820 POINT(37.676038068343864 -122.30987958749644) bank169820 +169821 POINT(36.97522326896882 -121.44559179500212) bank169821 +169822 POINT(38.10174833012112 -122.37665871029105) bank169822 +169823 POINT(38.600077924856606 -122.40815416883642) bank169823 +169824 POINT(37.29602185748716 -122.51638605526182) bank169824 +169825 POINT(38.5778958732281 -122.3519886836656) bank169825 +169826 POINT(37.52070398088523 -122.26528791037434) bank169826 +169827 POINT(38.66813994558159 -122.5829648055015) bank169827 +169828 POINT(38.55290181495231 -122.78142784702963) bank169828 +169829 POINT(38.04864304931687 -121.82487939826953) bank169829 +169830 POINT(38.49398499498373 -122.88561734063929) bank169830 +169831 POINT(37.955466201950614 -122.8990436814734) bank169831 +169832 POINT(38.10934537840477 -121.71919938373607) bank169832 +169833 POINT(37.58675132885086 -123.18631373361993) bank169833 +169834 POINT(38.24185245363982 -123.11569313250016) bank169834 +169835 POINT(37.26661328830259 -121.45957476569569) bank169835 +169836 POINT(38.026423172052304 -122.85394365596841) bank169836 +169837 POINT(37.40402893498503 -122.23281591573652) bank169837 +169838 POINT(38.58372083326624 -122.35192952711837) bank169838 +169839 POINT(36.98879145438163 -122.01604662114492) bank169839 +169840 POINT(37.69054254460113 -122.53523890396596) bank169840 +169841 POINT(38.321113688693366 -122.50030578437119) bank169841 +169842 POINT(37.96272029539839 -122.72080710888606) bank169842 +169843 POINT(37.205309636537 -123.15257315648249) bank169843 +169844 POINT(37.44194451297138 -122.091045934294) bank169844 +169845 POINT(38.68947974623354 -122.13985970231136) bank169845 +169846 POINT(37.81601314552981 -122.68231299970094) bank169846 +169847 POINT(37.11986899324214 -122.52258347915419) bank169847 +169848 POINT(37.482714131504345 -121.59170465999392) bank169848 +169849 POINT(37.92316951546939 -122.6944255815736) bank169849 +169850 POINT(38.52481315727173 -122.40040895744708) bank169850 +169851 POINT(37.37454768437294 -123.2270854604734) bank169851 +169852 POINT(37.155100736781606 -122.69772125397776) bank169852 +169853 POINT(38.343970975243366 -122.3503426393889) bank169853 +169854 POINT(38.219906750583604 -122.49401967625668) bank169854 +169855 POINT(38.757147819963 -123.20570022478297) bank169855 +169856 POINT(38.32314887007369 -122.88587215203877) bank169856 +169857 POINT(38.000103683232844 -122.34399259231229) bank169857 +169858 POINT(37.6205544676608 -121.5904738617505) bank169858 +169859 POINT(38.581845955897414 -122.80560802951878) bank169859 +169860 POINT(38.08307718990234 -122.85372475778298) bank169860 +169861 POINT(38.06890589716998 -122.5624336786013) bank169861 +169862 POINT(36.877811471337644 -122.49142459341275) bank169862 +169863 POINT(37.09214816643819 -122.81311180323365) bank169863 +169864 POINT(38.475716768051086 -123.11257693623051) bank169864 +169865 POINT(38.56856535065127 -121.71820931964258) bank169865 +169866 POINT(37.5862453392461 -123.40470735675667) bank169866 +169867 POINT(38.68327788538139 -122.26019539408807) bank169867 +169868 POINT(36.8117959024933 -121.97237311095664) bank169868 +169869 POINT(37.09865401795006 -122.52905524046488) bank169869 +169870 POINT(36.805403206838804 -122.3298896765942) bank169870 +169871 POINT(36.924184374555196 -121.58700933245838) bank169871 +169872 POINT(38.01086839937928 -123.09808380236892) bank169872 +169873 POINT(37.127901840464325 -121.83662164859031) bank169873 +169874 POINT(37.44038049786093 -122.56825166251134) bank169874 +169875 POINT(37.63647303524819 -121.65880485696403) bank169875 +169876 POINT(38.710123260966164 -123.10544308466672) bank169876 +169877 POINT(37.92250857389867 -122.09036949516077) bank169877 +169878 POINT(36.90512030864182 -121.74650145231074) bank169878 +169879 POINT(36.832932561499575 -121.58809647614355) bank169879 +169880 POINT(36.915516717718404 -122.17493616846726) bank169880 +169881 POINT(37.83709063485017 -121.91616566223828) bank169881 +169882 POINT(36.877280391407886 -122.12961057427583) bank169882 +169883 POINT(38.00256786295267 -122.50527339111964) bank169883 +169884 POINT(38.11728212945543 -122.10361761392932) bank169884 +169885 POINT(38.74997469355102 -123.22936606701315) bank169885 +169886 POINT(37.376346391490664 -122.4568946006173) bank169886 +169887 POINT(38.098669002710714 -121.58800039837848) bank169887 +169888 POINT(37.21273915846554 -122.00898337250155) bank169888 +169889 POINT(38.549753137742364 -122.23901810126048) bank169889 +169890 POINT(38.317131945879126 -121.59285686405728) bank169890 +169891 POINT(38.331821055459685 -123.14033873849847) bank169891 +169892 POINT(37.45849444378618 -121.63673438579697) bank169892 +169893 POINT(38.26336379188754 -122.29677889091921) bank169893 +169894 POINT(38.09074583536736 -123.25951456547233) bank169894 +169895 POINT(38.22196754722354 -122.6055804231286) bank169895 +169896 POINT(38.59854077881675 -123.28265527121191) bank169896 +169897 POINT(37.11933121707894 -122.98979020016067) bank169897 +169898 POINT(37.48717604714272 -121.50208930174394) bank169898 +169899 POINT(37.667139997370626 -123.12244234962732) bank169899 +169900 POINT(36.941486546179235 -121.46529819224165) bank169900 +169901 POINT(37.92243141867182 -121.45459183468203) bank169901 +169902 POINT(38.140947409247865 -123.2030267958188) bank169902 +169903 POINT(38.1747893968118 -121.4726592843565) bank169903 +169904 POINT(37.74809911487614 -122.27416528654798) bank169904 +169905 POINT(38.45080200385355 -122.33382309092747) bank169905 +169906 POINT(36.82508986738952 -121.57313992981408) bank169906 +169907 POINT(37.903576947783094 -122.27415662274322) bank169907 +169908 POINT(37.532425362226775 -121.48415092838378) bank169908 +169909 POINT(36.92874231828386 -122.49553176459324) bank169909 +169910 POINT(38.676963953210084 -122.44364769611578) bank169910 +169911 POINT(38.35937734962253 -121.5144844486695) bank169911 +169912 POINT(36.822933971928904 -122.66715479771526) bank169912 +169913 POINT(38.11629269159335 -121.54606868765735) bank169913 +169914 POINT(38.006783940305375 -121.49218323099574) bank169914 +169915 POINT(37.97707076379744 -122.90556244073235) bank169915 +169916 POINT(38.155615772927966 -122.28477143403074) bank169916 +169917 POINT(37.67194047535938 -122.08402589610886) bank169917 +169918 POINT(38.75119311472628 -123.11348192454341) bank169918 +169919 POINT(37.618148676883095 -122.36617651122562) bank169919 +169920 POINT(37.05701160196399 -123.101418011573) bank169920 +169921 POINT(37.19690622227524 -123.16098759473513) bank169921 +169922 POINT(37.602617308068986 -123.2587233789384) bank169922 +169923 POINT(37.41601045302486 -123.02162470295336) bank169923 +169924 POINT(38.0681891825995 -122.546837144757) bank169924 +169925 POINT(37.71191003351577 -121.8363976225062) bank169925 +169926 POINT(37.49414999530076 -122.62556170677864) bank169926 +169927 POINT(37.31083270752692 -122.87251066286946) bank169927 +169928 POINT(37.42949053191047 -121.86301196693626) bank169928 +169929 POINT(37.32265156868153 -122.54361463603189) bank169929 +169930 POINT(38.04619142404552 -121.7287002404768) bank169930 +169931 POINT(37.48681373763854 -122.35811445867945) bank169931 +169932 POINT(38.41818894769686 -121.45436277953199) bank169932 +169933 POINT(37.80109419129798 -123.20860744776252) bank169933 +169934 POINT(37.60359203337458 -122.43764492881368) bank169934 +169935 POINT(37.27959729380444 -122.30115691122683) bank169935 +169936 POINT(38.33135692198515 -123.11382087754177) bank169936 +169937 POINT(38.68832907845688 -122.37089160037974) bank169937 +169938 POINT(37.68699617595674 -121.8135517386066) bank169938 +169939 POINT(37.69436210012088 -122.07806481329439) bank169939 +169940 POINT(38.65535234286327 -123.41867892264582) bank169940 +169941 POINT(38.50851641398882 -123.03117742601988) bank169941 +169942 POINT(37.37293943288584 -121.5090802362383) bank169942 +169943 POINT(36.81459292797775 -123.09860695404636) bank169943 +169944 POINT(37.118909860732956 -121.69961012821926) bank169944 +169945 POINT(38.185768852467824 -121.52897130659191) bank169945 +169946 POINT(37.11561396443735 -122.73488958839505) bank169946 +169947 POINT(38.4546590942451 -121.63348477689719) bank169947 +169948 POINT(37.721023785112706 -122.36106087105692) bank169948 +169949 POINT(37.64333006386451 -121.73574228401866) bank169949 +169950 POINT(37.92916001462426 -122.52107797412728) bank169950 +169951 POINT(38.64361993919832 -122.95553698804142) bank169951 +169952 POINT(38.15344500926643 -121.95779077147239) bank169952 +169953 POINT(37.39786457567461 -121.98123373898065) bank169953 +169954 POINT(38.583622473605935 -122.74964908236423) bank169954 +169955 POINT(38.04887709439505 -122.22253751718125) bank169955 +169956 POINT(38.421956513738806 -122.81099380720414) bank169956 +169957 POINT(37.853375509382175 -123.00140476954938) bank169957 +169958 POINT(36.91466257557099 -123.16206954514473) bank169958 +169959 POINT(38.61312661391719 -121.61489064644115) bank169959 +169960 POINT(36.85579166497326 -121.74804038483961) bank169960 +169961 POINT(38.58570699586824 -121.53962587488884) bank169961 +169962 POINT(37.657487452522375 -121.4698979943038) bank169962 +169963 POINT(38.48700158219882 -121.742343285228) bank169963 +169964 POINT(37.22960359070439 -122.76726783894688) bank169964 +169965 POINT(37.183294044033964 -122.92223235654134) bank169965 +169966 POINT(37.68983322644548 -122.78349503319494) bank169966 +169967 POINT(36.85480719828936 -121.45528385951965) bank169967 +169968 POINT(37.401969517456465 -122.70746846290524) bank169968 +169969 POINT(37.29702874105782 -122.7553659198245) bank169969 +169970 POINT(38.24418612914452 -122.3992417745214) bank169970 +169971 POINT(37.200321207658305 -121.55037367984865) bank169971 +169972 POINT(37.30456548929525 -122.06412205324969) bank169972 +169973 POINT(37.68334238847342 -121.57699222703958) bank169973 +169974 POINT(37.90991109263827 -122.73133051097557) bank169974 +169975 POINT(38.759763793327146 -122.89598417025256) bank169975 +169976 POINT(37.124996208445275 -121.60068797017391) bank169976 +169977 POINT(36.84995764668341 -121.95313180215736) bank169977 +169978 POINT(37.27818227743211 -122.66716570154517) bank169978 +169979 POINT(37.47887880081063 -122.39715061395049) bank169979 +169980 POINT(38.06988965577848 -122.18380583863559) bank169980 +169981 POINT(38.14825134887167 -122.32803232819396) bank169981 +169982 POINT(37.23921942720498 -123.09591640403981) bank169982 +169983 POINT(38.61772293184001 -122.74739603969019) bank169983 +169984 POINT(38.336807061381634 -122.5431086425278) bank169984 +169985 POINT(37.42256709003964 -121.55368672197994) bank169985 +169986 POINT(37.76868739767482 -121.8468490910479) bank169986 +169987 POINT(38.00814214977535 -121.60658201819773) bank169987 +169988 POINT(37.9644742279507 -121.47286772548601) bank169988 +169989 POINT(37.97055613642302 -123.36500796780261) bank169989 +169990 POINT(38.72499952025328 -123.30985326466516) bank169990 +169991 POINT(38.035274411619234 -121.53635200079981) bank169991 +169992 POINT(38.049245001353704 -122.15306450499733) bank169992 +169993 POINT(37.34930881842162 -122.37401799996431) bank169993 +169994 POINT(37.14295381300139 -122.9874092733386) bank169994 +169995 POINT(37.363543380609364 -122.97274066203755) bank169995 +169996 POINT(38.681148408476645 -123.1833849097229) bank169996 +169997 POINT(37.65480575769371 -122.19824543835378) bank169997 +169998 POINT(38.2865112172519 -123.38218471733188) bank169998 +169999 POINT(37.735601176378395 -122.06187410776738) bank169999 +170000 POINT(38.44882986135131 -123.12717216472073) bank170000 +170001 POINT(37.56891660974816 -122.01860233922154) bank170001 +170002 POINT(37.06619831388948 -123.37085552903551) bank170002 +170003 POINT(37.416630374855 -121.73480243269161) bank170003 +170004 POINT(37.28463052353436 -122.67801816115764) bank170004 +170005 POINT(38.10817741166999 -123.02812892862173) bank170005 +170006 POINT(37.898597467078716 -121.68074911566116) bank170006 +170007 POINT(38.294838185841876 -123.26216706803525) bank170007 +170008 POINT(37.868248790382275 -122.30848851963974) bank170008 +170009 POINT(38.16305527287102 -122.90099565481819) bank170009 +170010 POINT(38.51812815872483 -121.9931342661353) bank170010 +170011 POINT(37.168330208637414 -123.07720743428315) bank170011 +170012 POINT(37.90880364832745 -121.80010067868648) bank170012 +170013 POINT(38.45096318663513 -122.80011348476285) bank170013 +170014 POINT(38.312964092261296 -122.89776063305061) bank170014 +170015 POINT(37.012628222851546 -122.90043576510087) bank170015 +170016 POINT(36.818000531148506 -122.95145523046422) bank170016 +170017 POINT(38.564889513929394 -121.86618037207803) bank170017 +170018 POINT(38.4726860657822 -121.74073948113568) bank170018 +170019 POINT(38.10157973862735 -122.72437746875552) bank170019 +170020 POINT(37.12772997079883 -122.72797164116238) bank170020 +170021 POINT(37.781887816208936 -122.8447987946806) bank170021 +170022 POINT(38.472496782567525 -123.32509896659018) bank170022 +170023 POINT(38.375662273068485 -123.09303277804273) bank170023 +170024 POINT(36.86981975250248 -121.6374689388822) bank170024 +170025 POINT(36.91418576510338 -122.68466875633922) bank170025 +170026 POINT(38.42583831420999 -121.90482371804038) bank170026 +170027 POINT(37.37430235470259 -123.40069954586112) bank170027 +170028 POINT(37.13376881686073 -121.99722215198199) bank170028 +170029 POINT(37.51877132501943 -122.92720127488224) bank170029 +170030 POINT(36.83488947541528 -122.40196641550517) bank170030 +170031 POINT(38.32355512467383 -122.59967894171956) bank170031 +170032 POINT(38.66679253092788 -122.0921297447955) bank170032 +170033 POINT(36.79103826774067 -121.49002556756919) bank170033 +170034 POINT(38.21064459979841 -123.01571112004738) bank170034 +170035 POINT(36.95373371911428 -122.87200365070571) bank170035 +170036 POINT(37.4697066575823 -122.36670100457661) bank170036 +170037 POINT(37.7967317106629 -121.69445720860764) bank170037 +170038 POINT(38.233330418086496 -121.51588380565947) bank170038 +170039 POINT(38.19033367914322 -122.04113256048419) bank170039 +170040 POINT(37.994210137873246 -121.6753254186876) bank170040 +170041 POINT(36.90085041586772 -121.91686977235112) bank170041 +170042 POINT(38.4115313344451 -121.6514913122773) bank170042 +170043 POINT(37.39578283809668 -122.68848544607634) bank170043 +170044 POINT(38.68776502223684 -123.26943078343318) bank170044 +170045 POINT(37.201832764047566 -122.84421007809544) bank170045 +170046 POINT(36.90364650395307 -123.28114692601773) bank170046 +170047 POINT(38.49337669582899 -122.5464413891143) bank170047 +170048 POINT(38.102316476886294 -121.92880172088702) bank170048 +170049 POINT(36.80222102396655 -123.24017761618806) bank170049 +170050 POINT(37.846724081760065 -122.67803113267415) bank170050 +170051 POINT(38.1566021079052 -121.92128699582521) bank170051 +170052 POINT(37.72998827160869 -122.53735875660216) bank170052 +170053 POINT(38.010506566522324 -122.15088104881187) bank170053 +170054 POINT(38.308671530468835 -123.1823736031182) bank170054 +170055 POINT(38.050491045988466 -122.6807709136101) bank170055 +170056 POINT(37.97774785009728 -123.048494583971) bank170056 +170057 POINT(38.26561785651989 -122.55854671663872) bank170057 +170058 POINT(36.83363420242511 -123.40337836832717) bank170058 +170059 POINT(37.022524046953386 -123.0348229290707) bank170059 +170060 POINT(37.44700368978405 -122.76044837244342) bank170060 +170061 POINT(37.648993726523656 -122.30956090197509) bank170061 +170062 POINT(37.41828506695909 -122.15184165702165) bank170062 +170063 POINT(37.57789246144567 -121.81887111602953) bank170063 +170064 POINT(38.19756379909305 -121.43512760915193) bank170064 +170065 POINT(37.7859473641056 -122.0506160310095) bank170065 +170066 POINT(37.49374368543767 -121.80303721463355) bank170066 +170067 POINT(37.69472505625957 -122.65447535621917) bank170067 +170068 POINT(38.160051828947054 -122.98208051393344) bank170068 +170069 POINT(37.41940807565819 -122.4039750701722) bank170069 +170070 POINT(36.958798116708834 -122.39089214877329) bank170070 +170071 POINT(36.8730717691082 -121.60192580552147) bank170071 +170072 POINT(38.57443439503683 -123.12243851475425) bank170072 +170073 POINT(38.26832504354008 -121.9657242744303) bank170073 +170074 POINT(37.164582089669686 -122.1786960865427) bank170074 +170075 POINT(36.83328321692998 -123.14691570669072) bank170075 +170076 POINT(36.98703899636417 -123.29202114438661) bank170076 +170077 POINT(38.74928722865729 -121.97437942661473) bank170077 +170078 POINT(36.78113922798402 -122.01076070744396) bank170078 +170079 POINT(37.35755564065717 -123.03257821752909) bank170079 +170080 POINT(37.50504473388653 -121.52739043653726) bank170080 +170081 POINT(38.15990828688887 -122.15107766573591) bank170081 +170082 POINT(37.79266770627974 -123.03944302964184) bank170082 +170083 POINT(38.07376652489771 -122.76293838447287) bank170083 +170084 POINT(37.49083344924103 -122.85191421187704) bank170084 +170085 POINT(38.02753273697962 -122.78365454094558) bank170085 +170086 POINT(37.09607794946987 -123.24171771606011) bank170086 +170087 POINT(37.09942184186924 -123.33100349047159) bank170087 +170088 POINT(38.7717357006536 -123.11936157256301) bank170088 +170089 POINT(38.61867184437367 -123.05460945697938) bank170089 +170090 POINT(38.19176869578397 -122.13889852211173) bank170090 +170091 POINT(37.02661068861349 -122.36495627206402) bank170091 +170092 POINT(37.323281225187245 -123.06538604518423) bank170092 +170093 POINT(38.17687506141132 -123.34027973704417) bank170093 +170094 POINT(37.63264194752955 -121.61082912974126) bank170094 +170095 POINT(37.2294606296546 -122.02747610645358) bank170095 +170096 POINT(37.01301279818912 -121.61870105871171) bank170096 +170097 POINT(36.97973248324637 -121.66902081619487) bank170097 +170098 POINT(37.102402657538974 -122.24715739979112) bank170098 +170099 POINT(38.56998112619229 -122.94223202087632) bank170099 +170100 POINT(37.068567532842 -122.56618582776616) bank170100 +170101 POINT(38.157710426114846 -122.74510858697924) bank170101 +170102 POINT(38.52789565232222 -123.37825659808406) bank170102 +170103 POINT(38.695313055144844 -122.45542153230537) bank170103 +170104 POINT(38.231565836480556 -122.49992096174373) bank170104 +170105 POINT(38.595572686361265 -122.70090544552521) bank170105 +170106 POINT(37.123182613611604 -121.49280592673433) bank170106 +170107 POINT(37.152594897186326 -122.19787744845418) bank170107 +170108 POINT(37.80648170605165 -122.74451750591993) bank170108 +170109 POINT(37.16540973842904 -122.54229092422258) bank170109 +170110 POINT(38.24930923390944 -122.51807313477494) bank170110 +170111 POINT(38.056809845356334 -121.61434002538651) bank170111 +170112 POINT(37.90611142559937 -122.96779375726564) bank170112 +170113 POINT(38.10561866556069 -121.58318263694959) bank170113 +170114 POINT(38.03978500599241 -121.5564446976939) bank170114 +170115 POINT(38.056080935323806 -122.49940511096392) bank170115 +170116 POINT(37.88760247434184 -121.62053103460326) bank170116 +170117 POINT(37.76468995238382 -121.50472737617524) bank170117 +170118 POINT(37.91530922225642 -121.618591138154) bank170118 +170119 POINT(37.53630710388006 -122.9551667485767) bank170119 +170120 POINT(38.45483465738643 -122.74540885617668) bank170120 +170121 POINT(38.28770688694368 -122.64242485728623) bank170121 +170122 POINT(37.146695123723056 -122.06825871612185) bank170122 +170123 POINT(36.918525487494094 -121.55441945001346) bank170123 +170124 POINT(37.689755320142424 -122.58418077723573) bank170124 +170125 POINT(38.1892346233019 -123.36757662564719) bank170125 +170126 POINT(38.36627322293315 -123.39141127461612) bank170126 +170127 POINT(37.77901793520453 -122.05941998505931) bank170127 +170128 POINT(37.17563443675053 -122.61206171969961) bank170128 +170129 POINT(37.0627594327858 -122.72637564754271) bank170129 +170130 POINT(38.25645022369702 -123.32309455853266) bank170130 +170131 POINT(38.262310531716956 -121.48059303360037) bank170131 +170132 POINT(37.77580679156622 -122.53461018180718) bank170132 +170133 POINT(37.96867518153215 -122.25601689993321) bank170133 +170134 POINT(38.607030129986356 -122.2371294231272) bank170134 +170135 POINT(37.263257509128344 -121.71414635512151) bank170135 +170136 POINT(37.73524058166006 -121.44579363255866) bank170136 +170137 POINT(38.24307499864003 -122.98503731649498) bank170137 +170138 POINT(37.92036585342332 -122.90043446026505) bank170138 +170139 POINT(38.72102216026614 -122.17444705877925) bank170139 +170140 POINT(37.30782376808369 -121.88846842960275) bank170140 +170141 POINT(38.25434549414719 -121.87256515698074) bank170141 +170142 POINT(37.72690795813156 -122.37618779954356) bank170142 +170143 POINT(38.2378580697876 -122.99076018743614) bank170143 +170144 POINT(38.311568343961156 -122.64779166508148) bank170144 +170145 POINT(38.73424356706528 -121.61393153299427) bank170145 +170146 POINT(37.64497555342851 -123.33936490497449) bank170146 +170147 POINT(36.85156703154859 -123.11112258687741) bank170147 +170148 POINT(38.31519153862404 -122.1002259721644) bank170148 +170149 POINT(38.51817139767568 -121.70405135541888) bank170149 +170150 POINT(37.91228897458886 -123.10018986408089) bank170150 +170151 POINT(36.996245364739686 -121.9540409984831) bank170151 +170152 POINT(37.082183155687225 -121.9074618008892) bank170152 +170153 POINT(37.846295149865504 -122.06419646172633) bank170153 +170154 POINT(38.72094867339935 -122.55639136362765) bank170154 +170155 POINT(37.82970836352559 -122.37956638200474) bank170155 +170156 POINT(37.12396249145421 -122.1384226402741) bank170156 +170157 POINT(36.94780251940327 -121.56250149308346) bank170157 +170158 POINT(37.55895989736799 -123.40580236979508) bank170158 +170159 POINT(36.94064030844916 -122.86186134913983) bank170159 +170160 POINT(38.721182573479574 -121.8740781455561) bank170160 +170161 POINT(36.813340815667246 -122.20142538658652) bank170161 +170162 POINT(37.276785129337334 -122.10349739134894) bank170162 +170163 POINT(38.25553844771322 -121.69672753998384) bank170163 +170164 POINT(37.29915487847413 -123.27694485937522) bank170164 +170165 POINT(37.969627378480624 -123.32988452424142) bank170165 +170166 POINT(37.04705168200371 -123.12708291092873) bank170166 +170167 POINT(38.35702168738202 -122.41374189710969) bank170167 +170168 POINT(36.988303277891205 -122.67164647563327) bank170168 +170169 POINT(38.44970169784004 -121.81368008798742) bank170169 +170170 POINT(38.03144303719682 -122.36565694000849) bank170170 +170171 POINT(38.751009704065844 -123.15390978530137) bank170171 +170172 POINT(38.56130005702236 -121.64329454478194) bank170172 +170173 POINT(38.52354158242574 -121.72817564079207) bank170173 +170174 POINT(37.78963478584402 -123.05060320694004) bank170174 +170175 POINT(37.642415459148445 -121.48150836878503) bank170175 +170176 POINT(36.85369139787585 -121.54610498105431) bank170176 +170177 POINT(37.4362731562823 -121.51687871860727) bank170177 +170178 POINT(38.46065965699749 -121.49196504884362) bank170178 +170179 POINT(37.419524081623535 -122.26060301596586) bank170179 +170180 POINT(37.349677431593335 -123.11032931328614) bank170180 +170181 POINT(37.551172754328356 -122.87613719860954) bank170181 +170182 POINT(36.96282315846758 -123.27020581057229) bank170182 +170183 POINT(36.79054551968738 -123.04945345258797) bank170183 +170184 POINT(37.484189463457426 -122.24199161707672) bank170184 +170185 POINT(37.11308287889393 -123.35650877988903) bank170185 +170186 POINT(37.94277532505851 -122.28844120543408) bank170186 +170187 POINT(38.01038354212702 -122.24177569230535) bank170187 +170188 POINT(38.399429464278946 -122.4453337169599) bank170188 +170189 POINT(37.65571769755972 -122.45840098562398) bank170189 +170190 POINT(38.34284563430097 -121.6852740764348) bank170190 +170191 POINT(37.67430173815925 -122.94319232723501) bank170191 +170192 POINT(38.08103648283536 -122.07663439446827) bank170192 +170193 POINT(38.28883682619618 -123.28124359806604) bank170193 +170194 POINT(37.32056339183604 -122.40160426957613) bank170194 +170195 POINT(37.92937790304131 -122.30371009179092) bank170195 +170196 POINT(38.68460546137821 -122.41688876925937) bank170196 +170197 POINT(36.80447056437328 -122.89724007891502) bank170197 +170198 POINT(38.727825558267654 -121.50605235534402) bank170198 +170199 POINT(37.52331653850093 -121.81175838782067) bank170199 +170200 POINT(38.39495006431939 -121.59365425223918) bank170200 +170201 POINT(37.862676679391946 -121.73843120190121) bank170201 +170202 POINT(37.77029055413298 -122.77435583070987) bank170202 +170203 POINT(37.22846553865077 -122.2605139795179) bank170203 +170204 POINT(36.78620887681209 -122.85746044435979) bank170204 +170205 POINT(37.64237495657717 -121.6513635802397) bank170205 +170206 POINT(37.83926847557061 -121.79236884468841) bank170206 +170207 POINT(38.53673190868513 -123.11827908080832) bank170207 +170208 POINT(37.009460365314204 -122.44205631185349) bank170208 +170209 POINT(38.548170853690365 -122.62231863560903) bank170209 +170210 POINT(38.03887755866919 -122.13469460679786) bank170210 +170211 POINT(37.10347968893528 -121.84213413224023) bank170211 +170212 POINT(37.48998427608633 -122.41978061295436) bank170212 +170213 POINT(37.28634265625778 -121.6076104750113) bank170213 +170214 POINT(37.370323054280846 -123.0448670733367) bank170214 +170215 POINT(38.3885438407735 -121.54346885463242) bank170215 +170216 POINT(37.66371552988742 -122.2819588639325) bank170216 +170217 POINT(37.05890414341838 -122.25899518852493) bank170217 +170218 POINT(37.9235371946103 -122.19352312218793) bank170218 +170219 POINT(37.50620773270499 -122.82843001925107) bank170219 +170220 POINT(37.580802759914235 -123.15101169240846) bank170220 +170221 POINT(38.19339529579111 -122.18575200291338) bank170221 +170222 POINT(37.084211790026984 -123.37872181727874) bank170222 +170223 POINT(37.79802172472758 -122.5089339068857) bank170223 +170224 POINT(37.38118658206006 -122.29742965639733) bank170224 +170225 POINT(37.50645242016251 -122.97225164868932) bank170225 +170226 POINT(36.958499874870576 -122.03355774243339) bank170226 +170227 POINT(38.24182562531909 -122.08789499614527) bank170227 +170228 POINT(36.799788589832005 -123.37630571727884) bank170228 +170229 POINT(38.505801015550254 -122.40445360354097) bank170229 +170230 POINT(38.556807856440024 -121.7439940706184) bank170230 +170231 POINT(38.231145054621734 -122.82011076454003) bank170231 +170232 POINT(37.404453376551864 -122.89203252350676) bank170232 +170233 POINT(38.0138575118281 -123.36477332781733) bank170233 +170234 POINT(37.7749880010076 -122.31312124551589) bank170234 +170235 POINT(38.40907081103893 -122.92520776704863) bank170235 +170236 POINT(37.986928124360176 -121.97294053060452) bank170236 +170237 POINT(37.17252505901909 -123.25926668010702) bank170237 +170238 POINT(37.69412482445315 -123.27439259011294) bank170238 +170239 POINT(38.40329575626697 -122.83660966409143) bank170239 +170240 POINT(38.131434098823284 -123.29769884322987) bank170240 +170241 POINT(38.2796903426334 -121.70408860506927) bank170241 +170242 POINT(37.412113103019855 -123.14221939570375) bank170242 +170243 POINT(37.76509745611424 -121.86887305650758) bank170243 +170244 POINT(36.80103826411537 -122.7841294233788) bank170244 +170245 POINT(38.178900131484994 -121.84230432373464) bank170245 +170246 POINT(37.40533217594932 -122.8507917894073) bank170246 +170247 POINT(38.09203983892218 -121.6463436093086) bank170247 +170248 POINT(36.904982387946276 -122.0817197535607) bank170248 +170249 POINT(37.179932956767004 -122.46089558837203) bank170249 +170250 POINT(37.878117625171924 -121.95371673627166) bank170250 +170251 POINT(37.291609528439395 -123.2619373846297) bank170251 +170252 POINT(38.25239754338031 -122.78936504852332) bank170252 +170253 POINT(37.68007078344024 -122.43749253384645) bank170253 +170254 POINT(38.47692640403497 -121.9047102817174) bank170254 +170255 POINT(36.896601386715524 -121.53409426356478) bank170255 +170256 POINT(37.95583649730756 -123.07389275091532) bank170256 +170257 POINT(38.13608166327561 -123.21747237581737) bank170257 +170258 POINT(37.48446521997344 -122.62762286270467) bank170258 +170259 POINT(38.26803910322492 -122.91488278354959) bank170259 +170260 POINT(37.17267912919089 -122.53870843023354) bank170260 +170261 POINT(38.11990487945075 -123.21766006770609) bank170261 +170262 POINT(38.4612374428401 -121.75231404919326) bank170262 +170263 POINT(38.61130791419 -122.80633191909635) bank170263 +170264 POINT(37.73402674432206 -122.43994530305679) bank170264 +170265 POINT(36.871694092121416 -122.29667661151888) bank170265 +170266 POINT(37.368567542304284 -121.93409151655827) bank170266 +170267 POINT(36.972669265441205 -121.63309298020629) bank170267 +170268 POINT(37.28077620876938 -121.59423899363237) bank170268 +170269 POINT(37.718094380116966 -122.07952409524702) bank170269 +170270 POINT(37.19223963534377 -122.85570478502174) bank170270 +170271 POINT(37.41362216598429 -122.25972089670675) bank170271 +170272 POINT(38.39126795225143 -122.11371513741433) bank170272 +170273 POINT(37.45880545152533 -121.83338003490964) bank170273 +170274 POINT(36.85709040875758 -122.03142907741359) bank170274 +170275 POINT(37.227180984200444 -121.79425341853417) bank170275 +170276 POINT(37.156471529712874 -122.07780643357698) bank170276 +170277 POINT(36.78120016590052 -123.13984428751498) bank170277 +170278 POINT(38.561994211801405 -122.03579082419245) bank170278 +170279 POINT(38.11309950570199 -121.50360391985235) bank170279 +170280 POINT(37.04554604972379 -122.40003044255675) bank170280 +170281 POINT(38.716335712828794 -121.69762800253979) bank170281 +170282 POINT(37.02251349194608 -122.25132520658367) bank170282 +170283 POINT(37.75221143135219 -122.70223410171036) bank170283 +170284 POINT(37.17696502509025 -121.94432414219632) bank170284 +170285 POINT(37.03875830176631 -122.21750818626306) bank170285 +170286 POINT(38.590774938898925 -121.67506365149626) bank170286 +170287 POINT(38.019806381058544 -122.5841700528597) bank170287 +170288 POINT(38.75288562649969 -123.0501476527991) bank170288 +170289 POINT(38.61649599103247 -121.63790219492715) bank170289 +170290 POINT(38.38132803220431 -122.21829839841803) bank170290 +170291 POINT(38.613396078690585 -121.66690850902329) bank170291 +170292 POINT(37.95931109156256 -122.56753507537196) bank170292 +170293 POINT(37.52392754102837 -121.83616013818211) bank170293 +170294 POINT(37.26036037634681 -122.8123284669743) bank170294 +170295 POINT(38.16268093829037 -122.61688240818768) bank170295 +170296 POINT(37.25241706516123 -122.69037012338214) bank170296 +170297 POINT(37.90050711131787 -121.993137823273) bank170297 +170298 POINT(38.25666615866696 -122.37478242887761) bank170298 +170299 POINT(37.044914751923244 -123.32144849467738) bank170299 +170300 POINT(37.27672545864709 -123.28872548146407) bank170300 +170301 POINT(36.80679352087868 -122.34514578067056) bank170301 +170302 POINT(38.740061073734964 -121.75791046182366) bank170302 +170303 POINT(37.13884317008411 -121.56768355229201) bank170303 +170304 POINT(37.06739538621586 -122.75543378056778) bank170304 +170305 POINT(38.10068165041517 -122.72304130688094) bank170305 +170306 POINT(37.51824023226709 -123.26661740903603) bank170306 +170307 POINT(36.81167414225917 -123.09529075468615) bank170307 +170308 POINT(37.13878089669684 -122.58441165762444) bank170308 +170309 POINT(38.124777040388665 -122.31086515872155) bank170309 +170310 POINT(37.12568427870403 -121.99491390627465) bank170310 +170311 POINT(38.45675758378838 -122.73536921818715) bank170311 +170312 POINT(37.25681993722202 -122.93603186100427) bank170312 +170313 POINT(37.233419855918015 -123.25675226539424) bank170313 +170314 POINT(37.11251779908205 -123.40637735109408) bank170314 +170315 POINT(38.345002172732435 -122.70649101454259) bank170315 +170316 POINT(38.4261709963985 -121.66947932165955) bank170316 +170317 POINT(38.14963255087128 -122.40021295811145) bank170317 +170318 POINT(38.15196141662415 -123.37457003739424) bank170318 +170319 POINT(37.580073068272625 -123.31050597128998) bank170319 +170320 POINT(37.03869272697549 -121.69166479034622) bank170320 +170321 POINT(37.21974418612032 -121.82434643958636) bank170321 +170322 POINT(38.427947364113436 -122.38125547463778) bank170322 +170323 POINT(38.473372174273926 -122.92067997965866) bank170323 +170324 POINT(38.370952709111144 -123.39247699534505) bank170324 +170325 POINT(38.08532346343213 -121.98596413455938) bank170325 +170326 POINT(38.770593943322005 -123.26039342740224) bank170326 +170327 POINT(38.18815060788665 -122.2213423725873) bank170327 +170328 POINT(36.88756552358418 -121.65746311920114) bank170328 +170329 POINT(37.337394198345415 -123.29146347075208) bank170329 +170330 POINT(37.09955556474613 -122.40842229113204) bank170330 +170331 POINT(38.49031534038624 -122.12060487379821) bank170331 +170332 POINT(37.03447814616714 -122.55909312764177) bank170332 +170333 POINT(38.074320961117714 -122.58972940737462) bank170333 +170334 POINT(37.1771764432982 -122.99868329658896) bank170334 +170335 POINT(37.547691196947156 -122.1383743740962) bank170335 +170336 POINT(37.49222020833869 -122.41258788991023) bank170336 +170337 POINT(38.1620667135361 -122.07872124733596) bank170337 +170338 POINT(38.04878045079922 -121.69080980607298) bank170338 +170339 POINT(37.779888426615045 -121.48688905940048) bank170339 +170340 POINT(38.66041305338721 -123.17013943057337) bank170340 +170341 POINT(38.24456020258413 -121.46350210938107) bank170341 +170342 POINT(37.21153997718721 -122.1288503465165) bank170342 +170343 POINT(38.264538125524865 -123.10528811909583) bank170343 +170344 POINT(37.13022801625491 -123.19635070602904) bank170344 +170345 POINT(38.51220527929531 -121.53613617792325) bank170345 +170346 POINT(37.604125693185736 -121.65152669026102) bank170346 +170347 POINT(36.82435645394136 -121.89659707033802) bank170347 +170348 POINT(37.584853548531136 -121.69707804969438) bank170348 +170349 POINT(37.965481104523285 -122.51791174264204) bank170349 +170350 POINT(37.667821084566405 -121.6779238513892) bank170350 +170351 POINT(37.27717310252489 -122.02848276853277) bank170351 +170352 POINT(37.16762895235256 -122.15758129025639) bank170352 +170353 POINT(38.19292381462935 -122.63447816403179) bank170353 +170354 POINT(36.860917304847824 -121.56255287252155) bank170354 +170355 POINT(37.81588194541037 -122.71080970385454) bank170355 +170356 POINT(36.98321898324057 -123.07223522787945) bank170356 +170357 POINT(37.99016611817118 -122.91653071702157) bank170357 +170358 POINT(36.93761265174948 -121.96805668499432) bank170358 +170359 POINT(38.5122443903302 -122.57309764014343) bank170359 +170360 POINT(37.30819028518228 -122.8822610737238) bank170360 +170361 POINT(37.390266870972454 -122.9197923427505) bank170361 +170362 POINT(37.431521355534315 -121.87440676724765) bank170362 +170363 POINT(37.4744111319682 -122.44282131889908) bank170363 +170364 POINT(37.36688167354511 -122.73243313504098) bank170364 +170365 POINT(37.389457923656714 -121.64188713157476) bank170365 +170366 POINT(37.44877530103909 -121.70104199057685) bank170366 +170367 POINT(38.56838192804032 -122.9233970583313) bank170367 +170368 POINT(38.45844778908231 -121.74078202901855) bank170368 +170369 POINT(37.29967501910339 -122.00329118724432) bank170369 +170370 POINT(37.77182267699059 -121.48562534450366) bank170370 +170371 POINT(38.215475982376766 -123.02479108496047) bank170371 +170372 POINT(38.46012635690743 -122.6423963194766) bank170372 +170373 POINT(37.64052560968567 -122.30049341769316) bank170373 +170374 POINT(37.09179829216387 -122.9401603672097) bank170374 +170375 POINT(37.36571298613883 -121.94969626664026) bank170375 +170376 POINT(38.241825789792266 -123.4000132628037) bank170376 +170377 POINT(37.01364795824424 -122.37877858119518) bank170377 +170378 POINT(37.504184468006656 -122.42563976017247) bank170378 +170379 POINT(37.24574059269038 -122.1156938601924) bank170379 +170380 POINT(38.34637365332782 -123.24251789678355) bank170380 +170381 POINT(38.718127248071994 -121.8475757244705) bank170381 +170382 POINT(38.241482469387485 -122.06020482970571) bank170382 +170383 POINT(38.32985966473752 -121.5705449926473) bank170383 +170384 POINT(38.751239605739855 -123.19139341290028) bank170384 +170385 POINT(37.129124380939686 -121.88181955555989) bank170385 +170386 POINT(38.16990214633296 -121.89968261345884) bank170386 +170387 POINT(37.78862622408055 -122.23722415076396) bank170387 +170388 POINT(37.304266860728916 -121.68701469589006) bank170388 +170389 POINT(37.898946869283506 -122.09491555195609) bank170389 +170390 POINT(37.87972887291474 -122.28162567644777) bank170390 +170391 POINT(38.02317759215678 -123.0036530999932) bank170391 +170392 POINT(36.95673595054535 -123.19522411100061) bank170392 +170393 POINT(38.67935002148053 -121.96827537288726) bank170393 +170394 POINT(38.06115276315444 -122.41406977344914) bank170394 +170395 POINT(38.22251567485141 -122.54669015663298) bank170395 +170396 POINT(37.91000600887969 -122.01316596237503) bank170396 +170397 POINT(36.91638294440755 -123.15365865471725) bank170397 +170398 POINT(37.79312078197033 -121.94931352414348) bank170398 +170399 POINT(37.85483716812866 -122.6773678052223) bank170399 +170400 POINT(37.28486987667558 -123.08697050727716) bank170400 +170401 POINT(38.0729775248106 -123.12693620911547) bank170401 +170402 POINT(37.570027378921864 -121.70721292896012) bank170402 +170403 POINT(38.416725743207124 -121.85426961642658) bank170403 +170404 POINT(37.95156468962101 -123.16379070173855) bank170404 +170405 POINT(36.954302338177044 -122.7441245207362) bank170405 +170406 POINT(37.257969046646096 -123.28240826137136) bank170406 +170407 POINT(38.47092595313513 -121.6985175435057) bank170407 +170408 POINT(38.53723344527778 -121.69512877814067) bank170408 +170409 POINT(36.83820576763926 -122.75456828717344) bank170409 +170410 POINT(36.79754500316872 -122.96753239026076) bank170410 +170411 POINT(37.69568508553273 -122.38778718932538) bank170411 +170412 POINT(38.50240219632543 -121.59381388089795) bank170412 +170413 POINT(36.95615646727637 -122.6776132477663) bank170413 +170414 POINT(38.44068038524611 -122.10231816706919) bank170414 +170415 POINT(38.4235990449227 -122.42733976328009) bank170415 +170416 POINT(37.538139072226684 -122.75250499918522) bank170416 +170417 POINT(37.12791489080578 -122.71733981641395) bank170417 +170418 POINT(37.458506117483566 -122.79908080221828) bank170418 +170419 POINT(37.45312943635842 -123.24185820224102) bank170419 +170420 POINT(38.39568692922609 -121.62216846694811) bank170420 +170421 POINT(37.74395495164113 -123.01617996490029) bank170421 +170422 POINT(37.32881860121415 -123.09657681612748) bank170422 +170423 POINT(37.260023148238915 -122.2959361188941) bank170423 +170424 POINT(37.9691454230499 -121.71597083639415) bank170424 +170425 POINT(38.299135634503436 -122.94880511577553) bank170425 +170426 POINT(37.95227255060831 -121.94951276567569) bank170426 +170427 POINT(38.512200839285406 -121.83941621961377) bank170427 +170428 POINT(38.51915817405238 -122.45898935095606) bank170428 +170429 POINT(38.5110533775886 -121.87488364635537) bank170429 +170430 POINT(36.793093045127264 -123.2583819752866) bank170430 +170431 POINT(37.865241635200384 -121.522533958921) bank170431 +170432 POINT(37.322725929141974 -121.58170965015564) bank170432 +170433 POINT(38.77201563929983 -121.77434309072669) bank170433 +170434 POINT(37.65294417656922 -121.45958881305698) bank170434 +170435 POINT(37.82089308966558 -121.43453175588283) bank170435 +170436 POINT(37.08106036187435 -122.98093096619344) bank170436 +170437 POINT(37.411712548490335 -121.42915053313408) bank170437 +170438 POINT(37.04400794951939 -122.88058429142413) bank170438 +170439 POINT(37.71697121055146 -122.49083922904771) bank170439 +170440 POINT(37.68711488295065 -123.17024104687903) bank170440 +170441 POINT(38.56403710736336 -122.96044533981947) bank170441 +170442 POINT(38.7649741924786 -121.95655015706258) bank170442 +170443 POINT(37.8050024773077 -122.24390566176476) bank170443 +170444 POINT(37.37561401690251 -122.28626602049825) bank170444 +170445 POINT(38.597639948682904 -122.73465053029088) bank170445 +170446 POINT(38.458853590798775 -121.72411268375141) bank170446 +170447 POINT(37.84094680329503 -121.79426514815567) bank170447 +170448 POINT(38.621316744209714 -123.05279739029248) bank170448 +170449 POINT(38.286043834030636 -122.79013146900307) bank170449 +170450 POINT(38.61681012122147 -122.17590438465284) bank170450 +170451 POINT(37.13970170537634 -122.61058327711231) bank170451 +170452 POINT(37.30018082461536 -122.7547399732953) bank170452 +170453 POINT(37.0917273088804 -122.01496590808154) bank170453 +170454 POINT(37.93686019063426 -123.03803766492973) bank170454 +170455 POINT(37.18470866877356 -121.53242788351346) bank170455 +170456 POINT(38.15229196626974 -121.79481647454722) bank170456 +170457 POINT(37.63434084344286 -122.39829193273269) bank170457 +170458 POINT(37.15747770099877 -122.90964252442289) bank170458 +170459 POINT(37.78924818874498 -122.56475978110016) bank170459 +170460 POINT(38.763281960218045 -123.12089272079366) bank170460 +170461 POINT(38.69219736426552 -121.43020851531274) bank170461 +170462 POINT(37.1033913590806 -122.25662817335007) bank170462 +170463 POINT(38.61911399002746 -123.00370707402247) bank170463 +170464 POINT(38.593684813733404 -121.88228696249992) bank170464 +170465 POINT(38.35122734925222 -121.66171427101762) bank170465 +170466 POINT(38.55481796722563 -122.16951764166842) bank170466 +170467 POINT(37.558905086950745 -121.9618380626561) bank170467 +170468 POINT(38.03300170758851 -122.48065911938987) bank170468 +170469 POINT(38.12397637743065 -123.10213454441548) bank170469 +170470 POINT(38.23816908237716 -121.72697996621096) bank170470 +170471 POINT(38.17773959809777 -122.93424982991026) bank170471 +170472 POINT(36.91047405070237 -122.75844069030863) bank170472 +170473 POINT(38.4587648616427 -122.38699558665745) bank170473 +170474 POINT(37.20249415574321 -122.82503239726992) bank170474 +170475 POINT(38.01992464881987 -122.9972040142089) bank170475 +170476 POINT(37.56209514472335 -123.03268131548792) bank170476 +170477 POINT(37.97546743897843 -121.89261925766604) bank170477 +170478 POINT(38.22642576598736 -121.64600028291427) bank170478 +170479 POINT(38.67733756763424 -121.43640525606551) bank170479 +170480 POINT(37.84481993840609 -123.20634610350531) bank170480 +170481 POINT(38.061494830572435 -122.57186285150377) bank170481 +170482 POINT(36.79898723254999 -122.25647628793052) bank170482 +170483 POINT(38.30289792181901 -121.59013966996069) bank170483 +170484 POINT(38.34952881811143 -123.20636770714714) bank170484 +170485 POINT(37.60681805349561 -121.96091881345244) bank170485 +170486 POINT(38.02533620032763 -122.5643544678979) bank170486 +170487 POINT(38.1470822497229 -123.28938300512138) bank170487 +170488 POINT(37.5705741254623 -122.54308339693867) bank170488 +170489 POINT(36.82829338141277 -121.57241295116181) bank170489 +170490 POINT(38.194988399158845 -123.12359059515991) bank170490 +170491 POINT(38.00049498770174 -122.78363805492936) bank170491 +170492 POINT(38.7146077462189 -122.82982706814241) bank170492 +170493 POINT(37.360135842010834 -122.72503238452624) bank170493 +170494 POINT(37.3986373766929 -121.68041617571382) bank170494 +170495 POINT(37.2910331710669 -122.4384215488916) bank170495 +170496 POINT(38.60267406471196 -121.95128706524719) bank170496 +170497 POINT(37.748917458585225 -122.84827884442177) bank170497 +170498 POINT(38.33691775887297 -122.61828418633301) bank170498 +170499 POINT(37.82810504798408 -122.20658323783073) bank170499 +170500 POINT(37.57935731089477 -122.70415104908184) bank170500 +170501 POINT(36.77981157931487 -122.52635401777658) bank170501 +170502 POINT(36.999184740068436 -121.95671093047213) bank170502 +170503 POINT(37.750212764078704 -122.080865214316) bank170503 +170504 POINT(38.04324684909154 -121.80796812727954) bank170504 +170505 POINT(37.553836273746555 -121.98542303560902) bank170505 +170506 POINT(37.9225436892153 -121.45468812117346) bank170506 +170507 POINT(36.78685447593449 -122.28596401736876) bank170507 +170508 POINT(38.4568502755639 -121.45704038648513) bank170508 +170509 POINT(37.2395493004458 -122.73146452618946) bank170509 +170510 POINT(38.07213620033261 -121.98223631520062) bank170510 +170511 POINT(38.630144374872906 -121.88441371875564) bank170511 +170512 POINT(36.78576171102508 -123.18604514299113) bank170512 +170513 POINT(38.72504167711207 -122.55690161133732) bank170513 +170514 POINT(37.07719714819203 -122.90220211913584) bank170514 +170515 POINT(37.45666984620163 -122.94410322498804) bank170515 +170516 POINT(36.92611266815082 -122.175858840942) bank170516 +170517 POINT(38.65554389633661 -122.55785692840112) bank170517 +170518 POINT(38.764219254825946 -123.12499598684234) bank170518 +170519 POINT(37.966370534118724 -122.53523079024922) bank170519 +170520 POINT(38.25481851747594 -122.68881287695628) bank170520 +170521 POINT(37.08392055365005 -121.4994414451939) bank170521 +170522 POINT(37.710351086119935 -121.4288335299453) bank170522 +170523 POINT(37.73391864137865 -122.88689142952575) bank170523 +170524 POINT(37.48722569799744 -123.10997505492624) bank170524 +170525 POINT(37.177792435271776 -122.38499373420818) bank170525 +170526 POINT(38.66557678206717 -122.41171052108446) bank170526 +170527 POINT(37.79029965567648 -121.58173882461362) bank170527 +170528 POINT(37.62046254006151 -123.28989321326732) bank170528 +170529 POINT(36.86175399014711 -123.06442901644196) bank170529 +170530 POINT(38.74703810334193 -121.67227165560838) bank170530 +170531 POINT(37.77492369432129 -122.86196359502958) bank170531 +170532 POINT(36.91106671180338 -122.51296093539568) bank170532 +170533 POINT(36.872806888339035 -122.74272678081132) bank170533 +170534 POINT(37.85869452483462 -122.80604119607433) bank170534 +170535 POINT(38.74264503135434 -122.14847222268826) bank170535 +170536 POINT(38.23778493774811 -121.86546592885215) bank170536 +170537 POINT(37.32781983569986 -122.68017339212453) bank170537 +170538 POINT(37.39392092247715 -121.97945399941274) bank170538 +170539 POINT(38.75962587199054 -121.98165299400027) bank170539 +170540 POINT(38.087873490160334 -123.0188959028628) bank170540 +170541 POINT(38.171866845983104 -122.12702030870291) bank170541 +170542 POINT(38.1060279328827 -121.61909630429574) bank170542 +170543 POINT(38.738058905825845 -122.66400050746297) bank170543 +170544 POINT(38.30510218665557 -121.7995726881635) bank170544 +170545 POINT(38.45949672197031 -122.7428828264316) bank170545 +170546 POINT(38.60086045026473 -121.70742948509802) bank170546 +170547 POINT(38.368638147942406 -121.71747723746896) bank170547 +170548 POINT(38.38635416104263 -123.30413717014818) bank170548 +170549 POINT(38.25817434463239 -123.31906372657947) bank170549 +170550 POINT(37.139865361006166 -123.03808799597371) bank170550 +170551 POINT(38.5303939848565 -122.78326716321371) bank170551 +170552 POINT(37.42417204691188 -122.25805076936959) bank170552 +170553 POINT(37.35540801876137 -122.7085856405331) bank170553 +170554 POINT(38.443851191528935 -122.70449552317109) bank170554 +170555 POINT(38.27355886463871 -122.38116602897404) bank170555 +170556 POINT(37.447804364164625 -121.61074773499494) bank170556 +170557 POINT(37.41148562924789 -123.33981038715494) bank170557 +170558 POINT(38.40033532130098 -123.34264888070702) bank170558 +170559 POINT(38.0558984257206 -122.81808621103751) bank170559 +170560 POINT(36.87162807046148 -121.71199005963845) bank170560 +170561 POINT(36.837523356006294 -122.69588695050588) bank170561 +170562 POINT(38.64984061988323 -123.30642635466653) bank170562 +170563 POINT(37.85688657665162 -123.41841190450516) bank170563 +170564 POINT(38.18588724144857 -123.21889450352694) bank170564 +170565 POINT(38.14712729069215 -121.74971434545786) bank170565 +170566 POINT(36.82937633251816 -123.38046249701534) bank170566 +170567 POINT(38.72537927406445 -122.99199923803319) bank170567 +170568 POINT(37.10560530252648 -122.04963639817734) bank170568 +170569 POINT(37.030768238133334 -123.33555988256428) bank170569 +170570 POINT(36.80115339662859 -123.06033905880179) bank170570 +170571 POINT(38.14119310668565 -123.19578173957888) bank170571 +170572 POINT(37.73483250477402 -122.06909694439172) bank170572 +170573 POINT(38.65043448002308 -121.63730352162904) bank170573 +170574 POINT(37.825532412799625 -122.23156907348253) bank170574 +170575 POINT(36.79101176845233 -123.34070559329362) bank170575 +170576 POINT(37.550738271889834 -123.38647255349072) bank170576 +170577 POINT(38.04135049914045 -122.51626007441318) bank170577 +170578 POINT(37.85251794987837 -121.89629189914612) bank170578 +170579 POINT(36.8287735976228 -121.80090231131057) bank170579 +170580 POINT(37.50062098476433 -122.95381934680202) bank170580 +170581 POINT(38.606188579898564 -121.94147218759404) bank170581 +170582 POINT(37.188961644761875 -123.2690239195768) bank170582 +170583 POINT(37.975937443348364 -122.68560465672458) bank170583 +170584 POINT(37.5759980063311 -122.7188609203111) bank170584 +170585 POINT(38.19991658753524 -122.41465890445305) bank170585 +170586 POINT(36.86342422726101 -122.44939063114182) bank170586 +170587 POINT(38.21616835608786 -122.40357545527104) bank170587 +170588 POINT(37.18475550008802 -123.20966678933468) bank170588 +170589 POINT(38.23524652372013 -121.64698157909174) bank170589 +170590 POINT(37.10381731177827 -121.72119685015707) bank170590 +170591 POINT(38.05495964000761 -122.41732585282466) bank170591 +170592 POINT(37.12487051661299 -123.40317087749469) bank170592 +170593 POINT(38.67841315015942 -122.6087541176915) bank170593 +170594 POINT(36.80515955473843 -122.97089920851995) bank170594 +170595 POINT(37.930536486720385 -122.65862256820014) bank170595 +170596 POINT(36.80338178347486 -122.08280061057884) bank170596 +170597 POINT(37.92094488143253 -123.20765654954721) bank170597 +170598 POINT(38.51637976031815 -122.1996757501326) bank170598 +170599 POINT(37.91688205796286 -122.41871574744287) bank170599 +170600 POINT(37.12263413043323 -121.69611281266283) bank170600 +170601 POINT(37.896325611021325 -121.95315131185694) bank170601 +170602 POINT(37.47115293134741 -121.44917635970846) bank170602 +170603 POINT(38.135240814797946 -122.05449158901942) bank170603 +170604 POINT(37.091855439054974 -122.75670386401545) bank170604 +170605 POINT(38.72112529801185 -122.07079157324584) bank170605 +170606 POINT(37.30835706238233 -123.22908530419235) bank170606 +170607 POINT(37.93213540475032 -122.60712955042597) bank170607 +170608 POINT(37.515036840922214 -122.89240710032105) bank170608 +170609 POINT(36.83876818296902 -122.97095734699428) bank170609 +170610 POINT(37.461308425426544 -121.90846398482246) bank170610 +170611 POINT(37.16961658881357 -123.25849059741229) bank170611 +170612 POINT(37.661109158090355 -122.00606242792806) bank170612 +170613 POINT(37.61245195670881 -122.11719888170695) bank170613 +170614 POINT(38.03425550199559 -123.40118751911336) bank170614 +170615 POINT(37.850191554046454 -122.40499817918139) bank170615 +170616 POINT(38.16099587543554 -122.74201510607509) bank170616 +170617 POINT(37.96880150911282 -121.66850565721813) bank170617 +170618 POINT(37.257751699082604 -121.9455506703021) bank170618 +170619 POINT(36.82651688150551 -121.61693053928379) bank170619 +170620 POINT(37.793004885438144 -122.97190521718797) bank170620 +170621 POINT(37.03523814940341 -122.5999927349549) bank170621 +170622 POINT(38.736460060632645 -122.1493865265199) bank170622 +170623 POINT(37.73336795166773 -122.85573781321335) bank170623 +170624 POINT(37.39293790672635 -122.11226993432349) bank170624 +170625 POINT(37.01463586776535 -123.28561961833103) bank170625 +170626 POINT(37.28177987571242 -122.74319722477526) bank170626 +170627 POINT(37.20219873514002 -121.83527909206445) bank170627 +170628 POINT(38.77430974360829 -121.68066030848463) bank170628 +170629 POINT(37.466129453825474 -121.86273897127721) bank170629 +170630 POINT(36.810845187564446 -122.32576083510102) bank170630 +170631 POINT(38.59633223477688 -122.94030888841478) bank170631 +170632 POINT(37.67064488993117 -123.10499957410728) bank170632 +170633 POINT(37.3789751335708 -122.0183935806344) bank170633 +170634 POINT(37.384428045720725 -122.61716006066563) bank170634 +170635 POINT(37.19067436651335 -122.33311963220102) bank170635 +170636 POINT(38.36613035241323 -122.12935712505602) bank170636 +170637 POINT(37.37200209624989 -122.86168093592282) bank170637 +170638 POINT(36.784558137664476 -121.96640402032519) bank170638 +170639 POINT(37.070208519821996 -121.69414026186303) bank170639 +170640 POINT(38.43996014594544 -123.07257284767944) bank170640 +170641 POINT(38.127063523013696 -123.27629001715326) bank170641 +170642 POINT(37.353079803052005 -121.70983524306723) bank170642 +170643 POINT(37.53088502672965 -121.82486605452517) bank170643 +170644 POINT(38.5270515152503 -123.04609684098627) bank170644 +170645 POINT(37.62011879023941 -122.12550136438512) bank170645 +170646 POINT(37.65672829796116 -123.0839478277308) bank170646 +170647 POINT(38.53698400226834 -121.83952459678234) bank170647 +170648 POINT(38.14544376892262 -121.78288822398474) bank170648 +170649 POINT(37.025813896266214 -122.52140273989933) bank170649 +170650 POINT(37.164781569952346 -121.499458356908) bank170650 +170651 POINT(38.708193612134345 -121.46035536153215) bank170651 +170652 POINT(38.320046588352696 -122.89002018456965) bank170652 +170653 POINT(38.08581893761616 -122.85016910369086) bank170653 +170654 POINT(38.44864891908306 -122.79062412243053) bank170654 +170655 POINT(37.84164338224007 -123.29113111698084) bank170655 +170656 POINT(36.99190410218399 -122.8341188272251) bank170656 +170657 POINT(38.38478396562943 -123.12809341214913) bank170657 +170658 POINT(38.57432964821063 -121.44518991855874) bank170658 +170659 POINT(37.55696688753663 -122.13145947823367) bank170659 +170660 POINT(38.40048747135233 -121.53526295944751) bank170660 +170661 POINT(37.374937391266826 -123.16721435757296) bank170661 +170662 POINT(37.03947281185696 -122.4243645911518) bank170662 +170663 POINT(38.08875215458181 -121.65183774318902) bank170663 +170664 POINT(36.778443339527485 -121.70553322764319) bank170664 +170665 POINT(36.86184595284101 -122.72566983877708) bank170665 +170666 POINT(37.828499085396686 -121.72798605160703) bank170666 +170667 POINT(37.19700268556933 -122.29440165548912) bank170667 +170668 POINT(37.091848412554384 -122.46070269200564) bank170668 +170669 POINT(37.43553654295236 -122.45363062015286) bank170669 +170670 POINT(37.819395550460264 -122.8590319698243) bank170670 +170671 POINT(37.10185247758743 -122.64985220716693) bank170671 +170672 POINT(38.05233239719049 -123.41293107336674) bank170672 +170673 POINT(37.875828513312506 -123.22676967250622) bank170673 +170674 POINT(37.692558595554004 -121.49586035496974) bank170674 +170675 POINT(36.776499889720434 -122.2455639144446) bank170675 +170676 POINT(38.087924739017915 -121.97927756731183) bank170676 +170677 POINT(38.24567130967345 -122.3915188468788) bank170677 +170678 POINT(36.989842412508835 -122.34260860453712) bank170678 +170679 POINT(37.45865947109173 -122.93303776315275) bank170679 +170680 POINT(37.0560851315712 -122.02787811043598) bank170680 +170681 POINT(37.03665031552358 -122.11094415292966) bank170681 +170682 POINT(37.425988943228674 -122.2299218874363) bank170682 +170683 POINT(37.17299168957453 -121.56097203712636) bank170683 +170684 POINT(38.4170701535073 -121.69505698485122) bank170684 +170685 POINT(38.42756889268779 -121.90905674591902) bank170685 +170686 POINT(37.746583686621605 -123.4177192236806) bank170686 +170687 POINT(38.24747745153216 -121.8123964904428) bank170687 +170688 POINT(37.24565518324957 -122.69389920120025) bank170688 +170689 POINT(38.16597562155175 -121.7744464909722) bank170689 +170690 POINT(38.53409192739136 -123.04123188621362) bank170690 +170691 POINT(38.10395714768934 -121.71503965832203) bank170691 +170692 POINT(36.83149489468514 -122.68569919649997) bank170692 +170693 POINT(38.208367669757216 -121.52463246493608) bank170693 +170694 POINT(36.901470468732406 -122.86592029039431) bank170694 +170695 POINT(38.273716528792335 -123.02960341156951) bank170695 +170696 POINT(38.464624288741675 -122.7209825746695) bank170696 +170697 POINT(36.83013858345029 -122.32128998920479) bank170697 +170698 POINT(38.768123691543174 -121.47131637493968) bank170698 +170699 POINT(36.777421469312905 -122.65427818605532) bank170699 +170700 POINT(38.382332316735045 -122.69686046148381) bank170700 +170701 POINT(37.25950540764557 -122.0452382712868) bank170701 +170702 POINT(38.70494662192361 -121.44947183562942) bank170702 +170703 POINT(37.263796965505904 -121.49435563900214) bank170703 +170704 POINT(38.405097653629575 -123.10862990521628) bank170704 +170705 POINT(38.77413529437004 -121.75909098915244) bank170705 +170706 POINT(38.42309397496628 -123.0547610967918) bank170706 +170707 POINT(38.39427490780968 -121.46115771735354) bank170707 +170708 POINT(37.00713705165296 -122.28654241859712) bank170708 +170709 POINT(37.12085107771521 -122.85600522553182) bank170709 +170710 POINT(38.16352764747064 -121.86017991868798) bank170710 +170711 POINT(36.93322805658825 -122.22599071378376) bank170711 +170712 POINT(38.50069010300039 -122.37380031870151) bank170712 +170713 POINT(38.63496085971591 -123.05538884555473) bank170713 +170714 POINT(37.83894937450647 -122.70702986268954) bank170714 +170715 POINT(37.43017195808521 -123.29408312570358) bank170715 +170716 POINT(37.20836506947204 -121.91413323619636) bank170716 +170717 POINT(37.53805126072944 -122.18390219157318) bank170717 +170718 POINT(38.174036769544024 -122.63586879966243) bank170718 +170719 POINT(37.7017317042576 -123.09022803766007) bank170719 +170720 POINT(37.16662561341192 -121.96061156691982) bank170720 +170721 POINT(38.3777169994742 -122.55072185835205) bank170721 +170722 POINT(37.996383456179856 -121.7590949185933) bank170722 +170723 POINT(36.96019178041581 -122.24879966743369) bank170723 +170724 POINT(38.76737496411208 -122.86130207556198) bank170724 +170725 POINT(37.432059510648976 -121.91403818451954) bank170725 +170726 POINT(37.963428840416775 -122.4214653324455) bank170726 +170727 POINT(37.36669857108162 -123.28736961692113) bank170727 +170728 POINT(37.65185010194761 -123.1766967052928) bank170728 +170729 POINT(37.659439290614536 -123.03528470341249) bank170729 +170730 POINT(36.95528692412719 -122.48004614921994) bank170730 +170731 POINT(38.59166277125597 -121.77719337472847) bank170731 +170732 POINT(38.74038090126565 -123.1175928276017) bank170732 +170733 POINT(37.59218716170168 -122.1392031598044) bank170733 +170734 POINT(38.619595929305554 -121.94418707441373) bank170734 +170735 POINT(38.516401243315606 -122.29103630925262) bank170735 +170736 POINT(38.18532255264311 -122.50776205186565) bank170736 +170737 POINT(37.611343813789205 -123.31686685006127) bank170737 +170738 POINT(38.760810479577295 -122.74382930215779) bank170738 +170739 POINT(37.11108477562738 -123.24328251729906) bank170739 +170740 POINT(37.08772792742513 -123.17194161946968) bank170740 +170741 POINT(36.851458155586876 -121.86688017454475) bank170741 +170742 POINT(36.920216004872486 -123.31787128375167) bank170742 +170743 POINT(36.82624464585307 -122.68770116989486) bank170743 +170744 POINT(36.96383359294242 -122.1669768520915) bank170744 +170745 POINT(37.10567333934081 -122.34776716560097) bank170745 +170746 POINT(37.55575550732232 -123.22818163126736) bank170746 +170747 POINT(37.66457407632379 -123.23948905426475) bank170747 +170748 POINT(38.37191082606811 -122.93869434075074) bank170748 +170749 POINT(38.256319084468174 -121.58772326806782) bank170749 +170750 POINT(37.92377900664777 -121.83864877201131) bank170750 +170751 POINT(37.21349336406088 -121.70205088976861) bank170751 +170752 POINT(37.20309635123035 -121.57829677217558) bank170752 +170753 POINT(36.78847651205937 -122.0531426273691) bank170753 +170754 POINT(37.013141771368474 -122.18996198478963) bank170754 +170755 POINT(38.230974486325735 -123.30005257128714) bank170755 +170756 POINT(36.796402703135996 -123.05307060313501) bank170756 +170757 POINT(37.174639306248935 -122.71444322497246) bank170757 +170758 POINT(38.64929853228135 -122.59249615871771) bank170758 +170759 POINT(37.74182914454301 -123.05635709680585) bank170759 +170760 POINT(37.600580653014404 -123.06901231997173) bank170760 +170761 POINT(36.89092327206193 -123.23851525850584) bank170761 +170762 POINT(38.68381416613465 -122.72062811571969) bank170762 +170763 POINT(37.25588758769514 -121.44413403112044) bank170763 +170764 POINT(37.79657800387065 -121.69320154147499) bank170764 +170765 POINT(37.44679100557602 -122.03809005525319) bank170765 +170766 POINT(38.265256823986945 -122.42124095540944) bank170766 +170767 POINT(38.395322118914336 -122.11369166638451) bank170767 +170768 POINT(37.02425623364653 -121.54398594072187) bank170768 +170769 POINT(37.86725709183781 -122.74341201443201) bank170769 +170770 POINT(37.643423900726994 -122.63217566132519) bank170770 +170771 POINT(37.44167691340519 -122.36432364657797) bank170771 +170772 POINT(37.50389851978729 -122.42837169798021) bank170772 +170773 POINT(37.75716661058882 -122.14399352316828) bank170773 +170774 POINT(38.51879775641357 -122.22135168925129) bank170774 +170775 POINT(36.85439244321809 -122.40562759733983) bank170775 +170776 POINT(37.853785017378236 -121.89628680486578) bank170776 +170777 POINT(37.22584194237514 -122.50591498261066) bank170777 +170778 POINT(38.112779563631236 -121.65041209534078) bank170778 +170779 POINT(37.16074679115313 -123.291984334323) bank170779 +170780 POINT(36.829808802218544 -122.17482029789194) bank170780 +170781 POINT(37.93971629115962 -122.29379208966468) bank170781 +170782 POINT(38.50331628194029 -123.2731462419878) bank170782 +170783 POINT(38.46736418615058 -121.99101436530935) bank170783 +170784 POINT(36.831531748312415 -121.91922123999842) bank170784 +170785 POINT(38.481514904598704 -121.85890908633431) bank170785 +170786 POINT(37.91706629598406 -122.04588655552833) bank170786 +170787 POINT(37.657816018081405 -122.70676406228799) bank170787 +170788 POINT(38.65878671838139 -122.5124946373712) bank170788 +170789 POINT(38.305437035308316 -122.8648270525124) bank170789 +170790 POINT(37.41817762264906 -121.78227198390061) bank170790 +170791 POINT(38.26902330760325 -122.03510385117474) bank170791 +170792 POINT(38.09875618065494 -122.65124574947829) bank170792 +170793 POINT(37.875180393032025 -122.76355841926134) bank170793 +170794 POINT(37.2566928984975 -121.85600225022628) bank170794 +170795 POINT(38.628138438875304 -121.52022780854283) bank170795 +170796 POINT(38.108027536157756 -121.57869943111488) bank170796 +170797 POINT(37.076091870187184 -123.00598452518963) bank170797 +170798 POINT(38.49190374088316 -121.99114457285542) bank170798 +170799 POINT(38.19962056969237 -121.95105450949339) bank170799 +170800 POINT(37.7560707771652 -122.99845157082814) bank170800 +170801 POINT(37.64412974854586 -122.33934939689661) bank170801 +170802 POINT(38.15950601908795 -122.90856198630483) bank170802 +170803 POINT(37.47290832725237 -121.78491943700249) bank170803 +170804 POINT(38.73267824665213 -123.31794441275977) bank170804 +170805 POINT(36.92393558572189 -122.6159354528139) bank170805 +170806 POINT(38.29637178602579 -122.30175648798222) bank170806 +170807 POINT(38.75248526236279 -122.95757700791879) bank170807 +170808 POINT(38.74332482645499 -122.64125607523579) bank170808 +170809 POINT(38.6909291586331 -123.22565865986911) bank170809 +170810 POINT(37.182290554811054 -123.11030861191608) bank170810 +170811 POINT(38.63573960794481 -121.75230106245337) bank170811 +170812 POINT(37.08277458240251 -122.03881251305783) bank170812 +170813 POINT(37.391174556048526 -122.42582459655469) bank170813 +170814 POINT(37.84047585087197 -122.99826129229777) bank170814 +170815 POINT(37.31013065846295 -123.29373223847934) bank170815 +170816 POINT(38.513268651738485 -122.56965464359057) bank170816 +170817 POINT(37.436896444323594 -123.17347849165124) bank170817 +170818 POINT(38.68110729220124 -123.08922540596099) bank170818 +170819 POINT(37.08814692531418 -122.55196172662629) bank170819 +170820 POINT(37.18332672958672 -122.68029556264639) bank170820 +170821 POINT(38.493884907828395 -121.67296561475625) bank170821 +170822 POINT(37.32353391701408 -122.17637315949659) bank170822 +170823 POINT(37.80443864596109 -122.16847861931532) bank170823 +170824 POINT(37.535523644369 -122.85932284276659) bank170824 +170825 POINT(38.530018733898444 -122.7500971542539) bank170825 +170826 POINT(38.16339773696441 -121.81393999104988) bank170826 +170827 POINT(37.788546947243724 -121.97902675102934) bank170827 +170828 POINT(38.683073100678556 -123.10311995767725) bank170828 +170829 POINT(38.6460242865581 -122.28623878560198) bank170829 +170830 POINT(38.39705434166689 -122.86662966691182) bank170830 +170831 POINT(37.938078329179156 -122.80560112626307) bank170831 +170832 POINT(38.61832565835177 -122.14529446410664) bank170832 +170833 POINT(37.08649394328544 -122.54182382488865) bank170833 +170834 POINT(38.20617979950717 -123.24648073620651) bank170834 +170835 POINT(38.5957432926297 -122.82294415445583) bank170835 +170836 POINT(37.568526536137945 -122.63991906247152) bank170836 +170837 POINT(38.730463391548156 -123.3750691488043) bank170837 +170838 POINT(38.70782414208945 -121.46794978151766) bank170838 +170839 POINT(36.779487590350236 -121.61163368776903) bank170839 +170840 POINT(37.538625201290316 -121.68681834504687) bank170840 +170841 POINT(38.421079603309025 -123.3595572915284) bank170841 +170842 POINT(36.96526240227572 -123.09906806825221) bank170842 +170843 POINT(38.54572121046714 -121.9572986972712) bank170843 +170844 POINT(37.67874336630423 -121.94375380429037) bank170844 +170845 POINT(37.37485245683596 -122.14193903467765) bank170845 +170846 POINT(37.76591650122268 -121.445662010487) bank170846 +170847 POINT(37.10879274261467 -122.69361037923011) bank170847 +170848 POINT(37.13998029645465 -121.74088708113202) bank170848 +170849 POINT(38.50492070881918 -123.18114583138498) bank170849 +170850 POINT(38.713885500378765 -121.6138834144096) bank170850 +170851 POINT(38.32947064878326 -122.19627900620446) bank170851 +170852 POINT(38.10090768942352 -122.1024504118876) bank170852 +170853 POINT(37.030499202541854 -122.90083201763129) bank170853 +170854 POINT(36.83906828847521 -123.08160807273039) bank170854 +170855 POINT(38.5738989223381 -123.15873035876865) bank170855 +170856 POINT(37.0789938930751 -122.57451312011419) bank170856 +170857 POINT(38.09055052240633 -121.55045483131006) bank170857 +170858 POINT(37.42644971578239 -122.85612541949801) bank170858 +170859 POINT(37.325462449846555 -122.30023233982133) bank170859 +170860 POINT(37.55132672640235 -123.29996301523622) bank170860 +170861 POINT(38.16871674480526 -122.15339284205058) bank170861 +170862 POINT(37.91185615574252 -123.21510691116882) bank170862 +170863 POINT(37.59015808587026 -121.67167690535365) bank170863 +170864 POINT(37.26969671319216 -123.26053056011108) bank170864 +170865 POINT(37.81716349926666 -122.57022137739996) bank170865 +170866 POINT(37.290435115541136 -122.79839153391305) bank170866 +170867 POINT(38.390245186418774 -123.28244266983504) bank170867 +170868 POINT(38.16130658080083 -122.42980082597644) bank170868 +170869 POINT(37.68462568579618 -122.079901833449) bank170869 +170870 POINT(37.13349792032207 -121.9611785753299) bank170870 +170871 POINT(37.61132971720813 -122.10890812069941) bank170871 +170872 POINT(37.365666409384175 -123.25261225412697) bank170872 +170873 POINT(37.69224824971346 -122.7373260380544) bank170873 +170874 POINT(37.60885340236225 -123.29119176723414) bank170874 +170875 POINT(37.28089578633026 -122.7650887893338) bank170875 +170876 POINT(38.39849487431334 -122.34874023243148) bank170876 +170877 POINT(38.64296819521671 -122.5691959544684) bank170877 +170878 POINT(38.45071019855948 -122.86614549993105) bank170878 +170879 POINT(38.477264247596636 -122.02732137281065) bank170879 +170880 POINT(37.233692422944145 -123.13804297300132) bank170880 +170881 POINT(37.72587644813683 -121.98820743064032) bank170881 +170882 POINT(37.36199307276628 -123.38742427765497) bank170882 +170883 POINT(37.31612055102836 -122.32381142344636) bank170883 +170884 POINT(37.52430428458758 -122.85840007247523) bank170884 +170885 POINT(36.93688355911129 -122.8759175831486) bank170885 +170886 POINT(37.925620655624265 -122.00424942361748) bank170886 +170887 POINT(38.169901427644625 -123.14502118952817) bank170887 +170888 POINT(37.94213879521571 -123.15960216978404) bank170888 +170889 POINT(38.386251229096935 -122.87047841491128) bank170889 +170890 POINT(37.4524263418313 -122.63880680362917) bank170890 +170891 POINT(38.00034211716692 -123.21967633428929) bank170891 +170892 POINT(37.692816592599605 -121.45816557056983) bank170892 +170893 POINT(37.16016351629942 -121.6122698350192) bank170893 +170894 POINT(38.501899639989816 -121.75088028643202) bank170894 +170895 POINT(38.06641048268568 -123.22465872113757) bank170895 +170896 POINT(38.17495527112165 -122.57372897572321) bank170896 +170897 POINT(37.15250742610178 -122.6916816558674) bank170897 +170898 POINT(37.945548258345 -121.75171787471584) bank170898 +170899 POINT(38.72127331852302 -122.54200722054777) bank170899 +170900 POINT(38.462077284734285 -123.23584823468106) bank170900 +170901 POINT(36.93460989810037 -121.45628480603172) bank170901 +170902 POINT(36.94265817694581 -122.38272961062863) bank170902 +170903 POINT(38.39536570583231 -121.5806121632007) bank170903 +170904 POINT(37.424631001526016 -122.23182531396635) bank170904 +170905 POINT(36.77536456671369 -121.60308117637005) bank170905 +170906 POINT(37.890052768876394 -122.72038772880481) bank170906 +170907 POINT(37.53887171279341 -121.81281586992705) bank170907 +170908 POINT(37.24677618953121 -122.84637979109124) bank170908 +170909 POINT(37.87105327953563 -123.11575318390202) bank170909 +170910 POINT(37.05225345768078 -121.77569684205692) bank170910 +170911 POINT(38.364491455207485 -122.28938890963) bank170911 +170912 POINT(38.72804456536029 -122.80178671935914) bank170912 +170913 POINT(37.71954930271067 -121.50097889569055) bank170913 +170914 POINT(38.27742097636765 -122.59745111758534) bank170914 +170915 POINT(37.33858640665783 -122.78643945242305) bank170915 +170916 POINT(37.025339173855585 -122.4301149391129) bank170916 +170917 POINT(38.44972698236744 -122.31699114070798) bank170917 +170918 POINT(37.3675122602772 -123.22395589941473) bank170918 +170919 POINT(38.47597302231315 -122.82426753324225) bank170919 +170920 POINT(37.05964872273255 -121.67591084865272) bank170920 +170921 POINT(37.19532454267653 -123.12149774279514) bank170921 +170922 POINT(38.746024106094886 -121.76434184551564) bank170922 +170923 POINT(37.37595673740409 -122.25851190526402) bank170923 +170924 POINT(38.39371761687765 -121.87625222662551) bank170924 +170925 POINT(37.351534462238234 -122.38532185258498) bank170925 +170926 POINT(38.35150441401236 -122.56639473065312) bank170926 +170927 POINT(37.77315634107976 -122.7843500864009) bank170927 +170928 POINT(37.23968179855788 -122.75426769660847) bank170928 +170929 POINT(36.93637332452919 -122.68442358576411) bank170929 +170930 POINT(37.75804483777872 -122.12098647669872) bank170930 +170931 POINT(38.637377306368705 -123.36691133159447) bank170931 +170932 POINT(37.7180800302342 -123.1000402787605) bank170932 +170933 POINT(37.623051993564 -122.91589807798587) bank170933 +170934 POINT(38.11701441710196 -123.19985494639421) bank170934 +170935 POINT(38.719690299033076 -123.19175279267965) bank170935 +170936 POINT(37.75349803786144 -122.39772349384455) bank170936 +170937 POINT(37.75081857928068 -122.42869009799453) bank170937 +170938 POINT(38.15754795217105 -122.7849645667653) bank170938 +170939 POINT(37.58098860787727 -123.06399613349761) bank170939 +170940 POINT(37.18131890869065 -123.08320304471343) bank170940 +170941 POINT(36.976047287271044 -122.40223521968485) bank170941 +170942 POINT(37.05134515603463 -121.58266338138598) bank170942 +170943 POINT(38.42620219996235 -123.24338672604028) bank170943 +170944 POINT(37.951583855614174 -122.35020014550442) bank170944 +170945 POINT(37.00371239207863 -122.72163970522138) bank170945 +170946 POINT(37.82479993102504 -121.97264657319761) bank170946 +170947 POINT(37.064655691671064 -123.13039788617985) bank170947 +170948 POINT(37.46103276355633 -123.11057775097856) bank170948 +170949 POINT(37.78785675138414 -121.4681327523949) bank170949 +170950 POINT(36.807031872532605 -122.67548357113114) bank170950 +170951 POINT(38.732795090913484 -121.54459225441177) bank170951 +170952 POINT(37.3673463165848 -122.08346904313971) bank170952 +170953 POINT(37.713316207640595 -121.60014843504617) bank170953 +170954 POINT(37.93273611736079 -122.56525942946699) bank170954 +170955 POINT(38.17139883551288 -121.44939814292762) bank170955 +170956 POINT(36.80842347850762 -123.18188282390615) bank170956 +170957 POINT(36.859642320064985 -121.94091105185964) bank170957 +170958 POINT(38.20658371841868 -121.82088259787469) bank170958 +170959 POINT(37.88613247355703 -122.50070367841984) bank170959 +170960 POINT(38.012217456133584 -122.75944896690368) bank170960 +170961 POINT(38.16678161037615 -122.31166077202467) bank170961 +170962 POINT(38.38693670610338 -123.27111528688927) bank170962 +170963 POINT(37.00264400716611 -122.05417459633748) bank170963 +170964 POINT(37.61796261540331 -122.88709679927153) bank170964 +170965 POINT(37.42421117471198 -122.9596088303569) bank170965 +170966 POINT(37.79960966168986 -122.59497717574278) bank170966 +170967 POINT(37.06904074846213 -122.29868360012928) bank170967 +170968 POINT(36.958959666057495 -122.36074055902229) bank170968 +170969 POINT(38.16943299364027 -122.20429429579754) bank170969 +170970 POINT(38.288026361416016 -121.67742463840227) bank170970 +170971 POINT(38.125854677090594 -121.93150559397276) bank170971 +170972 POINT(38.56253168109383 -122.21807567592121) bank170972 +170973 POINT(37.56444892488556 -121.61105079104156) bank170973 +170974 POINT(36.813465062467735 -121.55469811503147) bank170974 +170975 POINT(37.5551022310558 -122.60971562166279) bank170975 +170976 POINT(38.37686192739702 -121.93410679788349) bank170976 +170977 POINT(37.8948777887568 -123.18044265505893) bank170977 +170978 POINT(38.23230456480482 -121.77199288742968) bank170978 +170979 POINT(36.969772038673945 -121.579804964221) bank170979 +170980 POINT(37.21363171877878 -122.11311598015799) bank170980 +170981 POINT(37.00346439143312 -121.49799570336235) bank170981 +170982 POINT(37.825424075064035 -121.52941183638805) bank170982 +170983 POINT(37.59654639932564 -121.82795905281188) bank170983 +170984 POINT(37.55976429030315 -121.47325576203045) bank170984 +170985 POINT(36.905484964463675 -121.64045639851248) bank170985 +170986 POINT(38.54681989653417 -122.65449578904617) bank170986 +170987 POINT(37.1428788322414 -123.18402091038577) bank170987 +170988 POINT(38.12098455206636 -122.07624909754007) bank170988 +170989 POINT(37.79458245861218 -122.09433500444722) bank170989 +170990 POINT(37.05019959065735 -123.02271207236272) bank170990 +170991 POINT(36.920918668418906 -121.62760426781573) bank170991 +170992 POINT(38.55156452355095 -122.85228638776917) bank170992 +170993 POINT(37.550124389272725 -122.6650932998438) bank170993 +170994 POINT(37.46699092943476 -121.61201751401538) bank170994 +170995 POINT(38.35178889320436 -123.22775937211132) bank170995 +170996 POINT(36.86989389742878 -122.51571110621802) bank170996 +170997 POINT(38.4845668457009 -123.41222424348136) bank170997 +170998 POINT(37.399871769037404 -122.82085919121384) bank170998 +170999 POINT(37.942529766867125 -121.44360836222471) bank170999 +171000 POINT(37.648918661843226 -122.18950077690235) bank171000 +171001 POINT(38.50936233252344 -122.86183057722404) bank171001 +171002 POINT(37.18497213486797 -121.51306088418028) bank171002 +171003 POINT(38.65600906220801 -121.47652891080364) bank171003 +171004 POINT(37.843984085823635 -122.07044007995765) bank171004 +171005 POINT(38.45454437704238 -122.35479429587889) bank171005 +171006 POINT(37.69839365602783 -121.62029194962157) bank171006 +171007 POINT(38.13873719150365 -123.1364155009201) bank171007 +171008 POINT(38.76163152352604 -122.60177830606874) bank171008 +171009 POINT(37.35995651519736 -121.62518293673604) bank171009 +171010 POINT(37.89211430668 -123.3217107465254) bank171010 +171011 POINT(38.48777687569744 -121.51354482551056) bank171011 +171012 POINT(38.68792063500171 -122.74743904693555) bank171012 +171013 POINT(36.901382778343944 -123.2520277349121) bank171013 +171014 POINT(38.47313057501431 -121.94023135085708) bank171014 +171015 POINT(36.907928025351715 -121.87494471379195) bank171015 +171016 POINT(37.03784640574776 -123.33845260054909) bank171016 +171017 POINT(38.444193700209055 -121.6746183509851) bank171017 +171018 POINT(38.42334471469646 -121.94585132515957) bank171018 +171019 POINT(37.09740122469568 -122.72828223266603) bank171019 +171020 POINT(37.71590187614037 -123.28994848654318) bank171020 +171021 POINT(37.56163421066173 -121.91503765045572) bank171021 +171022 POINT(38.55134020222891 -122.07750369760187) bank171022 +171023 POINT(37.53745285637918 -122.88088865060853) bank171023 +171024 POINT(37.51662853654615 -123.21224588215414) bank171024 +171025 POINT(36.880373757135715 -122.65493590765811) bank171025 +171026 POINT(38.33231559729663 -122.35491382442375) bank171026 +171027 POINT(37.937377643895665 -122.61343516268415) bank171027 +171028 POINT(37.15490432744 -122.37238032171886) bank171028 +171029 POINT(37.840073179380745 -123.28178590112482) bank171029 +171030 POINT(37.63594728797945 -123.16279896959311) bank171030 +171031 POINT(37.29140548179156 -121.97491117448584) bank171031 +171032 POINT(37.04934116615257 -122.18710515163677) bank171032 +171033 POINT(38.54169529511397 -122.7537490046458) bank171033 +171034 POINT(37.99212718676495 -122.0336809010963) bank171034 +171035 POINT(37.72086779874549 -121.75680665549086) bank171035 +171036 POINT(37.65165130531145 -121.90010119886219) bank171036 +171037 POINT(38.5293105879652 -122.32711364825468) bank171037 +171038 POINT(36.82560350707175 -123.03001120343718) bank171038 +171039 POINT(36.952201735232 -122.79202031904197) bank171039 +171040 POINT(37.92327037965836 -122.6612021859803) bank171040 +171041 POINT(38.064878377378996 -122.95790394827205) bank171041 +171042 POINT(37.62382116112101 -122.6868769574401) bank171042 +171043 POINT(37.09578685703344 -122.42896615450115) bank171043 +171044 POINT(38.09902726690075 -122.03056782654177) bank171044 +171045 POINT(37.38186979715232 -122.80683728698794) bank171045 +171046 POINT(37.68440587345385 -122.40640279063551) bank171046 +171047 POINT(37.52539097047838 -123.24939006745898) bank171047 +171048 POINT(36.995737461091515 -123.27191736437392) bank171048 +171049 POINT(37.30087431554281 -122.58954405647734) bank171049 +171050 POINT(37.57678452933863 -122.9171835703636) bank171050 +171051 POINT(38.4891253208135 -121.64008098370137) bank171051 +171052 POINT(38.772113642744195 -123.09347122932574) bank171052 +171053 POINT(38.55070316134707 -123.30060532242405) bank171053 +171054 POINT(38.49117419948928 -121.56432713645914) bank171054 +171055 POINT(37.25375381102903 -122.07418397694212) bank171055 +171056 POINT(36.97727219541441 -121.84270471492518) bank171056 +171057 POINT(37.17530161411168 -123.0023884767542) bank171057 +171058 POINT(38.676402421524905 -122.68808122218903) bank171058 +171059 POINT(37.97969918757935 -122.92937578295754) bank171059 +171060 POINT(37.84998168081882 -122.53148413318526) bank171060 +171061 POINT(38.75609352981711 -122.6898409484079) bank171061 +171062 POINT(37.87229192270129 -122.72777634322628) bank171062 +171063 POINT(37.72531018859135 -123.11751522345286) bank171063 +171064 POINT(38.17814685072761 -122.89977457719257) bank171064 +171065 POINT(36.84140469819412 -122.72937056186576) bank171065 +171066 POINT(38.16398456229678 -121.61323684067875) bank171066 +171067 POINT(36.80546986585417 -122.48974445272235) bank171067 +171068 POINT(37.683050937103076 -121.71813301980202) bank171068 +171069 POINT(38.276397187189424 -121.60063456451608) bank171069 +171070 POINT(38.145791764277654 -121.47000635636755) bank171070 +171071 POINT(38.589588998732275 -121.94767181516858) bank171071 +171072 POINT(37.891204844941306 -122.94609427841684) bank171072 +171073 POINT(36.801115886292116 -122.54652499062797) bank171073 +171074 POINT(38.21443598597485 -122.42185601394105) bank171074 +171075 POINT(37.854955751396254 -122.45208234314644) bank171075 +171076 POINT(37.391534632005786 -122.18185883340041) bank171076 +171077 POINT(37.37953342923532 -122.75510105761498) bank171077 +171078 POINT(38.50485814967262 -122.44241000037937) bank171078 +171079 POINT(37.55766737847056 -122.56512345608517) bank171079 +171080 POINT(37.151885774834525 -121.4236767765789) bank171080 +171081 POINT(37.58674940908926 -121.60363198914905) bank171081 +171082 POINT(37.999013799161446 -122.81887281927965) bank171082 +171083 POINT(38.60825004687446 -122.59440936854232) bank171083 +171084 POINT(38.73434993415543 -122.2163135999926) bank171084 +171085 POINT(38.68950868259432 -122.42215020419464) bank171085 +171086 POINT(37.997765747548044 -122.52559016731773) bank171086 +171087 POINT(38.35815098325353 -122.1023099068165) bank171087 +171088 POINT(38.13543602235934 -123.08388243839711) bank171088 +171089 POINT(38.15636293845209 -123.34246447236839) bank171089 +171090 POINT(37.998855371455086 -122.40516734654375) bank171090 +171091 POINT(37.42768302581261 -123.39306264998757) bank171091 +171092 POINT(36.857375185742946 -122.78542167907439) bank171092 +171093 POINT(37.732534057665404 -122.45544467462044) bank171093 +171094 POINT(38.09346798158502 -123.12099304861013) bank171094 +171095 POINT(37.442211132357635 -122.54070439588655) bank171095 +171096 POINT(37.67850068282885 -121.49006265016126) bank171096 +171097 POINT(37.54262967786002 -121.84115519139179) bank171097 +171098 POINT(37.361865557040275 -121.4914536741315) bank171098 +171099 POINT(37.63074065437911 -121.58796044646074) bank171099 +171100 POINT(38.128686098480756 -122.09046908218012) bank171100 +171101 POINT(38.551873568010336 -121.76449225557752) bank171101 +171102 POINT(38.06565696599685 -121.90880651692311) bank171102 +171103 POINT(37.11397212973992 -121.50780420153025) bank171103 +171104 POINT(36.90170876242421 -122.36592013816912) bank171104 +171105 POINT(37.907487514151796 -122.28002199179889) bank171105 +171106 POINT(37.38684500556372 -122.0198841238268) bank171106 +171107 POINT(37.9702779728039 -122.42801938856459) bank171107 +171108 POINT(37.24257263399508 -122.6277948484301) bank171108 +171109 POINT(38.10264113937979 -121.52996350467319) bank171109 +171110 POINT(37.43479828728655 -121.42496884314997) bank171110 +171111 POINT(37.02223923867109 -121.44057268342526) bank171111 +171112 POINT(38.37160034291341 -121.60016197350296) bank171112 +171113 POINT(37.73398718553755 -122.18174355589125) bank171113 +171114 POINT(37.49490609054852 -121.98163279500618) bank171114 +171115 POINT(38.628869928428514 -122.54459191240652) bank171115 +171116 POINT(37.172477574135364 -123.41909845406742) bank171116 +171117 POINT(37.803399255309195 -122.24518069335056) bank171117 +171118 POINT(38.12856994877579 -122.83880962108196) bank171118 +171119 POINT(37.19802712732391 -122.93412172725581) bank171119 +171120 POINT(37.26844614717386 -123.06389881139509) bank171120 +171121 POINT(38.39613290065991 -122.42154361255932) bank171121 +171122 POINT(37.407052018142025 -123.36538903500572) bank171122 +171123 POINT(38.583604836392986 -122.46869216442597) bank171123 +171124 POINT(38.0504407178959 -123.02959367896474) bank171124 +171125 POINT(38.077011462794935 -123.04587022160824) bank171125 +171126 POINT(37.564747993822245 -121.45826491962173) bank171126 +171127 POINT(37.28901684232209 -122.23135151858516) bank171127 +171128 POINT(37.5449434756466 -121.55322250439835) bank171128 +171129 POINT(38.48099108900228 -121.89147790791475) bank171129 +171130 POINT(37.69084528816703 -122.46231401252876) bank171130 +171131 POINT(37.013270692625525 -122.45115328099331) bank171131 +171132 POINT(36.8104664803311 -122.06053728526763) bank171132 +171133 POINT(38.12547748441211 -123.28325010503609) bank171133 +171134 POINT(37.88530878211669 -121.42885497464732) bank171134 +171135 POINT(36.790115213241556 -123.09321016349973) bank171135 +171136 POINT(37.04201541506251 -122.99347588738712) bank171136 +171137 POINT(36.866250501777785 -122.80101479872972) bank171137 +171138 POINT(37.15643017617714 -123.2655008129133) bank171138 +171139 POINT(38.70667121683948 -122.82201631566923) bank171139 +171140 POINT(38.1754651790104 -122.97680690553871) bank171140 +171141 POINT(37.91990221898144 -121.74317948547687) bank171141 +171142 POINT(38.163005734741354 -121.59072971189255) bank171142 +171143 POINT(37.49989819756547 -122.09396302870887) bank171143 +171144 POINT(37.14455160673146 -122.45499054216263) bank171144 +171145 POINT(37.25217022591877 -122.92518560617813) bank171145 +171146 POINT(37.00310959857428 -122.06097629598611) bank171146 +171147 POINT(37.383748564316576 -121.73932426016121) bank171147 +171148 POINT(37.32710244218976 -121.9080336015346) bank171148 +171149 POINT(38.317294600802036 -121.53347191260141) bank171149 +171150 POINT(36.956015297608175 -121.61486386811285) bank171150 +171151 POINT(38.01234889557018 -123.148463532494) bank171151 +171152 POINT(37.64413054861752 -122.52791148663937) bank171152 +171153 POINT(38.554399109570724 -123.32369442454258) bank171153 +171154 POINT(37.14347149851378 -123.24538118664678) bank171154 +171155 POINT(38.63431080671328 -121.98957399931693) bank171155 +171156 POINT(37.73466028309711 -123.04233868925759) bank171156 +171157 POINT(37.35374624853648 -122.06759558290192) bank171157 +171158 POINT(37.857137818672996 -123.22858054814417) bank171158 +171159 POINT(37.952557588229396 -121.78799450012477) bank171159 +171160 POINT(38.36859681743018 -122.57749615869545) bank171160 +171161 POINT(38.59226304617873 -123.12638278266024) bank171161 +171162 POINT(38.22133690451989 -122.74561874444751) bank171162 +171163 POINT(37.522496578167186 -121.94735580006649) bank171163 +171164 POINT(38.651561899004165 -122.1389679183066) bank171164 +171165 POINT(37.40800328727015 -122.61513257722018) bank171165 +171166 POINT(36.973339498298735 -122.99663704368452) bank171166 +171167 POINT(38.171100682567825 -122.89573314595953) bank171167 +171168 POINT(38.44544954292676 -122.63378482694316) bank171168 +171169 POINT(36.977294045358526 -121.98982475909806) bank171169 +171170 POINT(38.51992307422176 -122.73255205444862) bank171170 +171171 POINT(38.242504157256946 -122.29883903049043) bank171171 +171172 POINT(36.99048826097395 -121.6874829601334) bank171172 +171173 POINT(38.59326207731107 -122.58284330314493) bank171173 +171174 POINT(37.09615682480021 -122.45720980563445) bank171174 +171175 POINT(37.96868057180765 -122.35065037004779) bank171175 +171176 POINT(37.97135865806926 -121.59295276313011) bank171176 +171177 POINT(36.79241762843686 -122.37862515440717) bank171177 +171178 POINT(38.40814508058652 -122.48500472589463) bank171178 +171179 POINT(37.92356381342849 -122.51347535126413) bank171179 +171180 POINT(37.08909043686611 -121.83593647661739) bank171180 +171181 POINT(37.81737681846403 -122.7445579090846) bank171181 +171182 POINT(37.24917728870984 -121.92540995102094) bank171182 +171183 POINT(38.24529247282473 -122.92746265048548) bank171183 +171184 POINT(37.169426829487556 -121.79010150243676) bank171184 +171185 POINT(36.8513822800057 -122.92270291975682) bank171185 +171186 POINT(38.24839463385574 -122.28090727905668) bank171186 +171187 POINT(36.94390912192438 -122.49951751056875) bank171187 +171188 POINT(37.34534230096064 -122.94379788899644) bank171188 +171189 POINT(38.641276400249964 -122.8010123465974) bank171189 +171190 POINT(37.72355440807648 -121.88856826085616) bank171190 +171191 POINT(38.50839474406745 -122.21378249355521) bank171191 +171192 POINT(37.34395241510399 -123.05244533495818) bank171192 +171193 POINT(38.63835781097962 -122.44498175033394) bank171193 +171194 POINT(37.28024408672915 -122.54570867240666) bank171194 +171195 POINT(36.87488927698617 -122.85927485411331) bank171195 +171196 POINT(36.95377035348746 -122.58603741460115) bank171196 +171197 POINT(37.004291113380084 -122.80002057238273) bank171197 +171198 POINT(38.01364032840842 -123.4094197197401) bank171198 +171199 POINT(37.59311672089457 -122.85471048747083) bank171199 +171200 POINT(38.58840599424319 -122.9623618374208) bank171200 +171201 POINT(37.96158118980037 -123.21810887509308) bank171201 +171202 POINT(38.047959753200935 -121.42509602851194) bank171202 +171203 POINT(38.446241938231715 -121.71474729256425) bank171203 +171204 POINT(37.151986972245915 -122.86994224010031) bank171204 +171205 POINT(37.5376968723345 -122.95922365029959) bank171205 +171206 POINT(36.98262341172364 -122.21998289238724) bank171206 +171207 POINT(37.057924435086626 -121.86850847634909) bank171207 +171208 POINT(38.680536445724876 -121.63562197988858) bank171208 +171209 POINT(37.48427006562412 -122.1853553903605) bank171209 +171210 POINT(37.9122573812144 -122.655825118016) bank171210 +171211 POINT(37.993199699260344 -122.28309061541675) bank171211 +171212 POINT(37.877157920799235 -122.54247983981199) bank171212 +171213 POINT(37.17937962061848 -122.25562731438355) bank171213 +171214 POINT(37.23154742119162 -122.61481498580049) bank171214 +171215 POINT(38.18620420832354 -122.78696457590347) bank171215 +171216 POINT(38.11780217352321 -121.43391705933308) bank171216 +171217 POINT(37.21237169795183 -122.0677326652511) bank171217 +171218 POINT(37.00173231861116 -122.7803199245049) bank171218 +171219 POINT(36.952427023859215 -122.97199930582485) bank171219 +171220 POINT(37.832712697540174 -121.60574535705481) bank171220 +171221 POINT(37.768673624964286 -122.20798802609441) bank171221 +171222 POINT(37.60831964856585 -122.04463300870485) bank171222 +171223 POINT(37.068376914046866 -123.34326108309689) bank171223 +171224 POINT(38.75751718431934 -122.53746184329282) bank171224 +171225 POINT(37.96295171279692 -122.3117873011653) bank171225 +171226 POINT(37.33857523415789 -121.44054104218023) bank171226 +171227 POINT(37.79130926063283 -122.29651902834624) bank171227 +171228 POINT(37.83826650469081 -122.76414270469576) bank171228 +171229 POINT(38.169451214387124 -121.96257174499422) bank171229 +171230 POINT(37.27811098771255 -121.65753094304992) bank171230 +171231 POINT(37.04945595284472 -121.91841732418234) bank171231 +171232 POINT(36.77513610416246 -121.47493694946581) bank171232 +171233 POINT(37.64333543177948 -121.61954908299295) bank171233 +171234 POINT(38.06940499740176 -122.18474820129282) bank171234 +171235 POINT(38.04656168772215 -121.96652029378917) bank171235 +171236 POINT(38.015655349349075 -123.24157320713388) bank171236 +171237 POINT(37.67055123460885 -121.59947689957933) bank171237 +171238 POINT(37.67327406400108 -121.65320643148439) bank171238 +171239 POINT(37.26227391780832 -122.90274907445377) bank171239 +171240 POINT(37.98407211981793 -122.18140896833684) bank171240 +171241 POINT(37.196087769577055 -122.28610453362275) bank171241 +171242 POINT(37.463745636631444 -121.91704133428358) bank171242 +171243 POINT(37.396058858930324 -122.1634683832326) bank171243 +171244 POINT(37.52086400178804 -122.01114025389866) bank171244 +171245 POINT(37.63540083715206 -122.11965507045096) bank171245 +171246 POINT(38.42502322724253 -122.79508125044376) bank171246 +171247 POINT(37.8267496111641 -123.3152494707509) bank171247 +171248 POINT(38.423196944542646 -121.47635268666174) bank171248 +171249 POINT(36.83809904829901 -122.193170592569) bank171249 +171250 POINT(38.771630458775334 -121.83439960086685) bank171250 +171251 POINT(38.346925796373206 -122.0446384163901) bank171251 +171252 POINT(38.287667881555905 -122.4558474752066) bank171252 +171253 POINT(38.404147444223725 -122.29631226496872) bank171253 +171254 POINT(37.71683932043447 -122.4354393110173) bank171254 +171255 POINT(38.40009992199002 -121.77158732797704) bank171255 +171256 POINT(38.50632035007907 -121.6996003190739) bank171256 +171257 POINT(36.84835046672479 -122.59760183018865) bank171257 +171258 POINT(37.73525262640285 -123.15780923499422) bank171258 +171259 POINT(38.58984973407706 -122.93601880817644) bank171259 +171260 POINT(37.74245747655722 -122.13928228308718) bank171260 +171261 POINT(38.634577232898444 -122.71835983725963) bank171261 +171262 POINT(37.17981240629203 -123.21014644689558) bank171262 +171263 POINT(37.419974854600696 -123.04458933740882) bank171263 +171264 POINT(38.071040264136094 -121.93409525206037) bank171264 +171265 POINT(38.221341148037446 -122.39154688739953) bank171265 +171266 POINT(37.4607523391923 -121.83217612679219) bank171266 +171267 POINT(37.09313782570147 -122.63913232710131) bank171267 +171268 POINT(37.886805511691065 -121.80111169280684) bank171268 +171269 POINT(36.90646778778523 -122.52517418022623) bank171269 +171270 POINT(37.05164413801686 -121.96466576727119) bank171270 +171271 POINT(37.58934969309158 -123.06638314976577) bank171271 +171272 POINT(38.05897120686512 -122.60952393885454) bank171272 +171273 POINT(38.01458459437423 -122.08259264128606) bank171273 +171274 POINT(37.70934684866551 -121.45192566190843) bank171274 +171275 POINT(37.574450534779814 -122.7910597545753) bank171275 +171276 POINT(37.64706242008809 -122.44807882691568) bank171276 +171277 POINT(37.584639235728204 -121.80571953636029) bank171277 +171278 POINT(38.732987868470595 -121.5948713000645) bank171278 +171279 POINT(38.61287155493054 -122.79256347570055) bank171279 +171280 POINT(37.10709979264494 -122.25329626818046) bank171280 +171281 POINT(38.08265007254848 -121.77418816280581) bank171281 +171282 POINT(38.402240500285096 -122.83467282975631) bank171282 +171283 POINT(38.045729561634325 -123.40222319913183) bank171283 +171284 POINT(37.89686236661113 -122.73148369597915) bank171284 +171285 POINT(38.32406060109424 -122.74064288480287) bank171285 +171286 POINT(37.117357557339446 -122.95751491014971) bank171286 +171287 POINT(36.80567641749221 -123.3614804263796) bank171287 +171288 POINT(38.15592394381672 -122.41367762228485) bank171288 +171289 POINT(37.93948500143328 -122.022668287618) bank171289 +171290 POINT(36.85987574598567 -122.92308726170884) bank171290 +171291 POINT(38.688397284514835 -122.4348321213928) bank171291 +171292 POINT(37.10069152166501 -121.74721776941195) bank171292 +171293 POINT(38.2728402813725 -123.39679408497564) bank171293 +171294 POINT(37.34978620863792 -122.31876451352372) bank171294 +171295 POINT(36.781872417268396 -122.82516829585929) bank171295 +171296 POINT(37.37265512329052 -122.53848194723771) bank171296 +171297 POINT(38.43390065693922 -122.78420474707077) bank171297 +171298 POINT(37.59176904791372 -121.52838683192236) bank171298 +171299 POINT(38.0544587540838 -122.98409453218002) bank171299 +171300 POINT(37.85781552006279 -121.9674718280136) bank171300 +171301 POINT(37.354902922371565 -123.29567272370748) bank171301 +171302 POINT(36.916553259543946 -123.04826849067254) bank171302 +171303 POINT(36.970905765149354 -122.78927471700916) bank171303 +171304 POINT(38.03533048631283 -122.610488054655) bank171304 +171305 POINT(37.71672887715723 -122.80838767471012) bank171305 +171306 POINT(38.18767302844942 -123.30983769454238) bank171306 +171307 POINT(38.73939142237408 -122.79446971100741) bank171307 +171308 POINT(38.10108631614047 -122.21584770099416) bank171308 +171309 POINT(37.35023850701427 -122.733496226181) bank171309 +171310 POINT(37.35613979203164 -122.84132051877206) bank171310 +171311 POINT(37.60960039206869 -121.55517568297337) bank171311 +171312 POINT(38.26173871089289 -121.91932109945357) bank171312 +171313 POINT(38.26220542707914 -122.71451015345052) bank171313 +171314 POINT(37.24348476218829 -122.41552754465518) bank171314 +171315 POINT(37.73457424535487 -121.87769881911707) bank171315 +171316 POINT(38.717953954469 -123.07086416674191) bank171316 +171317 POINT(38.33652856606818 -123.07361398557818) bank171317 +171318 POINT(37.39077197331987 -121.69727369495058) bank171318 +171319 POINT(38.64094638967714 -121.98503430794369) bank171319 +171320 POINT(37.03773928871606 -122.38566797564168) bank171320 +171321 POINT(37.58769177059314 -122.04750257941203) bank171321 +171322 POINT(37.3872347100412 -122.9259662023102) bank171322 +171323 POINT(37.1605620744071 -122.84237592611261) bank171323 +171324 POINT(38.348334281683975 -122.77557837412108) bank171324 +171325 POINT(37.42287854217762 -123.24138191700374) bank171325 +171326 POINT(37.83064151462336 -122.5856002109447) bank171326 +171327 POINT(38.39439040313957 -122.64137845156037) bank171327 +171328 POINT(37.72623811444427 -121.93326951079769) bank171328 +171329 POINT(37.27219134301587 -122.2139668223754) bank171329 +171330 POINT(38.67447628389739 -122.43036412755977) bank171330 +171331 POINT(38.22905541197464 -122.8860077491516) bank171331 +171332 POINT(37.09499212213629 -122.24908149607768) bank171332 +171333 POINT(37.92626654102256 -122.71614308237086) bank171333 +171334 POINT(36.922816967663856 -122.36390070770035) bank171334 +171335 POINT(37.241979962305756 -121.85659566110898) bank171335 +171336 POINT(38.553281700497735 -121.94165445707917) bank171336 +171337 POINT(38.581363683736285 -121.59602124957166) bank171337 +171338 POINT(37.412695758964404 -121.45967378490833) bank171338 +171339 POINT(37.13131514232148 -122.29747919032832) bank171339 +171340 POINT(37.126723836337334 -123.13361892359761) bank171340 +171341 POINT(38.38800625009483 -122.22691523794809) bank171341 +171342 POINT(37.68910380097204 -122.11158073211739) bank171342 +171343 POINT(38.479279851616226 -122.95441820780967) bank171343 +171344 POINT(38.39397344135791 -122.96683324171602) bank171344 +171345 POINT(37.34692188696645 -123.28322369996123) bank171345 +171346 POINT(38.06726496499276 -122.89979483088769) bank171346 +171347 POINT(38.449947667741846 -122.51014661162723) bank171347 +171348 POINT(37.379481277960096 -123.05391977211485) bank171348 +171349 POINT(37.87425519743098 -122.29116332703069) bank171349 +171350 POINT(38.22264366330219 -122.20668662417654) bank171350 +171351 POINT(38.01283313554782 -123.11906689434889) bank171351 +171352 POINT(37.398546640801776 -122.77541487090205) bank171352 +171353 POINT(37.33741825300067 -122.59166405452393) bank171353 +171354 POINT(37.620368024171455 -122.49486004435565) bank171354 +171355 POINT(37.400793837721984 -122.83967716506955) bank171355 +171356 POINT(38.07662324982083 -122.18951595282118) bank171356 +171357 POINT(38.74814734900102 -122.28395714396534) bank171357 +171358 POINT(37.61264038000377 -123.39482064671999) bank171358 +171359 POINT(38.70342454936189 -121.88137886670062) bank171359 +171360 POINT(37.64421726528303 -121.53937848804932) bank171360 +171361 POINT(36.9602921731707 -121.57782005422186) bank171361 +171362 POINT(37.722968706558106 -122.99938533283087) bank171362 +171363 POINT(37.07649279655115 -121.97765249060191) bank171363 +171364 POINT(37.73386891510522 -121.52012887908754) bank171364 +171365 POINT(37.81216540738443 -123.0314848803825) bank171365 +171366 POINT(37.904692347040815 -122.84005947688672) bank171366 +171367 POINT(38.61565228013506 -122.49516082375746) bank171367 +171368 POINT(36.82401730582243 -122.61095186881121) bank171368 +171369 POINT(37.959378296892325 -121.70230761981173) bank171369 +171370 POINT(37.642253285869764 -122.50573602723995) bank171370 +171371 POINT(37.29726715805269 -121.63951841618673) bank171371 +171372 POINT(38.62740630347334 -122.4031139756125) bank171372 +171373 POINT(37.06151211686509 -123.33462439739019) bank171373 +171374 POINT(38.00475570861234 -122.34459466455091) bank171374 +171375 POINT(37.37359319791638 -122.77638480279182) bank171375 +171376 POINT(36.93306553913664 -121.69162465975948) bank171376 +171377 POINT(37.305346720891784 -122.94657179058274) bank171377 +171378 POINT(36.9025663295879 -122.7869253261208) bank171378 +171379 POINT(38.39005203698906 -121.63833537176036) bank171379 +171380 POINT(37.331510426043515 -122.7519801484381) bank171380 +171381 POINT(37.562331871586466 -121.82938252636139) bank171381 +171382 POINT(36.99048607290906 -122.16598396556138) bank171382 +171383 POINT(37.50201199755623 -122.8929840000388) bank171383 +171384 POINT(37.428115590056905 -121.71664093725273) bank171384 +171385 POINT(37.4739023497004 -123.3646623507298) bank171385 +171386 POINT(37.136258350772664 -121.76838508753939) bank171386 +171387 POINT(38.140607405496134 -122.24967032909989) bank171387 +171388 POINT(38.52497769658557 -121.75288203398485) bank171388 +171389 POINT(37.69490851495526 -123.35085462612265) bank171389 +171390 POINT(37.92696863617963 -123.0558289977643) bank171390 +171391 POINT(37.64665770652977 -121.93601603710853) bank171391 +171392 POINT(37.627337474390245 -123.14868230268665) bank171392 +171393 POINT(38.29283699042467 -122.25286419026759) bank171393 +171394 POINT(37.576918476314 -122.81744126928905) bank171394 +171395 POINT(37.9419493825576 -121.8301690376759) bank171395 +171396 POINT(38.044799642552874 -122.16890426751549) bank171396 +171397 POINT(36.977990692427426 -123.06208848113883) bank171397 +171398 POINT(38.06307974724836 -122.70684837490944) bank171398 +171399 POINT(37.3622735512308 -122.09508807073988) bank171399 +171400 POINT(38.169331305155254 -122.83162009937928) bank171400 +171401 POINT(37.55342351863499 -122.9105994650937) bank171401 +171402 POINT(38.0402604968575 -122.80476409084301) bank171402 +171403 POINT(38.63190496150594 -121.62648468615376) bank171403 +171404 POINT(38.60133471390921 -123.01930946686586) bank171404 +171405 POINT(37.99260004780037 -122.01716863897313) bank171405 +171406 POINT(38.33837982154034 -122.55845560493924) bank171406 +171407 POINT(37.7478116692273 -122.28181184542579) bank171407 +171408 POINT(38.443996280855586 -122.64718905412906) bank171408 +171409 POINT(38.16993485684754 -121.66300129022338) bank171409 +171410 POINT(37.486076911260014 -122.77055537674215) bank171410 +171411 POINT(38.5382395081121 -122.68540469995705) bank171411 +171412 POINT(38.73613241646083 -121.71214015054173) bank171412 +171413 POINT(38.68309381207763 -121.63665234225921) bank171413 +171414 POINT(37.191270853522745 -122.91854098055623) bank171414 +171415 POINT(38.13836129823023 -123.1214529296143) bank171415 +171416 POINT(37.16552251520293 -122.46176357415884) bank171416 +171417 POINT(38.3212288838093 -121.48395185864287) bank171417 +171418 POINT(36.96840086080355 -122.084197174668) bank171418 +171419 POINT(38.3953824173544 -122.97917882529832) bank171419 +171420 POINT(37.555325914672565 -122.81163135964304) bank171420 +171421 POINT(38.67332254006223 -122.57076293757707) bank171421 +171422 POINT(37.10548314122701 -122.0297999288107) bank171422 +171423 POINT(37.5756176565837 -121.70942330006181) bank171423 +171424 POINT(38.15740297863778 -121.90680963676664) bank171424 +171425 POINT(37.23645337288734 -122.29579071237664) bank171425 +171426 POINT(38.153620098135384 -122.87719903035283) bank171426 +171427 POINT(38.42798734445462 -121.59690987112181) bank171427 +171428 POINT(37.92900035157848 -122.71738481649578) bank171428 +171429 POINT(37.91143613831663 -123.37182095804764) bank171429 +171430 POINT(36.82679459581631 -122.99857493559479) bank171430 +171431 POINT(36.80170352951133 -122.69850607564831) bank171431 +171432 POINT(38.17043522525715 -122.00524645415699) bank171432 +171433 POINT(38.205960647877234 -122.48138398513825) bank171433 +171434 POINT(38.58499453167202 -122.28908078779318) bank171434 +171435 POINT(37.96956814095215 -122.47150666275475) bank171435 +171436 POINT(38.15783272777523 -122.85404773225416) bank171436 +171437 POINT(37.81177246034691 -123.15346688613121) bank171437 +171438 POINT(37.61717643761801 -122.60795719548842) bank171438 +171439 POINT(38.68443274749319 -122.97069874870807) bank171439 +171440 POINT(37.87378309768615 -122.06065935209763) bank171440 +171441 POINT(36.924071523608376 -123.40568900142986) bank171441 +171442 POINT(37.32656926399649 -123.18360302448662) bank171442 +171443 POINT(37.77489851064696 -123.11261798692769) bank171443 +171444 POINT(37.39825929239079 -122.20285699502826) bank171444 +171445 POINT(36.930277177079645 -122.96047211275345) bank171445 +171446 POINT(37.8817700370162 -122.94047471452355) bank171446 +171447 POINT(38.41068288190334 -123.02199981675236) bank171447 +171448 POINT(37.7000200601863 -122.63653823091751) bank171448 +171449 POINT(38.59135828036302 -121.50243884600684) bank171449 +171450 POINT(37.60908547214686 -122.66912529408553) bank171450 +171451 POINT(37.74770469941424 -122.77729861096768) bank171451 +171452 POINT(38.61992137863587 -121.90159324240955) bank171452 +171453 POINT(38.514798820788 -121.43502744521896) bank171453 +171454 POINT(37.77497446066273 -121.468663441343) bank171454 +171455 POINT(37.24829941744518 -123.36900716666496) bank171455 +171456 POINT(36.970344520098664 -123.33164219835199) bank171456 +171457 POINT(36.95866458788468 -121.68617016197281) bank171457 +171458 POINT(37.711423850479164 -122.13002407101607) bank171458 +171459 POINT(38.022602464090276 -122.75525642533199) bank171459 +171460 POINT(38.334930446521334 -122.6438533883511) bank171460 +171461 POINT(37.18496197728183 -122.01157366829139) bank171461 +171462 POINT(38.69215019860715 -122.02848220183708) bank171462 +171463 POINT(38.47121560818368 -123.2241813851938) bank171463 +171464 POINT(38.648173559509125 -121.6072523280711) bank171464 +171465 POINT(37.89834103551027 -122.9650396793758) bank171465 +171466 POINT(37.0662671787162 -122.05038171316286) bank171466 +171467 POINT(38.404389881860254 -123.09795009784226) bank171467 +171468 POINT(38.51508863287223 -122.1110717769302) bank171468 +171469 POINT(37.90534814913369 -121.70547383883209) bank171469 +171470 POINT(37.03156426168974 -122.58934575545013) bank171470 +171471 POINT(38.74295939196826 -122.07390551740504) bank171471 +171472 POINT(37.70424020696022 -122.96544932539169) bank171472 +171473 POINT(38.15099406172773 -123.25597259407206) bank171473 +171474 POINT(38.45373311777924 -122.11222411691574) bank171474 +171475 POINT(38.18457142025949 -122.0180812689407) bank171475 +171476 POINT(37.03236386127614 -122.81856557999517) bank171476 +171477 POINT(37.75430861507969 -121.87519264078924) bank171477 +171478 POINT(37.17073305552263 -121.97092465070642) bank171478 +171479 POINT(37.5367365221364 -123.04712180018613) bank171479 +171480 POINT(38.36534362804624 -122.03033800154483) bank171480 +171481 POINT(37.49659395131049 -121.42573409230847) bank171481 +171482 POINT(38.54833641603711 -122.77346484316632) bank171482 +171483 POINT(36.864880857007215 -122.19628255878406) bank171483 +171484 POINT(38.20140736938273 -121.52070253819663) bank171484 +171485 POINT(37.851840914841915 -121.8207337260466) bank171485 +171486 POINT(38.14084143101612 -122.41312253780347) bank171486 +171487 POINT(36.83554636254632 -122.11306978899655) bank171487 +171488 POINT(37.426247292722195 -122.55956974042259) bank171488 +171489 POINT(37.565101277245155 -123.31666939133443) bank171489 +171490 POINT(37.87900821013981 -121.78058125991137) bank171490 +171491 POINT(37.05743122805619 -122.4419975607677) bank171491 +171492 POINT(38.505725087863745 -122.07099753626795) bank171492 +171493 POINT(37.28540865982274 -121.83665485190916) bank171493 +171494 POINT(36.849576582284286 -122.34974185171161) bank171494 +171495 POINT(38.41393238505347 -122.09290543737976) bank171495 +171496 POINT(38.15226707897968 -122.88029630298382) bank171496 +171497 POINT(38.32237227054852 -123.0683010103288) bank171497 +171498 POINT(37.036891566523146 -122.27083935721143) bank171498 +171499 POINT(36.90842403980994 -122.27901577353504) bank171499 +171500 POINT(37.7968361899574 -122.50908428216083) bank171500 +171501 POINT(37.08715085205437 -122.12982882822244) bank171501 +171502 POINT(38.57835605178899 -122.4836680586128) bank171502 +171503 POINT(37.43654932506215 -122.35193273494023) bank171503 +171504 POINT(38.39863575262956 -122.47303043268667) bank171504 +171505 POINT(37.21276907062199 -123.4026032744381) bank171505 +171506 POINT(37.02137325647889 -122.6815929041163) bank171506 +171507 POINT(37.36024746510397 -122.59558344156682) bank171507 +171508 POINT(37.81888093927094 -121.68644751837799) bank171508 +171509 POINT(36.95258493970674 -122.60935229008854) bank171509 +171510 POINT(37.73233640348622 -121.42513364612931) bank171510 +171511 POINT(36.858765566798745 -122.2236793047918) bank171511 +171512 POINT(37.44272893154114 -122.38243817010849) bank171512 +171513 POINT(38.08037196811292 -121.85409859374167) bank171513 +171514 POINT(38.281720368844326 -122.50504604013544) bank171514 +171515 POINT(37.127140124240356 -123.23467049599701) bank171515 +171516 POINT(38.31761477029907 -123.15100160522096) bank171516 +171517 POINT(37.85851052079165 -123.41363031000155) bank171517 +171518 POINT(37.22145588443935 -121.53760105688417) bank171518 +171519 POINT(38.6866888959251 -122.17942630749656) bank171519 +171520 POINT(37.412331163774766 -121.45529513605098) bank171520 +171521 POINT(36.97154370235713 -121.64383897142605) bank171521 +171522 POINT(38.75107671924248 -122.79689062503141) bank171522 +171523 POINT(38.71737656453533 -121.6866705637002) bank171523 +171524 POINT(36.847376110662296 -121.96437057203943) bank171524 +171525 POINT(37.18999757384448 -122.98545200362327) bank171525 +171526 POINT(37.946068560933476 -122.75907028616932) bank171526 +171527 POINT(37.62263467036061 -122.06655409180938) bank171527 +171528 POINT(38.75881753990421 -123.01787036168527) bank171528 +171529 POINT(37.668993829244265 -121.97949843544507) bank171529 +171530 POINT(38.37417102482978 -122.89071564618274) bank171530 +171531 POINT(37.53678643782341 -121.57151209119318) bank171531 +171532 POINT(37.749483082597784 -122.76270774918609) bank171532 +171533 POINT(37.08271611190431 -121.49012972750415) bank171533 +171534 POINT(38.15208676578861 -121.4226225041612) bank171534 +171535 POINT(37.89046778155396 -121.61587236659847) bank171535 +171536 POINT(37.56407965740446 -122.64046498696234) bank171536 +171537 POINT(37.975386673290316 -123.16594209985696) bank171537 +171538 POINT(38.7531888839402 -123.0244864171502) bank171538 +171539 POINT(38.42659691159814 -121.7237147953487) bank171539 +171540 POINT(36.86167759502707 -122.93642671952364) bank171540 +171541 POINT(38.60560578306157 -122.82245635580045) bank171541 +171542 POINT(37.12754718759644 -122.21275093243631) bank171542 +171543 POINT(38.04708256044299 -121.63022354751486) bank171543 +171544 POINT(36.87803497285501 -122.29072076363617) bank171544 +171545 POINT(37.54644328762236 -121.53689547633853) bank171545 +171546 POINT(37.59586287047657 -122.30800959413062) bank171546 +171547 POINT(38.54146871791845 -122.88836889003015) bank171547 +171548 POINT(37.774507435641986 -122.94487058188444) bank171548 +171549 POINT(38.60136711398203 -122.76823342073962) bank171549 +171550 POINT(38.033243236655956 -123.02375950162921) bank171550 +171551 POINT(36.9381426860849 -123.28800049997784) bank171551 +171552 POINT(37.30185083150584 -122.18277133359021) bank171552 +171553 POINT(38.32658989413321 -122.6296121269824) bank171553 +171554 POINT(38.13055009280717 -123.1019306391614) bank171554 +171555 POINT(38.63464514044433 -123.03313966575378) bank171555 +171556 POINT(38.12806050968237 -123.10570978168298) bank171556 +171557 POINT(38.105283919528716 -122.47310483675122) bank171557 +171558 POINT(38.676642642836406 -122.95676125594103) bank171558 +171559 POINT(37.603875118190984 -122.94366839734525) bank171559 +171560 POINT(36.78319665259384 -122.15547884672077) bank171560 +171561 POINT(37.716765234544155 -123.19081922317955) bank171561 +171562 POINT(37.454255058516836 -122.46792775839661) bank171562 +171563 POINT(37.0039730972597 -122.72011238707397) bank171563 +171564 POINT(38.590635923911904 -122.89031345385196) bank171564 +171565 POINT(37.57464079061063 -122.61753951513909) bank171565 +171566 POINT(38.57165727914435 -122.57965245474968) bank171566 +171567 POINT(38.49031628935569 -123.2222392023074) bank171567 +171568 POINT(37.605529941456375 -122.48852234479392) bank171568 +171569 POINT(37.79473849279202 -123.17927296643249) bank171569 +171570 POINT(37.54246306947172 -122.8798073911714) bank171570 +171571 POINT(37.95328753367937 -122.80898234590116) bank171571 +171572 POINT(38.45547012195942 -121.69380524390023) bank171572 +171573 POINT(38.39474794615537 -121.74106089823185) bank171573 +171574 POINT(37.55908486014096 -121.99951841176207) bank171574 +171575 POINT(38.647575621069336 -121.56295161093264) bank171575 +171576 POINT(38.37553530562313 -122.66504137626896) bank171576 +171577 POINT(36.77606774575436 -122.20159798243023) bank171577 +171578 POINT(36.90136657243962 -122.98382664703973) bank171578 +171579 POINT(37.804256868428745 -122.50013509305072) bank171579 +171580 POINT(38.26085396518594 -122.43581182514824) bank171580 +171581 POINT(38.56831173142992 -122.52684478008271) bank171581 +171582 POINT(38.33273570443559 -121.85150816098684) bank171582 +171583 POINT(38.08903276058009 -122.5019252399784) bank171583 +171584 POINT(38.66805939790684 -121.95940985799457) bank171584 +171585 POINT(37.90258801699828 -122.26097199498153) bank171585 +171586 POINT(38.28167472050236 -122.55300579160553) bank171586 +171587 POINT(37.37318929372578 -121.49692687487286) bank171587 +171588 POINT(37.105699978204356 -122.9104806725484) bank171588 +171589 POINT(38.568660089178444 -123.00095239790517) bank171589 +171590 POINT(37.25412275170538 -122.40976902425176) bank171590 +171591 POINT(38.413710287025864 -122.84585089512828) bank171591 +171592 POINT(37.47818224293866 -121.9996782710711) bank171592 +171593 POINT(38.607089359570054 -122.65854447728573) bank171593 +171594 POINT(36.99326852799332 -122.60018475520891) bank171594 +171595 POINT(37.690320292617024 -122.53118371701132) bank171595 +171596 POINT(36.89787684122045 -123.28855572821978) bank171596 +171597 POINT(37.19943069295959 -121.93031346670368) bank171597 +171598 POINT(38.23104115912269 -122.33752243062925) bank171598 +171599 POINT(38.48006691456227 -121.5242071114814) bank171599 +171600 POINT(38.40683538428564 -122.6242702934885) bank171600 +171601 POINT(37.41984786519702 -122.42379920268664) bank171601 +171602 POINT(37.49641731369789 -121.61195161563927) bank171602 +171603 POINT(37.96090178681711 -121.79531363657264) bank171603 +171604 POINT(37.454730674999 -121.57545532349992) bank171604 +171605 POINT(37.33097391348757 -122.13927004925678) bank171605 +171606 POINT(38.15513617233742 -123.33814652024627) bank171606 +171607 POINT(37.93533185350613 -122.03682675480017) bank171607 +171608 POINT(38.58182763784676 -123.4093769545278) bank171608 +171609 POINT(38.19744954392185 -122.62970432693578) bank171609 +171610 POINT(37.23923014650502 -121.97995550644748) bank171610 +171611 POINT(37.29730091452301 -122.28134006616496) bank171611 +171612 POINT(38.127634872607764 -122.34871160667507) bank171612 +171613 POINT(38.38137630522725 -122.40063232379389) bank171613 +171614 POINT(38.03070097750522 -122.58692136642375) bank171614 +171615 POINT(38.341514623414746 -121.71551113625007) bank171615 +171616 POINT(37.62552776599773 -122.70679878814174) bank171616 +171617 POINT(37.29633251550483 -123.31150456460489) bank171617 +171618 POINT(38.601239839231255 -123.38125069120099) bank171618 +171619 POINT(38.42923904077611 -122.36548846500453) bank171619 +171620 POINT(38.64982432585329 -122.49672378091489) bank171620 +171621 POINT(37.43726658801453 -123.15931979784771) bank171621 +171622 POINT(37.266604186818356 -122.68053312274988) bank171622 +171623 POINT(36.87556292406645 -122.69131448931789) bank171623 +171624 POINT(37.58525051000399 -121.99853210052916) bank171624 +171625 POINT(38.47052058627549 -122.88189932670576) bank171625 +171626 POINT(37.59527146005117 -122.55935687407327) bank171626 +171627 POINT(37.39163648731946 -122.38694103742856) bank171627 +171628 POINT(38.35114323980629 -123.1615297527993) bank171628 +171629 POINT(37.540052922310466 -121.52108347631437) bank171629 +171630 POINT(38.075603381998036 -122.38639953621019) bank171630 +171631 POINT(38.317256852781675 -123.06808643518183) bank171631 +171632 POINT(38.46275210326375 -123.36169446699398) bank171632 +171633 POINT(37.24384316597895 -123.1778255755362) bank171633 +171634 POINT(37.432417986216294 -122.72121604259125) bank171634 +171635 POINT(37.95279322580946 -121.9886046059928) bank171635 +171636 POINT(37.03422693148273 -123.10006271317737) bank171636 +171637 POINT(38.56565843002645 -122.97490835501085) bank171637 +171638 POINT(37.126796232587 -123.24542934150554) bank171638 +171639 POINT(37.0898362980215 -123.37969158650309) bank171639 +171640 POINT(38.162350835804496 -122.78925709969754) bank171640 +171641 POINT(37.98316639058989 -121.68011549173328) bank171641 +171642 POINT(37.13058855180964 -122.18646862585247) bank171642 +171643 POINT(37.00862093386991 -122.64227292962947) bank171643 +171644 POINT(37.88605084678192 -121.67375942045183) bank171644 +171645 POINT(38.515479111562065 -122.52276100816476) bank171645 +171646 POINT(38.25823714375319 -121.78685723925807) bank171646 +171647 POINT(37.333456114709676 -123.11794719497706) bank171647 +171648 POINT(37.16031271322448 -121.49941116531014) bank171648 +171649 POINT(36.91961795434017 -121.74326079358445) bank171649 +171650 POINT(37.279164624575195 -122.61540722866074) bank171650 +171651 POINT(38.438565707672275 -122.06674880975939) bank171651 +171652 POINT(37.09046302872866 -123.00406350077743) bank171652 +171653 POINT(37.02777961437448 -122.89028230230414) bank171653 +171654 POINT(37.3154723751486 -121.99838533647868) bank171654 +171655 POINT(37.67467361123649 -122.00761647963675) bank171655 +171656 POINT(37.59758262552542 -122.19821912854295) bank171656 +171657 POINT(37.14373002101324 -122.26624271400112) bank171657 +171658 POINT(37.7527577222857 -122.24314944042986) bank171658 +171659 POINT(37.89594487864162 -123.14301839602899) bank171659 +171660 POINT(38.76009900595693 -122.5017166341004) bank171660 +171661 POINT(37.31944290888251 -121.77010828520687) bank171661 +171662 POINT(37.01477204752504 -122.03881360425494) bank171662 +171663 POINT(38.56928937174066 -122.05289062392511) bank171663 +171664 POINT(37.86736612096297 -122.40335863807636) bank171664 +171665 POINT(36.89870861758064 -121.52647110391568) bank171665 +171666 POINT(37.01790841490251 -122.75675886953867) bank171666 +171667 POINT(37.26229137370495 -123.1821525317668) bank171667 +171668 POINT(37.19733853555832 -122.34433655056277) bank171668 +171669 POINT(38.40545528538737 -122.21209062730073) bank171669 +171670 POINT(37.590522589808685 -121.46350053272151) bank171670 +171671 POINT(37.66702373622131 -121.58338138571224) bank171671 +171672 POINT(37.69746477779339 -123.00980296864871) bank171672 +171673 POINT(37.41969024208375 -121.6180121582718) bank171673 +171674 POINT(38.479478549377546 -123.27262024553096) bank171674 +171675 POINT(38.354567634667966 -123.24222038242237) bank171675 +171676 POINT(37.071851377165004 -123.39756463510535) bank171676 +171677 POINT(38.07402557701572 -121.80490341851848) bank171677 +171678 POINT(36.98526373475669 -122.974519041281) bank171678 +171679 POINT(37.43208162801241 -122.85454048724193) bank171679 +171680 POINT(38.54035533065144 -121.71176200575725) bank171680 +171681 POINT(37.498217177875745 -123.37388085574473) bank171681 +171682 POINT(38.46457329553383 -122.42083360300747) bank171682 +171683 POINT(37.00772309108124 -122.306407099991) bank171683 +171684 POINT(38.10738966082486 -121.61657219950627) bank171684 +171685 POINT(37.28856017643505 -121.76424579075454) bank171685 +171686 POINT(37.53947926868435 -122.16913046545324) bank171686 +171687 POINT(37.7438025326678 -122.80167630437553) bank171687 +171688 POINT(36.80096142314175 -122.62236161932577) bank171688 +171689 POINT(37.52258136860292 -122.54060306044134) bank171689 +171690 POINT(37.31615734388866 -122.5284757323408) bank171690 +171691 POINT(37.70981993235961 -122.82239868238766) bank171691 +171692 POINT(37.546660812439356 -123.34297726666779) bank171692 +171693 POINT(38.48250165370248 -121.59820400982044) bank171693 +171694 POINT(37.357694672416386 -123.22216065194584) bank171694 +171695 POINT(36.79106970100719 -122.88104194235447) bank171695 +171696 POINT(38.768823219769565 -122.35019567812769) bank171696 +171697 POINT(37.43716684375863 -123.15925999855398) bank171697 +171698 POINT(38.260532140272026 -123.30724751587957) bank171698 +171699 POINT(37.03386777958282 -122.08845921224726) bank171699 +171700 POINT(36.99883986588643 -121.90906419859358) bank171700 +171701 POINT(38.42202074821119 -122.68904302709058) bank171701 +171702 POINT(37.61793386025059 -122.36750333067755) bank171702 +171703 POINT(38.19934501723477 -122.72198598564002) bank171703 +171704 POINT(38.37109857092918 -122.02938765074074) bank171704 +171705 POINT(38.41225824703095 -122.56011611145922) bank171705 +171706 POINT(37.80799083227507 -123.37747937771253) bank171706 +171707 POINT(38.48669088707665 -122.83787242058065) bank171707 +171708 POINT(38.10738120907694 -122.89592083376601) bank171708 +171709 POINT(37.9649762227106 -122.8600458948803) bank171709 +171710 POINT(37.80497025661834 -121.98920366472815) bank171710 +171711 POINT(37.471610597872974 -122.37044475408749) bank171711 +171712 POINT(38.32226229308626 -122.61716392612975) bank171712 +171713 POINT(38.552837471826166 -121.51352212243195) bank171713 +171714 POINT(37.835590063663986 -121.64796779232434) bank171714 +171715 POINT(37.28504104221645 -122.86904565675285) bank171715 +171716 POINT(36.84692497067187 -123.31334518400801) bank171716 +171717 POINT(37.89569777014905 -123.30913585669668) bank171717 +171718 POINT(37.25295902860555 -122.30094291291272) bank171718 +171719 POINT(37.53418334836645 -122.50426681691022) bank171719 +171720 POINT(38.75422639494795 -122.42389130347692) bank171720 +171721 POINT(38.30354301595739 -123.070182953067) bank171721 +171722 POINT(38.59976790752499 -121.71187224226574) bank171722 +171723 POINT(37.03851978438359 -121.90751611809607) bank171723 +171724 POINT(37.910000412545685 -121.62731581689516) bank171724 +171725 POINT(36.93591229364145 -121.78417326253378) bank171725 +171726 POINT(37.33626977862594 -121.6624354012242) bank171726 +171727 POINT(38.726536576338255 -121.8444064165472) bank171727 +171728 POINT(38.62022051650186 -122.69051848166853) bank171728 +171729 POINT(37.77650765295071 -122.3020431975664) bank171729 +171730 POINT(37.263584999357754 -122.52496048481804) bank171730 +171731 POINT(36.782844859284566 -122.65563346041651) bank171731 +171732 POINT(36.95078124544599 -122.45935764720295) bank171732 +171733 POINT(38.27146755264967 -122.07153837147538) bank171733 +171734 POINT(38.73064930034041 -121.86005230065902) bank171734 +171735 POINT(37.11918211201537 -122.87939792843612) bank171735 +171736 POINT(38.38495950632384 -123.09151061640883) bank171736 +171737 POINT(37.51599277531103 -123.27974703196428) bank171737 +171738 POINT(37.33733103205403 -122.83408844823) bank171738 +171739 POINT(37.575203743591565 -121.60096677210286) bank171739 +171740 POINT(38.669803139747124 -123.1643518303849) bank171740 +171741 POINT(37.72183497485065 -122.0472983528378) bank171741 +171742 POINT(36.78819403840919 -121.95931824981417) bank171742 +171743 POINT(38.386434676938364 -122.12642936189451) bank171743 +171744 POINT(38.412861635596094 -122.60568068721417) bank171744 +171745 POINT(38.20733434256103 -123.18115975884406) bank171745 +171746 POINT(37.02928327704809 -123.13133123809486) bank171746 +171747 POINT(37.000474275703404 -121.6366224092934) bank171747 +171748 POINT(37.85168631406295 -122.44089751465697) bank171748 +171749 POINT(37.81474186024054 -122.98773151691542) bank171749 +171750 POINT(37.905053979876584 -123.35534790210932) bank171750 +171751 POINT(36.86937520470479 -121.59514070230291) bank171751 +171752 POINT(36.99789057761064 -122.01750799058406) bank171752 +171753 POINT(37.75259160091396 -121.9200668161288) bank171753 +171754 POINT(38.30743276001937 -122.00527593422288) bank171754 +171755 POINT(38.09713193174968 -122.09971050310958) bank171755 +171756 POINT(38.40707476152071 -123.35360236605429) bank171756 +171757 POINT(37.38087843822603 -123.28220389118533) bank171757 +171758 POINT(38.34072589680474 -121.99009201544384) bank171758 +171759 POINT(38.098482459320586 -123.2419772909305) bank171759 +171760 POINT(37.367589275366406 -122.69693029327736) bank171760 +171761 POINT(37.92665071323197 -121.85746713476311) bank171761 +171762 POINT(37.94751146551532 -121.76645120539803) bank171762 +171763 POINT(37.55730426007428 -121.75559194252665) bank171763 +171764 POINT(38.15712691197458 -121.6856065440729) bank171764 +171765 POINT(38.541520427233465 -121.62654619307789) bank171765 +171766 POINT(37.34930477134458 -121.74732979955496) bank171766 +171767 POINT(37.92233241741111 -123.2330757410363) bank171767 +171768 POINT(36.93430252252663 -123.16325948870725) bank171768 +171769 POINT(37.90722657512319 -122.4599076888934) bank171769 +171770 POINT(38.19602519962175 -121.42453923576717) bank171770 +171771 POINT(38.58702902578054 -121.86693989872134) bank171771 +171772 POINT(38.71966117516236 -122.88855491221267) bank171772 +171773 POINT(37.54423379290351 -122.702438650184) bank171773 +171774 POINT(38.06885758292957 -123.26260189413435) bank171774 +171775 POINT(38.317490015475144 -123.22366251649575) bank171775 +171776 POINT(36.93646423927445 -123.06720391470334) bank171776 +171777 POINT(37.54000365706853 -122.43368395922805) bank171777 +171778 POINT(38.512655442923766 -122.24899743749299) bank171778 +171779 POINT(37.63362848699308 -122.98962826827034) bank171779 +171780 POINT(38.00633230322735 -121.86007876504645) bank171780 +171781 POINT(37.8955785488118 -122.13903389424442) bank171781 +171782 POINT(37.78247577851743 -122.27057029969815) bank171782 +171783 POINT(38.52279581799402 -121.51013607685859) bank171783 +171784 POINT(38.533494710509814 -123.06705247356233) bank171784 +171785 POINT(37.713386761735364 -121.82659204747743) bank171785 +171786 POINT(37.517111117525104 -122.81558112754561) bank171786 +171787 POINT(38.44038156713911 -123.26483663504455) bank171787 +171788 POINT(37.41859184458141 -123.11324376797593) bank171788 +171789 POINT(38.339188550494804 -121.94261601840684) bank171789 +171790 POINT(37.420994628788485 -123.10999118591323) bank171790 +171791 POINT(37.78585316499875 -123.18651713071522) bank171791 +171792 POINT(38.39710198931744 -122.45039782508863) bank171792 +171793 POINT(38.61835720537087 -122.56224641618604) bank171793 +171794 POINT(38.31396784596013 -122.21827182322725) bank171794 +171795 POINT(36.85424931499757 -121.70882632317193) bank171795 +171796 POINT(37.133586465688 -123.07714133577413) bank171796 +171797 POINT(38.64868823584685 -122.99160824858686) bank171797 +171798 POINT(38.52054054463201 -122.74667473795353) bank171798 +171799 POINT(37.439093515738755 -122.19360505700494) bank171799 +171800 POINT(38.2834522639215 -121.86650301569625) bank171800 +171801 POINT(36.864500562185775 -122.31876754107729) bank171801 +171802 POINT(38.47479358340067 -121.78489134865472) bank171802 +171803 POINT(36.88779569598272 -122.0256573170225) bank171803 +171804 POINT(37.554009613240545 -122.36848755549835) bank171804 +171805 POINT(37.37191139871389 -122.86150701610987) bank171805 +171806 POINT(37.90451513680233 -122.19713798342066) bank171806 +171807 POINT(37.82358371639951 -121.7158449070112) bank171807 +171808 POINT(36.789828492788544 -122.65218343620533) bank171808 +171809 POINT(37.17304109502917 -122.6284543047765) bank171809 +171810 POINT(37.126098798401465 -121.42435745326611) bank171810 +171811 POINT(37.51010069233085 -121.66689649005139) bank171811 +171812 POINT(37.35607686806243 -121.89794365606589) bank171812 +171813 POINT(37.136474566673996 -121.58574866081466) bank171813 +171814 POINT(38.546535176633334 -122.97753256370432) bank171814 +171815 POINT(38.53215724060786 -122.70054693042474) bank171815 +171816 POINT(38.08221923711711 -123.40459536248092) bank171816 +171817 POINT(38.53437314961252 -122.38190968476125) bank171817 +171818 POINT(37.464088888991384 -123.11188986077566) bank171818 +171819 POINT(38.451499613070474 -121.52512016587981) bank171819 +171820 POINT(36.920106713282614 -122.91153172462457) bank171820 +171821 POINT(38.24749933123959 -122.62801515021704) bank171821 +171822 POINT(37.06778666975636 -121.55341348535909) bank171822 +171823 POINT(37.27032816291004 -122.72602071013652) bank171823 +171824 POINT(38.43884704036429 -122.84562707515676) bank171824 +171825 POINT(38.46167838279082 -122.44587178416005) bank171825 +171826 POINT(38.69745989323091 -122.16638661147708) bank171826 +171827 POINT(37.91521376125601 -122.76969741220952) bank171827 +171828 POINT(37.86168121472297 -122.3035602550601) bank171828 +171829 POINT(37.70705503774876 -122.23502635218735) bank171829 +171830 POINT(37.22232790445227 -123.01355747313755) bank171830 +171831 POINT(37.961752637117065 -121.93405964376787) bank171831 +171832 POINT(38.55530034811173 -122.4078755648318) bank171832 +171833 POINT(38.61995027874534 -122.03271930725766) bank171833 +171834 POINT(38.109301259147706 -122.52699317179199) bank171834 +171835 POINT(38.67178877251501 -122.0078249885489) bank171835 +171836 POINT(38.694510490174935 -121.87423206322367) bank171836 +171837 POINT(38.59254580999206 -122.80840316387794) bank171837 +171838 POINT(38.442048952295764 -121.8692033456836) bank171838 +171839 POINT(36.937929747081434 -121.70946876378818) bank171839 +171840 POINT(38.74308263226723 -122.45107080415828) bank171840 +171841 POINT(37.97054399087903 -121.58369711285515) bank171841 +171842 POINT(37.33312204222432 -122.17049135114335) bank171842 +171843 POINT(38.64752555342383 -121.77059442042946) bank171843 +171844 POINT(38.112251097570855 -122.45823682965727) bank171844 +171845 POINT(37.50929212801427 -122.41070741131271) bank171845 +171846 POINT(38.12923548268708 -122.20901365434607) bank171846 +171847 POINT(38.336646704467796 -121.58615338630139) bank171847 +171848 POINT(38.30652009982834 -122.03917592242873) bank171848 +171849 POINT(37.18828577514025 -122.51046475716657) bank171849 +171850 POINT(37.04133086339795 -122.61068297391495) bank171850 +171851 POINT(36.99452499250839 -123.2998598064191) bank171851 +171852 POINT(37.88378754140495 -122.92061631625354) bank171852 +171853 POINT(37.62950506370765 -122.07323113975282) bank171853 +171854 POINT(37.03087347233758 -122.3865962778017) bank171854 +171855 POINT(36.934469286416366 -121.76633095439176) bank171855 +171856 POINT(38.33637273592357 -122.60012450516533) bank171856 +171857 POINT(38.22943355520909 -122.04262376105352) bank171857 +171858 POINT(36.78027630271069 -121.5767576480179) bank171858 +171859 POINT(37.64887825091036 -123.3390724951559) bank171859 +171860 POINT(36.83684889998397 -122.64828209458221) bank171860 +171861 POINT(38.24494166291277 -121.97284676529459) bank171861 +171862 POINT(37.798297107132704 -122.88164487030664) bank171862 +171863 POINT(37.40714878155473 -121.65722494803164) bank171863 +171864 POINT(38.0700006070236 -122.69115144586569) bank171864 +171865 POINT(37.51402222571155 -122.26606536635666) bank171865 +171866 POINT(36.976970009644006 -122.98759427544083) bank171866 +171867 POINT(38.7699875403372 -122.240182792935) bank171867 +171868 POINT(36.80769838946109 -121.4907735300054) bank171868 +171869 POINT(37.47005791327563 -122.27909391640029) bank171869 +171870 POINT(38.09667848514906 -122.94366084599831) bank171870 +171871 POINT(37.022148584648164 -122.73688526603647) bank171871 +171872 POINT(37.92172773922809 -122.1162543186172) bank171872 +171873 POINT(38.03073537194571 -121.74515067033553) bank171873 +171874 POINT(38.37098855154082 -122.28320591034313) bank171874 +171875 POINT(37.509018685050144 -121.77263605305906) bank171875 +171876 POINT(37.98985235414867 -123.37164110393536) bank171876 +171877 POINT(37.19311575336072 -122.72854622416816) bank171877 +171878 POINT(36.96325206907203 -122.23242958665963) bank171878 +171879 POINT(38.33316599065435 -122.22158699116181) bank171879 +171880 POINT(37.73979602828315 -122.38061721547841) bank171880 +171881 POINT(38.49503100676627 -123.28966505342325) bank171881 +171882 POINT(38.151107243121146 -122.871863505168) bank171882 +171883 POINT(38.61179920836139 -121.48536901052832) bank171883 +171884 POINT(37.084650319267865 -121.84604007155153) bank171884 +171885 POINT(37.950479302290674 -121.8473652913772) bank171885 +171886 POINT(37.077689610076646 -121.43043200886645) bank171886 +171887 POINT(38.49554349253309 -122.01088829964621) bank171887 +171888 POINT(37.03900875975031 -122.94942503698657) bank171888 +171889 POINT(38.752795113275425 -122.87534361467928) bank171889 +171890 POINT(38.186888455821546 -122.50332018763903) bank171890 +171891 POINT(38.186415927334394 -122.24940480435198) bank171891 +171892 POINT(38.288183854124945 -122.85839884572269) bank171892 +171893 POINT(36.83408338668658 -123.09813738325639) bank171893 +171894 POINT(38.092717239802035 -121.71195367968323) bank171894 +171895 POINT(37.453538283610264 -122.1805097648129) bank171895 +171896 POINT(38.659626174661206 -121.97249768109964) bank171896 +171897 POINT(37.677303012292974 -123.17999066711332) bank171897 +171898 POINT(37.33203649046561 -121.70064854277373) bank171898 +171899 POINT(37.92478150707811 -121.81031438379154) bank171899 +171900 POINT(38.3923943491483 -122.42390952222009) bank171900 +171901 POINT(36.83611954912852 -122.9259549418735) bank171901 +171902 POINT(38.441600545412044 -122.63697897974059) bank171902 +171903 POINT(38.440437054705576 -122.19536231662968) bank171903 +171904 POINT(37.083553376729554 -123.21783051221816) bank171904 +171905 POINT(37.39378712139157 -123.1994534214388) bank171905 +171906 POINT(38.771564127322655 -121.95229985277331) bank171906 +171907 POINT(37.25869403377557 -121.53425782228955) bank171907 +171908 POINT(36.86849900964716 -122.99347316989282) bank171908 +171909 POINT(37.446558572013785 -123.37038041063262) bank171909 +171910 POINT(37.206412119210334 -121.66328149706658) bank171910 +171911 POINT(37.992161526066134 -122.49529601240111) bank171911 +171912 POINT(38.74719265542474 -123.1704564951324) bank171912 +171913 POINT(37.64360054765085 -121.42381514195498) bank171913 +171914 POINT(38.15476290511835 -122.65728815359701) bank171914 +171915 POINT(37.08302001275734 -122.75782296594542) bank171915 +171916 POINT(37.637980645583255 -122.36170257982845) bank171916 +171917 POINT(38.420926170701144 -123.03815434258345) bank171917 +171918 POINT(36.84087319284623 -122.28695725880156) bank171918 +171919 POINT(37.32623432419938 -123.40472004973907) bank171919 +171920 POINT(37.514827650792334 -121.7947635033437) bank171920 +171921 POINT(37.38688582619247 -121.52238130618856) bank171921 +171922 POINT(38.71612930151079 -121.54369818881196) bank171922 +171923 POINT(37.1209552072711 -122.65585453862596) bank171923 +171924 POINT(37.63400662155149 -121.83454014572209) bank171924 +171925 POINT(38.212885586256604 -121.80640439514536) bank171925 +171926 POINT(37.07471385790802 -121.87682678378869) bank171926 +171927 POINT(37.607423685129596 -122.36881925240289) bank171927 +171928 POINT(37.154782232835295 -123.40276118728505) bank171928 +171929 POINT(37.9074651471367 -123.14956689462937) bank171929 +171930 POINT(37.684091047325296 -122.36289921524491) bank171930 +171931 POINT(37.25341190604148 -123.41289342816656) bank171931 +171932 POINT(37.60370201016088 -123.06967618817252) bank171932 +171933 POINT(37.51200268765189 -122.43813817679057) bank171933 +171934 POINT(37.696205310148834 -122.92070811079108) bank171934 +171935 POINT(37.60301420470768 -122.59050445040661) bank171935 +171936 POINT(38.05678630854895 -121.4822628227383) bank171936 +171937 POINT(38.444682333087336 -121.51970825489305) bank171937 +171938 POINT(37.68752976990388 -122.29017844812972) bank171938 +171939 POINT(37.23955218462767 -121.81001213890022) bank171939 +171940 POINT(38.29498162679971 -123.20264789891) bank171940 +171941 POINT(36.87686572035983 -122.14800090312917) bank171941 +171942 POINT(38.31208588927618 -122.17784371397534) bank171942 +171943 POINT(37.736646381550635 -122.43679564176705) bank171943 +171944 POINT(37.25655990426795 -122.99661301695699) bank171944 +171945 POINT(38.75891872314858 -121.92154815345563) bank171945 +171946 POINT(37.06968650615713 -122.93471264191298) bank171946 +171947 POINT(36.802533181083625 -122.27440066161695) bank171947 +171948 POINT(37.85675687211733 -122.59177595589064) bank171948 +171949 POINT(38.327425140258555 -123.22217651436875) bank171949 +171950 POINT(37.8336001225879 -123.32964565615046) bank171950 +171951 POINT(36.824763129855434 -122.82835364637583) bank171951 +171952 POINT(36.85455271116569 -121.90972524879362) bank171952 +171953 POINT(38.13316038499722 -123.02735574939243) bank171953 +171954 POINT(38.05857437413561 -122.48424965666703) bank171954 +171955 POINT(38.270420026511346 -123.34971939205776) bank171955 +171956 POINT(37.87897776890142 -123.2039271565078) bank171956 +171957 POINT(38.60999833768327 -122.11531083667799) bank171957 +171958 POINT(36.81374109261408 -122.76608872258083) bank171958 +171959 POINT(37.45579700329775 -123.05187940762102) bank171959 +171960 POINT(38.12717718752231 -122.91327731111166) bank171960 +171961 POINT(37.89914294458318 -122.54011525443731) bank171961 +171962 POINT(38.70847034844849 -123.26308910609501) bank171962 +171963 POINT(38.09599066379843 -123.31491675672781) bank171963 +171964 POINT(37.534579843187764 -122.47362475276068) bank171964 +171965 POINT(37.20413782820463 -121.4975169843362) bank171965 +171966 POINT(37.503813550854346 -122.84421817128674) bank171966 +171967 POINT(37.94218295485186 -123.20024343441034) bank171967 +171968 POINT(37.679752549098055 -122.0415006386336) bank171968 +171969 POINT(36.97255619958144 -122.94708191952432) bank171969 +171970 POINT(37.699819641782476 -122.54594831695469) bank171970 +171971 POINT(37.66104216614223 -122.16820046452725) bank171971 +171972 POINT(37.21829651639672 -122.3702196277419) bank171972 +171973 POINT(37.054960489746506 -122.15236145741328) bank171973 +171974 POINT(36.976871097778634 -121.95856341205737) bank171974 +171975 POINT(37.921849795422624 -122.11220093422456) bank171975 +171976 POINT(37.39933880774336 -121.65043026504799) bank171976 +171977 POINT(37.49894663023389 -121.84732580481136) bank171977 +171978 POINT(37.90403606379602 -122.67807579979183) bank171978 +171979 POINT(36.831955344499775 -122.88445996413778) bank171979 +171980 POINT(37.94433251303762 -122.77757946276228) bank171980 +171981 POINT(37.493876341349285 -121.99222215820099) bank171981 +171982 POINT(38.54778944577985 -122.0380141379796) bank171982 +171983 POINT(38.378153757527784 -122.66010492357789) bank171983 +171984 POINT(37.94217522394426 -122.20121594485256) bank171984 +171985 POINT(37.90194419255865 -123.24433482256379) bank171985 +171986 POINT(37.97951645340155 -121.92289231427347) bank171986 +171987 POINT(37.14198888148395 -121.90649103710508) bank171987 +171988 POINT(38.641909086897414 -122.13374746350573) bank171988 +171989 POINT(38.61236472256798 -121.62485894532527) bank171989 +171990 POINT(37.241378315279455 -122.76461782252933) bank171990 +171991 POINT(38.67997571366429 -122.48287744764994) bank171991 +171992 POINT(37.70766043974796 -123.10913355198123) bank171992 +171993 POINT(37.47392580166854 -122.34872667787977) bank171993 +171994 POINT(37.795716429242624 -122.59124468259134) bank171994 +171995 POINT(37.57033499380522 -122.84797501587195) bank171995 +171996 POINT(38.08004072340809 -122.46337441728717) bank171996 +171997 POINT(38.460592594009825 -123.01805968566119) bank171997 +171998 POINT(38.559223246785784 -121.76172957269651) bank171998 +171999 POINT(37.34903347304053 -121.43050615110512) bank171999 +172000 POINT(37.71016965569803 -122.1456490067454) bank172000 +172001 POINT(38.46009908277369 -123.17273615210549) bank172001 +172002 POINT(38.72017433529622 -122.16610419346885) bank172002 +172003 POINT(38.31018426758324 -121.58935811714741) bank172003 +172004 POINT(37.39870248821749 -122.20783857093613) bank172004 +172005 POINT(37.457491588259415 -121.9841425258187) bank172005 +172006 POINT(37.98671842065401 -121.56585780428922) bank172006 +172007 POINT(36.95896725488815 -123.40354717353249) bank172007 +172008 POINT(36.88923596661752 -121.89049358292034) bank172008 +172009 POINT(36.812774860480566 -121.63377412092565) bank172009 +172010 POINT(37.7910061169194 -121.91194147028142) bank172010 +172011 POINT(37.2859680031878 -122.1520869148523) bank172011 +172012 POINT(37.312950752651 -121.67136218124661) bank172012 +172013 POINT(38.52516302760588 -122.36338713239661) bank172013 +172014 POINT(38.40663560936492 -121.58174190740472) bank172014 +172015 POINT(38.75357380285991 -122.4476258953624) bank172015 +172016 POINT(37.17001754860941 -122.98064619577592) bank172016 +172017 POINT(37.385143848116414 -123.08669280450836) bank172017 +172018 POINT(37.165286879925766 -122.08181064908094) bank172018 +172019 POINT(38.476164564705066 -121.80186678124022) bank172019 +172020 POINT(38.50935798383594 -122.20048701227581) bank172020 +172021 POINT(37.033163993873956 -123.1804851977204) bank172021 +172022 POINT(38.15740064466603 -122.32447876747541) bank172022 +172023 POINT(38.347706778472734 -122.77430000811742) bank172023 +172024 POINT(37.71760872806832 -122.43834922255539) bank172024 +172025 POINT(37.200564205039285 -122.61813795259825) bank172025 +172026 POINT(38.542244098933736 -122.44786317358854) bank172026 +172027 POINT(37.68372732396689 -122.95233757045241) bank172027 +172028 POINT(37.56628084554338 -121.63934645888335) bank172028 +172029 POINT(36.98930905677339 -122.04070491333404) bank172029 +172030 POINT(37.13285569705147 -123.23859660719697) bank172030 +172031 POINT(36.7776538048422 -122.23719306282302) bank172031 +172032 POINT(37.14791348376234 -123.04390424253303) bank172032 +172033 POINT(37.23708914986048 -122.24797302921891) bank172033 +172034 POINT(36.95189422157263 -121.87939942253058) bank172034 +172035 POINT(37.325626940731674 -121.9055202167971) bank172035 +172036 POINT(37.62926429260965 -123.21293274881964) bank172036 +172037 POINT(38.75245222975929 -123.37622522531044) bank172037 +172038 POINT(37.52761751812971 -122.90434426025779) bank172038 +172039 POINT(36.8715438417287 -123.28569869422813) bank172039 +172040 POINT(38.44082348073682 -121.91033146882164) bank172040 +172041 POINT(37.91609893283861 -122.41527763252704) bank172041 +172042 POINT(37.95355472952676 -122.83799204174602) bank172042 +172043 POINT(37.50470378850894 -122.30734251768966) bank172043 +172044 POINT(37.93121104265325 -123.22660349225674) bank172044 +172045 POINT(37.94713261571948 -121.89306529215315) bank172045 +172046 POINT(37.90356187723007 -121.4232289867586) bank172046 +172047 POINT(38.62600892836565 -123.36849525872103) bank172047 +172048 POINT(37.955475698298024 -121.55047687404439) bank172048 +172049 POINT(38.426308575964015 -121.59903543819962) bank172049 +172050 POINT(38.31941989781215 -121.71256237786145) bank172050 +172051 POINT(37.6334409660733 -122.82096476115451) bank172051 +172052 POINT(37.541226428972095 -122.47077145500168) bank172052 +172053 POINT(37.06707854902429 -123.12388821237799) bank172053 +172054 POINT(36.98500404542963 -123.31063178705323) bank172054 +172055 POINT(36.839173007993885 -121.74834466591604) bank172055 +172056 POINT(38.64571071073003 -123.40974605897765) bank172056 +172057 POINT(38.013295622751535 -121.49253167082621) bank172057 +172058 POINT(37.452120981341785 -123.13579222574603) bank172058 +172059 POINT(38.149041423188116 -122.60925589645773) bank172059 +172060 POINT(38.76990155706572 -122.87133219497541) bank172060 +172061 POINT(37.757764524376086 -123.40241656185323) bank172061 +172062 POINT(38.261319851951825 -121.74565459057654) bank172062 +172063 POINT(38.37361376735485 -121.64102365227419) bank172063 +172064 POINT(37.12766089437184 -122.92605184422322) bank172064 +172065 POINT(37.774531899727386 -121.8701541404099) bank172065 +172066 POINT(37.97096349420838 -121.45347147193998) bank172066 +172067 POINT(37.955352505382706 -122.84147775610033) bank172067 +172068 POINT(36.899138126458126 -122.61695674551594) bank172068 +172069 POINT(37.21217049532439 -121.92040544123469) bank172069 +172070 POINT(37.26250023821576 -122.82319581693268) bank172070 +172071 POINT(37.47883077149436 -123.23145149621709) bank172071 +172072 POINT(37.01445849127435 -122.4903575689228) bank172072 +172073 POINT(37.842366892881756 -123.32890756669195) bank172073 +172074 POINT(38.64853911234114 -121.70595096020448) bank172074 +172075 POINT(37.30445780201918 -122.26290347184208) bank172075 +172076 POINT(38.324726858925324 -122.5231693883756) bank172076 +172077 POINT(37.17325780015863 -121.6130785464599) bank172077 +172078 POINT(37.77560984554922 -121.81290282811844) bank172078 +172079 POINT(38.057327119607976 -122.0462107651257) bank172079 +172080 POINT(38.58502433678027 -123.02346111034602) bank172080 +172081 POINT(37.33811547546388 -122.31434927902674) bank172081 +172082 POINT(38.01297304270692 -122.85140061581305) bank172082 +172083 POINT(36.918719705254304 -123.08719490319457) bank172083 +172084 POINT(38.0972448668825 -122.11877011290503) bank172084 +172085 POINT(36.813998393482365 -121.60949802008484) bank172085 +172086 POINT(36.79567491995622 -122.82833628286681) bank172086 +172087 POINT(38.73743720028326 -122.09663665471146) bank172087 +172088 POINT(38.066695935579894 -122.53101588223655) bank172088 +172089 POINT(37.53098433611471 -122.60265212276174) bank172089 +172090 POINT(37.53768837849429 -122.27639976006026) bank172090 +172091 POINT(37.32162065265834 -121.60532933040567) bank172091 +172092 POINT(37.725266826221535 -122.48134141234672) bank172092 +172093 POINT(38.38416384298994 -123.0413531804043) bank172093 +172094 POINT(37.138129426945525 -123.23512018352943) bank172094 +172095 POINT(37.691704443676365 -122.04156561038694) bank172095 +172096 POINT(36.85282695162505 -123.29710631504986) bank172096 +172097 POINT(38.034419184744806 -122.80159362173197) bank172097 +172098 POINT(38.33316785531671 -123.3495350469898) bank172098 +172099 POINT(38.043578062290116 -121.86350492341671) bank172099 +172100 POINT(37.02785231050911 -121.68219974525478) bank172100 +172101 POINT(37.30479114135936 -122.73636834603397) bank172101 +172102 POINT(38.253087339610154 -122.58749051206196) bank172102 +172103 POINT(37.31163534357678 -122.81994182218058) bank172103 +172104 POINT(37.13080144814213 -121.69330773827076) bank172104 +172105 POINT(37.57005480979004 -121.70961112169532) bank172105 +172106 POINT(37.66897207301957 -122.02856014831664) bank172106 +172107 POINT(36.84527003671782 -122.1740222310605) bank172107 +172108 POINT(37.42448733882684 -122.32965637312266) bank172108 +172109 POINT(37.235624774780035 -121.9411335713779) bank172109 +172110 POINT(38.098793266516076 -121.6902887788783) bank172110 +172111 POINT(37.97704946710805 -123.3393269498543) bank172111 +172112 POINT(37.2640737565458 -122.82870542357702) bank172112 +172113 POINT(37.068976614191 -121.67017582462395) bank172113 +172114 POINT(37.533921434198895 -122.41371368656367) bank172114 +172115 POINT(37.97025101300363 -121.63059629021508) bank172115 +172116 POINT(38.73084743642496 -123.34735889859128) bank172116 +172117 POINT(36.96190972907431 -122.03908599049421) bank172117 +172118 POINT(37.410649822955875 -121.90896029444642) bank172118 +172119 POINT(38.15661562935467 -122.521490888575) bank172119 +172120 POINT(38.64654406992312 -121.5480852094476) bank172120 +172121 POINT(38.6316780224287 -122.92926865064697) bank172121 +172122 POINT(37.971990282167745 -121.98405298678985) bank172122 +172123 POINT(38.03128329358946 -122.40349055554005) bank172123 +172124 POINT(38.08443748931517 -122.55208345625313) bank172124 +172125 POINT(38.46714651612964 -123.34546709467573) bank172125 +172126 POINT(38.62961310886503 -122.55871109294901) bank172126 +172127 POINT(37.17182137104152 -122.89548778492862) bank172127 +172128 POINT(38.353016775493025 -122.82602966960808) bank172128 +172129 POINT(38.337877029111695 -121.5724955674761) bank172129 +172130 POINT(37.22601152158247 -122.23661544999912) bank172130 +172131 POINT(38.16604213223802 -122.28403077365257) bank172131 +172132 POINT(38.461699796242335 -122.45378784333315) bank172132 +172133 POINT(38.5169969621938 -122.11368590625585) bank172133 +172134 POINT(38.72955069043711 -122.81193805878729) bank172134 +172135 POINT(36.868303368513786 -122.702519492703) bank172135 +172136 POINT(38.13550257614256 -123.05926053870331) bank172136 +172137 POINT(37.421636575986284 -122.80927246369752) bank172137 +172138 POINT(38.5904680210771 -121.74379732665494) bank172138 +172139 POINT(38.76189651402069 -122.36888804065087) bank172139 +172140 POINT(37.83833975028676 -122.70388314759805) bank172140 +172141 POINT(38.27860498511133 -122.59886729635288) bank172141 +172142 POINT(38.40481228335759 -123.06411512613045) bank172142 +172143 POINT(37.87546998326025 -122.98410752503483) bank172143 +172144 POINT(38.63970254494187 -121.94302189698692) bank172144 +172145 POINT(36.851715030733 -123.00739420786202) bank172145 +172146 POINT(37.59397594143594 -122.74260333966289) bank172146 +172147 POINT(37.423372220464664 -122.90799902580927) bank172147 +172148 POINT(38.554789540389095 -123.21945578970376) bank172148 +172149 POINT(38.72893377997142 -122.05946140135788) bank172149 +172150 POINT(37.642160314389876 -121.96092800856451) bank172150 +172151 POINT(37.62229086621933 -123.10870692214476) bank172151 +172152 POINT(37.948321003514906 -122.23935282130964) bank172152 +172153 POINT(38.27521159873946 -122.4242299531406) bank172153 +172154 POINT(37.39952650403899 -121.628007009019) bank172154 +172155 POINT(38.558380997104315 -122.80053979507719) bank172155 +172156 POINT(38.70779060794218 -122.4069665332485) bank172156 +172157 POINT(38.14565377392587 -123.3824275615952) bank172157 +172158 POINT(37.95915785181147 -123.33877800144072) bank172158 +172159 POINT(37.26411645537983 -121.72402169191862) bank172159 +172160 POINT(37.11743664625464 -122.5224048436793) bank172160 +172161 POINT(37.26833597185891 -121.54108100617931) bank172161 +172162 POINT(37.03638980720558 -122.0110781127163) bank172162 +172163 POINT(37.71482154132846 -121.68374979672708) bank172163 +172164 POINT(37.287176411754956 -122.09946458198422) bank172164 +172165 POINT(37.34631316695807 -121.48476971352498) bank172165 +172166 POINT(37.8256169880955 -122.23439372800082) bank172166 +172167 POINT(38.515243722414276 -123.4039449479283) bank172167 +172168 POINT(37.53568338000904 -121.85678577977671) bank172168 +172169 POINT(37.484253694491336 -122.45198193026933) bank172169 +172170 POINT(37.303317625178416 -122.65571450887509) bank172170 +172171 POINT(38.448866548513934 -123.0942349464788) bank172171 +172172 POINT(38.35704870753516 -121.70470014718308) bank172172 +172173 POINT(38.326071660804494 -123.10417206002191) bank172173 +172174 POINT(37.66259705498613 -123.2525633567471) bank172174 +172175 POINT(37.50471071486169 -123.4062890480998) bank172175 +172176 POINT(38.40436593241786 -121.60679195839401) bank172176 +172177 POINT(38.0310958705758 -121.80870698473942) bank172177 +172178 POINT(38.65587945865402 -121.46705115066695) bank172178 +172179 POINT(38.302298338137575 -122.26859953960255) bank172179 +172180 POINT(37.94503683340763 -123.17314457732905) bank172180 +172181 POINT(38.11058715876166 -122.45077911442573) bank172181 +172182 POINT(36.902339511674036 -122.86023829258596) bank172182 +172183 POINT(38.308550274863876 -122.05780356578418) bank172183 +172184 POINT(38.27237746754505 -122.56845672280674) bank172184 +172185 POINT(37.49868152410335 -121.83607584682598) bank172185 +172186 POINT(37.824860549533284 -122.68868540760465) bank172186 +172187 POINT(37.48754873690492 -123.0032340408731) bank172187 +172188 POINT(37.467207553121305 -122.75087116166038) bank172188 +172189 POINT(38.00721935686143 -122.54028350266385) bank172189 +172190 POINT(38.31357869753302 -121.94166681261382) bank172190 +172191 POINT(37.54697987275484 -123.1724883111339) bank172191 +172192 POINT(37.90645521521034 -121.55337123512217) bank172192 +172193 POINT(38.52230347085137 -122.33291171299246) bank172193 +172194 POINT(38.71308661125651 -122.890436574082) bank172194 +172195 POINT(38.31368371143952 -121.51257454616085) bank172195 +172196 POINT(36.784988270192294 -121.88991641389511) bank172196 +172197 POINT(37.88069240027906 -123.38689561876664) bank172197 +172198 POINT(38.10759468535822 -121.61550330167462) bank172198 +172199 POINT(38.278618273066485 -123.20123612359266) bank172199 +172200 POINT(37.85290297493434 -121.84494183041801) bank172200 +172201 POINT(37.11862873161415 -122.5263654565176) bank172201 +172202 POINT(38.394938390173806 -122.70245192967654) bank172202 +172203 POINT(38.69262545725682 -123.3200874452541) bank172203 +172204 POINT(38.01806148062299 -122.51161981849314) bank172204 +172205 POINT(38.490788974185875 -122.73137304906787) bank172205 +172206 POINT(36.80717764080321 -122.61069743264665) bank172206 +172207 POINT(38.06304276432429 -123.042799567824) bank172207 +172208 POINT(38.760093231523285 -123.04577409474027) bank172208 +172209 POINT(37.51155592485304 -122.74126052263236) bank172209 +172210 POINT(37.45834968931303 -121.72755263578902) bank172210 +172211 POINT(37.748424543744036 -121.52695580238507) bank172211 +172212 POINT(37.1673045831951 -122.76387860191339) bank172212 +172213 POINT(38.77307742557512 -123.09554806581178) bank172213 +172214 POINT(38.21832953375909 -121.76057066870024) bank172214 +172215 POINT(37.4341050824151 -122.84903418094679) bank172215 +172216 POINT(38.01672740308592 -122.31476317872453) bank172216 +172217 POINT(37.79190380903086 -121.96268462713472) bank172217 +172218 POINT(37.19859394307597 -122.70534842142749) bank172218 +172219 POINT(38.084046340972165 -121.77402572117435) bank172219 +172220 POINT(37.11909458415174 -121.47405797770055) bank172220 +172221 POINT(38.38858708548461 -123.00732046849267) bank172221 +172222 POINT(37.09728917204693 -121.5536658909358) bank172222 +172223 POINT(37.86115262034492 -122.08527598716469) bank172223 +172224 POINT(37.459842332892585 -121.49316908991865) bank172224 +172225 POINT(37.65820288990802 -121.51035647548792) bank172225 +172226 POINT(37.40471603557402 -123.16693515133836) bank172226 +172227 POINT(37.78029286326559 -122.44645292461495) bank172227 +172228 POINT(36.893186587402916 -122.68685444142486) bank172228 +172229 POINT(36.98535281561274 -123.27471757480866) bank172229 +172230 POINT(36.79388454294459 -122.51277758266077) bank172230 +172231 POINT(38.62146295259454 -122.35104641593622) bank172231 +172232 POINT(38.50675003235677 -122.05867546541738) bank172232 +172233 POINT(38.698597173228 -122.18850851625403) bank172233 +172234 POINT(38.39850518594984 -122.65540159615097) bank172234 +172235 POINT(37.963662482451625 -122.87588206544072) bank172235 +172236 POINT(38.24773247956542 -123.08649212698556) bank172236 +172237 POINT(38.151233742192794 -121.56414568410362) bank172237 +172238 POINT(38.49189700022287 -122.75051583865209) bank172238 +172239 POINT(38.03565132585617 -123.09563943699008) bank172239 +172240 POINT(37.83145330230666 -122.59227675659707) bank172240 +172241 POINT(38.756538808104764 -121.89527114291506) bank172241 +172242 POINT(38.29162885824909 -121.52267105740508) bank172242 +172243 POINT(37.33143777456123 -121.47289336632547) bank172243 +172244 POINT(38.25273259710666 -122.99891309831827) bank172244 +172245 POINT(37.184540190623025 -122.06375419009191) bank172245 +172246 POINT(38.730317532038434 -123.17974371731313) bank172246 +172247 POINT(36.90476094611538 -121.52533150508036) bank172247 +172248 POINT(38.3731567660352 -122.85625982972302) bank172248 +172249 POINT(37.815375713217776 -121.49609857026626) bank172249 +172250 POINT(36.954923826801064 -122.71516998124964) bank172250 +172251 POINT(37.321787851745526 -121.51033999441414) bank172251 +172252 POINT(37.01066123106349 -122.23466978088105) bank172252 +172253 POINT(38.246403786777876 -121.69291555747016) bank172253 +172254 POINT(36.935150735277965 -123.28945071075698) bank172254 +172255 POINT(36.829017760991654 -123.12866582142014) bank172255 +172256 POINT(38.444575117211286 -123.09958982359056) bank172256 +172257 POINT(37.51495504057397 -122.58600863381534) bank172257 +172258 POINT(38.66055811632541 -122.49435064591331) bank172258 +172259 POINT(37.276482461068724 -121.86551752496915) bank172259 +172260 POINT(37.794206124142725 -122.17530487769092) bank172260 +172261 POINT(37.327150726920365 -123.29159074716001) bank172261 +172262 POINT(36.79758411702238 -122.67267053315601) bank172262 +172263 POINT(37.7621980409484 -123.27324407004738) bank172263 +172264 POINT(38.44356977153743 -122.57747103241215) bank172264 +172265 POINT(37.38734790579308 -122.09650432386826) bank172265 +172266 POINT(38.75074690948607 -122.54271274679137) bank172266 +172267 POINT(37.91521341739133 -122.51275106255517) bank172267 +172268 POINT(38.24915732109104 -122.02813493990577) bank172268 +172269 POINT(37.148202346365196 -122.65951563031189) bank172269 +172270 POINT(37.98114140686346 -122.06629945019934) bank172270 +172271 POINT(37.57778321538313 -123.12306952245808) bank172271 +172272 POINT(37.51471217855615 -123.24522880619995) bank172272 +172273 POINT(38.06531101475789 -122.10992678199116) bank172273 +172274 POINT(38.73011967706038 -123.08585862798613) bank172274 +172275 POINT(36.85397454513109 -123.20168819865324) bank172275 +172276 POINT(38.210140229166186 -122.83151333081577) bank172276 +172277 POINT(37.61817649073305 -121.5428929903439) bank172277 +172278 POINT(38.19226692824761 -121.83136258086057) bank172278 +172279 POINT(37.44993732272004 -121.61251614122277) bank172279 +172280 POINT(36.83959719668804 -122.21307894839566) bank172280 +172281 POINT(38.653865730170594 -123.11012891178407) bank172281 +172282 POINT(38.681300735725124 -121.67573617713553) bank172282 +172283 POINT(37.13149822619369 -121.48613816753394) bank172283 +172284 POINT(38.34309658048107 -121.58930543349433) bank172284 +172285 POINT(37.6110741800303 -123.31956448498651) bank172285 +172286 POINT(37.466405467410084 -121.5740224298266) bank172286 +172287 POINT(37.99272947963482 -121.98763362166987) bank172287 +172288 POINT(37.30848746523393 -122.53671623795151) bank172288 +172289 POINT(38.55846253638317 -122.26335687250356) bank172289 +172290 POINT(38.662325643473096 -122.73829113832157) bank172290 +172291 POINT(38.655427016183445 -123.06493210197512) bank172291 +172292 POINT(37.34072364102973 -122.52846414387122) bank172292 +172293 POINT(37.86878027889696 -122.54599578498843) bank172293 +172294 POINT(36.959793695933904 -122.41939998075735) bank172294 +172295 POINT(38.712449346318586 -123.2784929267929) bank172295 +172296 POINT(38.71957138232763 -122.45192598151952) bank172296 +172297 POINT(38.276631477256444 -121.84802020291664) bank172297 +172298 POINT(36.982618491096005 -121.82489238917493) bank172298 +172299 POINT(37.6363540453557 -122.8391563666458) bank172299 +172300 POINT(37.50525473208459 -123.1581146095658) bank172300 +172301 POINT(37.94673358329251 -122.15882406738008) bank172301 +172302 POINT(37.637631701014065 -121.57056589013513) bank172302 +172303 POINT(37.37761477230157 -122.40657222282819) bank172303 +172304 POINT(37.83917528467049 -121.87262276417167) bank172304 +172305 POINT(37.24328903467027 -121.96204019860967) bank172305 +172306 POINT(36.87558877036378 -121.89164230733523) bank172306 +172307 POINT(37.22462901788679 -122.66989879987347) bank172307 +172308 POINT(37.0447770949412 -122.20448819630674) bank172308 +172309 POINT(38.09712366116529 -122.3271417921059) bank172309 +172310 POINT(38.75321543898575 -122.89126502041158) bank172310 +172311 POINT(37.32428697319343 -121.66762879513296) bank172311 +172312 POINT(37.01743669878775 -121.69073285542318) bank172312 +172313 POINT(37.074436703072735 -122.7013975789281) bank172313 +172314 POINT(38.70009915054445 -123.21655554849728) bank172314 +172315 POINT(37.19365738161385 -121.7173950220314) bank172315 +172316 POINT(37.397112124900694 -121.93599135294323) bank172316 +172317 POINT(37.76353723360894 -122.90932351804565) bank172317 +172318 POINT(38.390124005188866 -122.12192217595278) bank172318 +172319 POINT(38.03004201472238 -122.93733519738954) bank172319 +172320 POINT(38.52609542394799 -122.21148209535603) bank172320 +172321 POINT(37.39665616073735 -122.65758126843455) bank172321 +172322 POINT(37.42178828409379 -122.86495406161171) bank172322 +172323 POINT(38.55120554144403 -122.07728215978383) bank172323 +172324 POINT(38.408553541488224 -122.79499973326632) bank172324 +172325 POINT(37.099383514705174 -123.26713358433649) bank172325 +172326 POINT(38.16813575308527 -122.28739606666099) bank172326 +172327 POINT(37.66449166127739 -123.04195998508627) bank172327 +172328 POINT(37.75341273433494 -121.64365943701948) bank172328 +172329 POINT(38.21813087896056 -121.45715271817932) bank172329 +172330 POINT(37.77067595790364 -121.50318620531814) bank172330 +172331 POINT(37.388023617321565 -121.5136147391811) bank172331 +172332 POINT(37.58029889004715 -123.30266979559785) bank172332 +172333 POINT(38.21671481846482 -121.99647057854122) bank172333 +172334 POINT(37.031332060001624 -122.17987849934521) bank172334 +172335 POINT(38.495817506040645 -122.08348221477428) bank172335 +172336 POINT(37.180831964803836 -122.50507812336778) bank172336 +172337 POINT(38.028078635753864 -123.26435663587452) bank172337 +172338 POINT(37.650518065811355 -122.31628460508567) bank172338 +172339 POINT(38.477990952475295 -121.85866668156419) bank172339 +172340 POINT(38.60478954018309 -122.8500120383987) bank172340 +172341 POINT(37.41891283399429 -122.50619809655205) bank172341 +172342 POINT(37.294939029162926 -121.62122856631451) bank172342 +172343 POINT(37.50130399723352 -123.01138214704547) bank172343 +172344 POINT(37.954521814404366 -122.85257951842681) bank172344 +172345 POINT(38.53697800066821 -122.5518598167191) bank172345 +172346 POINT(37.949364476073285 -122.54926493235878) bank172346 +172347 POINT(37.309616540563965 -121.50733201196267) bank172347 +172348 POINT(37.010962799507 -121.77151154666765) bank172348 +172349 POINT(38.19433853599905 -121.54329312815668) bank172349 +172350 POINT(38.611359257602885 -122.10614877004014) bank172350 +172351 POINT(36.94697656558902 -122.61019539252882) bank172351 +172352 POINT(38.73595552888525 -122.91515782922376) bank172352 +172353 POINT(37.94588609433603 -122.13549686062059) bank172353 +172354 POINT(37.76805755670009 -121.79265267218469) bank172354 +172355 POINT(38.47446279521234 -123.35420607566607) bank172355 +172356 POINT(37.3996789258588 -122.51617761609498) bank172356 +172357 POINT(37.07852358586796 -123.36494556257705) bank172357 +172358 POINT(38.34298827306648 -122.3363492610335) bank172358 +172359 POINT(38.001579803346274 -123.34292531621023) bank172359 +172360 POINT(37.49123622287071 -122.08124473780201) bank172360 +172361 POINT(36.88577883328642 -123.26561022492784) bank172361 +172362 POINT(38.41455272057803 -121.66062838908238) bank172362 +172363 POINT(37.48945121880586 -123.09017665165514) bank172363 +172364 POINT(37.594969802374706 -121.73155302053648) bank172364 +172365 POINT(38.232611484606515 -122.46883048137305) bank172365 +172366 POINT(37.76092439101336 -122.0973177373345) bank172366 +172367 POINT(36.78877811371574 -123.24858627355624) bank172367 +172368 POINT(36.977718595729286 -123.0963370171174) bank172368 +172369 POINT(37.884856006153605 -123.14506722877262) bank172369 +172370 POINT(38.222986297859705 -123.36218026519452) bank172370 +172371 POINT(38.00417398329292 -121.72826938729575) bank172371 +172372 POINT(38.07670865816005 -121.98463608651967) bank172372 +172373 POINT(36.88500886141489 -121.85124310338507) bank172373 +172374 POINT(38.153114772395746 -122.29615788609532) bank172374 +172375 POINT(38.28701553332733 -122.04365354873602) bank172375 +172376 POINT(37.58964317886531 -122.74697912347487) bank172376 +172377 POINT(38.21757672164989 -121.71874442468736) bank172377 +172378 POINT(37.788994716219044 -122.42687233964111) bank172378 +172379 POINT(37.653226804325286 -121.77590559047343) bank172379 +172380 POINT(37.01701346518393 -122.27829261100035) bank172380 +172381 POINT(37.00949910537935 -122.44469218911944) bank172381 +172382 POINT(37.11405242286712 -121.97758462215778) bank172382 +172383 POINT(38.59791322396633 -121.90519408653039) bank172383 +172384 POINT(36.8494969502329 -123.08176809692488) bank172384 +172385 POINT(38.74056321528125 -121.6191056793272) bank172385 +172386 POINT(36.97305729318098 -122.97732399717755) bank172386 +172387 POINT(38.205549180314236 -121.56351725930551) bank172387 +172388 POINT(37.95740061911665 -121.43395893723397) bank172388 +172389 POINT(36.78170755140331 -122.1983493982786) bank172389 +172390 POINT(37.25439130387417 -122.15031892496206) bank172390 +172391 POINT(37.69004466070821 -123.02588716291083) bank172391 +172392 POINT(38.274491042556285 -122.41913301468126) bank172392 +172393 POINT(37.8898789421076 -122.0239345279032) bank172393 +172394 POINT(37.76828739599389 -122.32990522078593) bank172394 +172395 POINT(38.269664053491894 -123.29586852133852) bank172395 +172396 POINT(38.31356366722633 -122.02099637418583) bank172396 +172397 POINT(37.87334077423682 -122.81411044562367) bank172397 +172398 POINT(38.59333310275923 -122.29203817999041) bank172398 +172399 POINT(38.49832255785232 -122.12241685230346) bank172399 +172400 POINT(37.422188323532026 -122.25916525234696) bank172400 +172401 POINT(38.729285887639925 -122.41747860385904) bank172401 +172402 POINT(37.626659109212376 -121.66026239717176) bank172402 +172403 POINT(38.42344849928905 -121.96729548262358) bank172403 +172404 POINT(38.450383272213514 -122.59004167852872) bank172404 +172405 POINT(37.18687297182781 -123.41829659236564) bank172405 +172406 POINT(38.13378767185567 -122.82785429342442) bank172406 +172407 POINT(38.43231570539693 -121.49245874519428) bank172407 +172408 POINT(37.089861667783026 -122.55541474624161) bank172408 +172409 POINT(36.96592945433264 -122.46385284742789) bank172409 +172410 POINT(37.81239345346061 -121.6792524226839) bank172410 +172411 POINT(38.62183917039841 -122.77725737914857) bank172411 +172412 POINT(37.03382947687558 -122.86994668019634) bank172412 +172413 POINT(38.37264732242343 -122.87643841976984) bank172413 +172414 POINT(37.1823112052417 -122.56175982424591) bank172414 +172415 POINT(37.62871251379795 -121.44033275797483) bank172415 +172416 POINT(38.68697665198335 -122.50105037439123) bank172416 +172417 POINT(37.37051598702609 -121.72951024346389) bank172417 +172418 POINT(38.23474780361131 -123.13714183012861) bank172418 +172419 POINT(38.02113722071373 -121.89032194690881) bank172419 +172420 POINT(38.59510410478983 -123.39993329897231) bank172420 +172421 POINT(37.85772234554934 -121.62450236775882) bank172421 +172422 POINT(36.826495850266156 -122.88111932625651) bank172422 +172423 POINT(37.831325423916105 -123.29912081030936) bank172423 +172424 POINT(38.648340061930064 -121.67139967385671) bank172424 +172425 POINT(38.53935637788988 -121.44100202689674) bank172425 +172426 POINT(37.144248444860224 -122.32741017898528) bank172426 +172427 POINT(36.95074050292328 -122.58789966367856) bank172427 +172428 POINT(38.06050871267754 -122.80856034464526) bank172428 +172429 POINT(38.64939836656812 -123.06548328971395) bank172429 +172430 POINT(37.68572403944128 -121.66346852624942) bank172430 +172431 POINT(37.74302444102246 -122.8746437923146) bank172431 +172432 POINT(36.862363330835066 -122.96775094072082) bank172432 +172433 POINT(36.8579356955923 -123.04410171285315) bank172433 +172434 POINT(37.66811781279382 -122.79783701050106) bank172434 +172435 POINT(38.13386780363041 -122.81962159127707) bank172435 +172436 POINT(38.53403784386227 -121.68495346584119) bank172436 +172437 POINT(36.87676966833772 -121.69614848777351) bank172437 +172438 POINT(37.39970378237836 -122.31986574525955) bank172438 +172439 POINT(37.48418199312194 -123.13534919669203) bank172439 +172440 POINT(37.64773602396324 -122.5340066058413) bank172440 +172441 POINT(38.22417251879915 -122.87087733065626) bank172441 +172442 POINT(37.11267831577314 -122.78573201271394) bank172442 +172443 POINT(37.63454882977087 -123.29023919881126) bank172443 +172444 POINT(37.95736919153162 -122.27258192569677) bank172444 +172445 POINT(37.44515292547381 -122.35967618393323) bank172445 +172446 POINT(38.14297039283056 -121.72563505922932) bank172446 +172447 POINT(38.715113761983446 -122.29668266838422) bank172447 +172448 POINT(37.328032882391746 -121.96583637887767) bank172448 +172449 POINT(37.81709651794285 -122.58854743458176) bank172449 +172450 POINT(37.11043192805823 -121.83011872498128) bank172450 +172451 POINT(37.35444301152518 -123.28862789701755) bank172451 +172452 POINT(37.017807203967706 -122.17349230878098) bank172452 +172453 POINT(37.37109028009315 -122.5383335664555) bank172453 +172454 POINT(38.16265435465611 -122.16530317907515) bank172454 +172455 POINT(37.729993102722666 -122.37344359057666) bank172455 +172456 POINT(37.50455900340676 -122.7579675221815) bank172456 +172457 POINT(38.02618588855144 -121.45653694524293) bank172457 +172458 POINT(37.45645821355394 -121.9904350192672) bank172458 +172459 POINT(37.7742131545298 -123.37531257754027) bank172459 +172460 POINT(37.92199774734821 -121.79945765446246) bank172460 +172461 POINT(38.745479605413614 -121.78506982380615) bank172461 +172462 POINT(37.25658630900297 -123.34256986947396) bank172462 +172463 POINT(38.452303692137924 -121.73965215585397) bank172463 +172464 POINT(38.189091421366655 -122.20085647441842) bank172464 +172465 POINT(37.69875541787074 -122.81757697132609) bank172465 +172466 POINT(36.99413219489737 -122.1131317425208) bank172466 +172467 POINT(37.9919596999803 -122.38463175888681) bank172467 +172468 POINT(37.199374093174455 -122.04740873213471) bank172468 +172469 POINT(38.65900438215024 -121.6387416116138) bank172469 +172470 POINT(37.467508262331414 -122.14868100304075) bank172470 +172471 POINT(36.94154951026122 -123.24288117293527) bank172471 +172472 POINT(36.8286776566255 -122.26985226714389) bank172472 +172473 POINT(37.357893999881604 -122.51907082022994) bank172473 +172474 POINT(38.19580041550588 -123.26904658255225) bank172474 +172475 POINT(37.47371688675414 -121.81142437272146) bank172475 +172476 POINT(37.95073521337203 -121.61912872236003) bank172476 +172477 POINT(38.26219525911691 -122.22627448636521) bank172477 +172478 POINT(38.73522278372654 -121.82484924568614) bank172478 +172479 POINT(38.00287169722798 -123.35358625063787) bank172479 +172480 POINT(37.409694800712664 -123.35776192926866) bank172480 +172481 POINT(36.89987658546948 -121.55747489155209) bank172481 +172482 POINT(37.17605942770907 -122.08365464962742) bank172482 +172483 POINT(38.243471991897906 -123.06040839345518) bank172483 +172484 POINT(37.01832745427505 -121.62735510520791) bank172484 +172485 POINT(37.28366046492406 -122.14615433354156) bank172485 +172486 POINT(36.91955384478703 -122.16421233469814) bank172486 +172487 POINT(37.51486866307488 -122.45374014198202) bank172487 +172488 POINT(38.27501454734109 -123.39130293909992) bank172488 +172489 POINT(38.51355279912365 -121.49147119606293) bank172489 +172490 POINT(37.6873986337976 -122.44148069954102) bank172490 +172491 POINT(38.537314990790414 -121.60955424343173) bank172491 +172492 POINT(37.88747536665303 -122.38373185925468) bank172492 +172493 POINT(37.46484988173912 -122.90066600619119) bank172493 +172494 POINT(37.54057156924887 -121.79388715197845) bank172494 +172495 POINT(38.480510719493466 -122.1126245395812) bank172495 +172496 POINT(38.07961867221556 -121.71539470789527) bank172496 +172497 POINT(38.382495737419895 -123.34680036458802) bank172497 +172498 POINT(37.550416305373055 -122.49907592158945) bank172498 +172499 POINT(38.34788149966525 -121.87906354738597) bank172499 +172500 POINT(38.43926190664747 -122.3629896723686) bank172500 +172501 POINT(37.62739496554856 -121.99368295005294) bank172501 +172502 POINT(38.770673298796694 -121.79322992262038) bank172502 +172503 POINT(37.16352551192951 -123.10840997707282) bank172503 +172504 POINT(38.00032302093415 -122.85342568402986) bank172504 +172505 POINT(38.23004778454275 -122.4317706419179) bank172505 +172506 POINT(37.222967452093634 -122.26113203199961) bank172506 +172507 POINT(38.2551015420935 -122.0840647854956) bank172507 +172508 POINT(36.86078036439736 -122.13190103249124) bank172508 +172509 POINT(38.70123361108483 -122.00142876951938) bank172509 +172510 POINT(38.15397864071488 -121.77190822398092) bank172510 +172511 POINT(38.317110201420675 -122.80108758593589) bank172511 +172512 POINT(36.96530400702793 -121.9677686774876) bank172512 +172513 POINT(38.081194884840215 -122.72296303542767) bank172513 +172514 POINT(37.224624086082486 -123.02851916815871) bank172514 +172515 POINT(38.319599304444594 -123.12742841943745) bank172515 +172516 POINT(38.434641097708855 -121.90749362831852) bank172516 +172517 POINT(37.590023286024135 -122.35576049637453) bank172517 +172518 POINT(37.55466343112993 -122.3474837629103) bank172518 +172519 POINT(37.54020094266857 -123.17486731515713) bank172519 +172520 POINT(37.226260999819246 -122.17844639260605) bank172520 +172521 POINT(38.33190234091941 -121.74038377870266) bank172521 +172522 POINT(38.34974282194013 -121.89118272288795) bank172522 +172523 POINT(38.162824977247325 -123.39249197286284) bank172523 +172524 POINT(38.23480595323301 -122.40274122016757) bank172524 +172525 POINT(37.860498993143516 -121.71430690059988) bank172525 +172526 POINT(37.914433740540424 -122.16557645152561) bank172526 +172527 POINT(38.13162852460246 -122.41016886686576) bank172527 +172528 POINT(37.16711376831081 -122.41572244137602) bank172528 +172529 POINT(37.23605258373687 -121.56531177599746) bank172529 +172530 POINT(38.38896920168252 -121.70620261737432) bank172530 +172531 POINT(37.768210755835554 -121.66849282988147) bank172531 +172532 POINT(37.83962064447345 -123.17795715391625) bank172532 +172533 POINT(37.64692295789686 -121.78208010525947) bank172533 +172534 POINT(37.010844788832955 -122.53457842505007) bank172534 +172535 POINT(37.31632752465915 -122.24399164657318) bank172535 +172536 POINT(38.310810965951134 -122.92642525262858) bank172536 +172537 POINT(37.77531198811789 -122.95730416331106) bank172537 +172538 POINT(37.97938455009566 -121.9987674087939) bank172538 +172539 POINT(36.93594004511077 -121.99702953294056) bank172539 +172540 POINT(37.70047504303477 -122.72439436545072) bank172540 +172541 POINT(37.47623935525593 -122.64192920850537) bank172541 +172542 POINT(38.330855835637756 -123.19913240843269) bank172542 +172543 POINT(38.04830451122664 -121.77437005646499) bank172543 +172544 POINT(37.269085238696135 -122.46940665146052) bank172544 +172545 POINT(38.67484362814893 -122.29422999312312) bank172545 +172546 POINT(38.29862405550828 -122.56683173889459) bank172546 +172547 POINT(38.22967643657861 -122.81987445471522) bank172547 +172548 POINT(37.591911124250245 -122.5449127067727) bank172548 +172549 POINT(37.166758147413276 -122.42257711581686) bank172549 +172550 POINT(36.88047834192235 -121.42456112821485) bank172550 +172551 POINT(37.17403373825836 -122.43313059939409) bank172551 +172552 POINT(38.48600905472603 -122.78262764120251) bank172552 +172553 POINT(38.1689743887835 -122.58743609143622) bank172553 +172554 POINT(38.02982697297371 -121.4918062909812) bank172554 +172555 POINT(37.41751470695916 -122.66396997879139) bank172555 +172556 POINT(37.73637854370298 -122.06733833013529) bank172556 +172557 POINT(38.49276568042821 -122.87246231050324) bank172557 +172558 POINT(38.11135425164103 -122.79124836072312) bank172558 +172559 POINT(38.46207913466929 -123.12538915534245) bank172559 +172560 POINT(37.54926181706438 -122.4223494555857) bank172560 +172561 POINT(37.796363610173465 -122.00793469564462) bank172561 +172562 POINT(37.833190441266524 -121.50588863257805) bank172562 +172563 POINT(37.70258792521439 -122.6083895321363) bank172563 +172564 POINT(37.702267309513815 -122.1642186878756) bank172564 +172565 POINT(37.998657184838315 -123.31028418760178) bank172565 +172566 POINT(37.92457134714667 -123.16180936445411) bank172566 +172567 POINT(37.910332060271315 -122.57495636804587) bank172567 +172568 POINT(38.755670073515105 -122.13759226881227) bank172568 +172569 POINT(38.003447933655615 -121.60837720839388) bank172569 +172570 POINT(36.80213883821759 -121.55525629744825) bank172570 +172571 POINT(38.38122044355715 -123.38930560188072) bank172571 +172572 POINT(37.02706790740098 -121.60478479172986) bank172572 +172573 POINT(37.590135909646825 -122.97588962322624) bank172573 +172574 POINT(37.93257476527028 -121.68277430030247) bank172574 +172575 POINT(37.085013029233245 -122.06290455263327) bank172575 +172576 POINT(38.25734140037221 -121.7959553945093) bank172576 +172577 POINT(38.20123382128808 -122.22471932173441) bank172577 +172578 POINT(38.294124117465465 -122.97248252899074) bank172578 +172579 POINT(37.12425882458797 -123.19694959178412) bank172579 +172580 POINT(38.76871680729144 -122.88412470516367) bank172580 +172581 POINT(37.82030125961239 -122.50064658558057) bank172581 +172582 POINT(38.12981709612704 -121.55512013990669) bank172582 +172583 POINT(36.97531627665723 -122.3875319362344) bank172583 +172584 POINT(37.857447731592345 -122.81798764941502) bank172584 +172585 POINT(38.08081358919458 -121.43952850698976) bank172585 +172586 POINT(37.51255808658756 -122.73274419968175) bank172586 +172587 POINT(37.78734958709797 -122.91598035189497) bank172587 +172588 POINT(38.457327027087786 -121.6049018099477) bank172588 +172589 POINT(38.163118028954415 -122.9288692455512) bank172589 +172590 POINT(38.40118555886832 -121.95489740040387) bank172590 +172591 POINT(37.769462404399384 -121.80805976893375) bank172591 +172592 POINT(37.78491335925464 -122.27091475666516) bank172592 +172593 POINT(38.102803558716595 -122.85168140759124) bank172593 +172594 POINT(38.16260190362629 -122.01833017921794) bank172594 +172595 POINT(37.00855730199426 -122.14720874652525) bank172595 +172596 POINT(37.30493571279019 -121.81383667028801) bank172596 +172597 POINT(38.387145200233704 -121.43876095619109) bank172597 +172598 POINT(36.83618864669153 -123.19455636416752) bank172598 +172599 POINT(38.42664860254847 -121.50818207340285) bank172599 +172600 POINT(37.87677692772511 -121.89531264002447) bank172600 +172601 POINT(37.817930693851224 -122.84888431430625) bank172601 +172602 POINT(37.40256433203068 -122.31531465414947) bank172602 +172603 POINT(37.30384278407385 -121.74621086839447) bank172603 +172604 POINT(38.36755906826703 -122.96484850570639) bank172604 +172605 POINT(37.68263129141275 -121.62966043118071) bank172605 +172606 POINT(37.31971954614121 -123.03971929157593) bank172606 +172607 POINT(38.14192114312535 -122.76318186115218) bank172607 +172608 POINT(37.79481614076724 -121.50427695099626) bank172608 +172609 POINT(38.16475029675891 -122.44623679943872) bank172609 +172610 POINT(38.51887956094501 -121.43725564877805) bank172610 +172611 POINT(37.98290882845224 -123.35605243222737) bank172611 +172612 POINT(37.402527745601795 -122.67253712201354) bank172612 +172613 POINT(36.79240156407252 -123.00777853792914) bank172613 +172614 POINT(38.27362164386122 -121.89082575061116) bank172614 +172615 POINT(38.426889455539104 -123.12522967213889) bank172615 +172616 POINT(36.97019979123963 -122.9894943084374) bank172616 +172617 POINT(37.73142757985001 -121.45459737937779) bank172617 +172618 POINT(38.19094867080394 -121.77286517559925) bank172618 +172619 POINT(38.29044493160335 -122.43578025691711) bank172619 +172620 POINT(38.04052661141978 -123.14822147778658) bank172620 +172621 POINT(38.51522642684151 -121.53891117928072) bank172621 +172622 POINT(37.605926215949765 -122.34389080118169) bank172622 +172623 POINT(37.09151800799107 -121.92159044049188) bank172623 +172624 POINT(37.75308284431188 -123.11976856896612) bank172624 +172625 POINT(38.44437549493863 -122.81668442155708) bank172625 +172626 POINT(37.05096463775088 -123.1634523919949) bank172626 +172627 POINT(37.00086982468605 -122.755285270373) bank172627 +172628 POINT(37.425082793801245 -123.3703151826578) bank172628 +172629 POINT(38.058802097824355 -122.77315353929616) bank172629 +172630 POINT(36.941868941638546 -122.25284212649741) bank172630 +172631 POINT(38.34778545543071 -121.63856332602028) bank172631 +172632 POINT(38.62724449966757 -122.16997300350182) bank172632 +172633 POINT(38.56695279256052 -122.47308214111429) bank172633 +172634 POINT(37.10878258851148 -122.62451600344778) bank172634 +172635 POINT(38.50838086840054 -121.45497472783146) bank172635 +172636 POINT(37.27395842441861 -121.75052385771365) bank172636 +172637 POINT(36.88762240206608 -121.99190062732721) bank172637 +172638 POINT(38.518704221864034 -122.53804460825424) bank172638 +172639 POINT(37.86631789939805 -122.8296437463749) bank172639 +172640 POINT(37.20791908976518 -121.90641788693947) bank172640 +172641 POINT(37.45913422827093 -121.7880872766113) bank172641 +172642 POINT(37.27155591397436 -121.49303718664466) bank172642 +172643 POINT(37.97162924653604 -122.80689860017236) bank172643 +172644 POINT(38.33147395046045 -122.57763121995876) bank172644 +172645 POINT(38.398490268976126 -122.32672036153043) bank172645 +172646 POINT(37.24387730257245 -122.96447740992646) bank172646 +172647 POINT(38.51873936544401 -122.00365907892427) bank172647 +172648 POINT(37.027443638587954 -122.66461767786437) bank172648 +172649 POINT(38.76942475338774 -122.48269021452931) bank172649 +172650 POINT(38.4517265811562 -123.11153740137566) bank172650 +172651 POINT(38.61796451306469 -122.45364681226287) bank172651 +172652 POINT(37.10620234186671 -122.07924241569765) bank172652 +172653 POINT(37.012902796607555 -123.24889616027869) bank172653 +172654 POINT(37.61869478832802 -122.0850834860629) bank172654 +172655 POINT(37.68867501910278 -122.64398475419867) bank172655 +172656 POINT(37.33567079322411 -122.4771632200621) bank172656 +172657 POINT(37.13024908622309 -123.14528342821346) bank172657 +172658 POINT(37.27773341744539 -122.96796749652569) bank172658 +172659 POINT(37.5964722608265 -123.21456229653698) bank172659 +172660 POINT(38.16730265866237 -121.95329540344716) bank172660 +172661 POINT(37.56161424284087 -121.49896992629544) bank172661 +172662 POINT(38.137699222571136 -123.2689017524214) bank172662 +172663 POINT(38.09074347875926 -121.80562983458589) bank172663 +172664 POINT(38.548657430705674 -123.41194711277572) bank172664 +172665 POINT(37.07538330734085 -123.20074624531283) bank172665 +172666 POINT(38.52903213993006 -122.75359414812023) bank172666 +172667 POINT(37.79196316699804 -122.50236460072246) bank172667 +172668 POINT(37.6803487082548 -122.77046881786947) bank172668 +172669 POINT(38.38629791119845 -121.89003356312868) bank172669 +172670 POINT(37.095064563182085 -123.36417038952705) bank172670 +172671 POINT(37.17859862839584 -123.25388200212967) bank172671 +172672 POINT(36.95248725506727 -122.92672931220845) bank172672 +172673 POINT(37.46493970639717 -123.3126081001398) bank172673 +172674 POINT(37.80494886408514 -123.31379730417373) bank172674 +172675 POINT(38.742773595268574 -122.21922461008664) bank172675 +172676 POINT(38.687199950365056 -121.61332812962331) bank172676 +172677 POINT(38.00170952017744 -122.34643727626543) bank172677 +172678 POINT(37.789170986504 -122.53279541007598) bank172678 +172679 POINT(38.37923011638762 -122.47718514341096) bank172679 +172680 POINT(38.69990504426433 -122.82747652989647) bank172680 +172681 POINT(38.1471038070646 -122.59241080358629) bank172681 +172682 POINT(38.7370190453094 -123.07455076743632) bank172682 +172683 POINT(38.3771241219659 -122.66949491885768) bank172683 +172684 POINT(37.9025195498011 -122.29961990461211) bank172684 +172685 POINT(38.67318582142147 -121.75270819264504) bank172685 +172686 POINT(37.8474415656915 -122.31410566303133) bank172686 +172687 POINT(37.891836846147314 -122.90322885766662) bank172687 +172688 POINT(36.86026133686845 -123.23013780140424) bank172688 +172689 POINT(38.4987433375276 -122.1712052914502) bank172689 +172690 POINT(36.83664893512604 -123.23370163221371) bank172690 +172691 POINT(36.910242924574206 -121.66363815696701) bank172691 +172692 POINT(37.67268717323713 -121.57689563205348) bank172692 +172693 POINT(37.84555965359864 -123.13821704080192) bank172693 +172694 POINT(38.524176413872354 -122.41688998004848) bank172694 +172695 POINT(37.58336026773816 -122.56752258836369) bank172695 +172696 POINT(37.24809448248897 -122.52352396889334) bank172696 +172697 POINT(37.76877861162137 -122.18318666900907) bank172697 +172698 POINT(38.49020164771809 -122.33693081733543) bank172698 +172699 POINT(37.77366839997216 -123.2340579594178) bank172699 +172700 POINT(37.425991264634334 -122.88845442381914) bank172700 +172701 POINT(37.16834312384905 -122.90543679447839) bank172701 +172702 POINT(36.77593872518526 -122.87342953263033) bank172702 +172703 POINT(38.418925269202774 -122.39015404625117) bank172703 +172704 POINT(37.630034699737 -121.51619179609777) bank172704 +172705 POINT(37.02814095736688 -122.24682112164113) bank172705 +172706 POINT(37.65231322970361 -122.7700050177434) bank172706 +172707 POINT(38.60408008054826 -121.67556166280933) bank172707 +172708 POINT(38.28477049941764 -121.69144355685806) bank172708 +172709 POINT(37.30380360449649 -121.6552988869856) bank172709 +172710 POINT(38.028487861436425 -122.6283346324803) bank172710 +172711 POINT(37.94778812331005 -121.42890078342778) bank172711 +172712 POINT(37.92489194928215 -121.58898944236823) bank172712 +172713 POINT(37.241386540754014 -123.34250699182863) bank172713 +172714 POINT(38.29379851008566 -122.60868341724247) bank172714 +172715 POINT(37.8549704173695 -122.26445016211505) bank172715 +172716 POINT(38.75757821144055 -122.3819950972564) bank172716 +172717 POINT(37.596056234340566 -123.09415446758747) bank172717 +172718 POINT(37.371590487899624 -122.15011453063532) bank172718 +172719 POINT(38.04482487675566 -123.02561284125836) bank172719 +172720 POINT(38.004989269392624 -122.86750890148222) bank172720 +172721 POINT(38.70755536422248 -122.65272086483357) bank172721 +172722 POINT(38.36643335460686 -122.79152769502127) bank172722 +172723 POINT(37.318919485153735 -122.63426688424397) bank172723 +172724 POINT(37.20988479896744 -121.91687668682312) bank172724 +172725 POINT(38.61532565825291 -122.4616325954661) bank172725 +172726 POINT(37.96678847726262 -122.77975152011061) bank172726 +172727 POINT(37.55635681346563 -121.42886168624948) bank172727 +172728 POINT(37.363222939562796 -122.43696066012828) bank172728 +172729 POINT(37.94685030329219 -122.3207307621313) bank172729 +172730 POINT(36.77780370504442 -122.19828090296251) bank172730 +172731 POINT(37.81692564612416 -122.88922693116209) bank172731 +172732 POINT(38.596809354191045 -123.07270761282942) bank172732 +172733 POINT(37.63610977058808 -122.77853621804422) bank172733 +172734 POINT(37.99381385393909 -123.19040064059962) bank172734 +172735 POINT(37.402499644666996 -121.70431600882034) bank172735 +172736 POINT(38.376600975751316 -121.51598070070203) bank172736 +172737 POINT(38.04798567506873 -122.1509361109668) bank172737 +172738 POINT(37.74384084225273 -123.2306467414434) bank172738 +172739 POINT(38.72307459924356 -123.1861949663598) bank172739 +172740 POINT(37.56507683533862 -122.29549797865045) bank172740 +172741 POINT(38.66061962506105 -123.12925981937096) bank172741 +172742 POINT(37.68791349854792 -123.21546075186922) bank172742 +172743 POINT(37.66031469743444 -122.05306282062162) bank172743 +172744 POINT(37.766608707586144 -122.63508099441347) bank172744 +172745 POINT(38.483696011931144 -122.69560618094991) bank172745 +172746 POINT(37.78073584511658 -121.53804456948751) bank172746 +172747 POINT(38.142353727166544 -123.055869878463) bank172747 +172748 POINT(38.66148750545434 -123.36895139888472) bank172748 +172749 POINT(37.43624961530781 -122.41660779724704) bank172749 +172750 POINT(38.47533298458368 -123.05176195948086) bank172750 +172751 POINT(38.71780720852932 -122.71233805024376) bank172751 +172752 POINT(38.67952514348455 -121.79283171845844) bank172752 +172753 POINT(37.58833280887758 -122.50645196455862) bank172753 +172754 POINT(37.201162491134575 -121.55350196272016) bank172754 +172755 POINT(37.910757165371116 -122.70661648101311) bank172755 +172756 POINT(37.328483299905415 -122.68498755702288) bank172756 +172757 POINT(38.56775480354302 -122.14001219228497) bank172757 +172758 POINT(38.27908785412631 -122.4378979503453) bank172758 +172759 POINT(37.28994006399095 -122.65574690856823) bank172759 +172760 POINT(36.84270796960108 -122.27192957488964) bank172760 +172761 POINT(37.177581933727886 -123.04578426473395) bank172761 +172762 POINT(38.524073272531545 -123.05315316198521) bank172762 +172763 POINT(37.29569203713338 -121.58387621023057) bank172763 +172764 POINT(37.44815568201344 -122.43670936206867) bank172764 +172765 POINT(37.58431474544865 -122.32392460120559) bank172765 +172766 POINT(36.81333505868773 -123.14542453678327) bank172766 +172767 POINT(36.88084151969872 -123.35455860222112) bank172767 +172768 POINT(37.37288394351855 -121.79582770080746) bank172768 +172769 POINT(38.524088441215895 -122.42726555838323) bank172769 +172770 POINT(38.031484842860415 -122.23512330314588) bank172770 +172771 POINT(38.376593123892995 -123.11905857676355) bank172771 +172772 POINT(38.1278940984828 -122.5749645761994) bank172772 +172773 POINT(38.1656884779078 -121.7599722212929) bank172773 +172774 POINT(37.39117420936451 -122.71599948628368) bank172774 +172775 POINT(37.130754258106265 -121.86393585309331) bank172775 +172776 POINT(37.286995531430215 -123.21649567668656) bank172776 +172777 POINT(37.416563820621484 -122.09761497300556) bank172777 +172778 POINT(37.30228262446608 -122.56193201780484) bank172778 +172779 POINT(38.39387958654339 -122.01733446443254) bank172779 +172780 POINT(37.54966353058743 -122.27982285319439) bank172780 +172781 POINT(36.998543715554504 -123.41523461082832) bank172781 +172782 POINT(37.93257321574445 -123.25347326285325) bank172782 +172783 POINT(37.21792223848499 -123.03231745220242) bank172783 +172784 POINT(38.74109199465929 -122.03539841084678) bank172784 +172785 POINT(37.531370043837356 -121.70083097741025) bank172785 +172786 POINT(37.968277458404785 -122.23137450557138) bank172786 +172787 POINT(38.02004638879362 -122.52790598910951) bank172787 +172788 POINT(38.31104305271523 -121.92273736463517) bank172788 +172789 POINT(38.67959597947947 -122.65987220642948) bank172789 +172790 POINT(37.85087586004616 -123.18081412371181) bank172790 +172791 POINT(37.688362925467096 -122.63241328041558) bank172791 +172792 POINT(37.83031246576881 -122.25387424878657) bank172792 +172793 POINT(37.974277374019486 -123.06962462428085) bank172793 +172794 POINT(37.543800375509655 -122.9295432325058) bank172794 +172795 POINT(38.64248898595244 -123.38159728385935) bank172795 +172796 POINT(38.756209087783375 -122.62408036086697) bank172796 +172797 POINT(38.4636559475809 -121.57464191707376) bank172797 +172798 POINT(36.87854294502607 -123.41297193631672) bank172798 +172799 POINT(38.73455627222947 -123.40619090121069) bank172799 +172800 POINT(37.34391144671442 -122.36750164154935) bank172800 +172801 POINT(36.92255766822309 -122.37442111136104) bank172801 +172802 POINT(37.91546634730956 -121.69224108187277) bank172802 +172803 POINT(37.75617388795962 -122.06634432262247) bank172803 +172804 POINT(38.47893017411834 -121.98158537095458) bank172804 +172805 POINT(38.23212796843551 -123.18296361482304) bank172805 +172806 POINT(37.45081586969392 -121.42421030143107) bank172806 +172807 POINT(36.886718261558265 -122.91546469307043) bank172807 +172808 POINT(36.8510838280202 -122.0241019882115) bank172808 +172809 POINT(36.86516863645071 -122.90487376586405) bank172809 +172810 POINT(37.484221641227656 -122.32084996648811) bank172810 +172811 POINT(37.50229835727275 -123.09498899535382) bank172811 +172812 POINT(38.11311976976677 -121.4798957665843) bank172812 +172813 POINT(38.70948714270192 -123.1854025721752) bank172813 +172814 POINT(37.837017376762546 -122.72922596635571) bank172814 +172815 POINT(37.71236781393171 -122.59425981672808) bank172815 +172816 POINT(38.31235423767738 -121.64724572167792) bank172816 +172817 POINT(37.55649665609703 -122.82239730448929) bank172817 +172818 POINT(37.12303673101704 -122.25777588184283) bank172818 +172819 POINT(37.355654344848446 -122.5170461800926) bank172819 +172820 POINT(38.29264608188834 -122.66043907827299) bank172820 +172821 POINT(38.27101147836584 -122.98390697193543) bank172821 +172822 POINT(37.11477624999292 -123.1112027076249) bank172822 +172823 POINT(38.482456407356 -122.18387765167255) bank172823 +172824 POINT(38.092376810522694 -121.82534882136285) bank172824 +172825 POINT(38.391252028648786 -123.00997039464308) bank172825 +172826 POINT(38.65980086942451 -123.28707587900765) bank172826 +172827 POINT(38.61776657799605 -121.97517606003201) bank172827 +172828 POINT(37.78286964415249 -122.2762998208978) bank172828 +172829 POINT(38.7182738443636 -122.98372959850799) bank172829 +172830 POINT(38.75995674205738 -122.56850343820801) bank172830 +172831 POINT(37.041411974412824 -122.20366700886485) bank172831 +172832 POINT(37.976441531310854 -122.5700688322486) bank172832 +172833 POINT(38.156831965653424 -122.76597596764066) bank172833 +172834 POINT(36.81726304868216 -122.76453528026137) bank172834 +172835 POINT(38.54866790476298 -123.12367838656753) bank172835 +172836 POINT(38.68476067305533 -121.9653302576995) bank172836 +172837 POINT(37.88645183737586 -121.54940010041045) bank172837 +172838 POINT(38.2191182827482 -122.1924230149111) bank172838 +172839 POINT(37.06608452162479 -122.3898263314932) bank172839 +172840 POINT(37.380052551500846 -122.21981616437056) bank172840 +172841 POINT(37.47979411217852 -121.90676108594619) bank172841 +172842 POINT(37.813862045901026 -121.85547813178742) bank172842 +172843 POINT(37.740543858858665 -122.15984126676321) bank172843 +172844 POINT(38.7291319475567 -122.1402980045956) bank172844 +172845 POINT(38.50667072437368 -121.54017891026628) bank172845 +172846 POINT(37.08267124332829 -123.06107010543744) bank172846 +172847 POINT(37.670574451605624 -121.83605142672762) bank172847 +172848 POINT(38.06997794630183 -122.43280467631037) bank172848 +172849 POINT(37.282186568361155 -121.67593845415144) bank172849 +172850 POINT(38.171266533372886 -122.01825912722661) bank172850 +172851 POINT(38.16185134605897 -121.48548832994402) bank172851 +172852 POINT(38.30612908674989 -122.87705100962366) bank172852 +172853 POINT(37.73018154808111 -121.45885845746669) bank172853 +172854 POINT(37.435400590728925 -121.80825792686738) bank172854 +172855 POINT(36.98964532906818 -121.78321209161757) bank172855 +172856 POINT(37.607303829274905 -123.13871339443342) bank172856 +172857 POINT(37.99101331794865 -122.45472120996234) bank172857 +172858 POINT(37.37980948994364 -122.58712625041836) bank172858 +172859 POINT(38.513383969418804 -123.14092827614131) bank172859 +172860 POINT(36.779222105755444 -122.32576626713083) bank172860 +172861 POINT(38.13795183027036 -122.59788511635264) bank172861 +172862 POINT(38.64779671930211 -122.47479848545395) bank172862 +172863 POINT(38.6172446656527 -121.60988446210101) bank172863 +172864 POINT(38.08007744057101 -122.57908772884541) bank172864 +172865 POINT(37.25713526809944 -123.07841269605849) bank172865 +172866 POINT(37.54271014078639 -122.0130290481143) bank172866 +172867 POINT(38.11146911355747 -122.17096051900499) bank172867 +172868 POINT(37.55001127068034 -123.00988656627581) bank172868 +172869 POINT(37.15289705930612 -123.41899878339908) bank172869 +172870 POINT(38.04013961053327 -121.84382097095836) bank172870 +172871 POINT(37.948567611502135 -122.9985056328591) bank172871 +172872 POINT(37.12644798260877 -122.36861163790472) bank172872 +172873 POINT(37.95780369810566 -122.72487634421715) bank172873 +172874 POINT(37.78052952396301 -122.88723246116582) bank172874 +172875 POINT(38.7685103269626 -122.55364197415854) bank172875 +172876 POINT(36.98092738184967 -122.08067179987925) bank172876 +172877 POINT(36.90592343581596 -121.66327209807201) bank172877 +172878 POINT(38.67153245375887 -123.2003944635188) bank172878 +172879 POINT(37.82561031200039 -123.27121245413186) bank172879 +172880 POINT(38.40029269116945 -122.39056583768526) bank172880 +172881 POINT(37.51567740978905 -121.48089598232852) bank172881 +172882 POINT(37.456344659191316 -122.60380250781616) bank172882 +172883 POINT(36.84803795883942 -122.18715154286241) bank172883 +172884 POINT(37.719895587108795 -121.73384132928481) bank172884 +172885 POINT(37.12231045560276 -122.50440212959282) bank172885 +172886 POINT(37.77149118587922 -121.81490140349872) bank172886 +172887 POINT(37.548115017608595 -121.45667529477154) bank172887 +172888 POINT(37.6335735844764 -121.68221626920123) bank172888 +172889 POINT(38.51463757177889 -121.95102275116146) bank172889 +172890 POINT(38.399369128239 -122.00733483477936) bank172890 +172891 POINT(37.78468395966149 -121.91850016454843) bank172891 +172892 POINT(37.52847025804453 -122.47220590279753) bank172892 +172893 POINT(37.05347618917837 -123.1599833021519) bank172893 +172894 POINT(37.81789490872705 -122.06957916778022) bank172894 +172895 POINT(37.525356237330215 -122.3084882237579) bank172895 +172896 POINT(38.201433947964766 -121.69074486103665) bank172896 +172897 POINT(38.1373191588085 -122.90384740729033) bank172897 +172898 POINT(38.228968233726604 -121.75282599592897) bank172898 +172899 POINT(38.767639187180635 -121.4746447699943) bank172899 +172900 POINT(37.14827865989166 -121.99120938309264) bank172900 +172901 POINT(37.38002843840585 -122.75228674363439) bank172901 +172902 POINT(37.30184675841279 -121.9938862161861) bank172902 +172903 POINT(37.0619069039075 -123.21155342988324) bank172903 +172904 POINT(38.43181868431629 -121.58825132705574) bank172904 +172905 POINT(37.88629272276104 -123.08352470191687) bank172905 +172906 POINT(37.58541786160561 -121.5054325496302) bank172906 +172907 POINT(36.91177719780674 -121.69099232901738) bank172907 +172908 POINT(37.29100510060988 -122.39313462956282) bank172908 +172909 POINT(37.50535563270241 -121.66131905407335) bank172909 +172910 POINT(37.90617269887241 -121.82945231755964) bank172910 +172911 POINT(38.04198714785618 -121.93691556827459) bank172911 +172912 POINT(37.903465531475135 -123.23025560138208) bank172912 +172913 POINT(37.083187135434706 -123.35038175595713) bank172913 +172914 POINT(38.06248239340022 -121.66987933275355) bank172914 +172915 POINT(38.363585609549965 -121.9424326176714) bank172915 +172916 POINT(37.148377417859585 -122.13274848733002) bank172916 +172917 POINT(38.05906155642134 -121.87327234307774) bank172917 +172918 POINT(37.86566255616156 -122.94509713146105) bank172918 +172919 POINT(38.53441490288052 -123.01401639479623) bank172919 +172920 POINT(36.912014031440926 -122.21415320741053) bank172920 +172921 POINT(37.01545703432802 -121.69730226373235) bank172921 +172922 POINT(37.33373629700485 -123.11593674354003) bank172922 +172923 POINT(38.70534183834262 -122.00353815163506) bank172923 +172924 POINT(38.70572665435024 -122.83192498641779) bank172924 +172925 POINT(36.87413769369372 -121.63982586841425) bank172925 +172926 POINT(37.60929982326619 -121.7251003013652) bank172926 +172927 POINT(38.16825851293391 -122.67916097816035) bank172927 +172928 POINT(36.84329033547682 -122.16714576975316) bank172928 +172929 POINT(37.87377552964941 -121.62938164199406) bank172929 +172930 POINT(38.54523420997218 -122.28054813338346) bank172930 +172931 POINT(38.534128291458636 -121.54738957797292) bank172931 +172932 POINT(37.19877637061809 -121.8724933025696) bank172932 +172933 POINT(36.85605059318123 -123.32847531557714) bank172933 +172934 POINT(38.30338278289627 -122.82678360877375) bank172934 +172935 POINT(37.999116406823006 -122.70012813117448) bank172935 +172936 POINT(37.4401091445041 -121.44992075577767) bank172936 +172937 POINT(37.265135438662846 -122.40178367725325) bank172937 +172938 POINT(38.707906035948426 -122.05763153329319) bank172938 +172939 POINT(38.01378356453677 -122.36351526854334) bank172939 +172940 POINT(37.1922865731532 -121.61469874305602) bank172940 +172941 POINT(38.1241827601403 -122.09918876143544) bank172941 +172942 POINT(38.1043642864494 -122.35694772729578) bank172942 +172943 POINT(37.14698705408517 -121.7102501330316) bank172943 +172944 POINT(38.25971051099481 -122.34137329786377) bank172944 +172945 POINT(37.11998400097204 -122.44525427215807) bank172945 +172946 POINT(37.54123616488657 -122.8279955970606) bank172946 +172947 POINT(37.13363137951951 -122.70802371621055) bank172947 +172948 POINT(37.94950977459636 -121.67247772025682) bank172948 +172949 POINT(37.89065402585088 -122.49376967652677) bank172949 +172950 POINT(36.91037742417591 -122.08302912528899) bank172950 +172951 POINT(38.76980272892855 -123.29616072972458) bank172951 +172952 POINT(37.02676692653679 -121.87044081060559) bank172952 +172953 POINT(37.62239685138256 -121.79066862858447) bank172953 +172954 POINT(38.25814558824314 -123.03433924239067) bank172954 +172955 POINT(37.075361701209836 -123.02417514765509) bank172955 +172956 POINT(36.953739588960374 -123.17933146305812) bank172956 +172957 POINT(37.105188881163436 -121.59447851605782) bank172957 +172958 POINT(37.32991919217024 -122.94729906331642) bank172958 +172959 POINT(36.97630214010841 -122.60994358428187) bank172959 +172960 POINT(37.712427693169346 -121.76810485285085) bank172960 +172961 POINT(38.471689070848 -121.53744032554914) bank172961 +172962 POINT(38.154794787168335 -122.97291338607918) bank172962 +172963 POINT(37.07881592173761 -122.82375779680113) bank172963 +172964 POINT(38.08987396054424 -122.08567629881134) bank172964 +172965 POINT(38.23739990554602 -122.04781714057198) bank172965 +172966 POINT(38.391438302313446 -122.11740740119876) bank172966 +172967 POINT(37.78930853390919 -123.15568652510521) bank172967 +172968 POINT(37.40272376294192 -121.55859386648159) bank172968 +172969 POINT(37.93005919830513 -122.45828460757122) bank172969 +172970 POINT(37.321566218367764 -123.14650531075934) bank172970 +172971 POINT(37.47089956441404 -123.28349878959105) bank172971 +172972 POINT(37.94596730215054 -122.54234660264517) bank172972 +172973 POINT(36.941483783944044 -121.65095014308166) bank172973 +172974 POINT(37.94818032185667 -122.85586484527907) bank172974 +172975 POINT(38.51348099827153 -122.22577190001013) bank172975 +172976 POINT(37.457110805950315 -121.69042945119355) bank172976 +172977 POINT(37.857956844666305 -122.39258141535676) bank172977 +172978 POINT(37.128660452343 -121.5022770076385) bank172978 +172979 POINT(38.16459682546742 -122.8813542248722) bank172979 +172980 POINT(37.56923761350073 -121.89497922883724) bank172980 +172981 POINT(38.55213007048457 -121.475098106825) bank172981 +172982 POINT(36.801489654459196 -122.22767752175983) bank172982 +172983 POINT(37.549741807732886 -121.5162671551605) bank172983 +172984 POINT(37.77476495767528 -122.72664224602917) bank172984 +172985 POINT(38.47517154213139 -122.26292524548477) bank172985 +172986 POINT(37.96497858857041 -121.6898184936501) bank172986 +172987 POINT(37.819586830453005 -121.68521949520131) bank172987 +172988 POINT(37.7017506370067 -123.01325898410308) bank172988 +172989 POINT(37.69094967915543 -122.59634884365482) bank172989 +172990 POINT(37.54112790502858 -121.82091452978638) bank172990 +172991 POINT(38.258385259413636 -121.64603230278921) bank172991 +172992 POINT(37.83010530576607 -121.63879408431185) bank172992 +172993 POINT(38.075141282531206 -122.33901835967198) bank172993 +172994 POINT(36.95006016161402 -122.58857775940463) bank172994 +172995 POINT(38.737792406851916 -122.48412253069013) bank172995 +172996 POINT(38.5913110740011 -122.15786004327134) bank172996 +172997 POINT(38.376357568565076 -122.16548733931583) bank172997 +172998 POINT(37.47563103298032 -123.06734729626936) bank172998 +172999 POINT(37.90724493614616 -122.16435234446708) bank172999 +173000 POINT(38.24781619697944 -122.95141246264421) bank173000 +173001 POINT(37.62020773310448 -122.49869224314773) bank173001 +173002 POINT(36.79834559337602 -121.48711755832566) bank173002 +173003 POINT(37.99995646325391 -122.20947244597224) bank173003 +173004 POINT(37.39747785700539 -121.89400173519928) bank173004 +173005 POINT(37.630991721118164 -122.06649423363305) bank173005 +173006 POINT(36.8888448643205 -122.43884715668065) bank173006 +173007 POINT(38.450124249749955 -121.51549576468014) bank173007 +173008 POINT(37.26002361211977 -122.66674718526959) bank173008 +173009 POINT(38.71990996196167 -122.85096426630776) bank173009 +173010 POINT(38.01933401735323 -121.75693509291943) bank173010 +173011 POINT(37.921377506799104 -122.23538413147712) bank173011 +173012 POINT(36.841670712604646 -122.59003367052412) bank173012 +173013 POINT(36.976327680502365 -121.42228661834258) bank173013 +173014 POINT(37.6417534370147 -121.70881378811507) bank173014 +173015 POINT(37.327708332254595 -122.95168314365458) bank173015 +173016 POINT(37.35000419173886 -121.58063795063345) bank173016 +173017 POINT(37.7020297121165 -122.43908061822853) bank173017 +173018 POINT(38.207337809974135 -122.419468172646) bank173018 +173019 POINT(37.22596382129305 -122.9030491492678) bank173019 +173020 POINT(38.417160488807184 -122.05952069381685) bank173020 +173021 POINT(38.64233601190098 -123.13075470231594) bank173021 +173022 POINT(37.29463526074208 -122.35130368583897) bank173022 +173023 POINT(38.12084542251512 -122.82294115644312) bank173023 +173024 POINT(38.17166967462334 -123.05793024023865) bank173024 +173025 POINT(37.01910166658887 -123.21100597027397) bank173025 +173026 POINT(37.60009863885351 -123.25270290480408) bank173026 +173027 POINT(37.69523664025186 -122.23223608481902) bank173027 +173028 POINT(37.71066350931792 -122.67299259604194) bank173028 +173029 POINT(36.7926034823301 -123.10645548701378) bank173029 +173030 POINT(36.80731492433764 -122.97253384280808) bank173030 +173031 POINT(38.292972344003246 -121.55626468914089) bank173031 +173032 POINT(37.31237986496198 -121.5187922268708) bank173032 +173033 POINT(37.230279608226056 -122.52275559263848) bank173033 +173034 POINT(38.13350672645547 -122.54703108436061) bank173034 +173035 POINT(37.576995357954566 -122.4149776262045) bank173035 +173036 POINT(38.30578886343628 -122.69104424331093) bank173036 +173037 POINT(36.97873681156649 -122.45211110279317) bank173037 +173038 POINT(37.051454993572996 -121.47363432311626) bank173038 +173039 POINT(38.1470154673718 -121.80980584830075) bank173039 +173040 POINT(37.85056577615087 -122.10651577727252) bank173040 +173041 POINT(38.28713059670122 -121.44151377158758) bank173041 +173042 POINT(37.97870892303405 -122.71876801375775) bank173042 +173043 POINT(38.59733972729576 -121.50683693619825) bank173043 +173044 POINT(38.13661467891667 -121.53880360818616) bank173044 +173045 POINT(37.42477357885134 -123.35213849262394) bank173045 +173046 POINT(38.14574011769389 -123.16297697653137) bank173046 +173047 POINT(37.27660431971917 -123.27286918819547) bank173047 +173048 POINT(38.4407382656742 -122.08033092195544) bank173048 +173049 POINT(38.20533355109116 -122.09972946053267) bank173049 +173050 POINT(37.35871915083198 -121.54575000578293) bank173050 +173051 POINT(37.715215631283904 -123.02297913611913) bank173051 +173052 POINT(37.513280862337126 -122.66657659430835) bank173052 +173053 POINT(38.513536464148366 -122.84276136693873) bank173053 +173054 POINT(38.036736615535844 -121.6025932785172) bank173054 +173055 POINT(37.39061688338005 -121.55659209824394) bank173055 +173056 POINT(37.48463142718774 -123.0478193824069) bank173056 +173057 POINT(37.47831001089378 -122.71224864446135) bank173057 +173058 POINT(36.984178963757934 -121.65447048492715) bank173058 +173059 POINT(36.83140361173953 -123.0088171161382) bank173059 +173060 POINT(36.79000404657287 -123.07619490347604) bank173060 +173061 POINT(36.976460872658656 -121.4814230017636) bank173061 +173062 POINT(38.33861405141904 -122.30219719811559) bank173062 +173063 POINT(37.24094404809696 -121.82963750964358) bank173063 +173064 POINT(37.28054653041918 -121.46215224107897) bank173064 +173065 POINT(37.429222273608815 -122.62050793698687) bank173065 +173066 POINT(37.2312693493736 -121.94420938477359) bank173066 +173067 POINT(36.9320407220945 -122.08568062854265) bank173067 +173068 POINT(37.78857507481447 -121.43418666344901) bank173068 +173069 POINT(38.20987999167215 -122.24447078100638) bank173069 +173070 POINT(38.624769556734876 -122.97405649940882) bank173070 +173071 POINT(38.47046441536096 -122.66058173035248) bank173071 +173072 POINT(36.8610764103451 -123.15834950718279) bank173072 +173073 POINT(37.258198135746 -123.09684843788418) bank173073 +173074 POINT(38.70788882625268 -121.55711235498418) bank173074 +173075 POINT(37.228189038492644 -122.4411783543836) bank173075 +173076 POINT(37.19983611769323 -123.25249116450654) bank173076 +173077 POINT(38.657807488656616 -123.34744274728709) bank173077 +173078 POINT(37.444508414086776 -121.88706065081718) bank173078 +173079 POINT(37.02997472691523 -121.92778142312733) bank173079 +173080 POINT(37.907825132075935 -122.57332271026715) bank173080 +173081 POINT(36.90056469414235 -122.25422036048386) bank173081 +173082 POINT(37.96121453027408 -121.85682999116995) bank173082 +173083 POINT(38.60760307948462 -122.95672243313638) bank173083 +173084 POINT(37.87083296478095 -122.569445257475) bank173084 +173085 POINT(37.96163016377136 -122.0431232879867) bank173085 +173086 POINT(36.98169654445459 -123.06808359990467) bank173086 +173087 POINT(38.762625471115925 -122.18233605085348) bank173087 +173088 POINT(37.42423916257918 -121.89914159771261) bank173088 +173089 POINT(38.11434212271924 -122.63234474971195) bank173089 +173090 POINT(38.43030490137134 -122.19934798529553) bank173090 +173091 POINT(38.30377695060473 -122.16532610841668) bank173091 +173092 POINT(38.216137310207785 -123.40354812389695) bank173092 +173093 POINT(38.42209411039334 -121.48361544853306) bank173093 +173094 POINT(37.19331457971762 -121.98732570932503) bank173094 +173095 POINT(38.201536425758626 -122.19539584096205) bank173095 +173096 POINT(38.60187920109245 -122.06291918383126) bank173096 +173097 POINT(37.444538584856325 -122.49556571677904) bank173097 +173098 POINT(37.22434247884596 -121.51651431346605) bank173098 +173099 POINT(37.26029539237142 -122.58856408444068) bank173099 +173100 POINT(37.17180325562557 -122.8257119233222) bank173100 +173101 POINT(37.65866008848392 -121.76277293838439) bank173101 +173102 POINT(38.25255961381397 -122.40972847962887) bank173102 +173103 POINT(38.67416547666287 -122.97085474517766) bank173103 +173104 POINT(37.772648445438335 -123.40210586335854) bank173104 +173105 POINT(37.09102406228803 -122.97519795186538) bank173105 +173106 POINT(37.347514920085366 -121.97110936983239) bank173106 +173107 POINT(38.148926496976564 -121.74890734673677) bank173107 +173108 POINT(37.17718083029113 -122.50160852804323) bank173108 +173109 POINT(37.07991489944546 -122.5645540466551) bank173109 +173110 POINT(38.079445361706924 -122.88544497727983) bank173110 +173111 POINT(38.316398025561064 -123.3532969446082) bank173111 +173112 POINT(38.08673099300291 -122.46518311850275) bank173112 +173113 POINT(38.322060141081046 -122.66282605692743) bank173113 +173114 POINT(37.26540683395759 -121.81471266659223) bank173114 +173115 POINT(37.817017009761905 -123.20855481035008) bank173115 +173116 POINT(36.92644110838613 -122.3015279444876) bank173116 +173117 POINT(37.75708204904142 -123.15240217258989) bank173117 +173118 POINT(38.12487057436116 -121.63023701746948) bank173118 +173119 POINT(37.062612613830126 -122.03732648084413) bank173119 +173120 POINT(36.88603804139532 -123.36427775186613) bank173120 +173121 POINT(37.23391156581284 -122.81234815558511) bank173121 +173122 POINT(36.994640281981745 -123.37993441745225) bank173122 +173123 POINT(37.47956859678558 -122.29949866319629) bank173123 +173124 POINT(38.266342723035805 -123.11819098139573) bank173124 +173125 POINT(36.83568372866985 -122.03387056332205) bank173125 +173126 POINT(37.589742501645226 -122.74618029687423) bank173126 +173127 POINT(38.10469462095226 -121.42254442381429) bank173127 +173128 POINT(38.08321516141214 -123.26817189266221) bank173128 +173129 POINT(38.540865432594224 -121.92872182602515) bank173129 +173130 POINT(36.79960939072914 -121.57878226520717) bank173130 +173131 POINT(37.474423931834885 -121.7071605641468) bank173131 +173132 POINT(37.143331020086855 -121.93764412859333) bank173132 +173133 POINT(38.288835604212075 -122.59259048421455) bank173133 +173134 POINT(38.137335602003 -122.27603962564991) bank173134 +173135 POINT(37.46565617531231 -121.79177484803306) bank173135 +173136 POINT(36.99251025784092 -121.85676123331228) bank173136 +173137 POINT(37.0727393120063 -122.79251628141503) bank173137 +173138 POINT(37.958340039766654 -121.92350644324738) bank173138 +173139 POINT(37.19231135114419 -122.24402559789516) bank173139 +173140 POINT(37.46437212253568 -123.1073410879135) bank173140 +173141 POINT(36.812088115646986 -121.62256494649225) bank173141 +173142 POINT(36.88481886845307 -121.67501852167105) bank173142 +173143 POINT(38.518216062489984 -122.24550248914076) bank173143 +173144 POINT(38.69183874583455 -122.75315241003247) bank173144 +173145 POINT(37.04858196946447 -122.35038124210782) bank173145 +173146 POINT(37.10363146086899 -122.94509412281724) bank173146 +173147 POINT(38.75646871087069 -122.2193702371004) bank173147 +173148 POINT(36.860669382678715 -123.19118042203645) bank173148 +173149 POINT(37.6262972098698 -122.47639720918536) bank173149 +173150 POINT(37.87023097382925 -123.26845419736158) bank173150 +173151 POINT(36.95753342259139 -121.61240171308779) bank173151 +173152 POINT(37.621710256588926 -121.65212588024538) bank173152 +173153 POINT(37.027655370390846 -122.39537335574727) bank173153 +173154 POINT(36.78520296154713 -121.93912088446554) bank173154 +173155 POINT(36.86953670377689 -123.29507080379918) bank173155 +173156 POINT(37.913747958101865 -121.85467721576718) bank173156 +173157 POINT(37.96007517203251 -122.56409628070635) bank173157 +173158 POINT(38.36857489022518 -121.79253907151431) bank173158 +173159 POINT(38.68230047921651 -122.41356685745143) bank173159 +173160 POINT(37.85775214442753 -121.61970018689888) bank173160 +173161 POINT(37.751317440047 -121.73358100053142) bank173161 +173162 POINT(37.88412666507991 -122.53722930357361) bank173162 +173163 POINT(36.86353907039769 -122.20592308508375) bank173163 +173164 POINT(37.49641315975263 -121.84208204787839) bank173164 +173165 POINT(37.775152253333346 -122.85809645266808) bank173165 +173166 POINT(37.627841893112276 -121.77880185800738) bank173166 +173167 POINT(38.632796075243384 -122.26823547254425) bank173167 +173168 POINT(37.30468077549189 -122.39837265219022) bank173168 +173169 POINT(37.582147694625924 -123.24044374240344) bank173169 +173170 POINT(37.5848461268771 -122.46722447489076) bank173170 +173171 POINT(37.78205493766692 -123.21119871683146) bank173171 +173172 POINT(38.54484788039912 -122.29223327285297) bank173172 +173173 POINT(37.146282818671025 -123.29167220714614) bank173173 +173174 POINT(38.77065593865542 -122.55912865913903) bank173174 +173175 POINT(38.50628416408194 -122.97932273633903) bank173175 +173176 POINT(38.60301930264849 -123.32569667301834) bank173176 +173177 POINT(37.435612111736134 -121.52869503790595) bank173177 +173178 POINT(37.98370664455583 -122.895261668277) bank173178 +173179 POINT(38.556305986374824 -121.50780797377692) bank173179 +173180 POINT(38.535341436443424 -122.40439968064693) bank173180 +173181 POINT(37.869431607506776 -123.15109075342949) bank173181 +173182 POINT(37.73074070675744 -123.16151252442701) bank173182 +173183 POINT(37.96740872419229 -121.97818592175302) bank173183 +173184 POINT(37.594437015773224 -122.68678995816654) bank173184 +173185 POINT(38.335639701791564 -121.8788140430405) bank173185 +173186 POINT(38.63675954397144 -122.07524770793138) bank173186 +173187 POINT(38.21297779260782 -123.16057045140496) bank173187 +173188 POINT(37.21282319771193 -121.86379644822773) bank173188 +173189 POINT(37.28727284010821 -123.00928246545668) bank173189 +173190 POINT(37.73222212709783 -122.4125280039189) bank173190 +173191 POINT(37.05615905337888 -123.01051756983755) bank173191 +173192 POINT(37.64670035384176 -122.53224036281533) bank173192 +173193 POINT(37.15582326641049 -122.78340055330617) bank173193 +173194 POINT(37.07418700308295 -122.24968407844742) bank173194 +173195 POINT(37.02366253086027 -123.15921782771446) bank173195 +173196 POINT(37.94435076103827 -123.0497771122144) bank173196 +173197 POINT(37.52425124973959 -123.05246475671555) bank173197 +173198 POINT(37.22925713653142 -123.29148278584486) bank173198 +173199 POINT(37.39420264593103 -122.85489747359247) bank173199 +173200 POINT(38.49915449607462 -123.36995348923408) bank173200 +173201 POINT(37.497045164269416 -121.72654859346682) bank173201 +173202 POINT(38.33837318645346 -123.04227041562085) bank173202 +173203 POINT(37.601453322164105 -123.1579840641963) bank173203 +173204 POINT(38.12123084765445 -122.59716477653389) bank173204 +173205 POINT(37.10112295372586 -123.35974826550192) bank173205 +173206 POINT(38.34719490083691 -121.78652012327764) bank173206 +173207 POINT(37.012905200420434 -123.39114206866543) bank173207 +173208 POINT(38.42381173636744 -122.47935001949199) bank173208 +173209 POINT(37.40961640451646 -122.147106878865) bank173209 +173210 POINT(38.0870393760417 -121.4826320178539) bank173210 +173211 POINT(37.69343444520849 -121.85612751732012) bank173211 +173212 POINT(37.028876557602175 -122.35261251315599) bank173212 +173213 POINT(38.64588088676651 -122.07009913178572) bank173213 +173214 POINT(36.90758795224677 -122.04319141348017) bank173214 +173215 POINT(37.37039827007469 -122.91647990973219) bank173215 +173216 POINT(38.56618042312384 -122.61713782324837) bank173216 +173217 POINT(37.681213988780335 -121.70720037531152) bank173217 +173218 POINT(38.17173971817656 -123.02830587414118) bank173218 +173219 POINT(37.79368994643352 -121.53994579317128) bank173219 +173220 POINT(37.4526291451259 -123.28497469964066) bank173220 +173221 POINT(38.010771969488935 -122.58300307442848) bank173221 +173222 POINT(38.25344649860682 -122.64404705581128) bank173222 +173223 POINT(36.824006166764356 -122.93259590440485) bank173223 +173224 POINT(36.884687639517374 -121.59386359025756) bank173224 +173225 POINT(36.87731499121878 -122.68457093910678) bank173225 +173226 POINT(37.01915507361403 -122.13098441172492) bank173226 +173227 POINT(37.069026048455555 -121.49067225840065) bank173227 +173228 POINT(38.29951347133942 -122.50065496424799) bank173228 +173229 POINT(38.487576754338804 -122.99356746343926) bank173229 +173230 POINT(38.26844154929596 -122.16072606273094) bank173230 +173231 POINT(38.44466779961114 -123.2830744523595) bank173231 +173232 POINT(38.273577924599806 -122.92976865563254) bank173232 +173233 POINT(36.79461802471005 -121.79960170579113) bank173233 +173234 POINT(38.755433760750606 -121.58668520966019) bank173234 +173235 POINT(37.580232264507934 -121.42946703813567) bank173235 +173236 POINT(37.02783396090128 -123.29758282469794) bank173236 +173237 POINT(38.07079477127776 -123.29299478179857) bank173237 +173238 POINT(36.786073654935954 -123.11867364778269) bank173238 +173239 POINT(37.11414763949674 -123.04370265745881) bank173239 +173240 POINT(37.283350393303564 -122.46644520961345) bank173240 +173241 POINT(38.606111762115866 -122.11841881359331) bank173241 +173242 POINT(38.72060042651483 -122.06079013615027) bank173242 +173243 POINT(38.34474097328079 -121.46992255423545) bank173243 +173244 POINT(37.27814013403312 -121.56253674929116) bank173244 +173245 POINT(37.048002158529336 -123.29072269336427) bank173245 +173246 POINT(37.979462337711105 -122.475506069379) bank173246 +173247 POINT(37.890128037727415 -123.23225040636082) bank173247 +173248 POINT(37.48949639272971 -122.12530686529205) bank173248 +173249 POINT(37.39918817863254 -122.40503494571105) bank173249 +173250 POINT(37.404277106147404 -121.74213079003316) bank173250 +173251 POINT(38.7489390315543 -122.59745492978882) bank173251 +173252 POINT(37.16159094999109 -122.8941018900289) bank173252 +173253 POINT(38.76398437090745 -123.11563722553343) bank173253 +173254 POINT(37.32826254586116 -122.52432183619267) bank173254 +173255 POINT(36.97435787907655 -122.52822559969586) bank173255 +173256 POINT(38.75374390615484 -123.25140883806759) bank173256 +173257 POINT(37.84048159433819 -121.9628451130937) bank173257 +173258 POINT(37.77082713756989 -122.47852910176111) bank173258 +173259 POINT(38.28886224083598 -122.42177537093335) bank173259 +173260 POINT(37.313269954303806 -122.06260885157556) bank173260 +173261 POINT(36.867159548462595 -122.7480135650626) bank173261 +173262 POINT(38.4399139916131 -123.39621599413658) bank173262 +173263 POINT(38.75705276760087 -121.57635141738574) bank173263 +173264 POINT(37.554942829474115 -122.28580064384542) bank173264 +173265 POINT(38.16855091561409 -121.70365463238865) bank173265 +173266 POINT(38.435159379731545 -122.37624637194662) bank173266 +173267 POINT(37.015685050506725 -123.03385594214168) bank173267 +173268 POINT(36.83992073446157 -122.83408372472013) bank173268 +173269 POINT(37.9335117418292 -122.59548512942636) bank173269 +173270 POINT(37.71046422724333 -121.90604869348687) bank173270 +173271 POINT(38.46028430569616 -121.87880508476987) bank173271 +173272 POINT(37.65163595397674 -121.6622393552248) bank173272 +173273 POINT(36.82394970136609 -122.56556352740523) bank173273 +173274 POINT(36.82409343411677 -121.69646961043627) bank173274 +173275 POINT(37.98866508810149 -123.35891085521635) bank173275 +173276 POINT(38.41029191489727 -123.25048384402278) bank173276 +173277 POINT(38.20621787409239 -123.20084437897117) bank173277 +173278 POINT(37.28987015576946 -122.43509496369022) bank173278 +173279 POINT(37.51903308435886 -123.02595188201697) bank173279 +173280 POINT(37.207977173960806 -122.47463737275163) bank173280 +173281 POINT(37.648530046015715 -122.28609975799843) bank173281 +173282 POINT(37.95498599840449 -123.1759728179964) bank173282 +173283 POINT(38.468004068241584 -122.81406096278343) bank173283 +173284 POINT(37.55330859826583 -123.05921656886095) bank173284 +173285 POINT(38.75114258899433 -121.52755683214043) bank173285 +173286 POINT(36.89488420817663 -122.67406777983064) bank173286 +173287 POINT(37.8078715306939 -121.94407994532153) bank173287 +173288 POINT(37.16304523412312 -121.44709288374091) bank173288 +173289 POINT(38.532879920998795 -122.36798339559311) bank173289 +173290 POINT(36.98590565769422 -122.7705934476597) bank173290 +173291 POINT(37.17438526092752 -123.37885937841767) bank173291 +173292 POINT(37.071572952798746 -123.20807342876586) bank173292 +173293 POINT(37.740837549718044 -122.95641619826912) bank173293 +173294 POINT(38.0318006369335 -122.44184600770478) bank173294 +173295 POINT(37.99148956732602 -122.39417036729576) bank173295 +173296 POINT(38.20489345376012 -122.50373932207732) bank173296 +173297 POINT(37.335882634547275 -123.32174374998871) bank173297 +173298 POINT(38.62202172317903 -122.42826714974558) bank173298 +173299 POINT(37.954536740775026 -121.73422286404035) bank173299 +173300 POINT(37.389664175036366 -121.8787370458345) bank173300 +173301 POINT(37.258496356957366 -122.1416656517914) bank173301 +173302 POINT(37.46039784167639 -122.09355922727565) bank173302 +173303 POINT(38.10391477670913 -123.02338470455692) bank173303 +173304 POINT(37.697657279844314 -121.86279757353476) bank173304 +173305 POINT(38.47338183308863 -121.81613021716205) bank173305 +173306 POINT(37.09544070144866 -121.6395561809064) bank173306 +173307 POINT(38.74664138970454 -121.82425486360437) bank173307 +173308 POINT(37.95379423298264 -122.54335252034419) bank173308 +173309 POINT(37.52582929197621 -122.20063575200355) bank173309 +173310 POINT(38.67433676396716 -122.35179112052569) bank173310 +173311 POINT(37.986883457264426 -122.02762279820959) bank173311 +173312 POINT(36.84786333296691 -121.76987361517064) bank173312 +173313 POINT(38.605904439619124 -123.27570203668637) bank173313 +173314 POINT(38.36241439421876 -122.89598868099145) bank173314 +173315 POINT(37.76341200903924 -122.89090498616319) bank173315 +173316 POINT(36.98061277816473 -121.76875719704346) bank173316 +173317 POINT(38.31205372697852 -121.46967932267542) bank173317 +173318 POINT(38.67106506753272 -123.15706824718546) bank173318 +173319 POINT(37.72912048489442 -121.49845538716649) bank173319 +173320 POINT(38.079239247193115 -122.42979902496056) bank173320 +173321 POINT(37.10697199173275 -122.16425869722107) bank173321 +173322 POINT(37.18506814933662 -123.41264793239077) bank173322 +173323 POINT(37.21112608296258 -123.07733927691118) bank173323 +173324 POINT(37.265303615906596 -122.4900620703777) bank173324 +173325 POINT(37.19370591808133 -121.4562830167502) bank173325 +173326 POINT(38.04697029149241 -122.20377559371488) bank173326 +173327 POINT(36.97924084256574 -122.02043406279526) bank173327 +173328 POINT(36.9666119628783 -123.15596372552045) bank173328 +173329 POINT(38.685265303411015 -122.77899585417104) bank173329 +173330 POINT(38.618460659877115 -122.5899918158173) bank173330 +173331 POINT(36.82756829938628 -122.76664008938893) bank173331 +173332 POINT(38.67398178747142 -122.23327667071362) bank173332 +173333 POINT(38.16794929497016 -122.13628613983359) bank173333 +173334 POINT(36.941626229359194 -122.7857614173848) bank173334 +173335 POINT(36.994982473978595 -121.54207804021608) bank173335 +173336 POINT(37.09187783834509 -123.04933295923036) bank173336 +173337 POINT(38.34358140156904 -122.97216260764586) bank173337 +173338 POINT(36.79030525312471 -122.95043615203254) bank173338 +173339 POINT(38.2900568193874 -122.0248408124446) bank173339 +173340 POINT(37.11663230307833 -122.97020555334053) bank173340 +173341 POINT(37.23406996861693 -123.34324344001314) bank173341 +173342 POINT(38.2442296579371 -122.08477574879389) bank173342 +173343 POINT(38.21971205738658 -122.92126335957468) bank173343 +173344 POINT(37.312102043909256 -121.95405138961881) bank173344 +173345 POINT(38.2187596441166 -123.14446286889219) bank173345 +173346 POINT(37.85751727312731 -121.75833661608684) bank173346 +173347 POINT(37.106627067281536 -122.51942083538917) bank173347 +173348 POINT(37.09813226520054 -123.37591328467848) bank173348 +173349 POINT(38.58045752202932 -122.11410613237506) bank173349 +173350 POINT(37.469681613218725 -123.32963673549794) bank173350 +173351 POINT(37.56050234294457 -122.96183786328672) bank173351 +173352 POINT(37.45390237282141 -121.67427096205496) bank173352 +173353 POINT(36.98733634904053 -122.96985472316649) bank173353 +173354 POINT(37.140827249589805 -121.7783519981649) bank173354 +173355 POINT(37.51123789622305 -122.38390749581566) bank173355 +173356 POINT(37.78187304613987 -122.97641649467427) bank173356 +173357 POINT(37.20643738365962 -122.23807475423537) bank173357 +173358 POINT(36.993628255045564 -122.51936293344076) bank173358 +173359 POINT(37.85023677121108 -122.47166507713389) bank173359 +173360 POINT(37.564515775128335 -122.47267688895094) bank173360 +173361 POINT(38.546845061728305 -122.90847196140136) bank173361 +173362 POINT(38.328197321471585 -121.5515007412779) bank173362 +173363 POINT(38.128280256268326 -121.60411519257586) bank173363 +173364 POINT(37.1115497899736 -123.00745659328201) bank173364 +173365 POINT(38.59911174437668 -123.0383363638114) bank173365 +173366 POINT(36.99747073839576 -123.4104888404679) bank173366 +173367 POINT(38.75962005934699 -121.75932021696076) bank173367 +173368 POINT(38.76599574579831 -121.58316106535293) bank173368 +173369 POINT(38.68403480973494 -121.79898858839628) bank173369 +173370 POINT(38.59816524762138 -123.25640725568515) bank173370 +173371 POINT(38.68757714164503 -123.29218302222971) bank173371 +173372 POINT(38.34375139420289 -122.6698553996718) bank173372 +173373 POINT(38.288278328322974 -121.69616362209325) bank173373 +173374 POINT(37.71255604448743 -121.50315547684545) bank173374 +173375 POINT(37.02789477720246 -123.21483302261203) bank173375 +173376 POINT(38.00978076734504 -121.96955287585546) bank173376 +173377 POINT(37.572934520144734 -122.53220087956254) bank173377 +173378 POINT(37.459912655061345 -122.228436640202) bank173378 +173379 POINT(37.04519141271732 -122.66722246474258) bank173379 +173380 POINT(37.025463438912716 -122.48421165692623) bank173380 +173381 POINT(37.21009907930964 -123.1894425386883) bank173381 +173382 POINT(37.74251279806883 -121.56831946307268) bank173382 +173383 POINT(37.158419141490604 -122.56329869515335) bank173383 +173384 POINT(37.27826756553638 -123.08477802279702) bank173384 +173385 POINT(38.43453461973072 -122.40826545198871) bank173385 +173386 POINT(37.25356131385373 -121.63351223417375) bank173386 +173387 POINT(37.08447220171376 -121.98778156300968) bank173387 +173388 POINT(36.898057395471355 -122.00159334458161) bank173388 +173389 POINT(37.7254359798828 -121.57324142791381) bank173389 +173390 POINT(36.80780451448804 -121.77463993860744) bank173390 +173391 POINT(36.85124378060448 -122.90042168097331) bank173391 +173392 POINT(38.3756388334193 -121.75234452426837) bank173392 +173393 POINT(38.72752382329065 -123.06427177078827) bank173393 +173394 POINT(38.24269801819544 -123.07266560150619) bank173394 +173395 POINT(36.82553001429425 -121.44115457906253) bank173395 +173396 POINT(38.05550792335555 -122.74313145708697) bank173396 +173397 POINT(37.929730298558745 -122.85300284785909) bank173397 +173398 POINT(37.79123997965991 -122.42093826826644) bank173398 +173399 POINT(36.936218107844454 -123.14098751654949) bank173399 +173400 POINT(37.555132475971874 -121.74913175925407) bank173400 +173401 POINT(37.765208138218384 -123.22452425716767) bank173401 +173402 POINT(37.318138521114264 -122.12402103184094) bank173402 +173403 POINT(38.56715376848528 -123.09635744054492) bank173403 +173404 POINT(38.690119443753424 -121.57484164476104) bank173404 +173405 POINT(38.35567777223088 -123.12440104762352) bank173405 +173406 POINT(38.0120169943041 -123.15455878860598) bank173406 +173407 POINT(37.61900126595141 -121.88341737250177) bank173407 +173408 POINT(38.648832337401885 -121.91141085274118) bank173408 +173409 POINT(36.95758071497373 -122.17980400657568) bank173409 +173410 POINT(37.71781554676285 -122.10075112874708) bank173410 +173411 POINT(38.37132073016583 -123.37245648462317) bank173411 +173412 POINT(37.55340627778053 -122.71266850090612) bank173412 +173413 POINT(37.07827862389955 -122.8274365249778) bank173413 +173414 POINT(38.23108044599947 -121.77288961030585) bank173414 +173415 POINT(37.16426733417416 -122.30160970494656) bank173415 +173416 POINT(37.10566955745422 -122.01638294891471) bank173416 +173417 POINT(38.74160411917395 -121.81770729610011) bank173417 +173418 POINT(38.66198843298975 -121.75139267471211) bank173418 +173419 POINT(37.30525562690984 -121.770698920516) bank173419 +173420 POINT(38.090243549089294 -122.65250848802556) bank173420 +173421 POINT(37.577687529722596 -122.46519312239946) bank173421 +173422 POINT(37.78672730930755 -121.84090300952424) bank173422 +173423 POINT(37.499830055061935 -122.4446905749779) bank173423 +173424 POINT(38.242422630087724 -122.5321105526936) bank173424 +173425 POINT(36.93578635670921 -123.30082694287964) bank173425 +173426 POINT(37.41766388896475 -121.72542882710077) bank173426 +173427 POINT(37.18150227873871 -122.34744404467867) bank173427 +173428 POINT(38.02750471622269 -122.69799131734591) bank173428 +173429 POINT(37.9505119649934 -122.82752808963275) bank173429 +173430 POINT(37.88345943769881 -122.35083131646027) bank173430 +173431 POINT(37.15672771249591 -122.7938728503161) bank173431 +173432 POINT(38.14547258438838 -122.03065373846891) bank173432 +173433 POINT(38.70474751787667 -122.90890450507607) bank173433 +173434 POINT(38.76719654852102 -123.27992169617187) bank173434 +173435 POINT(38.354582257207696 -121.57857289638856) bank173435 +173436 POINT(38.52080223699467 -122.0590151707374) bank173436 +173437 POINT(37.04156682516635 -122.39102277873272) bank173437 +173438 POINT(38.560810736254574 -121.61531011314054) bank173438 +173439 POINT(38.13306518933396 -123.37152840526585) bank173439 +173440 POINT(38.62132292029769 -123.09438544314068) bank173440 +173441 POINT(38.029794796508995 -122.78276823942123) bank173441 +173442 POINT(37.70852512399231 -122.74226970215041) bank173442 +173443 POINT(38.431976305204394 -121.97367052784452) bank173443 +173444 POINT(37.4012000872393 -122.13164828162733) bank173444 +173445 POINT(37.40464785140383 -121.81597493542553) bank173445 +173446 POINT(37.96449083553984 -122.7933709707442) bank173446 +173447 POINT(38.161560288296876 -122.6356170573336) bank173447 +173448 POINT(37.58976201877597 -122.3269803186455) bank173448 +173449 POINT(38.60663014275057 -123.14983555938612) bank173449 +173450 POINT(38.48787294283907 -122.49160923979427) bank173450 +173451 POINT(36.90201982632121 -122.63357595436591) bank173451 +173452 POINT(38.764091529195454 -122.30700240496) bank173452 +173453 POINT(38.05178610506052 -122.97252288538374) bank173453 +173454 POINT(37.398702988545445 -123.25355343926391) bank173454 +173455 POINT(37.25020116963275 -121.64525558969162) bank173455 +173456 POINT(37.18943549578995 -122.50525500962048) bank173456 +173457 POINT(36.993889512475434 -121.92681330682626) bank173457 +173458 POINT(37.670162838694196 -123.01610798488855) bank173458 +173459 POINT(37.925999596276704 -121.94916463978146) bank173459 +173460 POINT(37.849153789889534 -121.67036721704724) bank173460 +173461 POINT(37.704857251753516 -121.99034611643361) bank173461 +173462 POINT(38.37297689948645 -122.39153316586449) bank173462 +173463 POINT(38.23282312495867 -122.04462312271681) bank173463 +173464 POINT(37.53247809275302 -122.62587409828423) bank173464 +173465 POINT(38.389014672439615 -122.83065827279927) bank173465 +173466 POINT(37.136980395640684 -123.30728572758031) bank173466 +173467 POINT(37.929917313756135 -122.48266002412788) bank173467 +173468 POINT(37.836629277849944 -121.44939511740134) bank173468 +173469 POINT(38.20338937412511 -121.76664195131633) bank173469 +173470 POINT(37.70388735897443 -121.94018783478577) bank173470 +173471 POINT(37.33695965444213 -123.31313299239103) bank173471 +173472 POINT(38.62814825905726 -122.10108225012942) bank173472 +173473 POINT(38.414090579231555 -123.25439156595449) bank173473 +173474 POINT(38.59583479719233 -122.99257108231984) bank173474 +173475 POINT(38.03416064390325 -121.5047256126583) bank173475 +173476 POINT(37.5950407825315 -121.77180151769079) bank173476 +173477 POINT(38.05749115792173 -122.05414455469159) bank173477 +173478 POINT(37.59345390648849 -122.27819725083974) bank173478 +173479 POINT(37.901335641675466 -123.12232333245113) bank173479 +173480 POINT(37.988331355749 -122.6715948550313) bank173480 +173481 POINT(37.34835604947389 -122.8960962437061) bank173481 +173482 POINT(38.45091870338294 -122.57833625103932) bank173482 +173483 POINT(38.576258709016884 -122.25309051175297) bank173483 +173484 POINT(37.9318666942388 -122.16221259510868) bank173484 +173485 POINT(38.242545262805066 -121.55985178183728) bank173485 +173486 POINT(37.88447988614958 -123.29148568499531) bank173486 +173487 POINT(37.73489538287478 -121.52364704898652) bank173487 +173488 POINT(37.52373955984115 -121.91908370309453) bank173488 +173489 POINT(38.628370112168646 -122.84662477079357) bank173489 +173490 POINT(38.49928442200518 -122.86202242199063) bank173490 +173491 POINT(38.62869171666647 -121.83300335504721) bank173491 +173492 POINT(38.67460403028703 -121.82852952707789) bank173492 +173493 POINT(38.603285201945134 -122.18254686625745) bank173493 +173494 POINT(37.13402676129978 -121.8835145175037) bank173494 +173495 POINT(38.15302625670426 -123.1571834050268) bank173495 +173496 POINT(37.83800819893248 -122.22663310516683) bank173496 +173497 POINT(37.25209840648189 -122.05679102181256) bank173497 +173498 POINT(37.04825075089195 -122.91559326043887) bank173498 +173499 POINT(38.14808546853828 -122.89881794523141) bank173499 +173500 POINT(38.610096224493695 -122.81020680703173) bank173500 +173501 POINT(37.52625394749398 -123.1397859314863) bank173501 +173502 POINT(37.12562596670258 -123.3900678201004) bank173502 +173503 POINT(38.016839656403434 -121.88088561921968) bank173503 +173504 POINT(37.237015972416906 -121.9132787761138) bank173504 +173505 POINT(37.81900857098426 -122.43212673943876) bank173505 +173506 POINT(37.37715629247143 -121.51942588944043) bank173506 +173507 POINT(37.502766393494696 -122.10549922330691) bank173507 +173508 POINT(37.52597273616914 -122.47768136597372) bank173508 +173509 POINT(38.58231136830241 -121.93582622678785) bank173509 +173510 POINT(38.006985509748986 -122.37753896377805) bank173510 +173511 POINT(38.074848086284824 -121.43762695165204) bank173511 +173512 POINT(37.33701188329415 -121.56998326837855) bank173512 +173513 POINT(38.20298400042436 -122.45895821675894) bank173513 +173514 POINT(36.98702938462171 -121.62406333754016) bank173514 +173515 POINT(37.99697204887251 -122.68593619650254) bank173515 +173516 POINT(37.43368611268362 -122.12194696716627) bank173516 +173517 POINT(37.81412057869225 -121.86271117099484) bank173517 +173518 POINT(38.42162082723182 -122.79680510624263) bank173518 +173519 POINT(37.79942229071248 -123.35309974155557) bank173519 +173520 POINT(37.19963310353835 -121.75935375711077) bank173520 +173521 POINT(37.931543810717855 -121.8497031873522) bank173521 +173522 POINT(38.627662017008696 -121.52208046596081) bank173522 +173523 POINT(38.104256395776815 -121.46917322387719) bank173523 +173524 POINT(37.52539089092094 -123.03762090609231) bank173524 +173525 POINT(37.73203146840246 -122.19603208050279) bank173525 +173526 POINT(38.249957237316025 -122.70537776825427) bank173526 +173527 POINT(37.14725436252645 -121.8753343594813) bank173527 +173528 POINT(37.9628097677721 -122.58880083205744) bank173528 +173529 POINT(37.518609968225796 -121.51141226099901) bank173529 +173530 POINT(37.79775222117424 -123.21263828440816) bank173530 +173531 POINT(38.06026930635387 -122.46237453157543) bank173531 +173532 POINT(37.429595111743396 -121.82679121904651) bank173532 +173533 POINT(37.83732352196772 -123.06932329060771) bank173533 +173534 POINT(37.298252041812844 -122.81416039102172) bank173534 +173535 POINT(38.47793325615199 -121.90821038158013) bank173535 +173536 POINT(37.6145773006655 -122.09288888965814) bank173536 +173537 POINT(37.6623270092294 -122.44701627750638) bank173537 +173538 POINT(37.946170539594476 -122.3492495236176) bank173538 +173539 POINT(38.39467772133974 -121.77566976005976) bank173539 +173540 POINT(37.31348854674533 -122.20458613210675) bank173540 +173541 POINT(37.97127063589073 -123.12705856566049) bank173541 +173542 POINT(38.62690883631134 -122.2729216132665) bank173542 +173543 POINT(38.08603773347349 -122.81999486115869) bank173543 +173544 POINT(37.23476569132783 -122.65682077264341) bank173544 +173545 POINT(38.37854717614143 -122.91710923874876) bank173545 +173546 POINT(38.09826901180349 -121.86027478459289) bank173546 +173547 POINT(37.85089862225592 -122.19887728510207) bank173547 +173548 POINT(37.05698418876094 -121.59987350482815) bank173548 +173549 POINT(38.70941644442715 -122.97021873540146) bank173549 +173550 POINT(38.51795835451616 -121.62083869679816) bank173550 +173551 POINT(38.010652321538345 -122.01616232914137) bank173551 +173552 POINT(37.79614675109548 -122.28984471037361) bank173552 +173553 POINT(37.79897470830419 -123.37057671714614) bank173553 +173554 POINT(38.076462309690925 -121.80346315108737) bank173554 +173555 POINT(37.01305147441187 -122.79330716648953) bank173555 +173556 POINT(37.567475352327286 -123.10782432579524) bank173556 +173557 POINT(36.99580360728569 -123.0131114338522) bank173557 +173558 POINT(37.4374393354349 -121.6729392065353) bank173558 +173559 POINT(37.95779920671666 -121.60070065172835) bank173559 +173560 POINT(37.44921763723093 -122.68292389256601) bank173560 +173561 POINT(38.502778861052576 -121.52999513654652) bank173561 +173562 POINT(37.718248963116146 -121.4928797001466) bank173562 +173563 POINT(38.09555237716266 -122.0787813067882) bank173563 +173564 POINT(38.06501820103402 -122.7860413629686) bank173564 +173565 POINT(38.611520484733425 -122.59173013910936) bank173565 +173566 POINT(38.06210810417482 -121.97968164460126) bank173566 +173567 POINT(38.00274418506785 -121.88345459853507) bank173567 +173568 POINT(37.016249698312706 -123.39973681927142) bank173568 +173569 POINT(36.88397236608127 -122.2641345976495) bank173569 +173570 POINT(37.17894433311036 -121.94939522995196) bank173570 +173571 POINT(38.2106071955007 -122.39311549113458) bank173571 +173572 POINT(37.458867885843944 -123.04150802926546) bank173572 +173573 POINT(37.21152664215788 -121.5172929492247) bank173573 +173574 POINT(38.56948478573177 -123.11179457101947) bank173574 +173575 POINT(38.755719137677204 -121.88672744341139) bank173575 +173576 POINT(37.503818993786616 -121.43566156756206) bank173576 +173577 POINT(37.4748883546704 -122.9260139696805) bank173577 +173578 POINT(37.4551160436493 -122.87715664662066) bank173578 +173579 POINT(37.820781562307374 -121.69616576544433) bank173579 +173580 POINT(37.868619718300415 -121.57147379272995) bank173580 +173581 POINT(38.4396940269684 -122.06485148006742) bank173581 +173582 POINT(37.09832315582758 -122.44537959482876) bank173582 +173583 POINT(37.74931023220675 -121.61578526721185) bank173583 +173584 POINT(38.11507624388119 -122.8805483272771) bank173584 +173585 POINT(37.41447394366085 -121.49159599454174) bank173585 +173586 POINT(38.703950166612344 -122.89722264837826) bank173586 +173587 POINT(37.99588820824893 -122.17726320050407) bank173587 +173588 POINT(36.814252881589695 -121.42230035615485) bank173588 +173589 POINT(38.23906899952729 -122.97833064016682) bank173589 +173590 POINT(37.835103320979215 -123.00795038338472) bank173590 +173591 POINT(37.390660575742636 -123.10777650574228) bank173591 +173592 POINT(38.442895072015354 -122.30196067708042) bank173592 +173593 POINT(37.32496952191421 -122.99937119819884) bank173593 +173594 POINT(38.288581327665554 -121.44747518017468) bank173594 +173595 POINT(38.258566331344994 -122.29914016359197) bank173595 +173596 POINT(38.005362599099726 -123.35859714643526) bank173596 +173597 POINT(37.507710913459086 -122.05984044565065) bank173597 +173598 POINT(38.52356837780947 -121.86168180329555) bank173598 +173599 POINT(37.16579222975451 -123.1535564974428) bank173599 +173600 POINT(37.40490032564804 -121.89592095063686) bank173600 +173601 POINT(38.187403099597596 -122.24064091662338) bank173601 +173602 POINT(37.48670603106881 -122.47579800684174) bank173602 +173603 POINT(38.142035087231015 -121.50464368751297) bank173603 +173604 POINT(37.01223632689301 -123.08774108400362) bank173604 +173605 POINT(37.223697130743155 -123.28605713386824) bank173605 +173606 POINT(38.407223573297856 -122.66867640612028) bank173606 +173607 POINT(38.4335866665142 -122.0764519909869) bank173607 +173608 POINT(37.148582849720405 -123.29342063318394) bank173608 +173609 POINT(36.999510798638376 -121.77914131084599) bank173609 +173610 POINT(38.34512105528929 -123.12529409941408) bank173610 +173611 POINT(37.48141231527568 -122.51654172853209) bank173611 +173612 POINT(37.485896895460456 -121.49143431048671) bank173612 +173613 POINT(38.604486526769776 -122.8648396152808) bank173613 +173614 POINT(37.727430479412845 -121.68578941059077) bank173614 +173615 POINT(38.650623989590315 -122.0447497131534) bank173615 +173616 POINT(37.030220966581076 -122.58561644607542) bank173616 +173617 POINT(37.45195745635135 -121.82594932836142) bank173617 +173618 POINT(38.28320739835897 -122.12573500468203) bank173618 +173619 POINT(38.74620990375557 -122.03946215481679) bank173619 +173620 POINT(37.75304159961114 -122.77358772653764) bank173620 +173621 POINT(37.055724837339724 -122.88546164433714) bank173621 +173622 POINT(37.386705253527936 -123.06961019839959) bank173622 +173623 POINT(36.99743224446091 -122.82511934895959) bank173623 +173624 POINT(37.175204525999675 -121.88551036652194) bank173624 +173625 POINT(36.90867898681301 -122.06969595861774) bank173625 +173626 POINT(37.36141172678224 -122.4946956083386) bank173626 +173627 POINT(38.06380429332057 -121.76524534834111) bank173627 +173628 POINT(38.288643799794876 -121.91583624673704) bank173628 +173629 POINT(38.16072093544839 -122.81133739599991) bank173629 +173630 POINT(37.052300091727574 -123.4116411393197) bank173630 +173631 POINT(38.17107901514906 -122.9629364321995) bank173631 +173632 POINT(37.643300501185415 -121.59218677679254) bank173632 +173633 POINT(37.1392140505667 -122.9809634743234) bank173633 +173634 POINT(37.93168126222613 -122.89802968309581) bank173634 +173635 POINT(38.128993325761456 -122.55710777387006) bank173635 +173636 POINT(38.4874289745833 -122.35409096671505) bank173636 +173637 POINT(38.43343308212518 -122.05928585124542) bank173637 +173638 POINT(37.89539672587681 -121.51234518784936) bank173638 +173639 POINT(37.87597034548029 -123.12182094245792) bank173639 +173640 POINT(37.57797085228787 -122.4282840086071) bank173640 +173641 POINT(37.86267883436143 -122.35843810066777) bank173641 +173642 POINT(38.32657105681728 -121.72831529631793) bank173642 +173643 POINT(37.4959378558745 -122.86690502409657) bank173643 +173644 POINT(37.18213538855403 -122.51432213905159) bank173644 +173645 POINT(37.10900671602798 -121.91978391465456) bank173645 +173646 POINT(37.7687118646047 -122.47086972070785) bank173646 +173647 POINT(37.54686848712527 -122.44320786940492) bank173647 +173648 POINT(37.51440115616541 -123.01499023851791) bank173648 +173649 POINT(37.719579656741615 -121.5985505559466) bank173649 +173650 POINT(38.63503307422185 -122.64705203163692) bank173650 +173651 POINT(38.29820808643709 -122.05413379475853) bank173651 +173652 POINT(38.601137215920346 -122.71808761391272) bank173652 +173653 POINT(37.97959904975398 -122.66353974126058) bank173653 +173654 POINT(37.85935521760475 -121.43721223760838) bank173654 +173655 POINT(37.69150469722076 -122.65801325237854) bank173655 +173656 POINT(37.49740407412418 -122.40928200770522) bank173656 +173657 POINT(38.70147304360143 -121.93349859169828) bank173657 +173658 POINT(37.51355876820628 -122.67062835078009) bank173658 +173659 POINT(37.25776429214598 -122.25426645885469) bank173659 +173660 POINT(37.56978359939726 -122.6257274361209) bank173660 +173661 POINT(37.34542833037455 -121.98698002556254) bank173661 +173662 POINT(37.51245755746845 -121.63309689521468) bank173662 +173663 POINT(37.97250018162271 -122.76584148924296) bank173663 +173664 POINT(37.41550895262278 -122.72761669059254) bank173664 +173665 POINT(37.600059453963674 -121.70699585507514) bank173665 +173666 POINT(37.32237086888577 -122.39642060847157) bank173666 +173667 POINT(38.4151176965396 -122.91793092774535) bank173667 +173668 POINT(36.91048345822097 -123.21489251626413) bank173668 +173669 POINT(37.52528698206827 -121.6017367332658) bank173669 +173670 POINT(36.860287197670985 -122.35355825156698) bank173670 +173671 POINT(37.36701268740513 -122.59089873658705) bank173671 +173672 POINT(38.358491013845196 -122.01714046094651) bank173672 +173673 POINT(37.0608583817221 -123.4049328494795) bank173673 +173674 POINT(36.87809083841228 -122.09035603322434) bank173674 +173675 POINT(37.66989533282616 -122.73783156755553) bank173675 +173676 POINT(36.913593692342744 -122.11929579840712) bank173676 +173677 POINT(37.528393251058326 -123.29143312925817) bank173677 +173678 POINT(37.66157923747423 -121.90996466862109) bank173678 +173679 POINT(37.401246583311305 -121.91379530931556) bank173679 +173680 POINT(36.95673032324038 -122.5613482661946) bank173680 +173681 POINT(37.454236585737874 -123.06040959353447) bank173681 +173682 POINT(37.305644852943715 -121.9482204746987) bank173682 +173683 POINT(37.06981200268845 -121.75556288624807) bank173683 +173684 POINT(37.6918376458372 -123.06220940814651) bank173684 +173685 POINT(37.172183980070244 -122.91374600212632) bank173685 +173686 POINT(38.31810725301835 -123.35082945529112) bank173686 +173687 POINT(37.24766664329676 -121.90490187797363) bank173687 +173688 POINT(37.31701118775993 -121.71454229820675) bank173688 +173689 POINT(37.31374290796605 -122.56672414280884) bank173689 +173690 POINT(37.71832253322961 -122.48487966607716) bank173690 +173691 POINT(37.69134483174093 -121.43607335815277) bank173691 +173692 POINT(37.620768506950704 -121.4933428483132) bank173692 +173693 POINT(38.10570679921946 -122.57535162367775) bank173693 +173694 POINT(37.121219431771074 -122.40047951285676) bank173694 +173695 POINT(36.808538401526896 -121.909844231617) bank173695 +173696 POINT(38.0060386537981 -122.67378567877073) bank173696 +173697 POINT(38.15367355186029 -122.24438010733621) bank173697 +173698 POINT(36.9361697393702 -121.49164514099245) bank173698 +173699 POINT(37.21357154205571 -122.82980021165277) bank173699 +173700 POINT(38.232767122686376 -123.25104330153059) bank173700 +173701 POINT(37.5361920396284 -122.96934701436581) bank173701 +173702 POINT(38.50755757738243 -122.33027106961096) bank173702 +173703 POINT(38.509793385834115 -122.28970129077022) bank173703 +173704 POINT(38.18429551658651 -123.41877842637685) bank173704 +173705 POINT(37.51949704108601 -121.68164088585897) bank173705 +173706 POINT(37.032087262628856 -122.1591813992997) bank173706 +173707 POINT(37.348572583312155 -123.0813540489692) bank173707 +173708 POINT(36.96012243841892 -121.8883313998172) bank173708 +173709 POINT(37.81190463603639 -121.58383622907007) bank173709 +173710 POINT(38.679813093034305 -121.4494281045362) bank173710 +173711 POINT(37.43213092714479 -122.16198644548149) bank173711 +173712 POINT(37.22078903445644 -122.63644631142772) bank173712 +173713 POINT(36.89161718050997 -122.68347177685172) bank173713 +173714 POINT(36.954496632966084 -122.46398794056061) bank173714 +173715 POINT(37.792097619752255 -121.74019664638054) bank173715 +173716 POINT(36.79183475842201 -122.8762688769563) bank173716 +173717 POINT(38.45694511571794 -122.98186468673018) bank173717 +173718 POINT(37.39698563396541 -121.57848826724981) bank173718 +173719 POINT(38.3359588989168 -122.33711929811369) bank173719 +173720 POINT(38.507532622243595 -121.98958946895168) bank173720 +173721 POINT(36.87262067224213 -123.336289336781) bank173721 +173722 POINT(37.300978839514066 -122.36403185199119) bank173722 +173723 POINT(38.73986355748857 -122.51041225087793) bank173723 +173724 POINT(37.66686297290123 -122.28615161859857) bank173724 +173725 POINT(38.09364585097452 -121.69313454506097) bank173725 +173726 POINT(37.50028112685211 -121.531664547899) bank173726 +173727 POINT(37.499963974648665 -122.45657891914313) bank173727 +173728 POINT(36.89472062121214 -122.14757857696935) bank173728 +173729 POINT(38.67961923048427 -123.36766987163486) bank173729 +173730 POINT(37.7148715411057 -121.57116821721841) bank173730 +173731 POINT(37.631151719537044 -122.54430555797869) bank173731 +173732 POINT(37.52895468695716 -122.8126115358545) bank173732 +173733 POINT(36.871852156473246 -121.53750425406268) bank173733 +173734 POINT(37.71183977774806 -121.8535640523126) bank173734 +173735 POINT(36.827697576880894 -123.21932384733915) bank173735 +173736 POINT(36.801673841741184 -122.15355468207265) bank173736 +173737 POINT(36.98384483051479 -121.85798218540063) bank173737 +173738 POINT(37.63254324154886 -121.4945372044658) bank173738 +173739 POINT(38.54795268220047 -121.58133571164385) bank173739 +173740 POINT(36.849294020182825 -122.29061223053793) bank173740 +173741 POINT(38.47848311636326 -122.9770264908904) bank173741 +173742 POINT(38.42918153513001 -121.73050612387952) bank173742 +173743 POINT(37.51792556612943 -123.27516802969116) bank173743 +173744 POINT(38.42369090231531 -123.34458270413609) bank173744 +173745 POINT(37.052223088630015 -121.54543048634244) bank173745 +173746 POINT(38.27964850870377 -121.96907784226737) bank173746 +173747 POINT(38.70842513328521 -122.28288588813935) bank173747 +173748 POINT(37.067000456914535 -121.68987801277828) bank173748 +173749 POINT(38.20881486721124 -121.95122017984352) bank173749 +173750 POINT(38.073303988078834 -121.90973086366455) bank173750 +173751 POINT(38.49415354035835 -121.68147532573398) bank173751 +173752 POINT(36.94313986335594 -122.9762704021212) bank173752 +173753 POINT(38.30221152676957 -121.946320089205) bank173753 +173754 POINT(38.14365229088183 -121.58883123980233) bank173754 +173755 POINT(38.239755661632564 -123.17576313464404) bank173755 +173756 POINT(38.02123588973478 -122.2661637205518) bank173756 +173757 POINT(37.07953256375704 -121.6643690298144) bank173757 +173758 POINT(37.37356020316506 -121.80420743720101) bank173758 +173759 POINT(37.89313388743837 -121.57334544828123) bank173759 +173760 POINT(37.5955833750601 -122.95442484626045) bank173760 +173761 POINT(37.10142534890843 -121.54712710455874) bank173761 +173762 POINT(36.9213246233337 -121.99897665410089) bank173762 +173763 POINT(38.31189897657019 -121.51732475421474) bank173763 +173764 POINT(37.48205413342724 -121.96080914505333) bank173764 +173765 POINT(37.647471456726166 -121.62369021334237) bank173765 +173766 POINT(36.87784689088991 -121.9165010738737) bank173766 +173767 POINT(38.11400108431824 -121.5383460497285) bank173767 +173768 POINT(37.806141231065354 -121.85993497961297) bank173768 +173769 POINT(38.03116920580888 -121.67393946305718) bank173769 +173770 POINT(37.735988655037744 -122.13461083674078) bank173770 +173771 POINT(38.594335070975035 -121.79910969542996) bank173771 +173772 POINT(37.92074712355959 -122.45598060745164) bank173772 +173773 POINT(36.85420656429581 -121.54912349214304) bank173773 +173774 POINT(38.055970747123524 -121.42624471558157) bank173774 +173775 POINT(38.139728200946124 -122.13706210208436) bank173775 +173776 POINT(37.21444882905291 -122.28197439951495) bank173776 +173777 POINT(38.64949901590021 -122.54340012745782) bank173777 +173778 POINT(37.88466512347642 -123.16937317239821) bank173778 +173779 POINT(37.29891487603794 -122.13370782931085) bank173779 +173780 POINT(38.3160983106176 -122.41350219248407) bank173780 +173781 POINT(36.92031748496131 -122.92836275595114) bank173781 +173782 POINT(38.65012208835945 -122.43841958549582) bank173782 +173783 POINT(38.370951398557196 -122.52665652185026) bank173783 +173784 POINT(38.375392786078855 -123.20316169036253) bank173784 +173785 POINT(37.580518039927824 -121.98591783728949) bank173785 +173786 POINT(36.87248202889816 -122.09069029472604) bank173786 +173787 POINT(38.07479384957097 -122.00734695521358) bank173787 +173788 POINT(37.078963234762135 -122.87379595972381) bank173788 +173789 POINT(37.30754635225684 -123.25766725029169) bank173789 +173790 POINT(38.18615948047567 -121.78679905625758) bank173790 +173791 POINT(38.17724171905048 -122.64600036789689) bank173791 +173792 POINT(37.57521025410122 -122.33569914921537) bank173792 +173793 POINT(37.62591250274677 -123.36268918450253) bank173793 +173794 POINT(36.815703433616804 -122.03437244926215) bank173794 +173795 POINT(37.88788758640353 -122.8394911729951) bank173795 +173796 POINT(36.99935769748197 -122.45561485895466) bank173796 +173797 POINT(36.948150639686965 -123.32159106819226) bank173797 +173798 POINT(37.197245302781624 -123.00788127087682) bank173798 +173799 POINT(38.25981333098886 -121.64980871230001) bank173799 +173800 POINT(38.212317209952786 -122.87725724832939) bank173800 +173801 POINT(37.82176592858568 -123.10109098211768) bank173801 +173802 POINT(37.05548539683806 -122.97842761551418) bank173802 +173803 POINT(37.90552630241461 -123.13911137269335) bank173803 +173804 POINT(37.52565516222539 -122.12362754129654) bank173804 +173805 POINT(36.92043332451396 -122.87910755895308) bank173805 +173806 POINT(37.281752290207585 -121.69659885586404) bank173806 +173807 POINT(37.34319222115237 -122.66310381287198) bank173807 +173808 POINT(37.505144349607605 -122.35647109262327) bank173808 +173809 POINT(37.54543875919158 -122.11594960361293) bank173809 +173810 POINT(37.521953678211084 -122.44330530210254) bank173810 +173811 POINT(37.62151784611337 -122.35451437552618) bank173811 +173812 POINT(37.000051222987324 -123.39874139883538) bank173812 +173813 POINT(38.01930557091919 -123.38132441952226) bank173813 +173814 POINT(38.27923469094486 -121.6440671229496) bank173814 +173815 POINT(38.26501093460253 -122.31218069485196) bank173815 +173816 POINT(38.52759961815971 -121.47835559021729) bank173816 +173817 POINT(38.37419971093321 -121.85962548079243) bank173817 +173818 POINT(38.416804341212966 -122.03076645639102) bank173818 +173819 POINT(37.62517248154996 -121.58534954230288) bank173819 +173820 POINT(37.65939085035138 -122.99125035474684) bank173820 +173821 POINT(37.90593541166459 -122.61614150908503) bank173821 +173822 POINT(37.44532701427792 -122.41603003795252) bank173822 +173823 POINT(37.10339205316204 -121.45736541149721) bank173823 +173824 POINT(37.62018458302944 -122.30981673400291) bank173824 +173825 POINT(37.55043276486215 -122.01601807785869) bank173825 +173826 POINT(37.140634593012464 -122.58357230100444) bank173826 +173827 POINT(37.2435629451653 -121.64531496532354) bank173827 +173828 POINT(37.90420752810224 -121.81585127477237) bank173828 +173829 POINT(37.46376544232057 -121.51108612769535) bank173829 +173830 POINT(37.642915288951755 -123.02180062842626) bank173830 +173831 POINT(37.63144528693887 -121.70008364676899) bank173831 +173832 POINT(37.14811713791784 -122.18163838241851) bank173832 +173833 POINT(38.260550609747554 -123.0216538383514) bank173833 +173834 POINT(38.56355892763864 -122.23754604776822) bank173834 +173835 POINT(37.298492657853316 -122.96160978561733) bank173835 +173836 POINT(38.56605663736276 -122.04004632413057) bank173836 +173837 POINT(37.94747006533506 -123.0162194385708) bank173837 +173838 POINT(37.609774863433294 -122.19326821282667) bank173838 +173839 POINT(36.9197602932672 -122.25693236300819) bank173839 +173840 POINT(37.31382959790568 -123.0872758351337) bank173840 +173841 POINT(37.359122231751556 -123.03049272246379) bank173841 +173842 POINT(38.181250188997865 -121.66679335928) bank173842 +173843 POINT(38.67531694836264 -122.08572033299848) bank173843 +173844 POINT(37.19509241222273 -123.00491330637192) bank173844 +173845 POINT(38.097507661973346 -122.84912438301336) bank173845 +173846 POINT(38.330229465518755 -122.5123334497606) bank173846 +173847 POINT(38.3711025953408 -122.37004561857302) bank173847 +173848 POINT(38.048432242695384 -122.6867772133475) bank173848 +173849 POINT(38.05765937352557 -122.16246654210386) bank173849 +173850 POINT(38.045777565738256 -122.32974734197657) bank173850 +173851 POINT(36.86308413530442 -121.60089760440167) bank173851 +173852 POINT(38.55432481402379 -122.5519706584289) bank173852 +173853 POINT(38.46511410777486 -121.51701835578962) bank173853 +173854 POINT(37.055854762119175 -122.75990761923188) bank173854 +173855 POINT(37.82122685974069 -122.84049760836334) bank173855 +173856 POINT(38.37900178807093 -123.06141467228105) bank173856 +173857 POINT(37.36583451169141 -121.52507357817687) bank173857 +173858 POINT(37.42051055436041 -122.7148927458625) bank173858 +173859 POINT(38.45756234802347 -121.6939124267445) bank173859 +173860 POINT(37.129158696716075 -123.41605869527424) bank173860 +173861 POINT(37.75788689341137 -122.074101131804) bank173861 +173862 POINT(38.71881572349386 -123.33945931537295) bank173862 +173863 POINT(37.31409018250027 -122.04822855714879) bank173863 +173864 POINT(37.04005626104508 -122.73361548721667) bank173864 +173865 POINT(38.026982094348675 -122.6208595316622) bank173865 +173866 POINT(38.03096734742286 -122.52218465174404) bank173866 +173867 POINT(37.630632113262465 -122.25932554356733) bank173867 +173868 POINT(38.12558459478954 -122.51633502896173) bank173868 +173869 POINT(36.99161181286598 -123.00181129584097) bank173869 +173870 POINT(38.762651688832 -122.55870549999027) bank173870 +173871 POINT(37.99277565980452 -122.53581010747234) bank173871 +173872 POINT(37.46184708235457 -123.01346416330081) bank173872 +173873 POINT(38.2579883754763 -123.35962016937268) bank173873 +173874 POINT(37.931317699312686 -123.02731642811037) bank173874 +173875 POINT(37.6736138426399 -122.12050099488006) bank173875 +173876 POINT(37.82229726370564 -123.17162595595534) bank173876 +173877 POINT(37.921871007275605 -121.95591010804402) bank173877 +173878 POINT(37.27223095543733 -122.032317855495) bank173878 +173879 POINT(37.76724180907987 -122.67598699326109) bank173879 +173880 POINT(37.18458578818959 -123.01615472840618) bank173880 +173881 POINT(36.78777709611409 -121.46654614266032) bank173881 +173882 POINT(38.27823414528437 -121.92775723033581) bank173882 +173883 POINT(36.96966134577721 -122.64812239316886) bank173883 +173884 POINT(37.109400526516865 -122.11038712727323) bank173884 +173885 POINT(38.3998991583053 -122.47677809017426) bank173885 +173886 POINT(37.49098457698769 -121.68408322066769) bank173886 +173887 POINT(37.18389373526526 -123.2596446123592) bank173887 +173888 POINT(38.120991929051605 -123.09454621389973) bank173888 +173889 POINT(37.62964593292755 -122.74864971658515) bank173889 +173890 POINT(38.65354622650574 -122.16940438825016) bank173890 +173891 POINT(38.68134938312928 -121.64141528856595) bank173891 +173892 POINT(38.33204035752756 -123.2978832436579) bank173892 +173893 POINT(37.434385101221686 -121.77293043925482) bank173893 +173894 POINT(37.98422627218839 -121.63709321659005) bank173894 +173895 POINT(38.58348771514521 -121.47184041062584) bank173895 +173896 POINT(37.70888344867193 -122.97499072156235) bank173896 +173897 POINT(38.443449988103296 -123.27135115626315) bank173897 +173898 POINT(38.227146020609105 -122.31937816113494) bank173898 +173899 POINT(38.485137617974125 -123.0767890018681) bank173899 +173900 POINT(37.59830845457884 -122.66178860345725) bank173900 +173901 POINT(37.757830408629474 -121.89095753415621) bank173901 +173902 POINT(38.357707430613885 -121.58280771684423) bank173902 +173903 POINT(37.59090642480585 -122.14867675955949) bank173903 +173904 POINT(36.89433798098934 -123.16400014660152) bank173904 +173905 POINT(37.277188692537855 -122.62686322274494) bank173905 +173906 POINT(38.376703865263416 -122.5873883992031) bank173906 +173907 POINT(38.710457927574566 -123.10647605803814) bank173907 +173908 POINT(38.52605499479867 -123.01307680313377) bank173908 +173909 POINT(36.99691410671056 -121.57968054581552) bank173909 +173910 POINT(36.93726190168379 -122.71014240925754) bank173910 +173911 POINT(38.51470438736115 -122.13600766001628) bank173911 +173912 POINT(38.536924905790755 -122.97675800307269) bank173912 +173913 POINT(36.93548931305088 -121.52828645553953) bank173913 +173914 POINT(38.72030204022836 -122.27657351005617) bank173914 +173915 POINT(36.87462841185289 -122.92434646284192) bank173915 +173916 POINT(37.20552369494444 -122.711508248912) bank173916 +173917 POINT(37.14172738531975 -121.53272859137482) bank173917 +173918 POINT(37.168776486207456 -122.83388290898186) bank173918 +173919 POINT(37.855222365669526 -122.79532590071996) bank173919 +173920 POINT(38.76250679792617 -123.1282697432508) bank173920 +173921 POINT(37.41367500713712 -123.1938479663035) bank173921 +173922 POINT(37.86766711171234 -122.81521564086866) bank173922 +173923 POINT(36.895672637916135 -123.05453106728044) bank173923 +173924 POINT(37.65954679802189 -122.58703032554313) bank173924 +173925 POINT(38.70342790014892 -121.64315453121911) bank173925 +173926 POINT(38.65304040620356 -122.38308171179062) bank173926 +173927 POINT(37.33068947207534 -121.9310017983084) bank173927 +173928 POINT(37.5978198549331 -123.04832508890426) bank173928 +173929 POINT(37.32614284140529 -121.75942130367932) bank173929 +173930 POINT(37.94269587349392 -123.37193548521435) bank173930 +173931 POINT(37.4259979455716 -122.29879967108823) bank173931 +173932 POINT(37.845329102407305 -122.64732836331545) bank173932 +173933 POINT(37.82912967289175 -122.89211384197547) bank173933 +173934 POINT(36.83135416439477 -122.6735170075316) bank173934 +173935 POINT(37.92778783262368 -122.75136825548097) bank173935 +173936 POINT(37.75920793200129 -122.76543166363281) bank173936 +173937 POINT(36.83202392207123 -121.98556199022781) bank173937 +173938 POINT(36.99516423692775 -122.44833617081582) bank173938 +173939 POINT(37.64498368248235 -122.31041867086515) bank173939 +173940 POINT(37.09021752955287 -122.14039522879395) bank173940 +173941 POINT(38.74990206462533 -122.19568207698856) bank173941 +173942 POINT(37.702297703571105 -122.95279955180175) bank173942 +173943 POINT(37.822702717284834 -123.02018627342837) bank173943 +173944 POINT(37.515213376271255 -122.23115214535852) bank173944 +173945 POINT(38.033650451895305 -122.25752139197589) bank173945 +173946 POINT(37.98592548074823 -123.38506428128073) bank173946 +173947 POINT(38.76532649870433 -123.35909189033687) bank173947 +173948 POINT(38.757852345286814 -123.37110932770699) bank173948 +173949 POINT(37.913126269435686 -123.03914464747508) bank173949 +173950 POINT(37.94662110492501 -122.55092188413583) bank173950 +173951 POINT(38.53502754386097 -123.12750075141905) bank173951 +173952 POINT(38.70149952604586 -121.62519071311733) bank173952 +173953 POINT(36.89107690669989 -121.62386415263465) bank173953 +173954 POINT(38.53362507266853 -123.02008604084588) bank173954 +173955 POINT(38.42939827456405 -122.99473375176348) bank173955 +173956 POINT(36.938241718877116 -121.9832348508349) bank173956 +173957 POINT(38.42978676632705 -122.08828447543974) bank173957 +173958 POINT(36.933972181687864 -121.88506437891044) bank173958 +173959 POINT(37.75069064191155 -122.37006934031214) bank173959 +173960 POINT(36.93928167614257 -121.90418333298086) bank173960 +173961 POINT(37.410136299403256 -121.47299721661824) bank173961 +173962 POINT(37.59581532281004 -122.60487169617542) bank173962 +173963 POINT(38.73819368456062 -122.3758263913235) bank173963 +173964 POINT(38.58552512615986 -123.24677932280433) bank173964 +173965 POINT(37.83259312909036 -121.48860008564607) bank173965 +173966 POINT(38.45692522864786 -123.04891583150159) bank173966 +173967 POINT(38.45924057829582 -122.87581020468818) bank173967 +173968 POINT(38.4331617765331 -122.65280560601902) bank173968 +173969 POINT(38.479164262816084 -122.90918195691032) bank173969 +173970 POINT(38.36233031844201 -123.4034117817251) bank173970 +173971 POINT(37.412530801108545 -122.25745757719024) bank173971 +173972 POINT(38.229588887914524 -121.7354496074481) bank173972 +173973 POINT(38.65667233013343 -123.16311791623491) bank173973 +173974 POINT(37.07049706259533 -122.6202301700516) bank173974 +173975 POINT(38.11042211976485 -122.86252804172429) bank173975 +173976 POINT(38.245458397136694 -122.26456172742544) bank173976 +173977 POINT(38.32659740570709 -121.77415513466826) bank173977 +173978 POINT(38.528960852646854 -123.00137223501825) bank173978 +173979 POINT(37.47814771710182 -122.6737936135338) bank173979 +173980 POINT(37.62102733727578 -121.71559712698253) bank173980 +173981 POINT(37.67953832722348 -123.35492451797282) bank173981 +173982 POINT(38.27025140651186 -123.23442689260511) bank173982 +173983 POINT(37.07872903423556 -121.57058420431468) bank173983 +173984 POINT(38.3045283159761 -121.90339887180308) bank173984 +173985 POINT(37.50947946638189 -122.56981486463147) bank173985 +173986 POINT(37.28178011738359 -122.31085464001885) bank173986 +173987 POINT(38.74539125078806 -122.1059886208724) bank173987 +173988 POINT(37.88061567581822 -122.65120029680057) bank173988 +173989 POINT(38.59010386048427 -123.27461609636748) bank173989 +173990 POINT(38.72376596097165 -122.67278454864913) bank173990 +173991 POINT(37.890130192796946 -122.8758563303199) bank173991 +173992 POINT(38.69746488061229 -121.97068474600198) bank173992 +173993 POINT(37.30143394269374 -121.47478190508532) bank173993 +173994 POINT(38.145797270837676 -123.05771916319216) bank173994 +173995 POINT(36.97565237118062 -122.72009813780889) bank173995 +173996 POINT(37.389900924407456 -122.4528123885745) bank173996 +173997 POINT(38.38629901685956 -122.43613772424143) bank173997 +173998 POINT(38.04064988779785 -123.28954252692658) bank173998 +173999 POINT(37.57933608849441 -122.69738443834082) bank173999 +174000 POINT(37.77610486802051 -121.98175652350676) bank174000 +174001 POINT(37.30160485768998 -122.02246632770994) bank174001 +174002 POINT(38.12876373349732 -121.81827578517188) bank174002 +174003 POINT(38.11920656282466 -123.19880911279866) bank174003 +174004 POINT(37.89930587934869 -122.63269138677511) bank174004 +174005 POINT(37.603803272638665 -122.26098538398807) bank174005 +174006 POINT(37.449411766108554 -122.60779157916227) bank174006 +174007 POINT(37.227374660843424 -122.74380460083206) bank174007 +174008 POINT(36.78308823488278 -121.60971103818864) bank174008 +174009 POINT(37.29425773693036 -122.87889252446946) bank174009 +174010 POINT(37.177276772102616 -121.72914992600155) bank174010 +174011 POINT(38.521000313929555 -122.00331918867941) bank174011 +174012 POINT(38.5424621634874 -122.81503264511319) bank174012 +174013 POINT(37.85154580748754 -123.35568344294562) bank174013 +174014 POINT(36.85833591361064 -123.31891710618522) bank174014 +174015 POINT(37.07498739787587 -121.74469009622369) bank174015 +174016 POINT(37.88110828469585 -123.01501061274662) bank174016 +174017 POINT(37.445181297711365 -123.15763218651031) bank174017 +174018 POINT(38.42932324705408 -122.9588522541576) bank174018 +174019 POINT(37.75323512322586 -122.64615065332512) bank174019 +174020 POINT(37.620269093552515 -122.03910121930808) bank174020 +174021 POINT(36.80039830538767 -123.37454050956495) bank174021 +174022 POINT(38.2300032880383 -123.34112493542627) bank174022 +174023 POINT(37.57734923310048 -122.27601983135945) bank174023 +174024 POINT(37.83163918014817 -123.1498485183275) bank174024 +174025 POINT(36.953692584597945 -122.7017288953398) bank174025 +174026 POINT(37.073896241273474 -121.50922295350186) bank174026 +174027 POINT(38.344383124291575 -122.30006042904351) bank174027 +174028 POINT(38.54775534308666 -121.71872975916385) bank174028 +174029 POINT(37.19575956838763 -121.55572444773999) bank174029 +174030 POINT(38.56884417493511 -122.1077201261836) bank174030 +174031 POINT(37.49494182830066 -121.9624428017985) bank174031 +174032 POINT(37.72806292571916 -121.74460217296509) bank174032 +174033 POINT(37.051211086284525 -121.7681361620178) bank174033 +174034 POINT(37.794605215477276 -122.57517446559194) bank174034 +174035 POINT(36.949485579148394 -121.74593180308406) bank174035 +174036 POINT(36.88379368040448 -122.04495219410313) bank174036 +174037 POINT(37.1764766289638 -122.09091738336011) bank174037 +174038 POINT(37.389977618937685 -122.59802381683642) bank174038 +174039 POINT(36.85560516495881 -122.41735026600377) bank174039 +174040 POINT(36.841534304663945 -123.32142380333849) bank174040 +174041 POINT(36.86873370905488 -121.87469993610785) bank174041 +174042 POINT(37.2881537881407 -121.76131141211087) bank174042 +174043 POINT(37.876016352680644 -122.93993555736945) bank174043 +174044 POINT(37.58635652361037 -122.67801059892902) bank174044 +174045 POINT(37.750220463791145 -122.59435350620856) bank174045 +174046 POINT(37.81575566917673 -123.39800412330295) bank174046 +174047 POINT(37.09264320674812 -122.0230958347832) bank174047 +174048 POINT(38.287539990212466 -122.63231899210552) bank174048 +174049 POINT(37.68128679975204 -123.39545811523791) bank174049 +174050 POINT(38.27728731063635 -122.53372579923516) bank174050 +174051 POINT(37.613114010650285 -121.62737425783784) bank174051 +174052 POINT(37.86992968667454 -122.46952612441741) bank174052 +174053 POINT(36.80386407866753 -121.76661696603134) bank174053 +174054 POINT(37.62206441484822 -122.78401863857822) bank174054 +174055 POINT(37.897240502853855 -122.32440868515083) bank174055 +174056 POINT(38.34285853350651 -121.62536572823203) bank174056 +174057 POINT(38.201743421947626 -122.47979647056259) bank174057 +174058 POINT(37.258611103023966 -122.42053023713919) bank174058 +174059 POINT(36.90432593371699 -122.62321327001216) bank174059 +174060 POINT(37.495440907672695 -121.73966837755343) bank174060 +174061 POINT(36.85855327578955 -122.29277087274677) bank174061 +174062 POINT(38.34193775117514 -122.40605333861046) bank174062 +174063 POINT(37.97570929057666 -123.20214908588164) bank174063 +174064 POINT(38.45084081222828 -122.91420180374848) bank174064 +174065 POINT(37.99453475784665 -123.19417567470718) bank174065 +174066 POINT(38.313603761392216 -121.961382596764) bank174066 +174067 POINT(38.06363787935282 -123.03389548496966) bank174067 +174068 POINT(38.551176369552266 -121.85585273376837) bank174068 +174069 POINT(37.04357809381859 -123.03415371860557) bank174069 +174070 POINT(38.69354692553565 -121.5330820880166) bank174070 +174071 POINT(37.80825115885375 -122.20379183315411) bank174071 +174072 POINT(38.37561607942222 -122.26092250604412) bank174072 +174073 POINT(36.913775668884064 -122.18538349391524) bank174073 +174074 POINT(36.90491039790565 -122.1694828539512) bank174074 +174075 POINT(38.53297770120043 -121.55548564086786) bank174075 +174076 POINT(37.90517528063999 -122.56604127899726) bank174076 +174077 POINT(37.176507098191536 -122.45553490328514) bank174077 +174078 POINT(38.762393782714454 -122.43662294075989) bank174078 +174079 POINT(36.88042400123182 -123.04453586444332) bank174079 +174080 POINT(37.09955975108953 -123.41022072095836) bank174080 +174081 POINT(37.24475908280059 -123.25933694760654) bank174081 +174082 POINT(37.67761456102377 -122.9331365834876) bank174082 +174083 POINT(36.95198502531047 -122.32484947965379) bank174083 +174084 POINT(37.216693354862265 -122.51323320774647) bank174084 +174085 POINT(37.0662559090341 -121.70351350591334) bank174085 +174086 POINT(38.05413703694037 -122.81710079978235) bank174086 +174087 POINT(37.940887808837275 -122.33264907091196) bank174087 +174088 POINT(37.807341322263376 -122.14349799117775) bank174088 +174089 POINT(38.339484629853686 -122.21437047122639) bank174089 +174090 POINT(37.270556703761464 -122.68891507798517) bank174090 +174091 POINT(37.18816640713675 -123.2273338456996) bank174091 +174092 POINT(38.44788147319995 -123.21603806469064) bank174092 +174093 POINT(37.74280658848286 -122.66832774862989) bank174093 +174094 POINT(38.457189575167824 -122.8869395254217) bank174094 +174095 POINT(38.0279160794407 -121.46399140472296) bank174095 +174096 POINT(37.82509162318871 -121.61791003400808) bank174096 +174097 POINT(38.186113062823694 -121.6670047815153) bank174097 +174098 POINT(38.10926778488739 -123.00715145694103) bank174098 +174099 POINT(37.69955063373899 -121.5173029525694) bank174099 +174100 POINT(36.839465581413066 -122.39102129705307) bank174100 +174101 POINT(36.84613799091859 -121.57372488548431) bank174101 +174102 POINT(38.046115692427826 -122.6731957251284) bank174102 +174103 POINT(38.171187516695156 -123.35277590238813) bank174103 +174104 POINT(38.14092053664413 -121.98236070762712) bank174104 +174105 POINT(37.831407521524135 -121.42504404652045) bank174105 +174106 POINT(37.46612476386798 -122.16045641706403) bank174106 +174107 POINT(37.29105480400728 -122.56043982226385) bank174107 +174108 POINT(38.606853454094 -122.71853142465035) bank174108 +174109 POINT(37.88052174830597 -122.28863292977333) bank174109 +174110 POINT(38.62940429964208 -121.82411955286824) bank174110 +174111 POINT(37.63641711043933 -122.59029247169565) bank174111 +174112 POINT(37.04986582370055 -122.70911113239902) bank174112 +174113 POINT(38.24809670523983 -121.7879566013827) bank174113 +174114 POINT(37.33797696034246 -122.94384239161575) bank174114 +174115 POINT(37.79008718869099 -122.28839261877805) bank174115 +174116 POINT(36.91453747796403 -122.17055833103718) bank174116 +174117 POINT(38.708743998199566 -122.9778778600751) bank174117 +174118 POINT(38.08983669001234 -122.9853595409191) bank174118 +174119 POINT(37.647384884596455 -121.76746215785786) bank174119 +174120 POINT(38.22467433794206 -122.3751493039606) bank174120 +174121 POINT(37.21486139472854 -122.44716987644229) bank174121 +174122 POINT(38.75972621730393 -123.37210588839883) bank174122 +174123 POINT(37.39395960477005 -122.6697160934329) bank174123 +174124 POINT(37.58180003906435 -121.80998343118353) bank174124 +174125 POINT(37.279206901256636 -122.44555743281109) bank174125 +174126 POINT(36.83436868055274 -123.29545687840347) bank174126 +174127 POINT(38.22835975305157 -123.19806246448837) bank174127 +174128 POINT(38.616107426064126 -122.19994795823786) bank174128 +174129 POINT(37.887336532691045 -123.18906264876993) bank174129 +174130 POINT(38.32831700151501 -122.47867598288433) bank174130 +174131 POINT(37.40263278682565 -121.98791761034683) bank174131 +174132 POINT(38.48804985147024 -122.6880084382712) bank174132 +174133 POINT(37.16371076508855 -123.27874562737712) bank174133 +174134 POINT(37.12927277380743 -121.78510910225792) bank174134 +174135 POINT(38.70311142910627 -122.9371713636977) bank174135 +174136 POINT(37.69159560065804 -123.3318949744551) bank174136 +174137 POINT(38.653136756821056 -121.53562861528846) bank174137 +174138 POINT(38.574464164620714 -122.77814282227645) bank174138 +174139 POINT(37.39442866012633 -121.51701182847019) bank174139 +174140 POINT(37.040556549404094 -123.33602501259699) bank174140 +174141 POINT(37.1182391860084 -121.66386444812416) bank174141 +174142 POINT(37.31017296994895 -122.44284791793162) bank174142 +174143 POINT(38.174819951474355 -121.66111077302935) bank174143 +174144 POINT(38.43314352714832 -122.09739324733869) bank174144 +174145 POINT(37.36311504575502 -122.08252003235874) bank174145 +174146 POINT(37.663299615610626 -123.12006327173751) bank174146 +174147 POINT(36.815047038591224 -121.93302426684399) bank174147 +174148 POINT(38.64582913499725 -122.2485149030213) bank174148 +174149 POINT(36.827026509415916 -122.22210163037387) bank174149 +174150 POINT(38.52583270173026 -121.70170023592541) bank174150 +174151 POINT(38.746726069875926 -123.25387554607369) bank174151 +174152 POINT(37.41186300650224 -122.26473449548945) bank174152 +174153 POINT(36.87489547358353 -122.80545789926437) bank174153 +174154 POINT(36.992305478604976 -122.10509280273014) bank174154 +174155 POINT(37.17672027360542 -122.66030837318864) bank174155 +174156 POINT(38.574853351142025 -121.45805413795956) bank174156 +174157 POINT(38.345162811727015 -122.43980812831555) bank174157 +174158 POINT(36.828685142909855 -122.31893130413158) bank174158 +174159 POINT(38.670694669675946 -121.59722433064454) bank174159 +174160 POINT(37.99975693242245 -122.28514614221608) bank174160 +174161 POINT(37.46509043293751 -122.27465880685196) bank174161 +174162 POINT(38.56259839682193 -121.89951291796508) bank174162 +174163 POINT(38.42182703743033 -122.39231872467035) bank174163 +174164 POINT(38.643052247940844 -122.70726056019171) bank174164 +174165 POINT(37.33222405061454 -122.36956531909821) bank174165 +174166 POINT(38.03507858136575 -121.74578550724216) bank174166 +174167 POINT(36.819833174547334 -122.82635488645434) bank174167 +174168 POINT(37.05238793438432 -121.76901178179689) bank174168 +174169 POINT(37.63675667430113 -122.10870774843127) bank174169 +174170 POINT(38.645642795346454 -122.12518617450145) bank174170 +174171 POINT(37.57637567365685 -121.9372136364405) bank174171 +174172 POINT(38.020919460471056 -122.56238257280499) bank174172 +174173 POINT(37.307050488918996 -122.05485629624717) bank174173 +174174 POINT(37.35494961869354 -121.88280024823851) bank174174 +174175 POINT(38.06624584036548 -122.99567905926396) bank174175 +174176 POINT(38.366910130675784 -122.38053508349633) bank174176 +174177 POINT(38.72994547477361 -122.87383155791588) bank174177 +174178 POINT(38.5453172085547 -121.97874762504375) bank174178 +174179 POINT(37.561199222534285 -122.45392914489923) bank174179 +174180 POINT(36.87038406852043 -122.97992612662519) bank174180 +174181 POINT(37.637769287510096 -123.17924736190422) bank174181 +174182 POINT(37.57321508049639 -121.48944655526931) bank174182 +174183 POINT(36.775487855117724 -123.02499156625132) bank174183 +174184 POINT(36.882210411250476 -121.6684619681157) bank174184 +174185 POINT(37.51697438165798 -122.50048868161453) bank174185 +174186 POINT(36.88927695609804 -122.62296881553038) bank174186 +174187 POINT(38.346244449504134 -122.62797521456608) bank174187 +174188 POINT(38.07796581136088 -122.16834528649312) bank174188 +174189 POINT(38.05001847657792 -123.2278848440189) bank174189 +174190 POINT(37.77970024735978 -122.80389738673347) bank174190 +174191 POINT(37.90680787835248 -121.74330397354974) bank174191 +174192 POINT(38.197901667976076 -123.41278684359075) bank174192 +174193 POINT(37.02854802716941 -123.11383887705067) bank174193 +174194 POINT(37.64407236862658 -122.43258701548277) bank174194 +174195 POINT(38.645989755992666 -122.36182438368371) bank174195 +174196 POINT(38.602296874236146 -122.1358248257352) bank174196 +174197 POINT(37.90568639558955 -122.00473715903206) bank174197 +174198 POINT(37.65245302662564 -123.36083526169061) bank174198 +174199 POINT(38.522017339682066 -122.87213275778387) bank174199 +174200 POINT(38.39131654965493 -122.41836089254434) bank174200 +174201 POINT(38.515910134914535 -122.45513653873843) bank174201 +174202 POINT(38.086125649745014 -121.61745162298881) bank174202 +174203 POINT(37.18890310888418 -121.65364199874904) bank174203 +174204 POINT(38.500951969966074 -121.43117106589456) bank174204 +174205 POINT(38.48180952000437 -122.13768446227182) bank174205 +174206 POINT(38.682562416297024 -122.57570586810402) bank174206 +174207 POINT(37.81470870601366 -122.7270825651089) bank174207 +174208 POINT(37.572587679020984 -121.50819549430655) bank174208 +174209 POINT(37.152705656670264 -122.63837828708373) bank174209 +174210 POINT(37.52399053466053 -122.67335770255136) bank174210 +174211 POINT(38.334181683292236 -121.5789967393898) bank174211 +174212 POINT(37.71244728259484 -121.99826557172587) bank174212 +174213 POINT(38.710225436739194 -123.08897138064253) bank174213 +174214 POINT(38.77049742322768 -121.77823186958983) bank174214 +174215 POINT(38.176125609194344 -121.91652775747733) bank174215 +174216 POINT(37.77822790483798 -121.51635474688041) bank174216 +174217 POINT(38.03960813209875 -122.07675993049767) bank174217 +174218 POINT(36.809111825692185 -121.49623105954093) bank174218 +174219 POINT(37.56037785774916 -123.1472825659249) bank174219 +174220 POINT(38.58512905219907 -123.10987011946534) bank174220 +174221 POINT(36.88642013730046 -122.89579274212926) bank174221 +174222 POINT(38.415595562971184 -123.25281014720954) bank174222 +174223 POINT(37.934239560842485 -122.4085153209599) bank174223 +174224 POINT(38.129466277087666 -122.64905286426162) bank174224 +174225 POINT(37.96595531984719 -121.70008196228174) bank174225 +174226 POINT(36.89332176189949 -122.24845353952016) bank174226 +174227 POINT(37.71296295405792 -123.21620342762498) bank174227 +174228 POINT(37.13790036170319 -122.97343835979397) bank174228 +174229 POINT(37.15361345025793 -121.96839800990296) bank174229 +174230 POINT(38.40503799542581 -121.4442496106272) bank174230 +174231 POINT(38.091597171366864 -121.49747985274382) bank174231 +174232 POINT(38.41440053609488 -121.99187479837015) bank174232 +174233 POINT(37.20357894996688 -123.24845611255377) bank174233 +174234 POINT(38.06026885240309 -122.59835738476364) bank174234 +174235 POINT(37.83012063457529 -122.33648834159821) bank174235 +174236 POINT(38.37579344443497 -123.14112502688755) bank174236 +174237 POINT(37.930930261545896 -121.63654779228848) bank174237 +174238 POINT(38.453299652862384 -121.59164016494312) bank174238 +174239 POINT(38.52806052119127 -121.85248095245727) bank174239 +174240 POINT(38.3963120475323 -122.64914852107502) bank174240 +174241 POINT(37.02246038120376 -122.77199048468849) bank174241 +174242 POINT(38.20477908854934 -121.99655259048114) bank174242 +174243 POINT(37.59892874229963 -122.30512262863239) bank174243 +174244 POINT(37.66656822121949 -123.3109947761437) bank174244 +174245 POINT(38.36978850313178 -122.35256516833172) bank174245 +174246 POINT(37.04086706635766 -122.68408170507422) bank174246 +174247 POINT(38.49538320189617 -122.45024675499765) bank174247 +174248 POINT(37.27023094219761 -122.73931621181892) bank174248 +174249 POINT(36.89509294261649 -123.29348552718355) bank174249 +174250 POINT(37.009732743877905 -121.90909293195493) bank174250 +174251 POINT(37.021391646101414 -122.2756596966852) bank174251 +174252 POINT(38.07895700042167 -121.48274879553455) bank174252 +174253 POINT(38.245278630201426 -121.63847541868202) bank174253 +174254 POINT(37.89993869379222 -122.80360628739983) bank174254 +174255 POINT(36.92042388803942 -122.57898480427029) bank174255 +174256 POINT(38.22280496726378 -122.745411607272) bank174256 +174257 POINT(37.32188605059332 -122.536316904127) bank174257 +174258 POINT(38.301505595804315 -122.81618206608275) bank174258 +174259 POINT(37.88794364378904 -123.22420649325835) bank174259 +174260 POINT(37.805161465778916 -121.79028063668066) bank174260 +174261 POINT(38.36930731726757 -123.37606997627253) bank174261 +174262 POINT(37.34187077215513 -122.88146841470558) bank174262 +174263 POINT(36.811486734961576 -122.43994293389103) bank174263 +174264 POINT(38.59549215084558 -122.87861745022678) bank174264 +174265 POINT(36.805049617797934 -122.78021274742396) bank174265 +174266 POINT(38.76357670562369 -121.74390214900399) bank174266 +174267 POINT(36.79993677140966 -122.55264102988811) bank174267 +174268 POINT(37.072288705281 -122.04004411441117) bank174268 +174269 POINT(38.040460450737086 -122.14684376532904) bank174269 +174270 POINT(37.67226291337738 -121.68073130042961) bank174270 +174271 POINT(38.33856462062015 -122.40048478762048) bank174271 +174272 POINT(36.776211003265736 -123.33658980506426) bank174272 +174273 POINT(38.53046869460909 -121.65253511852008) bank174273 +174274 POINT(38.6635619346767 -121.74571476200025) bank174274 +174275 POINT(37.323046077149364 -123.35334378737213) bank174275 +174276 POINT(38.6782644916152 -123.3202656378509) bank174276 +174277 POINT(36.807271647466216 -122.90726699039551) bank174277 +174278 POINT(38.556228085954665 -121.75632700667718) bank174278 +174279 POINT(37.59227721974485 -121.44886651818503) bank174279 +174280 POINT(37.459485468749754 -123.34438662153663) bank174280 +174281 POINT(37.17602414441571 -122.10970113514581) bank174281 +174282 POINT(37.083439335749105 -122.8091495208978) bank174282 +174283 POINT(37.13411615407682 -123.14911980995413) bank174283 +174284 POINT(38.357510795027196 -122.00876309580035) bank174284 +174285 POINT(38.72829902770901 -122.12454666486106) bank174285 +174286 POINT(38.260406687629654 -122.45590533175582) bank174286 +174287 POINT(36.84957961569739 -122.78866993288409) bank174287 +174288 POINT(38.535723769396974 -122.08844401355377) bank174288 +174289 POINT(38.5518178486154 -122.50600619582347) bank174289 +174290 POINT(38.123347783545405 -122.78734737384188) bank174290 +174291 POINT(37.822623691155115 -122.2263078948351) bank174291 +174292 POINT(37.77955725834643 -122.84778459715601) bank174292 +174293 POINT(37.752041815565455 -121.51070835204918) bank174293 +174294 POINT(37.07842544734787 -122.90588980317293) bank174294 +174295 POINT(37.006359859000185 -121.76762883141758) bank174295 +174296 POINT(37.06046315060884 -121.52894717641142) bank174296 +174297 POINT(37.25037610908364 -121.84943236027672) bank174297 +174298 POINT(37.02737649890343 -123.41285025099144) bank174298 +174299 POINT(37.25603406232541 -123.3680696505646) bank174299 +174300 POINT(38.01383745501357 -122.5381032384585) bank174300 +174301 POINT(38.092215337324305 -122.92097409116603) bank174301 +174302 POINT(38.72795678284444 -121.81974802322023) bank174302 +174303 POINT(37.592708602187834 -122.89270333556888) bank174303 +174304 POINT(36.795420080235345 -122.32094208502271) bank174304 +174305 POINT(38.49699296146169 -122.14704526224068) bank174305 +174306 POINT(38.47478075242344 -123.14316837501396) bank174306 +174307 POINT(37.26126956148387 -123.0366910496586) bank174307 +174308 POINT(38.67962561368347 -123.13605313111594) bank174308 +174309 POINT(37.531848894255674 -121.49809860577302) bank174309 +174310 POINT(37.5760421517486 -121.830978834533) bank174310 +174311 POINT(37.302032773058265 -122.16746744599095) bank174311 +174312 POINT(37.73753043472283 -121.60792170099957) bank174312 +174313 POINT(38.5077441184008 -122.47433120028359) bank174313 +174314 POINT(38.041645372255914 -121.80277989444821) bank174314 +174315 POINT(38.39409480062401 -122.55096383719214) bank174315 +174316 POINT(38.4192811879482 -122.62150429156705) bank174316 +174317 POINT(36.86550809715219 -121.56174916061127) bank174317 +174318 POINT(37.5275040385562 -122.80693440212771) bank174318 +174319 POINT(38.264718939957255 -122.57179286207207) bank174319 +174320 POINT(38.30862902801888 -122.74484465962983) bank174320 +174321 POINT(37.37048291702636 -123.14176848226744) bank174321 +174322 POINT(38.47309972065519 -122.91471601073606) bank174322 +174323 POINT(38.24145021979765 -122.82273882419001) bank174323 +174324 POINT(38.60385971893221 -121.50841349252012) bank174324 +174325 POINT(37.38398725664768 -123.02199636993299) bank174325 +174326 POINT(36.90409496317363 -121.48966113589692) bank174326 +174327 POINT(37.40341975123502 -121.46787189454137) bank174327 +174328 POINT(37.640741154582386 -123.27999294967528) bank174328 +174329 POINT(37.41202363631689 -122.67861109750037) bank174329 +174330 POINT(37.758230650419115 -121.78118870470468) bank174330 +174331 POINT(38.383597294850624 -121.5290976120766) bank174331 +174332 POINT(36.93663375974723 -122.17948293322637) bank174332 +174333 POINT(36.79827233873528 -122.02367267215101) bank174333 +174334 POINT(38.64234505676029 -121.73639797781993) bank174334 +174335 POINT(38.31238046830534 -122.08967116303867) bank174335 +174336 POINT(38.59678027751568 -123.35229426678568) bank174336 +174337 POINT(37.952864898391326 -121.70881262092982) bank174337 +174338 POINT(37.63912276091414 -122.30328864137803) bank174338 +174339 POINT(37.679251093168745 -121.45656643982434) bank174339 +174340 POINT(38.230663380151455 -122.76829837035217) bank174340 +174341 POINT(36.81243472662544 -122.95925739880472) bank174341 +174342 POINT(36.86298960264054 -121.88724719980972) bank174342 +174343 POINT(38.211090506004986 -122.2250246960821) bank174343 +174344 POINT(37.684519699998965 -122.5662230258155) bank174344 +174345 POINT(38.287039179836896 -121.59966556576747) bank174345 +174346 POINT(38.49635453613934 -123.10449635669974) bank174346 +174347 POINT(38.34589490606597 -121.98544270112608) bank174347 +174348 POINT(37.36711137161745 -122.6025014306967) bank174348 +174349 POINT(38.015406926944074 -122.54995843002484) bank174349 +174350 POINT(38.478875031232874 -123.1628361663974) bank174350 +174351 POINT(37.20836328298614 -122.0014758634023) bank174351 +174352 POINT(38.205773346916 -122.21758104735406) bank174352 +174353 POINT(37.485701307496676 -122.45832987860183) bank174353 +174354 POINT(38.39320102195503 -122.47995224770817) bank174354 +174355 POINT(38.12229664505866 -122.83832709399461) bank174355 +174356 POINT(38.48921654540198 -122.85689961701469) bank174356 +174357 POINT(37.01274943145892 -121.51094647695324) bank174357 +174358 POINT(38.39634650279468 -123.27092304670283) bank174358 +174359 POINT(38.029286983906 -122.44423699110946) bank174359 +174360 POINT(37.56563009439029 -123.02413161183435) bank174360 +174361 POINT(36.85137007710057 -122.86700525388846) bank174361 +174362 POINT(37.208375449657616 -122.74430983333848) bank174362 +174363 POINT(37.76012699135948 -123.28386195966756) bank174363 +174364 POINT(36.91364190900026 -122.42977296769836) bank174364 +174365 POINT(37.94684379453422 -123.25064188997018) bank174365 +174366 POINT(37.000084594527635 -123.30160697145331) bank174366 +174367 POINT(38.59073975333209 -123.33361254621171) bank174367 +174368 POINT(37.05371269089011 -123.14804710627288) bank174368 +174369 POINT(38.19371312178723 -122.84213546612447) bank174369 +174370 POINT(37.67311348900752 -122.62810395190166) bank174370 +174371 POINT(37.51641916004788 -121.65154908513446) bank174371 +174372 POINT(37.01668883355133 -121.91410613273406) bank174372 +174373 POINT(37.81249707346062 -122.89901317088601) bank174373 +174374 POINT(37.26001565827103 -121.69772483413925) bank174374 +174375 POINT(38.613478863276995 -121.44997735443387) bank174375 +174376 POINT(37.04307090532071 -123.2300788046781) bank174376 +174377 POINT(38.24441691681833 -122.95252330943315) bank174377 +174378 POINT(38.34482907227897 -122.34328001815334) bank174378 +174379 POINT(38.67434049417924 -122.70879741159324) bank174379 +174380 POINT(36.898292021668055 -122.70865574532358) bank174380 +174381 POINT(38.010698214803355 -122.83151666679271) bank174381 +174382 POINT(36.91159545315372 -122.30018764860748) bank174382 +174383 POINT(36.98749232928507 -121.75675181790511) bank174383 +174384 POINT(38.142227848947186 -121.61422331357763) bank174384 +174385 POINT(37.8239745872196 -123.29339648207372) bank174385 +174386 POINT(38.291545161683366 -122.40750288403362) bank174386 +174387 POINT(38.39810173375608 -122.0472371017788) bank174387 +174388 POINT(37.42085834237576 -123.35273181342225) bank174388 +174389 POINT(37.23331153519407 -122.10719249778397) bank174389 +174390 POINT(38.59731697971563 -122.638739989976) bank174390 +174391 POINT(37.74878419051585 -122.25578741553294) bank174391 +174392 POINT(37.10321026720227 -123.03216183862357) bank174392 +174393 POINT(38.45074284167761 -122.18094318285283) bank174393 +174394 POINT(36.80025707311231 -122.15017073399213) bank174394 +174395 POINT(38.24543956200135 -123.11646521557611) bank174395 +174396 POINT(37.062947334809685 -122.3324708376842) bank174396 +174397 POINT(38.00347286018198 -122.18770627602296) bank174397 +174398 POINT(37.62130421070549 -121.92812304585763) bank174398 +174399 POINT(37.55704480428285 -122.34885015486377) bank174399 +174400 POINT(38.5508280011658 -122.11675187107352) bank174400 +174401 POINT(37.57357662133841 -121.80777881241013) bank174401 +174402 POINT(37.438386082836885 -122.83462206865833) bank174402 +174403 POINT(37.87113143724325 -121.43540667247633) bank174403 +174404 POINT(38.70667352030848 -121.43255147704585) bank174404 +174405 POINT(37.70821982711484 -122.96389616365934) bank174405 +174406 POINT(37.723388400616216 -122.18067873469943) bank174406 +174407 POINT(37.05645826019695 -122.67324239818473) bank174407 +174408 POINT(38.509229687493864 -123.33973191869238) bank174408 +174409 POINT(38.385393742023986 -123.04922234521658) bank174409 +174410 POINT(37.636740450220344 -123.40687930958961) bank174410 +174411 POINT(38.19248994649717 -122.79078024556719) bank174411 +174412 POINT(37.15468290602876 -121.6713925194045) bank174412 +174413 POINT(38.764815082936906 -121.52690381512083) bank174413 +174414 POINT(38.137128699997945 -122.39044049184965) bank174414 +174415 POINT(37.74677616353562 -121.77303772402264) bank174415 +174416 POINT(36.97660380111495 -122.09872993444732) bank174416 +174417 POINT(37.37848797957048 -123.19515944334088) bank174417 +174418 POINT(38.26040641782471 -121.69472798010095) bank174418 +174419 POINT(38.591988729319525 -122.25866542109246) bank174419 +174420 POINT(38.2792991942161 -122.24015605665528) bank174420 +174421 POINT(37.104411106461704 -123.34606633217454) bank174421 +174422 POINT(37.01966478307585 -121.66395091106557) bank174422 +174423 POINT(37.32664350956649 -121.4946444074658) bank174423 +174424 POINT(37.61732794055603 -122.65357268940396) bank174424 +174425 POINT(37.28362714705124 -121.44483122227703) bank174425 +174426 POINT(37.8239039188248 -122.14416466894369) bank174426 +174427 POINT(36.956835492144684 -121.52356805102177) bank174427 +174428 POINT(37.240942384097856 -122.80724059197622) bank174428 +174429 POINT(37.05635826637158 -121.66039748777668) bank174429 +174430 POINT(36.797338390892044 -122.04087911948655) bank174430 +174431 POINT(38.24781208477646 -123.03529757067692) bank174431 +174432 POINT(38.38367629993979 -122.74014229847144) bank174432 +174433 POINT(36.803135294962566 -121.68603730705611) bank174433 +174434 POINT(36.79593206901424 -122.20873733263085) bank174434 +174435 POINT(37.32797014138179 -123.17297011549209) bank174435 +174436 POINT(37.828537189048625 -121.5196285776272) bank174436 +174437 POINT(37.239791615810724 -122.4196515628102) bank174437 +174438 POINT(38.138310467849266 -123.3855185425331) bank174438 +174439 POINT(36.83637403422417 -122.23391658854435) bank174439 +174440 POINT(37.166954333422545 -122.70337231920148) bank174440 +174441 POINT(37.120022049091716 -122.19687741801812) bank174441 +174442 POINT(37.553273311029024 -122.42922145876493) bank174442 +174443 POINT(38.66420986544539 -121.94197714308447) bank174443 +174444 POINT(38.414270055598344 -122.3978327938586) bank174444 +174445 POINT(37.67324293112654 -122.92045562604113) bank174445 +174446 POINT(37.85656538070594 -121.52665050326235) bank174446 +174447 POINT(36.79437579048618 -122.35119355876253) bank174447 +174448 POINT(38.65980471979788 -122.23778515678413) bank174448 +174449 POINT(38.698540656692074 -122.64209645305574) bank174449 +174450 POINT(36.96179819861125 -122.07871714696142) bank174450 +174451 POINT(36.84925399515186 -122.6464093083366) bank174451 +174452 POINT(38.162089278879975 -122.20019085069269) bank174452 +174453 POINT(38.38377560828396 -122.58567000063233) bank174453 +174454 POINT(38.51088515422231 -122.33513659603113) bank174454 +174455 POINT(36.83351839530606 -121.8373251494961) bank174455 +174456 POINT(38.07377146754239 -122.4149924923981) bank174456 +174457 POINT(37.85885203970326 -122.46614679610947) bank174457 +174458 POINT(38.25253697630672 -122.44503337625281) bank174458 +174459 POINT(37.44688315932724 -123.08799382592667) bank174459 +174460 POINT(38.34038309906924 -122.02662891263482) bank174460 +174461 POINT(36.90100798359425 -121.68141381541055) bank174461 +174462 POINT(37.025007150297526 -121.57603866526973) bank174462 +174463 POINT(37.17137512925323 -123.02185076511999) bank174463 +174464 POINT(37.39210432414934 -122.41256718449614) bank174464 +174465 POINT(37.7359168640798 -121.94301547123709) bank174465 +174466 POINT(37.95240539436575 -123.0319668320698) bank174466 +174467 POINT(38.60006624177707 -122.44789137212153) bank174467 +174468 POINT(37.20642116113194 -122.26309815920513) bank174468 +174469 POINT(38.166780784221224 -123.18090392829804) bank174469 +174470 POINT(37.37203306512202 -123.18492297457328) bank174470 +174471 POINT(37.71747383030054 -122.06450301874915) bank174471 +174472 POINT(37.059318914425255 -122.30612541426484) bank174472 +174473 POINT(36.982796426488534 -121.97919118410591) bank174473 +174474 POINT(38.54811403900231 -123.23895777256392) bank174474 +174475 POINT(37.81958217494338 -121.75507043474816) bank174475 +174476 POINT(37.28895335988881 -122.30724731237862) bank174476 +174477 POINT(37.10206132344908 -122.41177762531235) bank174477 +174478 POINT(38.574247180184074 -121.599376464308) bank174478 +174479 POINT(38.66353525951175 -122.43111197380357) bank174479 +174480 POINT(38.75902383354444 -121.72801906247074) bank174480 +174481 POINT(37.43848074642308 -123.24341574534716) bank174481 +174482 POINT(37.288666897470726 -121.93843323566811) bank174482 +174483 POINT(37.98830728266733 -122.68162848681553) bank174483 +174484 POINT(37.295373022093884 -123.04729388392633) bank174484 +174485 POINT(38.77036802403102 -122.80040808698294) bank174485 +174486 POINT(37.54261076191302 -121.4632292937492) bank174486 +174487 POINT(37.215933598582964 -122.43431807433339) bank174487 +174488 POINT(37.66091839134416 -123.25076526572026) bank174488 +174489 POINT(38.22962899389667 -121.94408525559669) bank174489 +174490 POINT(38.56451526873928 -122.15286206844229) bank174490 +174491 POINT(38.32473572938588 -122.34341667634754) bank174491 +174492 POINT(38.03725617001766 -122.53203876970827) bank174492 +174493 POINT(37.63285728566355 -122.03323377487318) bank174493 +174494 POINT(38.24563076746615 -122.77300473495774) bank174494 +174495 POINT(37.628672987231944 -123.14511069761632) bank174495 +174496 POINT(37.13379540134765 -122.38403428969228) bank174496 +174497 POINT(38.598656674577306 -122.37779192700262) bank174497 +174498 POINT(37.48550174039183 -122.48575748255189) bank174498 +174499 POINT(37.61300963512641 -123.37763888852102) bank174499 +174500 POINT(38.49106210600321 -123.18837693080512) bank174500 +174501 POINT(37.402480822054095 -122.5544599310562) bank174501 +174502 POINT(38.60641250792627 -123.31468658933731) bank174502 +174503 POINT(38.59343249743077 -121.65339117911739) bank174503 +174504 POINT(38.471712816764715 -122.82373434359245) bank174504 +174505 POINT(38.17350181070919 -123.13293742526344) bank174505 +174506 POINT(38.77179714666406 -123.00635005819464) bank174506 +174507 POINT(37.961338497416676 -122.92163128469087) bank174507 +174508 POINT(37.770885276358605 -122.93718758207477) bank174508 +174509 POINT(37.12888704517047 -122.69895312073913) bank174509 +174510 POINT(36.89334774134227 -121.85164163951481) bank174510 +174511 POINT(37.762331083462456 -122.4801431634151) bank174511 +174512 POINT(37.374992407941974 -121.59874710299515) bank174512 +174513 POINT(37.08465486365494 -122.8355337875609) bank174513 +174514 POINT(38.32023303760481 -121.89176000929051) bank174514 +174515 POINT(38.50728564261245 -123.17648649477741) bank174515 +174516 POINT(38.12163063248778 -122.33159929310312) bank174516 +174517 POINT(37.86521482173159 -121.91475287683703) bank174517 +174518 POINT(37.332079963118304 -121.63948597381605) bank174518 +174519 POINT(38.09660709991091 -122.8821077973244) bank174519 +174520 POINT(37.59974463001877 -122.57836128912794) bank174520 +174521 POINT(37.95850307397228 -122.39551916430055) bank174521 +174522 POINT(37.1085508590303 -122.56755636892117) bank174522 +174523 POINT(38.21172944572871 -122.3450803936005) bank174523 +174524 POINT(36.838281424833816 -121.9826567238883) bank174524 +174525 POINT(38.0319324725752 -123.35864453280249) bank174525 +174526 POINT(38.43635559277611 -122.05191977704519) bank174526 +174527 POINT(37.92515582606629 -122.11739676558612) bank174527 +174528 POINT(38.30536005769144 -121.50892358200944) bank174528 +174529 POINT(38.66470428714428 -121.44417103426932) bank174529 +174530 POINT(36.9303748341622 -122.47216609213258) bank174530 +174531 POINT(38.60247429504974 -121.66500685680975) bank174531 +174532 POINT(37.79265186485232 -122.59815257976315) bank174532 +174533 POINT(38.54786012753729 -121.634306595479) bank174533 +174534 POINT(37.16226481557459 -121.48178373251227) bank174534 +174535 POINT(38.738917470591005 -123.00047235457708) bank174535 +174536 POINT(37.619567457209605 -122.92030357837072) bank174536 +174537 POINT(38.41993534253687 -122.26561408387322) bank174537 +174538 POINT(37.42265131012372 -121.86237522798154) bank174538 +174539 POINT(36.95173497271434 -122.7714245095227) bank174539 +174540 POINT(38.730671231351415 -123.09782690601448) bank174540 +174541 POINT(37.25564084522128 -122.23334291180475) bank174541 +174542 POINT(37.18389051947713 -122.0577026760407) bank174542 +174543 POINT(37.41807501012991 -122.97882421406736) bank174543 +174544 POINT(37.5760688004622 -121.6680385235544) bank174544 +174545 POINT(38.12835454814037 -121.68241483354416) bank174545 +174546 POINT(37.120207010217705 -121.68596938165783) bank174546 +174547 POINT(38.02921711203631 -121.90715229996682) bank174547 +174548 POINT(37.319697022006814 -122.57109038823695) bank174548 +174549 POINT(36.979946891416816 -123.31061140574688) bank174549 +174550 POINT(37.81962831588828 -122.37633664278779) bank174550 +174551 POINT(37.20684445253571 -121.53367361523473) bank174551 +174552 POINT(37.60674821170338 -122.69660254977137) bank174552 +174553 POINT(37.00754611282477 -123.01724532229454) bank174553 +174554 POINT(37.38410438571756 -123.09716202265281) bank174554 +174555 POINT(37.75239467897356 -121.68665642596346) bank174555 +174556 POINT(37.08665701784475 -123.32443558499915) bank174556 +174557 POINT(37.2735451845749 -122.17938843303992) bank174557 +174558 POINT(37.880972242069035 -123.35165654601757) bank174558 +174559 POINT(37.07836496527635 -122.63436963149483) bank174559 +174560 POINT(38.651317184575944 -123.40772118457953) bank174560 +174561 POINT(37.91275002283168 -121.89788558089957) bank174561 +174562 POINT(37.31484484382662 -121.9210089384748) bank174562 +174563 POINT(37.87171223838376 -122.27629627054893) bank174563 +174564 POINT(37.3669820892433 -121.59534799812187) bank174564 +174565 POINT(37.83804504508939 -123.19819302033346) bank174565 +174566 POINT(36.84041047600105 -122.06933750417492) bank174566 +174567 POINT(37.04441798939712 -121.42296306614845) bank174567 +174568 POINT(38.3561447310327 -123.39593284333111) bank174568 +174569 POINT(38.06051543240096 -122.41931505299651) bank174569 +174570 POINT(38.451184510196434 -122.01693478058361) bank174570 +174571 POINT(37.48104307126085 -122.6120140736156) bank174571 +174572 POINT(37.1821622555474 -121.63111212168857) bank174572 +174573 POINT(38.7100995138892 -122.58228304614687) bank174573 +174574 POINT(38.59888180476245 -122.46639761482875) bank174574 +174575 POINT(36.790411493441724 -122.11375432450536) bank174575 +174576 POINT(37.72961097721716 -121.88725120332116) bank174576 +174577 POINT(37.00042622465716 -122.89858215500868) bank174577 +174578 POINT(38.01634764305586 -123.01330181342917) bank174578 +174579 POINT(38.24294095658296 -122.63201807947551) bank174579 +174580 POINT(37.76684543081724 -121.87079508050826) bank174580 +174581 POINT(36.8533752222865 -123.33277166652812) bank174581 +174582 POINT(37.029123938378994 -121.97356482752873) bank174582 +174583 POINT(37.946592333059066 -123.01615040322072) bank174583 +174584 POINT(36.95901907182112 -121.59040734256331) bank174584 +174585 POINT(38.51010562125673 -121.600932218831) bank174585 +174586 POINT(37.64690894226694 -123.00784168442576) bank174586 +174587 POINT(38.5408217516352 -123.29436758122962) bank174587 +174588 POINT(37.55180114685092 -123.0327123184095) bank174588 +174589 POINT(38.14766976349367 -122.81847941961549) bank174589 +174590 POINT(37.4392251898473 -123.38778873091334) bank174590 +174591 POINT(37.24141336206142 -123.00581118205056) bank174591 +174592 POINT(38.23479562447039 -123.27934038903065) bank174592 +174593 POINT(37.7659121158979 -122.1293815377379) bank174593 +174594 POINT(37.77804517840179 -121.43806379495065) bank174594 +174595 POINT(37.49676527166591 -122.90569189607257) bank174595 +174596 POINT(37.99280947015758 -122.55959906303772) bank174596 +174597 POINT(37.18252885603754 -122.41724923800352) bank174597 +174598 POINT(37.67744484662852 -123.31849280856025) bank174598 +174599 POINT(37.375442861668155 -122.85721492217525) bank174599 +174600 POINT(38.55270664543222 -123.07448120032498) bank174600 +174601 POINT(37.801109780452144 -123.07844282822886) bank174601 +174602 POINT(37.480980110703406 -122.37534373314588) bank174602 +174603 POINT(37.10555639304232 -123.40498444412015) bank174603 +174604 POINT(38.21662246415355 -122.40190025512466) bank174604 +174605 POINT(37.66026702447754 -122.19204370924626) bank174605 +174606 POINT(38.51541053760643 -122.25622537085535) bank174606 +174607 POINT(38.1242036909604 -121.6631242296075) bank174607 +174608 POINT(37.87631683890111 -123.17949693124146) bank174608 +174609 POINT(37.93852505336917 -122.32652891931556) bank174609 +174610 POINT(37.78575534463679 -122.96611262882135) bank174610 +174611 POINT(38.51538150291169 -123.0351377014929) bank174611 +174612 POINT(38.24857740289663 -122.67366948109773) bank174612 +174613 POINT(38.37902695960814 -122.99428468076253) bank174613 +174614 POINT(37.92946746434339 -122.026805346047) bank174614 +174615 POINT(37.416623045284425 -121.96007065131447) bank174615 +174616 POINT(38.23039087166977 -122.37365271807707) bank174616 +174617 POINT(38.4980053456512 -121.73114349642452) bank174617 +174618 POINT(38.27961728535967 -123.05964010729376) bank174618 +174619 POINT(37.35149901569517 -121.63455465975993) bank174619 +174620 POINT(37.9510708026169 -121.46560789254102) bank174620 +174621 POINT(37.44169157146965 -122.57950686394703) bank174621 +174622 POINT(37.927679586026066 -121.42420572875497) bank174622 +174623 POINT(38.25574860120233 -123.33944964914241) bank174623 +174624 POINT(37.95110728255377 -121.99884681176573) bank174624 +174625 POINT(37.565628020988385 -123.12204497754317) bank174625 +174626 POINT(37.040313982260585 -123.28069307070126) bank174626 +174627 POINT(37.41349588744747 -122.72749766881991) bank174627 +174628 POINT(38.22513344620081 -122.85786983407722) bank174628 +174629 POINT(38.18978905490823 -122.70468996284315) bank174629 +174630 POINT(38.35684789751502 -123.13290836618899) bank174630 +174631 POINT(38.56110365014675 -122.58712732856544) bank174631 +174632 POINT(38.44149627746445 -122.49964575288988) bank174632 +174633 POINT(38.12676068242296 -122.36162927139661) bank174633 +174634 POINT(37.66910724193537 -121.90615081240578) bank174634 +174635 POINT(37.0513712301858 -123.38717848181048) bank174635 +174636 POINT(36.81428695548981 -123.1298858599199) bank174636 +174637 POINT(37.22666550123717 -121.96068214078757) bank174637 +174638 POINT(37.93369315993437 -123.22326350755063) bank174638 +174639 POINT(37.93793535410179 -121.60699373259892) bank174639 +174640 POINT(38.06854214289914 -122.02433999275735) bank174640 +174641 POINT(37.607555628433374 -122.26914714618967) bank174641 +174642 POINT(37.305277051355404 -121.84526830932728) bank174642 +174643 POINT(38.255720440381864 -123.30814360367295) bank174643 +174644 POINT(37.781865304935785 -123.21604636331125) bank174644 +174645 POINT(38.557796292341216 -121.73421520624434) bank174645 +174646 POINT(38.74460241323132 -122.01545294201024) bank174646 +174647 POINT(38.45857047782048 -122.89373528569955) bank174647 +174648 POINT(37.06753612570313 -121.60357082495513) bank174648 +174649 POINT(38.180767005138335 -121.50053490410758) bank174649 +174650 POINT(38.54819130601037 -121.73031343932088) bank174650 +174651 POINT(37.6761296306908 -122.1337282481849) bank174651 +174652 POINT(36.91099182687006 -122.05223361223153) bank174652 +174653 POINT(37.306345331658534 -123.22834681709311) bank174653 +174654 POINT(37.867904913536215 -122.12024346683447) bank174654 +174655 POINT(37.122025152574 -123.41896898570835) bank174655 +174656 POINT(38.32983885179778 -121.58500772930343) bank174656 +174657 POINT(36.871244498713814 -122.29299116900086) bank174657 +174658 POINT(38.74999727829055 -122.43629041096003) bank174658 +174659 POINT(38.70450062846243 -123.03648965126608) bank174659 +174660 POINT(38.13404218592138 -122.01493514942516) bank174660 +174661 POINT(38.34911606552605 -122.2071681702235) bank174661 +174662 POINT(38.65345123891628 -121.8193972951241) bank174662 +174663 POINT(37.17566309790761 -122.19321675544998) bank174663 +174664 POINT(37.792838085083716 -123.40935751615805) bank174664 +174665 POINT(38.65994788058837 -121.60134102056847) bank174665 +174666 POINT(36.9739739368168 -122.56589073919552) bank174666 +174667 POINT(38.22222156274673 -121.45777887736519) bank174667 +174668 POINT(37.11166491139187 -122.46443146827525) bank174668 +174669 POINT(38.753057512808844 -122.94697850480651) bank174669 +174670 POINT(37.59530437423835 -121.71620315347288) bank174670 +174671 POINT(38.194375877148246 -122.18501865809111) bank174671 +174672 POINT(38.75718284568479 -123.27898688359468) bank174672 +174673 POINT(38.46248993804708 -122.52790348955838) bank174673 +174674 POINT(38.16706938635458 -123.0830404731215) bank174674 +174675 POINT(36.83193079091842 -121.48498947992871) bank174675 +174676 POINT(36.97381894994413 -123.13160408712329) bank174676 +174677 POINT(38.077567211071724 -122.39730473523746) bank174677 +174678 POINT(37.224090483243124 -122.6093081139863) bank174678 +174679 POINT(37.503326466589805 -122.9331717296688) bank174679 +174680 POINT(37.26016883830666 -121.8228021969242) bank174680 +174681 POINT(37.226319993409895 -121.6133293397218) bank174681 +174682 POINT(38.46923700962427 -123.2863232668948) bank174682 +174683 POINT(37.631699385138496 -121.88263151353094) bank174683 +174684 POINT(36.923597131702586 -122.76443825744917) bank174684 +174685 POINT(38.3002009698574 -122.00144358639109) bank174685 +174686 POINT(38.252108600330324 -122.80525296315508) bank174686 +174687 POINT(36.94287260117915 -122.2799061650983) bank174687 +174688 POINT(36.97171651959208 -122.01967619776349) bank174688 +174689 POINT(37.86833810361113 -121.82767633443929) bank174689 +174690 POINT(37.74801007655611 -122.41657549255712) bank174690 +174691 POINT(36.86646773708165 -123.35371984332116) bank174691 +174692 POINT(37.61189790527121 -122.19360183859588) bank174692 +174693 POINT(38.03319135153771 -121.44526671964125) bank174693 +174694 POINT(38.34077810143617 -123.17584871547065) bank174694 +174695 POINT(37.96613012086322 -122.07351555646441) bank174695 +174696 POINT(37.331229071687886 -122.83616549756042) bank174696 +174697 POINT(37.07944370871666 -121.76555993217069) bank174697 +174698 POINT(37.86438753478133 -123.36640246755246) bank174698 +174699 POINT(37.50308714597432 -122.47323001395327) bank174699 +174700 POINT(37.75495885735194 -123.18860202943988) bank174700 +174701 POINT(38.26321065474188 -121.94490103940406) bank174701 +174702 POINT(38.268864295030006 -121.93859592818251) bank174702 +174703 POINT(38.30985352564923 -123.1055176034078) bank174703 +174704 POINT(38.18597038545105 -121.75902075200908) bank174704 +174705 POINT(37.963472766495556 -121.81538912490437) bank174705 +174706 POINT(37.883711353998166 -122.32940720668596) bank174706 +174707 POINT(37.1321717943646 -122.2701385428717) bank174707 +174708 POINT(37.53765826363259 -123.07064722681591) bank174708 +174709 POINT(38.772794694280144 -122.45084934994334) bank174709 +174710 POINT(38.33266353216401 -122.67398066660739) bank174710 +174711 POINT(38.51274154379318 -123.40783437744165) bank174711 +174712 POINT(37.47379443940997 -121.63539685877443) bank174712 +174713 POINT(36.89798799911927 -121.5175847787238) bank174713 +174714 POINT(37.81952356380977 -123.11875157126056) bank174714 +174715 POINT(37.4845691299876 -122.41705998282626) bank174715 +174716 POINT(36.79399662044004 -122.46204945714196) bank174716 +174717 POINT(37.3945959729259 -123.17366605466245) bank174717 +174718 POINT(37.721322015461624 -121.7845777636886) bank174718 +174719 POINT(38.26097005003081 -123.30686373553506) bank174719 +174720 POINT(37.85067600168417 -122.92242976219816) bank174720 +174721 POINT(38.54012229502575 -122.35200041106894) bank174721 +174722 POINT(37.29249894993724 -123.32396512449995) bank174722 +174723 POINT(37.61898404966034 -121.49075399990649) bank174723 +174724 POINT(37.196367440057074 -121.97927603833105) bank174724 +174725 POINT(38.14939319589158 -123.39566517669448) bank174725 +174726 POINT(38.20763972656168 -123.30969753531534) bank174726 +174727 POINT(38.39473372437297 -122.5845158791135) bank174727 +174728 POINT(38.079126453125454 -122.5336280928249) bank174728 +174729 POINT(37.19446343059877 -122.68514891091718) bank174729 +174730 POINT(38.29306697610799 -123.21277076815659) bank174730 +174731 POINT(37.06710063142049 -122.52695505862465) bank174731 +174732 POINT(37.55605354581071 -122.60522527312739) bank174732 +174733 POINT(38.612269051431895 -121.92731762208105) bank174733 +174734 POINT(38.02544155073657 -123.15784895818216) bank174734 +174735 POINT(38.3151185612441 -122.5755569743997) bank174735 +174736 POINT(36.894951117385375 -121.98425619326858) bank174736 +174737 POINT(38.20909209833325 -121.87615508556019) bank174737 +174738 POINT(38.60098386886959 -122.86485018397119) bank174738 +174739 POINT(37.72434236034042 -122.91055615709834) bank174739 +174740 POINT(37.53590011504784 -123.23841707418393) bank174740 +174741 POINT(37.24561561684524 -122.399281852274) bank174741 +174742 POINT(37.69285076792266 -123.17365058167742) bank174742 +174743 POINT(37.26220744768789 -122.31698281310311) bank174743 +174744 POINT(37.64865426839714 -122.02775145732764) bank174744 +174745 POINT(37.08329853589639 -122.98237433032077) bank174745 +174746 POINT(37.75864712714146 -122.28007702233074) bank174746 +174747 POINT(38.15390767508455 -122.75970084237208) bank174747 +174748 POINT(37.582638506718595 -123.11682562719888) bank174748 +174749 POINT(37.394602069387275 -123.16565472645419) bank174749 +174750 POINT(38.284809269604786 -121.89952380967502) bank174750 +174751 POINT(36.79978110455413 -122.92247088983237) bank174751 +174752 POINT(37.51876573102924 -121.96734418612094) bank174752 +174753 POINT(38.50778761143112 -122.97817621329133) bank174753 +174754 POINT(37.02348630104243 -121.91473343740005) bank174754 +174755 POINT(37.73159575067498 -122.32891652961003) bank174755 +174756 POINT(37.52728529680025 -122.00795655954146) bank174756 +174757 POINT(37.89445630707397 -123.24448028060195) bank174757 +174758 POINT(37.90419957066941 -123.05219514646826) bank174758 +174759 POINT(37.46969760935022 -122.08124684399863) bank174759 +174760 POINT(38.43991076746996 -123.05695606887822) bank174760 +174761 POINT(37.990824875486474 -121.71685699411118) bank174761 +174762 POINT(38.43705763813729 -122.7359858147005) bank174762 +174763 POINT(36.835597034607744 -122.80519375114909) bank174763 +174764 POINT(37.6299900183927 -122.78886636169264) bank174764 +174765 POINT(36.89876963476478 -123.17177970098072) bank174765 +174766 POINT(37.63486090778604 -122.29655410262804) bank174766 +174767 POINT(37.8672270364691 -122.53512405008189) bank174767 +174768 POINT(37.835457270424726 -122.85074433081496) bank174768 +174769 POINT(38.253704259153345 -122.31449841176313) bank174769 +174770 POINT(36.944316810573866 -122.6564995355812) bank174770 +174771 POINT(38.14823730800713 -121.56671366953118) bank174771 +174772 POINT(38.47053273633763 -122.10271338244762) bank174772 +174773 POINT(38.25493532313482 -123.14504562129491) bank174773 +174774 POINT(38.75737531297199 -123.0395182346129) bank174774 +174775 POINT(38.73454930143764 -122.41995388554433) bank174775 +174776 POINT(37.98110044348191 -122.01708295094534) bank174776 +174777 POINT(37.67791467625715 -121.90172988020326) bank174777 +174778 POINT(38.34821715929985 -121.6381735985319) bank174778 +174779 POINT(38.21437510857533 -123.35002315517794) bank174779 +174780 POINT(37.30923669125035 -122.6184146784895) bank174780 +174781 POINT(37.44391546367227 -123.21536819942762) bank174781 +174782 POINT(37.9965041256588 -121.60281744723419) bank174782 +174783 POINT(37.36976475367679 -123.40540240248912) bank174783 +174784 POINT(37.07049466403061 -121.82631380464416) bank174784 +174785 POINT(37.73279904348862 -122.56269080903283) bank174785 +174786 POINT(37.64608964272749 -122.78413504167135) bank174786 +174787 POINT(37.07824429769836 -123.01150735764693) bank174787 +174788 POINT(37.952570890564296 -121.76142103551959) bank174788 +174789 POINT(37.621865682364394 -121.59311936052872) bank174789 +174790 POINT(37.35090602783584 -122.83879175139754) bank174790 +174791 POINT(37.80067512130293 -122.76817854827898) bank174791 +174792 POINT(37.83027234730314 -121.50097440120659) bank174792 +174793 POINT(37.11733943787062 -121.45015548794956) bank174793 +174794 POINT(38.22125247045138 -121.49104861589399) bank174794 +174795 POINT(38.26476126815299 -123.19474108847812) bank174795 +174796 POINT(37.459109255365384 -122.26601301706178) bank174796 +174797 POINT(38.28172872391876 -122.43767907694188) bank174797 +174798 POINT(37.20689071588312 -121.54664444565336) bank174798 +174799 POINT(38.15780573015863 -121.92970058739132) bank174799 +174800 POINT(38.65701905866725 -121.83987384498379) bank174800 +174801 POINT(38.28024608658864 -122.46386110599862) bank174801 +174802 POINT(37.06267899296492 -123.04928375982745) bank174802 +174803 POINT(37.15695115455834 -122.46930446469713) bank174803 +174804 POINT(37.93023615429058 -122.91093830457903) bank174804 +174805 POINT(37.44995700815756 -123.04947118489586) bank174805 +174806 POINT(38.62205225205828 -123.00575357119573) bank174806 +174807 POINT(38.02581610583392 -121.67370476771906) bank174807 +174808 POINT(37.22926476965022 -123.20939533626131) bank174808 +174809 POINT(37.19085662711026 -123.11201359277719) bank174809 +174810 POINT(37.81830979914678 -122.02088062745032) bank174810 +174811 POINT(36.91056511186295 -123.08351716212897) bank174811 +174812 POINT(38.01403310968671 -123.00119253101136) bank174812 +174813 POINT(37.25681382719374 -122.71705753131394) bank174813 +174814 POINT(37.78265786441819 -121.77083670599394) bank174814 +174815 POINT(37.78855263899535 -121.5612868219864) bank174815 +174816 POINT(38.43457700871327 -123.17252322998988) bank174816 +174817 POINT(37.18340465926357 -122.29865842282493) bank174817 +174818 POINT(37.055840703389116 -122.33698003621832) bank174818 +174819 POINT(37.136454682957215 -121.92928426569817) bank174819 +174820 POINT(38.36649793161325 -122.1038177987001) bank174820 +174821 POINT(36.82241514054845 -121.6997505556717) bank174821 +174822 POINT(37.934890761048514 -122.04500759357857) bank174822 +174823 POINT(37.05591375806447 -122.54296301230606) bank174823 +174824 POINT(37.69433100206342 -122.93163216124442) bank174824 +174825 POINT(36.779766315451376 -123.05930435481874) bank174825 +174826 POINT(38.20324657034461 -121.55765698753645) bank174826 +174827 POINT(36.983416053915555 -121.74521946054466) bank174827 +174828 POINT(38.7016506365994 -123.36468250272634) bank174828 +174829 POINT(37.69402397920791 -122.55286479338974) bank174829 +174830 POINT(37.812140091412715 -122.97783883183223) bank174830 +174831 POINT(37.692029812468505 -122.67949438970425) bank174831 +174832 POINT(37.203562958046255 -123.10242204024698) bank174832 +174833 POINT(38.77235195117495 -122.11476238281296) bank174833 +174834 POINT(36.89732703708137 -121.70684126763341) bank174834 +174835 POINT(37.241009436932416 -121.50987893042173) bank174835 +174836 POINT(38.06078434511287 -122.51215410843511) bank174836 +174837 POINT(38.76057703499102 -121.82536153567037) bank174837 +174838 POINT(37.048167853845065 -122.42479571833294) bank174838 +174839 POINT(38.636412735740954 -122.9763842921237) bank174839 +174840 POINT(37.148541287876604 -122.66278213103604) bank174840 +174841 POINT(37.72627577940554 -123.31839481545127) bank174841 +174842 POINT(37.73995979785567 -121.61732122719481) bank174842 +174843 POINT(38.48240974187297 -122.68886658868597) bank174843 +174844 POINT(37.06568340929758 -122.57231482898884) bank174844 +174845 POINT(36.852660623700714 -123.40440461106672) bank174845 +174846 POINT(37.67095275909172 -123.24313274208457) bank174846 +174847 POINT(37.70022191375998 -122.99054996454892) bank174847 +174848 POINT(37.43115120747514 -123.31021740048793) bank174848 +174849 POINT(37.85108613938926 -123.12214726771658) bank174849 +174850 POINT(37.32955027029418 -123.40608658036508) bank174850 +174851 POINT(38.38006844481803 -122.43091030327295) bank174851 +174852 POINT(37.56126137433123 -122.28911696028636) bank174852 +174853 POINT(36.86858590361703 -122.56967359374597) bank174853 +174854 POINT(38.29394660333421 -121.57853007307239) bank174854 +174855 POINT(37.201801329145056 -121.88709058323357) bank174855 +174856 POINT(38.558993838048 -122.04210070420127) bank174856 +174857 POINT(37.23077310397951 -121.86082859442438) bank174857 +174858 POINT(37.399699567775386 -121.99139246825128) bank174858 +174859 POINT(37.24007885650055 -122.272224133936) bank174859 +174860 POINT(38.522711887266944 -122.50136942106654) bank174860 +174861 POINT(38.54707865128184 -122.79662402984209) bank174861 +174862 POINT(38.27122077764759 -122.74955218083574) bank174862 +174863 POINT(38.472013073669174 -121.77982138030002) bank174863 +174864 POINT(38.77085304208118 -121.61061003743133) bank174864 +174865 POINT(38.055077794473206 -123.28555310871495) bank174865 +174866 POINT(38.68293955617754 -122.6941564737286) bank174866 +174867 POINT(37.01183756918004 -123.24394332881062) bank174867 +174868 POINT(37.17582466849393 -123.29610974055623) bank174868 +174869 POINT(38.02196519303035 -122.83445954983459) bank174869 +174870 POINT(37.083570001001824 -122.89884550868423) bank174870 +174871 POINT(38.76251259632287 -122.34581676823865) bank174871 +174872 POINT(37.538333373517375 -123.30886668796322) bank174872 +174873 POINT(38.52022190679666 -121.7021706076915) bank174873 +174874 POINT(36.89774352588511 -122.902865800417) bank174874 +174875 POINT(37.8600055834068 -122.62601958763403) bank174875 +174876 POINT(38.74720005618389 -123.02748607919867) bank174876 +174877 POINT(38.40892136123574 -123.33656314153139) bank174877 +174878 POINT(38.38222339386135 -122.22279927027888) bank174878 +174879 POINT(38.238247762606925 -122.61838527364945) bank174879 +174880 POINT(36.7846954347961 -123.13056047409262) bank174880 +174881 POINT(37.130196303546114 -122.20862759014389) bank174881 +174882 POINT(38.03277265153018 -122.24862954464857) bank174882 +174883 POINT(38.29274204640249 -123.39720094715776) bank174883 +174884 POINT(38.26537604547044 -122.3211365841992) bank174884 +174885 POINT(36.88410060603174 -123.06515160706084) bank174885 +174886 POINT(37.201692785408326 -121.71079623181676) bank174886 +174887 POINT(37.92244871981319 -121.86549447224249) bank174887 +174888 POINT(37.66506386301562 -123.1214722646857) bank174888 +174889 POINT(37.81254319588425 -122.58228494992873) bank174889 +174890 POINT(37.62605330367569 -121.69615728321271) bank174890 +174891 POINT(37.047308180053584 -121.92179880788045) bank174891 +174892 POINT(38.266282564406644 -122.05011055906269) bank174892 +174893 POINT(38.482110389627245 -122.82617065841185) bank174893 +174894 POINT(37.14449516919428 -121.6892391082307) bank174894 +174895 POINT(37.604117773692664 -121.6695418453302) bank174895 +174896 POINT(37.22659504818885 -122.89034615926545) bank174896 +174897 POINT(37.04983381675759 -122.58363261143855) bank174897 +174898 POINT(38.7199864991747 -122.01141979827325) bank174898 +174899 POINT(38.65568333734342 -122.91310701686264) bank174899 +174900 POINT(37.34588135910173 -123.12128733815352) bank174900 +174901 POINT(37.44070957750158 -121.87276052692107) bank174901 +174902 POINT(36.79236196373333 -122.04310969091027) bank174902 +174903 POINT(38.10867892535842 -122.95529957909545) bank174903 +174904 POINT(38.71312106701984 -121.66401207487006) bank174904 +174905 POINT(38.74453761465264 -122.27785227423469) bank174905 +174906 POINT(38.26639176485851 -122.84157775722998) bank174906 +174907 POINT(37.34006448364053 -121.71432126765235) bank174907 +174908 POINT(38.16172620287522 -122.79247332532293) bank174908 +174909 POINT(38.67197944995136 -122.29305172050452) bank174909 +174910 POINT(37.56024437936056 -121.42587337450776) bank174910 +174911 POINT(38.48782156120582 -123.11781341215574) bank174911 +174912 POINT(38.49802654869745 -121.68057851046326) bank174912 +174913 POINT(37.67466037029372 -122.64172606212945) bank174913 +174914 POINT(36.92793015893848 -122.58113681033461) bank174914 +174915 POINT(38.48198967099096 -122.34458503421709) bank174915 +174916 POINT(37.73150897196799 -123.18167710749655) bank174916 +174917 POINT(37.49238225265106 -122.56905080735108) bank174917 +174918 POINT(37.76327161428039 -122.09597910654884) bank174918 +174919 POINT(36.88336529904653 -122.66452931393036) bank174919 +174920 POINT(37.63928147903924 -121.67127298724013) bank174920 +174921 POINT(37.255348345122435 -122.31375268497504) bank174921 +174922 POINT(38.58490616781648 -123.31665183041204) bank174922 +174923 POINT(38.507013040628095 -122.04901053023927) bank174923 +174924 POINT(38.76017223226348 -122.22602381435563) bank174924 +174925 POINT(38.61588960219873 -122.86440305136205) bank174925 +174926 POINT(37.529620380066824 -122.17977999828237) bank174926 +174927 POINT(38.48551626713205 -123.16244256552567) bank174927 +174928 POINT(37.99916683239183 -122.85659940457103) bank174928 +174929 POINT(37.12799420286046 -121.5715871457396) bank174929 +174930 POINT(36.89718762736146 -122.67176465673084) bank174930 +174931 POINT(38.51945382986665 -122.47283654057829) bank174931 +174932 POINT(38.115684889478125 -123.28664875533299) bank174932 +174933 POINT(38.125958037332495 -122.7763635838358) bank174933 +174934 POINT(36.77897186612557 -122.76953696236217) bank174934 +174935 POINT(37.26699994066676 -121.59014787598439) bank174935 +174936 POINT(38.52828705894732 -123.27737956403287) bank174936 +174937 POINT(37.829230995938346 -122.68283859157759) bank174937 +174938 POINT(38.71275399677032 -123.12262147010493) bank174938 +174939 POINT(36.78698738559636 -122.50130534157014) bank174939 +174940 POINT(37.77861635877948 -121.68957969975001) bank174940 +174941 POINT(36.951071264165016 -122.28665393650486) bank174941 +174942 POINT(37.48982822647501 -123.32548136473692) bank174942 +174943 POINT(37.843152290076254 -122.38880273682453) bank174943 +174944 POINT(37.406091398655164 -122.84005622307959) bank174944 +174945 POINT(38.62042962641089 -122.34336178754214) bank174945 +174946 POINT(38.117513849545794 -122.68015910403717) bank174946 +174947 POINT(38.22121859455647 -123.02490426559346) bank174947 +174948 POINT(37.978047768131745 -123.27544206248736) bank174948 +174949 POINT(37.736943951810886 -123.14992534832814) bank174949 +174950 POINT(38.141701450658125 -121.56649712404294) bank174950 +174951 POINT(37.77054876678727 -122.37890050405531) bank174951 +174952 POINT(38.657943987783874 -123.31927996249269) bank174952 +174953 POINT(38.20400199857297 -121.71250317971408) bank174953 +174954 POINT(37.81581607357399 -123.2242238697408) bank174954 +174955 POINT(37.52933511087268 -121.81545188585321) bank174955 +174956 POINT(38.36064734022535 -122.96350738385206) bank174956 +174957 POINT(38.66119139917324 -123.11152607594596) bank174957 +174958 POINT(38.62040731427409 -121.61759382065246) bank174958 +174959 POINT(38.399575231783025 -122.69521606200522) bank174959 +174960 POINT(38.443649507814825 -122.69801230532858) bank174960 +174961 POINT(37.04692948991468 -122.1857825350658) bank174961 +174962 POINT(38.222706013604835 -121.98239190256322) bank174962 +174963 POINT(37.504875323739796 -122.19336768304498) bank174963 +174964 POINT(36.788712475979594 -121.67945584916502) bank174964 +174965 POINT(38.70937670387984 -122.13495554074355) bank174965 +174966 POINT(38.11703972885059 -122.62257042610466) bank174966 +174967 POINT(37.89634911581446 -122.33911443679877) bank174967 +174968 POINT(38.004098982233344 -123.03448185018866) bank174968 +174969 POINT(37.011116365655816 -121.51290339591759) bank174969 +174970 POINT(36.802062939760575 -123.28580148848985) bank174970 +174971 POINT(37.968070994535985 -121.45845133690204) bank174971 +174972 POINT(36.93570429755793 -123.39735086166912) bank174972 +174973 POINT(37.981372056050475 -122.8314973642616) bank174973 +174974 POINT(37.36846678131287 -121.89103050248285) bank174974 +174975 POINT(37.278088776649334 -122.59682403993381) bank174975 +174976 POINT(38.57919710970487 -123.28791786840695) bank174976 +174977 POINT(37.25032790914376 -121.64488803402827) bank174977 +174978 POINT(38.4545584296428 -122.16500908043996) bank174978 +174979 POINT(37.59743908013094 -121.96758832290185) bank174979 +174980 POINT(36.777449774764335 -123.28275265161056) bank174980 +174981 POINT(38.1426257354874 -121.4225806857168) bank174981 +174982 POINT(36.83640696676884 -122.96710118176918) bank174982 +174983 POINT(37.23566991291275 -122.15010266894826) bank174983 +174984 POINT(37.41278650641879 -121.46776311222047) bank174984 +174985 POINT(37.997300000927396 -121.62030086291513) bank174985 +174986 POINT(37.83197431252493 -122.56652081872998) bank174986 +174987 POINT(37.243168963921356 -121.77159760153822) bank174987 +174988 POINT(37.9384477445618 -121.83744939214) bank174988 +174989 POINT(37.780851368195115 -123.32749705968695) bank174989 +174990 POINT(38.18306848253782 -123.30060297795613) bank174990 +174991 POINT(36.854040965909775 -123.36289857219481) bank174991 +174992 POINT(38.32076731609083 -122.72517724641051) bank174992 +174993 POINT(37.163171950031845 -122.43611908671633) bank174993 +174994 POINT(37.496033752333595 -122.39602997161703) bank174994 +174995 POINT(36.82415631592326 -122.96478246151013) bank174995 +174996 POINT(37.37009493700202 -123.38597246129267) bank174996 +174997 POINT(37.344993047315306 -122.4587570252508) bank174997 +174998 POINT(38.507936452350464 -121.56406148789856) bank174998 +174999 POINT(38.62440433647377 -121.71058106037597) bank174999 +175000 POINT(37.53347221731597 -122.53920662338001) bank175000 +175001 POINT(37.72090523947329 -123.07477165815749) bank175001 +175002 POINT(37.07109784212948 -121.62578338782937) bank175002 +175003 POINT(38.19565125860388 -122.65992586463526) bank175003 +175004 POINT(37.54966363650825 -122.99151308322303) bank175004 +175005 POINT(37.293449512904715 -122.59696537870516) bank175005 +175006 POINT(38.061934923019166 -121.62581324391839) bank175006 +175007 POINT(37.72410547081167 -122.39636817486121) bank175007 +175008 POINT(37.435346486329145 -121.64867349183272) bank175008 +175009 POINT(37.63902715665316 -123.41730413432104) bank175009 +175010 POINT(36.855458388768724 -121.5869824730717) bank175010 +175011 POINT(36.9667912012209 -123.08622894318181) bank175011 +175012 POINT(38.47843267495242 -122.3327620901525) bank175012 +175013 POINT(37.64058867432007 -121.99962854163869) bank175013 +175014 POINT(36.914363382081454 -121.78486188319734) bank175014 +175015 POINT(37.11069179971386 -122.12133845223501) bank175015 +175016 POINT(38.70897793487322 -122.21725683021286) bank175016 +175017 POINT(37.927840743715834 -121.96271936885385) bank175017 +175018 POINT(37.14156375002514 -123.41276454052172) bank175018 +175019 POINT(37.432085062283114 -123.24354675762821) bank175019 +175020 POINT(37.84843445514867 -122.35077523103895) bank175020 +175021 POINT(37.654832610118014 -122.89754721003727) bank175021 +175022 POINT(38.11486501745783 -122.99013522830653) bank175022 +175023 POINT(38.77162794729074 -122.27758370848224) bank175023 +175024 POINT(38.3486969198128 -122.78593362308801) bank175024 +175025 POINT(37.94397182344732 -121.60813691357146) bank175025 +175026 POINT(38.19855510831391 -123.32382443352037) bank175026 +175027 POINT(36.93259102542423 -122.68372662092554) bank175027 +175028 POINT(36.878970447467836 -121.5684334605074) bank175028 +175029 POINT(37.491323017514986 -122.33310165793037) bank175029 +175030 POINT(37.06058510269828 -122.54190194839408) bank175030 +175031 POINT(38.072925383395315 -121.87949454391578) bank175031 +175032 POINT(36.89852110445657 -122.39463699161253) bank175032 +175033 POINT(36.90430602593973 -123.11277624044524) bank175033 +175034 POINT(38.677088004442226 -122.39991921684269) bank175034 +175035 POINT(38.699716899452184 -123.1616091762227) bank175035 +175036 POINT(37.1007866428886 -123.20929082066172) bank175036 +175037 POINT(37.381634097883065 -121.65998838981626) bank175037 +175038 POINT(37.28919292396995 -121.63272905214225) bank175038 +175039 POINT(38.13163777405103 -122.9303697918466) bank175039 +175040 POINT(37.40284017822454 -123.16108901730233) bank175040 +175041 POINT(37.982525628277195 -122.24430981855518) bank175041 +175042 POINT(38.28558359193355 -122.96202266554192) bank175042 +175043 POINT(37.54644413540676 -122.44107033685918) bank175043 +175044 POINT(37.51913666018575 -122.04022966350904) bank175044 +175045 POINT(38.313328636133846 -122.96979810144435) bank175045 +175046 POINT(37.351945009465354 -121.7111902037497) bank175046 +175047 POINT(38.576550500814015 -122.19479861278421) bank175047 +175048 POINT(37.430051395290754 -122.1925554664235) bank175048 +175049 POINT(38.180934333902016 -122.03192187501307) bank175049 +175050 POINT(37.65123434818894 -123.28654494536018) bank175050 +175051 POINT(37.825444479453296 -122.29113922905472) bank175051 +175052 POINT(38.63656163297125 -121.65511084424591) bank175052 +175053 POINT(37.26292831321174 -122.85400100988237) bank175053 +175054 POINT(37.74272919033896 -122.69544557101175) bank175054 +175055 POINT(36.79423787398188 -122.61271011512522) bank175055 +175056 POINT(37.303629756314834 -122.46879456806607) bank175056 +175057 POINT(37.71458160110115 -122.07451939041556) bank175057 +175058 POINT(37.91725785103968 -123.35720105311033) bank175058 +175059 POINT(37.32115397046354 -122.50546338684326) bank175059 +175060 POINT(36.91342154136238 -122.62575140765145) bank175060 +175061 POINT(38.57320205236658 -121.68161191880601) bank175061 +175062 POINT(37.46623661107562 -122.8351708483225) bank175062 +175063 POINT(38.51982152247696 -121.48739441280333) bank175063 +175064 POINT(37.997304954718466 -123.02869282362451) bank175064 +175065 POINT(37.69480473973437 -122.58061499703948) bank175065 +175066 POINT(37.546799018702366 -122.77039922241701) bank175066 +175067 POINT(37.91879509556409 -122.87919006773102) bank175067 +175068 POINT(36.92094246909027 -122.54628518758514) bank175068 +175069 POINT(38.7419639627582 -121.88118020744076) bank175069 +175070 POINT(37.98590375939335 -123.09403471770334) bank175070 +175071 POINT(37.468587633132415 -122.71921263239085) bank175071 +175072 POINT(36.924225642662584 -121.45679810181957) bank175072 +175073 POINT(38.03778806774544 -122.30399320378785) bank175073 +175074 POINT(37.968962705176516 -121.79623035845623) bank175074 +175075 POINT(36.946409109666746 -122.67974126445922) bank175075 +175076 POINT(38.10171177323578 -122.0518459629001) bank175076 +175077 POINT(37.878793812602694 -122.9938302958622) bank175077 +175078 POINT(38.17237415482933 -121.84691458405678) bank175078 +175079 POINT(36.88042194909742 -121.42380328112017) bank175079 +175080 POINT(37.11209258948556 -123.26787212447901) bank175080 +175081 POINT(37.41150255456273 -122.09549606292987) bank175081 +175082 POINT(38.275630621719245 -122.6826833816057) bank175082 +175083 POINT(37.81649241985639 -121.59000310841019) bank175083 +175084 POINT(38.17370198788161 -122.94942916804837) bank175084 +175085 POINT(38.671866996972675 -122.01580763327767) bank175085 +175086 POINT(37.58164464828725 -123.13391816834775) bank175086 +175087 POINT(37.72267237781776 -122.30344930996733) bank175087 +175088 POINT(37.2023573625922 -122.85670021353073) bank175088 +175089 POINT(37.264226210983104 -123.33591012184853) bank175089 +175090 POINT(37.063425721751 -122.28295374994204) bank175090 +175091 POINT(38.63282954149534 -121.83147033901587) bank175091 +175092 POINT(37.214549800833694 -121.68879228913796) bank175092 +175093 POINT(38.42066990199256 -121.56052773219538) bank175093 +175094 POINT(37.465308219095036 -122.06690617346194) bank175094 +175095 POINT(38.16955783552055 -122.36744364470499) bank175095 +175096 POINT(37.465379282583385 -122.44048763454495) bank175096 +175097 POINT(37.892877746639634 -122.19643353863623) bank175097 +175098 POINT(37.90868544330341 -122.6443755807205) bank175098 +175099 POINT(37.04521438480387 -122.74145514795855) bank175099 +175100 POINT(37.00654498526938 -122.81862595759338) bank175100 +175101 POINT(38.07070270758091 -123.26783658749919) bank175101 +175102 POINT(36.88423577266096 -122.90639585731024) bank175102 +175103 POINT(38.459151860791565 -122.94862100041588) bank175103 +175104 POINT(38.545902602638904 -122.2719336323913) bank175104 +175105 POINT(38.431419202840516 -121.77263577982157) bank175105 +175106 POINT(37.04045669640034 -121.65935477518724) bank175106 +175107 POINT(37.48181121023727 -123.09860658400397) bank175107 +175108 POINT(37.1062640112787 -121.51988484783783) bank175108 +175109 POINT(36.796162061616414 -122.73041960384757) bank175109 +175110 POINT(37.18582768799108 -123.21033845303997) bank175110 +175111 POINT(37.245042267669255 -122.51040096094145) bank175111 +175112 POINT(37.31733312074143 -123.0169206843154) bank175112 +175113 POINT(37.41116733171251 -123.14976062268224) bank175113 +175114 POINT(37.324455300490534 -121.63916169269594) bank175114 +175115 POINT(37.0628908580778 -123.2591891170731) bank175115 +175116 POINT(37.867984065601604 -122.3591541364021) bank175116 +175117 POINT(38.61076641009035 -122.16183682656863) bank175117 +175118 POINT(36.92292419466361 -122.66148576274172) bank175118 +175119 POINT(37.886374635104794 -122.2482905670148) bank175119 +175120 POINT(37.81256121984245 -122.02307151969654) bank175120 +175121 POINT(37.04357068502537 -122.10040131782198) bank175121 +175122 POINT(38.526399042085856 -121.75293360334545) bank175122 +175123 POINT(38.70235429741126 -121.83823398969153) bank175123 +175124 POINT(37.72958814351724 -122.74348224508347) bank175124 +175125 POINT(38.755846080147684 -123.14314835809532) bank175125 +175126 POINT(37.46330835449778 -123.04185417828339) bank175126 +175127 POINT(36.774979309987295 -122.4369021402633) bank175127 +175128 POINT(37.589675368681384 -122.67503446087977) bank175128 +175129 POINT(38.183110610204096 -123.02227200343441) bank175129 +175130 POINT(37.24342468452848 -122.67837406929478) bank175130 +175131 POINT(37.101236793658906 -122.74007837013772) bank175131 +175132 POINT(38.54554939389405 -123.218209664655) bank175132 +175133 POINT(37.247230770246986 -123.37882552451212) bank175133 +175134 POINT(37.8488337730767 -121.80510350802471) bank175134 +175135 POINT(38.23044235192483 -123.22536472700948) bank175135 +175136 POINT(37.55070790957789 -122.58162701802503) bank175136 +175137 POINT(36.8323808757603 -122.7378556276493) bank175137 +175138 POINT(37.15093201183912 -121.98963522431443) bank175138 +175139 POINT(37.70801043989639 -121.76962269644935) bank175139 +175140 POINT(36.97314061431992 -122.55314020445407) bank175140 +175141 POINT(37.1598532951334 -123.1005146677877) bank175141 +175142 POINT(37.61749367016203 -122.49778868880064) bank175142 +175143 POINT(38.36883368251262 -121.99254540427756) bank175143 +175144 POINT(38.312149955827365 -122.5113661612204) bank175144 +175145 POINT(37.63381705134114 -121.7842662166521) bank175145 +175146 POINT(36.822221177118024 -122.35334282004851) bank175146 +175147 POINT(38.77277459146222 -122.58945406649639) bank175147 +175148 POINT(38.2343796216182 -122.15983104691705) bank175148 +175149 POINT(37.56279871812711 -123.1375648724056) bank175149 +175150 POINT(38.023668334221874 -123.04128224828234) bank175150 +175151 POINT(36.830067987148475 -123.09423460177769) bank175151 +175152 POINT(38.76772353291331 -122.4262649259393) bank175152 +175153 POINT(38.59780748536447 -122.56262619830358) bank175153 +175154 POINT(37.81033324736867 -123.19230643869354) bank175154 +175155 POINT(37.48835213903452 -122.13572816078698) bank175155 +175156 POINT(37.088105476928405 -123.30054264969257) bank175156 +175157 POINT(38.031120123706046 -122.72914396942863) bank175157 +175158 POINT(37.06696757058308 -121.52298162241622) bank175158 +175159 POINT(38.20492642336566 -122.49441207030732) bank175159 +175160 POINT(37.08035191328841 -122.70424062441842) bank175160 +175161 POINT(37.77124683828148 -122.59703125909779) bank175161 +175162 POINT(37.783330344055656 -122.82162125746275) bank175162 +175163 POINT(37.51313390119752 -122.8589913032091) bank175163 +175164 POINT(37.14578600751931 -121.64234250336699) bank175164 +175165 POINT(38.640162951827016 -122.85916547422306) bank175165 +175166 POINT(37.17577606312404 -121.60075900608109) bank175166 +175167 POINT(36.88426239941962 -122.87501954156951) bank175167 +175168 POINT(37.556531686843876 -122.8722159338544) bank175168 +175169 POINT(36.93019857891793 -121.95559267313928) bank175169 +175170 POINT(37.435454599455156 -123.04103170996136) bank175170 +175171 POINT(37.83247604943946 -122.94184087389274) bank175171 +175172 POINT(36.84325793150329 -122.3774922116824) bank175172 +175173 POINT(37.074690375671906 -121.9874104831364) bank175173 +175174 POINT(37.90051306554047 -122.49056846990743) bank175174 +175175 POINT(38.263467723921835 -121.57449520954756) bank175175 +175176 POINT(37.99559403089618 -122.06667978452737) bank175176 +175177 POINT(37.170679425521854 -122.93101691805752) bank175177 +175178 POINT(37.79380318630745 -121.78412350711625) bank175178 +175179 POINT(37.669654824529815 -123.25473986475788) bank175179 +175180 POINT(37.33341746074821 -122.66341376341688) bank175180 +175181 POINT(38.7610814287145 -122.8295611135227) bank175181 +175182 POINT(37.65082558066943 -122.65148215461177) bank175182 +175183 POINT(38.32608201140314 -122.40664454733725) bank175183 +175184 POINT(37.13563851567133 -121.9611323980134) bank175184 +175185 POINT(37.234524513618446 -123.15790765243692) bank175185 +175186 POINT(36.829689197812016 -123.27343144563586) bank175186 +175187 POINT(37.004020127546156 -121.6404278912943) bank175187 +175188 POINT(36.90617628705362 -123.21813901598459) bank175188 +175189 POINT(37.28817257945843 -122.50295240053958) bank175189 +175190 POINT(37.19842859986256 -122.46259115267075) bank175190 +175191 POINT(38.23962227713615 -122.82865719574943) bank175191 +175192 POINT(38.245300419862424 -121.59650676207838) bank175192 +175193 POINT(37.9398948329241 -121.93448484236366) bank175193 +175194 POINT(38.38211829898026 -122.74559149133052) bank175194 +175195 POINT(37.88692993304654 -121.8307461912134) bank175195 +175196 POINT(37.512588145398716 -123.14993627719096) bank175196 +175197 POINT(37.36939970693201 -122.22658269115) bank175197 +175198 POINT(37.91444975694404 -121.77351349582784) bank175198 +175199 POINT(38.543619933962816 -121.85388933643536) bank175199 +175200 POINT(37.747982806759495 -123.024745840777) bank175200 +175201 POINT(38.6335609079718 -121.686440041873) bank175201 +175202 POINT(38.51056078122773 -121.83664026106491) bank175202 +175203 POINT(36.7829135013532 -121.47189787647798) bank175203 +175204 POINT(37.99039611834058 -122.18041595698641) bank175204 +175205 POINT(37.04234303304439 -121.87712170601807) bank175205 +175206 POINT(38.40761253738828 -122.61908375596015) bank175206 +175207 POINT(37.49373224869821 -121.95928012607236) bank175207 +175208 POINT(37.839010413509584 -122.53074073968274) bank175208 +175209 POINT(36.80904745822659 -122.07308786827737) bank175209 +175210 POINT(38.18803478542073 -122.8322582211505) bank175210 +175211 POINT(38.73096118281481 -123.10719283886789) bank175211 +175212 POINT(38.0439364496445 -123.11791907429269) bank175212 +175213 POINT(38.63001183659499 -122.33199826938872) bank175213 +175214 POINT(38.29975257279296 -122.37246849523991) bank175214 +175215 POINT(38.2398957317743 -121.84376294870357) bank175215 +175216 POINT(37.144092188475085 -122.23141106863564) bank175216 +175217 POINT(37.84080603057665 -121.76176166325448) bank175217 +175218 POINT(36.78280018029944 -122.12042528657062) bank175218 +175219 POINT(37.41092513445038 -121.97995953686569) bank175219 +175220 POINT(38.70892022611728 -122.49503403736871) bank175220 +175221 POINT(38.17693613987863 -121.8240323584232) bank175221 +175222 POINT(36.903479099531786 -123.31376680359922) bank175222 +175223 POINT(37.3342146276908 -123.27391943337697) bank175223 +175224 POINT(38.665137809453896 -121.6415691460315) bank175224 +175225 POINT(38.55169606668819 -123.16793567424207) bank175225 +175226 POINT(37.00348912347088 -122.08787231912105) bank175226 +175227 POINT(37.84580825122461 -122.37208358440606) bank175227 +175228 POINT(37.73142368796351 -122.54917124653026) bank175228 +175229 POINT(38.062838735106475 -123.06335469123867) bank175229 +175230 POINT(37.76998431461885 -122.3191453422866) bank175230 +175231 POINT(38.0179765365324 -123.03035515797217) bank175231 +175232 POINT(37.994454258085895 -122.02057857883732) bank175232 +175233 POINT(38.30625105551235 -122.05711563603518) bank175233 +175234 POINT(38.189735314301075 -122.10019892062127) bank175234 +175235 POINT(38.74408030708979 -122.85400173943144) bank175235 +175236 POINT(37.11344866357554 -122.12433494966037) bank175236 +175237 POINT(37.734162690711834 -123.03471769598465) bank175237 +175238 POINT(37.16588570999317 -121.48714987976257) bank175238 +175239 POINT(38.33993284475851 -121.91925128085963) bank175239 +175240 POINT(38.69142656161139 -122.8011271978208) bank175240 +175241 POINT(37.4876657729232 -121.78077552080802) bank175241 +175242 POINT(37.19076189369768 -122.94593050831871) bank175242 +175243 POINT(37.26208647697298 -121.81858308895221) bank175243 +175244 POINT(36.89284443818115 -122.0296671817281) bank175244 +175245 POINT(37.46477476975543 -122.44189044357287) bank175245 +175246 POINT(37.9385976944818 -123.37758988357128) bank175246 +175247 POINT(37.870330711441014 -122.71616401506543) bank175247 +175248 POINT(38.608906643856145 -123.23292079135483) bank175248 +175249 POINT(38.32789349450166 -122.03477444931868) bank175249 +175250 POINT(37.1490327025069 -122.34976732487229) bank175250 +175251 POINT(38.44111313998896 -122.67835932338082) bank175251 +175252 POINT(38.09588884576063 -121.60074562773326) bank175252 +175253 POINT(37.89664237575698 -121.50625548134647) bank175253 +175254 POINT(38.51163560894028 -122.68541572625176) bank175254 +175255 POINT(37.95867522844316 -121.47934953813653) bank175255 +175256 POINT(36.962437138363384 -122.81811250725175) bank175256 +175257 POINT(38.2686674614369 -122.96815602961502) bank175257 +175258 POINT(37.9242079900648 -122.01229059885316) bank175258 +175259 POINT(37.64019673693903 -122.53722115630596) bank175259 +175260 POINT(38.490461122019106 -123.24772533796946) bank175260 +175261 POINT(37.15072300577178 -123.05666518873883) bank175261 +175262 POINT(38.4642894523875 -122.8977697967814) bank175262 +175263 POINT(37.66350846183437 -123.0550818732628) bank175263 +175264 POINT(38.685268643809245 -122.15906228308974) bank175264 +175265 POINT(37.45170812754043 -123.33070798312531) bank175265 +175266 POINT(38.09409263996884 -123.11485909288977) bank175266 +175267 POINT(37.47817957995573 -122.93997619908227) bank175267 +175268 POINT(38.23504966644562 -122.60394721590042) bank175268 +175269 POINT(36.98339273390793 -122.16583441628933) bank175269 +175270 POINT(38.17027551974583 -122.52479484079653) bank175270 +175271 POINT(36.97293631001029 -122.98964203969953) bank175271 +175272 POINT(38.37104765172526 -122.25737489956349) bank175272 +175273 POINT(38.206011302975746 -122.97434261878911) bank175273 +175274 POINT(37.026995942801946 -121.59996393646321) bank175274 +175275 POINT(38.73216453551728 -122.69450303552449) bank175275 +175276 POINT(38.312609473205015 -123.00386909728867) bank175276 +175277 POINT(38.59089133049393 -123.26130449592766) bank175277 +175278 POINT(37.20882990416017 -121.45938497250664) bank175278 +175279 POINT(37.413113733825526 -121.58978553867179) bank175279 +175280 POINT(37.09414239979452 -122.54582504482731) bank175280 +175281 POINT(37.510274721024246 -122.23850765770786) bank175281 +175282 POINT(38.69721483837037 -122.0305961037851) bank175282 +175283 POINT(37.94242953602178 -123.02778779750258) bank175283 +175284 POINT(37.40884738742197 -121.62754160717742) bank175284 +175285 POINT(38.54589610233094 -122.51894069602443) bank175285 +175286 POINT(38.145049754071835 -122.50271203788516) bank175286 +175287 POINT(37.155637325245856 -121.90858759385168) bank175287 +175288 POINT(38.33813315598311 -122.54222745410178) bank175288 +175289 POINT(37.34689837592892 -122.08367092346764) bank175289 +175290 POINT(37.75799812947804 -121.56739769345955) bank175290 +175291 POINT(37.683123100429036 -121.73646104432231) bank175291 +175292 POINT(38.47938484324144 -122.43888570205017) bank175292 +175293 POINT(36.854695091658684 -122.3964288455187) bank175293 +175294 POINT(37.169335489272015 -122.60168217408251) bank175294 +175295 POINT(37.073533302567924 -122.52340227297368) bank175295 +175296 POINT(37.212010258863224 -122.58837885035918) bank175296 +175297 POINT(37.20428125119644 -121.67586822298972) bank175297 +175298 POINT(38.01983609016842 -123.29920723150445) bank175298 +175299 POINT(38.51363585967542 -121.51757660677832) bank175299 +175300 POINT(38.06051063793989 -123.40136240280185) bank175300 +175301 POINT(36.8407078527545 -122.00854446759466) bank175301 +175302 POINT(38.67129883713467 -121.86520038402851) bank175302 +175303 POINT(36.885103283912926 -123.00658016522995) bank175303 +175304 POINT(37.308239798973084 -123.13037999566548) bank175304 +175305 POINT(37.06396032587513 -122.51801034707867) bank175305 +175306 POINT(38.68423652201488 -121.44444453360643) bank175306 +175307 POINT(36.939579425983524 -122.65464828052674) bank175307 +175308 POINT(38.30181375498321 -123.19798821313218) bank175308 +175309 POINT(37.41740748500312 -123.34997838722647) bank175309 +175310 POINT(36.97504024017265 -122.48998745497136) bank175310 +175311 POINT(38.02932830283958 -122.98447738668533) bank175311 +175312 POINT(37.39549789911521 -122.46611772953015) bank175312 +175313 POINT(37.05846725776614 -123.08772534622328) bank175313 +175314 POINT(38.276577572689085 -122.62408276879026) bank175314 +175315 POINT(36.907278544020606 -122.55133254592509) bank175315 +175316 POINT(38.35414289024793 -122.83372288031897) bank175316 +175317 POINT(37.22402416446727 -122.5530601802823) bank175317 +175318 POINT(38.230246042014116 -122.55150676736768) bank175318 +175319 POINT(37.185397555251406 -122.93867472158318) bank175319 +175320 POINT(37.644982753851735 -122.2134449680151) bank175320 +175321 POINT(37.45510325038892 -122.02247644880904) bank175321 +175322 POINT(37.754358553783796 -121.47953841753207) bank175322 +175323 POINT(38.16335650058404 -122.3218323477889) bank175323 +175324 POINT(37.82837287980035 -123.19119897875119) bank175324 +175325 POINT(37.38026681187357 -122.837658184795) bank175325 +175326 POINT(36.82815402817373 -122.27720903359356) bank175326 +175327 POINT(37.78295723636511 -121.42712241534561) bank175327 +175328 POINT(37.45950320961737 -121.74493041705924) bank175328 +175329 POINT(36.99537114214115 -122.53739502608431) bank175329 +175330 POINT(36.9984997461809 -121.6120588600668) bank175330 +175331 POINT(37.99219829862952 -122.93626363115126) bank175331 +175332 POINT(37.96079537260473 -122.55503781784361) bank175332 +175333 POINT(38.29743798333213 -122.95118655110525) bank175333 +175334 POINT(36.98862251270497 -122.57506479252264) bank175334 +175335 POINT(37.85383654030702 -122.65017184921584) bank175335 +175336 POINT(38.43969855054125 -122.09503114734716) bank175336 +175337 POINT(37.20108710471466 -122.34051771193278) bank175337 +175338 POINT(36.981580050993664 -121.73434749868255) bank175338 +175339 POINT(37.57376147939844 -122.41390615794828) bank175339 +175340 POINT(37.89767830545995 -121.60508555362179) bank175340 +175341 POINT(37.15613306324319 -123.24840117605225) bank175341 +175342 POINT(38.24645678320453 -123.13597825751198) bank175342 +175343 POINT(37.3624346093012 -122.72510155702464) bank175343 +175344 POINT(37.1387776727509 -123.24347900972414) bank175344 +175345 POINT(37.60340464355164 -122.59107549915025) bank175345 +175346 POINT(38.21997834068246 -122.56078948160636) bank175346 +175347 POINT(37.74773751599537 -121.60623088433273) bank175347 +175348 POINT(38.17084241106805 -123.40216550184748) bank175348 +175349 POINT(37.872656803522645 -121.69980342140944) bank175349 +175350 POINT(38.40652316953847 -121.55820602434785) bank175350 +175351 POINT(37.59287625550629 -123.32966790766397) bank175351 +175352 POINT(37.03395920964749 -123.08083707084188) bank175352 +175353 POINT(37.58290307507044 -121.53975738294423) bank175353 +175354 POINT(36.95959595729858 -122.06079129978123) bank175354 +175355 POINT(37.351566171337964 -122.5298586552104) bank175355 +175356 POINT(38.3155689213775 -123.33545608356565) bank175356 +175357 POINT(38.464268271341034 -121.49322075606499) bank175357 +175358 POINT(37.85690035215834 -121.73967499749789) bank175358 +175359 POINT(37.283853520769576 -121.76343920567129) bank175359 +175360 POINT(37.618065238620304 -123.30207496774362) bank175360 +175361 POINT(38.15722172044825 -122.49257574386802) bank175361 +175362 POINT(37.316512247677245 -122.57201699681956) bank175362 +175363 POINT(38.50176919513682 -122.24135997375079) bank175363 +175364 POINT(36.90399075862387 -122.86892404679105) bank175364 +175365 POINT(37.850657384061414 -122.54065447343797) bank175365 +175366 POINT(37.341334220542436 -122.91608843789251) bank175366 +175367 POINT(36.8363208951417 -122.46232516061711) bank175367 +175368 POINT(37.95452183654571 -122.76638219405508) bank175368 +175369 POINT(37.97641784145695 -122.14402550101072) bank175369 +175370 POINT(36.834497756864685 -122.26483985929865) bank175370 +175371 POINT(37.790700907144135 -122.09773460054006) bank175371 +175372 POINT(38.74912438451744 -121.99453404360538) bank175372 +175373 POINT(38.599078620193474 -121.50701737401191) bank175373 +175374 POINT(37.82559118318518 -121.48201780384652) bank175374 +175375 POINT(38.35591846653688 -122.48760877516634) bank175375 +175376 POINT(37.55471504700945 -122.53820071269054) bank175376 +175377 POINT(38.01284093251814 -121.85057997178001) bank175377 +175378 POINT(37.60598788838286 -122.57769722229479) bank175378 +175379 POINT(37.61428535612549 -121.85034716290474) bank175379 +175380 POINT(37.48542305835247 -122.95369497816324) bank175380 +175381 POINT(37.842083831764434 -121.98886642623343) bank175381 +175382 POINT(37.03184965150621 -121.6189548836065) bank175382 +175383 POINT(38.513408979434566 -123.36992943197757) bank175383 +175384 POINT(38.6130723847761 -123.07451875133562) bank175384 +175385 POINT(37.99331657347199 -122.98913517928521) bank175385 +175386 POINT(38.52014635602231 -122.07958020457193) bank175386 +175387 POINT(36.917867168548696 -122.68759265637048) bank175387 +175388 POINT(37.42176966459907 -123.16099679974359) bank175388 +175389 POINT(37.13002963975337 -121.55518041693644) bank175389 +175390 POINT(38.17904393146095 -123.38486976670255) bank175390 +175391 POINT(37.49620678662673 -121.7555450755057) bank175391 +175392 POINT(36.80388896981733 -121.42714776236163) bank175392 +175393 POINT(36.85736029854465 -121.73139073921469) bank175393 +175394 POINT(36.940174878168335 -121.97949576259373) bank175394 +175395 POINT(37.021236052115746 -121.96103119344684) bank175395 +175396 POINT(38.28251425655964 -122.92827990808387) bank175396 +175397 POINT(38.09253505836504 -122.53952506385853) bank175397 +175398 POINT(37.20294113929379 -122.2914120455927) bank175398 +175399 POINT(37.55637500833026 -122.61074273277558) bank175399 +175400 POINT(36.94363802288299 -121.5504917718604) bank175400 +175401 POINT(37.45955460177923 -121.63948986194362) bank175401 +175402 POINT(37.99266929112163 -123.12273858886269) bank175402 +175403 POINT(37.626234378291656 -122.69354213740446) bank175403 +175404 POINT(37.09841310810808 -122.92169505918928) bank175404 +175405 POINT(37.001907502914335 -122.90390733911266) bank175405 +175406 POINT(37.494466983509085 -121.44343043143445) bank175406 +175407 POINT(37.64624293381097 -122.64907714109951) bank175407 +175408 POINT(38.40123573236524 -122.40985449664885) bank175408 +175409 POINT(36.905677870903276 -122.89323565885222) bank175409 +175410 POINT(38.33305187995705 -122.18349995203751) bank175410 +175411 POINT(37.83727761202006 -122.66331015620656) bank175411 +175412 POINT(37.94854875069324 -123.17439752075066) bank175412 +175413 POINT(36.852926119575116 -122.80653897437406) bank175413 +175414 POINT(37.19486175900862 -122.25935383579893) bank175414 +175415 POINT(38.50431971021845 -122.19481511133158) bank175415 +175416 POINT(37.145176627193614 -122.63124348066864) bank175416 +175417 POINT(37.460991214434124 -122.34784613334489) bank175417 +175418 POINT(38.52456328684603 -121.65709015197734) bank175418 +175419 POINT(37.87672783281629 -122.99000704928712) bank175419 +175420 POINT(37.38199802060365 -122.25648419232455) bank175420 +175421 POINT(38.77052710813598 -123.15162929140672) bank175421 +175422 POINT(36.85656570243617 -121.51788991557115) bank175422 +175423 POINT(38.503261729181695 -123.36926053199252) bank175423 +175424 POINT(37.927594877217636 -121.51044689716369) bank175424 +175425 POINT(37.99380534817557 -122.41303663544947) bank175425 +175426 POINT(36.932183364482974 -123.26423042572438) bank175426 +175427 POINT(37.03372483312798 -122.99761458543485) bank175427 +175428 POINT(38.5467530206143 -121.59058726546645) bank175428 +175429 POINT(36.839359006381514 -122.25479086748308) bank175429 +175430 POINT(38.54602754370643 -123.04444259562278) bank175430 +175431 POINT(38.106317307129146 -121.90796871054272) bank175431 +175432 POINT(38.70841381647409 -123.20055603322172) bank175432 +175433 POINT(37.94041146135911 -122.86808250533673) bank175433 +175434 POINT(37.16976421416419 -122.72050873579124) bank175434 +175435 POINT(37.19794770602625 -121.87765642187448) bank175435 +175436 POINT(38.39233832030104 -122.90837873789503) bank175436 +175437 POINT(36.93494611201101 -122.65618016451266) bank175437 +175438 POINT(37.0480978852617 -121.59739397053937) bank175438 +175439 POINT(37.13084229151787 -121.99833984104954) bank175439 +175440 POINT(37.82557638989791 -122.11413957624652) bank175440 +175441 POINT(37.36657624177322 -123.23186430481192) bank175441 +175442 POINT(37.42027142879637 -122.21407401696212) bank175442 +175443 POINT(38.18523023829129 -121.7818348417348) bank175443 +175444 POINT(36.98420893846632 -123.10545792464133) bank175444 +175445 POINT(37.91486773605949 -121.7104355475666) bank175445 +175446 POINT(37.4086120128905 -123.2426536169184) bank175446 +175447 POINT(37.77408972467 -122.3466681033578) bank175447 +175448 POINT(37.17875094822727 -122.62738218892854) bank175448 +175449 POINT(37.91211072645998 -122.15153546246722) bank175449 +175450 POINT(37.13211974418129 -123.38173198047487) bank175450 +175451 POINT(37.62724199258081 -122.78035363701063) bank175451 +175452 POINT(36.821351072964674 -121.93949916263342) bank175452 +175453 POINT(38.36524982765263 -121.69243477527493) bank175453 +175454 POINT(38.51536068598321 -123.21465818254107) bank175454 +175455 POINT(38.089825805818876 -122.67443076695662) bank175455 +175456 POINT(37.34362537627434 -123.03756423507551) bank175456 +175457 POINT(36.818952224490246 -123.30947578098733) bank175457 +175458 POINT(38.177695549647915 -122.2403747017567) bank175458 +175459 POINT(37.80925740446131 -122.95717386193402) bank175459 +175460 POINT(37.45275910805407 -121.80053528360058) bank175460 +175461 POINT(36.87580852095874 -122.622168986805) bank175461 +175462 POINT(38.27805964271613 -121.84884466591217) bank175462 +175463 POINT(37.11422757348632 -121.83666913232948) bank175463 +175464 POINT(37.30212027134469 -121.7305025698789) bank175464 +175465 POINT(37.47319346710017 -121.64061604709792) bank175465 +175466 POINT(38.34906259798703 -122.34443940263131) bank175466 +175467 POINT(38.334581435239336 -122.6984522271004) bank175467 +175468 POINT(36.99969942547138 -122.81859776471863) bank175468 +175469 POINT(37.47515275944315 -122.08966787321071) bank175469 +175470 POINT(37.970362351195384 -122.3687125981421) bank175470 +175471 POINT(38.182792551293765 -123.19073171457383) bank175471 +175472 POINT(38.22992460149894 -122.89114975430051) bank175472 +175473 POINT(38.09996631915284 -123.20148621220625) bank175473 +175474 POINT(37.567605193997885 -122.61959431832804) bank175474 +175475 POINT(38.06982222196099 -122.34236854928105) bank175475 +175476 POINT(36.943622580761414 -121.99946523502257) bank175476 +175477 POINT(37.80729155402402 -122.23803339789329) bank175477 +175478 POINT(37.26503396236049 -121.80726182326698) bank175478 +175479 POINT(38.284223508840086 -122.84369864209131) bank175479 +175480 POINT(37.26387633258943 -123.24894044047974) bank175480 +175481 POINT(38.66502358073974 -123.41295445965935) bank175481 +175482 POINT(37.255004024739115 -121.84694596558937) bank175482 +175483 POINT(37.85254447024847 -122.63790326587772) bank175483 +175484 POINT(37.16660120017779 -121.49954703074087) bank175484 +175485 POINT(36.80347123321132 -121.48560884810281) bank175485 +175486 POINT(37.048971687217154 -121.93458423573672) bank175486 +175487 POINT(38.02956412820878 -123.17579069886882) bank175487 +175488 POINT(37.81205682303076 -123.29599452599047) bank175488 +175489 POINT(38.43207424751013 -121.49196795182907) bank175489 +175490 POINT(37.44011824873715 -122.26157014005575) bank175490 +175491 POINT(36.978095847477874 -123.21686460588738) bank175491 +175492 POINT(37.162386013170206 -121.42789083999673) bank175492 +175493 POINT(38.71711559769781 -123.17982985138546) bank175493 +175494 POINT(37.28453278545677 -121.57011140912923) bank175494 +175495 POINT(37.69733159697869 -122.20433040870009) bank175495 +175496 POINT(37.13582954488781 -122.55159764344566) bank175496 +175497 POINT(37.8055784850236 -122.36790395495045) bank175497 +175498 POINT(37.22052639973115 -122.63591242361278) bank175498 +175499 POINT(38.678276495031625 -121.68650448725322) bank175499 +175500 POINT(37.572169883541406 -122.91504202816408) bank175500 +175501 POINT(36.78474567805487 -121.80979302291871) bank175501 +175502 POINT(37.96088279463117 -123.00049477893917) bank175502 +175503 POINT(37.820133113112675 -122.65160746634115) bank175503 +175504 POINT(38.200595281319224 -121.57082640840618) bank175504 +175505 POINT(37.059146733774305 -121.77140151033475) bank175505 +175506 POINT(37.99274255685589 -122.68488967169125) bank175506 +175507 POINT(37.41661839418398 -122.575760661816) bank175507 +175508 POINT(37.05622749355697 -121.44448865733136) bank175508 +175509 POINT(37.525781811631006 -123.33800732064687) bank175509 +175510 POINT(38.31707463549495 -123.34030849031579) bank175510 +175511 POINT(37.189581368562834 -122.7298794722394) bank175511 +175512 POINT(38.14139916946692 -121.90811490964357) bank175512 +175513 POINT(37.95404831068605 -121.46791949534013) bank175513 +175514 POINT(38.33552553370539 -122.64164588742206) bank175514 +175515 POINT(37.32210380575905 -122.396898989368) bank175515 +175516 POINT(38.57627491882794 -122.56461989873256) bank175516 +175517 POINT(38.69350376451228 -121.94347242449344) bank175517 +175518 POINT(37.19381162276649 -122.7359831319859) bank175518 +175519 POINT(38.35476714930695 -121.64070534653314) bank175519 +175520 POINT(37.826130405258006 -122.18779623986981) bank175520 +175521 POINT(37.657814308301944 -121.60736712233532) bank175521 +175522 POINT(37.27022021919009 -121.556525736747) bank175522 +175523 POINT(38.0494569599809 -122.78447937580987) bank175523 +175524 POINT(37.970319410166496 -121.45887453102348) bank175524 +175525 POINT(37.172679342000265 -123.30468791064729) bank175525 +175526 POINT(37.1137085609633 -122.68973433907816) bank175526 +175527 POINT(36.891635151038074 -122.73787725496318) bank175527 +175528 POINT(38.56187689015531 -122.17060162985945) bank175528 +175529 POINT(38.468920593351214 -123.19638263968295) bank175529 +175530 POINT(38.39316604679446 -122.1039413052734) bank175530 +175531 POINT(37.84288563852783 -123.13844057507222) bank175531 +175532 POINT(36.974753692229875 -122.50627760267777) bank175532 +175533 POINT(37.394171616737495 -122.67755708377554) bank175533 +175534 POINT(37.46539948404983 -121.95393724766568) bank175534 +175535 POINT(36.87119383527964 -122.81379897060518) bank175535 +175536 POINT(37.96464415777333 -121.71707221135564) bank175536 +175537 POINT(37.54990531294928 -122.92248248754136) bank175537 +175538 POINT(38.042739758795825 -121.78598672617251) bank175538 +175539 POINT(38.114233619326754 -123.0055651934794) bank175539 +175540 POINT(37.405168862174094 -122.26559110261354) bank175540 +175541 POINT(37.76393515677592 -122.43204027207321) bank175541 +175542 POINT(37.16855069251622 -122.52820651455426) bank175542 +175543 POINT(37.340162886991564 -121.93459704565574) bank175543 +175544 POINT(37.17978187281738 -123.31575421104273) bank175544 +175545 POINT(37.28311401010232 -122.87904622336924) bank175545 +175546 POINT(37.183551563787816 -122.46885001526807) bank175546 +175547 POINT(37.40470569379235 -123.30105247447094) bank175547 +175548 POINT(37.747005308657 -122.03103156852586) bank175548 +175549 POINT(37.80752605625448 -122.69685376015399) bank175549 +175550 POINT(37.208456899976206 -121.76337409853048) bank175550 +175551 POINT(38.55479186351656 -122.03833802251903) bank175551 +175552 POINT(38.21151632581917 -122.26929165734244) bank175552 +175553 POINT(37.96150882318334 -121.76994892774712) bank175553 +175554 POINT(38.33261525319091 -122.56454136365456) bank175554 +175555 POINT(37.89240483593502 -122.68850747602656) bank175555 +175556 POINT(37.71610239359268 -123.26016228701437) bank175556 +175557 POINT(37.96489988574478 -121.98413276796278) bank175557 +175558 POINT(37.022079801284185 -122.56223667348617) bank175558 +175559 POINT(37.260121988367644 -122.73023530167745) bank175559 +175560 POINT(37.8484393039025 -122.77685390065422) bank175560 +175561 POINT(37.41164699697969 -122.42722450020105) bank175561 +175562 POINT(37.99317975037235 -122.57642373045428) bank175562 +175563 POINT(37.9393130657963 -121.60927658974018) bank175563 +175564 POINT(38.74261524746181 -122.53421972118484) bank175564 +175565 POINT(37.48194345104566 -121.71701876854237) bank175565 +175566 POINT(36.79042874999218 -122.49067203674393) bank175566 +175567 POINT(36.9374883466347 -121.96411083103399) bank175567 +175568 POINT(38.582834745339696 -122.28945476344664) bank175568 +175569 POINT(38.739761850197034 -123.24191429872839) bank175569 +175570 POINT(37.76490452075114 -122.83201141628882) bank175570 +175571 POINT(37.05087788147947 -122.59745446871982) bank175571 +175572 POINT(38.420577658586666 -122.86711524130556) bank175572 +175573 POINT(37.270519130176105 -123.32879462605878) bank175573 +175574 POINT(38.13683883155434 -122.35339495459074) bank175574 +175575 POINT(37.374856312952836 -123.30566482770209) bank175575 +175576 POINT(38.14192848091634 -121.61571835259022) bank175576 +175577 POINT(37.49520125293567 -123.03719026344088) bank175577 +175578 POINT(38.70857187277026 -122.25443604374414) bank175578 +175579 POINT(37.46763287604225 -121.69727082033253) bank175579 +175580 POINT(36.879576924605516 -122.58850930516225) bank175580 +175581 POINT(36.90124721913753 -122.460725260124) bank175581 +175582 POINT(38.691988491387306 -122.68679676670361) bank175582 +175583 POINT(38.15153755808746 -123.28686326052801) bank175583 +175584 POINT(38.13939336033257 -121.49125564300651) bank175584 +175585 POINT(37.775009587184485 -123.28269910597952) bank175585 +175586 POINT(37.323102189763496 -121.59873950651017) bank175586 +175587 POINT(37.02747974193928 -122.26415370650798) bank175587 +175588 POINT(37.308732442219046 -122.88720501997663) bank175588 +175589 POINT(38.658177523252384 -123.10027786795067) bank175589 +175590 POINT(37.698533577673665 -123.41902145113052) bank175590 +175591 POINT(38.738667481408015 -123.05708649371078) bank175591 +175592 POINT(37.6653110179079 -122.67596756408194) bank175592 +175593 POINT(38.00717559499369 -123.18677804742633) bank175593 +175594 POINT(38.10652317997508 -122.72234478217612) bank175594 +175595 POINT(37.934355001342716 -122.81492003442065) bank175595 +175596 POINT(37.79586828113909 -122.52979299277634) bank175596 +175597 POINT(38.3059026104953 -121.47505974877727) bank175597 +175598 POINT(36.8867427946288 -123.18775946776647) bank175598 +175599 POINT(38.06734724940045 -123.0220290002037) bank175599 +175600 POINT(38.222904262208985 -121.54786338756615) bank175600 +175601 POINT(37.27887005388073 -122.44762410991099) bank175601 +175602 POINT(37.68731579929991 -123.22645854150238) bank175602 +175603 POINT(37.093518940787625 -122.33979150894932) bank175603 +175604 POINT(38.11161036642314 -122.17306138413142) bank175604 +175605 POINT(37.267492053430246 -122.94208135055965) bank175605 +175606 POINT(38.318217966280656 -121.61976362057891) bank175606 +175607 POINT(37.81635877271849 -123.19020821423433) bank175607 +175608 POINT(38.33750143984 -123.16929931383447) bank175608 +175609 POINT(38.298930828440355 -122.81607775150228) bank175609 +175610 POINT(37.79064853346817 -122.53450277090974) bank175610 +175611 POINT(38.007263692957274 -122.11928154066517) bank175611 +175612 POINT(37.28765192042088 -123.11257730202553) bank175612 +175613 POINT(37.2178550449936 -121.45800436041215) bank175613 +175614 POINT(37.82484037898409 -121.69619047987658) bank175614 +175615 POINT(38.65401318026522 -121.73384241135396) bank175615 +175616 POINT(37.230712470615025 -122.84021822813554) bank175616 +175617 POINT(37.860274513042164 -123.03433339301858) bank175617 +175618 POINT(37.64332120247066 -121.9449566199613) bank175618 +175619 POINT(37.092052122882635 -122.50863183553368) bank175619 +175620 POINT(36.94282293493929 -122.81913776422125) bank175620 +175621 POINT(37.956747433639926 -122.28236264441203) bank175621 +175622 POINT(38.4462030497147 -121.75139190175824) bank175622 +175623 POINT(38.15683633565134 -123.20676185184217) bank175623 +175624 POINT(36.87553143995192 -122.62237264516557) bank175624 +175625 POINT(38.04553188505936 -122.59590782624498) bank175625 +175626 POINT(37.01299277698188 -122.1796487161853) bank175626 +175627 POINT(37.89013415041969 -122.05062637276353) bank175627 +175628 POINT(36.809396845980736 -121.66489092447567) bank175628 +175629 POINT(38.37782275483093 -123.02371885718938) bank175629 +175630 POINT(37.424526861230525 -122.48458060214186) bank175630 +175631 POINT(38.69587333070436 -121.44581593529479) bank175631 +175632 POINT(37.46543722933506 -122.34077364863103) bank175632 +175633 POINT(37.768320701002956 -123.25951418059154) bank175633 +175634 POINT(37.184141055910445 -122.29157528015344) bank175634 +175635 POINT(38.63173195493205 -123.06879133061543) bank175635 +175636 POINT(37.00117827755811 -122.26603884945014) bank175636 +175637 POINT(37.69295943611928 -121.80859682460203) bank175637 +175638 POINT(37.13802414265265 -123.18019687738925) bank175638 +175639 POINT(37.02818354490737 -122.00897218126089) bank175639 +175640 POINT(36.92225906976692 -123.1125105465086) bank175640 +175641 POINT(37.26888869468489 -121.71703453611335) bank175641 +175642 POINT(37.53832203074308 -123.26603806850436) bank175642 +175643 POINT(36.77609227165001 -123.20724346421454) bank175643 +175644 POINT(37.08681404443135 -123.21675327801074) bank175644 +175645 POINT(38.16674012637089 -122.21928387925733) bank175645 +175646 POINT(38.41828715405491 -123.04247178317641) bank175646 +175647 POINT(37.60438965018208 -123.38354900666897) bank175647 +175648 POINT(38.01716053663945 -122.42908643915162) bank175648 +175649 POINT(38.048256965612886 -122.67893916907623) bank175649 +175650 POINT(38.044051400646495 -122.81517478107307) bank175650 +175651 POINT(37.18347583915938 -123.41126954877274) bank175651 +175652 POINT(38.5661112319333 -122.7705054380729) bank175652 +175653 POINT(37.05313015439193 -122.37374110803005) bank175653 +175654 POINT(38.057387371067 -121.42466067076059) bank175654 +175655 POINT(38.377584473835455 -122.81574441096744) bank175655 +175656 POINT(37.85741102904434 -123.35368623782578) bank175656 +175657 POINT(37.51966145261595 -121.71548756894647) bank175657 +175658 POINT(37.265734935578344 -121.47299128231845) bank175658 +175659 POINT(37.7719979390273 -121.43558621945601) bank175659 +175660 POINT(37.84525353094067 -122.05679701304186) bank175660 +175661 POINT(37.80377268502461 -121.43697698658912) bank175661 +175662 POINT(37.44886005046289 -121.98713969298144) bank175662 +175663 POINT(38.45115507692996 -122.33852710979968) bank175663 +175664 POINT(38.04383386822777 -122.82697069003831) bank175664 +175665 POINT(38.19166173645951 -121.97633027356143) bank175665 +175666 POINT(37.21608512257412 -123.38498594044204) bank175666 +175667 POINT(37.10021185573198 -121.77559316962572) bank175667 +175668 POINT(37.850111841506525 -123.16083175167532) bank175668 +175669 POINT(38.738722509022544 -121.67955143852296) bank175669 +175670 POINT(38.60975664076092 -121.58253971583741) bank175670 +175671 POINT(36.9674757601276 -121.81209910169669) bank175671 +175672 POINT(37.491549202993234 -122.50072402876665) bank175672 +175673 POINT(36.99467368937109 -122.63347546756339) bank175673 +175674 POINT(36.896875067830294 -122.37819057900629) bank175674 +175675 POINT(38.758855231206574 -122.76767713828438) bank175675 +175676 POINT(37.663186013885 -122.32319482972144) bank175676 +175677 POINT(38.04191851978712 -122.45275217817577) bank175677 +175678 POINT(37.850798833343276 -123.30877216713002) bank175678 +175679 POINT(37.214285878242045 -122.39640425396495) bank175679 +175680 POINT(37.475671249597916 -121.79402500979779) bank175680 +175681 POINT(38.10849315350456 -123.37622797537688) bank175681 +175682 POINT(38.14886073611719 -122.46300715853506) bank175682 +175683 POINT(38.43325358758475 -123.35996713865407) bank175683 +175684 POINT(37.147790084731064 -122.77152656621124) bank175684 +175685 POINT(37.12940406338696 -121.43449485760928) bank175685 +175686 POINT(38.01305089148018 -121.6705849564958) bank175686 +175687 POINT(37.05605375322388 -122.18555277166985) bank175687 +175688 POINT(38.214585064851384 -122.36007307045865) bank175688 +175689 POINT(38.76846552770722 -121.44182968212321) bank175689 +175690 POINT(38.313143716766355 -123.11089333303413) bank175690 +175691 POINT(38.75788525108232 -122.35860742604605) bank175691 +175692 POINT(37.841927191025945 -121.46229106818198) bank175692 +175693 POINT(36.94027872813627 -122.03438846461235) bank175693 +175694 POINT(38.158269354573584 -122.33487183671907) bank175694 +175695 POINT(38.471796639403905 -123.2331885561684) bank175695 +175696 POINT(37.78903910845215 -122.54512123759156) bank175696 +175697 POINT(37.55627259772189 -121.5900634086063) bank175697 +175698 POINT(37.865062770681384 -121.76856259768435) bank175698 +175699 POINT(37.78921795666241 -121.74723634496512) bank175699 +175700 POINT(38.08896929401438 -121.54477228351209) bank175700 +175701 POINT(38.54957768239032 -123.24643697204003) bank175701 +175702 POINT(38.13535423437785 -122.25586193505606) bank175702 +175703 POINT(38.228795802312156 -122.98326991296855) bank175703 +175704 POINT(37.457188612538076 -122.78724245351148) bank175704 +175705 POINT(37.45047242719123 -121.54288336527665) bank175705 +175706 POINT(38.59103507126144 -122.04842470780262) bank175706 +175707 POINT(37.28592951253158 -121.70946888820629) bank175707 +175708 POINT(37.67609510061116 -121.4819909591105) bank175708 +175709 POINT(36.85603514521381 -122.63463658173143) bank175709 +175710 POINT(37.57140336418223 -122.85320411768993) bank175710 +175711 POINT(37.49049942887935 -122.19220550911906) bank175711 +175712 POINT(37.414555374802674 -122.61073827352281) bank175712 +175713 POINT(38.50521572185218 -122.59912605154271) bank175713 +175714 POINT(37.71176496288799 -121.91460654047258) bank175714 +175715 POINT(36.831445208757 -123.2798889001892) bank175715 +175716 POINT(37.66037959499822 -122.94172485158266) bank175716 +175717 POINT(37.19973719624711 -123.0411331161702) bank175717 +175718 POINT(37.82995325787612 -123.06339584379103) bank175718 +175719 POINT(37.93895782126944 -121.52669812578024) bank175719 +175720 POINT(38.761382365396756 -121.59823703173272) bank175720 +175721 POINT(38.1607648375459 -123.00368695553786) bank175721 +175722 POINT(37.047618164704794 -123.38554182313877) bank175722 +175723 POINT(38.3481484345233 -121.73926201705714) bank175723 +175724 POINT(38.07286507007379 -122.4460764664663) bank175724 +175725 POINT(37.35469245182011 -121.48585476701652) bank175725 +175726 POINT(38.1134710389169 -122.87535615416708) bank175726 +175727 POINT(38.431022132631966 -122.27432926748192) bank175727 +175728 POINT(37.34041511574572 -123.31296935169225) bank175728 +175729 POINT(38.15100254184294 -122.7491283089771) bank175729 +175730 POINT(38.27281949838219 -121.79151922902973) bank175730 +175731 POINT(37.39322683705917 -123.0902411787369) bank175731 +175732 POINT(38.26959827454566 -122.06572568085001) bank175732 +175733 POINT(37.78318483715893 -122.5251537753291) bank175733 +175734 POINT(36.922951646635674 -121.98670967502477) bank175734 +175735 POINT(38.65521288197519 -122.34352893026306) bank175735 +175736 POINT(36.95720655111716 -123.34537522153498) bank175736 +175737 POINT(37.18025898472596 -122.23487921403111) bank175737 +175738 POINT(38.459497396804444 -121.58528451206625) bank175738 +175739 POINT(37.79083930469708 -123.26334820729267) bank175739 +175740 POINT(38.616783779814 -121.57989515544325) bank175740 +175741 POINT(38.58154992847692 -122.79589036137286) bank175741 +175742 POINT(38.68915726251721 -123.03942451971561) bank175742 +175743 POINT(37.38597793618193 -122.99692585146421) bank175743 +175744 POINT(37.5631892360842 -122.23547951336356) bank175744 +175745 POINT(38.12219835294267 -122.49356614471634) bank175745 +175746 POINT(38.067125427050144 -121.67653323769945) bank175746 +175747 POINT(38.49764459112673 -121.92671230002243) bank175747 +175748 POINT(38.549812037978704 -122.24500957312542) bank175748 +175749 POINT(38.72666867752261 -121.45812600730672) bank175749 +175750 POINT(37.864044881139264 -122.26138875467798) bank175750 +175751 POINT(37.74637927042265 -122.23968801629161) bank175751 +175752 POINT(37.30371012430969 -121.98760899343493) bank175752 +175753 POINT(37.577980785557706 -123.03506074531637) bank175753 +175754 POINT(37.1913568504231 -121.96108673388748) bank175754 +175755 POINT(38.60533391279073 -122.30093511291743) bank175755 +175756 POINT(38.41913095111612 -122.22741737809402) bank175756 +175757 POINT(37.07067393384619 -122.7445460810363) bank175757 +175758 POINT(38.30330697427577 -122.2093456174983) bank175758 +175759 POINT(37.97595591821937 -122.53934338368856) bank175759 +175760 POINT(37.416747115488285 -122.29357511137991) bank175760 +175761 POINT(38.47861762165501 -123.32082557613255) bank175761 +175762 POINT(37.64526174099056 -122.02058430039925) bank175762 +175763 POINT(37.74673941801368 -123.04999282132073) bank175763 +175764 POINT(37.27106407544306 -121.86035731921613) bank175764 +175765 POINT(38.580070285965746 -122.59234845870664) bank175765 +175766 POINT(37.22785931298376 -121.58001008296445) bank175766 +175767 POINT(38.71109169870369 -122.12414264767612) bank175767 +175768 POINT(38.135517772901935 -123.30509259868691) bank175768 +175769 POINT(36.9013724109908 -121.49554519217583) bank175769 +175770 POINT(38.33699308656948 -122.70076623002272) bank175770 +175771 POINT(37.030492121226786 -123.07693442190246) bank175771 +175772 POINT(37.55088407812981 -122.6256722905029) bank175772 +175773 POINT(37.21173114363 -122.03403255408942) bank175773 +175774 POINT(37.5744233678781 -122.78225564541096) bank175774 +175775 POINT(38.599412952288986 -122.78138706631236) bank175775 +175776 POINT(37.187781286160906 -121.86640989725106) bank175776 +175777 POINT(37.97909473687801 -123.01702323255901) bank175777 +175778 POINT(37.4441858165766 -121.86494530022401) bank175778 +175779 POINT(36.92261937041209 -122.2725496328572) bank175779 +175780 POINT(38.561272400097025 -121.85721874402073) bank175780 +175781 POINT(37.071456651745386 -121.78404760934764) bank175781 +175782 POINT(37.94145199268997 -122.86808315446066) bank175782 +175783 POINT(37.96156415522879 -122.38331221375113) bank175783 +175784 POINT(38.64165586377404 -122.7394628965874) bank175784 +175785 POINT(37.5280914530267 -122.3450955866006) bank175785 +175786 POINT(38.21692464769257 -122.54479574740564) bank175786 +175787 POINT(38.63636556927054 -121.90848279545646) bank175787 +175788 POINT(38.319730704334454 -122.98851358633937) bank175788 +175789 POINT(38.75418703143765 -122.8947505321526) bank175789 +175790 POINT(37.59578913852649 -121.8339419540989) bank175790 +175791 POINT(38.492217308402395 -122.78576137622657) bank175791 +175792 POINT(38.35702437358277 -122.75640626283784) bank175792 +175793 POINT(37.12074445207604 -123.09705801530251) bank175793 +175794 POINT(38.14950458320592 -121.90083220918945) bank175794 +175795 POINT(37.93965072157535 -123.3496665353951) bank175795 +175796 POINT(37.13299249862874 -122.2019397852272) bank175796 +175797 POINT(38.10968162457515 -122.4348822572809) bank175797 +175798 POINT(38.11319118174387 -122.90731368856511) bank175798 +175799 POINT(38.322125471507945 -121.90333955874031) bank175799 +175800 POINT(38.16691593703367 -122.29785191975361) bank175800 +175801 POINT(37.13412581627926 -123.35920959224181) bank175801 +175802 POINT(36.95925053497177 -121.7846810754121) bank175802 +175803 POINT(37.370659170888416 -122.93204595105806) bank175803 +175804 POINT(38.44295226995445 -122.54488993743709) bank175804 +175805 POINT(38.62700744972001 -121.52562549689489) bank175805 +175806 POINT(37.13132953773141 -123.11299125241194) bank175806 +175807 POINT(38.44196220657122 -121.7864574281673) bank175807 +175808 POINT(38.46112687842056 -122.5069062086591) bank175808 +175809 POINT(37.32360099417765 -123.17690329913763) bank175809 +175810 POINT(37.11346259454053 -122.46492693889009) bank175810 +175811 POINT(38.14514237384024 -123.2687972224256) bank175811 +175812 POINT(37.70853148612154 -122.91291025781103) bank175812 +175813 POINT(37.35851610494136 -121.90756814039239) bank175813 +175814 POINT(37.20345161464109 -122.86840248331653) bank175814 +175815 POINT(37.22613258104543 -121.90166830427566) bank175815 +175816 POINT(38.46768210281044 -122.78504233991286) bank175816 +175817 POINT(38.65338733860435 -122.6130986050005) bank175817 +175818 POINT(36.86261527799045 -122.71025977280796) bank175818 +175819 POINT(38.532061041761345 -121.45136077601066) bank175819 +175820 POINT(37.758424250660994 -121.91359452882647) bank175820 +175821 POINT(37.12641350334699 -121.80781224653563) bank175821 +175822 POINT(37.36990387062336 -123.20975247835145) bank175822 +175823 POINT(38.692792880280386 -122.74420524646979) bank175823 +175824 POINT(36.82484768423919 -121.56147009907994) bank175824 +175825 POINT(38.70908112741962 -123.31427857459634) bank175825 +175826 POINT(37.97045799699536 -121.67763152521675) bank175826 +175827 POINT(37.21916494468548 -122.66888424516493) bank175827 +175828 POINT(38.350624752459304 -122.67260483515939) bank175828 +175829 POINT(38.6146021312842 -122.71090140973828) bank175829 +175830 POINT(37.78747379152354 -122.28350534443902) bank175830 +175831 POINT(37.611045668596944 -121.44711542103374) bank175831 +175832 POINT(38.26471330337449 -123.24515230712858) bank175832 +175833 POINT(37.34952635498798 -122.88299030410505) bank175833 +175834 POINT(37.34929620870202 -121.85061707598635) bank175834 +175835 POINT(38.5915086185416 -121.62216258444427) bank175835 +175836 POINT(38.19013934791173 -122.96827280785749) bank175836 +175837 POINT(38.26383601479676 -123.05017459182751) bank175837 +175838 POINT(38.09796032229861 -122.4072817217806) bank175838 +175839 POINT(38.19965799106379 -122.09196399581324) bank175839 +175840 POINT(37.67745908497635 -123.19965663878449) bank175840 +175841 POINT(38.14239306154905 -122.71330575782669) bank175841 +175842 POINT(37.86792749392679 -122.1397652619769) bank175842 +175843 POINT(37.89725609864458 -122.95466012039381) bank175843 +175844 POINT(37.30561674728156 -121.42874659625674) bank175844 +175845 POINT(38.05987003521699 -122.54559274301666) bank175845 +175846 POINT(36.89299714672544 -121.59134493074001) bank175846 +175847 POINT(37.333180025213984 -122.48075043278055) bank175847 +175848 POINT(38.31793219943338 -122.45256515747602) bank175848 +175849 POINT(38.55306926271911 -122.14679291022185) bank175849 +175850 POINT(36.94722109625435 -123.07623368678239) bank175850 +175851 POINT(37.745526152115715 -122.96837383248693) bank175851 +175852 POINT(38.005529593417684 -122.1489174217417) bank175852 +175853 POINT(37.85472994291343 -123.06899075503385) bank175853 +175854 POINT(37.23229576771132 -123.19490972597764) bank175854 +175855 POINT(37.566195518762875 -122.6236692262021) bank175855 +175856 POINT(37.41673840721664 -122.91962055898982) bank175856 +175857 POINT(37.58964046182829 -121.97952968753418) bank175857 +175858 POINT(37.076321082593914 -122.68438767319824) bank175858 +175859 POINT(37.312304930708976 -122.77517749170616) bank175859 +175860 POINT(37.12288964461236 -121.8687110828856) bank175860 +175861 POINT(36.99516372878673 -123.15859547128248) bank175861 +175862 POINT(38.27328424796683 -122.22073583767886) bank175862 +175863 POINT(37.00885732790513 -121.49617286530486) bank175863 +175864 POINT(37.88639802322131 -121.60403295684658) bank175864 +175865 POINT(37.366642855544356 -123.29390929601885) bank175865 +175866 POINT(37.11909408076559 -123.19785599756781) bank175866 +175867 POINT(37.49081120330634 -122.86123450094281) bank175867 +175868 POINT(38.48003060932838 -121.91754542371204) bank175868 +175869 POINT(38.495303708965956 -122.10194735230492) bank175869 +175870 POINT(38.558708933672854 -123.26302631410964) bank175870 +175871 POINT(36.81625525310154 -121.61437906641028) bank175871 +175872 POINT(38.027592310343266 -122.41959151290482) bank175872 +175873 POINT(37.17413258548925 -122.96943513596855) bank175873 +175874 POINT(37.821214551762644 -122.64465660381599) bank175874 +175875 POINT(37.47537374548832 -122.70584216714995) bank175875 +175876 POINT(37.68076747763802 -121.98804739465642) bank175876 +175877 POINT(37.799288179975896 -122.69544280183621) bank175877 +175878 POINT(38.00546230672283 -122.60941696968851) bank175878 +175879 POINT(38.45055419812589 -122.35677980825632) bank175879 +175880 POINT(37.54685211017948 -122.18540539391226) bank175880 +175881 POINT(37.827065153511285 -121.8956962679405) bank175881 +175882 POINT(37.87186974019481 -122.8878616841742) bank175882 +175883 POINT(37.84564471509368 -121.7368554811433) bank175883 +175884 POINT(36.81338774855566 -123.2551872696023) bank175884 +175885 POINT(37.89663142464487 -122.10043706045725) bank175885 +175886 POINT(37.328175827519054 -123.0285286816246) bank175886 +175887 POINT(37.6255827879026 -123.16153541676044) bank175887 +175888 POINT(37.656674201498795 -123.24876955112467) bank175888 +175889 POINT(38.49992470771147 -122.91444163065093) bank175889 +175890 POINT(37.50236136538968 -121.7614937366385) bank175890 +175891 POINT(37.238591117734295 -122.07090927904235) bank175891 +175892 POINT(37.43548656473663 -122.72612828095168) bank175892 +175893 POINT(37.65647307804199 -121.86774881051065) bank175893 +175894 POINT(37.326416811895605 -121.98469833447255) bank175894 +175895 POINT(38.09249830425484 -121.71375270329435) bank175895 +175896 POINT(37.16097122159796 -123.23549243452644) bank175896 +175897 POINT(37.116092723879405 -121.82470789152056) bank175897 +175898 POINT(38.74698544003646 -123.0724317492321) bank175898 +175899 POINT(38.51931018290675 -122.31031870670944) bank175899 +175900 POINT(37.961449418960726 -122.38417999447084) bank175900 +175901 POINT(38.73692874286981 -121.5147375048554) bank175901 +175902 POINT(36.803320103271254 -121.52319936244123) bank175902 +175903 POINT(38.03656124040841 -122.3145968759766) bank175903 +175904 POINT(37.23668589214676 -123.23251888450882) bank175904 +175905 POINT(38.395476389459446 -122.18336520755842) bank175905 +175906 POINT(38.62097115397136 -121.96502825124375) bank175906 +175907 POINT(38.26616657597123 -121.46048991641403) bank175907 +175908 POINT(37.105410818144755 -122.295933666805) bank175908 +175909 POINT(38.188379177477636 -122.44912367986424) bank175909 +175910 POINT(38.73149372389375 -121.6365341384615) bank175910 +175911 POINT(37.746471600581486 -123.25433750296497) bank175911 +175912 POINT(37.433795975097034 -122.52283670578507) bank175912 +175913 POINT(37.615077090045844 -121.97577004469893) bank175913 +175914 POINT(37.27433125276953 -122.90866043967176) bank175914 +175915 POINT(36.81784335197857 -122.89567936681962) bank175915 +175916 POINT(37.80998572810547 -122.64865422478053) bank175916 +175917 POINT(36.79067674067688 -122.30419711983143) bank175917 +175918 POINT(37.98862319195615 -122.08935451609287) bank175918 +175919 POINT(37.193017112339305 -122.49733621850288) bank175919 +175920 POINT(38.249200348375005 -122.37693928503028) bank175920 +175921 POINT(37.833500255012424 -122.90777526837442) bank175921 +175922 POINT(37.754576868295146 -121.66390698679588) bank175922 +175923 POINT(37.7277949686713 -122.76386135727954) bank175923 +175924 POINT(37.07219555256794 -123.1397355253844) bank175924 +175925 POINT(37.94266341569419 -122.36152074589907) bank175925 +175926 POINT(38.520301485228885 -123.29071265559277) bank175926 +175927 POINT(38.20765120140704 -121.66055056737481) bank175927 +175928 POINT(37.52905638081082 -122.28574239095725) bank175928 +175929 POINT(37.492025017845926 -121.82770836875898) bank175929 +175930 POINT(37.163848738993266 -122.15529943311749) bank175930 +175931 POINT(38.17958646709492 -121.77059175843725) bank175931 +175932 POINT(37.39795761616858 -121.73734661928376) bank175932 +175933 POINT(37.07565993293578 -121.93201914851377) bank175933 +175934 POINT(38.6137865421519 -122.23142593747258) bank175934 +175935 POINT(37.84131231488096 -121.92198775634759) bank175935 +175936 POINT(38.38828578628745 -121.51869353348334) bank175936 +175937 POINT(38.4016083337361 -121.75975573425407) bank175937 +175938 POINT(37.32594837282742 -122.58139627023266) bank175938 +175939 POINT(37.80592193596245 -121.60646993924755) bank175939 +175940 POINT(37.08845362302828 -122.61549621745664) bank175940 +175941 POINT(38.018007610675575 -123.04844095074449) bank175941 +175942 POINT(37.80776222297699 -122.40708299858161) bank175942 +175943 POINT(37.61912590664164 -122.54288037259668) bank175943 +175944 POINT(37.2798136511454 -123.37735924144148) bank175944 +175945 POINT(38.179243773343295 -122.64603535389378) bank175945 +175946 POINT(38.21390285289661 -121.85300781623262) bank175946 +175947 POINT(36.78171702104276 -122.09196918707381) bank175947 +175948 POINT(37.47529402723537 -122.52260567412857) bank175948 +175949 POINT(37.0072571705825 -123.22499268423073) bank175949 +175950 POINT(38.52192208422086 -123.28279043378272) bank175950 +175951 POINT(38.0901237355464 -122.96250999199492) bank175951 +175952 POINT(37.91638859199157 -121.65440880348265) bank175952 +175953 POINT(37.15181783891937 -122.74660538656994) bank175953 +175954 POINT(37.433798540700344 -122.37458478705254) bank175954 +175955 POINT(38.76151167798776 -121.45886696429893) bank175955 +175956 POINT(37.801651488295306 -121.55249556860443) bank175956 +175957 POINT(37.371328321326146 -122.2374518363004) bank175957 +175958 POINT(37.7374125158243 -122.67449794791834) bank175958 +175959 POINT(38.4789629355977 -123.11045198549006) bank175959 +175960 POINT(38.22361032190291 -122.64378418986416) bank175960 +175961 POINT(38.342627821798516 -121.95019353468874) bank175961 +175962 POINT(37.55972459912811 -122.93336317856375) bank175962 +175963 POINT(38.59976205035788 -123.15576402083536) bank175963 +175964 POINT(37.551097420041515 -121.4868634745874) bank175964 +175965 POINT(37.761783328225434 -121.49234528878945) bank175965 +175966 POINT(37.27610390467392 -121.87517418877195) bank175966 +175967 POINT(37.87697180624821 -122.4379284034313) bank175967 +175968 POINT(37.741667543190786 -121.64184880229762) bank175968 +175969 POINT(37.076191577681335 -122.06679216067612) bank175969 +175970 POINT(37.35217588473994 -122.99239020165119) bank175970 +175971 POINT(38.763968039915255 -122.16681929461582) bank175971 +175972 POINT(38.38766096325825 -122.12219388581157) bank175972 +175973 POINT(37.076664976579096 -122.63354309618478) bank175973 +175974 POINT(37.21296803668828 -121.61143132179708) bank175974 +175975 POINT(37.42246897702609 -122.41939922338271) bank175975 +175976 POINT(38.49689766059329 -121.68869071337521) bank175976 +175977 POINT(37.81782190006212 -122.75210234215389) bank175977 +175978 POINT(36.98438632203732 -123.12006832373437) bank175978 +175979 POINT(37.9386774112933 -121.61487787838813) bank175979 +175980 POINT(38.766447419415556 -122.69913629473757) bank175980 +175981 POINT(36.969607757378625 -122.7406383533014) bank175981 +175982 POINT(37.97598231844533 -122.61172746314254) bank175982 +175983 POINT(36.92521111703626 -122.19337820961954) bank175983 +175984 POINT(38.00764793356598 -122.06617448177354) bank175984 +175985 POINT(37.15763104222199 -121.98520253749344) bank175985 +175986 POINT(38.73677169582594 -121.76277230490754) bank175986 +175987 POINT(37.5672680363327 -122.65772620502771) bank175987 +175988 POINT(37.61663198051595 -121.7865870092068) bank175988 +175989 POINT(38.55490400606499 -123.15586094949035) bank175989 +175990 POINT(37.409266877391744 -122.89552289947864) bank175990 +175991 POINT(38.52582054960286 -123.22572758946009) bank175991 +175992 POINT(37.94204208492246 -121.46860070044895) bank175992 +175993 POINT(37.68067527513753 -121.46580674449844) bank175993 +175994 POINT(38.10536774566973 -123.16053337472897) bank175994 +175995 POINT(37.31230293574583 -122.29201472529807) bank175995 +175996 POINT(36.90596579553614 -121.60980836650893) bank175996 +175997 POINT(36.793666669995226 -122.67463107610608) bank175997 +175998 POINT(38.463082048789666 -122.27018099663717) bank175998 +175999 POINT(37.0350877185994 -122.48637641843897) bank175999 +176000 POINT(36.96398069371858 -122.49047662336515) bank176000 +176001 POINT(38.69263225127728 -122.33313167618104) bank176001 +176002 POINT(37.033800584837664 -122.00431255029785) bank176002 +176003 POINT(37.6620688308736 -122.67042535714886) bank176003 +176004 POINT(38.56536487649477 -122.37443769251034) bank176004 +176005 POINT(38.06781630493968 -121.83800249445544) bank176005 +176006 POINT(36.81233866680824 -122.14511764747755) bank176006 +176007 POINT(38.727498531020906 -121.55106321485121) bank176007 +176008 POINT(36.94269889088434 -122.20267828044635) bank176008 +176009 POINT(37.568491619464666 -122.05644291164332) bank176009 +176010 POINT(38.15972018734445 -121.52437775063903) bank176010 +176011 POINT(38.160002443375205 -123.33090072537772) bank176011 +176012 POINT(38.41473040105041 -121.53460519057946) bank176012 +176013 POINT(38.435540715506804 -121.7857225457959) bank176013 +176014 POINT(37.394660038365686 -121.71931569790792) bank176014 +176015 POINT(36.971365463488674 -123.36854691704262) bank176015 +176016 POINT(38.62756163783062 -122.43673655589868) bank176016 +176017 POINT(37.32419008284018 -122.88544768274402) bank176017 +176018 POINT(37.913422393200825 -122.59487816328867) bank176018 +176019 POINT(37.82670171868349 -122.52836372941195) bank176019 +176020 POINT(36.94793296555816 -121.67005737672737) bank176020 +176021 POINT(37.656679647554235 -121.77537334671284) bank176021 +176022 POINT(37.79916896938765 -121.69602406284409) bank176022 +176023 POINT(38.306510840870395 -121.4876941612778) bank176023 +176024 POINT(37.44324446657204 -122.15551733674545) bank176024 +176025 POINT(37.9170283980252 -122.700782455987) bank176025 +176026 POINT(37.61540162045571 -122.35813694694154) bank176026 +176027 POINT(38.513357127488796 -122.47359799951526) bank176027 +176028 POINT(38.07305189030764 -122.54952770930515) bank176028 +176029 POINT(37.63916726065885 -122.23420888733939) bank176029 +176030 POINT(37.6865744913175 -122.58371298716666) bank176030 +176031 POINT(36.92986909314207 -122.0666308463775) bank176031 +176032 POINT(37.71866263300285 -121.89607255093235) bank176032 +176033 POINT(38.11090743843808 -121.86393910433647) bank176033 +176034 POINT(38.3723275746793 -121.4410081892772) bank176034 +176035 POINT(38.16544696930736 -122.94980672836904) bank176035 +176036 POINT(37.68612164912788 -122.77769550724327) bank176036 +176037 POINT(38.132512702247105 -121.59787499963365) bank176037 +176038 POINT(38.26868650096326 -123.35097835128323) bank176038 +176039 POINT(37.43718519147616 -121.71872689681742) bank176039 +176040 POINT(37.87863940163469 -121.77292355738857) bank176040 +176041 POINT(37.74847998904339 -123.13039008951205) bank176041 +176042 POINT(38.66720345244191 -122.41594676384271) bank176042 +176043 POINT(37.554884572981784 -121.83525879158074) bank176043 +176044 POINT(38.49891442184737 -123.34450395843697) bank176044 +176045 POINT(38.71207456352611 -121.82559767480328) bank176045 +176046 POINT(37.349367375864134 -122.57482306624837) bank176046 +176047 POINT(38.717553936468846 -123.23699951102525) bank176047 +176048 POINT(38.178853163709256 -123.23120874259996) bank176048 +176049 POINT(38.08149358851592 -121.98710719836531) bank176049 +176050 POINT(37.03141406483696 -123.2887674059191) bank176050 +176051 POINT(38.32798877904831 -122.06586139094242) bank176051 +176052 POINT(37.38316196165901 -123.32597378230322) bank176052 +176053 POINT(37.721469656832795 -122.69518550770826) bank176053 +176054 POINT(36.94267832118456 -121.49454828260336) bank176054 +176055 POINT(38.42212622575327 -122.80906791130867) bank176055 +176056 POINT(38.65276510941963 -121.66140810335334) bank176056 +176057 POINT(38.512894939976604 -121.51745770815805) bank176057 +176058 POINT(37.7078710170086 -121.42535932352601) bank176058 +176059 POINT(38.16202212434783 -122.12396056704314) bank176059 +176060 POINT(38.62598552891463 -122.58391711379701) bank176060 +176061 POINT(38.38114729090559 -122.99988981820876) bank176061 +176062 POINT(38.65038630584946 -121.55404975033719) bank176062 +176063 POINT(36.978326820950734 -122.61621399042971) bank176063 +176064 POINT(37.34570280155996 -123.09572568266108) bank176064 +176065 POINT(37.72914655359773 -121.48763599112505) bank176065 +176066 POINT(38.590665090792314 -122.8121569806869) bank176066 +176067 POINT(38.37685569812484 -122.55120225857283) bank176067 +176068 POINT(38.20351686186952 -122.04639380038559) bank176068 +176069 POINT(37.256059222919134 -121.60249371795649) bank176069 +176070 POINT(37.316269893539236 -122.32010548096119) bank176070 +176071 POINT(38.67313565000133 -122.70882007020562) bank176071 +176072 POINT(36.7857935600894 -122.09822072209423) bank176072 +176073 POINT(38.0721412052264 -123.18937582044998) bank176073 +176074 POINT(38.44075777053111 -121.9883315553088) bank176074 +176075 POINT(37.12446867473493 -123.13703040009621) bank176075 +176076 POINT(37.144530819138716 -122.67872133195253) bank176076 +176077 POINT(36.788837165110166 -122.14470449296871) bank176077 +176078 POINT(38.749403690224185 -123.33019238352634) bank176078 +176079 POINT(38.75359885934686 -121.59802217435981) bank176079 +176080 POINT(37.54784147960213 -123.41217369744525) bank176080 +176081 POINT(37.32713075600956 -123.07871825100908) bank176081 +176082 POINT(38.60582834946124 -122.39190242236849) bank176082 +176083 POINT(38.11465088796858 -121.76767316332031) bank176083 +176084 POINT(37.15160739611882 -121.63955934082912) bank176084 +176085 POINT(37.17166020647287 -122.0595204278861) bank176085 +176086 POINT(37.7009279224276 -122.87571400152207) bank176086 +176087 POINT(37.27208214767159 -122.49869866476965) bank176087 +176088 POINT(37.055355860109465 -123.12738119152722) bank176088 +176089 POINT(38.300978120619554 -122.81392346252868) bank176089 +176090 POINT(37.523038433803315 -121.73506009391953) bank176090 +176091 POINT(37.10763751009739 -121.83440408470247) bank176091 +176092 POINT(37.813604573647645 -123.4170914876204) bank176092 +176093 POINT(37.07761658057451 -123.20425797056366) bank176093 +176094 POINT(37.2735007735946 -123.09495739882) bank176094 +176095 POINT(36.81343109136851 -122.9631216541813) bank176095 +176096 POINT(36.83908049078012 -122.83474964070135) bank176096 +176097 POINT(36.8491721659347 -121.5345220305052) bank176097 +176098 POINT(38.41424799893504 -122.47031409525711) bank176098 +176099 POINT(37.738907547459746 -123.22424196812013) bank176099 +176100 POINT(37.638396837674755 -121.83122863100468) bank176100 +176101 POINT(38.315446255571096 -122.77419210762199) bank176101 +176102 POINT(38.115706153332695 -121.6227937149751) bank176102 +176103 POINT(37.36964774728655 -122.83113084035232) bank176103 +176104 POINT(38.24338831110633 -121.67531771196548) bank176104 +176105 POINT(38.55067573116439 -122.60460274457894) bank176105 +176106 POINT(38.106520067223734 -122.15423502048928) bank176106 +176107 POINT(37.625902256931795 -121.87023215546462) bank176107 +176108 POINT(37.78299061039312 -122.70930638283568) bank176108 +176109 POINT(37.62868070641747 -122.12295283087305) bank176109 +176110 POINT(38.18050545506136 -122.34245491107521) bank176110 +176111 POINT(37.11630960984574 -121.73583250400664) bank176111 +176112 POINT(37.29505904645602 -122.85485309970319) bank176112 +176113 POINT(37.517929638554136 -123.4023945375876) bank176113 +176114 POINT(37.23936622902942 -121.54891016473596) bank176114 +176115 POINT(37.98445317347225 -121.49326057402995) bank176115 +176116 POINT(37.05794379290387 -122.36023528831018) bank176116 +176117 POINT(38.48496381333347 -123.04497862729384) bank176117 +176118 POINT(37.28928800666141 -121.69076774453613) bank176118 +176119 POINT(37.412277705430824 -122.09908934425623) bank176119 +176120 POINT(37.18994339136131 -121.68477235879412) bank176120 +176121 POINT(38.34275304379253 -122.5864841039081) bank176121 +176122 POINT(37.70527267944559 -123.20761169149546) bank176122 +176123 POINT(38.08077837486246 -122.57422429664665) bank176123 +176124 POINT(38.6384759854941 -121.8870606880707) bank176124 +176125 POINT(37.09216254269619 -122.34996128059646) bank176125 +176126 POINT(36.87136064549762 -122.96794300351377) bank176126 +176127 POINT(37.97831558513381 -121.97554806592822) bank176127 +176128 POINT(37.967051815863584 -122.49633939782149) bank176128 +176129 POINT(37.15659707490853 -121.58220055983615) bank176129 +176130 POINT(38.11203871411872 -122.4395866955597) bank176130 +176131 POINT(38.251951672628195 -122.24147919212908) bank176131 +176132 POINT(38.625531996098076 -123.33696972518865) bank176132 +176133 POINT(38.08154697702103 -122.74824585828726) bank176133 +176134 POINT(36.84454016948969 -122.66406395273322) bank176134 +176135 POINT(36.822902212917484 -121.95231017942027) bank176135 +176136 POINT(38.52423316357061 -122.61420946912544) bank176136 +176137 POINT(38.730668440929406 -121.97131189232046) bank176137 +176138 POINT(37.71817859341845 -123.23355361578027) bank176138 +176139 POINT(38.18192705644918 -123.16711022041468) bank176139 +176140 POINT(37.093272234071065 -122.9004869882234) bank176140 +176141 POINT(37.74945714391376 -122.7223028930374) bank176141 +176142 POINT(37.02168254472629 -122.34245392090718) bank176142 +176143 POINT(36.96508597473859 -121.57216274396391) bank176143 +176144 POINT(38.67928116506738 -121.66671603808521) bank176144 +176145 POINT(38.400436638770785 -121.7200295801653) bank176145 +176146 POINT(36.89921972262088 -121.59152245752738) bank176146 +176147 POINT(37.69711981886906 -122.40682714724505) bank176147 +176148 POINT(37.28135890159061 -122.39555082031664) bank176148 +176149 POINT(37.29792820927893 -122.7573391338353) bank176149 +176150 POINT(37.918889020060135 -123.34130790200024) bank176150 +176151 POINT(37.81631950725946 -123.40476637393148) bank176151 +176152 POINT(37.832397466552685 -123.06273793756453) bank176152 +176153 POINT(37.340474953494706 -123.29382859368312) bank176153 +176154 POINT(38.394916014142524 -122.10847397103979) bank176154 +176155 POINT(37.1912738282928 -123.08216660368159) bank176155 +176156 POINT(37.51860386664488 -121.63903338857139) bank176156 +176157 POINT(36.90401965080067 -123.36021019770372) bank176157 +176158 POINT(38.58886212907976 -121.65977982333615) bank176158 +176159 POINT(37.91670254084144 -123.3591524860141) bank176159 +176160 POINT(38.07357489693447 -122.27581599124849) bank176160 +176161 POINT(37.0229274028526 -122.63531998184986) bank176161 +176162 POINT(37.44171954995446 -122.30464922339559) bank176162 +176163 POINT(38.371129829811274 -122.51828474846948) bank176163 +176164 POINT(38.074872200034385 -122.41128260164601) bank176164 +176165 POINT(37.11750182211059 -123.07169248589248) bank176165 +176166 POINT(37.296428190185814 -123.41735112254855) bank176166 +176167 POINT(38.0724946504077 -122.08122544088873) bank176167 +176168 POINT(36.959976918587515 -122.42471878077312) bank176168 +176169 POINT(37.99271232117692 -121.8181650199836) bank176169 +176170 POINT(37.508066856227146 -123.21924132321536) bank176170 +176171 POINT(37.66139340119044 -122.62876877221544) bank176171 +176172 POINT(36.96331296964365 -122.25808404404066) bank176172 +176173 POINT(37.94687210002281 -122.98175623834571) bank176173 +176174 POINT(37.20501462329418 -123.38199625136909) bank176174 +176175 POINT(38.585538117537325 -122.18536678086352) bank176175 +176176 POINT(38.00725185002252 -121.42321334392666) bank176176 +176177 POINT(37.31417653634159 -122.7359381989245) bank176177 +176178 POINT(38.297197599207166 -122.37769681602296) bank176178 +176179 POINT(38.244183187287646 -121.82952323986748) bank176179 +176180 POINT(37.01800436930001 -122.8451020615372) bank176180 +176181 POINT(37.18017345522158 -123.22018219011258) bank176181 +176182 POINT(38.225781376951865 -123.30541953387709) bank176182 +176183 POINT(37.37770313057874 -123.33423035600084) bank176183 +176184 POINT(36.8979520262682 -123.01581691492387) bank176184 +176185 POINT(37.65950769764078 -122.80821547564148) bank176185 +176186 POINT(37.271977430392084 -121.9753726962182) bank176186 +176187 POINT(38.40199575446747 -121.82775291500467) bank176187 +176188 POINT(37.60143865957155 -122.00328957498012) bank176188 +176189 POINT(37.342078499277925 -121.95490927406954) bank176189 +176190 POINT(38.493506084518465 -122.81035067525183) bank176190 +176191 POINT(37.907994904127335 -122.2700443440825) bank176191 +176192 POINT(36.833327471885255 -122.2073323767007) bank176192 +176193 POINT(37.16312615869206 -122.48115779764355) bank176193 +176194 POINT(37.35163953310336 -121.93716591949688) bank176194 +176195 POINT(38.501042215795714 -122.8394671991876) bank176195 +176196 POINT(37.82308539398554 -121.86746597792917) bank176196 +176197 POINT(37.5035992620593 -122.74000682439956) bank176197 +176198 POINT(38.46401257827073 -122.41851662911222) bank176198 +176199 POINT(37.26355835884761 -122.96484983049696) bank176199 +176200 POINT(38.102062815423196 -123.25766208894377) bank176200 +176201 POINT(37.92800635402405 -121.72812235853368) bank176201 +176202 POINT(36.978546344140035 -123.40921590680344) bank176202 +176203 POINT(37.98087516105677 -123.01941813444253) bank176203 +176204 POINT(37.56883081604723 -123.41483934452654) bank176204 +176205 POINT(38.50052673639339 -122.90891013222526) bank176205 +176206 POINT(37.49498241843322 -122.90847842815025) bank176206 +176207 POINT(38.67491701774644 -122.25939023980187) bank176207 +176208 POINT(38.31944510892662 -121.74282766929653) bank176208 +176209 POINT(37.83710542219929 -121.48812035949699) bank176209 +176210 POINT(36.90736157154735 -121.50951718044166) bank176210 +176211 POINT(38.08087359777639 -123.1878888016039) bank176211 +176212 POINT(37.88494163120732 -121.6409254354952) bank176212 +176213 POINT(36.88525274685686 -123.13857516772357) bank176213 +176214 POINT(38.53710254285048 -123.377919992342) bank176214 +176215 POINT(37.42582897899821 -123.07268811369642) bank176215 +176216 POINT(38.694615918367646 -121.75351247300787) bank176216 +176217 POINT(38.21552878827457 -121.48488340069866) bank176217 +176218 POINT(38.418009130381215 -121.82628352012172) bank176218 +176219 POINT(38.19510702894458 -122.110866983424) bank176219 +176220 POINT(38.50965904876692 -122.79132204706849) bank176220 +176221 POINT(37.20734489281618 -122.05181865984233) bank176221 +176222 POINT(36.96187223253495 -121.94111767609577) bank176222 +176223 POINT(37.12466685936995 -122.6059693764947) bank176223 +176224 POINT(37.145155620870796 -123.21881097138852) bank176224 +176225 POINT(36.92662510653487 -122.9381272805627) bank176225 +176226 POINT(38.560874805081234 -122.98668278359581) bank176226 +176227 POINT(37.41263098718139 -123.00120664909008) bank176227 +176228 POINT(37.23956694828864 -121.82682372451342) bank176228 +176229 POINT(37.1292304143773 -122.71303823028455) bank176229 +176230 POINT(38.3828334357456 -122.0393142468031) bank176230 +176231 POINT(37.60203622945385 -121.63409472539281) bank176231 +176232 POINT(37.28737128786545 -122.834123379908) bank176232 +176233 POINT(37.71049212364662 -122.61204416351809) bank176233 +176234 POINT(37.211705991037675 -121.59175933865656) bank176234 +176235 POINT(36.9539709235268 -122.3277713127629) bank176235 +176236 POINT(37.26422496652642 -122.42946615159521) bank176236 +176237 POINT(37.883748092455335 -123.13265375359313) bank176237 +176238 POINT(37.24558687499464 -121.80183374706856) bank176238 +176239 POINT(37.74222379749939 -122.95943133768866) bank176239 +176240 POINT(38.23879137356835 -122.92513444615587) bank176240 +176241 POINT(38.64847504372011 -122.87466822838132) bank176241 +176242 POINT(37.21385897139511 -122.69914485101242) bank176242 +176243 POINT(38.587684573551066 -122.86221061153981) bank176243 +176244 POINT(38.171241198788444 -123.26844014189989) bank176244 +176245 POINT(36.78546523988357 -121.46640585765891) bank176245 +176246 POINT(37.64236014432271 -122.72133053924179) bank176246 +176247 POINT(37.58782686595119 -122.65997101352212) bank176247 +176248 POINT(37.84177836844058 -122.27773112994527) bank176248 +176249 POINT(37.30111443358588 -122.67778623564091) bank176249 +176250 POINT(38.73970970393065 -123.25200921421839) bank176250 +176251 POINT(37.735091243036784 -122.69077773105626) bank176251 +176252 POINT(37.447417276105504 -122.73593264387156) bank176252 +176253 POINT(36.91637812395643 -122.33633924531145) bank176253 +176254 POINT(38.739527672540895 -122.23059447963647) bank176254 +176255 POINT(37.85345612738295 -122.56528553148756) bank176255 +176256 POINT(38.13788839837894 -122.18162632796803) bank176256 +176257 POINT(37.32152610739072 -123.27124510748568) bank176257 +176258 POINT(38.71112873619627 -122.70619790883205) bank176258 +176259 POINT(38.470974450069036 -122.08306051880164) bank176259 +176260 POINT(37.52843439173595 -122.09546170862242) bank176260 +176261 POINT(37.20625120099947 -123.32365169964467) bank176261 +176262 POINT(37.4604629722705 -121.89061530195345) bank176262 +176263 POINT(38.596869997159516 -122.44804812553103) bank176263 +176264 POINT(37.05299021925056 -123.18794782477218) bank176264 +176265 POINT(37.434821549916634 -122.21024127932645) bank176265 +176266 POINT(38.636320129198104 -122.75413039972422) bank176266 +176267 POINT(38.08460708569641 -122.69800534776722) bank176267 +176268 POINT(37.53230535920762 -123.15338582169868) bank176268 +176269 POINT(37.49699786952628 -121.66184871438293) bank176269 +176270 POINT(38.54069059285037 -122.88109169956103) bank176270 +176271 POINT(37.183445041262104 -122.11142432947895) bank176271 +176272 POINT(37.85689139127404 -122.0334975894602) bank176272 +176273 POINT(37.29772756070512 -122.39570575959564) bank176273 +176274 POINT(38.50381778073122 -121.59514606846913) bank176274 +176275 POINT(37.97083018961496 -121.60247746972964) bank176275 +176276 POINT(38.15468964747417 -123.29166893726916) bank176276 +176277 POINT(37.28401003655862 -123.11451050535459) bank176277 +176278 POINT(37.800632397732706 -123.32778448629665) bank176278 +176279 POINT(37.331564128887884 -121.46332190051824) bank176279 +176280 POINT(37.88560740114007 -122.62142563844941) bank176280 +176281 POINT(38.47193026363406 -123.10459552591655) bank176281 +176282 POINT(37.79516406556578 -121.68126522451679) bank176282 +176283 POINT(37.340592771091316 -122.26872985118166) bank176283 +176284 POINT(37.03147653221699 -122.55626342073064) bank176284 +176285 POINT(37.4274291091429 -122.5452920530329) bank176285 +176286 POINT(37.55960053770375 -121.43701287615136) bank176286 +176287 POINT(37.91538157120458 -122.84263430361098) bank176287 +176288 POINT(38.30741436576305 -123.25603698902303) bank176288 +176289 POINT(37.35552478699697 -121.93055534431204) bank176289 +176290 POINT(37.137092261743966 -121.77049619525039) bank176290 +176291 POINT(37.68543286742174 -123.25599016082369) bank176291 +176292 POINT(37.62550494260106 -122.09248416335534) bank176292 +176293 POINT(38.26032014520359 -122.72437435355666) bank176293 +176294 POINT(37.36835675778551 -121.76509776985262) bank176294 +176295 POINT(37.84890722126224 -122.23265417923861) bank176295 +176296 POINT(37.92487414231498 -122.81027244191426) bank176296 +176297 POINT(37.74447209658758 -121.64863634724688) bank176297 +176298 POINT(38.345956470758416 -122.14556914542061) bank176298 +176299 POINT(38.56360479608834 -122.49625050183573) bank176299 +176300 POINT(37.42967781566213 -122.37078597267329) bank176300 +176301 POINT(37.542092003692844 -122.63551120855985) bank176301 +176302 POINT(37.393794241531424 -122.0752333726241) bank176302 +176303 POINT(37.829088651211165 -123.14171692874169) bank176303 +176304 POINT(37.29727382898485 -122.90305253627454) bank176304 +176305 POINT(38.28984159347042 -121.64880522256242) bank176305 +176306 POINT(37.40872129896671 -122.50438132963106) bank176306 +176307 POINT(38.01276121898403 -121.48000427182257) bank176307 +176308 POINT(36.94633479944766 -121.59151811394862) bank176308 +176309 POINT(38.70011397414729 -123.38890338456652) bank176309 +176310 POINT(37.72963703048017 -122.45872406447559) bank176310 +176311 POINT(38.691349156964705 -122.82673167294423) bank176311 +176312 POINT(37.38215811027586 -122.47902952964958) bank176312 +176313 POINT(37.90540807083752 -121.59356340125086) bank176313 +176314 POINT(37.98274092454651 -122.52440529640344) bank176314 +176315 POINT(37.47959314158742 -123.21288302217951) bank176315 +176316 POINT(38.55267923578535 -123.06764575055216) bank176316 +176317 POINT(37.743735865746224 -121.64704573694611) bank176317 +176318 POINT(38.328869503406146 -121.48383318230314) bank176318 +176319 POINT(38.51278841679788 -121.51617116699894) bank176319 +176320 POINT(38.14793125973695 -121.89575558881833) bank176320 +176321 POINT(37.036911201952044 -123.23446362002647) bank176321 +176322 POINT(37.04089383901125 -122.34175490451447) bank176322 +176323 POINT(38.31165241525538 -122.85674759659742) bank176323 +176324 POINT(38.2907911139488 -122.43122073705855) bank176324 +176325 POINT(37.67331275707859 -121.55855003855898) bank176325 +176326 POINT(37.85469869267497 -122.67871154995204) bank176326 +176327 POINT(38.215633580439416 -122.04716896817109) bank176327 +176328 POINT(38.46272618350745 -122.39599721450178) bank176328 +176329 POINT(37.527430583905684 -122.20018137884709) bank176329 +176330 POINT(38.41068476633808 -121.54252761659914) bank176330 +176331 POINT(37.81305346420105 -122.90098578095775) bank176331 +176332 POINT(38.49274681919607 -122.07846422463865) bank176332 +176333 POINT(38.541039265319334 -122.6442256076874) bank176333 +176334 POINT(38.2415324316772 -122.55308453474075) bank176334 +176335 POINT(38.5928240428118 -122.45949327433613) bank176335 +176336 POINT(38.29811572760948 -122.73407570550883) bank176336 +176337 POINT(37.68553953465547 -123.01875146525255) bank176337 +176338 POINT(37.07722212218352 -122.59444095478796) bank176338 +176339 POINT(38.478130327818064 -122.50994434247455) bank176339 +176340 POINT(38.60242317761234 -121.74393841569537) bank176340 +176341 POINT(37.00884367426277 -122.8183562324239) bank176341 +176342 POINT(37.760557763802225 -121.52153378170952) bank176342 +176343 POINT(38.183210888119774 -123.14697905547186) bank176343 +176344 POINT(38.269634123459866 -123.18677096108989) bank176344 +176345 POINT(37.92741142127749 -123.40595511587493) bank176345 +176346 POINT(37.32577788341501 -123.33143315054667) bank176346 +176347 POINT(38.379228275261084 -121.51241441516596) bank176347 +176348 POINT(37.78088757424499 -122.89332166446738) bank176348 +176349 POINT(38.16519931994935 -123.20171233748188) bank176349 +176350 POINT(38.03296641380698 -123.32460588835173) bank176350 +176351 POINT(38.165085277168394 -122.1004349919688) bank176351 +176352 POINT(36.77739890044069 -122.08119204791753) bank176352 +176353 POINT(38.11448843265985 -122.21139551639905) bank176353 +176354 POINT(38.092516608347836 -122.47962399454796) bank176354 +176355 POINT(37.771407495251125 -123.04250234714014) bank176355 +176356 POINT(37.941349465520595 -121.5952846149582) bank176356 +176357 POINT(37.968585166505136 -121.6048155818825) bank176357 +176358 POINT(37.090275957626574 -121.92363680540613) bank176358 +176359 POINT(38.27893090849143 -121.91742469577986) bank176359 +176360 POINT(37.22965486220564 -122.53340971199822) bank176360 +176361 POINT(38.45199459206456 -122.9036513967595) bank176361 +176362 POINT(37.16791923516174 -122.05879862942251) bank176362 +176363 POINT(37.63712297958642 -122.47255570579661) bank176363 +176364 POINT(36.970741529289164 -123.39298593718178) bank176364 +176365 POINT(37.83720932942053 -121.77980939635655) bank176365 +176366 POINT(37.30356190600968 -122.16363608224664) bank176366 +176367 POINT(38.3847524441531 -122.69824016730232) bank176367 +176368 POINT(37.548416876101776 -122.62978847032815) bank176368 +176369 POINT(37.087382491894424 -122.34891162881208) bank176369 +176370 POINT(38.19920680369859 -121.62315262258142) bank176370 +176371 POINT(38.50731809245796 -123.28248652756781) bank176371 +176372 POINT(37.6308700260446 -121.78547791334294) bank176372 +176373 POINT(38.14098454938045 -122.68933832648293) bank176373 +176374 POINT(37.71839128400371 -122.78469851672745) bank176374 +176375 POINT(37.09618235819639 -123.0496765981324) bank176375 +176376 POINT(38.45744740431352 -122.75474191015813) bank176376 +176377 POINT(38.281640871475204 -121.64662217921968) bank176377 +176378 POINT(37.15275338459402 -121.98056295134566) bank176378 +176379 POINT(37.10366738476886 -121.97736365677834) bank176379 +176380 POINT(38.45810016895006 -121.71386136560106) bank176380 +176381 POINT(38.546033927242995 -122.16689075735026) bank176381 +176382 POINT(38.236413078682126 -122.16510193198657) bank176382 +176383 POINT(38.231312840698 -121.44806047647718) bank176383 +176384 POINT(38.55091297122701 -122.82138466020879) bank176384 +176385 POINT(38.20467079239869 -122.45853457983445) bank176385 +176386 POINT(37.30761951090687 -122.49009728229153) bank176386 +176387 POINT(38.679083241849845 -121.91236042380223) bank176387 +176388 POINT(38.07160015066646 -122.20680346575548) bank176388 +176389 POINT(37.556823795295315 -122.31109198979694) bank176389 +176390 POINT(37.16519542970993 -122.56350988628577) bank176390 +176391 POINT(38.5579202508376 -122.75057256102023) bank176391 +176392 POINT(38.15511439094021 -122.53749933518613) bank176392 +176393 POINT(37.418825821030886 -122.47942884512526) bank176393 +176394 POINT(38.52193133268791 -122.52925403098197) bank176394 +176395 POINT(37.31578258780953 -123.29586292086762) bank176395 +176396 POINT(38.214645135797184 -122.84352517205083) bank176396 +176397 POINT(37.02027278508764 -123.41131517790545) bank176397 +176398 POINT(38.012695555362896 -122.67198827512719) bank176398 +176399 POINT(38.00985581456944 -122.2728434140442) bank176399 +176400 POINT(37.87366890395397 -121.97460177473883) bank176400 +176401 POINT(36.8555477064537 -122.59262294563267) bank176401 +176402 POINT(37.98279781913168 -122.0258876235116) bank176402 +176403 POINT(37.34576443280601 -122.78544046327758) bank176403 +176404 POINT(38.55736839337432 -121.65981764217702) bank176404 +176405 POINT(37.02463068931102 -121.93130282926205) bank176405 +176406 POINT(37.41412937395129 -122.39242747658707) bank176406 +176407 POINT(37.21024256996481 -122.94000218045244) bank176407 +176408 POINT(36.908184595435095 -121.79677444519649) bank176408 +176409 POINT(38.12460728268984 -123.20448948706108) bank176409 +176410 POINT(38.635357087313025 -121.47188283118129) bank176410 +176411 POINT(36.886021705199724 -123.13632513601499) bank176411 +176412 POINT(37.733605212467985 -122.12467059364928) bank176412 +176413 POINT(38.29784704674714 -121.71786897856906) bank176413 +176414 POINT(37.59108589395356 -121.49213077848344) bank176414 +176415 POINT(38.325574811897894 -122.41001258280222) bank176415 +176416 POINT(37.91994033371405 -122.19062407953842) bank176416 +176417 POINT(38.25365094743998 -121.57183810873053) bank176417 +176418 POINT(37.69390756149612 -122.9374735090911) bank176418 +176419 POINT(38.01506099880536 -122.89879929773093) bank176419 +176420 POINT(38.538509660693606 -122.25251334686041) bank176420 +176421 POINT(37.278648228684915 -122.52181906495632) bank176421 +176422 POINT(38.15387017469072 -121.57138768186199) bank176422 +176423 POINT(37.83470611713929 -123.15556327403947) bank176423 +176424 POINT(37.33985288923237 -121.56138070823144) bank176424 +176425 POINT(37.05159850041943 -122.95538021271902) bank176425 +176426 POINT(37.903782447702724 -122.9219061433031) bank176426 +176427 POINT(38.23977056499096 -123.13054922931906) bank176427 +176428 POINT(37.40710991278008 -122.53243214563713) bank176428 +176429 POINT(37.94138951291627 -121.71952373314171) bank176429 +176430 POINT(37.68009396206574 -122.60078203455875) bank176430 +176431 POINT(37.15916372878517 -121.8785370698187) bank176431 +176432 POINT(38.12308798935409 -122.50704595305434) bank176432 +176433 POINT(38.63612088799711 -121.54829955306886) bank176433 +176434 POINT(37.40710465662825 -121.5213744879916) bank176434 +176435 POINT(38.64313994909591 -123.12686016163406) bank176435 +176436 POINT(36.81893288270914 -123.10222532499698) bank176436 +176437 POINT(36.954417515878475 -122.6195219739861) bank176437 +176438 POINT(38.4017172785982 -122.7868295930154) bank176438 +176439 POINT(37.75732925885568 -122.62402305643568) bank176439 +176440 POINT(37.45493302018725 -122.46147286184387) bank176440 +176441 POINT(37.515895193135435 -122.34269521954553) bank176441 +176442 POINT(36.848877671801084 -123.07370653217635) bank176442 +176443 POINT(37.74120014875524 -122.59883726201186) bank176443 +176444 POINT(38.06178124851266 -121.77725072568018) bank176444 +176445 POINT(36.884738528344876 -122.00445697011891) bank176445 +176446 POINT(37.62433465044515 -123.26636600819343) bank176446 +176447 POINT(37.790583285039716 -121.99125849985454) bank176447 +176448 POINT(37.354914606008734 -121.49106682877294) bank176448 +176449 POINT(38.232953402376005 -123.05546924029818) bank176449 +176450 POINT(37.79925953548204 -123.0473125253581) bank176450 +176451 POINT(37.37584392530685 -121.5148552325837) bank176451 +176452 POINT(38.690033439457046 -123.41900719462738) bank176452 +176453 POINT(36.990469172232736 -123.04621987143506) bank176453 +176454 POINT(38.38757465080487 -122.4955088031956) bank176454 +176455 POINT(38.00122325565569 -122.98538676540376) bank176455 +176456 POINT(38.249727613075 -122.51278999682837) bank176456 +176457 POINT(37.662184776661036 -121.98109512729579) bank176457 +176458 POINT(38.666777558478714 -122.6274915929258) bank176458 +176459 POINT(38.637403066088275 -122.49453063111326) bank176459 +176460 POINT(37.17872385057334 -121.74436577783887) bank176460 +176461 POINT(38.37303750067086 -122.53056574022308) bank176461 +176462 POINT(38.17100529530093 -121.59988823155302) bank176462 +176463 POINT(37.69804490111567 -122.04403232758493) bank176463 +176464 POINT(37.50592423605923 -122.07044074467173) bank176464 +176465 POINT(36.867269570056536 -121.43501957593587) bank176465 +176466 POINT(37.635926421171625 -123.15316586298898) bank176466 +176467 POINT(38.41041852291431 -121.54345239484748) bank176467 +176468 POINT(36.961746288009685 -121.65875375011174) bank176468 +176469 POINT(37.64035102709955 -122.08981497203227) bank176469 +176470 POINT(38.36196149008025 -122.18771019074877) bank176470 +176471 POINT(37.18359537278762 -122.89930311192691) bank176471 +176472 POINT(38.34281079777778 -123.37515868101485) bank176472 +176473 POINT(36.9438131895793 -123.06530013833199) bank176473 +176474 POINT(37.42165862812775 -123.18336509368739) bank176474 +176475 POINT(38.74296672990858 -121.42385328620342) bank176475 +176476 POINT(37.96753458888414 -122.31359562264758) bank176476 +176477 POINT(36.833214861723555 -122.7108871315731) bank176477 +176478 POINT(37.56813064530739 -122.07105734435278) bank176478 +176479 POINT(37.724069696944376 -122.88521096203819) bank176479 +176480 POINT(38.6497307227903 -122.68989217290024) bank176480 +176481 POINT(36.94944430945737 -123.01012782746493) bank176481 +176482 POINT(37.339402674156226 -122.64757107926259) bank176482 +176483 POINT(37.96177728839451 -121.78337984115034) bank176483 +176484 POINT(38.28417501356608 -123.07507694741726) bank176484 +176485 POINT(37.85049178253098 -121.6354811786454) bank176485 +176486 POINT(38.21332903022484 -122.32167736473116) bank176486 +176487 POINT(37.69690716026872 -121.92895101525622) bank176487 +176488 POINT(37.12249937963663 -123.39283831380504) bank176488 +176489 POINT(37.42750767973302 -121.64520579761412) bank176489 +176490 POINT(38.22059406253636 -122.78267897166006) bank176490 +176491 POINT(38.56433462079835 -121.43564793563851) bank176491 +176492 POINT(38.41216429830443 -122.59973464944086) bank176492 +176493 POINT(37.235537909484485 -122.65743925006578) bank176493 +176494 POINT(37.881381087273404 -122.8774125762772) bank176494 +176495 POINT(38.60335383868348 -122.5624318395568) bank176495 +176496 POINT(37.82035990283925 -123.13314691351063) bank176496 +176497 POINT(38.263778663401595 -122.36047065014954) bank176497 +176498 POINT(37.729618397507735 -122.666137607792) bank176498 +176499 POINT(38.2217714572261 -122.75207457088646) bank176499 +176500 POINT(38.20281314131921 -122.4910043120808) bank176500 +176501 POINT(38.53918777161098 -123.0135112040553) bank176501 +176502 POINT(38.47461596772856 -122.77828515315989) bank176502 +176503 POINT(38.4670930187521 -121.87203047098971) bank176503 +176504 POINT(36.97317418794833 -122.27960559184788) bank176504 +176505 POINT(37.707897253696125 -122.50625890964966) bank176505 +176506 POINT(37.26060991781279 -123.25111947076026) bank176506 +176507 POINT(38.699773933465906 -122.10621750749795) bank176507 +176508 POINT(38.54129568082429 -122.03897346746295) bank176508 +176509 POINT(37.94812368846294 -123.24695091292082) bank176509 +176510 POINT(38.27182785502319 -122.52367427441447) bank176510 +176511 POINT(38.751498811304906 -122.4203393768119) bank176511 +176512 POINT(37.884363038908006 -122.66831363093561) bank176512 +176513 POINT(37.44621379068579 -121.79818620115908) bank176513 +176514 POINT(38.23825219496219 -122.10782885741935) bank176514 +176515 POINT(36.90455030340234 -123.03373594245798) bank176515 +176516 POINT(37.82825246808558 -122.09645606283017) bank176516 +176517 POINT(38.615668173378026 -122.36916801050621) bank176517 +176518 POINT(37.73302737019727 -123.18384178284106) bank176518 +176519 POINT(36.9137731617807 -122.85827631067866) bank176519 +176520 POINT(37.167104681396744 -122.68575624430369) bank176520 +176521 POINT(38.46867145594231 -122.41737978882746) bank176521 +176522 POINT(38.44735371752829 -122.33858372518873) bank176522 +176523 POINT(36.85609810345382 -122.12160910382484) bank176523 +176524 POINT(37.26313044414286 -123.06239127567486) bank176524 +176525 POINT(38.16273949428139 -121.61605950923949) bank176525 +176526 POINT(37.01835743378726 -122.95584826398415) bank176526 +176527 POINT(37.686212467343644 -123.24674614868053) bank176527 +176528 POINT(37.637275799443096 -122.54014705382217) bank176528 +176529 POINT(38.21409676879682 -122.45353878350666) bank176529 +176530 POINT(37.0471607243833 -122.85859046398961) bank176530 +176531 POINT(38.05673460144071 -122.09351769082701) bank176531 +176532 POINT(38.426224425170375 -121.52042812295207) bank176532 +176533 POINT(38.58030048191436 -123.1199569797345) bank176533 +176534 POINT(38.76307746864798 -123.32123932767693) bank176534 +176535 POINT(37.471422976856765 -123.04810427954878) bank176535 +176536 POINT(38.226072846963646 -122.02477388915254) bank176536 +176537 POINT(37.58765562942711 -123.32722793274824) bank176537 +176538 POINT(38.49093717139221 -122.4882600071511) bank176538 +176539 POINT(37.58894269156581 -122.25991622571865) bank176539 +176540 POINT(38.134280877712165 -122.73970685682278) bank176540 +176541 POINT(37.93274169676214 -123.07273161271749) bank176541 +176542 POINT(37.55087846976059 -121.94644727489565) bank176542 +176543 POINT(37.69171163045649 -122.35569568328161) bank176543 +176544 POINT(37.44342312263332 -122.55284290850052) bank176544 +176545 POINT(38.134128981240444 -122.15772747402858) bank176545 +176546 POINT(37.4918608173898 -122.85757454042991) bank176546 +176547 POINT(37.076501242233604 -122.4756582755426) bank176547 +176548 POINT(38.12142521172724 -123.01412716076977) bank176548 +176549 POINT(38.202934299186964 -122.18105173523388) bank176549 +176550 POINT(37.677691911755815 -121.67564741156444) bank176550 +176551 POINT(36.88752919263453 -122.39491314758827) bank176551 +176552 POINT(37.3770306226462 -121.99731520640553) bank176552 +176553 POINT(37.4306019050264 -122.12942111712105) bank176553 +176554 POINT(37.509946403747314 -122.04268825804824) bank176554 +176555 POINT(37.39571139556918 -122.76938814378195) bank176555 +176556 POINT(38.36428277317721 -123.27595297319772) bank176556 +176557 POINT(38.544557676611674 -123.40040078829536) bank176557 +176558 POINT(38.5715958957291 -123.34828405625426) bank176558 +176559 POINT(38.06105289985651 -122.13130754034239) bank176559 +176560 POINT(36.92984098116254 -121.56719523633046) bank176560 +176561 POINT(38.436204033851816 -123.2260095039297) bank176561 +176562 POINT(38.42131077631553 -123.14750645865647) bank176562 +176563 POINT(37.96089416519399 -122.08013079279611) bank176563 +176564 POINT(38.23782293841799 -121.87282494659716) bank176564 +176565 POINT(38.49462479789728 -123.1026323393136) bank176565 +176566 POINT(37.96823889660898 -122.72828951631222) bank176566 +176567 POINT(38.11369936722343 -122.12247008911676) bank176567 +176568 POINT(38.32117924143577 -122.99644410369918) bank176568 +176569 POINT(38.06511237202742 -121.41973784910677) bank176569 +176570 POINT(38.68821278364627 -121.5657695066262) bank176570 +176571 POINT(37.81875742465225 -123.2003445017845) bank176571 +176572 POINT(38.0099016945945 -122.10301172232272) bank176572 +176573 POINT(38.73083026343158 -122.97790067325518) bank176573 +176574 POINT(38.753026642327974 -122.30758046039448) bank176574 +176575 POINT(37.88956875391491 -122.8643165354912) bank176575 +176576 POINT(37.95622906648471 -122.73663521104606) bank176576 +176577 POINT(38.762104908348974 -121.55292011211046) bank176577 +176578 POINT(37.78126641607077 -121.7089996732797) bank176578 +176579 POINT(38.65235074894506 -123.25858970571342) bank176579 +176580 POINT(37.51009443661623 -122.60060193261582) bank176580 +176581 POINT(37.81401830095871 -121.56384886698922) bank176581 +176582 POINT(37.8708728359407 -122.16738729771257) bank176582 +176583 POINT(36.918539432127865 -122.19529677268036) bank176583 +176584 POINT(37.326061943001264 -121.88235924506705) bank176584 +176585 POINT(37.19957861369825 -123.12892649384396) bank176585 +176586 POINT(38.229100895214835 -121.77823200088392) bank176586 +176587 POINT(38.30470515391236 -123.14312128914024) bank176587 +176588 POINT(38.13086611828673 -123.27517389922744) bank176588 +176589 POINT(37.76007128813201 -122.07639685349048) bank176589 +176590 POINT(37.04106957707398 -121.85823486154915) bank176590 +176591 POINT(37.43375642883181 -121.93900567808677) bank176591 +176592 POINT(37.88162609075986 -123.29663799877842) bank176592 +176593 POINT(38.75377587349696 -122.42253333790778) bank176593 +176594 POINT(36.953497061438924 -123.00155238417463) bank176594 +176595 POINT(36.79973458397501 -121.96763485540525) bank176595 +176596 POINT(37.8904819971507 -122.77656501210438) bank176596 +176597 POINT(38.62464467106181 -123.06292085843636) bank176597 +176598 POINT(37.5141785543352 -123.13764620023613) bank176598 +176599 POINT(37.968402145765474 -122.8722942775687) bank176599 +176600 POINT(36.9870458268554 -122.05831545510111) bank176600 +176601 POINT(38.011191624132394 -122.3912537800421) bank176601 +176602 POINT(37.505966872055815 -122.16759599441716) bank176602 +176603 POINT(38.68290430268982 -122.78436491487031) bank176603 +176604 POINT(38.427082147807575 -123.22869489802164) bank176604 +176605 POINT(36.97531807287703 -123.00459396894253) bank176605 +176606 POINT(36.99011077376257 -121.79312423991533) bank176606 +176607 POINT(37.3555021349492 -123.14441086258765) bank176607 +176608 POINT(37.67524463945254 -121.74056107085227) bank176608 +176609 POINT(38.1259701378641 -122.20062130123956) bank176609 +176610 POINT(36.95434359939911 -122.47084547312312) bank176610 +176611 POINT(36.995653178223876 -121.83560826810155) bank176611 +176612 POINT(37.94498122198641 -122.61015932027212) bank176612 +176613 POINT(38.40641134427635 -121.88774671282118) bank176613 +176614 POINT(38.16006151157506 -121.70633869298038) bank176614 +176615 POINT(38.334754935415596 -121.54234739527551) bank176615 +176616 POINT(38.08631101621685 -123.33479523995469) bank176616 +176617 POINT(37.884866183001385 -122.52789687363057) bank176617 +176618 POINT(38.173095606096624 -123.0523577760746) bank176618 +176619 POINT(36.78920514082509 -123.00838880173063) bank176619 +176620 POINT(38.75079917009598 -122.79186905423643) bank176620 +176621 POINT(38.72606504624894 -123.29963237896186) bank176621 +176622 POINT(37.78020542895314 -121.71582834420875) bank176622 +176623 POINT(37.21737331750998 -122.91411152904291) bank176623 +176624 POINT(38.45123993496196 -122.88444889052082) bank176624 +176625 POINT(37.97375405309444 -121.93761665705577) bank176625 +176626 POINT(38.75855514405591 -122.45294570307227) bank176626 +176627 POINT(36.85443180368038 -123.08100135927111) bank176627 +176628 POINT(36.85521450051682 -121.54556104169835) bank176628 +176629 POINT(38.02384588690637 -123.19851426352315) bank176629 +176630 POINT(37.696337463718244 -121.61065183224271) bank176630 +176631 POINT(37.629871709735134 -123.33555671887541) bank176631 +176632 POINT(37.6925764627292 -122.9862064319623) bank176632 +176633 POINT(37.92782257849353 -122.23344992936306) bank176633 +176634 POINT(37.55889249861012 -122.59899606163926) bank176634 +176635 POINT(38.54284640839492 -122.42318454501498) bank176635 +176636 POINT(37.67094589919964 -122.18423708011808) bank176636 +176637 POINT(38.14485726445952 -122.37621569370648) bank176637 +176638 POINT(38.26877227364171 -122.34686578775661) bank176638 +176639 POINT(37.52468596551934 -122.44181063297737) bank176639 +176640 POINT(38.157786482127165 -122.59779645437953) bank176640 +176641 POINT(37.40934657035406 -123.12087479061749) bank176641 +176642 POINT(38.46767137704643 -122.29844287037032) bank176642 +176643 POINT(37.68652997962987 -121.43689219074534) bank176643 +176644 POINT(37.350330937879974 -122.13612592058348) bank176644 +176645 POINT(38.15169842348777 -122.17533091560261) bank176645 +176646 POINT(37.539768964828816 -121.63016468659568) bank176646 +176647 POINT(37.071153267445794 -121.54836243358335) bank176647 +176648 POINT(36.904103727871 -121.82690370534006) bank176648 +176649 POINT(36.979766287357286 -121.71634814333746) bank176649 +176650 POINT(37.48293736978591 -121.581400308246) bank176650 +176651 POINT(37.27060474599394 -122.67358281417178) bank176651 +176652 POINT(38.012986170853935 -122.35440228935919) bank176652 +176653 POINT(37.74458440852974 -122.12821097022) bank176653 +176654 POINT(36.97155423495608 -122.07415186208353) bank176654 +176655 POINT(37.44559611332407 -122.09350549212908) bank176655 +176656 POINT(36.789095233697026 -122.42486024953668) bank176656 +176657 POINT(38.62098482498132 -121.68193291988682) bank176657 +176658 POINT(38.47550724620519 -121.92566368919599) bank176658 +176659 POINT(38.5496419666622 -122.82069643062964) bank176659 +176660 POINT(37.211623088192965 -123.31817941687905) bank176660 +176661 POINT(37.6709577088779 -121.63959348661793) bank176661 +176662 POINT(37.590653971218934 -123.21323047683642) bank176662 +176663 POINT(37.51489492105039 -122.35538434570014) bank176663 +176664 POINT(37.45732412767872 -123.32081811939733) bank176664 +176665 POINT(37.931473459260836 -121.82554857066972) bank176665 +176666 POINT(37.301201204811484 -123.36119668161022) bank176666 +176667 POINT(38.33650161048268 -121.74237558069747) bank176667 +176668 POINT(36.79837157315063 -122.5927220443788) bank176668 +176669 POINT(38.38895127935604 -123.38724049935188) bank176669 +176670 POINT(38.52052973985856 -121.88781607347909) bank176670 +176671 POINT(38.40480803398775 -122.88505870799308) bank176671 +176672 POINT(38.6209462201159 -122.0177364581156) bank176672 +176673 POINT(37.389074795444564 -123.17243564024312) bank176673 +176674 POINT(38.49773393268056 -123.04349437399043) bank176674 +176675 POINT(38.21426631362777 -123.0765733834704) bank176675 +176676 POINT(37.29650109381708 -122.52643603310266) bank176676 +176677 POINT(38.17794429306732 -122.15021183254336) bank176677 +176678 POINT(38.06727692477494 -122.26684660834258) bank176678 +176679 POINT(37.49884526493616 -122.93087356886667) bank176679 +176680 POINT(38.56581133379236 -122.5449301698017) bank176680 +176681 POINT(36.981750819076694 -121.90121730792082) bank176681 +176682 POINT(37.84264978503757 -123.20484711916299) bank176682 +176683 POINT(37.77601342465611 -121.94240643382194) bank176683 +176684 POINT(37.81680393588424 -122.1602008043376) bank176684 +176685 POINT(38.384734663607404 -122.30017332969963) bank176685 +176686 POINT(36.883822407026415 -122.13720049828176) bank176686 +176687 POINT(37.38324747068508 -122.86047563672757) bank176687 +176688 POINT(37.00514023272083 -122.87004866904083) bank176688 +176689 POINT(36.91330281507542 -122.09523929983371) bank176689 +176690 POINT(37.68391916281503 -123.34584286214) bank176690 +176691 POINT(38.53088871251333 -122.22306928383706) bank176691 +176692 POINT(37.79111127169418 -122.73014927632386) bank176692 +176693 POINT(37.83353451323565 -123.33817261120026) bank176693 +176694 POINT(38.12041805911868 -121.71394831508266) bank176694 +176695 POINT(37.54650313716474 -122.45791321598384) bank176695 +176696 POINT(38.04253502446202 -121.91494421979378) bank176696 +176697 POINT(38.10568343573411 -122.28070510087731) bank176697 +176698 POINT(37.743632010515356 -123.19523188246292) bank176698 +176699 POINT(36.99517765891741 -121.56998645918067) bank176699 +176700 POINT(37.35209383396954 -123.02460679171443) bank176700 +176701 POINT(37.52321574433295 -122.00968401377641) bank176701 +176702 POINT(38.367439589658694 -122.35095965966754) bank176702 +176703 POINT(37.08751951691924 -123.1825402695562) bank176703 +176704 POINT(37.10496984769375 -121.55295661469557) bank176704 +176705 POINT(37.06391802804836 -122.33908473610221) bank176705 +176706 POINT(38.40952168671972 -121.58728050091344) bank176706 +176707 POINT(38.490384305286426 -122.6879628303195) bank176707 +176708 POINT(38.390568761314626 -122.57773141247927) bank176708 +176709 POINT(38.67738267131685 -121.93658316401982) bank176709 +176710 POINT(37.896711824489685 -122.54608411749099) bank176710 +176711 POINT(37.06955563144236 -123.41391586911882) bank176711 +176712 POINT(37.63484096618493 -122.24787509486536) bank176712 +176713 POINT(37.263216038900694 -123.35385260321465) bank176713 +176714 POINT(37.74003509331934 -122.99432830591319) bank176714 +176715 POINT(38.722195331129114 -122.60381085874509) bank176715 +176716 POINT(37.96526756335344 -121.86102861644824) bank176716 +176717 POINT(38.5705491254038 -122.03891799553804) bank176717 +176718 POINT(38.62103927611596 -121.90435806463954) bank176718 +176719 POINT(38.15480625885949 -122.1590931927565) bank176719 +176720 POINT(37.38873017386234 -122.43863741505574) bank176720 +176721 POINT(37.22712603496899 -122.92178976650509) bank176721 +176722 POINT(37.524107202275644 -121.59101415718814) bank176722 +176723 POINT(36.951954590156234 -122.72718625253671) bank176723 +176724 POINT(38.19723850104303 -121.54186806591608) bank176724 +176725 POINT(38.053399522554486 -123.2691549154824) bank176725 +176726 POINT(37.539787618083594 -122.23386162363653) bank176726 +176727 POINT(37.473964396345735 -121.59640185268806) bank176727 +176728 POINT(37.38354264068905 -122.37678525597428) bank176728 +176729 POINT(38.02977427709141 -121.42922348398963) bank176729 +176730 POINT(38.3573767722415 -122.52778928302692) bank176730 +176731 POINT(38.66343336509461 -122.4848060333976) bank176731 +176732 POINT(37.13132508059762 -123.28804822029971) bank176732 +176733 POINT(37.486469167829185 -123.17182688900604) bank176733 +176734 POINT(37.13297331493075 -122.4192934097959) bank176734 +176735 POINT(38.204500319878605 -123.35960246842127) bank176735 +176736 POINT(38.61639509259258 -122.58191841687974) bank176736 +176737 POINT(36.88819983680408 -123.22461464307324) bank176737 +176738 POINT(37.31521181570469 -123.31640812046419) bank176738 +176739 POINT(37.581746419519064 -122.67308964494221) bank176739 +176740 POINT(37.665467498771285 -122.32524486936894) bank176740 +176741 POINT(37.715163954484694 -122.28638573750935) bank176741 +176742 POINT(38.71572494748171 -122.8796748394291) bank176742 +176743 POINT(36.98108656267063 -122.03326638782237) bank176743 +176744 POINT(37.123737651968504 -122.49141505311957) bank176744 +176745 POINT(37.00643358619361 -121.51846763250613) bank176745 +176746 POINT(37.243288154226285 -123.22803134205463) bank176746 +176747 POINT(38.11519601019742 -122.36194222334179) bank176747 +176748 POINT(37.15028555843033 -122.92752684960234) bank176748 +176749 POINT(38.29860623529625 -123.25480092515384) bank176749 +176750 POINT(38.51477725481804 -121.92321463876289) bank176750 +176751 POINT(37.36117524006321 -121.52771553766749) bank176751 +176752 POINT(37.07877520197177 -123.17250420147332) bank176752 +176753 POINT(38.31368375162516 -122.12530418931068) bank176753 +176754 POINT(37.088229560957515 -122.24551649722835) bank176754 +176755 POINT(36.87737234790103 -122.44499862940893) bank176755 +176756 POINT(36.95460361665083 -122.54362073372728) bank176756 +176757 POINT(38.667850795737984 -122.04204140741919) bank176757 +176758 POINT(38.15426367764714 -122.20833256978531) bank176758 +176759 POINT(37.80589017066501 -122.72245365448664) bank176759 +176760 POINT(36.81685956424474 -122.85273037821845) bank176760 +176761 POINT(37.40961524676211 -123.27163738590286) bank176761 +176762 POINT(38.28063085652025 -123.41207943624762) bank176762 +176763 POINT(36.98009041724342 -122.08168938634053) bank176763 +176764 POINT(37.15947582733661 -121.89286213147143) bank176764 +176765 POINT(38.675858288730836 -122.769536922249) bank176765 +176766 POINT(36.91248816666925 -122.6576046844328) bank176766 +176767 POINT(38.077583698108576 -121.74135950324592) bank176767 +176768 POINT(38.370027237622246 -123.30055912922474) bank176768 +176769 POINT(38.476145427681175 -123.12634308647085) bank176769 +176770 POINT(38.16679590292185 -122.85012395085984) bank176770 +176771 POINT(37.063409841387355 -121.82118196177397) bank176771 +176772 POINT(37.78905890812749 -121.82673247016359) bank176772 +176773 POINT(37.0967260341671 -123.0015642512508) bank176773 +176774 POINT(37.9474782991839 -123.24944167486497) bank176774 +176775 POINT(38.64779832920055 -121.62285081270196) bank176775 +176776 POINT(38.02724417825474 -122.42821576981827) bank176776 +176777 POINT(36.8405387711365 -122.96760632204861) bank176777 +176778 POINT(38.21263112292207 -123.36745583708061) bank176778 +176779 POINT(37.543910163843584 -121.74695776860472) bank176779 +176780 POINT(37.69321416228887 -122.3467215495136) bank176780 +176781 POINT(37.57527635561561 -122.3780404804527) bank176781 +176782 POINT(37.55262861872758 -122.24504723220721) bank176782 +176783 POINT(38.09352806099266 -123.30358924485613) bank176783 +176784 POINT(37.03764061841276 -121.80128344358506) bank176784 +176785 POINT(37.797223528350045 -122.04726833760321) bank176785 +176786 POINT(37.40123831164944 -122.79349489040376) bank176786 +176787 POINT(38.76366586255533 -122.03081132078597) bank176787 +176788 POINT(38.08936675230232 -122.98544916002069) bank176788 +176789 POINT(37.27439710477385 -122.64832336191003) bank176789 +176790 POINT(37.756609433459495 -122.2618170689068) bank176790 +176791 POINT(38.64796305391762 -122.37491746083231) bank176791 +176792 POINT(37.963108623618325 -122.32256461415119) bank176792 +176793 POINT(37.34894357628374 -122.24079773716056) bank176793 +176794 POINT(37.335741433994556 -122.28516564312118) bank176794 +176795 POINT(37.29140675509426 -122.02215511723072) bank176795 +176796 POINT(37.467650752570826 -121.62793838846993) bank176796 +176797 POINT(37.721091322643886 -121.61628103548168) bank176797 +176798 POINT(38.36754992224218 -122.14362432755357) bank176798 +176799 POINT(38.025267548064164 -123.02976920425651) bank176799 +176800 POINT(38.57760573968691 -122.57664206469806) bank176800 +176801 POINT(37.97057963875239 -122.78475072347672) bank176801 +176802 POINT(38.424836167585546 -123.16646629950044) bank176802 +176803 POINT(38.66153992562745 -122.36316569991013) bank176803 +176804 POINT(38.40899516769361 -123.00563494660281) bank176804 +176805 POINT(38.484425895213434 -122.16664117837418) bank176805 +176806 POINT(37.506227239647885 -122.40611107514685) bank176806 +176807 POINT(37.181337641167076 -122.71638472316863) bank176807 +176808 POINT(37.411753889210125 -122.8933757246269) bank176808 +176809 POINT(37.59127877382652 -123.24598736084172) bank176809 +176810 POINT(38.33999369553154 -121.81653237002295) bank176810 +176811 POINT(38.46908871341165 -122.42718789951354) bank176811 +176812 POINT(37.24801798580431 -123.06842353498132) bank176812 +176813 POINT(38.12679375093718 -122.41303473139307) bank176813 +176814 POINT(38.46694536805423 -122.85259780682404) bank176814 +176815 POINT(37.3079468454885 -122.91226793408107) bank176815 +176816 POINT(38.05966977733089 -121.67354748803584) bank176816 +176817 POINT(38.34137166142022 -121.99093339624916) bank176817 +176818 POINT(37.569615667336365 -122.87538361268543) bank176818 +176819 POINT(38.09196580174983 -121.80821543801474) bank176819 +176820 POINT(38.412040050808805 -121.43314328529083) bank176820 +176821 POINT(36.914787396702835 -121.46638210198323) bank176821 +176822 POINT(38.60040527395741 -122.29169641256112) bank176822 +176823 POINT(37.788591365167825 -122.23439783315541) bank176823 +176824 POINT(37.171838982335714 -121.59385230202709) bank176824 +176825 POINT(38.4790158314748 -123.11667033830437) bank176825 +176826 POINT(37.60291206658689 -123.05466307509695) bank176826 +176827 POINT(37.72688240882571 -121.55492161144073) bank176827 +176828 POINT(37.674860601305255 -122.75815890892058) bank176828 +176829 POINT(38.33312261208117 -122.22957701561042) bank176829 +176830 POINT(37.10335900980455 -122.6789268717144) bank176830 +176831 POINT(38.73905869899689 -121.55417703969086) bank176831 +176832 POINT(37.74851284449083 -121.43821530670547) bank176832 +176833 POINT(38.53815124957398 -121.47736108789893) bank176833 +176834 POINT(37.46457235248325 -121.42971332295551) bank176834 +176835 POINT(36.79627393839972 -122.91730651249287) bank176835 +176836 POINT(36.82580848119623 -122.05857685742434) bank176836 +176837 POINT(38.16614081091588 -122.79331024598726) bank176837 +176838 POINT(36.85967645507433 -121.42208675079371) bank176838 +176839 POINT(37.58759131445342 -122.94451505508813) bank176839 +176840 POINT(38.53184695059831 -122.72983154733357) bank176840 +176841 POINT(37.277347891795074 -123.3150371374274) bank176841 +176842 POINT(37.607050860896756 -122.22652466408765) bank176842 +176843 POINT(37.28950374810561 -122.71787573759313) bank176843 +176844 POINT(38.714482930921584 -123.38796311314736) bank176844 +176845 POINT(38.45434066473301 -122.95533566516339) bank176845 +176846 POINT(37.68490672686758 -123.35839461621468) bank176846 +176847 POINT(38.0994444092462 -122.66495803851943) bank176847 +176848 POINT(37.92119781637105 -123.0913856484243) bank176848 +176849 POINT(37.98647966821627 -121.83384184421915) bank176849 +176850 POINT(37.02127592641858 -122.76993511208558) bank176850 +176851 POINT(37.00450074052917 -123.1482914491674) bank176851 +176852 POINT(37.10729320143038 -121.5759159251289) bank176852 +176853 POINT(37.39110879495067 -122.4314379246187) bank176853 +176854 POINT(38.08162962625594 -121.75469130005773) bank176854 +176855 POINT(37.521631156753784 -122.05985118725143) bank176855 +176856 POINT(37.22098849419555 -122.0325301729473) bank176856 +176857 POINT(37.87859160265764 -122.21411472059958) bank176857 +176858 POINT(37.603644640191874 -123.37238593031977) bank176858 +176859 POINT(36.910088892946746 -123.20158822316496) bank176859 +176860 POINT(38.77105040280067 -123.09030437308066) bank176860 +176861 POINT(37.852379831442846 -123.26253123596784) bank176861 +176862 POINT(38.5661167995583 -121.90916940418222) bank176862 +176863 POINT(38.38441663179063 -122.27552489598087) bank176863 +176864 POINT(38.5747870925871 -123.08894562553913) bank176864 +176865 POINT(38.252513343758636 -121.55333150035294) bank176865 +176866 POINT(37.7816056300567 -121.6709872760311) bank176866 +176867 POINT(37.29962478940651 -121.89854245212774) bank176867 +176868 POINT(37.04939563286798 -123.15913790359089) bank176868 +176869 POINT(38.03013341849823 -123.14413592262288) bank176869 +176870 POINT(38.43066590518119 -122.94358001315847) bank176870 +176871 POINT(37.83020059819513 -121.76794507341147) bank176871 +176872 POINT(36.904850804804674 -121.97198751878564) bank176872 +176873 POINT(38.20676793401042 -122.34499985775705) bank176873 +176874 POINT(38.606919369986095 -122.8854215779928) bank176874 +176875 POINT(38.502882894261035 -121.62900632014814) bank176875 +176876 POINT(36.794489106677375 -123.12655238062766) bank176876 +176877 POINT(38.54318510255709 -121.65913959452702) bank176877 +176878 POINT(38.34304056497815 -121.700758560035) bank176878 +176879 POINT(38.23705710615143 -122.06269204455309) bank176879 +176880 POINT(37.028685568928786 -122.73865734564478) bank176880 +176881 POINT(38.72542120582939 -123.05847588141513) bank176881 +176882 POINT(38.52309756320319 -122.31068426314651) bank176882 +176883 POINT(37.48843623021528 -123.0903382081999) bank176883 +176884 POINT(37.722175891251254 -122.8156536815477) bank176884 +176885 POINT(37.32031350113849 -122.90414100334624) bank176885 +176886 POINT(37.53804401672038 -122.06399286583033) bank176886 +176887 POINT(37.63556793246351 -122.44529211449041) bank176887 +176888 POINT(38.527307742034026 -121.88447930979802) bank176888 +176889 POINT(38.14853820775078 -121.9083706493861) bank176889 +176890 POINT(38.26574261534629 -123.1724791228789) bank176890 +176891 POINT(38.131903418600906 -121.89530888918773) bank176891 +176892 POINT(38.01230947591384 -121.72512680842449) bank176892 +176893 POINT(36.93023641383671 -122.21356467457511) bank176893 +176894 POINT(38.53708926866787 -121.451300568828) bank176894 +176895 POINT(37.49718117367614 -122.75121694545669) bank176895 +176896 POINT(37.19063019039602 -122.69516319900784) bank176896 +176897 POINT(38.212071936287934 -122.99487834218804) bank176897 +176898 POINT(37.88769490693876 -122.35557990162805) bank176898 +176899 POINT(36.7907427054014 -121.95024903259886) bank176899 +176900 POINT(37.680055577300166 -121.9449359270426) bank176900 +176901 POINT(37.704763629830644 -122.88596205330443) bank176901 +176902 POINT(37.26803306259072 -123.04281013331912) bank176902 +176903 POINT(38.03383305873957 -122.05205062052569) bank176903 +176904 POINT(37.17264238390387 -123.13597357007013) bank176904 +176905 POINT(37.20085410250071 -122.10429987166351) bank176905 +176906 POINT(38.045661511262274 -122.85228696223061) bank176906 +176907 POINT(37.86169624532861 -122.88091842844504) bank176907 +176908 POINT(38.122129565964464 -122.86536118013315) bank176908 +176909 POINT(37.240547879512214 -122.07740118769509) bank176909 +176910 POINT(37.438536086303145 -122.03627367511433) bank176910 +176911 POINT(37.146889136912954 -121.55433986747548) bank176911 +176912 POINT(37.290074270334074 -122.28550734097063) bank176912 +176913 POINT(37.20551725710273 -121.74588809795503) bank176913 +176914 POINT(37.688632134742136 -123.02545185452023) bank176914 +176915 POINT(37.24495540234414 -121.79514280026564) bank176915 +176916 POINT(36.85863879411677 -122.30408201984858) bank176916 +176917 POINT(38.34687319474789 -121.72335310453795) bank176917 +176918 POINT(37.10027793314414 -123.23460541186547) bank176918 +176919 POINT(37.680379136569584 -122.01226246029096) bank176919 +176920 POINT(38.457225512746945 -123.21108572853421) bank176920 +176921 POINT(38.592104755103875 -121.48841953348047) bank176921 +176922 POINT(38.13519514888264 -122.3731726216656) bank176922 +176923 POINT(38.34554500624189 -122.99524259515002) bank176923 +176924 POINT(37.956658323700175 -122.69985331476528) bank176924 +176925 POINT(37.685918573654895 -121.63517518494925) bank176925 +176926 POINT(38.11972859301979 -122.02507361248416) bank176926 +176927 POINT(38.11759064313006 -122.93004153396883) bank176927 +176928 POINT(37.55002937372334 -123.07465066853361) bank176928 +176929 POINT(37.07642177261168 -122.493346335144) bank176929 +176930 POINT(37.300814221336886 -122.97661320388266) bank176930 +176931 POINT(37.31721265100641 -122.50897849015318) bank176931 +176932 POINT(37.73854263851778 -121.7315185642583) bank176932 +176933 POINT(36.86920214369969 -122.67235658671342) bank176933 +176934 POINT(38.25218060482499 -122.54385115292655) bank176934 +176935 POINT(38.392672352369104 -123.41594853397474) bank176935 +176936 POINT(37.94044545446638 -123.38604290261864) bank176936 +176937 POINT(37.78844766403658 -121.53864711645153) bank176937 +176938 POINT(38.54156156413178 -122.40768157001501) bank176938 +176939 POINT(37.46303231715788 -122.48634819958045) bank176939 +176940 POINT(37.04834556814467 -122.4963932796192) bank176940 +176941 POINT(37.477473925535506 -122.76793031751) bank176941 +176942 POINT(37.93877019749577 -123.1439622300308) bank176942 +176943 POINT(37.28253439951184 -121.82272605242389) bank176943 +176944 POINT(37.28190236105728 -122.35287184753038) bank176944 +176945 POINT(37.393676967491245 -123.31490394557203) bank176945 +176946 POINT(37.00454118424048 -122.06124762734981) bank176946 +176947 POINT(37.92789247480783 -122.59943558872833) bank176947 +176948 POINT(36.895462270611674 -123.078075043045) bank176948 +176949 POINT(36.99134098814371 -122.41300330090068) bank176949 +176950 POINT(36.78657204613108 -123.12450108174068) bank176950 +176951 POINT(37.62389456415866 -122.50570484870384) bank176951 +176952 POINT(37.267724110426336 -123.35462078406978) bank176952 +176953 POINT(37.04875323499171 -123.30516864899417) bank176953 +176954 POINT(37.48128193000266 -123.27184205135732) bank176954 +176955 POINT(38.00650771185986 -122.04095347246916) bank176955 +176956 POINT(37.49961901984837 -122.91147533893185) bank176956 +176957 POINT(38.12061179472952 -122.46349341851693) bank176957 +176958 POINT(37.285495701175115 -123.30547893757455) bank176958 +176959 POINT(37.535647103047665 -122.20914909638984) bank176959 +176960 POINT(37.71318507213063 -121.60561879161763) bank176960 +176961 POINT(37.007504453186385 -122.75836803182862) bank176961 +176962 POINT(38.16140875905005 -121.78385273358889) bank176962 +176963 POINT(38.02705587635821 -122.85715880332639) bank176963 +176964 POINT(37.94228615828221 -122.91047717019957) bank176964 +176965 POINT(37.40425992008083 -122.51608350171486) bank176965 +176966 POINT(37.0725670372845 -121.43629793478287) bank176966 +176967 POINT(37.323688583712475 -121.97641469709744) bank176967 +176968 POINT(37.48545081681907 -122.33416574859397) bank176968 +176969 POINT(36.84436862144574 -122.64650679677072) bank176969 +176970 POINT(36.892504322592 -122.41332261426973) bank176970 +176971 POINT(37.14635146788159 -121.62295958880013) bank176971 +176972 POINT(38.714180192757695 -122.71008545670476) bank176972 +176973 POINT(36.90009769988288 -122.72996134596663) bank176973 +176974 POINT(38.44911974227901 -123.17990583058463) bank176974 +176975 POINT(38.77023179140123 -121.81992817306366) bank176975 +176976 POINT(36.878967878880104 -122.41500358357014) bank176976 +176977 POINT(38.576762465274705 -122.20790424784268) bank176977 +176978 POINT(36.88850242524474 -122.38758353430396) bank176978 +176979 POINT(38.465106135097834 -121.78765078390418) bank176979 +176980 POINT(37.51510060566231 -121.79334365976464) bank176980 +176981 POINT(37.13521848755958 -122.32959507327055) bank176981 +176982 POINT(38.309822146364446 -122.24453092296797) bank176982 +176983 POINT(37.70646459158027 -121.53059255808003) bank176983 +176984 POINT(37.730604704321536 -123.3891988981391) bank176984 +176985 POINT(38.55658017549148 -121.96080634875773) bank176985 +176986 POINT(37.441131331343094 -122.72799040565101) bank176986 +176987 POINT(37.3042177441725 -122.34053618513545) bank176987 +176988 POINT(38.533840953013474 -122.84273835455178) bank176988 +176989 POINT(37.24191935676734 -121.98193082898938) bank176989 +176990 POINT(37.76224677099949 -122.13078923572358) bank176990 +176991 POINT(37.407935185360756 -122.61123854606417) bank176991 +176992 POINT(36.96381709985548 -122.67772253488103) bank176992 +176993 POINT(37.809872282613895 -122.48904618437226) bank176993 +176994 POINT(36.922441003362344 -122.97099618550966) bank176994 +176995 POINT(38.3532975438363 -122.60718148108364) bank176995 +176996 POINT(36.77830392505618 -121.84329509925861) bank176996 +176997 POINT(37.14172133726797 -121.45918152123538) bank176997 +176998 POINT(38.22961649851861 -121.85886422315127) bank176998 +176999 POINT(38.50773632360237 -121.8624787539088) bank176999 +177000 POINT(36.83469525502426 -121.75276230688848) bank177000 +177001 POINT(37.590932428217045 -122.18669493901272) bank177001 +177002 POINT(38.42333159278033 -121.69157379918528) bank177002 +177003 POINT(37.66964490103713 -121.53717576780372) bank177003 +177004 POINT(37.35939635600327 -121.86175701931896) bank177004 +177005 POINT(37.74390051431749 -121.79118014633441) bank177005 +177006 POINT(36.95602976184729 -122.79828710717067) bank177006 +177007 POINT(38.714527919370575 -121.842235669541) bank177007 +177008 POINT(38.06976721028842 -123.35835808417134) bank177008 +177009 POINT(38.091194837424496 -123.27347280781953) bank177009 +177010 POINT(36.92546411368552 -122.08510983032086) bank177010 +177011 POINT(38.608708846057446 -122.01697873962695) bank177011 +177012 POINT(37.755295546245335 -123.08184373487823) bank177012 +177013 POINT(38.19776170078146 -123.09777613263192) bank177013 +177014 POINT(38.50015142676819 -122.45702313827516) bank177014 +177015 POINT(36.78108365137232 -122.18582095814482) bank177015 +177016 POINT(37.29563476015282 -123.15989743221559) bank177016 +177017 POINT(37.71003675760648 -122.98274196657475) bank177017 +177018 POINT(37.871994746101876 -122.42338031060763) bank177018 +177019 POINT(37.652033879342156 -121.73538350971808) bank177019 +177020 POINT(38.03454030955977 -123.07633688150281) bank177020 +177021 POINT(37.077230258069285 -121.5698745989575) bank177021 +177022 POINT(37.265336092326194 -122.94937462483014) bank177022 +177023 POINT(38.174180970789145 -122.39386125461282) bank177023 +177024 POINT(37.765936775646 -122.28793183778377) bank177024 +177025 POINT(38.28221558923962 -121.7967149220563) bank177025 +177026 POINT(37.53160385486664 -123.13315258818977) bank177026 +177027 POINT(38.726060825073716 -121.92120863018575) bank177027 +177028 POINT(36.95763171023339 -123.02158373700412) bank177028 +177029 POINT(38.28889434276855 -123.21414676591542) bank177029 +177030 POINT(37.76559758287735 -122.51556244221213) bank177030 +177031 POINT(37.129652173454964 -121.77822060015947) bank177031 +177032 POINT(37.68667269613637 -122.44635114259184) bank177032 +177033 POINT(37.15020047102174 -121.97458555165164) bank177033 +177034 POINT(38.71700899286673 -121.98960487268242) bank177034 +177035 POINT(37.14092525287258 -122.63348781175705) bank177035 +177036 POINT(38.667430766565786 -122.4441289899908) bank177036 +177037 POINT(37.1714321410868 -122.4397285511693) bank177037 +177038 POINT(37.38992013484544 -122.49084292221224) bank177038 +177039 POINT(37.05593860960787 -122.904792939022) bank177039 +177040 POINT(37.121843471058334 -121.45738796399702) bank177040 +177041 POINT(37.30240426821084 -121.51445615849669) bank177041 +177042 POINT(37.68126875314086 -122.73379645597083) bank177042 +177043 POINT(37.92877658241634 -123.07598874428811) bank177043 +177044 POINT(38.610222395056866 -122.16971902993437) bank177044 +177045 POINT(37.52484481358259 -122.10489228257939) bank177045 +177046 POINT(38.07274322454722 -122.28997527051125) bank177046 +177047 POINT(38.17052139766275 -121.44864288139773) bank177047 +177048 POINT(38.74814368029366 -121.59229372657782) bank177048 +177049 POINT(38.157249886784975 -123.1538407595048) bank177049 +177050 POINT(37.136059118697816 -122.19775838800601) bank177050 +177051 POINT(38.732938390514924 -122.11304355163833) bank177051 +177052 POINT(37.20457330419373 -122.65770130206499) bank177052 +177053 POINT(38.43608387097333 -122.40096461915833) bank177053 +177054 POINT(37.44787768755945 -122.21399961289109) bank177054 +177055 POINT(37.31293142445507 -122.53527696451037) bank177055 +177056 POINT(37.793086794788216 -121.77424522620944) bank177056 +177057 POINT(37.83618620795345 -122.02997259631059) bank177057 +177058 POINT(38.753692233592844 -122.55592271095023) bank177058 +177059 POINT(37.05273395862002 -121.96163856342137) bank177059 +177060 POINT(38.56570309892446 -122.92894936919201) bank177060 +177061 POINT(38.27696773115583 -122.27781502838937) bank177061 +177062 POINT(36.88079697058368 -122.76088043780324) bank177062 +177063 POINT(37.68145869060457 -121.7833806215442) bank177063 +177064 POINT(38.19763782891274 -122.45444511003086) bank177064 +177065 POINT(37.09181877802997 -122.20727275904574) bank177065 +177066 POINT(38.525851179002586 -122.94097929614556) bank177066 +177067 POINT(37.97623586224425 -122.69576547025446) bank177067 +177068 POINT(38.616868640236845 -122.46354670053792) bank177068 +177069 POINT(37.021690351065985 -121.65558139513377) bank177069 +177070 POINT(38.20695456842438 -121.67773573355109) bank177070 +177071 POINT(38.004245074810825 -122.50855602357981) bank177071 +177072 POINT(37.36961253062806 -122.37012918202716) bank177072 +177073 POINT(38.63238798966726 -122.06893631210667) bank177073 +177074 POINT(37.39233884914157 -122.01906621906471) bank177074 +177075 POINT(37.59483685542875 -123.23951833698017) bank177075 +177076 POINT(37.20338524202779 -122.0734677341854) bank177076 +177077 POINT(37.50251528501089 -122.77771386455274) bank177077 +177078 POINT(37.573392783298246 -122.96503865201123) bank177078 +177079 POINT(37.06080272854805 -122.27995688024986) bank177079 +177080 POINT(38.1477590521393 -122.1070739676696) bank177080 +177081 POINT(37.04204856671223 -123.40482231816853) bank177081 +177082 POINT(37.520193146152515 -121.70959635467001) bank177082 +177083 POINT(38.31815397796086 -122.68367320029941) bank177083 +177084 POINT(37.948072184954114 -122.74358516107081) bank177084 +177085 POINT(37.74417676713336 -121.42459201371574) bank177085 +177086 POINT(37.15343343914122 -122.8943635924216) bank177086 +177087 POINT(38.24886612163274 -122.68856901244482) bank177087 +177088 POINT(37.53990559440723 -121.6489476401876) bank177088 +177089 POINT(37.75384416151189 -121.63347276814989) bank177089 +177090 POINT(38.3276286210907 -122.73175660013547) bank177090 +177091 POINT(38.53581761896323 -121.78927531164612) bank177091 +177092 POINT(38.106006772592366 -122.48605145480309) bank177092 +177093 POINT(38.27396632105239 -123.18775298792393) bank177093 +177094 POINT(37.875267723071865 -122.57644958677619) bank177094 +177095 POINT(38.6172719803227 -122.43613154981162) bank177095 +177096 POINT(38.39866440724548 -123.33995461656419) bank177096 +177097 POINT(37.48887520888534 -123.2431771517795) bank177097 +177098 POINT(37.17584931778352 -122.27216216192159) bank177098 +177099 POINT(36.98491680753824 -122.84410398191459) bank177099 +177100 POINT(38.00430859332218 -123.31945119358423) bank177100 +177101 POINT(37.26348286625685 -122.98333782500356) bank177101 +177102 POINT(38.72569807686328 -122.85505444540482) bank177102 +177103 POINT(38.37685328591266 -122.37390412353075) bank177103 +177104 POINT(36.811005555981694 -121.96070789380877) bank177104 +177105 POINT(37.06681715077224 -121.99770236786158) bank177105 +177106 POINT(37.31185468234345 -123.20023237640501) bank177106 +177107 POINT(38.06520714178764 -122.22337123495804) bank177107 +177108 POINT(37.97014521272762 -122.94217549478641) bank177108 +177109 POINT(38.60661239706972 -122.73933188986356) bank177109 +177110 POINT(37.658325291766175 -122.24172672129271) bank177110 +177111 POINT(37.17935202652912 -122.5540720140339) bank177111 +177112 POINT(37.373465884643956 -122.69181783828675) bank177112 +177113 POINT(38.69726319050182 -122.96148126027262) bank177113 +177114 POINT(37.668564303804054 -122.98296795131188) bank177114 +177115 POINT(37.79341317828742 -123.20539809623195) bank177115 +177116 POINT(37.95984156492501 -122.5836459708651) bank177116 +177117 POINT(37.261373277524726 -121.68643225786982) bank177117 +177118 POINT(36.84301210217513 -122.45473433496146) bank177118 +177119 POINT(38.736756272037404 -122.0780700871037) bank177119 +177120 POINT(37.79313145341178 -122.78042822086437) bank177120 +177121 POINT(38.48903468546543 -121.96722576756716) bank177121 +177122 POINT(37.367856557982286 -121.81687268393665) bank177122 +177123 POINT(37.488074393678 -122.31362603055825) bank177123 +177124 POINT(37.529897921990575 -122.23427833646545) bank177124 +177125 POINT(37.31305469846192 -122.89387464948855) bank177125 +177126 POINT(37.62902520681636 -123.28633274284067) bank177126 +177127 POINT(38.11772354301387 -123.08212126987709) bank177127 +177128 POINT(38.27867698998574 -122.79331223265883) bank177128 +177129 POINT(38.500039018169716 -122.64896940331397) bank177129 +177130 POINT(37.662496003012166 -122.77378044252741) bank177130 +177131 POINT(37.574803219181526 -122.9019667517582) bank177131 +177132 POINT(37.04209298382315 -122.3403534355562) bank177132 +177133 POINT(37.14144463436804 -122.33780538286928) bank177133 +177134 POINT(37.98097567545071 -121.91582013462369) bank177134 +177135 POINT(37.697814466686026 -122.58456275009976) bank177135 +177136 POINT(38.22409778170593 -123.33426979239405) bank177136 +177137 POINT(37.14333330967666 -121.49205511665592) bank177137 +177138 POINT(37.16660643136 -123.38139092801936) bank177138 +177139 POINT(38.4978653142061 -121.78224019804087) bank177139 +177140 POINT(37.761090688291624 -123.11751482560136) bank177140 +177141 POINT(38.49472386076039 -122.38820067498412) bank177141 +177142 POINT(37.75711192980781 -122.68178992153551) bank177142 +177143 POINT(37.03092283846546 -121.61744090770631) bank177143 +177144 POINT(37.929464322358555 -123.37520001050625) bank177144 +177145 POINT(38.5224683997133 -123.15602628093627) bank177145 +177146 POINT(38.668857177645776 -122.5649763757427) bank177146 +177147 POINT(38.24173821656968 -121.67151892092421) bank177147 +177148 POINT(37.255854359883834 -121.50183192455646) bank177148 +177149 POINT(38.49479773907917 -122.56759641826383) bank177149 +177150 POINT(37.71341688076222 -121.70813704210188) bank177150 +177151 POINT(38.33475836720425 -123.29390039414078) bank177151 +177152 POINT(37.70240700846357 -122.62185301811908) bank177152 +177153 POINT(37.07352503977563 -121.590377579589) bank177153 +177154 POINT(37.92871427360786 -121.48872765576331) bank177154 +177155 POINT(37.46035355793533 -122.01029280184798) bank177155 +177156 POINT(37.01468704182261 -122.66889927532988) bank177156 +177157 POINT(38.224480813556234 -122.53910850115655) bank177157 +177158 POINT(38.76282405123208 -122.85089384591272) bank177158 +177159 POINT(37.932362111774694 -122.35812687773262) bank177159 +177160 POINT(36.82313315664289 -121.98826561545003) bank177160 +177161 POINT(38.19252511381091 -121.61940684509563) bank177161 +177162 POINT(36.80651064461756 -122.81076146936446) bank177162 +177163 POINT(37.593944650037685 -122.10101824648835) bank177163 +177164 POINT(37.56866147065859 -122.38136853619487) bank177164 +177165 POINT(38.17365918759816 -122.56146748388187) bank177165 +177166 POINT(37.18375274727819 -121.58579591924888) bank177166 +177167 POINT(38.736492955395114 -122.22894396640869) bank177167 +177168 POINT(38.249066583339385 -122.50258824909412) bank177168 +177169 POINT(38.737597989638395 -123.0474493011576) bank177169 +177170 POINT(36.88158742903668 -121.76429396828753) bank177170 +177171 POINT(37.03517602030491 -123.33587178176062) bank177171 +177172 POINT(37.265708685190674 -122.09101996889181) bank177172 +177173 POINT(37.51219829266908 -123.2334696848577) bank177173 +177174 POINT(38.2184263888018 -122.17337652464833) bank177174 +177175 POINT(38.61597952259281 -122.67830598923139) bank177175 +177176 POINT(36.98499105570191 -122.96121622396244) bank177176 +177177 POINT(37.33311949195512 -122.49865980729845) bank177177 +177178 POINT(37.36085528854617 -121.97639251921774) bank177178 +177179 POINT(37.66130154812311 -121.49846762936606) bank177179 +177180 POINT(38.348542013544446 -123.29753915542035) bank177180 +177181 POINT(37.89382196804119 -123.2191892623999) bank177181 +177182 POINT(38.04242353369018 -123.25002668590186) bank177182 +177183 POINT(37.28485872744405 -122.36164311833481) bank177183 +177184 POINT(38.66465366459295 -122.77123663796372) bank177184 +177185 POINT(37.69669629628172 -121.71580829940604) bank177185 +177186 POINT(38.13515544258384 -122.83539506547835) bank177186 +177187 POINT(37.72629729942835 -122.73715332354445) bank177187 +177188 POINT(38.29975405135966 -121.88363981671151) bank177188 +177189 POINT(36.87949137331669 -121.44369128026737) bank177189 +177190 POINT(37.995698351266455 -121.65050242222875) bank177190 +177191 POINT(38.33313537232164 -122.12037656747675) bank177191 +177192 POINT(37.33951151576089 -121.64203682547796) bank177192 +177193 POINT(37.30271079535508 -123.026699391276) bank177193 +177194 POINT(38.10907630386602 -123.0518699599152) bank177194 +177195 POINT(36.861748096131905 -121.99248253887735) bank177195 +177196 POINT(37.408186203680536 -121.67823898918203) bank177196 +177197 POINT(38.70339463151056 -122.02879243220163) bank177197 +177198 POINT(37.25260337382388 -122.35794741562536) bank177198 +177199 POINT(38.733896972535845 -122.06706374672868) bank177199 +177200 POINT(38.3962250822319 -122.53452633762022) bank177200 +177201 POINT(36.86886434764813 -122.22491494262361) bank177201 +177202 POINT(37.88856753383939 -123.27251064717119) bank177202 +177203 POINT(37.34659829842442 -122.42426012286928) bank177203 +177204 POINT(37.80292221898542 -123.33943454669563) bank177204 +177205 POINT(37.71400655227231 -122.51198989145082) bank177205 +177206 POINT(38.231532485754656 -121.96128256775317) bank177206 +177207 POINT(36.86414258449558 -121.8805531771136) bank177207 +177208 POINT(38.0773999475066 -122.91863895217288) bank177208 +177209 POINT(38.4016917596014 -122.96150156880415) bank177209 +177210 POINT(37.90374454757697 -122.98531492360748) bank177210 +177211 POINT(38.60901548758107 -121.74734387238674) bank177211 +177212 POINT(38.694230635179125 -121.92267934993413) bank177212 +177213 POINT(38.44853925232451 -121.68303440593401) bank177213 +177214 POINT(37.35853557428653 -122.42555421985169) bank177214 +177215 POINT(38.25515840588378 -122.3536857545596) bank177215 +177216 POINT(38.202447276169636 -121.9197065988055) bank177216 +177217 POINT(37.4440327788105 -121.50114468399786) bank177217 +177218 POINT(37.45775686725455 -121.53882318366799) bank177218 +177219 POINT(38.10918234327839 -121.86158325791772) bank177219 +177220 POINT(38.246860813439724 -122.06773739308646) bank177220 +177221 POINT(38.19563189003 -122.55068046833746) bank177221 +177222 POINT(37.078556802326055 -122.42100517601686) bank177222 +177223 POINT(38.495635916256475 -122.83636700309296) bank177223 +177224 POINT(37.22254068842432 -123.00931776502554) bank177224 +177225 POINT(37.27428842853582 -122.48910729983821) bank177225 +177226 POINT(37.34392470661342 -122.49772566272456) bank177226 +177227 POINT(36.94429459508267 -121.97026999870218) bank177227 +177228 POINT(37.15532904709722 -122.93166006302741) bank177228 +177229 POINT(36.79508728594927 -122.71486595337876) bank177229 +177230 POINT(38.601759424874274 -122.4616447925254) bank177230 +177231 POINT(37.671362308226364 -122.16965745881531) bank177231 +177232 POINT(36.84415867440205 -123.10482046335716) bank177232 +177233 POINT(37.353278971996076 -122.66405882191891) bank177233 +177234 POINT(37.08180685837766 -123.24273158181273) bank177234 +177235 POINT(38.609468988851205 -123.16564146966655) bank177235 +177236 POINT(37.467221077444016 -122.64568464159562) bank177236 +177237 POINT(38.26793570879349 -122.47086595679743) bank177237 +177238 POINT(38.098829760340195 -122.34314700872049) bank177238 +177239 POINT(36.819727615330386 -121.67524889580274) bank177239 +177240 POINT(37.38358694100532 -121.53646133780211) bank177240 +177241 POINT(38.24162640888223 -122.51738482400867) bank177241 +177242 POINT(37.471203886551415 -122.45576599712977) bank177242 +177243 POINT(37.740538748722265 -122.14196640150693) bank177243 +177244 POINT(37.7240018215247 -121.54856972706713) bank177244 +177245 POINT(37.032707491076096 -122.99329674046983) bank177245 +177246 POINT(37.51457490691159 -121.94576397991464) bank177246 +177247 POINT(36.95976388051417 -123.16858630299096) bank177247 +177248 POINT(37.49312820892802 -122.00103950447222) bank177248 +177249 POINT(38.07926642108617 -123.32522793962153) bank177249 +177250 POINT(38.029593981759355 -121.78468552340205) bank177250 +177251 POINT(38.585848381939854 -123.03287352467893) bank177251 +177252 POINT(37.97669063150188 -123.31248983763054) bank177252 +177253 POINT(38.46331175787062 -121.48365693411603) bank177253 +177254 POINT(38.40236389477393 -121.54388203649873) bank177254 +177255 POINT(36.79998009822809 -123.00013927006476) bank177255 +177256 POINT(37.60331377416867 -121.61443795714926) bank177256 +177257 POINT(38.33491536474526 -123.17752280688887) bank177257 +177258 POINT(38.0914240015413 -122.36050008176865) bank177258 +177259 POINT(38.19106039230504 -123.23566497944104) bank177259 +177260 POINT(37.82859718477901 -123.30097987956073) bank177260 +177261 POINT(37.31998513115029 -121.84766006289247) bank177261 +177262 POINT(37.365625667315136 -122.96567798158866) bank177262 +177263 POINT(37.933253721630614 -123.11344629987387) bank177263 +177264 POINT(38.432053305903445 -122.86589706326227) bank177264 +177265 POINT(38.32446599570037 -122.37773282420507) bank177265 +177266 POINT(38.19535472203211 -122.17086507779665) bank177266 +177267 POINT(37.53163324561499 -121.9137204757999) bank177267 +177268 POINT(38.73832197769157 -123.27190985320077) bank177268 +177269 POINT(37.440410623352285 -122.57878409974171) bank177269 +177270 POINT(37.676763682972926 -123.39594111397156) bank177270 +177271 POINT(38.588312463683216 -123.03696841053522) bank177271 +177272 POINT(36.9798316686727 -123.23278773778824) bank177272 +177273 POINT(38.76455337692033 -123.37394235389917) bank177273 +177274 POINT(37.406900024725054 -122.77680946787555) bank177274 +177275 POINT(37.410730523799295 -123.15558463178346) bank177275 +177276 POINT(37.21955229443498 -121.66909793891223) bank177276 +177277 POINT(38.02624410360575 -123.110696918574) bank177277 +177278 POINT(38.3498321246428 -122.07403578133233) bank177278 +177279 POINT(37.491888361210904 -122.00863412046931) bank177279 +177280 POINT(38.02731646793943 -122.00142383857339) bank177280 +177281 POINT(38.3898150840582 -121.70385930567755) bank177281 +177282 POINT(37.3176630652028 -121.42814898380861) bank177282 +177283 POINT(37.45415647661556 -121.95081522721236) bank177283 +177284 POINT(37.0732628140441 -122.15900802460007) bank177284 +177285 POINT(38.30328909325256 -122.66255064171287) bank177285 +177286 POINT(36.795371639880834 -122.7980516653928) bank177286 +177287 POINT(38.25433589518084 -122.27798202790986) bank177287 +177288 POINT(38.163927947602346 -122.46933421496144) bank177288 +177289 POINT(38.53597984427032 -122.9561232340107) bank177289 +177290 POINT(37.76164584092812 -121.621718968565) bank177290 +177291 POINT(38.37780398528221 -122.33894509452685) bank177291 +177292 POINT(38.61932214093987 -122.95006431271845) bank177292 +177293 POINT(37.28484281066731 -121.4444596122867) bank177293 +177294 POINT(37.287645889899004 -121.94255086411063) bank177294 +177295 POINT(37.92733496407969 -122.74839569032716) bank177295 +177296 POINT(38.2198705999703 -122.90588539929455) bank177296 +177297 POINT(37.88181858039061 -121.59616655668756) bank177297 +177298 POINT(37.15274950039958 -121.82833569344128) bank177298 +177299 POINT(37.156475595268816 -121.89211994774335) bank177299 +177300 POINT(37.574657971287905 -122.148552709125) bank177300 +177301 POINT(36.99813285621141 -123.20039779154999) bank177301 +177302 POINT(36.85756123064023 -123.39413619418332) bank177302 +177303 POINT(38.22855587656769 -121.99793393715284) bank177303 +177304 POINT(37.703903799635114 -121.65711231446043) bank177304 +177305 POINT(38.73813581394527 -122.00733061684147) bank177305 +177306 POINT(37.21849676528862 -122.73653267960161) bank177306 +177307 POINT(37.10805241945629 -122.35391053668445) bank177307 +177308 POINT(37.57131252265194 -121.94849315169614) bank177308 +177309 POINT(38.5843381839679 -123.0793334027084) bank177309 +177310 POINT(37.2965770765809 -122.05011877136185) bank177310 +177311 POINT(38.66161100415449 -122.64996996650973) bank177311 +177312 POINT(36.927802943242064 -121.63377582410315) bank177312 +177313 POINT(37.56498472995307 -122.65000429929464) bank177313 +177314 POINT(37.229637511516636 -123.06003804964308) bank177314 +177315 POINT(36.794909114447634 -122.52335391852284) bank177315 +177316 POINT(36.99931621205644 -122.35426409545087) bank177316 +177317 POINT(36.95642048075217 -122.10267459845784) bank177317 +177318 POINT(38.68062454823347 -123.17904007340078) bank177318 +177319 POINT(37.31525810091037 -121.92401136272676) bank177319 +177320 POINT(38.64253777146425 -123.41819875831649) bank177320 +177321 POINT(37.34165668370312 -122.87067826674604) bank177321 +177322 POINT(38.389153082366505 -121.81026312350177) bank177322 +177323 POINT(38.5717029271583 -122.61624014433191) bank177323 +177324 POINT(37.47294647266945 -121.55936615073904) bank177324 +177325 POINT(37.282489173910506 -123.29468907470833) bank177325 +177326 POINT(36.87478233530375 -123.40805152534413) bank177326 +177327 POINT(37.86004629642793 -122.96170297049625) bank177327 +177328 POINT(37.06265035568198 -122.10697218414822) bank177328 +177329 POINT(37.88790720815467 -121.96208964046563) bank177329 +177330 POINT(37.61011689205302 -122.5773814571859) bank177330 +177331 POINT(36.96493995972427 -121.88720973688203) bank177331 +177332 POINT(38.65416245851039 -122.64163396316604) bank177332 +177333 POINT(36.79793987971207 -122.143542357567) bank177333 +177334 POINT(37.933566198145144 -121.70851113300411) bank177334 +177335 POINT(38.36917396532465 -123.07978707395971) bank177335 +177336 POINT(36.79888980940739 -121.61605709136903) bank177336 +177337 POINT(37.79523708448285 -122.04396280063739) bank177337 +177338 POINT(38.00005318470271 -122.34828900376013) bank177338 +177339 POINT(37.42164151100414 -121.67183127263259) bank177339 +177340 POINT(38.43421472384249 -122.96729704818483) bank177340 +177341 POINT(37.00619325174336 -121.98259304233514) bank177341 +177342 POINT(38.23834167032161 -121.81910532986275) bank177342 +177343 POINT(37.59435429956947 -122.7139698358522) bank177343 +177344 POINT(38.5694842338955 -122.72428718493273) bank177344 +177345 POINT(37.55287733665097 -122.30578047816292) bank177345 +177346 POINT(36.80941148064325 -122.06832507522284) bank177346 +177347 POINT(38.230459684955214 -123.12198356451937) bank177347 +177348 POINT(38.59309434872726 -122.2868358089581) bank177348 +177349 POINT(37.43341827250244 -121.62215191522914) bank177349 +177350 POINT(37.82662207271281 -122.82549144590652) bank177350 +177351 POINT(38.60602363815389 -123.21882146694256) bank177351 +177352 POINT(38.747181513821005 -121.95810774654984) bank177352 +177353 POINT(37.03956888840517 -122.5364692752061) bank177353 +177354 POINT(37.78109669685528 -123.37430720753152) bank177354 +177355 POINT(38.233500166912705 -121.96942769378208) bank177355 +177356 POINT(37.92922081509309 -122.41523541640656) bank177356 +177357 POINT(38.089811928789715 -121.92519397154527) bank177357 +177358 POINT(37.259133619836135 -122.60410060229351) bank177358 +177359 POINT(37.67517404064678 -123.3800265268437) bank177359 +177360 POINT(37.1804747415777 -123.41273209648395) bank177360 +177361 POINT(38.05441026389171 -123.0295630244309) bank177361 +177362 POINT(37.1176591104594 -122.8503970846344) bank177362 +177363 POINT(38.6068864874509 -122.7365973899226) bank177363 +177364 POINT(38.270874867669605 -122.89003484802416) bank177364 +177365 POINT(37.5774036879113 -122.05525542294495) bank177365 +177366 POINT(38.65963172096166 -123.28634150916264) bank177366 +177367 POINT(37.476543065084776 -123.20739047048265) bank177367 +177368 POINT(38.74293077578749 -122.1469951856935) bank177368 +177369 POINT(38.41042416322945 -123.09716624013116) bank177369 +177370 POINT(37.357087400878015 -122.54918496938069) bank177370 +177371 POINT(37.66350911084534 -121.81332816427107) bank177371 +177372 POINT(38.455398800057395 -122.77457351289765) bank177372 +177373 POINT(38.145736193902934 -122.6789922821507) bank177373 +177374 POINT(37.584177540486905 -121.97696378827358) bank177374 +177375 POINT(38.739021641130925 -123.37963778803066) bank177375 +177376 POINT(37.101101488936195 -123.2330853298347) bank177376 +177377 POINT(37.21890301272429 -121.96170415376143) bank177377 +177378 POINT(36.953721071023935 -123.20819413755675) bank177378 +177379 POINT(37.71194909090326 -121.72748647311087) bank177379 +177380 POINT(38.35949276592845 -121.62457065797672) bank177380 +177381 POINT(38.721828305812906 -122.56975347239495) bank177381 +177382 POINT(36.99518540488972 -122.54735534338258) bank177382 +177383 POINT(37.89408257575188 -121.88841540155087) bank177383 +177384 POINT(36.981878530613706 -122.06056670507226) bank177384 +177385 POINT(38.73289507069611 -123.33256491190284) bank177385 +177386 POINT(38.527588061250654 -121.95003958804844) bank177386 +177387 POINT(38.15062246841726 -122.72798504148889) bank177387 +177388 POINT(37.14639911008485 -122.36859371086877) bank177388 +177389 POINT(38.71898171549826 -122.84823172094126) bank177389 +177390 POINT(38.00856731569073 -121.98494974051694) bank177390 +177391 POINT(37.27828264633021 -122.14715904422651) bank177391 +177392 POINT(38.655354672512814 -122.43151923834917) bank177392 +177393 POINT(37.0095260292118 -122.5946018780718) bank177393 +177394 POINT(38.120328684915556 -122.76792071876633) bank177394 +177395 POINT(38.45950999174246 -122.74882337654928) bank177395 +177396 POINT(38.02679435325626 -121.74712662033573) bank177396 +177397 POINT(38.556095700866784 -122.14479146491844) bank177397 +177398 POINT(36.869817177107414 -122.16487848850055) bank177398 +177399 POINT(38.158718870741446 -123.24045549593586) bank177399 +177400 POINT(37.5644204331562 -123.09400432199902) bank177400 +177401 POINT(36.83075640774919 -123.39168303049755) bank177401 +177402 POINT(38.17654971511548 -121.48073392263986) bank177402 +177403 POINT(38.02541785549726 -122.7660631528971) bank177403 +177404 POINT(37.09800400959744 -122.450999937426) bank177404 +177405 POINT(38.77323469593606 -121.63201976516119) bank177405 +177406 POINT(37.15853846054612 -121.85991041290978) bank177406 +177407 POINT(37.28961025289181 -122.02927145772843) bank177407 +177408 POINT(38.63267047888561 -122.50898515987372) bank177408 +177409 POINT(37.825936734214366 -123.18395844113302) bank177409 +177410 POINT(37.60898215649769 -122.4496318881851) bank177410 +177411 POINT(37.50798866659961 -121.67694050835875) bank177411 +177412 POINT(37.408430337278695 -121.93833846224327) bank177412 +177413 POINT(38.43313548433061 -121.43044347974714) bank177413 +177414 POINT(38.637900720753116 -123.25424498138325) bank177414 +177415 POINT(38.67818539464753 -121.71767608488322) bank177415 +177416 POINT(38.090991170959654 -122.33395756366025) bank177416 +177417 POINT(37.846861245671676 -121.79130079697245) bank177417 +177418 POINT(37.67104611463775 -122.1357336676883) bank177418 +177419 POINT(37.90379752470512 -121.59478219355432) bank177419 +177420 POINT(38.652589406370886 -123.09628374666046) bank177420 +177421 POINT(36.87577071602823 -121.9190493171093) bank177421 +177422 POINT(37.37192024830837 -122.94786282046235) bank177422 +177423 POINT(38.48570007140756 -123.00228768939024) bank177423 +177424 POINT(37.147167210090714 -122.08369651397541) bank177424 +177425 POINT(37.53017151748979 -123.1989490631852) bank177425 +177426 POINT(37.31577684543361 -122.78071970793052) bank177426 +177427 POINT(38.061876375346564 -123.23249149467875) bank177427 +177428 POINT(38.002722186975205 -122.78992004353255) bank177428 +177429 POINT(37.40840129705445 -123.41922209539692) bank177429 +177430 POINT(37.90570755000316 -122.82406294910346) bank177430 +177431 POINT(37.78580568214007 -123.14581822094833) bank177431 +177432 POINT(38.229032670980175 -123.4173992854602) bank177432 +177433 POINT(37.714259991128216 -121.56082547185352) bank177433 +177434 POINT(38.265952624022454 -121.93463479552166) bank177434 +177435 POINT(38.2854270475669 -121.5763780733768) bank177435 +177436 POINT(37.37819637441261 -121.5105660619324) bank177436 +177437 POINT(38.12820424058345 -123.08692637715191) bank177437 +177438 POINT(38.52716016850712 -122.29202097398137) bank177438 +177439 POINT(37.16721494355928 -121.6647419701793) bank177439 +177440 POINT(37.10146798926156 -123.21037179135935) bank177440 +177441 POINT(37.08426728752109 -122.34544585242723) bank177441 +177442 POINT(38.18893285485524 -123.04721838372093) bank177442 +177443 POINT(38.423100751076674 -122.94616531770467) bank177443 +177444 POINT(37.64344929751282 -121.73386261290788) bank177444 +177445 POINT(38.759616445609154 -121.85974973431694) bank177445 +177446 POINT(37.25003650646787 -123.38121740754126) bank177446 +177447 POINT(38.06750190201694 -121.58721995192458) bank177447 +177448 POINT(38.26567678776291 -122.39335822885171) bank177448 +177449 POINT(37.36284029821462 -122.6664369852733) bank177449 +177450 POINT(38.2498139361268 -122.1462021623331) bank177450 +177451 POINT(38.34381983971643 -122.32229879224421) bank177451 +177452 POINT(37.62844150123042 -122.79546813772996) bank177452 +177453 POINT(38.567884862979895 -121.62381032104531) bank177453 +177454 POINT(37.964013871449026 -123.18770856702585) bank177454 +177455 POINT(37.18087535169619 -123.40361166795732) bank177455 +177456 POINT(38.45559180652597 -122.25867577790088) bank177456 +177457 POINT(36.95777665577746 -121.73003937349216) bank177457 +177458 POINT(37.814456209674624 -123.00300166517258) bank177458 +177459 POINT(37.13362670424709 -122.89535419981563) bank177459 +177460 POINT(37.45750991177221 -121.54628573890892) bank177460 +177461 POINT(38.581705432859096 -121.49591950342712) bank177461 +177462 POINT(37.23120843713444 -121.82408896413196) bank177462 +177463 POINT(37.918553822423 -121.48564681975678) bank177463 +177464 POINT(37.83253323947448 -122.69894188618271) bank177464 +177465 POINT(37.35290565704895 -123.37999295134806) bank177465 +177466 POINT(37.19393471545774 -122.4294443823729) bank177466 +177467 POINT(37.6963445820912 -122.2299539833036) bank177467 +177468 POINT(38.009591643116 -122.45716797734575) bank177468 +177469 POINT(37.91328919730177 -121.64132251900826) bank177469 +177470 POINT(37.90788267115176 -121.51263490997462) bank177470 +177471 POINT(37.448299088738814 -121.88012817056305) bank177471 +177472 POINT(37.16715528988832 -122.6773092849883) bank177472 +177473 POINT(38.26051266689687 -122.18717463212596) bank177473 +177474 POINT(37.40372403142688 -121.52907572912395) bank177474 +177475 POINT(38.3915443989149 -122.9501928041159) bank177475 +177476 POINT(37.3703291438233 -122.52398768488088) bank177476 +177477 POINT(38.61483809965233 -123.1501775772126) bank177477 +177478 POINT(37.94449962002309 -122.24553779342773) bank177478 +177479 POINT(38.31593718976883 -122.38206333494469) bank177479 +177480 POINT(38.59856622822121 -121.69234398494545) bank177480 +177481 POINT(38.669485205013935 -121.44182003292347) bank177481 +177482 POINT(36.84275471267749 -122.43427643376035) bank177482 +177483 POINT(37.77151907071443 -122.55068244305319) bank177483 +177484 POINT(38.33069544458673 -121.45898300977589) bank177484 +177485 POINT(37.19233511399085 -123.33871240824833) bank177485 +177486 POINT(38.128315928844955 -122.49100207517313) bank177486 +177487 POINT(38.73223867561386 -122.82346208860788) bank177487 +177488 POINT(37.48347868217322 -122.49567727752502) bank177488 +177489 POINT(37.687019862211635 -121.51845351973586) bank177489 +177490 POINT(37.935180519347455 -123.30187011486933) bank177490 +177491 POINT(36.97727588080346 -122.00707314632596) bank177491 +177492 POINT(37.02684084181556 -123.2181040250053) bank177492 +177493 POINT(37.189986200045325 -122.99382157808193) bank177493 +177494 POINT(37.31585794875145 -121.71674202016709) bank177494 +177495 POINT(37.663541908627685 -122.79343598985423) bank177495 +177496 POINT(38.34687148483079 -121.70371700142387) bank177496 +177497 POINT(37.68685363134233 -122.61395733794018) bank177497 +177498 POINT(37.769083173837856 -122.28959913159976) bank177498 +177499 POINT(38.535676949670545 -123.04086956581308) bank177499 +177500 POINT(36.81305544032897 -123.25455149430559) bank177500 +177501 POINT(38.67077963538252 -122.22971288532206) bank177501 +177502 POINT(37.01833769019415 -122.46646244484673) bank177502 +177503 POINT(37.931805009193944 -123.05103907936703) bank177503 +177504 POINT(37.402211733219445 -123.27943691306614) bank177504 +177505 POINT(38.48830332645121 -122.05754704310418) bank177505 +177506 POINT(38.7469180904051 -122.01068060399874) bank177506 +177507 POINT(37.30495651292674 -121.78638519497616) bank177507 +177508 POINT(36.94899211401708 -122.26435547942442) bank177508 +177509 POINT(37.81129293233852 -122.45285508435302) bank177509 +177510 POINT(38.29956285391686 -121.74907470367631) bank177510 +177511 POINT(38.16725774615966 -121.54497338641536) bank177511 +177512 POINT(36.99029853420678 -123.11629102383577) bank177512 +177513 POINT(38.61641919619535 -122.45594014531123) bank177513 +177514 POINT(38.382328637663434 -122.14774241128883) bank177514 +177515 POINT(37.71635448543276 -122.44917130019589) bank177515 +177516 POINT(37.05178687650585 -121.88321100458039) bank177516 +177517 POINT(37.33528023295795 -121.69553277379573) bank177517 +177518 POINT(38.56073553356426 -122.73407402083832) bank177518 +177519 POINT(38.71702420483841 -122.21669617356413) bank177519 +177520 POINT(37.0645837590019 -123.1221696108444) bank177520 +177521 POINT(37.76651138173132 -123.34779148825923) bank177521 +177522 POINT(36.86601439507681 -121.7866850102067) bank177522 +177523 POINT(38.37649753260983 -122.19426503899625) bank177523 +177524 POINT(37.064206083119835 -122.4194023481492) bank177524 +177525 POINT(37.29514386702262 -121.77714777240715) bank177525 +177526 POINT(38.4046833163778 -121.47486445495842) bank177526 +177527 POINT(38.56319618557396 -122.25844472723327) bank177527 +177528 POINT(38.6671289855239 -122.83923888798323) bank177528 +177529 POINT(38.07458401957497 -123.04128026920988) bank177529 +177530 POINT(37.92442736339915 -122.62752003203524) bank177530 +177531 POINT(38.47660501933796 -122.38084632486282) bank177531 +177532 POINT(37.041388641595105 -123.0525038943126) bank177532 +177533 POINT(37.68420642142998 -121.64865818734418) bank177533 +177534 POINT(36.98231521917539 -123.24477549767181) bank177534 +177535 POINT(37.93910172742642 -121.5536304241059) bank177535 +177536 POINT(37.14354373913631 -122.69443265028245) bank177536 +177537 POINT(38.61004907383445 -122.61609863880037) bank177537 +177538 POINT(38.103448418542534 -121.69599904950593) bank177538 +177539 POINT(37.48426381861759 -122.83856448074987) bank177539 +177540 POINT(36.786391556113465 -122.53022152670172) bank177540 +177541 POINT(37.613404105202854 -122.50225303418031) bank177541 +177542 POINT(38.674650051374506 -122.66708452192401) bank177542 +177543 POINT(36.868358062827944 -122.28799700401282) bank177543 +177544 POINT(36.94062698050426 -123.20503154298655) bank177544 +177545 POINT(37.73133347132263 -122.68700799540028) bank177545 +177546 POINT(36.8547734951846 -122.2616379460697) bank177546 +177547 POINT(38.081628207872605 -122.2561247730592) bank177547 +177548 POINT(38.54258566251326 -121.55341648523529) bank177548 +177549 POINT(38.173483001963156 -123.16490200301536) bank177549 +177550 POINT(37.31971340458945 -122.78907907955794) bank177550 +177551 POINT(38.19451214137647 -122.38582293466128) bank177551 +177552 POINT(37.17213383257163 -122.70491117909532) bank177552 +177553 POINT(38.71609646229702 -121.65771572738082) bank177553 +177554 POINT(38.27998463637746 -123.30845096022541) bank177554 +177555 POINT(36.85817729858691 -121.7572655752029) bank177555 +177556 POINT(37.23964110403416 -122.60042552935619) bank177556 +177557 POINT(37.544327044392446 -122.57443967891679) bank177557 +177558 POINT(38.43523603412029 -121.7615922420192) bank177558 +177559 POINT(38.47941344023523 -123.22423560802096) bank177559 +177560 POINT(38.72150691492933 -121.63995078147894) bank177560 +177561 POINT(37.94843109675999 -122.51609457888922) bank177561 +177562 POINT(37.58040038500815 -123.34518469867788) bank177562 +177563 POINT(37.57435755553911 -121.49103263007713) bank177563 +177564 POINT(37.6508543643601 -121.59432729216604) bank177564 +177565 POINT(37.17344065084083 -121.51242388784067) bank177565 +177566 POINT(37.84155767159406 -122.74837717405578) bank177566 +177567 POINT(38.16013008446845 -121.54624929532628) bank177567 +177568 POINT(37.04876870084276 -122.03680048370903) bank177568 +177569 POINT(37.56607530600772 -122.13721036285826) bank177569 +177570 POINT(36.801180360252744 -122.24006764189993) bank177570 +177571 POINT(37.63915530566284 -122.70159754784856) bank177571 +177572 POINT(36.8723064690536 -121.67416928891973) bank177572 +177573 POINT(38.53715213785446 -123.05603551208583) bank177573 +177574 POINT(37.561228252896974 -122.45893794561911) bank177574 +177575 POINT(38.74348552971506 -121.60703729730669) bank177575 +177576 POINT(37.66200159759526 -122.81498892888962) bank177576 +177577 POINT(38.409671825798554 -121.46706701195536) bank177577 +177578 POINT(37.86637296260729 -121.76938249166906) bank177578 +177579 POINT(37.8422738138733 -123.21719218970635) bank177579 +177580 POINT(37.4311622607305 -122.59315254685897) bank177580 +177581 POINT(37.094652267363074 -122.06314628396764) bank177581 +177582 POINT(36.93073086557913 -121.69877290948507) bank177582 +177583 POINT(37.207832061907105 -122.73442890198916) bank177583 +177584 POINT(38.14604722621218 -122.79840333225361) bank177584 +177585 POINT(37.529989291675285 -122.46399415233272) bank177585 +177586 POINT(38.63719692643674 -123.19057769022777) bank177586 +177587 POINT(38.765606213988406 -122.37987587366294) bank177587 +177588 POINT(37.480923012240424 -121.73846505785811) bank177588 +177589 POINT(36.96787205486271 -122.82308545677422) bank177589 +177590 POINT(38.684255453780544 -122.07059724173254) bank177590 +177591 POINT(37.025184085213304 -123.19902378154681) bank177591 +177592 POINT(37.532656476329585 -122.25727367453524) bank177592 +177593 POINT(38.474890173355575 -123.33831369900311) bank177593 +177594 POINT(37.04508238261907 -123.08078434349193) bank177594 +177595 POINT(37.588125561645825 -121.43734562016513) bank177595 +177596 POINT(37.626589601981465 -123.16969137921805) bank177596 +177597 POINT(37.79441903146481 -122.90833698445213) bank177597 +177598 POINT(37.05189604851839 -121.47859803837719) bank177598 +177599 POINT(38.3810473562384 -121.65587973382206) bank177599 +177600 POINT(37.067878637476916 -121.55248360673353) bank177600 +177601 POINT(37.794795748996144 -121.92714623095848) bank177601 +177602 POINT(37.81723194505854 -122.26595383812649) bank177602 +177603 POINT(36.96469853579138 -122.15292453882702) bank177603 +177604 POINT(37.505016520450226 -123.00534263441916) bank177604 +177605 POINT(38.024084925415785 -121.75078067181303) bank177605 +177606 POINT(37.22277937808297 -122.7774337106455) bank177606 +177607 POINT(38.3991646429345 -121.72031067450341) bank177607 +177608 POINT(36.97771195534975 -122.46913205904299) bank177608 +177609 POINT(37.82400441152203 -122.16382991097076) bank177609 +177610 POINT(37.845743654351004 -122.14877616979788) bank177610 +177611 POINT(37.552758492516126 -122.78459077244912) bank177611 +177612 POINT(38.585761360673715 -121.61842537671791) bank177612 +177613 POINT(37.273157322727734 -122.90765111612373) bank177613 +177614 POINT(38.63928128337416 -122.39536808706832) bank177614 +177615 POINT(37.67563835351218 -121.71275788541722) bank177615 +177616 POINT(37.41217986540173 -123.26676525463147) bank177616 +177617 POINT(37.5553279645328 -121.72668852300004) bank177617 +177618 POINT(36.85481193349352 -122.48313996848255) bank177618 +177619 POINT(37.03767903693433 -121.62616658553918) bank177619 +177620 POINT(37.525378833273784 -122.15161815611386) bank177620 +177621 POINT(37.34469380771258 -123.23731159727325) bank177621 +177622 POINT(37.20962147889787 -122.3207079814174) bank177622 +177623 POINT(37.32239526383423 -122.29676747250997) bank177623 +177624 POINT(38.74483085659938 -122.00701498774049) bank177624 +177625 POINT(36.84812432023352 -123.14724915612157) bank177625 +177626 POINT(37.01718555421498 -121.54770861426702) bank177626 +177627 POINT(38.049944884339894 -122.65997358537564) bank177627 +177628 POINT(37.370385296440276 -122.40033637635364) bank177628 +177629 POINT(37.44840498948775 -121.67268762439473) bank177629 +177630 POINT(37.3392078556011 -123.39138482000814) bank177630 +177631 POINT(37.99646563429097 -122.61444971736657) bank177631 +177632 POINT(38.763394578853955 -121.65142482630074) bank177632 +177633 POINT(36.78091156928843 -121.48917515242005) bank177633 +177634 POINT(37.28476966649686 -122.27765087431285) bank177634 +177635 POINT(38.39857100421305 -122.44668923882618) bank177635 +177636 POINT(38.338937474284045 -122.62764649873945) bank177636 +177637 POINT(37.07198778528353 -123.14572244373127) bank177637 +177638 POINT(36.96803821637751 -123.19843419311549) bank177638 +177639 POINT(37.2847285415346 -122.67473245416025) bank177639 +177640 POINT(38.55253932872923 -123.41796460507895) bank177640 +177641 POINT(37.60175441924834 -121.49641812569791) bank177641 +177642 POINT(37.194293969899874 -122.36095710424803) bank177642 +177643 POINT(37.87246431038797 -122.02918512933414) bank177643 +177644 POINT(38.15090881321003 -123.23551095697484) bank177644 +177645 POINT(36.88647678845053 -122.81193995135555) bank177645 +177646 POINT(37.68355026831423 -121.92244335335405) bank177646 +177647 POINT(37.34255241620004 -123.17987434006825) bank177647 +177648 POINT(38.682578810184275 -122.98966008488807) bank177648 +177649 POINT(36.84833968855266 -122.5784394627062) bank177649 +177650 POINT(37.96633031093274 -122.63743732989087) bank177650 +177651 POINT(37.3637681141123 -123.33794432893706) bank177651 +177652 POINT(37.743763122395116 -123.15582286125176) bank177652 +177653 POINT(37.54997827028253 -122.91310412260675) bank177653 +177654 POINT(36.81377329635201 -122.28404378634187) bank177654 +177655 POINT(36.96716472189749 -121.92117913596303) bank177655 +177656 POINT(37.070152491170056 -122.51136882043076) bank177656 +177657 POINT(37.55107279341937 -121.57323592249227) bank177657 +177658 POINT(38.75149405719272 -121.47813266497855) bank177658 +177659 POINT(37.75873127806911 -123.27916579868209) bank177659 +177660 POINT(37.60307242359891 -122.9972146647683) bank177660 +177661 POINT(37.37269728443387 -122.36719419980625) bank177661 +177662 POINT(38.17057910219001 -122.0635157795058) bank177662 +177663 POINT(38.432688484156124 -122.0594864684838) bank177663 +177664 POINT(38.26922652436589 -122.32503517113133) bank177664 +177665 POINT(37.90463644378911 -122.7752773266705) bank177665 +177666 POINT(37.55817881996341 -123.18876214865251) bank177666 +177667 POINT(37.49546822884516 -122.7472932678623) bank177667 +177668 POINT(37.18646318042315 -122.41349500227354) bank177668 +177669 POINT(37.54506159549558 -123.38643472269237) bank177669 +177670 POINT(38.586270155932844 -123.26178365251808) bank177670 +177671 POINT(37.13638481472903 -122.37286742151937) bank177671 +177672 POINT(38.61182180870643 -121.60434946565904) bank177672 +177673 POINT(37.396991113453055 -122.49233254400825) bank177673 +177674 POINT(36.866364884144716 -121.49745917146211) bank177674 +177675 POINT(37.47429034439943 -122.01462230350181) bank177675 +177676 POINT(37.52993273445557 -123.4036314550881) bank177676 +177677 POINT(37.632041480090585 -122.1513603671006) bank177677 +177678 POINT(38.18161405625477 -123.2540966844668) bank177678 +177679 POINT(37.952614602332794 -123.07859170091193) bank177679 +177680 POINT(38.49007662629937 -121.55541507388206) bank177680 +177681 POINT(38.7328885070799 -122.24939912561207) bank177681 +177682 POINT(38.35069106513519 -123.03579876265377) bank177682 +177683 POINT(37.866732225408526 -121.87474827371472) bank177683 +177684 POINT(37.792294043002535 -122.59179101651644) bank177684 +177685 POINT(37.041458531631804 -123.13159524444717) bank177685 +177686 POINT(38.38098216846782 -122.48852660368641) bank177686 +177687 POINT(36.935912398533425 -121.68892698330404) bank177687 +177688 POINT(37.36163010766064 -122.17207472713588) bank177688 +177689 POINT(38.70781184906981 -122.72530352022052) bank177689 +177690 POINT(38.57384506821285 -121.98518367307749) bank177690 +177691 POINT(37.79914482245808 -123.39092145837103) bank177691 +177692 POINT(38.31497481852785 -122.37787990020563) bank177692 +177693 POINT(37.095761301717324 -122.94914066925645) bank177693 +177694 POINT(37.116359630220984 -122.64913884354466) bank177694 +177695 POINT(37.84018108151712 -123.01053940422747) bank177695 +177696 POINT(37.72120858966652 -123.24381983535744) bank177696 +177697 POINT(37.220866579593896 -123.0157067383568) bank177697 +177698 POINT(38.51641311742067 -122.51728582518189) bank177698 +177699 POINT(37.59537034396878 -122.07236963240102) bank177699 +177700 POINT(37.7359441557514 -123.38571224834907) bank177700 +177701 POINT(38.32586839699199 -122.11370073132689) bank177701 +177702 POINT(37.333039377589266 -122.4961240339482) bank177702 +177703 POINT(36.94249316596185 -123.16873306037523) bank177703 +177704 POINT(37.88871358259633 -121.46755262519505) bank177704 +177705 POINT(36.81862278635273 -121.95585288042544) bank177705 +177706 POINT(36.88491719100498 -122.3111438359629) bank177706 +177707 POINT(38.40746801800822 -121.6358716639399) bank177707 +177708 POINT(38.27614020992528 -121.58508003475727) bank177708 +177709 POINT(36.841766717375286 -123.35377477314069) bank177709 +177710 POINT(36.894436797653384 -121.95966855018361) bank177710 +177711 POINT(37.78684001468556 -122.13617673382154) bank177711 +177712 POINT(37.52095255089451 -122.02655817914201) bank177712 +177713 POINT(38.15353268281963 -122.09884483497235) bank177713 +177714 POINT(37.84548022159968 -121.97363637540799) bank177714 +177715 POINT(37.654452861326064 -122.01549989919845) bank177715 +177716 POINT(37.75265413500339 -122.11724065340988) bank177716 +177717 POINT(37.464925404957945 -123.41164334347097) bank177717 +177718 POINT(37.29434833407262 -123.15870479450624) bank177718 +177719 POINT(38.05929855264855 -122.0307362236741) bank177719 +177720 POINT(37.47948070239963 -122.98602402752066) bank177720 +177721 POINT(37.2334912439313 -122.59306326282982) bank177721 +177722 POINT(38.02382661619327 -121.90855452671785) bank177722 +177723 POINT(37.50581140086452 -122.6043157881358) bank177723 +177724 POINT(38.74495519659335 -122.67397389693758) bank177724 +177725 POINT(36.8523888795839 -122.7752008571914) bank177725 +177726 POINT(37.61353140768509 -122.36443319802396) bank177726 +177727 POINT(37.73966543671958 -122.24117876078995) bank177727 +177728 POINT(36.89602207659918 -122.12414086780767) bank177728 +177729 POINT(37.1509222413178 -121.9379955888324) bank177729 +177730 POINT(38.02024321745744 -122.79416408139254) bank177730 +177731 POINT(38.35361375708675 -122.8989507990172) bank177731 +177732 POINT(38.30659450116063 -122.36647459689581) bank177732 +177733 POINT(37.95268656769525 -121.47925091615103) bank177733 +177734 POINT(36.987159020119755 -122.97564202830694) bank177734 +177735 POINT(38.157343456108066 -122.02535622064715) bank177735 +177736 POINT(37.49821550492163 -122.85635209385758) bank177736 +177737 POINT(37.18132174245213 -123.36606045289719) bank177737 +177738 POINT(38.663933081780975 -123.34177495980485) bank177738 +177739 POINT(38.37465011758113 -121.43860163735422) bank177739 +177740 POINT(38.60882958618908 -122.95834403797805) bank177740 +177741 POINT(36.939802981523016 -122.52970209573776) bank177741 +177742 POINT(38.37089183159791 -122.75056984709205) bank177742 +177743 POINT(37.65754617075976 -122.94181086696648) bank177743 +177744 POINT(37.39332065924588 -123.14437270176828) bank177744 +177745 POINT(38.63539797111838 -122.95364309083861) bank177745 +177746 POINT(38.150805589584536 -123.37729784218851) bank177746 +177747 POINT(36.83080693272414 -122.3682938511816) bank177747 +177748 POINT(38.2999073067532 -122.40901721516587) bank177748 +177749 POINT(37.71811441833941 -121.45146839761706) bank177749 +177750 POINT(38.44356839213083 -121.63148826087992) bank177750 +177751 POINT(37.21254948879406 -121.82550163076245) bank177751 +177752 POINT(36.8757016695051 -121.58718079442342) bank177752 +177753 POINT(38.101603160695284 -122.83321543800837) bank177753 +177754 POINT(38.41505239145285 -123.05924428008113) bank177754 +177755 POINT(37.435998227335276 -123.20314171058169) bank177755 +177756 POINT(38.619052937777646 -122.07626907941133) bank177756 +177757 POINT(37.18296640514533 -123.06673179666564) bank177757 +177758 POINT(38.12999850550022 -123.20879500822275) bank177758 +177759 POINT(36.77807235787131 -121.81561994596053) bank177759 +177760 POINT(36.99534966326019 -122.69752366343606) bank177760 +177761 POINT(38.20479226371044 -121.51032109898878) bank177761 +177762 POINT(38.408248035275065 -122.86000562524073) bank177762 +177763 POINT(38.72225777129527 -121.92546818555219) bank177763 +177764 POINT(38.07539210986835 -122.06905000528674) bank177764 +177765 POINT(38.439061173521246 -122.59765753775949) bank177765 +177766 POINT(37.148098605370535 -121.69695975068744) bank177766 +177767 POINT(38.34888932689341 -123.08708939466945) bank177767 +177768 POINT(38.0519503797472 -121.7794097465387) bank177768 +177769 POINT(38.0324464716811 -121.80235109624529) bank177769 +177770 POINT(37.93811927573184 -122.82278113175259) bank177770 +177771 POINT(37.358161101040324 -122.88948312740035) bank177771 +177772 POINT(37.379851974259246 -122.46750679050722) bank177772 +177773 POINT(36.81039408827009 -122.98382696151516) bank177773 +177774 POINT(36.98576882736995 -122.28374403592743) bank177774 +177775 POINT(37.08502031424106 -122.70990301089837) bank177775 +177776 POINT(37.64842753770524 -123.30604872111194) bank177776 +177777 POINT(37.776081063727226 -123.2058725998138) bank177777 +177778 POINT(37.40752560653877 -122.94127563894314) bank177778 +177779 POINT(37.5443609002404 -122.45481780343133) bank177779 +177780 POINT(37.81704926152173 -123.10592402226493) bank177780 +177781 POINT(37.7992721834558 -121.58451369671685) bank177781 +177782 POINT(37.679934646006885 -123.2827313427628) bank177782 +177783 POINT(38.37170919445023 -121.93739229866341) bank177783 +177784 POINT(37.69414062781576 -123.01567996538134) bank177784 +177785 POINT(36.95901334624312 -122.00632901261845) bank177785 +177786 POINT(38.5606670029038 -122.46711791517612) bank177786 +177787 POINT(37.19933928757286 -122.02049047017589) bank177787 +177788 POINT(38.342323702360744 -122.75197253319723) bank177788 +177789 POINT(37.47350367115425 -121.82787588586238) bank177789 +177790 POINT(37.911964378911854 -123.36041306749175) bank177790 +177791 POINT(37.75435998042766 -121.88838601307597) bank177791 +177792 POINT(37.948105736292156 -123.03813614032049) bank177792 +177793 POINT(37.61611028857057 -122.40676908579253) bank177793 +177794 POINT(37.654717030710565 -122.69148974640196) bank177794 +177795 POINT(37.5144752466331 -122.6349255260859) bank177795 +177796 POINT(38.09089466688425 -122.8113929438126) bank177796 +177797 POINT(37.549272891493615 -122.82313422731745) bank177797 +177798 POINT(37.29678735426755 -123.24039088580352) bank177798 +177799 POINT(38.495666225027506 -122.65150653207095) bank177799 +177800 POINT(38.042989337054685 -123.19800170072557) bank177800 +177801 POINT(38.682291159668345 -122.99999261277706) bank177801 +177802 POINT(37.02131355399932 -121.50513219496919) bank177802 +177803 POINT(36.831209544045116 -123.05490286420115) bank177803 +177804 POINT(37.22721165205464 -122.7096455986215) bank177804 +177805 POINT(37.902388078754676 -121.62016366994392) bank177805 +177806 POINT(37.96757685053483 -121.7866393002961) bank177806 +177807 POINT(36.843124443500024 -122.76785084006804) bank177807 +177808 POINT(38.02333540201235 -122.42771961950014) bank177808 +177809 POINT(37.86152672324119 -121.65532893651311) bank177809 +177810 POINT(38.20427580522233 -122.53958663104484) bank177810 +177811 POINT(38.19673517913973 -123.22114898329721) bank177811 +177812 POINT(38.02649909036358 -122.4380026179209) bank177812 +177813 POINT(37.55888811136042 -121.67186762508452) bank177813 +177814 POINT(36.87869143086439 -123.36747306263844) bank177814 +177815 POINT(36.991814386267954 -122.83541070914279) bank177815 +177816 POINT(36.82089587218451 -122.97076038707561) bank177816 +177817 POINT(37.16280146151063 -121.4570002574648) bank177817 +177818 POINT(37.839406949824294 -123.07029160211698) bank177818 +177819 POINT(37.193152163779 -121.48023569041082) bank177819 +177820 POINT(37.044710744610555 -122.17621849096216) bank177820 +177821 POINT(38.18366258277036 -121.84955318977853) bank177821 +177822 POINT(37.211181559971195 -123.28915872378602) bank177822 +177823 POINT(37.602673914169706 -122.76061891939611) bank177823 +177824 POINT(37.596198633141036 -123.21785309639986) bank177824 +177825 POINT(38.32460010894352 -122.34843006350009) bank177825 +177826 POINT(37.10628685357338 -121.44881360838832) bank177826 +177827 POINT(37.290015780618646 -121.49433554278413) bank177827 +177828 POINT(37.62552747478513 -122.16380236653752) bank177828 +177829 POINT(37.1179386098313 -122.40176980278302) bank177829 +177830 POINT(38.48266511029709 -122.49682484789652) bank177830 +177831 POINT(36.932773547818016 -121.98434274734115) bank177831 +177832 POINT(37.43160339740365 -122.0597823436116) bank177832 +177833 POINT(37.366998209721515 -121.93785923248986) bank177833 +177834 POINT(36.95837622674537 -122.88084486305793) bank177834 +177835 POINT(38.168145131837996 -121.72894981879222) bank177835 +177836 POINT(37.1729235928297 -123.29903010087794) bank177836 +177837 POINT(36.91977303041851 -121.62252601205614) bank177837 +177838 POINT(37.42356318732278 -122.7987775561372) bank177838 +177839 POINT(38.25658635895291 -122.44934010312235) bank177839 +177840 POINT(37.02479030879889 -121.43672279839092) bank177840 +177841 POINT(37.12586296158244 -123.23922148254452) bank177841 +177842 POINT(38.35800747353735 -123.14884673221239) bank177842 +177843 POINT(37.45575484213986 -123.29306940768265) bank177843 +177844 POINT(36.98336962166773 -122.88530595921013) bank177844 +177845 POINT(38.06886909800216 -122.1427830697681) bank177845 +177846 POINT(37.687448468490416 -122.26185856358042) bank177846 +177847 POINT(38.620782959892104 -122.37825395941233) bank177847 +177848 POINT(36.781328413981925 -122.18440228590448) bank177848 +177849 POINT(37.50144296504725 -121.99168025978366) bank177849 +177850 POINT(37.34812617085519 -122.96652080867165) bank177850 +177851 POINT(36.91635571224736 -122.03034071501371) bank177851 +177852 POINT(37.9730458509777 -121.55028656598004) bank177852 +177853 POINT(37.472711731560274 -122.65786928621017) bank177853 +177854 POINT(38.30439326654326 -121.75928241548499) bank177854 +177855 POINT(37.556353159766324 -121.6937447637437) bank177855 +177856 POINT(37.00655627365289 -122.71365479385214) bank177856 +177857 POINT(37.21572282327774 -121.7459953813853) bank177857 +177858 POINT(38.53551547004192 -123.00630536457189) bank177858 +177859 POINT(36.914699749543615 -122.9754769090424) bank177859 +177860 POINT(38.62486272768394 -121.45999841185217) bank177860 +177861 POINT(37.06738291066016 -122.26134977239985) bank177861 +177862 POINT(38.35930963302854 -122.38560027615333) bank177862 +177863 POINT(37.39794993813051 -122.24228761268601) bank177863 +177864 POINT(37.96535619414848 -122.0176717192739) bank177864 +177865 POINT(37.79960173670992 -122.32453278273915) bank177865 +177866 POINT(37.04735287880401 -122.27972790166028) bank177866 +177867 POINT(38.495917944710044 -122.20986771722717) bank177867 +177868 POINT(38.37675109859189 -121.61057632865538) bank177868 +177869 POINT(37.94801871023165 -122.34964892044029) bank177869 +177870 POINT(37.219434828805774 -121.42578119494524) bank177870 +177871 POINT(38.382613049715275 -121.57518973957777) bank177871 +177872 POINT(37.31176249454038 -122.71053309096662) bank177872 +177873 POINT(37.65012650751511 -122.71887616617286) bank177873 +177874 POINT(38.193422992372916 -122.10672394180315) bank177874 +177875 POINT(37.580700366715014 -122.56687985555007) bank177875 +177876 POINT(38.042521975014616 -122.73673791668521) bank177876 +177877 POINT(38.61509752668879 -122.68368470087738) bank177877 +177878 POINT(38.01663351482179 -121.55892183601412) bank177878 +177879 POINT(38.56423625641149 -122.42239091556266) bank177879 +177880 POINT(37.03240993970864 -122.00143231945587) bank177880 +177881 POINT(38.533846937396575 -123.22573928526361) bank177881 +177882 POINT(37.77186687852641 -121.82501901246863) bank177882 +177883 POINT(38.08834751314276 -122.76282949329396) bank177883 +177884 POINT(37.26898412282994 -122.66329464693976) bank177884 +177885 POINT(36.86821133297938 -123.40747150683306) bank177885 +177886 POINT(37.92491782021041 -122.98043630094031) bank177886 +177887 POINT(37.25345905754823 -122.12974296545302) bank177887 +177888 POINT(38.26541606324081 -122.45447861464021) bank177888 +177889 POINT(38.328600702440994 -122.91595671043343) bank177889 +177890 POINT(38.56477393161173 -121.747301967383) bank177890 +177891 POINT(38.38992402941449 -122.85891550142527) bank177891 +177892 POINT(37.46274760466099 -121.4195704107626) bank177892 +177893 POINT(37.44241255349045 -121.81696169792345) bank177893 +177894 POINT(36.803557935931 -123.33642011020046) bank177894 +177895 POINT(37.951101580365034 -122.99456251753081) bank177895 +177896 POINT(37.05007517903554 -123.30837081388856) bank177896 +177897 POINT(37.54629762087189 -123.13447736907239) bank177897 +177898 POINT(36.98672330104962 -122.98675103112853) bank177898 +177899 POINT(38.52296359573537 -122.65497744014758) bank177899 +177900 POINT(38.510341224639674 -122.2982154607391) bank177900 +177901 POINT(36.95369785447645 -121.72704114011869) bank177901 +177902 POINT(38.59465077703678 -122.96173933264717) bank177902 +177903 POINT(37.71908773738421 -122.60671891677148) bank177903 +177904 POINT(37.316054794376136 -122.73970742111626) bank177904 +177905 POINT(38.664123416130344 -122.90713544755407) bank177905 +177906 POINT(37.95066014247582 -122.47251675818282) bank177906 +177907 POINT(37.824194214054444 -122.16715901168608) bank177907 +177908 POINT(37.736881285539546 -121.55794488856607) bank177908 +177909 POINT(38.25097836887896 -122.6432132648279) bank177909 +177910 POINT(37.34344162911735 -123.17800941599039) bank177910 +177911 POINT(38.71635533180624 -122.47680696106895) bank177911 +177912 POINT(37.45354300907226 -122.31292566377606) bank177912 +177913 POINT(37.344434920258536 -123.12357164337627) bank177913 +177914 POINT(37.08362558002294 -122.11436175090071) bank177914 +177915 POINT(38.50292520296971 -122.43760504226016) bank177915 +177916 POINT(37.615169208771455 -121.7760095977173) bank177916 +177917 POINT(37.94955760729971 -122.01201346648963) bank177917 +177918 POINT(38.4311944983478 -121.62100032796849) bank177918 +177919 POINT(36.890775194773994 -123.2759526436814) bank177919 +177920 POINT(38.413068871955524 -122.42759253435005) bank177920 +177921 POINT(38.20235055692062 -122.61593055444759) bank177921 +177922 POINT(38.414018690184555 -123.16592615005034) bank177922 +177923 POINT(38.01786570516142 -123.2954258955997) bank177923 +177924 POINT(38.06329270748607 -122.37542297943308) bank177924 +177925 POINT(38.28463879693335 -122.75621958357023) bank177925 +177926 POINT(38.700165749930676 -121.50019521414781) bank177926 +177927 POINT(37.85251080382695 -122.22862994459943) bank177927 +177928 POINT(36.785039273141244 -121.7425522645463) bank177928 +177929 POINT(37.12081833389154 -122.6290381940898) bank177929 +177930 POINT(38.50503719763128 -123.11500545068334) bank177930 +177931 POINT(36.95757708177005 -121.98359648842118) bank177931 +177932 POINT(38.46794755533731 -123.01761835215697) bank177932 +177933 POINT(38.72677032886048 -121.99267450520334) bank177933 +177934 POINT(37.320716800025366 -122.0909468116641) bank177934 +177935 POINT(37.26114487176766 -122.78652885784564) bank177935 +177936 POINT(37.93365486780133 -122.88326093386708) bank177936 +177937 POINT(37.34975467245772 -122.215187598902) bank177937 +177938 POINT(36.925754906170184 -122.85305276445227) bank177938 +177939 POINT(36.79221268470679 -122.52340039486349) bank177939 +177940 POINT(37.73183096821112 -121.63030125880528) bank177940 +177941 POINT(38.27730388986313 -122.76349164250831) bank177941 +177942 POINT(37.481987019118854 -121.64892236895152) bank177942 +177943 POINT(37.62271284833499 -121.95330675224261) bank177943 +177944 POINT(37.33588657397764 -122.03206634132817) bank177944 +177945 POINT(36.82523835479428 -123.21180102001898) bank177945 +177946 POINT(37.387485877381465 -122.143141765881) bank177946 +177947 POINT(38.74526171106913 -122.56352771762) bank177947 +177948 POINT(38.482211463021095 -122.91729177717647) bank177948 +177949 POINT(38.63680410122299 -122.16764640132736) bank177949 +177950 POINT(37.46657873283603 -123.35336292108028) bank177950 +177951 POINT(38.28749430665109 -123.31083335361949) bank177951 +177952 POINT(38.46413640981003 -122.42624978546468) bank177952 +177953 POINT(37.20406761358718 -121.66399908519122) bank177953 +177954 POINT(38.4715419996619 -121.85362525955638) bank177954 +177955 POINT(37.709635614789015 -121.78158955906585) bank177955 +177956 POINT(38.38703584217967 -122.3596184860187) bank177956 +177957 POINT(37.93544479716923 -122.72059855980426) bank177957 +177958 POINT(37.83893376242022 -122.50314607640287) bank177958 +177959 POINT(38.66013303728642 -121.76067195108593) bank177959 +177960 POINT(37.431081757907194 -123.30591231520907) bank177960 +177961 POINT(38.2102993215684 -122.92592173416048) bank177961 +177962 POINT(37.655586522181714 -123.34369320612873) bank177962 +177963 POINT(37.676227573331765 -121.67318537919206) bank177963 +177964 POINT(36.96467540905346 -122.81115086628283) bank177964 +177965 POINT(38.05624547335608 -122.32593572686919) bank177965 +177966 POINT(37.23485025297377 -121.86850855314938) bank177966 +177967 POINT(37.727952735177205 -122.93783431923352) bank177967 +177968 POINT(37.00900184618094 -121.9327378538366) bank177968 +177969 POINT(37.87511479638021 -122.96446915464777) bank177969 +177970 POINT(38.24865154829817 -123.2231366410956) bank177970 +177971 POINT(38.40531821224306 -123.16447103107969) bank177971 +177972 POINT(36.933715682354475 -122.22431713817488) bank177972 +177973 POINT(37.50797353084065 -122.4091802722561) bank177973 +177974 POINT(37.690243937582785 -121.93399817755262) bank177974 +177975 POINT(37.09390643870335 -121.43216467259744) bank177975 +177976 POINT(36.850445141418774 -121.65006040148151) bank177976 +177977 POINT(37.47756874547962 -123.29009230000887) bank177977 +177978 POINT(37.12730410623107 -121.76662101667411) bank177978 +177979 POINT(37.62382565202445 -123.04041227749157) bank177979 +177980 POINT(38.47895740977873 -121.98366525494562) bank177980 +177981 POINT(37.568276853803745 -121.94512369915459) bank177981 +177982 POINT(38.03484089383069 -121.49559404916036) bank177982 +177983 POINT(37.14620094718733 -121.46925379916226) bank177983 +177984 POINT(37.69225769924202 -122.22180933757325) bank177984 +177985 POINT(38.5902611538569 -122.24600396966433) bank177985 +177986 POINT(38.48104149620294 -123.09266085513134) bank177986 +177987 POINT(38.56802402817659 -121.7964897296455) bank177987 +177988 POINT(38.44248495254347 -123.3127612905299) bank177988 +177989 POINT(36.88845233316463 -121.52598267749714) bank177989 +177990 POINT(37.734421479081675 -123.17214252641435) bank177990 +177991 POINT(37.36147946924796 -122.95752131675572) bank177991 +177992 POINT(38.35009423469744 -121.87957725563435) bank177992 +177993 POINT(38.39576921824104 -121.77012094190965) bank177993 +177994 POINT(38.141663850810104 -121.68313647446043) bank177994 +177995 POINT(38.67680039878162 -122.39195661379068) bank177995 +177996 POINT(37.01837978932113 -121.79194414517309) bank177996 +177997 POINT(37.950650997704315 -123.31606178946879) bank177997 +177998 POINT(38.444465616064576 -121.79734431022928) bank177998 +177999 POINT(37.866459759202804 -121.51821841740026) bank177999 +178000 POINT(38.72824664634664 -122.6921346395332) bank178000 +178001 POINT(38.006349344136986 -122.26173801079666) bank178001 +178002 POINT(38.39588635714858 -123.02413353837132) bank178002 +178003 POINT(36.883675038371756 -121.86494736034888) bank178003 +178004 POINT(37.55644871563722 -123.00928800141995) bank178004 +178005 POINT(38.11665239174824 -123.2871578013571) bank178005 +178006 POINT(38.48100926634813 -121.56861943328992) bank178006 +178007 POINT(36.78133689491421 -121.49763621573618) bank178007 +178008 POINT(37.6027683564711 -122.81293210970975) bank178008 +178009 POINT(37.109093372501725 -121.572623058646) bank178009 +178010 POINT(36.85730143889345 -121.73129602642473) bank178010 +178011 POINT(38.66375281087151 -122.17289592014392) bank178011 +178012 POINT(37.39982254633905 -121.88057832348825) bank178012 +178013 POINT(38.316378205370576 -122.69727579191202) bank178013 +178014 POINT(37.53312177495187 -122.77363532629639) bank178014 +178015 POINT(37.8564379806018 -122.14838785427652) bank178015 +178016 POINT(37.37786085743364 -122.429316977022) bank178016 +178017 POINT(38.23388407185344 -122.81928435699882) bank178017 +178018 POINT(36.84698897448689 -121.9175923282402) bank178018 +178019 POINT(37.61267941459009 -122.581170888132) bank178019 +178020 POINT(36.84856948623774 -121.61845690795016) bank178020 +178021 POINT(38.61750245990678 -123.26441358582998) bank178021 +178022 POINT(38.233545965023616 -123.27063829004805) bank178022 +178023 POINT(36.88685238113019 -122.6481815249566) bank178023 +178024 POINT(38.21619146156988 -123.33848461409228) bank178024 +178025 POINT(36.77924952722954 -122.73156758166456) bank178025 +178026 POINT(38.42266220199062 -123.40355321457842) bank178026 +178027 POINT(37.82002757269009 -122.99279417838224) bank178027 +178028 POINT(37.57530924213093 -121.90221738209708) bank178028 +178029 POINT(38.5703430962985 -122.48171782196268) bank178029 +178030 POINT(38.59869323137906 -123.41041649659448) bank178030 +178031 POINT(37.524079842571474 -123.34068246510537) bank178031 +178032 POINT(38.226044001851044 -122.16695806703757) bank178032 +178033 POINT(38.751326106100784 -123.36250446393144) bank178033 +178034 POINT(37.79945867568654 -122.45807524624098) bank178034 +178035 POINT(38.03677200112734 -121.72182652539296) bank178035 +178036 POINT(38.345613044610026 -121.48119673463093) bank178036 +178037 POINT(37.50027788734063 -121.64893709850827) bank178037 +178038 POINT(36.8959323296281 -123.06105873004532) bank178038 +178039 POINT(37.02892676203417 -122.80717622173445) bank178039 +178040 POINT(37.53261104484062 -123.3374052350271) bank178040 +178041 POINT(37.09878051093915 -123.34951065865702) bank178041 +178042 POINT(38.572013096533134 -121.43556990064079) bank178042 +178043 POINT(37.184279070118315 -122.41138532432049) bank178043 +178044 POINT(38.46772038527006 -122.66022653321508) bank178044 +178045 POINT(36.85044518255235 -122.78237721425461) bank178045 +178046 POINT(37.74401680816844 -122.70726059376057) bank178046 +178047 POINT(38.143894179505196 -121.88666022297343) bank178047 +178048 POINT(37.181831233156075 -123.13355247789256) bank178048 +178049 POINT(37.14388065219366 -122.90633830614345) bank178049 +178050 POINT(38.40031137621822 -122.0486993866368) bank178050 +178051 POINT(37.25241391715053 -123.34976202636406) bank178051 +178052 POINT(38.74984646845486 -121.90871029460325) bank178052 +178053 POINT(36.89779053447174 -122.57515015812724) bank178053 +178054 POINT(38.18543435816656 -123.41289619920629) bank178054 +178055 POINT(38.29562393875489 -122.07633995735024) bank178055 +178056 POINT(37.236635597780634 -122.6349675754827) bank178056 +178057 POINT(38.03482610201302 -122.2488862238932) bank178057 +178058 POINT(38.71364481499519 -121.883308284061) bank178058 +178059 POINT(38.679360188301615 -123.04359983027642) bank178059 +178060 POINT(36.83731847664275 -123.26242086180535) bank178060 +178061 POINT(37.360568341296336 -123.01721748026709) bank178061 +178062 POINT(38.5254254440261 -122.29697260190497) bank178062 +178063 POINT(38.056694892474006 -122.29684838628857) bank178063 +178064 POINT(37.04702250638202 -123.11919579230567) bank178064 +178065 POINT(37.766267150121806 -122.81628957638354) bank178065 +178066 POINT(36.937588627664326 -122.81436577222361) bank178066 +178067 POINT(37.031653001874126 -121.48379183408308) bank178067 +178068 POINT(37.687193814597684 -122.56085400155449) bank178068 +178069 POINT(38.06560325380471 -121.53079564584068) bank178069 +178070 POINT(37.978847073333995 -121.53444771095083) bank178070 +178071 POINT(37.46435227557465 -123.05730698200148) bank178071 +178072 POINT(37.71327741747546 -121.73422875542819) bank178072 +178073 POINT(37.63518742026283 -122.53082003988021) bank178073 +178074 POINT(38.13918622375466 -122.09350339012569) bank178074 +178075 POINT(38.69206571337123 -121.56315116810687) bank178075 +178076 POINT(37.52578070995905 -122.50468159568194) bank178076 +178077 POINT(38.53280629595801 -122.80327623200472) bank178077 +178078 POINT(37.85864891997452 -122.60045014413693) bank178078 +178079 POINT(37.70555681119498 -122.62554700335308) bank178079 +178080 POINT(37.517587370676814 -123.33120448588656) bank178080 +178081 POINT(36.99898024253981 -122.9695174236176) bank178081 +178082 POINT(37.10693356604264 -121.75525060536694) bank178082 +178083 POINT(38.61627813729071 -123.18277164413952) bank178083 +178084 POINT(38.241787786175465 -122.10488444409559) bank178084 +178085 POINT(36.871630039270066 -121.52209822008076) bank178085 +178086 POINT(38.76311196694449 -122.11083986864823) bank178086 +178087 POINT(37.605775817994854 -121.63066910470117) bank178087 +178088 POINT(37.24190317473997 -122.35017386437427) bank178088 +178089 POINT(37.38421280662434 -123.0163432727979) bank178089 +178090 POINT(38.0769852989601 -121.54494044695869) bank178090 +178091 POINT(38.65839926831215 -122.77184459482933) bank178091 +178092 POINT(38.4800008028706 -122.21332180010693) bank178092 +178093 POINT(38.6838105665297 -123.18579720080574) bank178093 +178094 POINT(38.53600513207748 -123.34047783483449) bank178094 +178095 POINT(38.707479374735584 -123.39826774762712) bank178095 +178096 POINT(37.417513491700404 -122.0664899000127) bank178096 +178097 POINT(37.333185162223295 -122.15613066874893) bank178097 +178098 POINT(37.43133647340254 -122.46620661940403) bank178098 +178099 POINT(36.93117832929566 -122.98920366396786) bank178099 +178100 POINT(36.910706060008096 -122.75795067506048) bank178100 +178101 POINT(37.515080011253836 -122.05391479958651) bank178101 +178102 POINT(38.615181674786676 -121.89788789781343) bank178102 +178103 POINT(37.898410644851886 -122.8888221652961) bank178103 +178104 POINT(38.03798671514724 -121.52527709470907) bank178104 +178105 POINT(37.26510217239833 -122.5956729564669) bank178105 +178106 POINT(38.02850704115646 -121.89086829489497) bank178106 +178107 POINT(37.6058042012093 -122.00349928510299) bank178107 +178108 POINT(38.693347784410825 -121.71738075582839) bank178108 +178109 POINT(38.09479616335748 -122.62460456591248) bank178109 +178110 POINT(36.896181939213356 -122.56735916544795) bank178110 +178111 POINT(38.0608903053942 -122.25521675762181) bank178111 +178112 POINT(36.98092429958585 -122.76037005098662) bank178112 +178113 POINT(38.28790746707965 -121.69138633576212) bank178113 +178114 POINT(38.05981491248059 -122.07722173539959) bank178114 +178115 POINT(36.98827230810198 -121.74497690601518) bank178115 +178116 POINT(37.50582001091084 -122.29363440984076) bank178116 +178117 POINT(38.153098678346964 -122.80620320433017) bank178117 +178118 POINT(38.61416096573037 -123.05336193706289) bank178118 +178119 POINT(37.439218581002045 -123.22381676761442) bank178119 +178120 POINT(37.18625006215527 -121.93080184748924) bank178120 +178121 POINT(37.46426829346289 -121.98071576224127) bank178121 +178122 POINT(37.79867015885579 -123.40566170636828) bank178122 +178123 POINT(38.49586220160303 -122.91247325882313) bank178123 +178124 POINT(37.747965031377376 -121.83302666291236) bank178124 +178125 POINT(36.77753320628102 -122.35140339433458) bank178125 +178126 POINT(38.40169002723062 -123.12894287295325) bank178126 +178127 POINT(38.40635263112583 -122.85858658476273) bank178127 +178128 POINT(38.70048527848981 -122.81121968865637) bank178128 +178129 POINT(37.742500193482954 -122.03802613717792) bank178129 +178130 POINT(38.66060675349046 -122.39915996330654) bank178130 +178131 POINT(38.4133084791095 -121.75156759872199) bank178131 +178132 POINT(37.11183558626685 -121.56958508200331) bank178132 +178133 POINT(37.63964407429008 -123.00428084152833) bank178133 +178134 POINT(36.91962288666968 -122.86809852009699) bank178134 +178135 POINT(38.198357744281395 -122.4327659661126) bank178135 +178136 POINT(38.406003468489104 -123.25003188196526) bank178136 +178137 POINT(36.819744281185855 -123.23072575271665) bank178137 +178138 POINT(38.19670836317137 -122.09933478747165) bank178138 +178139 POINT(37.88877359293074 -122.18208535928743) bank178139 +178140 POINT(38.32664840768281 -122.33298834670283) bank178140 +178141 POINT(37.13953632661978 -123.04185927355132) bank178141 +178142 POINT(37.992587008231084 -123.10906921004023) bank178142 +178143 POINT(38.774571445163076 -123.10408232644974) bank178143 +178144 POINT(38.27425499853669 -123.34015232926775) bank178144 +178145 POINT(38.45617566110327 -122.30921399266674) bank178145 +178146 POINT(36.77856642779413 -122.4897857321572) bank178146 +178147 POINT(37.94313594807249 -122.88754549082601) bank178147 +178148 POINT(36.85436603193734 -121.84156261625088) bank178148 +178149 POINT(37.30240400199454 -122.62591729113329) bank178149 +178150 POINT(36.91962497523361 -123.1511932082464) bank178150 +178151 POINT(38.08355947249099 -122.74055419611943) bank178151 +178152 POINT(37.073712295717826 -122.57630833254184) bank178152 +178153 POINT(38.57901171568401 -122.75167034339782) bank178153 +178154 POINT(36.90541933193974 -121.95477488861097) bank178154 +178155 POINT(37.49266181079067 -122.67570216976219) bank178155 +178156 POINT(37.90143289682939 -121.77900930282775) bank178156 +178157 POINT(37.98493236541768 -121.89191562184851) bank178157 +178158 POINT(38.009960670235564 -121.99160564554235) bank178158 +178159 POINT(37.840988782735266 -121.61613015975446) bank178159 +178160 POINT(36.81766877708916 -123.24515808507053) bank178160 +178161 POINT(38.495211447743515 -122.74859082725231) bank178161 +178162 POINT(38.51381295007678 -122.84457606921197) bank178162 +178163 POINT(37.500734302334266 -123.04315656704433) bank178163 +178164 POINT(38.1779784384573 -121.87894248935581) bank178164 +178165 POINT(37.755860505336784 -122.99000594260956) bank178165 +178166 POINT(38.471377616314015 -123.02304218843955) bank178166 +178167 POINT(37.25889322401376 -121.90552109195997) bank178167 +178168 POINT(37.04989888888864 -122.09176540016917) bank178168 +178169 POINT(37.39515475228098 -121.49690754429596) bank178169 +178170 POINT(37.13358192201958 -122.97334455419266) bank178170 +178171 POINT(36.923340092330506 -123.1650587656103) bank178171 +178172 POINT(37.800756985268855 -122.66377017606224) bank178172 +178173 POINT(38.586938814936374 -122.16401168509292) bank178173 +178174 POINT(36.95825989020763 -123.18051805504398) bank178174 +178175 POINT(38.157634779289076 -122.93495655882785) bank178175 +178176 POINT(38.0408947418932 -121.47491821089004) bank178176 +178177 POINT(37.89018475163008 -123.27465924874149) bank178177 +178178 POINT(38.662620665362 -122.47272580329998) bank178178 +178179 POINT(38.201791917750725 -122.93399566967078) bank178179 +178180 POINT(37.009311147131086 -122.38620900894375) bank178180 +178181 POINT(38.46622197542584 -122.26618511196625) bank178181 +178182 POINT(37.955587766261225 -121.86877360427478) bank178182 +178183 POINT(36.948976477754 -121.74427264292203) bank178183 +178184 POINT(37.93723294046449 -122.210178968363) bank178184 +178185 POINT(37.070207738076405 -122.44764901142032) bank178185 +178186 POINT(37.027896020937995 -122.62884403870974) bank178186 +178187 POINT(38.34677160329303 -123.37517977311398) bank178187 +178188 POINT(38.27290722396269 -121.93461049692489) bank178188 +178189 POINT(37.749425684262924 -123.36790610467824) bank178189 +178190 POINT(38.68022240593803 -122.89218257768054) bank178190 +178191 POINT(36.899276042899736 -121.91297547404368) bank178191 +178192 POINT(38.22803282836543 -122.08444149981233) bank178192 +178193 POINT(38.42821978605221 -122.42985031976606) bank178193 +178194 POINT(37.11806358038743 -122.5294043824431) bank178194 +178195 POINT(37.72956237653053 -122.71049119653065) bank178195 +178196 POINT(38.56465118234675 -122.54540443547532) bank178196 +178197 POINT(38.66253592938806 -122.24079533970628) bank178197 +178198 POINT(38.42366372862965 -122.67861047249312) bank178198 +178199 POINT(37.46337419166698 -121.47185978939255) bank178199 +178200 POINT(37.820990137535254 -121.48931022010296) bank178200 +178201 POINT(38.40948363369045 -122.27662166031038) bank178201 +178202 POINT(38.09888720947068 -121.66233253389106) bank178202 +178203 POINT(37.195660982469995 -122.01858432412689) bank178203 +178204 POINT(37.621227116932786 -121.53088638192408) bank178204 +178205 POINT(37.64580105508992 -121.64800934805473) bank178205 +178206 POINT(37.02760022714513 -121.50951689270505) bank178206 +178207 POINT(38.1094429131387 -123.4069966298808) bank178207 +178208 POINT(37.339012313042154 -121.57676798451091) bank178208 +178209 POINT(37.14500151104393 -122.06026139136453) bank178209 +178210 POINT(37.75909567203251 -123.18258843599422) bank178210 +178211 POINT(38.50413242211785 -121.83268971169343) bank178211 +178212 POINT(38.71927140273229 -123.33192801558435) bank178212 +178213 POINT(38.743682602755314 -121.62571921970611) bank178213 +178214 POINT(37.87268088470195 -122.06194638861423) bank178214 +178215 POINT(38.110525863185174 -122.98174363791634) bank178215 +178216 POINT(37.5868601295261 -122.13386503220725) bank178216 +178217 POINT(37.799623113023465 -122.42935644408661) bank178217 +178218 POINT(38.65777563270069 -122.48577624550445) bank178218 +178219 POINT(37.4854847073453 -121.58844700230179) bank178219 +178220 POINT(36.83515119597297 -121.93846749465882) bank178220 +178221 POINT(37.140905258586194 -123.19106005558648) bank178221 +178222 POINT(36.83586840433203 -121.54511801576707) bank178222 +178223 POINT(38.334578642835176 -122.81658473798377) bank178223 +178224 POINT(37.44266285135133 -121.83614195147581) bank178224 +178225 POINT(38.50732272900178 -122.64615191242999) bank178225 +178226 POINT(37.30652446407366 -123.24729208708308) bank178226 +178227 POINT(38.678498688912704 -121.97341641197922) bank178227 +178228 POINT(38.38762742498814 -121.84985338789429) bank178228 +178229 POINT(38.598779415690004 -122.84878917322636) bank178229 +178230 POINT(37.48948707054172 -122.72934147971097) bank178230 +178231 POINT(37.35867562543878 -123.23487013788474) bank178231 +178232 POINT(38.72155725072855 -122.73684840680232) bank178232 +178233 POINT(37.21780461148693 -122.31982715731405) bank178233 +178234 POINT(36.810098460663625 -122.79741053743778) bank178234 +178235 POINT(38.50146534904493 -121.77304593796194) bank178235 +178236 POINT(38.75133298489628 -121.85702361528108) bank178236 +178237 POINT(37.85571424369979 -123.3143908273331) bank178237 +178238 POINT(37.901341496020635 -122.02696724759328) bank178238 +178239 POINT(36.84891031761963 -122.8659559158686) bank178239 +178240 POINT(38.590892521806 -122.99431927777124) bank178240 +178241 POINT(37.367948276249734 -122.60270475113043) bank178241 +178242 POINT(37.19651004353803 -121.87734505365385) bank178242 +178243 POINT(36.932209564645454 -122.67718690499248) bank178243 +178244 POINT(36.96186521440164 -122.7572501316665) bank178244 +178245 POINT(38.6474953072928 -122.92224692294823) bank178245 +178246 POINT(37.86640266938536 -121.89481630167063) bank178246 +178247 POINT(38.3740322384144 -121.57384287088738) bank178247 +178248 POINT(38.180955673354696 -123.33709382735977) bank178248 +178249 POINT(37.99607214071431 -121.44796041905849) bank178249 +178250 POINT(37.27623662220404 -121.96396111086054) bank178250 +178251 POINT(38.3880456609752 -122.50123814632795) bank178251 +178252 POINT(38.02557127381857 -123.01607291648598) bank178252 +178253 POINT(38.20054052033211 -122.68173074492337) bank178253 +178254 POINT(38.03195992064125 -121.82113210689363) bank178254 +178255 POINT(38.164925284249755 -122.54542106779526) bank178255 +178256 POINT(37.00296543822597 -122.71362756552354) bank178256 +178257 POINT(38.31914405513268 -122.78111189211714) bank178257 +178258 POINT(38.478784188716574 -123.32896811311927) bank178258 +178259 POINT(37.774172737741196 -122.87073369642202) bank178259 +178260 POINT(37.268522987418 -121.55092582406957) bank178260 +178261 POINT(38.163059205333454 -122.20747097555288) bank178261 +178262 POINT(36.819222157432584 -121.98123571220107) bank178262 +178263 POINT(37.659512671922364 -122.48358724812739) bank178263 +178264 POINT(37.60875235922958 -123.27121664899973) bank178264 +178265 POINT(37.95621700073474 -122.23290975051393) bank178265 +178266 POINT(37.467584833977185 -122.1712344631913) bank178266 +178267 POINT(36.99580986428374 -122.73823823054897) bank178267 +178268 POINT(38.538965706187255 -121.65640701774163) bank178268 +178269 POINT(38.451858600138166 -121.58603633108743) bank178269 +178270 POINT(36.95763932846931 -121.64396153390045) bank178270 +178271 POINT(37.57316047007856 -122.55615935315602) bank178271 +178272 POINT(37.94094114650105 -122.50286711084965) bank178272 +178273 POINT(38.63391651353612 -123.04119911548243) bank178273 +178274 POINT(38.01762997447414 -123.28690039587276) bank178274 +178275 POINT(38.00274018920605 -121.48022607566288) bank178275 +178276 POINT(37.446828899288064 -121.59852046338007) bank178276 +178277 POINT(38.022638287196315 -121.67942591776165) bank178277 +178278 POINT(38.415625829964746 -123.0473237103606) bank178278 +178279 POINT(37.438467458137 -122.3591823493559) bank178279 +178280 POINT(37.06434991802749 -121.74536781924562) bank178280 +178281 POINT(38.100502916870845 -121.71861729562288) bank178281 +178282 POINT(38.42592176421928 -123.23899005114892) bank178282 +178283 POINT(38.43901186992014 -122.12550861272877) bank178283 +178284 POINT(38.08382913355158 -122.83071459171624) bank178284 +178285 POINT(37.179249775943084 -122.00666675389203) bank178285 +178286 POINT(38.01281464890382 -122.30328951599012) bank178286 +178287 POINT(38.540690730352715 -122.9832103990308) bank178287 +178288 POINT(38.40043843016472 -123.02887980764167) bank178288 +178289 POINT(37.94748515049239 -123.07033811187323) bank178289 +178290 POINT(36.985256647572285 -122.34749216424794) bank178290 +178291 POINT(38.192489635943396 -123.15963265417886) bank178291 +178292 POINT(37.45379717893082 -122.6542150116905) bank178292 +178293 POINT(38.153302928174206 -123.09034060132008) bank178293 +178294 POINT(36.99247172533301 -121.84001485609745) bank178294 +178295 POINT(37.76902240225613 -123.2228502339885) bank178295 +178296 POINT(37.69245319026017 -122.97592577853213) bank178296 +178297 POINT(37.873242242831694 -123.314274452552) bank178297 +178298 POINT(37.129018981210336 -122.83926175451418) bank178298 +178299 POINT(38.16143001732521 -122.08866533239191) bank178299 +178300 POINT(37.24389396301779 -123.31901790692156) bank178300 +178301 POINT(37.46438486121671 -121.92119058258896) bank178301 +178302 POINT(38.10326318504367 -123.3257552000705) bank178302 +178303 POINT(37.505061287523986 -122.2880832759866) bank178303 +178304 POINT(37.065639289612285 -122.6820526889631) bank178304 +178305 POINT(37.08968159735761 -121.7403061441807) bank178305 +178306 POINT(37.92462036873474 -122.9179565802149) bank178306 +178307 POINT(37.274061019543204 -123.1089653368801) bank178307 +178308 POINT(37.78200306886849 -123.31333833699374) bank178308 +178309 POINT(38.3418143370686 -122.68593457584925) bank178309 +178310 POINT(36.9216788194075 -122.9542980509422) bank178310 +178311 POINT(37.59415381651934 -122.54990584329158) bank178311 +178312 POINT(38.7402432792408 -121.4694858528452) bank178312 +178313 POINT(38.290056997375075 -122.50196100022396) bank178313 +178314 POINT(38.68757902997955 -122.25265633970521) bank178314 +178315 POINT(36.964061426242836 -123.23885366341145) bank178315 +178316 POINT(37.91147839656467 -122.71652522359312) bank178316 +178317 POINT(38.417775296696185 -123.31630834606031) bank178317 +178318 POINT(36.90719115388693 -121.48944084468611) bank178318 +178319 POINT(37.27705527819964 -123.12811699043762) bank178319 +178320 POINT(37.83408824310105 -122.30216975784144) bank178320 +178321 POINT(37.02986557004045 -121.96965694159213) bank178321 +178322 POINT(38.61761948032733 -123.39897305548574) bank178322 +178323 POINT(38.50715483423093 -122.5196260506737) bank178323 +178324 POINT(38.54134676610221 -121.99939759790192) bank178324 +178325 POINT(36.97592749608605 -122.80039486106732) bank178325 +178326 POINT(38.652903096452555 -123.08798376640347) bank178326 +178327 POINT(38.14361885142645 -122.37787475590156) bank178327 +178328 POINT(37.972939247872056 -122.42592841183642) bank178328 +178329 POINT(38.04228260134873 -121.69319672048023) bank178329 +178330 POINT(37.86464399601594 -122.18883550996742) bank178330 +178331 POINT(36.86304036530821 -123.18254819008175) bank178331 +178332 POINT(38.761572026858296 -122.32642628866314) bank178332 +178333 POINT(38.69796895852933 -122.8120704246324) bank178333 +178334 POINT(37.050862658337394 -122.3495183678276) bank178334 +178335 POINT(37.61613020059557 -122.51945845517305) bank178335 +178336 POINT(37.77554753495605 -121.6958742710148) bank178336 +178337 POINT(37.54418636402557 -121.58717757702046) bank178337 +178338 POINT(38.542461179670575 -121.72576683288175) bank178338 +178339 POINT(38.44216360082099 -123.32814821102603) bank178339 +178340 POINT(36.88379191069678 -121.91394480599251) bank178340 +178341 POINT(37.62278502644818 -122.46185341117271) bank178341 +178342 POINT(37.74563704830304 -121.95177221522174) bank178342 +178343 POINT(37.211013191842305 -122.60817993621276) bank178343 +178344 POINT(36.7890940498177 -122.48993233104555) bank178344 +178345 POINT(36.97542893931854 -123.08306312339421) bank178345 +178346 POINT(37.22570376227292 -121.97283903285467) bank178346 +178347 POINT(37.08928372554866 -121.85046555678466) bank178347 +178348 POINT(38.30569013256109 -122.77623538004865) bank178348 +178349 POINT(38.65188416209263 -121.55636519137325) bank178349 +178350 POINT(38.25232962246321 -122.58813163133951) bank178350 +178351 POINT(37.579951750689844 -122.05602341900402) bank178351 +178352 POINT(37.31847588273333 -123.3314980943201) bank178352 +178353 POINT(38.0162710792005 -121.84882748840187) bank178353 +178354 POINT(37.89419977421931 -122.9381860518231) bank178354 +178355 POINT(38.77312973201071 -122.62489572015731) bank178355 +178356 POINT(37.266111872173695 -122.00107223255385) bank178356 +178357 POINT(37.4131333154678 -123.233909264654) bank178357 +178358 POINT(38.44310912305305 -121.86577316388019) bank178358 +178359 POINT(37.72736479267077 -122.85798543861927) bank178359 +178360 POINT(36.89549041166307 -122.56068500182549) bank178360 +178361 POINT(38.005909202823254 -122.76160063077181) bank178361 +178362 POINT(38.41880213811732 -121.80351559624857) bank178362 +178363 POINT(37.606633423659474 -121.78496418884708) bank178363 +178364 POINT(38.05103906881483 -122.6234547255512) bank178364 +178365 POINT(38.15734585913975 -122.01290747244921) bank178365 +178366 POINT(38.444215316309815 -121.73699154877488) bank178366 +178367 POINT(38.016927978637 -123.0240842590835) bank178367 +178368 POINT(38.62771655355138 -122.05591822120053) bank178368 +178369 POINT(36.932244421267406 -122.48685888487336) bank178369 +178370 POINT(38.59770169193824 -123.03412531081932) bank178370 +178371 POINT(38.526207785787236 -122.69985479660059) bank178371 +178372 POINT(38.451742598892885 -123.3778742242067) bank178372 +178373 POINT(37.06124991179137 -122.19875402697976) bank178373 +178374 POINT(38.3096603238801 -122.32563327782282) bank178374 +178375 POINT(37.40635245793991 -121.87470652101302) bank178375 +178376 POINT(37.69346876551838 -122.5545353333795) bank178376 +178377 POINT(38.11439000278111 -121.95111029490299) bank178377 +178378 POINT(37.04139648461346 -122.93329204207443) bank178378 +178379 POINT(37.694845902189364 -123.11948265239253) bank178379 +178380 POINT(38.057511729392644 -121.55158379537826) bank178380 +178381 POINT(37.868544380029036 -122.8906002063369) bank178381 +178382 POINT(37.21222278765486 -121.67248421634879) bank178382 +178383 POINT(36.95188907458331 -122.67784117706158) bank178383 +178384 POINT(37.51486868665775 -122.75252224161149) bank178384 +178385 POINT(37.25448712400304 -122.14919227642933) bank178385 +178386 POINT(38.638736213901986 -122.66078590554122) bank178386 +178387 POINT(37.308836646547974 -122.27140090136344) bank178387 +178388 POINT(38.067124858482586 -122.2393917715482) bank178388 +178389 POINT(38.528143546699475 -121.9025015399704) bank178389 +178390 POINT(38.58616540542452 -122.95029950372208) bank178390 +178391 POINT(37.129732259403795 -122.99929527230927) bank178391 +178392 POINT(37.56668698299967 -122.59451082675703) bank178392 +178393 POINT(38.055388608226416 -122.11673277068314) bank178393 +178394 POINT(37.43531504138298 -122.40376722516504) bank178394 +178395 POINT(38.099816066837064 -121.82593119545919) bank178395 +178396 POINT(37.949882324636064 -123.38399693343399) bank178396 +178397 POINT(37.70305883225663 -122.56779256092794) bank178397 +178398 POINT(38.07329514404913 -123.23626440229944) bank178398 +178399 POINT(38.47798598607629 -122.14806666021884) bank178399 +178400 POINT(38.24026862695129 -122.33779417121931) bank178400 +178401 POINT(38.21146815479157 -123.08377806785056) bank178401 +178402 POINT(36.909712712861115 -121.5265855083053) bank178402 +178403 POINT(37.12193975564736 -122.61712730423443) bank178403 +178404 POINT(38.18356545756291 -122.4825746624128) bank178404 +178405 POINT(38.51380279792708 -122.37783328467164) bank178405 +178406 POINT(37.30219305049068 -122.44258286731015) bank178406 +178407 POINT(37.15616715165185 -123.11456825332752) bank178407 +178408 POINT(37.89473668832473 -123.10767030808469) bank178408 +178409 POINT(38.305585974036255 -122.5417715334992) bank178409 +178410 POINT(36.92644305309697 -123.01210616536729) bank178410 +178411 POINT(37.194651275798655 -122.97718145572313) bank178411 +178412 POINT(37.93952907082311 -122.80318398006432) bank178412 +178413 POINT(36.785243117842164 -122.36043876389043) bank178413 +178414 POINT(37.35580116208045 -122.45454487146249) bank178414 +178415 POINT(38.310902351956734 -121.98727995716403) bank178415 +178416 POINT(38.34927294518758 -122.2868400007341) bank178416 +178417 POINT(38.754867972540175 -123.36432647383464) bank178417 +178418 POINT(38.62377434390062 -122.27584105263647) bank178418 +178419 POINT(38.10220275201682 -122.87971840769596) bank178419 +178420 POINT(38.33381031140661 -121.56504820910388) bank178420 +178421 POINT(37.24625878880867 -122.94523468638711) bank178421 +178422 POINT(38.31761659529652 -123.00940954329326) bank178422 +178423 POINT(38.36125503131836 -123.11139529792035) bank178423 +178424 POINT(37.197499156309355 -123.40467265658918) bank178424 +178425 POINT(37.68081186163216 -123.03886167572739) bank178425 +178426 POINT(38.12161866794615 -123.21581944857219) bank178426 +178427 POINT(37.49254070122964 -122.73143549034303) bank178427 +178428 POINT(37.448457931994845 -121.98686931533227) bank178428 +178429 POINT(37.63461557775313 -123.33586804514594) bank178429 +178430 POINT(36.92506248925017 -122.23850112475218) bank178430 +178431 POINT(37.152217911715276 -123.36982459854359) bank178431 +178432 POINT(38.64442649270435 -123.28917370100329) bank178432 +178433 POINT(38.76821922915776 -122.0606719009844) bank178433 +178434 POINT(38.583262724699765 -122.65745315074952) bank178434 +178435 POINT(37.324861694212274 -121.87972937625915) bank178435 +178436 POINT(38.04911488055007 -121.65540503407048) bank178436 +178437 POINT(38.40208914275286 -122.46781738412005) bank178437 +178438 POINT(38.15867562993123 -122.1793041276097) bank178438 +178439 POINT(38.62737352457212 -122.53295679069156) bank178439 +178440 POINT(37.7232311297911 -122.99975018254753) bank178440 +178441 POINT(38.48176376497303 -121.8547216825879) bank178441 +178442 POINT(38.57341496728488 -123.18392753520139) bank178442 +178443 POINT(37.57990457240465 -121.79889171629902) bank178443 +178444 POINT(37.5673635863314 -121.6872912188521) bank178444 +178445 POINT(37.86222426079938 -123.18290628060178) bank178445 +178446 POINT(38.641118894444595 -123.13428786566318) bank178446 +178447 POINT(37.4960208086357 -121.8602743801788) bank178447 +178448 POINT(38.357818252662675 -121.97560212635173) bank178448 +178449 POINT(36.91062963178005 -123.1425681385435) bank178449 +178450 POINT(38.73389324595836 -123.22268451334521) bank178450 +178451 POINT(37.16900867329709 -123.12155053932369) bank178451 +178452 POINT(37.66894692100203 -122.85994874712077) bank178452 +178453 POINT(37.49641935543623 -123.33160886108345) bank178453 +178454 POINT(38.22018692797514 -123.02095516320946) bank178454 +178455 POINT(37.52585826965785 -121.73842934500891) bank178455 +178456 POINT(38.548797584371584 -121.6410955459465) bank178456 +178457 POINT(37.0586724373822 -122.78666203801085) bank178457 +178458 POINT(37.27454687848918 -122.14249976081672) bank178458 +178459 POINT(38.66047688412853 -121.60502520988294) bank178459 +178460 POINT(37.69440830483994 -122.43211190195258) bank178460 +178461 POINT(38.70844479162849 -122.3050380684033) bank178461 +178462 POINT(37.15111426997256 -122.17993499107196) bank178462 +178463 POINT(37.11446464117297 -121.91573033365601) bank178463 +178464 POINT(37.00520892574188 -122.94338837596587) bank178464 +178465 POINT(37.521984673938576 -121.7110890381903) bank178465 +178466 POINT(37.11164439146495 -121.8096796807511) bank178466 +178467 POINT(38.58106799166475 -122.43165239442817) bank178467 +178468 POINT(37.49341099313938 -122.65195045468751) bank178468 +178469 POINT(37.734363511052536 -122.41994137953519) bank178469 +178470 POINT(37.97151400019373 -123.41108077147244) bank178470 +178471 POINT(37.06310138122843 -122.69595666928102) bank178471 +178472 POINT(37.377840886509205 -122.44463778048456) bank178472 +178473 POINT(37.167978713648594 -121.81947733677072) bank178473 +178474 POINT(37.865189361259525 -121.7442162920902) bank178474 +178475 POINT(37.492487109120816 -122.75567928594772) bank178475 +178476 POINT(38.24133562249925 -121.44146927729159) bank178476 +178477 POINT(38.57023101991369 -122.77815853408356) bank178477 +178478 POINT(38.71457694094788 -122.43330789318824) bank178478 +178479 POINT(37.5289323229085 -123.05769199464345) bank178479 +178480 POINT(38.68115227309743 -123.05421591228946) bank178480 +178481 POINT(37.197766332730446 -122.32667651051008) bank178481 +178482 POINT(37.22311799409163 -122.63180939568187) bank178482 +178483 POINT(38.552317909221166 -122.48847747864866) bank178483 +178484 POINT(38.22852279053669 -123.40678096131467) bank178484 +178485 POINT(37.46236583532113 -122.91306125135819) bank178485 +178486 POINT(37.65357752382371 -122.77103609402107) bank178486 +178487 POINT(37.885532253084904 -123.3587638048941) bank178487 +178488 POINT(36.89775711597314 -123.2518037322825) bank178488 +178489 POINT(38.00166819867026 -121.43138115166344) bank178489 +178490 POINT(38.21125504859192 -122.97282755322207) bank178490 +178491 POINT(38.696396272628206 -122.81382365986178) bank178491 +178492 POINT(37.8238128146707 -122.39578277338052) bank178492 +178493 POINT(37.61624291013556 -122.70094496638362) bank178493 +178494 POINT(38.43639523893411 -122.28456458423705) bank178494 +178495 POINT(38.47973021658152 -123.04348046181215) bank178495 +178496 POINT(38.67946602869952 -122.01148592496004) bank178496 +178497 POINT(38.50727764978361 -121.66477277215056) bank178497 +178498 POINT(36.89679364640565 -121.67672497839595) bank178498 +178499 POINT(38.50087928550336 -121.83797107043762) bank178499 +178500 POINT(38.12731214977562 -121.65569105571399) bank178500 +178501 POINT(37.4664899564638 -123.2496733451791) bank178501 +178502 POINT(38.411027695877806 -123.25913383989536) bank178502 +178503 POINT(36.94525309169435 -122.95008569781704) bank178503 +178504 POINT(37.41691384924711 -121.50052464704109) bank178504 +178505 POINT(37.67807790094396 -122.10883413053526) bank178505 +178506 POINT(38.203551029430855 -122.98409829926521) bank178506 +178507 POINT(37.24088969756078 -121.43490370781471) bank178507 +178508 POINT(36.81848458773046 -121.880795141309) bank178508 +178509 POINT(37.246504045683366 -123.06125115562484) bank178509 +178510 POINT(37.30272762079314 -121.56145175644247) bank178510 +178511 POINT(36.81699980525514 -123.03398612785362) bank178511 +178512 POINT(37.53222252439341 -121.41941817663732) bank178512 +178513 POINT(38.68691004733521 -122.89508453509733) bank178513 +178514 POINT(38.49185452566846 -122.61584022816369) bank178514 +178515 POINT(37.153464158339006 -122.25271543097514) bank178515 +178516 POINT(38.632424018321466 -122.09446406232323) bank178516 +178517 POINT(38.3733678494829 -122.69046292401461) bank178517 +178518 POINT(37.99465934958454 -121.60257833171438) bank178518 +178519 POINT(37.25741895843596 -121.99370244900149) bank178519 +178520 POINT(38.59509502411225 -122.34692274922308) bank178520 +178521 POINT(37.632580878125054 -122.85922719728948) bank178521 +178522 POINT(37.39069048670095 -121.90175142799765) bank178522 +178523 POINT(38.568541035459035 -122.82420676721976) bank178523 +178524 POINT(37.75227674655068 -121.54395660901136) bank178524 +178525 POINT(36.89827702192376 -122.53680290367006) bank178525 +178526 POINT(36.9257599410292 -121.512709020411) bank178526 +178527 POINT(36.899263122823804 -122.89047124743857) bank178527 +178528 POINT(37.78104615870369 -121.92981655826257) bank178528 +178529 POINT(37.33512576980413 -122.27351756466275) bank178529 +178530 POINT(37.95381112953419 -122.55778994447786) bank178530 +178531 POINT(37.993724805157136 -122.45449170066084) bank178531 +178532 POINT(38.22655814606463 -122.49063096093661) bank178532 +178533 POINT(37.65529532846489 -121.85218828684486) bank178533 +178534 POINT(38.30788001274351 -121.61428797896173) bank178534 +178535 POINT(37.09985854224733 -122.36017636332204) bank178535 +178536 POINT(38.43587697868414 -122.13546818661557) bank178536 +178537 POINT(37.849437025904294 -123.14707907193375) bank178537 +178538 POINT(37.39560093268818 -123.38963788466182) bank178538 +178539 POINT(38.75606762574179 -122.88446285756275) bank178539 +178540 POINT(36.89113592963206 -121.538565803119) bank178540 +178541 POINT(37.09641121144772 -122.13140779852483) bank178541 +178542 POINT(38.52410670468769 -121.84007437643335) bank178542 +178543 POINT(37.72481334641699 -123.12905571423379) bank178543 +178544 POINT(38.223709097553716 -121.62156252764615) bank178544 +178545 POINT(38.1385464914364 -121.75451386618273) bank178545 +178546 POINT(38.16388126997182 -122.9524006961727) bank178546 +178547 POINT(37.515524532034114 -122.379021515086) bank178547 +178548 POINT(38.66880431598008 -121.56621075821127) bank178548 +178549 POINT(37.062526394110165 -122.60718810453035) bank178549 +178550 POINT(38.44263356568724 -122.95760454882566) bank178550 +178551 POINT(38.26142080109658 -122.78364570668796) bank178551 +178552 POINT(37.318435722033605 -122.10329596993583) bank178552 +178553 POINT(37.992536400035306 -122.50126523471538) bank178553 +178554 POINT(38.72439091549606 -121.91618355923008) bank178554 +178555 POINT(37.527347818360674 -122.21017158848133) bank178555 +178556 POINT(38.314492365558316 -122.7939048704371) bank178556 +178557 POINT(37.790712086030545 -122.69007150427177) bank178557 +178558 POINT(38.058281440322446 -121.74870997859874) bank178558 +178559 POINT(38.57707207009471 -123.35944569289178) bank178559 +178560 POINT(37.23702878309984 -122.64936553882474) bank178560 +178561 POINT(37.98052699648808 -121.54311868840244) bank178561 +178562 POINT(37.50826299732135 -121.8128301022631) bank178562 +178563 POINT(37.22730431929215 -123.28546222291651) bank178563 +178564 POINT(37.06150209973985 -122.35769866562218) bank178564 +178565 POINT(38.64049907338769 -122.50317254385395) bank178565 +178566 POINT(37.03229329206848 -122.36868437535975) bank178566 +178567 POINT(38.40821997842457 -121.83454807534159) bank178567 +178568 POINT(36.922776211481555 -122.95630300781671) bank178568 +178569 POINT(38.054395774048196 -122.9470451416109) bank178569 +178570 POINT(36.9999550374971 -122.8290402292723) bank178570 +178571 POINT(37.07339625947008 -123.29351025869938) bank178571 +178572 POINT(38.16189870633321 -122.2183566777193) bank178572 +178573 POINT(37.27259856963247 -121.96390962407308) bank178573 +178574 POINT(38.47436778732501 -121.75514532586348) bank178574 +178575 POINT(37.21393258649751 -123.10383672787931) bank178575 +178576 POINT(38.07330784357193 -123.12989378667709) bank178576 +178577 POINT(36.895441512279774 -121.51764778250438) bank178577 +178578 POINT(38.73904782173381 -122.82816740006686) bank178578 +178579 POINT(36.78154781225043 -122.72679343048624) bank178579 +178580 POINT(37.550484972247645 -122.50871703117934) bank178580 +178581 POINT(37.24823420716522 -121.72536543576366) bank178581 +178582 POINT(36.78199341291762 -121.42361666251684) bank178582 +178583 POINT(38.16569332355846 -122.54508926352145) bank178583 +178584 POINT(38.22699030479211 -121.69274784994182) bank178584 +178585 POINT(38.30668669884372 -122.48627793316544) bank178585 +178586 POINT(38.47052662176753 -123.30293432012087) bank178586 +178587 POINT(36.80043566875498 -122.0715851026308) bank178587 +178588 POINT(38.35029562956754 -122.98768917600509) bank178588 +178589 POINT(38.53646373583596 -121.62542545475792) bank178589 +178590 POINT(37.7306688347428 -121.72838151075575) bank178590 +178591 POINT(36.96776801530005 -122.1021873525528) bank178591 +178592 POINT(36.92946771949998 -122.78474416770392) bank178592 +178593 POINT(38.56172222719497 -121.99431328212906) bank178593 +178594 POINT(38.06240706523617 -122.3576786258062) bank178594 +178595 POINT(37.975503378251744 -123.18976928003971) bank178595 +178596 POINT(37.233861523188594 -123.09477714240869) bank178596 +178597 POINT(38.308790222189764 -121.82209731062511) bank178597 +178598 POINT(38.058784659836185 -123.39776652836645) bank178598 +178599 POINT(37.59466145462221 -122.68182935143892) bank178599 +178600 POINT(37.50590857936581 -121.87740861214434) bank178600 +178601 POINT(37.21652682380172 -122.86526357426685) bank178601 +178602 POINT(36.824600875355635 -121.90079120848306) bank178602 +178603 POINT(38.01515434261608 -121.615889575418) bank178603 +178604 POINT(37.05030463059358 -123.19985309248513) bank178604 +178605 POINT(37.217221516869614 -122.83648493910259) bank178605 +178606 POINT(37.18247981491403 -121.6579353058121) bank178606 +178607 POINT(38.518690016422894 -123.07522764418765) bank178607 +178608 POINT(36.83710609925287 -122.27666167726761) bank178608 +178609 POINT(38.0933952933632 -123.36020297670997) bank178609 +178610 POINT(36.924949074815785 -122.61909103104506) bank178610 +178611 POINT(37.60199253791274 -121.7323439551019) bank178611 +178612 POINT(37.099366575190764 -121.97000819343067) bank178612 +178613 POINT(36.83006475821196 -122.4231075230657) bank178613 +178614 POINT(37.72207918184812 -122.31874288854353) bank178614 +178615 POINT(37.114780895811904 -121.69258342360008) bank178615 +178616 POINT(38.14772307225366 -122.85055855883618) bank178616 +178617 POINT(37.0544486870828 -122.3318940173362) bank178617 +178618 POINT(38.55672943795087 -122.24705636977014) bank178618 +178619 POINT(37.453363062975555 -123.01751172916765) bank178619 +178620 POINT(38.13959395851863 -122.33633514778512) bank178620 +178621 POINT(38.04828226449355 -121.47365636605538) bank178621 +178622 POINT(37.389691439095344 -121.98313639351268) bank178622 +178623 POINT(38.409371028272304 -122.88935212665932) bank178623 +178624 POINT(37.02677433747197 -122.62823416685387) bank178624 +178625 POINT(37.77622750857334 -122.07736671639971) bank178625 +178626 POINT(38.522569162397964 -122.16464806986502) bank178626 +178627 POINT(38.74463066815335 -123.19739889878295) bank178627 +178628 POINT(38.42486466577508 -121.6395143530787) bank178628 +178629 POINT(37.604310550305186 -123.0961478632435) bank178629 +178630 POINT(37.1110389590753 -122.64667915679524) bank178630 +178631 POINT(37.69420252644982 -121.70919307115456) bank178631 +178632 POINT(38.56190257142852 -121.58589981216667) bank178632 +178633 POINT(37.88688689371777 -122.07759636296052) bank178633 +178634 POINT(38.61203922535366 -122.45254170563868) bank178634 +178635 POINT(38.43157642941593 -121.72278528813084) bank178635 +178636 POINT(37.6938086059489 -122.10699798889901) bank178636 +178637 POINT(36.98402723632072 -122.26217806304548) bank178637 +178638 POINT(37.57553144367279 -121.7263026882976) bank178638 +178639 POINT(38.38276997384834 -122.14958451970213) bank178639 +178640 POINT(37.14040959006337 -123.16743559420037) bank178640 +178641 POINT(38.09661647169609 -122.45955518004386) bank178641 +178642 POINT(37.201620500658905 -121.99100143959672) bank178642 +178643 POINT(38.15788895468096 -123.33242839395336) bank178643 +178644 POINT(37.62889003852242 -123.17270848830438) bank178644 +178645 POINT(38.32935201531496 -122.83194712129256) bank178645 +178646 POINT(38.227023887539275 -121.56437668557751) bank178646 +178647 POINT(38.18112478369616 -122.12396273970039) bank178647 +178648 POINT(37.906757295893705 -122.13342148158296) bank178648 +178649 POINT(38.08168961704166 -122.62516996051495) bank178649 +178650 POINT(38.123640136868744 -121.64328294341882) bank178650 +178651 POINT(38.341584050416806 -121.67862206492632) bank178651 +178652 POINT(38.515049762758366 -122.73192112738325) bank178652 +178653 POINT(38.05465480827245 -122.66045695584509) bank178653 +178654 POINT(38.673807486005906 -121.65852022830877) bank178654 +178655 POINT(37.14009737294989 -122.88667327821717) bank178655 +178656 POINT(37.25138671513324 -122.74635192348482) bank178656 +178657 POINT(38.32274351866633 -122.52980415293945) bank178657 +178658 POINT(37.68536500088074 -121.800332602034) bank178658 +178659 POINT(37.12651214946835 -121.93938095533404) bank178659 +178660 POINT(37.64816559170505 -121.8366599279446) bank178660 +178661 POINT(38.6634638193977 -123.26634602161198) bank178661 +178662 POINT(37.17887345117705 -123.0244711223873) bank178662 +178663 POINT(37.46498875167505 -122.66385062105154) bank178663 +178664 POINT(38.69104443307562 -121.77431619214016) bank178664 +178665 POINT(38.57190178113687 -123.2172494818358) bank178665 +178666 POINT(38.60522310220789 -122.27508271114162) bank178666 +178667 POINT(38.22731144573824 -121.7914103217357) bank178667 +178668 POINT(36.81943071047327 -121.47690513584027) bank178668 +178669 POINT(38.01255986738391 -122.42616521409806) bank178669 +178670 POINT(37.39156858693825 -122.88925355127631) bank178670 +178671 POINT(38.54960497233239 -121.70391712993906) bank178671 +178672 POINT(37.288726382242025 -122.80842679625275) bank178672 +178673 POINT(38.02074391900026 -121.60806971684542) bank178673 +178674 POINT(37.02016185444611 -121.66170202996769) bank178674 +178675 POINT(37.706747479860134 -122.18771033991588) bank178675 +178676 POINT(38.490548966021684 -122.15755181606137) bank178676 +178677 POINT(36.96460345177074 -121.99511865547288) bank178677 +178678 POINT(37.87250497690541 -122.6983486805674) bank178678 +178679 POINT(38.772999852914594 -122.0178892325465) bank178679 +178680 POINT(38.650271976227 -121.43848501558527) bank178680 +178681 POINT(37.60210604819856 -121.87772590013427) bank178681 +178682 POINT(37.174562502112444 -122.63959804365165) bank178682 +178683 POINT(38.14752555548959 -121.45238715047141) bank178683 +178684 POINT(38.642030495162246 -122.38696413021582) bank178684 +178685 POINT(37.20164652937116 -122.51562064008289) bank178685 +178686 POINT(36.788894478867476 -121.4205619720905) bank178686 +178687 POINT(37.552741123566264 -121.65056086089345) bank178687 +178688 POINT(37.21571213631216 -122.61084720737755) bank178688 +178689 POINT(37.85256366070523 -123.37122339443255) bank178689 +178690 POINT(38.39322614496691 -123.02459835691724) bank178690 +178691 POINT(37.10640832309285 -121.97840250255376) bank178691 +178692 POINT(38.290595777146216 -121.94556012221861) bank178692 +178693 POINT(37.78916124714549 -122.65216467976839) bank178693 +178694 POINT(37.49315918568322 -122.8889432441108) bank178694 +178695 POINT(37.61759614369515 -123.06914484890473) bank178695 +178696 POINT(37.68741022777303 -122.68484408385396) bank178696 +178697 POINT(37.41265608138147 -122.21696662190513) bank178697 +178698 POINT(38.57476408142329 -121.63625508604453) bank178698 +178699 POINT(38.11192192837238 -122.43211022772017) bank178699 +178700 POINT(37.507548935041434 -122.13163804934503) bank178700 +178701 POINT(36.95626319985005 -122.09938674416667) bank178701 +178702 POINT(37.928753385584216 -122.9315050251605) bank178702 +178703 POINT(37.13139026719242 -123.25750559690134) bank178703 +178704 POINT(38.008442910274596 -121.52584508588367) bank178704 +178705 POINT(37.197489501349125 -122.3306359478374) bank178705 +178706 POINT(38.2120350521258 -122.04322509143391) bank178706 +178707 POINT(37.62026370987341 -122.16131420880808) bank178707 +178708 POINT(37.25911494986273 -122.89283882763812) bank178708 +178709 POINT(38.7719074451036 -122.87784576723402) bank178709 +178710 POINT(38.54115545071852 -122.25366784898324) bank178710 +178711 POINT(37.68307281775892 -123.36644640106219) bank178711 +178712 POINT(38.13379690459095 -122.30720000792057) bank178712 +178713 POINT(37.49333300625019 -122.29857571139898) bank178713 +178714 POINT(38.28514630027334 -121.87613743035782) bank178714 +178715 POINT(37.44631304292724 -122.01867479123909) bank178715 +178716 POINT(36.8224322624725 -122.9280719836558) bank178716 +178717 POINT(37.40866082238238 -122.87213842889179) bank178717 +178718 POINT(37.596258221760486 -122.93383042090394) bank178718 +178719 POINT(38.16043360198354 -121.9558905540812) bank178719 +178720 POINT(38.24925109537074 -121.64663738264265) bank178720 +178721 POINT(38.189120867034404 -122.37002770494117) bank178721 +178722 POINT(37.390290342645876 -122.80950336367113) bank178722 +178723 POINT(37.65212634784732 -122.3631016134052) bank178723 +178724 POINT(37.25755925885498 -121.76794430489562) bank178724 +178725 POINT(38.09351620541336 -123.35373478231973) bank178725 +178726 POINT(36.946229066827875 -121.65254792870815) bank178726 +178727 POINT(37.19596256722555 -122.79832324112883) bank178727 +178728 POINT(37.5303937310062 -123.01493289215541) bank178728 +178729 POINT(37.356161879215726 -123.17598459409042) bank178729 +178730 POINT(37.66241586917059 -122.76062186180425) bank178730 +178731 POINT(37.00231092102362 -123.40709286484577) bank178731 +178732 POINT(37.59742978000979 -122.64568210111597) bank178732 +178733 POINT(37.87345544044764 -122.11475092170792) bank178733 +178734 POINT(38.33656821146167 -121.87590323922316) bank178734 +178735 POINT(36.80531537483057 -122.46322195883897) bank178735 +178736 POINT(37.96459794650444 -121.96140309678799) bank178736 +178737 POINT(37.10377403718021 -123.20458694088177) bank178737 +178738 POINT(38.44526334346629 -122.97527337617502) bank178738 +178739 POINT(38.43883772691993 -123.16976846952475) bank178739 +178740 POINT(38.77286211608695 -121.5933273610377) bank178740 +178741 POINT(36.877874508716545 -122.0969765391883) bank178741 +178742 POINT(36.9504545861964 -121.62318725319142) bank178742 +178743 POINT(37.167259047030356 -122.69778954622366) bank178743 +178744 POINT(37.30268234155352 -123.00744667088459) bank178744 +178745 POINT(38.000025030428425 -122.24998105033171) bank178745 +178746 POINT(38.17719687048454 -122.62409898215111) bank178746 +178747 POINT(37.37684170325048 -121.97345511245558) bank178747 +178748 POINT(37.10782856240966 -122.62731083944418) bank178748 +178749 POINT(37.24737406108399 -122.9598142002413) bank178749 +178750 POINT(37.16040348704864 -121.4812827549804) bank178750 +178751 POINT(38.0245938882539 -123.16963230186026) bank178751 +178752 POINT(37.90673428304574 -122.43686472214398) bank178752 +178753 POINT(37.34129561977606 -121.73534569663272) bank178753 +178754 POINT(37.159900697302035 -121.51184422168227) bank178754 +178755 POINT(38.48068559488502 -122.35313907240837) bank178755 +178756 POINT(37.11522516735975 -121.48377435411177) bank178756 +178757 POINT(37.0913654793615 -122.9045053552792) bank178757 +178758 POINT(37.53713502575221 -123.20602816502587) bank178758 +178759 POINT(36.88418004209408 -121.65058163303524) bank178759 +178760 POINT(36.77833507700301 -122.93386852649839) bank178760 +178761 POINT(37.60945797074248 -122.46608180497846) bank178761 +178762 POINT(38.04135075897875 -121.8642013857614) bank178762 +178763 POINT(37.9948944884997 -122.93724875241863) bank178763 +178764 POINT(37.33858688009002 -122.1549144014425) bank178764 +178765 POINT(38.16892716268162 -121.67883806594762) bank178765 +178766 POINT(38.68218449550409 -122.7494891723259) bank178766 +178767 POINT(36.93817207933065 -123.26439880008796) bank178767 +178768 POINT(38.19351845323263 -122.99882234487028) bank178768 +178769 POINT(38.19726715444892 -123.23504289670181) bank178769 +178770 POINT(37.81968660252405 -122.30453735387431) bank178770 +178771 POINT(37.77510473168553 -121.94541676674221) bank178771 +178772 POINT(38.43261686758414 -122.49063645790753) bank178772 +178773 POINT(37.007953984529266 -122.51768602488981) bank178773 +178774 POINT(38.118225339309795 -122.8877924791509) bank178774 +178775 POINT(36.80274913836376 -122.50084261482432) bank178775 +178776 POINT(37.430148287908686 -122.21652079506076) bank178776 +178777 POINT(38.432615000159224 -123.21798815377728) bank178777 +178778 POINT(37.41146554604451 -123.16064980147706) bank178778 +178779 POINT(36.953960535816684 -123.12485337158506) bank178779 +178780 POINT(38.47892812334845 -122.02243630172138) bank178780 +178781 POINT(38.253445152967565 -121.78197371675046) bank178781 +178782 POINT(37.301812769483426 -123.33250234394436) bank178782 +178783 POINT(38.59163700958932 -122.20917749735912) bank178783 +178784 POINT(37.93631941075969 -122.22967039468458) bank178784 +178785 POINT(38.74227515579543 -121.75794530029708) bank178785 +178786 POINT(38.263682738776666 -122.97827383991724) bank178786 +178787 POINT(37.56198325723199 -121.81697486444499) bank178787 +178788 POINT(38.11055013349118 -122.21912843805579) bank178788 +178789 POINT(38.209356336061916 -123.07760119045244) bank178789 +178790 POINT(38.73729231438016 -122.98270192425059) bank178790 +178791 POINT(37.48293580299654 -121.58410853628824) bank178791 +178792 POINT(38.58196108548129 -123.01792492686323) bank178792 +178793 POINT(37.67479026910791 -121.88370364721031) bank178793 +178794 POINT(37.94125620319312 -122.07092685826306) bank178794 +178795 POINT(37.059398651889474 -122.79830202628659) bank178795 +178796 POINT(37.01438623395403 -122.97742436212349) bank178796 +178797 POINT(37.34150003764092 -122.93965119342354) bank178797 +178798 POINT(38.27463196520771 -123.20245112727649) bank178798 +178799 POINT(38.678458138075506 -121.97638972644931) bank178799 +178800 POINT(37.736667265766194 -122.89536865588826) bank178800 +178801 POINT(38.583196702703596 -122.38912147230157) bank178801 +178802 POINT(36.994412538548765 -122.21103977370124) bank178802 +178803 POINT(36.84982841488189 -123.33912278369608) bank178803 +178804 POINT(38.375242539392865 -122.5663417471783) bank178804 +178805 POINT(37.74069607722905 -122.11151605638918) bank178805 +178806 POINT(37.071406660849725 -122.60481667723812) bank178806 +178807 POINT(37.641835802508076 -122.7154040379075) bank178807 +178808 POINT(37.971612357259616 -122.46560295525295) bank178808 +178809 POINT(38.344349850297746 -121.89093299434417) bank178809 +178810 POINT(37.97132434177934 -122.97719946536427) bank178810 +178811 POINT(37.277361681407136 -123.20927397231584) bank178811 +178812 POINT(38.34673426205749 -122.61102997092972) bank178812 +178813 POINT(36.94788618519758 -121.51926362900645) bank178813 +178814 POINT(38.7119379880431 -122.83884233107425) bank178814 +178815 POINT(38.44933042386942 -122.07058939510787) bank178815 +178816 POINT(38.558072736416406 -122.91114831945512) bank178816 +178817 POINT(38.54854166491 -122.39070017996514) bank178817 +178818 POINT(38.648380528511304 -123.19119418209866) bank178818 +178819 POINT(37.17430068614671 -122.7802127578506) bank178819 +178820 POINT(37.26013850854046 -122.97976122504221) bank178820 +178821 POINT(37.8934568588687 -122.5456909969416) bank178821 +178822 POINT(37.965580610168935 -121.45656912931632) bank178822 +178823 POINT(37.783760451904655 -122.58784358516229) bank178823 +178824 POINT(38.39579179325386 -122.34910926387522) bank178824 +178825 POINT(37.81107521815756 -122.0898080634382) bank178825 +178826 POINT(38.09524655269441 -121.73232412669209) bank178826 +178827 POINT(38.082741960420954 -121.916597505221) bank178827 +178828 POINT(38.33980449742635 -122.25940309364579) bank178828 +178829 POINT(38.07675897079207 -122.91775693463366) bank178829 +178830 POINT(37.92260518642283 -123.35152601261605) bank178830 +178831 POINT(37.030604214560576 -123.18427627678999) bank178831 +178832 POINT(36.85409586200842 -122.54023166464964) bank178832 +178833 POINT(37.644600364751795 -121.98121325358056) bank178833 +178834 POINT(37.81509005253872 -123.34198362161972) bank178834 +178835 POINT(37.44835347975125 -121.5965214661736) bank178835 +178836 POINT(37.20720399012606 -123.35428791207664) bank178836 +178837 POINT(37.785678878826474 -123.04693108954828) bank178837 +178838 POINT(37.1392507576222 -121.92603770582768) bank178838 +178839 POINT(38.193362886980815 -122.80162325234048) bank178839 +178840 POINT(37.09271733633831 -122.04423035912777) bank178840 +178841 POINT(37.479786252299625 -121.67501261709421) bank178841 +178842 POINT(38.452533486930754 -123.17676144124462) bank178842 +178843 POINT(37.20202790654247 -122.22919607526508) bank178843 +178844 POINT(37.591890402658 -122.81779261213433) bank178844 +178845 POINT(38.09626453222902 -122.42973743134617) bank178845 +178846 POINT(37.07443166180863 -122.85251188856968) bank178846 +178847 POINT(37.81087529700542 -122.746365916818) bank178847 +178848 POINT(37.90877736405502 -121.48565296775662) bank178848 +178849 POINT(37.50787420199284 -123.00462378009257) bank178849 +178850 POINT(37.752066683571314 -122.83051034127269) bank178850 +178851 POINT(37.04763816835906 -123.41232326785878) bank178851 +178852 POINT(38.26907148138509 -121.90053267710306) bank178852 +178853 POINT(38.154667410611275 -123.19607848839287) bank178853 +178854 POINT(37.61355734683068 -121.96733932479215) bank178854 +178855 POINT(37.85863115737263 -123.3838722776639) bank178855 +178856 POINT(38.35544870068746 -121.64386849628391) bank178856 +178857 POINT(37.49150043712696 -122.13847839815239) bank178857 +178858 POINT(38.31127617244466 -122.49432220223858) bank178858 +178859 POINT(38.3549039767427 -122.10777830026174) bank178859 +178860 POINT(37.34521930630682 -122.54565879001017) bank178860 +178861 POINT(37.00682238867243 -122.83546339928726) bank178861 +178862 POINT(37.13438707437025 -122.47952366399684) bank178862 +178863 POINT(37.81346119075731 -122.09506710413832) bank178863 +178864 POINT(38.36411697410388 -121.42757748016349) bank178864 +178865 POINT(37.16443088839836 -122.45285795007567) bank178865 +178866 POINT(37.38733824086528 -121.44474986684418) bank178866 +178867 POINT(38.729550412761355 -121.70673708529247) bank178867 +178868 POINT(38.62322975213017 -122.39887016050484) bank178868 +178869 POINT(38.2260114623617 -122.01390936616271) bank178869 +178870 POINT(37.88477381873975 -121.71869364961032) bank178870 +178871 POINT(37.119418559495145 -123.24723639709615) bank178871 +178872 POINT(37.45092071867753 -121.90256460253) bank178872 +178873 POINT(37.90350905377277 -122.408679932614) bank178873 +178874 POINT(38.29884196045848 -122.11778164518509) bank178874 +178875 POINT(37.45056410370015 -122.36842712740177) bank178875 +178876 POINT(37.34717897039339 -122.64975980033107) bank178876 +178877 POINT(36.98584565678606 -122.25886833757363) bank178877 +178878 POINT(37.962151396585476 -122.1261154822655) bank178878 +178879 POINT(38.04396334583564 -123.239401872535) bank178879 +178880 POINT(38.0244339302259 -123.11516425382631) bank178880 +178881 POINT(37.493164787404446 -121.50863609563017) bank178881 +178882 POINT(37.26620512922823 -122.37334906464162) bank178882 +178883 POINT(38.10207108483018 -121.97802841833602) bank178883 +178884 POINT(38.352848095873085 -122.09098331306443) bank178884 +178885 POINT(37.54173032366092 -121.46692979466582) bank178885 +178886 POINT(37.455769054288055 -122.76140434030002) bank178886 +178887 POINT(37.749642730921906 -122.86181380647349) bank178887 +178888 POINT(37.58838786517195 -123.33981390196624) bank178888 +178889 POINT(37.81913865252825 -123.23750918958686) bank178889 +178890 POINT(37.942334863709526 -122.97788379535734) bank178890 +178891 POINT(38.35619050109959 -121.81042026729116) bank178891 +178892 POINT(38.774727708105374 -123.24893095288651) bank178892 +178893 POINT(37.11281175876762 -123.12417622649902) bank178893 +178894 POINT(37.39730333754896 -122.01987777658341) bank178894 +178895 POINT(37.64098937598082 -123.31734411607641) bank178895 +178896 POINT(37.79178976545383 -123.21603837019993) bank178896 +178897 POINT(37.38709944445156 -122.74722615198824) bank178897 +178898 POINT(37.71131310748983 -122.37107465422497) bank178898 +178899 POINT(38.10047207466787 -122.31617313386316) bank178899 +178900 POINT(38.678837993121306 -121.92269914404041) bank178900 +178901 POINT(36.9327828394232 -121.47382415206758) bank178901 +178902 POINT(37.90469810119145 -121.78184554333264) bank178902 +178903 POINT(38.036007250861495 -122.95717383266746) bank178903 +178904 POINT(36.87429297633752 -122.12699888681801) bank178904 +178905 POINT(38.66277399190924 -122.47495751031768) bank178905 +178906 POINT(36.84667824950304 -122.32776447780057) bank178906 +178907 POINT(36.862192088571184 -121.43298905199258) bank178907 +178908 POINT(37.640245003411955 -123.14662271662976) bank178908 +178909 POINT(37.82942816473567 -121.89375464340485) bank178909 +178910 POINT(37.892925136462964 -122.53810947003159) bank178910 +178911 POINT(38.76048631625231 -122.95859519671959) bank178911 +178912 POINT(37.15147962784834 -123.34303341356174) bank178912 +178913 POINT(38.62252960963002 -121.60124140530361) bank178913 +178914 POINT(37.98203002020629 -121.9822357274274) bank178914 +178915 POINT(36.98806029476201 -123.40475927277124) bank178915 +178916 POINT(36.83147828913511 -122.97005440927595) bank178916 +178917 POINT(36.86619004955116 -121.50457561024062) bank178917 +178918 POINT(38.07646988103409 -122.07901561603553) bank178918 +178919 POINT(37.52754392749097 -122.07662352919554) bank178919 +178920 POINT(37.11776087899002 -122.93786026884085) bank178920 +178921 POINT(38.417694602461054 -121.5419518433041) bank178921 +178922 POINT(36.98102872246659 -123.2300152021149) bank178922 +178923 POINT(38.69873955336731 -122.57889745225184) bank178923 +178924 POINT(37.191492869678825 -122.53286246381376) bank178924 +178925 POINT(38.67745365360917 -123.19750682274751) bank178925 +178926 POINT(38.29340269842663 -121.44045331560703) bank178926 +178927 POINT(38.019197828718376 -122.54212447426646) bank178927 +178928 POINT(37.668058367627054 -123.04961719866156) bank178928 +178929 POINT(38.66408485835815 -122.81336484127613) bank178929 +178930 POINT(38.7116676105457 -121.57296173998282) bank178930 +178931 POINT(37.71967338571692 -122.15661638523038) bank178931 +178932 POINT(37.34880288274145 -122.9237678746969) bank178932 +178933 POINT(37.221782504641 -121.71641257358081) bank178933 +178934 POINT(36.84082112020175 -122.2989070196043) bank178934 +178935 POINT(36.809272524467666 -122.14554416754508) bank178935 +178936 POINT(38.662477322416926 -121.96847059523704) bank178936 +178937 POINT(37.66323720379127 -121.92749960406739) bank178937 +178938 POINT(37.82243752829305 -123.07821075000389) bank178938 +178939 POINT(36.77805893485197 -122.623594376846) bank178939 +178940 POINT(36.863713323110986 -121.52097702205981) bank178940 +178941 POINT(37.63403079125351 -122.60772309923942) bank178941 +178942 POINT(37.59126138152225 -122.93696185421152) bank178942 +178943 POINT(38.77405178199498 -122.88959680902568) bank178943 +178944 POINT(37.41818343954627 -121.9223156703573) bank178944 +178945 POINT(37.92904347974861 -122.58580996766587) bank178945 +178946 POINT(37.43762134989043 -123.33283229871675) bank178946 +178947 POINT(37.30039775281184 -122.27981100046203) bank178947 +178948 POINT(38.419822420084124 -123.09126434104094) bank178948 +178949 POINT(38.233518122640376 -122.82093597693991) bank178949 +178950 POINT(38.656949454652874 -122.64001265590744) bank178950 +178951 POINT(38.284792433242494 -122.27275728608105) bank178951 +178952 POINT(37.84875856915761 -123.08067721988327) bank178952 +178953 POINT(37.31968574828582 -121.54453272374757) bank178953 +178954 POINT(37.60148656925124 -122.98078295071562) bank178954 +178955 POINT(37.94847360037493 -122.96652014003959) bank178955 +178956 POINT(36.80599438892791 -123.07279235127733) bank178956 +178957 POINT(37.59618241073998 -122.9386884907616) bank178957 +178958 POINT(38.050675378410936 -123.07318296872772) bank178958 +178959 POINT(37.72578694740321 -122.63049145906879) bank178959 +178960 POINT(38.65399287282141 -122.75011039562406) bank178960 +178961 POINT(38.44631772184821 -122.8284915730509) bank178961 +178962 POINT(37.81283804783125 -123.3733250228732) bank178962 +178963 POINT(38.70731058229951 -123.21265822081786) bank178963 +178964 POINT(38.6937470819702 -122.53573713044379) bank178964 +178965 POINT(37.535440666966544 -123.35757969460882) bank178965 +178966 POINT(37.72333775366954 -122.8606014216869) bank178966 +178967 POINT(38.749422538914544 -122.12569116668325) bank178967 +178968 POINT(36.902042574991896 -123.2235403886279) bank178968 +178969 POINT(38.40054885936855 -122.03317311369669) bank178969 +178970 POINT(36.791671094081075 -122.5399394552004) bank178970 +178971 POINT(38.64280347182613 -123.02711709913213) bank178971 +178972 POINT(38.05619687615895 -122.65316798268861) bank178972 +178973 POINT(37.26565616487691 -122.1015188568257) bank178973 +178974 POINT(37.25736317130702 -122.58134110523848) bank178974 +178975 POINT(38.11893662561622 -123.37215156474797) bank178975 +178976 POINT(37.477058234346714 -123.14499530522619) bank178976 +178977 POINT(37.726532524087396 -122.88591152410321) bank178977 +178978 POINT(37.47305348915612 -121.4922690537502) bank178978 +178979 POINT(37.543647782993844 -122.42005859275086) bank178979 +178980 POINT(38.73077656735468 -121.8466440774965) bank178980 +178981 POINT(38.19868478815651 -122.21343915722358) bank178981 +178982 POINT(38.578642078336095 -121.65026734977934) bank178982 +178983 POINT(38.567332528587926 -122.68813065165354) bank178983 +178984 POINT(36.80135357722148 -122.84630151701249) bank178984 +178985 POINT(36.80902911701532 -121.59855800314473) bank178985 +178986 POINT(37.914347011101206 -122.1104109379668) bank178986 +178987 POINT(38.06753504931461 -122.1092179507222) bank178987 +178988 POINT(37.748175386363755 -122.71098263383517) bank178988 +178989 POINT(38.1731750917586 -121.82322932595306) bank178989 +178990 POINT(38.263736227337375 -122.03222849313468) bank178990 +178991 POINT(38.63279376479986 -123.03419227729594) bank178991 +178992 POINT(37.22822527084632 -122.87416742929187) bank178992 +178993 POINT(37.6629271522293 -123.38658587131779) bank178993 +178994 POINT(38.6448707376916 -122.39709447891701) bank178994 +178995 POINT(37.83362774906335 -122.41013949424243) bank178995 +178996 POINT(37.54457500035192 -121.5440703570935) bank178996 +178997 POINT(36.89344197448826 -122.6689963796099) bank178997 +178998 POINT(37.85456437917623 -122.96208150983468) bank178998 +178999 POINT(37.312032010795434 -122.87391995377054) bank178999 +179000 POINT(38.68512892999006 -122.83116820164658) bank179000 +179001 POINT(38.305691151634555 -122.42012123990511) bank179001 +179002 POINT(37.886187745041696 -123.33791598430183) bank179002 +179003 POINT(36.838507970836844 -121.79931011497459) bank179003 +179004 POINT(37.36750551084656 -123.07949319690626) bank179004 +179005 POINT(38.6924988450759 -121.71370731700071) bank179005 +179006 POINT(37.20121410078658 -122.62704389004763) bank179006 +179007 POINT(38.112824102901456 -122.85409394602807) bank179007 +179008 POINT(38.6854115071432 -122.35770794961526) bank179008 +179009 POINT(37.846924635990355 -121.43075161815766) bank179009 +179010 POINT(36.946870135001745 -121.84080862693504) bank179010 +179011 POINT(36.972315738131485 -121.8794126789663) bank179011 +179012 POINT(37.8070529104629 -122.75571704442672) bank179012 +179013 POINT(37.81884025097265 -122.05602223012055) bank179013 +179014 POINT(37.27834389449327 -122.07962653703184) bank179014 +179015 POINT(37.29903008136982 -122.26904376304238) bank179015 +179016 POINT(38.11451197753551 -121.54379175861385) bank179016 +179017 POINT(36.82462084047472 -122.79811821558123) bank179017 +179018 POINT(38.40139393399902 -121.96589691764244) bank179018 +179019 POINT(37.670670578464495 -121.82129308980085) bank179019 +179020 POINT(37.152291352900114 -122.72916660547692) bank179020 +179021 POINT(38.57065691874721 -123.38801663885485) bank179021 +179022 POINT(37.81124776419201 -121.49902458568302) bank179022 +179023 POINT(37.603685121564716 -122.15777461086715) bank179023 +179024 POINT(38.62622428395396 -122.10728055595344) bank179024 +179025 POINT(37.089206247932594 -121.61667993682661) bank179025 +179026 POINT(37.27855715038387 -122.75615626438893) bank179026 +179027 POINT(37.25173567249709 -121.7671372487841) bank179027 +179028 POINT(37.91861645752271 -122.64518839615297) bank179028 +179029 POINT(38.10731938697495 -122.61328020048968) bank179029 +179030 POINT(38.07878997690339 -122.52643504332549) bank179030 +179031 POINT(38.5967130407323 -123.26177334202816) bank179031 +179032 POINT(36.78817987966859 -122.47034978829758) bank179032 +179033 POINT(37.808658898066774 -122.60820914385623) bank179033 +179034 POINT(37.6247092143118 -123.08628310975939) bank179034 +179035 POINT(37.63461523063068 -123.11735138337271) bank179035 +179036 POINT(38.247094486295985 -122.3826699164428) bank179036 +179037 POINT(38.20671150873331 -122.45056627496554) bank179037 +179038 POINT(38.104808958409386 -121.43499064793195) bank179038 +179039 POINT(37.950522255593654 -123.13588638465247) bank179039 +179040 POINT(36.93603571508046 -123.22302201256927) bank179040 +179041 POINT(38.153225674439234 -121.54202789256668) bank179041 +179042 POINT(38.20572663935864 -121.45236082281414) bank179042 +179043 POINT(38.25224618860384 -122.34583255147624) bank179043 +179044 POINT(37.82652952429146 -122.99877388559636) bank179044 +179045 POINT(37.297192172293414 -123.09929755225475) bank179045 +179046 POINT(36.96681950771467 -121.53755044837543) bank179046 +179047 POINT(38.16133535876806 -122.16219926283885) bank179047 +179048 POINT(37.36266125489951 -121.62980217561639) bank179048 +179049 POINT(38.056298495670504 -122.72401091508354) bank179049 +179050 POINT(38.2005000687246 -122.50762887836906) bank179050 +179051 POINT(37.28014971240741 -122.92952130383982) bank179051 +179052 POINT(37.43312302410795 -122.42178538145416) bank179052 +179053 POINT(36.96314448977253 -122.17840445785461) bank179053 +179054 POINT(38.14072464472115 -122.2149790516243) bank179054 +179055 POINT(38.182434142434985 -122.55225808560729) bank179055 +179056 POINT(37.10861538948199 -121.90885273440259) bank179056 +179057 POINT(37.75484976671115 -122.2609228664007) bank179057 +179058 POINT(38.61535102487762 -121.87081611778157) bank179058 +179059 POINT(38.035331039720276 -122.50506983638194) bank179059 +179060 POINT(38.53638312570606 -122.2837147495542) bank179060 +179061 POINT(37.41961791912158 -122.35261722814101) bank179061 +179062 POINT(37.60767324267085 -123.29409619005014) bank179062 +179063 POINT(37.248224774103704 -122.71455444771522) bank179063 +179064 POINT(37.40447192959706 -123.3868079588892) bank179064 +179065 POINT(37.306351625042865 -121.71001803891566) bank179065 +179066 POINT(38.1098687044717 -123.22578595978665) bank179066 +179067 POINT(36.91474145974168 -123.0593623780976) bank179067 +179068 POINT(36.82689931286618 -123.3276214283836) bank179068 +179069 POINT(37.532136916969186 -121.55636481707971) bank179069 +179070 POINT(38.091870784326204 -122.42057549439293) bank179070 +179071 POINT(36.84482685539995 -121.91142048379257) bank179071 +179072 POINT(37.780723276034365 -122.60813325584986) bank179072 +179073 POINT(38.485769828908325 -122.17414382101977) bank179073 +179074 POINT(37.926392135851316 -123.08137663693476) bank179074 +179075 POINT(37.52793545182576 -122.65213521379668) bank179075 +179076 POINT(37.508655566172074 -122.50420796697027) bank179076 +179077 POINT(37.26215116052434 -122.01178807596104) bank179077 +179078 POINT(37.21776895556674 -123.38953227228176) bank179078 +179079 POINT(37.41483584871656 -122.3239484608294) bank179079 +179080 POINT(37.60795892748179 -122.58610660256446) bank179080 +179081 POINT(37.06967492537089 -122.0992866328003) bank179081 +179082 POINT(37.5458048376499 -122.50934110213734) bank179082 +179083 POINT(37.50673733914231 -122.27467181513023) bank179083 +179084 POINT(38.342265395881 -121.93433666039101) bank179084 +179085 POINT(37.56411798242137 -121.46511939256399) bank179085 +179086 POINT(38.58497932584437 -122.81045261763906) bank179086 +179087 POINT(38.43203702983708 -121.53107306946639) bank179087 +179088 POINT(36.99968438125079 -122.923114013225) bank179088 +179089 POINT(38.32406284823816 -121.71209185876815) bank179089 +179090 POINT(38.233698894820925 -121.58890450046914) bank179090 +179091 POINT(36.81544866694534 -121.49239473654052) bank179091 +179092 POINT(38.33852247446504 -121.64250425515623) bank179092 +179093 POINT(37.71222860274855 -121.50142961013455) bank179093 +179094 POINT(38.23652771688607 -123.15632126026664) bank179094 +179095 POINT(37.50536722669469 -122.14158273083854) bank179095 +179096 POINT(36.97042982346794 -123.0336908507936) bank179096 +179097 POINT(38.25010992663171 -122.26023775951403) bank179097 +179098 POINT(38.001700026784846 -121.8191836010194) bank179098 +179099 POINT(38.34712919181915 -121.5188408765994) bank179099 +179100 POINT(37.592802860213894 -121.68250482431813) bank179100 +179101 POINT(37.300462332140384 -122.61811003085751) bank179101 +179102 POINT(38.15021018376868 -122.9060880537004) bank179102 +179103 POINT(37.02374112895597 -121.77728610209834) bank179103 +179104 POINT(37.547089026326425 -123.40844647134765) bank179104 +179105 POINT(37.98852841876208 -122.30703493664186) bank179105 +179106 POINT(37.89940130862203 -121.98052834408516) bank179106 +179107 POINT(36.88951897914302 -122.6704435366492) bank179107 +179108 POINT(37.398516369530945 -121.8909296678519) bank179108 +179109 POINT(38.31252615033505 -122.77808859448407) bank179109 +179110 POINT(38.04873871805218 -122.55933016931844) bank179110 +179111 POINT(37.906367421277444 -121.82263930696753) bank179111 +179112 POINT(38.33411469203155 -122.84094754709596) bank179112 +179113 POINT(37.082192686467636 -122.39982138481356) bank179113 +179114 POINT(37.27407125352321 -123.3639011476762) bank179114 +179115 POINT(37.84817285920883 -122.76771720736122) bank179115 +179116 POINT(37.62706079129127 -122.13004120889914) bank179116 +179117 POINT(37.98319622955858 -122.14825409045729) bank179117 +179118 POINT(38.25038302279893 -121.8365890674662) bank179118 +179119 POINT(37.6988252124425 -123.09888640422727) bank179119 +179120 POINT(38.174880931179715 -121.8118717966422) bank179120 +179121 POINT(37.74434720947643 -122.96080896454427) bank179121 +179122 POINT(37.21027730808273 -123.37224879569749) bank179122 +179123 POINT(37.73878087857071 -122.85863347237772) bank179123 +179124 POINT(37.7771035950583 -122.28281753503164) bank179124 +179125 POINT(37.595198762822 -122.33562278158844) bank179125 +179126 POINT(38.23584524718471 -121.48913535259051) bank179126 +179127 POINT(36.820978272930596 -122.37534278843114) bank179127 +179128 POINT(38.73253038061061 -121.51309165245594) bank179128 +179129 POINT(36.95929347883435 -122.49274811053803) bank179129 +179130 POINT(38.76145198652164 -123.27387206715434) bank179130 +179131 POINT(36.981009918236225 -122.47902111456992) bank179131 +179132 POINT(38.089932011880926 -122.02275951670573) bank179132 +179133 POINT(38.28221446186485 -122.56307346028979) bank179133 +179134 POINT(38.25943339517413 -121.59907945542174) bank179134 +179135 POINT(37.36684965518137 -121.44586606281266) bank179135 +179136 POINT(37.32280472036444 -122.84352693384437) bank179136 +179137 POINT(37.25890125167706 -121.91480007670967) bank179137 +179138 POINT(36.85622021093318 -121.56293627366885) bank179138 +179139 POINT(37.21865810361842 -121.8281436696738) bank179139 +179140 POINT(38.36152940661432 -121.78050531055227) bank179140 +179141 POINT(38.51321613627573 -121.51093640755525) bank179141 +179142 POINT(38.1023914022848 -121.71464921863475) bank179142 +179143 POINT(37.47173981467983 -122.86447798425047) bank179143 +179144 POINT(37.215666403067424 -122.07825942497685) bank179144 +179145 POINT(37.48336384448836 -121.8438696200246) bank179145 +179146 POINT(36.86821022670332 -123.09008605225083) bank179146 +179147 POINT(37.80050235777095 -122.55935712695131) bank179147 +179148 POINT(38.613015318535375 -122.38621714601206) bank179148 +179149 POINT(38.74475469294743 -122.88861139204714) bank179149 +179150 POINT(38.7298639756855 -122.56738128334605) bank179150 +179151 POINT(36.823696800504074 -122.87390304745213) bank179151 +179152 POINT(36.872509749252686 -121.46008834342605) bank179152 +179153 POINT(37.27636337134088 -122.93611906022356) bank179153 +179154 POINT(37.145218352595755 -122.97873145049984) bank179154 +179155 POINT(37.85086028314598 -123.16083357280422) bank179155 +179156 POINT(37.63953196697346 -123.39513703201858) bank179156 +179157 POINT(38.43937910858156 -122.20951495201683) bank179157 +179158 POINT(38.079140540331146 -121.43653821023483) bank179158 +179159 POINT(37.0566353590122 -121.87617699848298) bank179159 +179160 POINT(37.056000940042026 -121.48148330489624) bank179160 +179161 POINT(37.899144392104766 -123.07553147339343) bank179161 +179162 POINT(37.60630853039061 -122.95223851233824) bank179162 +179163 POINT(37.87088780359647 -121.68252066486083) bank179163 +179164 POINT(38.706269739902815 -121.63855698765458) bank179164 +179165 POINT(37.54265342687097 -121.5416820875908) bank179165 +179166 POINT(38.41726929628233 -122.57380780719932) bank179166 +179167 POINT(37.77805707140479 -123.27382706165518) bank179167 +179168 POINT(37.41503327932499 -123.34945054031988) bank179168 +179169 POINT(37.22811721427939 -122.62503194281696) bank179169 +179170 POINT(38.450832743946826 -121.5219549426392) bank179170 +179171 POINT(38.3254978130187 -123.0189761868102) bank179171 +179172 POINT(37.2987781564757 -122.7430845130322) bank179172 +179173 POINT(37.58812079000513 -123.14384857070236) bank179173 +179174 POINT(37.88380200436932 -123.39884938283718) bank179174 +179175 POINT(37.78194645540889 -122.87135412003497) bank179175 +179176 POINT(36.88355489810314 -121.48922083968213) bank179176 +179177 POINT(37.41154563846689 -122.57054347931701) bank179177 +179178 POINT(37.615963707519455 -121.4573844559354) bank179178 +179179 POINT(36.88956561217882 -121.53753896070923) bank179179 +179180 POINT(38.25582054937166 -122.85998969023427) bank179180 +179181 POINT(37.108105597071514 -122.45891419486102) bank179181 +179182 POINT(37.17867277241612 -122.27102810017595) bank179182 +179183 POINT(38.0147507200413 -123.09637269557446) bank179183 +179184 POINT(38.404001018462665 -122.3374482697185) bank179184 +179185 POINT(38.14751891710908 -123.02415713798813) bank179185 +179186 POINT(38.27918657395423 -122.81984762820946) bank179186 +179187 POINT(38.717763475656746 -121.76107996672661) bank179187 +179188 POINT(38.753988079483165 -122.08562673882275) bank179188 +179189 POINT(37.592715487714415 -122.72989047361155) bank179189 +179190 POINT(38.15535299852605 -122.01655515694162) bank179190 +179191 POINT(37.6186205724357 -122.73404292787978) bank179191 +179192 POINT(36.872364745901464 -123.06426812531765) bank179192 +179193 POINT(36.949621938752344 -122.17758335958065) bank179193 +179194 POINT(38.00071841723502 -123.1270835886536) bank179194 +179195 POINT(38.4201731701998 -123.28129834929209) bank179195 +179196 POINT(38.16181729048052 -122.16340361364367) bank179196 +179197 POINT(37.8974072974599 -122.81308758949682) bank179197 +179198 POINT(38.733739629215016 -122.91858113501587) bank179198 +179199 POINT(37.20939420542382 -121.82665774007431) bank179199 +179200 POINT(38.318164767709106 -123.07326649512305) bank179200 +179201 POINT(38.20020488097061 -122.87374528654846) bank179201 +179202 POINT(37.39748816070991 -122.9341810819728) bank179202 +179203 POINT(37.25678602167216 -123.05575126049348) bank179203 +179204 POINT(37.608132951776774 -123.01294316723603) bank179204 +179205 POINT(38.68201873122857 -121.88158344655231) bank179205 +179206 POINT(38.08450573520707 -122.01211225129347) bank179206 +179207 POINT(37.70187753609865 -121.78736226045294) bank179207 +179208 POINT(37.70011475122617 -122.73030296082172) bank179208 +179209 POINT(38.589175344741236 -123.21738619305954) bank179209 +179210 POINT(37.72648615455068 -122.32812209748045) bank179210 +179211 POINT(37.62771760762057 -121.79965316545733) bank179211 +179212 POINT(38.18293656808527 -121.42175080003076) bank179212 +179213 POINT(38.627588438793 -122.08659083735334) bank179213 +179214 POINT(38.29077129406283 -122.071335274881) bank179214 +179215 POINT(38.2316981799416 -122.26781661113513) bank179215 +179216 POINT(37.68901407137401 -122.0801500556023) bank179216 +179217 POINT(36.877092511235745 -123.21006257983206) bank179217 +179218 POINT(36.87512839393565 -121.95873197695495) bank179218 +179219 POINT(38.746565349156015 -123.20497052761756) bank179219 +179220 POINT(36.79098338771945 -122.02302845082983) bank179220 +179221 POINT(38.2160204250145 -122.70085947001733) bank179221 +179222 POINT(37.18283109220501 -122.19796194309075) bank179222 +179223 POINT(37.179488413025034 -122.25761571876897) bank179223 +179224 POINT(37.020545476915196 -123.22021739952056) bank179224 +179225 POINT(37.972681153586464 -122.5053784603534) bank179225 +179226 POINT(37.698981801825774 -121.50738134984604) bank179226 +179227 POINT(37.495300358336465 -123.21604709058946) bank179227 +179228 POINT(37.08164291051569 -121.5949126159888) bank179228 +179229 POINT(38.57243173112344 -121.61980051034911) bank179229 +179230 POINT(37.61804607421696 -123.05854041700684) bank179230 +179231 POINT(38.09189347713312 -122.53118105495989) bank179231 +179232 POINT(37.78652269793737 -122.87408948593246) bank179232 +179233 POINT(37.34292382128629 -122.40573234324444) bank179233 +179234 POINT(38.6177601287562 -121.73409257755691) bank179234 +179235 POINT(38.44261160887841 -122.29029136266954) bank179235 +179236 POINT(36.869064282864535 -121.64758355452324) bank179236 +179237 POINT(38.478762087571205 -122.03139355142798) bank179237 +179238 POINT(38.63484039785454 -122.0286946780852) bank179238 +179239 POINT(38.16541219605765 -121.83585564240107) bank179239 +179240 POINT(38.33162145713897 -122.18852833709194) bank179240 +179241 POINT(38.155037566278125 -123.11508593948422) bank179241 +179242 POINT(36.976171249461075 -122.24061925651706) bank179242 +179243 POINT(37.71055938553895 -122.24689997515163) bank179243 +179244 POINT(38.22454536211445 -123.27051043108403) bank179244 +179245 POINT(38.67642592262838 -122.4205562548968) bank179245 +179246 POINT(37.064397050571664 -122.20368252130645) bank179246 +179247 POINT(38.60021035764542 -123.26272986672481) bank179247 +179248 POINT(38.68232661341867 -121.84948428074489) bank179248 +179249 POINT(37.82461285822513 -122.99849038440304) bank179249 +179250 POINT(38.63070612285987 -122.80844815491888) bank179250 +179251 POINT(36.874284864776904 -122.04069127168411) bank179251 +179252 POINT(38.21170232276116 -122.59333822816902) bank179252 +179253 POINT(37.29660426058527 -121.85180013144466) bank179253 +179254 POINT(38.74501316614932 -123.01686450936432) bank179254 +179255 POINT(37.957804602477566 -122.01551122259114) bank179255 +179256 POINT(38.48617387767891 -122.89883883646858) bank179256 +179257 POINT(36.798886454457495 -122.87870906148046) bank179257 +179258 POINT(38.27337534988272 -122.06841294528446) bank179258 +179259 POINT(37.22032697668242 -122.72611164501573) bank179259 +179260 POINT(37.748371277345804 -122.4524615633638) bank179260 +179261 POINT(37.48350858179429 -122.80223925215296) bank179261 +179262 POINT(37.11351003838849 -123.39236816535305) bank179262 +179263 POINT(37.09818538371469 -122.90527343263996) bank179263 +179264 POINT(37.630143869750945 -121.87483766516634) bank179264 +179265 POINT(37.31136826788942 -121.84389989443957) bank179265 +179266 POINT(38.76996501249351 -121.72439641074912) bank179266 +179267 POINT(38.32045509503089 -121.46027663902272) bank179267 +179268 POINT(37.98261865273999 -122.93358984113385) bank179268 +179269 POINT(38.26771201579568 -123.24908427672003) bank179269 +179270 POINT(36.89073333572013 -123.00471343556224) bank179270 +179271 POINT(38.257533824798585 -121.52163294430989) bank179271 +179272 POINT(37.592506973947295 -121.45566190231293) bank179272 +179273 POINT(37.89889996203137 -121.86975054092514) bank179273 +179274 POINT(37.957250563633345 -122.2454660717338) bank179274 +179275 POINT(38.03465459400122 -123.36431082174809) bank179275 +179276 POINT(37.03177787033581 -123.2195106470413) bank179276 +179277 POINT(38.65484346699381 -121.98867415063965) bank179277 +179278 POINT(37.86210685251321 -123.3972781423595) bank179278 +179279 POINT(38.2898942680825 -123.03413717578655) bank179279 +179280 POINT(37.81141769711026 -121.94898799668097) bank179280 +179281 POINT(38.03754790534503 -123.02835368600357) bank179281 +179282 POINT(38.479173517615784 -121.682212022268) bank179282 +179283 POINT(37.70342903231589 -121.58722259115699) bank179283 +179284 POINT(38.118452227727666 -122.90562645383828) bank179284 +179285 POINT(37.356147997155 -121.45317957497262) bank179285 +179286 POINT(37.23837558252133 -121.8478619934907) bank179286 +179287 POINT(37.12535693916031 -122.14856399666574) bank179287 +179288 POINT(38.76343242138466 -122.61220917641748) bank179288 +179289 POINT(36.842849435523284 -122.56782512742892) bank179289 +179290 POINT(37.25221689683575 -122.82973906208277) bank179290 +179291 POINT(37.25216555123423 -123.26070979392914) bank179291 +179292 POINT(37.35338125501594 -122.48348004275404) bank179292 +179293 POINT(37.34323827715794 -122.58349352459754) bank179293 +179294 POINT(38.43032153153162 -122.02656784412966) bank179294 +179295 POINT(38.45799908174679 -121.53238052651456) bank179295 +179296 POINT(37.30210273635523 -122.86149728084263) bank179296 +179297 POINT(37.101290814428125 -121.79035961170345) bank179297 +179298 POINT(36.990084949664336 -123.0508540873712) bank179298 +179299 POINT(37.91687248470455 -122.95414227093028) bank179299 +179300 POINT(37.78777119632286 -123.30262866633527) bank179300 +179301 POINT(38.31120142625948 -122.7352293724302) bank179301 +179302 POINT(38.73617007807941 -122.20367863111402) bank179302 +179303 POINT(36.867597938144755 -121.96355727167823) bank179303 +179304 POINT(37.68976386449145 -122.80220009422827) bank179304 +179305 POINT(38.372606858602325 -122.04823672139085) bank179305 +179306 POINT(37.86255284577564 -121.95141554505553) bank179306 +179307 POINT(38.62212492086765 -122.30057505394836) bank179307 +179308 POINT(37.576227245514715 -123.31281339850588) bank179308 +179309 POINT(37.405272353716896 -122.23652242341164) bank179309 +179310 POINT(37.91139733904582 -122.70132657770829) bank179310 +179311 POINT(38.66962534633999 -122.88748074921344) bank179311 +179312 POINT(37.88943826352228 -121.60071671080263) bank179312 +179313 POINT(37.69849375445043 -121.9684468440742) bank179313 +179314 POINT(38.76827499819326 -122.53310050506228) bank179314 +179315 POINT(38.13157157006181 -122.67533860187615) bank179315 +179316 POINT(37.30802664196736 -122.2128504991197) bank179316 +179317 POINT(38.4367924154987 -123.3016428871143) bank179317 +179318 POINT(36.87369857948887 -123.02453833933586) bank179318 +179319 POINT(38.02940382015609 -121.50860862016569) bank179319 +179320 POINT(36.9883044354464 -122.49648786035473) bank179320 +179321 POINT(38.48110102532322 -123.303542305524) bank179321 +179322 POINT(37.96306937314937 -122.88877848889943) bank179322 +179323 POINT(37.75502139610597 -122.46508018673691) bank179323 +179324 POINT(38.72904150103846 -122.69536990576232) bank179324 +179325 POINT(38.65179337595733 -122.55015872010776) bank179325 +179326 POINT(36.82663913950495 -121.68374000188908) bank179326 +179327 POINT(37.70672320395564 -123.34374240234479) bank179327 +179328 POINT(38.293906339761925 -121.66415573493526) bank179328 +179329 POINT(38.256873105192426 -122.3504101683745) bank179329 +179330 POINT(37.40989906035963 -121.88849677125799) bank179330 +179331 POINT(38.49405804871191 -123.09694852883474) bank179331 +179332 POINT(38.476670525536015 -122.39717989904823) bank179332 +179333 POINT(38.55326987430878 -123.02542280065052) bank179333 +179334 POINT(37.47994150706215 -121.43066663810527) bank179334 +179335 POINT(37.78148343201183 -122.22543829898177) bank179335 +179336 POINT(37.01659229718337 -121.643255741159) bank179336 +179337 POINT(36.93738629147667 -122.33642127099286) bank179337 +179338 POINT(37.75041332667434 -122.7915926582079) bank179338 +179339 POINT(37.652551931485355 -122.73119673521921) bank179339 +179340 POINT(38.30834226053687 -122.14022344678413) bank179340 +179341 POINT(37.32322817297674 -121.96723450506684) bank179341 +179342 POINT(38.449215567585384 -122.79456189588157) bank179342 +179343 POINT(38.710404490437924 -122.16747500176916) bank179343 +179344 POINT(38.016410580645534 -123.1021520811061) bank179344 +179345 POINT(38.60663648364082 -121.68901529307036) bank179345 +179346 POINT(36.9271693570999 -123.03208275760102) bank179346 +179347 POINT(38.48965217567608 -122.08171135560445) bank179347 +179348 POINT(38.0362464540791 -121.51404336640805) bank179348 +179349 POINT(36.89935618391942 -122.92541117303331) bank179349 +179350 POINT(38.02211404536618 -121.46947007379565) bank179350 +179351 POINT(38.67898033830564 -121.83563468530605) bank179351 +179352 POINT(37.471165605259785 -121.98108440859103) bank179352 +179353 POINT(38.07535205538282 -121.78275236801119) bank179353 +179354 POINT(38.40787356269314 -122.61309198056092) bank179354 +179355 POINT(37.99510893401873 -122.37647469069725) bank179355 +179356 POINT(38.42993916394598 -123.0817241018376) bank179356 +179357 POINT(38.01001185824631 -122.46589629835393) bank179357 +179358 POINT(36.956159473890345 -123.02799170236096) bank179358 +179359 POINT(37.013362672575305 -121.96568370891264) bank179359 +179360 POINT(38.70623070770902 -122.86436570272241) bank179360 +179361 POINT(37.951660813420375 -121.79691503306236) bank179361 +179362 POINT(38.646087482658274 -123.37368283992247) bank179362 +179363 POINT(36.90274870809053 -122.897030059234) bank179363 +179364 POINT(37.34043077582056 -121.65343709747687) bank179364 +179365 POINT(37.15537138923855 -122.61111020288341) bank179365 +179366 POINT(36.84203856809574 -122.70899668679013) bank179366 +179367 POINT(36.82994965549902 -123.32971443869891) bank179367 +179368 POINT(38.19195367442542 -121.70843696487569) bank179368 +179369 POINT(37.91827516649411 -122.05637944671217) bank179369 +179370 POINT(36.88191074885493 -121.76493850242356) bank179370 +179371 POINT(36.87527621926171 -121.94012740189073) bank179371 +179372 POINT(38.71445955184106 -122.02219195368782) bank179372 +179373 POINT(38.39791053298273 -121.85915450311462) bank179373 +179374 POINT(36.8038308468561 -122.6228361067316) bank179374 +179375 POINT(37.5496138618083 -122.18921834083751) bank179375 +179376 POINT(38.27875333099197 -122.5200753574876) bank179376 +179377 POINT(38.44693002834877 -123.1946255633741) bank179377 +179378 POINT(37.06077513604261 -121.84558422589922) bank179378 +179379 POINT(37.43055901887556 -122.9205517632749) bank179379 +179380 POINT(37.49266103042531 -123.19324885512304) bank179380 +179381 POINT(37.786768784954894 -121.70543056104522) bank179381 +179382 POINT(37.092071810278185 -122.28822806499487) bank179382 +179383 POINT(37.35392396985333 -122.21746567137465) bank179383 +179384 POINT(36.96063804942502 -123.34514662475222) bank179384 +179385 POINT(37.9296615190396 -123.23206403503652) bank179385 +179386 POINT(38.51946267690245 -123.05468974749762) bank179386 +179387 POINT(38.580763353263734 -122.91868670498434) bank179387 +179388 POINT(37.93393463207519 -122.83484322525342) bank179388 +179389 POINT(36.800017155121544 -122.78606236593497) bank179389 +179390 POINT(37.091881150889456 -121.52941956303758) bank179390 +179391 POINT(37.15480804768965 -122.79847197765918) bank179391 +179392 POINT(37.7578700991846 -122.66300737160748) bank179392 +179393 POINT(38.223085678046964 -121.90671022700928) bank179393 +179394 POINT(37.78336905624467 -122.80118625597999) bank179394 +179395 POINT(38.15706195335121 -122.97849721930676) bank179395 +179396 POINT(37.786390602089114 -122.37837246364587) bank179396 +179397 POINT(36.93623374824518 -123.28894213373647) bank179397 +179398 POINT(38.23365617851895 -123.41775821534591) bank179398 +179399 POINT(38.4865206474221 -122.62810107562919) bank179399 +179400 POINT(37.32907197721171 -122.65957629191168) bank179400 +179401 POINT(38.5610815195592 -122.21284050664865) bank179401 +179402 POINT(37.22007310855414 -122.9354373251251) bank179402 +179403 POINT(37.229976079867484 -121.86343891701677) bank179403 +179404 POINT(36.87456494124691 -122.90025577997488) bank179404 +179405 POINT(37.18580906427151 -121.61744626162476) bank179405 +179406 POINT(37.307053836298245 -121.93136992164648) bank179406 +179407 POINT(37.6346574774356 -121.8841061553031) bank179407 +179408 POINT(37.63134608220488 -122.41866917457689) bank179408 +179409 POINT(37.128925012885546 -122.76938217744474) bank179409 +179410 POINT(37.90996472192301 -121.99590819088843) bank179410 +179411 POINT(38.70901516455504 -123.27082790649075) bank179411 +179412 POINT(38.62569463481722 -123.06033257794172) bank179412 +179413 POINT(38.341873708492955 -121.73590518529825) bank179413 +179414 POINT(38.069874949392016 -122.522476336681) bank179414 +179415 POINT(37.514281476185964 -121.6774345672705) bank179415 +179416 POINT(38.41073841635262 -121.96272994724443) bank179416 +179417 POINT(38.043758335443805 -122.59085543797666) bank179417 +179418 POINT(37.885151572250024 -123.04363081643912) bank179418 +179419 POINT(37.614792701966095 -122.70091080158336) bank179419 +179420 POINT(37.7272888296758 -121.81804581198374) bank179420 +179421 POINT(38.63290613812788 -122.13767653712561) bank179421 +179422 POINT(37.69110766547589 -123.0240912941104) bank179422 +179423 POINT(38.439946678328795 -121.44766943405614) bank179423 +179424 POINT(37.474214836279685 -122.78694959264944) bank179424 +179425 POINT(38.7252374241268 -122.31299196214779) bank179425 +179426 POINT(36.99494169790201 -122.26628771472092) bank179426 +179427 POINT(37.58397909774075 -121.78146557806294) bank179427 +179428 POINT(37.42460817703201 -122.36105994898958) bank179428 +179429 POINT(38.66530816026753 -121.65366106751442) bank179429 +179430 POINT(37.47378292566438 -123.11170648804878) bank179430 +179431 POINT(38.49377918706754 -123.33178841349329) bank179431 +179432 POINT(37.441968554587575 -122.51116506016467) bank179432 +179433 POINT(38.27915807287334 -122.17383275678843) bank179433 +179434 POINT(38.296690701067064 -122.57319646201421) bank179434 +179435 POINT(36.85097849330792 -121.45560180598757) bank179435 +179436 POINT(37.65872252437749 -122.93392671881328) bank179436 +179437 POINT(37.75877007436428 -122.01084974950616) bank179437 +179438 POINT(36.78668484097526 -122.4317939917822) bank179438 +179439 POINT(38.62764796696252 -121.8334636102517) bank179439 +179440 POINT(36.946786245289324 -123.29538423174039) bank179440 +179441 POINT(38.5621444811495 -122.01060074188113) bank179441 +179442 POINT(38.014871178945036 -122.18139486358405) bank179442 +179443 POINT(37.730009865605155 -122.3766145574839) bank179443 +179444 POINT(38.25012448082322 -122.87963212301113) bank179444 +179445 POINT(37.09289597231972 -122.6149894054349) bank179445 +179446 POINT(38.20866243176111 -122.62250449344441) bank179446 +179447 POINT(38.416296163904605 -122.45182687922058) bank179447 +179448 POINT(36.912471910297874 -122.97063454944805) bank179448 +179449 POINT(38.72102991805646 -122.3602311106261) bank179449 +179450 POINT(37.42531017332396 -122.05979016237261) bank179450 +179451 POINT(37.824087370124744 -123.24806742265483) bank179451 +179452 POINT(37.34950776338913 -122.37181480960201) bank179452 +179453 POINT(37.7623641615359 -122.12668248415366) bank179453 +179454 POINT(37.771310934542015 -122.02864572256654) bank179454 +179455 POINT(37.63661735701078 -121.93555516621997) bank179455 +179456 POINT(38.66052146493809 -122.86788543745645) bank179456 +179457 POINT(37.78947894160249 -122.32149238101638) bank179457 +179458 POINT(38.22609491700639 -123.41325591049706) bank179458 +179459 POINT(38.01258675583719 -122.17164387965967) bank179459 +179460 POINT(38.06073255444022 -122.19178572934185) bank179460 +179461 POINT(38.539832563025776 -123.28326123263744) bank179461 +179462 POINT(37.4685169617911 -122.2908456609365) bank179462 +179463 POINT(38.693230736533785 -122.21578831598144) bank179463 +179464 POINT(36.957077494843965 -123.0781627691904) bank179464 +179465 POINT(37.393101298981605 -122.18981963002439) bank179465 +179466 POINT(37.58838316476236 -121.69538090256019) bank179466 +179467 POINT(38.16189542462261 -121.75536448863389) bank179467 +179468 POINT(37.23009575621105 -123.41165443531897) bank179468 +179469 POINT(37.11128813269524 -122.79000664133896) bank179469 +179470 POINT(38.72222052453593 -122.14522911747073) bank179470 +179471 POINT(37.084398776750355 -122.54126375186279) bank179471 +179472 POINT(37.41592183741412 -122.46306674557738) bank179472 +179473 POINT(37.18952632294294 -123.36169793225076) bank179473 +179474 POINT(38.03240733506731 -122.77265691019386) bank179474 +179475 POINT(38.20353069035332 -121.54484089544687) bank179475 +179476 POINT(37.96581842014832 -121.52955946790944) bank179476 +179477 POINT(37.871098433139686 -122.57261191295612) bank179477 +179478 POINT(38.77034876930135 -122.74576054695366) bank179478 +179479 POINT(38.06153595334232 -121.45288822910283) bank179479 +179480 POINT(38.388627322027965 -122.69660499460859) bank179480 +179481 POINT(38.21371995476548 -121.78637569486095) bank179481 +179482 POINT(37.554433139734186 -122.78982922324832) bank179482 +179483 POINT(37.3303593512873 -122.29239685488534) bank179483 +179484 POINT(38.320005866857294 -122.46015409591682) bank179484 +179485 POINT(38.47432832018269 -122.71385898962583) bank179485 +179486 POINT(37.81256421478226 -121.65960464096547) bank179486 +179487 POINT(37.39725890981892 -122.69247747534845) bank179487 +179488 POINT(36.79470193619002 -122.4273201973369) bank179488 +179489 POINT(38.435781449058524 -122.49997807068965) bank179489 +179490 POINT(37.40680766249441 -122.79678208231539) bank179490 +179491 POINT(37.52804486973994 -123.40343482829553) bank179491 +179492 POINT(38.73256299937411 -121.78175732406297) bank179492 +179493 POINT(37.76029768701305 -122.19354329972406) bank179493 +179494 POINT(38.71607148374477 -122.76091299359668) bank179494 +179495 POINT(38.037830430733806 -121.49804615912339) bank179495 +179496 POINT(37.252244142105894 -122.38064593203283) bank179496 +179497 POINT(38.50953137676745 -121.76778461713772) bank179497 +179498 POINT(38.529486861138665 -121.8450114304199) bank179498 +179499 POINT(37.25843520825624 -122.05616555103624) bank179499 +179500 POINT(37.39778263259982 -121.5348308799408) bank179500 +179501 POINT(37.51419443284917 -122.30536053526944) bank179501 +179502 POINT(37.42597730227332 -121.45361936181203) bank179502 +179503 POINT(37.33731986206884 -121.93809147790982) bank179503 +179504 POINT(37.154689117189236 -122.71060624153834) bank179504 +179505 POINT(36.90986294936253 -122.69819757091307) bank179505 +179506 POINT(38.07597822774886 -122.07579800715204) bank179506 +179507 POINT(38.21858276798988 -121.61567386018284) bank179507 +179508 POINT(38.40284545247335 -121.43911665285336) bank179508 +179509 POINT(38.44932621103119 -121.97846518356849) bank179509 +179510 POINT(38.16584599239878 -121.6315805993104) bank179510 +179511 POINT(38.02202474723527 -122.32569084103096) bank179511 +179512 POINT(38.70214410788308 -123.23091281193915) bank179512 +179513 POINT(37.40716567853739 -123.00524774438983) bank179513 +179514 POINT(38.158365268861296 -121.8658163576363) bank179514 +179515 POINT(37.43779933550792 -122.24498412089244) bank179515 +179516 POINT(38.23040189988426 -122.70806079756812) bank179516 +179517 POINT(38.469911608650236 -121.97826401129157) bank179517 +179518 POINT(37.359777316216054 -121.53238168089021) bank179518 +179519 POINT(38.62325330986306 -123.03907104805803) bank179519 +179520 POINT(37.501324263679194 -123.17857615304395) bank179520 +179521 POINT(38.772608242571444 -122.7591677959436) bank179521 +179522 POINT(38.71739150583347 -122.30003555973171) bank179522 +179523 POINT(38.20220946270407 -121.8236513276895) bank179523 +179524 POINT(36.84878896143803 -121.90339418996862) bank179524 +179525 POINT(36.97904704108893 -122.05395174645251) bank179525 +179526 POINT(37.88624532470227 -122.10463828172692) bank179526 +179527 POINT(38.614133444799805 -122.55286373670361) bank179527 +179528 POINT(37.020416997014614 -121.52141784173237) bank179528 +179529 POINT(37.51003516149982 -123.3035306512756) bank179529 +179530 POINT(38.34853046849544 -121.83712412150493) bank179530 +179531 POINT(38.22062527521127 -121.5277679552001) bank179531 +179532 POINT(37.739366595014424 -123.38519970833563) bank179532 +179533 POINT(37.84097385024802 -121.64183937925948) bank179533 +179534 POINT(37.271575666145296 -122.15001360219267) bank179534 +179535 POINT(38.40490758410847 -122.6113542288294) bank179535 +179536 POINT(38.22425523238107 -122.42960968334155) bank179536 +179537 POINT(38.23066643096242 -121.51412385659667) bank179537 +179538 POINT(36.80251753357652 -122.19050446120912) bank179538 +179539 POINT(36.993295983255535 -123.16529306109116) bank179539 +179540 POINT(37.795807508065664 -122.37863770638731) bank179540 +179541 POINT(36.92919610979457 -122.87387676913309) bank179541 +179542 POINT(36.818316421588 -121.59247284467244) bank179542 +179543 POINT(38.285580729807336 -121.90163143933587) bank179543 +179544 POINT(37.942031637777575 -121.59597659039585) bank179544 +179545 POINT(38.232710225924876 -122.87271052716315) bank179545 +179546 POINT(37.27157421943141 -122.09811457060543) bank179546 +179547 POINT(37.54226871147037 -121.50906112344423) bank179547 +179548 POINT(38.12376865441056 -121.63582905097591) bank179548 +179549 POINT(38.536212642339585 -122.35177108679426) bank179549 +179550 POINT(36.919632159698224 -122.26970094280588) bank179550 +179551 POINT(37.2724923380693 -121.57781265304057) bank179551 +179552 POINT(37.76246562334192 -122.20360018960244) bank179552 +179553 POINT(37.97829578535506 -123.05462091250924) bank179553 +179554 POINT(37.59863563868784 -121.49980267848892) bank179554 +179555 POINT(37.24272364177821 -122.37090680968386) bank179555 +179556 POINT(37.80236134972535 -122.43330643440257) bank179556 +179557 POINT(37.43233397778328 -122.30399965757044) bank179557 +179558 POINT(37.00485593908872 -121.69270752604766) bank179558 +179559 POINT(38.585370244611354 -122.54345125191576) bank179559 +179560 POINT(37.25002396220046 -123.07073386697668) bank179560 +179561 POINT(38.33419749131339 -122.7502717246765) bank179561 +179562 POINT(37.05365548385431 -121.88800713246617) bank179562 +179563 POINT(37.733678692866235 -123.08615093295401) bank179563 +179564 POINT(38.29584131480963 -122.8315278250298) bank179564 +179565 POINT(37.765933959248166 -123.10765983635724) bank179565 +179566 POINT(36.88202053083021 -122.62263548120546) bank179566 +179567 POINT(37.35671837526735 -122.60715523715571) bank179567 +179568 POINT(37.46074313591568 -121.90224144157862) bank179568 +179569 POINT(38.321621879064686 -122.13290540373619) bank179569 +179570 POINT(38.62033384562647 -121.42061152791265) bank179570 +179571 POINT(37.80179735839628 -122.21768241471868) bank179571 +179572 POINT(37.04614540458943 -122.38133009016008) bank179572 +179573 POINT(37.209876935324 -121.5293518508189) bank179573 +179574 POINT(37.285611265811 -122.11509046874181) bank179574 +179575 POINT(37.102668995721416 -121.82689563298828) bank179575 +179576 POINT(38.52750117342954 -123.0835658390231) bank179576 +179577 POINT(37.78614908655795 -122.63196690755024) bank179577 +179578 POINT(37.80796201108183 -122.19278631854053) bank179578 +179579 POINT(38.76486992574662 -121.88439864240657) bank179579 +179580 POINT(38.223955575986835 -122.98605212295517) bank179580 +179581 POINT(38.68689385606113 -121.80608574655972) bank179581 +179582 POINT(37.15106853222933 -122.7007347868051) bank179582 +179583 POINT(37.05806450753609 -121.59154719642977) bank179583 +179584 POINT(37.53290549986537 -122.27709399691302) bank179584 +179585 POINT(38.02391782155924 -122.79204978383828) bank179585 +179586 POINT(38.62166736575215 -122.9122120716746) bank179586 +179587 POINT(37.968081077591606 -122.99724592388417) bank179587 +179588 POINT(37.50922352929193 -122.72624365703973) bank179588 +179589 POINT(38.071343394768874 -122.22410648167428) bank179589 +179590 POINT(37.13973089236455 -123.25613802051174) bank179590 +179591 POINT(37.253022587535156 -123.11484153947973) bank179591 +179592 POINT(37.1676999889959 -122.45464695087124) bank179592 +179593 POINT(38.60913990465122 -122.15425944121495) bank179593 +179594 POINT(36.89700233938571 -122.36828102320688) bank179594 +179595 POINT(38.4939629320158 -121.98055161645831) bank179595 +179596 POINT(37.10729336227767 -123.26926549953443) bank179596 +179597 POINT(36.92522399534562 -121.95334263977863) bank179597 +179598 POINT(36.87297716669707 -122.58271976730016) bank179598 +179599 POINT(37.65381559758969 -122.93027158038944) bank179599 +179600 POINT(38.53443488563126 -122.38146122947954) bank179600 +179601 POINT(38.44317086752611 -121.9996446724089) bank179601 +179602 POINT(38.65198504448487 -123.33170388769776) bank179602 +179603 POINT(38.77149237313381 -123.17163551846717) bank179603 +179604 POINT(37.18693818375583 -123.01017946300568) bank179604 +179605 POINT(37.46650831819569 -122.78849688547044) bank179605 +179606 POINT(37.650363903946364 -122.08524541107126) bank179606 +179607 POINT(38.196817075870335 -122.91624410822946) bank179607 +179608 POINT(37.26998781681097 -121.9202246425712) bank179608 +179609 POINT(38.24133062872019 -121.71651503258808) bank179609 +179610 POINT(36.87317924248315 -122.5107009129127) bank179610 +179611 POINT(38.7144625179758 -123.04722857974005) bank179611 +179612 POINT(37.81557417189438 -123.14332195803566) bank179612 +179613 POINT(36.8645966672603 -121.53680519631875) bank179613 +179614 POINT(37.755501038385916 -122.06346964002131) bank179614 +179615 POINT(37.51640042780818 -121.98092477863375) bank179615 +179616 POINT(37.399414521941694 -122.1000608232789) bank179616 +179617 POINT(37.86809509934208 -121.81052572780365) bank179617 +179618 POINT(38.40173232348165 -121.6460951353695) bank179618 +179619 POINT(37.84183501169191 -121.90871069036626) bank179619 +179620 POINT(37.959922167875405 -122.06163221169646) bank179620 +179621 POINT(37.333813888688226 -121.7252876870158) bank179621 +179622 POINT(37.6872442158341 -123.34944954889748) bank179622 +179623 POINT(37.60573700057636 -121.52974195340721) bank179623 +179624 POINT(37.790668706249996 -122.46264451151428) bank179624 +179625 POINT(37.92384216790269 -122.54813266911702) bank179625 +179626 POINT(38.28612941237123 -121.86097021208688) bank179626 +179627 POINT(38.1785134501066 -122.27615991549486) bank179627 +179628 POINT(37.1915634859817 -122.35644335806246) bank179628 +179629 POINT(38.53653108268435 -122.53366323637267) bank179629 +179630 POINT(37.83281300858185 -122.96291170345037) bank179630 +179631 POINT(37.405612125595496 -123.17342927004479) bank179631 +179632 POINT(37.777568952592944 -122.74234373898925) bank179632 +179633 POINT(37.25615053985801 -123.36932710267658) bank179633 +179634 POINT(38.46494643381104 -122.26034135554322) bank179634 +179635 POINT(37.18377911673604 -121.67067403592029) bank179635 +179636 POINT(37.13944541918002 -121.91048875065397) bank179636 +179637 POINT(38.297657003034196 -122.98378476237814) bank179637 +179638 POINT(38.226802348692836 -123.17255793006287) bank179638 +179639 POINT(37.025668728002316 -122.8951958630354) bank179639 +179640 POINT(38.38513443321571 -121.51684196525277) bank179640 +179641 POINT(37.14236512861927 -122.99181370756052) bank179641 +179642 POINT(36.903700403487775 -122.07775533226328) bank179642 +179643 POINT(38.52321147216906 -122.52818999469577) bank179643 +179644 POINT(37.023081437839224 -122.293167124065) bank179644 +179645 POINT(38.10402496013276 -122.68122260709323) bank179645 +179646 POINT(37.85336623390479 -123.38018384782202) bank179646 +179647 POINT(38.08159183660549 -122.8052746517555) bank179647 +179648 POINT(37.13700119449988 -122.21469982632651) bank179648 +179649 POINT(37.011869473972624 -122.33870092187985) bank179649 +179650 POINT(37.956944550385515 -121.47401482181158) bank179650 +179651 POINT(38.16794678240028 -122.63119711766194) bank179651 +179652 POINT(38.47794084509926 -122.15138103100972) bank179652 +179653 POINT(38.18013113369849 -122.40928642411777) bank179653 +179654 POINT(37.50361962496331 -123.28848605503046) bank179654 +179655 POINT(37.39362768577236 -122.29770454975426) bank179655 +179656 POINT(37.51851340266715 -122.81906206891199) bank179656 +179657 POINT(38.24616661381481 -121.67666398434554) bank179657 +179658 POINT(38.43240278858096 -122.8685554398323) bank179658 +179659 POINT(37.82891216286696 -123.16664182120209) bank179659 +179660 POINT(37.42591375909979 -121.90586533971802) bank179660 +179661 POINT(37.096565671526925 -121.83703252952402) bank179661 +179662 POINT(36.90140826936192 -122.5770694424608) bank179662 +179663 POINT(37.85904431138915 -122.81432710715994) bank179663 +179664 POINT(38.20155017773717 -122.73317311266604) bank179664 +179665 POINT(36.932278374274745 -122.17787136132995) bank179665 +179666 POINT(38.039281442193115 -121.61403551708548) bank179666 +179667 POINT(38.47099418985267 -123.11720792348079) bank179667 +179668 POINT(38.430226665652896 -122.84321385861068) bank179668 +179669 POINT(37.24915715748196 -121.43430205437218) bank179669 +179670 POINT(37.779486035857836 -123.3229436015467) bank179670 +179671 POINT(37.445852088172316 -123.01169797404336) bank179671 +179672 POINT(37.80666497583766 -121.6762311811807) bank179672 +179673 POINT(37.77842725377538 -122.28968654049156) bank179673 +179674 POINT(37.84143347938424 -122.66109432815131) bank179674 +179675 POINT(37.78376089279376 -121.65845509300232) bank179675 +179676 POINT(37.08420563650204 -123.0347531084884) bank179676 +179677 POINT(38.77082159536034 -121.8502436134534) bank179677 +179678 POINT(38.729752807880786 -121.57598950588623) bank179678 +179679 POINT(37.04350670659518 -122.6760480195924) bank179679 +179680 POINT(37.908472758803086 -123.31832274226635) bank179680 +179681 POINT(37.64089614510353 -123.07190020761541) bank179681 +179682 POINT(37.611284388929356 -122.95446914153808) bank179682 +179683 POINT(38.33010511442438 -122.85174728414495) bank179683 +179684 POINT(38.250588983905 -121.42789450590845) bank179684 +179685 POINT(36.981535824017186 -123.12844815604963) bank179685 +179686 POINT(38.74735086298518 -121.87462877494012) bank179686 +179687 POINT(38.03091109581495 -123.18001480215734) bank179687 +179688 POINT(38.373368359838814 -121.93449216903385) bank179688 +179689 POINT(37.876314294028944 -121.8792921566419) bank179689 +179690 POINT(38.515839914018024 -122.99785241076312) bank179690 +179691 POINT(38.33879525280107 -122.77708972218558) bank179691 +179692 POINT(38.41153828115398 -123.34552976604218) bank179692 +179693 POINT(38.38442533917425 -121.95048553688707) bank179693 +179694 POINT(37.33773276232748 -122.15116680614213) bank179694 +179695 POINT(37.3155894683405 -123.31138821596332) bank179695 +179696 POINT(37.430643430502904 -122.8486789554373) bank179696 +179697 POINT(37.21315745906898 -123.24131585598076) bank179697 +179698 POINT(37.116323107226954 -122.85349643273419) bank179698 +179699 POINT(38.55649829409325 -121.86284961784891) bank179699 +179700 POINT(37.63203054187818 -122.75718122142021) bank179700 +179701 POINT(38.402076856771195 -122.86508844312118) bank179701 +179702 POINT(37.97729613435052 -121.58194682761021) bank179702 +179703 POINT(37.36847856513065 -121.97302457197274) bank179703 +179704 POINT(36.793045616429524 -121.91885432813291) bank179704 +179705 POINT(37.62652116745203 -122.22138607345863) bank179705 +179706 POINT(38.3488980200635 -123.39268873109853) bank179706 +179707 POINT(38.023749408504834 -122.01257573955866) bank179707 +179708 POINT(37.968168972032544 -121.5427773180652) bank179708 +179709 POINT(37.75385491664843 -122.82114916747581) bank179709 +179710 POINT(37.49269524737437 -122.89249671929112) bank179710 +179711 POINT(36.813991331678 -122.76558911867606) bank179711 +179712 POINT(38.537545526275956 -121.58885480523368) bank179712 +179713 POINT(38.758292484361526 -122.87050915953738) bank179713 +179714 POINT(38.366329509778794 -121.5262544212743) bank179714 +179715 POINT(36.938622132504584 -122.25613375373153) bank179715 +179716 POINT(37.17770765861049 -121.54154226894816) bank179716 +179717 POINT(37.85237656493626 -121.8856815886632) bank179717 +179718 POINT(38.128661129488286 -121.51004403220445) bank179718 +179719 POINT(36.910003621136475 -122.29496984595882) bank179719 +179720 POINT(38.15633063632519 -121.97949343687934) bank179720 +179721 POINT(38.46622092128541 -121.82307588313795) bank179721 +179722 POINT(38.193970241261724 -123.25448229703214) bank179722 +179723 POINT(38.31309711272384 -121.98171937249404) bank179723 +179724 POINT(38.25302888422119 -122.31672738857581) bank179724 +179725 POINT(38.46945824069407 -122.45989373104221) bank179725 +179726 POINT(37.32000757451888 -121.87921575910484) bank179726 +179727 POINT(37.27125610743983 -122.09013318824493) bank179727 +179728 POINT(36.783760060738 -123.26482808065997) bank179728 +179729 POINT(37.754449666084255 -122.56112468763357) bank179729 +179730 POINT(37.33573451170628 -122.75043850672802) bank179730 +179731 POINT(37.42592064732871 -122.7629638381147) bank179731 +179732 POINT(37.20067545972366 -121.5152167010806) bank179732 +179733 POINT(37.39866451086576 -122.626479685951) bank179733 +179734 POINT(37.7152593618865 -122.12739147522964) bank179734 +179735 POINT(36.79339327201919 -123.23381030613712) bank179735 +179736 POINT(37.94412087398687 -122.2847711404048) bank179736 +179737 POINT(37.41108796370837 -123.32483342117622) bank179737 +179738 POINT(38.21073786318994 -123.15251586935675) bank179738 +179739 POINT(38.244738611263585 -122.39270102615146) bank179739 +179740 POINT(36.79203548291099 -122.29985705644567) bank179740 +179741 POINT(37.91095862300446 -121.53889407421926) bank179741 +179742 POINT(38.23950882564265 -123.38665288413598) bank179742 +179743 POINT(38.05207652895407 -122.27090671225045) bank179743 +179744 POINT(36.84226595942341 -123.08984601459603) bank179744 +179745 POINT(37.94887890356371 -122.90031047736429) bank179745 +179746 POINT(37.10925401731437 -121.78170163193974) bank179746 +179747 POINT(37.01221018068578 -122.35297885090822) bank179747 +179748 POINT(36.79881923618502 -123.39670412269696) bank179748 +179749 POINT(38.461126598511996 -122.42902994784569) bank179749 +179750 POINT(38.32255130433336 -123.39046283128411) bank179750 +179751 POINT(37.89967046849675 -122.87535487006157) bank179751 +179752 POINT(37.88114998897334 -121.70976456054706) bank179752 +179753 POINT(37.85292177208631 -122.47314599342334) bank179753 +179754 POINT(38.63992638292865 -122.82991273735438) bank179754 +179755 POINT(38.008366246980884 -122.23362440339096) bank179755 +179756 POINT(37.50113872950983 -121.66824253660269) bank179756 +179757 POINT(37.84284488357893 -121.92671178653366) bank179757 +179758 POINT(37.512424783074714 -122.47499668175298) bank179758 +179759 POINT(36.86198324353168 -121.91461615582145) bank179759 +179760 POINT(38.643078033395504 -121.94643074664654) bank179760 +179761 POINT(37.74820011430416 -123.01486974417998) bank179761 +179762 POINT(37.77947758072939 -122.64871289195689) bank179762 +179763 POINT(38.01579146239143 -122.24848448377949) bank179763 +179764 POINT(37.565748849219574 -122.62755279677584) bank179764 +179765 POINT(38.50090817731682 -122.21818254215825) bank179765 +179766 POINT(37.5853816282803 -121.94898850863727) bank179766 +179767 POINT(38.733152329147934 -123.12322955661222) bank179767 +179768 POINT(37.27271113878005 -121.4983365527275) bank179768 +179769 POINT(37.77830029070488 -122.35323916749941) bank179769 +179770 POINT(38.1861908177697 -122.29403353813017) bank179770 +179771 POINT(38.307390362418126 -122.07933263298291) bank179771 +179772 POINT(38.47785340567904 -122.52632606891382) bank179772 +179773 POINT(37.44375396206325 -122.51135026036313) bank179773 +179774 POINT(37.630618055661415 -122.32502364849346) bank179774 +179775 POINT(38.046323118133486 -122.04371434231275) bank179775 +179776 POINT(37.96345034239546 -121.87356574556514) bank179776 +179777 POINT(37.78857328307319 -122.44579470042301) bank179777 +179778 POINT(38.410430120801756 -122.4059016648591) bank179778 +179779 POINT(38.72736824988115 -122.82377789280764) bank179779 +179780 POINT(37.78719928112311 -122.55504767636731) bank179780 +179781 POINT(38.41651663954971 -122.94546556666454) bank179781 +179782 POINT(38.15883546054884 -122.34577158455446) bank179782 +179783 POINT(38.090440137391965 -123.28778535633761) bank179783 +179784 POINT(37.569628911348836 -123.35572922675529) bank179784 +179785 POINT(36.86138642832073 -121.4896873490282) bank179785 +179786 POINT(37.731406716889715 -121.73013560225674) bank179786 +179787 POINT(38.02909831162685 -121.71691724090127) bank179787 +179788 POINT(37.36442951155701 -123.34036968035353) bank179788 +179789 POINT(38.21248429966531 -123.38493871400934) bank179789 +179790 POINT(37.271006535266146 -122.8981763776187) bank179790 +179791 POINT(37.293129423920924 -122.7425788971698) bank179791 +179792 POINT(38.40023674493639 -122.66355879358223) bank179792 +179793 POINT(38.34843694363798 -123.2480779120087) bank179793 +179794 POINT(38.533028176098654 -121.56705164585199) bank179794 +179795 POINT(38.1242384535733 -122.9387802900866) bank179795 +179796 POINT(37.71429045663796 -122.33138507393947) bank179796 +179797 POINT(38.71195392863205 -122.06951880967614) bank179797 +179798 POINT(37.35947893626988 -122.52347942245687) bank179798 +179799 POINT(36.960059887614214 -123.36455667467118) bank179799 +179800 POINT(37.90807958210297 -122.81919833955111) bank179800 +179801 POINT(36.848776045553976 -121.6486087979475) bank179801 +179802 POINT(37.65494125203778 -122.27321478961403) bank179802 +179803 POINT(37.855743554057206 -121.92885614880196) bank179803 +179804 POINT(38.374889161863976 -122.09837659442158) bank179804 +179805 POINT(38.255528706475836 -122.2666865858149) bank179805 +179806 POINT(38.192005517271724 -121.71927040411896) bank179806 +179807 POINT(37.830016343383775 -123.06612050629317) bank179807 +179808 POINT(38.6366128017769 -122.89818986277382) bank179808 +179809 POINT(38.22602287538956 -122.5685024384757) bank179809 +179810 POINT(38.42895153163771 -123.30967864971936) bank179810 +179811 POINT(38.067853538240556 -122.43904187113263) bank179811 +179812 POINT(37.03599790263368 -122.0364386445081) bank179812 +179813 POINT(37.810938570858006 -121.54755363082657) bank179813 +179814 POINT(38.4377446387157 -121.71062564546509) bank179814 +179815 POINT(37.5978324860923 -122.98575870515351) bank179815 +179816 POINT(38.53905453519779 -122.81792293595346) bank179816 +179817 POINT(38.63756695649216 -121.8918640230671) bank179817 +179818 POINT(37.07898743687028 -121.69969223987138) bank179818 +179819 POINT(37.541429897828046 -121.88770930894734) bank179819 +179820 POINT(38.56391267102578 -121.43707770614063) bank179820 +179821 POINT(38.18160396057767 -122.37768256372773) bank179821 +179822 POINT(38.57679278428206 -122.78395972320766) bank179822 +179823 POINT(37.83350863912014 -121.58292968356267) bank179823 +179824 POINT(38.60723592970113 -123.12730171929468) bank179824 +179825 POINT(36.91554755554543 -121.9883772526251) bank179825 +179826 POINT(37.09827267818165 -121.465590879731) bank179826 +179827 POINT(38.6045263794269 -122.35810507939294) bank179827 +179828 POINT(37.11320914299311 -123.18081897850386) bank179828 +179829 POINT(36.79309932122007 -122.92462940992746) bank179829 +179830 POINT(38.74032488613039 -122.2529037739903) bank179830 +179831 POINT(37.6201506131973 -123.24799831569665) bank179831 +179832 POINT(38.29285309960474 -122.87295387661753) bank179832 +179833 POINT(38.19242740758776 -121.60151101145699) bank179833 +179834 POINT(36.77779613625241 -122.11962117303328) bank179834 +179835 POINT(38.578804828767886 -123.13025645050728) bank179835 +179836 POINT(37.19731010931649 -122.85499109139282) bank179836 +179837 POINT(37.19783571410589 -121.8830329612658) bank179837 +179838 POINT(38.25883113507171 -123.11541526118872) bank179838 +179839 POINT(38.06092886593335 -123.30849903770846) bank179839 +179840 POINT(37.10698989963371 -123.25474597158428) bank179840 +179841 POINT(37.879400521564214 -122.20559100937808) bank179841 +179842 POINT(37.32586690700549 -121.53058335288472) bank179842 +179843 POINT(38.355378854709855 -122.08604957441719) bank179843 +179844 POINT(37.65115089757156 -122.4447534335076) bank179844 +179845 POINT(38.31975214147867 -123.11660182256264) bank179845 +179846 POINT(37.854666834214356 -121.68635604099026) bank179846 +179847 POINT(37.24611853143188 -122.58137292590442) bank179847 +179848 POINT(37.83215261526429 -122.89180071681115) bank179848 +179849 POINT(37.95163768866306 -123.00028571368298) bank179849 +179850 POINT(37.75148497413546 -121.9635686065612) bank179850 +179851 POINT(38.49591057564373 -122.94222902950344) bank179851 +179852 POINT(37.120569629827656 -121.99882954377507) bank179852 +179853 POINT(37.80654572269071 -122.69152354926207) bank179853 +179854 POINT(38.4721531753567 -122.20274151564102) bank179854 +179855 POINT(36.9447393562498 -121.538167020457) bank179855 +179856 POINT(37.33098263115411 -122.03068542465839) bank179856 +179857 POINT(37.509970520031715 -123.22393629697777) bank179857 +179858 POINT(37.40870924240975 -121.79594900136873) bank179858 +179859 POINT(37.59675858757651 -122.33457268843148) bank179859 +179860 POINT(38.452504807755986 -121.52442217054043) bank179860 +179861 POINT(38.03265697215054 -121.45398935902206) bank179861 +179862 POINT(38.494605309999095 -121.72648125671887) bank179862 +179863 POINT(36.99023755761597 -122.79866817019254) bank179863 +179864 POINT(37.01066920957839 -121.99441923426858) bank179864 +179865 POINT(37.12498548680785 -122.73320665763802) bank179865 +179866 POINT(38.41247506986592 -121.57463883895922) bank179866 +179867 POINT(37.832653252051486 -122.9469667449956) bank179867 +179868 POINT(38.3875246330796 -122.51048188366194) bank179868 +179869 POINT(37.90507780021105 -123.19700529322559) bank179869 +179870 POINT(38.04446599436703 -121.99965718682826) bank179870 +179871 POINT(37.45515557928776 -123.15353771485435) bank179871 +179872 POINT(36.90095278445782 -122.70643712238407) bank179872 +179873 POINT(37.70235220439159 -121.92150828531447) bank179873 +179874 POINT(37.84438582501605 -123.32750339308006) bank179874 +179875 POINT(37.85023089191586 -121.77618389912593) bank179875 +179876 POINT(37.603578870366164 -122.84318303216101) bank179876 +179877 POINT(36.83411083476564 -121.71004520868178) bank179877 +179878 POINT(36.829873905844636 -122.48201545624711) bank179878 +179879 POINT(37.23069929298729 -121.85175658207025) bank179879 +179880 POINT(37.301688869994095 -122.0190571355917) bank179880 +179881 POINT(38.65921488127787 -122.19547270611919) bank179881 +179882 POINT(38.02967520972604 -123.07473972771382) bank179882 +179883 POINT(37.72596912897409 -122.8203229015143) bank179883 +179884 POINT(37.8366534166696 -122.99504248564922) bank179884 +179885 POINT(37.774600510122674 -121.42147304502356) bank179885 +179886 POINT(38.751414142910164 -121.94503572730719) bank179886 +179887 POINT(37.215571487462746 -122.88412362897319) bank179887 +179888 POINT(37.118086099449044 -122.2197458822749) bank179888 +179889 POINT(38.67959496484207 -122.80491949059393) bank179889 +179890 POINT(38.119963218130714 -123.02120840673163) bank179890 +179891 POINT(36.86646652197213 -123.24832996789627) bank179891 +179892 POINT(36.81809303467457 -122.7558630726272) bank179892 +179893 POINT(37.9886037921014 -123.07100404787744) bank179893 +179894 POINT(38.25156647514298 -122.97792763400417) bank179894 +179895 POINT(38.35778116993485 -123.39321135869045) bank179895 +179896 POINT(37.17586640580031 -123.2594470380202) bank179896 +179897 POINT(37.80302299662676 -122.85747615226279) bank179897 +179898 POINT(37.049069741794575 -122.2088563111186) bank179898 +179899 POINT(37.953217237584404 -122.72590041968358) bank179899 +179900 POINT(37.60960574601642 -121.64712982609262) bank179900 +179901 POINT(37.12270923946757 -122.04276127769879) bank179901 +179902 POINT(38.7734808875718 -122.4964650603878) bank179902 +179903 POINT(36.7924718939862 -123.1193239051319) bank179903 +179904 POINT(37.68432132513861 -122.79557168382095) bank179904 +179905 POINT(37.56381821568658 -121.49625068357832) bank179905 +179906 POINT(37.075358484080155 -122.45615051159186) bank179906 +179907 POINT(37.35792087621178 -121.87645586349794) bank179907 +179908 POINT(37.07897129255737 -122.97834143549639) bank179908 +179909 POINT(38.09313386362833 -122.47634407044491) bank179909 +179910 POINT(38.070039121608986 -122.95395286405228) bank179910 +179911 POINT(37.94287802571357 -122.5449436627522) bank179911 +179912 POINT(37.12306858342864 -122.5686931628542) bank179912 +179913 POINT(38.414060669856845 -122.2732262012398) bank179913 +179914 POINT(37.11041933417009 -121.79091621547428) bank179914 +179915 POINT(37.854601301574256 -123.13837628278787) bank179915 +179916 POINT(37.939967682508026 -122.5225305853688) bank179916 +179917 POINT(37.52232425421068 -121.81886818018768) bank179917 +179918 POINT(37.26418858706507 -121.93148069146623) bank179918 +179919 POINT(38.59284976727752 -123.25557320812393) bank179919 +179920 POINT(37.49597230443398 -122.6317853770049) bank179920 +179921 POINT(37.31617074373979 -122.83893521376825) bank179921 +179922 POINT(38.06203790747338 -123.35733085320872) bank179922 +179923 POINT(37.40202020251465 -121.52686533992892) bank179923 +179924 POINT(37.89454582154347 -122.68382775461046) bank179924 +179925 POINT(37.92319642138034 -123.15111358936977) bank179925 +179926 POINT(36.810659413526054 -123.26333030736126) bank179926 +179927 POINT(37.519267031523576 -121.94769566081038) bank179927 +179928 POINT(37.970467189720246 -121.87624048370206) bank179928 +179929 POINT(38.26023830468155 -122.90931750600156) bank179929 +179930 POINT(38.0432848414931 -122.11899145964074) bank179930 +179931 POINT(37.58187415110645 -122.37909962280209) bank179931 +179932 POINT(38.00774827657174 -122.38959620988645) bank179932 +179933 POINT(37.21525918587446 -122.10969408105504) bank179933 +179934 POINT(37.711663227498256 -122.17225943861705) bank179934 +179935 POINT(37.09866279192271 -122.55937134206953) bank179935 +179936 POINT(38.68125814050589 -122.52573126446465) bank179936 +179937 POINT(37.86970376778951 -121.84575961250395) bank179937 +179938 POINT(38.2880734870839 -122.69054704992266) bank179938 +179939 POINT(37.9500674645636 -122.8541192531558) bank179939 +179940 POINT(38.02156809296378 -122.24794636640665) bank179940 +179941 POINT(38.459842341626846 -123.02814790950498) bank179941 +179942 POINT(37.31607789990796 -122.06778444621203) bank179942 +179943 POINT(36.98834200033591 -122.18818743037255) bank179943 +179944 POINT(36.98175246700808 -122.03506462450434) bank179944 +179945 POINT(36.87487762190884 -122.33095908785185) bank179945 +179946 POINT(38.42038073284931 -121.83350014710454) bank179946 +179947 POINT(38.614602523452554 -123.30741120519926) bank179947 +179948 POINT(38.0715124272826 -121.7852989806437) bank179948 +179949 POINT(38.330369244972545 -121.83967953978585) bank179949 +179950 POINT(38.039748486808726 -123.29269488750451) bank179950 +179951 POINT(37.935543545455516 -122.67638029024852) bank179951 +179952 POINT(36.81937757728551 -121.70594441309728) bank179952 +179953 POINT(37.91417325427276 -122.01123603074706) bank179953 +179954 POINT(38.15704911501606 -121.84178660604235) bank179954 +179955 POINT(38.29607967263683 -121.598213296785) bank179955 +179956 POINT(38.13585327659825 -122.18824295406999) bank179956 +179957 POINT(38.20911549342688 -122.92947985300131) bank179957 +179958 POINT(37.48792523666593 -122.32674636884693) bank179958 +179959 POINT(38.75099234280707 -121.66273019955135) bank179959 +179960 POINT(37.574816980052155 -122.23865903259173) bank179960 +179961 POINT(38.72528786457905 -122.2329206876002) bank179961 +179962 POINT(38.53431870173146 -121.8396799575284) bank179962 +179963 POINT(37.524006257271026 -123.32931325595148) bank179963 +179964 POINT(38.667270533250175 -123.21921705181948) bank179964 +179965 POINT(38.221100530091775 -122.47429212701735) bank179965 +179966 POINT(37.80804167926638 -122.26759817834804) bank179966 +179967 POINT(36.90628980562268 -122.09820456030343) bank179967 +179968 POINT(38.1843956741257 -122.59531566129992) bank179968 +179969 POINT(37.79579767247677 -123.06493953715855) bank179969 +179970 POINT(38.07181110375881 -122.85880996128473) bank179970 +179971 POINT(37.9402243763204 -123.269940301251) bank179971 +179972 POINT(38.65549307141928 -123.3144755072318) bank179972 +179973 POINT(38.760785880422404 -121.5085547581032) bank179973 +179974 POINT(37.70058248283482 -122.05212379792255) bank179974 +179975 POINT(38.468810964607464 -122.03294557357682) bank179975 +179976 POINT(37.88583322798424 -123.29577824122002) bank179976 +179977 POINT(37.214252688496494 -123.2678076869395) bank179977 +179978 POINT(37.99776324326113 -121.47686843416902) bank179978 +179979 POINT(38.012762773849445 -122.06275131114313) bank179979 +179980 POINT(37.28267497549575 -123.41756040739469) bank179980 +179981 POINT(37.766479588943405 -121.6879246072368) bank179981 +179982 POINT(36.89037359473484 -121.69912142900641) bank179982 +179983 POINT(36.906676990996864 -123.11736916218155) bank179983 +179984 POINT(38.75066557129499 -121.71117040364003) bank179984 +179985 POINT(38.47861173827256 -122.48829643845393) bank179985 +179986 POINT(37.90906682183319 -121.883396962104) bank179986 +179987 POINT(38.212050884169756 -122.48925928370151) bank179987 +179988 POINT(36.83467893306254 -122.96733183421561) bank179988 +179989 POINT(38.503697185975696 -122.26809641668291) bank179989 +179990 POINT(36.913831523378455 -122.87519306713024) bank179990 +179991 POINT(37.18298356318667 -122.73343240419659) bank179991 +179992 POINT(37.185833195494496 -122.51324710287957) bank179992 +179993 POINT(36.913581491030406 -121.74212174084789) bank179993 +179994 POINT(37.33356088735432 -122.0434751591971) bank179994 +179995 POINT(37.44279722815536 -122.40051329177557) bank179995 +179996 POINT(38.388634110262124 -122.83374058145046) bank179996 +179997 POINT(38.41004295812922 -123.06965601297242) bank179997 +179998 POINT(37.80980425118099 -121.49501553902844) bank179998 +179999 POINT(38.00786674240307 -122.11624306046224) bank179999 +180000 POINT(37.634336469479976 -122.9677802788772) bank180000 +180001 POINT(38.672461097793175 -121.90717364067955) bank180001 +180002 POINT(38.6474758156051 -123.00585318953893) bank180002 +180003 POINT(37.56577252870382 -122.20429340444035) bank180003 +180004 POINT(37.15829555159114 -122.88814068528836) bank180004 +180005 POINT(37.84064722464255 -122.85419992766114) bank180005 +180006 POINT(37.9076882627172 -121.45568856045057) bank180006 +180007 POINT(38.450806419644294 -122.50455077675122) bank180007 +180008 POINT(37.81291125956095 -121.49182675482503) bank180008 +180009 POINT(37.47424509875686 -122.90943141448906) bank180009 +180010 POINT(37.3921178786723 -121.60049755995851) bank180010 +180011 POINT(37.77696663973856 -122.16031976429119) bank180011 +180012 POINT(37.85488650129029 -121.64323797817897) bank180012 +180013 POINT(37.022341194074606 -121.98445331315821) bank180013 +180014 POINT(37.08480332428873 -123.050829577082) bank180014 +180015 POINT(37.8848885009283 -122.59426607507697) bank180015 +180016 POINT(37.7801475415448 -121.9014773735959) bank180016 +180017 POINT(37.658088224322 -121.94222724693644) bank180017 +180018 POINT(36.912596303097075 -121.92113506940728) bank180018 +180019 POINT(37.129767019246195 -121.7979210851828) bank180019 +180020 POINT(38.49144453551655 -121.44205092865677) bank180020 +180021 POINT(38.42979584508412 -122.61929831349661) bank180021 +180022 POINT(38.15884333167445 -122.53363012486855) bank180022 +180023 POINT(38.327976745721315 -121.55864907882902) bank180023 +180024 POINT(38.00959523741153 -122.05767660198819) bank180024 +180025 POINT(37.14341922341739 -123.32870481510581) bank180025 +180026 POINT(38.581781690045446 -122.43973726662522) bank180026 +180027 POINT(37.474523999666204 -123.36872351147919) bank180027 +180028 POINT(36.93063551684647 -122.31399939724399) bank180028 +180029 POINT(38.65072645627433 -122.84882801762691) bank180029 +180030 POINT(36.890584300600615 -121.7298720677083) bank180030 +180031 POINT(37.45695271492706 -123.14852817143327) bank180031 +180032 POINT(37.647401893218905 -123.05251966460503) bank180032 +180033 POINT(37.633298723652125 -122.84928390170592) bank180033 +180034 POINT(37.91047892496177 -121.93423032847757) bank180034 +180035 POINT(37.30955277206884 -121.8736405709642) bank180035 +180036 POINT(38.40929536190819 -121.8247014082423) bank180036 +180037 POINT(38.32710870817861 -121.55360719171897) bank180037 +180038 POINT(37.878743502853176 -122.52349807340116) bank180038 +180039 POINT(37.8062058870359 -122.93945457391175) bank180039 +180040 POINT(38.6738313478553 -122.0205055420958) bank180040 +180041 POINT(38.704630714313815 -123.12667023545383) bank180041 +180042 POINT(38.218167291759805 -121.76374681837194) bank180042 +180043 POINT(38.25240428044369 -122.39104873179829) bank180043 +180044 POINT(37.63789715796892 -121.87541096128058) bank180044 +180045 POINT(37.78949546018357 -123.22341109677906) bank180045 +180046 POINT(37.870335514461615 -122.47127018793225) bank180046 +180047 POINT(38.671785475200785 -122.5590925876505) bank180047 +180048 POINT(38.28913487850576 -122.49908201692521) bank180048 +180049 POINT(38.34097791875813 -122.93365843373897) bank180049 +180050 POINT(37.92126419824572 -122.45182942476042) bank180050 +180051 POINT(36.850411268633664 -122.39163984931845) bank180051 +180052 POINT(37.25742112758887 -122.42469907619326) bank180052 +180053 POINT(38.34541054937142 -122.64337042007224) bank180053 +180054 POINT(36.90167080035041 -123.22680517298089) bank180054 +180055 POINT(38.42489618090822 -122.403518130934) bank180055 +180056 POINT(37.135028569277615 -123.31402388368912) bank180056 +180057 POINT(37.097657069837666 -123.2721918424504) bank180057 +180058 POINT(36.852346294337764 -123.38154009979759) bank180058 +180059 POINT(37.61265183608786 -121.7139198405258) bank180059 +180060 POINT(38.2947424241184 -122.62426797626163) bank180060 +180061 POINT(36.89139166214841 -121.7545623290968) bank180061 +180062 POINT(37.717986689215394 -122.50145681729879) bank180062 +180063 POINT(37.689395577357566 -122.01504452106985) bank180063 +180064 POINT(38.67705124656129 -121.93700661770025) bank180064 +180065 POINT(37.88623137519971 -123.20960707261469) bank180065 +180066 POINT(38.37916649973028 -122.73989233007188) bank180066 +180067 POINT(38.503598028953895 -122.09942293851921) bank180067 +180068 POINT(38.2966519253279 -122.37460596587158) bank180068 +180069 POINT(38.54137124124802 -122.06213845618568) bank180069 +180070 POINT(37.94351931316528 -122.46702761048228) bank180070 +180071 POINT(38.393655750996224 -122.76596629369483) bank180071 +180072 POINT(37.82645272301817 -121.75841879132307) bank180072 +180073 POINT(38.64892073191353 -122.61532015218852) bank180073 +180074 POINT(38.501749206017756 -122.60579835104824) bank180074 +180075 POINT(37.05695194735244 -122.09742142110936) bank180075 +180076 POINT(37.41259273474841 -123.29806824606158) bank180076 +180077 POINT(36.93382464846962 -122.6398375298536) bank180077 +180078 POINT(37.47053637069853 -123.292639168594) bank180078 +180079 POINT(37.402668814137634 -122.81898106935677) bank180079 +180080 POINT(36.97232921956898 -121.88066180772448) bank180080 +180081 POINT(37.81638297006823 -122.31748498074143) bank180081 +180082 POINT(38.35296311333054 -121.70094222087492) bank180082 +180083 POINT(38.71833772809212 -123.37241691378622) bank180083 +180084 POINT(37.16912343445218 -121.45752834889133) bank180084 +180085 POINT(38.086252304026736 -121.92178798690641) bank180085 +180086 POINT(38.197605852235704 -121.87181676187741) bank180086 +180087 POINT(37.327945273145865 -121.97306879198143) bank180087 +180088 POINT(38.71950488579681 -121.97219214678624) bank180088 +180089 POINT(38.47787408663576 -123.1243196719918) bank180089 +180090 POINT(38.293689085678785 -123.27705181852963) bank180090 +180091 POINT(37.626423424710495 -123.24274965893473) bank180091 +180092 POINT(38.714049940521605 -122.4350872765291) bank180092 +180093 POINT(37.881996549961286 -121.89074260181027) bank180093 +180094 POINT(38.29204512613084 -123.16055890397283) bank180094 +180095 POINT(38.657393918015686 -122.75900924160835) bank180095 +180096 POINT(36.90114358583864 -122.12472222090567) bank180096 +180097 POINT(36.81656663150191 -122.75796391034261) bank180097 +180098 POINT(36.951209463065595 -123.0539230947967) bank180098 +180099 POINT(38.697758441075365 -123.06435388855928) bank180099 +180100 POINT(38.49661250541281 -122.14484126209538) bank180100 +180101 POINT(37.181365950968974 -122.28185231571841) bank180101 +180102 POINT(36.9906457118104 -122.87687344569075) bank180102 +180103 POINT(37.47610422282521 -123.02288385712666) bank180103 +180104 POINT(37.17987026354972 -121.59986472829081) bank180104 +180105 POINT(37.13319837886891 -121.49039717915193) bank180105 +180106 POINT(38.681157974769405 -122.06556566706507) bank180106 +180107 POINT(36.91860212138624 -122.83436165636282) bank180107 +180108 POINT(37.282527933487906 -122.07704588893498) bank180108 +180109 POINT(37.6633711554763 -121.60195421134044) bank180109 +180110 POINT(37.7005833557537 -121.47541996658275) bank180110 +180111 POINT(36.78639427470682 -121.79266883484776) bank180111 +180112 POINT(38.61393162682353 -121.83546416016102) bank180112 +180113 POINT(37.84067623515963 -123.34820647306324) bank180113 +180114 POINT(37.08305684477733 -121.62421597568553) bank180114 +180115 POINT(37.93745580428949 -123.06908299814891) bank180115 +180116 POINT(37.73098576760251 -121.84261983272259) bank180116 +180117 POINT(38.695646545688085 -122.68802668565239) bank180117 +180118 POINT(38.2269656953647 -121.90602579944398) bank180118 +180119 POINT(37.52314468751489 -121.97323265332562) bank180119 +180120 POINT(37.17863322398223 -122.93954458506929) bank180120 +180121 POINT(36.83384701430405 -121.46695523171554) bank180121 +180122 POINT(37.07814717753249 -122.67043637945183) bank180122 +180123 POINT(38.481971679871556 -121.88885393789103) bank180123 +180124 POINT(37.96764108925263 -123.37218354688828) bank180124 +180125 POINT(38.134731883938656 -122.09439564440497) bank180125 +180126 POINT(37.4433633676997 -123.41018033462775) bank180126 +180127 POINT(38.64514598671702 -122.16285423932094) bank180127 +180128 POINT(37.30098015659344 -122.8598672231988) bank180128 +180129 POINT(38.371574029397415 -122.2651501223323) bank180129 +180130 POINT(37.24814994392358 -121.45245493369524) bank180130 +180131 POINT(37.94311283663302 -122.9756946232114) bank180131 +180132 POINT(37.01675042668735 -121.96271738612569) bank180132 +180133 POINT(37.434868288620734 -121.5461326315173) bank180133 +180134 POINT(38.37835050656264 -122.06673412634035) bank180134 +180135 POINT(37.4075505529611 -122.66658126539627) bank180135 +180136 POINT(37.96351674422577 -121.53518203821687) bank180136 +180137 POINT(38.552692914118815 -123.03122064393888) bank180137 +180138 POINT(38.5997178228821 -122.38399605399911) bank180138 +180139 POINT(37.38122391386995 -122.5319501018592) bank180139 +180140 POINT(37.689602265502714 -122.56068680456318) bank180140 +180141 POINT(37.006908047678664 -121.90807680738027) bank180141 +180142 POINT(37.25109896133416 -123.19409239963385) bank180142 +180143 POINT(37.319350140391855 -122.85818448269792) bank180143 +180144 POINT(37.352398704205065 -122.82061674437742) bank180144 +180145 POINT(38.26320814805142 -121.90314270563854) bank180145 +180146 POINT(37.06251831703627 -122.80143839387998) bank180146 +180147 POINT(36.80396767557164 -121.65929720452206) bank180147 +180148 POINT(36.94039126250618 -123.18658949418912) bank180148 +180149 POINT(38.20931839029983 -122.86170999424446) bank180149 +180150 POINT(36.96552888597542 -121.69074847100083) bank180150 +180151 POINT(37.02178436567445 -122.85617593221662) bank180151 +180152 POINT(38.5321809793193 -122.97535528568008) bank180152 +180153 POINT(38.06871373903695 -121.92973057969128) bank180153 +180154 POINT(38.57653838207123 -122.12016210692512) bank180154 +180155 POINT(38.21160792247696 -122.17501935414066) bank180155 +180156 POINT(38.22164327606108 -123.04034741300201) bank180156 +180157 POINT(38.02565823525591 -122.70179392892189) bank180157 +180158 POINT(37.22463367069742 -123.11736408932599) bank180158 +180159 POINT(38.3883497302 -122.12207864590992) bank180159 +180160 POINT(38.3799023378506 -121.77581485082995) bank180160 +180161 POINT(37.58041707371728 -122.3241283847108) bank180161 +180162 POINT(37.484963896817604 -122.02859360278846) bank180162 +180163 POINT(38.59897687754529 -121.79331051821606) bank180163 +180164 POINT(38.442591428822446 -121.5587810074302) bank180164 +180165 POINT(37.55637454255241 -121.42406668708705) bank180165 +180166 POINT(37.16795900572831 -122.56962053709479) bank180166 +180167 POINT(37.33547724350844 -123.21872537948947) bank180167 +180168 POINT(37.624119930766824 -122.7612949724672) bank180168 +180169 POINT(38.65504438172273 -121.46346895914554) bank180169 +180170 POINT(37.022791760230476 -122.89890336975603) bank180170 +180171 POINT(37.96249059739824 -122.7770915927231) bank180171 +180172 POINT(37.39194917881542 -121.43558014416492) bank180172 +180173 POINT(38.766746927305014 -121.43246234632765) bank180173 +180174 POINT(38.419856466542015 -122.07550453545333) bank180174 +180175 POINT(36.83843058612281 -122.18719753269421) bank180175 +180176 POINT(36.84409252416867 -121.43079930998182) bank180176 +180177 POINT(38.42952348539153 -121.88566546437141) bank180177 +180178 POINT(38.62887484591178 -122.28765185118576) bank180178 +180179 POINT(38.433742782226474 -122.34553705971557) bank180179 +180180 POINT(38.38321088015365 -123.05290155851061) bank180180 +180181 POINT(37.11336140232827 -123.14125085180137) bank180181 +180182 POINT(37.372165515195434 -122.48618948840752) bank180182 +180183 POINT(37.58605477866902 -122.93484429579067) bank180183 +180184 POINT(38.69377417382147 -121.45733926786512) bank180184 +180185 POINT(37.251632525571274 -122.73159435267853) bank180185 +180186 POINT(37.859015919203024 -122.09725776629098) bank180186 +180187 POINT(37.57600062013512 -121.84444321589298) bank180187 +180188 POINT(37.33605942045235 -121.83605921593023) bank180188 +180189 POINT(36.81427470522637 -121.67799475443562) bank180189 +180190 POINT(38.05949703249325 -122.58772395948169) bank180190 +180191 POINT(38.36529239023257 -121.76809057063466) bank180191 +180192 POINT(37.726756378675034 -122.69493015429667) bank180192 +180193 POINT(37.01201123021476 -121.52469495535446) bank180193 +180194 POINT(38.68310772160971 -123.18880899640541) bank180194 +180195 POINT(37.96453566865404 -122.77172910491414) bank180195 +180196 POINT(38.59664580919967 -121.85840701464618) bank180196 +180197 POINT(37.630416369167 -123.14971206586337) bank180197 +180198 POINT(36.98628192374322 -122.29401710475193) bank180198 +180199 POINT(37.71513929519201 -123.41126164873373) bank180199 +180200 POINT(38.651635716747585 -123.16474119293612) bank180200 +180201 POINT(38.47721269273656 -121.97438140284846) bank180201 +180202 POINT(37.58035267245313 -122.14591655537055) bank180202 +180203 POINT(37.698012850227464 -121.8511496389275) bank180203 +180204 POINT(37.15334904364584 -123.0922731857891) bank180204 +180205 POINT(37.14076405985851 -123.2171394706933) bank180205 +180206 POINT(38.437718982247645 -121.51383675294511) bank180206 +180207 POINT(37.27246340083151 -122.66902398379173) bank180207 +180208 POINT(38.593023784447325 -122.56267744821392) bank180208 +180209 POINT(38.20579298752997 -121.71782682589158) bank180209 +180210 POINT(38.14723259563276 -121.74680213331223) bank180210 +180211 POINT(37.399969603487115 -122.43026014808981) bank180211 +180212 POINT(37.928431501416355 -122.10160721026841) bank180212 +180213 POINT(38.5381902488327 -121.89950881990895) bank180213 +180214 POINT(38.045249213382085 -123.26788183707691) bank180214 +180215 POINT(38.61644833684609 -121.65393195460466) bank180215 +180216 POINT(38.3550257888203 -123.24554007955551) bank180216 +180217 POINT(37.56679838949813 -123.31008258638498) bank180217 +180218 POINT(37.79186733322463 -121.9063822547284) bank180218 +180219 POINT(37.062021631309996 -122.20429238743732) bank180219 +180220 POINT(38.67847470957898 -122.33896321020872) bank180220 +180221 POINT(37.135071007782535 -121.91974187995002) bank180221 +180222 POINT(37.92231946546952 -122.82153856024898) bank180222 +180223 POINT(38.747307324338784 -122.99714635339512) bank180223 +180224 POINT(37.538552710242534 -123.33307599125986) bank180224 +180225 POINT(37.44965114477673 -123.2377502197221) bank180225 +180226 POINT(38.614383367603985 -122.43651208374233) bank180226 +180227 POINT(37.72815160180942 -122.90086785076335) bank180227 +180228 POINT(37.265449916220966 -121.57594703336969) bank180228 +180229 POINT(37.389986235244095 -123.14267076702555) bank180229 +180230 POINT(38.37193696685194 -121.82574373297233) bank180230 +180231 POINT(38.47859959986386 -121.70092266506734) bank180231 +180232 POINT(37.7259082435191 -121.48903813539681) bank180232 +180233 POINT(38.574083099649194 -121.84704669167286) bank180233 +180234 POINT(38.45153549822698 -122.72323172126188) bank180234 +180235 POINT(38.16206428185701 -122.46415556871814) bank180235 +180236 POINT(38.76991498354693 -122.9289693641076) bank180236 +180237 POINT(37.43791330434263 -121.74603376834821) bank180237 +180238 POINT(37.98182816344413 -121.97785715076516) bank180238 +180239 POINT(37.791160677062265 -123.26820135394459) bank180239 +180240 POINT(37.15110690216309 -123.19982127330235) bank180240 +180241 POINT(36.793130028681134 -121.72140295686968) bank180241 +180242 POINT(38.076873355142624 -123.07059000116166) bank180242 +180243 POINT(38.096405753556894 -121.79396628920242) bank180243 +180244 POINT(38.29314067552042 -122.27613147832047) bank180244 +180245 POINT(37.4895393031031 -122.5291246060699) bank180245 +180246 POINT(38.0170817113231 -121.90923279504196) bank180246 +180247 POINT(37.51003222112318 -123.16886457249534) bank180247 +180248 POINT(38.175568762825286 -123.25062196921834) bank180248 +180249 POINT(37.81677441101986 -122.89448140260969) bank180249 +180250 POINT(38.05135922313318 -122.09459063274483) bank180250 +180251 POINT(38.062199178598725 -122.06394462185499) bank180251 +180252 POINT(36.7899964171709 -123.23038839128056) bank180252 +180253 POINT(36.79182418874383 -123.02525367129299) bank180253 +180254 POINT(38.70590089915865 -122.27545323120704) bank180254 +180255 POINT(38.29921502006039 -123.35763917606195) bank180255 +180256 POINT(37.407280102146814 -121.67215574244675) bank180256 +180257 POINT(38.46381264005663 -122.29369694908623) bank180257 +180258 POINT(38.74564121299767 -123.10335030901891) bank180258 +180259 POINT(37.00387195385877 -121.49522736409364) bank180259 +180260 POINT(38.10007080927049 -122.40619587284256) bank180260 +180261 POINT(38.23931113291941 -122.9996488184584) bank180261 +180262 POINT(36.93086207471078 -121.70898278780619) bank180262 +180263 POINT(37.03500806470515 -121.8197304451318) bank180263 +180264 POINT(37.70543429824484 -123.22131652127833) bank180264 +180265 POINT(37.285444931465854 -121.70226554856679) bank180265 +180266 POINT(38.134224963601184 -122.48404185031494) bank180266 +180267 POINT(36.83689440016332 -122.42177770653564) bank180267 +180268 POINT(37.91480395533894 -123.07070125777678) bank180268 +180269 POINT(38.436092556429344 -121.93497931724593) bank180269 +180270 POINT(38.20791965133391 -123.4153227629772) bank180270 +180271 POINT(38.67018804301698 -121.48077822156291) bank180271 +180272 POINT(38.23377957357987 -122.80742570895991) bank180272 +180273 POINT(38.13423788326781 -122.41072271438019) bank180273 +180274 POINT(37.35745362108186 -121.60251626921264) bank180274 +180275 POINT(38.303348197573285 -123.38395257172644) bank180275 +180276 POINT(37.67281857047458 -122.79890397081077) bank180276 +180277 POINT(37.318746154044625 -122.8806598454938) bank180277 +180278 POINT(37.9545497030704 -121.89353051983093) bank180278 +180279 POINT(38.14866892674507 -123.26261282954815) bank180279 +180280 POINT(38.09688335215307 -121.83368667342924) bank180280 +180281 POINT(38.759793596389976 -122.25180477343024) bank180281 +180282 POINT(38.53948676306481 -123.25292786637496) bank180282 +180283 POINT(36.81800179613848 -123.30955978479315) bank180283 +180284 POINT(38.24462903741877 -122.76504993301185) bank180284 +180285 POINT(37.268600510442745 -123.15606823840496) bank180285 +180286 POINT(38.28896568761996 -123.19586119459444) bank180286 +180287 POINT(38.389295009138976 -122.9325143168023) bank180287 +180288 POINT(38.0537390126521 -121.62641363844872) bank180288 +180289 POINT(38.255110272938936 -121.918238945896) bank180289 +180290 POINT(36.996113146529645 -122.91619551203034) bank180290 +180291 POINT(37.262111834274734 -122.90302534786483) bank180291 +180292 POINT(38.35798706473165 -121.53362123271712) bank180292 +180293 POINT(36.93507321562034 -121.49119988570672) bank180293 +180294 POINT(36.96252892087513 -122.35121570816608) bank180294 +180295 POINT(38.670666060489424 -122.96914115808202) bank180295 +180296 POINT(37.04824386837605 -123.10708687736523) bank180296 +180297 POINT(38.17334688294537 -122.62082169508136) bank180297 +180298 POINT(38.34230188802892 -121.77677300233998) bank180298 +180299 POINT(37.39998249132618 -121.99386500244154) bank180299 +180300 POINT(38.443819465850424 -122.00615359627588) bank180300 +180301 POINT(37.486345087627896 -122.75569211263084) bank180301 +180302 POINT(38.755453823637126 -123.05676917881655) bank180302 +180303 POINT(38.44650766674862 -122.17782291518519) bank180303 +180304 POINT(38.73037575308992 -122.65763447411418) bank180304 +180305 POINT(38.36422088092364 -121.83083730633705) bank180305 +180306 POINT(37.920106362450795 -121.92250572528525) bank180306 +180307 POINT(38.09093282649483 -122.82405554054549) bank180307 +180308 POINT(37.08625251517298 -122.5073255280831) bank180308 +180309 POINT(38.479088029933756 -121.72753404249328) bank180309 +180310 POINT(38.05861798226497 -121.72368221818535) bank180310 +180311 POINT(38.67136639304285 -123.1743209001219) bank180311 +180312 POINT(38.06087676007325 -122.0053464302679) bank180312 +180313 POINT(38.33901528202281 -122.5314173144114) bank180313 +180314 POINT(37.76302650991048 -122.57762565129046) bank180314 +180315 POINT(37.31807763086155 -121.8753098993049) bank180315 +180316 POINT(36.79054491952071 -121.66215543090674) bank180316 +180317 POINT(38.165799266962885 -122.96977541379918) bank180317 +180318 POINT(37.65537516869005 -122.51178696303884) bank180318 +180319 POINT(38.03877437675239 -122.31857921530143) bank180319 +180320 POINT(38.44647230507738 -121.66719412526393) bank180320 +180321 POINT(38.53663741895176 -121.70392479412595) bank180321 +180322 POINT(37.78603250452215 -122.30152105000973) bank180322 +180323 POINT(37.459774659906905 -123.14605134050566) bank180323 +180324 POINT(38.480809706870986 -123.28652417978357) bank180324 +180325 POINT(38.135397770799585 -122.46325564624163) bank180325 +180326 POINT(38.416734910551625 -122.85987378632488) bank180326 +180327 POINT(38.36185076936634 -121.95822146064036) bank180327 +180328 POINT(38.76467915238131 -123.04823009906404) bank180328 +180329 POINT(37.79867708838182 -121.92870677532734) bank180329 +180330 POINT(37.37302504288593 -122.02841429472427) bank180330 +180331 POINT(38.40967506448927 -123.32878575746406) bank180331 +180332 POINT(38.0794391057391 -121.95384204360377) bank180332 +180333 POINT(37.698855854651704 -121.46412733323066) bank180333 +180334 POINT(37.13711558075287 -123.13019714444494) bank180334 +180335 POINT(37.712832888902874 -121.52211014248151) bank180335 +180336 POINT(37.31518327765265 -123.00451031050407) bank180336 +180337 POINT(37.03182289166124 -121.969310997217) bank180337 +180338 POINT(38.688968881640264 -122.45337479726417) bank180338 +180339 POINT(38.266776488711336 -121.42862698442438) bank180339 +180340 POINT(37.915124013291184 -122.10422954050303) bank180340 +180341 POINT(37.86880519957672 -122.10076224961584) bank180341 +180342 POINT(38.20111090718077 -122.52657525694663) bank180342 +180343 POINT(36.81284049004714 -121.66115296280049) bank180343 +180344 POINT(37.33305764135634 -121.77618028143354) bank180344 +180345 POINT(37.00816254695205 -121.90275480360403) bank180345 +180346 POINT(37.045725005091235 -122.98557919341972) bank180346 +180347 POINT(37.41238918081696 -123.0038272714473) bank180347 +180348 POINT(36.96161509037944 -122.85840889961432) bank180348 +180349 POINT(38.721799728477166 -122.79774509005007) bank180349 +180350 POINT(36.94096410034761 -122.63957736364306) bank180350 +180351 POINT(37.6104724301801 -121.62367327520893) bank180351 +180352 POINT(38.33090035599695 -122.80150440160334) bank180352 +180353 POINT(38.15230639366044 -122.90991961507734) bank180353 +180354 POINT(38.31243669406732 -122.0573145894694) bank180354 +180355 POINT(37.97570368993156 -123.09515688986332) bank180355 +180356 POINT(38.61011743194261 -122.3031571562728) bank180356 +180357 POINT(37.660888066991404 -123.11203228052167) bank180357 +180358 POINT(37.634088567694874 -122.71509129571912) bank180358 +180359 POINT(38.330795464982295 -122.19201778762128) bank180359 +180360 POINT(37.42003027038934 -121.67566656412232) bank180360 +180361 POINT(37.30929126859722 -122.80749969132737) bank180361 +180362 POINT(38.724948394999565 -122.49012284512573) bank180362 +180363 POINT(37.738790047154346 -121.81482108087296) bank180363 +180364 POINT(38.528603234560904 -123.35970749410039) bank180364 +180365 POINT(37.01513780549819 -122.82962505646312) bank180365 +180366 POINT(37.430646158443466 -122.0147225772315) bank180366 +180367 POINT(38.63246038005544 -122.97197866921495) bank180367 +180368 POINT(37.37157273587939 -122.077747135529) bank180368 +180369 POINT(38.00410383438398 -122.26722897900572) bank180369 +180370 POINT(38.497795423377404 -122.75713296566693) bank180370 +180371 POINT(38.35333796595455 -123.25378798687524) bank180371 +180372 POINT(38.01445021094312 -121.5164420941428) bank180372 +180373 POINT(38.32794792934863 -121.482047369072) bank180373 +180374 POINT(37.56629130558905 -121.71416143528067) bank180374 +180375 POINT(37.77391178186076 -122.23567292008542) bank180375 +180376 POINT(38.530217378479435 -122.14584554237129) bank180376 +180377 POINT(38.113779763512 -122.39110414535688) bank180377 +180378 POINT(37.01665439064362 -121.91175260759933) bank180378 +180379 POINT(37.411785049834265 -122.48010521289754) bank180379 +180380 POINT(36.9969974564922 -121.49589257006723) bank180380 +180381 POINT(37.925879455485955 -121.9764110781789) bank180381 +180382 POINT(37.83363228607232 -123.24734859317898) bank180382 +180383 POINT(37.842620184696585 -121.91913529547479) bank180383 +180384 POINT(38.44804571401221 -123.04416752644381) bank180384 +180385 POINT(38.11931545249135 -123.09358750814705) bank180385 +180386 POINT(37.24200197202884 -123.21970135931997) bank180386 +180387 POINT(37.0078822659182 -122.90051188765881) bank180387 +180388 POINT(37.48919376837192 -121.67857596371456) bank180388 +180389 POINT(37.27148129114768 -122.72944926583314) bank180389 +180390 POINT(37.885563846469445 -121.51265975038564) bank180390 +180391 POINT(37.32848458521853 -122.14045900218338) bank180391 +180392 POINT(37.98456819514229 -121.56227824852986) bank180392 +180393 POINT(37.21951606261385 -121.72624877158754) bank180393 +180394 POINT(37.971743452250564 -121.4679811932691) bank180394 +180395 POINT(37.192610697668684 -122.19329310866374) bank180395 +180396 POINT(37.65041208039533 -121.78693194522758) bank180396 +180397 POINT(38.5358839406623 -121.85012191429907) bank180397 +180398 POINT(38.17378865884376 -121.99832125610965) bank180398 +180399 POINT(37.923352458657455 -122.51740127323016) bank180399 +180400 POINT(38.261472504605194 -121.73054577258468) bank180400 +180401 POINT(37.90078392775023 -122.97691980828189) bank180401 +180402 POINT(38.53498651885128 -122.86190127199104) bank180402 +180403 POINT(38.185141007070044 -121.56364865458343) bank180403 +180404 POINT(37.997871715337794 -123.3444439154948) bank180404 +180405 POINT(37.428075566914586 -122.36905739781571) bank180405 +180406 POINT(37.09153234486974 -122.17197843809987) bank180406 +180407 POINT(38.25312405406263 -122.7968846639269) bank180407 +180408 POINT(36.825266585739186 -121.93934606286699) bank180408 +180409 POINT(38.57643061958099 -121.67697320917401) bank180409 +180410 POINT(38.58286453069282 -122.5834173641308) bank180410 +180411 POINT(38.26672032052472 -122.31651131286294) bank180411 +180412 POINT(37.27948020941592 -123.07945252120061) bank180412 +180413 POINT(36.97526650049447 -122.43792514534003) bank180413 +180414 POINT(38.50009534339398 -121.45288221259796) bank180414 +180415 POINT(37.12232723251461 -122.5120901575014) bank180415 +180416 POINT(38.50911923679127 -121.73378845064991) bank180416 +180417 POINT(38.37038367130535 -121.91984917811862) bank180417 +180418 POINT(37.15959747470243 -122.48627069803533) bank180418 +180419 POINT(37.16558728040911 -121.55021474767325) bank180419 +180420 POINT(37.04822366795524 -122.71813876598203) bank180420 +180421 POINT(38.71245492309149 -123.2550470190664) bank180421 +180422 POINT(37.652931823933095 -121.52789809954918) bank180422 +180423 POINT(37.485251306765505 -121.59061575454685) bank180423 +180424 POINT(38.717324187283644 -121.84186298433889) bank180424 +180425 POINT(38.00306017932469 -123.2241663556623) bank180425 +180426 POINT(37.49221389532314 -123.19017865459034) bank180426 +180427 POINT(37.141217526275135 -122.52202345403903) bank180427 +180428 POINT(38.763876788851505 -123.00064203605613) bank180428 +180429 POINT(38.55916395477548 -122.75968634720819) bank180429 +180430 POINT(38.611520519537635 -123.00825907422116) bank180430 +180431 POINT(37.58284864640328 -121.82176699573671) bank180431 +180432 POINT(38.26047700916089 -122.2337352418629) bank180432 +180433 POINT(37.96680234967403 -122.06199463930179) bank180433 +180434 POINT(38.61761547646657 -122.4170632369355) bank180434 +180435 POINT(37.296833807687044 -122.24435273890161) bank180435 +180436 POINT(37.91674517322405 -122.2190935203863) bank180436 +180437 POINT(37.140680950175984 -122.6459143355211) bank180437 +180438 POINT(37.408094446766164 -122.20660721186206) bank180438 +180439 POINT(37.89572843151415 -122.51192358009656) bank180439 +180440 POINT(38.50758215245739 -121.91203422126162) bank180440 +180441 POINT(37.47331979640705 -123.34497880329086) bank180441 +180442 POINT(37.73937361709459 -121.55906211501444) bank180442 +180443 POINT(37.20492822716282 -123.22654611113342) bank180443 +180444 POINT(38.536465813518085 -123.13421151492544) bank180444 +180445 POINT(38.59258415887065 -123.08327236537048) bank180445 +180446 POINT(37.46370512648084 -122.64958379213667) bank180446 +180447 POINT(38.00436316511581 -122.21462685929546) bank180447 +180448 POINT(37.240189832801086 -123.30631357909522) bank180448 +180449 POINT(36.813386595267126 -122.10381691735944) bank180449 +180450 POINT(38.1246823950139 -121.74635962993301) bank180450 +180451 POINT(38.46211459349904 -121.94646699348843) bank180451 +180452 POINT(37.87898819748105 -121.76983994355922) bank180452 +180453 POINT(38.22403138148721 -122.7588638894782) bank180453 +180454 POINT(37.78487677949401 -122.47070722881247) bank180454 +180455 POINT(38.295910505668004 -122.72356979578147) bank180455 +180456 POINT(38.392012405413475 -123.32665477082651) bank180456 +180457 POINT(38.15207008682845 -122.20361819330617) bank180457 +180458 POINT(37.822609759378594 -122.24525292498807) bank180458 +180459 POINT(38.20998583794753 -122.32275978116927) bank180459 +180460 POINT(37.42543749236557 -121.7836393041793) bank180460 +180461 POINT(37.96466712817144 -122.40829122788915) bank180461 +180462 POINT(37.67831255511717 -122.95766540505925) bank180462 +180463 POINT(37.20111111106647 -122.08030775078035) bank180463 +180464 POINT(38.008048510058714 -122.03405930853152) bank180464 +180465 POINT(37.660938808704834 -122.05969405964795) bank180465 +180466 POINT(37.963544690089215 -122.662945998033) bank180466 +180467 POINT(38.19970152722434 -121.681609203272) bank180467 +180468 POINT(38.064930211266315 -122.51038355783268) bank180468 +180469 POINT(37.799949926080735 -123.0614233599788) bank180469 +180470 POINT(36.83685693603398 -121.58179088750579) bank180470 +180471 POINT(37.46589374681418 -122.6263820303449) bank180471 +180472 POINT(37.825500606797696 -123.24501109686223) bank180472 +180473 POINT(38.07778606313676 -123.26905651531845) bank180473 +180474 POINT(38.206533495995366 -123.01746087881807) bank180474 +180475 POINT(37.979869745026946 -122.12596278679813) bank180475 +180476 POINT(36.89204108922359 -122.5502763510187) bank180476 +180477 POINT(37.90909688191902 -121.43249577369481) bank180477 +180478 POINT(38.42092226553558 -121.8546640003664) bank180478 +180479 POINT(37.26480159882915 -123.38328474780492) bank180479 +180480 POINT(37.94501319923423 -123.03837099475741) bank180480 +180481 POINT(38.053223625491135 -122.18641990458791) bank180481 +180482 POINT(36.901871993751186 -122.81047987840294) bank180482 +180483 POINT(38.57542404422683 -122.83638021396541) bank180483 +180484 POINT(38.497758785128354 -121.52080444391542) bank180484 +180485 POINT(37.97474708173667 -122.58904643259163) bank180485 +180486 POINT(37.57691972606819 -122.88119022409067) bank180486 +180487 POINT(38.422264702562195 -123.25013138527257) bank180487 +180488 POINT(38.18605692274461 -122.34949035663392) bank180488 +180489 POINT(37.9845336664925 -122.3715804004627) bank180489 +180490 POINT(37.02987343748094 -121.81759776193354) bank180490 +180491 POINT(37.02639728038953 -121.67769405105079) bank180491 +180492 POINT(38.201812341355534 -121.81568652738153) bank180492 +180493 POINT(38.13943920644065 -121.7516283396749) bank180493 +180494 POINT(38.201725696380045 -121.56027823286752) bank180494 +180495 POINT(38.33138462162191 -122.44733005539848) bank180495 +180496 POINT(37.685975997014964 -122.22218106692206) bank180496 +180497 POINT(37.184224710491875 -121.53613213907681) bank180497 +180498 POINT(38.76154373775811 -121.85472296255702) bank180498 +180499 POINT(38.08365589607558 -122.02428025500645) bank180499 +180500 POINT(37.882497257591346 -122.42167011999494) bank180500 +180501 POINT(37.381290662287 -122.23336639454946) bank180501 +180502 POINT(38.302908408919926 -123.10696613264365) bank180502 +180503 POINT(37.984400851148585 -122.03139487213771) bank180503 +180504 POINT(37.2586308654883 -122.77421543963294) bank180504 +180505 POINT(37.91477610304337 -122.49585146380778) bank180505 +180506 POINT(38.67548699359034 -123.10161153969803) bank180506 +180507 POINT(37.713879231442725 -122.72310659907679) bank180507 +180508 POINT(37.58113108759967 -122.58261919984434) bank180508 +180509 POINT(37.12300405460692 -123.03315463675865) bank180509 +180510 POINT(38.302040508265456 -123.24166758459153) bank180510 +180511 POINT(38.6016176595324 -121.93793503551) bank180511 +180512 POINT(37.600514217647294 -122.45565880972217) bank180512 +180513 POINT(37.89470660180716 -121.79872393146768) bank180513 +180514 POINT(38.22363252152654 -121.65373379175442) bank180514 +180515 POINT(38.01252082067872 -122.55020649994118) bank180515 +180516 POINT(36.787070145804336 -122.81344822161782) bank180516 +180517 POINT(37.255613879466985 -121.54502275458904) bank180517 +180518 POINT(37.858062101386814 -122.56719449919913) bank180518 +180519 POINT(37.02992944299153 -122.64172968251872) bank180519 +180520 POINT(37.24615075845534 -121.46712061949424) bank180520 +180521 POINT(37.023291222951364 -122.80032776554451) bank180521 +180522 POINT(37.068979703934176 -122.50937595108692) bank180522 +180523 POINT(38.68631883478167 -121.81127367238786) bank180523 +180524 POINT(37.56710299703411 -121.61515068738233) bank180524 +180525 POINT(38.28545433524223 -122.70477155533209) bank180525 +180526 POINT(37.06100408949769 -122.22637264502976) bank180526 +180527 POINT(38.032825575094265 -123.19037391660093) bank180527 +180528 POINT(38.502358863536394 -123.05413969068181) bank180528 +180529 POINT(37.868446856148985 -122.45391744601346) bank180529 +180530 POINT(38.65389599761785 -122.6567381352209) bank180530 +180531 POINT(38.23373165037113 -122.83661841316528) bank180531 +180532 POINT(37.9503466952557 -121.81677719038765) bank180532 +180533 POINT(38.676437353649874 -122.4309309364985) bank180533 +180534 POINT(37.91419684999827 -122.5401716049159) bank180534 +180535 POINT(38.571542756340286 -122.75264932133392) bank180535 +180536 POINT(37.87384080600184 -123.28972867363507) bank180536 +180537 POINT(36.85273184540592 -122.59151165520892) bank180537 +180538 POINT(37.41007827254704 -122.00733540455805) bank180538 +180539 POINT(38.054595987449346 -122.15026301700081) bank180539 +180540 POINT(37.89060226300873 -122.195399977692) bank180540 +180541 POINT(36.98052211349095 -123.04319126949328) bank180541 +180542 POINT(37.08637700452572 -121.56048346568691) bank180542 +180543 POINT(38.297311195817606 -122.14129739185071) bank180543 +180544 POINT(38.26913786194091 -122.61478452031956) bank180544 +180545 POINT(38.0186755301073 -122.02410990659922) bank180545 +180546 POINT(38.10366106309011 -122.92315135901605) bank180546 +180547 POINT(36.80199264995728 -122.66555368317424) bank180547 +180548 POINT(38.637343338229385 -122.42357292924834) bank180548 +180549 POINT(37.718889818822426 -121.56316491639976) bank180549 +180550 POINT(38.02162425400605 -121.78541445096818) bank180550 +180551 POINT(37.058518135236405 -122.57559080655159) bank180551 +180552 POINT(37.3438052257513 -121.45878153681895) bank180552 +180553 POINT(38.48845389010443 -123.14066365798878) bank180553 +180554 POINT(37.52168923700239 -122.0571492858792) bank180554 +180555 POINT(37.332292627153244 -121.62994063239574) bank180555 +180556 POINT(37.17069021133969 -121.54452081570092) bank180556 +180557 POINT(37.81227268895817 -122.20706455598842) bank180557 +180558 POINT(38.66003657711438 -122.33995593814714) bank180558 +180559 POINT(36.81333821528838 -122.12329686331108) bank180559 +180560 POINT(37.82959060927742 -121.47604803655466) bank180560 +180561 POINT(38.18117157682001 -122.24163837395716) bank180561 +180562 POINT(37.59328471840625 -123.21482679078488) bank180562 +180563 POINT(38.64170421355302 -122.64343280858395) bank180563 +180564 POINT(37.55360128868843 -121.49889765496874) bank180564 +180565 POINT(38.463952461866484 -122.9995732343441) bank180565 +180566 POINT(37.37635945223598 -122.95004235906922) bank180566 +180567 POINT(37.886090084969375 -121.85014437213539) bank180567 +180568 POINT(38.71449976254022 -121.79253431610427) bank180568 +180569 POINT(38.47100714458628 -121.75484756836488) bank180569 +180570 POINT(36.82264542771998 -122.70014497174265) bank180570 +180571 POINT(38.36420873669186 -122.46360812730927) bank180571 +180572 POINT(37.437222134412565 -123.36535562286888) bank180572 +180573 POINT(38.047273801086796 -123.14165516221281) bank180573 +180574 POINT(37.741239175714405 -123.07578761913281) bank180574 +180575 POINT(38.47190341805679 -122.82629303748462) bank180575 +180576 POINT(38.41396144376764 -122.59935477774704) bank180576 +180577 POINT(38.38444663103124 -122.96888039397462) bank180577 +180578 POINT(36.8189464072353 -123.11920684752705) bank180578 +180579 POINT(37.5121633539459 -122.53172039386945) bank180579 +180580 POINT(38.48634899011031 -121.97475599474667) bank180580 +180581 POINT(37.30435839169779 -123.19887186154828) bank180581 +180582 POINT(36.915090543842226 -123.3316537867197) bank180582 +180583 POINT(38.053911829692254 -122.53406762947441) bank180583 +180584 POINT(38.36077660958738 -122.92916319052316) bank180584 +180585 POINT(38.47010207173775 -121.84274520832926) bank180585 +180586 POINT(37.499137968476404 -121.97128941270446) bank180586 +180587 POINT(38.10446336912581 -122.13973003165947) bank180587 +180588 POINT(38.13179623021084 -122.19992176999051) bank180588 +180589 POINT(38.543601493156594 -122.25322457871016) bank180589 +180590 POINT(37.57510250916157 -123.14630521518512) bank180590 +180591 POINT(37.748476181646694 -121.58190081948216) bank180591 +180592 POINT(38.20335659714703 -122.26067559896931) bank180592 +180593 POINT(37.554961319413756 -122.75948662145966) bank180593 +180594 POINT(37.590985606342315 -122.8622883444308) bank180594 +180595 POINT(37.940001108826685 -123.14206049446655) bank180595 +180596 POINT(37.00490795248376 -123.19201271054473) bank180596 +180597 POINT(37.132964075861764 -121.47759343915875) bank180597 +180598 POINT(37.58830762125449 -121.50399269944006) bank180598 +180599 POINT(37.75795579190583 -121.5095255107732) bank180599 +180600 POINT(37.153528203942756 -122.78133773260888) bank180600 +180601 POINT(37.472753666117555 -122.61924101327313) bank180601 +180602 POINT(36.811167754329475 -122.69367587254622) bank180602 +180603 POINT(36.81081767527488 -123.32008565322431) bank180603 +180604 POINT(38.18410314474371 -121.8028421527789) bank180604 +180605 POINT(38.6363017455068 -121.6350729912791) bank180605 +180606 POINT(37.63060654596629 -122.05731917461794) bank180606 +180607 POINT(38.076095612925336 -122.05282161662673) bank180607 +180608 POINT(36.92602927695502 -121.83970386723192) bank180608 +180609 POINT(37.40481030878397 -122.93556621734785) bank180609 +180610 POINT(38.48931922986054 -122.24520309576242) bank180610 +180611 POINT(38.76427237324194 -121.90207894665153) bank180611 +180612 POINT(38.16955686292442 -122.52151043002505) bank180612 +180613 POINT(37.50156056623136 -123.20466639038283) bank180613 +180614 POINT(37.37070621593545 -123.39965842508971) bank180614 +180615 POINT(36.93933328046353 -121.65667874543672) bank180615 +180616 POINT(37.78996197155488 -121.68158196793576) bank180616 +180617 POINT(37.5515592754386 -121.48878921907553) bank180617 +180618 POINT(37.463963273212514 -122.9986160888743) bank180618 +180619 POINT(37.72131233390541 -122.86297787911244) bank180619 +180620 POINT(38.5997480931271 -122.40152433678341) bank180620 +180621 POINT(37.98784459376218 -122.59463458134377) bank180621 +180622 POINT(38.01202411942538 -122.87525773206957) bank180622 +180623 POINT(36.91552105304723 -122.03631634267397) bank180623 +180624 POINT(38.149532300980134 -121.58772707835178) bank180624 +180625 POINT(36.79191378412939 -121.76575932079496) bank180625 +180626 POINT(38.603107961700495 -122.2922118549297) bank180626 +180627 POINT(37.04937638853273 -122.25721604896418) bank180627 +180628 POINT(37.74097709756472 -122.61654259752572) bank180628 +180629 POINT(37.65004291176242 -121.8874356772143) bank180629 +180630 POINT(37.42547158911171 -122.50492167310198) bank180630 +180631 POINT(38.61593358104808 -122.40109581675799) bank180631 +180632 POINT(37.586919820075494 -121.77996979126715) bank180632 +180633 POINT(37.85958388851516 -122.16811948902877) bank180633 +180634 POINT(36.86019958553191 -122.46909350840303) bank180634 +180635 POINT(37.224469910140485 -122.7346870189765) bank180635 +180636 POINT(38.03943694511527 -122.48584080011648) bank180636 +180637 POINT(37.073684237149344 -123.39484086604872) bank180637 +180638 POINT(38.37367137122696 -121.6444127641427) bank180638 +180639 POINT(37.19672453985567 -122.4577675089255) bank180639 +180640 POINT(37.08743435368703 -122.66054161581533) bank180640 +180641 POINT(37.97023610187531 -122.93550903037593) bank180641 +180642 POINT(37.32181276288061 -121.43799271922543) bank180642 +180643 POINT(38.197913763222495 -121.8079650638547) bank180643 +180644 POINT(38.026994112170755 -122.72583410360599) bank180644 +180645 POINT(38.42780533357881 -122.39951580744005) bank180645 +180646 POINT(38.6459713776925 -121.94333840930734) bank180646 +180647 POINT(38.57763770015232 -121.72558411406133) bank180647 +180648 POINT(38.215618555250686 -122.57893139627396) bank180648 +180649 POINT(37.00520646329692 -122.36349154168238) bank180649 +180650 POINT(38.395972554307626 -122.88070196671855) bank180650 +180651 POINT(37.00943726008001 -121.9150873071498) bank180651 +180652 POINT(38.44307105950784 -122.35138147845011) bank180652 +180653 POINT(37.5320470364789 -121.5876939067145) bank180653 +180654 POINT(38.68023529815304 -122.82006305031601) bank180654 +180655 POINT(38.582218839415894 -123.1067691730952) bank180655 +180656 POINT(36.98464068988481 -122.20729769150212) bank180656 +180657 POINT(36.77573697579306 -123.39268631865713) bank180657 +180658 POINT(38.40232702383656 -122.2851912694921) bank180658 +180659 POINT(37.785820262853775 -122.23257800542703) bank180659 +180660 POINT(38.531456274533724 -123.34468479600383) bank180660 +180661 POINT(37.34739813862216 -121.83724714384905) bank180661 +180662 POINT(37.1392656372602 -121.74570798063861) bank180662 +180663 POINT(38.04914773517359 -122.11759888068458) bank180663 +180664 POINT(36.78201315963118 -123.03111778493876) bank180664 +180665 POINT(37.240211334172464 -121.96631182082933) bank180665 +180666 POINT(38.23424754364778 -123.25614084117935) bank180666 +180667 POINT(38.71133775575359 -122.81747584569943) bank180667 +180668 POINT(36.89749158839101 -123.21364408321038) bank180668 +180669 POINT(37.98016703655161 -122.57045734687073) bank180669 +180670 POINT(37.00414826571632 -123.07194570497715) bank180670 +180671 POINT(37.061813318872 -122.40238491751181) bank180671 +180672 POINT(38.13374410844072 -122.83953813522156) bank180672 +180673 POINT(37.127237810235265 -121.81956148693276) bank180673 +180674 POINT(37.39231734198159 -121.61001044073318) bank180674 +180675 POINT(38.61449368601953 -121.60507731525011) bank180675 +180676 POINT(38.758382375029534 -123.12697997865855) bank180676 +180677 POINT(36.857986960411054 -123.34821972956229) bank180677 +180678 POINT(38.337957518844945 -122.89125257127543) bank180678 +180679 POINT(38.46049641781715 -122.48763139562745) bank180679 +180680 POINT(37.93652396072454 -123.23243502541227) bank180680 +180681 POINT(37.3198275360962 -122.95727153831898) bank180681 +180682 POINT(37.57033326101311 -122.7382715373502) bank180682 +180683 POINT(37.17513119108862 -121.54983598618868) bank180683 +180684 POINT(37.043358630540816 -122.66082288541082) bank180684 +180685 POINT(38.33028522399401 -122.96529173194565) bank180685 +180686 POINT(37.44787178963584 -122.59077378334142) bank180686 +180687 POINT(38.63681666443827 -123.18757177602139) bank180687 +180688 POINT(37.3906151925913 -122.3952015861848) bank180688 +180689 POINT(38.256363175611455 -122.98450550749706) bank180689 +180690 POINT(37.30818826169363 -122.12842884854032) bank180690 +180691 POINT(37.46180514828444 -121.7300193749133) bank180691 +180692 POINT(37.54631429335651 -122.31890440781544) bank180692 +180693 POINT(37.05071803190396 -121.4477979422941) bank180693 +180694 POINT(37.13262854461351 -122.54480348019379) bank180694 +180695 POINT(37.94421943346905 -123.33594440648184) bank180695 +180696 POINT(38.42487289222568 -122.83658458226374) bank180696 +180697 POINT(38.72373408077061 -121.80284754559386) bank180697 +180698 POINT(38.01757165915236 -122.93994323733587) bank180698 +180699 POINT(37.275702700664255 -121.66812424739462) bank180699 +180700 POINT(38.45697507070191 -122.40935799653275) bank180700 +180701 POINT(37.066124874204085 -122.3088796002443) bank180701 +180702 POINT(37.73628664756908 -122.16059708644006) bank180702 +180703 POINT(37.11114512663045 -123.02438624883385) bank180703 +180704 POINT(37.77890236593438 -123.2333442085471) bank180704 +180705 POINT(38.66750661173117 -122.13044046237297) bank180705 +180706 POINT(38.58647215013284 -122.8338760107061) bank180706 +180707 POINT(38.243855976597445 -122.93391056960523) bank180707 +180708 POINT(38.35966664114394 -121.56347415244045) bank180708 +180709 POINT(37.08244369440945 -121.91863871670627) bank180709 +180710 POINT(37.32175622783638 -122.25681262148626) bank180710 +180711 POINT(37.901158283686776 -122.35386044463613) bank180711 +180712 POINT(38.0837389512075 -121.61342893946015) bank180712 +180713 POINT(37.805252263892626 -122.23451185461708) bank180713 +180714 POINT(37.45964945124841 -122.76413660221135) bank180714 +180715 POINT(38.39843484179731 -122.97450914638208) bank180715 +180716 POINT(37.834087472651824 -123.24302433556245) bank180716 +180717 POINT(36.89004937309372 -121.97069131174499) bank180717 +180718 POINT(37.40657044879437 -122.26868221323653) bank180718 +180719 POINT(38.31396813787267 -122.64902736580706) bank180719 +180720 POINT(37.7806801080548 -122.38161283960056) bank180720 +180721 POINT(36.81183414709459 -122.72429671764694) bank180721 +180722 POINT(38.11281882607644 -122.75338987900207) bank180722 +180723 POINT(37.05107017107608 -122.20751525197907) bank180723 +180724 POINT(36.97811570468302 -121.87066380916723) bank180724 +180725 POINT(38.64400813809113 -123.15467647767959) bank180725 +180726 POINT(36.91044335147522 -122.41610832383581) bank180726 +180727 POINT(36.93673503863575 -122.2978513437393) bank180727 +180728 POINT(38.538041018835976 -121.50388854640832) bank180728 +180729 POINT(37.879407518808854 -122.40408931980464) bank180729 +180730 POINT(37.29605891503984 -121.60703130611704) bank180730 +180731 POINT(38.71406431164505 -123.32538446863754) bank180731 +180732 POINT(36.84451838537123 -121.46178636184301) bank180732 +180733 POINT(38.35524679366654 -122.51132918535481) bank180733 +180734 POINT(36.82928748158251 -122.17518330225015) bank180734 +180735 POINT(38.60141436967831 -122.63666386759886) bank180735 +180736 POINT(37.57892716640283 -122.45624086272204) bank180736 +180737 POINT(38.3181631435183 -122.30445540359091) bank180737 +180738 POINT(38.70793831078385 -122.4820892588813) bank180738 +180739 POINT(36.96575022118445 -122.36831887565933) bank180739 +180740 POINT(36.91131204520657 -122.67871042809618) bank180740 +180741 POINT(38.18803807442088 -123.0196513122659) bank180741 +180742 POINT(37.60916230790341 -122.101706813927) bank180742 +180743 POINT(38.30597441610479 -122.55793466657731) bank180743 +180744 POINT(37.625448013311996 -123.13850586270968) bank180744 +180745 POINT(38.278066463035124 -122.01797961585616) bank180745 +180746 POINT(36.92712284695725 -123.21546668972985) bank180746 +180747 POINT(36.930317561459425 -122.79086975684619) bank180747 +180748 POINT(37.20629917696895 -122.78548970598655) bank180748 +180749 POINT(38.4202301492751 -122.65715330355002) bank180749 +180750 POINT(37.023935678980784 -121.56584874713515) bank180750 +180751 POINT(37.09625971143909 -121.52315377788418) bank180751 +180752 POINT(37.2401149222555 -122.19869524158914) bank180752 +180753 POINT(38.07310431425856 -122.29340358834038) bank180753 +180754 POINT(37.23295972018007 -122.491470647324) bank180754 +180755 POINT(36.843213008064275 -122.79529733532861) bank180755 +180756 POINT(38.184058174396675 -122.54147503142113) bank180756 +180757 POINT(38.48765999507127 -123.11139177790325) bank180757 +180758 POINT(37.085450603399586 -122.11192943109651) bank180758 +180759 POINT(37.697232479947566 -122.59782566629889) bank180759 +180760 POINT(38.53554002237649 -122.46755747166121) bank180760 +180761 POINT(37.17171591755307 -121.91151825573768) bank180761 +180762 POINT(38.51879767405537 -121.72022070407668) bank180762 +180763 POINT(37.65436804583614 -122.34169641215328) bank180763 +180764 POINT(36.99114577519508 -123.04226976239575) bank180764 +180765 POINT(38.07594915987134 -122.44415329220864) bank180765 +180766 POINT(37.42201253446216 -122.44914210399186) bank180766 +180767 POINT(37.75104486345147 -122.85500650392427) bank180767 +180768 POINT(38.60093899005384 -121.97153362754649) bank180768 +180769 POINT(37.207453127404634 -122.68332266282785) bank180769 +180770 POINT(37.45430686177257 -121.63677182420466) bank180770 +180771 POINT(38.07191497791999 -122.26084896280649) bank180771 +180772 POINT(37.13223369606945 -122.04735666594084) bank180772 +180773 POINT(38.690155156487705 -121.98130115022123) bank180773 +180774 POINT(38.28269438757225 -122.26307468052134) bank180774 +180775 POINT(38.46799963681759 -121.62093005960267) bank180775 +180776 POINT(38.76569725711635 -122.89195642427002) bank180776 +180777 POINT(37.27295886126735 -123.27677690760818) bank180777 +180778 POINT(38.094756940840874 -121.67080095582601) bank180778 +180779 POINT(38.446848243032726 -123.35504046135961) bank180779 +180780 POINT(37.15941459579533 -121.99683709601675) bank180780 +180781 POINT(37.11543127028195 -122.30447926891708) bank180781 +180782 POINT(37.6247005148711 -121.62371583729522) bank180782 +180783 POINT(38.376224761484686 -123.39674698412587) bank180783 +180784 POINT(36.98260158089688 -122.97441137586954) bank180784 +180785 POINT(37.57014838772608 -122.05932383259218) bank180785 +180786 POINT(38.08744530522253 -122.76853729293005) bank180786 +180787 POINT(38.24461143541321 -121.86226450658896) bank180787 +180788 POINT(37.0591614212176 -121.92963788910576) bank180788 +180789 POINT(36.837063408223216 -121.42883352023101) bank180789 +180790 POINT(38.1193878221316 -121.58968977530556) bank180790 +180791 POINT(37.82983310272287 -123.24060828458364) bank180791 +180792 POINT(38.360404116610475 -122.38738492183012) bank180792 +180793 POINT(38.207631761241146 -123.06833906601938) bank180793 +180794 POINT(37.39467005736727 -122.95426333154586) bank180794 +180795 POINT(37.69166444910702 -123.00972482659914) bank180795 +180796 POINT(37.77199245784955 -121.46587508214765) bank180796 +180797 POINT(36.82183370326115 -122.24315146020862) bank180797 +180798 POINT(38.663273272255445 -122.64230472394725) bank180798 +180799 POINT(37.89136855665199 -121.7876586566975) bank180799 +180800 POINT(37.38996261395929 -121.73772546490383) bank180800 +180801 POINT(36.888635747980686 -122.94039049037904) bank180801 +180802 POINT(38.058973491853656 -123.34921896991543) bank180802 +180803 POINT(36.79561582669062 -122.65712932323974) bank180803 +180804 POINT(36.871645363227955 -121.9669493017118) bank180804 +180805 POINT(38.60658775018301 -122.4973430646023) bank180805 +180806 POINT(37.86002570917383 -123.23766214998338) bank180806 +180807 POINT(37.6274515457795 -121.62106196243184) bank180807 +180808 POINT(38.36567837671688 -122.14905520092917) bank180808 +180809 POINT(37.9912436490149 -121.95086056812744) bank180809 +180810 POINT(37.14995880008923 -121.98152877935004) bank180810 +180811 POINT(37.3665175672658 -123.01041668155568) bank180811 +180812 POINT(38.44481592961314 -123.27584923951909) bank180812 +180813 POINT(37.75431952216017 -122.55223615171415) bank180813 +180814 POINT(36.93463099409915 -122.52850193775265) bank180814 +180815 POINT(37.55945328431743 -123.16433369012826) bank180815 +180816 POINT(37.436055721499294 -123.03993578820662) bank180816 +180817 POINT(37.62458378619418 -122.7426636768024) bank180817 +180818 POINT(37.730716408739156 -122.39903210622411) bank180818 +180819 POINT(37.872996292209876 -122.11496724774396) bank180819 +180820 POINT(37.43192997869422 -122.19330090758774) bank180820 +180821 POINT(37.858750964139176 -121.52178725895828) bank180821 +180822 POINT(38.39308930246315 -122.5981673667516) bank180822 +180823 POINT(36.92964380040188 -122.68229097389045) bank180823 +180824 POINT(38.381431754311535 -121.61357382869653) bank180824 +180825 POINT(36.95515061863993 -123.0988996225197) bank180825 +180826 POINT(37.13837182270889 -122.8104223588855) bank180826 +180827 POINT(37.858939701166555 -121.71154319473894) bank180827 +180828 POINT(37.37173285672265 -122.12933077847475) bank180828 +180829 POINT(36.85619254423294 -121.98155830764534) bank180829 +180830 POINT(36.79873104403305 -122.20035357623495) bank180830 +180831 POINT(37.75129666330816 -122.49719175747684) bank180831 +180832 POINT(36.90547025054304 -122.08288292044844) bank180832 +180833 POINT(37.509646389543455 -121.94020692876492) bank180833 +180834 POINT(37.96322608974998 -121.52381001367898) bank180834 +180835 POINT(38.11168783774467 -123.40436471425343) bank180835 +180836 POINT(38.019750999997 -123.02496372204929) bank180836 +180837 POINT(37.04346597686532 -122.65953045490411) bank180837 +180838 POINT(37.315801707714314 -123.15781972761233) bank180838 +180839 POINT(38.20764518207868 -121.43079624739438) bank180839 +180840 POINT(36.8351741389382 -123.15029981117252) bank180840 +180841 POINT(37.169625988955616 -123.05411073192116) bank180841 +180842 POINT(37.813796374655176 -121.70791456980874) bank180842 +180843 POINT(37.219324486830786 -121.51538563988524) bank180843 +180844 POINT(38.04904221582108 -123.17498211084582) bank180844 +180845 POINT(38.39875039345212 -122.70440365094055) bank180845 +180846 POINT(37.086045476013545 -122.90662163715884) bank180846 +180847 POINT(37.26910072430676 -122.86874166091525) bank180847 +180848 POINT(36.91525342137672 -121.96980199705725) bank180848 +180849 POINT(38.59404062054555 -122.24351610268293) bank180849 +180850 POINT(36.98662087197443 -122.88646280522019) bank180850 +180851 POINT(37.10049171220345 -121.94428664687284) bank180851 +180852 POINT(37.18945244965485 -121.97679273024468) bank180852 +180853 POINT(37.003645455136265 -122.21342662851227) bank180853 +180854 POINT(37.77604097032617 -122.91511150238254) bank180854 +180855 POINT(37.84234846650403 -121.7356621707061) bank180855 +180856 POINT(38.733856063503666 -121.54889400927959) bank180856 +180857 POINT(37.40418928277705 -123.40593711252158) bank180857 +180858 POINT(37.13619079095665 -121.54326583564158) bank180858 +180859 POINT(37.6071710372631 -123.36919978282062) bank180859 +180860 POINT(37.319583658675825 -121.78970174972198) bank180860 +180861 POINT(38.380872586794254 -121.87354354899577) bank180861 +180862 POINT(38.57024483807935 -122.3597270739035) bank180862 +180863 POINT(37.875313480056406 -122.3265579509847) bank180863 +180864 POINT(37.929466479650195 -121.74988096133477) bank180864 +180865 POINT(37.60777802578277 -122.34767632700658) bank180865 +180866 POINT(38.13716109002517 -122.05035322478075) bank180866 +180867 POINT(38.63058890983729 -123.1400577253968) bank180867 +180868 POINT(37.75764140976006 -122.98113543679507) bank180868 +180869 POINT(38.54814002676376 -122.97318845226758) bank180869 +180870 POINT(38.19259805564883 -123.10407188478071) bank180870 +180871 POINT(37.329319462687785 -122.12428743293522) bank180871 +180872 POINT(37.477617673559614 -121.73716684965207) bank180872 +180873 POINT(38.68969739163476 -122.96549770454514) bank180873 +180874 POINT(38.60159231813476 -121.72663164087655) bank180874 +180875 POINT(38.2528787419979 -122.88945656015137) bank180875 +180876 POINT(37.2303169231522 -121.89150978503588) bank180876 +180877 POINT(38.17416522898541 -123.33622248466715) bank180877 +180878 POINT(37.67806489482533 -121.8731804987148) bank180878 +180879 POINT(37.68943459784035 -121.8374903463171) bank180879 +180880 POINT(37.15295773648948 -121.99838057581692) bank180880 +180881 POINT(37.6272942427362 -121.66054034136369) bank180881 +180882 POINT(38.344222255275014 -121.85803603234079) bank180882 +180883 POINT(37.95710518733142 -121.77308852530577) bank180883 +180884 POINT(38.48679993528198 -123.07172300113594) bank180884 +180885 POINT(37.13237542369935 -122.68194829515987) bank180885 +180886 POINT(37.63656905306695 -122.15735805641906) bank180886 +180887 POINT(38.37035729901199 -121.63547923386584) bank180887 +180888 POINT(37.418126884433434 -121.45075608618971) bank180888 +180889 POINT(38.66696318911926 -122.12822417074885) bank180889 +180890 POINT(37.25748883819192 -121.71015807426647) bank180890 +180891 POINT(36.79127857738401 -122.48468282183636) bank180891 +180892 POINT(37.8394548638891 -122.12520630974352) bank180892 +180893 POINT(38.25597365791897 -122.61683431755526) bank180893 +180894 POINT(38.20738004338123 -122.46622199311928) bank180894 +180895 POINT(37.09334845379504 -123.14137592036073) bank180895 +180896 POINT(38.184358840031294 -122.94291805172615) bank180896 +180897 POINT(37.118205518512084 -122.87720882122348) bank180897 +180898 POINT(38.198718063659555 -122.42801596400884) bank180898 +180899 POINT(38.605433706551175 -121.88713290769003) bank180899 +180900 POINT(37.036620309009535 -121.95811302249878) bank180900 +180901 POINT(38.525714028878674 -122.98072647095837) bank180901 +180902 POINT(36.81199728707582 -122.34756896055323) bank180902 +180903 POINT(37.14996417237933 -121.85100699978405) bank180903 +180904 POINT(37.07679059115399 -122.4223226816299) bank180904 +180905 POINT(37.399761907094586 -122.95547331600909) bank180905 +180906 POINT(38.165319364288074 -123.10934276231923) bank180906 +180907 POINT(37.88114250647749 -121.55126887787553) bank180907 +180908 POINT(37.326513595631454 -122.43444144776777) bank180908 +180909 POINT(37.52157218896279 -123.25527572787628) bank180909 +180910 POINT(37.47307894201911 -122.9514257397404) bank180910 +180911 POINT(37.73597558340433 -122.90656193497799) bank180911 +180912 POINT(37.468733174491 -122.91194277772273) bank180912 +180913 POINT(37.42172589120855 -121.81523776879118) bank180913 +180914 POINT(37.62807477615537 -123.23303441498197) bank180914 +180915 POINT(38.63228535233533 -121.76503660408446) bank180915 +180916 POINT(37.78371261704778 -123.25251934144485) bank180916 +180917 POINT(38.446945692622485 -122.25878935159686) bank180917 +180918 POINT(37.66186943470027 -121.43654797945328) bank180918 +180919 POINT(37.49969903297189 -122.76436845511009) bank180919 +180920 POINT(38.37033677205681 -122.29292171922874) bank180920 +180921 POINT(38.72621363044013 -123.31822382496566) bank180921 +180922 POINT(36.85731435273828 -122.73798100168075) bank180922 +180923 POINT(37.44714186719143 -122.2306921899059) bank180923 +180924 POINT(38.38008366042336 -122.16799852350684) bank180924 +180925 POINT(38.39440320660381 -122.31353317333966) bank180925 +180926 POINT(36.82031546021269 -122.77262358599368) bank180926 +180927 POINT(37.39534652866909 -122.83913475281629) bank180927 +180928 POINT(38.72195920973791 -123.23210455808446) bank180928 +180929 POINT(38.59164587412931 -121.47533191569225) bank180929 +180930 POINT(37.82714366517717 -123.24260566419866) bank180930 +180931 POINT(38.71760461302079 -122.99363178027502) bank180931 +180932 POINT(37.10437521317278 -122.31179916966686) bank180932 +180933 POINT(38.25902544970303 -122.08598944934116) bank180933 +180934 POINT(38.451080481539066 -121.82400121946483) bank180934 +180935 POINT(38.000959559569885 -123.07700715752763) bank180935 +180936 POINT(37.14038603328014 -123.07104592217502) bank180936 +180937 POINT(36.79366969073926 -122.2744221223175) bank180937 +180938 POINT(36.863417306985134 -122.63404610944724) bank180938 +180939 POINT(37.32852442859464 -123.23269408363538) bank180939 +180940 POINT(37.90669221920027 -121.58680744168382) bank180940 +180941 POINT(38.7171933631267 -121.84577611812948) bank180941 +180942 POINT(37.99962442831954 -122.7727133046615) bank180942 +180943 POINT(37.79317604298011 -121.42421085956008) bank180943 +180944 POINT(37.70933156854973 -121.6816034091913) bank180944 +180945 POINT(38.340573596670694 -122.29368503219277) bank180945 +180946 POINT(38.22743933724422 -121.52579288628793) bank180946 +180947 POINT(38.72460888341872 -122.89253769412778) bank180947 +180948 POINT(38.12680848101327 -122.83399545450185) bank180948 +180949 POINT(38.11646451274325 -121.89488843171307) bank180949 +180950 POINT(37.19404709651521 -121.84516862514097) bank180950 +180951 POINT(37.522323609027886 -121.83153362844784) bank180951 +180952 POINT(37.23369093364711 -121.6486534697093) bank180952 +180953 POINT(38.32472274565443 -121.89349229766411) bank180953 +180954 POINT(38.296453387906325 -122.038920080848) bank180954 +180955 POINT(38.407526051107816 -121.72321983959567) bank180955 +180956 POINT(37.233493339129886 -122.13636937976302) bank180956 +180957 POINT(38.705161005483916 -123.28614647917543) bank180957 +180958 POINT(37.631982870072186 -121.50738130772673) bank180958 +180959 POINT(37.366822845023954 -123.03433496838231) bank180959 +180960 POINT(37.00556517577711 -122.46218995485721) bank180960 +180961 POINT(38.32134362133162 -122.55409436619608) bank180961 +180962 POINT(38.43969191756262 -123.11040486687445) bank180962 +180963 POINT(37.58102876386061 -122.83783102412127) bank180963 +180964 POINT(37.449525922008206 -123.01299994412015) bank180964 +180965 POINT(37.537357674906055 -122.302457133242) bank180965 +180966 POINT(38.06263875840215 -122.43519088526516) bank180966 +180967 POINT(38.729124743228276 -122.62335826710044) bank180967 +180968 POINT(37.56247481987783 -123.18095834005682) bank180968 +180969 POINT(37.76594673897642 -122.64857646537791) bank180969 +180970 POINT(37.23723085044932 -123.03708511094938) bank180970 +180971 POINT(37.442748975547794 -123.19856886207342) bank180971 +180972 POINT(37.92768483988146 -122.18287996896431) bank180972 +180973 POINT(37.57192220607557 -123.35129064033144) bank180973 +180974 POINT(37.91539541003813 -123.01542974120747) bank180974 +180975 POINT(36.89788432479862 -123.12633388632949) bank180975 +180976 POINT(36.94942647706678 -122.88641219278935) bank180976 +180977 POINT(38.556602467651125 -122.18567388300356) bank180977 +180978 POINT(37.257749969340196 -122.95906059010773) bank180978 +180979 POINT(37.30416338907223 -123.29760741761636) bank180979 +180980 POINT(37.172585636483156 -121.64147046530464) bank180980 +180981 POINT(38.13360581170573 -123.39723823749031) bank180981 +180982 POINT(38.35403181329288 -123.0206619530825) bank180982 +180983 POINT(37.42052444071619 -123.05658653440263) bank180983 +180984 POINT(36.9972769534499 -123.11042610631178) bank180984 +180985 POINT(38.18122190580887 -121.87031250196155) bank180985 +180986 POINT(37.253273680719104 -122.47835878882027) bank180986 +180987 POINT(37.686659319683685 -122.08889797948609) bank180987 +180988 POINT(37.731848192153436 -122.00449783938011) bank180988 +180989 POINT(37.174083303834266 -122.19748719801596) bank180989 +180990 POINT(38.16315190572988 -122.25057174893674) bank180990 +180991 POINT(37.057175378452825 -122.75252445099727) bank180991 +180992 POINT(38.00503544812904 -122.11520578400955) bank180992 +180993 POINT(37.81761039976747 -123.24782233531606) bank180993 +180994 POINT(37.06816170231159 -122.8546515331337) bank180994 +180995 POINT(37.32439972702045 -123.09512515082857) bank180995 +180996 POINT(37.975669936112816 -122.12742083519149) bank180996 +180997 POINT(38.118438418382205 -122.80224151935339) bank180997 +180998 POINT(37.43554221242967 -121.54042674821326) bank180998 +180999 POINT(37.12745997941494 -123.18762346885131) bank180999 +181000 POINT(37.114675851103435 -122.59126106091887) bank181000 +181001 POINT(37.53806856458281 -122.09118902634646) bank181001 +181002 POINT(37.47345078458749 -121.76969449936487) bank181002 +181003 POINT(37.325878914742205 -122.54487684198871) bank181003 +181004 POINT(38.346590982727506 -121.62694650668824) bank181004 +181005 POINT(38.48299822600954 -123.12027467358581) bank181005 +181006 POINT(38.47747774658662 -122.36653389235873) bank181006 +181007 POINT(38.44922917860446 -122.28288438770521) bank181007 +181008 POINT(36.80406158488838 -122.99248293052287) bank181008 +181009 POINT(38.723416764489805 -123.0080243959432) bank181009 +181010 POINT(38.36297405003486 -123.15572114989176) bank181010 +181011 POINT(38.61770477550339 -122.07029023257648) bank181011 +181012 POINT(38.66618181261687 -121.6643849577386) bank181012 +181013 POINT(36.86451627247002 -121.9337277026261) bank181013 +181014 POINT(37.17251687913121 -122.28244842016342) bank181014 +181015 POINT(37.26112245287297 -123.05886473626468) bank181015 +181016 POINT(38.649798203347515 -121.80766701157683) bank181016 +181017 POINT(38.274457050711334 -123.13835120914555) bank181017 +181018 POINT(38.41222215275269 -122.44162226084794) bank181018 +181019 POINT(38.14551244049853 -122.38373439919884) bank181019 +181020 POINT(38.35128222441319 -122.34376464022677) bank181020 +181021 POINT(38.430534950551376 -122.80198883456207) bank181021 +181022 POINT(37.486423706011394 -122.54427232109373) bank181022 +181023 POINT(37.92852757971821 -122.8060819615051) bank181023 +181024 POINT(37.6960308760586 -121.49886434314593) bank181024 +181025 POINT(38.66982402111808 -121.46780029107092) bank181025 +181026 POINT(37.61194821288841 -123.21276178708473) bank181026 +181027 POINT(38.394161385246875 -123.28349567727423) bank181027 +181028 POINT(38.05028196405473 -121.74552296796622) bank181028 +181029 POINT(38.28836334772461 -122.69290565048502) bank181029 +181030 POINT(37.51748336205259 -122.86648356217883) bank181030 +181031 POINT(38.080489626258725 -122.24230303800742) bank181031 +181032 POINT(38.104980072777316 -122.18084926170269) bank181032 +181033 POINT(38.41806524390994 -121.87252879856837) bank181033 +181034 POINT(37.52769759474028 -121.51682661873429) bank181034 +181035 POINT(37.78114828891997 -122.11573336721408) bank181035 +181036 POINT(38.569485326232574 -122.7770379581841) bank181036 +181037 POINT(37.18834912744627 -122.19687563216566) bank181037 +181038 POINT(38.182143689592365 -123.21206814144921) bank181038 +181039 POINT(38.44768853594378 -122.5053298597514) bank181039 +181040 POINT(37.879750226335794 -123.24296626212005) bank181040 +181041 POINT(37.89354879849376 -123.05828524221143) bank181041 +181042 POINT(36.89012341961737 -121.7490325627472) bank181042 +181043 POINT(37.69234172260094 -122.32113413890032) bank181043 +181044 POINT(38.07730014801971 -122.72468066525084) bank181044 +181045 POINT(38.42642197520641 -121.44367356541315) bank181045 +181046 POINT(38.49191099958767 -121.83609221193613) bank181046 +181047 POINT(38.25211439585424 -122.40300221066006) bank181047 +181048 POINT(37.00746798661554 -122.18107498886471) bank181048 +181049 POINT(37.55129007350998 -122.15958993647801) bank181049 +181050 POINT(36.82718807959527 -121.70472076830525) bank181050 +181051 POINT(37.59220554694901 -122.20678062633527) bank181051 +181052 POINT(38.04581764869621 -122.19415969950818) bank181052 +181053 POINT(38.63256405867482 -122.59052847132288) bank181053 +181054 POINT(37.370719043235304 -123.28904739500967) bank181054 +181055 POINT(37.90223024479067 -122.95395830124264) bank181055 +181056 POINT(37.73209721144362 -123.28464763760387) bank181056 +181057 POINT(38.71699773288851 -122.74589500917567) bank181057 +181058 POINT(37.027918625918424 -122.66863776376042) bank181058 +181059 POINT(38.55516154125179 -121.89960671003732) bank181059 +181060 POINT(38.272512636576344 -121.79143835563369) bank181060 +181061 POINT(37.980774159017926 -121.61322400599713) bank181061 +181062 POINT(37.51699194276526 -122.59893918879884) bank181062 +181063 POINT(37.81669796854306 -122.9051095603276) bank181063 +181064 POINT(38.18657279746657 -122.27936923572848) bank181064 +181065 POINT(37.68062300059619 -122.4851040885736) bank181065 +181066 POINT(37.07644408093666 -122.4665437331813) bank181066 +181067 POINT(37.42401271002412 -123.01609750915352) bank181067 +181068 POINT(38.67375546675397 -121.44964832618054) bank181068 +181069 POINT(36.85828840294589 -123.13954079826895) bank181069 +181070 POINT(38.51014790680134 -121.63686126979258) bank181070 +181071 POINT(37.28055262569659 -122.93853902734293) bank181071 +181072 POINT(38.50129243308355 -122.60628077452874) bank181072 +181073 POINT(38.62137843527317 -123.10671911339128) bank181073 +181074 POINT(38.541235231951845 -122.12251407142378) bank181074 +181075 POINT(37.23593272999399 -121.50666919092589) bank181075 +181076 POINT(37.93503205847266 -121.48068182694165) bank181076 +181077 POINT(37.14529138786322 -122.08733258205353) bank181077 +181078 POINT(38.097174425001825 -122.89787820909622) bank181078 +181079 POINT(36.795622326404796 -122.02655658184334) bank181079 +181080 POINT(37.04314681505579 -121.9813590450125) bank181080 +181081 POINT(37.55126933744441 -121.65546542070422) bank181081 +181082 POINT(37.520012030642015 -123.2081437587693) bank181082 +181083 POINT(37.48838931985352 -122.81816592957482) bank181083 +181084 POINT(37.417423145266824 -122.50029994566103) bank181084 +181085 POINT(38.72388975457038 -121.61761346664339) bank181085 +181086 POINT(37.094540928969465 -121.82275379162799) bank181086 +181087 POINT(37.766547546402315 -122.58446279798734) bank181087 +181088 POINT(38.69157897520257 -123.09895828078955) bank181088 +181089 POINT(37.361466937425575 -121.48019552525946) bank181089 +181090 POINT(37.85121046591321 -122.23930539980421) bank181090 +181091 POINT(36.992068320125014 -122.1893192970275) bank181091 +181092 POINT(38.44580369000034 -123.23761339484005) bank181092 +181093 POINT(38.56692034135929 -123.25348786885239) bank181093 +181094 POINT(38.34003345118073 -122.3117843173451) bank181094 +181095 POINT(36.79333651433925 -122.56567849939073) bank181095 +181096 POINT(37.62255947376179 -122.20511176984026) bank181096 +181097 POINT(36.917243932675085 -122.74211492915474) bank181097 +181098 POINT(38.35198275050503 -122.8225128283141) bank181098 +181099 POINT(37.895730623997466 -121.90516098044361) bank181099 +181100 POINT(37.680843516543504 -123.22000028992062) bank181100 +181101 POINT(37.04481961467352 -122.5742131619088) bank181101 +181102 POINT(38.62611117201788 -121.53272357024963) bank181102 +181103 POINT(37.6525535582264 -123.35089685760006) bank181103 +181104 POINT(36.987219222083425 -122.9740450450584) bank181104 +181105 POINT(37.661711850946816 -121.9944352478236) bank181105 +181106 POINT(37.9018425955211 -122.33666793502844) bank181106 +181107 POINT(37.24626146525035 -122.93283729001968) bank181107 +181108 POINT(37.352603800535384 -123.0795064355755) bank181108 +181109 POINT(38.49498997457179 -122.72475307851757) bank181109 +181110 POINT(38.45368401511361 -123.22535678280327) bank181110 +181111 POINT(38.162865695445376 -122.22702214120037) bank181111 +181112 POINT(37.60963318876349 -121.63747378890238) bank181112 +181113 POINT(38.57570045409323 -122.35738624024575) bank181113 +181114 POINT(38.21239766556431 -121.8730966990513) bank181114 +181115 POINT(37.2494539691624 -122.80006645202494) bank181115 +181116 POINT(37.179779078311554 -122.71682213987832) bank181116 +181117 POINT(36.78239726084614 -121.46507081352962) bank181117 +181118 POINT(38.12919456654229 -123.36743402384948) bank181118 +181119 POINT(37.01235294837285 -122.13276551829811) bank181119 +181120 POINT(37.60141710868311 -121.8079521729663) bank181120 +181121 POINT(37.85869595506364 -121.58761095991643) bank181121 +181122 POINT(37.08601111901097 -123.01819276641469) bank181122 +181123 POINT(37.776652085819876 -122.85167032125742) bank181123 +181124 POINT(37.79097211028824 -122.12410492906443) bank181124 +181125 POINT(37.09983803983449 -122.4070550479836) bank181125 +181126 POINT(37.84444831369147 -122.59697917763253) bank181126 +181127 POINT(38.67366663563816 -122.27088312509156) bank181127 +181128 POINT(37.717438741114044 -122.74189853261612) bank181128 +181129 POINT(37.949314142007815 -121.44866022977737) bank181129 +181130 POINT(37.15237669604885 -123.06521181948015) bank181130 +181131 POINT(37.208562444103976 -121.52912155945359) bank181131 +181132 POINT(38.630698867796745 -121.77930702975034) bank181132 +181133 POINT(37.70487496083949 -123.07846538546329) bank181133 +181134 POINT(38.00128357495611 -122.47347431594865) bank181134 +181135 POINT(37.95162755776543 -122.81898740888066) bank181135 +181136 POINT(38.34147497777416 -122.83580674767103) bank181136 +181137 POINT(37.73180127047686 -122.45138812946377) bank181137 +181138 POINT(37.86280378539287 -122.58456671739391) bank181138 +181139 POINT(37.489002612896996 -122.07075760715448) bank181139 +181140 POINT(37.78337925401366 -123.35085886916727) bank181140 +181141 POINT(37.81291405657527 -122.28088653636499) bank181141 +181142 POINT(38.75610760237485 -122.60028592238577) bank181142 +181143 POINT(38.05603023886165 -123.07629655764862) bank181143 +181144 POINT(37.68676848850631 -122.61501799457248) bank181144 +181145 POINT(38.442709466070816 -122.81580211337041) bank181145 +181146 POINT(38.74316795700424 -123.23374015653224) bank181146 +181147 POINT(38.74694662137749 -121.63456968899662) bank181147 +181148 POINT(38.25163700798354 -122.87266229220573) bank181148 +181149 POINT(37.13690531860598 -121.48065945912026) bank181149 +181150 POINT(38.35848364388504 -121.8050801397111) bank181150 +181151 POINT(37.665408769811144 -122.70489469758793) bank181151 +181152 POINT(37.96778875033953 -121.66213823428053) bank181152 +181153 POINT(38.16213174573375 -121.76942837231289) bank181153 +181154 POINT(37.23899136910681 -122.16210518012743) bank181154 +181155 POINT(37.04726816946738 -121.55276101616245) bank181155 +181156 POINT(37.27588945781649 -122.71070593012439) bank181156 +181157 POINT(37.29833912961353 -122.59145480453604) bank181157 +181158 POINT(37.51450757328335 -122.22106648874973) bank181158 +181159 POINT(36.995880581059524 -121.94056369952315) bank181159 +181160 POINT(37.51273740785666 -122.54712877500202) bank181160 +181161 POINT(36.990000632804794 -121.83723901818408) bank181161 +181162 POINT(37.708076756888765 -122.63929267215316) bank181162 +181163 POINT(36.83935841134809 -121.46547944587682) bank181163 +181164 POINT(38.70616222280575 -121.92248276098057) bank181164 +181165 POINT(37.99716226187809 -122.84532423133719) bank181165 +181166 POINT(37.48932219566912 -122.48585093109861) bank181166 +181167 POINT(37.889919782815554 -122.15566731127562) bank181167 +181168 POINT(37.84072513198704 -122.75015448323427) bank181168 +181169 POINT(38.73478373596382 -121.73764661444665) bank181169 +181170 POINT(37.70009419792313 -122.65977444716628) bank181170 +181171 POINT(38.641132909302165 -122.03024518003276) bank181171 +181172 POINT(38.69352109025661 -123.197636025857) bank181172 +181173 POINT(37.94311881931962 -122.1549825984023) bank181173 +181174 POINT(38.66068843694244 -122.60955629411619) bank181174 +181175 POINT(37.19825917139375 -123.3996344051549) bank181175 +181176 POINT(37.72620742232958 -121.41942271338542) bank181176 +181177 POINT(37.47836440710793 -122.68707761843139) bank181177 +181178 POINT(37.75666851452135 -122.12574271669997) bank181178 +181179 POINT(38.07886098616303 -122.75801140135162) bank181179 +181180 POINT(37.19752030986308 -122.23042775529285) bank181180 +181181 POINT(36.96504475746708 -121.53465109857599) bank181181 +181182 POINT(37.860674892324276 -123.20246651931436) bank181182 +181183 POINT(38.531219745385485 -122.99247947522947) bank181183 +181184 POINT(37.05369674218614 -122.69188741684621) bank181184 +181185 POINT(38.25227863893421 -121.96040258709745) bank181185 +181186 POINT(37.03689192607144 -122.52816342505865) bank181186 +181187 POINT(37.84240093006627 -122.5268170604689) bank181187 +181188 POINT(36.832900009174864 -121.51573854187244) bank181188 +181189 POINT(37.296134871568874 -123.13986948442876) bank181189 +181190 POINT(37.51527088363492 -123.40165499354961) bank181190 +181191 POINT(37.08426747870286 -122.487426826084) bank181191 +181192 POINT(37.28167490967852 -122.1758107914293) bank181192 +181193 POINT(38.11706859599128 -123.3371221951147) bank181193 +181194 POINT(38.06332312864626 -122.73945225977523) bank181194 +181195 POINT(38.620253973147065 -121.47821166155549) bank181195 +181196 POINT(37.28278260404444 -123.13827254491791) bank181196 +181197 POINT(37.410844173809764 -123.21064030109731) bank181197 +181198 POINT(37.17817095600503 -122.82587342895702) bank181198 +181199 POINT(37.932053347642494 -123.12395172276618) bank181199 +181200 POINT(38.685685534471105 -122.53437162771561) bank181200 +181201 POINT(37.57912929992345 -121.6368547430249) bank181201 +181202 POINT(37.35441183106603 -121.8832464879572) bank181202 +181203 POINT(37.28723683441337 -122.49693996824168) bank181203 +181204 POINT(38.54727893542471 -121.725326105979) bank181204 +181205 POINT(37.46259613641208 -121.4710581910376) bank181205 +181206 POINT(38.25740048345322 -123.31077324870672) bank181206 +181207 POINT(38.59790478588978 -122.92716758938603) bank181207 +181208 POINT(37.86928580720464 -122.30851868715709) bank181208 +181209 POINT(38.06785205439938 -122.85144822346126) bank181209 +181210 POINT(38.35884609943611 -122.62542027639819) bank181210 +181211 POINT(37.910290615095136 -121.91755356468148) bank181211 +181212 POINT(37.95208283458039 -122.98297871633987) bank181212 +181213 POINT(38.161817526981004 -123.02826873465654) bank181213 +181214 POINT(38.20334227746703 -121.78962917830526) bank181214 +181215 POINT(37.717391469475565 -122.99969861827594) bank181215 +181216 POINT(37.394990600886224 -122.89215049848964) bank181216 +181217 POINT(37.22565363728286 -122.62043211241682) bank181217 +181218 POINT(38.531840512190165 -122.39679481073044) bank181218 +181219 POINT(37.53810398091671 -122.8261576784128) bank181219 +181220 POINT(36.781296632428464 -122.82890346262997) bank181220 +181221 POINT(38.19539424393414 -122.29233615154808) bank181221 +181222 POINT(38.1225453619884 -123.16156468458627) bank181222 +181223 POINT(37.576615582762614 -121.62497976415807) bank181223 +181224 POINT(37.14372562102161 -122.72520264091298) bank181224 +181225 POINT(38.68321931762804 -122.42787655683053) bank181225 +181226 POINT(37.183763938135286 -122.266290444778) bank181226 +181227 POINT(37.49855376763909 -122.43475364330509) bank181227 +181228 POINT(38.316228912954244 -122.61260617777425) bank181228 +181229 POINT(37.464655867821136 -121.98261044520588) bank181229 +181230 POINT(38.13496148046119 -122.76643266723225) bank181230 +181231 POINT(38.662410803595755 -123.25895609560733) bank181231 +181232 POINT(37.12615895675708 -123.36254709487943) bank181232 +181233 POINT(37.304030094734586 -122.67702959860509) bank181233 +181234 POINT(37.22124944509417 -123.17153776138382) bank181234 +181235 POINT(38.739741773193636 -122.3866348166331) bank181235 +181236 POINT(36.943988344361806 -123.03012098182633) bank181236 +181237 POINT(38.26753604307266 -123.09837711096978) bank181237 +181238 POINT(37.10282591713383 -122.52064457744943) bank181238 +181239 POINT(37.361139269315984 -122.08465533099167) bank181239 +181240 POINT(38.40667923313323 -122.83717140217527) bank181240 +181241 POINT(37.72431497475856 -121.8954191714726) bank181241 +181242 POINT(38.28922459568834 -121.61461086706672) bank181242 +181243 POINT(38.270691989287066 -122.09826113223507) bank181243 +181244 POINT(38.08283942288059 -122.34006752459933) bank181244 +181245 POINT(37.495507734039506 -121.4376306173628) bank181245 +181246 POINT(38.25699722094083 -121.60953482451987) bank181246 +181247 POINT(37.20102764543596 -123.39709037583911) bank181247 +181248 POINT(36.995030332852004 -122.03791928953206) bank181248 +181249 POINT(38.19999779288285 -122.00246973246635) bank181249 +181250 POINT(38.39218946985935 -121.49150467779984) bank181250 +181251 POINT(36.977070587266645 -122.28294490600986) bank181251 +181252 POINT(38.008761447818834 -121.80434841196261) bank181252 +181253 POINT(37.7361057514289 -121.80487006739206) bank181253 +181254 POINT(37.425252553311125 -123.1551668224478) bank181254 +181255 POINT(37.504955271650225 -123.25765676310724) bank181255 +181256 POINT(38.593771785013274 -122.01803458527556) bank181256 +181257 POINT(37.104484721317554 -121.44394987440876) bank181257 +181258 POINT(38.38127614621491 -121.42671509868174) bank181258 +181259 POINT(37.19462186609774 -121.87868483722127) bank181259 +181260 POINT(37.111800476006515 -121.45196182457171) bank181260 +181261 POINT(38.73822082454994 -122.24331976718805) bank181261 +181262 POINT(38.73505112320675 -123.19421471022974) bank181262 +181263 POINT(37.84927820358593 -122.95134223032463) bank181263 +181264 POINT(38.62152531668078 -122.7998062087264) bank181264 +181265 POINT(37.80748390463874 -123.0343786948559) bank181265 +181266 POINT(37.335208571482966 -123.2455775560043) bank181266 +181267 POINT(37.75780874799472 -121.99240085462125) bank181267 +181268 POINT(37.42615761782023 -121.72914037632594) bank181268 +181269 POINT(36.92887643011862 -122.31640692314573) bank181269 +181270 POINT(37.89644517623514 -122.02660970295253) bank181270 +181271 POINT(38.540182783866086 -123.20896000385974) bank181271 +181272 POINT(37.927649773849524 -123.40315201808073) bank181272 +181273 POINT(38.017014239765224 -121.45660128339883) bank181273 +181274 POINT(38.0170764732898 -123.38486456083143) bank181274 +181275 POINT(38.60725112443614 -122.49794449160984) bank181275 +181276 POINT(37.799559683657314 -122.5893982339721) bank181276 +181277 POINT(36.87712805242175 -122.47242281996043) bank181277 +181278 POINT(38.313574763111454 -121.6808579717016) bank181278 +181279 POINT(37.45260110652769 -121.80563457228843) bank181279 +181280 POINT(37.73779315228885 -123.30013180917668) bank181280 +181281 POINT(37.81055188889412 -121.66227283410514) bank181281 +181282 POINT(37.425868299403106 -123.15626708616558) bank181282 +181283 POINT(38.146785922295805 -122.53657963444472) bank181283 +181284 POINT(38.612190159788526 -121.52846797010258) bank181284 +181285 POINT(38.496365170364655 -121.82349849443123) bank181285 +181286 POINT(38.66381315301466 -122.62039960871536) bank181286 +181287 POINT(38.48272290260127 -121.81711624775667) bank181287 +181288 POINT(37.94790848227879 -122.0729966978384) bank181288 +181289 POINT(37.90109655378381 -121.57968348569952) bank181289 +181290 POINT(37.8814329549421 -121.53476943964871) bank181290 +181291 POINT(38.037907187965345 -122.24361214894103) bank181291 +181292 POINT(37.01661834973805 -121.88722712509478) bank181292 +181293 POINT(37.71699901890665 -123.20159570859201) bank181293 +181294 POINT(37.505395677354635 -122.66739259149921) bank181294 +181295 POINT(38.289054008390245 -123.06892916653932) bank181295 +181296 POINT(37.197757780544514 -123.3438485124623) bank181296 +181297 POINT(37.40010042862063 -123.00258759252172) bank181297 +181298 POINT(36.85522968696499 -122.79788681189554) bank181298 +181299 POINT(37.00240722072147 -122.353662707065) bank181299 +181300 POINT(38.764717325748144 -122.28419853210944) bank181300 +181301 POINT(38.243495316922 -123.27706266885183) bank181301 +181302 POINT(36.939353173274164 -121.87320076033957) bank181302 +181303 POINT(37.68639590924856 -122.89233349737452) bank181303 +181304 POINT(37.76740979991423 -122.17720371281024) bank181304 +181305 POINT(37.27148680362315 -123.26799509153736) bank181305 +181306 POINT(38.1207965587891 -121.83735646013254) bank181306 +181307 POINT(37.870140836033585 -123.13710370571953) bank181307 +181308 POINT(36.79994578421841 -122.92100557071761) bank181308 +181309 POINT(38.06639026066313 -121.81397471193038) bank181309 +181310 POINT(37.891921464946655 -123.08228232997494) bank181310 +181311 POINT(38.374116898185854 -122.2919979870852) bank181311 +181312 POINT(37.147122223998224 -123.29301542286643) bank181312 +181313 POINT(37.07984980021456 -122.4425731596864) bank181313 +181314 POINT(37.86458663191489 -121.42205480050343) bank181314 +181315 POINT(37.570378849429346 -122.70123307479697) bank181315 +181316 POINT(36.80001142729727 -121.70243354598314) bank181316 +181317 POINT(37.935176604737684 -121.44052996215683) bank181317 +181318 POINT(38.1754397070336 -122.08598428939904) bank181318 +181319 POINT(36.93194101778391 -121.89138671173967) bank181319 +181320 POINT(37.405766303375124 -122.88721394598046) bank181320 +181321 POINT(38.15213262285581 -123.31339152414279) bank181321 +181322 POINT(38.42503455564007 -122.43836580165488) bank181322 +181323 POINT(37.559014155000895 -121.78079424674384) bank181323 +181324 POINT(37.33343726077763 -122.94376233621391) bank181324 +181325 POINT(37.906122730357275 -122.28468029948476) bank181325 +181326 POINT(38.54333761348609 -123.24681935676742) bank181326 +181327 POINT(37.37718945736126 -122.18029105231406) bank181327 +181328 POINT(37.1208282043313 -122.57782201536718) bank181328 +181329 POINT(37.44583887778631 -122.43048953279363) bank181329 +181330 POINT(38.29348305969918 -122.16664083405803) bank181330 +181331 POINT(38.228021079019356 -122.84325896736365) bank181331 +181332 POINT(37.48838769469415 -122.62161346760148) bank181332 +181333 POINT(37.78547885890439 -122.4852540022299) bank181333 +181334 POINT(38.299895331286656 -122.79919137882905) bank181334 +181335 POINT(37.301530014497295 -123.28594828721377) bank181335 +181336 POINT(37.0144668290126 -122.92738961244164) bank181336 +181337 POINT(38.63759680601031 -122.06765202114005) bank181337 +181338 POINT(38.60843546646524 -122.98265340987813) bank181338 +181339 POINT(38.33772242391868 -122.57203054213412) bank181339 +181340 POINT(38.054178405254206 -122.27521639173241) bank181340 +181341 POINT(38.66606387700972 -121.57123352585333) bank181341 +181342 POINT(37.19919089295238 -123.22331325143224) bank181342 +181343 POINT(38.602664412788265 -122.13434551528795) bank181343 +181344 POINT(38.512402673423125 -121.51839249041763) bank181344 +181345 POINT(36.78766499031895 -123.34683221119009) bank181345 +181346 POINT(37.19841681301781 -121.57825583094697) bank181346 +181347 POINT(38.73673507595057 -122.21338395731821) bank181347 +181348 POINT(38.53026845603111 -122.23059525450654) bank181348 +181349 POINT(36.886500889340674 -122.92311523921093) bank181349 +181350 POINT(36.800233209686645 -122.51144961585227) bank181350 +181351 POINT(38.491876462797514 -122.35500850579182) bank181351 +181352 POINT(38.12018696443199 -122.65908951079801) bank181352 +181353 POINT(37.0655424821832 -122.76545502032718) bank181353 +181354 POINT(37.56551922605863 -121.88709219135927) bank181354 +181355 POINT(38.39042345852308 -121.4683745822956) bank181355 +181356 POINT(38.4720505457612 -123.03321602262427) bank181356 +181357 POINT(37.74385937980241 -122.46456175974552) bank181357 +181358 POINT(37.6966325955659 -122.71499840569136) bank181358 +181359 POINT(38.288297391386514 -121.92148422025063) bank181359 +181360 POINT(37.50894202141273 -122.40129486727463) bank181360 +181361 POINT(38.617642462832265 -121.59978726946596) bank181361 +181362 POINT(37.132521549440945 -121.93757158472421) bank181362 +181363 POINT(38.60702831810528 -121.64008538480752) bank181363 +181364 POINT(36.82901565999335 -123.10741099062241) bank181364 +181365 POINT(38.535927633262965 -121.70049655436584) bank181365 +181366 POINT(37.13025300607817 -121.5015950127032) bank181366 +181367 POINT(38.430556965761575 -122.54398693517331) bank181367 +181368 POINT(38.02857625870513 -121.60381385560024) bank181368 +181369 POINT(38.69141768262705 -121.71900950585912) bank181369 +181370 POINT(37.50223905752768 -122.78741498912936) bank181370 +181371 POINT(38.069903132156625 -122.81094104870172) bank181371 +181372 POINT(37.2142278473883 -122.7360756735406) bank181372 +181373 POINT(37.178647327854385 -121.55363762844607) bank181373 +181374 POINT(37.86326309962224 -122.69461314690439) bank181374 +181375 POINT(37.43149919783832 -123.37153458828452) bank181375 +181376 POINT(38.729795352598025 -123.40440315428206) bank181376 +181377 POINT(38.37242500040588 -123.30686811036307) bank181377 +181378 POINT(36.884201484202805 -122.7137849810931) bank181378 +181379 POINT(37.6908742417106 -122.72440483452353) bank181379 +181380 POINT(37.064112219402276 -122.00667761354578) bank181380 +181381 POINT(37.474120657373334 -122.99775241613654) bank181381 +181382 POINT(38.74786122032344 -121.5153320052599) bank181382 +181383 POINT(38.66696053250288 -122.19867920912232) bank181383 +181384 POINT(38.61585519559137 -123.14065609996219) bank181384 +181385 POINT(36.8917930706822 -123.11804933479472) bank181385 +181386 POINT(36.90188825910426 -121.58539783822263) bank181386 +181387 POINT(37.77774986626212 -123.03440390344258) bank181387 +181388 POINT(37.97732937773475 -121.70790467920853) bank181388 +181389 POINT(38.093988021457335 -122.46721496748197) bank181389 +181390 POINT(37.88385577418408 -122.46440528490113) bank181390 +181391 POINT(37.89829817130704 -122.8347661307833) bank181391 +181392 POINT(37.021500638101756 -121.54571619971559) bank181392 +181393 POINT(38.21407516522858 -122.11908291325855) bank181393 +181394 POINT(38.0736612216143 -122.42403974277003) bank181394 +181395 POINT(36.94006560248546 -122.6249225338714) bank181395 +181396 POINT(37.07468961752427 -123.25089643688018) bank181396 +181397 POINT(36.96560256053825 -122.73434644609083) bank181397 +181398 POINT(36.80656406478417 -122.08326594075233) bank181398 +181399 POINT(36.846750181457864 -122.28987439447872) bank181399 +181400 POINT(38.442816251777074 -122.52379508775243) bank181400 +181401 POINT(36.98220973990728 -122.53168400474429) bank181401 +181402 POINT(38.49039957717524 -123.3792374076949) bank181402 +181403 POINT(37.02821616055516 -122.01259116221176) bank181403 +181404 POINT(38.295216247622214 -122.53979267984019) bank181404 +181405 POINT(38.60097542326251 -123.29100771513474) bank181405 +181406 POINT(37.13371949251608 -121.84641705344276) bank181406 +181407 POINT(36.97798415873186 -121.85330443660791) bank181407 +181408 POINT(38.43748303832127 -121.87404846755881) bank181408 +181409 POINT(37.27534180378244 -122.90259010188623) bank181409 +181410 POINT(37.82048292537043 -121.82017240772056) bank181410 +181411 POINT(38.456176768479196 -122.45955854703543) bank181411 +181412 POINT(37.625900743118734 -121.62982160045598) bank181412 +181413 POINT(37.59698658205306 -121.87172193625408) bank181413 +181414 POINT(38.02340112287906 -123.12981261980087) bank181414 +181415 POINT(38.57541441712449 -122.52772883113695) bank181415 +181416 POINT(37.39644811529568 -121.78763830051842) bank181416 +181417 POINT(37.62601059061097 -121.53598324255603) bank181417 +181418 POINT(38.31769320130951 -122.85369204920681) bank181418 +181419 POINT(38.55997820676498 -123.31967827303532) bank181419 +181420 POINT(36.93139075675152 -122.02441187567192) bank181420 +181421 POINT(37.50894141327743 -122.48045170119914) bank181421 +181422 POINT(36.832339565675966 -122.24558962514075) bank181422 +181423 POINT(37.09086615206037 -123.04248309507454) bank181423 +181424 POINT(38.311354743863035 -122.21984342839899) bank181424 +181425 POINT(37.21895699184569 -122.49836346674562) bank181425 +181426 POINT(37.340149425690285 -121.65990240564923) bank181426 +181427 POINT(37.03256684513158 -122.24690532677086) bank181427 +181428 POINT(38.36314452367425 -123.1028652269295) bank181428 +181429 POINT(37.74721267354522 -123.21960206554695) bank181429 +181430 POINT(38.21648074852024 -123.294690809726) bank181430 +181431 POINT(37.33790955390387 -122.03506433223711) bank181431 +181432 POINT(37.79607814241561 -122.34790642052694) bank181432 +181433 POINT(37.53892368065368 -123.3331525436163) bank181433 +181434 POINT(37.98767459986418 -123.08100986306061) bank181434 +181435 POINT(36.926487827535055 -122.86593263467036) bank181435 +181436 POINT(38.69555092900724 -122.29874682083323) bank181436 +181437 POINT(37.5155880331022 -121.50169836281245) bank181437 +181438 POINT(38.45666485449766 -121.46694381422954) bank181438 +181439 POINT(36.91544588102292 -123.06245111232096) bank181439 +181440 POINT(38.2075000966714 -122.61626833302292) bank181440 +181441 POINT(37.17406942675793 -122.28951488922597) bank181441 +181442 POINT(38.71551694132428 -122.83603135781843) bank181442 +181443 POINT(37.31553292836646 -123.36487520646968) bank181443 +181444 POINT(37.8423536273266 -122.02819275919045) bank181444 +181445 POINT(37.79827284270303 -121.89494901945788) bank181445 +181446 POINT(37.507101294605796 -122.30213747150844) bank181446 +181447 POINT(37.42607214344345 -122.65482679327502) bank181447 +181448 POINT(38.41823584522987 -122.49171959334255) bank181448 +181449 POINT(38.69012074170455 -122.19041801513765) bank181449 +181450 POINT(37.631348505227656 -123.16748952637732) bank181450 +181451 POINT(37.79583865570064 -121.7442617234202) bank181451 +181452 POINT(36.867641103783704 -122.86605322672551) bank181452 +181453 POINT(37.80834240864786 -121.71637529005781) bank181453 +181454 POINT(37.150902191147715 -121.4428455112465) bank181454 +181455 POINT(37.223613030602294 -122.22105819201525) bank181455 +181456 POINT(37.324458779144805 -122.30150094146552) bank181456 +181457 POINT(37.53621426430948 -122.76561347174663) bank181457 +181458 POINT(38.53742860790591 -121.94041155520208) bank181458 +181459 POINT(37.92865817216581 -121.67854670621244) bank181459 +181460 POINT(37.278494755011444 -122.55867412164093) bank181460 +181461 POINT(37.642383970622454 -121.72042451933397) bank181461 +181462 POINT(37.29145988271898 -123.15937747251424) bank181462 +181463 POINT(37.83770593623256 -121.89725930112019) bank181463 +181464 POINT(38.51077728594011 -121.46535118410459) bank181464 +181465 POINT(38.50022345762096 -122.55708373851965) bank181465 +181466 POINT(38.03093394225327 -121.77674968941622) bank181466 +181467 POINT(37.306406945875324 -123.29157385071852) bank181467 +181468 POINT(37.24151416736007 -121.58032812558586) bank181468 +181469 POINT(38.73134053707458 -121.73007870208245) bank181469 +181470 POINT(37.662680363763556 -121.73972943147388) bank181470 +181471 POINT(38.33532994548754 -122.61590961560258) bank181471 +181472 POINT(37.10074980445458 -121.51359845429583) bank181472 +181473 POINT(37.363301995410175 -121.84905799575077) bank181473 +181474 POINT(38.104878721119924 -122.38286381331498) bank181474 +181475 POINT(36.808290185795485 -123.19135816019539) bank181475 +181476 POINT(36.93115223859215 -121.46590437393267) bank181476 +181477 POINT(36.929513690804626 -123.30067649395832) bank181477 +181478 POINT(37.916334278026014 -122.15750608917266) bank181478 +181479 POINT(37.9992264201875 -121.72448576825495) bank181479 +181480 POINT(36.8642214466939 -123.41261809469265) bank181480 +181481 POINT(37.30254621645856 -122.84242292518904) bank181481 +181482 POINT(38.48555236045137 -123.18308124162074) bank181482 +181483 POINT(37.59375395116425 -121.70024462343945) bank181483 +181484 POINT(36.86430175444164 -122.64667857022303) bank181484 +181485 POINT(38.04570763386267 -121.4202797095031) bank181485 +181486 POINT(38.32390998244345 -122.24857791998858) bank181486 +181487 POINT(37.97863185856494 -122.2233202468771) bank181487 +181488 POINT(38.14962522529026 -121.96237442576829) bank181488 +181489 POINT(38.59114347135581 -123.17286984220141) bank181489 +181490 POINT(38.57949250891788 -123.40129351979051) bank181490 +181491 POINT(37.00832568515475 -123.17386123489456) bank181491 +181492 POINT(37.78595221777233 -122.52916899152359) bank181492 +181493 POINT(38.52135089748212 -122.9410413203496) bank181493 +181494 POINT(38.30884340460532 -122.88707816823435) bank181494 +181495 POINT(38.21493215466454 -122.7571704260907) bank181495 +181496 POINT(36.89025487829717 -122.12483775837896) bank181496 +181497 POINT(38.37210948759876 -122.4172482760392) bank181497 +181498 POINT(38.40447034840849 -122.46086047237654) bank181498 +181499 POINT(37.81142974409508 -122.17723824515429) bank181499 +181500 POINT(37.12620949427943 -122.55760053481562) bank181500 +181501 POINT(36.80678145320497 -122.67425166275041) bank181501 +181502 POINT(37.078793815092446 -122.18966585508227) bank181502 +181503 POINT(38.00324680936886 -122.64375786254553) bank181503 +181504 POINT(36.897923870114816 -121.54956241891051) bank181504 +181505 POINT(38.35576019462539 -123.0979488631851) bank181505 +181506 POINT(38.467264088347534 -122.53604929514361) bank181506 +181507 POINT(36.96315732666347 -121.85076419655329) bank181507 +181508 POINT(38.44595490711745 -123.22920308132625) bank181508 +181509 POINT(38.29712419037571 -122.34657907920176) bank181509 +181510 POINT(37.21565694055851 -123.3718426872964) bank181510 +181511 POINT(38.62363089732526 -123.36673765028543) bank181511 +181512 POINT(37.14991149955431 -123.38940848095173) bank181512 +181513 POINT(37.87117885968798 -123.27819656441943) bank181513 +181514 POINT(38.644710083549384 -123.05192356010947) bank181514 +181515 POINT(37.65863452684279 -122.83129300800682) bank181515 +181516 POINT(37.41472654879345 -121.93796909578202) bank181516 +181517 POINT(37.734567467044094 -122.63798531817572) bank181517 +181518 POINT(37.889593078004516 -123.04511865083141) bank181518 +181519 POINT(38.618155478074016 -121.82837770773985) bank181519 +181520 POINT(36.878268446279336 -122.77531246420442) bank181520 +181521 POINT(36.86853582138412 -123.16217182447883) bank181521 +181522 POINT(37.36220109930835 -122.61232756014749) bank181522 +181523 POINT(38.642789758922014 -123.02236202116103) bank181523 +181524 POINT(37.83019084408162 -122.3344555635172) bank181524 +181525 POINT(38.3168119564854 -122.90049435870844) bank181525 +181526 POINT(37.89808394820751 -122.63822293237617) bank181526 +181527 POINT(37.32199103475748 -123.0644866147785) bank181527 +181528 POINT(38.16919230507348 -123.39365241074358) bank181528 +181529 POINT(36.89237681577872 -122.87986540468457) bank181529 +181530 POINT(38.610171408748286 -123.39700396582114) bank181530 +181531 POINT(37.49792196808081 -121.97947276461991) bank181531 +181532 POINT(37.970379062224865 -121.71728011528452) bank181532 +181533 POINT(38.3242222995425 -122.942485355641) bank181533 +181534 POINT(38.03053116170243 -121.89023115684738) bank181534 +181535 POINT(37.922481727876374 -123.16520336330294) bank181535 +181536 POINT(38.504618998603284 -122.53120880765397) bank181536 +181537 POINT(38.01980955599251 -122.85459371751004) bank181537 +181538 POINT(37.0810800616458 -123.19885180973465) bank181538 +181539 POINT(38.49704883307613 -122.96043073127241) bank181539 +181540 POINT(37.08647859267069 -121.51907964502128) bank181540 +181541 POINT(37.914576002876125 -122.73560966129696) bank181541 +181542 POINT(37.39539715819015 -121.47276844981194) bank181542 +181543 POINT(38.211660500206236 -121.91944148683653) bank181543 +181544 POINT(38.00898317047202 -122.17358441425927) bank181544 +181545 POINT(36.86397301781652 -123.2512655587051) bank181545 +181546 POINT(38.03655098076789 -121.6088800235365) bank181546 +181547 POINT(36.95413762297188 -122.77094247150396) bank181547 +181548 POINT(38.30203896746805 -122.72268106883904) bank181548 +181549 POINT(38.28599227651661 -122.94128725355364) bank181549 +181550 POINT(36.84337807858667 -122.17695977671164) bank181550 +181551 POINT(37.98461247967051 -121.9803874760053) bank181551 +181552 POINT(37.24473098616345 -121.4218311494735) bank181552 +181553 POINT(38.47867649104191 -122.51620629142722) bank181553 +181554 POINT(38.11318505501415 -122.85784889294375) bank181554 +181555 POINT(38.22447216684242 -121.66235623965783) bank181555 +181556 POINT(38.265494805374125 -122.26836116739419) bank181556 +181557 POINT(38.09128097410558 -121.76689456070216) bank181557 +181558 POINT(38.74740141860524 -123.41406666305463) bank181558 +181559 POINT(38.33157813834317 -122.91333922000831) bank181559 +181560 POINT(38.58286517217752 -121.82502383513898) bank181560 +181561 POINT(37.900360235912046 -122.81637505085179) bank181561 +181562 POINT(38.39005251232438 -122.43540877693196) bank181562 +181563 POINT(37.94834174389453 -122.69630887225294) bank181563 +181564 POINT(37.60421753026589 -121.70519961606553) bank181564 +181565 POINT(38.64574999003544 -122.23439692542564) bank181565 +181566 POINT(37.97759011391938 -122.89864217057955) bank181566 +181567 POINT(38.3714650113644 -121.90986166837276) bank181567 +181568 POINT(38.62096197442641 -122.17922836591643) bank181568 +181569 POINT(38.36554253595957 -121.70119399541953) bank181569 +181570 POINT(37.966528020301645 -122.03440509594071) bank181570 +181571 POINT(38.17072627172472 -123.08673303524658) bank181571 +181572 POINT(36.87442156436937 -122.18364342014848) bank181572 +181573 POINT(38.71693123368824 -122.93426053995053) bank181573 +181574 POINT(37.328335904350354 -122.387323995775) bank181574 +181575 POINT(37.36904322497656 -121.62417701240545) bank181575 +181576 POINT(37.29763090579954 -121.52901960368203) bank181576 +181577 POINT(37.39978200754942 -121.49230099431863) bank181577 +181578 POINT(37.40309677831803 -122.71006006554205) bank181578 +181579 POINT(36.820298579340495 -123.18120826806674) bank181579 +181580 POINT(37.02806280488756 -122.12144018584092) bank181580 +181581 POINT(37.26648895758154 -122.99542026246598) bank181581 +181582 POINT(36.90286623565504 -121.6468613824907) bank181582 +181583 POINT(37.870153385550275 -122.29038975771935) bank181583 +181584 POINT(38.06663152980892 -122.61009024618565) bank181584 +181585 POINT(38.41951527704026 -121.71054592930322) bank181585 +181586 POINT(37.73589432030542 -122.93671726224315) bank181586 +181587 POINT(37.69106831996282 -121.88877018962978) bank181587 +181588 POINT(38.77214286074925 -121.8022287174965) bank181588 +181589 POINT(37.86684804791537 -121.63280955461757) bank181589 +181590 POINT(37.379602818311945 -122.04673467746848) bank181590 +181591 POINT(36.836491534002505 -121.86371252729057) bank181591 +181592 POINT(38.096849295999064 -123.12146341821247) bank181592 +181593 POINT(37.82250325866474 -121.78382812424154) bank181593 +181594 POINT(37.389735115882495 -121.94838534247047) bank181594 +181595 POINT(36.85019901068897 -121.96110233245045) bank181595 +181596 POINT(36.951528060571455 -121.82897232927861) bank181596 +181597 POINT(37.75765763706646 -121.89014598663606) bank181597 +181598 POINT(38.387466808787664 -123.09235661058979) bank181598 +181599 POINT(37.647036858007844 -121.68055584884884) bank181599 +181600 POINT(38.591002776940826 -121.58579303025135) bank181600 +181601 POINT(38.079481406186545 -122.23257881972545) bank181601 +181602 POINT(37.24997804534655 -122.0340170120806) bank181602 +181603 POINT(37.497970624142155 -122.06344814538288) bank181603 +181604 POINT(38.0472670524271 -122.75924298592145) bank181604 +181605 POINT(37.06670992457718 -122.91246752944153) bank181605 +181606 POINT(38.53083169541717 -122.43753006500559) bank181606 +181607 POINT(37.795055475354474 -121.80731614512881) bank181607 +181608 POINT(38.07030568016242 -122.02472115358219) bank181608 +181609 POINT(37.7854845721169 -122.60584741627079) bank181609 +181610 POINT(37.29607887692012 -122.6831086400744) bank181610 +181611 POINT(37.43910547256351 -123.2242961869123) bank181611 +181612 POINT(37.67245883974158 -121.85180423907167) bank181612 +181613 POINT(38.658840679476036 -122.43796111115483) bank181613 +181614 POINT(38.09076330633937 -122.29222083427646) bank181614 +181615 POINT(38.05090959586695 -123.27782600827726) bank181615 +181616 POINT(38.483294042091536 -122.89224331533183) bank181616 +181617 POINT(37.46709812541019 -123.03093554702436) bank181617 +181618 POINT(38.18616419267868 -121.55755918492277) bank181618 +181619 POINT(38.07899058727288 -123.24258155161318) bank181619 +181620 POINT(38.41098161891494 -121.90903661651737) bank181620 +181621 POINT(38.274357199937306 -122.03534278924776) bank181621 +181622 POINT(36.85117830531205 -121.55487768697287) bank181622 +181623 POINT(38.17736825209334 -122.08747902374641) bank181623 +181624 POINT(37.946576480969966 -121.84436170469836) bank181624 +181625 POINT(37.17765747419899 -121.86846061882052) bank181625 +181626 POINT(38.16568386118325 -123.37453472128949) bank181626 +181627 POINT(36.843681796250316 -122.23767411399079) bank181627 +181628 POINT(36.86225231301755 -122.97691690638197) bank181628 +181629 POINT(37.622364374562984 -122.92792476457426) bank181629 +181630 POINT(37.12178291583338 -122.63376240364393) bank181630 +181631 POINT(37.017813332515864 -122.96794735354108) bank181631 +181632 POINT(37.83540031148823 -122.71921394355051) bank181632 +181633 POINT(38.75082698399885 -122.67179346111804) bank181633 +181634 POINT(37.414651385826986 -121.46879122781475) bank181634 +181635 POINT(36.83295030216915 -121.7664588701834) bank181635 +181636 POINT(38.09449187716444 -121.68928399903982) bank181636 +181637 POINT(37.977186275664934 -121.62447047592485) bank181637 +181638 POINT(37.05844684771921 -123.03512164491326) bank181638 +181639 POINT(38.26674264065283 -123.39358744059686) bank181639 +181640 POINT(36.93915304476471 -123.05257043542204) bank181640 +181641 POINT(38.461729321498716 -122.76650696663866) bank181641 +181642 POINT(36.80585143388942 -121.91341904055021) bank181642 +181643 POINT(36.82013693370176 -122.2316399751186) bank181643 +181644 POINT(37.14290974552391 -122.41658844589323) bank181644 +181645 POINT(38.054351205553004 -122.44557392595051) bank181645 +181646 POINT(37.557516794773434 -122.30664986771737) bank181646 +181647 POINT(38.18863771168898 -121.5416276791977) bank181647 +181648 POINT(37.15624610291962 -121.4385327369472) bank181648 +181649 POINT(38.46460164151746 -121.88992853875445) bank181649 +181650 POINT(38.51582963645512 -121.54391898667836) bank181650 +181651 POINT(37.333729952571716 -121.70180769602871) bank181651 +181652 POINT(37.840076466299536 -121.67179165761797) bank181652 +181653 POINT(37.09134947192033 -122.8710517956597) bank181653 +181654 POINT(38.559703267486306 -123.18262755838508) bank181654 +181655 POINT(37.19270976180113 -122.67317037658084) bank181655 +181656 POINT(37.93896797131251 -122.62016346924992) bank181656 +181657 POINT(36.92448160345529 -121.49389277062413) bank181657 +181658 POINT(38.03349899052169 -121.79427564188343) bank181658 +181659 POINT(36.967833432750915 -122.9295439178434) bank181659 +181660 POINT(38.25066750606573 -122.78979419555182) bank181660 +181661 POINT(37.14245218231156 -122.6028949553953) bank181661 +181662 POINT(38.57004397853705 -122.5740037249156) bank181662 +181663 POINT(37.00731271644034 -122.6919081744826) bank181663 +181664 POINT(37.90495729117991 -122.32365209734185) bank181664 +181665 POINT(37.09772731553865 -121.87536239545169) bank181665 +181666 POINT(37.84221038912638 -123.13881465559443) bank181666 +181667 POINT(37.85982268095388 -122.82245784192976) bank181667 +181668 POINT(37.55079086222379 -122.857438592613) bank181668 +181669 POINT(37.348621111242466 -121.4448336860712) bank181669 +181670 POINT(37.93396787358737 -122.4207483392036) bank181670 +181671 POINT(37.21867480965992 -123.02542362861382) bank181671 +181672 POINT(37.18915338676641 -121.42311205822817) bank181672 +181673 POINT(38.28753386240778 -122.64617001149072) bank181673 +181674 POINT(36.93949941279614 -123.27022406780928) bank181674 +181675 POINT(37.5513673097039 -122.66338363504428) bank181675 +181676 POINT(37.03815734115932 -123.13875346044578) bank181676 +181677 POINT(36.90563078751204 -122.84060934431805) bank181677 +181678 POINT(38.624222556426275 -122.09046988893043) bank181678 +181679 POINT(38.60621687756663 -122.56947463903343) bank181679 +181680 POINT(38.4377668634861 -123.32437373344975) bank181680 +181681 POINT(38.093383044669146 -121.81250314934444) bank181681 +181682 POINT(37.86471558285717 -121.62191855857327) bank181682 +181683 POINT(38.69753237293424 -122.05613406246069) bank181683 +181684 POINT(38.4059302259788 -122.24576703989268) bank181684 +181685 POINT(38.00675445848081 -122.92448522350003) bank181685 +181686 POINT(36.79109019216445 -121.60237618984789) bank181686 +181687 POINT(37.32318030615285 -121.41962035967958) bank181687 +181688 POINT(38.412339482478764 -123.35788355982334) bank181688 +181689 POINT(37.934149482106584 -122.75226510310854) bank181689 +181690 POINT(36.84462135618323 -122.15218860559484) bank181690 +181691 POINT(37.604502611641166 -122.90403460547458) bank181691 +181692 POINT(38.323072218398536 -121.44499764680481) bank181692 +181693 POINT(37.17030729496586 -122.80513232706365) bank181693 +181694 POINT(37.14904726812802 -122.52973012808887) bank181694 +181695 POINT(38.49432834693999 -121.939446855883) bank181695 +181696 POINT(38.17791209743026 -121.56495102292203) bank181696 +181697 POINT(37.13935490975204 -121.65486741284549) bank181697 +181698 POINT(37.45654132549613 -123.36057990149455) bank181698 +181699 POINT(38.48586804645578 -123.27593446776137) bank181699 +181700 POINT(37.16428280978465 -123.33920791119306) bank181700 +181701 POINT(38.34231031592408 -122.61311752067637) bank181701 +181702 POINT(38.569521122030395 -123.28661091858658) bank181702 +181703 POINT(37.81235754730922 -121.75645906188885) bank181703 +181704 POINT(38.69245757212543 -122.4128492338062) bank181704 +181705 POINT(38.08963997710693 -122.15124183724954) bank181705 +181706 POINT(37.09143434154981 -122.47071825376844) bank181706 +181707 POINT(38.15887210654915 -122.88853062225594) bank181707 +181708 POINT(37.941718894426465 -121.64221712600774) bank181708 +181709 POINT(37.15144061120973 -122.47277271749137) bank181709 +181710 POINT(37.31099707716311 -121.52795225612348) bank181710 +181711 POINT(37.16801808256622 -122.97766031611695) bank181711 +181712 POINT(37.56764089750363 -122.0425057283867) bank181712 +181713 POINT(37.877470508078545 -122.0440629182876) bank181713 +181714 POINT(37.97895558049129 -121.55621356945268) bank181714 +181715 POINT(38.634474766357854 -121.8417686373754) bank181715 +181716 POINT(38.512968050178436 -122.73459368526424) bank181716 +181717 POINT(37.09851599049323 -123.28656727600135) bank181717 +181718 POINT(37.60257462939225 -121.64145255384153) bank181718 +181719 POINT(37.86861825769934 -123.19008174208683) bank181719 +181720 POINT(37.73112112157949 -123.35478366072121) bank181720 +181721 POINT(38.563388109458714 -122.55230173210242) bank181721 +181722 POINT(38.62448237878878 -122.73895971772872) bank181722 +181723 POINT(38.30033201051866 -122.68922584690202) bank181723 +181724 POINT(37.50087497937582 -122.82543194327391) bank181724 +181725 POINT(38.28883344931931 -121.77269786417632) bank181725 +181726 POINT(37.07195957167536 -123.36870114083405) bank181726 +181727 POINT(37.161315454893625 -123.2761918546157) bank181727 +181728 POINT(37.0139663861674 -122.47430199901497) bank181728 +181729 POINT(38.11726195999445 -122.88243534883117) bank181729 +181730 POINT(37.99998730241538 -122.08427525189374) bank181730 +181731 POINT(37.401839255611876 -122.24234648177439) bank181731 +181732 POINT(37.57367791804671 -121.61717260863607) bank181732 +181733 POINT(38.50778146658816 -122.97995803708847) bank181733 +181734 POINT(37.39153508812643 -123.33954615289511) bank181734 +181735 POINT(38.386235078319096 -121.68394415429516) bank181735 +181736 POINT(37.272172115497234 -122.5240357984671) bank181736 +181737 POINT(37.39424336885758 -122.50046139043216) bank181737 +181738 POINT(36.93853930694503 -122.96035957285797) bank181738 +181739 POINT(37.004143679936256 -123.36233239567679) bank181739 +181740 POINT(37.522470671947275 -122.46063161491844) bank181740 +181741 POINT(38.47894011443877 -123.36837674027822) bank181741 +181742 POINT(38.15541497454731 -122.27871630228384) bank181742 +181743 POINT(37.61080234680075 -123.02325090597927) bank181743 +181744 POINT(37.86473152848904 -122.40701697563368) bank181744 +181745 POINT(37.42782769106459 -122.36380012514711) bank181745 +181746 POINT(37.14445052978167 -122.30772335615269) bank181746 +181747 POINT(36.7830132846226 -122.80213746189368) bank181747 +181748 POINT(38.587169267929696 -123.10951640243717) bank181748 +181749 POINT(37.81908993662955 -123.35268857192598) bank181749 +181750 POINT(38.58047793307744 -123.39893387339457) bank181750 +181751 POINT(36.963445882646084 -123.27700459141488) bank181751 +181752 POINT(37.24049474297457 -123.05476839784573) bank181752 +181753 POINT(38.076122422530375 -121.52917019741275) bank181753 +181754 POINT(38.72775840772988 -121.49061739280694) bank181754 +181755 POINT(37.18790441409696 -123.30395982091954) bank181755 +181756 POINT(37.83241800454614 -122.6804875935994) bank181756 +181757 POINT(38.72122803603999 -123.09929143166036) bank181757 +181758 POINT(38.19273013977 -122.10941379486158) bank181758 +181759 POINT(38.083664947546815 -121.70006024996702) bank181759 +181760 POINT(38.67148646028497 -121.52178088707647) bank181760 +181761 POINT(38.186798083551864 -123.18588849660466) bank181761 +181762 POINT(38.33466833638105 -123.37574369013876) bank181762 +181763 POINT(38.626075402664746 -122.07493596555166) bank181763 +181764 POINT(38.094808530493836 -122.95579922543048) bank181764 +181765 POINT(36.9088978391955 -121.7288929838301) bank181765 +181766 POINT(37.42516000548535 -122.34162337257855) bank181766 +181767 POINT(37.90351381051422 -122.72568399083603) bank181767 +181768 POINT(38.64217698488909 -121.66706240615953) bank181768 +181769 POINT(37.522115219702265 -121.56180899540448) bank181769 +181770 POINT(37.888596209897074 -121.70666755947931) bank181770 +181771 POINT(38.74019699692571 -122.39917814564421) bank181771 +181772 POINT(37.64815798112646 -122.1529866768158) bank181772 +181773 POINT(37.352029047836986 -123.02532696353626) bank181773 +181774 POINT(37.72863375366071 -122.57559933785012) bank181774 +181775 POINT(36.970235762102654 -122.3511770175548) bank181775 +181776 POINT(37.51733791257652 -122.90928492622079) bank181776 +181777 POINT(38.08148833924836 -121.78022934991381) bank181777 +181778 POINT(37.26158072047053 -122.62120044208136) bank181778 +181779 POINT(38.444926976032086 -123.31174145050984) bank181779 +181780 POINT(38.58022823424373 -123.01651068407173) bank181780 +181781 POINT(38.4270939904994 -121.7551876779585) bank181781 +181782 POINT(37.12635032515381 -122.13952144233087) bank181782 +181783 POINT(38.6616167443836 -122.0936136991493) bank181783 +181784 POINT(38.149067129460086 -122.16376008005908) bank181784 +181785 POINT(37.233409283638835 -122.45420103818748) bank181785 +181786 POINT(38.718543205039126 -121.77603925360545) bank181786 +181787 POINT(38.684792801781036 -122.3194725051416) bank181787 +181788 POINT(37.94422168512778 -122.08469188983524) bank181788 +181789 POINT(37.8633788366663 -122.01537310729124) bank181789 +181790 POINT(36.81781077380308 -123.18465461599912) bank181790 +181791 POINT(37.57600795926524 -123.27449412376157) bank181791 +181792 POINT(37.332586131285396 -122.11189804449513) bank181792 +181793 POINT(36.91806817545635 -122.71303987480039) bank181793 +181794 POINT(37.64544202459699 -121.713685920041) bank181794 +181795 POINT(38.250512183065624 -123.41119771161259) bank181795 +181796 POINT(36.90912489971002 -122.71837855316257) bank181796 +181797 POINT(37.643141271242236 -123.18619950670978) bank181797 +181798 POINT(36.852434998111 -122.56773389248949) bank181798 +181799 POINT(38.56129974152116 -122.83550334729298) bank181799 +181800 POINT(37.19339308465551 -122.7919546940514) bank181800 +181801 POINT(38.58290698971255 -121.51090129009054) bank181801 +181802 POINT(38.53127656443017 -122.6497668381385) bank181802 +181803 POINT(37.95189148491251 -122.81246826057303) bank181803 +181804 POINT(38.12929696466696 -122.97419852426526) bank181804 +181805 POINT(37.33464919898441 -123.28819890222215) bank181805 +181806 POINT(37.70487783333256 -121.74123432727215) bank181806 +181807 POINT(38.221245649917336 -121.78492157301937) bank181807 +181808 POINT(38.44660941728909 -122.00911493550507) bank181808 +181809 POINT(37.75070242663451 -121.84710329958328) bank181809 +181810 POINT(38.34955365161459 -122.27306626813302) bank181810 +181811 POINT(38.308596099924465 -122.96102862420217) bank181811 +181812 POINT(38.155457652255954 -122.27432192784212) bank181812 +181813 POINT(38.06464600303895 -121.59935436207797) bank181813 +181814 POINT(37.964458765789594 -122.93097370373668) bank181814 +181815 POINT(37.9748627561215 -122.47181885444834) bank181815 +181816 POINT(38.2056828783335 -122.9163028067973) bank181816 +181817 POINT(38.550584361884844 -121.47589442602238) bank181817 +181818 POINT(37.00786745246479 -122.0871238710341) bank181818 +181819 POINT(38.772369826579755 -121.87191431591602) bank181819 +181820 POINT(38.06585074884385 -121.49230758893083) bank181820 +181821 POINT(36.90770507539632 -121.49349696482928) bank181821 +181822 POINT(37.73111869677516 -122.14071290345721) bank181822 +181823 POINT(37.23237238978584 -122.52649096461425) bank181823 +181824 POINT(37.02662151227428 -121.47593631555208) bank181824 +181825 POINT(37.49790306571141 -123.1815860764193) bank181825 +181826 POINT(38.739439456733436 -122.16917759082921) bank181826 +181827 POINT(37.9470117832098 -122.18899983921368) bank181827 +181828 POINT(38.74765597561705 -122.44884505208525) bank181828 +181829 POINT(38.41177994639852 -122.15655619128384) bank181829 +181830 POINT(37.73889320108271 -123.05061479514006) bank181830 +181831 POINT(36.90523541060672 -121.4856683910819) bank181831 +181832 POINT(37.382519619351186 -123.16426520768884) bank181832 +181833 POINT(38.08625874011031 -122.94442971920263) bank181833 +181834 POINT(38.018877364101535 -123.26357590378245) bank181834 +181835 POINT(37.19830407001921 -121.5747589157837) bank181835 +181836 POINT(38.28541595764291 -122.65540252303559) bank181836 +181837 POINT(38.223847587910114 -122.22771738373014) bank181837 +181838 POINT(37.17139530995459 -121.59137374127971) bank181838 +181839 POINT(37.562700475396944 -123.33243804722235) bank181839 +181840 POINT(37.66830167129213 -123.18754117506992) bank181840 +181841 POINT(37.9601349262742 -122.2587318882621) bank181841 +181842 POINT(38.55546948262673 -122.04224177214131) bank181842 +181843 POINT(37.081957644705035 -123.24866822986908) bank181843 +181844 POINT(38.50655464624788 -122.64017853465309) bank181844 +181845 POINT(38.749686824358534 -123.05838564416315) bank181845 +181846 POINT(37.06235817287229 -123.2653436991044) bank181846 +181847 POINT(37.17014356368436 -122.07532739003564) bank181847 +181848 POINT(38.75078573994357 -121.96421110513167) bank181848 +181849 POINT(37.14051244799634 -122.78226972911692) bank181849 +181850 POINT(37.17983680627448 -122.28006200689298) bank181850 +181851 POINT(37.06129834270099 -122.21340182477275) bank181851 +181852 POINT(38.6077949278297 -122.68406593541737) bank181852 +181853 POINT(37.44115962709696 -121.8413250703694) bank181853 +181854 POINT(38.41722767107448 -122.83038361903002) bank181854 +181855 POINT(38.05108380435665 -122.43387751424697) bank181855 +181856 POINT(37.51785695087939 -122.0562232430192) bank181856 +181857 POINT(37.640594705254365 -122.90583290391871) bank181857 +181858 POINT(38.165527940509456 -123.03993065636381) bank181858 +181859 POINT(36.85705854766627 -122.449386403833) bank181859 +181860 POINT(37.48034669555107 -123.07683075626102) bank181860 +181861 POINT(37.57289079048585 -122.30772029757982) bank181861 +181862 POINT(37.809520877071705 -123.15059005715558) bank181862 +181863 POINT(37.37826732325516 -121.88695257774147) bank181863 +181864 POINT(37.930196886154505 -121.74958684543643) bank181864 +181865 POINT(38.00922280298155 -122.07444976229023) bank181865 +181866 POINT(37.90374428769259 -123.41882174241414) bank181866 +181867 POINT(36.905891309023374 -122.21227772403154) bank181867 +181868 POINT(38.62934668866265 -122.34436573427337) bank181868 +181869 POINT(37.047213390739316 -122.40162231316397) bank181869 +181870 POINT(37.35656548250156 -122.55199325275862) bank181870 +181871 POINT(37.20193637846587 -123.02180761308753) bank181871 +181872 POINT(37.48862046054665 -123.05148258552676) bank181872 +181873 POINT(37.199928513926785 -122.92147766191218) bank181873 +181874 POINT(38.186023982940576 -121.72294303181076) bank181874 +181875 POINT(38.69543463205632 -123.02311145405874) bank181875 +181876 POINT(37.62107294171269 -121.54348421323978) bank181876 +181877 POINT(36.77807655938632 -122.22060233836253) bank181877 +181878 POINT(37.55442138260137 -121.4498890471317) bank181878 +181879 POINT(37.19649488709181 -122.41543532109492) bank181879 +181880 POINT(38.20657496249983 -122.7917568420455) bank181880 +181881 POINT(38.45622124426465 -122.46700607389266) bank181881 +181882 POINT(37.52674765000119 -123.16341878943908) bank181882 +181883 POINT(37.51296856567267 -122.37560617047181) bank181883 +181884 POINT(38.72187902017182 -122.18233078256006) bank181884 +181885 POINT(38.533878429213466 -123.1703456621041) bank181885 +181886 POINT(38.48327611221836 -122.26897724979071) bank181886 +181887 POINT(37.38770347375103 -123.40141676622936) bank181887 +181888 POINT(37.557518436728344 -121.77590148260654) bank181888 +181889 POINT(37.997579745189576 -122.45833939815107) bank181889 +181890 POINT(38.026064625847795 -122.57060497222578) bank181890 +181891 POINT(37.85643140545911 -121.60611336096122) bank181891 +181892 POINT(38.57951649181366 -122.05953892305877) bank181892 +181893 POINT(37.07904504029899 -122.8706680540518) bank181893 +181894 POINT(38.09904713973227 -122.27198880251073) bank181894 +181895 POINT(37.76970499873787 -122.05095411147646) bank181895 +181896 POINT(36.892879153765605 -121.87584007694842) bank181896 +181897 POINT(38.50036305799193 -122.37413308248655) bank181897 +181898 POINT(37.3810660301252 -122.74292718889461) bank181898 +181899 POINT(37.48884500237862 -123.04428380885341) bank181899 +181900 POINT(37.281683344501026 -122.02302579792503) bank181900 +181901 POINT(37.87891489972917 -122.4394008222939) bank181901 +181902 POINT(38.241055848395376 -122.13415807387419) bank181902 +181903 POINT(37.90552646947335 -122.69025333392639) bank181903 +181904 POINT(38.611674745268665 -122.61636746942771) bank181904 +181905 POINT(38.00532126340591 -123.3947134096302) bank181905 +181906 POINT(37.268959364983964 -121.64025865586503) bank181906 +181907 POINT(36.9779064218348 -122.48219503377899) bank181907 +181908 POINT(38.592222988122444 -122.47802757545739) bank181908 +181909 POINT(37.241647442804926 -123.32568098111442) bank181909 +181910 POINT(37.606067375065344 -122.15353978266441) bank181910 +181911 POINT(38.050327126342765 -122.47525859769287) bank181911 +181912 POINT(38.212932794703924 -121.66865681819667) bank181912 +181913 POINT(38.46908180455144 -121.97220793001694) bank181913 +181914 POINT(37.28586139538889 -122.95307274506044) bank181914 +181915 POINT(36.82231563672972 -121.75510842239699) bank181915 +181916 POINT(37.91595761966583 -121.7970042888109) bank181916 +181917 POINT(38.3330531656249 -122.92295330771307) bank181917 +181918 POINT(36.86478049376075 -123.402604901327) bank181918 +181919 POINT(37.333617515672024 -121.44540763029727) bank181919 +181920 POINT(38.19850013536841 -121.92409976173163) bank181920 +181921 POINT(38.352895977786964 -122.38558131424695) bank181921 +181922 POINT(37.435161708314396 -122.44270994583054) bank181922 +181923 POINT(37.50959224532883 -123.33067880475056) bank181923 +181924 POINT(37.55785982005373 -122.5440298031055) bank181924 +181925 POINT(37.37028456313907 -123.22244719511122) bank181925 +181926 POINT(37.065732833345024 -123.32604483301806) bank181926 +181927 POINT(36.86762737649917 -121.62974687891936) bank181927 +181928 POINT(37.435863223099425 -122.95391860897374) bank181928 +181929 POINT(38.33196684400054 -122.20152114286437) bank181929 +181930 POINT(37.89233620424875 -122.10690038749466) bank181930 +181931 POINT(37.680669775445175 -121.82881480513089) bank181931 +181932 POINT(37.82160409137057 -123.22638658817357) bank181932 +181933 POINT(37.93231674439866 -122.10383807931956) bank181933 +181934 POINT(37.54692523322321 -122.93113284914605) bank181934 +181935 POINT(37.88338233471167 -121.69245974364497) bank181935 +181936 POINT(36.775151927146375 -122.34825033670266) bank181936 +181937 POINT(37.05352601895291 -122.24650096626596) bank181937 +181938 POINT(38.64334336096376 -123.25563713702145) bank181938 +181939 POINT(37.146596696785444 -123.08473145963319) bank181939 +181940 POINT(37.695383057457654 -122.45021443182814) bank181940 +181941 POINT(37.95723210262766 -123.06608115287317) bank181941 +181942 POINT(37.62287058873651 -121.50624036081105) bank181942 +181943 POINT(38.32459883851949 -122.12865588287524) bank181943 +181944 POINT(38.025915438795764 -121.58611882932686) bank181944 +181945 POINT(37.237731984578716 -122.10885993983239) bank181945 +181946 POINT(38.77027201459453 -121.46795840262668) bank181946 +181947 POINT(38.671095431032654 -123.17577508774527) bank181947 +181948 POINT(38.38333309393569 -121.69636110380233) bank181948 +181949 POINT(37.79218733294634 -121.43325888597221) bank181949 +181950 POINT(38.65736935869109 -122.54091136762021) bank181950 +181951 POINT(37.83038609140159 -122.97723554437171) bank181951 +181952 POINT(38.68976609290484 -122.82046577368162) bank181952 +181953 POINT(38.19731135614393 -121.83911459470309) bank181953 +181954 POINT(37.219309908552894 -122.99112747352555) bank181954 +181955 POINT(36.84734169091586 -122.32367068755472) bank181955 +181956 POINT(37.35840459000759 -122.7342529048562) bank181956 +181957 POINT(37.264960992033096 -122.69196548478767) bank181957 +181958 POINT(38.120827519183344 -122.2629134481477) bank181958 +181959 POINT(37.39540949037977 -122.59713412623981) bank181959 +181960 POINT(37.73348841145096 -123.21425934276208) bank181960 +181961 POINT(38.41414947537331 -122.97584143661281) bank181961 +181962 POINT(37.21622850242632 -122.57440359834108) bank181962 +181963 POINT(38.32605115521091 -123.01449703373778) bank181963 +181964 POINT(38.632632089053814 -121.77199066337444) bank181964 +181965 POINT(37.90201053191054 -122.7149050982479) bank181965 +181966 POINT(37.84120330650918 -121.7323219385536) bank181966 +181967 POINT(37.6588687452933 -122.95218593848809) bank181967 +181968 POINT(38.56226139948051 -122.13458175849895) bank181968 +181969 POINT(37.73189600569658 -121.90026867758063) bank181969 +181970 POINT(36.85760866162381 -122.26944677168363) bank181970 +181971 POINT(37.247135078524465 -123.29807480355433) bank181971 +181972 POINT(37.041114581243214 -121.71729806434423) bank181972 +181973 POINT(38.74151527743729 -123.16415521980794) bank181973 +181974 POINT(36.88629346336095 -121.46827195441242) bank181974 +181975 POINT(38.59545874559707 -122.11455053776197) bank181975 +181976 POINT(38.52373677838676 -123.29364684980801) bank181976 +181977 POINT(36.94495205790789 -121.85116594937995) bank181977 +181978 POINT(37.60269933307393 -123.06023856966787) bank181978 +181979 POINT(37.391471122100334 -122.28427842809445) bank181979 +181980 POINT(37.169110220473485 -121.86152085489167) bank181980 +181981 POINT(37.058458970962846 -121.7450977033814) bank181981 +181982 POINT(36.905284927269385 -122.51767946928766) bank181982 +181983 POINT(36.91352874805802 -122.87185262071021) bank181983 +181984 POINT(36.907054675847725 -122.09360314142847) bank181984 +181985 POINT(38.099199062870554 -122.62617938132719) bank181985 +181986 POINT(37.033090867426246 -123.2339742189494) bank181986 +181987 POINT(36.95420365175834 -122.89497597341686) bank181987 +181988 POINT(36.80155517918215 -121.97794850275264) bank181988 +181989 POINT(36.967473302215915 -121.72551486458062) bank181989 +181990 POINT(37.6260342923725 -121.70509582376512) bank181990 +181991 POINT(37.79407708967866 -122.49641357115277) bank181991 +181992 POINT(37.2769945397454 -122.31241662680019) bank181992 +181993 POINT(37.0378900087366 -121.53072316825168) bank181993 +181994 POINT(37.460469499780416 -122.30243545038039) bank181994 +181995 POINT(36.83159684313912 -123.02121518635644) bank181995 +181996 POINT(37.821781945008595 -123.11840926615137) bank181996 +181997 POINT(36.801387708955936 -123.31227444965617) bank181997 +181998 POINT(37.45641777307622 -123.40922132024) bank181998 +181999 POINT(38.75402834170103 -121.80968411155449) bank181999 +182000 POINT(36.934370232258765 -121.43536132454483) bank182000 +182001 POINT(37.192110590461525 -122.20198960736468) bank182001 +182002 POINT(36.832758445460776 -123.18941248231665) bank182002 +182003 POINT(38.39135327484987 -121.45767288331673) bank182003 +182004 POINT(37.272849896106166 -123.20504119945166) bank182004 +182005 POINT(36.81907042582913 -123.37733655061493) bank182005 +182006 POINT(38.75614765181855 -121.64942505534798) bank182006 +182007 POINT(37.325101658552654 -123.22186097081446) bank182007 +182008 POINT(37.98833113640461 -122.48502528570454) bank182008 +182009 POINT(37.343748350240986 -123.27654879928585) bank182009 +182010 POINT(37.31099436068333 -121.62106086033596) bank182010 +182011 POINT(38.08360366826311 -122.93688355457377) bank182011 +182012 POINT(38.25575769064816 -122.92100340826137) bank182012 +182013 POINT(38.18208631226943 -122.79017206993362) bank182013 +182014 POINT(37.158355264249245 -121.84538373509419) bank182014 +182015 POINT(38.56905692132618 -121.67083060253562) bank182015 +182016 POINT(38.46164950290257 -121.59559730578678) bank182016 +182017 POINT(37.64349745044695 -122.10835735713691) bank182017 +182018 POINT(38.07052953314239 -122.05398168114857) bank182018 +182019 POINT(38.56731687413041 -123.40649502769078) bank182019 +182020 POINT(37.92359956464984 -123.26256769570358) bank182020 +182021 POINT(38.57404820278925 -123.05443823234027) bank182021 +182022 POINT(38.47053784978384 -122.29500387587298) bank182022 +182023 POINT(36.85992421033907 -123.05816203803306) bank182023 +182024 POINT(37.92090302921952 -121.98565549782732) bank182024 +182025 POINT(37.352528890566695 -121.53517083811552) bank182025 +182026 POINT(37.94324750204692 -122.10078643065641) bank182026 +182027 POINT(37.12922050431124 -122.1497067142702) bank182027 +182028 POINT(37.90682852378298 -122.47334764963668) bank182028 +182029 POINT(37.32495935516178 -122.06845083169713) bank182029 +182030 POINT(38.250114440351695 -123.17012242708626) bank182030 +182031 POINT(36.93993218729968 -123.09055291832657) bank182031 +182032 POINT(38.75232147859147 -121.55080155106631) bank182032 +182033 POINT(38.693983762411406 -121.77575114351892) bank182033 +182034 POINT(38.62851575836155 -122.02349272650204) bank182034 +182035 POINT(37.262509944878346 -122.85948321337399) bank182035 +182036 POINT(37.38762231256685 -121.47888063034934) bank182036 +182037 POINT(37.46999619761407 -121.55151991923361) bank182037 +182038 POINT(37.683206646584054 -123.27312923132143) bank182038 +182039 POINT(37.25892530695448 -123.14320124916175) bank182039 +182040 POINT(38.461945165855056 -121.89531424776797) bank182040 +182041 POINT(37.905663957872434 -121.67035630187729) bank182041 +182042 POINT(37.781257673864744 -121.65135462234325) bank182042 +182043 POINT(37.38942443446553 -122.8744285409755) bank182043 +182044 POINT(37.29289901899482 -123.19932731947775) bank182044 +182045 POINT(36.8698561773761 -123.11373766586858) bank182045 +182046 POINT(37.453779510897306 -121.59844817337611) bank182046 +182047 POINT(38.188705398066105 -121.89663445119574) bank182047 +182048 POINT(38.71121652654371 -121.60453118792098) bank182048 +182049 POINT(37.40297376805978 -123.06926255405645) bank182049 +182050 POINT(37.545106492362635 -123.41032229163478) bank182050 +182051 POINT(38.60862375939629 -122.47789315939309) bank182051 +182052 POINT(38.27354761558644 -121.80518584536323) bank182052 +182053 POINT(38.44796303283568 -122.27054618007892) bank182053 +182054 POINT(37.76942134183552 -122.96171440391578) bank182054 +182055 POINT(36.94148281733515 -123.27686953675224) bank182055 +182056 POINT(37.19812193224861 -122.7720319297168) bank182056 +182057 POINT(38.61996865777548 -123.06072972366914) bank182057 +182058 POINT(37.241720491393224 -123.38398774336544) bank182058 +182059 POINT(37.83135002708855 -121.70890990881415) bank182059 +182060 POINT(37.00230922693124 -123.24288207146085) bank182060 +182061 POINT(37.62877491211492 -122.6298916754994) bank182061 +182062 POINT(38.67806585942887 -122.80787932611426) bank182062 +182063 POINT(37.51677413183983 -121.84911279458268) bank182063 +182064 POINT(37.93522250829642 -122.291976129729) bank182064 +182065 POINT(37.13039252261878 -122.35415165048342) bank182065 +182066 POINT(38.27828765180466 -123.00588221031279) bank182066 +182067 POINT(37.29604203199172 -122.14449218749374) bank182067 +182068 POINT(37.1664804753674 -122.87990974647859) bank182068 +182069 POINT(38.09295482912874 -121.85189773003896) bank182069 +182070 POINT(37.860590930292915 -122.58309894635477) bank182070 +182071 POINT(37.62055826052759 -122.90537787496203) bank182071 +182072 POINT(37.63843087515127 -122.36823207698671) bank182072 +182073 POINT(37.16842059173801 -122.57469552879098) bank182073 +182074 POINT(38.61791807039978 -121.56112704844313) bank182074 +182075 POINT(37.9650139901119 -123.3108480220759) bank182075 +182076 POINT(38.431369772319464 -121.97721648283456) bank182076 +182077 POINT(37.39056863344848 -122.39894553585704) bank182077 +182078 POINT(37.50282240313596 -121.76503731393484) bank182078 +182079 POINT(36.794174505028515 -121.60392408940902) bank182079 +182080 POINT(37.49826393143026 -121.71653668092205) bank182080 +182081 POINT(37.36548623797507 -122.32281265291275) bank182081 +182082 POINT(38.0511379456304 -122.77027628751624) bank182082 +182083 POINT(37.24987213034409 -122.9907931707265) bank182083 +182084 POINT(37.67835385831638 -122.40925142093188) bank182084 +182085 POINT(36.864557434992314 -122.78730822505982) bank182085 +182086 POINT(38.692496877711804 -121.47257239577159) bank182086 +182087 POINT(37.046563896094185 -123.09283176814745) bank182087 +182088 POINT(38.138870157811255 -122.32567922453407) bank182088 +182089 POINT(37.36928952627844 -121.74564569802287) bank182089 +182090 POINT(38.77077593774186 -122.32880416093622) bank182090 +182091 POINT(38.213914371330475 -123.26734227026773) bank182091 +182092 POINT(37.55381276334664 -121.43074811631011) bank182092 +182093 POINT(37.120532762204434 -121.97824550595014) bank182093 +182094 POINT(36.81941631880668 -121.85067589981978) bank182094 +182095 POINT(37.38350893169244 -121.4575191496605) bank182095 +182096 POINT(37.45743819540806 -122.81690372448453) bank182096 +182097 POINT(36.951937150636766 -123.13338378217091) bank182097 +182098 POINT(38.06424823666854 -122.05357904157549) bank182098 +182099 POINT(38.720188868960776 -121.79571835424852) bank182099 +182100 POINT(37.91384983627333 -122.60369099787725) bank182100 +182101 POINT(38.279384608250766 -122.20762323389806) bank182101 +182102 POINT(37.9199647124806 -122.79393849302443) bank182102 +182103 POINT(37.66558768750174 -122.09460674650208) bank182103 +182104 POINT(38.69499806280804 -121.61769870940788) bank182104 +182105 POINT(37.316949603874974 -123.14927018787337) bank182105 +182106 POINT(38.313208258261554 -122.84275493147196) bank182106 +182107 POINT(37.49635575284044 -122.26768864438269) bank182107 +182108 POINT(36.90799022507207 -122.82231659244113) bank182108 +182109 POINT(37.667258106896846 -121.6602204612745) bank182109 +182110 POINT(38.33221845164928 -122.15522680022052) bank182110 +182111 POINT(37.76400009285757 -121.87041908367604) bank182111 +182112 POINT(36.87828726876619 -122.19280032534562) bank182112 +182113 POINT(38.50043312866511 -122.35800433401864) bank182113 +182114 POINT(38.135546690233035 -122.0257034274246) bank182114 +182115 POINT(37.218986463228035 -122.52017434571046) bank182115 +182116 POINT(38.55869294666853 -121.57759407893158) bank182116 +182117 POINT(37.43444472485431 -121.65601652207052) bank182117 +182118 POINT(38.425609125147666 -121.77522133328523) bank182118 +182119 POINT(37.66189113173911 -123.18446544071818) bank182119 +182120 POINT(37.00343542351657 -122.30688187166288) bank182120 +182121 POINT(38.39048054406043 -122.9780470966492) bank182121 +182122 POINT(38.12864218432901 -121.4882486877701) bank182122 +182123 POINT(37.202225019711896 -121.74640099647365) bank182123 +182124 POINT(36.82611297370554 -121.67694543519542) bank182124 +182125 POINT(37.63284190067054 -121.51337970668281) bank182125 +182126 POINT(37.26004013028858 -122.08433360412153) bank182126 +182127 POINT(37.18281485651608 -121.87150875603002) bank182127 +182128 POINT(36.98445613483169 -122.02514109375893) bank182128 +182129 POINT(37.930689058183404 -122.8685131362293) bank182129 +182130 POINT(37.74102305403642 -123.09913610259248) bank182130 +182131 POINT(37.81184865918194 -123.06395501678728) bank182131 +182132 POINT(38.545674387374554 -121.76892277388187) bank182132 +182133 POINT(37.470537123445915 -122.75670437691348) bank182133 +182134 POINT(38.453915567214096 -122.21717547508764) bank182134 +182135 POINT(37.510568909061895 -122.14066212169503) bank182135 +182136 POINT(37.471001136448805 -121.49266686495795) bank182136 +182137 POINT(36.92657044356133 -121.9225819101351) bank182137 +182138 POINT(37.97538318408571 -122.66651367649943) bank182138 +182139 POINT(38.35588809886156 -122.61490192320487) bank182139 +182140 POINT(37.193285085736335 -123.26771965553179) bank182140 +182141 POINT(37.87563709621427 -122.20811827644692) bank182141 +182142 POINT(36.97840627422277 -122.57704949059027) bank182142 +182143 POINT(38.34285733597897 -121.54218586181422) bank182143 +182144 POINT(37.49258300332703 -123.08062744698232) bank182144 +182145 POINT(36.872726588643715 -123.25593264986263) bank182145 +182146 POINT(36.920202875362435 -122.33125557175019) bank182146 +182147 POINT(37.38147944068763 -122.61578637381103) bank182147 +182148 POINT(38.368662545868624 -123.03349451892367) bank182148 +182149 POINT(37.63874839077446 -123.34034570942725) bank182149 +182150 POINT(38.36077168752081 -122.79609332408731) bank182150 +182151 POINT(38.179573395121125 -122.20613880116132) bank182151 +182152 POINT(37.12472864398913 -121.56037388304726) bank182152 +182153 POINT(37.70729898172278 -123.37739977419668) bank182153 +182154 POINT(37.76468202200444 -122.11253156348297) bank182154 +182155 POINT(37.214420128668074 -123.21905910564506) bank182155 +182156 POINT(37.98121065662926 -122.45572743638236) bank182156 +182157 POINT(37.00365179669181 -121.4623612761654) bank182157 +182158 POINT(38.26848306449323 -122.29021483149084) bank182158 +182159 POINT(37.37703835789006 -123.13838107671091) bank182159 +182160 POINT(38.07519603952011 -121.72330853716787) bank182160 +182161 POINT(36.79678214341227 -122.46652201463121) bank182161 +182162 POINT(37.2755720318321 -122.86518966544068) bank182162 +182163 POINT(37.940385543404496 -123.35629877383157) bank182163 +182164 POINT(38.25798643776986 -122.71273112919572) bank182164 +182165 POINT(38.19745299964116 -121.92369186333838) bank182165 +182166 POINT(37.908917028637006 -122.01262823782776) bank182166 +182167 POINT(37.07260327254232 -121.95843151867761) bank182167 +182168 POINT(38.29892894400925 -123.31019667640433) bank182168 +182169 POINT(36.88916569179382 -122.2062768654167) bank182169 +182170 POINT(37.107537940445496 -121.8290853770579) bank182170 +182171 POINT(38.06344676876818 -123.09116668498314) bank182171 +182172 POINT(38.09922597491645 -122.0384523179335) bank182172 +182173 POINT(38.1803767756143 -122.59335144254129) bank182173 +182174 POINT(38.35395103979185 -121.53100494982394) bank182174 +182175 POINT(37.671492152314315 -123.17788577370759) bank182175 +182176 POINT(37.52026246238776 -121.77190841827638) bank182176 +182177 POINT(38.176151084523354 -122.04569500311041) bank182177 +182178 POINT(37.91187138979939 -121.89215588577562) bank182178 +182179 POINT(38.23253266976851 -121.50340781358926) bank182179 +182180 POINT(37.28290418414584 -122.35842596825022) bank182180 +182181 POINT(38.45806214940198 -122.20454924276895) bank182181 +182182 POINT(37.957163268496025 -122.37391302786685) bank182182 +182183 POINT(38.08568222232907 -122.3406651745392) bank182183 +182184 POINT(37.722092658634445 -122.52703255912043) bank182184 +182185 POINT(37.7739847201666 -122.3453747127776) bank182185 +182186 POINT(37.24029008317321 -121.91947578404036) bank182186 +182187 POINT(37.23901014803581 -121.60883608720754) bank182187 +182188 POINT(36.83959759497128 -123.22037657319154) bank182188 +182189 POINT(37.99568394077128 -122.82268163709793) bank182189 +182190 POINT(38.65917811935948 -123.40154609191231) bank182190 +182191 POINT(37.32233229764375 -121.91213925668269) bank182191 +182192 POINT(36.82315425012138 -123.33229733260261) bank182192 +182193 POINT(37.98262584562253 -122.86776283154363) bank182193 +182194 POINT(37.33229525564015 -121.85426056609177) bank182194 +182195 POINT(38.23275497136871 -121.76500603155588) bank182195 +182196 POINT(37.636554287023124 -123.02667800687333) bank182196 +182197 POINT(38.03322968477178 -121.46475321853273) bank182197 +182198 POINT(38.355672835987924 -122.80617998893256) bank182198 +182199 POINT(37.43999735904197 -122.69778698655755) bank182199 +182200 POINT(37.83252435305468 -123.17167326200281) bank182200 +182201 POINT(36.85287216521914 -121.66066245444426) bank182201 +182202 POINT(37.7499106277664 -121.78169097262221) bank182202 +182203 POINT(36.873381235395414 -122.0596367698185) bank182203 +182204 POINT(38.11590462755565 -122.7333346937535) bank182204 +182205 POINT(36.97646240085678 -121.87244188959261) bank182205 +182206 POINT(37.532342123583945 -121.68269272264165) bank182206 +182207 POINT(37.36414008735509 -123.21001861223034) bank182207 +182208 POINT(37.60519979498877 -122.78702797295651) bank182208 +182209 POINT(37.848919481003364 -122.47948556552183) bank182209 +182210 POINT(37.448055058685206 -121.90255039367106) bank182210 +182211 POINT(37.137115442317324 -122.727140122225) bank182211 +182212 POINT(38.28060395132824 -121.63689163834854) bank182212 +182213 POINT(38.59241843550699 -123.13348695881969) bank182213 +182214 POINT(37.29468622804742 -122.59804036174987) bank182214 +182215 POINT(36.777070119257594 -122.49032100054421) bank182215 +182216 POINT(37.86971879848856 -122.81927939565553) bank182216 +182217 POINT(38.65202818422552 -121.46508124188415) bank182217 +182218 POINT(37.01797943076593 -123.3249806537553) bank182218 +182219 POINT(38.12144896641734 -122.57044017751069) bank182219 +182220 POINT(37.41779645634068 -123.36960469237613) bank182220 +182221 POINT(37.41693096013577 -122.0520487102886) bank182221 +182222 POINT(38.1728480423682 -122.68059900438408) bank182222 +182223 POINT(38.43763352457386 -123.16506479554435) bank182223 +182224 POINT(36.98268955535424 -122.60152347194956) bank182224 +182225 POINT(37.83770345050632 -122.94109597034318) bank182225 +182226 POINT(37.39498420903252 -122.871531725303) bank182226 +182227 POINT(37.65164207720266 -122.84595580757025) bank182227 +182228 POINT(37.77632942255386 -122.30826101815303) bank182228 +182229 POINT(38.09473978091033 -122.12983094960678) bank182229 +182230 POINT(36.917466962032165 -123.02852623430297) bank182230 +182231 POINT(37.90203529474797 -122.42308593699298) bank182231 +182232 POINT(38.30750340277945 -121.79307314185971) bank182232 +182233 POINT(36.96923091079482 -121.70650933746599) bank182233 +182234 POINT(38.563060593183124 -121.78494463781459) bank182234 +182235 POINT(37.09492367981619 -121.51506502968242) bank182235 +182236 POINT(38.228266448124764 -123.41259709305005) bank182236 +182237 POINT(38.03608292732771 -123.0018534199095) bank182237 +182238 POINT(36.92616524157764 -121.93041803753164) bank182238 +182239 POINT(37.07376915807572 -121.61647633665683) bank182239 +182240 POINT(37.21428150520634 -121.79058915187264) bank182240 +182241 POINT(37.890974544568884 -122.17765365201888) bank182241 +182242 POINT(37.01980643758032 -122.92177121738419) bank182242 +182243 POINT(38.632049101290804 -121.75847217991868) bank182243 +182244 POINT(37.69610635163063 -122.18301222969242) bank182244 +182245 POINT(38.74086647266457 -122.52496817495187) bank182245 +182246 POINT(38.750528904082444 -121.68467449785304) bank182246 +182247 POINT(38.56754368697786 -122.12209130252644) bank182247 +182248 POINT(36.90148951686827 -121.82870911152622) bank182248 +182249 POINT(38.63606305937412 -122.35549902589752) bank182249 +182250 POINT(37.580344134923024 -122.38134587626699) bank182250 +182251 POINT(38.68076657691316 -121.85521655928338) bank182251 +182252 POINT(37.87426417659552 -122.42666080387856) bank182252 +182253 POINT(38.75978804233206 -122.9377338874249) bank182253 +182254 POINT(37.257682619192224 -123.12473031051582) bank182254 +182255 POINT(36.78014012783214 -123.24736488366733) bank182255 +182256 POINT(38.70245866667197 -121.76609141799076) bank182256 +182257 POINT(38.14892743535978 -122.06748851699672) bank182257 +182258 POINT(36.86190566580491 -123.13716872007383) bank182258 +182259 POINT(37.45096111256776 -122.46190036081771) bank182259 +182260 POINT(38.13847060567337 -122.0097287917706) bank182260 +182261 POINT(37.327753283214896 -121.86046012501103) bank182261 +182262 POINT(38.36674035398814 -121.45398700538964) bank182262 +182263 POINT(36.82657734851889 -122.04793705547881) bank182263 +182264 POINT(37.37692477970276 -122.70338649948826) bank182264 +182265 POINT(38.11608976774072 -122.09440767476141) bank182265 +182266 POINT(38.67411560436269 -122.71384350176824) bank182266 +182267 POINT(38.338346487830236 -121.88990313282402) bank182267 +182268 POINT(37.17663725777747 -122.41058674775873) bank182268 +182269 POINT(38.235985745500145 -121.49355259220141) bank182269 +182270 POINT(37.01272047220007 -122.46886135660561) bank182270 +182271 POINT(37.134193786068316 -121.63694486261542) bank182271 +182272 POINT(38.5252329876957 -123.12027620937657) bank182272 +182273 POINT(37.87629459536037 -122.88697766214499) bank182273 +182274 POINT(36.992707679830005 -123.06824111161161) bank182274 +182275 POINT(37.42612230522886 -122.98653807089211) bank182275 +182276 POINT(38.61096422958958 -122.12029806940058) bank182276 +182277 POINT(37.82124798830096 -122.9616189381845) bank182277 +182278 POINT(37.51766490775067 -123.29015181164664) bank182278 +182279 POINT(36.85236889404209 -122.63464479754734) bank182279 +182280 POINT(38.53303882252856 -121.80027591736051) bank182280 +182281 POINT(37.85050660787197 -121.76718113151315) bank182281 +182282 POINT(37.81220124106336 -121.98540533889845) bank182282 +182283 POINT(38.46832554472393 -122.75736114811487) bank182283 +182284 POINT(38.0694299439442 -121.8157073086425) bank182284 +182285 POINT(38.00124419468148 -122.6265628660003) bank182285 +182286 POINT(38.70542101969998 -123.3382685743318) bank182286 +182287 POINT(38.69270966105604 -121.79308052402615) bank182287 +182288 POINT(37.567401225333064 -123.16935857050532) bank182288 +182289 POINT(37.298263430472275 -122.65865760151333) bank182289 +182290 POINT(38.71211238772032 -122.88139125592485) bank182290 +182291 POINT(38.40700722018284 -123.03081988870032) bank182291 +182292 POINT(37.468472813579325 -121.74850783405316) bank182292 +182293 POINT(38.003745285692276 -121.56883717525945) bank182293 +182294 POINT(38.65758563782856 -122.54358124824903) bank182294 +182295 POINT(37.137335048804616 -122.7388949091565) bank182295 +182296 POINT(38.34670507959718 -122.3274644119164) bank182296 +182297 POINT(36.81060092211224 -123.23511788678508) bank182297 +182298 POINT(37.54980363916141 -122.89507734922726) bank182298 +182299 POINT(37.590889890328114 -123.18633320467956) bank182299 +182300 POINT(37.00146926023315 -123.26255765706208) bank182300 +182301 POINT(37.94503611341848 -122.28629381579783) bank182301 +182302 POINT(36.92449265709012 -121.7626338195172) bank182302 +182303 POINT(38.07672517423415 -123.22676311784166) bank182303 +182304 POINT(37.13811177259976 -122.30346064190765) bank182304 +182305 POINT(38.75683022099002 -122.66211991573161) bank182305 +182306 POINT(37.224258041157334 -122.11433464600076) bank182306 +182307 POINT(38.096040198670764 -122.95255062889197) bank182307 +182308 POINT(37.12970313578645 -122.90865203796868) bank182308 +182309 POINT(38.677532481921986 -121.62473253352103) bank182309 +182310 POINT(37.05247388053495 -122.85771463558991) bank182310 +182311 POINT(37.91790045466132 -121.73736299279443) bank182311 +182312 POINT(37.051879974846315 -122.20994653148497) bank182312 +182313 POINT(37.03389552317181 -122.43452206624657) bank182313 +182314 POINT(38.708804251891685 -122.47619728859765) bank182314 +182315 POINT(38.222096169471605 -122.56400573820476) bank182315 +182316 POINT(38.119494141929856 -121.84738230039372) bank182316 +182317 POINT(37.43687558944039 -121.87731907302592) bank182317 +182318 POINT(37.24121090391081 -122.18751352325623) bank182318 +182319 POINT(37.19710167253851 -122.28181072981506) bank182319 +182320 POINT(38.42064462460814 -122.15710000493144) bank182320 +182321 POINT(37.25568552359763 -122.229479581369) bank182321 +182322 POINT(37.24040649788552 -122.97501434305046) bank182322 +182323 POINT(37.853678089205054 -123.13452471090322) bank182323 +182324 POINT(37.35906218069046 -122.65585383966148) bank182324 +182325 POINT(36.87543489705204 -121.59341254457688) bank182325 +182326 POINT(38.12244015725317 -122.08574807012688) bank182326 +182327 POINT(38.56473951836025 -122.31999167647082) bank182327 +182328 POINT(37.76966912069465 -122.13259760792488) bank182328 +182329 POINT(37.901439450870484 -123.05268723760162) bank182329 +182330 POINT(38.448497233407835 -122.23891070410328) bank182330 +182331 POINT(38.630680719024014 -121.44118543610233) bank182331 +182332 POINT(38.37015551222964 -122.54187249482426) bank182332 +182333 POINT(37.583844676117856 -122.70088311094538) bank182333 +182334 POINT(38.5297388940593 -123.0138211637524) bank182334 +182335 POINT(37.49619604124837 -122.3182677478863) bank182335 +182336 POINT(37.472650716616414 -122.79516607865563) bank182336 +182337 POINT(37.780757885227764 -122.33589865900815) bank182337 +182338 POINT(37.26143109474278 -122.93326181307681) bank182338 +182339 POINT(36.80372514929923 -122.42452930134789) bank182339 +182340 POINT(38.736087449351864 -122.28293596010724) bank182340 +182341 POINT(37.685450878095956 -122.59297499702102) bank182341 +182342 POINT(37.88878564205894 -121.84052157092555) bank182342 +182343 POINT(37.31041624742767 -122.61733000478607) bank182343 +182344 POINT(38.12672190627291 -121.7418158526571) bank182344 +182345 POINT(38.627670964396245 -121.62550486984757) bank182345 +182346 POINT(37.45413751052334 -123.07824068355562) bank182346 +182347 POINT(37.57539859995735 -122.88976105129593) bank182347 +182348 POINT(38.590105464101164 -122.8550632411678) bank182348 +182349 POINT(36.781681179809226 -121.9120863772891) bank182349 +182350 POINT(36.99009258367531 -122.65219704260946) bank182350 +182351 POINT(37.61727310471173 -121.4880628975931) bank182351 +182352 POINT(37.29745331920248 -121.61861830364539) bank182352 +182353 POINT(37.75893019160742 -122.18843483566548) bank182353 +182354 POINT(38.13593923202563 -122.35049337800618) bank182354 +182355 POINT(36.90590490164479 -122.49332320076637) bank182355 +182356 POINT(38.049373355646786 -121.50417784463797) bank182356 +182357 POINT(36.904792350022625 -123.2996496280535) bank182357 +182358 POINT(37.582615713804564 -121.4364211974653) bank182358 +182359 POINT(38.62660741968404 -121.68531707937197) bank182359 +182360 POINT(37.375056893994966 -121.68295130872785) bank182360 +182361 POINT(38.47809303152849 -122.73654427744906) bank182361 +182362 POINT(37.0097758480114 -121.59223501657233) bank182362 +182363 POINT(37.280218140588566 -123.00211966854745) bank182363 +182364 POINT(36.91438442938632 -122.0666444099605) bank182364 +182365 POINT(37.91497616271197 -121.67714720751007) bank182365 +182366 POINT(37.76731122614503 -122.5670956161334) bank182366 +182367 POINT(37.33767319950015 -121.87630189887103) bank182367 +182368 POINT(37.259473405014205 -122.73873949056484) bank182368 +182369 POINT(37.489812640903835 -121.96839537422764) bank182369 +182370 POINT(38.18360862352326 -121.7498713007286) bank182370 +182371 POINT(38.05670576590713 -122.96676447524855) bank182371 +182372 POINT(37.92675105815713 -122.69343680129526) bank182372 +182373 POINT(38.31128001255809 -123.18832870804059) bank182373 +182374 POINT(37.71441245620871 -123.34133399075773) bank182374 +182375 POINT(37.17370993365931 -122.40086965671333) bank182375 +182376 POINT(37.01660857572824 -121.73574235989824) bank182376 +182377 POINT(38.13735068270438 -123.0675874557122) bank182377 +182378 POINT(36.848794228874496 -122.25934375131015) bank182378 +182379 POINT(38.09359014173178 -123.20658263843241) bank182379 +182380 POINT(38.732224416479745 -122.91278013051048) bank182380 +182381 POINT(36.89478424585333 -123.25819608594702) bank182381 +182382 POINT(37.514165893087586 -122.63002118348554) bank182382 +182383 POINT(38.28216132097984 -122.85298164835827) bank182383 +182384 POINT(37.830374673044645 -121.78378673522779) bank182384 +182385 POINT(37.55673196102729 -122.78504281526772) bank182385 +182386 POINT(38.7114066691331 -122.59729011967164) bank182386 +182387 POINT(38.37606172751908 -122.89620459677352) bank182387 +182388 POINT(37.405455725911665 -121.74396503773119) bank182388 +182389 POINT(38.67431793801312 -122.15508527818584) bank182389 +182390 POINT(37.07471598938231 -122.24533246928549) bank182390 +182391 POINT(37.80666321037582 -122.58849135640575) bank182391 +182392 POINT(38.161909708255266 -122.15973710166087) bank182392 +182393 POINT(38.57557009767998 -121.64535093882489) bank182393 +182394 POINT(37.93940897313073 -122.55451742460524) bank182394 +182395 POINT(38.28000528935824 -122.70812892946677) bank182395 +182396 POINT(38.18250116538144 -122.89216359288048) bank182396 +182397 POINT(37.802947358779676 -122.65528013784004) bank182397 +182398 POINT(38.47497127890307 -122.99671293372427) bank182398 +182399 POINT(38.40783970849906 -122.54478103914487) bank182399 +182400 POINT(37.45730405715545 -123.31479082240303) bank182400 +182401 POINT(38.29443264882928 -122.53749416904921) bank182401 +182402 POINT(37.76973148123442 -122.01015664525086) bank182402 +182403 POINT(37.26382651186049 -121.99624123793892) bank182403 +182404 POINT(38.04179976319175 -122.45821994349009) bank182404 +182405 POINT(36.98763845614745 -121.51749964496396) bank182405 +182406 POINT(37.41602093262866 -121.83146376327649) bank182406 +182407 POINT(36.83440160263966 -122.37198814440085) bank182407 +182408 POINT(37.86189105219666 -121.52125335039193) bank182408 +182409 POINT(38.73746233896564 -123.36783166012178) bank182409 +182410 POINT(37.64485477198866 -121.4372636679468) bank182410 +182411 POINT(37.991752395279306 -122.39153790357766) bank182411 +182412 POINT(37.489317218928505 -122.93546253056707) bank182412 +182413 POINT(38.52628725250061 -122.52231487318387) bank182413 +182414 POINT(38.35525395301384 -122.14703797681584) bank182414 +182415 POINT(38.167375920113145 -121.51055403946155) bank182415 +182416 POINT(38.731851824926224 -121.64450522221368) bank182416 +182417 POINT(38.18832372935144 -122.16710132008542) bank182417 +182418 POINT(38.72954197428057 -121.99769712184481) bank182418 +182419 POINT(37.14761328407411 -122.6296659675861) bank182419 +182420 POINT(38.5565232704871 -121.80313082790823) bank182420 +182421 POINT(38.39869078504116 -122.00476520592431) bank182421 +182422 POINT(38.46671941985249 -121.90870763655983) bank182422 +182423 POINT(37.218353270612816 -123.27155138049733) bank182423 +182424 POINT(37.46164489937787 -121.9076333157153) bank182424 +182425 POINT(36.9854377166491 -122.65257996042297) bank182425 +182426 POINT(37.47670821646836 -122.82841534437863) bank182426 +182427 POINT(37.98331189837591 -122.9918618147419) bank182427 +182428 POINT(37.18675855079732 -123.4103689762785) bank182428 +182429 POINT(37.04051707950781 -122.20204647121547) bank182429 +182430 POINT(37.31799455845652 -122.95067408614719) bank182430 +182431 POINT(37.569092166615405 -122.83257170865807) bank182431 +182432 POINT(37.786200467222294 -122.3692958620431) bank182432 +182433 POINT(36.89679939392258 -122.48904223572583) bank182433 +182434 POINT(38.492457118755915 -121.55906520976197) bank182434 +182435 POINT(36.95523133843236 -121.50693946093432) bank182435 +182436 POINT(38.75168302296068 -123.17593747643352) bank182436 +182437 POINT(38.27690459704458 -121.66439691707421) bank182437 +182438 POINT(38.70001952929299 -122.90580667617878) bank182438 +182439 POINT(37.12607671874217 -122.01567284440107) bank182439 +182440 POINT(38.41548031325646 -123.1026462146142) bank182440 +182441 POINT(37.1657024145462 -122.29523003970372) bank182441 +182442 POINT(38.39384179366812 -122.1752111396049) bank182442 +182443 POINT(37.12104411106222 -123.39070959775249) bank182443 +182444 POINT(38.121640376919835 -121.68162678865018) bank182444 +182445 POINT(38.26973131578863 -123.36332380528219) bank182445 +182446 POINT(38.1070201965105 -122.52193291592715) bank182446 +182447 POINT(37.46132888029729 -122.37094822146234) bank182447 +182448 POINT(37.594397671693145 -121.51279947300787) bank182448 +182449 POINT(36.82429924327768 -123.22407374872476) bank182449 +182450 POINT(38.60004083312542 -123.33893608373668) bank182450 +182451 POINT(37.64641115902581 -122.8578436955886) bank182451 +182452 POINT(38.11934416585367 -123.21982620164509) bank182452 +182453 POINT(38.178442842608746 -122.05960006640728) bank182453 +182454 POINT(37.82218630580292 -122.53937044433434) bank182454 +182455 POINT(37.908132351931584 -123.40928655314605) bank182455 +182456 POINT(36.85229882152774 -122.71372829894783) bank182456 +182457 POINT(37.58988535054369 -121.43003203073249) bank182457 +182458 POINT(37.23194286043686 -122.83904947022202) bank182458 +182459 POINT(36.84217904039906 -123.37881895399485) bank182459 +182460 POINT(38.41781883082538 -122.12902869641127) bank182460 +182461 POINT(38.27287342492891 -122.66023719435736) bank182461 +182462 POINT(38.30661535980041 -122.12020398607518) bank182462 +182463 POINT(37.933309008783425 -122.48771605115805) bank182463 +182464 POINT(38.6657453324817 -122.74562312052785) bank182464 +182465 POINT(38.56751966686487 -121.4384914678412) bank182465 +182466 POINT(37.40820702372216 -122.67419255044106) bank182466 +182467 POINT(37.11746284250479 -121.46639422721074) bank182467 +182468 POINT(38.434720583228774 -121.78415101749833) bank182468 +182469 POINT(37.243829809133004 -122.27295996054289) bank182469 +182470 POINT(36.82297441910603 -122.59348311686075) bank182470 +182471 POINT(37.204576350926274 -122.2162056703793) bank182471 +182472 POINT(37.21722860961253 -122.8087017838719) bank182472 +182473 POINT(37.90545743122411 -122.78640482897745) bank182473 +182474 POINT(37.474312376237 -121.79597549857075) bank182474 +182475 POINT(36.95800316933276 -122.61020858486627) bank182475 +182476 POINT(36.791065029807775 -122.69269069172036) bank182476 +182477 POINT(37.57452683792044 -122.55389391670701) bank182477 +182478 POINT(37.529865207279975 -122.46685839484573) bank182478 +182479 POINT(37.03219524091121 -121.50282325321959) bank182479 +182480 POINT(38.57888368640888 -122.3923221418135) bank182480 +182481 POINT(37.415850828112724 -123.25238025256739) bank182481 +182482 POINT(37.729449044901095 -121.72811179954142) bank182482 +182483 POINT(38.302072560525346 -122.25320152585708) bank182483 +182484 POINT(37.46698700862755 -122.50609345719145) bank182484 +182485 POINT(36.99916945891084 -121.80831340481988) bank182485 +182486 POINT(38.01395201547333 -123.21787513843982) bank182486 +182487 POINT(37.640928730005534 -121.60993438118605) bank182487 +182488 POINT(38.508158277179774 -122.14815687348161) bank182488 +182489 POINT(37.948182040721214 -121.68154891629537) bank182489 +182490 POINT(38.36649843765694 -121.88218754868909) bank182490 +182491 POINT(38.626175102593685 -123.40764175593712) bank182491 +182492 POINT(37.11031002983588 -123.04500868820412) bank182492 +182493 POINT(38.165941201321715 -121.68964509948037) bank182493 +182494 POINT(38.56939924911507 -122.09981902697336) bank182494 +182495 POINT(37.66291483769866 -121.53843326781605) bank182495 +182496 POINT(37.98802656539726 -121.88249263036475) bank182496 +182497 POINT(36.97912737340455 -122.29449410270685) bank182497 +182498 POINT(37.45772635106187 -122.91514525478593) bank182498 +182499 POINT(37.39978195374817 -122.82921195692612) bank182499 +182500 POINT(38.711673388602335 -122.70486591364393) bank182500 +182501 POINT(36.950369242487355 -123.12057103267372) bank182501 +182502 POINT(37.582229640315106 -123.04767047939704) bank182502 +182503 POINT(37.72821933146057 -122.4391591160338) bank182503 +182504 POINT(37.075038941549685 -123.41179240381794) bank182504 +182505 POINT(37.21087158647136 -123.39976689388666) bank182505 +182506 POINT(38.20065023218036 -122.68435819960833) bank182506 +182507 POINT(37.00577780160552 -123.08216084526175) bank182507 +182508 POINT(37.759345046209454 -122.9327252679288) bank182508 +182509 POINT(37.833764807410176 -122.10616041351494) bank182509 +182510 POINT(38.34931496486556 -122.90189815522972) bank182510 +182511 POINT(37.13982184808204 -121.57478067889532) bank182511 +182512 POINT(38.43378349369978 -121.71538994817112) bank182512 +182513 POINT(37.70719832922477 -122.89556267513528) bank182513 +182514 POINT(37.824641018735086 -122.86242547629784) bank182514 +182515 POINT(38.535516142989934 -121.81090906186995) bank182515 +182516 POINT(38.23085419839629 -121.52852491363188) bank182516 +182517 POINT(37.23067371968525 -123.12651349395614) bank182517 +182518 POINT(37.35906432340406 -122.10595406030855) bank182518 +182519 POINT(38.013621523747794 -122.53945531464274) bank182519 +182520 POINT(36.77738610516894 -122.07826663015838) bank182520 +182521 POINT(37.09688041751432 -123.28901845062856) bank182521 +182522 POINT(37.25477671054255 -123.23878931625309) bank182522 +182523 POINT(38.52429306765732 -121.68640785448977) bank182523 +182524 POINT(38.61912699291635 -123.3123062977005) bank182524 +182525 POINT(37.97475316808158 -122.68421377966253) bank182525 +182526 POINT(38.470672016863944 -122.85206759736006) bank182526 +182527 POINT(36.930232871646844 -121.46606092339404) bank182527 +182528 POINT(37.97324001385675 -121.45398119601532) bank182528 +182529 POINT(37.769940108614165 -122.86938850062181) bank182529 +182530 POINT(38.14972975284426 -123.2080037614677) bank182530 +182531 POINT(37.31882390766254 -121.74835774195601) bank182531 +182532 POINT(37.296226671884085 -121.57089897755702) bank182532 +182533 POINT(38.652240967140564 -122.0258665972456) bank182533 +182534 POINT(38.158505826158084 -121.81234583843323) bank182534 +182535 POINT(37.09897072571375 -121.66584732570261) bank182535 +182536 POINT(38.409011176493294 -121.52807824577873) bank182536 +182537 POINT(38.061797863302154 -121.52479838448892) bank182537 +182538 POINT(36.792685075307475 -122.10734665951313) bank182538 +182539 POINT(37.81693610985434 -122.67470392749568) bank182539 +182540 POINT(37.33960650704526 -121.85623358723481) bank182540 +182541 POINT(37.095021257545966 -121.79588538788686) bank182541 +182542 POINT(37.929024251038584 -123.18958386632778) bank182542 +182543 POINT(36.84629049662693 -121.75229389916375) bank182543 +182544 POINT(36.946191382036275 -122.4982574307179) bank182544 +182545 POINT(38.546613484646016 -122.2541283740528) bank182545 +182546 POINT(37.94298814769623 -123.16579848884359) bank182546 +182547 POINT(38.47018339238129 -121.44840465366855) bank182547 +182548 POINT(38.2048869584159 -121.47425128714025) bank182548 +182549 POINT(36.866200007555726 -122.366922192863) bank182549 +182550 POINT(37.685518630844506 -122.43767084390119) bank182550 +182551 POINT(37.23924129736227 -122.13334690658944) bank182551 +182552 POINT(36.94769066963966 -123.18056359984898) bank182552 +182553 POINT(38.675434173113445 -121.9051158455633) bank182553 +182554 POINT(37.026240201313016 -122.73799233699076) bank182554 +182555 POINT(38.3225862499705 -122.35665173463987) bank182555 +182556 POINT(37.72022332850605 -122.67377718299633) bank182556 +182557 POINT(37.664207669292246 -121.62798204582364) bank182557 +182558 POINT(37.11488015857575 -122.60369421437014) bank182558 +182559 POINT(38.6284360110879 -123.2909818106798) bank182559 +182560 POINT(37.85492301565274 -122.77560625697221) bank182560 +182561 POINT(37.611309243887334 -122.50715500791904) bank182561 +182562 POINT(38.55634869701623 -122.34062488895182) bank182562 +182563 POINT(37.886472998785955 -122.31593776650168) bank182563 +182564 POINT(38.120183564514285 -122.47412276515794) bank182564 +182565 POINT(38.68000227952607 -122.96413996016364) bank182565 +182566 POINT(36.89179887430657 -122.62018831727616) bank182566 +182567 POINT(36.80078863762407 -122.2849958743437) bank182567 +182568 POINT(37.12521590687764 -121.64919660455861) bank182568 +182569 POINT(36.89894490308125 -122.69650833183859) bank182569 +182570 POINT(38.31655022142035 -121.56207639278894) bank182570 +182571 POINT(37.46050609521997 -121.99157392676176) bank182571 +182572 POINT(37.708729608057745 -122.8946126646028) bank182572 +182573 POINT(36.78535991895657 -121.94901745118285) bank182573 +182574 POINT(38.52569523923345 -122.66042910869888) bank182574 +182575 POINT(37.70837703038396 -122.66742151424125) bank182575 +182576 POINT(37.30755846585656 -122.98223563881335) bank182576 +182577 POINT(37.2836880767481 -123.29165138459072) bank182577 +182578 POINT(37.04634387061769 -121.84689581476289) bank182578 +182579 POINT(36.898311136329156 -121.86345098704135) bank182579 +182580 POINT(37.50862202395615 -122.65771289207919) bank182580 +182581 POINT(37.27940569712181 -122.44045485602459) bank182581 +182582 POINT(38.56979143436449 -121.80560743837691) bank182582 +182583 POINT(37.60959784864269 -123.13735965382082) bank182583 +182584 POINT(37.01155175361922 -122.14409224665692) bank182584 +182585 POINT(37.52578243845456 -122.45869345868446) bank182585 +182586 POINT(37.99063698720454 -121.75829646581698) bank182586 +182587 POINT(37.24798793433498 -122.68311421114915) bank182587 +182588 POINT(37.13515358111654 -121.63583416436374) bank182588 +182589 POINT(37.05732267490491 -123.142495210882) bank182589 +182590 POINT(37.9289370738794 -121.82551267611986) bank182590 +182591 POINT(37.05722950780212 -123.39895733183462) bank182591 +182592 POINT(38.06966528289486 -122.60310464964846) bank182592 +182593 POINT(37.67911934855139 -122.17761111343994) bank182593 +182594 POINT(37.62933867897636 -122.01675043799048) bank182594 +182595 POINT(38.66771365103145 -122.57573756820177) bank182595 +182596 POINT(37.20009360029041 -122.10177072034242) bank182596 +182597 POINT(38.7672650195966 -122.68467138588282) bank182597 +182598 POINT(37.23450336476012 -123.17780339425248) bank182598 +182599 POINT(38.5921849621609 -122.564600522166) bank182599 +182600 POINT(38.6784628765197 -121.78925582684302) bank182600 +182601 POINT(37.193171840666146 -122.61400407725279) bank182601 +182602 POINT(38.50487345125462 -122.98246391676409) bank182602 +182603 POINT(37.65235482766205 -122.05030313978762) bank182603 +182604 POINT(37.03274147861703 -121.98670743453454) bank182604 +182605 POINT(37.1622937086276 -121.62714404055042) bank182605 +182606 POINT(38.76985785636878 -122.29623345182105) bank182606 +182607 POINT(38.50866125426469 -122.09215196942247) bank182607 +182608 POINT(37.956422389492666 -121.59359455018046) bank182608 +182609 POINT(37.85381020921955 -122.66221756775295) bank182609 +182610 POINT(37.13853308798702 -121.43058573946871) bank182610 +182611 POINT(37.47853028203354 -123.31148212339268) bank182611 +182612 POINT(37.27981328428371 -122.5633915946182) bank182612 +182613 POINT(37.08697704750248 -123.08002427975735) bank182613 +182614 POINT(37.58820507281791 -122.98001500295247) bank182614 +182615 POINT(37.53801033813146 -122.45471702251058) bank182615 +182616 POINT(36.9156648046035 -123.10444338117209) bank182616 +182617 POINT(38.116706978378495 -121.51345711808605) bank182617 +182618 POINT(38.132837882732986 -122.08597693312936) bank182618 +182619 POINT(37.70812482237132 -123.41540652482232) bank182619 +182620 POINT(37.79960364016153 -123.37226877627319) bank182620 +182621 POINT(36.82182909276536 -122.43153219518338) bank182621 +182622 POINT(38.69165401082974 -122.62716539268702) bank182622 +182623 POINT(38.6765595193721 -122.19394651636313) bank182623 +182624 POINT(36.98503711885125 -122.54704034002825) bank182624 +182625 POINT(37.488187889337034 -123.10040140590917) bank182625 +182626 POINT(37.48415351279073 -122.95861112031648) bank182626 +182627 POINT(37.33525601501246 -122.16950514120906) bank182627 +182628 POINT(37.967783108399125 -121.96404980349979) bank182628 +182629 POINT(38.682111162676314 -123.17325748623561) bank182629 +182630 POINT(38.08374231116572 -121.41990114906652) bank182630 +182631 POINT(37.2910205848476 -121.9571731756482) bank182631 +182632 POINT(38.16350022631964 -121.70465700619539) bank182632 +182633 POINT(37.0730884395125 -122.97340704009159) bank182633 +182634 POINT(38.33077749674251 -122.7687350297676) bank182634 +182635 POINT(38.0865315450454 -123.2663502561157) bank182635 +182636 POINT(37.299725943547124 -122.08137697374589) bank182636 +182637 POINT(38.66822589992416 -123.12027122735782) bank182637 +182638 POINT(38.351423215263395 -123.35469759139268) bank182638 +182639 POINT(37.94613339220051 -121.95462866219337) bank182639 +182640 POINT(37.987897849543 -122.76863140618129) bank182640 +182641 POINT(37.86252359643161 -121.44667617249122) bank182641 +182642 POINT(37.55157693330435 -122.06171704964402) bank182642 +182643 POINT(37.58694427521892 -121.71075240926051) bank182643 +182644 POINT(37.04377446476128 -122.2776327853815) bank182644 +182645 POINT(37.757934129347476 -122.12710237735615) bank182645 +182646 POINT(37.95709441242302 -123.411303498601) bank182646 +182647 POINT(37.80191381221628 -122.040919350661) bank182647 +182648 POINT(37.92770465756773 -122.57220768258361) bank182648 +182649 POINT(37.96285649631758 -123.3125773969048) bank182649 +182650 POINT(36.99034228313035 -121.72876029991045) bank182650 +182651 POINT(37.411606333602464 -121.61182409380018) bank182651 +182652 POINT(37.09161974573637 -121.5262538576215) bank182652 +182653 POINT(38.37369603266953 -122.82817123182829) bank182653 +182654 POINT(37.78080192836776 -123.01292673559637) bank182654 +182655 POINT(36.94907681535954 -123.37036430321186) bank182655 +182656 POINT(38.56966887593229 -121.7252515429603) bank182656 +182657 POINT(37.815495121836875 -121.74033755254612) bank182657 +182658 POINT(37.81761055393127 -122.50251215934423) bank182658 +182659 POINT(37.090844923498494 -122.42484060585844) bank182659 +182660 POINT(37.35208806023181 -122.57892585481625) bank182660 +182661 POINT(37.13958021563424 -121.82390118294114) bank182661 +182662 POINT(36.843270876733854 -122.2562079973851) bank182662 +182663 POINT(38.00133641996862 -122.88996656994571) bank182663 +182664 POINT(38.19290531321674 -123.3988630223008) bank182664 +182665 POINT(38.48909249516867 -122.13515711212114) bank182665 +182666 POINT(37.57125843520902 -123.27252190760353) bank182666 +182667 POINT(38.149584539507366 -122.15799413654045) bank182667 +182668 POINT(38.16661765306671 -121.84767346623674) bank182668 +182669 POINT(37.493150953928776 -122.32956766666688) bank182669 +182670 POINT(36.918149863640096 -121.84423375516903) bank182670 +182671 POINT(37.49061651969619 -122.75163625851665) bank182671 +182672 POINT(37.43858836267297 -122.16748621412746) bank182672 +182673 POINT(38.428145666292274 -122.69105096795673) bank182673 +182674 POINT(37.929144365842035 -122.99562753955439) bank182674 +182675 POINT(37.107288720614925 -123.3385730375052) bank182675 +182676 POINT(36.9015093338338 -122.84793813045798) bank182676 +182677 POINT(38.02689924157182 -122.10901138015461) bank182677 +182678 POINT(36.785498613388505 -122.99201711844026) bank182678 +182679 POINT(38.01260501764703 -122.34213519653129) bank182679 +182680 POINT(37.07381718781821 -121.82854966937707) bank182680 +182681 POINT(37.8693693003482 -122.32824776556127) bank182681 +182682 POINT(37.20748432661823 -123.32784756464598) bank182682 +182683 POINT(38.72607081105857 -122.63244999673458) bank182683 +182684 POINT(37.785996125269136 -122.13579125822008) bank182684 +182685 POINT(37.83363045752133 -121.52130015513815) bank182685 +182686 POINT(38.11247851719204 -123.12312403353324) bank182686 +182687 POINT(38.042772510601985 -122.16854363804443) bank182687 +182688 POINT(37.26288830624212 -121.78056031853033) bank182688 +182689 POINT(38.2944056544291 -122.55227554666907) bank182689 +182690 POINT(38.42104992643772 -122.5877272750039) bank182690 +182691 POINT(37.71655573422552 -122.1703768710286) bank182691 +182692 POINT(38.332232564377364 -122.86970471008675) bank182692 +182693 POINT(38.47216035468564 -122.37769974346836) bank182693 +182694 POINT(38.7471367249545 -121.47275616789068) bank182694 +182695 POINT(38.7557802208206 -121.66577711803498) bank182695 +182696 POINT(38.428362960661424 -122.94730957490331) bank182696 +182697 POINT(37.26849933573189 -121.80033738962543) bank182697 +182698 POINT(38.67664043037779 -122.60369458732104) bank182698 +182699 POINT(37.1777074955851 -122.33785511625179) bank182699 +182700 POINT(38.41602633130773 -123.37609754040767) bank182700 +182701 POINT(37.77146918434886 -122.8814729944806) bank182701 +182702 POINT(38.55855694593175 -122.59535409770724) bank182702 +182703 POINT(38.66881996075585 -121.60506560342321) bank182703 +182704 POINT(38.629110300364985 -122.06153839153255) bank182704 +182705 POINT(36.9373018648161 -123.07744821813206) bank182705 +182706 POINT(36.89730705162842 -122.77600627986453) bank182706 +182707 POINT(38.398704668773256 -122.50288941149066) bank182707 +182708 POINT(37.57608231045254 -122.59523464306469) bank182708 +182709 POINT(37.752549320362974 -121.43344145284311) bank182709 +182710 POINT(36.93128908422209 -121.63742144446341) bank182710 +182711 POINT(37.60489128252725 -121.89187612351276) bank182711 +182712 POINT(38.06947492320367 -123.40348756388285) bank182712 +182713 POINT(37.849850712765246 -122.16455467282464) bank182713 +182714 POINT(38.26425063467322 -122.89124855625374) bank182714 +182715 POINT(38.51102481833559 -122.39856413154297) bank182715 +182716 POINT(37.57543393436425 -123.17418689487448) bank182716 +182717 POINT(37.21441120667241 -123.37599692470712) bank182717 +182718 POINT(38.2020131387776 -122.4929508704317) bank182718 +182719 POINT(37.2896761835035 -121.96669473208628) bank182719 +182720 POINT(37.05713636357007 -121.56404683348633) bank182720 +182721 POINT(37.76154145192494 -122.99359631032195) bank182721 +182722 POINT(38.450376368873954 -122.3091463500742) bank182722 +182723 POINT(38.657853589721256 -122.47275996525687) bank182723 +182724 POINT(38.588745362734706 -122.02109993989714) bank182724 +182725 POINT(37.748873178692364 -121.50008713228662) bank182725 +182726 POINT(37.746816950868904 -122.44316325324743) bank182726 +182727 POINT(37.1229066677799 -122.24377078644716) bank182727 +182728 POINT(38.19554998337289 -122.1499388734244) bank182728 +182729 POINT(37.42109319985565 -122.15132272799822) bank182729 +182730 POINT(38.41021612844982 -122.52649714392669) bank182730 +182731 POINT(36.926527420200955 -121.82743816906813) bank182731 +182732 POINT(37.56581734148248 -122.4464020077776) bank182732 +182733 POINT(38.55381064488063 -122.59622929838615) bank182733 +182734 POINT(37.33297126144588 -121.73806066862119) bank182734 +182735 POINT(38.011136475458855 -122.89520361391352) bank182735 +182736 POINT(38.181629527882876 -121.74616696964836) bank182736 +182737 POINT(37.28430914357623 -122.69351902613232) bank182737 +182738 POINT(38.44824354846058 -122.48145552414714) bank182738 +182739 POINT(38.53647235155919 -123.14681096772716) bank182739 +182740 POINT(37.124381103125685 -122.68703763106576) bank182740 +182741 POINT(37.45076903019316 -122.15036760093457) bank182741 +182742 POINT(38.26297961465449 -122.8861945880272) bank182742 +182743 POINT(37.74966288843691 -122.19190173937804) bank182743 +182744 POINT(38.086232059813206 -122.50537109141) bank182744 +182745 POINT(38.53798654275045 -122.57725465484772) bank182745 +182746 POINT(36.937750290687156 -122.7955642646212) bank182746 +182747 POINT(38.29012159361547 -121.50752954782389) bank182747 +182748 POINT(37.23648145155838 -122.56299068448304) bank182748 +182749 POINT(38.246432445483656 -121.46508346957803) bank182749 +182750 POINT(37.46635011007736 -123.29315212542453) bank182750 +182751 POINT(37.77298697220362 -121.89429280297814) bank182751 +182752 POINT(37.07498635983498 -121.68184884111353) bank182752 +182753 POINT(37.05540836412371 -123.07334115035297) bank182753 +182754 POINT(37.22300769789071 -121.98788820700106) bank182754 +182755 POINT(38.460267991347244 -121.99754066672408) bank182755 +182756 POINT(37.15939071574735 -121.57207893766376) bank182756 +182757 POINT(37.3268460449491 -122.42889751315907) bank182757 +182758 POINT(37.7271107801042 -121.5658890648429) bank182758 +182759 POINT(37.08160482409809 -123.21966786643112) bank182759 +182760 POINT(37.876935429089194 -122.51455478504286) bank182760 +182761 POINT(37.0045281166479 -121.99204493103436) bank182761 +182762 POINT(37.02923293989587 -122.30228800071522) bank182762 +182763 POINT(38.51733633937721 -122.21882812414479) bank182763 +182764 POINT(37.52958183834094 -121.73272448999663) bank182764 +182765 POINT(37.4528609031898 -123.01056863174395) bank182765 +182766 POINT(38.07303354087665 -122.42109334953808) bank182766 +182767 POINT(37.774212179750705 -122.70293529994346) bank182767 +182768 POINT(37.358574873562496 -121.7859759849892) bank182768 +182769 POINT(38.76720469856472 -123.21846306343815) bank182769 +182770 POINT(37.39104535736 -121.9998649171861) bank182770 +182771 POINT(37.40228216566157 -122.50156365623975) bank182771 +182772 POINT(37.92053219526799 -123.17496022301087) bank182772 +182773 POINT(38.629964622546325 -123.29520659131781) bank182773 +182774 POINT(37.949316469501404 -122.5249862919253) bank182774 +182775 POINT(37.68784857622842 -121.94017963828284) bank182775 +182776 POINT(38.27981952525157 -122.8467135553027) bank182776 +182777 POINT(38.340590985551046 -121.69499096769687) bank182777 +182778 POINT(38.068097275365936 -121.52349573235351) bank182778 +182779 POINT(38.076339805521854 -123.29779143828821) bank182779 +182780 POINT(36.948729066976576 -122.88655060982418) bank182780 +182781 POINT(37.6072714261728 -122.41927730465864) bank182781 +182782 POINT(37.750098491881246 -123.15651274281745) bank182782 +182783 POINT(38.46708231257756 -122.70676798358255) bank182783 +182784 POINT(37.48743117050597 -123.40054605432027) bank182784 +182785 POINT(38.606361210627945 -123.27225953603882) bank182785 +182786 POINT(37.40365705558187 -123.37288970604332) bank182786 +182787 POINT(38.22253543115368 -122.27104953960854) bank182787 +182788 POINT(37.30295457355996 -121.91836962609334) bank182788 +182789 POINT(38.66036188951723 -121.97554804251598) bank182789 +182790 POINT(37.52258207974113 -121.44074433645265) bank182790 +182791 POINT(38.608510642517544 -122.27959610013708) bank182791 +182792 POINT(37.59160933842068 -121.86392987808884) bank182792 +182793 POINT(37.15853223597186 -123.18951738926245) bank182793 +182794 POINT(38.149266631325084 -122.1192121134615) bank182794 +182795 POINT(38.3154821055803 -122.62371109171262) bank182795 +182796 POINT(37.92805444689212 -122.88030841170495) bank182796 +182797 POINT(37.39909331615104 -122.28811753149215) bank182797 +182798 POINT(37.98286109732793 -121.68956482492605) bank182798 +182799 POINT(38.676344479030554 -122.59166645755882) bank182799 +182800 POINT(38.37166826183584 -123.0051359937355) bank182800 +182801 POINT(37.92969217571663 -122.9036476911683) bank182801 +182802 POINT(38.484796624295214 -122.27543939870884) bank182802 +182803 POINT(37.41100037134351 -123.01660373765995) bank182803 +182804 POINT(37.051766803872766 -122.94775487263361) bank182804 +182805 POINT(37.47203556825708 -122.50393788184076) bank182805 +182806 POINT(37.84968546520427 -122.25143887469885) bank182806 +182807 POINT(38.25497474830264 -122.38009439108052) bank182807 +182808 POINT(37.73687015673782 -122.44768074760523) bank182808 +182809 POINT(37.969490638424354 -122.03438049281749) bank182809 +182810 POINT(38.1315767968827 -122.31389441326476) bank182810 +182811 POINT(37.487762899355154 -122.69940554733464) bank182811 +182812 POINT(37.08862680881324 -122.64866019198365) bank182812 +182813 POINT(36.80570582599722 -123.28948994356611) bank182813 +182814 POINT(37.240166457709925 -122.98928498404585) bank182814 +182815 POINT(37.0729767278449 -122.03892417359059) bank182815 +182816 POINT(37.33376256356379 -121.90280429644582) bank182816 +182817 POINT(36.94398132401339 -122.456240384546) bank182817 +182818 POINT(37.84076671885185 -121.9422516169735) bank182818 +182819 POINT(38.596747047061534 -122.54214599086963) bank182819 +182820 POINT(38.39954519585954 -122.50532916882226) bank182820 +182821 POINT(38.75106638283067 -123.21758171710566) bank182821 +182822 POINT(37.082835633612966 -122.81592344318621) bank182822 +182823 POINT(36.87584454736395 -122.57773019337107) bank182823 +182824 POINT(38.24667184861159 -121.66104957041027) bank182824 +182825 POINT(37.551323046537526 -122.6561730114527) bank182825 +182826 POINT(36.81599550641037 -121.6875421330167) bank182826 +182827 POINT(38.72625625436377 -123.2450980993767) bank182827 +182828 POINT(38.63903641089014 -123.02203480490046) bank182828 +182829 POINT(38.387217376555405 -121.7431036676597) bank182829 +182830 POINT(38.426701471370755 -122.88833555211734) bank182830 +182831 POINT(38.47237301177765 -121.8053420686524) bank182831 +182832 POINT(38.266946355143354 -123.12760274846966) bank182832 +182833 POINT(37.10003455961453 -121.67134338042855) bank182833 +182834 POINT(38.18065925287597 -122.3739611928269) bank182834 +182835 POINT(37.83378172288334 -123.10955262887407) bank182835 +182836 POINT(38.48742549954327 -122.48950439774333) bank182836 +182837 POINT(37.414279657512104 -122.02020536397231) bank182837 +182838 POINT(37.713446429334695 -122.30586191328615) bank182838 +182839 POINT(37.27330336290616 -122.43187838625204) bank182839 +182840 POINT(38.16342230551005 -123.08173925774607) bank182840 +182841 POINT(38.15140946693428 -121.4537600590072) bank182841 +182842 POINT(38.457180831942516 -123.41400061168831) bank182842 +182843 POINT(38.35324289872663 -121.92030850962817) bank182843 +182844 POINT(36.991066749678666 -121.44165270945898) bank182844 +182845 POINT(37.36458568352357 -123.24063694851195) bank182845 +182846 POINT(37.41619999922416 -123.1961187836371) bank182846 +182847 POINT(36.97557469609637 -121.88869365129587) bank182847 +182848 POINT(37.25790096427496 -121.49300391207915) bank182848 +182849 POINT(37.99539355212234 -123.15470648275587) bank182849 +182850 POINT(36.87985246303638 -122.61669312318247) bank182850 +182851 POINT(37.369069168213564 -123.28090694443117) bank182851 +182852 POINT(38.30125297368796 -122.11442708223088) bank182852 +182853 POINT(37.420495778189476 -121.8055551934209) bank182853 +182854 POINT(38.255026048365806 -122.47743226949322) bank182854 +182855 POINT(37.713940870705684 -122.89854482957023) bank182855 +182856 POINT(36.96359486499776 -121.93623770912119) bank182856 +182857 POINT(37.36826810315978 -122.05312585293976) bank182857 +182858 POINT(37.616528193529284 -121.83546569370898) bank182858 +182859 POINT(38.37718578248424 -122.74811429488543) bank182859 +182860 POINT(38.240911458180605 -122.23194864735585) bank182860 +182861 POINT(38.700666575461675 -121.55476414108452) bank182861 +182862 POINT(38.3008988479465 -122.60459813192992) bank182862 +182863 POINT(38.056434878115674 -123.0269916884874) bank182863 +182864 POINT(38.158067114145446 -122.59818297795292) bank182864 +182865 POINT(37.9593709122569 -123.38755889518082) bank182865 +182866 POINT(37.66646010431901 -121.991224739898) bank182866 +182867 POINT(37.00830356343157 -121.5056127342281) bank182867 +182868 POINT(38.309703327800726 -121.83872381757831) bank182868 +182869 POINT(37.82201831564084 -121.7243597025014) bank182869 +182870 POINT(37.320340277003886 -122.45351625060557) bank182870 +182871 POINT(38.37777946657236 -121.56293323321786) bank182871 +182872 POINT(36.99941289420672 -121.91586348820067) bank182872 +182873 POINT(36.91329073419537 -122.14966004538248) bank182873 +182874 POINT(37.500742650711786 -123.2625575043228) bank182874 +182875 POINT(37.32820525888678 -121.4407835173254) bank182875 +182876 POINT(38.31263329721153 -123.26015156551642) bank182876 +182877 POINT(37.00033701766966 -123.38937000705018) bank182877 +182878 POINT(37.596276843157845 -121.84247220162628) bank182878 +182879 POINT(38.103319759831464 -121.75829796545472) bank182879 +182880 POINT(37.47007591925363 -121.52682415194398) bank182880 +182881 POINT(37.94311340574977 -122.41614742723404) bank182881 +182882 POINT(38.55933094960563 -122.01315770185015) bank182882 +182883 POINT(38.48019882368523 -122.15229379841955) bank182883 +182884 POINT(38.32845288812053 -122.29762582027328) bank182884 +182885 POINT(38.059347824646906 -122.13646934557477) bank182885 +182886 POINT(37.97201613833818 -122.79782092238695) bank182886 +182887 POINT(38.46881161234472 -122.11402505525108) bank182887 +182888 POINT(38.04168039831435 -122.53489263534443) bank182888 +182889 POINT(38.503958164466 -122.61002993092517) bank182889 +182890 POINT(38.59373404187537 -122.47864801514051) bank182890 +182891 POINT(37.107725383793806 -121.53932567538506) bank182891 +182892 POINT(37.07268284477537 -122.45026675227055) bank182892 +182893 POINT(37.54142761132421 -122.62887011717542) bank182893 +182894 POINT(37.587043476374454 -122.84230531360882) bank182894 +182895 POINT(37.93092825971662 -122.8124297476023) bank182895 +182896 POINT(37.13767128577203 -122.01558144063783) bank182896 +182897 POINT(38.5629138368505 -122.7394413626727) bank182897 +182898 POINT(37.10563019670751 -122.33680895590435) bank182898 +182899 POINT(37.05281712560445 -122.76290798076353) bank182899 +182900 POINT(38.29250186861411 -122.25328350193908) bank182900 +182901 POINT(38.136930271116555 -123.33792557374929) bank182901 +182902 POINT(36.989351616563724 -122.80417330974115) bank182902 +182903 POINT(38.362441285778885 -122.42680234710241) bank182903 +182904 POINT(37.143793564550265 -122.01053917571063) bank182904 +182905 POINT(37.986673012235705 -122.6155618315502) bank182905 +182906 POINT(37.324210236437025 -123.133794357434) bank182906 +182907 POINT(38.28857794849931 -121.47445998683304) bank182907 +182908 POINT(38.320699589109736 -121.48126165252933) bank182908 +182909 POINT(38.48419489004557 -122.83273975779699) bank182909 +182910 POINT(37.57308212396952 -121.43726546445187) bank182910 +182911 POINT(38.392386162360225 -121.69246984795824) bank182911 +182912 POINT(37.168148752450435 -122.47365210456924) bank182912 +182913 POINT(37.49886084788424 -122.35701998129905) bank182913 +182914 POINT(37.91142176073344 -122.4875517612382) bank182914 +182915 POINT(37.93265010823182 -122.5628819837243) bank182915 +182916 POINT(37.35425962075063 -122.24070958280527) bank182916 +182917 POINT(38.63649074606029 -122.04852738199335) bank182917 +182918 POINT(38.13089090795882 -123.30851844436465) bank182918 +182919 POINT(37.9622062726175 -123.31831378522661) bank182919 +182920 POINT(38.41672168892807 -122.0247956350737) bank182920 +182921 POINT(37.13069071920585 -122.61553950110607) bank182921 +182922 POINT(37.07377411821684 -122.97306267989607) bank182922 +182923 POINT(38.61444909906795 -122.37785070189732) bank182923 +182924 POINT(37.34585190287542 -121.5823210230188) bank182924 +182925 POINT(37.04192842116676 -122.13006135042524) bank182925 +182926 POINT(37.38053530213533 -122.28543622313319) bank182926 +182927 POINT(38.458316788970116 -121.78966781880587) bank182927 +182928 POINT(37.639372658350815 -121.53495286442457) bank182928 +182929 POINT(36.92833957298043 -121.66062608031065) bank182929 +182930 POINT(37.18805832627595 -122.1345212395734) bank182930 +182931 POINT(37.21651620733672 -121.80219411418278) bank182931 +182932 POINT(38.10887337920145 -121.62053573922339) bank182932 +182933 POINT(37.51817941455359 -122.57324035816333) bank182933 +182934 POINT(37.083394231748635 -122.83125515439504) bank182934 +182935 POINT(38.7060334199722 -122.1388229253672) bank182935 +182936 POINT(38.15849282687695 -122.8796049965225) bank182936 +182937 POINT(37.15555925772971 -122.98398805812816) bank182937 +182938 POINT(38.59212093060252 -122.63378615242762) bank182938 +182939 POINT(38.414371114949915 -121.86231835735062) bank182939 +182940 POINT(38.2189577294346 -121.63587544708061) bank182940 +182941 POINT(37.24710441221174 -121.4288499076764) bank182941 +182942 POINT(36.99751044309578 -123.38448654109465) bank182942 +182943 POINT(38.70207669116339 -122.53658913778432) bank182943 +182944 POINT(36.90058107197653 -123.03145440475365) bank182944 +182945 POINT(38.32342213334143 -122.89555583182273) bank182945 +182946 POINT(38.39034274956274 -121.89718738573892) bank182946 +182947 POINT(37.27418053265329 -122.93043100852134) bank182947 +182948 POINT(37.08289463332651 -123.18094497523792) bank182948 +182949 POINT(37.85514816584792 -122.59059986533538) bank182949 +182950 POINT(38.59980976100643 -122.08219033780698) bank182950 +182951 POINT(38.468823197321534 -121.46029044573766) bank182951 +182952 POINT(37.71022034216602 -123.0618027829342) bank182952 +182953 POINT(38.199910396875694 -121.59620821441187) bank182953 +182954 POINT(36.8031318904639 -123.18350331046096) bank182954 +182955 POINT(37.4441762550053 -121.92452722105314) bank182955 +182956 POINT(37.93990110299584 -122.6889667206465) bank182956 +182957 POINT(38.68941906524121 -123.3292414745615) bank182957 +182958 POINT(37.55385095604708 -122.8178665750308) bank182958 +182959 POINT(36.95420422541909 -123.05379001734464) bank182959 +182960 POINT(37.454161647252654 -121.95943302940178) bank182960 +182961 POINT(37.09104727392164 -123.36160396727344) bank182961 +182962 POINT(38.56648191675045 -122.0333340355958) bank182962 +182963 POINT(38.254611093907684 -121.82591818558485) bank182963 +182964 POINT(38.23759420679306 -122.59017770335372) bank182964 +182965 POINT(38.569649216265404 -121.59698093670933) bank182965 +182966 POINT(37.91738962728702 -121.56818524073928) bank182966 +182967 POINT(37.73189149752087 -121.88058934522208) bank182967 +182968 POINT(36.91472451004904 -122.97251889734575) bank182968 +182969 POINT(38.68320877461413 -123.24480633099603) bank182969 +182970 POINT(37.97511797462109 -121.42479546915993) bank182970 +182971 POINT(38.298388656698776 -122.40762639871785) bank182971 +182972 POINT(38.03654651135612 -122.55569382599855) bank182972 +182973 POINT(37.36765529554993 -122.67387291715308) bank182973 +182974 POINT(37.83543816207215 -122.0035398369548) bank182974 +182975 POINT(38.6189003687772 -121.47051872833624) bank182975 +182976 POINT(37.41607650756647 -122.79548971534962) bank182976 +182977 POINT(37.578639386051314 -122.2429291107804) bank182977 +182978 POINT(38.28319050049885 -122.9915216426264) bank182978 +182979 POINT(38.086760236860805 -122.46180056258142) bank182979 +182980 POINT(37.753365362794746 -122.06772325190562) bank182980 +182981 POINT(38.024131320667784 -122.80872641041722) bank182981 +182982 POINT(37.63585139070345 -123.030258756169) bank182982 +182983 POINT(37.37422524638662 -123.31284654867142) bank182983 +182984 POINT(38.54155826510715 -121.53972117743353) bank182984 +182985 POINT(36.93755616000019 -123.1566791271219) bank182985 +182986 POINT(36.96790961893075 -121.65670278666502) bank182986 +182987 POINT(38.61343885837118 -123.28706051301803) bank182987 +182988 POINT(37.394141721691675 -122.42918654958422) bank182988 +182989 POINT(37.36547996484198 -123.10281680560239) bank182989 +182990 POINT(37.2536218825225 -121.55843905119737) bank182990 +182991 POINT(38.428731595955945 -122.03241552623395) bank182991 +182992 POINT(36.93911749080577 -123.09176499314329) bank182992 +182993 POINT(36.81883668175974 -122.77401428985823) bank182993 +182994 POINT(38.64030462454758 -122.68660071022875) bank182994 +182995 POINT(37.53054068614774 -122.60152225075421) bank182995 +182996 POINT(36.85742144401921 -122.37490635456956) bank182996 +182997 POINT(38.737675803031586 -123.03321201232117) bank182997 +182998 POINT(37.083384886349215 -121.49300993068246) bank182998 +182999 POINT(37.377033705170064 -122.06845298579242) bank182999 +183000 POINT(38.5618808029463 -122.91632661435185) bank183000 +183001 POINT(37.48175911238597 -122.24026814634159) bank183001 +183002 POINT(36.94708880149808 -122.70812061282518) bank183002 +183003 POINT(37.743988086880535 -121.67406448811974) bank183003 +183004 POINT(37.98015872361273 -121.77778007131994) bank183004 +183005 POINT(37.56510371731805 -123.28460766968125) bank183005 +183006 POINT(38.34418360134331 -121.9333548569636) bank183006 +183007 POINT(37.93723113188921 -123.01432070419573) bank183007 +183008 POINT(38.301786246863394 -123.08770982021277) bank183008 +183009 POINT(38.5496277285345 -121.6544409004348) bank183009 +183010 POINT(37.38093053930969 -122.2957159458473) bank183010 +183011 POINT(37.41569815844771 -121.88796965473857) bank183011 +183012 POINT(38.38479090569688 -123.12435791463659) bank183012 +183013 POINT(38.70189176221127 -122.66486516473921) bank183013 +183014 POINT(37.47842086255135 -123.39992678230321) bank183014 +183015 POINT(37.92850806884388 -121.70799861376665) bank183015 +183016 POINT(37.02211329496137 -123.28257996844165) bank183016 +183017 POINT(37.607770028874576 -123.35521528453582) bank183017 +183018 POINT(37.14525414438557 -121.59631577214465) bank183018 +183019 POINT(38.015501835301265 -121.82131860055344) bank183019 +183020 POINT(37.93868338564696 -121.5717269161029) bank183020 +183021 POINT(37.64816203973744 -121.73886061593002) bank183021 +183022 POINT(37.7353016439899 -121.77352675230001) bank183022 +183023 POINT(37.24630601556114 -121.6278705765816) bank183023 +183024 POINT(37.2788361596858 -122.08914954041349) bank183024 +183025 POINT(37.89749615729019 -122.11045243461913) bank183025 +183026 POINT(38.75752512626823 -122.90020558134252) bank183026 +183027 POINT(36.95331445132457 -122.95378231843674) bank183027 +183028 POINT(38.70856850774715 -123.06689189507631) bank183028 +183029 POINT(38.2942414965721 -122.57078135337048) bank183029 +183030 POINT(37.791852426355675 -122.87689835936942) bank183030 +183031 POINT(37.35996763972905 -123.03002740435541) bank183031 +183032 POINT(38.14057562829726 -122.71941060003432) bank183032 +183033 POINT(38.505609608436224 -122.832919802688) bank183033 +183034 POINT(36.95083919598302 -121.72451912196122) bank183034 +183035 POINT(38.42008648147483 -121.87679925516592) bank183035 +183036 POINT(37.17633900201557 -122.63741990442468) bank183036 +183037 POINT(36.9186989234073 -122.4186633005365) bank183037 +183038 POINT(37.43525735898486 -122.25837118135391) bank183038 +183039 POINT(36.78183894020892 -121.72245824690012) bank183039 +183040 POINT(38.20026421022245 -122.85257127686315) bank183040 +183041 POINT(37.941009368400735 -123.07154282085872) bank183041 +183042 POINT(37.25236533505797 -122.03906808846313) bank183042 +183043 POINT(38.52840982220696 -123.09200032819103) bank183043 +183044 POINT(38.59885906472407 -122.38514835655093) bank183044 +183045 POINT(38.414006480948096 -122.05756772956205) bank183045 +183046 POINT(38.56700420634945 -121.4778919495533) bank183046 +183047 POINT(37.74173797402031 -122.90255399197923) bank183047 +183048 POINT(37.288092546262945 -121.72720864125846) bank183048 +183049 POINT(38.4788670334141 -121.581293965541) bank183049 +183050 POINT(37.244471482622906 -123.17249329610256) bank183050 +183051 POINT(38.502109625066055 -122.9881492605107) bank183051 +183052 POINT(38.0448241263855 -122.11489983491906) bank183052 +183053 POINT(37.91523609591733 -122.74213793979737) bank183053 +183054 POINT(38.58172712432185 -122.51932546975088) bank183054 +183055 POINT(37.07242528457666 -122.08145697227327) bank183055 +183056 POINT(37.75324009721492 -121.6007648060786) bank183056 +183057 POINT(38.44305378981685 -122.30201419138143) bank183057 +183058 POINT(37.72855505655628 -121.63259419392163) bank183058 +183059 POINT(37.262604980029124 -122.01891648069699) bank183059 +183060 POINT(38.6487829808701 -122.39372402929676) bank183060 +183061 POINT(38.2271170290976 -121.44198410016271) bank183061 +183062 POINT(37.59940415438697 -122.29745016609385) bank183062 +183063 POINT(38.55617628034491 -122.69146819203647) bank183063 +183064 POINT(37.4330602510141 -121.47204695822023) bank183064 +183065 POINT(38.25605958976423 -121.71253306604481) bank183065 +183066 POINT(38.313166581462795 -122.69525298747868) bank183066 +183067 POINT(38.468444119570655 -121.7037592710847) bank183067 +183068 POINT(38.26630074360412 -121.46963210790628) bank183068 +183069 POINT(38.53031300039237 -122.91016391153357) bank183069 +183070 POINT(37.37881824435369 -121.87366476438487) bank183070 +183071 POINT(37.80003201933718 -122.7140350324834) bank183071 +183072 POINT(36.90574951519809 -121.8441967950903) bank183072 +183073 POINT(37.50304915423376 -122.2940924975084) bank183073 +183074 POINT(38.21771267275421 -122.07392745070013) bank183074 +183075 POINT(38.248347522949835 -122.55134459732777) bank183075 +183076 POINT(37.95991015334449 -121.56387353330705) bank183076 +183077 POINT(37.34258531441283 -123.15980254004423) bank183077 +183078 POINT(37.13621433204449 -122.63098967045076) bank183078 +183079 POINT(37.21241279066595 -122.84197467655804) bank183079 +183080 POINT(37.00524375599831 -121.94281551342921) bank183080 +183081 POINT(38.511705859453784 -122.10053696193145) bank183081 +183082 POINT(37.79437296149027 -122.62236936769234) bank183082 +183083 POINT(38.64596549231619 -123.25679348335694) bank183083 +183084 POINT(38.11887281016516 -122.04686973408124) bank183084 +183085 POINT(37.438331320138666 -122.39694069629971) bank183085 +183086 POINT(37.9540011712356 -121.63751418418327) bank183086 +183087 POINT(38.67644464674915 -122.16281849518255) bank183087 +183088 POINT(37.23603425452531 -122.13263059648389) bank183088 +183089 POINT(37.22386188292049 -121.75142507399838) bank183089 +183090 POINT(37.803794578599465 -122.00546153632278) bank183090 +183091 POINT(38.33502379451399 -121.99142520916757) bank183091 +183092 POINT(36.79749670732622 -122.44753201594438) bank183092 +183093 POINT(37.90870368796758 -122.58732207386915) bank183093 +183094 POINT(36.85493442545446 -122.20551447597622) bank183094 +183095 POINT(38.14836114964599 -123.1792514311426) bank183095 +183096 POINT(38.03479331840001 -122.26827319995705) bank183096 +183097 POINT(37.22239601176916 -121.7205993468222) bank183097 +183098 POINT(37.70257070144731 -121.59838383647042) bank183098 +183099 POINT(37.39093722811181 -122.95200627599128) bank183099 +183100 POINT(38.16272205102491 -121.9260568524386) bank183100 +183101 POINT(37.27730607920987 -122.03008260365567) bank183101 +183102 POINT(38.741180817243965 -122.73524460498658) bank183102 +183103 POINT(37.93655800104589 -122.22508683046406) bank183103 +183104 POINT(37.51694079073659 -122.2471612885832) bank183104 +183105 POINT(38.111730572310385 -122.02322166596379) bank183105 +183106 POINT(38.62811648931011 -123.22868479709784) bank183106 +183107 POINT(38.519631737987794 -121.48487713372872) bank183107 +183108 POINT(37.58768051366446 -121.82443592886497) bank183108 +183109 POINT(37.61260083718024 -121.51664770145636) bank183109 +183110 POINT(37.313780409936385 -122.74188775723813) bank183110 +183111 POINT(38.659570985394616 -122.2654958587937) bank183111 +183112 POINT(38.066016541007905 -122.60323239589509) bank183112 +183113 POINT(38.13037939598316 -122.12159791359949) bank183113 +183114 POINT(37.359506176241 -121.42724970212329) bank183114 +183115 POINT(36.94280147642985 -123.31578499565765) bank183115 +183116 POINT(38.735583692405555 -122.76543171196771) bank183116 +183117 POINT(37.68451953564829 -123.22891938712998) bank183117 +183118 POINT(37.80239581318253 -121.6082497735978) bank183118 +183119 POINT(38.76767133599761 -122.39518574559658) bank183119 +183120 POINT(37.058592784729704 -122.63893911189976) bank183120 +183121 POINT(37.15098093738851 -123.16450711970819) bank183121 +183122 POINT(38.08260497852716 -121.75299450628685) bank183122 +183123 POINT(37.82668114956286 -123.0664132246135) bank183123 +183124 POINT(37.817534813043274 -122.90350887782712) bank183124 +183125 POINT(37.08800105938824 -123.28983808587854) bank183125 +183126 POINT(37.44380589724224 -122.83612933336971) bank183126 +183127 POINT(37.44810598383808 -122.16483289778047) bank183127 +183128 POINT(38.532286193680704 -122.88129687557499) bank183128 +183129 POINT(36.86468179633518 -122.28965316428966) bank183129 +183130 POINT(36.80881487463401 -122.12953593199525) bank183130 +183131 POINT(37.93562016596437 -123.35833235470822) bank183131 +183132 POINT(37.32340417613784 -122.43451385949759) bank183132 +183133 POINT(37.88435529834533 -123.19323015533331) bank183133 +183134 POINT(37.55468628972545 -122.29183328322299) bank183134 +183135 POINT(37.50516933520511 -122.14177867186453) bank183135 +183136 POINT(37.245836525788235 -122.19243960010164) bank183136 +183137 POINT(37.85590353163903 -122.59152397813853) bank183137 +183138 POINT(38.56072001661777 -122.12282570772297) bank183138 +183139 POINT(38.224764408884404 -122.55933666154687) bank183139 +183140 POINT(37.032087274602084 -122.33982733858443) bank183140 +183141 POINT(38.324799523587465 -122.77159257883618) bank183141 +183142 POINT(36.867416292375786 -122.84008184291613) bank183142 +183143 POINT(36.98383218761723 -123.34391216762764) bank183143 +183144 POINT(37.69424389908499 -122.0701099639794) bank183144 +183145 POINT(38.315618647231645 -122.84629680745289) bank183145 +183146 POINT(38.33458577008567 -122.5618607270702) bank183146 +183147 POINT(36.86228279728571 -122.20295368753686) bank183147 +183148 POINT(37.356186733136596 -122.061605589778) bank183148 +183149 POINT(37.04589347964451 -122.59984727007189) bank183149 +183150 POINT(38.476095379141995 -121.51750299249494) bank183150 +183151 POINT(38.12057243403826 -122.925892175154) bank183151 +183152 POINT(37.59979553309056 -122.50371562571308) bank183152 +183153 POINT(37.657356863165745 -121.87944538955618) bank183153 +183154 POINT(38.02941693140934 -122.74643445518785) bank183154 +183155 POINT(37.36556869375002 -122.19223479004228) bank183155 +183156 POINT(37.5862090978335 -121.43165656725411) bank183156 +183157 POINT(37.205262653941624 -121.74458149358328) bank183157 +183158 POINT(38.348082870230584 -121.71824252857373) bank183158 +183159 POINT(37.95211047626812 -122.27340301853) bank183159 +183160 POINT(37.6204086621179 -123.10449659180208) bank183160 +183161 POINT(37.34231449775464 -121.83808727482202) bank183161 +183162 POINT(38.27576707091673 -121.73966915315826) bank183162 +183163 POINT(38.03170051099337 -121.56104746950979) bank183163 +183164 POINT(37.91131588639398 -122.15345436379909) bank183164 +183165 POINT(37.51906177936994 -121.99675025234534) bank183165 +183166 POINT(37.86542418292458 -123.03415449253134) bank183166 +183167 POINT(37.97335268303384 -121.83558897051047) bank183167 +183168 POINT(37.045444507622484 -122.93856433882482) bank183168 +183169 POINT(38.764787383198794 -122.33601579702217) bank183169 +183170 POINT(38.33348894097267 -121.5391517655234) bank183170 +183171 POINT(37.68174467040867 -122.14153301774317) bank183171 +183172 POINT(38.77114181035022 -123.31520063118724) bank183172 +183173 POINT(37.1089976971625 -123.35149355818464) bank183173 +183174 POINT(37.067729991192124 -123.40273713914219) bank183174 +183175 POINT(37.477165188342966 -121.86052197768258) bank183175 +183176 POINT(37.51235888487337 -121.87536319965722) bank183176 +183177 POINT(37.99966072432837 -121.76878651769775) bank183177 +183178 POINT(36.93136943678411 -121.434094974112) bank183178 +183179 POINT(38.45840635801294 -121.70403330711171) bank183179 +183180 POINT(37.65007462581534 -121.42319957451544) bank183180 +183181 POINT(37.0271121126412 -121.62097598991171) bank183181 +183182 POINT(37.04037616134596 -121.8874093647087) bank183182 +183183 POINT(38.09655363639611 -121.53002132422961) bank183183 +183184 POINT(37.91503898453588 -123.11616754064426) bank183184 +183185 POINT(38.33826039979995 -121.65388144487316) bank183185 +183186 POINT(37.98399366818913 -123.06630018673314) bank183186 +183187 POINT(38.37087190136582 -121.62952021191485) bank183187 +183188 POINT(38.752262269617866 -122.62108946837655) bank183188 +183189 POINT(37.528888379181076 -122.89024161068266) bank183189 +183190 POINT(38.52122275225043 -122.52453990765355) bank183190 +183191 POINT(38.523372668589005 -121.73826050584336) bank183191 +183192 POINT(37.643401199000174 -121.72296567597903) bank183192 +183193 POINT(37.973517295939246 -121.88756918093831) bank183193 +183194 POINT(37.08450776633254 -121.7285128419677) bank183194 +183195 POINT(37.64233205121984 -123.21489804935574) bank183195 +183196 POINT(38.67154427352886 -123.1700687776871) bank183196 +183197 POINT(38.18549052520562 -121.5753205169187) bank183197 +183198 POINT(38.44153862943859 -122.28743601962974) bank183198 +183199 POINT(38.375434550853065 -122.89440297548542) bank183199 +183200 POINT(37.67946654753809 -122.68217830385275) bank183200 +183201 POINT(37.670766905109886 -121.78242209257786) bank183201 +183202 POINT(37.31818423640946 -121.78218662498548) bank183202 +183203 POINT(37.14910745863713 -121.6893755864039) bank183203 +183204 POINT(37.32869254823018 -123.144835559732) bank183204 +183205 POINT(38.20621817866934 -121.57551728118682) bank183205 +183206 POINT(38.0103980207052 -121.8051553851969) bank183206 +183207 POINT(38.29212709926572 -123.30017392054927) bank183207 +183208 POINT(37.86545514610101 -123.11262958202758) bank183208 +183209 POINT(36.832855057711384 -121.54273055324073) bank183209 +183210 POINT(38.69767818709368 -121.63896399371018) bank183210 +183211 POINT(37.21055457345606 -121.96722373621697) bank183211 +183212 POINT(37.36760238133087 -122.10179741536068) bank183212 +183213 POINT(37.50503283511302 -122.37227458596256) bank183213 +183214 POINT(38.196710341675505 -123.00630197451632) bank183214 +183215 POINT(38.3189272602651 -122.61549614292164) bank183215 +183216 POINT(38.27481262027656 -122.84002018776373) bank183216 +183217 POINT(38.605058055514114 -122.25426549345579) bank183217 +183218 POINT(38.701284677330456 -122.33073655650452) bank183218 +183219 POINT(38.67953538336088 -123.00332431029891) bank183219 +183220 POINT(37.25525875377939 -122.2520020690407) bank183220 +183221 POINT(38.20715373996044 -123.05851381432048) bank183221 +183222 POINT(36.96508053265659 -122.70767260488083) bank183222 +183223 POINT(37.19098509392924 -122.39385300018225) bank183223 +183224 POINT(37.418560825015184 -123.29049446900878) bank183224 +183225 POINT(36.846389433210945 -121.89867398215551) bank183225 +183226 POINT(37.590347944398026 -122.36990669464154) bank183226 +183227 POINT(37.23464253463957 -122.97158035505457) bank183227 +183228 POINT(38.264664812202994 -123.20846626789334) bank183228 +183229 POINT(38.15029928406458 -121.59988380289569) bank183229 +183230 POINT(37.4213067314532 -121.79935450576029) bank183230 +183231 POINT(37.718463489260806 -123.26786887066848) bank183231 +183232 POINT(37.07417507356775 -123.34587098128594) bank183232 +183233 POINT(38.24949582358292 -121.58390352279403) bank183233 +183234 POINT(38.187862567432234 -122.31074617856473) bank183234 +183235 POINT(38.167615128608205 -122.66229997181631) bank183235 +183236 POINT(37.45223504407806 -123.16008919650366) bank183236 +183237 POINT(37.7461123225935 -122.97763650493354) bank183237 +183238 POINT(37.6446765220678 -121.57514384966686) bank183238 +183239 POINT(37.087664929805335 -122.22639646320417) bank183239 +183240 POINT(37.73868098622906 -122.42761923887892) bank183240 +183241 POINT(37.2543350158612 -122.78652667658314) bank183241 +183242 POINT(37.22352218358251 -122.5863678072789) bank183242 +183243 POINT(38.6827576883917 -121.61858754141554) bank183243 +183244 POINT(37.50820358338402 -121.85277727490656) bank183244 +183245 POINT(37.84384025788846 -121.44622557019365) bank183245 +183246 POINT(37.30919319792543 -121.9517827215134) bank183246 +183247 POINT(37.98914800187342 -121.46454145644834) bank183247 +183248 POINT(37.28256706681534 -122.24688883855971) bank183248 +183249 POINT(38.68947180737492 -121.70104179480596) bank183249 +183250 POINT(38.74426515647581 -121.84858861577) bank183250 +183251 POINT(38.636797907181865 -123.3838218866046) bank183251 +183252 POINT(38.35391854814026 -122.04997177708923) bank183252 +183253 POINT(36.93865955342299 -121.67851174508496) bank183253 +183254 POINT(37.540214895858064 -121.7013063681434) bank183254 +183255 POINT(37.56582216388089 -122.62106640359951) bank183255 +183256 POINT(37.90095164898413 -122.03547990574432) bank183256 +183257 POINT(37.03173087497496 -122.0477185256372) bank183257 +183258 POINT(36.83822666717785 -121.68536685305477) bank183258 +183259 POINT(37.17226186427405 -122.15483313153335) bank183259 +183260 POINT(37.18603165492393 -122.4482716648398) bank183260 +183261 POINT(38.277700314456006 -122.87439688347108) bank183261 +183262 POINT(36.804616349956376 -123.32111815720822) bank183262 +183263 POINT(37.579917745172956 -123.21840339025306) bank183263 +183264 POINT(37.70588826556378 -121.72114935308441) bank183264 +183265 POINT(38.239564887149804 -121.76396280230634) bank183265 +183266 POINT(36.91542684157482 -121.54083600095096) bank183266 +183267 POINT(37.140896394362045 -122.46133608584739) bank183267 +183268 POINT(38.43638786385043 -123.05465455788166) bank183268 +183269 POINT(37.656774840453885 -121.54474688232857) bank183269 +183270 POINT(37.33215216144121 -122.65287148834088) bank183270 +183271 POINT(36.885625852640146 -121.96769093625166) bank183271 +183272 POINT(38.364197930253354 -122.93104189895418) bank183272 +183273 POINT(37.090407035555984 -121.628509953391) bank183273 +183274 POINT(37.967888623316085 -121.95598520831896) bank183274 +183275 POINT(38.61477880083751 -121.50107069998376) bank183275 +183276 POINT(37.57462683922785 -121.89164277941296) bank183276 +183277 POINT(37.404379488021256 -122.44052060506148) bank183277 +183278 POINT(38.650614664710695 -123.20978130403304) bank183278 +183279 POINT(37.62163091914913 -123.32306876625276) bank183279 +183280 POINT(37.96357635073923 -121.44520458230735) bank183280 +183281 POINT(37.22884892654312 -121.65842894802107) bank183281 +183282 POINT(37.147512158579 -121.89452684504053) bank183282 +183283 POINT(36.901673564511604 -123.25722925204678) bank183283 +183284 POINT(38.232048211605736 -123.06154554613467) bank183284 +183285 POINT(37.983903494048995 -122.51617668995326) bank183285 +183286 POINT(36.99096570046431 -123.16739572131041) bank183286 +183287 POINT(38.33511584133402 -121.46091911086216) bank183287 +183288 POINT(38.483605940372925 -121.64856923999338) bank183288 +183289 POINT(36.84289474748788 -122.15191093472818) bank183289 +183290 POINT(36.821324293968416 -121.66307884404559) bank183290 +183291 POINT(38.29909109978794 -122.14234330129) bank183291 +183292 POINT(36.893069427711104 -122.97752417443162) bank183292 +183293 POINT(38.193944989421475 -122.47198019188686) bank183293 +183294 POINT(36.81149977007181 -121.99521620328109) bank183294 +183295 POINT(36.778496742947894 -123.21942767523569) bank183295 +183296 POINT(37.577037339309065 -121.66578100367667) bank183296 +183297 POINT(37.2481120307347 -122.08595141249043) bank183297 +183298 POINT(38.70086959280589 -123.273303372548) bank183298 +183299 POINT(36.8970313938416 -122.18667236200305) bank183299 +183300 POINT(36.79125200528812 -122.8999103703265) bank183300 +183301 POINT(37.389001355146036 -121.88341914432368) bank183301 +183302 POINT(37.656374830470924 -122.35523254588315) bank183302 +183303 POINT(38.068921367242325 -122.80956340177056) bank183303 +183304 POINT(38.42878564074674 -122.08816796829802) bank183304 +183305 POINT(36.86652382984399 -122.90629369147959) bank183305 +183306 POINT(37.761433384603414 -122.15830261798399) bank183306 +183307 POINT(38.18509034455802 -121.95862233271555) bank183307 +183308 POINT(38.46140283596377 -122.77190577176032) bank183308 +183309 POINT(37.84281156044237 -121.65415248842334) bank183309 +183310 POINT(38.19803676003595 -121.64483209191539) bank183310 +183311 POINT(38.42042981067249 -123.29957622196719) bank183311 +183312 POINT(37.41787468062062 -121.43713585359198) bank183312 +183313 POINT(38.02108778627033 -121.49928997330606) bank183313 +183314 POINT(37.71978131840851 -122.72499370369162) bank183314 +183315 POINT(38.042882119303144 -122.17180863140189) bank183315 +183316 POINT(38.47146437127641 -122.48579712203494) bank183316 +183317 POINT(38.02634443435372 -122.9401677258032) bank183317 +183318 POINT(37.50060351080599 -121.70705205673923) bank183318 +183319 POINT(37.2085509378386 -121.49074839642748) bank183319 +183320 POINT(38.31092416240362 -121.94905694561885) bank183320 +183321 POINT(38.68422869495104 -121.60047247732734) bank183321 +183322 POINT(38.55265126937918 -121.6911887807776) bank183322 +183323 POINT(37.8740170463941 -122.01947036293377) bank183323 +183324 POINT(37.929079830895155 -121.62255534987273) bank183324 +183325 POINT(37.18523892400483 -122.35496268391249) bank183325 +183326 POINT(36.77949415332266 -122.2163049592925) bank183326 +183327 POINT(38.17931689337045 -121.91106860326765) bank183327 +183328 POINT(37.82382847557941 -122.62667172477063) bank183328 +183329 POINT(38.54783098950147 -123.34670728700928) bank183329 +183330 POINT(38.57803394265341 -121.60483740832376) bank183330 +183331 POINT(38.19784392634494 -122.87855054272374) bank183331 +183332 POINT(38.65256937151702 -122.43177806289422) bank183332 +183333 POINT(37.183212467093426 -122.20712550693341) bank183333 +183334 POINT(37.335587075199456 -122.71846798643315) bank183334 +183335 POINT(36.798260590745464 -122.88808496033819) bank183335 +183336 POINT(36.79795891022967 -122.57287769354988) bank183336 +183337 POINT(38.63279200471108 -123.11590124911596) bank183337 +183338 POINT(38.61913682650181 -122.36306800313358) bank183338 +183339 POINT(37.86905969254629 -122.7717011554593) bank183339 +183340 POINT(37.15505899504979 -123.2193196137151) bank183340 +183341 POINT(38.570568577242575 -122.8662865173652) bank183341 +183342 POINT(38.5524110658672 -123.10982470632867) bank183342 +183343 POINT(37.11826824921776 -121.4321145863045) bank183343 +183344 POINT(37.46692443911315 -122.13150522616962) bank183344 +183345 POINT(36.98249319977109 -122.94325713822215) bank183345 +183346 POINT(38.47833778710646 -122.56687923337745) bank183346 +183347 POINT(37.54169342642418 -121.478255739324) bank183347 +183348 POINT(38.341893695552336 -122.65430059101277) bank183348 +183349 POINT(38.76090302999365 -123.1424130349593) bank183349 +183350 POINT(37.150835105457524 -123.21084639722231) bank183350 +183351 POINT(38.498723700044515 -123.23422387944422) bank183351 +183352 POINT(38.220807402577044 -122.57931518646431) bank183352 +183353 POINT(36.80972354701102 -123.01048494957658) bank183353 +183354 POINT(37.83214887100345 -122.12155369573148) bank183354 +183355 POINT(37.022412011370825 -122.75117546911758) bank183355 +183356 POINT(37.68576889411453 -121.4350055680675) bank183356 +183357 POINT(37.55969020370673 -122.55968923735794) bank183357 +183358 POINT(36.78500726429158 -123.01217700975263) bank183358 +183359 POINT(36.96160010746481 -122.35113114945428) bank183359 +183360 POINT(37.100948042074066 -122.97189133353322) bank183360 +183361 POINT(37.10228257270828 -122.9364588974114) bank183361 +183362 POINT(36.842298126551675 -121.50843672454626) bank183362 +183363 POINT(38.33735206711009 -121.71595345564195) bank183363 +183364 POINT(37.058753813176686 -123.12800556577928) bank183364 +183365 POINT(38.595177517895515 -122.64038103209619) bank183365 +183366 POINT(37.71079700329684 -122.30638865979617) bank183366 +183367 POINT(37.46577815472172 -122.05817962950577) bank183367 +183368 POINT(38.5234547449008 -123.04080799494177) bank183368 +183369 POINT(36.81531354874784 -122.6316544175997) bank183369 +183370 POINT(37.231190856945034 -122.18995504390801) bank183370 +183371 POINT(37.4513209953476 -121.49380375974135) bank183371 +183372 POINT(38.26373108322291 -121.54016711583479) bank183372 +183373 POINT(36.94021636178648 -121.70188219632622) bank183373 +183374 POINT(38.02863284470694 -121.89548296440739) bank183374 +183375 POINT(38.53662607914869 -123.33277381903345) bank183375 +183376 POINT(38.23620180983768 -122.33690874578058) bank183376 +183377 POINT(38.0251221422511 -122.05259796329585) bank183377 +183378 POINT(37.6663503824639 -121.82313477307136) bank183378 +183379 POINT(37.46818722566386 -122.95284667793584) bank183379 +183380 POINT(38.02929311472829 -123.22097668183217) bank183380 +183381 POINT(38.74564307709735 -121.83454023074593) bank183381 +183382 POINT(37.41828097064585 -122.93934477136685) bank183382 +183383 POINT(37.97789473428771 -122.30989168610012) bank183383 +183384 POINT(38.275852532639554 -121.97300875256488) bank183384 +183385 POINT(38.53000452202731 -123.32198402268664) bank183385 +183386 POINT(38.74132238920882 -122.27372413634893) bank183386 +183387 POINT(38.06310631826295 -122.82372308406016) bank183387 +183388 POINT(38.689704778950826 -122.18284930685425) bank183388 +183389 POINT(37.864958756004434 -121.6494739237481) bank183389 +183390 POINT(38.705188977312 -122.30130498355656) bank183390 +183391 POINT(38.28624601073341 -121.98987775044938) bank183391 +183392 POINT(36.884554355284045 -122.14446298001802) bank183392 +183393 POINT(38.044228137948885 -123.25071561553673) bank183393 +183394 POINT(37.5213850993425 -122.41867377307054) bank183394 +183395 POINT(36.91460273430219 -123.21108414573607) bank183395 +183396 POINT(37.31351969377882 -123.35904049708768) bank183396 +183397 POINT(37.92854998417222 -122.48413201585507) bank183397 +183398 POINT(38.386935563324755 -121.46242938353808) bank183398 +183399 POINT(38.20883882983703 -122.19910233794107) bank183399 +183400 POINT(38.742734291468174 -123.21947220981654) bank183400 +183401 POINT(37.19855742167915 -121.93451279852982) bank183401 +183402 POINT(38.12161653087017 -123.21248901534163) bank183402 +183403 POINT(37.66946385569452 -122.22466422858494) bank183403 +183404 POINT(37.26557224485623 -122.57832542402953) bank183404 +183405 POINT(38.41991825075124 -123.36674759364219) bank183405 +183406 POINT(38.338808182657424 -123.41279151876937) bank183406 +183407 POINT(38.68279447795615 -123.30407857659489) bank183407 +183408 POINT(38.558511907469594 -123.41232091388729) bank183408 +183409 POINT(38.41677092071149 -121.72953025811404) bank183409 +183410 POINT(37.90031286319629 -122.9252617588758) bank183410 +183411 POINT(37.86147109380605 -123.37366897745181) bank183411 +183412 POINT(37.649777576048216 -121.95999233278411) bank183412 +183413 POINT(37.77882329271688 -122.81125221577084) bank183413 +183414 POINT(37.84757159335922 -121.55260969914691) bank183414 +183415 POINT(38.376859715323775 -122.34029625121926) bank183415 +183416 POINT(37.69092116489222 -122.1101525594093) bank183416 +183417 POINT(37.234688842586586 -123.27351290170135) bank183417 +183418 POINT(38.65248919554408 -121.64468958133318) bank183418 +183419 POINT(38.658540160984394 -122.08945017827361) bank183419 +183420 POINT(37.696941109566566 -122.06059840456126) bank183420 +183421 POINT(37.43374341549316 -121.6646134300928) bank183421 +183422 POINT(37.66482952468183 -121.72174537514151) bank183422 +183423 POINT(37.08243536304156 -122.51041122077318) bank183423 +183424 POINT(36.8425548520339 -122.26842558954509) bank183424 +183425 POINT(37.63843745927172 -123.3173192456733) bank183425 +183426 POINT(38.121645407316876 -123.31671586514115) bank183426 +183427 POINT(37.705666256702635 -121.56057853902202) bank183427 +183428 POINT(37.4339064382765 -123.40777989390047) bank183428 +183429 POINT(38.362599151355326 -122.56290524908755) bank183429 +183430 POINT(37.17120679521648 -121.43756965726857) bank183430 +183431 POINT(37.657915772270734 -121.52458900606392) bank183431 +183432 POINT(36.94786805893769 -121.6532046965707) bank183432 +183433 POINT(37.391636343517895 -122.26883834597236) bank183433 +183434 POINT(37.53610399071405 -121.78815696839801) bank183434 +183435 POINT(37.78751090319177 -122.74834431619404) bank183435 +183436 POINT(37.15974377585452 -122.78042230499156) bank183436 +183437 POINT(38.016161357353106 -123.17598612587075) bank183437 +183438 POINT(37.851183996501035 -123.11249721414381) bank183438 +183439 POINT(37.812873275286975 -121.70383317217528) bank183439 +183440 POINT(37.446197766417825 -122.3969353635831) bank183440 +183441 POINT(37.3825360379895 -121.75912804631015) bank183441 +183442 POINT(37.55947742022182 -122.20953425482818) bank183442 +183443 POINT(37.24702634636698 -122.66731995778451) bank183443 +183444 POINT(37.071159515262174 -123.13323656384931) bank183444 +183445 POINT(37.9505894804885 -123.37111816132263) bank183445 +183446 POINT(38.28431831537264 -122.77346447586419) bank183446 +183447 POINT(38.3674233863645 -123.1927918440583) bank183447 +183448 POINT(38.17072960185128 -121.54163532237908) bank183448 +183449 POINT(37.28050985764944 -122.24428084673835) bank183449 +183450 POINT(36.776558631218904 -122.83667903721593) bank183450 +183451 POINT(37.84207065912264 -123.19709696698555) bank183451 +183452 POINT(37.353461507614846 -122.80610046174185) bank183452 +183453 POINT(37.81908119772202 -122.4392906007322) bank183453 +183454 POINT(37.13795888109385 -122.4375956403441) bank183454 +183455 POINT(37.247177434217924 -123.11670096533307) bank183455 +183456 POINT(37.61815603903454 -122.91838114742005) bank183456 +183457 POINT(37.96881099695976 -122.69300561041011) bank183457 +183458 POINT(36.892903691663086 -122.30140895127255) bank183458 +183459 POINT(37.282813850705104 -122.78205886602207) bank183459 +183460 POINT(38.48606916653127 -122.35696740540757) bank183460 +183461 POINT(37.71018873747221 -121.97334648054115) bank183461 +183462 POINT(37.241736061317496 -123.16996301773837) bank183462 +183463 POINT(38.17948468571878 -122.35521793029547) bank183463 +183464 POINT(38.03941140704902 -122.32391707804223) bank183464 +183465 POINT(37.42688576033859 -122.82918042139497) bank183465 +183466 POINT(37.42427596502523 -123.26461224743122) bank183466 +183467 POINT(37.95813692508406 -122.50402786773323) bank183467 +183468 POINT(38.447875475728736 -123.27943147001652) bank183468 +183469 POINT(36.955645978561726 -121.51715610064629) bank183469 +183470 POINT(37.23900322730369 -122.81059925758947) bank183470 +183471 POINT(36.81358998895039 -122.79159103126989) bank183471 +183472 POINT(36.87305435416994 -122.34757836035179) bank183472 +183473 POINT(38.381913587917474 -123.4033803425377) bank183473 +183474 POINT(38.596672386436154 -123.39843411552278) bank183474 +183475 POINT(37.01756207982061 -121.60177398862139) bank183475 +183476 POINT(38.540573113905005 -122.87462597303167) bank183476 +183477 POINT(38.50615490878674 -122.17796268092972) bank183477 +183478 POINT(38.41643090284009 -122.83446923658818) bank183478 +183479 POINT(38.70244676325237 -122.16990963166054) bank183479 +183480 POINT(38.2212865145742 -123.31620956095473) bank183480 +183481 POINT(38.307378134075975 -123.29658978864272) bank183481 +183482 POINT(37.996447331534 -123.26251426728433) bank183482 +183483 POINT(38.576679988615716 -121.64655329823655) bank183483 +183484 POINT(37.04076358999392 -122.03841461608916) bank183484 +183485 POINT(37.1119380374554 -122.51554874934114) bank183485 +183486 POINT(38.22763542476258 -122.96262918445066) bank183486 +183487 POINT(37.436343867429926 -122.21481735740008) bank183487 +183488 POINT(37.42532052971013 -121.86058933601564) bank183488 +183489 POINT(38.36530825638997 -121.89603927419556) bank183489 +183490 POINT(37.93022277173608 -121.80918265406157) bank183490 +183491 POINT(37.1958645901814 -123.07082895267312) bank183491 +183492 POINT(38.71229962243797 -121.80122671910433) bank183492 +183493 POINT(38.38343925516677 -122.72363724477381) bank183493 +183494 POINT(37.12974131524095 -122.14577307491784) bank183494 +183495 POINT(36.97520556737101 -121.47014562172325) bank183495 +183496 POINT(37.916275412679084 -122.30535867106121) bank183496 +183497 POINT(37.8411356446811 -122.58998842269325) bank183497 +183498 POINT(38.718102655054 -122.98771259557797) bank183498 +183499 POINT(38.14027714923019 -122.67133555360105) bank183499 +183500 POINT(38.208128977685 -122.25056653770396) bank183500 +183501 POINT(38.7162001424048 -122.00755349764214) bank183501 +183502 POINT(38.29167462197015 -122.26063262547015) bank183502 +183503 POINT(38.22959161673698 -121.83942253505502) bank183503 +183504 POINT(37.317973937546135 -122.1391777848099) bank183504 +183505 POINT(38.109900390160725 -122.3004182909793) bank183505 +183506 POINT(36.907366090995936 -123.29936747002846) bank183506 +183507 POINT(38.62392526181701 -123.22049889416675) bank183507 +183508 POINT(36.77977220905706 -121.54055527852503) bank183508 +183509 POINT(37.994477726832464 -122.58616390845806) bank183509 +183510 POINT(38.07051296490261 -121.8614771955894) bank183510 +183511 POINT(38.378711311395485 -122.26743669082373) bank183511 +183512 POINT(37.29066928867888 -123.19759467712262) bank183512 +183513 POINT(38.5633745500029 -122.00871136948925) bank183513 +183514 POINT(37.38872822298738 -121.91606449084314) bank183514 +183515 POINT(37.56235182863988 -121.66842873323614) bank183515 +183516 POINT(37.52300469798977 -122.32724599070173) bank183516 +183517 POINT(38.008987896619324 -121.94462734145077) bank183517 +183518 POINT(36.92489775284361 -122.96755594706873) bank183518 +183519 POINT(37.845349862553086 -122.77684215616627) bank183519 +183520 POINT(37.94497878000457 -123.15409312428439) bank183520 +183521 POINT(36.98850519425267 -123.27525271593555) bank183521 +183522 POINT(36.867031723718675 -121.45734928437206) bank183522 +183523 POINT(38.393782323032795 -123.1493947534465) bank183523 +183524 POINT(38.22348767523679 -123.2686272644139) bank183524 +183525 POINT(37.85826452194191 -123.17308910458814) bank183525 +183526 POINT(38.13510984771122 -122.54316899524805) bank183526 +183527 POINT(38.68389538611399 -122.33471059391596) bank183527 +183528 POINT(37.68745482758646 -121.74816126601928) bank183528 +183529 POINT(37.170092145251274 -123.25737387401338) bank183529 +183530 POINT(36.859925782663055 -123.04910520197095) bank183530 +183531 POINT(38.63624135764836 -123.04122152349791) bank183531 +183532 POINT(38.29646506725653 -121.65549900465786) bank183532 +183533 POINT(37.32273744143085 -122.3162290271375) bank183533 +183534 POINT(38.31004972409779 -121.81808011188524) bank183534 +183535 POINT(38.07228837716714 -121.67259473370477) bank183535 +183536 POINT(37.31951429990804 -122.4396948717896) bank183536 +183537 POINT(38.244727867788725 -122.66715629687955) bank183537 +183538 POINT(37.95827415284512 -122.16237565371576) bank183538 +183539 POINT(37.4976858966354 -122.27733937818975) bank183539 +183540 POINT(37.926048118873915 -123.0908062994001) bank183540 +183541 POINT(38.528886951189364 -121.59895938892141) bank183541 +183542 POINT(38.71472022501591 -122.61868091094493) bank183542 +183543 POINT(37.053786641648 -122.74827914689367) bank183543 +183544 POINT(37.924902433945164 -121.73981193619892) bank183544 +183545 POINT(37.61321446677703 -123.09089643597751) bank183545 +183546 POINT(38.16825407117577 -122.4967032693312) bank183546 +183547 POINT(38.204393762145976 -122.78809431812132) bank183547 +183548 POINT(37.34550226763665 -122.88322149724193) bank183548 +183549 POINT(37.62205468692254 -121.52506410186005) bank183549 +183550 POINT(37.93190133885046 -122.40561131446361) bank183550 +183551 POINT(37.287188205204 -122.61846361035641) bank183551 +183552 POINT(38.148026494323624 -122.05548548734613) bank183552 +183553 POINT(37.77616773737885 -122.77922440778964) bank183553 +183554 POINT(38.75349089141925 -122.99454956857558) bank183554 +183555 POINT(37.49695309513591 -122.51469900909059) bank183555 +183556 POINT(37.763901837921644 -123.31373300851115) bank183556 +183557 POINT(38.234971773218206 -123.39662968853142) bank183557 +183558 POINT(38.59246116648785 -122.19041312394678) bank183558 +183559 POINT(37.328204269713645 -122.5299054361548) bank183559 +183560 POINT(37.60857109404514 -122.28863267339648) bank183560 +183561 POINT(37.62077093144195 -122.38753720622852) bank183561 +183562 POINT(36.94933474182053 -121.70757395862252) bank183562 +183563 POINT(37.60653808305947 -122.12977395497171) bank183563 +183564 POINT(38.216412080116356 -122.57765168289313) bank183564 +183565 POINT(38.47578326068363 -122.29956972387014) bank183565 +183566 POINT(37.35998373930108 -121.69047625078424) bank183566 +183567 POINT(38.36121799293508 -122.24709621398104) bank183567 +183568 POINT(37.01503679161578 -122.72112064800812) bank183568 +183569 POINT(37.82291917775715 -122.36218152020919) bank183569 +183570 POINT(38.20289940277053 -122.92912544353156) bank183570 +183571 POINT(37.930930072456526 -122.49755194341115) bank183571 +183572 POINT(37.361136903941535 -121.61467068508566) bank183572 +183573 POINT(37.00971244953994 -122.2720715466693) bank183573 +183574 POINT(38.32058713056697 -121.89861586714336) bank183574 +183575 POINT(38.653136809060264 -122.3501763883762) bank183575 +183576 POINT(37.62558701576625 -123.20020636586702) bank183576 +183577 POINT(37.89289084233031 -122.11401334566328) bank183577 +183578 POINT(37.97609453193857 -123.30456432554828) bank183578 +183579 POINT(38.60666487575765 -121.51729089603643) bank183579 +183580 POINT(38.66875290837423 -122.32390182894589) bank183580 +183581 POINT(38.500513330238675 -123.32406345448804) bank183581 +183582 POINT(37.304030037585704 -123.05061337336089) bank183582 +183583 POINT(37.20079532287483 -122.43559724799287) bank183583 +183584 POINT(37.59397772550374 -122.88406946149854) bank183584 +183585 POINT(36.97700200763484 -121.53251994515435) bank183585 +183586 POINT(38.11484178039716 -122.08115553083005) bank183586 +183587 POINT(37.57930711852193 -122.03233541066594) bank183587 +183588 POINT(38.03415673446648 -121.96626391916473) bank183588 +183589 POINT(37.99627634122958 -121.8459032651725) bank183589 +183590 POINT(38.43855662900418 -122.85555349831495) bank183590 +183591 POINT(38.27300871395099 -122.72778795874225) bank183591 +183592 POINT(36.82725385545058 -122.16020704596659) bank183592 +183593 POINT(38.58910417951678 -123.11430303381226) bank183593 +183594 POINT(37.78595365910464 -122.91800134877015) bank183594 +183595 POINT(36.89169065831539 -122.34014348330669) bank183595 +183596 POINT(38.26902322390628 -121.87709399186042) bank183596 +183597 POINT(38.188759723807486 -123.11950303463362) bank183597 +183598 POINT(38.685562824019314 -122.95919941080773) bank183598 +183599 POINT(37.26405382632708 -122.67659781203561) bank183599 +183600 POINT(37.514486475802215 -123.08569869731932) bank183600 +183601 POINT(38.118216945354284 -123.06511644534423) bank183601 +183602 POINT(37.02416517287392 -122.36951591456281) bank183602 +183603 POINT(38.04952315308755 -121.94584283165008) bank183603 +183604 POINT(37.01141038921746 -121.89011807607092) bank183604 +183605 POINT(37.40826053058665 -121.61505933478047) bank183605 +183606 POINT(37.283043785846544 -123.00085617972923) bank183606 +183607 POINT(37.3765361559971 -122.88099570416122) bank183607 +183608 POINT(38.49101168378681 -122.80543844389986) bank183608 +183609 POINT(37.52495485004408 -123.20890651968071) bank183609 +183610 POINT(38.5226561111406 -122.6172662445404) bank183610 +183611 POINT(38.46041948972555 -122.83551128629139) bank183611 +183612 POINT(38.249207769365775 -122.21096297029682) bank183612 +183613 POINT(37.71833848612576 -123.23093388166679) bank183613 +183614 POINT(37.724525872977225 -122.32525899478365) bank183614 +183615 POINT(36.78463023477004 -122.44917937971651) bank183615 +183616 POINT(36.84780507952423 -122.97931483499765) bank183616 +183617 POINT(38.6292536456086 -122.4502682995451) bank183617 +183618 POINT(36.8911063850884 -122.6932661484287) bank183618 +183619 POINT(38.62984189942671 -122.09771354200687) bank183619 +183620 POINT(37.0110143084739 -122.34600475391392) bank183620 +183621 POINT(37.532342685466176 -121.99910928449282) bank183621 +183622 POINT(37.3703437995624 -123.06006098438128) bank183622 +183623 POINT(36.85228385853164 -123.3525293633221) bank183623 +183624 POINT(36.99994463001255 -121.53289809513318) bank183624 +183625 POINT(38.47076375581532 -121.90493490786969) bank183625 +183626 POINT(37.71203520163309 -123.37471126038788) bank183626 +183627 POINT(37.38837350377337 -121.49052848107463) bank183627 +183628 POINT(38.04915530717937 -121.66186137480773) bank183628 +183629 POINT(38.12898069204341 -121.76255724596729) bank183629 +183630 POINT(37.45052325913148 -122.53691254764038) bank183630 +183631 POINT(38.053580124815085 -122.34501955138433) bank183631 +183632 POINT(37.74503952298722 -121.87794493872592) bank183632 +183633 POINT(37.51121276595147 -123.3620865148414) bank183633 +183634 POINT(38.7389396891124 -122.10538251858026) bank183634 +183635 POINT(36.840867406136724 -121.83622859273117) bank183635 +183636 POINT(38.72079031206435 -122.09502156210361) bank183636 +183637 POINT(37.63102056498618 -122.23819009418439) bank183637 +183638 POINT(37.543616129091845 -122.58078800645843) bank183638 +183639 POINT(36.786262927197946 -123.36566891174944) bank183639 +183640 POINT(36.987014876909825 -123.09528539207574) bank183640 +183641 POINT(36.830825499887794 -122.29594200389897) bank183641 +183642 POINT(37.37686672599964 -123.29033510831914) bank183642 +183643 POINT(37.221407183902166 -123.11933478761591) bank183643 +183644 POINT(38.11660436963082 -122.15639000656483) bank183644 +183645 POINT(37.619154385004585 -123.37219818644978) bank183645 +183646 POINT(37.331928261348 -122.89128565325969) bank183646 +183647 POINT(37.618522943848475 -122.15005283652418) bank183647 +183648 POINT(37.55895942092519 -122.31440158240092) bank183648 +183649 POINT(37.68949282064879 -122.47059034339556) bank183649 +183650 POINT(37.747590448891884 -121.9463531474391) bank183650 +183651 POINT(36.92142674549151 -123.21727793005593) bank183651 +183652 POINT(37.45082085411209 -122.32085593305614) bank183652 +183653 POINT(37.541628710743716 -122.04039386017683) bank183653 +183654 POINT(37.37344192496743 -121.6226476246668) bank183654 +183655 POINT(37.01469505625141 -123.13343067945607) bank183655 +183656 POINT(36.91358893077034 -121.63844321798587) bank183656 +183657 POINT(38.46117777115269 -122.04050615391553) bank183657 +183658 POINT(36.947982538902416 -122.58083990676508) bank183658 +183659 POINT(37.017732660927145 -122.05503057758757) bank183659 +183660 POINT(37.958589302869775 -122.70556742736073) bank183660 +183661 POINT(37.36586667579781 -121.98955477884182) bank183661 +183662 POINT(37.61954522496815 -122.07712271474253) bank183662 +183663 POINT(37.66159772500372 -123.35941089956737) bank183663 +183664 POINT(37.428124147949184 -123.29172326810841) bank183664 +183665 POINT(37.86796216004876 -122.59782073955049) bank183665 +183666 POINT(37.93789948718104 -123.38702147859071) bank183666 +183667 POINT(36.97322849095102 -121.44605969203153) bank183667 +183668 POINT(38.3873174364018 -123.29230492650362) bank183668 +183669 POINT(38.04108518616357 -122.8092813762041) bank183669 +183670 POINT(37.46300333338864 -121.5318775983413) bank183670 +183671 POINT(38.277084594519046 -122.00002240083701) bank183671 +183672 POINT(38.471061710376304 -122.16609213957085) bank183672 +183673 POINT(36.960499554554254 -121.74426734171472) bank183673 +183674 POINT(36.8520363909906 -121.9726597640444) bank183674 +183675 POINT(38.01322380487855 -122.38499033686398) bank183675 +183676 POINT(38.09077468013446 -122.84483096474248) bank183676 +183677 POINT(37.77054360417727 -123.08847928984943) bank183677 +183678 POINT(37.29075053592452 -123.07393982728846) bank183678 +183679 POINT(38.73497735172976 -122.5074820702423) bank183679 +183680 POINT(37.11954577412018 -121.5892781734843) bank183680 +183681 POINT(37.247924989662295 -122.28282616369653) bank183681 +183682 POINT(37.83719424990584 -122.63214880193145) bank183682 +183683 POINT(38.00692529875454 -122.59188170326904) bank183683 +183684 POINT(37.467112003685315 -122.2250381553653) bank183684 +183685 POINT(37.465441153036636 -121.71745974524562) bank183685 +183686 POINT(38.29974027635737 -122.19413053633069) bank183686 +183687 POINT(36.8207344380117 -122.26861542762036) bank183687 +183688 POINT(37.639829163305414 -122.81840522895358) bank183688 +183689 POINT(38.53546092551731 -122.72806011255611) bank183689 +183690 POINT(38.31379316729359 -122.10343701248246) bank183690 +183691 POINT(38.48657535043346 -122.75192301665749) bank183691 +183692 POINT(37.01679060397945 -122.6209545648522) bank183692 +183693 POINT(37.961000971252105 -123.09653175238967) bank183693 +183694 POINT(37.85051776953454 -122.67873953279484) bank183694 +183695 POINT(37.70498379090184 -122.79902266569344) bank183695 +183696 POINT(38.155389265806164 -121.64611233549846) bank183696 +183697 POINT(37.52735489542758 -122.03500757913524) bank183697 +183698 POINT(37.60972650053541 -121.83708121607458) bank183698 +183699 POINT(38.15786793494428 -123.31995794649374) bank183699 +183700 POINT(38.624747248463265 -122.92721182393109) bank183700 +183701 POINT(37.019437475602054 -122.58425285350852) bank183701 +183702 POINT(38.571496691672046 -121.64273656425553) bank183702 +183703 POINT(36.94209065252811 -122.66653119936505) bank183703 +183704 POINT(37.71510899514055 -121.54453401424063) bank183704 +183705 POINT(37.690507957844005 -122.92318555785518) bank183705 +183706 POINT(37.37211230372589 -122.29220353869766) bank183706 +183707 POINT(37.30155343612544 -122.28950205364703) bank183707 +183708 POINT(38.030052283269825 -121.74529213859881) bank183708 +183709 POINT(38.67177945053045 -121.90634984023656) bank183709 +183710 POINT(37.18080571466507 -121.62048023183245) bank183710 +183711 POINT(37.8859176535901 -122.55559819263883) bank183711 +183712 POINT(37.619635634263716 -121.76543488532066) bank183712 +183713 POINT(38.128756998785164 -123.35837241323549) bank183713 +183714 POINT(37.28793665408884 -122.82532533034862) bank183714 +183715 POINT(38.701804075344285 -122.41729887373937) bank183715 +183716 POINT(36.93107792241309 -121.78547436115412) bank183716 +183717 POINT(38.59674029737815 -121.9609772396142) bank183717 +183718 POINT(37.695442247728515 -122.89379523255379) bank183718 +183719 POINT(37.892499869773744 -122.02897489123015) bank183719 +183720 POINT(37.27428049681926 -123.15188713272445) bank183720 +183721 POINT(37.049574595462545 -122.38246164084384) bank183721 +183722 POINT(37.89250741363082 -121.83188890732676) bank183722 +183723 POINT(38.71996653287443 -122.22795225556561) bank183723 +183724 POINT(37.23717797496713 -121.96015137280702) bank183724 +183725 POINT(37.872473980701514 -123.28421811438584) bank183725 +183726 POINT(38.089138976556505 -123.40715286424026) bank183726 +183727 POINT(37.852714045442035 -122.87208718326978) bank183727 +183728 POINT(37.84609123659255 -121.6155452994999) bank183728 +183729 POINT(37.547734058846146 -122.59881720140764) bank183729 +183730 POINT(36.99566505529853 -122.70240378626306) bank183730 +183731 POINT(38.134865380346824 -123.37281999394548) bank183731 +183732 POINT(37.99561994062963 -123.37165847375879) bank183732 +183733 POINT(37.015051723337166 -121.72893881782247) bank183733 +183734 POINT(37.67520752388926 -123.19121563175683) bank183734 +183735 POINT(38.36670109491201 -121.5060136540181) bank183735 +183736 POINT(38.76082825394178 -122.83456200306793) bank183736 +183737 POINT(38.29732821386975 -121.65566956784376) bank183737 +183738 POINT(38.233953968255115 -122.73229807668639) bank183738 +183739 POINT(38.66371010732512 -121.58411279020841) bank183739 +183740 POINT(38.69740958259898 -122.65584506899215) bank183740 +183741 POINT(37.43698851821786 -121.55459263476516) bank183741 +183742 POINT(38.17177258350821 -122.03682626538314) bank183742 +183743 POINT(37.21590310741341 -121.78774522886576) bank183743 +183744 POINT(37.6753631466855 -122.47350071003595) bank183744 +183745 POINT(36.85539924263469 -122.05354785162005) bank183745 +183746 POINT(37.97760083602953 -121.53641773716326) bank183746 +183747 POINT(37.94344703802077 -122.31405809516674) bank183747 +183748 POINT(37.14190548145646 -121.84227887251954) bank183748 +183749 POINT(37.13242003576692 -121.95309082028116) bank183749 +183750 POINT(37.60879666982808 -122.32693226828827) bank183750 +183751 POINT(37.424053116951804 -121.47514388815765) bank183751 +183752 POINT(38.14291488149766 -122.1015684595238) bank183752 +183753 POINT(37.40185551041564 -123.4144557559241) bank183753 +183754 POINT(37.25927150867655 -121.8236348149148) bank183754 +183755 POINT(38.127299049553415 -123.04656599704988) bank183755 +183756 POINT(38.155587083008676 -122.46529517021702) bank183756 +183757 POINT(38.40595743530456 -122.37833087043029) bank183757 +183758 POINT(37.64493186345671 -122.22356031859344) bank183758 +183759 POINT(37.756168417301105 -123.2642315526934) bank183759 +183760 POINT(37.58292804244055 -121.50801500950924) bank183760 +183761 POINT(37.90274384078501 -121.67808332511946) bank183761 +183762 POINT(37.13392742831967 -123.17700962298574) bank183762 +183763 POINT(38.36741033226923 -121.92908535581037) bank183763 +183764 POINT(37.54997326313548 -121.8044955259932) bank183764 +183765 POINT(38.01223592639218 -122.10083303941944) bank183765 +183766 POINT(37.81467102578152 -123.36880665167261) bank183766 +183767 POINT(37.30790775678981 -121.84807732803186) bank183767 +183768 POINT(38.493960270641026 -121.74446884316936) bank183768 +183769 POINT(38.02139397874742 -122.76000084186721) bank183769 +183770 POINT(37.423148318117214 -122.96996503433998) bank183770 +183771 POINT(37.49313642880303 -121.51504027867477) bank183771 +183772 POINT(37.6184596601726 -122.08823160234118) bank183772 +183773 POINT(37.97182747368817 -122.16692047480974) bank183773 +183774 POINT(37.4992241220115 -122.10038674419992) bank183774 +183775 POINT(38.125254707257554 -121.97679514690535) bank183775 +183776 POINT(37.49511289702086 -122.54088135562687) bank183776 +183777 POINT(38.514754975958766 -123.19236447829685) bank183777 +183778 POINT(38.55897600187091 -122.29616316852272) bank183778 +183779 POINT(37.62723755095822 -123.24091559429965) bank183779 +183780 POINT(38.63227199794082 -123.15572938572606) bank183780 +183781 POINT(37.34728587984152 -122.78343269522618) bank183781 +183782 POINT(37.22796284867186 -122.56676131541069) bank183782 +183783 POINT(37.733510804365686 -122.68999744166567) bank183783 +183784 POINT(38.33944619391819 -123.28251124161419) bank183784 +183785 POINT(38.266083959699934 -121.77400082686158) bank183785 +183786 POINT(38.113665218821694 -122.82093248626752) bank183786 +183787 POINT(38.089644226390284 -121.53013122991801) bank183787 +183788 POINT(37.957237451396026 -123.09907309417328) bank183788 +183789 POINT(38.09882170062366 -122.09859957922961) bank183789 +183790 POINT(37.288423638291704 -121.7308405713567) bank183790 +183791 POINT(38.569149199666064 -121.64024773788906) bank183791 +183792 POINT(37.7453219192661 -121.71009216086432) bank183792 +183793 POINT(38.355224810912446 -121.72913445123265) bank183793 +183794 POINT(38.559639892633385 -122.1467219925009) bank183794 +183795 POINT(36.840882339470014 -122.61332247930504) bank183795 +183796 POINT(37.24117791855982 -122.19770315867308) bank183796 +183797 POINT(37.880938659143006 -122.5164994166709) bank183797 +183798 POINT(37.785050271677214 -122.19851793650822) bank183798 +183799 POINT(38.535904370760875 -122.55070488105835) bank183799 +183800 POINT(37.97469617169536 -122.72394594994421) bank183800 +183801 POINT(37.34689263265392 -122.81441683800618) bank183801 +183802 POINT(36.909622776400454 -121.49052694100259) bank183802 +183803 POINT(37.332224121735706 -122.77760961819787) bank183803 +183804 POINT(37.33824121052872 -122.68231145820484) bank183804 +183805 POINT(38.2444505036276 -121.45643518334741) bank183805 +183806 POINT(38.090049833210635 -121.98049242633144) bank183806 +183807 POINT(38.39013447129038 -121.4222486489261) bank183807 +183808 POINT(37.25559872567244 -122.8503398618771) bank183808 +183809 POINT(37.38869304432253 -123.36145127440192) bank183809 +183810 POINT(38.35570764033092 -121.90174906518615) bank183810 +183811 POINT(37.04537736393096 -123.28396024478948) bank183811 +183812 POINT(36.9017164807386 -122.48492839274317) bank183812 +183813 POINT(37.15078690571249 -121.4768206880154) bank183813 +183814 POINT(38.188929309761114 -121.44878345572666) bank183814 +183815 POINT(38.281332247115806 -122.28300951176274) bank183815 +183816 POINT(37.04132759004937 -122.1991873100143) bank183816 +183817 POINT(38.310576208252016 -123.0651759346111) bank183817 +183818 POINT(37.33642791200794 -123.1288993912957) bank183818 +183819 POINT(37.17625409607357 -122.41908749698803) bank183819 +183820 POINT(38.363000011117684 -121.44500275820667) bank183820 +183821 POINT(37.810634112918315 -122.44338490945489) bank183821 +183822 POINT(37.39497921205596 -121.85592732343272) bank183822 +183823 POINT(38.24039465876467 -123.13527096122708) bank183823 +183824 POINT(37.809110805031395 -121.73301718634723) bank183824 +183825 POINT(37.995030862006494 -123.4091258613815) bank183825 +183826 POINT(38.14529089898633 -122.49843826791113) bank183826 +183827 POINT(37.90376873642071 -122.48579044970813) bank183827 +183828 POINT(37.09079646885596 -122.77870478910705) bank183828 +183829 POINT(38.26441164226018 -123.15299865464797) bank183829 +183830 POINT(36.94849231433182 -122.38393315491489) bank183830 +183831 POINT(37.46763962023945 -122.69449976884835) bank183831 +183832 POINT(37.14244465486704 -122.79164943555779) bank183832 +183833 POINT(38.29983146281988 -122.51252089106687) bank183833 +183834 POINT(37.150306487713166 -122.01085835523111) bank183834 +183835 POINT(36.993541994330876 -122.09325603823306) bank183835 +183836 POINT(36.88867755345779 -121.77155381958477) bank183836 +183837 POINT(38.15829287245035 -121.93414559344318) bank183837 +183838 POINT(37.0418759925634 -122.47390165998543) bank183838 +183839 POINT(38.23757078547256 -121.46095431819285) bank183839 +183840 POINT(38.63538056849409 -122.60660110296149) bank183840 +183841 POINT(38.32474750354546 -121.47126396859933) bank183841 +183842 POINT(38.41113324099413 -123.23151355572452) bank183842 +183843 POINT(37.01876619462415 -122.16364581236198) bank183843 +183844 POINT(38.154975927024104 -122.56434138764519) bank183844 +183845 POINT(38.101311451161855 -123.0209744634702) bank183845 +183846 POINT(37.88244735346241 -123.29267812209596) bank183846 +183847 POINT(37.16922214892417 -123.3467914019004) bank183847 +183848 POINT(38.10808822088245 -122.23336813855389) bank183848 +183849 POINT(37.416497088242444 -122.38789489443108) bank183849 +183850 POINT(38.52222575563913 -121.49575530889514) bank183850 +183851 POINT(37.038032275752684 -121.49604050394271) bank183851 +183852 POINT(37.95979905068129 -122.92022308195995) bank183852 +183853 POINT(38.248196947868436 -122.05732054441562) bank183853 +183854 POINT(38.66891104027381 -121.95174001084669) bank183854 +183855 POINT(37.903875734650136 -122.90452956000117) bank183855 +183856 POINT(38.104446268791875 -122.93855805964209) bank183856 +183857 POINT(37.60371883271636 -121.51650642327547) bank183857 +183858 POINT(38.762479266207215 -122.3654416766891) bank183858 +183859 POINT(37.65805536374563 -122.4725932921786) bank183859 +183860 POINT(37.187481894585204 -121.54405310734467) bank183860 +183861 POINT(38.34735290754452 -121.88770388596541) bank183861 +183862 POINT(38.57277382255469 -121.49183566967142) bank183862 +183863 POINT(37.03837995510724 -122.82341365148152) bank183863 +183864 POINT(38.488144357848114 -121.54072559235125) bank183864 +183865 POINT(38.67266393097462 -122.68578305444656) bank183865 +183866 POINT(37.45459551653434 -121.65863792186812) bank183866 +183867 POINT(37.94466793689661 -122.2508996383784) bank183867 +183868 POINT(37.826635680405204 -122.38469869050373) bank183868 +183869 POINT(37.39691863005535 -122.79568863045691) bank183869 +183870 POINT(38.38769062884134 -121.42769273810855) bank183870 +183871 POINT(37.504659346258094 -122.18267103146803) bank183871 +183872 POINT(38.50504607893186 -122.90193205311985) bank183872 +183873 POINT(38.53621411678606 -121.99641313739257) bank183873 +183874 POINT(38.653983673513906 -122.8356267309784) bank183874 +183875 POINT(37.36435672684263 -122.01263942414326) bank183875 +183876 POINT(36.84003047977224 -123.23872903014303) bank183876 +183877 POINT(38.19142220325801 -121.68172943196016) bank183877 +183878 POINT(36.83474750734597 -123.39342485995752) bank183878 +183879 POINT(38.07344044548541 -123.27426063289877) bank183879 +183880 POINT(38.27956809306957 -121.60905310702711) bank183880 +183881 POINT(38.48504958900513 -121.49945216952553) bank183881 +183882 POINT(37.09928914242112 -122.24383482996029) bank183882 +183883 POINT(38.2381743764609 -121.62389668671908) bank183883 +183884 POINT(38.68223799854044 -121.89025403416598) bank183884 +183885 POINT(37.97681094175737 -122.16833282805068) bank183885 +183886 POINT(38.231998316267315 -121.59968270228829) bank183886 +183887 POINT(38.06266205292955 -121.7389257069936) bank183887 +183888 POINT(37.48950470285461 -122.89672201119208) bank183888 +183889 POINT(37.3392393268778 -122.57429098552609) bank183889 +183890 POINT(37.503758698160574 -122.09314827252389) bank183890 +183891 POINT(37.557497792835164 -122.28327206551222) bank183891 +183892 POINT(36.92233099397513 -122.95105345502625) bank183892 +183893 POINT(38.272378463405076 -121.56412316629448) bank183893 +183894 POINT(38.69016043329344 -121.76289405736185) bank183894 +183895 POINT(37.98134695917658 -123.07747311621476) bank183895 +183896 POINT(37.38378583351129 -121.80140531666363) bank183896 +183897 POINT(38.557931606742926 -123.16704576259058) bank183897 +183898 POINT(37.60533618600501 -123.32265848464941) bank183898 +183899 POINT(37.2808137100005 -121.84508259707033) bank183899 +183900 POINT(37.00362303920943 -123.13270332430362) bank183900 +183901 POINT(37.513544412581844 -123.27123703972997) bank183901 +183902 POINT(38.44474770397645 -122.8353031711607) bank183902 +183903 POINT(38.01242893085711 -121.6520834331643) bank183903 +183904 POINT(38.14197724079373 -123.2831024813337) bank183904 +183905 POINT(37.736414446448535 -122.8694628756079) bank183905 +183906 POINT(37.37272414764705 -122.25453820051312) bank183906 +183907 POINT(38.654044333320336 -122.26296524656057) bank183907 +183908 POINT(38.22402824193989 -121.94258584286698) bank183908 +183909 POINT(37.73123051055386 -121.88403293697465) bank183909 +183910 POINT(37.711652574802066 -122.96208887180899) bank183910 +183911 POINT(37.78753449151875 -122.3740470044801) bank183911 +183912 POINT(38.375086098414975 -122.71792293550381) bank183912 +183913 POINT(38.498534488862056 -121.59354827929458) bank183913 +183914 POINT(38.552204222621164 -122.13337295442935) bank183914 +183915 POINT(37.0315985613539 -122.9610663085268) bank183915 +183916 POINT(36.985381314798055 -122.29347945230269) bank183916 +183917 POINT(37.44636405644445 -121.98005768111123) bank183917 +183918 POINT(37.80671970856669 -122.46852278036907) bank183918 +183919 POINT(36.866767288376295 -122.57393250994187) bank183919 +183920 POINT(37.841676037918916 -122.1145306540697) bank183920 +183921 POINT(37.48391691530561 -123.27532310002499) bank183921 +183922 POINT(38.26935583052873 -122.27969909620094) bank183922 +183923 POINT(37.04880277783287 -122.15707884021693) bank183923 +183924 POINT(37.68226434204013 -122.76660887744968) bank183924 +183925 POINT(37.06084721529868 -123.0362887862009) bank183925 +183926 POINT(38.16905011317119 -123.01741567164886) bank183926 +183927 POINT(36.93133352812252 -122.9907509116318) bank183927 +183928 POINT(37.11211890744676 -121.89860980164542) bank183928 +183929 POINT(38.593893933450104 -123.3399500145004) bank183929 +183930 POINT(36.93182007559449 -123.06615813992596) bank183930 +183931 POINT(38.73421384114411 -122.29742375575674) bank183931 +183932 POINT(38.20988831841927 -122.72034254697697) bank183932 +183933 POINT(38.53850929994814 -121.57141901685132) bank183933 +183934 POINT(37.09753465066627 -123.19644047836537) bank183934 +183935 POINT(38.706135621074985 -121.5120057558037) bank183935 +183936 POINT(37.96946312787462 -122.0542949374696) bank183936 +183937 POINT(38.47161028135395 -123.19731879072897) bank183937 +183938 POINT(37.65361576782998 -122.83732988203414) bank183938 +183939 POINT(37.10286953278172 -121.89298521723022) bank183939 +183940 POINT(38.540179320810395 -121.785438427348) bank183940 +183941 POINT(38.65484133645498 -121.84207421165631) bank183941 +183942 POINT(37.76048840446742 -122.2138191463906) bank183942 +183943 POINT(37.85433872515039 -121.59555891294576) bank183943 +183944 POINT(37.92765392756528 -121.67035290228907) bank183944 +183945 POINT(37.38475134303289 -122.75680822235772) bank183945 +183946 POINT(37.69432096692622 -123.09991443338006) bank183946 +183947 POINT(38.31786544149836 -122.02788134476381) bank183947 +183948 POINT(37.59663764128098 -122.91597747011056) bank183948 +183949 POINT(37.43943035664432 -123.09667451471098) bank183949 +183950 POINT(38.60603096400437 -122.07615845157449) bank183950 +183951 POINT(37.761130126122154 -123.03128810455394) bank183951 +183952 POINT(38.429435068604505 -122.97414526007421) bank183952 +183953 POINT(38.13765737367262 -121.87945603133699) bank183953 +183954 POINT(38.65065921565239 -123.24774312863596) bank183954 +183955 POINT(38.51049190300854 -122.54305516369692) bank183955 +183956 POINT(36.83688800638294 -123.35390267667483) bank183956 +183957 POINT(37.691779193332074 -123.185328135004) bank183957 +183958 POINT(38.75223915354511 -122.50434516510957) bank183958 +183959 POINT(37.859649098491786 -122.12806343136488) bank183959 +183960 POINT(38.15471157881942 -121.50435195518492) bank183960 +183961 POINT(37.92157284882714 -122.85850143016538) bank183961 +183962 POINT(37.99006534849728 -122.81949954744908) bank183962 +183963 POINT(38.40686172320535 -122.7238499476421) bank183963 +183964 POINT(37.13526731378558 -121.89923798213583) bank183964 +183965 POINT(37.70967679600816 -121.88513619564091) bank183965 +183966 POINT(37.630204776235246 -123.10223389319589) bank183966 +183967 POINT(38.07644043739859 -122.24848339831588) bank183967 +183968 POINT(36.93403956516419 -123.22744419255832) bank183968 +183969 POINT(38.1441717437723 -122.66496498851886) bank183969 +183970 POINT(36.930257476718126 -122.52188071743569) bank183970 +183971 POINT(37.98053356595264 -123.33007480853158) bank183971 +183972 POINT(38.24168119565916 -121.73532130819295) bank183972 +183973 POINT(38.49671777019273 -122.1677961153863) bank183973 +183974 POINT(38.74406098232151 -121.89285200416965) bank183974 +183975 POINT(38.70579869121534 -122.98372507108687) bank183975 +183976 POINT(37.62689413012757 -122.58245059088917) bank183976 +183977 POINT(37.58063000139222 -121.90189739291152) bank183977 +183978 POINT(37.05480632899074 -121.55516369536625) bank183978 +183979 POINT(38.066193296837305 -122.72153226711902) bank183979 +183980 POINT(38.68646101995615 -121.72149551891196) bank183980 +183981 POINT(36.98102605381083 -122.01616387841023) bank183981 +183982 POINT(37.6777066369695 -122.12903082984182) bank183982 +183983 POINT(37.830318821430765 -123.23976945284518) bank183983 +183984 POINT(38.5714851877286 -121.53402023159403) bank183984 +183985 POINT(37.032609519152736 -121.97707720551507) bank183985 +183986 POINT(38.14470993635087 -121.61984925947613) bank183986 +183987 POINT(38.43540851410636 -123.28081109266745) bank183987 +183988 POINT(36.955779482440455 -123.41197585518826) bank183988 +183989 POINT(37.66660008019322 -122.26695344200743) bank183989 +183990 POINT(37.1799267412115 -122.89677064905521) bank183990 +183991 POINT(37.36150577557295 -122.41551023819065) bank183991 +183992 POINT(38.69509916165675 -122.71764321103677) bank183992 +183993 POINT(38.471433862286766 -122.69735036786443) bank183993 +183994 POINT(37.06933006508997 -122.0084823552113) bank183994 +183995 POINT(36.78854729395795 -123.36599942133564) bank183995 +183996 POINT(38.37766037370655 -121.63040015295451) bank183996 +183997 POINT(38.186890874557356 -121.56806433877288) bank183997 +183998 POINT(38.694545831307934 -122.02576566505408) bank183998 +183999 POINT(36.83673489096307 -121.94131830740105) bank183999 +184000 POINT(37.65450949054469 -123.14965242539455) bank184000 +184001 POINT(38.65845692022909 -122.39473373652278) bank184001 +184002 POINT(38.73817168473253 -123.32095568948252) bank184002 +184003 POINT(37.75346620109977 -122.81366581557391) bank184003 +184004 POINT(37.62028696410775 -121.69973330341426) bank184004 +184005 POINT(37.3911622164334 -122.56976012266777) bank184005 +184006 POINT(37.74743319750666 -121.80527249909485) bank184006 +184007 POINT(38.0205690329419 -122.25175434958948) bank184007 +184008 POINT(38.40575910066547 -122.2196925952846) bank184008 +184009 POINT(37.77282104689712 -123.05057732148262) bank184009 +184010 POINT(37.52159133527547 -121.55361989238534) bank184010 +184011 POINT(37.86842534558604 -121.89563520505263) bank184011 +184012 POINT(37.3347553365709 -121.72150332788402) bank184012 +184013 POINT(38.222842023316474 -123.00041265599933) bank184013 +184014 POINT(38.212938789997374 -121.65874502515094) bank184014 +184015 POINT(37.89258991366811 -122.86622168252956) bank184015 +184016 POINT(38.489354036204745 -122.15548475283734) bank184016 +184017 POINT(38.436475964978584 -122.1254408071709) bank184017 +184018 POINT(37.97539738997756 -122.15529023863853) bank184018 +184019 POINT(38.76775651448276 -123.20874496142834) bank184019 +184020 POINT(38.04201172231519 -122.10562868073208) bank184020 +184021 POINT(37.624495933920585 -122.80965866849375) bank184021 +184022 POINT(37.214300766808456 -123.0044387506947) bank184022 +184023 POINT(37.293172765670356 -122.86509895430008) bank184023 +184024 POINT(37.50148409327507 -123.04779691215802) bank184024 +184025 POINT(38.47074228080642 -122.61933330081905) bank184025 +184026 POINT(37.708409524336105 -121.75112115005629) bank184026 +184027 POINT(38.170625879066485 -121.76045451779247) bank184027 +184028 POINT(37.95919099561137 -122.8998465898434) bank184028 +184029 POINT(38.10883686564807 -121.85161546624805) bank184029 +184030 POINT(37.81859181816664 -123.30094057677653) bank184030 +184031 POINT(37.02937020110859 -122.78807053321604) bank184031 +184032 POINT(38.54128909129836 -122.37036322256283) bank184032 +184033 POINT(37.7387313750786 -121.8628227355624) bank184033 +184034 POINT(37.1348287820483 -121.89970026121821) bank184034 +184035 POINT(37.48621513664041 -122.6691175044358) bank184035 +184036 POINT(38.4222629929179 -122.1184245312702) bank184036 +184037 POINT(38.0390778692336 -122.07818117701137) bank184037 +184038 POINT(38.31799383242762 -123.32628158866169) bank184038 +184039 POINT(38.60555437975789 -123.32536729538182) bank184039 +184040 POINT(37.046632272177014 -121.95831092891378) bank184040 +184041 POINT(37.17185810449242 -122.98420629386203) bank184041 +184042 POINT(37.65499107604251 -122.86970472821227) bank184042 +184043 POINT(38.11409075025577 -122.04193942596763) bank184043 +184044 POINT(38.06699741337537 -123.41596863806528) bank184044 +184045 POINT(37.38796026816959 -123.242928793938) bank184045 +184046 POINT(38.7140422261084 -122.59697119511895) bank184046 +184047 POINT(38.12125443502511 -122.42536688097616) bank184047 +184048 POINT(37.53159342331722 -121.83047659322543) bank184048 +184049 POINT(37.39439765551603 -122.66908430849111) bank184049 +184050 POINT(37.31516051790856 -122.54208612714898) bank184050 +184051 POINT(37.19224486347782 -122.06887424715055) bank184051 +184052 POINT(38.44218114615711 -123.05344288608842) bank184052 +184053 POINT(38.64832939212697 -123.1331409974001) bank184053 +184054 POINT(37.60249428079243 -122.63327016299633) bank184054 +184055 POINT(37.58142498218653 -122.61954633821426) bank184055 +184056 POINT(37.21775748028652 -122.58913654931382) bank184056 +184057 POINT(38.49200511366257 -121.77367992453028) bank184057 +184058 POINT(37.4305424614707 -121.56762014894026) bank184058 +184059 POINT(38.478890783898485 -121.52628364124269) bank184059 +184060 POINT(37.63330588580519 -122.73278351082617) bank184060 +184061 POINT(37.867256623521804 -123.26588242960837) bank184061 +184062 POINT(37.29911659839499 -121.66518210692479) bank184062 +184063 POINT(37.927685950698574 -122.59164041681389) bank184063 +184064 POINT(38.770655288920985 -122.13866144980061) bank184064 +184065 POINT(38.38876370679003 -121.8408882951179) bank184065 +184066 POINT(37.97872667594607 -122.44714061114287) bank184066 +184067 POINT(37.2879388773542 -123.18731174044211) bank184067 +184068 POINT(38.44912218173933 -122.9496316508216) bank184068 +184069 POINT(36.91372941911169 -122.83514939422352) bank184069 +184070 POINT(37.59829249310788 -121.77538342734272) bank184070 +184071 POINT(38.409911366534836 -122.18953658900737) bank184071 +184072 POINT(37.58496834710447 -122.8636613298724) bank184072 +184073 POINT(36.781131398729606 -122.66930168338487) bank184073 +184074 POINT(38.50036055477271 -122.95399728766236) bank184074 +184075 POINT(38.09945502542517 -122.6348067003987) bank184075 +184076 POINT(37.30388024454644 -122.93909237509065) bank184076 +184077 POINT(37.36463456861492 -122.92012367900357) bank184077 +184078 POINT(38.46763703358154 -123.05744185160042) bank184078 +184079 POINT(37.159840355957776 -122.36663929309898) bank184079 +184080 POINT(38.02191727033316 -121.44548079591421) bank184080 +184081 POINT(36.924588524486055 -121.51549569872381) bank184081 +184082 POINT(38.01913979026539 -122.00740502587949) bank184082 +184083 POINT(37.25128138273832 -123.38042929456455) bank184083 +184084 POINT(37.38857201999037 -122.20594150951004) bank184084 +184085 POINT(37.40523510619991 -122.24941546554366) bank184085 +184086 POINT(37.84659162295251 -121.71940901058552) bank184086 +184087 POINT(37.701890602095574 -121.81257124793339) bank184087 +184088 POINT(37.504866017632025 -121.53043540534296) bank184088 +184089 POINT(38.75794811595807 -122.62693950671553) bank184089 +184090 POINT(37.58124732487677 -121.8177447176911) bank184090 +184091 POINT(38.20820989928798 -121.62022756405065) bank184091 +184092 POINT(38.64325907068264 -121.61356981972366) bank184092 +184093 POINT(36.985459888711354 -122.151274564365) bank184093 +184094 POINT(37.61131223900997 -122.00329987558365) bank184094 +184095 POINT(37.02572559140377 -121.68345032013475) bank184095 +184096 POINT(38.010059628907925 -122.48772058257487) bank184096 +184097 POINT(37.888608543470355 -122.78713514683639) bank184097 +184098 POINT(38.531289971146265 -122.30619094429184) bank184098 +184099 POINT(38.570767530540124 -122.94998725684509) bank184099 +184100 POINT(38.52895711850255 -122.58113073707241) bank184100 +184101 POINT(38.32388207777342 -123.10939917273492) bank184101 +184102 POINT(36.845584825308784 -122.0311118718022) bank184102 +184103 POINT(38.502882840390704 -121.42927368070829) bank184103 +184104 POINT(38.27069693425524 -123.27760233246211) bank184104 +184105 POINT(37.50458904741874 -122.7168345507407) bank184105 +184106 POINT(37.62189833396917 -122.01720635096646) bank184106 +184107 POINT(37.421421348522074 -122.25929744212446) bank184107 +184108 POINT(36.97219318559927 -122.2178485751206) bank184108 +184109 POINT(36.93514324674332 -122.69629930196902) bank184109 +184110 POINT(38.03650634638946 -121.62812862815188) bank184110 +184111 POINT(37.98033246277487 -122.56266845681932) bank184111 +184112 POINT(38.5319263197124 -123.26278459855621) bank184112 +184113 POINT(37.041857856605844 -122.02890233424829) bank184113 +184114 POINT(37.30595289182122 -122.25465945634724) bank184114 +184115 POINT(37.49917237447418 -122.56648925860817) bank184115 +184116 POINT(37.88477124340812 -123.28163069181588) bank184116 +184117 POINT(36.85075847918192 -122.94647473816286) bank184117 +184118 POINT(37.392125003198544 -122.00260547160796) bank184118 +184119 POINT(37.66975987258564 -122.73708735824071) bank184119 +184120 POINT(36.87342587513956 -121.44593741697527) bank184120 +184121 POINT(38.2711953941045 -123.02120985550502) bank184121 +184122 POINT(37.480777449561735 -122.9279786168522) bank184122 +184123 POINT(37.86856483671619 -123.05527967458399) bank184123 +184124 POINT(38.202338629002874 -121.97297574229493) bank184124 +184125 POINT(37.68077083016214 -122.26824575537064) bank184125 +184126 POINT(38.67915753561429 -122.70951767075476) bank184126 +184127 POINT(36.81912468906762 -121.73882958709657) bank184127 +184128 POINT(37.81484755235002 -121.455679201756) bank184128 +184129 POINT(38.032253456185494 -122.32209676961672) bank184129 +184130 POINT(38.26934450487769 -123.25634365832357) bank184130 +184131 POINT(38.19013550852936 -121.95836176972502) bank184131 +184132 POINT(38.75467467505529 -121.97622953601068) bank184132 +184133 POINT(37.115167502776735 -122.1217896052068) bank184133 +184134 POINT(38.24891712580058 -121.94538898975043) bank184134 +184135 POINT(38.17469986012476 -122.58567170727197) bank184135 +184136 POINT(38.07235044188754 -122.24204241098018) bank184136 +184137 POINT(38.28814540345657 -121.77388811749519) bank184137 +184138 POINT(38.462150937364626 -121.74554940519907) bank184138 +184139 POINT(37.38740240863551 -122.3163474376922) bank184139 +184140 POINT(38.76993726761971 -123.20432665307857) bank184140 +184141 POINT(37.6804526987339 -122.76148276616564) bank184141 +184142 POINT(38.19715994603795 -121.49262667432411) bank184142 +184143 POINT(36.99088732542905 -122.9411666909291) bank184143 +184144 POINT(36.781251488691254 -123.03296351848508) bank184144 +184145 POINT(37.03169172657732 -122.34430917646121) bank184145 +184146 POINT(37.10822644579825 -123.01893732046305) bank184146 +184147 POINT(38.04844366332706 -123.3120112834382) bank184147 +184148 POINT(37.1465077723382 -122.64377018830606) bank184148 +184149 POINT(37.44965859312242 -122.93219039260366) bank184149 +184150 POINT(38.437864294327746 -122.73836160058629) bank184150 +184151 POINT(38.21701688383174 -123.11099272324195) bank184151 +184152 POINT(37.902105572887976 -122.87604761475141) bank184152 +184153 POINT(37.47379008817535 -122.84253117928327) bank184153 +184154 POINT(36.86114995156594 -123.24350911373311) bank184154 +184155 POINT(38.76409606053859 -122.57954794984425) bank184155 +184156 POINT(38.41459116770176 -122.49824864253512) bank184156 +184157 POINT(37.49926265117785 -122.85946265994518) bank184157 +184158 POINT(36.793369233473 -121.69757755870056) bank184158 +184159 POINT(38.595339445231915 -122.26121491642773) bank184159 +184160 POINT(37.99662796351146 -121.69025221632707) bank184160 +184161 POINT(36.94797266913285 -123.31277621575158) bank184161 +184162 POINT(37.946977844710176 -123.08945466118995) bank184162 +184163 POINT(37.06376332960673 -121.71535813313098) bank184163 +184164 POINT(38.64327567431046 -122.71624680788666) bank184164 +184165 POINT(38.421113841911534 -122.86796469928743) bank184165 +184166 POINT(37.17319080327027 -122.60562887896374) bank184166 +184167 POINT(37.42665600111017 -123.20575510723634) bank184167 +184168 POINT(37.754983541276964 -123.22145352011485) bank184168 +184169 POINT(37.81760826410874 -123.24728305782199) bank184169 +184170 POINT(37.98490316100944 -121.4428968436484) bank184170 +184171 POINT(37.05897282887159 -123.05013542146264) bank184171 +184172 POINT(37.83816631442341 -122.79653135632341) bank184172 +184173 POINT(38.50855287433174 -122.50991501536583) bank184173 +184174 POINT(37.744803432221985 -121.7160085652978) bank184174 +184175 POINT(38.31922079757724 -121.53764022622677) bank184175 +184176 POINT(36.96642296072724 -123.31363727977981) bank184176 +184177 POINT(38.03961578753113 -123.18745642517064) bank184177 +184178 POINT(37.15468952096923 -123.24937649110983) bank184178 +184179 POINT(38.22720060694737 -122.54937936973344) bank184179 +184180 POINT(37.53085511659875 -122.47039185135968) bank184180 +184181 POINT(36.96825908686952 -121.9293854470609) bank184181 +184182 POINT(38.68450422576016 -122.4131920297163) bank184182 +184183 POINT(37.39905957362038 -123.08308907643213) bank184183 +184184 POINT(38.189012637453665 -122.04698031873629) bank184184 +184185 POINT(37.528913741328246 -123.15753736785341) bank184185 +184186 POINT(37.81692414597069 -122.96060228928674) bank184186 +184187 POINT(36.78343171804875 -123.08194097840891) bank184187 +184188 POINT(38.512792088473326 -123.1727638055903) bank184188 +184189 POINT(38.25098263036102 -122.87839044239468) bank184189 +184190 POINT(38.690216439415536 -122.18074064577378) bank184190 +184191 POINT(38.03331004283838 -122.146102012784) bank184191 +184192 POINT(37.58696343976567 -121.87521546140857) bank184192 +184193 POINT(37.57161589948154 -123.17120703675668) bank184193 +184194 POINT(37.90750241531222 -121.94261677738399) bank184194 +184195 POINT(37.93609268454145 -123.07567493199512) bank184195 +184196 POINT(37.230205758205436 -122.4373435519474) bank184196 +184197 POINT(36.96433211879665 -122.83970759916265) bank184197 +184198 POINT(37.09863042494505 -121.4434176006541) bank184198 +184199 POINT(37.88874452731268 -122.53211380940856) bank184199 +184200 POINT(38.11465555017339 -123.35219281040946) bank184200 +184201 POINT(37.37430643182554 -123.28074886754254) bank184201 +184202 POINT(37.402601441531395 -121.7585921426103) bank184202 +184203 POINT(37.01391178049955 -122.39696703587535) bank184203 +184204 POINT(36.828878613689774 -122.48368321375126) bank184204 +184205 POINT(37.381697869903 -122.54021624601117) bank184205 +184206 POINT(38.20093782897751 -122.16599134061505) bank184206 +184207 POINT(37.60671315480182 -123.10348531335872) bank184207 +184208 POINT(38.205053258312745 -123.13618770117671) bank184208 +184209 POINT(36.95253432731722 -123.02821379171391) bank184209 +184210 POINT(38.031802808567754 -122.25873275140275) bank184210 +184211 POINT(37.64558478174092 -121.5674474676033) bank184211 +184212 POINT(38.271085108241344 -122.80676224107422) bank184212 +184213 POINT(38.54773003121871 -122.425742924427) bank184213 +184214 POINT(37.92720323753654 -121.45662203455527) bank184214 +184215 POINT(37.17303568814243 -121.8307350713366) bank184215 +184216 POINT(38.32158140746148 -121.42372004430288) bank184216 +184217 POINT(38.59381521407612 -121.438238728267) bank184217 +184218 POINT(37.901266921508004 -123.07980430246111) bank184218 +184219 POINT(36.946407013300366 -122.63023152649663) bank184219 +184220 POINT(37.32486416265062 -121.99952189669514) bank184220 +184221 POINT(38.10819236644063 -121.79040466121307) bank184221 +184222 POINT(37.66978300012461 -122.20107991625497) bank184222 +184223 POINT(37.29637199756128 -122.08930062939407) bank184223 +184224 POINT(37.10682696237263 -122.48725617889875) bank184224 +184225 POINT(36.93341730335265 -122.80786059019579) bank184225 +184226 POINT(36.83209252540016 -123.02209542351235) bank184226 +184227 POINT(37.64482137217166 -121.96812470305181) bank184227 +184228 POINT(37.17788805486329 -121.62509892683084) bank184228 +184229 POINT(37.28502075876257 -121.64098914710554) bank184229 +184230 POINT(37.21752464757115 -121.85208405212038) bank184230 +184231 POINT(38.02182476639251 -121.4701124927519) bank184231 +184232 POINT(38.74161190605064 -121.46770493290495) bank184232 +184233 POINT(37.49870109504269 -121.8429575212948) bank184233 +184234 POINT(37.10388472295165 -122.2871269113449) bank184234 +184235 POINT(38.092946462551765 -121.91005424945301) bank184235 +184236 POINT(37.65833236722194 -123.17130263385218) bank184236 +184237 POINT(37.1525508680886 -121.70761067095296) bank184237 +184238 POINT(37.536835307317666 -122.2052376645191) bank184238 +184239 POINT(37.253165018299484 -123.03339166343206) bank184239 +184240 POINT(38.76968482343075 -122.15584411862578) bank184240 +184241 POINT(36.88840413569916 -122.81570104888263) bank184241 +184242 POINT(37.96039072511247 -121.718243236727) bank184242 +184243 POINT(37.22798514929702 -121.50139120223068) bank184243 +184244 POINT(37.939459953385956 -122.68377425271471) bank184244 +184245 POINT(38.43928133578886 -122.90072480151697) bank184245 +184246 POINT(37.61358496203858 -121.55130486829289) bank184246 +184247 POINT(37.83183822952509 -122.70140147288986) bank184247 +184248 POINT(37.04488802894559 -122.95171191519633) bank184248 +184249 POINT(37.43704040092214 -123.21187104782577) bank184249 +184250 POINT(37.79314127139026 -122.89461104348634) bank184250 +184251 POINT(38.29561486802126 -123.18155941735311) bank184251 +184252 POINT(37.22676472270473 -123.41441682501369) bank184252 +184253 POINT(37.69510508286105 -121.93207121444789) bank184253 +184254 POINT(38.15648302163766 -123.13894468452732) bank184254 +184255 POINT(38.59895542373175 -121.44402592162935) bank184255 +184256 POINT(36.94320805043572 -122.46710357611703) bank184256 +184257 POINT(38.55456045123408 -121.85520771815553) bank184257 +184258 POINT(37.57596977282346 -121.87571918246688) bank184258 +184259 POINT(38.75283615212173 -122.85302385456028) bank184259 +184260 POINT(36.84978614128048 -123.30357808131731) bank184260 +184261 POINT(36.924045182940056 -123.06756666445183) bank184261 +184262 POINT(38.38538659610872 -122.20388538366662) bank184262 +184263 POINT(37.01432854521732 -123.16878199622603) bank184263 +184264 POINT(38.45067779431203 -122.21694037102883) bank184264 +184265 POINT(37.18300519303395 -122.59961390071247) bank184265 +184266 POINT(36.81702896115124 -122.1173159330069) bank184266 +184267 POINT(38.26031256090103 -123.1265010978347) bank184267 +184268 POINT(38.672604068884375 -122.14443856294628) bank184268 +184269 POINT(37.6003803132449 -121.85058440195671) bank184269 +184270 POINT(38.16834161526923 -121.43237706532277) bank184270 +184271 POINT(37.476307940290305 -121.48472653052075) bank184271 +184272 POINT(37.51473858899158 -122.49668265020246) bank184272 +184273 POINT(37.41428096146548 -122.07729304050828) bank184273 +184274 POINT(38.139648516337054 -123.05433405247292) bank184274 +184275 POINT(37.75458271355313 -122.80804912033834) bank184275 +184276 POINT(37.99033538694595 -122.63768404282806) bank184276 +184277 POINT(37.07029080962919 -123.20363311669391) bank184277 +184278 POINT(36.884117392669125 -122.48762346819255) bank184278 +184279 POINT(37.02763312920223 -122.74070271830338) bank184279 +184280 POINT(37.74801065238494 -121.8340242706906) bank184280 +184281 POINT(37.382410664320375 -122.32991094300004) bank184281 +184282 POINT(36.88456412236721 -123.00355135490182) bank184282 +184283 POINT(37.388874343657626 -122.30437321297032) bank184283 +184284 POINT(38.20488943182065 -123.20325579288513) bank184284 +184285 POINT(38.61142532955941 -122.45836577627432) bank184285 +184286 POINT(37.58554854659499 -122.23471703998806) bank184286 +184287 POINT(37.213479414842325 -122.88797117552632) bank184287 +184288 POINT(37.732566656782126 -121.75735873826126) bank184288 +184289 POINT(37.87401046109476 -122.59261346223573) bank184289 +184290 POINT(38.76789108178496 -121.43169433484938) bank184290 +184291 POINT(37.218256778956935 -121.48467704446198) bank184291 +184292 POINT(36.884203089754614 -121.98590782932145) bank184292 +184293 POINT(37.78546493275895 -121.7780246636693) bank184293 +184294 POINT(37.565561143197044 -121.8848415635663) bank184294 +184295 POINT(37.07321533673427 -123.38645849555913) bank184295 +184296 POINT(37.366957926842474 -122.26982902580485) bank184296 +184297 POINT(38.73857178281968 -121.58522985286008) bank184297 +184298 POINT(37.84102488178052 -121.5433141259848) bank184298 +184299 POINT(36.956965176102266 -123.32220291634341) bank184299 +184300 POINT(36.77898611983922 -123.2940788249256) bank184300 +184301 POINT(37.09792973707825 -122.69261662879816) bank184301 +184302 POINT(37.07603770510956 -122.02356631947406) bank184302 +184303 POINT(38.20322927244533 -123.20514656667852) bank184303 +184304 POINT(38.27438874158723 -122.75648433637551) bank184304 +184305 POINT(38.54299514949375 -123.17573050579492) bank184305 +184306 POINT(38.15669497624282 -123.00860204086153) bank184306 +184307 POINT(37.02042659929971 -122.09217962665402) bank184307 +184308 POINT(37.77000598670885 -121.86574476878627) bank184308 +184309 POINT(36.7882058146564 -121.49805371543674) bank184309 +184310 POINT(38.02402350570962 -122.05221329840936) bank184310 +184311 POINT(38.735206228190656 -123.23651052320245) bank184311 +184312 POINT(38.25979085357897 -121.73089502926645) bank184312 +184313 POINT(36.992042924404416 -122.86223353996436) bank184313 +184314 POINT(38.39009400631441 -123.17984810160638) bank184314 +184315 POINT(38.23689227630324 -121.54402321562321) bank184315 +184316 POINT(37.87572536193752 -123.18940814509693) bank184316 +184317 POINT(37.80368479925252 -122.00166405299836) bank184317 +184318 POINT(37.710688380203614 -122.35458335277056) bank184318 +184319 POINT(37.20564275997986 -122.94772409587401) bank184319 +184320 POINT(36.99029603107448 -123.29058655411845) bank184320 +184321 POINT(37.99057951910467 -121.81396438329105) bank184321 +184322 POINT(38.39343173313443 -122.71813590363597) bank184322 +184323 POINT(38.54509266447123 -121.54199018104504) bank184323 +184324 POINT(37.238141246240815 -121.83224353986132) bank184324 +184325 POINT(37.87710711227819 -123.21078390274893) bank184325 +184326 POINT(38.2504040077372 -123.34159601555086) bank184326 +184327 POINT(37.31541508380003 -122.58314014118105) bank184327 +184328 POINT(37.30203171722373 -121.6441661890702) bank184328 +184329 POINT(38.47053615116516 -122.94276374947613) bank184329 +184330 POINT(38.31747851785772 -122.19373709386872) bank184330 +184331 POINT(38.397489894640486 -122.3845826764558) bank184331 +184332 POINT(37.578373899702356 -123.01661159483675) bank184332 +184333 POINT(38.284879503617255 -121.44646022508883) bank184333 +184334 POINT(37.56505991214422 -122.2314599622538) bank184334 +184335 POINT(36.79370790262507 -121.80139375836288) bank184335 +184336 POINT(37.80212969803801 -122.23709693802459) bank184336 +184337 POINT(37.43826454033788 -123.32823339594502) bank184337 +184338 POINT(38.341690734054616 -123.05507124898037) bank184338 +184339 POINT(37.858924863488724 -121.83618379235688) bank184339 +184340 POINT(38.37355489419803 -123.30220170694145) bank184340 +184341 POINT(37.972192542955675 -122.3921293937796) bank184341 +184342 POINT(37.95397047138507 -122.98777819975635) bank184342 +184343 POINT(36.78221995168783 -122.56109369476512) bank184343 +184344 POINT(37.9285008690654 -122.08123130233177) bank184344 +184345 POINT(38.05309469303425 -123.3817623162755) bank184345 +184346 POINT(37.65507759483997 -121.65479997676522) bank184346 +184347 POINT(36.850079521725846 -123.19015917769008) bank184347 +184348 POINT(37.30700368404149 -121.71809531113016) bank184348 +184349 POINT(38.24222183410813 -122.80386730190224) bank184349 +184350 POINT(37.68487846933765 -122.67075806784466) bank184350 +184351 POINT(37.46336901084296 -121.6819354477871) bank184351 +184352 POINT(38.54865302803484 -121.98718379804957) bank184352 +184353 POINT(38.07459076109894 -122.27010954831235) bank184353 +184354 POINT(36.80091701077617 -123.32611749843474) bank184354 +184355 POINT(38.708823419916655 -122.57683697462002) bank184355 +184356 POINT(36.927669842986106 -121.98864701678654) bank184356 +184357 POINT(37.918506390595844 -121.95168206973199) bank184357 +184358 POINT(38.55840093504131 -122.64447210417366) bank184358 +184359 POINT(38.7135992144852 -122.63651582062215) bank184359 +184360 POINT(37.14558083680143 -121.87122933853222) bank184360 +184361 POINT(38.50208394788702 -123.38302347290247) bank184361 +184362 POINT(38.73019713672413 -122.32640924954376) bank184362 +184363 POINT(37.93586974343764 -121.8527431272512) bank184363 +184364 POINT(37.41816917786891 -121.92297153851848) bank184364 +184365 POINT(37.5382303637764 -123.05019728394696) bank184365 +184366 POINT(37.81969256336111 -123.33816855748506) bank184366 +184367 POINT(37.88817813454779 -122.93629213852239) bank184367 +184368 POINT(38.64413326459057 -122.4828372722786) bank184368 +184369 POINT(37.631276679505056 -121.81507845847501) bank184369 +184370 POINT(38.46590053038569 -123.31401447038085) bank184370 +184371 POINT(38.61901069739068 -123.33302153170119) bank184371 +184372 POINT(37.25354424941799 -122.78291151047486) bank184372 +184373 POINT(38.57493273264696 -122.63615984571891) bank184373 +184374 POINT(37.82443693016521 -122.17125331720443) bank184374 +184375 POINT(38.27469127813685 -123.41441354594507) bank184375 +184376 POINT(37.26818898790554 -122.9804628890761) bank184376 +184377 POINT(37.680804852137335 -122.55989864738406) bank184377 +184378 POINT(37.06444006981745 -121.5038596961465) bank184378 +184379 POINT(37.25354628645162 -123.24355709918666) bank184379 +184380 POINT(38.108157885899054 -122.45898546925596) bank184380 +184381 POINT(38.46245114244934 -121.68860006666384) bank184381 +184382 POINT(38.05668364511623 -121.92883814283584) bank184382 +184383 POINT(37.83853573179435 -122.98042074854885) bank184383 +184384 POINT(38.361800729742626 -122.35446254893769) bank184384 +184385 POINT(38.32552864851949 -121.56920198748243) bank184385 +184386 POINT(37.6058882039123 -122.97250426534272) bank184386 +184387 POINT(37.083980700261264 -122.35135641473764) bank184387 +184388 POINT(36.86500690821635 -122.02236275273044) bank184388 +184389 POINT(37.136140388110526 -122.3691880553957) bank184389 +184390 POINT(37.55291448639166 -123.07438633570213) bank184390 +184391 POINT(37.797505664266566 -122.30561599399702) bank184391 +184392 POINT(37.671954498289914 -122.88921821581826) bank184392 +184393 POINT(38.05565325532655 -123.1058654199581) bank184393 +184394 POINT(36.77508745954682 -122.71382600408165) bank184394 +184395 POINT(37.351786679046334 -122.20122520235871) bank184395 +184396 POINT(38.258712802197216 -123.16645392323534) bank184396 +184397 POINT(38.278131712768335 -122.86051431311138) bank184397 +184398 POINT(37.52638898824662 -121.64793301702066) bank184398 +184399 POINT(38.6621568649112 -123.33479007213997) bank184399 +184400 POINT(38.5619723542292 -121.52443891361797) bank184400 +184401 POINT(38.741198949532034 -121.43288161998082) bank184401 +184402 POINT(37.12191962109656 -123.00805719608542) bank184402 +184403 POINT(37.815628649549964 -121.70842857544237) bank184403 +184404 POINT(37.942263147851946 -122.22347903833068) bank184404 +184405 POINT(36.99577296518337 -122.83240085411829) bank184405 +184406 POINT(37.526346954941246 -122.77069490126348) bank184406 +184407 POINT(37.57152374617497 -121.54380839653876) bank184407 +184408 POINT(38.33344647534551 -122.84622722221931) bank184408 +184409 POINT(38.12452475044787 -122.13211865233744) bank184409 +184410 POINT(38.601684735945334 -123.25373414373452) bank184410 +184411 POINT(36.91570387589154 -121.98821948749173) bank184411 +184412 POINT(37.89703392624311 -121.6125608728163) bank184412 +184413 POINT(38.7470172411528 -123.32273853290926) bank184413 +184414 POINT(36.82527813617726 -123.23261214338632) bank184414 +184415 POINT(36.93613915794721 -122.17410423071064) bank184415 +184416 POINT(36.888127971064925 -123.34308213131953) bank184416 +184417 POINT(38.22925250294172 -122.2638527974777) bank184417 +184418 POINT(36.84691058468507 -122.94847998500262) bank184418 +184419 POINT(38.756954051588195 -121.92418160506907) bank184419 +184420 POINT(38.69964805684038 -123.0714074613389) bank184420 +184421 POINT(37.66121007394299 -121.74496610309399) bank184421 +184422 POINT(36.88394764137585 -121.99380639975016) bank184422 +184423 POINT(37.52569674391317 -122.54215081982156) bank184423 +184424 POINT(38.493299404298234 -121.85865597301986) bank184424 +184425 POINT(38.16918582133691 -121.96539505618726) bank184425 +184426 POINT(38.34138640344212 -122.03674571812864) bank184426 +184427 POINT(37.40803115702634 -121.71060126275678) bank184427 +184428 POINT(38.22026667355166 -121.75851290762336) bank184428 +184429 POINT(38.55580517692452 -122.32834857018335) bank184429 +184430 POINT(38.54227445985395 -122.92973832434194) bank184430 +184431 POINT(38.50178480386808 -122.67930615946393) bank184431 +184432 POINT(37.89561059910803 -121.97371822596158) bank184432 +184433 POINT(38.49397597994063 -121.66209953952121) bank184433 +184434 POINT(38.766986348957545 -122.31629743921238) bank184434 +184435 POINT(38.23608134754048 -122.86831708316387) bank184435 +184436 POINT(37.12130881349411 -123.27559640805407) bank184436 +184437 POINT(38.36561964702566 -121.83966740104894) bank184437 +184438 POINT(37.29982677827211 -122.58183350541584) bank184438 +184439 POINT(38.019877651155625 -122.50878638695659) bank184439 +184440 POINT(38.101492527865254 -122.98708283389405) bank184440 +184441 POINT(37.786395549998986 -121.47934175487947) bank184441 +184442 POINT(36.894220944397034 -122.36829726855045) bank184442 +184443 POINT(38.672788636817344 -123.1705028812586) bank184443 +184444 POINT(37.557595181055 -122.63239487782694) bank184444 +184445 POINT(37.520979289557914 -122.20110791524515) bank184445 +184446 POINT(37.164426653065554 -121.53793237226247) bank184446 +184447 POINT(37.941913450322126 -122.57257174552133) bank184447 +184448 POINT(36.87004239449964 -121.87725766399447) bank184448 +184449 POINT(37.32905176591981 -122.06149151135942) bank184449 +184450 POINT(38.586204248142536 -123.16898989587956) bank184450 +184451 POINT(38.45613063892057 -121.48122561290519) bank184451 +184452 POINT(37.57911987583931 -121.50009462163497) bank184452 +184453 POINT(37.242904373172735 -121.86818849766544) bank184453 +184454 POINT(38.44196245500535 -122.7429294787573) bank184454 +184455 POINT(36.871231617876255 -122.98828852942086) bank184455 +184456 POINT(38.01368358669635 -121.76903099558648) bank184456 +184457 POINT(38.21920925500484 -122.52323306921588) bank184457 +184458 POINT(38.5000361904468 -122.80649833157892) bank184458 +184459 POINT(37.84656565560863 -121.47574320295945) bank184459 +184460 POINT(37.86933222826593 -122.17465591388145) bank184460 +184461 POINT(37.435649864979275 -121.6996652359778) bank184461 +184462 POINT(37.355379530063544 -122.561891997386) bank184462 +184463 POINT(37.226727178502216 -122.40422281360372) bank184463 +184464 POINT(37.02501704293088 -121.98286427846348) bank184464 +184465 POINT(37.72326931547465 -121.46848375479203) bank184465 +184466 POINT(38.09369943345771 -121.66823522661032) bank184466 +184467 POINT(37.92648562847166 -123.07750095885385) bank184467 +184468 POINT(36.830555870500554 -123.2375971389994) bank184468 +184469 POINT(36.78903952512822 -122.45661843705697) bank184469 +184470 POINT(37.912946956064495 -123.37649002301974) bank184470 +184471 POINT(38.030229398486235 -121.60870254789731) bank184471 +184472 POINT(38.337576510971004 -122.4828942963479) bank184472 +184473 POINT(37.21856877249304 -123.40427022201125) bank184473 +184474 POINT(37.86885124424043 -122.59102550367435) bank184474 +184475 POINT(37.62125654074771 -122.1171919831466) bank184475 +184476 POINT(38.23806170243329 -121.59180186275637) bank184476 +184477 POINT(38.31564081614227 -122.19037289094761) bank184477 +184478 POINT(36.79483467557099 -122.10796791925199) bank184478 +184479 POINT(38.11424840086522 -121.84525603845788) bank184479 +184480 POINT(37.84340207297541 -121.48004799683072) bank184480 +184481 POINT(37.348885992507775 -122.47072610966958) bank184481 +184482 POINT(36.80584803691515 -123.3592540615719) bank184482 +184483 POINT(38.664409303854384 -122.93746347330747) bank184483 +184484 POINT(37.46598954804429 -122.02119081625474) bank184484 +184485 POINT(37.806447968690875 -122.66155371842034) bank184485 +184486 POINT(37.906726545332184 -121.82881074655403) bank184486 +184487 POINT(37.57430802998843 -123.29713971689064) bank184487 +184488 POINT(37.204132048053104 -122.72016554379547) bank184488 +184489 POINT(38.04090069857295 -123.4044494696978) bank184489 +184490 POINT(37.008735530826215 -121.5039111023094) bank184490 +184491 POINT(38.42855598421666 -122.00276386129069) bank184491 +184492 POINT(36.90126066233097 -122.13139676223054) bank184492 +184493 POINT(37.81071502332203 -123.08063514341997) bank184493 +184494 POINT(37.075340816023115 -123.24668982217241) bank184494 +184495 POINT(38.202773421238916 -123.4095056623221) bank184495 +184496 POINT(36.9109862991147 -123.3476098087917) bank184496 +184497 POINT(38.053815289856985 -122.09154044624991) bank184497 +184498 POINT(37.591805717943416 -121.70228051033358) bank184498 +184499 POINT(37.87871792920976 -123.38293558521852) bank184499 +184500 POINT(37.01693742888783 -121.53495986071871) bank184500 +184501 POINT(36.89355723115217 -121.53527843206402) bank184501 +184502 POINT(37.52137463715163 -122.9466280549711) bank184502 +184503 POINT(38.306717518164874 -122.58154404727742) bank184503 +184504 POINT(38.26044929330271 -123.00279991910737) bank184504 +184505 POINT(38.12819462563972 -121.55176889379052) bank184505 +184506 POINT(38.561515834030615 -123.27839726350706) bank184506 +184507 POINT(37.57976140329551 -122.59296986968894) bank184507 +184508 POINT(36.94257591402418 -122.32959411352299) bank184508 +184509 POINT(37.33731162735128 -122.60300749402455) bank184509 +184510 POINT(37.51748876165267 -122.716471488752) bank184510 +184511 POINT(37.049954645471054 -122.73277675373483) bank184511 +184512 POINT(36.989808290698114 -122.47674882290633) bank184512 +184513 POINT(38.01475619542791 -123.41678294491466) bank184513 +184514 POINT(37.02180921256183 -122.69048989222516) bank184514 +184515 POINT(37.3011631614341 -123.31182234338807) bank184515 +184516 POINT(38.119084802468144 -122.51749105909622) bank184516 +184517 POINT(38.40985843976233 -121.84104828935061) bank184517 +184518 POINT(36.94543846295057 -121.80835124015177) bank184518 +184519 POINT(38.041807688012 -123.17492453882005) bank184519 +184520 POINT(37.38739004767777 -121.73212069489) bank184520 +184521 POINT(37.89673350054355 -121.91536740449443) bank184521 +184522 POINT(37.342154364475604 -121.62309973249957) bank184522 +184523 POINT(37.55176173957898 -121.91294572780694) bank184523 +184524 POINT(37.79630945409911 -121.43272193652594) bank184524 +184525 POINT(37.65610628207209 -122.19656330186146) bank184525 +184526 POINT(37.983147364811785 -122.65071024721475) bank184526 +184527 POINT(37.66855951909018 -123.24699113515842) bank184527 +184528 POINT(37.45241900465876 -121.64535719133232) bank184528 +184529 POINT(38.57710783991522 -122.34453015308) bank184529 +184530 POINT(36.81995155384946 -122.27923087322934) bank184530 +184531 POINT(38.420000744121424 -121.5574383959989) bank184531 +184532 POINT(37.88510976939062 -122.1254719420193) bank184532 +184533 POINT(37.786397119134115 -122.29570103225674) bank184533 +184534 POINT(37.4054514827978 -122.03501242944245) bank184534 +184535 POINT(38.46075701998045 -122.40698695409118) bank184535 +184536 POINT(38.196906524824016 -121.46004251124648) bank184536 +184537 POINT(38.003007767039634 -121.8024614351287) bank184537 +184538 POINT(36.844609541383946 -122.88307209984528) bank184538 +184539 POINT(37.13736390479579 -123.2676197078196) bank184539 +184540 POINT(37.57279158218883 -121.84444944207299) bank184540 +184541 POINT(38.63060078609338 -121.96152223847729) bank184541 +184542 POINT(37.91427464043278 -123.17799713199632) bank184542 +184543 POINT(38.599800376204016 -121.99240984571888) bank184543 +184544 POINT(38.34671071633216 -122.10889332688842) bank184544 +184545 POINT(38.50605989843915 -122.60171012655003) bank184545 +184546 POINT(38.6899113259457 -122.37337583922819) bank184546 +184547 POINT(37.40905065051489 -123.36293349815503) bank184547 +184548 POINT(38.25130310391937 -122.44309748297476) bank184548 +184549 POINT(36.84983805171007 -121.51650703970712) bank184549 +184550 POINT(37.077012318981055 -122.20126558217324) bank184550 +184551 POINT(37.133586223129385 -122.31816607227842) bank184551 +184552 POINT(37.726320293233925 -122.98767502727898) bank184552 +184553 POINT(37.29932708549419 -121.53890480667366) bank184553 +184554 POINT(36.96039534902974 -121.63962004979034) bank184554 +184555 POINT(38.72038903324151 -122.257661942139) bank184555 +184556 POINT(37.083809288534155 -122.46371304907902) bank184556 +184557 POINT(37.42835656631948 -123.27500448565651) bank184557 +184558 POINT(37.12427240703957 -121.950862917684) bank184558 +184559 POINT(37.25787337188353 -121.497399800947) bank184559 +184560 POINT(38.666850353881394 -122.27146660528707) bank184560 +184561 POINT(38.31864665877002 -123.34588971067001) bank184561 +184562 POINT(38.384807380157035 -121.4922231598661) bank184562 +184563 POINT(37.0550891589579 -123.36538322266495) bank184563 +184564 POINT(37.59083898811462 -121.45208913323233) bank184564 +184565 POINT(37.58843224084086 -122.45070784617309) bank184565 +184566 POINT(37.513320330581266 -122.65050267495644) bank184566 +184567 POINT(38.513609518322006 -121.80736698084256) bank184567 +184568 POINT(38.15524911634461 -122.0956916969063) bank184568 +184569 POINT(37.01085251217415 -122.25679058835891) bank184569 +184570 POINT(37.441836156364516 -121.94370816848745) bank184570 +184571 POINT(37.516304345426484 -122.70009815302781) bank184571 +184572 POINT(37.050741401144464 -122.40101747704088) bank184572 +184573 POINT(37.943273701059056 -123.07992558310404) bank184573 +184574 POINT(38.60947088380195 -121.76042679973136) bank184574 +184575 POINT(36.84282337458292 -123.09187647650909) bank184575 +184576 POINT(38.098316136799745 -122.06472534523144) bank184576 +184577 POINT(38.34686752636133 -121.49747439414168) bank184577 +184578 POINT(38.39336510509032 -122.21747471395207) bank184578 +184579 POINT(36.98018607187621 -122.57120391346908) bank184579 +184580 POINT(37.38820144326719 -123.05382674512798) bank184580 +184581 POINT(38.28816109672066 -122.1948890604767) bank184581 +184582 POINT(38.14520830328298 -121.71338183850783) bank184582 +184583 POINT(37.81186650543779 -123.04646075669835) bank184583 +184584 POINT(38.67860327467196 -121.80649418840589) bank184584 +184585 POINT(37.20051580355045 -122.7646541912565) bank184585 +184586 POINT(37.57600689508938 -122.41410290177161) bank184586 +184587 POINT(38.56114288426963 -122.42897580409027) bank184587 +184588 POINT(37.44620958272545 -122.76161913036535) bank184588 +184589 POINT(38.0484009538203 -122.16453647653994) bank184589 +184590 POINT(37.02279680045437 -122.15351210950857) bank184590 +184591 POINT(38.64889220942778 -122.50367859471015) bank184591 +184592 POINT(37.416382613965894 -122.60571617005249) bank184592 +184593 POINT(37.31329843694941 -122.89322061427606) bank184593 +184594 POINT(37.973302532995866 -121.42752344859998) bank184594 +184595 POINT(38.75575047192298 -121.5404068639329) bank184595 +184596 POINT(38.57547006388746 -123.30660335743723) bank184596 +184597 POINT(37.51821062620628 -122.23415228389393) bank184597 +184598 POINT(37.21849066924846 -122.22761655127952) bank184598 +184599 POINT(38.687270259384434 -121.87618548598653) bank184599 +184600 POINT(37.76578873776848 -123.22896312091741) bank184600 +184601 POINT(37.18519150268738 -122.18336268872767) bank184601 +184602 POINT(38.606630870308095 -122.52992450962213) bank184602 +184603 POINT(37.406445553172595 -122.57139574885981) bank184603 +184604 POINT(37.04992945438683 -121.7650897864521) bank184604 +184605 POINT(37.77560664283251 -122.16397808011133) bank184605 +184606 POINT(38.54414936887069 -121.7582492926827) bank184606 +184607 POINT(37.62012813252548 -123.32459088662067) bank184607 +184608 POINT(37.78249649741501 -123.2382307650422) bank184608 +184609 POINT(38.104294031939936 -122.03238098987009) bank184609 +184610 POINT(37.59462542521381 -121.73699319594441) bank184610 +184611 POINT(38.26271512553196 -122.95297702362262) bank184611 +184612 POINT(38.696415082193894 -123.33847304031218) bank184612 +184613 POINT(38.10354835421976 -122.28705077427618) bank184613 +184614 POINT(36.94885815147857 -122.96481041553818) bank184614 +184615 POINT(37.968152906991996 -121.42565316950082) bank184615 +184616 POINT(38.320574189455755 -121.91972162528897) bank184616 +184617 POINT(38.73385713404979 -122.71655815484733) bank184617 +184618 POINT(37.607448441639136 -122.15753367942389) bank184618 +184619 POINT(37.17379407737671 -121.73088701715112) bank184619 +184620 POINT(38.70603924978415 -121.91436439541138) bank184620 +184621 POINT(37.78025410944017 -122.68967953153987) bank184621 +184622 POINT(37.830508643903684 -122.68122352128037) bank184622 +184623 POINT(37.630221821490125 -122.80740604620547) bank184623 +184624 POINT(37.72879314948665 -121.93689899764378) bank184624 +184625 POINT(38.52594474967799 -121.8720266579962) bank184625 +184626 POINT(37.181833679700645 -122.34409942930814) bank184626 +184627 POINT(36.86746146723232 -123.41135537409565) bank184627 +184628 POINT(37.49704922011229 -122.17205507449916) bank184628 +184629 POINT(38.19292194038954 -121.97260359810288) bank184629 +184630 POINT(37.7440822846046 -122.83416372005142) bank184630 +184631 POINT(38.41291913702197 -121.6799945513845) bank184631 +184632 POINT(37.69779060298501 -122.06534702233681) bank184632 +184633 POINT(37.49684028376858 -122.92587570665445) bank184633 +184634 POINT(38.52437545020766 -121.72780126616293) bank184634 +184635 POINT(38.444458327203385 -121.56742053829586) bank184635 +184636 POINT(37.015683458722876 -122.94825530828466) bank184636 +184637 POINT(38.315575970722826 -123.29438764447802) bank184637 +184638 POINT(37.202515417445575 -121.53640385801742) bank184638 +184639 POINT(38.565323740058105 -121.47285867334651) bank184639 +184640 POINT(37.41934129073993 -122.22544097278035) bank184640 +184641 POINT(38.01594447098203 -121.97177346452915) bank184641 +184642 POINT(38.53749319561472 -122.31908114930444) bank184642 +184643 POINT(37.90092896216765 -122.72666748988854) bank184643 +184644 POINT(38.354724572168436 -122.0599522713489) bank184644 +184645 POINT(37.60414366856761 -122.7998311556122) bank184645 +184646 POINT(38.21209394727004 -122.95121968262313) bank184646 +184647 POINT(37.093155834280886 -122.19559000750213) bank184647 +184648 POINT(38.53125923470612 -122.93436506850857) bank184648 +184649 POINT(38.628701003665526 -121.79666720196406) bank184649 +184650 POINT(38.459544910117785 -122.73456769696497) bank184650 +184651 POINT(37.89976082511201 -121.63064550811599) bank184651 +184652 POINT(37.09887128501873 -123.02682796438971) bank184652 +184653 POINT(37.040012244512255 -121.55128772300179) bank184653 +184654 POINT(38.73513431129304 -121.46399383476061) bank184654 +184655 POINT(37.64636492460174 -122.39312301624005) bank184655 +184656 POINT(37.29782206724775 -123.09006807572837) bank184656 +184657 POINT(37.28398499696085 -123.15111809456025) bank184657 +184658 POINT(38.71886098872884 -123.01071300377069) bank184658 +184659 POINT(37.10525196269914 -122.3500288550085) bank184659 +184660 POINT(38.11061120555517 -122.07728405033208) bank184660 +184661 POINT(36.881784877460106 -121.82037509998098) bank184661 +184662 POINT(38.1124397614808 -122.91516783194619) bank184662 +184663 POINT(36.81275235550214 -122.3862356748192) bank184663 +184664 POINT(37.43519675811706 -121.73772003826157) bank184664 +184665 POINT(38.54253296346312 -121.61222078821326) bank184665 +184666 POINT(37.667795112759855 -123.26938703309617) bank184666 +184667 POINT(36.89461679977562 -122.63651803608832) bank184667 +184668 POINT(38.3157062310374 -121.80607415760115) bank184668 +184669 POINT(38.34976741095374 -122.10571657215047) bank184669 +184670 POINT(37.226325846210614 -122.30472892882666) bank184670 +184671 POINT(37.851669611298426 -121.7516668710622) bank184671 +184672 POINT(38.05716074626965 -123.3423574206854) bank184672 +184673 POINT(38.500344959068784 -121.72110047738293) bank184673 +184674 POINT(36.882026295800244 -122.01706846353282) bank184674 +184675 POINT(36.907445586839785 -122.70887575277392) bank184675 +184676 POINT(36.828462934289135 -121.84653315566094) bank184676 +184677 POINT(36.91417480389323 -122.52938965044487) bank184677 +184678 POINT(38.768939625504586 -121.80286045995945) bank184678 +184679 POINT(37.76234294468221 -122.87989915253421) bank184679 +184680 POINT(37.04913572055781 -121.99704451233414) bank184680 +184681 POINT(38.406953271693745 -122.2793341799355) bank184681 +184682 POINT(37.67312933367617 -122.8883464117961) bank184682 +184683 POINT(38.657535427859145 -122.24840991114304) bank184683 +184684 POINT(38.72448480465999 -122.92920792953032) bank184684 +184685 POINT(38.3622325684866 -122.99323590598839) bank184685 +184686 POINT(37.63863688080639 -122.0143521647956) bank184686 +184687 POINT(38.69289526272162 -123.17024685293373) bank184687 +184688 POINT(38.64778398128036 -121.91092448827972) bank184688 +184689 POINT(37.84312124329088 -122.90637258241736) bank184689 +184690 POINT(38.4441045153888 -121.66087141245745) bank184690 +184691 POINT(37.93213125173799 -122.87634341225092) bank184691 +184692 POINT(37.12198047762733 -121.48783539606461) bank184692 +184693 POINT(38.645618384241494 -122.1200891541519) bank184693 +184694 POINT(38.57092961097427 -121.49495098862083) bank184694 +184695 POINT(38.07254103299808 -123.35703443797775) bank184695 +184696 POINT(37.92264202359914 -121.55398605772332) bank184696 +184697 POINT(38.57617884526746 -122.18487527539183) bank184697 +184698 POINT(36.792141764917915 -121.78052893720788) bank184698 +184699 POINT(37.27826821086353 -121.75979382183314) bank184699 +184700 POINT(37.58418756401946 -122.8066779191181) bank184700 +184701 POINT(36.908213386217916 -123.33796127433219) bank184701 +184702 POINT(37.688449242038374 -121.9735389229749) bank184702 +184703 POINT(37.29691825388253 -122.09205165628569) bank184703 +184704 POINT(38.52254138351527 -123.01618056975187) bank184704 +184705 POINT(36.797686131987284 -121.46328798470773) bank184705 +184706 POINT(37.48718822943852 -122.06713531825235) bank184706 +184707 POINT(37.331512631800564 -122.76379471877912) bank184707 +184708 POINT(38.31757687421303 -122.775187034143) bank184708 +184709 POINT(37.61764515672412 -121.7040029963261) bank184709 +184710 POINT(37.09154599484965 -122.90946026688111) bank184710 +184711 POINT(37.026817674126676 -122.97705822599603) bank184711 +184712 POINT(38.52827429389096 -123.24170929221624) bank184712 +184713 POINT(38.20899195107866 -121.55181341675483) bank184713 +184714 POINT(38.33291357718446 -122.67860847563473) bank184714 +184715 POINT(37.44702933979626 -121.80531820647641) bank184715 +184716 POINT(37.52200076721003 -121.81561974859186) bank184716 +184717 POINT(37.23816064286666 -122.69023157439689) bank184717 +184718 POINT(37.092829713113275 -122.60781075708731) bank184718 +184719 POINT(37.929630220640554 -123.25366105744591) bank184719 +184720 POINT(36.982747928825304 -122.31661945564379) bank184720 +184721 POINT(38.65908391306992 -123.03361863595133) bank184721 +184722 POINT(38.45745818028985 -123.41656609408778) bank184722 +184723 POINT(38.1327013296487 -122.17912934085096) bank184723 +184724 POINT(38.77453055458602 -121.93964142558205) bank184724 +184725 POINT(38.630242573532904 -121.52042013752578) bank184725 +184726 POINT(37.81319472690247 -122.07768786303748) bank184726 +184727 POINT(36.917372764458555 -123.23732220196017) bank184727 +184728 POINT(37.0547574537887 -121.50153570308753) bank184728 +184729 POINT(36.920730056664226 -122.72012956260315) bank184729 +184730 POINT(38.248926944641205 -122.02494875330501) bank184730 +184731 POINT(37.40430713226399 -123.24069224800704) bank184731 +184732 POINT(38.49770941248918 -122.56776577719921) bank184732 +184733 POINT(38.205474453301434 -122.39277000807994) bank184733 +184734 POINT(38.67514300852101 -122.47593821776655) bank184734 +184735 POINT(38.38302923829355 -121.8745078336963) bank184735 +184736 POINT(37.206849070113606 -121.43888731756594) bank184736 +184737 POINT(37.746036045433364 -121.5277381818951) bank184737 +184738 POINT(38.69818561967445 -122.40756754273906) bank184738 +184739 POINT(37.678993497670035 -121.80850249917526) bank184739 +184740 POINT(37.97318147027272 -122.1688115865427) bank184740 +184741 POINT(38.727903467200534 -121.98244160966787) bank184741 +184742 POINT(38.264042215372555 -122.65941474501022) bank184742 +184743 POINT(37.1909607593519 -121.5397377112376) bank184743 +184744 POINT(37.29543164133654 -121.51203611762215) bank184744 +184745 POINT(36.88009289539773 -122.8029765437534) bank184745 +184746 POINT(37.3955627060253 -122.01875381058186) bank184746 +184747 POINT(37.776557756392805 -123.09522776122071) bank184747 +184748 POINT(36.81949215258288 -122.02901365556563) bank184748 +184749 POINT(36.82552530742145 -122.3596654278962) bank184749 +184750 POINT(38.28084334930697 -122.38544573388586) bank184750 +184751 POINT(38.411243774140466 -123.11045798435252) bank184751 +184752 POINT(37.97817269689142 -121.61257701502758) bank184752 +184753 POINT(38.07995340792235 -121.63823968510806) bank184753 +184754 POINT(38.63205601142218 -121.50348151484654) bank184754 +184755 POINT(37.119916604041684 -122.37239117826171) bank184755 +184756 POINT(37.84649525967106 -122.10527125225845) bank184756 +184757 POINT(38.1895340029467 -123.19662093689577) bank184757 +184758 POINT(38.08559988209313 -122.949472218557) bank184758 +184759 POINT(38.29456366044428 -121.83224656206687) bank184759 +184760 POINT(38.77139923889182 -123.38324972224251) bank184760 +184761 POINT(37.389809092483986 -121.67183116140731) bank184761 +184762 POINT(37.128645766381304 -123.12261407165315) bank184762 +184763 POINT(37.293252144335504 -122.01354535797267) bank184763 +184764 POINT(37.25944124351141 -123.05403875494311) bank184764 +184765 POINT(38.17799555501117 -121.51306362967757) bank184765 +184766 POINT(38.4665061966971 -122.56132897692544) bank184766 +184767 POINT(37.51904199952848 -122.55288788394675) bank184767 +184768 POINT(37.922321818274014 -122.95080558159158) bank184768 +184769 POINT(38.77271435259655 -123.0393555567614) bank184769 +184770 POINT(37.71840165144067 -123.09263680795661) bank184770 +184771 POINT(37.044506943782785 -123.38814080116926) bank184771 +184772 POINT(38.16077509950348 -122.95649596128719) bank184772 +184773 POINT(36.79418757349034 -123.06023933699946) bank184773 +184774 POINT(37.17833276924152 -123.06410931563468) bank184774 +184775 POINT(38.75254671197092 -123.2527180117769) bank184775 +184776 POINT(36.831540003752394 -121.99260773635339) bank184776 +184777 POINT(37.03693831466639 -121.51728585695261) bank184777 +184778 POINT(38.542138126603646 -122.64399536372106) bank184778 +184779 POINT(37.45587022898999 -122.02743473728941) bank184779 +184780 POINT(38.61419342703338 -123.40570867990056) bank184780 +184781 POINT(37.28295467338723 -122.85790721672306) bank184781 +184782 POINT(37.79430042244649 -122.16177874225855) bank184782 +184783 POINT(37.39163031725592 -123.10551146127493) bank184783 +184784 POINT(37.97889828956577 -123.33435818016278) bank184784 +184785 POINT(37.27035301393266 -122.86819011868441) bank184785 +184786 POINT(37.879526514537005 -122.60467100057632) bank184786 +184787 POINT(37.55946745300827 -121.82946996695772) bank184787 +184788 POINT(37.779841705738725 -123.33320441939286) bank184788 +184789 POINT(37.740240180331305 -122.12853515140354) bank184789 +184790 POINT(37.311867223658794 -122.22025607684024) bank184790 +184791 POINT(38.652283125073865 -122.16925896074655) bank184791 +184792 POINT(38.53110259029226 -123.3234646146263) bank184792 +184793 POINT(37.21534815191382 -122.60815910606829) bank184793 +184794 POINT(37.01999322063064 -122.42607895378225) bank184794 +184795 POINT(36.8962761056177 -122.30606042613422) bank184795 +184796 POINT(38.52687035711219 -122.81548329586053) bank184796 +184797 POINT(36.99862821665412 -122.46685679086562) bank184797 +184798 POINT(37.61431240321501 -123.27664518842302) bank184798 +184799 POINT(37.788631017726345 -123.26169506400424) bank184799 +184800 POINT(37.05211672909673 -122.81843509866046) bank184800 +184801 POINT(36.92527484089527 -122.7638512300247) bank184801 +184802 POINT(37.870490968728625 -122.27877493385883) bank184802 +184803 POINT(37.225185875193205 -121.43245023829603) bank184803 +184804 POINT(36.97569129534601 -121.4742592570716) bank184804 +184805 POINT(37.05892050921616 -121.93991074047878) bank184805 +184806 POINT(38.11775888592966 -121.7729741912376) bank184806 +184807 POINT(37.37738302160513 -122.71179208226812) bank184807 +184808 POINT(37.939872955558755 -121.87534515959386) bank184808 +184809 POINT(37.749152812583425 -122.39326996698738) bank184809 +184810 POINT(37.74758571181781 -122.30876776893446) bank184810 +184811 POINT(37.88141414280989 -121.74480690604948) bank184811 +184812 POINT(37.58854398001856 -123.05339964405903) bank184812 +184813 POINT(38.097358572060095 -123.24664106963986) bank184813 +184814 POINT(38.576992630110844 -122.28004369282851) bank184814 +184815 POINT(38.27377981635692 -123.34113307133764) bank184815 +184816 POINT(38.0774260244817 -122.58477985755476) bank184816 +184817 POINT(37.44230525170875 -121.84086093221866) bank184817 +184818 POINT(37.62293487815552 -122.67396685391947) bank184818 +184819 POINT(38.720071779952 -121.93568486643302) bank184819 +184820 POINT(37.076117888834005 -122.48162997344426) bank184820 +184821 POINT(38.23584527792802 -122.29519494794617) bank184821 +184822 POINT(37.44765873023817 -121.9038431835429) bank184822 +184823 POINT(38.77371735256231 -122.63816265151658) bank184823 +184824 POINT(37.44259519719518 -123.16101896771596) bank184824 +184825 POINT(38.206730303903804 -122.03044727326815) bank184825 +184826 POINT(37.30209677401315 -122.17385786986591) bank184826 +184827 POINT(37.18007952002925 -123.32531201224278) bank184827 +184828 POINT(38.04416014793259 -121.71167554677051) bank184828 +184829 POINT(38.41876441887437 -123.33531122495663) bank184829 +184830 POINT(38.50666087107173 -122.11940644959627) bank184830 +184831 POINT(37.84285441306337 -122.63878684021263) bank184831 +184832 POINT(37.42966512143586 -121.78755058721627) bank184832 +184833 POINT(37.42109223775469 -122.75797977777927) bank184833 +184834 POINT(37.13690163109728 -123.37087634321476) bank184834 +184835 POINT(38.01802323106558 -122.20708185996422) bank184835 +184836 POINT(37.53606985736181 -121.84344945165186) bank184836 +184837 POINT(37.68488501658692 -123.06700023284768) bank184837 +184838 POINT(38.19431023828671 -122.91866173694879) bank184838 +184839 POINT(37.374759789477125 -121.96073663716903) bank184839 +184840 POINT(37.6908839023892 -121.65162891158896) bank184840 +184841 POINT(37.56922767532379 -122.24596126271648) bank184841 +184842 POINT(38.70118800622197 -121.7196776659926) bank184842 +184843 POINT(36.819944188903236 -121.98980768641422) bank184843 +184844 POINT(37.09213941004423 -122.47593179470437) bank184844 +184845 POINT(36.874904434878495 -122.65883777532186) bank184845 +184846 POINT(37.06396752636788 -123.20217115160706) bank184846 +184847 POINT(36.81535448712243 -121.56117786771325) bank184847 +184848 POINT(37.44215517211645 -123.1591788262629) bank184848 +184849 POINT(36.9643704507544 -123.32373208480927) bank184849 +184850 POINT(37.52677140094095 -122.42545376390024) bank184850 +184851 POINT(37.6222025499915 -122.30789047184714) bank184851 +184852 POINT(37.47083772697805 -123.11079111429262) bank184852 +184853 POINT(37.06955317868523 -122.82452899843801) bank184853 +184854 POINT(38.332419493265256 -123.0951675725823) bank184854 +184855 POINT(37.62390846211082 -123.09241791489927) bank184855 +184856 POINT(37.530289966817506 -122.73968853806774) bank184856 +184857 POINT(37.75789292493301 -122.11610461640127) bank184857 +184858 POINT(37.61329731405677 -123.15258158618283) bank184858 +184859 POINT(37.4802736145539 -123.26849550664063) bank184859 +184860 POINT(36.94774198516458 -121.6895575668333) bank184860 +184861 POINT(38.15662830946756 -122.09173287682162) bank184861 +184862 POINT(38.12472626482196 -122.9275794198869) bank184862 +184863 POINT(38.5960272669105 -121.95921680371616) bank184863 +184864 POINT(38.15133353553657 -122.86309506411303) bank184864 +184865 POINT(37.500727100526674 -123.10582666039346) bank184865 +184866 POINT(37.03154961885926 -123.1791236586227) bank184866 +184867 POINT(38.28232285828073 -121.93072125645666) bank184867 +184868 POINT(38.41878905987618 -122.23975891417686) bank184868 +184869 POINT(37.97963614979854 -122.15530592161733) bank184869 +184870 POINT(38.67404961575189 -122.58164607681513) bank184870 +184871 POINT(38.00303219083138 -122.8031138018846) bank184871 +184872 POINT(38.494798330783254 -122.82077051095355) bank184872 +184873 POINT(36.97902649014765 -121.80858046910045) bank184873 +184874 POINT(37.68284911088851 -122.21098357083468) bank184874 +184875 POINT(37.58002153688184 -123.35418639905173) bank184875 +184876 POINT(38.73283436407864 -123.14559240718035) bank184876 +184877 POINT(36.94715280011837 -121.60916031877754) bank184877 +184878 POINT(37.461468935974196 -122.95839084845373) bank184878 +184879 POINT(37.736606316543345 -121.61523744358657) bank184879 +184880 POINT(37.358192690462154 -123.39342431518823) bank184880 +184881 POINT(38.20063344069911 -122.6248256431392) bank184881 +184882 POINT(37.572681138140645 -122.02635077308939) bank184882 +184883 POINT(37.56993724377237 -121.79864558994608) bank184883 +184884 POINT(38.09188802523548 -123.25429298218228) bank184884 +184885 POINT(38.02771130108236 -122.20901601063022) bank184885 +184886 POINT(38.09938738591549 -123.09003462951021) bank184886 +184887 POINT(38.728417526797024 -122.32658992174821) bank184887 +184888 POINT(37.50375295152767 -122.22492591452158) bank184888 +184889 POINT(37.12448353625479 -121.85588022764509) bank184889 +184890 POINT(37.32005646321203 -122.09635671741597) bank184890 +184891 POINT(37.27537058786935 -123.36963786637199) bank184891 +184892 POINT(38.5619044715682 -123.12422691610591) bank184892 +184893 POINT(38.221921977725216 -122.37825696778383) bank184893 +184894 POINT(37.50907803863668 -121.66211875235918) bank184894 +184895 POINT(38.596963405865075 -122.15730691736003) bank184895 +184896 POINT(38.49984101955717 -122.96620134270644) bank184896 +184897 POINT(37.489935075155856 -123.10907838935421) bank184897 +184898 POINT(38.059634428093645 -122.86918429919236) bank184898 +184899 POINT(37.75573659654097 -121.85814449456625) bank184899 +184900 POINT(38.37266252469243 -123.21331656757005) bank184900 +184901 POINT(37.934080096763864 -121.69200920545296) bank184901 +184902 POINT(38.40904697991942 -121.74189359118348) bank184902 +184903 POINT(38.62510450591848 -121.45708231494395) bank184903 +184904 POINT(37.268310327861265 -122.36339004675554) bank184904 +184905 POINT(37.19408541939495 -122.7118184274136) bank184905 +184906 POINT(37.719744073613114 -123.27371716431128) bank184906 +184907 POINT(37.87429292966826 -123.08115366243736) bank184907 +184908 POINT(37.47748780125165 -121.66485896496981) bank184908 +184909 POINT(38.57152115685218 -121.90199267066315) bank184909 +184910 POINT(37.48989343417644 -122.19524287766099) bank184910 +184911 POINT(37.69475230676507 -122.4466352214946) bank184911 +184912 POINT(37.046548150373695 -121.67418532049675) bank184912 +184913 POINT(38.425675850040285 -122.15546873262976) bank184913 +184914 POINT(37.25996127093264 -121.93695390582306) bank184914 +184915 POINT(37.24406504931511 -122.05964210076917) bank184915 +184916 POINT(37.43534469362854 -121.87158428551744) bank184916 +184917 POINT(38.18590092103767 -121.90311728571486) bank184917 +184918 POINT(38.3045235850926 -123.13871924647704) bank184918 +184919 POINT(38.015073735537186 -122.16165917037364) bank184919 +184920 POINT(37.52973713762975 -122.07645412774228) bank184920 +184921 POINT(37.56459615050076 -122.32481992455263) bank184921 +184922 POINT(36.877400109356074 -122.10062546882837) bank184922 +184923 POINT(38.73543441475558 -122.21857887654002) bank184923 +184924 POINT(36.86971558694318 -122.49430692547726) bank184924 +184925 POINT(38.76171475765585 -122.32941114670939) bank184925 +184926 POINT(36.812663450234325 -122.65595597175744) bank184926 +184927 POINT(36.98758137199297 -123.17551228644636) bank184927 +184928 POINT(37.05454319532173 -121.78393768027378) bank184928 +184929 POINT(38.76222566848344 -123.1279532364178) bank184929 +184930 POINT(37.771991865184766 -122.71308171392242) bank184930 +184931 POINT(37.01546062461761 -122.68928716233034) bank184931 +184932 POINT(38.086306254932296 -121.54768393375599) bank184932 +184933 POINT(37.242996781388435 -122.02742748545354) bank184933 +184934 POINT(38.05289537692125 -122.01105164151319) bank184934 +184935 POINT(37.103335257099125 -121.63314210886628) bank184935 +184936 POINT(36.91457341284839 -121.95509986408932) bank184936 +184937 POINT(37.176185038286704 -122.15552590652499) bank184937 +184938 POINT(37.26437621605864 -122.38278101080923) bank184938 +184939 POINT(38.10463831878245 -122.67217304602993) bank184939 +184940 POINT(37.08324126787507 -123.038158477285) bank184940 +184941 POINT(37.87113034583435 -123.09154457203081) bank184941 +184942 POINT(36.9855198817534 -122.11824298698478) bank184942 +184943 POINT(37.48681530966206 -123.35324014538719) bank184943 +184944 POINT(38.560654622204915 -122.59331485529306) bank184944 +184945 POINT(37.21268320483149 -123.34167215025346) bank184945 +184946 POINT(37.23638190453564 -122.41366804074482) bank184946 +184947 POINT(37.59549720265973 -123.16059650210438) bank184947 +184948 POINT(37.39972534735089 -122.20461658769315) bank184948 +184949 POINT(38.46942608691735 -123.19160748665969) bank184949 +184950 POINT(38.00838669609078 -123.38775072889341) bank184950 +184951 POINT(38.12184276855646 -121.99139474152959) bank184951 +184952 POINT(36.87983758655197 -121.95614508687456) bank184952 +184953 POINT(38.69573888786085 -123.12540659019966) bank184953 +184954 POINT(38.448019004571464 -122.56505356209667) bank184954 +184955 POINT(37.534391357772684 -121.52887543792102) bank184955 +184956 POINT(38.39923107808842 -123.1420792640801) bank184956 +184957 POINT(37.85937254539628 -121.46291629996877) bank184957 +184958 POINT(37.87902464126897 -122.24935771517941) bank184958 +184959 POINT(37.344594414464076 -123.35085366039836) bank184959 +184960 POINT(36.82517065521324 -121.66781490944895) bank184960 +184961 POINT(38.43616418676605 -123.07569205243324) bank184961 +184962 POINT(38.408190737695506 -122.78670700698977) bank184962 +184963 POINT(36.91524738238808 -122.30920028204947) bank184963 +184964 POINT(37.81862214228279 -121.8066648807678) bank184964 +184965 POINT(38.38092290600882 -122.84590525621913) bank184965 +184966 POINT(38.682863808584024 -121.49924569285945) bank184966 +184967 POINT(37.063489362033394 -121.60718627600338) bank184967 +184968 POINT(37.480880789176 -121.74584586803137) bank184968 +184969 POINT(37.72946722968657 -123.1409124655248) bank184969 +184970 POINT(37.96870995724287 -122.51832072591925) bank184970 +184971 POINT(37.94944956011593 -123.4048970818157) bank184971 +184972 POINT(36.855339073307 -123.23596065755908) bank184972 +184973 POINT(37.28536410469976 -121.79943726683068) bank184973 +184974 POINT(38.19256542000548 -122.77802483747558) bank184974 +184975 POINT(38.21991689042452 -123.37952640688364) bank184975 +184976 POINT(38.496535327850744 -121.67698946503675) bank184976 +184977 POINT(38.29771187875488 -122.14922138466513) bank184977 +184978 POINT(38.01952467181958 -121.8478674187652) bank184978 +184979 POINT(37.51597903130449 -122.84957794593433) bank184979 +184980 POINT(37.02031919921709 -122.805727693084) bank184980 +184981 POINT(37.187307669493485 -122.92126731705159) bank184981 +184982 POINT(37.30902323919738 -121.56985374204095) bank184982 +184983 POINT(37.7870869241396 -121.61327738393078) bank184983 +184984 POINT(37.731767501603656 -122.55063119544995) bank184984 +184985 POINT(37.95653991423601 -121.75928330515346) bank184985 +184986 POINT(37.1461485671609 -121.57328817122493) bank184986 +184987 POINT(37.53632716302199 -122.56088912715981) bank184987 +184988 POINT(37.33597131278348 -122.28220287292113) bank184988 +184989 POINT(38.2596482050028 -121.64028010558093) bank184989 +184990 POINT(37.09420155661808 -122.07830815089919) bank184990 +184991 POINT(37.0904015172249 -122.59907296076874) bank184991 +184992 POINT(38.41694016544819 -122.22918240867627) bank184992 +184993 POINT(38.512185737915104 -122.6087913929201) bank184993 +184994 POINT(37.3924095674486 -122.94581119748625) bank184994 +184995 POINT(37.7853274298358 -121.54209155505654) bank184995 +184996 POINT(37.925214586358955 -122.61031813891555) bank184996 +184997 POINT(38.053922220231854 -121.75651561454575) bank184997 +184998 POINT(36.799246674901084 -121.56243045536229) bank184998 +184999 POINT(37.252519744752945 -121.84052010362643) bank184999 +185000 POINT(38.606983966167626 -122.88468029877858) bank185000 +185001 POINT(38.54599805545575 -122.48326533459435) bank185001 +185002 POINT(37.64300059187845 -123.09853078225053) bank185002 +185003 POINT(37.983901074534145 -122.34512774007058) bank185003 +185004 POINT(37.518497490606215 -121.60374517799124) bank185004 +185005 POINT(37.784697126310746 -122.55938779576255) bank185005 +185006 POINT(38.11494537786865 -122.59873137185319) bank185006 +185007 POINT(38.13195076159566 -121.52972909968086) bank185007 +185008 POINT(37.664214746533126 -121.4296835222129) bank185008 +185009 POINT(36.84012489692848 -121.44570946688422) bank185009 +185010 POINT(37.65945985170829 -122.82833427574059) bank185010 +185011 POINT(38.55087420365344 -121.88889420712047) bank185011 +185012 POINT(38.76086868403451 -122.36685134274515) bank185012 +185013 POINT(37.587428331116556 -122.94779452037129) bank185013 +185014 POINT(36.99259313290811 -121.42355024523073) bank185014 +185015 POINT(37.92783966327124 -123.19615049467653) bank185015 +185016 POINT(37.41694759953506 -122.28914543738844) bank185016 +185017 POINT(38.08686081460077 -121.82331888209795) bank185017 +185018 POINT(38.47850772771575 -122.80824448551877) bank185018 +185019 POINT(38.699126945172125 -123.00505725301214) bank185019 +185020 POINT(37.64860556695851 -122.99489173613239) bank185020 +185021 POINT(37.848409954970236 -122.14557550255071) bank185021 +185022 POINT(37.77761286397049 -122.59426671234472) bank185022 +185023 POINT(37.67759417137863 -121.6468663785986) bank185023 +185024 POINT(37.02348605789541 -122.21740277484005) bank185024 +185025 POINT(37.505933396805375 -122.21949206293442) bank185025 +185026 POINT(37.87560003411576 -123.14395781130983) bank185026 +185027 POINT(37.18038110743394 -123.4193747685913) bank185027 +185028 POINT(36.81100505378639 -122.57869937959113) bank185028 +185029 POINT(38.04589787935448 -122.1589424402973) bank185029 +185030 POINT(38.278289381722395 -121.5871958208778) bank185030 +185031 POINT(37.647300291820095 -122.96648308195107) bank185031 +185032 POINT(37.96688664896351 -122.10354586240088) bank185032 +185033 POINT(38.673526836279734 -121.76809705386468) bank185033 +185034 POINT(38.04419525423504 -122.77836989504023) bank185034 +185035 POINT(38.36458807670113 -122.52692041173252) bank185035 +185036 POINT(37.467811822874246 -123.10649361818912) bank185036 +185037 POINT(37.21863619989161 -121.67674806643662) bank185037 +185038 POINT(37.96199805684401 -121.62948040545689) bank185038 +185039 POINT(38.150504418805475 -122.38784973520677) bank185039 +185040 POINT(37.39298614385851 -122.11860505468793) bank185040 +185041 POINT(36.94582202459641 -122.98049734588147) bank185041 +185042 POINT(38.120345547172654 -121.46136988781146) bank185042 +185043 POINT(38.72407690816549 -123.21129528904882) bank185043 +185044 POINT(37.372203887598005 -121.73379677817329) bank185044 +185045 POINT(38.73653516585325 -121.77245329293308) bank185045 +185046 POINT(38.470307911004625 -122.42316640104582) bank185046 +185047 POINT(36.95683328648535 -121.55942836036941) bank185047 +185048 POINT(37.7486001488801 -122.55143006289138) bank185048 +185049 POINT(38.157704441955474 -122.91677964521605) bank185049 +185050 POINT(37.80014929499142 -121.9931702269875) bank185050 +185051 POINT(37.73640748475918 -122.35667032422552) bank185051 +185052 POINT(38.17528243128858 -122.6160665062256) bank185052 +185053 POINT(37.98941693015594 -122.61145671851868) bank185053 +185054 POINT(37.24213966731956 -122.49779989887905) bank185054 +185055 POINT(38.46216185811056 -123.20947784977687) bank185055 +185056 POINT(38.698615394044474 -121.78956753165933) bank185056 +185057 POINT(38.55137967667086 -121.5289281980921) bank185057 +185058 POINT(36.79214647220658 -123.18483204014215) bank185058 +185059 POINT(37.395039803082994 -121.8673999283939) bank185059 +185060 POINT(37.13031419882945 -123.06938128282411) bank185060 +185061 POINT(38.65803386384663 -121.56214981559889) bank185061 +185062 POINT(37.75992708604037 -121.77628494776046) bank185062 +185063 POINT(38.70690326375 -122.08614536023167) bank185063 +185064 POINT(38.27406455549806 -122.2244464379957) bank185064 +185065 POINT(38.28138223907307 -121.48476327061516) bank185065 +185066 POINT(36.89065088912966 -122.33367028039373) bank185066 +185067 POINT(38.43704739327838 -122.47049023081622) bank185067 +185068 POINT(37.672236631710284 -122.50295834933287) bank185068 +185069 POINT(37.09753345111133 -122.17271677492894) bank185069 +185070 POINT(36.88701753446474 -121.58432467860453) bank185070 +185071 POINT(37.66542740362841 -121.79622391377731) bank185071 +185072 POINT(38.130147765375035 -122.12751311010507) bank185072 +185073 POINT(37.34465314428239 -122.10098461309559) bank185073 +185074 POINT(37.94610815602446 -122.03653192607707) bank185074 +185075 POINT(38.73424937404225 -122.29186873422638) bank185075 +185076 POINT(38.28409881977893 -122.13392348017419) bank185076 +185077 POINT(38.32145408775398 -121.48052468330904) bank185077 +185078 POINT(38.389137758866575 -121.82395979169641) bank185078 +185079 POINT(38.47398492893046 -123.26129202886376) bank185079 +185080 POINT(38.69377653647718 -122.13498913494368) bank185080 +185081 POINT(38.477751932277585 -122.75761457016088) bank185081 +185082 POINT(37.96811750190582 -123.22333775180145) bank185082 +185083 POINT(37.83196334736096 -121.75396900505419) bank185083 +185084 POINT(38.194261965745916 -121.77104948752913) bank185084 +185085 POINT(38.19013561053697 -122.3812057092116) bank185085 +185086 POINT(36.83186091985564 -121.50597965043745) bank185086 +185087 POINT(37.33184316490285 -122.31954046721849) bank185087 +185088 POINT(36.98671073670513 -121.99286010912728) bank185088 +185089 POINT(37.7935043176292 -122.97382751411595) bank185089 +185090 POINT(38.016973987989694 -121.75015883252776) bank185090 +185091 POINT(38.28313409372934 -123.27668130141707) bank185091 +185092 POINT(36.89342168522192 -121.5776751158212) bank185092 +185093 POINT(37.79592370445395 -122.92100638394564) bank185093 +185094 POINT(38.403228586157 -121.74710675228724) bank185094 +185095 POINT(36.95995347534034 -123.17995021575636) bank185095 +185096 POINT(37.39185002753844 -122.03512091196505) bank185096 +185097 POINT(38.24627493573607 -121.51680397446732) bank185097 +185098 POINT(37.00830043405357 -123.03519998468514) bank185098 +185099 POINT(38.20629168148171 -121.80389533403205) bank185099 +185100 POINT(38.43396174710602 -121.63027559370273) bank185100 +185101 POINT(38.75817123929963 -122.82234970510993) bank185101 +185102 POINT(37.88223812920343 -122.92931614536798) bank185102 +185103 POINT(38.39688896912972 -122.60805940110161) bank185103 +185104 POINT(37.483913441736284 -122.26700146445008) bank185104 +185105 POINT(36.81427594025201 -121.55060681593835) bank185105 +185106 POINT(37.67302127791028 -122.23030124019229) bank185106 +185107 POINT(36.92643954827075 -122.73033813036416) bank185107 +185108 POINT(37.45668091290087 -121.64409367550871) bank185108 +185109 POINT(36.98392674707453 -123.37111099614799) bank185109 +185110 POINT(38.17267318844227 -121.50958820286873) bank185110 +185111 POINT(37.976479622890444 -121.72639804049403) bank185111 +185112 POINT(37.51173261631169 -121.99897976657559) bank185112 +185113 POINT(37.09091204768221 -121.76243297748414) bank185113 +185114 POINT(37.53756983392922 -122.00315859082235) bank185114 +185115 POINT(37.94692704101429 -123.18210910932531) bank185115 +185116 POINT(36.803948959738506 -123.25105664588692) bank185116 +185117 POINT(38.21708150628933 -123.15337797490925) bank185117 +185118 POINT(36.886260563079006 -123.37209924905719) bank185118 +185119 POINT(38.469868900276865 -122.14326409896158) bank185119 +185120 POINT(38.02125905847201 -122.14036815151937) bank185120 +185121 POINT(38.76218594388651 -122.58782113670708) bank185121 +185122 POINT(38.53654296265646 -123.12501066767956) bank185122 +185123 POINT(38.3552645431426 -122.78450418747212) bank185123 +185124 POINT(37.72895145895422 -122.60022802711198) bank185124 +185125 POINT(38.49246562198049 -123.01926939143387) bank185125 +185126 POINT(38.2064380372323 -123.00689750455851) bank185126 +185127 POINT(37.105763586470225 -122.39605643510178) bank185127 +185128 POINT(37.70355125253385 -121.70919639185193) bank185128 +185129 POINT(38.330936642881 -122.73821470164748) bank185129 +185130 POINT(38.34377012416988 -123.18076208700865) bank185130 +185131 POINT(38.00298560926949 -123.06608442031383) bank185131 +185132 POINT(37.19044030795533 -122.92805410715017) bank185132 +185133 POINT(38.43054641103178 -123.154327564333) bank185133 +185134 POINT(37.42691980473331 -121.42865178631004) bank185134 +185135 POINT(38.56779597467074 -122.4050664100642) bank185135 +185136 POINT(38.21955634440194 -122.38265537167781) bank185136 +185137 POINT(37.19268910920406 -121.47509077601619) bank185137 +185138 POINT(37.37634812954174 -122.24196204080879) bank185138 +185139 POINT(38.28993193438668 -121.95556639597922) bank185139 +185140 POINT(38.43163760454668 -123.10757727022111) bank185140 +185141 POINT(37.14259401200237 -122.98171516524016) bank185141 +185142 POINT(37.60056863125626 -121.8389952495918) bank185142 +185143 POINT(37.4116731006123 -122.72829916153606) bank185143 +185144 POINT(37.43069394011681 -123.04346131415109) bank185144 +185145 POINT(37.76292720094509 -121.4748655879524) bank185145 +185146 POINT(36.85999865203625 -123.2834248865305) bank185146 +185147 POINT(36.7766259002093 -123.30719716623264) bank185147 +185148 POINT(38.20691809917882 -123.08999831410642) bank185148 +185149 POINT(37.70008695872233 -123.01224856808943) bank185149 +185150 POINT(38.66993279530703 -122.29881290775724) bank185150 +185151 POINT(38.73346973806219 -122.54868688455096) bank185151 +185152 POINT(36.855562222162774 -123.34144202252705) bank185152 +185153 POINT(36.825107548909514 -122.91693269371376) bank185153 +185154 POINT(38.5939461979082 -122.69814569975323) bank185154 +185155 POINT(37.31243043828127 -122.2062722665309) bank185155 +185156 POINT(37.02627128414006 -122.39788948467137) bank185156 +185157 POINT(38.20950290792834 -121.50541283874736) bank185157 +185158 POINT(37.79943713633649 -122.84154666227954) bank185158 +185159 POINT(37.95611988635603 -121.93397797607835) bank185159 +185160 POINT(38.73558753260294 -122.1438469596673) bank185160 +185161 POINT(36.8240671316566 -122.36524642242723) bank185161 +185162 POINT(37.691801127411566 -122.6144149319362) bank185162 +185163 POINT(38.557671083975876 -123.37943999311483) bank185163 +185164 POINT(37.06193375771843 -123.05200733607006) bank185164 +185165 POINT(37.292227050286364 -122.92809239084032) bank185165 +185166 POINT(36.86370354227829 -121.55676537708877) bank185166 +185167 POINT(36.89544599770823 -122.92550451852345) bank185167 +185168 POINT(37.5048502976339 -123.09826624522742) bank185168 +185169 POINT(37.93573633099896 -123.37526337574614) bank185169 +185170 POINT(38.51180921315853 -122.74250065072852) bank185170 +185171 POINT(37.69450073904112 -122.50744163554265) bank185171 +185172 POINT(38.14643540038443 -123.31024703061317) bank185172 +185173 POINT(37.33778653181623 -121.4330309558993) bank185173 +185174 POINT(37.65717089286299 -123.04259011138372) bank185174 +185175 POINT(37.69839655022035 -122.4831297771319) bank185175 +185176 POINT(38.46270109871825 -123.31242131344035) bank185176 +185177 POINT(38.294630718992885 -122.8484742591937) bank185177 +185178 POINT(37.03887322881972 -121.91294079977285) bank185178 +185179 POINT(37.682618465287675 -122.13876256458619) bank185179 +185180 POINT(37.8083318468698 -122.22542093161304) bank185180 +185181 POINT(36.821411738880194 -122.89481944245681) bank185181 +185182 POINT(37.015519484885964 -121.95634288151408) bank185182 +185183 POINT(37.74805714009578 -122.54958133391645) bank185183 +185184 POINT(38.36222067670787 -122.92210988560458) bank185184 +185185 POINT(38.05292179939809 -122.23165690599046) bank185185 +185186 POINT(37.74567622464529 -122.15665879055233) bank185186 +185187 POINT(38.025737976573346 -122.62086953772322) bank185187 +185188 POINT(37.4735917094497 -123.27398113395394) bank185188 +185189 POINT(37.550724730316595 -121.74729634803019) bank185189 +185190 POINT(37.177069782573014 -121.92821573436893) bank185190 +185191 POINT(38.638941292359426 -122.39094211002238) bank185191 +185192 POINT(36.83829515957891 -123.28307212992286) bank185192 +185193 POINT(38.37822900547443 -123.31984425085898) bank185193 +185194 POINT(38.612173494654044 -123.17224993216047) bank185194 +185195 POINT(36.93306545942747 -122.41285296325266) bank185195 +185196 POINT(37.58798307983946 -123.07073194715842) bank185196 +185197 POINT(37.76091539315591 -122.26221634135537) bank185197 +185198 POINT(38.42345981183655 -122.17229882067804) bank185198 +185199 POINT(38.00216735120639 -122.24927902168231) bank185199 +185200 POINT(38.384072427324305 -122.63372501296348) bank185200 +185201 POINT(38.53149021844364 -122.29001248957002) bank185201 +185202 POINT(37.54744906445041 -121.82678795768295) bank185202 +185203 POINT(37.908364713104156 -121.57924374572148) bank185203 +185204 POINT(37.76838751859555 -122.81267599564816) bank185204 +185205 POINT(37.75299886077821 -121.87382427740745) bank185205 +185206 POINT(38.74401677597619 -122.90067556676163) bank185206 +185207 POINT(38.722844003108975 -121.42179151086607) bank185207 +185208 POINT(37.878663736056566 -122.55733830033056) bank185208 +185209 POINT(38.42167646307152 -123.13754075314608) bank185209 +185210 POINT(38.098280922472206 -121.70627202397277) bank185210 +185211 POINT(37.24026957522572 -122.46297595147294) bank185211 +185212 POINT(38.62025831020515 -122.703376769908) bank185212 +185213 POINT(37.611195113983925 -122.99964663604803) bank185213 +185214 POINT(38.41760006036739 -122.94458227754936) bank185214 +185215 POINT(37.80141476062381 -123.1433238325684) bank185215 +185216 POINT(37.272781509094884 -122.2358915662821) bank185216 +185217 POINT(37.68676786644872 -121.69971338579762) bank185217 +185218 POINT(37.5690919213491 -122.53243215772812) bank185218 +185219 POINT(38.20426588812219 -121.67871249383178) bank185219 +185220 POINT(38.36112080867258 -122.64240486037072) bank185220 +185221 POINT(38.33803436769061 -121.83764944491128) bank185221 +185222 POINT(38.33505555020192 -123.2531534659451) bank185222 +185223 POINT(37.6188456535057 -121.73238283910986) bank185223 +185224 POINT(37.607261520837156 -122.87587127321342) bank185224 +185225 POINT(37.61406944808381 -121.89011882544352) bank185225 +185226 POINT(38.37143464747858 -122.4179250009465) bank185226 +185227 POINT(38.35742947926266 -121.52820250867974) bank185227 +185228 POINT(37.98050697091031 -123.00264579414575) bank185228 +185229 POINT(38.26796823962411 -122.20547194488381) bank185229 +185230 POINT(36.98599076529131 -122.67172663571681) bank185230 +185231 POINT(36.92134256254959 -122.69120088501107) bank185231 +185232 POINT(37.59059900022779 -123.26847899302045) bank185232 +185233 POINT(38.72126333519025 -121.75933515051472) bank185233 +185234 POINT(36.84905285803176 -123.07616396777615) bank185234 +185235 POINT(38.70663125677892 -123.04177186739433) bank185235 +185236 POINT(36.82642257016607 -122.79956820604396) bank185236 +185237 POINT(38.66378557828351 -122.46671732509182) bank185237 +185238 POINT(38.56652328565155 -123.20984882068436) bank185238 +185239 POINT(37.49013564825818 -122.96459479842733) bank185239 +185240 POINT(38.0601604174015 -122.30294106699353) bank185240 +185241 POINT(38.4903750920507 -123.39572877820426) bank185241 +185242 POINT(38.672920775247086 -122.72159581088036) bank185242 +185243 POINT(36.983382710459495 -122.06761903133871) bank185243 +185244 POINT(38.43514372834491 -123.27600008081765) bank185244 +185245 POINT(37.19579714671728 -122.97518652122038) bank185245 +185246 POINT(38.52800313428606 -121.69740162244797) bank185246 +185247 POINT(37.09167801797782 -121.46693398712273) bank185247 +185248 POINT(38.59563309008586 -122.94599833025345) bank185248 +185249 POINT(38.06326149577341 -121.62329773731143) bank185249 +185250 POINT(37.43394337895775 -121.83641395551149) bank185250 +185251 POINT(37.72491202097708 -121.47281084473914) bank185251 +185252 POINT(37.184181588503584 -122.59049131776104) bank185252 +185253 POINT(38.32812862652565 -122.23481913525362) bank185253 +185254 POINT(38.7575785133633 -122.04211939407543) bank185254 +185255 POINT(38.387327388454686 -121.42640121125407) bank185255 +185256 POINT(37.04087659412583 -122.44649095479623) bank185256 +185257 POINT(37.19509961950825 -123.2893982012698) bank185257 +185258 POINT(37.99881521428189 -121.46424437541678) bank185258 +185259 POINT(37.77258673738117 -122.51420848389822) bank185259 +185260 POINT(37.87216197395519 -123.12953540511317) bank185260 +185261 POINT(38.086561342302225 -122.56206932916989) bank185261 +185262 POINT(37.8583925728018 -121.446460215111) bank185262 +185263 POINT(37.84310088395086 -122.93986257397927) bank185263 +185264 POINT(37.6794151480482 -122.98614914124391) bank185264 +185265 POINT(38.249748963181595 -122.94597249786464) bank185265 +185266 POINT(38.21450991492594 -122.32069587161388) bank185266 +185267 POINT(38.74807521754113 -121.77675271590175) bank185267 +185268 POINT(38.0361431209387 -123.07400335407819) bank185268 +185269 POINT(37.5931427936141 -121.71079098636939) bank185269 +185270 POINT(38.26000686263317 -122.47122545141642) bank185270 +185271 POINT(38.65625789910431 -122.67277339108882) bank185271 +185272 POINT(38.664146137435324 -121.94274816192463) bank185272 +185273 POINT(37.49905239511169 -122.9467880823292) bank185273 +185274 POINT(37.36360652897498 -122.68546189221976) bank185274 +185275 POINT(38.06925991098206 -122.70538999919836) bank185275 +185276 POINT(37.62527437332096 -121.899332056319) bank185276 +185277 POINT(37.95319443153271 -122.30720483367459) bank185277 +185278 POINT(37.594786715945844 -122.33946994637871) bank185278 +185279 POINT(37.00798027600578 -122.29305779488303) bank185279 +185280 POINT(37.749628236793995 -122.90343570569789) bank185280 +185281 POINT(37.97717323477524 -121.75953181705685) bank185281 +185282 POINT(36.860937125703586 -122.82653888087974) bank185282 +185283 POINT(37.14393672569769 -121.82884888861773) bank185283 +185284 POINT(37.04389106413574 -121.7615897787686) bank185284 +185285 POINT(37.309055355193216 -121.73945392369511) bank185285 +185286 POINT(37.893749975514886 -122.65352478030519) bank185286 +185287 POINT(37.28163287318288 -121.99946963344426) bank185287 +185288 POINT(38.09828368926961 -121.50763055216136) bank185288 +185289 POINT(38.516845335944204 -121.90804247889427) bank185289 +185290 POINT(38.749058731757174 -121.5168451862516) bank185290 +185291 POINT(37.904168532082295 -121.84635391292446) bank185291 +185292 POINT(37.865761969749855 -122.20568502069054) bank185292 +185293 POINT(37.5239516725678 -122.50232826095665) bank185293 +185294 POINT(37.793048808929406 -121.90808030239144) bank185294 +185295 POINT(38.39920124975258 -122.02745470566907) bank185295 +185296 POINT(38.00811839466614 -123.09755318237096) bank185296 +185297 POINT(37.38504013755133 -121.46368701736043) bank185297 +185298 POINT(36.97298094774464 -121.76464581596365) bank185298 +185299 POINT(37.618306465172516 -121.92295978992777) bank185299 +185300 POINT(37.85093182028871 -123.38312803843344) bank185300 +185301 POINT(38.3984837999638 -122.93121028536254) bank185301 +185302 POINT(37.58803235468257 -122.48424554928314) bank185302 +185303 POINT(38.68593252366197 -121.45730324850628) bank185303 +185304 POINT(38.02621895219129 -122.40854365308178) bank185304 +185305 POINT(37.57464322747687 -121.58151479600558) bank185305 +185306 POINT(37.48848637365396 -123.01309842964977) bank185306 +185307 POINT(38.53954479873162 -122.58498545029667) bank185307 +185308 POINT(38.62512131324822 -122.93744136179866) bank185308 +185309 POINT(38.6563724589905 -121.65525457734044) bank185309 +185310 POINT(38.45084374315613 -122.19246905730364) bank185310 +185311 POINT(38.18978467223373 -122.46585931516779) bank185311 +185312 POINT(38.29800752939514 -121.53392998752517) bank185312 +185313 POINT(38.0755931579739 -122.66472950188314) bank185313 +185314 POINT(36.92381728832191 -121.93596052095043) bank185314 +185315 POINT(37.02916552930305 -122.82900162839898) bank185315 +185316 POINT(38.69373740162506 -122.14431289069438) bank185316 +185317 POINT(38.108427958602 -122.27076382461945) bank185317 +185318 POINT(37.69691539214815 -121.7340957573051) bank185318 +185319 POINT(37.15315614842093 -121.84644431248554) bank185319 +185320 POINT(38.71025529783779 -121.99824280989439) bank185320 +185321 POINT(38.36481422702546 -122.79756529238061) bank185321 +185322 POINT(38.165729574377714 -122.21918707152803) bank185322 +185323 POINT(37.770325753519074 -121.91251713207231) bank185323 +185324 POINT(37.02326103177096 -121.74471132852817) bank185324 +185325 POINT(37.902183237051524 -122.94687149761941) bank185325 +185326 POINT(37.049778213585434 -122.02710706141985) bank185326 +185327 POINT(36.794351953312386 -122.1211134752379) bank185327 +185328 POINT(38.42400559392848 -122.69070273968768) bank185328 +185329 POINT(37.7283064667295 -123.40553043948896) bank185329 +185330 POINT(37.518233629838655 -123.38089113626636) bank185330 +185331 POINT(37.37873945667044 -122.0598943080047) bank185331 +185332 POINT(37.02321011710725 -122.09472119170731) bank185332 +185333 POINT(37.894999878812776 -122.2141652125597) bank185333 +185334 POINT(38.74936462431325 -121.82723312141829) bank185334 +185335 POINT(37.03716596802065 -122.46594963040673) bank185335 +185336 POINT(38.64098626874382 -122.66289061133023) bank185336 +185337 POINT(38.082166894131696 -121.71202359954438) bank185337 +185338 POINT(38.11179132377959 -122.14547753367772) bank185338 +185339 POINT(37.60438209486499 -121.6272555734074) bank185339 +185340 POINT(38.758178738309915 -122.0094643680827) bank185340 +185341 POINT(37.526960870436625 -121.72519372063793) bank185341 +185342 POINT(38.35547921161094 -122.64733413479291) bank185342 +185343 POINT(37.605743114292096 -121.63861016484381) bank185343 +185344 POINT(38.48039825338974 -122.93626403661011) bank185344 +185345 POINT(36.853615745451236 -122.8617598057465) bank185345 +185346 POINT(37.85726027827318 -122.47966068496669) bank185346 +185347 POINT(37.795643766978216 -122.79208592575307) bank185347 +185348 POINT(37.948129201768765 -123.29531181703395) bank185348 +185349 POINT(36.82717874266658 -123.28092126352007) bank185349 +185350 POINT(37.71635609667657 -121.78272257445828) bank185350 +185351 POINT(37.159999968877834 -123.31262462488004) bank185351 +185352 POINT(37.673859893283485 -122.63154615800008) bank185352 +185353 POINT(37.01551355664472 -121.96987245777811) bank185353 +185354 POINT(38.71166091141242 -121.79252288054779) bank185354 +185355 POINT(37.652318267531975 -121.87780917910699) bank185355 +185356 POINT(38.64168781172646 -122.67455776673029) bank185356 +185357 POINT(37.676123994056226 -122.3613079207752) bank185357 +185358 POINT(37.279923198887005 -123.15693186591137) bank185358 +185359 POINT(38.26920522394449 -123.023119269943) bank185359 +185360 POINT(37.66294409614175 -122.83254491055551) bank185360 +185361 POINT(37.581113130343034 -121.94144610579595) bank185361 +185362 POINT(38.1333583333972 -123.29828064340177) bank185362 +185363 POINT(37.05920868904594 -122.341440260634) bank185363 +185364 POINT(37.54424848433259 -122.64262183566956) bank185364 +185365 POINT(38.41299785253928 -122.71023461626145) bank185365 +185366 POINT(37.402540176327356 -121.74795416503707) bank185366 +185367 POINT(38.77349387689734 -122.93048980106354) bank185367 +185368 POINT(38.657739974037774 -122.93228551694241) bank185368 +185369 POINT(38.492158633484415 -122.05033937178862) bank185369 +185370 POINT(38.18580790829486 -122.95783834995783) bank185370 +185371 POINT(37.47394683022866 -122.60244625321363) bank185371 +185372 POINT(37.80631202633965 -122.5054117254713) bank185372 +185373 POINT(37.34766272478248 -121.76521667762165) bank185373 +185374 POINT(38.08147994596667 -122.42685785455903) bank185374 +185375 POINT(36.92205016081783 -121.73665065071718) bank185375 +185376 POINT(37.39121556638521 -123.04319518003616) bank185376 +185377 POINT(38.0495151033817 -122.59052413769744) bank185377 +185378 POINT(38.594556266206126 -122.49013213285946) bank185378 +185379 POINT(38.71576576527287 -122.85597114139874) bank185379 +185380 POINT(38.730646504807105 -122.19551389467844) bank185380 +185381 POINT(37.882042719636964 -122.12940591048928) bank185381 +185382 POINT(38.43768053418261 -122.4976533421969) bank185382 +185383 POINT(37.89554097156347 -123.21393422347754) bank185383 +185384 POINT(36.87080735293198 -122.84861884596576) bank185384 +185385 POINT(38.02843131258657 -122.11118645872855) bank185385 +185386 POINT(37.7354165026539 -122.20138631688695) bank185386 +185387 POINT(37.7326991787354 -121.54835632699421) bank185387 +185388 POINT(38.30000192495776 -122.6919534643971) bank185388 +185389 POINT(37.624240747003746 -122.49715585088217) bank185389 +185390 POINT(37.244059452574874 -121.83727245425334) bank185390 +185391 POINT(37.65311055527839 -121.84700369799555) bank185391 +185392 POINT(37.78925432941687 -121.9065241090537) bank185392 +185393 POINT(37.86250761927466 -122.58709564625549) bank185393 +185394 POINT(37.82697965107205 -121.89545311662363) bank185394 +185395 POINT(36.96349172649505 -123.10841334395359) bank185395 +185396 POINT(38.375942974211455 -122.13763196786446) bank185396 +185397 POINT(38.30619696772094 -123.13032130673963) bank185397 +185398 POINT(37.239259795275956 -122.56182955515825) bank185398 +185399 POINT(36.79872257128058 -122.61137106400211) bank185399 +185400 POINT(37.84359293729895 -122.0063871858692) bank185400 +185401 POINT(38.046002376018386 -122.0297593031313) bank185401 +185402 POINT(36.849488285909565 -122.57198685453277) bank185402 +185403 POINT(37.76650557691024 -123.32764274195924) bank185403 +185404 POINT(38.213515842699664 -122.97234666860115) bank185404 +185405 POINT(37.18871611648954 -121.82523581056314) bank185405 +185406 POINT(37.39003708270316 -122.06651926350779) bank185406 +185407 POINT(37.77161378839617 -121.50763562517888) bank185407 +185408 POINT(37.04278414123487 -123.12212972520514) bank185408 +185409 POINT(37.16940520795212 -122.38112814923744) bank185409 +185410 POINT(37.44650869633284 -123.30485480881953) bank185410 +185411 POINT(38.69898681787493 -122.67237618850218) bank185411 +185412 POINT(37.15167860089054 -123.11776906636634) bank185412 +185413 POINT(38.08207075666763 -123.39205283554126) bank185413 +185414 POINT(38.11487913402591 -121.79461036618775) bank185414 +185415 POINT(37.02843269425527 -122.6123548804175) bank185415 +185416 POINT(38.582036035140476 -122.03790110249709) bank185416 +185417 POINT(38.01895849410573 -123.33726988180065) bank185417 +185418 POINT(37.43140506628966 -121.7869856400174) bank185418 +185419 POINT(37.328899429900545 -122.97640257821243) bank185419 +185420 POINT(36.913417228726175 -121.93961843450663) bank185420 +185421 POINT(38.36689199870823 -123.12690795680103) bank185421 +185422 POINT(37.29725211226872 -122.58736315334455) bank185422 +185423 POINT(37.08560688883192 -121.4196167616662) bank185423 +185424 POINT(36.87896477820009 -123.20209841210179) bank185424 +185425 POINT(37.8573020527528 -123.07296827090047) bank185425 +185426 POINT(37.77279180685883 -122.67427257186344) bank185426 +185427 POINT(38.05986584961321 -123.14972820910877) bank185427 +185428 POINT(37.971219901932464 -121.84902753230114) bank185428 +185429 POINT(38.15362392886412 -121.8707503183412) bank185429 +185430 POINT(38.534320326571596 -121.806219633454) bank185430 +185431 POINT(37.0687354063642 -123.33643264427214) bank185431 +185432 POINT(38.35159995480103 -121.5888605611708) bank185432 +185433 POINT(38.26817853561824 -122.25699150717915) bank185433 +185434 POINT(37.48997832739977 -121.82920248198558) bank185434 +185435 POINT(37.84680496391906 -121.96400148794244) bank185435 +185436 POINT(38.47858897140106 -121.56941025798213) bank185436 +185437 POINT(37.146174587223285 -121.82091630058743) bank185437 +185438 POINT(38.26506773196144 -121.68479187207369) bank185438 +185439 POINT(37.76868698408985 -122.09314223242019) bank185439 +185440 POINT(37.303082623141286 -123.20356804692277) bank185440 +185441 POINT(36.86961710523479 -123.07658163634967) bank185441 +185442 POINT(37.26324245150725 -122.34704635163094) bank185442 +185443 POINT(37.49017074641794 -122.98468442223316) bank185443 +185444 POINT(37.11508137865234 -123.1541330610164) bank185444 +185445 POINT(36.97376982662464 -121.87282134663597) bank185445 +185446 POINT(37.285056790432506 -123.17415434655412) bank185446 +185447 POINT(38.25468445693779 -123.24284168226312) bank185447 +185448 POINT(38.70738902074045 -122.10739399560852) bank185448 +185449 POINT(37.97767580512096 -121.87694329234327) bank185449 +185450 POINT(38.241113261239214 -122.04524792751445) bank185450 +185451 POINT(38.582987572537654 -122.63499751678934) bank185451 +185452 POINT(37.33883365491848 -121.72540285426452) bank185452 +185453 POINT(38.72736689703318 -122.14408629558768) bank185453 +185454 POINT(38.68673383669663 -122.5991961922931) bank185454 +185455 POINT(37.68551503247969 -121.74240099341485) bank185455 +185456 POINT(37.75101618535552 -123.12292510823369) bank185456 +185457 POINT(38.22503652307162 -123.09413565321752) bank185457 +185458 POINT(37.08369563129373 -122.70496883344755) bank185458 +185459 POINT(36.78307745831232 -122.59738706039602) bank185459 +185460 POINT(37.17830786990421 -121.60164221134386) bank185460 +185461 POINT(38.2525189121362 -123.20742501474083) bank185461 +185462 POINT(37.28730136576707 -121.90474742947131) bank185462 +185463 POINT(36.79382157568997 -122.68467504435817) bank185463 +185464 POINT(37.65835103604672 -121.78714439566804) bank185464 +185465 POINT(38.596443599567806 -122.8532548505319) bank185465 +185466 POINT(37.148248902445594 -122.82161398424448) bank185466 +185467 POINT(37.309066279852814 -121.90910303429514) bank185467 +185468 POINT(38.24109225192237 -121.4330330005509) bank185468 +185469 POINT(37.253098543699195 -121.94984268633364) bank185469 +185470 POINT(37.22599411788827 -122.88359928501954) bank185470 +185471 POINT(37.23997919107763 -122.10357284657434) bank185471 +185472 POINT(37.37448750210763 -123.06171760426426) bank185472 +185473 POINT(37.10122923499966 -122.26354468935168) bank185473 +185474 POINT(37.743014953723986 -123.3841865706652) bank185474 +185475 POINT(38.25568955282736 -122.29853144215036) bank185475 +185476 POINT(37.5447045164826 -123.09422715718037) bank185476 +185477 POINT(37.659632890931555 -121.85363094624613) bank185477 +185478 POINT(37.280629944540124 -123.35503634728703) bank185478 +185479 POINT(38.32486364777333 -122.3566131487421) bank185479 +185480 POINT(37.29089569001985 -123.33863906039441) bank185480 +185481 POINT(36.924508482654424 -122.3674751339574) bank185481 +185482 POINT(38.465979036620745 -121.9182632608432) bank185482 +185483 POINT(37.787174455771805 -122.16516856647472) bank185483 +185484 POINT(38.2078600724974 -121.45292042514578) bank185484 +185485 POINT(37.83974569223248 -122.5173857803774) bank185485 +185486 POINT(37.48454935611508 -123.36223025803667) bank185486 +185487 POINT(37.98742925331945 -122.11011332281295) bank185487 +185488 POINT(37.03056890114296 -123.38136177695455) bank185488 +185489 POINT(38.24749838661269 -122.09851637133141) bank185489 +185490 POINT(37.68025224915572 -122.35396200983458) bank185490 +185491 POINT(38.38142404638074 -121.68786629323961) bank185491 +185492 POINT(36.906458427414805 -122.40490048631642) bank185492 +185493 POINT(38.74719323013073 -122.38802458684151) bank185493 +185494 POINT(37.519724496249694 -122.07443412835211) bank185494 +185495 POINT(38.39185418347287 -121.94541894077098) bank185495 +185496 POINT(38.36853279173329 -122.20934022896272) bank185496 +185497 POINT(38.54343725978475 -121.95653658924718) bank185497 +185498 POINT(37.65416167429384 -123.1661548933478) bank185498 +185499 POINT(37.89778277013221 -121.72325811786092) bank185499 +185500 POINT(38.30097267299453 -122.92584840863107) bank185500 +185501 POINT(37.081704874979295 -121.46155669763289) bank185501 +185502 POINT(38.354223357182335 -123.32853189238803) bank185502 +185503 POINT(38.56513023625162 -121.81168820584281) bank185503 +185504 POINT(38.64245997092482 -122.99347004754223) bank185504 +185505 POINT(38.621956360167985 -122.01397494206148) bank185505 +185506 POINT(36.88501500487256 -122.52744814917362) bank185506 +185507 POINT(38.42614417181508 -123.08540878876875) bank185507 +185508 POINT(36.911643253190284 -122.20065699087917) bank185508 +185509 POINT(37.763802750679524 -123.0054505782844) bank185509 +185510 POINT(37.25061572360417 -122.64087710138783) bank185510 +185511 POINT(37.33000352626926 -122.59068994066055) bank185511 +185512 POINT(38.680372073572975 -121.93688632240247) bank185512 +185513 POINT(36.84403865654111 -121.92344898332681) bank185513 +185514 POINT(36.81092783776521 -122.08795788089937) bank185514 +185515 POINT(37.2217709706072 -122.04583231178344) bank185515 +185516 POINT(38.57805355257044 -123.33488984334643) bank185516 +185517 POINT(37.757840136245065 -122.37430210794922) bank185517 +185518 POINT(38.03678123485777 -123.06019566404848) bank185518 +185519 POINT(37.73559513815993 -122.42861477123778) bank185519 +185520 POINT(37.54957565188461 -121.83758829173955) bank185520 +185521 POINT(37.62615712802138 -122.01732739928565) bank185521 +185522 POINT(37.65599898992649 -122.79225654646682) bank185522 +185523 POINT(38.56386665048035 -122.48029231121727) bank185523 +185524 POINT(37.65264956410728 -123.23883529961454) bank185524 +185525 POINT(37.15269221911169 -122.60826676180162) bank185525 +185526 POINT(38.17057040835803 -122.55605630074908) bank185526 +185527 POINT(37.8165301038683 -121.76800078387686) bank185527 +185528 POINT(36.80558487417249 -121.66713418678256) bank185528 +185529 POINT(37.77890037558002 -122.59132583800071) bank185529 +185530 POINT(37.53955874479991 -123.03590260931871) bank185530 +185531 POINT(38.54422270914917 -122.96966031041535) bank185531 +185532 POINT(37.500474570656344 -122.34367173534648) bank185532 +185533 POINT(37.95861211514434 -123.16279565099003) bank185533 +185534 POINT(38.77232895151139 -122.33764666876056) bank185534 +185535 POINT(38.19184120747376 -122.186293357956) bank185535 +185536 POINT(37.1673710360857 -121.50755966945415) bank185536 +185537 POINT(38.14989903033365 -122.51210811257164) bank185537 +185538 POINT(37.87145519967745 -123.34196730119672) bank185538 +185539 POINT(38.6130903808151 -122.41915225544578) bank185539 +185540 POINT(37.61350313585604 -122.1584394720073) bank185540 +185541 POINT(38.5022341140517 -122.00537192079103) bank185541 +185542 POINT(37.38366100841125 -122.0047843575705) bank185542 +185543 POINT(38.520110234127785 -123.31990783619548) bank185543 +185544 POINT(37.583563989988015 -121.57898881240664) bank185544 +185545 POINT(38.65985726734162 -122.46173524293363) bank185545 +185546 POINT(37.83285296856887 -122.19128215669046) bank185546 +185547 POINT(36.84824523370529 -122.48174862769258) bank185547 +185548 POINT(37.248587581454004 -121.68286637385837) bank185548 +185549 POINT(37.599817094445385 -122.47993953660777) bank185549 +185550 POINT(36.775714603760505 -122.63544010074439) bank185550 +185551 POINT(36.77742435613421 -121.5863359275941) bank185551 +185552 POINT(37.58499680896977 -122.72519097915773) bank185552 +185553 POINT(36.965576314316145 -123.26466094361726) bank185553 +185554 POINT(38.65558792721512 -122.4223347067412) bank185554 +185555 POINT(38.71882646705798 -121.61999684593208) bank185555 +185556 POINT(38.29495208014073 -122.9363724910596) bank185556 +185557 POINT(38.5264063941066 -122.39973857211862) bank185557 +185558 POINT(37.311531700763794 -121.55216988248492) bank185558 +185559 POINT(37.8219977155252 -121.57753067919184) bank185559 +185560 POINT(37.74948569682666 -121.8741161160096) bank185560 +185561 POINT(37.43627950422997 -123.18270499605318) bank185561 +185562 POINT(38.51564026855963 -121.42607639520699) bank185562 +185563 POINT(37.270169396467665 -123.36456712980893) bank185563 +185564 POINT(38.38702926853812 -122.29668434868219) bank185564 +185565 POINT(37.557532888612585 -122.32120014783044) bank185565 +185566 POINT(37.712688381382314 -121.74860369446787) bank185566 +185567 POINT(38.13875473001509 -122.28531332717374) bank185567 +185568 POINT(37.80686147735841 -123.23590235210551) bank185568 +185569 POINT(38.1338963050363 -121.76977863478459) bank185569 +185570 POINT(38.682513365644056 -123.12563806320617) bank185570 +185571 POINT(37.94216439064651 -123.0156354299547) bank185571 +185572 POINT(37.80792820947455 -122.7616751847854) bank185572 +185573 POINT(38.29687284627712 -121.69507020694759) bank185573 +185574 POINT(37.86943901551606 -121.43793394465744) bank185574 +185575 POINT(38.54161814837502 -122.19686134179017) bank185575 +185576 POINT(38.62757785419446 -123.34673346628583) bank185576 +185577 POINT(38.4462575623498 -122.64356928831609) bank185577 +185578 POINT(37.84383976742796 -122.66506243536146) bank185578 +185579 POINT(38.24640520386879 -122.0992993098333) bank185579 +185580 POINT(37.43328287387973 -122.01663067357272) bank185580 +185581 POINT(38.46214364187789 -123.30907413525871) bank185581 +185582 POINT(36.84900719690394 -121.8572211695573) bank185582 +185583 POINT(38.49541793254843 -122.8766216737917) bank185583 +185584 POINT(38.5649479279993 -121.68588372298235) bank185584 +185585 POINT(38.45200780106316 -121.57513657439475) bank185585 +185586 POINT(38.445527048311824 -122.73272121759601) bank185586 +185587 POINT(38.18481078721001 -123.16838697092886) bank185587 +185588 POINT(38.22971877018364 -122.17304406524656) bank185588 +185589 POINT(36.85846250151144 -122.46980065432072) bank185589 +185590 POINT(37.04197639405344 -122.64414353955449) bank185590 +185591 POINT(38.12261451780862 -122.59758023218811) bank185591 +185592 POINT(37.443060395612676 -121.7550857878519) bank185592 +185593 POINT(38.43660709432647 -121.99935484003562) bank185593 +185594 POINT(37.51784835867679 -121.5950264122701) bank185594 +185595 POINT(36.92466502874129 -122.14137634614454) bank185595 +185596 POINT(37.79941064028931 -122.24204793411876) bank185596 +185597 POINT(37.72756275846087 -121.82180080482037) bank185597 +185598 POINT(36.795276731306295 -121.73187728921826) bank185598 +185599 POINT(37.586788555843505 -122.22466440087926) bank185599 +185600 POINT(36.87147050511005 -123.21954629769479) bank185600 +185601 POINT(37.73812742588863 -122.91022632429502) bank185601 +185602 POINT(37.00020423634115 -122.59545460645474) bank185602 +185603 POINT(37.60203963348537 -122.9685713636654) bank185603 +185604 POINT(37.33013509674104 -122.3589681330896) bank185604 +185605 POINT(37.22595733242637 -122.82259638384616) bank185605 +185606 POINT(36.82844433715209 -123.04237191407313) bank185606 +185607 POINT(36.91134318525963 -122.50000517090027) bank185607 +185608 POINT(37.12632962962808 -123.15065794927085) bank185608 +185609 POINT(37.131320868430485 -122.64161581261438) bank185609 +185610 POINT(38.354398643924895 -123.37306151663947) bank185610 +185611 POINT(37.58230054531335 -122.04585540492752) bank185611 +185612 POINT(38.05757056291121 -121.75975696839328) bank185612 +185613 POINT(38.5223251638201 -121.967710029983) bank185613 +185614 POINT(37.200015881378725 -122.92213757098857) bank185614 +185615 POINT(37.90670270294682 -122.22451180542474) bank185615 +185616 POINT(36.860187802569556 -121.62291640154345) bank185616 +185617 POINT(37.02259163004029 -121.75734361963038) bank185617 +185618 POINT(36.85222991075503 -121.50125759320919) bank185618 +185619 POINT(38.4388008864717 -122.78404971488173) bank185619 +185620 POINT(37.90243225760257 -122.19318434609247) bank185620 +185621 POINT(38.491649413456706 -122.94399355378586) bank185621 +185622 POINT(38.05813446890419 -122.97695693688827) bank185622 +185623 POINT(37.25242079580951 -122.4269211921946) bank185623 +185624 POINT(37.472399127213436 -122.01810537062447) bank185624 +185625 POINT(38.20918986894773 -121.64194461294272) bank185625 +185626 POINT(37.814164601799725 -122.68591988757926) bank185626 +185627 POINT(37.269764950635306 -121.57292810076088) bank185627 +185628 POINT(38.19401785610112 -122.89502945667525) bank185628 +185629 POINT(38.7744638947841 -123.30694312175267) bank185629 +185630 POINT(38.221069862159204 -122.7031046521955) bank185630 +185631 POINT(37.820285241768495 -121.75365221687359) bank185631 +185632 POINT(36.78815091503307 -122.42176421708326) bank185632 +185633 POINT(37.73964749460293 -122.74274484784107) bank185633 +185634 POINT(38.68445120385221 -122.36404823029636) bank185634 +185635 POINT(38.1779058422562 -122.23122137572157) bank185635 +185636 POINT(37.03475499806291 -122.56159584025723) bank185636 +185637 POINT(38.71827122485405 -123.277752365755) bank185637 +185638 POINT(38.247356614019644 -121.65908915548994) bank185638 +185639 POINT(37.058406612024754 -122.6133586311503) bank185639 +185640 POINT(38.10245317685516 -123.2274412752489) bank185640 +185641 POINT(36.908801500556706 -122.8701062449803) bank185641 +185642 POINT(37.22986947309399 -122.67733411305636) bank185642 +185643 POINT(37.72923240656623 -123.01800501861656) bank185643 +185644 POINT(38.30645684446056 -122.66391267054445) bank185644 +185645 POINT(37.27510613297938 -122.73149597192355) bank185645 +185646 POINT(37.34776839181846 -122.8941122736532) bank185646 +185647 POINT(37.62542050240852 -123.09813860040835) bank185647 +185648 POINT(38.52653019918482 -121.6174530761336) bank185648 +185649 POINT(36.832678466322015 -123.22615582415835) bank185649 +185650 POINT(37.77001438689344 -121.71909932278973) bank185650 +185651 POINT(38.16510189148158 -122.68302279240316) bank185651 +185652 POINT(38.5251780438377 -121.77261900794063) bank185652 +185653 POINT(38.00798276646406 -123.25708095830272) bank185653 +185654 POINT(37.09464972205347 -122.2962831509301) bank185654 +185655 POINT(38.59687315122756 -122.29164615371303) bank185655 +185656 POINT(38.36348231203678 -122.29092499211494) bank185656 +185657 POINT(36.93722606749668 -121.84450929688849) bank185657 +185658 POINT(37.147073640432204 -123.01607902321028) bank185658 +185659 POINT(37.403011600507185 -122.6426546775313) bank185659 +185660 POINT(37.40616648720219 -122.20607137577714) bank185660 +185661 POINT(37.71554455496752 -122.11840953326467) bank185661 +185662 POINT(37.017556764402336 -122.76234697589062) bank185662 +185663 POINT(38.08420324308226 -122.74613962117475) bank185663 +185664 POINT(38.33356664286079 -123.28675836009718) bank185664 +185665 POINT(37.116693562523 -121.78239810181039) bank185665 +185666 POINT(37.776238200061975 -122.25510430251525) bank185666 +185667 POINT(37.002195873127924 -121.49443212361683) bank185667 +185668 POINT(37.449091124793554 -123.08590531706244) bank185668 +185669 POINT(37.56779360848046 -123.33414619743012) bank185669 +185670 POINT(37.62499410509559 -123.31593393177154) bank185670 +185671 POINT(37.67909685332979 -122.80655236691413) bank185671 +185672 POINT(38.75771627103876 -122.77718533271965) bank185672 +185673 POINT(38.725956365860846 -122.49854446704815) bank185673 +185674 POINT(38.09216546318007 -121.88538068736584) bank185674 +185675 POINT(37.690571966283734 -122.60285718006047) bank185675 +185676 POINT(38.31040350107128 -122.34788920036034) bank185676 +185677 POINT(38.684235543530825 -121.57605487385091) bank185677 +185678 POINT(37.79640310758362 -121.46024520455451) bank185678 +185679 POINT(37.49790765262145 -121.49901584709501) bank185679 +185680 POINT(38.00112287583962 -122.28342860558836) bank185680 +185681 POINT(37.59540975774456 -123.25239977154591) bank185681 +185682 POINT(37.36531805050319 -121.60399034847505) bank185682 +185683 POINT(37.2982181953415 -121.54557054992789) bank185683 +185684 POINT(37.708845679232866 -121.7931803407415) bank185684 +185685 POINT(37.71154814849422 -123.18424158586397) bank185685 +185686 POINT(37.385249615377184 -123.03368596603234) bank185686 +185687 POINT(36.88494402567772 -121.43734184785319) bank185687 +185688 POINT(37.322944672488994 -122.76652864072743) bank185688 +185689 POINT(38.63922044896816 -123.37424404759567) bank185689 +185690 POINT(38.14283814866088 -122.71919312394485) bank185690 +185691 POINT(38.44203870523349 -122.57062857923623) bank185691 +185692 POINT(38.163857114672716 -123.31887973994955) bank185692 +185693 POINT(36.892455156782994 -121.88803620968422) bank185693 +185694 POINT(38.51845530683235 -121.42190866337289) bank185694 +185695 POINT(36.980750194168074 -121.99031207540803) bank185695 +185696 POINT(37.23586325495443 -122.1933918990395) bank185696 +185697 POINT(37.99826024082332 -122.74603314406984) bank185697 +185698 POINT(37.858050601952236 -122.29590495978495) bank185698 +185699 POINT(38.50650164927406 -122.17213136797011) bank185699 +185700 POINT(38.52149239378144 -121.4273228029598) bank185700 +185701 POINT(37.462228440832675 -122.07332215617294) bank185701 +185702 POINT(37.87007156020713 -123.16107696756572) bank185702 +185703 POINT(38.725531575699456 -121.79135713185424) bank185703 +185704 POINT(37.31160342048793 -121.84216520662588) bank185704 +185705 POINT(38.179142359885915 -122.25419700013131) bank185705 +185706 POINT(38.39105788897301 -122.19637286158921) bank185706 +185707 POINT(37.590440963980114 -123.26695687021737) bank185707 +185708 POINT(38.280076672904514 -121.66816791413382) bank185708 +185709 POINT(37.947229982836774 -122.2800899299532) bank185709 +185710 POINT(37.65619184420741 -122.12510232101121) bank185710 +185711 POINT(38.508661534596826 -121.98592864503428) bank185711 +185712 POINT(37.54857835252436 -123.26835569761329) bank185712 +185713 POINT(38.140078124749664 -123.36833147651227) bank185713 +185714 POINT(37.68562942520842 -122.57666002218104) bank185714 +185715 POINT(37.94399828639688 -121.86278900123018) bank185715 +185716 POINT(37.61405321543946 -121.50375349107648) bank185716 +185717 POINT(37.6165669963025 -122.74573161620727) bank185717 +185718 POINT(37.55036822419092 -123.39896954822794) bank185718 +185719 POINT(37.26178341946759 -121.91599963994834) bank185719 +185720 POINT(37.43068067665434 -122.86560063417835) bank185720 +185721 POINT(38.056592751698986 -122.52192405832739) bank185721 +185722 POINT(38.27801607768937 -121.95453240329057) bank185722 +185723 POINT(37.045824123180886 -123.22623941667483) bank185723 +185724 POINT(37.367796986820714 -122.24614955002586) bank185724 +185725 POINT(37.993482635807545 -122.29364131585258) bank185725 +185726 POINT(38.43340807189138 -122.85011866601275) bank185726 +185727 POINT(38.51334066482157 -122.37465000451884) bank185727 +185728 POINT(38.385795091659396 -121.44541110043373) bank185728 +185729 POINT(38.096771361476236 -122.23190750388575) bank185729 +185730 POINT(38.2652955810979 -121.58759308049738) bank185730 +185731 POINT(38.393056488147735 -122.9396167840306) bank185731 +185732 POINT(38.37396000425385 -121.66266399636808) bank185732 +185733 POINT(37.20299802483313 -121.61811611533798) bank185733 +185734 POINT(37.417051481350384 -122.73310971019127) bank185734 +185735 POINT(37.924125083449084 -122.65863288094185) bank185735 +185736 POINT(38.22817658776145 -121.50612285661148) bank185736 +185737 POINT(38.47007543765561 -122.855968348079) bank185737 +185738 POINT(37.67401373419797 -123.22217957184179) bank185738 +185739 POINT(37.36590054494934 -121.50919741555263) bank185739 +185740 POINT(37.93394624070161 -122.5378188797481) bank185740 +185741 POINT(37.419792793069426 -122.5764838726155) bank185741 +185742 POINT(37.69652137876264 -123.08026432810826) bank185742 +185743 POINT(36.90378670211919 -121.76755107308833) bank185743 +185744 POINT(37.55171773253328 -122.69831650021412) bank185744 +185745 POINT(37.96375236371027 -123.38952003563774) bank185745 +185746 POINT(37.24648005492878 -123.31322046188895) bank185746 +185747 POINT(37.73633941580281 -122.2147966304715) bank185747 +185748 POINT(38.00588214502505 -123.31827558993791) bank185748 +185749 POINT(37.0769274809175 -122.60695845213876) bank185749 +185750 POINT(37.65735985172358 -122.59444552661647) bank185750 +185751 POINT(37.9755536459672 -122.95283801542456) bank185751 +185752 POINT(38.58597399621273 -121.59731085686879) bank185752 +185753 POINT(37.18878980082505 -122.42639720448213) bank185753 +185754 POINT(38.468079113245004 -122.32514195098132) bank185754 +185755 POINT(38.272255188481786 -122.99169217591036) bank185755 +185756 POINT(37.460282492753436 -122.09959703047997) bank185756 +185757 POINT(38.383277759829 -122.22045104458368) bank185757 +185758 POINT(38.63834085840419 -121.92035929468041) bank185758 +185759 POINT(37.8156948373181 -122.45746562661371) bank185759 +185760 POINT(37.51182617999372 -121.59917713798406) bank185760 +185761 POINT(37.33897406458852 -121.94234627358246) bank185761 +185762 POINT(38.72030245686713 -122.5194639154913) bank185762 +185763 POINT(38.356322710812556 -123.40250059940838) bank185763 +185764 POINT(36.813624359225365 -122.4392573184413) bank185764 +185765 POINT(38.63590655792337 -122.94063091815968) bank185765 +185766 POINT(36.956054353293275 -121.8479444559009) bank185766 +185767 POINT(36.94811460109823 -121.69385414215637) bank185767 +185768 POINT(38.48047136396727 -122.64603761831373) bank185768 +185769 POINT(38.06305259377524 -121.71983740073294) bank185769 +185770 POINT(37.84563521206816 -123.08269999849885) bank185770 +185771 POINT(37.52312285873577 -122.79573332435984) bank185771 +185772 POINT(38.47835850869243 -121.93770395021616) bank185772 +185773 POINT(38.18057709438488 -123.31707869338348) bank185773 +185774 POINT(38.030569499580274 -122.12398240501133) bank185774 +185775 POINT(38.14773563673465 -121.55849344086974) bank185775 +185776 POINT(38.08785995464543 -123.24620893615457) bank185776 +185777 POINT(38.20102808774533 -123.16781394145704) bank185777 +185778 POINT(37.60722773984533 -122.5148737671239) bank185778 +185779 POINT(38.569282333225935 -123.02706097893808) bank185779 +185780 POINT(37.52548203091678 -122.03017749550271) bank185780 +185781 POINT(37.639583508300184 -121.59605052534515) bank185781 +185782 POINT(38.245605208822546 -123.02081905269286) bank185782 +185783 POINT(37.20678614660702 -121.79491017813488) bank185783 +185784 POINT(36.94142878594258 -121.926699424386) bank185784 +185785 POINT(37.091973618347126 -122.66241741583899) bank185785 +185786 POINT(37.790403063667235 -122.23707980895966) bank185786 +185787 POINT(38.219244174654605 -121.69840405398936) bank185787 +185788 POINT(37.76776049675879 -121.53143994183202) bank185788 +185789 POINT(38.024337020579864 -122.34748418614495) bank185789 +185790 POINT(37.577441311220966 -123.13385698547837) bank185790 +185791 POINT(38.37821676592827 -122.90042947254888) bank185791 +185792 POINT(37.30470688268491 -121.8999527603897) bank185792 +185793 POINT(37.46076876758318 -122.78739043502384) bank185793 +185794 POINT(38.215201985304155 -122.14835481151273) bank185794 +185795 POINT(37.78846561629343 -121.8476614287504) bank185795 +185796 POINT(38.33704839677555 -123.3163762581576) bank185796 +185797 POINT(37.923990302944645 -121.72344654006274) bank185797 +185798 POINT(38.38914239164683 -121.99745655410646) bank185798 +185799 POINT(38.037324863530706 -122.3857029256898) bank185799 +185800 POINT(36.83733978577717 -122.07171235237523) bank185800 +185801 POINT(37.090643149975016 -122.83809501387478) bank185801 +185802 POINT(38.521169057281426 -122.07365314693962) bank185802 +185803 POINT(38.53081800599966 -122.20563539571316) bank185803 +185804 POINT(37.8631648107997 -122.64117141193515) bank185804 +185805 POINT(38.474199879437386 -122.88911358732037) bank185805 +185806 POINT(37.37467647562365 -123.26930552298296) bank185806 +185807 POINT(38.570583650842195 -122.26417182577218) bank185807 +185808 POINT(36.82730605294815 -121.97481509268177) bank185808 +185809 POINT(37.885353734365566 -122.62532634998392) bank185809 +185810 POINT(37.037845958787216 -121.82903268848399) bank185810 +185811 POINT(37.17404403905905 -122.3487149169364) bank185811 +185812 POINT(37.24230382953413 -122.9361695900022) bank185812 +185813 POINT(38.17173620493605 -123.07027263067705) bank185813 +185814 POINT(37.550357235575575 -123.21730849686958) bank185814 +185815 POINT(38.5522047031016 -122.46742725179867) bank185815 +185816 POINT(37.01801246139095 -121.44095986031019) bank185816 +185817 POINT(37.285229860286975 -121.66873442199712) bank185817 +185818 POINT(38.33095361846516 -123.25163082676511) bank185818 +185819 POINT(37.81362704314553 -123.2434577454208) bank185819 +185820 POINT(38.672927552842474 -122.20192824258173) bank185820 +185821 POINT(38.491115653064504 -123.33996996369956) bank185821 +185822 POINT(38.02130297352131 -121.77071678442854) bank185822 +185823 POINT(37.00398866311292 -122.07533210058932) bank185823 +185824 POINT(38.05292313833958 -121.71203870227717) bank185824 +185825 POINT(37.97670203459512 -121.63493349093451) bank185825 +185826 POINT(37.588332773778944 -121.51541657482755) bank185826 +185827 POINT(37.043339048119165 -121.46743242921454) bank185827 +185828 POINT(37.09891138441602 -122.17401695625206) bank185828 +185829 POINT(38.59337007975393 -121.65171103381304) bank185829 +185830 POINT(37.355508020853115 -122.24872846774895) bank185830 +185831 POINT(36.83449551427202 -121.64792850874167) bank185831 +185832 POINT(37.80602632759182 -123.13240188464022) bank185832 +185833 POINT(37.068443679475784 -121.43905136992484) bank185833 +185834 POINT(37.70768892891079 -121.62586973167883) bank185834 +185835 POINT(38.26129608988267 -122.86207354302087) bank185835 +185836 POINT(38.615109543476464 -122.52230614787707) bank185836 +185837 POINT(37.66042779658726 -123.12057057038464) bank185837 +185838 POINT(38.32531653550855 -123.23730345134493) bank185838 +185839 POINT(37.089502442160764 -121.83271750081589) bank185839 +185840 POINT(38.701484960079895 -122.01450176296899) bank185840 +185841 POINT(38.5799873048576 -122.331309139294) bank185841 +185842 POINT(36.82312756132766 -122.92795467969138) bank185842 +185843 POINT(38.096777219756575 -121.88337534727943) bank185843 +185844 POINT(38.3499567053939 -122.66337763092336) bank185844 +185845 POINT(37.23869965476817 -121.54044508466916) bank185845 +185846 POINT(37.07922761588371 -122.88053845656557) bank185846 +185847 POINT(37.16239075905514 -123.34225755963871) bank185847 +185848 POINT(37.225815448061034 -121.56532916174253) bank185848 +185849 POINT(37.68664637771507 -122.24891320129154) bank185849 +185850 POINT(38.30040512104392 -122.41355937347325) bank185850 +185851 POINT(37.12536586479507 -121.87219508567838) bank185851 +185852 POINT(38.06489159099197 -121.68780832735641) bank185852 +185853 POINT(36.84259767939473 -121.83430729509774) bank185853 +185854 POINT(37.14572527743416 -122.47052394931683) bank185854 +185855 POINT(38.24376056084482 -121.53718677347788) bank185855 +185856 POINT(37.762980402687376 -122.39092987500668) bank185856 +185857 POINT(38.5528905909235 -121.6251422420047) bank185857 +185858 POINT(36.959097123288814 -123.06555014162521) bank185858 +185859 POINT(37.905915614097594 -122.6025723008595) bank185859 +185860 POINT(37.54259857395468 -121.89108796652052) bank185860 +185861 POINT(37.11601350422692 -121.71281363374698) bank185861 +185862 POINT(37.27844168615999 -123.20689154189061) bank185862 +185863 POINT(37.66893965926238 -122.00400673125743) bank185863 +185864 POINT(38.26975241505958 -123.38623688819054) bank185864 +185865 POINT(37.52068987742253 -122.79744639572637) bank185865 +185866 POINT(37.85533879153998 -122.33555983683256) bank185866 +185867 POINT(38.051906771859514 -123.22333294173089) bank185867 +185868 POINT(37.793404646087915 -121.76378011597843) bank185868 +185869 POINT(36.89425283349136 -121.7634027085186) bank185869 +185870 POINT(37.59391518412812 -122.08275223894147) bank185870 +185871 POINT(38.446697608558864 -121.69525024209642) bank185871 +185872 POINT(38.02032920984325 -122.62455048454389) bank185872 +185873 POINT(37.14436693502725 -122.72344702742474) bank185873 +185874 POINT(36.904351963506876 -121.75848538206385) bank185874 +185875 POINT(37.77279704364183 -122.1414316719902) bank185875 +185876 POINT(38.144268474366676 -122.07798409472122) bank185876 +185877 POINT(37.45392962101758 -121.78752385644783) bank185877 +185878 POINT(37.05437625391185 -122.84482525536532) bank185878 +185879 POINT(38.01734906663657 -122.96198592084744) bank185879 +185880 POINT(38.34176573996391 -122.02618950045365) bank185880 +185881 POINT(36.799379877590376 -121.50561773971985) bank185881 +185882 POINT(37.50989934573083 -122.96923113542262) bank185882 +185883 POINT(37.65428934213321 -123.10373418906234) bank185883 +185884 POINT(37.182216666165544 -121.95325190425011) bank185884 +185885 POINT(37.578897583867956 -122.51351223216459) bank185885 +185886 POINT(37.37507809400981 -122.78314133784215) bank185886 +185887 POINT(36.84285553642437 -122.38487921232039) bank185887 +185888 POINT(37.76740050196327 -121.61068963281747) bank185888 +185889 POINT(38.14927355509704 -121.65287127042292) bank185889 +185890 POINT(37.27406099509045 -122.86865740863087) bank185890 +185891 POINT(37.77689433792265 -122.79918327523633) bank185891 +185892 POINT(36.91211854375503 -121.63180351130816) bank185892 +185893 POINT(37.649189497586185 -122.23986630298644) bank185893 +185894 POINT(38.29019068516211 -122.21416506407958) bank185894 +185895 POINT(38.52406001646934 -121.75527477311348) bank185895 +185896 POINT(38.64875370411301 -121.73735119885752) bank185896 +185897 POINT(37.28582136120544 -122.13340808359195) bank185897 +185898 POINT(37.23183097682551 -121.44855066905299) bank185898 +185899 POINT(37.687920679568236 -122.05461767890341) bank185899 +185900 POINT(38.046938506653646 -122.33700075212398) bank185900 +185901 POINT(37.568093962690504 -122.70544155589594) bank185901 +185902 POINT(38.53269039557742 -122.35131126231269) bank185902 +185903 POINT(36.92030857963286 -123.30789999938736) bank185903 +185904 POINT(38.57415646512512 -123.16140635470892) bank185904 +185905 POINT(37.79818807133815 -123.091991458817) bank185905 +185906 POINT(37.274483875873464 -121.54304738712665) bank185906 +185907 POINT(38.426116587370785 -121.76176044920355) bank185907 +185908 POINT(36.81083899387773 -122.79197350680252) bank185908 +185909 POINT(38.11696870253284 -122.59983526765066) bank185909 +185910 POINT(38.47612460515629 -122.23632148947799) bank185910 +185911 POINT(37.86460666988657 -121.77736721369797) bank185911 +185912 POINT(38.67353664455848 -122.78747769662425) bank185912 +185913 POINT(37.7775412261931 -122.10394638898498) bank185913 +185914 POINT(38.689315348204765 -121.6938813612916) bank185914 +185915 POINT(37.359662721751164 -122.36513907945563) bank185915 +185916 POINT(37.176902178350005 -122.59473018844804) bank185916 +185917 POINT(37.34583274118663 -122.47284314326457) bank185917 +185918 POINT(38.147696336320635 -121.65103288896114) bank185918 +185919 POINT(37.05516661407569 -123.24036728981278) bank185919 +185920 POINT(38.00304731974707 -122.93322113049304) bank185920 +185921 POINT(37.01006480913324 -123.08325878172134) bank185921 +185922 POINT(37.81848640879117 -121.96761262140268) bank185922 +185923 POINT(37.577371394614055 -121.46121235300357) bank185923 +185924 POINT(37.38501520518949 -123.292735484299) bank185924 +185925 POINT(38.627210249200175 -121.6401306197375) bank185925 +185926 POINT(37.54701385665027 -123.23775815991571) bank185926 +185927 POINT(37.32483976501499 -123.18702327107296) bank185927 +185928 POINT(36.966753862123994 -121.98704300997582) bank185928 +185929 POINT(37.56797308719303 -123.04769084681625) bank185929 +185930 POINT(38.46562594162831 -121.51055930945202) bank185930 +185931 POINT(37.310777930119094 -122.4392027183857) bank185931 +185932 POINT(37.9299898009191 -123.04891128268041) bank185932 +185933 POINT(37.957361206157465 -121.78947092122823) bank185933 +185934 POINT(37.06948034250413 -121.78343705751834) bank185934 +185935 POINT(38.535818188752216 -123.36108106685585) bank185935 +185936 POINT(38.21285681648476 -122.62264826189704) bank185936 +185937 POINT(37.583628466358434 -121.49588074201796) bank185937 +185938 POINT(38.484294658792024 -122.05317243056795) bank185938 +185939 POINT(37.87291915106525 -122.21031334349207) bank185939 +185940 POINT(36.909189105775724 -121.81889863419472) bank185940 +185941 POINT(37.67851173015732 -121.68463451253753) bank185941 +185942 POINT(38.10033493652082 -122.51046361113683) bank185942 +185943 POINT(38.65693112698027 -122.25308730163891) bank185943 +185944 POINT(38.35260188414412 -123.14303445515013) bank185944 +185945 POINT(38.76543092435015 -123.28537861828342) bank185945 +185946 POINT(36.863078853312565 -122.43645334867507) bank185946 +185947 POINT(38.11642484168983 -122.95930740234625) bank185947 +185948 POINT(38.13579457943706 -123.27646111886504) bank185948 +185949 POINT(38.73431687202225 -122.33255745713123) bank185949 +185950 POINT(37.42530956606677 -122.24643661452082) bank185950 +185951 POINT(37.67499574717501 -122.66405469903675) bank185951 +185952 POINT(38.58159622660341 -123.06862814713482) bank185952 +185953 POINT(36.87431387762407 -122.96547455908835) bank185953 +185954 POINT(37.48109342548666 -122.17227818453938) bank185954 +185955 POINT(38.57201810917188 -123.37700992959883) bank185955 +185956 POINT(37.40599593452184 -121.85965723222502) bank185956 +185957 POINT(36.79505110953024 -122.88400802696306) bank185957 +185958 POINT(37.58384392809791 -122.54370574271339) bank185958 +185959 POINT(37.28529697452601 -122.57077737019165) bank185959 +185960 POINT(37.9060654436838 -121.81548341927859) bank185960 +185961 POINT(38.03966878972063 -122.51904348363325) bank185961 +185962 POINT(37.95187880572677 -122.2598944107514) bank185962 +185963 POINT(38.27586811280473 -122.42816938043082) bank185963 +185964 POINT(37.866980831056225 -123.23241263766981) bank185964 +185965 POINT(37.87818365238999 -122.31207542453853) bank185965 +185966 POINT(37.96235422834248 -122.15137514851537) bank185966 +185967 POINT(38.34658062320158 -123.28063372705124) bank185967 +185968 POINT(38.30543448900859 -123.35678657545304) bank185968 +185969 POINT(37.82476575996792 -121.83304940744014) bank185969 +185970 POINT(38.360012992706906 -122.35097112933637) bank185970 +185971 POINT(37.30214814869959 -123.08335539342244) bank185971 +185972 POINT(37.93264680326255 -123.39136525013632) bank185972 +185973 POINT(37.370449697985094 -121.72336980713773) bank185973 +185974 POINT(38.26622928502429 -121.76001038222346) bank185974 +185975 POINT(37.304504724114025 -122.74012979503435) bank185975 +185976 POINT(38.349051967442335 -121.60376665904468) bank185976 +185977 POINT(37.561403939785215 -122.01469320257682) bank185977 +185978 POINT(38.275853051360244 -121.82319671743724) bank185978 +185979 POINT(37.396975959494995 -121.51026653355261) bank185979 +185980 POINT(37.34549702703935 -123.23728131011016) bank185980 +185981 POINT(36.96169872887686 -122.21535423208819) bank185981 +185982 POINT(37.00806659522355 -122.58550502846151) bank185982 +185983 POINT(37.6564739362624 -122.54719788232728) bank185983 +185984 POINT(36.81830989921576 -122.09564919214418) bank185984 +185985 POINT(37.058717499460606 -121.92839020534001) bank185985 +185986 POINT(38.71041607093291 -122.66424796655407) bank185986 +185987 POINT(36.938748115028424 -121.59984930411164) bank185987 +185988 POINT(37.95832826959914 -121.75745156690567) bank185988 +185989 POINT(37.500302801055746 -122.67787629646539) bank185989 +185990 POINT(38.34817837222872 -122.76405011013446) bank185990 +185991 POINT(37.50645070302757 -122.83562865460677) bank185991 +185992 POINT(36.83709531656572 -121.52856852633025) bank185992 +185993 POINT(38.67626452103765 -123.24363347227748) bank185993 +185994 POINT(37.058418096473986 -122.04409101426565) bank185994 +185995 POINT(38.66486690064592 -122.45647866224212) bank185995 +185996 POINT(37.91764398447162 -122.02138489819752) bank185996 +185997 POINT(37.25573721114903 -121.53847197782243) bank185997 +185998 POINT(38.481785016095465 -122.5466396880756) bank185998 +185999 POINT(38.72566361722877 -122.46955587275822) bank185999 +186000 POINT(37.294191625325745 -123.30957362698415) bank186000 +186001 POINT(37.769030769117656 -123.26574539293789) bank186001 +186002 POINT(38.1933373666652 -122.89024046844388) bank186002 +186003 POINT(37.78080167950773 -123.22836729371106) bank186003 +186004 POINT(37.64824818621765 -122.38128196667786) bank186004 +186005 POINT(37.38267832653459 -122.34320043848635) bank186005 +186006 POINT(36.80885524908975 -122.39416104389308) bank186006 +186007 POINT(38.51078467110272 -122.60424163512369) bank186007 +186008 POINT(38.365759323547564 -123.22888441655927) bank186008 +186009 POINT(38.326916142959405 -121.86921120139706) bank186009 +186010 POINT(38.35668914598084 -122.94195181599807) bank186010 +186011 POINT(38.20543162416496 -122.51176402835414) bank186011 +186012 POINT(38.62323086109454 -122.24761783762808) bank186012 +186013 POINT(38.58937596141174 -122.31453978306475) bank186013 +186014 POINT(37.21392087262491 -122.49467976337583) bank186014 +186015 POINT(37.72447278623137 -121.74765573746647) bank186015 +186016 POINT(37.960847161629765 -123.24120158713427) bank186016 +186017 POINT(37.1526465120514 -123.12799709251239) bank186017 +186018 POINT(38.707520063033314 -122.28228710220556) bank186018 +186019 POINT(37.85123944642099 -122.16524858927792) bank186019 +186020 POINT(38.75251008680905 -123.31765119174386) bank186020 +186021 POINT(38.47287419225363 -122.68610842238968) bank186021 +186022 POINT(38.19161981006886 -122.3716930610493) bank186022 +186023 POINT(37.87119488551997 -121.64679828255795) bank186023 +186024 POINT(37.26765595558212 -122.33126999848074) bank186024 +186025 POINT(38.00403029315312 -123.21885138196873) bank186025 +186026 POINT(37.999662069100225 -122.49975658406497) bank186026 +186027 POINT(38.42500871186238 -121.62272822664089) bank186027 +186028 POINT(37.489767336321364 -121.88640978218736) bank186028 +186029 POINT(37.530630933298525 -123.06829577992252) bank186029 +186030 POINT(38.30481966179211 -121.50552217220064) bank186030 +186031 POINT(38.486270194230826 -122.07115788815405) bank186031 +186032 POINT(37.280319927501495 -121.83377262614472) bank186032 +186033 POINT(38.54336214918569 -123.18919477014133) bank186033 +186034 POINT(37.975333766351724 -122.65777720244157) bank186034 +186035 POINT(38.735230599154214 -122.71695872371802) bank186035 +186036 POINT(38.51983134042401 -121.64917718546954) bank186036 +186037 POINT(37.952331816191766 -121.49066528422964) bank186037 +186038 POINT(38.49073255350112 -122.72736884698989) bank186038 +186039 POINT(37.81560616420788 -123.25086499970277) bank186039 +186040 POINT(38.66995041535186 -122.14715714321285) bank186040 +186041 POINT(37.73955012760879 -121.52537854626303) bank186041 +186042 POINT(37.77530493564918 -122.97745185553083) bank186042 +186043 POINT(37.55377136866517 -123.18690232229912) bank186043 +186044 POINT(37.73285138125882 -122.83484901062894) bank186044 +186045 POINT(38.157405139481504 -122.44262034284513) bank186045 +186046 POINT(38.21283532396137 -122.93398308896714) bank186046 +186047 POINT(37.7428588215869 -122.2493026326653) bank186047 +186048 POINT(38.25062040002784 -122.69618334850466) bank186048 +186049 POINT(38.19926858355756 -121.97847516607493) bank186049 +186050 POINT(38.04338338967457 -122.88810369813893) bank186050 +186051 POINT(38.5910248524176 -122.97981200054488) bank186051 +186052 POINT(38.053432040438224 -122.93583733831447) bank186052 +186053 POINT(37.83986157977039 -121.51559383957112) bank186053 +186054 POINT(38.08750756297073 -121.4664845932372) bank186054 +186055 POINT(37.6631008223841 -122.33867499900876) bank186055 +186056 POINT(37.87650597381772 -121.49356908965534) bank186056 +186057 POINT(37.959744307616184 -123.23220632814972) bank186057 +186058 POINT(36.90533903726063 -122.05720299837633) bank186058 +186059 POINT(37.1027616615797 -122.06844285233309) bank186059 +186060 POINT(38.43995950194125 -122.42708863357609) bank186060 +186061 POINT(38.5425452466465 -122.5540229608419) bank186061 +186062 POINT(37.30597796230823 -122.87357674279335) bank186062 +186063 POINT(37.7223507713813 -121.54704342966835) bank186063 +186064 POINT(36.949484559347326 -121.90552411775437) bank186064 +186065 POINT(38.011685617457864 -123.15431966396193) bank186065 +186066 POINT(37.912435164763885 -122.065574990406) bank186066 +186067 POINT(38.33953028626732 -122.66590601983377) bank186067 +186068 POINT(37.103607190999895 -122.49224450446137) bank186068 +186069 POINT(38.527948984031184 -121.83438879765193) bank186069 +186070 POINT(36.89223113086347 -122.69931554158204) bank186070 +186071 POINT(38.25655568573862 -122.9218005760301) bank186071 +186072 POINT(38.144729751344116 -123.4147199481708) bank186072 +186073 POINT(36.99499949720484 -121.81175051781892) bank186073 +186074 POINT(37.89786817465821 -121.87119784813333) bank186074 +186075 POINT(37.116758787141684 -122.72021409876743) bank186075 +186076 POINT(38.04618278041203 -121.45754101510391) bank186076 +186077 POINT(37.11949337721189 -122.64135479430935) bank186077 +186078 POINT(36.84026564570776 -121.75216905355103) bank186078 +186079 POINT(36.79456456072962 -121.90698613906703) bank186079 +186080 POINT(37.53436674920467 -122.2253997040253) bank186080 +186081 POINT(38.698122593947645 -123.23770471141613) bank186081 +186082 POINT(37.83197531754909 -123.41054064275454) bank186082 +186083 POINT(37.945585589066475 -122.61180757127944) bank186083 +186084 POINT(38.350091145091596 -123.3741873030621) bank186084 +186085 POINT(36.98412764119076 -122.83063173765949) bank186085 +186086 POINT(37.27259996838059 -122.31452258316598) bank186086 +186087 POINT(38.02765018166508 -122.80641268682106) bank186087 +186088 POINT(38.368380807623865 -123.02401912641999) bank186088 +186089 POINT(37.51120164089562 -121.58226600995312) bank186089 +186090 POINT(36.95781410932639 -122.68492069398695) bank186090 +186091 POINT(38.696727126160255 -122.62801450899322) bank186091 +186092 POINT(37.36403615425494 -121.71564508602533) bank186092 +186093 POINT(37.66194349615985 -123.08462924067479) bank186093 +186094 POINT(37.122594967371896 -123.27782557492306) bank186094 +186095 POINT(37.13391937564872 -122.1654040384114) bank186095 +186096 POINT(38.37299268032366 -122.8724935509154) bank186096 +186097 POINT(37.274889724265854 -122.54835980492649) bank186097 +186098 POINT(36.91651741551918 -122.85943375115218) bank186098 +186099 POINT(37.739560468777235 -121.75879394285194) bank186099 +186100 POINT(37.94335813226162 -121.52014768348589) bank186100 +186101 POINT(38.2613620593918 -122.55355281277505) bank186101 +186102 POINT(36.844212631496816 -121.88797244475356) bank186102 +186103 POINT(38.55668136755788 -122.41784802701136) bank186103 +186104 POINT(36.93318808816993 -121.65094638320681) bank186104 +186105 POINT(37.596003539551255 -122.9537107350284) bank186105 +186106 POINT(38.709177314792704 -123.39836820137121) bank186106 +186107 POINT(36.964642677839905 -122.45769692152551) bank186107 +186108 POINT(38.694284396377945 -121.92731282941338) bank186108 +186109 POINT(36.83148187619279 -122.72716964274755) bank186109 +186110 POINT(37.79473302501036 -121.60312509887503) bank186110 +186111 POINT(37.38198982192918 -123.04774115936256) bank186111 +186112 POINT(38.187005779814136 -123.36801778183191) bank186112 +186113 POINT(37.16669688946718 -122.91580300212458) bank186113 +186114 POINT(37.446136658508905 -122.73788782484726) bank186114 +186115 POINT(36.842123156255504 -121.95067759795256) bank186115 +186116 POINT(37.82959168564387 -122.63573327352152) bank186116 +186117 POINT(38.16068581082398 -122.0976705322693) bank186117 +186118 POINT(36.87524675279103 -123.03243463975282) bank186118 +186119 POINT(38.359931945906894 -122.27609553399311) bank186119 +186120 POINT(37.57904412968185 -122.55626320812429) bank186120 +186121 POINT(37.97037043440252 -122.21269608077341) bank186121 +186122 POINT(37.17507909391051 -121.82326210451158) bank186122 +186123 POINT(37.15401787658684 -121.71584211119257) bank186123 +186124 POINT(37.00860495076844 -121.49847781309784) bank186124 +186125 POINT(38.19382398636161 -121.68844750055683) bank186125 +186126 POINT(38.34246407621367 -121.61257251768848) bank186126 +186127 POINT(38.384269219946596 -123.04454239355283) bank186127 +186128 POINT(38.2248115162973 -122.52709612597975) bank186128 +186129 POINT(38.19611205644625 -122.09722957236917) bank186129 +186130 POINT(37.65912970683792 -122.87196318268973) bank186130 +186131 POINT(38.312246086928276 -122.64116790322315) bank186131 +186132 POINT(37.40131325534622 -122.34978628816923) bank186132 +186133 POINT(37.43529387036028 -121.99786642783855) bank186133 +186134 POINT(36.98699769443316 -123.19415187850723) bank186134 +186135 POINT(38.39038839102385 -121.51395851235775) bank186135 +186136 POINT(37.856394929443454 -122.87996584157115) bank186136 +186137 POINT(37.79144169301317 -123.12456872394276) bank186137 +186138 POINT(38.11751774931687 -122.75284769903267) bank186138 +186139 POINT(38.18130841307922 -121.64294032795542) bank186139 +186140 POINT(38.09239722354253 -121.48279337791266) bank186140 +186141 POINT(37.94432362817037 -122.40381552362525) bank186141 +186142 POINT(37.84856709585232 -121.59308730280856) bank186142 +186143 POINT(38.248837255370056 -123.02516069268478) bank186143 +186144 POINT(38.2018548132418 -122.23243310808704) bank186144 +186145 POINT(38.42720416536404 -122.28908970568389) bank186145 +186146 POINT(37.93297595265641 -122.94594663916389) bank186146 +186147 POINT(38.41482714974236 -122.50218480322066) bank186147 +186148 POINT(37.13313018572601 -122.18411210079636) bank186148 +186149 POINT(37.92745709216995 -121.83058075284023) bank186149 +186150 POINT(37.90861339472123 -123.24166514893585) bank186150 +186151 POINT(37.871445631674895 -122.31202643311357) bank186151 +186152 POINT(38.491052918928 -122.11349727154067) bank186152 +186153 POINT(38.28758348229438 -121.8321090387341) bank186153 +186154 POINT(37.58648720168884 -122.08591018673343) bank186154 +186155 POINT(36.8459751812121 -121.75328190109218) bank186155 +186156 POINT(37.37389323449856 -121.83163652739249) bank186156 +186157 POINT(38.262202459040374 -121.65390300047531) bank186157 +186158 POINT(37.118505489370975 -122.04672787173506) bank186158 +186159 POINT(37.95252038883487 -122.43835663287012) bank186159 +186160 POINT(38.39649753805092 -122.5157355828563) bank186160 +186161 POINT(36.977583130369865 -121.8718363057561) bank186161 +186162 POINT(37.078902182297945 -123.16928548415889) bank186162 +186163 POINT(38.22108590335809 -122.70106041679298) bank186163 +186164 POINT(37.02725530931247 -123.0441968473755) bank186164 +186165 POINT(38.15162007390113 -122.86767194813426) bank186165 +186166 POINT(37.41760352431896 -121.56963582841719) bank186166 +186167 POINT(37.98395064580428 -122.32896520717301) bank186167 +186168 POINT(36.79386316418534 -123.07314626755993) bank186168 +186169 POINT(37.816690572891744 -122.89920698580234) bank186169 +186170 POINT(36.953128469362134 -123.14378800130284) bank186170 +186171 POINT(37.54657606485859 -122.44330281275762) bank186171 +186172 POINT(37.256169470447645 -122.87845541277609) bank186172 +186173 POINT(38.508557112646564 -123.09343798678873) bank186173 +186174 POINT(37.869755741555196 -122.03206058763227) bank186174 +186175 POINT(36.95779543724199 -122.0915365150837) bank186175 +186176 POINT(38.54412010034479 -123.03206712210161) bank186176 +186177 POINT(37.484139381098444 -122.24996465286647) bank186177 +186178 POINT(36.99832706200638 -122.08780139139934) bank186178 +186179 POINT(38.10098641923012 -123.07170089333164) bank186179 +186180 POINT(37.20146044733098 -123.0502189903874) bank186180 +186181 POINT(38.35248743709958 -122.1550750572766) bank186181 +186182 POINT(38.09623236792131 -122.7343550558207) bank186182 +186183 POINT(38.43360253508222 -121.51941339629275) bank186183 +186184 POINT(37.218163342263985 -122.61922947693772) bank186184 +186185 POINT(38.20758001456491 -122.28226987361693) bank186185 +186186 POINT(38.0589279963035 -121.89849216271567) bank186186 +186187 POINT(36.854895224361876 -123.20811730090495) bank186187 +186188 POINT(37.003650112010234 -121.80025215157588) bank186188 +186189 POINT(36.99157209221399 -122.97234247338926) bank186189 +186190 POINT(38.34963211999943 -121.47035105939568) bank186190 +186191 POINT(38.443865369006865 -122.29307219815735) bank186191 +186192 POINT(36.79493024119188 -122.0867279442767) bank186192 +186193 POINT(38.34979604252208 -122.1953605414281) bank186193 +186194 POINT(36.784313716599094 -121.7603310259347) bank186194 +186195 POINT(37.59263326266133 -122.48522654707043) bank186195 +186196 POINT(37.80980060522343 -121.91834567854245) bank186196 +186197 POINT(38.07397512809489 -122.27839510481493) bank186197 +186198 POINT(37.00236025357538 -121.91390922824581) bank186198 +186199 POINT(37.07407952015687 -123.21391910345488) bank186199 +186200 POINT(37.425375232908884 -123.18648505530727) bank186200 +186201 POINT(36.7952678265989 -123.351951578745) bank186201 +186202 POINT(37.80580256396021 -121.54471841942647) bank186202 +186203 POINT(37.81653763026681 -121.8150502255239) bank186203 +186204 POINT(37.7638616737212 -122.69804130745781) bank186204 +186205 POINT(38.38512478540364 -121.61391501573473) bank186205 +186206 POINT(37.08918542719846 -122.96252451945601) bank186206 +186207 POINT(37.11272329131865 -121.52940901189685) bank186207 +186208 POINT(38.36984593365372 -122.0768088310213) bank186208 +186209 POINT(38.38564004482716 -122.21928226056419) bank186209 +186210 POINT(38.107963939062316 -121.78483360315285) bank186210 +186211 POINT(36.91929034309008 -121.82425625872969) bank186211 +186212 POINT(37.408169294208506 -121.77526417265835) bank186212 +186213 POINT(38.3955516509277 -121.42965107846439) bank186213 +186214 POINT(38.51539023807002 -122.77615391831978) bank186214 +186215 POINT(37.65970784820865 -123.05411430134885) bank186215 +186216 POINT(37.01916367366461 -122.13626608143109) bank186216 +186217 POINT(36.989732053474214 -121.51348739594107) bank186217 +186218 POINT(37.701140890727594 -122.81197813074459) bank186218 +186219 POINT(38.61051591179692 -122.45706684359767) bank186219 +186220 POINT(38.0444311054539 -121.8465098883838) bank186220 +186221 POINT(36.91707416063025 -121.82967653222201) bank186221 +186222 POINT(37.918764702273 -123.35367391359983) bank186222 +186223 POINT(37.693023940141494 -123.0300502689588) bank186223 +186224 POINT(37.391855663668565 -122.23940399227435) bank186224 +186225 POINT(37.864804986096075 -122.03942281943583) bank186225 +186226 POINT(38.56667521873156 -123.02479981565334) bank186226 +186227 POINT(38.478038522401434 -123.22957381233516) bank186227 +186228 POINT(37.15197509050912 -122.4993625044843) bank186228 +186229 POINT(37.19445730507249 -122.5238998199509) bank186229 +186230 POINT(38.6613434144298 -123.09384084601392) bank186230 +186231 POINT(38.67777935866787 -123.27341679416928) bank186231 +186232 POINT(38.33725450080726 -121.84957909660297) bank186232 +186233 POINT(38.66949717662303 -122.25992840670452) bank186233 +186234 POINT(36.9151375661727 -122.01885733001394) bank186234 +186235 POINT(37.289277325426475 -121.80608398226768) bank186235 +186236 POINT(37.702563180379286 -122.13065332567835) bank186236 +186237 POINT(38.37128909300584 -122.14745668175644) bank186237 +186238 POINT(38.55844651780065 -122.12366689648012) bank186238 +186239 POINT(37.21346824884313 -123.19567143598833) bank186239 +186240 POINT(37.09343109268053 -123.415726271538) bank186240 +186241 POINT(38.44372107610659 -122.11823460749277) bank186241 +186242 POINT(38.76805668880989 -122.03448507392795) bank186242 +186243 POINT(38.76738434406069 -121.83676849624393) bank186243 +186244 POINT(37.138087367478064 -123.12636674342068) bank186244 +186245 POINT(38.126540827853255 -122.17399805771637) bank186245 +186246 POINT(38.62946891183068 -122.07944108025654) bank186246 +186247 POINT(37.56961715307836 -121.9341830043665) bank186247 +186248 POINT(38.53239930127727 -122.21334993551933) bank186248 +186249 POINT(37.61131897844864 -122.504406698197) bank186249 +186250 POINT(38.29111967624346 -121.5338717873171) bank186250 +186251 POINT(38.60303717808296 -122.67953008368788) bank186251 +186252 POINT(38.51654200785837 -123.06375760261032) bank186252 +186253 POINT(37.036849967900565 -121.61332927164214) bank186253 +186254 POINT(37.28135582002952 -121.43385870277221) bank186254 +186255 POINT(38.205145565384825 -122.82523478031713) bank186255 +186256 POINT(37.92378437639891 -122.53829954193799) bank186256 +186257 POINT(37.61199772516125 -121.4954204965782) bank186257 +186258 POINT(38.29885183964789 -122.15347050699893) bank186258 +186259 POINT(37.654408392223246 -122.46345189287132) bank186259 +186260 POINT(37.178273666324344 -121.95644118115753) bank186260 +186261 POINT(38.3693291691509 -121.65933144025858) bank186261 +186262 POINT(38.41770247742702 -121.90899402286645) bank186262 +186263 POINT(37.310020928575554 -123.31399964295787) bank186263 +186264 POINT(37.46002737418272 -123.22618228182019) bank186264 +186265 POINT(37.28030441825071 -123.20347313885716) bank186265 +186266 POINT(37.58578467335496 -121.88500120119559) bank186266 +186267 POINT(37.901799831573996 -123.37338151540479) bank186267 +186268 POINT(36.86378615840291 -122.7797222866031) bank186268 +186269 POINT(37.5225231463569 -121.90829302099515) bank186269 +186270 POINT(37.32563753284061 -122.99926014155334) bank186270 +186271 POINT(38.30852780076587 -121.6416327132234) bank186271 +186272 POINT(38.60666823682276 -122.25582667110731) bank186272 +186273 POINT(37.712793597845355 -122.6644214076247) bank186273 +186274 POINT(37.2580686718405 -122.05929066677808) bank186274 +186275 POINT(38.00434470867103 -122.59911472158213) bank186275 +186276 POINT(38.72213934673934 -122.56118847896653) bank186276 +186277 POINT(36.8830149414397 -121.84629756506413) bank186277 +186278 POINT(38.257572009593495 -122.16396143399217) bank186278 +186279 POINT(37.577376255230206 -122.87915494610459) bank186279 +186280 POINT(38.65661154544913 -122.00571142578569) bank186280 +186281 POINT(36.89592846341526 -122.76561203826058) bank186281 +186282 POINT(37.558249559568715 -121.6857177725369) bank186282 +186283 POINT(37.155006081335294 -122.68392183506889) bank186283 +186284 POINT(38.76200081904617 -122.45776293792102) bank186284 +186285 POINT(38.00112846906393 -121.80488434290652) bank186285 +186286 POINT(38.338589520340186 -122.87195935994738) bank186286 +186287 POINT(37.163575622198266 -122.4398321569588) bank186287 +186288 POINT(38.373402493527095 -123.37972704098074) bank186288 +186289 POINT(37.4676263296858 -122.4749742767201) bank186289 +186290 POINT(37.55691674744181 -121.54159554301695) bank186290 +186291 POINT(37.63195367432758 -122.21825423102732) bank186291 +186292 POINT(37.610510777415186 -122.72918123749791) bank186292 +186293 POINT(38.30215330543086 -122.7508889627628) bank186293 +186294 POINT(38.65606279527826 -123.29341985390501) bank186294 +186295 POINT(38.432614580603065 -121.67695643677473) bank186295 +186296 POINT(36.94462847194903 -121.87529831883744) bank186296 +186297 POINT(37.9198216923756 -122.89211951904367) bank186297 +186298 POINT(36.782441103834174 -123.33657407595612) bank186298 +186299 POINT(37.61596547647872 -122.67026818745465) bank186299 +186300 POINT(37.473008608097345 -122.4335547217818) bank186300 +186301 POINT(38.690892407360735 -122.17682235444175) bank186301 +186302 POINT(37.19079115865923 -123.02147484085984) bank186302 +186303 POINT(36.80259958488741 -123.25588195816384) bank186303 +186304 POINT(38.46954443142154 -122.14537966619251) bank186304 +186305 POINT(38.06541137862182 -122.84923104722712) bank186305 +186306 POINT(36.78051683436633 -121.72796666971053) bank186306 +186307 POINT(38.05645795970195 -122.54881217861772) bank186307 +186308 POINT(38.30426557774462 -123.05520429706966) bank186308 +186309 POINT(37.88899787476423 -123.17202651054387) bank186309 +186310 POINT(38.02433329644912 -122.55024411364512) bank186310 +186311 POINT(37.022108908635154 -123.31440537205316) bank186311 +186312 POINT(38.64547634828853 -123.21710034080587) bank186312 +186313 POINT(38.38157956901806 -122.30074898042993) bank186313 +186314 POINT(37.107790125783424 -121.49599567007093) bank186314 +186315 POINT(37.39419533069911 -121.75882778594254) bank186315 +186316 POINT(36.92756409002762 -121.77999318423642) bank186316 +186317 POINT(36.79943855813871 -122.87421345943778) bank186317 +186318 POINT(38.353882118039245 -121.61918765213157) bank186318 +186319 POINT(37.15419571761493 -122.76767894277194) bank186319 +186320 POINT(38.08923731133287 -122.19419134824854) bank186320 +186321 POINT(37.01395932531415 -121.99658812570317) bank186321 +186322 POINT(37.181811704902934 -121.61731050153858) bank186322 +186323 POINT(37.15152054984152 -122.64948169410985) bank186323 +186324 POINT(36.78148200653439 -122.06342241206367) bank186324 +186325 POINT(37.00443040552636 -123.35169429994113) bank186325 +186326 POINT(38.28568407227691 -123.34987218999996) bank186326 +186327 POINT(37.28106495281639 -122.47853419064616) bank186327 +186328 POINT(37.4918405092967 -122.76454337915091) bank186328 +186329 POINT(37.576040904768085 -122.09139007311704) bank186329 +186330 POINT(37.77030538503264 -123.22318595158971) bank186330 +186331 POINT(38.047687831325945 -122.62670486824848) bank186331 +186332 POINT(38.42843463307029 -123.33934296554882) bank186332 +186333 POINT(37.02013583893978 -122.37731228908382) bank186333 +186334 POINT(37.77054660578164 -122.2857253822155) bank186334 +186335 POINT(37.06869369691432 -121.6771494193831) bank186335 +186336 POINT(36.81057882325106 -122.31025660558743) bank186336 +186337 POINT(38.43466681385909 -121.84759691544238) bank186337 +186338 POINT(38.551845839399455 -122.28779670747755) bank186338 +186339 POINT(37.7063571473553 -123.40086272144889) bank186339 +186340 POINT(37.90676607068708 -123.07509394989496) bank186340 +186341 POINT(37.137037715388225 -121.49228066655753) bank186341 +186342 POINT(37.489070761749666 -121.99270597349468) bank186342 +186343 POINT(37.93048646320153 -122.92816032832862) bank186343 +186344 POINT(37.07663021352856 -121.66980047060979) bank186344 +186345 POINT(38.731706840858564 -122.42206305580731) bank186345 +186346 POINT(37.818000573194674 -122.29153506022863) bank186346 +186347 POINT(36.795456390601515 -121.75952378861474) bank186347 +186348 POINT(38.61128340014477 -123.27395731363093) bank186348 +186349 POINT(37.1329676794864 -123.07812490169857) bank186349 +186350 POINT(37.72998950568203 -121.83543489414322) bank186350 +186351 POINT(37.35478353327433 -121.99183016035091) bank186351 +186352 POINT(38.4758243816794 -123.02276680449089) bank186352 +186353 POINT(37.58554397349585 -122.37686409572069) bank186353 +186354 POINT(37.89082536357911 -122.17564908157938) bank186354 +186355 POINT(37.635829017808454 -122.31089416331722) bank186355 +186356 POINT(38.441399687213924 -122.16246887207087) bank186356 +186357 POINT(36.969689562615336 -122.398967354239) bank186357 +186358 POINT(38.38330542860808 -122.51957430228003) bank186358 +186359 POINT(37.50747568298445 -122.99143215956671) bank186359 +186360 POINT(37.670251733771956 -121.81514994556838) bank186360 +186361 POINT(36.8000464347536 -121.8977462834438) bank186361 +186362 POINT(36.94023877133555 -122.42272971665291) bank186362 +186363 POINT(37.08193974812149 -122.63969196323407) bank186363 +186364 POINT(38.11832705635393 -122.86099153383783) bank186364 +186365 POINT(37.96203616827262 -123.38514719330898) bank186365 +186366 POINT(37.936043683526854 -123.35550601177675) bank186366 +186367 POINT(38.07408721085534 -122.95259472595708) bank186367 +186368 POINT(37.87019911412667 -122.01241244427959) bank186368 +186369 POINT(36.985309200323705 -121.46761200097555) bank186369 +186370 POINT(37.76324224991325 -121.51484570644095) bank186370 +186371 POINT(38.70319082014974 -122.97863290295747) bank186371 +186372 POINT(37.14130393288636 -122.9793915858856) bank186372 +186373 POINT(38.21552881169313 -122.36032311300383) bank186373 +186374 POINT(38.0294583126233 -123.29988820738534) bank186374 +186375 POINT(38.29500959032422 -121.62828069146511) bank186375 +186376 POINT(38.09335152473544 -123.13529760172541) bank186376 +186377 POINT(36.882697788583094 -122.89931431131087) bank186377 +186378 POINT(36.81092041476291 -122.336961290702) bank186378 +186379 POINT(38.16374904081532 -123.16275272906222) bank186379 +186380 POINT(37.41528370764105 -122.11946694808695) bank186380 +186381 POINT(38.183139454873064 -123.31780883426266) bank186381 +186382 POINT(38.325845633807184 -121.98691683353921) bank186382 +186383 POINT(37.20436840738619 -122.13972833380187) bank186383 +186384 POINT(37.173754619622734 -123.06704948318921) bank186384 +186385 POINT(37.623237485512746 -121.62310051408204) bank186385 +186386 POINT(37.43903144117432 -122.98773765590926) bank186386 +186387 POINT(37.702117032782404 -123.19433421345114) bank186387 +186388 POINT(37.056094995691076 -123.22578174169817) bank186388 +186389 POINT(37.783522956510694 -121.52547624587005) bank186389 +186390 POINT(37.30433470601467 -121.55035682417169) bank186390 +186391 POINT(38.36729479978496 -123.03137377344447) bank186391 +186392 POINT(37.7194736830206 -122.89853024627371) bank186392 +186393 POINT(36.83893713568732 -121.66992001512583) bank186393 +186394 POINT(37.91929881505412 -122.66153252755298) bank186394 +186395 POINT(37.32833072232901 -122.40858761594112) bank186395 +186396 POINT(37.55508345168202 -121.70892540414276) bank186396 +186397 POINT(37.773107281781506 -123.24500412132785) bank186397 +186398 POINT(37.93064549379983 -122.41997285472999) bank186398 +186399 POINT(38.431911189527256 -122.29990900311745) bank186399 +186400 POINT(37.93245599906768 -121.66413098754938) bank186400 +186401 POINT(38.71511786090292 -122.37535937878447) bank186401 +186402 POINT(38.055172035564695 -121.51071038407923) bank186402 +186403 POINT(37.9325177116108 -122.86903847184787) bank186403 +186404 POINT(37.82735420900231 -121.79265419556039) bank186404 +186405 POINT(38.30575581236015 -121.7671179611308) bank186405 +186406 POINT(38.37609270945989 -122.53128812996769) bank186406 +186407 POINT(36.94593232459317 -123.09923833288806) bank186407 +186408 POINT(37.016670708126355 -121.57844046918748) bank186408 +186409 POINT(37.82097934789752 -123.3804501714486) bank186409 +186410 POINT(36.80224414147362 -123.34584455848864) bank186410 +186411 POINT(37.66617387816709 -122.02437297207817) bank186411 +186412 POINT(38.298240049388546 -121.84164311210094) bank186412 +186413 POINT(38.39857821587904 -123.16351612612051) bank186413 +186414 POINT(37.980517738512134 -123.30710772309945) bank186414 +186415 POINT(36.81378825979818 -122.53286215197514) bank186415 +186416 POINT(37.30292359082045 -123.23700208675271) bank186416 +186417 POINT(38.75081369324633 -123.29852073434944) bank186417 +186418 POINT(38.48717557404185 -121.6486728811966) bank186418 +186419 POINT(37.49407876827104 -123.10502195033548) bank186419 +186420 POINT(38.06628493827221 -121.76278086112676) bank186420 +186421 POINT(37.532243684187804 -121.92833532898801) bank186421 +186422 POINT(37.39801832710741 -122.31493674874825) bank186422 +186423 POINT(38.609606462084045 -122.19583536712597) bank186423 +186424 POINT(38.53913807610689 -122.31715928962765) bank186424 +186425 POINT(38.278222673862814 -122.64827683947848) bank186425 +186426 POINT(36.80191660999941 -122.46036519125903) bank186426 +186427 POINT(36.92853674109594 -121.47243606558575) bank186427 +186428 POINT(36.81890622890049 -123.31219585351089) bank186428 +186429 POINT(37.27105579937157 -122.9792006423734) bank186429 +186430 POINT(38.56137188719986 -122.69397427518004) bank186430 +186431 POINT(38.03885078948124 -121.51229967645855) bank186431 +186432 POINT(37.645114364527075 -122.41418221469407) bank186432 +186433 POINT(37.709846145950635 -122.59698777531545) bank186433 +186434 POINT(38.25433845117391 -123.19660416059855) bank186434 +186435 POINT(37.74775820476833 -122.03007960180571) bank186435 +186436 POINT(36.927023124911635 -122.03713808884947) bank186436 +186437 POINT(37.96704367292502 -122.46211653061573) bank186437 +186438 POINT(38.615190429943915 -123.31097955248133) bank186438 +186439 POINT(37.74656514224247 -123.15225405288095) bank186439 +186440 POINT(37.647880679157495 -122.49502214853011) bank186440 +186441 POINT(38.10583551088602 -122.72239712462034) bank186441 +186442 POINT(36.931806270760596 -122.3495603849894) bank186442 +186443 POINT(37.422140594239295 -121.95447718180807) bank186443 +186444 POINT(38.545762647718654 -122.31813161300242) bank186444 +186445 POINT(37.89872623474798 -122.69298560390733) bank186445 +186446 POINT(37.619501070185834 -122.62762912721391) bank186446 +186447 POINT(37.769113794322436 -123.34942165516291) bank186447 +186448 POINT(37.427077783147766 -122.59605801855287) bank186448 +186449 POINT(38.13578102218381 -121.511973939) bank186449 +186450 POINT(38.41474126459014 -123.37798480320438) bank186450 +186451 POINT(37.876769815696655 -122.51308270115709) bank186451 +186452 POINT(37.571042230948116 -123.22675660263866) bank186452 +186453 POINT(36.87483237605514 -122.50118325361387) bank186453 +186454 POINT(38.18561735356803 -121.85277148679882) bank186454 +186455 POINT(37.20058182039124 -123.24346064205369) bank186455 +186456 POINT(37.76728347233663 -121.56140607885423) bank186456 +186457 POINT(38.3395983954968 -122.49795407773894) bank186457 +186458 POINT(37.03065438420887 -121.74349813372096) bank186458 +186459 POINT(37.204037477888704 -121.84516251713211) bank186459 +186460 POINT(37.020082750810005 -123.31395553020641) bank186460 +186461 POINT(38.04891918230747 -122.7340181691437) bank186461 +186462 POINT(36.943542203760614 -121.83940821006354) bank186462 +186463 POINT(37.32954722290939 -122.93302718363054) bank186463 +186464 POINT(37.906391742406385 -123.25013801507073) bank186464 +186465 POINT(36.86420609562762 -122.01645493139904) bank186465 +186466 POINT(38.01133476362434 -122.49837166733647) bank186466 +186467 POINT(38.36478362090582 -122.46652016992626) bank186467 +186468 POINT(38.54957693651292 -121.60258423379423) bank186468 +186469 POINT(36.77884696400838 -121.99891936349567) bank186469 +186470 POINT(37.43670771611641 -122.50285651893502) bank186470 +186471 POINT(38.10470865005709 -122.92620588824188) bank186471 +186472 POINT(37.558946280184706 -122.66756433084663) bank186472 +186473 POINT(38.54867120037995 -123.1371629207434) bank186473 +186474 POINT(37.300079524703506 -123.02563576050872) bank186474 +186475 POINT(37.29351806536048 -123.02047047886393) bank186475 +186476 POINT(38.573451797161354 -122.14936810298086) bank186476 +186477 POINT(36.795566332731084 -122.4459489066391) bank186477 +186478 POINT(38.71519333332268 -122.61517840613126) bank186478 +186479 POINT(37.00908403061682 -122.8889366775857) bank186479 +186480 POINT(38.659942628925926 -121.53161785227722) bank186480 +186481 POINT(38.15956014534167 -122.40141480458234) bank186481 +186482 POINT(38.628284712214736 -122.06602097605196) bank186482 +186483 POINT(38.69559324034498 -122.20944568768387) bank186483 +186484 POINT(38.240377502045924 -123.33919670971677) bank186484 +186485 POINT(38.22189676944494 -121.6293610607903) bank186485 +186486 POINT(38.71859697687386 -122.60668848461893) bank186486 +186487 POINT(37.59738437755854 -123.0672016732526) bank186487 +186488 POINT(36.879138246839084 -121.80580708113443) bank186488 +186489 POINT(36.894920826509846 -122.56284346286131) bank186489 +186490 POINT(38.35386407195819 -122.46561833894113) bank186490 +186491 POINT(37.16299834257009 -122.57372786291212) bank186491 +186492 POINT(36.83475683614307 -122.59646672008952) bank186492 +186493 POINT(37.16979058036339 -122.57796061069719) bank186493 +186494 POINT(37.76095823162746 -122.05247300821772) bank186494 +186495 POINT(37.221938455420045 -123.00732887150978) bank186495 +186496 POINT(38.58678741465977 -122.15856875129535) bank186496 +186497 POINT(38.72873514776646 -122.18765616505611) bank186497 +186498 POINT(37.477751008566756 -123.34841580110336) bank186498 +186499 POINT(38.5879105331565 -122.50305061155949) bank186499 +186500 POINT(37.22297203494231 -122.75594929940569) bank186500 +186501 POINT(38.41194921932471 -122.3168097219519) bank186501 +186502 POINT(37.15614800031667 -122.37679898918655) bank186502 +186503 POINT(37.824955827970115 -122.37310541755686) bank186503 +186504 POINT(37.5645830803858 -122.5982507791901) bank186504 +186505 POINT(38.00411554163406 -122.14142642886372) bank186505 +186506 POINT(37.747137834099846 -121.4535453491703) bank186506 +186507 POINT(38.1873544993098 -122.24364067331756) bank186507 +186508 POINT(38.565392397147676 -123.4140332259956) bank186508 +186509 POINT(37.51413888571004 -121.92701765415117) bank186509 +186510 POINT(38.13794326895488 -122.30952066549331) bank186510 +186511 POINT(37.72753081641655 -123.32702656308469) bank186511 +186512 POINT(37.61877126275525 -123.35087068316116) bank186512 +186513 POINT(38.00575573488837 -121.60840634010206) bank186513 +186514 POINT(38.23107918424655 -123.12497728099144) bank186514 +186515 POINT(38.759122884007624 -121.76299915816398) bank186515 +186516 POINT(37.634230724623315 -122.45599479596139) bank186516 +186517 POINT(36.99612066560774 -122.03811589588018) bank186517 +186518 POINT(37.04762023305649 -122.9784117324555) bank186518 +186519 POINT(36.93836791460634 -121.79934466822262) bank186519 +186520 POINT(38.00662272140863 -121.70014939138703) bank186520 +186521 POINT(37.10261324621371 -122.80765402621536) bank186521 +186522 POINT(38.153335227441936 -123.1694968605602) bank186522 +186523 POINT(38.11125736998855 -123.38401608512466) bank186523 +186524 POINT(37.70438250655811 -121.99179196314184) bank186524 +186525 POINT(36.80935615618961 -123.00846594109717) bank186525 +186526 POINT(37.738361943004854 -121.70909087738168) bank186526 +186527 POINT(37.455262795141955 -121.84728785461346) bank186527 +186528 POINT(36.937617703543395 -122.73920229790473) bank186528 +186529 POINT(37.36949315783229 -121.7501293119021) bank186529 +186530 POINT(37.9414913344926 -122.5444879908393) bank186530 +186531 POINT(38.60321267759146 -122.84350847295518) bank186531 +186532 POINT(36.85004331095483 -122.0262231702898) bank186532 +186533 POINT(37.639936107571664 -122.84458207526954) bank186533 +186534 POINT(37.89288771147401 -123.20780791258889) bank186534 +186535 POINT(38.3796635577811 -123.03099484216794) bank186535 +186536 POINT(37.4236391191978 -122.76502389961497) bank186536 +186537 POINT(38.22736145070181 -122.28805386486638) bank186537 +186538 POINT(37.71935298970696 -121.78611039126706) bank186538 +186539 POINT(38.72037317794174 -122.71931518156934) bank186539 +186540 POINT(37.47771498535644 -122.67586926956038) bank186540 +186541 POINT(38.678744662050896 -123.39322833592237) bank186541 +186542 POINT(37.08435484146582 -122.10843179912085) bank186542 +186543 POINT(38.62321940220602 -122.9666492154836) bank186543 +186544 POINT(36.965822742973565 -122.5083109722459) bank186544 +186545 POINT(38.63565513831552 -122.27233324883025) bank186545 +186546 POINT(36.95725774955373 -122.66055580059695) bank186546 +186547 POINT(37.14143923962339 -122.99113396670597) bank186547 +186548 POINT(38.754019037084326 -123.32942321089162) bank186548 +186549 POINT(37.66489859962712 -121.67625265068213) bank186549 +186550 POINT(37.424923745389705 -122.11711619895925) bank186550 +186551 POINT(37.66630697148046 -123.23272455807256) bank186551 +186552 POINT(37.92619166669073 -122.69683669096541) bank186552 +186553 POINT(37.38329643478651 -122.42096533456736) bank186553 +186554 POINT(38.578056410419 -121.9384891037038) bank186554 +186555 POINT(38.43450645913113 -121.72911323563221) bank186555 +186556 POINT(36.94666755235442 -122.74351976893915) bank186556 +186557 POINT(37.280673268858635 -122.98788351097188) bank186557 +186558 POINT(37.899248192613754 -122.64817664282353) bank186558 +186559 POINT(38.201204628399104 -122.2700851596443) bank186559 +186560 POINT(37.73238342261198 -123.29018632516987) bank186560 +186561 POINT(37.20514891009525 -121.96520972550731) bank186561 +186562 POINT(37.28174185421812 -122.86961906051086) bank186562 +186563 POINT(38.28410149209027 -121.72621569149705) bank186563 +186564 POINT(38.167328033839986 -121.63980174119824) bank186564 +186565 POINT(37.26164610244953 -122.78254362147986) bank186565 +186566 POINT(38.71696206055898 -122.50317758097209) bank186566 +186567 POINT(37.66856180991636 -121.46113685110079) bank186567 +186568 POINT(37.800601922903404 -121.81032175143145) bank186568 +186569 POINT(38.34351074144709 -122.44535395610737) bank186569 +186570 POINT(37.077624797663255 -121.72149595217947) bank186570 +186571 POINT(37.990620373387365 -122.13885026235177) bank186571 +186572 POINT(36.96354024375438 -122.48019453660794) bank186572 +186573 POINT(37.077696107983975 -123.00260215561845) bank186573 +186574 POINT(37.53404956088654 -122.17494950443188) bank186574 +186575 POINT(38.31284273104071 -122.65192103012937) bank186575 +186576 POINT(38.551593007264685 -123.15123017817923) bank186576 +186577 POINT(37.585948731300356 -123.04047638448915) bank186577 +186578 POINT(37.21624304401363 -122.67809811822167) bank186578 +186579 POINT(37.22904603623834 -122.6518920220012) bank186579 +186580 POINT(36.86175050954666 -123.30478401579582) bank186580 +186581 POINT(37.64485966500638 -122.41724885573534) bank186581 +186582 POINT(38.76893210155835 -121.47439952590996) bank186582 +186583 POINT(37.33924221949713 -121.54138991660946) bank186583 +186584 POINT(38.51840743373465 -121.45024564068893) bank186584 +186585 POINT(38.57280870871539 -121.52264669780571) bank186585 +186586 POINT(38.14066242916058 -122.43925705780107) bank186586 +186587 POINT(38.227075841567164 -122.43604525447894) bank186587 +186588 POINT(37.17617984662421 -122.1166422382803) bank186588 +186589 POINT(38.58884771049638 -121.43350211357787) bank186589 +186590 POINT(37.31113205794898 -123.02328665879304) bank186590 +186591 POINT(36.853915132486776 -121.86480650474009) bank186591 +186592 POINT(37.01212263814006 -122.97905882762637) bank186592 +186593 POINT(38.33484526807852 -122.65380528210048) bank186593 +186594 POINT(37.83665243426377 -121.70686570707589) bank186594 +186595 POINT(38.34678527185545 -122.61406071664989) bank186595 +186596 POINT(37.17418646128593 -122.13260323215809) bank186596 +186597 POINT(37.599733622408976 -122.8633385231196) bank186597 +186598 POINT(37.244104995627865 -122.94761838749065) bank186598 +186599 POINT(37.6077891617769 -122.99795209855536) bank186599 +186600 POINT(38.44773180657072 -122.32057444016688) bank186600 +186601 POINT(38.18665158079547 -122.41776405453967) bank186601 +186602 POINT(36.977893180213165 -122.01929565828266) bank186602 +186603 POINT(38.17787771520214 -121.85660545168753) bank186603 +186604 POINT(38.64668753783256 -123.35249851837983) bank186604 +186605 POINT(38.73958137065545 -122.11983996472827) bank186605 +186606 POINT(37.95558934758887 -121.4537742473433) bank186606 +186607 POINT(38.02821006416202 -123.04667952784811) bank186607 +186608 POINT(37.00525486573216 -122.91612974004825) bank186608 +186609 POINT(37.993792079759515 -121.52856975551083) bank186609 +186610 POINT(38.72071582839899 -121.76264420876475) bank186610 +186611 POINT(38.40478107284552 -122.37815452371636) bank186611 +186612 POINT(38.43180521125401 -122.07547020585888) bank186612 +186613 POINT(37.09708413965203 -121.50464588093635) bank186613 +186614 POINT(37.17987327641111 -122.76071271899579) bank186614 +186615 POINT(37.63263516656113 -121.44704023686273) bank186615 +186616 POINT(38.70208143251552 -122.12948548210153) bank186616 +186617 POINT(37.1542032239026 -122.97668730525251) bank186617 +186618 POINT(37.61055588208944 -123.04075094718063) bank186618 +186619 POINT(37.635910585426394 -121.97146627699728) bank186619 +186620 POINT(38.75322694899672 -121.49024162729725) bank186620 +186621 POINT(36.96581257222155 -122.14477185761358) bank186621 +186622 POINT(37.73718720304847 -121.51373299047029) bank186622 +186623 POINT(37.11014649784051 -122.61692476055993) bank186623 +186624 POINT(37.37838616482013 -122.82852921869407) bank186624 +186625 POINT(37.126096324936974 -122.78558819529367) bank186625 +186626 POINT(37.24461456972163 -122.60988944410921) bank186626 +186627 POINT(38.418716394823676 -121.81376044843772) bank186627 +186628 POINT(37.06708980725388 -122.30732925783592) bank186628 +186629 POINT(36.970858765498605 -121.83381404814028) bank186629 +186630 POINT(38.18184373427265 -122.91387518872298) bank186630 +186631 POINT(37.640979684834754 -121.80772527651519) bank186631 +186632 POINT(38.65708280648228 -122.75106367223212) bank186632 +186633 POINT(36.97052247339892 -122.12530501297205) bank186633 +186634 POINT(38.49965011428488 -122.1945618062097) bank186634 +186635 POINT(38.369556070787986 -122.69534611976322) bank186635 +186636 POINT(38.248725245954475 -122.18580264736946) bank186636 +186637 POINT(37.611388898279394 -121.46513231142056) bank186637 +186638 POINT(37.86785176629733 -123.1406498792281) bank186638 +186639 POINT(38.55131376642074 -122.8848506541126) bank186639 +186640 POINT(38.167286231077625 -122.14562169720125) bank186640 +186641 POINT(38.665118062970905 -121.86631910639595) bank186641 +186642 POINT(38.4396560308617 -121.62966707706086) bank186642 +186643 POINT(37.99199887543628 -122.87351720210995) bank186643 +186644 POINT(36.87045055374043 -123.22662405736271) bank186644 +186645 POINT(37.51006092961273 -121.58929338204933) bank186645 +186646 POINT(38.11869660568879 -123.16288245975846) bank186646 +186647 POINT(37.45505839138066 -123.07687053004051) bank186647 +186648 POINT(38.08366403834895 -121.56662801280311) bank186648 +186649 POINT(37.340520357576175 -122.19736260634608) bank186649 +186650 POINT(37.16763445189252 -121.64005094655653) bank186650 +186651 POINT(37.63510180880173 -122.86506703725686) bank186651 +186652 POINT(38.38173187248935 -122.63474355204816) bank186652 +186653 POINT(36.94068023732598 -121.59855843326312) bank186653 +186654 POINT(37.76984717794146 -121.68393416631476) bank186654 +186655 POINT(38.600768724135214 -121.67763687953494) bank186655 +186656 POINT(38.40718355796396 -122.40125609785034) bank186656 +186657 POINT(38.70906564054116 -123.15920779993095) bank186657 +186658 POINT(37.586163862177514 -123.16785461299972) bank186658 +186659 POINT(37.308630011069326 -121.82685672190837) bank186659 +186660 POINT(38.67960460509704 -122.21551984788698) bank186660 +186661 POINT(37.26592877704742 -122.09864353644505) bank186661 +186662 POINT(37.72459356033645 -123.4113853503958) bank186662 +186663 POINT(38.14834526959837 -121.9147007097998) bank186663 +186664 POINT(37.86061106934124 -122.38992861341032) bank186664 +186665 POINT(38.66752903864454 -121.89912250910415) bank186665 +186666 POINT(37.589078130044975 -123.23185823614783) bank186666 +186667 POINT(37.70043089085282 -121.81734330276146) bank186667 +186668 POINT(37.76775372948753 -121.50424401934391) bank186668 +186669 POINT(37.97796364885089 -122.44838621261275) bank186669 +186670 POINT(37.93075489340426 -121.90690698973745) bank186670 +186671 POINT(38.441119195772245 -123.28248323302813) bank186671 +186672 POINT(38.49684238287667 -122.53869501630092) bank186672 +186673 POINT(37.90361157172502 -123.15505216957276) bank186673 +186674 POINT(37.60699680489167 -122.7977876880341) bank186674 +186675 POINT(36.83549612424325 -121.59115976756608) bank186675 +186676 POINT(37.09431586414401 -122.41110573890842) bank186676 +186677 POINT(38.15818086428249 -121.9408122792057) bank186677 +186678 POINT(38.37602330719551 -122.48660686448243) bank186678 +186679 POINT(36.85630616931307 -122.00699906768875) bank186679 +186680 POINT(37.56557381596247 -122.252291917145) bank186680 +186681 POINT(36.9932377595565 -122.86480504698477) bank186681 +186682 POINT(37.44578848675805 -123.03806596729012) bank186682 +186683 POINT(38.53405953069117 -121.9189236894182) bank186683 +186684 POINT(37.50180647582435 -121.48802591628126) bank186684 +186685 POINT(38.419528258173024 -122.67474587200343) bank186685 +186686 POINT(36.87825171119978 -123.09820380376968) bank186686 +186687 POINT(37.42463371055919 -122.64732528277452) bank186687 +186688 POINT(37.14379680608843 -122.1230106523124) bank186688 +186689 POINT(37.78825689343476 -121.8003817165114) bank186689 +186690 POINT(37.50645898885148 -122.14843015695357) bank186690 +186691 POINT(37.494753594213435 -123.32686365685399) bank186691 +186692 POINT(38.16095639810735 -122.2176990579976) bank186692 +186693 POINT(38.231243508334664 -123.3770441058922) bank186693 +186694 POINT(38.330737488372485 -121.76131616771482) bank186694 +186695 POINT(36.80765669706553 -121.62284417808681) bank186695 +186696 POINT(38.39416096914932 -121.94167180790825) bank186696 +186697 POINT(37.02139185411243 -121.96586147777263) bank186697 +186698 POINT(36.897642460337124 -123.26824173282802) bank186698 +186699 POINT(37.48157548528373 -121.87173671671601) bank186699 +186700 POINT(37.2828974225231 -122.19258431265247) bank186700 +186701 POINT(38.4716223430093 -123.36603801180226) bank186701 +186702 POINT(37.771994892940604 -122.91397040364878) bank186702 +186703 POINT(38.15046624799344 -121.76632243422834) bank186703 +186704 POINT(38.39315106448391 -121.53807258406793) bank186704 +186705 POINT(38.55014937968674 -121.69301994747639) bank186705 +186706 POINT(38.07050951074344 -122.77955246304248) bank186706 +186707 POINT(38.23698871184427 -122.13859250394376) bank186707 +186708 POINT(38.29572727396412 -121.54520615517897) bank186708 +186709 POINT(38.13165966534654 -122.71402361980364) bank186709 +186710 POINT(36.84221278326446 -122.28646571596163) bank186710 +186711 POINT(36.81147853397054 -121.6307495333475) bank186711 +186712 POINT(37.75624544906516 -121.80691587364784) bank186712 +186713 POINT(38.3736378183458 -122.19873037096234) bank186713 +186714 POINT(38.19527822912758 -123.16202659291358) bank186714 +186715 POINT(38.08977515912463 -122.13201117462643) bank186715 +186716 POINT(38.67917431994983 -122.20328020938148) bank186716 +186717 POINT(38.405439797353964 -121.92744868034391) bank186717 +186718 POINT(37.60693414041313 -122.6853126124135) bank186718 +186719 POINT(37.03313366847295 -121.5088684111817) bank186719 +186720 POINT(38.15660283037058 -121.7094593429595) bank186720 +186721 POINT(37.42852065962177 -122.5109063739898) bank186721 +186722 POINT(37.20990515780277 -123.41505847934512) bank186722 +186723 POINT(38.66918429807872 -122.87114629122456) bank186723 +186724 POINT(38.34251142403673 -121.54696885752217) bank186724 +186725 POINT(37.836253293748356 -122.60685642530554) bank186725 +186726 POINT(37.16438055067839 -121.44910412437316) bank186726 +186727 POINT(37.79178149062051 -123.01035314260902) bank186727 +186728 POINT(37.239566701473024 -123.41506049003566) bank186728 +186729 POINT(37.14678226123359 -122.00805136670591) bank186729 +186730 POINT(38.3034152224558 -122.38800798386613) bank186730 +186731 POINT(36.81726542432084 -122.65420913260559) bank186731 +186732 POINT(38.21557011166594 -122.099788660296) bank186732 +186733 POINT(37.27769590365904 -122.41952266470072) bank186733 +186734 POINT(38.51193526014461 -123.080916446707) bank186734 +186735 POINT(38.2001519058506 -121.84035321154724) bank186735 +186736 POINT(38.20955997443993 -122.28772281557781) bank186736 +186737 POINT(37.32405759659061 -123.31694418316204) bank186737 +186738 POINT(38.52452241025425 -122.58763449111595) bank186738 +186739 POINT(37.751691029118604 -122.89939046185417) bank186739 +186740 POINT(38.153397805579694 -122.21997219887028) bank186740 +186741 POINT(38.60939576319125 -122.37081340871367) bank186741 +186742 POINT(38.5816162237042 -122.01141731356022) bank186742 +186743 POINT(37.28537908128278 -123.03060287847673) bank186743 +186744 POINT(38.22349342582175 -122.06664385743208) bank186744 +186745 POINT(37.15481526130861 -123.28895605861574) bank186745 +186746 POINT(38.163663600512585 -122.35410854888664) bank186746 +186747 POINT(36.96192888705629 -121.6639842428084) bank186747 +186748 POINT(38.670500197334725 -123.13709516691854) bank186748 +186749 POINT(36.9776546510664 -122.20764681458895) bank186749 +186750 POINT(37.643150600522006 -121.90106604258355) bank186750 +186751 POINT(37.73748270398024 -121.94402809687861) bank186751 +186752 POINT(37.499954487960416 -121.78500584981899) bank186752 +186753 POINT(38.34013841096949 -122.90991829265363) bank186753 +186754 POINT(37.529009093638784 -122.92307856799945) bank186754 +186755 POINT(37.11607143357756 -122.51363877539809) bank186755 +186756 POINT(38.23106751024377 -122.74083431816241) bank186756 +186757 POINT(38.48595963335909 -122.90378385916252) bank186757 +186758 POINT(38.03792175922528 -121.73612851901916) bank186758 +186759 POINT(37.26924993832163 -122.56548396208747) bank186759 +186760 POINT(38.47380477747067 -121.52330139040075) bank186760 +186761 POINT(37.441484553271984 -122.84624508062883) bank186761 +186762 POINT(37.42796432964887 -121.43747262624213) bank186762 +186763 POINT(37.25435720466217 -122.22865709450251) bank186763 +186764 POINT(38.07285953216261 -123.34843298236186) bank186764 +186765 POINT(37.514689311644084 -122.7816655802779) bank186765 +186766 POINT(37.79578085976946 -123.19976742142849) bank186766 +186767 POINT(38.479871941364046 -122.45914959889127) bank186767 +186768 POINT(38.3830647316939 -122.46763512912577) bank186768 +186769 POINT(38.689768437971644 -123.25663549270085) bank186769 +186770 POINT(37.17410316270695 -123.25223444978755) bank186770 +186771 POINT(36.99426901176715 -121.74206724834582) bank186771 +186772 POINT(38.58498278300444 -122.41953361777084) bank186772 +186773 POINT(38.07907965342484 -121.66340278899962) bank186773 +186774 POINT(37.49224747285735 -122.64559116115228) bank186774 +186775 POINT(38.53584232829395 -121.96509696629478) bank186775 +186776 POINT(37.31600211879973 -122.00946285806198) bank186776 +186777 POINT(36.82307041931287 -122.30749055856204) bank186777 +186778 POINT(37.15666747990965 -121.90079324957807) bank186778 +186779 POINT(38.584730054700785 -121.4739288042424) bank186779 +186780 POINT(36.79156410079374 -121.91486355963026) bank186780 +186781 POINT(37.8187677837743 -121.75999187634815) bank186781 +186782 POINT(38.16352977226426 -122.01729610238168) bank186782 +186783 POINT(36.821236288035955 -122.0932606289878) bank186783 +186784 POINT(36.845852206529585 -121.9544331872669) bank186784 +186785 POINT(37.095616587115025 -123.09904445044515) bank186785 +186786 POINT(36.90904848962201 -122.1499872292529) bank186786 +186787 POINT(38.629334511951285 -123.20092381662539) bank186787 +186788 POINT(37.28641174614689 -123.36851814158221) bank186788 +186789 POINT(38.107687665607955 -122.52732041577462) bank186789 +186790 POINT(37.94844263409144 -122.24648225080038) bank186790 +186791 POINT(38.19631210942256 -122.6576974127154) bank186791 +186792 POINT(38.155637965734584 -121.96553996008592) bank186792 +186793 POINT(36.8222310108535 -123.40958238750443) bank186793 +186794 POINT(37.57748356884474 -121.68107224332252) bank186794 +186795 POINT(37.74560491859106 -123.04855458576742) bank186795 +186796 POINT(37.90989218871876 -122.45742310479089) bank186796 +186797 POINT(38.17425953401159 -121.63303280947714) bank186797 +186798 POINT(36.83458671003623 -123.30082867584883) bank186798 +186799 POINT(37.63429508184248 -121.858511882427) bank186799 +186800 POINT(37.83889743468723 -123.28646324539551) bank186800 +186801 POINT(38.71877281415604 -122.66947386240935) bank186801 +186802 POINT(37.268624296959 -122.79414666186973) bank186802 +186803 POINT(38.677847459740974 -122.77131445836149) bank186803 +186804 POINT(36.89145992992525 -123.14659372965677) bank186804 +186805 POINT(37.23606304339986 -122.82225843847144) bank186805 +186806 POINT(37.001645071490756 -122.18799073220464) bank186806 +186807 POINT(37.393195567410075 -122.66767105325201) bank186807 +186808 POINT(38.13519595025091 -123.13814032883407) bank186808 +186809 POINT(38.178739013768784 -121.90715549578151) bank186809 +186810 POINT(37.69053421804312 -122.88613080605165) bank186810 +186811 POINT(37.67459202885273 -123.35829558243545) bank186811 +186812 POINT(38.299059528462266 -122.01312706077437) bank186812 +186813 POINT(38.72052926231193 -122.17212345831608) bank186813 +186814 POINT(37.58015518460233 -123.40894756787084) bank186814 +186815 POINT(38.139548610467166 -122.01799545345504) bank186815 +186816 POINT(37.612384681554474 -123.10271608733017) bank186816 +186817 POINT(37.52176802720565 -121.80757441509553) bank186817 +186818 POINT(37.373958397261575 -123.26760633607769) bank186818 +186819 POINT(37.44222513352574 -123.24938569454602) bank186819 +186820 POINT(37.95011092171795 -121.74274444693286) bank186820 +186821 POINT(36.856617076406444 -123.11007188482934) bank186821 +186822 POINT(37.68666872667738 -122.95202274252756) bank186822 +186823 POINT(38.170030125848896 -121.97949880993978) bank186823 +186824 POINT(37.21417447775786 -122.7657614862208) bank186824 +186825 POINT(38.19111559469289 -123.31561011618092) bank186825 +186826 POINT(38.55840335365123 -123.20947651389393) bank186826 +186827 POINT(37.30520260411721 -122.20062392713561) bank186827 +186828 POINT(37.52519442339399 -121.74884601525304) bank186828 +186829 POINT(37.93074133263323 -122.90462484996651) bank186829 +186830 POINT(37.994568203820315 -121.74214049424705) bank186830 +186831 POINT(37.781107528881556 -122.63861206376946) bank186831 +186832 POINT(38.134175013632024 -122.1875474415312) bank186832 +186833 POINT(37.13131085479995 -122.78241774688112) bank186833 +186834 POINT(37.492668713241635 -122.68284389807586) bank186834 +186835 POINT(37.370177326506806 -123.27297867562933) bank186835 +186836 POINT(37.100389056605785 -123.37752959063764) bank186836 +186837 POINT(37.97942233521793 -123.06025598339463) bank186837 +186838 POINT(36.815694369784445 -122.43188028169541) bank186838 +186839 POINT(37.52243145251599 -122.2050915849579) bank186839 +186840 POINT(38.263738712962926 -121.9626876391549) bank186840 +186841 POINT(38.7594733179124 -122.7088208493917) bank186841 +186842 POINT(38.035187731218585 -122.3679344131186) bank186842 +186843 POINT(37.81225911835781 -122.29929872459215) bank186843 +186844 POINT(37.04012694109313 -121.77618140861598) bank186844 +186845 POINT(38.6575003596172 -122.48814544408626) bank186845 +186846 POINT(38.68957569689338 -121.90578565676348) bank186846 +186847 POINT(36.92229948209525 -122.35633358205577) bank186847 +186848 POINT(38.41974923592563 -122.47263629252424) bank186848 +186849 POINT(37.843153106959086 -123.19081079289722) bank186849 +186850 POINT(37.61508237251994 -122.20317756181531) bank186850 +186851 POINT(37.2463381962647 -123.3417431395653) bank186851 +186852 POINT(37.499394979680154 -122.52250453452444) bank186852 +186853 POINT(37.095985112303495 -121.43906693232486) bank186853 +186854 POINT(37.521928275915144 -122.54072830603005) bank186854 +186855 POINT(37.685393048814504 -121.53630897717946) bank186855 +186856 POINT(37.45479405580167 -122.73565389279885) bank186856 +186857 POINT(38.70395092887651 -122.60382867582277) bank186857 +186858 POINT(38.17755601904905 -122.35191010882721) bank186858 +186859 POINT(36.790712611212115 -121.89711659792293) bank186859 +186860 POINT(38.41780255530906 -121.44803636439644) bank186860 +186861 POINT(37.93670043326392 -122.72133069832958) bank186861 +186862 POINT(37.57461388750314 -123.14500370743497) bank186862 +186863 POINT(38.562019817581024 -123.19965873493425) bank186863 +186864 POINT(37.78737799203006 -122.73130267339167) bank186864 +186865 POINT(38.713752487496876 -123.3397208248511) bank186865 +186866 POINT(37.20078498662365 -122.00825803727216) bank186866 +186867 POINT(37.46215529483931 -123.14863456363653) bank186867 +186868 POINT(38.54617186528485 -123.28273174939719) bank186868 +186869 POINT(38.22220120643642 -122.23203582361239) bank186869 +186870 POINT(37.43649050233865 -123.39963300939495) bank186870 +186871 POINT(37.83729248273217 -122.1385390960662) bank186871 +186872 POINT(38.656573306149824 -122.61631421602303) bank186872 +186873 POINT(38.742277996838325 -121.82187427640142) bank186873 +186874 POINT(37.47343917789243 -122.57347513260483) bank186874 +186875 POINT(37.96797900249775 -122.15167292364019) bank186875 +186876 POINT(37.98852804780936 -121.7340337587575) bank186876 +186877 POINT(37.56682682383067 -122.18281730993962) bank186877 +186878 POINT(38.1405885421828 -122.20611283021645) bank186878 +186879 POINT(37.11812539254056 -122.48363902974748) bank186879 +186880 POINT(37.6776319322031 -123.36719999293527) bank186880 +186881 POINT(37.13989833241883 -122.1405686059418) bank186881 +186882 POINT(36.80825149065326 -122.71032563242451) bank186882 +186883 POINT(38.39009968794775 -122.93466388871848) bank186883 +186884 POINT(36.80317472395503 -121.84767607226098) bank186884 +186885 POINT(37.67839878509429 -121.54406688299336) bank186885 +186886 POINT(37.220031278730865 -122.08762700143949) bank186886 +186887 POINT(38.751909219463414 -121.61890835026163) bank186887 +186888 POINT(37.36716795190154 -121.4415522474408) bank186888 +186889 POINT(38.64173383632732 -122.85680317794632) bank186889 +186890 POINT(38.228336372301364 -122.80665526141783) bank186890 +186891 POINT(36.932633060772154 -121.42508295259204) bank186891 +186892 POINT(36.89275857231742 -123.09181517111209) bank186892 +186893 POINT(37.06977064548715 -122.13449910445213) bank186893 +186894 POINT(37.083744556102474 -121.49482560868638) bank186894 +186895 POINT(36.90562728394737 -122.65689618713871) bank186895 +186896 POINT(36.82711607232687 -122.95612116498303) bank186896 +186897 POINT(38.34845636741938 -121.46876117730949) bank186897 +186898 POINT(36.91963549111478 -123.35543412294153) bank186898 +186899 POINT(38.04068211529824 -122.80294954163762) bank186899 +186900 POINT(38.01978673215856 -121.85689901291508) bank186900 +186901 POINT(38.57271625237153 -123.3573657642203) bank186901 +186902 POINT(38.55478563447452 -122.55622719215461) bank186902 +186903 POINT(37.74161491927063 -121.9125959151094) bank186903 +186904 POINT(37.79893072013553 -122.64246239942503) bank186904 +186905 POINT(37.98850787178632 -122.26939364337912) bank186905 +186906 POINT(36.86740290084446 -121.77085843105368) bank186906 +186907 POINT(38.06657975418686 -123.24432233290793) bank186907 +186908 POINT(37.582841303221265 -122.34170518201881) bank186908 +186909 POINT(37.26497468887269 -121.56967538845596) bank186909 +186910 POINT(37.96778554075706 -122.41434389445563) bank186910 +186911 POINT(38.61901530369383 -123.09920178978318) bank186911 +186912 POINT(37.01615167310748 -123.2734225962338) bank186912 +186913 POINT(37.786796363074274 -122.45778250231463) bank186913 +186914 POINT(36.823442236216366 -121.71521819983913) bank186914 +186915 POINT(38.10681711992378 -122.71798535813147) bank186915 +186916 POINT(37.99062869696656 -122.6498589582364) bank186916 +186917 POINT(38.530825743988736 -121.44453216524583) bank186917 +186918 POINT(37.11505582266567 -122.98279459499807) bank186918 +186919 POINT(37.693913679704714 -122.7447721968975) bank186919 +186920 POINT(38.095839339315404 -122.25063740427093) bank186920 +186921 POINT(36.828658864467215 -122.31266535502705) bank186921 +186922 POINT(38.022674972869694 -122.54351528713015) bank186922 +186923 POINT(36.88054816852226 -122.527193541875) bank186923 +186924 POINT(38.1131608917905 -122.87883302975328) bank186924 +186925 POINT(38.01282397601827 -122.21968747212226) bank186925 +186926 POINT(38.744465825561 -122.0574191780912) bank186926 +186927 POINT(38.710844093169314 -121.45373894081632) bank186927 +186928 POINT(38.06611677084828 -123.01672992356929) bank186928 +186929 POINT(37.95823679049216 -123.08006498027159) bank186929 +186930 POINT(37.78183522698549 -122.58905643467693) bank186930 +186931 POINT(37.53759090423672 -122.20586361607509) bank186931 +186932 POINT(38.35920991584635 -123.23111120808781) bank186932 +186933 POINT(38.62765773388234 -122.37877330330103) bank186933 +186934 POINT(37.94904859177588 -123.20921287852052) bank186934 +186935 POINT(37.77827593199855 -122.43573991553282) bank186935 +186936 POINT(38.11853849268481 -122.84152888626315) bank186936 +186937 POINT(38.60637013600925 -121.92696955240578) bank186937 +186938 POINT(36.957516133826694 -122.87617930541532) bank186938 +186939 POINT(37.75680654845785 -121.8519605315215) bank186939 +186940 POINT(38.3769729278493 -121.74025883057485) bank186940 +186941 POINT(37.41641551668879 -122.1228914824298) bank186941 +186942 POINT(37.7545466295379 -121.5660710898497) bank186942 +186943 POINT(38.629201643351486 -121.46213663424815) bank186943 +186944 POINT(38.17097881364376 -123.19005049443045) bank186944 +186945 POINT(37.290879902707694 -122.29935886887549) bank186945 +186946 POINT(37.747860947921566 -122.30055404725819) bank186946 +186947 POINT(38.66886636984548 -122.03593213135422) bank186947 +186948 POINT(37.42312736381674 -122.0715617179375) bank186948 +186949 POINT(38.33281852272462 -122.57423051211445) bank186949 +186950 POINT(37.78994492443674 -121.96064547851253) bank186950 +186951 POINT(36.84304546467197 -123.06126010836259) bank186951 +186952 POINT(38.13406711903294 -123.09348017342718) bank186952 +186953 POINT(36.967687523206365 -123.01285164767543) bank186953 +186954 POINT(36.8000804085054 -123.09805828184484) bank186954 +186955 POINT(37.67938817101902 -122.03767491987375) bank186955 +186956 POINT(38.49702541444656 -122.33378028271531) bank186956 +186957 POINT(38.63440006906214 -122.40352518415078) bank186957 +186958 POINT(38.522628215905236 -122.93782107350026) bank186958 +186959 POINT(37.23856314859262 -122.37699873024576) bank186959 +186960 POINT(37.97601492830447 -122.23060817952128) bank186960 +186961 POINT(38.60460189225224 -121.51144344354942) bank186961 +186962 POINT(38.58097673745178 -123.28961986221114) bank186962 +186963 POINT(37.17719773497474 -123.2040922044134) bank186963 +186964 POINT(38.05923413532452 -123.3822872412905) bank186964 +186965 POINT(37.408655321273216 -122.77398162067755) bank186965 +186966 POINT(37.27291704752697 -122.02714479849116) bank186966 +186967 POINT(38.46069139409905 -123.08781427915024) bank186967 +186968 POINT(37.328524931320366 -123.11986541294705) bank186968 +186969 POINT(36.81582047708464 -121.92796167147635) bank186969 +186970 POINT(38.649226553780814 -121.67074484170382) bank186970 +186971 POINT(38.50473116448031 -123.17698259200138) bank186971 +186972 POINT(37.25154865361322 -122.3469501221856) bank186972 +186973 POINT(38.25514011584994 -122.9168554742408) bank186973 +186974 POINT(37.90025880766861 -123.15069959391919) bank186974 +186975 POINT(37.33079972198182 -122.03862335205565) bank186975 +186976 POINT(37.786243342082024 -123.15291007034233) bank186976 +186977 POINT(38.15637893083202 -121.85856259337322) bank186977 +186978 POINT(38.02509729227729 -122.62504101142433) bank186978 +186979 POINT(37.88194343499047 -122.28280513899297) bank186979 +186980 POINT(37.53232359899107 -122.75637949483159) bank186980 +186981 POINT(37.26150815075311 -122.2400307168548) bank186981 +186982 POINT(37.031491938893005 -122.94575331675743) bank186982 +186983 POINT(38.58783531809036 -122.74652271004487) bank186983 +186984 POINT(37.1556008419488 -122.34108418068594) bank186984 +186985 POINT(37.36239355901218 -121.55242671510653) bank186985 +186986 POINT(36.893704343678884 -121.66119873874689) bank186986 +186987 POINT(37.71548700663174 -121.91698356994047) bank186987 +186988 POINT(38.63276783903446 -121.96799074584584) bank186988 +186989 POINT(37.800050488710475 -123.24915112553234) bank186989 +186990 POINT(37.94903942000106 -122.56151395481491) bank186990 +186991 POINT(38.10093432952486 -121.44329502056729) bank186991 +186992 POINT(37.83767712373778 -123.22773961653097) bank186992 +186993 POINT(37.12515647604446 -121.82692510809967) bank186993 +186994 POINT(37.77218686935479 -121.50438707720919) bank186994 +186995 POINT(38.28711474608478 -123.35610599906057) bank186995 +186996 POINT(37.58581196112587 -121.92557637327889) bank186996 +186997 POINT(38.45426777834809 -121.74964471058139) bank186997 +186998 POINT(37.50144416518065 -121.70957125318127) bank186998 +186999 POINT(37.98530263728604 -122.96094923744583) bank186999 +187000 POINT(37.34640253555211 -121.6072729407401) bank187000 +187001 POINT(37.7911633749242 -121.84078847897962) bank187001 +187002 POINT(37.694032754503574 -122.0807241299071) bank187002 +187003 POINT(36.86967754055448 -121.94615161577508) bank187003 +187004 POINT(36.96239645260958 -122.06512241443096) bank187004 +187005 POINT(37.23132348474581 -121.91313661026031) bank187005 +187006 POINT(37.26741119287708 -122.9759387110688) bank187006 +187007 POINT(38.15772347018026 -123.31902513918264) bank187007 +187008 POINT(37.567345126636255 -123.11868272061342) bank187008 +187009 POINT(37.52445362658185 -123.05647574224258) bank187009 +187010 POINT(37.60242192783751 -123.18380039956665) bank187010 +187011 POINT(38.634539777655014 -121.98543954534571) bank187011 +187012 POINT(37.256049971372654 -121.44110074690609) bank187012 +187013 POINT(37.409611583867786 -122.9730740103381) bank187013 +187014 POINT(37.10035471278397 -121.94002468935282) bank187014 +187015 POINT(37.36094212188548 -121.9128165870628) bank187015 +187016 POINT(36.86389540187867 -122.97850924939492) bank187016 +187017 POINT(37.060048900783976 -122.50512678199908) bank187017 +187018 POINT(38.74267852279128 -122.11024190133556) bank187018 +187019 POINT(37.05353250740022 -123.13670146096392) bank187019 +187020 POINT(38.20120937912667 -121.72056148796872) bank187020 +187021 POINT(36.84129905123283 -121.79683088371708) bank187021 +187022 POINT(37.807304108726115 -122.89074424809513) bank187022 +187023 POINT(38.08377938355004 -123.28414098538798) bank187023 +187024 POINT(37.57579229917584 -121.84604460417916) bank187024 +187025 POINT(38.72690691890514 -121.58776955396787) bank187025 +187026 POINT(36.96749698593581 -121.57343037389194) bank187026 +187027 POINT(38.57132399237938 -122.5709964155637) bank187027 +187028 POINT(37.93867439655786 -122.52461541009109) bank187028 +187029 POINT(37.30539936848784 -121.70043758494057) bank187029 +187030 POINT(37.479642214301506 -121.59495645130542) bank187030 +187031 POINT(38.337908592366254 -123.3715324449711) bank187031 +187032 POINT(38.41375828125943 -122.23107743856707) bank187032 +187033 POINT(36.933517582569294 -122.79890419733351) bank187033 +187034 POINT(38.684951481474926 -122.7655307328556) bank187034 +187035 POINT(37.1122892374393 -123.1634899337673) bank187035 +187036 POINT(37.23952489789948 -122.72884315094498) bank187036 +187037 POINT(37.664009351412595 -122.91318119474755) bank187037 +187038 POINT(37.76706294279672 -122.35106327790518) bank187038 +187039 POINT(37.04508582983176 -122.94283654668136) bank187039 +187040 POINT(38.24338796265895 -122.54704472309268) bank187040 +187041 POINT(37.59949031324609 -121.47339828360595) bank187041 +187042 POINT(37.46977091124678 -121.88832293984782) bank187042 +187043 POINT(37.831602309308835 -121.93263726000146) bank187043 +187044 POINT(38.54845357546712 -123.09998329031428) bank187044 +187045 POINT(37.25870924115838 -122.23792785431678) bank187045 +187046 POINT(37.92507479068781 -122.14290930589705) bank187046 +187047 POINT(37.43383345907286 -121.63859398916485) bank187047 +187048 POINT(37.65547137978787 -122.01888235113816) bank187048 +187049 POINT(38.18397099141389 -123.220536215216) bank187049 +187050 POINT(38.21665020187927 -123.30424356528317) bank187050 +187051 POINT(37.71874734656128 -122.98002859204365) bank187051 +187052 POINT(36.78206785325824 -122.36825781721353) bank187052 +187053 POINT(37.898324451204346 -122.4117483902245) bank187053 +187054 POINT(38.164471451091046 -123.18004437791167) bank187054 +187055 POINT(37.00418895929015 -122.53333999359118) bank187055 +187056 POINT(38.536511120903214 -122.5599429641599) bank187056 +187057 POINT(37.82257710879372 -121.48355772813728) bank187057 +187058 POINT(37.38967735330177 -122.5111803225776) bank187058 +187059 POINT(37.871786793818174 -122.83487612900744) bank187059 +187060 POINT(38.10612812446161 -121.55842789805915) bank187060 +187061 POINT(38.369797642326596 -123.40041271208331) bank187061 +187062 POINT(36.88755499903141 -123.29536778093653) bank187062 +187063 POINT(36.88611292472534 -122.32320723487548) bank187063 +187064 POINT(37.81130003030926 -122.83469049100107) bank187064 +187065 POINT(37.60684405312149 -122.3168430024305) bank187065 +187066 POINT(36.87275923875323 -122.05145529594724) bank187066 +187067 POINT(37.937644379094024 -122.11435298489683) bank187067 +187068 POINT(37.25347573499643 -122.4246759325749) bank187068 +187069 POINT(37.37084784387601 -122.75818238756452) bank187069 +187070 POINT(38.72888962344645 -122.64743957695957) bank187070 +187071 POINT(37.341630418901666 -122.74638553960396) bank187071 +187072 POINT(37.07220962293786 -122.98644219834112) bank187072 +187073 POINT(36.78805449395186 -122.75435628679642) bank187073 +187074 POINT(37.105561636110636 -123.22495955181802) bank187074 +187075 POINT(38.60520573495831 -122.18076163655599) bank187075 +187076 POINT(37.18385644804626 -122.35989672312716) bank187076 +187077 POINT(37.31195546666884 -122.02349258950463) bank187077 +187078 POINT(37.24723434623416 -122.34313182389616) bank187078 +187079 POINT(36.968834794120895 -121.87302306019517) bank187079 +187080 POINT(37.456817872120766 -122.54357134350809) bank187080 +187081 POINT(37.29489458619582 -122.09238779003229) bank187081 +187082 POINT(37.44143494668771 -123.2197628175102) bank187082 +187083 POINT(37.552172297801164 -122.20554034029428) bank187083 +187084 POINT(37.21931935504492 -122.32194754880902) bank187084 +187085 POINT(36.90411655118539 -122.03005872119974) bank187085 +187086 POINT(37.67730363862686 -121.58405221854433) bank187086 +187087 POINT(36.95275950057594 -123.33873439748574) bank187087 +187088 POINT(38.07759801988781 -122.7453771648698) bank187088 +187089 POINT(38.47847000046932 -123.40650094360393) bank187089 +187090 POINT(37.1847251344914 -122.62054133846785) bank187090 +187091 POINT(38.375649880835546 -122.37332911863216) bank187091 +187092 POINT(38.08834789747356 -122.06020593913276) bank187092 +187093 POINT(38.08347004704147 -121.68998922966955) bank187093 +187094 POINT(38.047403410091555 -123.40892092516252) bank187094 +187095 POINT(37.44484054036555 -122.05600288134856) bank187095 +187096 POINT(37.65978685898934 -122.66517414938268) bank187096 +187097 POINT(37.680029549622716 -122.28077906688813) bank187097 +187098 POINT(38.238708789398785 -122.75944228529904) bank187098 +187099 POINT(37.85568411925615 -122.48344491958436) bank187099 +187100 POINT(36.93523398398699 -123.35027451891717) bank187100 +187101 POINT(38.217420708657784 -123.34878775578149) bank187101 +187102 POINT(38.11207534408992 -122.39365647191224) bank187102 +187103 POINT(38.078400195199094 -121.57639281313485) bank187103 +187104 POINT(37.18554356075023 -122.53263750971122) bank187104 +187105 POINT(37.516753014670016 -123.27378658636275) bank187105 +187106 POINT(38.43003868242264 -122.90691075319353) bank187106 +187107 POINT(38.616149548247456 -122.38156177093535) bank187107 +187108 POINT(36.84207110944099 -122.30552093047353) bank187108 +187109 POINT(38.77198675888662 -121.88380428460397) bank187109 +187110 POINT(38.31931339322557 -123.1735705592502) bank187110 +187111 POINT(37.59241952410742 -122.73558789861048) bank187111 +187112 POINT(37.28605499457467 -121.70752303076449) bank187112 +187113 POINT(37.73531602204018 -121.63577437108441) bank187113 +187114 POINT(38.07566925092657 -122.2874253695105) bank187114 +187115 POINT(38.164257232739914 -122.57881212723598) bank187115 +187116 POINT(37.570319090383045 -121.98761384718543) bank187116 +187117 POINT(37.45850347197276 -123.31739204782785) bank187117 +187118 POINT(38.05656410936037 -121.92503248250905) bank187118 +187119 POINT(38.0649091525314 -122.92929491242573) bank187119 +187120 POINT(38.193315487391146 -122.44655897734863) bank187120 +187121 POINT(38.139104342599545 -123.01744827402746) bank187121 +187122 POINT(37.36612968148079 -122.24687746041431) bank187122 +187123 POINT(37.50099271036126 -122.58303196377913) bank187123 +187124 POINT(38.18025455718396 -122.91599520471715) bank187124 +187125 POINT(37.90806914825404 -123.0792532224994) bank187125 +187126 POINT(38.349929478523315 -122.96847804117738) bank187126 +187127 POINT(36.778925116472124 -121.95681898735108) bank187127 +187128 POINT(38.44857453443243 -122.47399161911133) bank187128 +187129 POINT(37.11865840833061 -122.02406182333957) bank187129 +187130 POINT(36.88623264223615 -121.49785859043384) bank187130 +187131 POINT(36.91695781893859 -123.01511541569978) bank187131 +187132 POINT(38.1663122502635 -122.20171903196226) bank187132 +187133 POINT(37.225412225063266 -121.69517234586455) bank187133 +187134 POINT(38.175016814962206 -122.4940793740051) bank187134 +187135 POINT(37.65439519754033 -122.14325357089778) bank187135 +187136 POINT(37.97860650539996 -121.98686828214856) bank187136 +187137 POINT(38.70418098507843 -122.2140962849138) bank187137 +187138 POINT(38.181668888338066 -122.16026143890151) bank187138 +187139 POINT(38.31129076166336 -123.02278819666549) bank187139 +187140 POINT(37.868111445798974 -121.43471096901581) bank187140 +187141 POINT(36.85076430660857 -121.63649375551546) bank187141 +187142 POINT(37.270871983836116 -121.82408578577018) bank187142 +187143 POINT(37.02811086630291 -123.15717566189086) bank187143 +187144 POINT(38.368078849567006 -122.7258279487643) bank187144 +187145 POINT(38.67180679743081 -121.67239848043745) bank187145 +187146 POINT(37.514852757074685 -122.09063563999626) bank187146 +187147 POINT(37.89631305724439 -123.20882094635522) bank187147 +187148 POINT(37.60213727975531 -123.34790870944677) bank187148 +187149 POINT(38.57059256996942 -121.66944315816252) bank187149 +187150 POINT(37.16103048776062 -122.50403283622012) bank187150 +187151 POINT(37.38229229757619 -122.62682873813603) bank187151 +187152 POINT(37.71830757818925 -123.14283648687571) bank187152 +187153 POINT(37.775647475453255 -121.42784166886838) bank187153 +187154 POINT(36.83841053178382 -122.99283522426562) bank187154 +187155 POINT(38.52254323140436 -121.59586475233722) bank187155 +187156 POINT(37.42407169862324 -122.4108997972951) bank187156 +187157 POINT(36.95664100719979 -122.0350911504142) bank187157 +187158 POINT(37.48293671271743 -121.46312454720538) bank187158 +187159 POINT(38.7099682787499 -123.29071345048594) bank187159 +187160 POINT(38.14717614700421 -121.62079323000358) bank187160 +187161 POINT(37.38265284272211 -121.62533612401113) bank187161 +187162 POINT(38.72074288943974 -121.70793985898518) bank187162 +187163 POINT(37.05075387753164 -122.76944723118491) bank187163 +187164 POINT(38.295018809900114 -123.15929118394396) bank187164 +187165 POINT(37.44666789755611 -122.94190516375257) bank187165 +187166 POINT(37.642306548879894 -121.69293631557207) bank187166 +187167 POINT(37.257185735024095 -122.42809263303019) bank187167 +187168 POINT(37.83854410299489 -121.70701390931177) bank187168 +187169 POINT(38.37105008118185 -123.24160284857537) bank187169 +187170 POINT(38.45341678577949 -122.10676185430414) bank187170 +187171 POINT(37.751448627967804 -123.23839464537947) bank187171 +187172 POINT(36.841472209300726 -122.42127611769938) bank187172 +187173 POINT(36.90488959138363 -121.46157689061357) bank187173 +187174 POINT(38.52832830178555 -122.95944162911897) bank187174 +187175 POINT(38.01464653050237 -122.87256284315022) bank187175 +187176 POINT(38.30293370281796 -122.88175061038254) bank187176 +187177 POINT(38.28808001318951 -121.9521603930523) bank187177 +187178 POINT(38.43191036292188 -121.97549316011744) bank187178 +187179 POINT(37.422472723600706 -122.91232220386252) bank187179 +187180 POINT(37.80652576818059 -122.99024948808572) bank187180 +187181 POINT(37.72411411310195 -121.82517994336585) bank187181 +187182 POINT(38.162024790240324 -122.04614964338072) bank187182 +187183 POINT(37.40631832736009 -122.2524487249265) bank187183 +187184 POINT(38.20336035051059 -121.51487663107905) bank187184 +187185 POINT(37.23651456945023 -123.39979879720727) bank187185 +187186 POINT(36.91219643614704 -122.92362795699125) bank187186 +187187 POINT(38.71478002441801 -122.98481913901298) bank187187 +187188 POINT(38.359328156568466 -122.21242027969116) bank187188 +187189 POINT(38.72124814848958 -122.76411080596498) bank187189 +187190 POINT(38.15276725266856 -122.42849079564891) bank187190 +187191 POINT(37.874596939661124 -121.85164354168407) bank187191 +187192 POINT(36.91310242100657 -122.63461497629676) bank187192 +187193 POINT(37.871078750556805 -122.48347356072806) bank187193 +187194 POINT(38.135325099972945 -121.70080421891547) bank187194 +187195 POINT(37.98603312115479 -122.35736428294) bank187195 +187196 POINT(37.699543071251036 -121.94402858124514) bank187196 +187197 POINT(37.58975364885032 -122.29429971191922) bank187197 +187198 POINT(38.180012609595074 -122.6259347625664) bank187198 +187199 POINT(37.97111327439712 -122.51685339497033) bank187199 +187200 POINT(37.13965405255158 -121.87489243231596) bank187200 +187201 POINT(37.32311883243001 -122.61345466086183) bank187201 +187202 POINT(38.485366853742285 -122.03470574274836) bank187202 +187203 POINT(38.219524177456314 -122.51827345816652) bank187203 +187204 POINT(36.99703162087681 -121.89283310598363) bank187204 +187205 POINT(37.98596721867059 -123.03888727881181) bank187205 +187206 POINT(38.75087035316663 -122.1080129702657) bank187206 +187207 POINT(37.98144088901104 -123.01174438150413) bank187207 +187208 POINT(37.09696869111045 -121.73774475135164) bank187208 +187209 POINT(38.11168169178801 -123.31405875287983) bank187209 +187210 POINT(37.950919959565276 -122.80154817736155) bank187210 +187211 POINT(38.245736743082006 -123.13909898417396) bank187211 +187212 POINT(38.55115348261689 -121.45545542586008) bank187212 +187213 POINT(37.204664924003744 -122.32153810999021) bank187213 +187214 POINT(36.990722374565735 -123.04340944671223) bank187214 +187215 POINT(38.48803643612266 -122.62292343839808) bank187215 +187216 POINT(37.01453388613243 -121.67486405126789) bank187216 +187217 POINT(36.928016312708856 -122.51996667173194) bank187217 +187218 POINT(37.26001191330953 -121.99604440663477) bank187218 +187219 POINT(38.685452977742976 -122.68919046807916) bank187219 +187220 POINT(38.08708382742295 -122.41153512124512) bank187220 +187221 POINT(37.13523405788253 -123.31633875755202) bank187221 +187222 POINT(37.46182039622031 -122.88422679725505) bank187222 +187223 POINT(37.23195790559397 -122.95892825076844) bank187223 +187224 POINT(37.35298198254199 -122.7949034470378) bank187224 +187225 POINT(38.71468304138292 -122.35338929548618) bank187225 +187226 POINT(37.80930020304438 -121.6653838895496) bank187226 +187227 POINT(38.771837441242695 -122.422280900007) bank187227 +187228 POINT(37.903485841633746 -121.48019465499208) bank187228 +187229 POINT(37.25377866688655 -122.0343779242051) bank187229 +187230 POINT(37.19773879867009 -122.10354458903808) bank187230 +187231 POINT(38.68850413735332 -121.61570477480487) bank187231 +187232 POINT(38.12372773718367 -123.3047188375396) bank187232 +187233 POINT(37.846966508561444 -122.8424781086124) bank187233 +187234 POINT(38.659798225024154 -121.48645891741685) bank187234 +187235 POINT(36.86167096454725 -122.16425565922128) bank187235 +187236 POINT(36.93487646603293 -123.20086015207563) bank187236 +187237 POINT(37.32039009611789 -122.4853060831217) bank187237 +187238 POINT(37.42682546657056 -122.81088464923977) bank187238 +187239 POINT(38.23076202392827 -121.48076562777294) bank187239 +187240 POINT(37.09034955024912 -121.43510192454234) bank187240 +187241 POINT(37.61085800463828 -123.3894546195747) bank187241 +187242 POINT(37.561354048991205 -122.68806475194464) bank187242 +187243 POINT(37.74378218170788 -123.09028702536204) bank187243 +187244 POINT(37.92914943397518 -122.86519440922027) bank187244 +187245 POINT(38.10421257367168 -122.7391399279698) bank187245 +187246 POINT(38.21596023357874 -122.29804059841733) bank187246 +187247 POINT(37.92515476088014 -121.94330004843322) bank187247 +187248 POINT(36.85638479964113 -122.29001967277483) bank187248 +187249 POINT(36.96598168477043 -121.52750852759341) bank187249 +187250 POINT(37.880510753046636 -122.22029669934847) bank187250 +187251 POINT(38.47714164944607 -122.6137889894656) bank187251 +187252 POINT(37.28578263018173 -121.4342177339156) bank187252 +187253 POINT(37.96366381513564 -122.58663794197489) bank187253 +187254 POINT(36.99255485771229 -122.2034349117748) bank187254 +187255 POINT(37.55547372401454 -122.9630188094893) bank187255 +187256 POINT(37.22983014210725 -121.61058063719544) bank187256 +187257 POINT(37.751682490923066 -123.14577111345349) bank187257 +187258 POINT(37.521098141474226 -122.92604719747776) bank187258 +187259 POINT(38.42599990283743 -122.77383015974685) bank187259 +187260 POINT(38.008764888093076 -122.69361038343303) bank187260 +187261 POINT(38.550017419480525 -121.85699514766246) bank187261 +187262 POINT(37.72987101950735 -121.6421263702715) bank187262 +187263 POINT(38.31153307036004 -122.25242825468838) bank187263 +187264 POINT(36.80907060769001 -122.85211348031383) bank187264 +187265 POINT(38.28188508448709 -122.20903720925268) bank187265 +187266 POINT(38.77298426007982 -122.87906027853731) bank187266 +187267 POINT(38.13118989899539 -122.71810044613738) bank187267 +187268 POINT(37.61975944481311 -121.73321965712226) bank187268 +187269 POINT(37.305676804075645 -123.09764061727873) bank187269 +187270 POINT(37.2986466908626 -122.38069045020795) bank187270 +187271 POINT(38.16156560509918 -122.27886486747583) bank187271 +187272 POINT(37.15560164153033 -121.5121539046407) bank187272 +187273 POINT(38.12634343750249 -121.61951589182546) bank187273 +187274 POINT(37.87204769244189 -122.9163004267172) bank187274 +187275 POINT(38.46153449445155 -122.28591118521295) bank187275 +187276 POINT(38.589250869759056 -122.2746959873278) bank187276 +187277 POINT(36.81742372743616 -121.88607986822282) bank187277 +187278 POINT(36.92528244711206 -121.78405989926495) bank187278 +187279 POINT(37.96752664218268 -122.83208968067417) bank187279 +187280 POINT(38.54813964804102 -122.19520094758717) bank187280 +187281 POINT(36.81995848345208 -122.05945249335339) bank187281 +187282 POINT(37.87274702623331 -121.96591164660033) bank187282 +187283 POINT(37.652980254285744 -121.55729122925732) bank187283 +187284 POINT(37.7803802289344 -121.47850932465553) bank187284 +187285 POINT(38.494362819775596 -122.95426395742496) bank187285 +187286 POINT(37.043660055804885 -121.88384772334632) bank187286 +187287 POINT(38.009493327586306 -122.81545533920952) bank187287 +187288 POINT(36.819469172074704 -121.89533663125127) bank187288 +187289 POINT(37.932694446965854 -121.63129644789699) bank187289 +187290 POINT(37.949879188169746 -122.87826365203091) bank187290 +187291 POINT(38.45061174672523 -121.55542706589654) bank187291 +187292 POINT(37.081567924721256 -121.96920432536251) bank187292 +187293 POINT(37.535650389947236 -122.65209054931671) bank187293 +187294 POINT(38.3274841490552 -121.63069815307463) bank187294 +187295 POINT(38.416464291296585 -122.9599951011297) bank187295 +187296 POINT(37.63369066403624 -122.10686795844704) bank187296 +187297 POINT(38.41849622095101 -121.55505885946607) bank187297 +187298 POINT(37.66335044100577 -121.53370127214747) bank187298 +187299 POINT(37.070800674777445 -122.86742646962196) bank187299 +187300 POINT(37.6206519740468 -123.00871684837992) bank187300 +187301 POINT(37.808516713292626 -122.25753976753829) bank187301 +187302 POINT(36.89315431985591 -122.90435988884389) bank187302 +187303 POINT(36.963455370469426 -121.91170376950284) bank187303 +187304 POINT(38.65674976294005 -122.81393504619525) bank187304 +187305 POINT(37.155713615440696 -122.82696873048883) bank187305 +187306 POINT(37.96513549602428 -121.86701342585978) bank187306 +187307 POINT(37.53312083655286 -122.80327571994663) bank187307 +187308 POINT(38.383322793733456 -121.46315266086565) bank187308 +187309 POINT(38.71215359390751 -122.75163095883958) bank187309 +187310 POINT(37.04252259161899 -121.74868643858908) bank187310 +187311 POINT(37.15264938252559 -122.14740208717191) bank187311 +187312 POINT(37.12629912889012 -122.57000067638796) bank187312 +187313 POINT(37.95474676784935 -121.57154684724912) bank187313 +187314 POINT(38.20580820525976 -123.14275171109048) bank187314 +187315 POINT(38.26727923790499 -122.0913229099172) bank187315 +187316 POINT(37.621120113816694 -121.55154483646362) bank187316 +187317 POINT(36.93791839765191 -121.78058067270058) bank187317 +187318 POINT(38.30378586304138 -123.09626021425834) bank187318 +187319 POINT(38.427046827648816 -121.58859326502079) bank187319 +187320 POINT(37.075828226834275 -122.48613350514904) bank187320 +187321 POINT(37.04696667327492 -122.45218941222183) bank187321 +187322 POINT(37.01373969374414 -122.26942749703582) bank187322 +187323 POINT(36.95520738880069 -122.1769599574595) bank187323 +187324 POINT(37.02544635417502 -121.96697254709015) bank187324 +187325 POINT(38.163443748397164 -122.09647254970207) bank187325 +187326 POINT(38.21161706049412 -121.75983258031687) bank187326 +187327 POINT(38.153486432254155 -123.21495287084576) bank187327 +187328 POINT(37.44260925397321 -122.39170557458375) bank187328 +187329 POINT(36.78798680116547 -122.94491940066085) bank187329 +187330 POINT(38.32962382050663 -123.03649295271573) bank187330 +187331 POINT(38.026739576003166 -121.90281215517635) bank187331 +187332 POINT(38.27431869734266 -122.03225820361916) bank187332 +187333 POINT(38.73745942355595 -122.23919168708028) bank187333 +187334 POINT(37.0673115485112 -123.24962560536481) bank187334 +187335 POINT(38.64750990431107 -122.53781082357736) bank187335 +187336 POINT(37.19723876930045 -123.32105342306639) bank187336 +187337 POINT(38.39646621837761 -122.34645022867568) bank187337 +187338 POINT(38.72896354502736 -122.39001040258123) bank187338 +187339 POINT(38.01656432134548 -121.58781660917745) bank187339 +187340 POINT(37.64631339406466 -121.83875179035117) bank187340 +187341 POINT(37.66316469540892 -122.11770392255367) bank187341 +187342 POINT(37.94469757288941 -122.61310742157455) bank187342 +187343 POINT(37.74656217819282 -121.89248425386852) bank187343 +187344 POINT(38.04628714356314 -123.24862374556206) bank187344 +187345 POINT(38.283306015226856 -123.08077873363132) bank187345 +187346 POINT(37.2457538150935 -121.87683560317413) bank187346 +187347 POINT(37.57910380353552 -123.3449656105551) bank187347 +187348 POINT(37.230533025499014 -123.02338752045917) bank187348 +187349 POINT(37.67140785458444 -122.29003853133352) bank187349 +187350 POINT(38.378209268570394 -122.97617695416893) bank187350 +187351 POINT(38.40204498161599 -123.31127322465638) bank187351 +187352 POINT(37.5190054238159 -122.68947751964556) bank187352 +187353 POINT(38.34387116550199 -123.23228867584977) bank187353 +187354 POINT(38.1601383420897 -122.05205714785171) bank187354 +187355 POINT(37.51117070956894 -122.94073294267749) bank187355 +187356 POINT(37.582471514749635 -123.12639114564693) bank187356 +187357 POINT(37.47714159613255 -122.88848653007682) bank187357 +187358 POINT(38.63676404629905 -123.00541564678971) bank187358 +187359 POINT(37.7896061927115 -122.68771844688337) bank187359 +187360 POINT(38.27980398654252 -121.95185468038278) bank187360 +187361 POINT(38.44399331668862 -123.18316217812628) bank187361 +187362 POINT(36.97012732948054 -122.17191405553075) bank187362 +187363 POINT(38.43001746907882 -122.2044848327293) bank187363 +187364 POINT(38.473732526157406 -121.80611609837608) bank187364 +187365 POINT(37.95736721691598 -123.26630047971169) bank187365 +187366 POINT(36.89596586167676 -122.0821964001188) bank187366 +187367 POINT(37.49309213421079 -122.22679784864746) bank187367 +187368 POINT(36.87312658457917 -122.29659809464943) bank187368 +187369 POINT(37.913959269306865 -122.86826793845499) bank187369 +187370 POINT(37.268497540662445 -122.86112349801304) bank187370 +187371 POINT(38.39473602846352 -122.23374774014516) bank187371 +187372 POINT(37.36364358259043 -122.54626272331858) bank187372 +187373 POINT(36.77922911804614 -123.2169577267768) bank187373 +187374 POINT(37.97049040226432 -121.72853245256415) bank187374 +187375 POINT(36.886426663528944 -122.82083972774086) bank187375 +187376 POINT(37.51337046917838 -123.03320609542665) bank187376 +187377 POINT(37.5984874911328 -121.88545720150036) bank187377 +187378 POINT(38.160302600847466 -121.49724860635651) bank187378 +187379 POINT(38.71321249518231 -122.97611925398486) bank187379 +187380 POINT(38.0925614089186 -122.0052415305852) bank187380 +187381 POINT(38.12282317253322 -122.08569338291039) bank187381 +187382 POINT(38.7707471899493 -122.68011012541086) bank187382 +187383 POINT(36.80778123415943 -123.33229848988815) bank187383 +187384 POINT(38.62935361421903 -122.94333418568536) bank187384 +187385 POINT(37.03419040869232 -123.08678865220024) bank187385 +187386 POINT(37.82510593038004 -122.11577863331057) bank187386 +187387 POINT(37.240499757328635 -121.93126516338849) bank187387 +187388 POINT(37.048662962926535 -123.36934904237302) bank187388 +187389 POINT(38.73453775773704 -123.15422280890606) bank187389 +187390 POINT(37.93933484416016 -122.8726674462302) bank187390 +187391 POINT(38.36370294410941 -121.53679179755925) bank187391 +187392 POINT(37.311245257281925 -121.93793268249281) bank187392 +187393 POINT(37.74142221834356 -123.06835037040672) bank187393 +187394 POINT(37.2366408228448 -122.36997118453824) bank187394 +187395 POINT(37.6678697273548 -122.56518222455166) bank187395 +187396 POINT(37.24780832270877 -123.3446860702368) bank187396 +187397 POINT(38.05183574977846 -122.01271546418398) bank187397 +187398 POINT(38.451923569597525 -123.16341945592586) bank187398 +187399 POINT(37.30704491906531 -121.7051606969338) bank187399 +187400 POINT(37.431562116248706 -123.14643774554489) bank187400 +187401 POINT(37.04734291543601 -123.07497892391702) bank187401 +187402 POINT(37.16496702457033 -123.38433690577175) bank187402 +187403 POINT(36.89073315290444 -122.19953130873029) bank187403 +187404 POINT(37.77874740773553 -122.22324276954531) bank187404 +187405 POINT(37.67087606233754 -122.90876524635712) bank187405 +187406 POINT(37.97847713784362 -121.66320896554004) bank187406 +187407 POINT(38.70977490334972 -123.00127860901739) bank187407 +187408 POINT(36.823964981639506 -121.82776768512899) bank187408 +187409 POINT(37.067727538208224 -121.68342491182158) bank187409 +187410 POINT(36.93894625684525 -122.56343081689434) bank187410 +187411 POINT(37.11952770636748 -123.34894747458986) bank187411 +187412 POINT(37.28642428549745 -123.22670525658823) bank187412 +187413 POINT(37.35371021013027 -122.13872202797727) bank187413 +187414 POINT(38.51534345307634 -121.54913466429969) bank187414 +187415 POINT(37.214125882428974 -122.83646059824268) bank187415 +187416 POINT(37.76933175069513 -122.5723744246029) bank187416 +187417 POINT(37.49199792828437 -123.41210116050964) bank187417 +187418 POINT(37.76682786028419 -121.9285047179013) bank187418 +187419 POINT(36.94131093987766 -121.5237413069165) bank187419 +187420 POINT(36.92675875693762 -122.20832032644208) bank187420 +187421 POINT(37.69913080443078 -122.27977438783876) bank187421 +187422 POINT(37.96279129994583 -122.58080668564047) bank187422 +187423 POINT(37.19268220537529 -123.15346869987748) bank187423 +187424 POINT(38.35927705403238 -122.23841245465285) bank187424 +187425 POINT(37.62050662734752 -123.02627869679331) bank187425 +187426 POINT(37.39393876826897 -122.00449364136101) bank187426 +187427 POINT(37.94987723400081 -122.01918441316178) bank187427 +187428 POINT(38.213579466987206 -122.38369132294976) bank187428 +187429 POINT(38.04965172574756 -121.43766530215697) bank187429 +187430 POINT(37.990129316974006 -123.16145735596136) bank187430 +187431 POINT(38.10525820671539 -121.47736229481804) bank187431 +187432 POINT(37.3886124741672 -123.37038909179654) bank187432 +187433 POINT(38.35422069960386 -122.99628052720072) bank187433 +187434 POINT(37.16316849364368 -123.1637807757426) bank187434 +187435 POINT(37.91392049259113 -123.26814093483782) bank187435 +187436 POINT(38.686967694893774 -122.70446841963366) bank187436 +187437 POINT(37.01496003334577 -123.2189602064705) bank187437 +187438 POINT(38.1319032199425 -122.37957548321691) bank187438 +187439 POINT(38.406502222937625 -123.08481249818557) bank187439 +187440 POINT(38.61005707081865 -122.86150979020933) bank187440 +187441 POINT(37.74153117799405 -122.7092801028812) bank187441 +187442 POINT(38.71117904765751 -121.98667481400406) bank187442 +187443 POINT(38.09857156612989 -123.07348507946355) bank187443 +187444 POINT(36.89892113224228 -123.41605143722046) bank187444 +187445 POINT(36.86214091518421 -122.06338514011726) bank187445 +187446 POINT(37.41536655661472 -122.80868857853936) bank187446 +187447 POINT(38.460658541495384 -123.09358896090352) bank187447 +187448 POINT(38.49159055445655 -122.69373797122516) bank187448 +187449 POINT(37.19726934471655 -123.3745032613016) bank187449 +187450 POINT(37.053215253811246 -121.93959879528809) bank187450 +187451 POINT(38.58314827062538 -122.9961906595665) bank187451 +187452 POINT(38.57154017015543 -122.15305221934996) bank187452 +187453 POINT(38.579186037778996 -123.12344818532918) bank187453 +187454 POINT(38.41559931666311 -122.36649630910436) bank187454 +187455 POINT(37.4765870415306 -122.16558042250895) bank187455 +187456 POINT(38.47750866297859 -122.64418397938478) bank187456 +187457 POINT(37.555941168614126 -121.60686564395478) bank187457 +187458 POINT(38.646687481499555 -123.3619634089504) bank187458 +187459 POINT(38.663823892442345 -121.90567033175807) bank187459 +187460 POINT(37.67811707578477 -122.6643498198213) bank187460 +187461 POINT(37.110101957483735 -122.37645483199344) bank187461 +187462 POINT(37.60652533241952 -122.38002740047413) bank187462 +187463 POINT(37.56591222702814 -122.18095658778404) bank187463 +187464 POINT(38.32665432835216 -122.24412723179117) bank187464 +187465 POINT(37.180936729686266 -122.830965977315) bank187465 +187466 POINT(37.24812058335506 -123.39684496218544) bank187466 +187467 POINT(37.157960818715814 -122.89094632529866) bank187467 +187468 POINT(37.684905582131535 -122.55163730689455) bank187468 +187469 POINT(37.197229700496024 -121.65316964613275) bank187469 +187470 POINT(38.33408605178608 -122.66531668789854) bank187470 +187471 POINT(37.457032095960706 -123.08595489770248) bank187471 +187472 POINT(38.45476116853524 -122.04942684294076) bank187472 +187473 POINT(37.043739856538096 -121.56960887076292) bank187473 +187474 POINT(38.04352425516068 -122.97785922349468) bank187474 +187475 POINT(38.49428603891894 -121.70894791694425) bank187475 +187476 POINT(38.38025562922086 -122.00830608559347) bank187476 +187477 POINT(38.715906177698386 -123.39014974048843) bank187477 +187478 POINT(37.878429869945 -121.74614432141622) bank187478 +187479 POINT(37.026075261168224 -121.44912421586635) bank187479 +187480 POINT(37.70928985499181 -123.03188437852617) bank187480 +187481 POINT(37.43772712088081 -121.42928917040138) bank187481 +187482 POINT(36.80314363103693 -122.82454430648548) bank187482 +187483 POINT(38.16784565638509 -122.82542305115162) bank187483 +187484 POINT(37.85686305797774 -122.14782905529087) bank187484 +187485 POINT(37.41250804513294 -121.79994299632946) bank187485 +187486 POINT(37.47081790729723 -122.34049687619853) bank187486 +187487 POINT(38.74280870241018 -122.66178583825142) bank187487 +187488 POINT(37.651671620199046 -123.22480014951564) bank187488 +187489 POINT(37.53355669033556 -123.34317368600729) bank187489 +187490 POINT(37.397059980095115 -123.41867823295378) bank187490 +187491 POINT(37.355434462960694 -122.41359993519008) bank187491 +187492 POINT(36.94669019865847 -121.81479423471022) bank187492 +187493 POINT(38.18447890380827 -122.33761618312717) bank187493 +187494 POINT(38.498779168073604 -122.75753424103594) bank187494 +187495 POINT(37.67939198027596 -123.19182552738545) bank187495 +187496 POINT(37.19099450880531 -122.23104960747587) bank187496 +187497 POINT(38.539398726098675 -121.60300796052579) bank187497 +187498 POINT(36.86739108500034 -121.5601324003115) bank187498 +187499 POINT(37.41051604576054 -122.39399056634706) bank187499 +187500 POINT(38.063798368984294 -121.52520283639579) bank187500 +187501 POINT(38.385008723491744 -122.30886136854276) bank187501 +187502 POINT(36.95226327776543 -121.56425805153451) bank187502 +187503 POINT(37.32440837193075 -123.25419397201249) bank187503 +187504 POINT(36.9442188578608 -123.22452270220055) bank187504 +187505 POINT(38.58933230335866 -122.70551840074975) bank187505 +187506 POINT(37.35038290383538 -122.90276653515264) bank187506 +187507 POINT(38.436696002142796 -123.21127337738096) bank187507 +187508 POINT(37.379922949528485 -121.61774447453473) bank187508 +187509 POINT(38.25597675306756 -122.66933162434202) bank187509 +187510 POINT(38.530617454832885 -122.76744771435068) bank187510 +187511 POINT(38.74781581630194 -121.66671766220927) bank187511 +187512 POINT(37.99153952995732 -122.1788246377946) bank187512 +187513 POINT(37.542808745138366 -121.8722091256984) bank187513 +187514 POINT(36.9586023246421 -121.51792264333935) bank187514 +187515 POINT(37.5746655090051 -122.50403249197788) bank187515 +187516 POINT(37.36024269313838 -122.42341061180784) bank187516 +187517 POINT(37.13205110354174 -122.7993722746687) bank187517 +187518 POINT(37.53543390397859 -122.06916269855317) bank187518 +187519 POINT(38.20971557346022 -123.17156541119982) bank187519 +187520 POINT(37.948785326886444 -122.707258853878) bank187520 +187521 POINT(38.255255846687035 -121.70184572728) bank187521 +187522 POINT(38.26183596634901 -122.15551512577001) bank187522 +187523 POINT(36.79527760528375 -121.82580677391543) bank187523 +187524 POINT(38.70358662168426 -121.97079562609565) bank187524 +187525 POINT(38.435637376972934 -122.75846751227061) bank187525 +187526 POINT(37.731761842118885 -123.14158491917483) bank187526 +187527 POINT(38.74675376927131 -122.69661606915642) bank187527 +187528 POINT(38.07724405612838 -123.0401705227789) bank187528 +187529 POINT(37.91717622252628 -121.49030053391915) bank187529 +187530 POINT(37.13493641881127 -123.30132342321572) bank187530 +187531 POINT(37.818851773374085 -122.3700474937246) bank187531 +187532 POINT(38.02171181291162 -121.55678177542171) bank187532 +187533 POINT(38.30455209223312 -122.4288647327376) bank187533 +187534 POINT(38.03353927836903 -123.27416578865052) bank187534 +187535 POINT(37.14267663761648 -122.01218964002811) bank187535 +187536 POINT(38.26496692067919 -123.25328482230917) bank187536 +187537 POINT(37.12348957459394 -122.8342871770443) bank187537 +187538 POINT(36.84528140682285 -122.22765533012463) bank187538 +187539 POINT(38.03571288488714 -121.80519660202573) bank187539 +187540 POINT(38.59473525712113 -122.16173120320023) bank187540 +187541 POINT(38.00898774816099 -123.2346871696817) bank187541 +187542 POINT(37.332876534409216 -121.7732617746412) bank187542 +187543 POINT(38.305234439938076 -122.70560733363524) bank187543 +187544 POINT(37.18436929115162 -122.82453424939905) bank187544 +187545 POINT(36.997493377412404 -122.42439832163807) bank187545 +187546 POINT(37.39938487228237 -123.1558480953084) bank187546 +187547 POINT(37.82552926103055 -123.26838784132947) bank187547 +187548 POINT(37.27437154304527 -122.27461513543282) bank187548 +187549 POINT(38.22107783547388 -122.84291371005389) bank187549 +187550 POINT(37.5519777165687 -122.70224807237189) bank187550 +187551 POINT(36.99266947945776 -123.24408998877836) bank187551 +187552 POINT(38.05450754772775 -123.2365214087932) bank187552 +187553 POINT(38.250408667577254 -121.76173637286327) bank187553 +187554 POINT(38.76490573115774 -122.22390494617862) bank187554 +187555 POINT(37.06613662408212 -123.0716364165287) bank187555 +187556 POINT(37.648782374163694 -121.66121372000907) bank187556 +187557 POINT(36.82953650896971 -122.4978216498856) bank187557 +187558 POINT(38.26572665467474 -122.84089056005571) bank187558 +187559 POINT(38.76232230667752 -121.5755688679968) bank187559 +187560 POINT(36.82844064601676 -122.55833723723757) bank187560 +187561 POINT(37.268275329380955 -122.70003130823712) bank187561 +187562 POINT(37.84373421025852 -122.4177041735739) bank187562 +187563 POINT(36.89886934637083 -122.54894748004612) bank187563 +187564 POINT(38.774830130495666 -121.89342256569854) bank187564 +187565 POINT(37.0831356677212 -121.71619766556191) bank187565 +187566 POINT(38.20333646399975 -122.60398485861097) bank187566 +187567 POINT(37.21239490691576 -122.93484110666772) bank187567 +187568 POINT(38.02836431195774 -121.93776341685444) bank187568 +187569 POINT(37.660911724275046 -123.15507259230944) bank187569 +187570 POINT(36.82010573256478 -123.26611843885019) bank187570 +187571 POINT(38.319774000229295 -123.26854194107497) bank187571 +187572 POINT(37.06640978975766 -121.4671800058954) bank187572 +187573 POINT(36.81525086560302 -121.87347129563629) bank187573 +187574 POINT(38.352251111055494 -122.39344974778669) bank187574 +187575 POINT(38.090702732695505 -123.11332387569753) bank187575 +187576 POINT(37.6548864706697 -122.98597131371965) bank187576 +187577 POINT(37.491764261348315 -123.40743476815807) bank187577 +187578 POINT(37.218900396046585 -121.82577800694844) bank187578 +187579 POINT(38.691195956233024 -122.93459109310561) bank187579 +187580 POINT(38.71504988121413 -122.41527685016509) bank187580 +187581 POINT(37.24895573551943 -122.25859091205476) bank187581 +187582 POINT(37.69020013486346 -122.79794135392392) bank187582 +187583 POINT(37.0871677064278 -121.96379362314475) bank187583 +187584 POINT(37.06275633008117 -121.97645346833875) bank187584 +187585 POINT(37.17567909851315 -123.07996011323335) bank187585 +187586 POINT(37.37055173911801 -122.66938050794207) bank187586 +187587 POINT(38.77045780975239 -122.83609898789929) bank187587 +187588 POINT(36.79051811671875 -121.783497190994) bank187588 +187589 POINT(38.486749257013706 -122.1416167742338) bank187589 +187590 POINT(37.104105385470525 -123.0304400977112) bank187590 +187591 POINT(38.38152672565931 -122.07473358216869) bank187591 +187592 POINT(37.01192709018359 -123.08671153283596) bank187592 +187593 POINT(37.93309615803699 -122.7935447782765) bank187593 +187594 POINT(38.07983370065391 -122.13774219972379) bank187594 +187595 POINT(38.69184915966892 -121.57851887084261) bank187595 +187596 POINT(37.60668660373553 -123.07670493464838) bank187596 +187597 POINT(36.88387237193409 -122.123671892031) bank187597 +187598 POINT(37.043511696587274 -122.12770330229695) bank187598 +187599 POINT(38.68206793049416 -122.28037834077378) bank187599 +187600 POINT(37.808501935625536 -122.43366825146691) bank187600 +187601 POINT(36.83376544997558 -122.60352147624926) bank187601 +187602 POINT(37.17232585394663 -122.9531667901657) bank187602 +187603 POINT(38.70382833309581 -123.31141865363122) bank187603 +187604 POINT(36.93651778730707 -122.35409907755319) bank187604 +187605 POINT(37.92672064718243 -121.77823044191035) bank187605 +187606 POINT(38.70680484358482 -121.83781324752178) bank187606 +187607 POINT(38.72353321354738 -122.40949023896822) bank187607 +187608 POINT(37.34284828122548 -121.90135539602925) bank187608 +187609 POINT(37.649710989248256 -122.5449747641419) bank187609 +187610 POINT(37.4417484910203 -121.77737517174398) bank187610 +187611 POINT(38.45470332285554 -122.70798119671116) bank187611 +187612 POINT(37.734161502776125 -123.07530854135618) bank187612 +187613 POINT(37.65503040523595 -122.12393090055008) bank187613 +187614 POINT(37.666387807103796 -122.59858823318095) bank187614 +187615 POINT(38.62187345687314 -123.04586277804633) bank187615 +187616 POINT(37.408086660921334 -122.57415799058272) bank187616 +187617 POINT(37.05325365632208 -122.35848048609188) bank187617 +187618 POINT(38.452089724071115 -123.180920622243) bank187618 +187619 POINT(38.068662962045885 -122.1179872137797) bank187619 +187620 POINT(37.00165670395621 -121.49358003227009) bank187620 +187621 POINT(37.21474779631907 -122.5589525299712) bank187621 +187622 POINT(37.04210183644156 -122.57650079231817) bank187622 +187623 POINT(38.67702980560532 -122.35503246387746) bank187623 +187624 POINT(37.1214499804075 -123.21509433428984) bank187624 +187625 POINT(37.555374525085206 -123.29688948345488) bank187625 +187626 POINT(38.69818569410443 -121.54488880461582) bank187626 +187627 POINT(38.45331583051847 -123.0832906070682) bank187627 +187628 POINT(37.93170689311991 -122.01941414449615) bank187628 +187629 POINT(37.030909753725254 -122.57642313171489) bank187629 +187630 POINT(38.24499581981911 -123.04836442731329) bank187630 +187631 POINT(38.721566701014765 -121.69601253964223) bank187631 +187632 POINT(37.89901757303384 -121.67800555006905) bank187632 +187633 POINT(38.06447658890675 -123.02579203510486) bank187633 +187634 POINT(37.79111461493908 -123.39855082351603) bank187634 +187635 POINT(37.558240444675775 -121.55296665573076) bank187635 +187636 POINT(37.31655127484317 -122.94105805914235) bank187636 +187637 POINT(37.24789382347978 -122.96534894234277) bank187637 +187638 POINT(37.447593472709336 -122.99420609424301) bank187638 +187639 POINT(37.66604891239351 -122.6697836678004) bank187639 +187640 POINT(37.796783505738986 -121.72115545517985) bank187640 +187641 POINT(38.30236375173219 -121.70037096594419) bank187641 +187642 POINT(36.95040934821061 -123.23852442657454) bank187642 +187643 POINT(38.19191879710983 -122.73991622399149) bank187643 +187644 POINT(38.443524637488935 -122.56575238085145) bank187644 +187645 POINT(37.1566000546593 -122.86111529098348) bank187645 +187646 POINT(37.020306631717396 -121.97085854990229) bank187646 +187647 POINT(37.95863271162626 -122.54641745909024) bank187647 +187648 POINT(37.891593903729195 -121.71369003795623) bank187648 +187649 POINT(37.088614654064294 -122.5911975700273) bank187649 +187650 POINT(37.213964151478 -122.74813900783583) bank187650 +187651 POINT(37.97778894437818 -121.99902104887923) bank187651 +187652 POINT(37.726485118217354 -122.38441221059351) bank187652 +187653 POINT(37.72615463775175 -122.79753965757257) bank187653 +187654 POINT(37.459929962805965 -122.27335757080627) bank187654 +187655 POINT(38.173634746068004 -121.925073997301) bank187655 +187656 POINT(38.249924056301786 -121.60763484716593) bank187656 +187657 POINT(37.95056985387055 -122.52230334296402) bank187657 +187658 POINT(38.62400836940766 -121.72366758294842) bank187658 +187659 POINT(37.53490071783016 -122.54295621670958) bank187659 +187660 POINT(37.345903842316 -121.4865551774596) bank187660 +187661 POINT(36.81679758629599 -123.3460748550607) bank187661 +187662 POINT(36.819654593505966 -122.80041430022412) bank187662 +187663 POINT(36.89250877245199 -122.98294754867419) bank187663 +187664 POINT(38.62272554254232 -121.49333815582216) bank187664 +187665 POINT(38.66918532558105 -121.84469431592564) bank187665 +187666 POINT(38.13671318030858 -121.55248829698147) bank187666 +187667 POINT(36.876654918546826 -122.27509917930973) bank187667 +187668 POINT(37.85670209224522 -122.12697111821382) bank187668 +187669 POINT(38.45029987952022 -122.33304485205295) bank187669 +187670 POINT(38.073619635157165 -122.30979998162758) bank187670 +187671 POINT(37.61671802045511 -122.2442259050718) bank187671 +187672 POINT(38.00039567251124 -122.4786785394448) bank187672 +187673 POINT(37.63659006433996 -123.35716251944712) bank187673 +187674 POINT(36.97364269842751 -123.32638233898044) bank187674 +187675 POINT(37.383078560181126 -122.50443138754149) bank187675 +187676 POINT(38.26814298524012 -123.18334959186537) bank187676 +187677 POINT(37.75849031640932 -123.03941538196352) bank187677 +187678 POINT(36.92626889263736 -123.05996736688968) bank187678 +187679 POINT(38.66519730889216 -122.69467600259327) bank187679 +187680 POINT(37.57114676219261 -122.62186140970913) bank187680 +187681 POINT(37.36299959504312 -123.32195931619196) bank187681 +187682 POINT(36.99678121270385 -123.33547269937314) bank187682 +187683 POINT(38.55876781828607 -121.5553681181298) bank187683 +187684 POINT(38.304567245353596 -122.2487289532172) bank187684 +187685 POINT(37.61368934306955 -123.32031003505003) bank187685 +187686 POINT(38.12587805924585 -122.57204604486662) bank187686 +187687 POINT(38.106483077952724 -122.9484827719576) bank187687 +187688 POINT(38.21622215777777 -121.84145081690976) bank187688 +187689 POINT(38.0137144607549 -122.31570302084585) bank187689 +187690 POINT(37.77920249335138 -122.82693164486514) bank187690 +187691 POINT(37.46831652565302 -121.84904346802837) bank187691 +187692 POINT(38.72349483719003 -123.41266379025674) bank187692 +187693 POINT(38.30261679280382 -121.89695040443259) bank187693 +187694 POINT(38.346143582568956 -121.77143844908676) bank187694 +187695 POINT(37.56377526795251 -121.88055550251848) bank187695 +187696 POINT(36.84986772055373 -123.33661639493717) bank187696 +187697 POINT(38.18658251866741 -122.84290824427843) bank187697 +187698 POINT(38.284197066689146 -122.55284520185054) bank187698 +187699 POINT(37.60518631552025 -122.96894062541577) bank187699 +187700 POINT(37.9510565602476 -122.43336098218238) bank187700 +187701 POINT(36.778939748284074 -121.42895372835522) bank187701 +187702 POINT(37.31631912330383 -121.80845975728037) bank187702 +187703 POINT(37.314916477103026 -122.79605194220507) bank187703 +187704 POINT(36.79420258010186 -122.81719477781353) bank187704 +187705 POINT(37.10051923531768 -121.59570730509783) bank187705 +187706 POINT(37.2299286950251 -121.80830978004776) bank187706 +187707 POINT(38.661083007081366 -121.42194505132677) bank187707 +187708 POINT(37.869164034649295 -123.21042174545556) bank187708 +187709 POINT(37.217903976367694 -122.24235236006506) bank187709 +187710 POINT(38.35892739546601 -121.88000234765457) bank187710 +187711 POINT(37.16341483032572 -122.14080460995446) bank187711 +187712 POINT(37.60774685292358 -122.96752130357524) bank187712 +187713 POINT(37.199275395894745 -122.5056194812829) bank187713 +187714 POINT(37.79712081052 -122.71567814138305) bank187714 +187715 POINT(38.398190790102554 -121.44965836923268) bank187715 +187716 POINT(38.163473946789416 -122.83503757050008) bank187716 +187717 POINT(38.73058672083539 -122.19130614851962) bank187717 +187718 POINT(37.83703888969587 -121.97091465628411) bank187718 +187719 POINT(37.068201829508844 -121.66028817626398) bank187719 +187720 POINT(37.18333084374273 -123.40537181569267) bank187720 +187721 POINT(38.31436360708217 -122.45371524636981) bank187721 +187722 POINT(37.30473642623634 -122.08929750241077) bank187722 +187723 POINT(37.19563069382909 -122.06662287835542) bank187723 +187724 POINT(37.42863227832216 -122.81527475918544) bank187724 +187725 POINT(37.8398458701256 -122.45934840584341) bank187725 +187726 POINT(38.231601121757336 -122.51243330309075) bank187726 +187727 POINT(37.54406933074298 -123.32098115850235) bank187727 +187728 POINT(37.33843231668435 -122.33897735717645) bank187728 +187729 POINT(38.24939983586859 -123.02903131020055) bank187729 +187730 POINT(37.3533336089977 -123.30045080954461) bank187730 +187731 POINT(36.85504391442959 -122.31008068356279) bank187731 +187732 POINT(38.70433324418659 -122.25496810389723) bank187732 +187733 POINT(37.90949011697376 -121.53888132727258) bank187733 +187734 POINT(37.29622222450965 -122.57698216155683) bank187734 +187735 POINT(38.277952216682365 -122.80780888270257) bank187735 +187736 POINT(37.73328802311413 -121.97708471581484) bank187736 +187737 POINT(37.56653911803622 -121.56689526002809) bank187737 +187738 POINT(38.476948649431236 -122.7404177508792) bank187738 +187739 POINT(37.144823966456535 -121.56833766666661) bank187739 +187740 POINT(37.12458906257967 -123.15566219829759) bank187740 +187741 POINT(37.17751478144179 -122.91459655315434) bank187741 +187742 POINT(37.585747561352406 -123.21429491606875) bank187742 +187743 POINT(38.75425107458565 -123.2729950917473) bank187743 +187744 POINT(37.47893154537617 -121.60791860508314) bank187744 +187745 POINT(36.83961065827331 -122.13125426189818) bank187745 +187746 POINT(37.140980348830894 -121.93850241137172) bank187746 +187747 POINT(37.480822818073726 -121.76574624445578) bank187747 +187748 POINT(37.39489024718894 -121.54748188798192) bank187748 +187749 POINT(38.68871357715642 -122.82203849935173) bank187749 +187750 POINT(37.539021185168814 -121.89933747678538) bank187750 +187751 POINT(38.05017800810589 -122.51795605574996) bank187751 +187752 POINT(37.963232182534014 -123.14692785770295) bank187752 +187753 POINT(37.094758838916995 -122.78978951426531) bank187753 +187754 POINT(36.92550796089271 -122.44837390460314) bank187754 +187755 POINT(36.85175512403469 -122.75366087389206) bank187755 +187756 POINT(37.43051868560765 -122.99535619819216) bank187756 +187757 POINT(37.4844022766817 -123.39420013225724) bank187757 +187758 POINT(37.32140508328233 -122.10103986356366) bank187758 +187759 POINT(36.863709550104865 -123.36013216201215) bank187759 +187760 POINT(38.21570355047109 -122.81184293271818) bank187760 +187761 POINT(38.546788935333694 -122.53162826603122) bank187761 +187762 POINT(37.95424269137274 -123.28749730128173) bank187762 +187763 POINT(37.53373329240539 -121.57649959301658) bank187763 +187764 POINT(37.95649295762386 -122.02796144581914) bank187764 +187765 POINT(36.96748559664673 -121.5054676517517) bank187765 +187766 POINT(37.863027972839525 -122.96816379206497) bank187766 +187767 POINT(37.78747953710467 -122.53241145146072) bank187767 +187768 POINT(36.89689822714141 -122.47077311362035) bank187768 +187769 POINT(37.06214402649612 -121.98951486291124) bank187769 +187770 POINT(38.521914773844706 -122.98844727170885) bank187770 +187771 POINT(37.296023983873106 -121.62141210414372) bank187771 +187772 POINT(38.37315205005334 -123.02232042106586) bank187772 +187773 POINT(37.115248204320956 -123.24493698093812) bank187773 +187774 POINT(37.83907801187997 -122.05616618447712) bank187774 +187775 POINT(36.96867065322525 -121.47270647829185) bank187775 +187776 POINT(37.63218245361946 -123.19090509070296) bank187776 +187777 POINT(38.6750134202875 -121.8594236397729) bank187777 +187778 POINT(38.697788347519406 -121.78239451524102) bank187778 +187779 POINT(37.41830381818001 -123.2356884098916) bank187779 +187780 POINT(37.64315257900562 -121.84426213939417) bank187780 +187781 POINT(37.8984916167067 -121.69124595515318) bank187781 +187782 POINT(37.70609084850724 -122.40000314900375) bank187782 +187783 POINT(37.83562883743919 -121.56051828089197) bank187783 +187784 POINT(37.69115866510352 -122.2303493331788) bank187784 +187785 POINT(37.799137633715524 -123.0141966243397) bank187785 +187786 POINT(38.716015799798264 -122.08051604309695) bank187786 +187787 POINT(38.25350202087907 -121.91546058942058) bank187787 +187788 POINT(38.620382422239025 -122.29384860313291) bank187788 +187789 POINT(38.07958294626967 -123.28584894457833) bank187789 +187790 POINT(37.19508828705345 -122.84835522517261) bank187790 +187791 POINT(38.05219206645177 -123.37481250425303) bank187791 +187792 POINT(36.83061691744279 -121.99581811250913) bank187792 +187793 POINT(37.31409086732845 -122.26767286623996) bank187793 +187794 POINT(36.93655003755693 -123.26689997547781) bank187794 +187795 POINT(37.20871493495038 -122.0479551136957) bank187795 +187796 POINT(37.96469434405085 -121.92028120118711) bank187796 +187797 POINT(37.91283996000551 -122.30335375755213) bank187797 +187798 POINT(36.83854147161449 -122.31576903273057) bank187798 +187799 POINT(38.51178323969477 -123.11733244950776) bank187799 +187800 POINT(37.26797249221752 -122.22164857209972) bank187800 +187801 POINT(38.03915251079909 -122.48281676342494) bank187801 +187802 POINT(37.17239737887174 -123.22643067244663) bank187802 +187803 POINT(37.98826825380144 -122.48099162453504) bank187803 +187804 POINT(37.977835859494924 -121.42529128784518) bank187804 +187805 POINT(37.080494433315 -122.10797454311206) bank187805 +187806 POINT(38.42317924868738 -122.39514058286586) bank187806 +187807 POINT(38.316867661022165 -121.80317653904615) bank187807 +187808 POINT(37.3154324967364 -121.69302261747089) bank187808 +187809 POINT(38.34773681752968 -122.90151275025265) bank187809 +187810 POINT(37.39146619133306 -123.11771700664971) bank187810 +187811 POINT(38.51042413500387 -121.96928841359158) bank187811 +187812 POINT(37.53789383588205 -122.54618471306374) bank187812 +187813 POINT(38.42581025130875 -122.6145044898042) bank187813 +187814 POINT(38.12888530722468 -121.72804221388876) bank187814 +187815 POINT(37.79112615123234 -123.39099003856451) bank187815 +187816 POINT(38.70484411987905 -122.24346666747228) bank187816 +187817 POINT(37.6644048595542 -122.70744415619592) bank187817 +187818 POINT(37.88423310207894 -121.91883717605045) bank187818 +187819 POINT(37.653784636316416 -121.99058647890455) bank187819 +187820 POINT(37.364117989209916 -121.566544576775) bank187820 +187821 POINT(38.02150632154074 -123.31555826902495) bank187821 +187822 POINT(37.698851842287674 -121.60893382535225) bank187822 +187823 POINT(37.78148657373077 -121.9680139043802) bank187823 +187824 POINT(38.04776914865182 -122.09338559171938) bank187824 +187825 POINT(36.955020382980074 -121.87440226201795) bank187825 +187826 POINT(38.690463849093824 -122.89472408285923) bank187826 +187827 POINT(37.758823838260405 -122.58208424410421) bank187827 +187828 POINT(38.41344536075226 -122.79577147751459) bank187828 +187829 POINT(37.06059161128805 -122.24928417951843) bank187829 +187830 POINT(37.85937724867325 -123.30896375704982) bank187830 +187831 POINT(36.98927359367216 -122.91290157096014) bank187831 +187832 POINT(37.93874634869867 -122.43778983166716) bank187832 +187833 POINT(37.206947306117534 -123.28020000585188) bank187833 +187834 POINT(37.37754711705653 -122.24355706541856) bank187834 +187835 POINT(37.22776376139172 -122.22431309195343) bank187835 +187836 POINT(37.048531765822425 -122.3038079792058) bank187836 +187837 POINT(37.579133754940365 -123.15026610963653) bank187837 +187838 POINT(38.23921915991904 -121.9240287005238) bank187838 +187839 POINT(38.293425973959124 -121.90981863072665) bank187839 +187840 POINT(36.947898420473514 -123.23050600101566) bank187840 +187841 POINT(37.908441352751304 -123.24509462711387) bank187841 +187842 POINT(37.12258649488774 -122.93920427312405) bank187842 +187843 POINT(38.61096616758579 -122.8390100512469) bank187843 +187844 POINT(38.31571883432498 -123.40325311881655) bank187844 +187845 POINT(37.80594464238962 -121.93061576439166) bank187845 +187846 POINT(38.22475282272277 -121.71562070052826) bank187846 +187847 POINT(37.65974719600864 -123.38991743736948) bank187847 +187848 POINT(38.72766575482902 -122.57428949894846) bank187848 +187849 POINT(37.96249267468275 -121.61381836832663) bank187849 +187850 POINT(36.82773094233859 -122.28428096292855) bank187850 +187851 POINT(38.17081630986358 -121.53478214085918) bank187851 +187852 POINT(37.846412178955056 -123.24737895755925) bank187852 +187853 POINT(37.20835791160363 -122.95019417820117) bank187853 +187854 POINT(38.535053247203244 -122.16231589715836) bank187854 +187855 POINT(37.35030147071501 -123.39222071833902) bank187855 +187856 POINT(37.390470948155865 -123.21635912572985) bank187856 +187857 POINT(38.38674735363836 -123.17765684259511) bank187857 +187858 POINT(37.51901720414739 -122.8060897747603) bank187858 +187859 POINT(37.521874723991004 -122.72666341002164) bank187859 +187860 POINT(37.915060139251686 -121.90703257829523) bank187860 +187861 POINT(38.510151323433 -121.58041082985505) bank187861 +187862 POINT(38.252438432994715 -121.61886467341047) bank187862 +187863 POINT(37.46892776473318 -121.86581495484579) bank187863 +187864 POINT(38.186529558620144 -123.38013159032153) bank187864 +187865 POINT(37.09199594033857 -121.91678601373222) bank187865 +187866 POINT(38.558210412745154 -123.18369218972586) bank187866 +187867 POINT(37.16105386075018 -122.90184230370491) bank187867 +187868 POINT(38.10489293265556 -123.0296077493336) bank187868 +187869 POINT(38.289771323288264 -122.20536779887892) bank187869 +187870 POINT(36.86152170590495 -122.84583401821375) bank187870 +187871 POINT(37.431258575011654 -122.26635280931964) bank187871 +187872 POINT(38.113519066949486 -122.34495627516773) bank187872 +187873 POINT(36.81362864235951 -122.44573394320359) bank187873 +187874 POINT(37.825320101455716 -122.14986530173276) bank187874 +187875 POINT(37.474596265139105 -122.93903237268442) bank187875 +187876 POINT(37.63620392526698 -122.65894117047509) bank187876 +187877 POINT(37.133671110800606 -122.6574946535873) bank187877 +187878 POINT(37.610757588019304 -121.5751348179533) bank187878 +187879 POINT(38.440236096904876 -122.99899420329876) bank187879 +187880 POINT(38.174603163492215 -121.58764970561086) bank187880 +187881 POINT(37.62945628496461 -122.34823984141111) bank187881 +187882 POINT(36.78624945487542 -122.27671305862073) bank187882 +187883 POINT(37.47586142798256 -121.69668643248285) bank187883 +187884 POINT(37.647343034356304 -123.03366767904937) bank187884 +187885 POINT(37.168673376226266 -122.3200762397611) bank187885 +187886 POINT(38.40572850945784 -122.22763655831494) bank187886 +187887 POINT(38.235475037793286 -122.6089037719745) bank187887 +187888 POINT(38.19270320545823 -122.00945494934568) bank187888 +187889 POINT(37.11481958416061 -121.52039419890922) bank187889 +187890 POINT(38.37235340652284 -121.79580736689194) bank187890 +187891 POINT(38.705121994692426 -122.93203474470691) bank187891 +187892 POINT(36.93348459728271 -121.53909649112755) bank187892 +187893 POINT(37.88677436809444 -122.09740947206947) bank187893 +187894 POINT(37.649878699324766 -122.52996061698084) bank187894 +187895 POINT(37.02036378844478 -121.82279065100074) bank187895 +187896 POINT(38.446877044814045 -122.27195842693394) bank187896 +187897 POINT(38.68627323685456 -122.65351008464206) bank187897 +187898 POINT(38.712184998402236 -123.10495449741165) bank187898 +187899 POINT(38.185893234319195 -123.32486481004908) bank187899 +187900 POINT(37.46550466003346 -122.54717687634763) bank187900 +187901 POINT(37.55218213736192 -121.61091977322099) bank187901 +187902 POINT(37.038705514889855 -121.48154473937178) bank187902 +187903 POINT(38.57870358715831 -121.45850855640957) bank187903 +187904 POINT(38.07733536316007 -123.14310908280646) bank187904 +187905 POINT(37.36870572694287 -122.367762721183) bank187905 +187906 POINT(38.0011476520164 -121.95027754935303) bank187906 +187907 POINT(37.528448351718005 -122.65541375486347) bank187907 +187908 POINT(37.297340020824976 -121.72910351462767) bank187908 +187909 POINT(38.555458168272544 -123.18343246583034) bank187909 +187910 POINT(38.32961245771502 -122.37264651052695) bank187910 +187911 POINT(37.37839853844146 -122.93063218629813) bank187911 +187912 POINT(37.03414595218068 -123.06613987824393) bank187912 +187913 POINT(37.16315458579949 -122.2743356935442) bank187913 +187914 POINT(37.82676613125819 -122.17544078906643) bank187914 +187915 POINT(37.69333305352257 -122.85786733431704) bank187915 +187916 POINT(38.46708166685254 -121.84286950684101) bank187916 +187917 POINT(37.375455370472785 -122.20505513312816) bank187917 +187918 POINT(38.52700312383139 -122.41905957188368) bank187918 +187919 POINT(38.43856277239208 -122.2782488365077) bank187919 +187920 POINT(37.71759580914979 -123.35204226746106) bank187920 +187921 POINT(37.07213465751178 -122.46698780298534) bank187921 +187922 POINT(38.32400365927349 -121.56889505492937) bank187922 +187923 POINT(37.584525992901234 -122.32221413372295) bank187923 +187924 POINT(37.24729289043922 -122.20851801205158) bank187924 +187925 POINT(37.358321300863786 -122.29998019161035) bank187925 +187926 POINT(37.446922851733625 -122.96572399725746) bank187926 +187927 POINT(37.770034769543194 -122.33337337418985) bank187927 +187928 POINT(37.25170565838914 -123.25103841916253) bank187928 +187929 POINT(38.52370232350986 -123.13709550554574) bank187929 +187930 POINT(37.54459336136981 -122.71360333771617) bank187930 +187931 POINT(38.66178098895351 -121.54232792787488) bank187931 +187932 POINT(38.05054947740741 -122.08258136004854) bank187932 +187933 POINT(38.74664117861998 -122.48633122974415) bank187933 +187934 POINT(37.20461646023739 -122.17288769068685) bank187934 +187935 POINT(38.36868816034348 -122.07932554903766) bank187935 +187936 POINT(38.08726121664371 -122.75037053229076) bank187936 +187937 POINT(37.03543411814896 -122.90912842252548) bank187937 +187938 POINT(38.018465620186305 -123.4055544899419) bank187938 +187939 POINT(38.69147434756369 -121.91766269435274) bank187939 +187940 POINT(36.99369548642415 -123.01903883141901) bank187940 +187941 POINT(37.49716173235335 -122.52756207509229) bank187941 +187942 POINT(37.90849568203692 -123.11621407493428) bank187942 +187943 POINT(38.345914207627914 -122.30170155633165) bank187943 +187944 POINT(37.197273081964184 -121.53577534421521) bank187944 +187945 POINT(38.14872235865434 -122.91102036733636) bank187945 +187946 POINT(37.47058301928406 -123.00048172981052) bank187946 +187947 POINT(36.87517320670626 -121.49408270994576) bank187947 +187948 POINT(38.56276083969442 -122.53393483923746) bank187948 +187949 POINT(38.078307318497345 -122.42824441880883) bank187949 +187950 POINT(37.14679435795318 -121.46924176436413) bank187950 +187951 POINT(37.19276821728606 -122.29992850278856) bank187951 +187952 POINT(38.575559541648474 -121.82334803617697) bank187952 +187953 POINT(36.90454754094389 -122.2890917070392) bank187953 +187954 POINT(37.90340755912577 -122.20979656238218) bank187954 +187955 POINT(37.69222630969285 -122.64046486100895) bank187955 +187956 POINT(37.10926614509357 -121.8492852304257) bank187956 +187957 POINT(37.48323851888504 -122.59502969359049) bank187957 +187958 POINT(36.902135023580385 -121.47111986965764) bank187958 +187959 POINT(37.7371409056852 -122.6704375601483) bank187959 +187960 POINT(37.488582564318854 -122.21500892985313) bank187960 +187961 POINT(37.04729127486121 -123.12945935150735) bank187961 +187962 POINT(37.637647108577795 -123.14642953907644) bank187962 +187963 POINT(38.42670012298858 -122.43752623435523) bank187963 +187964 POINT(37.47212440780685 -122.67889383470758) bank187964 +187965 POINT(37.53646876474857 -122.73430600383878) bank187965 +187966 POINT(37.194301436184574 -122.56594027795808) bank187966 +187967 POINT(38.21082861701428 -121.45907735330783) bank187967 +187968 POINT(37.43979240966746 -121.68662074574551) bank187968 +187969 POINT(38.45088681271655 -122.50784070826828) bank187969 +187970 POINT(38.17859224644317 -122.54257426923148) bank187970 +187971 POINT(36.92625168866107 -123.33560332662647) bank187971 +187972 POINT(37.30752208521666 -122.52939509141638) bank187972 +187973 POINT(36.98424799829948 -121.82723209176265) bank187973 +187974 POINT(36.97610457526267 -123.06707068332896) bank187974 +187975 POINT(38.717856209909684 -121.52201961396472) bank187975 +187976 POINT(38.13714305492581 -123.1472811921417) bank187976 +187977 POINT(36.96257080777609 -122.74155612729557) bank187977 +187978 POINT(37.342359910453375 -121.88609014086026) bank187978 +187979 POINT(38.46141346873791 -122.94587449583278) bank187979 +187980 POINT(37.77710082086467 -122.26216054565215) bank187980 +187981 POINT(37.87557802114821 -122.90501092996337) bank187981 +187982 POINT(38.207542364930305 -122.83401865313105) bank187982 +187983 POINT(37.496053666559796 -122.53334785835351) bank187983 +187984 POINT(36.91774387381114 -122.28117781123764) bank187984 +187985 POINT(38.211112015982714 -123.32861216537226) bank187985 +187986 POINT(38.427993557013885 -122.83631142914422) bank187986 +187987 POINT(36.92560809285611 -122.40549853148028) bank187987 +187988 POINT(37.389718183498246 -121.51957851841304) bank187988 +187989 POINT(38.53723945881033 -121.53330603317652) bank187989 +187990 POINT(38.69887119576463 -122.15849683571847) bank187990 +187991 POINT(38.20381751565629 -121.6470682496117) bank187991 +187992 POINT(38.633405912775416 -123.13805893223567) bank187992 +187993 POINT(38.529853025940334 -122.50320342282889) bank187993 +187994 POINT(37.04249829210557 -121.87128651496282) bank187994 +187995 POINT(38.00929013865878 -123.23623593783825) bank187995 +187996 POINT(37.48189395182427 -122.26717836589063) bank187996 +187997 POINT(37.57816678622947 -122.5206389289152) bank187997 +187998 POINT(38.135819746011194 -122.67303378495197) bank187998 +187999 POINT(37.98982705805476 -123.35837692018345) bank187999 +188000 POINT(37.28717615232513 -122.66332971081977) bank188000 +188001 POINT(37.50470402627937 -122.59594959488933) bank188001 +188002 POINT(37.553005184276934 -122.91459927765806) bank188002 +188003 POINT(38.35638363904537 -121.78791832835391) bank188003 +188004 POINT(37.6097832048634 -122.74767319146365) bank188004 +188005 POINT(38.48898727303906 -122.28229042517687) bank188005 +188006 POINT(38.009109287369334 -123.3901955177565) bank188006 +188007 POINT(38.102987862729094 -121.4781458472849) bank188007 +188008 POINT(38.66635585174505 -121.67143914208432) bank188008 +188009 POINT(36.92276586369675 -122.70376080808964) bank188009 +188010 POINT(38.51972974149516 -121.92504242490101) bank188010 +188011 POINT(37.50277667973425 -121.92580127266811) bank188011 +188012 POINT(38.69644035062558 -121.61897316776256) bank188012 +188013 POINT(37.78726437160728 -122.14446390823818) bank188013 +188014 POINT(38.57423596778498 -122.31272303272381) bank188014 +188015 POINT(37.10031997500522 -123.30545303285358) bank188015 +188016 POINT(37.28556835204418 -122.5608239017273) bank188016 +188017 POINT(38.204415531175826 -122.99334250975427) bank188017 +188018 POINT(37.91047099536314 -123.17332882029503) bank188018 +188019 POINT(36.96186754755582 -122.6116184986327) bank188019 +188020 POINT(37.96637679527382 -122.25104419996394) bank188020 +188021 POINT(38.58201894661237 -123.102578244924) bank188021 +188022 POINT(37.80424497841912 -121.91558675902816) bank188022 +188023 POINT(37.63359487591555 -121.58715692464551) bank188023 +188024 POINT(38.76172462074712 -123.26058431041953) bank188024 +188025 POINT(38.64356744891766 -122.53341443709931) bank188025 +188026 POINT(38.12543694175382 -123.2348399077476) bank188026 +188027 POINT(38.22777097709246 -122.13534279266278) bank188027 +188028 POINT(38.458846356481764 -123.06679860373912) bank188028 +188029 POINT(36.9137881050808 -121.98243675035808) bank188029 +188030 POINT(37.085662116379964 -122.76055571525609) bank188030 +188031 POINT(38.64944320188784 -123.28978282172834) bank188031 +188032 POINT(37.032801060792714 -121.64343301006502) bank188032 +188033 POINT(37.43631962337778 -122.29791606699865) bank188033 +188034 POINT(38.338740096528824 -122.36384008215752) bank188034 +188035 POINT(38.288685937582876 -122.27052545416831) bank188035 +188036 POINT(37.37507606919778 -123.41836173891947) bank188036 +188037 POINT(37.96126077163671 -121.94174978652607) bank188037 +188038 POINT(38.19457747952929 -122.80815306749597) bank188038 +188039 POINT(38.603356267146 -121.52545471631963) bank188039 +188040 POINT(37.46411061380855 -121.61061839771031) bank188040 +188041 POINT(37.887848636875056 -121.7952448276996) bank188041 +188042 POINT(38.72461575884618 -121.66227359919529) bank188042 +188043 POINT(37.53862621645473 -121.44253068616082) bank188043 +188044 POINT(37.75100027846251 -122.74835907158386) bank188044 +188045 POINT(37.95320742615814 -122.66228181049898) bank188045 +188046 POINT(36.95624220187748 -122.46694011424742) bank188046 +188047 POINT(37.47890827352458 -121.92704854533508) bank188047 +188048 POINT(37.60847830493872 -123.28136138126206) bank188048 +188049 POINT(37.92247441946966 -122.26628211993528) bank188049 +188050 POINT(38.340035380960224 -121.68600509819267) bank188050 +188051 POINT(36.83090528308933 -123.34173516469815) bank188051 +188052 POINT(36.981158837617265 -123.35615877468776) bank188052 +188053 POINT(38.50538329261375 -122.83093476255075) bank188053 +188054 POINT(37.23604340479578 -122.6072243739697) bank188054 +188055 POINT(36.97687278079728 -122.75854423849161) bank188055 +188056 POINT(37.97542486956461 -123.14847222837192) bank188056 +188057 POINT(37.50062916096668 -122.89777160061392) bank188057 +188058 POINT(38.643714685545895 -121.99514419878884) bank188058 +188059 POINT(37.959096816910325 -122.72166709128103) bank188059 +188060 POINT(37.9100385274021 -123.31175138404603) bank188060 +188061 POINT(38.69641744356137 -122.62438400141575) bank188061 +188062 POINT(37.096643080006466 -121.83775455055235) bank188062 +188063 POINT(37.190543310091066 -122.38995605732809) bank188063 +188064 POINT(38.0551694227746 -122.40571152294767) bank188064 +188065 POINT(37.695338795763156 -122.56172913742516) bank188065 +188066 POINT(37.85494076440435 -122.5772413713172) bank188066 +188067 POINT(38.695862888728115 -122.49758534314456) bank188067 +188068 POINT(37.95813677223279 -122.42082292296745) bank188068 +188069 POINT(37.76226135143176 -121.52708514733439) bank188069 +188070 POINT(38.73303310370832 -123.10196087400881) bank188070 +188071 POINT(38.68956569544795 -122.77423663913189) bank188071 +188072 POINT(37.258665536459674 -121.79639515726805) bank188072 +188073 POINT(37.37558278678423 -122.29105389135951) bank188073 +188074 POINT(37.5064844003544 -121.5364255054223) bank188074 +188075 POINT(38.03673223006579 -122.30594274169901) bank188075 +188076 POINT(37.99715911704884 -121.71354240709856) bank188076 +188077 POINT(38.011145440929404 -122.70680324422982) bank188077 +188078 POINT(38.1104963246608 -122.88534050158015) bank188078 +188079 POINT(38.51406286964708 -122.64572597670639) bank188079 +188080 POINT(38.77223615041536 -122.75553693853969) bank188080 +188081 POINT(38.20783337155354 -122.26543768472342) bank188081 +188082 POINT(37.904161061188084 -123.32612058718884) bank188082 +188083 POINT(37.1334965025526 -123.08762112101648) bank188083 +188084 POINT(38.34911222733602 -123.29468223293068) bank188084 +188085 POINT(36.889714421668444 -122.34058879564373) bank188085 +188086 POINT(37.84911935986689 -121.88470602261725) bank188086 +188087 POINT(38.58176764558225 -122.42463850988331) bank188087 +188088 POINT(37.08358913283566 -121.4819916292427) bank188088 +188089 POINT(37.65625310785652 -121.90306158529745) bank188089 +188090 POINT(36.877904811058556 -123.03293262790494) bank188090 +188091 POINT(38.324098032755586 -122.84811308452348) bank188091 +188092 POINT(38.51563771431309 -121.67034482484543) bank188092 +188093 POINT(37.91477216756268 -122.73719010519095) bank188093 +188094 POINT(38.43540795069086 -122.11770557264586) bank188094 +188095 POINT(38.04073423553607 -122.08796590148137) bank188095 +188096 POINT(37.829429911691 -122.85062623230759) bank188096 +188097 POINT(37.32073819402481 -122.16880428966431) bank188097 +188098 POINT(38.498119829371554 -121.97917076360704) bank188098 +188099 POINT(37.649729156293276 -123.19382736739348) bank188099 +188100 POINT(38.07328175185558 -121.49984512625555) bank188100 +188101 POINT(38.20776186589232 -121.82328114412373) bank188101 +188102 POINT(37.76315383625385 -122.78526964726407) bank188102 +188103 POINT(38.30232983412959 -122.78147923816238) bank188103 +188104 POINT(37.30469540810383 -121.66305824924149) bank188104 +188105 POINT(38.32550379488351 -123.27231691948774) bank188105 +188106 POINT(38.39993066835188 -122.54172976093167) bank188106 +188107 POINT(38.040311129285996 -123.24129434388492) bank188107 +188108 POINT(38.3914253375091 -122.50079007977038) bank188108 +188109 POINT(37.044111955889704 -121.51975297131034) bank188109 +188110 POINT(37.790047306477376 -123.00283760262155) bank188110 +188111 POINT(37.41828218878922 -123.23718469152882) bank188111 +188112 POINT(37.64256906126727 -123.17440426000933) bank188112 +188113 POINT(38.370992941764456 -122.16610518340079) bank188113 +188114 POINT(38.11152454531709 -121.6857123047931) bank188114 +188115 POINT(38.72859989006515 -121.46313396573906) bank188115 +188116 POINT(36.86055428895891 -122.79664148075852) bank188116 +188117 POINT(37.367459221092055 -122.61061466843756) bank188117 +188118 POINT(38.29704530847785 -121.58603552102826) bank188118 +188119 POINT(38.20386463158131 -122.33197167785) bank188119 +188120 POINT(38.23816376384457 -122.98099801107305) bank188120 +188121 POINT(37.55362098578077 -122.51247299741048) bank188121 +188122 POINT(37.34343245408838 -122.58880585910471) bank188122 +188123 POINT(38.01133122696083 -121.72186096888606) bank188123 +188124 POINT(36.82615807080485 -122.19270853054182) bank188124 +188125 POINT(37.49921101356566 -121.90471051378793) bank188125 +188126 POINT(38.60541945339208 -122.8546037065921) bank188126 +188127 POINT(37.865902095038834 -123.08867807873446) bank188127 +188128 POINT(37.170166820217034 -122.84656168418068) bank188128 +188129 POINT(37.648773303424164 -122.08395816715675) bank188129 +188130 POINT(38.19076216491817 -122.34953398689477) bank188130 +188131 POINT(37.12281865754387 -122.80017262999806) bank188131 +188132 POINT(38.211923918401894 -122.99612405940525) bank188132 +188133 POINT(37.14265456034415 -121.49129092519455) bank188133 +188134 POINT(38.70849245661167 -122.57102841785957) bank188134 +188135 POINT(38.143558741492974 -122.96383574359909) bank188135 +188136 POINT(38.546408286161025 -123.38050829627731) bank188136 +188137 POINT(36.85051766504122 -122.29056154697989) bank188137 +188138 POINT(37.935852050770904 -121.50561513552273) bank188138 +188139 POINT(37.02332729999285 -122.76560723123512) bank188139 +188140 POINT(37.09658357888337 -122.7959351334905) bank188140 +188141 POINT(37.260290931795396 -121.58486329028665) bank188141 +188142 POINT(36.9656113135536 -123.20067333237888) bank188142 +188143 POINT(37.28309945203043 -122.00916428667003) bank188143 +188144 POINT(38.75201131035257 -121.97579277989868) bank188144 +188145 POINT(37.163148317105126 -121.49436111194154) bank188145 +188146 POINT(37.76361091927567 -123.09584810362925) bank188146 +188147 POINT(37.53713868401117 -122.55906368447627) bank188147 +188148 POINT(37.69662410909651 -122.14234024132377) bank188148 +188149 POINT(37.26836269297557 -121.64347990072149) bank188149 +188150 POINT(36.8049678823421 -122.26247404004883) bank188150 +188151 POINT(36.80863502664108 -121.62488720639043) bank188151 +188152 POINT(38.565040726219735 -123.22970868452674) bank188152 +188153 POINT(38.21783118870724 -121.70138908854351) bank188153 +188154 POINT(38.356211834683414 -121.60665559252858) bank188154 +188155 POINT(38.240020334487845 -122.93409240101705) bank188155 +188156 POINT(37.455569435203685 -122.21938253248224) bank188156 +188157 POINT(36.930240826070346 -121.72567325413344) bank188157 +188158 POINT(37.30643952317651 -123.0622210265508) bank188158 +188159 POINT(38.59213022498858 -122.56787828188862) bank188159 +188160 POINT(38.363323823337424 -121.77861324998702) bank188160 +188161 POINT(37.65795503186265 -122.04574144351781) bank188161 +188162 POINT(37.76198717064645 -121.80863795059453) bank188162 +188163 POINT(37.84712358992613 -121.7384273861558) bank188163 +188164 POINT(38.27733641010595 -122.67118247250075) bank188164 +188165 POINT(36.81078450199913 -121.69505200238761) bank188165 +188166 POINT(36.987111701267764 -123.17688484253301) bank188166 +188167 POINT(37.054249953892665 -122.89304196931646) bank188167 +188168 POINT(37.02976931134684 -122.95772371354472) bank188168 +188169 POINT(38.33105314053417 -123.32441462080733) bank188169 +188170 POINT(38.3024907353544 -121.80689098869786) bank188170 +188171 POINT(38.63320407139838 -122.66430919197704) bank188171 +188172 POINT(37.281033397350285 -121.96521638044999) bank188172 +188173 POINT(36.97684944800983 -122.28699689373973) bank188173 +188174 POINT(37.04851446592825 -122.63073197527453) bank188174 +188175 POINT(37.991446447695154 -123.10497072967682) bank188175 +188176 POINT(37.52786888411271 -122.12915349963133) bank188176 +188177 POINT(37.90969446444147 -123.30237705973997) bank188177 +188178 POINT(37.18926324000432 -123.41144440274064) bank188178 +188179 POINT(37.49000850653989 -121.46325037682615) bank188179 +188180 POINT(37.10947179085723 -122.70773043492161) bank188180 +188181 POINT(38.67100067878669 -121.72954781122793) bank188181 +188182 POINT(37.3896766030028 -122.98047396307875) bank188182 +188183 POINT(37.273066243307326 -122.07975728629003) bank188183 +188184 POINT(37.90150666094863 -121.83049225370661) bank188184 +188185 POINT(38.02053060353419 -122.80490547368359) bank188185 +188186 POINT(38.34146705600172 -123.19829229871036) bank188186 +188187 POINT(36.87497297774939 -121.83665392592769) bank188187 +188188 POINT(38.75874706312718 -123.11102704211562) bank188188 +188189 POINT(37.87621933967282 -122.56861478293943) bank188189 +188190 POINT(38.735217489401485 -121.49119439323495) bank188190 +188191 POINT(37.53427505665236 -123.17584968062955) bank188191 +188192 POINT(37.57294782863093 -122.42387752568713) bank188192 +188193 POINT(37.73679580119133 -122.17424144448232) bank188193 +188194 POINT(38.346369246525754 -122.63711264380787) bank188194 +188195 POINT(37.303877375442845 -122.00163916941298) bank188195 +188196 POINT(37.62770283022332 -122.42662644839707) bank188196 +188197 POINT(38.479807082068874 -122.54965528480831) bank188197 +188198 POINT(38.12347463590328 -123.40017554089201) bank188198 +188199 POINT(37.42779546231293 -122.41626760371138) bank188199 +188200 POINT(37.37912464808318 -122.836948051791) bank188200 +188201 POINT(38.20736631538405 -123.33159574051628) bank188201 +188202 POINT(36.92015454809774 -123.17256606912723) bank188202 +188203 POINT(38.343107383453095 -122.38557104867112) bank188203 +188204 POINT(37.97795787208723 -121.4353629468545) bank188204 +188205 POINT(37.402550761102226 -121.56281214978084) bank188205 +188206 POINT(38.21329354308757 -123.26499246164379) bank188206 +188207 POINT(38.375866030222234 -122.14952257477285) bank188207 +188208 POINT(36.843450261731405 -121.47286121548602) bank188208 +188209 POINT(38.61822481932604 -121.73948389899029) bank188209 +188210 POINT(38.65193138535198 -121.45555640684202) bank188210 +188211 POINT(38.37067016528414 -122.54658439385442) bank188211 +188212 POINT(37.66804572084605 -122.26225794302414) bank188212 +188213 POINT(37.148313353340626 -121.61236094369355) bank188213 +188214 POINT(37.24232014811931 -123.20549840113785) bank188214 +188215 POINT(37.495456647248055 -121.9298557164891) bank188215 +188216 POINT(38.40648355583723 -121.92958032849224) bank188216 +188217 POINT(36.88306023630018 -123.35452990547667) bank188217 +188218 POINT(36.98012844618151 -122.74530384740152) bank188218 +188219 POINT(38.074160028719035 -121.62835249755597) bank188219 +188220 POINT(38.41347100414409 -121.73357942257464) bank188220 +188221 POINT(38.76875107363032 -122.44063783693369) bank188221 +188222 POINT(37.34825023361981 -122.78566434318131) bank188222 +188223 POINT(37.57921899974437 -121.83652265147325) bank188223 +188224 POINT(37.39776572453609 -122.29017347198516) bank188224 +188225 POINT(37.662175747540076 -122.9642187250696) bank188225 +188226 POINT(37.470847680187255 -122.1967939416202) bank188226 +188227 POINT(37.22989538354085 -122.61555968832525) bank188227 +188228 POINT(37.507731402050915 -122.25298814094525) bank188228 +188229 POINT(37.29341440229258 -121.55669992128416) bank188229 +188230 POINT(38.676729110200185 -122.31337472923963) bank188230 +188231 POINT(38.65885656256363 -121.54869603285324) bank188231 +188232 POINT(37.04503659674938 -122.46682819738186) bank188232 +188233 POINT(38.14625828446384 -122.30681350364999) bank188233 +188234 POINT(37.275474512855276 -121.59964013876954) bank188234 +188235 POINT(38.72162026079329 -122.05555223197725) bank188235 +188236 POINT(38.24226968135314 -122.23221152654187) bank188236 +188237 POINT(38.708104211895105 -123.18242512826178) bank188237 +188238 POINT(36.794752688968664 -122.09510734902867) bank188238 +188239 POINT(36.822022210403276 -123.12128363214933) bank188239 +188240 POINT(37.42999062484776 -123.15885740560188) bank188240 +188241 POINT(37.06486062612226 -122.07881009299477) bank188241 +188242 POINT(38.44096965701384 -123.2596672009492) bank188242 +188243 POINT(37.672506177256615 -122.48560684544326) bank188243 +188244 POINT(37.729209815621005 -123.31316710259398) bank188244 +188245 POINT(37.45673124926469 -121.71163586506187) bank188245 +188246 POINT(37.71820815843772 -122.24143063466902) bank188246 +188247 POINT(38.56646319417574 -122.92446983237967) bank188247 +188248 POINT(38.68910686745424 -122.43189601617856) bank188248 +188249 POINT(38.661478297392314 -121.53752338671215) bank188249 +188250 POINT(38.0243883405589 -121.94492284588816) bank188250 +188251 POINT(38.636500845417075 -122.9269531903829) bank188251 +188252 POINT(36.78478163781063 -122.70724207055311) bank188252 +188253 POINT(38.26050946429791 -121.89230938654497) bank188253 +188254 POINT(38.433108086867136 -122.88331543071028) bank188254 +188255 POINT(37.07708675277943 -121.82710417638951) bank188255 +188256 POINT(37.472436007706875 -121.94071637777014) bank188256 +188257 POINT(37.7919048963207 -121.79431866071126) bank188257 +188258 POINT(37.29932898933684 -123.17414796000138) bank188258 +188259 POINT(37.119938976730985 -122.65644598542177) bank188259 +188260 POINT(36.9038933704142 -121.70477988363338) bank188260 +188261 POINT(38.6059907110031 -122.06633643318132) bank188261 +188262 POINT(37.37808686189879 -122.05730615449062) bank188262 +188263 POINT(38.30188924540441 -122.87108091389443) bank188263 +188264 POINT(37.56668155034505 -122.24659385123095) bank188264 +188265 POINT(37.19456258623485 -122.60763642707478) bank188265 +188266 POINT(37.83120452221614 -123.18369307591789) bank188266 +188267 POINT(37.01970387679592 -121.48942746873124) bank188267 +188268 POINT(37.26245050684449 -121.9524757234) bank188268 +188269 POINT(37.93570855777186 -122.26964883588809) bank188269 +188270 POINT(37.10344773377127 -122.59198906553843) bank188270 +188271 POINT(37.589410638416176 -122.00791240024923) bank188271 +188272 POINT(38.187136903307575 -122.41461716305012) bank188272 +188273 POINT(37.47646134281309 -122.79626650629965) bank188273 +188274 POINT(38.465570224689664 -121.94100569325256) bank188274 +188275 POINT(36.86085848321961 -122.40664338467217) bank188275 +188276 POINT(36.94560694583046 -122.44755351906691) bank188276 +188277 POINT(37.79000968970369 -122.19247144145812) bank188277 +188278 POINT(37.72559475073581 -122.69212406837333) bank188278 +188279 POINT(38.75648374650589 -122.93363488685253) bank188279 +188280 POINT(38.13144953760361 -121.79664317351529) bank188280 +188281 POINT(38.16952940056868 -121.79645551596863) bank188281 +188282 POINT(38.2846391320462 -122.89530762673937) bank188282 +188283 POINT(37.6414495909379 -121.80460952696278) bank188283 +188284 POINT(37.02521796148267 -122.78222619936345) bank188284 +188285 POINT(37.75608666036655 -122.90546377181577) bank188285 +188286 POINT(37.154921964003535 -122.67112092189144) bank188286 +188287 POINT(37.26895336923374 -122.39142847381976) bank188287 +188288 POINT(37.40106382738901 -122.41307806946229) bank188288 +188289 POINT(37.77365911499791 -121.8878733665396) bank188289 +188290 POINT(36.80040128595632 -121.85146530580344) bank188290 +188291 POINT(37.235846430694835 -122.17727637368661) bank188291 +188292 POINT(37.63677604332707 -122.50375276184963) bank188292 +188293 POINT(37.76123823562107 -122.98325727396397) bank188293 +188294 POINT(36.778226913011665 -121.69945780873158) bank188294 +188295 POINT(38.33792961073932 -122.58858627667856) bank188295 +188296 POINT(37.461415244572336 -123.18735927592434) bank188296 +188297 POINT(38.533156540285155 -122.52714356514416) bank188297 +188298 POINT(37.76634016081344 -123.30548022480409) bank188298 +188299 POINT(37.55669826872614 -122.94282481615875) bank188299 +188300 POINT(38.655041547776484 -122.28934647526532) bank188300 +188301 POINT(37.55609405762745 -122.40733649168709) bank188301 +188302 POINT(38.7152373917523 -122.69553766045476) bank188302 +188303 POINT(37.594059178679224 -121.93210689626446) bank188303 +188304 POINT(38.2211913399352 -121.91733947336397) bank188304 +188305 POINT(37.95387589782827 -122.30585979949097) bank188305 +188306 POINT(38.3447117513762 -121.99055655683773) bank188306 +188307 POINT(37.516297557825546 -121.86303069441239) bank188307 +188308 POINT(37.86277392879259 -123.11532943355786) bank188308 +188309 POINT(38.614256967918514 -122.81013633275055) bank188309 +188310 POINT(38.205974127519774 -122.44679104448399) bank188310 +188311 POINT(37.45964216067173 -123.02078227650712) bank188311 +188312 POINT(38.22182117428697 -121.87592059475705) bank188312 +188313 POINT(37.36014009923303 -122.74788990644221) bank188313 +188314 POINT(38.22751004401684 -123.20132427741483) bank188314 +188315 POINT(38.53769804217051 -122.73118402837632) bank188315 +188316 POINT(38.60297766418546 -122.8809580280065) bank188316 +188317 POINT(38.035532748797756 -122.96848531444077) bank188317 +188318 POINT(37.260822820550814 -122.30261608845736) bank188318 +188319 POINT(38.57583975784204 -122.23400297202471) bank188319 +188320 POINT(38.1929375593709 -121.78707169393849) bank188320 +188321 POINT(37.24633510882164 -122.78009021981856) bank188321 +188322 POINT(37.9905974406894 -121.98754465684841) bank188322 +188323 POINT(37.764463918023786 -121.51023323698232) bank188323 +188324 POINT(38.726109033460325 -121.64205692470104) bank188324 +188325 POINT(37.053994865261416 -122.7646382110788) bank188325 +188326 POINT(38.337646918680505 -121.4441915503251) bank188326 +188327 POINT(37.321004953650885 -122.40246412119048) bank188327 +188328 POINT(37.72792779135384 -122.44245001916404) bank188328 +188329 POINT(38.40674163077661 -122.89327444672568) bank188329 +188330 POINT(37.544700843050705 -122.4934430876045) bank188330 +188331 POINT(38.74123987322962 -121.58212672838684) bank188331 +188332 POINT(38.64847400548628 -122.37976148564654) bank188332 +188333 POINT(38.58759484480297 -122.02607948458406) bank188333 +188334 POINT(37.98446751800999 -123.03963607874125) bank188334 +188335 POINT(38.75887288680098 -123.23766539804146) bank188335 +188336 POINT(37.83238506915761 -123.2861235188605) bank188336 +188337 POINT(37.091570917899865 -122.61291070406476) bank188337 +188338 POINT(38.13576570937457 -122.42850415543418) bank188338 +188339 POINT(37.16376545213827 -122.81691524344406) bank188339 +188340 POINT(38.40138083999575 -123.21171205922512) bank188340 +188341 POINT(37.242024251657014 -122.06402881714942) bank188341 +188342 POINT(37.53761219442535 -122.12311826105346) bank188342 +188343 POINT(37.51833167703273 -123.36975555853147) bank188343 +188344 POINT(37.36023881710055 -122.73100514692062) bank188344 +188345 POINT(38.014823602986745 -121.6138800817388) bank188345 +188346 POINT(37.095592152757334 -122.42295577082214) bank188346 +188347 POINT(37.24468374254512 -122.5491935945675) bank188347 +188348 POINT(37.682862156159246 -122.13255475877409) bank188348 +188349 POINT(36.815557115355915 -123.11305398078456) bank188349 +188350 POINT(36.882480654533026 -123.14159121790415) bank188350 +188351 POINT(38.52241743677974 -123.0796163456502) bank188351 +188352 POINT(37.774520264097994 -122.78422833839612) bank188352 +188353 POINT(38.03178356169046 -123.30846644437528) bank188353 +188354 POINT(37.58389409756545 -122.31022492087263) bank188354 +188355 POINT(38.30377466283998 -121.89342478423745) bank188355 +188356 POINT(36.84835375819092 -121.50011089546358) bank188356 +188357 POINT(37.18289795176546 -122.98310989041366) bank188357 +188358 POINT(38.472279234871884 -122.04660573982848) bank188358 +188359 POINT(37.33047883386043 -122.24787584004387) bank188359 +188360 POINT(37.81509719276566 -123.364728352088) bank188360 +188361 POINT(38.58656496761258 -122.44401696952893) bank188361 +188362 POINT(38.08964512370153 -123.01932099941449) bank188362 +188363 POINT(37.51381561006197 -123.17874688805104) bank188363 +188364 POINT(36.78961861952493 -122.40388479813483) bank188364 +188365 POINT(38.03833211062557 -123.14153935824527) bank188365 +188366 POINT(37.957984696614076 -122.19070461591419) bank188366 +188367 POINT(38.33498202084597 -122.88403828160979) bank188367 +188368 POINT(37.62326804082996 -123.19609012188694) bank188368 +188369 POINT(37.533759342170946 -123.13216090360224) bank188369 +188370 POINT(37.75686806325412 -122.74846692374975) bank188370 +188371 POINT(38.71160453463473 -121.53166609349074) bank188371 +188372 POINT(37.15536685633763 -121.48560599145081) bank188372 +188373 POINT(37.37267590434646 -122.36081777135975) bank188373 +188374 POINT(36.874496736094656 -121.5698482300973) bank188374 +188375 POINT(38.193312716915926 -122.96895113717753) bank188375 +188376 POINT(37.37739555867464 -121.67763076255419) bank188376 +188377 POINT(38.752559692063606 -122.08220757954493) bank188377 +188378 POINT(38.31909661476462 -122.31796257715914) bank188378 +188379 POINT(37.127646968052865 -123.38049695266346) bank188379 +188380 POINT(37.52608971231693 -121.50675158535931) bank188380 +188381 POINT(38.714713503058086 -123.01586386049058) bank188381 +188382 POINT(38.43412082691521 -123.27344159231737) bank188382 +188383 POINT(37.99141994529147 -121.449148948214) bank188383 +188384 POINT(37.887464889515755 -121.81083925415908) bank188384 +188385 POINT(38.33077911393544 -122.63772552221313) bank188385 +188386 POINT(38.40814492261915 -121.81139541299463) bank188386 +188387 POINT(37.8650661943965 -122.89810645787068) bank188387 +188388 POINT(37.6549356248072 -122.76430211493627) bank188388 +188389 POINT(37.63581586011036 -122.12137331083255) bank188389 +188390 POINT(38.11016684306602 -121.78062903148513) bank188390 +188391 POINT(37.975615272309255 -122.13744749715323) bank188391 +188392 POINT(36.888110264174024 -122.15549147764601) bank188392 +188393 POINT(38.221448151134126 -122.35197902879484) bank188393 +188394 POINT(38.36637674081159 -122.41879768588657) bank188394 +188395 POINT(37.985044840299594 -122.65306638100849) bank188395 +188396 POINT(37.846226448509576 -121.8428643013675) bank188396 +188397 POINT(37.82396940654992 -123.21928911900599) bank188397 +188398 POINT(37.154885806555 -122.58618270308763) bank188398 +188399 POINT(37.59333316229345 -122.30669416182756) bank188399 +188400 POINT(38.03207421739037 -123.21009715146455) bank188400 +188401 POINT(36.858226722485895 -123.0364225811133) bank188401 +188402 POINT(37.256833021301134 -123.30244594388599) bank188402 +188403 POINT(37.58380855955053 -123.01576998983923) bank188403 +188404 POINT(36.81239829989004 -122.53890555477328) bank188404 +188405 POINT(38.218941226951046 -123.26169847207959) bank188405 +188406 POINT(37.824803031783055 -122.63033365159878) bank188406 +188407 POINT(38.64259735859364 -122.02740668313547) bank188407 +188408 POINT(38.220232496778685 -121.45601113871034) bank188408 +188409 POINT(37.491039224505336 -121.67326588970228) bank188409 +188410 POINT(38.71435800860598 -123.2546606627608) bank188410 +188411 POINT(37.64221241165603 -122.03242404638802) bank188411 +188412 POINT(37.8560599417987 -123.00886387537685) bank188412 +188413 POINT(37.59754280217778 -122.69262271921589) bank188413 +188414 POINT(38.58822512066667 -123.21528799610998) bank188414 +188415 POINT(37.6844210665545 -121.6852483609204) bank188415 +188416 POINT(37.09006995353949 -123.21554496163358) bank188416 +188417 POINT(37.332000456632684 -122.97402175013112) bank188417 +188418 POINT(37.15162968072615 -122.26446736805806) bank188418 +188419 POINT(37.75852061101836 -121.49858322912145) bank188419 +188420 POINT(38.52119574729144 -122.11449733728021) bank188420 +188421 POINT(37.96780587699285 -122.14582664014529) bank188421 +188422 POINT(38.17610229185629 -121.74944641945412) bank188422 +188423 POINT(37.24453194598615 -122.60924564012605) bank188423 +188424 POINT(36.83489676848259 -121.98870037362623) bank188424 +188425 POINT(37.19561121546148 -122.1984576637222) bank188425 +188426 POINT(36.809813760545275 -123.3695861566584) bank188426 +188427 POINT(36.95367740138021 -122.22772601519365) bank188427 +188428 POINT(37.30087287858104 -121.9254915358215) bank188428 +188429 POINT(38.07442438928024 -121.70067748348987) bank188429 +188430 POINT(37.43484337857383 -122.8190184117589) bank188430 +188431 POINT(38.575797226926696 -121.75535185008714) bank188431 +188432 POINT(36.95926458254636 -122.98237962808855) bank188432 +188433 POINT(37.17945590553527 -123.02079965130625) bank188433 +188434 POINT(37.9100954103065 -122.12124112052824) bank188434 +188435 POINT(38.15054482556812 -122.91733287419541) bank188435 +188436 POINT(37.50326874161379 -122.90523699006935) bank188436 +188437 POINT(37.081188103663315 -122.3910526394795) bank188437 +188438 POINT(37.19369342534846 -122.13248252414732) bank188438 +188439 POINT(36.91634002049631 -122.43327043692318) bank188439 +188440 POINT(37.18717421313668 -123.30279686528642) bank188440 +188441 POINT(37.427807081591205 -121.45622632433715) bank188441 +188442 POINT(37.120103755207886 -122.92872469236912) bank188442 +188443 POINT(37.91909901189619 -122.34277839992447) bank188443 +188444 POINT(37.90551784714619 -122.74277048045225) bank188444 +188445 POINT(38.00632435321554 -123.20161539265081) bank188445 +188446 POINT(37.61064434028542 -123.123278464164) bank188446 +188447 POINT(38.38704843292955 -122.49965987262895) bank188447 +188448 POINT(37.944669062170654 -122.51390480196054) bank188448 +188449 POINT(37.82877953831093 -123.39414462413693) bank188449 +188450 POINT(37.001937208729004 -121.88044856398187) bank188450 +188451 POINT(37.310020861789376 -122.71616718973199) bank188451 +188452 POINT(37.49749689771488 -123.19473429699957) bank188452 +188453 POINT(37.16154905451655 -123.1293873869096) bank188453 +188454 POINT(37.36160381426627 -121.61186737136367) bank188454 +188455 POINT(37.83694958550763 -122.65965268393977) bank188455 +188456 POINT(37.546900453697525 -122.98283569263053) bank188456 +188457 POINT(37.62909705193108 -122.36748110142061) bank188457 +188458 POINT(38.56782980821365 -122.85369657902433) bank188458 +188459 POINT(38.21171905191013 -123.16966768227601) bank188459 +188460 POINT(38.22409346407274 -123.28745980293989) bank188460 +188461 POINT(38.45768685308868 -122.3836293441088) bank188461 +188462 POINT(37.41813880503077 -123.22372762748611) bank188462 +188463 POINT(38.09887669118063 -123.34715112939905) bank188463 +188464 POINT(37.7463512570473 -122.3492533480829) bank188464 +188465 POINT(36.83170083556189 -122.69697139261592) bank188465 +188466 POINT(37.39041723835246 -122.59183514081953) bank188466 +188467 POINT(37.56160489096759 -122.40560621245653) bank188467 +188468 POINT(38.25414010768971 -122.44529812333312) bank188468 +188469 POINT(37.35533244446709 -121.62883299417626) bank188469 +188470 POINT(38.33732673485312 -121.61426390860588) bank188470 +188471 POINT(38.113463430120056 -122.43198599290236) bank188471 +188472 POINT(38.51181472297851 -121.73113241225157) bank188472 +188473 POINT(38.668816694178 -122.4959085387479) bank188473 +188474 POINT(37.25446802009098 -123.19206983668028) bank188474 +188475 POINT(37.68903214749197 -121.42602477757454) bank188475 +188476 POINT(36.87169115553166 -122.63892097261689) bank188476 +188477 POINT(37.79031196288663 -123.3122238571983) bank188477 +188478 POINT(38.72717343221475 -122.97727127419407) bank188478 +188479 POINT(37.68670705495316 -122.26026856128924) bank188479 +188480 POINT(36.929342217231415 -121.88359423797559) bank188480 +188481 POINT(37.21603016333742 -123.37051586573381) bank188481 +188482 POINT(38.34596525809013 -121.91201283135867) bank188482 +188483 POINT(36.8990129180671 -122.81504245515573) bank188483 +188484 POINT(37.748993466298444 -122.43709174227928) bank188484 +188485 POINT(38.585620916075314 -122.43152162006454) bank188485 +188486 POINT(37.7023054960879 -121.5462169178021) bank188486 +188487 POINT(38.3712456023028 -122.79255199664166) bank188487 +188488 POINT(38.28129112137882 -121.50629362576903) bank188488 +188489 POINT(38.43683526258919 -122.27827083334667) bank188489 +188490 POINT(37.110459163280034 -122.25055920932873) bank188490 +188491 POINT(37.907906117407656 -122.92973005359566) bank188491 +188492 POINT(37.14377771959458 -121.68447621668302) bank188492 +188493 POINT(38.33137241529506 -123.00984541410608) bank188493 +188494 POINT(38.607603034710394 -123.26766413338198) bank188494 +188495 POINT(37.055906759827735 -123.20365483355111) bank188495 +188496 POINT(38.30627777683869 -121.77840831323209) bank188496 +188497 POINT(36.9128609251654 -122.37981844555075) bank188497 +188498 POINT(37.93474664802141 -122.95079606798927) bank188498 +188499 POINT(36.92868345295994 -121.47303986169544) bank188499 +188500 POINT(37.23209413690318 -123.22101473623772) bank188500 +188501 POINT(38.55252272769878 -122.14137581850977) bank188501 +188502 POINT(37.99517727225863 -121.84581871338162) bank188502 +188503 POINT(38.45182629501619 -122.52961022883294) bank188503 +188504 POINT(37.83245320079316 -122.46587146160473) bank188504 +188505 POINT(37.32275064079052 -121.87024688013727) bank188505 +188506 POINT(38.06389304771038 -123.18076167853944) bank188506 +188507 POINT(38.05235181146204 -122.14705691223467) bank188507 +188508 POINT(38.022664118274285 -122.3108307078484) bank188508 +188509 POINT(38.10759186215811 -123.13997208093713) bank188509 +188510 POINT(38.77305393875274 -123.3513596672171) bank188510 +188511 POINT(38.30643788986262 -121.70101970676694) bank188511 +188512 POINT(37.77003641123778 -122.84329979097718) bank188512 +188513 POINT(37.9820289493915 -123.3829279249728) bank188513 +188514 POINT(36.804917926682116 -123.18388404169788) bank188514 +188515 POINT(37.464073768260604 -121.6216483521504) bank188515 +188516 POINT(37.56956943430936 -121.56424873748678) bank188516 +188517 POINT(37.984372644488175 -122.79821845544734) bank188517 +188518 POINT(38.613234438783124 -121.61177410198133) bank188518 +188519 POINT(37.297867288853226 -121.73848677969497) bank188519 +188520 POINT(37.67701392388658 -123.1028840223313) bank188520 +188521 POINT(37.83287027962258 -122.97347944656359) bank188521 +188522 POINT(37.966543450007556 -122.80779299446104) bank188522 +188523 POINT(37.4469568138539 -123.19096358544913) bank188523 +188524 POINT(37.669033073497516 -123.21268867612704) bank188524 +188525 POINT(38.323337123003895 -121.76786034197681) bank188525 +188526 POINT(37.2888132599696 -121.86698841179748) bank188526 +188527 POINT(38.18630526461581 -122.16129209839896) bank188527 +188528 POINT(36.85569542171689 -122.41211333914134) bank188528 +188529 POINT(38.56813662688762 -122.59330915130246) bank188529 +188530 POINT(37.1637735965623 -123.04162739391961) bank188530 +188531 POINT(38.088211420176414 -123.37263670480112) bank188531 +188532 POINT(38.39416761674974 -122.97749390115915) bank188532 +188533 POINT(37.16881717728597 -122.08808754530385) bank188533 +188534 POINT(36.86465213317974 -122.63456632787376) bank188534 +188535 POINT(38.33377815261974 -121.82656195313955) bank188535 +188536 POINT(37.70578182650797 -121.74387672304636) bank188536 +188537 POINT(38.584761875399806 -121.8587042269613) bank188537 +188538 POINT(38.30322639158688 -122.49163377265154) bank188538 +188539 POINT(36.86047658392539 -122.30079533300027) bank188539 +188540 POINT(38.35287597366334 -121.75978703016024) bank188540 +188541 POINT(36.88708360304679 -121.77576275148903) bank188541 +188542 POINT(37.56891427375369 -123.0363601193918) bank188542 +188543 POINT(37.33880115541916 -121.9883837739224) bank188543 +188544 POINT(38.160498702863734 -122.9097863606695) bank188544 +188545 POINT(38.23030907895938 -122.65703385721206) bank188545 +188546 POINT(37.75992015016845 -121.94175868827364) bank188546 +188547 POINT(37.835143372749954 -121.56684850729314) bank188547 +188548 POINT(37.55034572153415 -123.13114478937136) bank188548 +188549 POINT(37.019625544141235 -123.08950894625688) bank188549 +188550 POINT(37.29559434293308 -122.39712927591059) bank188550 +188551 POINT(38.449652156413606 -121.76525657179675) bank188551 +188552 POINT(37.76668694989815 -122.4926123210777) bank188552 +188553 POINT(37.53846429129302 -121.97919501267998) bank188553 +188554 POINT(38.628743609298965 -123.2890358717136) bank188554 +188555 POINT(37.170477501708966 -121.65168519513374) bank188555 +188556 POINT(36.98399529402611 -122.72084668500388) bank188556 +188557 POINT(36.998565028820195 -123.1391220912119) bank188557 +188558 POINT(38.33837731660927 -123.20355659272438) bank188558 +188559 POINT(38.54575293069314 -122.36961297067964) bank188559 +188560 POINT(38.54229032386478 -123.40523894861478) bank188560 +188561 POINT(37.01705750954834 -123.03776662535547) bank188561 +188562 POINT(37.49562549725068 -122.13565136660166) bank188562 +188563 POINT(37.06009025455785 -122.25263295034266) bank188563 +188564 POINT(37.38182831112362 -122.19019834308124) bank188564 +188565 POINT(37.912351219354875 -121.56231710580816) bank188565 +188566 POINT(37.268493193456145 -123.05851008764384) bank188566 +188567 POINT(38.19066435038723 -122.88587846605678) bank188567 +188568 POINT(36.91181596425549 -121.88661662594704) bank188568 +188569 POINT(36.82060772868583 -122.00168088299071) bank188569 +188570 POINT(37.164687599264155 -122.4340296960824) bank188570 +188571 POINT(37.91036150528162 -121.85465604354636) bank188571 +188572 POINT(38.19954992238762 -122.49189210765257) bank188572 +188573 POINT(37.267467151523206 -122.6191296768613) bank188573 +188574 POINT(38.212668665020075 -122.65571482867969) bank188574 +188575 POINT(38.61611663306708 -122.91037028242665) bank188575 +188576 POINT(38.24996314383076 -123.20914840432002) bank188576 +188577 POINT(37.27950253698265 -122.38196196138342) bank188577 +188578 POINT(37.936274127178635 -122.66138056066802) bank188578 +188579 POINT(37.17801806070723 -122.3365682426634) bank188579 +188580 POINT(38.518813874846686 -123.13431537502498) bank188580 +188581 POINT(38.187237669584476 -121.74951542716603) bank188581 +188582 POINT(36.94691951479522 -122.49543827373581) bank188582 +188583 POINT(37.751090735769395 -121.87986272160576) bank188583 +188584 POINT(37.14165757754373 -121.91130853854692) bank188584 +188585 POINT(37.15477280652716 -122.06520302379177) bank188585 +188586 POINT(37.78030086386531 -121.85334473257853) bank188586 +188587 POINT(38.04630267840577 -122.42798273299526) bank188587 +188588 POINT(37.44561927264626 -123.36933425699544) bank188588 +188589 POINT(36.905275838399675 -122.69611108353463) bank188589 +188590 POINT(38.362448300983274 -122.37446307787518) bank188590 +188591 POINT(38.488222355133985 -121.5157037543882) bank188591 +188592 POINT(37.8809204102092 -121.79029621491067) bank188592 +188593 POINT(37.349526970764515 -121.46745779930956) bank188593 +188594 POINT(37.19639770691608 -122.3061078675393) bank188594 +188595 POINT(37.73540562448683 -123.08210845230995) bank188595 +188596 POINT(38.25933559972818 -121.87119339992097) bank188596 +188597 POINT(37.130906846732785 -122.42762684446316) bank188597 +188598 POINT(37.57676869675914 -122.74656113348695) bank188598 +188599 POINT(37.696141018974984 -122.29590856450714) bank188599 +188600 POINT(36.91824976267485 -122.2171969266338) bank188600 +188601 POINT(38.05302076190104 -122.54803517850912) bank188601 +188602 POINT(36.875082221060175 -121.92754033241718) bank188602 +188603 POINT(38.46372600622471 -122.81253732376945) bank188603 +188604 POINT(38.59006227536051 -122.27490556710916) bank188604 +188605 POINT(36.806155642546024 -121.86134026258324) bank188605 +188606 POINT(38.10473789471124 -122.38251275492469) bank188606 +188607 POINT(38.48906143649623 -122.23028451380996) bank188607 +188608 POINT(38.61273915170472 -121.94220406104402) bank188608 +188609 POINT(37.828210848566734 -123.39986945308671) bank188609 +188610 POINT(38.1503796764346 -122.96031580164369) bank188610 +188611 POINT(38.517746269304375 -122.73106167565643) bank188611 +188612 POINT(37.51196367040893 -123.33616599536909) bank188612 +188613 POINT(38.62174090168213 -123.39660604009084) bank188613 +188614 POINT(38.63474619243406 -122.2782504156849) bank188614 +188615 POINT(38.716221952668036 -121.97670421319422) bank188615 +188616 POINT(38.34533287019002 -122.57760957724578) bank188616 +188617 POINT(38.27086497642739 -121.57331358696479) bank188617 +188618 POINT(38.152100562933576 -122.67806953716037) bank188618 +188619 POINT(38.452840280952095 -123.28759160110337) bank188619 +188620 POINT(37.358658014164945 -122.81759619313985) bank188620 +188621 POINT(37.0345317969618 -122.84793275884638) bank188621 +188622 POINT(38.322775115002976 -122.67843432847118) bank188622 +188623 POINT(38.39331049839873 -122.78651563959396) bank188623 +188624 POINT(37.95197315591828 -122.42722613394078) bank188624 +188625 POINT(37.382184901016075 -121.99218347862846) bank188625 +188626 POINT(38.17060554164706 -123.24832235141022) bank188626 +188627 POINT(37.35393326317828 -122.5545342430435) bank188627 +188628 POINT(36.9833758509245 -122.86671622186299) bank188628 +188629 POINT(36.99616629785681 -123.37791734611254) bank188629 +188630 POINT(38.47396423583029 -122.84467433886047) bank188630 +188631 POINT(37.877816484816194 -121.52343934933761) bank188631 +188632 POINT(38.34113419580244 -121.46648648765412) bank188632 +188633 POINT(37.716822172476114 -122.30386205437524) bank188633 +188634 POINT(37.73284049667896 -122.32149707923048) bank188634 +188635 POINT(36.875687998903146 -121.89746444756878) bank188635 +188636 POINT(37.776223856367245 -122.5046893152364) bank188636 +188637 POINT(38.55654113788778 -122.80370393161981) bank188637 +188638 POINT(37.112246979801654 -121.78972910494268) bank188638 +188639 POINT(38.43922557631698 -122.47982292147229) bank188639 +188640 POINT(38.00688797409352 -121.88284855860024) bank188640 +188641 POINT(37.578660122071 -123.21693959068757) bank188641 +188642 POINT(36.795281666117624 -122.43272169687823) bank188642 +188643 POINT(38.49003604918958 -122.33435109601878) bank188643 +188644 POINT(36.85999653986576 -122.81752665685806) bank188644 +188645 POINT(38.63104471153065 -122.14598545849259) bank188645 +188646 POINT(37.39412117968611 -121.45256674848838) bank188646 +188647 POINT(38.27974884438978 -123.26021996570871) bank188647 +188648 POINT(38.28258518143983 -123.27220573957143) bank188648 +188649 POINT(38.01408925510137 -122.96387791675984) bank188649 +188650 POINT(38.132228946438 -122.34096664812886) bank188650 +188651 POINT(37.33285393277234 -122.18036264319174) bank188651 +188652 POINT(38.12905018177765 -121.65500421605594) bank188652 +188653 POINT(38.625077505820634 -122.81556379025483) bank188653 +188654 POINT(38.43400637826962 -121.44988386270178) bank188654 +188655 POINT(37.2087970143173 -122.16624981032543) bank188655 +188656 POINT(36.95602179170181 -122.65717681259252) bank188656 +188657 POINT(37.99375010901215 -122.85495794203769) bank188657 +188658 POINT(38.12541572025221 -122.74719398844042) bank188658 +188659 POINT(37.014915687057986 -122.6985155089652) bank188659 +188660 POINT(37.680598367135815 -121.85911994418247) bank188660 +188661 POINT(38.124124316073825 -121.78425522525282) bank188661 +188662 POINT(37.962550686373135 -123.40268337996173) bank188662 +188663 POINT(38.51555027771373 -122.12079764123375) bank188663 +188664 POINT(37.32048330848091 -121.57689418511376) bank188664 +188665 POINT(36.897329909538456 -122.80197379978527) bank188665 +188666 POINT(37.198213629563526 -123.32947558978532) bank188666 +188667 POINT(38.169824672443596 -122.92578853865464) bank188667 +188668 POINT(37.4015261354918 -122.41190162091223) bank188668 +188669 POINT(37.51441910301074 -123.13269975895162) bank188669 +188670 POINT(36.78474194124219 -123.19555240211456) bank188670 +188671 POINT(38.63607301376235 -122.44177212881723) bank188671 +188672 POINT(37.42806817043935 -122.68454379063897) bank188672 +188673 POINT(36.89650027607364 -122.40762921768335) bank188673 +188674 POINT(36.940121980893494 -121.73040298320053) bank188674 +188675 POINT(36.93696136628341 -122.8460605487452) bank188675 +188676 POINT(37.95534741896934 -123.24376898115257) bank188676 +188677 POINT(38.68058457644874 -122.70825299278043) bank188677 +188678 POINT(37.183329643921525 -123.2370843466076) bank188678 +188679 POINT(38.65776721199953 -123.01203385207079) bank188679 +188680 POINT(37.31332006511024 -122.04470192372825) bank188680 +188681 POINT(37.308850727560596 -123.10893753684687) bank188681 +188682 POINT(36.911324985194305 -121.69208211248083) bank188682 +188683 POINT(37.988548062913594 -121.4211100287561) bank188683 +188684 POINT(38.58653955829416 -122.29904832366311) bank188684 +188685 POINT(38.02934002255862 -122.92914546599599) bank188685 +188686 POINT(38.21523441962115 -122.1853684691802) bank188686 +188687 POINT(38.40939526588376 -123.20061105715844) bank188687 +188688 POINT(38.026296284550696 -121.52372380551668) bank188688 +188689 POINT(37.633464481691675 -121.84766548927452) bank188689 +188690 POINT(38.116867954152276 -123.04434793588035) bank188690 +188691 POINT(36.80936511430883 -122.86677415515427) bank188691 +188692 POINT(37.81684803136172 -121.71285417358278) bank188692 +188693 POINT(37.31960846881926 -121.93512838397331) bank188693 +188694 POINT(37.659398678981866 -121.99836292844135) bank188694 +188695 POINT(37.571170940755145 -122.39887998893104) bank188695 +188696 POINT(36.88997900107371 -122.27283325084447) bank188696 +188697 POINT(37.364836022058974 -121.58003621468555) bank188697 +188698 POINT(37.826884120285186 -122.10653627462928) bank188698 +188699 POINT(37.75138934036897 -122.58250650947551) bank188699 +188700 POINT(38.06029703032515 -123.20868682273273) bank188700 +188701 POINT(37.11720655532007 -121.58382026325866) bank188701 +188702 POINT(37.53877805100961 -122.6079404888226) bank188702 +188703 POINT(36.960344206380135 -121.90062776934316) bank188703 +188704 POINT(36.91211379572536 -123.38610945353527) bank188704 +188705 POINT(37.063008679304374 -122.48013462936497) bank188705 +188706 POINT(36.89682370484461 -122.56078173045897) bank188706 +188707 POINT(37.867708772399524 -123.25364466422168) bank188707 +188708 POINT(37.3776411130197 -121.70972591711521) bank188708 +188709 POINT(38.214380242136755 -123.12693191485504) bank188709 +188710 POINT(36.77569284393086 -121.43889258714248) bank188710 +188711 POINT(38.54977271207743 -122.80409226619328) bank188711 +188712 POINT(37.72718183443615 -123.16905674337549) bank188712 +188713 POINT(38.20211156733321 -122.21856193670361) bank188713 +188714 POINT(37.268338110007775 -122.17269870048214) bank188714 +188715 POINT(37.21894328037933 -122.18818942225603) bank188715 +188716 POINT(37.58336364357097 -122.76096515573683) bank188716 +188717 POINT(38.708179234695905 -121.55144406891232) bank188717 +188718 POINT(36.78212306805864 -122.20336147453071) bank188718 +188719 POINT(37.42126536922679 -122.74700119762551) bank188719 +188720 POINT(36.78668762163342 -121.93398131924612) bank188720 +188721 POINT(38.13989410977876 -121.80219520852981) bank188721 +188722 POINT(37.20656101878402 -122.60855392232216) bank188722 +188723 POINT(38.303246201708134 -122.87916902012873) bank188723 +188724 POINT(37.81907333189011 -122.84015474602622) bank188724 +188725 POINT(37.58285862882831 -122.67615041301686) bank188725 +188726 POINT(37.83024458252983 -123.047813992246) bank188726 +188727 POINT(37.1098416319589 -122.1545245651668) bank188727 +188728 POINT(37.141405271937465 -122.81140383408078) bank188728 +188729 POINT(37.160845307513966 -122.80748012785631) bank188729 +188730 POINT(37.824163149749516 -123.00157795958543) bank188730 +188731 POINT(37.96078479161436 -123.18235169310749) bank188731 +188732 POINT(38.07308341874038 -121.46341612617756) bank188732 +188733 POINT(37.879931417273106 -122.69264221301678) bank188733 +188734 POINT(38.20939577759106 -122.76496045817215) bank188734 +188735 POINT(36.8850399720577 -121.80150663432309) bank188735 +188736 POINT(37.61604490105788 -122.8463374325789) bank188736 +188737 POINT(37.63009865050869 -122.08489410631374) bank188737 +188738 POINT(37.474653093737246 -121.50918560724232) bank188738 +188739 POINT(37.101797046749475 -122.62419425457442) bank188739 +188740 POINT(38.31563742532712 -121.81756995653109) bank188740 +188741 POINT(38.23451238017242 -122.79268298023007) bank188741 +188742 POINT(37.045998495653684 -123.29713586902552) bank188742 +188743 POINT(37.680643409789866 -122.02471939464232) bank188743 +188744 POINT(38.64824151981465 -121.46676247060138) bank188744 +188745 POINT(36.95944674492185 -122.08480179732324) bank188745 +188746 POINT(37.43771656730896 -121.86319215673322) bank188746 +188747 POINT(37.051256726817726 -122.21838334944816) bank188747 +188748 POINT(37.73014316507975 -123.06100574620265) bank188748 +188749 POINT(38.24414334059643 -121.76740819380957) bank188749 +188750 POINT(37.396827112593826 -122.56325113137203) bank188750 +188751 POINT(37.40890874290717 -123.11551561921449) bank188751 +188752 POINT(37.23505946025614 -122.85742690523902) bank188752 +188753 POINT(37.71328087941549 -121.91281671548349) bank188753 +188754 POINT(37.7148271364088 -122.10440123473289) bank188754 +188755 POINT(37.485183937469664 -122.06563338688285) bank188755 +188756 POINT(37.602424432103795 -121.87457350394759) bank188756 +188757 POINT(37.23130746554004 -122.24832742915446) bank188757 +188758 POINT(38.00685848176462 -123.01687480906621) bank188758 +188759 POINT(37.09404835423117 -122.9020932449709) bank188759 +188760 POINT(38.62319588557526 -122.07936210173418) bank188760 +188761 POINT(36.80620127921258 -123.14539581249954) bank188761 +188762 POINT(37.83754449195992 -123.07981365452176) bank188762 +188763 POINT(38.18501919035939 -123.18149041463882) bank188763 +188764 POINT(36.7944037284107 -122.15978319707959) bank188764 +188765 POINT(37.31756004267046 -123.03716187811584) bank188765 +188766 POINT(37.96609018320553 -122.84143532759701) bank188766 +188767 POINT(38.460855121924105 -121.92474860652112) bank188767 +188768 POINT(37.05401945299995 -122.0242807340511) bank188768 +188769 POINT(38.321941802918055 -123.03975957046214) bank188769 +188770 POINT(37.85420340865084 -122.44426069808141) bank188770 +188771 POINT(37.02897269670165 -122.7504985832043) bank188771 +188772 POINT(37.71980862061202 -122.1605926508511) bank188772 +188773 POINT(38.23036080229318 -122.33179315229768) bank188773 +188774 POINT(36.80413756364078 -122.43679198565744) bank188774 +188775 POINT(37.73456498767243 -122.2965249762112) bank188775 +188776 POINT(36.97017097054782 -123.09992607490977) bank188776 +188777 POINT(38.26248324102575 -121.46095448205284) bank188777 +188778 POINT(36.88267909601209 -121.51376701660196) bank188778 +188779 POINT(38.14318662380347 -121.72776721818084) bank188779 +188780 POINT(37.29569616988598 -122.03610339989626) bank188780 +188781 POINT(38.00329722147388 -121.89822775712672) bank188781 +188782 POINT(37.517611567429775 -121.5835022711376) bank188782 +188783 POINT(38.38481833070645 -122.8123658998988) bank188783 +188784 POINT(37.98327042972136 -122.58122890979368) bank188784 +188785 POINT(38.61917578105587 -122.42556417218393) bank188785 +188786 POINT(37.12360997951107 -121.69362561184471) bank188786 +188787 POINT(38.00852075789321 -121.762686125757) bank188787 +188788 POINT(38.59150949002315 -123.22171743668649) bank188788 +188789 POINT(38.36303923383972 -123.21938536188924) bank188789 +188790 POINT(37.41846735843637 -122.55664307494477) bank188790 +188791 POINT(38.403576168045284 -122.42660687495365) bank188791 +188792 POINT(36.87685513834045 -121.70744166264306) bank188792 +188793 POINT(37.4616888774948 -121.81696246068675) bank188793 +188794 POINT(37.57080514279274 -122.70446422097952) bank188794 +188795 POINT(37.13243149320051 -123.29018722827291) bank188795 +188796 POINT(37.91731130516834 -123.08988551142563) bank188796 +188797 POINT(36.97093609954255 -123.20902933614201) bank188797 +188798 POINT(37.57908615617074 -122.72371882466203) bank188798 +188799 POINT(38.58335539060751 -122.67066490953965) bank188799 +188800 POINT(37.41607405898133 -123.34596682061488) bank188800 +188801 POINT(38.37103304972185 -123.32235232992335) bank188801 +188802 POINT(37.14572886704982 -122.26633553306444) bank188802 +188803 POINT(37.405967393417804 -121.77774422636757) bank188803 +188804 POINT(36.886140310933364 -122.18798580111006) bank188804 +188805 POINT(37.279102743055056 -123.28039543265727) bank188805 +188806 POINT(37.953969625645875 -122.21749646874999) bank188806 +188807 POINT(37.37407755599131 -122.26811101017424) bank188807 +188808 POINT(37.187036604883765 -122.67630422531171) bank188808 +188809 POINT(37.137076322231124 -122.49429417795656) bank188809 +188810 POINT(37.72725553820371 -123.35732361062074) bank188810 +188811 POINT(37.16160849211463 -121.5114065713656) bank188811 +188812 POINT(37.87358660043027 -122.44140709844345) bank188812 +188813 POINT(38.519153621207344 -121.60144114933745) bank188813 +188814 POINT(37.19714739065597 -121.5678867484925) bank188814 +188815 POINT(37.08404319133555 -123.01787563251857) bank188815 +188816 POINT(37.806457619685894 -123.35263424757349) bank188816 +188817 POINT(37.701953926562744 -122.35632974160055) bank188817 +188818 POINT(37.854956313036226 -121.4222925108725) bank188818 +188819 POINT(36.92233336315762 -122.34918744877616) bank188819 +188820 POINT(36.9655150133073 -121.85496727118253) bank188820 +188821 POINT(38.30052159111456 -122.86175492693613) bank188821 +188822 POINT(38.21306184175591 -121.55699058057655) bank188822 +188823 POINT(37.04089531959146 -121.543919576198) bank188823 +188824 POINT(37.99085732570533 -122.45577105975218) bank188824 +188825 POINT(37.62941938618643 -123.2769000930957) bank188825 +188826 POINT(37.388918203617735 -121.62914179323556) bank188826 +188827 POINT(38.68072159669852 -123.03735698627143) bank188827 +188828 POINT(38.222199768051645 -121.47833765217129) bank188828 +188829 POINT(37.2007911588858 -121.87737048203535) bank188829 +188830 POINT(38.667117276042234 -121.9029639543512) bank188830 +188831 POINT(38.542514650704184 -121.4563849846664) bank188831 +188832 POINT(38.38575512703633 -122.58991309936415) bank188832 +188833 POINT(38.05155497510576 -123.16958323150199) bank188833 +188834 POINT(38.21100872242653 -122.11049012213641) bank188834 +188835 POINT(38.25544829000806 -122.42306790206504) bank188835 +188836 POINT(37.133037521502516 -122.77857847774581) bank188836 +188837 POINT(37.59409529188996 -122.54859125952073) bank188837 +188838 POINT(38.651840534007604 -121.50856646211132) bank188838 +188839 POINT(37.327753769741115 -122.94649766932037) bank188839 +188840 POINT(38.11274393225657 -122.56178000940037) bank188840 +188841 POINT(37.601401302522625 -122.8267454765233) bank188841 +188842 POINT(37.4848831264791 -122.18266220545365) bank188842 +188843 POINT(37.3703720112395 -123.25106084345623) bank188843 +188844 POINT(36.83653636560449 -123.19843690378173) bank188844 +188845 POINT(38.65355973400905 -121.72705016010383) bank188845 +188846 POINT(37.54219889014719 -122.35227136043098) bank188846 +188847 POINT(36.829255722861326 -123.16431399645471) bank188847 +188848 POINT(37.72408202758378 -122.4897971493952) bank188848 +188849 POINT(38.77453703473844 -123.25276904864087) bank188849 +188850 POINT(37.53040975986597 -122.44116076962646) bank188850 +188851 POINT(36.85746967267809 -122.99741475498254) bank188851 +188852 POINT(37.387007083301654 -121.6977845351892) bank188852 +188853 POINT(38.3526700773488 -121.86259203809263) bank188853 +188854 POINT(38.60881913780889 -121.91970086354395) bank188854 +188855 POINT(37.60674664751046 -123.18349905006646) bank188855 +188856 POINT(37.680075926539665 -122.73500917647863) bank188856 +188857 POINT(37.1079846767359 -121.82310393535745) bank188857 +188858 POINT(37.73231218726972 -122.89894730379484) bank188858 +188859 POINT(37.820397404890066 -122.4713550711035) bank188859 +188860 POINT(36.94486897163124 -122.16784178590478) bank188860 +188861 POINT(37.788137262970885 -122.87947036736897) bank188861 +188862 POINT(37.69229635415384 -123.33741434626145) bank188862 +188863 POINT(38.485144345939624 -122.21421287444669) bank188863 +188864 POINT(38.29014366856673 -122.40120323071689) bank188864 +188865 POINT(37.72068708043645 -121.90140787737364) bank188865 +188866 POINT(38.515049282802295 -121.67988449710634) bank188866 +188867 POINT(37.899517264589896 -123.1304219173332) bank188867 +188868 POINT(37.435494399741515 -122.46892156622789) bank188868 +188869 POINT(38.66889474801165 -122.67516657431165) bank188869 +188870 POINT(37.60153079025072 -121.82662882791745) bank188870 +188871 POINT(37.564444612698836 -122.99507594432832) bank188871 +188872 POINT(37.386914268632225 -123.01700083477245) bank188872 +188873 POINT(37.0779426616538 -122.53448410761854) bank188873 +188874 POINT(38.45467601347959 -123.11000629575823) bank188874 +188875 POINT(37.25290868238698 -122.3498806362241) bank188875 +188876 POINT(37.57118310716736 -122.22471219531681) bank188876 +188877 POINT(37.599430217794556 -122.27278752888851) bank188877 +188878 POINT(36.788954247028954 -122.55530881565863) bank188878 +188879 POINT(38.45953514311988 -121.84931686603868) bank188879 +188880 POINT(37.503139925584726 -123.24446683405004) bank188880 +188881 POINT(37.52312867542515 -121.60346164813801) bank188881 +188882 POINT(38.68359040618546 -122.24711813929923) bank188882 +188883 POINT(37.504490192480915 -121.8623413315497) bank188883 +188884 POINT(38.56101163862174 -122.40873239592926) bank188884 +188885 POINT(38.47782291934701 -121.46408948911821) bank188885 +188886 POINT(37.47735309553036 -122.67943756049291) bank188886 +188887 POINT(37.756067782052924 -121.58871092930198) bank188887 +188888 POINT(38.03436091466598 -123.13575602930459) bank188888 +188889 POINT(38.246495632971644 -123.24431708294946) bank188889 +188890 POINT(38.30458202765365 -121.89671317719719) bank188890 +188891 POINT(37.91942504179757 -123.07165741262094) bank188891 +188892 POINT(38.52427142218807 -121.43425870672961) bank188892 +188893 POINT(38.61306586528721 -123.25781615505238) bank188893 +188894 POINT(37.86581511630561 -123.09801148053522) bank188894 +188895 POINT(37.56295688043725 -121.4572294822205) bank188895 +188896 POINT(37.7289961140155 -122.87879292781574) bank188896 +188897 POINT(37.0114280872198 -122.71009015529671) bank188897 +188898 POINT(38.154732697713996 -121.74505550176306) bank188898 +188899 POINT(38.04410586249077 -123.18568736582762) bank188899 +188900 POINT(38.040221915958526 -121.48884246886621) bank188900 +188901 POINT(38.236617391128604 -122.70800700994162) bank188901 +188902 POINT(36.94251476793888 -121.70606959989392) bank188902 +188903 POINT(38.7284562596319 -122.28250530192109) bank188903 +188904 POINT(38.27306661360366 -123.24381413690804) bank188904 +188905 POINT(37.46948080174847 -121.75012488263403) bank188905 +188906 POINT(38.23687460562944 -123.1607775052532) bank188906 +188907 POINT(38.22333016363882 -123.11867643657853) bank188907 +188908 POINT(38.58318330960437 -122.31801909015314) bank188908 +188909 POINT(38.20315812088111 -121.98465531295383) bank188909 +188910 POINT(38.73618163407116 -122.08831127526611) bank188910 +188911 POINT(37.07228149829995 -122.26596683493375) bank188911 +188912 POINT(37.67285679671273 -122.99823974883739) bank188912 +188913 POINT(38.009436938936666 -122.378775631726) bank188913 +188914 POINT(38.24535015421929 -122.51161211267606) bank188914 +188915 POINT(38.066534399188576 -122.3154896142172) bank188915 +188916 POINT(38.14931208963112 -121.58664735156498) bank188916 +188917 POINT(37.07593559559215 -121.8911954023052) bank188917 +188918 POINT(36.805581748542004 -121.97536529918135) bank188918 +188919 POINT(37.217107955059035 -122.45048436592755) bank188919 +188920 POINT(37.21111062533673 -122.14063597987683) bank188920 +188921 POINT(37.01641559567541 -122.24953331492006) bank188921 +188922 POINT(38.69254764747434 -122.84825018341832) bank188922 +188923 POINT(37.303252980035964 -121.89438569239228) bank188923 +188924 POINT(38.58778878842194 -121.5264294348781) bank188924 +188925 POINT(37.58755704760421 -121.43906548027833) bank188925 +188926 POINT(38.28815139140869 -122.61741733514549) bank188926 +188927 POINT(38.07240642126681 -121.76868368221142) bank188927 +188928 POINT(37.037538841574765 -122.57396417801642) bank188928 +188929 POINT(38.69358982651639 -123.19141876363553) bank188929 +188930 POINT(37.80376524601284 -122.1910934619224) bank188930 +188931 POINT(37.73769331210248 -123.21841571037584) bank188931 +188932 POINT(37.34046389872957 -121.72937296864627) bank188932 +188933 POINT(37.57307724003687 -122.36204163014587) bank188933 +188934 POINT(37.5708249314565 -123.15236575081393) bank188934 +188935 POINT(37.01787207808308 -123.41027945174514) bank188935 +188936 POINT(38.52168415581995 -122.30908765995528) bank188936 +188937 POINT(36.930168672375416 -121.73093316049332) bank188937 +188938 POINT(38.5938456785363 -121.50559068569264) bank188938 +188939 POINT(36.90295987625805 -122.25389182168706) bank188939 +188940 POINT(38.22494257390479 -122.03485698574777) bank188940 +188941 POINT(37.85397532677802 -122.06660377718346) bank188941 +188942 POINT(36.955790653039344 -122.45292337601813) bank188942 +188943 POINT(37.58302544331924 -123.14936955854651) bank188943 +188944 POINT(37.55639369772216 -123.14035791786631) bank188944 +188945 POINT(37.76041405336494 -122.27890695321229) bank188945 +188946 POINT(37.297175993506734 -123.3425528589976) bank188946 +188947 POINT(38.24849460414674 -122.86670009292915) bank188947 +188948 POINT(38.00760495018413 -122.75000445996344) bank188948 +188949 POINT(38.77318875816795 -122.99500262585288) bank188949 +188950 POINT(37.932525971559755 -121.51096189377108) bank188950 +188951 POINT(37.31570940503286 -121.87559560616715) bank188951 +188952 POINT(37.38175438297651 -123.40109683347112) bank188952 +188953 POINT(36.92421767492013 -121.72126239980797) bank188953 +188954 POINT(38.77462580257636 -121.54280330205349) bank188954 +188955 POINT(37.66065720466708 -122.26140794253413) bank188955 +188956 POINT(37.738960713878164 -123.07620234806143) bank188956 +188957 POINT(37.48322706806122 -121.82382114448076) bank188957 +188958 POINT(38.25429703585674 -122.8506903302566) bank188958 +188959 POINT(37.876050216966455 -122.68004648665914) bank188959 +188960 POINT(38.670765942825014 -121.77608395032242) bank188960 +188961 POINT(37.77306964559933 -121.67802113449044) bank188961 +188962 POINT(37.89436942820796 -121.48439904985158) bank188962 +188963 POINT(37.77575326596246 -122.5453997709285) bank188963 +188964 POINT(38.508730950755435 -121.57239561411184) bank188964 +188965 POINT(37.75690062333734 -121.87692529179773) bank188965 +188966 POINT(37.57831600766404 -123.13243219796772) bank188966 +188967 POINT(36.777603704960384 -123.1734322505255) bank188967 +188968 POINT(37.223466850741175 -122.87544034209667) bank188968 +188969 POINT(37.94604560412729 -122.83368684429652) bank188969 +188970 POINT(36.997829417227926 -123.33039906433088) bank188970 +188971 POINT(37.470769039498315 -122.8080848333498) bank188971 +188972 POINT(37.53453551061259 -121.57973597056103) bank188972 +188973 POINT(37.170652334808224 -123.11738546732921) bank188973 +188974 POINT(36.841712217400264 -123.30896857997821) bank188974 +188975 POINT(38.472164494179545 -121.56014021126099) bank188975 +188976 POINT(37.124985737366536 -123.09711059672415) bank188976 +188977 POINT(37.06784073894156 -121.75136776768194) bank188977 +188978 POINT(37.53464557505224 -123.14559185137985) bank188978 +188979 POINT(37.43871922305676 -121.68618534848392) bank188979 +188980 POINT(36.99568668199497 -122.72610186156777) bank188980 +188981 POINT(37.251419339664494 -123.09642779495933) bank188981 +188982 POINT(37.30116242616014 -122.28570026897954) bank188982 +188983 POINT(38.5900707337064 -122.17780155188571) bank188983 +188984 POINT(38.2949060849637 -121.84456779618768) bank188984 +188985 POINT(36.85227401597972 -122.2945242754639) bank188985 +188986 POINT(38.314426795476265 -123.30225489366045) bank188986 +188987 POINT(38.594139012166394 -122.26083468214298) bank188987 +188988 POINT(38.447408578732414 -123.11060866086652) bank188988 +188989 POINT(37.84815741246255 -122.75928764719643) bank188989 +188990 POINT(38.57940489696711 -123.06832266722456) bank188990 +188991 POINT(38.09265827877432 -121.87539106667757) bank188991 +188992 POINT(37.66801540039167 -122.82196837594375) bank188992 +188993 POINT(36.92908251931029 -123.32762065860588) bank188993 +188994 POINT(36.88477235292133 -123.27780602995925) bank188994 +188995 POINT(38.451254945151156 -123.38918830943099) bank188995 +188996 POINT(36.97949791268092 -121.42769871501834) bank188996 +188997 POINT(37.42427411753375 -122.38908321027027) bank188997 +188998 POINT(38.18956224200045 -123.33953419268285) bank188998 +188999 POINT(37.529604545685785 -123.02681279542988) bank188999 +189000 POINT(37.151722359577676 -123.14893949684661) bank189000 +189001 POINT(37.270515545157025 -122.65555884789266) bank189001 +189002 POINT(37.38528434384024 -122.4151308762548) bank189002 +189003 POINT(36.91369442777546 -122.77974266579773) bank189003 +189004 POINT(37.9384427112987 -121.45016748405767) bank189004 +189005 POINT(38.14175463261802 -122.23870750484976) bank189005 +189006 POINT(38.24402993229734 -122.76477093522996) bank189006 +189007 POINT(38.38007069330266 -121.67481122708767) bank189007 +189008 POINT(38.25164876248204 -121.78669707205779) bank189008 +189009 POINT(37.73368638604483 -121.77087260043758) bank189009 +189010 POINT(37.10378562094216 -121.91430965155786) bank189010 +189011 POINT(38.52503001418237 -122.33963742067631) bank189011 +189012 POINT(37.59956553474788 -121.77363611302304) bank189012 +189013 POINT(38.223817905782205 -123.26569451632123) bank189013 +189014 POINT(38.53988074840891 -122.02162148665803) bank189014 +189015 POINT(37.07363521582819 -122.33017137146491) bank189015 +189016 POINT(38.58031456329115 -122.62239036952124) bank189016 +189017 POINT(36.85214928070632 -122.8466592026582) bank189017 +189018 POINT(38.527996269991974 -122.25769083802722) bank189018 +189019 POINT(37.64962504014205 -121.41994847548914) bank189019 +189020 POINT(37.961319557573276 -122.45561922495715) bank189020 +189021 POINT(36.80064645882064 -121.70034080664999) bank189021 +189022 POINT(37.37559057728839 -121.49632916602259) bank189022 +189023 POINT(37.108769960709814 -123.24224124621568) bank189023 +189024 POINT(37.16601131983002 -121.4913811727499) bank189024 +189025 POINT(37.28388754174192 -122.18834473345666) bank189025 +189026 POINT(36.8751246113575 -121.9334983631294) bank189026 +189027 POINT(38.369653960217335 -122.5578598207264) bank189027 +189028 POINT(38.41217336704747 -121.44186890344209) bank189028 +189029 POINT(37.17958069945051 -122.51940208533526) bank189029 +189030 POINT(37.04685034806068 -122.05351310967923) bank189030 +189031 POINT(38.35702145824782 -122.82669586699377) bank189031 +189032 POINT(38.059748995016406 -123.05398000896429) bank189032 +189033 POINT(37.347619328977935 -121.7521139087603) bank189033 +189034 POINT(37.19733237663977 -122.23289261123983) bank189034 +189035 POINT(38.34356951319004 -122.47590156927876) bank189035 +189036 POINT(37.34878537761172 -122.28289550421246) bank189036 +189037 POINT(38.12674059518194 -123.22080973756859) bank189037 +189038 POINT(38.05297176360834 -122.33745215394075) bank189038 +189039 POINT(37.6720217278021 -121.99831542593886) bank189039 +189040 POINT(38.698920015135606 -122.83384011360977) bank189040 +189041 POINT(38.143961974176406 -123.16302955348483) bank189041 +189042 POINT(36.85891967495749 -121.68695548342538) bank189042 +189043 POINT(36.871748269229776 -123.13029928552176) bank189043 +189044 POINT(38.17873495129692 -122.47808137061628) bank189044 +189045 POINT(37.57019755996058 -122.56817179945168) bank189045 +189046 POINT(38.52074851450615 -122.0070906409566) bank189046 +189047 POINT(36.815769679278525 -122.13731574574359) bank189047 +189048 POINT(37.63791859512057 -121.66369969326705) bank189048 +189049 POINT(37.65544058674339 -122.28609963703876) bank189049 +189050 POINT(37.514247304241664 -122.35875124610187) bank189050 +189051 POINT(38.29312419001458 -122.93055646374292) bank189051 +189052 POINT(38.223404225443424 -121.68654755042643) bank189052 +189053 POINT(37.340243367393676 -122.96219131619073) bank189053 +189054 POINT(37.50345549419137 -122.36395752441308) bank189054 +189055 POINT(38.73063015321839 -121.45153802254976) bank189055 +189056 POINT(37.42330294365997 -122.76743248933428) bank189056 +189057 POINT(37.10691760289236 -121.91970446808229) bank189057 +189058 POINT(38.509334791628305 -121.45861990709562) bank189058 +189059 POINT(37.591763464488395 -121.69131057404448) bank189059 +189060 POINT(36.91312381368397 -122.63657703100694) bank189060 +189061 POINT(38.21191897611619 -122.06747057356743) bank189061 +189062 POINT(37.00176832551988 -122.23045300939651) bank189062 +189063 POINT(37.076819089255174 -122.58826005627773) bank189063 +189064 POINT(38.42352405071217 -122.43933261392027) bank189064 +189065 POINT(37.69770341420684 -122.05445920965981) bank189065 +189066 POINT(38.29839884034792 -122.93890118577568) bank189066 +189067 POINT(38.72944490938983 -121.98079052474831) bank189067 +189068 POINT(38.39140047274783 -123.12870888871058) bank189068 +189069 POINT(38.26437777571449 -122.61606419930791) bank189069 +189070 POINT(37.14069163764392 -122.42859879587522) bank189070 +189071 POINT(37.88678211229233 -122.02686694987327) bank189071 +189072 POINT(37.173524753932774 -122.99641306232338) bank189072 +189073 POINT(38.357463982017926 -123.1660985885854) bank189073 +189074 POINT(37.97391432272405 -122.06979612302506) bank189074 +189075 POINT(38.509433093499865 -121.86676891492084) bank189075 +189076 POINT(37.236117642075676 -122.70127090409609) bank189076 +189077 POINT(37.54931200630002 -122.97787499289763) bank189077 +189078 POINT(38.06486290130705 -122.87610578679096) bank189078 +189079 POINT(37.49093435156718 -122.8512524105998) bank189079 +189080 POINT(38.321024617600834 -122.68982775371974) bank189080 +189081 POINT(37.62557292286282 -122.52505894820267) bank189081 +189082 POINT(36.78133543971568 -123.10060732567854) bank189082 +189083 POINT(37.439314199015584 -121.59560139828409) bank189083 +189084 POINT(38.46721200631234 -123.40118858894908) bank189084 +189085 POINT(37.90305237597087 -121.42586967690613) bank189085 +189086 POINT(38.61898904509568 -122.25073882083974) bank189086 +189087 POINT(38.4410233423302 -121.9482151780045) bank189087 +189088 POINT(38.39085184168281 -123.3291716647226) bank189088 +189089 POINT(37.12387672138157 -122.91201514337017) bank189089 +189090 POINT(38.38897732841052 -122.5623515666395) bank189090 +189091 POINT(38.1980262337135 -121.92002791421113) bank189091 +189092 POINT(38.34247218135376 -122.0647342497114) bank189092 +189093 POINT(37.531099701835636 -122.78577493188605) bank189093 +189094 POINT(37.27647971784845 -121.69311876940695) bank189094 +189095 POINT(37.09324145541735 -123.29797838150972) bank189095 +189096 POINT(38.226011595446586 -122.77500954941031) bank189096 +189097 POINT(38.13703331317109 -121.73316208510974) bank189097 +189098 POINT(38.069946032216066 -122.04926601045925) bank189098 +189099 POINT(37.28663517456411 -122.81369509701905) bank189099 +189100 POINT(38.4247212262912 -121.55259878933445) bank189100 +189101 POINT(38.06595536190866 -121.86396170557208) bank189101 +189102 POINT(37.53429959537554 -122.75859334851924) bank189102 +189103 POINT(36.814075336363175 -121.65105176454041) bank189103 +189104 POINT(37.000273729019426 -121.65991020371138) bank189104 +189105 POINT(37.35464590677547 -122.43837379912392) bank189105 +189106 POINT(37.23190498322685 -122.61139588685845) bank189106 +189107 POINT(36.93623863321711 -121.72642838131462) bank189107 +189108 POINT(38.71485198103689 -122.71093125228374) bank189108 +189109 POINT(38.316816932342576 -121.76416243956776) bank189109 +189110 POINT(38.035719098276026 -122.99059198286936) bank189110 +189111 POINT(37.7075758117995 -121.47017814975266) bank189111 +189112 POINT(37.23957599880524 -121.58929593298994) bank189112 +189113 POINT(37.73251345260992 -122.26476693454447) bank189113 +189114 POINT(37.88550741335694 -122.5454928369447) bank189114 +189115 POINT(38.013039957939405 -122.72466352674637) bank189115 +189116 POINT(38.76765973887299 -123.25327897202457) bank189116 +189117 POINT(37.9106988035582 -121.59625489006729) bank189117 +189118 POINT(37.08773816433046 -122.07217254618863) bank189118 +189119 POINT(37.892779027325936 -122.59855390751412) bank189119 +189120 POINT(38.23644670652106 -122.54589661656055) bank189120 +189121 POINT(37.835890751566154 -121.67067528296872) bank189121 +189122 POINT(37.6004791697721 -123.01028501555098) bank189122 +189123 POINT(37.89519812213631 -121.77863805284422) bank189123 +189124 POINT(37.687531808254924 -123.01268937305834) bank189124 +189125 POINT(38.62296291068435 -121.90726100140763) bank189125 +189126 POINT(38.36094204577188 -122.79164466830171) bank189126 +189127 POINT(37.62455098690111 -121.97048999565644) bank189127 +189128 POINT(38.167708460583796 -121.88574946598162) bank189128 +189129 POINT(38.00572404616674 -122.6343382059166) bank189129 +189130 POINT(38.20826721752974 -121.99492351446345) bank189130 +189131 POINT(37.49399183915572 -122.48097352739855) bank189131 +189132 POINT(38.34851494674901 -121.66426905044234) bank189132 +189133 POINT(37.3378150200127 -121.73373382555388) bank189133 +189134 POINT(37.046304931494625 -123.4022957556905) bank189134 +189135 POINT(37.864293936568785 -121.44053809126507) bank189135 +189136 POINT(36.80789196376017 -121.71783102159316) bank189136 +189137 POINT(37.03594620834016 -122.7944795777676) bank189137 +189138 POINT(37.81766365515324 -123.16671008176095) bank189138 +189139 POINT(37.920919594016276 -122.33005905930754) bank189139 +189140 POINT(37.14912484813891 -123.17559109822447) bank189140 +189141 POINT(38.178724643178775 -122.29003796054177) bank189141 +189142 POINT(37.44615458007404 -122.50637341665005) bank189142 +189143 POINT(37.7571292928447 -122.87617216440087) bank189143 +189144 POINT(36.84356927252716 -122.8606426308356) bank189144 +189145 POINT(37.91846071033326 -122.40630089158573) bank189145 +189146 POINT(38.13291384432707 -121.64072572086623) bank189146 +189147 POINT(38.33304587882468 -121.71940458378567) bank189147 +189148 POINT(37.50158536614421 -122.31979075739942) bank189148 +189149 POINT(36.941326856883606 -122.111138653962) bank189149 +189150 POINT(37.92825511158726 -122.29264651095897) bank189150 +189151 POINT(37.42645609630277 -122.90254139515008) bank189151 +189152 POINT(37.06876795009995 -123.2589825048446) bank189152 +189153 POINT(38.4079569128837 -122.70754608220216) bank189153 +189154 POINT(38.39597177990335 -122.16501345877461) bank189154 +189155 POINT(38.39712590074323 -123.27895632426022) bank189155 +189156 POINT(38.015457200633456 -122.03837940035741) bank189156 +189157 POINT(37.49249761500358 -122.01698089751574) bank189157 +189158 POINT(38.015895223249146 -122.75911311614482) bank189158 +189159 POINT(37.05972547941361 -121.65197933039123) bank189159 +189160 POINT(36.8465445808221 -121.73839206575221) bank189160 +189161 POINT(37.723505523605894 -122.91877222127093) bank189161 +189162 POINT(38.34869071534208 -123.32936693180181) bank189162 +189163 POINT(36.96436020759312 -122.3040696518056) bank189163 +189164 POINT(38.117313303527915 -123.22268941283522) bank189164 +189165 POINT(38.580702360205585 -122.90530752049293) bank189165 +189166 POINT(37.61559490134548 -122.0508345692336) bank189166 +189167 POINT(38.442502304423456 -123.21638947454997) bank189167 +189168 POINT(38.575625347139386 -121.55174830717407) bank189168 +189169 POINT(37.617728625887494 -122.85010684798382) bank189169 +189170 POINT(37.778801054635714 -123.02655826035712) bank189170 +189171 POINT(36.86093928999509 -121.81684194332284) bank189171 +189172 POINT(36.811532748835056 -123.33508428962972) bank189172 +189173 POINT(38.511191794883246 -122.0342935063827) bank189173 +189174 POINT(37.70124524341682 -122.16782106136372) bank189174 +189175 POINT(37.489936315155184 -122.98502096702246) bank189175 +189176 POINT(36.8056863272491 -122.86997003368396) bank189176 +189177 POINT(37.545205780064265 -123.29000907059482) bank189177 +189178 POINT(37.77599807056434 -121.86125481941542) bank189178 +189179 POINT(37.941555984804644 -122.86354685028564) bank189179 +189180 POINT(38.03582549667019 -121.73459290386874) bank189180 +189181 POINT(37.03720506126722 -122.05638046338323) bank189181 +189182 POINT(37.04114739787975 -122.1155664615808) bank189182 +189183 POINT(38.04569772975583 -123.02669628786352) bank189183 +189184 POINT(38.18067081159357 -122.62083091122584) bank189184 +189185 POINT(38.27598922059222 -122.19674655115858) bank189185 +189186 POINT(38.24391990752697 -122.65779519156695) bank189186 +189187 POINT(37.099620766298486 -121.83028305429966) bank189187 +189188 POINT(37.313460635225255 -121.61470054762067) bank189188 +189189 POINT(38.57154974835256 -122.80394227933621) bank189189 +189190 POINT(38.517620819272246 -122.64740472166291) bank189190 +189191 POINT(38.62355125068812 -123.16362446831371) bank189191 +189192 POINT(36.953081591029246 -121.60384547903418) bank189192 +189193 POINT(38.239011440820306 -122.02699607979291) bank189193 +189194 POINT(37.423964881866304 -121.72067216068473) bank189194 +189195 POINT(38.36617164819489 -123.17764830874147) bank189195 +189196 POINT(38.176999209692845 -122.55450661810585) bank189196 +189197 POINT(38.45262275512927 -123.20393084417633) bank189197 +189198 POINT(38.701578508734656 -122.91672158498135) bank189198 +189199 POINT(37.60785265905124 -122.13510388251203) bank189199 +189200 POINT(38.02889718847397 -122.84692373917414) bank189200 +189201 POINT(38.44570085296946 -121.52859953362042) bank189201 +189202 POINT(38.71574893283944 -123.3680563840949) bank189202 +189203 POINT(38.070020191477575 -122.2785307722614) bank189203 +189204 POINT(36.87567128962924 -121.62786267937476) bank189204 +189205 POINT(37.16903923874218 -121.86198940184991) bank189205 +189206 POINT(37.579071319634224 -121.98323327735196) bank189206 +189207 POINT(38.448752581451956 -122.92818961177872) bank189207 +189208 POINT(38.3003646554612 -122.95099916631591) bank189208 +189209 POINT(37.22238749182387 -122.1226765394552) bank189209 +189210 POINT(38.32832862019015 -122.88137144862834) bank189210 +189211 POINT(37.72906378132518 -122.57721636628969) bank189211 +189212 POINT(38.0542262598096 -122.80200841708533) bank189212 +189213 POINT(37.6120908822154 -121.92365386580651) bank189213 +189214 POINT(36.91873541190423 -121.78868539498326) bank189214 +189215 POINT(37.790476269253176 -122.37508754678558) bank189215 +189216 POINT(37.284318519740815 -121.79129988254972) bank189216 +189217 POINT(38.11396666000029 -121.85466077591373) bank189217 +189218 POINT(37.2119081476957 -122.46093371768481) bank189218 +189219 POINT(38.57648988700372 -122.11382996709705) bank189219 +189220 POINT(38.727974321521316 -123.28856613905359) bank189220 +189221 POINT(37.271973278331494 -122.89430175831482) bank189221 +189222 POINT(36.8839929197082 -122.51008800371547) bank189222 +189223 POINT(37.84135095433075 -122.59497686750206) bank189223 +189224 POINT(37.99698284166514 -122.99616069179008) bank189224 +189225 POINT(38.12492159574084 -123.01045850591125) bank189225 +189226 POINT(38.12605305622896 -121.42776952803906) bank189226 +189227 POINT(37.08819642145517 -123.29446247483143) bank189227 +189228 POINT(38.50349856906292 -123.36089118704582) bank189228 +189229 POINT(37.07621076309088 -122.71240851687095) bank189229 +189230 POINT(37.40923827636351 -122.22674140915238) bank189230 +189231 POINT(37.92514014615388 -123.26188428158024) bank189231 +189232 POINT(38.70645696906279 -123.18059520781212) bank189232 +189233 POINT(38.05628062721598 -122.11509630473154) bank189233 +189234 POINT(38.048192849493844 -121.49912683615726) bank189234 +189235 POINT(38.48856186522744 -122.0915334016492) bank189235 +189236 POINT(37.25045102593163 -122.15472383182814) bank189236 +189237 POINT(38.539169477668 -122.95449279124539) bank189237 +189238 POINT(37.77873470210217 -121.90060921006302) bank189238 +189239 POINT(38.51960338205729 -121.7561801042865) bank189239 +189240 POINT(38.0188927374024 -122.25428257314526) bank189240 +189241 POINT(37.488565003064025 -121.49435992431891) bank189241 +189242 POINT(37.39951936715086 -122.7104853945968) bank189242 +189243 POINT(37.51909615102575 -123.03693207733271) bank189243 +189244 POINT(37.489015173830666 -121.78645811332889) bank189244 +189245 POINT(37.84404052922429 -122.4943973542705) bank189245 +189246 POINT(38.50544177798492 -121.63939900783852) bank189246 +189247 POINT(36.937153556055534 -122.56933912536978) bank189247 +189248 POINT(38.052952087137335 -122.73652819120463) bank189248 +189249 POINT(38.30198878279614 -122.87320344632451) bank189249 +189250 POINT(37.76669181292844 -123.10215466069631) bank189250 +189251 POINT(37.184458376484024 -121.46367408234093) bank189251 +189252 POINT(38.66980404674326 -121.9688355016175) bank189252 +189253 POINT(37.270747330278844 -123.15881615203814) bank189253 +189254 POINT(36.84822035835531 -123.03254923912856) bank189254 +189255 POINT(38.56476937338925 -122.31582355677965) bank189255 +189256 POINT(38.05727486052973 -123.25469093095386) bank189256 +189257 POINT(37.7752701319436 -122.59582875234089) bank189257 +189258 POINT(37.3527965925795 -123.17024910459345) bank189258 +189259 POINT(38.72306124900812 -122.78146567862083) bank189259 +189260 POINT(37.66274394001242 -121.85884049448694) bank189260 +189261 POINT(38.08020595985293 -122.84998492855588) bank189261 +189262 POINT(37.924866002811314 -122.85323305035921) bank189262 +189263 POINT(37.169164052462754 -121.47160811148731) bank189263 +189264 POINT(37.224270601349644 -121.94025126542448) bank189264 +189265 POINT(37.79979367336114 -123.37731101602127) bank189265 +189266 POINT(37.442793668255575 -123.39509617938721) bank189266 +189267 POINT(38.758175417579054 -121.92210029811085) bank189267 +189268 POINT(38.75569775716434 -122.21598483622985) bank189268 +189269 POINT(38.49997596362946 -121.65049787993568) bank189269 +189270 POINT(37.26581934026023 -121.55515386299481) bank189270 +189271 POINT(37.27718315105085 -122.30479663811577) bank189271 +189272 POINT(38.08097299991584 -122.47514241904142) bank189272 +189273 POINT(38.67413617714171 -122.80861091227536) bank189273 +189274 POINT(38.001746793690124 -122.84994085850172) bank189274 +189275 POINT(37.704018356140935 -122.60022864957575) bank189275 +189276 POINT(38.311253125334815 -122.45597572236582) bank189276 +189277 POINT(37.36809064279649 -123.10088494897205) bank189277 +189278 POINT(37.37421524742147 -123.3908886815216) bank189278 +189279 POINT(37.8947885999721 -122.31442906146155) bank189279 +189280 POINT(38.036463451103344 -122.26512050332568) bank189280 +189281 POINT(38.447086753397706 -121.88140207777911) bank189281 +189282 POINT(36.93405024328064 -122.98505731534719) bank189282 +189283 POINT(38.739598990218404 -121.46968673646168) bank189283 +189284 POINT(37.93210631532662 -122.58281982833313) bank189284 +189285 POINT(38.40117707169293 -123.30985887692147) bank189285 +189286 POINT(37.71643252581721 -122.31523615416845) bank189286 +189287 POINT(38.543216573731165 -121.65543403863788) bank189287 +189288 POINT(37.050372236216546 -122.60939277090142) bank189288 +189289 POINT(38.15865160609203 -121.92446613464236) bank189289 +189290 POINT(38.179229930282 -122.84686833171801) bank189290 +189291 POINT(38.25124072305043 -122.3583928108772) bank189291 +189292 POINT(37.379628910565714 -122.64200498110995) bank189292 +189293 POINT(37.928763586381486 -123.26684102186005) bank189293 +189294 POINT(38.55342050428923 -121.89974779877694) bank189294 +189295 POINT(37.868301532359105 -121.53893996985747) bank189295 +189296 POINT(38.3053122901037 -123.3701603537425) bank189296 +189297 POINT(37.22883169824194 -121.65774056704608) bank189297 +189298 POINT(38.590774107494724 -121.62169068066902) bank189298 +189299 POINT(37.15789999602614 -122.82944158999804) bank189299 +189300 POINT(37.41078130223789 -121.42464793940354) bank189300 +189301 POINT(37.84845059612219 -123.17875208711256) bank189301 +189302 POINT(38.04496343769248 -122.94038118755415) bank189302 +189303 POINT(37.00283116878987 -122.61595685840466) bank189303 +189304 POINT(37.99474491901059 -121.87937339459306) bank189304 +189305 POINT(37.864819955616085 -121.55125646515141) bank189305 +189306 POINT(37.08949849039215 -122.8580656094236) bank189306 +189307 POINT(37.945741939961124 -122.94761123137641) bank189307 +189308 POINT(37.94625364944573 -121.98200951739045) bank189308 +189309 POINT(37.45782657486822 -122.99617617150751) bank189309 +189310 POINT(38.6173820814737 -121.78613541014553) bank189310 +189311 POINT(37.35015520779508 -122.947182168332) bank189311 +189312 POINT(38.083650298945244 -123.17715490195238) bank189312 +189313 POINT(37.86964653721098 -122.71664785249236) bank189313 +189314 POINT(37.448319919685744 -122.49377289191285) bank189314 +189315 POINT(37.20483961633802 -122.22962426424095) bank189315 +189316 POINT(37.23846422527767 -123.00476258423934) bank189316 +189317 POINT(38.42652313199178 -122.60744560381254) bank189317 +189318 POINT(37.498686538819335 -121.90097474272773) bank189318 +189319 POINT(38.3205428185815 -122.22386167575019) bank189319 +189320 POINT(38.40468680898712 -123.27004676058573) bank189320 +189321 POINT(37.624458220813715 -123.17327911175404) bank189321 +189322 POINT(37.053294474990594 -122.82110980515648) bank189322 +189323 POINT(37.16708444111129 -122.67865017569555) bank189323 +189324 POINT(38.20705107105628 -122.15832451391404) bank189324 +189325 POINT(37.962016314513505 -122.2486282315762) bank189325 +189326 POINT(38.37242765472703 -122.88408654970254) bank189326 +189327 POINT(38.38621066469211 -122.49869782567158) bank189327 +189328 POINT(38.27451317640928 -122.35787782552077) bank189328 +189329 POINT(37.10353796375352 -121.65634298028519) bank189329 +189330 POINT(37.61529525646198 -123.02662832286383) bank189330 +189331 POINT(36.977939384833164 -121.57195025197663) bank189331 +189332 POINT(38.348834816512245 -122.6325232603247) bank189332 +189333 POINT(36.80294766123997 -122.63199645086729) bank189333 +189334 POINT(37.28399814825203 -122.51003444439156) bank189334 +189335 POINT(37.78534044511853 -123.12686764524321) bank189335 +189336 POINT(37.63351415133245 -123.15492759709772) bank189336 +189337 POINT(37.449596797927406 -121.611907216044) bank189337 +189338 POINT(38.25582752605074 -121.96915168249326) bank189338 +189339 POINT(38.29198091952058 -121.55926935058241) bank189339 +189340 POINT(37.92279952112774 -122.68848488438245) bank189340 +189341 POINT(37.76260030845286 -122.84437993943943) bank189341 +189342 POINT(37.38199012754876 -123.01164166496694) bank189342 +189343 POINT(38.00395435056745 -121.90666681536769) bank189343 +189344 POINT(38.74455808742425 -122.15492896320954) bank189344 +189345 POINT(36.99456132502758 -122.46926072280182) bank189345 +189346 POINT(37.20948439004049 -122.31705582998528) bank189346 +189347 POINT(37.43448168205971 -122.04492598736482) bank189347 +189348 POINT(37.07600973785884 -123.07340055858046) bank189348 +189349 POINT(37.77502454978459 -122.2594390633022) bank189349 +189350 POINT(37.514722446681816 -122.24140079009914) bank189350 +189351 POINT(37.38782327743914 -122.54405332274351) bank189351 +189352 POINT(38.10287581966071 -122.50464743841212) bank189352 +189353 POINT(38.32372192234248 -122.8211864618148) bank189353 +189354 POINT(37.24528508162725 -121.80153946937591) bank189354 +189355 POINT(37.082098700377266 -122.3659704532444) bank189355 +189356 POINT(37.46756369117591 -121.96157841036845) bank189356 +189357 POINT(37.05127111867021 -122.44528491931649) bank189357 +189358 POINT(38.04810441767723 -121.68509124038579) bank189358 +189359 POINT(37.17514262299077 -121.73109960221963) bank189359 +189360 POINT(37.23866200649861 -122.7370699760882) bank189360 +189361 POINT(37.55383552450797 -121.66657385515641) bank189361 +189362 POINT(37.010900121293645 -123.31452426179399) bank189362 +189363 POINT(38.383572750034695 -123.0201772381123) bank189363 +189364 POINT(37.649913042904835 -122.29735179179445) bank189364 +189365 POINT(36.96564797555879 -121.7235788848273) bank189365 +189366 POINT(37.8020362619188 -122.96598703515858) bank189366 +189367 POINT(36.87726338627835 -123.41843391248709) bank189367 +189368 POINT(36.93253786711685 -121.87529440027005) bank189368 +189369 POINT(37.54083828020859 -122.39578269637359) bank189369 +189370 POINT(38.35479273385171 -122.78076217742702) bank189370 +189371 POINT(38.26435604003034 -123.00974642421691) bank189371 +189372 POINT(37.44423451801833 -122.46301278549457) bank189372 +189373 POINT(38.688946676511996 -121.76661600608847) bank189373 +189374 POINT(37.78410611168359 -123.26208405101555) bank189374 +189375 POINT(37.88413342923653 -123.39336458516131) bank189375 +189376 POINT(38.697529499425166 -122.58559163854724) bank189376 +189377 POINT(37.850459867597934 -122.23604044487024) bank189377 +189378 POINT(38.04339435306262 -121.83027249196637) bank189378 +189379 POINT(38.72242555016891 -122.0762911642267) bank189379 +189380 POINT(38.064252950900595 -122.29590666589714) bank189380 +189381 POINT(37.51376436940686 -123.1205195318958) bank189381 +189382 POINT(38.3091321747157 -122.37089778173804) bank189382 +189383 POINT(37.95808507369806 -122.07440408599973) bank189383 +189384 POINT(36.795346609855066 -122.16893545576563) bank189384 +189385 POINT(37.36818941360569 -121.42669519578831) bank189385 +189386 POINT(37.006393067632125 -121.93671156164623) bank189386 +189387 POINT(38.54136439663841 -121.7059503318024) bank189387 +189388 POINT(37.65835310291095 -122.19966493419898) bank189388 +189389 POINT(38.71533843157888 -122.18166445811373) bank189389 +189390 POINT(38.52801961711465 -122.33714635096857) bank189390 +189391 POINT(38.57555065210007 -122.88093894255411) bank189391 +189392 POINT(38.28873340492682 -122.8313858378734) bank189392 +189393 POINT(37.691073621059935 -121.55177727465461) bank189393 +189394 POINT(37.95958245766459 -121.6095923079184) bank189394 +189395 POINT(37.851050487670086 -122.6355422721936) bank189395 +189396 POINT(38.515335558104006 -122.99516240086461) bank189396 +189397 POINT(37.51820330374304 -123.17761976482898) bank189397 +189398 POINT(38.64872843600896 -123.00715159143489) bank189398 +189399 POINT(38.519777861588395 -122.50480850479804) bank189399 +189400 POINT(37.545869674482425 -122.66243402374835) bank189400 +189401 POINT(38.38323820265123 -122.64667329106861) bank189401 +189402 POINT(37.26723768731675 -122.40658723537427) bank189402 +189403 POINT(38.21751731650295 -121.94611661767865) bank189403 +189404 POINT(37.955987851603226 -123.00853233036953) bank189404 +189405 POINT(37.533753089441795 -122.85556842169149) bank189405 +189406 POINT(38.58173755534305 -121.5934714833791) bank189406 +189407 POINT(37.37715578124614 -121.46293025209097) bank189407 +189408 POINT(36.973977586932044 -121.77327324746221) bank189408 +189409 POINT(37.068752769897664 -122.12372889365132) bank189409 +189410 POINT(38.05984244248413 -121.98480097524155) bank189410 +189411 POINT(38.65259955599118 -121.73996961256886) bank189411 +189412 POINT(37.5040095605695 -123.08733160865151) bank189412 +189413 POINT(37.18963952961936 -121.76609004179794) bank189413 +189414 POINT(36.93353483089111 -122.70316528647923) bank189414 +189415 POINT(37.01181577529686 -123.271007727819) bank189415 +189416 POINT(36.876480489786736 -122.5451073648326) bank189416 +189417 POINT(38.23058410157133 -122.35341531877862) bank189417 +189418 POINT(38.31921548778842 -122.98721798006885) bank189418 +189419 POINT(38.27802247147576 -123.29771062844205) bank189419 +189420 POINT(38.23632273487537 -123.11603565221257) bank189420 +189421 POINT(38.733231502741674 -123.07811765539098) bank189421 +189422 POINT(37.01583757605624 -123.02302645848509) bank189422 +189423 POINT(38.50270166339497 -122.16376042039641) bank189423 +189424 POINT(38.718592858087305 -123.03922136712539) bank189424 +189425 POINT(37.70687514740402 -122.47240845975888) bank189425 +189426 POINT(36.906110122987414 -122.33020697800848) bank189426 +189427 POINT(37.43801726096274 -121.89461617191643) bank189427 +189428 POINT(38.19142152822012 -121.71656258438344) bank189428 +189429 POINT(37.309510688803336 -122.94629193285176) bank189429 +189430 POINT(38.54880327030186 -121.81492864843537) bank189430 +189431 POINT(38.77013071411864 -121.57083361746018) bank189431 +189432 POINT(38.32191117433148 -123.10085482131447) bank189432 +189433 POINT(38.71027736575722 -123.0324124582952) bank189433 +189434 POINT(37.06606590209596 -122.74458910915632) bank189434 +189435 POINT(38.15850767761585 -123.28286523518513) bank189435 +189436 POINT(37.39019046197291 -122.64578387399472) bank189436 +189437 POINT(37.57102805924518 -121.62394749488334) bank189437 +189438 POINT(38.436061335952644 -121.99938090433682) bank189438 +189439 POINT(38.759831718583 -123.25451112082312) bank189439 +189440 POINT(37.91954228035691 -122.19811372273897) bank189440 +189441 POINT(38.74333224039606 -123.06346552794898) bank189441 +189442 POINT(37.889154787076194 -123.29963342918012) bank189442 +189443 POINT(38.56185130276369 -121.97610970906496) bank189443 +189444 POINT(37.99399881184834 -123.07967467717728) bank189444 +189445 POINT(38.056245118259675 -122.46882214809771) bank189445 +189446 POINT(37.82272483130795 -122.3334030000091) bank189446 +189447 POINT(37.19633674150137 -123.37934153342083) bank189447 +189448 POINT(37.11548554620025 -121.53648371876311) bank189448 +189449 POINT(37.60908764105426 -122.79623837081617) bank189449 +189450 POINT(37.609529119998584 -121.84873148222) bank189450 +189451 POINT(37.892128005681585 -122.9208240025389) bank189451 +189452 POINT(38.40948882039519 -122.81645198886892) bank189452 +189453 POINT(36.811437887931596 -121.72514255009126) bank189453 +189454 POINT(37.143762503876985 -121.65842236168685) bank189454 +189455 POINT(36.91883919193755 -122.82804826311559) bank189455 +189456 POINT(36.98695055662828 -121.83960489889175) bank189456 +189457 POINT(38.538913691677216 -122.21388345194745) bank189457 +189458 POINT(37.49028418575127 -122.35414697582421) bank189458 +189459 POINT(37.873498265474396 -122.11470472679322) bank189459 +189460 POINT(37.66291840412418 -121.76189770939612) bank189460 +189461 POINT(37.87524260352317 -122.10955108935472) bank189461 +189462 POINT(38.32141893549425 -121.4364516359828) bank189462 +189463 POINT(38.588600755688866 -122.68496950816424) bank189463 +189464 POINT(37.167866846978086 -122.42652977381991) bank189464 +189465 POINT(37.29583513043447 -121.82889077839208) bank189465 +189466 POINT(37.41389059491111 -122.49584724657555) bank189466 +189467 POINT(38.771603063951446 -123.00877962366923) bank189467 +189468 POINT(37.01140011188227 -122.24605381267588) bank189468 +189469 POINT(36.92742459814928 -123.27796303342832) bank189469 +189470 POINT(36.84898248562532 -122.16400399526259) bank189470 +189471 POINT(38.61625548789184 -123.00759466659329) bank189471 +189472 POINT(37.50316888578112 -121.77742576491117) bank189472 +189473 POINT(37.312327051713446 -122.19034999680629) bank189473 +189474 POINT(37.33754586371889 -122.47678895901302) bank189474 +189475 POINT(38.710028080267335 -122.77914563441817) bank189475 +189476 POINT(38.315522169594125 -121.97930454743883) bank189476 +189477 POINT(36.8844275950158 -123.14731168962076) bank189477 +189478 POINT(37.443385697329646 -121.94918891352359) bank189478 +189479 POINT(37.25490383972043 -121.86646211101016) bank189479 +189480 POINT(37.021952285923476 -123.4108744921524) bank189480 +189481 POINT(36.882330762785585 -122.75427406861607) bank189481 +189482 POINT(37.12075037666465 -122.28628494892855) bank189482 +189483 POINT(38.28868975443632 -123.32894376204817) bank189483 +189484 POINT(37.74705379875067 -122.77809216589995) bank189484 +189485 POINT(37.82032351484164 -122.1027129614194) bank189485 +189486 POINT(37.141812882803855 -123.30883247543596) bank189486 +189487 POINT(38.35991557264143 -121.93835873283975) bank189487 +189488 POINT(37.86546779315315 -122.42900566518162) bank189488 +189489 POINT(37.987802837530666 -122.85476519184944) bank189489 +189490 POINT(38.575186395807016 -121.67303018632853) bank189490 +189491 POINT(38.48879226827796 -122.87407462330665) bank189491 +189492 POINT(37.745687398124595 -122.88911157538433) bank189492 +189493 POINT(38.77092453226646 -122.76322978320957) bank189493 +189494 POINT(37.335862139006224 -121.95231178487633) bank189494 +189495 POINT(37.600422274254576 -123.3727730119982) bank189495 +189496 POINT(38.03475674696769 -121.87080292873628) bank189496 +189497 POINT(37.36891743638825 -122.35613763222388) bank189497 +189498 POINT(38.62944406473611 -122.9406203168908) bank189498 +189499 POINT(37.713928614474035 -122.75941183646933) bank189499 +189500 POINT(37.55353450652036 -121.67332407490265) bank189500 +189501 POINT(38.145645026600306 -121.95159478776978) bank189501 +189502 POINT(37.52283586478779 -121.66947892889515) bank189502 +189503 POINT(38.206930880548285 -121.81986674825976) bank189503 +189504 POINT(36.9546155308476 -122.88792067453788) bank189504 +189505 POINT(37.692763023465155 -121.47936066326943) bank189505 +189506 POINT(37.331203634019815 -121.75202847165728) bank189506 +189507 POINT(38.029537161100144 -123.16669258366231) bank189507 +189508 POINT(37.27867096217346 -122.750374588619) bank189508 +189509 POINT(37.21277036135203 -122.45460244774591) bank189509 +189510 POINT(38.54281086870221 -121.83667014746266) bank189510 +189511 POINT(38.7121448412139 -123.29073025435012) bank189511 +189512 POINT(37.34017626637219 -123.1318728834975) bank189512 +189513 POINT(38.45767183482073 -122.52617865441862) bank189513 +189514 POINT(37.57015159849935 -122.29516034492255) bank189514 +189515 POINT(38.04483044269394 -122.6895667633926) bank189515 +189516 POINT(36.829211860554224 -121.58076421580797) bank189516 +189517 POINT(36.94617815384947 -122.81949744382014) bank189517 +189518 POINT(37.46491815016382 -122.18058213815834) bank189518 +189519 POINT(38.23930776663695 -123.19292761308661) bank189519 +189520 POINT(37.9525265962894 -121.70157791824131) bank189520 +189521 POINT(38.155236024668426 -121.47631412291157) bank189521 +189522 POINT(38.272169255518136 -121.75035988674537) bank189522 +189523 POINT(37.76380837244244 -122.23951956366366) bank189523 +189524 POINT(36.905940453001534 -122.86231679050027) bank189524 +189525 POINT(38.25405380574132 -121.532504698423) bank189525 +189526 POINT(38.39732958533124 -122.91568681199118) bank189526 +189527 POINT(38.44575059166945 -121.56329415600574) bank189527 +189528 POINT(36.91432377246624 -122.08625294253609) bank189528 +189529 POINT(37.396101404780886 -123.33655606270561) bank189529 +189530 POINT(38.40968796807068 -122.8979191208277) bank189530 +189531 POINT(37.315449399336856 -122.60550251837665) bank189531 +189532 POINT(36.791378086678336 -122.28875918930426) bank189532 +189533 POINT(38.569973612506956 -122.41561435137518) bank189533 +189534 POINT(37.31729307635633 -122.11001099458039) bank189534 +189535 POINT(37.41082714420341 -121.85742114593344) bank189535 +189536 POINT(38.761109016814856 -122.34262016942448) bank189536 +189537 POINT(38.68649062285248 -122.72721731266417) bank189537 +189538 POINT(38.36516238298976 -122.34282849110087) bank189538 +189539 POINT(38.650014729016476 -123.32398395916033) bank189539 +189540 POINT(37.38339124913248 -122.03796397642756) bank189540 +189541 POINT(38.54059061240667 -122.14741400771959) bank189541 +189542 POINT(37.52164915220117 -121.59175757313612) bank189542 +189543 POINT(38.14725662005296 -122.4187737846835) bank189543 +189544 POINT(37.754337606255774 -123.20393826049305) bank189544 +189545 POINT(37.47338572957957 -121.79509032422429) bank189545 +189546 POINT(38.51875283088581 -122.62768917292163) bank189546 +189547 POINT(37.68558901947016 -122.3825213155462) bank189547 +189548 POINT(38.55763453761632 -123.34716114275608) bank189548 +189549 POINT(37.99429208644537 -122.98510439265715) bank189549 +189550 POINT(37.65790912183498 -122.73660484387277) bank189550 +189551 POINT(37.51343562022554 -122.98012048682975) bank189551 +189552 POINT(37.52491418589458 -122.59619990910323) bank189552 +189553 POINT(37.53099344868163 -121.89864702445986) bank189553 +189554 POINT(37.41783158665623 -121.55531101596006) bank189554 +189555 POINT(37.26389582339363 -122.97622009974577) bank189555 +189556 POINT(37.36832391455666 -121.45844843177683) bank189556 +189557 POINT(38.24603504716377 -123.11399294623725) bank189557 +189558 POINT(36.9590076201136 -122.82309949652749) bank189558 +189559 POINT(37.86469351872236 -122.52551141036297) bank189559 +189560 POINT(36.87461119254243 -123.14721063292626) bank189560 +189561 POINT(38.53972252280366 -122.91877103647931) bank189561 +189562 POINT(37.730018062163786 -123.10129350447471) bank189562 +189563 POINT(38.447026555969984 -122.66002549472208) bank189563 +189564 POINT(38.02489848728072 -122.0572437974511) bank189564 +189565 POINT(38.615888901528095 -122.06091714389396) bank189565 +189566 POINT(37.81989382048439 -122.22269900368323) bank189566 +189567 POINT(37.66986448775477 -121.64994633206229) bank189567 +189568 POINT(37.39881151235385 -122.65668718053784) bank189568 +189569 POINT(37.20354389670018 -123.15809249162861) bank189569 +189570 POINT(37.049360508954024 -122.96429003609717) bank189570 +189571 POINT(37.30019911591741 -122.05864462443996) bank189571 +189572 POINT(36.775215761968276 -121.4571778123152) bank189572 +189573 POINT(37.55934044216345 -121.91621461551844) bank189573 +189574 POINT(38.0358435509131 -123.3291612565006) bank189574 +189575 POINT(38.061923356874715 -122.34753316949475) bank189575 +189576 POINT(38.117929874550015 -123.40582670142396) bank189576 +189577 POINT(38.51321571051563 -123.02182202886435) bank189577 +189578 POINT(37.48078684353267 -122.44846303966926) bank189578 +189579 POINT(37.72355538364851 -123.2299371882051) bank189579 +189580 POINT(36.87161274639711 -121.7058780850686) bank189580 +189581 POINT(38.119634902166936 -123.00202983758243) bank189581 +189582 POINT(37.67531782827571 -121.71363757184686) bank189582 +189583 POINT(37.15261860974248 -123.39799791257515) bank189583 +189584 POINT(37.35276423571609 -122.17673878016436) bank189584 +189585 POINT(38.7330357982598 -122.37242284730536) bank189585 +189586 POINT(37.7896599104939 -122.9211411229392) bank189586 +189587 POINT(37.50647623206461 -122.11384421259324) bank189587 +189588 POINT(36.88028154858982 -123.19371384355337) bank189588 +189589 POINT(38.73942860588888 -122.97682753290454) bank189589 +189590 POINT(38.0620859449209 -121.74789755255125) bank189590 +189591 POINT(38.339638213737196 -123.34636813198442) bank189591 +189592 POINT(38.5539345245851 -122.96336702783343) bank189592 +189593 POINT(38.517696315835515 -121.50921217849402) bank189593 +189594 POINT(37.7318848707481 -122.73385042830317) bank189594 +189595 POINT(37.08709603984465 -122.4099422664298) bank189595 +189596 POINT(36.89336251807245 -123.31824678297066) bank189596 +189597 POINT(37.84795026461724 -123.19526231986386) bank189597 +189598 POINT(36.91713356437326 -121.5136850384583) bank189598 +189599 POINT(37.39592157580553 -123.20830230636889) bank189599 +189600 POINT(37.618375472981185 -121.59365880200052) bank189600 +189601 POINT(38.64502265187654 -123.18777834902234) bank189601 +189602 POINT(36.96013701881096 -122.81224113388353) bank189602 +189603 POINT(37.585278952316806 -122.70495694562723) bank189603 +189604 POINT(37.36853883788256 -122.42881190833566) bank189604 +189605 POINT(38.589202906159656 -122.73010721447793) bank189605 +189606 POINT(37.62411464196847 -122.51605918244874) bank189606 +189607 POINT(37.17930696296516 -121.91900016606556) bank189607 +189608 POINT(38.689628328385666 -122.94504608050934) bank189608 +189609 POINT(38.34001494974268 -123.39054866276746) bank189609 +189610 POINT(38.461702630121 -121.57195825908474) bank189610 +189611 POINT(37.16068688072777 -122.74466529346014) bank189611 +189612 POINT(38.52464412023566 -122.36817170319733) bank189612 +189613 POINT(37.87206012958686 -123.05856484350133) bank189613 +189614 POINT(37.56074536218236 -122.65168123663483) bank189614 +189615 POINT(37.220967070454 -123.04239559196373) bank189615 +189616 POINT(38.36144237045562 -122.90254892737178) bank189616 +189617 POINT(38.63961143616578 -123.05195752616544) bank189617 +189618 POINT(36.829808332389085 -122.43205583164449) bank189618 +189619 POINT(38.39429318190945 -122.6039062578455) bank189619 +189620 POINT(38.138467585797656 -121.8294053944046) bank189620 +189621 POINT(38.04288517235698 -121.749432601418) bank189621 +189622 POINT(37.18116195076929 -122.7908966440146) bank189622 +189623 POINT(37.016417946080246 -122.57334382140299) bank189623 +189624 POINT(37.406849518271464 -122.48326792442259) bank189624 +189625 POINT(36.86966286267425 -123.25785484191942) bank189625 +189626 POINT(37.45126273137193 -122.50672628550475) bank189626 +189627 POINT(38.17023566358953 -122.91083936260695) bank189627 +189628 POINT(37.12629682611729 -123.14159899050406) bank189628 +189629 POINT(37.36429818911706 -122.32454100571695) bank189629 +189630 POINT(38.34298049143159 -123.14852322200245) bank189630 +189631 POINT(37.61216520381657 -122.93941291822044) bank189631 +189632 POINT(37.209589667892665 -122.76030202902676) bank189632 +189633 POINT(37.263982168966024 -122.1105123723148) bank189633 +189634 POINT(38.46573809007513 -122.8001839478624) bank189634 +189635 POINT(37.81522732733413 -121.81796765354296) bank189635 +189636 POINT(38.084138800954676 -122.24528353054792) bank189636 +189637 POINT(38.217241757982215 -122.1417223152559) bank189637 +189638 POINT(36.81329979531227 -122.57488487685961) bank189638 +189639 POINT(37.409293297011246 -123.02539068017568) bank189639 +189640 POINT(37.05603575863414 -121.83272747246976) bank189640 +189641 POINT(37.908544929465904 -123.24843654870624) bank189641 +189642 POINT(38.01371659628276 -121.8334258945966) bank189642 +189643 POINT(36.83787102921486 -121.9412627905813) bank189643 +189644 POINT(37.38989518813073 -122.52785445205907) bank189644 +189645 POINT(38.65289078155823 -122.17121241587213) bank189645 +189646 POINT(38.31601786740865 -123.40069022623557) bank189646 +189647 POINT(38.416789282995865 -123.32832210310592) bank189647 +189648 POINT(38.23064423951473 -121.9171828988571) bank189648 +189649 POINT(38.00469897772208 -121.80408809050712) bank189649 +189650 POINT(38.400634126811596 -122.10118664030048) bank189650 +189651 POINT(38.30707168202763 -122.78860152285908) bank189651 +189652 POINT(38.40631531519819 -121.72776515451064) bank189652 +189653 POINT(38.64793694345502 -121.84879783401237) bank189653 +189654 POINT(36.797751272134406 -122.87399327384894) bank189654 +189655 POINT(38.2410608855039 -123.12359033969281) bank189655 +189656 POINT(38.00545249471733 -122.14522833414205) bank189656 +189657 POINT(36.910885298073985 -122.39133894209489) bank189657 +189658 POINT(38.43649917482665 -122.80566125009898) bank189658 +189659 POINT(37.16788058786028 -122.59311839187424) bank189659 +189660 POINT(38.318870960014124 -121.65201057724012) bank189660 +189661 POINT(37.84833485378229 -122.52818992520166) bank189661 +189662 POINT(38.27253862265195 -122.5156855930799) bank189662 +189663 POINT(38.64304897273954 -122.99198944047185) bank189663 +189664 POINT(38.56665879105168 -122.71995398887503) bank189664 +189665 POINT(37.274810266774516 -122.76433140163697) bank189665 +189666 POINT(37.24734607553218 -122.03766027106438) bank189666 +189667 POINT(38.63215047997017 -123.04585463303505) bank189667 +189668 POINT(38.09738250303866 -122.6639302867436) bank189668 +189669 POINT(37.63728817039286 -122.61511584945902) bank189669 +189670 POINT(38.09635072594348 -122.87697239337986) bank189670 +189671 POINT(37.698624262412054 -122.7402421399395) bank189671 +189672 POINT(38.40036585150304 -121.92695132659607) bank189672 +189673 POINT(38.5254946901019 -121.71871899807066) bank189673 +189674 POINT(36.83237622183222 -121.70749232660158) bank189674 +189675 POINT(37.25891272638043 -122.60409688143288) bank189675 +189676 POINT(37.12221926001383 -121.63810593484764) bank189676 +189677 POINT(37.38332920780075 -121.59442750319971) bank189677 +189678 POINT(37.444695882147855 -122.15899199831382) bank189678 +189679 POINT(37.0315560537845 -122.3355364848196) bank189679 +189680 POINT(37.31859639530174 -121.87664116342008) bank189680 +189681 POINT(38.26730308606516 -122.08363500777413) bank189681 +189682 POINT(38.70477966317413 -121.86553130436921) bank189682 +189683 POINT(37.44419282317701 -122.75433863393698) bank189683 +189684 POINT(37.239496125636286 -122.684083338699) bank189684 +189685 POINT(37.12787709981592 -122.30842882613443) bank189685 +189686 POINT(37.32469945038334 -122.81124189713942) bank189686 +189687 POINT(37.191099063947725 -121.97396725275274) bank189687 +189688 POINT(37.061497777376 -122.14509259193318) bank189688 +189689 POINT(38.6960571734592 -122.39860505410984) bank189689 +189690 POINT(37.48431134755541 -122.46930717535862) bank189690 +189691 POINT(38.73008873827875 -122.2438243063823) bank189691 +189692 POINT(38.48529993668297 -123.21711194612158) bank189692 +189693 POINT(37.02386081518802 -122.08217187013561) bank189693 +189694 POINT(38.55855297943839 -121.49173442990346) bank189694 +189695 POINT(36.93277513471229 -122.57892182048107) bank189695 +189696 POINT(37.53945022800401 -123.40478250926947) bank189696 +189697 POINT(38.470200967197805 -123.32307634002929) bank189697 +189698 POINT(36.95101194756605 -122.59229328710263) bank189698 +189699 POINT(36.854936529960305 -122.49286718751478) bank189699 +189700 POINT(36.971234018983566 -122.13222197920278) bank189700 +189701 POINT(37.135611443485644 -121.8128388038987) bank189701 +189702 POINT(38.26357747603577 -122.13459150386807) bank189702 +189703 POINT(37.9248518765785 -121.44883281271703) bank189703 +189704 POINT(38.08304575505389 -121.68638220735009) bank189704 +189705 POINT(37.251547182699994 -122.32158212223369) bank189705 +189706 POINT(38.19068974539126 -122.62236419269202) bank189706 +189707 POINT(37.22648153902468 -121.42647545063053) bank189707 +189708 POINT(37.541433145405136 -121.85987490224092) bank189708 +189709 POINT(37.08871077756186 -122.12234128876091) bank189709 +189710 POINT(38.25662307551447 -122.27264922542489) bank189710 +189711 POINT(37.1536952339114 -122.31331097319693) bank189711 +189712 POINT(38.229052343108776 -122.87812081679964) bank189712 +189713 POINT(38.12325615079209 -122.32628808404503) bank189713 +189714 POINT(38.24945832826669 -122.05340904726509) bank189714 +189715 POINT(38.632811125323535 -123.33395952187651) bank189715 +189716 POINT(38.53072292266873 -121.79857161919476) bank189716 +189717 POINT(38.29479025970542 -123.20121099516707) bank189717 +189718 POINT(37.47856787551508 -122.62801812117667) bank189718 +189719 POINT(38.65082143797885 -123.2194335102476) bank189719 +189720 POINT(36.999263594548204 -122.80288581603516) bank189720 +189721 POINT(36.79403472885041 -122.99135895810268) bank189721 +189722 POINT(37.7654710709671 -122.6519946007904) bank189722 +189723 POINT(36.941482268376916 -122.67247373448055) bank189723 +189724 POINT(37.96752401434094 -123.1817074469716) bank189724 +189725 POINT(38.22739682150581 -121.98470597939675) bank189725 +189726 POINT(37.75136723500908 -121.93469070211927) bank189726 +189727 POINT(37.2862899117618 -121.80414265083883) bank189727 +189728 POINT(38.379578464134276 -122.79155066374867) bank189728 +189729 POINT(38.0140217093149 -121.55559140901927) bank189729 +189730 POINT(37.16243101303771 -121.96781534417885) bank189730 +189731 POINT(38.178918172122174 -122.07730832652521) bank189731 +189732 POINT(37.59095231652775 -123.10301510455186) bank189732 +189733 POINT(38.29756315557159 -122.14519288295732) bank189733 +189734 POINT(37.97191210178992 -122.23029956249344) bank189734 +189735 POINT(38.28627024178695 -122.7378180593097) bank189735 +189736 POINT(38.03414737911287 -122.2203841567004) bank189736 +189737 POINT(37.33257197027703 -122.02356286546485) bank189737 +189738 POINT(38.10439637073778 -121.87161511381368) bank189738 +189739 POINT(37.6501233192622 -123.32117263460788) bank189739 +189740 POINT(37.90051599270211 -121.97262765007798) bank189740 +189741 POINT(38.037969772587836 -122.31877903086408) bank189741 +189742 POINT(36.786981670301394 -123.05800101763558) bank189742 +189743 POINT(37.12148129170479 -121.96335964987432) bank189743 +189744 POINT(37.46968885122228 -123.12113401612265) bank189744 +189745 POINT(38.257616425029305 -121.92693195086613) bank189745 +189746 POINT(38.186753885863155 -122.00163555391877) bank189746 +189747 POINT(37.866292693288834 -121.67609747013344) bank189747 +189748 POINT(38.35676846768273 -122.95824361675834) bank189748 +189749 POINT(37.69052862058446 -123.02400934615453) bank189749 +189750 POINT(37.656434166871875 -122.86213117738994) bank189750 +189751 POINT(38.406879040076426 -122.1477375443311) bank189751 +189752 POINT(37.95383084445362 -122.73847052507108) bank189752 +189753 POINT(37.93765375049498 -123.06742872776596) bank189753 +189754 POINT(37.744575655185464 -121.47247190479463) bank189754 +189755 POINT(37.54992597560271 -122.06795767651546) bank189755 +189756 POINT(36.94515253180528 -121.77829109231236) bank189756 +189757 POINT(37.85918799423069 -123.26893629431599) bank189757 +189758 POINT(38.05286893063359 -122.95910785832412) bank189758 +189759 POINT(36.953254144122475 -123.3658960378487) bank189759 +189760 POINT(37.84820883694982 -121.72189482907773) bank189760 +189761 POINT(36.786378161561345 -121.63636396579932) bank189761 +189762 POINT(37.504267324885745 -122.705045146749) bank189762 +189763 POINT(37.18997608738134 -123.08112950133307) bank189763 +189764 POINT(36.824865241368045 -122.87587111293594) bank189764 +189765 POINT(38.51036515132288 -122.65157215258753) bank189765 +189766 POINT(38.20699301086378 -123.07613420190131) bank189766 +189767 POINT(37.21850407138132 -121.92365185936075) bank189767 +189768 POINT(37.35848256503522 -121.54041952047943) bank189768 +189769 POINT(38.02567526436642 -121.64628688600894) bank189769 +189770 POINT(37.38925439838992 -121.43035589445539) bank189770 +189771 POINT(37.04471954038066 -121.8499967867381) bank189771 +189772 POINT(38.72236519367641 -123.34339655560696) bank189772 +189773 POINT(37.50784313364793 -121.58850494484525) bank189773 +189774 POINT(38.07028087623949 -123.16524322590192) bank189774 +189775 POINT(37.07949893596157 -121.54616243150404) bank189775 +189776 POINT(37.4958243266849 -122.05251493364487) bank189776 +189777 POINT(37.603256082914626 -122.33991421357734) bank189777 +189778 POINT(38.44516212177311 -123.3398654615269) bank189778 +189779 POINT(38.664666755287946 -121.76393410697887) bank189779 +189780 POINT(37.81388232220856 -123.00100598723881) bank189780 +189781 POINT(38.32117344881428 -122.39544384491708) bank189781 +189782 POINT(37.836748567727696 -121.54579010231177) bank189782 +189783 POINT(37.636129642035804 -121.9717933716218) bank189783 +189784 POINT(38.74932346592675 -121.4779198617654) bank189784 +189785 POINT(37.210822383767116 -122.08068687870237) bank189785 +189786 POINT(38.072241343829695 -123.41839042509851) bank189786 +189787 POINT(38.15741755371736 -121.49950620894721) bank189787 +189788 POINT(37.53818195938806 -122.4782411108213) bank189788 +189789 POINT(38.30116075413188 -122.49814114330017) bank189789 +189790 POINT(38.43485047798077 -121.81830178116964) bank189790 +189791 POINT(38.32805628568794 -122.59098135486195) bank189791 +189792 POINT(38.45147532594446 -123.39892225500535) bank189792 +189793 POINT(38.6244844960097 -121.49731757581132) bank189793 +189794 POINT(36.880522115238726 -122.76056269364632) bank189794 +189795 POINT(37.389772482161014 -122.78478626332868) bank189795 +189796 POINT(38.4971424630809 -121.45389165566978) bank189796 +189797 POINT(37.52739938923202 -122.09318772486034) bank189797 +189798 POINT(37.97490350976211 -122.702370316081) bank189798 +189799 POINT(38.02549351519113 -122.56291102362775) bank189799 +189800 POINT(37.69648792751423 -122.57413816486563) bank189800 +189801 POINT(37.66738583816865 -122.29618833585981) bank189801 +189802 POINT(36.78542545940399 -123.35619956955391) bank189802 +189803 POINT(37.19903482177464 -122.42467774831114) bank189803 +189804 POINT(38.52472000725731 -122.84241563558172) bank189804 +189805 POINT(37.54597899667407 -122.05590419372999) bank189805 +189806 POINT(38.14594203545304 -122.07237050413416) bank189806 +189807 POINT(37.94836644692798 -121.83905207265569) bank189807 +189808 POINT(36.985902244565196 -121.5693628826385) bank189808 +189809 POINT(37.38709550784323 -122.4353993907443) bank189809 +189810 POINT(38.41883548383728 -123.23505461469135) bank189810 +189811 POINT(37.56104375738023 -122.08690564698891) bank189811 +189812 POINT(36.8422122372825 -122.50056067290477) bank189812 +189813 POINT(36.98551684767789 -121.56653025686428) bank189813 +189814 POINT(37.178388291144174 -121.70480617179004) bank189814 +189815 POINT(37.04301503578591 -121.65030213314728) bank189815 +189816 POINT(37.878475235524725 -121.51179536837498) bank189816 +189817 POINT(38.39436227734869 -123.34874874847476) bank189817 +189818 POINT(38.1599805479077 -123.36843891843365) bank189818 +189819 POINT(37.8958077780915 -121.5321580460728) bank189819 +189820 POINT(38.31426804312163 -122.63193505553343) bank189820 +189821 POINT(37.29528039225942 -122.62408953738343) bank189821 +189822 POINT(38.24080222675228 -121.65947667968672) bank189822 +189823 POINT(38.203957658390074 -123.1567647863879) bank189823 +189824 POINT(38.71579043618707 -122.71001586623198) bank189824 +189825 POINT(37.94250578394805 -122.11831059942465) bank189825 +189826 POINT(38.47962912159425 -122.32920547178489) bank189826 +189827 POINT(38.42316530743236 -122.47430109598183) bank189827 +189828 POINT(37.068514749143894 -122.08154514614216) bank189828 +189829 POINT(38.65945716231269 -122.04275495261764) bank189829 +189830 POINT(37.805522526455974 -122.84506563296758) bank189830 +189831 POINT(38.42721182266371 -122.49251248015509) bank189831 +189832 POINT(38.344825205602326 -123.38301813716585) bank189832 +189833 POINT(38.426044459081155 -122.36098956949864) bank189833 +189834 POINT(36.94176988765815 -122.03950579476518) bank189834 +189835 POINT(37.65135991401541 -123.20551930563151) bank189835 +189836 POINT(37.75948759664619 -121.62888706335809) bank189836 +189837 POINT(37.371699055142074 -122.30674084514762) bank189837 +189838 POINT(37.39111881023409 -122.03777156098752) bank189838 +189839 POINT(38.73425652026466 -121.9392461864363) bank189839 +189840 POINT(37.46881566983595 -121.9513862072171) bank189840 +189841 POINT(38.698621255840806 -122.28698554910582) bank189841 +189842 POINT(38.55415375155447 -122.8916743719754) bank189842 +189843 POINT(38.70131356316107 -122.04071990133313) bank189843 +189844 POINT(38.707477291339444 -121.71565168093151) bank189844 +189845 POINT(37.760704571562144 -121.488687110131) bank189845 +189846 POINT(37.283461413739374 -122.47812289663833) bank189846 +189847 POINT(38.50460179027552 -122.83532881334119) bank189847 +189848 POINT(37.375378062322426 -121.72132184469604) bank189848 +189849 POINT(36.803869712997205 -123.22940936595947) bank189849 +189850 POINT(38.75504380583358 -122.02904904889081) bank189850 +189851 POINT(36.86637665408999 -121.89898697429673) bank189851 +189852 POINT(38.60746489245845 -122.6639665484604) bank189852 +189853 POINT(38.11408250034663 -122.54367331192654) bank189853 +189854 POINT(37.984749010665794 -122.70452719119011) bank189854 +189855 POINT(37.88566604145535 -122.37874151313562) bank189855 +189856 POINT(36.84264498082756 -121.58542168007536) bank189856 +189857 POINT(38.08266510383522 -122.65184783970466) bank189857 +189858 POINT(36.88277794751508 -122.07311158474607) bank189858 +189859 POINT(36.88941790970771 -121.46573158333871) bank189859 +189860 POINT(38.73589368821218 -121.43314549330671) bank189860 +189861 POINT(38.19406519346473 -121.4472719661571) bank189861 +189862 POINT(37.0044488471864 -121.73209234931443) bank189862 +189863 POINT(37.97379126297406 -122.91815404528685) bank189863 +189864 POINT(38.39753660199367 -121.86879163474981) bank189864 +189865 POINT(38.74809041371475 -122.72234209784085) bank189865 +189866 POINT(37.06752390908732 -121.80516172101927) bank189866 +189867 POINT(38.65694448614954 -121.75318826947178) bank189867 +189868 POINT(38.696750898882875 -122.66620902754063) bank189868 +189869 POINT(37.35412592602692 -122.97346231092374) bank189869 +189870 POINT(37.784776516832224 -122.89905622745187) bank189870 +189871 POINT(38.45032906061858 -121.99486603759756) bank189871 +189872 POINT(38.22786146396511 -122.94513644490813) bank189872 +189873 POINT(38.768259089384856 -123.11549611999935) bank189873 +189874 POINT(38.324750233579145 -121.74302382515766) bank189874 +189875 POINT(38.195829993832824 -121.94598393220538) bank189875 +189876 POINT(38.304578988102215 -123.38829256597909) bank189876 +189877 POINT(36.82600447764679 -122.26998202093704) bank189877 +189878 POINT(38.01612257846168 -122.45958088949968) bank189878 +189879 POINT(37.34823916669435 -123.27663271591976) bank189879 +189880 POINT(38.49217711772079 -122.92448244131477) bank189880 +189881 POINT(37.87009977508608 -121.6884776081529) bank189881 +189882 POINT(37.187773686308596 -122.67781942230553) bank189882 +189883 POINT(36.97858036249615 -122.36209315049085) bank189883 +189884 POINT(37.74089115296996 -123.23150910719446) bank189884 +189885 POINT(37.49855634902696 -121.88873012290146) bank189885 +189886 POINT(37.60194392205482 -121.84486082121133) bank189886 +189887 POINT(37.62132054087407 -122.80023225177548) bank189887 +189888 POINT(37.7000796823373 -122.84491808281432) bank189888 +189889 POINT(37.02494453644409 -122.94445469318543) bank189889 +189890 POINT(37.53946564609238 -121.81901169363492) bank189890 +189891 POINT(38.1859045819394 -123.15949510926967) bank189891 +189892 POINT(37.95355579298391 -122.60641722203358) bank189892 +189893 POINT(37.14224503696853 -123.06298306502661) bank189893 +189894 POINT(38.56892411822869 -123.35688636461498) bank189894 +189895 POINT(38.561542826735256 -121.96837629663959) bank189895 +189896 POINT(37.9760114240741 -121.75261545262862) bank189896 +189897 POINT(38.42723313357309 -122.88107012953031) bank189897 +189898 POINT(38.50594960456402 -122.16969766204251) bank189898 +189899 POINT(37.232002427247004 -123.34619768480512) bank189899 +189900 POINT(37.50803201745265 -122.22546909008997) bank189900 +189901 POINT(37.04096519761413 -121.7910432392599) bank189901 +189902 POINT(38.01076957592135 -122.77310818795597) bank189902 +189903 POINT(38.2746158037778 -123.19008482280576) bank189903 +189904 POINT(37.29712344256974 -123.28440234937311) bank189904 +189905 POINT(38.28378593157157 -122.47044917994305) bank189905 +189906 POINT(37.026639991975884 -122.64908781791996) bank189906 +189907 POINT(37.61663101942899 -122.1675907932184) bank189907 +189908 POINT(38.76620328471298 -123.01369866026079) bank189908 +189909 POINT(37.608922195659915 -122.40543468017016) bank189909 +189910 POINT(37.90747617542959 -121.62186925460031) bank189910 +189911 POINT(38.3146286369853 -121.70324541963149) bank189911 +189912 POINT(36.877295969761136 -121.49157016428283) bank189912 +189913 POINT(37.60925896664682 -123.36019599086669) bank189913 +189914 POINT(38.51894694352493 -123.32981724492514) bank189914 +189915 POINT(37.966543569831344 -121.68854208460316) bank189915 +189916 POINT(38.09737693876337 -122.73293404963019) bank189916 +189917 POINT(37.66554459637063 -121.58307830455043) bank189917 +189918 POINT(38.33611405000964 -121.95471655994972) bank189918 +189919 POINT(38.07047039209051 -121.93390385825583) bank189919 +189920 POINT(37.99042618213456 -121.44746668639182) bank189920 +189921 POINT(36.794924383849526 -121.78372934869375) bank189921 +189922 POINT(37.35354617120524 -122.97390810123996) bank189922 +189923 POINT(37.80547665599599 -123.00042448263409) bank189923 +189924 POINT(37.94349330498562 -122.76302815115328) bank189924 +189925 POINT(37.067285355647215 -122.77903161421928) bank189925 +189926 POINT(38.731864929429236 -122.1607050708637) bank189926 +189927 POINT(38.32682484306457 -122.42668103885786) bank189927 +189928 POINT(36.95492925854042 -121.83745229735213) bank189928 +189929 POINT(38.00731029769332 -123.25947955100277) bank189929 +189930 POINT(37.61572894841323 -122.45448879000558) bank189930 +189931 POINT(38.36738938051647 -121.75644968268752) bank189931 +189932 POINT(37.949167312643176 -121.75258379635842) bank189932 +189933 POINT(36.89637131538505 -122.11616815951368) bank189933 +189934 POINT(38.64679298209688 -121.55959135385447) bank189934 +189935 POINT(38.39042932417946 -122.8340887692469) bank189935 +189936 POINT(37.4386961965878 -121.5678598092635) bank189936 +189937 POINT(38.728048600905694 -121.9554394760207) bank189937 +189938 POINT(37.10093508927932 -121.87502666474566) bank189938 +189939 POINT(37.71591841088323 -122.2583764783645) bank189939 +189940 POINT(37.702143207968234 -123.230718205973) bank189940 +189941 POINT(38.504266461110404 -123.36570909161664) bank189941 +189942 POINT(38.39735335261803 -122.55045259103258) bank189942 +189943 POINT(38.655247562520586 -123.11971809822685) bank189943 +189944 POINT(37.43984849899938 -122.93252685326668) bank189944 +189945 POINT(37.881590607126526 -122.54832339203294) bank189945 +189946 POINT(37.19034938982504 -121.64446897295777) bank189946 +189947 POINT(37.69469786768847 -122.47999510230377) bank189947 +189948 POINT(38.38286522410383 -123.17913845159676) bank189948 +189949 POINT(37.15508364490361 -121.8788413650254) bank189949 +189950 POINT(38.66927351928712 -121.43552120159146) bank189950 +189951 POINT(36.866616480700955 -121.49343721212834) bank189951 +189952 POINT(38.60720216974186 -122.27990697509725) bank189952 +189953 POINT(38.28683179028757 -122.7396171814848) bank189953 +189954 POINT(37.43678342928761 -122.46219191821218) bank189954 +189955 POINT(38.484513521903295 -122.11119320294777) bank189955 +189956 POINT(38.35403626515961 -121.69517398206963) bank189956 +189957 POINT(37.75180676256653 -121.72294421440861) bank189957 +189958 POINT(37.123520626653196 -122.92300801114386) bank189958 +189959 POINT(37.74332713844218 -122.4160621889105) bank189959 +189960 POINT(37.68604463191294 -122.1278496843852) bank189960 +189961 POINT(38.62557047218473 -121.46467129426615) bank189961 +189962 POINT(38.15829558983494 -121.44249770356751) bank189962 +189963 POINT(37.64005389076937 -122.76305093130136) bank189963 +189964 POINT(38.656295566150604 -122.36797713065341) bank189964 +189965 POINT(37.92004057476026 -122.00006384683294) bank189965 +189966 POINT(38.25263763761734 -123.304381675823) bank189966 +189967 POINT(38.29271928525268 -122.70358452013183) bank189967 +189968 POINT(37.43428978760068 -123.18566256634399) bank189968 +189969 POINT(37.320416990740085 -122.11999232697308) bank189969 +189970 POINT(37.29780560230122 -123.03899866152338) bank189970 +189971 POINT(37.546469509795266 -122.64207095664266) bank189971 +189972 POINT(37.80795258270803 -122.67414997511558) bank189972 +189973 POINT(37.936940847100104 -122.88376010023961) bank189973 +189974 POINT(38.205293652361796 -123.05107715465442) bank189974 +189975 POINT(38.24850099612253 -122.13510201172463) bank189975 +189976 POINT(36.86517312904209 -122.71493586465684) bank189976 +189977 POINT(37.733551061597446 -122.23426147038205) bank189977 +189978 POINT(37.881245342124046 -123.20118754405823) bank189978 +189979 POINT(36.954372996253774 -122.24043528840211) bank189979 +189980 POINT(37.44578818089566 -123.08917593585457) bank189980 +189981 POINT(37.06547073007004 -122.4054185293238) bank189981 +189982 POINT(38.56330271434187 -123.08657575492755) bank189982 +189983 POINT(38.37424009935676 -122.57196888296156) bank189983 +189984 POINT(37.20309644760371 -122.67734098583809) bank189984 +189985 POINT(38.278610211015014 -122.41324605431096) bank189985 +189986 POINT(37.410911661291095 -121.5400291633951) bank189986 +189987 POINT(38.008565933495234 -122.3117149224887) bank189987 +189988 POINT(37.715972495717544 -122.78127812290896) bank189988 +189989 POINT(38.207221020647225 -121.5800231425494) bank189989 +189990 POINT(37.46063336946338 -121.70320831596955) bank189990 +189991 POINT(37.10649237950545 -122.91071785418718) bank189991 +189992 POINT(38.03299478014738 -121.8850881770239) bank189992 +189993 POINT(36.829531634382874 -121.949782548745) bank189993 +189994 POINT(36.81347237325043 -122.99461092045772) bank189994 +189995 POINT(37.587250248491245 -122.15829637923306) bank189995 +189996 POINT(37.167438881537656 -122.57354039603499) bank189996 +189997 POINT(37.07627341005687 -121.76918246034764) bank189997 +189998 POINT(36.96575098708902 -121.66623531333661) bank189998 +189999 POINT(37.39228542547668 -122.40881053388847) bank189999 +190000 POINT(38.65125203959505 -123.1432821454334) bank190000 +190001 POINT(37.984053030279824 -123.18707131548838) bank190001 +190002 POINT(37.108609675868166 -123.34864488800753) bank190002 +190003 POINT(38.44743814397636 -123.03413013282444) bank190003 +190004 POINT(37.204881082507164 -123.23234861734983) bank190004 +190005 POINT(38.35695995521943 -123.41133180993803) bank190005 +190006 POINT(38.752236032971325 -122.28959846330719) bank190006 +190007 POINT(37.99009063800608 -121.59906564124053) bank190007 +190008 POINT(37.58421238270554 -121.48906213567346) bank190008 +190009 POINT(38.43524534543149 -122.42082584297134) bank190009 +190010 POINT(38.544576957495714 -121.95124350137284) bank190010 +190011 POINT(37.91416421158716 -122.58666771073135) bank190011 +190012 POINT(37.0076514290223 -122.50035760968244) bank190012 +190013 POINT(38.637913227270154 -122.27026530962868) bank190013 +190014 POINT(38.67579662378568 -122.12595421013904) bank190014 +190015 POINT(38.14517762619187 -122.94567324451464) bank190015 +190016 POINT(37.380782563245575 -122.33216953655221) bank190016 +190017 POINT(37.00723505423032 -121.61173711230363) bank190017 +190018 POINT(37.47513404006865 -122.9077803819361) bank190018 +190019 POINT(38.60888612520482 -122.7284095288215) bank190019 +190020 POINT(38.27938800971009 -123.31196472128799) bank190020 +190021 POINT(37.78868085463036 -122.80939360597326) bank190021 +190022 POINT(37.81319208019392 -122.66443490882338) bank190022 +190023 POINT(38.365601649649875 -121.9284576374727) bank190023 +190024 POINT(37.59358709230847 -123.26604153721968) bank190024 +190025 POINT(37.46381170817275 -122.47773945618098) bank190025 +190026 POINT(38.212800220401164 -122.42238882755362) bank190026 +190027 POINT(38.61976998724337 -122.17213715502005) bank190027 +190028 POINT(38.16756231942948 -122.36769736767755) bank190028 +190029 POINT(38.73098386289952 -121.68785172062489) bank190029 +190030 POINT(37.20406152417003 -122.83831511070687) bank190030 +190031 POINT(38.16446993028921 -123.0716037168529) bank190031 +190032 POINT(38.51414503486741 -121.62496297425119) bank190032 +190033 POINT(37.73456604478417 -121.6228371155296) bank190033 +190034 POINT(37.98090186717497 -122.88947636788438) bank190034 +190035 POINT(37.44469325148684 -121.47445966217812) bank190035 +190036 POINT(38.225282699402065 -122.57854715146857) bank190036 +190037 POINT(37.597405993499734 -123.1096260889969) bank190037 +190038 POINT(37.12759210604008 -122.64006867194786) bank190038 +190039 POINT(38.529372188044384 -122.39586094048522) bank190039 +190040 POINT(36.80078850395581 -122.1574486985823) bank190040 +190041 POINT(38.19772179234852 -123.4127375980767) bank190041 +190042 POINT(37.031691509594495 -122.88721893584476) bank190042 +190043 POINT(37.19723908655439 -121.74232243997861) bank190043 +190044 POINT(38.44990449989541 -122.1575435554653) bank190044 +190045 POINT(37.178248993963365 -122.49238647814802) bank190045 +190046 POINT(37.991098987453064 -122.07138935906269) bank190046 +190047 POINT(38.39256083805645 -123.0056164785855) bank190047 +190048 POINT(38.483296865192116 -121.97408596567455) bank190048 +190049 POINT(38.21717339681656 -122.66423599603596) bank190049 +190050 POINT(37.56924341317747 -122.77657051266532) bank190050 +190051 POINT(37.551055285890286 -121.42052295849093) bank190051 +190052 POINT(37.49019760536718 -122.75451335354876) bank190052 +190053 POINT(38.75854988589504 -121.85032383852271) bank190053 +190054 POINT(38.217132338037516 -122.03016117881464) bank190054 +190055 POINT(38.293548777352775 -123.15423195253882) bank190055 +190056 POINT(38.69479859071746 -122.58734630514249) bank190056 +190057 POINT(37.281079361018655 -123.39771272880647) bank190057 +190058 POINT(37.91827137695388 -121.53607733075121) bank190058 +190059 POINT(36.95691626570921 -121.53223911715668) bank190059 +190060 POINT(38.02456939299057 -121.89007791880202) bank190060 +190061 POINT(38.267271707258786 -122.87163356267924) bank190061 +190062 POINT(37.90274235959072 -121.76607040117052) bank190062 +190063 POINT(37.14659168096219 -121.9552702965897) bank190063 +190064 POINT(37.52768320620098 -122.0453931242674) bank190064 +190065 POINT(36.97421929823183 -122.08569406601207) bank190065 +190066 POINT(38.60361463959177 -122.43762614215255) bank190066 +190067 POINT(37.138221313323925 -121.8017891638465) bank190067 +190068 POINT(38.75838617278497 -121.55157321873097) bank190068 +190069 POINT(38.60026557742495 -122.90043039977058) bank190069 +190070 POINT(37.95117222563471 -121.5870815342047) bank190070 +190071 POINT(37.08217749827893 -123.10150412747838) bank190071 +190072 POINT(37.809836208657664 -123.3134294474817) bank190072 +190073 POINT(37.27846483020509 -122.66034098213312) bank190073 +190074 POINT(38.649394193865476 -122.72150895579563) bank190074 +190075 POINT(38.49565695252105 -122.76731592948018) bank190075 +190076 POINT(37.80195249608802 -121.7303721436479) bank190076 +190077 POINT(36.81101452848171 -122.07575135827855) bank190077 +190078 POINT(38.056585998289826 -123.24232929920764) bank190078 +190079 POINT(37.61500489813322 -121.50261442674628) bank190079 +190080 POINT(38.19576120963851 -121.72113777959035) bank190080 +190081 POINT(37.57251027848149 -123.05258902561516) bank190081 +190082 POINT(37.350003488165484 -122.80558336824936) bank190082 +190083 POINT(37.45607787100528 -123.41264149938883) bank190083 +190084 POINT(38.1632440010817 -122.84549150349433) bank190084 +190085 POINT(37.97797372022963 -122.63503028120503) bank190085 +190086 POINT(37.53205798835153 -121.79027696759702) bank190086 +190087 POINT(37.71285307960721 -122.93073640559582) bank190087 +190088 POINT(37.12534578537205 -123.41171178927794) bank190088 +190089 POINT(37.60773216257129 -121.8419249474314) bank190089 +190090 POINT(37.653780309559956 -122.64136275221679) bank190090 +190091 POINT(38.208396709103134 -121.81343143746308) bank190091 +190092 POINT(38.12933799201992 -121.57057363926769) bank190092 +190093 POINT(37.558454317940964 -122.08717859795246) bank190093 +190094 POINT(36.98732701925004 -122.52340796312038) bank190094 +190095 POINT(38.15360770716354 -121.84690942788967) bank190095 +190096 POINT(38.71254077415182 -123.18965405965845) bank190096 +190097 POINT(37.8375470013328 -122.17225091596872) bank190097 +190098 POINT(38.44991710169351 -121.584908311713) bank190098 +190099 POINT(36.788490988752734 -122.53394002973278) bank190099 +190100 POINT(38.71356784525078 -121.76122131323852) bank190100 +190101 POINT(38.04874086557331 -122.78052757913454) bank190101 +190102 POINT(38.35199893781199 -121.86098292053123) bank190102 +190103 POINT(38.752777458486364 -123.10948311930643) bank190103 +190104 POINT(37.072675588940854 -123.11022632085242) bank190104 +190105 POINT(38.40415707997766 -121.50698838817651) bank190105 +190106 POINT(36.97601119484111 -121.70731595052686) bank190106 +190107 POINT(37.12314675930903 -122.1291070477643) bank190107 +190108 POINT(38.201031553060595 -123.17834543495478) bank190108 +190109 POINT(38.263664385694035 -122.36193566256469) bank190109 +190110 POINT(37.6237999214783 -121.71180377348513) bank190110 +190111 POINT(38.45305750711444 -121.89547871406216) bank190111 +190112 POINT(37.22165079326791 -123.28154224530782) bank190112 +190113 POINT(38.46456547832933 -122.8114425208622) bank190113 +190114 POINT(37.39707546927755 -122.00975929755715) bank190114 +190115 POINT(37.48592732702787 -121.67368657537382) bank190115 +190116 POINT(37.87748047643688 -122.79312106897495) bank190116 +190117 POINT(37.19126187217598 -121.51060230695047) bank190117 +190118 POINT(38.7676729684302 -121.53300180947761) bank190118 +190119 POINT(37.84405907613831 -122.15348274304735) bank190119 +190120 POINT(38.248212941089335 -123.20627360182527) bank190120 +190121 POINT(37.80241392989588 -121.8193974299045) bank190121 +190122 POINT(37.47060445705981 -123.05791707445466) bank190122 +190123 POINT(37.980005974613675 -122.37317698720508) bank190123 +190124 POINT(37.71404344973006 -121.88462613382002) bank190124 +190125 POINT(37.11889624907329 -123.07179317431377) bank190125 +190126 POINT(37.92164846369158 -122.56474906354939) bank190126 +190127 POINT(36.89654496662937 -121.73071144128187) bank190127 +190128 POINT(37.58511989452383 -121.94348759195029) bank190128 +190129 POINT(37.302100468160894 -123.29761804837436) bank190129 +190130 POINT(36.99181147567717 -122.18111923234886) bank190130 +190131 POINT(38.733393394446374 -122.7772376877846) bank190131 +190132 POINT(37.17945297207272 -122.16779249141413) bank190132 +190133 POINT(38.76066910346155 -121.99716857943628) bank190133 +190134 POINT(36.86722244751372 -121.71615802620326) bank190134 +190135 POINT(37.91912726618688 -122.11575462840176) bank190135 +190136 POINT(37.206613827823986 -123.2154374119001) bank190136 +190137 POINT(38.71222331171163 -122.68109439238148) bank190137 +190138 POINT(38.010812572004625 -122.94559685130237) bank190138 +190139 POINT(37.506925174866495 -121.72519057916776) bank190139 +190140 POINT(37.08777145083389 -122.44589326976936) bank190140 +190141 POINT(36.82400529744968 -123.21956578866022) bank190141 +190142 POINT(38.673262949922346 -121.69250663411727) bank190142 +190143 POINT(36.88098363372184 -123.08771928637279) bank190143 +190144 POINT(38.02240837974709 -122.03676731922458) bank190144 +190145 POINT(37.946287157273346 -121.80638188728643) bank190145 +190146 POINT(37.28472498861022 -122.71517853278468) bank190146 +190147 POINT(37.44854300489502 -121.65312750845128) bank190147 +190148 POINT(37.11149825904287 -122.17686080375321) bank190148 +190149 POINT(37.44504689490957 -122.24755029628002) bank190149 +190150 POINT(38.37809213333589 -121.92162588298366) bank190150 +190151 POINT(38.29920444395699 -123.27135808001576) bank190151 +190152 POINT(38.1220673403409 -122.66617416574972) bank190152 +190153 POINT(37.47892885544453 -123.3296959077848) bank190153 +190154 POINT(38.46158878959989 -123.29412373032928) bank190154 +190155 POINT(38.73356698379878 -123.33483175555895) bank190155 +190156 POINT(38.610872837296625 -123.12662185603841) bank190156 +190157 POINT(37.781486303838015 -121.78422422499602) bank190157 +190158 POINT(37.60021066198385 -123.01509209367784) bank190158 +190159 POINT(37.730932395237396 -123.10451103889402) bank190159 +190160 POINT(38.74884210123472 -122.10058842882339) bank190160 +190161 POINT(37.031490885336886 -121.95777891265026) bank190161 +190162 POINT(37.571100381933526 -121.52849773917389) bank190162 +190163 POINT(37.0899641477331 -122.99013669395639) bank190163 +190164 POINT(37.95432880179609 -122.88990999506512) bank190164 +190165 POINT(37.87161032817544 -121.91390164356669) bank190165 +190166 POINT(37.44965079078442 -121.96216733768513) bank190166 +190167 POINT(38.53983759912641 -121.9082469721572) bank190167 +190168 POINT(38.414761009060236 -122.72469021322797) bank190168 +190169 POINT(38.09308075723368 -122.81368040083737) bank190169 +190170 POINT(38.71813912251517 -122.43840119447776) bank190170 +190171 POINT(37.9113794642827 -122.65928390383915) bank190171 +190172 POINT(38.32000722076627 -123.4058621731067) bank190172 +190173 POINT(37.601207206472004 -123.17442266956667) bank190173 +190174 POINT(37.97986185254979 -121.68085706596929) bank190174 +190175 POINT(37.02406270648912 -121.85714170205354) bank190175 +190176 POINT(38.70177329719249 -123.3272928281366) bank190176 +190177 POINT(38.235861596480206 -121.8455710122473) bank190177 +190178 POINT(37.95070323603209 -122.973287366937) bank190178 +190179 POINT(38.70560605312929 -122.9071095544603) bank190179 +190180 POINT(37.03005525823041 -122.67519789578027) bank190180 +190181 POINT(37.58278999004082 -122.1608043596625) bank190181 +190182 POINT(36.95396578867377 -121.85364698518768) bank190182 +190183 POINT(38.3913377575226 -122.23870643655292) bank190183 +190184 POINT(36.97004210098876 -122.65935162725852) bank190184 +190185 POINT(38.631403640335414 -121.93281701395317) bank190185 +190186 POINT(37.731686928637856 -123.35020620273097) bank190186 +190187 POINT(38.02555927583662 -122.08414678808269) bank190187 +190188 POINT(38.4049896019261 -122.03289531764524) bank190188 +190189 POINT(37.7024350280551 -121.90006023275888) bank190189 +190190 POINT(38.66432146909711 -122.35901878127088) bank190190 +190191 POINT(38.581782253604764 -122.42750949905295) bank190191 +190192 POINT(37.019326292490604 -122.33732823060849) bank190192 +190193 POINT(38.07942984957906 -122.30845904362934) bank190193 +190194 POINT(36.82363991445017 -121.4708769970119) bank190194 +190195 POINT(37.31428570251861 -122.32617095321896) bank190195 +190196 POINT(37.42069513541479 -121.46220512374295) bank190196 +190197 POINT(38.62790130318315 -121.83254101834245) bank190197 +190198 POINT(37.95492037072326 -121.77806810060974) bank190198 +190199 POINT(38.604708081338 -121.54881180990859) bank190199 +190200 POINT(38.53987581039656 -122.02842773121414) bank190200 +190201 POINT(38.50591486248537 -121.42109144539421) bank190201 +190202 POINT(38.67222108542989 -122.32758940505374) bank190202 +190203 POINT(37.96511933345954 -122.39233003051365) bank190203 +190204 POINT(38.11461275975455 -122.19426949581212) bank190204 +190205 POINT(37.002669005834726 -123.24756321026027) bank190205 +190206 POINT(37.152743877510034 -122.79926148130514) bank190206 +190207 POINT(38.74906593763735 -122.35963208693589) bank190207 +190208 POINT(38.72580471202268 -121.4619404830876) bank190208 +190209 POINT(38.490970989015224 -122.65798770369076) bank190209 +190210 POINT(38.19931286492169 -121.55266171311892) bank190210 +190211 POINT(37.263587813229556 -122.71076642428044) bank190211 +190212 POINT(37.96485386622996 -122.27559511860161) bank190212 +190213 POINT(37.611164743283794 -122.13544168673087) bank190213 +190214 POINT(38.752699286357 -123.19736399216548) bank190214 +190215 POINT(38.002315002037705 -121.98550205836926) bank190215 +190216 POINT(38.520041711966556 -122.96417047842831) bank190216 +190217 POINT(36.96093880639574 -122.60116804367156) bank190217 +190218 POINT(37.7512087581826 -122.62102339442642) bank190218 +190219 POINT(38.45933063918162 -122.55076925761041) bank190219 +190220 POINT(38.52626687592993 -122.59298210947745) bank190220 +190221 POINT(38.366343256962324 -122.04746082848573) bank190221 +190222 POINT(37.35629558919678 -123.29687713730144) bank190222 +190223 POINT(38.727233794950735 -123.12960496220055) bank190223 +190224 POINT(38.367127055824554 -123.38832828232306) bank190224 +190225 POINT(36.840995342070975 -122.13066060262177) bank190225 +190226 POINT(38.33365949309687 -122.07360484340079) bank190226 +190227 POINT(38.08655509264123 -123.25043689932656) bank190227 +190228 POINT(38.07022735794038 -122.95190032116784) bank190228 +190229 POINT(37.94475130314086 -122.6898893796935) bank190229 +190230 POINT(36.9948105957074 -122.56650204778614) bank190230 +190231 POINT(37.318189030444515 -122.79879108741588) bank190231 +190232 POINT(37.605866684342516 -123.21180392686306) bank190232 +190233 POINT(37.4364927387784 -121.44313359378093) bank190233 +190234 POINT(37.20774954213246 -122.88201225893204) bank190234 +190235 POINT(38.23248966588141 -122.3086985884247) bank190235 +190236 POINT(36.816791921938055 -121.91115929084935) bank190236 +190237 POINT(37.945906660349166 -121.81090065358698) bank190237 +190238 POINT(37.492664610980505 -122.67891272715582) bank190238 +190239 POINT(38.670362523375495 -122.63964251843042) bank190239 +190240 POINT(38.09666587232895 -123.27466286543607) bank190240 +190241 POINT(37.56445849869228 -122.17656495919803) bank190241 +190242 POINT(38.00204991918128 -122.14703665060136) bank190242 +190243 POINT(36.78892229363267 -121.50845333779793) bank190243 +190244 POINT(38.37408019230443 -122.63375225490614) bank190244 +190245 POINT(37.60292794923715 -123.22287018176397) bank190245 +190246 POINT(37.820033057456364 -123.35815469105223) bank190246 +190247 POINT(36.86560764937766 -122.3393417699006) bank190247 +190248 POINT(36.971165186694435 -122.24260455275201) bank190248 +190249 POINT(37.30929825273429 -122.85641789928019) bank190249 +190250 POINT(37.077739740358204 -122.99842169516441) bank190250 +190251 POINT(38.62488043608888 -122.14347398631863) bank190251 +190252 POINT(37.67015324501676 -122.80855277395715) bank190252 +190253 POINT(37.0536829730042 -123.09413407015647) bank190253 +190254 POINT(37.55210283386933 -123.10898208117277) bank190254 +190255 POINT(37.40151181305484 -123.33217659535828) bank190255 +190256 POINT(38.33922043719214 -122.35443555085189) bank190256 +190257 POINT(37.72540348251855 -121.68801260701386) bank190257 +190258 POINT(38.26994679893598 -122.65368034035909) bank190258 +190259 POINT(37.21958842337099 -122.24845941720504) bank190259 +190260 POINT(38.15726875029858 -122.1144105903922) bank190260 +190261 POINT(38.28047424387416 -123.32366372707095) bank190261 +190262 POINT(37.52902278379467 -121.8346792527931) bank190262 +190263 POINT(37.22397865032081 -122.06942073957805) bank190263 +190264 POINT(37.63682700967043 -121.64320608084097) bank190264 +190265 POINT(37.07633035924002 -121.69759590929853) bank190265 +190266 POINT(38.29590949753236 -122.34268376755878) bank190266 +190267 POINT(38.644323527384635 -121.61547682818386) bank190267 +190268 POINT(37.57081371260703 -123.07560505586466) bank190268 +190269 POINT(37.860601658099725 -122.03813443532785) bank190269 +190270 POINT(36.90787846424373 -122.30725521486225) bank190270 +190271 POINT(38.77474460530602 -122.6641276560396) bank190271 +190272 POINT(37.81167681909136 -121.5164738417873) bank190272 +190273 POINT(37.7667132659387 -122.1990818947276) bank190273 +190274 POINT(37.754116344649944 -121.60747827114967) bank190274 +190275 POINT(38.60957487068366 -122.30870707117748) bank190275 +190276 POINT(37.79973618856812 -122.56700795970157) bank190276 +190277 POINT(37.30344872646318 -122.13393939403528) bank190277 +190278 POINT(38.24810907121996 -121.6758336078685) bank190278 +190279 POINT(37.92589638508426 -121.7477400960349) bank190279 +190280 POINT(38.120987483120125 -121.57866011330275) bank190280 +190281 POINT(38.452261332818495 -122.16951341750953) bank190281 +190282 POINT(38.68604081306638 -123.08331281405994) bank190282 +190283 POINT(36.864562419642006 -121.65474287337615) bank190283 +190284 POINT(37.55759566348544 -123.27512988347826) bank190284 +190285 POINT(37.38995301473096 -122.30584082727411) bank190285 +190286 POINT(37.1445856588152 -121.87348814349922) bank190286 +190287 POINT(37.03418027569791 -123.14005477415972) bank190287 +190288 POINT(37.23688558821274 -122.6416579538851) bank190288 +190289 POINT(37.60616899294221 -122.76967697261728) bank190289 +190290 POINT(38.39214274198941 -123.4098631121708) bank190290 +190291 POINT(38.31908638249742 -123.18132524051205) bank190291 +190292 POINT(36.83696879683852 -123.25563974283352) bank190292 +190293 POINT(38.08755303485631 -121.91695093417292) bank190293 +190294 POINT(37.99037878397606 -122.64940545595903) bank190294 +190295 POINT(36.98673592189876 -122.70935094682753) bank190295 +190296 POINT(37.86963094917295 -122.19869566691797) bank190296 +190297 POINT(37.35778228511454 -121.93739178504177) bank190297 +190298 POINT(38.00585386394026 -123.05860685274486) bank190298 +190299 POINT(37.04119237974089 -123.32648216338698) bank190299 +190300 POINT(37.551579098291064 -121.8987090041087) bank190300 +190301 POINT(38.35302211135384 -121.66817038287732) bank190301 +190302 POINT(37.61229622281396 -122.58065970119418) bank190302 +190303 POINT(37.51428869076983 -121.70696476328037) bank190303 +190304 POINT(36.937733646736774 -123.0429937228042) bank190304 +190305 POINT(38.35510716390327 -121.76055988994727) bank190305 +190306 POINT(38.52693880141352 -121.87429239661253) bank190306 +190307 POINT(37.05206144113645 -121.64188789206798) bank190307 +190308 POINT(36.93035711330275 -123.27492197012276) bank190308 +190309 POINT(37.69081004756529 -122.01978199399935) bank190309 +190310 POINT(37.60781939406551 -122.14804672094624) bank190310 +190311 POINT(36.83190316546472 -122.56854349886537) bank190311 +190312 POINT(38.42671851134773 -121.66975599278062) bank190312 +190313 POINT(38.31865725227019 -123.22571590035302) bank190313 +190314 POINT(38.20205599470537 -122.33092748403392) bank190314 +190315 POINT(38.65913773954243 -123.16424395168765) bank190315 +190316 POINT(37.51229629806036 -121.7855366754414) bank190316 +190317 POINT(38.691810089460056 -122.25925239573824) bank190317 +190318 POINT(37.900116652716264 -121.70602412728394) bank190318 +190319 POINT(37.93909713535011 -122.06492322916628) bank190319 +190320 POINT(38.681165458558155 -123.0255595611829) bank190320 +190321 POINT(37.35706906042024 -122.8535441562591) bank190321 +190322 POINT(37.27074562808945 -122.01413581666625) bank190322 +190323 POINT(37.75492839210256 -123.06070713676564) bank190323 +190324 POINT(38.41914496979809 -121.7521179253074) bank190324 +190325 POINT(38.488654838365974 -123.41458764186793) bank190325 +190326 POINT(36.9493587717298 -123.24334583019083) bank190326 +190327 POINT(37.868312745821115 -122.32644351988496) bank190327 +190328 POINT(38.5945672553255 -122.20291110468378) bank190328 +190329 POINT(37.44084802468313 -121.78485243942859) bank190329 +190330 POINT(36.88665100626062 -122.53146684513652) bank190330 +190331 POINT(38.40785676696807 -123.16959101270439) bank190331 +190332 POINT(37.45621834735802 -122.65499650970999) bank190332 +190333 POINT(37.32062283657777 -123.31432599747963) bank190333 +190334 POINT(38.612365398188686 -121.49802459634525) bank190334 +190335 POINT(37.59327659637356 -121.99263739746249) bank190335 +190336 POINT(37.92689689775227 -122.33033893567966) bank190336 +190337 POINT(37.74349329420732 -121.55049375379356) bank190337 +190338 POINT(36.94018013451395 -123.19530946396061) bank190338 +190339 POINT(37.810755880039345 -121.65895438142182) bank190339 +190340 POINT(36.920375918119774 -121.87391704050253) bank190340 +190341 POINT(37.787892418405164 -121.6274324830445) bank190341 +190342 POINT(37.850814594810906 -122.0926107142165) bank190342 +190343 POINT(37.64861583826829 -121.8954573434504) bank190343 +190344 POINT(36.9785533202829 -122.81574044710307) bank190344 +190345 POINT(37.94455751053735 -122.98033246077185) bank190345 +190346 POINT(38.394720526596195 -123.35750441164205) bank190346 +190347 POINT(37.952283279346474 -122.99794843150016) bank190347 +190348 POINT(38.406466750431576 -122.94853525902963) bank190348 +190349 POINT(37.713237918156956 -121.64945414206228) bank190349 +190350 POINT(38.256821362343274 -122.92290709709218) bank190350 +190351 POINT(37.725163820750076 -123.32592545574177) bank190351 +190352 POINT(38.7745213244282 -121.92235913437378) bank190352 +190353 POINT(37.55134617634436 -122.12417479122247) bank190353 +190354 POINT(37.024581085677674 -123.15638833701033) bank190354 +190355 POINT(37.57285104637698 -122.2726372088604) bank190355 +190356 POINT(37.78889912002956 -122.74586094372805) bank190356 +190357 POINT(37.90816191545883 -123.16929091982881) bank190357 +190358 POINT(38.17637445909067 -121.95792620473323) bank190358 +190359 POINT(36.84108117921517 -121.43522772476372) bank190359 +190360 POINT(37.31202376011698 -122.56871292588195) bank190360 +190361 POINT(38.14307478167101 -122.72684224330452) bank190361 +190362 POINT(37.93645676836801 -122.63623784690365) bank190362 +190363 POINT(38.41691821986827 -121.5138678233474) bank190363 +190364 POINT(37.04444998346908 -122.30808860110577) bank190364 +190365 POINT(37.32851591212391 -122.48250603880135) bank190365 +190366 POINT(37.66335429114094 -122.94882063141723) bank190366 +190367 POINT(37.175658449464414 -123.30223654077824) bank190367 +190368 POINT(38.111427917595535 -123.21881518333089) bank190368 +190369 POINT(37.68752851875317 -122.92436116623438) bank190369 +190370 POINT(38.036183838034916 -121.90297666752168) bank190370 +190371 POINT(37.55098979432057 -121.77274963941456) bank190371 +190372 POINT(36.97765730101856 -123.374118756254) bank190372 +190373 POINT(37.508956952140046 -122.54713112396404) bank190373 +190374 POINT(37.63124947295248 -122.51872918644722) bank190374 +190375 POINT(37.20496025445624 -122.96899732645488) bank190375 +190376 POINT(38.34153911506207 -122.67333725299613) bank190376 +190377 POINT(37.45193338165909 -122.92108486549446) bank190377 +190378 POINT(38.173509892501826 -122.87333250196316) bank190378 +190379 POINT(36.84860203089746 -121.89389142924813) bank190379 +190380 POINT(37.36314481289656 -122.29227291390454) bank190380 +190381 POINT(37.18627483845495 -123.36845699634145) bank190381 +190382 POINT(37.533770346965376 -121.70530660540203) bank190382 +190383 POINT(38.47925048901489 -121.4648044129661) bank190383 +190384 POINT(36.77994812710368 -122.0189681578005) bank190384 +190385 POINT(37.01926602161127 -122.9902243849649) bank190385 +190386 POINT(38.36008529394846 -122.71929525363353) bank190386 +190387 POINT(37.36932150027332 -121.73945205891967) bank190387 +190388 POINT(37.98568798555979 -121.71176673768021) bank190388 +190389 POINT(37.27806341644233 -122.28278426305381) bank190389 +190390 POINT(37.180574915606535 -122.4788162673629) bank190390 +190391 POINT(38.3901916473758 -122.32241553930102) bank190391 +190392 POINT(37.11108854604714 -123.31221766021977) bank190392 +190393 POINT(37.30066269972034 -122.4308438322122) bank190393 +190394 POINT(37.09876068414291 -122.23583157139484) bank190394 +190395 POINT(37.529170696985055 -123.2457025259047) bank190395 +190396 POINT(37.92323485438598 -121.50811510677376) bank190396 +190397 POINT(37.8188332867725 -123.31878720825812) bank190397 +190398 POINT(37.512606407655376 -121.51633528330929) bank190398 +190399 POINT(38.597950669087844 -122.29149621490592) bank190399 +190400 POINT(38.28480673829577 -123.0634331095094) bank190400 +190401 POINT(37.07084202895758 -122.84625579428162) bank190401 +190402 POINT(37.040194692417835 -122.87019050369676) bank190402 +190403 POINT(36.90106914608751 -121.49914026606245) bank190403 +190404 POINT(37.30351218563302 -121.5414982835849) bank190404 +190405 POINT(37.04233768063336 -121.5743682194502) bank190405 +190406 POINT(37.72215518638208 -121.86929626719437) bank190406 +190407 POINT(38.70507737520756 -121.43746928270906) bank190407 +190408 POINT(38.691038498853324 -122.93706909516087) bank190408 +190409 POINT(37.3756396538937 -122.88661197276119) bank190409 +190410 POINT(37.244437062256566 -122.76771551871617) bank190410 +190411 POINT(37.206476405795954 -122.2510901310494) bank190411 +190412 POINT(37.565890266221395 -122.45305085762283) bank190412 +190413 POINT(36.93099803252487 -121.60773670881265) bank190413 +190414 POINT(38.506941277395015 -122.83728999535347) bank190414 +190415 POINT(37.86130386058053 -123.13401051888815) bank190415 +190416 POINT(37.48309255953303 -121.84371917563043) bank190416 +190417 POINT(38.65344718232672 -122.09762756853185) bank190417 +190418 POINT(38.0435797248154 -121.70110177069941) bank190418 +190419 POINT(36.89291574857529 -123.0251900490147) bank190419 +190420 POINT(38.60745594947643 -122.69707188895606) bank190420 +190421 POINT(38.476682082482064 -121.74785286556695) bank190421 +190422 POINT(37.81479500145915 -123.16092379010608) bank190422 +190423 POINT(38.74257501070893 -122.26026811178211) bank190423 +190424 POINT(36.97928563791237 -121.95002763084727) bank190424 +190425 POINT(38.34833432458697 -122.26693133455353) bank190425 +190426 POINT(36.80670643279242 -121.77709173239337) bank190426 +190427 POINT(37.356376046384725 -122.25504425217191) bank190427 +190428 POINT(38.41269992895899 -121.77407876700762) bank190428 +190429 POINT(37.25975195987499 -122.82805029795202) bank190429 +190430 POINT(37.91149332088289 -121.73606299955654) bank190430 +190431 POINT(37.467044810559216 -122.87466028936628) bank190431 +190432 POINT(38.60806166722261 -121.6605926102961) bank190432 +190433 POINT(36.97192628474919 -121.78810714584596) bank190433 +190434 POINT(37.984120755046135 -121.45923133244239) bank190434 +190435 POINT(38.50763987988567 -122.10390859021963) bank190435 +190436 POINT(37.99509042676925 -122.43287187739723) bank190436 +190437 POINT(37.87049132545339 -121.68556909257568) bank190437 +190438 POINT(36.93278364928077 -122.50585092385505) bank190438 +190439 POINT(38.077167276286694 -122.58647713944663) bank190439 +190440 POINT(37.69836197705074 -121.6132719973764) bank190440 +190441 POINT(38.32759137913912 -122.65777956105765) bank190441 +190442 POINT(37.49627389222631 -122.35410415821701) bank190442 +190443 POINT(37.46019322438379 -122.07396430087437) bank190443 +190444 POINT(38.056092624864675 -121.52982021785309) bank190444 +190445 POINT(38.21839302826368 -122.03477709736794) bank190445 +190446 POINT(38.53562568414696 -121.65709949248244) bank190446 +190447 POINT(38.15264079453454 -122.27723234874415) bank190447 +190448 POINT(38.545803511072144 -122.28624018041837) bank190448 +190449 POINT(38.30450384027821 -122.5075796676831) bank190449 +190450 POINT(37.57184601290492 -122.05644812218864) bank190450 +190451 POINT(37.012137617812314 -122.1119656139246) bank190451 +190452 POINT(37.681328190315845 -122.15489744342412) bank190452 +190453 POINT(37.545344561920196 -122.4105604054399) bank190453 +190454 POINT(37.13640757201621 -122.13571184239595) bank190454 +190455 POINT(38.46178341760901 -123.04125042462822) bank190455 +190456 POINT(38.30028163961707 -121.86852650330046) bank190456 +190457 POINT(37.39410850562242 -122.0497376002442) bank190457 +190458 POINT(37.819507882579494 -123.15883095165849) bank190458 +190459 POINT(36.917183082056724 -121.5281196122063) bank190459 +190460 POINT(37.946857068182084 -121.52712818937054) bank190460 +190461 POINT(38.722383273066214 -123.03760951569328) bank190461 +190462 POINT(38.47991574840942 -123.39944955478143) bank190462 +190463 POINT(38.44708280695576 -123.01381226231933) bank190463 +190464 POINT(38.51502053141283 -121.7789602129649) bank190464 +190465 POINT(38.10950349786205 -121.91149584871975) bank190465 +190466 POINT(36.80249999087551 -122.00161360801326) bank190466 +190467 POINT(37.629020685898595 -121.7409125359591) bank190467 +190468 POINT(37.209345403122725 -122.00896011371326) bank190468 +190469 POINT(38.553352032435164 -121.61686141561745) bank190469 +190470 POINT(36.80237396082766 -122.93389347561094) bank190470 +190471 POINT(37.74384084305354 -121.4561325277876) bank190471 +190472 POINT(37.66998929762779 -123.05280241956653) bank190472 +190473 POINT(38.30391692170053 -123.26748262993871) bank190473 +190474 POINT(38.018924359177284 -122.49063678151857) bank190474 +190475 POINT(37.61829651004007 -123.40450786644956) bank190475 +190476 POINT(37.741016080461556 -121.86823752769496) bank190476 +190477 POINT(37.09199499181392 -121.93620177959485) bank190477 +190478 POINT(37.14608453078854 -123.20754708860248) bank190478 +190479 POINT(36.86268219291035 -122.6118315647164) bank190479 +190480 POINT(38.38089700225111 -121.99673378089182) bank190480 +190481 POINT(37.49515812308575 -122.26525541974085) bank190481 +190482 POINT(37.01968981792902 -122.34697680258084) bank190482 +190483 POINT(38.67232352994503 -122.00076264066048) bank190483 +190484 POINT(38.00861445199073 -122.30340742235629) bank190484 +190485 POINT(38.16089375606453 -122.6136522614567) bank190485 +190486 POINT(38.31013484702597 -122.04672524359115) bank190486 +190487 POINT(38.547090940489134 -122.702553254416) bank190487 +190488 POINT(36.924608894546544 -121.91304661601987) bank190488 +190489 POINT(38.271359566602534 -122.83536391279246) bank190489 +190490 POINT(37.19947484146057 -123.06876529556646) bank190490 +190491 POINT(37.63944717946519 -121.8293578474197) bank190491 +190492 POINT(37.766965525695134 -121.55232320687601) bank190492 +190493 POINT(38.127927283162705 -121.82556236673054) bank190493 +190494 POINT(37.229678021041266 -122.75629313411275) bank190494 +190495 POINT(38.736746236171626 -123.26296909009969) bank190495 +190496 POINT(36.95095864303792 -122.15763231038997) bank190496 +190497 POINT(38.71643486984208 -122.31180423575782) bank190497 +190498 POINT(38.450172931156246 -121.91827900681143) bank190498 +190499 POINT(37.04012931021084 -122.85385459984956) bank190499 +190500 POINT(38.18200643576667 -122.87178926760902) bank190500 +190501 POINT(38.17045312443491 -122.02435227541574) bank190501 +190502 POINT(38.52512439896504 -121.55697699237543) bank190502 +190503 POINT(37.7612175826226 -123.16250436039469) bank190503 +190504 POINT(38.013293047721056 -123.27365577877248) bank190504 +190505 POINT(37.64115084261713 -121.7623793911502) bank190505 +190506 POINT(37.57846275064956 -122.22164031522914) bank190506 +190507 POINT(38.06850248386089 -122.15839925745058) bank190507 +190508 POINT(37.0738980999475 -121.48879557774814) bank190508 +190509 POINT(37.62657518708766 -122.90694310026957) bank190509 +190510 POINT(37.16761029122805 -122.72214450170094) bank190510 +190511 POINT(38.343146355800506 -122.80458266645061) bank190511 +190512 POINT(36.79183477200397 -121.47457372160746) bank190512 +190513 POINT(37.39261806990002 -122.47015259249936) bank190513 +190514 POINT(37.38484293925911 -122.45011042387424) bank190514 +190515 POINT(37.276871115703756 -121.57515763121155) bank190515 +190516 POINT(38.090112233674986 -122.42141528332911) bank190516 +190517 POINT(37.56428556650164 -122.91458230634944) bank190517 +190518 POINT(37.15040397410372 -121.71538140346075) bank190518 +190519 POINT(38.50848199110028 -121.46990461044855) bank190519 +190520 POINT(37.206287407699314 -123.2999008098439) bank190520 +190521 POINT(38.355612755162845 -122.66555578975586) bank190521 +190522 POINT(37.0820921824219 -122.75511478922458) bank190522 +190523 POINT(37.09754729458237 -122.22610450083458) bank190523 +190524 POINT(36.78871602236994 -121.42735169032998) bank190524 +190525 POINT(37.06544868877058 -123.35437345366472) bank190525 +190526 POINT(37.312221257790924 -123.39739833141368) bank190526 +190527 POINT(38.13857525106022 -121.7403235560671) bank190527 +190528 POINT(37.85984758247384 -122.70917308249903) bank190528 +190529 POINT(37.69945460973929 -123.33752243620806) bank190529 +190530 POINT(38.18452702621975 -123.33992343013745) bank190530 +190531 POINT(37.49613566853506 -122.78766081740677) bank190531 +190532 POINT(38.01335410189432 -121.93251177857978) bank190532 +190533 POINT(38.73555314235686 -123.20764364645481) bank190533 +190534 POINT(37.97487292166569 -122.81942833822386) bank190534 +190535 POINT(38.32181388061514 -123.00066298008842) bank190535 +190536 POINT(36.83873997852462 -121.9045009789575) bank190536 +190537 POINT(36.85387504342668 -122.64902889560432) bank190537 +190538 POINT(38.388614302792924 -121.95755370251106) bank190538 +190539 POINT(38.133014702867875 -122.05852884961375) bank190539 +190540 POINT(37.38026233379327 -122.44669351080559) bank190540 +190541 POINT(37.228002678204994 -122.17823012213181) bank190541 +190542 POINT(38.23680972190854 -122.94939997285172) bank190542 +190543 POINT(37.0495766784419 -123.03434219455129) bank190543 +190544 POINT(37.53951968360439 -122.06827837669093) bank190544 +190545 POINT(36.9001544805848 -123.27171658674023) bank190545 +190546 POINT(37.674723298950454 -123.24363430576658) bank190546 +190547 POINT(38.32307955795553 -123.36774953068308) bank190547 +190548 POINT(37.886320307088106 -123.14544644667156) bank190548 +190549 POINT(38.378632062462835 -121.50062210173928) bank190549 +190550 POINT(38.719418277261155 -122.9268129374205) bank190550 +190551 POINT(37.58997961618831 -122.48276227181061) bank190551 +190552 POINT(38.416905694421025 -121.68328651383223) bank190552 +190553 POINT(38.36528588923116 -122.4298181468074) bank190553 +190554 POINT(38.243811016441065 -121.9730927046078) bank190554 +190555 POINT(37.65426027273757 -123.15457485906883) bank190555 +190556 POINT(37.31003734698972 -123.12632206222051) bank190556 +190557 POINT(37.97507314436304 -123.14901277782369) bank190557 +190558 POINT(37.934649519810435 -122.32336417395166) bank190558 +190559 POINT(37.703471974714034 -123.13854602506865) bank190559 +190560 POINT(36.86601446749038 -122.32260438484933) bank190560 +190561 POINT(37.694809495609135 -122.81762200542411) bank190561 +190562 POINT(37.506435618968986 -123.36871148743835) bank190562 +190563 POINT(38.26985775769494 -121.93594583188897) bank190563 +190564 POINT(37.479901913812476 -121.62496097335426) bank190564 +190565 POINT(36.82317620559036 -121.48432760831456) bank190565 +190566 POINT(36.928323804115756 -121.98309025474954) bank190566 +190567 POINT(38.49809687843419 -122.4377477481687) bank190567 +190568 POINT(37.124097316525045 -122.3134576034715) bank190568 +190569 POINT(37.56811888755738 -122.51985935080067) bank190569 +190570 POINT(38.62779311217767 -122.1572040025073) bank190570 +190571 POINT(38.26969863736701 -123.30650493994243) bank190571 +190572 POINT(38.12208650118203 -121.43063460777523) bank190572 +190573 POINT(38.17716381851996 -121.42909179526862) bank190573 +190574 POINT(37.325448384625545 -121.5323109261192) bank190574 +190575 POINT(38.281168204084096 -122.90697468044169) bank190575 +190576 POINT(36.8582101835291 -121.9250369127379) bank190576 +190577 POINT(38.23073859776347 -123.29448585995866) bank190577 +190578 POINT(37.99683182600079 -121.5016904258422) bank190578 +190579 POINT(38.370746398060696 -121.44867667686275) bank190579 +190580 POINT(37.01469675515769 -122.96078755604566) bank190580 +190581 POINT(37.38296619908184 -122.85361964744547) bank190581 +190582 POINT(38.045556813312736 -122.29697145089249) bank190582 +190583 POINT(37.74838886095418 -122.20463303673452) bank190583 +190584 POINT(38.31111660764433 -122.62142711153106) bank190584 +190585 POINT(37.1384172969491 -122.96943593204838) bank190585 +190586 POINT(37.547489485951935 -123.17627382770893) bank190586 +190587 POINT(37.41246833601531 -122.10406774896593) bank190587 +190588 POINT(37.17990489372081 -122.39097924451798) bank190588 +190589 POINT(37.46108992292061 -122.72734131256773) bank190589 +190590 POINT(37.990513552445194 -122.03816325776312) bank190590 +190591 POINT(37.39380393466949 -121.49339955478825) bank190591 +190592 POINT(38.16254976155943 -122.27047422568232) bank190592 +190593 POINT(38.65335869263066 -121.62853389095257) bank190593 +190594 POINT(38.50010700079506 -122.53936803289155) bank190594 +190595 POINT(37.28978200529551 -122.608920663911) bank190595 +190596 POINT(38.737109811547704 -123.18987071219541) bank190596 +190597 POINT(38.657711942415915 -121.78553486504782) bank190597 +190598 POINT(37.810486947507854 -123.0194334194069) bank190598 +190599 POINT(37.94837671658123 -121.89713667942813) bank190599 +190600 POINT(38.49205058437633 -121.66310178058453) bank190600 +190601 POINT(38.56556029527513 -122.35786170433389) bank190601 +190602 POINT(38.75854368288955 -121.68499331345673) bank190602 +190603 POINT(37.22435318738456 -122.60647115012868) bank190603 +190604 POINT(38.354311136140495 -122.47703467472512) bank190604 +190605 POINT(37.11590841611473 -121.44522366132692) bank190605 +190606 POINT(37.64330638003517 -122.51437057746531) bank190606 +190607 POINT(37.664444880346124 -123.16948599078252) bank190607 +190608 POINT(37.41179253194614 -121.80417543348663) bank190608 +190609 POINT(37.23005420010695 -121.83571711212444) bank190609 +190610 POINT(38.176824242752836 -121.54541895388513) bank190610 +190611 POINT(36.81675993355236 -122.59120155427262) bank190611 +190612 POINT(37.56532100926137 -122.9049683025117) bank190612 +190613 POINT(36.964959180647426 -122.21139317527562) bank190613 +190614 POINT(36.84023602711246 -122.55749074831826) bank190614 +190615 POINT(38.453236843023085 -122.21730511024076) bank190615 +190616 POINT(38.61364466522981 -123.34457427250975) bank190616 +190617 POINT(38.193898629830564 -121.56225193084866) bank190617 +190618 POINT(37.90479808082698 -122.23271203078876) bank190618 +190619 POINT(38.05921458882049 -122.66070502857065) bank190619 +190620 POINT(38.50687607957466 -122.66143892641136) bank190620 +190621 POINT(38.29988122516188 -122.94725160091646) bank190621 +190622 POINT(37.68527159171698 -122.8630468259306) bank190622 +190623 POINT(37.80717682165646 -122.52968513111337) bank190623 +190624 POINT(36.94521510104296 -123.0411114215746) bank190624 +190625 POINT(38.50689775117418 -121.9697127855634) bank190625 +190626 POINT(37.08930467910927 -121.70906055042016) bank190626 +190627 POINT(37.40295249267198 -123.0671126946322) bank190627 +190628 POINT(36.979701199991375 -121.60163676796576) bank190628 +190629 POINT(38.41979978819926 -121.44327650573582) bank190629 +190630 POINT(37.16189533080813 -123.40636818285421) bank190630 +190631 POINT(38.217167034130256 -123.05353280922125) bank190631 +190632 POINT(37.74495384001095 -122.90200946078717) bank190632 +190633 POINT(36.98715930723955 -122.06995215014474) bank190633 +190634 POINT(38.28602212036955 -122.21118165409904) bank190634 +190635 POINT(38.1455111710447 -122.59703772529522) bank190635 +190636 POINT(36.97968358676331 -122.90392729099175) bank190636 +190637 POINT(36.9224952538914 -121.86926306528396) bank190637 +190638 POINT(36.84794820909515 -123.24107463792612) bank190638 +190639 POINT(37.31811689900555 -123.14131395537241) bank190639 +190640 POINT(37.116789585310656 -123.08509790900337) bank190640 +190641 POINT(36.791665805403625 -121.64086714904079) bank190641 +190642 POINT(37.17766655166759 -121.48995408570683) bank190642 +190643 POINT(36.82507380043298 -122.56415025746736) bank190643 +190644 POINT(36.84981138780625 -121.54002013791464) bank190644 +190645 POINT(38.017132086731124 -121.46859575590474) bank190645 +190646 POINT(37.38543784768581 -122.51808704780925) bank190646 +190647 POINT(36.98819013325227 -121.53447872547041) bank190647 +190648 POINT(38.19349150928753 -122.0909492299266) bank190648 +190649 POINT(37.18059985392436 -123.27492208700899) bank190649 +190650 POINT(38.39680362773612 -123.00328064825865) bank190650 +190651 POINT(38.64062817789693 -123.37662242963293) bank190651 +190652 POINT(38.42442170665133 -122.01770858361252) bank190652 +190653 POINT(38.71850568680592 -122.82765239373803) bank190653 +190654 POINT(37.22694761642695 -123.32133397403011) bank190654 +190655 POINT(36.846187963485264 -122.33382509240461) bank190655 +190656 POINT(36.92619265307074 -122.92570012188575) bank190656 +190657 POINT(37.597587170162264 -121.53006569030454) bank190657 +190658 POINT(37.39283294042154 -121.8562323018102) bank190658 +190659 POINT(36.952199545912286 -122.42569041700546) bank190659 +190660 POINT(38.29840017239556 -122.60064828255895) bank190660 +190661 POINT(38.646525950665136 -122.42748400769689) bank190661 +190662 POINT(38.030089770629395 -121.82993413548438) bank190662 +190663 POINT(37.528463375082865 -121.42051021144485) bank190663 +190664 POINT(37.12211559850714 -122.27625548835488) bank190664 +190665 POINT(36.93782624290154 -122.30066905454724) bank190665 +190666 POINT(37.068778830916685 -122.96811381001227) bank190666 +190667 POINT(37.17891758038905 -121.93176940074537) bank190667 +190668 POINT(38.708262589957094 -121.978307885761) bank190668 +190669 POINT(36.92716618796956 -122.16033044246315) bank190669 +190670 POINT(37.078410967917264 -122.39547666806271) bank190670 +190671 POINT(37.08393864961972 -123.16593323066223) bank190671 +190672 POINT(38.06330366850994 -122.85774136742903) bank190672 +190673 POINT(37.59941582377563 -122.12464694997591) bank190673 +190674 POINT(36.84397156574715 -123.07519398414962) bank190674 +190675 POINT(38.531100626773785 -123.02373622398558) bank190675 +190676 POINT(38.72708855121291 -123.30429203784132) bank190676 +190677 POINT(38.47994956440154 -121.56176845304022) bank190677 +190678 POINT(37.3975185831402 -122.81855592909186) bank190678 +190679 POINT(37.92330690248926 -123.41389313515548) bank190679 +190680 POINT(38.0717544295825 -122.86968534566944) bank190680 +190681 POINT(37.707000613150235 -121.87154928267806) bank190681 +190682 POINT(37.52662328279185 -122.37359374111948) bank190682 +190683 POINT(38.351776625786094 -122.43579265369573) bank190683 +190684 POINT(37.94006455124631 -122.59655963955007) bank190684 +190685 POINT(38.60734535057628 -123.38269456079527) bank190685 +190686 POINT(38.718099775832705 -122.28540762053228) bank190686 +190687 POINT(37.57084886516255 -122.12555367554111) bank190687 +190688 POINT(37.05253298863058 -123.38693550583483) bank190688 +190689 POINT(38.479967799943665 -122.40625758303128) bank190689 +190690 POINT(37.42404085041721 -122.4506860800417) bank190690 +190691 POINT(37.76383190123161 -122.78570671329385) bank190691 +190692 POINT(36.79534193958184 -122.29681700255122) bank190692 +190693 POINT(37.29118435799934 -122.24948839084252) bank190693 +190694 POINT(38.19527949070815 -121.59563902624765) bank190694 +190695 POINT(36.97409317451798 -122.12526624312073) bank190695 +190696 POINT(37.53597974123068 -122.47322488077073) bank190696 +190697 POINT(37.659462878888625 -122.43509772025321) bank190697 +190698 POINT(37.465191929914006 -122.19243782638188) bank190698 +190699 POINT(37.84418155337484 -122.05651316052912) bank190699 +190700 POINT(37.57283232970979 -123.09120147770491) bank190700 +190701 POINT(37.69006893511603 -122.5699440399917) bank190701 +190702 POINT(36.88838666265929 -122.58323671310119) bank190702 +190703 POINT(37.770681889890895 -122.43394610934358) bank190703 +190704 POINT(38.54451016675152 -123.30640908320359) bank190704 +190705 POINT(38.48229511003605 -122.28355069201864) bank190705 +190706 POINT(37.72001491347864 -121.69407658398012) bank190706 +190707 POINT(38.64225331683327 -121.5175753590617) bank190707 +190708 POINT(37.51431310291279 -122.93915231165893) bank190708 +190709 POINT(38.371885335910065 -121.95218432872815) bank190709 +190710 POINT(37.7075741316258 -123.18330966465281) bank190710 +190711 POINT(37.92066585822883 -121.82894071780188) bank190711 +190712 POINT(38.17236640199975 -122.75195749120007) bank190712 +190713 POINT(37.186978887725644 -122.85945810721958) bank190713 +190714 POINT(38.39455625053235 -122.8608619583257) bank190714 +190715 POINT(38.13182435246207 -123.17501184735791) bank190715 +190716 POINT(37.05245495371584 -123.18724701373291) bank190716 +190717 POINT(37.96312289757015 -121.72898250504839) bank190717 +190718 POINT(38.26215008224034 -122.73139254273092) bank190718 +190719 POINT(37.94178343848148 -123.11357670683215) bank190719 +190720 POINT(37.620414455018604 -122.87974149123376) bank190720 +190721 POINT(37.55933673334898 -122.68790600727122) bank190721 +190722 POINT(37.47184849057762 -122.59623314813234) bank190722 +190723 POINT(38.587960157325455 -121.49554023695443) bank190723 +190724 POINT(38.15281627293689 -121.85892628834043) bank190724 +190725 POINT(36.95882248580929 -121.50317554594169) bank190725 +190726 POINT(38.70224272280891 -121.855362667496) bank190726 +190727 POINT(38.75736866873611 -123.20652574935276) bank190727 +190728 POINT(38.444273547958595 -121.8889988638622) bank190728 +190729 POINT(37.94084977126381 -123.32727407890297) bank190729 +190730 POINT(38.37723541563696 -123.00020892453617) bank190730 +190731 POINT(37.61287403534457 -123.41498544727233) bank190731 +190732 POINT(36.88708373322665 -121.75881301138598) bank190732 +190733 POINT(38.04123540416007 -122.31124574593541) bank190733 +190734 POINT(37.97717827649481 -121.9540539559233) bank190734 +190735 POINT(37.95525820392768 -121.81791677405486) bank190735 +190736 POINT(37.59699042613513 -121.68907215333032) bank190736 +190737 POINT(38.39566883216562 -122.3837597360278) bank190737 +190738 POINT(37.141770379241585 -122.31822755450237) bank190738 +190739 POINT(37.78248269325798 -123.00321716670648) bank190739 +190740 POINT(38.20230976628759 -123.15323497831409) bank190740 +190741 POINT(37.6687383967486 -122.44597742239027) bank190741 +190742 POINT(37.73014224443381 -121.90331342792224) bank190742 +190743 POINT(38.455832400649435 -122.40507515731369) bank190743 +190744 POINT(38.14055626501527 -122.37239082736356) bank190744 +190745 POINT(38.46726902255059 -123.07143775483466) bank190745 +190746 POINT(37.65893132698458 -123.14510427948366) bank190746 +190747 POINT(38.70939903845436 -122.47240622289146) bank190747 +190748 POINT(38.74612964738803 -121.8809281931121) bank190748 +190749 POINT(37.149251435807216 -121.44290604966243) bank190749 +190750 POINT(38.31831751836262 -121.64930657632098) bank190750 +190751 POINT(36.928109580530055 -121.93664229911246) bank190751 +190752 POINT(37.13898376689692 -122.43228996576056) bank190752 +190753 POINT(37.69120273773978 -123.13371286600814) bank190753 +190754 POINT(37.803822008185755 -123.34139472286165) bank190754 +190755 POINT(37.851190871611934 -123.16991769236265) bank190755 +190756 POINT(37.28568851100724 -122.38666475947758) bank190756 +190757 POINT(37.91218034517411 -121.90306344072414) bank190757 +190758 POINT(37.55863703032718 -122.89098941214539) bank190758 +190759 POINT(38.13987604018955 -122.50521815922366) bank190759 +190760 POINT(36.93765489017116 -121.80292320527381) bank190760 +190761 POINT(36.86689088266718 -123.07106115964567) bank190761 +190762 POINT(36.968147042561355 -122.47899926242059) bank190762 +190763 POINT(37.77791981561178 -122.91002843229303) bank190763 +190764 POINT(36.8272133402908 -123.2438712967905) bank190764 +190765 POINT(37.13045451220672 -121.59294001653284) bank190765 +190766 POINT(37.28031749027285 -123.41574452277483) bank190766 +190767 POINT(38.00267814204526 -122.4588820430939) bank190767 +190768 POINT(37.63038320163463 -121.70481349584257) bank190768 +190769 POINT(38.263403151985855 -121.92807972955748) bank190769 +190770 POINT(37.23603743201278 -123.07129044551081) bank190770 +190771 POINT(37.52052356923477 -121.6496099653576) bank190771 +190772 POINT(38.114913570798116 -122.15058593520804) bank190772 +190773 POINT(37.181938151097285 -122.72508452325587) bank190773 +190774 POINT(37.49114640492034 -123.39857902821636) bank190774 +190775 POINT(37.1053328238316 -121.5917374211944) bank190775 +190776 POINT(37.60585732347974 -121.62823551853829) bank190776 +190777 POINT(38.4630129294925 -123.31803980845886) bank190777 +190778 POINT(37.024860398843884 -122.19485661526377) bank190778 +190779 POINT(37.08155683845089 -123.09713455643569) bank190779 +190780 POINT(38.31236321388076 -121.71368155742948) bank190780 +190781 POINT(37.030812140536035 -123.34017346041918) bank190781 +190782 POINT(37.92887271990384 -123.28536751643357) bank190782 +190783 POINT(37.3661106538647 -122.80231343121223) bank190783 +190784 POINT(38.1266935294181 -121.92950846821563) bank190784 +190785 POINT(37.42095293931668 -122.80148744292876) bank190785 +190786 POINT(38.329533660558596 -122.5198425464695) bank190786 +190787 POINT(37.45430167358735 -123.33997834798768) bank190787 +190788 POINT(37.206149848247584 -121.78291408009143) bank190788 +190789 POINT(37.08841515284985 -122.34455090224947) bank190789 +190790 POINT(38.47311543283356 -121.8445624218731) bank190790 +190791 POINT(38.597114123125564 -122.2493540132908) bank190791 +190792 POINT(38.315379015680655 -121.69462591627955) bank190792 +190793 POINT(37.53733372893328 -122.57054863977864) bank190793 +190794 POINT(37.15528004880731 -122.35100151141259) bank190794 +190795 POINT(36.90507262452805 -123.1148886774889) bank190795 +190796 POINT(38.400855713667525 -123.35676647028718) bank190796 +190797 POINT(38.1852350523908 -121.96454474000815) bank190797 +190798 POINT(36.832142572191586 -123.00412019334655) bank190798 +190799 POINT(37.370012897856135 -122.33951563327021) bank190799 +190800 POINT(38.274815463373194 -121.91857043697655) bank190800 +190801 POINT(37.15057945364333 -121.6248043360768) bank190801 +190802 POINT(37.00538087148902 -122.45045138019623) bank190802 +190803 POINT(37.30626220188908 -123.1527966034903) bank190803 +190804 POINT(36.92704652425931 -122.43283039928906) bank190804 +190805 POINT(37.46029390766061 -122.33978279307193) bank190805 +190806 POINT(38.28394071724245 -121.81229747751472) bank190806 +190807 POINT(37.17720428608128 -122.79311785630932) bank190807 +190808 POINT(37.950585280861716 -122.92780849336408) bank190808 +190809 POINT(38.61957500736726 -121.86351657674872) bank190809 +190810 POINT(37.23562786361475 -121.82612437531257) bank190810 +190811 POINT(38.567350100992044 -122.86398233701836) bank190811 +190812 POINT(38.19454336357309 -122.97665586015806) bank190812 +190813 POINT(38.02746751806984 -121.87287268047349) bank190813 +190814 POINT(38.54346648455648 -121.62094486961362) bank190814 +190815 POINT(36.8197491931726 -123.10569325315649) bank190815 +190816 POINT(37.21011497570377 -121.82238764767071) bank190816 +190817 POINT(38.54416307950126 -122.32856940280621) bank190817 +190818 POINT(37.46521920581876 -122.62792734778115) bank190818 +190819 POINT(37.33053911466957 -122.06576732513868) bank190819 +190820 POINT(38.57627321494633 -122.15757286840113) bank190820 +190821 POINT(38.19233435775602 -123.38416427555065) bank190821 +190822 POINT(37.6773488704425 -123.25251652319743) bank190822 +190823 POINT(37.23493840102375 -123.37318226814014) bank190823 +190824 POINT(37.43774569086847 -121.84927332690526) bank190824 +190825 POINT(38.188819093004554 -123.29651409482575) bank190825 +190826 POINT(36.87788083132649 -122.78492481844741) bank190826 +190827 POINT(36.89837358431854 -122.22265772658069) bank190827 +190828 POINT(37.41376323501358 -123.08157056851715) bank190828 +190829 POINT(37.63532464989386 -123.18379470081835) bank190829 +190830 POINT(36.78103160268058 -123.17961962502554) bank190830 +190831 POINT(37.272029578089516 -122.65682050461078) bank190831 +190832 POINT(37.29193767121987 -121.85544973600679) bank190832 +190833 POINT(37.714765170579646 -122.71138433429083) bank190833 +190834 POINT(37.03421013996919 -122.08339391862259) bank190834 +190835 POINT(37.46660969520108 -121.91414137266771) bank190835 +190836 POINT(37.193367428090056 -122.2379517411977) bank190836 +190837 POINT(38.44616429817079 -121.56739572981432) bank190837 +190838 POINT(37.91141121331265 -122.25745597351319) bank190838 +190839 POINT(37.162880371247795 -123.02999101012927) bank190839 +190840 POINT(38.16068050682161 -122.88998666806289) bank190840 +190841 POINT(38.07978543180326 -122.89792307669924) bank190841 +190842 POINT(37.89589878846924 -122.49497271265744) bank190842 +190843 POINT(37.14514359969041 -122.33352447068114) bank190843 +190844 POINT(37.59243914075367 -122.92144340575386) bank190844 +190845 POINT(37.37664889435482 -122.24919522121128) bank190845 +190846 POINT(37.435295633775034 -122.26123197497803) bank190846 +190847 POINT(38.688954618439844 -122.46695997310367) bank190847 +190848 POINT(36.870391240222624 -122.11399240020864) bank190848 +190849 POINT(38.650704305164304 -121.67752728278256) bank190849 +190850 POINT(37.38429623907708 -122.23081231927594) bank190850 +190851 POINT(37.25385732428654 -121.73113553248444) bank190851 +190852 POINT(38.171226638845276 -122.4682864269961) bank190852 +190853 POINT(37.79381733251334 -121.63390117744154) bank190853 +190854 POINT(37.41009482703223 -123.01165705887203) bank190854 +190855 POINT(38.74177575948901 -122.53547408446157) bank190855 +190856 POINT(37.41018524006973 -121.4207717961026) bank190856 +190857 POINT(38.19272785330582 -123.01383922986682) bank190857 +190858 POINT(36.87722562059947 -122.04668134350511) bank190858 +190859 POINT(37.282264771206655 -122.35137512359852) bank190859 +190860 POINT(38.47760540655505 -123.37680931556923) bank190860 +190861 POINT(37.94258033877472 -123.23222701065333) bank190861 +190862 POINT(37.649085655249536 -122.37337196298563) bank190862 +190863 POINT(38.58526786688685 -122.19982655434212) bank190863 +190864 POINT(38.14153271252481 -122.82466927332536) bank190864 +190865 POINT(36.81418675786872 -121.93757552889193) bank190865 +190866 POINT(38.4308018320591 -121.71204995656655) bank190866 +190867 POINT(37.73561812736201 -123.21815758284214) bank190867 +190868 POINT(38.368836531945846 -121.72991143638775) bank190868 +190869 POINT(38.680654066290934 -121.5494411979613) bank190869 +190870 POINT(38.255704604800336 -123.33293586954737) bank190870 +190871 POINT(37.019892547651374 -121.81562662057776) bank190871 +190872 POINT(37.45852475263472 -122.15786596474591) bank190872 +190873 POINT(37.69235667811771 -121.54260136876452) bank190873 +190874 POINT(38.626859607905914 -122.25722071557438) bank190874 +190875 POINT(37.32350336629086 -122.14790610074832) bank190875 +190876 POINT(37.23500918643472 -122.10237206657483) bank190876 +190877 POINT(36.89397345307669 -122.70540388968125) bank190877 +190878 POINT(36.9717563545712 -122.14506657183219) bank190878 +190879 POINT(37.26671975692338 -123.1939040492618) bank190879 +190880 POINT(36.80508879744468 -121.70370934601773) bank190880 +190881 POINT(37.35021126349984 -122.83067743693442) bank190881 +190882 POINT(38.29985373820065 -123.05254696504012) bank190882 +190883 POINT(38.187720815210135 -121.78627304514468) bank190883 +190884 POINT(37.78206489652305 -121.45436168335408) bank190884 +190885 POINT(38.352106152609174 -123.1307302317975) bank190885 +190886 POINT(38.471514405510135 -123.38624145571833) bank190886 +190887 POINT(37.38904406697286 -121.8314896616419) bank190887 +190888 POINT(38.24377015225641 -122.44231997233722) bank190888 +190889 POINT(38.43941346969537 -122.4624185806333) bank190889 +190890 POINT(37.39733343533936 -121.93406577194645) bank190890 +190891 POINT(38.40530471149374 -121.50399698450316) bank190891 +190892 POINT(36.83695943633283 -122.77741428430674) bank190892 +190893 POINT(36.90581722367176 -123.39232850762158) bank190893 +190894 POINT(37.69195893186804 -121.9359471759203) bank190894 +190895 POINT(37.3484331365137 -122.87554705030188) bank190895 +190896 POINT(37.83518507606583 -122.93992401951496) bank190896 +190897 POINT(38.17012824337609 -121.65532145102496) bank190897 +190898 POINT(36.95971538470362 -122.09571319310746) bank190898 +190899 POINT(38.4495863473637 -122.59552413851797) bank190899 +190900 POINT(38.25590547269492 -122.18498591817814) bank190900 +190901 POINT(37.24000120363669 -121.7329969366003) bank190901 +190902 POINT(37.74807351841066 -122.45517225792021) bank190902 +190903 POINT(38.61578666998058 -121.83317444545568) bank190903 +190904 POINT(38.12013621249545 -122.94567663948344) bank190904 +190905 POINT(37.50181794377057 -123.11958974795758) bank190905 +190906 POINT(37.113734873382214 -122.60672918669889) bank190906 +190907 POINT(37.064084889687 -123.16892783014475) bank190907 +190908 POINT(38.50288968689516 -121.56062031034863) bank190908 +190909 POINT(37.41909420028406 -121.47976872479437) bank190909 +190910 POINT(37.473996613287426 -121.80930677289228) bank190910 +190911 POINT(37.34297368971234 -121.9297329133195) bank190911 +190912 POINT(37.26053781827599 -122.09338495426287) bank190912 +190913 POINT(37.33286857242368 -122.7228644049639) bank190913 +190914 POINT(38.625859326933764 -121.48341522810455) bank190914 +190915 POINT(37.3636337810143 -121.55780879749378) bank190915 +190916 POINT(37.28501018545707 -121.52920139938286) bank190916 +190917 POINT(38.54019964965008 -122.26401152202475) bank190917 +190918 POINT(37.02787184856238 -122.32791611535167) bank190918 +190919 POINT(37.335648660497505 -121.85308341939512) bank190919 +190920 POINT(37.58459469736262 -123.21630903698284) bank190920 +190921 POINT(37.64403408232605 -123.25124876261022) bank190921 +190922 POINT(36.849753990918956 -122.04325585693444) bank190922 +190923 POINT(37.6401736302255 -122.71690593926971) bank190923 +190924 POINT(37.95780086410085 -123.15634537092916) bank190924 +190925 POINT(36.94755603010123 -123.32238345202391) bank190925 +190926 POINT(37.46530876793084 -122.62947824697844) bank190926 +190927 POINT(38.39118916666445 -121.78095508161006) bank190927 +190928 POINT(38.26159162714056 -122.84853208272325) bank190928 +190929 POINT(38.0207339105571 -122.34184472985183) bank190929 +190930 POINT(38.173334494691204 -123.2530536225128) bank190930 +190931 POINT(38.13400455133107 -122.23357720103641) bank190931 +190932 POINT(37.4300686126458 -122.80202826329362) bank190932 +190933 POINT(37.72582423748125 -121.93406585141413) bank190933 +190934 POINT(37.92180308818726 -121.8200468636068) bank190934 +190935 POINT(36.93879596967142 -123.22726053318398) bank190935 +190936 POINT(37.20959601784794 -122.83809764203646) bank190936 +190937 POINT(37.553751981766844 -121.46357046042083) bank190937 +190938 POINT(37.55841926556503 -122.27604217709995) bank190938 +190939 POINT(37.05274293976366 -122.22170728515472) bank190939 +190940 POINT(37.90677597481503 -121.96511484692225) bank190940 +190941 POINT(37.62333693584067 -122.51445857545008) bank190941 +190942 POINT(37.92159175309805 -123.18428498346353) bank190942 +190943 POINT(38.297659524293884 -122.51067707240465) bank190943 +190944 POINT(37.86746352814819 -122.33528226008849) bank190944 +190945 POINT(36.844839800004905 -122.83080425346179) bank190945 +190946 POINT(38.71847082485867 -121.71952020710053) bank190946 +190947 POINT(38.765887842621396 -121.7201437237101) bank190947 +190948 POINT(37.981045306009676 -121.42426487785761) bank190948 +190949 POINT(36.807564185854204 -121.42120797649606) bank190949 +190950 POINT(37.42208158602993 -121.6941436351215) bank190950 +190951 POINT(38.415730334211005 -123.06496006551754) bank190951 +190952 POINT(37.30744466725344 -121.66023765023205) bank190952 +190953 POINT(37.91579053914994 -121.97946092936449) bank190953 +190954 POINT(37.675789549773484 -122.82219131386279) bank190954 +190955 POINT(38.73833568614235 -121.78065021756898) bank190955 +190956 POINT(37.58657061871602 -122.79724596892154) bank190956 +190957 POINT(37.45318802663621 -122.00442982487182) bank190957 +190958 POINT(36.88630751394391 -122.63649881609896) bank190958 +190959 POINT(38.55312179069779 -121.92883161736707) bank190959 +190960 POINT(37.88616703437286 -123.23245804150119) bank190960 +190961 POINT(37.37200741494264 -123.12224296066965) bank190961 +190962 POINT(37.44696551923435 -122.52466554551042) bank190962 +190963 POINT(37.438050866636495 -122.25368964852402) bank190963 +190964 POINT(38.72501361262636 -122.91229773917874) bank190964 +190965 POINT(37.82986329129174 -122.51964053742148) bank190965 +190966 POINT(36.861548369687995 -121.9291798140758) bank190966 +190967 POINT(38.654849891231635 -122.34056177592856) bank190967 +190968 POINT(38.29936406096688 -122.71948429661964) bank190968 +190969 POINT(37.89321301953583 -123.09383507576247) bank190969 +190970 POINT(37.10678915112803 -122.00466562839563) bank190970 +190971 POINT(36.8029460713758 -122.60120590742845) bank190971 +190972 POINT(38.62464738987475 -121.7864536616212) bank190972 +190973 POINT(38.44276263549075 -122.06656192481248) bank190973 +190974 POINT(38.73106994051777 -122.0361326994788) bank190974 +190975 POINT(37.20166355326318 -122.48744790859767) bank190975 +190976 POINT(37.322804823416355 -121.93886647145949) bank190976 +190977 POINT(38.03859011133808 -122.53159657474522) bank190977 +190978 POINT(37.327858089707036 -121.84407274750605) bank190978 +190979 POINT(37.41003858046285 -123.32762653201391) bank190979 +190980 POINT(37.579805058520016 -123.34169314137581) bank190980 +190981 POINT(37.074950893525504 -123.00513333343261) bank190981 +190982 POINT(36.85923477209917 -123.15104498438376) bank190982 +190983 POINT(38.60049803476394 -122.85216333878516) bank190983 +190984 POINT(37.28124297964683 -123.41061803067304) bank190984 +190985 POINT(38.058383774790656 -122.04046338667187) bank190985 +190986 POINT(38.51016884619147 -121.60821726905878) bank190986 +190987 POINT(38.262110645332854 -123.33347828039953) bank190987 +190988 POINT(37.1086437282083 -122.7060764916617) bank190988 +190989 POINT(36.92971741512561 -121.69854945047906) bank190989 +190990 POINT(38.0769028153822 -122.1968622952738) bank190990 +190991 POINT(37.78137927595568 -122.76252170607414) bank190991 +190992 POINT(36.874630392729365 -123.21524552450134) bank190992 +190993 POINT(37.938125628891726 -123.12336666363666) bank190993 +190994 POINT(37.67536071245508 -122.72264253851012) bank190994 +190995 POINT(38.452446602569516 -122.88343241658289) bank190995 +190996 POINT(37.00937283385367 -122.41708380594513) bank190996 +190997 POINT(38.72950410344315 -121.59501424490644) bank190997 +190998 POINT(36.95254211620585 -123.21949026728993) bank190998 +190999 POINT(38.4982262019413 -122.77447668783472) bank190999 +191000 POINT(38.00161115030584 -123.32404030253197) bank191000 +191001 POINT(37.74602734590387 -122.36287714870893) bank191001 +191002 POINT(38.61676452345225 -123.20245336548422) bank191002 +191003 POINT(38.655937432546786 -122.30689168371066) bank191003 +191004 POINT(37.39874522091667 -122.00768870022362) bank191004 +191005 POINT(36.80929786467455 -122.21082714122134) bank191005 +191006 POINT(37.90963042176313 -121.6002680861579) bank191006 +191007 POINT(38.08260092598175 -122.8708560014518) bank191007 +191008 POINT(37.06733394353801 -122.62240943513181) bank191008 +191009 POINT(37.04079455867468 -122.71489062720626) bank191009 +191010 POINT(38.54068523457458 -122.27335020970632) bank191010 +191011 POINT(36.92592938371764 -121.63360757208882) bank191011 +191012 POINT(38.04160181212683 -122.58678459192816) bank191012 +191013 POINT(37.91121044405024 -122.27466520097086) bank191013 +191014 POINT(37.127223408545454 -121.45067542608258) bank191014 +191015 POINT(37.35979768477528 -122.84405700506414) bank191015 +191016 POINT(37.82620579461943 -122.00562564597975) bank191016 +191017 POINT(38.02579413296094 -122.8302892956699) bank191017 +191018 POINT(37.4694028561579 -123.3211046940759) bank191018 +191019 POINT(37.27850248214791 -121.90182110779037) bank191019 +191020 POINT(38.10983191411538 -122.24155459479947) bank191020 +191021 POINT(38.22808263481319 -121.85967744628492) bank191021 +191022 POINT(37.05775989680978 -121.61081875397885) bank191022 +191023 POINT(38.27207121984077 -121.86492436970858) bank191023 +191024 POINT(38.60573996352471 -121.69182923461423) bank191024 +191025 POINT(38.45447005194298 -123.37769502302658) bank191025 +191026 POINT(38.29667536020241 -123.22366975180836) bank191026 +191027 POINT(37.40457741141545 -122.8328022457317) bank191027 +191028 POINT(37.701816694566084 -122.91114375188104) bank191028 +191029 POINT(38.76652791999654 -121.67031876999498) bank191029 +191030 POINT(37.25855325130625 -122.47878060140192) bank191030 +191031 POINT(38.230069774600715 -123.27625612546807) bank191031 +191032 POINT(36.81833347799143 -121.93556179590537) bank191032 +191033 POINT(37.10769700189412 -121.43448180020978) bank191033 +191034 POINT(36.9637614232997 -122.20610623861086) bank191034 +191035 POINT(37.20258602176554 -123.03566013840441) bank191035 +191036 POINT(37.821037793739706 -122.00769205192904) bank191036 +191037 POINT(37.15624506965158 -122.85293347889763) bank191037 +191038 POINT(38.396585591134446 -122.86071817290126) bank191038 +191039 POINT(36.96034348900971 -122.60133708455535) bank191039 +191040 POINT(38.35312759810351 -123.08986796627299) bank191040 +191041 POINT(38.170300224226345 -123.4029885035486) bank191041 +191042 POINT(37.80110103464191 -121.85376870873795) bank191042 +191043 POINT(36.87959930091809 -121.81843491623049) bank191043 +191044 POINT(36.895575617179354 -122.38270575404579) bank191044 +191045 POINT(37.419524334758265 -122.89418462618399) bank191045 +191046 POINT(37.008962985465466 -122.52461515414264) bank191046 +191047 POINT(37.87346834245826 -122.55960342936035) bank191047 +191048 POINT(37.48557763905221 -122.8271289006813) bank191048 +191049 POINT(38.59670842657562 -122.06281597832883) bank191049 +191050 POINT(38.532587246584974 -122.12174349164945) bank191050 +191051 POINT(38.463407757004816 -122.28800023449783) bank191051 +191052 POINT(37.817186507855034 -122.97411728502458) bank191052 +191053 POINT(38.48075289435667 -121.71895904978115) bank191053 +191054 POINT(36.79011198247553 -122.86238024817766) bank191054 +191055 POINT(37.583108444170236 -121.86783390091215) bank191055 +191056 POINT(36.81811241447756 -121.56308029607446) bank191056 +191057 POINT(37.94796235461501 -122.22363972242451) bank191057 +191058 POINT(38.25813207057389 -122.57816241906835) bank191058 +191059 POINT(38.1925457668774 -121.83354057184471) bank191059 +191060 POINT(38.05532188442165 -121.79526155610193) bank191060 +191061 POINT(37.604581003384055 -122.48389471957204) bank191061 +191062 POINT(38.52386323753355 -122.57862524779755) bank191062 +191063 POINT(38.166565219876816 -122.53432755249436) bank191063 +191064 POINT(36.85441171477925 -122.84288997280689) bank191064 +191065 POINT(38.378781461494455 -123.13769165307225) bank191065 +191066 POINT(38.73574387034758 -121.64496097255606) bank191066 +191067 POINT(37.90648269176291 -122.268197121046) bank191067 +191068 POINT(37.76609202555252 -123.40269194421778) bank191068 +191069 POINT(38.32493576288023 -123.10292493171623) bank191069 +191070 POINT(38.752896819189296 -122.7845808527056) bank191070 +191071 POINT(38.4650781151175 -123.0280481166763) bank191071 +191072 POINT(38.715605541615574 -122.20746300185587) bank191072 +191073 POINT(37.85467801497552 -122.62116501626762) bank191073 +191074 POINT(37.84720675430901 -122.5460489810295) bank191074 +191075 POINT(38.43914124263171 -122.2241834581744) bank191075 +191076 POINT(36.785958259453565 -121.45117217888352) bank191076 +191077 POINT(37.569326542563275 -122.38695983295919) bank191077 +191078 POINT(37.23751741516611 -123.14577706555603) bank191078 +191079 POINT(38.12979629712572 -122.71027695188155) bank191079 +191080 POINT(38.023608073896526 -121.6443015706059) bank191080 +191081 POINT(37.50957471181147 -123.41654083332342) bank191081 +191082 POINT(38.64829182687748 -122.69434897622803) bank191082 +191083 POINT(36.98738783226247 -122.1629376432269) bank191083 +191084 POINT(38.23566253713525 -122.60859726403467) bank191084 +191085 POINT(37.52704765432403 -122.98693430986069) bank191085 +191086 POINT(37.26412041921813 -122.66516541417154) bank191086 +191087 POINT(38.13481315588965 -121.97214416191363) bank191087 +191088 POINT(38.42402490696074 -122.62412784003618) bank191088 +191089 POINT(37.62636142559508 -121.7954088059385) bank191089 +191090 POINT(37.416282725683516 -122.52588323249525) bank191090 +191091 POINT(37.454105138056875 -122.83486175032115) bank191091 +191092 POINT(38.47747536545455 -122.89805696684111) bank191092 +191093 POINT(37.0624037243512 -123.33686554354566) bank191093 +191094 POINT(38.513998703721725 -122.97903493344876) bank191094 +191095 POINT(38.48946776052716 -122.18058010914214) bank191095 +191096 POINT(38.67045741743842 -123.12752239113163) bank191096 +191097 POINT(37.642945337153336 -123.32201755256426) bank191097 +191098 POINT(38.600676603379334 -121.93519042037347) bank191098 +191099 POINT(38.441090114356925 -122.49787786085739) bank191099 +191100 POINT(37.13624842383754 -122.9938296350836) bank191100 +191101 POINT(38.17860065061512 -122.50690759921352) bank191101 +191102 POINT(38.18380879565515 -122.32407411920168) bank191102 +191103 POINT(37.97818612577492 -122.5127472821522) bank191103 +191104 POINT(38.52996606712319 -121.51579506473021) bank191104 +191105 POINT(38.060428358877004 -121.43511402665979) bank191105 +191106 POINT(37.0224423089538 -123.07871074202339) bank191106 +191107 POINT(38.36504202781609 -123.33250585003596) bank191107 +191108 POINT(38.27324070395808 -123.31147099504672) bank191108 +191109 POINT(38.60482803562615 -122.82329002006459) bank191109 +191110 POINT(37.64599869440416 -122.63279749731913) bank191110 +191111 POINT(37.2195695736646 -121.51989058516158) bank191111 +191112 POINT(37.30042576935348 -123.05859476880428) bank191112 +191113 POINT(38.181829006534564 -122.10824824870434) bank191113 +191114 POINT(36.871719871417746 -122.6089797215712) bank191114 +191115 POINT(37.20868567334458 -123.16880815368415) bank191115 +191116 POINT(37.29863651425027 -122.52845295685742) bank191116 +191117 POINT(37.652622645376184 -121.82584694117956) bank191117 +191118 POINT(38.01448669561288 -121.77040850681931) bank191118 +191119 POINT(38.23159826886313 -122.63271506680775) bank191119 +191120 POINT(36.837199545322186 -122.77321709325977) bank191120 +191121 POINT(37.68359746626097 -122.31193707374833) bank191121 +191122 POINT(37.64110369007649 -123.35675906610862) bank191122 +191123 POINT(38.11797314900242 -122.62976474394826) bank191123 +191124 POINT(38.45666104404161 -123.07478119709616) bank191124 +191125 POINT(38.03828854001818 -122.23224128845357) bank191125 +191126 POINT(36.87202223444219 -122.1865607280472) bank191126 +191127 POINT(38.51462465952371 -121.60644814950166) bank191127 +191128 POINT(38.41872471997983 -123.08213106858454) bank191128 +191129 POINT(38.454578774702625 -123.26126980818067) bank191129 +191130 POINT(38.00525401818981 -122.19073638170671) bank191130 +191131 POINT(37.154282447654346 -121.59534633902574) bank191131 +191132 POINT(37.8632015362626 -122.42176447317958) bank191132 +191133 POINT(37.56885365246314 -122.73914943876844) bank191133 +191134 POINT(37.92256422764255 -122.85129525707616) bank191134 +191135 POINT(37.108765605370834 -123.35000853250412) bank191135 +191136 POINT(38.105365385276194 -121.74458465428086) bank191136 +191137 POINT(37.721631862578256 -121.43318215572499) bank191137 +191138 POINT(38.382406213100474 -121.54995095609138) bank191138 +191139 POINT(37.39053508501543 -121.9607022844564) bank191139 +191140 POINT(38.228625223163 -122.69885424495062) bank191140 +191141 POINT(37.90773098898497 -121.94895128047901) bank191141 +191142 POINT(38.64682365227617 -122.23114228043855) bank191142 +191143 POINT(38.661737531645876 -123.16452258925047) bank191143 +191144 POINT(38.75747319602974 -122.96924696198057) bank191144 +191145 POINT(38.24013668569839 -121.45912333072522) bank191145 +191146 POINT(38.412103459483276 -122.59527307290617) bank191146 +191147 POINT(37.33796323169409 -121.72511965336504) bank191147 +191148 POINT(37.413433765056155 -121.53954312051077) bank191148 +191149 POINT(37.99557511862415 -121.43045985392146) bank191149 +191150 POINT(37.910579392321985 -123.1474056890214) bank191150 +191151 POINT(37.360377117179446 -123.20161664943227) bank191151 +191152 POINT(37.53382911753822 -122.39592640572145) bank191152 +191153 POINT(37.32771732500073 -121.9280619361391) bank191153 +191154 POINT(37.66012277115236 -123.10615043897036) bank191154 +191155 POINT(38.046064293828465 -121.47640462990753) bank191155 +191156 POINT(38.01973811109374 -122.76911251234134) bank191156 +191157 POINT(38.162303755448875 -121.78633205532064) bank191157 +191158 POINT(38.73303072263037 -122.76707531684278) bank191158 +191159 POINT(37.60855547406931 -121.93144007904598) bank191159 +191160 POINT(38.33711804107626 -122.14253856883232) bank191160 +191161 POINT(37.78935315234644 -122.8782218307553) bank191161 +191162 POINT(38.22648574106849 -122.55194889745312) bank191162 +191163 POINT(37.36200289548123 -122.80927912305464) bank191163 +191164 POINT(37.46705902980128 -122.09576257119001) bank191164 +191165 POINT(38.39072813074617 -123.1892970127331) bank191165 +191166 POINT(38.39539142227972 -121.89831971453198) bank191166 +191167 POINT(38.27692276166085 -122.48611254686794) bank191167 +191168 POINT(37.95350568050323 -122.51117958207168) bank191168 +191169 POINT(38.566203737016245 -122.81321536382956) bank191169 +191170 POINT(38.56268047532011 -122.12141477617386) bank191170 +191171 POINT(38.12649052493588 -121.75236836885449) bank191171 +191172 POINT(37.09235479111843 -122.98783620367216) bank191172 +191173 POINT(38.317933661774106 -122.58032846797607) bank191173 +191174 POINT(37.84546496931992 -123.14822102293266) bank191174 +191175 POINT(37.90740656487293 -122.16385634039791) bank191175 +191176 POINT(38.48537086417211 -122.24628700730953) bank191176 +191177 POINT(37.66942411194621 -121.52787724525679) bank191177 +191178 POINT(38.051923203990995 -121.60425224798543) bank191178 +191179 POINT(38.50429285530937 -123.00887504214309) bank191179 +191180 POINT(37.84303994154559 -122.08365220277287) bank191180 +191181 POINT(37.53692516983214 -123.3764411655592) bank191181 +191182 POINT(37.37130342985721 -122.36955022262873) bank191182 +191183 POINT(37.21483706985986 -121.4631861683794) bank191183 +191184 POINT(37.94479351129148 -121.89534901473213) bank191184 +191185 POINT(38.72077927867495 -121.94423339878958) bank191185 +191186 POINT(38.70241333705638 -123.36743117851125) bank191186 +191187 POINT(38.245089390180205 -123.20392958377184) bank191187 +191188 POINT(37.80261199094712 -122.54577132756266) bank191188 +191189 POINT(37.78302542083553 -122.94615118106503) bank191189 +191190 POINT(38.029932866125634 -121.74169586588476) bank191190 +191191 POINT(38.30553892302362 -121.50919183382935) bank191191 +191192 POINT(37.283060095873594 -122.06351079971991) bank191192 +191193 POINT(37.05194656282184 -122.88181847946966) bank191193 +191194 POINT(38.1816053851478 -121.4516353083365) bank191194 +191195 POINT(37.419154537970776 -122.1636884377652) bank191195 +191196 POINT(37.73774158357667 -121.92529683208797) bank191196 +191197 POINT(38.56008098453319 -122.10419465597583) bank191197 +191198 POINT(38.56982901847118 -122.6756846105649) bank191198 +191199 POINT(36.78627164696786 -122.59296940331193) bank191199 +191200 POINT(36.92483006587585 -121.78347319303103) bank191200 +191201 POINT(37.7244407518607 -122.44742825051536) bank191201 +191202 POINT(37.30572051600309 -123.25403446772744) bank191202 +191203 POINT(38.58340371649746 -121.9281779428068) bank191203 +191204 POINT(37.94823993033232 -122.59080103424887) bank191204 +191205 POINT(37.6252020248039 -121.92671196244564) bank191205 +191206 POINT(37.393679222346414 -122.11063249500145) bank191206 +191207 POINT(38.008981598247686 -123.0663651671008) bank191207 +191208 POINT(37.167117763878295 -122.80506201585303) bank191208 +191209 POINT(38.69540671246624 -123.40158171044118) bank191209 +191210 POINT(38.63992480278313 -122.39062547788954) bank191210 +191211 POINT(37.855253349095186 -123.28119262141563) bank191211 +191212 POINT(37.78710180058467 -122.45916061232607) bank191212 +191213 POINT(37.47314938370203 -122.32182198345222) bank191213 +191214 POINT(37.18444555612309 -122.10930390149858) bank191214 +191215 POINT(37.112379646996914 -122.52249227526657) bank191215 +191216 POINT(37.68368165864751 -122.99963810857716) bank191216 +191217 POINT(38.27637507634907 -121.51069677228304) bank191217 +191218 POINT(36.8637979073755 -121.58108440512927) bank191218 +191219 POINT(38.082510847977915 -122.31474880497997) bank191219 +191220 POINT(38.3283146855239 -121.63133803904297) bank191220 +191221 POINT(36.95047382201357 -121.47786057880074) bank191221 +191222 POINT(38.75817063064223 -121.58056676644951) bank191222 +191223 POINT(38.024721703863264 -123.09843225018535) bank191223 +191224 POINT(37.38176588396564 -122.05300964978596) bank191224 +191225 POINT(38.038696227068314 -123.25673455635663) bank191225 +191226 POINT(38.301477833471196 -123.22492763559005) bank191226 +191227 POINT(37.931559883069774 -123.05476770924808) bank191227 +191228 POINT(37.399779186429114 -121.6984704288198) bank191228 +191229 POINT(37.40853197100797 -122.87468850902357) bank191229 +191230 POINT(37.53217503357363 -122.97568915472073) bank191230 +191231 POINT(38.40003884907475 -122.67162243279306) bank191231 +191232 POINT(37.06289217255242 -122.88626324391976) bank191232 +191233 POINT(38.081459094734846 -122.53477029916903) bank191233 +191234 POINT(38.25501492274263 -122.66136739439571) bank191234 +191235 POINT(38.7090142637252 -122.19689796708843) bank191235 +191236 POINT(38.76986531991694 -122.90001355620217) bank191236 +191237 POINT(38.29786243278438 -121.89301295064153) bank191237 +191238 POINT(38.48713683843806 -121.61485566051947) bank191238 +191239 POINT(38.559857671608356 -122.27537077059952) bank191239 +191240 POINT(37.22459438822425 -122.2556771028199) bank191240 +191241 POINT(37.7741271635721 -122.50568821298346) bank191241 +191242 POINT(37.18847795020937 -122.15235080866059) bank191242 +191243 POINT(37.472334146960975 -122.65057904294365) bank191243 +191244 POINT(37.0443521952343 -123.12547343241201) bank191244 +191245 POINT(38.03233246434074 -123.30382390628257) bank191245 +191246 POINT(38.102684209107096 -121.50585442363632) bank191246 +191247 POINT(36.9145234598164 -122.5986184220143) bank191247 +191248 POINT(36.796787747727734 -122.03485630205063) bank191248 +191249 POINT(37.39173328856513 -121.77451815231008) bank191249 +191250 POINT(36.95993720041011 -121.77640527352597) bank191250 +191251 POINT(38.62019229699323 -122.8527682956528) bank191251 +191252 POINT(36.92147505430832 -123.30807739280499) bank191252 +191253 POINT(36.97455099786929 -123.11147223993214) bank191253 +191254 POINT(36.79580742348934 -121.48264182426124) bank191254 +191255 POINT(38.06368759611048 -121.75398279502231) bank191255 +191256 POINT(36.79258367444849 -122.40837414036714) bank191256 +191257 POINT(38.49953124585839 -122.3221000279384) bank191257 +191258 POINT(37.47767816364645 -122.95591788138584) bank191258 +191259 POINT(38.68988595885045 -122.15241474080919) bank191259 +191260 POINT(38.70063272132481 -121.59340913022565) bank191260 +191261 POINT(37.695591240257365 -123.32949164357177) bank191261 +191262 POINT(37.800289576877056 -122.04448630131724) bank191262 +191263 POINT(38.131799115652846 -122.50853993898274) bank191263 +191264 POINT(36.85888430285112 -122.79957121356249) bank191264 +191265 POINT(37.62698120834037 -122.33121228528168) bank191265 +191266 POINT(38.13306887686654 -121.78012109502794) bank191266 +191267 POINT(36.785280425721275 -123.25831088582488) bank191267 +191268 POINT(38.580634603365965 -122.83902490974927) bank191268 +191269 POINT(38.198389438723744 -122.16133753029939) bank191269 +191270 POINT(37.66980064700403 -121.63002323489128) bank191270 +191271 POINT(37.67580501979911 -123.03001577068531) bank191271 +191272 POINT(36.91140738871617 -122.67830656508275) bank191272 +191273 POINT(37.09852701977683 -122.54907754930345) bank191273 +191274 POINT(37.26513100133942 -123.26235072400125) bank191274 +191275 POINT(37.59575819629173 -121.96824180570209) bank191275 +191276 POINT(37.922068687490096 -123.0850071744301) bank191276 +191277 POINT(37.64733461934441 -122.43415079664416) bank191277 +191278 POINT(38.40869271310924 -122.28212630872895) bank191278 +191279 POINT(36.99546655653119 -123.02684144072842) bank191279 +191280 POINT(37.94745860213714 -123.09652120026385) bank191280 +191281 POINT(38.23385563375201 -121.94295620058234) bank191281 +191282 POINT(38.19677479434324 -121.50412045466642) bank191282 +191283 POINT(37.62293354004289 -121.9273490433023) bank191283 +191284 POINT(37.98547827452216 -122.04990011740422) bank191284 +191285 POINT(38.06578514744427 -122.78749643966107) bank191285 +191286 POINT(37.78576157158071 -121.88725932286472) bank191286 +191287 POINT(38.39594174096605 -122.85795907126978) bank191287 +191288 POINT(38.22285487158707 -122.14283159973819) bank191288 +191289 POINT(38.68933797771462 -123.10859055141842) bank191289 +191290 POINT(37.965934713913086 -121.4700473249172) bank191290 +191291 POINT(37.10221678518636 -123.01481045585858) bank191291 +191292 POINT(36.943995645104984 -122.7959475378628) bank191292 +191293 POINT(37.15547620469186 -123.28414976942011) bank191293 +191294 POINT(38.15038930486027 -122.54050105945701) bank191294 +191295 POINT(37.54603943109875 -122.19123735089914) bank191295 +191296 POINT(37.94393937685879 -122.59435629801179) bank191296 +191297 POINT(37.73760679793884 -122.47300551571382) bank191297 +191298 POINT(37.60458500419841 -121.46423904479343) bank191298 +191299 POINT(37.366375319882316 -123.23763256793251) bank191299 +191300 POINT(38.517896746617694 -122.61519717445873) bank191300 +191301 POINT(37.096829882144256 -122.84399875850123) bank191301 +191302 POINT(37.13914430040433 -123.40837069306876) bank191302 +191303 POINT(37.54652245678066 -121.55257799205606) bank191303 +191304 POINT(37.40261098131457 -121.69513036768723) bank191304 +191305 POINT(37.178855296751614 -122.90695076389419) bank191305 +191306 POINT(38.69244337752388 -122.13254999225428) bank191306 +191307 POINT(38.056163245056624 -122.19121552666391) bank191307 +191308 POINT(36.98676816236105 -122.95980831148077) bank191308 +191309 POINT(38.5486334555399 -121.91438873929143) bank191309 +191310 POINT(38.56026760122417 -123.19873185823873) bank191310 +191311 POINT(37.25826548093622 -122.58853621117876) bank191311 +191312 POINT(36.91458599584772 -122.38413074356488) bank191312 +191313 POINT(37.84750283314448 -122.56118684949688) bank191313 +191314 POINT(37.20596347013333 -122.03731906616981) bank191314 +191315 POINT(38.57168652598536 -121.88921802960971) bank191315 +191316 POINT(37.80391869299 -122.66668667514153) bank191316 +191317 POINT(37.14041770839907 -123.23228043368051) bank191317 +191318 POINT(38.250172839966574 -123.20428075803012) bank191318 +191319 POINT(38.203099935682964 -122.63936622417971) bank191319 +191320 POINT(37.583233358259406 -121.86690810456166) bank191320 +191321 POINT(38.53186761528398 -123.36448726028951) bank191321 +191322 POINT(37.55696759189307 -122.93950574522513) bank191322 +191323 POINT(37.01153994238193 -122.11683500264452) bank191323 +191324 POINT(37.05573092009709 -123.11596129760682) bank191324 +191325 POINT(38.176964397195135 -122.1819656192333) bank191325 +191326 POINT(38.760746978537625 -122.36000891904762) bank191326 +191327 POINT(37.670750404311605 -122.68016104948917) bank191327 +191328 POINT(37.08369949077348 -123.26840532590899) bank191328 +191329 POINT(37.469128059532494 -123.38369730223484) bank191329 +191330 POINT(37.702557327740294 -121.69057132808308) bank191330 +191331 POINT(38.76435257017355 -122.65855007027498) bank191331 +191332 POINT(37.41338643242977 -121.95482492943354) bank191332 +191333 POINT(38.394101804822306 -123.41643096957884) bank191333 +191334 POINT(38.41207229492229 -121.5778595057211) bank191334 +191335 POINT(36.824703574519866 -122.86208727206179) bank191335 +191336 POINT(37.114737408416005 -122.63493583628737) bank191336 +191337 POINT(37.14302675775779 -123.04066213845542) bank191337 +191338 POINT(37.75849341603552 -122.08655909381369) bank191338 +191339 POINT(38.617932387439616 -123.41382100179533) bank191339 +191340 POINT(38.74509659762981 -121.4668208579834) bank191340 +191341 POINT(37.516290603388825 -121.7524771183939) bank191341 +191342 POINT(37.038955270758464 -121.77741095946382) bank191342 +191343 POINT(37.18300913529517 -122.70174206148113) bank191343 +191344 POINT(37.19409743389729 -121.94257688677166) bank191344 +191345 POINT(38.04911436980226 -122.59485905482003) bank191345 +191346 POINT(38.27401252809252 -122.17624197822) bank191346 +191347 POINT(38.68630745464451 -121.65172268400474) bank191347 +191348 POINT(37.34242941462039 -123.31904684819837) bank191348 +191349 POINT(37.8811273433385 -123.03534496831472) bank191349 +191350 POINT(37.80380943798454 -122.03597552954612) bank191350 +191351 POINT(37.99109862160206 -122.81362826044445) bank191351 +191352 POINT(38.40513735411476 -123.14075632196604) bank191352 +191353 POINT(36.89775767240146 -122.04587689041367) bank191353 +191354 POINT(38.4044376054461 -121.67219614767433) bank191354 +191355 POINT(37.57561477638002 -122.4581636186756) bank191355 +191356 POINT(38.5545019091084 -121.95054392311808) bank191356 +191357 POINT(37.03470355823504 -122.24204908783729) bank191357 +191358 POINT(38.364476747746934 -122.71100060634815) bank191358 +191359 POINT(38.29683882262203 -122.77765645889764) bank191359 +191360 POINT(37.99632006135307 -121.9558599291429) bank191360 +191361 POINT(37.43834222747799 -122.92614142076775) bank191361 +191362 POINT(38.683334255260945 -122.16630785956761) bank191362 +191363 POINT(36.84583683935914 -122.51863003625978) bank191363 +191364 POINT(37.72419803282325 -122.85149958304758) bank191364 +191365 POINT(37.72221705986627 -121.69774797047921) bank191365 +191366 POINT(37.64952036001018 -121.79952430090877) bank191366 +191367 POINT(38.64667866673843 -123.31027949971536) bank191367 +191368 POINT(38.513320130204455 -123.31382756615656) bank191368 +191369 POINT(36.963017319241 -121.82159956848254) bank191369 +191370 POINT(38.38269483738865 -122.39647618845892) bank191370 +191371 POINT(38.46581968928357 -121.93204377074184) bank191371 +191372 POINT(38.190251470746695 -121.73130642668693) bank191372 +191373 POINT(38.16124884554748 -122.82072069866341) bank191373 +191374 POINT(38.27035175365067 -123.35402382246184) bank191374 +191375 POINT(38.04882332848201 -121.66848632852223) bank191375 +191376 POINT(37.13131728758379 -121.45248447855948) bank191376 +191377 POINT(38.02076426115623 -121.42113317630547) bank191377 +191378 POINT(37.7700319401849 -122.45307251583785) bank191378 +191379 POINT(37.48495061771243 -123.16010392521511) bank191379 +191380 POINT(38.37487546887433 -121.58413450157494) bank191380 +191381 POINT(37.28405818325764 -121.43473646744721) bank191381 +191382 POINT(37.8992358153752 -122.730695886117) bank191382 +191383 POINT(38.14629943048037 -122.62256883429392) bank191383 +191384 POINT(37.27150527860492 -123.12276902019927) bank191384 +191385 POINT(37.31988888215283 -122.33917868146268) bank191385 +191386 POINT(38.018426548080114 -121.99371629019812) bank191386 +191387 POINT(37.4248111116601 -122.87138207851079) bank191387 +191388 POINT(38.18017983573146 -122.07247909107565) bank191388 +191389 POINT(38.27064809590565 -122.35398845817006) bank191389 +191390 POINT(38.45948091880603 -122.42338265806822) bank191390 +191391 POINT(37.7975727382761 -122.2688956440199) bank191391 +191392 POINT(38.68733603639578 -122.11662719522036) bank191392 +191393 POINT(37.67506653961606 -121.54337092009385) bank191393 +191394 POINT(38.53788341745225 -122.5964974796779) bank191394 +191395 POINT(37.76922527624671 -121.52675661942475) bank191395 +191396 POINT(36.837555456770595 -122.45577618177649) bank191396 +191397 POINT(37.90467780468392 -123.00540757873188) bank191397 +191398 POINT(36.83827228509979 -122.95461968601875) bank191398 +191399 POINT(38.3962378734641 -123.32800279493593) bank191399 +191400 POINT(37.85361001776624 -122.32833706301555) bank191400 +191401 POINT(37.527776165931115 -121.76760848085664) bank191401 +191402 POINT(37.10308098968864 -123.21064457890807) bank191402 +191403 POINT(38.32374961108576 -121.74123226999201) bank191403 +191404 POINT(37.20221783521303 -122.1191284964108) bank191404 +191405 POINT(37.029168098882565 -121.52375772295571) bank191405 +191406 POINT(38.303275063920815 -122.6886859520814) bank191406 +191407 POINT(38.62437181387083 -122.42476212527411) bank191407 +191408 POINT(38.4440029873848 -123.39986389736761) bank191408 +191409 POINT(37.91359261755743 -121.97964060193891) bank191409 +191410 POINT(37.79245721716874 -121.7157198328141) bank191410 +191411 POINT(38.447770033081845 -122.67693995908718) bank191411 +191412 POINT(38.131143840551964 -123.1034906950034) bank191412 +191413 POINT(38.051913372527444 -122.78404112166517) bank191413 +191414 POINT(37.84758725548345 -121.65427834367148) bank191414 +191415 POINT(38.55942669832236 -122.31973006566284) bank191415 +191416 POINT(37.72719966826561 -122.12779725530324) bank191416 +191417 POINT(38.64255530490339 -122.53972804631351) bank191417 +191418 POINT(37.94285752531793 -122.23192099262124) bank191418 +191419 POINT(38.73511281594653 -122.19226133283593) bank191419 +191420 POINT(37.57650045725974 -122.65113979438702) bank191420 +191421 POINT(37.26383067600529 -121.89212262257303) bank191421 +191422 POINT(37.33033498276434 -122.02210494714421) bank191422 +191423 POINT(37.09257348499526 -122.56709625229485) bank191423 +191424 POINT(37.79734037020524 -122.70427531841037) bank191424 +191425 POINT(38.35927059163325 -122.94593969596708) bank191425 +191426 POINT(37.213864166423335 -123.03348415524628) bank191426 +191427 POINT(37.415103213271905 -123.17823755700539) bank191427 +191428 POINT(36.809998024453 -122.94224507039291) bank191428 +191429 POINT(36.86612823476019 -123.0568314917186) bank191429 +191430 POINT(37.609020598258574 -121.58351699608428) bank191430 +191431 POINT(37.88694436123978 -122.4227018285701) bank191431 +191432 POINT(37.53452107443809 -121.70035879770843) bank191432 +191433 POINT(37.339495388791825 -121.42464170539175) bank191433 +191434 POINT(37.88742375506469 -121.753690774797) bank191434 +191435 POINT(37.6168041201256 -122.04894642662464) bank191435 +191436 POINT(37.26020251366701 -122.20207974151309) bank191436 +191437 POINT(38.767493318183355 -122.98787574893699) bank191437 +191438 POINT(38.70231553380399 -122.23487247875357) bank191438 +191439 POINT(37.16357804780442 -121.54637230214283) bank191439 +191440 POINT(37.22269166129076 -122.19413047847772) bank191440 +191441 POINT(38.553575720247196 -121.69780986274519) bank191441 +191442 POINT(36.83849620425674 -123.37099033725121) bank191442 +191443 POINT(37.864300303580414 -123.39996838866008) bank191443 +191444 POINT(37.95467818612387 -122.1714152284198) bank191444 +191445 POINT(37.513129814593306 -122.46323616636009) bank191445 +191446 POINT(38.20678815196189 -123.1183139298825) bank191446 +191447 POINT(38.349740053340554 -123.22986698102301) bank191447 +191448 POINT(37.28776310679679 -123.27745976086321) bank191448 +191449 POINT(38.108790204015904 -121.498071034989) bank191449 +191450 POINT(38.04025788376705 -122.35595735666767) bank191450 +191451 POINT(37.28272974400519 -121.61077087011121) bank191451 +191452 POINT(37.30354318281387 -121.83315411989392) bank191452 +191453 POINT(38.155862324986224 -122.5574140443862) bank191453 +191454 POINT(38.12082225155006 -122.97117277334792) bank191454 +191455 POINT(37.70006316636653 -122.34967154903367) bank191455 +191456 POINT(38.65571495201633 -123.27161001470203) bank191456 +191457 POINT(37.75172938347978 -122.46177614073265) bank191457 +191458 POINT(37.48679973904513 -122.66080816058827) bank191458 +191459 POINT(38.046329691619285 -121.56464884559469) bank191459 +191460 POINT(38.36769851264504 -122.28226425515759) bank191460 +191461 POINT(36.8784390770367 -122.47005972171446) bank191461 +191462 POINT(38.761450886040976 -122.94301931295627) bank191462 +191463 POINT(37.34769026497325 -121.8945282336232) bank191463 +191464 POINT(37.14228266157166 -122.37075278861083) bank191464 +191465 POINT(37.82610457078698 -121.97774526515819) bank191465 +191466 POINT(37.966314034815525 -123.13051492008728) bank191466 +191467 POINT(37.6068897324846 -122.23073659794574) bank191467 +191468 POINT(37.72738915249339 -122.94120376530368) bank191468 +191469 POINT(36.836043844695325 -122.5044499787719) bank191469 +191470 POINT(38.14432231591821 -123.09902768957623) bank191470 +191471 POINT(36.8996441705264 -122.46257405241155) bank191471 +191472 POINT(37.63881685626903 -122.04962236860524) bank191472 +191473 POINT(36.97204081048954 -122.97109183461257) bank191473 +191474 POINT(37.74964958397616 -122.79447992466767) bank191474 +191475 POINT(38.50965566605888 -122.32011454204695) bank191475 +191476 POINT(37.112275353737296 -122.50117562728181) bank191476 +191477 POINT(37.455662993835276 -122.32758087310262) bank191477 +191478 POINT(38.679292808148986 -122.93475142114687) bank191478 +191479 POINT(38.14128383051687 -122.60606402557544) bank191479 +191480 POINT(38.43684144132134 -121.96707690412546) bank191480 +191481 POINT(37.69797224175445 -122.66314179150177) bank191481 +191482 POINT(36.85262591050575 -122.1926468374481) bank191482 +191483 POINT(38.674153376524536 -122.8063906930416) bank191483 +191484 POINT(37.9250692151016 -122.71454349955654) bank191484 +191485 POINT(37.19189929523832 -123.01180595871662) bank191485 +191486 POINT(37.236065075992315 -122.437865768568) bank191486 +191487 POINT(37.029093469679594 -122.8517504267153) bank191487 +191488 POINT(38.18533079142566 -122.51543154310559) bank191488 +191489 POINT(37.27726528542501 -121.77196220599622) bank191489 +191490 POINT(38.20574986569021 -121.75507470507765) bank191490 +191491 POINT(38.31107669849525 -122.0321243493344) bank191491 +191492 POINT(38.53651434427304 -121.84826264465941) bank191492 +191493 POINT(37.71952458206603 -122.47080172957698) bank191493 +191494 POINT(37.983127526283255 -123.08620276570436) bank191494 +191495 POINT(37.699704833032094 -122.48199153633479) bank191495 +191496 POINT(38.24880974543331 -122.36916046965555) bank191496 +191497 POINT(37.242469710975094 -122.38331175188725) bank191497 +191498 POINT(37.07633639205511 -121.45626142921365) bank191498 +191499 POINT(37.39408556987982 -122.52684992950321) bank191499 +191500 POINT(38.36146517764499 -122.4572156253252) bank191500 +191501 POINT(37.99994558402433 -121.77796267183842) bank191501 +191502 POINT(37.83880163629347 -122.02681061735834) bank191502 +191503 POINT(36.97527756162388 -123.41111482043864) bank191503 +191504 POINT(36.929869216583974 -121.9435864731999) bank191504 +191505 POINT(36.997414136433356 -122.61783489064786) bank191505 +191506 POINT(38.61968752917574 -122.15084764151523) bank191506 +191507 POINT(37.43891730968913 -122.22681272702913) bank191507 +191508 POINT(37.86787409324484 -122.22618203248888) bank191508 +191509 POINT(37.00456626097953 -123.1795229827042) bank191509 +191510 POINT(36.808448948657016 -121.80213764934827) bank191510 +191511 POINT(37.34398122325221 -121.71577575518191) bank191511 +191512 POINT(38.440161508933045 -121.6546436413489) bank191512 +191513 POINT(37.72925574539411 -121.85332112917696) bank191513 +191514 POINT(37.79826181017499 -121.47454194703162) bank191514 +191515 POINT(38.08850592399243 -122.69559131341448) bank191515 +191516 POINT(37.02581737359331 -123.15316018201435) bank191516 +191517 POINT(36.940725602836665 -122.42895126199218) bank191517 +191518 POINT(36.93958972371097 -123.17442578180297) bank191518 +191519 POINT(38.72534397079618 -122.53864145957522) bank191519 +191520 POINT(37.320468969416275 -123.19830663871397) bank191520 +191521 POINT(36.82299212176149 -121.71799282123627) bank191521 +191522 POINT(37.60484867081959 -122.3769664077637) bank191522 +191523 POINT(37.746041415992615 -122.88952243590663) bank191523 +191524 POINT(36.871087130685055 -122.26872533584884) bank191524 +191525 POINT(38.73514576330408 -122.8944150172406) bank191525 +191526 POINT(36.82398693304661 -122.2775539010358) bank191526 +191527 POINT(37.57505884933381 -121.54464681634117) bank191527 +191528 POINT(38.334194829953965 -121.5886118398449) bank191528 +191529 POINT(37.28017465975091 -122.19161381192926) bank191529 +191530 POINT(38.075874312706226 -123.2412146683444) bank191530 +191531 POINT(37.61875242454994 -121.75553644524341) bank191531 +191532 POINT(37.628191548323805 -121.5339534815717) bank191532 +191533 POINT(38.32444304928852 -121.88222637724267) bank191533 +191534 POINT(36.7821673576556 -122.45279223646943) bank191534 +191535 POINT(37.74512818818932 -121.6267857517984) bank191535 +191536 POINT(38.50047764221317 -121.74750262883197) bank191536 +191537 POINT(37.890957953669805 -122.51437831861003) bank191537 +191538 POINT(38.27649632096978 -122.62288888880033) bank191538 +191539 POINT(37.93688611821664 -121.91478113017588) bank191539 +191540 POINT(37.32672297164178 -122.9900246346058) bank191540 +191541 POINT(38.22390068594477 -123.04674736148321) bank191541 +191542 POINT(38.28269662160297 -122.14516890774784) bank191542 +191543 POINT(38.26417545947238 -122.3492871717478) bank191543 +191544 POINT(37.211537464301315 -121.86667386077853) bank191544 +191545 POINT(37.1788903885839 -122.9923272598789) bank191545 +191546 POINT(37.06221245573872 -122.91407636933066) bank191546 +191547 POINT(37.61944742684775 -122.15042046494426) bank191547 +191548 POINT(37.69758607650524 -122.87977980259666) bank191548 +191549 POINT(37.054985866951846 -123.06826362017688) bank191549 +191550 POINT(37.88738626055341 -122.3652600072904) bank191550 +191551 POINT(37.63646661103808 -122.632860101821) bank191551 +191552 POINT(37.50836221909516 -123.0605639884718) bank191552 +191553 POINT(37.5875909525591 -122.7866267036922) bank191553 +191554 POINT(38.49198008366363 -122.57782535969893) bank191554 +191555 POINT(38.10021450044377 -122.30826132037595) bank191555 +191556 POINT(37.017698005547395 -122.27395014212196) bank191556 +191557 POINT(36.95416728719394 -122.44341125021354) bank191557 +191558 POINT(38.420192416557384 -123.13495273525295) bank191558 +191559 POINT(37.17967080843213 -122.2135207789331) bank191559 +191560 POINT(38.05115830736316 -122.50407258517804) bank191560 +191561 POINT(37.03061470739506 -123.13674842542247) bank191561 +191562 POINT(38.76213694054381 -121.58993727538379) bank191562 +191563 POINT(38.21694745407302 -121.73325700757837) bank191563 +191564 POINT(38.25427040307315 -122.5053219896433) bank191564 +191565 POINT(36.89117237826339 -121.70789247852193) bank191565 +191566 POINT(37.45767790422534 -121.71591894716302) bank191566 +191567 POINT(36.885816145045546 -121.8901213469258) bank191567 +191568 POINT(38.66650609203142 -122.31528524917711) bank191568 +191569 POINT(37.73507961047641 -121.7061894338261) bank191569 +191570 POINT(38.008623526982 -122.20987393463325) bank191570 +191571 POINT(38.045047623634666 -121.888489168735) bank191571 +191572 POINT(37.401306305561725 -123.02582859791099) bank191572 +191573 POINT(38.68269623026925 -122.96032915920341) bank191573 +191574 POINT(37.46645900905906 -122.16838920105228) bank191574 +191575 POINT(38.28871011396784 -122.86692145295544) bank191575 +191576 POINT(38.74150045012503 -121.62483003038056) bank191576 +191577 POINT(37.40709162885781 -122.0325208269652) bank191577 +191578 POINT(36.973447791611214 -121.87022725669792) bank191578 +191579 POINT(38.515615413772906 -122.63782551513401) bank191579 +191580 POINT(37.62187002814722 -123.17672882428062) bank191580 +191581 POINT(38.71791608388102 -121.7490735874734) bank191581 +191582 POINT(37.61119874739578 -121.9694935278182) bank191582 +191583 POINT(37.759819890367304 -121.9822609411857) bank191583 +191584 POINT(38.08711269875929 -121.64512899679309) bank191584 +191585 POINT(38.50247788805393 -122.4466451416347) bank191585 +191586 POINT(37.41012226783036 -122.31949573220747) bank191586 +191587 POINT(37.52831178561653 -122.99894895610947) bank191587 +191588 POINT(36.94641404098992 -122.23121228578643) bank191588 +191589 POINT(38.345617435886176 -122.52038731999981) bank191589 +191590 POINT(37.09280857183186 -123.09169970487082) bank191590 +191591 POINT(37.6520919711604 -122.3122250126512) bank191591 +191592 POINT(37.97096302945725 -122.57107637193221) bank191592 +191593 POINT(36.88175466581172 -122.94327259749072) bank191593 +191594 POINT(38.3957149357138 -122.20740893898078) bank191594 +191595 POINT(38.64312657161188 -122.79666709702617) bank191595 +191596 POINT(38.35456806442201 -122.27747560752944) bank191596 +191597 POINT(38.44901446163328 -122.40400804568314) bank191597 +191598 POINT(38.491302969813326 -122.2658781662222) bank191598 +191599 POINT(38.030093623257464 -123.01897885301088) bank191599 +191600 POINT(36.97592786557902 -122.5357444221991) bank191600 +191601 POINT(37.304420917024224 -122.40067707819354) bank191601 +191602 POINT(36.84500540122915 -121.9417061789766) bank191602 +191603 POINT(37.91203136907218 -122.51894457617092) bank191603 +191604 POINT(36.976879471501974 -122.40325458277557) bank191604 +191605 POINT(38.75372328154023 -122.22242149790826) bank191605 +191606 POINT(37.97045394973096 -122.53944643749203) bank191606 +191607 POINT(36.971483172867586 -122.7486997681073) bank191607 +191608 POINT(37.302394662018244 -121.51156913601905) bank191608 +191609 POINT(37.879076192593025 -122.81030864433217) bank191609 +191610 POINT(37.0576973533275 -121.8740774824756) bank191610 +191611 POINT(38.17127775671621 -121.70229068060574) bank191611 +191612 POINT(37.22880260423766 -121.5403748702195) bank191612 +191613 POINT(38.10503203741025 -121.52722630896699) bank191613 +191614 POINT(38.76915132109368 -122.87909095101698) bank191614 +191615 POINT(37.90387776139278 -121.6016422663554) bank191615 +191616 POINT(36.847794876701535 -123.27367661455527) bank191616 +191617 POINT(36.86531125612974 -122.23243666474595) bank191617 +191618 POINT(38.245361117986455 -121.80204475896367) bank191618 +191619 POINT(38.09569429276773 -123.06672907973214) bank191619 +191620 POINT(36.83454335950036 -123.14452194829454) bank191620 +191621 POINT(38.31956714176548 -123.0081491745529) bank191621 +191622 POINT(36.94000976065735 -122.54938106551997) bank191622 +191623 POINT(37.51050840588628 -122.10018889164624) bank191623 +191624 POINT(37.07927348651616 -122.50886438768813) bank191624 +191625 POINT(36.79437159700239 -123.4136100510477) bank191625 +191626 POINT(38.026702551547544 -122.75334455469002) bank191626 +191627 POINT(36.836636152155194 -122.34494613895657) bank191627 +191628 POINT(37.683921588723905 -122.0362214242948) bank191628 +191629 POINT(38.30104688624472 -123.24728275508859) bank191629 +191630 POINT(37.43285519779171 -122.7444470790411) bank191630 +191631 POINT(38.37136294353477 -122.1174520523319) bank191631 +191632 POINT(37.57475291915135 -121.99926677208263) bank191632 +191633 POINT(38.0802450097339 -121.95564277507879) bank191633 +191634 POINT(37.006225661567896 -121.99617190528089) bank191634 +191635 POINT(37.44026213087561 -122.89989645257883) bank191635 +191636 POINT(37.66161136228549 -123.25584220284877) bank191636 +191637 POINT(37.089570059120064 -122.30441906368846) bank191637 +191638 POINT(38.33385394285441 -122.37028921037346) bank191638 +191639 POINT(38.58736990154281 -123.04383291432312) bank191639 +191640 POINT(37.196206064415264 -122.87339890127224) bank191640 +191641 POINT(38.05953029791123 -121.81200955310499) bank191641 +191642 POINT(38.773566852074524 -121.7534940168789) bank191642 +191643 POINT(36.98734181541452 -122.76172976396113) bank191643 +191644 POINT(38.034611545361216 -121.95333606785388) bank191644 +191645 POINT(36.972512031679855 -121.53547591486128) bank191645 +191646 POINT(37.90390981734449 -121.59611920772659) bank191646 +191647 POINT(37.71488714617088 -123.3409157282973) bank191647 +191648 POINT(37.03249480865619 -122.34290859168834) bank191648 +191649 POINT(37.951962260252664 -123.02703958004432) bank191649 +191650 POINT(37.39921113680704 -123.18471927195102) bank191650 +191651 POINT(38.65748633672139 -122.6923203294062) bank191651 +191652 POINT(36.92443698803022 -123.39991684598984) bank191652 +191653 POINT(38.17712908191578 -123.1132011403074) bank191653 +191654 POINT(36.99206185627405 -121.83619397105248) bank191654 +191655 POINT(37.568151344461576 -121.87641770100755) bank191655 +191656 POINT(38.10281636802267 -122.0440032021714) bank191656 +191657 POINT(38.556908945377835 -122.62071440562) bank191657 +191658 POINT(37.52581549459662 -122.39690281086023) bank191658 +191659 POINT(37.68598824252348 -122.55821372546583) bank191659 +191660 POINT(38.22291636890804 -122.17742748957016) bank191660 +191661 POINT(38.354989458896334 -122.99856980726163) bank191661 +191662 POINT(36.87335196565318 -122.09191014515066) bank191662 +191663 POINT(37.36934050648591 -122.46035501814926) bank191663 +191664 POINT(36.86409759067941 -122.67673280544102) bank191664 +191665 POINT(37.36987823659411 -121.52884931511345) bank191665 +191666 POINT(38.2075093233475 -122.247484994755) bank191666 +191667 POINT(38.04153193262501 -123.33448467768025) bank191667 +191668 POINT(36.921275847442544 -121.99120070878946) bank191668 +191669 POINT(37.681576792952306 -123.03454724459978) bank191669 +191670 POINT(38.264545851190675 -121.69939294254868) bank191670 +191671 POINT(37.48700022592532 -122.55488044011622) bank191671 +191672 POINT(37.109275490177325 -123.34236875301085) bank191672 +191673 POINT(38.02229183546236 -121.42372471028402) bank191673 +191674 POINT(37.07336085307062 -123.24769938397806) bank191674 +191675 POINT(37.74405425721936 -122.33526516799265) bank191675 +191676 POINT(38.5520430271305 -123.03290223180319) bank191676 +191677 POINT(38.50086241796183 -122.17690821513672) bank191677 +191678 POINT(37.15104002082995 -122.8326689550643) bank191678 +191679 POINT(37.09015226452658 -123.31235482982149) bank191679 +191680 POINT(37.777460587522484 -121.78639778480544) bank191680 +191681 POINT(36.93170520827673 -121.42179741654486) bank191681 +191682 POINT(37.397630064817456 -122.63826825436708) bank191682 +191683 POINT(36.903202348286825 -121.82098187130137) bank191683 +191684 POINT(38.53860546788308 -123.13483971248483) bank191684 +191685 POINT(37.55088213410287 -121.52298216693065) bank191685 +191686 POINT(37.91282861324673 -123.30521355210485) bank191686 +191687 POINT(38.661742541062715 -122.89761621710818) bank191687 +191688 POINT(37.385420743789155 -122.18999602242997) bank191688 +191689 POINT(37.35268522279131 -123.2777986068586) bank191689 +191690 POINT(37.36040994067395 -122.47426763098727) bank191690 +191691 POINT(37.79027283788096 -123.19539660297569) bank191691 +191692 POINT(38.356967200072454 -123.05308548550846) bank191692 +191693 POINT(38.07621551557104 -122.83898342804102) bank191693 +191694 POINT(37.81563039711778 -122.91492402996488) bank191694 +191695 POINT(36.879769298601055 -122.7901687770555) bank191695 +191696 POINT(37.35837985246418 -122.98105051849592) bank191696 +191697 POINT(38.60902275759759 -123.20839328576957) bank191697 +191698 POINT(38.69139452118161 -122.66186381078629) bank191698 +191699 POINT(38.62566196566261 -122.05386504708896) bank191699 +191700 POINT(37.36940961464629 -123.27824391327697) bank191700 +191701 POINT(37.775683197360955 -122.38737814204566) bank191701 +191702 POINT(38.275704427344344 -122.68284732387984) bank191702 +191703 POINT(37.71255106188647 -123.34180709117848) bank191703 +191704 POINT(37.52872283132685 -121.56058690755368) bank191704 +191705 POINT(37.69867394469246 -122.20019145056182) bank191705 +191706 POINT(37.06934586850466 -123.29138162242042) bank191706 +191707 POINT(37.60445573107394 -121.4943404059544) bank191707 +191708 POINT(37.57676445356737 -122.89640946479665) bank191708 +191709 POINT(38.05005958122776 -123.0342722916209) bank191709 +191710 POINT(37.37079653865816 -121.63777266784254) bank191710 +191711 POINT(38.47555920053866 -121.89201678039453) bank191711 +191712 POINT(38.34217551187786 -121.60690993412632) bank191712 +191713 POINT(38.192952795166505 -121.99112381390248) bank191713 +191714 POINT(37.46743348967262 -121.98256279949888) bank191714 +191715 POINT(37.507229442088445 -122.57051681944343) bank191715 +191716 POINT(38.55529772810061 -122.51733515215206) bank191716 +191717 POINT(37.405516916319705 -123.08763353641109) bank191717 +191718 POINT(37.67683584160772 -122.67333731936209) bank191718 +191719 POINT(37.31836259377292 -122.12268107576942) bank191719 +191720 POINT(37.03408828438267 -122.25146994481563) bank191720 +191721 POINT(38.7550778621899 -121.66734402123528) bank191721 +191722 POINT(37.50366425960918 -122.11556503679994) bank191722 +191723 POINT(38.624453840390494 -122.28243494446377) bank191723 +191724 POINT(38.21264506127723 -122.27955097557108) bank191724 +191725 POINT(36.84751772826052 -123.09760722050859) bank191725 +191726 POINT(37.303209842930436 -122.64297294801699) bank191726 +191727 POINT(37.59631012333015 -122.21453299329235) bank191727 +191728 POINT(37.745726142177936 -122.34896183676888) bank191728 +191729 POINT(37.59500345637672 -122.92157615241678) bank191729 +191730 POINT(38.416545036458686 -121.84290968577145) bank191730 +191731 POINT(36.997416544620116 -122.21323334716591) bank191731 +191732 POINT(37.426301188415216 -121.50294762313824) bank191732 +191733 POINT(38.73326223586929 -122.60635392043905) bank191733 +191734 POINT(38.44860331108145 -122.33587842511803) bank191734 +191735 POINT(37.00459377691435 -121.57101444421866) bank191735 +191736 POINT(37.95678143903953 -121.58107685641988) bank191736 +191737 POINT(37.423316044758174 -122.47043231028945) bank191737 +191738 POINT(38.453569494043435 -123.285084137548) bank191738 +191739 POINT(37.85409491929154 -121.86281891068514) bank191739 +191740 POINT(36.933823864758715 -122.4174453879021) bank191740 +191741 POINT(37.60099287545507 -122.3102039463883) bank191741 +191742 POINT(38.689785055819 -122.73815051395316) bank191742 +191743 POINT(36.94860693234409 -122.92149246526412) bank191743 +191744 POINT(38.70687844358419 -123.35535334741512) bank191744 +191745 POINT(37.26847941447752 -121.49046058089021) bank191745 +191746 POINT(38.276938606305045 -122.38517000594418) bank191746 +191747 POINT(37.33331140275181 -123.17319935565038) bank191747 +191748 POINT(37.324862484617654 -121.7664706155957) bank191748 +191749 POINT(37.33798071247716 -121.64383195937597) bank191749 +191750 POINT(38.48746865665621 -121.4295951095096) bank191750 +191751 POINT(38.65447693168116 -123.28388764379113) bank191751 +191752 POINT(37.964862785607394 -122.1205424836519) bank191752 +191753 POINT(37.63831429690842 -122.16224148014783) bank191753 +191754 POINT(38.55333922464086 -121.62832260546048) bank191754 +191755 POINT(38.77087501463535 -121.66955488239202) bank191755 +191756 POINT(37.326336095117625 -122.1001744183714) bank191756 +191757 POINT(38.05090259614699 -123.20010540289238) bank191757 +191758 POINT(36.780847709014594 -123.20604119383324) bank191758 +191759 POINT(38.15794414374447 -121.8503939164073) bank191759 +191760 POINT(37.86702703459918 -123.11960048268224) bank191760 +191761 POINT(37.47206798965955 -121.68839896256024) bank191761 +191762 POINT(38.6996619341241 -123.37340482849658) bank191762 +191763 POINT(38.16473786005571 -122.98218863596595) bank191763 +191764 POINT(38.16798748663786 -121.61828736261363) bank191764 +191765 POINT(36.80878218982547 -123.15706133778023) bank191765 +191766 POINT(37.17924275881063 -122.92249090348872) bank191766 +191767 POINT(37.783622321780264 -122.0585072748026) bank191767 +191768 POINT(38.35618598254854 -121.99670783086032) bank191768 +191769 POINT(38.41551461979227 -122.36857191992581) bank191769 +191770 POINT(38.54143102350312 -122.65147168584683) bank191770 +191771 POINT(38.06886600978638 -122.02096403008622) bank191771 +191772 POINT(37.72564405559658 -121.7694648676067) bank191772 +191773 POINT(38.51515275590238 -122.660100034055) bank191773 +191774 POINT(38.1265833259694 -121.88345499073995) bank191774 +191775 POINT(37.20155078040926 -123.08397056973861) bank191775 +191776 POINT(38.42710566998249 -122.45653624848026) bank191776 +191777 POINT(37.5208527140285 -122.10364598987819) bank191777 +191778 POINT(37.26773335592778 -122.61653407316373) bank191778 +191779 POINT(38.49779322437588 -121.83589976248092) bank191779 +191780 POINT(36.959029612510186 -122.35437310875832) bank191780 +191781 POINT(38.36518609941852 -122.96362946577594) bank191781 +191782 POINT(37.85707994396167 -122.95015281865577) bank191782 +191783 POINT(38.40237090903829 -121.64969885161139) bank191783 +191784 POINT(37.70887268634978 -123.36119762988524) bank191784 +191785 POINT(37.49632314637773 -122.73471096998612) bank191785 +191786 POINT(38.261032167896786 -122.37376778420092) bank191786 +191787 POINT(37.707005294434246 -122.36654075839824) bank191787 +191788 POINT(37.350805240213745 -122.51643118337894) bank191788 +191789 POINT(38.735128126159985 -122.61825056348499) bank191789 +191790 POINT(37.03079514960611 -122.39754525800436) bank191790 +191791 POINT(37.94512432162006 -123.31333380920667) bank191791 +191792 POINT(38.294928319814 -122.6529135699641) bank191792 +191793 POINT(37.60807422310195 -122.07520575333464) bank191793 +191794 POINT(38.19784136958606 -121.60206367841423) bank191794 +191795 POINT(37.68028220574798 -122.68724547550066) bank191795 +191796 POINT(37.46477415466729 -123.14423406354396) bank191796 +191797 POINT(38.49997203236178 -121.5734734684807) bank191797 +191798 POINT(37.206116256060525 -121.50696795272714) bank191798 +191799 POINT(38.007389841405896 -122.48162786000988) bank191799 +191800 POINT(37.82852886340888 -122.6064022496525) bank191800 +191801 POINT(37.71692750923124 -121.48737283178521) bank191801 +191802 POINT(36.869513544491184 -121.43824806937282) bank191802 +191803 POINT(37.85768765113579 -123.24389863616624) bank191803 +191804 POINT(37.46500989162612 -123.34648383754806) bank191804 +191805 POINT(37.639793597271705 -122.67707490350323) bank191805 +191806 POINT(36.86345299600837 -123.384949281053) bank191806 +191807 POINT(38.409999210925136 -122.67745288533237) bank191807 +191808 POINT(38.666991704109016 -122.1232650691048) bank191808 +191809 POINT(36.9641414458738 -122.75881016722856) bank191809 +191810 POINT(37.988177758637484 -122.39150638624957) bank191810 +191811 POINT(37.7695617157371 -122.23942859139243) bank191811 +191812 POINT(38.27731078261789 -122.74002019002288) bank191812 +191813 POINT(37.05988943459535 -122.14175308633762) bank191813 +191814 POINT(37.742303864640476 -123.33127398660116) bank191814 +191815 POINT(36.88571632313371 -122.34973296936674) bank191815 +191816 POINT(37.17829217153294 -123.39025037959406) bank191816 +191817 POINT(36.820041266378055 -122.30244658766226) bank191817 +191818 POINT(37.92149828057821 -123.04178029702092) bank191818 +191819 POINT(38.605000809355836 -121.77399356170172) bank191819 +191820 POINT(38.29075695623755 -122.20416741769586) bank191820 +191821 POINT(38.020973054321836 -122.06508834458604) bank191821 +191822 POINT(38.39117946331894 -121.82942325743356) bank191822 +191823 POINT(37.65896647262545 -122.52116746460992) bank191823 +191824 POINT(38.73127929201119 -121.80165584933505) bank191824 +191825 POINT(38.543691709209 -122.90946660132818) bank191825 +191826 POINT(37.4914931600145 -123.02993748623199) bank191826 +191827 POINT(38.489827087474254 -123.15071080752026) bank191827 +191828 POINT(38.28346384973821 -122.11529429152468) bank191828 +191829 POINT(37.8476576341033 -122.18542398635768) bank191829 +191830 POINT(37.861201485257254 -122.19209875862106) bank191830 +191831 POINT(38.695494883304534 -122.776273589029) bank191831 +191832 POINT(37.43033706783109 -123.27985534803722) bank191832 +191833 POINT(37.04908934106377 -122.68109068657363) bank191833 +191834 POINT(36.818373441469035 -121.82463828210675) bank191834 +191835 POINT(38.66565559376778 -122.09570374603996) bank191835 +191836 POINT(38.65885888735122 -122.74196284955254) bank191836 +191837 POINT(37.13770328779808 -123.38926727986886) bank191837 +191838 POINT(37.45697532861864 -122.68915553304444) bank191838 +191839 POINT(38.21473570947151 -122.71790694456826) bank191839 +191840 POINT(37.818708493011975 -122.60160458297591) bank191840 +191841 POINT(37.46310023251796 -122.70966420403262) bank191841 +191842 POINT(37.14744439713117 -121.81648476461339) bank191842 +191843 POINT(38.18439266070875 -123.29056248531519) bank191843 +191844 POINT(38.24459227147252 -121.5808294337709) bank191844 +191845 POINT(37.137221732927856 -122.67045879321643) bank191845 +191846 POINT(37.5009307138076 -122.50704874431787) bank191846 +191847 POINT(37.89131034170693 -121.65712966757992) bank191847 +191848 POINT(36.90584692260744 -122.63747356736614) bank191848 +191849 POINT(37.08782626069785 -122.82303890549042) bank191849 +191850 POINT(37.67869712522269 -122.12885382934226) bank191850 +191851 POINT(37.83067014328375 -122.97271815239799) bank191851 +191852 POINT(37.710803954100854 -122.09173493968717) bank191852 +191853 POINT(36.89265903701382 -121.5174931997037) bank191853 +191854 POINT(37.12007577909013 -121.9174781632709) bank191854 +191855 POINT(38.26298925510878 -122.46087532543035) bank191855 +191856 POINT(37.10433341313263 -121.67922588493204) bank191856 +191857 POINT(37.836418314467416 -121.4511923465726) bank191857 +191858 POINT(38.152924114280395 -122.72531488925318) bank191858 +191859 POINT(36.956318208976555 -122.05491535482285) bank191859 +191860 POINT(38.079701346669424 -122.91721300028338) bank191860 +191861 POINT(38.64982348224412 -123.00793615342843) bank191861 +191862 POINT(38.07937095045823 -122.71123447790924) bank191862 +191863 POINT(37.960950922403555 -122.58552769502919) bank191863 +191864 POINT(38.398167610052894 -122.73738174862494) bank191864 +191865 POINT(38.06158821801917 -122.39185191507009) bank191865 +191866 POINT(37.61408624306196 -121.72738778925547) bank191866 +191867 POINT(36.787839598496326 -123.25882159316546) bank191867 +191868 POINT(37.290880189434866 -123.30818664691999) bank191868 +191869 POINT(38.58636697430863 -122.27495304837787) bank191869 +191870 POINT(37.992964493577546 -121.73279425279023) bank191870 +191871 POINT(38.72599980818396 -122.36198610397153) bank191871 +191872 POINT(37.82095552027843 -122.54180940104617) bank191872 +191873 POINT(38.61222084715294 -123.03352531466642) bank191873 +191874 POINT(38.07981927301772 -123.24660619433031) bank191874 +191875 POINT(38.20086011469073 -122.14490876532452) bank191875 +191876 POINT(38.465509351717486 -122.1175227634236) bank191876 +191877 POINT(37.57256013873538 -123.26741242447304) bank191877 +191878 POINT(37.53786591898665 -122.3115476511629) bank191878 +191879 POINT(38.533428750128735 -122.66671088913778) bank191879 +191880 POINT(37.43230496623451 -121.55905537282635) bank191880 +191881 POINT(37.02156821956546 -121.9290742610624) bank191881 +191882 POINT(37.20144031345745 -121.53059996331764) bank191882 +191883 POINT(37.827739394754545 -122.41392537936751) bank191883 +191884 POINT(36.989075837563135 -122.77107928226977) bank191884 +191885 POINT(37.0920229015676 -123.00598955710173) bank191885 +191886 POINT(38.10361572143891 -122.20921029708389) bank191886 +191887 POINT(38.371090601102225 -122.97337866242627) bank191887 +191888 POINT(37.24264185102937 -122.84787127589942) bank191888 +191889 POINT(38.69420308384019 -122.90787894665004) bank191889 +191890 POINT(37.87512546989166 -122.27542619625223) bank191890 +191891 POINT(38.67672639943073 -122.49850827188489) bank191891 +191892 POINT(37.324709255940284 -122.13588470736455) bank191892 +191893 POINT(36.9458476018211 -123.28922740930255) bank191893 +191894 POINT(37.36696431799096 -122.84535280818069) bank191894 +191895 POINT(38.394180839309335 -121.4856265383135) bank191895 +191896 POINT(37.19762435794517 -122.79293161285378) bank191896 +191897 POINT(38.37963765369766 -123.26488055537877) bank191897 +191898 POINT(38.53940887138179 -122.97991992587998) bank191898 +191899 POINT(38.744050411870546 -122.80032918078312) bank191899 +191900 POINT(37.7147900370054 -123.35013638385942) bank191900 +191901 POINT(37.17513095603999 -122.80988764333881) bank191901 +191902 POINT(38.19989054086986 -122.64088623918862) bank191902 +191903 POINT(37.06830353486911 -122.10227711502627) bank191903 +191904 POINT(38.692643108763114 -122.0110463677397) bank191904 +191905 POINT(37.58610810968516 -121.64822450110542) bank191905 +191906 POINT(38.51331671205204 -121.48371434158791) bank191906 +191907 POINT(38.06870756368059 -121.45555375515148) bank191907 +191908 POINT(38.65869808088758 -123.08853375415674) bank191908 +191909 POINT(37.12150922575394 -122.32439978126146) bank191909 +191910 POINT(37.43768925217014 -121.55325199291002) bank191910 +191911 POINT(37.76176135861481 -123.13335542596002) bank191911 +191912 POINT(37.53173086224418 -121.59474851196579) bank191912 +191913 POINT(37.58032214113339 -122.34280573796889) bank191913 +191914 POINT(37.705099601025815 -122.2667731794457) bank191914 +191915 POINT(38.45417171212804 -123.21866089872896) bank191915 +191916 POINT(37.391012510032795 -122.15476480299111) bank191916 +191917 POINT(37.66999968038309 -122.66641820047519) bank191917 +191918 POINT(37.38427571024202 -123.27248283220234) bank191918 +191919 POINT(38.492034650877784 -122.83255876073436) bank191919 +191920 POINT(38.65333120759129 -122.74521666701855) bank191920 +191921 POINT(37.76296731203635 -122.47935173476053) bank191921 +191922 POINT(36.88909739022632 -123.11083772856583) bank191922 +191923 POINT(37.26672654998257 -123.32963086137926) bank191923 +191924 POINT(37.40385318502822 -122.2149009127476) bank191924 +191925 POINT(38.764207573766285 -123.27820478776766) bank191925 +191926 POINT(37.93371341632167 -122.35689538547349) bank191926 +191927 POINT(38.25557335372662 -123.33471055127778) bank191927 +191928 POINT(37.842736394847826 -122.52990919777926) bank191928 +191929 POINT(37.168971854943436 -123.23313505226355) bank191929 +191930 POINT(37.80574768322423 -122.36086909097776) bank191930 +191931 POINT(36.896381678102884 -122.76606319645859) bank191931 +191932 POINT(38.134899134144334 -122.89531712857979) bank191932 +191933 POINT(37.96148651100742 -122.89192874011843) bank191933 +191934 POINT(38.77059037362234 -123.19422820597941) bank191934 +191935 POINT(36.7804490264417 -123.35207887175677) bank191935 +191936 POINT(38.45693012725535 -121.557310709939) bank191936 +191937 POINT(37.46226649599932 -121.72724923563783) bank191937 +191938 POINT(37.22485137309698 -121.98750860408087) bank191938 +191939 POINT(37.25597345022835 -123.0869813465502) bank191939 +191940 POINT(37.73281644595933 -122.93377642010134) bank191940 +191941 POINT(37.91387567027429 -123.32167440080364) bank191941 +191942 POINT(38.007002346369234 -122.014685045963) bank191942 +191943 POINT(37.13742038366831 -122.45876247368797) bank191943 +191944 POINT(37.482126012902455 -123.04647322137504) bank191944 +191945 POINT(37.00042438679675 -122.09350460816951) bank191945 +191946 POINT(37.090842047064626 -121.51431195440664) bank191946 +191947 POINT(38.68598715960043 -122.97941284087564) bank191947 +191948 POINT(38.76728998592635 -122.98425021827647) bank191948 +191949 POINT(37.936866569213045 -122.17926860694045) bank191949 +191950 POINT(38.66711679842662 -121.82160696803841) bank191950 +191951 POINT(36.87143817001981 -122.4805060350377) bank191951 +191952 POINT(38.144949302540766 -121.4627296179982) bank191952 +191953 POINT(37.58700192223218 -121.80851426951621) bank191953 +191954 POINT(38.433970274845976 -122.92172065955334) bank191954 +191955 POINT(37.67749385873576 -121.66217524357772) bank191955 +191956 POINT(37.34791551134897 -122.68799637504247) bank191956 +191957 POINT(37.806175712480865 -122.63812635569252) bank191957 +191958 POINT(37.28533546702699 -122.43796839877174) bank191958 +191959 POINT(37.24843794973435 -121.94495726415573) bank191959 +191960 POINT(37.94237529824819 -122.24969412774296) bank191960 +191961 POINT(37.66397811581366 -123.27291533913234) bank191961 +191962 POINT(37.620828918936645 -122.81734779877007) bank191962 +191963 POINT(37.46905781355808 -123.28457096233618) bank191963 +191964 POINT(37.18505504918723 -122.06673994068278) bank191964 +191965 POINT(37.06798195473818 -122.06105094295042) bank191965 +191966 POINT(38.13490030807418 -121.70917668497212) bank191966 +191967 POINT(37.27585440354891 -122.92433119253619) bank191967 +191968 POINT(38.03595323316381 -121.6569456900657) bank191968 +191969 POINT(38.75836286510616 -122.60766648645074) bank191969 +191970 POINT(37.66366819411671 -122.3531868710954) bank191970 +191971 POINT(37.5322196509189 -122.86417480861263) bank191971 +191972 POINT(37.51963269819087 -121.51147731939864) bank191972 +191973 POINT(37.478946125410324 -122.32674202485627) bank191973 +191974 POINT(37.624873804773955 -121.67175687873016) bank191974 +191975 POINT(37.1606019821267 -121.4937776872586) bank191975 +191976 POINT(37.803193846459195 -121.88569017579027) bank191976 +191977 POINT(38.60920737225068 -121.91920585045462) bank191977 +191978 POINT(37.17633959869301 -122.13872029554751) bank191978 +191979 POINT(37.62033306238098 -122.02004463169455) bank191979 +191980 POINT(38.088768692401146 -122.1986731812998) bank191980 +191981 POINT(37.63353177711671 -121.67197985682189) bank191981 +191982 POINT(37.01578934390273 -122.71195623757168) bank191982 +191983 POINT(37.930557172864724 -121.68646128532127) bank191983 +191984 POINT(37.74614544314585 -122.02858115022416) bank191984 +191985 POINT(37.452687713057266 -122.32442668822625) bank191985 +191986 POINT(38.6394196247281 -122.28430621418356) bank191986 +191987 POINT(36.851826955784176 -122.39416754973833) bank191987 +191988 POINT(36.81930190764814 -123.09255246958372) bank191988 +191989 POINT(37.02389482966145 -122.91151747247507) bank191989 +191990 POINT(37.42675001304619 -123.40946211201224) bank191990 +191991 POINT(37.74082378838818 -122.86561607936461) bank191991 +191992 POINT(36.84656241347236 -123.37169953328645) bank191992 +191993 POINT(37.734467863972576 -121.87044119261185) bank191993 +191994 POINT(38.52104049755918 -121.84428102877948) bank191994 +191995 POINT(37.665163672442105 -123.05070731026663) bank191995 +191996 POINT(38.58760429511913 -122.46013274193727) bank191996 +191997 POINT(37.65201232883494 -123.01126634561173) bank191997 +191998 POINT(38.0145229380057 -121.88433374563002) bank191998 +191999 POINT(38.04100232466732 -122.37338061053089) bank191999 +192000 POINT(37.330506650146894 -122.60459926379359) bank192000 +192001 POINT(37.44007768467887 -121.93247029836962) bank192001 +192002 POINT(37.53180595958192 -122.77059064763945) bank192002 +192003 POINT(37.1875392207855 -123.16445501893709) bank192003 +192004 POINT(38.148320140273796 -123.36800566431887) bank192004 +192005 POINT(37.67096830799082 -121.80561003616958) bank192005 +192006 POINT(38.007087827488995 -122.16214594168643) bank192006 +192007 POINT(38.31900515216409 -121.448584803472) bank192007 +192008 POINT(37.45424496196346 -122.46658479615984) bank192008 +192009 POINT(37.443116051284825 -122.8446123209421) bank192009 +192010 POINT(38.474005615912105 -122.69917406562999) bank192010 +192011 POINT(37.83250249922338 -123.08413502840858) bank192011 +192012 POINT(37.83053617660415 -123.06872201248432) bank192012 +192013 POINT(38.14694292262815 -122.0803071307242) bank192013 +192014 POINT(37.68025516723356 -122.0692075616087) bank192014 +192015 POINT(37.31386035725973 -122.90513316177775) bank192015 +192016 POINT(37.72836625934075 -121.9340168448703) bank192016 +192017 POINT(38.121134040712484 -122.81435937533908) bank192017 +192018 POINT(37.39277880969095 -122.46734938539618) bank192018 +192019 POINT(38.65787767950583 -121.44679817823938) bank192019 +192020 POINT(37.24551819609118 -122.41450346156925) bank192020 +192021 POINT(38.569423126643436 -121.70296043193966) bank192021 +192022 POINT(38.626491073252865 -121.43317806822348) bank192022 +192023 POINT(38.662571005904496 -123.34279323539704) bank192023 +192024 POINT(36.88333428435317 -122.31072171558428) bank192024 +192025 POINT(37.15229416278314 -123.28079764660146) bank192025 +192026 POINT(37.30757739086302 -123.28400462314887) bank192026 +192027 POINT(36.88022229138123 -122.09448036144597) bank192027 +192028 POINT(36.96660659010123 -121.83051190799372) bank192028 +192029 POINT(37.04123060300479 -122.45535982062069) bank192029 +192030 POINT(37.5284784621314 -121.49471430440818) bank192030 +192031 POINT(38.48235700668867 -121.5779187128067) bank192031 +192032 POINT(37.95882898110942 -121.88981232438015) bank192032 +192033 POINT(36.94956070015695 -122.89933972896738) bank192033 +192034 POINT(37.5355951495635 -122.47287661768718) bank192034 +192035 POINT(38.375317140809685 -121.48170423797924) bank192035 +192036 POINT(37.22240327136377 -123.10953608742827) bank192036 +192037 POINT(38.493712875705036 -122.28459603057328) bank192037 +192038 POINT(37.623413221136055 -122.31043640678841) bank192038 +192039 POINT(37.68580184414041 -122.08780586143374) bank192039 +192040 POINT(37.43194474519697 -123.11915051671531) bank192040 +192041 POINT(38.436935695124035 -122.72754363218235) bank192041 +192042 POINT(37.434957670825334 -122.2972601954572) bank192042 +192043 POINT(38.44142158706285 -121.4776271761142) bank192043 +192044 POINT(37.017837288612476 -122.73731623239021) bank192044 +192045 POINT(36.83747490225568 -122.11523018258703) bank192045 +192046 POINT(37.83778881147792 -122.23747796388501) bank192046 +192047 POINT(38.34563020761718 -121.67832902884928) bank192047 +192048 POINT(38.255752991967576 -121.47761418453916) bank192048 +192049 POINT(37.14429465710947 -121.5421207356283) bank192049 +192050 POINT(38.29194101229925 -122.81266704141409) bank192050 +192051 POINT(37.27251424820136 -122.56432072408789) bank192051 +192052 POINT(38.491536590038244 -123.20144135762773) bank192052 +192053 POINT(37.14741009367705 -121.95608796012746) bank192053 +192054 POINT(37.25356811777319 -122.46252612958644) bank192054 +192055 POINT(38.54778540247143 -122.7417282392598) bank192055 +192056 POINT(36.824939246194795 -123.23644156098354) bank192056 +192057 POINT(38.41333888823242 -121.44626373927319) bank192057 +192058 POINT(37.574762159375986 -122.3868970886769) bank192058 +192059 POINT(37.02602898471432 -122.04331714533015) bank192059 +192060 POINT(37.3681177686357 -123.05396266597035) bank192060 +192061 POINT(37.40064089441617 -123.10675297847885) bank192061 +192062 POINT(37.4955429084548 -122.33552575962025) bank192062 +192063 POINT(37.634971151997526 -122.17032933524831) bank192063 +192064 POINT(37.45100606009815 -123.21603710845545) bank192064 +192065 POINT(38.723031762441494 -122.86126270598544) bank192065 +192066 POINT(38.493061071775465 -121.99026076231081) bank192066 +192067 POINT(37.79480494391706 -121.63249083177516) bank192067 +192068 POINT(37.611293697734176 -122.37845223284995) bank192068 +192069 POINT(37.0146961100318 -122.77851143556668) bank192069 +192070 POINT(38.646615026964 -122.31174254742312) bank192070 +192071 POINT(37.069885487408726 -122.91590559747071) bank192071 +192072 POINT(37.03833181202274 -122.05062642144603) bank192072 +192073 POINT(37.33668107765903 -123.40680110544224) bank192073 +192074 POINT(38.42518768589076 -121.60369795358551) bank192074 +192075 POINT(37.99310989834771 -122.16957855572718) bank192075 +192076 POINT(36.867137151658326 -123.36263080277435) bank192076 +192077 POINT(38.41537631411206 -123.04456708724048) bank192077 +192078 POINT(38.74328414280474 -122.85224256543748) bank192078 +192079 POINT(37.934569358982586 -122.6030275618551) bank192079 +192080 POINT(38.36047612167302 -122.07536600120532) bank192080 +192081 POINT(37.14757380511776 -121.6944060266804) bank192081 +192082 POINT(37.27051978509007 -121.71668129861969) bank192082 +192083 POINT(36.99967819750198 -121.97858313987021) bank192083 +192084 POINT(38.55441467682623 -123.14061772805016) bank192084 +192085 POINT(38.36737118822362 -123.10664571269243) bank192085 +192086 POINT(37.7695088550616 -121.44080487081811) bank192086 +192087 POINT(37.88547825701522 -122.94265052228093) bank192087 +192088 POINT(38.04233336574524 -121.89988364581883) bank192088 +192089 POINT(38.179211161406215 -122.49900183376158) bank192089 +192090 POINT(38.52915312514052 -123.3294489147055) bank192090 +192091 POINT(38.76599202932947 -123.19123659779616) bank192091 +192092 POINT(37.43906206725649 -121.58265098700183) bank192092 +192093 POINT(38.15383421864666 -122.1045908485697) bank192093 +192094 POINT(38.1204841046754 -123.27810249363748) bank192094 +192095 POINT(38.03094491686894 -122.3882238958757) bank192095 +192096 POINT(38.53345657601508 -122.49218208279903) bank192096 +192097 POINT(37.85070158289631 -121.44742831626301) bank192097 +192098 POINT(37.40317221785346 -122.44929450921093) bank192098 +192099 POINT(37.08634415539347 -121.66108148587256) bank192099 +192100 POINT(38.3312607548135 -122.97573225960504) bank192100 +192101 POINT(37.73081614110773 -122.04602219851594) bank192101 +192102 POINT(38.637922426037505 -122.8544161853386) bank192102 +192103 POINT(38.09971650234259 -122.87710921770947) bank192103 +192104 POINT(38.42995519507732 -122.96142349164946) bank192104 +192105 POINT(37.27851133693437 -123.38988926562286) bank192105 +192106 POINT(38.63717408368756 -122.86286891351106) bank192106 +192107 POINT(38.55443469631489 -122.31659545166382) bank192107 +192108 POINT(37.438898301349184 -123.41580961679857) bank192108 +192109 POINT(37.849186861351626 -122.46077966601028) bank192109 +192110 POINT(38.382262678605926 -123.41046192702468) bank192110 +192111 POINT(37.67060863276235 -121.70547460550247) bank192111 +192112 POINT(37.87743239544542 -121.73695326771484) bank192112 +192113 POINT(38.61323677473037 -122.78371508635246) bank192113 +192114 POINT(36.83563692007639 -122.09942159900649) bank192114 +192115 POINT(38.57903846979109 -121.61237016642977) bank192115 +192116 POINT(38.69311376578437 -122.62511967594727) bank192116 +192117 POINT(36.77610298808055 -121.96502164763736) bank192117 +192118 POINT(37.925147989423685 -121.71594387245126) bank192118 +192119 POINT(38.25727239776906 -123.27497170774294) bank192119 +192120 POINT(37.44827754091953 -121.87441788459239) bank192120 +192121 POINT(37.05181740000344 -122.978897030254) bank192121 +192122 POINT(37.1627191303042 -121.73488409632637) bank192122 +192123 POINT(38.45961699351364 -121.62927130570807) bank192123 +192124 POINT(36.87119459075165 -122.10109613682137) bank192124 +192125 POINT(37.22990496059453 -122.44121976344091) bank192125 +192126 POINT(37.849419038590746 -122.6918863377407) bank192126 +192127 POINT(38.385444795569676 -121.75090425440517) bank192127 +192128 POINT(37.77904447741627 -121.56963001608435) bank192128 +192129 POINT(36.98906404150151 -121.70059130427047) bank192129 +192130 POINT(37.439431560097944 -121.69542897165162) bank192130 +192131 POINT(37.41494678616514 -122.15435537379325) bank192131 +192132 POINT(37.393022127943965 -122.87641266643926) bank192132 +192133 POINT(37.88862885964442 -121.64464237608703) bank192133 +192134 POINT(37.701889300947315 -121.51784242109875) bank192134 +192135 POINT(36.803087394682876 -121.83265375861967) bank192135 +192136 POINT(38.39576627899251 -121.44230893487659) bank192136 +192137 POINT(37.02205660502548 -122.33723005863752) bank192137 +192138 POINT(37.10567639463784 -121.95090439731835) bank192138 +192139 POINT(38.54163249411505 -121.80839839203334) bank192139 +192140 POINT(38.11426057132625 -123.28916985683348) bank192140 +192141 POINT(37.012883378966 -121.74488246164088) bank192141 +192142 POINT(38.61704701537777 -122.31998188620398) bank192142 +192143 POINT(38.6200677693259 -121.99107334849974) bank192143 +192144 POINT(36.87751100901984 -121.96351824426478) bank192144 +192145 POINT(37.14658714641167 -123.21965024342524) bank192145 +192146 POINT(38.67565984599302 -122.85407547589062) bank192146 +192147 POINT(38.555348978203774 -121.92670463154019) bank192147 +192148 POINT(37.103206712640656 -121.64439264746329) bank192148 +192149 POINT(38.383753568252985 -122.51240127511286) bank192149 +192150 POINT(37.17474673080617 -121.68351747018683) bank192150 +192151 POINT(36.93162437260994 -122.70476398079653) bank192151 +192152 POINT(37.33526478196761 -121.5666031162713) bank192152 +192153 POINT(37.1883669959787 -122.54328750578354) bank192153 +192154 POINT(37.79978591931524 -122.5474997185274) bank192154 +192155 POINT(38.206538744140595 -123.35758872482162) bank192155 +192156 POINT(38.68409600244035 -121.85210876858268) bank192156 +192157 POINT(37.37484918564407 -121.9135292126066) bank192157 +192158 POINT(37.29111963801121 -123.06391554458497) bank192158 +192159 POINT(38.54319123735234 -122.43288935118477) bank192159 +192160 POINT(36.7940885767218 -121.51207231326873) bank192160 +192161 POINT(38.34213844677725 -122.32667346148219) bank192161 +192162 POINT(38.15190180964466 -123.03002467884171) bank192162 +192163 POINT(37.75659591341456 -123.16780902063483) bank192163 +192164 POINT(38.64485615996401 -122.59329205880945) bank192164 +192165 POINT(38.60739612095685 -121.79699099517806) bank192165 +192166 POINT(36.782536897056026 -121.73536532516853) bank192166 +192167 POINT(36.87025408757913 -123.24121223877643) bank192167 +192168 POINT(37.857518281249305 -121.4294395875961) bank192168 +192169 POINT(37.50220864544124 -122.72475203062653) bank192169 +192170 POINT(38.132305780183735 -123.1316238505523) bank192170 +192171 POINT(37.455701539515495 -121.85388214345485) bank192171 +192172 POINT(36.861495078566364 -122.19960928379182) bank192172 +192173 POINT(37.268246378703495 -122.95767015590472) bank192173 +192174 POINT(38.43106898758932 -122.44545678865788) bank192174 +192175 POINT(37.65606706492812 -123.24237271844795) bank192175 +192176 POINT(37.34791473102095 -123.33608612115071) bank192176 +192177 POINT(37.87284558922676 -121.74311397117515) bank192177 +192178 POINT(37.5276050876022 -123.40041607730807) bank192178 +192179 POINT(37.82562960079971 -122.3391215093898) bank192179 +192180 POINT(37.81119645610373 -123.36397862018002) bank192180 +192181 POINT(37.915415007295366 -123.33701601447133) bank192181 +192182 POINT(37.80689764016803 -122.73706252493675) bank192182 +192183 POINT(37.95573986684832 -122.95484742629316) bank192183 +192184 POINT(38.091840442783216 -121.84215614852069) bank192184 +192185 POINT(36.8995197827068 -122.16153534443426) bank192185 +192186 POINT(37.92868262195556 -123.34289890931156) bank192186 +192187 POINT(36.775370092206146 -121.85060464735348) bank192187 +192188 POINT(37.56540432278555 -122.85341128881663) bank192188 +192189 POINT(37.329756495995085 -123.2737671828699) bank192189 +192190 POINT(37.08664945333575 -122.54468908941881) bank192190 +192191 POINT(38.066759211032746 -122.47319871832241) bank192191 +192192 POINT(38.49204907916198 -122.38140930096377) bank192192 +192193 POINT(37.87658145636673 -122.5271807383918) bank192193 +192194 POINT(36.96905684265019 -121.71956412408319) bank192194 +192195 POINT(38.649132794513314 -123.0730644654068) bank192195 +192196 POINT(37.538674273611505 -122.8917267345159) bank192196 +192197 POINT(38.16793696829157 -121.56152303322679) bank192197 +192198 POINT(38.010245398283516 -123.18156003676577) bank192198 +192199 POINT(37.49121857542817 -122.88895115472906) bank192199 +192200 POINT(37.01645654174165 -122.56103293510812) bank192200 +192201 POINT(38.40758863149692 -123.30688404216077) bank192201 +192202 POINT(37.940752351897814 -122.0092613573598) bank192202 +192203 POINT(38.63629036075519 -123.16505184408145) bank192203 +192204 POINT(37.37326158411565 -123.19123325047404) bank192204 +192205 POINT(37.981911756269305 -121.73659790550121) bank192205 +192206 POINT(38.238603136755245 -123.39454574023935) bank192206 +192207 POINT(38.40385216012713 -122.87093540461265) bank192207 +192208 POINT(37.823483311250826 -123.37527937029695) bank192208 +192209 POINT(37.54603787093639 -121.9867712317843) bank192209 +192210 POINT(38.55698407984752 -122.83134916957702) bank192210 +192211 POINT(37.23125592253235 -122.73582334268526) bank192211 +192212 POINT(38.4860127225725 -122.11275547852827) bank192212 +192213 POINT(38.73997804883445 -122.04393323197391) bank192213 +192214 POINT(38.116635235345946 -122.3649103295277) bank192214 +192215 POINT(37.4362737093943 -121.77197949485746) bank192215 +192216 POINT(37.819513030900296 -121.79183160007841) bank192216 +192217 POINT(38.51912462815765 -122.57321919811346) bank192217 +192218 POINT(37.13166912286371 -121.55731909165041) bank192218 +192219 POINT(37.75973643127349 -121.50343180952697) bank192219 +192220 POINT(36.89497398724913 -122.30202223197401) bank192220 +192221 POINT(38.37595934375361 -123.40995002274502) bank192221 +192222 POINT(37.71163765102117 -123.08154417351015) bank192222 +192223 POINT(37.42408712945581 -122.50473390455294) bank192223 +192224 POINT(37.16140958638926 -122.18573579741332) bank192224 +192225 POINT(38.23984583659426 -123.37187299238522) bank192225 +192226 POINT(36.96275358023259 -123.21698684997828) bank192226 +192227 POINT(37.9495926387182 -122.8886911358776) bank192227 +192228 POINT(37.79335374469343 -121.8400185374034) bank192228 +192229 POINT(38.17793034169849 -121.86046134754845) bank192229 +192230 POINT(37.21697436451196 -122.76978226323378) bank192230 +192231 POINT(37.75472269735934 -122.27948836254076) bank192231 +192232 POINT(37.84340996061248 -122.74789487411631) bank192232 +192233 POINT(38.45978841156589 -122.33712358611491) bank192233 +192234 POINT(38.350287390703436 -122.54583342652762) bank192234 +192235 POINT(38.669049270442706 -122.62013182559136) bank192235 +192236 POINT(37.40361128067981 -122.671032052518) bank192236 +192237 POINT(36.894597404170476 -123.24234240982898) bank192237 +192238 POINT(37.79965436112981 -122.42697212218906) bank192238 +192239 POINT(38.273785353662014 -122.47795233282548) bank192239 +192240 POINT(38.70521909535609 -122.78738705339642) bank192240 +192241 POINT(37.0635834900791 -122.24410912804801) bank192241 +192242 POINT(37.29807887471796 -121.93480533850756) bank192242 +192243 POINT(37.55679815172359 -122.47205797060187) bank192243 +192244 POINT(38.76717277202179 -121.67472658604301) bank192244 +192245 POINT(37.40385423764297 -122.02112620650308) bank192245 +192246 POINT(38.63900978390702 -123.40483571533383) bank192246 +192247 POINT(37.85458573673549 -122.02539191747086) bank192247 +192248 POINT(36.7766821820964 -122.37452641616706) bank192248 +192249 POINT(38.26208401690233 -122.75260197505348) bank192249 +192250 POINT(37.74879778851508 -121.53909536762941) bank192250 +192251 POINT(37.64976485251998 -123.3519262868484) bank192251 +192252 POINT(38.0645513999483 -123.02957253313497) bank192252 +192253 POINT(38.18819491783678 -121.97638926108152) bank192253 +192254 POINT(36.8385550936866 -121.80108298294981) bank192254 +192255 POINT(37.54951026525629 -121.43869850629027) bank192255 +192256 POINT(38.73395857397758 -122.51323479470453) bank192256 +192257 POINT(38.20981703340258 -122.71060826071343) bank192257 +192258 POINT(37.179931843627955 -122.54891355487106) bank192258 +192259 POINT(37.26141560218899 -122.7632033728332) bank192259 +192260 POINT(38.62847064550597 -123.17336192704377) bank192260 +192261 POINT(37.975622186652984 -121.79571625976818) bank192261 +192262 POINT(37.03243151554777 -121.47309757496562) bank192262 +192263 POINT(37.13778967815158 -122.60446473689301) bank192263 +192264 POINT(38.654436225386846 -123.0968547040612) bank192264 +192265 POINT(38.15599336674294 -123.07463940031028) bank192265 +192266 POINT(37.44202866945762 -123.29080943625289) bank192266 +192267 POINT(38.247624769987766 -121.89166346206834) bank192267 +192268 POINT(37.64495661047778 -122.55990647979613) bank192268 +192269 POINT(37.80865015552195 -121.54543234270955) bank192269 +192270 POINT(38.62596087593174 -122.78558712217715) bank192270 +192271 POINT(38.25750994799273 -123.35820269526879) bank192271 +192272 POINT(37.78244059267604 -121.88006895814786) bank192272 +192273 POINT(36.89927877180424 -123.156366628542) bank192273 +192274 POINT(38.600088848767015 -122.47712695443919) bank192274 +192275 POINT(37.789804694267225 -122.83300456598957) bank192275 +192276 POINT(37.418812750189744 -122.37030730337638) bank192276 +192277 POINT(37.32484662047896 -121.80380265187375) bank192277 +192278 POINT(36.95983026339265 -122.86494695057903) bank192278 +192279 POINT(36.844164037374185 -123.04866508778193) bank192279 +192280 POINT(37.083427381408164 -122.1676712572012) bank192280 +192281 POINT(37.52129783274409 -123.01455295602769) bank192281 +192282 POINT(37.55620503849606 -122.58824315325187) bank192282 +192283 POINT(38.46644989206152 -122.63778960428444) bank192283 +192284 POINT(36.89882987021695 -122.64669487400518) bank192284 +192285 POINT(37.210051977728604 -122.94559510474654) bank192285 +192286 POINT(37.98710158206782 -122.42935361839704) bank192286 +192287 POINT(38.16736324590917 -122.80272534820729) bank192287 +192288 POINT(37.943652556476934 -121.95629405878188) bank192288 +192289 POINT(37.44317584270807 -123.32937288549016) bank192289 +192290 POINT(36.87238990254763 -123.31101442606499) bank192290 +192291 POINT(37.92535492507315 -123.33749777598703) bank192291 +192292 POINT(37.80726610923451 -122.28166965800759) bank192292 +192293 POINT(37.33705377257314 -121.96272722304072) bank192293 +192294 POINT(37.58150202322984 -122.91291046889026) bank192294 +192295 POINT(36.809672398481254 -122.7019867828966) bank192295 +192296 POINT(38.599666634245736 -122.43610486478285) bank192296 +192297 POINT(38.15483943748468 -123.04697637245576) bank192297 +192298 POINT(37.24802162091318 -122.21937774813719) bank192298 +192299 POINT(38.43910273277981 -122.36659607274397) bank192299 +192300 POINT(38.53374640348203 -121.9207884411586) bank192300 +192301 POINT(37.70746362984761 -123.41037268630731) bank192301 +192302 POINT(37.735475338268365 -122.91481987303473) bank192302 +192303 POINT(38.725878570954656 -121.64296936859772) bank192303 +192304 POINT(37.298391044730785 -122.4569709972684) bank192304 +192305 POINT(37.33366181632574 -122.54605475722076) bank192305 +192306 POINT(37.500860304564135 -123.14880060535154) bank192306 +192307 POINT(36.99648799779932 -122.30505363369302) bank192307 +192308 POINT(38.73034108552385 -122.2978075822186) bank192308 +192309 POINT(38.09649054509138 -123.26399099905267) bank192309 +192310 POINT(38.29362898493693 -122.86220079778643) bank192310 +192311 POINT(37.32740202764154 -121.82573980941348) bank192311 +192312 POINT(37.399147468164784 -121.43640454268854) bank192312 +192313 POINT(36.93699717727414 -123.28646415404891) bank192313 +192314 POINT(37.987557911072464 -122.16228284552484) bank192314 +192315 POINT(38.36445757023429 -122.33677680838899) bank192315 +192316 POINT(37.630439610894875 -122.58472335877227) bank192316 +192317 POINT(38.50684534058534 -121.71140890149933) bank192317 +192318 POINT(38.12684726391713 -122.87218428540368) bank192318 +192319 POINT(38.76032665491633 -123.04310858619253) bank192319 +192320 POINT(37.32630867438501 -122.32408418064922) bank192320 +192321 POINT(38.289359291133714 -122.51441613224996) bank192321 +192322 POINT(38.68519472305596 -123.34968000680676) bank192322 +192323 POINT(37.574602272025984 -121.65412333898841) bank192323 +192324 POINT(38.417564898013275 -123.40539909384098) bank192324 +192325 POINT(37.17834026218317 -122.70465812830848) bank192325 +192326 POINT(38.2104376243952 -121.45904748769294) bank192326 +192327 POINT(37.70213768199226 -122.53282316671927) bank192327 +192328 POINT(36.78397013733148 -123.39051712645302) bank192328 +192329 POINT(38.32326391335316 -122.35492130456785) bank192329 +192330 POINT(38.16291176762305 -123.40921592599248) bank192330 +192331 POINT(38.30444431417068 -122.32395212052474) bank192331 +192332 POINT(38.615635971226986 -121.6361561021088) bank192332 +192333 POINT(38.1935016416553 -121.56454115193735) bank192333 +192334 POINT(37.910070272081065 -121.48128956243285) bank192334 +192335 POINT(37.0326283879171 -122.04639238774745) bank192335 +192336 POINT(37.793862870121316 -122.5826432556483) bank192336 +192337 POINT(38.06941952755477 -122.6905772116945) bank192337 +192338 POINT(38.15672355193395 -122.31870558846975) bank192338 +192339 POINT(36.776041310386226 -123.28280447337877) bank192339 +192340 POINT(37.36676915563638 -123.21442888681312) bank192340 +192341 POINT(38.05118664216547 -123.03830175714492) bank192341 +192342 POINT(37.500138401273006 -122.84017934985073) bank192342 +192343 POINT(37.24294977292923 -122.46364046387069) bank192343 +192344 POINT(37.592977090826025 -122.54085783010777) bank192344 +192345 POINT(37.96982369798792 -122.2923249418294) bank192345 +192346 POINT(37.964274170389075 -123.00713661343302) bank192346 +192347 POINT(37.615528095738895 -122.64859243671623) bank192347 +192348 POINT(37.94950703532487 -123.18188636060665) bank192348 +192349 POINT(38.088341015691235 -121.95287966653335) bank192349 +192350 POINT(37.193490593749 -122.74990749607792) bank192350 +192351 POINT(37.740009206113115 -121.9845966425212) bank192351 +192352 POINT(38.25023825340053 -123.20307720374029) bank192352 +192353 POINT(37.34645465445065 -122.2295389575225) bank192353 +192354 POINT(38.12114924533096 -122.18094895752448) bank192354 +192355 POINT(37.37377966028333 -122.24013713013323) bank192355 +192356 POINT(37.51151584333686 -122.2005953100536) bank192356 +192357 POINT(37.571098389723346 -123.30677408291986) bank192357 +192358 POINT(37.79659483082913 -122.00332001875324) bank192358 +192359 POINT(36.78349311837912 -123.20999665099914) bank192359 +192360 POINT(37.94505900934795 -121.727905470004) bank192360 +192361 POINT(38.1454028492354 -122.69160441582413) bank192361 +192362 POINT(37.52335141910869 -123.2275709163357) bank192362 +192363 POINT(37.32869136243331 -122.30003051995199) bank192363 +192364 POINT(38.01383223291981 -122.69450749172933) bank192364 +192365 POINT(38.25685563687917 -123.31901063459897) bank192365 +192366 POINT(37.240162456957904 -122.70506734971842) bank192366 +192367 POINT(38.01291336378767 -123.29808027684342) bank192367 +192368 POINT(38.16898393031553 -121.68726505019617) bank192368 +192369 POINT(37.55913143611263 -122.19224657711548) bank192369 +192370 POINT(36.794345198844695 -121.86781306169134) bank192370 +192371 POINT(38.1848973675985 -121.79825757414449) bank192371 +192372 POINT(37.21422032769751 -121.44445206152145) bank192372 +192373 POINT(37.052312287705725 -121.68360386975216) bank192373 +192374 POINT(38.55466253879149 -123.30394056886155) bank192374 +192375 POINT(37.00425054799511 -122.6304075116673) bank192375 +192376 POINT(37.539248126124065 -121.46797275998426) bank192376 +192377 POINT(36.823653254908585 -121.91861828044782) bank192377 +192378 POINT(36.92183377268105 -121.66923589417974) bank192378 +192379 POINT(38.27516353985723 -122.92287116594419) bank192379 +192380 POINT(36.781730774200774 -121.61898318006082) bank192380 +192381 POINT(37.54176868500435 -123.38834039197516) bank192381 +192382 POINT(37.156698925534904 -122.21733686796819) bank192382 +192383 POINT(38.36535563734708 -122.02580456784573) bank192383 +192384 POINT(37.74368439893605 -122.61765603645732) bank192384 +192385 POINT(38.40472394423054 -122.00384486298051) bank192385 +192386 POINT(38.61026245506014 -121.85647797004015) bank192386 +192387 POINT(37.30632775365096 -121.79553361165316) bank192387 +192388 POINT(37.52993347556073 -122.1464915188339) bank192388 +192389 POINT(37.41867554391815 -123.2428360800554) bank192389 +192390 POINT(38.258856441505074 -121.91536138241925) bank192390 +192391 POINT(38.411079236512144 -121.92661265863644) bank192391 +192392 POINT(37.04462562101195 -122.2240596044121) bank192392 +192393 POINT(37.816899590785816 -123.1339583505397) bank192393 +192394 POINT(38.26635711507329 -121.91766193777012) bank192394 +192395 POINT(38.30061391027579 -122.21774177300097) bank192395 +192396 POINT(38.432208703350994 -123.25078752733387) bank192396 +192397 POINT(37.235862866629226 -122.88938208444337) bank192397 +192398 POINT(38.0431893752712 -121.75942712148797) bank192398 +192399 POINT(38.554673679548145 -122.62736023984613) bank192399 +192400 POINT(36.93059187001414 -122.06821212911971) bank192400 +192401 POINT(38.77443413790572 -122.15628563535493) bank192401 +192402 POINT(36.8872672053751 -123.18205643576081) bank192402 +192403 POINT(37.114195896002556 -122.19020568007102) bank192403 +192404 POINT(38.045988115543395 -121.5929848349687) bank192404 +192405 POINT(38.10617595271639 -122.88367286615627) bank192405 +192406 POINT(36.897644793552956 -123.23351783250654) bank192406 +192407 POINT(36.965043300621545 -122.13398891586677) bank192407 +192408 POINT(38.09882956886062 -122.3649108819233) bank192408 +192409 POINT(37.97545905564306 -122.65735000561489) bank192409 +192410 POINT(36.988058046156674 -121.76331775509749) bank192410 +192411 POINT(38.16855492427809 -121.69454837790565) bank192411 +192412 POINT(36.804705924951314 -123.28970196873472) bank192412 +192413 POINT(37.46849005883258 -121.7840211572844) bank192413 +192414 POINT(37.729656634747755 -122.92552327238865) bank192414 +192415 POINT(37.61644382281736 -122.44287252688281) bank192415 +192416 POINT(37.653993666938845 -122.94625486937683) bank192416 +192417 POINT(37.939110867570285 -123.28268387114527) bank192417 +192418 POINT(37.524415407105884 -121.82053625959182) bank192418 +192419 POINT(37.75155087350632 -122.22882175802552) bank192419 +192420 POINT(37.54179781712569 -123.41696861194775) bank192420 +192421 POINT(38.36758851756971 -121.66671795763655) bank192421 +192422 POINT(37.54079380028283 -122.07608320484704) bank192422 +192423 POINT(38.19028129549116 -122.72836041575054) bank192423 +192424 POINT(38.32061163913489 -123.1822917167142) bank192424 +192425 POINT(37.85068338326969 -122.33926260467209) bank192425 +192426 POINT(37.213687466500964 -123.34751392485346) bank192426 +192427 POINT(37.48900955895372 -121.85588431163005) bank192427 +192428 POINT(37.202450795693 -123.20981540251866) bank192428 +192429 POINT(38.66540083775553 -121.81470443369055) bank192429 +192430 POINT(38.46980734658913 -122.1117864692594) bank192430 +192431 POINT(38.372457856188205 -122.4484240318829) bank192431 +192432 POINT(38.483637132494486 -123.09554445115914) bank192432 +192433 POINT(37.23993675778652 -123.06501894241121) bank192433 +192434 POINT(36.90233375446136 -122.81450974178999) bank192434 +192435 POINT(37.95075923653171 -121.54867978490766) bank192435 +192436 POINT(38.35887407989865 -122.01598163277106) bank192436 +192437 POINT(37.26576485193386 -121.42241560275211) bank192437 +192438 POINT(37.447233699101645 -122.03457080630515) bank192438 +192439 POINT(37.03214638270507 -123.23313690239374) bank192439 +192440 POINT(37.07183453320616 -121.925483070491) bank192440 +192441 POINT(38.32464691784136 -123.35241421271118) bank192441 +192442 POINT(38.73783152953346 -123.10282193321608) bank192442 +192443 POINT(36.832816421743736 -122.81601137358253) bank192443 +192444 POINT(37.72702750754006 -123.28220652684635) bank192444 +192445 POINT(37.74571773361662 -123.27794718786868) bank192445 +192446 POINT(38.45828178226552 -123.07657472285928) bank192446 +192447 POINT(37.032616910671464 -121.47228163073629) bank192447 +192448 POINT(36.883730388410214 -122.22190724151486) bank192448 +192449 POINT(36.894802778483715 -122.96880085092879) bank192449 +192450 POINT(37.567637899151485 -123.29003648050752) bank192450 +192451 POINT(37.54726425323255 -122.5001815032019) bank192451 +192452 POINT(37.48920013631869 -121.92821854518634) bank192452 +192453 POINT(37.951303861781135 -122.57871580523357) bank192453 +192454 POINT(37.455901090781744 -123.36363943938854) bank192454 +192455 POINT(37.74300358717426 -122.52189772529759) bank192455 +192456 POINT(37.63302085885198 -123.18838674504835) bank192456 +192457 POINT(37.40029134761812 -123.41088355869582) bank192457 +192458 POINT(38.564773281379786 -122.07033417003728) bank192458 +192459 POINT(38.18169359303487 -121.78873021027749) bank192459 +192460 POINT(38.40963402516643 -121.89376121108373) bank192460 +192461 POINT(37.40339059727855 -122.12137287476459) bank192461 +192462 POINT(37.350295248960414 -123.36354402250407) bank192462 +192463 POINT(37.98073086191167 -123.0460437116707) bank192463 +192464 POINT(38.42337171079296 -123.26887304811547) bank192464 +192465 POINT(37.40475402362503 -122.3094691349681) bank192465 +192466 POINT(37.67325073135076 -121.79474627993487) bank192466 +192467 POINT(37.10999790696848 -122.99767958866364) bank192467 +192468 POINT(37.42021137332496 -122.21389696234321) bank192468 +192469 POINT(38.303757406367474 -122.12288235080767) bank192469 +192470 POINT(37.23754662284376 -123.3725525266929) bank192470 +192471 POINT(38.51772438966758 -123.12538615971125) bank192471 +192472 POINT(38.217484400809695 -123.24602159129319) bank192472 +192473 POINT(38.748929944555705 -122.13065336260534) bank192473 +192474 POINT(38.07959420866642 -123.03712731315322) bank192474 +192475 POINT(37.46074335062722 -122.07298620450786) bank192475 +192476 POINT(38.25114356683704 -123.38844788037095) bank192476 +192477 POINT(38.77126829752403 -121.74329010583155) bank192477 +192478 POINT(38.708255528889 -122.2206600152399) bank192478 +192479 POINT(37.704051692052566 -121.65010308186699) bank192479 +192480 POINT(38.66864225485778 -123.00886157431185) bank192480 +192481 POINT(38.29911691039617 -121.59559799680298) bank192481 +192482 POINT(37.48913524710174 -121.7372721341278) bank192482 +192483 POINT(37.40806629881826 -123.12352991794982) bank192483 +192484 POINT(38.68964887842577 -121.72740678927275) bank192484 +192485 POINT(38.45366854212949 -121.43232177030248) bank192485 +192486 POINT(38.32018317451936 -121.99702589744511) bank192486 +192487 POINT(38.575480037452486 -122.59363786322744) bank192487 +192488 POINT(36.7871945883015 -122.05628744310036) bank192488 +192489 POINT(38.69535444879634 -122.87793380507155) bank192489 +192490 POINT(37.14119119840737 -122.1110822204985) bank192490 +192491 POINT(37.812641214535304 -121.43056104072598) bank192491 +192492 POINT(37.52187373287388 -123.05937539830268) bank192492 +192493 POINT(37.25562470440472 -122.8360857822837) bank192493 +192494 POINT(36.956253546222406 -122.4450985492739) bank192494 +192495 POINT(38.019932678436206 -122.23756815854216) bank192495 +192496 POINT(38.599000548701085 -122.52259994130908) bank192496 +192497 POINT(37.920144871879224 -121.87980185493119) bank192497 +192498 POINT(37.16137343557343 -122.45903697963281) bank192498 +192499 POINT(37.41047475462534 -122.83750479595017) bank192499 +192500 POINT(37.01719413795877 -122.45297904709078) bank192500 +192501 POINT(37.73147191818379 -123.05115345117834) bank192501 +192502 POINT(36.97065290387925 -121.89896212193489) bank192502 +192503 POINT(38.34200124899003 -123.1213268196657) bank192503 +192504 POINT(37.69208654562756 -122.74907879588773) bank192504 +192505 POINT(38.41213539874305 -122.98007308683955) bank192505 +192506 POINT(37.18767599917331 -123.04272154324164) bank192506 +192507 POINT(36.90964691783848 -122.5973285585507) bank192507 +192508 POINT(37.99955753379363 -121.96202507312817) bank192508 +192509 POINT(37.305606394509 -122.11228305612092) bank192509 +192510 POINT(37.89053959070559 -123.32844803560297) bank192510 +192511 POINT(37.58431326394028 -121.54194943510409) bank192511 +192512 POINT(38.57398825600496 -122.32985473230195) bank192512 +192513 POINT(37.05356347403823 -122.26294517868628) bank192513 +192514 POINT(37.31372661574111 -121.58796473135874) bank192514 +192515 POINT(38.6593579679953 -122.82036835748026) bank192515 +192516 POINT(37.363177712066346 -121.54362331365456) bank192516 +192517 POINT(37.91794033111992 -122.96729375219438) bank192517 +192518 POINT(38.70911759307448 -123.27881920541942) bank192518 +192519 POINT(38.43264624041414 -122.15845864948625) bank192519 +192520 POINT(38.66494475222745 -122.65770769601836) bank192520 +192521 POINT(38.424108433928524 -121.5888998306425) bank192521 +192522 POINT(38.20737883641142 -121.79476404673049) bank192522 +192523 POINT(36.89985452310469 -121.60278401784234) bank192523 +192524 POINT(37.04664374634968 -121.82952904992135) bank192524 +192525 POINT(36.947127639387695 -122.77955293069522) bank192525 +192526 POINT(36.831284737802584 -123.05274744327983) bank192526 +192527 POINT(37.1829600834385 -122.72030141618298) bank192527 +192528 POINT(38.154004370563 -121.80050560393296) bank192528 +192529 POINT(38.237282795925594 -122.58510980730794) bank192529 +192530 POINT(38.66603203574468 -121.67597409066782) bank192530 +192531 POINT(37.2799774863551 -121.61860947514229) bank192531 +192532 POINT(38.71205677673703 -122.36707197139422) bank192532 +192533 POINT(38.658520275615224 -122.10093863779952) bank192533 +192534 POINT(36.942655359444686 -121.42957505710669) bank192534 +192535 POINT(38.14453251080277 -121.7574067406364) bank192535 +192536 POINT(37.03079852191433 -122.47077478414204) bank192536 +192537 POINT(36.875101665560344 -122.84693855410016) bank192537 +192538 POINT(36.99427986516016 -122.16027944572993) bank192538 +192539 POINT(37.4654982206608 -122.1751679800842) bank192539 +192540 POINT(38.14178966469818 -122.28235908064795) bank192540 +192541 POINT(38.77418458970514 -122.27674661493081) bank192541 +192542 POINT(36.97893350519596 -121.50118568576117) bank192542 +192543 POINT(37.83207799885623 -121.44223311955453) bank192543 +192544 POINT(38.04786656690714 -122.78165228812432) bank192544 +192545 POINT(38.18500632469747 -122.98556791206573) bank192545 +192546 POINT(38.59428723182954 -121.62044553162751) bank192546 +192547 POINT(37.134529034648835 -121.88382741938213) bank192547 +192548 POINT(37.021031097255516 -123.29755857105883) bank192548 +192549 POINT(36.78685843806257 -122.78718996783947) bank192549 +192550 POINT(38.47842537139133 -122.05764136855518) bank192550 +192551 POINT(37.81162442960861 -122.41355355499155) bank192551 +192552 POINT(38.27360940992836 -122.90693829509551) bank192552 +192553 POINT(37.9599795893166 -122.7481332107677) bank192553 +192554 POINT(37.759734692803434 -122.23002222713687) bank192554 +192555 POINT(38.29439124654542 -122.39751507983998) bank192555 +192556 POINT(38.246689214729514 -122.91430629825427) bank192556 +192557 POINT(37.395426955245895 -121.73780766460422) bank192557 +192558 POINT(36.8576413821811 -121.54631740781531) bank192558 +192559 POINT(37.03306721265087 -121.50750812225445) bank192559 +192560 POINT(37.98208406910706 -121.88220803172547) bank192560 +192561 POINT(37.34492662496236 -122.04429595019015) bank192561 +192562 POINT(37.81511694753718 -121.43656135623017) bank192562 +192563 POINT(38.16930890148675 -122.46147883899519) bank192563 +192564 POINT(37.70540680160083 -122.86827602115541) bank192564 +192565 POINT(37.125609028708304 -123.30307138818762) bank192565 +192566 POINT(37.78677368220754 -123.17181531909085) bank192566 +192567 POINT(38.53042548121299 -121.47752181343468) bank192567 +192568 POINT(37.53131279880233 -122.32017252277026) bank192568 +192569 POINT(37.410693433413016 -122.8657453189465) bank192569 +192570 POINT(38.68166035635293 -121.89812941168658) bank192570 +192571 POINT(37.29705756050724 -122.03546791630512) bank192571 +192572 POINT(37.389624716942365 -121.94742792418255) bank192572 +192573 POINT(38.44268415840857 -122.4313182699437) bank192573 +192574 POINT(37.97147103581348 -122.73123904677297) bank192574 +192575 POINT(37.48984641429698 -122.15657921825333) bank192575 +192576 POINT(37.732207099832394 -122.02720810094253) bank192576 +192577 POINT(37.38204681411903 -123.27829670275328) bank192577 +192578 POINT(38.65800717697828 -121.48533668295029) bank192578 +192579 POINT(37.01806261060904 -123.19876890120456) bank192579 +192580 POINT(37.49392900120625 -122.51906891728632) bank192580 +192581 POINT(37.132289423918074 -122.58475241489768) bank192581 +192582 POINT(37.968284712422445 -122.86234944569745) bank192582 +192583 POINT(37.55021293890155 -123.27948087129724) bank192583 +192584 POINT(37.748544483635 -122.92837186548819) bank192584 +192585 POINT(38.73429942319278 -123.15259840908601) bank192585 +192586 POINT(37.94023416036664 -122.3708129222422) bank192586 +192587 POINT(37.19370133860616 -122.92537604093353) bank192587 +192588 POINT(38.23844703179551 -123.03134369206387) bank192588 +192589 POINT(38.25816748610942 -121.8014669223935) bank192589 +192590 POINT(38.30877795982238 -121.95614237955985) bank192590 +192591 POINT(37.590609265704764 -122.82401907795031) bank192591 +192592 POINT(37.740005663647004 -121.53454557807108) bank192592 +192593 POINT(37.590961391277084 -121.74271533998804) bank192593 +192594 POINT(38.251904131952585 -122.97308563761486) bank192594 +192595 POINT(37.24100542611979 -122.20726106719613) bank192595 +192596 POINT(38.562816450904116 -122.8981203850894) bank192596 +192597 POINT(38.58372400684967 -122.62104291260918) bank192597 +192598 POINT(37.90057141851467 -123.04891480395206) bank192598 +192599 POINT(38.36463278112591 -121.64553539179288) bank192599 +192600 POINT(37.08199745677678 -121.74011227853103) bank192600 +192601 POINT(38.20000210686484 -121.67696019350754) bank192601 +192602 POINT(36.9264402345306 -122.43069721445701) bank192602 +192603 POINT(37.8701485609292 -122.38821878084073) bank192603 +192604 POINT(38.44479571500182 -122.70615939863309) bank192604 +192605 POINT(38.06640740273567 -123.35588035062025) bank192605 +192606 POINT(36.86263754882927 -123.36194423473732) bank192606 +192607 POINT(37.61324946406263 -122.73961454619501) bank192607 +192608 POINT(37.676186946582035 -122.64601653820097) bank192608 +192609 POINT(38.54539768448784 -121.91290994282173) bank192609 +192610 POINT(38.57863021673826 -122.94172461341668) bank192610 +192611 POINT(36.94384472527382 -121.60402588426886) bank192611 +192612 POINT(38.36267371793215 -122.48283839848814) bank192612 +192613 POINT(38.4490674517061 -122.54006687213565) bank192613 +192614 POINT(38.687255215534684 -123.11971440017889) bank192614 +192615 POINT(37.42456284354786 -121.66860947183203) bank192615 +192616 POINT(37.76292688093821 -123.14856149099303) bank192616 +192617 POINT(38.03053218704774 -121.7885197070969) bank192617 +192618 POINT(37.3086545406279 -122.78500241903922) bank192618 +192619 POINT(37.6391542922972 -122.84323878803787) bank192619 +192620 POINT(38.20847273831264 -122.40917751197998) bank192620 +192621 POINT(37.70077161946282 -121.44547973503845) bank192621 +192622 POINT(38.51561832195061 -123.33377671218793) bank192622 +192623 POINT(38.11625606911891 -122.06274034768468) bank192623 +192624 POINT(37.62896156501169 -122.34599416046208) bank192624 +192625 POINT(38.21270867835516 -122.5670988787751) bank192625 +192626 POINT(38.11793082365054 -122.56306683859005) bank192626 +192627 POINT(38.637073789181564 -121.66313034306816) bank192627 +192628 POINT(38.75792073637936 -123.23917743216403) bank192628 +192629 POINT(37.71756761940915 -122.40336622448636) bank192629 +192630 POINT(37.00868360391853 -122.99369308740505) bank192630 +192631 POINT(37.528771446000455 -121.48398261347552) bank192631 +192632 POINT(38.693400546655525 -122.45553185067176) bank192632 +192633 POINT(37.62975442988522 -123.41066640066929) bank192633 +192634 POINT(36.96225831816068 -122.92864365022949) bank192634 +192635 POINT(37.270918150226606 -121.54168955380545) bank192635 +192636 POINT(37.49538310031169 -123.00446855870018) bank192636 +192637 POINT(37.719650491720245 -121.53965565659686) bank192637 +192638 POINT(38.724905887686 -123.1208176916272) bank192638 +192639 POINT(38.01642786908522 -122.18098373588883) bank192639 +192640 POINT(37.484317693060866 -121.51355819151958) bank192640 +192641 POINT(38.43428661413267 -122.95160869038133) bank192641 +192642 POINT(38.348945163125414 -123.19464749444607) bank192642 +192643 POINT(37.99568893922718 -122.2413139594009) bank192643 +192644 POINT(38.27886778399673 -122.01137345527243) bank192644 +192645 POINT(37.07561990265004 -122.45995116067297) bank192645 +192646 POINT(37.101759075289905 -122.62151653016872) bank192646 +192647 POINT(37.68604391557508 -122.25170100669116) bank192647 +192648 POINT(38.74020895779414 -123.1975422929241) bank192648 +192649 POINT(37.62390875304701 -122.40026295420853) bank192649 +192650 POINT(38.23125488686708 -123.01792107273698) bank192650 +192651 POINT(38.6001538962946 -122.70477339558158) bank192651 +192652 POINT(38.44699836380138 -122.56707421264353) bank192652 +192653 POINT(36.97388947230894 -121.54457368067112) bank192653 +192654 POINT(37.20170660701399 -123.17258979419373) bank192654 +192655 POINT(37.140326412678334 -122.43879715500529) bank192655 +192656 POINT(37.56360328723653 -123.21132360289577) bank192656 +192657 POINT(38.63407643136434 -122.38462921107669) bank192657 +192658 POINT(36.87549754575467 -123.12020488081451) bank192658 +192659 POINT(37.71670982290679 -121.98987901917793) bank192659 +192660 POINT(38.44724308909498 -121.76269628056883) bank192660 +192661 POINT(37.81682643575783 -123.33567754117628) bank192661 +192662 POINT(36.87619568402082 -122.6818239371225) bank192662 +192663 POINT(37.418956199786024 -121.50817739894042) bank192663 +192664 POINT(37.08503472538545 -123.38728025508367) bank192664 +192665 POINT(37.68095015160005 -121.82721391821745) bank192665 +192666 POINT(38.76612945566829 -123.3538683090398) bank192666 +192667 POINT(36.90297479241321 -121.62562425807278) bank192667 +192668 POINT(38.43057582656553 -123.39009712893164) bank192668 +192669 POINT(38.46162197849319 -121.5184148891756) bank192669 +192670 POINT(37.623843438671955 -123.23070142210747) bank192670 +192671 POINT(38.5480970298006 -122.93745908858654) bank192671 +192672 POINT(37.66125942122274 -121.92683566597894) bank192672 +192673 POINT(38.61208937409854 -122.33715428408696) bank192673 +192674 POINT(37.60958347561264 -122.70752354935242) bank192674 +192675 POINT(38.01784815956123 -121.95112197239943) bank192675 +192676 POINT(37.669264247902134 -122.19890218679113) bank192676 +192677 POINT(38.67762239823801 -121.53547979647843) bank192677 +192678 POINT(37.96394412205526 -122.5538049777524) bank192678 +192679 POINT(37.8252223394077 -123.27705239002238) bank192679 +192680 POINT(37.43120994723725 -122.7500874548572) bank192680 +192681 POINT(37.04892712794308 -121.97168996003191) bank192681 +192682 POINT(38.52604879116506 -122.94448501337787) bank192682 +192683 POINT(37.59549679707249 -122.82382692321589) bank192683 +192684 POINT(38.60367852213659 -122.43046771650756) bank192684 +192685 POINT(37.77600997346197 -121.72448769592081) bank192685 +192686 POINT(37.82448628790056 -122.53576929915862) bank192686 +192687 POINT(38.151416974466386 -122.52675206217486) bank192687 +192688 POINT(38.487845373187376 -121.49080823700832) bank192688 +192689 POINT(37.48281851988094 -122.4394451819615) bank192689 +192690 POINT(37.33478807137267 -121.92053188673961) bank192690 +192691 POINT(38.13262057565303 -121.85137414830133) bank192691 +192692 POINT(37.44560348626865 -121.74492472899028) bank192692 +192693 POINT(38.594769283516406 -121.64467682297364) bank192693 +192694 POINT(36.914444977624164 -121.7325386783096) bank192694 +192695 POINT(37.48521784290379 -121.65641374085581) bank192695 +192696 POINT(37.4959404795082 -123.12868317190487) bank192696 +192697 POINT(38.3863273740641 -121.60399917220964) bank192697 +192698 POINT(37.83480636895912 -122.91692699112433) bank192698 +192699 POINT(37.911667244815106 -123.04370904738037) bank192699 +192700 POINT(38.691165822413325 -122.81485864361866) bank192700 +192701 POINT(37.496511651252945 -121.52217452631542) bank192701 +192702 POINT(37.39568458175034 -121.91123058698739) bank192702 +192703 POINT(37.38163458030274 -123.19602198418242) bank192703 +192704 POINT(38.58860480822625 -121.57926309254304) bank192704 +192705 POINT(37.55777365963203 -121.91541213718456) bank192705 +192706 POINT(38.1835312006354 -122.2993528490454) bank192706 +192707 POINT(36.78067897882588 -121.86710672418873) bank192707 +192708 POINT(38.54332495526608 -122.6805247580138) bank192708 +192709 POINT(37.53161665347681 -122.57667775820221) bank192709 +192710 POINT(37.53164281906008 -122.64621424174308) bank192710 +192711 POINT(37.0731252629331 -122.29704303633912) bank192711 +192712 POINT(36.88690645423044 -122.61321454044366) bank192712 +192713 POINT(38.738854448755085 -123.00613106961845) bank192713 +192714 POINT(38.52129806767296 -122.86178882297104) bank192714 +192715 POINT(37.705218119851686 -122.93573607355955) bank192715 +192716 POINT(38.275000862453425 -123.22775901073344) bank192716 +192717 POINT(37.179133462945906 -122.3748847584082) bank192717 +192718 POINT(37.47141883889421 -122.1848272606796) bank192718 +192719 POINT(37.24920831390667 -123.17523503204609) bank192719 +192720 POINT(37.30484979102155 -123.14434552523493) bank192720 +192721 POINT(37.75087796952839 -122.04533687282822) bank192721 +192722 POINT(38.58216249974056 -121.47325645269703) bank192722 +192723 POINT(37.90338187941085 -122.1766934622712) bank192723 +192724 POINT(38.05608010383344 -122.70607184987813) bank192724 +192725 POINT(37.197943317249624 -122.83238483057131) bank192725 +192726 POINT(38.22333871177844 -121.99561996585399) bank192726 +192727 POINT(38.61684171821394 -122.1392836574866) bank192727 +192728 POINT(38.52350346999874 -122.4217945046967) bank192728 +192729 POINT(37.81661875033709 -121.89172451256879) bank192729 +192730 POINT(38.68322733540176 -123.0874735077555) bank192730 +192731 POINT(38.643644404907796 -122.29407778179625) bank192731 +192732 POINT(38.68957029875553 -123.0189197675421) bank192732 +192733 POINT(37.973223324840575 -122.66601723501864) bank192733 +192734 POINT(37.91801442866307 -121.86021536374628) bank192734 +192735 POINT(37.10792805264769 -123.20802623573515) bank192735 +192736 POINT(37.39279243830861 -122.04309919278688) bank192736 +192737 POINT(37.413109424524876 -121.64153570479766) bank192737 +192738 POINT(36.9626782384085 -122.58240074934412) bank192738 +192739 POINT(38.243310258524865 -121.69015041321443) bank192739 +192740 POINT(38.32285894234074 -123.39703262591432) bank192740 +192741 POINT(36.781228634838094 -122.54749603586909) bank192741 +192742 POINT(38.204016046317655 -121.60348695471502) bank192742 +192743 POINT(36.975437829990256 -122.86125054907129) bank192743 +192744 POINT(37.46799279249631 -123.4035371546256) bank192744 +192745 POINT(38.53005036366087 -122.50090818390359) bank192745 +192746 POINT(37.56758865198697 -122.02911815517304) bank192746 +192747 POINT(38.613431165929164 -123.3106134666855) bank192747 +192748 POINT(37.47326550351981 -121.99794929372766) bank192748 +192749 POINT(38.39367722582982 -122.60116744301811) bank192749 +192750 POINT(38.27068114024911 -121.72237160608405) bank192750 +192751 POINT(37.4299933335869 -123.35577749991782) bank192751 +192752 POINT(37.57016681900144 -121.51948536164737) bank192752 +192753 POINT(37.04986800274092 -122.00523625391286) bank192753 +192754 POINT(36.947190208972316 -123.25591659909028) bank192754 +192755 POINT(37.580195145469226 -123.24004093565685) bank192755 +192756 POINT(37.83596216650701 -122.4044947274373) bank192756 +192757 POINT(37.2966348908595 -122.65466869190307) bank192757 +192758 POINT(37.09907782520456 -123.16424041671874) bank192758 +192759 POINT(36.853886202450546 -122.49434790659319) bank192759 +192760 POINT(38.25144460936352 -122.38094442577375) bank192760 +192761 POINT(37.95109033325773 -121.81041534507044) bank192761 +192762 POINT(37.22918364890331 -122.45600137464835) bank192762 +192763 POINT(37.70962422821649 -122.53788704484325) bank192763 +192764 POINT(37.9603956509268 -121.58396670473664) bank192764 +192765 POINT(37.85544840558591 -123.34042864003042) bank192765 +192766 POINT(37.743493030594884 -123.09701040509749) bank192766 +192767 POINT(38.366729240229716 -122.8295189271022) bank192767 +192768 POINT(36.952072363701795 -122.85703792854935) bank192768 +192769 POINT(37.608212815094596 -121.54406487344528) bank192769 +192770 POINT(38.14493616030967 -123.15783438516831) bank192770 +192771 POINT(37.36685149986762 -122.61416105463542) bank192771 +192772 POINT(36.91013121454431 -121.8117636875868) bank192772 +192773 POINT(36.81502417335126 -122.29203858417962) bank192773 +192774 POINT(38.18912527792695 -122.50949958361774) bank192774 +192775 POINT(36.97557560041737 -123.18725016269278) bank192775 +192776 POINT(37.577632394720965 -122.64443062468047) bank192776 +192777 POINT(38.28213691687684 -122.8179316432102) bank192777 +192778 POINT(38.1415885067802 -121.42044232033312) bank192778 +192779 POINT(37.346856153351546 -122.20321486933766) bank192779 +192780 POINT(38.734352815244605 -122.91405334611933) bank192780 +192781 POINT(38.573138907567156 -121.86324188762595) bank192781 +192782 POINT(38.555364624936416 -122.8974465052678) bank192782 +192783 POINT(38.67245052793163 -123.02071670808542) bank192783 +192784 POINT(38.48197384716991 -123.24539230752272) bank192784 +192785 POINT(37.35311683418313 -121.66918343512144) bank192785 +192786 POINT(37.45893468718787 -122.45213556353103) bank192786 +192787 POINT(37.253221914096876 -122.85396655796461) bank192787 +192788 POINT(37.8221921299091 -122.73976650203316) bank192788 +192789 POINT(37.74124000727403 -121.91959416238343) bank192789 +192790 POINT(38.70496366171883 -122.1666407538707) bank192790 +192791 POINT(38.19238271622846 -122.78882426468309) bank192791 +192792 POINT(38.19516617480322 -123.01622798515761) bank192792 +192793 POINT(38.251327557480714 -122.67555867744608) bank192793 +192794 POINT(38.13811393188861 -121.9129649218272) bank192794 +192795 POINT(37.0711616372907 -122.76628557665326) bank192795 +192796 POINT(37.81071894703466 -122.1898846072406) bank192796 +192797 POINT(38.15147761700509 -121.74805030118779) bank192797 +192798 POINT(37.81362549607642 -122.55507508108961) bank192798 +192799 POINT(38.666207730838416 -122.66006596787253) bank192799 +192800 POINT(38.55839005904256 -122.16155899954872) bank192800 +192801 POINT(38.67350121485897 -122.30450502845511) bank192801 +192802 POINT(38.13795039354337 -121.91032749427931) bank192802 +192803 POINT(37.71059892845982 -122.52550036544807) bank192803 +192804 POINT(37.274315702433114 -123.07009120108972) bank192804 +192805 POINT(37.755192191829266 -121.69068454640686) bank192805 +192806 POINT(38.626090632332335 -123.26696962967424) bank192806 +192807 POINT(38.10067947998712 -121.46346842806123) bank192807 +192808 POINT(37.82075297118117 -121.65696928354497) bank192808 +192809 POINT(37.82611593070421 -121.78343579086318) bank192809 +192810 POINT(37.54938006183119 -123.3302502295919) bank192810 +192811 POINT(37.75992385230917 -123.28933483415464) bank192811 +192812 POINT(38.325140525254945 -121.53279805163852) bank192812 +192813 POINT(37.89879826052217 -121.52280380628164) bank192813 +192814 POINT(38.24782602082665 -123.06124582678892) bank192814 +192815 POINT(38.096135786541424 -122.2840564877263) bank192815 +192816 POINT(37.980716352289676 -122.7287377526157) bank192816 +192817 POINT(38.64239957992031 -122.87612775107318) bank192817 +192818 POINT(38.20468624759649 -122.02122650873116) bank192818 +192819 POINT(37.07326175849054 -122.06420971419098) bank192819 +192820 POINT(37.406266099562316 -122.61770235548775) bank192820 +192821 POINT(37.850030700752846 -122.01232030241958) bank192821 +192822 POINT(38.013894464249866 -122.05836751524998) bank192822 +192823 POINT(38.6933772061803 -121.8654940581428) bank192823 +192824 POINT(37.33606738178145 -122.71221738461513) bank192824 +192825 POINT(36.93406299092992 -122.38182243410374) bank192825 +192826 POINT(38.63597450208518 -121.67946413197376) bank192826 +192827 POINT(38.26699202377273 -122.60348293141617) bank192827 +192828 POINT(37.84705740930039 -121.48049534010845) bank192828 +192829 POINT(37.55522078909331 -121.97495471265339) bank192829 +192830 POINT(38.47339938188186 -122.70999046421582) bank192830 +192831 POINT(38.585754252919 -121.95467800380418) bank192831 +192832 POINT(37.67837581127565 -122.85443762592372) bank192832 +192833 POINT(36.868410481657854 -121.56231059002835) bank192833 +192834 POINT(36.87461454211658 -121.91993054329426) bank192834 +192835 POINT(38.49158206594636 -122.40274518074047) bank192835 +192836 POINT(37.07773009348535 -121.94758453475337) bank192836 +192837 POINT(37.58566827967913 -121.47978612199068) bank192837 +192838 POINT(37.99809736181565 -122.35662012764051) bank192838 +192839 POINT(37.61186838371739 -121.89291288012184) bank192839 +192840 POINT(38.155341192357334 -121.52473389278317) bank192840 +192841 POINT(37.80032132742761 -122.77415246673544) bank192841 +192842 POINT(36.91874357208273 -123.0595850076707) bank192842 +192843 POINT(37.912575506929734 -121.56287393930025) bank192843 +192844 POINT(37.53781580982762 -123.25926167770965) bank192844 +192845 POINT(37.02126561768377 -121.59304095604404) bank192845 +192846 POINT(36.99501894758158 -122.6641888338416) bank192846 +192847 POINT(38.143718723189394 -121.64574545785975) bank192847 +192848 POINT(37.389106983956715 -121.89126930822684) bank192848 +192849 POINT(38.121835708453816 -121.84565827955177) bank192849 +192850 POINT(38.026674666786306 -123.18692819404839) bank192850 +192851 POINT(38.704455053125585 -122.06399324670689) bank192851 +192852 POINT(38.50636069289726 -121.94377749557555) bank192852 +192853 POINT(38.6754621812807 -121.57897967420173) bank192853 +192854 POINT(37.54392693542098 -122.62733869217229) bank192854 +192855 POINT(37.684602733311095 -122.67129628943549) bank192855 +192856 POINT(37.23905772743911 -122.20388124961696) bank192856 +192857 POINT(37.722038874072624 -122.80294836430473) bank192857 +192858 POINT(37.33961353219791 -121.63522729836438) bank192858 +192859 POINT(37.41535877464677 -123.3843409167363) bank192859 +192860 POINT(37.02762300005209 -123.39787140783146) bank192860 +192861 POINT(36.785302681576965 -121.83818529620788) bank192861 +192862 POINT(38.736504669454604 -122.6715994233089) bank192862 +192863 POINT(37.59493706605793 -122.68064494632341) bank192863 +192864 POINT(36.779914797835254 -121.99471531967468) bank192864 +192865 POINT(38.677166772720774 -121.46813678251299) bank192865 +192866 POINT(37.81092934333537 -122.6845720968867) bank192866 +192867 POINT(37.48263276741404 -122.97956578628012) bank192867 +192868 POINT(37.94419936980506 -123.05666212566801) bank192868 +192869 POINT(37.43555126837503 -121.76217628238571) bank192869 +192870 POINT(37.581115999565064 -123.36812263762042) bank192870 +192871 POINT(38.133617349913486 -122.5354588403252) bank192871 +192872 POINT(38.738402243886675 -121.50691698206019) bank192872 +192873 POINT(38.594346717557706 -122.61453907659948) bank192873 +192874 POINT(37.824246323533565 -122.58806666313527) bank192874 +192875 POINT(37.97352230481606 -122.88311432826016) bank192875 +192876 POINT(36.840894217581784 -123.40189769390342) bank192876 +192877 POINT(37.35117960490506 -122.46582785601444) bank192877 +192878 POINT(38.06283527052469 -121.80424558139052) bank192878 +192879 POINT(37.62888700203644 -121.44333536089015) bank192879 +192880 POINT(38.09455559662182 -123.13737018005648) bank192880 +192881 POINT(36.837221998843475 -122.39448946564231) bank192881 +192882 POINT(37.91040530626096 -121.44756427072515) bank192882 +192883 POINT(37.15664284037686 -121.72514661270746) bank192883 +192884 POINT(38.437082360945546 -123.01510587050294) bank192884 +192885 POINT(37.14878940868722 -121.89945858624452) bank192885 +192886 POINT(38.34553871920883 -123.2010231933654) bank192886 +192887 POINT(36.77870061586375 -121.50070489602233) bank192887 +192888 POINT(36.92537003720069 -122.03074540574477) bank192888 +192889 POINT(37.537050031772125 -123.31498816455813) bank192889 +192890 POINT(36.80075306367832 -122.53916904029285) bank192890 +192891 POINT(37.19000080861194 -123.09896306916295) bank192891 +192892 POINT(38.272061974401595 -123.01717212516482) bank192892 +192893 POINT(36.84890381651811 -122.11489865753697) bank192893 +192894 POINT(38.58812059277682 -122.5331211943044) bank192894 +192895 POINT(37.306035448125385 -122.62975780498175) bank192895 +192896 POINT(37.336274228060184 -122.08994393818601) bank192896 +192897 POINT(37.02965787094357 -122.3085734084307) bank192897 +192898 POINT(37.118174906245756 -121.91910503479258) bank192898 +192899 POINT(36.820444775055826 -122.49894050776916) bank192899 +192900 POINT(38.34769562115877 -122.75189359929402) bank192900 +192901 POINT(37.77008696128706 -121.90444059441278) bank192901 +192902 POINT(36.908754271876056 -122.86020996776394) bank192902 +192903 POINT(37.649803252146256 -122.03952070912021) bank192903 +192904 POINT(37.17264980266651 -121.87333414183936) bank192904 +192905 POINT(37.065429216141574 -122.68853732734725) bank192905 +192906 POINT(37.37416811731365 -123.35271216091071) bank192906 +192907 POINT(38.68442675875105 -122.86789619847397) bank192907 +192908 POINT(37.158380280713565 -122.94851119546198) bank192908 +192909 POINT(37.37142670430015 -123.22583304305894) bank192909 +192910 POINT(38.457885546604814 -122.54806872739876) bank192910 +192911 POINT(37.791417086104694 -122.27346464921902) bank192911 +192912 POINT(38.13150455949986 -122.9863610086074) bank192912 +192913 POINT(38.257486638640046 -122.01996037216497) bank192913 +192914 POINT(37.99787263989613 -122.5294009588125) bank192914 +192915 POINT(37.05357543047612 -123.0353947949843) bank192915 +192916 POINT(38.1037824271666 -121.55002192505374) bank192916 +192917 POINT(37.73349529728496 -122.37674312345787) bank192917 +192918 POINT(38.711712924092716 -122.27422570691199) bank192918 +192919 POINT(38.09157338337833 -122.4316531212784) bank192919 +192920 POINT(38.41323423784577 -121.55195868420901) bank192920 +192921 POINT(37.920426743051166 -122.50813441121151) bank192921 +192922 POINT(37.79078675061558 -121.68449850527018) bank192922 +192923 POINT(38.007667306950346 -123.04955009595827) bank192923 +192924 POINT(37.83999788799608 -122.15747006330699) bank192924 +192925 POINT(37.76330615012572 -121.8148216946838) bank192925 +192926 POINT(38.763721285444554 -122.92569084320624) bank192926 +192927 POINT(38.598378921770944 -122.63124269109395) bank192927 +192928 POINT(38.12978622985992 -122.97206558394865) bank192928 +192929 POINT(38.49871602018354 -122.6049821785471) bank192929 +192930 POINT(38.71191445585898 -122.2875626087) bank192930 +192931 POINT(37.57673740045169 -122.78399258315642) bank192931 +192932 POINT(37.15956941080837 -121.87056616116263) bank192932 +192933 POINT(38.081249460034506 -122.00966824716532) bank192933 +192934 POINT(37.96856456928677 -121.48394897156676) bank192934 +192935 POINT(37.53317832908309 -122.10459163870169) bank192935 +192936 POINT(38.098936887953975 -121.80838753395625) bank192936 +192937 POINT(36.855064921189644 -122.02346594190443) bank192937 +192938 POINT(38.13282061872181 -121.50012185441958) bank192938 +192939 POINT(37.04920371558393 -122.10380633551382) bank192939 +192940 POINT(38.605888673040624 -122.6791310867902) bank192940 +192941 POINT(37.11477310749206 -121.97649614976898) bank192941 +192942 POINT(37.70861896238095 -122.41753748403234) bank192942 +192943 POINT(37.49942595093677 -122.73939947224478) bank192943 +192944 POINT(38.27098966947807 -121.94070237623289) bank192944 +192945 POINT(38.43846872927447 -121.43249573400774) bank192945 +192946 POINT(37.95647292758192 -123.41301946493073) bank192946 +192947 POINT(36.82826011615602 -123.20047122255848) bank192947 +192948 POINT(38.19038178154948 -122.58430198122012) bank192948 +192949 POINT(36.925978190567655 -121.8154434986218) bank192949 +192950 POINT(37.74749357781966 -122.01988168212326) bank192950 +192951 POINT(38.5122130068675 -122.26778248445497) bank192951 +192952 POINT(37.14690649684496 -121.51032506178557) bank192952 +192953 POINT(38.61315091963081 -122.58008515396831) bank192953 +192954 POINT(37.31243389995364 -122.53651059855386) bank192954 +192955 POINT(37.8115491878977 -122.82014521384387) bank192955 +192956 POINT(37.46496089292125 -123.03450427896183) bank192956 +192957 POINT(38.71587897651226 -122.51728298143014) bank192957 +192958 POINT(37.370141758730306 -121.7640643899582) bank192958 +192959 POINT(38.74868349852313 -122.16634420588368) bank192959 +192960 POINT(38.31931286482154 -123.35089448014516) bank192960 +192961 POINT(37.61451478362659 -122.0784950946803) bank192961 +192962 POINT(38.60655568165659 -123.37233457644778) bank192962 +192963 POINT(38.19726162013335 -122.99796711260038) bank192963 +192964 POINT(37.05428812469689 -122.27786810736058) bank192964 +192965 POINT(37.76824916643888 -122.86237633869943) bank192965 +192966 POINT(37.39923976254163 -122.83787447927877) bank192966 +192967 POINT(38.756715826998054 -122.19669557977937) bank192967 +192968 POINT(38.69105739379089 -122.92866314050393) bank192968 +192969 POINT(38.456416612956986 -121.65866054343446) bank192969 +192970 POINT(38.39743708583143 -122.4739037832167) bank192970 +192971 POINT(38.060367520681176 -123.3486600158463) bank192971 +192972 POINT(38.18586323488887 -123.41747132604088) bank192972 +192973 POINT(38.64129836019409 -121.8322382442085) bank192973 +192974 POINT(38.60858341375305 -123.4120393648334) bank192974 +192975 POINT(37.629969303192034 -123.00447706958981) bank192975 +192976 POINT(38.60886283788024 -121.95125121273199) bank192976 +192977 POINT(38.228164071333545 -121.45114257014133) bank192977 +192978 POINT(38.01155933372845 -121.43209296609622) bank192978 +192979 POINT(37.56218566757778 -121.99962051119503) bank192979 +192980 POINT(37.196724358572816 -121.76175715362056) bank192980 +192981 POINT(37.44114364071628 -123.26651598313774) bank192981 +192982 POINT(38.453207854655744 -121.9846071239522) bank192982 +192983 POINT(38.19816860948612 -122.3503648913402) bank192983 +192984 POINT(38.77277700120556 -122.51309597158848) bank192984 +192985 POINT(37.31814863227201 -122.1180776897296) bank192985 +192986 POINT(36.83479749269358 -122.02816234501809) bank192986 +192987 POINT(38.07645715943415 -121.89976187201093) bank192987 +192988 POINT(37.99578076440308 -121.89077515348235) bank192988 +192989 POINT(38.22233299176124 -123.3212382685109) bank192989 +192990 POINT(37.79553072287101 -121.83011118117223) bank192990 +192991 POINT(37.68891303589118 -123.3660602769584) bank192991 +192992 POINT(37.8470553036449 -121.8299286515306) bank192992 +192993 POINT(37.94100202846645 -121.84363655839505) bank192993 +192994 POINT(37.56433863872537 -122.54334723139615) bank192994 +192995 POINT(38.04308553655425 -121.45692432250651) bank192995 +192996 POINT(37.3257860835582 -123.17485769333805) bank192996 +192997 POINT(38.12807621241886 -121.80823918253854) bank192997 +192998 POINT(37.734711230969374 -122.16296507743192) bank192998 +192999 POINT(36.82850556691413 -122.67314860546082) bank192999 +193000 POINT(37.76232690647764 -122.18208837635369) bank193000 +193001 POINT(36.98943494336331 -122.07496629195938) bank193001 +193002 POINT(38.71978766171722 -123.25081872485332) bank193002 +193003 POINT(37.71068697077835 -121.87302188346564) bank193003 +193004 POINT(37.7718924736366 -122.78892785323085) bank193004 +193005 POINT(37.350029065474736 -122.78510672902222) bank193005 +193006 POINT(37.718640452232265 -122.22426403763949) bank193006 +193007 POINT(37.92045447093418 -123.20377060443049) bank193007 +193008 POINT(36.96082372062175 -123.31756527137657) bank193008 +193009 POINT(38.634154100851745 -122.90576295212898) bank193009 +193010 POINT(37.48879598723528 -121.48696093778895) bank193010 +193011 POINT(37.41181994934193 -122.20025828143103) bank193011 +193012 POINT(37.80914347346491 -123.02097723244245) bank193012 +193013 POINT(37.10156342259958 -121.57888672971232) bank193013 +193014 POINT(36.91258496692744 -122.40884297745406) bank193014 +193015 POINT(37.30522586841869 -122.61418814961196) bank193015 +193016 POINT(37.03760213161828 -123.16106602599666) bank193016 +193017 POINT(37.41814777348309 -121.58320184472392) bank193017 +193018 POINT(38.522932604904014 -122.83229966907733) bank193018 +193019 POINT(38.181734720891875 -122.17802823084519) bank193019 +193020 POINT(38.22552436637531 -122.22565783700938) bank193020 +193021 POINT(38.74637703699697 -123.04682257752276) bank193021 +193022 POINT(36.836742764412826 -122.28493105994136) bank193022 +193023 POINT(37.31864019776213 -122.97258721139033) bank193023 +193024 POINT(37.902912260159646 -122.98970559857982) bank193024 +193025 POINT(38.351480712830224 -121.78954601192143) bank193025 +193026 POINT(38.47091032446644 -123.30933849839509) bank193026 +193027 POINT(38.434681629728985 -123.0155308212454) bank193027 +193028 POINT(37.46986405203165 -122.71090915474609) bank193028 +193029 POINT(38.60396642712298 -121.55135960979038) bank193029 +193030 POINT(37.97125942023824 -122.20120086195608) bank193030 +193031 POINT(37.50265429952252 -122.71563839752586) bank193031 +193032 POINT(38.062288289517475 -122.9024683008592) bank193032 +193033 POINT(37.50536740187354 -121.854026119872) bank193033 +193034 POINT(37.74445525747276 -121.97321751744558) bank193034 +193035 POINT(38.62225735508887 -122.25340891200618) bank193035 +193036 POINT(37.17749523642874 -122.46159177664977) bank193036 +193037 POINT(38.138723250106345 -122.93775384059624) bank193037 +193038 POINT(37.065467007417496 -122.51011568981647) bank193038 +193039 POINT(37.25198219487586 -122.37838010647106) bank193039 +193040 POINT(36.907381417748546 -121.5713156834619) bank193040 +193041 POINT(38.4460687481494 -122.79126083545141) bank193041 +193042 POINT(37.91007249553848 -123.39986228933675) bank193042 +193043 POINT(38.608519190098704 -122.96355549550644) bank193043 +193044 POINT(36.86116608752917 -121.75121581000205) bank193044 +193045 POINT(37.97206981537692 -122.25538092768087) bank193045 +193046 POINT(36.94424209477652 -122.86048093995556) bank193046 +193047 POINT(37.792083446462556 -122.43366154730376) bank193047 +193048 POINT(38.34093125326719 -122.87811114617722) bank193048 +193049 POINT(37.50022737449189 -122.01108657233867) bank193049 +193050 POINT(37.773053682864564 -122.47057991713149) bank193050 +193051 POINT(38.46064906032866 -122.1923276773271) bank193051 +193052 POINT(38.00336576290951 -121.95142001841833) bank193052 +193053 POINT(37.261901572165414 -122.6129979539699) bank193053 +193054 POINT(37.184201890426934 -121.8158585332762) bank193054 +193055 POINT(37.45107558530082 -122.04225036446557) bank193055 +193056 POINT(38.70934531141283 -122.67121799882672) bank193056 +193057 POINT(37.89219218519116 -122.81087073099556) bank193057 +193058 POINT(38.194036445266185 -122.77195183698106) bank193058 +193059 POINT(38.05439808030056 -123.23899121435286) bank193059 +193060 POINT(37.91271636231812 -122.26618120984487) bank193060 +193061 POINT(37.490672334172636 -122.90479669012906) bank193061 +193062 POINT(37.24973059923302 -121.87297030270723) bank193062 +193063 POINT(37.0210535164412 -121.58410842986628) bank193063 +193064 POINT(37.41135508175809 -122.83294683118773) bank193064 +193065 POINT(37.513245949537215 -122.817064053162) bank193065 +193066 POINT(37.49392853129238 -121.8198521684628) bank193066 +193067 POINT(37.06785875397773 -121.63417049316175) bank193067 +193068 POINT(37.62311440356056 -122.04624170968009) bank193068 +193069 POINT(38.348459859849854 -121.93304336706666) bank193069 +193070 POINT(38.65474271063118 -121.80616718925482) bank193070 +193071 POINT(38.44604786357458 -122.5451117000465) bank193071 +193072 POINT(38.66963239473885 -122.776848885109) bank193072 +193073 POINT(38.48939028701616 -121.8679942854371) bank193073 +193074 POINT(37.112591486328796 -121.79827475209167) bank193074 +193075 POINT(38.27354395854794 -122.88210416234506) bank193075 +193076 POINT(36.87394362896019 -123.38865023246115) bank193076 +193077 POINT(36.85522766599503 -121.76068860278262) bank193077 +193078 POINT(37.131874840521306 -122.26512872729704) bank193078 +193079 POINT(36.99513226105578 -122.99896962694876) bank193079 +193080 POINT(36.938679616439366 -123.14670519145501) bank193080 +193081 POINT(37.55738917266779 -122.56447996680366) bank193081 +193082 POINT(38.25068946044077 -122.87759725585248) bank193082 +193083 POINT(36.81830413888712 -122.16869623722638) bank193083 +193084 POINT(38.44674307398575 -121.86833458076516) bank193084 +193085 POINT(37.46105864247541 -121.9440440667166) bank193085 +193086 POINT(38.1146409692932 -123.15620169993188) bank193086 +193087 POINT(38.374900882417364 -123.14032512222923) bank193087 +193088 POINT(38.09518252792613 -123.3966195023054) bank193088 +193089 POINT(37.32731775178409 -122.79381197709633) bank193089 +193090 POINT(38.17762382399316 -122.85721741886171) bank193090 +193091 POINT(38.15572042353909 -121.82485465991503) bank193091 +193092 POINT(38.626800152512224 -122.95136720205795) bank193092 +193093 POINT(38.53618571980686 -123.4064827253301) bank193093 +193094 POINT(37.48485333776468 -121.58136554168338) bank193094 +193095 POINT(37.86178494354774 -121.7391458376858) bank193095 +193096 POINT(38.12728577723742 -121.56417128074526) bank193096 +193097 POINT(36.82465929997055 -121.80712171521637) bank193097 +193098 POINT(37.01658650643338 -122.94568712273085) bank193098 +193099 POINT(37.42114984977558 -121.71078307272657) bank193099 +193100 POINT(37.42981351324691 -122.41712089635554) bank193100 +193101 POINT(38.15357554982891 -121.51598432455482) bank193101 +193102 POINT(38.76555148093646 -122.88882788111651) bank193102 +193103 POINT(38.46188270543535 -123.22170929309654) bank193103 +193104 POINT(37.06794893435983 -122.65738244807781) bank193104 +193105 POINT(37.76258167943843 -123.24992369245868) bank193105 +193106 POINT(37.52094106016454 -122.82159858907418) bank193106 +193107 POINT(36.86852712830271 -123.36129577830847) bank193107 +193108 POINT(37.920892788666464 -122.22747226070818) bank193108 +193109 POINT(37.33464361578519 -123.26666684640826) bank193109 +193110 POINT(38.321227412800795 -122.53511460303405) bank193110 +193111 POINT(37.46762815095636 -123.31842901989334) bank193111 +193112 POINT(36.916908326160254 -121.97897226137019) bank193112 +193113 POINT(38.41447004726235 -122.82303858875892) bank193113 +193114 POINT(37.41832672995231 -121.80480785716902) bank193114 +193115 POINT(37.95731627636807 -122.83035571154082) bank193115 +193116 POINT(36.89596530472588 -122.38165296131403) bank193116 +193117 POINT(38.28504893815924 -121.49374336660526) bank193117 +193118 POINT(38.60693898879007 -122.552043327437) bank193118 +193119 POINT(37.093249319664196 -122.8275544174167) bank193119 +193120 POINT(37.131431999773 -122.45493832037484) bank193120 +193121 POINT(37.805318963648645 -123.37770197932447) bank193121 +193122 POINT(37.10736344656511 -123.16361156501891) bank193122 +193123 POINT(37.9940029589079 -121.65982322530633) bank193123 +193124 POINT(37.271567208490275 -123.403320233505) bank193124 +193125 POINT(37.02080659217276 -121.8388573610011) bank193125 +193126 POINT(37.63815704570222 -122.5459757660554) bank193126 +193127 POINT(37.70392226327447 -122.39315911953489) bank193127 +193128 POINT(36.96355926958479 -123.18762998187826) bank193128 +193129 POINT(37.06749604816606 -123.04346530472338) bank193129 +193130 POINT(37.91170012838214 -121.57192929377922) bank193130 +193131 POINT(37.14092233788132 -123.0188332198346) bank193131 +193132 POINT(36.84086626933603 -122.74278189161623) bank193132 +193133 POINT(37.27889566552995 -122.23146975055423) bank193133 +193134 POINT(37.93800260755805 -121.89414779390742) bank193134 +193135 POINT(36.80136180328376 -122.93338033198492) bank193135 +193136 POINT(37.70945117965115 -122.21886950736429) bank193136 +193137 POINT(37.993201187934986 -122.15774668991943) bank193137 +193138 POINT(37.601716771712624 -122.62909574352871) bank193138 +193139 POINT(37.880230304759934 -122.96943530229329) bank193139 +193140 POINT(38.76620557423614 -122.43711001398367) bank193140 +193141 POINT(38.738392106173556 -122.41798172177077) bank193141 +193142 POINT(37.53356612901909 -121.99168477257616) bank193142 +193143 POINT(37.93294026226926 -122.7253877615589) bank193143 +193144 POINT(37.15965733805134 -121.96620893955831) bank193144 +193145 POINT(38.09854337772162 -122.54335886356266) bank193145 +193146 POINT(37.593441220945145 -122.36464123539156) bank193146 +193147 POINT(38.464851021839976 -121.97575037642258) bank193147 +193148 POINT(37.25577227751949 -121.6539311886497) bank193148 +193149 POINT(38.219180696867404 -122.81817212069603) bank193149 +193150 POINT(38.76303944244052 -121.75343957206368) bank193150 +193151 POINT(37.00868605772112 -121.45676937689349) bank193151 +193152 POINT(36.783376245594894 -123.15459693854721) bank193152 +193153 POINT(38.0080822567521 -122.3837393118011) bank193153 +193154 POINT(37.87058987689921 -121.70992672120352) bank193154 +193155 POINT(38.64413471597324 -122.70740254811805) bank193155 +193156 POINT(36.986498376970346 -123.23117281171827) bank193156 +193157 POINT(37.47749108835573 -122.51052018233115) bank193157 +193158 POINT(37.448355970938344 -122.05604008679047) bank193158 +193159 POINT(38.20018910742884 -121.49899235201732) bank193159 +193160 POINT(37.70980025636717 -123.35615298481109) bank193160 +193161 POINT(38.0411077723266 -122.58396265895121) bank193161 +193162 POINT(37.95408117205051 -122.62554997819032) bank193162 +193163 POINT(38.42126667959395 -122.60486875024027) bank193163 +193164 POINT(36.94755166369492 -122.51218984838559) bank193164 +193165 POINT(38.01251993857779 -121.72580226691629) bank193165 +193166 POINT(36.996418453078206 -122.87451613233858) bank193166 +193167 POINT(37.631391688983825 -122.71012169183669) bank193167 +193168 POINT(37.90171611040019 -122.81378400482569) bank193168 +193169 POINT(38.50698936751194 -121.86315594339344) bank193169 +193170 POINT(37.0817774030766 -122.31387666760754) bank193170 +193171 POINT(37.6452669854369 -122.3476288162331) bank193171 +193172 POINT(38.651850895120546 -122.2000202124144) bank193172 +193173 POINT(38.49002976208708 -121.64114760830176) bank193173 +193174 POINT(36.97259740669041 -121.9042253159922) bank193174 +193175 POINT(38.753446321733335 -122.43750067032585) bank193175 +193176 POINT(38.64462701133567 -121.88212170886963) bank193176 +193177 POINT(37.46591163992381 -123.22485944523498) bank193177 +193178 POINT(38.3140451310622 -121.50053094242739) bank193178 +193179 POINT(38.723459296455935 -121.85256836556668) bank193179 +193180 POINT(38.61187472223029 -122.17777603768099) bank193180 +193181 POINT(38.5881788670408 -122.82351859915502) bank193181 +193182 POINT(37.08347120824255 -122.20196493986299) bank193182 +193183 POINT(36.943919144945745 -121.92054748377082) bank193183 +193184 POINT(38.278561694774275 -123.02504688672089) bank193184 +193185 POINT(37.1321473240668 -122.6534304625163) bank193185 +193186 POINT(38.719010974520295 -122.12034507039883) bank193186 +193187 POINT(37.096634916402934 -121.92555381064639) bank193187 +193188 POINT(37.088568589966684 -121.94232088548485) bank193188 +193189 POINT(38.36509694306285 -122.50879550236861) bank193189 +193190 POINT(38.35351764356898 -123.07472725662593) bank193190 +193191 POINT(36.89845666023527 -122.05097852011878) bank193191 +193192 POINT(37.269908986789346 -121.81791443263201) bank193192 +193193 POINT(37.680154378369764 -121.59971045205917) bank193193 +193194 POINT(37.87372458639362 -123.35664207483103) bank193194 +193195 POINT(37.75379307710025 -123.37453708192673) bank193195 +193196 POINT(38.61472093270671 -121.90816827540485) bank193196 +193197 POINT(37.02017725887119 -122.46022051022861) bank193197 +193198 POINT(38.37033344537253 -123.01230556829051) bank193198 +193199 POINT(38.07651452753263 -122.1723961306491) bank193199 +193200 POINT(37.911956800627266 -123.19894213131946) bank193200 +193201 POINT(38.32820170300597 -123.36365382815828) bank193201 +193202 POINT(37.64831751180607 -122.86861039545627) bank193202 +193203 POINT(38.007962642179045 -121.64796918898092) bank193203 +193204 POINT(37.43944531267194 -123.18973364008117) bank193204 +193205 POINT(37.340330911980736 -121.51699679125444) bank193205 +193206 POINT(37.589232257900505 -121.85358577641942) bank193206 +193207 POINT(37.17409133309608 -121.77857867822911) bank193207 +193208 POINT(37.99261469129839 -121.71916201551596) bank193208 +193209 POINT(37.91544765130034 -121.6520078926185) bank193209 +193210 POINT(37.3830597789117 -122.40217495864937) bank193210 +193211 POINT(36.91530917579679 -121.4299983603103) bank193211 +193212 POINT(37.53323026994194 -122.10015164350905) bank193212 +193213 POINT(38.241611763553834 -122.12084616227565) bank193213 +193214 POINT(37.590848035599144 -121.4410349180992) bank193214 +193215 POINT(37.322532183436444 -123.01474100377561) bank193215 +193216 POINT(37.98835329008739 -121.50761799734697) bank193216 +193217 POINT(37.27855566285057 -122.28837603087487) bank193217 +193218 POINT(38.447130995631575 -123.03739240585415) bank193218 +193219 POINT(38.68957270583116 -122.44536395137868) bank193219 +193220 POINT(38.507839630746744 -122.92337998301124) bank193220 +193221 POINT(37.46432481911419 -121.85221025322276) bank193221 +193222 POINT(37.5587996646367 -122.96845060682608) bank193222 +193223 POINT(38.66494215691821 -121.77209669580294) bank193223 +193224 POINT(36.8334983363719 -122.42827952192296) bank193224 +193225 POINT(38.256918958831015 -122.80909197366117) bank193225 +193226 POINT(38.16642441325786 -121.63994707256066) bank193226 +193227 POINT(36.89624788029937 -121.63616428432876) bank193227 +193228 POINT(37.61932534526461 -122.81824895248786) bank193228 +193229 POINT(36.80018113398263 -122.77225682226587) bank193229 +193230 POINT(37.77210150553347 -123.22075827977105) bank193230 +193231 POINT(37.70288969014951 -121.9221336104952) bank193231 +193232 POINT(37.24651154832871 -121.98373229958233) bank193232 +193233 POINT(38.64192356989534 -122.84318569442645) bank193233 +193234 POINT(37.531245314209635 -122.33159186162958) bank193234 +193235 POINT(37.518699232973745 -121.83893923671415) bank193235 +193236 POINT(36.86823046633509 -121.60031579308612) bank193236 +193237 POINT(37.785259992001976 -122.37765852932036) bank193237 +193238 POINT(38.66025256107532 -122.15698415783774) bank193238 +193239 POINT(37.455887112790215 -122.98165465867069) bank193239 +193240 POINT(37.800358570065455 -122.9845480781037) bank193240 +193241 POINT(38.34703327820599 -121.78150437940153) bank193241 +193242 POINT(38.05776330457358 -121.96953892136783) bank193242 +193243 POINT(37.949691065902904 -121.86766407663868) bank193243 +193244 POINT(38.134822652078846 -121.88457470865877) bank193244 +193245 POINT(37.13466697974035 -122.34617562211461) bank193245 +193246 POINT(37.67746694433435 -121.9089139265204) bank193246 +193247 POINT(37.60174520104883 -122.04597132859624) bank193247 +193248 POINT(37.01346336524998 -123.2616877885598) bank193248 +193249 POINT(37.326285491146585 -122.33545380100607) bank193249 +193250 POINT(37.13816483997281 -122.97479006101304) bank193250 +193251 POINT(38.277899355661546 -121.62675240608671) bank193251 +193252 POINT(38.61441922909214 -122.00648742425365) bank193252 +193253 POINT(38.7061288141118 -122.0303915544066) bank193253 +193254 POINT(38.72583740596184 -121.58961500227885) bank193254 +193255 POINT(37.14069993863072 -122.2608342426612) bank193255 +193256 POINT(37.252029353427815 -121.87386440433237) bank193256 +193257 POINT(38.64836405074206 -121.95138603910162) bank193257 +193258 POINT(38.49409043058738 -121.83219517197364) bank193258 +193259 POINT(37.43245987471242 -121.60843614583206) bank193259 +193260 POINT(37.10801949065747 -123.09839291826489) bank193260 +193261 POINT(37.43097170879406 -122.349210663832) bank193261 +193262 POINT(37.344998968927676 -122.6161972914518) bank193262 +193263 POINT(37.41284143422108 -122.93750075389983) bank193263 +193264 POINT(37.45587608182321 -122.34349811844454) bank193264 +193265 POINT(37.23040364953505 -121.4687577708455) bank193265 +193266 POINT(36.82360415778866 -122.06404530251564) bank193266 +193267 POINT(38.269425131481434 -121.66433644574991) bank193267 +193268 POINT(38.39230222678123 -122.34843411451344) bank193268 +193269 POINT(37.86753249082475 -122.24219576118593) bank193269 +193270 POINT(38.7375442314106 -123.2394841276086) bank193270 +193271 POINT(37.179553747519776 -121.69527951755761) bank193271 +193272 POINT(37.55378080530248 -122.21515857881913) bank193272 +193273 POINT(38.41256638694124 -121.71723394379997) bank193273 +193274 POINT(38.06424086480938 -121.6077573643229) bank193274 +193275 POINT(36.91955883669179 -122.54076172908377) bank193275 +193276 POINT(37.29621931907608 -123.19504834675223) bank193276 +193277 POINT(38.20214805178617 -122.39138387026095) bank193277 +193278 POINT(37.06938537602806 -122.16442196369046) bank193278 +193279 POINT(37.89535184679728 -122.39818357281617) bank193279 +193280 POINT(38.21682139835957 -122.25652474928019) bank193280 +193281 POINT(37.85284891214856 -123.23318498684942) bank193281 +193282 POINT(38.25110389167891 -123.37377907695796) bank193282 +193283 POINT(36.79317699300768 -122.64120810632578) bank193283 +193284 POINT(37.212618234146234 -123.28325486461857) bank193284 +193285 POINT(37.610315434405784 -123.38291360061585) bank193285 +193286 POINT(37.122337758778066 -121.42469746148193) bank193286 +193287 POINT(37.48840674438835 -121.49714936030618) bank193287 +193288 POINT(38.449856437086105 -121.61272336672313) bank193288 +193289 POINT(37.972811811416506 -122.77080512737822) bank193289 +193290 POINT(37.84741797586217 -123.05698229992734) bank193290 +193291 POINT(37.908883303730704 -122.73418070868264) bank193291 +193292 POINT(37.53428748459856 -122.7226735183093) bank193292 +193293 POINT(37.824386143415666 -122.12469714828724) bank193293 +193294 POINT(38.415975175217135 -121.74274928143068) bank193294 +193295 POINT(38.40161492248007 -122.15720149197844) bank193295 +193296 POINT(38.33941501969948 -121.94989190746712) bank193296 +193297 POINT(37.23325767763251 -121.53023896767247) bank193297 +193298 POINT(38.52531435363829 -121.97170260688435) bank193298 +193299 POINT(38.59893688796884 -123.27846552487622) bank193299 +193300 POINT(38.66950099927676 -122.31523039747167) bank193300 +193301 POINT(38.14761808907314 -122.4532819328271) bank193301 +193302 POINT(36.88231965596719 -121.52058706157648) bank193302 +193303 POINT(38.49433288884238 -122.4371507429111) bank193303 +193304 POINT(37.67358477986025 -121.50192311467251) bank193304 +193305 POINT(36.80099901300439 -121.90211936615928) bank193305 +193306 POINT(38.03250277967564 -121.61729218338792) bank193306 +193307 POINT(37.430284111181926 -121.43497873151699) bank193307 +193308 POINT(37.54931336304959 -123.15369215488238) bank193308 +193309 POINT(38.18488947874281 -121.97956213057591) bank193309 +193310 POINT(37.72567765449644 -122.48503470895503) bank193310 +193311 POINT(37.263355327086636 -123.24604921814624) bank193311 +193312 POINT(36.79216178171363 -122.76144020213964) bank193312 +193313 POINT(37.168101341040206 -122.83800283737563) bank193313 +193314 POINT(38.5121694303232 -123.28317641237902) bank193314 +193315 POINT(38.601999357150284 -121.50901324776403) bank193315 +193316 POINT(38.08397144324887 -123.27285515913606) bank193316 +193317 POINT(37.58632972547882 -122.7142586230866) bank193317 +193318 POINT(37.88573268205134 -121.80968665204067) bank193318 +193319 POINT(38.4882968894845 -121.9420467485003) bank193319 +193320 POINT(37.62360505041223 -122.18149014736598) bank193320 +193321 POINT(38.749137356460146 -123.38186817110989) bank193321 +193322 POINT(37.18258622188241 -121.96704962539326) bank193322 +193323 POINT(37.00155370277128 -121.6673541015579) bank193323 +193324 POINT(37.699747165213914 -122.80046786343928) bank193324 +193325 POINT(38.65888952489071 -123.09784075874657) bank193325 +193326 POINT(37.98267220853201 -121.81984251412023) bank193326 +193327 POINT(37.62916006749582 -123.2119605409907) bank193327 +193328 POINT(38.20552639350081 -122.0438341978701) bank193328 +193329 POINT(38.74987575238992 -121.57468663166941) bank193329 +193330 POINT(38.757460538720004 -122.76958772848532) bank193330 +193331 POINT(37.08452480579704 -123.11764845974315) bank193331 +193332 POINT(38.164831052057046 -123.25630565104515) bank193332 +193333 POINT(37.16013106301407 -121.7900692995414) bank193333 +193334 POINT(37.205930536996625 -122.37316906051417) bank193334 +193335 POINT(38.41429308189269 -121.80570567368505) bank193335 +193336 POINT(38.38653434456932 -122.3605641662147) bank193336 +193337 POINT(38.43949202056744 -121.62611348676117) bank193337 +193338 POINT(37.69409730345535 -122.09367378253204) bank193338 +193339 POINT(37.85236945467157 -123.20229969962843) bank193339 +193340 POINT(38.017712976325626 -122.65947449395497) bank193340 +193341 POINT(38.25741216332497 -122.57800364774977) bank193341 +193342 POINT(38.03956426704603 -122.1188462043803) bank193342 +193343 POINT(36.80803409208049 -123.36642797720992) bank193343 +193344 POINT(37.03910330413905 -122.4448621203342) bank193344 +193345 POINT(37.06959317629986 -121.97783723215521) bank193345 +193346 POINT(37.63274596162088 -123.39698173344544) bank193346 +193347 POINT(38.76447278163371 -122.52431349974789) bank193347 +193348 POINT(38.09624472546462 -121.96682856033365) bank193348 +193349 POINT(38.32990700654753 -122.39431781673233) bank193349 +193350 POINT(38.144292526091526 -121.65727007040653) bank193350 +193351 POINT(37.06450066523009 -123.40192628521899) bank193351 +193352 POINT(37.20562220676534 -122.68617756363706) bank193352 +193353 POINT(37.065442249255135 -121.67140909402336) bank193353 +193354 POINT(38.14475991975737 -122.68081084531384) bank193354 +193355 POINT(37.44445342098647 -121.67214084705783) bank193355 +193356 POINT(37.96491149320697 -122.73174492562583) bank193356 +193357 POINT(37.06749064619701 -122.33365402296951) bank193357 +193358 POINT(37.03921005784629 -122.04288423574407) bank193358 +193359 POINT(36.91445574454249 -121.7065596375082) bank193359 +193360 POINT(38.58035484096745 -123.3052268306372) bank193360 +193361 POINT(38.030049008634386 -122.36942567114599) bank193361 +193362 POINT(38.460291902372575 -122.19372038854517) bank193362 +193363 POINT(37.54641144779929 -123.22554711690711) bank193363 +193364 POINT(37.82090919830651 -122.60253622111514) bank193364 +193365 POINT(38.66185994553356 -122.53083692756233) bank193365 +193366 POINT(37.57633024817208 -123.22399380124622) bank193366 +193367 POINT(37.40629082236551 -122.50961559797759) bank193367 +193368 POINT(38.265713370946685 -123.33767802704548) bank193368 +193369 POINT(37.53280496703523 -121.66680908373228) bank193369 +193370 POINT(37.269437688058844 -122.13399176641853) bank193370 +193371 POINT(38.32226645325775 -121.84215664587761) bank193371 +193372 POINT(37.02208827445535 -122.13689687912917) bank193372 +193373 POINT(37.68049819048537 -123.28162336372574) bank193373 +193374 POINT(38.31724237722346 -122.22174556034211) bank193374 +193375 POINT(37.93481574241433 -121.73970603309412) bank193375 +193376 POINT(37.85259894058042 -122.97620490770572) bank193376 +193377 POINT(37.00478200862917 -122.78356730943594) bank193377 +193378 POINT(37.90912586664272 -123.22127794733485) bank193378 +193379 POINT(37.92647306912505 -122.91026358922463) bank193379 +193380 POINT(36.786436309326795 -123.27130080999478) bank193380 +193381 POINT(38.76830390087269 -121.56512464973117) bank193381 +193382 POINT(37.61038105465846 -123.01767195186345) bank193382 +193383 POINT(38.01717468764708 -122.88568872568536) bank193383 +193384 POINT(38.00102772918464 -122.78157657101141) bank193384 +193385 POINT(37.30490116820664 -122.1803285566124) bank193385 +193386 POINT(37.820420331032906 -121.93115913921052) bank193386 +193387 POINT(37.78730125698146 -123.1073994990769) bank193387 +193388 POINT(37.06596590457473 -122.41708532424204) bank193388 +193389 POINT(37.46111988299985 -123.29928837431163) bank193389 +193390 POINT(38.350735078006 -122.35290447052712) bank193390 +193391 POINT(37.338057477818985 -121.79347043029462) bank193391 +193392 POINT(38.6374002611119 -122.13200735736947) bank193392 +193393 POINT(38.156046505226584 -123.20152254508902) bank193393 +193394 POINT(38.04320683001197 -122.52583766292992) bank193394 +193395 POINT(37.010919325002995 -121.99445631033427) bank193395 +193396 POINT(37.51230920782102 -123.34574931157955) bank193396 +193397 POINT(38.150365076282505 -123.11828135075584) bank193397 +193398 POINT(37.52641477822204 -123.17615710742223) bank193398 +193399 POINT(37.02229068910062 -121.58306922939981) bank193399 +193400 POINT(37.550604429966626 -123.01594922527153) bank193400 +193401 POINT(37.37614834102614 -123.15111773364652) bank193401 +193402 POINT(36.93736615685771 -122.19849484046406) bank193402 +193403 POINT(38.66064219722494 -122.44712535228072) bank193403 +193404 POINT(38.196775350498385 -122.61770993222072) bank193404 +193405 POINT(37.0566801137746 -122.28863788311621) bank193405 +193406 POINT(38.288720404472045 -122.52238244977309) bank193406 +193407 POINT(38.08641984972898 -122.41004009191792) bank193407 +193408 POINT(36.95576369325019 -123.13815220430055) bank193408 +193409 POINT(37.59409284550397 -122.0813727288691) bank193409 +193410 POINT(37.77569963794324 -122.12057791604238) bank193410 +193411 POINT(37.66228432616171 -123.07777347475451) bank193411 +193412 POINT(38.4209058722752 -122.22503851901284) bank193412 +193413 POINT(37.138638315879746 -122.5765007492676) bank193413 +193414 POINT(37.06128312768545 -122.23347991142265) bank193414 +193415 POINT(37.833373711000135 -123.22861895032767) bank193415 +193416 POINT(37.08222118719472 -123.09602511768735) bank193416 +193417 POINT(38.55944277157336 -121.4865868056588) bank193417 +193418 POINT(37.89779632611064 -121.81880106696845) bank193418 +193419 POINT(37.23895360858932 -123.0725920944308) bank193419 +193420 POINT(37.013144433481926 -121.717293124663) bank193420 +193421 POINT(37.3184968376924 -122.02603584065098) bank193421 +193422 POINT(37.78439236871809 -122.92677362434229) bank193422 +193423 POINT(38.057765252616655 -121.82676474942676) bank193423 +193424 POINT(36.98869500176828 -123.01143425182435) bank193424 +193425 POINT(38.117943067914574 -121.84763066099586) bank193425 +193426 POINT(38.70123871743503 -123.09674528972971) bank193426 +193427 POINT(38.4456061617988 -123.21139333142065) bank193427 +193428 POINT(36.819142478615426 -122.57957932239789) bank193428 +193429 POINT(38.542467401652736 -122.06181364031313) bank193429 +193430 POINT(37.18875909377401 -122.0208890318985) bank193430 +193431 POINT(38.235327012653315 -122.38599227906461) bank193431 +193432 POINT(38.770566525960064 -122.48829273035237) bank193432 +193433 POINT(38.74819366023918 -121.47214510157814) bank193433 +193434 POINT(37.801768392922426 -121.60327784051736) bank193434 +193435 POINT(37.17882257227869 -122.77335987389763) bank193435 +193436 POINT(37.63147697337854 -122.9022853494616) bank193436 +193437 POINT(37.28893610031282 -123.23222903132384) bank193437 +193438 POINT(37.84726577657886 -122.04164444562487) bank193438 +193439 POINT(37.51752165852223 -121.65939058660517) bank193439 +193440 POINT(37.054505681001885 -122.27803650782484) bank193440 +193441 POINT(38.62707923367401 -123.1620122304975) bank193441 +193442 POINT(37.22102191717582 -123.3644953884192) bank193442 +193443 POINT(38.17113178618802 -122.55790814486235) bank193443 +193444 POINT(37.171387977760574 -123.31043106573449) bank193444 +193445 POINT(38.16131768686453 -121.7948051732769) bank193445 +193446 POINT(38.57418347019549 -121.51775121053477) bank193446 +193447 POINT(38.41316119593456 -122.36299089387556) bank193447 +193448 POINT(38.764937746179 -122.02886230184767) bank193448 +193449 POINT(38.12926036171961 -121.52343344945535) bank193449 +193450 POINT(38.4611348415105 -121.43302394462006) bank193450 +193451 POINT(38.625896009781414 -122.26615845880535) bank193451 +193452 POINT(37.07651754018899 -123.38984275076993) bank193452 +193453 POINT(38.44251038737898 -122.97206682133817) bank193453 +193454 POINT(36.928059985452606 -122.13913946614308) bank193454 +193455 POINT(38.156233097190764 -122.13131426420026) bank193455 +193456 POINT(36.78981940765764 -122.73573936553724) bank193456 +193457 POINT(38.337147705329606 -122.10771018840079) bank193457 +193458 POINT(36.799374971471984 -123.29179652157173) bank193458 +193459 POINT(36.950131044804465 -121.56239685336115) bank193459 +193460 POINT(37.09591126127658 -122.19580992760453) bank193460 +193461 POINT(36.83108804303075 -123.38224566746882) bank193461 +193462 POINT(38.4542250143282 -121.90625936067134) bank193462 +193463 POINT(38.27485844756058 -121.6326827219584) bank193463 +193464 POINT(37.56200510305289 -121.83984359920507) bank193464 +193465 POINT(38.122856308086824 -122.4065385579985) bank193465 +193466 POINT(37.71154377195603 -122.9179843571315) bank193466 +193467 POINT(38.120291064206015 -122.33055740172658) bank193467 +193468 POINT(36.88920245625139 -122.66673600487024) bank193468 +193469 POINT(38.14490930567376 -121.64295711060295) bank193469 +193470 POINT(37.66881176616364 -122.78748345569444) bank193470 +193471 POINT(38.29512892309589 -123.11367370942645) bank193471 +193472 POINT(37.64546213885653 -123.12818662727324) bank193472 +193473 POINT(38.72933403473957 -122.07736669643596) bank193473 +193474 POINT(37.210424499191966 -122.1476566316784) bank193474 +193475 POINT(37.653792685902296 -121.44153585000954) bank193475 +193476 POINT(37.099703816991166 -122.36147538736667) bank193476 +193477 POINT(38.66662151297398 -123.11916225861155) bank193477 +193478 POINT(37.97815237176255 -123.255567784854) bank193478 +193479 POINT(37.47943113812137 -122.47500563004364) bank193479 +193480 POINT(36.910602077095156 -122.81869528049366) bank193480 +193481 POINT(37.3989629583236 -122.59889157768714) bank193481 +193482 POINT(36.81416816164222 -121.43270038538252) bank193482 +193483 POINT(37.3725937057389 -122.63195070088574) bank193483 +193484 POINT(38.4430361797643 -122.46987740303793) bank193484 +193485 POINT(38.07117011572303 -123.05293542933101) bank193485 +193486 POINT(37.388720795924314 -123.06369494983156) bank193486 +193487 POINT(38.67127531074355 -122.63640053953755) bank193487 +193488 POINT(36.79238292115074 -121.54376302541488) bank193488 +193489 POINT(38.04198430949259 -122.49728520891662) bank193489 +193490 POINT(38.13179094967615 -121.5292939186465) bank193490 +193491 POINT(38.49818690683518 -122.17974623186184) bank193491 +193492 POINT(38.30238376616343 -123.22693644977785) bank193492 +193493 POINT(37.446206058315575 -122.52173930855017) bank193493 +193494 POINT(36.968376649703124 -122.50658204895996) bank193494 +193495 POINT(38.51653700675034 -122.84535169889465) bank193495 +193496 POINT(36.98435051537937 -122.55808856049552) bank193496 +193497 POINT(38.366293424565335 -121.45810656631336) bank193497 +193498 POINT(37.064790968849394 -121.89869021047599) bank193498 +193499 POINT(37.393256429249575 -122.1202733992556) bank193499 +193500 POINT(38.642226765509456 -122.06451251477158) bank193500 +193501 POINT(37.183782491903145 -121.78811728735305) bank193501 +193502 POINT(38.41133663670907 -122.21794527169828) bank193502 +193503 POINT(37.67776364170346 -121.5418167203812) bank193503 +193504 POINT(38.580538531887846 -123.357087345614) bank193504 +193505 POINT(37.74830783372687 -122.38558041892348) bank193505 +193506 POINT(37.51682559375548 -121.84781483790101) bank193506 +193507 POINT(38.16703686332072 -122.59336987812539) bank193507 +193508 POINT(38.50993591486026 -122.21139964725867) bank193508 +193509 POINT(38.14050856707587 -121.89432731215754) bank193509 +193510 POINT(37.30937217121846 -122.34676359921887) bank193510 +193511 POINT(37.69704928987279 -123.35292650194309) bank193511 +193512 POINT(38.42097565202374 -122.81764669943907) bank193512 +193513 POINT(37.65929172159872 -122.59997953915774) bank193513 +193514 POINT(37.360526231250404 -122.9183972861148) bank193514 +193515 POINT(37.67725241788752 -121.71262098911546) bank193515 +193516 POINT(38.0852253382669 -121.751056968068) bank193516 +193517 POINT(38.23530066566655 -122.00079595063234) bank193517 +193518 POINT(38.60726740593585 -122.79675248177904) bank193518 +193519 POINT(37.52416401060276 -121.95856159106539) bank193519 +193520 POINT(38.67214752980955 -121.56747898457625) bank193520 +193521 POINT(37.0328393245782 -122.56819328347525) bank193521 +193522 POINT(36.843287765287116 -121.66910978023604) bank193522 +193523 POINT(37.024502623781586 -122.91963212860357) bank193523 +193524 POINT(37.21611236233937 -122.42460724023988) bank193524 +193525 POINT(36.83980817913636 -123.31129673552992) bank193525 +193526 POINT(38.01197712510318 -123.27505791289519) bank193526 +193527 POINT(38.233862155409376 -123.19518838290811) bank193527 +193528 POINT(37.3847413153283 -123.01051591665309) bank193528 +193529 POINT(37.846317460984835 -121.90586961134058) bank193529 +193530 POINT(38.16280167791769 -122.59904985650634) bank193530 +193531 POINT(37.570707122319554 -122.56900317148194) bank193531 +193532 POINT(38.64096839498416 -121.97663706918044) bank193532 +193533 POINT(38.72747977857502 -123.1303473334915) bank193533 +193534 POINT(38.20138317221688 -122.96925373855831) bank193534 +193535 POINT(37.32852900514358 -122.06489898265144) bank193535 +193536 POINT(38.1735897666508 -122.46750394633669) bank193536 +193537 POINT(37.51077012228056 -121.50068869353623) bank193537 +193538 POINT(37.84803126608874 -123.36478424940785) bank193538 +193539 POINT(37.54783595230482 -123.05137354533024) bank193539 +193540 POINT(38.73235035487924 -122.96699131354057) bank193540 +193541 POINT(36.87340529198383 -122.17986564528215) bank193541 +193542 POINT(38.502135596118144 -123.19818964633527) bank193542 +193543 POINT(38.72775518246911 -122.5533723593504) bank193543 +193544 POINT(37.17027789352682 -122.63211200020531) bank193544 +193545 POINT(38.430394546938416 -122.05663238105723) bank193545 +193546 POINT(37.53997836238383 -122.66617394555492) bank193546 +193547 POINT(37.54654222952471 -122.67705099328965) bank193547 +193548 POINT(38.50847392293386 -123.41302214718377) bank193548 +193549 POINT(38.63747022522018 -121.6438440322403) bank193549 +193550 POINT(37.14388457859558 -122.9284172522387) bank193550 +193551 POINT(38.46696606947131 -121.55591751597034) bank193551 +193552 POINT(38.63978431572232 -122.19630211047487) bank193552 +193553 POINT(37.357577897173776 -122.72557550482924) bank193553 +193554 POINT(38.15694172837863 -122.57382150048056) bank193554 +193555 POINT(37.48920352474457 -122.61778902853952) bank193555 +193556 POINT(38.10978453516947 -121.82566153544101) bank193556 +193557 POINT(37.44078114806027 -123.06641779037389) bank193557 +193558 POINT(38.28994471503126 -122.33263921488633) bank193558 +193559 POINT(36.99501274938122 -122.42360108380615) bank193559 +193560 POINT(38.138099186944345 -122.51636696840542) bank193560 +193561 POINT(37.84430021858363 -122.86586775510739) bank193561 +193562 POINT(37.58912120277346 -123.33446785079067) bank193562 +193563 POINT(38.08861550379796 -121.49438152677013) bank193563 +193564 POINT(37.02717752922437 -123.04327445748967) bank193564 +193565 POINT(37.4488285422141 -122.88768635149381) bank193565 +193566 POINT(37.21564337436013 -121.77387829527602) bank193566 +193567 POINT(36.85039389796197 -122.75900369363852) bank193567 +193568 POINT(38.00289086531853 -121.61025821978438) bank193568 +193569 POINT(37.62015822247184 -121.58866425243778) bank193569 +193570 POINT(37.70483929045374 -122.98506668390766) bank193570 +193571 POINT(38.421355272196436 -121.85184474029649) bank193571 +193572 POINT(36.99050383057041 -122.01211910732276) bank193572 +193573 POINT(38.597712517325974 -122.27089815419053) bank193573 +193574 POINT(36.919240151032334 -121.55299703540219) bank193574 +193575 POINT(38.585026063757674 -122.04511581783059) bank193575 +193576 POINT(37.52264133273941 -121.89696797987058) bank193576 +193577 POINT(37.94523624045806 -122.2827488066187) bank193577 +193578 POINT(37.113541753476696 -122.06144625617742) bank193578 +193579 POINT(37.212810094999796 -122.81104896879694) bank193579 +193580 POINT(37.61916056042225 -123.21084395860302) bank193580 +193581 POINT(36.983790587912864 -122.63581283231235) bank193581 +193582 POINT(38.027486170472656 -122.12485783910721) bank193582 +193583 POINT(38.42462065962065 -122.01432137612282) bank193583 +193584 POINT(37.81637861689843 -123.28587071428738) bank193584 +193585 POINT(36.82023232073598 -123.22538238490884) bank193585 +193586 POINT(38.11304162571943 -122.47404627191777) bank193586 +193587 POINT(38.46329076298857 -123.21795456178471) bank193587 +193588 POINT(36.85059509499934 -122.26981897341837) bank193588 +193589 POINT(38.46838741402803 -123.23208208720307) bank193589 +193590 POINT(37.34263327467272 -121.67548764947988) bank193590 +193591 POINT(37.246317517006666 -121.62052603158942) bank193591 +193592 POINT(38.34464839360258 -122.54786040731076) bank193592 +193593 POINT(38.07664717548382 -122.35219778775144) bank193593 +193594 POINT(37.371718125863765 -122.00264941122406) bank193594 +193595 POINT(37.13380241534636 -122.68958102328389) bank193595 +193596 POINT(37.87198890079143 -121.63203466519592) bank193596 +193597 POINT(37.60745132451152 -121.42322717908016) bank193597 +193598 POINT(38.666976256691896 -122.8036143721548) bank193598 +193599 POINT(37.53600322698497 -123.24359376666509) bank193599 +193600 POINT(37.229732515271294 -122.48305051499445) bank193600 +193601 POINT(38.05511182029292 -121.96491664951493) bank193601 +193602 POINT(37.01984294256957 -121.74122075220708) bank193602 +193603 POINT(38.62803130453768 -122.70862208115055) bank193603 +193604 POINT(37.961467121367114 -123.39006703147965) bank193604 +193605 POINT(38.59668825284895 -122.28729872082477) bank193605 +193606 POINT(37.537335495046705 -122.87923664283235) bank193606 +193607 POINT(37.12375935699246 -121.85494281364981) bank193607 +193608 POINT(37.12706888233342 -122.4617720351283) bank193608 +193609 POINT(38.03906997233412 -123.29002139366551) bank193609 +193610 POINT(37.4366089168175 -121.68174270922982) bank193610 +193611 POINT(37.679001630205434 -123.29665832699143) bank193611 +193612 POINT(37.08571253451411 -121.7413844672541) bank193612 +193613 POINT(37.66844058473377 -122.28897634561957) bank193613 +193614 POINT(37.95951377808072 -123.00436681025273) bank193614 +193615 POINT(38.187621606207195 -122.42927275585555) bank193615 +193616 POINT(38.69203470658891 -121.54980373166883) bank193616 +193617 POINT(38.11384993358377 -122.39883160066118) bank193617 +193618 POINT(37.83909781342382 -121.5982202630253) bank193618 +193619 POINT(36.91979679057283 -122.22210747969726) bank193619 +193620 POINT(37.41080949042733 -122.41110799657253) bank193620 +193621 POINT(37.138376319312634 -121.61802411593621) bank193621 +193622 POINT(37.87489983132032 -121.72065595533995) bank193622 +193623 POINT(38.31960073708663 -121.45880446580149) bank193623 +193624 POINT(37.96738814889148 -122.67222368700142) bank193624 +193625 POINT(38.59254456900455 -123.18779828918973) bank193625 +193626 POINT(37.899376166185746 -122.03158260892903) bank193626 +193627 POINT(38.220770122009085 -123.27899853420101) bank193627 +193628 POINT(37.35388561459503 -121.93069027782522) bank193628 +193629 POINT(38.014950345840774 -122.09092945579957) bank193629 +193630 POINT(37.98078982838258 -122.07234439342947) bank193630 +193631 POINT(37.345861618988536 -123.26384516970869) bank193631 +193632 POINT(38.605531927691885 -122.18734518734372) bank193632 +193633 POINT(38.53273336408026 -121.60151917922764) bank193633 +193634 POINT(38.62134384366571 -122.745113569342) bank193634 +193635 POINT(37.55507596688202 -123.15829496425421) bank193635 +193636 POINT(37.05536131204244 -123.28385533811934) bank193636 +193637 POINT(38.06940873371724 -122.86521720426487) bank193637 +193638 POINT(37.975590074903565 -121.91837688507881) bank193638 +193639 POINT(37.10058549723236 -123.14612175010389) bank193639 +193640 POINT(37.424371603133984 -122.44166600549988) bank193640 +193641 POINT(38.368005997710306 -122.40058402597535) bank193641 +193642 POINT(37.381407544723395 -123.10411650853932) bank193642 +193643 POINT(36.792738061408784 -122.73289414186166) bank193643 +193644 POINT(37.62687164985311 -122.88501127708675) bank193644 +193645 POINT(37.941053903969554 -123.15401327920314) bank193645 +193646 POINT(37.86708944565633 -121.7015781119954) bank193646 +193647 POINT(38.01318785146405 -121.72488590593596) bank193647 +193648 POINT(36.92549117921725 -122.72602252542693) bank193648 +193649 POINT(36.929415141695245 -122.73516717780423) bank193649 +193650 POINT(36.80186643694675 -122.97606098246861) bank193650 +193651 POINT(37.99863651446629 -121.85872820888717) bank193651 +193652 POINT(38.49539100279893 -122.8823225188665) bank193652 +193653 POINT(38.26448270661642 -121.4953340163407) bank193653 +193654 POINT(38.355849143101814 -122.47820945470046) bank193654 +193655 POINT(36.8774940718559 -123.09379289960266) bank193655 +193656 POINT(38.192734450514564 -121.87743608529209) bank193656 +193657 POINT(38.0863349587122 -121.63692371040588) bank193657 +193658 POINT(38.08350411178942 -122.13459562456647) bank193658 +193659 POINT(37.396484873647445 -122.48412158216931) bank193659 +193660 POINT(37.71676402214829 -123.00809211501024) bank193660 +193661 POINT(37.923119940527776 -122.7325872323951) bank193661 +193662 POINT(38.571899957047506 -121.91800954154851) bank193662 +193663 POINT(37.040751127917964 -122.98215872211156) bank193663 +193664 POINT(38.48351695144331 -122.23497271024723) bank193664 +193665 POINT(38.69738861380597 -122.41864039817764) bank193665 +193666 POINT(37.94269983415896 -121.64298217979216) bank193666 +193667 POINT(38.294385601393756 -121.92572758179752) bank193667 +193668 POINT(37.83998722917523 -122.43521912806733) bank193668 +193669 POINT(36.92420156976448 -123.22491312278169) bank193669 +193670 POINT(38.51246180245467 -123.38254409838092) bank193670 +193671 POINT(37.85418094473959 -122.74756969477184) bank193671 +193672 POINT(36.98056007997135 -122.16713537995739) bank193672 +193673 POINT(36.899418712308695 -122.86072731878441) bank193673 +193674 POINT(38.76552879737711 -122.67940151912673) bank193674 +193675 POINT(36.823625922218156 -123.2037296324906) bank193675 +193676 POINT(37.89803381306989 -121.63307913838916) bank193676 +193677 POINT(37.20432725491041 -122.27210674759093) bank193677 +193678 POINT(38.5988247061303 -122.64589207175682) bank193678 +193679 POINT(38.48977867340854 -123.01564049890783) bank193679 +193680 POINT(37.774462627233504 -122.60121358628551) bank193680 +193681 POINT(37.20813232183563 -122.53156309028898) bank193681 +193682 POINT(36.958815236734125 -122.08471911750054) bank193682 +193683 POINT(36.99149731358531 -123.0816182121769) bank193683 +193684 POINT(37.16666011637192 -122.50908053941356) bank193684 +193685 POINT(38.2035583562539 -122.58580779051206) bank193685 +193686 POINT(38.37616349099424 -121.60445374607916) bank193686 +193687 POINT(36.78385414561127 -123.32423404003286) bank193687 +193688 POINT(37.31429943810476 -122.51443283359642) bank193688 +193689 POINT(38.11899762132323 -121.67440952801998) bank193689 +193690 POINT(37.89827035751868 -122.28549791855484) bank193690 +193691 POINT(37.97341960016163 -121.98127027688675) bank193691 +193692 POINT(37.65620759323098 -122.60607367651626) bank193692 +193693 POINT(38.630771170940534 -121.83786439004147) bank193693 +193694 POINT(38.34360188658696 -122.70398230163184) bank193694 +193695 POINT(37.92468094407436 -122.2978255006855) bank193695 +193696 POINT(38.41941077588566 -122.50519307166715) bank193696 +193697 POINT(37.21288094546514 -121.67665438508624) bank193697 +193698 POINT(37.20857047383993 -122.86360193515986) bank193698 +193699 POINT(38.073456747700725 -122.24226490083667) bank193699 +193700 POINT(38.69759272808991 -122.85604207209215) bank193700 +193701 POINT(37.69858481520012 -121.79001629358608) bank193701 +193702 POINT(37.49948292487957 -122.59600107167775) bank193702 +193703 POINT(37.466780353050986 -121.79950990732769) bank193703 +193704 POINT(38.01711222451793 -122.15413307002878) bank193704 +193705 POINT(38.293870319164704 -122.90862394974394) bank193705 +193706 POINT(37.45797817588205 -122.98507718824122) bank193706 +193707 POINT(38.75951721012071 -123.3588829902799) bank193707 +193708 POINT(37.60132255466991 -121.7888006588731) bank193708 +193709 POINT(36.82183497715607 -122.8033884181257) bank193709 +193710 POINT(38.26001732634766 -122.11630214774999) bank193710 +193711 POINT(37.910913391590285 -122.57626055004013) bank193711 +193712 POINT(38.52015892106388 -121.65642149175149) bank193712 +193713 POINT(36.88550398326495 -121.79208232608956) bank193713 +193714 POINT(37.38732601136735 -122.51504208924214) bank193714 +193715 POINT(38.09705241014611 -121.93987747470229) bank193715 +193716 POINT(37.46097538315142 -121.45233196813564) bank193716 +193717 POINT(38.726615780827345 -122.76395440246463) bank193717 +193718 POINT(37.98820327147987 -121.80081976486278) bank193718 +193719 POINT(37.163027658326115 -121.95214827124488) bank193719 +193720 POINT(36.87184565506007 -122.32997987440302) bank193720 +193721 POINT(36.801286048067674 -121.77460179609271) bank193721 +193722 POINT(38.20711407033572 -121.8394897944462) bank193722 +193723 POINT(37.64255045746358 -122.66158836818904) bank193723 +193724 POINT(38.40182782272384 -122.92376745588456) bank193724 +193725 POINT(37.66521603810885 -121.57368873534597) bank193725 +193726 POINT(36.81819118446324 -123.04498371712491) bank193726 +193727 POINT(37.2606391350196 -123.15973981522858) bank193727 +193728 POINT(37.803631216612025 -122.18916922761545) bank193728 +193729 POINT(37.128657098306846 -122.23451951736115) bank193729 +193730 POINT(37.06558864977711 -122.94111156104151) bank193730 +193731 POINT(37.625761992504856 -121.89855390368143) bank193731 +193732 POINT(36.98482710671909 -122.21734384220736) bank193732 +193733 POINT(38.31992412664748 -122.29312308310953) bank193733 +193734 POINT(37.85524302847731 -122.13975570573648) bank193734 +193735 POINT(37.86304142524379 -121.44355475089216) bank193735 +193736 POINT(37.116021056192686 -122.99418310773748) bank193736 +193737 POINT(38.39888383566122 -121.55909664242775) bank193737 +193738 POINT(38.363756239843426 -122.76644525529787) bank193738 +193739 POINT(37.69548195983178 -121.49271407242094) bank193739 +193740 POINT(38.38172855808885 -121.93191591801371) bank193740 +193741 POINT(37.93739299567312 -123.2809670889892) bank193741 +193742 POINT(37.271534493600136 -121.6119150592464) bank193742 +193743 POINT(37.8293433556612 -123.24089717746176) bank193743 +193744 POINT(37.31927413223602 -122.06485242833448) bank193744 +193745 POINT(37.26721199964918 -122.48259066271304) bank193745 +193746 POINT(37.14325142070953 -122.39034697679577) bank193746 +193747 POINT(37.88218824346187 -121.74599122549928) bank193747 +193748 POINT(38.32496743523452 -122.25571586205065) bank193748 +193749 POINT(37.528315394347175 -121.8520388489807) bank193749 +193750 POINT(37.218933865627946 -122.84299060922383) bank193750 +193751 POINT(37.20744875898616 -122.42111290181441) bank193751 +193752 POINT(38.55054443611162 -122.96804269314723) bank193752 +193753 POINT(36.78184367408139 -122.34214541625647) bank193753 +193754 POINT(38.194541242991505 -123.38991529051317) bank193754 +193755 POINT(38.24365988843777 -122.4980790330748) bank193755 +193756 POINT(38.42055859454366 -122.62788503490773) bank193756 +193757 POINT(38.2203033546332 -122.57384897179918) bank193757 +193758 POINT(36.826417111283654 -122.30535718643226) bank193758 +193759 POINT(38.33151304660684 -122.60758552943292) bank193759 +193760 POINT(37.432520978699486 -123.17848802534508) bank193760 +193761 POINT(38.71916103286211 -123.40656026665394) bank193761 +193762 POINT(37.2964751024558 -122.11375766662736) bank193762 +193763 POINT(37.83597658887804 -123.00222566107739) bank193763 +193764 POINT(37.248096880677856 -123.07843984669653) bank193764 +193765 POINT(37.921071937868554 -121.6536618517991) bank193765 +193766 POINT(36.87928139300448 -121.72313364544047) bank193766 +193767 POINT(36.8076835339238 -123.34465226332169) bank193767 +193768 POINT(38.36148957935789 -123.23687583172135) bank193768 +193769 POINT(37.553660553186084 -121.42317270598468) bank193769 +193770 POINT(37.85010671754 -122.05802933087654) bank193770 +193771 POINT(37.20332318038561 -121.61580654121374) bank193771 +193772 POINT(37.311641859572404 -121.9529017174561) bank193772 +193773 POINT(36.86000462585147 -123.21317553661876) bank193773 +193774 POINT(37.99522147483393 -121.92159936559679) bank193774 +193775 POINT(38.068563308318055 -123.31454688861986) bank193775 +193776 POINT(37.48735113719035 -123.2466538145961) bank193776 +193777 POINT(38.32142796847583 -122.3650221127798) bank193777 +193778 POINT(38.70652033617277 -122.67469355568329) bank193778 +193779 POINT(38.55284435984282 -123.21318302778582) bank193779 +193780 POINT(36.81764819480187 -123.00394911648073) bank193780 +193781 POINT(37.59763391042571 -122.25083879141889) bank193781 +193782 POINT(37.275266669803855 -122.15793219698485) bank193782 +193783 POINT(38.34900803929734 -122.02079452786464) bank193783 +193784 POINT(37.04385647557556 -121.87809833804566) bank193784 +193785 POINT(38.18321020589361 -122.12084702442439) bank193785 +193786 POINT(37.65455557396984 -121.76937108602611) bank193786 +193787 POINT(37.34852858604344 -122.13380125762924) bank193787 +193788 POINT(37.30423688337831 -121.86337822735689) bank193788 +193789 POINT(37.35828323201413 -122.71207192382656) bank193789 +193790 POINT(37.685258260404055 -122.02891963923257) bank193790 +193791 POINT(36.810425548355056 -122.83677790944037) bank193791 +193792 POINT(36.94989473620916 -121.50858092845387) bank193792 +193793 POINT(37.38833973030746 -121.78523794691243) bank193793 +193794 POINT(37.79492435660955 -122.45332794236943) bank193794 +193795 POINT(38.16258083323641 -122.50310058665002) bank193795 +193796 POINT(38.08702291293323 -121.65225455261081) bank193796 +193797 POINT(38.46544725322282 -122.20849533683149) bank193797 +193798 POINT(38.162352191335785 -122.12998413127187) bank193798 +193799 POINT(36.83431724128949 -121.47062267419003) bank193799 +193800 POINT(38.37360292036762 -123.34613997178725) bank193800 +193801 POINT(37.775150392178624 -122.20727870389561) bank193801 +193802 POINT(37.942072742207536 -122.52126072448796) bank193802 +193803 POINT(38.33058694807156 -122.92122284778432) bank193803 +193804 POINT(36.79100991830497 -122.79512784609207) bank193804 +193805 POINT(36.79335511402993 -122.36443060021469) bank193805 +193806 POINT(37.83405496709432 -123.26154144923582) bank193806 +193807 POINT(38.344343899583784 -123.15869534664893) bank193807 +193808 POINT(37.01651109828447 -121.8504033858178) bank193808 +193809 POINT(37.358483214749974 -123.04963514659012) bank193809 +193810 POINT(37.461684358061454 -122.00482206306988) bank193810 +193811 POINT(38.44632947072207 -122.62363421986666) bank193811 +193812 POINT(36.806015114498344 -121.86750116865316) bank193812 +193813 POINT(36.815177036803014 -121.52338453198662) bank193813 +193814 POINT(37.783764605001785 -122.13225293822464) bank193814 +193815 POINT(38.37375515655246 -121.44370718423248) bank193815 +193816 POINT(38.76626086154567 -122.39673689674815) bank193816 +193817 POINT(37.48287663397792 -123.36768229307489) bank193817 +193818 POINT(38.3069954273837 -123.05315835079267) bank193818 +193819 POINT(36.81164678709916 -122.83360799374606) bank193819 +193820 POINT(37.36110797694948 -122.12141336097885) bank193820 +193821 POINT(37.01812984721603 -123.06074919900657) bank193821 +193822 POINT(37.228081500663194 -123.17719511410183) bank193822 +193823 POINT(36.96818876152905 -122.78615260756116) bank193823 +193824 POINT(37.95578208231357 -122.07943827356819) bank193824 +193825 POINT(37.45088073065355 -122.69084630094335) bank193825 +193826 POINT(38.61653599665742 -121.86647079369953) bank193826 +193827 POINT(37.19290524489824 -122.41058904017467) bank193827 +193828 POINT(38.23939812900378 -121.95901887639602) bank193828 +193829 POINT(36.87436843792495 -122.56611115071367) bank193829 +193830 POINT(38.11211282423526 -123.15567955009408) bank193830 +193831 POINT(37.92521318265082 -121.73638458633506) bank193831 +193832 POINT(38.77441180119981 -122.37652031570781) bank193832 +193833 POINT(37.30923587732014 -121.68562880810309) bank193833 +193834 POINT(37.638327476793314 -121.90555666995621) bank193834 +193835 POINT(38.38045923400797 -121.87622032602798) bank193835 +193836 POINT(37.39532419062113 -123.3798089225207) bank193836 +193837 POINT(37.39550307783568 -122.62673869822572) bank193837 +193838 POINT(38.33197213370151 -121.97234594481206) bank193838 +193839 POINT(36.8405131889322 -122.26792952185109) bank193839 +193840 POINT(37.54955314260051 -121.85071132618476) bank193840 +193841 POINT(37.21199726491578 -122.16160043323008) bank193841 +193842 POINT(37.7754172689807 -122.13875761014873) bank193842 +193843 POINT(38.153592739923994 -121.8674391138808) bank193843 +193844 POINT(37.60909971663232 -121.8318680103672) bank193844 +193845 POINT(38.701619430722275 -122.18773920366043) bank193845 +193846 POINT(38.44755297651226 -122.66157422228629) bank193846 +193847 POINT(38.50678972853721 -122.53351234583688) bank193847 +193848 POINT(38.54022249850773 -122.5490331068439) bank193848 +193849 POINT(37.8470116770666 -121.70144718062355) bank193849 +193850 POINT(37.741543189824185 -121.96289238180852) bank193850 +193851 POINT(37.50289870344723 -122.70810606506669) bank193851 +193852 POINT(37.23867281261339 -123.41160177741777) bank193852 +193853 POINT(37.22802639964623 -121.84806951652847) bank193853 +193854 POINT(37.567578566260664 -122.43844488974644) bank193854 +193855 POINT(38.544850332387696 -122.44073129325044) bank193855 +193856 POINT(37.54551393465609 -121.75346069031274) bank193856 +193857 POINT(38.08192398751194 -123.12494931111922) bank193857 +193858 POINT(37.457769924696734 -123.35203210140149) bank193858 +193859 POINT(38.40896783607121 -123.41782687740323) bank193859 +193860 POINT(37.292382043764064 -122.44043249345377) bank193860 +193861 POINT(37.32882974327342 -122.28369132315785) bank193861 +193862 POINT(37.40875966388957 -122.68831828928312) bank193862 +193863 POINT(37.355176515090626 -122.52213251371732) bank193863 +193864 POINT(37.00681894310969 -122.20233200442823) bank193864 +193865 POINT(36.88040125089952 -123.19938779508512) bank193865 +193866 POINT(37.952781277151225 -121.57302835016533) bank193866 +193867 POINT(36.78677960613792 -122.14390144510561) bank193867 +193868 POINT(38.32799003288488 -121.74095012389711) bank193868 +193869 POINT(36.93797617247486 -121.57587621731115) bank193869 +193870 POINT(38.24968849402737 -121.89619662646773) bank193870 +193871 POINT(36.961965748038814 -123.1085225122017) bank193871 +193872 POINT(38.215783088272644 -121.51511664897866) bank193872 +193873 POINT(37.3151020241755 -123.0625600021141) bank193873 +193874 POINT(38.25532492979211 -122.89909778212957) bank193874 +193875 POINT(37.82447601652105 -122.36809285615558) bank193875 +193876 POINT(38.291849836276846 -122.3638333912567) bank193876 +193877 POINT(38.30843162409314 -123.03218036638326) bank193877 +193878 POINT(38.65955097352169 -121.7228324207807) bank193878 +193879 POINT(37.7827520295914 -122.85534444519352) bank193879 +193880 POINT(37.94273705174294 -122.332824051243) bank193880 +193881 POINT(37.1757860509324 -121.80398039391487) bank193881 +193882 POINT(37.127436497212514 -121.54478750231306) bank193882 +193883 POINT(37.52208102188913 -122.92311971532521) bank193883 +193884 POINT(37.48938935546128 -122.33849690577576) bank193884 +193885 POINT(38.62269827959911 -122.90384118356765) bank193885 +193886 POINT(36.84431301935953 -121.67626367429865) bank193886 +193887 POINT(37.362733496585804 -122.19389718891195) bank193887 +193888 POINT(37.961443428418825 -121.45152072756163) bank193888 +193889 POINT(37.96828262248368 -123.0019135444751) bank193889 +193890 POINT(38.65223297967942 -122.30549510055123) bank193890 +193891 POINT(38.06361493863503 -121.62051920359022) bank193891 +193892 POINT(37.036112039467405 -121.45410377608312) bank193892 +193893 POINT(37.2097974471512 -121.56329529266192) bank193893 +193894 POINT(38.25149719363409 -122.59905513567836) bank193894 +193895 POINT(37.04186520543298 -121.8802466896239) bank193895 +193896 POINT(37.69354689655719 -121.82582309058034) bank193896 +193897 POINT(37.96602305452438 -122.79702517335173) bank193897 +193898 POINT(37.82248792103665 -121.63592816321446) bank193898 +193899 POINT(38.54038921685963 -122.18466994820118) bank193899 +193900 POINT(37.205427997698614 -122.39630578930256) bank193900 +193901 POINT(38.43709997005263 -122.94877209464644) bank193901 +193902 POINT(38.6379027065572 -122.52343146459462) bank193902 +193903 POINT(37.15131173766655 -123.24645934449802) bank193903 +193904 POINT(37.94620723201094 -122.68164724675488) bank193904 +193905 POINT(36.895714578618204 -121.62153425878095) bank193905 +193906 POINT(37.599806017774085 -121.72712477615988) bank193906 +193907 POINT(37.77515293088491 -121.78674891125858) bank193907 +193908 POINT(37.81040753255124 -122.6989521928641) bank193908 +193909 POINT(37.82555482526609 -121.69614081790507) bank193909 +193910 POINT(37.4211398324868 -122.79950088189366) bank193910 +193911 POINT(36.97852039867026 -122.11564367453043) bank193911 +193912 POINT(37.95132364652683 -121.63966127381329) bank193912 +193913 POINT(36.82823781943712 -121.62101886441205) bank193913 +193914 POINT(37.911214484605985 -123.320466628304) bank193914 +193915 POINT(38.56762322336223 -121.80907181835511) bank193915 +193916 POINT(37.39716075131297 -121.72896460442225) bank193916 +193917 POINT(37.43493295887979 -122.49307101849652) bank193917 +193918 POINT(37.543414252239245 -121.84235654220784) bank193918 +193919 POINT(38.249278193204574 -123.39855863408923) bank193919 +193920 POINT(37.726744239991795 -122.45475830601234) bank193920 +193921 POINT(37.88558711642672 -122.7508234508386) bank193921 +193922 POINT(38.555555487431775 -122.72697918003367) bank193922 +193923 POINT(37.435384550492586 -121.94551397422833) bank193923 +193924 POINT(38.128766568753115 -122.93546218653995) bank193924 +193925 POINT(37.18460933045577 -123.11413872632761) bank193925 +193926 POINT(37.510400260156324 -122.13734212946295) bank193926 +193927 POINT(37.94786515782646 -122.59690575519673) bank193927 +193928 POINT(36.989214311403636 -122.24233775836157) bank193928 +193929 POINT(36.86304309518889 -122.05190364670322) bank193929 +193930 POINT(37.34077743992935 -122.8227942061297) bank193930 +193931 POINT(38.316577990907525 -121.84861716576754) bank193931 +193932 POINT(37.53337250630793 -121.7000782426615) bank193932 +193933 POINT(38.04110563275801 -122.95408461514617) bank193933 +193934 POINT(37.747450385077215 -122.2615943993633) bank193934 +193935 POINT(36.97861197411141 -122.82911775485584) bank193935 +193936 POINT(36.85347828669508 -123.17225266898001) bank193936 +193937 POINT(38.38770727758838 -121.97905953566543) bank193937 +193938 POINT(37.95865400114356 -123.39997465473873) bank193938 +193939 POINT(37.30665695587747 -123.1163525246156) bank193939 +193940 POINT(36.818566214709826 -122.4892667772961) bank193940 +193941 POINT(37.93970945017033 -122.42438349487237) bank193941 +193942 POINT(38.58082636805581 -123.31400860440165) bank193942 +193943 POINT(38.29854274447127 -122.80154231672412) bank193943 +193944 POINT(38.58656927248681 -122.07680624677779) bank193944 +193945 POINT(37.836396083224045 -121.44948144614796) bank193945 +193946 POINT(37.34557030663681 -122.92896092635272) bank193946 +193947 POINT(37.16381397994026 -122.8169899298159) bank193947 +193948 POINT(37.11657075152396 -121.72217499422551) bank193948 +193949 POINT(37.89008754818369 -122.1798553786883) bank193949 +193950 POINT(38.20065896202614 -122.80936300784602) bank193950 +193951 POINT(38.40028610281012 -123.1203967829964) bank193951 +193952 POINT(37.579357769357884 -121.72203370776947) bank193952 +193953 POINT(38.27812197721038 -122.29298293172482) bank193953 +193954 POINT(38.41177918245666 -121.58026250360487) bank193954 +193955 POINT(37.52837738433783 -122.04577650193069) bank193955 +193956 POINT(37.25198295238648 -122.95746700574419) bank193956 +193957 POINT(38.28257997296718 -122.21889668974258) bank193957 +193958 POINT(38.00879459038929 -121.56489292182582) bank193958 +193959 POINT(38.001645012601124 -123.1905260276933) bank193959 +193960 POINT(37.30454861889641 -122.15405649702755) bank193960 +193961 POINT(38.673414664890245 -121.42677078335443) bank193961 +193962 POINT(37.72691596303123 -122.77844561175817) bank193962 +193963 POINT(37.626777523965714 -123.07405634228998) bank193963 +193964 POINT(37.21273824850731 -122.70387930146968) bank193964 +193965 POINT(36.84880827030331 -122.34732817776307) bank193965 +193966 POINT(37.41985623842762 -123.27511618749736) bank193966 +193967 POINT(37.67833973540209 -122.98632889539729) bank193967 +193968 POINT(38.51331284024995 -122.8408234631992) bank193968 +193969 POINT(38.464037538657905 -122.05588843521386) bank193969 +193970 POINT(37.72272530918127 -122.18825798437307) bank193970 +193971 POINT(38.20692634736751 -121.72478376239346) bank193971 +193972 POINT(37.21632835251316 -122.4052898574368) bank193972 +193973 POINT(36.815651409751545 -122.94703327694374) bank193973 +193974 POINT(37.581287480940816 -121.54782247689943) bank193974 +193975 POINT(38.34966861650632 -122.282679250068) bank193975 +193976 POINT(36.921667103261925 -121.87243370027933) bank193976 +193977 POINT(38.48404917992177 -122.48603417202993) bank193977 +193978 POINT(37.53893671312404 -122.41888689978096) bank193978 +193979 POINT(36.83352990258768 -122.37933107227853) bank193979 +193980 POINT(38.40332133478762 -123.28245987313285) bank193980 +193981 POINT(37.703032976615624 -121.57096913040002) bank193981 +193982 POINT(37.08638410930945 -121.62184646635798) bank193982 +193983 POINT(37.44398324731358 -121.93097967630436) bank193983 +193984 POINT(37.45467775712108 -121.65414481972354) bank193984 +193985 POINT(37.35451154361789 -121.67924332483348) bank193985 +193986 POINT(37.56577640431243 -122.47632337119855) bank193986 +193987 POINT(38.14733260202411 -122.17366521970598) bank193987 +193988 POINT(36.863387374000965 -122.95304091428274) bank193988 +193989 POINT(36.914508876070705 -122.61601422998316) bank193989 +193990 POINT(36.78798343206135 -122.89075198086883) bank193990 +193991 POINT(38.53020300457201 -123.0600738574712) bank193991 +193992 POINT(38.72877899558988 -123.16586759990496) bank193992 +193993 POINT(37.42202872593982 -122.1685576968167) bank193993 +193994 POINT(38.41321607609518 -123.27406139116415) bank193994 +193995 POINT(37.01465343454727 -123.06387829919323) bank193995 +193996 POINT(37.25746389803196 -122.44513541672576) bank193996 +193997 POINT(36.97879087062945 -122.20373798826093) bank193997 +193998 POINT(38.34543499585451 -122.4176558008937) bank193998 +193999 POINT(37.35815380719979 -122.57040532788618) bank193999 +194000 POINT(37.699241294511445 -122.80662536070241) bank194000 +194001 POINT(36.977654669660026 -121.61364122260828) bank194001 +194002 POINT(37.90065130999763 -122.22779977058633) bank194002 +194003 POINT(38.76961068791235 -121.93678560344136) bank194003 +194004 POINT(37.13537709418919 -122.68508753857546) bank194004 +194005 POINT(37.78304901747338 -123.15548556267413) bank194005 +194006 POINT(38.68750251838037 -122.39549361669398) bank194006 +194007 POINT(37.273763260721424 -122.03773289020222) bank194007 +194008 POINT(38.125282637718826 -123.22487152296908) bank194008 +194009 POINT(37.91502629876822 -122.18604783198826) bank194009 +194010 POINT(38.08564730836246 -122.23590734721073) bank194010 +194011 POINT(37.689859573319474 -123.31322135690075) bank194011 +194012 POINT(37.08471300396629 -122.12512573768382) bank194012 +194013 POINT(38.23338250999411 -122.37188575642763) bank194013 +194014 POINT(38.72462450244746 -122.0613675548082) bank194014 +194015 POINT(37.765939352273286 -123.34077683961104) bank194015 +194016 POINT(36.81168983998037 -121.48003797842351) bank194016 +194017 POINT(37.89205205033382 -122.88768999389062) bank194017 +194018 POINT(37.157882972915615 -122.67414683035692) bank194018 +194019 POINT(37.24846731438698 -121.7229670136607) bank194019 +194020 POINT(37.17767315368252 -123.14274937406839) bank194020 +194021 POINT(37.474159551140595 -122.17992594860902) bank194021 +194022 POINT(37.65369491069487 -123.09110696549288) bank194022 +194023 POINT(37.304955544124084 -121.49095155945703) bank194023 +194024 POINT(38.736072289020406 -121.75808380153987) bank194024 +194025 POINT(37.14678498368488 -121.97365591348536) bank194025 +194026 POINT(38.13652438365128 -122.78070171516981) bank194026 +194027 POINT(36.80820476376242 -121.56540367780836) bank194027 +194028 POINT(37.47974568404932 -121.97714652994004) bank194028 +194029 POINT(37.55845886409015 -121.63063544728455) bank194029 +194030 POINT(37.45028293398761 -123.17049935861105) bank194030 +194031 POINT(36.983849017256894 -122.64053644923933) bank194031 +194032 POINT(37.6231334614682 -122.27080046374638) bank194032 +194033 POINT(36.93216817368584 -121.73990001139016) bank194033 +194034 POINT(37.45210446190831 -121.54460131486897) bank194034 +194035 POINT(38.4423726029459 -123.27574899002163) bank194035 +194036 POINT(37.97865001417733 -122.64028135466008) bank194036 +194037 POINT(38.23269613574262 -122.74865717730663) bank194037 +194038 POINT(37.92979019487012 -123.22999532512657) bank194038 +194039 POINT(37.53472352170619 -121.5528868787853) bank194039 +194040 POINT(37.491179819884415 -123.35086075043488) bank194040 +194041 POINT(38.473380832015756 -123.32783247336441) bank194041 +194042 POINT(37.26450592467479 -121.8859467544618) bank194042 +194043 POINT(37.56167215118578 -122.76360421691142) bank194043 +194044 POINT(37.801316744839745 -122.42425810001133) bank194044 +194045 POINT(37.7863706070599 -122.73311137676554) bank194045 +194046 POINT(38.279635183500666 -122.97987087010658) bank194046 +194047 POINT(38.55248382758332 -121.52199133411432) bank194047 +194048 POINT(36.813051459932666 -122.19994179505564) bank194048 +194049 POINT(37.20855745234669 -123.03007978659078) bank194049 +194050 POINT(37.5256026647985 -123.36092246446191) bank194050 +194051 POINT(38.46025400636755 -121.89879396151507) bank194051 +194052 POINT(38.53605578989705 -121.94079103285502) bank194052 +194053 POINT(38.06119884446291 -123.28905330329418) bank194053 +194054 POINT(38.23904990016542 -122.91388505508856) bank194054 +194055 POINT(38.24456763519971 -121.44790176760436) bank194055 +194056 POINT(38.505489319274815 -122.83323125248974) bank194056 +194057 POINT(36.84535674974924 -122.23497847148919) bank194057 +194058 POINT(37.18220116391167 -122.11590293807336) bank194058 +194059 POINT(37.623624002575845 -122.68884579190507) bank194059 +194060 POINT(38.734232046251286 -123.32910404683138) bank194060 +194061 POINT(36.87809734627692 -121.9524051384679) bank194061 +194062 POINT(38.292920720104775 -121.89448683633547) bank194062 +194063 POINT(38.52535223945525 -121.88767428487594) bank194063 +194064 POINT(37.03729568684124 -121.71147502631644) bank194064 +194065 POINT(38.49652870020505 -121.96466599714478) bank194065 +194066 POINT(38.4037410547016 -122.31938519431583) bank194066 +194067 POINT(37.19156306003764 -122.9615213378855) bank194067 +194068 POINT(36.81514252206396 -121.70069833297643) bank194068 +194069 POINT(38.551783491039785 -123.15566376823203) bank194069 +194070 POINT(37.802934088981765 -121.65093099324031) bank194070 +194071 POINT(37.981768074632996 -123.38482325388311) bank194071 +194072 POINT(36.87763873881988 -121.75621403801561) bank194072 +194073 POINT(37.705293010266836 -121.86915205773255) bank194073 +194074 POINT(37.932412461519675 -122.81820563017891) bank194074 +194075 POINT(38.29097721173442 -122.00968086251575) bank194075 +194076 POINT(36.97930511639063 -122.35675162967827) bank194076 +194077 POINT(38.46677915566063 -122.09194170087902) bank194077 +194078 POINT(37.20007695429172 -121.42847144575373) bank194078 +194079 POINT(37.67350837112388 -122.82887976729637) bank194079 +194080 POINT(38.60206071678146 -122.38943742077129) bank194080 +194081 POINT(38.03756362071004 -122.7499080316326) bank194081 +194082 POINT(38.29951132797167 -123.36128002209513) bank194082 +194083 POINT(37.32415247503088 -122.54408028673785) bank194083 +194084 POINT(36.96969968862742 -121.62702609689703) bank194084 +194085 POINT(36.910304444841785 -122.09221883353644) bank194085 +194086 POINT(37.91057969526972 -122.92533162732917) bank194086 +194087 POINT(37.86734865833122 -122.7091764479344) bank194087 +194088 POINT(37.27580151152378 -122.8256695945143) bank194088 +194089 POINT(38.69105207907442 -122.98237998281111) bank194089 +194090 POINT(38.096091545389505 -122.23965828189228) bank194090 +194091 POINT(37.77201747020037 -122.81809646888745) bank194091 +194092 POINT(36.89778326197922 -121.58811407538695) bank194092 +194093 POINT(37.77000347231962 -123.18956633979293) bank194093 +194094 POINT(37.89538675350271 -122.13800732900505) bank194094 +194095 POINT(37.952217690827034 -122.49836430771461) bank194095 +194096 POINT(37.768545447338816 -123.2926606498522) bank194096 +194097 POINT(37.56341585875291 -123.02910115545478) bank194097 +194098 POINT(37.54296724148991 -122.54033836737914) bank194098 +194099 POINT(37.8993178079342 -121.44744237923699) bank194099 +194100 POINT(37.374030662697436 -123.05094643699555) bank194100 +194101 POINT(37.58487908019403 -122.79819954563563) bank194101 +194102 POINT(38.2427965028998 -122.85255916897286) bank194102 +194103 POINT(38.408389644597264 -122.7835901132549) bank194103 +194104 POINT(37.60122694400435 -122.77926455544232) bank194104 +194105 POINT(37.895607620830155 -123.20490412207221) bank194105 +194106 POINT(37.7827756130015 -122.249993393542) bank194106 +194107 POINT(38.177636192491946 -123.18518408470916) bank194107 +194108 POINT(36.94293231272829 -122.58155259776458) bank194108 +194109 POINT(37.318503475909026 -121.55650480021461) bank194109 +194110 POINT(38.69474541299233 -123.19689196392605) bank194110 +194111 POINT(38.10618872965035 -121.70144965902921) bank194111 +194112 POINT(37.545063910575884 -121.80309994010527) bank194112 +194113 POINT(37.12785563502561 -122.24289107237266) bank194113 +194114 POINT(37.52382397331211 -122.65076276147519) bank194114 +194115 POINT(38.228332509696656 -123.29189664697518) bank194115 +194116 POINT(38.18859102750751 -122.8878013312029) bank194116 +194117 POINT(38.587678104983894 -123.21973105917033) bank194117 +194118 POINT(38.22663382684888 -121.60879479621279) bank194118 +194119 POINT(38.54801555914867 -121.88340682880899) bank194119 +194120 POINT(37.723763347225926 -121.53343399130033) bank194120 +194121 POINT(37.95473012055214 -121.68433141262477) bank194121 +194122 POINT(37.935409118478674 -121.85874870647565) bank194122 +194123 POINT(37.14122171825276 -122.88188440144) bank194123 +194124 POINT(36.92601539368782 -122.75182731948047) bank194124 +194125 POINT(36.95809345918667 -121.66698112352628) bank194125 +194126 POINT(37.31675404689587 -122.22951166443362) bank194126 +194127 POINT(37.70666034156299 -123.08230836223258) bank194127 +194128 POINT(38.40126312007517 -122.47225499982592) bank194128 +194129 POINT(37.870217355181985 -123.26228925694372) bank194129 +194130 POINT(37.65468135517643 -122.34832354901317) bank194130 +194131 POINT(36.976699886700445 -121.9960793886483) bank194131 +194132 POINT(37.741443033214544 -123.16449107448683) bank194132 +194133 POINT(36.87621892338836 -123.22979628562109) bank194133 +194134 POINT(36.86850316856091 -122.1365630425974) bank194134 +194135 POINT(37.61205040472504 -121.90390587342418) bank194135 +194136 POINT(37.61347375420012 -122.90985778104661) bank194136 +194137 POINT(37.199500925672545 -123.04470843306895) bank194137 +194138 POINT(37.31103874071587 -123.19465664674678) bank194138 +194139 POINT(37.185848767332764 -122.67629950081357) bank194139 +194140 POINT(37.38182364682581 -121.64986457918863) bank194140 +194141 POINT(38.203915246309435 -121.49362665637459) bank194141 +194142 POINT(38.176913844278495 -122.33486263193141) bank194142 +194143 POINT(37.92519088469098 -123.22722683194688) bank194143 +194144 POINT(37.45856823933368 -122.59518541689638) bank194144 +194145 POINT(38.66308882438877 -123.39892952546751) bank194145 +194146 POINT(38.634815917899004 -121.61059811967475) bank194146 +194147 POINT(37.04219713158654 -122.21264331960444) bank194147 +194148 POINT(37.83566507976609 -122.88107255232522) bank194148 +194149 POINT(37.67421496176742 -121.69326851142567) bank194149 +194150 POINT(37.22871685037821 -123.21057817135333) bank194150 +194151 POINT(37.16421929565384 -123.2933036163035) bank194151 +194152 POINT(37.467246920545115 -121.47179903847007) bank194152 +194153 POINT(37.15106435590775 -121.4726824899771) bank194153 +194154 POINT(37.257280525886706 -121.99555623040345) bank194154 +194155 POINT(38.2403071981872 -122.25798411229566) bank194155 +194156 POINT(37.567937766371784 -122.39687308558706) bank194156 +194157 POINT(37.95945999732929 -123.24123388756789) bank194157 +194158 POINT(38.62615647489639 -122.61014764340825) bank194158 +194159 POINT(38.338362329522674 -122.14375377266539) bank194159 +194160 POINT(38.268958631954014 -121.62503835344143) bank194160 +194161 POINT(38.43085185209913 -122.47307718112049) bank194161 +194162 POINT(37.12649019312877 -123.23942749936056) bank194162 +194163 POINT(37.21776686840959 -122.61757495928984) bank194163 +194164 POINT(36.923958767161814 -123.07511359820208) bank194164 +194165 POINT(38.02701118837972 -122.37664917369275) bank194165 +194166 POINT(36.93632533797708 -122.51934764022162) bank194166 +194167 POINT(37.42204035763959 -122.41436416585461) bank194167 +194168 POINT(37.77050394382944 -123.09669611907253) bank194168 +194169 POINT(37.598153313529174 -123.11195577444138) bank194169 +194170 POINT(38.04794308204268 -122.2810532527706) bank194170 +194171 POINT(38.27138229073233 -121.63500261136122) bank194171 +194172 POINT(37.74919622534859 -123.09196519340135) bank194172 +194173 POINT(38.6394414580495 -122.04273163269959) bank194173 +194174 POINT(38.5803256613094 -121.58558188083298) bank194174 +194175 POINT(38.691353814904986 -123.38130510550455) bank194175 +194176 POINT(36.80785146323976 -123.00700276035695) bank194176 +194177 POINT(38.597105394096474 -122.25537868919848) bank194177 +194178 POINT(37.58852222266873 -121.44200319984861) bank194178 +194179 POINT(37.917458355356345 -122.12786591525445) bank194179 +194180 POINT(36.77852926265229 -122.04616599925761) bank194180 +194181 POINT(36.79133281879071 -122.8844407882964) bank194181 +194182 POINT(37.5001599478207 -121.92520564201763) bank194182 +194183 POINT(38.45628207855036 -122.68319351223664) bank194183 +194184 POINT(37.99835763391949 -123.3357579625344) bank194184 +194185 POINT(37.627651447746025 -121.57394037803935) bank194185 +194186 POINT(37.134936545856 -121.82233884147296) bank194186 +194187 POINT(38.43433996745514 -121.57795389216942) bank194187 +194188 POINT(38.220057005410794 -121.87109936695971) bank194188 +194189 POINT(36.874335697093656 -122.41915422055062) bank194189 +194190 POINT(38.759530271126536 -122.91781748364848) bank194190 +194191 POINT(37.77231317858056 -122.27835740251551) bank194191 +194192 POINT(37.81715228764185 -121.42602578101713) bank194192 +194193 POINT(37.76399424103191 -122.13114477799753) bank194193 +194194 POINT(37.85556744286369 -122.36379583624635) bank194194 +194195 POINT(38.06211209977835 -122.72845866725618) bank194195 +194196 POINT(36.801829258469404 -122.19939257831223) bank194196 +194197 POINT(37.72010645012151 -121.67778526007737) bank194197 +194198 POINT(37.97084073818009 -121.67898919757852) bank194198 +194199 POINT(38.628822934278894 -121.95062700524385) bank194199 +194200 POINT(38.13177432014229 -122.78257200441965) bank194200 +194201 POINT(38.461537390488246 -121.76272312470337) bank194201 +194202 POINT(38.3152180885388 -123.18942331037461) bank194202 +194203 POINT(37.91676347963835 -122.94073325605338) bank194203 +194204 POINT(36.83228222356541 -121.51000473088322) bank194204 +194205 POINT(37.127824531162275 -123.18475206001365) bank194205 +194206 POINT(37.48267776656546 -122.34114292113753) bank194206 +194207 POINT(37.398163415905806 -123.1175608515506) bank194207 +194208 POINT(38.319912334642844 -122.78805197691673) bank194208 +194209 POINT(38.24258064326167 -122.63073045165285) bank194209 +194210 POINT(37.31235525001078 -122.6308201710757) bank194210 +194211 POINT(37.52217366772238 -121.85318179931454) bank194211 +194212 POINT(37.12910128215815 -121.93499234247864) bank194212 +194213 POINT(37.518563750483935 -122.46228740017055) bank194213 +194214 POINT(37.47311697536429 -122.12080789414402) bank194214 +194215 POINT(37.10057831756631 -122.64817715393953) bank194215 +194216 POINT(37.839602559573216 -121.77888894821857) bank194216 +194217 POINT(38.21376644349721 -121.9547914447536) bank194217 +194218 POINT(38.26766180220366 -122.06733797493366) bank194218 +194219 POINT(38.32283842801978 -121.4363071850682) bank194219 +194220 POINT(38.26491864502525 -123.2019081216029) bank194220 +194221 POINT(38.77156631539219 -122.89872815353834) bank194221 +194222 POINT(38.708532147087084 -121.79798807858421) bank194222 +194223 POINT(37.10516872989145 -123.35659866702584) bank194223 +194224 POINT(37.82619325231364 -122.2248335371671) bank194224 +194225 POINT(38.47086199092281 -122.08340442817753) bank194225 +194226 POINT(37.1691934681169 -122.45553717055982) bank194226 +194227 POINT(37.833998076494886 -122.37240127128507) bank194227 +194228 POINT(38.523627086879735 -123.36613799325443) bank194228 +194229 POINT(36.78591461681796 -121.77476960000455) bank194229 +194230 POINT(38.09206081465575 -123.3511571171394) bank194230 +194231 POINT(37.40855747200389 -122.19664470989423) bank194231 +194232 POINT(37.66551350312477 -121.4297361961696) bank194232 +194233 POINT(36.944461791142956 -123.09498651772483) bank194233 +194234 POINT(38.76488633289215 -122.8004141742475) bank194234 +194235 POINT(37.891224456731884 -123.38570292300956) bank194235 +194236 POINT(38.27440015936267 -123.263024015174) bank194236 +194237 POINT(38.204361594867 -122.23073367058221) bank194237 +194238 POINT(38.08369764634998 -121.49106614828028) bank194238 +194239 POINT(37.69050937628571 -121.42311554833958) bank194239 +194240 POINT(37.5558303420145 -122.31145612686491) bank194240 +194241 POINT(36.90899526506019 -122.45570969191628) bank194241 +194242 POINT(38.01497432877758 -122.27838890775962) bank194242 +194243 POINT(37.2005043246257 -122.31711423099625) bank194243 +194244 POINT(37.99542571752745 -121.85484335257628) bank194244 +194245 POINT(36.817691346746386 -121.98191603301382) bank194245 +194246 POINT(36.92142197724821 -122.39955155569179) bank194246 +194247 POINT(37.61886904860202 -123.05913992318769) bank194247 +194248 POINT(37.25933178556596 -123.30215261918917) bank194248 +194249 POINT(36.83682358506286 -122.9412209009181) bank194249 +194250 POINT(36.89647817839065 -122.85481842693166) bank194250 +194251 POINT(37.24586240803813 -122.40853914392623) bank194251 +194252 POINT(38.74360215444607 -123.40904798130197) bank194252 +194253 POINT(37.670936562308064 -122.47547502807265) bank194253 +194254 POINT(37.36232007545952 -121.94915202161344) bank194254 +194255 POINT(38.425102482395886 -121.54538870272472) bank194255 +194256 POINT(38.267095820803306 -122.18949047710754) bank194256 +194257 POINT(38.31947462897066 -123.17855930455576) bank194257 +194258 POINT(37.99984825279737 -122.86070858296422) bank194258 +194259 POINT(37.35736547531952 -122.31388223810718) bank194259 +194260 POINT(37.45068137625844 -122.41774516533945) bank194260 +194261 POINT(37.19985494778247 -123.24386905968277) bank194261 +194262 POINT(37.92464129598165 -121.90768825728358) bank194262 +194263 POINT(37.722256900295044 -122.52399998678997) bank194263 +194264 POINT(38.559158553533024 -123.16392191123353) bank194264 +194265 POINT(38.27610759267282 -122.20382561894533) bank194265 +194266 POINT(37.6388749619581 -122.24663375065373) bank194266 +194267 POINT(37.891109054291945 -122.97848250302906) bank194267 +194268 POINT(38.705545006344906 -121.88020610400555) bank194268 +194269 POINT(36.78369585547991 -122.56008575805984) bank194269 +194270 POINT(38.45943139740406 -122.1578394545207) bank194270 +194271 POINT(37.299575621906406 -122.09057291883732) bank194271 +194272 POINT(37.95303451070597 -122.86767934090115) bank194272 +194273 POINT(37.409363303637676 -123.36586344209341) bank194273 +194274 POINT(38.471085672295885 -121.74801484884921) bank194274 +194275 POINT(37.70072718767186 -122.35594421919517) bank194275 +194276 POINT(37.87892523431339 -122.22989059733705) bank194276 +194277 POINT(37.629072628782154 -123.38063620285914) bank194277 +194278 POINT(37.76211788273996 -122.77626103458526) bank194278 +194279 POINT(37.16264877414771 -123.01799896676613) bank194279 +194280 POINT(37.789193751228595 -123.17591442277694) bank194280 +194281 POINT(37.587477020731484 -121.43897441104582) bank194281 +194282 POINT(38.19160736375183 -122.19815956819677) bank194282 +194283 POINT(38.24400647797255 -121.64559243719398) bank194283 +194284 POINT(37.46043109898445 -121.76149571242401) bank194284 +194285 POINT(38.13671214201879 -122.83405462498793) bank194285 +194286 POINT(37.19297677437154 -121.61021036023091) bank194286 +194287 POINT(37.956140090340924 -123.28010801207675) bank194287 +194288 POINT(38.13499842187213 -122.82478895633989) bank194288 +194289 POINT(38.22846035027123 -122.5776652016092) bank194289 +194290 POINT(38.080653337338724 -122.24997210544734) bank194290 +194291 POINT(37.99876280905466 -121.77017728312961) bank194291 +194292 POINT(36.96715745442123 -122.99724493619297) bank194292 +194293 POINT(38.539872741165745 -123.20573526425926) bank194293 +194294 POINT(36.88804449445558 -122.09407072244089) bank194294 +194295 POINT(38.77091895890232 -122.04463350262331) bank194295 +194296 POINT(36.93251901441789 -121.67015005906183) bank194296 +194297 POINT(38.75883131637271 -123.27129433890052) bank194297 +194298 POINT(38.13371120084352 -122.3074109098545) bank194298 +194299 POINT(37.605762679041185 -122.35882560826563) bank194299 +194300 POINT(37.107450955007764 -122.43411688348064) bank194300 +194301 POINT(38.3980326636741 -121.91422485342744) bank194301 +194302 POINT(38.188018900750755 -121.98186115829631) bank194302 +194303 POINT(37.510562327446785 -121.77360452341603) bank194303 +194304 POINT(37.40312421083981 -122.10354875658993) bank194304 +194305 POINT(38.106313368978185 -122.51672910172788) bank194305 +194306 POINT(36.88232977782444 -123.25696334483003) bank194306 +194307 POINT(37.81359322690149 -123.23237782263705) bank194307 +194308 POINT(37.71148036479219 -122.62407560373323) bank194308 +194309 POINT(37.15207121506375 -122.33040748000552) bank194309 +194310 POINT(37.696533827264666 -122.33958155086009) bank194310 +194311 POINT(38.56313945818883 -123.24968218932571) bank194311 +194312 POINT(37.55189189279653 -122.02914214442116) bank194312 +194313 POINT(37.90331343800129 -123.17204585844574) bank194313 +194314 POINT(37.71170897645517 -121.84025252541797) bank194314 +194315 POINT(38.06551762674728 -122.6392234378172) bank194315 +194316 POINT(37.537591093821305 -122.92837327572119) bank194316 +194317 POINT(38.653017286637436 -122.43638825676743) bank194317 +194318 POINT(37.132237413933034 -121.5788871944616) bank194318 +194319 POINT(38.14501305562851 -121.98584213107875) bank194319 +194320 POINT(38.23532646023936 -121.74859195829991) bank194320 +194321 POINT(37.19723972628836 -122.35476157792857) bank194321 +194322 POINT(38.39724861294945 -122.92090237222847) bank194322 +194323 POINT(36.900854659112774 -122.52683212930128) bank194323 +194324 POINT(38.09895842593363 -121.97348076728711) bank194324 +194325 POINT(36.97462258143261 -122.37161020600088) bank194325 +194326 POINT(37.00297024963583 -122.07822881589165) bank194326 +194327 POINT(38.018395566974625 -123.09441753998944) bank194327 +194328 POINT(38.248517728506194 -122.54345759632275) bank194328 +194329 POINT(38.66242838309622 -122.50594917582691) bank194329 +194330 POINT(38.04047330053844 -123.10429577638078) bank194330 +194331 POINT(38.37520242300473 -122.04793100715366) bank194331 +194332 POINT(37.24692881238454 -122.45466477252654) bank194332 +194333 POINT(37.65016033917771 -123.31563906909935) bank194333 +194334 POINT(38.44282470783414 -123.01793846610765) bank194334 +194335 POINT(36.97061306859529 -122.06160542529693) bank194335 +194336 POINT(38.31969764391845 -121.50685878908888) bank194336 +194337 POINT(38.57217636914785 -121.86246223911283) bank194337 +194338 POINT(37.50061736969464 -122.77460427018956) bank194338 +194339 POINT(37.49521377678204 -122.50955124280296) bank194339 +194340 POINT(37.28128507787395 -122.99522764508536) bank194340 +194341 POINT(37.303057322113 -121.63496126801893) bank194341 +194342 POINT(37.11712410125345 -123.40278143591259) bank194342 +194343 POINT(38.39353277791204 -121.66782679128826) bank194343 +194344 POINT(37.88507895651771 -122.78382591258385) bank194344 +194345 POINT(37.15375689075587 -123.03209467683578) bank194345 +194346 POINT(37.19784056323823 -122.99565435922322) bank194346 +194347 POINT(38.34352736016032 -122.9331948680613) bank194347 +194348 POINT(38.27837428964672 -121.91344418634895) bank194348 +194349 POINT(38.57915084973225 -121.43780947243373) bank194349 +194350 POINT(37.85160672029811 -122.29479149091823) bank194350 +194351 POINT(36.82107813614158 -122.20885427823976) bank194351 +194352 POINT(38.145231469962745 -123.20060088289493) bank194352 +194353 POINT(38.33401146073777 -122.65824532544666) bank194353 +194354 POINT(38.76019176408188 -122.54376698178352) bank194354 +194355 POINT(37.518877811745995 -123.2423353837887) bank194355 +194356 POINT(36.791124107351514 -121.98719547878133) bank194356 +194357 POINT(37.53791781792984 -122.216506950225) bank194357 +194358 POINT(37.96075845298142 -121.51494298217946) bank194358 +194359 POINT(38.73171402756442 -123.15259275342046) bank194359 +194360 POINT(38.28413877745405 -122.99545233903856) bank194360 +194361 POINT(36.941770232400856 -123.37634543858826) bank194361 +194362 POINT(38.596535380714215 -123.27050746032762) bank194362 +194363 POINT(38.090066295379025 -122.06231860402613) bank194363 +194364 POINT(37.39110031227698 -121.96940104730474) bank194364 +194365 POINT(37.27281878969805 -122.31355160006832) bank194365 +194366 POINT(38.278645140322055 -123.02570608471851) bank194366 +194367 POINT(38.27393016221684 -122.44424314698263) bank194367 +194368 POINT(37.43549136819602 -121.458043346509) bank194368 +194369 POINT(37.04963165665207 -122.46181436385871) bank194369 +194370 POINT(37.53885995361196 -122.64172572797064) bank194370 +194371 POINT(37.65188941587634 -121.48665918124198) bank194371 +194372 POINT(37.80443765098215 -121.55576137604619) bank194372 +194373 POINT(38.524704745051366 -121.92541046901836) bank194373 +194374 POINT(37.20050768025715 -123.10888260378715) bank194374 +194375 POINT(38.65865304174303 -122.6799956324348) bank194375 +194376 POINT(37.796064624157246 -123.33691990066288) bank194376 +194377 POINT(37.08805723777234 -122.1833101383461) bank194377 +194378 POINT(38.08501621070115 -121.64722621089771) bank194378 +194379 POINT(37.1811252741602 -121.89758357335091) bank194379 +194380 POINT(38.57152059435526 -123.28380360757404) bank194380 +194381 POINT(36.934299502187045 -123.04895703397008) bank194381 +194382 POINT(37.68200347093244 -121.4324176435004) bank194382 +194383 POINT(38.758038737396475 -122.1718864410828) bank194383 +194384 POINT(37.587465783288216 -123.41687911000358) bank194384 +194385 POINT(38.53011569283162 -122.19778617431832) bank194385 +194386 POINT(37.704568499480615 -121.57073344308661) bank194386 +194387 POINT(37.60767863781489 -121.69698888683553) bank194387 +194388 POINT(36.900050986746116 -121.67758855322802) bank194388 +194389 POINT(37.19443886955888 -122.69927570981999) bank194389 +194390 POINT(38.48548640094233 -121.43574646215548) bank194390 +194391 POINT(36.784364599231225 -122.99477187661067) bank194391 +194392 POINT(37.69043682731608 -122.96326330496369) bank194392 +194393 POINT(37.418467189455676 -122.28096678212732) bank194393 +194394 POINT(36.893140870975174 -121.67973779591594) bank194394 +194395 POINT(37.96546870261822 -123.29355784148262) bank194395 +194396 POINT(36.85048655172534 -121.5288273093642) bank194396 +194397 POINT(37.25649041680661 -123.3067425243764) bank194397 +194398 POINT(37.345944067288144 -122.89696570024714) bank194398 +194399 POINT(37.86301542835663 -123.06310261318627) bank194399 +194400 POINT(38.51255856709037 -121.95893136896514) bank194400 +194401 POINT(38.75012594393696 -123.05195420130863) bank194401 +194402 POINT(37.69890645969811 -122.37828803341917) bank194402 +194403 POINT(37.26393274132578 -122.11455133685118) bank194403 +194404 POINT(37.46569200803584 -123.23852672117987) bank194404 +194405 POINT(37.20422927289263 -122.53841102562326) bank194405 +194406 POINT(37.41608370502376 -122.22724061005563) bank194406 +194407 POINT(37.480980110052535 -123.19727909940143) bank194407 +194408 POINT(38.17413895243742 -122.54385437540806) bank194408 +194409 POINT(37.25869262744626 -121.9514687492411) bank194409 +194410 POINT(38.07495386585386 -122.03924970602532) bank194410 +194411 POINT(38.0602859608545 -121.44651918420624) bank194411 +194412 POINT(37.6653136902471 -123.0941075821768) bank194412 +194413 POINT(38.42565666548406 -122.00863514180575) bank194413 +194414 POINT(38.513171669969026 -121.48077543510139) bank194414 +194415 POINT(37.48466047677415 -122.71486575206816) bank194415 +194416 POINT(37.387649623265744 -121.61771015291271) bank194416 +194417 POINT(38.7619302676658 -122.16546802501765) bank194417 +194418 POINT(37.85833524496386 -122.3276432485428) bank194418 +194419 POINT(36.91792961800724 -122.94434576440105) bank194419 +194420 POINT(36.99181415135758 -121.89423041677755) bank194420 +194421 POINT(38.7208270146868 -122.18357593077477) bank194421 +194422 POINT(36.91095494060298 -122.4251041899802) bank194422 +194423 POINT(37.821378082919914 -123.01581638905829) bank194423 +194424 POINT(37.63912404916462 -122.30491470956615) bank194424 +194425 POINT(36.88849888673106 -122.92594992840424) bank194425 +194426 POINT(36.99642371273602 -123.33023019830142) bank194426 +194427 POINT(37.62750110007657 -122.14722563808732) bank194427 +194428 POINT(36.792315802949446 -122.31536908370784) bank194428 +194429 POINT(38.48903727450219 -122.31703745481327) bank194429 +194430 POINT(37.3361904820324 -122.80888057326108) bank194430 +194431 POINT(37.3026966381429 -122.03404433101964) bank194431 +194432 POINT(38.05018343151589 -123.366503907908) bank194432 +194433 POINT(37.185987613095556 -121.49752665414536) bank194433 +194434 POINT(38.33015372141645 -122.4018844127263) bank194434 +194435 POINT(37.87788507341835 -122.37851754196372) bank194435 +194436 POINT(37.62380243651429 -123.23779925517712) bank194436 +194437 POINT(37.631899346996036 -122.85515849612013) bank194437 +194438 POINT(38.48926235383945 -122.9099943984346) bank194438 +194439 POINT(38.12523482536792 -121.75826133102288) bank194439 +194440 POINT(37.08805108446745 -121.60077236699516) bank194440 +194441 POINT(37.820140242298216 -122.57843917963207) bank194441 +194442 POINT(37.658244680852675 -123.19924291428663) bank194442 +194443 POINT(37.918092054131634 -122.72928217730718) bank194443 +194444 POINT(37.18266823413632 -123.34562018270559) bank194444 +194445 POINT(38.53827838953786 -121.5209896848347) bank194445 +194446 POINT(37.02856840689307 -121.42014588004882) bank194446 +194447 POINT(37.59252690598287 -121.71174732135891) bank194447 +194448 POINT(37.09386948769156 -123.30519725947582) bank194448 +194449 POINT(38.29388217797037 -121.52705401392454) bank194449 +194450 POINT(38.75011044886011 -121.44250124818637) bank194450 +194451 POINT(38.459946678260145 -122.24300625800709) bank194451 +194452 POINT(38.526397842725096 -122.9957286932483) bank194452 +194453 POINT(38.088069110448565 -122.65170175987909) bank194453 +194454 POINT(37.11756139934653 -123.26445178203412) bank194454 +194455 POINT(38.52602677531818 -122.90804892526177) bank194455 +194456 POINT(38.600181471872574 -122.14162021561378) bank194456 +194457 POINT(36.97737810916124 -122.44362112300608) bank194457 +194458 POINT(36.91969380953043 -122.55617121344912) bank194458 +194459 POINT(37.55195213505983 -121.73964167303544) bank194459 +194460 POINT(38.743888313588336 -122.85941696569188) bank194460 +194461 POINT(37.448226074503104 -122.35146085761815) bank194461 +194462 POINT(38.38108923466571 -121.43826337288127) bank194462 +194463 POINT(38.54329270957172 -122.58764053091205) bank194463 +194464 POINT(38.082090473490986 -122.89246090026147) bank194464 +194465 POINT(38.314486270535326 -122.97899716588296) bank194465 +194466 POINT(37.26397964015706 -123.15735057211863) bank194466 +194467 POINT(38.67557841487326 -123.00764533140485) bank194467 +194468 POINT(38.60320997663323 -122.51299418175299) bank194468 +194469 POINT(37.43798922955049 -121.8267720087716) bank194469 +194470 POINT(37.302264197117836 -122.99448414042193) bank194470 +194471 POINT(38.64342237556512 -123.39997282947402) bank194471 +194472 POINT(38.31283625023916 -122.82696696570473) bank194472 +194473 POINT(38.37774494932674 -121.49791325202241) bank194473 +194474 POINT(38.05248348454314 -122.74516290539803) bank194474 +194475 POINT(36.87554770692224 -122.95650230380014) bank194475 +194476 POINT(37.25862124241794 -122.30897644912264) bank194476 +194477 POINT(38.76574141975978 -123.39577740523625) bank194477 +194478 POINT(38.06060524401385 -121.82777868709483) bank194478 +194479 POINT(36.82073021424705 -122.2965121884522) bank194479 +194480 POINT(38.10348869846295 -121.89960546365467) bank194480 +194481 POINT(38.37677908737925 -121.9106826861021) bank194481 +194482 POINT(38.229481849917434 -121.92117809355183) bank194482 +194483 POINT(37.0210687343726 -122.62187499567153) bank194483 +194484 POINT(37.85904870757832 -123.07563710256986) bank194484 +194485 POINT(37.00455426732496 -121.90507438852664) bank194485 +194486 POINT(38.39875026180868 -121.8707086281845) bank194486 +194487 POINT(38.07674945437881 -121.47075196663947) bank194487 +194488 POINT(37.86046018676499 -122.9597600539352) bank194488 +194489 POINT(38.5232368327104 -121.83099158982014) bank194489 +194490 POINT(37.864525344546465 -122.68700509558876) bank194490 +194491 POINT(38.13272818275729 -121.64634023794693) bank194491 +194492 POINT(38.40101711203926 -122.72495826485465) bank194492 +194493 POINT(38.34883694352964 -122.27398659829002) bank194493 +194494 POINT(37.67964237402812 -122.67730904688702) bank194494 +194495 POINT(37.25952549230468 -121.99873433838002) bank194495 +194496 POINT(36.92530244243998 -122.38334195489645) bank194496 +194497 POINT(38.29351952457649 -122.89296965353218) bank194497 +194498 POINT(38.23206123165218 -122.01464946601553) bank194498 +194499 POINT(38.410227487855714 -122.7582655144671) bank194499 +194500 POINT(37.73029292232127 -122.16216609183634) bank194500 +194501 POINT(37.024511228389734 -121.86374751943556) bank194501 +194502 POINT(38.67867075288073 -121.44508757090685) bank194502 +194503 POINT(38.190551768805705 -121.8003409728917) bank194503 +194504 POINT(38.073007593675655 -123.35868979535643) bank194504 +194505 POINT(38.746271017404204 -122.35473499584985) bank194505 +194506 POINT(38.248739579328316 -122.25829642004699) bank194506 +194507 POINT(38.06171350483342 -122.15396699297297) bank194507 +194508 POINT(37.4620169242594 -121.85049928144139) bank194508 +194509 POINT(36.95122581504177 -122.40114076728254) bank194509 +194510 POINT(38.657434029582376 -121.5160443651927) bank194510 +194511 POINT(38.624367148616535 -121.57842662588325) bank194511 +194512 POINT(37.211236489931736 -122.98396629195557) bank194512 +194513 POINT(37.93330840614587 -123.35188758912771) bank194513 +194514 POINT(38.192078375587 -121.80652303685234) bank194514 +194515 POINT(38.60985115425529 -122.81619581680094) bank194515 +194516 POINT(36.78331527634722 -123.06651268137907) bank194516 +194517 POINT(38.13509676151869 -123.34021841346694) bank194517 +194518 POINT(37.73091789117478 -122.95534031161081) bank194518 +194519 POINT(37.268072829593144 -123.27820503343311) bank194519 +194520 POINT(37.253089450830366 -122.80457300218727) bank194520 +194521 POINT(38.17464586565266 -122.42998273915754) bank194521 +194522 POINT(37.10004075036006 -123.05890822516275) bank194522 +194523 POINT(37.492893160328556 -122.53691273889493) bank194523 +194524 POINT(37.7595188846392 -122.79442639017859) bank194524 +194525 POINT(37.36362996670948 -121.95249689037536) bank194525 +194526 POINT(38.02709772591423 -121.88154773020325) bank194526 +194527 POINT(37.59397829184987 -122.30939299392618) bank194527 +194528 POINT(36.9502435060251 -122.27015551963348) bank194528 +194529 POINT(37.96820715830775 -122.8423765258153) bank194529 +194530 POINT(38.73478515695519 -121.53742119105348) bank194530 +194531 POINT(37.53252429529822 -123.12995321186654) bank194531 +194532 POINT(38.43132915686617 -121.85911238984976) bank194532 +194533 POINT(37.69570290444226 -121.80047310882907) bank194533 +194534 POINT(37.88334779238073 -121.68482984586933) bank194534 +194535 POINT(37.615102236044024 -121.9772573130344) bank194535 +194536 POINT(38.38262841102072 -122.02986828506812) bank194536 +194537 POINT(38.69262108280913 -122.03857542135634) bank194537 +194538 POINT(38.2010421641598 -121.72750644075856) bank194538 +194539 POINT(37.50933848256135 -121.85636465918375) bank194539 +194540 POINT(37.09468338076239 -122.66274104544573) bank194540 +194541 POINT(37.37380559312396 -121.98271956697873) bank194541 +194542 POINT(38.4459813681307 -122.68682361837973) bank194542 +194543 POINT(37.2217930252639 -122.81513200633492) bank194543 +194544 POINT(37.74348281830384 -122.50392366252345) bank194544 +194545 POINT(38.45769445551853 -123.0682815063127) bank194545 +194546 POINT(37.68935107638855 -123.10925274310219) bank194546 +194547 POINT(37.949337717509756 -122.4828457633897) bank194547 +194548 POINT(37.77214167673947 -122.42750850006922) bank194548 +194549 POINT(38.77232517025242 -122.38978992882062) bank194549 +194550 POINT(38.194867362291134 -122.84489662109736) bank194550 +194551 POINT(38.08037070533229 -123.2348715433752) bank194551 +194552 POINT(37.279561178893495 -122.63410014252165) bank194552 +194553 POINT(37.102354555118616 -122.88420789510589) bank194553 +194554 POINT(37.99399998715768 -123.11126182553116) bank194554 +194555 POINT(36.81272477393441 -123.14405021148652) bank194555 +194556 POINT(38.71770950960745 -123.33399689233264) bank194556 +194557 POINT(37.39341690031003 -122.24994307735223) bank194557 +194558 POINT(36.826825730425796 -122.99148701383143) bank194558 +194559 POINT(37.363688836522975 -122.88894047733864) bank194559 +194560 POINT(37.68787248437427 -121.88265575058197) bank194560 +194561 POINT(36.80309338437733 -123.04844894437268) bank194561 +194562 POINT(38.380889560693866 -123.41766280158075) bank194562 +194563 POINT(36.97822998515646 -121.68681164339914) bank194563 +194564 POINT(38.335107271617595 -121.87758194726449) bank194564 +194565 POINT(36.862395824699206 -123.17002215462968) bank194565 +194566 POINT(38.20567608231571 -122.95569453236487) bank194566 +194567 POINT(38.63936843368108 -122.17485621085335) bank194567 +194568 POINT(36.88441700689306 -122.10020869083931) bank194568 +194569 POINT(38.13476082566241 -121.80001247630214) bank194569 +194570 POINT(37.61051463797538 -122.18169889635928) bank194570 +194571 POINT(38.51942358044558 -123.16227363674724) bank194571 +194572 POINT(37.91501820135692 -122.3287111021815) bank194572 +194573 POINT(38.14791466072281 -121.82741501230348) bank194573 +194574 POINT(38.2473586867336 -121.60589258414193) bank194574 +194575 POINT(38.12854231745442 -122.13741999328407) bank194575 +194576 POINT(38.65367902681008 -121.97493885422404) bank194576 +194577 POINT(37.14961756581381 -122.24712911498902) bank194577 +194578 POINT(38.25159560933846 -121.5557875600027) bank194578 +194579 POINT(38.394285245476 -123.14724634875752) bank194579 +194580 POINT(38.22978663643876 -122.17474880146625) bank194580 +194581 POINT(38.31130279314061 -122.36769653024527) bank194581 +194582 POINT(37.15445873468636 -121.93915971528274) bank194582 +194583 POINT(38.283670929424616 -122.33864114181483) bank194583 +194584 POINT(37.74433219325876 -121.65912332700515) bank194584 +194585 POINT(37.64822293857578 -122.77105084493573) bank194585 +194586 POINT(37.94106257857908 -123.19604000974289) bank194586 +194587 POINT(37.92114265036799 -122.20013929411792) bank194587 +194588 POINT(37.87056302958378 -122.09235736885971) bank194588 +194589 POINT(36.88647032336288 -122.98671214205874) bank194589 +194590 POINT(37.39535691747319 -121.99293145591129) bank194590 +194591 POINT(36.92006678336506 -122.59309631952239) bank194591 +194592 POINT(37.146861043393116 -122.0779055876542) bank194592 +194593 POINT(37.58120804157006 -121.67259650657128) bank194593 +194594 POINT(37.230723348239245 -121.94872951644672) bank194594 +194595 POINT(37.19585242530131 -123.35922358025736) bank194595 +194596 POINT(38.70847478092777 -123.08546029983219) bank194596 +194597 POINT(37.00761693057184 -123.23467279456297) bank194597 +194598 POINT(38.29940478742893 -121.77103629666819) bank194598 +194599 POINT(37.049169961674394 -122.2255537561482) bank194599 +194600 POINT(38.44277014033962 -122.97958872421496) bank194600 +194601 POINT(37.05248288347231 -121.47092858379281) bank194601 +194602 POINT(37.9078308065525 -121.90325693227325) bank194602 +194603 POINT(37.08932059043644 -123.28352457080075) bank194603 +194604 POINT(37.33012218269596 -122.4810653012909) bank194604 +194605 POINT(38.0230398717579 -122.18725545733973) bank194605 +194606 POINT(37.56333668939877 -123.393320232484) bank194606 +194607 POINT(37.145524043500004 -123.35325638667904) bank194607 +194608 POINT(37.67365410885484 -122.08394301620176) bank194608 +194609 POINT(37.497868514543285 -122.07036586011489) bank194609 +194610 POINT(38.37107593236207 -121.66871523236591) bank194610 +194611 POINT(37.24227359033721 -121.8123810863483) bank194611 +194612 POINT(37.06350791785084 -123.286079114687) bank194612 +194613 POINT(37.77368338210737 -122.51491187573632) bank194613 +194614 POINT(37.923096703586516 -121.86064822852354) bank194614 +194615 POINT(38.05764622146105 -123.0632581993619) bank194615 +194616 POINT(38.61124918329738 -121.94013691963758) bank194616 +194617 POINT(36.814637387083906 -123.34445633862809) bank194617 +194618 POINT(38.414696111309524 -122.89887694202703) bank194618 +194619 POINT(37.708290990668544 -123.06328908991219) bank194619 +194620 POINT(38.59221465526406 -122.76101750595245) bank194620 +194621 POINT(37.42116660048434 -122.76602954140414) bank194621 +194622 POINT(37.79452055537584 -121.68335424345487) bank194622 +194623 POINT(38.48338914324114 -122.08962913607627) bank194623 +194624 POINT(37.146036652161435 -121.81224087529367) bank194624 +194625 POINT(37.454273130067826 -122.38498581937411) bank194625 +194626 POINT(38.374001507123445 -123.08436369127578) bank194626 +194627 POINT(38.29939763907781 -123.15091888640625) bank194627 +194628 POINT(37.785168253397465 -122.64128351422134) bank194628 +194629 POINT(37.304969026571314 -123.11959434208086) bank194629 +194630 POINT(38.57291862756038 -121.84512728824919) bank194630 +194631 POINT(38.1360944372379 -123.09085442898784) bank194631 +194632 POINT(38.07925800473901 -121.58641019130081) bank194632 +194633 POINT(37.757450848923156 -123.01268989508283) bank194633 +194634 POINT(37.48416742627701 -121.50215553791455) bank194634 +194635 POINT(38.28087043304824 -123.27112894651852) bank194635 +194636 POINT(36.94405732276939 -122.21491785268296) bank194636 +194637 POINT(37.580499740807866 -123.02512186013035) bank194637 +194638 POINT(38.199494875376345 -123.09550482872136) bank194638 +194639 POINT(37.25787283752785 -122.06283406663329) bank194639 +194640 POINT(38.235538908350534 -122.33134585094767) bank194640 +194641 POINT(38.243743099211585 -122.91325321617505) bank194641 +194642 POINT(38.681414310628305 -123.27869516974809) bank194642 +194643 POINT(38.63059288690256 -121.7012491041412) bank194643 +194644 POINT(38.662594227690946 -122.08455098593551) bank194644 +194645 POINT(38.04548172943128 -122.89882845663993) bank194645 +194646 POINT(37.809385016615444 -121.49178891656261) bank194646 +194647 POINT(38.67421402561756 -122.06287942041288) bank194647 +194648 POINT(37.49610155207524 -123.22534809404081) bank194648 +194649 POINT(37.23445529235394 -122.46422437761771) bank194649 +194650 POINT(38.53852877293549 -122.73235598726791) bank194650 +194651 POINT(38.6439403152189 -122.21077341995813) bank194651 +194652 POINT(38.21407077083771 -122.27804195821074) bank194652 +194653 POINT(38.299957401863736 -122.93326923010284) bank194653 +194654 POINT(37.85731262926529 -122.18280143012515) bank194654 +194655 POINT(38.097603221811916 -123.23664937426618) bank194655 +194656 POINT(38.73415811113022 -121.47983348891826) bank194656 +194657 POINT(37.510708049197184 -123.06877912585475) bank194657 +194658 POINT(36.96575818317936 -122.9953773766106) bank194658 +194659 POINT(37.45214697983578 -122.70300614821252) bank194659 +194660 POINT(38.71731337230652 -122.23366053056816) bank194660 +194661 POINT(38.22524941407278 -122.15690129621522) bank194661 +194662 POINT(37.54749956472081 -122.7340939569751) bank194662 +194663 POINT(36.899943386625345 -122.10580456138804) bank194663 +194664 POINT(37.19550805344936 -122.373568142865) bank194664 +194665 POINT(36.84980705333888 -122.63597139620165) bank194665 +194666 POINT(38.44086970310331 -122.03200061532341) bank194666 +194667 POINT(37.153095028686515 -122.18305301908681) bank194667 +194668 POINT(37.04473101569766 -122.81122245500003) bank194668 +194669 POINT(38.03727639401211 -121.70312251497833) bank194669 +194670 POINT(37.67822714855788 -121.8860808152314) bank194670 +194671 POINT(37.85852289072484 -122.98108740564177) bank194671 +194672 POINT(38.60107965188536 -121.44487416380002) bank194672 +194673 POINT(37.903144775817985 -121.91891447129113) bank194673 +194674 POINT(38.46135119534063 -122.29116588415494) bank194674 +194675 POINT(36.90954138215645 -122.18058018218521) bank194675 +194676 POINT(36.992377260353955 -122.51889155718712) bank194676 +194677 POINT(38.43013931922997 -121.91596678828473) bank194677 +194678 POINT(38.028917963539705 -122.45218274026523) bank194678 +194679 POINT(37.70190632424373 -121.64787892659255) bank194679 +194680 POINT(37.256637419809664 -122.28552803661537) bank194680 +194681 POINT(37.884020370308235 -122.86929885181895) bank194681 +194682 POINT(37.90167762211468 -122.37639642614073) bank194682 +194683 POINT(36.928631333720375 -123.07404694589158) bank194683 +194684 POINT(37.48563837648919 -123.40618507183561) bank194684 +194685 POINT(36.79135799662057 -122.35200489429084) bank194685 +194686 POINT(38.51535277373275 -122.69242658121766) bank194686 +194687 POINT(38.68777371157358 -122.74947766449122) bank194687 +194688 POINT(38.3701133334126 -122.93227977615226) bank194688 +194689 POINT(37.09270514155981 -122.7451039496183) bank194689 +194690 POINT(37.33991772811704 -123.19244402527487) bank194690 +194691 POINT(38.3751977362143 -123.11235165730714) bank194691 +194692 POINT(37.09441578646347 -122.30887749596978) bank194692 +194693 POINT(37.31638191803001 -122.50385250033719) bank194693 +194694 POINT(37.620824534610925 -121.65385115281505) bank194694 +194695 POINT(38.3058693388948 -122.49435132975627) bank194695 +194696 POINT(37.41104781445372 -122.19878763255456) bank194696 +194697 POINT(38.01891749479569 -122.30286155224158) bank194697 +194698 POINT(38.3545630373073 -123.064920329627) bank194698 +194699 POINT(38.123848811434875 -122.82027872111384) bank194699 +194700 POINT(37.34890194370618 -121.82179341868753) bank194700 +194701 POINT(37.73169017570821 -122.63773721126832) bank194701 +194702 POINT(37.24662945084855 -122.65495757770444) bank194702 +194703 POINT(37.98294512255831 -121.64941638796006) bank194703 +194704 POINT(37.89761738067442 -122.33219011922895) bank194704 +194705 POINT(37.32360640290507 -121.67549197501803) bank194705 +194706 POINT(37.64287862217491 -122.22786152887153) bank194706 +194707 POINT(38.031758249143515 -122.41148981323016) bank194707 +194708 POINT(37.41135533941903 -122.13526532578335) bank194708 +194709 POINT(37.44657486635636 -122.42501615449157) bank194709 +194710 POINT(38.7668020338797 -122.22793237237077) bank194710 +194711 POINT(38.654732678408365 -122.94129083878013) bank194711 +194712 POINT(36.84869324244745 -122.09931881985432) bank194712 +194713 POINT(38.3034640366067 -122.53712073427992) bank194713 +194714 POINT(36.78457140775476 -123.25740775594893) bank194714 +194715 POINT(38.431167538373096 -121.53234182734415) bank194715 +194716 POINT(38.283501653795014 -122.05899904271321) bank194716 +194717 POINT(38.533374414785825 -122.03694731622909) bank194717 +194718 POINT(38.071213131898126 -123.0135006358595) bank194718 +194719 POINT(38.45782671466743 -122.51339477683022) bank194719 +194720 POINT(38.299789715444675 -123.33502184437408) bank194720 +194721 POINT(37.640530626987136 -122.0507730422475) bank194721 +194722 POINT(38.28715085301221 -123.07125269496541) bank194722 +194723 POINT(36.860374496221226 -122.68122252289984) bank194723 +194724 POINT(37.673539669225995 -122.2687303473601) bank194724 +194725 POINT(37.33343620039347 -121.49375518088877) bank194725 +194726 POINT(38.25536662353105 -121.8742788600784) bank194726 +194727 POINT(37.68898763753746 -122.3795449836667) bank194727 +194728 POINT(38.54616592125252 -122.80732735977314) bank194728 +194729 POINT(37.29831154322254 -122.09867883038122) bank194729 +194730 POINT(37.77079174397434 -121.92812538827681) bank194730 +194731 POINT(37.79825419162095 -123.03148078663273) bank194731 +194732 POINT(38.35873717510117 -123.18702151784413) bank194732 +194733 POINT(37.9079486945617 -121.86066715018262) bank194733 +194734 POINT(37.55388976791189 -121.76729660015211) bank194734 +194735 POINT(38.52753848881473 -121.53197609657497) bank194735 +194736 POINT(37.43650820990042 -122.72896292548427) bank194736 +194737 POINT(36.91633416425579 -122.63140271902542) bank194737 +194738 POINT(38.7095179629792 -122.87251290291697) bank194738 +194739 POINT(38.4006861242975 -122.85560234817494) bank194739 +194740 POINT(37.890397975089556 -122.39975419705885) bank194740 +194741 POINT(37.97030897075953 -123.29587018579) bank194741 +194742 POINT(38.46731343752743 -121.51045596644843) bank194742 +194743 POINT(37.50364681714023 -122.85187892880089) bank194743 +194744 POINT(36.80712623687557 -121.47856840175835) bank194744 +194745 POINT(38.4589448569991 -121.45527723657106) bank194745 +194746 POINT(37.03081083782615 -122.03017272724325) bank194746 +194747 POINT(38.738622312332204 -122.34357132363138) bank194747 +194748 POINT(37.9412360262061 -122.87549805899627) bank194748 +194749 POINT(38.11776483406273 -121.9451566881323) bank194749 +194750 POINT(36.913466063912644 -123.00111510657712) bank194750 +194751 POINT(37.958486749912325 -122.4500645832423) bank194751 +194752 POINT(38.76031160175003 -122.12109967694263) bank194752 +194753 POINT(38.3518385752228 -123.16884166596486) bank194753 +194754 POINT(37.87494000803261 -123.16676210755584) bank194754 +194755 POINT(38.41170945159841 -122.44541805301228) bank194755 +194756 POINT(38.13117779713424 -121.4394004625755) bank194756 +194757 POINT(38.08794280413668 -122.95749234343882) bank194757 +194758 POINT(38.411313347321034 -121.93088914803195) bank194758 +194759 POINT(37.55325449320819 -122.94275918105883) bank194759 +194760 POINT(37.48622793930682 -123.39016969823837) bank194760 +194761 POINT(38.17761735618941 -122.44641131753221) bank194761 +194762 POINT(37.98586400224771 -122.84662151558962) bank194762 +194763 POINT(38.09249004950166 -121.99163384696517) bank194763 +194764 POINT(37.79934458884173 -121.65885417991976) bank194764 +194765 POINT(37.868100229485634 -122.65608607578824) bank194765 +194766 POINT(38.4644861051337 -122.79477342246454) bank194766 +194767 POINT(38.20580112116267 -122.60936715499916) bank194767 +194768 POINT(38.38120110505232 -122.02099395032981) bank194768 +194769 POINT(37.28352149477592 -122.13263809361162) bank194769 +194770 POINT(37.84595224415063 -123.36156253803568) bank194770 +194771 POINT(38.31252124007812 -121.85637879422997) bank194771 +194772 POINT(38.035895242827095 -122.7811566150331) bank194772 +194773 POINT(38.072901364904084 -122.45354548066668) bank194773 +194774 POINT(37.66905747520418 -121.62208308351273) bank194774 +194775 POINT(37.9963463604536 -121.8895847179067) bank194775 +194776 POINT(37.08763429644775 -123.04739895613277) bank194776 +194777 POINT(38.51301925054024 -123.13619209101174) bank194777 +194778 POINT(37.64463690080905 -123.08622352734034) bank194778 +194779 POINT(36.97652036908454 -123.40459000820087) bank194779 +194780 POINT(38.34213159919015 -121.50912162107123) bank194780 +194781 POINT(38.05213123143955 -122.61330216232419) bank194781 +194782 POINT(38.119787852138195 -122.02962204688973) bank194782 +194783 POINT(37.529383547747074 -122.5750332179186) bank194783 +194784 POINT(37.1164320706112 -122.83749685614539) bank194784 +194785 POINT(37.06063399040551 -121.85949742867652) bank194785 +194786 POINT(38.61856790485058 -123.15838573926982) bank194786 +194787 POINT(37.24105903569688 -121.94987123094121) bank194787 +194788 POINT(37.178595991232996 -122.8616683570201) bank194788 +194789 POINT(37.48969933579436 -122.8321382244746) bank194789 +194790 POINT(37.118590974371145 -122.26154493037022) bank194790 +194791 POINT(38.07066446003514 -121.6627802323966) bank194791 +194792 POINT(38.661285510241555 -122.09330342368568) bank194792 +194793 POINT(38.483786633349325 -122.39981223531322) bank194793 +194794 POINT(38.69371143271577 -121.85236998625864) bank194794 +194795 POINT(37.86996442520515 -121.83511419048602) bank194795 +194796 POINT(37.730187328796525 -123.32903104883113) bank194796 +194797 POINT(38.07114144449908 -122.16800946629377) bank194797 +194798 POINT(38.37942448404907 -123.30938126071239) bank194798 +194799 POINT(37.057496122013184 -122.32155945789476) bank194799 +194800 POINT(37.89160026731418 -122.028243298035) bank194800 +194801 POINT(38.270875059963075 -121.42706495652233) bank194801 +194802 POINT(37.619182744427704 -121.77946213876494) bank194802 +194803 POINT(37.55214920146311 -123.1487124023871) bank194803 +194804 POINT(37.797338284688415 -122.70699471529733) bank194804 +194805 POINT(37.44912459713668 -121.67340751955534) bank194805 +194806 POINT(37.1582936288747 -122.47362765305648) bank194806 +194807 POINT(38.2573077023978 -122.72070493866381) bank194807 +194808 POINT(38.235033238796895 -123.08823375302276) bank194808 +194809 POINT(37.76513118286054 -122.23085699788037) bank194809 +194810 POINT(38.3443103094878 -122.28320075629242) bank194810 +194811 POINT(38.55963477053158 -122.11577671030764) bank194811 +194812 POINT(38.724284030459934 -123.379524746279) bank194812 +194813 POINT(37.81606315094891 -122.33883389928704) bank194813 +194814 POINT(37.77336558082187 -122.43564787042736) bank194814 +194815 POINT(36.89553803164763 -122.5885464469371) bank194815 +194816 POINT(37.53355016958989 -122.94891987219022) bank194816 +194817 POINT(37.93023211828553 -123.2545380765506) bank194817 +194818 POINT(38.23685911336708 -122.02538177393379) bank194818 +194819 POINT(37.703396682153425 -121.58179883137986) bank194819 +194820 POINT(36.85723259650571 -122.67275090307994) bank194820 +194821 POINT(37.0185916318632 -122.96746826986518) bank194821 +194822 POINT(36.81730249643093 -121.62992486654832) bank194822 +194823 POINT(36.993397677449245 -122.60727731161504) bank194823 +194824 POINT(37.6677045808938 -123.19445917190242) bank194824 +194825 POINT(38.772628417841 -121.92485863078602) bank194825 +194826 POINT(37.664891032295415 -123.36291369039175) bank194826 +194827 POINT(37.85595006803624 -122.32422455622161) bank194827 +194828 POINT(38.014114116230516 -122.9612674086211) bank194828 +194829 POINT(37.7621665517439 -123.22644940308459) bank194829 +194830 POINT(37.453076618706014 -122.50719714000898) bank194830 +194831 POINT(37.009632205406476 -122.71395803935461) bank194831 +194832 POINT(37.096524639134834 -123.24502505977473) bank194832 +194833 POINT(38.07416441176666 -121.87681782805475) bank194833 +194834 POINT(37.62254386783031 -121.69949488107142) bank194834 +194835 POINT(36.89468215864054 -122.08869385599354) bank194835 +194836 POINT(37.40505069184616 -122.88908934006608) bank194836 +194837 POINT(37.38750278296103 -121.55880615471368) bank194837 +194838 POINT(37.35679762671104 -123.18307421965454) bank194838 +194839 POINT(37.79787940381715 -123.19679307125945) bank194839 +194840 POINT(38.741387441292716 -123.3693937981194) bank194840 +194841 POINT(38.72917858129492 -121.66054426351428) bank194841 +194842 POINT(37.527632250751594 -122.92361846496898) bank194842 +194843 POINT(36.82993062852633 -121.69718159089071) bank194843 +194844 POINT(38.310620642120575 -122.32093723333843) bank194844 +194845 POINT(37.71413084830266 -122.29119397678548) bank194845 +194846 POINT(38.50333564132592 -123.01616677700537) bank194846 +194847 POINT(38.25245039125276 -122.59861824473903) bank194847 +194848 POINT(36.9167953113444 -122.49235255795442) bank194848 +194849 POINT(36.98869913857118 -121.63827721306109) bank194849 +194850 POINT(38.425281654143475 -123.00352029676615) bank194850 +194851 POINT(37.120701903140706 -122.92693474570677) bank194851 +194852 POINT(37.69188091850908 -123.3183635635364) bank194852 +194853 POINT(38.19823104070726 -123.3372186468415) bank194853 +194854 POINT(38.71531507419051 -121.88293767470275) bank194854 +194855 POINT(38.719405486353835 -122.61157306096356) bank194855 +194856 POINT(38.75428714811863 -122.18565445146788) bank194856 +194857 POINT(36.80264889681516 -121.68439318016253) bank194857 +194858 POINT(37.6756552920266 -123.238275162711) bank194858 +194859 POINT(38.74544546371664 -122.16663132117517) bank194859 +194860 POINT(38.024553208277446 -122.75365715120276) bank194860 +194861 POINT(37.70109809169204 -121.73483202629464) bank194861 +194862 POINT(38.51546166271631 -122.10627351668676) bank194862 +194863 POINT(37.191336148693495 -122.0405922588352) bank194863 +194864 POINT(37.886511943890255 -121.4771460241123) bank194864 +194865 POINT(38.72981294666403 -122.50643588245961) bank194865 +194866 POINT(37.99962856544797 -121.74412651853487) bank194866 +194867 POINT(38.49808474407596 -121.71842861541563) bank194867 +194868 POINT(36.79112468188169 -121.56299683894221) bank194868 +194869 POINT(37.84447194287044 -121.6864168482089) bank194869 +194870 POINT(38.74307448716597 -123.04847781996862) bank194870 +194871 POINT(37.378906924676585 -121.6408006411265) bank194871 +194872 POINT(38.046120374125515 -121.89765378982692) bank194872 +194873 POINT(38.5987193315124 -123.35888079808149) bank194873 +194874 POINT(36.797879344025255 -123.04549174720822) bank194874 +194875 POINT(38.14134778075988 -122.46386205885005) bank194875 +194876 POINT(37.91594919688945 -121.45826267502554) bank194876 +194877 POINT(36.915671255879765 -122.92488391142517) bank194877 +194878 POINT(37.62609997506382 -123.17279117137966) bank194878 +194879 POINT(38.442865330486775 -122.3718948484692) bank194879 +194880 POINT(37.37374060117796 -121.62398576987914) bank194880 +194881 POINT(38.64401264829515 -122.36414732577946) bank194881 +194882 POINT(38.290328958576346 -123.26915492099239) bank194882 +194883 POINT(38.7734856411998 -122.1202074768653) bank194883 +194884 POINT(36.86939846631531 -122.3188908741097) bank194884 +194885 POINT(38.65327331088019 -123.07535697437736) bank194885 +194886 POINT(36.971956413681646 -121.72864418662527) bank194886 +194887 POINT(37.92175827056172 -122.44621825595634) bank194887 +194888 POINT(37.14536267488509 -123.06561418316748) bank194888 +194889 POINT(38.24797116315138 -122.67819778968119) bank194889 +194890 POINT(37.36985586798945 -121.97031882472305) bank194890 +194891 POINT(38.13473109935095 -122.72993271909878) bank194891 +194892 POINT(38.514321827532214 -121.80582384204968) bank194892 +194893 POINT(36.909956366146226 -122.64891845015181) bank194893 +194894 POINT(37.52318600216092 -123.060912951886) bank194894 +194895 POINT(38.07186742937354 -123.27734905606455) bank194895 +194896 POINT(37.23993121100513 -122.90063193634502) bank194896 +194897 POINT(38.71391442203872 -122.29117113830159) bank194897 +194898 POINT(37.010045763905424 -121.95547087362395) bank194898 +194899 POINT(38.72573686903919 -122.60302882234603) bank194899 +194900 POINT(38.48334108451856 -122.3873854391219) bank194900 +194901 POINT(38.46203726274429 -122.5623578627249) bank194901 +194902 POINT(37.234048526921484 -122.95951805889243) bank194902 +194903 POINT(37.63995606013769 -121.46623725048525) bank194903 +194904 POINT(36.95210727076205 -121.97944143446878) bank194904 +194905 POINT(37.06936013447493 -122.82470757112885) bank194905 +194906 POINT(38.530658706766424 -122.1431356677041) bank194906 +194907 POINT(38.290729425543546 -122.99417157203649) bank194907 +194908 POINT(37.50844909311757 -122.35691587021022) bank194908 +194909 POINT(38.08531558065671 -122.36407769974834) bank194909 +194910 POINT(37.68212963078294 -122.98399675469817) bank194910 +194911 POINT(38.110506543395445 -121.45533006123347) bank194911 +194912 POINT(38.24386385882054 -122.22403323949402) bank194912 +194913 POINT(38.745273104937446 -121.45486924740402) bank194913 +194914 POINT(36.97851242052064 -122.5949287690838) bank194914 +194915 POINT(37.865998762945 -121.85783901119792) bank194915 +194916 POINT(38.65538938856537 -122.90578922496563) bank194916 +194917 POINT(38.35955380566187 -121.74750688714079) bank194917 +194918 POINT(37.81333026561884 -123.19433702772176) bank194918 +194919 POINT(38.28986850781597 -121.62208615910055) bank194919 +194920 POINT(37.18800918772688 -123.16599230314698) bank194920 +194921 POINT(37.22858353621202 -123.01136964015976) bank194921 +194922 POINT(38.1557760069258 -121.79882676912904) bank194922 +194923 POINT(37.070569202990605 -121.69587491157208) bank194923 +194924 POINT(37.36465692933182 -122.69996957733044) bank194924 +194925 POINT(37.93744315957013 -122.334142027803) bank194925 +194926 POINT(38.27854868725273 -121.58221035622283) bank194926 +194927 POINT(38.03284931460249 -122.86845274717801) bank194927 +194928 POINT(37.70698592110985 -122.1031234686194) bank194928 +194929 POINT(38.75526028031972 -123.00942338956995) bank194929 +194930 POINT(37.67284699422989 -122.77782588466005) bank194930 +194931 POINT(36.85785359046332 -121.9783808388898) bank194931 +194932 POINT(37.294734030209135 -121.900069923192) bank194932 +194933 POINT(36.779806109531734 -121.56352665052415) bank194933 +194934 POINT(38.420870722465416 -122.62053416464704) bank194934 +194935 POINT(37.0695874119498 -121.46015468251244) bank194935 +194936 POINT(37.09028798452152 -121.79414438098799) bank194936 +194937 POINT(37.76549381365595 -122.44398368072603) bank194937 +194938 POINT(38.545506792910125 -121.86236291120444) bank194938 +194939 POINT(37.67266266433222 -122.87803553581578) bank194939 +194940 POINT(38.742652646475065 -123.14869571681456) bank194940 +194941 POINT(38.65523889429544 -121.78199862817033) bank194941 +194942 POINT(38.037369074738386 -122.57540603750246) bank194942 +194943 POINT(38.389425702489795 -123.40482981486257) bank194943 +194944 POINT(38.25160613027992 -122.28948316646877) bank194944 +194945 POINT(37.021730238946965 -123.32643158799479) bank194945 +194946 POINT(38.42833537636848 -122.64684380790523) bank194946 +194947 POINT(38.02005931911105 -121.69123416985724) bank194947 +194948 POINT(38.52655799243893 -122.47211476542293) bank194948 +194949 POINT(37.797002069447664 -122.51905197637005) bank194949 +194950 POINT(37.01167614664119 -122.56400737557466) bank194950 +194951 POINT(36.87823997320637 -123.3591268059253) bank194951 +194952 POINT(38.061135411499684 -123.28354908635752) bank194952 +194953 POINT(37.5741534348264 -121.48415921034) bank194953 +194954 POINT(36.9982256012162 -121.43220934874239) bank194954 +194955 POINT(38.50333823853285 -121.52285259216877) bank194955 +194956 POINT(36.929800769936435 -121.4833101664486) bank194956 +194957 POINT(38.504167812331616 -122.02143882712355) bank194957 +194958 POINT(36.949513368333605 -121.84875315171534) bank194958 +194959 POINT(38.14436379955056 -123.16703212780384) bank194959 +194960 POINT(38.6103078664729 -121.91914961920904) bank194960 +194961 POINT(36.859645557013394 -121.48400613870301) bank194961 +194962 POINT(36.838030245911256 -122.3571132092892) bank194962 +194963 POINT(38.12060425597032 -122.68341801350654) bank194963 +194964 POINT(36.94448691684693 -122.68111291366698) bank194964 +194965 POINT(36.783105531540876 -123.21417896586604) bank194965 +194966 POINT(37.22068992545269 -122.90225670577829) bank194966 +194967 POINT(37.66447097568283 -122.89546648841552) bank194967 +194968 POINT(38.62868009303071 -122.35287498146579) bank194968 +194969 POINT(37.136694927940425 -122.99511445895803) bank194969 +194970 POINT(38.750029591362484 -121.48341233845063) bank194970 +194971 POINT(38.174688247778626 -122.70039949219625) bank194971 +194972 POINT(36.901191101713714 -122.58305386299156) bank194972 +194973 POINT(38.04642460689686 -123.00575306129231) bank194973 +194974 POINT(37.347275800902594 -122.04451839614867) bank194974 +194975 POINT(37.218228805234396 -121.61753865009855) bank194975 +194976 POINT(38.221912274119106 -122.34649214791868) bank194976 +194977 POINT(38.26172581268057 -122.1703671397242) bank194977 +194978 POINT(38.070565557903805 -121.52706948643386) bank194978 +194979 POINT(38.70053045012948 -121.52939859811447) bank194979 +194980 POINT(36.91292597999492 -121.71330321415226) bank194980 +194981 POINT(37.85262399285567 -122.56416841709684) bank194981 +194982 POINT(38.66952794947374 -123.24664562990168) bank194982 +194983 POINT(38.62164373493713 -122.35625188516838) bank194983 +194984 POINT(38.49784039467297 -122.05837202517631) bank194984 +194985 POINT(37.43439840831782 -122.43783898118848) bank194985 +194986 POINT(37.07687954141953 -121.86647462991195) bank194986 +194987 POINT(36.99691990561049 -121.65301801111644) bank194987 +194988 POINT(38.506707736200454 -122.43004027355938) bank194988 +194989 POINT(37.08320429688055 -123.25155502667542) bank194989 +194990 POINT(38.72863313088805 -121.75060331854444) bank194990 +194991 POINT(38.66399738143584 -122.31757562725117) bank194991 +194992 POINT(38.56638747937122 -123.35660354983582) bank194992 +194993 POINT(36.78039719544459 -121.57330150966628) bank194993 +194994 POINT(36.78347550755854 -121.79332321096463) bank194994 +194995 POINT(37.769966908373995 -123.18957517779816) bank194995 +194996 POINT(37.20118447990555 -122.7659707254978) bank194996 +194997 POINT(36.81712328263788 -121.42568702992719) bank194997 +194998 POINT(38.0359241858986 -123.21491740579191) bank194998 +194999 POINT(37.10133199245148 -122.92722726077061) bank194999 +195000 POINT(38.502012885373055 -122.78604525635448) bank195000 +195001 POINT(38.0938553950261 -123.22380633238677) bank195001 +195002 POINT(38.72546917012417 -122.29267120610912) bank195002 +195003 POINT(38.246597394310356 -121.644571822272) bank195003 +195004 POINT(37.93730077128143 -123.41467710423943) bank195004 +195005 POINT(38.67759282477908 -123.35222332480063) bank195005 +195006 POINT(37.13328753709308 -122.63192602137443) bank195006 +195007 POINT(37.09499975179795 -122.08125289499182) bank195007 +195008 POINT(37.18786202687583 -121.91048126440087) bank195008 +195009 POINT(36.93832996015095 -123.37763819077881) bank195009 +195010 POINT(36.897390980997116 -121.66562107970435) bank195010 +195011 POINT(38.75044219885228 -122.4723418147458) bank195011 +195012 POINT(37.79606631946517 -121.93948666902574) bank195012 +195013 POINT(36.96774452396614 -122.34256272215023) bank195013 +195014 POINT(37.31331793779201 -121.90492610388131) bank195014 +195015 POINT(37.868001648998245 -122.31164176578721) bank195015 +195016 POINT(37.293793977681695 -122.10606466690504) bank195016 +195017 POINT(38.26845234501456 -121.43565345750662) bank195017 +195018 POINT(37.56201526072884 -122.80649355437271) bank195018 +195019 POINT(36.82978990771641 -123.12470321389495) bank195019 +195020 POINT(38.011900489802535 -123.01584757915572) bank195020 +195021 POINT(37.130636297956435 -123.38604820364759) bank195021 +195022 POINT(37.046180898140186 -122.18804671370526) bank195022 +195023 POINT(37.60646180682437 -121.55346556273781) bank195023 +195024 POINT(37.52904288234282 -122.38494429807542) bank195024 +195025 POINT(36.97400327012681 -122.49389233308759) bank195025 +195026 POINT(37.19188176739245 -122.1162958196392) bank195026 +195027 POINT(38.05180046238491 -121.90421201365675) bank195027 +195028 POINT(38.38987040512009 -122.02581513801385) bank195028 +195029 POINT(38.42973734246663 -121.9856295730422) bank195029 +195030 POINT(37.30358707517602 -122.61428986195804) bank195030 +195031 POINT(38.7362326037781 -122.99046757195192) bank195031 +195032 POINT(38.13436759931812 -123.27893064874841) bank195032 +195033 POINT(37.732947894641384 -123.05460673111683) bank195033 +195034 POINT(38.713743159665434 -121.42571586383596) bank195034 +195035 POINT(36.858363284943295 -122.8516943311685) bank195035 +195036 POINT(36.97221348351476 -122.07041423817383) bank195036 +195037 POINT(37.5172534347135 -121.78019407227303) bank195037 +195038 POINT(38.70861582641616 -121.87978704792519) bank195038 +195039 POINT(38.20836690520519 -121.73007182716401) bank195039 +195040 POINT(37.02675365442472 -121.69322195612551) bank195040 +195041 POINT(37.87261066696584 -122.29920772189833) bank195041 +195042 POINT(37.548989285164794 -121.63414513037999) bank195042 +195043 POINT(37.96158986592174 -121.53825459056775) bank195043 +195044 POINT(38.49107504639764 -122.99933236153932) bank195044 +195045 POINT(38.72592492887218 -121.77045158662885) bank195045 +195046 POINT(37.59247567777324 -122.43777147480884) bank195046 +195047 POINT(38.71185226524642 -121.76188014837128) bank195047 +195048 POINT(37.51580435940877 -121.47987608199938) bank195048 +195049 POINT(38.59508400104913 -122.79714338071264) bank195049 +195050 POINT(37.92369522547139 -122.38920924485713) bank195050 +195051 POINT(37.68443106514771 -122.92163855457275) bank195051 +195052 POINT(37.49444205775701 -122.68056484439768) bank195052 +195053 POINT(37.72009336380745 -122.52959429015978) bank195053 +195054 POINT(37.93489604998921 -121.42353371110289) bank195054 +195055 POINT(36.78134987557447 -122.57248961664473) bank195055 +195056 POINT(37.944121283478424 -122.68590120745692) bank195056 +195057 POINT(38.75691343226571 -121.91726070649783) bank195057 +195058 POINT(37.589464719229284 -121.79062819291356) bank195058 +195059 POINT(37.48978458038925 -123.06693310732142) bank195059 +195060 POINT(37.01079207824953 -122.61656686889079) bank195060 +195061 POINT(38.48902803117836 -122.2005741525064) bank195061 +195062 POINT(37.36336794883447 -122.54055538100238) bank195062 +195063 POINT(38.62536357155208 -121.5496971078336) bank195063 +195064 POINT(37.08968765925166 -121.69417232084503) bank195064 +195065 POINT(37.90612548879975 -121.82699833144949) bank195065 +195066 POINT(38.36900748531677 -122.90758641285082) bank195066 +195067 POINT(36.821226267760686 -121.85182903495247) bank195067 +195068 POINT(36.961165964206906 -121.5488147048297) bank195068 +195069 POINT(37.74800051661179 -122.36790712240376) bank195069 +195070 POINT(37.52096665779506 -121.53676428984028) bank195070 +195071 POINT(37.85547723823227 -121.52618173480147) bank195071 +195072 POINT(38.763519667855924 -121.92845443741331) bank195072 +195073 POINT(37.475216812666844 -121.78961476554021) bank195073 +195074 POINT(38.348112035639225 -121.42300159827636) bank195074 +195075 POINT(36.92736004297305 -123.16535676509352) bank195075 +195076 POINT(37.35188380823922 -121.5425797436829) bank195076 +195077 POINT(38.268203426373354 -121.81938657822572) bank195077 +195078 POINT(37.00644741758925 -121.84788368299755) bank195078 +195079 POINT(37.60451243826699 -122.26533149262362) bank195079 +195080 POINT(37.36955691531451 -123.24290808790523) bank195080 +195081 POINT(37.133082516470246 -122.94744242006784) bank195081 +195082 POINT(37.03418540014067 -123.11769627448467) bank195082 +195083 POINT(38.290778707465215 -121.79359588698932) bank195083 +195084 POINT(37.00783049281958 -121.66561069622976) bank195084 +195085 POINT(38.626990645494615 -121.77409709111885) bank195085 +195086 POINT(38.73285716047447 -123.24709849863879) bank195086 +195087 POINT(37.14321810420102 -122.36741750543995) bank195087 +195088 POINT(37.87204412024956 -123.39282626988322) bank195088 +195089 POINT(37.458397464908394 -122.19259001924168) bank195089 +195090 POINT(38.50860182513744 -122.83194319918768) bank195090 +195091 POINT(37.42786731639693 -122.86484236771494) bank195091 +195092 POINT(38.18616006829802 -122.21664776317365) bank195092 +195093 POINT(38.04607959977983 -123.14223903426746) bank195093 +195094 POINT(37.94196774435079 -122.21581252822268) bank195094 +195095 POINT(37.468599409205886 -122.57717297943674) bank195095 +195096 POINT(37.3032107904852 -121.59908857980157) bank195096 +195097 POINT(36.90707692596315 -122.90459244937227) bank195097 +195098 POINT(37.79008417992176 -123.23408207702956) bank195098 +195099 POINT(36.97493160190181 -123.1924614694762) bank195099 +195100 POINT(37.70974530740709 -122.10579475667232) bank195100 +195101 POINT(38.19354367684836 -122.07395996174104) bank195101 +195102 POINT(37.75280928601659 -122.0068849004622) bank195102 +195103 POINT(37.43224419581202 -122.71416895352473) bank195103 +195104 POINT(37.05347895864484 -122.734808347684) bank195104 +195105 POINT(38.26543009156974 -122.85890011202133) bank195105 +195106 POINT(37.77977097457452 -121.85642492291758) bank195106 +195107 POINT(37.21033781334135 -123.18842355575036) bank195107 +195108 POINT(38.43420518615315 -123.00251894438114) bank195108 +195109 POINT(36.917855541367736 -122.25851601686928) bank195109 +195110 POINT(37.366409094592754 -123.4029049788744) bank195110 +195111 POINT(38.48660537305437 -123.23630816704922) bank195111 +195112 POINT(37.25235210966564 -122.29034921981892) bank195112 +195113 POINT(38.65160316769339 -121.6519571444468) bank195113 +195114 POINT(37.99216718785484 -121.55274833843343) bank195114 +195115 POINT(38.57792095465456 -123.23992691966592) bank195115 +195116 POINT(38.01160704271885 -123.05373770337035) bank195116 +195117 POINT(38.70766695894275 -123.24397967522052) bank195117 +195118 POINT(37.689070657663585 -122.62876740858296) bank195118 +195119 POINT(38.36859348123419 -122.81579044606633) bank195119 +195120 POINT(37.54305324076459 -123.14206587018006) bank195120 +195121 POINT(37.087234432752325 -122.11616169625788) bank195121 +195122 POINT(38.335554336142025 -122.24006383902729) bank195122 +195123 POINT(37.719072357127345 -122.38700605135244) bank195123 +195124 POINT(38.5364871365469 -122.85203206322629) bank195124 +195125 POINT(38.27498831793367 -121.48756253448212) bank195125 +195126 POINT(36.96072238576671 -122.97492775219693) bank195126 +195127 POINT(37.74763145822646 -122.10394599357858) bank195127 +195128 POINT(37.25820944751005 -122.58219470716307) bank195128 +195129 POINT(38.47906601794937 -121.90022748587032) bank195129 +195130 POINT(37.819434824582025 -121.72644197678524) bank195130 +195131 POINT(38.77072806240163 -122.23559641318549) bank195131 +195132 POINT(37.23375868218543 -122.21654158832301) bank195132 +195133 POINT(37.277857156609606 -121.87610146151513) bank195133 +195134 POINT(36.81763880372036 -123.39834387788248) bank195134 +195135 POINT(37.46850081134761 -122.61845839106704) bank195135 +195136 POINT(36.974644892226664 -122.65188514593682) bank195136 +195137 POINT(36.92333312046905 -122.6926432059711) bank195137 +195138 POINT(36.9272756128361 -122.22347890284175) bank195138 +195139 POINT(38.317746717369765 -122.97005560893061) bank195139 +195140 POINT(37.578294865951285 -121.45080218694068) bank195140 +195141 POINT(38.41637502624616 -121.89267887793518) bank195141 +195142 POINT(38.468265536153986 -122.867010751414) bank195142 +195143 POINT(37.14792492060205 -122.74091282997786) bank195143 +195144 POINT(36.85386292979343 -123.17046599091793) bank195144 +195145 POINT(37.45097386207015 -121.71480456124604) bank195145 +195146 POINT(37.06953570953689 -122.99477071035957) bank195146 +195147 POINT(37.63038334220818 -121.7208300173276) bank195147 +195148 POINT(38.76472271809305 -122.54729663712635) bank195148 +195149 POINT(37.51706082164186 -121.76556696112343) bank195149 +195150 POINT(38.28060514127518 -122.99498183129458) bank195150 +195151 POINT(38.16605750371257 -122.22074025236616) bank195151 +195152 POINT(38.73523346094851 -121.72915488695617) bank195152 +195153 POINT(37.13730602915183 -122.71343340542128) bank195153 +195154 POINT(37.79699629571558 -122.550185558842) bank195154 +195155 POINT(37.50859359108844 -121.83209136139341) bank195155 +195156 POINT(37.3261256586288 -123.34017973465926) bank195156 +195157 POINT(37.79092242437474 -122.0291985227809) bank195157 +195158 POINT(38.462139317529754 -123.09871247058025) bank195158 +195159 POINT(37.18032770418256 -123.05175480205125) bank195159 +195160 POINT(36.86559203357409 -122.52849187615857) bank195160 +195161 POINT(37.83723807295828 -122.11433415268334) bank195161 +195162 POINT(38.653200196522604 -123.06678036935736) bank195162 +195163 POINT(37.50994823003577 -123.2625034321672) bank195163 +195164 POINT(38.38276272497764 -122.83906576522297) bank195164 +195165 POINT(38.315385549411715 -122.8460047420247) bank195165 +195166 POINT(38.24352716487966 -122.05388799285195) bank195166 +195167 POINT(38.146411444724876 -122.85151344251896) bank195167 +195168 POINT(36.92042257291934 -123.28118321950136) bank195168 +195169 POINT(38.15054419704422 -122.10293589400669) bank195169 +195170 POINT(37.43369971500269 -122.05695337035658) bank195170 +195171 POINT(37.55582458750868 -121.5112592585731) bank195171 +195172 POINT(38.11598264825315 -122.3500865203478) bank195172 +195173 POINT(37.57781364673553 -121.98758643833617) bank195173 +195174 POINT(37.864796144808516 -122.45015371705155) bank195174 +195175 POINT(36.9175692327755 -122.49923438809004) bank195175 +195176 POINT(37.96805810387176 -123.01341906609267) bank195176 +195177 POINT(37.4644759983563 -121.53666394022056) bank195177 +195178 POINT(38.64959013272386 -122.04636397735817) bank195178 +195179 POINT(37.43912911329904 -122.7730995722102) bank195179 +195180 POINT(37.12755960036945 -122.02447805370024) bank195180 +195181 POINT(37.4134146334833 -121.72020451806142) bank195181 +195182 POINT(38.733520157986845 -122.91005066221626) bank195182 +195183 POINT(37.90512990898566 -121.70016514962167) bank195183 +195184 POINT(38.03226439979432 -121.71392849703085) bank195184 +195185 POINT(37.73215346627134 -121.82913924756897) bank195185 +195186 POINT(36.93360336377315 -122.28532140430042) bank195186 +195187 POINT(37.30634834475018 -121.55648721783892) bank195187 +195188 POINT(36.97777207782173 -122.7224048608331) bank195188 +195189 POINT(37.79930706709025 -121.55114477851575) bank195189 +195190 POINT(37.067408231438485 -123.25088037888072) bank195190 +195191 POINT(38.470642161290215 -123.28777092216151) bank195191 +195192 POINT(38.067044041147405 -121.42780653948728) bank195192 +195193 POINT(36.87098690833146 -122.64783022835091) bank195193 +195194 POINT(38.2041490546469 -121.75768971184058) bank195194 +195195 POINT(37.54277429875404 -122.59699099465072) bank195195 +195196 POINT(38.66714287620688 -121.82268467521018) bank195196 +195197 POINT(38.28667741232231 -121.84449182386851) bank195197 +195198 POINT(38.592283054638926 -122.14203223052961) bank195198 +195199 POINT(37.391710550729606 -122.418679266379) bank195199 +195200 POINT(38.5522176020784 -121.48012068476294) bank195200 +195201 POINT(38.19021250875549 -121.696467116858) bank195201 +195202 POINT(37.516454746962644 -122.07976609656309) bank195202 +195203 POINT(37.683779306560076 -121.63056807684757) bank195203 +195204 POINT(37.32864065926067 -123.14712099076623) bank195204 +195205 POINT(38.07400613112709 -122.0341509577247) bank195205 +195206 POINT(38.2349871904221 -121.46967634654742) bank195206 +195207 POINT(37.587282356423046 -121.49854744811255) bank195207 +195208 POINT(37.4712114325361 -121.49339130579529) bank195208 +195209 POINT(37.16239323592609 -121.87058701914961) bank195209 +195210 POINT(37.28842220003023 -122.71296424252003) bank195210 +195211 POINT(37.57647777751574 -121.43784445522074) bank195211 +195212 POINT(36.83738472172214 -122.71473477364177) bank195212 +195213 POINT(38.24707933822787 -122.61416968310047) bank195213 +195214 POINT(37.273469157158694 -122.17040556562146) bank195214 +195215 POINT(36.9288027250594 -122.95737046873765) bank195215 +195216 POINT(37.96248975378863 -122.43968070612067) bank195216 +195217 POINT(38.595213311723555 -122.9250361062202) bank195217 +195218 POINT(36.96959741278382 -123.20826468834863) bank195218 +195219 POINT(38.74459383524535 -121.73990453076648) bank195219 +195220 POINT(37.25754968791736 -121.59925857072454) bank195220 +195221 POINT(38.468133993902335 -122.21620612110566) bank195221 +195222 POINT(37.22590485048013 -123.01745035774786) bank195222 +195223 POINT(37.70375189911095 -121.81941731433021) bank195223 +195224 POINT(37.31273926091758 -122.78309779978773) bank195224 +195225 POINT(37.52235634990894 -123.0332211024265) bank195225 +195226 POINT(37.08050368073109 -122.06668241903269) bank195226 +195227 POINT(37.58146284860021 -122.17724866142778) bank195227 +195228 POINT(38.7031942626522 -123.00517249643843) bank195228 +195229 POINT(38.0994223190444 -122.48293539288146) bank195229 +195230 POINT(37.23015272094521 -122.27472745725363) bank195230 +195231 POINT(38.744414665313485 -123.30970699860232) bank195231 +195232 POINT(37.1753519679221 -122.95939857366152) bank195232 +195233 POINT(37.315307340220876 -122.96171563949905) bank195233 +195234 POINT(37.66929761293757 -123.08584276361034) bank195234 +195235 POINT(37.85137886845619 -121.50983747371387) bank195235 +195236 POINT(38.692438438942176 -121.89072057037079) bank195236 +195237 POINT(38.56317856439909 -123.18401054126944) bank195237 +195238 POINT(36.834199058911466 -122.79075075585597) bank195238 +195239 POINT(37.085912813644114 -122.16491806205848) bank195239 +195240 POINT(38.18908214308353 -122.14088821765326) bank195240 +195241 POINT(37.63605976275548 -122.30369125158856) bank195241 +195242 POINT(36.928132070524306 -122.06554092222937) bank195242 +195243 POINT(37.4030061344745 -121.9659480717519) bank195243 +195244 POINT(37.88881015536593 -121.75234995886683) bank195244 +195245 POINT(37.92618159475908 -121.90329907727488) bank195245 +195246 POINT(36.90948001435078 -122.46094536768673) bank195246 +195247 POINT(38.590779811257335 -122.2630695904923) bank195247 +195248 POINT(37.94838560359672 -123.11747885603144) bank195248 +195249 POINT(37.00768926968276 -123.19366438311373) bank195249 +195250 POINT(38.19571908231926 -122.41701718560695) bank195250 +195251 POINT(36.9378049050057 -122.86586846368795) bank195251 +195252 POINT(37.71099157208388 -122.13594693534613) bank195252 +195253 POINT(38.02138178989595 -123.34266984310949) bank195253 +195254 POINT(37.844423666643515 -122.02712218626962) bank195254 +195255 POINT(37.01957867766237 -122.16043946848542) bank195255 +195256 POINT(37.97458022422683 -122.36247714947449) bank195256 +195257 POINT(38.716515115540766 -121.71941232983865) bank195257 +195258 POINT(37.04615888656167 -122.03950219852094) bank195258 +195259 POINT(38.69686102299323 -123.21143602563407) bank195259 +195260 POINT(37.394625009378515 -121.44140834954253) bank195260 +195261 POINT(37.42166344545029 -122.47140083173367) bank195261 +195262 POINT(36.893817622044985 -122.14789818541709) bank195262 +195263 POINT(36.80291277925519 -122.15740763581647) bank195263 +195264 POINT(37.41418698342904 -122.32098659636814) bank195264 +195265 POINT(37.23566760339502 -122.18665732979358) bank195265 +195266 POINT(38.27858313699235 -121.47193672045046) bank195266 +195267 POINT(37.237105573026604 -122.44760566235848) bank195267 +195268 POINT(38.66855825080953 -122.44081216411246) bank195268 +195269 POINT(38.39381013185891 -122.30637765797353) bank195269 +195270 POINT(36.981229584576674 -122.12355559431175) bank195270 +195271 POINT(37.68357082009178 -122.64851346718241) bank195271 +195272 POINT(38.21362730474986 -123.39160155831088) bank195272 +195273 POINT(38.175702550326065 -122.74024864614131) bank195273 +195274 POINT(38.027111595272615 -122.40429444790007) bank195274 +195275 POINT(38.73267184855796 -121.51761346160778) bank195275 +195276 POINT(37.308888573956295 -123.10848189810554) bank195276 +195277 POINT(36.90338762775531 -123.37337198283102) bank195277 +195278 POINT(37.77112566264083 -121.65309861911726) bank195278 +195279 POINT(37.23786934018942 -122.67284244227496) bank195279 +195280 POINT(37.91627113406503 -121.51671599449638) bank195280 +195281 POINT(38.4387824349285 -123.41455557458043) bank195281 +195282 POINT(37.91194376005767 -122.1133684006535) bank195282 +195283 POINT(37.49282708057783 -121.97376999481344) bank195283 +195284 POINT(38.2896744691429 -122.50584065532611) bank195284 +195285 POINT(38.591031371814566 -121.43263782677072) bank195285 +195286 POINT(37.09755168951653 -121.81578159193488) bank195286 +195287 POINT(37.1546246969246 -122.66326144234871) bank195287 +195288 POINT(37.56532301766877 -121.69865573047157) bank195288 +195289 POINT(37.50293437544178 -123.16459778400619) bank195289 +195290 POINT(37.08657039286672 -122.6796290809784) bank195290 +195291 POINT(38.37920150690544 -121.73658780918213) bank195291 +195292 POINT(36.92847343004942 -122.64392423700512) bank195292 +195293 POINT(37.929494523093055 -122.90045196074915) bank195293 +195294 POINT(37.135954578481176 -121.61755093113325) bank195294 +195295 POINT(36.891980696004026 -121.78935625242438) bank195295 +195296 POINT(38.642071981271656 -123.41060666077031) bank195296 +195297 POINT(37.81983100738073 -123.22426503399737) bank195297 +195298 POINT(36.86778247246863 -122.47280560640536) bank195298 +195299 POINT(36.88305894659267 -122.51333960639953) bank195299 +195300 POINT(38.152446748042706 -122.85649456593323) bank195300 +195301 POINT(37.3256866486953 -123.26649017392549) bank195301 +195302 POINT(37.27894130997638 -121.72118296013728) bank195302 +195303 POINT(38.56573349556264 -122.17241801845451) bank195303 +195304 POINT(37.18756343362287 -122.35587672741082) bank195304 +195305 POINT(37.361776546481266 -122.54957900792027) bank195305 +195306 POINT(38.74585513673088 -121.91700677234164) bank195306 +195307 POINT(38.198158583772674 -121.65829324386965) bank195307 +195308 POINT(37.16610407254002 -122.71929491292094) bank195308 +195309 POINT(38.367573887892995 -122.03545068585615) bank195309 +195310 POINT(37.71762770472564 -122.14918829862444) bank195310 +195311 POINT(38.14898391612148 -122.13539986468805) bank195311 +195312 POINT(37.44707428742337 -121.81378721917172) bank195312 +195313 POINT(38.39543682414412 -122.49547172717206) bank195313 +195314 POINT(37.63986088142631 -123.41291082046651) bank195314 +195315 POINT(36.99067030290748 -121.94951030672375) bank195315 +195316 POINT(37.51589029239652 -122.58645596689382) bank195316 +195317 POINT(36.97539330188494 -122.3786534342846) bank195317 +195318 POINT(38.41386971889048 -122.68852890000599) bank195318 +195319 POINT(37.800101081935885 -121.92167921168142) bank195319 +195320 POINT(37.89499736630828 -122.7388689870628) bank195320 +195321 POINT(38.537257830157145 -123.28217613336648) bank195321 +195322 POINT(37.41716300990946 -122.42239556675041) bank195322 +195323 POINT(37.46395547890502 -122.76715808039107) bank195323 +195324 POINT(38.11368201679766 -121.46025606104959) bank195324 +195325 POINT(37.5905386451242 -121.67396440403199) bank195325 +195326 POINT(37.711842291614616 -123.29571791143628) bank195326 +195327 POINT(38.64226161657708 -121.99838980920636) bank195327 +195328 POINT(37.71344982607478 -121.90944754094284) bank195328 +195329 POINT(37.96058052167061 -121.90276098119418) bank195329 +195330 POINT(38.13667750658895 -123.14738056519312) bank195330 +195331 POINT(37.50536683060413 -122.49092947140468) bank195331 +195332 POINT(38.44388267026449 -121.72767319862048) bank195332 +195333 POINT(38.69638186623405 -122.38618143351667) bank195333 +195334 POINT(37.29676477846602 -122.99969206357609) bank195334 +195335 POINT(36.99120986955573 -123.25957548669372) bank195335 +195336 POINT(37.31943424588909 -122.78326615394217) bank195336 +195337 POINT(38.17043478143982 -122.57976022673006) bank195337 +195338 POINT(37.75894281583092 -122.91798954490635) bank195338 +195339 POINT(37.66105187456005 -121.62156576901866) bank195339 +195340 POINT(38.73055598107526 -122.48885768705546) bank195340 +195341 POINT(37.11380889507749 -122.74669574685402) bank195341 +195342 POINT(36.89549807711846 -122.50140408460796) bank195342 +195343 POINT(37.08277069960107 -122.16515556601846) bank195343 +195344 POINT(38.50002290722015 -123.12978855923983) bank195344 +195345 POINT(37.05064942734859 -122.61084287989003) bank195345 +195346 POINT(37.680564166829285 -121.71936186099404) bank195346 +195347 POINT(37.21084502101443 -122.69144289750619) bank195347 +195348 POINT(36.7982511387868 -121.63642215739353) bank195348 +195349 POINT(38.24545127558352 -123.17340484458295) bank195349 +195350 POINT(37.33268458569449 -122.02708136931663) bank195350 +195351 POINT(37.33029753068559 -122.16524151954337) bank195351 +195352 POINT(37.018182107636235 -122.58176948729341) bank195352 +195353 POINT(37.10622003666997 -122.67312292510236) bank195353 +195354 POINT(36.93478898836872 -123.03227922000866) bank195354 +195355 POINT(37.73540055773132 -121.80012073276856) bank195355 +195356 POINT(37.85763118477211 -121.76815547997735) bank195356 +195357 POINT(38.59349892038703 -123.22972014841504) bank195357 +195358 POINT(38.75776891277221 -121.45583834554735) bank195358 +195359 POINT(37.63389606776883 -122.53768488228648) bank195359 +195360 POINT(37.39891258287323 -122.73005561188555) bank195360 +195361 POINT(38.71512139855889 -122.64528731642159) bank195361 +195362 POINT(37.60391361243888 -121.75602840781681) bank195362 +195363 POINT(37.68439401346272 -121.4600006449392) bank195363 +195364 POINT(37.79738657828876 -121.46854360689882) bank195364 +195365 POINT(36.84691640538651 -123.15089946225069) bank195365 +195366 POINT(37.01738958644115 -122.28297450522302) bank195366 +195367 POINT(38.40924784391118 -123.22661909760848) bank195367 +195368 POINT(38.16316736503234 -121.54991810398207) bank195368 +195369 POINT(38.76316747398283 -122.38799807749902) bank195369 +195370 POINT(37.093136890345455 -122.45802251290984) bank195370 +195371 POINT(38.05110144221608 -122.04353464999987) bank195371 +195372 POINT(37.41713527547731 -122.32579687261585) bank195372 +195373 POINT(38.33809914271781 -121.4800694950588) bank195373 +195374 POINT(37.15490598743983 -122.55109061088417) bank195374 +195375 POINT(37.575652413353424 -122.900302278954) bank195375 +195376 POINT(38.20502373037267 -121.46494276222286) bank195376 +195377 POINT(38.05952370720967 -122.25168384963303) bank195377 +195378 POINT(38.65968949268174 -122.71362983197663) bank195378 +195379 POINT(38.06765874270163 -122.52141615514016) bank195379 +195380 POINT(37.28107544725383 -123.1387612527906) bank195380 +195381 POINT(37.88750632057346 -122.43428115167804) bank195381 +195382 POINT(38.199923964920764 -123.38962169360111) bank195382 +195383 POINT(37.97713752306239 -121.6650373240182) bank195383 +195384 POINT(37.89082835371026 -122.57391878877331) bank195384 +195385 POINT(38.76593499806658 -121.79732376194093) bank195385 +195386 POINT(38.72329314331468 -123.22414267069064) bank195386 +195387 POINT(38.54014456605632 -121.99835322718177) bank195387 +195388 POINT(37.319821123057935 -121.73429789766924) bank195388 +195389 POINT(38.00021097581987 -121.59119544002348) bank195389 +195390 POINT(38.14781864156134 -122.5707978585176) bank195390 +195391 POINT(38.39873716708853 -122.85565220693015) bank195391 +195392 POINT(38.05288449588339 -122.73035969870975) bank195392 +195393 POINT(38.58264866569287 -121.88493939754109) bank195393 +195394 POINT(36.86122695038672 -123.02808958045964) bank195394 +195395 POINT(38.74499431328433 -121.45291304457609) bank195395 +195396 POINT(37.04693753882677 -123.02840602759852) bank195396 +195397 POINT(37.30110295755082 -123.25941777335848) bank195397 +195398 POINT(37.479969567654 -122.87265460468797) bank195398 +195399 POINT(37.76854556796023 -122.8027394819751) bank195399 +195400 POINT(37.89506142477596 -121.8931386982929) bank195400 +195401 POINT(38.38410192179398 -123.15260995069931) bank195401 +195402 POINT(37.54724049481771 -122.42460541044439) bank195402 +195403 POINT(37.19481808066725 -121.95519942115558) bank195403 +195404 POINT(37.043665936318106 -122.51160382115921) bank195404 +195405 POINT(38.43692969916163 -122.59951964873561) bank195405 +195406 POINT(38.750178797968054 -123.39296738271636) bank195406 +195407 POINT(36.94855963110603 -122.50987808402458) bank195407 +195408 POINT(38.54042879947376 -123.01973339576836) bank195408 +195409 POINT(37.147470771899755 -122.41345430856124) bank195409 +195410 POINT(37.78098937512698 -121.59715730130024) bank195410 +195411 POINT(38.18123482235628 -121.77481891439523) bank195411 +195412 POINT(38.46431138769407 -123.37963848779951) bank195412 +195413 POINT(37.48442052314936 -122.22747859310483) bank195413 +195414 POINT(36.79757606339155 -123.12903158262701) bank195414 +195415 POINT(38.591754199895384 -122.98920954211705) bank195415 +195416 POINT(37.789657240760725 -121.74548387710736) bank195416 +195417 POINT(38.32478100867199 -121.95161309790146) bank195417 +195418 POINT(37.533333672962684 -122.47593280487855) bank195418 +195419 POINT(36.956624313605516 -122.64835900100502) bank195419 +195420 POINT(37.16237684398008 -121.79765435943543) bank195420 +195421 POINT(37.52793275664242 -122.59887854388593) bank195421 +195422 POINT(38.24950513799047 -122.61435962500092) bank195422 +195423 POINT(36.88406407484144 -121.84148348813459) bank195423 +195424 POINT(38.05501914695622 -122.34048912468417) bank195424 +195425 POINT(38.44909936847898 -123.29952628691211) bank195425 +195426 POINT(36.83676999749382 -122.77079614491855) bank195426 +195427 POINT(38.35379063502229 -121.59048370103979) bank195427 +195428 POINT(37.632969349823824 -122.63831927682784) bank195428 +195429 POINT(37.14246800245971 -122.49006624007892) bank195429 +195430 POINT(38.33780741469064 -121.62751072846709) bank195430 +195431 POINT(37.4511956657414 -122.66588836410104) bank195431 +195432 POINT(38.20389021559703 -121.49055178849683) bank195432 +195433 POINT(37.933925309010526 -121.43058744329781) bank195433 +195434 POINT(36.79827423774427 -123.26480660439627) bank195434 +195435 POINT(38.731188834179854 -122.24528315070835) bank195435 +195436 POINT(37.488321212206095 -123.02952790513056) bank195436 +195437 POINT(36.90030126492882 -122.71461730568119) bank195437 +195438 POINT(37.7151472337757 -122.21476484130346) bank195438 +195439 POINT(37.09196828528003 -121.48073154788327) bank195439 +195440 POINT(38.58616733065271 -123.10822461993548) bank195440 +195441 POINT(37.359450843917564 -122.84021593997213) bank195441 +195442 POINT(38.55717879158451 -121.50667544933728) bank195442 +195443 POINT(38.0138764259258 -121.77256411256002) bank195443 +195444 POINT(37.04143408769096 -121.75887241577934) bank195444 +195445 POINT(37.5874286424626 -122.63292042514689) bank195445 +195446 POINT(38.64308538405357 -121.87762995360127) bank195446 +195447 POINT(38.73117673004786 -122.74809214713571) bank195447 +195448 POINT(38.72792276162252 -121.59459366921165) bank195448 +195449 POINT(37.85712376426683 -122.34808359640917) bank195449 +195450 POINT(38.634824434132454 -122.89358124834686) bank195450 +195451 POINT(37.19940868356307 -121.9210401258418) bank195451 +195452 POINT(38.2983248301002 -121.74358859111054) bank195452 +195453 POINT(37.91704338314906 -121.73744810624072) bank195453 +195454 POINT(38.358320757857584 -122.82077164382002) bank195454 +195455 POINT(37.465386286420156 -121.77705357205213) bank195455 +195456 POINT(38.16691667328036 -122.97824109441301) bank195456 +195457 POINT(37.35844625147395 -121.99482536295255) bank195457 +195458 POINT(38.70204254981823 -123.19629081514712) bank195458 +195459 POINT(37.86018465990934 -122.15983271929372) bank195459 +195460 POINT(37.509739900687244 -121.72530995994902) bank195460 +195461 POINT(37.4279788850618 -122.00051706373002) bank195461 +195462 POINT(38.574889988016785 -122.01521641069047) bank195462 +195463 POINT(38.174741360654316 -122.45920275676933) bank195463 +195464 POINT(38.462570861149906 -122.37046608887832) bank195464 +195465 POINT(38.526060737665055 -121.63932018266101) bank195465 +195466 POINT(37.13465599118366 -122.15299829961612) bank195466 +195467 POINT(38.094458251397015 -121.7064789531208) bank195467 +195468 POINT(36.917840384814575 -122.08358994126134) bank195468 +195469 POINT(37.340825355439044 -121.81688588360016) bank195469 +195470 POINT(37.036079765295554 -122.78063031074998) bank195470 +195471 POINT(36.892730782528226 -122.31062660195421) bank195471 +195472 POINT(37.78125186097529 -122.43837785595015) bank195472 +195473 POINT(38.46313831310894 -121.68083589763175) bank195473 +195474 POINT(38.19089023098951 -123.37909234940093) bank195474 +195475 POINT(37.127805864527325 -123.03522168680846) bank195475 +195476 POINT(36.824802576784755 -122.50044971069694) bank195476 +195477 POINT(38.32166263279687 -123.2761420465225) bank195477 +195478 POINT(38.01514935192007 -121.85791137136387) bank195478 +195479 POINT(37.9395850807926 -121.66087568419863) bank195479 +195480 POINT(38.084262659451745 -122.3604991847879) bank195480 +195481 POINT(37.58060956653331 -121.44345930652433) bank195481 +195482 POINT(38.69904943268904 -123.03079892627424) bank195482 +195483 POINT(38.6984476647626 -123.19053971484459) bank195483 +195484 POINT(37.53931105517604 -121.52742691457065) bank195484 +195485 POINT(37.10776405979699 -122.3844330606908) bank195485 +195486 POINT(38.38147490167121 -123.18697902623391) bank195486 +195487 POINT(37.159433950537284 -121.69680538141681) bank195487 +195488 POINT(36.9649268477897 -122.54796212246376) bank195488 +195489 POINT(38.62922957443601 -121.77329199545365) bank195489 +195490 POINT(37.72690167195913 -123.14364682974904) bank195490 +195491 POINT(37.5479405068944 -122.78373200645923) bank195491 +195492 POINT(36.94547236879863 -122.71332129520192) bank195492 +195493 POINT(37.50560496294696 -122.48521645084219) bank195493 +195494 POINT(37.37834433089089 -122.15206047726537) bank195494 +195495 POINT(37.800324016517614 -123.05571411378685) bank195495 +195496 POINT(37.048664404453476 -122.46341041075868) bank195496 +195497 POINT(38.73171161022627 -123.08679613127711) bank195497 +195498 POINT(38.53865724418427 -121.86846858143099) bank195498 +195499 POINT(37.790718535592994 -122.24982093584929) bank195499 +195500 POINT(38.04860024983812 -123.04521592911517) bank195500 +195501 POINT(38.60323795673358 -122.08192509536117) bank195501 +195502 POINT(37.285185426873156 -122.58264483426501) bank195502 +195503 POINT(36.80610810051152 -121.49699203358215) bank195503 +195504 POINT(38.40349963924432 -123.29760732165722) bank195504 +195505 POINT(37.6871420812492 -122.22051485537429) bank195505 +195506 POINT(38.02382135631086 -121.8047850008932) bank195506 +195507 POINT(37.90728883447201 -123.17356510287003) bank195507 +195508 POINT(38.158660836841705 -122.48333912637523) bank195508 +195509 POINT(38.48365492281878 -122.07180325224203) bank195509 +195510 POINT(38.572564095095004 -123.13245375278906) bank195510 +195511 POINT(37.014390615754806 -122.24404591844977) bank195511 +195512 POINT(37.057198885512406 -121.53322999850849) bank195512 +195513 POINT(37.64420653412874 -122.89541273936821) bank195513 +195514 POINT(37.45070959657429 -122.25872167898653) bank195514 +195515 POINT(37.871749370613635 -122.60667176384825) bank195515 +195516 POINT(38.2959968567849 -122.24430325507691) bank195516 +195517 POINT(38.033879883561035 -121.4968051132336) bank195517 +195518 POINT(37.79266779006153 -123.1232921728403) bank195518 +195519 POINT(37.6180510237005 -122.7172932676711) bank195519 +195520 POINT(38.41026994870724 -123.41928579859787) bank195520 +195521 POINT(36.79063535940857 -122.97568550860626) bank195521 +195522 POINT(38.601581560257664 -122.90586837878207) bank195522 +195523 POINT(37.00080799288369 -122.63504862001177) bank195523 +195524 POINT(38.13973426918703 -123.00051634558588) bank195524 +195525 POINT(37.43210896519336 -122.16199264285733) bank195525 +195526 POINT(38.709374737721504 -122.94301139652808) bank195526 +195527 POINT(37.250712520357396 -121.72914300015398) bank195527 +195528 POINT(37.16645061524766 -122.62346934480259) bank195528 +195529 POINT(38.0108402184196 -122.75036999943956) bank195529 +195530 POINT(38.211688963175135 -122.24793829645571) bank195530 +195531 POINT(37.9572297010868 -121.57102813997494) bank195531 +195532 POINT(38.15811581747992 -121.60873204191827) bank195532 +195533 POINT(38.39795801559359 -122.03222197719566) bank195533 +195534 POINT(36.828542170442645 -122.3173601999794) bank195534 +195535 POINT(37.6444145283567 -122.90451308615593) bank195535 +195536 POINT(37.23271261570674 -121.6899610632876) bank195536 +195537 POINT(36.82849250115495 -122.12740293544556) bank195537 +195538 POINT(36.96277427548715 -123.35004055782167) bank195538 +195539 POINT(38.67571840465377 -123.32827188678704) bank195539 +195540 POINT(37.36673300954386 -122.82446446400822) bank195540 +195541 POINT(37.85627424489428 -122.67506677470831) bank195541 +195542 POINT(38.52683452516858 -122.11717667162954) bank195542 +195543 POINT(37.94562519029494 -121.8523650316846) bank195543 +195544 POINT(37.50688860899243 -122.97865582535348) bank195544 +195545 POINT(38.756161162628345 -121.75945653367165) bank195545 +195546 POINT(38.16155429413306 -121.6255633348815) bank195546 +195547 POINT(37.51343103772283 -121.8072280909265) bank195547 +195548 POINT(37.68351580178363 -122.34542147239704) bank195548 +195549 POINT(37.647895555374234 -123.1881015637528) bank195549 +195550 POINT(37.56412897070101 -121.5357151449703) bank195550 +195551 POINT(36.81266974460567 -121.82029859190516) bank195551 +195552 POINT(38.384540549487255 -122.25455084881652) bank195552 +195553 POINT(38.23328750785161 -122.1295491646624) bank195553 +195554 POINT(38.534800297833925 -123.36333437595268) bank195554 +195555 POINT(38.008038563137035 -122.92168458837412) bank195555 +195556 POINT(37.24380887528573 -121.91826935804033) bank195556 +195557 POINT(37.03106819803558 -122.48079698390379) bank195557 +195558 POINT(38.29728763688891 -122.23959171222563) bank195558 +195559 POINT(38.420648019262444 -122.02804162386748) bank195559 +195560 POINT(37.76300860780334 -121.51455165491922) bank195560 +195561 POINT(37.424734775975885 -121.95336608114933) bank195561 +195562 POINT(38.06306287212881 -123.30567180199277) bank195562 +195563 POINT(36.857107468541464 -122.33614334524485) bank195563 +195564 POINT(37.78821029454089 -123.15204375672958) bank195564 +195565 POINT(38.130544602274334 -121.83412303361224) bank195565 +195566 POINT(38.005572380147754 -122.3381406903021) bank195566 +195567 POINT(38.64839038758514 -123.06088277361857) bank195567 +195568 POINT(36.955651768137606 -122.44213603939308) bank195568 +195569 POINT(37.9298496466385 -123.08999067299378) bank195569 +195570 POINT(37.21631885184743 -123.36765479890092) bank195570 +195571 POINT(38.061925843521415 -122.48872181438573) bank195571 +195572 POINT(38.69417379452263 -121.92998171600031) bank195572 +195573 POINT(37.820151593505464 -122.69960938755858) bank195573 +195574 POINT(37.65574278386906 -121.72394834647298) bank195574 +195575 POINT(38.59394868832121 -122.2610180907929) bank195575 +195576 POINT(38.09451748472223 -122.72087762620765) bank195576 +195577 POINT(37.3951572090608 -122.26720306854253) bank195577 +195578 POINT(37.15918691858093 -123.1887985995519) bank195578 +195579 POINT(38.09419341923444 -123.1744969737939) bank195579 +195580 POINT(37.05262032670271 -122.15675476460125) bank195580 +195581 POINT(38.774404594781 -121.53714738416126) bank195581 +195582 POINT(37.73882717423721 -122.40014871891815) bank195582 +195583 POINT(38.275431577347184 -121.62840961478672) bank195583 +195584 POINT(38.46922705974515 -121.9956419255408) bank195584 +195585 POINT(38.00458501166163 -121.96243868228055) bank195585 +195586 POINT(38.721737020049304 -121.79994984224955) bank195586 +195587 POINT(37.36820312528852 -123.21915154572945) bank195587 +195588 POINT(37.12258943242227 -122.62157526730815) bank195588 +195589 POINT(37.163001300549844 -123.11897816564984) bank195589 +195590 POINT(37.734210189890774 -122.46452475812238) bank195590 +195591 POINT(37.98320487772237 -122.41694451904256) bank195591 +195592 POINT(37.3239561452474 -122.56475453896088) bank195592 +195593 POINT(38.540050189245754 -122.86613334842431) bank195593 +195594 POINT(38.7573388684398 -122.57410172001504) bank195594 +195595 POINT(38.466918600124295 -122.58369938855729) bank195595 +195596 POINT(38.769794582666634 -122.0207196757683) bank195596 +195597 POINT(38.50032365576158 -121.95689109970266) bank195597 +195598 POINT(37.328818557690816 -122.15007237864596) bank195598 +195599 POINT(38.0808315559643 -123.1175073430838) bank195599 +195600 POINT(38.05050607900748 -122.8699106840451) bank195600 +195601 POINT(36.81447535517196 -121.84581181749125) bank195601 +195602 POINT(36.82746528624841 -122.54446184321766) bank195602 +195603 POINT(38.64921921239438 -121.53037715509176) bank195603 +195604 POINT(38.6344244670141 -122.53480718580218) bank195604 +195605 POINT(38.66612143052189 -121.80540415764786) bank195605 +195606 POINT(38.073774432279855 -122.31059108123623) bank195606 +195607 POINT(38.67988901421294 -121.98599488533549) bank195607 +195608 POINT(37.26239071172194 -121.72254322675686) bank195608 +195609 POINT(37.03622547551495 -122.01381425677909) bank195609 +195610 POINT(37.89922673409723 -122.61346960294047) bank195610 +195611 POINT(38.024146129638986 -121.92320559060376) bank195611 +195612 POINT(37.069883808428145 -123.0546098678358) bank195612 +195613 POINT(36.91813491700465 -123.07689433179183) bank195613 +195614 POINT(37.32257668381663 -123.13280100813529) bank195614 +195615 POINT(37.87796373961131 -123.37830469843362) bank195615 +195616 POINT(37.528080898238194 -123.16384353470079) bank195616 +195617 POINT(38.65787158184892 -121.52279652029702) bank195617 +195618 POINT(37.35393223211944 -122.92702159573572) bank195618 +195619 POINT(37.33445528524069 -123.00146364106617) bank195619 +195620 POINT(38.76044703704704 -121.6039404638566) bank195620 +195621 POINT(38.33872440628344 -121.8325163684451) bank195621 +195622 POINT(37.48855451882919 -122.29612152950357) bank195622 +195623 POINT(38.50858940296422 -123.01338636318805) bank195623 +195624 POINT(38.2544291760199 -123.07066567446088) bank195624 +195625 POINT(37.4787493251544 -122.066426845888) bank195625 +195626 POINT(37.18911707327943 -123.10469816681176) bank195626 +195627 POINT(37.003089722604756 -122.47322930632754) bank195627 +195628 POINT(36.88020403371239 -121.84313751386699) bank195628 +195629 POINT(37.680918169043686 -121.67966249924626) bank195629 +195630 POINT(37.65499326572167 -121.88987475259042) bank195630 +195631 POINT(38.02194105347261 -122.28441671534776) bank195631 +195632 POINT(37.721682275798074 -121.43548835502175) bank195632 +195633 POINT(38.190936387469435 -121.85001979113106) bank195633 +195634 POINT(37.63854009605473 -122.98139685990468) bank195634 +195635 POINT(38.55192251260339 -122.73182058773197) bank195635 +195636 POINT(38.50695042128671 -123.31397916817608) bank195636 +195637 POINT(37.176545964301106 -122.20836734473244) bank195637 +195638 POINT(38.673191041599054 -122.54014840089796) bank195638 +195639 POINT(38.06444337921558 -121.9568717893838) bank195639 +195640 POINT(37.65318986865521 -123.27396450359632) bank195640 +195641 POINT(38.53484889214455 -122.20285205558125) bank195641 +195642 POINT(37.859191391613116 -121.70100885940472) bank195642 +195643 POINT(37.23410897813076 -123.35106972804262) bank195643 +195644 POINT(37.356357810413556 -122.67469085297687) bank195644 +195645 POINT(37.75829063313004 -122.66982432513808) bank195645 +195646 POINT(36.83852739559277 -123.15505181962833) bank195646 +195647 POINT(38.480642802166216 -122.54892715652444) bank195647 +195648 POINT(37.86709726647697 -123.28124014530167) bank195648 +195649 POINT(37.604193205570205 -123.21096375090168) bank195649 +195650 POINT(38.36473150423763 -121.62795090788651) bank195650 +195651 POINT(38.18895261586548 -122.18332328635589) bank195651 +195652 POINT(37.5004707385187 -122.62371658730247) bank195652 +195653 POINT(37.80981158691987 -122.04703381856557) bank195653 +195654 POINT(37.29955547560606 -123.16025818803827) bank195654 +195655 POINT(37.86531461078401 -121.58224685001741) bank195655 +195656 POINT(37.267403004011996 -121.7816997721185) bank195656 +195657 POINT(36.987275110993046 -121.98061795333561) bank195657 +195658 POINT(38.14038194922164 -123.07251870092527) bank195658 +195659 POINT(38.646214103466626 -122.96203193898944) bank195659 +195660 POINT(37.27913402777757 -122.90136780414306) bank195660 +195661 POINT(36.79361190413624 -122.54974665066263) bank195661 +195662 POINT(36.81376427240019 -123.17564676095424) bank195662 +195663 POINT(37.74853931287872 -123.00985853535448) bank195663 +195664 POINT(37.81494881881164 -122.47745602005504) bank195664 +195665 POINT(37.14817272339936 -123.03479896327914) bank195665 +195666 POINT(36.91978891913235 -122.587646148164) bank195666 +195667 POINT(37.54537860009816 -122.3625202385424) bank195667 +195668 POINT(38.45788000148627 -121.91856493631316) bank195668 +195669 POINT(37.09404993539976 -121.91129830834981) bank195669 +195670 POINT(38.49564552821907 -122.30006819416137) bank195670 +195671 POINT(36.8124493823001 -122.34106935217213) bank195671 +195672 POINT(38.23450938732681 -121.79187296886286) bank195672 +195673 POINT(38.3250329411069 -122.53480656061986) bank195673 +195674 POINT(38.389326500636734 -121.9579850301825) bank195674 +195675 POINT(36.93350745676488 -123.00902196488232) bank195675 +195676 POINT(36.88617492402068 -123.26218474880052) bank195676 +195677 POINT(38.725650000239646 -123.33519764587814) bank195677 +195678 POINT(38.66542646347315 -123.33244904726608) bank195678 +195679 POINT(36.86870826676097 -121.45769838903882) bank195679 +195680 POINT(36.876228219863755 -123.07383111808095) bank195680 +195681 POINT(37.62265576764839 -123.3106446790026) bank195681 +195682 POINT(37.82330150845553 -123.0877896021223) bank195682 +195683 POINT(37.161994201063195 -121.8332341793013) bank195683 +195684 POINT(38.19263793872249 -121.81930350883282) bank195684 +195685 POINT(37.683836547938796 -122.95020243886397) bank195685 +195686 POINT(38.00135563108755 -121.75449702841779) bank195686 +195687 POINT(38.36791095631503 -122.80535052285813) bank195687 +195688 POINT(38.30436276487588 -122.43191055271541) bank195688 +195689 POINT(36.862786225961635 -121.56334206164283) bank195689 +195690 POINT(38.61867847035186 -123.36040858946662) bank195690 +195691 POINT(36.826659342169464 -122.02780060545358) bank195691 +195692 POINT(38.194023437103 -122.03334109127616) bank195692 +195693 POINT(38.71342593763991 -122.80396466263635) bank195693 +195694 POINT(36.947723187290535 -123.38245707887451) bank195694 +195695 POINT(38.06648807274911 -122.34869132070182) bank195695 +195696 POINT(36.98737156656618 -122.23109141901459) bank195696 +195697 POINT(37.97323341535755 -122.9280928863627) bank195697 +195698 POINT(36.77755045464117 -123.1193948725396) bank195698 +195699 POINT(37.89428830043289 -123.19113210465855) bank195699 +195700 POINT(37.300184607910886 -121.9450007265268) bank195700 +195701 POINT(37.93924132570957 -122.1989560694578) bank195701 +195702 POINT(37.82293519281738 -121.6602582912551) bank195702 +195703 POINT(37.884526183150186 -122.79312373903231) bank195703 +195704 POINT(37.75790347206111 -122.8751002375386) bank195704 +195705 POINT(36.8406442397404 -122.8386697353325) bank195705 +195706 POINT(37.84478302801417 -121.62231175658043) bank195706 +195707 POINT(38.69812509962535 -122.98105380637145) bank195707 +195708 POINT(37.077340908178506 -123.38614153727258) bank195708 +195709 POINT(37.36236843617501 -121.6053801382392) bank195709 +195710 POINT(37.89385297018933 -122.4652160392404) bank195710 +195711 POINT(37.58807544574613 -122.45570033786596) bank195711 +195712 POINT(38.292467952513505 -122.43231780806407) bank195712 +195713 POINT(37.42620580196573 -123.00848298128459) bank195713 +195714 POINT(37.198886442791164 -122.91946188205847) bank195714 +195715 POINT(38.53097719889643 -121.88711522380483) bank195715 +195716 POINT(38.4415556930886 -122.88726229704915) bank195716 +195717 POINT(37.423225492665026 -123.27174975064229) bank195717 +195718 POINT(36.796950395411415 -121.87681747330323) bank195718 +195719 POINT(38.334352401055284 -122.63752288560201) bank195719 +195720 POINT(38.45479288029577 -121.73541721081767) bank195720 +195721 POINT(37.537184798974124 -122.1995409742993) bank195721 +195722 POINT(37.465748510527256 -122.49809905313707) bank195722 +195723 POINT(36.91203006547154 -122.01440564574166) bank195723 +195724 POINT(36.79542493351199 -122.35065159251035) bank195724 +195725 POINT(37.40277069062047 -123.14565524334324) bank195725 +195726 POINT(37.44351857532335 -123.02963110370526) bank195726 +195727 POINT(37.19932219893289 -121.94074684493076) bank195727 +195728 POINT(38.440155408791504 -122.68833865547646) bank195728 +195729 POINT(37.59834886217331 -122.74005874713082) bank195729 +195730 POINT(36.84879588216195 -123.25975862340944) bank195730 +195731 POINT(37.545180638417875 -122.0889517588061) bank195731 +195732 POINT(38.22362049224687 -122.37525689962982) bank195732 +195733 POINT(36.93023547323639 -122.61037180589257) bank195733 +195734 POINT(37.55271574980338 -122.60479302187035) bank195734 +195735 POINT(38.092071162245745 -122.84492245856424) bank195735 +195736 POINT(38.09067897034766 -122.44268047508017) bank195736 +195737 POINT(38.49309991515966 -121.6058790230795) bank195737 +195738 POINT(37.28644555792854 -122.07760938584856) bank195738 +195739 POINT(38.151961989369894 -121.58608310434332) bank195739 +195740 POINT(37.21995086305713 -121.59373471105286) bank195740 +195741 POINT(38.18947822549105 -121.44789728231287) bank195741 +195742 POINT(37.84893102294237 -122.32437745633331) bank195742 +195743 POINT(36.85748214012946 -122.0348920951985) bank195743 +195744 POINT(37.713683840296284 -122.14199527765494) bank195744 +195745 POINT(38.17874423589514 -122.57986734338915) bank195745 +195746 POINT(37.381034532745524 -122.42019439147441) bank195746 +195747 POINT(37.02222565895284 -123.3166299397669) bank195747 +195748 POINT(38.423272233368564 -122.89629778595422) bank195748 +195749 POINT(37.053009860383646 -121.94156884696844) bank195749 +195750 POINT(38.76966831296188 -121.65013149971968) bank195750 +195751 POINT(37.2472177164339 -122.71345994199031) bank195751 +195752 POINT(37.15058509199714 -122.81026850101932) bank195752 +195753 POINT(38.36654407040741 -122.34502004931493) bank195753 +195754 POINT(38.32122665279787 -121.42573506756806) bank195754 +195755 POINT(38.04857164188958 -123.21280093283943) bank195755 +195756 POINT(37.29179495688207 -122.79640027641526) bank195756 +195757 POINT(37.99993633310008 -122.5895181708047) bank195757 +195758 POINT(38.623074024565994 -122.00345984920091) bank195758 +195759 POINT(37.79935729870768 -121.94393667481758) bank195759 +195760 POINT(38.73345124651184 -121.85616748153856) bank195760 +195761 POINT(36.871742494753086 -122.2184413269276) bank195761 +195762 POINT(38.18626344572599 -121.69505365271301) bank195762 +195763 POINT(37.62893628634233 -123.18313911687241) bank195763 +195764 POINT(38.306357790885244 -122.67220219026996) bank195764 +195765 POINT(38.10960396193312 -122.35754823332368) bank195765 +195766 POINT(37.91928425133427 -121.44747294897685) bank195766 +195767 POINT(37.94565164309572 -122.56055873048555) bank195767 +195768 POINT(38.76046760962953 -123.18021091170363) bank195768 +195769 POINT(37.40323276863405 -123.25396767268079) bank195769 +195770 POINT(37.44787793155916 -121.95254013110556) bank195770 +195771 POINT(37.58461594092199 -123.2317586682819) bank195771 +195772 POINT(37.31676852215969 -122.01920827069205) bank195772 +195773 POINT(36.94984623432382 -122.48485376185984) bank195773 +195774 POINT(38.16829649738537 -121.63913249145965) bank195774 +195775 POINT(37.44664367284692 -122.34391097265807) bank195775 +195776 POINT(38.77349258721749 -122.09237886888914) bank195776 +195777 POINT(37.34561555699966 -122.83529641986506) bank195777 +195778 POINT(38.401614655823 -123.05144439162792) bank195778 +195779 POINT(37.25361390636503 -122.28713739769738) bank195779 +195780 POINT(37.88459433416392 -121.4759745577059) bank195780 +195781 POINT(36.892696120866546 -122.42329016113084) bank195781 +195782 POINT(38.06359877571837 -121.82699850850312) bank195782 +195783 POINT(36.778006044744906 -121.85188236209848) bank195783 +195784 POINT(36.84491897220381 -122.38028975806591) bank195784 +195785 POINT(37.74782419178992 -122.52704984928575) bank195785 +195786 POINT(38.51165259029726 -121.51252285294309) bank195786 +195787 POINT(37.796430549812705 -122.05591833243464) bank195787 +195788 POINT(38.283982856239874 -121.95693516249857) bank195788 +195789 POINT(38.52723336452889 -122.83489488182421) bank195789 +195790 POINT(37.461172942297765 -122.38884515579763) bank195790 +195791 POINT(37.279215961559416 -123.32100638009584) bank195791 +195792 POINT(37.012369109351205 -122.33766545656243) bank195792 +195793 POINT(37.77842942371439 -121.63633372070309) bank195793 +195794 POINT(37.05301149662171 -122.08974671620321) bank195794 +195795 POINT(36.79924187541817 -121.81004725348316) bank195795 +195796 POINT(36.871607993281756 -122.6656517945612) bank195796 +195797 POINT(38.49801746880607 -122.57235134386643) bank195797 +195798 POINT(37.2544671302956 -122.02735674822306) bank195798 +195799 POINT(38.351289942858116 -122.24728047319826) bank195799 +195800 POINT(37.11436628299689 -121.69321159558004) bank195800 +195801 POINT(37.07972575746929 -121.74892239733589) bank195801 +195802 POINT(37.382915634048864 -121.5164742968437) bank195802 +195803 POINT(37.480680253141536 -122.80076822802738) bank195803 +195804 POINT(37.91263590889577 -121.80815543048845) bank195804 +195805 POINT(38.49933367384776 -122.69321573724102) bank195805 +195806 POINT(37.17600313977108 -123.38954706669975) bank195806 +195807 POINT(38.10203779642508 -121.90341518609752) bank195807 +195808 POINT(37.35939986869572 -122.80069983678018) bank195808 +195809 POINT(36.825098116943394 -123.05394533796795) bank195809 +195810 POINT(37.125081403669455 -121.96509545296195) bank195810 +195811 POINT(37.19841674614769 -123.30538212215544) bank195811 +195812 POINT(37.05494641248653 -122.0945818634699) bank195812 +195813 POINT(37.689861044287625 -123.39535919332691) bank195813 +195814 POINT(37.344237374283345 -123.16326880452269) bank195814 +195815 POINT(37.45253185231099 -122.29411272218755) bank195815 +195816 POINT(38.380236060819776 -123.07869084911098) bank195816 +195817 POINT(38.742657654911724 -123.29776493025551) bank195817 +195818 POINT(36.89954771873968 -122.49473649410271) bank195818 +195819 POINT(36.80224899921351 -122.57552518610035) bank195819 +195820 POINT(37.162324759230515 -122.82806048541241) bank195820 +195821 POINT(37.06938579821228 -123.4066117421309) bank195821 +195822 POINT(37.30302036444792 -121.43926826339599) bank195822 +195823 POINT(37.62929121260434 -123.18496679299147) bank195823 +195824 POINT(37.05349977316613 -122.41675120187126) bank195824 +195825 POINT(37.71706505615312 -121.74395310121388) bank195825 +195826 POINT(37.698954919714794 -122.4829354951464) bank195826 +195827 POINT(37.04866305613616 -121.56432479653539) bank195827 +195828 POINT(38.506306453113496 -123.20807903476577) bank195828 +195829 POINT(38.56980262443683 -122.41165094282918) bank195829 +195830 POINT(38.68408209959813 -123.32482466543557) bank195830 +195831 POINT(38.18802374534684 -121.90008822913215) bank195831 +195832 POINT(38.41842383036967 -123.14427194186929) bank195832 +195833 POINT(37.3887510934351 -123.3872428076998) bank195833 +195834 POINT(37.27059866129283 -121.64733295685652) bank195834 +195835 POINT(37.4252991150135 -123.03643297808092) bank195835 +195836 POINT(38.33521738374169 -123.24438204611295) bank195836 +195837 POINT(37.06012792891477 -123.18386840321836) bank195837 +195838 POINT(37.79942760095006 -123.12736191794112) bank195838 +195839 POINT(38.396578698123506 -121.88603866500652) bank195839 +195840 POINT(36.98701095840595 -122.5115876423656) bank195840 +195841 POINT(38.723515990936654 -121.94762258850098) bank195841 +195842 POINT(37.974828491552344 -122.33925129415503) bank195842 +195843 POINT(38.323837106745714 -122.78957418869173) bank195843 +195844 POINT(36.81577330720596 -121.6820594740459) bank195844 +195845 POINT(37.98759218177894 -122.93261473360519) bank195845 +195846 POINT(38.538681122095994 -121.74956324063086) bank195846 +195847 POINT(37.81138317003476 -122.04251826886748) bank195847 +195848 POINT(37.40057463110391 -123.03936462470429) bank195848 +195849 POINT(38.39279199627678 -123.35664478074023) bank195849 +195850 POINT(37.82943311192033 -121.64547797516666) bank195850 +195851 POINT(37.95801936954662 -122.84283019805629) bank195851 +195852 POINT(36.890737641602705 -123.41346434811476) bank195852 +195853 POINT(37.6554412468792 -122.17713514573165) bank195853 +195854 POINT(37.04554845657635 -123.07228352317102) bank195854 +195855 POINT(38.09700709063875 -121.83665302691551) bank195855 +195856 POINT(38.16290749805509 -122.59213208373299) bank195856 +195857 POINT(36.88645479473392 -122.67372852764966) bank195857 +195858 POINT(37.637275603940616 -122.86795345979579) bank195858 +195859 POINT(38.30616674443595 -122.21290014741002) bank195859 +195860 POINT(38.56838987437023 -122.9615289283018) bank195860 +195861 POINT(38.19543024591636 -123.37441712391305) bank195861 +195862 POINT(38.15969124488587 -121.6648220192075) bank195862 +195863 POINT(37.57643652843642 -121.96747629157312) bank195863 +195864 POINT(38.591306643136 -122.66388434190979) bank195864 +195865 POINT(37.07154989087696 -122.64174380822507) bank195865 +195866 POINT(38.323731970651906 -122.36042550536385) bank195866 +195867 POINT(37.75800054614033 -121.49086217776635) bank195867 +195868 POINT(38.463101089525495 -121.96215643089903) bank195868 +195869 POINT(37.67491156505782 -123.38594798072411) bank195869 +195870 POINT(38.44796516036018 -123.2478713952867) bank195870 +195871 POINT(38.454393323037735 -121.61099196716441) bank195871 +195872 POINT(38.26311514081769 -122.21183463667064) bank195872 +195873 POINT(37.004603040305504 -121.63368055288564) bank195873 +195874 POINT(37.08278655494339 -123.39499383588738) bank195874 +195875 POINT(37.09599492707685 -122.30341011681978) bank195875 +195876 POINT(36.82230552868215 -123.37077897167441) bank195876 +195877 POINT(37.4596025809379 -121.9863354238996) bank195877 +195878 POINT(38.427098716800984 -121.93361522850658) bank195878 +195879 POINT(37.90157521955004 -122.40092082330679) bank195879 +195880 POINT(38.76866471135547 -122.1687701541611) bank195880 +195881 POINT(37.37621878262487 -123.14983939777825) bank195881 +195882 POINT(38.242543666936335 -121.74755983538674) bank195882 +195883 POINT(38.64351515178992 -122.47369027657453) bank195883 +195884 POINT(37.0464418054253 -122.18935674032069) bank195884 +195885 POINT(37.86282960350734 -122.85876431163717) bank195885 +195886 POINT(38.478127271616415 -122.15130623724797) bank195886 +195887 POINT(37.46466771457477 -123.2293166567174) bank195887 +195888 POINT(37.080235758044026 -123.29377938345715) bank195888 +195889 POINT(38.61013093787361 -122.58632016032227) bank195889 +195890 POINT(36.80139539795263 -123.4024698771522) bank195890 +195891 POINT(37.26069765033965 -122.0643206287037) bank195891 +195892 POINT(37.45439383012314 -121.60501154249185) bank195892 +195893 POINT(38.40672995997311 -122.52940338847455) bank195893 +195894 POINT(37.26750675340463 -123.31713856623466) bank195894 +195895 POINT(38.69222737961199 -122.63139749058433) bank195895 +195896 POINT(37.429036778356455 -122.11268560316051) bank195896 +195897 POINT(38.73470980480725 -122.8194394248768) bank195897 +195898 POINT(37.16167873427342 -121.5986856057652) bank195898 +195899 POINT(36.9502918985027 -123.25410569979225) bank195899 +195900 POINT(37.564274856134915 -123.37155857960835) bank195900 +195901 POINT(36.986429281008796 -122.59081766289411) bank195901 +195902 POINT(38.60421752934987 -123.06612197401837) bank195902 +195903 POINT(38.24070313233035 -122.12294045018474) bank195903 +195904 POINT(38.21246686556699 -121.80088140820803) bank195904 +195905 POINT(36.894733142381895 -121.79393752785239) bank195905 +195906 POINT(37.14144725045823 -121.9890729430406) bank195906 +195907 POINT(37.478764033656844 -122.84231659030335) bank195907 +195908 POINT(38.70610378649812 -122.6147957616639) bank195908 +195909 POINT(37.18295601979201 -122.25641344781386) bank195909 +195910 POINT(37.20187446012214 -122.92475453370398) bank195910 +195911 POINT(36.827802618566515 -122.67559691265791) bank195911 +195912 POINT(38.235662445100076 -123.00369731534333) bank195912 +195913 POINT(38.196847421557045 -122.8897005642286) bank195913 +195914 POINT(37.275845284654274 -122.11898382785282) bank195914 +195915 POINT(38.539055360613354 -122.60830251052501) bank195915 +195916 POINT(37.375308400193965 -122.98313201974008) bank195916 +195917 POINT(38.380420713703465 -122.36377528165518) bank195917 +195918 POINT(37.13617393434634 -123.2228853630333) bank195918 +195919 POINT(37.39635636905435 -121.49736845014895) bank195919 +195920 POINT(38.04914556625881 -122.65022513383045) bank195920 +195921 POINT(38.001731341863206 -122.4692804184288) bank195921 +195922 POINT(37.52839410228454 -123.136992625248) bank195922 +195923 POINT(38.58927260862517 -123.29698236836911) bank195923 +195924 POINT(36.932134757179085 -121.64612904630843) bank195924 +195925 POINT(37.61604717910489 -122.11298202982178) bank195925 +195926 POINT(38.59361013826508 -121.80280583809704) bank195926 +195927 POINT(37.15652657782898 -122.81401907349596) bank195927 +195928 POINT(38.43755958960599 -121.51175316601832) bank195928 +195929 POINT(37.504878473210695 -123.16521269168561) bank195929 +195930 POINT(37.12485376267567 -122.74028561423634) bank195930 +195931 POINT(37.38071313861437 -122.78952696398312) bank195931 +195932 POINT(36.91498100557862 -122.84616801945558) bank195932 +195933 POINT(38.22133186016457 -123.41077304791523) bank195933 +195934 POINT(38.15701918292131 -122.36862137248248) bank195934 +195935 POINT(37.366737252061256 -122.58675327354979) bank195935 +195936 POINT(36.94048248444508 -123.20458455010348) bank195936 +195937 POINT(37.65594109803286 -122.76341116133754) bank195937 +195938 POINT(38.000305079331156 -123.07103051216036) bank195938 +195939 POINT(37.06011382789831 -123.13218929234684) bank195939 +195940 POINT(37.62684364287089 -121.47691501228802) bank195940 +195941 POINT(37.763685695303984 -122.58138771433349) bank195941 +195942 POINT(38.08411798999793 -121.87761001749001) bank195942 +195943 POINT(37.65541875206491 -122.14011044367811) bank195943 +195944 POINT(38.304923529012946 -123.33996311469052) bank195944 +195945 POINT(38.57332687376412 -121.51301743690048) bank195945 +195946 POINT(37.71518862988263 -123.41284077167566) bank195946 +195947 POINT(38.714382020044354 -121.66943529110262) bank195947 +195948 POINT(37.49284543649131 -123.02594280920691) bank195948 +195949 POINT(36.7890181542025 -121.85774045736132) bank195949 +195950 POINT(37.67102460848306 -123.07720864108624) bank195950 +195951 POINT(38.50218895216493 -122.85661946402077) bank195951 +195952 POINT(37.011835189669995 -122.0793764295381) bank195952 +195953 POINT(38.410881231691754 -122.69763135762547) bank195953 +195954 POINT(36.878421865656655 -122.86534491487146) bank195954 +195955 POINT(36.90147206849898 -121.48251666295354) bank195955 +195956 POINT(38.388499601585195 -121.83107822397604) bank195956 +195957 POINT(37.19852835149371 -122.64349206971507) bank195957 +195958 POINT(38.29879288330573 -121.9014690600704) bank195958 +195959 POINT(37.854739657443766 -122.67563547353029) bank195959 +195960 POINT(38.12260958052087 -123.29713652297357) bank195960 +195961 POINT(38.45432658145321 -123.21008863253209) bank195961 +195962 POINT(38.645231756837056 -122.4909009375562) bank195962 +195963 POINT(37.38920765562406 -121.44844165427305) bank195963 +195964 POINT(38.52637273333461 -122.38035678585577) bank195964 +195965 POINT(38.244222869733555 -121.73462865130455) bank195965 +195966 POINT(37.70549786410545 -122.34098171860451) bank195966 +195967 POINT(37.98056237100854 -123.36187238584682) bank195967 +195968 POINT(38.75290841529005 -121.92367843245438) bank195968 +195969 POINT(37.7035503250359 -122.02521305207225) bank195969 +195970 POINT(37.02455119183133 -122.22916642179979) bank195970 +195971 POINT(38.297257150714294 -121.97761982595702) bank195971 +195972 POINT(38.11969883631948 -122.74117962199946) bank195972 +195973 POINT(37.70463332671227 -122.32951306793645) bank195973 +195974 POINT(38.33063084938695 -123.14587051143393) bank195974 +195975 POINT(36.84859466360121 -122.19111014045035) bank195975 +195976 POINT(38.11149831293012 -122.88408464402444) bank195976 +195977 POINT(37.09050375328051 -122.41296180535109) bank195977 +195978 POINT(37.00527778270358 -122.01018490013271) bank195978 +195979 POINT(37.32897911976714 -122.29130732915569) bank195979 +195980 POINT(38.26300225644375 -121.87467169001786) bank195980 +195981 POINT(38.53164634351931 -121.53842132815696) bank195981 +195982 POINT(38.65755674914017 -122.80991795684626) bank195982 +195983 POINT(38.19114865836012 -122.38841269919732) bank195983 +195984 POINT(37.01234503487944 -122.4554731484718) bank195984 +195985 POINT(37.48758764202347 -121.87311550162562) bank195985 +195986 POINT(38.55087554268586 -122.44172675429631) bank195986 +195987 POINT(37.37926494972402 -122.60961400925845) bank195987 +195988 POINT(38.39233643265179 -123.28581195469113) bank195988 +195989 POINT(38.10647499333426 -121.43334084555771) bank195989 +195990 POINT(37.04723121963066 -122.44684620094651) bank195990 +195991 POINT(38.51538287690236 -121.74169861409973) bank195991 +195992 POINT(37.68729779036488 -122.94034105113683) bank195992 +195993 POINT(37.981187830965624 -122.75706690439117) bank195993 +195994 POINT(37.65004502031397 -122.06370423243833) bank195994 +195995 POINT(38.45752751918218 -121.53170476740176) bank195995 +195996 POINT(37.31369588785675 -121.48736680360254) bank195996 +195997 POINT(36.840031751002414 -122.76794380315495) bank195997 +195998 POINT(37.654983017822744 -122.13822631023393) bank195998 +195999 POINT(38.53204425911137 -122.64732890225783) bank195999 +196000 POINT(37.184403769056765 -121.78109418939086) bank196000 +196001 POINT(38.774338200984985 -121.78908370596994) bank196001 +196002 POINT(38.02838297439292 -122.04245123098194) bank196002 +196003 POINT(38.23262801731882 -121.65797183689817) bank196003 +196004 POINT(37.406432563171506 -123.25809499720842) bank196004 +196005 POINT(37.133082248178326 -122.79690521434128) bank196005 +196006 POINT(37.97292226191279 -122.06522383894315) bank196006 +196007 POINT(38.42359588365429 -122.3749111791227) bank196007 +196008 POINT(37.2367350234299 -121.87342363987834) bank196008 +196009 POINT(38.22758779155961 -123.33133530830197) bank196009 +196010 POINT(37.22814451370347 -122.80545571075905) bank196010 +196011 POINT(37.40378331913436 -121.85141036560628) bank196011 +196012 POINT(36.969800918125514 -122.99800928632) bank196012 +196013 POINT(38.16750972739404 -121.89163433615953) bank196013 +196014 POINT(37.10951243643078 -122.76267878788602) bank196014 +196015 POINT(38.66029561536939 -123.24746648941085) bank196015 +196016 POINT(36.96705396126772 -122.39856445587145) bank196016 +196017 POINT(37.21375961754843 -121.91180540824874) bank196017 +196018 POINT(36.838164343435224 -122.45426843300687) bank196018 +196019 POINT(36.86881458608806 -122.84848860207813) bank196019 +196020 POINT(38.03910813184065 -122.4352082019116) bank196020 +196021 POINT(36.82356769202494 -121.94835084719028) bank196021 +196022 POINT(37.64976675375417 -121.57014670466852) bank196022 +196023 POINT(37.63504409646819 -122.50192086822378) bank196023 +196024 POINT(37.9794385833374 -122.07690824055284) bank196024 +196025 POINT(37.96267385357799 -122.39611486822115) bank196025 +196026 POINT(38.0672581592314 -121.86896143649567) bank196026 +196027 POINT(37.95004552817188 -123.2916523608468) bank196027 +196028 POINT(38.4298255529113 -123.1948535802058) bank196028 +196029 POINT(37.876498968613134 -123.038208968793) bank196029 +196030 POINT(37.54400051207091 -121.67407964969736) bank196030 +196031 POINT(38.60784764636974 -122.63195893736925) bank196031 +196032 POINT(37.026626196084656 -121.5573964145996) bank196032 +196033 POINT(37.1548846521001 -123.14896389373592) bank196033 +196034 POINT(37.95564008330554 -123.09410103713407) bank196034 +196035 POINT(37.77953221715318 -121.55467932871822) bank196035 +196036 POINT(37.02168417369422 -122.19820612570494) bank196036 +196037 POINT(38.53499465183413 -122.490283670886) bank196037 +196038 POINT(38.72108190435914 -122.67828211462167) bank196038 +196039 POINT(37.49457991441965 -122.13350660085624) bank196039 +196040 POINT(37.16355590711796 -121.82601896887199) bank196040 +196041 POINT(37.52066350018438 -122.42793791492466) bank196041 +196042 POINT(36.982586568295346 -123.29207126756864) bank196042 +196043 POINT(37.21355925106206 -121.6660108622403) bank196043 +196044 POINT(37.213098342844795 -122.89227415757722) bank196044 +196045 POINT(37.773655710312546 -122.42409571923476) bank196045 +196046 POINT(38.6011723569038 -121.7748011537179) bank196046 +196047 POINT(38.10324186976747 -121.5095533729977) bank196047 +196048 POINT(38.29086497717623 -122.66296252423857) bank196048 +196049 POINT(37.82270756958726 -123.40976897085783) bank196049 +196050 POINT(38.60446842750652 -122.34091060096706) bank196050 +196051 POINT(38.040658231411676 -123.21878775418062) bank196051 +196052 POINT(38.46966231923825 -122.57851056778412) bank196052 +196053 POINT(38.15723754614006 -122.1818529881154) bank196053 +196054 POINT(38.37092342958628 -121.48572552938666) bank196054 +196055 POINT(38.10664730764209 -122.70595948774108) bank196055 +196056 POINT(37.14084002313552 -122.75228477389896) bank196056 +196057 POINT(37.118338794832255 -122.27880580993035) bank196057 +196058 POINT(37.76682329539885 -122.8254520897397) bank196058 +196059 POINT(37.413022645393575 -121.86716382275833) bank196059 +196060 POINT(38.10120336216081 -122.97516102952665) bank196060 +196061 POINT(37.89934243450173 -121.99606180238955) bank196061 +196062 POINT(38.39069746914758 -122.65993223028279) bank196062 +196063 POINT(37.41585318425949 -122.543266068304) bank196063 +196064 POINT(38.5508094690752 -122.427693095369) bank196064 +196065 POINT(37.288462095128786 -122.62136724661178) bank196065 +196066 POINT(37.21070888781287 -123.37464301454315) bank196066 +196067 POINT(38.73395731755091 -123.039895514734) bank196067 +196068 POINT(37.78596197657223 -122.59400646253758) bank196068 +196069 POINT(37.71352489916507 -122.76401272550011) bank196069 +196070 POINT(38.706726046806516 -122.094019807327) bank196070 +196071 POINT(38.299633255615916 -122.46118170522963) bank196071 +196072 POINT(37.32001136514425 -121.7675633802515) bank196072 +196073 POINT(38.34321541640215 -121.89961656249478) bank196073 +196074 POINT(37.29734703611163 -122.51288182702177) bank196074 +196075 POINT(36.8951012733834 -121.77985152329654) bank196075 +196076 POINT(38.154726934867256 -122.22799608926675) bank196076 +196077 POINT(37.88177656866073 -122.51160395853178) bank196077 +196078 POINT(37.087894317050164 -122.95194011175248) bank196078 +196079 POINT(38.33613301345861 -121.53431432442841) bank196079 +196080 POINT(38.07834332882181 -122.31696787610447) bank196080 +196081 POINT(37.68402533215892 -121.80954566114077) bank196081 +196082 POINT(37.81462791731413 -122.75507216016427) bank196082 +196083 POINT(36.82347618048904 -123.24053659063402) bank196083 +196084 POINT(38.32828484430926 -122.24078360959206) bank196084 +196085 POINT(37.91085062421864 -122.61951767152462) bank196085 +196086 POINT(37.92309107539989 -122.69305464935023) bank196086 +196087 POINT(38.13645732354363 -121.4888406724065) bank196087 +196088 POINT(37.54819142198486 -122.79888466383873) bank196088 +196089 POINT(37.35256341688386 -123.16331632320022) bank196089 +196090 POINT(38.49196651635398 -121.82985941432459) bank196090 +196091 POINT(36.962445569245325 -121.53043096442671) bank196091 +196092 POINT(37.37077173527886 -121.43139794764383) bank196092 +196093 POINT(38.06971989103084 -123.03013802728069) bank196093 +196094 POINT(38.63143755526984 -122.10290368449472) bank196094 +196095 POINT(37.82107948698779 -123.04684569357624) bank196095 +196096 POINT(38.40268048762223 -123.19965306681046) bank196096 +196097 POINT(37.75634485777858 -123.20512147055905) bank196097 +196098 POINT(37.875421661660134 -122.75420850016349) bank196098 +196099 POINT(37.82823255871379 -122.04186442167212) bank196099 +196100 POINT(38.500245454285256 -122.6560316558419) bank196100 +196101 POINT(37.297692320677676 -122.42201239861461) bank196101 +196102 POINT(38.18092238363463 -121.68819871823777) bank196102 +196103 POINT(38.25590670871545 -122.4648437079447) bank196103 +196104 POINT(37.66889563108501 -122.51539396690323) bank196104 +196105 POINT(37.131212181212874 -122.44835580809917) bank196105 +196106 POINT(38.74584551650622 -122.63075934495207) bank196106 +196107 POINT(36.97307679320317 -122.51292345938187) bank196107 +196108 POINT(38.40523509910373 -122.52089727887129) bank196108 +196109 POINT(37.58276933852567 -122.40631159918874) bank196109 +196110 POINT(36.96285743452615 -122.23733912398347) bank196110 +196111 POINT(38.531990995465776 -122.00734908195177) bank196111 +196112 POINT(37.34826242742467 -121.89544409767377) bank196112 +196113 POINT(38.55780883978982 -123.41907093397859) bank196113 +196114 POINT(37.94593860089258 -122.81738885012174) bank196114 +196115 POINT(38.614482848378444 -121.8923476131606) bank196115 +196116 POINT(37.62948862615947 -122.08773394908094) bank196116 +196117 POINT(37.33608753722284 -122.75406857086294) bank196117 +196118 POINT(38.13906697456162 -122.42811060296532) bank196118 +196119 POINT(38.22728960668069 -122.73947485301692) bank196119 +196120 POINT(38.039587223608926 -121.49746020513167) bank196120 +196121 POINT(37.66867566686981 -121.97948325914031) bank196121 +196122 POINT(37.56581882677818 -121.85894174055454) bank196122 +196123 POINT(38.17031763077514 -121.8476172414623) bank196123 +196124 POINT(38.587054273025814 -123.1881183965585) bank196124 +196125 POINT(38.53992510458264 -121.57295914633832) bank196125 +196126 POINT(38.14759433020863 -122.4244092496451) bank196126 +196127 POINT(38.06707017404293 -122.1348582982519) bank196127 +196128 POINT(37.00667876627289 -121.63709099606316) bank196128 +196129 POINT(38.74619645780707 -123.2994339671288) bank196129 +196130 POINT(37.91427857788289 -122.17112498492395) bank196130 +196131 POINT(37.76095961125029 -122.07421126550148) bank196131 +196132 POINT(38.0754713528224 -121.54564520844683) bank196132 +196133 POINT(38.27481488374938 -122.97971097651138) bank196133 +196134 POINT(38.73318648249769 -121.95162720352677) bank196134 +196135 POINT(36.88233968773057 -122.87685983635245) bank196135 +196136 POINT(37.11332811300699 -121.93866793898844) bank196136 +196137 POINT(37.99949563436534 -122.093538338956) bank196137 +196138 POINT(37.20532548227353 -122.87901043193537) bank196138 +196139 POINT(37.1903950419366 -122.88415241705685) bank196139 +196140 POINT(36.86421026153802 -121.66583079888969) bank196140 +196141 POINT(37.08526244859454 -122.51764457737961) bank196141 +196142 POINT(38.71700324587033 -121.6517767758311) bank196142 +196143 POINT(37.81632314622724 -121.74777214379338) bank196143 +196144 POINT(38.10193295908561 -123.19319278577709) bank196144 +196145 POINT(37.008183355059856 -123.41906098470638) bank196145 +196146 POINT(38.32343709124876 -122.03241017951552) bank196146 +196147 POINT(36.853527074640816 -122.21141106552587) bank196147 +196148 POINT(38.07553401071707 -122.89645900622492) bank196148 +196149 POINT(37.76851071572485 -122.9006721103394) bank196149 +196150 POINT(37.365005808242124 -122.76106002791099) bank196150 +196151 POINT(38.650280726922574 -121.76313789047194) bank196151 +196152 POINT(38.3982912144995 -121.9946225460988) bank196152 +196153 POINT(38.41398200778795 -122.73147338953956) bank196153 +196154 POINT(37.09794633911474 -122.79642900817585) bank196154 +196155 POINT(38.52021658091678 -122.07513855834453) bank196155 +196156 POINT(37.509964577603846 -122.41088197345994) bank196156 +196157 POINT(37.5443407847393 -122.31873051036077) bank196157 +196158 POINT(38.12070207131856 -121.59073987666892) bank196158 +196159 POINT(38.1620130136615 -121.79631959537988) bank196159 +196160 POINT(37.08414850838175 -121.9241198577009) bank196160 +196161 POINT(38.773002808386366 -122.19978750949743) bank196161 +196162 POINT(37.53279224792596 -121.68574068503615) bank196162 +196163 POINT(37.362089007988644 -122.82333024899496) bank196163 +196164 POINT(37.36588883818413 -122.35388772879878) bank196164 +196165 POINT(38.069818429599216 -123.12696137139127) bank196165 +196166 POINT(38.25211525062633 -123.06984572410393) bank196166 +196167 POINT(37.43678427162338 -121.96749608428905) bank196167 +196168 POINT(36.77676898794644 -122.10607101119557) bank196168 +196169 POINT(37.71388189347432 -122.57422717547567) bank196169 +196170 POINT(38.44889200504499 -121.42275441692621) bank196170 +196171 POINT(37.24389286233478 -121.65298289663903) bank196171 +196172 POINT(37.23077423510634 -122.27125994366044) bank196172 +196173 POINT(38.5834940891459 -122.83760453115198) bank196173 +196174 POINT(37.26656461243839 -122.01305491168051) bank196174 +196175 POINT(36.81084693676674 -122.25633176930381) bank196175 +196176 POINT(37.203023776807754 -121.97390347257205) bank196176 +196177 POINT(37.575464775832316 -122.45051466591636) bank196177 +196178 POINT(37.18432518911873 -122.67543788914044) bank196178 +196179 POINT(37.26940581547483 -122.86554501381293) bank196179 +196180 POINT(36.84299928399228 -122.90136000349223) bank196180 +196181 POINT(37.14331875790325 -121.70990215877472) bank196181 +196182 POINT(37.21097644968998 -122.79068010542461) bank196182 +196183 POINT(37.86889035558953 -122.0517355214761) bank196183 +196184 POINT(38.03327666787155 -122.9197662115474) bank196184 +196185 POINT(37.02784153772964 -123.29531183179353) bank196185 +196186 POINT(36.85475491715276 -123.38722508603664) bank196186 +196187 POINT(38.605486764876375 -122.11798575090924) bank196187 +196188 POINT(37.501050747747975 -121.74967930380733) bank196188 +196189 POINT(36.857614355464975 -122.76817545126661) bank196189 +196190 POINT(38.14805833462882 -123.31871014410794) bank196190 +196191 POINT(38.6120273961272 -123.05310981502633) bank196191 +196192 POINT(38.58616042677369 -123.39330503607512) bank196192 +196193 POINT(37.157443684765944 -123.20308532499715) bank196193 +196194 POINT(37.036653185615236 -121.73010034648648) bank196194 +196195 POINT(37.85375536576675 -123.08657293067391) bank196195 +196196 POINT(38.537964114537935 -121.51529894180902) bank196196 +196197 POINT(38.2927058432704 -122.62690518170304) bank196197 +196198 POINT(36.82473971826717 -121.94307040396623) bank196198 +196199 POINT(36.925653729798185 -122.3633648128509) bank196199 +196200 POINT(38.033769035813776 -121.61916076170588) bank196200 +196201 POINT(37.86640071175454 -121.92911836277368) bank196201 +196202 POINT(37.67125367132257 -121.57948566224755) bank196202 +196203 POINT(37.456443895011944 -121.57035754317904) bank196203 +196204 POINT(36.986281431002276 -122.91557739788374) bank196204 +196205 POINT(38.54629790476632 -122.4496903757256) bank196205 +196206 POINT(37.193975885531444 -122.52059974755348) bank196206 +196207 POINT(37.965105541217014 -123.36986786656021) bank196207 +196208 POINT(37.548722309947934 -122.2450241952426) bank196208 +196209 POINT(36.98424694769803 -121.59422110911238) bank196209 +196210 POINT(37.949759546012736 -122.59250409235457) bank196210 +196211 POINT(38.284194091768775 -122.62588788633477) bank196211 +196212 POINT(37.46087593159289 -122.58999394704688) bank196212 +196213 POINT(37.10077499114609 -122.52220629392346) bank196213 +196214 POINT(38.35880496782676 -121.99884093801808) bank196214 +196215 POINT(37.1064648378029 -122.07858651941751) bank196215 +196216 POINT(37.54656811705651 -121.48278309380092) bank196216 +196217 POINT(37.98994985179966 -122.35669911536976) bank196217 +196218 POINT(38.05541623737116 -122.45658822332541) bank196218 +196219 POINT(37.099249450586434 -123.01875012147285) bank196219 +196220 POINT(37.82997986311623 -122.90681009650099) bank196220 +196221 POINT(37.19392427873165 -122.20830127188282) bank196221 +196222 POINT(37.927098014164635 -121.47612261970671) bank196222 +196223 POINT(37.44514787147899 -123.10861663153844) bank196223 +196224 POINT(37.6110207058902 -121.6045769781875) bank196224 +196225 POINT(38.22262400554263 -121.84497529684913) bank196225 +196226 POINT(38.31932392461883 -122.81598915191529) bank196226 +196227 POINT(38.169300932004354 -122.40857023679841) bank196227 +196228 POINT(38.220156291681114 -123.30161288188968) bank196228 +196229 POINT(38.611221146165136 -122.82498755671841) bank196229 +196230 POINT(37.12901682037533 -122.30498244094757) bank196230 +196231 POINT(38.02869826624745 -121.6579452620759) bank196231 +196232 POINT(36.8143980322523 -121.54154614551615) bank196232 +196233 POINT(38.25344637814255 -121.6846328176182) bank196233 +196234 POINT(37.87675770474253 -122.90313697534653) bank196234 +196235 POINT(38.38624995079172 -122.19528974194814) bank196235 +196236 POINT(38.64462551624103 -122.87117259963928) bank196236 +196237 POINT(38.15444714093559 -121.45682497590848) bank196237 +196238 POINT(38.603024811033 -122.08603104837375) bank196238 +196239 POINT(36.86147405152064 -121.48746952499658) bank196239 +196240 POINT(37.918264895618464 -123.33381819266612) bank196240 +196241 POINT(37.785416274842895 -123.2727785208909) bank196241 +196242 POINT(37.136834308119774 -123.1535218539223) bank196242 +196243 POINT(36.77993189342342 -121.45238314284387) bank196243 +196244 POINT(37.9622042312752 -121.47950594525341) bank196244 +196245 POINT(37.31373832768598 -122.31412640239766) bank196245 +196246 POINT(38.419869311817415 -121.50986301696499) bank196246 +196247 POINT(38.11247083026801 -123.29786712829257) bank196247 +196248 POINT(38.102595666402884 -122.36531495872566) bank196248 +196249 POINT(37.45634821648604 -123.05907693419051) bank196249 +196250 POINT(38.57126288201092 -122.59969587210654) bank196250 +196251 POINT(38.1994474171176 -122.07406973847752) bank196251 +196252 POINT(36.78029331981147 -122.98682085013924) bank196252 +196253 POINT(38.046002797069306 -122.91212903784567) bank196253 +196254 POINT(38.69744361416538 -122.79237526484026) bank196254 +196255 POINT(37.253576941259105 -123.31701782200129) bank196255 +196256 POINT(38.407759490933564 -123.24174111551004) bank196256 +196257 POINT(36.80148899585005 -122.49877549801273) bank196257 +196258 POINT(38.51218389042097 -122.15518037434076) bank196258 +196259 POINT(36.994626339517865 -121.73182313124784) bank196259 +196260 POINT(38.61475981448642 -121.46564456221344) bank196260 +196261 POINT(37.59213053197784 -122.16409016987626) bank196261 +196262 POINT(38.07823608234503 -122.2340518936708) bank196262 +196263 POINT(37.06799548039242 -121.50541213127322) bank196263 +196264 POINT(38.09237376617121 -122.99065894799344) bank196264 +196265 POINT(36.86149229286125 -122.55763300342846) bank196265 +196266 POINT(37.19561187909266 -122.07375024912842) bank196266 +196267 POINT(38.067346197299216 -121.91049757387387) bank196267 +196268 POINT(37.4126336180877 -121.51664206529605) bank196268 +196269 POINT(37.09451988345151 -123.35343736606066) bank196269 +196270 POINT(37.794103182270476 -121.84192552435144) bank196270 +196271 POINT(37.78518647207899 -123.18899290304793) bank196271 +196272 POINT(38.53502026520541 -123.06591270598118) bank196272 +196273 POINT(37.46281422454495 -122.0543396043757) bank196273 +196274 POINT(37.869523542009546 -121.53680926394719) bank196274 +196275 POINT(37.707417164289616 -121.96233922571021) bank196275 +196276 POINT(37.426867376910174 -123.40568557513144) bank196276 +196277 POINT(37.03738750189453 -122.5110249824104) bank196277 +196278 POINT(38.45526838414573 -122.58436188404605) bank196278 +196279 POINT(38.519780821888 -121.68665355833895) bank196279 +196280 POINT(37.245795683511204 -123.24344950882285) bank196280 +196281 POINT(36.82037805349821 -123.31566004651071) bank196281 +196282 POINT(36.832928539914484 -121.70857528423026) bank196282 +196283 POINT(37.768026260672634 -122.60066028146052) bank196283 +196284 POINT(37.17074649941677 -121.60237134744092) bank196284 +196285 POINT(37.97552681848329 -122.02867786230243) bank196285 +196286 POINT(37.911879628421865 -122.62697018240277) bank196286 +196287 POINT(38.11726335982995 -122.4283351988161) bank196287 +196288 POINT(37.916831026312394 -122.1010644524697) bank196288 +196289 POINT(38.39240929301601 -122.1536867460827) bank196289 +196290 POINT(37.215898884951514 -122.20526314073938) bank196290 +196291 POINT(37.038152025215815 -122.32027916273651) bank196291 +196292 POINT(37.998065638060375 -121.85853768792856) bank196292 +196293 POINT(36.78465176757202 -121.80672948614969) bank196293 +196294 POINT(38.48558459955388 -122.14751114997505) bank196294 +196295 POINT(36.95105177922157 -122.20627280241348) bank196295 +196296 POINT(37.99231281354636 -122.66476026640893) bank196296 +196297 POINT(36.934026085445204 -122.48729143010449) bank196297 +196298 POINT(38.18095403474475 -123.24807825976809) bank196298 +196299 POINT(37.691107797676835 -122.95982970185206) bank196299 +196300 POINT(37.52764188295909 -123.28670080508596) bank196300 +196301 POINT(38.02604227503685 -122.75863395443957) bank196301 +196302 POINT(37.08924262964296 -123.25422448858032) bank196302 +196303 POINT(36.92677496010649 -123.08941285744021) bank196303 +196304 POINT(36.865705355706815 -122.63733676910736) bank196304 +196305 POINT(38.2757130378263 -122.47021203721128) bank196305 +196306 POINT(37.08798192883696 -121.72571817828555) bank196306 +196307 POINT(37.68171781946296 -122.08656495843535) bank196307 +196308 POINT(37.68177388062665 -123.00401748675723) bank196308 +196309 POINT(38.25690310653849 -121.56382685540575) bank196309 +196310 POINT(37.90099905806092 -122.97171287606552) bank196310 +196311 POINT(37.0882378281012 -122.82819696283322) bank196311 +196312 POINT(37.113551202314746 -123.26525567583832) bank196312 +196313 POINT(38.29744898067952 -122.0190244228539) bank196313 +196314 POINT(37.94772308311859 -123.10708990127581) bank196314 +196315 POINT(36.8704782823486 -123.31465060001615) bank196315 +196316 POINT(38.02200553617513 -122.9812673936877) bank196316 +196317 POINT(36.94539332833573 -123.02493825212551) bank196317 +196318 POINT(38.12015591801386 -121.76047546620798) bank196318 +196319 POINT(36.77624596694455 -122.15312108604356) bank196319 +196320 POINT(38.50144572920103 -122.97575282978963) bank196320 +196321 POINT(38.41407758629978 -121.94169075301349) bank196321 +196322 POINT(37.33327112043209 -121.44431543882264) bank196322 +196323 POINT(38.47180213869545 -123.28072785793432) bank196323 +196324 POINT(38.45023448447057 -123.19882069540665) bank196324 +196325 POINT(37.422283397946444 -121.52729444913864) bank196325 +196326 POINT(37.00674724741674 -121.93561784964983) bank196326 +196327 POINT(36.88249916782455 -122.80129243251305) bank196327 +196328 POINT(38.70629210464232 -122.15620310665122) bank196328 +196329 POINT(37.86818865435559 -122.20383200865449) bank196329 +196330 POINT(37.4435473255031 -122.45451912140717) bank196330 +196331 POINT(38.194643060370154 -122.45338989584654) bank196331 +196332 POINT(38.414509166477515 -123.36295986782457) bank196332 +196333 POINT(37.09067296051965 -122.26245602875433) bank196333 +196334 POINT(38.643963026872235 -122.60085632494032) bank196334 +196335 POINT(37.222884335019174 -122.15412764376333) bank196335 +196336 POINT(38.2304042795321 -122.02851625757286) bank196336 +196337 POINT(38.67726832259269 -122.8456414934453) bank196337 +196338 POINT(37.60820067652885 -123.10706125712866) bank196338 +196339 POINT(36.91232433752725 -123.30027569333427) bank196339 +196340 POINT(37.24130417723977 -123.11862390575595) bank196340 +196341 POINT(36.990888183613805 -122.38528735365274) bank196341 +196342 POINT(38.46818211138157 -122.96475822179919) bank196342 +196343 POINT(38.71032950222039 -121.56677218300064) bank196343 +196344 POINT(38.05156961917001 -123.14967649620074) bank196344 +196345 POINT(36.99536981830321 -121.93059505517932) bank196345 +196346 POINT(37.51384254086697 -122.96686055884621) bank196346 +196347 POINT(38.734421676584226 -121.52826845893188) bank196347 +196348 POINT(37.18070674169145 -122.53904701876232) bank196348 +196349 POINT(37.594765189608424 -122.68590199225396) bank196349 +196350 POINT(37.166507073615506 -123.40203928690777) bank196350 +196351 POINT(37.17229370920096 -121.51653784474925) bank196351 +196352 POINT(37.38339922726688 -122.74054383717299) bank196352 +196353 POINT(36.98411625523904 -121.45414882287598) bank196353 +196354 POINT(37.693184761936315 -123.32104833748959) bank196354 +196355 POINT(38.323690876479176 -123.27406288790597) bank196355 +196356 POINT(37.51185978546153 -121.64911294555135) bank196356 +196357 POINT(38.559312973633666 -123.10745864880987) bank196357 +196358 POINT(37.98045875616157 -122.06499576648103) bank196358 +196359 POINT(37.45507748573888 -122.98952498368328) bank196359 +196360 POINT(38.42604182196257 -123.27563784351378) bank196360 +196361 POINT(37.64340111745279 -121.90488927988842) bank196361 +196362 POINT(38.65185826376717 -122.75290450559058) bank196362 +196363 POINT(37.5504632169253 -122.52674693539761) bank196363 +196364 POINT(38.200930365734465 -122.64590502553894) bank196364 +196365 POINT(37.75714996617506 -121.46689739261316) bank196365 +196366 POINT(38.4737690013648 -122.38585006550365) bank196366 +196367 POINT(38.495370768370364 -122.09036024320234) bank196367 +196368 POINT(38.30072438290927 -122.66142834204096) bank196368 +196369 POINT(36.81416801069781 -122.78310432563578) bank196369 +196370 POINT(38.63865296275336 -121.548469877591) bank196370 +196371 POINT(37.85545341426279 -122.68282330372706) bank196371 +196372 POINT(37.454975279071455 -122.94685732726958) bank196372 +196373 POINT(37.54665280933346 -122.7275828693763) bank196373 +196374 POINT(36.90743337706509 -123.05742576330327) bank196374 +196375 POINT(37.572304285736486 -121.47054214809681) bank196375 +196376 POINT(38.15902340273509 -122.60227660412177) bank196376 +196377 POINT(38.49542402387654 -122.74619214676284) bank196377 +196378 POINT(37.71147289666404 -121.97577199857476) bank196378 +196379 POINT(38.56114707158169 -122.55377020658352) bank196379 +196380 POINT(36.94434810223277 -122.55764545223234) bank196380 +196381 POINT(37.97504411936613 -121.52506062881339) bank196381 +196382 POINT(37.2714407719749 -121.73910344659154) bank196382 +196383 POINT(37.29527849142079 -121.97593581709069) bank196383 +196384 POINT(38.03459318258104 -122.17914477305601) bank196384 +196385 POINT(38.45395457780311 -121.87239015629598) bank196385 +196386 POINT(37.837746160199664 -122.80183404448694) bank196386 +196387 POINT(37.23845488682281 -122.89305000819108) bank196387 +196388 POINT(38.34567077543665 -122.24369260423674) bank196388 +196389 POINT(37.148760794706426 -122.84747886296267) bank196389 +196390 POINT(38.403401825128554 -122.53508830613332) bank196390 +196391 POINT(37.37494412916786 -122.26996822867443) bank196391 +196392 POINT(37.45743689262064 -123.19918603816457) bank196392 +196393 POINT(38.16147662600996 -123.19365633728421) bank196393 +196394 POINT(37.01678178387325 -122.19923884740166) bank196394 +196395 POINT(36.920329271679364 -122.95004190224728) bank196395 +196396 POINT(38.31335901234252 -122.22826739442404) bank196396 +196397 POINT(37.0327888378575 -121.47628183579776) bank196397 +196398 POINT(37.51587868199496 -122.42444547247004) bank196398 +196399 POINT(37.408291881680235 -122.24692179542284) bank196399 +196400 POINT(38.233777470968185 -122.54237022998687) bank196400 +196401 POINT(38.59155926503728 -122.96643954843019) bank196401 +196402 POINT(37.65992085699631 -122.8241861666138) bank196402 +196403 POINT(36.77934716351584 -122.35951910387772) bank196403 +196404 POINT(38.486459928461635 -122.20176638281382) bank196404 +196405 POINT(37.81044973439682 -123.15514761455081) bank196405 +196406 POINT(37.64503326314369 -123.0339142059027) bank196406 +196407 POINT(37.505999837523184 -121.66235284167227) bank196407 +196408 POINT(37.922687303301345 -122.41454115575392) bank196408 +196409 POINT(37.38963044587849 -122.32701554516483) bank196409 +196410 POINT(38.21742085699266 -122.41583756994079) bank196410 +196411 POINT(38.10117648767684 -122.89519280898051) bank196411 +196412 POINT(37.22324968827014 -122.89952096553229) bank196412 +196413 POINT(37.69043477017287 -123.39787935208052) bank196413 +196414 POINT(38.17781227170943 -122.78279850114426) bank196414 +196415 POINT(38.033183415101085 -121.94452283375054) bank196415 +196416 POINT(38.38058283746 -123.41739336416481) bank196416 +196417 POINT(38.71483364541071 -122.34700331508016) bank196417 +196418 POINT(37.822157843430446 -122.26815278706907) bank196418 +196419 POINT(36.90552221321971 -122.19800679129459) bank196419 +196420 POINT(38.578932061443524 -121.81403099935744) bank196420 +196421 POINT(38.27183700093437 -123.02699377290887) bank196421 +196422 POINT(38.539742013286684 -122.65768388825752) bank196422 +196423 POINT(38.771771326983036 -121.98356489546393) bank196423 +196424 POINT(36.80992621587836 -123.3252012972164) bank196424 +196425 POINT(37.81488655424601 -123.08976428714382) bank196425 +196426 POINT(38.326354614072784 -122.83597020377269) bank196426 +196427 POINT(37.9722756160483 -123.3635771349098) bank196427 +196428 POINT(38.41204592025051 -121.58543364239274) bank196428 +196429 POINT(38.61013956783508 -121.71729878849429) bank196429 +196430 POINT(37.09059140066412 -122.22350334367914) bank196430 +196431 POINT(38.461285002072515 -122.7781598678981) bank196431 +196432 POINT(37.41954151694705 -122.96195467651191) bank196432 +196433 POINT(37.684248362283604 -121.66185921732269) bank196433 +196434 POINT(37.18685488055056 -121.95394283162757) bank196434 +196435 POINT(37.756756897414654 -122.04840873217967) bank196435 +196436 POINT(37.27021639531594 -121.95611740631685) bank196436 +196437 POINT(36.83010783721688 -121.9657852922258) bank196437 +196438 POINT(37.4125060250084 -122.61451804682281) bank196438 +196439 POINT(37.48151434484709 -121.67715495297371) bank196439 +196440 POINT(37.47788903442764 -122.36447020233054) bank196440 +196441 POINT(38.017898462870036 -122.73511234118861) bank196441 +196442 POINT(38.739077686932674 -122.9930994130345) bank196442 +196443 POINT(38.177950036899844 -121.7067515678584) bank196443 +196444 POINT(38.05330425314458 -121.63575713851053) bank196444 +196445 POINT(38.00979320560296 -123.28847772032259) bank196445 +196446 POINT(37.55257514297899 -121.68860485130257) bank196446 +196447 POINT(36.993983827346185 -122.67100258284233) bank196447 +196448 POINT(37.75977655437978 -122.41617177729084) bank196448 +196449 POINT(37.63791270241831 -123.41513379829362) bank196449 +196450 POINT(37.210637784331496 -122.75357247118896) bank196450 +196451 POINT(36.880928935561705 -121.52921407341286) bank196451 +196452 POINT(37.70107411662623 -122.80526633663877) bank196452 +196453 POINT(36.857381923538476 -121.54061110415562) bank196453 +196454 POINT(36.81216488717791 -122.53685256872585) bank196454 +196455 POINT(38.581437832198894 -122.11490379071756) bank196455 +196456 POINT(37.117379924500966 -121.81478742636891) bank196456 +196457 POINT(36.85797658078743 -121.70651967161118) bank196457 +196458 POINT(38.38162135160986 -123.24122881163308) bank196458 +196459 POINT(36.81302444713078 -122.77597509844335) bank196459 +196460 POINT(38.212713671129926 -121.68277589038394) bank196460 +196461 POINT(37.53157179272175 -122.98717026719696) bank196461 +196462 POINT(36.8071930470533 -121.8148709421501) bank196462 +196463 POINT(37.232267312080495 -123.35122687543856) bank196463 +196464 POINT(37.5218309181898 -122.92969110842604) bank196464 +196465 POINT(38.54128954541009 -122.24425067490411) bank196465 +196466 POINT(37.904175461469215 -122.65868239117097) bank196466 +196467 POINT(37.68025018120702 -122.133166531295) bank196467 +196468 POINT(38.365359374675585 -122.02865073766002) bank196468 +196469 POINT(37.27317013455409 -123.31764253894916) bank196469 +196470 POINT(36.86644723669358 -121.43572496133697) bank196470 +196471 POINT(37.738489131946515 -122.79327642270209) bank196471 +196472 POINT(38.51616496924426 -122.27170920478726) bank196472 +196473 POINT(38.3802863793363 -121.84453470724915) bank196473 +196474 POINT(37.832809453750585 -121.99990917613438) bank196474 +196475 POINT(37.56958131162258 -122.98860671182076) bank196475 +196476 POINT(38.29971467602615 -122.06287123507737) bank196476 +196477 POINT(37.07860432532446 -121.73705053743122) bank196477 +196478 POINT(38.20731146260361 -122.94504589808237) bank196478 +196479 POINT(36.91169666011169 -122.388073142016) bank196479 +196480 POINT(38.0923008182203 -123.40900258799496) bank196480 +196481 POINT(37.698304695985115 -123.32777031085111) bank196481 +196482 POINT(37.514162348348904 -121.7070299872126) bank196482 +196483 POINT(37.484534928034634 -121.68910094580487) bank196483 +196484 POINT(38.64102468576673 -122.02580606936928) bank196484 +196485 POINT(38.408353910467085 -122.55295703018976) bank196485 +196486 POINT(37.04532721010492 -122.3966529733879) bank196486 +196487 POINT(37.00341955948295 -122.21176539363593) bank196487 +196488 POINT(38.0774082268657 -121.97519282229598) bank196488 +196489 POINT(37.79610773527693 -122.25863665714493) bank196489 +196490 POINT(37.10103787072478 -121.85710590916182) bank196490 +196491 POINT(37.95104525018838 -122.61846242819085) bank196491 +196492 POINT(36.921162891854905 -121.95929697268365) bank196492 +196493 POINT(36.90652310762343 -122.13311310651721) bank196493 +196494 POINT(37.59936647980594 -122.63376198146457) bank196494 +196495 POINT(36.8150823351065 -123.11254151119228) bank196495 +196496 POINT(37.12709966790641 -121.81996059437292) bank196496 +196497 POINT(36.90920285674937 -123.38373635251276) bank196497 +196498 POINT(37.23595490806779 -123.02618023984364) bank196498 +196499 POINT(37.09257172490592 -123.10986531773187) bank196499 +196500 POINT(38.46439291681246 -123.3996234727784) bank196500 +196501 POINT(38.53620849013291 -122.7096114146709) bank196501 +196502 POINT(36.77855195412908 -123.33532816953277) bank196502 +196503 POINT(37.31779324382414 -122.3462813490503) bank196503 +196504 POINT(38.63141584710485 -123.3373953226506) bank196504 +196505 POINT(38.234823793946376 -123.3607145245758) bank196505 +196506 POINT(38.183972041002555 -122.67007677684015) bank196506 +196507 POINT(38.144980710113664 -122.79727170814571) bank196507 +196508 POINT(37.51083135667041 -122.14794405422215) bank196508 +196509 POINT(37.56390766732292 -122.82385598471076) bank196509 +196510 POINT(38.33250368387532 -122.09653604466246) bank196510 +196511 POINT(38.393613747029804 -122.75279363303518) bank196511 +196512 POINT(37.75366709494358 -121.54271905503171) bank196512 +196513 POINT(38.11928236745834 -122.25253415172413) bank196513 +196514 POINT(37.91647009490042 -121.42916115576654) bank196514 +196515 POINT(38.72539880645492 -123.10599511594035) bank196515 +196516 POINT(37.21207659964593 -122.30660613892903) bank196516 +196517 POINT(37.16819845307366 -122.22562344364286) bank196517 +196518 POINT(37.20815800823361 -123.21568186723121) bank196518 +196519 POINT(38.390481825249594 -123.33560676006937) bank196519 +196520 POINT(37.672291818417925 -123.28154991835294) bank196520 +196521 POINT(38.38263949417109 -121.77080202378998) bank196521 +196522 POINT(37.8715498719558 -122.02689888684007) bank196522 +196523 POINT(38.30707292168904 -121.8639024474927) bank196523 +196524 POINT(37.74916431701395 -122.48840724647552) bank196524 +196525 POINT(38.19823266782564 -122.8156540484708) bank196525 +196526 POINT(38.773189666967426 -122.22137392965698) bank196526 +196527 POINT(37.94477993412476 -122.55139408310377) bank196527 +196528 POINT(38.33893705176624 -121.87053886896383) bank196528 +196529 POINT(36.81975555424594 -121.52357298857257) bank196529 +196530 POINT(37.75157985873216 -123.11469695327747) bank196530 +196531 POINT(37.875450263930304 -122.40947777212448) bank196531 +196532 POINT(38.0824082510813 -121.98737109580108) bank196532 +196533 POINT(36.91886356970427 -123.03441541980632) bank196533 +196534 POINT(37.03166741559207 -123.12331387375225) bank196534 +196535 POINT(38.6674887843679 -122.71227274033743) bank196535 +196536 POINT(37.78600990602422 -121.81563287846286) bank196536 +196537 POINT(38.505141873006146 -123.35356570710437) bank196537 +196538 POINT(37.92503804422885 -123.06523531247974) bank196538 +196539 POINT(37.012958368022886 -122.6052234562709) bank196539 +196540 POINT(38.516378623340906 -123.1765849772989) bank196540 +196541 POINT(38.51783329660038 -122.91499250019064) bank196541 +196542 POINT(37.776905532338645 -123.23767458960846) bank196542 +196543 POINT(37.49303428120133 -123.40674846035776) bank196543 +196544 POINT(38.25055608021917 -121.94390583733248) bank196544 +196545 POINT(38.19367487826644 -122.04954731319388) bank196545 +196546 POINT(38.10631398923624 -121.85232910715088) bank196546 +196547 POINT(38.72436907773823 -122.1445961496909) bank196547 +196548 POINT(37.0118300500199 -122.19414207512696) bank196548 +196549 POINT(38.477470198721605 -122.55613777069452) bank196549 +196550 POINT(37.62793729410585 -123.3572897353377) bank196550 +196551 POINT(38.57249485826001 -123.38723723867233) bank196551 +196552 POINT(37.630934102376436 -121.49716282393422) bank196552 +196553 POINT(38.39308613962023 -121.68078945966654) bank196553 +196554 POINT(36.9843455857446 -122.5140025323272) bank196554 +196555 POINT(38.68050942619176 -122.84091257162801) bank196555 +196556 POINT(38.589921654035635 -121.63053245143655) bank196556 +196557 POINT(37.28778794294697 -122.85953282348989) bank196557 +196558 POINT(38.31977653900888 -123.16286410810994) bank196558 +196559 POINT(37.80453014131113 -121.97367441080674) bank196559 +196560 POINT(37.71354753862973 -121.62457880808793) bank196560 +196561 POINT(37.08382651477118 -121.50580731881023) bank196561 +196562 POINT(38.35802995914327 -122.94269139446308) bank196562 +196563 POINT(38.450583086002155 -122.59065055356334) bank196563 +196564 POINT(37.373402076217424 -122.50276289945074) bank196564 +196565 POINT(38.451613822981265 -123.09684478895711) bank196565 +196566 POINT(38.49055566427487 -122.31783861145539) bank196566 +196567 POINT(37.42776132744521 -122.46179974636453) bank196567 +196568 POINT(38.292196775149684 -122.50942366182663) bank196568 +196569 POINT(38.413972462182116 -122.99323252386232) bank196569 +196570 POINT(37.077799291588136 -121.58424014566219) bank196570 +196571 POINT(38.147895049217766 -122.98369729627944) bank196571 +196572 POINT(37.37575103753742 -121.6336671387691) bank196572 +196573 POINT(37.026330014544094 -123.35541962222987) bank196573 +196574 POINT(37.72142704751749 -122.18526390131795) bank196574 +196575 POINT(38.422653246956365 -122.68693705115241) bank196575 +196576 POINT(38.29956177251658 -122.90789985567164) bank196576 +196577 POINT(38.28839212181594 -121.93854409924919) bank196577 +196578 POINT(37.21427136440766 -123.30632401328926) bank196578 +196579 POINT(38.73377487654087 -122.21070999572206) bank196579 +196580 POINT(37.13897087623006 -121.44666889005673) bank196580 +196581 POINT(38.538991745983 -122.83288654017764) bank196581 +196582 POINT(37.325002912962084 -122.68088001777528) bank196582 +196583 POINT(37.34716744021428 -121.93953856617377) bank196583 +196584 POINT(38.48544215761739 -122.33492156847919) bank196584 +196585 POINT(38.29812317079468 -122.65027235663027) bank196585 +196586 POINT(37.97137695877305 -122.15692868135469) bank196586 +196587 POINT(38.17819056334885 -123.26507865110145) bank196587 +196588 POINT(37.729264486154975 -122.74737931563232) bank196588 +196589 POINT(37.009285642415385 -122.79207277833177) bank196589 +196590 POINT(38.553366445134735 -122.0146013207732) bank196590 +196591 POINT(37.68407104718774 -121.79853466410914) bank196591 +196592 POINT(37.47274565026483 -121.86742209582458) bank196592 +196593 POINT(36.99731621533126 -121.50737099218972) bank196593 +196594 POINT(38.500562425791955 -122.55820018515796) bank196594 +196595 POINT(38.51521801507972 -122.67080761976177) bank196595 +196596 POINT(36.80059687100197 -123.29043236876694) bank196596 +196597 POINT(38.107121427650554 -122.8269712186187) bank196597 +196598 POINT(37.97426713989716 -123.22026980935472) bank196598 +196599 POINT(37.427590703105686 -121.47557974630502) bank196599 +196600 POINT(36.777637910273214 -122.09876068843572) bank196600 +196601 POINT(38.19259243538753 -122.91836049038841) bank196601 +196602 POINT(37.19194072333037 -122.35780263675616) bank196602 +196603 POINT(38.52660906638556 -121.73266397112819) bank196603 +196604 POINT(37.595273224015834 -123.18920690397381) bank196604 +196605 POINT(36.79084972034153 -122.76041080052053) bank196605 +196606 POINT(37.941235299486266 -121.90211300339831) bank196606 +196607 POINT(36.78369142483429 -122.83211961901492) bank196607 +196608 POINT(37.46965818203942 -122.41343247643707) bank196608 +196609 POINT(37.038446751103685 -122.21976264518827) bank196609 +196610 POINT(37.75329120661373 -122.03617832306976) bank196610 +196611 POINT(37.45684442887234 -121.82969350899938) bank196611 +196612 POINT(37.44368698348276 -122.92460459248956) bank196612 +196613 POINT(37.89306526046097 -122.17565411502258) bank196613 +196614 POINT(38.03120783587595 -121.57910168272683) bank196614 +196615 POINT(37.34077332348604 -121.45226795208907) bank196615 +196616 POINT(38.591270124816155 -122.3963647790091) bank196616 +196617 POINT(38.41428208338343 -122.46744353822669) bank196617 +196618 POINT(38.57518512287366 -123.0831009589558) bank196618 +196619 POINT(38.48491173317298 -123.04945768206723) bank196619 +196620 POINT(37.677389775234005 -123.27217895937945) bank196620 +196621 POINT(37.84803837629592 -123.10847414839124) bank196621 +196622 POINT(37.91986236982919 -121.48443284985916) bank196622 +196623 POINT(38.51177111827669 -123.1300901566481) bank196623 +196624 POINT(37.17536664376434 -122.85400068451422) bank196624 +196625 POINT(37.27373011981298 -123.06388788558736) bank196625 +196626 POINT(38.259236127526506 -123.13499195221368) bank196626 +196627 POINT(36.878620650836844 -122.02042402143604) bank196627 +196628 POINT(37.910660913980664 -122.66594530487617) bank196628 +196629 POINT(37.31067771407049 -121.68994406253368) bank196629 +196630 POINT(36.94699846621172 -123.15406988053503) bank196630 +196631 POINT(38.13593614470667 -121.453148801347) bank196631 +196632 POINT(37.17122238241021 -121.50053501156152) bank196632 +196633 POINT(37.96844117663016 -121.70638888575925) bank196633 +196634 POINT(38.430709683757975 -121.55084250533461) bank196634 +196635 POINT(37.19427820898337 -123.08550934619608) bank196635 +196636 POINT(38.327581396774555 -122.15967533216765) bank196636 +196637 POINT(38.213212743904876 -121.96639333834521) bank196637 +196638 POINT(36.95940125731566 -122.89650100726317) bank196638 +196639 POINT(37.23414582092235 -123.21854192245087) bank196639 +196640 POINT(38.763597080405056 -122.74277396364218) bank196640 +196641 POINT(37.83904325286178 -121.6332352719835) bank196641 +196642 POINT(36.88755491290188 -121.99620080449375) bank196642 +196643 POINT(38.11209322404314 -122.29993541991574) bank196643 +196644 POINT(38.500170208305505 -123.32449922326862) bank196644 +196645 POINT(37.344937055621585 -122.7657055272107) bank196645 +196646 POINT(38.108302284810456 -121.75664574570696) bank196646 +196647 POINT(37.241159071253925 -123.24127241876931) bank196647 +196648 POINT(37.320590911540236 -121.86972690370597) bank196648 +196649 POINT(37.07759672998588 -122.18362394592437) bank196649 +196650 POINT(37.00215399780058 -123.22531850390324) bank196650 +196651 POINT(36.790426371779375 -122.71058782860763) bank196651 +196652 POINT(37.610020655045176 -123.00996564547503) bank196652 +196653 POINT(37.76476193094951 -122.19850421233058) bank196653 +196654 POINT(38.67494308666295 -121.89697763568593) bank196654 +196655 POINT(37.70070584395629 -123.25173237583213) bank196655 +196656 POINT(37.15822595079949 -121.69835531168314) bank196656 +196657 POINT(37.65003890357932 -121.70550432603132) bank196657 +196658 POINT(37.322719363227094 -123.11838492825133) bank196658 +196659 POINT(37.297897059104464 -121.99438729732019) bank196659 +196660 POINT(36.78543370141952 -122.38291647891275) bank196660 +196661 POINT(38.59404076895046 -121.7015164784959) bank196661 +196662 POINT(38.505182729739815 -123.23934489150233) bank196662 +196663 POINT(37.16533764471507 -122.4471761338595) bank196663 +196664 POINT(37.82229272167645 -122.05601397870677) bank196664 +196665 POINT(38.0582659161383 -121.98185766509036) bank196665 +196666 POINT(37.236494347324275 -122.59863069462969) bank196666 +196667 POINT(37.48512477042398 -122.3600878940501) bank196667 +196668 POINT(37.30880893193573 -122.50465727979687) bank196668 +196669 POINT(37.16470450231243 -121.9705744358576) bank196669 +196670 POINT(36.94393466118399 -123.30839193677413) bank196670 +196671 POINT(38.72438022281965 -122.56087764832047) bank196671 +196672 POINT(38.61034624969458 -123.00651687684174) bank196672 +196673 POINT(37.02753716393159 -121.97232651856056) bank196673 +196674 POINT(38.453191429435925 -122.01334198804273) bank196674 +196675 POINT(38.02278824258935 -122.74358373728654) bank196675 +196676 POINT(38.46404955290031 -122.2472567785491) bank196676 +196677 POINT(38.392404131494864 -122.42724000290104) bank196677 +196678 POINT(38.00855729651444 -122.66993478162668) bank196678 +196679 POINT(38.74127544386559 -121.44482541742344) bank196679 +196680 POINT(37.73393173691971 -123.00830259992281) bank196680 +196681 POINT(37.67248766713597 -122.43234064994607) bank196681 +196682 POINT(38.466140877747605 -122.81433864173381) bank196682 +196683 POINT(36.81526694503908 -122.02255121340703) bank196683 +196684 POINT(37.88989949328721 -121.55380856531669) bank196684 +196685 POINT(37.74904519772559 -121.57752068374799) bank196685 +196686 POINT(38.235735940716786 -121.65796489258952) bank196686 +196687 POINT(37.94569869712541 -123.02152709792814) bank196687 +196688 POINT(38.59173133777399 -122.57780151360285) bank196688 +196689 POINT(38.338400673512005 -122.44348988159085) bank196689 +196690 POINT(37.125462468852646 -121.7445585200958) bank196690 +196691 POINT(37.96392695970969 -122.36032893909034) bank196691 +196692 POINT(38.570379342486945 -121.61062490690969) bank196692 +196693 POINT(38.65933673624901 -123.37366470088638) bank196693 +196694 POINT(37.41809462583332 -122.0563734888298) bank196694 +196695 POINT(37.84258751263312 -122.36480843534241) bank196695 +196696 POINT(38.487069179223674 -121.8056840273528) bank196696 +196697 POINT(37.98322282016534 -122.67432179267084) bank196697 +196698 POINT(38.63387706514871 -121.63716429364523) bank196698 +196699 POINT(37.167030954054624 -123.33298933729802) bank196699 +196700 POINT(38.497680165248525 -121.7689123764865) bank196700 +196701 POINT(38.20503911990939 -123.21126488297635) bank196701 +196702 POINT(37.62090429649343 -122.76433778107202) bank196702 +196703 POINT(38.06146134020616 -121.42451697611013) bank196703 +196704 POINT(37.4957619636902 -122.03258785080732) bank196704 +196705 POINT(36.93219501504991 -122.39201447483329) bank196705 +196706 POINT(36.947218740696535 -121.48930029263325) bank196706 +196707 POINT(37.21152549174724 -122.39711425477368) bank196707 +196708 POINT(37.88694436339272 -122.22747180583852) bank196708 +196709 POINT(38.72821122311072 -122.28064433313074) bank196709 +196710 POINT(37.110347290948276 -122.23909149759105) bank196710 +196711 POINT(37.26572238412159 -123.39255552923647) bank196711 +196712 POINT(38.64706454318612 -123.19449183355727) bank196712 +196713 POINT(38.4924104323864 -121.9546897942755) bank196713 +196714 POINT(38.08406846490248 -122.04064410753108) bank196714 +196715 POINT(37.71521884380883 -121.61356857423277) bank196715 +196716 POINT(37.297508395549535 -122.47670953670911) bank196716 +196717 POINT(38.25754105076775 -123.17761591930879) bank196717 +196718 POINT(37.539319720231184 -122.16743420741479) bank196718 +196719 POINT(38.04254287667994 -122.48279528919059) bank196719 +196720 POINT(38.3281376621622 -121.56129466409585) bank196720 +196721 POINT(38.58878566101156 -121.49611172924214) bank196721 +196722 POINT(37.08074800311585 -122.07682242234995) bank196722 +196723 POINT(38.66144181604622 -121.82053190502967) bank196723 +196724 POINT(37.92166503558233 -121.45059472772722) bank196724 +196725 POINT(37.27621144471582 -121.97250511139502) bank196725 +196726 POINT(37.260982607294366 -122.9362263381479) bank196726 +196727 POINT(37.70455766961498 -123.0759848642228) bank196727 +196728 POINT(38.56676561022127 -122.93725341745333) bank196728 +196729 POINT(38.56185347562525 -122.19883750920748) bank196729 +196730 POINT(38.22426583113566 -122.05784881649163) bank196730 +196731 POINT(38.04448847239062 -123.02031717298304) bank196731 +196732 POINT(36.82371004260636 -121.76869441298544) bank196732 +196733 POINT(38.16930826825798 -123.02900981614337) bank196733 +196734 POINT(37.450017765475046 -121.80685726363825) bank196734 +196735 POINT(38.3729476782539 -121.84331661609693) bank196735 +196736 POINT(37.34222499546325 -121.57780691020378) bank196736 +196737 POINT(38.574590979357545 -122.91621258305027) bank196737 +196738 POINT(37.614422676887386 -122.77462251530578) bank196738 +196739 POINT(37.64540553119425 -122.61733100896247) bank196739 +196740 POINT(37.623929209048875 -121.48454554061374) bank196740 +196741 POINT(37.79526226931836 -122.4871639506632) bank196741 +196742 POINT(36.784641677548024 -123.16859564307215) bank196742 +196743 POINT(37.0798697033586 -123.3920419739371) bank196743 +196744 POINT(37.01288055204351 -121.57435124944212) bank196744 +196745 POINT(38.36328916225761 -122.23657082440107) bank196745 +196746 POINT(37.083652325435736 -122.534402250401) bank196746 +196747 POINT(37.34201483130847 -121.56588461856244) bank196747 +196748 POINT(38.362745528773765 -122.86422666482619) bank196748 +196749 POINT(37.09244787062058 -122.03473290653491) bank196749 +196750 POINT(38.691006009445815 -123.1793384980935) bank196750 +196751 POINT(38.64976989507165 -122.71230539584779) bank196751 +196752 POINT(36.99928600156493 -122.65401914364861) bank196752 +196753 POINT(37.3170482020384 -121.56984489350768) bank196753 +196754 POINT(37.79643294806575 -123.02009480888398) bank196754 +196755 POINT(37.24547200606558 -122.26116062053866) bank196755 +196756 POINT(37.06065941537328 -122.06287016197186) bank196756 +196757 POINT(38.63627449824694 -122.5115663933505) bank196757 +196758 POINT(36.90759094591293 -123.01110379836635) bank196758 +196759 POINT(37.70873863100721 -123.1776665583405) bank196759 +196760 POINT(37.4594155858277 -122.14756303358402) bank196760 +196761 POINT(38.304257682479374 -121.64093867410088) bank196761 +196762 POINT(38.445745630100525 -121.5677548111294) bank196762 +196763 POINT(37.98244402252228 -122.87565045709547) bank196763 +196764 POINT(38.008961077273284 -121.48184312605524) bank196764 +196765 POINT(38.65767231420821 -123.28862685824478) bank196765 +196766 POINT(37.10234163097777 -122.30099874487262) bank196766 +196767 POINT(38.210828153171654 -122.3130514433671) bank196767 +196768 POINT(36.79010491360006 -122.70065729266129) bank196768 +196769 POINT(37.48486900328462 -122.92496605115903) bank196769 +196770 POINT(37.79892958436786 -121.74020141077945) bank196770 +196771 POINT(37.59428681199657 -123.18617994225092) bank196771 +196772 POINT(38.22094301207176 -121.70565197901163) bank196772 +196773 POINT(38.11209767480255 -121.77225930941529) bank196773 +196774 POINT(36.81334688652519 -122.08929697969131) bank196774 +196775 POINT(37.79704705467358 -122.59157075513608) bank196775 +196776 POINT(37.96636773202034 -123.2658860200405) bank196776 +196777 POINT(38.2238716281572 -122.8622086577098) bank196777 +196778 POINT(38.7327673816505 -122.91654537977927) bank196778 +196779 POINT(37.696119832728215 -122.99236742571145) bank196779 +196780 POINT(37.84509376528236 -121.6097284621234) bank196780 +196781 POINT(38.48625931236374 -121.51127394943623) bank196781 +196782 POINT(37.30226933956534 -123.38541330581948) bank196782 +196783 POINT(36.88407199219738 -122.39387404204807) bank196783 +196784 POINT(37.860388438870615 -123.21311984810399) bank196784 +196785 POINT(37.18382379661283 -121.59592812453862) bank196785 +196786 POINT(37.646799467621534 -122.16156080288017) bank196786 +196787 POINT(38.198299833284835 -121.44660606743125) bank196787 +196788 POINT(37.09972695565317 -122.26706559570694) bank196788 +196789 POINT(37.11548910521219 -121.63860486429938) bank196789 +196790 POINT(38.50761931118134 -123.02792060074161) bank196790 +196791 POINT(37.16990821645333 -122.38994930336774) bank196791 +196792 POINT(37.724518881978625 -122.93873893677687) bank196792 +196793 POINT(37.202080544188135 -121.65885141272928) bank196793 +196794 POINT(37.417154719353015 -122.97449460994665) bank196794 +196795 POINT(38.449858712150004 -122.71327674648113) bank196795 +196796 POINT(37.39426806286836 -122.9320954673042) bank196796 +196797 POINT(37.00668795179821 -123.41198778224143) bank196797 +196798 POINT(37.0784819687603 -121.49060859202423) bank196798 +196799 POINT(38.67985556872886 -121.42200067458656) bank196799 +196800 POINT(38.58467676398116 -122.00639515989022) bank196800 +196801 POINT(38.4296741391192 -123.26494593298015) bank196801 +196802 POINT(38.10558175575711 -123.4186503287888) bank196802 +196803 POINT(37.047738375685626 -121.65157010859473) bank196803 +196804 POINT(38.059569331089264 -122.29662596146765) bank196804 +196805 POINT(37.29785254457281 -122.82956581167414) bank196805 +196806 POINT(38.760483723788404 -121.8194469867186) bank196806 +196807 POINT(37.81320106703481 -123.27447159580748) bank196807 +196808 POINT(38.76395927422897 -122.12746724645982) bank196808 +196809 POINT(37.889490548333036 -122.91044025936306) bank196809 +196810 POINT(37.51705858088229 -122.89199746594538) bank196810 +196811 POINT(38.57624365277612 -121.58174832114024) bank196811 +196812 POINT(38.76512247394791 -123.05649566748052) bank196812 +196813 POINT(37.13462615331011 -123.30513248509787) bank196813 +196814 POINT(37.59381520009867 -122.38510628659554) bank196814 +196815 POINT(37.78938711833089 -122.79931113941032) bank196815 +196816 POINT(37.720220396263315 -121.57327329232663) bank196816 +196817 POINT(38.25491261647464 -122.94406682637374) bank196817 +196818 POINT(38.28902731969174 -123.22103231053947) bank196818 +196819 POINT(37.66881458882413 -123.26470054564471) bank196819 +196820 POINT(37.292431125416165 -122.92199315348108) bank196820 +196821 POINT(37.22648557086255 -122.28548017664492) bank196821 +196822 POINT(37.22163212330875 -123.14250725122146) bank196822 +196823 POINT(38.04376374843671 -121.8604074162385) bank196823 +196824 POINT(38.069131272711935 -121.42698255869045) bank196824 +196825 POINT(38.38206788268314 -122.58304121905968) bank196825 +196826 POINT(37.175622368335524 -122.40286581442828) bank196826 +196827 POINT(38.3589124978438 -121.9430337768429) bank196827 +196828 POINT(37.89054966367411 -123.00071701291178) bank196828 +196829 POINT(38.366532726629806 -122.01248455709586) bank196829 +196830 POINT(37.458343437063505 -122.61712944576341) bank196830 +196831 POINT(37.55851455262496 -121.42275504924757) bank196831 +196832 POINT(36.93252652377988 -122.69102362520212) bank196832 +196833 POINT(37.89572144045671 -121.71676616162135) bank196833 +196834 POINT(37.100165592035566 -122.83989891096547) bank196834 +196835 POINT(37.542624320523416 -121.45342693771548) bank196835 +196836 POINT(37.860271460949356 -122.82000439544858) bank196836 +196837 POINT(38.699293420587786 -122.58612779666674) bank196837 +196838 POINT(36.901426818428746 -122.27645670821089) bank196838 +196839 POINT(38.49706617784725 -123.0679701003177) bank196839 +196840 POINT(38.362442674774925 -122.96877332112868) bank196840 +196841 POINT(37.35136948341756 -122.13767888739352) bank196841 +196842 POINT(38.468522590237264 -123.28370401435993) bank196842 +196843 POINT(37.10069926577434 -122.37224770556318) bank196843 +196844 POINT(37.207015241014 -122.90949432449055) bank196844 +196845 POINT(38.01396290229687 -121.74914458672828) bank196845 +196846 POINT(38.37789131639881 -121.9193112039511) bank196846 +196847 POINT(37.94080575783472 -123.31332648377159) bank196847 +196848 POINT(38.00052704586308 -122.47374991551193) bank196848 +196849 POINT(37.25200060113373 -121.97046944731238) bank196849 +196850 POINT(38.17421651648286 -122.72011912334726) bank196850 +196851 POINT(38.654626598130896 -123.30228652928213) bank196851 +196852 POINT(37.482447737414084 -122.64498875359133) bank196852 +196853 POINT(37.59324306320335 -122.34503696596757) bank196853 +196854 POINT(38.25438861706152 -123.14531542655719) bank196854 +196855 POINT(36.799022997444425 -121.94804680233702) bank196855 +196856 POINT(38.60872644751897 -123.09279536793908) bank196856 +196857 POINT(37.10341361633703 -122.21029594175315) bank196857 +196858 POINT(37.71132452787445 -122.44164336255167) bank196858 +196859 POINT(37.481301910495425 -123.10445623528736) bank196859 +196860 POINT(38.584582867475945 -122.57667771040327) bank196860 +196861 POINT(38.77016034642492 -121.85460798080771) bank196861 +196862 POINT(38.37051996937568 -122.34956860407404) bank196862 +196863 POINT(37.52026974501149 -122.00673365396918) bank196863 +196864 POINT(36.896342986789 -122.45095751383874) bank196864 +196865 POINT(37.18986180261161 -121.89260207626188) bank196865 +196866 POINT(38.4547406949543 -123.023141573371) bank196866 +196867 POINT(37.57885485407267 -121.8439389466263) bank196867 +196868 POINT(37.251962177401325 -122.12439047560711) bank196868 +196869 POINT(38.04942879742165 -121.66092913661709) bank196869 +196870 POINT(37.89205784498484 -122.27793077952924) bank196870 +196871 POINT(37.01389268883883 -122.931600566925) bank196871 +196872 POINT(37.36474867360804 -121.99001615153573) bank196872 +196873 POINT(38.10345478767814 -121.554162681761) bank196873 +196874 POINT(38.35724214997893 -123.22615619423861) bank196874 +196875 POINT(37.460192459918055 -123.4058269769989) bank196875 +196876 POINT(37.53101349775527 -122.39565140282313) bank196876 +196877 POINT(38.344942237714946 -121.52778962034162) bank196877 +196878 POINT(38.34646040365589 -122.745078972704) bank196878 +196879 POINT(37.149872691156226 -122.41477328467876) bank196879 +196880 POINT(38.738334114286864 -122.65864989642583) bank196880 +196881 POINT(37.259400835788476 -123.1902593057227) bank196881 +196882 POINT(37.69961890615953 -123.049358228238) bank196882 +196883 POINT(36.79862993561594 -123.2879925133005) bank196883 +196884 POINT(38.64025790939423 -123.3551568168635) bank196884 +196885 POINT(36.85102937284745 -123.31187955427174) bank196885 +196886 POINT(38.11269969813544 -123.15967002231763) bank196886 +196887 POINT(38.514593529832894 -123.17371610687623) bank196887 +196888 POINT(37.97906653140751 -122.20123199729461) bank196888 +196889 POINT(37.158343649044404 -123.33750924788785) bank196889 +196890 POINT(38.75325970505108 -121.73916649892814) bank196890 +196891 POINT(37.70453725910591 -121.73709233796856) bank196891 +196892 POINT(38.298471061028025 -122.66848719952779) bank196892 +196893 POINT(36.82398747766912 -122.12890360563433) bank196893 +196894 POINT(38.47877975690432 -123.08633137903651) bank196894 +196895 POINT(38.67870718679185 -122.95071021467396) bank196895 +196896 POINT(38.108330088691986 -122.87891984070197) bank196896 +196897 POINT(38.370733792447055 -122.56500204494908) bank196897 +196898 POINT(37.236684940727514 -121.89550746602725) bank196898 +196899 POINT(38.74486149915798 -122.58609272297488) bank196899 +196900 POINT(37.552337695116535 -122.86308630637211) bank196900 +196901 POINT(36.98037434659687 -122.66299882278611) bank196901 +196902 POINT(36.79762283778196 -122.45531852403805) bank196902 +196903 POINT(38.38462294899789 -121.86090723651988) bank196903 +196904 POINT(37.431673800124756 -121.537211359697) bank196904 +196905 POINT(38.312627778776374 -121.55775000499933) bank196905 +196906 POINT(37.60384531760969 -121.48552804864775) bank196906 +196907 POINT(37.068477303085466 -121.68473141250865) bank196907 +196908 POINT(36.78784671009283 -123.25085482341815) bank196908 +196909 POINT(38.572399726111826 -122.3766754098917) bank196909 +196910 POINT(37.04464479111131 -122.00373499874162) bank196910 +196911 POINT(38.721944635041034 -121.57264382345336) bank196911 +196912 POINT(37.79895725381097 -121.51172679657815) bank196912 +196913 POINT(37.19183041244942 -121.81653652070293) bank196913 +196914 POINT(37.86691412767696 -122.4164786130037) bank196914 +196915 POINT(38.54156764606477 -122.02980548020264) bank196915 +196916 POINT(37.13530217710603 -121.68731290041971) bank196916 +196917 POINT(38.248528846062904 -121.65509228016941) bank196917 +196918 POINT(37.17664062754462 -122.37549371147486) bank196918 +196919 POINT(36.80562896206732 -123.18368127308831) bank196919 +196920 POINT(37.39255361648033 -121.78398814917271) bank196920 +196921 POINT(37.100726050196094 -123.13933386512186) bank196921 +196922 POINT(38.42932053717765 -121.70927011313402) bank196922 +196923 POINT(37.17178704072644 -121.87820553434128) bank196923 +196924 POINT(37.70554081510112 -122.39057856586255) bank196924 +196925 POINT(37.0698479150328 -121.69179539406181) bank196925 +196926 POINT(37.69960684771683 -121.58730451150973) bank196926 +196927 POINT(38.34682468479057 -122.955057117259) bank196927 +196928 POINT(37.59301327325495 -122.77491866170823) bank196928 +196929 POINT(37.71563905582979 -122.71079248967268) bank196929 +196930 POINT(38.10000508949469 -122.05554016265684) bank196930 +196931 POINT(38.3570932476996 -121.94047626752064) bank196931 +196932 POINT(37.42000680626862 -121.63308253839968) bank196932 +196933 POINT(37.387471575253194 -121.91554420844841) bank196933 +196934 POINT(37.074921142605525 -121.9631511921668) bank196934 +196935 POINT(37.01807553086983 -121.77768631438614) bank196935 +196936 POINT(38.43858376409199 -123.17803682666246) bank196936 +196937 POINT(38.68676102149003 -121.81941283251635) bank196937 +196938 POINT(37.843855086598765 -122.43321902425905) bank196938 +196939 POINT(38.4247962696528 -123.21081306894523) bank196939 +196940 POINT(36.86833588087653 -122.49698791108247) bank196940 +196941 POINT(37.82972267518638 -121.70421743134146) bank196941 +196942 POINT(38.49912372599655 -122.52810740475906) bank196942 +196943 POINT(38.564985452399576 -123.1592308272832) bank196943 +196944 POINT(38.544549819513435 -121.62996741894678) bank196944 +196945 POINT(38.41470728225881 -121.71664602418225) bank196945 +196946 POINT(38.35432641147631 -122.49469002584381) bank196946 +196947 POINT(38.31303485571856 -122.49789019494776) bank196947 +196948 POINT(37.90779957243587 -122.22094406669045) bank196948 +196949 POINT(38.31146252019539 -121.64232206020604) bank196949 +196950 POINT(36.98749953477701 -122.84649957211545) bank196950 +196951 POINT(37.8784818098893 -123.32079943585373) bank196951 +196952 POINT(38.60756548896463 -121.6761900349891) bank196952 +196953 POINT(38.20515986155933 -122.17862316177703) bank196953 +196954 POINT(38.37230609165608 -122.24628339343678) bank196954 +196955 POINT(37.776963529578794 -123.17227718581518) bank196955 +196956 POINT(37.543304600222925 -122.41350235568446) bank196956 +196957 POINT(36.80438679251314 -122.90878369568605) bank196957 +196958 POINT(38.25231635231601 -122.20453234509128) bank196958 +196959 POINT(38.52453665300207 -122.4030326308779) bank196959 +196960 POINT(38.33822752993697 -123.38971648515948) bank196960 +196961 POINT(38.38875572815061 -121.76056720122016) bank196961 +196962 POINT(38.59259603183981 -122.1312108349237) bank196962 +196963 POINT(38.28566531943159 -122.62819643293989) bank196963 +196964 POINT(37.597823033088126 -122.56338350914864) bank196964 +196965 POINT(37.195081487178854 -122.078496911648) bank196965 +196966 POINT(37.856096598982724 -121.85477330573082) bank196966 +196967 POINT(38.65053995882597 -121.56660657504783) bank196967 +196968 POINT(38.4706722878333 -122.92952058304775) bank196968 +196969 POINT(36.924773044168894 -122.2695776338326) bank196969 +196970 POINT(38.344797488711585 -122.31947258486018) bank196970 +196971 POINT(36.78180906141276 -122.60663148506708) bank196971 +196972 POINT(38.025581937425706 -121.61525720514327) bank196972 +196973 POINT(38.36470600482463 -122.44486127501803) bank196973 +196974 POINT(37.937047021564574 -122.6892311359378) bank196974 +196975 POINT(37.11839399157842 -123.28232069984925) bank196975 +196976 POINT(37.82653359042549 -121.98664639626409) bank196976 +196977 POINT(37.61632295902047 -122.42757717138983) bank196977 +196978 POINT(37.874613428192816 -121.99598755507401) bank196978 +196979 POINT(37.82395554296443 -121.5020842567371) bank196979 +196980 POINT(37.543884793503864 -122.64025941338693) bank196980 +196981 POINT(38.38869141378048 -122.2902927521646) bank196981 +196982 POINT(37.4659858445647 -122.56549482184097) bank196982 +196983 POINT(37.0738196740371 -121.92386467745015) bank196983 +196984 POINT(38.71682461325584 -122.61805071758995) bank196984 +196985 POINT(38.71537314724848 -122.55696742870272) bank196985 +196986 POINT(37.125075500975 -123.216951235171) bank196986 +196987 POINT(38.387205557114925 -121.79984293624669) bank196987 +196988 POINT(37.40324984273561 -122.22959413820732) bank196988 +196989 POINT(38.094119816831565 -123.17365406264526) bank196989 +196990 POINT(37.1804424048733 -122.43481535508289) bank196990 +196991 POINT(38.06184770882618 -122.14106398496018) bank196991 +196992 POINT(38.310421450103135 -121.77506982211692) bank196992 +196993 POINT(38.76465108845367 -122.3818577650226) bank196993 +196994 POINT(38.15488243855761 -122.88436669366392) bank196994 +196995 POINT(37.255659182670044 -123.11494162443726) bank196995 +196996 POINT(37.36701471170785 -121.54634260055388) bank196996 +196997 POINT(37.9500155603217 -121.64766489823249) bank196997 +196998 POINT(38.21949588608587 -122.8721227824659) bank196998 +196999 POINT(37.17474867673771 -121.84280359876851) bank196999 +197000 POINT(37.25800977943692 -121.86214308973409) bank197000 +197001 POINT(36.848590375761006 -123.00892213799773) bank197001 +197002 POINT(38.66927005607412 -122.95824379612718) bank197002 +197003 POINT(37.43254679620773 -122.54713549899326) bank197003 +197004 POINT(38.07584271140678 -122.94887896976066) bank197004 +197005 POINT(38.58214163517343 -122.31599940928231) bank197005 +197006 POINT(38.46309414980746 -122.62530852532853) bank197006 +197007 POINT(38.33387812093151 -121.95764055528466) bank197007 +197008 POINT(36.83608711142399 -122.23954794816197) bank197008 +197009 POINT(38.518616033253394 -122.35417700632902) bank197009 +197010 POINT(36.79009424620652 -122.7255306919979) bank197010 +197011 POINT(38.25103366300491 -123.22712079750917) bank197011 +197012 POINT(37.94304978981449 -121.46938637641527) bank197012 +197013 POINT(37.42120128988033 -121.54612994219173) bank197013 +197014 POINT(37.78733993966418 -121.42183153967932) bank197014 +197015 POINT(38.636866857295644 -122.34237660312368) bank197015 +197016 POINT(36.92638337100559 -123.21492087520845) bank197016 +197017 POINT(38.511234322081144 -122.97993020264853) bank197017 +197018 POINT(38.22361260561203 -121.52469066122745) bank197018 +197019 POINT(38.5813424570693 -121.63584552187544) bank197019 +197020 POINT(38.21434738815161 -122.73762522025498) bank197020 +197021 POINT(38.77201456730541 -121.74484106906286) bank197021 +197022 POINT(37.561999421742186 -122.6537652841748) bank197022 +197023 POINT(37.71021797492702 -122.82734718442231) bank197023 +197024 POINT(38.36282639060362 -123.13280233429127) bank197024 +197025 POINT(37.38131079875804 -121.66718915009338) bank197025 +197026 POINT(38.569590114407156 -122.17542544629049) bank197026 +197027 POINT(36.897879980143074 -122.34305606720791) bank197027 +197028 POINT(37.71976469449104 -122.02239952733186) bank197028 +197029 POINT(37.0450153507261 -123.2451430385211) bank197029 +197030 POINT(38.22550134785501 -122.45239951469622) bank197030 +197031 POINT(37.88999591157383 -122.21641628280489) bank197031 +197032 POINT(37.48960067629949 -123.1372600530825) bank197032 +197033 POINT(37.11858396194801 -122.78245757872043) bank197033 +197034 POINT(37.95004106033256 -121.97653617690177) bank197034 +197035 POINT(37.29535733773506 -123.22827727489606) bank197035 +197036 POINT(37.54678265345775 -121.6214950911865) bank197036 +197037 POINT(38.25085933296629 -122.51469099127019) bank197037 +197038 POINT(37.666585188191206 -123.2250221397289) bank197038 +197039 POINT(37.12440550888142 -123.20889283901225) bank197039 +197040 POINT(36.820295863854746 -122.72307007701944) bank197040 +197041 POINT(36.86130497233533 -122.40585824470223) bank197041 +197042 POINT(38.084568138576934 -122.1085506343826) bank197042 +197043 POINT(37.626934918740936 -122.77732655801987) bank197043 +197044 POINT(37.262710345761704 -121.8297567640822) bank197044 +197045 POINT(38.194951743339296 -123.06094774831351) bank197045 +197046 POINT(37.57022240847339 -122.50481011329715) bank197046 +197047 POINT(37.74856782003478 -123.05742052254195) bank197047 +197048 POINT(37.59602837687982 -123.25307939236791) bank197048 +197049 POINT(37.34400789311756 -122.66506998462631) bank197049 +197050 POINT(37.409451281081004 -122.85033614295129) bank197050 +197051 POINT(37.802561956250095 -122.60078752629664) bank197051 +197052 POINT(38.42420754982604 -123.1390356839997) bank197052 +197053 POINT(37.37760068510255 -121.69289674922834) bank197053 +197054 POINT(38.37797710149302 -121.61094411882394) bank197054 +197055 POINT(37.04031017430101 -122.55454142302445) bank197055 +197056 POINT(37.52799133230406 -123.00989471306126) bank197056 +197057 POINT(37.770842595381154 -122.64422154642688) bank197057 +197058 POINT(37.6385002773111 -123.32990372888969) bank197058 +197059 POINT(37.46287391149864 -123.40816327029609) bank197059 +197060 POINT(37.268293966974824 -123.34896552095398) bank197060 +197061 POINT(36.77532639474986 -121.71421602784031) bank197061 +197062 POINT(38.62394684185823 -122.16167255022081) bank197062 +197063 POINT(37.969635472387395 -122.78349466554057) bank197063 +197064 POINT(37.46962215438995 -123.26826625749307) bank197064 +197065 POINT(38.582299518478465 -122.65943495016107) bank197065 +197066 POINT(37.93997077476607 -121.67060656818002) bank197066 +197067 POINT(37.74616532162268 -122.50941833766544) bank197067 +197068 POINT(37.08400275610689 -123.00932625512884) bank197068 +197069 POINT(38.275141418874654 -121.86871297232784) bank197069 +197070 POINT(36.83739775810406 -122.98518232473688) bank197070 +197071 POINT(37.90755366926 -121.57992273916186) bank197071 +197072 POINT(37.69529763842364 -123.04318618317718) bank197072 +197073 POINT(38.565189452887815 -123.18909517142406) bank197073 +197074 POINT(37.03739724256049 -122.1063692756858) bank197074 +197075 POINT(38.406303614596155 -121.65315857382429) bank197075 +197076 POINT(38.062797664413495 -122.87768009827208) bank197076 +197077 POINT(38.56691688054406 -122.37550505753298) bank197077 +197078 POINT(37.27458300562966 -121.86671456126538) bank197078 +197079 POINT(37.52109833210136 -122.53888978358182) bank197079 +197080 POINT(37.689745638824995 -123.29610697657148) bank197080 +197081 POINT(37.64765933263917 -123.12762299921056) bank197081 +197082 POINT(36.88407447782475 -123.07463006135518) bank197082 +197083 POINT(38.32493042878629 -123.02044673987587) bank197083 +197084 POINT(37.99488566603339 -123.01351776834122) bank197084 +197085 POINT(38.57815644358049 -121.45533546624456) bank197085 +197086 POINT(37.65372966811221 -121.88304890118113) bank197086 +197087 POINT(38.20936438626694 -123.12812382284142) bank197087 +197088 POINT(38.10299061002895 -122.73198670162776) bank197088 +197089 POINT(37.61261747178044 -121.45478817738932) bank197089 +197090 POINT(37.502092433221776 -122.75670024837004) bank197090 +197091 POINT(37.84543830208613 -121.73583688767755) bank197091 +197092 POINT(37.71152396138093 -121.74419715428539) bank197092 +197093 POINT(36.856998700357096 -121.76958718961748) bank197093 +197094 POINT(38.22029621490276 -123.23927494969848) bank197094 +197095 POINT(37.23169417066968 -122.22534888248182) bank197095 +197096 POINT(38.061937767175465 -122.53490736328001) bank197096 +197097 POINT(37.6098226324417 -122.12865360187732) bank197097 +197098 POINT(38.24613277104117 -123.05419753714567) bank197098 +197099 POINT(38.674441460239336 -122.93575139124768) bank197099 +197100 POINT(38.19746188792796 -122.3869978604457) bank197100 +197101 POINT(37.32015944861556 -121.83876923718631) bank197101 +197102 POINT(37.2866743019711 -121.61582572768125) bank197102 +197103 POINT(38.66093949688442 -123.30201944321728) bank197103 +197104 POINT(37.237723296471785 -122.25291774199809) bank197104 +197105 POINT(37.46744596616442 -123.32870609792293) bank197105 +197106 POINT(37.28219093467047 -121.65544811656265) bank197106 +197107 POINT(38.008082618725254 -122.29033295980133) bank197107 +197108 POINT(38.27267011810929 -121.93295133968986) bank197108 +197109 POINT(38.58757451450856 -121.54743668984163) bank197109 +197110 POINT(38.13320328332585 -123.00353033426258) bank197110 +197111 POINT(37.300281700191476 -122.6650189138116) bank197111 +197112 POINT(38.47706557015974 -123.17058029086294) bank197112 +197113 POINT(38.153696129466 -123.22103941483363) bank197113 +197114 POINT(37.22346114355676 -123.32966424442908) bank197114 +197115 POINT(37.8893165819747 -121.51012912422168) bank197115 +197116 POINT(38.554901344658816 -121.76280913354026) bank197116 +197117 POINT(37.87093176881936 -122.24591602860339) bank197117 +197118 POINT(37.68650686789974 -121.70034557761822) bank197118 +197119 POINT(36.88286676860775 -122.43068284801507) bank197119 +197120 POINT(38.493737949703174 -121.82397395849694) bank197120 +197121 POINT(38.29443343947251 -121.71902859192248) bank197121 +197122 POINT(37.520784922354274 -121.8097328954137) bank197122 +197123 POINT(37.73126268452942 -122.71183352695105) bank197123 +197124 POINT(37.684006228811924 -122.11508486813207) bank197124 +197125 POINT(38.0089691488399 -121.69138141493866) bank197125 +197126 POINT(37.24536721576249 -121.5299328480447) bank197126 +197127 POINT(37.638408349527346 -122.67553099276738) bank197127 +197128 POINT(36.82456157194932 -122.15228063059122) bank197128 +197129 POINT(37.98889727155682 -122.72916139225063) bank197129 +197130 POINT(36.96370762820484 -122.39948523820645) bank197130 +197131 POINT(37.7991432271638 -121.71404237307861) bank197131 +197132 POINT(37.23810175286602 -123.16908581393447) bank197132 +197133 POINT(38.3490893018979 -122.303960557013) bank197133 +197134 POINT(38.354180073094795 -122.96577580338254) bank197134 +197135 POINT(36.85237651173705 -121.96442005365812) bank197135 +197136 POINT(37.719461147626454 -122.40104394367131) bank197136 +197137 POINT(38.35749966295359 -121.67278393352616) bank197137 +197138 POINT(36.880285318740576 -121.64385636810307) bank197138 +197139 POINT(38.619924901148856 -122.54525731416356) bank197139 +197140 POINT(37.76335299224702 -122.39767575646825) bank197140 +197141 POINT(38.00316578756294 -121.5080612369332) bank197141 +197142 POINT(38.31599483705429 -121.59449410258331) bank197142 +197143 POINT(37.52671716896553 -123.14241846288986) bank197143 +197144 POINT(37.97231106671571 -121.69812553741448) bank197144 +197145 POINT(37.993434935487656 -121.94157393577512) bank197145 +197146 POINT(38.64015279733464 -122.12099103457419) bank197146 +197147 POINT(37.824375377729424 -123.36529324184531) bank197147 +197148 POINT(37.93968488943242 -122.19508783619935) bank197148 +197149 POINT(37.92005331346843 -122.21733061325088) bank197149 +197150 POINT(38.74622891128779 -121.4988731977951) bank197150 +197151 POINT(38.762321222174705 -122.53907154917259) bank197151 +197152 POINT(38.64768587866419 -121.92822513871724) bank197152 +197153 POINT(38.276786328824485 -121.45281458028333) bank197153 +197154 POINT(38.20434910714956 -122.4772485959673) bank197154 +197155 POINT(37.299163212336694 -122.6827005639031) bank197155 +197156 POINT(38.33279281751501 -121.86797937605832) bank197156 +197157 POINT(38.299397736474056 -122.53041885104561) bank197157 +197158 POINT(36.803644806789684 -123.3984272150756) bank197158 +197159 POINT(37.600511700831866 -122.18806320521695) bank197159 +197160 POINT(37.42545153488881 -121.85598826014161) bank197160 +197161 POINT(37.23794232403196 -121.89615554333642) bank197161 +197162 POINT(38.24751384516378 -121.79605322613908) bank197162 +197163 POINT(38.0072523749929 -123.1617399757965) bank197163 +197164 POINT(36.900470565080745 -121.63056713712116) bank197164 +197165 POINT(37.113275946980764 -122.35014198066035) bank197165 +197166 POINT(38.06077623567306 -121.4418171765472) bank197166 +197167 POINT(36.99901317955618 -121.57256578225115) bank197167 +197168 POINT(37.4458449534919 -122.55138732026775) bank197168 +197169 POINT(38.64165837857327 -122.24761403853083) bank197169 +197170 POINT(37.43404663112121 -122.9103430775552) bank197170 +197171 POINT(38.73774362513616 -121.84048479441167) bank197171 +197172 POINT(37.941464689665054 -122.28753345429753) bank197172 +197173 POINT(37.84536614365775 -123.17028132753524) bank197173 +197174 POINT(38.103150879471805 -122.83850875969584) bank197174 +197175 POINT(37.71595147911232 -123.20423858778183) bank197175 +197176 POINT(38.005270443899235 -122.17456393543755) bank197176 +197177 POINT(38.40987767402835 -121.93456673429763) bank197177 +197178 POINT(37.221187027000234 -122.03516399242102) bank197178 +197179 POINT(38.58166373253992 -122.08792502777109) bank197179 +197180 POINT(36.81972030046393 -122.34188799835566) bank197180 +197181 POINT(37.6252874178967 -122.84203961114801) bank197181 +197182 POINT(37.851278874571314 -122.55824376299076) bank197182 +197183 POINT(38.533891547162796 -122.39830211789996) bank197183 +197184 POINT(38.39474464902115 -123.36139437962237) bank197184 +197185 POINT(38.58234084887243 -122.45689587483409) bank197185 +197186 POINT(37.12207808852355 -123.32481297121471) bank197186 +197187 POINT(36.93060962909603 -122.36620986847726) bank197187 +197188 POINT(38.08063140000646 -122.53286787274307) bank197188 +197189 POINT(37.10575697307929 -122.22205127302384) bank197189 +197190 POINT(37.16411519255339 -121.88248578208075) bank197190 +197191 POINT(38.02931566332224 -122.22983924979417) bank197191 +197192 POINT(37.20558844771456 -122.0874267208133) bank197192 +197193 POINT(37.80016017410626 -122.95382530503495) bank197193 +197194 POINT(37.84894376742948 -122.14264304888738) bank197194 +197195 POINT(37.169952161634825 -122.7439784092042) bank197195 +197196 POINT(37.57158096478019 -121.52173116349549) bank197196 +197197 POINT(37.63949098773254 -123.09550642535243) bank197197 +197198 POINT(37.7649284848444 -123.2671575389278) bank197198 +197199 POINT(38.747643835677906 -121.76517668263753) bank197199 +197200 POINT(37.48179230936593 -121.8121182359792) bank197200 +197201 POINT(38.65873455281209 -122.7959937472839) bank197201 +197202 POINT(37.84074680105044 -122.68889823325213) bank197202 +197203 POINT(38.13899339646522 -122.134021049268) bank197203 +197204 POINT(38.364479714270615 -122.67104481246461) bank197204 +197205 POINT(38.028875284605164 -121.50526539830014) bank197205 +197206 POINT(37.145036255733196 -121.42148658915656) bank197206 +197207 POINT(38.71749924284387 -121.92242876796614) bank197207 +197208 POINT(36.95500198360389 -122.34034806631117) bank197208 +197209 POINT(36.90259214515184 -121.89181646979816) bank197209 +197210 POINT(37.426164874447046 -121.64431608335511) bank197210 +197211 POINT(37.17540154551397 -121.86192582581226) bank197211 +197212 POINT(38.581017727249915 -121.6087307519263) bank197212 +197213 POINT(37.968121519420265 -122.12575457547793) bank197213 +197214 POINT(36.79342408943229 -122.11331704698424) bank197214 +197215 POINT(38.13601330633392 -122.16223776632472) bank197215 +197216 POINT(37.28376706766166 -123.34910843850328) bank197216 +197217 POINT(37.60586830226538 -122.26152296821986) bank197217 +197218 POINT(38.554122526973224 -122.53266684264612) bank197218 +197219 POINT(38.25112738799229 -121.72903125728186) bank197219 +197220 POINT(37.22660365275623 -121.99368083454134) bank197220 +197221 POINT(37.054540951283464 -122.79049398838855) bank197221 +197222 POINT(38.464033104414455 -121.65445205349651) bank197222 +197223 POINT(38.06488580871863 -122.80847321178516) bank197223 +197224 POINT(37.78281161243645 -122.1854630346202) bank197224 +197225 POINT(38.32983966199411 -122.00153618362086) bank197225 +197226 POINT(37.480852932487934 -122.66431636596373) bank197226 +197227 POINT(38.19789808336009 -122.77550192108649) bank197227 +197228 POINT(38.49529360942408 -122.43149527665368) bank197228 +197229 POINT(37.78541586060408 -122.72635415430997) bank197229 +197230 POINT(38.13517735989332 -122.90085755592501) bank197230 +197231 POINT(37.912408538173345 -122.42334831840576) bank197231 +197232 POINT(36.85330204226295 -122.29898924749287) bank197232 +197233 POINT(36.944891478125726 -121.84880785363234) bank197233 +197234 POINT(38.63524560083678 -122.22182866790111) bank197234 +197235 POINT(38.01064004443762 -122.58509894047803) bank197235 +197236 POINT(37.169048251105295 -122.5893708322935) bank197236 +197237 POINT(38.55836656862979 -121.91630986488194) bank197237 +197238 POINT(38.626242921889755 -122.96556585636024) bank197238 +197239 POINT(38.08684590979798 -122.47602054080623) bank197239 +197240 POINT(36.9223379426649 -123.06405550328797) bank197240 +197241 POINT(37.39283247327518 -122.24919043740447) bank197241 +197242 POINT(36.89982211765088 -121.88947659578685) bank197242 +197243 POINT(38.058043100656654 -122.47285943258333) bank197243 +197244 POINT(36.88163358451234 -123.1425308264678) bank197244 +197245 POINT(37.424958673445275 -122.17248936566588) bank197245 +197246 POINT(38.414532822522894 -121.84341521173957) bank197246 +197247 POINT(36.85637420454715 -122.0920015056017) bank197247 +197248 POINT(38.30148249264348 -121.83297278932098) bank197248 +197249 POINT(37.16188568120237 -121.93465351547272) bank197249 +197250 POINT(38.479700159839716 -122.76776150684216) bank197250 +197251 POINT(38.187106640612726 -121.6844199170464) bank197251 +197252 POINT(38.23503268228431 -121.56445520373317) bank197252 +197253 POINT(37.31176269409712 -122.9736174350753) bank197253 +197254 POINT(38.563923826256776 -121.80278763196056) bank197254 +197255 POINT(37.48138351075219 -123.39641064643632) bank197255 +197256 POINT(38.48917461439436 -122.60168047008192) bank197256 +197257 POINT(38.29420933614863 -122.61123739211584) bank197257 +197258 POINT(37.341449996264295 -123.01972918191343) bank197258 +197259 POINT(38.47095116757086 -121.42208618553856) bank197259 +197260 POINT(37.836135040759046 -121.45407909764552) bank197260 +197261 POINT(38.25473001691489 -121.64213554518001) bank197261 +197262 POINT(37.780169505481 -121.68651416981623) bank197262 +197263 POINT(38.58361634125857 -122.23216877235205) bank197263 +197264 POINT(38.75969136911509 -122.81335528066417) bank197264 +197265 POINT(36.91847626587597 -122.55349647212111) bank197265 +197266 POINT(38.21512799204292 -122.88920004344116) bank197266 +197267 POINT(37.31829042596059 -121.67570488210517) bank197267 +197268 POINT(37.137851559225574 -123.11777621204897) bank197268 +197269 POINT(37.866862467027644 -121.94149594923077) bank197269 +197270 POINT(37.735956730959415 -122.02061334640882) bank197270 +197271 POINT(37.640962606092245 -122.25032274767933) bank197271 +197272 POINT(37.09314605920453 -122.53313125834427) bank197272 +197273 POINT(36.794901684487215 -122.271156410466) bank197273 +197274 POINT(36.910276508957644 -122.48992449977759) bank197274 +197275 POINT(38.363496783802546 -122.04823999380943) bank197275 +197276 POINT(37.93530641839224 -121.78466274628967) bank197276 +197277 POINT(38.7200023723872 -122.6091023665177) bank197277 +197278 POINT(37.607106074133085 -121.68347946164364) bank197278 +197279 POINT(36.81977693858863 -121.43492386835406) bank197279 +197280 POINT(37.96617309971608 -123.24971002545439) bank197280 +197281 POINT(37.113011671687545 -122.28048755644377) bank197281 +197282 POINT(37.24636095261472 -123.15254344344822) bank197282 +197283 POINT(38.31396919423862 -122.57480540347993) bank197283 +197284 POINT(37.35821580019614 -122.05951006216341) bank197284 +197285 POINT(37.449595690379795 -121.9139840985366) bank197285 +197286 POINT(38.75311970510784 -122.95486745699385) bank197286 +197287 POINT(38.1004560582295 -121.85085866383687) bank197287 +197288 POINT(38.343316442141194 -123.1529904374005) bank197288 +197289 POINT(38.60378987186029 -121.71273906374266) bank197289 +197290 POINT(38.33331094092951 -121.72543053588068) bank197290 +197291 POINT(38.115558243979414 -121.85931007208812) bank197291 +197292 POINT(37.410160373511424 -122.26375384675474) bank197292 +197293 POINT(38.19378352833713 -121.60093487963975) bank197293 +197294 POINT(37.22174274899229 -121.94071755330573) bank197294 +197295 POINT(37.982652610741304 -122.26107798554072) bank197295 +197296 POINT(37.79228102555042 -121.66177145601509) bank197296 +197297 POINT(37.06818058265094 -122.0889116860741) bank197297 +197298 POINT(37.50634074705373 -122.80107363465295) bank197298 +197299 POINT(37.58356635802747 -122.594358861245) bank197299 +197300 POINT(38.483501371152755 -123.20507326668447) bank197300 +197301 POINT(38.22495168469107 -121.7514746588497) bank197301 +197302 POINT(38.69586669072682 -123.06752519751264) bank197302 +197303 POINT(37.31634739010807 -123.10382123259649) bank197303 +197304 POINT(37.993721294594046 -122.84446142628161) bank197304 +197305 POINT(37.02114350569852 -122.23505114669341) bank197305 +197306 POINT(37.466096133606435 -123.1194656601024) bank197306 +197307 POINT(38.37966888203233 -121.5533795810454) bank197307 +197308 POINT(37.04435977052095 -122.88552253719833) bank197308 +197309 POINT(37.10688299652042 -122.04228892978486) bank197309 +197310 POINT(38.52071282172328 -122.3534493040706) bank197310 +197311 POINT(38.075188069019795 -122.36507516613777) bank197311 +197312 POINT(38.5897203404076 -122.87549042230044) bank197312 +197313 POINT(38.10007232315217 -123.32548165602127) bank197313 +197314 POINT(37.291150650783216 -123.25989497761256) bank197314 +197315 POINT(38.36040198695306 -123.06412171046142) bank197315 +197316 POINT(38.24319926685975 -121.59598434215285) bank197316 +197317 POINT(38.028853107273775 -122.86647457973683) bank197317 +197318 POINT(37.45853745507539 -123.23343900638297) bank197318 +197319 POINT(36.91634925046132 -123.30802157761913) bank197319 +197320 POINT(37.05349547428167 -122.8303498653888) bank197320 +197321 POINT(37.38396006379801 -122.53951153790906) bank197321 +197322 POINT(37.655793309211695 -121.74330603546412) bank197322 +197323 POINT(38.6475185899944 -123.29348327446256) bank197323 +197324 POINT(38.66856819058286 -122.72537993525803) bank197324 +197325 POINT(37.80429846004025 -123.2243744462793) bank197325 +197326 POINT(37.53937096255619 -121.92779114556915) bank197326 +197327 POINT(37.715609266403305 -123.20122105652031) bank197327 +197328 POINT(37.974839755803615 -122.83761734895093) bank197328 +197329 POINT(37.68267325180654 -123.13941566809076) bank197329 +197330 POINT(37.02240916865945 -123.17873554141268) bank197330 +197331 POINT(37.39266310317371 -121.86912581472878) bank197331 +197332 POINT(37.01576504523271 -122.46289846487859) bank197332 +197333 POINT(37.607530102349244 -123.16076244037683) bank197333 +197334 POINT(36.81693118137201 -121.49599912979878) bank197334 +197335 POINT(37.532518373620796 -123.33668396895786) bank197335 +197336 POINT(36.80093659494453 -121.68089826993203) bank197336 +197337 POINT(37.79622045770433 -122.64895978432618) bank197337 +197338 POINT(36.80153125111727 -122.0699662660012) bank197338 +197339 POINT(37.654600845713695 -123.38824390316145) bank197339 +197340 POINT(37.775573331781295 -121.47103396069241) bank197340 +197341 POINT(37.167546821203196 -123.37166516980005) bank197341 +197342 POINT(38.06257245707215 -122.73323585273718) bank197342 +197343 POINT(37.627970450379706 -122.69023059172234) bank197343 +197344 POINT(37.70667242192954 -122.11466483775334) bank197344 +197345 POINT(36.77799831141757 -122.47957187152048) bank197345 +197346 POINT(37.4022060111829 -122.7636681510921) bank197346 +197347 POINT(37.20128257407107 -121.59974012319087) bank197347 +197348 POINT(37.19313065749978 -122.89359679575551) bank197348 +197349 POINT(38.49514513817801 -122.26179216065016) bank197349 +197350 POINT(37.60298413869028 -122.82849931051084) bank197350 +197351 POINT(38.510021114552856 -121.50781877560007) bank197351 +197352 POINT(38.56910760572867 -122.36360250100195) bank197352 +197353 POINT(37.060889299083534 -122.16907630414921) bank197353 +197354 POINT(37.78632758387073 -121.8429129770957) bank197354 +197355 POINT(38.09313654343945 -121.92965480191464) bank197355 +197356 POINT(38.43892760007738 -122.57681088927764) bank197356 +197357 POINT(36.82081461029111 -122.93155623038112) bank197357 +197358 POINT(37.951093992669314 -122.25913778021524) bank197358 +197359 POINT(37.77433263079949 -122.22346860522367) bank197359 +197360 POINT(37.51805016189545 -123.18751112014915) bank197360 +197361 POINT(38.25025151833497 -122.97428379513332) bank197361 +197362 POINT(37.31219717589595 -123.37209870907526) bank197362 +197363 POINT(36.84380355461025 -122.94296905258726) bank197363 +197364 POINT(37.77640741376675 -122.33099260450376) bank197364 +197365 POINT(38.39391092582221 -122.92893970541529) bank197365 +197366 POINT(37.43054537178384 -123.17859786911531) bank197366 +197367 POINT(37.30916524884484 -121.52777560116357) bank197367 +197368 POINT(38.5157849040131 -123.26963944779918) bank197368 +197369 POINT(38.15799423334847 -121.81788999131068) bank197369 +197370 POINT(36.84138902391213 -121.75937777388778) bank197370 +197371 POINT(36.81154512507238 -121.97530065434349) bank197371 +197372 POINT(37.10379169264232 -121.70066910819655) bank197372 +197373 POINT(36.85637453937417 -122.14396454804039) bank197373 +197374 POINT(37.23072605837043 -121.66681786209975) bank197374 +197375 POINT(38.46009213396111 -123.18921454966163) bank197375 +197376 POINT(38.717465284592706 -123.39024570474031) bank197376 +197377 POINT(38.533860430710426 -123.05220159586905) bank197377 +197378 POINT(38.64278761698126 -123.09687647662874) bank197378 +197379 POINT(38.32371661362566 -122.27224848200403) bank197379 +197380 POINT(37.91769923146207 -123.05646358702136) bank197380 +197381 POINT(37.364511361893754 -121.84238395311031) bank197381 +197382 POINT(37.803608521331526 -122.6867831775427) bank197382 +197383 POINT(38.37018672788763 -123.2725255329155) bank197383 +197384 POINT(38.6043398840397 -122.47972038658675) bank197384 +197385 POINT(36.898451836815866 -122.36801298164868) bank197385 +197386 POINT(37.49458411173727 -123.31919453247426) bank197386 +197387 POINT(37.917643392549245 -122.83369367101977) bank197387 +197388 POINT(37.05888449040811 -122.24782931514788) bank197388 +197389 POINT(37.32427017070793 -122.93110510845814) bank197389 +197390 POINT(37.06567011416162 -121.51677849792866) bank197390 +197391 POINT(38.20886096992369 -122.70086177192827) bank197391 +197392 POINT(38.10950840358363 -122.89295709996321) bank197392 +197393 POINT(37.436514324380724 -122.48252296684052) bank197393 +197394 POINT(37.45505063273848 -121.82328128751476) bank197394 +197395 POINT(37.162515915733316 -122.56032692633408) bank197395 +197396 POINT(38.77403856359648 -122.75563201879069) bank197396 +197397 POINT(38.33148238665608 -122.66984668201044) bank197397 +197398 POINT(37.85653557453596 -121.50014380345236) bank197398 +197399 POINT(37.04581898675612 -123.15582647831258) bank197399 +197400 POINT(37.56797836651894 -121.77057975733607) bank197400 +197401 POINT(38.60967756241206 -122.47699853883783) bank197401 +197402 POINT(38.46357810628388 -122.60020218497361) bank197402 +197403 POINT(37.23657019502711 -122.74830639937208) bank197403 +197404 POINT(37.26297117186923 -123.27157696853685) bank197404 +197405 POINT(38.131562018714796 -122.80449947781918) bank197405 +197406 POINT(38.641079951695914 -121.48358886763891) bank197406 +197407 POINT(37.62945971510225 -121.771084940175) bank197407 +197408 POINT(38.55061840958452 -121.84002780889706) bank197408 +197409 POINT(36.84361199498268 -121.4216953125463) bank197409 +197410 POINT(37.60362510388668 -122.22991270588476) bank197410 +197411 POINT(38.31537365915699 -122.38166399621153) bank197411 +197412 POINT(38.717399352407 -121.97150037171292) bank197412 +197413 POINT(38.27442985641354 -121.8378486206856) bank197413 +197414 POINT(37.45172347615413 -121.6434312399068) bank197414 +197415 POINT(37.2854796171649 -122.75277447085736) bank197415 +197416 POINT(38.66422278625285 -122.99941718966895) bank197416 +197417 POINT(38.29814339872777 -122.57073414396578) bank197417 +197418 POINT(38.30701057284494 -122.18071782621101) bank197418 +197419 POINT(36.89253779877295 -121.84683472177824) bank197419 +197420 POINT(37.83106526180329 -122.67055798618236) bank197420 +197421 POINT(37.52779599844632 -122.03249298364898) bank197421 +197422 POINT(38.46530667416039 -122.80623150156698) bank197422 +197423 POINT(37.052021976429835 -121.94386281473432) bank197423 +197424 POINT(38.5532671706583 -122.68290505054858) bank197424 +197425 POINT(37.17211737471444 -122.732426920857) bank197425 +197426 POINT(37.31866316422691 -122.38618494824273) bank197426 +197427 POINT(38.38995361197165 -122.54018495044473) bank197427 +197428 POINT(37.0639971811837 -122.77857064654094) bank197428 +197429 POINT(38.102874569071574 -123.12771497116339) bank197429 +197430 POINT(38.07633146895787 -122.79960989877152) bank197430 +197431 POINT(38.573276201964305 -122.89859227449988) bank197431 +197432 POINT(37.53959300376697 -123.15791569802896) bank197432 +197433 POINT(38.599179069924496 -122.19512042349257) bank197433 +197434 POINT(37.614860257521954 -121.78789637485974) bank197434 +197435 POINT(37.66099542615783 -122.40420722343397) bank197435 +197436 POINT(38.12768615352437 -121.59269885858218) bank197436 +197437 POINT(37.163279900378456 -123.38272544332035) bank197437 +197438 POINT(36.93177332674562 -123.15976432754931) bank197438 +197439 POINT(37.95951466054056 -121.4788007304076) bank197439 +197440 POINT(37.64676778522973 -123.04805736251882) bank197440 +197441 POINT(38.49996299311802 -123.20100198469697) bank197441 +197442 POINT(37.59679670818512 -122.98509059388552) bank197442 +197443 POINT(38.72356084449213 -123.10242531058785) bank197443 +197444 POINT(37.158117145771655 -122.17855566742081) bank197444 +197445 POINT(38.36273945163069 -123.21285366660855) bank197445 +197446 POINT(38.58576973570769 -121.88710411523616) bank197446 +197447 POINT(37.6893101849001 -122.0948104269075) bank197447 +197448 POINT(37.167090568984705 -121.4717181672873) bank197448 +197449 POINT(37.86041028131404 -123.41785237874115) bank197449 +197450 POINT(37.28883984132075 -121.83574800788558) bank197450 +197451 POINT(36.922376115120606 -123.2673306320966) bank197451 +197452 POINT(38.130086924487244 -122.78788470118384) bank197452 +197453 POINT(38.70563787987096 -121.47850038395448) bank197453 +197454 POINT(37.74320146415506 -122.12491733334534) bank197454 +197455 POINT(37.14371202492107 -122.65598119623706) bank197455 +197456 POINT(37.747598047322526 -122.28631341474322) bank197456 +197457 POINT(37.26710242667886 -121.8300492436623) bank197457 +197458 POINT(37.170964466826035 -123.26291135137319) bank197458 +197459 POINT(36.922998310176915 -122.504202794631) bank197459 +197460 POINT(37.535088832094296 -122.09808803176954) bank197460 +197461 POINT(36.828274397464035 -121.79911368803955) bank197461 +197462 POINT(38.48584783398849 -122.16768860360268) bank197462 +197463 POINT(37.44749921906847 -123.33395279325578) bank197463 +197464 POINT(37.219101028030984 -122.70272867158337) bank197464 +197465 POINT(38.39142010104318 -123.29147223090055) bank197465 +197466 POINT(37.04433787993417 -123.277225508321) bank197466 +197467 POINT(38.405844222195455 -122.46595341966552) bank197467 +197468 POINT(36.842283548108995 -121.61559680529805) bank197468 +197469 POINT(38.24122163812786 -123.04729052000516) bank197469 +197470 POINT(37.95474078289965 -121.82149700379951) bank197470 +197471 POINT(37.074162487655215 -123.16239775011648) bank197471 +197472 POINT(37.190042895468395 -121.84262916102654) bank197472 +197473 POINT(38.04477223252299 -122.40599426925877) bank197473 +197474 POINT(38.46641462161944 -122.95749033590536) bank197474 +197475 POINT(37.96961539121042 -123.10052877801954) bank197475 +197476 POINT(37.42399351137158 -122.9758280759911) bank197476 +197477 POINT(38.31049154298326 -121.68623601541921) bank197477 +197478 POINT(38.143259293090175 -123.16025721841903) bank197478 +197479 POINT(37.50004241234703 -122.96046623144856) bank197479 +197480 POINT(38.0172933185208 -121.8489619496752) bank197480 +197481 POINT(37.66770274325392 -122.74021288038726) bank197481 +197482 POINT(38.737480336187154 -121.47950804021475) bank197482 +197483 POINT(37.87432566773006 -122.93293177739253) bank197483 +197484 POINT(37.73588981660912 -122.24982419148614) bank197484 +197485 POINT(36.87349020896082 -123.26140708325897) bank197485 +197486 POINT(37.666611521191356 -122.19291466217183) bank197486 +197487 POINT(37.43530105841415 -121.47037109896476) bank197487 +197488 POINT(37.835147830759325 -122.98973016252971) bank197488 +197489 POINT(38.19175031873581 -123.31793973525268) bank197489 +197490 POINT(38.388723029714214 -121.90687684367421) bank197490 +197491 POINT(36.95307599800179 -121.43026958007138) bank197491 +197492 POINT(37.48894613829062 -121.66838527752529) bank197492 +197493 POINT(36.86793442027511 -122.44710998955513) bank197493 +197494 POINT(37.900750057041684 -123.03389919447663) bank197494 +197495 POINT(37.69506883938682 -122.7089318653371) bank197495 +197496 POINT(38.4714998010061 -121.80410151378454) bank197496 +197497 POINT(38.53045929261406 -122.57313347736246) bank197497 +197498 POINT(37.14337517534507 -121.78344775306563) bank197498 +197499 POINT(37.222395513285576 -123.2103771581548) bank197499 +197500 POINT(38.438553102452765 -121.66994918142805) bank197500 +197501 POINT(38.063349441757076 -122.38086112603935) bank197501 +197502 POINT(38.312861701570604 -122.47328918802299) bank197502 +197503 POINT(37.703550599315776 -122.26390067053241) bank197503 +197504 POINT(38.58026770643724 -122.30322064790955) bank197504 +197505 POINT(37.337054668476576 -123.19131824538934) bank197505 +197506 POINT(37.791240617302606 -122.71207849269273) bank197506 +197507 POINT(37.474945390312996 -123.3001111040768) bank197507 +197508 POINT(37.168609404190796 -121.68252609462868) bank197508 +197509 POINT(36.83752463891323 -122.97641648743885) bank197509 +197510 POINT(37.04999069948633 -122.96539404761525) bank197510 +197511 POINT(38.45951646992164 -123.17572686259844) bank197511 +197512 POINT(37.20096706561829 -122.2308601924356) bank197512 +197513 POINT(38.113243048391546 -121.54108524018334) bank197513 +197514 POINT(37.17566422039486 -123.27239903395589) bank197514 +197515 POINT(38.14596279658894 -122.89025752574392) bank197515 +197516 POINT(37.51395045617203 -121.82249393921964) bank197516 +197517 POINT(38.397415254274584 -122.27129748636007) bank197517 +197518 POINT(37.8394306574921 -122.40440572120775) bank197518 +197519 POINT(37.62199673083368 -122.28852339727354) bank197519 +197520 POINT(37.71993822862824 -122.77691550619491) bank197520 +197521 POINT(37.01157778679662 -123.142950250774) bank197521 +197522 POINT(36.82376858053689 -122.99925068250185) bank197522 +197523 POINT(37.04472619152414 -123.40109587722706) bank197523 +197524 POINT(37.19716180159646 -122.56509844988497) bank197524 +197525 POINT(37.4048881168052 -122.45611113424476) bank197525 +197526 POINT(38.22901586673375 -122.61672370723328) bank197526 +197527 POINT(36.99105120135323 -121.59428465150171) bank197527 +197528 POINT(36.78816956109747 -122.77068952379797) bank197528 +197529 POINT(38.603034373185785 -122.86431342089135) bank197529 +197530 POINT(37.897248427759834 -121.78716327132813) bank197530 +197531 POINT(38.327600794723395 -123.38980401568584) bank197531 +197532 POINT(37.066392765513164 -122.8133945598674) bank197532 +197533 POINT(38.24570158717266 -122.24184796293936) bank197533 +197534 POINT(37.610851825189556 -122.03530498513132) bank197534 +197535 POINT(37.744356971090156 -123.23434851337879) bank197535 +197536 POINT(38.463155856719084 -121.66456745275529) bank197536 +197537 POINT(36.7982073219134 -122.69324584420765) bank197537 +197538 POINT(38.14674749964365 -123.39385098704065) bank197538 +197539 POINT(37.85746085320178 -122.7427344018459) bank197539 +197540 POINT(37.94926261913411 -122.65542019493292) bank197540 +197541 POINT(38.22657875974237 -121.89229627756684) bank197541 +197542 POINT(37.23469667338261 -121.68127462886463) bank197542 +197543 POINT(38.62312811185479 -121.55391988762501) bank197543 +197544 POINT(37.48735348291084 -123.0242328349659) bank197544 +197545 POINT(38.19919787241232 -122.10353116036299) bank197545 +197546 POINT(37.431906642198804 -121.83043841477988) bank197546 +197547 POINT(38.02421978317508 -123.25415387451781) bank197547 +197548 POINT(38.14758199671634 -121.97673414964703) bank197548 +197549 POINT(38.06670476000302 -122.35621463521538) bank197549 +197550 POINT(37.314837215112895 -123.03559340774127) bank197550 +197551 POINT(37.175546592356206 -122.09606314627963) bank197551 +197552 POINT(37.00880409310207 -122.75226138073373) bank197552 +197553 POINT(38.232941604531696 -121.725197561084) bank197553 +197554 POINT(37.05767052554262 -122.39098018634873) bank197554 +197555 POINT(37.74467194762288 -122.26297856473951) bank197555 +197556 POINT(37.612438656587386 -121.98228448380905) bank197556 +197557 POINT(38.245363553944294 -121.5809830089771) bank197557 +197558 POINT(37.472886750546294 -123.07309809354027) bank197558 +197559 POINT(38.6935472717786 -122.98550976188801) bank197559 +197560 POINT(37.02498249083618 -123.07438282868475) bank197560 +197561 POINT(37.014899142139974 -121.4247765852007) bank197561 +197562 POINT(36.819818267821724 -121.8470775242388) bank197562 +197563 POINT(38.015218119572424 -123.11705004412164) bank197563 +197564 POINT(37.117965960396354 -122.47844367394337) bank197564 +197565 POINT(37.83902469535705 -121.99362682479699) bank197565 +197566 POINT(37.45510565807548 -122.84061625087092) bank197566 +197567 POINT(38.55182316566773 -122.5455559505379) bank197567 +197568 POINT(38.25292880846865 -122.06608361926153) bank197568 +197569 POINT(37.149558341656416 -122.5609256768047) bank197569 +197570 POINT(37.42897640383422 -122.08532075243298) bank197570 +197571 POINT(37.812077415988654 -123.27422593155705) bank197571 +197572 POINT(37.16058601917065 -121.94807822767454) bank197572 +197573 POINT(37.805825445152635 -121.65141627933865) bank197573 +197574 POINT(38.108335163808334 -121.7649500756629) bank197574 +197575 POINT(38.725537833405156 -122.66617201435653) bank197575 +197576 POINT(38.74661542288654 -122.23302188744648) bank197576 +197577 POINT(37.92044496951959 -122.39020251194687) bank197577 +197578 POINT(37.10850504042509 -123.21626754022176) bank197578 +197579 POINT(37.862867893951666 -122.63766534934672) bank197579 +197580 POINT(37.47275029822823 -122.73241227870416) bank197580 +197581 POINT(38.60083093630801 -122.76013244991645) bank197581 +197582 POINT(37.16138709485361 -121.62263518796371) bank197582 +197583 POINT(37.00069358044955 -121.92677392839371) bank197583 +197584 POINT(36.937059064876024 -122.62802881849288) bank197584 +197585 POINT(37.09696328677583 -122.93407916510856) bank197585 +197586 POINT(37.64936869015414 -123.03345905943416) bank197586 +197587 POINT(37.529116478136565 -122.24797485163825) bank197587 +197588 POINT(38.383236445565416 -123.32474927025223) bank197588 +197589 POINT(38.52201842386882 -121.99522938573445) bank197589 +197590 POINT(37.125907965689954 -122.18705821988675) bank197590 +197591 POINT(37.78945055626776 -121.638345635415) bank197591 +197592 POINT(37.434172065340746 -123.01849123897092) bank197592 +197593 POINT(37.95782410816953 -122.83098305966658) bank197593 +197594 POINT(37.21617543325384 -123.275646441066) bank197594 +197595 POINT(37.75572543818759 -123.27679509830318) bank197595 +197596 POINT(37.26768172424534 -122.56436507419721) bank197596 +197597 POINT(37.73416229407278 -123.34040644021502) bank197597 +197598 POINT(38.58292356948532 -121.89534209199198) bank197598 +197599 POINT(37.654659811492614 -123.4189531791894) bank197599 +197600 POINT(37.51130426241411 -123.1759370188121) bank197600 +197601 POINT(37.03372847643816 -122.87377478028782) bank197601 +197602 POINT(37.33847695901547 -122.28742051878702) bank197602 +197603 POINT(38.033149822777986 -123.03430767979471) bank197603 +197604 POINT(37.50067610449947 -121.76265430703937) bank197604 +197605 POINT(37.38167896105857 -122.92160301818699) bank197605 +197606 POINT(36.82625878493216 -122.60078461515991) bank197606 +197607 POINT(38.011933981373815 -121.65299058402208) bank197607 +197608 POINT(37.85947314222152 -122.17676472270469) bank197608 +197609 POINT(38.25773993723757 -122.88533814056886) bank197609 +197610 POINT(37.40245455680888 -121.42823051680998) bank197610 +197611 POINT(36.97725660769954 -122.99078384694565) bank197611 +197612 POINT(38.033114155423625 -123.03943894433779) bank197612 +197613 POINT(37.550820883378535 -122.77872982288581) bank197613 +197614 POINT(37.83091648133687 -122.28519417968297) bank197614 +197615 POINT(37.52320634322429 -122.85554126435609) bank197615 +197616 POINT(37.34805130956365 -122.34411012808559) bank197616 +197617 POINT(37.841207891361 -122.25357199024381) bank197617 +197618 POINT(38.40456089545939 -122.3717541079729) bank197618 +197619 POINT(38.129792004871256 -122.19109419628269) bank197619 +197620 POINT(38.50343013266712 -122.12768523151078) bank197620 +197621 POINT(38.50892334020849 -123.01665291886174) bank197621 +197622 POINT(38.56820230502636 -122.49449669311139) bank197622 +197623 POINT(38.24526917983442 -122.79295060357983) bank197623 +197624 POINT(38.75401423007811 -121.43284119040923) bank197624 +197625 POINT(36.998253326595695 -122.73036698221753) bank197625 +197626 POINT(38.05883331702852 -122.01339618746732) bank197626 +197627 POINT(37.405294848888545 -122.95029377222531) bank197627 +197628 POINT(37.26203951336939 -123.34693865153884) bank197628 +197629 POINT(37.729487261807606 -122.74825766371237) bank197629 +197630 POINT(37.77017624344746 -121.45446307289097) bank197630 +197631 POINT(37.59448866794391 -122.4026068241505) bank197631 +197632 POINT(36.85982469309309 -123.15800744856372) bank197632 +197633 POINT(37.1718765349161 -122.75670866939029) bank197633 +197634 POINT(38.73096576259028 -122.05931819361017) bank197634 +197635 POINT(38.28221049354099 -121.78099302409403) bank197635 +197636 POINT(38.170482383896584 -122.51409681523607) bank197636 +197637 POINT(37.71470390288737 -122.46508483960469) bank197637 +197638 POINT(37.25936997577328 -121.60687376859325) bank197638 +197639 POINT(37.63853476844056 -121.6799955239803) bank197639 +197640 POINT(37.686937231190754 -121.44593271911017) bank197640 +197641 POINT(37.162235797314636 -123.38330190770711) bank197641 +197642 POINT(38.35257694072824 -121.46314763119257) bank197642 +197643 POINT(36.79862232275325 -122.42090853845387) bank197643 +197644 POINT(38.76848432449276 -122.24942417592075) bank197644 +197645 POINT(37.966730821065816 -121.77862200152342) bank197645 +197646 POINT(37.46354342226415 -122.3131580779533) bank197646 +197647 POINT(38.14034298573964 -121.42089988619063) bank197647 +197648 POINT(38.45787192025919 -121.70974484497424) bank197648 +197649 POINT(37.210124203328526 -122.86912614409086) bank197649 +197650 POINT(38.24359710336329 -122.28245979347106) bank197650 +197651 POINT(37.27165731122034 -121.98258027101936) bank197651 +197652 POINT(37.24962959188787 -122.2001969618779) bank197652 +197653 POINT(37.32150663445436 -121.92452470159337) bank197653 +197654 POINT(36.88225432326374 -122.74405575497521) bank197654 +197655 POINT(38.15629107012718 -121.4978318138477) bank197655 +197656 POINT(37.508683784047896 -121.4716457247381) bank197656 +197657 POINT(38.481641437583924 -122.27355966208884) bank197657 +197658 POINT(38.24583154603156 -122.21495241651343) bank197658 +197659 POINT(38.39768948911192 -122.19428637409543) bank197659 +197660 POINT(37.99856054141713 -121.55286075624201) bank197660 +197661 POINT(38.013148987626266 -122.78209118770063) bank197661 +197662 POINT(38.22123170434663 -121.90581755933512) bank197662 +197663 POINT(38.504720625206204 -121.76862955174145) bank197663 +197664 POINT(37.975587951711205 -121.90322523986245) bank197664 +197665 POINT(37.492211761225356 -121.78633922392775) bank197665 +197666 POINT(36.80441350988087 -122.25197546170972) bank197666 +197667 POINT(38.16125585900046 -123.4116276448795) bank197667 +197668 POINT(37.79226102398312 -121.4806898062684) bank197668 +197669 POINT(38.54636593223028 -121.99726822335634) bank197669 +197670 POINT(37.01854485862211 -122.33215994274094) bank197670 +197671 POINT(38.69911671495008 -121.55529019122933) bank197671 +197672 POINT(38.4547558035743 -123.09718581860855) bank197672 +197673 POINT(37.07547808690092 -122.56050037844867) bank197673 +197674 POINT(38.55470422509702 -121.66425530841826) bank197674 +197675 POINT(38.69614697909002 -121.56464465608293) bank197675 +197676 POINT(38.11418632266368 -121.44542144026445) bank197676 +197677 POINT(37.215142906946745 -122.49975183692165) bank197677 +197678 POINT(37.28805032389957 -121.76488741485902) bank197678 +197679 POINT(36.94014443220051 -122.81752625488703) bank197679 +197680 POINT(37.41200154702716 -122.16931829496156) bank197680 +197681 POINT(38.64381570110678 -122.9616587693609) bank197681 +197682 POINT(37.651079907580396 -122.42948151595392) bank197682 +197683 POINT(37.883057306821144 -122.14357758167355) bank197683 +197684 POINT(37.34430225760325 -121.50137007972587) bank197684 +197685 POINT(38.23154767394303 -122.7764322628257) bank197685 +197686 POINT(37.34561777155549 -122.47396002115394) bank197686 +197687 POINT(37.3459707712395 -121.97040369287247) bank197687 +197688 POINT(37.11048631637699 -123.01506229757804) bank197688 +197689 POINT(37.47137327580607 -123.36231935529963) bank197689 +197690 POINT(38.527208553461215 -122.39354253628532) bank197690 +197691 POINT(38.48780921088478 -122.68997304430033) bank197691 +197692 POINT(37.25212770204597 -122.57260165784263) bank197692 +197693 POINT(38.4526695977119 -122.9742147193343) bank197693 +197694 POINT(38.07473847475373 -121.72906706758734) bank197694 +197695 POINT(38.63966721780633 -122.47826849057437) bank197695 +197696 POINT(37.8929721209415 -122.26151466494122) bank197696 +197697 POINT(36.803945287663964 -121.65253321431429) bank197697 +197698 POINT(38.49752311980832 -122.77851504223777) bank197698 +197699 POINT(37.166521072454536 -121.67756965323579) bank197699 +197700 POINT(37.91181575588202 -122.62037295191493) bank197700 +197701 POINT(38.41790788571004 -121.59119102675403) bank197701 +197702 POINT(37.66029303033808 -123.20465806258258) bank197702 +197703 POINT(37.43514141165134 -123.11254305723236) bank197703 +197704 POINT(38.504160790371294 -121.78677203725951) bank197704 +197705 POINT(37.65581346420798 -122.8412875839642) bank197705 +197706 POINT(36.91609114045992 -121.752514105093) bank197706 +197707 POINT(37.156405647204885 -122.08341049509615) bank197707 +197708 POINT(37.300659614735004 -122.65026837936827) bank197708 +197709 POINT(38.437056639786 -122.74348290379521) bank197709 +197710 POINT(38.09861217493039 -121.45751684822265) bank197710 +197711 POINT(38.42033140999926 -121.56677229301502) bank197711 +197712 POINT(38.05379180210224 -122.0131002664193) bank197712 +197713 POINT(37.06573582107115 -123.3099613681336) bank197713 +197714 POINT(38.639858496171435 -121.92011020299825) bank197714 +197715 POINT(37.34825090872662 -121.56697075040431) bank197715 +197716 POINT(37.98760168308235 -121.57537164405619) bank197716 +197717 POINT(37.043927227417186 -122.2802892728098) bank197717 +197718 POINT(38.444288166346986 -122.25240432908106) bank197718 +197719 POINT(37.28898325236268 -122.6033520740633) bank197719 +197720 POINT(37.54930170469559 -122.57404038069616) bank197720 +197721 POINT(38.31893163422687 -123.33297835011398) bank197721 +197722 POINT(37.048302461325235 -122.17204086906253) bank197722 +197723 POINT(37.94934601775464 -122.77262299933516) bank197723 +197724 POINT(37.68949416594132 -122.70587225819133) bank197724 +197725 POINT(37.64049852967665 -121.78345523991067) bank197725 +197726 POINT(37.74969680445357 -122.26632689487255) bank197726 +197727 POINT(37.42665789434916 -122.13043314906389) bank197727 +197728 POINT(38.65170970598031 -122.2836942806588) bank197728 +197729 POINT(38.00315327841455 -122.42178582042433) bank197729 +197730 POINT(36.840525674980455 -123.06495123620107) bank197730 +197731 POINT(38.29233186848344 -121.99892155951966) bank197731 +197732 POINT(38.181098536469335 -123.38392765178433) bank197732 +197733 POINT(36.86674921293602 -122.85488522127197) bank197733 +197734 POINT(38.73819958056775 -121.91379631027016) bank197734 +197735 POINT(36.99864664372366 -122.67312692937259) bank197735 +197736 POINT(37.762996228795984 -121.75769938953604) bank197736 +197737 POINT(38.62038571690381 -123.03264316866301) bank197737 +197738 POINT(37.043269999951185 -123.15377121692224) bank197738 +197739 POINT(38.6137776987273 -122.75004232936028) bank197739 +197740 POINT(36.9841199147972 -123.13771360582922) bank197740 +197741 POINT(37.400955795994975 -123.22573952443548) bank197741 +197742 POINT(37.21210235696447 -123.0001881620273) bank197742 +197743 POINT(37.90578982365707 -121.97841869721266) bank197743 +197744 POINT(37.096058355535625 -121.81904716539375) bank197744 +197745 POINT(36.9472399299728 -121.48882582972176) bank197745 +197746 POINT(36.86289254219281 -121.72333423125767) bank197746 +197747 POINT(37.487133218006235 -121.56605839294038) bank197747 +197748 POINT(38.67881995206105 -122.52429739479606) bank197748 +197749 POINT(36.80498544911696 -121.81300186830319) bank197749 +197750 POINT(38.406260031204326 -122.32852599072537) bank197750 +197751 POINT(38.39094828552586 -121.90567011517905) bank197751 +197752 POINT(37.341846096633375 -121.50827263918407) bank197752 +197753 POINT(38.03253236168462 -122.02445270015849) bank197753 +197754 POINT(37.64843241506838 -122.08679669479851) bank197754 +197755 POINT(38.62032240797171 -122.71037171582566) bank197755 +197756 POINT(37.97728041179404 -121.81214747002377) bank197756 +197757 POINT(38.24091846040979 -121.59562412155793) bank197757 +197758 POINT(37.707828043652654 -122.22134203723007) bank197758 +197759 POINT(38.527443099083605 -122.49611488292153) bank197759 +197760 POINT(38.76615546851825 -122.2052636205918) bank197760 +197761 POINT(38.452228386394516 -122.51787982906582) bank197761 +197762 POINT(37.40419899956595 -121.9926639565376) bank197762 +197763 POINT(36.80648755290783 -122.12187245293657) bank197763 +197764 POINT(37.27089937930781 -123.25744540355579) bank197764 +197765 POINT(38.27558635199133 -122.13518947423469) bank197765 +197766 POINT(38.01118038925388 -121.74428802626991) bank197766 +197767 POINT(38.11220801969225 -122.41005533486434) bank197767 +197768 POINT(37.624563041285725 -123.27769880819615) bank197768 +197769 POINT(38.6702411695582 -121.98479638938613) bank197769 +197770 POINT(38.76305578527436 -122.05635039638335) bank197770 +197771 POINT(36.951515169548 -122.36432186608846) bank197771 +197772 POINT(37.7306678133054 -123.30058638170898) bank197772 +197773 POINT(38.72383638526141 -123.33320897909013) bank197773 +197774 POINT(37.53966822900265 -122.33249989159978) bank197774 +197775 POINT(37.29702340728155 -122.61237872018962) bank197775 +197776 POINT(38.108339036481105 -122.16581938676546) bank197776 +197777 POINT(37.4313611485252 -122.53466621225579) bank197777 +197778 POINT(37.32231756935516 -123.37724080795172) bank197778 +197779 POINT(37.858093244395754 -122.08875006454646) bank197779 +197780 POINT(38.66234058248645 -122.28833168851041) bank197780 +197781 POINT(37.11134736934881 -122.72290296274693) bank197781 +197782 POINT(37.20712397997768 -122.4422648769887) bank197782 +197783 POINT(37.679864038575055 -121.82489528006163) bank197783 +197784 POINT(38.17234933478541 -121.89040305269342) bank197784 +197785 POINT(38.70480771465656 -122.33968281905152) bank197785 +197786 POINT(37.842452490505764 -121.85574814379909) bank197786 +197787 POINT(37.11791307120629 -123.09699803428381) bank197787 +197788 POINT(38.69564511003759 -123.39095698849623) bank197788 +197789 POINT(37.56587746463074 -121.65950848549497) bank197789 +197790 POINT(38.388056591172244 -122.00433000081503) bank197790 +197791 POINT(37.194036118353985 -122.71939925641713) bank197791 +197792 POINT(37.14582401067598 -121.5172126586005) bank197792 +197793 POINT(38.26829997182635 -123.17803116137287) bank197793 +197794 POINT(38.29128745311183 -121.56319874502397) bank197794 +197795 POINT(38.732007891537805 -122.41863959584543) bank197795 +197796 POINT(37.77631833224186 -123.05267478998638) bank197796 +197797 POINT(38.24625633598594 -122.52343253310742) bank197797 +197798 POINT(37.637972079057946 -122.76388135159347) bank197798 +197799 POINT(36.834246723606284 -123.31467828047877) bank197799 +197800 POINT(38.10584801025247 -122.64313716203837) bank197800 +197801 POINT(37.03397842929694 -122.79776902688032) bank197801 +197802 POINT(38.66656097751226 -122.5501042777003) bank197802 +197803 POINT(36.878492605414536 -123.02851145631703) bank197803 +197804 POINT(38.73423040172564 -122.06277965720464) bank197804 +197805 POINT(37.663516176901666 -122.94093302028773) bank197805 +197806 POINT(38.268020586769865 -122.2102652792182) bank197806 +197807 POINT(37.26307383734353 -121.88890298639306) bank197807 +197808 POINT(37.456987640985844 -122.23325067822259) bank197808 +197809 POINT(36.83535430645317 -121.93384400910742) bank197809 +197810 POINT(38.051952882227305 -121.83782307807095) bank197810 +197811 POINT(38.08546965086331 -123.33057192217807) bank197811 +197812 POINT(37.94215848359855 -122.52098530429772) bank197812 +197813 POINT(37.23271102887043 -122.72029579180379) bank197813 +197814 POINT(38.3405413103266 -121.54367504382125) bank197814 +197815 POINT(38.16915254129454 -121.78363617525511) bank197815 +197816 POINT(37.73755769352819 -122.72539494026088) bank197816 +197817 POINT(38.546684434843264 -121.64980916988799) bank197817 +197818 POINT(37.07260500709766 -121.75512580293625) bank197818 +197819 POINT(38.44876225231769 -122.47609053511833) bank197819 +197820 POINT(38.03735665674376 -121.83340758288823) bank197820 +197821 POINT(38.13404713316584 -123.21515270185466) bank197821 +197822 POINT(37.88586771109384 -123.24047925656875) bank197822 +197823 POINT(38.28022284284306 -123.26082243782065) bank197823 +197824 POINT(38.41277506492745 -123.1106094943118) bank197824 +197825 POINT(38.45172916541289 -122.34532795998886) bank197825 +197826 POINT(37.82582142173305 -122.02089805507319) bank197826 +197827 POINT(38.71275960790558 -122.41429115984998) bank197827 +197828 POINT(37.58049784174555 -121.9445835374392) bank197828 +197829 POINT(37.009245296019714 -123.3878799903095) bank197829 +197830 POINT(38.510468365740444 -123.07933196804211) bank197830 +197831 POINT(37.44101164809484 -123.13730375004172) bank197831 +197832 POINT(38.44810826886013 -122.44975299285186) bank197832 +197833 POINT(37.54322237761553 -121.44727035726314) bank197833 +197834 POINT(37.3571647647604 -122.46601978323882) bank197834 +197835 POINT(38.54795649378131 -121.4991096758131) bank197835 +197836 POINT(37.2511798039573 -123.11091963445753) bank197836 +197837 POINT(37.528512369008354 -121.54337627747749) bank197837 +197838 POINT(37.23991716051208 -123.22100180661437) bank197838 +197839 POINT(38.349231587535854 -122.75467979629136) bank197839 +197840 POINT(37.546022019157604 -123.34997959992184) bank197840 +197841 POINT(38.40050297087963 -123.09985107615873) bank197841 +197842 POINT(38.04203668362829 -122.3452650827314) bank197842 +197843 POINT(38.68826218206203 -122.12923343007151) bank197843 +197844 POINT(37.33803162467632 -121.73467790017746) bank197844 +197845 POINT(38.22805456126721 -122.62812059747695) bank197845 +197846 POINT(37.26265709862058 -122.32599313503559) bank197846 +197847 POINT(38.64720786774819 -121.82426618611021) bank197847 +197848 POINT(38.6815581451542 -122.47992097228898) bank197848 +197849 POINT(37.52392651039 -121.94464677405605) bank197849 +197850 POINT(37.75174471219744 -123.1951718360762) bank197850 +197851 POINT(38.22958898202258 -121.69757063063312) bank197851 +197852 POINT(38.131480859482 -123.00906586743076) bank197852 +197853 POINT(38.11080643360034 -122.60036113517398) bank197853 +197854 POINT(38.15011452305481 -122.98902719774533) bank197854 +197855 POINT(38.089281848700836 -121.84394117521023) bank197855 +197856 POINT(37.44825680963484 -122.21224752297036) bank197856 +197857 POINT(38.50078594360145 -121.58585334851988) bank197857 +197858 POINT(36.80529513922874 -122.7302956909834) bank197858 +197859 POINT(38.070398452710315 -123.15529755697635) bank197859 +197860 POINT(36.804807638348976 -122.63497236515471) bank197860 +197861 POINT(37.00906102118141 -122.71873541322317) bank197861 +197862 POINT(37.48639630577829 -121.62510753649221) bank197862 +197863 POINT(37.149175275759404 -122.28902695047665) bank197863 +197864 POINT(37.45162770645855 -122.27135486538462) bank197864 +197865 POINT(37.99826845992128 -123.31356182717914) bank197865 +197866 POINT(36.813807775706145 -122.7221597567944) bank197866 +197867 POINT(37.60173353201299 -122.84067599697485) bank197867 +197868 POINT(38.34438756922257 -123.02888551215862) bank197868 +197869 POINT(37.20984996073344 -122.87463798360184) bank197869 +197870 POINT(37.9784464624832 -121.90161725157739) bank197870 +197871 POINT(36.87674307199608 -122.24018886217556) bank197871 +197872 POINT(38.69256648494535 -122.46586399680943) bank197872 +197873 POINT(38.11308596547447 -121.8323155022322) bank197873 +197874 POINT(37.051200895857 -123.27487024986235) bank197874 +197875 POINT(37.530472801260565 -122.86769129501796) bank197875 +197876 POINT(38.02098779604054 -121.49528276940896) bank197876 +197877 POINT(38.421515856664456 -121.48281767213415) bank197877 +197878 POINT(38.288174433378046 -121.64259155252196) bank197878 +197879 POINT(37.098599003769245 -121.88915121589793) bank197879 +197880 POINT(37.80416838783717 -121.57787556305115) bank197880 +197881 POINT(36.81211627232212 -122.65700278291459) bank197881 +197882 POINT(36.897329766190374 -123.31364427018471) bank197882 +197883 POINT(38.169732323898586 -122.50845592147897) bank197883 +197884 POINT(38.50153534874735 -123.33798299489325) bank197884 +197885 POINT(37.48811101635832 -121.66887577982243) bank197885 +197886 POINT(36.87150562310494 -123.39223271006992) bank197886 +197887 POINT(37.363338140874546 -122.40792164558614) bank197887 +197888 POINT(37.53181870781757 -123.19669233407757) bank197888 +197889 POINT(37.60200324346566 -122.49711485791107) bank197889 +197890 POINT(37.7931812442831 -122.21939106663876) bank197890 +197891 POINT(37.606824625167896 -121.84899920100824) bank197891 +197892 POINT(38.33649391520067 -121.8261287408514) bank197892 +197893 POINT(37.257189427176904 -123.41833796831642) bank197893 +197894 POINT(36.817677784599326 -122.35328565097993) bank197894 +197895 POINT(37.03007352273863 -122.14026872222641) bank197895 +197896 POINT(38.51817984623324 -122.31303914135952) bank197896 +197897 POINT(38.45605482137787 -121.48869868038746) bank197897 +197898 POINT(38.50504890908698 -123.17202439591922) bank197898 +197899 POINT(37.73263680671161 -121.95024068995859) bank197899 +197900 POINT(38.43777544731733 -123.19866169710544) bank197900 +197901 POINT(37.78004519004573 -122.73025393994605) bank197901 +197902 POINT(38.07496088465481 -122.44141664804023) bank197902 +197903 POINT(37.67992729205546 -122.30243404953852) bank197903 +197904 POINT(38.1479813642187 -121.99453723815353) bank197904 +197905 POINT(37.05158091064525 -121.85379003137366) bank197905 +197906 POINT(38.252827330126564 -121.75041446859419) bank197906 +197907 POINT(38.54822063552529 -122.17361408182865) bank197907 +197908 POINT(37.11290728745982 -123.37443538141714) bank197908 +197909 POINT(38.62884858569519 -121.9047999459983) bank197909 +197910 POINT(36.89180882216456 -122.69236631975757) bank197910 +197911 POINT(38.54647152514697 -122.42847356003819) bank197911 +197912 POINT(38.52192638390028 -121.61061965780131) bank197912 +197913 POINT(37.93006358982663 -121.89469279145216) bank197913 +197914 POINT(37.40935000345325 -121.91576481915848) bank197914 +197915 POINT(38.62186422863524 -122.43551299992062) bank197915 +197916 POINT(37.427199728084354 -122.87100845058585) bank197916 +197917 POINT(36.98227777686423 -122.98910084895402) bank197917 +197918 POINT(37.13295697367162 -121.51225592959122) bank197918 +197919 POINT(37.73294773407263 -121.43803697994379) bank197919 +197920 POINT(38.433783055054384 -123.20837848997544) bank197920 +197921 POINT(36.85662523490383 -122.81882833408454) bank197921 +197922 POINT(37.30820943338646 -121.98508216157921) bank197922 +197923 POINT(37.67054298851725 -121.42631335642368) bank197923 +197924 POINT(38.503435704335764 -122.39403268539814) bank197924 +197925 POINT(37.55933537602082 -122.6504569056918) bank197925 +197926 POINT(37.8252089948069 -123.04436471176776) bank197926 +197927 POINT(36.83188554253681 -122.07523300546153) bank197927 +197928 POINT(38.201567066490284 -122.62156990252292) bank197928 +197929 POINT(38.098737538904984 -122.02066161600088) bank197929 +197930 POINT(37.91087289875098 -121.74560194488424) bank197930 +197931 POINT(38.60976606631644 -123.01147173864229) bank197931 +197932 POINT(38.056937546203315 -123.00491388719213) bank197932 +197933 POINT(37.548027761047635 -122.09926293180258) bank197933 +197934 POINT(36.959582875050366 -121.64344885531082) bank197934 +197935 POINT(37.610260618407125 -122.83403614979221) bank197935 +197936 POINT(37.58540527585817 -122.5747889381964) bank197936 +197937 POINT(37.29750763886411 -123.19280047442933) bank197937 +197938 POINT(38.57031631542757 -122.794053429061) bank197938 +197939 POINT(37.2184722204209 -122.53662238653764) bank197939 +197940 POINT(38.51498302819537 -122.16715134473841) bank197940 +197941 POINT(36.89282724755988 -122.4824772787754) bank197941 +197942 POINT(36.99543450395241 -121.87462571697748) bank197942 +197943 POINT(38.25912774142421 -122.44159885808894) bank197943 +197944 POINT(37.163825322215224 -121.71901848694206) bank197944 +197945 POINT(38.008629493506525 -122.18262287755009) bank197945 +197946 POINT(37.54809977439154 -122.49404397393724) bank197946 +197947 POINT(38.54337466812015 -121.89007059505299) bank197947 +197948 POINT(38.561156750717004 -121.62244683554465) bank197948 +197949 POINT(37.88924069612295 -122.19958877862851) bank197949 +197950 POINT(37.8818902110952 -123.21478338963148) bank197950 +197951 POINT(37.761901024970626 -122.9481446520911) bank197951 +197952 POINT(38.773545007509696 -123.35599193335081) bank197952 +197953 POINT(37.356363372599766 -122.9059570082669) bank197953 +197954 POINT(37.177366024451736 -123.35670540738403) bank197954 +197955 POINT(37.87824752048452 -122.71164798495175) bank197955 +197956 POINT(38.722956071023496 -122.29322756929656) bank197956 +197957 POINT(38.37228076202906 -123.11368513873082) bank197957 +197958 POINT(37.40213656525295 -122.71936761150896) bank197958 +197959 POINT(38.66054867710264 -121.81058088067728) bank197959 +197960 POINT(37.70458994045938 -122.1515760473167) bank197960 +197961 POINT(38.397444130113556 -122.06763033059768) bank197961 +197962 POINT(37.608215127227744 -121.82580898751031) bank197962 +197963 POINT(36.94193451361931 -122.29446876381337) bank197963 +197964 POINT(37.03087054972562 -122.76867835534944) bank197964 +197965 POINT(37.84997296726602 -123.06456060238538) bank197965 +197966 POINT(38.25201333691658 -122.45019149968101) bank197966 +197967 POINT(37.44053917484622 -122.2646945265036) bank197967 +197968 POINT(37.88023155860069 -121.68333705971216) bank197968 +197969 POINT(37.110971957117734 -122.68241298196784) bank197969 +197970 POINT(37.550949268129116 -122.11414902645677) bank197970 +197971 POINT(38.434095129270766 -122.11936795599252) bank197971 +197972 POINT(38.676817025831056 -122.30185225479113) bank197972 +197973 POINT(37.53318828586165 -122.55641576181588) bank197973 +197974 POINT(37.57092353048392 -122.59364638368014) bank197974 +197975 POINT(38.20520631631632 -122.9372523084097) bank197975 +197976 POINT(38.463984551396344 -123.2593226165962) bank197976 +197977 POINT(38.173919673346276 -122.26750754038821) bank197977 +197978 POINT(36.83437901243534 -122.14148981490301) bank197978 +197979 POINT(38.19963945785316 -121.99493340150119) bank197979 +197980 POINT(37.1583108024939 -122.46389770916632) bank197980 +197981 POINT(37.05025859775523 -122.31577716686277) bank197981 +197982 POINT(37.0412379797403 -123.09793534510149) bank197982 +197983 POINT(37.900142362253746 -121.68203606857197) bank197983 +197984 POINT(37.640173487113906 -122.78161605713466) bank197984 +197985 POINT(38.045657804870345 -123.10399984524892) bank197985 +197986 POINT(38.61064601665764 -121.4400304642208) bank197986 +197987 POINT(37.77362197283192 -121.68334578465141) bank197987 +197988 POINT(36.929608472945894 -121.91143998721724) bank197988 +197989 POINT(38.016617714031405 -121.46735112058805) bank197989 +197990 POINT(38.149797505458096 -122.8742328729904) bank197990 +197991 POINT(38.540844298959314 -121.63565347983969) bank197991 +197992 POINT(37.10335742583397 -122.07283770727201) bank197992 +197993 POINT(36.85659772343755 -123.10513473179961) bank197993 +197994 POINT(37.16509834394833 -122.21862754088464) bank197994 +197995 POINT(36.952754824154496 -123.15613694786526) bank197995 +197996 POINT(37.5225247196934 -121.65996826664241) bank197996 +197997 POINT(37.164402366099864 -123.09205225714457) bank197997 +197998 POINT(37.019788366547786 -122.5684384942663) bank197998 +197999 POINT(38.13593313505195 -123.25352803925932) bank197999 +198000 POINT(37.26008236983981 -121.48567408277417) bank198000 +198001 POINT(37.48711592731607 -122.7180521059762) bank198001 +198002 POINT(38.2222808931278 -122.2386882437964) bank198002 +198003 POINT(38.58080817890611 -122.05854749581155) bank198003 +198004 POINT(38.48323710470831 -121.83964442724847) bank198004 +198005 POINT(38.28095240645649 -122.03996601020368) bank198005 +198006 POINT(38.50699302216687 -122.5154894404618) bank198006 +198007 POINT(37.665286847855455 -122.71504444266523) bank198007 +198008 POINT(38.612656556776116 -122.82307203076934) bank198008 +198009 POINT(38.18716144461036 -121.69729481493148) bank198009 +198010 POINT(38.422430420273166 -122.66307468945301) bank198010 +198011 POINT(37.26610592000619 -123.13006487384365) bank198011 +198012 POINT(37.859479930261806 -122.93330743353808) bank198012 +198013 POINT(37.28471131111614 -123.22942794564392) bank198013 +198014 POINT(38.18594878943324 -121.9227820565039) bank198014 +198015 POINT(38.343435807223464 -123.3985257875619) bank198015 +198016 POINT(38.47810377959661 -122.7144649188855) bank198016 +198017 POINT(37.87442646168852 -121.70427936601368) bank198017 +198018 POINT(38.457963341542964 -122.40400391424117) bank198018 +198019 POINT(38.74043815494671 -122.11608099103613) bank198019 +198020 POINT(37.084144536872046 -123.03596518426785) bank198020 +198021 POINT(38.27056304469622 -122.88705608191476) bank198021 +198022 POINT(37.95032236459152 -122.35570373356862) bank198022 +198023 POINT(37.834816420091464 -123.23915806967004) bank198023 +198024 POINT(38.53793857802203 -122.66757967555104) bank198024 +198025 POINT(37.003054396321986 -122.98085268263891) bank198025 +198026 POINT(37.573616263090024 -121.72584044363356) bank198026 +198027 POINT(37.96361583233051 -121.61005260403866) bank198027 +198028 POINT(38.03575696543403 -122.94773905789621) bank198028 +198029 POINT(38.60868990682799 -122.15394183214421) bank198029 +198030 POINT(36.88950668116482 -123.17888316473876) bank198030 +198031 POINT(37.75900830434358 -122.30499671785404) bank198031 +198032 POINT(38.17270852060273 -122.37984101125149) bank198032 +198033 POINT(37.45004937509392 -122.62823729357443) bank198033 +198034 POINT(38.22129581002951 -122.31912638470916) bank198034 +198035 POINT(37.39687106436441 -121.52591206572149) bank198035 +198036 POINT(37.27397654138945 -121.54702507290355) bank198036 +198037 POINT(38.24617678412846 -122.53165091865505) bank198037 +198038 POINT(36.9251530154589 -121.50210549730903) bank198038 +198039 POINT(38.66994104814283 -122.08659006932545) bank198039 +198040 POINT(37.15227355962037 -122.11520983913552) bank198040 +198041 POINT(38.3498008402734 -122.61013495236176) bank198041 +198042 POINT(37.806180903057076 -121.87048766510574) bank198042 +198043 POINT(38.008628289556526 -122.55218403348309) bank198043 +198044 POINT(37.9346989154268 -123.19291500990967) bank198044 +198045 POINT(36.77834028660105 -121.9975245032867) bank198045 +198046 POINT(38.2069434520786 -122.4377590810671) bank198046 +198047 POINT(37.71990817239631 -121.73194877628663) bank198047 +198048 POINT(36.8569923645392 -122.6002251286667) bank198048 +198049 POINT(37.22272120880007 -122.5401700557185) bank198049 +198050 POINT(37.16315698241013 -123.21903637088207) bank198050 +198051 POINT(37.34941690742727 -122.92605552235541) bank198051 +198052 POINT(37.558732159346704 -122.51131683130339) bank198052 +198053 POINT(37.051836012429575 -123.4088960963885) bank198053 +198054 POINT(37.97836874332977 -123.09467493272265) bank198054 +198055 POINT(36.79422115426982 -121.44361762483717) bank198055 +198056 POINT(37.186713557017974 -122.13488024059306) bank198056 +198057 POINT(37.291571832501134 -123.38496853356583) bank198057 +198058 POINT(36.860039912644744 -122.99597802539044) bank198058 +198059 POINT(37.701041180461814 -121.7541935035018) bank198059 +198060 POINT(37.45794433128348 -121.79884683291165) bank198060 +198061 POINT(38.3470298512355 -122.60604682154825) bank198061 +198062 POINT(37.42538381320159 -121.49486836694547) bank198062 +198063 POINT(36.927964600567556 -121.75390635038659) bank198063 +198064 POINT(38.154407395431924 -121.96122756477294) bank198064 +198065 POINT(38.71736139905217 -121.65707456353246) bank198065 +198066 POINT(37.780622793378 -123.22059021463829) bank198066 +198067 POINT(37.227214024834524 -121.46557088052751) bank198067 +198068 POINT(37.12648167015017 -121.95158106263482) bank198068 +198069 POINT(37.55207625108041 -122.56563879810811) bank198069 +198070 POINT(38.26210561876764 -121.54070057751709) bank198070 +198071 POINT(38.38745678816162 -123.31271033439627) bank198071 +198072 POINT(38.431169223681 -122.45707513947883) bank198072 +198073 POINT(38.28282191753685 -121.53516616763359) bank198073 +198074 POINT(38.18953708704784 -121.88955332505314) bank198074 +198075 POINT(38.25222751074568 -123.18542252639008) bank198075 +198076 POINT(37.03678003218849 -122.38577696575781) bank198076 +198077 POINT(37.852365320638526 -123.1240824737841) bank198077 +198078 POINT(36.94101756986123 -121.74476019897413) bank198078 +198079 POINT(37.463496547519355 -121.43301232776899) bank198079 +198080 POINT(38.458194732498924 -123.14569987223325) bank198080 +198081 POINT(37.322640014553315 -122.98523990898074) bank198081 +198082 POINT(38.195403759096415 -122.75180613777074) bank198082 +198083 POINT(37.03506223567053 -122.82194492145395) bank198083 +198084 POINT(37.12852457914904 -121.46119683097932) bank198084 +198085 POINT(37.47856902116166 -122.54207040300476) bank198085 +198086 POINT(37.530508844680824 -122.13517345308512) bank198086 +198087 POINT(37.179801850119 -122.72758623532656) bank198087 +198088 POINT(38.66061632776734 -122.32520341732291) bank198088 +198089 POINT(38.06358219868801 -122.8444133606138) bank198089 +198090 POINT(38.31861172539882 -122.33788385097931) bank198090 +198091 POINT(38.24917539384644 -121.81521125122956) bank198091 +198092 POINT(37.33135092494142 -122.49256552290811) bank198092 +198093 POINT(37.07567218359046 -123.23373717146482) bank198093 +198094 POINT(38.74229011526444 -122.78733166292989) bank198094 +198095 POINT(36.84256043006703 -123.17368944591853) bank198095 +198096 POINT(38.33611020689987 -123.40980711169158) bank198096 +198097 POINT(38.131459024786096 -123.14037320428739) bank198097 +198098 POINT(37.6698754956235 -123.31256011163305) bank198098 +198099 POINT(38.346676514940036 -121.7536094748362) bank198099 +198100 POINT(38.162860523200656 -122.64047718342292) bank198100 +198101 POINT(38.65048821549519 -121.86243919718191) bank198101 +198102 POINT(37.26169883121637 -123.17806194814641) bank198102 +198103 POINT(37.64265107289307 -121.85071370567152) bank198103 +198104 POINT(38.0924618250302 -122.27121420035552) bank198104 +198105 POINT(38.10597996909524 -123.19469883252177) bank198105 +198106 POINT(38.726460696696776 -121.55304294920732) bank198106 +198107 POINT(38.624488263334456 -121.48309885789267) bank198107 +198108 POINT(37.73799401453383 -121.93367348947478) bank198108 +198109 POINT(37.46409704862076 -121.64507653529793) bank198109 +198110 POINT(36.84830750373554 -123.05178234273978) bank198110 +198111 POINT(38.37400041371939 -122.10090994372258) bank198111 +198112 POINT(38.68898593556521 -122.99603921149219) bank198112 +198113 POINT(38.1033628617536 -122.45642395250897) bank198113 +198114 POINT(38.37072519945239 -122.23698200684002) bank198114 +198115 POINT(37.98757152341008 -122.46070651468641) bank198115 +198116 POINT(38.096254091645946 -123.17996632535785) bank198116 +198117 POINT(38.15864260534809 -123.0856184643669) bank198117 +198118 POINT(37.25804604275143 -123.25962605273762) bank198118 +198119 POINT(38.06278163512376 -121.50221357267998) bank198119 +198120 POINT(36.99476897323024 -122.9570273636591) bank198120 +198121 POINT(38.61829383083838 -121.67999481285915) bank198121 +198122 POINT(37.16210592521758 -121.94662849267041) bank198122 +198123 POINT(37.43466416727688 -121.47127515050639) bank198123 +198124 POINT(38.50475690207465 -122.5830510413487) bank198124 +198125 POINT(38.17341530171865 -121.78524075937162) bank198125 +198126 POINT(37.520214422092785 -122.68110272582737) bank198126 +198127 POINT(38.10274438434139 -122.1328184945761) bank198127 +198128 POINT(36.92574182603641 -123.37633650770495) bank198128 +198129 POINT(36.89906588564129 -122.633215888833) bank198129 +198130 POINT(37.59303017599937 -122.34159508545739) bank198130 +198131 POINT(38.293931389746795 -121.7458674137815) bank198131 +198132 POINT(38.223600386308505 -122.12071988995936) bank198132 +198133 POINT(38.44645054527751 -122.73584322648678) bank198133 +198134 POINT(38.247866245210524 -122.8003499713232) bank198134 +198135 POINT(37.20871789325627 -122.38149415657034) bank198135 +198136 POINT(37.27722758537371 -122.07371007862682) bank198136 +198137 POINT(38.35892567028646 -122.9196695201802) bank198137 +198138 POINT(37.45497406429467 -122.20132363620256) bank198138 +198139 POINT(38.59465443567648 -123.36370019496937) bank198139 +198140 POINT(38.029291560218304 -123.11509511038251) bank198140 +198141 POINT(37.34373951326614 -122.64170877179542) bank198141 +198142 POINT(37.235012527377904 -123.38588402243451) bank198142 +198143 POINT(38.04075344976629 -121.9897541299507) bank198143 +198144 POINT(37.780247195601774 -121.9870213921751) bank198144 +198145 POINT(38.7560316646951 -122.6756747582536) bank198145 +198146 POINT(38.07086924420844 -122.58883360134804) bank198146 +198147 POINT(38.30572241264468 -123.13217091330117) bank198147 +198148 POINT(38.28073078044251 -122.5478958604694) bank198148 +198149 POINT(37.502065258693825 -121.80719751996287) bank198149 +198150 POINT(37.45921944139278 -121.64859805371542) bank198150 +198151 POINT(36.92955541406255 -122.07707551596819) bank198151 +198152 POINT(38.460654784014 -122.29076645802384) bank198152 +198153 POINT(38.32044654713357 -122.5043932303846) bank198153 +198154 POINT(36.96420239217307 -122.8236370789272) bank198154 +198155 POINT(36.99813475180406 -122.68045393712751) bank198155 +198156 POINT(37.988690757274604 -122.27651747620978) bank198156 +198157 POINT(38.63204608181213 -122.04767559106898) bank198157 +198158 POINT(38.74265752291956 -122.05920300565128) bank198158 +198159 POINT(37.44013740202129 -123.11189482113564) bank198159 +198160 POINT(37.65522792620605 -121.74233059779871) bank198160 +198161 POINT(37.83915743478281 -123.30562972781479) bank198161 +198162 POINT(37.58181800263461 -122.96107179294778) bank198162 +198163 POINT(38.20285908318069 -122.32976158953491) bank198163 +198164 POINT(37.30829051145614 -122.3305760003041) bank198164 +198165 POINT(37.71310588517527 -121.53627648232134) bank198165 +198166 POINT(37.39105794881126 -121.67846374459953) bank198166 +198167 POINT(37.775837814653514 -121.95253057500351) bank198167 +198168 POINT(37.36568901155371 -122.63530435553226) bank198168 +198169 POINT(37.67069743596672 -123.14637047430794) bank198169 +198170 POINT(38.08056526996829 -121.8641838089195) bank198170 +198171 POINT(37.952289747814014 -121.76812766145703) bank198171 +198172 POINT(38.440931361972 -121.63493728892806) bank198172 +198173 POINT(38.53393991926417 -122.5283588265578) bank198173 +198174 POINT(37.09187697836851 -123.24794476734915) bank198174 +198175 POINT(36.78883664739711 -122.61529891623874) bank198175 +198176 POINT(36.92001491447991 -122.84671159765914) bank198176 +198177 POINT(37.27188678580677 -121.50961488103104) bank198177 +198178 POINT(37.98810261852475 -122.09127782012334) bank198178 +198179 POINT(38.03749673944825 -121.90111242876286) bank198179 +198180 POINT(38.22519150682209 -121.75825857745127) bank198180 +198181 POINT(38.173565641884295 -122.21813993129182) bank198181 +198182 POINT(37.17579124430569 -122.07337671356777) bank198182 +198183 POINT(36.83884406142403 -122.11671606335855) bank198183 +198184 POINT(37.036652191406354 -123.17363428614547) bank198184 +198185 POINT(38.27960993038549 -121.84498208423868) bank198185 +198186 POINT(37.54536228514065 -122.65612369664184) bank198186 +198187 POINT(37.86140326989084 -122.0024103531326) bank198187 +198188 POINT(38.021538413942906 -122.32671623075261) bank198188 +198189 POINT(36.85708731313732 -121.5245991879234) bank198189 +198190 POINT(38.01345901132567 -121.91919372303018) bank198190 +198191 POINT(37.699139353366625 -121.62888193749731) bank198191 +198192 POINT(37.51297241852524 -123.31347794852385) bank198192 +198193 POINT(37.03571362602717 -123.22792025429621) bank198193 +198194 POINT(36.891699103586056 -121.45099219710048) bank198194 +198195 POINT(38.64755372041816 -122.70056857728663) bank198195 +198196 POINT(38.17686907075472 -122.00196279181448) bank198196 +198197 POINT(37.99871738391037 -123.01651010398727) bank198197 +198198 POINT(37.009466490233955 -122.26481290907991) bank198198 +198199 POINT(37.40589257946226 -123.25291903102246) bank198199 +198200 POINT(37.72660125980191 -123.2601311700448) bank198200 +198201 POINT(36.89772088878474 -123.09583561683596) bank198201 +198202 POINT(37.26446138581432 -122.75506076808547) bank198202 +198203 POINT(38.24648695950854 -123.17126168037315) bank198203 +198204 POINT(38.40046509928197 -121.82144697245127) bank198204 +198205 POINT(37.135771657882515 -121.99501221902672) bank198205 +198206 POINT(38.41390564327511 -122.93609023468764) bank198206 +198207 POINT(37.96370510237963 -122.0042885810852) bank198207 +198208 POINT(36.840034920075176 -122.33922434625376) bank198208 +198209 POINT(37.427026411008676 -121.4838904045035) bank198209 +198210 POINT(36.82812444795075 -122.93152939494708) bank198210 +198211 POINT(37.533052526822466 -122.73652037702671) bank198211 +198212 POINT(37.333346609677456 -122.45157422865371) bank198212 +198213 POINT(36.899315886353605 -121.89600324589688) bank198213 +198214 POINT(37.65005635671853 -121.67138515582988) bank198214 +198215 POINT(37.10669092879663 -123.37508165827619) bank198215 +198216 POINT(37.050629140229816 -121.93463461259574) bank198216 +198217 POINT(37.796027993306865 -122.48832981216646) bank198217 +198218 POINT(37.95020389421329 -123.1960985159574) bank198218 +198219 POINT(38.03457776417592 -123.30541914335635) bank198219 +198220 POINT(38.695333386456866 -122.24668390258103) bank198220 +198221 POINT(37.54714100297914 -122.09093451254583) bank198221 +198222 POINT(37.1440932342529 -123.30664854290727) bank198222 +198223 POINT(37.03077008387136 -123.39405407971763) bank198223 +198224 POINT(37.909755318157956 -121.59214404181478) bank198224 +198225 POINT(37.11410451761051 -121.5972344036157) bank198225 +198226 POINT(37.6235836266888 -123.40832266507788) bank198226 +198227 POINT(38.10375736468019 -122.9493808970671) bank198227 +198228 POINT(37.361277204476465 -121.74149470425546) bank198228 +198229 POINT(37.02142792495917 -123.21347022588239) bank198229 +198230 POINT(37.93126909630109 -121.93431754507672) bank198230 +198231 POINT(37.404419459834344 -122.50799328084618) bank198231 +198232 POINT(38.727335214167574 -122.61112405612019) bank198232 +198233 POINT(38.475093824461396 -122.81256599456903) bank198233 +198234 POINT(38.12143565542087 -123.10292624798058) bank198234 +198235 POINT(38.422407445343794 -122.30322250923376) bank198235 +198236 POINT(37.35229781706659 -121.94033471304049) bank198236 +198237 POINT(37.147434648691394 -122.16354724157375) bank198237 +198238 POINT(37.00242792935647 -123.07010579974674) bank198238 +198239 POINT(37.99196503373081 -122.05041018422423) bank198239 +198240 POINT(37.539856973808526 -122.85824899815309) bank198240 +198241 POINT(37.62689446795931 -122.4112113996717) bank198241 +198242 POINT(38.71087234456831 -121.56701524683753) bank198242 +198243 POINT(38.107072753824234 -123.4020854201687) bank198243 +198244 POINT(38.72217026401708 -123.2121001909848) bank198244 +198245 POINT(37.79381404976235 -122.68481871631758) bank198245 +198246 POINT(38.51991683764332 -122.48738748831356) bank198246 +198247 POINT(36.8501277147122 -121.79558222732362) bank198247 +198248 POINT(37.59919552375972 -122.03338001597675) bank198248 +198249 POINT(38.27955611114204 -122.25350946569417) bank198249 +198250 POINT(38.377953166236615 -122.39076756675219) bank198250 +198251 POINT(37.29898710174576 -121.58366722435456) bank198251 +198252 POINT(38.558419463884974 -123.3815170903036) bank198252 +198253 POINT(38.15727195813011 -123.27444121861313) bank198253 +198254 POINT(37.931036673264764 -121.9579042276769) bank198254 +198255 POINT(37.4628833350277 -123.2511724939975) bank198255 +198256 POINT(37.068344231668455 -123.01401369260249) bank198256 +198257 POINT(37.14245234655649 -122.10384410726913) bank198257 +198258 POINT(37.63686412160435 -123.03993451127572) bank198258 +198259 POINT(36.88295351038894 -122.37867255227722) bank198259 +198260 POINT(37.24531432730297 -123.35552062417608) bank198260 +198261 POINT(37.32712498174167 -121.69459472296786) bank198261 +198262 POINT(38.56881441193098 -123.22487835275297) bank198262 +198263 POINT(38.76438434778037 -122.29143761041148) bank198263 +198264 POINT(38.72073463392487 -122.13520338678475) bank198264 +198265 POINT(37.64179528951853 -121.71231306001134) bank198265 +198266 POINT(38.60118157471881 -123.10295312876816) bank198266 +198267 POINT(38.09461735736119 -121.55111245524178) bank198267 +198268 POINT(38.50156477041588 -122.50704286397234) bank198268 +198269 POINT(37.56487740058623 -122.38216937155462) bank198269 +198270 POINT(36.94891784930004 -122.98300767817925) bank198270 +198271 POINT(38.632897869399386 -122.67198420408828) bank198271 +198272 POINT(37.24601786844769 -123.27340835540527) bank198272 +198273 POINT(38.60956468577567 -121.55779227617403) bank198273 +198274 POINT(38.044571410907025 -121.65754907928513) bank198274 +198275 POINT(37.90421274521615 -122.55496987914583) bank198275 +198276 POINT(38.09549953848295 -122.8317202101696) bank198276 +198277 POINT(36.896472500422064 -122.26069275373676) bank198277 +198278 POINT(37.53622041490984 -121.75108211806092) bank198278 +198279 POINT(37.466842372936824 -121.49140264501044) bank198279 +198280 POINT(37.0849317302823 -121.87543481022833) bank198280 +198281 POINT(38.26514500102655 -121.83137416025745) bank198281 +198282 POINT(37.52180224998524 -121.7776616336377) bank198282 +198283 POINT(38.16372525929612 -121.7258269926523) bank198283 +198284 POINT(38.15142463034409 -122.58355668332332) bank198284 +198285 POINT(38.02925292704762 -122.8099888259891) bank198285 +198286 POINT(37.21383879183151 -122.64220103171539) bank198286 +198287 POINT(37.54233316114001 -121.5985782874059) bank198287 +198288 POINT(38.15164252108428 -122.17405815927584) bank198288 +198289 POINT(38.27608209139256 -122.11803087060579) bank198289 +198290 POINT(38.70194862250108 -121.61608243629475) bank198290 +198291 POINT(36.835714171704126 -122.02541035374502) bank198291 +198292 POINT(38.00181525614737 -121.85979495264736) bank198292 +198293 POINT(37.041124024516826 -122.98730729071761) bank198293 +198294 POINT(38.466563482581236 -123.06296250958175) bank198294 +198295 POINT(37.7565176540615 -121.81019878866859) bank198295 +198296 POINT(37.92197512472203 -121.99159815494015) bank198296 +198297 POINT(37.24531921440512 -121.43368677362714) bank198297 +198298 POINT(37.26933758781921 -122.23955065945069) bank198298 +198299 POINT(37.05767304829859 -122.96092454636926) bank198299 +198300 POINT(37.6555971779873 -122.00595564032949) bank198300 +198301 POINT(38.20408749360272 -122.39143073974269) bank198301 +198302 POINT(38.54670604585819 -122.93219405866918) bank198302 +198303 POINT(38.76337173435374 -122.96563831957027) bank198303 +198304 POINT(38.525534598594675 -122.34948918734284) bank198304 +198305 POINT(38.49763395812753 -122.20219732588919) bank198305 +198306 POINT(38.55006535845378 -121.80637458001895) bank198306 +198307 POINT(38.23868305521011 -122.91913790189157) bank198307 +198308 POINT(38.64712551498229 -122.32648563357549) bank198308 +198309 POINT(37.295509760043906 -121.93106218974106) bank198309 +198310 POINT(38.4949182030599 -123.35856396418906) bank198310 +198311 POINT(37.950225804605594 -122.61654769245308) bank198311 +198312 POINT(37.40645333911409 -121.88496657333108) bank198312 +198313 POINT(36.82781137739149 -121.94653441129627) bank198313 +198314 POINT(37.07666882729517 -122.3581578003993) bank198314 +198315 POINT(38.24551575451742 -121.65985162583351) bank198315 +198316 POINT(36.87129869411545 -121.71572912592057) bank198316 +198317 POINT(38.14196886829204 -122.25344359219875) bank198317 +198318 POINT(38.02922550902932 -121.84711975505662) bank198318 +198319 POINT(38.5242536824441 -121.46035910665825) bank198319 +198320 POINT(38.59614955772887 -122.25104355356575) bank198320 +198321 POINT(36.97976800062396 -122.92251165544792) bank198321 +198322 POINT(36.82692088911719 -123.23549765733806) bank198322 +198323 POINT(37.12621904891421 -122.92578067185931) bank198323 +198324 POINT(38.094340193574574 -122.59451025958595) bank198324 +198325 POINT(38.108743817545815 -122.39195438184112) bank198325 +198326 POINT(37.19127941223308 -121.50660307112554) bank198326 +198327 POINT(37.46561490017471 -122.49039348358703) bank198327 +198328 POINT(37.540603163577735 -121.51173821837696) bank198328 +198329 POINT(38.49518542928289 -122.93133294345043) bank198329 +198330 POINT(36.829534056916415 -122.941362621683) bank198330 +198331 POINT(37.971194777027655 -122.92242375462139) bank198331 +198332 POINT(38.631751269905145 -122.124349461511) bank198332 +198333 POINT(37.98938939723418 -123.10178078696272) bank198333 +198334 POINT(37.288446567820046 -122.71406692008367) bank198334 +198335 POINT(38.04800081417283 -122.42448244599085) bank198335 +198336 POINT(38.33268224998061 -121.76891068883262) bank198336 +198337 POINT(37.156196276627306 -121.8040754226486) bank198337 +198338 POINT(37.722600870080875 -122.84396337024046) bank198338 +198339 POINT(38.44069464148546 -123.07904230531956) bank198339 +198340 POINT(37.80091041912971 -123.0190811769133) bank198340 +198341 POINT(37.45702591645617 -122.60449245277621) bank198341 +198342 POINT(37.521626915544495 -123.39675094902083) bank198342 +198343 POINT(38.240313789894934 -122.04348767891689) bank198343 +198344 POINT(38.73285922085866 -122.38140755623999) bank198344 +198345 POINT(38.47025732642848 -121.9361487308435) bank198345 +198346 POINT(38.556778880046615 -121.70282254109046) bank198346 +198347 POINT(38.50687735071574 -123.1510117394007) bank198347 +198348 POINT(37.537750868044895 -123.26931187848774) bank198348 +198349 POINT(38.49868734145766 -121.71514353864022) bank198349 +198350 POINT(38.34373762836201 -121.60941250144607) bank198350 +198351 POINT(38.52608875083787 -122.70530132740454) bank198351 +198352 POINT(38.394213232377375 -123.19956698778827) bank198352 +198353 POINT(36.80482485456599 -122.21250144668625) bank198353 +198354 POINT(37.871612292767956 -122.68303169741145) bank198354 +198355 POINT(37.48794368424427 -123.23544417493633) bank198355 +198356 POINT(38.5543487237143 -123.20513716050854) bank198356 +198357 POINT(38.17350463360765 -123.0444607283107) bank198357 +198358 POINT(38.42464021773968 -121.73420899637418) bank198358 +198359 POINT(36.77569370456945 -122.84856727196085) bank198359 +198360 POINT(38.14411417399004 -122.17280541310718) bank198360 +198361 POINT(37.22137433935044 -122.79759391882243) bank198361 +198362 POINT(37.86846468217363 -122.98081508806048) bank198362 +198363 POINT(38.00430187935389 -122.81749994146669) bank198363 +198364 POINT(36.892440933284554 -123.16729028537877) bank198364 +198365 POINT(37.69315522097992 -122.24723205084533) bank198365 +198366 POINT(36.9698731497842 -123.28794700582945) bank198366 +198367 POINT(38.04315581601265 -123.40260284692421) bank198367 +198368 POINT(37.96730714264459 -122.28300350869988) bank198368 +198369 POINT(37.59222707239798 -121.52750606237356) bank198369 +198370 POINT(38.33344595921225 -122.18147857560244) bank198370 +198371 POINT(38.56679995260668 -122.29399221948825) bank198371 +198372 POINT(37.49384083093527 -121.94026826715414) bank198372 +198373 POINT(37.734704249337604 -122.30841627570926) bank198373 +198374 POINT(37.42171763027737 -121.45874263168908) bank198374 +198375 POINT(37.79696785544069 -122.13866651359155) bank198375 +198376 POINT(36.967078245924164 -122.24810866249493) bank198376 +198377 POINT(37.29044475470975 -122.6725629991119) bank198377 +198378 POINT(37.5980615373116 -122.27008939744589) bank198378 +198379 POINT(38.25695834901582 -121.55843973458316) bank198379 +198380 POINT(36.89105197548723 -121.7975707177392) bank198380 +198381 POINT(36.84118967881617 -123.23109752411034) bank198381 +198382 POINT(37.61965965541959 -123.30976121120706) bank198382 +198383 POINT(37.31987580694283 -122.56422094851395) bank198383 +198384 POINT(38.27179492527311 -122.7863480438145) bank198384 +198385 POINT(38.54097006964865 -122.54008405006506) bank198385 +198386 POINT(37.184439215556395 -122.39621838766773) bank198386 +198387 POINT(38.21746067331052 -123.24272864205066) bank198387 +198388 POINT(37.16560296132679 -122.33876114565875) bank198388 +198389 POINT(38.09713487358779 -123.41787874201505) bank198389 +198390 POINT(38.69108546552202 -121.46138011931494) bank198390 +198391 POINT(37.35106673076136 -121.91269866974433) bank198391 +198392 POINT(36.88837725697011 -122.59346151437676) bank198392 +198393 POINT(38.66705664691731 -121.55119913077013) bank198393 +198394 POINT(37.442117816457525 -122.43034662718321) bank198394 +198395 POINT(36.805664629027945 -122.67407494109075) bank198395 +198396 POINT(38.55118215681964 -122.90438305878376) bank198396 +198397 POINT(36.947216126498006 -123.3354253251789) bank198397 +198398 POINT(38.77308855652227 -122.1159076999173) bank198398 +198399 POINT(37.708226425445304 -121.65818434343389) bank198399 +198400 POINT(37.76752051850282 -122.8250616320134) bank198400 +198401 POINT(37.142876840586105 -121.53763682673497) bank198401 +198402 POINT(38.70741863961087 -122.0716545354259) bank198402 +198403 POINT(36.804730540391226 -122.71177120035166) bank198403 +198404 POINT(37.37732807278631 -121.50928758554558) bank198404 +198405 POINT(38.153264198593206 -121.80336859004703) bank198405 +198406 POINT(37.05171360069507 -123.14915311131891) bank198406 +198407 POINT(38.31622079531401 -121.6838408763635) bank198407 +198408 POINT(38.49932152153135 -123.17241332386986) bank198408 +198409 POINT(37.01837510268861 -121.53351456316317) bank198409 +198410 POINT(37.3260777236397 -122.05239880372046) bank198410 +198411 POINT(37.36086273972529 -123.41804891221715) bank198411 +198412 POINT(38.774280557758466 -121.43251488054321) bank198412 +198413 POINT(37.977292131974416 -121.61950182378065) bank198413 +198414 POINT(37.66880639570743 -122.53400173957456) bank198414 +198415 POINT(38.46452314145746 -121.57852967811769) bank198415 +198416 POINT(38.13696195720636 -122.20210573097044) bank198416 +198417 POINT(37.88998652902023 -122.0038744010128) bank198417 +198418 POINT(38.531626215191544 -121.4902148126525) bank198418 +198419 POINT(37.010949140005664 -121.51893355960937) bank198419 +198420 POINT(38.70712888418782 -121.72921972589707) bank198420 +198421 POINT(38.09373014538252 -122.76450175609662) bank198421 +198422 POINT(38.25043518106772 -122.44588127414423) bank198422 +198423 POINT(37.12623055494508 -122.76565631944814) bank198423 +198424 POINT(38.58429465747221 -123.02961620485084) bank198424 +198425 POINT(37.530292158995486 -121.87352143703083) bank198425 +198426 POINT(37.21976894516878 -122.43230321992705) bank198426 +198427 POINT(37.263896950680504 -122.0427859198046) bank198427 +198428 POINT(37.202416113178955 -122.98803900385239) bank198428 +198429 POINT(37.86848924583725 -123.08908897305564) bank198429 +198430 POINT(37.16391879669431 -122.9879596686556) bank198430 +198431 POINT(37.270922666017974 -123.33682242929814) bank198431 +198432 POINT(37.79542025795706 -121.49126119525589) bank198432 +198433 POINT(37.89145250538612 -122.85358196385891) bank198433 +198434 POINT(36.793155649012775 -121.6435474995831) bank198434 +198435 POINT(37.13165419121581 -122.24359995252688) bank198435 +198436 POINT(38.725489141978485 -123.28017972799236) bank198436 +198437 POINT(38.520369987671636 -122.1974017444273) bank198437 +198438 POINT(37.551671724226374 -123.00162333253652) bank198438 +198439 POINT(37.49391837734618 -122.01592083740472) bank198439 +198440 POINT(37.32663902569587 -122.64597031261447) bank198440 +198441 POINT(37.315633674540834 -122.62966588021672) bank198441 +198442 POINT(37.63492370802644 -122.29645601177114) bank198442 +198443 POINT(38.465286884407085 -121.64874416670249) bank198443 +198444 POINT(37.110099620384155 -122.19634307364431) bank198444 +198445 POINT(36.89271400981839 -122.43897705257729) bank198445 +198446 POINT(37.878243998773485 -123.20895656239628) bank198446 +198447 POINT(37.09328695451794 -121.82348113221809) bank198447 +198448 POINT(38.54177145657384 -122.16850869669439) bank198448 +198449 POINT(38.58508084987648 -122.84165892400262) bank198449 +198450 POINT(38.73971589481307 -123.30869313011397) bank198450 +198451 POINT(36.93295422111458 -122.13521963643073) bank198451 +198452 POINT(37.21333074360012 -121.73621998967639) bank198452 +198453 POINT(37.054016063047214 -122.28040890429503) bank198453 +198454 POINT(38.104414776498906 -122.01343100009244) bank198454 +198455 POINT(37.27166774235711 -123.23253602527237) bank198455 +198456 POINT(36.914538798662626 -122.13140485913118) bank198456 +198457 POINT(37.23816631538864 -121.94536453037824) bank198457 +198458 POINT(36.947794922449596 -121.43005817502578) bank198458 +198459 POINT(37.341785056713704 -122.97589744019943) bank198459 +198460 POINT(38.51239088299675 -122.4886702339134) bank198460 +198461 POINT(37.524158265969945 -121.52347904733635) bank198461 +198462 POINT(38.13314720916933 -121.93361598054362) bank198462 +198463 POINT(38.732717514477514 -123.31781032387363) bank198463 +198464 POINT(38.169626479546174 -123.1782647848406) bank198464 +198465 POINT(37.3578589390082 -121.99014341806544) bank198465 +198466 POINT(38.759564891458304 -122.17977388955585) bank198466 +198467 POINT(38.26701928973796 -122.18287847914397) bank198467 +198468 POINT(38.07051792506023 -122.09272893719185) bank198468 +198469 POINT(38.752577373211125 -121.76133675794689) bank198469 +198470 POINT(36.89377383373616 -122.08757483546447) bank198470 +198471 POINT(38.44282877507587 -121.76717670189504) bank198471 +198472 POINT(36.840403699498545 -122.3009733785479) bank198472 +198473 POINT(37.23175898318272 -122.04759182159746) bank198473 +198474 POINT(37.90243163600931 -123.40235449151636) bank198474 +198475 POINT(38.539698353135506 -122.13221724727912) bank198475 +198476 POINT(37.94163389944037 -123.04281438252899) bank198476 +198477 POINT(36.990448273648845 -121.50167811720816) bank198477 +198478 POINT(38.32602143316415 -122.3062139256784) bank198478 +198479 POINT(38.55882966256451 -121.82884267574265) bank198479 +198480 POINT(38.31533623253234 -123.40951863139149) bank198480 +198481 POINT(37.11026106544234 -122.81309385211497) bank198481 +198482 POINT(37.588514304436465 -123.3954957583212) bank198482 +198483 POINT(37.366470917457406 -123.15677554755955) bank198483 +198484 POINT(37.48219084406287 -123.39595968528474) bank198484 +198485 POINT(38.012517057371 -121.98475019488265) bank198485 +198486 POINT(37.56311617313093 -121.97218675210564) bank198486 +198487 POINT(38.354018789450905 -123.09834763359817) bank198487 +198488 POINT(37.25230399535947 -123.2725290704734) bank198488 +198489 POINT(38.08497370035396 -122.75133756103423) bank198489 +198490 POINT(37.65574650957019 -121.95391503373412) bank198490 +198491 POINT(38.255146595695294 -121.46005720588435) bank198491 +198492 POINT(38.572617832664235 -123.13348852930788) bank198492 +198493 POINT(36.799731573455716 -122.78988280902152) bank198493 +198494 POINT(37.48825575872589 -122.59825813425354) bank198494 +198495 POINT(37.60026388517119 -122.1872624528857) bank198495 +198496 POINT(37.55866686253171 -123.17045145577545) bank198496 +198497 POINT(37.699190666343114 -122.26722488596617) bank198497 +198498 POINT(37.30593176255761 -122.9332453867145) bank198498 +198499 POINT(37.03248441018594 -121.68240551870882) bank198499 +198500 POINT(37.68401863299554 -123.26557186550613) bank198500 +198501 POINT(37.34596984652625 -123.14849344350223) bank198501 +198502 POINT(36.86260283325581 -121.89639162032634) bank198502 +198503 POINT(37.71433497855503 -122.03846265784944) bank198503 +198504 POINT(37.56880493887877 -123.36894318065858) bank198504 +198505 POINT(38.020249120676354 -123.07065328186802) bank198505 +198506 POINT(36.809000464967355 -122.41611337928744) bank198506 +198507 POINT(37.177850556388535 -122.15094457060974) bank198507 +198508 POINT(37.93299145272189 -122.95529488503136) bank198508 +198509 POINT(38.34893670952507 -122.38921986494269) bank198509 +198510 POINT(37.0979201478909 -121.75390777019116) bank198510 +198511 POINT(37.16471287560973 -121.73632030521937) bank198511 +198512 POINT(37.17800802211629 -122.34013379125851) bank198512 +198513 POINT(37.057768099045184 -121.63703887696319) bank198513 +198514 POINT(37.50716212859497 -123.07916896619223) bank198514 +198515 POINT(37.95903193278476 -123.08416241169226) bank198515 +198516 POINT(37.62487279117912 -122.66924165724052) bank198516 +198517 POINT(38.090343628894566 -123.2402329332547) bank198517 +198518 POINT(37.391933670858585 -122.40832897542278) bank198518 +198519 POINT(37.240061122082096 -121.63093893892268) bank198519 +198520 POINT(37.55796278522565 -122.29034093274069) bank198520 +198521 POINT(38.11947923431471 -123.0857211641538) bank198521 +198522 POINT(37.97160265211339 -122.39656478954348) bank198522 +198523 POINT(37.16186208610422 -121.7255352246856) bank198523 +198524 POINT(38.06176012790898 -121.86688562444216) bank198524 +198525 POINT(37.92041143163073 -123.20363836846781) bank198525 +198526 POINT(36.9954465691856 -122.30351742293436) bank198526 +198527 POINT(37.634044265377305 -123.20030398992914) bank198527 +198528 POINT(37.99026670605231 -122.10897289061637) bank198528 +198529 POINT(38.46419098701241 -122.90531807514245) bank198529 +198530 POINT(37.20976568248544 -123.3252289532844) bank198530 +198531 POINT(37.31509041440682 -123.29766113425994) bank198531 +198532 POINT(37.81931511173402 -122.79348779738456) bank198532 +198533 POINT(38.694790734291985 -123.3169202658637) bank198533 +198534 POINT(37.50012884731946 -121.71171432945233) bank198534 +198535 POINT(37.78648094190384 -121.45881356284437) bank198535 +198536 POINT(37.08874817729916 -122.46589593844811) bank198536 +198537 POINT(37.954543242483 -122.8101248200093) bank198537 +198538 POINT(37.471740678529386 -122.09162730312552) bank198538 +198539 POINT(37.900304249075965 -122.53931323687927) bank198539 +198540 POINT(38.30334075135848 -122.08964726382953) bank198540 +198541 POINT(38.49297254846398 -122.35726285974461) bank198541 +198542 POINT(38.15112481839703 -122.28639225178343) bank198542 +198543 POINT(38.069171240920426 -122.75637149538657) bank198543 +198544 POINT(37.83925045993893 -121.51875260861175) bank198544 +198545 POINT(38.27057183998826 -121.55300742500918) bank198545 +198546 POINT(36.96126231188072 -121.92019123417103) bank198546 +198547 POINT(37.17397109934739 -121.57922627357293) bank198547 +198548 POINT(37.06199572818759 -121.42723850219326) bank198548 +198549 POINT(38.296922836659185 -122.79696477142205) bank198549 +198550 POINT(38.092839738330234 -122.74203178606349) bank198550 +198551 POINT(37.52150544806664 -122.19180911457643) bank198551 +198552 POINT(37.40085652830006 -122.75390941685146) bank198552 +198553 POINT(37.73185036041169 -123.37711645294692) bank198553 +198554 POINT(37.24939649864097 -122.8415506406271) bank198554 +198555 POINT(37.989931770162826 -123.04925705805157) bank198555 +198556 POINT(38.70504813961867 -122.63116738866508) bank198556 +198557 POINT(38.213368009337636 -122.15506367984092) bank198557 +198558 POINT(36.84202481055481 -123.12317167247026) bank198558 +198559 POINT(37.57088216241451 -123.34480126302033) bank198559 +198560 POINT(37.776133100999424 -121.92513909568737) bank198560 +198561 POINT(37.49691664092203 -122.66935181472509) bank198561 +198562 POINT(38.325253841347354 -122.76653095715203) bank198562 +198563 POINT(37.88081489285885 -122.61186819780227) bank198563 +198564 POINT(37.678994836976045 -122.23854292215937) bank198564 +198565 POINT(38.552701933398865 -122.15442193395181) bank198565 +198566 POINT(37.333495203152 -121.48610063312607) bank198566 +198567 POINT(38.32713684682565 -122.21347845309748) bank198567 +198568 POINT(37.67962573575617 -122.25443691864714) bank198568 +198569 POINT(37.469578710000434 -122.86526117096815) bank198569 +198570 POINT(37.43824917292005 -122.33539863529401) bank198570 +198571 POINT(36.877636786524924 -122.25928989041961) bank198571 +198572 POINT(37.45439518558341 -123.11405815805156) bank198572 +198573 POINT(38.12973051975872 -123.22627403168329) bank198573 +198574 POINT(37.85665788156438 -123.23745023456158) bank198574 +198575 POINT(37.48253183225182 -122.26723923063115) bank198575 +198576 POINT(38.24610064840669 -123.28886363999652) bank198576 +198577 POINT(37.7577733233066 -121.91993383625724) bank198577 +198578 POINT(37.622931986072025 -123.13249838541404) bank198578 +198579 POINT(38.42247960362153 -121.45808380650723) bank198579 +198580 POINT(37.94125502629838 -123.02025046358307) bank198580 +198581 POINT(37.87200446241964 -122.26654215036298) bank198581 +198582 POINT(37.12561577965037 -121.95654953507061) bank198582 +198583 POINT(37.66435089670686 -121.64619710328634) bank198583 +198584 POINT(38.40969146962851 -122.0828246933447) bank198584 +198585 POINT(37.089221414130215 -121.54653091738575) bank198585 +198586 POINT(37.60479920694963 -122.73274613658364) bank198586 +198587 POINT(37.74466227893877 -121.89002868342159) bank198587 +198588 POINT(38.60470994347191 -121.67181660788536) bank198588 +198589 POINT(37.21391056210733 -122.18244781175386) bank198589 +198590 POINT(36.870722486640716 -122.91039829014439) bank198590 +198591 POINT(37.22686261449767 -123.0005443168854) bank198591 +198592 POINT(37.00724437416983 -121.8810301665828) bank198592 +198593 POINT(38.580661380324585 -121.44399545126005) bank198593 +198594 POINT(38.49357321860937 -122.59663746339804) bank198594 +198595 POINT(37.835776809336885 -123.33581466068152) bank198595 +198596 POINT(37.50381872034513 -123.00056317196182) bank198596 +198597 POINT(37.7655441747759 -122.17170532049802) bank198597 +198598 POINT(38.60057247182531 -121.79480534690023) bank198598 +198599 POINT(37.33399009114197 -122.66372034612786) bank198599 +198600 POINT(38.05148319274558 -122.84101434921851) bank198600 +198601 POINT(37.056720631155294 -122.84768422458896) bank198601 +198602 POINT(38.348282049794115 -121.52492654263887) bank198602 +198603 POINT(38.68301839896001 -121.89704388760931) bank198603 +198604 POINT(37.91880060488691 -121.95032893317982) bank198604 +198605 POINT(37.45106942074388 -123.39239980325469) bank198605 +198606 POINT(38.54055328056001 -122.0340860438243) bank198606 +198607 POINT(37.58760105088677 -122.52980670681535) bank198607 +198608 POINT(38.00389782135068 -123.37534290654204) bank198608 +198609 POINT(38.17596568359864 -121.70068121070707) bank198609 +198610 POINT(37.0360628111615 -122.15184729455119) bank198610 +198611 POINT(37.882815894029896 -121.60435110626067) bank198611 +198612 POINT(38.55183486295382 -123.1584848469294) bank198612 +198613 POINT(37.90415929078516 -122.31820593746423) bank198613 +198614 POINT(36.821799357442536 -122.71793507938595) bank198614 +198615 POINT(38.53843500816272 -121.80706924096275) bank198615 +198616 POINT(36.896283210914085 -123.21672178946787) bank198616 +198617 POINT(37.58083808500058 -121.87221431333055) bank198617 +198618 POINT(38.3572293629404 -121.73541529922994) bank198618 +198619 POINT(37.32131751374595 -123.28134179417904) bank198619 +198620 POINT(38.08458654662286 -122.53299507748838) bank198620 +198621 POINT(38.39965743687825 -123.22500145890385) bank198621 +198622 POINT(37.08025222805293 -122.29949359832855) bank198622 +198623 POINT(36.81932398561397 -122.93796192438494) bank198623 +198624 POINT(38.13581069706706 -121.56412999560779) bank198624 +198625 POINT(37.1810266761502 -122.35114026075993) bank198625 +198626 POINT(38.10309676333106 -122.5622818581183) bank198626 +198627 POINT(37.17285540174036 -123.14064843136921) bank198627 +198628 POINT(37.2104347497109 -121.5924985979806) bank198628 +198629 POINT(38.4309421026331 -122.49444180099765) bank198629 +198630 POINT(37.62406471608149 -122.55495167246914) bank198630 +198631 POINT(37.95703326995107 -122.00272368285607) bank198631 +198632 POINT(38.18261286638013 -123.14993014018744) bank198632 +198633 POINT(38.71677347409832 -122.0017378542826) bank198633 +198634 POINT(38.10842751271672 -123.28369296620644) bank198634 +198635 POINT(37.226433549183525 -121.69520696866118) bank198635 +198636 POINT(38.42487640515496 -122.842817334465) bank198636 +198637 POINT(38.165589480258454 -121.47021520112847) bank198637 +198638 POINT(37.74449801817136 -122.1299709056748) bank198638 +198639 POINT(38.08338836328304 -122.6236189728989) bank198639 +198640 POINT(37.89448251568831 -121.81917240983907) bank198640 +198641 POINT(38.23144447558124 -122.91536365632288) bank198641 +198642 POINT(37.068880923197526 -121.43623190489697) bank198642 +198643 POINT(37.00540724430626 -123.1382602117695) bank198643 +198644 POINT(37.47367924144545 -123.03233865407411) bank198644 +198645 POINT(37.6061073820026 -122.20602030444722) bank198645 +198646 POINT(38.65379332640185 -122.61837615768144) bank198646 +198647 POINT(37.18790241899332 -123.00977456746924) bank198647 +198648 POINT(37.917726111782294 -122.72044790296366) bank198648 +198649 POINT(38.11569244070966 -122.04396413787644) bank198649 +198650 POINT(37.62564711312156 -122.13663472109793) bank198650 +198651 POINT(37.72232121304998 -121.50991737164381) bank198651 +198652 POINT(38.07933931970284 -121.486482554664) bank198652 +198653 POINT(36.89911661931533 -122.1602581613476) bank198653 +198654 POINT(37.56695723376323 -121.44521175410203) bank198654 +198655 POINT(37.58891330373653 -122.55394950344754) bank198655 +198656 POINT(38.203912135080046 -121.76658702347233) bank198656 +198657 POINT(37.733422141913174 -123.37161875980945) bank198657 +198658 POINT(37.05715478809878 -122.12318362651592) bank198658 +198659 POINT(36.873171700954536 -121.98568984273433) bank198659 +198660 POINT(37.18474998617761 -121.71414674850743) bank198660 +198661 POINT(37.73328017673629 -123.40028899741355) bank198661 +198662 POINT(37.54427536426692 -122.02354767196789) bank198662 +198663 POINT(38.15937314212041 -122.11423649347913) bank198663 +198664 POINT(38.45219673607426 -122.38792630440167) bank198664 +198665 POINT(36.99088036644045 -123.24484504689437) bank198665 +198666 POINT(37.75426335855197 -122.04035743663371) bank198666 +198667 POINT(38.377128509091605 -121.86013329624693) bank198667 +198668 POINT(38.09389228627836 -121.46348275852118) bank198668 +198669 POINT(37.39984664256788 -122.43728232105742) bank198669 +198670 POINT(37.19918896296853 -121.88295683520043) bank198670 +198671 POINT(37.9212624181777 -121.9516048006878) bank198671 +198672 POINT(37.802377528957784 -123.2545619432775) bank198672 +198673 POINT(38.02124745901896 -123.03746699713737) bank198673 +198674 POINT(37.84517036259041 -122.9329776045397) bank198674 +198675 POINT(37.73848633925289 -121.92383834580048) bank198675 +198676 POINT(37.65973832651517 -122.45467235109368) bank198676 +198677 POINT(37.739766693327724 -121.58323670029755) bank198677 +198678 POINT(38.15624052172639 -123.41567976118553) bank198678 +198679 POINT(37.282092183778495 -122.88777845866284) bank198679 +198680 POINT(38.61489226040993 -121.53587351367818) bank198680 +198681 POINT(36.95930076017379 -122.12504397222594) bank198681 +198682 POINT(37.25855685859268 -123.25486284652376) bank198682 +198683 POINT(38.387005417751084 -122.71344576850552) bank198683 +198684 POINT(37.56331836249972 -122.02828920907913) bank198684 +198685 POINT(38.68669970995611 -122.6558625996968) bank198685 +198686 POINT(37.27182796552771 -123.10470386976108) bank198686 +198687 POINT(37.16355712520274 -123.22261658104371) bank198687 +198688 POINT(36.87837164075265 -121.4461538902885) bank198688 +198689 POINT(36.827610425538325 -121.61574651990216) bank198689 +198690 POINT(38.636506844973255 -121.72124372669006) bank198690 +198691 POINT(37.85477961330345 -121.93658427676785) bank198691 +198692 POINT(37.83039218988204 -123.15538155899176) bank198692 +198693 POINT(37.97167627371758 -122.98176333811318) bank198693 +198694 POINT(36.94358392290666 -123.16194800875294) bank198694 +198695 POINT(37.278954854630506 -122.41217652829471) bank198695 +198696 POINT(38.42542869093066 -121.85152881190244) bank198696 +198697 POINT(37.04958865267592 -123.27599515566247) bank198697 +198698 POINT(38.75037234866812 -122.11088440876846) bank198698 +198699 POINT(37.170823218017986 -122.89329099328799) bank198699 +198700 POINT(38.18716087105065 -121.46547551997085) bank198700 +198701 POINT(38.73557903212769 -123.07254544786706) bank198701 +198702 POINT(36.926927313287266 -122.71165609069008) bank198702 +198703 POINT(38.707656358091306 -122.93875070448631) bank198703 +198704 POINT(37.84656433579413 -121.5045882147188) bank198704 +198705 POINT(38.315194702876646 -122.9354403702956) bank198705 +198706 POINT(38.73893531158078 -123.04775500177122) bank198706 +198707 POINT(38.38400709843574 -122.49480067310222) bank198707 +198708 POINT(37.14299698770102 -121.42940062902036) bank198708 +198709 POINT(38.39987589083785 -123.37515594896934) bank198709 +198710 POINT(37.923802486608814 -123.32312051199249) bank198710 +198711 POINT(37.720378254115914 -122.30648765283226) bank198711 +198712 POINT(36.926627589827746 -123.10443184032779) bank198712 +198713 POINT(37.9433454795723 -122.06057665653437) bank198713 +198714 POINT(38.18910035921643 -122.66070489808664) bank198714 +198715 POINT(38.46928141192108 -121.8854637883628) bank198715 +198716 POINT(36.817138761357185 -122.9754352134848) bank198716 +198717 POINT(37.11883191573314 -122.86994411850732) bank198717 +198718 POINT(37.059565222444014 -121.7335495910743) bank198718 +198719 POINT(36.84461767678895 -122.19109450111782) bank198719 +198720 POINT(38.332515994091445 -122.44636127837782) bank198720 +198721 POINT(38.3413061687177 -122.02141544717034) bank198721 +198722 POINT(38.60032826898304 -123.17915675740875) bank198722 +198723 POINT(37.82400266955037 -122.77294714929837) bank198723 +198724 POINT(36.890770978717676 -121.47929687808926) bank198724 +198725 POINT(37.78920005091936 -122.78631984349649) bank198725 +198726 POINT(37.392408067107674 -121.70070622096175) bank198726 +198727 POINT(36.97753301141299 -121.65129563687005) bank198727 +198728 POINT(38.35039735026351 -123.00357632985252) bank198728 +198729 POINT(36.852767615432434 -122.05073056892566) bank198729 +198730 POINT(38.18184744560206 -121.52762560989557) bank198730 +198731 POINT(37.79918297449804 -122.48926603175119) bank198731 +198732 POINT(37.32616389284804 -123.39360465209785) bank198732 +198733 POINT(36.90292436206667 -122.73078658899175) bank198733 +198734 POINT(37.276286209508804 -122.78659060894562) bank198734 +198735 POINT(37.6362956180588 -122.09063413631294) bank198735 +198736 POINT(38.61611881134537 -123.01550545434199) bank198736 +198737 POINT(37.591972906908794 -121.5873653968926) bank198737 +198738 POINT(36.794203284192044 -122.82874961277263) bank198738 +198739 POINT(37.896988171440896 -122.85619634482325) bank198739 +198740 POINT(36.9122656571582 -123.23069318994845) bank198740 +198741 POINT(38.491197165771815 -121.44679961485379) bank198741 +198742 POINT(37.33709764117354 -121.69813819470548) bank198742 +198743 POINT(37.69710764825587 -121.55441502799817) bank198743 +198744 POINT(38.05932720426784 -121.92816143246694) bank198744 +198745 POINT(36.965366111677895 -121.44877352314366) bank198745 +198746 POINT(37.070300162421844 -123.11153143479686) bank198746 +198747 POINT(38.16345752993777 -122.34789802656198) bank198747 +198748 POINT(36.77658392979851 -123.02426818084842) bank198748 +198749 POINT(37.29565801652655 -121.6043447103152) bank198749 +198750 POINT(36.827094793585935 -122.5111405685589) bank198750 +198751 POINT(38.28561784994733 -121.7235628941875) bank198751 +198752 POINT(37.77977395476687 -122.11690358320492) bank198752 +198753 POINT(37.35278535806908 -123.02472059579353) bank198753 +198754 POINT(37.00523291375565 -122.13401723786808) bank198754 +198755 POINT(38.20803710920787 -121.82947996138896) bank198755 +198756 POINT(37.65968163640213 -122.06865867574703) bank198756 +198757 POINT(37.2699149842201 -121.56813548332508) bank198757 +198758 POINT(38.441327493335784 -122.43632527346841) bank198758 +198759 POINT(38.61919290340656 -122.30048408277264) bank198759 +198760 POINT(38.76358175038559 -121.92834295197429) bank198760 +198761 POINT(37.92362681053546 -122.05383796205923) bank198761 +198762 POINT(37.272823055942695 -121.5858184204426) bank198762 +198763 POINT(38.503586346611186 -122.14279902777452) bank198763 +198764 POINT(37.46208848183139 -123.2094765863639) bank198764 +198765 POINT(38.405231163112056 -123.05515729202946) bank198765 +198766 POINT(37.144339637242595 -121.52181822400883) bank198766 +198767 POINT(38.634238388893195 -121.45083452529097) bank198767 +198768 POINT(37.78168083941577 -122.46193360475543) bank198768 +198769 POINT(37.5060972990469 -122.76364918505503) bank198769 +198770 POINT(37.46731783224468 -121.66215725730623) bank198770 +198771 POINT(37.90201242460223 -122.87821037819897) bank198771 +198772 POINT(38.69630802000236 -122.5671261479128) bank198772 +198773 POINT(37.23785259393277 -123.12706512858975) bank198773 +198774 POINT(37.10684906308065 -123.10435797301584) bank198774 +198775 POINT(37.05182016655989 -123.28417226578398) bank198775 +198776 POINT(36.90622728821816 -121.7189161319973) bank198776 +198777 POINT(38.69742100233969 -123.32086478945746) bank198777 +198778 POINT(38.45781619287937 -123.20751284320194) bank198778 +198779 POINT(38.26550545740447 -123.19154321801481) bank198779 +198780 POINT(38.14452076530494 -122.60166587145284) bank198780 +198781 POINT(38.3794153995638 -122.82412905523758) bank198781 +198782 POINT(38.3225365640553 -121.4734752837276) bank198782 +198783 POINT(36.881874480001045 -122.58254658078874) bank198783 +198784 POINT(37.0749915581748 -121.81115265913158) bank198784 +198785 POINT(37.89190944375777 -123.05923414695708) bank198785 +198786 POINT(37.869762750021984 -122.01971225113569) bank198786 +198787 POINT(37.18014020602632 -122.23377846605138) bank198787 +198788 POINT(37.90089503072824 -121.4344466718726) bank198788 +198789 POINT(38.32981412370109 -122.3025531117228) bank198789 +198790 POINT(37.4739544087142 -123.21080754544508) bank198790 +198791 POINT(36.81651142448195 -121.59719924185046) bank198791 +198792 POINT(38.136492638646935 -121.52237040963566) bank198792 +198793 POINT(38.07508933344125 -123.00043034137342) bank198793 +198794 POINT(37.85014684811348 -122.72971615355708) bank198794 +198795 POINT(37.59143512028916 -121.6561190992907) bank198795 +198796 POINT(38.02451072233416 -122.14451474667588) bank198796 +198797 POINT(37.968526220960634 -122.83224401370958) bank198797 +198798 POINT(36.94855293006358 -123.24310536522584) bank198798 +198799 POINT(37.66676207150262 -123.1629721067389) bank198799 +198800 POINT(38.72932916745965 -122.66129636566109) bank198800 +198801 POINT(37.94195547478725 -122.98290000490111) bank198801 +198802 POINT(38.51649564439911 -122.3633417952103) bank198802 +198803 POINT(38.639132228497175 -121.72914515587757) bank198803 +198804 POINT(38.416130805927025 -122.98665681945418) bank198804 +198805 POINT(37.93016409068881 -122.0694820976159) bank198805 +198806 POINT(37.95219815836 -123.37113764242864) bank198806 +198807 POINT(37.2692482425194 -122.17334170081251) bank198807 +198808 POINT(38.09762237149674 -121.48230593837047) bank198808 +198809 POINT(38.342785762270275 -122.80732366148074) bank198809 +198810 POINT(37.25765512096344 -122.98999203504803) bank198810 +198811 POINT(37.52314203305646 -122.5937985976829) bank198811 +198812 POINT(36.88564502281391 -122.35978833486273) bank198812 +198813 POINT(38.02892483241679 -122.93352313000202) bank198813 +198814 POINT(38.45658243699214 -122.87448954071712) bank198814 +198815 POINT(37.458817497088255 -121.49420712588643) bank198815 +198816 POINT(37.11917860381311 -122.10616875431113) bank198816 +198817 POINT(37.31433061288235 -122.94555495345482) bank198817 +198818 POINT(37.66816566819708 -121.95058001610005) bank198818 +198819 POINT(37.2031235678375 -122.19468000782128) bank198819 +198820 POINT(37.666169729513946 -122.89874679517028) bank198820 +198821 POINT(37.2739898109952 -122.50538707863741) bank198821 +198822 POINT(37.98565971138518 -122.03607192936383) bank198822 +198823 POINT(37.573797031524606 -123.08611688053566) bank198823 +198824 POINT(37.352824460738354 -123.17552749796494) bank198824 +198825 POINT(36.88668198177795 -121.51090411758302) bank198825 +198826 POINT(37.47516456182803 -122.44301084784433) bank198826 +198827 POINT(38.270655964976996 -123.13077121848256) bank198827 +198828 POINT(38.32189799067362 -123.32285533465625) bank198828 +198829 POINT(37.90369845221963 -122.47238899544904) bank198829 +198830 POINT(37.27268383623968 -121.71353491949759) bank198830 +198831 POINT(37.02352213250267 -122.86974929922772) bank198831 +198832 POINT(38.30912270465419 -121.53514590303368) bank198832 +198833 POINT(36.89504974444425 -122.66667763827486) bank198833 +198834 POINT(37.39415628096799 -121.54599112944817) bank198834 +198835 POINT(36.87759911221607 -122.3958750702466) bank198835 +198836 POINT(36.83134218866027 -123.41919058361542) bank198836 +198837 POINT(37.09461658147581 -122.93801821566669) bank198837 +198838 POINT(37.30659715959966 -121.79574913836994) bank198838 +198839 POINT(37.3900597716039 -121.96095269308337) bank198839 +198840 POINT(37.61878103839718 -122.15693636199192) bank198840 +198841 POINT(38.133546999307505 -122.41832046346248) bank198841 +198842 POINT(38.40558987093261 -121.92170953724516) bank198842 +198843 POINT(37.272737239838904 -121.43770760346086) bank198843 +198844 POINT(37.8597348983869 -122.9529725255243) bank198844 +198845 POINT(38.49725451472819 -121.82177685548857) bank198845 +198846 POINT(37.34813608056044 -121.47939047295127) bank198846 +198847 POINT(38.48169835365681 -121.87461420792171) bank198847 +198848 POINT(37.07028567459354 -123.05211029129754) bank198848 +198849 POINT(37.36446006156381 -122.80787661333353) bank198849 +198850 POINT(37.68738334594334 -123.18922581492139) bank198850 +198851 POINT(36.81040800060584 -123.17497200251438) bank198851 +198852 POINT(37.15551430791441 -122.51811668562281) bank198852 +198853 POINT(38.40796271335537 -122.30555410134829) bank198853 +198854 POINT(38.737980131632014 -121.65076727956846) bank198854 +198855 POINT(37.753358110563205 -123.12486776316278) bank198855 +198856 POINT(37.80216309563386 -123.22268654035253) bank198856 +198857 POINT(38.33122017820102 -122.49130365822279) bank198857 +198858 POINT(37.97678063963664 -121.93046502355807) bank198858 +198859 POINT(37.1916979623002 -123.21998044528304) bank198859 +198860 POINT(38.24386557615988 -122.55892445594654) bank198860 +198861 POINT(37.08885704627588 -123.12832404909629) bank198861 +198862 POINT(37.42905149227134 -122.32603274570299) bank198862 +198863 POINT(38.334792324540004 -122.01583620885164) bank198863 +198864 POINT(38.74723082586819 -122.52480478493742) bank198864 +198865 POINT(37.18711351258178 -122.89483328205078) bank198865 +198866 POINT(37.08143880145058 -122.6095496812571) bank198866 +198867 POINT(37.99869382205445 -122.36400513955576) bank198867 +198868 POINT(37.77745821304586 -123.04206213043103) bank198868 +198869 POINT(37.88904651664223 -122.27838357730428) bank198869 +198870 POINT(38.11944798485349 -122.9166986142017) bank198870 +198871 POINT(38.00603105345205 -122.00906942276156) bank198871 +198872 POINT(38.31640905489785 -123.03442284185573) bank198872 +198873 POINT(37.001620928047316 -122.62486502784724) bank198873 +198874 POINT(37.40392153888975 -121.59273887243309) bank198874 +198875 POINT(37.04550728532116 -122.76850949554914) bank198875 +198876 POINT(37.727662725904764 -123.1767114706685) bank198876 +198877 POINT(36.84370809631062 -123.34095420379758) bank198877 +198878 POINT(37.7519389723083 -122.13880594838693) bank198878 +198879 POINT(38.19526648698262 -123.2106332353479) bank198879 +198880 POINT(38.21494297949994 -121.52736038916774) bank198880 +198881 POINT(38.02633173346058 -121.52782617509199) bank198881 +198882 POINT(36.94673512620173 -121.64276801800031) bank198882 +198883 POINT(38.529084492086064 -123.267192734278) bank198883 +198884 POINT(37.212763683020135 -122.72763296661152) bank198884 +198885 POINT(37.161958417852155 -122.98061634916677) bank198885 +198886 POINT(37.343185794332925 -122.61491404693817) bank198886 +198887 POINT(37.6293450507183 -122.19927122594397) bank198887 +198888 POINT(37.72991266816242 -123.02929499320136) bank198888 +198889 POINT(37.59366707861844 -122.3547055285045) bank198889 +198890 POINT(38.22185923824715 -122.48318491486232) bank198890 +198891 POINT(38.460955794197986 -122.71285033736999) bank198891 +198892 POINT(37.184812194202834 -122.15976833879375) bank198892 +198893 POINT(37.2663862205287 -123.11531926267243) bank198893 +198894 POINT(38.10288884345453 -121.74425569590177) bank198894 +198895 POINT(37.8011217934152 -121.86966213065332) bank198895 +198896 POINT(38.20024277074629 -122.43069377047954) bank198896 +198897 POINT(38.096478096173115 -121.7197538091545) bank198897 +198898 POINT(37.32864790860242 -122.6129098538967) bank198898 +198899 POINT(38.35549260408043 -121.9792839092364) bank198899 +198900 POINT(37.26121052181906 -122.22529939531047) bank198900 +198901 POINT(38.34986353496085 -123.30844998778015) bank198901 +198902 POINT(38.07783327304309 -123.31749657448887) bank198902 +198903 POINT(37.314866849355695 -122.87798417190436) bank198903 +198904 POINT(37.16862997708305 -122.40559404444843) bank198904 +198905 POINT(37.97736273848898 -123.3556481292059) bank198905 +198906 POINT(38.46714070203316 -122.82973902813353) bank198906 +198907 POINT(38.148885809352244 -123.32541959166541) bank198907 +198908 POINT(37.43418710648119 -123.08096133207884) bank198908 +198909 POINT(37.65538843580374 -122.52402421230052) bank198909 +198910 POINT(38.566124402466365 -121.66299960583211) bank198910 +198911 POINT(36.81026184064542 -122.44028633069335) bank198911 +198912 POINT(37.823647671858005 -123.08305364299046) bank198912 +198913 POINT(37.516488252133726 -122.38750789932946) bank198913 +198914 POINT(37.1425081304197 -121.78896618996109) bank198914 +198915 POINT(38.38345637690318 -123.38828124233446) bank198915 +198916 POINT(38.463493773735706 -123.23001743019329) bank198916 +198917 POINT(37.899614479708056 -122.38908901550637) bank198917 +198918 POINT(38.7262165093826 -121.78628195234606) bank198918 +198919 POINT(37.73642898345149 -123.28571879163107) bank198919 +198920 POINT(38.6371510550342 -121.64491028341568) bank198920 +198921 POINT(38.66389737586752 -123.1890711669305) bank198921 +198922 POINT(38.720859283035225 -123.1693505268767) bank198922 +198923 POINT(38.37300999450721 -121.57005085079146) bank198923 +198924 POINT(38.660657543638145 -121.89892404431404) bank198924 +198925 POINT(37.77745199965756 -121.85697673622674) bank198925 +198926 POINT(38.504375930044255 -122.39833466383145) bank198926 +198927 POINT(37.26605308888889 -123.19502098140163) bank198927 +198928 POINT(37.55604073211823 -122.80468010015163) bank198928 +198929 POINT(38.01407860123243 -122.62914885743967) bank198929 +198930 POINT(36.80453828381988 -122.66527072210008) bank198930 +198931 POINT(38.477261002194886 -122.07315151786212) bank198931 +198932 POINT(37.435279252803255 -122.81251174803508) bank198932 +198933 POINT(37.4581872739012 -122.25594185632599) bank198933 +198934 POINT(37.44529084559365 -121.42213258413675) bank198934 +198935 POINT(37.82213752704453 -121.55105399430118) bank198935 +198936 POINT(38.74727391313448 -122.6461635812215) bank198936 +198937 POINT(37.89054777492755 -122.77042180625992) bank198937 +198938 POINT(36.99574788215226 -123.31919359079058) bank198938 +198939 POINT(37.747019258953394 -122.75623217712491) bank198939 +198940 POINT(37.93498812088684 -121.47512063601759) bank198940 +198941 POINT(38.156214420610965 -122.97594301786755) bank198941 +198942 POINT(37.26354992839303 -123.32758582278016) bank198942 +198943 POINT(37.03623091605128 -123.10833933576627) bank198943 +198944 POINT(37.117796451383406 -121.87983637638912) bank198944 +198945 POINT(37.013830160324176 -122.4927609975367) bank198945 +198946 POINT(37.01018127636441 -123.04417383051008) bank198946 +198947 POINT(36.98331671055664 -122.71624168087207) bank198947 +198948 POINT(38.00649334731267 -122.4283986341352) bank198948 +198949 POINT(36.92111892940169 -122.4419903763862) bank198949 +198950 POINT(37.697587272680764 -122.62803922011955) bank198950 +198951 POINT(36.806909890007965 -121.87737677631297) bank198951 +198952 POINT(36.891019354237216 -122.99768341624166) bank198952 +198953 POINT(37.61699036750607 -122.4922356694881) bank198953 +198954 POINT(37.84288383791354 -121.4424189412411) bank198954 +198955 POINT(37.13745450261254 -121.8235731115136) bank198955 +198956 POINT(37.270052421642475 -121.73594926919397) bank198956 +198957 POINT(38.692626839953505 -122.0378773545747) bank198957 +198958 POINT(38.361202811960126 -121.54690975726972) bank198958 +198959 POINT(38.224127612196575 -122.85634892725079) bank198959 +198960 POINT(37.946185590621056 -122.50769705440511) bank198960 +198961 POINT(38.62435239763697 -122.49304900854922) bank198961 +198962 POINT(37.39176609888451 -123.02037809582045) bank198962 +198963 POINT(38.63490466497716 -123.04900313455187) bank198963 +198964 POINT(37.930144826678045 -121.5767589368952) bank198964 +198965 POINT(36.84309248361881 -121.73578307275749) bank198965 +198966 POINT(37.309349919620765 -121.78496031730788) bank198966 +198967 POINT(38.29120782532287 -122.01078225728034) bank198967 +198968 POINT(37.444192472567 -123.38633706413799) bank198968 +198969 POINT(37.18360265504086 -121.63966145341178) bank198969 +198970 POINT(37.847894072116375 -123.30095298722603) bank198970 +198971 POINT(37.221498615385215 -122.01821414281362) bank198971 +198972 POINT(36.84592667887231 -122.55849461089703) bank198972 +198973 POINT(37.37858675053525 -123.01970860112026) bank198973 +198974 POINT(38.0129891519988 -121.63974368214889) bank198974 +198975 POINT(37.68678841127657 -122.82641060859255) bank198975 +198976 POINT(38.73579033384954 -123.25041766130236) bank198976 +198977 POINT(36.807148911407346 -123.12103426306915) bank198977 +198978 POINT(38.53532149260866 -122.82383930060374) bank198978 +198979 POINT(37.95981233062504 -123.10299832189096) bank198979 +198980 POINT(37.41212965065297 -122.7342358033757) bank198980 +198981 POINT(38.03972190578733 -122.20294593984923) bank198981 +198982 POINT(37.64173239204047 -121.447001261353) bank198982 +198983 POINT(36.9016114874364 -123.04037792597587) bank198983 +198984 POINT(37.65145706717591 -122.52258508587484) bank198984 +198985 POINT(37.23139931885915 -121.65785515450142) bank198985 +198986 POINT(36.999973699000556 -123.07575628591705) bank198986 +198987 POINT(36.93609927106284 -123.1519570914745) bank198987 +198988 POINT(38.6847090532458 -122.09109145698086) bank198988 +198989 POINT(38.490563164995 -121.47990471500151) bank198989 +198990 POINT(38.2181106238927 -122.57653353292623) bank198990 +198991 POINT(36.99857460911723 -123.27357496715757) bank198991 +198992 POINT(38.67016511994553 -123.3634648448185) bank198992 +198993 POINT(37.50357068916153 -121.98175437280456) bank198993 +198994 POINT(37.39890327349387 -122.7087123174408) bank198994 +198995 POINT(37.82715783940124 -122.22943695885293) bank198995 +198996 POINT(38.248650739450525 -123.29787068359771) bank198996 +198997 POINT(38.41077351324823 -122.90853633896037) bank198997 +198998 POINT(36.85371559058948 -121.7021384616302) bank198998 +198999 POINT(38.4045310350279 -122.35881358369865) bank198999 +199000 POINT(38.23160537126728 -122.2325298445097) bank199000 +199001 POINT(36.79183732359042 -122.53482899291528) bank199001 +199002 POINT(37.976743314550085 -123.26925308865185) bank199002 +199003 POINT(37.04773325782103 -122.39955919083954) bank199003 +199004 POINT(36.88414475941271 -122.55387173326795) bank199004 +199005 POINT(38.15941742149372 -121.90327953167444) bank199005 +199006 POINT(37.94024573229638 -122.36105013373594) bank199006 +199007 POINT(36.80253961346994 -121.51504978289) bank199007 +199008 POINT(38.73844356314752 -122.69389871313047) bank199008 +199009 POINT(38.16245279225552 -122.07279861921486) bank199009 +199010 POINT(38.23534983110487 -123.12641903239984) bank199010 +199011 POINT(38.51336390277771 -121.68372349772181) bank199011 +199012 POINT(37.25998048196914 -122.50843221505366) bank199012 +199013 POINT(36.958452643887455 -122.55828497176391) bank199013 +199014 POINT(36.93432766978922 -121.42607462179798) bank199014 +199015 POINT(37.49875919270352 -123.12730410438635) bank199015 +199016 POINT(38.33491848815921 -123.1364583961862) bank199016 +199017 POINT(37.803819313421776 -121.47188633994409) bank199017 +199018 POINT(38.488138089886945 -122.31170999013239) bank199018 +199019 POINT(37.20583924482532 -122.48879696509228) bank199019 +199020 POINT(37.34764823395101 -123.19587442991572) bank199020 +199021 POINT(37.07920266662759 -122.33451143855721) bank199021 +199022 POINT(38.69417120178779 -122.5503478967864) bank199022 +199023 POINT(37.805818563360305 -122.21381296363165) bank199023 +199024 POINT(37.89390586907548 -121.96173026967016) bank199024 +199025 POINT(38.69357131122211 -122.79098806596525) bank199025 +199026 POINT(36.97477643441455 -122.70908006882794) bank199026 +199027 POINT(37.34750518888253 -121.85006008800636) bank199027 +199028 POINT(36.97107660945128 -121.47119403319327) bank199028 +199029 POINT(37.36104629627165 -123.03158783517321) bank199029 +199030 POINT(36.8026580556922 -122.6554916952725) bank199030 +199031 POINT(37.57678999031277 -122.51550963219566) bank199031 +199032 POINT(37.311545788738435 -123.19888704906677) bank199032 +199033 POINT(38.60483537025529 -122.34106645126852) bank199033 +199034 POINT(37.01700391914768 -123.16893727841872) bank199034 +199035 POINT(38.73572800507606 -123.3198169848002) bank199035 +199036 POINT(38.755793858706795 -121.71398718515775) bank199036 +199037 POINT(38.72659950116825 -121.92636026628247) bank199037 +199038 POINT(38.24883722412064 -122.94442864706956) bank199038 +199039 POINT(38.25630364738738 -122.24442524611885) bank199039 +199040 POINT(37.864397096483884 -122.9227116002203) bank199040 +199041 POINT(37.9472698986914 -121.92727586724898) bank199041 +199042 POINT(38.20895599145336 -123.37306199836335) bank199042 +199043 POINT(37.03660518241001 -123.38963708800932) bank199043 +199044 POINT(36.85491817788218 -123.00453513950423) bank199044 +199045 POINT(38.572120099426286 -121.87344150888238) bank199045 +199046 POINT(37.40242341250546 -121.6998148697254) bank199046 +199047 POINT(37.083177658200476 -121.65682922926928) bank199047 +199048 POINT(38.341094287988156 -121.9553414754924) bank199048 +199049 POINT(38.17180386945705 -122.07110692458001) bank199049 +199050 POINT(38.55189443590976 -121.57043176185998) bank199050 +199051 POINT(37.8551952176475 -121.72701547059741) bank199051 +199052 POINT(37.64965396964254 -123.40113455367016) bank199052 +199053 POINT(38.306770511513314 -121.428979015844) bank199053 +199054 POINT(37.214308316750234 -123.17187463496509) bank199054 +199055 POINT(37.288574695238324 -121.50122829502229) bank199055 +199056 POINT(38.709670040147465 -122.26824045344344) bank199056 +199057 POINT(37.818345882890206 -122.7066189961183) bank199057 +199058 POINT(38.41394478136416 -122.34409029341087) bank199058 +199059 POINT(38.76250268599005 -122.42268180602561) bank199059 +199060 POINT(38.682634540759786 -122.172136448209) bank199060 +199061 POINT(37.36154533480343 -121.85586986418704) bank199061 +199062 POINT(37.643088403514106 -123.00751137430238) bank199062 +199063 POINT(37.06208746122796 -122.32237626699462) bank199063 +199064 POINT(37.32155655785245 -122.33398074189866) bank199064 +199065 POINT(38.447427593247454 -122.5090199116964) bank199065 +199066 POINT(38.58464260326509 -122.63475978107482) bank199066 +199067 POINT(38.774471368169024 -123.31866263592671) bank199067 +199068 POINT(38.18518584142224 -122.08096321405542) bank199068 +199069 POINT(36.83073007518854 -121.61394943052272) bank199069 +199070 POINT(37.69405616302615 -123.31422653397206) bank199070 +199071 POINT(37.9239868724509 -122.95874460584122) bank199071 +199072 POINT(38.71400173138246 -122.39156416072332) bank199072 +199073 POINT(37.25967813926113 -123.33441205490607) bank199073 +199074 POINT(36.89386847469424 -121.73352792707685) bank199074 +199075 POINT(38.53011495198251 -121.67054617687596) bank199075 +199076 POINT(37.054954392327176 -123.25395060583502) bank199076 +199077 POINT(38.14594555227788 -122.61816598009219) bank199077 +199078 POINT(38.39521780966092 -122.62584480206706) bank199078 +199079 POINT(38.39212454875138 -122.22530378426119) bank199079 +199080 POINT(37.1746314856203 -123.17707199872004) bank199080 +199081 POINT(38.59999414380821 -122.78843430104831) bank199081 +199082 POINT(38.66873254203923 -122.9502071965744) bank199082 +199083 POINT(37.47296499256838 -123.20064088840047) bank199083 +199084 POINT(38.66852216508739 -122.0494579984206) bank199084 +199085 POINT(38.604735599123664 -122.73499205514139) bank199085 +199086 POINT(36.862186823318495 -121.7620448255269) bank199086 +199087 POINT(36.992584440759295 -123.09663518713188) bank199087 +199088 POINT(38.17739919084773 -122.72633787730565) bank199088 +199089 POINT(37.141450521215745 -122.2833215578433) bank199089 +199090 POINT(38.46312500409328 -122.15308717894949) bank199090 +199091 POINT(37.51332074809829 -122.07498550886048) bank199091 +199092 POINT(36.82915944574559 -121.43243788038508) bank199092 +199093 POINT(36.94330354473503 -121.51061990159553) bank199093 +199094 POINT(37.66582110851455 -123.07204107288129) bank199094 +199095 POINT(38.56265914691181 -122.1067406382792) bank199095 +199096 POINT(38.205758523847955 -122.53399848343025) bank199096 +199097 POINT(37.139592071138765 -121.61611746477152) bank199097 +199098 POINT(36.98723300372891 -121.62915154979683) bank199098 +199099 POINT(37.816826755380646 -121.7054136329237) bank199099 +199100 POINT(36.85706569302924 -121.4894045892132) bank199100 +199101 POINT(37.027521026096764 -122.81093934850561) bank199101 +199102 POINT(36.962033388184025 -123.0971640233365) bank199102 +199103 POINT(36.84976433642738 -123.16782334713938) bank199103 +199104 POINT(38.37059623621053 -122.1527199554682) bank199104 +199105 POINT(37.91223577738595 -121.79731723367242) bank199105 +199106 POINT(37.55541091916506 -122.47138653888909) bank199106 +199107 POINT(37.67460191630844 -122.22920956568383) bank199107 +199108 POINT(38.462110056144894 -121.73416219623422) bank199108 +199109 POINT(38.04263131463429 -121.44938884803538) bank199109 +199110 POINT(37.257382854931684 -121.6790209896955) bank199110 +199111 POINT(37.59245226894251 -121.81559597130571) bank199111 +199112 POINT(38.66680913187358 -122.67300532739452) bank199112 +199113 POINT(36.994824116822436 -121.52761081900154) bank199113 +199114 POINT(37.81039182931827 -121.9808092584831) bank199114 +199115 POINT(38.50067502167263 -123.00143062085871) bank199115 +199116 POINT(37.33194103371074 -123.22050127316929) bank199116 +199117 POINT(37.48619728761436 -122.91128681581962) bank199117 +199118 POINT(37.14043078960316 -121.61842403543822) bank199118 +199119 POINT(37.912323675591914 -122.4926574266778) bank199119 +199120 POINT(38.26966258860972 -123.07309425684396) bank199120 +199121 POINT(37.82420474692066 -121.86895360071816) bank199121 +199122 POINT(37.59862549492736 -121.8154313305774) bank199122 +199123 POINT(37.649093214295405 -121.97637934181084) bank199123 +199124 POINT(36.981492936984814 -121.90558709702448) bank199124 +199125 POINT(38.04490991050442 -122.1062526872953) bank199125 +199126 POINT(38.4564893795487 -122.14791751715593) bank199126 +199127 POINT(37.4170338067338 -122.81638408745268) bank199127 +199128 POINT(38.334475193412864 -121.65591838812968) bank199128 +199129 POINT(37.277640830966696 -122.50572668114876) bank199129 +199130 POINT(38.64562061094439 -121.74833597395384) bank199130 +199131 POINT(37.23335491490793 -122.8422724245374) bank199131 +199132 POINT(37.95828664074862 -122.08241765073431) bank199132 +199133 POINT(38.481510865709765 -121.58242691120914) bank199133 +199134 POINT(37.13435176576223 -123.04426565626342) bank199134 +199135 POINT(38.37499508182323 -121.65862920738778) bank199135 +199136 POINT(37.736739078507526 -122.79904799459455) bank199136 +199137 POINT(38.423918773375874 -121.71063367300427) bank199137 +199138 POINT(38.11866374397457 -122.87124167647126) bank199138 +199139 POINT(37.7340347205901 -122.74524294950494) bank199139 +199140 POINT(38.55516688714981 -122.16202694743858) bank199140 +199141 POINT(38.013960998408805 -121.84787754808835) bank199141 +199142 POINT(38.3411452468175 -121.61508497725845) bank199142 +199143 POINT(37.57389510191525 -121.44213012950976) bank199143 +199144 POINT(38.196722829888444 -122.17265066455526) bank199144 +199145 POINT(37.59888898625355 -123.07049393830685) bank199145 +199146 POINT(38.01025240653647 -123.30912716467626) bank199146 +199147 POINT(37.00372096437231 -122.83064387414802) bank199147 +199148 POINT(38.221112160524555 -121.96781630472806) bank199148 +199149 POINT(38.30803192773792 -122.44854012097476) bank199149 +199150 POINT(38.37744208716806 -123.2946203289316) bank199150 +199151 POINT(37.01137616993833 -122.39784007412084) bank199151 +199152 POINT(38.683124444390515 -122.2514850607477) bank199152 +199153 POINT(37.90676949811388 -122.49258824808133) bank199153 +199154 POINT(38.565460967795524 -122.10752828802194) bank199154 +199155 POINT(37.37859891842574 -121.93146931530859) bank199155 +199156 POINT(38.14723683204316 -122.71499807190636) bank199156 +199157 POINT(37.51693925847418 -122.00490176470181) bank199157 +199158 POINT(36.85758865308438 -123.3446708730543) bank199158 +199159 POINT(37.21882992763985 -123.14498096315815) bank199159 +199160 POINT(37.312730428400506 -121.69920371975074) bank199160 +199161 POINT(37.310576347490944 -121.99208383116881) bank199161 +199162 POINT(38.1013494824182 -122.79419674728767) bank199162 +199163 POINT(38.601628971217515 -122.51510024967303) bank199163 +199164 POINT(37.20280572053293 -122.73334470982486) bank199164 +199165 POINT(38.70533908845575 -122.59890590971408) bank199165 +199166 POINT(38.00006964490461 -122.83097123505824) bank199166 +199167 POINT(37.26116509672541 -123.03168908206924) bank199167 +199168 POINT(37.87859770063587 -121.84851795120574) bank199168 +199169 POINT(37.46648618279545 -122.31524262785359) bank199169 +199170 POINT(37.61816253408162 -122.54804861718246) bank199170 +199171 POINT(37.541838831040536 -121.75734461245672) bank199171 +199172 POINT(37.15392908961591 -123.10640846326869) bank199172 +199173 POINT(37.531632138439164 -122.62291061247346) bank199173 +199174 POINT(38.30768655857785 -122.0993104062715) bank199174 +199175 POINT(36.84104755291666 -121.80133062765164) bank199175 +199176 POINT(38.22096451232177 -122.11967520984992) bank199176 +199177 POINT(37.9976790573398 -122.93030317454688) bank199177 +199178 POINT(37.460469727387796 -123.14023066163423) bank199178 +199179 POINT(37.69888237641504 -123.41252943462644) bank199179 +199180 POINT(37.026638799593755 -121.76791417976827) bank199180 +199181 POINT(38.024232640453086 -123.28672310403043) bank199181 +199182 POINT(36.92261655939072 -122.78621179967034) bank199182 +199183 POINT(36.78967159541254 -122.04766497579028) bank199183 +199184 POINT(37.57547987650439 -122.06640712779792) bank199184 +199185 POINT(37.78856859098516 -122.1903098020206) bank199185 +199186 POINT(38.2218385966748 -122.33842024837993) bank199186 +199187 POINT(38.59040200244659 -121.46394198531534) bank199187 +199188 POINT(38.3299503081683 -122.78811434912458) bank199188 +199189 POINT(38.641653862177854 -121.79339787471672) bank199189 +199190 POINT(36.78906834067118 -122.85805995242369) bank199190 +199191 POINT(36.99811918212253 -121.79034535997664) bank199191 +199192 POINT(38.47661367843502 -123.0603470551155) bank199192 +199193 POINT(37.26439798055235 -123.0305440388852) bank199193 +199194 POINT(36.835839801214476 -121.88232230207598) bank199194 +199195 POINT(38.66850552602046 -123.18168078547077) bank199195 +199196 POINT(38.36419700987803 -121.54501010446731) bank199196 +199197 POINT(36.94988213439904 -122.55941729052533) bank199197 +199198 POINT(37.28426081595827 -122.14640897261081) bank199198 +199199 POINT(38.002732668749466 -122.48139497609186) bank199199 +199200 POINT(38.46196119493697 -122.36000617025306) bank199200 +199201 POINT(37.06894443618464 -123.09884110366386) bank199201 +199202 POINT(37.246669698779854 -122.72465183869338) bank199202 +199203 POINT(38.51976735030271 -122.11492252259089) bank199203 +199204 POINT(37.966831089027046 -122.18018891788743) bank199204 +199205 POINT(37.77395332070047 -122.70964424741722) bank199205 +199206 POINT(36.87930573067252 -122.24031956733303) bank199206 +199207 POINT(37.38057336338591 -121.92102720155607) bank199207 +199208 POINT(38.30218630308836 -122.89392008665489) bank199208 +199209 POINT(37.449461810920745 -121.89261112401107) bank199209 +199210 POINT(37.36263346167817 -123.07341977542602) bank199210 +199211 POINT(37.723980807883024 -122.75523074184927) bank199211 +199212 POINT(37.983527596142864 -122.47003051679413) bank199212 +199213 POINT(37.87100289326905 -122.07082985199985) bank199213 +199214 POINT(38.469184436178786 -121.85683487902533) bank199214 +199215 POINT(37.94243840689285 -122.5847504073063) bank199215 +199216 POINT(37.24822192314226 -122.20476692651059) bank199216 +199217 POINT(38.09930242241295 -122.58847075262312) bank199217 +199218 POINT(38.567168862819 -121.60322973322212) bank199218 +199219 POINT(38.52137522732192 -122.25421989311026) bank199219 +199220 POINT(37.15840457204131 -123.0232026207685) bank199220 +199221 POINT(36.91130310004093 -122.34132773783692) bank199221 +199222 POINT(38.74229882173908 -123.40651991185226) bank199222 +199223 POINT(38.31013711377138 -123.32431573691568) bank199223 +199224 POINT(37.2749153810799 -121.7906236025717) bank199224 +199225 POINT(38.03421324573476 -121.65020544569234) bank199225 +199226 POINT(37.24958018526328 -123.30369340642689) bank199226 +199227 POINT(38.47994947490106 -123.06984011529788) bank199227 +199228 POINT(38.34057639867232 -122.20804863616203) bank199228 +199229 POINT(37.631187578108694 -121.63064886043009) bank199229 +199230 POINT(38.75612294037219 -121.52554330113276) bank199230 +199231 POINT(37.879337829735064 -122.63986704481887) bank199231 +199232 POINT(38.26458496840804 -121.45095682793962) bank199232 +199233 POINT(37.16522068553832 -122.42599044481159) bank199233 +199234 POINT(37.98699629590519 -122.37913249791029) bank199234 +199235 POINT(38.15709040075102 -122.9596332284677) bank199235 +199236 POINT(37.98897292955608 -122.74200418968029) bank199236 +199237 POINT(38.63691885800645 -123.33463884153655) bank199237 +199238 POINT(38.20205749482947 -121.90228991121987) bank199238 +199239 POINT(37.904126679289604 -123.34458227518556) bank199239 +199240 POINT(37.503545570360124 -121.94841198338739) bank199240 +199241 POINT(38.52253981689869 -121.50141658778782) bank199241 +199242 POINT(37.4873459267608 -121.75847128996082) bank199242 +199243 POINT(38.07484605134998 -122.10206014532301) bank199243 +199244 POINT(36.901355892780174 -121.61610880318331) bank199244 +199245 POINT(38.27271501441824 -122.5482345620018) bank199245 +199246 POINT(37.42565006561888 -123.36149599290826) bank199246 +199247 POINT(36.963261207764816 -121.47437444815513) bank199247 +199248 POINT(38.19060431884563 -122.97779035878288) bank199248 +199249 POINT(37.633815359349 -122.7944382866355) bank199249 +199250 POINT(37.62487660254936 -122.91014777284094) bank199250 +199251 POINT(37.73751183600846 -123.11782022130585) bank199251 +199252 POINT(37.383759802479126 -123.00882848035383) bank199252 +199253 POINT(36.9193497872625 -121.5727341292015) bank199253 +199254 POINT(38.21131308546303 -122.53381844000894) bank199254 +199255 POINT(37.15123896152828 -121.84006243410319) bank199255 +199256 POINT(37.75545681832373 -121.68405550559724) bank199256 +199257 POINT(38.022228629676306 -122.44149959644042) bank199257 +199258 POINT(38.717495051689525 -121.56237224496813) bank199258 +199259 POINT(38.61801755328624 -121.50446363968163) bank199259 +199260 POINT(38.19441840199165 -122.23880729029884) bank199260 +199261 POINT(37.70633462965414 -122.57356778018293) bank199261 +199262 POINT(38.089236578281096 -123.1208255209882) bank199262 +199263 POINT(38.15749200889346 -121.71106624493277) bank199263 +199264 POINT(38.67329242233247 -122.39789813802436) bank199264 +199265 POINT(38.128443790717014 -123.31987726562011) bank199265 +199266 POINT(37.74260167921514 -121.85447277893392) bank199266 +199267 POINT(38.042872142183136 -122.36325310113615) bank199267 +199268 POINT(38.01109589382745 -123.10846281090008) bank199268 +199269 POINT(37.52948722435122 -121.67600411467028) bank199269 +199270 POINT(38.36152709681181 -122.42609151501003) bank199270 +199271 POINT(38.20464338925498 -122.76196257529526) bank199271 +199272 POINT(38.14027126958577 -122.70907226856207) bank199272 +199273 POINT(38.4359176353336 -122.82453987085236) bank199273 +199274 POINT(38.23444028247774 -121.58085957258572) bank199274 +199275 POINT(38.38000603192547 -121.72886823344415) bank199275 +199276 POINT(36.89504930533647 -122.8453554425087) bank199276 +199277 POINT(37.05629166679107 -121.65066677607153) bank199277 +199278 POINT(36.995912160030244 -121.75679756240247) bank199278 +199279 POINT(37.1679624744509 -122.98577931017711) bank199279 +199280 POINT(37.051577826559864 -122.7751148920656) bank199280 +199281 POINT(37.45325304904521 -123.36203752812719) bank199281 +199282 POINT(38.41468294743987 -122.45868392386022) bank199282 +199283 POINT(37.68384018939999 -122.36267093077396) bank199283 +199284 POINT(37.09812614943629 -122.38131258835703) bank199284 +199285 POINT(37.890836093870874 -122.22840342511569) bank199285 +199286 POINT(38.05957637664807 -123.00403390801918) bank199286 +199287 POINT(38.02324093838088 -122.27581264548783) bank199287 +199288 POINT(38.27695704624145 -122.14471631016532) bank199288 +199289 POINT(38.004617241598005 -123.34704122510345) bank199289 +199290 POINT(38.1326497282831 -123.17191979276225) bank199290 +199291 POINT(37.42118788066823 -121.98010077864213) bank199291 +199292 POINT(37.26049105747532 -122.36096428264344) bank199292 +199293 POINT(37.44715969475355 -121.51856565751663) bank199293 +199294 POINT(37.28614239055065 -123.0073272215948) bank199294 +199295 POINT(36.81625057733817 -122.0902262836814) bank199295 +199296 POINT(38.53615532438117 -122.96873401262013) bank199296 +199297 POINT(37.17211570043819 -121.4729110047118) bank199297 +199298 POINT(37.730630417114334 -123.28882055478894) bank199298 +199299 POINT(36.84172664552865 -121.60487131961798) bank199299 +199300 POINT(38.61704293757326 -122.77002346890087) bank199300 +199301 POINT(36.905770674614025 -122.6748847212142) bank199301 +199302 POINT(37.537424775552225 -123.38220997706524) bank199302 +199303 POINT(38.750001097600716 -122.44030905933194) bank199303 +199304 POINT(37.142507668490516 -122.91502221289699) bank199304 +199305 POINT(38.37956648057825 -122.4085960996501) bank199305 +199306 POINT(36.79643093748525 -121.72778237086925) bank199306 +199307 POINT(37.08571829325458 -123.34492600938758) bank199307 +199308 POINT(37.918025582692486 -121.6014604739076) bank199308 +199309 POINT(37.76451438837298 -121.69860725901604) bank199309 +199310 POINT(36.9145407231618 -121.9714139134716) bank199310 +199311 POINT(38.09321519900308 -123.07019836784083) bank199311 +199312 POINT(38.00088289797452 -122.60708528042395) bank199312 +199313 POINT(38.50736223212276 -123.33692195501897) bank199313 +199314 POINT(38.193780709831884 -121.57549444350943) bank199314 +199315 POINT(37.577530218206476 -122.5766001409017) bank199315 +199316 POINT(38.2955693321569 -122.47207262720399) bank199316 +199317 POINT(37.49797531346941 -122.67350122577669) bank199317 +199318 POINT(38.342829449315005 -123.28370059804061) bank199318 +199319 POINT(36.870702077202886 -121.6806954654728) bank199319 +199320 POINT(38.37711223621228 -123.31216220758402) bank199320 +199321 POINT(37.016036393795346 -121.53708932593437) bank199321 +199322 POINT(37.15443178078303 -122.64642371532324) bank199322 +199323 POINT(37.52480898763867 -121.65383970315342) bank199323 +199324 POINT(37.30953679748216 -121.51103693719736) bank199324 +199325 POINT(37.443676787380795 -123.14955088974018) bank199325 +199326 POINT(37.613757960903264 -121.83123190140131) bank199326 +199327 POINT(38.06976597570265 -122.99653249488235) bank199327 +199328 POINT(37.62816758302988 -123.09293384039196) bank199328 +199329 POINT(37.8983937703013 -122.12108642738232) bank199329 +199330 POINT(38.391697908557134 -121.68818062008667) bank199330 +199331 POINT(38.67713978219074 -123.26118802119238) bank199331 +199332 POINT(37.84608548044401 -122.51674644866347) bank199332 +199333 POINT(37.75119641212483 -122.88789667481456) bank199333 +199334 POINT(38.257107220277135 -122.92412312446946) bank199334 +199335 POINT(38.619443009892386 -121.82710694982246) bank199335 +199336 POINT(38.29998112214666 -121.44338182083516) bank199336 +199337 POINT(37.1374357220962 -121.55241502604383) bank199337 +199338 POINT(37.20508926292919 -121.91186318511087) bank199338 +199339 POINT(38.2545951593037 -122.57621860421659) bank199339 +199340 POINT(36.85665221056667 -122.9904210354742) bank199340 +199341 POINT(38.039279316346395 -121.84106465254591) bank199341 +199342 POINT(38.7304861722952 -122.53765538737038) bank199342 +199343 POINT(38.41390417153913 -123.09033896467952) bank199343 +199344 POINT(38.01483081275027 -123.36324504102458) bank199344 +199345 POINT(37.35092812489712 -122.7645743013633) bank199345 +199346 POINT(37.18211214657656 -122.82001777611298) bank199346 +199347 POINT(38.169176809423654 -122.51295269178551) bank199347 +199348 POINT(37.53202523618541 -122.30387542958428) bank199348 +199349 POINT(37.39450876707103 -123.14853511327891) bank199349 +199350 POINT(37.283744158591404 -122.07060326063385) bank199350 +199351 POINT(37.08396910141217 -121.93521801337747) bank199351 +199352 POINT(38.30047234081612 -122.81828728113584) bank199352 +199353 POINT(37.9407541278663 -122.95864318500628) bank199353 +199354 POINT(38.05892157716374 -123.32048013954315) bank199354 +199355 POINT(36.868299199065646 -122.19584243606619) bank199355 +199356 POINT(37.72776954350206 -122.24993914965515) bank199356 +199357 POINT(38.58649118694765 -121.4459669416674) bank199357 +199358 POINT(37.479445071067516 -122.5580502203111) bank199358 +199359 POINT(38.28688194512369 -122.34658003480862) bank199359 +199360 POINT(38.219717656072284 -122.3893349023235) bank199360 +199361 POINT(38.3125569532182 -121.89555844869044) bank199361 +199362 POINT(36.90786857984713 -123.20692491696366) bank199362 +199363 POINT(36.918581648127976 -122.1623349015958) bank199363 +199364 POINT(36.89745935172999 -122.81238117833283) bank199364 +199365 POINT(37.621508563457645 -121.58755936362748) bank199365 +199366 POINT(37.55754264457625 -121.46809446476779) bank199366 +199367 POINT(37.87755868778333 -122.644574796592) bank199367 +199368 POINT(38.71022091817904 -121.63599186421024) bank199368 +199369 POINT(36.92523165307968 -122.04919124112004) bank199369 +199370 POINT(37.49144771795337 -121.72725832661543) bank199370 +199371 POINT(37.34964847853339 -122.02777507908213) bank199371 +199372 POINT(37.841080898986995 -121.80970876439306) bank199372 +199373 POINT(37.22642960514371 -121.69608026476278) bank199373 +199374 POINT(38.34766327300167 -122.7393133472533) bank199374 +199375 POINT(37.34053246399923 -121.67821179571908) bank199375 +199376 POINT(38.049834757366355 -121.70240265103267) bank199376 +199377 POINT(37.025237146536696 -123.1950635917094) bank199377 +199378 POINT(37.92994607228748 -122.95293340577508) bank199378 +199379 POINT(38.17435157465734 -122.8983985583432) bank199379 +199380 POINT(36.94802643990085 -123.26475676993273) bank199380 +199381 POINT(37.1098679127884 -121.65514007913582) bank199381 +199382 POINT(38.197895953942286 -122.62574858037594) bank199382 +199383 POINT(37.35481185772456 -122.76830194484849) bank199383 +199384 POINT(36.82428944617492 -122.36681794059741) bank199384 +199385 POINT(38.52985723919246 -123.00079491756068) bank199385 +199386 POINT(38.2206641783407 -121.56237447069122) bank199386 +199387 POINT(37.45911595390175 -122.36512883586937) bank199387 +199388 POINT(38.2387215481338 -121.73429210438252) bank199388 +199389 POINT(37.835506213183336 -122.7904963627494) bank199389 +199390 POINT(37.25751485167803 -123.19792673189781) bank199390 +199391 POINT(38.14266946178147 -123.00902430302632) bank199391 +199392 POINT(38.427955951768 -121.45903270988043) bank199392 +199393 POINT(37.08452349700883 -121.85164131048225) bank199393 +199394 POINT(37.93125679125955 -123.0980035809348) bank199394 +199395 POINT(36.88533973879575 -122.4832790320504) bank199395 +199396 POINT(38.58134541387761 -122.96948273491707) bank199396 +199397 POINT(37.85646178014582 -123.2967150441977) bank199397 +199398 POINT(36.84403538132367 -123.04948771744938) bank199398 +199399 POINT(37.10015463559526 -121.60110134807158) bank199399 +199400 POINT(36.90014141206175 -121.72261554539504) bank199400 +199401 POINT(37.450808327892055 -122.96945176166828) bank199401 +199402 POINT(36.843483181247414 -121.56976210385544) bank199402 +199403 POINT(38.01981950944345 -123.30476259965803) bank199403 +199404 POINT(38.21525119027313 -122.22788243413908) bank199404 +199405 POINT(37.64932549593547 -122.60774988970228) bank199405 +199406 POINT(37.89552845273848 -121.99657317525175) bank199406 +199407 POINT(37.6098761430672 -121.46170165871754) bank199407 +199408 POINT(37.34297574690577 -123.29054758849554) bank199408 +199409 POINT(36.80842427891127 -123.40503911405408) bank199409 +199410 POINT(37.98260759490862 -121.9965970807009) bank199410 +199411 POINT(37.32954419284705 -121.62015358612021) bank199411 +199412 POINT(37.08363891639495 -123.2471393017079) bank199412 +199413 POINT(37.44574082504385 -122.8039503965948) bank199413 +199414 POINT(37.858500396898 -122.86541174547274) bank199414 +199415 POINT(37.07774592060995 -122.29606392658438) bank199415 +199416 POINT(37.39832258788982 -121.51012725895734) bank199416 +199417 POINT(38.29358390789954 -121.81608219737274) bank199417 +199418 POINT(36.90644143314789 -123.24816522780782) bank199418 +199419 POINT(38.53598911474516 -123.24245070161243) bank199419 +199420 POINT(38.50669940115092 -121.67413684465573) bank199420 +199421 POINT(38.41854454167353 -122.10779100879299) bank199421 +199422 POINT(36.986703492826145 -121.71748411677014) bank199422 +199423 POINT(38.745004425057154 -122.95952573002279) bank199423 +199424 POINT(38.09886626919363 -122.82981812824586) bank199424 +199425 POINT(38.01561839331785 -121.46803941264886) bank199425 +199426 POINT(37.315374015134026 -122.47970466050619) bank199426 +199427 POINT(37.6715031086184 -122.11518819919434) bank199427 +199428 POINT(37.49192614419635 -122.31189888445063) bank199428 +199429 POINT(37.65180441536323 -122.54800540993259) bank199429 +199430 POINT(38.225663931203705 -122.38254734682226) bank199430 +199431 POINT(37.9626531711444 -122.93454336763715) bank199431 +199432 POINT(37.7425246781522 -122.49396185562473) bank199432 +199433 POINT(37.1200211228573 -122.586374598906) bank199433 +199434 POINT(36.852697364536645 -121.75674080205815) bank199434 +199435 POINT(37.85596022283742 -122.50866317322463) bank199435 +199436 POINT(37.853510682180776 -122.83725776312232) bank199436 +199437 POINT(37.096138187781236 -123.0098329009181) bank199437 +199438 POINT(37.34275310843293 -122.14897560294045) bank199438 +199439 POINT(36.91197857407691 -123.37545077164701) bank199439 +199440 POINT(37.352835321060915 -122.48694484786338) bank199440 +199441 POINT(38.36515561048454 -121.98983127641822) bank199441 +199442 POINT(38.02869277625952 -123.16901867882954) bank199442 +199443 POINT(37.33494432115754 -122.83587988858571) bank199443 +199444 POINT(37.708094907862666 -121.45427132375586) bank199444 +199445 POINT(37.32948669878533 -122.23320504245518) bank199445 +199446 POINT(37.18093515072213 -122.09650666801465) bank199446 +199447 POINT(37.96866647751815 -122.03268784690108) bank199447 +199448 POINT(37.14141930636394 -121.98930073898902) bank199448 +199449 POINT(38.493603757685456 -121.7366952161489) bank199449 +199450 POINT(37.78161899708939 -122.90528004405635) bank199450 +199451 POINT(37.60961898921132 -122.69390415231474) bank199451 +199452 POINT(37.524744436249655 -122.5007858654338) bank199452 +199453 POINT(37.41327306353574 -121.99355487765308) bank199453 +199454 POINT(37.42519070256832 -123.16803247499013) bank199454 +199455 POINT(37.73660301538105 -122.72656535329081) bank199455 +199456 POINT(37.38177271742362 -122.8128090178468) bank199456 +199457 POINT(37.93969447350538 -122.33446026891387) bank199457 +199458 POINT(37.08244146075352 -121.47322809914003) bank199458 +199459 POINT(38.0548981350579 -121.55223706005495) bank199459 +199460 POINT(37.91645456447738 -121.99655583397228) bank199460 +199461 POINT(37.30794242683289 -122.73966990138162) bank199461 +199462 POINT(37.579230163025926 -122.52983772539984) bank199462 +199463 POINT(37.642718550198836 -121.98163825323027) bank199463 +199464 POINT(37.23041299002682 -122.86556621468243) bank199464 +199465 POINT(36.83422275769135 -122.26619413128155) bank199465 +199466 POINT(36.79518844886421 -121.9765689817198) bank199466 +199467 POINT(38.575499388974414 -123.00117610561055) bank199467 +199468 POINT(37.02049990093174 -121.66821805825799) bank199468 +199469 POINT(37.38429616639547 -121.44199875900956) bank199469 +199470 POINT(37.47926656405994 -122.69449500425384) bank199470 +199471 POINT(37.277434268595556 -121.47100270888357) bank199471 +199472 POINT(36.845615981318446 -123.13002676821237) bank199472 +199473 POINT(38.37594257197639 -122.94130561176584) bank199473 +199474 POINT(38.20231543049864 -122.04769169877153) bank199474 +199475 POINT(37.14732978002369 -123.18885587643905) bank199475 +199476 POINT(37.93251371138154 -123.09674508058268) bank199476 +199477 POINT(38.23816876361196 -122.91446293280941) bank199477 +199478 POINT(36.879230892369485 -123.20060928324881) bank199478 +199479 POINT(37.693763535047864 -121.46467530251464) bank199479 +199480 POINT(38.312980617728215 -122.50553118160025) bank199480 +199481 POINT(37.13515649494112 -123.09157580988608) bank199481 +199482 POINT(38.047098563978224 -123.02092378139983) bank199482 +199483 POINT(38.70048578235378 -123.40290499070328) bank199483 +199484 POINT(37.01455366997102 -123.08035022206045) bank199484 +199485 POINT(37.684777935020456 -121.84957568507478) bank199485 +199486 POINT(38.401889460731915 -121.48069603871465) bank199486 +199487 POINT(37.331913814876685 -122.29499687347825) bank199487 +199488 POINT(38.609163413321255 -122.25563346844335) bank199488 +199489 POINT(38.17581496320203 -122.17673454372213) bank199489 +199490 POINT(38.755108327849314 -121.5014393221918) bank199490 +199491 POINT(38.56409626933039 -121.74237718746993) bank199491 +199492 POINT(38.15417875288526 -123.23367712334297) bank199492 +199493 POINT(38.3728981478793 -122.43510730371224) bank199493 +199494 POINT(38.03110803968734 -123.18653627211133) bank199494 +199495 POINT(36.89099271593692 -122.05767748056063) bank199495 +199496 POINT(37.757297513686964 -122.98661625174358) bank199496 +199497 POINT(37.74582592370654 -122.73092640227462) bank199497 +199498 POINT(36.97129893426076 -122.87875543163348) bank199498 +199499 POINT(38.49414291426038 -122.07747493756267) bank199499 +199500 POINT(37.87127077441816 -122.16240485525269) bank199500 +199501 POINT(37.748595014257354 -122.59129339763528) bank199501 +199502 POINT(37.095403589034625 -123.11950803443514) bank199502 +199503 POINT(37.36966648552649 -122.81583260061821) bank199503 +199504 POINT(36.97529013661281 -123.28975135791563) bank199504 +199505 POINT(38.562928263634 -122.86510689088104) bank199505 +199506 POINT(37.829119269842515 -121.78790917160768) bank199506 +199507 POINT(38.739068619098745 -122.63474353702918) bank199507 +199508 POINT(37.92844614297649 -122.70040433068093) bank199508 +199509 POINT(37.51099843665487 -122.43482370991768) bank199509 +199510 POINT(37.90273075171349 -122.41123169292149) bank199510 +199511 POINT(36.97553988261888 -121.7097537125502) bank199511 +199512 POINT(38.54879349454553 -121.6465925641736) bank199512 +199513 POINT(37.116207460745564 -122.69143264284791) bank199513 +199514 POINT(37.53512768911841 -122.60878079825478) bank199514 +199515 POINT(38.708283370196256 -121.8351728282344) bank199515 +199516 POINT(36.835997753523266 -122.97466584010718) bank199516 +199517 POINT(37.46842687664781 -121.4954152312627) bank199517 +199518 POINT(38.532535982496455 -121.56577004206424) bank199518 +199519 POINT(38.292317411625696 -122.36622232518222) bank199519 +199520 POINT(38.72122860617244 -121.87786323471911) bank199520 +199521 POINT(38.6620371777644 -121.69224606792196) bank199521 +199522 POINT(37.71305398582288 -122.11385127518356) bank199522 +199523 POINT(37.74409427201763 -121.57386044834766) bank199523 +199524 POINT(37.17474646895962 -123.02811692890543) bank199524 +199525 POINT(37.789436834071495 -122.48759787520021) bank199525 +199526 POINT(37.28776796518591 -122.42386364772194) bank199526 +199527 POINT(36.95964438853293 -122.60669555869418) bank199527 +199528 POINT(37.66366382378697 -123.07915560760419) bank199528 +199529 POINT(36.77791883205797 -121.43547048860951) bank199529 +199530 POINT(37.53546451515573 -121.9974403998516) bank199530 +199531 POINT(37.33667137526419 -121.5623490101002) bank199531 +199532 POINT(37.28073597689306 -121.88603926191581) bank199532 +199533 POINT(38.452854483326824 -121.50511796321263) bank199533 +199534 POINT(38.61396162709851 -122.01598652718509) bank199534 +199535 POINT(37.453033316166035 -122.78057242566112) bank199535 +199536 POINT(38.24258986075123 -121.94487770085465) bank199536 +199537 POINT(37.20910683363504 -122.34894399213988) bank199537 +199538 POINT(37.747196883180145 -123.40716593111223) bank199538 +199539 POINT(38.21893525948608 -122.2281022774494) bank199539 +199540 POINT(38.74090324721263 -121.73982250340211) bank199540 +199541 POINT(36.980616676424916 -123.16935673653703) bank199541 +199542 POINT(37.35453759417078 -122.84681586196989) bank199542 +199543 POINT(38.17370438232275 -123.17980682614869) bank199543 +199544 POINT(38.578982000191886 -122.16967249284647) bank199544 +199545 POINT(37.6987901614141 -122.31430007060241) bank199545 +199546 POINT(38.075441008029856 -122.6192712218676) bank199546 +199547 POINT(36.85201688973594 -122.61633785153508) bank199547 +199548 POINT(37.74578206777972 -121.60606745088751) bank199548 +199549 POINT(37.2381299102704 -123.11623556695889) bank199549 +199550 POINT(37.50839805974919 -121.87415078023945) bank199550 +199551 POINT(37.08580010921143 -123.11380761569136) bank199551 +199552 POINT(38.340203162687274 -122.10609576147476) bank199552 +199553 POINT(38.358223307370416 -122.59458532985796) bank199553 +199554 POINT(37.49601950321998 -121.9103112725297) bank199554 +199555 POINT(37.257207477654745 -122.2209993113251) bank199555 +199556 POINT(36.87101171157522 -122.23589904494965) bank199556 +199557 POINT(37.08838736543606 -123.14336571798667) bank199557 +199558 POINT(38.38644970040524 -121.8992501618169) bank199558 +199559 POINT(37.39687052598805 -121.5128785881319) bank199559 +199560 POINT(38.24868285991793 -122.36719517288968) bank199560 +199561 POINT(38.55283036272667 -122.03152992980677) bank199561 +199562 POINT(37.58817258216539 -122.80113863843029) bank199562 +199563 POINT(37.305005194158255 -121.43376746957517) bank199563 +199564 POINT(37.99295409355793 -121.91503741122449) bank199564 +199565 POINT(37.459423478265144 -123.30091661631789) bank199565 +199566 POINT(38.74169587620262 -122.34909848971797) bank199566 +199567 POINT(37.8152559908013 -122.05723402630002) bank199567 +199568 POINT(38.18704151267765 -123.40161458650036) bank199568 +199569 POINT(37.84384280328653 -121.84333119563561) bank199569 +199570 POINT(37.842073809885136 -122.01476998826435) bank199570 +199571 POINT(38.143154406013466 -123.24217211019935) bank199571 +199572 POINT(37.0426350064343 -122.90180567576901) bank199572 +199573 POINT(37.73403062365238 -122.62641236873742) bank199573 +199574 POINT(36.857818996872105 -121.62246659598726) bank199574 +199575 POINT(37.20204849950718 -123.37995138069819) bank199575 +199576 POINT(37.21936764235515 -123.34976595344901) bank199576 +199577 POINT(37.416210437164 -122.46663947048627) bank199577 +199578 POINT(37.07397189522009 -121.78948075017902) bank199578 +199579 POINT(37.78433089545186 -122.48081691748042) bank199579 +199580 POINT(38.53008920244722 -122.09173068150648) bank199580 +199581 POINT(38.55065683680451 -123.13525897305655) bank199581 +199582 POINT(38.16228372561786 -121.69777417070546) bank199582 +199583 POINT(37.02615741842785 -122.53226446579635) bank199583 +199584 POINT(38.36872662615842 -121.78108754451031) bank199584 +199585 POINT(37.069491959812794 -122.87409119870227) bank199585 +199586 POINT(37.683720653480705 -121.80298547963444) bank199586 +199587 POINT(38.508790209862994 -123.32925990316163) bank199587 +199588 POINT(37.05811844518328 -121.47543628359868) bank199588 +199589 POINT(37.16654784657969 -122.03436790240245) bank199589 +199590 POINT(37.60131620041668 -123.34423042334873) bank199590 +199591 POINT(37.957093466633886 -121.60927829115623) bank199591 +199592 POINT(37.61102698973012 -122.68207852875089) bank199592 +199593 POINT(37.588342831905685 -123.06252790846135) bank199593 +199594 POINT(37.316022568721266 -121.63155770915571) bank199594 +199595 POINT(38.647632568688294 -121.56800844689562) bank199595 +199596 POINT(38.2728618127951 -121.94946233346796) bank199596 +199597 POINT(37.69055380737902 -121.43046867584869) bank199597 +199598 POINT(37.67263588305722 -121.49896838043473) bank199598 +199599 POINT(37.91946352237001 -122.58841780366691) bank199599 +199600 POINT(37.94325279221421 -122.37436953470846) bank199600 +199601 POINT(37.93976891467143 -122.64083768433213) bank199601 +199602 POINT(37.26733899848467 -121.78701255176047) bank199602 +199603 POINT(36.854699802369 -122.55511932360415) bank199603 +199604 POINT(38.707281057499806 -121.84758256041168) bank199604 +199605 POINT(38.23784449408464 -122.75788983138874) bank199605 +199606 POINT(38.073714453687195 -123.14058421101652) bank199606 +199607 POINT(37.93480897755056 -121.78199116424481) bank199607 +199608 POINT(37.253191358474695 -123.06853766884676) bank199608 +199609 POINT(38.661940805611735 -121.64545014131059) bank199609 +199610 POINT(38.119229582839765 -122.07540231942622) bank199610 +199611 POINT(38.624771445931586 -122.46238097429965) bank199611 +199612 POINT(38.306686445959734 -122.02648160731046) bank199612 +199613 POINT(38.23783409078104 -121.76829493440806) bank199613 +199614 POINT(38.397865029824246 -123.21173274143005) bank199614 +199615 POINT(38.30613304889072 -121.69717852152691) bank199615 +199616 POINT(37.77381872685844 -122.99560393774874) bank199616 +199617 POINT(38.50460952620855 -122.00928445753918) bank199617 +199618 POINT(37.91906088547128 -123.30130342679757) bank199618 +199619 POINT(38.56952989924374 -121.58997661082502) bank199619 +199620 POINT(37.25105945436715 -123.34349680548074) bank199620 +199621 POINT(37.75338456237141 -122.84760276953087) bank199621 +199622 POINT(37.90286920066405 -122.61960209120744) bank199622 +199623 POINT(37.532940541601356 -121.8903997714464) bank199623 +199624 POINT(37.37673547067691 -121.75150879369845) bank199624 +199625 POINT(37.130892086390126 -122.68569847880609) bank199625 +199626 POINT(37.52950543623757 -123.10520372066331) bank199626 +199627 POINT(37.490564062486904 -122.21380841821708) bank199627 +199628 POINT(37.83938284081447 -123.15517505421619) bank199628 +199629 POINT(38.029755596388604 -122.03487815553201) bank199629 +199630 POINT(38.71128921019687 -122.39208338358667) bank199630 +199631 POINT(38.18517295694392 -122.70524830598775) bank199631 +199632 POINT(36.78162059252668 -122.9613086849027) bank199632 +199633 POINT(36.841363211173245 -122.60504158681455) bank199633 +199634 POINT(38.69318788160898 -121.98544219393598) bank199634 +199635 POINT(37.526706024973684 -122.2696890636316) bank199635 +199636 POINT(37.27688508544158 -122.54603068537699) bank199636 +199637 POINT(38.22251431564285 -122.09862370859902) bank199637 +199638 POINT(37.50158479649957 -121.67149775082507) bank199638 +199639 POINT(37.69367723413169 -123.35274066365305) bank199639 +199640 POINT(37.687163818278705 -122.8452270254913) bank199640 +199641 POINT(38.24336973465199 -123.40260577704579) bank199641 +199642 POINT(36.997810249168396 -122.27011864155509) bank199642 +199643 POINT(37.02454931141479 -122.66202230823832) bank199643 +199644 POINT(37.015309009750844 -121.79804112111464) bank199644 +199645 POINT(37.04444480058362 -122.81813250492382) bank199645 +199646 POINT(36.79504320736838 -122.05987686918333) bank199646 +199647 POINT(37.14866673090234 -121.71710930371108) bank199647 +199648 POINT(37.325360001911754 -123.22480495271084) bank199648 +199649 POINT(37.74975470736574 -123.19455541530048) bank199649 +199650 POINT(37.17263462834651 -123.39292042261124) bank199650 +199651 POINT(36.829648591966496 -121.64858259741895) bank199651 +199652 POINT(38.071841524136886 -122.34023162474139) bank199652 +199653 POINT(37.72981761436815 -123.08252641662139) bank199653 +199654 POINT(37.422439016424306 -122.63102291434062) bank199654 +199655 POINT(37.15669103327773 -122.9266471592178) bank199655 +199656 POINT(37.93666796550342 -123.34131484644954) bank199656 +199657 POINT(37.05570619097674 -122.8895930209143) bank199657 +199658 POINT(37.9284325350187 -121.57639745024903) bank199658 +199659 POINT(37.68167918819015 -122.50718379709159) bank199659 +199660 POINT(38.610543101327885 -121.43350347220894) bank199660 +199661 POINT(38.627619281164165 -122.47096971318113) bank199661 +199662 POINT(38.339414722002225 -121.8131507297975) bank199662 +199663 POINT(38.0595985501528 -122.89733172446229) bank199663 +199664 POINT(37.16001781409985 -121.57675406239186) bank199664 +199665 POINT(37.78022145481083 -122.34669559860819) bank199665 +199666 POINT(38.09094920343807 -121.90872707273961) bank199666 +199667 POINT(38.14362243768359 -122.18932114343428) bank199667 +199668 POINT(38.51576746173585 -122.42580788083215) bank199668 +199669 POINT(38.233622007500145 -122.94573602313673) bank199669 +199670 POINT(37.39496074508485 -122.1239201278681) bank199670 +199671 POINT(38.11091276585635 -123.38297519099876) bank199671 +199672 POINT(37.41523412942044 -122.05261696576083) bank199672 +199673 POINT(36.97762148148452 -123.02507162690414) bank199673 +199674 POINT(36.88771953910258 -122.57226180594716) bank199674 +199675 POINT(36.856287858011754 -121.71350810734674) bank199675 +199676 POINT(37.05182785911293 -122.40174650776179) bank199676 +199677 POINT(38.21918516864347 -122.36553315001325) bank199677 +199678 POINT(38.18075002404191 -121.70735034671785) bank199678 +199679 POINT(38.34225337959722 -123.30003282105213) bank199679 +199680 POINT(38.27226330696821 -123.20190116422613) bank199680 +199681 POINT(36.85429447898085 -121.99826527858545) bank199681 +199682 POINT(37.67053057162756 -121.780705025419) bank199682 +199683 POINT(38.74109850712456 -123.23928901546263) bank199683 +199684 POINT(37.60980892159179 -121.65542830610612) bank199684 +199685 POINT(37.861759977459634 -122.74057474535824) bank199685 +199686 POINT(37.15869953877859 -121.42409767686847) bank199686 +199687 POINT(38.76016590911053 -122.57739520038629) bank199687 +199688 POINT(38.42351608366678 -121.71524697274366) bank199688 +199689 POINT(37.032803056239686 -121.85369428626434) bank199689 +199690 POINT(38.70557885474351 -121.92630247614771) bank199690 +199691 POINT(37.992313920773114 -121.90346621898112) bank199691 +199692 POINT(37.60763663156436 -121.7659183560557) bank199692 +199693 POINT(37.2097052126896 -123.30275378213933) bank199693 +199694 POINT(36.81644646995414 -121.4332888711923) bank199694 +199695 POINT(37.60046724780744 -122.00340676303682) bank199695 +199696 POINT(37.42138587919202 -121.98611004813313) bank199696 +199697 POINT(38.35042020848671 -121.63106265949109) bank199697 +199698 POINT(37.07287726294419 -121.51143862993312) bank199698 +199699 POINT(37.734521948395795 -122.89300069094607) bank199699 +199700 POINT(37.297137359909236 -122.76302569008274) bank199700 +199701 POINT(38.772066062110895 -122.54156703927106) bank199701 +199702 POINT(37.82009225214465 -121.91587605555534) bank199702 +199703 POINT(38.659616771506506 -122.14310389356446) bank199703 +199704 POINT(36.86099618696535 -122.1165237721068) bank199704 +199705 POINT(38.13569557007299 -121.44629238322675) bank199705 +199706 POINT(37.6205337671182 -121.924727956022) bank199706 +199707 POINT(38.532893438243114 -122.0635872451989) bank199707 +199708 POINT(38.43537517025417 -121.866510416379) bank199708 +199709 POINT(36.98184063365028 -122.41534256735665) bank199709 +199710 POINT(37.36829261947611 -122.47534466952787) bank199710 +199711 POINT(37.239040102682544 -123.35621582878679) bank199711 +199712 POINT(37.79317669523827 -122.08184233749834) bank199712 +199713 POINT(37.34137130837958 -122.62725795789072) bank199713 +199714 POINT(38.34723124289905 -121.52197524399794) bank199714 +199715 POINT(37.667752206023756 -121.61861586219439) bank199715 +199716 POINT(37.88543440370578 -122.15198874041678) bank199716 +199717 POINT(37.55653876506057 -122.88064227250351) bank199717 +199718 POINT(38.56490314583182 -122.76371681540803) bank199718 +199719 POINT(38.715500248422394 -121.8866181857053) bank199719 +199720 POINT(36.87072179704317 -122.9445792204543) bank199720 +199721 POINT(37.00845182821099 -122.39580916343786) bank199721 +199722 POINT(38.055425075366756 -122.62736128021201) bank199722 +199723 POINT(37.35089062967386 -122.87877483735177) bank199723 +199724 POINT(37.3622643728506 -123.18109076920197) bank199724 +199725 POINT(38.32610067007668 -122.79082229616552) bank199725 +199726 POINT(37.973904193151206 -122.37796096507364) bank199726 +199727 POINT(36.95229144473994 -122.12986080875208) bank199727 +199728 POINT(38.6772277706716 -122.50088053063082) bank199728 +199729 POINT(38.529255546120076 -122.61338349644426) bank199729 +199730 POINT(37.44244910153704 -121.84728382777023) bank199730 +199731 POINT(38.29225503703995 -123.29352013556604) bank199731 +199732 POINT(38.24708654749655 -122.1206175697234) bank199732 +199733 POINT(37.68830191946378 -122.26290950567342) bank199733 +199734 POINT(36.94492023179176 -123.28672066846266) bank199734 +199735 POINT(38.713513914814726 -122.20837907270241) bank199735 +199736 POINT(37.2515265020142 -122.98552296484617) bank199736 +199737 POINT(37.86341972926087 -122.16562740924986) bank199737 +199738 POINT(36.88909365451975 -122.54348286453565) bank199738 +199739 POINT(36.86370352343157 -121.93854339242546) bank199739 +199740 POINT(38.28177679784783 -122.03398232893376) bank199740 +199741 POINT(38.02931546047947 -121.97142996783766) bank199741 +199742 POINT(37.173915045635425 -121.46390814038357) bank199742 +199743 POINT(36.841254015304244 -122.96557632641817) bank199743 +199744 POINT(38.768922748150715 -122.23088728851694) bank199744 +199745 POINT(38.19611259786303 -122.16774428392918) bank199745 +199746 POINT(38.52841020386589 -122.5083562916844) bank199746 +199747 POINT(38.71041214513534 -122.55818204583025) bank199747 +199748 POINT(37.72700015133878 -122.1863060766816) bank199748 +199749 POINT(38.11192389436416 -122.51587785956947) bank199749 +199750 POINT(38.07403900041901 -121.4366788486765) bank199750 +199751 POINT(37.74268880542546 -122.3460553297464) bank199751 +199752 POINT(37.62090895609551 -121.81864455681911) bank199752 +199753 POINT(37.23733409031251 -121.57438386323811) bank199753 +199754 POINT(38.1269351964339 -122.31009491101116) bank199754 +199755 POINT(36.900967057086106 -122.3577762419396) bank199755 +199756 POINT(38.43334513739342 -122.86628605742976) bank199756 +199757 POINT(37.184868910361644 -123.07477905102724) bank199757 +199758 POINT(37.30064601318024 -123.23277846966747) bank199758 +199759 POINT(38.35250651501598 -122.71267193448656) bank199759 +199760 POINT(37.23842888437673 -122.75007896626755) bank199760 +199761 POINT(38.01458682926186 -122.06760540951234) bank199761 +199762 POINT(38.19743649184997 -122.49910197066389) bank199762 +199763 POINT(38.032334710150835 -122.61426633619729) bank199763 +199764 POINT(37.484333803582295 -121.76187122132781) bank199764 +199765 POINT(37.231245363679896 -121.52956624783852) bank199765 +199766 POINT(37.60680541273012 -121.81859202086808) bank199766 +199767 POINT(37.7462845653144 -121.79440373223093) bank199767 +199768 POINT(38.23377989338778 -122.09684420962354) bank199768 +199769 POINT(37.10466567417223 -122.52558484199659) bank199769 +199770 POINT(37.56283873343689 -122.45952290104606) bank199770 +199771 POINT(37.29687132532804 -122.12622756734835) bank199771 +199772 POINT(36.94520410188334 -121.8774605596956) bank199772 +199773 POINT(36.84611593692031 -121.83429198677867) bank199773 +199774 POINT(37.58874798612882 -121.63289236725943) bank199774 +199775 POINT(37.26691365111334 -123.2146655234426) bank199775 +199776 POINT(37.88813690083571 -122.555750465218) bank199776 +199777 POINT(37.56061848985295 -122.93720042881479) bank199777 +199778 POINT(37.83037288988307 -122.57817706702366) bank199778 +199779 POINT(37.2799390205667 -122.88521116112557) bank199779 +199780 POINT(37.011853023101196 -123.01661234030442) bank199780 +199781 POINT(37.62390819661822 -121.94497172169429) bank199781 +199782 POINT(37.69750445151571 -122.06896997532024) bank199782 +199783 POINT(38.77294656517083 -123.13020262201367) bank199783 +199784 POINT(36.94224864154959 -122.73498436715164) bank199784 +199785 POINT(38.000688567917656 -121.8598088185549) bank199785 +199786 POINT(37.31285062984822 -122.4259937539143) bank199786 +199787 POINT(38.63551532316935 -122.3121142718845) bank199787 +199788 POINT(38.57321119483048 -122.38785209502632) bank199788 +199789 POINT(37.26695488548858 -122.36809680824022) bank199789 +199790 POINT(37.62328269048351 -121.97474891166581) bank199790 +199791 POINT(37.54140899737099 -122.85461185639703) bank199791 +199792 POINT(37.37605755512691 -122.4630695112946) bank199792 +199793 POINT(36.924600280086295 -121.59044992708492) bank199793 +199794 POINT(38.1811415724762 -122.78837311400312) bank199794 +199795 POINT(38.65598078754875 -123.17529883254593) bank199795 +199796 POINT(37.697616893402774 -122.93818526312701) bank199796 +199797 POINT(38.10089904827307 -122.18405804434406) bank199797 +199798 POINT(37.507786693454904 -122.86726025313301) bank199798 +199799 POINT(37.49363921670574 -121.49660238771006) bank199799 +199800 POINT(37.06877649307958 -123.28866907797496) bank199800 +199801 POINT(38.41433575016057 -121.94562143615146) bank199801 +199802 POINT(38.17184541366893 -121.59404251135328) bank199802 +199803 POINT(38.457514178848555 -121.90315064574213) bank199803 +199804 POINT(38.44519435328622 -123.1004287242987) bank199804 +199805 POINT(37.35351073024398 -121.52046927789894) bank199805 +199806 POINT(37.9010835049221 -121.99132205536206) bank199806 +199807 POINT(38.07013007166784 -121.92550933234497) bank199807 +199808 POINT(37.85211493863628 -123.14085218780701) bank199808 +199809 POINT(37.23428361805321 -122.36940997128173) bank199809 +199810 POINT(37.90648261558493 -122.46286449984416) bank199810 +199811 POINT(36.94238244094955 -123.39654558815322) bank199811 +199812 POINT(37.83682800175893 -121.46908249490498) bank199812 +199813 POINT(37.233093552182254 -122.65740795480261) bank199813 +199814 POINT(38.577164852304435 -122.5291233494347) bank199814 +199815 POINT(38.613889725242 -122.26378316633257) bank199815 +199816 POINT(38.14213867047563 -122.14186155963336) bank199816 +199817 POINT(38.625840838915266 -121.61682875258198) bank199817 +199818 POINT(37.85271464611691 -122.10031646258618) bank199818 +199819 POINT(38.72800719518307 -121.79767899950805) bank199819 +199820 POINT(37.40102401989251 -122.43545097999191) bank199820 +199821 POINT(36.883146203986705 -122.53340067032234) bank199821 +199822 POINT(38.28095724655236 -122.99615203491844) bank199822 +199823 POINT(37.00685362963199 -121.95648597867404) bank199823 +199824 POINT(38.58301296135529 -122.44023503095316) bank199824 +199825 POINT(38.33397534412429 -123.07478632121742) bank199825 +199826 POINT(36.818399374669845 -121.76857174175247) bank199826 +199827 POINT(36.776854969564916 -121.70347624125822) bank199827 +199828 POINT(38.66554857047382 -121.56340138647231) bank199828 +199829 POINT(37.93662957118005 -123.368395509687) bank199829 +199830 POINT(37.23841182550101 -121.51383758162356) bank199830 +199831 POINT(38.21361589018344 -121.4390527898126) bank199831 +199832 POINT(37.53874485978058 -122.45499840632046) bank199832 +199833 POINT(36.89805415262115 -122.29460494294159) bank199833 +199834 POINT(37.90977431493198 -122.58673560283196) bank199834 +199835 POINT(37.83604606090479 -123.06185581733983) bank199835 +199836 POINT(38.12078675893522 -122.74923863616661) bank199836 +199837 POINT(37.31542382093712 -123.3833909079188) bank199837 +199838 POINT(37.51091126189784 -123.1259536540301) bank199838 +199839 POINT(38.70282137794012 -121.60678916620516) bank199839 +199840 POINT(37.51329969275948 -121.74592074485182) bank199840 +199841 POINT(37.58694788704407 -122.93603896592231) bank199841 +199842 POINT(38.37414405410054 -122.34461739208837) bank199842 +199843 POINT(37.72343170826364 -123.30690549073154) bank199843 +199844 POINT(37.56319848349297 -121.74179740850222) bank199844 +199845 POINT(37.78666206290119 -122.06561524031912) bank199845 +199846 POINT(36.997347748877864 -121.5922976952233) bank199846 +199847 POINT(37.751233621866554 -122.63547250538409) bank199847 +199848 POINT(38.51118290816558 -122.26865785182055) bank199848 +199849 POINT(37.65448522705278 -122.98468121845491) bank199849 +199850 POINT(37.00314042643417 -122.8642112806743) bank199850 +199851 POINT(36.886600642873816 -122.0300651938243) bank199851 +199852 POINT(37.91723812802208 -122.24339105579489) bank199852 +199853 POINT(38.04849463838118 -121.91703602786521) bank199853 +199854 POINT(38.13176694764219 -121.72734869072491) bank199854 +199855 POINT(37.87826875930042 -122.61740900326826) bank199855 +199856 POINT(38.230009717686436 -122.90892865967334) bank199856 +199857 POINT(37.71065344594376 -123.34471980649569) bank199857 +199858 POINT(37.68040156308329 -122.26596001507707) bank199858 +199859 POINT(37.64191442688168 -122.4696617116453) bank199859 +199860 POINT(38.61532292029153 -123.3701996165219) bank199860 +199861 POINT(38.28931525895781 -123.40667627334567) bank199861 +199862 POINT(38.670641472834006 -123.39673048694672) bank199862 +199863 POINT(37.83434126618355 -122.93800073487586) bank199863 +199864 POINT(37.1620881506383 -122.97211037699893) bank199864 +199865 POINT(37.82779292033373 -121.71234499769369) bank199865 +199866 POINT(37.3904272386778 -121.59283479336925) bank199866 +199867 POINT(37.56156109613065 -123.19353603250937) bank199867 +199868 POINT(38.59029407530643 -122.77117007458607) bank199868 +199869 POINT(37.79804285123375 -122.91600184178671) bank199869 +199870 POINT(38.57212037243157 -123.00269980637826) bank199870 +199871 POINT(37.007448810955914 -123.08143782026039) bank199871 +199872 POINT(37.66851851854321 -121.96657875141727) bank199872 +199873 POINT(37.652689872039005 -122.83245104984157) bank199873 +199874 POINT(37.73150554652424 -122.27980245446176) bank199874 +199875 POINT(37.57686990936466 -122.12885837698116) bank199875 +199876 POINT(37.317264033626884 -122.30369847035735) bank199876 +199877 POINT(37.94127106800423 -123.34459924504519) bank199877 +199878 POINT(37.262171501915255 -122.69958181576592) bank199878 +199879 POINT(37.94933290129426 -123.31554152341133) bank199879 +199880 POINT(36.932721911832125 -121.67668156439854) bank199880 +199881 POINT(38.35765047763965 -123.3694354936432) bank199881 +199882 POINT(38.677584279683494 -122.53090697261638) bank199882 +199883 POINT(37.36723186798593 -122.01976401982026) bank199883 +199884 POINT(38.57569967588529 -121.79921548371232) bank199884 +199885 POINT(37.170915769482114 -121.96439350135104) bank199885 +199886 POINT(38.63016186345739 -122.3546700582576) bank199886 +199887 POINT(37.2322325594233 -123.10141057800426) bank199887 +199888 POINT(37.07881038591124 -122.69592592688605) bank199888 +199889 POINT(37.17792850186967 -122.5899231009903) bank199889 +199890 POINT(38.629435051045036 -122.98088860752551) bank199890 +199891 POINT(38.48723645443327 -123.2891430553011) bank199891 +199892 POINT(36.85673253822591 -122.64143563510856) bank199892 +199893 POINT(38.27770176476604 -122.04166390600702) bank199893 +199894 POINT(38.76316544979931 -122.3897609502237) bank199894 +199895 POINT(38.51727580789937 -123.23802552508633) bank199895 +199896 POINT(38.29258163292135 -122.05799723130005) bank199896 +199897 POINT(38.650958720471536 -122.52139267985929) bank199897 +199898 POINT(37.88346603601432 -122.59090835630589) bank199898 +199899 POINT(38.409681766883416 -121.8446309630793) bank199899 +199900 POINT(37.70469638458432 -122.20879028917376) bank199900 +199901 POINT(38.20165644170236 -122.61114964162415) bank199901 +199902 POINT(38.44502391047136 -123.05940294109568) bank199902 +199903 POINT(38.24326976763878 -122.64636629683369) bank199903 +199904 POINT(37.79925249972277 -122.18123420705594) bank199904 +199905 POINT(38.34099937659726 -122.42928433893186) bank199905 +199906 POINT(37.12351871588724 -122.11389745208076) bank199906 +199907 POINT(38.07754456276686 -122.59462824372987) bank199907 +199908 POINT(38.205371101055555 -121.99598796918634) bank199908 +199909 POINT(37.01742168751418 -123.17243533866181) bank199909 +199910 POINT(38.28436452903156 -122.19803190471221) bank199910 +199911 POINT(38.353264623359124 -121.55505258287903) bank199911 +199912 POINT(38.631218317361736 -122.63669650781655) bank199912 +199913 POINT(38.74523797886822 -122.53429279868107) bank199913 +199914 POINT(37.319435756425854 -121.72652910272154) bank199914 +199915 POINT(37.54776565164329 -123.27322432280559) bank199915 +199916 POINT(38.32817098331689 -123.2825386536544) bank199916 +199917 POINT(38.0538151940236 -122.69261050605768) bank199917 +199918 POINT(37.81939439027598 -122.86772336717502) bank199918 +199919 POINT(37.59329745707364 -122.79986097154651) bank199919 +199920 POINT(37.497606102298555 -122.67031479895002) bank199920 +199921 POINT(37.26808313568666 -123.11751536900604) bank199921 +199922 POINT(37.263538300308795 -123.35785407823705) bank199922 +199923 POINT(38.018085808714005 -123.35727674467117) bank199923 +199924 POINT(37.19718147950145 -122.23648616558737) bank199924 +199925 POINT(38.274918424703124 -122.08580240602804) bank199925 +199926 POINT(37.809748235278335 -121.9174581885453) bank199926 +199927 POINT(36.905822930277274 -122.48442505953155) bank199927 +199928 POINT(37.47278168173009 -122.33526246222479) bank199928 +199929 POINT(37.225425309694295 -121.48316444892436) bank199929 +199930 POINT(36.91846993145687 -122.66351481794909) bank199930 +199931 POINT(37.97775973598181 -122.31943526893532) bank199931 +199932 POINT(37.54099245367683 -122.32619628411004) bank199932 +199933 POINT(37.79868689964308 -122.2679134077421) bank199933 +199934 POINT(38.7003157693483 -121.90129073311566) bank199934 +199935 POINT(36.82490854303519 -123.04434180173931) bank199935 +199936 POINT(36.780900972767924 -121.542744328847) bank199936 +199937 POINT(38.415475841956386 -121.99436636718187) bank199937 +199938 POINT(37.44568158342086 -122.54934956683472) bank199938 +199939 POINT(37.84086665479603 -123.04185678606198) bank199939 +199940 POINT(37.936615745016475 -122.64607382867777) bank199940 +199941 POINT(38.22867052504896 -123.22682395654492) bank199941 +199942 POINT(37.86277660926954 -123.36055125365579) bank199942 +199943 POINT(37.801327109800084 -121.87934059771588) bank199943 +199944 POINT(36.86221544826897 -121.98164964746829) bank199944 +199945 POINT(37.02311739371143 -121.7723481435513) bank199945 +199946 POINT(37.34428675510974 -122.26204617973195) bank199946 +199947 POINT(37.16965287691896 -122.98950879597865) bank199947 +199948 POINT(36.84924894287159 -123.00597006353605) bank199948 +199949 POINT(38.43835785082301 -123.31196324652609) bank199949 +199950 POINT(38.53174092990798 -123.41279588184275) bank199950 +199951 POINT(38.75450099371913 -121.51638737263771) bank199951 +199952 POINT(36.949196539486344 -121.59591193613171) bank199952 +199953 POINT(37.77424766069698 -122.53588581230818) bank199953 +199954 POINT(38.48825291663481 -121.8450082671836) bank199954 +199955 POINT(37.791268479231015 -121.74358284653928) bank199955 +199956 POINT(38.055212707742726 -123.30943925314665) bank199956 +199957 POINT(38.213782338620824 -122.54044081439545) bank199957 +199958 POINT(37.50391727581821 -121.7859843142198) bank199958 +199959 POINT(38.71966098363088 -121.53918097636121) bank199959 +199960 POINT(37.16207669987317 -123.06456099877167) bank199960 +199961 POINT(38.164096512746916 -122.3030919947507) bank199961 +199962 POINT(38.399601557589975 -122.57434599406217) bank199962 +199963 POINT(38.00260368341836 -123.0253038640586) bank199963 +199964 POINT(37.35121441861659 -122.10140280944717) bank199964 +199965 POINT(36.80733711195495 -123.08505198983973) bank199965 +199966 POINT(37.666202241228 -121.78957499812863) bank199966 +199967 POINT(38.310920145883124 -122.37568640732495) bank199967 +199968 POINT(37.10261019172824 -122.94267465122104) bank199968 +199969 POINT(38.033910085014284 -122.76638144566904) bank199969 +199970 POINT(36.96721901560808 -121.49584994990511) bank199970 +199971 POINT(37.09487511031312 -122.90691159247007) bank199971 +199972 POINT(37.28538851112734 -122.23195140006811) bank199972 +199973 POINT(37.167351545421596 -122.11633149689995) bank199973 +199974 POINT(37.62721258292321 -122.20460610335377) bank199974 +199975 POINT(37.32379403008178 -121.61509140730696) bank199975 +199976 POINT(37.06873574995303 -121.508600973097) bank199976 +199977 POINT(38.2888625485234 -123.18794193401432) bank199977 +199978 POINT(38.75552324137809 -121.79173233180394) bank199978 +199979 POINT(37.12758117084709 -121.867092611777) bank199979 +199980 POINT(37.54776603725335 -121.6312257016849) bank199980 +199981 POINT(37.24526761679586 -121.70181427736911) bank199981 +199982 POINT(37.25926558684621 -121.55948328303376) bank199982 +199983 POINT(38.56431503711176 -122.99220977305067) bank199983 +199984 POINT(36.831648593930375 -122.37280258599844) bank199984 +199985 POINT(37.73173462889223 -122.17064796314648) bank199985 +199986 POINT(37.31380701873798 -123.30197432526876) bank199986 +199987 POINT(37.41117057301311 -122.41278058969424) bank199987 +199988 POINT(38.11296919839651 -123.06848000906426) bank199988 +199989 POINT(37.296996300413774 -122.35899948937619) bank199989 +199990 POINT(38.55701550995962 -122.66945952019834) bank199990 +199991 POINT(37.97979368703362 -123.32654968227052) bank199991 +199992 POINT(38.68001767090207 -121.7209214407274) bank199992 +199993 POINT(37.83652799803407 -122.23336732160021) bank199993 +199994 POINT(37.740619283113475 -122.16679020537882) bank199994 +199995 POINT(38.6589233659295 -122.13925889634828) bank199995 +199996 POINT(38.3549315285339 -122.71840242261818) bank199996 +199997 POINT(37.74354398304942 -122.51721144772974) bank199997 +199998 POINT(38.694217711279066 -123.22845490545268) bank199998 +199999 POINT(37.8154884631528 -122.679604277726) bank199999 diff --git a/spark/common/src/test/resources/knn/queries-large-skewed.csv b/spark/common/src/test/resources/knn/queries-large-skewed.csv new file mode 100644 index 0000000000..2883c49e69 --- /dev/null +++ b/spark/common/src/test/resources/knn/queries-large-skewed.csv @@ -0,0 +1,199981 @@ +1 POINT(20.0 60.0) bank1 +21 POINT(41.06581944577888 73.31615251351968) bank21 +22 POINT(39.731104837384095 74.92491051478515) bank22 +23 POINT(39.76431741716018 74.36574348983969) bank23 +24 POINT(41.342112923966354 74.63306409401739) bank24 +25 POINT(41.051996514312236 73.96112708958357) bank25 +26 POINT(40.845889116085154 73.1552893374506) bank26 +27 POINT(41.40256108110186 73.93542009308608) bank27 +28 POINT(40.69343389464201 73.23929230244245) bank28 +29 POINT(39.76849311506865 73.91422869553591) bank29 +30 POINT(40.4630508879632 73.23950960164505) bank30 +31 POINT(40.66362144999636 74.85942116563137) bank31 +32 POINT(41.449779131121694 74.58385702035902) bank32 +33 POINT(40.645115197588666 74.91249420365196) bank33 +34 POINT(40.197487267756486 73.82249232732843) bank34 +35 POINT(39.87356614900357 73.66572364615054) bank35 +36 POINT(40.469591008964954 74.92210850819771) bank36 +37 POINT(41.20226703447333 73.87038699367994) bank37 +38 POINT(41.65339104022265 74.6511097409152) bank38 +39 POINT(41.19879907291347 74.74477353725884) bank39 +40 POINT(40.11306489047681 73.18804412282742) bank40 +41 POINT(41.06017358480731 74.35179533337502) bank41 +42 POINT(40.711392474473534 74.84830291372651) bank42 +43 POINT(39.80416891184534 73.05764381485413) bank43 +44 POINT(40.13228548261097 74.82292494455957) bank44 +45 POINT(40.42250593229736 74.14398025820545) bank45 +46 POINT(41.40373132270695 73.92842265199826) bank46 +47 POINT(41.33348933462816 73.42276806882008) bank47 +48 POINT(41.585734453492165 73.8214136932506) bank48 +49 POINT(40.7018392380655 73.27184481798118) bank49 +50 POINT(41.48813610271762 73.37783087145522) bank50 +51 POINT(40.64623199364555 74.66748756591636) bank51 +52 POINT(39.72931677748655 73.25023760993041) bank52 +53 POINT(40.21296837212506 74.93177851639554) bank53 +54 POINT(40.34339612069318 73.42884347945446) bank54 +55 POINT(39.878645812857485 74.0689599447922) bank55 +56 POINT(40.0603141745698 74.00257171345234) bank56 +57 POINT(41.23481425798163 74.02011233405794) bank57 +58 POINT(40.14515990019877 74.10056362669901) bank58 +59 POINT(40.548271480294375 73.70422002922278) bank59 +60 POINT(41.525624195657905 73.4296810619291) bank60 +61 POINT(40.76510379378682 74.62005925533028) bank61 +62 POINT(40.94423043959472 74.44422351798096) bank62 +63 POINT(41.390984369705926 74.83894282630759) bank63 +64 POINT(40.45141051784711 74.50732356737201) bank64 +65 POINT(41.181924447893934 73.35925286105211) bank65 +66 POINT(40.18803429875317 74.81229558193145) bank66 +67 POINT(40.77128106106234 74.53233490384861) bank67 +68 POINT(40.272066786709296 73.86572993371212) bank68 +69 POINT(39.94853384947618 73.84537107456364) bank69 +70 POINT(41.4582795284433 74.5330287313871) bank70 +71 POINT(40.695334572240604 74.66239079660464) bank71 +72 POINT(40.2537507265174 73.43577551968441) bank72 +73 POINT(40.025931497043054 74.23678929452919) bank73 +74 POINT(40.492007245632585 74.07040330927714) bank74 +75 POINT(40.823827963293226 73.14697685036136) bank75 +76 POINT(39.86924452900078 73.5941649436671) bank76 +77 POINT(40.57770499372453 74.71059179980047) bank77 +78 POINT(39.855066084256634 73.7648502645565) bank78 +79 POINT(40.57430102945947 74.14341354776) bank79 +80 POINT(41.57519663830354 74.09427573107344) bank80 +81 POINT(39.90608653267727 74.8158084999649) bank81 +82 POINT(40.066337768222034 74.92048131414685) bank82 +83 POINT(41.652879459798825 74.51853822662851) bank83 +84 POINT(40.832354461490205 73.43370774451415) bank84 +85 POINT(40.8529802098403 74.9235246991929) bank85 +86 POINT(39.844289456007964 73.29690485936291) bank86 +87 POINT(39.789328516172226 74.7956424108067) bank87 +88 POINT(40.853699434632404 73.06707106427366) bank88 +89 POINT(40.764575782568926 73.11377000497134) bank89 +90 POINT(40.01182905373546 74.4777671722751) bank90 +91 POINT(39.99035125012364 73.91551160144553) bank91 +92 POINT(40.55410895038554 74.34587084387836) bank92 +93 POINT(41.56518857302801 74.43234947417037) bank93 +94 POINT(40.86002261847456 73.86783719902219) bank94 +95 POINT(40.25960476597971 73.89088935901134) bank95 +96 POINT(41.60830210911807 74.83300818778675) bank96 +97 POINT(40.737349930198704 73.51320890837594) bank97 +98 POINT(41.38808051392903 74.38955845727753) bank98 +99 POINT(40.83355123022717 73.44695577466486) bank99 +100 POINT(41.3358608860437 73.71281417777158) bank100 +101 POINT(40.15989945660665 73.28413162006679) bank101 +102 POINT(40.51994625173181 74.64645074663792) bank102 +103 POINT(40.39897694775312 74.4301993505061) bank103 +104 POINT(40.627930821072994 73.08035503424504) bank104 +105 POINT(41.29115146906304 74.45483885975469) bank105 +106 POINT(41.05064930221403 73.00849257956176) bank106 +107 POINT(41.29758481220464 73.49905848416941) bank107 +108 POINT(41.63799079769131 74.45674797481976) bank108 +109 POINT(40.81550839292955 73.8127912872102) bank109 +110 POINT(40.756065652397446 74.41104074548052) bank110 +111 POINT(41.65006289757128 73.10167715593957) bank111 +112 POINT(41.0872898796558 73.67429971914571) bank112 +113 POINT(41.202845575078236 74.74084458071025) bank113 +114 POINT(40.952714582500455 73.97435891900828) bank114 +115 POINT(40.54193484340729 73.31499871809933) bank115 +116 POINT(40.84007555058505 73.50519097283383) bank116 +117 POINT(39.89123096527545 73.20270759561829) bank117 +118 POINT(40.022193692839686 74.39469284144349) bank118 +119 POINT(40.32692770131308 74.14479091445095) bank119 +120 POINT(40.77294080624322 74.60160958059545) bank120 +121 POINT(41.08735241354985 74.80283461700304) bank121 +122 POINT(41.21215913645694 74.7237936636135) bank122 +123 POINT(39.94204929900814 74.75734231766518) bank123 +124 POINT(40.57927948860291 74.23324975501325) bank124 +125 POINT(40.94600198312699 74.13394724643125) bank125 +126 POINT(40.21969980104651 74.61624666690815) bank126 +127 POINT(39.99245470994518 74.4353718046087) bank127 +128 POINT(39.76223227915984 73.61119315259133) bank128 +129 POINT(40.38682807562497 74.44366793408545) bank129 +130 POINT(41.50526818347663 74.8457634768798) bank130 +131 POINT(40.161133167431444 73.94203279336188) bank131 +132 POINT(39.741933891907856 73.28771511223664) bank132 +133 POINT(41.40956154841853 73.39838131750923) bank133 +134 POINT(40.61502089238796 74.84465762964938) bank134 +135 POINT(40.64627160701166 74.14844818900144) bank135 +136 POINT(41.43492909007293 73.48348162333544) bank136 +137 POINT(39.8700621684604 73.98681519712723) bank137 +138 POINT(40.65745033780199 73.71188543043645) bank138 +139 POINT(40.70367281899129 74.28831214892247) bank139 +140 POINT(40.675709331826944 74.92797010425538) bank140 +141 POINT(41.65459050027455 73.37072675723701) bank141 +142 POINT(39.81450094390565 74.63366474040964) bank142 +143 POINT(40.316882445467876 73.34028351993486) bank143 +144 POINT(40.74452546434576 74.72714283862109) bank144 +145 POINT(41.1329553150619 74.09884178850415) bank145 +146 POINT(40.09485469283948 73.40330920672639) bank146 +147 POINT(39.89161045491935 74.57378209416922) bank147 +148 POINT(41.0640056806631 73.64053982344065) bank148 +149 POINT(39.96486679959063 73.42490054176781) bank149 +150 POINT(40.641210211747165 73.43427149714174) bank150 +151 POINT(40.65606665887042 74.48764834873019) bank151 +152 POINT(40.869980984023854 74.17966538725328) bank152 +153 POINT(40.94970830803556 74.47151906987986) bank153 +154 POINT(40.92887548081454 74.80082780437482) bank154 +155 POINT(40.48053811606845 74.57093639470835) bank155 +156 POINT(41.320014107648525 73.55726809580848) bank156 +157 POINT(41.6996726152133 73.35220774659115) bank157 +158 POINT(40.27646301084157 73.13197007502652) bank158 +159 POINT(40.26428271418656 73.49205891026979) bank159 +160 POINT(40.1293600681478 73.92451320032927) bank160 +161 POINT(39.80493161838553 74.05748542368514) bank161 +162 POINT(40.98664019803544 73.18786990687074) bank162 +163 POINT(41.2935522712301 73.35386298140689) bank163 +164 POINT(40.95662451731741 74.03593859955295) bank164 +165 POINT(40.00984901574794 73.6523406347481) bank165 +166 POINT(40.68406192970555 74.78199924687816) bank166 +167 POINT(40.159238657075896 73.78518837447413) bank167 +168 POINT(41.36812651843057 74.78367210157019) bank168 +169 POINT(41.590177140429965 74.1076669695176) bank169 +170 POINT(39.84773389464917 74.77279250129989) bank170 +171 POINT(40.562858956230755 74.8121123630373) bank171 +172 POINT(40.926478904194866 74.34980670145481) bank172 +173 POINT(41.45030475998076 73.13609236617381) bank173 +174 POINT(40.30565591919923 73.76725783919217) bank174 +175 POINT(40.94965907405436 73.87221294419267) bank175 +176 POINT(40.41804502109328 73.44997537857738) bank176 +177 POINT(40.963131571003345 74.47935443305595) bank177 +178 POINT(40.59054424383208 73.53573624837428) bank178 +179 POINT(40.237073096298985 74.20661948897181) bank179 +180 POINT(39.986319274012416 74.78860314844361) bank180 +181 POINT(41.708642118486885 73.7030683648872) bank181 +182 POINT(41.07418562749102 74.93367367369846) bank182 +183 POINT(39.97235436797619 73.80835495417375) bank183 +184 POINT(40.373798209672415 73.48676118650776) bank184 +185 POINT(40.918865952473865 73.78352209049652) bank185 +186 POINT(40.60131187485333 74.58904356182698) bank186 +187 POINT(41.23392525964452 73.76336329007063) bank187 +188 POINT(41.26879482450348 73.39308158992856) bank188 +189 POINT(41.581296714477595 73.9917980994601) bank189 +190 POINT(40.918441078608375 73.987204867688) bank190 +191 POINT(40.69536460602192 74.43715390773161) bank191 +192 POINT(40.079649270172 73.16407180174625) bank192 +193 POINT(41.1200283878795 73.747057031817) bank193 +194 POINT(41.60284993074811 74.64992399569083) bank194 +195 POINT(41.120599556334646 73.37538888528961) bank195 +196 POINT(40.515134331951465 74.3762870419448) bank196 +197 POINT(39.82750289599114 74.56065229336323) bank197 +198 POINT(40.18718778710328 74.94241998701531) bank198 +199 POINT(41.29968535150116 74.0058233968939) bank199 +200 POINT(40.91446322495467 74.49407835230186) bank200 +201 POINT(39.75170276749889 73.12607019491043) bank201 +202 POINT(40.31122987582112 73.20469861803825) bank202 +203 POINT(40.01314928167871 73.7784125557554) bank203 +204 POINT(41.22551245929302 73.34944659416767) bank204 +205 POINT(41.29901318431203 73.38048121380447) bank205 +206 POINT(41.63389586287678 73.07008033080118) bank206 +207 POINT(40.23910481933518 74.15633560660561) bank207 +208 POINT(41.346516052732795 73.1746464664371) bank208 +209 POINT(41.36395070740641 74.65617672359839) bank209 +210 POINT(40.61342011973859 73.73126301323266) bank210 +211 POINT(39.81442728091876 73.02178200470046) bank211 +212 POINT(41.664385420687466 73.36809004874522) bank212 +213 POINT(40.015890664111595 74.99945117829735) bank213 +214 POINT(41.433880948838805 74.08780540584016) bank214 +215 POINT(41.02735314588396 74.57976076210225) bank215 +216 POINT(41.47388380503496 74.43540045426774) bank216 +217 POINT(40.68264746715881 73.72140615312637) bank217 +218 POINT(41.04990700939505 74.14151413339219) bank218 +219 POINT(41.663093876810684 74.74232346882914) bank219 +220 POINT(40.43291983106233 74.4407052739587) bank220 +221 POINT(41.50607069417338 73.15088978446384) bank221 +222 POINT(41.18334453321127 73.08586712244544) bank222 +223 POINT(41.64066568443788 74.06711720742612) bank223 +224 POINT(41.56692149506183 73.71668474560767) bank224 +225 POINT(40.22561432807917 74.8781358383198) bank225 +226 POINT(41.369910520347474 73.22793796622778) bank226 +227 POINT(39.90077698243562 74.87817533246854) bank227 +228 POINT(39.822029726156785 74.84337832170617) bank228 +229 POINT(39.92718826815736 73.08759044692229) bank229 +230 POINT(41.70161205988886 74.50582600253311) bank230 +231 POINT(40.55803925482779 73.34502460926008) bank231 +232 POINT(41.26021817668108 73.69944142506034) bank232 +233 POINT(40.565438594540346 73.51435905781692) bank233 +234 POINT(40.46036032873704 73.22102397360325) bank234 +235 POINT(40.042151750490156 73.17200958719465) bank235 +236 POINT(39.859112374079224 74.83889842399351) bank236 +237 POINT(41.707660812508465 74.11851470970555) bank237 +238 POINT(41.045578861458964 73.98995794309945) bank238 +239 POINT(40.76437672563554 74.93719227063875) bank239 +240 POINT(39.89746439707771 74.76120131428885) bank240 +241 POINT(41.14312005008216 73.98914440645245) bank241 +242 POINT(41.09169901665848 74.21097905709419) bank242 +243 POINT(40.6503615796225 73.88251546597607) bank243 +244 POINT(40.22301575749994 74.44003759324161) bank244 +245 POINT(41.6109394526927 74.29583307380382) bank245 +246 POINT(40.073290505238 74.71987139229952) bank246 +247 POINT(41.634382845552835 74.75111725768576) bank247 +248 POINT(40.10585417628162 74.02279067593358) bank248 +249 POINT(39.76116846661542 73.9444322970475) bank249 +250 POINT(39.733442995260695 74.15214054576857) bank250 +251 POINT(40.98072251228327 74.91202168414374) bank251 +252 POINT(41.560728570459126 74.72759162694553) bank252 +253 POINT(40.271177306934206 74.43803938763479) bank253 +254 POINT(40.04947587183717 73.09282008984934) bank254 +255 POINT(39.9323685715696 74.81017872812248) bank255 +256 POINT(40.29775333281715 74.89417451981173) bank256 +257 POINT(40.30733496354397 73.54453308175063) bank257 +258 POINT(40.6449870553512 73.80951428051242) bank258 +259 POINT(41.56933463912405 74.8973871347808) bank259 +260 POINT(40.29088001067592 73.40389368799724) bank260 +261 POINT(40.556498054943766 74.32207211276423) bank261 +262 POINT(40.559013763603915 73.65004721900398) bank262 +263 POINT(40.372668823420256 73.59430700473067) bank263 +264 POINT(40.16968104677062 73.85789416979487) bank264 +265 POINT(40.66811693965853 73.18178645174254) bank265 +266 POINT(41.656695389496875 74.45496164497378) bank266 +267 POINT(39.97067301309256 74.20268521446673) bank267 +268 POINT(40.86542169164868 74.46600686586758) bank268 +269 POINT(41.41662151023077 74.80950294356131) bank269 +270 POINT(40.9517309020595 73.10099440123356) bank270 +271 POINT(39.884088351416246 74.8592107982744) bank271 +272 POINT(40.724477359109464 73.9924675881573) bank272 +273 POINT(40.607276888957884 73.78783087595116) bank273 +274 POINT(40.01350221598104 74.25590647855543) bank274 +275 POINT(40.77322892500053 73.122352122922) bank275 +276 POINT(41.12871024219873 74.37286439578976) bank276 +277 POINT(40.129701292085244 74.98647537534676) bank277 +278 POINT(40.71802747265145 73.93426917562286) bank278 +279 POINT(41.3970653850667 73.7095652412077) bank279 +280 POINT(39.75596846711683 73.10855843274366) bank280 +281 POINT(40.04011450507375 73.7484678358026) bank281 +282 POINT(39.891407090720385 73.06521099413797) bank282 +283 POINT(40.814713759816506 73.49703167897216) bank283 +284 POINT(40.70131761091722 74.58809766200562) bank284 +285 POINT(40.30822852129172 73.31820573945451) bank285 +286 POINT(40.107979234185684 73.55166235799906) bank286 +287 POINT(41.47963091678337 74.27266975577639) bank287 +288 POINT(40.14314011332264 73.57132696457086) bank288 +289 POINT(39.918025952497345 74.05247863547868) bank289 +290 POINT(41.370448272510984 73.0763154499945) bank290 +291 POINT(41.47681415137008 73.54874814283997) bank291 +292 POINT(41.69814440807961 73.2158121330238) bank292 +293 POINT(40.719304519830196 73.46169789522598) bank293 +294 POINT(41.688028268719606 73.83520684472518) bank294 +295 POINT(40.540745013878386 74.0778924910912) bank295 +296 POINT(40.50804760029607 73.5079443338868) bank296 +297 POINT(40.964407333712636 74.17914682583269) bank297 +298 POINT(40.28750468133093 74.06350687357885) bank298 +299 POINT(39.89090131161704 73.28437964664757) bank299 +300 POINT(40.12169962118721 74.37176886660187) bank300 +301 POINT(39.85363040005769 73.0317060651415) bank301 +302 POINT(40.239474839165325 73.01928689672425) bank302 +303 POINT(40.245358905364604 73.24741693241202) bank303 +304 POINT(40.58128222453431 74.20889269022507) bank304 +305 POINT(40.474443979582034 74.43002397816248) bank305 +306 POINT(40.405692045280254 74.80077482892003) bank306 +307 POINT(41.365750987429436 74.06504237127314) bank307 +308 POINT(40.36514481677877 73.76396463098895) bank308 +309 POINT(40.09771031918968 74.12176445550236) bank309 +310 POINT(41.301752603271105 73.38847516868036) bank310 +311 POINT(40.00039535247572 74.8259056281748) bank311 +312 POINT(41.282118025014185 74.2986483860822) bank312 +313 POINT(41.059243352186996 73.05096039555454) bank313 +314 POINT(40.84503628969791 74.25488891046317) bank314 +315 POINT(40.89544092790333 73.17162074790934) bank315 +316 POINT(40.30419313417699 74.512532298024) bank316 +317 POINT(40.491142259568136 74.57068483965753) bank317 +318 POINT(40.55045291236406 73.47361030494514) bank318 +319 POINT(40.316529893226914 73.21002859050324) bank319 +320 POINT(41.34922535849626 73.18801295106222) bank320 +321 POINT(41.393110856897785 74.07391592629637) bank321 +322 POINT(40.59003049394579 73.87167653450838) bank322 +323 POINT(41.43013101013158 73.73527631313979) bank323 +324 POINT(41.610177030613926 74.35182394810927) bank324 +325 POINT(41.456700607464796 73.11335534497789) bank325 +326 POINT(40.11916713664674 73.71028444477169) bank326 +327 POINT(41.636198288884316 74.2028169830461) bank327 +328 POINT(41.31363811792267 74.3580045196793) bank328 +329 POINT(40.060482705444294 74.0630245150127) bank329 +330 POINT(41.337343314361036 73.76152099425609) bank330 +331 POINT(41.015876005267145 74.05547749981658) bank331 +332 POINT(41.169738477003904 74.42230878911003) bank332 +333 POINT(41.708199875726116 73.91766131642737) bank333 +334 POINT(39.900564082288064 73.5373118331166) bank334 +335 POINT(40.79809199793646 73.69169873179924) bank335 +336 POINT(40.8518459469896 74.79480809382386) bank336 +337 POINT(40.2207490056407 74.1516413009798) bank337 +338 POINT(40.9372792428843 73.64022092349978) bank338 +339 POINT(40.23083814391979 73.51113513542097) bank339 +340 POINT(39.98302729177475 73.60762471753083) bank340 +341 POINT(41.63795905143725 74.9816788100588) bank341 +342 POINT(40.242848833488615 73.72680827047928) bank342 +343 POINT(39.846320393652455 74.63394131333217) bank343 +344 POINT(40.0584417677022 73.87349421988696) bank344 +345 POINT(40.10189793735253 74.01032048044102) bank345 +346 POINT(41.40699670037928 73.01432197893607) bank346 +347 POINT(40.07925550266074 73.8163317156244) bank347 +348 POINT(41.303224264477514 73.91064181645434) bank348 +349 POINT(41.41944767068615 73.03461247729788) bank349 +350 POINT(41.51073704444816 73.57059023479226) bank350 +351 POINT(41.588766613037286 73.6108934674709) bank351 +352 POINT(40.456903049578735 73.36986929254405) bank352 +353 POINT(40.87821363385991 73.6833570062946) bank353 +354 POINT(40.6344456145144 74.45326703173227) bank354 +355 POINT(40.83994522616666 73.04026643455244) bank355 +356 POINT(40.77646365390417 74.74736496604662) bank356 +357 POINT(40.43076912497401 73.81387400820577) bank357 +358 POINT(40.50568096862457 73.99347943947718) bank358 +359 POINT(41.46999350341836 73.43590993920151) bank359 +360 POINT(40.11740506310093 74.92042164688353) bank360 +361 POINT(40.19580115574588 74.82446385877294) bank361 +362 POINT(39.85424495514289 73.18383548235965) bank362 +363 POINT(41.18041365943532 73.51916569070657) bank363 +364 POINT(40.73502086035233 73.70187287150793) bank364 +365 POINT(41.479644805925815 73.61453582831014) bank365 +366 POINT(41.16889882969693 73.31897885544932) bank366 +367 POINT(40.588126991756575 73.54661711316618) bank367 +368 POINT(41.16149809485998 73.15974202761477) bank368 +369 POINT(40.2907766866598 73.05555238525655) bank369 +370 POINT(41.674442912121165 73.23683107034448) bank370 +371 POINT(39.93310594250056 73.49690795650936) bank371 +372 POINT(39.7934593837385 73.36457803054668) bank372 +373 POINT(41.328194430837705 73.07906355090908) bank373 +374 POINT(41.25349166201138 74.05726346292958) bank374 +375 POINT(40.846463242327374 74.312976225462) bank375 +376 POINT(41.05478825570458 73.71495563826753) bank376 +377 POINT(40.58987492282758 74.22023735454205) bank377 +378 POINT(41.47986779600603 74.2512588786768) bank378 +379 POINT(40.203854213749906 73.51125451489945) bank379 +380 POINT(41.50653132143959 73.56401407706001) bank380 +381 POINT(40.68542279801473 74.02482764342722) bank381 +382 POINT(40.56545564577419 73.82542166629204) bank382 +383 POINT(41.26746732920973 74.42331094311527) bank383 +384 POINT(40.27795303231526 74.27810470032318) bank384 +385 POINT(41.10575639659159 73.28046931481296) bank385 +386 POINT(41.16629882386423 74.91283785575426) bank386 +387 POINT(40.450170786873855 73.63558294328165) bank387 +388 POINT(41.27808512790256 73.54139430623691) bank388 +389 POINT(40.36000607771635 73.89262039514222) bank389 +390 POINT(39.73891631856692 73.09957422619622) bank390 +391 POINT(40.20229400878335 74.51548629304783) bank391 +392 POINT(39.79001474258846 73.51920036325787) bank392 +393 POINT(40.50911941526331 74.75781794873426) bank393 +394 POINT(40.429431927588546 74.61142001566496) bank394 +395 POINT(40.393291748055205 74.85087336029916) bank395 +396 POINT(41.640633853131256 74.27061970390643) bank396 +397 POINT(40.11211442366426 73.19340118376039) bank397 +398 POINT(40.94679126748071 74.29038147296522) bank398 +399 POINT(41.49940263448972 73.36635393325123) bank399 +400 POINT(40.648919927130876 73.14670236558261) bank400 +401 POINT(40.61422429912864 74.81757426706007) bank401 +402 POINT(41.05948451499763 73.82902801074718) bank402 +403 POINT(41.13019400643428 74.98553882258057) bank403 +404 POINT(40.69986675335419 74.92488849212337) bank404 +405 POINT(40.7077872436719 74.22050148384227) bank405 +406 POINT(39.90464837697152 74.73861475480359) bank406 +407 POINT(41.39504858346349 73.34142081753726) bank407 +408 POINT(41.636043849888544 74.56268462400222) bank408 +409 POINT(40.53753645059989 74.32923219379508) bank409 +410 POINT(41.391872892036034 73.07257338674479) bank410 +411 POINT(41.270148207165704 73.23578193161015) bank411 +412 POINT(40.557814869272 73.58619067872321) bank412 +413 POINT(40.629565497957174 74.94274773987566) bank413 +414 POINT(41.449793372838286 73.78965521546867) bank414 +415 POINT(40.97529513223536 74.32740050895532) bank415 +416 POINT(40.24381912683691 74.27966907618605) bank416 +417 POINT(41.475747248259715 74.4125378416229) bank417 +418 POINT(39.85350704622573 73.82578124248944) bank418 +419 POINT(40.64827008089575 74.67843158638516) bank419 +420 POINT(40.40117911176268 73.643303903634) bank420 +421 POINT(39.761306653097 73.34638303123819) bank421 +422 POINT(41.42495828339178 74.46511494339745) bank422 +423 POINT(39.72267640190367 74.10753535212673) bank423 +424 POINT(41.38435367101107 74.87345215979266) bank424 +425 POINT(41.039657919148176 73.49794171807598) bank425 +426 POINT(40.40099244823375 73.61940268290287) bank426 +427 POINT(41.085257989644894 74.4704373528325) bank427 +428 POINT(40.951428165497695 73.84681602050755) bank428 +429 POINT(41.554638498819884 73.83059551807948) bank429 +430 POINT(40.99121130009512 73.97944987728299) bank430 +431 POINT(40.63550238873225 73.86320200727572) bank431 +432 POINT(41.41663951727042 74.34060132743011) bank432 +433 POINT(41.400028300014554 74.96004467059807) bank433 +434 POINT(39.769571013959094 74.44476288074719) bank434 +435 POINT(40.44762249870861 73.15992714709276) bank435 +436 POINT(41.45542916406496 74.56694950562354) bank436 +437 POINT(40.84002498050415 73.8036271986836) bank437 +438 POINT(40.1248628171799 74.67586868678542) bank438 +439 POINT(41.36957744765218 73.17805599080488) bank439 +440 POINT(41.16428198141124 74.10756608767582) bank440 +441 POINT(41.34846224296861 74.66956456643767) bank441 +442 POINT(41.51669099724674 73.34640004643173) bank442 +443 POINT(41.095603142894895 74.60338065018183) bank443 +444 POINT(40.961756678481 74.46872793523579) bank444 +445 POINT(40.46266636013962 74.9953200130801) bank445 +446 POINT(39.77832713617833 74.61619551707247) bank446 +447 POINT(39.82217993491176 74.25864698888387) bank447 +448 POINT(40.26407106233637 73.15788475508744) bank448 +449 POINT(41.30993928904722 74.4563806342514) bank449 +450 POINT(41.05090963149963 74.1385588627732) bank450 +451 POINT(39.7389109416287 74.21623135460601) bank451 +452 POINT(41.13945900198836 73.44249132972962) bank452 +453 POINT(40.70341636659862 74.92383920356498) bank453 +454 POINT(40.76720434028414 73.19432601930926) bank454 +455 POINT(41.16449030834832 73.27426761609588) bank455 +456 POINT(40.44747999891086 74.76229286119052) bank456 +457 POINT(41.640716307878726 74.07482818036006) bank457 +458 POINT(40.95987025815197 73.72632124469924) bank458 +459 POINT(40.36166345959649 73.09787579896489) bank459 +460 POINT(40.996711278900406 74.94790500727467) bank460 +461 POINT(41.485425848461425 74.26740058212864) bank461 +462 POINT(40.372756999656296 73.56516405119746) bank462 +463 POINT(41.31454563578042 74.02270591140325) bank463 +464 POINT(40.055795311475464 74.68464156469777) bank464 +465 POINT(41.09402092697821 73.65463731248545) bank465 +466 POINT(41.025477708385345 74.84994297037478) bank466 +467 POINT(41.25846628165319 73.08436630316338) bank467 +468 POINT(41.644263042831824 74.39242027945537) bank468 +469 POINT(40.304754283191144 73.29793950176133) bank469 +470 POINT(39.79227148254578 74.66482597446817) bank470 +471 POINT(40.080016571243554 73.86043847885956) bank471 +472 POINT(40.5081211525263 74.70807416483757) bank472 +473 POINT(41.11257013479774 73.83221072265782) bank473 +474 POINT(40.45692832218772 74.19993177887649) bank474 +475 POINT(40.788767169443375 73.50416774278008) bank475 +476 POINT(41.34759470971106 73.44868328294534) bank476 +477 POINT(40.5524898320676 74.99614126183127) bank477 +478 POINT(41.18844220770829 74.11779600071002) bank478 +479 POINT(41.05827290301157 74.7092176313412) bank479 +480 POINT(40.196101271511154 73.52450892107991) bank480 +481 POINT(40.287230389941485 74.95627932016046) bank481 +482 POINT(40.965293543252756 74.24183683665876) bank482 +483 POINT(41.500082802599366 74.81507430057856) bank483 +484 POINT(39.83792739998159 74.42501273686885) bank484 +485 POINT(40.693268090544976 74.54739939010744) bank485 +486 POINT(40.891645497583646 74.78307356088229) bank486 +487 POINT(39.97048665491289 74.68715683259765) bank487 +488 POINT(41.19043885402947 74.98152828892776) bank488 +489 POINT(41.14008620315059 74.60362997956406) bank489 +490 POINT(41.65903434271434 73.96980998982401) bank490 +491 POINT(40.658515008110946 73.92658919464833) bank491 +492 POINT(40.26182848331196 74.11186578116798) bank492 +493 POINT(40.7935714241861 74.37210000652688) bank493 +494 POINT(39.76041725061135 73.83751675711588) bank494 +495 POINT(40.131666124823944 74.97687111869205) bank495 +496 POINT(40.40282284669838 73.72093925132735) bank496 +497 POINT(40.169105197319254 73.08816977026373) bank497 +498 POINT(41.5448402218413 74.29148842407614) bank498 +499 POINT(41.444420770821964 73.13650019315241) bank499 +500 POINT(40.71894897393015 74.5797571750394) bank500 +501 POINT(41.462547299688936 73.57683000295857) bank501 +502 POINT(40.89209627709276 74.03754441786143) bank502 +503 POINT(41.11729208741678 74.60138276925854) bank503 +504 POINT(40.399340730934576 73.90562478640422) bank504 +505 POINT(41.117972706628315 74.37772975342455) bank505 +506 POINT(41.55954731396491 74.95536380501943) bank506 +507 POINT(41.5299553874891 73.97578555052137) bank507 +508 POINT(41.12752929040378 74.4225302955408) bank508 +509 POINT(40.732130703816566 73.51958311804248) bank509 +510 POINT(40.84279418416792 74.76762855189246) bank510 +511 POINT(40.15685533093491 74.55608356510676) bank511 +512 POINT(40.95304509114754 73.79926553383197) bank512 +513 POINT(40.93955872715389 74.2403805811284) bank513 +514 POINT(41.16840575128378 74.1987892650866) bank514 +515 POINT(41.28225513089296 73.7447380240594) bank515 +516 POINT(39.93588311336383 73.97650772319848) bank516 +517 POINT(40.27904270994347 74.12462193953876) bank517 +518 POINT(41.07046197930173 74.17110001822789) bank518 +519 POINT(39.79467845796929 73.22909141912268) bank519 +520 POINT(41.3816099448387 73.08656604004705) bank520 +521 POINT(40.77321031484793 73.8702610058893) bank521 +522 POINT(41.57563845208946 73.41536780714829) bank522 +523 POINT(41.12130463351871 74.13639313430458) bank523 +524 POINT(40.098009500362274 73.43420398838192) bank524 +525 POINT(39.861304505615145 73.44115588640915) bank525 +526 POINT(40.33962500455178 74.5646057419027) bank526 +527 POINT(39.950037239653085 73.5748474465944) bank527 +528 POINT(41.66714574491622 73.74896451034824) bank528 +529 POINT(40.43380443997735 74.1516244532986) bank529 +530 POINT(41.073752721592065 74.97803258572958) bank530 +531 POINT(40.675405730905055 74.60908013714646) bank531 +532 POINT(40.89086890423561 73.56932661080127) bank532 +533 POINT(41.26914562488658 74.68918466529625) bank533 +534 POINT(41.457279983563275 74.70289371848708) bank534 +535 POINT(41.659802509419826 74.38432526848916) bank535 +536 POINT(40.13618705795135 74.91454305754017) bank536 +537 POINT(41.43375031848683 73.79164472510215) bank537 +538 POINT(41.66443526489095 74.59360300876163) bank538 +539 POINT(41.25474237788897 74.02066579248257) bank539 +540 POINT(40.516637636354474 73.71325523347929) bank540 +541 POINT(41.349572517123896 73.45279897723508) bank541 +542 POINT(41.522191391813266 73.25160819413199) bank542 +543 POINT(41.08993359158122 73.813006182124) bank543 +544 POINT(39.934097787822196 74.66420697465496) bank544 +545 POINT(40.02063078944624 73.12888702803014) bank545 +546 POINT(41.4885285831061 74.44923136213751) bank546 +547 POINT(41.53878364488659 73.60729863132553) bank547 +548 POINT(41.44007770275932 73.65153849695136) bank548 +549 POINT(40.63553786370903 74.2679827195942) bank549 +550 POINT(40.18565656242042 73.8330967090564) bank550 +551 POINT(41.60845509840364 73.8264636346808) bank551 +552 POINT(41.24512806381724 73.68597995287419) bank552 +553 POINT(39.763180378362804 74.04916065070783) bank553 +554 POINT(41.69170408015075 73.52262741287551) bank554 +555 POINT(41.35815564954148 73.15252069987524) bank555 +556 POINT(40.57828930643784 74.80398576939791) bank556 +557 POINT(41.566434795297376 73.77149662708294) bank557 +558 POINT(40.94296350091006 73.484013118596) bank558 +559 POINT(41.12773497309226 73.99520491147416) bank559 +560 POINT(41.43339421540635 73.29373549872707) bank560 +561 POINT(39.9282733831939 73.06529623590588) bank561 +562 POINT(41.17894600901806 74.61213503289619) bank562 +563 POINT(40.59189240404976 74.262897310435) bank563 +564 POINT(40.13923797741826 74.11928254186547) bank564 +565 POINT(41.64348906709982 73.14086237668899) bank565 +566 POINT(41.066701376581584 74.60120849678958) bank566 +567 POINT(41.570016829576154 73.57158750898967) bank567 +568 POINT(40.238859577917474 74.47144657032139) bank568 +569 POINT(39.76241219264008 74.80590307412659) bank569 +570 POINT(40.555006436812334 74.90830840573715) bank570 +571 POINT(39.75877780685559 73.42333996104074) bank571 +572 POINT(41.216272792986246 73.26945544041389) bank572 +573 POINT(39.950000927105805 73.42907940365481) bank573 +574 POINT(40.39845882043687 74.63214647741736) bank574 +575 POINT(39.88157076265186 74.27983668564734) bank575 +576 POINT(41.24034781921684 73.65939057640634) bank576 +577 POINT(40.64768363627277 73.71463925184828) bank577 +578 POINT(40.07100744570455 74.6729385740763) bank578 +579 POINT(41.296587251530326 73.85605828368024) bank579 +580 POINT(41.137869635907535 73.9673998388237) bank580 +581 POINT(40.67645029499588 73.69670571061701) bank581 +582 POINT(41.197321234894034 73.49748130753017) bank582 +583 POINT(40.41958757962312 74.23685797572232) bank583 +584 POINT(39.82073798492282 74.57309614360733) bank584 +585 POINT(40.04786585064892 74.16472152756695) bank585 +586 POINT(40.192943768724916 74.34957384651605) bank586 +587 POINT(40.966479622250006 74.6589422873842) bank587 +588 POINT(40.30874418916073 73.23066967270752) bank588 +589 POINT(39.77336861176559 74.91934955761604) bank589 +590 POINT(40.603575559720454 74.87693536254396) bank590 +591 POINT(41.33540231957702 74.56340178911806) bank591 +592 POINT(39.81273826914109 73.58414258423697) bank592 +593 POINT(41.28232686249709 74.032627046276) bank593 +594 POINT(40.29820658285849 73.77626532253741) bank594 +595 POINT(40.59566430960468 73.13433860631075) bank595 +596 POINT(41.476135803652795 73.96778730692286) bank596 +597 POINT(40.336184358368534 74.25052500606517) bank597 +598 POINT(41.311262479666 74.52125064031839) bank598 +599 POINT(40.476301005013546 74.39919486247288) bank599 +600 POINT(40.8349870638619 73.67974543858955) bank600 +601 POINT(40.64045238258807 73.39134129663044) bank601 +602 POINT(41.07093436034458 73.06753538128974) bank602 +603 POINT(39.93912478446974 73.4166561446198) bank603 +604 POINT(41.05819853181485 74.51837705249125) bank604 +605 POINT(40.63862748273161 73.58836972296606) bank605 +606 POINT(41.381791384026805 73.837422798098) bank606 +607 POINT(40.023736555802024 73.48041090791182) bank607 +608 POINT(39.81554691413068 73.99403217086478) bank608 +609 POINT(41.66430400848355 73.5868124625546) bank609 +610 POINT(41.503651154411365 74.29867968183125) bank610 +611 POINT(41.61323856683183 74.50496673842032) bank611 +612 POINT(40.20735961986196 73.59793813233064) bank612 +613 POINT(40.7982449944016 73.31384153984177) bank613 +614 POINT(40.13687566860581 74.33088520563571) bank614 +615 POINT(41.43110697643417 74.3052298024311) bank615 +616 POINT(41.31433178688623 74.94682084564418) bank616 +617 POINT(41.019105770195665 74.5393276995005) bank617 +618 POINT(40.63948799177611 74.34918781554924) bank618 +619 POINT(40.766601649811456 74.09999908207884) bank619 +620 POINT(40.62986874848902 74.01085653812927) bank620 +621 POINT(40.849124170895095 74.79114054176844) bank621 +622 POINT(41.561126943474946 74.71648262028832) bank622 +623 POINT(41.48379085616238 74.79533512582691) bank623 +624 POINT(40.83351902977408 73.02994285181519) bank624 +625 POINT(41.70963826370925 74.77519753415517) bank625 +626 POINT(41.53752106510085 74.86276780547831) bank626 +627 POINT(40.29206358490799 73.93975572351235) bank627 +628 POINT(40.21368476622916 73.55309531909705) bank628 +629 POINT(40.63847735327813 73.58506902700714) bank629 +630 POINT(40.39981721761555 73.32203122321845) bank630 +631 POINT(40.63167912140842 73.76649588933476) bank631 +632 POINT(41.30607543565042 73.13414879001434) bank632 +633 POINT(40.40014855101972 74.90834648446379) bank633 +634 POINT(41.66883464205162 74.29372516079452) bank634 +635 POINT(40.94459015918365 74.4700160332661) bank635 +636 POINT(39.83198107113111 73.87780698168427) bank636 +637 POINT(39.73185817562268 74.41983411635378) bank637 +638 POINT(40.020838172560694 74.96833216889948) bank638 +639 POINT(41.50238266869581 74.97033664402956) bank639 +640 POINT(40.45333222145566 74.97988507246814) bank640 +641 POINT(41.141255624045876 74.46248549444941) bank641 +642 POINT(40.1199143384945 73.18974242017792) bank642 +643 POINT(41.325067821322754 74.59524195066402) bank643 +644 POINT(40.11545981338941 74.54284271551697) bank644 +645 POINT(41.445638068377924 74.66265698289753) bank645 +646 POINT(40.97304440737927 74.37752054133631) bank646 +647 POINT(39.84238609268113 73.30249233226571) bank647 +648 POINT(40.519997612314846 74.31497430050075) bank648 +649 POINT(40.349703178604315 73.28122926942443) bank649 +650 POINT(40.8870596406982 73.2703811170012) bank650 +651 POINT(39.82066690950521 74.57206347259581) bank651 +652 POINT(41.59483741550534 73.11620872555004) bank652 +653 POINT(41.1612030990379 74.32490504206206) bank653 +654 POINT(40.34629299825034 74.9359659611051) bank654 +655 POINT(41.47983182435297 74.0132349354472) bank655 +656 POINT(39.83531576586469 74.42376440578218) bank656 +657 POINT(41.13416932947516 73.93869808780416) bank657 +658 POINT(40.71906632437848 73.34227052450191) bank658 +659 POINT(40.164115777401776 73.32429018107062) bank659 +660 POINT(40.026441514391806 73.06291300904842) bank660 +661 POINT(41.27888952706049 74.20753065617714) bank661 +662 POINT(40.03888111553744 73.13725269019129) bank662 +663 POINT(41.57285328173032 74.32559564829367) bank663 +664 POINT(41.42199008650432 73.70479688814643) bank664 +665 POINT(40.15618320970652 74.08998752265181) bank665 +666 POINT(41.03221613973843 73.20936937130058) bank666 +667 POINT(39.82670816126233 74.4131015786873) bank667 +668 POINT(40.84021488438433 73.31874067184384) bank668 +669 POINT(40.25040424374859 73.16919367003948) bank669 +670 POINT(40.21919818600198 73.57016120497813) bank670 +671 POINT(40.895437223647974 74.88357057786783) bank671 +672 POINT(40.899484983278974 74.13507341569107) bank672 +673 POINT(40.983275724201576 73.97750377426817) bank673 +674 POINT(40.42412699929759 74.63960549366621) bank674 +675 POINT(41.70792168251024 73.11450069656058) bank675 +676 POINT(39.88317792422846 74.34005537465247) bank676 +677 POINT(41.4373997692524 74.52371958355681) bank677 +678 POINT(39.97969458634788 74.37911316949537) bank678 +679 POINT(40.61624452038465 74.74112395043218) bank679 +680 POINT(39.97926630341391 74.2018798964248) bank680 +681 POINT(40.135303699895175 74.02470844669794) bank681 +682 POINT(40.43994379795805 74.97514658568291) bank682 +683 POINT(41.54393075412746 73.10707405688984) bank683 +684 POINT(39.80912325946391 73.76681230853359) bank684 +685 POINT(40.278037272007204 74.70235167408569) bank685 +686 POINT(40.354348413449415 73.9960329041515) bank686 +687 POINT(40.632699735409915 74.52245706883534) bank687 +688 POINT(40.053411219747744 74.18290479591472) bank688 +689 POINT(40.053200079941654 73.23550864118097) bank689 +690 POINT(39.941121786180254 74.55266363467794) bank690 +691 POINT(40.00090659341104 73.1376403778446) bank691 +692 POINT(40.24558722281865 73.5959522216665) bank692 +693 POINT(39.97744180190912 74.8353233772732) bank693 +694 POINT(41.51264233268723 74.24216236261283) bank694 +695 POINT(41.04653160065311 74.95081481040921) bank695 +696 POINT(40.937025432201345 74.53489013626522) bank696 +697 POINT(40.15103572696658 73.6934078193588) bank697 +698 POINT(40.781574584508135 73.69330694523944) bank698 +699 POINT(40.75849153855929 73.66453931897045) bank699 +700 POINT(41.52063901956505 74.6814420821871) bank700 +701 POINT(40.43444279303958 73.28033956107005) bank701 +702 POINT(40.846492676910714 74.24642245976787) bank702 +703 POINT(40.202940408600114 73.18586151523562) bank703 +704 POINT(39.872202538807 74.80240277864857) bank704 +705 POINT(40.890595565831305 74.66795133888395) bank705 +706 POINT(39.85917115739371 73.63735948298185) bank706 +707 POINT(40.77570124018162 74.4339472377973) bank707 +708 POINT(40.65492376272394 74.280999955524) bank708 +709 POINT(41.39547046635302 74.45820117165135) bank709 +710 POINT(40.635252838197296 74.48348421034186) bank710 +711 POINT(41.62802574971771 74.68326423670177) bank711 +712 POINT(40.32438619950467 73.98518805302932) bank712 +713 POINT(39.91669957852196 74.45097770805327) bank713 +714 POINT(41.41066161432873 74.8444149617244) bank714 +715 POINT(41.64350990877061 73.01068844784176) bank715 +716 POINT(40.86607654775943 73.98096108332614) bank716 +717 POINT(40.85434518244463 74.32974054478146) bank717 +718 POINT(39.77138531118251 74.80301613852272) bank718 +719 POINT(41.1107028240055 73.9323618197317) bank719 +720 POINT(40.72800260120533 73.18820272615073) bank720 +721 POINT(41.54014490502954 73.15043189382527) bank721 +722 POINT(41.528952234216504 73.45741162100846) bank722 +723 POINT(40.768584286542975 73.64025415086476) bank723 +724 POINT(40.01437629113523 74.30260502973793) bank724 +725 POINT(39.724194179020664 74.72882894968434) bank725 +726 POINT(41.455946575298015 73.48897842614802) bank726 +727 POINT(40.60114242303664 74.79247400780979) bank727 +728 POINT(40.35229201999464 73.80468432564605) bank728 +729 POINT(39.810884050272335 73.34462748673683) bank729 +730 POINT(41.17521388531927 73.82994958405027) bank730 +731 POINT(41.492189295495685 74.00020942016036) bank731 +732 POINT(40.75058197266184 73.90467547992357) bank732 +733 POINT(40.32879683790456 74.2400967435335) bank733 +734 POINT(40.091912316143826 73.56062942373637) bank734 +735 POINT(40.564662161064135 73.39693065472294) bank735 +736 POINT(39.861176643621576 74.77126063114893) bank736 +737 POINT(40.763393879722635 74.53166097577196) bank737 +738 POINT(39.99565951809632 74.68452372967006) bank738 +739 POINT(40.36824496263861 73.56396842825653) bank739 +740 POINT(40.45353345627835 73.39513848612361) bank740 +741 POINT(39.877400176684255 73.61997919814296) bank741 +742 POINT(41.630281313848336 73.02180186902334) bank742 +743 POINT(40.174836999119044 73.57962867907574) bank743 +744 POINT(39.7230808233886 74.91606991297061) bank744 +745 POINT(39.93217448820871 74.45403440960123) bank745 +746 POINT(40.68665220671236 73.67685223322998) bank746 +747 POINT(40.90222049472332 74.98741813454497) bank747 +748 POINT(39.79034112884518 73.25041765795383) bank748 +749 POINT(41.13375351540835 74.87084186640163) bank749 +750 POINT(41.17370353355294 74.35391773609865) bank750 +751 POINT(40.2650943746776 74.54172566999334) bank751 +752 POINT(40.61598409173611 73.12261507496065) bank752 +753 POINT(39.74011996990486 73.65878420311144) bank753 +754 POINT(41.49419665120656 73.32133747147198) bank754 +755 POINT(39.90664204583569 73.7338950318792) bank755 +756 POINT(40.400046253102744 73.44730733592453) bank756 +757 POINT(41.473636893443896 74.61109309836654) bank757 +758 POINT(41.07440989351884 73.6129187504395) bank758 +759 POINT(40.04504648489044 73.65956639576876) bank759 +760 POINT(40.971508260482175 73.61878119844049) bank760 +761 POINT(40.1360298134203 73.53551392810434) bank761 +762 POINT(41.27880526167936 73.98022617891525) bank762 +763 POINT(40.680865661427774 74.62721625230489) bank763 +764 POINT(40.13808134463903 73.77968668151233) bank764 +765 POINT(41.38451293081922 74.12971006920043) bank765 +766 POINT(40.2275792657192 74.6431510357846) bank766 +767 POINT(41.58223203608605 73.81848860915201) bank767 +768 POINT(40.891354340055585 74.339019550614) bank768 +769 POINT(41.037843828576285 73.92681434714142) bank769 +770 POINT(41.656567144313115 73.65581026042067) bank770 +771 POINT(41.376981954077245 73.60209110255421) bank771 +772 POINT(39.99648836429529 73.04900584894027) bank772 +773 POINT(40.54823466028537 73.88460831397293) bank773 +774 POINT(41.364411234789145 73.11794997033144) bank774 +775 POINT(40.226082484105625 73.70553491972308) bank775 +776 POINT(39.8956843913968 74.812163480204) bank776 +777 POINT(41.21823209203909 74.39874086176445) bank777 +778 POINT(40.97017830164922 74.88477442039515) bank778 +779 POINT(39.84364498665533 73.66538814690101) bank779 +780 POINT(40.66574272266409 73.40767290542738) bank780 +781 POINT(39.76197944135502 73.84736510067721) bank781 +782 POINT(40.80097346924662 73.21949699264857) bank782 +783 POINT(39.96722037734367 74.71496312912832) bank783 +784 POINT(41.66452693925787 73.17286967037025) bank784 +785 POINT(40.29163775851117 73.70482359024481) bank785 +786 POINT(40.00414152297294 73.50114261899115) bank786 +787 POINT(40.43062909860901 74.4129451092706) bank787 +788 POINT(41.532701643805076 74.71883223431365) bank788 +789 POINT(41.700100373766126 73.47807872325822) bank789 +790 POINT(41.378435143053125 74.6875632518831) bank790 +791 POINT(41.565221468805134 74.46310195601048) bank791 +792 POINT(41.5604748458769 73.67303488292579) bank792 +793 POINT(40.86076298019929 74.97037260524799) bank793 +794 POINT(41.667539566253595 74.02072529601595) bank794 +795 POINT(40.89529390340305 73.17303959014102) bank795 +796 POINT(40.8654774877268 73.04149030537809) bank796 +797 POINT(41.43065635325859 73.51527100571835) bank797 +798 POINT(41.34325326306735 73.67372498911634) bank798 +799 POINT(41.12757545005628 74.53279164332942) bank799 +800 POINT(40.802827990905136 73.8219889946999) bank800 +801 POINT(40.05670114018632 74.51264617318085) bank801 +802 POINT(41.34122114235114 73.1232319388918) bank802 +803 POINT(40.40309699825461 74.21574542396615) bank803 +804 POINT(40.82705992776061 74.65347539101299) bank804 +805 POINT(40.78089033712798 73.97415845554195) bank805 +806 POINT(40.5845126719073 74.86322141531748) bank806 +807 POINT(41.27595251187856 74.99379431574505) bank807 +808 POINT(40.24537595922492 73.13175815871642) bank808 +809 POINT(40.34125391455481 74.12653512761193) bank809 +810 POINT(40.49718214434 73.28382421045293) bank810 +811 POINT(40.04028262101038 74.24964839796246) bank811 +812 POINT(40.921232363095925 74.53801509785292) bank812 +813 POINT(41.50502341304585 74.78833696380134) bank813 +814 POINT(39.852003010943356 74.56453087078296) bank814 +815 POINT(40.952113660303255 73.71220940391106) bank815 +816 POINT(39.76883638649114 74.4947047410455) bank816 +817 POINT(39.88657636874315 74.45156912741993) bank817 +818 POINT(41.05929613255255 73.42162884613764) bank818 +819 POINT(40.239572843226306 74.3291230548364) bank819 +820 POINT(41.237309811016814 73.60084231577463) bank820 +821 POINT(41.420668439889575 73.48531481205274) bank821 +822 POINT(40.22372609650141 74.68660479555066) bank822 +823 POINT(40.03148310336856 73.19349200190888) bank823 +824 POINT(40.811671168664276 74.06398473796493) bank824 +825 POINT(39.72418954048096 74.27458804102298) bank825 +826 POINT(40.309161952733895 73.02986057603793) bank826 +827 POINT(39.869119010359185 74.52751194878583) bank827 +828 POINT(41.49666976750336 74.96379281961775) bank828 +829 POINT(40.56469514672321 74.46769771105464) bank829 +830 POINT(40.60905230404345 73.02439411684324) bank830 +831 POINT(41.3732106635696 73.04688725366968) bank831 +832 POINT(40.47870894318356 74.68905301352211) bank832 +833 POINT(40.77976258621605 73.63229945977393) bank833 +834 POINT(41.23434725333727 74.69431806986273) bank834 +835 POINT(41.02124825437944 73.06993206820215) bank835 +836 POINT(41.50429502641945 74.92137341605607) bank836 +837 POINT(40.896373886611165 74.09711403247961) bank837 +838 POINT(40.333378643131965 73.74475289274031) bank838 +839 POINT(39.903624276112495 73.07677293828125) bank839 +840 POINT(40.08184713743192 74.43830306331765) bank840 +841 POINT(40.78781438039542 73.53560711564272) bank841 +842 POINT(41.30518499494277 73.33207962323904) bank842 +843 POINT(40.64683094517637 74.14092014057454) bank843 +844 POINT(40.900904779298955 74.73330492493344) bank844 +845 POINT(40.35612360651319 73.34534621235176) bank845 +846 POINT(39.77972191147384 73.2715837810141) bank846 +847 POINT(40.148245017764545 73.58062833666202) bank847 +848 POINT(39.74644857299085 73.29758382470072) bank848 +849 POINT(40.8282903842239 74.00179821550698) bank849 +850 POINT(40.15136937279132 74.67962552789989) bank850 +851 POINT(41.63488778552302 74.08465420004448) bank851 +852 POINT(40.88345027595799 74.07231033146653) bank852 +853 POINT(41.381535201733634 74.31051839947655) bank853 +854 POINT(41.53793839222944 73.70782002908324) bank854 +855 POINT(40.72634521425838 73.07085740205821) bank855 +856 POINT(40.37135655092827 74.14318793038993) bank856 +857 POINT(40.756712295411425 73.37004981680722) bank857 +858 POINT(40.26979735987512 73.09491665381648) bank858 +859 POINT(39.892587550581666 74.01051223012279) bank859 +860 POINT(41.35625065059755 74.96797422203838) bank860 +861 POINT(41.63450687331568 73.61308600563724) bank861 +862 POINT(40.97785997372645 73.23875079326314) bank862 +863 POINT(41.68284764446911 74.33896175064024) bank863 +864 POINT(41.45041694639426 74.13168813377291) bank864 +865 POINT(39.94560681464895 74.65195272641725) bank865 +866 POINT(41.595243279031976 74.42560920404974) bank866 +867 POINT(40.38404735639989 73.5833535568879) bank867 +868 POINT(40.07326247102565 73.15589901055897) bank868 +869 POINT(41.21043472091655 73.38740355519701) bank869 +870 POINT(41.17209259313493 74.06158574512388) bank870 +871 POINT(40.01322382333876 74.63033489633293) bank871 +872 POINT(41.70974340119764 74.96793837013688) bank872 +873 POINT(40.882909212501545 74.94602943964978) bank873 +874 POINT(41.509479907370654 73.98402366732051) bank874 +875 POINT(40.74954910954158 73.91994890147475) bank875 +876 POINT(40.95288947312652 73.10522419839361) bank876 +877 POINT(40.24675819110787 73.10017452119556) bank877 +878 POINT(41.02374959697551 73.17524635738552) bank878 +879 POINT(41.30920203670125 74.55023570405227) bank879 +880 POINT(40.91338765910062 74.8165455423085) bank880 +881 POINT(41.47182729679077 74.74501708688133) bank881 +882 POINT(40.241217164559764 73.57395013709755) bank882 +883 POINT(39.837744246322906 73.83552478066538) bank883 +884 POINT(40.988806204936424 74.9008154648969) bank884 +885 POINT(40.78990272535578 74.84768142593128) bank885 +886 POINT(40.29268775691667 74.12909936906728) bank886 +887 POINT(40.185089124577786 73.65857659060447) bank887 +888 POINT(41.140651706520664 74.09967540095678) bank888 +889 POINT(40.77828355449012 74.05274902423486) bank889 +890 POINT(40.36412581139183 73.73792475401495) bank890 +891 POINT(40.385499956711556 73.54237773572456) bank891 +892 POINT(40.44218871801432 73.38438578047466) bank892 +893 POINT(40.964012443747556 73.40238396926942) bank893 +894 POINT(40.64048731951748 73.42020795096114) bank894 +895 POINT(40.38976777141345 73.03207821436563) bank895 +896 POINT(40.70170477932141 74.50401580273368) bank896 +897 POINT(41.660788806871246 74.61656484466458) bank897 +898 POINT(40.86733580827588 73.62824512347846) bank898 +899 POINT(40.84797302470752 74.71480789978709) bank899 +900 POINT(40.93367428557451 74.13268916205567) bank900 +901 POINT(40.9627006803531 73.4604016189228) bank901 +902 POINT(40.39108579521024 73.91757440687755) bank902 +903 POINT(41.00335219101349 73.95817196603991) bank903 +904 POINT(40.426129424526316 73.49907416739569) bank904 +905 POINT(41.39822117326825 74.04383219119568) bank905 +906 POINT(39.84808430386965 73.68600538948336) bank906 +907 POINT(40.11558295456105 73.65396260354416) bank907 +908 POINT(41.396170245491334 74.13747583192783) bank908 +909 POINT(40.78957687336199 74.67905490976331) bank909 +910 POINT(40.272408731913615 73.98349336060906) bank910 +911 POINT(41.145178782785536 74.24068079265223) bank911 +912 POINT(40.21651608375523 74.28072034430848) bank912 +913 POINT(39.71432491285647 73.5249940931101) bank913 +914 POINT(41.38879232213316 74.52235878105603) bank914 +915 POINT(40.46879534382234 74.97160845622042) bank915 +916 POINT(40.98912558610052 74.8027214511462) bank916 +917 POINT(40.50106332043161 73.64264330064694) bank917 +918 POINT(40.84750356541929 73.72802054060932) bank918 +919 POINT(40.15999769815995 74.84380997681816) bank919 +920 POINT(39.99382028363753 74.24137483036597) bank920 +921 POINT(39.8732076376949 73.75971136693452) bank921 +922 POINT(40.91274489477387 74.69894714486281) bank922 +923 POINT(40.44108113902058 74.03123708204475) bank923 +924 POINT(40.15970185161485 73.72514269141767) bank924 +925 POINT(40.16350528698767 74.1672663360413) bank925 +926 POINT(41.16657814630976 74.74008020418253) bank926 +927 POINT(39.96765003319807 73.4867087455367) bank927 +928 POINT(41.582713711673065 73.45127203285739) bank928 +929 POINT(40.691639480023206 74.79798863720498) bank929 +930 POINT(40.05551968456581 73.66900638813587) bank930 +931 POINT(40.67605213905331 73.477142312282) bank931 +932 POINT(40.71193332558084 74.1857454416044) bank932 +933 POINT(40.63336008155497 73.63032605669869) bank933 +934 POINT(40.647922513869574 74.37143392765263) bank934 +935 POINT(41.321556414647645 73.3630396012381) bank935 +936 POINT(40.24096839487705 73.9416868582145) bank936 +937 POINT(41.50526753625721 73.72631899020804) bank937 +938 POINT(41.1887740398019 74.13740236399963) bank938 +939 POINT(40.42893748465535 73.69088791105303) bank939 +940 POINT(39.847894243435 74.67178753146996) bank940 +941 POINT(40.727612426536155 73.64923335397809) bank941 +942 POINT(39.796171104749504 73.64581014275733) bank942 +943 POINT(40.44820146970677 74.56253693113523) bank943 +944 POINT(41.061549206820295 74.00634550877534) bank944 +945 POINT(41.28578893940152 74.06131095997333) bank945 +946 POINT(39.85622162904727 74.70291099991277) bank946 +947 POINT(40.512351301898235 73.0198534710337) bank947 +948 POINT(41.6904705460297 74.73492703974662) bank948 +949 POINT(39.71405348851084 73.812623810835) bank949 +950 POINT(40.19141208076101 74.18495807328416) bank950 +951 POINT(40.88384622398571 74.71213888821532) bank951 +952 POINT(41.676395238463634 73.31397896054474) bank952 +953 POINT(41.476851103901296 74.28387163854586) bank953 +954 POINT(40.22054907290426 73.12479295590619) bank954 +955 POINT(40.866206889811465 73.36842519370623) bank955 +956 POINT(41.12034181620955 73.6783074196939) bank956 +957 POINT(41.3296693974279 74.15335333641431) bank957 +958 POINT(40.79681989983057 74.68915244103087) bank958 +959 POINT(39.79345515855761 74.20754133192662) bank959 +960 POINT(41.20772958712935 73.56330945212773) bank960 +961 POINT(41.45847318762229 73.8202729804067) bank961 +962 POINT(40.83814162767988 74.72217211502829) bank962 +963 POINT(40.21591146187766 74.43407266203009) bank963 +964 POINT(40.53943575981487 74.32544815170627) bank964 +965 POINT(40.50643453346587 74.02225234544919) bank965 +966 POINT(41.040956971846605 74.37467139825479) bank966 +967 POINT(41.55386266930374 74.74119249136785) bank967 +968 POINT(40.616896328961445 74.75007395260828) bank968 +969 POINT(41.01554840804387 73.62793319720149) bank969 +970 POINT(40.70637947638559 73.6013895675506) bank970 +971 POINT(40.95821235241877 74.88755425329465) bank971 +972 POINT(39.932021739825906 73.61694303003021) bank972 +973 POINT(41.31730441687223 73.77565252195868) bank973 +974 POINT(40.7723024394802 74.65576153451822) bank974 +975 POINT(41.40916502787735 74.13613529204359) bank975 +976 POINT(40.13234850957327 73.77683033424967) bank976 +977 POINT(40.76367137313976 74.92598482457963) bank977 +978 POINT(40.19080275127016 74.83240093312716) bank978 +979 POINT(39.746798512730486 74.46602595819378) bank979 +980 POINT(40.939110588820526 74.54543112309902) bank980 +981 POINT(40.36207866673478 74.85508768937974) bank981 +982 POINT(41.30946763957233 74.9648265808925) bank982 +983 POINT(41.506041475176595 74.64248479829675) bank983 +984 POINT(40.02534638988376 74.39775683488234) bank984 +985 POINT(39.87701863113275 73.01509776107932) bank985 +986 POINT(40.48788000215725 74.27742907647146) bank986 +987 POINT(40.58899321856677 73.51351767375024) bank987 +988 POINT(40.453527936891646 73.11761754490253) bank988 +989 POINT(40.267090107091356 73.36947316290365) bank989 +990 POINT(40.76608671168685 74.22952703826446) bank990 +991 POINT(41.01898974803299 73.47130609889633) bank991 +992 POINT(41.48959445889416 73.5796215327691) bank992 +993 POINT(41.23258384135195 73.77443829527674) bank993 +994 POINT(41.57594516784006 73.25962303861382) bank994 +995 POINT(41.39939090754917 73.49212972253044) bank995 +996 POINT(40.017245024849025 74.28869112353432) bank996 +997 POINT(40.03317377745662 73.20176476146302) bank997 +998 POINT(39.974342310224074 73.19028542140487) bank998 +999 POINT(39.85484447649993 74.79864696472796) bank999 +1000 POINT(41.17005360703377 74.01530044918603) bank1000 +1001 POINT(41.44695040033882 74.12412282169187) bank1001 +1002 POINT(41.4051697070627 73.68186636858393) bank1002 +1003 POINT(40.52938444290806 74.21433703837624) bank1003 +1004 POINT(40.96647388733793 74.58485877076835) bank1004 +1005 POINT(41.41594008034624 74.32475515035439) bank1005 +1006 POINT(41.13571005070873 74.28108778460837) bank1006 +1007 POINT(41.10178333228053 74.53091116957381) bank1007 +1008 POINT(41.129863511466 74.52713638605844) bank1008 +1009 POINT(40.81339428453267 73.25278314907676) bank1009 +1010 POINT(40.09305451423634 73.10134038961361) bank1010 +1011 POINT(41.16039095133389 74.44138198891216) bank1011 +1012 POINT(39.796961228895434 74.7120453381739) bank1012 +1013 POINT(41.65363009861796 73.22308252053328) bank1013 +1014 POINT(40.094312194991865 73.44322427303095) bank1014 +1015 POINT(41.37176848741112 73.5380217129431) bank1015 +1016 POINT(41.425702627936744 73.6266032461509) bank1016 +1017 POINT(40.95305304598291 73.96698230757602) bank1017 +1018 POINT(41.416771748629714 73.43922148302914) bank1018 +1019 POINT(39.75708804645359 74.47248406811863) bank1019 +1020 POINT(39.90257034860052 73.88404746623584) bank1020 +1021 POINT(40.79287694578211 74.09969860970746) bank1021 +1022 POINT(40.12690403257182 73.26471792418363) bank1022 +1023 POINT(41.66073911968262 74.90583285520673) bank1023 +1024 POINT(41.326352701911695 73.44488147071807) bank1024 +1025 POINT(39.88861814696453 74.14165912067037) bank1025 +1026 POINT(41.21835742903017 74.73524697633846) bank1026 +1027 POINT(40.16166151819651 74.62602735681757) bank1027 +1028 POINT(41.24433889930193 73.49873590889705) bank1028 +1029 POINT(40.03997613539075 73.5892027477974) bank1029 +1030 POINT(40.61433283541095 74.08918106675262) bank1030 +1031 POINT(41.51974383243356 73.69782018108033) bank1031 +1032 POINT(40.17765721244281 74.50252720742454) bank1032 +1033 POINT(41.29955211292837 74.16046196444445) bank1033 +1034 POINT(41.13582535200683 74.59168400697887) bank1034 +1035 POINT(41.699249868728955 73.37554452469469) bank1035 +1036 POINT(41.55097796028476 73.22555400397262) bank1036 +1037 POINT(40.114921015074835 74.00295648901914) bank1037 +1038 POINT(40.37236247098049 74.43627552032594) bank1038 +1039 POINT(40.64200908990798 73.80822124593101) bank1039 +1040 POINT(40.65564520832939 73.33445848536638) bank1040 +1041 POINT(39.956468797360515 73.48377541453448) bank1041 +1042 POINT(40.391379145578234 73.54314833230394) bank1042 +1043 POINT(40.97662288367117 73.49687223678575) bank1043 +1044 POINT(40.55027971077048 74.2523484394051) bank1044 +1045 POINT(40.94173953754158 73.79218284746571) bank1045 +1046 POINT(40.636705530702024 73.32693080676735) bank1046 +1047 POINT(39.79572045141138 73.37451824880954) bank1047 +1048 POINT(40.48058822275193 74.41488479926568) bank1048 +1049 POINT(40.34182448738499 74.37313771182798) bank1049 +1050 POINT(40.56304780069766 73.2037566329995) bank1050 +1051 POINT(40.61645826828421 73.01358468125159) bank1051 +1052 POINT(41.25773094069745 73.94855191547333) bank1052 +1053 POINT(40.397830974054074 73.06444359856066) bank1053 +1054 POINT(41.5761486070056 74.35478063386255) bank1054 +1055 POINT(39.98818367500683 74.67948149613865) bank1055 +1056 POINT(41.0983484312679 74.02237410025238) bank1056 +1057 POINT(41.3925180364157 74.3261853092058) bank1057 +1058 POINT(40.176580268871035 73.52677267244128) bank1058 +1059 POINT(39.75908978610326 73.03524441868709) bank1059 +1060 POINT(40.567166101625894 74.1593421177932) bank1060 +1061 POINT(40.998661636298074 74.54706657499182) bank1061 +1062 POINT(40.963902282755406 74.74111115288733) bank1062 +1063 POINT(39.863095236927855 73.29805694168772) bank1063 +1064 POINT(40.63397957785488 73.72073548431119) bank1064 +1065 POINT(41.46213698996708 73.98090764124281) bank1065 +1066 POINT(41.28712767932362 74.52183331120023) bank1066 +1067 POINT(41.37250301839661 73.90988591606151) bank1067 +1068 POINT(40.6798769482782 73.23074425162646) bank1068 +1069 POINT(40.97335864415123 73.09993048139762) bank1069 +1070 POINT(40.95200179853712 74.21907580737874) bank1070 +1071 POINT(41.169120092066706 73.4625753885818) bank1071 +1072 POINT(40.8223250339681 74.81872947713744) bank1072 +1073 POINT(41.62064746553556 73.11316270883181) bank1073 +1074 POINT(41.483399428286035 74.34117571382271) bank1074 +1075 POINT(40.37611725064634 73.449809153606) bank1075 +1076 POINT(39.88079029321959 74.346422572144) bank1076 +1077 POINT(40.89296000395764 74.14140199010967) bank1077 +1078 POINT(41.6423636310856 74.11015857303813) bank1078 +1079 POINT(40.026865142533374 73.98302100669922) bank1079 +1080 POINT(40.40813365327683 73.81284231454777) bank1080 +1081 POINT(40.42114243640476 74.69918745186462) bank1081 +1082 POINT(40.07267329850522 73.83011159813822) bank1082 +1083 POINT(40.47852219075301 74.25376796431428) bank1083 +1084 POINT(40.912693182345635 73.55417808146991) bank1084 +1085 POINT(41.555559369174134 73.65246455862032) bank1085 +1086 POINT(41.17332926967475 73.0716455941315) bank1086 +1087 POINT(40.75693034067648 73.8266132045925) bank1087 +1088 POINT(41.25842826871526 73.18325085715414) bank1088 +1089 POINT(41.28157392485638 74.586849211191) bank1089 +1090 POINT(40.75150647494251 73.92499610012956) bank1090 +1091 POINT(40.42250990463613 73.16362021672327) bank1091 +1092 POINT(40.36449147112075 73.84910139913451) bank1092 +1093 POINT(41.327585839172144 73.58310330658621) bank1093 +1094 POINT(39.93890739449184 74.84744202000681) bank1094 +1095 POINT(40.770316764464944 74.9059711179274) bank1095 +1096 POINT(39.83674271092237 74.91066087262854) bank1096 +1097 POINT(40.94136443864919 73.9212644292405) bank1097 +1098 POINT(41.063642193790216 73.17528496446802) bank1098 +1099 POINT(41.012141044317445 74.2727785771013) bank1099 +1100 POINT(40.95141501130219 74.17568547330515) bank1100 +1101 POINT(40.65500085020122 74.9076233517826) bank1101 +1102 POINT(41.13727880350942 74.67805085775022) bank1102 +1103 POINT(39.868198353548 73.17527594883522) bank1103 +1104 POINT(41.32393910697551 73.52064227213383) bank1104 +1105 POINT(41.5891788566897 74.75757071099642) bank1105 +1106 POINT(40.27867559967926 74.53368684603086) bank1106 +1107 POINT(40.28509745179384 74.72172663032805) bank1107 +1108 POINT(40.33043987608199 73.45511744637056) bank1108 +1109 POINT(40.609531762149764 74.80165521740246) bank1109 +1110 POINT(39.977769885863694 74.72245262351542) bank1110 +1111 POINT(41.08391758994783 73.22867889723494) bank1111 +1112 POINT(40.454900742647034 74.79787314122481) bank1112 +1113 POINT(41.69848134810421 73.39814092008669) bank1113 +1114 POINT(39.80975935443062 73.99034424460281) bank1114 +1115 POINT(40.33071175198161 74.37567022902053) bank1115 +1116 POINT(41.6635594255056 73.07179739702607) bank1116 +1117 POINT(41.19586206579255 74.07727364961711) bank1117 +1118 POINT(40.53540075798694 73.09998025535394) bank1118 +1119 POINT(40.295803395629825 73.46009521253104) bank1119 +1120 POINT(41.04037534156092 74.156526439796) bank1120 +1121 POINT(40.265443100661905 74.71422715111963) bank1121 +1122 POINT(40.915443016221 74.96299750145053) bank1122 +1123 POINT(40.52156543084363 74.37860066158751) bank1123 +1124 POINT(41.5779483480844 74.3882327750654) bank1124 +1125 POINT(40.29831196381007 73.54193745147478) bank1125 +1126 POINT(40.671674498189276 73.44491485385497) bank1126 +1127 POINT(40.6324127198713 73.98491183320868) bank1127 +1128 POINT(41.3142293286691 73.15894546771422) bank1128 +1129 POINT(40.51760417982334 73.74623161850704) bank1129 +1130 POINT(41.24644088074655 74.94485135592276) bank1130 +1131 POINT(41.14760116139341 74.65357158924286) bank1131 +1132 POINT(40.3402684494061 74.51701255478588) bank1132 +1133 POINT(41.05190927397588 74.10217683120455) bank1133 +1134 POINT(40.88494769655163 73.86965220725101) bank1134 +1135 POINT(41.302586659564035 73.57201590197651) bank1135 +1136 POINT(39.97546903071034 73.40563322974256) bank1136 +1137 POINT(41.19270990597986 73.25536413616743) bank1137 +1138 POINT(40.12841742666426 73.16918351589412) bank1138 +1139 POINT(40.54092825868611 73.1917241941014) bank1139 +1140 POINT(40.72506941552691 74.77644722514337) bank1140 +1141 POINT(40.37671934070565 73.10204815617892) bank1141 +1142 POINT(40.19607807573189 74.22619540043502) bank1142 +1143 POINT(41.522162235929265 73.00692822778342) bank1143 +1144 POINT(41.36597528948055 73.71847176572433) bank1144 +1145 POINT(40.133306610048706 74.59325294542356) bank1145 +1146 POINT(41.504373744719814 73.7663613819451) bank1146 +1147 POINT(41.555382787480106 74.79971875022571) bank1147 +1148 POINT(40.4656684221502 73.72501131943591) bank1148 +1149 POINT(40.90925688085396 74.47417620630414) bank1149 +1150 POINT(40.93599591729913 74.90150678607219) bank1150 +1151 POINT(40.02240874761199 74.44023550871069) bank1151 +1152 POINT(40.622293794629066 74.05462907053318) bank1152 +1153 POINT(40.0204299076852 74.46536953821006) bank1153 +1154 POINT(41.012847206967436 75.00541948610359) bank1154 +1155 POINT(39.851330781761845 73.57755166354784) bank1155 +1156 POINT(40.68986039537529 73.59634142184252) bank1156 +1157 POINT(39.92877562471123 73.85927590837233) bank1157 +1158 POINT(40.7646307571709 74.8307184387493) bank1158 +1159 POINT(41.4479706604525 74.42794423646306) bank1159 +1160 POINT(39.8745276030789 73.84048476802556) bank1160 +1161 POINT(41.084730463768494 74.24543436342557) bank1161 +1162 POINT(41.16704144958018 73.91187461197866) bank1162 +1163 POINT(39.78569570385006 73.91066442750348) bank1163 +1164 POINT(40.80438628053973 73.59164792407529) bank1164 +1165 POINT(40.723028608952525 74.26161730721277) bank1165 +1166 POINT(39.725048518211906 74.50080549653401) bank1166 +1167 POINT(39.943475374524134 74.25332410218562) bank1167 +1168 POINT(39.75546242682157 74.74758537246392) bank1168 +1169 POINT(39.940013264152945 74.44030628069366) bank1169 +1170 POINT(40.23021906612413 73.52094256644902) bank1170 +1171 POINT(41.0010124648223 73.68515381242199) bank1171 +1172 POINT(40.04024935284076 73.92452673458824) bank1172 +1173 POINT(40.30115463306783 73.70073609476802) bank1173 +1174 POINT(41.24314559719586 73.16327438680196) bank1174 +1175 POINT(40.5335803661183 73.41132606732145) bank1175 +1176 POINT(41.566075596960744 74.73293738599678) bank1176 +1177 POINT(39.98273949149333 73.64658276001812) bank1177 +1178 POINT(40.54124653626709 73.33837145507734) bank1178 +1179 POINT(40.37157272927294 73.17307647918292) bank1179 +1180 POINT(39.96398335992873 74.94429769008049) bank1180 +1181 POINT(41.24318870317754 73.55819665172498) bank1181 +1182 POINT(40.32050036065347 74.3109677609108) bank1182 +1183 POINT(40.86178858675889 74.61670630270899) bank1183 +1184 POINT(40.502932843866894 74.39542388166808) bank1184 +1185 POINT(41.4692624264016 73.98115840083273) bank1185 +1186 POINT(40.25977550662661 74.62456216423067) bank1186 +1187 POINT(40.04906911268502 73.62967857172848) bank1187 +1188 POINT(40.44343576138949 73.43122106728957) bank1188 +1189 POINT(40.62461981264446 73.02716811242172) bank1189 +1190 POINT(41.345095957085235 74.27365755630363) bank1190 +1191 POINT(41.39889432645461 74.5174747757158) bank1191 +1192 POINT(40.94430449216892 73.80085289818265) bank1192 +1193 POINT(41.14497194786321 74.38739416674625) bank1193 +1194 POINT(41.519710832248386 74.96428530750404) bank1194 +1195 POINT(40.70821079884253 73.69570700336975) bank1195 +1196 POINT(41.367631553247456 73.23742511683277) bank1196 +1197 POINT(39.909483265952886 73.0510300893613) bank1197 +1198 POINT(41.09469550738054 73.09310753506905) bank1198 +1199 POINT(41.49988142965855 73.20061119074826) bank1199 +1200 POINT(40.53022477930467 74.3176036435011) bank1200 +1201 POINT(40.491101759825256 73.33947040757636) bank1201 +1202 POINT(40.03041203956848 74.44589091134806) bank1202 +1203 POINT(40.359410823492105 74.31414979285987) bank1203 +1204 POINT(40.486225945553954 73.13251507008248) bank1204 +1205 POINT(41.60002213743881 74.13256623061302) bank1205 +1206 POINT(40.267961166264094 74.40542822372923) bank1206 +1207 POINT(41.68913725785511 73.50984893063729) bank1207 +1208 POINT(39.84066112925211 73.70093748380326) bank1208 +1209 POINT(41.68368340442732 73.84746584610114) bank1209 +1210 POINT(40.24327898549528 74.68461217704701) bank1210 +1211 POINT(39.748970569449746 73.74564475871358) bank1211 +1212 POINT(39.830670351886205 74.74055690429377) bank1212 +1213 POINT(41.42790075721652 73.37553152130027) bank1213 +1214 POINT(41.56294703761298 73.35891193369551) bank1214 +1215 POINT(40.82453281229981 74.85679200103644) bank1215 +1216 POINT(40.97643492751039 73.74379748451145) bank1216 +1217 POINT(41.45007753232228 73.05689739179705) bank1217 +1218 POINT(40.999479731951105 74.7588211581574) bank1218 +1219 POINT(41.47929101442822 74.16992849437635) bank1219 +1220 POINT(41.63970959264235 73.12548902694812) bank1220 +1221 POINT(41.20362389962135 74.15440525235954) bank1221 +1222 POINT(39.78525729285518 73.11466578867498) bank1222 +1223 POINT(41.601266440673626 73.27763690246037) bank1223 +1224 POINT(40.622539020066505 73.50062262072274) bank1224 +1225 POINT(41.034572454563516 73.29847539344422) bank1225 +1226 POINT(41.54324609175914 73.08380910395505) bank1226 +1227 POINT(40.93013305028618 73.98912412850075) bank1227 +1228 POINT(41.26903554014699 73.65313391318523) bank1228 +1229 POINT(40.61911991848946 74.21667262132566) bank1229 +1230 POINT(41.136497444859266 73.51278026054288) bank1230 +1231 POINT(40.85401143592304 73.72360392694235) bank1231 +1232 POINT(40.060621822202535 74.48437518947071) bank1232 +1233 POINT(41.36003261680587 74.06733574692768) bank1233 +1234 POINT(40.623087746127574 74.64650242332596) bank1234 +1235 POINT(40.98398785698739 74.42082943927743) bank1235 +1236 POINT(40.64161717618513 73.65513975966196) bank1236 +1237 POINT(39.83648040849844 73.81803519103308) bank1237 +1238 POINT(41.65291043129222 74.66967538202807) bank1238 +1239 POINT(40.3771672625853 74.3076521755875) bank1239 +1240 POINT(41.34084696639066 73.09261303021614) bank1240 +1241 POINT(41.34801388000127 74.96694291999817) bank1241 +1242 POINT(40.49889274985182 74.04407834373517) bank1242 +1243 POINT(40.532300504575886 73.35089597050387) bank1243 +1244 POINT(41.12777763623478 74.35273923315741) bank1244 +1245 POINT(39.93836101905777 73.44524607265414) bank1245 +1246 POINT(41.40469150488909 73.44330606997411) bank1246 +1247 POINT(41.60264282071231 74.50905155210314) bank1247 +1248 POINT(40.27248756104084 73.4857431903611) bank1248 +1249 POINT(41.57406853945653 73.11999360000894) bank1249 +1250 POINT(40.83746996408784 74.552874910409) bank1250 +1251 POINT(39.936848985370695 73.54783272795338) bank1251 +1252 POINT(41.57992588270393 74.63568143454653) bank1252 +1253 POINT(40.70686743411428 73.11164012251665) bank1253 +1254 POINT(39.84822919140968 74.5302803647059) bank1254 +1255 POINT(41.535319477996495 73.0655061674593) bank1255 +1256 POINT(39.73381651624518 74.85272576349267) bank1256 +1257 POINT(41.27493588480284 74.5261575188115) bank1257 +1258 POINT(40.421312443318605 73.63093985986224) bank1258 +1259 POINT(40.81941256725277 73.95009924139372) bank1259 +1260 POINT(41.429852806193736 73.51718115622623) bank1260 +1261 POINT(39.804212644075754 74.95583259091691) bank1261 +1262 POINT(40.64438729109633 73.69480212158822) bank1262 +1263 POINT(40.47925546138027 73.68350583120589) bank1263 +1264 POINT(40.63508325034344 74.0804077086721) bank1264 +1265 POINT(41.2103606103513 73.40064242217343) bank1265 +1266 POINT(40.77634627979149 74.33680178180667) bank1266 +1267 POINT(40.51783887626525 73.15511667267387) bank1267 +1268 POINT(40.91444143236493 73.9346915148601) bank1268 +1269 POINT(41.623572019258766 73.17336652495307) bank1269 +1270 POINT(40.90501744363863 74.73876725661881) bank1270 +1271 POINT(40.69693028745855 74.89532989438794) bank1271 +1272 POINT(40.78268401847412 73.26037564935187) bank1272 +1273 POINT(41.52418840079634 73.43438254234063) bank1273 +1274 POINT(40.19799342257936 73.34652792051054) bank1274 +1275 POINT(41.05246188268944 73.40457997595395) bank1275 +1276 POINT(40.72479895047326 73.92153096355965) bank1276 +1277 POINT(41.67550934205825 74.90910236158656) bank1277 +1278 POINT(40.06640666147523 73.27217946034027) bank1278 +1279 POINT(40.667919122852446 73.46288017427226) bank1279 +1280 POINT(41.04826211557801 74.11203863155947) bank1280 +1281 POINT(41.301671398662386 74.71197761974841) bank1281 +1282 POINT(40.06359542907189 74.01599469351072) bank1282 +1283 POINT(41.59690736414453 73.7946360073583) bank1283 +1284 POINT(40.83483538911093 73.5630003838575) bank1284 +1285 POINT(40.289984926234176 73.75445063729956) bank1285 +1286 POINT(40.19957066758844 73.94142444422907) bank1286 +1287 POINT(40.79620886716637 74.74051675199445) bank1287 +1288 POINT(41.4793789540958 73.16391016074137) bank1288 +1289 POINT(40.69443507938608 74.168025351761) bank1289 +1290 POINT(40.50702347319136 74.97621678505139) bank1290 +1291 POINT(41.50895476197453 74.74445965387622) bank1291 +1292 POINT(40.79896973574806 74.24073757039417) bank1292 +1293 POINT(39.77431432930362 73.08676976875724) bank1293 +1294 POINT(40.0569911825933 74.61458147145926) bank1294 +1295 POINT(40.287773222452785 74.56009180789691) bank1295 +1296 POINT(40.85309618284277 74.99930353710718) bank1296 +1297 POINT(40.39114467990407 73.53110659031354) bank1297 +1298 POINT(40.4458729899661 73.07899744328304) bank1298 +1299 POINT(39.95285754376298 74.68978086844143) bank1299 +1300 POINT(39.99857988558443 74.94251657884318) bank1300 +1301 POINT(41.336092569443835 74.12446927262782) bank1301 +1302 POINT(41.049330881065906 73.2872734333463) bank1302 +1303 POINT(41.38396142760007 74.60076802938332) bank1303 +1304 POINT(41.191280498686645 73.85380283174) bank1304 +1305 POINT(39.73935468856973 74.2573374666328) bank1305 +1306 POINT(41.05714290793983 73.96539678316148) bank1306 +1307 POINT(40.73823202416642 73.91492499918074) bank1307 +1308 POINT(40.310134025305565 73.1258288559543) bank1308 +1309 POINT(41.35592352411963 74.89893813813678) bank1309 +1310 POINT(41.02456867343281 74.87153581413112) bank1310 +1311 POINT(40.09166350927008 74.8595894493909) bank1311 +1312 POINT(41.45227397940147 73.5982183050236) bank1312 +1313 POINT(41.57360450788093 73.70140783548862) bank1313 +1314 POINT(40.88194522152526 73.34050548754553) bank1314 +1315 POINT(40.73711375316239 73.19408947008462) bank1315 +1316 POINT(41.12573489769655 73.71069583942592) bank1316 +1317 POINT(40.96403946748289 74.12758362012983) bank1317 +1318 POINT(40.631867057295175 74.8563232232543) bank1318 +1319 POINT(40.62460822244293 74.61382522114114) bank1319 +1320 POINT(40.66677114657382 73.58002036244623) bank1320 +1321 POINT(41.536127265872345 74.39888871219893) bank1321 +1322 POINT(40.4854794898578 74.76900353048191) bank1322 +1323 POINT(40.322451388520186 74.45586552297519) bank1323 +1324 POINT(40.624810958574656 74.46500285308608) bank1324 +1325 POINT(40.14533119153834 74.28599928809871) bank1325 +1326 POINT(40.0525869437253 73.38476912814077) bank1326 +1327 POINT(40.07908103323959 73.36306952707113) bank1327 +1328 POINT(40.30284449643136 73.92581639109977) bank1328 +1329 POINT(41.18538312606395 73.30340215800528) bank1329 +1330 POINT(41.168098852258964 73.58874707196428) bank1330 +1331 POINT(40.70012616972052 74.21151272511933) bank1331 +1332 POINT(41.4379909159318 73.54480027613548) bank1332 +1333 POINT(41.36018506543085 74.36781277061331) bank1333 +1334 POINT(40.97105235633399 74.07555758073444) bank1334 +1335 POINT(39.9155785346021 73.1116057113062) bank1335 +1336 POINT(40.632847084904874 73.49848367905497) bank1336 +1337 POINT(41.406803238006546 73.52529706179959) bank1337 +1338 POINT(40.27727353319512 73.05612176830152) bank1338 +1339 POINT(40.426325193141395 74.79992437625496) bank1339 +1340 POINT(40.063640045979895 73.73198584608409) bank1340 +1341 POINT(41.52156840868929 73.69523437698876) bank1341 +1342 POINT(41.31938449273718 74.77325172070753) bank1342 +1343 POINT(40.28970769940578 74.40474248295423) bank1343 +1344 POINT(39.93805528887395 73.62648767714244) bank1344 +1345 POINT(40.260106588723424 73.87882732383288) bank1345 +1346 POINT(41.07780721026268 74.94812010505987) bank1346 +1347 POINT(39.94193179698629 74.63012154137728) bank1347 +1348 POINT(40.9590437131146 73.47337174775676) bank1348 +1349 POINT(40.072971960390305 74.49379840488432) bank1349 +1350 POINT(41.03469141118455 74.73955101601946) bank1350 +1351 POINT(41.57386622861903 73.76332963874972) bank1351 +1352 POINT(41.63503000318983 73.31377218659307) bank1352 +1353 POINT(41.469138838651006 74.14776025999556) bank1353 +1354 POINT(40.170545021731726 73.73555771237096) bank1354 +1355 POINT(41.63001656462063 73.37885026211781) bank1355 +1356 POINT(40.58969746590965 73.80351752497235) bank1356 +1357 POINT(41.665782886636244 73.20317124885959) bank1357 +1358 POINT(40.03983710443753 74.67712254547163) bank1358 +1359 POINT(41.333236444291586 73.6330516212564) bank1359 +1360 POINT(40.21495614418925 73.94837663905645) bank1360 +1361 POINT(40.60808657207909 74.49848774269151) bank1361 +1362 POINT(41.48965909968406 73.73650462121242) bank1362 +1363 POINT(40.55941745182184 73.82516099301463) bank1363 +1364 POINT(41.36330594130675 75.00520895132433) bank1364 +1365 POINT(40.81758969005508 74.68825510030825) bank1365 +1366 POINT(40.82873054879774 73.16378931012274) bank1366 +1367 POINT(41.277398705787036 74.63067698114253) bank1367 +1368 POINT(40.216251263427615 74.75676668672739) bank1368 +1369 POINT(41.610395096504035 73.45406599445187) bank1369 +1370 POINT(40.31937591121939 74.67977853849361) bank1370 +1371 POINT(40.248519486024655 74.33415917601394) bank1371 +1372 POINT(40.29554475647923 73.35708304681941) bank1372 +1373 POINT(40.378832233114984 75.00475335747068) bank1373 +1374 POINT(39.94679321448426 73.9329504461645) bank1374 +1375 POINT(39.74075985142764 74.45005138211492) bank1375 +1376 POINT(40.63257348280311 74.13931887190178) bank1376 +1377 POINT(41.08072370652309 74.31299774427319) bank1377 +1378 POINT(41.324535309718094 73.59460489528719) bank1378 +1379 POINT(40.28216241351834 74.15352522213792) bank1379 +1380 POINT(41.09768607559526 74.63540234453252) bank1380 +1381 POINT(40.5373620256238 73.37126024911336) bank1381 +1382 POINT(39.726145503122325 73.87175102289072) bank1382 +1383 POINT(40.197274187118545 73.47169331678887) bank1383 +1384 POINT(40.50714259263708 74.79858094479039) bank1384 +1385 POINT(41.40459977333516 73.0104328411792) bank1385 +1386 POINT(39.90729450079089 73.36877772778277) bank1386 +1387 POINT(40.95888483627178 73.69631369602101) bank1387 +1388 POINT(39.77773413082732 73.59749911886345) bank1388 +1389 POINT(40.81620535232791 73.53424300184268) bank1389 +1390 POINT(40.43185048855876 74.52322190280601) bank1390 +1391 POINT(41.51276731569483 74.00847081751239) bank1391 +1392 POINT(39.903978748836096 73.59107323651772) bank1392 +1393 POINT(40.74311101178301 73.25242462990147) bank1393 +1394 POINT(41.52664838847695 74.63884738011384) bank1394 +1395 POINT(40.486249922476844 73.25890790954966) bank1395 +1396 POINT(40.13669888577334 74.1241738358839) bank1396 +1397 POINT(41.14505524704245 74.46242492920409) bank1397 +1398 POINT(41.692985851381856 73.85894967540808) bank1398 +1399 POINT(41.37046655944995 73.95535864977047) bank1399 +1400 POINT(40.713762444385985 73.21148210751944) bank1400 +1401 POINT(41.38420722284335 73.44479460661697) bank1401 +1402 POINT(41.40372218663852 74.41122727385716) bank1402 +1403 POINT(41.07634437232345 73.49225004113772) bank1403 +1404 POINT(40.31070332136185 73.90287547577877) bank1404 +1405 POINT(39.811753771073064 74.3153425805288) bank1405 +1406 POINT(40.62109753379494 73.30349423499953) bank1406 +1407 POINT(40.09352605987892 74.24735879473778) bank1407 +1408 POINT(39.952559604871155 74.8555991708387) bank1408 +1409 POINT(40.93385374188487 74.13688204992697) bank1409 +1410 POINT(41.54098657051936 73.59471609200313) bank1410 +1411 POINT(39.96161356909988 73.97351267420305) bank1411 +1412 POINT(39.737694113812985 74.12137321666134) bank1412 +1413 POINT(39.905374856202734 73.49578197044686) bank1413 +1414 POINT(41.45732710373562 73.5634240429419) bank1414 +1415 POINT(40.940468673095815 74.60628117007626) bank1415 +1416 POINT(41.67263186359446 74.22211840441439) bank1416 +1417 POINT(40.53971836219436 73.2993140291303) bank1417 +1418 POINT(41.25927811134409 73.8908798614653) bank1418 +1419 POINT(41.41058347227835 74.7420766442919) bank1419 +1420 POINT(39.83181617768101 74.00655089066282) bank1420 +1421 POINT(41.168496147900896 73.99836420864082) bank1421 +1422 POINT(41.609659447543216 73.48989052305241) bank1422 +1423 POINT(41.68997961634162 74.93577342274675) bank1423 +1424 POINT(41.65896465481246 73.45021975230082) bank1424 +1425 POINT(40.67150646825044 74.21498455850966) bank1425 +1426 POINT(41.292445154660875 73.70046939550099) bank1426 +1427 POINT(40.636891018825665 74.97693261706569) bank1427 +1428 POINT(40.85650565018342 73.68216171433187) bank1428 +1429 POINT(40.33357385162019 73.5656707311363) bank1429 +1430 POINT(41.225682844233326 73.63341226370834) bank1430 +1431 POINT(40.605976196048765 74.5684616110596) bank1431 +1432 POINT(40.65582628003331 73.84794608830278) bank1432 +1433 POINT(41.16696046535828 74.91836334929643) bank1433 +1434 POINT(40.03155741040315 74.57232128896298) bank1434 +1435 POINT(40.00062149811557 73.68250247657708) bank1435 +1436 POINT(40.16109246290948 74.92182299456069) bank1436 +1437 POINT(41.59391373198006 73.38132918606998) bank1437 +1438 POINT(40.44023099492565 73.91650224147227) bank1438 +1439 POINT(41.64745136219486 74.91423036771644) bank1439 +1440 POINT(40.23511428549571 74.35157439267667) bank1440 +1441 POINT(41.040549297601935 73.02646229588038) bank1441 +1442 POINT(40.31138099344867 74.29175728434144) bank1442 +1443 POINT(40.02630472439296 74.22990261558009) bank1443 +1444 POINT(40.27075550494002 73.3535352948099) bank1444 +1445 POINT(41.14498455047111 74.18784594780314) bank1445 +1446 POINT(41.37018067937926 73.93844049738486) bank1446 +1447 POINT(41.073493181447034 73.81149757997491) bank1447 +1448 POINT(41.2567837422043 74.29558556103486) bank1448 +1449 POINT(40.3065866349062 73.69686862275736) bank1449 +1450 POINT(41.4580091466134 73.52153061731087) bank1450 +1451 POINT(40.66526831559842 73.16984820650858) bank1451 +1452 POINT(41.684520068379214 74.87973063460784) bank1452 +1453 POINT(40.71862406924189 73.85436104076301) bank1453 +1454 POINT(40.9860383958544 73.20840789441685) bank1454 +1455 POINT(40.60641634797405 74.40987847169463) bank1455 +1456 POINT(40.47535685499651 74.25847891083326) bank1456 +1457 POINT(40.47777661924471 73.97987337288623) bank1457 +1458 POINT(40.87228300873612 74.06961077731052) bank1458 +1459 POINT(40.37464090198063 74.65314771770531) bank1459 +1460 POINT(41.593820548922814 73.30484844015353) bank1460 +1461 POINT(39.98157937330224 74.2696427603788) bank1461 +1462 POINT(40.7407471958272 74.42006243576039) bank1462 +1463 POINT(41.2806167966249 73.47404082975302) bank1463 +1464 POINT(40.51646886630018 74.76337760537687) bank1464 +1465 POINT(41.65310608526776 73.0400267391153) bank1465 +1466 POINT(39.82140991088565 74.08000463974321) bank1466 +1467 POINT(40.27075126424182 74.76636191159355) bank1467 +1468 POINT(40.6643375698059 73.22947018978695) bank1468 +1469 POINT(41.35783188233139 74.46541465625386) bank1469 +1470 POINT(40.308116368689966 74.37211534541136) bank1470 +1471 POINT(40.208312626913056 73.27100816874385) bank1471 +1472 POINT(41.16732947866173 74.02960419235967) bank1472 +1473 POINT(40.67642427247189 73.70433671753841) bank1473 +1474 POINT(41.63639212701255 74.46988536859558) bank1474 +1475 POINT(40.94512535078479 74.41863045758781) bank1475 +1476 POINT(41.45616960896899 74.56575571948534) bank1476 +1477 POINT(39.99395880578312 74.96767924006669) bank1477 +1478 POINT(39.76658854986191 73.36837313296112) bank1478 +1479 POINT(40.55497144712099 74.2519329256626) bank1479 +1480 POINT(40.84055466567682 74.60477228864325) bank1480 +1481 POINT(40.41130360087654 74.8322516369806) bank1481 +1482 POINT(40.276638185436084 73.80445733668104) bank1482 +1483 POINT(40.56492941039849 74.12016163210734) bank1483 +1484 POINT(41.70591668517311 73.4012245879656) bank1484 +1485 POINT(39.857193885401195 73.54322916366092) bank1485 +1486 POINT(39.989399939823606 74.8111964911204) bank1486 +1487 POINT(41.19540772363044 74.37434252891832) bank1487 +1488 POINT(40.52458592144548 74.51499782658473) bank1488 +1489 POINT(40.99900197240186 73.56594650023592) bank1489 +1490 POINT(40.72768640999006 73.34563098775304) bank1490 +1491 POINT(40.419560944572964 74.49195832025875) bank1491 +1492 POINT(40.5752319482858 73.88756764720101) bank1492 +1493 POINT(41.701893692486486 73.12765975326315) bank1493 +1494 POINT(39.81040053756777 74.88758467321219) bank1494 +1495 POINT(41.366634364991675 73.02089435071574) bank1495 +1496 POINT(40.11886104920544 74.76984859635871) bank1496 +1497 POINT(41.43960675908305 74.95094408584364) bank1497 +1498 POINT(39.78513151782377 73.11431964225902) bank1498 +1499 POINT(39.80366596961749 74.80367100629606) bank1499 +1500 POINT(40.84974859517362 74.98184667154305) bank1500 +1501 POINT(41.67159115121923 74.43348742680439) bank1501 +1502 POINT(40.39692176973761 74.66946841223807) bank1502 +1503 POINT(40.03549972414178 73.13184518315288) bank1503 +1504 POINT(41.24040197728074 74.93031906686699) bank1504 +1505 POINT(40.18603301687564 73.4363948811343) bank1505 +1506 POINT(40.591056203650865 73.86754719714266) bank1506 +1507 POINT(41.260628499073924 74.11839650774846) bank1507 +1508 POINT(40.69193360865644 73.55331131494293) bank1508 +1509 POINT(40.208233987912635 74.59229641502812) bank1509 +1510 POINT(40.531263180160934 73.59747183248123) bank1510 +1511 POINT(41.0374382870314 74.92852319825536) bank1511 +1512 POINT(40.76783489140891 74.57541683820733) bank1512 +1513 POINT(41.23834937169399 74.0256125623495) bank1513 +1514 POINT(40.65112325655885 73.95650286566985) bank1514 +1515 POINT(41.176312025747784 74.82151384986327) bank1515 +1516 POINT(41.545260954516905 74.93417165391219) bank1516 +1517 POINT(40.23738926606982 74.71831224576438) bank1517 +1518 POINT(41.66834784736466 74.47343157659739) bank1518 +1519 POINT(40.650259253255115 74.45210905789945) bank1519 +1520 POINT(40.45837198810852 73.69525013645651) bank1520 +1521 POINT(40.79948712961781 74.9185353697028) bank1521 +1522 POINT(40.286857000162506 73.63026232886585) bank1522 +1523 POINT(41.65179117561256 73.40441181966817) bank1523 +1524 POINT(40.56824473224478 74.65350549900384) bank1524 +1525 POINT(40.94737819868412 74.81353047419094) bank1525 +1526 POINT(40.059095374467745 73.69754501414101) bank1526 +1527 POINT(41.62833137882662 74.6438345708697) bank1527 +1528 POINT(40.16911801806002 73.53095065742652) bank1528 +1529 POINT(40.54532233798102 74.55357474007542) bank1529 +1530 POINT(40.08819662787009 74.32359989829571) bank1530 +1531 POINT(39.73958448802092 73.91622144796187) bank1531 +1532 POINT(41.29461977608703 74.25124599736824) bank1532 +1533 POINT(40.32169647951375 73.27600824825487) bank1533 +1534 POINT(41.55974406407317 73.23063880283934) bank1534 +1535 POINT(40.76796155771115 74.62644551289645) bank1535 +1536 POINT(40.235952565963764 73.1433729957089) bank1536 +1537 POINT(40.1817512954381 74.92563004355607) bank1537 +1538 POINT(40.37600355786775 74.73029421596826) bank1538 +1539 POINT(39.99242319391762 73.85814576711302) bank1539 +1540 POINT(40.39590658967217 73.9004258108194) bank1540 +1541 POINT(40.89320725507525 74.07632241636311) bank1541 +1542 POINT(40.58380620860258 74.37570378636276) bank1542 +1543 POINT(40.659617551565496 74.19843326583322) bank1543 +1544 POINT(41.62852385462085 74.60104250467758) bank1544 +1545 POINT(40.3578932220671 74.14067835369511) bank1545 +1546 POINT(40.64795815842716 74.75703668793824) bank1546 +1547 POINT(39.778297532828596 74.28871242925554) bank1547 +1548 POINT(41.22297472257084 73.3440263335114) bank1548 +1549 POINT(41.12427506800968 73.29924622574725) bank1549 +1550 POINT(41.15833901956398 73.93013631038066) bank1550 +1551 POINT(39.756421253759996 73.10227864421354) bank1551 +1552 POINT(41.04142061487044 74.50095078645832) bank1552 +1553 POINT(40.81925776592708 74.84368123045937) bank1553 +1554 POINT(40.61755542787458 74.33825119052729) bank1554 +1555 POINT(41.65372917524116 74.62048697080553) bank1555 +1556 POINT(40.198143414908735 73.10687461398906) bank1556 +1557 POINT(41.0060067734369 73.45206450324297) bank1557 +1558 POINT(39.992950583745554 74.2593829079241) bank1558 +1559 POINT(41.698916599276174 74.9727171203163) bank1559 +1560 POINT(41.48135667511787 73.16810991420584) bank1560 +1561 POINT(41.39324107632642 73.67733025132746) bank1561 +1562 POINT(40.92101469779399 73.08419484066086) bank1562 +1563 POINT(40.662187883094234 73.94524902315254) bank1563 +1564 POINT(40.076401305962584 73.36497208448506) bank1564 +1565 POINT(40.209276987146986 73.27239696061343) bank1565 +1566 POINT(39.74918334881553 74.30908586562865) bank1566 +1567 POINT(40.10083803631234 73.70352384180363) bank1567 +1568 POINT(41.43328307139335 74.81483743658761) bank1568 +1569 POINT(40.37166751479373 73.8538719221851) bank1569 +1570 POINT(40.77668743559373 73.36214136551858) bank1570 +1571 POINT(40.66197136128631 73.96193641691902) bank1571 +1572 POINT(40.62880084725472 74.76766608022146) bank1572 +1573 POINT(41.101929021527084 74.74084218311305) bank1573 +1574 POINT(39.89053773170442 73.66672974510372) bank1574 +1575 POINT(41.40042182907461 73.38922110234753) bank1575 +1576 POINT(41.14339176468676 73.49771801149718) bank1576 +1577 POINT(40.287529103256034 74.65828727658445) bank1577 +1578 POINT(40.66164937744317 73.64366172230517) bank1578 +1579 POINT(40.224234041809694 74.20920176948799) bank1579 +1580 POINT(39.76348314285577 73.1511099260481) bank1580 +1581 POINT(41.681878658183436 73.13955515827215) bank1581 +1582 POINT(40.14480607190769 73.20187625015397) bank1582 +1583 POINT(40.80041390697499 73.44803119917403) bank1583 +1584 POINT(40.840463536447395 73.01038804159286) bank1584 +1585 POINT(39.931047416982366 73.61826153164183) bank1585 +1586 POINT(41.23908314917775 74.82049723116197) bank1586 +1587 POINT(41.5116674324701 74.28677143461572) bank1587 +1588 POINT(39.84775455142998 73.78698023961914) bank1588 +1589 POINT(40.749326963354505 73.50914043227621) bank1589 +1590 POINT(41.102986171624345 74.2983108276136) bank1590 +1591 POINT(40.71819472271379 73.99954160584151) bank1591 +1592 POINT(41.64221008717997 74.15712899613045) bank1592 +1593 POINT(40.56824596898586 73.57320146835558) bank1593 +1594 POINT(40.1159307966057 73.01420299136211) bank1594 +1595 POINT(40.178541666793365 74.97132047987797) bank1595 +1596 POINT(41.11995937282778 73.0346055516005) bank1596 +1597 POINT(41.10380781992238 74.14270466360749) bank1597 +1598 POINT(40.47857327037184 74.58024391620532) bank1598 +1599 POINT(40.59491200106659 73.88383540469502) bank1599 +1600 POINT(40.57220870166847 74.22343195177302) bank1600 +1601 POINT(39.73289490716125 73.17771694018171) bank1601 +1602 POINT(41.287281721159424 73.13020432847213) bank1602 +1603 POINT(41.25308549382528 73.04631554127236) bank1603 +1604 POINT(39.8801016493208 73.59545764810798) bank1604 +1605 POINT(40.03483296149638 74.78165806846079) bank1605 +1606 POINT(39.8529695878022 73.03465703667045) bank1606 +1607 POINT(40.321855424257116 74.887324954291) bank1607 +1608 POINT(40.96978721766433 74.61077913823479) bank1608 +1609 POINT(39.931335526412866 73.6511606569652) bank1609 +1610 POINT(40.54732143586935 74.47675909101741) bank1610 +1611 POINT(41.23531917194389 74.9106656333733) bank1611 +1612 POINT(40.43360375789347 73.9070917082028) bank1612 +1613 POINT(40.49001902052859 73.43250990569823) bank1613 +1614 POINT(41.2696356388806 74.14781816353295) bank1614 +1615 POINT(41.61385324428484 74.08672742501612) bank1615 +1616 POINT(41.23389952947913 73.03787467121883) bank1616 +1617 POINT(40.48353278653486 73.81329917323595) bank1617 +1618 POINT(41.29754584627902 74.15765292224383) bank1618 +1619 POINT(40.76015978214932 73.62295708033194) bank1619 +1620 POINT(40.71447026556193 73.39425636791219) bank1620 +1621 POINT(41.65767549889935 74.79853162603231) bank1621 +1622 POINT(41.21920114277338 73.73528195228934) bank1622 +1623 POINT(39.73351857480639 74.44448680668695) bank1623 +1624 POINT(41.6067685674775 74.20096210666733) bank1624 +1625 POINT(40.65231756299538 74.7876473015549) bank1625 +1626 POINT(41.45274945730009 73.41756579154243) bank1626 +1627 POINT(41.04377618407312 74.60700697261575) bank1627 +1628 POINT(41.59613749025394 74.08160445053298) bank1628 +1629 POINT(40.130407098099155 74.93998807434953) bank1629 +1630 POINT(41.4095503155049 74.0388248197713) bank1630 +1631 POINT(41.68266446792894 73.45198890397451) bank1631 +1632 POINT(39.97775313722773 73.90956759062261) bank1632 +1633 POINT(40.21332629807161 74.06000167866354) bank1633 +1634 POINT(41.03238535556428 74.35329840882395) bank1634 +1635 POINT(41.09158826868858 73.05355590813664) bank1635 +1636 POINT(40.83464167776755 73.38050813601718) bank1636 +1637 POINT(39.80829477420181 74.4178978310941) bank1637 +1638 POINT(41.15616197947234 74.3707531105449) bank1638 +1639 POINT(39.87303621058301 74.1448608560227) bank1639 +1640 POINT(40.49115550932306 73.98739050903993) bank1640 +1641 POINT(40.99276891832855 74.1479862286201) bank1641 +1642 POINT(41.19653116877221 73.09503329897957) bank1642 +1643 POINT(40.98522275285329 74.11113666256311) bank1643 +1644 POINT(40.35490214158009 73.90836009327751) bank1644 +1645 POINT(39.946667579967674 74.50662805735224) bank1645 +1646 POINT(40.8226388318633 74.0677408495207) bank1646 +1647 POINT(40.53667406182975 74.05722328083208) bank1647 +1648 POINT(40.03276484790866 74.54725577034968) bank1648 +1649 POINT(40.107060774113194 74.81441450402795) bank1649 +1650 POINT(40.69161147232235 74.43229252515678) bank1650 +1651 POINT(41.297348275515795 74.91472810629972) bank1651 +1652 POINT(40.32867638047068 74.85489007500587) bank1652 +1653 POINT(41.23732324275921 74.90997667065638) bank1653 +1654 POINT(40.65116361266176 74.60880858344647) bank1654 +1655 POINT(40.73738005661603 74.5218540833716) bank1655 +1656 POINT(40.35182351153189 74.84847385888428) bank1656 +1657 POINT(39.94493031032884 73.24231938076822) bank1657 +1658 POINT(39.99424636308753 73.70469809367657) bank1658 +1659 POINT(41.127480101771354 74.33926093596834) bank1659 +1660 POINT(41.36281821750063 74.32976360878904) bank1660 +1661 POINT(40.71003167877365 74.42574239295165) bank1661 +1662 POINT(40.40916140843688 74.0005685556109) bank1662 +1663 POINT(40.81361471014925 74.0984082045547) bank1663 +1664 POINT(40.38163882961264 73.14103338769286) bank1664 +1665 POINT(40.82311863558021 74.53691414256377) bank1665 +1666 POINT(41.13983418705326 73.61237215000793) bank1666 +1667 POINT(41.03736264089742 73.443553758361) bank1667 +1668 POINT(41.07578829956801 73.76852378852186) bank1668 +1669 POINT(40.496685935426186 73.61574625522273) bank1669 +1670 POINT(40.864750784026796 74.0640930060486) bank1670 +1671 POINT(40.710729950601845 73.55064260194905) bank1671 +1672 POINT(40.76701275714075 74.39910085376863) bank1672 +1673 POINT(39.83232760418011 73.94956994362693) bank1673 +1674 POINT(41.549666063951925 73.0493712962224) bank1674 +1675 POINT(40.30304453241215 74.34888654893913) bank1675 +1676 POINT(41.29151679461925 73.12242266245141) bank1676 +1677 POINT(41.56407646302963 73.47701747890554) bank1677 +1678 POINT(39.77620187385843 73.53794320588102) bank1678 +1679 POINT(40.313563330656365 74.88209436255246) bank1679 +1680 POINT(40.867809072601254 73.60280007212577) bank1680 +1681 POINT(40.72398635249548 73.1132587965839) bank1681 +1682 POINT(41.029037761986395 73.38959273683987) bank1682 +1683 POINT(39.76013650159967 74.14149604044532) bank1683 +1684 POINT(40.39108567485969 73.61707783609043) bank1684 +1685 POINT(40.91326868491255 74.43650069640088) bank1685 +1686 POINT(39.896804937259965 74.17125789401017) bank1686 +1687 POINT(40.065304371956934 74.0583061813835) bank1687 +1688 POINT(40.47128078270193 74.88524872091133) bank1688 +1689 POINT(41.25531802601977 74.83345899214761) bank1689 +1690 POINT(40.063104704409476 74.43184016843998) bank1690 +1691 POINT(41.609817735932516 74.40984444809297) bank1691 +1692 POINT(40.87873251232076 73.18093212067883) bank1692 +1693 POINT(41.58979783731771 74.63285956192458) bank1693 +1694 POINT(40.26267678352846 73.5897976258137) bank1694 +1695 POINT(40.65675393778287 74.03701550025659) bank1695 +1696 POINT(40.6442772692314 73.86391719236453) bank1696 +1697 POINT(41.22678800763271 73.12030992183918) bank1697 +1698 POINT(40.12935569117296 74.33743151855464) bank1698 +1699 POINT(41.504040730894545 74.84790482781857) bank1699 +1700 POINT(40.71439094666909 73.50580718330257) bank1700 +1701 POINT(40.73871946066207 74.11889944743406) bank1701 +1702 POINT(40.96622118636591 73.3186451814727) bank1702 +1703 POINT(40.03405182572982 74.5864276650732) bank1703 +1704 POINT(39.8158388654673 73.24951984714868) bank1704 +1705 POINT(41.6816657342177 73.81762339387372) bank1705 +1706 POINT(41.08534282993199 73.31635162160532) bank1706 +1707 POINT(40.20910754321232 74.61500642381282) bank1707 +1708 POINT(39.72980009690191 74.78702390440526) bank1708 +1709 POINT(41.38508224639563 74.17887301414873) bank1709 +1710 POINT(40.31160267962122 74.72181627139891) bank1710 +1711 POINT(40.81604769251621 74.3572986664574) bank1711 +1712 POINT(40.23010315617757 74.7459296813007) bank1712 +1713 POINT(40.23840130882981 74.4868547154459) bank1713 +1714 POINT(40.815449554788344 74.24216459356298) bank1714 +1715 POINT(41.28698737867267 73.08332016812447) bank1715 +1716 POINT(39.71890481218269 74.92892001748764) bank1716 +1717 POINT(40.59235265080888 74.18258481138061) bank1717 +1718 POINT(41.348211786960185 73.21401127774084) bank1718 +1719 POINT(41.54771259917629 74.85271503562872) bank1719 +1720 POINT(40.84658513296057 74.80432119385526) bank1720 +1721 POINT(40.483615833709116 73.82455060645802) bank1721 +1722 POINT(40.543156369696916 73.62295605823881) bank1722 +1723 POINT(40.06797373503279 74.29411088956596) bank1723 +1724 POINT(41.405607210672585 73.22955034120744) bank1724 +1725 POINT(40.426734623638126 74.83523197230794) bank1725 +1726 POINT(40.51925821971195 73.22194702979854) bank1726 +1727 POINT(39.75539371755099 73.12437556349187) bank1727 +1728 POINT(41.094008545751166 73.38881362819949) bank1728 +1729 POINT(41.163347384982984 74.00246599784408) bank1729 +1730 POINT(39.91648512368056 74.58607139979347) bank1730 +1731 POINT(39.72831045871067 74.70380265399895) bank1731 +1732 POINT(41.423723606088764 73.42565035795266) bank1732 +1733 POINT(41.2291472795628 73.11030989805603) bank1733 +1734 POINT(40.54050294689971 73.20567171117021) bank1734 +1735 POINT(41.41101481312092 73.08113018119985) bank1735 +1736 POINT(40.487979103100855 74.51246822230657) bank1736 +1737 POINT(40.5615850931614 73.44616439272205) bank1737 +1738 POINT(41.35276350303859 73.09946555105326) bank1738 +1739 POINT(41.24279823545119 73.42531746272213) bank1739 +1740 POINT(40.80533907899765 74.4713412078005) bank1740 +1741 POINT(40.88421957534037 73.50544418935866) bank1741 +1742 POINT(39.81177000699319 74.65952337198468) bank1742 +1743 POINT(40.38687764696723 74.85808352409497) bank1743 +1744 POINT(39.90403345751615 74.6695345795896) bank1744 +1745 POINT(39.783219283610364 73.30069793782341) bank1745 +1746 POINT(40.67921077898046 74.46271232482376) bank1746 +1747 POINT(40.86253876626092 74.44437143635044) bank1747 +1748 POINT(41.02541203858715 74.0829172546683) bank1748 +1749 POINT(40.29455438878677 74.51317983536457) bank1749 +1750 POINT(40.11053340157854 73.88542024819876) bank1750 +1751 POINT(40.89957932003304 74.70340418819968) bank1751 +1752 POINT(40.76616998143407 73.59414155519653) bank1752 +1753 POINT(40.18009702564621 74.56684775604391) bank1753 +1754 POINT(39.957208817866686 74.70379631308292) bank1754 +1755 POINT(41.55255089358176 74.19582582600444) bank1755 +1756 POINT(41.58256087246046 73.71549282491914) bank1756 +1757 POINT(40.35956216443656 73.0134208584296) bank1757 +1758 POINT(40.9750133021695 73.92802034547039) bank1758 +1759 POINT(40.26892290407716 74.54380198567735) bank1759 +1760 POINT(40.475377890042495 74.04545371927293) bank1760 +1761 POINT(39.90810099354363 74.46769119142311) bank1761 +1762 POINT(40.46243471547583 73.53833049933117) bank1762 +1763 POINT(40.160346473015906 73.90579394810733) bank1763 +1764 POINT(41.35667664360713 74.63606124256424) bank1764 +1765 POINT(41.637868596147705 73.44026671226872) bank1765 +1766 POINT(40.73774986049494 74.87063989820197) bank1766 +1767 POINT(40.17132335452705 74.2926950705961) bank1767 +1768 POINT(41.23431688379612 73.96436291326157) bank1768 +1769 POINT(39.870745668308764 74.55513335511193) bank1769 +1770 POINT(40.08825241334012 74.89778482359044) bank1770 +1771 POINT(41.42619933708739 73.47286204362432) bank1771 +1772 POINT(41.63015569329055 74.37861904125167) bank1772 +1773 POINT(41.38608836983022 74.48269110139277) bank1773 +1774 POINT(40.5548118269683 74.88844474099639) bank1774 +1775 POINT(40.841216312522576 73.7391361988312) bank1775 +1776 POINT(41.097819074045475 73.27388084560079) bank1776 +1777 POINT(40.393901690417664 74.93663488055601) bank1777 +1778 POINT(41.32276344628113 74.25011278400648) bank1778 +1779 POINT(40.95772076832528 73.20527582361332) bank1779 +1780 POINT(41.423699462690905 73.22288813506492) bank1780 +1781 POINT(41.53985464770733 74.65652530548127) bank1781 +1782 POINT(41.159674817639605 73.4941492885757) bank1782 +1783 POINT(41.53367665328024 74.90154626922778) bank1783 +1784 POINT(41.34208024926479 74.56611213216468) bank1784 +1785 POINT(39.78417363598798 73.21002128624154) bank1785 +1786 POINT(41.677340660499226 74.04687060571186) bank1786 +1787 POINT(41.08366543926623 74.88929553119141) bank1787 +1788 POINT(41.52697122165463 74.46098787127443) bank1788 +1789 POINT(40.22389387008085 73.1339724956536) bank1789 +1790 POINT(40.13099604560093 73.31410913193474) bank1790 +1791 POINT(40.05410788235208 73.2226923869859) bank1791 +1792 POINT(40.76027555153523 74.63942574315696) bank1792 +1793 POINT(39.857142107993596 73.69333503916371) bank1793 +1794 POINT(40.262867141582504 73.76665969635775) bank1794 +1795 POINT(39.753940723249045 73.46523174828816) bank1795 +1796 POINT(40.02989170121656 73.22539922734964) bank1796 +1797 POINT(41.38226438398643 74.37089935987342) bank1797 +1798 POINT(39.91904294479553 74.09086174315867) bank1798 +1799 POINT(41.169779641665734 73.27628810139282) bank1799 +1800 POINT(41.49717303496023 74.78907867547004) bank1800 +1801 POINT(40.19132542883946 73.61096666844246) bank1801 +1802 POINT(40.61805961293502 73.45336596508616) bank1802 +1803 POINT(41.29629930273038 73.46531273170584) bank1803 +1804 POINT(40.67102181603758 73.73315620155886) bank1804 +1805 POINT(41.61684559871262 73.50406033770919) bank1805 +1806 POINT(40.24572160232708 74.17532715422901) bank1806 +1807 POINT(40.99777034290117 74.0505058994242) bank1807 +1808 POINT(41.00344767615928 74.40640927082929) bank1808 +1809 POINT(41.33537944221359 74.91381569465652) bank1809 +1810 POINT(41.379290107047616 74.60441355874359) bank1810 +1811 POINT(39.98358657700324 74.4256436055964) bank1811 +1812 POINT(40.73556427314539 73.26075117302794) bank1812 +1813 POINT(41.40183972586345 74.3355515087568) bank1813 +1814 POINT(40.1484641861622 74.61812206951963) bank1814 +1815 POINT(39.94998189834484 73.41014048629326) bank1815 +1816 POINT(40.94595190867504 73.83631383164564) bank1816 +1817 POINT(40.92738400843389 73.98099323937808) bank1817 +1818 POINT(41.60543285929302 73.9664248199954) bank1818 +1819 POINT(39.8800154363136 73.13794446088505) bank1819 +1820 POINT(40.09944332837922 74.44892115469848) bank1820 +1821 POINT(41.0740541076089 73.43729863792808) bank1821 +1822 POINT(39.97061201634833 73.57251331302805) bank1822 +1823 POINT(40.70990893979555 73.37840000135093) bank1823 +1824 POINT(41.533629314672396 74.74502559044949) bank1824 +1825 POINT(41.20387858277081 73.43692372502939) bank1825 +1826 POINT(40.03835302540325 73.72776965152772) bank1826 +1827 POINT(40.42341631315987 74.30520536798467) bank1827 +1828 POINT(41.161404930201854 73.44447527481648) bank1828 +1829 POINT(40.11208062135901 74.74948945184106) bank1829 +1830 POINT(41.50960798957538 74.79254194469948) bank1830 +1831 POINT(40.22305336520278 74.07158353818201) bank1831 +1832 POINT(40.41987025950227 73.44197249466862) bank1832 +1833 POINT(40.086423714244276 74.35285447772034) bank1833 +1834 POINT(40.01788987800559 73.80562046905732) bank1834 +1835 POINT(41.50330086773868 73.48238874044557) bank1835 +1836 POINT(41.03501371955352 74.66523357298354) bank1836 +1837 POINT(39.76415815223088 73.82094212991721) bank1837 +1838 POINT(41.04572756070819 74.18392671013984) bank1838 +1839 POINT(39.94929048279203 73.63728572413754) bank1839 +1840 POINT(41.20234698438816 73.0226511666831) bank1840 +1841 POINT(41.538441947638695 74.47867425370335) bank1841 +1842 POINT(40.210122805702824 74.50892946326312) bank1842 +1843 POINT(40.34492526415771 74.89298290595225) bank1843 +1844 POINT(39.88190343731578 74.30173113208187) bank1844 +1845 POINT(40.80674038760286 74.86656396240126) bank1845 +1846 POINT(40.229230267580995 73.31233939273051) bank1846 +1847 POINT(40.17918359109798 74.45499435428692) bank1847 +1848 POINT(40.8232189804908 73.76564448008209) bank1848 +1849 POINT(40.84332310469476 74.47713823345453) bank1849 +1850 POINT(41.4203076801259 74.42244270390233) bank1850 +1851 POINT(41.43195296595479 74.5366571052444) bank1851 +1852 POINT(40.73799697837059 74.39226215323414) bank1852 +1853 POINT(39.99246785784597 73.99254081759388) bank1853 +1854 POINT(41.663372595098366 74.70890606185854) bank1854 +1855 POINT(40.80795761633545 73.95308419211561) bank1855 +1856 POINT(40.62731545882326 74.81665576847567) bank1856 +1857 POINT(39.8172615660918 74.54891191445458) bank1857 +1858 POINT(41.445245142531924 73.10868072353604) bank1858 +1859 POINT(40.84000365369934 73.2650731655124) bank1859 +1860 POINT(40.30280031524366 73.83829110968922) bank1860 +1861 POINT(40.53340215440729 73.19910785896862) bank1861 +1862 POINT(41.10024743045167 74.12611067143014) bank1862 +1863 POINT(41.100889963753204 74.6326133178651) bank1863 +1864 POINT(40.48457301603875 74.87697858085191) bank1864 +1865 POINT(40.73930715852884 74.47831560463133) bank1865 +1866 POINT(40.29833894784987 74.41314442816095) bank1866 +1867 POINT(40.06429531187331 74.80724704756445) bank1867 +1868 POINT(40.0950261040363 74.04077789255248) bank1868 +1869 POINT(41.499119156689666 74.37040006842756) bank1869 +1870 POINT(41.64531478574287 73.3827737184756) bank1870 +1871 POINT(40.77895063261163 74.66935213799152) bank1871 +1872 POINT(39.86337683030349 74.91139019744607) bank1872 +1873 POINT(40.243059543380426 74.75055846315594) bank1873 +1874 POINT(41.53797499822466 73.36573156205117) bank1874 +1875 POINT(40.99589422253617 74.91810767861317) bank1875 +1876 POINT(41.10208839260138 73.03550657425274) bank1876 +1877 POINT(40.89430498021297 74.87177008866308) bank1877 +1878 POINT(40.244746736875335 74.81236382104971) bank1878 +1879 POINT(40.104066246880585 74.11482530003995) bank1879 +1880 POINT(41.58809438861611 73.91530484299004) bank1880 +1881 POINT(40.04476431592113 74.35726854909248) bank1881 +1882 POINT(41.63961926578767 73.21377946156328) bank1882 +1883 POINT(41.16205885692402 74.08941230267517) bank1883 +1884 POINT(40.20227819104255 73.1059419346673) bank1884 +1885 POINT(40.176139226469445 74.14412878603143) bank1885 +1886 POINT(40.67272137525198 73.17921092765192) bank1886 +1887 POINT(40.3198415679168 74.9122831580226) bank1887 +1888 POINT(40.5928713389492 74.59371806137173) bank1888 +1889 POINT(39.79867308129503 74.74140169347564) bank1889 +1890 POINT(41.377066818041946 74.26499858127595) bank1890 +1891 POINT(40.603105026729885 74.7038987452396) bank1891 +1892 POINT(40.12815480390034 73.86285170157105) bank1892 +1893 POINT(40.77093361933828 74.67671286888742) bank1893 +1894 POINT(40.79592275538716 74.05306808522417) bank1894 +1895 POINT(41.590606129552626 73.52372455808796) bank1895 +1896 POINT(40.69383628545483 74.4202186298952) bank1896 +1897 POINT(39.92335914956365 73.33190322055597) bank1897 +1898 POINT(39.86612634159527 74.45888642345388) bank1898 +1899 POINT(41.41041503532232 73.7931157968181) bank1899 +1900 POINT(40.257869623317454 74.9959639261864) bank1900 +1901 POINT(41.69901315482249 74.94643729306782) bank1901 +1902 POINT(39.91426573599166 73.30389706062742) bank1902 +1903 POINT(41.51725965069943 73.65436202302737) bank1903 +1904 POINT(41.44370046041067 73.72824097654765) bank1904 +1905 POINT(41.53964285067887 74.47072520550812) bank1905 +1906 POINT(40.77407781226213 74.7381835025696) bank1906 +1907 POINT(40.624203946247306 74.35887798776803) bank1907 +1908 POINT(41.094297900965195 73.74474399102627) bank1908 +1909 POINT(41.68200787759428 74.1183419729616) bank1909 +1910 POINT(40.50780477643323 74.10921919527188) bank1910 +1911 POINT(41.51356293739145 74.63111482585839) bank1911 +1912 POINT(40.69768527886276 73.72068067067681) bank1912 +1913 POINT(41.519242843145946 74.1663601727788) bank1913 +1914 POINT(39.94242147347866 74.68753949656251) bank1914 +1915 POINT(40.067476759056106 73.0795728217815) bank1915 +1916 POINT(41.415336893511245 73.98163856885685) bank1916 +1917 POINT(40.933236257927135 74.8181850627016) bank1917 +1918 POINT(40.35371145218628 73.1776402585926) bank1918 +1919 POINT(40.16349819140367 73.14289406146233) bank1919 +1920 POINT(41.07563875652223 74.42961831320682) bank1920 +1921 POINT(40.49275647989814 73.08004736774619) bank1921 +1922 POINT(40.93083278577787 73.90268052343896) bank1922 +1923 POINT(39.86733932022501 74.76976349208576) bank1923 +1924 POINT(40.78245422025899 74.38196552931484) bank1924 +1925 POINT(39.99336021001977 74.46879649014785) bank1925 +1926 POINT(41.50361964264655 74.33231260163576) bank1926 +1927 POINT(41.16439791799027 74.24285182912766) bank1927 +1928 POINT(40.661903313886214 74.87032886988979) bank1928 +1929 POINT(41.31411044159729 74.92495596143416) bank1929 +1930 POINT(40.94352305294247 73.54712874511594) bank1930 +1931 POINT(41.402466604433265 73.13927965140881) bank1931 +1932 POINT(40.500365157914125 73.29271216347608) bank1932 +1933 POINT(40.35709868504149 73.8075275317515) bank1933 +1934 POINT(40.335680069126575 73.28043131290313) bank1934 +1935 POINT(39.71594376012764 73.88985027050717) bank1935 +1936 POINT(41.562481185847645 73.54347006743258) bank1936 +1937 POINT(41.32387583379603 74.56078833939306) bank1937 +1938 POINT(40.77554203959594 74.3106844665189) bank1938 +1939 POINT(41.681362167312905 74.6382120823675) bank1939 +1940 POINT(40.594442310845736 74.90210563521566) bank1940 +1941 POINT(41.20702353992201 73.79164667622953) bank1941 +1942 POINT(39.892504667282914 73.96579506260498) bank1942 +1943 POINT(41.32169260478924 73.75996088145763) bank1943 +1944 POINT(40.74775507572887 74.26717283397645) bank1944 +1945 POINT(40.30108054466296 73.60712799650894) bank1945 +1946 POINT(41.51649622716152 74.6029170776397) bank1946 +1947 POINT(40.5992225842531 74.7677531789591) bank1947 +1948 POINT(40.45744692882717 74.66957129090483) bank1948 +1949 POINT(41.6859291284765 73.94943480022135) bank1949 +1950 POINT(40.69551798831059 73.71206379393004) bank1950 +1951 POINT(39.999926693487865 73.30171490024671) bank1951 +1952 POINT(40.09918722155109 73.72960348711973) bank1952 +1953 POINT(39.97555841385397 73.19597587225272) bank1953 +1954 POINT(40.82870795105097 73.49189142836605) bank1954 +1955 POINT(39.995740473992925 74.39655270364351) bank1955 +1956 POINT(39.99729947604728 73.91755079094398) bank1956 +1957 POINT(40.007956447256596 74.9451104027188) bank1957 +1958 POINT(40.729160132136265 74.00206011391818) bank1958 +1959 POINT(41.184773551209155 73.75635425533507) bank1959 +1960 POINT(39.951502345589354 73.2608030445071) bank1960 +1961 POINT(40.86666040401091 73.00642719095315) bank1961 +1962 POINT(40.930561256281315 74.14433471789475) bank1962 +1963 POINT(40.261663306891855 73.65238933999615) bank1963 +1964 POINT(41.08212174829928 74.19220588183418) bank1964 +1965 POINT(40.53713588616013 73.3986473279286) bank1965 +1966 POINT(41.58715882254434 73.12217822762496) bank1966 +1967 POINT(41.326423654739244 74.55924185068672) bank1967 +1968 POINT(40.76922847740127 73.93750045344883) bank1968 +1969 POINT(40.1606350332722 73.73959284957886) bank1969 +1970 POINT(40.030694084226 73.22116848596752) bank1970 +1971 POINT(41.66582963547265 74.9335194173888) bank1971 +1972 POINT(40.589248483952446 73.25230784746199) bank1972 +1973 POINT(40.30272809376779 73.84234735117748) bank1973 +1974 POINT(41.16174128325713 73.42974095492785) bank1974 +1975 POINT(40.06396164721795 74.96137074005755) bank1975 +1976 POINT(40.290582246622066 73.97043847760122) bank1976 +1977 POINT(40.34137085320762 73.03016946764747) bank1977 +1978 POINT(40.83385849639163 74.86774313261684) bank1978 +1979 POINT(40.33235589803389 73.96134298698385) bank1979 +1980 POINT(41.575726265754355 74.1472173174464) bank1980 +1981 POINT(41.35366338259894 73.14413318538944) bank1981 +1982 POINT(39.795940795296964 73.02133409406635) bank1982 +1983 POINT(41.231204855228285 73.15414352185276) bank1983 +1984 POINT(40.058223649762255 73.83499658555385) bank1984 +1985 POINT(41.27507399327573 73.6086855726553) bank1985 +1986 POINT(40.000359351980144 74.36480710660919) bank1986 +1987 POINT(41.693783521008264 74.89394029374203) bank1987 +1988 POINT(41.68984007779222 73.44007831142049) bank1988 +1989 POINT(40.07674759612166 73.13741561493585) bank1989 +1990 POINT(40.00111941640351 74.80958028304613) bank1990 +1991 POINT(40.932671045416036 73.68925305114561) bank1991 +1992 POINT(39.71555985983818 74.11066217525075) bank1992 +1993 POINT(40.00456294626949 73.52731099691476) bank1993 +1994 POINT(40.7572514395109 74.22562593815411) bank1994 +1995 POINT(41.234427265050904 73.90471002065584) bank1995 +1996 POINT(41.1549706602569 74.42653079388664) bank1996 +1997 POINT(40.343821437891634 74.78215466249497) bank1997 +1998 POINT(40.16731319174257 73.76206877220741) bank1998 +1999 POINT(39.838096399785464 73.85798374195875) bank1999 +2000 POINT(40.82165502518133 74.35416447973634) bank2000 +2001 POINT(41.19333251539148 73.4002902608249) bank2001 +2002 POINT(39.71856302459307 74.46335391042716) bank2002 +2003 POINT(40.001125574544425 73.13341996420225) bank2003 +2004 POINT(41.661999093208046 73.12900429678871) bank2004 +2005 POINT(41.12023316168304 73.45511410067283) bank2005 +2006 POINT(40.47223651582333 73.03950875484816) bank2006 +2007 POINT(40.34650845164486 73.8009763618074) bank2007 +2008 POINT(41.01146237832626 73.9914211040178) bank2008 +2009 POINT(41.03270059696428 74.761428055857) bank2009 +2010 POINT(40.79224650003632 73.21805295069326) bank2010 +2011 POINT(40.79521909322862 74.36505510453217) bank2011 +2012 POINT(40.85829556430288 73.00791679369819) bank2012 +2013 POINT(41.41681030726597 74.14051796697362) bank2013 +2014 POINT(40.43556224693882 73.46218059375585) bank2014 +2015 POINT(39.79982950294424 73.72094755718123) bank2015 +2016 POINT(40.97084702560472 74.34964163846475) bank2016 +2017 POINT(39.83189862207415 74.5598462885273) bank2017 +2018 POINT(40.799872091351496 74.96245547350087) bank2018 +2019 POINT(40.53202292411058 73.7676353407423) bank2019 +2020 POINT(40.31156736160817 74.23241754387102) bank2020 +2021 POINT(41.48898832379032 75.00421550006718) bank2021 +2022 POINT(40.93958247174601 74.35865488418999) bank2022 +2023 POINT(40.40583507375042 73.25175265920167) bank2023 +2024 POINT(41.329537507563096 73.81861222029093) bank2024 +2025 POINT(40.867857154866876 74.34266121273392) bank2025 +2026 POINT(41.070190051791776 74.19567130064459) bank2026 +2027 POINT(40.64464637486792 73.58358187861712) bank2027 +2028 POINT(40.106570086745336 73.5248933639008) bank2028 +2029 POINT(40.10294645271482 74.55385066663517) bank2029 +2030 POINT(40.614126095395065 73.52332425647657) bank2030 +2031 POINT(39.81620349007351 73.16823295918978) bank2031 +2032 POINT(39.887927636638366 74.5053378290072) bank2032 +2033 POINT(39.858847091002936 74.34299213209474) bank2033 +2034 POINT(39.82540755541065 73.45881454018892) bank2034 +2035 POINT(40.81648338750529 73.60077982261839) bank2035 +2036 POINT(40.86069647153826 73.45597160892682) bank2036 +2037 POINT(40.11975637714296 73.5801289726895) bank2037 +2038 POINT(40.760506391555225 74.37800140149189) bank2038 +2039 POINT(41.377871546017474 73.96756208518572) bank2039 +2040 POINT(41.07410660085886 73.95808379513248) bank2040 +2041 POINT(40.67006933966391 74.23441174618844) bank2041 +2042 POINT(41.50950677782514 74.81374637936148) bank2042 +2043 POINT(39.805133916833995 73.3037556116849) bank2043 +2044 POINT(40.62187871572424 74.00788007855259) bank2044 +2045 POINT(40.70009252443846 73.69760986180042) bank2045 +2046 POINT(40.49747689233794 73.83444210792015) bank2046 +2047 POINT(40.92978464543233 73.71034116810833) bank2047 +2048 POINT(40.02697004021286 74.72289768551703) bank2048 +2049 POINT(41.48194907066278 74.32739035592938) bank2049 +2050 POINT(40.51271905215928 74.35128553196631) bank2050 +2051 POINT(40.859896561130434 73.52807344696384) bank2051 +2052 POINT(40.21792827206095 74.32991384319183) bank2052 +2053 POINT(41.39411673613025 74.75247025538899) bank2053 +2054 POINT(40.467678301893514 73.6641668628516) bank2054 +2055 POINT(41.19982173949325 73.6576204990064) bank2055 +2056 POINT(39.73858634322882 74.60335789091651) bank2056 +2057 POINT(40.545964550166175 74.17737599733917) bank2057 +2058 POINT(39.87762069315606 74.38558791902882) bank2058 +2059 POINT(41.32834790372643 74.51001082727555) bank2059 +2060 POINT(40.48980621857294 73.83122340736598) bank2060 +2061 POINT(40.00905192110525 73.17552329852961) bank2061 +2062 POINT(39.78323223751993 73.80837665160028) bank2062 +2063 POINT(39.89029285202383 73.39736570356416) bank2063 +2064 POINT(41.2071982591113 74.54863802735876) bank2064 +2065 POINT(41.65760626693112 73.12045218341964) bank2065 +2066 POINT(40.26398724372987 74.27814302117802) bank2066 +2067 POINT(40.857690612928934 74.35275019078155) bank2067 +2068 POINT(40.20650910980867 73.77018847302378) bank2068 +2069 POINT(40.983138546420996 74.86349346672864) bank2069 +2070 POINT(41.579284194095976 73.22657055329309) bank2070 +2071 POINT(41.061142690984546 74.04449336243668) bank2071 +2072 POINT(40.90679031901496 74.26335005026557) bank2072 +2073 POINT(41.55388948261445 73.68838877798446) bank2073 +2074 POINT(40.69231997845161 74.57012221806654) bank2074 +2075 POINT(41.27387372278096 74.57029060037254) bank2075 +2076 POINT(39.89523094151986 73.4848754867543) bank2076 +2077 POINT(39.92749845868835 74.81552016981328) bank2077 +2078 POINT(40.462680169142914 74.21667902111355) bank2078 +2079 POINT(41.055687780144915 73.73785880339292) bank2079 +2080 POINT(40.87154910980561 73.82259585943874) bank2080 +2081 POINT(40.979048653979575 74.41934461802693) bank2081 +2082 POINT(40.1165303586767 73.68421833138038) bank2082 +2083 POINT(41.327558951065996 73.8994693347583) bank2083 +2084 POINT(41.16954535152458 74.0849701914673) bank2084 +2085 POINT(40.747153824063325 73.17765808252442) bank2085 +2086 POINT(40.23053569295654 74.8616235024532) bank2086 +2087 POINT(40.783212498861985 73.52445569544568) bank2087 +2088 POINT(40.2570455481258 74.64214493957634) bank2088 +2089 POINT(40.85560887220255 73.55197784806184) bank2089 +2090 POINT(39.91167492853901 74.9276850936884) bank2090 +2091 POINT(41.318496766038706 73.60462557283097) bank2091 +2092 POINT(40.79514916973957 74.64109722358378) bank2092 +2093 POINT(40.389524479043104 74.79003745101238) bank2093 +2094 POINT(40.803540162456706 73.58710450788237) bank2094 +2095 POINT(40.7738408276216 73.51360651347817) bank2095 +2096 POINT(39.97839697447771 73.21154700746952) bank2096 +2097 POINT(41.39300542099471 73.14753367947736) bank2097 +2098 POINT(41.53265228579836 73.30723264241082) bank2098 +2099 POINT(40.76091316289537 74.20054762866224) bank2099 +2100 POINT(40.2518881381209 73.596602343005) bank2100 +2101 POINT(40.636794129603764 73.72858967847097) bank2101 +2102 POINT(40.211157020106924 73.17960425219044) bank2102 +2103 POINT(41.14418804372438 73.35644299417915) bank2103 +2104 POINT(40.79130489012985 73.80089593208172) bank2104 +2105 POINT(40.830294321910145 73.36188146499336) bank2105 +2106 POINT(40.169661833394805 73.4071355849302) bank2106 +2107 POINT(41.52874065457194 73.78852233955041) bank2107 +2108 POINT(40.1497666324507 73.0904992353603) bank2108 +2109 POINT(40.79909903049066 74.58765308382971) bank2109 +2110 POINT(41.038309476236186 74.90302451410497) bank2110 +2111 POINT(40.81687966205241 73.47959190399585) bank2111 +2112 POINT(40.66818096572407 73.56536193722162) bank2112 +2113 POINT(40.53710220317064 74.93671898213452) bank2113 +2114 POINT(41.534567015610655 74.5624525296081) bank2114 +2115 POINT(41.514712530577015 74.18190191723437) bank2115 +2116 POINT(40.96631742781094 73.07868153349473) bank2116 +2117 POINT(40.5390359835602 74.64219748754282) bank2117 +2118 POINT(40.792826020278454 74.65907478006356) bank2118 +2119 POINT(41.0652521902778 74.75901131705665) bank2119 +2120 POINT(39.91399983517865 73.62716781230736) bank2120 +2121 POINT(41.615858452228395 74.22186438850906) bank2121 +2122 POINT(41.02853823815888 73.78326539078601) bank2122 +2123 POINT(39.90408779715559 73.19775917156828) bank2123 +2124 POINT(40.5310872392075 73.50309551253471) bank2124 +2125 POINT(40.47046057934911 74.1691616242257) bank2125 +2126 POINT(39.947390780689794 74.73928226996233) bank2126 +2127 POINT(41.28075510416886 74.44364305368886) bank2127 +2128 POINT(41.7030169341021 73.99062214317313) bank2128 +2129 POINT(39.90515477104886 73.20249940651826) bank2129 +2130 POINT(40.62277571625181 74.4952042187261) bank2130 +2131 POINT(41.12913982819508 73.35732283063325) bank2131 +2132 POINT(40.295614730432796 73.23369078097087) bank2132 +2133 POINT(41.639620776966254 74.16065945221469) bank2133 +2134 POINT(39.879870524122325 74.58670226575315) bank2134 +2135 POINT(40.3054156650329 73.9627241163011) bank2135 +2136 POINT(41.61497679741251 73.03096296584769) bank2136 +2137 POINT(40.72516733259081 74.90211707888321) bank2137 +2138 POINT(39.752694461295675 74.38872642701867) bank2138 +2139 POINT(39.955044283737735 74.61345361962017) bank2139 +2140 POINT(41.58247698101198 74.15105387861719) bank2140 +2141 POINT(39.802899110555224 74.43347564644864) bank2141 +2142 POINT(41.25242621218668 73.68956795144581) bank2142 +2143 POINT(40.25470175297862 73.32068952234417) bank2143 +2144 POINT(40.4479786769549 74.5684328704737) bank2144 +2145 POINT(41.644220445153174 73.91210076616343) bank2145 +2146 POINT(41.60745824783325 74.05682070523788) bank2146 +2147 POINT(40.13178136028974 74.84154232962955) bank2147 +2148 POINT(39.72677633588684 73.81061074595335) bank2148 +2149 POINT(41.44096842799114 73.59896200795606) bank2149 +2150 POINT(40.14581064086454 74.96900067721135) bank2150 +2151 POINT(40.55136020944818 74.81168688897544) bank2151 +2152 POINT(41.613231994858694 74.47867095884332) bank2152 +2153 POINT(40.60006820373728 73.11180160253869) bank2153 +2154 POINT(40.28186342417454 73.45737138839888) bank2154 +2155 POINT(40.43761021854762 74.97384189104652) bank2155 +2156 POINT(41.446522370384734 73.35164394684217) bank2156 +2157 POINT(40.64571288426958 74.79584915238341) bank2157 +2158 POINT(40.90584674546641 73.54202193756359) bank2158 +2159 POINT(41.089834688383874 74.56359421845471) bank2159 +2160 POINT(41.56511326802489 74.43522449181447) bank2160 +2161 POINT(41.3831635770933 74.80013494610589) bank2161 +2162 POINT(41.359611855975224 75.00550134291943) bank2162 +2163 POINT(40.6907535279287 73.29842865043334) bank2163 +2164 POINT(39.89149376145669 74.17422910561241) bank2164 +2165 POINT(39.982609963171946 74.98360610715041) bank2165 +2166 POINT(39.95083718418581 73.81347409657793) bank2166 +2167 POINT(40.16004743782261 73.2628859190087) bank2167 +2168 POINT(39.88803598983129 73.73422382648171) bank2168 +2169 POINT(40.863655043653786 73.56100270584916) bank2169 +2170 POINT(39.87340466470554 74.50022655002758) bank2170 +2171 POINT(40.39628911335589 74.39623322107126) bank2171 +2172 POINT(39.96315098904507 73.90947279488086) bank2172 +2173 POINT(40.226494529175035 74.26905986236818) bank2173 +2174 POINT(40.94034572079807 73.04784491219077) bank2174 +2175 POINT(40.40445218782671 74.84257875483219) bank2175 +2176 POINT(41.288795575977566 74.95269603309913) bank2176 +2177 POINT(39.78502385739806 74.23653130114975) bank2177 +2178 POINT(39.84861987040523 73.07408463672287) bank2178 +2179 POINT(40.32291645987543 74.33716229396168) bank2179 +2180 POINT(40.60730852649454 74.51930740455495) bank2180 +2181 POINT(40.5483490615547 74.69323485752744) bank2181 +2182 POINT(40.181092220529806 73.47904254816086) bank2182 +2183 POINT(41.59928983595209 73.12824686061916) bank2183 +2184 POINT(39.84250340626561 73.13210291140841) bank2184 +2185 POINT(41.51362363207693 73.44033607816115) bank2185 +2186 POINT(40.54047670106342 73.3670303157181) bank2186 +2187 POINT(41.67273982353487 74.82433991816626) bank2187 +2188 POINT(41.65287244199204 74.83241243013201) bank2188 +2189 POINT(41.69450619816381 74.67851314440533) bank2189 +2190 POINT(41.07757227416851 73.69115702062432) bank2190 +2191 POINT(40.055589871157906 74.63346737764442) bank2191 +2192 POINT(40.72900798993778 73.80236967973804) bank2192 +2193 POINT(40.76468601617639 74.20281286531824) bank2193 +2194 POINT(41.16601455430923 73.77287785644292) bank2194 +2195 POINT(39.722858646956105 73.48109392030369) bank2195 +2196 POINT(41.08371173267835 74.76566128026194) bank2196 +2197 POINT(41.48997856591699 74.0664895955678) bank2197 +2198 POINT(40.83552949345751 73.281717520183) bank2198 +2199 POINT(40.263755765379855 74.52137465485698) bank2199 +2200 POINT(40.83958672721859 73.09042138128191) bank2200 +2201 POINT(40.57944985312575 74.0809549082467) bank2201 +2202 POINT(41.378513195301814 73.30351692976257) bank2202 +2203 POINT(40.17363661429184 73.93046529895078) bank2203 +2204 POINT(40.95368565442208 73.14558578734848) bank2204 +2205 POINT(40.367492060273484 73.8014687726361) bank2205 +2206 POINT(40.844743773572795 74.07513521336774) bank2206 +2207 POINT(39.75377787515399 73.13283343991046) bank2207 +2208 POINT(41.626055316587845 73.94470460145449) bank2208 +2209 POINT(40.02066902431059 73.89136230238354) bank2209 +2210 POINT(39.906710760141415 73.56132378718792) bank2210 +2211 POINT(40.197926177702676 73.6722736756863) bank2211 +2212 POINT(40.51939532872095 73.7238769236884) bank2212 +2213 POINT(40.026210612038355 73.78946171224355) bank2213 +2214 POINT(40.44453001525585 73.13193155313206) bank2214 +2215 POINT(40.27274246012994 74.5546075954704) bank2215 +2216 POINT(40.72411805526508 74.4224435394809) bank2216 +2217 POINT(40.655519463148536 73.4907484076416) bank2217 +2218 POINT(40.58491459376542 74.52694191546267) bank2218 +2219 POINT(40.47854721117666 73.72504319121462) bank2219 +2220 POINT(41.592647335931154 73.91281757374166) bank2220 +2221 POINT(40.68150232216997 73.80344662849446) bank2221 +2222 POINT(41.59693885198141 74.25712503342237) bank2222 +2223 POINT(40.924268934295085 74.63831327831362) bank2223 +2224 POINT(40.58210510612743 74.18440038628587) bank2224 +2225 POINT(40.22678724148609 74.3214702423001) bank2225 +2226 POINT(40.963523285852766 73.2599637279622) bank2226 +2227 POINT(41.59564979611001 74.75523314336701) bank2227 +2228 POINT(40.264803219972144 74.95906420816507) bank2228 +2229 POINT(39.76709489862293 74.59231351984947) bank2229 +2230 POINT(39.84523779935292 74.88534795729338) bank2230 +2231 POINT(41.165898156042786 73.73094818002562) bank2231 +2232 POINT(41.692877915736844 74.45177213580075) bank2232 +2233 POINT(40.63474711415908 74.60290930435701) bank2233 +2234 POINT(41.11649688242495 73.83620538602632) bank2234 +2235 POINT(39.775943471312644 73.3370908170886) bank2235 +2236 POINT(39.897431966437956 74.98472154284663) bank2236 +2237 POINT(40.642752159575835 73.67393961830189) bank2237 +2238 POINT(40.12712159038243 73.01426983858885) bank2238 +2239 POINT(39.843853373786295 73.90255633487011) bank2239 +2240 POINT(39.796007069082485 74.53051951056486) bank2240 +2241 POINT(41.596043866757746 74.57701413892124) bank2241 +2242 POINT(41.07905274425374 73.49118162017035) bank2242 +2243 POINT(39.79106666287815 74.4980220444546) bank2243 +2244 POINT(40.43977101486713 74.58278417197026) bank2244 +2245 POINT(40.42113973870703 73.66730872986595) bank2245 +2246 POINT(39.96680664182908 73.5276031126983) bank2246 +2247 POINT(39.95619351688326 73.05773748071871) bank2247 +2248 POINT(39.95381133939467 74.17299496352933) bank2248 +2249 POINT(40.80352266357401 74.50058169101257) bank2249 +2250 POINT(41.33073268537361 73.0649081997976) bank2250 +2251 POINT(40.05496338754764 73.38338596392501) bank2251 +2252 POINT(41.345858636834414 74.85254326736018) bank2252 +2253 POINT(40.57950310823063 73.22523903775506) bank2253 +2254 POINT(40.75980843060165 74.05765531894693) bank2254 +2255 POINT(41.084440763153765 73.54675683637119) bank2255 +2256 POINT(40.20972252820067 74.38184952630289) bank2256 +2257 POINT(40.82253989915162 73.52326277708548) bank2257 +2258 POINT(40.77539574382101 73.45589581913384) bank2258 +2259 POINT(41.114645364963316 73.07508805787167) bank2259 +2260 POINT(41.21866624346494 74.45270338314685) bank2260 +2261 POINT(40.4730602627384 74.516367204213) bank2261 +2262 POINT(40.08830816973781 73.81158029921188) bank2262 +2263 POINT(41.15610474307745 74.4788335039189) bank2263 +2264 POINT(40.33937580749822 74.98877359969795) bank2264 +2265 POINT(40.000207981696775 74.36232593097007) bank2265 +2266 POINT(40.37135940163601 73.53162975312436) bank2266 +2267 POINT(40.02918590505893 73.87515688465244) bank2267 +2268 POINT(40.29631646354852 74.61887204069168) bank2268 +2269 POINT(39.98763918778331 74.54981111292642) bank2269 +2270 POINT(40.39810984397836 73.25765176124919) bank2270 +2271 POINT(40.58445281217537 74.56772138558907) bank2271 +2272 POINT(41.529899578157895 74.11214973939062) bank2272 +2273 POINT(40.802974019657064 73.84271893691485) bank2273 +2274 POINT(40.13750711345052 74.01814542602017) bank2274 +2275 POINT(41.10352760899201 73.58152884878339) bank2275 +2276 POINT(41.64798962660972 73.09852578144879) bank2276 +2277 POINT(41.46453724471082 73.57707739166933) bank2277 +2278 POINT(40.73624438295038 73.231218503101) bank2278 +2279 POINT(41.34814666232225 73.24392606624791) bank2279 +2280 POINT(40.9509019773134 74.9642034872195) bank2280 +2281 POINT(40.637009410258386 74.39839125337427) bank2281 +2282 POINT(39.82009992465059 74.48292183692226) bank2282 +2283 POINT(41.06855472563155 74.74280366420021) bank2283 +2284 POINT(41.10809501694783 74.1315966895561) bank2284 +2285 POINT(41.15772248350375 74.87750828517954) bank2285 +2286 POINT(41.28337954105539 74.74164828136597) bank2286 +2287 POINT(41.23899179238178 73.57940034779534) bank2287 +2288 POINT(40.8554457957625 73.27345856554037) bank2288 +2289 POINT(41.62979916703791 74.17824174594186) bank2289 +2290 POINT(39.78783022674107 74.78377803906952) bank2290 +2291 POINT(40.34291221186628 73.2279478297506) bank2291 +2292 POINT(39.83459359428959 74.4715163686765) bank2292 +2293 POINT(39.86970467679757 73.59434360296747) bank2293 +2294 POINT(41.240862132809774 73.10526855036068) bank2294 +2295 POINT(40.025145599736604 74.63016743987212) bank2295 +2296 POINT(40.284625342946526 74.0763360327908) bank2296 +2297 POINT(41.03815801552933 73.43299300440592) bank2297 +2298 POINT(41.06559308570817 73.17589642923019) bank2298 +2299 POINT(41.632031529232556 74.19318201714054) bank2299 +2300 POINT(40.78413148686034 73.10211733858394) bank2300 +2301 POINT(41.399347190364395 74.27026486075923) bank2301 +2302 POINT(40.72099564439872 73.27158138330265) bank2302 +2303 POINT(41.20429580884976 74.59192471630104) bank2303 +2304 POINT(41.21558305211473 73.88621417550432) bank2304 +2305 POINT(41.235155705024596 73.66110744308044) bank2305 +2306 POINT(40.9390089680135 73.01977516558877) bank2306 +2307 POINT(41.23309562223881 73.19829439116515) bank2307 +2308 POINT(40.14396703314373 73.4776139174308) bank2308 +2309 POINT(39.8995137904757 74.04381476810008) bank2309 +2310 POINT(41.23627485115135 74.16021273923053) bank2310 +2311 POINT(40.971672208148995 74.92440860047152) bank2311 +2312 POINT(41.07719411758431 74.75774843050847) bank2312 +2313 POINT(40.589072727858024 74.1185867893949) bank2313 +2314 POINT(40.74902011961815 74.80128469948075) bank2314 +2315 POINT(40.1726519822658 74.83501932515418) bank2315 +2316 POINT(41.64833962434744 73.69877254213802) bank2316 +2317 POINT(40.99237731450463 73.38180753609606) bank2317 +2318 POINT(40.076505794492476 73.45579792112278) bank2318 +2319 POINT(40.8421223828039 73.13180979184865) bank2319 +2320 POINT(39.92812819668575 74.61241037495563) bank2320 +2321 POINT(40.1173914096507 74.20593626848205) bank2321 +2322 POINT(40.790426454309056 73.2028054616651) bank2322 +2323 POINT(39.77972125475487 73.47628008397315) bank2323 +2324 POINT(39.960959512296014 73.81683273071438) bank2324 +2325 POINT(40.40224742944107 74.71235661199991) bank2325 +2326 POINT(41.086781241967714 74.94181489201961) bank2326 +2327 POINT(40.49217780227243 74.09085600638521) bank2327 +2328 POINT(40.93039760560972 74.68875270023624) bank2328 +2329 POINT(40.2651961151413 74.11484979254938) bank2329 +2330 POINT(40.38336168905782 73.71851723192388) bank2330 +2331 POINT(40.56112467909749 73.8906484717972) bank2331 +2332 POINT(41.57503320516409 74.21041206143767) bank2332 +2333 POINT(41.46416726792841 74.32168378563775) bank2333 +2334 POINT(39.90493730854422 73.27791808400839) bank2334 +2335 POINT(40.57721975220988 74.63716397434052) bank2335 +2336 POINT(40.930863031519614 74.25629270430011) bank2336 +2337 POINT(40.811518268870735 74.02959636150264) bank2337 +2338 POINT(39.98915015791684 73.99765321466887) bank2338 +2339 POINT(41.416962693768355 74.01974439124561) bank2339 +2340 POINT(41.45676705794587 73.26749250321637) bank2340 +2341 POINT(40.00726610824032 73.98101134619327) bank2341 +2342 POINT(40.781759075787164 74.53002867833021) bank2342 +2343 POINT(41.07438863354996 74.5001857545166) bank2343 +2344 POINT(39.872668065914795 74.62993685012601) bank2344 +2345 POINT(40.739407419521385 73.68198648007144) bank2345 +2346 POINT(40.92530583588324 74.0620955912878) bank2346 +2347 POINT(41.64807635557735 74.26127537979995) bank2347 +2348 POINT(39.818995022731876 73.09197647868208) bank2348 +2349 POINT(40.70612124032317 73.65449430043333) bank2349 +2350 POINT(40.377005999583375 74.32487911119422) bank2350 +2351 POINT(40.80064994969199 74.6344367040037) bank2351 +2352 POINT(41.10703288897661 74.69015888720824) bank2352 +2353 POINT(41.16964117007523 74.83970879796155) bank2353 +2354 POINT(41.104209007616454 73.99893270996833) bank2354 +2355 POINT(40.002545927428976 73.84457939672068) bank2355 +2356 POINT(40.433381277935446 73.34190887107408) bank2356 +2357 POINT(40.861593050734 73.32014785646398) bank2357 +2358 POINT(40.50067087089532 73.28609614920795) bank2358 +2359 POINT(41.6546973660959 74.35900523245859) bank2359 +2360 POINT(40.523529627104004 74.03849078032789) bank2360 +2361 POINT(40.840835423637635 74.07173295567834) bank2361 +2362 POINT(40.95249362039682 74.15154009872357) bank2362 +2363 POINT(40.361040274813256 73.21886788354239) bank2363 +2364 POINT(40.44246360860745 73.38766916663386) bank2364 +2365 POINT(40.09969833965681 74.88947582735042) bank2365 +2366 POINT(40.97943893127731 73.45262830190251) bank2366 +2367 POINT(40.719984832412266 73.1383188941684) bank2367 +2368 POINT(40.92713174914913 73.04088393808325) bank2368 +2369 POINT(40.48185461631388 74.3033468144711) bank2369 +2370 POINT(41.34367800196132 73.06443440208946) bank2370 +2371 POINT(40.86865558606836 74.6702486032058) bank2371 +2372 POINT(40.20733965109307 73.1017315423401) bank2372 +2373 POINT(41.706482008937954 74.89903092109556) bank2373 +2374 POINT(39.72596517644543 73.32093181858862) bank2374 +2375 POINT(41.39056419751434 74.7408644506512) bank2375 +2376 POINT(41.63340770940954 73.62232585613029) bank2376 +2377 POINT(40.96565446464103 74.88261348558818) bank2377 +2378 POINT(40.019548192741155 73.24401270426436) bank2378 +2379 POINT(41.32639685122151 73.36544577019285) bank2379 +2380 POINT(40.899225318213134 74.47322848119312) bank2380 +2381 POINT(40.380037020675864 74.30052178035298) bank2381 +2382 POINT(41.53891014067107 73.88365147953155) bank2382 +2383 POINT(41.302981536413746 74.19009456887498) bank2383 +2384 POINT(40.80235607422939 73.01169656307741) bank2384 +2385 POINT(40.64076253791715 73.14432741530788) bank2385 +2386 POINT(40.73169831627925 74.0518315277709) bank2386 +2387 POINT(41.40219588562984 74.1911283794032) bank2387 +2388 POINT(40.38131533046067 74.30305314834679) bank2388 +2389 POINT(40.622146528199046 74.11846730552787) bank2389 +2390 POINT(41.310400618295894 74.0402213972751) bank2390 +2391 POINT(41.65095127202838 73.60319512598994) bank2391 +2392 POINT(39.96034289867077 74.38145932322303) bank2392 +2393 POINT(40.5162463599388 73.39945776729054) bank2393 +2394 POINT(40.19163302509793 73.43571429645742) bank2394 +2395 POINT(40.613268526760756 73.10306800249052) bank2395 +2396 POINT(41.479169111473055 74.15697468727836) bank2396 +2397 POINT(41.47037504855316 74.28790914758537) bank2397 +2398 POINT(41.57270181952009 73.93088219360912) bank2398 +2399 POINT(41.576117486446876 74.7713800149248) bank2399 +2400 POINT(39.79590190604227 74.08836623936344) bank2400 +2401 POINT(40.71210078011585 73.833673114398) bank2401 +2402 POINT(40.80836908394264 73.84974488235655) bank2402 +2403 POINT(41.40502886034352 74.4600922832733) bank2403 +2404 POINT(40.85775167949874 74.0131669209702) bank2404 +2405 POINT(39.86452019861066 74.41330397906877) bank2405 +2406 POINT(40.9186783208343 73.14334397135941) bank2406 +2407 POINT(39.962040200225175 74.98324194374216) bank2407 +2408 POINT(41.17443798247873 73.13190960397218) bank2408 +2409 POINT(41.4032861102721 74.57667477708144) bank2409 +2410 POINT(41.32666680802098 73.54502000716613) bank2410 +2411 POINT(40.924359695202284 74.10686560634832) bank2411 +2412 POINT(41.55599903897754 73.61936063039214) bank2412 +2413 POINT(41.55891302991537 74.75591276569507) bank2413 +2414 POINT(40.90053111713283 74.0088447745543) bank2414 +2415 POINT(40.537524361020765 74.2403083211639) bank2415 +2416 POINT(41.20565808985107 73.03498073843394) bank2416 +2417 POINT(39.878528636840805 73.7658044520564) bank2417 +2418 POINT(40.57322364350752 74.19834261238557) bank2418 +2419 POINT(39.919677569301385 74.53194136580217) bank2419 +2420 POINT(40.06905537698661 73.5923684854285) bank2420 +2421 POINT(40.91836700111051 74.45406970707795) bank2421 +2422 POINT(41.57053472923504 74.75556068946949) bank2422 +2423 POINT(41.13278968409323 74.93050787329588) bank2423 +2424 POINT(41.56561243230756 74.22221782041004) bank2424 +2425 POINT(40.78248145154171 73.41839702273504) bank2425 +2426 POINT(40.71100102062589 73.44388796974756) bank2426 +2427 POINT(41.061899758479555 73.19351389011332) bank2427 +2428 POINT(41.11365677367106 74.30241186560723) bank2428 +2429 POINT(41.640538908288924 73.60194636750164) bank2429 +2430 POINT(41.27409554354525 74.61642514096877) bank2430 +2431 POINT(41.23081252191161 73.67951832733678) bank2431 +2432 POINT(41.27887221959617 74.44788517391663) bank2432 +2433 POINT(41.31951210157361 73.20898137582789) bank2433 +2434 POINT(40.971453877338476 73.51583921280816) bank2434 +2435 POINT(41.02873769879616 74.97524596880903) bank2435 +2436 POINT(40.507341877179634 74.3303180085029) bank2436 +2437 POINT(41.023663361814684 74.74348880397653) bank2437 +2438 POINT(40.195004023972665 74.00381558418208) bank2438 +2439 POINT(40.46775680145857 74.48364608018512) bank2439 +2440 POINT(41.00155038061985 74.26693655762675) bank2440 +2441 POINT(40.65281137339508 74.22088682216709) bank2441 +2442 POINT(41.397238643071546 75.0045562953954) bank2442 +2443 POINT(41.08995337990365 73.57535180198693) bank2443 +2444 POINT(40.84311268804368 73.0273996618175) bank2444 +2445 POINT(41.605300641051194 73.81926993706473) bank2445 +2446 POINT(40.86609703357068 73.06390085232134) bank2446 +2447 POINT(40.99343909684187 74.62134775138473) bank2447 +2448 POINT(41.09005692152617 73.67090201004898) bank2448 +2449 POINT(40.123799993809364 74.76768634489447) bank2449 +2450 POINT(41.090260302949915 73.2858015099551) bank2450 +2451 POINT(40.23155674323249 73.42029331446615) bank2451 +2452 POINT(40.216528952860514 74.25016565156915) bank2452 +2453 POINT(40.123854907890184 74.36989295614222) bank2453 +2454 POINT(39.92788288525851 74.54980709370192) bank2454 +2455 POINT(41.0052726599092 73.70521440170629) bank2455 +2456 POINT(40.354166090279854 74.88530208031763) bank2456 +2457 POINT(40.63196105690377 73.5149222826406) bank2457 +2458 POINT(41.092781255430125 73.6345913826487) bank2458 +2459 POINT(41.13172269046823 74.44256356136509) bank2459 +2460 POINT(41.26135040980703 74.79475256524205) bank2460 +2461 POINT(40.42650600415096 74.828374914339) bank2461 +2462 POINT(39.84085791164421 73.74506975774536) bank2462 +2463 POINT(40.281325237308906 74.9769324092417) bank2463 +2464 POINT(40.898934918920105 74.17667561381093) bank2464 +2465 POINT(41.48213018468562 73.03191985763091) bank2465 +2466 POINT(40.14172677925661 73.9538317012943) bank2466 +2467 POINT(39.749882097697224 73.40829585382133) bank2467 +2468 POINT(40.67986872458424 73.53184083136823) bank2468 +2469 POINT(40.15945162422977 73.58396189000929) bank2469 +2470 POINT(40.46409945257621 73.30469653437474) bank2470 +2471 POINT(41.173425149158376 74.22435055080344) bank2471 +2472 POINT(41.182401798065975 74.34049933541834) bank2472 +2473 POINT(41.69493414089453 74.98804670081932) bank2473 +2474 POINT(40.442489518234645 74.56742725816933) bank2474 +2475 POINT(41.07215862436016 74.51146382180453) bank2475 +2476 POINT(40.93995506033709 73.05312103140609) bank2476 +2477 POINT(39.84655124526172 73.74857800682557) bank2477 +2478 POINT(40.65275678574729 73.6472648266537) bank2478 +2479 POINT(39.71379669738789 74.92889726430691) bank2479 +2480 POINT(39.73513503342453 73.5559286268052) bank2480 +2481 POINT(40.26389415527586 73.15029277952138) bank2481 +2482 POINT(40.51659284837552 74.87339997318419) bank2482 +2483 POINT(40.55878159364019 74.57163489083305) bank2483 +2484 POINT(41.651775224765714 74.88941391149642) bank2484 +2485 POINT(40.27016152444889 73.14031633811125) bank2485 +2486 POINT(40.751451647841506 74.82303585882462) bank2486 +2487 POINT(40.35996911929985 74.62665142553108) bank2487 +2488 POINT(41.552311683630926 74.25802988015936) bank2488 +2489 POINT(40.08192380279703 73.63585711966446) bank2489 +2490 POINT(40.96090278025429 73.84592677589129) bank2490 +2491 POINT(41.48056700544564 74.17862769755) bank2491 +2492 POINT(40.29825367349259 73.9745343464742) bank2492 +2493 POINT(40.72231340595458 73.98927199725402) bank2493 +2494 POINT(39.779029997564656 74.21608254492405) bank2494 +2495 POINT(41.13853690768414 73.84835447745706) bank2495 +2496 POINT(40.05868302725729 73.72744204774352) bank2496 +2497 POINT(40.75326538269857 74.24743505433398) bank2497 +2498 POINT(40.64177575138672 73.333392175469) bank2498 +2499 POINT(40.34661944277816 74.52637224740737) bank2499 +2500 POINT(40.12796889545896 74.69044539763763) bank2500 +2501 POINT(40.24304360754382 73.03531380356745) bank2501 +2502 POINT(39.928464678357365 74.53682186000185) bank2502 +2503 POINT(40.97910976324926 73.78273564983873) bank2503 +2504 POINT(41.173974690882 73.71470747923958) bank2504 +2505 POINT(40.3041761330067 73.80814272067629) bank2505 +2506 POINT(39.90437057012218 74.78803942924802) bank2506 +2507 POINT(41.52615501656817 73.58794726686129) bank2507 +2508 POINT(39.797800276852136 73.06321077076267) bank2508 +2509 POINT(41.17083078432831 74.73972827203465) bank2509 +2510 POINT(41.33345162707103 73.88559852970444) bank2510 +2511 POINT(39.91513532035983 73.5568983270625) bank2511 +2512 POINT(40.76885195012052 74.61116280958707) bank2512 +2513 POINT(39.9278350233643 73.80954420606918) bank2513 +2514 POINT(39.83815464302448 73.41739183908336) bank2514 +2515 POINT(40.024087766512665 74.69889595386287) bank2515 +2516 POINT(41.21807196545213 73.81104475259714) bank2516 +2517 POINT(41.455216808751956 73.83396055838496) bank2517 +2518 POINT(41.520784025946995 73.01008910517406) bank2518 +2519 POINT(41.56131357672487 74.86054147129842) bank2519 +2520 POINT(39.905791392094784 74.13818394584356) bank2520 +2521 POINT(40.13036730996636 74.67995503894966) bank2521 +2522 POINT(40.367019301338225 74.57717096556739) bank2522 +2523 POINT(40.05861469943961 73.02939783164803) bank2523 +2524 POINT(40.98317507208252 73.33529160617012) bank2524 +2525 POINT(40.69587320011408 74.05564981565138) bank2525 +2526 POINT(41.27406563424526 74.58034728389018) bank2526 +2527 POINT(40.17560471282982 74.13975011647997) bank2527 +2528 POINT(40.059039648737766 73.55378626483764) bank2528 +2529 POINT(41.34927470666222 74.22576321022309) bank2529 +2530 POINT(41.53529656692356 73.16328048980787) bank2530 +2531 POINT(41.4797914114679 73.88321652375231) bank2531 +2532 POINT(40.28443897031784 74.15219715955193) bank2532 +2533 POINT(41.42038643791773 73.9266805021221) bank2533 +2534 POINT(41.40381149852433 73.0908373454754) bank2534 +2535 POINT(40.448130888118996 74.00070914369105) bank2535 +2536 POINT(40.707744725930446 74.43604563708928) bank2536 +2537 POINT(40.10075987398515 73.56495634495533) bank2537 +2538 POINT(40.10659373730657 74.731457003802) bank2538 +2539 POINT(40.65083529346244 73.37757022510822) bank2539 +2540 POINT(41.69710775144016 74.77430534231344) bank2540 +2541 POINT(40.9802211007648 74.09562759815906) bank2541 +2542 POINT(41.191201722590414 74.16875488415639) bank2542 +2543 POINT(40.06251398239611 73.8522121051851) bank2543 +2544 POINT(41.156957130984466 74.13774253786214) bank2544 +2545 POINT(40.14329258175995 74.83531033424497) bank2545 +2546 POINT(40.453922808021666 74.16900077845521) bank2546 +2547 POINT(39.762280246741966 74.02323628505624) bank2547 +2548 POINT(40.15822707728836 73.22796174608303) bank2548 +2549 POINT(40.33166348333301 73.41143598046817) bank2549 +2550 POINT(40.191318931148054 73.16253831404096) bank2550 +2551 POINT(40.042630832049184 73.73233967534745) bank2551 +2552 POINT(40.834428745926836 73.17060950974744) bank2552 +2553 POINT(40.215655193699334 74.06175675424537) bank2553 +2554 POINT(41.588527109351084 74.52589148253986) bank2554 +2555 POINT(41.41171652159822 73.5563343181665) bank2555 +2556 POINT(41.358367022574335 73.83507439701891) bank2556 +2557 POINT(41.57923402113619 73.43859072112343) bank2557 +2558 POINT(41.11877285639769 73.6318435888768) bank2558 +2559 POINT(40.39171642871388 74.33932712926075) bank2559 +2560 POINT(41.349724275788645 74.56782875230623) bank2560 +2561 POINT(40.35468489316541 73.90262988980678) bank2561 +2562 POINT(41.710921774290796 74.3051898836248) bank2562 +2563 POINT(41.360448243084605 74.89938913921266) bank2563 +2564 POINT(39.799659563270275 74.15025363725904) bank2564 +2565 POINT(40.75520398593384 74.10398525512728) bank2565 +2566 POINT(41.24559097735407 74.92889415894015) bank2566 +2567 POINT(41.6066882571771 74.99427060253915) bank2567 +2568 POINT(40.419400526897455 73.52470577762945) bank2568 +2569 POINT(40.666786995835764 73.50343236225966) bank2569 +2570 POINT(40.086253341804714 73.0271022793925) bank2570 +2571 POINT(41.6773728947719 73.04349266965468) bank2571 +2572 POINT(39.78270846942992 73.18689685897164) bank2572 +2573 POINT(39.985485525128425 73.9239986544659) bank2573 +2574 POINT(41.39352069031108 74.51443856720834) bank2574 +2575 POINT(39.751038840055486 73.47466013386631) bank2575 +2576 POINT(41.02208181901051 74.09377553182338) bank2576 +2577 POINT(41.48355257801476 74.23204057815913) bank2577 +2578 POINT(41.3177514153191 73.6497094869689) bank2578 +2579 POINT(39.7287530259731 73.28164189233746) bank2579 +2580 POINT(40.65962305618491 73.12768016819618) bank2580 +2581 POINT(40.82223601243437 73.80549679893734) bank2581 +2582 POINT(39.79613592146203 74.13337567408935) bank2582 +2583 POINT(41.29172913457093 74.81729488016616) bank2583 +2584 POINT(40.09129416186365 73.23420447694512) bank2584 +2585 POINT(40.87944974003168 73.30535417509728) bank2585 +2586 POINT(40.22008701072919 73.6273826579117) bank2586 +2587 POINT(41.06431018542752 74.01910949190311) bank2587 +2588 POINT(40.586031181674834 74.65422482309867) bank2588 +2589 POINT(41.58398778059691 73.32025511179495) bank2589 +2590 POINT(41.19380385485494 73.59608827174306) bank2590 +2591 POINT(40.2624525172977 73.14996061299527) bank2591 +2592 POINT(41.208484536827 73.33136740936959) bank2592 +2593 POINT(40.16908213129176 74.82178667871702) bank2593 +2594 POINT(41.53240648135532 73.51769174102235) bank2594 +2595 POINT(39.83036386864608 73.05857140836065) bank2595 +2596 POINT(40.5363288182615 73.03081246574895) bank2596 +2597 POINT(40.45668768462605 74.71086105448248) bank2597 +2598 POINT(41.42178503761318 74.27924552428048) bank2598 +2599 POINT(40.1623267159136 74.69369386543852) bank2599 +2600 POINT(41.71192645487251 75.00289525350578) bank2600 +2601 POINT(41.39953184160219 73.04227295505704) bank2601 +2602 POINT(40.29720184048992 74.04800460375444) bank2602 +2603 POINT(41.075226579562006 74.61567206310364) bank2603 +2604 POINT(40.465042576802325 73.57201016270447) bank2604 +2605 POINT(41.54012469428024 74.88819123931988) bank2605 +2606 POINT(39.90105713966211 73.07709628646056) bank2606 +2607 POINT(40.19212885393562 74.04844637443938) bank2607 +2608 POINT(40.48401619350491 74.97376750866312) bank2608 +2609 POINT(41.39415152118333 73.53756838935904) bank2609 +2610 POINT(41.03071961280418 73.87612032927771) bank2610 +2611 POINT(40.21004014744644 73.35241878146705) bank2611 +2612 POINT(41.633214017632135 74.2932542143613) bank2612 +2613 POINT(40.825046634112276 73.29538953891733) bank2613 +2614 POINT(41.57989865685562 73.87767442447047) bank2614 +2615 POINT(41.24422547844558 74.72626219745149) bank2615 +2616 POINT(41.658861537265274 74.29655839755563) bank2616 +2617 POINT(40.39250915518845 73.40796027531925) bank2617 +2618 POINT(39.99345988740705 73.21354663677971) bank2618 +2619 POINT(40.07016459441546 74.00453982469453) bank2619 +2620 POINT(40.45479056103983 73.74090725457849) bank2620 +2621 POINT(40.324669684980556 73.28011958829102) bank2621 +2622 POINT(40.946384344313074 73.07891786017386) bank2622 +2623 POINT(41.43812220164303 74.81416848328749) bank2623 +2624 POINT(41.16287492134439 73.67614772081178) bank2624 +2625 POINT(40.628228694355116 73.36509930442487) bank2625 +2626 POINT(40.05274199573558 74.55413884093943) bank2626 +2627 POINT(41.11642954950857 73.5684901651033) bank2627 +2628 POINT(40.6095133006385 73.75629834198746) bank2628 +2629 POINT(41.61574255864436 74.77417077759637) bank2629 +2630 POINT(40.57934004412192 74.19154412239881) bank2630 +2631 POINT(40.798481556678674 73.88226029655898) bank2631 +2632 POINT(41.03010456195831 74.68425014794971) bank2632 +2633 POINT(41.21948785638511 73.11993242448182) bank2633 +2634 POINT(41.51359913024182 73.63259101418983) bank2634 +2635 POINT(40.92282703014725 73.18564237809638) bank2635 +2636 POINT(39.89626517662509 74.85190908330148) bank2636 +2637 POINT(40.31849277808429 74.25592710789373) bank2637 +2638 POINT(41.47964454754694 74.93536106689726) bank2638 +2639 POINT(40.67899834073157 73.9857030172804) bank2639 +2640 POINT(41.42414715090171 74.37019597774935) bank2640 +2641 POINT(40.88459258429614 73.35158800073256) bank2641 +2642 POINT(40.364677270578525 73.40684183141288) bank2642 +2643 POINT(41.52910500078846 73.42242781470569) bank2643 +2644 POINT(40.60953693788398 74.75947019314816) bank2644 +2645 POINT(40.1208727775541 73.97199797242438) bank2645 +2646 POINT(41.57959445909663 74.3615947652545) bank2646 +2647 POINT(40.76733521828574 73.2559595210589) bank2647 +2648 POINT(41.155510613741605 73.2498646227014) bank2648 +2649 POINT(39.92455072203336 74.75145712477085) bank2649 +2650 POINT(40.03644958204623 74.78157512547445) bank2650 +2651 POINT(40.45867138522507 73.80029112493055) bank2651 +2652 POINT(41.577270083653666 73.99547317622262) bank2652 +2653 POINT(40.9726020426596 73.70289975902264) bank2653 +2654 POINT(40.89521950686521 73.45378591000575) bank2654 +2655 POINT(41.02305348187851 74.50206287807524) bank2655 +2656 POINT(39.76433890528961 73.58340996512155) bank2656 +2657 POINT(40.907898119932746 73.17012403184097) bank2657 +2658 POINT(40.32718859489369 73.42894925225214) bank2658 +2659 POINT(40.11620798890054 73.3862989125987) bank2659 +2660 POINT(40.27668948552845 73.07205641171981) bank2660 +2661 POINT(40.83843127544398 74.92670299683549) bank2661 +2662 POINT(40.578941874524546 74.29615049831902) bank2662 +2663 POINT(41.57090259639574 74.77686391562538) bank2663 +2664 POINT(41.39391310446168 73.02634148053197) bank2664 +2665 POINT(39.88644090552791 73.37515652022242) bank2665 +2666 POINT(41.67334933747958 73.4142602815568) bank2666 +2667 POINT(40.36115427824945 74.90037412094227) bank2667 +2668 POINT(40.44250545933581 73.68302726487347) bank2668 +2669 POINT(40.879200527549955 74.43466581118254) bank2669 +2670 POINT(41.695532316576454 73.26303846453256) bank2670 +2671 POINT(40.957460242640224 73.86883557398623) bank2671 +2672 POINT(41.34715824585053 74.36046247937203) bank2672 +2673 POINT(39.895123960534725 73.53470984125718) bank2673 +2674 POINT(41.03166659326662 74.19918443892449) bank2674 +2675 POINT(40.6308989185154 74.69488452424167) bank2675 +2676 POINT(40.62559774723334 73.17865145694013) bank2676 +2677 POINT(40.98652461972054 73.46304614062339) bank2677 +2678 POINT(41.209539150844165 73.36912311088888) bank2678 +2679 POINT(40.61992987785758 73.33125877525389) bank2679 +2680 POINT(41.129210585125705 73.14249207182024) bank2680 +2681 POINT(40.36276362523165 73.20715931899976) bank2681 +2682 POINT(41.597075321117906 73.80389368506813) bank2682 +2683 POINT(40.59940654666307 73.71276035034158) bank2683 +2684 POINT(40.365708552043714 73.7288264310719) bank2684 +2685 POINT(40.56474178483531 73.71871534222412) bank2685 +2686 POINT(41.26521139103354 73.77874264607522) bank2686 +2687 POINT(39.99720008302897 73.1777491580313) bank2687 +2688 POINT(40.23320279710988 73.75683624539275) bank2688 +2689 POINT(39.98523034947571 73.61694040981433) bank2689 +2690 POINT(39.88380138658022 73.3653739530821) bank2690 +2691 POINT(40.821812891760494 73.10680185237912) bank2691 +2692 POINT(40.17472989901119 73.83270452626303) bank2692 +2693 POINT(39.91621619335198 74.30163529171928) bank2693 +2694 POINT(41.110975289284184 74.47992221297446) bank2694 +2695 POINT(41.18676262010966 73.18933925905365) bank2695 +2696 POINT(40.558406977432405 73.12423000206819) bank2696 +2697 POINT(40.465617758408214 74.28420076908574) bank2697 +2698 POINT(40.59985376583219 73.45531778490383) bank2698 +2699 POINT(41.43773775130635 73.86027846917362) bank2699 +2700 POINT(40.097603953253774 73.13190784937571) bank2700 +2701 POINT(41.028918444777574 73.58163156537672) bank2701 +2702 POINT(39.790158680834416 73.95782793850314) bank2702 +2703 POINT(40.28408898668827 73.07039006559178) bank2703 +2704 POINT(41.38677870004181 73.88107719757492) bank2704 +2705 POINT(39.96108223037869 74.21991434916868) bank2705 +2706 POINT(40.74430793498907 73.45086554750458) bank2706 +2707 POINT(41.06858940699159 73.0512954891701) bank2707 +2708 POINT(40.14532108548179 73.56929715612408) bank2708 +2709 POINT(40.184053554970106 73.2778807447756) bank2709 +2710 POINT(40.97797638861525 74.58713899070615) bank2710 +2711 POINT(39.92501795391312 74.6166664110742) bank2711 +2712 POINT(40.135558439620006 73.59963682751714) bank2712 +2713 POINT(41.39849540897802 74.28021881293532) bank2713 +2714 POINT(40.63712511778521 74.38083637122179) bank2714 +2715 POINT(41.547077927117 73.6747476510461) bank2715 +2716 POINT(41.36243700359671 73.42381359623947) bank2716 +2717 POINT(40.319992828576645 73.86435205183295) bank2717 +2718 POINT(41.53002714297207 74.62366939832043) bank2718 +2719 POINT(41.50541006743957 73.18422814133879) bank2719 +2720 POINT(40.70087568629051 74.0598609219376) bank2720 +2721 POINT(41.141574520988534 73.49084464470005) bank2721 +2722 POINT(39.86266440331043 74.83719989835744) bank2722 +2723 POINT(40.33445721897954 73.1050249255218) bank2723 +2724 POINT(41.51650506861093 73.3700837796643) bank2724 +2725 POINT(41.020699493385735 74.62127543229387) bank2725 +2726 POINT(40.144459349331285 74.27909894598923) bank2726 +2727 POINT(40.795469969337724 74.96331034078439) bank2727 +2728 POINT(40.00479664974317 74.67521119153348) bank2728 +2729 POINT(39.8444434743944 73.10089282222408) bank2729 +2730 POINT(39.854328824294235 73.84053427575698) bank2730 +2731 POINT(41.441547770800696 74.68717490533996) bank2731 +2732 POINT(41.04766945664763 73.7546276441039) bank2732 +2733 POINT(40.552033267655304 74.1913775080655) bank2733 +2734 POINT(41.604461130045564 74.97687876226102) bank2734 +2735 POINT(41.38485701368323 74.07217605372593) bank2735 +2736 POINT(39.96382467455321 73.94706698512903) bank2736 +2737 POINT(40.80246161338815 73.8257844530058) bank2737 +2738 POINT(40.010303382462865 73.62646258802948) bank2738 +2739 POINT(40.9471239284763 74.55916200123372) bank2739 +2740 POINT(40.43673640032833 74.88028865704575) bank2740 +2741 POINT(41.44337305783558 74.59064327874971) bank2741 +2742 POINT(40.22193487250449 74.97105060633479) bank2742 +2743 POINT(41.0282918205316 73.63384628169844) bank2743 +2744 POINT(41.49818724835882 74.16829948563122) bank2744 +2745 POINT(40.73138310664046 74.37651710888488) bank2745 +2746 POINT(40.744664272901886 74.15865242610617) bank2746 +2747 POINT(40.684754633699626 73.90128968780395) bank2747 +2748 POINT(40.43544665622237 73.49886736958322) bank2748 +2749 POINT(41.044944139778494 74.56601407215773) bank2749 +2750 POINT(41.21970979428591 74.86396302579058) bank2750 +2751 POINT(40.10701281399233 73.57153337704344) bank2751 +2752 POINT(41.31043478182846 74.0218484846262) bank2752 +2753 POINT(40.31749770045389 73.24545501373237) bank2753 +2754 POINT(40.89402347031051 73.3098567372765) bank2754 +2755 POINT(40.1901202422386 74.95827388864595) bank2755 +2756 POINT(39.90512459965771 74.30088765202329) bank2756 +2757 POINT(41.703292563549454 74.74205039309544) bank2757 +2758 POINT(41.32029625954665 74.85418175829072) bank2758 +2759 POINT(39.85628275490222 73.21317120080172) bank2759 +2760 POINT(40.66985236384489 74.28961449071483) bank2760 +2761 POINT(40.487857316849656 74.7038005790683) bank2761 +2762 POINT(41.419074360883826 74.85040383439664) bank2762 +2763 POINT(39.88877750088386 74.21335639016088) bank2763 +2764 POINT(41.52063766689299 74.9278703723836) bank2764 +2765 POINT(41.298878767253946 73.76630643247634) bank2765 +2766 POINT(40.463588719133966 73.04764881725454) bank2766 +2767 POINT(41.56330273837226 74.0087337396196) bank2767 +2768 POINT(40.44846281713734 74.73542946135895) bank2768 +2769 POINT(40.3181656912268 73.03354359532078) bank2769 +2770 POINT(40.04878029294154 74.93727076596531) bank2770 +2771 POINT(40.30199092358618 74.89148507513262) bank2771 +2772 POINT(40.5556670155334 74.84752529362636) bank2772 +2773 POINT(41.22431990376297 73.20326900984719) bank2773 +2774 POINT(40.70977693025627 74.8462010200375) bank2774 +2775 POINT(41.36536188508879 73.46782968602147) bank2775 +2776 POINT(39.8889616499312 74.52213853225626) bank2776 +2777 POINT(40.983391205377686 73.93065499183344) bank2777 +2778 POINT(40.34022351219288 73.51556893937617) bank2778 +2779 POINT(41.25025712237185 74.5144598783354) bank2779 +2780 POINT(39.98147322359776 74.45386431874306) bank2780 +2781 POINT(41.39409727309083 73.77592408745011) bank2781 +2782 POINT(41.047228270135754 73.00689550476066) bank2782 +2783 POINT(40.188533838236914 74.46063097124501) bank2783 +2784 POINT(41.09364346353051 73.6027127058335) bank2784 +2785 POINT(39.83005232370616 74.84418576125593) bank2785 +2786 POINT(39.71681619014719 74.51820038578533) bank2786 +2787 POINT(40.68715307146851 74.09556262568147) bank2787 +2788 POINT(41.40407906346679 73.6663828018999) bank2788 +2789 POINT(40.42473036501861 74.5415640031086) bank2789 +2790 POINT(41.391097523652576 74.02080180097774) bank2790 +2791 POINT(39.844924144155456 73.7442354631223) bank2791 +2792 POINT(40.7768764532443 74.21722529831769) bank2792 +2793 POINT(39.89995784683527 73.65921858688029) bank2793 +2794 POINT(40.90875579615634 73.07507902297895) bank2794 +2795 POINT(40.480390701758374 74.77242321856914) bank2795 +2796 POINT(40.99189529748031 74.35011088929545) bank2796 +2797 POINT(41.030738195802805 73.37134799899145) bank2797 +2798 POINT(41.27265595400009 74.8450755698644) bank2798 +2799 POINT(41.526980073482456 73.2545684130873) bank2799 +2800 POINT(40.822477768678596 73.44381989787044) bank2800 +2801 POINT(41.115734936025326 74.03485710809254) bank2801 +2802 POINT(40.80704518504032 73.49971740263861) bank2802 +2803 POINT(39.96983908053197 73.28828330824366) bank2803 +2804 POINT(40.1613883629061 73.99814581816919) bank2804 +2805 POINT(41.65669400715587 73.20755619262195) bank2805 +2806 POINT(39.831712085735035 74.88796115420682) bank2806 +2807 POINT(41.29149659762529 74.78744899466395) bank2807 +2808 POINT(40.01799109098404 74.73358123034771) bank2808 +2809 POINT(41.04906731461787 74.8876547964203) bank2809 +2810 POINT(39.95187833284893 74.36189979274073) bank2810 +2811 POINT(41.60105116146917 73.3155372517517) bank2811 +2812 POINT(40.63917666323818 74.2801551152694) bank2812 +2813 POINT(40.27885039889051 74.73839168007957) bank2813 +2814 POINT(40.804746766993134 74.92369498675615) bank2814 +2815 POINT(40.27155468903511 73.56192476664131) bank2815 +2816 POINT(39.8382150778324 73.10128780016002) bank2816 +2817 POINT(40.498888787147074 73.26623981965649) bank2817 +2818 POINT(40.80016721416677 74.77267360209188) bank2818 +2819 POINT(40.227432700507926 73.18974721500467) bank2819 +2820 POINT(39.80736373588049 74.03283089123389) bank2820 +2821 POINT(40.412613964041064 74.10678612582052) bank2821 +2822 POINT(40.91015930413853 73.78855707994212) bank2822 +2823 POINT(39.79363687549867 74.92160351059046) bank2823 +2824 POINT(40.304094422387884 74.0686528784492) bank2824 +2825 POINT(40.84189324856937 74.34333192031178) bank2825 +2826 POINT(41.07447040717287 73.109611355849) bank2826 +2827 POINT(41.451467259499296 73.80826709734258) bank2827 +2828 POINT(41.36401613491544 73.15024456397897) bank2828 +2829 POINT(41.60454209964857 74.8382704556804) bank2829 +2830 POINT(39.777576459645374 73.68551189017865) bank2830 +2831 POINT(41.091689253062434 74.78082012380466) bank2831 +2832 POINT(39.90649148388104 73.55484750981411) bank2832 +2833 POINT(41.50155000696621 74.04325879603473) bank2833 +2834 POINT(39.93725749489735 73.85311672351402) bank2834 +2835 POINT(40.39419214197972 73.33425121412023) bank2835 +2836 POINT(40.10962647029229 74.22772889729362) bank2836 +2837 POINT(40.985874540267865 73.64069195260336) bank2837 +2838 POINT(40.5660904288148 74.83134213361335) bank2838 +2839 POINT(40.14803681307411 73.77114076489799) bank2839 +2840 POINT(40.91008997356827 74.03640850869994) bank2840 +2841 POINT(39.945397378847524 73.40209638038925) bank2841 +2842 POINT(40.88115186572495 74.64209761286568) bank2842 +2843 POINT(41.593086460906854 74.62071547321192) bank2843 +2844 POINT(41.61711221215625 74.00220513214035) bank2844 +2845 POINT(41.36072645155695 73.29518889980629) bank2845 +2846 POINT(40.59786288569095 73.66588793803174) bank2846 +2847 POINT(40.47686666524719 74.38757620190964) bank2847 +2848 POINT(41.191092173620014 73.34111267986624) bank2848 +2849 POINT(40.273536489358406 73.36189569711864) bank2849 +2850 POINT(41.57281368533471 73.52426065678989) bank2850 +2851 POINT(40.30935104170827 73.25739289775308) bank2851 +2852 POINT(39.925208536052835 74.27407346849911) bank2852 +2853 POINT(41.2046432093515 73.32231275208214) bank2853 +2854 POINT(40.86580672612852 73.55521969516734) bank2854 +2855 POINT(40.69666701030092 73.61643859966641) bank2855 +2856 POINT(40.94635621216146 74.48211253395604) bank2856 +2857 POINT(40.69317376536991 73.19714472713468) bank2857 +2858 POINT(40.992202152873084 74.34872746488426) bank2858 +2859 POINT(41.048248130546064 74.92612312835686) bank2859 +2860 POINT(40.61744751305871 74.61733174044498) bank2860 +2861 POINT(39.78283857221031 73.90129369753419) bank2861 +2862 POINT(40.730827027578925 73.03142513596863) bank2862 +2863 POINT(41.403769624050305 74.76701051746032) bank2863 +2864 POINT(41.024332100771 74.68926855603083) bank2864 +2865 POINT(40.10439086264722 73.65989501900773) bank2865 +2866 POINT(41.228826152786034 74.44849129582742) bank2866 +2867 POINT(40.55303918937167 74.30924387748942) bank2867 +2868 POINT(41.14221216091181 74.85855287343531) bank2868 +2869 POINT(40.17330253685771 74.73400739668189) bank2869 +2870 POINT(40.57539136025942 74.45734771652897) bank2870 +2871 POINT(40.95696233134915 74.47909235957506) bank2871 +2872 POINT(40.78887033764288 74.72891727254859) bank2872 +2873 POINT(39.87828040894489 74.22341661810343) bank2873 +2874 POINT(40.72828273248904 74.57177067252105) bank2874 +2875 POINT(39.90550448766024 73.5124722189243) bank2875 +2876 POINT(40.09877062021651 73.3692561552382) bank2876 +2877 POINT(41.376803357694705 73.35456720173788) bank2877 +2878 POINT(39.7582006329255 74.3062038292365) bank2878 +2879 POINT(40.63005933574172 74.83027559592803) bank2879 +2880 POINT(40.84948482027587 74.63019990588015) bank2880 +2881 POINT(40.161545974882266 74.40766597827633) bank2881 +2882 POINT(41.377896508942015 74.40493008651637) bank2882 +2883 POINT(40.04650658381025 74.1291888708836) bank2883 +2884 POINT(39.85597818827464 73.14417076672592) bank2884 +2885 POINT(39.874315630816156 73.20038112638301) bank2885 +2886 POINT(40.33815392536162 74.46053361798911) bank2886 +2887 POINT(41.67461316162811 73.0921065987031) bank2887 +2888 POINT(41.32633976799475 74.78293930810094) bank2888 +2889 POINT(41.65498118831724 74.22572196277251) bank2889 +2890 POINT(39.95429004722609 74.47958020686339) bank2890 +2891 POINT(40.186243451655685 74.55651727643685) bank2891 +2892 POINT(39.94453023911871 73.19001315730377) bank2892 +2893 POINT(39.87909615412644 73.56923353296526) bank2893 +2894 POINT(40.612512549517746 73.19055756737392) bank2894 +2895 POINT(41.457041084786404 74.06564784264263) bank2895 +2896 POINT(40.24261457332026 74.03310901839073) bank2896 +2897 POINT(41.64233694638681 73.37123883051417) bank2897 +2898 POINT(40.37260108728362 74.21777597382828) bank2898 +2899 POINT(41.12581059299647 74.47291722897182) bank2899 +2900 POINT(40.973962861156586 73.95357165625467) bank2900 +2901 POINT(40.25858554891821 74.68074092074667) bank2901 +2902 POINT(41.60068031676034 73.75666419996242) bank2902 +2903 POINT(40.58258751695535 74.83765155748344) bank2903 +2904 POINT(39.897287508942604 73.45066230120617) bank2904 +2905 POINT(41.55381082288747 73.9149244915113) bank2905 +2906 POINT(40.379589876978315 73.80949117631249) bank2906 +2907 POINT(40.455325772222395 73.11758630654009) bank2907 +2908 POINT(40.175364200539626 74.76887612260815) bank2908 +2909 POINT(40.473160356769384 73.80863709666583) bank2909 +2910 POINT(40.91864503829596 73.9454774621011) bank2910 +2911 POINT(40.261405529134244 74.78317748319562) bank2911 +2912 POINT(39.82803193459806 74.93221579666276) bank2912 +2913 POINT(39.82086183804909 73.22197948351311) bank2913 +2914 POINT(41.22503177498891 74.3901306261494) bank2914 +2915 POINT(41.4998395028275 73.28122627891128) bank2915 +2916 POINT(39.892799341928445 74.50248035915018) bank2916 +2917 POINT(40.79602203156202 73.5361416406772) bank2917 +2918 POINT(40.40472751380138 74.00018960979534) bank2918 +2919 POINT(41.03319749298018 74.31965629974472) bank2919 +2920 POINT(40.56285492865437 73.81592615641404) bank2920 +2921 POINT(40.637838384377446 73.79541179665812) bank2921 +2922 POINT(40.28888169057192 74.58763884856619) bank2922 +2923 POINT(40.32952875174519 74.17972007451908) bank2923 +2924 POINT(40.559443785289766 73.91674393675838) bank2924 +2925 POINT(41.16875972274802 73.55880675446416) bank2925 +2926 POINT(40.515983263530785 74.56404027915075) bank2926 +2927 POINT(39.74035138367128 74.67915104986413) bank2927 +2928 POINT(40.3506691844944 73.97760966870237) bank2928 +2929 POINT(40.91442157745921 74.01852269589389) bank2929 +2930 POINT(40.59213680308259 73.52674675686563) bank2930 +2931 POINT(41.61373967650396 73.90246914434398) bank2931 +2932 POINT(40.2285231488372 73.3122828290471) bank2932 +2933 POINT(41.14247748995031 73.85453178559027) bank2933 +2934 POINT(40.20360885965626 74.05700936292142) bank2934 +2935 POINT(41.125362889719675 73.72072896207844) bank2935 +2936 POINT(40.29964061980486 74.89848164352345) bank2936 +2937 POINT(40.3357059483712 74.2343077452524) bank2937 +2938 POINT(40.9083882035646 73.71536930574618) bank2938 +2939 POINT(40.43943037881691 73.90961749317374) bank2939 +2940 POINT(41.60539777977998 74.03164503743032) bank2940 +2941 POINT(39.71706167517088 74.66270690041279) bank2941 +2942 POINT(41.508464498669916 75.00395389469917) bank2942 +2943 POINT(39.899796435115846 74.68023856271851) bank2943 +2944 POINT(40.26452736336053 73.77252533721902) bank2944 +2945 POINT(41.292856372170085 73.7535831901748) bank2945 +2946 POINT(41.05244107340317 73.69115677112536) bank2946 +2947 POINT(41.16544808190209 73.12007297072473) bank2947 +2948 POINT(39.772764280288605 74.66569589589344) bank2948 +2949 POINT(40.61207703553695 74.89723344179885) bank2949 +2950 POINT(40.469946075028716 74.12206641352422) bank2950 +2951 POINT(41.61918984294816 74.93688412462346) bank2951 +2952 POINT(41.616430595349286 74.87831312409857) bank2952 +2953 POINT(39.764785977526635 74.67841528320464) bank2953 +2954 POINT(41.11895287342451 73.38601890087169) bank2954 +2955 POINT(39.79671565670346 74.42351330475306) bank2955 +2956 POINT(39.85786411273727 74.85958002617949) bank2956 +2957 POINT(39.76575402250707 74.98845296820134) bank2957 +2958 POINT(40.19811246587458 73.28254171762029) bank2958 +2959 POINT(40.57281694222443 73.56022382799523) bank2959 +2960 POINT(41.13075842419385 74.7174377458335) bank2960 +2961 POINT(40.57539623872213 73.33926334441266) bank2961 +2962 POINT(41.63008359270057 74.45968865523069) bank2962 +2963 POINT(41.403093758156814 74.01145097894039) bank2963 +2964 POINT(41.01749285825668 74.8161552614116) bank2964 +2965 POINT(40.01686064461778 74.46579785436879) bank2965 +2966 POINT(40.409663544178095 73.91493703236247) bank2966 +2967 POINT(40.10333695123335 74.895352755607) bank2967 +2968 POINT(40.71810591458887 73.61179593551984) bank2968 +2969 POINT(40.45633121115972 73.41001607370062) bank2969 +2970 POINT(41.27197421537546 73.9053331164325) bank2970 +2971 POINT(41.27138278734465 74.03156742601949) bank2971 +2972 POINT(40.52463427771546 73.72314760833429) bank2972 +2973 POINT(41.41834761680231 74.35080178869805) bank2973 +2974 POINT(41.099845747847766 73.47865786655842) bank2974 +2975 POINT(41.391921853936985 74.56997446969598) bank2975 +2976 POINT(41.25015834205893 74.05702762857203) bank2976 +2977 POINT(39.92304602737272 73.65401614455027) bank2977 +2978 POINT(40.510087103261554 73.9060002447809) bank2978 +2979 POINT(39.91955474309649 74.67889222638433) bank2979 +2980 POINT(39.79324021068534 73.64625923532266) bank2980 +2981 POINT(40.27911108958962 73.05135814979282) bank2981 +2982 POINT(40.471837448936654 73.70023308942363) bank2982 +2983 POINT(41.19881763785223 74.74629933199411) bank2983 +2984 POINT(40.420476511496894 74.17971290282523) bank2984 +2985 POINT(41.01357239235023 73.79004157960952) bank2985 +2986 POINT(40.96220723711373 74.58418632604331) bank2986 +2987 POINT(41.53499807425288 73.2845216561456) bank2987 +2988 POINT(40.48893385895653 73.62304294695879) bank2988 +2989 POINT(40.924253122142495 73.64059736293609) bank2989 +2990 POINT(40.569554217192426 74.80777818146552) bank2990 +2991 POINT(39.90001997125757 73.09194749992339) bank2991 +2992 POINT(39.9422240623775 73.12402706534672) bank2992 +2993 POINT(40.420098560587874 73.10029683820909) bank2993 +2994 POINT(41.69785712752786 73.02419665299857) bank2994 +2995 POINT(39.72573764592743 74.85032273942265) bank2995 +2996 POINT(40.897205739516025 74.53600068401508) bank2996 +2997 POINT(40.35304546052005 74.03884762563065) bank2997 +2998 POINT(41.52577694383319 73.68728235459271) bank2998 +2999 POINT(41.443746468342944 74.24185752781376) bank2999 +3000 POINT(40.14364278052967 74.45937813814716) bank3000 +3001 POINT(41.67597225111166 73.75251460096628) bank3001 +3002 POINT(40.01686000231245 73.58686003841109) bank3002 +3003 POINT(40.77241649378011 74.76191667785987) bank3003 +3004 POINT(40.76447483732143 74.17719094225049) bank3004 +3005 POINT(40.780976580666184 73.78535032423879) bank3005 +3006 POINT(39.9579371613294 74.5361666109048) bank3006 +3007 POINT(39.96314985880309 73.46206675645699) bank3007 +3008 POINT(40.48021972591559 74.31200972918835) bank3008 +3009 POINT(40.46440203168729 74.49134141833076) bank3009 +3010 POINT(41.347834525205734 73.4140390773968) bank3010 +3011 POINT(40.573985875545 74.33286356504041) bank3011 +3012 POINT(40.62018574909218 74.47548859937305) bank3012 +3013 POINT(41.00062298534777 74.86427434334004) bank3013 +3014 POINT(40.11045488018858 73.38761196713135) bank3014 +3015 POINT(39.813512815909846 73.92012396458851) bank3015 +3016 POINT(39.82992403945976 73.73702682992426) bank3016 +3017 POINT(40.20691102862639 73.96312359789277) bank3017 +3018 POINT(41.49905722192791 74.6477339280843) bank3018 +3019 POINT(41.51217341936943 73.19992167291879) bank3019 +3020 POINT(40.74373584842021 73.30038831481477) bank3020 +3021 POINT(41.62527508458478 74.74157233376974) bank3021 +3022 POINT(41.28590661547818 74.34821489660219) bank3022 +3023 POINT(39.9358094929675 74.59593618065252) bank3023 +3024 POINT(41.09537313927995 74.74280635549103) bank3024 +3025 POINT(40.57766973940572 74.13571543822212) bank3025 +3026 POINT(39.930829673497925 74.85274533277008) bank3026 +3027 POINT(39.95733313113235 73.22691645324443) bank3027 +3028 POINT(41.31243261249471 73.77833894679384) bank3028 +3029 POINT(40.75495180310518 73.09509699391275) bank3029 +3030 POINT(41.25936424098922 74.98503446915328) bank3030 +3031 POINT(41.008328200560975 74.60793623166222) bank3031 +3032 POINT(41.0439927001339 73.62392586841963) bank3032 +3033 POINT(40.40313286828903 74.32927600456209) bank3033 +3034 POINT(41.039960543465405 73.31113736340977) bank3034 +3035 POINT(41.51771638098491 73.12934109006903) bank3035 +3036 POINT(40.10126196314486 74.10002690405007) bank3036 +3037 POINT(41.21569080402716 74.66490836570091) bank3037 +3038 POINT(41.17932999162149 74.33796404651066) bank3038 +3039 POINT(41.13346190274913 73.0664857249251) bank3039 +3040 POINT(39.90310089781015 73.62475614742046) bank3040 +3041 POINT(40.398465237954454 74.65764807938832) bank3041 +3042 POINT(39.91109290892238 74.67842005927928) bank3042 +3043 POINT(40.87578362957246 74.99149641317743) bank3043 +3044 POINT(41.596647347667755 74.09680020503914) bank3044 +3045 POINT(41.3095371266782 73.29864306388207) bank3045 +3046 POINT(40.965198581774466 73.03627073382606) bank3046 +3047 POINT(40.28595627412999 74.5137905449848) bank3047 +3048 POINT(39.91044592215395 74.13111824789875) bank3048 +3049 POINT(40.10594593043237 74.48761580675254) bank3049 +3050 POINT(40.92519672443288 73.78968798213559) bank3050 +3051 POINT(41.49308639783423 73.61264605991919) bank3051 +3052 POINT(41.68445160102254 73.05572470406238) bank3052 +3053 POINT(40.314974577467176 74.34856664571717) bank3053 +3054 POINT(41.421850229777796 73.07054132198488) bank3054 +3055 POINT(40.346691573302884 74.63342468953155) bank3055 +3056 POINT(41.462501718252305 74.45284454643702) bank3056 +3057 POINT(40.72795809066483 73.56226939644225) bank3057 +3058 POINT(40.62867897298716 73.15992851232933) bank3058 +3059 POINT(40.615918314781155 73.09835750214614) bank3059 +3060 POINT(41.138012596628585 73.64825787392493) bank3060 +3061 POINT(41.58279224866609 74.10933369951313) bank3061 +3062 POINT(39.779996787102874 74.28939511490348) bank3062 +3063 POINT(41.29723181996345 74.67690642909274) bank3063 +3064 POINT(41.55354744187099 73.32694909203909) bank3064 +3065 POINT(40.94313759079409 73.46774083663949) bank3065 +3066 POINT(41.02366234399753 74.41289718554454) bank3066 +3067 POINT(40.54573524270256 74.04067747617387) bank3067 +3068 POINT(39.911098547486176 74.15733275674998) bank3068 +3069 POINT(41.64909858653911 74.39232333507695) bank3069 +3070 POINT(40.30414317046114 74.20716405357378) bank3070 +3071 POINT(39.803857459780914 74.06511777828509) bank3071 +3072 POINT(40.21471122204922 73.6935266329406) bank3072 +3073 POINT(40.56482485251505 74.66576697963713) bank3073 +3074 POINT(40.96952842652813 73.97688971764325) bank3074 +3075 POINT(40.12379173780475 74.76272258940963) bank3075 +3076 POINT(41.28728156169507 74.95584156474513) bank3076 +3077 POINT(40.80350478220416 74.85578733894583) bank3077 +3078 POINT(40.3451672155783 73.93911767856424) bank3078 +3079 POINT(40.9678673885568 73.75175817858235) bank3079 +3080 POINT(41.62165997151092 74.52450296050881) bank3080 +3081 POINT(39.91014375420846 74.63860252393947) bank3081 +3082 POINT(40.32798267330887 73.72094877095567) bank3082 +3083 POINT(39.93039416759133 73.26299290857918) bank3083 +3084 POINT(40.10161076807904 74.0408019725517) bank3084 +3085 POINT(39.86733302065261 73.41881273514335) bank3085 +3086 POINT(39.95986576161649 73.91763943127236) bank3086 +3087 POINT(41.07178211465226 73.858158480307) bank3087 +3088 POINT(41.37355942133133 73.62123974209672) bank3088 +3089 POINT(40.511306897308735 73.54064696423276) bank3089 +3090 POINT(40.175325994017435 73.57360412331835) bank3090 +3091 POINT(39.85623899979419 73.30056291929135) bank3091 +3092 POINT(41.44487072647148 73.4805056005162) bank3092 +3093 POINT(40.85529286911631 73.56572026771092) bank3093 +3094 POINT(41.115774730764485 74.39936979954618) bank3094 +3095 POINT(40.63142623915975 73.90821244573202) bank3095 +3096 POINT(39.7577337785969 73.52466102383717) bank3096 +3097 POINT(41.52917011928034 74.7213434945281) bank3097 +3098 POINT(41.659273457253036 73.90953778912987) bank3098 +3099 POINT(40.144150727964984 74.44361206282049) bank3099 +3100 POINT(40.97839682503798 73.3407403789646) bank3100 +3101 POINT(40.896482981561 74.84228077492786) bank3101 +3102 POINT(40.5010790672527 74.26388059709525) bank3102 +3103 POINT(40.32146971640121 73.20169037175914) bank3103 +3104 POINT(40.38963701550407 73.72968382169009) bank3104 +3105 POINT(39.91022966700518 73.98628755449171) bank3105 +3106 POINT(40.12403664836228 73.24815867111897) bank3106 +3107 POINT(40.520094467764935 73.52237016399611) bank3107 +3108 POINT(41.23043215236721 74.84012486184139) bank3108 +3109 POINT(41.517310311707625 73.62162139683244) bank3109 +3110 POINT(39.81825707326007 74.45774386812879) bank3110 +3111 POINT(41.03116183278728 73.74255386130659) bank3111 +3112 POINT(40.07094672443351 74.29917201877915) bank3112 +3113 POINT(40.68810659785799 74.21981155223834) bank3113 +3114 POINT(40.06886210557279 73.46587617118658) bank3114 +3115 POINT(40.53957135164412 73.81135799619034) bank3115 +3116 POINT(40.43260994368656 74.54483879733023) bank3116 +3117 POINT(40.540578943753886 74.94557418321602) bank3117 +3118 POINT(40.82440441638053 73.43028619132042) bank3118 +3119 POINT(40.62283871235537 73.81788443720866) bank3119 +3120 POINT(41.15857577664937 74.5175577065148) bank3120 +3121 POINT(41.512652934527495 73.33414812577827) bank3121 +3122 POINT(41.0557435568928 74.19635604519885) bank3122 +3123 POINT(40.95632538180381 74.0136113219354) bank3123 +3124 POINT(41.46684518961462 74.76195578091361) bank3124 +3125 POINT(39.825977735827905 73.45486798693615) bank3125 +3126 POINT(41.24292781361135 73.30871913488824) bank3126 +3127 POINT(40.1628701699601 73.79147352830627) bank3127 +3128 POINT(40.71750055657237 74.92134885074178) bank3128 +3129 POINT(40.22907259099672 73.75369015072953) bank3129 +3130 POINT(40.88955651061696 73.1496111532682) bank3130 +3131 POINT(39.772060414167115 74.25373916364732) bank3131 +3132 POINT(40.83786271902207 73.2868315700234) bank3132 +3133 POINT(41.64677973913575 73.42772489413622) bank3133 +3134 POINT(40.87065263584301 73.86432704290944) bank3134 +3135 POINT(40.94282269585072 73.48312252093186) bank3135 +3136 POINT(40.78721470201557 74.75338176636409) bank3136 +3137 POINT(41.060071385009145 73.00813115073913) bank3137 +3138 POINT(40.043116670029335 73.76888363200304) bank3138 +3139 POINT(41.56334419262229 73.051491930991) bank3139 +3140 POINT(39.87294717803071 73.94079048150007) bank3140 +3141 POINT(41.47846221482559 74.3254409464663) bank3141 +3142 POINT(41.48510801025803 73.15431678446778) bank3142 +3143 POINT(40.72673659603849 74.43624976824243) bank3143 +3144 POINT(40.41863460245827 74.34083062718213) bank3144 +3145 POINT(39.967329664074185 73.8616433048167) bank3145 +3146 POINT(40.64821656202331 73.72353613544064) bank3146 +3147 POINT(40.1637012657343 73.61428635243259) bank3147 +3148 POINT(40.99809960322013 74.68939390654911) bank3148 +3149 POINT(41.57608629595468 74.01287551452158) bank3149 +3150 POINT(40.11923570233327 74.49027934530318) bank3150 +3151 POINT(40.32561334543032 73.76392191353872) bank3151 +3152 POINT(40.34013996755939 74.64616740485555) bank3152 +3153 POINT(39.92936487094095 74.64595730508354) bank3153 +3154 POINT(39.72076850858434 74.75314215573066) bank3154 +3155 POINT(39.85858944524378 73.57994536425917) bank3155 +3156 POINT(40.831816792712466 74.53894140291847) bank3156 +3157 POINT(40.23439408307405 74.66213907206351) bank3157 +3158 POINT(41.32290010403009 74.04145651218008) bank3158 +3159 POINT(41.44588336337371 73.63684778039448) bank3159 +3160 POINT(40.96167828631576 73.89179396992795) bank3160 +3161 POINT(40.29354213964954 73.17505538691476) bank3161 +3162 POINT(40.067802004786124 73.4787469922962) bank3162 +3163 POINT(41.216049246493505 73.08719618894847) bank3163 +3164 POINT(40.712117571509026 73.20469941913412) bank3164 +3165 POINT(40.17632034643609 74.23060167815852) bank3165 +3166 POINT(40.7475184696777 73.93333203929278) bank3166 +3167 POINT(39.80145124763138 73.07260055615117) bank3167 +3168 POINT(40.76112700853776 74.84406681431682) bank3168 +3169 POINT(41.410773290154424 74.57437434194603) bank3169 +3170 POINT(41.476164005428735 74.05300069674689) bank3170 +3171 POINT(41.44703164910622 73.93480471614588) bank3171 +3172 POINT(40.82340480738461 73.12456548027772) bank3172 +3173 POINT(39.97312037677259 74.24715593369555) bank3173 +3174 POINT(41.437696407492226 73.60435740256894) bank3174 +3175 POINT(40.67515234046613 73.35569972718474) bank3175 +3176 POINT(39.9331275355159 74.95893569770936) bank3176 +3177 POINT(39.944240109677565 73.9774634717701) bank3177 +3178 POINT(40.435799358753485 74.7625184668211) bank3178 +3179 POINT(41.49964544606777 73.52736498560323) bank3179 +3180 POINT(40.94583150791601 73.85181750615062) bank3180 +3181 POINT(40.34261219386214 73.46752750018013) bank3181 +3182 POINT(40.72100841149283 73.42548583119958) bank3182 +3183 POINT(41.246148393250024 73.41944665241193) bank3183 +3184 POINT(40.94597809074849 73.80183981783368) bank3184 +3185 POINT(40.204014338875055 73.07555655150793) bank3185 +3186 POINT(40.94081208343942 74.88963454338725) bank3186 +3187 POINT(40.78134222374277 74.79549518990865) bank3187 +3188 POINT(40.60291664368129 74.93033982334633) bank3188 +3189 POINT(41.22288473077572 74.6737461584068) bank3189 +3190 POINT(40.447323115293685 73.89534053407188) bank3190 +3191 POINT(40.46808762411795 73.25061479608145) bank3191 +3192 POINT(40.592449315508354 74.5902809975255) bank3192 +3193 POINT(41.480952824671824 73.95258854702779) bank3193 +3194 POINT(39.97074886382708 74.30124561341414) bank3194 +3195 POINT(40.53822770917549 73.2943528010536) bank3195 +3196 POINT(39.74463989052868 74.53765351837855) bank3196 +3197 POINT(40.38298706119657 74.05605890371498) bank3197 +3198 POINT(39.93240628967757 74.34139418278961) bank3198 +3199 POINT(41.18184775134603 74.93083482837298) bank3199 +3200 POINT(40.621077066870896 73.42943224757654) bank3200 +3201 POINT(40.90895531208416 73.94594021724984) bank3201 +3202 POINT(39.80249865602463 73.06484658299888) bank3202 +3203 POINT(40.67727787119632 73.73841739038073) bank3203 +3204 POINT(40.62081935442599 73.2901256539242) bank3204 +3205 POINT(40.7234375281241 73.96894935028045) bank3205 +3206 POINT(40.64855948054551 74.05462139443847) bank3206 +3207 POINT(40.22266638354846 73.60860523076127) bank3207 +3208 POINT(39.90253554276754 73.30096803232564) bank3208 +3209 POINT(40.3171078354931 74.0277906979513) bank3209 +3210 POINT(40.899532842079005 73.95333663221909) bank3210 +3211 POINT(41.11489502650449 73.76096844841747) bank3211 +3212 POINT(40.84555207451655 73.23963582405725) bank3212 +3213 POINT(41.45319762031105 73.84484827255777) bank3213 +3214 POINT(39.98970122128272 74.32167820872442) bank3214 +3215 POINT(39.83886753765161 73.74805908151109) bank3215 +3216 POINT(41.1740753890399 73.1732965372134) bank3216 +3217 POINT(41.70658082394915 73.13899602792871) bank3217 +3218 POINT(40.24438845012859 74.6699108265561) bank3218 +3219 POINT(41.28264574999484 73.44140576899485) bank3219 +3220 POINT(41.119900323533344 73.30687030631155) bank3220 +3221 POINT(40.516541459873196 74.6799121465389) bank3221 +3222 POINT(41.57079945541695 73.59513399255603) bank3222 +3223 POINT(39.79467182015949 74.64701785165035) bank3223 +3224 POINT(41.235938717483 73.37846013183314) bank3224 +3225 POINT(40.30594667464974 74.44433323828191) bank3225 +3226 POINT(39.91524891548066 74.88959232610412) bank3226 +3227 POINT(41.485202994224764 73.89967384484102) bank3227 +3228 POINT(40.90737846688211 74.21946496307396) bank3228 +3229 POINT(41.57234691430111 74.13854663433938) bank3229 +3230 POINT(41.283608995726965 74.96523406390226) bank3230 +3231 POINT(39.96154104422349 73.9176154602668) bank3231 +3232 POINT(40.435513755355885 74.56943825335937) bank3232 +3233 POINT(40.84132932498399 74.33334458793776) bank3233 +3234 POINT(40.092255898034715 73.79932270412642) bank3234 +3235 POINT(41.31668722674524 73.33410381623833) bank3235 +3236 POINT(41.42964467007711 74.15586400848242) bank3236 +3237 POINT(40.205300345489796 74.16488600944182) bank3237 +3238 POINT(40.476560942124706 73.01999668395445) bank3238 +3239 POINT(40.21081435503011 74.2151912833766) bank3239 +3240 POINT(39.969022360977 74.10717288769496) bank3240 +3241 POINT(41.088929384321574 74.387883999115) bank3241 +3242 POINT(40.88694115612119 73.63519768249267) bank3242 +3243 POINT(39.897369143001924 73.26924839660515) bank3243 +3244 POINT(41.355887036020896 74.9870672252386) bank3244 +3245 POINT(41.009357494067515 73.06150579851727) bank3245 +3246 POINT(41.03158489882982 74.28104234279144) bank3246 +3247 POINT(40.58337557487778 74.49373925719094) bank3247 +3248 POINT(40.91114554650805 74.56605838566061) bank3248 +3249 POINT(40.534907653510324 74.28299039514522) bank3249 +3250 POINT(40.76455193274626 74.2410539513519) bank3250 +3251 POINT(41.15455950140047 73.12181398633312) bank3251 +3252 POINT(40.745104456118725 74.08866790065363) bank3252 +3253 POINT(40.40130813964645 73.52070769521133) bank3253 +3254 POINT(40.02643769780958 74.04740212768833) bank3254 +3255 POINT(41.69442612404966 73.59399345100601) bank3255 +3256 POINT(41.27068701820438 75.00403558706105) bank3256 +3257 POINT(40.18097423541169 74.17378289985061) bank3257 +3258 POINT(41.026998613576936 74.47041192594845) bank3258 +3259 POINT(40.91921955512042 74.94973129811375) bank3259 +3260 POINT(41.24607008495306 73.89932088447539) bank3260 +3261 POINT(40.64397338262718 74.27214881328943) bank3261 +3262 POINT(41.602202101577596 74.23245037759136) bank3262 +3263 POINT(41.46435867941591 73.48377761316206) bank3263 +3264 POINT(40.79006836181619 74.75362924351451) bank3264 +3265 POINT(40.95749102904837 73.08721062367111) bank3265 +3266 POINT(40.58485470313023 73.41354875627611) bank3266 +3267 POINT(41.065415455350596 74.55816825022269) bank3267 +3268 POINT(41.162584383150865 73.49118362783037) bank3268 +3269 POINT(40.538346606483785 74.09792273998842) bank3269 +3270 POINT(40.15993273086495 74.3266457663683) bank3270 +3271 POINT(41.59046976394446 74.45272375897078) bank3271 +3272 POINT(40.68508690811079 74.47003729499734) bank3272 +3273 POINT(39.976863974426635 74.6301414341394) bank3273 +3274 POINT(40.09160572310883 74.45979847253038) bank3274 +3275 POINT(39.86153365050074 73.87260487011137) bank3275 +3276 POINT(41.2717938076258 73.13386977728238) bank3276 +3277 POINT(40.473726826864805 74.62637359644195) bank3277 +3278 POINT(41.25691391901494 74.31139852591497) bank3278 +3279 POINT(41.18858427566643 74.37070400176717) bank3279 +3280 POINT(39.99209628037048 73.78337867710711) bank3280 +3281 POINT(41.13003118513458 73.94182638987085) bank3281 +3282 POINT(41.129830631044335 74.63505157997848) bank3282 +3283 POINT(40.53637655529532 74.74285447943409) bank3283 +3284 POINT(39.96266809541667 74.33592154252548) bank3284 +3285 POINT(40.022927916600985 73.17799199104549) bank3285 +3286 POINT(40.433111500260125 73.76244460444757) bank3286 +3287 POINT(41.26203977379658 73.97212618315143) bank3287 +3288 POINT(40.81904744697006 74.59798577557746) bank3288 +3289 POINT(39.7224121349637 74.26366683434925) bank3289 +3290 POINT(40.7446706638922 74.85219421009147) bank3290 +3291 POINT(39.92738420193548 74.71109818465084) bank3291 +3292 POINT(39.88745610707203 73.62652184574827) bank3292 +3293 POINT(40.55787009805541 74.44877147378006) bank3293 +3294 POINT(40.70429640531321 73.79729839474142) bank3294 +3295 POINT(40.926314319688124 73.4725325313165) bank3295 +3296 POINT(40.70226033095471 74.68073553081383) bank3296 +3297 POINT(41.65432599615169 74.65959428446597) bank3297 +3298 POINT(40.25357504713334 73.05497197119865) bank3298 +3299 POINT(40.47288686735098 73.36819325912207) bank3299 +3300 POINT(41.09033804520731 74.49083434192528) bank3300 +3301 POINT(39.95650775905845 74.79461979111453) bank3301 +3302 POINT(40.91860204157761 73.30217971623549) bank3302 +3303 POINT(40.91233631850886 74.50988366286319) bank3303 +3304 POINT(40.04317708879136 73.21371519716274) bank3304 +3305 POINT(41.54620643807214 74.89297726036754) bank3305 +3306 POINT(41.21113338984403 74.92452501269365) bank3306 +3307 POINT(39.716765476470876 73.63142826557245) bank3307 +3308 POINT(40.211500400869426 73.3537748081305) bank3308 +3309 POINT(40.84396481644288 74.48820203405513) bank3309 +3310 POINT(39.77476187292163 74.56283745571265) bank3310 +3311 POINT(40.041712774631186 74.07936793018595) bank3311 +3312 POINT(40.51583350383122 73.05288152764011) bank3312 +3313 POINT(40.49040249498195 73.81142687127682) bank3313 +3314 POINT(40.41319865449681 74.87498488369981) bank3314 +3315 POINT(41.46600432668191 74.20590827212403) bank3315 +3316 POINT(40.66061258470084 74.2853424121569) bank3316 +3317 POINT(41.47510188486454 73.6175272156486) bank3317 +3318 POINT(41.47643653121452 73.50460936411119) bank3318 +3319 POINT(40.92924904914627 73.99252068026468) bank3319 +3320 POINT(40.8062521855122 73.41198496947554) bank3320 +3321 POINT(40.95555784024584 73.90410754572596) bank3321 +3322 POINT(41.03901228374006 73.51575026446419) bank3322 +3323 POINT(40.990974655863305 74.16958511328806) bank3323 +3324 POINT(39.985285604608414 73.59677860820436) bank3324 +3325 POINT(40.32708531144544 74.5082283207972) bank3325 +3326 POINT(41.30648463592483 73.6456236921048) bank3326 +3327 POINT(40.44840171001008 74.62074594339502) bank3327 +3328 POINT(40.003610776603 74.77550967967433) bank3328 +3329 POINT(40.373607460597164 74.01290500918707) bank3329 +3330 POINT(40.066985974087245 74.49146709990448) bank3330 +3331 POINT(40.50233144713512 73.62185815845741) bank3331 +3332 POINT(41.63604008521613 73.87936367351047) bank3332 +3333 POINT(41.698391482971864 73.62648656336106) bank3333 +3334 POINT(40.47350010310388 73.9919681658089) bank3334 +3335 POINT(40.11453001396335 74.54196997040302) bank3335 +3336 POINT(39.716456749351266 74.34883289023108) bank3336 +3337 POINT(41.57074762371464 74.99756081762588) bank3337 +3338 POINT(40.49601298288546 74.76350711422081) bank3338 +3339 POINT(40.075709012497576 73.30260770236015) bank3339 +3340 POINT(40.80826705861987 73.07190020709524) bank3340 +3341 POINT(39.7394053586755 73.69494734102896) bank3341 +3342 POINT(40.71854735629809 74.3058995355016) bank3342 +3343 POINT(41.43169774410918 74.39347365058352) bank3343 +3344 POINT(40.59933677411434 73.879956173533) bank3344 +3345 POINT(41.37053411870169 73.3221964530154) bank3345 +3346 POINT(40.30410082986319 74.8507241320333) bank3346 +3347 POINT(40.952273529427906 74.99121965178477) bank3347 +3348 POINT(40.42461796033566 74.28890599014477) bank3348 +3349 POINT(40.25420588049388 74.00164005703836) bank3349 +3350 POINT(40.542355912616515 73.24227005772751) bank3350 +3351 POINT(41.00464804649716 74.00588761751155) bank3351 +3352 POINT(40.97790752902171 74.07435019639797) bank3352 +3353 POINT(40.04386753196215 73.33576023803981) bank3353 +3354 POINT(40.10466625643579 73.19073254280794) bank3354 +3355 POINT(40.89520011972843 73.90406518666656) bank3355 +3356 POINT(41.41254248703417 74.84207330523358) bank3356 +3357 POINT(41.54190095590859 74.13704305428134) bank3357 +3358 POINT(40.83666731164455 74.10213888426257) bank3358 +3359 POINT(40.5913193380362 73.80444747209515) bank3359 +3360 POINT(40.3813287538494 73.51601481002379) bank3360 +3361 POINT(41.1723114536993 74.38172522381983) bank3361 +3362 POINT(41.18573975972572 74.0803484303663) bank3362 +3363 POINT(39.73291000262423 73.05412604811289) bank3363 +3364 POINT(40.27259984037114 73.80673413335609) bank3364 +3365 POINT(40.939616379706536 73.95032467223781) bank3365 +3366 POINT(39.893870905174175 74.11550124284003) bank3366 +3367 POINT(40.70395379622727 74.79194077184964) bank3367 +3368 POINT(41.08950541902303 74.56055075389851) bank3368 +3369 POINT(41.38840930266995 74.25865985546581) bank3369 +3370 POINT(39.78981584391474 73.2612835148582) bank3370 +3371 POINT(39.782258396899685 74.91190456661393) bank3371 +3372 POINT(40.52888714040058 73.53863791694779) bank3372 +3373 POINT(41.369604734297596 73.56481196591174) bank3373 +3374 POINT(39.840053695029845 73.41478637474737) bank3374 +3375 POINT(40.05096673544932 73.15155195132681) bank3375 +3376 POINT(41.71095198229636 74.58846373936025) bank3376 +3377 POINT(40.098710537324116 74.25918054666172) bank3377 +3378 POINT(40.06465059438043 74.97307638970588) bank3378 +3379 POINT(39.89895107658717 74.05985286058176) bank3379 +3380 POINT(40.031006151302435 73.04618441017838) bank3380 +3381 POINT(40.54983694110643 73.62776275840055) bank3381 +3382 POINT(40.17334084511968 73.9375316014851) bank3382 +3383 POINT(40.43687346922303 73.9676852351019) bank3383 +3384 POINT(40.921854942107274 73.25715387736263) bank3384 +3385 POINT(40.69924503785006 74.92378907934523) bank3385 +3386 POINT(40.56885587170647 74.02826868850917) bank3386 +3387 POINT(40.69407614103977 74.89456147268601) bank3387 +3388 POINT(41.71001033475638 73.98251636340893) bank3388 +3389 POINT(39.92440453041259 73.52011884643697) bank3389 +3390 POINT(40.953947971713134 74.9485956554024) bank3390 +3391 POINT(39.74849101645836 74.65036107175112) bank3391 +3392 POINT(41.32225487946989 73.1934900530296) bank3392 +3393 POINT(40.106018971063435 74.2437094178576) bank3393 +3394 POINT(40.954523619655426 73.25292196726528) bank3394 +3395 POINT(41.083223950077844 73.05286155282523) bank3395 +3396 POINT(41.3045306706486 74.79863984605987) bank3396 +3397 POINT(40.69368255255316 73.01772865989969) bank3397 +3398 POINT(39.914370814343414 73.70420590783735) bank3398 +3399 POINT(41.69827486144925 73.52787468236183) bank3399 +3400 POINT(41.01018720654561 73.84424193460703) bank3400 +3401 POINT(41.04311168280599 74.20722064960043) bank3401 +3402 POINT(40.5645013392547 73.24233393710448) bank3402 +3403 POINT(40.01855843497852 74.34009405097153) bank3403 +3404 POINT(40.714620932727065 74.73837832881924) bank3404 +3405 POINT(40.603323060182184 74.7583800435817) bank3405 +3406 POINT(41.66841480757897 73.35640222341823) bank3406 +3407 POINT(41.20975075143251 74.16404174390911) bank3407 +3408 POINT(40.825766845079706 73.6786880308435) bank3408 +3409 POINT(39.86650391007382 74.85355831057993) bank3409 +3410 POINT(41.104674932971555 73.76417537574814) bank3410 +3411 POINT(41.36889254713132 74.42209900734527) bank3411 +3412 POINT(40.783263406343565 74.89181209298276) bank3412 +3413 POINT(40.22223292569018 74.51035337952065) bank3413 +3414 POINT(39.774571943005306 73.11300828934338) bank3414 +3415 POINT(39.71443486044466 73.43509755820637) bank3415 +3416 POINT(39.96133681224393 74.6813061649709) bank3416 +3417 POINT(40.97098932057194 73.07125606264471) bank3417 +3418 POINT(39.99801467350329 74.83082273331236) bank3418 +3419 POINT(41.16290010977893 73.5961148363627) bank3419 +3420 POINT(41.085664479646354 73.95426422683526) bank3420 +3421 POINT(39.95785484289877 74.56943529264171) bank3421 +3422 POINT(40.13476042614856 74.84106240667418) bank3422 +3423 POINT(40.190925664005945 73.61348813319348) bank3423 +3424 POINT(41.29421523594636 73.69387139549201) bank3424 +3425 POINT(41.65445067792892 74.2890559936187) bank3425 +3426 POINT(40.64513921159257 74.71018186826979) bank3426 +3427 POINT(40.15609216728441 73.76158768470614) bank3427 +3428 POINT(41.601724554800654 75.00332250589145) bank3428 +3429 POINT(40.1896168952086 73.61029051164249) bank3429 +3430 POINT(40.6363709373141 74.81660209512388) bank3430 +3431 POINT(41.24317525977107 74.36623037846425) bank3431 +3432 POINT(40.96060088829058 73.92759625807987) bank3432 +3433 POINT(40.79750309952064 74.91888939839205) bank3433 +3434 POINT(39.8532640559315 74.30977387515142) bank3434 +3435 POINT(40.14541693099321 73.85434443941321) bank3435 +3436 POINT(41.39490066066039 74.24759380488516) bank3436 +3437 POINT(40.521785024070326 73.26096076977623) bank3437 +3438 POINT(40.16406415242819 73.7720617095281) bank3438 +3439 POINT(40.704121952227716 74.15773543956061) bank3439 +3440 POINT(40.79939538492015 73.55155471482298) bank3440 +3441 POINT(41.12946056730602 73.13263716243124) bank3441 +3442 POINT(41.38366007369263 74.34423472800114) bank3442 +3443 POINT(41.33255761669392 74.6090993707664) bank3443 +3444 POINT(41.15490830272952 74.05655504816761) bank3444 +3445 POINT(40.25942181440568 73.35332511318938) bank3445 +3446 POINT(41.703573380109056 73.98692288669348) bank3446 +3447 POINT(39.80143899284952 73.39146142988683) bank3447 +3448 POINT(41.27911781677529 73.76966182126817) bank3448 +3449 POINT(41.386272006079 73.38830869075802) bank3449 +3450 POINT(40.70789663864236 74.09507534503955) bank3450 +3451 POINT(39.90752484091409 74.65565282292047) bank3451 +3452 POINT(40.69572000533238 74.77117307832067) bank3452 +3453 POINT(40.929124411733575 73.94865314252053) bank3453 +3454 POINT(40.09714432109753 73.80714610386303) bank3454 +3455 POINT(40.02181748203161 74.52344826296327) bank3455 +3456 POINT(40.26024330825142 74.54713364804644) bank3456 +3457 POINT(41.00400253331855 73.54021419029708) bank3457 +3458 POINT(40.96808683058605 73.09462445907965) bank3458 +3459 POINT(40.86622930913165 74.2488197696375) bank3459 +3460 POINT(40.64371177154021 74.6141116025101) bank3460 +3461 POINT(41.16843500862167 73.55052786605765) bank3461 +3462 POINT(40.14455688798042 74.77795019235758) bank3462 +3463 POINT(40.88465005451826 74.4071298815349) bank3463 +3464 POINT(41.16745673033053 74.7219896141535) bank3464 +3465 POINT(40.50323029575341 73.61303073331558) bank3465 +3466 POINT(40.470875668286695 74.22319595668748) bank3466 +3467 POINT(39.968850675166046 74.13329348614067) bank3467 +3468 POINT(41.68838266390201 74.95461865310507) bank3468 +3469 POINT(40.66602719203593 73.65744253039006) bank3469 +3470 POINT(41.38534930083939 74.42503287832244) bank3470 +3471 POINT(39.82599268110717 74.14279278660732) bank3471 +3472 POINT(41.321770593574605 74.6748476856921) bank3472 +3473 POINT(40.77000531434342 73.31308745799174) bank3473 +3474 POINT(40.59038136477785 74.88509146814543) bank3474 +3475 POINT(40.64133588526919 73.80792674036714) bank3475 +3476 POINT(41.29225482683924 74.04127559437296) bank3476 +3477 POINT(40.90510479807927 74.62465334470478) bank3477 +3478 POINT(40.11258119499652 74.01668642018812) bank3478 +3479 POINT(40.030791254577125 74.2440860805514) bank3479 +3480 POINT(41.40980859466964 73.05392763758788) bank3480 +3481 POINT(40.06243232950159 74.21293266730918) bank3481 +3482 POINT(40.17795528590055 74.46425520032112) bank3482 +3483 POINT(40.04562548245077 74.90289949311276) bank3483 +3484 POINT(40.29852865163495 73.85738305330118) bank3484 +3485 POINT(40.22219390948682 73.91563917795139) bank3485 +3486 POINT(41.165830543640915 73.59468707722563) bank3486 +3487 POINT(40.856335720819196 73.0455250740239) bank3487 +3488 POINT(40.216713432346474 74.96792347778477) bank3488 +3489 POINT(41.52229240039708 73.93164545574332) bank3489 +3490 POINT(40.58147954209398 74.58628819147125) bank3490 +3491 POINT(41.05525219314165 74.19152864216407) bank3491 +3492 POINT(40.15390684785789 73.97875966658803) bank3492 +3493 POINT(41.289297696902565 73.43223508573678) bank3493 +3494 POINT(39.72146639139681 74.99060000691988) bank3494 +3495 POINT(40.52014469051891 74.71552637799373) bank3495 +3496 POINT(40.47180002934325 73.2700809216603) bank3496 +3497 POINT(40.61320876088016 73.63644467724691) bank3497 +3498 POINT(40.564789595471396 74.56993610924005) bank3498 +3499 POINT(41.17064839731959 73.63467397480032) bank3499 +3500 POINT(40.434079320219645 73.42803145128121) bank3500 +3501 POINT(41.27396653145132 74.80169631549307) bank3501 +3502 POINT(39.74649270788604 73.91320675528013) bank3502 +3503 POINT(40.72486880642096 73.4611395130828) bank3503 +3504 POINT(41.09951490609709 74.95505258018379) bank3504 +3505 POINT(40.5187688951126 73.01426236572985) bank3505 +3506 POINT(40.71687239676424 73.52854003178847) bank3506 +3507 POINT(41.44296100907356 73.18889741892856) bank3507 +3508 POINT(40.35840453659728 74.34022059569516) bank3508 +3509 POINT(41.520035322033735 74.09342591505458) bank3509 +3510 POINT(40.44999253010786 73.11920227226481) bank3510 +3511 POINT(40.12470898689474 74.55619143349385) bank3511 +3512 POINT(40.95604030584079 73.62135756046366) bank3512 +3513 POINT(40.92495261427803 74.63660930625414) bank3513 +3514 POINT(40.58603996068313 74.5519580783939) bank3514 +3515 POINT(40.02197481947051 74.82901356560714) bank3515 +3516 POINT(41.20660495186629 73.53374603878723) bank3516 +3517 POINT(41.11637847550167 74.94914818922791) bank3517 +3518 POINT(40.70650287821797 74.98550336156522) bank3518 +3519 POINT(40.81042286966717 73.50647033097619) bank3519 +3520 POINT(41.147070257086305 74.51100955344117) bank3520 +3521 POINT(41.155670053783695 74.43225726969843) bank3521 +3522 POINT(41.511011956765024 73.61742591221977) bank3522 +3523 POINT(39.80561261143016 74.98050827336877) bank3523 +3524 POINT(40.1026103911505 74.6424991716805) bank3524 +3525 POINT(41.51801063802732 74.21850728321023) bank3525 +3526 POINT(40.95716327628184 74.7130874335753) bank3526 +3527 POINT(40.5358936782082 74.01746171326276) bank3527 +3528 POINT(41.57771307821358 73.7867345661937) bank3528 +3529 POINT(41.29755755649014 73.17231321457797) bank3529 +3530 POINT(40.30002127049183 74.09675147907541) bank3530 +3531 POINT(41.100311141608316 74.46630277787254) bank3531 +3532 POINT(41.31546032545751 74.77121136320535) bank3532 +3533 POINT(41.31283498117322 73.01949882283643) bank3533 +3534 POINT(39.746697661910105 73.48525876387065) bank3534 +3535 POINT(40.37595366028514 74.93489217568124) bank3535 +3536 POINT(40.26596230957763 73.82293100687637) bank3536 +3537 POINT(39.74224049279336 73.57070854747492) bank3537 +3538 POINT(40.13418133693267 74.42211661581766) bank3538 +3539 POINT(41.299402008710025 74.95078323298597) bank3539 +3540 POINT(40.285414645215944 74.83734042978864) bank3540 +3541 POINT(40.91117800043674 73.15577050854928) bank3541 +3542 POINT(40.65989012448554 74.72602426795977) bank3542 +3543 POINT(41.36416711196932 74.12259614811457) bank3543 +3544 POINT(40.047627099686096 74.74035697429575) bank3544 +3545 POINT(41.14232301565388 74.570549190634) bank3545 +3546 POINT(40.11624934844068 74.13980678456265) bank3546 +3547 POINT(39.92923569284083 74.59924358099767) bank3547 +3548 POINT(41.21591716770559 74.46080634680942) bank3548 +3549 POINT(41.14681176369434 73.82655141279139) bank3549 +3550 POINT(39.75163475455513 74.2755192417953) bank3550 +3551 POINT(39.92040008872122 74.8414652287369) bank3551 +3552 POINT(40.36237009551856 73.8941222880157) bank3552 +3553 POINT(40.73588278700007 74.7331753952123) bank3553 +3554 POINT(41.411183979165706 74.42484819976632) bank3554 +3555 POINT(40.10568152415806 73.2280912377327) bank3555 +3556 POINT(40.51358959148897 73.0978012211185) bank3556 +3557 POINT(40.99253005621031 73.53294059092403) bank3557 +3558 POINT(39.84184425132188 73.01120115168827) bank3558 +3559 POINT(40.68484230464592 74.51020278154564) bank3559 +3560 POINT(40.85323170020859 73.16880833597453) bank3560 +3561 POINT(40.652825443587055 73.47435385741954) bank3561 +3562 POINT(41.61114166840182 73.35328581901987) bank3562 +3563 POINT(39.99728555342593 73.38143478482856) bank3563 +3564 POINT(41.37001667146926 73.45539135433285) bank3564 +3565 POINT(40.16859754964804 73.84702221748898) bank3565 +3566 POINT(40.38719025034355 74.9494099943249) bank3566 +3567 POINT(40.5190539844504 74.70434726179708) bank3567 +3568 POINT(39.90008156472719 74.7365400418499) bank3568 +3569 POINT(40.00254699888259 74.63337444136484) bank3569 +3570 POINT(40.170229214707106 74.71785064795792) bank3570 +3571 POINT(40.94950319220733 73.84437931271384) bank3571 +3572 POINT(41.261144200110536 73.49475512404646) bank3572 +3573 POINT(39.73988495500646 74.45173522509671) bank3573 +3574 POINT(40.8354656729375 73.11743646504789) bank3574 +3575 POINT(39.99645091011439 73.78957548023402) bank3575 +3576 POINT(41.39190164877731 73.11637725451405) bank3576 +3577 POINT(40.09028012502155 73.53902768379761) bank3577 +3578 POINT(41.680250019179184 74.9536635757217) bank3578 +3579 POINT(40.448668321473335 73.49657571593276) bank3579 +3580 POINT(41.466897488501246 73.93721882845064) bank3580 +3581 POINT(41.16954827472371 74.12161664462626) bank3581 +3582 POINT(40.91870168470805 74.37704631459154) bank3582 +3583 POINT(40.76603123502316 74.2146323298737) bank3583 +3584 POINT(41.4366737765673 74.75693406357196) bank3584 +3585 POINT(41.0748941022266 73.1580279806702) bank3585 +3586 POINT(40.16776170140727 73.96052316482178) bank3586 +3587 POINT(40.95119981195615 73.64268969263054) bank3587 +3588 POINT(40.351661319129015 74.84568293892448) bank3588 +3589 POINT(41.39072978188266 74.63932131010041) bank3589 +3590 POINT(41.417038418420695 74.56028888039413) bank3590 +3591 POINT(39.82359408926147 74.72001380021291) bank3591 +3592 POINT(41.64451451448727 73.03890099080436) bank3592 +3593 POINT(40.32936098047478 74.43160667950218) bank3593 +3594 POINT(39.763775208401306 74.06282598746351) bank3594 +3595 POINT(40.330794708577784 73.93562888914548) bank3595 +3596 POINT(40.844680991248474 73.41309803520521) bank3596 +3597 POINT(40.291508136553624 74.19445717291161) bank3597 +3598 POINT(40.0085410659198 74.8423798932117) bank3598 +3599 POINT(40.79791247929585 74.36331705845728) bank3599 +3600 POINT(40.50789114711543 73.71793349709853) bank3600 +3601 POINT(40.140114144442855 73.20998492177793) bank3601 +3602 POINT(40.94595653650518 74.17181398274685) bank3602 +3603 POINT(39.84741895829912 73.29428121869043) bank3603 +3604 POINT(40.75636833972222 73.23667019745386) bank3604 +3605 POINT(40.9826012047133 73.34598559777383) bank3605 +3606 POINT(40.7726229876172 74.13254627693132) bank3606 +3607 POINT(40.84836559289817 74.91226497406949) bank3607 +3608 POINT(41.38054644167422 74.62092947687465) bank3608 +3609 POINT(40.73341916392311 74.24403202649992) bank3609 +3610 POINT(40.30729864826285 74.7209835339972) bank3610 +3611 POINT(40.68428669606809 74.52192434134031) bank3611 +3612 POINT(41.34965850200909 74.21102860069233) bank3612 +3613 POINT(40.98237867635922 73.99988425901223) bank3613 +3614 POINT(41.10298280106355 73.6058838622239) bank3614 +3615 POINT(41.565435822478946 74.2364022614678) bank3615 +3616 POINT(41.23215233454534 74.51924277971725) bank3616 +3617 POINT(39.9727595453726 74.66819618233863) bank3617 +3618 POINT(40.90835954449056 74.0918909363405) bank3618 +3619 POINT(41.401782845672344 74.60201977008055) bank3619 +3620 POINT(41.0905640691585 73.24994593552574) bank3620 +3621 POINT(39.86113241499619 73.66750515449921) bank3621 +3622 POINT(40.474482781322315 74.36330897366358) bank3622 +3623 POINT(41.28673957716351 74.53004026438312) bank3623 +3624 POINT(41.22061276259061 74.73109915664247) bank3624 +3625 POINT(41.711243441306266 74.87724643861394) bank3625 +3626 POINT(39.98338400452571 74.31765207909764) bank3626 +3627 POINT(41.09333514160048 74.63304639590648) bank3627 +3628 POINT(40.466617027984654 74.33702219929786) bank3628 +3629 POINT(40.211504097837945 73.28309299589056) bank3629 +3630 POINT(40.904216412736425 73.5039211469625) bank3630 +3631 POINT(41.27222050132369 73.07217483053515) bank3631 +3632 POINT(40.877346509120564 74.76216777284405) bank3632 +3633 POINT(40.80477456664981 74.86123159028227) bank3633 +3634 POINT(41.40211385331065 73.67622776911358) bank3634 +3635 POINT(40.03584305202349 73.20150586793973) bank3635 +3636 POINT(40.60511376233482 73.68737987004441) bank3636 +3637 POINT(40.316897094503254 74.70576994548344) bank3637 +3638 POINT(40.712011980652214 73.11488166047626) bank3638 +3639 POINT(41.23819853995025 73.70590359123308) bank3639 +3640 POINT(40.8286308939175 74.3985893178904) bank3640 +3641 POINT(41.35718744788606 73.72301557120626) bank3641 +3642 POINT(40.07534827488751 73.46634942941796) bank3642 +3643 POINT(40.11722744387773 73.13496488926208) bank3643 +3644 POINT(41.17806748014477 74.36819121896555) bank3644 +3645 POINT(41.55502993378188 73.554847377668) bank3645 +3646 POINT(40.004000159086 73.8514275197952) bank3646 +3647 POINT(40.705978263081 73.17364033576088) bank3647 +3648 POINT(41.461086167773416 73.50545631485058) bank3648 +3649 POINT(40.740867053929925 73.3489351262408) bank3649 +3650 POINT(41.429061784663205 73.28910689913847) bank3650 +3651 POINT(40.61065098433191 74.20094266254324) bank3651 +3652 POINT(41.62129535839723 74.7857850612617) bank3652 +3653 POINT(39.958399293575496 73.79796027436699) bank3653 +3654 POINT(40.32972461106397 75.00146251597039) bank3654 +3655 POINT(41.59921110818339 74.25845985226813) bank3655 +3656 POINT(40.36159920329344 73.9846972769857) bank3656 +3657 POINT(41.33916743205518 73.45888567828504) bank3657 +3658 POINT(39.98915814522627 73.29361862883978) bank3658 +3659 POINT(40.10552895553367 74.73046403217455) bank3659 +3660 POINT(40.42388714777298 74.54325129296785) bank3660 +3661 POINT(41.52462487190256 73.18370274389183) bank3661 +3662 POINT(40.3760513908636 73.15807850862659) bank3662 +3663 POINT(40.926612546951965 74.53855194476287) bank3663 +3664 POINT(40.06942298031546 73.66009835344228) bank3664 +3665 POINT(41.58011660747818 74.56425244244727) bank3665 +3666 POINT(41.052218628104455 73.09880858270836) bank3666 +3667 POINT(39.84913333788114 74.4434072152278) bank3667 +3668 POINT(40.10834237081345 73.74249631397034) bank3668 +3669 POINT(41.53998895183741 74.68862483719116) bank3669 +3670 POINT(40.42189849962614 74.63762676738837) bank3670 +3671 POINT(40.521818524829904 74.30236186760102) bank3671 +3672 POINT(41.25968618696797 73.17520407671901) bank3672 +3673 POINT(40.94059338080467 74.03770431166238) bank3673 +3674 POINT(40.17707739432606 73.03859172285938) bank3674 +3675 POINT(39.96259123704483 74.09883903808682) bank3675 +3676 POINT(40.63071118012708 74.63630078502946) bank3676 +3677 POINT(40.90448497049905 73.4605760478478) bank3677 +3678 POINT(39.85504109922223 74.3439736407295) bank3678 +3679 POINT(40.33361538208514 74.68709688460562) bank3679 +3680 POINT(40.5673976789211 73.27732555503314) bank3680 +3681 POINT(39.724864235777744 73.47718800572723) bank3681 +3682 POINT(40.33659329138494 73.38373114475134) bank3682 +3683 POINT(41.29328114094809 74.12604093783584) bank3683 +3684 POINT(41.52883142510215 74.33340322288628) bank3684 +3685 POINT(41.427601989327655 73.91976998085757) bank3685 +3686 POINT(41.460761821663745 73.65295818315677) bank3686 +3687 POINT(39.92503110072632 73.10115969080753) bank3687 +3688 POINT(40.77515972363319 74.6383607536054) bank3688 +3689 POINT(40.17668816158511 73.28886514309158) bank3689 +3690 POINT(41.11360116879745 74.81718328919024) bank3690 +3691 POINT(41.443310479487806 73.51570853217004) bank3691 +3692 POINT(41.02825917532628 73.99624730642373) bank3692 +3693 POINT(41.39017311340994 73.09333210140515) bank3693 +3694 POINT(41.47462744318786 73.64764920795568) bank3694 +3695 POINT(40.34077870674185 73.12510404069198) bank3695 +3696 POINT(40.19652485613282 74.34480353570157) bank3696 +3697 POINT(40.630505250252156 73.12292215228848) bank3697 +3698 POINT(41.11500359531732 73.51863203385004) bank3698 +3699 POINT(40.67856391190184 74.92782049847862) bank3699 +3700 POINT(40.23348393379252 74.32558256952272) bank3700 +3701 POINT(40.23609861525579 73.48417323963413) bank3701 +3702 POINT(41.21854531635247 74.05419829352506) bank3702 +3703 POINT(40.98693649995534 74.00778089565645) bank3703 +3704 POINT(40.94272960467276 74.14903000167898) bank3704 +3705 POINT(40.40663861635559 74.86824631181771) bank3705 +3706 POINT(39.87153125793588 73.49891611404699) bank3706 +3707 POINT(41.4123057139001 73.07215980110051) bank3707 +3708 POINT(41.642060547779394 73.0842797685018) bank3708 +3709 POINT(40.286449429093615 74.23826663888954) bank3709 +3710 POINT(40.69946769864023 73.32092861100087) bank3710 +3711 POINT(40.609325966631204 73.88108289675743) bank3711 +3712 POINT(39.813118337779 73.72267379073544) bank3712 +3713 POINT(40.861434172897006 73.69016517851682) bank3713 +3714 POINT(40.91829916968614 74.24406912520318) bank3714 +3715 POINT(40.08489365020618 73.04358241485757) bank3715 +3716 POINT(40.40269268193917 74.34942705800343) bank3716 +3717 POINT(40.792089644153016 74.68900811596389) bank3717 +3718 POINT(41.68040793724739 74.19116092795365) bank3718 +3719 POINT(39.89694326202425 74.88066370775024) bank3719 +3720 POINT(41.297830821820725 73.96024360913668) bank3720 +3721 POINT(40.74856743671226 74.87381627568912) bank3721 +3722 POINT(40.6826597530821 74.49606708482263) bank3722 +3723 POINT(40.850388444868486 74.56311433940017) bank3723 +3724 POINT(40.083586289546375 74.04743613085654) bank3724 +3725 POINT(41.41784528961405 73.52293586868528) bank3725 +3726 POINT(40.27880548037009 73.33858137248623) bank3726 +3727 POINT(40.27773857862771 73.61068732039264) bank3727 +3728 POINT(40.93365808694488 73.34377761208424) bank3728 +3729 POINT(41.61059906506454 73.54316814763743) bank3729 +3730 POINT(40.621208619318224 73.94381246058539) bank3730 +3731 POINT(40.12784025841391 73.46846864287212) bank3731 +3732 POINT(41.42694422056193 74.36420661061044) bank3732 +3733 POINT(41.58690286123599 74.06365467324738) bank3733 +3734 POINT(40.009433241383704 73.73030496219228) bank3734 +3735 POINT(40.356786477799226 73.48008892038011) bank3735 +3736 POINT(41.62607218461035 73.52909192769735) bank3736 +3737 POINT(40.71612851129538 73.01568494560672) bank3737 +3738 POINT(40.05348469228834 73.82998690214625) bank3738 +3739 POINT(41.565247683288895 73.72188107564546) bank3739 +3740 POINT(39.81950000147033 74.84563684741636) bank3740 +3741 POINT(40.26482396250635 74.96878663781663) bank3741 +3742 POINT(41.57030859453706 73.53850907951058) bank3742 +3743 POINT(40.02449898660806 74.35980413018449) bank3743 +3744 POINT(40.20220619582077 73.86471164132517) bank3744 +3745 POINT(40.50566301456488 73.27581706817479) bank3745 +3746 POINT(41.10242914499348 74.30589451214153) bank3746 +3747 POINT(41.06951382576925 73.82626122706348) bank3747 +3748 POINT(39.7197557440361 73.5707833533381) bank3748 +3749 POINT(41.56995707873999 73.04230171817079) bank3749 +3750 POINT(40.03758839824645 73.85273005045413) bank3750 +3751 POINT(40.06787959861984 73.59303830993375) bank3751 +3752 POINT(41.12975004558482 73.13180745311872) bank3752 +3753 POINT(40.8123982041841 74.21774538823931) bank3753 +3754 POINT(40.37563876580113 74.39245232872148) bank3754 +3755 POINT(41.2004033082617 74.56072671689537) bank3755 +3756 POINT(40.01254441848187 74.56362532755021) bank3756 +3757 POINT(41.50164964345949 74.92594523102086) bank3757 +3758 POINT(41.66223937459017 73.53116115070029) bank3758 +3759 POINT(41.26235744394301 74.02366677544812) bank3759 +3760 POINT(41.55110918214103 74.31023220508007) bank3760 +3761 POINT(40.6940615341207 73.91443620806214) bank3761 +3762 POINT(39.866991653838966 73.43846414760021) bank3762 +3763 POINT(40.632712197111196 73.20402405586329) bank3763 +3764 POINT(41.307334499862534 73.10128318576783) bank3764 +3765 POINT(41.66900077242893 74.06041303971007) bank3765 +3766 POINT(40.09050047497799 73.77794710288978) bank3766 +3767 POINT(40.99010268045133 74.26607499415799) bank3767 +3768 POINT(40.48794815973986 74.31388865270611) bank3768 +3769 POINT(40.51116861471607 73.99749597156442) bank3769 +3770 POINT(40.70539919019314 73.98449406246343) bank3770 +3771 POINT(40.39841857590531 74.99694430832484) bank3771 +3772 POINT(40.59125841107755 74.0932603048637) bank3772 +3773 POINT(39.902759937177414 74.80876122741365) bank3773 +3774 POINT(41.122800396160116 74.25899590451563) bank3774 +3775 POINT(40.113379089973215 74.2656677132072) bank3775 +3776 POINT(39.866721623019984 74.01572158741705) bank3776 +3777 POINT(41.48557014126586 73.27170595120451) bank3777 +3778 POINT(41.66129403810719 74.90381718530293) bank3778 +3779 POINT(40.964650073380014 73.4522905011966) bank3779 +3780 POINT(40.27180983474912 73.40482270994622) bank3780 +3781 POINT(40.27395846914144 73.23234838369382) bank3781 +3782 POINT(39.828840372810824 74.35186384549823) bank3782 +3783 POINT(40.316848884397885 74.58460192120117) bank3783 +3784 POINT(40.18581291155154 73.32719885516177) bank3784 +3785 POINT(40.859002747473 74.25227667006176) bank3785 +3786 POINT(40.41634231213224 74.22270824946632) bank3786 +3787 POINT(40.845401967084165 73.513668222447) bank3787 +3788 POINT(40.94773719757295 74.14897836057902) bank3788 +3789 POINT(41.41345148622008 73.07197452045106) bank3789 +3790 POINT(40.95751188435957 74.58117491267916) bank3790 +3791 POINT(41.14915102803207 74.3836694031586) bank3791 +3792 POINT(40.1157877684752 74.59683989474364) bank3792 +3793 POINT(41.311235229403586 73.66364273461646) bank3793 +3794 POINT(40.698640501483034 74.45653839919738) bank3794 +3795 POINT(40.503122098272954 74.86615529545394) bank3795 +3796 POINT(40.35487183417164 74.6184048785881) bank3796 +3797 POINT(41.38441137717304 74.11744470049877) bank3797 +3798 POINT(40.783519202047515 73.341785314796) bank3798 +3799 POINT(40.66719775523366 74.29942549486597) bank3799 +3800 POINT(40.10714233264383 74.90060365280705) bank3800 +3801 POINT(41.49422186351625 74.11400331762027) bank3801 +3802 POINT(40.86587000754763 73.15058316121606) bank3802 +3803 POINT(40.47512536850768 74.44145216508227) bank3803 +3804 POINT(41.674251281529564 74.43234977294284) bank3804 +3805 POINT(39.957553523735086 73.07936213981642) bank3805 +3806 POINT(40.89288464933876 73.69437967130064) bank3806 +3807 POINT(40.564739556515626 73.36935943608519) bank3807 +3808 POINT(41.35529202210642 73.20255362107663) bank3808 +3809 POINT(39.760136543940355 74.1188586081792) bank3809 +3810 POINT(39.84983385674927 74.27860182731054) bank3810 +3811 POINT(41.0345965114602 74.51150416709594) bank3811 +3812 POINT(41.37994772401445 73.40671836809709) bank3812 +3813 POINT(40.195809908016074 74.36693737131654) bank3813 +3814 POINT(39.724922617754906 74.40717673846471) bank3814 +3815 POINT(40.66898701101161 73.88831748703653) bank3815 +3816 POINT(40.15421620971507 74.05997316890726) bank3816 +3817 POINT(40.61025684830044 74.13935442961133) bank3817 +3818 POINT(40.78189914280895 73.96606828033178) bank3818 +3819 POINT(40.55743254396491 74.69112428307513) bank3819 +3820 POINT(40.56739332737339 74.21107805276148) bank3820 +3821 POINT(41.11074806741042 74.56263671439824) bank3821 +3822 POINT(41.630467658889195 73.07418455950092) bank3822 +3823 POINT(40.86436587872558 73.9897932010404) bank3823 +3824 POINT(41.37290218578378 74.48698969360576) bank3824 +3825 POINT(40.187328126735636 74.12826211386512) bank3825 +3826 POINT(41.52902773154743 74.63652009751038) bank3826 +3827 POINT(40.81306454355085 73.12987530718303) bank3827 +3828 POINT(40.179571169487176 74.73934952602151) bank3828 +3829 POINT(40.38650065859477 73.67344365630885) bank3829 +3830 POINT(40.3303086382664 74.21739587104065) bank3830 +3831 POINT(40.56142343822398 73.64529700597626) bank3831 +3832 POINT(40.43463919005651 73.32321160367265) bank3832 +3833 POINT(40.34338250063669 74.38334977490831) bank3833 +3834 POINT(39.763395070670825 74.99558060203327) bank3834 +3835 POINT(41.497945774231475 74.46364741937417) bank3835 +3836 POINT(41.10077712097768 74.86418299488682) bank3836 +3837 POINT(39.797500689847425 74.04236227947366) bank3837 +3838 POINT(40.344982611398024 73.86774960081894) bank3838 +3839 POINT(40.922459822384084 74.39348749940014) bank3839 +3840 POINT(40.74301796865166 74.62133707498424) bank3840 +3841 POINT(41.42794114402308 73.69245405880949) bank3841 +3842 POINT(40.048620993603 73.93235629140062) bank3842 +3843 POINT(41.48992269318745 73.85924423743677) bank3843 +3844 POINT(41.681063205857775 73.76005053067462) bank3844 +3845 POINT(40.28115926883244 73.86562225312463) bank3845 +3846 POINT(41.45136115196783 73.40019263232148) bank3846 +3847 POINT(40.336857865951444 74.14752329735943) bank3847 +3848 POINT(39.86115578932957 74.52099561053694) bank3848 +3849 POINT(40.384428594812846 73.16194790525667) bank3849 +3850 POINT(40.437985910678606 73.43678038675039) bank3850 +3851 POINT(41.36161253315178 73.3308998663461) bank3851 +3852 POINT(41.472292894806365 73.18499060157397) bank3852 +3853 POINT(41.15911602645069 73.36022051854489) bank3853 +3854 POINT(39.853888483442745 73.13228552355994) bank3854 +3855 POINT(41.26486993310496 74.13722452743987) bank3855 +3856 POINT(40.84585323623968 74.6955078210367) bank3856 +3857 POINT(40.0896111515596 74.12205026910907) bank3857 +3858 POINT(40.83234062871864 74.31736021036764) bank3858 +3859 POINT(40.73470929552737 74.01745362291713) bank3859 +3860 POINT(39.73292124005328 73.13328177073657) bank3860 +3861 POINT(41.441715265201026 73.48444086927283) bank3861 +3862 POINT(40.65191722968448 73.4784060926433) bank3862 +3863 POINT(41.49263205761117 74.98821166853112) bank3863 +3864 POINT(40.866591352621775 73.49408118659454) bank3864 +3865 POINT(41.42289822805196 73.14067499953896) bank3865 +3866 POINT(40.7454072000517 73.9634408546553) bank3866 +3867 POINT(39.96252634795001 74.77656257847742) bank3867 +3868 POINT(41.23309636547033 74.71472765275894) bank3868 +3869 POINT(40.95660933491152 73.55174317351513) bank3869 +3870 POINT(41.14163601998692 74.33691465409764) bank3870 +3871 POINT(40.60092294218532 74.42872303192615) bank3871 +3872 POINT(40.78634485759234 74.83677543381661) bank3872 +3873 POINT(40.64614993290816 74.34387661765132) bank3873 +3874 POINT(41.38191051366654 73.08804470128722) bank3874 +3875 POINT(40.02849587928228 73.37819761782487) bank3875 +3876 POINT(40.42077772732162 74.21278683586664) bank3876 +3877 POINT(41.11580957055733 74.51710513273531) bank3877 +3878 POINT(41.10392647539099 74.3444555415297) bank3878 +3879 POINT(40.062466961188505 74.75418941484776) bank3879 +3880 POINT(41.54142908101511 74.96480857598982) bank3880 +3881 POINT(41.641873220154054 73.38992192308544) bank3881 +3882 POINT(39.884024913152345 74.13675404693673) bank3882 +3883 POINT(40.38589603015867 74.73354900145907) bank3883 +3884 POINT(40.21349938869735 74.6170784972201) bank3884 +3885 POINT(41.474335014113535 73.4053757492461) bank3885 +3886 POINT(40.68063659212189 74.78301098818737) bank3886 +3887 POINT(40.75531858085375 74.85870167543617) bank3887 +3888 POINT(40.82016099403245 73.02331550344564) bank3888 +3889 POINT(40.660454568299315 73.07014635438142) bank3889 +3890 POINT(40.31708297099557 74.32430640350809) bank3890 +3891 POINT(40.406356250734845 74.67232325455812) bank3891 +3892 POINT(41.57066431372318 74.94437600406032) bank3892 +3893 POINT(40.161109200627564 73.24192680399236) bank3893 +3894 POINT(40.328295623382644 73.19746309312573) bank3894 +3895 POINT(41.63671981616599 74.4459474871779) bank3895 +3896 POINT(40.38699572564941 73.51244484994544) bank3896 +3897 POINT(40.05879886337778 74.32512546049624) bank3897 +3898 POINT(41.266254495732014 74.46122793489835) bank3898 +3899 POINT(40.198480498903116 73.53614775419977) bank3899 +3900 POINT(40.160356024861066 74.87048243899159) bank3900 +3901 POINT(41.19251482590043 73.996006805447) bank3901 +3902 POINT(40.29657321801059 73.83052882905724) bank3902 +3903 POINT(40.606725070469246 74.41868004309306) bank3903 +3904 POINT(40.186066117980495 73.74336349509288) bank3904 +3905 POINT(40.79052468415352 74.38682805759032) bank3905 +3906 POINT(40.36606853336032 73.87797531323557) bank3906 +3907 POINT(41.62162965938744 74.99403461406183) bank3907 +3908 POINT(40.542401662844824 73.96977744215911) bank3908 +3909 POINT(41.203769940543054 73.97482903085735) bank3909 +3910 POINT(41.289913489488555 74.28299140553861) bank3910 +3911 POINT(41.49548934471566 73.37411171457121) bank3911 +3912 POINT(40.07859065108494 73.12386373065503) bank3912 +3913 POINT(41.119504205713945 73.35316266900723) bank3913 +3914 POINT(39.89258325191823 73.01623148898304) bank3914 +3915 POINT(40.27660451462552 73.9887219084781) bank3915 +3916 POINT(40.90122086073745 74.89875262158094) bank3916 +3917 POINT(41.03988182605093 74.17024100271341) bank3917 +3918 POINT(40.29290688689557 73.05593726351961) bank3918 +3919 POINT(41.69085801033463 74.77584918860336) bank3919 +3920 POINT(41.69901277887695 74.51370709086471) bank3920 +3921 POINT(40.67864787421122 74.69745568937685) bank3921 +3922 POINT(41.34709990943562 74.95160257792604) bank3922 +3923 POINT(40.611382090849965 73.47352886170198) bank3923 +3924 POINT(41.481464895384 73.87169001230453) bank3924 +3925 POINT(39.948861804721176 74.74553852134157) bank3925 +3926 POINT(40.631022991421894 74.37835716408736) bank3926 +3927 POINT(41.5178161522706 74.27953840119191) bank3927 +3928 POINT(40.45993203643785 74.15663133068446) bank3928 +3929 POINT(40.15106916061028 74.21895831124603) bank3929 +3930 POINT(41.43642840532609 73.52919263031663) bank3930 +3931 POINT(40.45153112954921 73.65150454828628) bank3931 +3932 POINT(41.2534524765664 73.0906569153761) bank3932 +3933 POINT(40.89537876160397 74.14304887233806) bank3933 +3934 POINT(40.21536725854909 73.709885869662) bank3934 +3935 POINT(40.141420461129805 73.50227124075924) bank3935 +3936 POINT(40.26774930519638 73.39549604947196) bank3936 +3937 POINT(40.479790581562206 74.36891270446763) bank3937 +3938 POINT(41.167367413172855 74.7297521448312) bank3938 +3939 POINT(40.28423345761105 74.55362923182786) bank3939 +3940 POINT(41.41270752364547 74.24245838295985) bank3940 +3941 POINT(40.69533072657061 73.45703312240369) bank3941 +3942 POINT(40.51058727870579 74.69966112585595) bank3942 +3943 POINT(40.08356139137984 74.12459324710387) bank3943 +3944 POINT(41.33828177653827 73.30897456799568) bank3944 +3945 POINT(40.43798551656581 73.39350241531703) bank3945 +3946 POINT(40.1253483273987 74.35046220201454) bank3946 +3947 POINT(41.637100530806066 73.39195241792359) bank3947 +3948 POINT(41.49083806763146 74.87048331619617) bank3948 +3949 POINT(40.7431701445022 73.90204559576698) bank3949 +3950 POINT(40.30634296748242 73.79834125523782) bank3950 +3951 POINT(40.6438892838532 73.05786329471225) bank3951 +3952 POINT(41.42539280028247 73.84356431640398) bank3952 +3953 POINT(40.17596874039508 74.0257256029911) bank3953 +3954 POINT(40.35379431155862 73.33302704224896) bank3954 +3955 POINT(41.08137591558821 73.02136801952791) bank3955 +3956 POINT(40.32501632960507 74.04524127478092) bank3956 +3957 POINT(39.74933492460195 73.15407016052814) bank3957 +3958 POINT(41.43975793941851 74.01468544728658) bank3958 +3959 POINT(41.00353034129177 73.86080963740203) bank3959 +3960 POINT(39.88805967993766 74.0301283768714) bank3960 +3961 POINT(41.61643039650696 74.93063570129993) bank3961 +3962 POINT(41.24271754662325 74.12701665138835) bank3962 +3963 POINT(41.187427223677865 74.81732275247882) bank3963 +3964 POINT(41.16736024855189 74.14596739786721) bank3964 +3965 POINT(39.94917388928718 74.6966007302029) bank3965 +3966 POINT(40.371521128260724 74.58661417985972) bank3966 +3967 POINT(41.06161917521885 73.79127980168104) bank3967 +3968 POINT(40.221112793601705 74.91267019396074) bank3968 +3969 POINT(41.173988652564056 73.37626631008254) bank3969 +3970 POINT(41.3983764773852 74.94730856503173) bank3970 +3971 POINT(40.11831204159625 73.88152669552879) bank3971 +3972 POINT(40.079429679980024 73.27619988785325) bank3972 +3973 POINT(39.807809465890955 74.05945651184838) bank3973 +3974 POINT(40.94869193608425 74.51161833249223) bank3974 +3975 POINT(40.79442920085953 73.77826419242561) bank3975 +3976 POINT(41.46274442998012 73.36987436795324) bank3976 +3977 POINT(39.93498527804458 73.03018586286295) bank3977 +3978 POINT(39.75506556447593 74.44842578302334) bank3978 +3979 POINT(41.527950601264536 73.32968515968554) bank3979 +3980 POINT(41.16735060194573 73.71124073393412) bank3980 +3981 POINT(40.02047043577028 74.76848676044786) bank3981 +3982 POINT(40.997111937190475 73.42465507388316) bank3982 +3983 POINT(41.26876996696608 73.9862213630886) bank3983 +3984 POINT(40.012865810590164 73.1584311192207) bank3984 +3985 POINT(40.65842134451001 74.18014572610785) bank3985 +3986 POINT(40.45497352873002 73.00669450489039) bank3986 +3987 POINT(40.16464134947642 73.28520036476176) bank3987 +3988 POINT(41.20524680773484 73.00791971470578) bank3988 +3989 POINT(41.44395431876004 74.07502434184646) bank3989 +3990 POINT(40.16715003307246 73.59802687841541) bank3990 +3991 POINT(40.42638724376498 74.23546913732044) bank3991 +3992 POINT(39.844593849984264 73.73961779360742) bank3992 +3993 POINT(41.14678346025801 73.08686394809875) bank3993 +3994 POINT(41.57926765262349 73.1016566044594) bank3994 +3995 POINT(40.08922517537886 73.35966142410346) bank3995 +3996 POINT(40.06315539655248 73.42701098931298) bank3996 +3997 POINT(41.05313320367739 74.66605095182628) bank3997 +3998 POINT(40.284982154726485 73.83062567510453) bank3998 +3999 POINT(40.044037060096464 74.68843846854426) bank3999 +4000 POINT(41.114152007047444 73.20129714503693) bank4000 +4001 POINT(40.04768818845573 74.97786385819994) bank4001 +4002 POINT(41.67623026094487 74.89436881285087) bank4002 +4003 POINT(40.32794946358159 74.36539606240956) bank4003 +4004 POINT(41.64395337666643 73.9926989696222) bank4004 +4005 POINT(40.94085021233894 73.07234244785121) bank4005 +4006 POINT(41.56984948772757 73.21425366081294) bank4006 +4007 POINT(39.986282950726654 73.55423070039322) bank4007 +4008 POINT(41.71241119710366 73.33363385604167) bank4008 +4009 POINT(41.68415682219296 74.23006485188438) bank4009 +4010 POINT(41.243979775072795 73.51401841988401) bank4010 +4011 POINT(41.140267930762406 73.37840848594799) bank4011 +4012 POINT(41.49445166483146 73.390509949396) bank4012 +4013 POINT(41.14555313238701 73.37147448047583) bank4013 +4014 POINT(41.208987175899836 73.85341241877418) bank4014 +4015 POINT(40.39451796546546 74.467897969086) bank4015 +4016 POINT(39.841541037160106 74.15939343276064) bank4016 +4017 POINT(41.41364023753703 73.44445359781977) bank4017 +4018 POINT(40.72166395158174 73.46294011657409) bank4018 +4019 POINT(40.72037111591868 73.57012092871044) bank4019 +4020 POINT(40.69886743213689 73.09299518516409) bank4020 +4021 POINT(40.63761630926174 73.81857518468563) bank4021 +4022 POINT(40.95336617274364 73.72230595981063) bank4022 +4023 POINT(40.49810537555953 74.58594373302944) bank4023 +4024 POINT(40.84614915940208 74.42599544313997) bank4024 +4025 POINT(40.59770089394265 74.04024672225313) bank4025 +4026 POINT(39.81978476761986 73.50126832206229) bank4026 +4027 POINT(41.08298951725717 73.0581140564245) bank4027 +4028 POINT(39.84717832206399 73.76071158072926) bank4028 +4029 POINT(39.770222955754384 74.70163539483765) bank4029 +4030 POINT(40.73547102622195 73.28734904111126) bank4030 +4031 POINT(40.884513219731424 74.58135716882352) bank4031 +4032 POINT(40.6563551133986 74.37011634219928) bank4032 +4033 POINT(40.45340709274759 73.71426989666313) bank4033 +4034 POINT(40.485577435083044 74.86766694175317) bank4034 +4035 POINT(40.877357595701824 73.62674141682696) bank4035 +4036 POINT(41.47116625423656 73.07823800407353) bank4036 +4037 POINT(41.02718996218651 73.67542856175787) bank4037 +4038 POINT(40.530026410869475 73.91646533109052) bank4038 +4039 POINT(39.73804814346219 73.902652406136) bank4039 +4040 POINT(41.39644442598639 74.6629849805727) bank4040 +4041 POINT(40.25424653593047 74.6856975738659) bank4041 +4042 POINT(40.735150442329115 73.71598312059324) bank4042 +4043 POINT(40.22763969422657 74.42894514491003) bank4043 +4044 POINT(41.708685015100734 73.59468463297789) bank4044 +4045 POINT(41.410312258335345 74.29994644999562) bank4045 +4046 POINT(41.67237802894842 73.27630544851019) bank4046 +4047 POINT(40.79109783468972 73.33858386070956) bank4047 +4048 POINT(41.44088015219547 73.73554762522876) bank4048 +4049 POINT(39.78034493909729 73.77621769045798) bank4049 +4050 POINT(40.600924345908325 73.72107245511322) bank4050 +4051 POINT(40.860644819210826 74.48682113934204) bank4051 +4052 POINT(41.17664121098757 73.18524720242264) bank4052 +4053 POINT(40.42068984055205 73.09144516939362) bank4053 +4054 POINT(41.23971744701403 73.32390054807104) bank4054 +4055 POINT(41.58314099425656 74.19986074040854) bank4055 +4056 POINT(39.74426516511312 74.40344045221863) bank4056 +4057 POINT(40.41283186857119 74.77844858896324) bank4057 +4058 POINT(41.31837645422029 73.05845466430719) bank4058 +4059 POINT(41.35582261930328 74.41156895632085) bank4059 +4060 POINT(39.77238062172505 73.73178016680383) bank4060 +4061 POINT(41.60849062961638 73.45870663824027) bank4061 +4062 POINT(41.519845228849235 73.41391408856393) bank4062 +4063 POINT(41.636587378332486 74.18350947533891) bank4063 +4064 POINT(40.86432837948367 74.69996805401884) bank4064 +4065 POINT(41.4204288622773 73.69808915873033) bank4065 +4066 POINT(40.73694363390791 74.76481399474217) bank4066 +4067 POINT(41.48453176197495 74.43763941385339) bank4067 +4068 POINT(41.321866345612364 73.72561377481848) bank4068 +4069 POINT(41.440213163247904 74.85972746500416) bank4069 +4070 POINT(39.8820105735158 74.16730668347795) bank4070 +4071 POINT(39.96030887021953 73.27461422430292) bank4071 +4072 POINT(40.4096446744913 74.15058551106102) bank4072 +4073 POINT(40.79047993949631 73.14690813995334) bank4073 +4074 POINT(40.81119692251116 73.17772714703683) bank4074 +4075 POINT(40.29121282845941 74.9650598489323) bank4075 +4076 POINT(39.79736602379925 74.284508728583) bank4076 +4077 POINT(40.02388140606184 73.15558930344581) bank4077 +4078 POINT(40.494950343435924 74.4164365547204) bank4078 +4079 POINT(40.607679339498375 73.31008401898877) bank4079 +4080 POINT(40.53709115512439 73.34289418767288) bank4080 +4081 POINT(40.84924964680597 74.05103666073428) bank4081 +4082 POINT(39.82064046790451 74.78878867532903) bank4082 +4083 POINT(40.736704351813486 73.87448559243647) bank4083 +4084 POINT(41.634753290046085 73.25462644822153) bank4084 +4085 POINT(40.05738087573465 74.99291290683804) bank4085 +4086 POINT(39.88682624189274 74.10350891356786) bank4086 +4087 POINT(39.829766090133745 73.64569911056311) bank4087 +4088 POINT(40.449350949179475 73.72130850504114) bank4088 +4089 POINT(40.4089945429431 73.74470439292084) bank4089 +4090 POINT(39.81775139906103 73.55178373217147) bank4090 +4091 POINT(39.78396790889439 74.5753105376631) bank4091 +4092 POINT(40.94192690082496 73.71360186949524) bank4092 +4093 POINT(41.068585233668095 73.64070769694894) bank4093 +4094 POINT(40.864226324014176 73.01897415648695) bank4094 +4095 POINT(40.48420590952348 74.44368952881335) bank4095 +4096 POINT(40.53309551880991 74.17707140727069) bank4096 +4097 POINT(40.73432792143345 74.0484140437059) bank4097 +4098 POINT(41.41274981338227 74.99662856701399) bank4098 +4099 POINT(40.17047533615135 73.53663292947077) bank4099 +4100 POINT(41.58112367308232 73.47920749833094) bank4100 +4101 POINT(40.71450970571766 73.71733365759886) bank4101 +4102 POINT(40.6100222845416 73.9239334329796) bank4102 +4103 POINT(41.16222055207313 74.10359680874524) bank4103 +4104 POINT(41.12358305155778 74.85643711123109) bank4104 +4105 POINT(39.91280181891133 74.07320701195702) bank4105 +4106 POINT(40.659666083365536 73.05181165372701) bank4106 +4107 POINT(41.68166901417899 74.7979505820862) bank4107 +4108 POINT(40.580793242037984 74.22525594334786) bank4108 +4109 POINT(40.811510006776736 73.57500442172785) bank4109 +4110 POINT(40.48506077093848 74.53796374840402) bank4110 +4111 POINT(40.93430934817596 73.31031113318164) bank4111 +4112 POINT(41.51306641285002 73.09442751071087) bank4112 +4113 POINT(40.82724496430624 74.9876266274827) bank4113 +4114 POINT(39.97000265100516 74.20039969459262) bank4114 +4115 POINT(40.81903012558737 74.30475951273614) bank4115 +4116 POINT(40.81805920587777 74.118922817183) bank4116 +4117 POINT(40.83434547656064 74.85996777955064) bank4117 +4118 POINT(40.32609691629842 74.4683791901618) bank4118 +4119 POINT(41.18285966547893 74.16252796599571) bank4119 +4120 POINT(41.66316389831126 73.56106815958235) bank4120 +4121 POINT(39.92548875222468 73.71361552386307) bank4121 +4122 POINT(41.69089596969809 74.1020947511924) bank4122 +4123 POINT(41.67968566322852 74.05479496319) bank4123 +4124 POINT(40.116013768162325 74.76125809931898) bank4124 +4125 POINT(41.006188212554484 74.08103201852657) bank4125 +4126 POINT(40.878014673278926 74.81995563270229) bank4126 +4127 POINT(40.68836138030681 73.98351348789753) bank4127 +4128 POINT(39.759008920196955 73.67572256138789) bank4128 +4129 POINT(41.316896080921175 74.89061823632244) bank4129 +4130 POINT(40.645034685674524 74.88859366510565) bank4130 +4131 POINT(39.89560614046248 74.19667251022891) bank4131 +4132 POINT(41.64360969257814 74.43107081725255) bank4132 +4133 POINT(41.52066394983007 73.0604703442897) bank4133 +4134 POINT(40.43973268848065 73.91633243415382) bank4134 +4135 POINT(40.28264208559062 73.59223187779014) bank4135 +4136 POINT(41.07466265026603 74.43749818545564) bank4136 +4137 POINT(41.06109714526016 73.712624932171) bank4137 +4138 POINT(41.46246802486459 74.00277078972807) bank4138 +4139 POINT(41.10063151507231 73.80830441706513) bank4139 +4140 POINT(41.70244116067672 73.73843046004366) bank4140 +4141 POINT(40.32603676077986 74.35079719764288) bank4141 +4142 POINT(41.47698642205134 73.50611605071862) bank4142 +4143 POINT(39.83311974843595 73.17920578025998) bank4143 +4144 POINT(40.00555503032143 74.91203125940746) bank4144 +4145 POINT(41.13703080395925 73.96307935190872) bank4145 +4146 POINT(40.54875402322895 73.68403272552166) bank4146 +4147 POINT(39.905154785051536 73.70833469910305) bank4147 +4148 POINT(40.4079002860853 74.33126221898124) bank4148 +4149 POINT(40.02601151815704 74.1109716471529) bank4149 +4150 POINT(40.57615617949974 74.45876459875569) bank4150 +4151 POINT(40.514192314736725 74.5828210011745) bank4151 +4152 POINT(41.4667225828415 73.58735842799537) bank4152 +4153 POINT(39.86941337479089 73.03612250706806) bank4153 +4154 POINT(40.32475702676534 73.40702072320798) bank4154 +4155 POINT(41.04696976638287 74.84932143815685) bank4155 +4156 POINT(41.401787573188976 73.28754438295984) bank4156 +4157 POINT(41.076378936382824 73.49008085380669) bank4157 +4158 POINT(41.456186988783095 74.5535243385769) bank4158 +4159 POINT(40.2078146819779 73.29144817696793) bank4159 +4160 POINT(41.58613620714147 73.6224598937666) bank4160 +4161 POINT(41.419737466179654 74.38089485554333) bank4161 +4162 POINT(40.00304950938699 74.42103085186054) bank4162 +4163 POINT(41.189021408906974 73.5883602714542) bank4163 +4164 POINT(41.69660501119981 73.86357482782881) bank4164 +4165 POINT(39.91354888779051 74.33779312188722) bank4165 +4166 POINT(40.30405053487602 74.03144677167393) bank4166 +4167 POINT(40.892299164737146 73.72892701699874) bank4167 +4168 POINT(41.55609640980496 73.38089763716675) bank4168 +4169 POINT(41.070004382670255 73.23323486022154) bank4169 +4170 POINT(41.680129135962844 73.31099602085413) bank4170 +4171 POINT(41.623620738644725 73.55348285288899) bank4171 +4172 POINT(40.06243639364911 73.28534864660561) bank4172 +4173 POINT(41.32145878012312 74.41779144283821) bank4173 +4174 POINT(39.9617597129893 73.73554961739812) bank4174 +4175 POINT(39.866754428543366 74.6181368972134) bank4175 +4176 POINT(40.05543746169073 74.65539113597949) bank4176 +4177 POINT(40.115094102976954 73.44077489415204) bank4177 +4178 POINT(40.2572799317139 73.54427359263913) bank4178 +4179 POINT(40.18771046571981 73.1788165118999) bank4179 +4180 POINT(41.22952824359143 73.82202367949755) bank4180 +4181 POINT(39.930731559340266 74.86751734263225) bank4181 +4182 POINT(39.99794431597977 73.04096426933833) bank4182 +4183 POINT(41.39940791757616 73.52796751602105) bank4183 +4184 POINT(40.85513474251441 73.53069653339207) bank4184 +4185 POINT(40.39637430660153 74.58891764086668) bank4185 +4186 POINT(40.50889363665054 73.11445797711082) bank4186 +4187 POINT(40.53257314582399 73.80336224735925) bank4187 +4188 POINT(39.90714561901054 74.80713121349852) bank4188 +4189 POINT(40.41626794851482 74.99169286018511) bank4189 +4190 POINT(41.146005088159654 73.50267092232856) bank4190 +4191 POINT(40.78791874907385 74.32515618732025) bank4191 +4192 POINT(40.70213619704646 74.5931401890012) bank4192 +4193 POINT(40.195317884665094 73.03003252932655) bank4193 +4194 POINT(40.527862572467875 73.86653753216095) bank4194 +4195 POINT(41.64567247738743 74.2290753678274) bank4195 +4196 POINT(39.95138584768983 74.90520581048703) bank4196 +4197 POINT(41.04761160192268 73.9661819295035) bank4197 +4198 POINT(40.61773421919378 74.83682058626523) bank4198 +4199 POINT(40.79297649161808 74.02688395898451) bank4199 +4200 POINT(40.575999862248544 74.76832798040519) bank4200 +4201 POINT(41.55941786325229 73.50690230792948) bank4201 +4202 POINT(41.19969587940858 73.17131765076775) bank4202 +4203 POINT(41.66592953902232 74.33769330325326) bank4203 +4204 POINT(40.27808843528826 74.84586485675025) bank4204 +4205 POINT(41.095043337274255 73.27982154354804) bank4205 +4206 POINT(40.30630010838123 74.87959807150759) bank4206 +4207 POINT(41.44881353873335 73.59870793109879) bank4207 +4208 POINT(41.46321245604207 73.60069823724879) bank4208 +4209 POINT(40.06201314027542 74.19734240141315) bank4209 +4210 POINT(40.66329917975122 73.64636410123303) bank4210 +4211 POINT(40.58977656805089 73.3481844627155) bank4211 +4212 POINT(41.638794866566414 73.08630717573983) bank4212 +4213 POINT(39.78510070716249 73.87840926210781) bank4213 +4214 POINT(40.51645114035402 74.62766856329507) bank4214 +4215 POINT(41.41077991897837 74.65771475882943) bank4215 +4216 POINT(40.06081996890123 74.41092188729154) bank4216 +4217 POINT(40.0392427081861 74.98943697345358) bank4217 +4218 POINT(41.66227768793444 74.44207137629937) bank4218 +4219 POINT(40.246724174990035 74.77451744379258) bank4219 +4220 POINT(40.06799404112311 74.62017219139709) bank4220 +4221 POINT(39.89055400777169 74.25607263092945) bank4221 +4222 POINT(40.82041463897122 73.01584108338541) bank4222 +4223 POINT(41.54136999798632 74.16720333560723) bank4223 +4224 POINT(40.548175342600445 73.42457652316345) bank4224 +4225 POINT(40.02447805328024 73.64204854252769) bank4225 +4226 POINT(40.24810930893666 74.19484563117176) bank4226 +4227 POINT(40.8178205934258 74.35647869022807) bank4227 +4228 POINT(41.290837650836615 74.20454070183997) bank4228 +4229 POINT(41.35696427617503 73.25896009218401) bank4229 +4230 POINT(40.46615340940819 73.93975863537803) bank4230 +4231 POINT(41.4354432141859 73.88103844953264) bank4231 +4232 POINT(41.161456504333884 74.21883328270499) bank4232 +4233 POINT(40.63385190542404 73.61780242242179) bank4233 +4234 POINT(40.183009577596046 74.35107561776115) bank4234 +4235 POINT(40.071314170585026 74.12367189179172) bank4235 +4236 POINT(41.48268052776872 74.99662749769325) bank4236 +4237 POINT(39.80398728965793 73.9580059302277) bank4237 +4238 POINT(40.91343180472195 73.22985199173479) bank4238 +4239 POINT(40.78815537842413 74.10832030602423) bank4239 +4240 POINT(40.35525516411718 74.4257634595972) bank4240 +4241 POINT(41.26382064354785 74.6704813143213) bank4241 +4242 POINT(41.21468931484723 73.54844552359008) bank4242 +4243 POINT(41.27397295006619 73.18357910968794) bank4243 +4244 POINT(40.661376801229686 73.3614566544086) bank4244 +4245 POINT(40.36398276867332 73.9251892166416) bank4245 +4246 POINT(40.64778088919457 74.94963533276733) bank4246 +4247 POINT(39.90511619574398 74.96334839454248) bank4247 +4248 POINT(41.50633874247915 74.7388141216633) bank4248 +4249 POINT(40.15803691259006 73.83645921956004) bank4249 +4250 POINT(41.709130039184906 73.76884972078362) bank4250 +4251 POINT(40.060258961775496 74.44261271384316) bank4251 +4252 POINT(39.96170531508939 73.55356739154091) bank4252 +4253 POINT(39.94898802509339 74.71471530783238) bank4253 +4254 POINT(41.11681541973709 73.41907902860547) bank4254 +4255 POINT(40.106124946680666 74.66701604123035) bank4255 +4256 POINT(40.04415100104596 74.15627264307054) bank4256 +4257 POINT(40.67615580337315 74.26479001063144) bank4257 +4258 POINT(41.05627133929029 73.94222784320156) bank4258 +4259 POINT(41.14858348323923 74.70705267373073) bank4259 +4260 POINT(40.51743624488867 73.80594556859324) bank4260 +4261 POINT(40.70386413094369 73.39784064285836) bank4261 +4262 POINT(40.36855454257095 73.44144000476442) bank4262 +4263 POINT(41.135306715869106 74.66698266581623) bank4263 +4264 POINT(39.997585399895776 74.06672784573844) bank4264 +4265 POINT(41.10171938216107 74.7341682617344) bank4265 +4266 POINT(40.684416762045984 73.5991076137419) bank4266 +4267 POINT(40.03333035417538 74.98928035364477) bank4267 +4268 POINT(39.878579681183275 74.28973872515454) bank4268 +4269 POINT(40.25589786215143 74.71783290777687) bank4269 +4270 POINT(41.16866924038819 74.15956371180728) bank4270 +4271 POINT(41.14513899807106 73.15216012378097) bank4271 +4272 POINT(41.36569845772725 74.892585040064) bank4272 +4273 POINT(40.84727513803209 73.7218769357489) bank4273 +4274 POINT(40.88053290569621 73.08823350864964) bank4274 +4275 POINT(40.23310068827565 73.46911765545048) bank4275 +4276 POINT(41.55001791968852 73.3280181866617) bank4276 +4277 POINT(40.308845782904 74.17963576909536) bank4277 +4278 POINT(39.925175917618326 74.23264187026639) bank4278 +4279 POINT(40.107119912415826 73.37437882808207) bank4279 +4280 POINT(40.11778444408181 73.3179175659047) bank4280 +4281 POINT(40.220100060429196 74.92148939820703) bank4281 +4282 POINT(41.26526373939276 74.77741447003199) bank4282 +4283 POINT(41.017272259079085 74.07931117713352) bank4283 +4284 POINT(40.60593960532501 73.11464230597522) bank4284 +4285 POINT(39.821419952971574 74.42875888175291) bank4285 +4286 POINT(41.44504240311167 73.35209245010594) bank4286 +4287 POINT(41.06230197644969 73.60415879615803) bank4287 +4288 POINT(40.55729533018994 74.64357681589676) bank4288 +4289 POINT(40.86557788524982 73.35049420608398) bank4289 +4290 POINT(41.104637355143375 73.22239782998165) bank4290 +4291 POINT(40.741026759992565 74.2659971530835) bank4291 +4292 POINT(40.639882401786785 73.98735007719146) bank4292 +4293 POINT(40.62323148671838 73.19769635887118) bank4293 +4294 POINT(41.537442168064274 73.90890006912907) bank4294 +4295 POINT(41.063059119129335 73.32022088579585) bank4295 +4296 POINT(40.47443484845175 73.27028614837006) bank4296 +4297 POINT(40.04084205328389 73.04209691976483) bank4297 +4298 POINT(41.28107663503962 73.38636201900643) bank4298 +4299 POINT(40.066654900714134 74.44969633733324) bank4299 +4300 POINT(39.90326798557047 73.64625276902534) bank4300 +4301 POINT(40.34183853081234 74.04650191215228) bank4301 +4302 POINT(40.92757799082261 74.69383970350455) bank4302 +4303 POINT(41.35204195248936 73.889410056676) bank4303 +4304 POINT(41.45277644998666 74.40408989006222) bank4304 +4305 POINT(39.74628831664647 73.17724261634031) bank4305 +4306 POINT(39.73288401236335 74.09371352502544) bank4306 +4307 POINT(41.63297969762919 74.08247316089985) bank4307 +4308 POINT(41.46170449325757 73.51047741053324) bank4308 +4309 POINT(41.29131731325839 73.6010917441212) bank4309 +4310 POINT(39.713276107566536 74.65690747130705) bank4310 +4311 POINT(40.04503663301696 73.7556289303876) bank4311 +4312 POINT(41.664879787568715 74.7971235732537) bank4312 +4313 POINT(40.60612458830306 73.36543989903718) bank4313 +4314 POINT(41.634811737440586 74.82898853513866) bank4314 +4315 POINT(39.86945965316315 73.84720069260842) bank4315 +4316 POINT(40.86388635049147 74.48271576269876) bank4316 +4317 POINT(40.7030184558259 74.74564884801154) bank4317 +4318 POINT(40.13456123858459 74.01242586444495) bank4318 +4319 POINT(40.05256124695326 74.85706537826172) bank4319 +4320 POINT(41.38322968772867 73.8065468751104) bank4320 +4321 POINT(41.288892095803654 74.96555347153426) bank4321 +4322 POINT(41.19578253239534 74.48746707647022) bank4322 +4323 POINT(41.54034498317299 73.5074619484195) bank4323 +4324 POINT(39.971890280227015 74.1813535421997) bank4324 +4325 POINT(41.09011851289775 74.33265647961777) bank4325 +4326 POINT(40.48371576090266 74.17156991632055) bank4326 +4327 POINT(40.740439958798554 74.55339614735647) bank4327 +4328 POINT(40.40784344002629 74.03473906839214) bank4328 +4329 POINT(40.04119326730437 73.40205570539453) bank4329 +4330 POINT(40.23443778857085 73.55449471525738) bank4330 +4331 POINT(39.94357028299818 74.32594484304816) bank4331 +4332 POINT(41.42575286302294 73.47649657792982) bank4332 +4333 POINT(41.55339257797913 74.6635607447793) bank4333 +4334 POINT(41.615267727099095 73.81982887691439) bank4334 +4335 POINT(39.92649691486349 73.5264482810761) bank4335 +4336 POINT(41.072436261327844 74.192662425142) bank4336 +4337 POINT(41.399387112427114 74.9141820267033) bank4337 +4338 POINT(41.15890215762827 73.64560492301844) bank4338 +4339 POINT(41.603611804215596 73.4417395022839) bank4339 +4340 POINT(41.38388311608348 73.70793515391664) bank4340 +4341 POINT(39.76008692730481 74.40578799673659) bank4341 +4342 POINT(40.647573586707445 73.25666318515437) bank4342 +4343 POINT(41.50893101642737 74.49744854801152) bank4343 +4344 POINT(41.023112805034565 74.05813337035269) bank4344 +4345 POINT(40.93405495563559 73.06476849150137) bank4345 +4346 POINT(41.091335901714466 73.32916135429156) bank4346 +4347 POINT(40.09151880337571 74.69097330700858) bank4347 +4348 POINT(41.53934246943134 74.13051538903561) bank4348 +4349 POINT(40.35791408559174 74.46702952212208) bank4349 +4350 POINT(40.34599589027221 74.83136926628309) bank4350 +4351 POINT(41.44263109730105 74.50523567574758) bank4351 +4352 POINT(40.88167013336929 73.07045646934533) bank4352 +4353 POINT(40.854885810349536 74.89346974185318) bank4353 +4354 POINT(41.43877323263786 74.37001544000513) bank4354 +4355 POINT(40.79582185544885 73.94933946373321) bank4355 +4356 POINT(41.51086911172707 73.88378159453016) bank4356 +4357 POINT(40.663932757113656 73.6848141498174) bank4357 +4358 POINT(41.47330383513752 73.83706503181713) bank4358 +4359 POINT(40.728944771594314 73.13075721041474) bank4359 +4360 POINT(40.10925526177099 73.34363356483094) bank4360 +4361 POINT(40.48459234189899 74.00287992737815) bank4361 +4362 POINT(41.25345845920411 73.94806151730323) bank4362 +4363 POINT(40.0940132331832 73.66129133468752) bank4363 +4364 POINT(39.75772627822146 73.79540836075894) bank4364 +4365 POINT(41.29374155805051 74.14288178371908) bank4365 +4366 POINT(41.1727134283835 73.46135052099011) bank4366 +4367 POINT(41.6978213330187 74.16578541687785) bank4367 +4368 POINT(41.62807270467586 73.11747338375332) bank4368 +4369 POINT(40.3752814382951 74.45035108930539) bank4369 +4370 POINT(40.33099253390184 73.67939402712595) bank4370 +4371 POINT(40.679859663330994 73.03479747215736) bank4371 +4372 POINT(41.273117118392165 74.76451162968254) bank4372 +4373 POINT(40.21758141691598 73.63217768572402) bank4373 +4374 POINT(40.443487295442374 73.93327671975446) bank4374 +4375 POINT(40.49159112055357 74.65171789016486) bank4375 +4376 POINT(40.85437533656841 74.76058572278457) bank4376 +4377 POINT(40.26913476812774 74.26740401197023) bank4377 +4378 POINT(40.67300999341418 73.11092350988585) bank4378 +4379 POINT(40.239153448473566 74.98713977428028) bank4379 +4380 POINT(40.78552670952661 74.54776253868799) bank4380 +4381 POINT(40.17383302157452 74.63369609756204) bank4381 +4382 POINT(39.906643455094915 73.98330888189034) bank4382 +4383 POINT(41.10910607986256 74.7254568003268) bank4383 +4384 POINT(39.87608982549478 74.1996241177147) bank4384 +4385 POINT(40.128741309386264 73.19373779418828) bank4385 +4386 POINT(40.44634846326776 74.84316992992626) bank4386 +4387 POINT(40.92439252484351 74.84138326472474) bank4387 +4388 POINT(40.88968080189572 73.49811186976581) bank4388 +4389 POINT(40.51526874554485 73.27363693604363) bank4389 +4390 POINT(41.621713888646305 73.09862024117908) bank4390 +4391 POINT(40.991148324815974 74.01321251091893) bank4391 +4392 POINT(41.14468700895138 73.0979275990433) bank4392 +4393 POINT(40.251401766616524 74.03293273513377) bank4393 +4394 POINT(40.43565386875348 73.66159056182853) bank4394 +4395 POINT(40.39516057876796 73.21399240871426) bank4395 +4396 POINT(41.58490280827501 74.98238415387343) bank4396 +4397 POINT(39.94202147718369 74.25793048446599) bank4397 +4398 POINT(40.33392062623973 74.11672847312558) bank4398 +4399 POINT(39.9750359588379 74.66445856475262) bank4399 +4400 POINT(40.79309583495234 73.23938044065001) bank4400 +4401 POINT(40.50388752986967 74.4397721387956) bank4401 +4402 POINT(39.85700020953343 73.92633536288602) bank4402 +4403 POINT(40.36642426478203 74.64663096790561) bank4403 +4404 POINT(39.84598866373334 74.74577383425206) bank4404 +4405 POINT(40.60741164919047 74.7264190319657) bank4405 +4406 POINT(40.934465789613114 73.5851728745998) bank4406 +4407 POINT(40.07293246443797 73.89605698384244) bank4407 +4408 POINT(40.24175634766425 73.39676078069334) bank4408 +4409 POINT(40.64393287413183 74.70740151195493) bank4409 +4410 POINT(41.16349850530282 74.59046448714238) bank4410 +4411 POINT(41.38949284833022 73.8047327635037) bank4411 +4412 POINT(41.40816860486581 73.64448926298617) bank4412 +4413 POINT(41.16222463718904 73.10921882071803) bank4413 +4414 POINT(40.925226985116694 74.94056401564148) bank4414 +4415 POINT(41.320166573394836 73.22967197258212) bank4415 +4416 POINT(40.041772143658065 74.27844747172406) bank4416 +4417 POINT(39.845339980274304 73.07069833612648) bank4417 +4418 POINT(40.48005542263677 74.22314124138296) bank4418 +4419 POINT(40.338735294424936 74.89995573145079) bank4419 +4420 POINT(40.08257263661537 74.26113015393454) bank4420 +4421 POINT(40.631507600542676 73.94574063736167) bank4421 +4422 POINT(41.32143740835099 74.78210059498959) bank4422 +4423 POINT(40.99418577077676 73.72604841526872) bank4423 +4424 POINT(41.42003087519685 73.53193774474907) bank4424 +4425 POINT(40.23285670260088 73.10295404150553) bank4425 +4426 POINT(41.14283189552795 74.37332373947551) bank4426 +4427 POINT(40.10839395815832 74.67701277163243) bank4427 +4428 POINT(41.05744178031561 74.24192260911641) bank4428 +4429 POINT(40.101543465963665 74.68441968458576) bank4429 +4430 POINT(41.6587005869878 74.84239467826619) bank4430 +4431 POINT(41.639491961366126 73.62186553267664) bank4431 +4432 POINT(40.43757701161501 74.41820513025401) bank4432 +4433 POINT(41.62663837023935 74.29124774583946) bank4433 +4434 POINT(39.73289534549876 74.70935597550725) bank4434 +4435 POINT(40.780031212887614 73.47185417695255) bank4435 +4436 POINT(41.59007308048556 73.95225813353451) bank4436 +4437 POINT(41.200934939123485 74.62679093400568) bank4437 +4438 POINT(40.37732560955007 73.98567053632217) bank4438 +4439 POINT(40.37407251214097 74.94803142188351) bank4439 +4440 POINT(40.01346817703272 73.80012348921151) bank4440 +4441 POINT(39.92911427032974 74.9525581893874) bank4441 +4442 POINT(41.085926698089104 73.13375786016844) bank4442 +4443 POINT(39.96637398408697 73.41400987763392) bank4443 +4444 POINT(41.587885427955506 73.68383140433097) bank4444 +4445 POINT(41.672418262135594 73.82260204245672) bank4445 +4446 POINT(39.820430941855555 74.88626061093007) bank4446 +4447 POINT(40.30553180278728 73.20530480436894) bank4447 +4448 POINT(41.088096129557016 74.43554387607372) bank4448 +4449 POINT(41.50445704832253 73.42284603917696) bank4449 +4450 POINT(39.785307858507686 74.06236744450854) bank4450 +4451 POINT(39.94392546097133 73.95072981143717) bank4451 +4452 POINT(41.62446329346394 73.9224147458399) bank4452 +4453 POINT(41.45830584070247 73.52706595593432) bank4453 +4454 POINT(40.47388972304522 74.87956488621958) bank4454 +4455 POINT(39.87028621692019 74.71893463569056) bank4455 +4456 POINT(40.29666255069649 74.51334061377817) bank4456 +4457 POINT(40.79774401522272 74.0808256673936) bank4457 +4458 POINT(39.83793845665219 74.22459347281732) bank4458 +4459 POINT(39.864355380514205 74.75226429444021) bank4459 +4460 POINT(39.987824932177816 73.37894820297271) bank4460 +4461 POINT(40.006774692427264 73.16041926230287) bank4461 +4462 POINT(40.06757416879722 74.80583967296833) bank4462 +4463 POINT(40.6829606682627 74.78625885133606) bank4463 +4464 POINT(39.81243185366718 73.8141924066854) bank4464 +4465 POINT(40.37170691095628 74.52246172311912) bank4465 +4466 POINT(40.37297686385512 74.98888158567975) bank4466 +4467 POINT(40.22648776734956 74.86552219985599) bank4467 +4468 POINT(41.364072102130834 74.6801666530719) bank4468 +4469 POINT(41.298773307987744 73.18182096553579) bank4469 +4470 POINT(41.29589419247629 73.09698995261496) bank4470 +4471 POINT(40.69555294757539 73.66939115311204) bank4471 +4472 POINT(40.98617603376279 74.68846287047624) bank4472 +4473 POINT(39.875411181371575 73.59185893864576) bank4473 +4474 POINT(39.82536252113246 74.26863578486402) bank4474 +4475 POINT(40.95855331291695 73.9635803616547) bank4475 +4476 POINT(40.421882159477896 73.23930979142295) bank4476 +4477 POINT(41.41927243217736 74.30718803580743) bank4477 +4478 POINT(40.44496710462489 73.53660165065001) bank4478 +4479 POINT(41.69318982735809 74.7336602503997) bank4479 +4480 POINT(41.106702724624505 73.26143787180736) bank4480 +4481 POINT(40.223090097605905 73.16511376417778) bank4481 +4482 POINT(41.21037906198953 74.27153045537064) bank4482 +4483 POINT(40.610144608270225 74.84368221937969) bank4483 +4484 POINT(41.6762887712629 74.6125252727457) bank4484 +4485 POINT(40.07433637541671 74.77613620648339) bank4485 +4486 POINT(40.44454827770842 73.2201590726327) bank4486 +4487 POINT(40.248039812447864 73.79021084420306) bank4487 +4488 POINT(41.30196416259557 74.36569122919354) bank4488 +4489 POINT(40.65238009748183 73.9819056551975) bank4489 +4490 POINT(40.19662194680728 73.11240031596334) bank4490 +4491 POINT(41.64298624179961 73.92446107583764) bank4491 +4492 POINT(40.38014206567014 74.13848939049987) bank4492 +4493 POINT(40.94259465728592 73.07999097059465) bank4493 +4494 POINT(40.812517417981404 73.57263007181089) bank4494 +4495 POINT(41.56930600305259 73.58048093119311) bank4495 +4496 POINT(41.408899440320546 74.77414672121668) bank4496 +4497 POINT(40.3616278080628 73.42364473242385) bank4497 +4498 POINT(41.53037394975364 73.83994886981394) bank4498 +4499 POINT(40.01090829713152 73.3983768416151) bank4499 +4500 POINT(41.15669064372827 74.65777098481122) bank4500 +4501 POINT(41.021846349007106 74.63457320947147) bank4501 +4502 POINT(41.35500772765127 74.677874894119) bank4502 +4503 POINT(41.141041376356796 73.6330082515038) bank4503 +4504 POINT(39.982793846987384 73.62907988208899) bank4504 +4505 POINT(39.95970104615894 74.10510437847418) bank4505 +4506 POINT(41.032020581220266 74.4193606185999) bank4506 +4507 POINT(41.61000152420931 74.43146557794444) bank4507 +4508 POINT(40.627669318690494 74.2213685086161) bank4508 +4509 POINT(40.277275455208546 74.13420948813321) bank4509 +4510 POINT(40.25737297117871 74.92046788438151) bank4510 +4511 POINT(40.31655205232103 74.58018119965165) bank4511 +4512 POINT(41.1883399333829 74.29054999941341) bank4512 +4513 POINT(41.40118999186207 73.67694951925417) bank4513 +4514 POINT(41.47491897343254 73.89926046117412) bank4514 +4515 POINT(41.25045819007115 74.50686371405341) bank4515 +4516 POINT(39.899299589785585 73.46499779681064) bank4516 +4517 POINT(41.49560803404675 73.81276139980481) bank4517 +4518 POINT(39.75534518705139 73.74225312503715) bank4518 +4519 POINT(40.78203082120383 73.82374907420589) bank4519 +4520 POINT(39.84845219959701 74.69365687443968) bank4520 +4521 POINT(40.81117888856481 74.67814580193182) bank4521 +4522 POINT(39.78978044391674 74.51160189262752) bank4522 +4523 POINT(41.238478478401014 74.96698464116486) bank4523 +4524 POINT(40.86187404342494 73.48557417959036) bank4524 +4525 POINT(41.678209622859754 73.06202253468484) bank4525 +4526 POINT(41.003589994609925 73.20568454827144) bank4526 +4527 POINT(39.995144173122874 73.87540052960215) bank4527 +4528 POINT(39.75015263373429 73.81896951897292) bank4528 +4529 POINT(40.21504481356554 74.71929129043484) bank4529 +4530 POINT(41.5496923157656 74.48099382608261) bank4530 +4531 POINT(39.96327130902051 73.5030586818737) bank4531 +4532 POINT(41.5046245430743 73.6017424221461) bank4532 +4533 POINT(41.590443888276084 73.84565091354439) bank4533 +4534 POINT(40.758182297549844 74.49092988053192) bank4534 +4535 POINT(41.22816156060768 73.68799257457837) bank4535 +4536 POINT(41.19020055846065 74.48977744586323) bank4536 +4537 POINT(40.406305826371614 74.85172088602772) bank4537 +4538 POINT(40.15152797875274 74.18650757094521) bank4538 +4539 POINT(40.861657451316546 74.17344426945206) bank4539 +4540 POINT(40.580023835438915 74.07588696916122) bank4540 +4541 POINT(41.089577804726 73.60494554978561) bank4541 +4542 POINT(41.50380459305533 74.46269168534839) bank4542 +4543 POINT(41.40416685660567 73.35218489821644) bank4543 +4544 POINT(40.369585691883216 73.4428811351401) bank4544 +4545 POINT(41.0511910624786 73.13592242159841) bank4545 +4546 POINT(40.3057588870049 74.42728878120442) bank4546 +4547 POINT(41.22099639320396 73.51992267540746) bank4547 +4548 POINT(41.35552053128938 74.85325239474099) bank4548 +4549 POINT(40.14158149329153 74.79424337963536) bank4549 +4550 POINT(39.8889996587822 73.74715076480496) bank4550 +4551 POINT(41.161903840921426 73.82994572735505) bank4551 +4552 POINT(40.810076851364684 74.4806028630117) bank4552 +4553 POINT(39.74310275818632 73.40597762805926) bank4553 +4554 POINT(40.62145719528766 73.21352980910223) bank4554 +4555 POINT(41.67480382649421 73.26582727241197) bank4555 +4556 POINT(39.769606020091445 73.09594817751145) bank4556 +4557 POINT(41.68302961271858 74.50047746992794) bank4557 +4558 POINT(39.94430370664727 73.64288920495754) bank4558 +4559 POINT(40.32276015980136 73.41060688920781) bank4559 +4560 POINT(40.92470933829078 74.32848548105812) bank4560 +4561 POINT(41.375296128836936 74.18552345341467) bank4561 +4562 POINT(41.550477403063766 73.83877266310255) bank4562 +4563 POINT(39.74632488846618 74.95428112864855) bank4563 +4564 POINT(41.48993629267825 73.86135727756124) bank4564 +4565 POINT(40.93664907575973 73.71852196014535) bank4565 +4566 POINT(41.187740320669576 73.27483809799523) bank4566 +4567 POINT(40.08621626125718 73.06232258431397) bank4567 +4568 POINT(40.42813974127032 73.59397058928606) bank4568 +4569 POINT(41.58582290051195 73.48400223483515) bank4569 +4570 POINT(40.203629413246894 73.29796729747294) bank4570 +4571 POINT(41.0289816484379 73.42784058801575) bank4571 +4572 POINT(40.4199026970459 73.34112006258158) bank4572 +4573 POINT(39.811257726928794 74.91377718655264) bank4573 +4574 POINT(41.07550775452243 73.82962611786606) bank4574 +4575 POINT(40.95570172539801 73.01501196458157) bank4575 +4576 POINT(40.06434479872175 74.65092932777851) bank4576 +4577 POINT(40.27653029067145 73.56602331686163) bank4577 +4578 POINT(40.67088221035923 73.67217056273185) bank4578 +4579 POINT(41.409009692335744 73.91661343411158) bank4579 +4580 POINT(41.57013637193951 73.99211703994382) bank4580 +4581 POINT(39.98506944629588 73.6230949429803) bank4581 +4582 POINT(41.65904781078321 74.2500831539452) bank4582 +4583 POINT(40.598832127782146 73.61767124575965) bank4583 +4584 POINT(41.399609202908856 74.37754049754035) bank4584 +4585 POINT(41.523614549282165 73.26548401973038) bank4585 +4586 POINT(41.34936156888876 73.90500625568284) bank4586 +4587 POINT(40.45445283313853 74.75625891685627) bank4587 +4588 POINT(41.51788926500379 74.64375668423678) bank4588 +4589 POINT(40.378191175372855 74.3773853991408) bank4589 +4590 POINT(40.73933630933877 74.71076184414694) bank4590 +4591 POINT(40.03842137014671 73.84800492515792) bank4591 +4592 POINT(40.95014459390541 73.72080615304024) bank4592 +4593 POINT(41.44712977084342 74.63209005295846) bank4593 +4594 POINT(41.04185411707885 73.2984135728801) bank4594 +4595 POINT(39.86663409404766 73.99377997724324) bank4595 +4596 POINT(40.688652306245025 74.22603471834103) bank4596 +4597 POINT(39.746877639264966 73.62274193349174) bank4597 +4598 POINT(40.27889635297877 74.1555716236068) bank4598 +4599 POINT(40.32723741881033 74.37895576313868) bank4599 +4600 POINT(41.25731517153244 74.30237444251473) bank4600 +4601 POINT(41.08687318982621 73.73147007678804) bank4601 +4602 POINT(41.323174340932106 73.30758392737182) bank4602 +4603 POINT(39.81586749315035 74.33856984283784) bank4603 +4604 POINT(40.8040111883268 73.69816044658977) bank4604 +4605 POINT(41.04409748737456 73.68924553162542) bank4605 +4606 POINT(40.22006610644826 73.98114441842378) bank4606 +4607 POINT(41.191551930149274 74.76897203138589) bank4607 +4608 POINT(40.533786921715965 74.44986710564352) bank4608 +4609 POINT(41.558949413020194 74.99599131303682) bank4609 +4610 POINT(41.569108362290706 74.74083184885) bank4610 +4611 POINT(40.133965184022486 74.15206619169858) bank4611 +4612 POINT(40.39868940931634 74.04839742939694) bank4612 +4613 POINT(40.09053638513748 73.57136709291787) bank4613 +4614 POINT(41.233626059861365 74.30424508025575) bank4614 +4615 POINT(40.63572072838652 74.79395360470828) bank4615 +4616 POINT(41.670597523665954 73.64537841059717) bank4616 +4617 POINT(40.03481686510209 73.33875999710321) bank4617 +4618 POINT(39.98379817505951 74.31513685170546) bank4618 +4619 POINT(40.60906060313137 74.56489863383652) bank4619 +4620 POINT(40.02794004103423 73.36663377886568) bank4620 +4621 POINT(40.877452610676976 74.78613307995234) bank4621 +4622 POINT(40.4738618530809 73.53459779213672) bank4622 +4623 POINT(41.45786429788321 74.61913951506743) bank4623 +4624 POINT(39.74069586811713 74.67635086578926) bank4624 +4625 POINT(39.995719909510115 74.10780329437404) bank4625 +4626 POINT(41.50554886496721 74.07240582989327) bank4626 +4627 POINT(41.42513532145251 74.85167782109717) bank4627 +4628 POINT(40.54670808156915 73.11704660608872) bank4628 +4629 POINT(39.881435085737515 73.90761651989253) bank4629 +4630 POINT(40.83858378542754 74.58489114199769) bank4630 +4631 POINT(41.49107845241526 73.65407014248825) bank4631 +4632 POINT(41.39731585600601 73.54545663627327) bank4632 +4633 POINT(39.7798898536444 74.52091239118083) bank4633 +4634 POINT(39.83171669675905 74.512775179135) bank4634 +4635 POINT(41.33380981294374 73.53615379383315) bank4635 +4636 POINT(40.48196658236838 73.20749901382479) bank4636 +4637 POINT(41.30732625069288 74.64888633409483) bank4637 +4638 POINT(41.004755891450806 74.74510971050545) bank4638 +4639 POINT(40.04554564085269 73.51069862702879) bank4639 +4640 POINT(41.66313330581128 73.90924838935243) bank4640 +4641 POINT(41.31779696092516 73.68705009465714) bank4641 +4642 POINT(40.86858043155027 73.53077816123539) bank4642 +4643 POINT(40.49311068099727 74.70501500405666) bank4643 +4644 POINT(39.882741843229304 73.57844922754616) bank4644 +4645 POINT(41.67733905822686 73.52385134927331) bank4645 +4646 POINT(41.35353305822976 74.36176503107889) bank4646 +4647 POINT(40.60950482608281 73.24539529771388) bank4647 +4648 POINT(39.89575859074029 73.17169159691761) bank4648 +4649 POINT(41.641432764432665 74.51572300895951) bank4649 +4650 POINT(41.08595540113682 74.02198351032234) bank4650 +4651 POINT(41.6509365314798 73.36938447414786) bank4651 +4652 POINT(39.894078672254686 73.47332861690766) bank4652 +4653 POINT(40.765293309976045 73.51415115541943) bank4653 +4654 POINT(40.43683001418 74.5131477702566) bank4654 +4655 POINT(41.69402860182687 74.78254249217507) bank4655 +4656 POINT(39.85832072235575 74.39004505102366) bank4656 +4657 POINT(40.80960432308299 73.91043363313275) bank4657 +4658 POINT(39.88230806210588 73.30182055449121) bank4658 +4659 POINT(40.75135751414255 73.4658500925201) bank4659 +4660 POINT(40.904834557791574 73.76609985284) bank4660 +4661 POINT(40.74598813705048 74.32702239274948) bank4661 +4662 POINT(41.665312464722376 73.6319877858545) bank4662 +4663 POINT(40.74888809791391 73.17622301943639) bank4663 +4664 POINT(41.37531780765789 74.4949627840069) bank4664 +4665 POINT(40.072455189931965 74.61881992900737) bank4665 +4666 POINT(41.335115598728144 73.95976417316255) bank4666 +4667 POINT(40.60449935300513 73.94765495487843) bank4667 +4668 POINT(40.85101809028963 73.04726460839805) bank4668 +4669 POINT(41.35832037467795 74.5643521043196) bank4669 +4670 POINT(40.98354763334251 74.81925930772329) bank4670 +4671 POINT(40.30387072359786 73.83375801637534) bank4671 +4672 POINT(40.68268334967668 73.43037558360089) bank4672 +4673 POINT(41.168968831059395 73.24698047673137) bank4673 +4674 POINT(40.88480330768185 74.17747099184452) bank4674 +4675 POINT(40.76255073612109 74.12802570579359) bank4675 +4676 POINT(39.82161303119386 73.46947286233723) bank4676 +4677 POINT(39.88257615140825 73.63078820931905) bank4677 +4678 POINT(40.91436209844173 74.46381855370224) bank4678 +4679 POINT(40.45491424969815 73.48534033203444) bank4679 +4680 POINT(39.89780445980155 73.70069762671943) bank4680 +4681 POINT(41.2037935739129 73.68776119403599) bank4681 +4682 POINT(41.40143178330231 74.12949261228214) bank4682 +4683 POINT(39.73516243956373 73.35067832207733) bank4683 +4684 POINT(39.99318816043802 74.13635869553768) bank4684 +4685 POINT(40.919109675322694 74.98304887653543) bank4685 +4686 POINT(41.09175443961099 73.98189911278911) bank4686 +4687 POINT(41.64536103848822 74.22397845404744) bank4687 +4688 POINT(40.67816519603533 73.2961890213665) bank4688 +4689 POINT(40.8698442840248 74.35587007897195) bank4689 +4690 POINT(40.88895099981248 74.22699581234858) bank4690 +4691 POINT(41.15141718557076 73.07866907509988) bank4691 +4692 POINT(40.23845838544502 74.37917471768134) bank4692 +4693 POINT(39.762982852104464 74.6217788299351) bank4693 +4694 POINT(41.43752880679101 73.91229444697125) bank4694 +4695 POINT(40.22358025250196 73.22075126662955) bank4695 +4696 POINT(40.492107021680596 73.15639036891353) bank4696 +4697 POINT(40.2969218000188 73.63431145529883) bank4697 +4698 POINT(40.68704646267548 73.72761675061885) bank4698 +4699 POINT(41.353542389858795 73.99839125269989) bank4699 +4700 POINT(40.31895067618758 73.64408974029897) bank4700 +4701 POINT(40.30682948842727 74.77131752720338) bank4701 +4702 POINT(41.011114898615624 74.564359283817) bank4702 +4703 POINT(40.451530037824284 74.60858011600295) bank4703 +4704 POINT(39.75420527174934 74.43232060115848) bank4704 +4705 POINT(40.94257056384632 74.87462238122455) bank4705 +4706 POINT(39.959703600791606 74.26194880700503) bank4706 +4707 POINT(40.75710214835256 73.08412852648843) bank4707 +4708 POINT(40.89311907274702 74.96810440656523) bank4708 +4709 POINT(40.07483122018512 74.98872561965257) bank4709 +4710 POINT(41.42277881817719 74.984490985164) bank4710 +4711 POINT(41.58801415848673 73.08575197055394) bank4711 +4712 POINT(40.13187653404951 74.50706131866242) bank4712 +4713 POINT(41.709188666425035 73.15601118607847) bank4713 +4714 POINT(40.86176102115832 73.59217784302226) bank4714 +4715 POINT(39.71306353212336 74.79676189931487) bank4715 +4716 POINT(41.01731970167493 74.94778009123917) bank4716 +4717 POINT(40.53504683812039 73.82069062788904) bank4717 +4718 POINT(40.65509828645638 74.61920512249326) bank4718 +4719 POINT(40.978507783657776 74.08680318150489) bank4719 +4720 POINT(41.57134923476714 73.63342093274403) bank4720 +4721 POINT(40.179499108599444 73.46250755762588) bank4721 +4722 POINT(40.172393916291824 74.90999875281233) bank4722 +4723 POINT(40.61696249226966 74.78042091726495) bank4723 +4724 POINT(41.43284662013242 73.41593677059105) bank4724 +4725 POINT(40.52399157112415 73.30981844596059) bank4725 +4726 POINT(41.06066189333235 73.00656895468768) bank4726 +4727 POINT(39.873374994169 74.93534540239358) bank4727 +4728 POINT(41.51822897315524 74.83860192142733) bank4728 +4729 POINT(41.1644041766852 74.26177753169404) bank4729 +4730 POINT(41.60140629300391 73.33196513091663) bank4730 +4731 POINT(40.61196737294951 73.18001600721055) bank4731 +4732 POINT(41.64269488432963 74.14009433974594) bank4732 +4733 POINT(39.78423540629551 74.39603855890836) bank4733 +4734 POINT(41.28895974673089 74.00594434908643) bank4734 +4735 POINT(41.2884398387394 74.69077282411583) bank4735 +4736 POINT(40.44715765346027 74.67913678539057) bank4736 +4737 POINT(40.046694123162155 74.21950609947744) bank4737 +4738 POINT(41.3533353048324 73.5713178131306) bank4738 +4739 POINT(40.12445826026088 73.39475202055067) bank4739 +4740 POINT(41.171474798962926 74.2050668706759) bank4740 +4741 POINT(40.785650956275624 73.98783601580368) bank4741 +4742 POINT(39.90449469114282 74.13706859070122) bank4742 +4743 POINT(41.424338801194146 74.836427385805) bank4743 +4744 POINT(40.2564305863742 74.45704441109993) bank4744 +4745 POINT(40.891927839847156 74.89456418397543) bank4745 +4746 POINT(41.071929452015254 74.52164922314489) bank4746 +4747 POINT(40.117024301219594 74.27637240286958) bank4747 +4748 POINT(41.4292825822325 73.20936398027602) bank4748 +4749 POINT(40.96191328791855 73.84194811093275) bank4749 +4750 POINT(41.62914055731552 74.11588059095027) bank4750 +4751 POINT(39.75638513573116 73.1266340164884) bank4751 +4752 POINT(40.46336355271034 74.13401384401747) bank4752 +4753 POINT(40.56496628723125 73.01686731085009) bank4753 +4754 POINT(41.378625285668626 74.95161269671897) bank4754 +4755 POINT(41.32248451151701 73.86392004194425) bank4755 +4756 POINT(41.08169174998457 74.09593201894336) bank4756 +4757 POINT(39.81055315593845 73.35543521367934) bank4757 +4758 POINT(40.37617000249816 74.97666967522734) bank4758 +4759 POINT(41.26853759981316 74.37612303979672) bank4759 +4760 POINT(40.893638786119226 74.02231515671212) bank4760 +4761 POINT(41.11624053987458 74.69007323237622) bank4761 +4762 POINT(39.877782338770054 74.89438261063572) bank4762 +4763 POINT(40.31304488010181 73.40642605000161) bank4763 +4764 POINT(40.689479647400994 74.91442304117815) bank4764 +4765 POINT(39.9654527848454 74.61423908990506) bank4765 +4766 POINT(40.76241139001766 74.56158507175142) bank4766 +4767 POINT(41.438079811887945 73.08524267773358) bank4767 +4768 POINT(41.696186196240625 73.10646546504364) bank4768 +4769 POINT(40.785904862752155 73.83078424419877) bank4769 +4770 POINT(40.434301018772885 74.74733670048614) bank4770 +4771 POINT(41.08440240533965 73.36505943662684) bank4771 +4772 POINT(41.37506150077453 73.0554806911508) bank4772 +4773 POINT(41.43421786645944 73.0397889939583) bank4773 +4774 POINT(39.76322936495515 74.9338275522914) bank4774 +4775 POINT(41.57317428238052 73.89451048537099) bank4775 +4776 POINT(40.96097707064687 74.35199433358811) bank4776 +4777 POINT(41.012024811515616 74.44997989325225) bank4777 +4778 POINT(41.31847042526918 73.53307809303465) bank4778 +4779 POINT(40.07428728146179 73.43158527222563) bank4779 +4780 POINT(41.68410851704544 73.66160008378057) bank4780 +4781 POINT(40.702087787038394 74.2006511474988) bank4781 +4782 POINT(39.97758082796749 73.75043612998243) bank4782 +4783 POINT(41.448474829826814 74.4100372366729) bank4783 +4784 POINT(41.09015962281088 74.88969799630951) bank4784 +4785 POINT(41.700687101319744 73.19558103929481) bank4785 +4786 POINT(40.46592577556932 73.9914938388338) bank4786 +4787 POINT(39.83624522679979 74.64007039258146) bank4787 +4788 POINT(40.66022549533246 74.22407240903354) bank4788 +4789 POINT(41.33400395023846 74.26768923737743) bank4789 +4790 POINT(40.20080609297789 74.60634494128304) bank4790 +4791 POINT(40.583447883256184 73.6793369544971) bank4791 +4792 POINT(40.711304233466954 74.20114176981849) bank4792 +4793 POINT(40.926609175499124 74.046602189801) bank4793 +4794 POINT(41.64572531317951 73.91795965710419) bank4794 +4795 POINT(40.422919201584484 73.92806107814786) bank4795 +4796 POINT(40.166353230048614 74.2235017833145) bank4796 +4797 POINT(41.28513428887479 74.63729733651415) bank4797 +4798 POINT(40.38207499569779 73.43175366627852) bank4798 +4799 POINT(40.80998075537708 73.27790636940391) bank4799 +4800 POINT(40.904860531492076 73.87174720247934) bank4800 +4801 POINT(40.851919979370706 74.96503016724165) bank4801 +4802 POINT(41.16294063440737 73.17320397307134) bank4802 +4803 POINT(39.82390477403266 74.57110993307236) bank4803 +4804 POINT(40.42777908445258 74.56152576727555) bank4804 +4805 POINT(40.21629556785759 74.05679859654269) bank4805 +4806 POINT(41.07949059590142 73.52219855103998) bank4806 +4807 POINT(39.86124015935452 74.6608695183903) bank4807 +4808 POINT(41.68190811702318 74.82230344862782) bank4808 +4809 POINT(41.33115022860609 74.23292750181284) bank4809 +4810 POINT(41.19286939565865 73.27857824424918) bank4810 +4811 POINT(40.37755959752482 73.75036336004668) bank4811 +4812 POINT(40.68922160987266 74.83941051844712) bank4812 +4813 POINT(40.996767089214615 74.58668376224661) bank4813 +4814 POINT(39.80638342423519 73.56190908524418) bank4814 +4815 POINT(41.14030554090728 74.29094875771229) bank4815 +4816 POINT(41.403407053318894 74.11157674670913) bank4816 +4817 POINT(40.38559666483825 73.13939578448303) bank4817 +4818 POINT(40.86038827791084 73.5899502715564) bank4818 +4819 POINT(41.1537383771945 73.17774750173508) bank4819 +4820 POINT(40.92238446901229 74.51981282282141) bank4820 +4821 POINT(41.28825855076498 73.7532142506642) bank4821 +4822 POINT(40.4989427405587 74.26779049294865) bank4822 +4823 POINT(41.49792709614226 73.27080615430069) bank4823 +4824 POINT(41.36475363861225 73.7263199114087) bank4824 +4825 POINT(41.541863936121146 73.35020024131875) bank4825 +4826 POINT(39.99105712004156 74.1852236264771) bank4826 +4827 POINT(41.37262489471608 73.10004158419784) bank4827 +4828 POINT(40.1862184254805 73.43335685000783) bank4828 +4829 POINT(40.53677238812339 74.34390797105483) bank4829 +4830 POINT(41.61472890488156 73.7658127616651) bank4830 +4831 POINT(40.68633449482437 74.73231408517427) bank4831 +4832 POINT(40.928303668150605 73.6381916504176) bank4832 +4833 POINT(40.84107116934131 73.73985932275097) bank4833 +4834 POINT(41.66363912987397 74.2039103804371) bank4834 +4835 POINT(41.03499271944305 73.77884306408636) bank4835 +4836 POINT(41.10345741220487 73.14377562594785) bank4836 +4837 POINT(40.716058040510994 73.47739451486524) bank4837 +4838 POINT(40.28530483115855 73.81004822178649) bank4838 +4839 POINT(41.655535595764206 74.44839859073457) bank4839 +4840 POINT(41.134882754893866 74.1238349627549) bank4840 +4841 POINT(40.37965444954566 74.09366423794641) bank4841 +4842 POINT(40.14807955386595 74.66172012485728) bank4842 +4843 POINT(41.32279974632513 74.5029558245333) bank4843 +4844 POINT(41.00809123678471 74.95889509015343) bank4844 +4845 POINT(39.976185596485195 73.49311941711743) bank4845 +4846 POINT(40.05695994160735 74.31049591748263) bank4846 +4847 POINT(41.259009550462515 74.41701202134999) bank4847 +4848 POINT(40.39117929871901 74.46027326031911) bank4848 +4849 POINT(41.11897747042058 73.20820163328803) bank4849 +4850 POINT(39.88870636925086 75.00555330906187) bank4850 +4851 POINT(40.3201214397953 74.54913837997248) bank4851 +4852 POINT(40.80522640790703 74.34830153676376) bank4852 +4853 POINT(40.107375412484416 74.59306442112363) bank4853 +4854 POINT(40.77305549287805 74.27975148836475) bank4854 +4855 POINT(40.133877759658226 73.86070932887735) bank4855 +4856 POINT(41.66789759018512 73.9167823069456) bank4856 +4857 POINT(39.966419779026644 74.28873302136688) bank4857 +4858 POINT(41.11506730753005 74.40195140544293) bank4858 +4859 POINT(40.47213317531261 73.75570941918392) bank4859 +4860 POINT(39.78454903015223 74.55977698676513) bank4860 +4861 POINT(40.95868323267114 73.50734815604591) bank4861 +4862 POINT(40.475994347589705 73.47266289363189) bank4862 +4863 POINT(40.97878890354324 73.43480781536518) bank4863 +4864 POINT(40.445428946660506 74.30465748868701) bank4864 +4865 POINT(41.296152968316484 74.55140635013454) bank4865 +4866 POINT(40.28301260839324 74.3170592217225) bank4866 +4867 POINT(40.35203834827446 74.16489842323611) bank4867 +4868 POINT(40.02220177298952 73.97136555476278) bank4868 +4869 POINT(41.04917824131877 74.24398321550545) bank4869 +4870 POINT(41.51115085301756 73.60491245636116) bank4870 +4871 POINT(40.19254748440604 73.6819916019974) bank4871 +4872 POINT(41.4239333833 73.6730070148381) bank4872 +4873 POINT(40.91635314308847 74.44106621982633) bank4873 +4874 POINT(39.81002593377086 73.63823249270037) bank4874 +4875 POINT(40.818464180071736 73.15192704188983) bank4875 +4876 POINT(40.43998020968717 74.635263862283) bank4876 +4877 POINT(41.08357826389779 73.33897725266334) bank4877 +4878 POINT(40.526363762210785 74.27261909481248) bank4878 +4879 POINT(41.23551188768443 73.56611359911702) bank4879 +4880 POINT(40.06563377720475 73.99822923782965) bank4880 +4881 POINT(41.66665707777551 74.18228609736711) bank4881 +4882 POINT(41.096311447131306 74.75026964412949) bank4882 +4883 POINT(40.299911549309144 74.8157763362684) bank4883 +4884 POINT(41.290904869102995 73.04922432426334) bank4884 +4885 POINT(40.58127489087983 74.00046208777653) bank4885 +4886 POINT(40.8162994751816 73.33690314053801) bank4886 +4887 POINT(40.26602697425457 74.5444683598762) bank4887 +4888 POINT(41.12823637530436 74.31327241473721) bank4888 +4889 POINT(41.19662428459195 74.03359028181366) bank4889 +4890 POINT(39.93363242768048 73.91066356288526) bank4890 +4891 POINT(41.67410950883662 74.72846894508686) bank4891 +4892 POINT(40.3917576480356 73.27893539145404) bank4892 +4893 POINT(41.3725017280309 73.90920943732313) bank4893 +4894 POINT(40.81485108397873 74.76977652928377) bank4894 +4895 POINT(40.45916727003408 74.71657594264404) bank4895 +4896 POINT(39.80438511689104 73.81469373540402) bank4896 +4897 POINT(41.34981892809733 73.84064107571729) bank4897 +4898 POINT(40.016636350645356 74.28855026052531) bank4898 +4899 POINT(41.66359359092445 74.97217443787126) bank4899 +4900 POINT(41.31360005340847 74.696705540414) bank4900 +4901 POINT(40.592664051275435 74.50027539284788) bank4901 +4902 POINT(40.42207783389013 73.74125434079599) bank4902 +4903 POINT(41.13307117055128 74.79253969617807) bank4903 +4904 POINT(40.93045665227939 74.84440821957155) bank4904 +4905 POINT(40.105442730692886 73.52170115209871) bank4905 +4906 POINT(41.08200443912869 74.61018624654527) bank4906 +4907 POINT(41.13006903026979 74.62638917555557) bank4907 +4908 POINT(39.87576693299696 73.41196105753771) bank4908 +4909 POINT(41.24057685626869 74.58342517783012) bank4909 +4910 POINT(39.80608920235727 74.61769751952203) bank4910 +4911 POINT(40.10490951187584 73.07894313540606) bank4911 +4912 POINT(41.39286385994922 73.95158621261771) bank4912 +4913 POINT(40.1813292980479 73.60837724757455) bank4913 +4914 POINT(39.77224182367283 74.583826487144) bank4914 +4915 POINT(40.71700866656065 74.01400487568627) bank4915 +4916 POINT(41.06004962837225 73.56045937121587) bank4916 +4917 POINT(40.5688869383162 74.97109891839935) bank4917 +4918 POINT(41.01376658419851 74.22761364933575) bank4918 +4919 POINT(41.20911671621671 73.73505312043564) bank4919 +4920 POINT(41.569186741311576 73.70263121004255) bank4920 +4921 POINT(41.56524155981895 73.7662145040542) bank4921 +4922 POINT(39.81200941515505 73.9442338632374) bank4922 +4923 POINT(40.02190785205036 74.81019544206295) bank4923 +4924 POINT(41.29946930660392 74.05974264656159) bank4924 +4925 POINT(39.71754324022392 73.68907232712519) bank4925 +4926 POINT(39.97285536274297 74.96087573094425) bank4926 +4927 POINT(40.1306345606221 74.65628143582285) bank4927 +4928 POINT(39.726594264568156 73.71188962564119) bank4928 +4929 POINT(40.79442862393978 74.7496842195626) bank4929 +4930 POINT(41.65114595274698 74.61300918552497) bank4930 +4931 POINT(41.50071268434873 73.30754359029433) bank4931 +4932 POINT(40.13738028132842 74.08875240944035) bank4932 +4933 POINT(41.143728866362544 74.35757620027395) bank4933 +4934 POINT(41.12410087361378 73.32363032519228) bank4934 +4935 POINT(39.79335019597596 74.44996747468008) bank4935 +4936 POINT(40.58791092158626 73.88831808947833) bank4936 +4937 POINT(39.75489478986707 74.91252789608181) bank4937 +4938 POINT(40.706620373325016 73.23252039851066) bank4938 +4939 POINT(41.401143305826494 73.40467585705095) bank4939 +4940 POINT(40.00434268116365 73.49948196128318) bank4940 +4941 POINT(40.038942725443995 74.77880195517076) bank4941 +4942 POINT(41.511522126360624 74.56888009015229) bank4942 +4943 POINT(41.438467842397365 74.20272117441495) bank4943 +4944 POINT(41.305224588460526 74.40357460838803) bank4944 +4945 POINT(40.166667609232036 74.66324694939641) bank4945 +4946 POINT(40.45390898609287 73.1034367092204) bank4946 +4947 POINT(40.97246378607704 73.6649500285649) bank4947 +4948 POINT(39.82178583993129 73.63732335677227) bank4948 +4949 POINT(39.77052547997361 74.57562420204133) bank4949 +4950 POINT(40.58767578293183 73.81617175498276) bank4950 +4951 POINT(41.269765176878046 73.88497888746461) bank4951 +4952 POINT(40.934834190386525 73.74370216021184) bank4952 +4953 POINT(40.67082575683552 73.49036739342955) bank4953 +4954 POINT(40.763667237236696 74.52195379907229) bank4954 +4955 POINT(40.923297306992005 73.58234317807975) bank4955 +4956 POINT(41.06603238778231 74.90995771667059) bank4956 +4957 POINT(39.76565500613058 73.07297896570019) bank4957 +4958 POINT(39.86807171989324 74.19809988691811) bank4958 +4959 POINT(41.34958531828359 73.76763363537717) bank4959 +4960 POINT(40.53353439015719 74.21431078738632) bank4960 +4961 POINT(41.007307487064416 74.07117494013652) bank4961 +4962 POINT(39.78055657409495 73.278792019658) bank4962 +4963 POINT(41.019641008792135 74.6340892429239) bank4963 +4964 POINT(41.57706580143397 73.21276319618752) bank4964 +4965 POINT(40.655926857698326 73.03959305926786) bank4965 +4966 POINT(41.5081585765992 74.25659413798945) bank4966 +4967 POINT(39.804517372233484 73.75776619596171) bank4967 +4968 POINT(40.49985557671463 73.68091770632438) bank4968 +4969 POINT(41.09306250061873 74.17076127664679) bank4969 +4970 POINT(40.21257299155934 74.13392200690325) bank4970 +4971 POINT(41.50986688178939 74.71836622922481) bank4971 +4972 POINT(40.837125683054296 73.20568178609741) bank4972 +4973 POINT(39.76142849557098 73.48640462059902) bank4973 +4974 POINT(40.54048311981169 73.46961815104524) bank4974 +4975 POINT(40.52355373594773 74.56242425209456) bank4975 +4976 POINT(41.37370156719129 74.13357616844662) bank4976 +4977 POINT(40.987447281882275 73.96826645261386) bank4977 +4978 POINT(40.71160684376353 74.79030216649494) bank4978 +4979 POINT(40.383401809321825 73.34678616721466) bank4979 +4980 POINT(41.31849271226502 74.39935514963535) bank4980 +4981 POINT(41.426035247728386 74.53846787696463) bank4981 +4982 POINT(40.35143195403995 74.3315300530967) bank4982 +4983 POINT(40.53520015448257 74.4784920038952) bank4983 +4984 POINT(40.94199266401453 74.51906367063708) bank4984 +4985 POINT(40.42643002442091 73.20527410275952) bank4985 +4986 POINT(40.01680376166759 74.30559407669689) bank4986 +4987 POINT(41.036305100355484 73.41734965053878) bank4987 +4988 POINT(41.66501048833454 73.82963510117835) bank4988 +4989 POINT(41.53792015332252 74.03661402818136) bank4989 +4990 POINT(40.7979694764754 73.14885248763086) bank4990 +4991 POINT(40.47342706378983 73.70847672933336) bank4991 +4992 POINT(40.1645245475593 74.69240414559194) bank4992 +4993 POINT(40.6917862917393 73.50312328230399) bank4993 +4994 POINT(41.52307226207075 73.21205102023004) bank4994 +4995 POINT(40.47774249357023 74.77951623719198) bank4995 +4996 POINT(40.47915385819339 73.5407489603749) bank4996 +4997 POINT(40.2720293993466 74.26557896980509) bank4997 +4998 POINT(40.01747988731171 73.40457224042909) bank4998 +4999 POINT(40.699219173149594 73.32861031172543) bank4999 +5000 POINT(41.450183659580404 73.45452281695266) bank5000 +5001 POINT(40.167186143532795 74.80237207906549) bank5001 +5002 POINT(40.24378125745171 74.787550433056) bank5002 +5003 POINT(40.11152672133003 73.20367896619594) bank5003 +5004 POINT(41.52133753649665 74.66312529614042) bank5004 +5005 POINT(40.617188845859545 73.46847627664583) bank5005 +5006 POINT(40.986022500899075 74.78006534186083) bank5006 +5007 POINT(39.900610749100416 74.14059945000479) bank5007 +5008 POINT(40.262784178721766 73.30264007976066) bank5008 +5009 POINT(40.95689684749842 74.1234064373287) bank5009 +5010 POINT(39.950775348887326 74.59734661480803) bank5010 +5011 POINT(40.63726055663539 74.8534477538163) bank5011 +5012 POINT(40.488259619621175 73.40874013502355) bank5012 +5013 POINT(40.24536265468574 73.09180437122048) bank5013 +5014 POINT(39.85122630654219 74.43727852591739) bank5014 +5015 POINT(40.9413684241724 73.2672809402548) bank5015 +5016 POINT(40.0285785775333 73.5139822128223) bank5016 +5017 POINT(41.63126681116605 73.51423102223383) bank5017 +5018 POINT(40.62349569354426 73.31901607647691) bank5018 +5019 POINT(41.08553645749107 73.7521472933392) bank5019 +5020 POINT(40.06562297362127 73.34725457078889) bank5020 +5021 POINT(41.12800762923612 74.46332448617764) bank5021 +5022 POINT(40.596010709292486 74.79875941346175) bank5022 +5023 POINT(40.45905267227644 74.62254753968496) bank5023 +5024 POINT(41.4114290793354 74.93205588406823) bank5024 +5025 POINT(40.65217740453506 74.29427128022685) bank5025 +5026 POINT(40.52556873325748 73.19925859804977) bank5026 +5027 POINT(40.57915958814545 74.80797389586233) bank5027 +5028 POINT(39.72902314978508 74.97876117004671) bank5028 +5029 POINT(41.4898349939733 74.82862212169783) bank5029 +5030 POINT(40.656600240048945 73.48456738765667) bank5030 +5031 POINT(41.59168909796107 74.69106451866138) bank5031 +5032 POINT(41.463234988583125 74.03164812250029) bank5032 +5033 POINT(41.12967077657152 73.42374361107917) bank5033 +5034 POINT(39.87811963759296 73.86149491290395) bank5034 +5035 POINT(41.028390441194276 73.70874911632467) bank5035 +5036 POINT(41.07061921096552 74.23741892136893) bank5036 +5037 POINT(41.40818893697139 74.54872828662525) bank5037 +5038 POINT(40.27346503455343 74.77510944813729) bank5038 +5039 POINT(40.93272204415371 73.5116675771763) bank5039 +5040 POINT(40.169275891542014 73.52487099760258) bank5040 +5041 POINT(41.27175638867173 74.48888266412555) bank5041 +5042 POINT(39.957064734572725 74.01832003183101) bank5042 +5043 POINT(40.12144368510613 73.67162537178116) bank5043 +5044 POINT(41.25410290763949 74.67188544102858) bank5044 +5045 POINT(40.37047791107757 73.48071908943358) bank5045 +5046 POINT(41.408263699223376 74.64709930791737) bank5046 +5047 POINT(41.27961994793585 73.32693739312955) bank5047 +5048 POINT(40.0341891232599 74.1495936509908) bank5048 +5049 POINT(40.05511157392866 74.98056679047305) bank5049 +5050 POINT(40.381930310776056 74.82072454442022) bank5050 +5051 POINT(40.62954943775355 73.58617570082431) bank5051 +5052 POINT(40.57071988915332 73.39410257096357) bank5052 +5053 POINT(40.13712298278175 74.38256152708593) bank5053 +5054 POINT(41.18164086035592 74.54141798980724) bank5054 +5055 POINT(39.71464712483002 73.03104020752806) bank5055 +5056 POINT(40.068060116817115 73.45253957603785) bank5056 +5057 POINT(39.72175146223809 74.00959748159369) bank5057 +5058 POINT(41.59774894880243 74.91181005899233) bank5058 +5059 POINT(40.785519208292165 74.97233575129447) bank5059 +5060 POINT(40.666791682085794 74.88832493060134) bank5060 +5061 POINT(40.07072449312421 73.29398805822069) bank5061 +5062 POINT(41.36261719866195 74.58367297297183) bank5062 +5063 POINT(40.27218416036223 73.76655773453982) bank5063 +5064 POINT(40.23880146823308 73.18936409916999) bank5064 +5065 POINT(40.91749009469289 74.96440110858993) bank5065 +5066 POINT(40.129140454343435 73.12828423399814) bank5066 +5067 POINT(39.753628619618105 73.81416032915907) bank5067 +5068 POINT(40.53584473560716 73.8701648062333) bank5068 +5069 POINT(39.84689732072778 73.7050060857448) bank5069 +5070 POINT(41.00502371148008 73.31604867536643) bank5070 +5071 POINT(40.40133299483279 74.70528486868116) bank5071 +5072 POINT(41.53772912446413 73.31521001486475) bank5072 +5073 POINT(40.27846671400546 73.99824913174113) bank5073 +5074 POINT(41.30458185239148 73.47870175793837) bank5074 +5075 POINT(41.04910966230396 74.04027043833871) bank5075 +5076 POINT(39.875591374407186 73.97227187634824) bank5076 +5077 POINT(41.60644627583819 73.25441512608197) bank5077 +5078 POINT(39.91680119153835 74.62083577824879) bank5078 +5079 POINT(41.012720312002934 73.87436371670579) bank5079 +5080 POINT(40.660236803593655 74.68995932972894) bank5080 +5081 POINT(40.97196419274304 73.28427982056559) bank5081 +5082 POINT(41.61162989124756 73.18759257596189) bank5082 +5083 POINT(41.06192136974219 74.85732949570472) bank5083 +5084 POINT(41.27232293231485 74.07223306926439) bank5084 +5085 POINT(40.394147932196134 73.73887376137709) bank5085 +5086 POINT(40.1930154182882 74.20130967715396) bank5086 +5087 POINT(40.654421126720386 73.08657748814285) bank5087 +5088 POINT(41.07944098071189 74.4004661925827) bank5088 +5089 POINT(40.68314630130797 73.86930262193788) bank5089 +5090 POINT(40.00476246225401 74.3586213421281) bank5090 +5091 POINT(39.79145840824039 74.92367710338632) bank5091 +5092 POINT(41.04675260143251 74.30332842509269) bank5092 +5093 POINT(41.59766711029607 74.39083509681356) bank5093 +5094 POINT(40.643402600960165 74.80736797840149) bank5094 +5095 POINT(41.35307296561942 74.42828235373224) bank5095 +5096 POINT(40.288628986880376 73.28345046900195) bank5096 +5097 POINT(40.01689784797213 74.97098092756879) bank5097 +5098 POINT(40.05183588681186 73.26322482610435) bank5098 +5099 POINT(41.5810443775457 73.54107341608194) bank5099 +5100 POINT(41.15370707557615 74.19687866006203) bank5100 +5101 POINT(41.570678059473266 74.8552643714226) bank5101 +5102 POINT(40.88163663597436 74.28194885182745) bank5102 +5103 POINT(40.438933197936954 74.65680908783193) bank5103 +5104 POINT(40.60576602985541 74.89345981400236) bank5104 +5105 POINT(40.60894482544514 73.77911214931343) bank5105 +5106 POINT(40.07211903250216 74.73194789508412) bank5106 +5107 POINT(40.29091523796673 73.26295648375493) bank5107 +5108 POINT(41.19974219102909 73.70294176567688) bank5108 +5109 POINT(40.863192916164905 73.01668133025737) bank5109 +5110 POINT(41.64365668759233 73.33511747961168) bank5110 +5111 POINT(40.05421490410059 74.49532500528703) bank5111 +5112 POINT(39.94638178416692 73.79764314891317) bank5112 +5113 POINT(40.14253738717104 74.65561455281605) bank5113 +5114 POINT(41.1849364883115 74.82254676599194) bank5114 +5115 POINT(41.09706171726343 74.70141364948496) bank5115 +5116 POINT(41.56010112164142 73.11879636347616) bank5116 +5117 POINT(40.15141174920503 74.01314380581367) bank5117 +5118 POINT(40.58419669488119 74.04833445565686) bank5118 +5119 POINT(40.38007581956675 73.48536767669556) bank5119 +5120 POINT(41.45762377325015 73.86549646958866) bank5120 +5121 POINT(40.59934483371303 74.51918796941457) bank5121 +5122 POINT(39.784964807819286 74.37882505379277) bank5122 +5123 POINT(41.280303261345395 74.86580518466486) bank5123 +5124 POINT(40.80769240672513 73.10355564517718) bank5124 +5125 POINT(40.319760558973094 74.66839198922281) bank5125 +5126 POINT(40.29148658834611 74.10210994813544) bank5126 +5127 POINT(41.567590282635415 74.47697584537522) bank5127 +5128 POINT(41.18681561479304 73.30548767399607) bank5128 +5129 POINT(39.86991907867065 74.19874441859932) bank5129 +5130 POINT(39.99647343350333 74.63127857443207) bank5130 +5131 POINT(40.509068147631446 73.1571412271515) bank5131 +5132 POINT(41.32796360524303 73.53267321770392) bank5132 +5133 POINT(40.21330248006261 74.71357956585031) bank5133 +5134 POINT(40.79159116387625 73.02256811792725) bank5134 +5135 POINT(39.917740083123526 73.54438521068161) bank5135 +5136 POINT(40.34645428685025 74.78923963108164) bank5136 +5137 POINT(40.53381029601433 74.73797466273737) bank5137 +5138 POINT(40.02627467626592 74.6124263927874) bank5138 +5139 POINT(41.21593268524483 73.57810487210791) bank5139 +5140 POINT(41.47045049379894 74.26694146472943) bank5140 +5141 POINT(41.45227822554449 74.9500291087124) bank5141 +5142 POINT(39.81499331153063 73.96106003734691) bank5142 +5143 POINT(40.47796597339676 74.07188368449702) bank5143 +5144 POINT(41.18872135088909 73.92654263257236) bank5144 +5145 POINT(40.532571395788665 73.7892014159381) bank5145 +5146 POINT(40.99212373485662 73.61336360464219) bank5146 +5147 POINT(40.82593649583365 73.68051181327587) bank5147 +5148 POINT(40.84133521728053 73.12228725301512) bank5148 +5149 POINT(40.7099879830768 73.58907074020044) bank5149 +5150 POINT(39.79697024490628 74.16156320553496) bank5150 +5151 POINT(39.86951807476968 74.16804147286511) bank5151 +5152 POINT(40.841469105516616 73.30174196047426) bank5152 +5153 POINT(40.87073414883849 74.21002238176145) bank5153 +5154 POINT(41.19217128809023 73.7370210418732) bank5154 +5155 POINT(40.2519379023946 74.26909165517347) bank5155 +5156 POINT(41.34872306201526 73.08223432246726) bank5156 +5157 POINT(41.00402756756041 73.8283512133484) bank5157 +5158 POINT(41.48080767168087 74.88912951032528) bank5158 +5159 POINT(39.97182109771136 73.11563686481055) bank5159 +5160 POINT(40.00934962639906 74.21841620990206) bank5160 +5161 POINT(40.16460589066573 74.72678659371405) bank5161 +5162 POINT(39.94653989257561 74.30016678760146) bank5162 +5163 POINT(40.37411650169028 74.28254640914501) bank5163 +5164 POINT(40.598639548637976 73.78957164419978) bank5164 +5165 POINT(39.87347636075046 73.79215023425827) bank5165 +5166 POINT(40.307034155887685 73.55903318432203) bank5166 +5167 POINT(40.83166557888737 73.53636887342785) bank5167 +5168 POINT(41.10712271568395 73.05192773616481) bank5168 +5169 POINT(40.815294443896576 73.33532739075699) bank5169 +5170 POINT(41.11260077717495 73.66508460445834) bank5170 +5171 POINT(40.13029081035192 74.32194917521392) bank5171 +5172 POINT(40.19168749828328 74.56233719441195) bank5172 +5173 POINT(41.505776315409506 73.63442515937737) bank5173 +5174 POINT(40.84967562705872 74.88879689639427) bank5174 +5175 POINT(40.4981784993476 73.61520705215769) bank5175 +5176 POINT(40.03708920423399 73.3033765002364) bank5176 +5177 POINT(41.35501862093368 74.24069573541529) bank5177 +5178 POINT(40.07508497608127 73.06400237459134) bank5178 +5179 POINT(39.93941215918834 73.1475921200075) bank5179 +5180 POINT(41.4214387583673 73.60523032414515) bank5180 +5181 POINT(41.59030654668597 74.58929654271333) bank5181 +5182 POINT(39.71869429926844 74.9914343805376) bank5182 +5183 POINT(40.15593575975284 73.42707116108384) bank5183 +5184 POINT(41.37050804190966 74.63286917046932) bank5184 +5185 POINT(39.9148817567642 74.96679911443934) bank5185 +5186 POINT(39.85598426985945 73.76467596784856) bank5186 +5187 POINT(41.494587234580415 73.40966259837423) bank5187 +5188 POINT(40.57729786734482 74.2709466628331) bank5188 +5189 POINT(41.181857417540314 74.924946182162) bank5189 +5190 POINT(40.468802966366 74.69998560194342) bank5190 +5191 POINT(40.222479121600124 73.14445884238886) bank5191 +5192 POINT(41.36023536774881 73.83066951600155) bank5192 +5193 POINT(39.90173407793661 73.92793425552365) bank5193 +5194 POINT(40.3061239101635 74.6896800640444) bank5194 +5195 POINT(40.51847465700188 73.14577611641715) bank5195 +5196 POINT(40.79200906825029 73.04309245951404) bank5196 +5197 POINT(41.457023171473956 73.54108091927505) bank5197 +5198 POINT(39.86391951970092 74.75652079633103) bank5198 +5199 POINT(39.812895574084955 74.83134201944563) bank5199 +5200 POINT(40.12288611108802 74.67713534961115) bank5200 +5201 POINT(41.07018162428898 73.31330230088106) bank5201 +5202 POINT(40.72780263145184 73.02228236434587) bank5202 +5203 POINT(40.00062812557248 74.3254320175613) bank5203 +5204 POINT(40.50019103083509 74.66717163871085) bank5204 +5205 POINT(41.63392604527105 74.65748109219716) bank5205 +5206 POINT(40.72542016434711 73.00838825034941) bank5206 +5207 POINT(41.502307703016875 73.68824904333317) bank5207 +5208 POINT(40.21040739676805 74.27018444144457) bank5208 +5209 POINT(40.66341526617842 74.51853486059473) bank5209 +5210 POINT(39.74966046378421 73.6070296920171) bank5210 +5211 POINT(41.427363612724385 73.59825532226562) bank5211 +5212 POINT(41.44988529292446 73.83391503473177) bank5212 +5213 POINT(40.6163659366926 73.58242894318299) bank5213 +5214 POINT(40.22326572467485 73.14866471597891) bank5214 +5215 POINT(41.423780862085394 73.26650641568321) bank5215 +5216 POINT(41.38608348264642 74.08029086996643) bank5216 +5217 POINT(40.23639877962354 73.28423577373104) bank5217 +5218 POINT(39.8803797533767 73.97550117996192) bank5218 +5219 POINT(41.63846146286014 73.46578263371038) bank5219 +5220 POINT(39.893855585725504 74.67596211796972) bank5220 +5221 POINT(40.69998804092579 74.6084079752938) bank5221 +5222 POINT(41.06039817966474 73.65123687059385) bank5222 +5223 POINT(39.98901555996294 74.09954530780465) bank5223 +5224 POINT(40.91874377613582 74.33821174084481) bank5224 +5225 POINT(41.23400580805747 74.20815218037222) bank5225 +5226 POINT(41.348091633375525 74.29242203075673) bank5226 +5227 POINT(40.37143766299325 74.19195315609115) bank5227 +5228 POINT(40.35003805589961 73.83433966721228) bank5228 +5229 POINT(41.53987330429031 73.97403709621267) bank5229 +5230 POINT(41.43631386395314 73.79426701490297) bank5230 +5231 POINT(41.55059858769962 73.50795980547518) bank5231 +5232 POINT(41.52603078257982 73.17868974960166) bank5232 +5233 POINT(40.8175187759688 73.98446179493068) bank5233 +5234 POINT(41.649689988763974 73.58593966731294) bank5234 +5235 POINT(39.88507022076865 73.39958522032627) bank5235 +5236 POINT(41.61026563517775 73.63755690952327) bank5236 +5237 POINT(40.98133843446447 73.93787262105494) bank5237 +5238 POINT(40.44444030500362 74.8692877909573) bank5238 +5239 POINT(41.2964195728679 74.50091128869187) bank5239 +5240 POINT(40.164254455779236 73.63464921903781) bank5240 +5241 POINT(39.75631320490355 74.41429963141832) bank5241 +5242 POINT(39.85598869991861 74.12284763018035) bank5242 +5243 POINT(41.15710196411226 74.93659431333782) bank5243 +5244 POINT(40.25484252771844 74.37732278380197) bank5244 +5245 POINT(40.50682355072612 73.62031592304444) bank5245 +5246 POINT(41.510178257819916 74.55727929503288) bank5246 +5247 POINT(40.791254448066994 74.47691622264271) bank5247 +5248 POINT(40.69964838999193 74.64016430369533) bank5248 +5249 POINT(41.405411114475335 73.05417799542442) bank5249 +5250 POINT(39.79151934077714 73.15941800359172) bank5250 +5251 POINT(39.938785888448415 73.67767390397738) bank5251 +5252 POINT(40.02828561326649 74.15692898148083) bank5252 +5253 POINT(40.81472387027935 74.21970719905599) bank5253 +5254 POINT(40.566968974646144 74.21214640175775) bank5254 +5255 POINT(39.81701320453992 74.97266567536596) bank5255 +5256 POINT(40.87186953083421 73.3881083409868) bank5256 +5257 POINT(41.28536487757726 74.4190795774777) bank5257 +5258 POINT(40.44660804491133 74.64307312761285) bank5258 +5259 POINT(41.44756027190245 74.2621639299448) bank5259 +5260 POINT(40.008996586625656 73.13310713322323) bank5260 +5261 POINT(40.76797829852806 73.50372137408857) bank5261 +5262 POINT(39.770897509313734 74.44560884887416) bank5262 +5263 POINT(39.860490601895 73.30477922397793) bank5263 +5264 POINT(40.620110880870946 74.41878290631504) bank5264 +5265 POINT(39.97670373070232 73.78623315448837) bank5265 +5266 POINT(40.10980894497203 74.26743900555213) bank5266 +5267 POINT(40.029120541194345 74.59690899406021) bank5267 +5268 POINT(41.48609295873801 73.61423869734433) bank5268 +5269 POINT(40.722065622795064 74.09047322529001) bank5269 +5270 POINT(40.21118029801663 74.22274128019282) bank5270 +5271 POINT(40.96808375447736 74.27413953156375) bank5271 +5272 POINT(39.761779598129536 73.35883738279553) bank5272 +5273 POINT(39.765440810961366 74.27443675990389) bank5273 +5274 POINT(40.93481808205554 74.06646486273819) bank5274 +5275 POINT(40.21222971451458 73.24525586102031) bank5275 +5276 POINT(40.63024155780967 73.46732832061669) bank5276 +5277 POINT(40.50400073564961 73.84795673383859) bank5277 +5278 POINT(39.77760549619599 74.05464909740448) bank5278 +5279 POINT(41.605900562963505 74.79820437188782) bank5279 +5280 POINT(41.560373261168344 74.7339714486937) bank5280 +5281 POINT(39.77365372707571 74.65191329229943) bank5281 +5282 POINT(40.65108617929222 74.8667866716012) bank5282 +5283 POINT(39.83511746588351 74.63497824894579) bank5283 +5284 POINT(41.02521022504769 73.74728066871062) bank5284 +5285 POINT(40.35595969010805 74.7718597185216) bank5285 +5286 POINT(40.290820878822835 73.98077608541958) bank5286 +5287 POINT(40.484203945117464 74.8137300721749) bank5287 +5288 POINT(40.67381649153761 73.61841788468314) bank5288 +5289 POINT(40.11974660627422 74.8398113463247) bank5289 +5290 POINT(40.61455401941298 73.89075393951293) bank5290 +5291 POINT(41.51173964194024 74.16913722932921) bank5291 +5292 POINT(40.27652078415367 73.55352201474871) bank5292 +5293 POINT(40.23496547897411 73.48914375153505) bank5293 +5294 POINT(40.52037631708453 73.84073508364136) bank5294 +5295 POINT(41.38287564789697 74.9350491297175) bank5295 +5296 POINT(41.009504090528594 73.49516365922233) bank5296 +5297 POINT(40.809021628330214 74.90788310542871) bank5297 +5298 POINT(41.42619992341706 74.74638974280803) bank5298 +5299 POINT(40.71530185249781 74.77750365411313) bank5299 +5300 POINT(41.45577481725248 73.01000060912362) bank5300 +5301 POINT(39.85550776385768 73.53490733255921) bank5301 +5302 POINT(40.6409341411527 74.83502717461863) bank5302 +5303 POINT(41.03067725854505 74.84885028291686) bank5303 +5304 POINT(41.20588940590191 74.97646825006348) bank5304 +5305 POINT(39.76494562422183 73.54510395904558) bank5305 +5306 POINT(40.893260850839795 73.04817031561909) bank5306 +5307 POINT(40.536502263209634 73.32852307867496) bank5307 +5308 POINT(40.730438392644324 74.93887979011355) bank5308 +5309 POINT(40.7923404366685 74.58658748250805) bank5309 +5310 POINT(39.750380885134426 74.70306225899922) bank5310 +5311 POINT(41.31876329064748 73.01277003006204) bank5311 +5312 POINT(39.73372438764318 74.53927554700897) bank5312 +5313 POINT(40.993444114233114 73.42649192481575) bank5313 +5314 POINT(39.85908966126942 73.32729038943634) bank5314 +5315 POINT(40.81277989699521 74.42743592285326) bank5315 +5316 POINT(40.01929396650023 73.11795536553389) bank5316 +5317 POINT(41.31095940196968 73.67387237382644) bank5317 +5318 POINT(41.46553700582491 74.71095504771385) bank5318 +5319 POINT(41.70382268100039 73.69244889694068) bank5319 +5320 POINT(40.31832368607606 74.55110219374431) bank5320 +5321 POINT(41.495601652107716 73.8140800794343) bank5321 +5322 POINT(39.96764225328635 73.11339536674491) bank5322 +5323 POINT(41.582665308158646 73.63825810789153) bank5323 +5324 POINT(40.15884308050849 73.11789696240146) bank5324 +5325 POINT(40.973285993315876 73.97626095154268) bank5325 +5326 POINT(40.451274078029144 73.62662684184835) bank5326 +5327 POINT(40.62724752242497 74.34162660801503) bank5327 +5328 POINT(40.602926219011515 74.04837319481054) bank5328 +5329 POINT(40.612997909510966 73.18536472237419) bank5329 +5330 POINT(41.20583115490668 74.35092034963976) bank5330 +5331 POINT(40.191591581792245 73.9389779233633) bank5331 +5332 POINT(40.51441454154758 74.71976553923248) bank5332 +5333 POINT(40.180743435412225 73.1346351782269) bank5333 +5334 POINT(40.34150445662618 74.37581259964661) bank5334 +5335 POINT(39.932109852201556 74.10238623244165) bank5335 +5336 POINT(40.79279748270103 73.44808988982871) bank5336 +5337 POINT(40.912051070998174 74.12453436341818) bank5337 +5338 POINT(40.40242755486754 74.21278498476026) bank5338 +5339 POINT(39.967732710597566 73.9069155743487) bank5339 +5340 POINT(41.41933304027663 74.94937321524387) bank5340 +5341 POINT(39.9812353525066 73.8640662958277) bank5341 +5342 POINT(41.307421186615315 73.75754681141184) bank5342 +5343 POINT(40.23488633865999 74.3546940601769) bank5343 +5344 POINT(41.171482071028066 73.5175005362582) bank5344 +5345 POINT(41.10883362629221 74.54620553533034) bank5345 +5346 POINT(40.03760117314358 74.39198511933554) bank5346 +5347 POINT(40.284074607007184 74.97196861289726) bank5347 +5348 POINT(41.49123539668708 74.7448801783518) bank5348 +5349 POINT(41.34284035037556 74.81778482404381) bank5349 +5350 POINT(39.887804913909044 73.19233647097589) bank5350 +5351 POINT(41.33477320652941 74.44058280774652) bank5351 +5352 POINT(39.837269608906304 73.73033711332306) bank5352 +5353 POINT(40.64985580632679 73.51166351934482) bank5353 +5354 POINT(41.3717281880764 73.58116081067568) bank5354 +5355 POINT(41.147682337696395 74.4488303451659) bank5355 +5356 POINT(41.17539292564755 74.85142880464298) bank5356 +5357 POINT(41.46807477065554 73.23802772661597) bank5357 +5358 POINT(40.05200476425106 73.61399323750572) bank5358 +5359 POINT(40.71801155698805 74.22663941046824) bank5359 +5360 POINT(40.3273451446387 74.54445599668969) bank5360 +5361 POINT(40.43729305408375 74.65587584650223) bank5361 +5362 POINT(40.7458199410911 73.5548972539778) bank5362 +5363 POINT(40.163878212307544 73.03589024456136) bank5363 +5364 POINT(41.122638777785205 74.92929945457887) bank5364 +5365 POINT(41.598872712357455 73.70083027610877) bank5365 +5366 POINT(40.83198834854766 73.40428536672277) bank5366 +5367 POINT(39.72075834114442 74.74660064303961) bank5367 +5368 POINT(40.92928422968276 73.92999960267846) bank5368 +5369 POINT(39.9067811733292 74.81271690950187) bank5369 +5370 POINT(40.11576990977099 74.46802714383128) bank5370 +5371 POINT(40.69417814089368 74.93085219315698) bank5371 +5372 POINT(40.37652635546544 73.01499426535612) bank5372 +5373 POINT(40.00762733354006 73.26486587857042) bank5373 +5374 POINT(39.74035272220149 74.3589016179835) bank5374 +5375 POINT(41.09919329406645 74.58599755313269) bank5375 +5376 POINT(40.783352387485756 74.82695212422948) bank5376 +5377 POINT(40.67750101541118 74.22872684676825) bank5377 +5378 POINT(41.529466618286065 74.11423962478251) bank5378 +5379 POINT(40.2415892815439 74.09634673946371) bank5379 +5380 POINT(41.18895645493877 74.09832447362052) bank5380 +5381 POINT(40.31631807996036 74.73823779535263) bank5381 +5382 POINT(41.63455296181803 73.42808389444191) bank5382 +5383 POINT(40.722958046378665 73.30487190231356) bank5383 +5384 POINT(41.20012465014854 74.12216866502143) bank5384 +5385 POINT(41.37514546583709 73.23527310989462) bank5385 +5386 POINT(41.0059659603791 74.55682851656773) bank5386 +5387 POINT(41.66609293707741 73.94845028706409) bank5387 +5388 POINT(40.81917654130071 73.56580989469906) bank5388 +5389 POINT(40.56301737661824 74.36340323704198) bank5389 +5390 POINT(39.81243691736478 73.338228495253) bank5390 +5391 POINT(40.30421635440768 73.61003221911315) bank5391 +5392 POINT(39.98118634212876 73.40947822741035) bank5392 +5393 POINT(40.232993232784075 74.34617869410631) bank5393 +5394 POINT(41.54528098076654 74.11564668689255) bank5394 +5395 POINT(40.614135923744456 73.3399344638284) bank5395 +5396 POINT(39.782423072470785 74.36449623714458) bank5396 +5397 POINT(41.109657166975175 74.6228818351954) bank5397 +5398 POINT(40.6441337908868 73.39507055615127) bank5398 +5399 POINT(41.34293914985931 73.56858554914395) bank5399 +5400 POINT(40.65503109414365 74.55297837432377) bank5400 +5401 POINT(41.35314445771362 74.99579861795148) bank5401 +5402 POINT(41.4264501042966 74.89636545282399) bank5402 +5403 POINT(40.65563689516594 73.28094216195386) bank5403 +5404 POINT(40.53946737564902 73.02650073093801) bank5404 +5405 POINT(40.77213375541978 74.29843742565082) bank5405 +5406 POINT(41.30716250989262 73.50130137247845) bank5406 +5407 POINT(40.98772659560578 73.84429861165249) bank5407 +5408 POINT(39.85035284712951 73.67056928733456) bank5408 +5409 POINT(39.91046199943549 73.91839926499739) bank5409 +5410 POINT(40.24570705865755 74.69472840749884) bank5410 +5411 POINT(41.06252085837701 73.27137864075522) bank5411 +5412 POINT(40.21209101829175 74.5866686644869) bank5412 +5413 POINT(41.141100593911844 73.70499409001299) bank5413 +5414 POINT(40.15537517301117 73.07487162806326) bank5414 +5415 POINT(41.66223705169083 74.31165221460161) bank5415 +5416 POINT(39.75830026346719 74.19966501495905) bank5416 +5417 POINT(41.614236485726124 74.65511232743343) bank5417 +5418 POINT(41.64640001379044 74.2593386664416) bank5418 +5419 POINT(40.328382633668205 74.24599032608327) bank5419 +5420 POINT(40.95265228089953 74.81447759322242) bank5420 +5421 POINT(40.56170483733532 74.9915233146035) bank5421 +5422 POINT(40.76546318695617 73.7817955212256) bank5422 +5423 POINT(40.85207373833252 74.33968712092626) bank5423 +5424 POINT(40.54716127116332 74.82758058987982) bank5424 +5425 POINT(41.62864789926114 73.69474428308332) bank5425 +5426 POINT(40.866627842078024 74.30742809880553) bank5426 +5427 POINT(41.29921279753357 74.51772351562558) bank5427 +5428 POINT(40.5192215171073 73.48255078247394) bank5428 +5429 POINT(41.504380713691766 73.9182697228152) bank5429 +5430 POINT(41.16475500951502 73.08867464631672) bank5430 +5431 POINT(40.026845931999816 74.11368037234764) bank5431 +5432 POINT(40.45615429329539 74.98199031971399) bank5432 +5433 POINT(40.01256955384643 73.02229310209613) bank5433 +5434 POINT(39.96346139807747 73.85369517865674) bank5434 +5435 POINT(40.569389631496136 73.77526176047007) bank5435 +5436 POINT(41.20753473156641 73.12089533903435) bank5436 +5437 POINT(41.2191455741403 74.85329027770234) bank5437 +5438 POINT(40.94122445234121 74.8020198580463) bank5438 +5439 POINT(41.27131319059292 73.13992382386658) bank5439 +5440 POINT(41.14336186874631 73.73305932378041) bank5440 +5441 POINT(40.76095391737417 73.6292421400307) bank5441 +5442 POINT(39.74221297002786 74.07336825765294) bank5442 +5443 POINT(41.311844420166615 74.81918226133384) bank5443 +5444 POINT(40.617851928006516 73.29588941881583) bank5444 +5445 POINT(40.31615926459768 74.54006030241388) bank5445 +5446 POINT(40.600540159796054 73.15049650738474) bank5446 +5447 POINT(41.26256774545908 74.27614625810524) bank5447 +5448 POINT(41.3481092175046 73.11453371195749) bank5448 +5449 POINT(39.806376087604754 74.85272370257181) bank5449 +5450 POINT(40.91559738395586 74.69963316976845) bank5450 +5451 POINT(41.03246583408798 73.06757705890368) bank5451 +5452 POINT(40.608938077173526 74.18998448607313) bank5452 +5453 POINT(40.01366027383396 73.30565618772108) bank5453 +5454 POINT(40.38502019077633 74.29115780961166) bank5454 +5455 POINT(39.81605549464669 73.69386838155236) bank5455 +5456 POINT(41.69321818661437 73.31929873302329) bank5456 +5457 POINT(41.5775595232562 74.30753532124736) bank5457 +5458 POINT(39.952180250616514 73.21893736650671) bank5458 +5459 POINT(40.06806342084203 74.88786843133255) bank5459 +5460 POINT(41.48649253832726 74.28818385517613) bank5460 +5461 POINT(41.069323504115026 73.26207801960165) bank5461 +5462 POINT(41.493436220617426 73.04422449518498) bank5462 +5463 POINT(41.58287105556625 73.12784154639078) bank5463 +5464 POINT(40.32482059561045 73.55227839258292) bank5464 +5465 POINT(41.2679602220981 74.06507233978436) bank5465 +5466 POINT(40.448731400794884 74.76181407330587) bank5466 +5467 POINT(41.676525295361934 73.10379395176855) bank5467 +5468 POINT(40.54479488919624 73.34199692667316) bank5468 +5469 POINT(41.43827371514835 74.90231626414453) bank5469 +5470 POINT(40.459112918195345 74.77438380545478) bank5470 +5471 POINT(41.508942686111226 74.26107761642969) bank5471 +5472 POINT(40.50775708694232 74.65340650400601) bank5472 +5473 POINT(39.77442898737047 73.21607504866091) bank5473 +5474 POINT(39.88157268685189 74.49500644617068) bank5474 +5475 POINT(39.757607165141025 73.06294612936554) bank5475 +5476 POINT(41.534284465226946 74.5856756794649) bank5476 +5477 POINT(40.3801441354359 74.15619171259519) bank5477 +5478 POINT(40.47387629112744 74.89286769118081) bank5478 +5479 POINT(41.54774173907641 73.08011782529819) bank5479 +5480 POINT(40.81953865314443 73.8041788632736) bank5480 +5481 POINT(41.489617617350575 74.86170243448672) bank5481 +5482 POINT(41.55901033752358 73.48750394824711) bank5482 +5483 POINT(41.207165955337516 74.88888867704631) bank5483 +5484 POINT(40.69058512794512 73.92392442102026) bank5484 +5485 POINT(40.56416833392554 74.69855151828142) bank5485 +5486 POINT(40.60784132558994 73.48744420235246) bank5486 +5487 POINT(41.289670714255095 73.85217820735781) bank5487 +5488 POINT(40.02598052082011 74.67159917541596) bank5488 +5489 POINT(39.807039214744876 73.7382586496846) bank5489 +5490 POINT(41.26784021660855 74.6731622853389) bank5490 +5491 POINT(40.091256391708406 74.23949581022657) bank5491 +5492 POINT(41.54076300531983 74.13263049364565) bank5492 +5493 POINT(39.75099506582016 74.18967739635018) bank5493 +5494 POINT(40.57708925334484 74.22061842853502) bank5494 +5495 POINT(40.22145108285846 73.19140410324488) bank5495 +5496 POINT(41.26890387662327 74.74784311886725) bank5496 +5497 POINT(40.24776368488685 74.25214135856206) bank5497 +5498 POINT(41.70702017858123 73.451639913437) bank5498 +5499 POINT(40.51353801734928 73.50234197775374) bank5499 +5500 POINT(40.451272939778995 74.11237027915979) bank5500 +5501 POINT(41.1008561783649 73.91373503675996) bank5501 +5502 POINT(41.04964811614275 74.5512736774402) bank5502 +5503 POINT(41.35449302831255 74.65112050316384) bank5503 +5504 POINT(39.9696390632992 74.49899066970438) bank5504 +5505 POINT(41.563439208507496 73.29763880540926) bank5505 +5506 POINT(40.255024771094796 73.59177350075555) bank5506 +5507 POINT(40.69153920265177 74.40597251475867) bank5507 +5508 POINT(40.61506946321328 74.75860607159615) bank5508 +5509 POINT(41.07568611867044 73.76631841463188) bank5509 +5510 POINT(41.21799305549607 73.00623580162036) bank5510 +5511 POINT(40.43359888007934 73.32394960944636) bank5511 +5512 POINT(40.53265120307471 74.86274384362497) bank5512 +5513 POINT(40.87047262518909 74.79028073312391) bank5513 +5514 POINT(41.41094641284726 73.15941533411605) bank5514 +5515 POINT(41.29821057124118 73.94868317659055) bank5515 +5516 POINT(40.411348789006844 73.22631130454452) bank5516 +5517 POINT(40.75441176312514 74.57485352676119) bank5517 +5518 POINT(40.83302179796944 74.69174091568084) bank5518 +5519 POINT(41.055303761410556 74.82559216769269) bank5519 +5520 POINT(40.67753558420466 73.86988417155759) bank5520 +5521 POINT(40.400732345007825 74.52040467184602) bank5521 +5522 POINT(40.49559073715081 73.22798864183518) bank5522 +5523 POINT(40.086692561614214 73.35185412012913) bank5523 +5524 POINT(39.87229088893721 74.1555202272113) bank5524 +5525 POINT(40.87763432970672 73.69686914220954) bank5525 +5526 POINT(40.727568606216565 74.41515227975945) bank5526 +5527 POINT(41.50951603596036 74.24842941107629) bank5527 +5528 POINT(39.7866145825913 73.43830619824797) bank5528 +5529 POINT(39.93311069958606 73.14857044950544) bank5529 +5530 POINT(40.04169850208274 73.35180633124831) bank5530 +5531 POINT(40.091072401643814 73.7915959667982) bank5531 +5532 POINT(41.17439991411017 73.65852346702908) bank5532 +5533 POINT(41.07889540505281 74.7377175218502) bank5533 +5534 POINT(41.261420897549165 74.0541656422894) bank5534 +5535 POINT(40.23509454853833 73.7647008065405) bank5535 +5536 POINT(41.425745913620766 74.54270798256503) bank5536 +5537 POINT(40.225974644795215 73.91414555576742) bank5537 +5538 POINT(40.463956183066614 74.90830904245361) bank5538 +5539 POINT(40.68555777106649 73.92156140912869) bank5539 +5540 POINT(40.52368356993291 73.56406314081576) bank5540 +5541 POINT(40.41406294833578 74.39830153251037) bank5541 +5542 POINT(41.04703847842622 74.97551014635428) bank5542 +5543 POINT(39.77983425873048 73.33614502370061) bank5543 +5544 POINT(40.53283443475568 73.96096146416308) bank5544 +5545 POINT(41.500936829449266 73.13440873411126) bank5545 +5546 POINT(40.05982167525764 74.0156666292791) bank5546 +5547 POINT(40.30025458168207 74.55888017508313) bank5547 +5548 POINT(41.691507130633276 74.5207041259743) bank5548 +5549 POINT(40.21970701642077 74.39588996221673) bank5549 +5550 POINT(39.83039727585715 74.62209532241144) bank5550 +5551 POINT(39.729980087005096 74.91907888020565) bank5551 +5552 POINT(41.02590351110315 73.47117467827064) bank5552 +5553 POINT(40.164326936914826 73.15904606586324) bank5553 +5554 POINT(39.97383946473062 73.85796474856946) bank5554 +5555 POINT(40.33896286797961 73.54322829678038) bank5555 +5556 POINT(40.87638647269808 73.28742527310006) bank5556 +5557 POINT(40.86527259151665 74.57519897127379) bank5557 +5558 POINT(40.231187047919065 74.0909430313772) bank5558 +5559 POINT(40.72465316450867 74.3174633319617) bank5559 +5560 POINT(41.25242337238174 74.6671105267935) bank5560 +5561 POINT(40.099464800281446 75.00448353638232) bank5561 +5562 POINT(39.74872328640375 73.87396160407147) bank5562 +5563 POINT(40.986798754650216 73.03801650535289) bank5563 +5564 POINT(40.37366476801346 74.96421345001914) bank5564 +5565 POINT(40.34571094535679 74.00568631642184) bank5565 +5566 POINT(40.20294671129248 74.85059747038862) bank5566 +5567 POINT(40.77477874220767 73.59354216016706) bank5567 +5568 POINT(40.58197314054416 73.81405273181902) bank5568 +5569 POINT(39.76469506713469 73.70485821156025) bank5569 +5570 POINT(39.85288555181196 74.68601860297474) bank5570 +5571 POINT(40.84114930944051 74.75300469408826) bank5571 +5572 POINT(40.04908175532466 73.55699627150874) bank5572 +5573 POINT(40.2899762652662 73.58089220299205) bank5573 +5574 POINT(40.940859359045255 73.74562460452114) bank5574 +5575 POINT(40.35339089522056 74.5961571902218) bank5575 +5576 POINT(40.4948980813907 73.80867734145971) bank5576 +5577 POINT(41.12442879044457 73.36667715337536) bank5577 +5578 POINT(41.18954268473218 73.85180411499138) bank5578 +5579 POINT(40.208877570429834 74.11166426844117) bank5579 +5580 POINT(40.61487763665727 74.14794385236179) bank5580 +5581 POINT(41.572190215374874 74.45451729752703) bank5581 +5582 POINT(40.577347110998396 74.34491616120683) bank5582 +5583 POINT(40.812055783699456 74.69342971823902) bank5583 +5584 POINT(41.31425623204307 74.92504281049166) bank5584 +5585 POINT(41.23875943514703 74.65458314305724) bank5585 +5586 POINT(40.489878992008094 73.58348173243635) bank5586 +5587 POINT(40.428752668917696 73.70517192558225) bank5587 +5588 POINT(41.321494729300696 74.82982560197973) bank5588 +5589 POINT(41.478414881069455 73.39651510316403) bank5589 +5590 POINT(39.797363101833234 73.2085487759761) bank5590 +5591 POINT(41.20202280864811 74.45081695191126) bank5591 +5592 POINT(39.76104869995047 73.17143380265439) bank5592 +5593 POINT(41.0765945232934 74.90754178889958) bank5593 +5594 POINT(40.27016970866829 73.77206581822371) bank5594 +5595 POINT(41.10033184083207 74.5854865717315) bank5595 +5596 POINT(41.41506506241399 74.54101051602218) bank5596 +5597 POINT(41.16508756330697 74.93170832412974) bank5597 +5598 POINT(41.44592890556923 74.70949500549254) bank5598 +5599 POINT(41.635986309158085 74.47545659010345) bank5599 +5600 POINT(40.14096223203526 74.50672560405962) bank5600 +5601 POINT(40.256138216151676 74.64827518156531) bank5601 +5602 POINT(40.48176156204318 74.30047295467938) bank5602 +5603 POINT(40.972763059551944 73.5532789125504) bank5603 +5604 POINT(41.41483578889923 74.16278306303768) bank5604 +5605 POINT(40.09493938895403 74.98660115743844) bank5605 +5606 POINT(40.16267450188377 74.30000494335724) bank5606 +5607 POINT(40.5241423817126 73.10834311949107) bank5607 +5608 POINT(41.38059712329785 74.17455563491986) bank5608 +5609 POINT(39.930571861108284 73.19506120580594) bank5609 +5610 POINT(40.90691331127758 74.8343143220099) bank5610 +5611 POINT(40.22074350715339 74.95832910205463) bank5611 +5612 POINT(40.57346399385373 73.17869975812249) bank5612 +5613 POINT(41.549135185301296 74.82813322005964) bank5613 +5614 POINT(41.61956828918944 73.62728917797757) bank5614 +5615 POINT(40.184243370279106 73.07067747915093) bank5615 +5616 POINT(41.383328137333045 73.50840726335639) bank5616 +5617 POINT(40.13323804535341 74.12699321146358) bank5617 +5618 POINT(40.389764543984995 74.37606267793869) bank5618 +5619 POINT(40.47287633363595 73.10906480953354) bank5619 +5620 POINT(39.92581625189748 73.84599242917128) bank5620 +5621 POINT(41.45146041857811 73.5892217999962) bank5621 +5622 POINT(41.29583483587005 73.3553875616175) bank5622 +5623 POINT(41.244639138781075 74.89867756434663) bank5623 +5624 POINT(40.3031747237506 74.87382401826636) bank5624 +5625 POINT(40.26351210702183 73.66823996455352) bank5625 +5626 POINT(40.85576972142206 75.00587885811245) bank5626 +5627 POINT(41.709325316112675 73.8580032428083) bank5627 +5628 POINT(39.83666742801788 74.15633714417108) bank5628 +5629 POINT(40.47115966125178 74.02206787011359) bank5629 +5630 POINT(40.02701154495 74.04707389311436) bank5630 +5631 POINT(39.947029079088765 73.25888984023346) bank5631 +5632 POINT(39.88772253183676 73.8664266229814) bank5632 +5633 POINT(40.07889066572036 74.07055176821832) bank5633 +5634 POINT(40.026820877430566 73.08874278359765) bank5634 +5635 POINT(40.14334228923909 73.08677393410782) bank5635 +5636 POINT(40.47146509652266 73.8361855900261) bank5636 +5637 POINT(39.941041000255005 73.90483372948673) bank5637 +5638 POINT(40.31996096944292 74.09064720187311) bank5638 +5639 POINT(41.35088352776548 73.44331675512343) bank5639 +5640 POINT(40.28043284155575 74.1926831501637) bank5640 +5641 POINT(40.404263374480756 73.20751961971061) bank5641 +5642 POINT(40.131784626449765 74.73457197337427) bank5642 +5643 POINT(40.087334992190534 74.02717743633261) bank5643 +5644 POINT(40.43960665314057 74.99732326885672) bank5644 +5645 POINT(40.600891683557094 74.59067511945011) bank5645 +5646 POINT(41.52095975826982 73.30028382447708) bank5646 +5647 POINT(41.47706578769661 73.44140450622145) bank5647 +5648 POINT(41.106014883986006 74.0389273811883) bank5648 +5649 POINT(41.702515109337405 74.7414404201091) bank5649 +5650 POINT(41.37937026073979 73.54572716809938) bank5650 +5651 POINT(41.611258226971636 74.84439101850293) bank5651 +5652 POINT(40.32466115263353 74.46376802771668) bank5652 +5653 POINT(40.01495810129681 74.67796121157689) bank5653 +5654 POINT(41.55112722161566 74.43325103144055) bank5654 +5655 POINT(41.47176172244339 74.29232459791159) bank5655 +5656 POINT(40.01371723735983 73.93953529480817) bank5656 +5657 POINT(40.08697528371946 74.69220647402057) bank5657 +5658 POINT(40.8118357233204 74.90250323011682) bank5658 +5659 POINT(40.24277170587144 74.57499863661364) bank5659 +5660 POINT(41.06964068979099 74.10224753119401) bank5660 +5661 POINT(41.488182745027096 73.4936051722219) bank5661 +5662 POINT(41.378272848552726 73.448327721177) bank5662 +5663 POINT(39.742969640678446 73.1318178807927) bank5663 +5664 POINT(41.17443788923343 73.96466851891059) bank5664 +5665 POINT(40.106318084407846 73.82218558375139) bank5665 +5666 POINT(41.022928713901585 73.90066776108903) bank5666 +5667 POINT(39.77160485585921 73.68946993945538) bank5667 +5668 POINT(40.39939592121788 73.87321437134308) bank5668 +5669 POINT(41.43515463931033 73.20504548038637) bank5669 +5670 POINT(40.964958107285966 73.41734354974434) bank5670 +5671 POINT(40.82293597898048 74.4749477153022) bank5671 +5672 POINT(41.317037682319324 74.08573100949229) bank5672 +5673 POINT(41.410402448082685 74.10659551901864) bank5673 +5674 POINT(40.144974330550724 73.02682836704172) bank5674 +5675 POINT(40.054632521515344 74.41648306005136) bank5675 +5676 POINT(40.05184104792354 73.38707080243756) bank5676 +5677 POINT(40.25158092382945 73.35188098153108) bank5677 +5678 POINT(40.485522751337164 73.36796426666048) bank5678 +5679 POINT(39.81492542461579 73.44398978845622) bank5679 +5680 POINT(40.0469295851919 73.24412273710489) bank5680 +5681 POINT(41.70625152808407 73.25361897610085) bank5681 +5682 POINT(41.65429860796219 73.31260570513028) bank5682 +5683 POINT(39.81260199427557 74.49994228691796) bank5683 +5684 POINT(40.532908382607154 74.70621019269504) bank5684 +5685 POINT(41.22920305230619 73.67688744999215) bank5685 +5686 POINT(40.48417449710244 74.83077030093662) bank5686 +5687 POINT(41.086564024856145 73.3415717702104) bank5687 +5688 POINT(40.62123686814426 74.46358430703552) bank5688 +5689 POINT(39.777670054563245 73.74401900531102) bank5689 +5690 POINT(41.5152254224771 74.0031675543126) bank5690 +5691 POINT(40.682732195219536 73.87502262324594) bank5691 +5692 POINT(41.703224734412494 73.69808161013555) bank5692 +5693 POINT(41.38044316804455 73.22341060773705) bank5693 +5694 POINT(41.114711094762114 73.11855567447209) bank5694 +5695 POINT(40.04953592873065 74.12776276388193) bank5695 +5696 POINT(41.04511987212709 73.21790585413432) bank5696 +5697 POINT(40.47129889882538 73.96138143264902) bank5697 +5698 POINT(40.07280777362314 73.32019253282621) bank5698 +5699 POINT(40.74261081379596 73.236488427216) bank5699 +5700 POINT(40.524865728576884 73.49384375599206) bank5700 +5701 POINT(41.309861194859344 73.45151542031653) bank5701 +5702 POINT(41.39596917260182 74.713407935417) bank5702 +5703 POINT(41.04410385911691 74.81543674913648) bank5703 +5704 POINT(40.00801960105402 74.3434444454198) bank5704 +5705 POINT(39.912699806404525 73.48302588798967) bank5705 +5706 POINT(41.292509125796045 73.52514306400289) bank5706 +5707 POINT(40.385319556314826 74.11855701512218) bank5707 +5708 POINT(40.200164683506664 74.05687959464242) bank5708 +5709 POINT(40.11371814141392 73.87276089743703) bank5709 +5710 POINT(40.57489408994461 73.03507473420161) bank5710 +5711 POINT(40.57069660292136 74.79695105573606) bank5711 +5712 POINT(40.73994873752919 74.7893483344638) bank5712 +5713 POINT(39.87944743517511 74.49296351470407) bank5713 +5714 POINT(40.87786347026016 74.86651423296755) bank5714 +5715 POINT(41.38188562946683 73.9246647046685) bank5715 +5716 POINT(40.94460013134048 74.40954075563788) bank5716 +5717 POINT(39.88322683974127 73.20382038737154) bank5717 +5718 POINT(40.46296359740138 73.32449980414088) bank5718 +5719 POINT(40.153897346959916 73.49024963041039) bank5719 +5720 POINT(41.23337817776295 73.14856737513279) bank5720 +5721 POINT(40.67026663900786 73.07550530446699) bank5721 +5722 POINT(41.339660771419865 74.30754165684348) bank5722 +5723 POINT(40.832191312152 74.63170223587073) bank5723 +5724 POINT(41.65729327610258 73.23753348741981) bank5724 +5725 POINT(39.749470980761174 73.82405953003003) bank5725 +5726 POINT(41.04399995181821 74.86792330500207) bank5726 +5727 POINT(40.44141189695004 73.32676237849033) bank5727 +5728 POINT(39.7756429407859 74.64458492261068) bank5728 +5729 POINT(39.74649216709412 73.69269722371466) bank5729 +5730 POINT(40.85915704905736 74.57672807692096) bank5730 +5731 POINT(40.6232783752164 73.4318053686204) bank5731 +5732 POINT(40.445429075153775 74.18717302063177) bank5732 +5733 POINT(40.28525661599602 74.65496437217867) bank5733 +5734 POINT(41.20092094553037 74.73756173927379) bank5734 +5735 POINT(39.954951846298066 74.27400226262854) bank5735 +5736 POINT(40.007371227765844 73.8759355096091) bank5736 +5737 POINT(40.530757082189155 73.36687245076071) bank5737 +5738 POINT(41.06744655880056 74.28296963341128) bank5738 +5739 POINT(41.35941305118865 74.27346801252911) bank5739 +5740 POINT(40.78729929527389 73.36006935675623) bank5740 +5741 POINT(40.645735614495024 73.78899129818711) bank5741 +5742 POINT(39.97130717264856 73.55794879991555) bank5742 +5743 POINT(39.77816738432814 73.50297142246136) bank5743 +5744 POINT(40.60991214911116 74.48078951123871) bank5744 +5745 POINT(40.459360638345274 74.11193610239519) bank5745 +5746 POINT(39.77458142175668 73.90337392414216) bank5746 +5747 POINT(39.73745243319687 73.68653562782643) bank5747 +5748 POINT(40.845104325396335 73.28294429512) bank5748 +5749 POINT(41.09654730977244 73.46836345889149) bank5749 +5750 POINT(39.73181787714898 74.96007034009523) bank5750 +5751 POINT(41.031032543944114 74.53591309329812) bank5751 +5752 POINT(40.56331011952861 73.23722063237173) bank5752 +5753 POINT(40.22447250078016 73.84503567382191) bank5753 +5754 POINT(41.101733158905404 74.56411049126223) bank5754 +5755 POINT(41.58154647811992 73.07653030851766) bank5755 +5756 POINT(41.544809434356786 74.46378974516047) bank5756 +5757 POINT(40.17198280464987 74.41638900329485) bank5757 +5758 POINT(41.47722018739195 74.70021348420227) bank5758 +5759 POINT(41.05986519866892 74.38292051205465) bank5759 +5760 POINT(39.903063639200376 74.01670379878713) bank5760 +5761 POINT(40.96603415092498 73.68345170666105) bank5761 +5762 POINT(40.149091995426076 74.8231035228574) bank5762 +5763 POINT(40.52669278715238 73.43302881105076) bank5763 +5764 POINT(40.25432384862818 74.39661987740095) bank5764 +5765 POINT(41.08534166526201 74.24310050266644) bank5765 +5766 POINT(41.19439695125791 73.66428043645821) bank5766 +5767 POINT(40.95267729088134 73.10045118859433) bank5767 +5768 POINT(40.72096769475904 73.45857743589556) bank5768 +5769 POINT(40.877830461442876 73.52767156239759) bank5769 +5770 POINT(39.756475723132134 74.5226790431679) bank5770 +5771 POINT(39.977826814107125 73.33394482892915) bank5771 +5772 POINT(39.72829043248773 74.25471986877562) bank5772 +5773 POINT(40.26236969902078 73.97335795427806) bank5773 +5774 POINT(40.34779490832311 74.0070874078561) bank5774 +5775 POINT(40.37660936278633 74.18304811695708) bank5775 +5776 POINT(40.79258278620563 73.51736929013155) bank5776 +5777 POINT(41.462259352245994 73.57030135598316) bank5777 +5778 POINT(40.77389527320113 73.85548911833494) bank5778 +5779 POINT(39.79962968837108 73.98122751732197) bank5779 +5780 POINT(39.89816484319603 74.1499912573692) bank5780 +5781 POINT(41.35667025691467 73.74998467007242) bank5781 +5782 POINT(41.654154033130226 74.25333997410425) bank5782 +5783 POINT(40.2516505004579 73.21499986095459) bank5783 +5784 POINT(41.5138420103919 74.50289470093864) bank5784 +5785 POINT(40.63337533831281 74.15353477986038) bank5785 +5786 POINT(40.13039127563863 74.72614848033464) bank5786 +5787 POINT(41.09062641921865 74.77979071120325) bank5787 +5788 POINT(41.12433732089671 73.8595474123387) bank5788 +5789 POINT(40.835792346579495 74.95715388509072) bank5789 +5790 POINT(40.69639137101372 74.99485331045402) bank5790 +5791 POINT(41.00061199200322 73.03048073781028) bank5791 +5792 POINT(40.66314506153803 73.99649449780033) bank5792 +5793 POINT(41.06045671075209 73.92797877439106) bank5793 +5794 POINT(40.91535561797894 73.99602982124023) bank5794 +5795 POINT(41.56637710238369 73.90056134770181) bank5795 +5796 POINT(40.34268976418412 73.23326114598811) bank5796 +5797 POINT(40.02265340823648 73.07342686311277) bank5797 +5798 POINT(40.104865621767566 73.74839247415663) bank5798 +5799 POINT(41.14686320917034 74.53367995321899) bank5799 +5800 POINT(41.48178320529736 73.77879905980602) bank5800 +5801 POINT(40.23467842776706 74.75553083252433) bank5801 +5802 POINT(40.592722330767344 73.91270495472291) bank5802 +5803 POINT(41.19092804633766 74.45016339002396) bank5803 +5804 POINT(41.17962754505785 74.08844078889551) bank5804 +5805 POINT(40.1340604912256 73.17521543049538) bank5805 +5806 POINT(40.04932078498208 73.16391912542912) bank5806 +5807 POINT(40.621332584486375 74.89230717442668) bank5807 +5808 POINT(40.748663606333324 73.73895968464045) bank5808 +5809 POINT(40.45297320611305 73.46949861220884) bank5809 +5810 POINT(40.35499807931439 74.13791020503095) bank5810 +5811 POINT(40.26290159627079 74.34386085205742) bank5811 +5812 POINT(41.54832953095729 74.51979272790831) bank5812 +5813 POINT(41.607352691547675 74.26710148973908) bank5813 +5814 POINT(41.654019385622426 73.92183325439971) bank5814 +5815 POINT(39.905645222742855 73.06193861004893) bank5815 +5816 POINT(40.95765313892871 73.7943646742375) bank5816 +5817 POINT(40.858325924629284 74.07918352490952) bank5817 +5818 POINT(40.97378675997104 73.059409111828) bank5818 +5819 POINT(40.606789353228 74.68432480482632) bank5819 +5820 POINT(41.702734274616674 74.47080334309035) bank5820 +5821 POINT(39.77782966410205 73.19330560156217) bank5821 +5822 POINT(41.14095441349422 73.93824650954886) bank5822 +5823 POINT(40.29992589555659 74.25536199971855) bank5823 +5824 POINT(40.31198383953173 74.14261259507728) bank5824 +5825 POINT(39.988605493306956 73.31032283654054) bank5825 +5826 POINT(39.88511612884408 74.4976581488188) bank5826 +5827 POINT(40.742650185096196 73.43579090482694) bank5827 +5828 POINT(41.075488652742976 74.859169695507) bank5828 +5829 POINT(41.36373065096441 74.19318476915528) bank5829 +5830 POINT(41.20687128550426 74.87980135596882) bank5830 +5831 POINT(40.582044563222766 74.84022663483526) bank5831 +5832 POINT(40.32569913498504 74.02460058991592) bank5832 +5833 POINT(41.239604696114526 74.77127477201829) bank5833 +5834 POINT(41.085209092064794 74.8062107338576) bank5834 +5835 POINT(41.19308795989194 74.76051606933626) bank5835 +5836 POINT(40.51356758244703 74.87535845527212) bank5836 +5837 POINT(40.626839052066515 74.12738386791455) bank5837 +5838 POINT(41.332428932311196 74.61881385875996) bank5838 +5839 POINT(41.117445189085025 73.67485695578327) bank5839 +5840 POINT(40.351002660597324 73.83122496400746) bank5840 +5841 POINT(39.83874725415286 74.46061481832204) bank5841 +5842 POINT(41.216193755083296 73.1927552382237) bank5842 +5843 POINT(41.26535244366659 74.14305095766021) bank5843 +5844 POINT(40.071338977300904 74.09322628319215) bank5844 +5845 POINT(41.70380505950677 73.63031485831105) bank5845 +5846 POINT(41.0210925417511 73.01958453862429) bank5846 +5847 POINT(40.09563490766142 74.53945576152091) bank5847 +5848 POINT(41.30977675384185 74.5531366705138) bank5848 +5849 POINT(40.51060610368732 73.96555630832759) bank5849 +5850 POINT(40.64047264757269 74.83736390019536) bank5850 +5851 POINT(40.73891333256705 74.6181224180756) bank5851 +5852 POINT(41.50769907689646 74.6227915716252) bank5852 +5853 POINT(40.80717156762059 73.16245957800244) bank5853 +5854 POINT(40.910192236825054 74.4064892894616) bank5854 +5855 POINT(40.919637478043796 73.21293925945162) bank5855 +5856 POINT(40.766297292865666 74.79496827159974) bank5856 +5857 POINT(39.84765443207044 73.14882448237752) bank5857 +5858 POINT(40.1450938247433 74.35175162970181) bank5858 +5859 POINT(40.01739812680196 73.96336329682643) bank5859 +5860 POINT(39.920239087194474 73.92525233913153) bank5860 +5861 POINT(40.566040040351 73.8010566675484) bank5861 +5862 POINT(39.785444753940915 74.0164861357267) bank5862 +5863 POINT(40.374108843543084 74.43456532363744) bank5863 +5864 POINT(39.94370907370656 73.38850748230098) bank5864 +5865 POINT(40.0674613883979 74.00119789027899) bank5865 +5866 POINT(40.11019676522654 74.20357275407943) bank5866 +5867 POINT(40.25858368086981 74.35646532037683) bank5867 +5868 POINT(41.58464417298179 73.0189819685206) bank5868 +5869 POINT(40.20933795959396 73.67517713182136) bank5869 +5870 POINT(41.53273219011883 74.5786617514057) bank5870 +5871 POINT(41.68372321818039 74.92291907204682) bank5871 +5872 POINT(40.45950422637843 74.79835670972932) bank5872 +5873 POINT(40.40184487478046 74.2176163817089) bank5873 +5874 POINT(40.76290139641044 74.52539209382017) bank5874 +5875 POINT(40.730019581532 73.34722973948953) bank5875 +5876 POINT(41.122578996919344 74.79167732172704) bank5876 +5877 POINT(41.06278243594861 74.32983975167949) bank5877 +5878 POINT(39.912514980757685 73.3801523884359) bank5878 +5879 POINT(40.752167574351375 74.74587233858152) bank5879 +5880 POINT(40.680958005455864 74.80693553561974) bank5880 +5881 POINT(40.025922685929466 74.13918394897469) bank5881 +5882 POINT(40.60223448784935 74.07864139115314) bank5882 +5883 POINT(40.86818657582822 74.25173594948248) bank5883 +5884 POINT(40.022643805149464 73.63107750229923) bank5884 +5885 POINT(40.408105587924304 73.73286198110567) bank5885 +5886 POINT(41.10948555838793 74.46711592431393) bank5886 +5887 POINT(40.256780626316754 74.13669288901549) bank5887 +5888 POINT(39.8454146432419 74.33007449023327) bank5888 +5889 POINT(40.22800151088852 74.38588055772752) bank5889 +5890 POINT(39.85846297320953 73.98039299229794) bank5890 +5891 POINT(40.544621282931935 73.97059533921505) bank5891 +5892 POINT(40.09777248703151 73.64558663730878) bank5892 +5893 POINT(39.909535135993025 74.85067884202559) bank5893 +5894 POINT(41.22242591330129 74.09650975794122) bank5894 +5895 POINT(41.41567230887413 74.30716238178894) bank5895 +5896 POINT(40.71991139761443 73.49765906031965) bank5896 +5897 POINT(41.09748183371523 74.65113420619144) bank5897 +5898 POINT(40.94509430586062 73.50657926985971) bank5898 +5899 POINT(41.70548096191615 73.13489447056372) bank5899 +5900 POINT(40.78467901890284 74.43889881968933) bank5900 +5901 POINT(40.84232594144511 73.03064652119856) bank5901 +5902 POINT(41.3581972268312 74.29960386880936) bank5902 +5903 POINT(41.58873167035713 73.87524744907729) bank5903 +5904 POINT(39.88097986771542 74.85250262992237) bank5904 +5905 POINT(40.485842417495085 74.07455745067277) bank5905 +5906 POINT(41.311363730298126 73.92700547282186) bank5906 +5907 POINT(40.450839271584655 73.73652431604889) bank5907 +5908 POINT(41.45983050793307 74.75006415258258) bank5908 +5909 POINT(40.56254104318979 73.98117175247198) bank5909 +5910 POINT(41.24740119112113 73.1483152858205) bank5910 +5911 POINT(40.67135340117348 73.09889186546921) bank5911 +5912 POINT(41.30405453453601 74.86218515737986) bank5912 +5913 POINT(41.12034079340052 74.84109394743555) bank5913 +5914 POINT(40.02652891165328 74.56726009794636) bank5914 +5915 POINT(40.02605093793045 73.15179308577119) bank5915 +5916 POINT(40.217901034834334 74.97120893592138) bank5916 +5917 POINT(40.83592497575519 74.86718535930935) bank5917 +5918 POINT(41.13884169525892 74.51849568205203) bank5918 +5919 POINT(41.3732106106683 73.75276826338406) bank5919 +5920 POINT(40.915375702851996 73.20582718496364) bank5920 +5921 POINT(41.09754286927534 73.9293590243385) bank5921 +5922 POINT(40.797170865865034 73.2051099003504) bank5922 +5923 POINT(41.09036733044914 74.53171716129401) bank5923 +5924 POINT(40.154856097268684 74.14656683384585) bank5924 +5925 POINT(41.3086238478046 74.85679427127458) bank5925 +5926 POINT(40.125331980316446 73.97405903659414) bank5926 +5927 POINT(40.349609522798595 73.58592422209702) bank5927 +5928 POINT(40.670898319234176 74.65083443166695) bank5928 +5929 POINT(41.62035972058601 73.65177573134261) bank5929 +5930 POINT(41.5510972232233 73.70723757375539) bank5930 +5931 POINT(41.018886720994345 74.54317928565754) bank5931 +5932 POINT(39.755774304036734 73.22368179146325) bank5932 +5933 POINT(39.856137254961276 73.8514708960472) bank5933 +5934 POINT(40.06372644576382 74.9175231141148) bank5934 +5935 POINT(41.602915240229564 73.78850595430592) bank5935 +5936 POINT(39.960744283398924 73.72409600195795) bank5936 +5937 POINT(41.057985856242 73.88423454126264) bank5937 +5938 POINT(40.37328371410876 74.34415521844696) bank5938 +5939 POINT(40.728720866736374 74.11221347941535) bank5939 +5940 POINT(40.41813906848595 74.12882780240079) bank5940 +5941 POINT(39.975940956086724 74.15083990107031) bank5941 +5942 POINT(39.785911848653036 73.27544300204667) bank5942 +5943 POINT(40.26280293614056 73.07824927030492) bank5943 +5944 POINT(40.60209821617316 73.68392337748301) bank5944 +5945 POINT(40.883604460982134 73.60681499958311) bank5945 +5946 POINT(41.36242801549696 74.89530837803568) bank5946 +5947 POINT(40.19267397037617 74.29609171552582) bank5947 +5948 POINT(39.76898369063312 74.6789006961682) bank5948 +5949 POINT(41.15964865065622 73.22880948801051) bank5949 +5950 POINT(41.29519158523693 74.40423585668304) bank5950 +5951 POINT(41.43988727857558 73.94619784695087) bank5951 +5952 POINT(41.28769453199848 73.61705961135674) bank5952 +5953 POINT(40.918066370397476 74.96101534510936) bank5953 +5954 POINT(40.702394308709586 74.80881187469497) bank5954 +5955 POINT(39.92538581435397 74.49001294510175) bank5955 +5956 POINT(41.65948917831767 74.14096911194999) bank5956 +5957 POINT(40.25591885337184 73.0588696411033) bank5957 +5958 POINT(41.50102923737594 73.48881498767011) bank5958 +5959 POINT(41.08976035045287 74.30193709498782) bank5959 +5960 POINT(40.60220772806435 74.15792335840266) bank5960 +5961 POINT(41.59001761125582 74.86843997575033) bank5961 +5962 POINT(40.564196461678655 74.26108153962146) bank5962 +5963 POINT(40.22623392160118 73.30845754579794) bank5963 +5964 POINT(40.612362825959465 74.51484121683475) bank5964 +5965 POINT(39.73145664632932 74.71200244891105) bank5965 +5966 POINT(39.98786844067494 73.79700826542421) bank5966 +5967 POINT(40.12546792025178 73.84946130494123) bank5967 +5968 POINT(41.678002068779605 74.48746176853814) bank5968 +5969 POINT(40.14486234256573 73.52904139776346) bank5969 +5970 POINT(41.191256091275996 74.72445547031963) bank5970 +5971 POINT(40.734722860367384 73.78962325634497) bank5971 +5972 POINT(41.56680340855055 73.98115074659032) bank5972 +5973 POINT(40.41080143557002 74.08120835470356) bank5973 +5974 POINT(40.32720574092517 74.7904595280075) bank5974 +5975 POINT(40.95779978935171 73.58186801880507) bank5975 +5976 POINT(41.578804893003024 73.5683611637355) bank5976 +5977 POINT(39.96774826756453 74.28835901357579) bank5977 +5978 POINT(40.421665141204755 74.03029445371104) bank5978 +5979 POINT(40.19172169598699 74.97491369274054) bank5979 +5980 POINT(41.425328777682196 73.58514646180296) bank5980 +5981 POINT(41.14932537465726 73.77409804288298) bank5981 +5982 POINT(41.220367357610584 74.1454197536103) bank5982 +5983 POINT(41.18650841076309 74.04642584881636) bank5983 +5984 POINT(40.08842264640587 73.657323357384) bank5984 +5985 POINT(40.99763194843713 74.16676554937165) bank5985 +5986 POINT(40.15301560153319 74.58133989361878) bank5986 +5987 POINT(41.39898215615946 74.19364691687991) bank5987 +5988 POINT(40.68140190362505 74.8696161117361) bank5988 +5989 POINT(40.8337863720161 74.44270835020326) bank5989 +5990 POINT(40.613314548521714 74.22694580649508) bank5990 +5991 POINT(40.60687976488195 73.1895540563785) bank5991 +5992 POINT(41.32810908669591 73.60182768820609) bank5992 +5993 POINT(40.827013394397156 73.45708882833677) bank5993 +5994 POINT(39.84534903356384 73.96662895768422) bank5994 +5995 POINT(40.586027354617414 73.60445694715057) bank5995 +5996 POINT(40.74290856698258 74.22460220408782) bank5996 +5997 POINT(41.2800118674859 74.26843699289074) bank5997 +5998 POINT(41.084220085022366 73.85668097443477) bank5998 +5999 POINT(39.975969834979935 73.09882843087921) bank5999 +6000 POINT(40.45285011914864 74.12108776715449) bank6000 +6001 POINT(40.30489239429663 74.82413081349858) bank6001 +6002 POINT(41.68666275809307 74.40083017539463) bank6002 +6003 POINT(40.36914937345427 74.03872090740703) bank6003 +6004 POINT(40.788723235864545 73.64798666913038) bank6004 +6005 POINT(40.552461405463085 73.2013885901019) bank6005 +6006 POINT(40.86123357253737 74.00272231205072) bank6006 +6007 POINT(41.32563917212215 73.72765822963305) bank6007 +6008 POINT(40.68808350495724 74.22208451830198) bank6008 +6009 POINT(40.38825876952454 74.01831887620635) bank6009 +6010 POINT(40.33272531801048 74.30666611825255) bank6010 +6011 POINT(40.348200932088474 73.95431200072996) bank6011 +6012 POINT(41.06994538239893 73.79410506435379) bank6012 +6013 POINT(39.87246443056789 73.91465207071141) bank6013 +6014 POINT(40.47660331880708 73.76293498707469) bank6014 +6015 POINT(41.025148475812315 73.7459800435762) bank6015 +6016 POINT(40.59090368795901 73.92390487694243) bank6016 +6017 POINT(39.97185150658065 74.5937532587891) bank6017 +6018 POINT(41.4837652410493 74.54088595575345) bank6018 +6019 POINT(40.286309249288166 73.19213963493108) bank6019 +6020 POINT(41.553184408454264 73.5069493931964) bank6020 +6021 POINT(40.82955882369845 74.81538415888318) bank6021 +6022 POINT(39.99040265714586 74.59186483999291) bank6022 +6023 POINT(40.90065728453336 73.68810685375384) bank6023 +6024 POINT(40.15763492288369 73.61694848915253) bank6024 +6025 POINT(40.412899693111704 74.45647997589461) bank6025 +6026 POINT(41.48810600193439 73.20974644945518) bank6026 +6027 POINT(40.16752865154585 74.4644230750285) bank6027 +6028 POINT(41.6597757485733 73.34634358450528) bank6028 +6029 POINT(41.4048305533573 74.27822768789275) bank6029 +6030 POINT(39.944822621878764 74.19291094524026) bank6030 +6031 POINT(40.41716458521811 73.74516502480343) bank6031 +6032 POINT(41.16032016958425 73.1543063121946) bank6032 +6033 POINT(40.01566237481219 73.64149282917504) bank6033 +6034 POINT(41.410545481157094 73.62533815071008) bank6034 +6035 POINT(40.52857910527307 74.89112140424027) bank6035 +6036 POINT(40.14979724651685 73.81254875162514) bank6036 +6037 POINT(39.944691188005656 74.17158334037265) bank6037 +6038 POINT(40.96691633939003 73.46433660825826) bank6038 +6039 POINT(41.09790110978224 73.22361401064336) bank6039 +6040 POINT(40.079327122464775 74.83576048250629) bank6040 +6041 POINT(41.56100507883868 74.82374792043112) bank6041 +6042 POINT(40.37242657528389 73.69376686497425) bank6042 +6043 POINT(39.987783739451345 73.71639646074313) bank6043 +6044 POINT(41.515425910178216 73.55399315021401) bank6044 +6045 POINT(40.770565546953215 74.95457576667992) bank6045 +6046 POINT(39.88025832691435 74.389511717412) bank6046 +6047 POINT(40.14593803478329 73.74290187860728) bank6047 +6048 POINT(40.251668098871 74.45233985600701) bank6048 +6049 POINT(40.63952585286781 73.79100246184016) bank6049 +6050 POINT(41.52340305050175 74.56262557015052) bank6050 +6051 POINT(41.643230112902046 74.8978255097841) bank6051 +6052 POINT(40.54933474006528 74.24553840010182) bank6052 +6053 POINT(40.28813335326652 74.38418200818525) bank6053 +6054 POINT(40.86450332496732 73.82798198137765) bank6054 +6055 POINT(40.99081168101531 74.8625049481287) bank6055 +6056 POINT(41.503314995781416 74.09957726908344) bank6056 +6057 POINT(40.54441676204783 74.12516192756152) bank6057 +6058 POINT(41.5165041082516 73.92251540465676) bank6058 +6059 POINT(41.222321636175906 73.76780702389945) bank6059 +6060 POINT(40.207735193329746 73.96292202450802) bank6060 +6061 POINT(40.3341510989878 73.49640509597457) bank6061 +6062 POINT(41.56243012326482 74.1747461455277) bank6062 +6063 POINT(40.07917321963711 73.57693394706034) bank6063 +6064 POINT(40.53520091194428 73.48154088347863) bank6064 +6065 POINT(40.73873196973235 74.41981940028062) bank6065 +6066 POINT(41.032626988267715 73.2051678726268) bank6066 +6067 POINT(40.56023840778667 73.08756954085688) bank6067 +6068 POINT(40.58602421036375 73.2075648080169) bank6068 +6069 POINT(40.757556469262255 73.24124680679003) bank6069 +6070 POINT(40.2490691689196 73.24757794359243) bank6070 +6071 POINT(39.91146979444467 74.17983118937288) bank6071 +6072 POINT(40.66985568001605 74.9927732780437) bank6072 +6073 POINT(40.98241411736606 74.57792665893048) bank6073 +6074 POINT(40.21584832081923 74.73869950230858) bank6074 +6075 POINT(39.80512862944476 73.05686181784526) bank6075 +6076 POINT(40.53328325548339 73.7491327328387) bank6076 +6077 POINT(39.77196577735961 73.65535085109626) bank6077 +6078 POINT(39.79547849007807 74.47955432643244) bank6078 +6079 POINT(41.15404989556699 74.14265501391193) bank6079 +6080 POINT(40.96261356845805 73.09607262919434) bank6080 +6081 POINT(39.840763016251394 73.622304177463) bank6081 +6082 POINT(40.86194649913334 73.8085700406493) bank6082 +6083 POINT(39.7531021101074 74.56089252636593) bank6083 +6084 POINT(39.8527898301832 74.86409445980487) bank6084 +6085 POINT(41.71006628399215 73.84704001706191) bank6085 +6086 POINT(41.08147993327514 74.25091480574214) bank6086 +6087 POINT(40.41806857178359 74.6301578729869) bank6087 +6088 POINT(39.98264847137264 73.96748189277551) bank6088 +6089 POINT(41.126959791372016 74.72960606149003) bank6089 +6090 POINT(40.625174361053304 73.67929342150208) bank6090 +6091 POINT(41.252597881802075 74.26702302966865) bank6091 +6092 POINT(41.51148479714847 73.53687203070015) bank6092 +6093 POINT(40.732108224606584 74.15724266594633) bank6093 +6094 POINT(41.32347040611802 74.16955508882603) bank6094 +6095 POINT(39.94448715306084 73.12409500299897) bank6095 +6096 POINT(39.90785303638899 74.9290640683879) bank6096 +6097 POINT(40.66960520856853 73.58884858880215) bank6097 +6098 POINT(40.05291006061339 74.59463570677507) bank6098 +6099 POINT(40.446136093713186 74.49543424925893) bank6099 +6100 POINT(40.979436292577766 73.35187847865078) bank6100 +6101 POINT(39.95352748642501 74.57962667660125) bank6101 +6102 POINT(41.65867050988886 73.6766271603824) bank6102 +6103 POINT(39.79569058463065 74.69228211829531) bank6103 +6104 POINT(39.78589408058671 73.54806592669165) bank6104 +6105 POINT(40.71070070550335 74.18142048456355) bank6105 +6106 POINT(40.27273920759496 73.9266537771251) bank6106 +6107 POINT(40.31549877178031 73.39159308305943) bank6107 +6108 POINT(41.36632351018872 73.52752746954103) bank6108 +6109 POINT(39.98448681788029 73.17105899895235) bank6109 +6110 POINT(40.348868408690045 74.32943072238258) bank6110 +6111 POINT(39.77674073253163 73.77097684153114) bank6111 +6112 POINT(39.93037667664732 73.68873105215008) bank6112 +6113 POINT(40.578405498187095 74.61769513807857) bank6113 +6114 POINT(40.14035825829066 73.46255663269461) bank6114 +6115 POINT(40.33617119047548 73.42801060819998) bank6115 +6116 POINT(41.48939937800962 74.62444240460178) bank6116 +6117 POINT(40.790325897070495 73.2241463044854) bank6117 +6118 POINT(40.284759179034964 73.80557309417368) bank6118 +6119 POINT(41.29656007037914 73.89830195245194) bank6119 +6120 POINT(40.020866145605574 74.66426895724317) bank6120 +6121 POINT(40.25363896881847 73.845635707796) bank6121 +6122 POINT(39.86903004302336 73.12627971029511) bank6122 +6123 POINT(41.22753045470443 74.42570486645518) bank6123 +6124 POINT(39.73095442285738 73.70224158724402) bank6124 +6125 POINT(40.80947365388307 74.40498943060005) bank6125 +6126 POINT(40.0849901878951 74.59187228011737) bank6126 +6127 POINT(39.863458271886486 73.54358248280927) bank6127 +6128 POINT(41.600215453941495 73.82050942903307) bank6128 +6129 POINT(41.08912963507007 74.65732996482055) bank6129 +6130 POINT(41.144268592539106 73.95437263223576) bank6130 +6131 POINT(41.073080155032244 74.9994004632046) bank6131 +6132 POINT(40.36238754167993 74.3703139983478) bank6132 +6133 POINT(40.102107004705815 73.42459198332874) bank6133 +6134 POINT(40.5601787765597 74.07029448438398) bank6134 +6135 POINT(41.42479618069199 73.850451198436) bank6135 +6136 POINT(39.81625044457985 74.30013179074719) bank6136 +6137 POINT(41.29981133629558 73.22452134158355) bank6137 +6138 POINT(41.18080584157447 74.0690721743715) bank6138 +6139 POINT(41.692788416334814 73.72448157609495) bank6139 +6140 POINT(40.51319647710487 73.28336639443798) bank6140 +6141 POINT(40.798980816543434 74.69763378090205) bank6141 +6142 POINT(40.57125262003858 73.62529585273117) bank6142 +6143 POINT(39.73833842985961 73.66734307795397) bank6143 +6144 POINT(41.388055861481675 73.9278552630644) bank6144 +6145 POINT(40.28615242839028 74.2738790174247) bank6145 +6146 POINT(39.75818431095462 74.73549372118384) bank6146 +6147 POINT(40.77621137933757 74.93983932887566) bank6147 +6148 POINT(39.79250368352705 73.33463448271674) bank6148 +6149 POINT(41.064113839631624 73.66164843699364) bank6149 +6150 POINT(41.694525062570534 74.20125035033725) bank6150 +6151 POINT(41.0734340111192 73.64205335376194) bank6151 +6152 POINT(41.285966054501166 74.65351125397093) bank6152 +6153 POINT(41.52559876473138 74.2694030802401) bank6153 +6154 POINT(40.25001019861119 74.36601571076248) bank6154 +6155 POINT(40.96926462272287 73.12904860492266) bank6155 +6156 POINT(41.65256930474305 73.32821139769078) bank6156 +6157 POINT(41.38457887985626 73.69189984903713) bank6157 +6158 POINT(39.961714043149 73.81816642009802) bank6158 +6159 POINT(40.417284877605994 73.15973983716877) bank6159 +6160 POINT(40.935472771368126 74.40855745538319) bank6160 +6161 POINT(39.72714774802519 74.7028062005671) bank6161 +6162 POINT(40.751280307171285 73.19116915328978) bank6162 +6163 POINT(39.8716742735499 73.16687869341112) bank6163 +6164 POINT(39.995582624118306 73.72695266213104) bank6164 +6165 POINT(41.14836786627967 73.99706598879445) bank6165 +6166 POINT(41.32563498064793 74.91300634345596) bank6166 +6167 POINT(41.22183001565545 73.97723132714428) bank6167 +6168 POINT(40.53024530945301 73.8238373978615) bank6168 +6169 POINT(40.202067633568426 74.58510370439753) bank6169 +6170 POINT(40.97112083467852 73.28979789539663) bank6170 +6171 POINT(40.059194367536755 73.11055043830686) bank6171 +6172 POINT(41.342044490283506 73.2264464743373) bank6172 +6173 POINT(40.935617397485444 73.37691389511934) bank6173 +6174 POINT(40.20846960916427 74.80970057865603) bank6174 +6175 POINT(39.93232229837268 74.60009165844754) bank6175 +6176 POINT(40.03483980786394 74.02397641575537) bank6176 +6177 POINT(39.75566856242395 73.47251460881897) bank6177 +6178 POINT(41.57171538557479 74.18743910158572) bank6178 +6179 POINT(40.38281733939858 74.47884627125012) bank6179 +6180 POINT(39.96901325221236 74.04684251529271) bank6180 +6181 POINT(39.9409024220678 74.05975995291423) bank6181 +6182 POINT(40.1751263272368 73.45202102147435) bank6182 +6183 POINT(41.67312898659917 74.43484830914738) bank6183 +6184 POINT(41.31803376193544 73.06990367703297) bank6184 +6185 POINT(40.529120002850384 73.10021936684396) bank6185 +6186 POINT(41.4753763698927 74.865257477902) bank6186 +6187 POINT(39.782421407373704 74.32077128175357) bank6187 +6188 POINT(40.53579520876474 74.95240632187893) bank6188 +6189 POINT(40.925955973189005 73.87559903220516) bank6189 +6190 POINT(40.31479496187213 74.18834155994041) bank6190 +6191 POINT(39.758475577852764 73.1552391416507) bank6191 +6192 POINT(40.7873608556673 73.72307399334152) bank6192 +6193 POINT(40.86791313183587 74.95538995709876) bank6193 +6194 POINT(39.726371750555224 74.49204069831346) bank6194 +6195 POINT(41.0306598091247 73.44815331461758) bank6195 +6196 POINT(40.98117387530765 73.73450760263835) bank6196 +6197 POINT(41.05348421173143 74.502411089068) bank6197 +6198 POINT(40.82047781851491 74.63843529265094) bank6198 +6199 POINT(41.662397077523465 74.50838123780387) bank6199 +6200 POINT(39.73002634202944 74.90595141003602) bank6200 +6201 POINT(41.096613924413035 73.69791591213861) bank6201 +6202 POINT(40.999789216358 73.75026353100573) bank6202 +6203 POINT(41.599103706181424 74.66872518496291) bank6203 +6204 POINT(41.09080234705857 74.6426252177322) bank6204 +6205 POINT(41.54584994646163 73.62165088539037) bank6205 +6206 POINT(39.90818812795179 73.24550058654353) bank6206 +6207 POINT(41.48361704900287 74.49468559680513) bank6207 +6208 POINT(40.16625664176936 73.98017376356557) bank6208 +6209 POINT(41.28084490222627 74.89102887771057) bank6209 +6210 POINT(40.7455006651345 73.91277258388318) bank6210 +6211 POINT(40.17038513621046 73.56521634647014) bank6211 +6212 POINT(40.459933396135796 74.39101262954883) bank6212 +6213 POINT(39.74161171664516 73.86115205135074) bank6213 +6214 POINT(40.29840536466889 73.25547553221251) bank6214 +6215 POINT(41.530705864858575 73.24677221391634) bank6215 +6216 POINT(39.731526191721066 73.22795152212291) bank6216 +6217 POINT(41.11990015143744 73.33406554837113) bank6217 +6218 POINT(40.684647985274125 74.25921373791131) bank6218 +6219 POINT(40.624703210464496 73.89191530180229) bank6219 +6220 POINT(39.8274921515968 73.93295315114439) bank6220 +6221 POINT(41.48634365831736 74.61312040777277) bank6221 +6222 POINT(41.65809034042689 73.76059579455502) bank6222 +6223 POINT(41.19021323718616 74.61393326429388) bank6223 +6224 POINT(41.27882923277247 74.67810629565324) bank6224 +6225 POINT(40.114093725657135 74.7397253817555) bank6225 +6226 POINT(41.64452832898732 73.73228280013696) bank6226 +6227 POINT(40.016486403552705 74.25434522154833) bank6227 +6228 POINT(40.73578395156941 74.15010122426126) bank6228 +6229 POINT(41.5390720511961 73.29461512350544) bank6229 +6230 POINT(40.91416584098282 73.24865917019837) bank6230 +6231 POINT(40.73482727856524 73.82100661279887) bank6231 +6232 POINT(40.97624937630363 74.08939610777573) bank6232 +6233 POINT(41.615179038067765 74.2610103768079) bank6233 +6234 POINT(40.323658110670884 74.31184878826622) bank6234 +6235 POINT(40.00709817969463 73.5491528220473) bank6235 +6236 POINT(41.59219843173331 73.90650405336831) bank6236 +6237 POINT(40.30128251784586 73.0269366682476) bank6237 +6238 POINT(41.34723274562978 73.86431611202691) bank6238 +6239 POINT(40.7903279127255 74.33275728016787) bank6239 +6240 POINT(40.152985892432 74.53718226100779) bank6240 +6241 POINT(40.03363667363982 74.4538170721373) bank6241 +6242 POINT(41.36681841686018 74.15982870968945) bank6242 +6243 POINT(41.091684825828665 73.2703046574544) bank6243 +6244 POINT(41.64102504479421 74.95292597317318) bank6244 +6245 POINT(40.55349428146828 74.17677892976215) bank6245 +6246 POINT(41.659003704986276 73.83299942096941) bank6246 +6247 POINT(39.921349051065384 74.49678030013155) bank6247 +6248 POINT(40.991368765665605 74.33079088143322) bank6248 +6249 POINT(39.984135283409486 74.47363871652796) bank6249 +6250 POINT(39.94164444278493 74.88975206781595) bank6250 +6251 POINT(41.33064827496971 74.59802331713787) bank6251 +6252 POINT(40.33998856243743 74.08546639680085) bank6252 +6253 POINT(40.30808846961479 74.90512188177999) bank6253 +6254 POINT(40.29311334018244 74.79462630009951) bank6254 +6255 POINT(39.826205427251615 74.04616383469178) bank6255 +6256 POINT(40.87873918858793 73.55095800361154) bank6256 +6257 POINT(40.9099551563573 74.9794911731555) bank6257 +6258 POINT(40.38998997932266 73.72528198501001) bank6258 +6259 POINT(41.390371086255755 73.81922977563111) bank6259 +6260 POINT(39.965302578783046 73.71350843765059) bank6260 +6261 POINT(39.744177966969346 73.06502208948058) bank6261 +6262 POINT(40.098058445612494 73.93587456427) bank6262 +6263 POINT(39.89597008794675 74.16310334139621) bank6263 +6264 POINT(40.07088789447496 74.02232265556697) bank6264 +6265 POINT(40.500591141282975 73.61367097173589) bank6265 +6266 POINT(41.70324361626355 74.37360952676293) bank6266 +6267 POINT(40.53181096044603 74.74557488928828) bank6267 +6268 POINT(40.217977748514485 73.91967681111728) bank6268 +6269 POINT(39.910447897057445 74.98047650800322) bank6269 +6270 POINT(41.27000461199699 74.78252042559292) bank6270 +6271 POINT(40.77760989032634 74.38194930631086) bank6271 +6272 POINT(40.291063731097516 73.35615057714514) bank6272 +6273 POINT(40.59812839810101 73.43288220266165) bank6273 +6274 POINT(41.578522541289665 73.9937901375425) bank6274 +6275 POINT(40.540433935548975 73.0682237546067) bank6275 +6276 POINT(41.62847466506994 73.74925462855411) bank6276 +6277 POINT(41.51241427465966 74.65479631467554) bank6277 +6278 POINT(40.732802251588794 73.55284051147868) bank6278 +6279 POINT(39.93781947282622 73.0389717261751) bank6279 +6280 POINT(40.57857657938444 74.5880776181146) bank6280 +6281 POINT(40.10286217370189 74.55706274484994) bank6281 +6282 POINT(39.98711153504028 73.61677977359851) bank6282 +6283 POINT(40.9651254916906 73.84356740015558) bank6283 +6284 POINT(40.33178835600008 73.86688478823959) bank6284 +6285 POINT(41.077532830551505 73.43312461501671) bank6285 +6286 POINT(40.56756948433584 74.20436799625361) bank6286 +6287 POINT(41.00213551890569 73.25664282300292) bank6287 +6288 POINT(41.20889978874062 73.6748562041444) bank6288 +6289 POINT(40.46302180931162 74.56703322854852) bank6289 +6290 POINT(40.47077221618223 73.82499071646583) bank6290 +6291 POINT(40.97512865357273 74.97734915028363) bank6291 +6292 POINT(40.00176437621 73.23969303784355) bank6292 +6293 POINT(41.35471389766561 74.97729283272784) bank6293 +6294 POINT(41.31049133540953 73.11074239116515) bank6294 +6295 POINT(40.586800798391806 73.20982770470508) bank6295 +6296 POINT(41.167753749923804 74.6731831594791) bank6296 +6297 POINT(41.048558873596555 73.23676206641932) bank6297 +6298 POINT(41.28045388636492 74.74784081020738) bank6298 +6299 POINT(40.22536858567496 74.8369890634529) bank6299 +6300 POINT(40.41704300804177 73.58322830284301) bank6300 +6301 POINT(40.13011286494811 73.82467116665794) bank6301 +6302 POINT(41.5942684983841 74.60257919021225) bank6302 +6303 POINT(41.23188290959417 74.91080305064453) bank6303 +6304 POINT(39.839549985437245 74.56881877144407) bank6304 +6305 POINT(40.92565877480028 73.58443896540113) bank6305 +6306 POINT(39.99993925745443 73.51392566734626) bank6306 +6307 POINT(41.42979632356466 74.38578900109118) bank6307 +6308 POINT(40.546696914214266 73.6037112469101) bank6308 +6309 POINT(40.24372364898 74.51707553712265) bank6309 +6310 POINT(40.001769771246565 73.59688261360674) bank6310 +6311 POINT(39.8629286902421 74.01179369155068) bank6311 +6312 POINT(41.0173106612082 73.55953242057663) bank6312 +6313 POINT(39.796802411348466 75.00006887118677) bank6313 +6314 POINT(40.073108415135735 73.85631747083899) bank6314 +6315 POINT(40.15684984456696 74.4915725600295) bank6315 +6316 POINT(40.17009813805077 73.86395971348459) bank6316 +6317 POINT(40.38917102421672 74.55873472318639) bank6317 +6318 POINT(41.259675944172365 74.45270345899142) bank6318 +6319 POINT(40.175154494185925 74.06107093312951) bank6319 +6320 POINT(41.07649877040721 73.3583052803178) bank6320 +6321 POINT(40.73410505901028 74.924474290885) bank6321 +6322 POINT(40.188619075238215 74.89690882105971) bank6322 +6323 POINT(41.244789031632294 74.64161862222731) bank6323 +6324 POINT(41.08577200196787 73.92794551919965) bank6324 +6325 POINT(40.64350259998376 73.33375360723663) bank6325 +6326 POINT(39.99170700240593 74.75280162602309) bank6326 +6327 POINT(40.06232468173344 74.61175393223878) bank6327 +6328 POINT(41.5309630079148 73.60074826979454) bank6328 +6329 POINT(40.97775141939708 74.48291520317112) bank6329 +6330 POINT(40.25664515781206 73.01139572185544) bank6330 +6331 POINT(40.297602493156994 74.78550490981861) bank6331 +6332 POINT(40.32214353891913 73.04368401284842) bank6332 +6333 POINT(40.27576548679407 73.05780791853242) bank6333 +6334 POINT(39.73174578120356 74.71953198169263) bank6334 +6335 POINT(40.40781377675339 73.08609765379772) bank6335 +6336 POINT(40.31352225283321 74.37402704728304) bank6336 +6337 POINT(40.5439295811262 74.71821873197091) bank6337 +6338 POINT(40.80615610169861 73.27688653622899) bank6338 +6339 POINT(41.35051299583351 74.74167600523606) bank6339 +6340 POINT(41.46191132930045 73.3539752734407) bank6340 +6341 POINT(40.8268910831125 74.72324983535555) bank6341 +6342 POINT(40.34742553138864 73.27910208991578) bank6342 +6343 POINT(40.297025113179856 73.84458884975942) bank6343 +6344 POINT(41.42886400213267 73.85059497002966) bank6344 +6345 POINT(41.02343031948412 73.79670880116741) bank6345 +6346 POINT(41.49969989744621 74.12124145284666) bank6346 +6347 POINT(41.207424562739384 73.29483157431804) bank6347 +6348 POINT(39.97827517683605 73.60225625666189) bank6348 +6349 POINT(41.22482709415572 73.78089872378816) bank6349 +6350 POINT(40.32513874456648 73.51738670296712) bank6350 +6351 POINT(41.089207208843355 73.49093592546494) bank6351 +6352 POINT(40.90519825779245 73.30710085292723) bank6352 +6353 POINT(39.9996787445625 74.57407156057536) bank6353 +6354 POINT(41.318405086757075 73.93359819560239) bank6354 +6355 POINT(40.37403792853614 73.54720023312346) bank6355 +6356 POINT(41.63849319242823 74.52323873977448) bank6356 +6357 POINT(40.24125875127547 74.6263253576175) bank6357 +6358 POINT(41.30897478821387 73.22387482587044) bank6358 +6359 POINT(41.53746583826005 74.77885026654113) bank6359 +6360 POINT(40.56082080730594 73.47109868740661) bank6360 +6361 POINT(40.375599846721656 74.76377747427996) bank6361 +6362 POINT(41.37316233298219 73.92711440027198) bank6362 +6363 POINT(39.76045272040458 74.22660480699955) bank6363 +6364 POINT(40.5141584405874 73.24488779523504) bank6364 +6365 POINT(40.728901946968875 73.83113277281343) bank6365 +6366 POINT(40.339795655027544 74.03905506235097) bank6366 +6367 POINT(40.69481731823512 73.39676250579109) bank6367 +6368 POINT(40.46462446386635 74.56167904528931) bank6368 +6369 POINT(40.297516940099044 73.39785905697802) bank6369 +6370 POINT(40.6907635887536 74.00613697640465) bank6370 +6371 POINT(40.923781789080714 75.00324622910566) bank6371 +6372 POINT(39.73702841543282 73.51377052245768) bank6372 +6373 POINT(41.02540522914723 74.69441697085767) bank6373 +6374 POINT(41.31673728336583 74.70921887341338) bank6374 +6375 POINT(40.93907762193693 73.63221079450256) bank6375 +6376 POINT(40.77325980362051 73.87944872234645) bank6376 +6377 POINT(40.918831639833186 74.19144753692285) bank6377 +6378 POINT(40.16984630468205 73.3077616426751) bank6378 +6379 POINT(41.553818144735295 74.67919246170898) bank6379 +6380 POINT(41.19931580130502 73.25724074487823) bank6380 +6381 POINT(41.65434373625537 73.40459702113154) bank6381 +6382 POINT(40.72227449107788 73.44555994567655) bank6382 +6383 POINT(41.07876494081451 74.06274970891917) bank6383 +6384 POINT(39.77093129578303 73.9482735077878) bank6384 +6385 POINT(41.441757126006394 73.4385612069268) bank6385 +6386 POINT(39.75601956602929 74.14817725207945) bank6386 +6387 POINT(41.14083488764645 74.04433068360672) bank6387 +6388 POINT(41.35600017925793 74.5388436372407) bank6388 +6389 POINT(39.87749784206444 74.56708938575264) bank6389 +6390 POINT(40.85191748403636 73.60438280831484) bank6390 +6391 POINT(41.65985604625203 73.89885946933072) bank6391 +6392 POINT(41.654365736707376 73.15732527972435) bank6392 +6393 POINT(40.64894785091449 74.06576002356927) bank6393 +6394 POINT(41.486401772891945 73.5327862552035) bank6394 +6395 POINT(39.73136830337533 73.80579368717694) bank6395 +6396 POINT(40.92955682917615 73.09828009869588) bank6396 +6397 POINT(41.52105202744831 74.9605247282726) bank6397 +6398 POINT(40.45081139155093 73.4614952389185) bank6398 +6399 POINT(41.26099899276011 74.18013294469877) bank6399 +6400 POINT(40.830742532339066 73.474782786154) bank6400 +6401 POINT(40.14600680057137 73.10951519786869) bank6401 +6402 POINT(40.304788374065964 74.63494206434248) bank6402 +6403 POINT(40.45522569757031 73.47185683944662) bank6403 +6404 POINT(41.54189896482822 73.81134937539962) bank6404 +6405 POINT(41.71165874480709 73.0594538192446) bank6405 +6406 POINT(40.36375196674387 74.353412426945) bank6406 +6407 POINT(40.76769238062204 73.71242196243698) bank6407 +6408 POINT(40.797668139797096 73.79955410779648) bank6408 +6409 POINT(39.7294386472238 74.50512540125841) bank6409 +6410 POINT(41.24521928425362 73.43772103222021) bank6410 +6411 POINT(41.32578580497006 74.94502387515584) bank6411 +6412 POINT(41.07904204318776 74.6146171834446) bank6412 +6413 POINT(40.0997895223471 74.67414642809993) bank6413 +6414 POINT(40.85906892957651 73.51415065314858) bank6414 +6415 POINT(41.186726260586774 74.26561147338371) bank6415 +6416 POINT(41.31714576441388 73.66945862049072) bank6416 +6417 POINT(40.65921654395614 73.05070291892191) bank6417 +6418 POINT(39.82098285569323 74.752878691505) bank6418 +6419 POINT(40.01477459088743 74.14023764150616) bank6419 +6420 POINT(41.01985619044549 74.41065942881815) bank6420 +6421 POINT(41.21829344311476 74.62800855256701) bank6421 +6422 POINT(40.27626142516658 73.05830767419619) bank6422 +6423 POINT(41.234172397397685 74.66382749915404) bank6423 +6424 POINT(40.57333295847311 73.53440395791448) bank6424 +6425 POINT(41.545978059141866 74.90370449632718) bank6425 +6426 POINT(40.76069930695242 73.29673828920808) bank6426 +6427 POINT(41.112722512270345 73.71792853271747) bank6427 +6428 POINT(39.8733131375461 74.28160463396729) bank6428 +6429 POINT(39.84632973780155 74.67475201521192) bank6429 +6430 POINT(39.787401864989604 74.87069571781272) bank6430 +6431 POINT(39.823338419545934 74.89142668768396) bank6431 +6432 POINT(40.83503247197294 73.23458963367626) bank6432 +6433 POINT(40.35498653731148 73.9190419455314) bank6433 +6434 POINT(40.07481033792839 74.74261539852914) bank6434 +6435 POINT(41.55834867056925 74.3478240568476) bank6435 +6436 POINT(39.712865340851494 74.8134352305946) bank6436 +6437 POINT(39.86938602301098 74.24830024628663) bank6437 +6438 POINT(40.37522705504563 74.85811956077234) bank6438 +6439 POINT(40.855654125809245 73.53066095017961) bank6439 +6440 POINT(41.243716441516156 74.10876002057468) bank6440 +6441 POINT(41.06836604031377 74.14825124469941) bank6441 +6442 POINT(40.431417316431684 74.5775967950341) bank6442 +6443 POINT(41.17345322382092 74.98465952209935) bank6443 +6444 POINT(41.182070830395524 73.16900697828339) bank6444 +6445 POINT(40.56379641413541 73.4887226268676) bank6445 +6446 POINT(40.509246534564134 74.58543261944261) bank6446 +6447 POINT(39.96743912891639 73.661377826278) bank6447 +6448 POINT(40.61093183456177 74.26974072234812) bank6448 +6449 POINT(41.28450061933295 74.5460913213008) bank6449 +6450 POINT(41.23644052197044 73.03660646835804) bank6450 +6451 POINT(41.227333073228024 73.7232148816331) bank6451 +6452 POINT(41.174855428682875 73.49701301669955) bank6452 +6453 POINT(41.30271943675745 73.12964820134717) bank6453 +6454 POINT(39.850439910880034 73.80334412958948) bank6454 +6455 POINT(39.90056205353805 73.61081385686303) bank6455 +6456 POINT(41.57859785242607 73.61354715789639) bank6456 +6457 POINT(40.89307445680507 73.45052966368608) bank6457 +6458 POINT(41.10266550629812 73.05431584049914) bank6458 +6459 POINT(41.55040961380505 74.58457740253783) bank6459 +6460 POINT(40.22951297175659 74.59255944021133) bank6460 +6461 POINT(39.733185339077586 74.35542586984772) bank6461 +6462 POINT(41.31490304261932 74.19459125110323) bank6462 +6463 POINT(39.921485569773964 73.16089923377251) bank6463 +6464 POINT(40.13155443779557 73.74369411784394) bank6464 +6465 POINT(40.12272422361696 73.28527923248421) bank6465 +6466 POINT(40.380319210688285 73.14706019782598) bank6466 +6467 POINT(41.65400127388809 74.21086833781098) bank6467 +6468 POINT(41.34500613101348 73.05319894407509) bank6468 +6469 POINT(40.36552997029447 73.07762243773156) bank6469 +6470 POINT(39.746240625647125 73.45200504767472) bank6470 +6471 POINT(40.03726494837419 73.9012545279924) bank6471 +6472 POINT(40.492306622176514 73.09100815461676) bank6472 +6473 POINT(41.48454106194219 73.59494396770755) bank6473 +6474 POINT(41.63140350416844 73.7183115026377) bank6474 +6475 POINT(41.11212700439083 73.81328826614437) bank6475 +6476 POINT(41.59958594854878 73.79047850802603) bank6476 +6477 POINT(41.22730438349659 74.31454279423103) bank6477 +6478 POINT(41.35489665510943 74.81943405164591) bank6478 +6479 POINT(39.92496454793438 73.0400568777493) bank6479 +6480 POINT(41.62789437959242 73.49927797911845) bank6480 +6481 POINT(39.84869506514921 74.29826871723775) bank6481 +6482 POINT(40.209986685315954 74.53010325834948) bank6482 +6483 POINT(40.08918954092016 73.90736608002301) bank6483 +6484 POINT(41.37530121121053 74.30171238149653) bank6484 +6485 POINT(41.117182509122934 74.00360281187372) bank6485 +6486 POINT(39.75815367810033 74.88967349816535) bank6486 +6487 POINT(40.5080640513169 73.57909117825253) bank6487 +6488 POINT(41.555072562837196 74.87781126169862) bank6488 +6489 POINT(41.37575605421142 73.95780979845019) bank6489 +6490 POINT(40.71195810527814 74.12973905797568) bank6490 +6491 POINT(40.238205919823564 73.57134809162581) bank6491 +6492 POINT(41.53789120030969 73.5322174554985) bank6492 +6493 POINT(40.53506558059831 74.86201930148069) bank6493 +6494 POINT(40.38042460969296 74.31479521302975) bank6494 +6495 POINT(40.34824066613527 73.7880991700127) bank6495 +6496 POINT(40.6816610684767 73.11179235379659) bank6496 +6497 POINT(40.41772273933903 73.67894453920854) bank6497 +6498 POINT(40.029289495884726 74.66614626705606) bank6498 +6499 POINT(39.90190740025404 74.7617277812397) bank6499 +6500 POINT(40.75693242784944 74.98710831517381) bank6500 +6501 POINT(40.176757389256835 73.33160088277421) bank6501 +6502 POINT(40.22737598491311 74.94868162964676) bank6502 +6503 POINT(41.66730092358523 74.72075367676788) bank6503 +6504 POINT(40.443783425534555 74.11714047793971) bank6504 +6505 POINT(41.027646763110205 73.49180638357659) bank6505 +6506 POINT(40.99520085499391 74.40392498602509) bank6506 +6507 POINT(39.7900176288467 74.94672450246432) bank6507 +6508 POINT(40.70099336682039 74.31237936678009) bank6508 +6509 POINT(41.257785918447006 73.29231856130605) bank6509 +6510 POINT(40.77566743600037 73.13302150309457) bank6510 +6511 POINT(40.55760192768643 73.0442529814413) bank6511 +6512 POINT(40.49935394586816 73.18638783237574) bank6512 +6513 POINT(40.6437530048609 74.47131411600108) bank6513 +6514 POINT(40.37936019495505 74.7205514304617) bank6514 +6515 POINT(41.38205845562532 74.60337819647795) bank6515 +6516 POINT(41.55330937797905 74.69063690831925) bank6516 +6517 POINT(41.67781557239498 74.18105354092573) bank6517 +6518 POINT(40.80011469334056 74.293006055725) bank6518 +6519 POINT(41.52448793460674 74.96069525423196) bank6519 +6520 POINT(41.68267881945296 74.68016392649206) bank6520 +6521 POINT(40.67629292896402 74.71284154049054) bank6521 +6522 POINT(40.39568820563153 73.9288326307085) bank6522 +6523 POINT(40.13731160101805 74.91356460594987) bank6523 +6524 POINT(41.39203129203928 74.8736996254477) bank6524 +6525 POINT(40.94243888826582 74.8217228234717) bank6525 +6526 POINT(41.246867258684745 73.46718440533448) bank6526 +6527 POINT(40.902814732473004 74.41968320545892) bank6527 +6528 POINT(40.826966769803605 74.1163191070373) bank6528 +6529 POINT(39.96578516184249 74.48081588681937) bank6529 +6530 POINT(41.49291566141855 73.37078908318148) bank6530 +6531 POINT(41.46083977301824 73.29121436428484) bank6531 +6532 POINT(41.013127512312565 73.26440345884475) bank6532 +6533 POINT(40.32311655957393 73.06822354061522) bank6533 +6534 POINT(40.52674823321082 73.71853228719942) bank6534 +6535 POINT(39.9756679909142 74.83507392711184) bank6535 +6536 POINT(39.719159556960996 73.7100435993735) bank6536 +6537 POINT(40.635327577074136 74.41355948728528) bank6537 +6538 POINT(41.1113808095415 73.26799068264813) bank6538 +6539 POINT(40.88483665873395 74.79048679147832) bank6539 +6540 POINT(40.200166501869134 74.1642841297386) bank6540 +6541 POINT(40.81112809004191 74.5132429203535) bank6541 +6542 POINT(40.60266022240116 73.90140490887983) bank6542 +6543 POINT(40.05824374313288 73.4615861109123) bank6543 +6544 POINT(40.05379883599954 74.34552143673852) bank6544 +6545 POINT(41.02713828359078 73.457517340485) bank6545 +6546 POINT(41.041561729607636 73.08603243375032) bank6546 +6547 POINT(41.42708488573517 74.40052155342937) bank6547 +6548 POINT(40.581802044967425 73.14764822362258) bank6548 +6549 POINT(41.58589925535646 73.3418915621103) bank6549 +6550 POINT(40.986269807281886 73.25465919049729) bank6550 +6551 POINT(40.41447018394788 74.45169630151152) bank6551 +6552 POINT(40.48954478947931 73.87247479873295) bank6552 +6553 POINT(39.92906546639347 73.69552942542839) bank6553 +6554 POINT(40.12775203058587 74.37880757536985) bank6554 +6555 POINT(41.082878783002705 74.99573282536099) bank6555 +6556 POINT(40.2787502969463 73.53680404409636) bank6556 +6557 POINT(41.43189319263803 73.81416402215854) bank6557 +6558 POINT(39.86350809864845 74.47007298703433) bank6558 +6559 POINT(41.48348551619649 74.89181205445918) bank6559 +6560 POINT(40.285556180794906 73.78546622583458) bank6560 +6561 POINT(39.93950775703206 73.23161786608617) bank6561 +6562 POINT(40.899251982459276 74.3182812255921) bank6562 +6563 POINT(40.99443709879282 74.4542303512426) bank6563 +6564 POINT(41.15282351337434 73.20351212391061) bank6564 +6565 POINT(40.68328733899968 74.0757334825746) bank6565 +6566 POINT(41.305757722978754 74.77745021963086) bank6566 +6567 POINT(41.278774848119966 73.33945587988713) bank6567 +6568 POINT(41.37724114636228 74.08156544905562) bank6568 +6569 POINT(41.637617301167346 73.70211662735586) bank6569 +6570 POINT(41.45049915098821 74.4438077784092) bank6570 +6571 POINT(40.037864933912466 73.48654241348196) bank6571 +6572 POINT(41.076591951173114 74.67040661645999) bank6572 +6573 POINT(40.39799476830315 74.50511549039646) bank6573 +6574 POINT(40.42590533451207 74.67462002239755) bank6574 +6575 POINT(40.825662088159355 73.42168710015211) bank6575 +6576 POINT(41.26704243696619 74.03416258920844) bank6576 +6577 POINT(41.37262085986822 74.62200898926308) bank6577 +6578 POINT(40.401204846856594 73.07291686706745) bank6578 +6579 POINT(40.72838057020308 73.49672243438688) bank6579 +6580 POINT(41.14736925543774 73.13145385204085) bank6580 +6581 POINT(39.73132361333529 73.06181736656734) bank6581 +6582 POINT(41.095504030832664 73.96352795210946) bank6582 +6583 POINT(40.64757630591758 74.35347523042309) bank6583 +6584 POINT(40.583299573559216 73.07102026135335) bank6584 +6585 POINT(40.979638411103814 74.98671418222189) bank6585 +6586 POINT(40.11924520981843 73.70850326783327) bank6586 +6587 POINT(39.82421647553455 74.68400138827175) bank6587 +6588 POINT(39.91220073181778 74.74770846414175) bank6588 +6589 POINT(41.61007436829224 74.06562092478745) bank6589 +6590 POINT(39.766515093208085 74.37227980461769) bank6590 +6591 POINT(40.230301196534974 73.55102097283928) bank6591 +6592 POINT(39.92820928915348 73.15491223323922) bank6592 +6593 POINT(40.56483178878605 73.2936387598085) bank6593 +6594 POINT(41.371805011141 74.16117251011637) bank6594 +6595 POINT(41.235081237104694 74.04379485140302) bank6595 +6596 POINT(40.56104922634554 73.09044715289491) bank6596 +6597 POINT(40.205663268931104 73.16023557161554) bank6597 +6598 POINT(40.25554798482923 73.49269470120807) bank6598 +6599 POINT(40.322569808991155 74.34615009224159) bank6599 +6600 POINT(41.33973534532783 74.92772304340089) bank6600 +6601 POINT(41.600954894148515 73.39309977624696) bank6601 +6602 POINT(40.49749440622399 73.04322133479491) bank6602 +6603 POINT(40.7902417478023 74.51014845689122) bank6603 +6604 POINT(39.95944927752244 73.53926665025972) bank6604 +6605 POINT(40.72148923114351 75.00085165691414) bank6605 +6606 POINT(41.37739343876307 73.16568875861445) bank6606 +6607 POINT(41.11462611342786 74.36103190195588) bank6607 +6608 POINT(40.8919860268573 74.12699152679288) bank6608 +6609 POINT(40.215377211438586 73.57172245416439) bank6609 +6610 POINT(40.08302796861044 73.99509366411533) bank6610 +6611 POINT(41.15259692832498 73.33552286960288) bank6611 +6612 POINT(41.31577770026476 74.03520180200783) bank6612 +6613 POINT(41.144790206012196 74.12940753476255) bank6613 +6614 POINT(40.36334152086073 74.05926217470609) bank6614 +6615 POINT(41.35381746963222 73.21933733495403) bank6615 +6616 POINT(39.84529837971425 74.83552590177436) bank6616 +6617 POINT(40.015868599446044 73.75523498350391) bank6617 +6618 POINT(41.41682483700604 74.83635567499728) bank6618 +6619 POINT(41.49514500125036 73.50887937361645) bank6619 +6620 POINT(40.97291266141828 73.13025463613583) bank6620 +6621 POINT(41.33830517977132 73.8502380814922) bank6621 +6622 POINT(41.70213216811514 73.80712273506337) bank6622 +6623 POINT(40.388595175007644 73.5945845250647) bank6623 +6624 POINT(40.43348571390696 73.55075419345143) bank6624 +6625 POINT(41.30564135613179 74.26932882941477) bank6625 +6626 POINT(41.422365483820535 73.39715322258832) bank6626 +6627 POINT(40.63000158518537 73.99295804428957) bank6627 +6628 POINT(40.160385820729175 74.96495690209505) bank6628 +6629 POINT(41.13299376888631 73.2518234793086) bank6629 +6630 POINT(41.609170749075865 73.12218271581212) bank6630 +6631 POINT(40.996432288202236 74.97642905117517) bank6631 +6632 POINT(41.577663291880114 73.29806426700497) bank6632 +6633 POINT(41.519144549229544 74.32723743293519) bank6633 +6634 POINT(41.37545319056115 74.93592504838438) bank6634 +6635 POINT(39.92173430850318 74.76866995124756) bank6635 +6636 POINT(40.87828942604075 74.69705125724916) bank6636 +6637 POINT(41.01041471948874 74.56303967745862) bank6637 +6638 POINT(41.614313728814054 74.19090123233724) bank6638 +6639 POINT(40.759425856355975 74.89183907964) bank6639 +6640 POINT(40.61580403459195 74.26552461085717) bank6640 +6641 POINT(40.808331829444874 74.12112703318098) bank6641 +6642 POINT(39.724014954443 73.94518329523315) bank6642 +6643 POINT(41.08880385725357 73.83752247656426) bank6643 +6644 POINT(41.09100355154689 73.96592088268683) bank6644 +6645 POINT(41.48534210465919 74.2617373127447) bank6645 +6646 POINT(41.127653593118644 73.06753143792936) bank6646 +6647 POINT(40.316644910780155 73.97605928110751) bank6647 +6648 POINT(41.1179219845108 73.25980387443033) bank6648 +6649 POINT(39.76486655419984 74.68142448778526) bank6649 +6650 POINT(41.15825644895536 73.95663382684461) bank6650 +6651 POINT(40.289849218291124 74.03950289014949) bank6651 +6652 POINT(41.12353701865578 73.71555829174702) bank6652 +6653 POINT(41.326973314377135 73.75035707051872) bank6653 +6654 POINT(41.09645877065495 73.84934629398401) bank6654 +6655 POINT(41.242844467053565 73.16834849902571) bank6655 +6656 POINT(41.348929178989536 73.90301526678621) bank6656 +6657 POINT(40.99480462044188 73.19931484991996) bank6657 +6658 POINT(39.907624590872096 73.75112918250247) bank6658 +6659 POINT(39.861504727986315 74.81976005032534) bank6659 +6660 POINT(41.24019705386827 73.60466502559598) bank6660 +6661 POINT(39.83984334574446 73.92513360278546) bank6661 +6662 POINT(40.42913435113021 73.62205277409589) bank6662 +6663 POINT(40.630758100804094 73.78155114092027) bank6663 +6664 POINT(40.291980850054074 73.53401029908058) bank6664 +6665 POINT(41.10234071724996 73.35040199291689) bank6665 +6666 POINT(40.54853415368459 73.6904110356408) bank6666 +6667 POINT(40.93286086635381 73.0880201699548) bank6667 +6668 POINT(40.82314428120637 74.73627940231215) bank6668 +6669 POINT(39.89886660350069 74.67006870716723) bank6669 +6670 POINT(39.988060243791885 74.77658689391687) bank6670 +6671 POINT(40.480700337036915 73.98401266485307) bank6671 +6672 POINT(39.75578459945267 74.40287779722796) bank6672 +6673 POINT(39.98581366161847 73.22177235781902) bank6673 +6674 POINT(40.555797413935466 73.54265520127035) bank6674 +6675 POINT(40.86207252490965 73.40740294589479) bank6675 +6676 POINT(40.70239392487601 74.13307840403701) bank6676 +6677 POINT(40.84046925144603 73.17893776774793) bank6677 +6678 POINT(40.77856190287395 74.63665882699138) bank6678 +6679 POINT(40.36052411767246 74.58001723336623) bank6679 +6680 POINT(41.11273741880124 74.23151979349358) bank6680 +6681 POINT(41.34404890174322 74.04976404552924) bank6681 +6682 POINT(41.24357417958602 74.00460590019527) bank6682 +6683 POINT(40.74084017914059 74.10383208015352) bank6683 +6684 POINT(40.75043346647542 74.37633996389184) bank6684 +6685 POINT(41.03240604004737 73.54062946986778) bank6685 +6686 POINT(40.46907003147291 74.62143781593808) bank6686 +6687 POINT(40.53164116220821 73.36147920054303) bank6687 +6688 POINT(40.729548288188624 73.2263118629223) bank6688 +6689 POINT(41.68383548736678 73.92611562963334) bank6689 +6690 POINT(40.00979610265138 74.14414042750273) bank6690 +6691 POINT(41.17529332483483 73.23176562930571) bank6691 +6692 POINT(40.09906193815784 74.92339426544814) bank6692 +6693 POINT(41.25500140116892 74.32382241330428) bank6693 +6694 POINT(40.48690837416933 74.05027245151963) bank6694 +6695 POINT(41.527012547882414 74.97379803083648) bank6695 +6696 POINT(41.27117802208302 73.54647016986891) bank6696 +6697 POINT(41.110917658624786 73.18232547242695) bank6697 +6698 POINT(41.69211966628218 74.85091380231425) bank6698 +6699 POINT(40.2260502245299 73.45213315314984) bank6699 +6700 POINT(39.923934115285824 73.42284329424332) bank6700 +6701 POINT(40.273649546050706 74.4219606930181) bank6701 +6702 POINT(39.81531943554923 74.07701972513549) bank6702 +6703 POINT(40.45814013951226 74.84236655021535) bank6703 +6704 POINT(41.18662128670766 73.19913109318158) bank6704 +6705 POINT(41.25987618827556 73.83431109457881) bank6705 +6706 POINT(40.23162672658738 74.43632453364427) bank6706 +6707 POINT(40.94001357774377 74.30738993499804) bank6707 +6708 POINT(40.310476482863216 74.07037407287201) bank6708 +6709 POINT(40.47673519547486 73.33181811464787) bank6709 +6710 POINT(40.70781136120413 74.18712192979646) bank6710 +6711 POINT(40.73184942049153 74.37631727896402) bank6711 +6712 POINT(40.58822109937003 74.33505846072335) bank6712 +6713 POINT(40.34422736729985 73.18932921896081) bank6713 +6714 POINT(40.5557776053588 73.39499323278768) bank6714 +6715 POINT(41.23684210506202 74.4921016057961) bank6715 +6716 POINT(41.528648108045736 74.50842599699925) bank6716 +6717 POINT(39.760124270866406 75.00383667781215) bank6717 +6718 POINT(40.729016353347774 74.51965058548244) bank6718 +6719 POINT(40.953942892504514 73.57875273436198) bank6719 +6720 POINT(40.37410030212576 74.88395304416409) bank6720 +6721 POINT(41.02774100430119 74.00480829937477) bank6721 +6722 POINT(40.13239149467666 74.69843889448651) bank6722 +6723 POINT(39.84115571080707 74.40249506663494) bank6723 +6724 POINT(41.53111346223785 73.96487910198847) bank6724 +6725 POINT(39.831098465100126 74.17558441830319) bank6725 +6726 POINT(40.20612673096512 73.8706891224849) bank6726 +6727 POINT(39.74327291238029 73.56786578788159) bank6727 +6728 POINT(40.96109164072213 74.27646968429384) bank6728 +6729 POINT(39.909175522070626 73.67363380725665) bank6729 +6730 POINT(41.38038050157072 73.31815550493305) bank6730 +6731 POINT(40.523876734254394 74.57265842358453) bank6731 +6732 POINT(40.814061968569916 73.2039527621256) bank6732 +6733 POINT(41.295831464333695 73.0945244866044) bank6733 +6734 POINT(40.38625619251638 73.43936272545271) bank6734 +6735 POINT(41.38046039727919 73.0867035419087) bank6735 +6736 POINT(41.694953375173476 74.99276693765717) bank6736 +6737 POINT(40.91135272581615 74.7718882953188) bank6737 +6738 POINT(39.8208137479478 73.89993532944406) bank6738 +6739 POINT(41.59227400674775 73.60134603992137) bank6739 +6740 POINT(41.122366025677344 74.39609097189) bank6740 +6741 POINT(40.08990709609586 73.98773566530976) bank6741 +6742 POINT(40.51043608298057 74.00365106648104) bank6742 +6743 POINT(41.06665000289578 73.06540080284772) bank6743 +6744 POINT(41.69714320756531 74.67043883193804) bank6744 +6745 POINT(40.85588552390814 74.99537577058011) bank6745 +6746 POINT(39.95573598299453 74.49409637276699) bank6746 +6747 POINT(41.26231434338246 73.92195964740047) bank6747 +6748 POINT(40.70611380070207 74.30334517826677) bank6748 +6749 POINT(41.30874060621038 74.99771651276507) bank6749 +6750 POINT(40.990709673319195 74.10086971994579) bank6750 +6751 POINT(40.832515748646266 73.58270349085784) bank6751 +6752 POINT(40.68595722755652 73.83601563263693) bank6752 +6753 POINT(41.15473009305227 73.08447052816288) bank6753 +6754 POINT(41.511341297124254 74.13640212487836) bank6754 +6755 POINT(40.02759393082151 74.36268224691557) bank6755 +6756 POINT(40.83075324607474 73.831542196142) bank6756 +6757 POINT(40.79964699691543 73.84617711765445) bank6757 +6758 POINT(41.67525232113528 73.45938638998729) bank6758 +6759 POINT(40.01117654864728 74.32886432608136) bank6759 +6760 POINT(41.624541709662765 73.05281002543346) bank6760 +6761 POINT(40.8183110064259 74.91312573250927) bank6761 +6762 POINT(40.40849729391152 73.32019522206171) bank6762 +6763 POINT(40.759367981034735 74.3194039254038) bank6763 +6764 POINT(40.151800841300584 73.84284502897907) bank6764 +6765 POINT(41.34018618248073 73.58229063371512) bank6765 +6766 POINT(40.83333426342021 74.3071419054042) bank6766 +6767 POINT(40.94925164475166 73.04723000643658) bank6767 +6768 POINT(40.91156773413285 73.25508741788157) bank6768 +6769 POINT(41.398986176606876 73.08235647668631) bank6769 +6770 POINT(41.44230920633069 73.19074261085343) bank6770 +6771 POINT(40.40113044280399 74.20280276454616) bank6771 +6772 POINT(40.60632321962662 73.13394167099946) bank6772 +6773 POINT(41.27451329644112 73.6486690906885) bank6773 +6774 POINT(41.47329935903064 73.73224814090014) bank6774 +6775 POINT(40.25016969231503 74.3274515087427) bank6775 +6776 POINT(41.092807837690785 74.50378139877566) bank6776 +6777 POINT(41.576102077565885 73.10406133015378) bank6777 +6778 POINT(39.90534668260763 73.57019667094755) bank6778 +6779 POINT(40.00880866285233 73.64789486986257) bank6779 +6780 POINT(41.126582668531 73.70444565256857) bank6780 +6781 POINT(40.58675659397879 73.53342800825374) bank6781 +6782 POINT(41.261244396688 74.8091743986265) bank6782 +6783 POINT(41.06382365573423 73.8688420771213) bank6783 +6784 POINT(40.70881815025309 73.29494123354463) bank6784 +6785 POINT(39.948227224306756 73.53312866957879) bank6785 +6786 POINT(40.32879097947592 74.97728112278504) bank6786 +6787 POINT(41.43675477323705 73.01786397513092) bank6787 +6788 POINT(40.69410767968996 74.30178640942752) bank6788 +6789 POINT(41.197373680495325 74.08737324895117) bank6789 +6790 POINT(40.36470365166365 73.61932201006344) bank6790 +6791 POINT(40.916873960104226 74.66751590401992) bank6791 +6792 POINT(39.83677652222435 73.052257540457) bank6792 +6793 POINT(41.507424061371196 74.75277618771084) bank6793 +6794 POINT(41.68141171046731 73.64822996178545) bank6794 +6795 POINT(41.514853232236376 73.69965635177527) bank6795 +6796 POINT(41.410387055787474 73.39120187725447) bank6796 +6797 POINT(40.796587830836316 74.31952096334811) bank6797 +6798 POINT(40.44510597551856 74.8099491245451) bank6798 +6799 POINT(40.56908252268354 74.71223500876906) bank6799 +6800 POINT(40.90694900934856 73.77608654041708) bank6800 +6801 POINT(40.02614442337113 74.46889926679401) bank6801 +6802 POINT(40.396670548190535 73.81520988823678) bank6802 +6803 POINT(41.371190318539796 73.35761036488164) bank6803 +6804 POINT(39.72663931777147 74.88778611493795) bank6804 +6805 POINT(41.70258446220517 74.97016812118517) bank6805 +6806 POINT(40.51737039233342 73.69292734651117) bank6806 +6807 POINT(40.91359288310443 74.2207518131059) bank6807 +6808 POINT(41.204798736027406 73.0788303037545) bank6808 +6809 POINT(41.32841651704104 74.57138434041083) bank6809 +6810 POINT(40.42520278777994 74.79788312643402) bank6810 +6811 POINT(40.06921428437183 74.00401434709376) bank6811 +6812 POINT(41.086566618137105 73.85167419828089) bank6812 +6813 POINT(40.262472352907295 74.97003991499263) bank6813 +6814 POINT(40.77450934661071 73.13374455496752) bank6814 +6815 POINT(41.170821386398266 73.02765682239198) bank6815 +6816 POINT(41.40482801326687 73.15748092549345) bank6816 +6817 POINT(40.23265301508833 73.34170018277331) bank6817 +6818 POINT(40.57043604222892 73.05792989109304) bank6818 +6819 POINT(41.48456041265355 73.45327231685827) bank6819 +6820 POINT(41.25718194904881 73.42038847200288) bank6820 +6821 POINT(40.078020567517555 73.39560773199852) bank6821 +6822 POINT(41.664764606149085 74.14134672539426) bank6822 +6823 POINT(41.37129516468816 74.7472362570018) bank6823 +6824 POINT(41.301605832049546 73.7418476409876) bank6824 +6825 POINT(40.41876190772939 73.42106622958943) bank6825 +6826 POINT(39.73755458328533 73.40298978425845) bank6826 +6827 POINT(40.31831367862469 74.27695477543985) bank6827 +6828 POINT(40.999085022100324 73.21666379624067) bank6828 +6829 POINT(40.356390384317336 74.84303416283524) bank6829 +6830 POINT(40.7982265827672 74.98678977728925) bank6830 +6831 POINT(40.57439234539777 73.10928414997072) bank6831 +6832 POINT(39.81944177886072 73.10864195756578) bank6832 +6833 POINT(41.22414999181908 73.36706215350378) bank6833 +6834 POINT(41.268435455907905 73.56659613829493) bank6834 +6835 POINT(40.09030715274175 74.68942879241597) bank6835 +6836 POINT(41.5591637294439 74.61202352986956) bank6836 +6837 POINT(40.91731253179416 74.59938279088635) bank6837 +6838 POINT(40.56383890416622 74.33167049569599) bank6838 +6839 POINT(41.24031271455007 74.10180670188265) bank6839 +6840 POINT(40.24559278309623 73.27537728571698) bank6840 +6841 POINT(40.292209981052885 73.29713117632915) bank6841 +6842 POINT(41.29284007982909 73.16687089906576) bank6842 +6843 POINT(40.4362391404031 73.60709506460508) bank6843 +6844 POINT(40.907999804114596 74.4982336765318) bank6844 +6845 POINT(40.28412947265694 73.7614945723038) bank6845 +6846 POINT(40.92662579613139 74.89308296273775) bank6846 +6847 POINT(40.454987548463016 73.38833378705854) bank6847 +6848 POINT(40.11002997252322 73.1488398686305) bank6848 +6849 POINT(40.2930084164515 73.85005274910674) bank6849 +6850 POINT(39.76853234286325 73.70915679915119) bank6850 +6851 POINT(39.94179159097916 74.79093602137486) bank6851 +6852 POINT(41.16198050715179 74.67958544975797) bank6852 +6853 POINT(41.45869248815492 73.43267080518413) bank6853 +6854 POINT(40.35122338560885 73.80954828582908) bank6854 +6855 POINT(40.803589998578104 73.72831425420178) bank6855 +6856 POINT(40.10977283296754 74.06933853570726) bank6856 +6857 POINT(40.7779966213825 74.52422866882543) bank6857 +6858 POINT(40.21999580595484 73.4293822233957) bank6858 +6859 POINT(40.105396519027956 73.81745011322793) bank6859 +6860 POINT(41.60458514065823 73.31682765246676) bank6860 +6861 POINT(39.97363309284694 73.87008977250517) bank6861 +6862 POINT(41.2365414278721 73.71044980757121) bank6862 +6863 POINT(40.24260129936902 73.19986231673231) bank6863 +6864 POINT(40.38967058956776 73.76324096222928) bank6864 +6865 POINT(41.18629228713428 74.21251556458058) bank6865 +6866 POINT(40.26047261193835 73.07457692597103) bank6866 +6867 POINT(41.46975058873485 73.42661290455945) bank6867 +6868 POINT(40.01639508876456 74.27322929597555) bank6868 +6869 POINT(40.00989217300437 73.00722784612361) bank6869 +6870 POINT(40.35375077952001 74.18632667234532) bank6870 +6871 POINT(40.44910982578841 73.85416192687796) bank6871 +6872 POINT(41.63398190526345 73.89747972709259) bank6872 +6873 POINT(41.2518873024145 74.87353349993181) bank6873 +6874 POINT(40.78094856887812 74.66813485206615) bank6874 +6875 POINT(41.06622535534633 73.72453450061933) bank6875 +6876 POINT(40.17450987263801 73.29438017984815) bank6876 +6877 POINT(40.71889695045528 73.52770996025973) bank6877 +6878 POINT(40.28422959083686 74.60720636906703) bank6878 +6879 POINT(41.66299313525593 73.17679307245909) bank6879 +6880 POINT(39.90217675536279 74.37779057682508) bank6880 +6881 POINT(40.29049103054991 73.70112793011383) bank6881 +6882 POINT(40.85361221481374 73.22335165468068) bank6882 +6883 POINT(39.949968333868526 73.36202723091604) bank6883 +6884 POINT(41.253170277619056 74.72915524697797) bank6884 +6885 POINT(40.03235695940014 73.34719107196368) bank6885 +6886 POINT(40.784023234575606 74.39281644402193) bank6886 +6887 POINT(40.27657911678854 74.75986916061872) bank6887 +6888 POINT(39.97298604849398 73.19613335088086) bank6888 +6889 POINT(41.16734688309394 74.91650407755812) bank6889 +6890 POINT(41.57681902930155 73.13577552301932) bank6890 +6891 POINT(40.558996872276005 74.2226988578368) bank6891 +6892 POINT(41.36821913774548 74.19725396986959) bank6892 +6893 POINT(41.59790822101045 74.55985664260653) bank6893 +6894 POINT(40.88080877911021 74.94117337049133) bank6894 +6895 POINT(40.76347132668682 74.03855003979746) bank6895 +6896 POINT(40.30037479861813 74.5977809304911) bank6896 +6897 POINT(41.195289895043466 74.96413745786315) bank6897 +6898 POINT(40.559889340270125 74.90820416948749) bank6898 +6899 POINT(41.20149112075299 73.0140154148522) bank6899 +6900 POINT(41.080505143510685 73.09951510036592) bank6900 +6901 POINT(41.337625062299274 74.41640120917145) bank6901 +6902 POINT(39.77822685888849 73.3644135661258) bank6902 +6903 POINT(40.01450562332857 74.3017952542949) bank6903 +6904 POINT(39.80953925090585 73.23492466568969) bank6904 +6905 POINT(41.41809476715845 73.65378632509236) bank6905 +6906 POINT(41.359629386540036 74.84301094621638) bank6906 +6907 POINT(41.34001392201338 73.73453330227258) bank6907 +6908 POINT(39.72146167470042 74.26660121993892) bank6908 +6909 POINT(40.35038201515505 74.52613814916563) bank6909 +6910 POINT(39.94941068856346 74.85646996219339) bank6910 +6911 POINT(40.193163684738785 73.27381375382134) bank6911 +6912 POINT(40.9289268003586 73.88586686232527) bank6912 +6913 POINT(40.01503314372759 73.86351435440606) bank6913 +6914 POINT(40.088367843796966 74.52784291694185) bank6914 +6915 POINT(40.31639377387171 73.76796383650532) bank6915 +6916 POINT(39.74186548860298 74.86999212818313) bank6916 +6917 POINT(41.61728529315194 74.47074517917912) bank6917 +6918 POINT(41.20435108520349 74.09887536138241) bank6918 +6919 POINT(40.123933076059615 73.48684625857253) bank6919 +6920 POINT(41.63861796833626 74.35537464786108) bank6920 +6921 POINT(39.81128434122535 74.80085941675495) bank6921 +6922 POINT(41.69327711395473 74.68080615726396) bank6922 +6923 POINT(41.5035388806621 74.71872072345882) bank6923 +6924 POINT(40.59031656767142 74.39658114696599) bank6924 +6925 POINT(41.25994780571374 74.90882858315706) bank6925 +6926 POINT(40.778755353315724 73.34901225870257) bank6926 +6927 POINT(40.40835378828822 73.96970793505686) bank6927 +6928 POINT(40.07291779650393 74.76779657402646) bank6928 +6929 POINT(40.78886921881277 73.78737920807002) bank6929 +6930 POINT(41.28151108500629 73.78498279458469) bank6930 +6931 POINT(39.8554906624814 73.81368100376055) bank6931 +6932 POINT(41.17546894088042 73.78549645984073) bank6932 +6933 POINT(41.69019103174341 73.41773292376227) bank6933 +6934 POINT(40.61485797981013 74.05001921559631) bank6934 +6935 POINT(41.27093422079911 74.54543475850697) bank6935 +6936 POINT(40.690690469667274 73.38982396498159) bank6936 +6937 POINT(41.545333790229115 73.53574896620566) bank6937 +6938 POINT(40.80793522731052 73.56232049859787) bank6938 +6939 POINT(40.288324545383766 73.94883111451857) bank6939 +6940 POINT(40.87039224600382 74.87762677275738) bank6940 +6941 POINT(40.421298074530256 73.13764069686896) bank6941 +6942 POINT(41.05228139647619 74.73984979855263) bank6942 +6943 POINT(40.600688013238205 74.26775683102215) bank6943 +6944 POINT(40.201956348078646 74.18963270416627) bank6944 +6945 POINT(39.818289224227165 74.71740746225174) bank6945 +6946 POINT(40.839250087197826 73.48280688221084) bank6946 +6947 POINT(41.05268119864717 74.24045883494115) bank6947 +6948 POINT(41.67139383022739 73.08299498237736) bank6948 +6949 POINT(40.650435187379664 74.84122176881867) bank6949 +6950 POINT(40.14268209702709 73.79176543432956) bank6950 +6951 POINT(40.783546241584936 73.1593780960012) bank6951 +6952 POINT(41.441280195648666 73.19597130553107) bank6952 +6953 POINT(40.41510955635838 74.58620741795004) bank6953 +6954 POINT(41.280017588984336 73.22262121586274) bank6954 +6955 POINT(40.21223881679551 74.39047248146713) bank6955 +6956 POINT(41.09117868320601 74.90711244119575) bank6956 +6957 POINT(39.7831624219165 73.90980909145459) bank6957 +6958 POINT(40.80280457246192 73.18074978627781) bank6958 +6959 POINT(40.333428243005834 73.47771872051331) bank6959 +6960 POINT(40.647118086931236 73.34298573925165) bank6960 +6961 POINT(40.655397814150625 74.34471841753287) bank6961 +6962 POINT(41.09937110338463 73.89241917359737) bank6962 +6963 POINT(39.78257327711538 73.09405024439384) bank6963 +6964 POINT(40.832179433102525 74.84906057208217) bank6964 +6965 POINT(40.460943001396174 73.07569854500652) bank6965 +6966 POINT(40.99527670452332 73.64886347940336) bank6966 +6967 POINT(40.56562998392826 73.02548531385158) bank6967 +6968 POINT(41.27057120417768 74.54168069745555) bank6968 +6969 POINT(40.15885866984526 74.26950155922577) bank6969 +6970 POINT(41.02279532059337 74.10439502598656) bank6970 +6971 POINT(40.02140838515913 73.39872710735595) bank6971 +6972 POINT(39.96624021724417 73.45111131068468) bank6972 +6973 POINT(40.883176592642734 74.67215853576629) bank6973 +6974 POINT(39.77716494257244 73.25413856260631) bank6974 +6975 POINT(40.92280157298825 74.2383227896025) bank6975 +6976 POINT(41.3812791327222 73.87398356945533) bank6976 +6977 POINT(40.339409594519545 74.51663674316983) bank6977 +6978 POINT(41.35659197258551 74.21859606738425) bank6978 +6979 POINT(41.42991657143142 73.23689124573914) bank6979 +6980 POINT(41.111969638584306 73.9959566056614) bank6980 +6981 POINT(41.086054491431426 74.87001058964213) bank6981 +6982 POINT(40.44195022615688 74.56714984685519) bank6982 +6983 POINT(41.063801910887776 74.61599781066727) bank6983 +6984 POINT(39.758910536368234 74.60855019222942) bank6984 +6985 POINT(40.860420201867974 74.3264880159978) bank6985 +6986 POINT(41.67027931884171 74.9519397401946) bank6986 +6987 POINT(40.475714861197766 73.56813024436491) bank6987 +6988 POINT(40.55315154081164 73.66078435897093) bank6988 +6989 POINT(41.40862116544341 73.93962348253123) bank6989 +6990 POINT(40.18114352358391 73.07699273806328) bank6990 +6991 POINT(39.90765983171729 74.39496013726962) bank6991 +6992 POINT(41.4616977306173 74.45016070204228) bank6992 +6993 POINT(41.51812478197423 73.18216783120143) bank6993 +6994 POINT(41.68742066508205 73.27601605085367) bank6994 +6995 POINT(40.662952393864195 74.23149346174658) bank6995 +6996 POINT(39.81561876837187 73.29287699868337) bank6996 +6997 POINT(41.2559556697353 74.5727067529938) bank6997 +6998 POINT(39.78774731983395 74.72730234513907) bank6998 +6999 POINT(41.408991633054214 73.53453623602604) bank6999 +7000 POINT(40.26481920266636 73.97442299945914) bank7000 +7001 POINT(41.68139211641801 74.48087733746313) bank7001 +7002 POINT(40.47771198829894 73.9909570698868) bank7002 +7003 POINT(40.468665066206306 73.39435170822317) bank7003 +7004 POINT(41.440967077111125 73.49997445979984) bank7004 +7005 POINT(40.690694070776985 73.61265421694333) bank7005 +7006 POINT(41.53267712841893 74.6178145446626) bank7006 +7007 POINT(40.6945979904547 74.5286188650567) bank7007 +7008 POINT(40.02928791489997 73.50590746402375) bank7008 +7009 POINT(41.0957780032493 73.95373549431969) bank7009 +7010 POINT(41.32052703025479 73.8549331217462) bank7010 +7011 POINT(41.657984937719036 73.33054568944624) bank7011 +7012 POINT(41.55732723497524 74.32453818005416) bank7012 +7013 POINT(40.790933028980696 73.64471208890544) bank7013 +7014 POINT(39.991436685902386 74.16155794314177) bank7014 +7015 POINT(39.868817466114095 73.93346596687223) bank7015 +7016 POINT(40.76844785940441 73.06705553052682) bank7016 +7017 POINT(39.98627263700012 73.0757188392863) bank7017 +7018 POINT(41.41393055983822 73.11152176009564) bank7018 +7019 POINT(39.928635767572814 74.61429146066521) bank7019 +7020 POINT(41.26235951424359 73.07243049023471) bank7020 +7021 POINT(40.71264183052966 74.06273521971323) bank7021 +7022 POINT(40.83942466332889 74.6096452412699) bank7022 +7023 POINT(41.366596611000546 73.1567700394372) bank7023 +7024 POINT(41.43273374733539 73.42151627803172) bank7024 +7025 POINT(40.98958682745265 73.1555454484925) bank7025 +7026 POINT(40.10683590909414 74.83469172599789) bank7026 +7027 POINT(41.12822113712346 74.33493731419495) bank7027 +7028 POINT(41.10471258182759 73.40140714452015) bank7028 +7029 POINT(40.510171300102826 73.55053131451444) bank7029 +7030 POINT(40.26398414238243 73.1024354998596) bank7030 +7031 POINT(41.63407888782046 74.00759926106389) bank7031 +7032 POINT(40.362688012260456 74.37331939725985) bank7032 +7033 POINT(40.218663463917736 74.11657915020155) bank7033 +7034 POINT(39.97260699264622 74.68889643934813) bank7034 +7035 POINT(40.17560032527139 73.44800106178222) bank7035 +7036 POINT(41.32231803295773 73.09727976982073) bank7036 +7037 POINT(41.5236067129832 73.6623032679656) bank7037 +7038 POINT(41.02345381131738 73.61047402365222) bank7038 +7039 POINT(39.92745436447221 73.9694567358481) bank7039 +7040 POINT(39.894789924559426 73.05064324479697) bank7040 +7041 POINT(41.590033295702014 74.83757121784106) bank7041 +7042 POINT(41.225311760650136 74.96054294635886) bank7042 +7043 POINT(39.78678837599131 73.82935562409506) bank7043 +7044 POINT(40.727147183724604 74.43493796591594) bank7044 +7045 POINT(40.23464079360057 73.74814554213218) bank7045 +7046 POINT(39.93267308712634 73.907564062552) bank7046 +7047 POINT(41.0831467477111 74.58301895942083) bank7047 +7048 POINT(40.50649484575317 73.7140082369525) bank7048 +7049 POINT(41.56467143325615 73.9584080469386) bank7049 +7050 POINT(40.484963274258845 73.69464294214124) bank7050 +7051 POINT(40.348826911104084 73.74094429744721) bank7051 +7052 POINT(41.15695298027922 74.91755688860334) bank7052 +7053 POINT(40.0436288942439 74.69569031698161) bank7053 +7054 POINT(41.26024661547462 74.61543856386736) bank7054 +7055 POINT(40.20534000792171 73.51088250051171) bank7055 +7056 POINT(41.57709825521363 74.52778278942684) bank7056 +7057 POINT(41.705995957113544 75.00162798829946) bank7057 +7058 POINT(40.62471444048998 73.3414244555724) bank7058 +7059 POINT(40.37028694795521 74.20471415595071) bank7059 +7060 POINT(40.032793841480725 73.36668452626849) bank7060 +7061 POINT(40.742136261264825 74.10477867711138) bank7061 +7062 POINT(41.44285653085381 73.7345157876219) bank7062 +7063 POINT(41.46950677933225 74.8131454993674) bank7063 +7064 POINT(40.47852725117871 73.87678899042872) bank7064 +7065 POINT(41.30650543084143 73.90907028609381) bank7065 +7066 POINT(41.07382372995301 73.68383527464145) bank7066 +7067 POINT(39.775794496530516 73.86143661175109) bank7067 +7068 POINT(40.81483684031601 74.11096895607994) bank7068 +7069 POINT(40.23866042681884 74.46515582049466) bank7069 +7070 POINT(40.885096367557736 74.98215114151539) bank7070 +7071 POINT(39.75973502087976 73.80584298424994) bank7071 +7072 POINT(40.41612625507109 73.11499033609172) bank7072 +7073 POINT(40.73533173786344 74.07244383907073) bank7073 +7074 POINT(41.69769566108489 73.84036506806034) bank7074 +7075 POINT(39.95349913019771 73.25914425568592) bank7075 +7076 POINT(40.912850029655395 73.2279869680098) bank7076 +7077 POINT(39.80460666859622 74.24529511464448) bank7077 +7078 POINT(41.58777955884181 74.40015021530064) bank7078 +7079 POINT(40.594518386873425 73.22070899026386) bank7079 +7080 POINT(40.016939709499354 73.05555239481684) bank7080 +7081 POINT(40.59850069720528 73.122740618219) bank7081 +7082 POINT(41.57056664383663 73.15840447891141) bank7082 +7083 POINT(40.24775647452471 74.40036425808646) bank7083 +7084 POINT(39.892910558508476 73.20964283179507) bank7084 +7085 POINT(41.236313381642866 73.02395181907147) bank7085 +7086 POINT(41.47969065332056 73.49771415375663) bank7086 +7087 POINT(40.309868898716815 73.48890624853503) bank7087 +7088 POINT(40.094125628568236 74.83594030375237) bank7088 +7089 POINT(40.19805147729062 73.63158286465729) bank7089 +7090 POINT(40.912622236324566 73.90737081427358) bank7090 +7091 POINT(39.81437043169928 73.35672380944257) bank7091 +7092 POINT(40.730400991297074 73.22669366228246) bank7092 +7093 POINT(39.78315641341951 73.07678921695975) bank7093 +7094 POINT(40.14613374487387 74.88140027290366) bank7094 +7095 POINT(40.872515532186796 74.81342048208278) bank7095 +7096 POINT(41.56712534723014 74.96631777165315) bank7096 +7097 POINT(40.845905931802704 73.57922854724015) bank7097 +7098 POINT(40.82775815486453 73.22639679171739) bank7098 +7099 POINT(39.86509154291693 73.39807062418414) bank7099 +7100 POINT(40.467427441914246 73.42557870340036) bank7100 +7101 POINT(41.47617032965107 74.10049405999871) bank7101 +7102 POINT(41.348956526562 74.8727906510706) bank7102 +7103 POINT(40.37793164933373 74.50358354538888) bank7103 +7104 POINT(39.83570737617297 73.29100868351105) bank7104 +7105 POINT(40.709764947860435 74.12739129053102) bank7105 +7106 POINT(39.94609222665277 74.97533917903998) bank7106 +7107 POINT(41.27444212851001 74.44313286425208) bank7107 +7108 POINT(41.42678334187008 73.24317247431503) bank7108 +7109 POINT(39.78423757146078 73.25785224832737) bank7109 +7110 POINT(40.53657886345378 73.6823996143102) bank7110 +7111 POINT(41.16215578987998 73.23134935206139) bank7111 +7112 POINT(40.00058404129809 73.94557671071475) bank7112 +7113 POINT(41.527351809550744 73.81172818009203) bank7113 +7114 POINT(41.21560638700581 73.93500620494396) bank7114 +7115 POINT(41.34205600618489 74.42726570411487) bank7115 +7116 POINT(40.234811970490405 74.89912015629746) bank7116 +7117 POINT(40.50525403127486 74.0498643070616) bank7117 +7118 POINT(41.12522062647575 74.62264453079892) bank7118 +7119 POINT(39.82837586860169 73.3173215569319) bank7119 +7120 POINT(40.72242216585619 73.30442149640021) bank7120 +7121 POINT(40.39854236521373 73.10203793990281) bank7121 +7122 POINT(41.56384634632511 73.34697690916892) bank7122 +7123 POINT(40.146592793900986 74.36295868778187) bank7123 +7124 POINT(39.84313561444202 73.33398089954534) bank7124 +7125 POINT(40.10063268877315 74.17617340465115) bank7125 +7126 POINT(41.67771794578052 74.50372910398876) bank7126 +7127 POINT(39.760960858417185 73.26495910156748) bank7127 +7128 POINT(39.98728379948917 74.7134870028597) bank7128 +7129 POINT(39.927449435153285 74.72559505517795) bank7129 +7130 POINT(40.043169123058874 74.36188122234422) bank7130 +7131 POINT(41.05068804619703 74.0382700617972) bank7131 +7132 POINT(41.22683769744478 74.64088060675186) bank7132 +7133 POINT(40.15463039625245 74.89406486931486) bank7133 +7134 POINT(40.55372260206569 74.96752808296276) bank7134 +7135 POINT(40.00909731928391 74.72215741535393) bank7135 +7136 POINT(41.590528059222116 73.13303766981608) bank7136 +7137 POINT(41.492735163843676 74.63880024606593) bank7137 +7138 POINT(40.26522505512867 73.95201229347342) bank7138 +7139 POINT(41.66107301099067 74.69775788346205) bank7139 +7140 POINT(40.222868419704625 73.67300598800335) bank7140 +7141 POINT(40.936867582543265 73.74713078061231) bank7141 +7142 POINT(40.09019930747262 73.2373967361584) bank7142 +7143 POINT(40.84349215568591 74.4938425397392) bank7143 +7144 POINT(40.79337629561205 73.43863740755353) bank7144 +7145 POINT(40.43998343339075 73.55492042801366) bank7145 +7146 POINT(41.701322179560776 74.04862718040597) bank7146 +7147 POINT(41.48994653488358 73.83121601845914) bank7147 +7148 POINT(40.207759976122716 73.2155366999815) bank7148 +7149 POINT(39.79354353725838 74.13218367325989) bank7149 +7150 POINT(39.924360328078635 74.88894671825938) bank7150 +7151 POINT(41.15192836263557 73.07800136805402) bank7151 +7152 POINT(40.31457778634165 73.67716586025269) bank7152 +7153 POINT(40.29604261575278 73.57708455165938) bank7153 +7154 POINT(41.09041657725129 73.52096011986245) bank7154 +7155 POINT(40.88183031928857 73.83541006252152) bank7155 +7156 POINT(40.06391854534581 73.94005888186352) bank7156 +7157 POINT(40.46973206531294 73.30851014422164) bank7157 +7158 POINT(39.923864913864584 74.4310416115242) bank7158 +7159 POINT(40.53902442806722 74.14081540783417) bank7159 +7160 POINT(40.47169187084454 74.64271285011986) bank7160 +7161 POINT(41.337181813354185 74.17762257659587) bank7161 +7162 POINT(39.85882210634004 73.91317438599197) bank7162 +7163 POINT(40.191251017053006 73.48118937300111) bank7163 +7164 POINT(41.399835554784524 73.15155706595827) bank7164 +7165 POINT(40.15950744351044 74.33266696236196) bank7165 +7166 POINT(41.44258379961712 73.64403404789772) bank7166 +7167 POINT(39.88168437545505 74.16077698653855) bank7167 +7168 POINT(40.87571346071746 74.46721067931705) bank7168 +7169 POINT(40.93862609662961 74.72004786696361) bank7169 +7170 POINT(40.067685944184696 73.70709794949704) bank7170 +7171 POINT(40.90172259196025 74.44526927819474) bank7171 +7172 POINT(39.893836197240034 74.4381903787386) bank7172 +7173 POINT(41.53968635755113 74.19409632327586) bank7173 +7174 POINT(41.497537278346535 74.72353463629264) bank7174 +7175 POINT(39.91437620578368 74.84756297197548) bank7175 +7176 POINT(40.69970238782582 73.97865384923074) bank7176 +7177 POINT(40.995983093610214 74.72332217899351) bank7177 +7178 POINT(41.40376042213704 74.69763976994616) bank7178 +7179 POINT(39.86260915694446 73.99934440898906) bank7179 +7180 POINT(41.18839685916922 74.51251846725962) bank7180 +7181 POINT(40.833498162401256 74.60696533440361) bank7181 +7182 POINT(40.22020209572959 74.59157164873838) bank7182 +7183 POINT(41.090041223363365 74.21431422992676) bank7183 +7184 POINT(40.28938292963407 74.93018396253389) bank7184 +7185 POINT(41.225997402351226 73.65010680458037) bank7185 +7186 POINT(40.653176452892815 74.32870429558014) bank7186 +7187 POINT(41.010621618043864 74.91783673429016) bank7187 +7188 POINT(39.72362963105297 74.29302969937788) bank7188 +7189 POINT(41.203787894235646 74.07139300793608) bank7189 +7190 POINT(41.63729918916151 74.67723475314158) bank7190 +7191 POINT(41.42399014461175 73.51098097837142) bank7191 +7192 POINT(40.3131901880059 73.88003119235341) bank7192 +7193 POINT(40.216978679990795 74.38694000436539) bank7193 +7194 POINT(41.6001399405823 74.34125028869136) bank7194 +7195 POINT(40.332085839481884 74.80547575184039) bank7195 +7196 POINT(39.96314030965535 74.39665138153029) bank7196 +7197 POINT(41.42923375881765 73.25371826817846) bank7197 +7198 POINT(40.474197429599045 73.01101075575909) bank7198 +7199 POINT(39.89146736959856 73.5051501742287) bank7199 +7200 POINT(40.06895218970395 73.44616728737101) bank7200 +7201 POINT(40.24126062609761 74.1787062608497) bank7201 +7202 POINT(41.18112741233319 73.31331815140936) bank7202 +7203 POINT(40.25347957570567 73.41039186560506) bank7203 +7204 POINT(40.817200219133376 73.79060722972497) bank7204 +7205 POINT(40.89043527892571 73.49806741055434) bank7205 +7206 POINT(41.5399636454313 74.28124008541627) bank7206 +7207 POINT(39.88131674128884 74.35702827251058) bank7207 +7208 POINT(39.90087847293071 73.51612269629314) bank7208 +7209 POINT(41.15567127256627 73.09313434243462) bank7209 +7210 POINT(41.04932482861191 74.65608148488789) bank7210 +7211 POINT(41.445156304812244 73.39438658702146) bank7211 +7212 POINT(41.08258243258175 74.12439151277178) bank7212 +7213 POINT(40.03706814006018 74.7819005111117) bank7213 +7214 POINT(41.70760298466487 74.81699199722685) bank7214 +7215 POINT(39.75755052926108 74.03671999226303) bank7215 +7216 POINT(40.92442210862028 73.15585131125879) bank7216 +7217 POINT(41.35832763251175 74.12114474393498) bank7217 +7218 POINT(40.74443370741907 73.63438737649282) bank7218 +7219 POINT(41.34750598306909 73.2660683758744) bank7219 +7220 POINT(41.33195050533072 74.95674407689606) bank7220 +7221 POINT(39.9916359440525 74.1913516546621) bank7221 +7222 POINT(40.47739450308204 74.26101431396205) bank7222 +7223 POINT(39.85248018866809 73.03555220737996) bank7223 +7224 POINT(40.49329760024336 74.02558474480144) bank7224 +7225 POINT(41.10988923044832 73.12021741421745) bank7225 +7226 POINT(40.828091513982194 74.87638479647826) bank7226 +7227 POINT(40.533267024226454 74.35303461645289) bank7227 +7228 POINT(40.35619852684074 73.37435008042394) bank7228 +7229 POINT(39.981845887355995 74.80582542101632) bank7229 +7230 POINT(41.2861732932772 74.85892074346793) bank7230 +7231 POINT(40.73084504336954 73.76072632883249) bank7231 +7232 POINT(41.548437019974514 74.30842359496309) bank7232 +7233 POINT(40.11713828709889 73.80122386502472) bank7233 +7234 POINT(41.01922249056987 74.46339423831304) bank7234 +7235 POINT(40.27295633898209 74.71064325303567) bank7235 +7236 POINT(41.28890892352391 73.39491541317928) bank7236 +7237 POINT(40.146930517784256 73.758931302085) bank7237 +7238 POINT(41.28250306956669 74.89764231802152) bank7238 +7239 POINT(41.66033496724406 74.55196619014794) bank7239 +7240 POINT(41.29603643588372 73.66738776829774) bank7240 +7241 POINT(41.299781555745874 74.19639354340073) bank7241 +7242 POINT(40.014914254191545 74.87976999253402) bank7242 +7243 POINT(40.948515548989114 73.54844145955731) bank7243 +7244 POINT(40.38209241255088 74.20169959244978) bank7244 +7245 POINT(41.21461909427405 73.53386095936234) bank7245 +7246 POINT(40.78308213970377 73.49175779205602) bank7246 +7247 POINT(40.732554921906015 73.89788286861813) bank7247 +7248 POINT(39.92024928276363 74.94546622075056) bank7248 +7249 POINT(41.12187934645604 74.27318068928903) bank7249 +7250 POINT(41.19619447833944 73.90985308333047) bank7250 +7251 POINT(40.660807542579576 74.21542300587669) bank7251 +7252 POINT(40.736987649654715 73.95532975635965) bank7252 +7253 POINT(39.861895997108206 74.85692523127008) bank7253 +7254 POINT(41.44907260535969 73.92714421663892) bank7254 +7255 POINT(40.30669714910675 74.82661783461585) bank7255 +7256 POINT(41.52241150360679 73.99940966582702) bank7256 +7257 POINT(40.99737898650336 74.80228637485601) bank7257 +7258 POINT(40.558011739829624 74.67777432143016) bank7258 +7259 POINT(40.436269353602114 73.02480185164413) bank7259 +7260 POINT(40.47794647370647 74.20210821529201) bank7260 +7261 POINT(40.1163509026057 74.17557076601862) bank7261 +7262 POINT(40.33036077109914 74.83270438657587) bank7262 +7263 POINT(40.07296301065855 74.06073854203431) bank7263 +7264 POINT(40.96038313502436 73.87509888299279) bank7264 +7265 POINT(39.895747429319826 73.03314194609386) bank7265 +7266 POINT(39.88790439239205 73.5926853461267) bank7266 +7267 POINT(40.09035850655141 74.71294123906381) bank7267 +7268 POINT(40.96211798192158 73.14105046496226) bank7268 +7269 POINT(39.89383934095064 74.98706491921773) bank7269 +7270 POINT(40.086493605222515 74.00698316040743) bank7270 +7271 POINT(40.21869544941866 73.99022613647071) bank7271 +7272 POINT(39.7575884276358 74.49507703309114) bank7272 +7273 POINT(40.72926152782782 74.6510473347161) bank7273 +7274 POINT(39.8239135343501 73.06645013660183) bank7274 +7275 POINT(41.47057140417539 74.49377040584662) bank7275 +7276 POINT(40.35765446875843 74.63818218936109) bank7276 +7277 POINT(41.62926641466678 74.36708489957522) bank7277 +7278 POINT(40.922683289438346 73.69428638726984) bank7278 +7279 POINT(40.90897449592312 73.47026293227904) bank7279 +7280 POINT(40.64953201613631 74.15469529787437) bank7280 +7281 POINT(40.6523850123428 73.09447374519495) bank7281 +7282 POINT(39.8039132330577 74.37290062817782) bank7282 +7283 POINT(41.16758278062611 74.15272834322943) bank7283 +7284 POINT(41.35514922262593 73.29792685132709) bank7284 +7285 POINT(40.7876999673339 74.68088449941486) bank7285 +7286 POINT(41.50263563322077 74.86336759917558) bank7286 +7287 POINT(41.504874806671864 73.8596777905803) bank7287 +7288 POINT(41.6965770968554 73.96018826021827) bank7288 +7289 POINT(40.0738616640742 73.46940637946177) bank7289 +7290 POINT(40.964039290489985 74.2443674710454) bank7290 +7291 POINT(41.2830445137449 74.68331713632529) bank7291 +7292 POINT(39.99525990304026 73.62327282290454) bank7292 +7293 POINT(40.418796508148766 73.67082712962981) bank7293 +7294 POINT(39.90833754886587 74.40133720997775) bank7294 +7295 POINT(41.57056342682515 74.11479843887821) bank7295 +7296 POINT(40.27151468690546 74.75918853892817) bank7296 +7297 POINT(40.971639238791326 73.19000806004381) bank7297 +7298 POINT(40.77508962975373 73.12086887853707) bank7298 +7299 POINT(40.294316633078154 74.13652522565953) bank7299 +7300 POINT(41.067525430976275 74.38456052961571) bank7300 +7301 POINT(39.9951951306099 73.18553488249296) bank7301 +7302 POINT(41.705969807404706 73.921299358879) bank7302 +7303 POINT(41.1651407815717 73.56054242256522) bank7303 +7304 POINT(41.50852513172224 73.13916356822061) bank7304 +7305 POINT(40.59176851412068 74.36268853574988) bank7305 +7306 POINT(41.51885620904323 74.79004702194958) bank7306 +7307 POINT(40.646850329319605 73.47586191294378) bank7307 +7308 POINT(40.5368680997762 73.47482162536394) bank7308 +7309 POINT(41.26908272943619 74.52560443394222) bank7309 +7310 POINT(40.482752666940804 74.18496157539788) bank7310 +7311 POINT(41.18911429240568 73.01172213212111) bank7311 +7312 POINT(39.71352443659989 73.42799348261717) bank7312 +7313 POINT(39.82603902704373 74.3695679314727) bank7313 +7314 POINT(41.07847954241953 73.9693562627182) bank7314 +7315 POINT(40.479811433039686 73.84876507446268) bank7315 +7316 POINT(41.51856715240135 73.76133465470768) bank7316 +7317 POINT(40.7915607434714 73.24362302107735) bank7317 +7318 POINT(39.8968810377867 74.28271493026594) bank7318 +7319 POINT(40.64663396554783 73.86430625966184) bank7319 +7320 POINT(41.65208401962625 73.95294504441242) bank7320 +7321 POINT(39.937068916586924 73.34147424469697) bank7321 +7322 POINT(41.211334771372854 73.59802013770292) bank7322 +7323 POINT(41.285932561519495 74.77062385858285) bank7323 +7324 POINT(41.39383579768446 74.70201688045083) bank7324 +7325 POINT(40.9001268645463 73.56437825340433) bank7325 +7326 POINT(41.11175553674683 74.14535809789538) bank7326 +7327 POINT(40.598046406390594 73.20968364376758) bank7327 +7328 POINT(41.15695782481519 74.8562293049815) bank7328 +7329 POINT(41.310259238584436 74.79219710421452) bank7329 +7330 POINT(40.278876642743334 74.7187937836046) bank7330 +7331 POINT(40.22301530928203 73.0609501739608) bank7331 +7332 POINT(40.234545833517195 74.33312221883183) bank7332 +7333 POINT(40.80849740134587 73.00791940221447) bank7333 +7334 POINT(41.657965189404415 74.42573586210813) bank7334 +7335 POINT(40.70145894060635 73.59004719243235) bank7335 +7336 POINT(40.650782740332424 73.18327649425007) bank7336 +7337 POINT(41.546706158501124 74.4030448061369) bank7337 +7338 POINT(40.48323385789685 73.84868350861338) bank7338 +7339 POINT(41.447204114052795 74.21183400274424) bank7339 +7340 POINT(40.79758520687314 74.49121259701371) bank7340 +7341 POINT(40.690370744178225 73.85958239546663) bank7341 +7342 POINT(41.495594028368195 74.81394936309559) bank7342 +7343 POINT(41.16551885954275 73.02741710917965) bank7343 +7344 POINT(40.67665873881519 73.34635310710479) bank7344 +7345 POINT(40.542070653673235 73.97091385317171) bank7345 +7346 POINT(40.62977471964231 74.26269672864457) bank7346 +7347 POINT(40.254636904089494 74.2123568845051) bank7347 +7348 POINT(40.995804012292524 74.43111662896285) bank7348 +7349 POINT(40.65891545371585 73.50331245021005) bank7349 +7350 POINT(40.21420317939964 74.15042721366565) bank7350 +7351 POINT(40.469412877690154 73.06647782825756) bank7351 +7352 POINT(40.307094340203285 74.94255409288509) bank7352 +7353 POINT(39.99971753054095 74.02639545417739) bank7353 +7354 POINT(41.29493708099171 74.15405303542401) bank7354 +7355 POINT(40.732190681082805 74.3859536791728) bank7355 +7356 POINT(39.94257593694863 73.60507474440851) bank7356 +7357 POINT(39.97289641693983 74.02765235472447) bank7357 +7358 POINT(40.41349179550708 74.6557931698174) bank7358 +7359 POINT(41.015182122382384 74.67090836449302) bank7359 +7360 POINT(41.64300513606023 74.64555802602698) bank7360 +7361 POINT(41.27135310674539 73.54836500298958) bank7361 +7362 POINT(41.55682916155609 73.41020732319701) bank7362 +7363 POINT(40.432572197263234 74.80354121674986) bank7363 +7364 POINT(39.83767388575187 73.0645290794471) bank7364 +7365 POINT(40.03149696718749 74.14730866629375) bank7365 +7366 POINT(41.65623944245761 73.01134721090366) bank7366 +7367 POINT(41.55604637909147 74.98981764350472) bank7367 +7368 POINT(40.09212993232632 74.18602908799085) bank7368 +7369 POINT(40.684025283088175 74.4688974954592) bank7369 +7370 POINT(40.181436472040104 74.0996190532329) bank7370 +7371 POINT(39.7150837863663 74.74123399562065) bank7371 +7372 POINT(39.946913534366274 73.0704286509601) bank7372 +7373 POINT(40.07077628194727 74.41010525727572) bank7373 +7374 POINT(39.94588195438147 74.8314556362905) bank7374 +7375 POINT(40.951860405586366 73.30609983502815) bank7375 +7376 POINT(40.62542284859668 74.50402710505475) bank7376 +7377 POINT(40.01028245008427 73.31823135005719) bank7377 +7378 POINT(39.969747692597025 73.82682531373993) bank7378 +7379 POINT(40.03558510068072 74.74895403709465) bank7379 +7380 POINT(41.57584704870532 73.65993135437473) bank7380 +7381 POINT(40.876571089547355 73.99551403834634) bank7381 +7382 POINT(39.96918381925545 74.63705251954555) bank7382 +7383 POINT(41.495784902922544 74.54335868269966) bank7383 +7384 POINT(41.01668451707498 73.83072941119774) bank7384 +7385 POINT(40.49998677489721 74.31557920585614) bank7385 +7386 POINT(39.92766538223246 73.35185086182129) bank7386 +7387 POINT(40.82763433602596 73.16752897272417) bank7387 +7388 POINT(39.929047623928454 73.43625262537445) bank7388 +7389 POINT(41.15622887128205 73.77345637463951) bank7389 +7390 POINT(41.56928099716219 73.08707557410685) bank7390 +7391 POINT(41.0218509091987 74.00432315708427) bank7391 +7392 POINT(40.48136704726542 74.70132164609456) bank7392 +7393 POINT(41.600430045528924 73.04584355389412) bank7393 +7394 POINT(39.873303692840544 73.58067866022586) bank7394 +7395 POINT(40.71564687027473 74.9254573870106) bank7395 +7396 POINT(41.322861393418314 74.17419061994718) bank7396 +7397 POINT(40.45551398190257 73.65847584989938) bank7397 +7398 POINT(40.151035379103625 74.03952086783856) bank7398 +7399 POINT(39.82712752246668 74.00824373718147) bank7399 +7400 POINT(39.7146941857272 73.74756138131005) bank7400 +7401 POINT(41.34167429983969 74.6995657242753) bank7401 +7402 POINT(40.401271314324084 74.63336238248955) bank7402 +7403 POINT(40.72430333515272 73.4420293821203) bank7403 +7404 POINT(40.60392933132233 73.88716114238446) bank7404 +7405 POINT(41.06047655796207 73.6116117286758) bank7405 +7406 POINT(40.8257503788461 74.786597657133) bank7406 +7407 POINT(40.77591330643124 73.83007918049961) bank7407 +7408 POINT(40.94085819839668 73.91110326793941) bank7408 +7409 POINT(39.914997271998956 73.57630646595153) bank7409 +7410 POINT(39.95858627858693 73.02261260700769) bank7410 +7411 POINT(41.669045257661054 74.0383084608647) bank7411 +7412 POINT(40.214562356164166 73.24798761557435) bank7412 +7413 POINT(40.01618268668801 74.73187038466482) bank7413 +7414 POINT(41.371235020860716 74.8192984304942) bank7414 +7415 POINT(40.53751596417465 74.61540540433221) bank7415 +7416 POINT(39.738687638195934 74.0324646115131) bank7416 +7417 POINT(41.20805285083846 73.60177944819638) bank7417 +7418 POINT(41.23202296548224 74.9036384387807) bank7418 +7419 POINT(41.394557156455136 73.97247800958138) bank7419 +7420 POINT(41.396059500291194 73.27846892139557) bank7420 +7421 POINT(40.924543685745235 73.84730554033426) bank7421 +7422 POINT(40.53550295218573 74.472739115455) bank7422 +7423 POINT(40.715064719519525 73.73855511650773) bank7423 +7424 POINT(39.8194769348562 73.15518288317712) bank7424 +7425 POINT(41.47890465216139 73.64929063420101) bank7425 +7426 POINT(40.143428421531475 74.61135273444835) bank7426 +7427 POINT(40.15966947651351 74.6335489917069) bank7427 +7428 POINT(40.987377959804654 74.52338869827375) bank7428 +7429 POINT(40.78460357744531 74.62947315605219) bank7429 +7430 POINT(41.68514050824533 73.47434203842589) bank7430 +7431 POINT(41.196923890826895 73.77302578453607) bank7431 +7432 POINT(39.783567143139905 74.51395383015375) bank7432 +7433 POINT(40.086382752696224 74.8953863102106) bank7433 +7434 POINT(41.391369537334384 73.87618338223092) bank7434 +7435 POINT(40.21451309458182 74.03796813721125) bank7435 +7436 POINT(40.05257063062868 73.74064597156578) bank7436 +7437 POINT(40.38248069319321 74.67066471686502) bank7437 +7438 POINT(40.137507173685016 74.19676025836175) bank7438 +7439 POINT(40.25709182629167 73.97177021394741) bank7439 +7440 POINT(40.921221956393985 74.6949830229774) bank7440 +7441 POINT(40.65081512032248 73.1079346911437) bank7441 +7442 POINT(40.81145450012675 73.48467632374005) bank7442 +7443 POINT(40.73275446167595 74.20475982645645) bank7443 +7444 POINT(41.68142022958521 73.92854838782178) bank7444 +7445 POINT(41.37081072054473 74.2493540500102) bank7445 +7446 POINT(40.904524249172624 74.40988611992753) bank7446 +7447 POINT(41.110920772917375 73.77320479045684) bank7447 +7448 POINT(40.48488571146964 74.24193814249539) bank7448 +7449 POINT(39.95434340252235 74.95253492254992) bank7449 +7450 POINT(41.44871424887476 73.17000987996369) bank7450 +7451 POINT(40.183009515402865 73.37753292505673) bank7451 +7452 POINT(41.25776502036858 74.68471077650361) bank7452 +7453 POINT(41.6477389569399 73.36451277548107) bank7453 +7454 POINT(39.89963184098778 73.89171728469746) bank7454 +7455 POINT(40.19517729585616 73.60315069139749) bank7455 +7456 POINT(40.28917589332797 73.34449537463476) bank7456 +7457 POINT(40.183985493044446 73.40423346694735) bank7457 +7458 POINT(40.363537244359684 74.03576463993528) bank7458 +7459 POINT(41.273556038736125 74.07907872271115) bank7459 +7460 POINT(39.964289411079804 74.64838572116898) bank7460 +7461 POINT(40.16891353321496 74.01161275009544) bank7461 +7462 POINT(40.307789502134156 73.15438487401381) bank7462 +7463 POINT(41.349490688676546 74.80039841348388) bank7463 +7464 POINT(40.87095508214422 73.47897711138963) bank7464 +7465 POINT(40.104077910606044 73.37056134979363) bank7465 +7466 POINT(41.56317517613734 74.76674194282637) bank7466 +7467 POINT(40.359754742433466 74.57140658222009) bank7467 +7468 POINT(41.50298108260163 74.23920414722164) bank7468 +7469 POINT(41.52966313125747 74.99671189867054) bank7469 +7470 POINT(41.61911634705569 74.24942442059037) bank7470 +7471 POINT(40.91464146089029 73.04083115980823) bank7471 +7472 POINT(41.351504666485276 73.54683150520621) bank7472 +7473 POINT(40.242728095821256 74.232646872549) bank7473 +7474 POINT(39.72396933883197 74.86993771378084) bank7474 +7475 POINT(41.36971488611838 73.42936046568367) bank7475 +7476 POINT(39.79806255869078 73.71857256588459) bank7476 +7477 POINT(40.52453455180412 74.4124262414236) bank7477 +7478 POINT(41.00941640971708 73.23660889996088) bank7478 +7479 POINT(40.485211970945265 74.05569857465169) bank7479 +7480 POINT(41.08612630635688 73.44370467829387) bank7480 +7481 POINT(41.01537774442149 73.27248314314359) bank7481 +7482 POINT(41.217165356792194 73.6505505186306) bank7482 +7483 POINT(41.252046529231094 74.29711130479707) bank7483 +7484 POINT(40.56850876299723 73.23952876440703) bank7484 +7485 POINT(41.34797030028336 74.89339838201289) bank7485 +7486 POINT(40.42659752712712 74.63892971921129) bank7486 +7487 POINT(41.10055523253884 74.14145841261018) bank7487 +7488 POINT(41.67951127293724 73.06721020482243) bank7488 +7489 POINT(41.134060051800674 74.1058115808743) bank7489 +7490 POINT(40.46523604772836 73.96217174070281) bank7490 +7491 POINT(40.49055767194023 73.76444400898143) bank7491 +7492 POINT(41.70260247932428 74.21676864165606) bank7492 +7493 POINT(41.61039237732651 74.00921003400461) bank7493 +7494 POINT(40.900718288833254 73.45086507017017) bank7494 +7495 POINT(40.972481997132135 74.665061159764) bank7495 +7496 POINT(41.25528594181185 74.76832222554039) bank7496 +7497 POINT(39.97850270780125 74.22981267183776) bank7497 +7498 POINT(41.60913202871072 73.30248196058345) bank7498 +7499 POINT(41.552903705133716 73.76542347427043) bank7499 +7500 POINT(41.677358836102606 74.29380460124116) bank7500 +7501 POINT(39.85746245809349 73.37378782157197) bank7501 +7502 POINT(41.49455833740409 73.1135202816055) bank7502 +7503 POINT(41.10823132414426 74.97888030159685) bank7503 +7504 POINT(40.6773587800379 74.05587611309508) bank7504 +7505 POINT(39.950637422769624 74.28899980899898) bank7505 +7506 POINT(41.36158112922071 74.45145617951849) bank7506 +7507 POINT(40.51971671382011 73.13583535801105) bank7507 +7508 POINT(40.13884817320971 73.13653886330547) bank7508 +7509 POINT(41.31277155429431 73.89262815341056) bank7509 +7510 POINT(41.04927388352729 74.80216748748624) bank7510 +7511 POINT(40.60691950136191 74.7595466501476) bank7511 +7512 POINT(41.64796454557302 73.73256844257766) bank7512 +7513 POINT(41.551232542959376 73.9065357748617) bank7513 +7514 POINT(40.9805313746423 73.85412658694648) bank7514 +7515 POINT(40.78386979508686 73.94733546852737) bank7515 +7516 POINT(40.627713095567366 74.40787909535706) bank7516 +7517 POINT(40.32022968954393 74.15452068336457) bank7517 +7518 POINT(40.26528169004138 74.16011683948294) bank7518 +7519 POINT(40.518850589328196 73.43005357331592) bank7519 +7520 POINT(39.953837803206426 73.82856048432636) bank7520 +7521 POINT(41.24434821967452 74.9207602040833) bank7521 +7522 POINT(41.13264256847788 74.29381373176145) bank7522 +7523 POINT(40.302762604917774 74.71908393874975) bank7523 +7524 POINT(41.17638342593761 73.12079806429097) bank7524 +7525 POINT(41.20713918276413 73.23833915669778) bank7525 +7526 POINT(40.078182093822704 74.82871818755129) bank7526 +7527 POINT(41.22442613200208 73.38308132883469) bank7527 +7528 POINT(40.16195904998536 74.64582408639812) bank7528 +7529 POINT(40.23409589502991 74.88754859241661) bank7529 +7530 POINT(40.927440180420945 74.87527768165752) bank7530 +7531 POINT(40.71985184687588 74.58361843681725) bank7531 +7532 POINT(41.502547600333244 74.26259811504175) bank7532 +7533 POINT(41.21913395614103 73.43589176492156) bank7533 +7534 POINT(40.07947158222261 74.0675299545137) bank7534 +7535 POINT(39.93491391858543 73.96005854324538) bank7535 +7536 POINT(41.47100831157126 74.65524675495128) bank7536 +7537 POINT(41.07912150741919 74.17356743921927) bank7537 +7538 POINT(40.17520403500017 74.2346359351509) bank7538 +7539 POINT(41.25692443457922 74.14146754713542) bank7539 +7540 POINT(40.92798346146676 73.58330204593494) bank7540 +7541 POINT(39.84295851040448 73.25467951354923) bank7541 +7542 POINT(41.17627708521516 74.25393424855267) bank7542 +7543 POINT(41.42849341013319 74.30071449878356) bank7543 +7544 POINT(40.45119804375203 73.1253819650018) bank7544 +7545 POINT(39.81547903979313 73.41952107406404) bank7545 +7546 POINT(40.37390611159492 73.36495890029244) bank7546 +7547 POINT(40.43954255651431 73.214468638778) bank7547 +7548 POINT(40.18438976214822 73.47350192362516) bank7548 +7549 POINT(39.85683493012554 74.98416113879384) bank7549 +7550 POINT(39.783798429054194 73.48028778574695) bank7550 +7551 POINT(40.777281600940064 73.96498445533899) bank7551 +7552 POINT(41.46669419922695 73.97270023656182) bank7552 +7553 POINT(41.337428409561866 74.17427932577314) bank7553 +7554 POINT(40.85808367545247 74.81102088060355) bank7554 +7555 POINT(40.19466007882858 73.46126938367154) bank7555 +7556 POINT(40.43071825770054 74.45022845411593) bank7556 +7557 POINT(40.587842410783495 73.65077918031386) bank7557 +7558 POINT(41.65891308194777 73.08247708354804) bank7558 +7559 POINT(40.7779998649441 73.71156523357735) bank7559 +7560 POINT(40.73043219583175 74.66625189273029) bank7560 +7561 POINT(39.79344724765928 74.04128571420625) bank7561 +7562 POINT(40.61501682065418 74.40080155732471) bank7562 +7563 POINT(41.61593854947151 73.40182661786497) bank7563 +7564 POINT(40.22087712357724 74.54778694013972) bank7564 +7565 POINT(39.733756328412056 73.07327649086736) bank7565 +7566 POINT(41.67541416574342 73.71685392717004) bank7566 +7567 POINT(41.690858980748864 73.25693883137563) bank7567 +7568 POINT(41.29066031532566 73.3393832394215) bank7568 +7569 POINT(40.27803394744499 73.3367362695089) bank7569 +7570 POINT(41.595822058151384 73.13158504458681) bank7570 +7571 POINT(40.43370456931776 74.39932293275459) bank7571 +7572 POINT(40.12617546565036 74.69451309775131) bank7572 +7573 POINT(41.52344350694261 73.74532189888) bank7573 +7574 POINT(40.66172202562447 73.24119994188753) bank7574 +7575 POINT(41.56767594727226 74.57674061143784) bank7575 +7576 POINT(41.71145098790422 73.84887265715044) bank7576 +7577 POINT(40.777094473621304 73.09720687738073) bank7577 +7578 POINT(41.50375687955359 73.245061848851) bank7578 +7579 POINT(39.81581156577975 73.93030938754688) bank7579 +7580 POINT(41.686889172188856 74.87481502973257) bank7580 +7581 POINT(41.054608094179265 73.06820330530846) bank7581 +7582 POINT(41.36332709894236 73.57051775315067) bank7582 +7583 POINT(40.19798643418127 74.91759292259307) bank7583 +7584 POINT(40.47948176048957 73.97836114232518) bank7584 +7585 POINT(40.874554596345924 73.46573987662794) bank7585 +7586 POINT(41.16829455008598 73.74619623074) bank7586 +7587 POINT(41.632930364785274 73.58381383668613) bank7587 +7588 POINT(40.803208625818485 73.59247214296371) bank7588 +7589 POINT(41.00905218658473 73.65400662781369) bank7589 +7590 POINT(40.281852309383446 73.41629256657636) bank7590 +7591 POINT(41.61862034479924 73.63716557253964) bank7591 +7592 POINT(40.65811572450253 73.12378188539972) bank7592 +7593 POINT(40.99563780932603 74.36575659652519) bank7593 +7594 POINT(39.772531006091825 73.4765948644271) bank7594 +7595 POINT(40.29205927414035 74.53674961397166) bank7595 +7596 POINT(41.1488697086055 73.64566363846727) bank7596 +7597 POINT(40.776882504521524 74.26520510603162) bank7597 +7598 POINT(40.10660969748353 73.39242911674565) bank7598 +7599 POINT(39.81802154824139 73.67978002768822) bank7599 +7600 POINT(41.415336030682795 73.08067097540871) bank7600 +7601 POINT(39.865616460135875 73.46331919435185) bank7601 +7602 POINT(39.85801893404782 74.17040364121611) bank7602 +7603 POINT(40.3225434482459 74.69493673543838) bank7603 +7604 POINT(40.91502034735394 74.67122714001255) bank7604 +7605 POINT(40.40047327902745 73.78756864430795) bank7605 +7606 POINT(41.0016624445 73.13456955894213) bank7606 +7607 POINT(39.959507749529386 74.64225696699125) bank7607 +7608 POINT(41.23685356665678 73.13406210463559) bank7608 +7609 POINT(41.164395704927074 74.93531932521653) bank7609 +7610 POINT(41.3802891695407 73.24196241160033) bank7610 +7611 POINT(41.684682294347255 73.53165779696654) bank7611 +7612 POINT(39.843678930227 74.27846628599438) bank7612 +7613 POINT(40.84123412167265 74.45533994195831) bank7613 +7614 POINT(41.67619844521841 73.91470057823713) bank7614 +7615 POINT(39.78402550469483 73.3310909789031) bank7615 +7616 POINT(40.63257005677305 74.96367784776976) bank7616 +7617 POINT(41.266760709265014 73.31979138277083) bank7617 +7618 POINT(41.00138599137723 74.4992984543099) bank7618 +7619 POINT(40.34576492899943 73.20175489063016) bank7619 +7620 POINT(41.080506149973836 74.58891413318592) bank7620 +7621 POINT(41.706225459400216 73.35088966750348) bank7621 +7622 POINT(41.15617599037569 73.7603425078213) bank7622 +7623 POINT(39.968603490658296 74.89781961812567) bank7623 +7624 POINT(41.691740957620205 74.17290263480268) bank7624 +7625 POINT(41.33171836222718 74.10548829233824) bank7625 +7626 POINT(41.272637398226266 73.03797206668457) bank7626 +7627 POINT(41.39027522006352 74.67581548405275) bank7627 +7628 POINT(41.121184564346834 74.21692985484763) bank7628 +7629 POINT(40.57398831316608 74.61736917396117) bank7629 +7630 POINT(39.82953642627111 74.20490212237566) bank7630 +7631 POINT(41.419214481619704 74.11707638540001) bank7631 +7632 POINT(40.68899505182831 74.65944091723381) bank7632 +7633 POINT(40.232005108504104 74.84997128132771) bank7633 +7634 POINT(41.27002276720308 74.4539234715922) bank7634 +7635 POINT(39.84091870419232 73.22989444762585) bank7635 +7636 POINT(41.65838926334807 74.08429348730779) bank7636 +7637 POINT(40.779210179787285 73.09110551344703) bank7637 +7638 POINT(40.05653130291767 74.1692064268441) bank7638 +7639 POINT(40.08674240057937 73.44720405762202) bank7639 +7640 POINT(40.99347240103635 73.65497678637657) bank7640 +7641 POINT(39.97935399512623 74.09344759024563) bank7641 +7642 POINT(40.17735562604538 74.6764430233566) bank7642 +7643 POINT(40.03909448143617 73.26682038614568) bank7643 +7644 POINT(41.00904557663004 73.5606890104175) bank7644 +7645 POINT(40.88254705126187 74.26234453371231) bank7645 +7646 POINT(39.90680786628636 74.15919909733155) bank7646 +7647 POINT(41.27480767062243 73.56794016343636) bank7647 +7648 POINT(41.570062652482086 73.50593618642979) bank7648 +7649 POINT(41.55375391360722 74.84234960594618) bank7649 +7650 POINT(40.94617165443828 74.84230114498304) bank7650 +7651 POINT(39.858936110924674 73.57234550299734) bank7651 +7652 POINT(39.867526466673624 74.32355231261168) bank7652 +7653 POINT(40.065298344300054 74.50385121270331) bank7653 +7654 POINT(40.62981253373675 73.26853424995734) bank7654 +7655 POINT(41.03870919887078 74.73359382435237) bank7655 +7656 POINT(41.2833299872077 73.45431225669817) bank7656 +7657 POINT(40.01475938685982 73.85328242462359) bank7657 +7658 POINT(41.41579004166124 74.5605305944633) bank7658 +7659 POINT(40.138876177246274 73.93199955360633) bank7659 +7660 POINT(40.6555363204302 73.94987454841207) bank7660 +7661 POINT(39.90372214927947 73.8939060020038) bank7661 +7662 POINT(41.41002154587943 73.60006293516112) bank7662 +7663 POINT(41.075814358884266 73.30888959116106) bank7663 +7664 POINT(39.79136696599149 73.74896345628763) bank7664 +7665 POINT(41.62952480195419 74.32632950311245) bank7665 +7666 POINT(40.19988666455554 73.0386970542963) bank7666 +7667 POINT(40.777701274978284 74.08165808583956) bank7667 +7668 POINT(40.674473521213486 73.71301389322174) bank7668 +7669 POINT(41.58006421210729 73.5746331693795) bank7669 +7670 POINT(39.92340281796174 73.45050194413507) bank7670 +7671 POINT(40.18337056148232 73.36564634426247) bank7671 +7672 POINT(41.66921147145604 74.92416969276694) bank7672 +7673 POINT(40.99110517023701 74.1204359421534) bank7673 +7674 POINT(40.133470802989656 74.25825315517388) bank7674 +7675 POINT(40.71865305665954 74.22595057393409) bank7675 +7676 POINT(41.14195224050259 74.02458533575457) bank7676 +7677 POINT(41.114626551315254 73.83445443182221) bank7677 +7678 POINT(40.357116685554644 74.19666959248875) bank7678 +7679 POINT(40.988077978501664 74.85682554028415) bank7679 +7680 POINT(40.613299048721906 74.95232084611241) bank7680 +7681 POINT(41.27912730555746 73.83316856945216) bank7681 +7682 POINT(40.28846820541314 73.06939772466139) bank7682 +7683 POINT(40.7138281119155 74.84457138502191) bank7683 +7684 POINT(40.9939617973746 74.83590880983829) bank7684 +7685 POINT(40.760097791525695 73.06564821583623) bank7685 +7686 POINT(40.75586852121355 73.96092244231471) bank7686 +7687 POINT(40.54572379751968 74.67764811795621) bank7687 +7688 POINT(39.954135804186876 74.5827583720277) bank7688 +7689 POINT(39.992406933384785 73.13168935468094) bank7689 +7690 POINT(41.218096799247164 74.6775958809096) bank7690 +7691 POINT(41.5211450726277 74.03267330495487) bank7691 +7692 POINT(39.860722512959235 74.67861801335897) bank7692 +7693 POINT(40.11957094180932 73.964756149641) bank7693 +7694 POINT(41.52241113715118 74.7197963124796) bank7694 +7695 POINT(41.37772108542995 74.3722111640059) bank7695 +7696 POINT(40.78790435678199 74.11575708402374) bank7696 +7697 POINT(40.242918244093886 74.27684065270549) bank7697 +7698 POINT(41.08110420837336 74.33235422079551) bank7698 +7699 POINT(39.99810767042659 74.93939927979036) bank7699 +7700 POINT(39.794911381241256 73.41307208168591) bank7700 +7701 POINT(39.809946551859696 74.46898574024529) bank7701 +7702 POINT(40.060378691832454 73.98980550665848) bank7702 +7703 POINT(40.55135138363153 74.35397033818728) bank7703 +7704 POINT(39.88399395521563 74.7517139119147) bank7704 +7705 POINT(40.79482656071223 73.66986517463498) bank7705 +7706 POINT(40.60939717660827 73.46312694487304) bank7706 +7707 POINT(41.51384482883557 73.15293614995134) bank7707 +7708 POINT(40.558495237348254 73.55538971406196) bank7708 +7709 POINT(40.0838375445363 74.36218478487315) bank7709 +7710 POINT(40.259870142581846 73.8457579404719) bank7710 +7711 POINT(40.97666494761635 73.57401141617345) bank7711 +7712 POINT(40.99499421285301 74.50621204085645) bank7712 +7713 POINT(40.448261968823346 73.72217779630678) bank7713 +7714 POINT(40.96928290640855 73.49874721145359) bank7714 +7715 POINT(40.961616441181846 74.2371037573236) bank7715 +7716 POINT(39.80597404714578 74.13604683421447) bank7716 +7717 POINT(41.094421176786724 73.42965722200073) bank7717 +7718 POINT(41.29784699274426 74.41878374426862) bank7718 +7719 POINT(39.95412527681438 73.6978750248518) bank7719 +7720 POINT(41.1085170160231 74.11305159053735) bank7720 +7721 POINT(40.5194036127113 74.6156142992106) bank7721 +7722 POINT(40.33760167602028 73.33504881313362) bank7722 +7723 POINT(40.27428453919755 74.16608437937592) bank7723 +7724 POINT(40.58272997722783 74.39778934392221) bank7724 +7725 POINT(39.98366131513177 73.56771590128348) bank7725 +7726 POINT(40.26645192919041 73.13206552942367) bank7726 +7727 POINT(40.8134329460856 74.65567142248624) bank7727 +7728 POINT(39.7143579841175 73.07750016058965) bank7728 +7729 POINT(40.557501511337165 74.11832184705288) bank7729 +7730 POINT(40.32417844708557 73.12590027365441) bank7730 +7731 POINT(40.86233857036299 74.81739045995606) bank7731 +7732 POINT(39.79441954175216 73.84244750997308) bank7732 +7733 POINT(41.297530956982946 74.13973218102193) bank7733 +7734 POINT(39.82540360756061 73.08827298279473) bank7734 +7735 POINT(41.07738406905109 74.56235652961979) bank7735 +7736 POINT(41.686322658427784 73.56192434562492) bank7736 +7737 POINT(41.14087619079983 74.11993799411658) bank7737 +7738 POINT(40.09167500781191 73.59891229423968) bank7738 +7739 POINT(39.98153296765389 74.92157655856198) bank7739 +7740 POINT(41.15427362185846 74.23589559675963) bank7740 +7741 POINT(41.124451569167505 73.63964273332525) bank7741 +7742 POINT(41.632396739879475 73.48678272875668) bank7742 +7743 POINT(41.19716514028293 73.99479356038918) bank7743 +7744 POINT(40.21979127172164 73.40663591168837) bank7744 +7745 POINT(40.487182607008776 74.17885433015572) bank7745 +7746 POINT(40.60911683350226 74.91613159407525) bank7746 +7747 POINT(40.09936814375959 74.32413750866796) bank7747 +7748 POINT(39.8604288831504 74.09448737383978) bank7748 +7749 POINT(41.321542790518336 73.59558718357287) bank7749 +7750 POINT(39.97642804514173 74.37349929173084) bank7750 +7751 POINT(41.0701631189148 74.01208804612118) bank7751 +7752 POINT(41.095827394107936 74.8296011534276) bank7752 +7753 POINT(41.59698346172843 73.17897635109631) bank7753 +7754 POINT(41.25792402302947 74.13857389108112) bank7754 +7755 POINT(40.06008052883709 74.57363029475432) bank7755 +7756 POINT(40.00884286151433 73.44384111867298) bank7756 +7757 POINT(40.49113878184219 73.61785686261544) bank7757 +7758 POINT(39.94824743175209 74.89384742902335) bank7758 +7759 POINT(40.73106519328273 73.20131283924877) bank7759 +7760 POINT(40.73995672696248 74.28143453527235) bank7760 +7761 POINT(40.232048709772215 73.7469629896271) bank7761 +7762 POINT(39.91564888330514 73.58703688178082) bank7762 +7763 POINT(41.06135018720858 74.20097129477419) bank7763 +7764 POINT(41.17410396463942 74.45388307966651) bank7764 +7765 POINT(40.097287785268826 73.98540826824826) bank7765 +7766 POINT(41.33278009956094 73.92292723704891) bank7766 +7767 POINT(40.223730634371144 74.25293770014547) bank7767 +7768 POINT(39.896724541318605 74.34915757852164) bank7768 +7769 POINT(40.44229039292032 74.80257379783255) bank7769 +7770 POINT(39.793062741010274 73.17420913053215) bank7770 +7771 POINT(40.69384085057408 74.06062547735468) bank7771 +7772 POINT(40.83847793884419 73.58816388654498) bank7772 +7773 POINT(39.925647895212805 74.40407203641189) bank7773 +7774 POINT(39.907936598067494 73.5880205751885) bank7774 +7775 POINT(40.07238150333898 74.52400854477067) bank7775 +7776 POINT(41.32466521874351 73.72877047668514) bank7776 +7777 POINT(41.58626918982597 74.46256174716284) bank7777 +7778 POINT(40.208994979420225 74.60171729295749) bank7778 +7779 POINT(40.804087656163006 73.65242302204385) bank7779 +7780 POINT(40.4340867320594 74.76102212391747) bank7780 +7781 POINT(41.24997271747481 73.79660503227116) bank7781 +7782 POINT(40.32852539734276 74.3765444289106) bank7782 +7783 POINT(40.74653085464303 73.23617298136162) bank7783 +7784 POINT(41.007474376417306 73.75700069535652) bank7784 +7785 POINT(40.35311529574665 73.66109791269488) bank7785 +7786 POINT(40.23310498421417 73.49271200512956) bank7786 +7787 POINT(40.259013631946935 73.16187271058942) bank7787 +7788 POINT(41.13189541994255 74.99617319428936) bank7788 +7789 POINT(40.34687499861654 74.18822081837621) bank7789 +7790 POINT(40.11417539556618 74.8500207455357) bank7790 +7791 POINT(41.55604832261247 73.8061016914831) bank7791 +7792 POINT(41.32885067632535 74.12016484947851) bank7792 +7793 POINT(41.5127456846715 74.77585142115426) bank7793 +7794 POINT(41.48034645544316 74.11880483235801) bank7794 +7795 POINT(40.806359860059366 74.40977664363031) bank7795 +7796 POINT(41.14548081853014 73.38807544406872) bank7796 +7797 POINT(40.102989973473754 74.75213042563554) bank7797 +7798 POINT(41.47264266003336 74.27809141030141) bank7798 +7799 POINT(40.71483067143762 74.56361267994907) bank7799 +7800 POINT(41.64951127352325 73.37550903992215) bank7800 +7801 POINT(41.08435788931535 73.08982042786205) bank7801 +7802 POINT(41.172008833725485 73.65714103795277) bank7802 +7803 POINT(40.88312088787367 74.16209565734283) bank7803 +7804 POINT(41.18752553450985 73.27750608620705) bank7804 +7805 POINT(40.563101898312695 74.44175340323449) bank7805 +7806 POINT(40.30036856286473 73.30031443795558) bank7806 +7807 POINT(40.807279667790404 73.17442534208529) bank7807 +7808 POINT(40.71349499212847 73.77398652063825) bank7808 +7809 POINT(41.574462664831884 73.12645131500368) bank7809 +7810 POINT(40.95768175056019 73.20300850745647) bank7810 +7811 POINT(41.32923348459709 74.5474578400196) bank7811 +7812 POINT(41.687402127416746 74.09364954092196) bank7812 +7813 POINT(41.630060011034026 73.6715142998516) bank7813 +7814 POINT(40.37274752098645 73.25266689736054) bank7814 +7815 POINT(41.58803901664068 73.63189342647337) bank7815 +7816 POINT(40.486194768852805 74.60461845821811) bank7816 +7817 POINT(41.540773594974624 73.24089854719534) bank7817 +7818 POINT(39.85294437157586 74.97616418458665) bank7818 +7819 POINT(40.464625038291764 73.67883013752419) bank7819 +7820 POINT(40.43162595815223 74.41144430307426) bank7820 +7821 POINT(39.97330171630475 73.24364353638975) bank7821 +7822 POINT(40.34584295551932 74.30085865015097) bank7822 +7823 POINT(41.54590396636596 73.45658182797423) bank7823 +7824 POINT(41.588197520495235 73.92747070396315) bank7824 +7825 POINT(40.23779045587771 73.84946861996181) bank7825 +7826 POINT(41.580098835856106 73.4806845436198) bank7826 +7827 POINT(40.25005569971921 74.09085595662417) bank7827 +7828 POINT(39.714347894360714 73.30798969253345) bank7828 +7829 POINT(39.73945261077413 74.50050511409023) bank7829 +7830 POINT(40.53367343883267 74.79421715487989) bank7830 +7831 POINT(41.00597634035851 73.68919635112971) bank7831 +7832 POINT(39.98233321053794 73.18272755614979) bank7832 +7833 POINT(41.61951559832655 74.37523530499455) bank7833 +7834 POINT(40.733058105518516 73.58348106061966) bank7834 +7835 POINT(41.42089541773484 73.42575272768288) bank7835 +7836 POINT(39.826634698192144 73.1311992955874) bank7836 +7837 POINT(40.37377323430906 74.3007113765239) bank7837 +7838 POINT(39.875441263454604 74.80504186356859) bank7838 +7839 POINT(39.81476002364928 73.35813519397803) bank7839 +7840 POINT(41.232536142839066 74.24668910460248) bank7840 +7841 POINT(41.51340192683719 74.29666878513754) bank7841 +7842 POINT(39.926267260751615 73.96769798138445) bank7842 +7843 POINT(41.391228694672066 74.84848455997484) bank7843 +7844 POINT(41.03320533307862 75.00071667371604) bank7844 +7845 POINT(40.116139916765015 73.54474383722422) bank7845 +7846 POINT(41.15968531684933 74.55776315533542) bank7846 +7847 POINT(41.699832639441034 73.06193822675891) bank7847 +7848 POINT(39.786099046834146 74.60456792087857) bank7848 +7849 POINT(40.83101884175698 73.52445074073152) bank7849 +7850 POINT(40.41579335683582 74.2107730101309) bank7850 +7851 POINT(39.72101557191503 74.44138690995457) bank7851 +7852 POINT(39.894418717581445 74.50918818135354) bank7852 +7853 POINT(40.78630725476325 74.1141005542123) bank7853 +7854 POINT(40.20138244665818 74.44885156146552) bank7854 +7855 POINT(41.49503548995508 73.69495229547461) bank7855 +7856 POINT(40.64505523303974 74.99788245684516) bank7856 +7857 POINT(40.99887875801368 73.43010337887998) bank7857 +7858 POINT(41.5812741524165 73.38280165941464) bank7858 +7859 POINT(41.62005906018038 73.76996402275431) bank7859 +7860 POINT(41.04076877008144 73.0705813674353) bank7860 +7861 POINT(39.976413487185944 74.37880019953225) bank7861 +7862 POINT(40.68766531518157 73.76457878729111) bank7862 +7863 POINT(41.616691075488276 74.37866573038023) bank7863 +7864 POINT(41.575991490188606 73.46595664040531) bank7864 +7865 POINT(40.0473443025111 74.45770546309093) bank7865 +7866 POINT(40.29632888004301 74.55273196413296) bank7866 +7867 POINT(40.68674167401118 74.75834834076636) bank7867 +7868 POINT(41.43057342096087 74.31985217018999) bank7868 +7869 POINT(41.02094591547869 74.05270084708259) bank7869 +7870 POINT(40.513313587587945 74.37492680496425) bank7870 +7871 POINT(39.88612174594657 74.43174194214187) bank7871 +7872 POINT(41.215163771108465 74.43134631184645) bank7872 +7873 POINT(40.9842814277383 74.06490289070743) bank7873 +7874 POINT(41.16800125101829 73.86793748309132) bank7874 +7875 POINT(40.57845154132741 73.97137231788226) bank7875 +7876 POINT(41.42658413764107 74.2216340101749) bank7876 +7877 POINT(40.663577505958806 73.92792862702323) bank7877 +7878 POINT(40.298672392746404 74.8192971021562) bank7878 +7879 POINT(39.8972769190489 74.25403344775341) bank7879 +7880 POINT(40.0366245758437 74.43784330480372) bank7880 +7881 POINT(40.84447718762058 73.19783002974881) bank7881 +7882 POINT(41.29100266032939 74.26005804231495) bank7882 +7883 POINT(40.54070708895926 73.03563381380073) bank7883 +7884 POINT(39.72984878368966 74.64725415467946) bank7884 +7885 POINT(39.727199141900684 73.64896009686181) bank7885 +7886 POINT(40.28620504646227 74.78587708164065) bank7886 +7887 POINT(40.79391687254494 74.57483807343176) bank7887 +7888 POINT(39.97806356885174 73.3422318434447) bank7888 +7889 POINT(40.49505996282593 74.36674880939904) bank7889 +7890 POINT(40.918363166754155 74.49838040282168) bank7890 +7891 POINT(40.7792274051292 74.08875161180242) bank7891 +7892 POINT(39.74808941075778 73.17733166043293) bank7892 +7893 POINT(41.681040049552465 74.27082106411596) bank7893 +7894 POINT(40.25094494338468 74.4344333200089) bank7894 +7895 POINT(40.21966745907994 73.19469411643225) bank7895 +7896 POINT(41.34210006366104 73.99854456372206) bank7896 +7897 POINT(41.36209988827009 74.62790285344383) bank7897 +7898 POINT(40.137817997841296 73.70691632640808) bank7898 +7899 POINT(40.56635513677808 74.4458653061335) bank7899 +7900 POINT(40.171789306581324 74.24432712854176) bank7900 +7901 POINT(41.48290981836798 74.2610540001112) bank7901 +7902 POINT(40.58135660275641 74.6474347561456) bank7902 +7903 POINT(40.35497273828183 74.20070091640646) bank7903 +7904 POINT(40.61954261361239 74.3619968897257) bank7904 +7905 POINT(40.59837586979447 73.72469119776551) bank7905 +7906 POINT(41.08392085190248 74.34357176312673) bank7906 +7907 POINT(40.53754170408471 74.9195650275785) bank7907 +7908 POINT(40.168474829178244 74.89093069632841) bank7908 +7909 POINT(40.09162986497407 73.38749161532282) bank7909 +7910 POINT(40.90355843919653 74.3034604060047) bank7910 +7911 POINT(41.65231391189634 73.3770647849837) bank7911 +7912 POINT(40.2321982530933 74.89887626354295) bank7912 +7913 POINT(41.66680281289464 74.66743305494484) bank7913 +7914 POINT(40.26504742608734 73.32059200882732) bank7914 +7915 POINT(39.9086293678222 74.86585634842776) bank7915 +7916 POINT(40.04918927546285 74.66625493297913) bank7916 +7917 POINT(41.111248953660024 74.7380191422798) bank7917 +7918 POINT(40.50401421612252 74.33276021811463) bank7918 +7919 POINT(41.013807854150386 73.15585456722397) bank7919 +7920 POINT(40.713788798677825 74.47294333016303) bank7920 +7921 POINT(41.48094263578194 73.9437337409626) bank7921 +7922 POINT(41.500353697173566 73.99875605166073) bank7922 +7923 POINT(41.179724135098375 73.8213938617684) bank7923 +7924 POINT(39.861917205157745 74.82264796260641) bank7924 +7925 POINT(41.26613851326427 73.32384935601547) bank7925 +7926 POINT(41.225101977185865 73.18987811833799) bank7926 +7927 POINT(40.55019365763493 74.5857752956886) bank7927 +7928 POINT(41.24860162677002 73.16630698468542) bank7928 +7929 POINT(40.3919927005939 73.1509461878397) bank7929 +7930 POINT(40.62352618445669 74.30010938536124) bank7930 +7931 POINT(41.63501238722894 73.2276403552667) bank7931 +7932 POINT(41.20026445742444 73.72429048080103) bank7932 +7933 POINT(41.14197419988204 73.58787198755954) bank7933 +7934 POINT(41.24069405096303 74.38608080766952) bank7934 +7935 POINT(39.919529421758504 73.76936317537645) bank7935 +7936 POINT(40.61488703221013 73.13003105013277) bank7936 +7937 POINT(40.241613419955456 74.55619306478124) bank7937 +7938 POINT(41.43663278559504 74.4079465935334) bank7938 +7939 POINT(40.58367699209238 74.63619005941923) bank7939 +7940 POINT(41.21015728762805 74.88764146436232) bank7940 +7941 POINT(40.30750821976604 74.50310321492573) bank7941 +7942 POINT(40.99115324292497 74.96492580262809) bank7942 +7943 POINT(40.144232910830084 74.59696500700987) bank7943 +7944 POINT(41.66747272717448 74.32233096930489) bank7944 +7945 POINT(41.15490906886135 73.84064865360637) bank7945 +7946 POINT(39.94670943225724 73.51313740864069) bank7946 +7947 POINT(40.351307135819894 74.50363803900073) bank7947 +7948 POINT(41.40065525385778 74.55275322426449) bank7948 +7949 POINT(40.170846378608 74.5681044260326) bank7949 +7950 POINT(41.50122307431257 73.23339721774698) bank7950 +7951 POINT(40.94322314343483 74.4684565990448) bank7951 +7952 POINT(40.555088284023554 73.20686100391472) bank7952 +7953 POINT(40.68344554157462 74.92757127026184) bank7953 +7954 POINT(41.48700485686974 73.13882441678201) bank7954 +7955 POINT(40.30741925601117 74.99934102034905) bank7955 +7956 POINT(41.681421998167274 73.51628886780671) bank7956 +7957 POINT(40.28257146219033 74.65733951631735) bank7957 +7958 POINT(41.04536290334748 73.89969464718419) bank7958 +7959 POINT(40.9113573906801 73.1224033164643) bank7959 +7960 POINT(40.283513349024496 74.12907938693749) bank7960 +7961 POINT(41.30856090108869 74.38172162008772) bank7961 +7962 POINT(41.05461698922102 73.68349297214563) bank7962 +7963 POINT(41.49956522415672 74.64082876213055) bank7963 +7964 POINT(41.064824866113575 74.43446632807922) bank7964 +7965 POINT(39.837969967786265 73.08885185576243) bank7965 +7966 POINT(39.91232264922341 73.56475432664789) bank7966 +7967 POINT(40.625928861708644 74.40232193154745) bank7967 +7968 POINT(40.159772337492605 74.77548463234946) bank7968 +7969 POINT(40.88474705979288 73.6048716453656) bank7969 +7970 POINT(41.70188004824612 74.12763032198971) bank7970 +7971 POINT(40.54043102314291 74.97177783857967) bank7971 +7972 POINT(39.935051622663615 73.43380257325263) bank7972 +7973 POINT(40.57069290665739 74.11526131851406) bank7973 +7974 POINT(40.9956449940516 74.86008754326839) bank7974 +7975 POINT(41.01803459486688 74.53606858118268) bank7975 +7976 POINT(39.775263441633335 74.54334566013586) bank7976 +7977 POINT(39.906399430733686 73.40912439725064) bank7977 +7978 POINT(40.98280106693811 74.78948746015763) bank7978 +7979 POINT(40.07961368473656 73.99236721653733) bank7979 +7980 POINT(41.6217126933666 74.94058784235226) bank7980 +7981 POINT(39.81200211596364 74.41758020923535) bank7981 +7982 POINT(40.06693354690199 73.58012117453659) bank7982 +7983 POINT(39.759791349148735 74.01422876815329) bank7983 +7984 POINT(40.506308711295105 74.56340271407477) bank7984 +7985 POINT(39.85630001179883 73.44236818447781) bank7985 +7986 POINT(39.87320009404669 73.32968422636613) bank7986 +7987 POINT(40.22086691922792 74.13515371748385) bank7987 +7988 POINT(41.70645250175279 73.82594414626467) bank7988 +7989 POINT(40.288183211289635 73.70785433445695) bank7989 +7990 POINT(41.67760069819798 74.59700321352221) bank7990 +7991 POINT(41.40017382105603 73.77048796766584) bank7991 +7992 POINT(39.92382768792412 73.07850841673728) bank7992 +7993 POINT(39.97798508068401 73.45519181789184) bank7993 +7994 POINT(40.247165247472935 73.24927432159373) bank7994 +7995 POINT(40.569104352831296 74.0909306877984) bank7995 +7996 POINT(41.52765377218202 74.32889231663465) bank7996 +7997 POINT(39.769896924266135 73.84052319434984) bank7997 +7998 POINT(40.860548282832276 74.59161611551268) bank7998 +7999 POINT(41.630161887240696 73.25873478702677) bank7999 +8000 POINT(40.27574706039452 74.46973162777805) bank8000 +8001 POINT(41.30951099650127 74.27728489187736) bank8001 +8002 POINT(40.53024656510212 74.52834025591252) bank8002 +8003 POINT(40.13640537296071 74.06949799450743) bank8003 +8004 POINT(40.9065820268335 74.16250938756873) bank8004 +8005 POINT(40.4017120436875 74.37364926396879) bank8005 +8006 POINT(41.000020299107476 74.98769966438806) bank8006 +8007 POINT(39.78434932388279 73.79735660104127) bank8007 +8008 POINT(40.86505342555464 74.32410003172134) bank8008 +8009 POINT(40.65810691167871 73.66900189239213) bank8009 +8010 POINT(41.2513640046421 73.38235904860154) bank8010 +8011 POINT(40.73799902337131 74.5307695609693) bank8011 +8012 POINT(41.31243777237755 73.82375919572816) bank8012 +8013 POINT(40.07560632242935 74.21452241501402) bank8013 +8014 POINT(41.198969566997235 74.77297854602902) bank8014 +8015 POINT(40.52497069832418 73.7424495160394) bank8015 +8016 POINT(41.244619962967334 74.62367358197787) bank8016 +8017 POINT(40.95717165468034 73.58412064860364) bank8017 +8018 POINT(39.81973000330301 74.24422079687155) bank8018 +8019 POINT(41.40995358151702 74.36800171176208) bank8019 +8020 POINT(41.19809824838845 74.22617946902163) bank8020 +8021 POINT(41.085897313261846 74.99986984875252) bank8021 +8022 POINT(40.1411786235417 73.80880218047612) bank8022 +8023 POINT(40.33615550206268 74.72161841253195) bank8023 +8024 POINT(40.869867166415425 73.05475121201572) bank8024 +8025 POINT(40.18611167925304 74.8081107076871) bank8025 +8026 POINT(40.618566904260156 74.80911617097414) bank8026 +8027 POINT(41.03662041577731 73.23074072334799) bank8027 +8028 POINT(41.42086050425668 74.75405262489323) bank8028 +8029 POINT(40.906121296982974 74.51575315835724) bank8029 +8030 POINT(41.16301747867038 74.84051225140209) bank8030 +8031 POINT(40.202932073206135 73.83025386920406) bank8031 +8032 POINT(40.20946727135245 74.85789175885714) bank8032 +8033 POINT(39.730813863163426 74.00190886647837) bank8033 +8034 POINT(40.194914283163754 74.44384631992334) bank8034 +8035 POINT(41.021279211586375 74.99050370956861) bank8035 +8036 POINT(41.3796631629528 74.34563118121297) bank8036 +8037 POINT(40.29574991445873 73.84135955677105) bank8037 +8038 POINT(40.93554173250899 74.95066163842651) bank8038 +8039 POINT(40.73686244442393 73.82746378511831) bank8039 +8040 POINT(41.39071143830814 74.21472856359021) bank8040 +8041 POINT(40.377115181560285 73.60283669356348) bank8041 +8042 POINT(41.29277612868192 73.3012755714188) bank8042 +8043 POINT(41.295918298265654 74.96528884665084) bank8043 +8044 POINT(40.74499137427011 74.47246289649084) bank8044 +8045 POINT(40.5288370618766 74.55076273750583) bank8045 +8046 POINT(41.2879703552055 74.43156365089543) bank8046 +8047 POINT(39.94509652319063 74.95094700098377) bank8047 +8048 POINT(39.74319180654581 73.48199845458619) bank8048 +8049 POINT(39.809603735711285 73.70499658782357) bank8049 +8050 POINT(41.54918634106648 73.06172682951437) bank8050 +8051 POINT(40.13028588170231 73.33296453306045) bank8051 +8052 POINT(41.63713636162625 74.08135390551173) bank8052 +8053 POINT(40.12536571685377 73.43509453528239) bank8053 +8054 POINT(39.91284355448639 73.18912427985093) bank8054 +8055 POINT(41.24230257415566 75.00276547123627) bank8055 +8056 POINT(40.05174281144265 74.1672133321936) bank8056 +8057 POINT(40.18198501915306 73.79304625435104) bank8057 +8058 POINT(40.00865233659892 73.67884057724392) bank8058 +8059 POINT(40.34037521862854 73.88915931061064) bank8059 +8060 POINT(39.891558131512134 73.0534262437916) bank8060 +8061 POINT(39.935067834448915 73.8262585311619) bank8061 +8062 POINT(41.18500795549638 74.448237306982) bank8062 +8063 POINT(40.9521658743435 74.50514894785408) bank8063 +8064 POINT(41.58854115762581 74.5873511515444) bank8064 +8065 POINT(39.85192189534192 73.51302466484015) bank8065 +8066 POINT(41.27475794206612 74.97712392637135) bank8066 +8067 POINT(40.60612263274281 74.42141414915204) bank8067 +8068 POINT(40.19538081858481 73.67647964340127) bank8068 +8069 POINT(40.6315441702459 73.79152936610271) bank8069 +8070 POINT(41.16261236780136 74.56353475266062) bank8070 +8071 POINT(39.94555426765378 73.82210840169033) bank8071 +8072 POINT(40.246517251250964 73.77551161888353) bank8072 +8073 POINT(41.04606768652917 73.28429933641898) bank8073 +8074 POINT(40.320243180592776 73.36130146778551) bank8074 +8075 POINT(40.2165380822647 73.85154406769564) bank8075 +8076 POINT(40.3064678689974 73.17496637234227) bank8076 +8077 POINT(40.62366363839959 74.48355672073161) bank8077 +8078 POINT(40.61934453860193 73.76691305071384) bank8078 +8079 POINT(40.87894341779972 73.10587709731348) bank8079 +8080 POINT(41.315386594062765 73.54219765922845) bank8080 +8081 POINT(39.8227800162962 73.192788031222) bank8081 +8082 POINT(39.7891070669404 73.13156010184821) bank8082 +8083 POINT(40.77430208299365 74.37497881938026) bank8083 +8084 POINT(40.782930040605706 74.92869973558781) bank8084 +8085 POINT(40.96543548121144 73.21889032569351) bank8085 +8086 POINT(41.04046362082487 73.34441213934927) bank8086 +8087 POINT(39.80514259130804 74.76350394945463) bank8087 +8088 POINT(40.550490886283534 74.29543017485196) bank8088 +8089 POINT(41.407095553236246 73.38129931207187) bank8089 +8090 POINT(40.7596794116953 73.56441219498919) bank8090 +8091 POINT(40.76314897367211 74.07521792567591) bank8091 +8092 POINT(41.08684616368965 74.70322674898986) bank8092 +8093 POINT(40.44510592043082 73.69245448394584) bank8093 +8094 POINT(40.9599526472137 74.25087320950918) bank8094 +8095 POINT(41.24636604944446 73.9186374776194) bank8095 +8096 POINT(40.725039826159964 73.22539528294705) bank8096 +8097 POINT(41.180471343044125 74.31029186534549) bank8097 +8098 POINT(40.368269581784695 73.59790980081601) bank8098 +8099 POINT(41.67490947460407 74.20472075230121) bank8099 +8100 POINT(40.204789631381516 74.95216548119777) bank8100 +8101 POINT(40.64543002002144 73.4934916078702) bank8101 +8102 POINT(41.15735591288302 74.17120689551443) bank8102 +8103 POINT(40.87870946782666 73.56087368323179) bank8103 +8104 POINT(39.819410936672156 74.27957235806747) bank8104 +8105 POINT(41.41579081555778 73.06009428828241) bank8105 +8106 POINT(41.20234831316939 73.32436563731532) bank8106 +8107 POINT(40.496511001698856 74.24344145339423) bank8107 +8108 POINT(41.33455330193572 73.56313731731154) bank8108 +8109 POINT(40.79064594634961 73.60553292082083) bank8109 +8110 POINT(40.40954045686151 74.43190979114063) bank8110 +8111 POINT(40.151148244653974 73.61089160289322) bank8111 +8112 POINT(40.85314846977487 74.20657958510458) bank8112 +8113 POINT(41.249521795527166 74.99391675144923) bank8113 +8114 POINT(39.794664917828044 74.13967527984757) bank8114 +8115 POINT(39.87256503665923 73.29621278183701) bank8115 +8116 POINT(39.848893918565786 74.52395660659329) bank8116 +8117 POINT(41.57962205221738 73.47001212125377) bank8117 +8118 POINT(41.68264797214881 73.84595718804498) bank8118 +8119 POINT(40.24958645866134 74.10926755858036) bank8119 +8120 POINT(40.51987941091085 73.35749199522125) bank8120 +8121 POINT(40.83564571990963 74.11483332963317) bank8121 +8122 POINT(40.639695966222085 74.9815316986618) bank8122 +8123 POINT(40.66831885434926 74.4875458278135) bank8123 +8124 POINT(40.96180004943278 74.50626738148281) bank8124 +8125 POINT(41.67859969032571 73.44146675690567) bank8125 +8126 POINT(39.78236191055704 73.69019823413485) bank8126 +8127 POINT(41.503879128318786 73.70648790256132) bank8127 +8128 POINT(41.21167001308762 73.21594999661085) bank8128 +8129 POINT(41.46096273804821 74.2576080551619) bank8129 +8130 POINT(40.121164461713555 73.36521707608658) bank8130 +8131 POINT(40.937071422866296 73.74746074192458) bank8131 +8132 POINT(40.52486657030217 74.0696009729667) bank8132 +8133 POINT(40.69192989972049 74.64391511551156) bank8133 +8134 POINT(39.95845202010104 74.74974217281776) bank8134 +8135 POINT(41.58590325471369 73.98730222117219) bank8135 +8136 POINT(41.40021396607955 74.13751708119997) bank8136 +8137 POINT(41.452379988868756 74.54509224842438) bank8137 +8138 POINT(41.526951418521016 73.49130608512355) bank8138 +8139 POINT(40.811316108314955 74.07064021863978) bank8139 +8140 POINT(39.90344600658783 73.82229354718584) bank8140 +8141 POINT(40.602881913476146 73.93873416326367) bank8141 +8142 POINT(40.444096007347945 73.36739720212195) bank8142 +8143 POINT(40.813452976595876 73.67906657612545) bank8143 +8144 POINT(41.217121774184314 74.12840021243537) bank8144 +8145 POINT(41.63678226442151 74.53852550610456) bank8145 +8146 POINT(40.07037583992924 73.09076860947943) bank8146 +8147 POINT(41.68081001407079 74.05506470146749) bank8147 +8148 POINT(41.123906350275135 73.87251490362948) bank8148 +8149 POINT(41.09218524567926 73.06555220562429) bank8149 +8150 POINT(40.08860643576784 73.16924130706204) bank8150 +8151 POINT(41.65471920126007 73.16092270156429) bank8151 +8152 POINT(41.019411501861505 73.1664106684728) bank8152 +8153 POINT(41.01283945387316 74.93973575533428) bank8153 +8154 POINT(40.127400240603585 73.58004939508878) bank8154 +8155 POINT(40.65586532387096 73.0782831052699) bank8155 +8156 POINT(40.44332646037757 73.0966152322556) bank8156 +8157 POINT(41.016061440291764 73.46067604509179) bank8157 +8158 POINT(40.04179340098161 74.19750777830063) bank8158 +8159 POINT(40.850995238031665 73.2107709038691) bank8159 +8160 POINT(40.127740926867276 73.88081213248292) bank8160 +8161 POINT(40.710656758684785 74.55293565585784) bank8161 +8162 POINT(40.66445329397846 74.73770217007996) bank8162 +8163 POINT(40.62944959661903 73.86870279271689) bank8163 +8164 POINT(40.96910139255501 73.1116162542282) bank8164 +8165 POINT(41.57685211969814 74.66151925936197) bank8165 +8166 POINT(41.63489386449439 73.37421692706393) bank8166 +8167 POINT(41.40570395011604 73.39246851456947) bank8167 +8168 POINT(40.754951485015965 73.24742498524259) bank8168 +8169 POINT(41.578355498638075 73.70908306861877) bank8169 +8170 POINT(39.83844271742714 73.5590399930747) bank8170 +8171 POINT(39.75984572926247 74.39508893667568) bank8171 +8172 POINT(41.50055355745198 73.13695777435801) bank8172 +8173 POINT(40.290082096930085 73.85759522487078) bank8173 +8174 POINT(41.41184767704457 73.57436687750958) bank8174 +8175 POINT(41.04104609990556 74.46609974680507) bank8175 +8176 POINT(40.114967641998334 74.6558279498445) bank8176 +8177 POINT(40.049392376319396 74.64851194863175) bank8177 +8178 POINT(40.824606127683154 74.47716106590326) bank8178 +8179 POINT(39.7951472988369 74.59666923829127) bank8179 +8180 POINT(40.21992434283618 74.23617897097611) bank8180 +8181 POINT(40.542572826111716 73.27089117260445) bank8181 +8182 POINT(40.82572469843381 73.60076141969569) bank8182 +8183 POINT(40.07345859714477 73.60451884987751) bank8183 +8184 POINT(39.93817325532063 73.38514496644807) bank8184 +8185 POINT(39.877210012377354 74.19995281082329) bank8185 +8186 POINT(41.26294271219078 74.88898427800417) bank8186 +8187 POINT(41.21278163749448 73.58340677199867) bank8187 +8188 POINT(41.381770534160786 73.16911500459966) bank8188 +8189 POINT(40.475304326386954 74.45311469332738) bank8189 +8190 POINT(41.1789872839763 74.48397926867324) bank8190 +8191 POINT(40.88463567811364 73.79759251124085) bank8191 +8192 POINT(39.88738958092788 73.05125407901721) bank8192 +8193 POINT(41.159420842874994 74.35476909046194) bank8193 +8194 POINT(40.51842374244657 74.47084628206943) bank8194 +8195 POINT(40.98938130228678 73.6505168820308) bank8195 +8196 POINT(40.94233082440955 73.7692476908546) bank8196 +8197 POINT(41.100717217878305 74.66040376035362) bank8197 +8198 POINT(40.10850400867792 73.74555266883377) bank8198 +8199 POINT(39.73731535649774 74.47387892185378) bank8199 +8200 POINT(41.27662360435093 73.06621801190559) bank8200 +8201 POINT(40.644916087603036 74.37665044732285) bank8201 +8202 POINT(40.28646173292073 73.14371679074384) bank8202 +8203 POINT(40.33245662108792 74.07204480306532) bank8203 +8204 POINT(40.851193650085925 74.61034733950082) bank8204 +8205 POINT(41.3689006887781 74.87921703178812) bank8205 +8206 POINT(41.14145950146212 74.91717442093599) bank8206 +8207 POINT(41.40942746528827 73.39830132160961) bank8207 +8208 POINT(40.907377642951836 74.46452656423338) bank8208 +8209 POINT(41.57152712846868 73.28339154562137) bank8209 +8210 POINT(40.072148079791475 73.82570303012747) bank8210 +8211 POINT(41.364255973866634 74.28024111644463) bank8211 +8212 POINT(40.18477049764464 73.20494192852685) bank8212 +8213 POINT(40.796412532798485 73.54363299106268) bank8213 +8214 POINT(40.2590339903218 73.32600914557955) bank8214 +8215 POINT(40.38737899354244 74.40365577792029) bank8215 +8216 POINT(39.99303867065063 74.04538779472361) bank8216 +8217 POINT(41.1292706383671 73.38997596635524) bank8217 +8218 POINT(41.68652757589734 74.31040875930196) bank8218 +8219 POINT(40.08032789848968 73.26772796229679) bank8219 +8220 POINT(41.632959953350564 74.82426295187277) bank8220 +8221 POINT(40.92913954475637 74.0778599448671) bank8221 +8222 POINT(41.161938890633515 74.8214646118247) bank8222 +8223 POINT(40.71224296535586 73.54970746942404) bank8223 +8224 POINT(41.64110665163554 74.71902827258884) bank8224 +8225 POINT(39.99956378005339 74.49938228822573) bank8225 +8226 POINT(40.17164768672892 74.86440312340976) bank8226 +8227 POINT(40.04217746471352 73.49769895650658) bank8227 +8228 POINT(41.691738956268786 73.62864988325275) bank8228 +8229 POINT(40.774542044352344 74.94967244549298) bank8229 +8230 POINT(40.986946814959296 74.52247544869682) bank8230 +8231 POINT(39.72926904686884 74.89621057191943) bank8231 +8232 POINT(39.89463488040739 73.1229110001058) bank8232 +8233 POINT(40.82336250446028 73.90813900245976) bank8233 +8234 POINT(40.03524963303569 74.32070439626494) bank8234 +8235 POINT(40.41550904375738 74.75898056203133) bank8235 +8236 POINT(41.462160031867825 73.7142910757159) bank8236 +8237 POINT(40.70785357055345 74.45614111792403) bank8237 +8238 POINT(39.735632020443695 73.64818492588005) bank8238 +8239 POINT(40.61376361629771 74.27422888262046) bank8239 +8240 POINT(41.66496651254588 73.87881209982558) bank8240 +8241 POINT(40.012227175113 74.4819766722233) bank8241 +8242 POINT(40.27751781615821 74.74286305340007) bank8242 +8243 POINT(39.89214394950251 74.62518476627324) bank8243 +8244 POINT(40.57074473377954 73.58729403444173) bank8244 +8245 POINT(41.225988548673755 74.4111401941537) bank8245 +8246 POINT(39.93735529603145 74.7262631231032) bank8246 +8247 POINT(41.54506305137585 74.62343228121271) bank8247 +8248 POINT(41.16644220095964 73.96707464573441) bank8248 +8249 POINT(41.054195221855466 73.96417258011812) bank8249 +8250 POINT(40.09411002227453 74.24415574254553) bank8250 +8251 POINT(41.08800403207671 73.75549298472527) bank8251 +8252 POINT(39.96377299266507 74.92917585667014) bank8252 +8253 POINT(41.293752390868 73.43586643155066) bank8253 +8254 POINT(41.503754319526976 73.80148373658919) bank8254 +8255 POINT(40.352494109455094 74.07453752922825) bank8255 +8256 POINT(40.744448258824356 74.18110057318526) bank8256 +8257 POINT(40.12962177318859 73.42618303001224) bank8257 +8258 POINT(39.75448289716516 74.50625879682535) bank8258 +8259 POINT(40.3716392476391 74.11570653826726) bank8259 +8260 POINT(41.58925269949475 73.91452953274676) bank8260 +8261 POINT(40.14492790301412 73.76590329433132) bank8261 +8262 POINT(39.926949329159164 73.688653281072) bank8262 +8263 POINT(40.901779396633266 74.22547851224772) bank8263 +8264 POINT(39.84214804342885 73.23574279864478) bank8264 +8265 POINT(40.2078811707825 74.24790008193216) bank8265 +8266 POINT(39.887783270915115 74.54409792441561) bank8266 +8267 POINT(40.70420037784762 73.4348689778012) bank8267 +8268 POINT(41.26011061536646 74.03572194091569) bank8268 +8269 POINT(41.23585453979132 73.09645950909318) bank8269 +8270 POINT(41.1829394225632 74.65149454679216) bank8270 +8271 POINT(41.2481695405865 73.17624011048365) bank8271 +8272 POINT(39.866997277926785 74.30573619383566) bank8272 +8273 POINT(40.83530501825642 74.67919722550123) bank8273 +8274 POINT(41.23551732836796 74.80057384343492) bank8274 +8275 POINT(41.26671556702901 73.13413042929281) bank8275 +8276 POINT(40.65588413367205 73.44366619347143) bank8276 +8277 POINT(39.96427879954751 74.15489730090061) bank8277 +8278 POINT(40.69850450548309 74.78898306819583) bank8278 +8279 POINT(41.67491053619611 73.70558757376475) bank8279 +8280 POINT(41.194019073525304 73.29641043414186) bank8280 +8281 POINT(40.519950195254 74.47901235647426) bank8281 +8282 POINT(41.53547360587355 74.70120223389391) bank8282 +8283 POINT(41.54496611295011 74.2181141282796) bank8283 +8284 POINT(40.04682535651659 73.22417194499293) bank8284 +8285 POINT(41.43900218317411 74.31755496570969) bank8285 +8286 POINT(41.18783493556274 73.01318103371871) bank8286 +8287 POINT(40.11293436662735 73.28743285639251) bank8287 +8288 POINT(39.7134663508541 73.70555148574222) bank8288 +8289 POINT(41.329892377232404 74.65176845267871) bank8289 +8290 POINT(40.937526118328364 74.82552332987628) bank8290 +8291 POINT(39.9205759109365 74.91061691339019) bank8291 +8292 POINT(41.36523491062753 74.03622882132795) bank8292 +8293 POINT(40.904357520064835 74.11387450362176) bank8293 +8294 POINT(39.74831716014843 74.81295928678256) bank8294 +8295 POINT(39.78383038713254 73.27732502576991) bank8295 +8296 POINT(41.05689254440431 74.44959282604147) bank8296 +8297 POINT(41.34266239372377 73.87078934704242) bank8297 +8298 POINT(40.78747553684246 74.54239555771386) bank8298 +8299 POINT(41.579642021123135 73.14745516293618) bank8299 +8300 POINT(41.69733620846412 73.20290253392368) bank8300 +8301 POINT(41.531237552399034 74.63161454470445) bank8301 +8302 POINT(39.8770473946589 73.76366124007173) bank8302 +8303 POINT(40.779577038664264 73.078702360461) bank8303 +8304 POINT(40.84520301224936 73.09093669912608) bank8304 +8305 POINT(41.19909807077888 74.1148334762205) bank8305 +8306 POINT(41.68208196522916 73.06753790955804) bank8306 +8307 POINT(40.24387685363876 74.55305960085697) bank8307 +8308 POINT(39.97559628435394 74.25818067690074) bank8308 +8309 POINT(40.527120830031606 74.60973250596427) bank8309 +8310 POINT(40.2054320385097 73.50234186726996) bank8310 +8311 POINT(41.12666889540766 74.58580104481138) bank8311 +8312 POINT(40.08560367251452 74.99286003698697) bank8312 +8313 POINT(39.715060967157704 73.17009266745032) bank8313 +8314 POINT(41.192707179875825 73.04964459408421) bank8314 +8315 POINT(39.97084883822238 74.68634066725728) bank8315 +8316 POINT(40.48249764391817 74.65661452213276) bank8316 +8317 POINT(40.41519599147653 74.74196568832704) bank8317 +8318 POINT(40.09329131062691 73.5931754429806) bank8318 +8319 POINT(40.848933101639034 73.8392435221962) bank8319 +8320 POINT(41.153501189634504 74.53543308073749) bank8320 +8321 POINT(40.45210184975276 74.08608941355034) bank8321 +8322 POINT(39.822622681693126 73.21833452526603) bank8322 +8323 POINT(40.63297894679671 74.52470960310592) bank8323 +8324 POINT(40.11232104067368 74.12053923033244) bank8324 +8325 POINT(40.15062088450364 73.50925440950463) bank8325 +8326 POINT(40.817675716912156 74.86281952217243) bank8326 +8327 POINT(40.28809794058912 73.81303110916156) bank8327 +8328 POINT(39.79942224062793 74.3462515565704) bank8328 +8329 POINT(40.25326289139097 73.32858289998347) bank8329 +8330 POINT(40.434176255718405 73.74796672690044) bank8330 +8331 POINT(41.276740890488334 74.79995670156002) bank8331 +8332 POINT(41.086868238918434 73.26103138245819) bank8332 +8333 POINT(40.88823540498656 74.01559803929129) bank8333 +8334 POINT(40.90044790353735 74.7649374497058) bank8334 +8335 POINT(40.99623777699812 73.89691969693624) bank8335 +8336 POINT(40.74958029013407 74.85720527303312) bank8336 +8337 POINT(39.93033783575769 73.14694163142737) bank8337 +8338 POINT(40.95957268819426 73.30496054075346) bank8338 +8339 POINT(40.81377887079931 74.26704768997831) bank8339 +8340 POINT(41.21767917070935 74.23451662425501) bank8340 +8341 POINT(40.28509843463613 73.67304909703293) bank8341 +8342 POINT(40.910511025261435 74.41584369601546) bank8342 +8343 POINT(40.70683265116094 73.52481294553188) bank8343 +8344 POINT(40.19129625689485 73.3600035254761) bank8344 +8345 POINT(40.69591425415021 73.32873070086904) bank8345 +8346 POINT(40.836213393321 74.45988599739951) bank8346 +8347 POINT(41.11055614687701 73.58610038217523) bank8347 +8348 POINT(41.03683376683227 73.33984118627423) bank8348 +8349 POINT(41.10923511882066 73.48401581628563) bank8349 +8350 POINT(40.81688334650688 73.89090276152592) bank8350 +8351 POINT(40.59911237834653 74.35454743488162) bank8351 +8352 POINT(40.16820445941416 74.8772070182125) bank8352 +8353 POINT(41.02485886591161 74.36776816730952) bank8353 +8354 POINT(41.33219572410244 73.34393236030076) bank8354 +8355 POINT(41.65874985052009 74.03681917701262) bank8355 +8356 POINT(41.04897604829155 74.80994712264457) bank8356 +8357 POINT(41.40836643242994 74.54235652662209) bank8357 +8358 POINT(39.81313328663271 73.88488025057231) bank8358 +8359 POINT(40.128037634630935 73.88993724192713) bank8359 +8360 POINT(40.87376345061909 73.73068314198694) bank8360 +8361 POINT(40.52100048147755 74.57653655329436) bank8361 +8362 POINT(39.88054574177631 73.93841586934481) bank8362 +8363 POINT(40.588901123802515 74.50890783166497) bank8363 +8364 POINT(40.868308404008516 74.7997552536969) bank8364 +8365 POINT(40.00786063522405 73.23984757753692) bank8365 +8366 POINT(40.06804579541977 74.69801927124551) bank8366 +8367 POINT(39.828083724498875 73.59440404029968) bank8367 +8368 POINT(40.758337397371186 73.74725473962937) bank8368 +8369 POINT(40.10088651538843 73.32841715638729) bank8369 +8370 POINT(40.80358368209163 73.14444137411351) bank8370 +8371 POINT(39.737858716846176 74.69240471393476) bank8371 +8372 POINT(39.95176251073424 74.17750043043657) bank8372 +8373 POINT(41.247966114187875 74.14128755401008) bank8373 +8374 POINT(41.01740627462422 73.64970929486508) bank8374 +8375 POINT(40.990094688988655 73.36614468858369) bank8375 +8376 POINT(40.16559092572118 74.28770038260551) bank8376 +8377 POINT(39.99991697557265 73.43292546228088) bank8377 +8378 POINT(40.31106789361972 73.38252770467287) bank8378 +8379 POINT(40.27036310017127 74.05046238112621) bank8379 +8380 POINT(40.39492274828498 73.93598497065838) bank8380 +8381 POINT(41.35893430227726 73.44359576797378) bank8381 +8382 POINT(39.94770163847584 73.0661099171509) bank8382 +8383 POINT(40.37562617978584 73.58530985160952) bank8383 +8384 POINT(40.18721518882906 74.72129621032369) bank8384 +8385 POINT(40.985987247714874 73.91399857226574) bank8385 +8386 POINT(39.802400797395045 73.76356142195567) bank8386 +8387 POINT(41.30697512813086 74.93791515743504) bank8387 +8388 POINT(41.43871345438522 74.55968591800291) bank8388 +8389 POINT(39.75182324336865 74.86473993606653) bank8389 +8390 POINT(41.316992383319004 74.2809363470973) bank8390 +8391 POINT(39.87208667552366 74.99820491754706) bank8391 +8392 POINT(41.02965943004819 74.20470211580533) bank8392 +8393 POINT(40.831994549609135 74.70573298596881) bank8393 +8394 POINT(39.73761715902231 73.3618007082471) bank8394 +8395 POINT(40.87108283452674 73.25885050888877) bank8395 +8396 POINT(40.99697174268648 74.08098701865434) bank8396 +8397 POINT(41.57163252127367 74.36138607122943) bank8397 +8398 POINT(40.75419638298334 73.29531306343029) bank8398 +8399 POINT(40.36407072253286 73.91169483818206) bank8399 +8400 POINT(41.274029973533324 74.22033527602133) bank8400 +8401 POINT(41.13068235249533 74.3226500306304) bank8401 +8402 POINT(40.296987255159415 74.8830926607249) bank8402 +8403 POINT(41.07268097988548 74.63915403467283) bank8403 +8404 POINT(41.323320035026335 74.79819116142569) bank8404 +8405 POINT(40.35848167702508 74.29537606346928) bank8405 +8406 POINT(41.48111346145367 73.9702855988994) bank8406 +8407 POINT(40.74446376478846 74.9784183377601) bank8407 +8408 POINT(40.73835029742284 73.77510973525794) bank8408 +8409 POINT(40.40609296772771 74.20557450586374) bank8409 +8410 POINT(41.71201762340146 73.32985002426634) bank8410 +8411 POINT(40.620052156569784 73.41728484237586) bank8411 +8412 POINT(39.913280726743004 74.46314564028788) bank8412 +8413 POINT(40.07217401786639 73.81389929820969) bank8413 +8414 POINT(40.54381701979477 74.0003317327858) bank8414 +8415 POINT(39.97114993248042 73.706722828046) bank8415 +8416 POINT(40.87796843894652 74.88266457976019) bank8416 +8417 POINT(40.18004501017388 74.88215701502492) bank8417 +8418 POINT(40.35307213462227 73.67881352032472) bank8418 +8419 POINT(39.84154286451317 74.48143506923307) bank8419 +8420 POINT(41.28876657490326 74.76732395045626) bank8420 +8421 POINT(39.96574401721386 74.91625578058242) bank8421 +8422 POINT(41.20351930040344 74.90718243371654) bank8422 +8423 POINT(40.12168787999221 74.42089785123508) bank8423 +8424 POINT(40.00141428241484 73.62561453971504) bank8424 +8425 POINT(41.01909399947609 74.6508442684493) bank8425 +8426 POINT(40.47677079138722 73.96452195806208) bank8426 +8427 POINT(39.89033679007781 74.1523161638006) bank8427 +8428 POINT(41.517244146381216 73.22513904017511) bank8428 +8429 POINT(40.291875163112245 74.7339727075205) bank8429 +8430 POINT(40.26446145861929 75.0041823371171) bank8430 +8431 POINT(40.293611455078 74.380878576422) bank8431 +8432 POINT(41.13669691735012 73.86986757075577) bank8432 +8433 POINT(40.24782381763975 73.39303070235565) bank8433 +8434 POINT(40.935919433534615 74.59835676031196) bank8434 +8435 POINT(40.369124644870126 73.93153432299768) bank8435 +8436 POINT(40.27951546624449 74.29002241421418) bank8436 +8437 POINT(41.65623787324792 74.59764937344595) bank8437 +8438 POINT(40.63750242376939 74.39382261654724) bank8438 +8439 POINT(40.80944123781727 74.73329235402328) bank8439 +8440 POINT(41.145940913552195 74.86162186236174) bank8440 +8441 POINT(40.89717544789273 74.61836004707504) bank8441 +8442 POINT(40.094966917435784 74.46296684014908) bank8442 +8443 POINT(40.18555204644757 73.07377103840966) bank8443 +8444 POINT(41.24835855661195 73.76998703021535) bank8444 +8445 POINT(40.733530025636206 73.0225220542984) bank8445 +8446 POINT(40.9176596376393 73.9166035846658) bank8446 +8447 POINT(40.45665737762169 74.38959364344933) bank8447 +8448 POINT(41.46796419418599 73.27184583774581) bank8448 +8449 POINT(40.702638095573114 73.3639968208996) bank8449 +8450 POINT(40.174104851532306 74.12489293499003) bank8450 +8451 POINT(40.14471984040408 73.56100814633201) bank8451 +8452 POINT(40.70921687592393 73.1131416957654) bank8452 +8453 POINT(41.35503110003986 74.32935292828793) bank8453 +8454 POINT(41.36738056542396 73.81614325331654) bank8454 +8455 POINT(40.50696051213088 74.66414619423344) bank8455 +8456 POINT(40.69734388144478 74.95763130120802) bank8456 +8457 POINT(40.57465476433634 74.55362968129306) bank8457 +8458 POINT(39.93060595812792 73.6678436950231) bank8458 +8459 POINT(40.9571847626614 74.7690187147274) bank8459 +8460 POINT(39.8894711914223 74.77159559744355) bank8460 +8461 POINT(41.42689819774781 73.1422708200601) bank8461 +8462 POINT(40.039583086901175 73.20683565534999) bank8462 +8463 POINT(41.01800635554924 73.27255983453486) bank8463 +8464 POINT(40.633925564064995 74.11411474511544) bank8464 +8465 POINT(40.76535993632446 74.94413027659218) bank8465 +8466 POINT(40.02486849748302 74.01967564777905) bank8466 +8467 POINT(40.19367794341 74.68251150556894) bank8467 +8468 POINT(41.32746075809885 74.53607386264501) bank8468 +8469 POINT(40.76582676764605 74.20100134314781) bank8469 +8470 POINT(40.31405199099383 74.57650552836522) bank8470 +8471 POINT(40.050556508553136 73.17274079577575) bank8471 +8472 POINT(41.69607523261584 73.99269926005529) bank8472 +8473 POINT(40.22082596907155 74.98547793753525) bank8473 +8474 POINT(40.488134665482804 73.14180102856112) bank8474 +8475 POINT(39.947891964264606 74.45293930853518) bank8475 +8476 POINT(40.04124644278337 73.86169166159762) bank8476 +8477 POINT(39.969167663579384 74.34231071005861) bank8477 +8478 POINT(41.063087368000375 73.6969743757084) bank8478 +8479 POINT(41.5140003067095 73.37793224042817) bank8479 +8480 POINT(41.521701970526124 74.9824878715749) bank8480 +8481 POINT(41.366573347870904 74.21117509437562) bank8481 +8482 POINT(40.81482925335616 74.53644463360843) bank8482 +8483 POINT(40.670322053707075 74.76438823735892) bank8483 +8484 POINT(41.41432463616592 74.77833765420507) bank8484 +8485 POINT(40.0457989255117 73.34484525344807) bank8485 +8486 POINT(40.78257011611638 73.6413558186388) bank8486 +8487 POINT(41.21283530144304 74.0019558494878) bank8487 +8488 POINT(41.010712837533816 74.29215333157893) bank8488 +8489 POINT(40.53012284018233 73.81789374247269) bank8489 +8490 POINT(41.15270338646118 74.92434386368339) bank8490 +8491 POINT(41.14674606293199 73.88515860318003) bank8491 +8492 POINT(40.814532310141594 73.17603947369412) bank8492 +8493 POINT(40.198019723015655 73.10326079039376) bank8493 +8494 POINT(40.05633390029029 73.81696359958006) bank8494 +8495 POINT(39.779434377360495 73.08476670228937) bank8495 +8496 POINT(39.876123844431184 73.58627414717236) bank8496 +8497 POINT(40.39409768574842 74.0099300545666) bank8497 +8498 POINT(39.72770288211091 73.60224982734847) bank8498 +8499 POINT(40.300782352352606 74.43519251572621) bank8499 +8500 POINT(39.923758936892206 73.381181901677) bank8500 +8501 POINT(41.65325954148063 74.59578837429339) bank8501 +8502 POINT(40.83551032374254 73.22205176412389) bank8502 +8503 POINT(40.8345742554741 73.83175107716265) bank8503 +8504 POINT(40.148967978003114 73.51181033318028) bank8504 +8505 POINT(40.724572192910465 74.45221714172351) bank8505 +8506 POINT(40.71382399064007 73.7343990987232) bank8506 +8507 POINT(39.88384373846308 73.27891687492965) bank8507 +8508 POINT(41.50227539539238 73.55631276655352) bank8508 +8509 POINT(40.694337621669526 74.40837315779393) bank8509 +8510 POINT(40.63590776751748 74.22684554617415) bank8510 +8511 POINT(40.954341158400766 74.05335227411915) bank8511 +8512 POINT(40.704749667462686 74.61713107939725) bank8512 +8513 POINT(41.17185030208431 74.94389739139791) bank8513 +8514 POINT(39.92486704348325 74.29871830802473) bank8514 +8515 POINT(40.15059008334027 74.06853004745327) bank8515 +8516 POINT(40.84594320919541 74.02188728769966) bank8516 +8517 POINT(41.350474026431826 74.40456835535716) bank8517 +8518 POINT(40.147919509271276 73.3562700320214) bank8518 +8519 POINT(41.036170138439125 73.42462536559285) bank8519 +8520 POINT(39.90680296624252 74.1795982797955) bank8520 +8521 POINT(41.619966573990034 74.52531551369667) bank8521 +8522 POINT(40.83922810960067 73.62200267879707) bank8522 +8523 POINT(40.58181981232504 73.35619042843001) bank8523 +8524 POINT(39.89195550755393 74.41242583361299) bank8524 +8525 POINT(41.08286603609542 74.4566392356375) bank8525 +8526 POINT(40.69881043675302 73.36341741485163) bank8526 +8527 POINT(40.75844690584309 74.21891398425291) bank8527 +8528 POINT(40.10517047518719 74.94086974809764) bank8528 +8529 POINT(40.78517160071385 74.28602865633809) bank8529 +8530 POINT(40.890936478409174 74.55488426669318) bank8530 +8531 POINT(41.49302605787014 73.50716999955648) bank8531 +8532 POINT(40.22086141146098 74.87438882799574) bank8532 +8533 POINT(40.22085944336132 74.94233426352652) bank8533 +8534 POINT(41.16668071785267 73.54193524369951) bank8534 +8535 POINT(39.84217605088896 74.03375652292985) bank8535 +8536 POINT(40.412414242366495 73.97864920922225) bank8536 +8537 POINT(40.6221286765796 73.34937612581669) bank8537 +8538 POINT(41.285804983170166 74.50264952353365) bank8538 +8539 POINT(39.80115212573074 73.52641271577491) bank8539 +8540 POINT(40.583886106035195 74.38270081473327) bank8540 +8541 POINT(41.59402504483068 73.76698552182776) bank8541 +8542 POINT(41.61012055203054 73.8407259892192) bank8542 +8543 POINT(40.2352920094535 74.33438351963947) bank8543 +8544 POINT(40.41399315372599 73.73722987454165) bank8544 +8545 POINT(41.12569772056699 73.15707399735714) bank8545 +8546 POINT(40.83322584251914 73.50670120115147) bank8546 +8547 POINT(40.432862740371874 73.60390063358753) bank8547 +8548 POINT(40.45458753319912 73.10383499995707) bank8548 +8549 POINT(40.50106863905194 74.91389852996721) bank8549 +8550 POINT(41.66238066173173 73.79643722477856) bank8550 +8551 POINT(41.148964669974795 74.06398336855408) bank8551 +8552 POINT(40.1098206927846 74.10407882763671) bank8552 +8553 POINT(39.86290491332139 73.44546060538346) bank8553 +8554 POINT(39.892373517944826 73.29263724763689) bank8554 +8555 POINT(40.029691004116046 73.2192732295883) bank8555 +8556 POINT(40.466721236645576 73.73665709471987) bank8556 +8557 POINT(41.272680732245625 74.74131154270974) bank8557 +8558 POINT(39.9471215643267 74.64349486853428) bank8558 +8559 POINT(41.19660644803871 74.72672566974396) bank8559 +8560 POINT(39.910477343511864 73.65714722113105) bank8560 +8561 POINT(40.865259880915836 74.25649916601388) bank8561 +8562 POINT(40.10551776750956 73.25970694753563) bank8562 +8563 POINT(41.35359829542735 74.02088072741817) bank8563 +8564 POINT(40.20716692373961 73.67027896586777) bank8564 +8565 POINT(41.301928047886186 74.73612687542729) bank8565 +8566 POINT(40.84846564371036 74.87044637662147) bank8566 +8567 POINT(40.48513004333717 73.58863861443113) bank8567 +8568 POINT(39.94361825287458 73.18632210384946) bank8568 +8569 POINT(40.56990691712154 73.4611202228753) bank8569 +8570 POINT(41.25633526416981 74.13214621159787) bank8570 +8571 POINT(41.48304393813283 73.40776911733927) bank8571 +8572 POINT(39.95487277286495 74.4175409637277) bank8572 +8573 POINT(39.86183430076339 73.7751247720845) bank8573 +8574 POINT(41.646029973989755 74.51178586089091) bank8574 +8575 POINT(39.92610936211837 74.31170011762103) bank8575 +8576 POINT(41.516528339313744 74.9095560037698) bank8576 +8577 POINT(39.74749204981743 74.59443692251226) bank8577 +8578 POINT(40.23465902169954 73.08400212362045) bank8578 +8579 POINT(40.60489783914082 73.63043558696333) bank8579 +8580 POINT(41.41161376681903 74.87906454468522) bank8580 +8581 POINT(40.228304534215695 74.62504369679384) bank8581 +8582 POINT(40.327399360023676 73.32823049822275) bank8582 +8583 POINT(39.77989882972808 74.62022311576588) bank8583 +8584 POINT(41.61059375440101 74.40283419250459) bank8584 +8585 POINT(40.725163951817834 74.41720750446501) bank8585 +8586 POINT(41.545714759494174 73.82724599843436) bank8586 +8587 POINT(39.770425690887826 73.83581343410489) bank8587 +8588 POINT(39.9783699331566 73.05261999272165) bank8588 +8589 POINT(40.712514621430174 73.3841881143285) bank8589 +8590 POINT(41.641628238301344 74.11314933020411) bank8590 +8591 POINT(40.126619226059326 73.37073727640424) bank8591 +8592 POINT(41.07433836806914 73.89250374548189) bank8592 +8593 POINT(39.725474494527276 73.89550310868992) bank8593 +8594 POINT(39.71400243417206 73.21906463073786) bank8594 +8595 POINT(39.96863459973506 74.63084128274272) bank8595 +8596 POINT(41.267527873039135 73.44184800509423) bank8596 +8597 POINT(39.819007506850085 73.07616008935685) bank8597 +8598 POINT(41.52772444329446 73.73372397043647) bank8598 +8599 POINT(41.219437982244735 74.98205969189792) bank8599 +8600 POINT(41.5712071492137 73.02109736744441) bank8600 +8601 POINT(40.48260042168858 74.6151276248385) bank8601 +8602 POINT(40.6660324174858 74.77281208862657) bank8602 +8603 POINT(40.516074302762675 74.58136855483919) bank8603 +8604 POINT(40.08741264532215 73.06397655245978) bank8604 +8605 POINT(40.988199802761926 74.71026901535451) bank8605 +8606 POINT(41.38836632512319 74.36924060731569) bank8606 +8607 POINT(40.91714162380267 74.51555388113854) bank8607 +8608 POINT(41.448606140215205 73.70852161340184) bank8608 +8609 POINT(39.778483053747806 73.67631493672485) bank8609 +8610 POINT(41.32441472075602 74.23126552393623) bank8610 +8611 POINT(40.067008866333225 73.20725617536077) bank8611 +8612 POINT(39.86724726420187 74.7681621660185) bank8612 +8613 POINT(41.37221648335071 73.51723529470682) bank8613 +8614 POINT(40.013259452980655 73.1215633639373) bank8614 +8615 POINT(39.907122083265186 74.07110425150151) bank8615 +8616 POINT(39.77198978644591 74.31252803128942) bank8616 +8617 POINT(40.118237940420705 73.25012900951111) bank8617 +8618 POINT(39.83046018786996 73.25258848490694) bank8618 +8619 POINT(40.265705281512794 74.3537199320405) bank8619 +8620 POINT(39.83841521265651 74.45750062481484) bank8620 +8621 POINT(39.90665411524622 74.83330633669897) bank8621 +8622 POINT(39.83288412598251 74.68098197506492) bank8622 +8623 POINT(39.7366329413705 74.37824693668477) bank8623 +8624 POINT(40.12720077060201 74.5835780854519) bank8624 +8625 POINT(41.11124713600116 73.93741456733663) bank8625 +8626 POINT(40.26485329244698 73.37484724090633) bank8626 +8627 POINT(39.989812688481194 74.28172520083372) bank8627 +8628 POINT(41.12450646706976 73.74535262839866) bank8628 +8629 POINT(39.93839461690249 73.00688823003499) bank8629 +8630 POINT(40.42071182479662 74.2015181676087) bank8630 +8631 POINT(39.877787580449194 73.44538696498137) bank8631 +8632 POINT(41.224933338902126 73.23684382814764) bank8632 +8633 POINT(40.4476179677891 74.23923165534094) bank8633 +8634 POINT(41.517054512933 74.25261576858796) bank8634 +8635 POINT(40.84128370934887 74.58615283569388) bank8635 +8636 POINT(41.63787391494488 73.65122382984703) bank8636 +8637 POINT(41.655107715130114 73.86243708506488) bank8637 +8638 POINT(40.998889708230934 73.71360863277732) bank8638 +8639 POINT(40.66372920217762 74.85138964290303) bank8639 +8640 POINT(40.3660974643493 74.54283248055694) bank8640 +8641 POINT(40.13896245930779 74.90121115550811) bank8641 +8642 POINT(39.818941473524994 73.37824683618376) bank8642 +8643 POINT(40.47954107852414 73.00974656441593) bank8643 +8644 POINT(40.58227094507938 74.82768029691461) bank8644 +8645 POINT(41.56295411477974 74.28849757446784) bank8645 +8646 POINT(41.08741725011733 74.75444374568373) bank8646 +8647 POINT(41.5340864662264 73.10357813257357) bank8647 +8648 POINT(39.93138513724668 74.77393246642244) bank8648 +8649 POINT(41.55539087690657 74.99743399142109) bank8649 +8650 POINT(40.790008464053 73.37001716110497) bank8650 +8651 POINT(41.51251654605763 73.89087791675132) bank8651 +8652 POINT(41.17140016330947 74.21028335080345) bank8652 +8653 POINT(41.051024755955275 74.87383575557249) bank8653 +8654 POINT(39.86656774232587 73.12722563759122) bank8654 +8655 POINT(41.379146225664954 73.9368799148829) bank8655 +8656 POINT(39.92897522058133 73.69394718974067) bank8656 +8657 POINT(40.99150101810485 73.35316545884677) bank8657 +8658 POINT(41.04781936360256 74.80983838968673) bank8658 +8659 POINT(41.41534968615538 73.54833797291087) bank8659 +8660 POINT(39.76481615592788 74.95716249421145) bank8660 +8661 POINT(40.533050660238175 73.57027021792088) bank8661 +8662 POINT(41.159010585811295 73.92978378013478) bank8662 +8663 POINT(40.17431693614868 74.18964613155289) bank8663 +8664 POINT(39.932755164173145 74.56586074748944) bank8664 +8665 POINT(41.146071979581926 73.85667816800967) bank8665 +8666 POINT(39.77071248050133 73.79248743077035) bank8666 +8667 POINT(41.68203674869722 74.12309766979998) bank8667 +8668 POINT(40.69614219507798 73.76154334096486) bank8668 +8669 POINT(40.402346292603525 73.0965308883145) bank8669 +8670 POINT(41.067882155460644 73.6836328819981) bank8670 +8671 POINT(40.43160109080901 73.66953486889511) bank8671 +8672 POINT(40.99001306403832 73.02497927642051) bank8672 +8673 POINT(40.1805743287562 73.94979779216321) bank8673 +8674 POINT(40.139957572577174 74.9816346169272) bank8674 +8675 POINT(41.317605989898 74.70598979167505) bank8675 +8676 POINT(41.28404910917123 74.28393332384243) bank8676 +8677 POINT(40.193255525329654 74.23242143888763) bank8677 +8678 POINT(40.381840549301565 73.2579016291488) bank8678 +8679 POINT(39.792546070541675 73.84824259880614) bank8679 +8680 POINT(40.368014312336186 73.35257340407932) bank8680 +8681 POINT(41.11768916507389 73.0616339340566) bank8681 +8682 POINT(40.67710511849445 74.76908015566917) bank8682 +8683 POINT(41.14608913956015 73.5440292294475) bank8683 +8684 POINT(41.52903462243493 73.21429479554406) bank8684 +8685 POINT(40.986384898105364 74.81824256858008) bank8685 +8686 POINT(40.31239318490944 74.53449256789663) bank8686 +8687 POINT(39.995381824557555 73.4957609011826) bank8687 +8688 POINT(41.35668746054451 74.05524149975112) bank8688 +8689 POINT(41.687476132836856 74.76877185032234) bank8689 +8690 POINT(41.06281898477385 73.2015894020806) bank8690 +8691 POINT(40.85653214763874 73.11020121480453) bank8691 +8692 POINT(41.2814074375492 73.19971565221913) bank8692 +8693 POINT(40.10149625021875 74.20515409022079) bank8693 +8694 POINT(39.993641291569425 73.61171318501621) bank8694 +8695 POINT(40.87882959836125 74.95089692416211) bank8695 +8696 POINT(41.13106757186532 74.59309380432693) bank8696 +8697 POINT(40.994952307679185 74.83583337596608) bank8697 +8698 POINT(40.73210526222513 73.19343675583059) bank8698 +8699 POINT(40.749713483387666 74.89611654149776) bank8699 +8700 POINT(40.45850078039958 74.34055816678723) bank8700 +8701 POINT(41.55471566183376 74.78688608857343) bank8701 +8702 POINT(40.85759333350287 73.44908035666076) bank8702 +8703 POINT(41.69473004932786 73.21833452337196) bank8703 +8704 POINT(39.76972282217805 74.67940135830818) bank8704 +8705 POINT(41.4810998189785 74.90422769907245) bank8705 +8706 POINT(40.73339786244119 74.799468210174) bank8706 +8707 POINT(39.851957707097135 73.15596702078926) bank8707 +8708 POINT(39.99184974058111 74.52167973494336) bank8708 +8709 POINT(40.757456772601444 73.79355569970124) bank8709 +8710 POINT(40.61795162177908 74.05778803683123) bank8710 +8711 POINT(40.43764494009668 74.54094654140043) bank8711 +8712 POINT(41.358001064652406 74.7382485187402) bank8712 +8713 POINT(41.50107308770322 74.02229314708192) bank8713 +8714 POINT(41.616638004335435 73.52914627873349) bank8714 +8715 POINT(41.63775681557322 74.0854453342362) bank8715 +8716 POINT(40.26245159420678 74.53976440416525) bank8716 +8717 POINT(41.592025840596285 73.66607547264461) bank8717 +8718 POINT(41.55268510337088 73.89582739603235) bank8718 +8719 POINT(41.204979576819056 74.7336962193639) bank8719 +8720 POINT(40.37193477670549 74.1967568060776) bank8720 +8721 POINT(40.19446728074297 73.43693331445613) bank8721 +8722 POINT(40.200075714692 73.95508070953608) bank8722 +8723 POINT(40.186471787599025 73.56718848141327) bank8723 +8724 POINT(40.19438590527428 74.25773792732085) bank8724 +8725 POINT(41.41262835671891 73.97186142209212) bank8725 +8726 POINT(39.998504215243834 73.6641739128917) bank8726 +8727 POINT(41.554902098984094 73.2763906714145) bank8727 +8728 POINT(39.98355068211795 73.53317382483631) bank8728 +8729 POINT(40.73328357940219 74.84518553825) bank8729 +8730 POINT(41.07437765575283 73.47938226557785) bank8730 +8731 POINT(41.172765629348824 74.65498087194588) bank8731 +8732 POINT(41.530602896346394 73.55840205800929) bank8732 +8733 POINT(41.18714423438096 73.16056714374145) bank8733 +8734 POINT(41.38703896846342 74.37457670626279) bank8734 +8735 POINT(40.512072887575314 73.15117105593755) bank8735 +8736 POINT(41.62710328007314 73.04977129648685) bank8736 +8737 POINT(41.468363783573906 73.13941352265708) bank8737 +8738 POINT(41.05132116579967 74.71784975231489) bank8738 +8739 POINT(41.4570320429856 74.88654847203539) bank8739 +8740 POINT(40.345979735802686 74.91103298757312) bank8740 +8741 POINT(41.18991034318994 73.97711818571513) bank8741 +8742 POINT(41.49441596125147 74.39356250407675) bank8742 +8743 POINT(39.941988803581495 73.58028935685705) bank8743 +8744 POINT(40.03309275487458 74.76671317542547) bank8744 +8745 POINT(41.13656590645588 73.36499781827217) bank8745 +8746 POINT(40.82633238346506 73.28983770449531) bank8746 +8747 POINT(40.83297570523388 73.23594981746834) bank8747 +8748 POINT(39.88066274685637 74.48411993230883) bank8748 +8749 POINT(41.5462700416231 74.56854389182884) bank8749 +8750 POINT(40.86824457203562 73.76031937719945) bank8750 +8751 POINT(40.65798483875778 73.41233355114932) bank8751 +8752 POINT(40.2053443395693 74.8554971386419) bank8752 +8753 POINT(39.94878447151196 74.18870174254037) bank8753 +8754 POINT(41.257752982001605 74.2663323471342) bank8754 +8755 POINT(39.90536822521408 74.91911486013036) bank8755 +8756 POINT(41.28269164097707 74.76625082299387) bank8756 +8757 POINT(40.71362226805727 74.2369150114205) bank8757 +8758 POINT(41.28014991941909 74.93975803044985) bank8758 +8759 POINT(41.61873758953752 73.48078081241428) bank8759 +8760 POINT(39.891670842617394 73.50534663088877) bank8760 +8761 POINT(40.88257316994722 73.26628709683423) bank8761 +8762 POINT(40.626031612696465 74.692058323501) bank8762 +8763 POINT(41.46934244440021 73.7718985598133) bank8763 +8764 POINT(40.682479350179406 73.52132885364381) bank8764 +8765 POINT(39.80597216754455 74.05818135438653) bank8765 +8766 POINT(40.91478371902342 73.48505780748161) bank8766 +8767 POINT(40.0659506860832 73.10786207230885) bank8767 +8768 POINT(41.30531812063461 74.76043547421506) bank8768 +8769 POINT(40.75786098672763 73.24365416648556) bank8769 +8770 POINT(39.957992638908166 74.4441267344358) bank8770 +8771 POINT(39.766595431671895 74.11543311757298) bank8771 +8772 POINT(40.098445835256484 73.0242763567686) bank8772 +8773 POINT(40.59590531671987 74.31057644597689) bank8773 +8774 POINT(41.24527860575241 73.24368383792905) bank8774 +8775 POINT(40.41041572335697 73.78039384314067) bank8775 +8776 POINT(41.051838688914295 73.02796466190425) bank8776 +8777 POINT(40.33432085270832 73.27177047471022) bank8777 +8778 POINT(41.410714212625365 74.92308067948639) bank8778 +8779 POINT(39.91708230209485 73.70515408647287) bank8779 +8780 POINT(41.47581960959659 74.71416823477283) bank8780 +8781 POINT(41.45180994423265 74.29920289063338) bank8781 +8782 POINT(41.442761073751605 73.2883552610868) bank8782 +8783 POINT(40.37802612003146 73.1148816812773) bank8783 +8784 POINT(40.10480381823896 73.41516599601917) bank8784 +8785 POINT(40.49584567344759 73.28532245704814) bank8785 +8786 POINT(41.156019188576565 73.20063403327228) bank8786 +8787 POINT(41.27054654326627 74.54484178249784) bank8787 +8788 POINT(40.00897115967796 74.36522038093882) bank8788 +8789 POINT(39.821625492198336 74.44976986416417) bank8789 +8790 POINT(40.50137757200478 74.93641391480892) bank8790 +8791 POINT(40.96460638500281 74.7832203447168) bank8791 +8792 POINT(40.94616161502476 73.90488678528247) bank8792 +8793 POINT(39.74301508547337 73.64616063760408) bank8793 +8794 POINT(40.493522028014524 73.6999629328266) bank8794 +8795 POINT(40.32891639666914 74.3877298516206) bank8795 +8796 POINT(41.488128450153354 74.27459150302172) bank8796 +8797 POINT(41.071162728249575 74.66143966087219) bank8797 +8798 POINT(40.278305473069125 73.35721618630447) bank8798 +8799 POINT(39.74617539325252 73.69745068952939) bank8799 +8800 POINT(40.7230800814974 74.03818182221887) bank8800 +8801 POINT(41.513215853920926 73.39354745860918) bank8801 +8802 POINT(39.868605088706104 73.28443889784705) bank8802 +8803 POINT(41.17865264643587 74.37242021421545) bank8803 +8804 POINT(41.49892550109868 73.68845141638099) bank8804 +8805 POINT(40.68245518022912 73.09263918315538) bank8805 +8806 POINT(40.888656575784644 73.49771678143453) bank8806 +8807 POINT(41.49114935541428 74.48473702262366) bank8807 +8808 POINT(40.0027422066211 73.21884412478343) bank8808 +8809 POINT(39.94807300939556 74.4420067943475) bank8809 +8810 POINT(41.13816196808683 74.98749252473525) bank8810 +8811 POINT(40.57425731733234 74.25794993829032) bank8811 +8812 POINT(40.103486109927 73.45616543395407) bank8812 +8813 POINT(40.626211430751404 73.38629943376792) bank8813 +8814 POINT(39.98177047968019 74.4525146888606) bank8814 +8815 POINT(39.79548359059509 74.05928573989956) bank8815 +8816 POINT(41.27610669872956 74.3066701397659) bank8816 +8817 POINT(40.87115764768988 74.6681618143556) bank8817 +8818 POINT(39.72738487530409 74.93372007390872) bank8818 +8819 POINT(39.78671160441718 73.06433881176291) bank8819 +8820 POINT(39.86799993387411 73.64037394844084) bank8820 +8821 POINT(40.974742007151 74.57594143418642) bank8821 +8822 POINT(41.24797759533036 73.93448316673135) bank8822 +8823 POINT(41.63304946394874 73.0561418466887) bank8823 +8824 POINT(40.80351563824731 73.90361099201122) bank8824 +8825 POINT(40.65900691954837 74.1127884612071) bank8825 +8826 POINT(41.340577019969835 73.83615363977736) bank8826 +8827 POINT(41.093408449542736 74.95464246860296) bank8827 +8828 POINT(41.42507255663267 74.24817855493868) bank8828 +8829 POINT(41.11753449160969 73.97922302020484) bank8829 +8830 POINT(40.57022156469151 73.45159512606175) bank8830 +8831 POINT(40.32855290376422 74.97015736659014) bank8831 +8832 POINT(40.06496725225439 73.22796993128371) bank8832 +8833 POINT(40.2535603498092 73.81350273278024) bank8833 +8834 POINT(39.9752313480096 73.66355486296938) bank8834 +8835 POINT(41.03533683236763 73.74414234483595) bank8835 +8836 POINT(39.945283155279846 73.58188544363674) bank8836 +8837 POINT(40.54759146751142 73.02181227487753) bank8837 +8838 POINT(39.7938311161911 73.47064963205338) bank8838 +8839 POINT(40.101072759117315 73.30412570916263) bank8839 +8840 POINT(39.78308619374657 74.40306601813155) bank8840 +8841 POINT(41.25095420637195 74.85751906268928) bank8841 +8842 POINT(40.468485303225826 74.01993082352891) bank8842 +8843 POINT(41.28651947275858 73.71537719482569) bank8843 +8844 POINT(40.58686993806895 73.61154660604986) bank8844 +8845 POINT(40.56607236130629 74.35923169075359) bank8845 +8846 POINT(40.64440169783573 73.28740749557501) bank8846 +8847 POINT(41.02913231704426 74.77480814809756) bank8847 +8848 POINT(39.80916188450254 74.3364020091785) bank8848 +8849 POINT(41.689932570230425 74.86021464869206) bank8849 +8850 POINT(41.00597871216256 73.43173196222847) bank8850 +8851 POINT(40.30401270302407 74.1746096531793) bank8851 +8852 POINT(40.5507737272324 74.5500555620028) bank8852 +8853 POINT(40.700745723405824 73.040851769737) bank8853 +8854 POINT(41.62330310431513 74.61900335514447) bank8854 +8855 POINT(40.31311380170951 75.00091998627326) bank8855 +8856 POINT(40.022246323614205 74.79292674415034) bank8856 +8857 POINT(41.43164862183927 74.92626306142299) bank8857 +8858 POINT(40.773307706116135 73.42376490411168) bank8858 +8859 POINT(41.69778672621924 73.5631580155626) bank8859 +8860 POINT(41.4328561241965 74.31572331290785) bank8860 +8861 POINT(40.17944856862 74.89664077023536) bank8861 +8862 POINT(41.55211138301793 74.55230795169852) bank8862 +8863 POINT(40.80362068172311 74.66766482629043) bank8863 +8864 POINT(41.1713423440754 74.68124084331535) bank8864 +8865 POINT(40.18006801745339 74.51641478913075) bank8865 +8866 POINT(40.37083735135581 74.67585213593303) bank8866 +8867 POINT(41.41556415059369 74.87817267816527) bank8867 +8868 POINT(39.84732618921933 74.694773877225) bank8868 +8869 POINT(41.54063661239247 74.52659823777152) bank8869 +8870 POINT(40.0837825078383 74.59264402481291) bank8870 +8871 POINT(41.27824934665357 73.54536086833848) bank8871 +8872 POINT(40.770493983659584 74.24942761186709) bank8872 +8873 POINT(40.487216713472804 74.21899693325217) bank8873 +8874 POINT(39.98175939533028 74.72378108875625) bank8874 +8875 POINT(40.501822784630406 74.37227709463454) bank8875 +8876 POINT(40.58237433636034 74.70470906400561) bank8876 +8877 POINT(39.92521263955292 74.62079398634162) bank8877 +8878 POINT(40.088932947 74.89903060686336) bank8878 +8879 POINT(41.28265034092409 74.28378764917164) bank8879 +8880 POINT(39.94188506391008 74.47584140969703) bank8880 +8881 POINT(39.8928475996969 73.57131280682768) bank8881 +8882 POINT(41.46451344501905 73.77221025904527) bank8882 +8883 POINT(41.30448401271083 73.23469511301646) bank8883 +8884 POINT(41.150081908921464 73.05598538098494) bank8884 +8885 POINT(41.422388447428666 73.29810850382982) bank8885 +8886 POINT(40.204013144949094 74.12196153821318) bank8886 +8887 POINT(41.36264904107707 74.59324678823121) bank8887 +8888 POINT(40.43025666310322 74.18850683646816) bank8888 +8889 POINT(41.082132383028124 73.80658701065477) bank8889 +8890 POINT(41.70069168473603 73.9019790679133) bank8890 +8891 POINT(40.57504802298216 74.02989162653884) bank8891 +8892 POINT(41.424729919401024 73.01273544782714) bank8892 +8893 POINT(41.067022278457685 73.17233920139714) bank8893 +8894 POINT(41.0337684821181 73.30808345888268) bank8894 +8895 POINT(39.916761892052754 73.98056382584667) bank8895 +8896 POINT(41.69077635524313 73.054086987764) bank8896 +8897 POINT(40.53473284105942 73.85406896718324) bank8897 +8898 POINT(40.835490010474786 73.46129550378357) bank8898 +8899 POINT(40.38465473394427 74.89631926416531) bank8899 +8900 POINT(39.82332543913371 74.91782088211399) bank8900 +8901 POINT(40.16186605506309 73.13662585079243) bank8901 +8902 POINT(41.33975762855443 73.21961474555914) bank8902 +8903 POINT(40.29377021381424 74.86136480333089) bank8903 +8904 POINT(41.32468311425653 73.27923498623362) bank8904 +8905 POINT(41.57169414116243 74.14472627173491) bank8905 +8906 POINT(41.242254934008145 74.238978723975) bank8906 +8907 POINT(40.232145259292665 74.02655580622961) bank8907 +8908 POINT(39.722379370398514 74.55222244135044) bank8908 +8909 POINT(40.164116772790095 73.33001540556273) bank8909 +8910 POINT(41.30449743551856 73.69520358519948) bank8910 +8911 POINT(41.334177643553794 73.09643013978899) bank8911 +8912 POINT(40.93806529827276 73.60772345476285) bank8912 +8913 POINT(40.19620135093453 74.57985682426244) bank8913 +8914 POINT(39.99409840114861 73.20673166481528) bank8914 +8915 POINT(41.538396099262826 74.07795514731681) bank8915 +8916 POINT(39.84772012260696 73.0474338272305) bank8916 +8917 POINT(40.940411305166045 74.69476482710448) bank8917 +8918 POINT(41.10248746963058 73.76978714437912) bank8918 +8919 POINT(41.219776356993165 73.70109627009458) bank8919 +8920 POINT(41.59566312602368 73.94541834815337) bank8920 +8921 POINT(41.00247687744006 74.0328999615382) bank8921 +8922 POINT(41.437193543571986 73.41793462915594) bank8922 +8923 POINT(41.56207921306557 73.64032575672138) bank8923 +8924 POINT(39.91265999640796 74.10463858317017) bank8924 +8925 POINT(41.466345651219555 74.21633410075071) bank8925 +8926 POINT(41.52789561818028 74.96657595713509) bank8926 +8927 POINT(39.99242987667135 73.76699200823903) bank8927 +8928 POINT(41.67234752360562 74.83442565680205) bank8928 +8929 POINT(39.938855067133545 73.30722575818234) bank8929 +8930 POINT(41.328426129809955 73.89859682707757) bank8930 +8931 POINT(39.997568364146176 73.62625959597567) bank8931 +8932 POINT(41.28763373536592 74.17052077487618) bank8932 +8933 POINT(41.14359669771621 74.97104377026392) bank8933 +8934 POINT(40.55372972187195 73.9409152216575) bank8934 +8935 POINT(40.94244274635698 74.00655680789835) bank8935 +8936 POINT(39.73616235431987 74.95052604290045) bank8936 +8937 POINT(40.89994509129123 74.3777934833052) bank8937 +8938 POINT(40.54779670364835 74.25227365084002) bank8938 +8939 POINT(40.6606890825365 73.84556718843795) bank8939 +8940 POINT(40.20209225178152 73.07875274265281) bank8940 +8941 POINT(41.10007636412357 73.87068443523829) bank8941 +8942 POINT(39.94284630162136 73.99791721252103) bank8942 +8943 POINT(40.05191131981611 74.22135467163409) bank8943 +8944 POINT(40.99765053650364 73.58372083877813) bank8944 +8945 POINT(41.57530074753078 73.75943935810864) bank8945 +8946 POINT(39.75374824484779 73.2928714994679) bank8946 +8947 POINT(40.68476052212698 73.73034189348235) bank8947 +8948 POINT(40.2897615516582 73.87979204085153) bank8948 +8949 POINT(40.20130270044777 74.80627149585254) bank8949 +8950 POINT(40.752153454183414 74.7003986578173) bank8950 +8951 POINT(40.224186869324555 74.51402976028656) bank8951 +8952 POINT(41.2758846614834 73.07401241228585) bank8952 +8953 POINT(40.633466992298736 74.70506018367072) bank8953 +8954 POINT(40.780165134322075 74.79504235489213) bank8954 +8955 POINT(41.2886969401198 74.6779310525529) bank8955 +8956 POINT(40.77826257760284 74.37470059087705) bank8956 +8957 POINT(40.0197137834703 74.07027719605954) bank8957 +8958 POINT(41.295269998351436 74.76841511081524) bank8958 +8959 POINT(40.403139360649206 73.7247760966248) bank8959 +8960 POINT(40.86198939427594 73.91024026954729) bank8960 +8961 POINT(40.386274343825804 73.51410238379472) bank8961 +8962 POINT(40.952731852088085 74.49864394203763) bank8962 +8963 POINT(40.335158270319546 74.5242723306902) bank8963 +8964 POINT(41.421086366654876 74.56207925702374) bank8964 +8965 POINT(41.64673991185457 73.93822290271049) bank8965 +8966 POINT(41.052538980404755 73.9068755440155) bank8966 +8967 POINT(40.02723192117254 73.14179885940196) bank8967 +8968 POINT(40.183111851082714 74.24632981959125) bank8968 +8969 POINT(39.79616643509586 73.69297539821085) bank8969 +8970 POINT(39.88789340334877 74.42787340420493) bank8970 +8971 POINT(41.30223156798509 74.76277185129629) bank8971 +8972 POINT(39.73260343986722 73.54887861846115) bank8972 +8973 POINT(41.588610941750694 73.92927529575222) bank8973 +8974 POINT(41.34838108538472 74.06516318115081) bank8974 +8975 POINT(40.20557717868518 74.00667262081251) bank8975 +8976 POINT(41.612565521287564 74.28466459517921) bank8976 +8977 POINT(41.2559099612862 74.63437192446688) bank8977 +8978 POINT(40.11496969640738 74.38109074266706) bank8978 +8979 POINT(40.71206652979941 74.51778373404106) bank8979 +8980 POINT(41.165736956288804 73.26985041855255) bank8980 +8981 POINT(41.65443247275292 73.11823357389964) bank8981 +8982 POINT(40.95544066111543 73.07719233613085) bank8982 +8983 POINT(41.14361225220252 73.90593277227124) bank8983 +8984 POINT(40.162803196597125 74.97924062765604) bank8984 +8985 POINT(40.9779867453678 74.33239373771998) bank8985 +8986 POINT(41.658862177021334 74.60357935059893) bank8986 +8987 POINT(40.900655943176496 73.16445136596853) bank8987 +8988 POINT(41.30618651453531 74.91448379393255) bank8988 +8989 POINT(40.622887969282324 74.18743568844886) bank8989 +8990 POINT(41.56198218708686 74.74889338386683) bank8990 +8991 POINT(40.930732183563215 73.17986895296512) bank8991 +8992 POINT(40.319536521568104 74.40635521779679) bank8992 +8993 POINT(40.10645328430021 74.79296645518124) bank8993 +8994 POINT(40.931288343412085 75.00348736769197) bank8994 +8995 POINT(41.56783413642632 73.92735113777394) bank8995 +8996 POINT(41.35793641373973 73.00980213965958) bank8996 +8997 POINT(40.2084055237559 74.47492069170875) bank8997 +8998 POINT(40.529941638711335 74.64825683524947) bank8998 +8999 POINT(40.9241326131534 73.78122787906207) bank8999 +9000 POINT(41.13605464322767 74.7404910847605) bank9000 +9001 POINT(41.3172040277256 73.97988537140677) bank9001 +9002 POINT(40.243654121240915 74.10965888154831) bank9002 +9003 POINT(40.59726595801633 73.78450547888342) bank9003 +9004 POINT(40.34595696873917 73.67294138006778) bank9004 +9005 POINT(40.015747395345706 73.58002091059409) bank9005 +9006 POINT(39.86202198549944 73.79632447373733) bank9006 +9007 POINT(41.1071228427038 73.48674256516856) bank9007 +9008 POINT(41.63181114441916 73.24934232525155) bank9008 +9009 POINT(41.57748815058413 74.92743013671354) bank9009 +9010 POINT(39.92454706989631 73.93300697147988) bank9010 +9011 POINT(41.16849568482036 74.65964858490669) bank9011 +9012 POINT(41.234683997521124 73.33874549111928) bank9012 +9013 POINT(41.04934771871869 74.67677621009018) bank9013 +9014 POINT(40.63707283866458 74.93130469986734) bank9014 +9015 POINT(41.452267128324316 73.42307826932775) bank9015 +9016 POINT(41.072986549669274 74.71110726589887) bank9016 +9017 POINT(40.40663214618839 74.89680301461877) bank9017 +9018 POINT(40.96489516868241 74.62882569609198) bank9018 +9019 POINT(40.81214320328259 73.60894105414046) bank9019 +9020 POINT(40.42173843134373 74.29067867780896) bank9020 +9021 POINT(41.43493700145365 73.62393334528875) bank9021 +9022 POINT(41.42919679112157 73.89818226115229) bank9022 +9023 POINT(41.04340971927243 74.53168864730183) bank9023 +9024 POINT(40.62394364224109 74.56069745728321) bank9024 +9025 POINT(40.189287303699125 74.06419035697444) bank9025 +9026 POINT(40.461056205926155 73.96451008204345) bank9026 +9027 POINT(39.970088881265745 74.11809930363802) bank9027 +9028 POINT(41.059987965391436 74.60837064843658) bank9028 +9029 POINT(40.52463540562147 73.25213328548597) bank9029 +9030 POINT(40.949148752789554 74.23508407432995) bank9030 +9031 POINT(40.70810407294329 73.78857035738221) bank9031 +9032 POINT(41.451007379061416 73.41978700746404) bank9032 +9033 POINT(40.0823550558833 74.2208481289761) bank9033 +9034 POINT(40.39790437350541 74.02577951485416) bank9034 +9035 POINT(39.843233783350975 74.32755150726815) bank9035 +9036 POINT(41.48649304062378 73.97361107506413) bank9036 +9037 POINT(41.474942923719304 73.66995513565115) bank9037 +9038 POINT(39.99192476786093 74.62079043023147) bank9038 +9039 POINT(41.414535502935195 73.47894974385066) bank9039 +9040 POINT(40.00816696953864 73.84745199322188) bank9040 +9041 POINT(40.885502491948536 73.88442712247286) bank9041 +9042 POINT(40.52818442790055 74.82803557254195) bank9042 +9043 POINT(40.363655615617674 74.68683871416512) bank9043 +9044 POINT(39.81014381611892 74.11881382640739) bank9044 +9045 POINT(41.4581680338725 73.158491564789) bank9045 +9046 POINT(40.34350808345479 73.72704904244029) bank9046 +9047 POINT(39.91932852600574 73.17130457498263) bank9047 +9048 POINT(41.54285285519867 73.15775801727466) bank9048 +9049 POINT(40.793063263659185 74.87388371007955) bank9049 +9050 POINT(40.65261720497004 74.94023125413283) bank9050 +9051 POINT(40.10613259919071 73.18821074452649) bank9051 +9052 POINT(41.5216683161817 73.90413393167053) bank9052 +9053 POINT(41.700576932656745 73.46977009416815) bank9053 +9054 POINT(40.2579521786303 73.05658753379974) bank9054 +9055 POINT(40.106115337082 73.40527602135646) bank9055 +9056 POINT(39.998818252936175 74.35362308463905) bank9056 +9057 POINT(41.03349541296062 74.97504439051438) bank9057 +9058 POINT(40.928720388949245 74.7919887594659) bank9058 +9059 POINT(41.18961214752415 73.22883090306816) bank9059 +9060 POINT(40.33475326061645 74.59680837397119) bank9060 +9061 POINT(41.297693366862916 74.17134337351254) bank9061 +9062 POINT(41.54205186507517 73.29336798583151) bank9062 +9063 POINT(40.43374336640258 73.60865377633861) bank9063 +9064 POINT(41.699664502203504 74.31351875166361) bank9064 +9065 POINT(41.58176177737838 74.72741216459768) bank9065 +9066 POINT(39.71331133110322 73.31973487118978) bank9066 +9067 POINT(41.54213504120608 74.27293492974623) bank9067 +9068 POINT(40.92068328043936 73.95699359280276) bank9068 +9069 POINT(41.4459202304701 73.30837619780644) bank9069 +9070 POINT(41.47541981261944 73.35773308798538) bank9070 +9071 POINT(41.19746013108345 73.04563587881785) bank9071 +9072 POINT(41.123106770758646 73.8624008411051) bank9072 +9073 POINT(41.250350902418944 74.42886742977157) bank9073 +9074 POINT(40.101274854232756 73.39306639237186) bank9074 +9075 POINT(41.61008530379912 74.8382767827339) bank9075 +9076 POINT(39.7806507113351 74.62256524430434) bank9076 +9077 POINT(40.1422414573191 73.18932363176813) bank9077 +9078 POINT(41.40225722506173 74.0243432326824) bank9078 +9079 POINT(41.01632801221393 73.49392497492619) bank9079 +9080 POINT(41.64112468340148 73.27972487208302) bank9080 +9081 POINT(41.49723808138135 73.95405789936329) bank9081 +9082 POINT(41.562356164766605 73.67969509429676) bank9082 +9083 POINT(40.4669335311814 74.48069813310889) bank9083 +9084 POINT(40.61700957567529 73.70345439415415) bank9084 +9085 POINT(41.09861501007878 74.70675476623852) bank9085 +9086 POINT(39.723132993847486 74.45634171511281) bank9086 +9087 POINT(41.670305029555195 73.23145353271015) bank9087 +9088 POINT(40.79487797284664 73.56666039655529) bank9088 +9089 POINT(39.850625821996346 73.19407090814018) bank9089 +9090 POINT(39.85420554286911 74.0481331351314) bank9090 +9091 POINT(40.707646867743215 73.13136470371659) bank9091 +9092 POINT(40.94612595939577 74.99063089030001) bank9092 +9093 POINT(40.414078414818974 74.97621698433757) bank9093 +9094 POINT(40.698176047706596 74.32376945467549) bank9094 +9095 POINT(41.358472732444596 73.96879643678682) bank9095 +9096 POINT(41.2014628985552 74.0284200840832) bank9096 +9097 POINT(40.560993346282245 74.5156591772393) bank9097 +9098 POINT(40.449707851724554 73.13675422011183) bank9098 +9099 POINT(41.65350059387118 74.14650105166693) bank9099 +9100 POINT(41.551700341921396 73.44859380007986) bank9100 +9101 POINT(40.117925384054274 73.53678889443637) bank9101 +9102 POINT(39.953299380723955 73.17044618005664) bank9102 +9103 POINT(40.29833351679013 74.21666015668761) bank9103 +9104 POINT(41.136194319284144 73.51349517800969) bank9104 +9105 POINT(41.20785670938091 74.53856993326467) bank9105 +9106 POINT(40.71569821804517 73.22177714381124) bank9106 +9107 POINT(41.19285335409117 73.56257625421031) bank9107 +9108 POINT(41.43054739691335 73.98885711499386) bank9108 +9109 POINT(40.37613192031675 74.83643820622001) bank9109 +9110 POINT(40.66082790518959 74.27761588570009) bank9110 +9111 POINT(40.28418795406473 74.58875883533274) bank9111 +9112 POINT(40.66533378497527 74.02460603076888) bank9112 +9113 POINT(39.72489182410075 73.4483462378308) bank9113 +9114 POINT(40.810309312685476 74.84143775702316) bank9114 +9115 POINT(40.503912015364044 74.46170598886297) bank9115 +9116 POINT(39.85037728285559 73.23836874537118) bank9116 +9117 POINT(39.969747755231225 74.24288153488332) bank9117 +9118 POINT(41.15577905299917 73.88189382953243) bank9118 +9119 POINT(41.57422613463348 74.7547838745645) bank9119 +9120 POINT(40.59211856134985 74.50765979777326) bank9120 +9121 POINT(40.87908346546036 73.77515165948188) bank9121 +9122 POINT(39.837447273727875 73.52167713943028) bank9122 +9123 POINT(41.42037476273388 74.07563151095756) bank9123 +9124 POINT(40.67700875762801 73.46249460974458) bank9124 +9125 POINT(40.400118101138915 73.98668299648898) bank9125 +9126 POINT(41.11210310067281 74.57695858322766) bank9126 +9127 POINT(41.6602246750396 73.08018972063493) bank9127 +9128 POINT(40.2984866485427 74.53612181931052) bank9128 +9129 POINT(40.46241082027081 73.56942563636282) bank9129 +9130 POINT(41.481415776226335 73.64661345785505) bank9130 +9131 POINT(40.745427458993916 73.93389256728994) bank9131 +9132 POINT(40.90712365503637 74.31234363244093) bank9132 +9133 POINT(41.222182201090355 74.39086310374093) bank9133 +9134 POINT(39.75202295226989 74.78337799223856) bank9134 +9135 POINT(41.46573214536661 73.92340237663798) bank9135 +9136 POINT(41.27053323517405 73.46545783949826) bank9136 +9137 POINT(41.63676597013738 74.59637087669523) bank9137 +9138 POINT(39.81869981125767 73.23888510442917) bank9138 +9139 POINT(41.353832483266224 73.46610378685277) bank9139 +9140 POINT(40.230496095389256 74.16475676208825) bank9140 +9141 POINT(40.683893731843185 73.34294979020038) bank9141 +9142 POINT(40.08294968304545 74.54650115956099) bank9142 +9143 POINT(41.09581359113104 73.48043059453019) bank9143 +9144 POINT(40.45758000924691 73.55247050709184) bank9144 +9145 POINT(40.14890470083503 73.19850219674312) bank9145 +9146 POINT(39.772211683861954 74.10934953520176) bank9146 +9147 POINT(40.64790007914542 74.56236430295169) bank9147 +9148 POINT(40.48332696681865 74.77272063269446) bank9148 +9149 POINT(41.56379059096478 73.52854435679211) bank9149 +9150 POINT(40.74457416230085 73.39730116366997) bank9150 +9151 POINT(40.78872600662928 73.16416776122448) bank9151 +9152 POINT(41.17391871342086 73.15154643638891) bank9152 +9153 POINT(40.42355389212898 73.82049190903928) bank9153 +9154 POINT(39.780974679698986 74.99456240780388) bank9154 +9155 POINT(39.860207529120316 73.47494748120424) bank9155 +9156 POINT(40.47372482827218 74.63975251679416) bank9156 +9157 POINT(40.66995964022891 74.37361022356946) bank9157 +9158 POINT(41.438461858247194 73.68608206575847) bank9158 +9159 POINT(41.325353476293714 74.56057644444594) bank9159 +9160 POINT(40.57379006754903 74.30711106090523) bank9160 +9161 POINT(41.41107143047234 73.0554202530945) bank9161 +9162 POINT(41.25170149610663 74.8285699307925) bank9162 +9163 POINT(40.63696985452505 74.52365259826102) bank9163 +9164 POINT(41.70893397127289 73.62771098830302) bank9164 +9165 POINT(41.093365469267 73.06057385076012) bank9165 +9166 POINT(40.33404524369971 73.11150792647817) bank9166 +9167 POINT(41.1780871824149 74.41769008132046) bank9167 +9168 POINT(41.666794878087074 73.01158146131961) bank9168 +9169 POINT(40.689893088315266 73.59514201255692) bank9169 +9170 POINT(41.4602026642784 73.63829830880444) bank9170 +9171 POINT(40.34141260189237 73.24660969482429) bank9171 +9172 POINT(41.04284963453025 74.37589370432384) bank9172 +9173 POINT(39.91583496390692 73.29106542093317) bank9173 +9174 POINT(40.94181340288586 74.97337222964045) bank9174 +9175 POINT(41.57784083173972 74.65094437545773) bank9175 +9176 POINT(39.8919912780466 74.81575221813644) bank9176 +9177 POINT(39.847401162029456 74.98708524402397) bank9177 +9178 POINT(40.5273537986608 74.13125709182903) bank9178 +9179 POINT(39.773074765463925 73.10007890532644) bank9179 +9180 POINT(41.05208488602198 73.64766986500683) bank9180 +9181 POINT(40.77612423862718 74.14151143154801) bank9181 +9182 POINT(41.39682333759795 74.543302912253) bank9182 +9183 POINT(40.713381932505555 73.47383615179866) bank9183 +9184 POINT(40.38355039361684 73.61998032639872) bank9184 +9185 POINT(40.424845883125194 74.894303565696) bank9185 +9186 POINT(41.62222853035829 73.58721161871871) bank9186 +9187 POINT(41.6228064028335 75.00289299700697) bank9187 +9188 POINT(41.05130918531774 73.26529471947548) bank9188 +9189 POINT(40.99778828563819 73.05723517530868) bank9189 +9190 POINT(39.95672799567755 74.04602859531485) bank9190 +9191 POINT(40.63182949287551 74.56020223854718) bank9191 +9192 POINT(41.435219322079995 73.45148801207912) bank9192 +9193 POINT(40.67147405963841 73.77694673180034) bank9193 +9194 POINT(41.287121419662554 74.80590359632761) bank9194 +9195 POINT(40.33637626636649 73.4953736645988) bank9195 +9196 POINT(40.08523189648399 73.24197790676749) bank9196 +9197 POINT(40.666814544000076 73.59167045862115) bank9197 +9198 POINT(40.34655314742531 74.71951361031923) bank9198 +9199 POINT(41.411673271126105 74.03580823741115) bank9199 +9200 POINT(41.63158053145227 73.18287070110777) bank9200 +9201 POINT(40.63434137420727 74.78337000597563) bank9201 +9202 POINT(40.38748221014913 73.66882955834205) bank9202 +9203 POINT(41.07674183630274 74.85596331972138) bank9203 +9204 POINT(40.679300938098464 73.18817929135149) bank9204 +9205 POINT(41.660758827884834 73.05346597779048) bank9205 +9206 POINT(41.563945857451515 73.17681985574771) bank9206 +9207 POINT(41.10685564881575 74.41348582356527) bank9207 +9208 POINT(40.78644068152476 74.17772166103123) bank9208 +9209 POINT(40.877494668477645 74.93949663541241) bank9209 +9210 POINT(41.075811786308755 74.14372529897003) bank9210 +9211 POINT(40.90393673418049 74.23908226180055) bank9211 +9212 POINT(41.65700006801309 74.2739021549439) bank9212 +9213 POINT(40.63926090318495 73.84339229684782) bank9213 +9214 POINT(41.39985602981953 74.10956422038765) bank9214 +9215 POINT(40.787856049025976 74.56551301548913) bank9215 +9216 POINT(39.72802333856135 73.72208806721824) bank9216 +9217 POINT(40.1486867297452 74.87713521168331) bank9217 +9218 POINT(40.23594986035307 73.46204887783192) bank9218 +9219 POINT(40.72665192240734 73.8345135993299) bank9219 +9220 POINT(40.351636356752905 73.87229401515657) bank9220 +9221 POINT(41.270502406211435 73.51521408551277) bank9221 +9222 POINT(40.32722287128296 74.54748784251474) bank9222 +9223 POINT(41.49460209433955 74.34093284306589) bank9223 +9224 POINT(40.21484108256223 73.63244702167736) bank9224 +9225 POINT(40.79188200664683 73.84062693725635) bank9225 +9226 POINT(40.535050211058284 74.16380882045044) bank9226 +9227 POINT(40.305130453139824 73.24719134853397) bank9227 +9228 POINT(40.934661871034145 74.8552281151687) bank9228 +9229 POINT(41.35666149994089 73.84680221719378) bank9229 +9230 POINT(39.9571458986933 73.50167640134234) bank9230 +9231 POINT(40.23532714883533 74.23072831436443) bank9231 +9232 POINT(41.52186817482677 74.9370836573579) bank9232 +9233 POINT(41.605513323856506 73.2574614145358) bank9233 +9234 POINT(40.38194388217617 73.40613840977525) bank9234 +9235 POINT(41.498766859391345 74.41814093970538) bank9235 +9236 POINT(41.420502733056416 74.56046170329643) bank9236 +9237 POINT(40.20323332313539 73.68111629627076) bank9237 +9238 POINT(39.94695228017392 74.39181461579552) bank9238 +9239 POINT(40.83682269483901 73.0122004082617) bank9239 +9240 POINT(41.37870903771447 74.22964685829172) bank9240 +9241 POINT(41.586147784166364 73.07120720699034) bank9241 +9242 POINT(41.65494457690267 73.69318670905174) bank9242 +9243 POINT(41.35875390995577 73.08777645989571) bank9243 +9244 POINT(39.96650145642939 74.55468718377578) bank9244 +9245 POINT(41.66536743975044 74.3645778541522) bank9245 +9246 POINT(40.51953914362611 74.31639032286004) bank9246 +9247 POINT(40.254869270838086 73.50305372341731) bank9247 +9248 POINT(40.7764987618023 73.09565129784403) bank9248 +9249 POINT(40.489992401461464 73.89926131429533) bank9249 +9250 POINT(41.05199844925215 74.26833718098489) bank9250 +9251 POINT(40.190554519802774 74.1969516885296) bank9251 +9252 POINT(39.766222680809534 73.83610792869368) bank9252 +9253 POINT(41.29752410777298 74.04253533847917) bank9253 +9254 POINT(40.211420692580305 73.9607320008704) bank9254 +9255 POINT(39.74685094665063 74.5656104325175) bank9255 +9256 POINT(41.18028394101969 73.06862611118021) bank9256 +9257 POINT(40.12168523102029 73.13514510022425) bank9257 +9258 POINT(39.90590602416527 73.93200461428293) bank9258 +9259 POINT(40.03048660087039 73.49157726248573) bank9259 +9260 POINT(40.68846780259407 73.35161458712211) bank9260 +9261 POINT(40.027878753575045 73.59376875969582) bank9261 +9262 POINT(39.78586862891561 74.91791339061183) bank9262 +9263 POINT(39.924527739389916 73.83827127239289) bank9263 +9264 POINT(41.479140825946025 75.00150471437901) bank9264 +9265 POINT(40.173980211146244 73.40891127359411) bank9265 +9266 POINT(41.30017143703649 73.42924281852524) bank9266 +9267 POINT(40.078270224146145 74.96775681658572) bank9267 +9268 POINT(40.8918573192271 73.1632817894003) bank9268 +9269 POINT(41.4426541881754 74.08858251965962) bank9269 +9270 POINT(41.22481324407273 73.78224305382268) bank9270 +9271 POINT(39.829432972046526 73.68167902387059) bank9271 +9272 POINT(40.19845572084622 73.97096590294319) bank9272 +9273 POINT(41.335964899922686 73.4899974408297) bank9273 +9274 POINT(40.867813266933474 73.20933127778947) bank9274 +9275 POINT(40.602418258444786 73.06067384052913) bank9275 +9276 POINT(40.261581038030975 74.80057872927932) bank9276 +9277 POINT(41.639486478483555 73.11515642353618) bank9277 +9278 POINT(39.97817526648615 73.05914520702088) bank9278 +9279 POINT(41.37071517528832 74.87016051989596) bank9279 +9280 POINT(41.23878966830073 74.17379770739083) bank9280 +9281 POINT(39.97439769899002 73.32240613013045) bank9281 +9282 POINT(40.97595197292681 74.61138544810518) bank9282 +9283 POINT(39.765798911574926 74.7357837591777) bank9283 +9284 POINT(41.16491961102104 73.65740650416728) bank9284 +9285 POINT(41.44507117245657 74.9039398550291) bank9285 +9286 POINT(40.15668327008553 74.518251736404) bank9286 +9287 POINT(40.65536768303763 73.23003764039673) bank9287 +9288 POINT(39.82831620437934 74.28544262942707) bank9288 +9289 POINT(40.788700472520915 73.76726607970133) bank9289 +9290 POINT(41.343568178411495 74.89014110379823) bank9290 +9291 POINT(41.39584275805476 73.80213246167474) bank9291 +9292 POINT(41.284510178338785 73.37173370580734) bank9292 +9293 POINT(41.440570837353015 74.38956925013139) bank9293 +9294 POINT(41.36908497805362 73.03721386077623) bank9294 +9295 POINT(40.26014510812069 73.54752832971897) bank9295 +9296 POINT(40.94174660367168 74.76388551726684) bank9296 +9297 POINT(41.16097213853513 74.84563568479179) bank9297 +9298 POINT(40.37399993785302 73.09598614568816) bank9298 +9299 POINT(40.77841675043306 74.0366239353327) bank9299 +9300 POINT(40.34284187115036 74.32314261197396) bank9300 +9301 POINT(41.31012804449186 73.8547614695569) bank9301 +9302 POINT(39.99177971680132 73.17262615088988) bank9302 +9303 POINT(39.78077101247977 73.16524705921813) bank9303 +9304 POINT(40.49529744593052 74.8588034556099) bank9304 +9305 POINT(40.895867602913874 73.81274264770805) bank9305 +9306 POINT(40.145711581575455 74.16679314829685) bank9306 +9307 POINT(39.74887514066749 74.15797494383939) bank9307 +9308 POINT(39.881256814185335 73.32930572323214) bank9308 +9309 POINT(39.85552232275065 74.34070092746437) bank9309 +9310 POINT(40.06788001789188 74.00559157449293) bank9310 +9311 POINT(40.97502078569295 74.84558975297261) bank9311 +9312 POINT(40.50981096324616 73.08959618132114) bank9312 +9313 POINT(41.08741322515872 73.97805071715057) bank9313 +9314 POINT(40.721083065700775 74.37204890610484) bank9314 +9315 POINT(40.619046533503194 73.47024620171541) bank9315 +9316 POINT(40.51356207098877 73.00777737777663) bank9316 +9317 POINT(40.00153618620709 74.82211236695788) bank9317 +9318 POINT(40.71609186552385 73.38869174763772) bank9318 +9319 POINT(40.58573891508908 73.87561195002311) bank9319 +9320 POINT(41.12550399797033 73.69988895902097) bank9320 +9321 POINT(41.44231476506415 73.7977657548376) bank9321 +9322 POINT(39.757837595333605 74.05582684567767) bank9322 +9323 POINT(40.146069146717394 74.61923458205595) bank9323 +9324 POINT(41.351400375974265 73.72018385225245) bank9324 +9325 POINT(40.676223019763185 74.32456932339875) bank9325 +9326 POINT(40.775554833245515 73.29505987660045) bank9326 +9327 POINT(40.541790479293354 74.8485075134883) bank9327 +9328 POINT(40.84830439826353 73.46102029383889) bank9328 +9329 POINT(40.30723874073463 74.33274018400436) bank9329 +9330 POINT(39.966231374227185 74.20423539423209) bank9330 +9331 POINT(41.059666132979025 73.31805876907434) bank9331 +9332 POINT(40.802854785892634 73.24406601107142) bank9332 +9333 POINT(40.84167702179657 73.36895401560045) bank9333 +9334 POINT(41.385151565667655 74.05069661101815) bank9334 +9335 POINT(41.66358272650331 74.4988700918063) bank9335 +9336 POINT(40.08583406331106 74.7135479038672) bank9336 +9337 POINT(39.80716373600799 73.74576414331675) bank9337 +9338 POINT(39.966398881170974 74.31885679186313) bank9338 +9339 POINT(41.10247752983202 73.62624717956716) bank9339 +9340 POINT(39.91709845756083 73.51770333257201) bank9340 +9341 POINT(39.89862529992601 74.16279435100445) bank9341 +9342 POINT(41.219756217223235 73.79174322742132) bank9342 +9343 POINT(40.91890855764352 74.92463226323629) bank9343 +9344 POINT(39.93396553449738 73.41125394532506) bank9344 +9345 POINT(40.838568135792116 74.9396743384094) bank9345 +9346 POINT(41.02374878244236 74.48450469773606) bank9346 +9347 POINT(41.265234747191386 74.77831252047596) bank9347 +9348 POINT(39.82309388640928 74.7276710398261) bank9348 +9349 POINT(41.703019576808195 73.39816562342418) bank9349 +9350 POINT(40.512321686042974 73.60867219813133) bank9350 +9351 POINT(40.307734094513705 74.62616851651454) bank9351 +9352 POINT(41.02841007678169 73.07015057937222) bank9352 +9353 POINT(39.97209236863009 73.84470750150805) bank9353 +9354 POINT(41.42360437087488 74.92976164206253) bank9354 +9355 POINT(39.930165251260924 74.50378797328653) bank9355 +9356 POINT(41.22634834311222 74.17021652954011) bank9356 +9357 POINT(40.67857713389829 73.39440086437507) bank9357 +9358 POINT(40.62548523963682 74.97798725355827) bank9358 +9359 POINT(41.47934289663157 74.82381891664897) bank9359 +9360 POINT(41.70055390056734 73.08118161276715) bank9360 +9361 POINT(41.32416383024929 73.87008275126873) bank9361 +9362 POINT(41.099227217605744 73.38919910030882) bank9362 +9363 POINT(40.487115001412896 73.74890720697907) bank9363 +9364 POINT(39.97782742575198 73.72952825720174) bank9364 +9365 POINT(39.76540552930361 74.07547415437804) bank9365 +9366 POINT(40.197352577620265 73.72297323257388) bank9366 +9367 POINT(41.15936917485088 73.79105229579163) bank9367 +9368 POINT(40.43882925537039 74.25572888315975) bank9368 +9369 POINT(40.42901306456251 73.19541321162856) bank9369 +9370 POINT(41.587521456133985 74.37693347138688) bank9370 +9371 POINT(39.814109383950246 74.74225506725571) bank9371 +9372 POINT(40.11529447313094 74.8186824754632) bank9372 +9373 POINT(40.028252805306366 73.10133159249152) bank9373 +9374 POINT(40.24276967669407 74.56474837228659) bank9374 +9375 POINT(41.24257291279646 74.46903093827677) bank9375 +9376 POINT(40.50216291723389 74.55677513665862) bank9376 +9377 POINT(40.4997581804366 73.05081487361416) bank9377 +9378 POINT(41.40298554617105 73.59150560908454) bank9378 +9379 POINT(40.99159589699542 73.06893205763276) bank9379 +9380 POINT(40.976157744502224 74.31740676345815) bank9380 +9381 POINT(41.61606022472961 74.99961362965853) bank9381 +9382 POINT(40.137360006727505 73.83213756162633) bank9382 +9383 POINT(40.08442391594133 73.38942017219725) bank9383 +9384 POINT(40.65188711961231 73.12931458921416) bank9384 +9385 POINT(40.2366058336425 74.46590404699482) bank9385 +9386 POINT(39.90896979083237 74.90370587748492) bank9386 +9387 POINT(40.99267144188933 74.88572060861995) bank9387 +9388 POINT(40.438736187650754 73.33484821153264) bank9388 +9389 POINT(39.82358094990928 74.0655644547803) bank9389 +9390 POINT(40.90749023622088 73.01614038623546) bank9390 +9391 POINT(41.00357640896965 74.31493526102567) bank9391 +9392 POINT(41.086056488968765 73.15963021501899) bank9392 +9393 POINT(40.63428653970507 73.56778637972067) bank9393 +9394 POINT(41.08930554927811 73.51595110153939) bank9394 +9395 POINT(40.06738685351573 73.75682009690044) bank9395 +9396 POINT(40.38176459914289 73.1396186135047) bank9396 +9397 POINT(40.14535260087869 74.32312723712228) bank9397 +9398 POINT(40.44237361268408 73.14353109333167) bank9398 +9399 POINT(40.71389304149331 74.92520179957043) bank9399 +9400 POINT(41.42380097123958 73.99744963021736) bank9400 +9401 POINT(39.98124665683704 73.18350018395832) bank9401 +9402 POINT(40.80231953150602 73.80178259693149) bank9402 +9403 POINT(40.50361993330131 73.93133923828842) bank9403 +9404 POINT(39.92766364813601 74.54532758025523) bank9404 +9405 POINT(40.83003574422919 74.84579648571038) bank9405 +9406 POINT(41.32218472978021 73.04658483655145) bank9406 +9407 POINT(40.28378662889615 74.72321898897879) bank9407 +9408 POINT(40.13135333186979 74.99078448607115) bank9408 +9409 POINT(40.85620378742923 74.45145546236438) bank9409 +9410 POINT(39.96712226966351 73.16021886014153) bank9410 +9411 POINT(40.71423473797384 73.13165582564997) bank9411 +9412 POINT(40.50993052424613 73.07022792783673) bank9412 +9413 POINT(41.57171274399555 74.65456680236873) bank9413 +9414 POINT(41.50619006774295 74.07112491477132) bank9414 +9415 POINT(41.51340415144916 74.66037515753708) bank9415 +9416 POINT(41.55469192531284 74.05734489366351) bank9416 +9417 POINT(39.79338564749099 73.26811663676983) bank9417 +9418 POINT(40.23203963383035 73.57566256122881) bank9418 +9419 POINT(40.193886519842806 73.42631735634927) bank9419 +9420 POINT(40.39965072188973 73.82144525190259) bank9420 +9421 POINT(39.80035457687634 74.67196793396737) bank9421 +9422 POINT(40.67620430587463 74.5588204470876) bank9422 +9423 POINT(40.2316743583259 74.44044956879928) bank9423 +9424 POINT(41.1820870085584 73.32777066756593) bank9424 +9425 POINT(40.28714006708214 74.59443291882936) bank9425 +9426 POINT(40.814615825820816 73.72580937449635) bank9426 +9427 POINT(41.070189574687284 74.83506280732426) bank9427 +9428 POINT(41.305305558097785 73.16748719363096) bank9428 +9429 POINT(39.79207686568058 73.89807959612126) bank9429 +9430 POINT(39.94088002744035 74.37274266345386) bank9430 +9431 POINT(40.686936109868405 74.37137155376155) bank9431 +9432 POINT(40.86866623796823 74.63140293289455) bank9432 +9433 POINT(40.011026462524555 73.521620818492) bank9433 +9434 POINT(40.21800872758295 74.1205506434716) bank9434 +9435 POINT(39.969343175740576 73.88664797712148) bank9435 +9436 POINT(40.77931243283409 74.25146522651269) bank9436 +9437 POINT(40.74964038153342 73.45587811785688) bank9437 +9438 POINT(40.32147964699443 74.65304381268453) bank9438 +9439 POINT(41.20742626413423 74.01922707351365) bank9439 +9440 POINT(41.665063821292016 74.83658037300879) bank9440 +9441 POINT(40.11141105957124 74.68303690965536) bank9441 +9442 POINT(41.34503580109637 74.92527814110458) bank9442 +9443 POINT(39.75760387352043 73.99732884343726) bank9443 +9444 POINT(39.74403388637479 73.22583953253528) bank9444 +9445 POINT(39.96695083577313 74.55571145895941) bank9445 +9446 POINT(40.35211250717074 74.60818969285518) bank9446 +9447 POINT(40.01050713871484 73.93313267717842) bank9447 +9448 POINT(41.11156373990671 74.7541707472997) bank9448 +9449 POINT(41.59201479925369 74.42830114225549) bank9449 +9450 POINT(40.605785578520916 73.86925193890231) bank9450 +9451 POINT(41.63164118830721 74.95238580411515) bank9451 +9452 POINT(41.28455404058833 73.9249347901941) bank9452 +9453 POINT(39.876962214872044 74.90462104017858) bank9453 +9454 POINT(40.9590085310772 74.45461865681123) bank9454 +9455 POINT(40.91547570335156 73.82294710525163) bank9455 +9456 POINT(41.15931554181247 73.0200447105568) bank9456 +9457 POINT(39.933554978067775 73.93849031945287) bank9457 +9458 POINT(41.50174312624486 74.31762907772229) bank9458 +9459 POINT(41.645516689156224 74.51183026769225) bank9459 +9460 POINT(40.587271051073884 73.9221259094945) bank9460 +9461 POINT(39.7931282343379 73.11689526112954) bank9461 +9462 POINT(40.25039909511211 73.73933853611689) bank9462 +9463 POINT(41.46818546693093 73.67241564976965) bank9463 +9464 POINT(41.13145290982631 73.965401879189) bank9464 +9465 POINT(41.56404810627527 73.80087291659335) bank9465 +9466 POINT(40.309908350428444 74.8965735763428) bank9466 +9467 POINT(41.15155664122491 74.25762603398493) bank9467 +9468 POINT(40.47513802130861 74.65229099260517) bank9468 +9469 POINT(39.98602512143994 73.39839079343491) bank9469 +9470 POINT(39.79934820203396 73.63199495293647) bank9470 +9471 POINT(41.47895630928488 74.97439404101371) bank9471 +9472 POINT(39.97102476400368 73.5260509721249) bank9472 +9473 POINT(40.09262019293154 74.03795272034336) bank9473 +9474 POINT(40.989130932109354 73.8805138590634) bank9474 +9475 POINT(39.81397427945583 74.32713282008486) bank9475 +9476 POINT(40.33058839776307 73.11990023202533) bank9476 +9477 POINT(40.82958282609507 74.06108804619655) bank9477 +9478 POINT(39.81590377636873 74.77385587170473) bank9478 +9479 POINT(40.07666667154905 73.11698328403054) bank9479 +9480 POINT(41.54522389158271 73.98604933697253) bank9480 +9481 POINT(40.66030533104185 74.62544855903295) bank9481 +9482 POINT(39.99999905671662 73.40030411510496) bank9482 +9483 POINT(40.45105663056966 74.21429197863615) bank9483 +9484 POINT(41.61583256003596 73.55724391649377) bank9484 +9485 POINT(41.648233369388514 74.32118863317788) bank9485 +9486 POINT(41.326806964031434 73.93798105638112) bank9486 +9487 POINT(41.604464411680304 74.97893463367984) bank9487 +9488 POINT(40.25059883797536 74.11824309562569) bank9488 +9489 POINT(40.70854008333166 73.04368483589298) bank9489 +9490 POINT(41.62865588574937 74.7094524453385) bank9490 +9491 POINT(40.60801163397666 73.76052863941516) bank9491 +9492 POINT(40.996251682258745 73.44626718091341) bank9492 +9493 POINT(40.23191809554309 74.99640221545471) bank9493 +9494 POINT(39.743840188726516 73.68326676686544) bank9494 +9495 POINT(41.67652024342442 73.61065733654917) bank9495 +9496 POINT(40.49449303820736 73.90069424009795) bank9496 +9497 POINT(39.79648028813172 74.93248054368152) bank9497 +9498 POINT(40.157342742641845 74.35015827326197) bank9498 +9499 POINT(41.446334385926086 73.02348712469276) bank9499 +9500 POINT(40.79423454700654 74.54803856361268) bank9500 +9501 POINT(40.12059635823596 74.54492862001241) bank9501 +9502 POINT(39.80744589111973 73.94427677642405) bank9502 +9503 POINT(40.7144104192469 73.48566568682187) bank9503 +9504 POINT(39.930625382639924 73.70091274351783) bank9504 +9505 POINT(41.24864685521592 74.97507944122815) bank9505 +9506 POINT(40.77431587576876 73.10171956801581) bank9506 +9507 POINT(41.53915092686648 73.3327681744794) bank9507 +9508 POINT(40.5602190161357 73.28041730402143) bank9508 +9509 POINT(39.762406458381705 73.00626842751842) bank9509 +9510 POINT(40.23897335074057 73.44777359885613) bank9510 +9511 POINT(40.38581154644083 73.62388231391851) bank9511 +9512 POINT(39.964417409349906 74.77902667320198) bank9512 +9513 POINT(40.49456033113994 73.46138368328799) bank9513 +9514 POINT(41.4231593356113 73.28567333857093) bank9514 +9515 POINT(40.16658856747275 74.1570594285522) bank9515 +9516 POINT(41.344426685397664 74.67021797337782) bank9516 +9517 POINT(41.412277310432316 74.70596175809865) bank9517 +9518 POINT(40.663904830462094 73.5460599553443) bank9518 +9519 POINT(40.667225191950074 74.57088834054066) bank9519 +9520 POINT(41.15341942033368 74.74382534847092) bank9520 +9521 POINT(41.59954622710308 74.64372939314975) bank9521 +9522 POINT(41.660334020662 74.09326703527306) bank9522 +9523 POINT(41.682565535806084 74.1317591623748) bank9523 +9524 POINT(41.54145066097343 73.48369052050353) bank9524 +9525 POINT(39.79650379421111 74.61320801260518) bank9525 +9526 POINT(41.44796897267073 73.9617184668066) bank9526 +9527 POINT(39.911023030568536 73.22648411978625) bank9527 +9528 POINT(39.819878588982725 74.70110573833608) bank9528 +9529 POINT(39.937446860514626 73.43464580654548) bank9529 +9530 POINT(40.60282169243407 74.52767939641988) bank9530 +9531 POINT(40.72736516359827 74.15015759212194) bank9531 +9532 POINT(39.85248945824871 73.9607430625175) bank9532 +9533 POINT(41.70957804026764 73.2896527139602) bank9533 +9534 POINT(41.634467000782074 73.6687342119893) bank9534 +9535 POINT(40.888848168626005 73.89374854065929) bank9535 +9536 POINT(41.64867805558773 73.32614955244708) bank9536 +9537 POINT(40.70477083925059 74.61353719957845) bank9537 +9538 POINT(40.698574802595964 74.63043055511956) bank9538 +9539 POINT(41.28853284528459 74.7868222333029) bank9539 +9540 POINT(39.81206786137904 73.47059476734665) bank9540 +9541 POINT(40.870536472911446 73.12876182837242) bank9541 +9542 POINT(40.44169714764803 74.721958372575) bank9542 +9543 POINT(41.09683062664608 73.76434091840332) bank9543 +9544 POINT(39.909239335426705 74.00959612383377) bank9544 +9545 POINT(40.65476003961958 73.34013405071256) bank9545 +9546 POINT(40.40267160930543 74.17350531448113) bank9546 +9547 POINT(40.89469900482679 73.22447441608942) bank9547 +9548 POINT(40.54812021990987 74.15980846121927) bank9548 +9549 POINT(40.39102618042143 73.32562556868697) bank9549 +9550 POINT(41.26561615641998 73.84999329976309) bank9550 +9551 POINT(39.789837351148996 73.05653043787741) bank9551 +9552 POINT(40.392089718532894 74.09746253330935) bank9552 +9553 POINT(40.677702350605024 74.40267049103711) bank9553 +9554 POINT(40.052373200552104 73.66759951656431) bank9554 +9555 POINT(40.10292089949135 73.44243775303181) bank9555 +9556 POINT(40.16352346208704 74.72236064000336) bank9556 +9557 POINT(39.98764675111247 74.13729716908836) bank9557 +9558 POINT(41.24159256844399 73.39474767197949) bank9558 +9559 POINT(40.83912534064326 74.39248460808797) bank9559 +9560 POINT(41.476858415297485 73.19535565233008) bank9560 +9561 POINT(41.521993175612266 73.71129574936096) bank9561 +9562 POINT(40.69326559844224 73.22308671661929) bank9562 +9563 POINT(40.78279628121011 73.81603330369768) bank9563 +9564 POINT(40.92490005617016 74.2503804319327) bank9564 +9565 POINT(40.08535345807671 73.82786113891957) bank9565 +9566 POINT(41.4844666158383 73.71084116464121) bank9566 +9567 POINT(40.479576886217934 74.61597007918738) bank9567 +9568 POINT(40.415212902137085 74.90419442071277) bank9568 +9569 POINT(41.55510465418032 73.12950939710039) bank9569 +9570 POINT(41.361103613937146 73.11329308798855) bank9570 +9571 POINT(39.84414161204535 73.84387822950427) bank9571 +9572 POINT(41.54697915120379 73.7063820906323) bank9572 +9573 POINT(40.916239463126445 73.9864753635224) bank9573 +9574 POINT(39.73974520886811 73.45142900455177) bank9574 +9575 POINT(40.30963720717182 74.01689383435144) bank9575 +9576 POINT(41.32451244442658 73.43518150350987) bank9576 +9577 POINT(40.41617386892399 74.86259815085573) bank9577 +9578 POINT(39.92647168473107 74.08149417332724) bank9578 +9579 POINT(41.36640689183869 73.63962257666583) bank9579 +9580 POINT(40.92675756854569 73.58421475227651) bank9580 +9581 POINT(41.487911551033065 74.58637029416191) bank9581 +9582 POINT(40.33264697587814 73.89441862108656) bank9582 +9583 POINT(40.968317855006575 73.95747653051757) bank9583 +9584 POINT(41.175992479188494 73.00872201702019) bank9584 +9585 POINT(40.93684985909203 74.84156055710051) bank9585 +9586 POINT(40.40257929423368 74.76717131393653) bank9586 +9587 POINT(41.23783739013238 74.31338074796477) bank9587 +9588 POINT(41.37657553738005 74.85270403557064) bank9588 +9589 POINT(41.048534405167814 73.98100048444391) bank9589 +9590 POINT(39.87509601080951 73.61595333768774) bank9590 +9591 POINT(41.2449504286255 74.24299514456732) bank9591 +9592 POINT(40.144910880348505 73.25516615530114) bank9592 +9593 POINT(41.646903378581335 74.64214507087914) bank9593 +9594 POINT(39.71990627293456 74.48757542055631) bank9594 +9595 POINT(40.531541858026436 73.66863252423093) bank9595 +9596 POINT(40.95119319282804 73.39791130510046) bank9596 +9597 POINT(41.53482147727718 73.84732823816776) bank9597 +9598 POINT(40.84479013183092 73.0907524218043) bank9598 +9599 POINT(40.36463066879975 73.6556652738862) bank9599 +9600 POINT(40.60526731919631 74.69293004955013) bank9600 +9601 POINT(39.72955192008981 74.1464763360784) bank9601 +9602 POINT(41.608270637417796 74.14897132013195) bank9602 +9603 POINT(40.54338870432194 73.99172571968434) bank9603 +9604 POINT(39.75232189616306 74.74846581921308) bank9604 +9605 POINT(40.42718703649785 73.63145106569904) bank9605 +9606 POINT(40.11525078110541 73.41223260228799) bank9606 +9607 POINT(40.618637978031536 73.15985480274543) bank9607 +9608 POINT(41.48267630391595 73.84105042753902) bank9608 +9609 POINT(40.49354388073117 73.508384088264) bank9609 +9610 POINT(39.82306571450222 73.8780024605015) bank9610 +9611 POINT(40.69626916122165 74.94485033284016) bank9611 +9612 POINT(40.6706784452902 74.78505832860007) bank9612 +9613 POINT(40.607415763837075 74.3166474353252) bank9613 +9614 POINT(40.072996009644704 74.76826299422105) bank9614 +9615 POINT(40.03243847421223 73.37237816724496) bank9615 +9616 POINT(40.72525829340366 74.54081185572272) bank9616 +9617 POINT(41.61625011110739 74.12280272155738) bank9617 +9618 POINT(40.7747105918609 74.97995035672976) bank9618 +9619 POINT(40.33521636611225 74.90998043106836) bank9619 +9620 POINT(40.40997654397161 74.29786157072157) bank9620 +9621 POINT(39.9100830437805 73.91195633123239) bank9621 +9622 POINT(40.145394012670664 74.65487104429994) bank9622 +9623 POINT(41.01817614710704 74.88229783269028) bank9623 +9624 POINT(39.774059459049134 74.38946354280084) bank9624 +9625 POINT(40.85536028849154 74.63148935168009) bank9625 +9626 POINT(40.10739248870151 73.10144585663431) bank9626 +9627 POINT(40.49448738046195 73.51692917821028) bank9627 +9628 POINT(40.517861919072736 73.58623025053438) bank9628 +9629 POINT(41.627007303011176 74.15796568361839) bank9629 +9630 POINT(41.06960526342114 73.44096972744822) bank9630 +9631 POINT(40.32553666997407 73.962184598706) bank9631 +9632 POINT(40.28476036829814 73.39265516298902) bank9632 +9633 POINT(40.35776206776299 74.85403240251371) bank9633 +9634 POINT(40.1658873459942 74.76854186501858) bank9634 +9635 POINT(39.73939464855708 73.83748423010934) bank9635 +9636 POINT(40.981233754091186 73.75190249438526) bank9636 +9637 POINT(41.445017715271014 73.6161307732128) bank9637 +9638 POINT(40.61734958047598 74.75971375533464) bank9638 +9639 POINT(41.35222421295662 74.78831223385446) bank9639 +9640 POINT(41.61270306993412 74.37102440825957) bank9640 +9641 POINT(40.106056395243144 73.92589374743353) bank9641 +9642 POINT(41.57974624614729 74.8727003047413) bank9642 +9643 POINT(40.46166077839921 74.22462547307914) bank9643 +9644 POINT(40.987596351945584 73.30363004602485) bank9644 +9645 POINT(40.27340772902589 74.92635772840957) bank9645 +9646 POINT(40.61148609765716 73.88737056722127) bank9646 +9647 POINT(41.02738377139557 74.82516982674497) bank9647 +9648 POINT(41.38913482462535 73.06097793057874) bank9648 +9649 POINT(40.80927774210636 74.20247600916106) bank9649 +9650 POINT(41.679089606972326 74.49076811304525) bank9650 +9651 POINT(41.075533351577654 74.15218558002842) bank9651 +9652 POINT(40.98221981950789 74.13328357430777) bank9652 +9653 POINT(40.24582568700722 74.5637958388795) bank9653 +9654 POINT(41.511437026313665 74.71710594473898) bank9654 +9655 POINT(40.19464279288609 73.66576045605201) bank9655 +9656 POINT(40.49721618976236 74.87292015840616) bank9656 +9657 POINT(40.67426095035445 74.72279054009593) bank9657 +9658 POINT(40.31037298290751 73.66463789176889) bank9658 +9659 POINT(40.03566500827581 73.2513753934887) bank9659 +9660 POINT(41.409194980510385 73.41886146458435) bank9660 +9661 POINT(39.92581860633514 73.08687579917364) bank9661 +9662 POINT(41.49160842029949 74.18724572916284) bank9662 +9663 POINT(40.53629001957543 73.21236430751999) bank9663 +9664 POINT(39.817604802223244 73.27448700852032) bank9664 +9665 POINT(41.08544537009062 74.93236815075817) bank9665 +9666 POINT(41.272207857459186 73.81825037750502) bank9666 +9667 POINT(40.273081748978655 73.49806845406539) bank9667 +9668 POINT(40.202761187624674 74.03314172036693) bank9668 +9669 POINT(40.21585016120511 74.15339388347319) bank9669 +9670 POINT(41.18652416267679 73.89844813327086) bank9670 +9671 POINT(41.709988296323615 73.451629697757) bank9671 +9672 POINT(40.502208428242355 74.33981637088269) bank9672 +9673 POINT(40.09492543583897 74.69661254486614) bank9673 +9674 POINT(41.336282370826446 74.89401082900702) bank9674 +9675 POINT(41.28608611696206 73.78134278786018) bank9675 +9676 POINT(41.06242935399431 74.72862784760788) bank9676 +9677 POINT(41.23265602773232 73.26715748577456) bank9677 +9678 POINT(40.741928636862546 74.78382186012472) bank9678 +9679 POINT(40.23710871128802 74.46300125677705) bank9679 +9680 POINT(40.432259251195006 73.12879801497222) bank9680 +9681 POINT(40.132878554630466 74.76309221449914) bank9681 +9682 POINT(40.38275342051562 75.0010124749284) bank9682 +9683 POINT(41.236191553491246 73.35149163774601) bank9683 +9684 POINT(40.84151913162011 73.79766178969388) bank9684 +9685 POINT(40.97685452801596 73.87533327857086) bank9685 +9686 POINT(41.597435800312205 73.7457030034347) bank9686 +9687 POINT(39.893570897727756 73.2640575465983) bank9687 +9688 POINT(41.491837885253666 73.38713360642785) bank9688 +9689 POINT(41.18321284960477 74.4681623947706) bank9689 +9690 POINT(39.81423610668174 73.1330626581337) bank9690 +9691 POINT(40.826364931383324 73.06427971996183) bank9691 +9692 POINT(41.39549424119671 73.32661166763693) bank9692 +9693 POINT(41.56821677400426 73.5482638500577) bank9693 +9694 POINT(40.06997021710423 73.90354241791549) bank9694 +9695 POINT(39.78286880684092 74.3353945640864) bank9695 +9696 POINT(41.50725382519079 74.5488941144818) bank9696 +9697 POINT(41.612566720618545 73.6444601135242) bank9697 +9698 POINT(40.74532022790034 73.34077424599191) bank9698 +9699 POINT(40.25057927042856 74.72498720920122) bank9699 +9700 POINT(39.9436004926254 73.96405578114896) bank9700 +9701 POINT(40.54394974732157 73.71319290350075) bank9701 +9702 POINT(41.18838798025929 74.51240405234482) bank9702 +9703 POINT(39.87916145954348 73.42363747049933) bank9703 +9704 POINT(40.299446317558754 74.98706533189664) bank9704 +9705 POINT(41.524755240719635 74.0736731572743) bank9705 +9706 POINT(40.704169047198654 74.76094902529033) bank9706 +9707 POINT(40.33392102200049 74.81465802013942) bank9707 +9708 POINT(40.49577351552894 74.02879889717627) bank9708 +9709 POINT(40.94085577025355 73.66418553221008) bank9709 +9710 POINT(40.339847595077785 73.43775431079072) bank9710 +9711 POINT(40.13246753863677 73.64111586847874) bank9711 +9712 POINT(41.6986154029937 74.75519486667) bank9712 +9713 POINT(41.54076808860643 73.57188135106945) bank9713 +9714 POINT(39.716604844235775 73.56735026997407) bank9714 +9715 POINT(40.61942221159629 73.41981271676114) bank9715 +9716 POINT(40.74635672493489 73.65662115194132) bank9716 +9717 POINT(40.46133832096558 73.57204134720035) bank9717 +9718 POINT(41.023863654650256 73.54423863858392) bank9718 +9719 POINT(41.121432653499284 73.21627525836259) bank9719 +9720 POINT(40.44779157155951 74.38554032287861) bank9720 +9721 POINT(40.18422542437525 74.31974777699237) bank9721 +9722 POINT(41.5702925844568 73.17792520209476) bank9722 +9723 POINT(40.03772976056377 74.71410135529241) bank9723 +9724 POINT(41.108690882711414 74.47637795610844) bank9724 +9725 POINT(40.03510612816214 73.90496841750004) bank9725 +9726 POINT(40.44742807592722 73.30758587319409) bank9726 +9727 POINT(39.748399344885975 74.9275882030602) bank9727 +9728 POINT(40.13148225126617 74.5705667446815) bank9728 +9729 POINT(41.090769937494436 73.39626959198169) bank9729 +9730 POINT(40.26133056812264 74.96114180170383) bank9730 +9731 POINT(41.51318009538434 73.16054515794114) bank9731 +9732 POINT(41.28666349608307 74.9875150637755) bank9732 +9733 POINT(39.810916136591715 74.4913841329533) bank9733 +9734 POINT(41.250568463276 73.83748238980112) bank9734 +9735 POINT(41.496424290137725 74.66048504196822) bank9735 +9736 POINT(40.07530966091902 73.35487835837132) bank9736 +9737 POINT(40.42949081628486 74.50946888030587) bank9737 +9738 POINT(40.87977640071996 74.50394954036113) bank9738 +9739 POINT(41.22973354980643 73.24998274009617) bank9739 +9740 POINT(40.3240108930179 73.83097296804476) bank9740 +9741 POINT(41.478279677649745 73.46683034296268) bank9741 +9742 POINT(41.44518710772488 73.38950068745248) bank9742 +9743 POINT(39.84381015359785 73.18740314308852) bank9743 +9744 POINT(41.48165905505944 73.07496783198613) bank9744 +9745 POINT(39.886528659049844 74.51306306562408) bank9745 +9746 POINT(40.87669090375388 73.45951642239625) bank9746 +9747 POINT(40.553176269551216 73.20382527885026) bank9747 +9748 POINT(41.072308843146835 74.70325837642775) bank9748 +9749 POINT(41.115334897177945 73.09783193596768) bank9749 +9750 POINT(41.337861437339846 73.88346452728025) bank9750 +9751 POINT(40.62892578311868 74.70543365571883) bank9751 +9752 POINT(41.56599465166238 74.55530454198393) bank9752 +9753 POINT(40.53105776384357 74.50225473410302) bank9753 +9754 POINT(39.749430273089615 73.13755303404317) bank9754 +9755 POINT(41.38370940149145 73.38709320336386) bank9755 +9756 POINT(40.58654306663605 73.79080255973928) bank9756 +9757 POINT(40.32893113368462 73.73115112311191) bank9757 +9758 POINT(40.484081422053805 74.63972930720406) bank9758 +9759 POINT(41.030207959070246 74.58820460354455) bank9759 +9760 POINT(40.81849738230973 73.49499540599852) bank9760 +9761 POINT(41.500755332200114 73.86010433478647) bank9761 +9762 POINT(40.084093075335815 74.09503730491065) bank9762 +9763 POINT(41.47006082923109 73.99136073999037) bank9763 +9764 POINT(40.071127267363686 74.71927580570686) bank9764 +9765 POINT(40.708782498979964 73.66386829772183) bank9765 +9766 POINT(40.307662052450226 73.65899669379118) bank9766 +9767 POINT(40.829277345673745 74.66741891603853) bank9767 +9768 POINT(40.22506748344706 75.00452757720868) bank9768 +9769 POINT(41.38710387132028 73.21825500016419) bank9769 +9770 POINT(41.18448559553475 74.76254726286456) bank9770 +9771 POINT(40.75001069833607 74.92738907404804) bank9771 +9772 POINT(40.33860178683509 73.12276001010882) bank9772 +9773 POINT(40.68082831654314 74.73162932230574) bank9773 +9774 POINT(40.34162915897668 74.55613281526675) bank9774 +9775 POINT(41.703935492332086 74.05729648340301) bank9775 +9776 POINT(41.280990407006975 74.21495538426787) bank9776 +9777 POINT(40.62502033581269 74.96932147406879) bank9777 +9778 POINT(41.63957155135176 74.09597579525402) bank9778 +9779 POINT(40.23738494161754 74.47060578419804) bank9779 +9780 POINT(39.75936853532132 73.7372032104162) bank9780 +9781 POINT(40.70019444050027 74.7449426927177) bank9781 +9782 POINT(41.48227315887184 74.96214061156897) bank9782 +9783 POINT(40.191854808507436 73.74219566504294) bank9783 +9784 POINT(41.46212439917783 74.05625997294169) bank9784 +9785 POINT(40.19045011439797 73.2062686670343) bank9785 +9786 POINT(40.62257526113822 74.10147441037013) bank9786 +9787 POINT(39.83994350349634 73.74725879972492) bank9787 +9788 POINT(41.691944649432756 74.1894046884833) bank9788 +9789 POINT(40.67613148541013 74.31605959055726) bank9789 +9790 POINT(39.762387710991455 74.38186188418257) bank9790 +9791 POINT(39.90887961238892 74.8614037518478) bank9791 +9792 POINT(40.421114257622506 74.32105918725244) bank9792 +9793 POINT(40.06122529635904 74.85996381466242) bank9793 +9794 POINT(40.41151299367106 73.46878357514676) bank9794 +9795 POINT(40.76978065293517 73.52940387647637) bank9795 +9796 POINT(40.27881702236687 73.57680111179818) bank9796 +9797 POINT(41.34071849654273 74.21821790098879) bank9797 +9798 POINT(40.29727700772132 74.75020441491894) bank9798 +9799 POINT(40.475785401506975 74.95138249015096) bank9799 +9800 POINT(40.6205880011358 74.3819070374273) bank9800 +9801 POINT(41.58070005222283 73.30898138650143) bank9801 +9802 POINT(40.80915594073735 73.11856464884514) bank9802 +9803 POINT(40.014667243061815 73.57455993308082) bank9803 +9804 POINT(40.86408432485537 73.98394213450311) bank9804 +9805 POINT(40.17368393947157 74.98743138468001) bank9805 +9806 POINT(40.06243027061383 74.46615312762114) bank9806 +9807 POINT(40.50106102055798 74.87170776903568) bank9807 +9808 POINT(40.181999957093765 74.46201273945447) bank9808 +9809 POINT(41.13858560902942 74.3956640732598) bank9809 +9810 POINT(40.81981303326757 73.18219840425085) bank9810 +9811 POINT(40.28048111419641 73.7798619168555) bank9811 +9812 POINT(40.266551681538644 73.64767089221156) bank9812 +9813 POINT(41.46330140035598 73.28708419892763) bank9813 +9814 POINT(40.512943147553386 74.18347014910034) bank9814 +9815 POINT(41.070900919703675 74.0291908606271) bank9815 +9816 POINT(39.880359058180915 73.73034118069667) bank9816 +9817 POINT(41.03462042087043 73.58865071414584) bank9817 +9818 POINT(41.62339028007704 73.65126043318811) bank9818 +9819 POINT(40.81583900610074 73.77389034686742) bank9819 +9820 POINT(40.33436143461749 74.45273863256593) bank9820 +9821 POINT(41.334480498204066 74.25515032486801) bank9821 +9822 POINT(41.004578789562316 74.88054013853605) bank9822 +9823 POINT(41.217278282939475 73.60179267852621) bank9823 +9824 POINT(40.338717325098436 73.56605160838895) bank9824 +9825 POINT(39.85321104750426 73.94669247369326) bank9825 +9826 POINT(40.414612424383115 74.98666564141618) bank9826 +9827 POINT(41.25730460445917 73.45256198932584) bank9827 +9828 POINT(39.90937909926718 73.96688830075378) bank9828 +9829 POINT(40.164224872741805 74.50856851242045) bank9829 +9830 POINT(41.07076482689861 74.64169166760881) bank9830 +9831 POINT(40.67955409911477 73.23424941669956) bank9831 +9832 POINT(40.228740798130424 74.65402282379169) bank9832 +9833 POINT(41.66037573918392 73.1826767365669) bank9833 +9834 POINT(40.239795252886495 74.76650072832435) bank9834 +9835 POINT(41.46125375215705 74.53933586531966) bank9835 +9836 POINT(40.83930975767624 73.21864358405433) bank9836 +9837 POINT(39.72938895124835 74.75361226065469) bank9837 +9838 POINT(41.362762004834174 73.14542083322691) bank9838 +9839 POINT(40.70932207717249 73.87911450472694) bank9839 +9840 POINT(41.64334781714535 74.8682565066389) bank9840 +9841 POINT(39.857523781352604 73.74206834672552) bank9841 +9842 POINT(40.61994310238342 73.75996462909025) bank9842 +9843 POINT(40.22215120655256 73.76734389407586) bank9843 +9844 POINT(41.22476389607029 74.15749143412323) bank9844 +9845 POINT(40.15842004806871 74.6745131775252) bank9845 +9846 POINT(41.47418635519857 74.26946936272437) bank9846 +9847 POINT(40.88378665790879 73.1062292805185) bank9847 +9848 POINT(40.002303981413256 73.2887559030102) bank9848 +9849 POINT(41.56094745258543 73.78339260939471) bank9849 +9850 POINT(40.552853076435326 74.49965380852439) bank9850 +9851 POINT(40.237668245674534 73.4275206552364) bank9851 +9852 POINT(40.89716955189353 73.58676970773529) bank9852 +9853 POINT(41.055163310670366 74.16437025543414) bank9853 +9854 POINT(41.064072200335545 74.98793594341134) bank9854 +9855 POINT(40.565350517161235 74.9528300969705) bank9855 +9856 POINT(40.80177016344614 73.2929084952223) bank9856 +9857 POINT(41.44900338133974 73.29334227515368) bank9857 +9858 POINT(40.913643499177695 74.15034185187542) bank9858 +9859 POINT(40.71256153321523 74.60063753456441) bank9859 +9860 POINT(41.49876570554822 73.04038316598722) bank9860 +9861 POINT(40.5315820503512 74.16410455234735) bank9861 +9862 POINT(40.097454013807415 74.92316957764233) bank9862 +9863 POINT(39.856953555480096 73.95670716858335) bank9863 +9864 POINT(40.73804949566772 73.77033300638418) bank9864 +9865 POINT(41.59174617607875 74.22101070279341) bank9865 +9866 POINT(39.718016289557134 74.18819361062896) bank9866 +9867 POINT(39.96173071148584 73.45119258822677) bank9867 +9868 POINT(41.417333266599464 74.82381855186968) bank9868 +9869 POINT(40.52586368761744 73.996166323874) bank9869 +9870 POINT(40.27801298503219 73.68610587958254) bank9870 +9871 POINT(40.26161583858964 74.5158803739395) bank9871 +9872 POINT(41.00542498195732 73.84356598461014) bank9872 +9873 POINT(40.47506164317041 74.99480985164575) bank9873 +9874 POINT(40.27277294953512 74.73161146267599) bank9874 +9875 POINT(40.41616514158113 73.2604335588523) bank9875 +9876 POINT(41.382596038683786 74.00873393047144) bank9876 +9877 POINT(40.42262219071371 73.44848262406475) bank9877 +9878 POINT(40.775104910834855 73.86260486891325) bank9878 +9879 POINT(40.16991821378274 73.07966196166626) bank9879 +9880 POINT(40.826838670074196 74.92219276344298) bank9880 +9881 POINT(41.632033814587665 74.74331770034345) bank9881 +9882 POINT(40.27842997029583 74.87081996397768) bank9882 +9883 POINT(40.71158911212938 74.38051573800288) bank9883 +9884 POINT(40.573454232101696 73.19158371473968) bank9884 +9885 POINT(40.469740976012545 73.09423156899791) bank9885 +9886 POINT(41.70121041747457 73.7656980298263) bank9886 +9887 POINT(41.17909553213498 73.26370879009882) bank9887 +9888 POINT(41.4968613417928 74.06564688027647) bank9888 +9889 POINT(41.66287829973906 73.56662764311571) bank9889 +9890 POINT(40.86631050900818 73.81016092294468) bank9890 +9891 POINT(40.82146992854421 74.75820817245825) bank9891 +9892 POINT(41.570456925026896 74.64759526925828) bank9892 +9893 POINT(41.59482128270529 73.12140783829739) bank9893 +9894 POINT(40.42022428225555 73.96264553609589) bank9894 +9895 POINT(40.38338335602498 74.23158469619577) bank9895 +9896 POINT(41.61580945584667 74.89003091920723) bank9896 +9897 POINT(40.42713502735794 73.37049802650549) bank9897 +9898 POINT(40.62055582460625 73.92469866969977) bank9898 +9899 POINT(41.08749605213049 73.0834580724267) bank9899 +9900 POINT(41.50733782873806 74.0813970149454) bank9900 +9901 POINT(40.88018334477576 74.06899950938283) bank9901 +9902 POINT(41.454122087507116 73.48383823595543) bank9902 +9903 POINT(41.11025176896242 73.90452606479748) bank9903 +9904 POINT(40.605123386957004 73.12154295704333) bank9904 +9905 POINT(40.812402254159885 73.17457531143461) bank9905 +9906 POINT(41.30689543292472 74.93855564333633) bank9906 +9907 POINT(40.607983193587955 73.40408138289033) bank9907 +9908 POINT(41.22723571877412 74.15048685004261) bank9908 +9909 POINT(41.24833755167883 74.21849578900787) bank9909 +9910 POINT(39.87514179424099 73.44643899157226) bank9910 +9911 POINT(40.14041416620517 74.44561437597127) bank9911 +9912 POINT(40.26447976419274 74.6781574853313) bank9912 +9913 POINT(40.88147879404548 73.27968335596002) bank9913 +9914 POINT(40.42419967230277 73.60275010614718) bank9914 +9915 POINT(41.6383043324026 74.29620468882311) bank9915 +9916 POINT(40.30881646851431 74.01590079097683) bank9916 +9917 POINT(40.88227411497187 74.59252177804018) bank9917 +9918 POINT(40.73471277136859 73.90041302330948) bank9918 +9919 POINT(40.33567227859711 74.0376802178725) bank9919 +9920 POINT(40.28561683666903 73.32186302995947) bank9920 +9921 POINT(40.903482436980056 73.2277787167572) bank9921 +9922 POINT(41.27840858149708 74.17456837663798) bank9922 +9923 POINT(39.954497720517594 74.62039261513819) bank9923 +9924 POINT(41.51549713314318 74.91355541577516) bank9924 +9925 POINT(41.35410755583134 74.57169993564509) bank9925 +9926 POINT(41.19438643229758 74.46749014091236) bank9926 +9927 POINT(40.215236360117004 73.32790705802627) bank9927 +9928 POINT(40.43490020951961 74.71436133158755) bank9928 +9929 POINT(39.76807825506515 73.9849783680863) bank9929 +9930 POINT(41.24774366933871 74.30028728012181) bank9930 +9931 POINT(40.36204275471955 73.11431808275066) bank9931 +9932 POINT(39.837924336270824 73.42814798862338) bank9932 +9933 POINT(40.16828230735757 73.39884681337801) bank9933 +9934 POINT(40.11204876332041 74.76234854913889) bank9934 +9935 POINT(40.90497121943566 74.74665506486475) bank9935 +9936 POINT(40.37758805854531 73.67062037618321) bank9936 +9937 POINT(40.38820732054923 74.70475966591681) bank9937 +9938 POINT(40.477659186378496 73.1082095609844) bank9938 +9939 POINT(39.816356485475346 74.34144202606323) bank9939 +9940 POINT(39.96653659260287 74.33492582366934) bank9940 +9941 POINT(39.760409767209296 74.43491167604716) bank9941 +9942 POINT(39.89888855384481 74.31328564471804) bank9942 +9943 POINT(40.47854107661693 74.78159555539159) bank9943 +9944 POINT(39.80607962762336 73.92110161296237) bank9944 +9945 POINT(39.780635475165305 74.48674662661325) bank9945 +9946 POINT(40.19869119532022 73.69676145423668) bank9946 +9947 POINT(40.160966550479024 73.38727160633665) bank9947 +9948 POINT(40.04005684320065 74.79681859763235) bank9948 +9949 POINT(39.91859687266841 74.00302005158478) bank9949 +9950 POINT(41.07142461721566 74.07652828351634) bank9950 +9951 POINT(40.33594151246022 73.75285149346097) bank9951 +9952 POINT(41.18499617188965 74.85949665154556) bank9952 +9953 POINT(39.80625178926359 74.12118091135656) bank9953 +9954 POINT(41.44037708980981 73.86828620847916) bank9954 +9955 POINT(40.087899331507145 74.93297971861526) bank9955 +9956 POINT(40.94934854952089 73.5077939561835) bank9956 +9957 POINT(39.87823449443722 74.50430878227951) bank9957 +9958 POINT(40.69763376176328 74.02857983011343) bank9958 +9959 POINT(39.812334939695425 74.56817340065194) bank9959 +9960 POINT(41.56745018027971 73.41819674368416) bank9960 +9961 POINT(40.60157399720768 73.36567649440141) bank9961 +9962 POINT(39.71810871618031 74.08753758328284) bank9962 +9963 POINT(40.92564273174907 73.9164188095715) bank9963 +9964 POINT(41.16172570107819 73.95028925996284) bank9964 +9965 POINT(40.142405103790146 73.10925524860751) bank9965 +9966 POINT(41.29269993299386 74.46623153785137) bank9966 +9967 POINT(41.01858440840775 73.54381669779056) bank9967 +9968 POINT(39.85105727827594 74.60710522648152) bank9968 +9969 POINT(41.28206942490087 73.25325217520955) bank9969 +9970 POINT(40.684745439177334 73.18144387024658) bank9970 +9971 POINT(40.8450210470246 73.69623250165272) bank9971 +9972 POINT(41.27604532403809 73.2570126018918) bank9972 +9973 POINT(40.01097801810978 74.80898262338715) bank9973 +9974 POINT(41.383747269502685 73.10522171019915) bank9974 +9975 POINT(40.981202268551584 73.1393931329605) bank9975 +9976 POINT(41.38455740727912 73.51027656739356) bank9976 +9977 POINT(40.20694231657178 73.5594937418645) bank9977 +9978 POINT(40.3840266787274 73.82893456450503) bank9978 +9979 POINT(40.40116978793886 73.46598927071683) bank9979 +9980 POINT(41.69916671048818 73.55209064302305) bank9980 +9981 POINT(39.772492057759436 74.52003225453682) bank9981 +9982 POINT(41.53878023850207 73.10315552440426) bank9982 +9983 POINT(41.484068462755005 74.44973559043432) bank9983 +9984 POINT(40.87962774798961 74.69059745102203) bank9984 +9985 POINT(40.3995844042005 73.53291681909798) bank9985 +9986 POINT(41.348076413111784 73.152454383003) bank9986 +9987 POINT(40.17883971030205 74.68738791718165) bank9987 +9988 POINT(40.81189253402061 74.47475172627648) bank9988 +9989 POINT(40.81883461601532 74.44661705081563) bank9989 +9990 POINT(40.195774550127716 73.64024802880644) bank9990 +9991 POINT(41.62702256647065 73.05249338306164) bank9991 +9992 POINT(40.625711875460496 73.76428649520213) bank9992 +9993 POINT(40.79380811958711 73.99696503473798) bank9993 +9994 POINT(40.05383755478768 73.21104850882708) bank9994 +9995 POINT(40.50109333110891 73.67943722157852) bank9995 +9996 POINT(41.070570892707565 74.52236771782637) bank9996 +9997 POINT(40.711123469570076 74.4087302642922) bank9997 +9998 POINT(41.34200064752556 74.19259889068995) bank9998 +9999 POINT(41.214877429264924 73.24005249708391) bank9999 +10000 POINT(40.46413315290752 74.87966695883998) bank10000 +10001 POINT(41.563225626209366 74.76860105722392) bank10001 +10002 POINT(41.06789118955947 74.3435624660799) bank10002 +10003 POINT(41.51592113485854 73.23834493354342) bank10003 +10004 POINT(41.17887756792343 74.38993276752387) bank10004 +10005 POINT(41.373183396394296 73.47750968900895) bank10005 +10006 POINT(39.80703875984137 74.41830326570955) bank10006 +10007 POINT(40.97396799151484 74.0769048258872) bank10007 +10008 POINT(41.238527437413886 73.42229810068862) bank10008 +10009 POINT(41.18520997805303 73.7037419609283) bank10009 +10010 POINT(40.97249252175737 74.7692295510395) bank10010 +10011 POINT(39.74477218780414 74.1764584677753) bank10011 +10012 POINT(41.19416704380169 73.51724382582039) bank10012 +10013 POINT(39.74185086588434 73.3522832940213) bank10013 +10014 POINT(41.182782822057106 74.9880410620237) bank10014 +10015 POINT(39.82730082541973 74.75158030942252) bank10015 +10016 POINT(40.32369898861258 73.89591648949826) bank10016 +10017 POINT(40.026905330089114 73.10417394634389) bank10017 +10018 POINT(41.063352145989846 74.39578237686334) bank10018 +10019 POINT(39.79850030092909 73.95965598520307) bank10019 +10020 POINT(41.56451476121304 74.42975525451195) bank10020 +10021 POINT(40.07362013192185 74.02540192647743) bank10021 +10022 POINT(41.01264110231997 74.74742691935934) bank10022 +10023 POINT(40.18302082970167 74.29984324019054) bank10023 +10024 POINT(41.22708395431268 74.96360692896381) bank10024 +10025 POINT(39.9948830872602 74.81856597287559) bank10025 +10026 POINT(40.07771316241472 74.05418320350111) bank10026 +10027 POINT(40.194283174405186 74.91185254557219) bank10027 +10028 POINT(40.12480147504381 74.67762930039414) bank10028 +10029 POINT(40.85246840414259 73.55709933736) bank10029 +10030 POINT(39.870264613677776 74.9231303890113) bank10030 +10031 POINT(41.627148667398025 73.87677260742072) bank10031 +10032 POINT(40.35290723021792 73.47185567085988) bank10032 +10033 POINT(40.55017413379122 73.12229552825983) bank10033 +10034 POINT(41.1197746972698 73.29811294287806) bank10034 +10035 POINT(39.9210431994946 73.27954440061984) bank10035 +10036 POINT(40.608614844990825 73.53948294675232) bank10036 +10037 POINT(41.2067344632822 74.490651635583) bank10037 +10038 POINT(41.34170101057196 74.63028704815858) bank10038 +10039 POINT(40.794560466002224 73.03371577006835) bank10039 +10040 POINT(40.47753513537276 74.2774232052382) bank10040 +10041 POINT(41.03443907690468 74.17754432556283) bank10041 +10042 POINT(41.47499315521194 74.16822743571296) bank10042 +10043 POINT(41.477139273195554 73.71342417361632) bank10043 +10044 POINT(40.831944507895784 74.57763562321145) bank10044 +10045 POINT(39.795130140742614 73.94166804806778) bank10045 +10046 POINT(41.593553289494366 73.8149472727462) bank10046 +10047 POINT(40.039903030129 74.42925873005983) bank10047 +10048 POINT(39.813163254427096 73.21464878143959) bank10048 +10049 POINT(41.175786222026154 73.10185692038849) bank10049 +10050 POINT(39.82272879596855 73.68168329240282) bank10050 +10051 POINT(40.037595491664916 73.26591000719071) bank10051 +10052 POINT(40.52099202489258 73.20113274071254) bank10052 +10053 POINT(39.914864159099224 74.4711451528595) bank10053 +10054 POINT(40.65714185859847 74.02411483602822) bank10054 +10055 POINT(40.630103039317426 74.91727703330383) bank10055 +10056 POINT(41.622644065438806 73.37010987174604) bank10056 +10057 POINT(41.698743100464824 73.6576425897405) bank10057 +10058 POINT(41.31640724421144 73.48923071243452) bank10058 +10059 POINT(41.44538481492408 74.514560473216) bank10059 +10060 POINT(39.815703966826916 74.77814799760557) bank10060 +10061 POINT(40.02166244690099 74.36187711621969) bank10061 +10062 POINT(40.995274958037164 74.91602954679544) bank10062 +10063 POINT(41.04858295906395 74.11598212765755) bank10063 +10064 POINT(39.98874047798191 74.381033985397) bank10064 +10065 POINT(40.26328706132553 74.36908680901605) bank10065 +10066 POINT(41.33566749123876 74.62083877857575) bank10066 +10067 POINT(40.91905437241644 73.59643574889867) bank10067 +10068 POINT(40.90812836452736 74.10518786297284) bank10068 +10069 POINT(41.644889079568074 74.30218521187298) bank10069 +10070 POINT(40.216281287151865 74.68762946672896) bank10070 +10071 POINT(40.499829043174685 73.27016886421255) bank10071 +10072 POINT(40.804626974047366 74.17152794300392) bank10072 +10073 POINT(40.69502743928118 73.04349018633174) bank10073 +10074 POINT(41.115588441979085 74.0408904313544) bank10074 +10075 POINT(41.45160080904045 74.53604363500475) bank10075 +10076 POINT(41.01238805911456 74.99039492146366) bank10076 +10077 POINT(40.68419963015109 73.58311291572475) bank10077 +10078 POINT(40.76074640262063 73.84989971598925) bank10078 +10079 POINT(40.741259511813325 73.07983642104298) bank10079 +10080 POINT(41.340194981000074 74.75072399539728) bank10080 +10081 POINT(40.03344571171163 73.02008076822189) bank10081 +10082 POINT(40.4267081415102 74.19990394205769) bank10082 +10083 POINT(41.37811640211 74.60885982900676) bank10083 +10084 POINT(41.25565238229555 73.75427568701335) bank10084 +10085 POINT(41.46254534836171 73.98912536415943) bank10085 +10086 POINT(40.160693784392244 73.01673000430094) bank10086 +10087 POINT(41.00986821410128 73.23330202080902) bank10087 +10088 POINT(40.26566701233653 73.41267779875108) bank10088 +10089 POINT(41.05515765870224 73.89195638442735) bank10089 +10090 POINT(41.4467658632849 73.81047029045156) bank10090 +10091 POINT(40.142842619791686 73.54716713673832) bank10091 +10092 POINT(41.614216084944076 73.09789494852693) bank10092 +10093 POINT(41.67323795745927 74.11108126684663) bank10093 +10094 POINT(41.24071768867708 73.3792840880543) bank10094 +10095 POINT(41.0466001638567 73.73017872090233) bank10095 +10096 POINT(40.66542819170686 74.38992017196205) bank10096 +10097 POINT(39.717467510548296 74.19725986442248) bank10097 +10098 POINT(40.662634723702034 74.38341635585535) bank10098 +10099 POINT(40.34713511481825 73.18688387601813) bank10099 +10100 POINT(40.380652321215265 74.7974739960356) bank10100 +10101 POINT(40.80416021271417 74.35296849118215) bank10101 +10102 POINT(40.11231168021326 74.0631008177988) bank10102 +10103 POINT(41.5003622687148 73.29644454085349) bank10103 +10104 POINT(40.957163836541724 74.57245189471355) bank10104 +10105 POINT(41.076704103841244 73.05276137023321) bank10105 +10106 POINT(41.47015278103337 74.76199537385018) bank10106 +10107 POINT(41.122068280574695 73.54070613444529) bank10107 +10108 POINT(41.26276353469108 73.33565533544055) bank10108 +10109 POINT(41.42578030912092 74.78985683895608) bank10109 +10110 POINT(41.08339135862975 73.36532810056877) bank10110 +10111 POINT(40.41233397637645 74.47490924161868) bank10111 +10112 POINT(41.222155609899964 73.13278494398216) bank10112 +10113 POINT(39.83962076985758 74.52950186941368) bank10113 +10114 POINT(39.965730133537285 73.22383939545544) bank10114 +10115 POINT(41.24688224352991 74.80594056016794) bank10115 +10116 POINT(41.08125339314926 74.82347324496465) bank10116 +10117 POINT(41.45502299436915 74.03670147980341) bank10117 +10118 POINT(40.10107519612492 73.20460583381278) bank10118 +10119 POINT(40.480250176075714 73.58545201110516) bank10119 +10120 POINT(40.886330203012434 74.8618494399572) bank10120 +10121 POINT(39.99264582648159 74.18784472143706) bank10121 +10122 POINT(41.40384119874208 74.6382733651925) bank10122 +10123 POINT(40.01272477567306 73.10989639774584) bank10123 +10124 POINT(40.207122567215315 74.95991085463069) bank10124 +10125 POINT(41.08075015876807 74.29968402578027) bank10125 +10126 POINT(40.56855582216301 73.20581807828314) bank10126 +10127 POINT(40.49805348863971 73.02800954522309) bank10127 +10128 POINT(40.39356061774739 73.07318948798583) bank10128 +10129 POINT(41.061590191593105 74.78989984119151) bank10129 +10130 POINT(40.02336902649843 73.51909414200745) bank10130 +10131 POINT(41.47318725034592 73.31207791283477) bank10131 +10132 POINT(40.22549323561169 74.68977852714964) bank10132 +10133 POINT(40.23273531890534 74.55715192620823) bank10133 +10134 POINT(40.724699190758116 74.87624007526784) bank10134 +10135 POINT(41.29018301745099 74.00165295178509) bank10135 +10136 POINT(41.66129341307985 73.95416811931868) bank10136 +10137 POINT(41.235452318790145 74.11288000702045) bank10137 +10138 POINT(40.52131699816017 73.92752664381348) bank10138 +10139 POINT(41.493511102387984 74.18316633872055) bank10139 +10140 POINT(39.96645419306374 74.37491374144959) bank10140 +10141 POINT(40.640245462433356 73.69193623917583) bank10141 +10142 POINT(39.8360612904379 74.08753034341855) bank10142 +10143 POINT(40.20780697733695 74.02756631986675) bank10143 +10144 POINT(41.48851264871617 73.8215389353673) bank10144 +10145 POINT(41.481708561082584 73.5433347220534) bank10145 +10146 POINT(40.3187846082884 74.91080450692584) bank10146 +10147 POINT(40.45329571300453 74.46443251307093) bank10147 +10148 POINT(39.97757212442437 73.00651375920633) bank10148 +10149 POINT(39.826735382497674 73.20813012026667) bank10149 +10150 POINT(39.73464315995795 73.98953009472505) bank10150 +10151 POINT(40.07398802577175 74.71261344475013) bank10151 +10152 POINT(41.25398279763726 73.77383184467938) bank10152 +10153 POINT(40.66093188551891 73.68933716528008) bank10153 +10154 POINT(40.02168769352634 74.55750582766764) bank10154 +10155 POINT(40.18931646944205 74.55137456461152) bank10155 +10156 POINT(40.37531790315933 73.43374103907443) bank10156 +10157 POINT(40.620608354400005 73.97427241790308) bank10157 +10158 POINT(40.98865183447001 73.32362649628669) bank10158 +10159 POINT(40.727452074632296 73.55880253003845) bank10159 +10160 POINT(41.68557534553847 74.27260114054197) bank10160 +10161 POINT(40.71028592823775 74.31166159188015) bank10161 +10162 POINT(39.87382180614486 74.9185978797453) bank10162 +10163 POINT(40.815033463515434 73.44994607514184) bank10163 +10164 POINT(41.693538661784025 74.70365320775117) bank10164 +10165 POINT(40.72583184977463 74.37008726011449) bank10165 +10166 POINT(39.90128583700922 73.60598263836816) bank10166 +10167 POINT(40.16684767606843 73.98654899546563) bank10167 +10168 POINT(40.256078651439985 74.38514005217105) bank10168 +10169 POINT(40.4425812280414 73.58816013842423) bank10169 +10170 POINT(41.64868973851314 74.04968265129114) bank10170 +10171 POINT(41.165855810216684 74.05907729247521) bank10171 +10172 POINT(39.768515825559035 73.32977795782651) bank10172 +10173 POINT(41.39146498512651 74.80099204072548) bank10173 +10174 POINT(41.31428639004473 74.33183051166877) bank10174 +10175 POINT(41.39855640516306 74.3810348090706) bank10175 +10176 POINT(40.99545914659027 74.91503951294287) bank10176 +10177 POINT(39.90688690674435 74.92970050063175) bank10177 +10178 POINT(39.80128259354451 73.33040069214486) bank10178 +10179 POINT(40.4200252993232 73.04257388519905) bank10179 +10180 POINT(41.127352954757825 73.41949571094155) bank10180 +10181 POINT(40.75150200909283 74.83910441658051) bank10181 +10182 POINT(41.533054775973525 73.0461980597369) bank10182 +10183 POINT(40.42347539220107 73.70688919059967) bank10183 +10184 POINT(40.11946538320727 74.50741415111328) bank10184 +10185 POINT(41.12832607018066 74.09508648489107) bank10185 +10186 POINT(40.179644085999506 73.6888310051882) bank10186 +10187 POINT(41.37735309099974 73.41928510642074) bank10187 +10188 POINT(41.01839400445625 74.97083760721166) bank10188 +10189 POINT(41.35349751515163 74.74148421490183) bank10189 +10190 POINT(41.45693910642768 74.4448917613083) bank10190 +10191 POINT(39.90694960523838 73.2844146841116) bank10191 +10192 POINT(40.19905481418955 74.05261033992285) bank10192 +10193 POINT(41.00890672131927 74.46371807577587) bank10193 +10194 POINT(40.790361660615005 73.37680280761866) bank10194 +10195 POINT(41.572375618741795 73.2993864216121) bank10195 +10196 POINT(40.64259379183424 74.03902809322412) bank10196 +10197 POINT(40.33405133149957 73.34760026223793) bank10197 +10198 POINT(41.230664536645385 74.02644882734006) bank10198 +10199 POINT(40.180709870084215 73.41613551121702) bank10199 +10200 POINT(40.982051505793045 74.09791900420589) bank10200 +10201 POINT(39.74410438276385 73.84377045363973) bank10201 +10202 POINT(40.09223058857997 74.54916161353817) bank10202 +10203 POINT(41.29502646560379 73.45239152823788) bank10203 +10204 POINT(40.1897310019318 73.2958903445605) bank10204 +10205 POINT(40.765982252449874 73.21475767600036) bank10205 +10206 POINT(41.27913003344124 74.05373040534306) bank10206 +10207 POINT(40.55936818392669 74.17240092269891) bank10207 +10208 POINT(41.59078516062654 73.66638042326602) bank10208 +10209 POINT(40.223377745722594 74.63504605519978) bank10209 +10210 POINT(39.80575987201165 74.58009121613036) bank10210 +10211 POINT(40.38778188678496 74.561230919838) bank10211 +10212 POINT(40.61152472273989 74.58264934228424) bank10212 +10213 POINT(39.85452930759668 74.4049774180011) bank10213 +10214 POINT(39.778507516345556 74.38482280031299) bank10214 +10215 POINT(40.168706048216194 73.71600101146655) bank10215 +10216 POINT(41.532633946491394 73.62382176415444) bank10216 +10217 POINT(40.39770484700266 74.99723451515545) bank10217 +10218 POINT(40.600722794893514 74.97402939652449) bank10218 +10219 POINT(40.533978395774405 73.44215980788131) bank10219 +10220 POINT(41.61941903149884 74.07723211660873) bank10220 +10221 POINT(39.83301825174838 73.27967235419109) bank10221 +10222 POINT(41.32161992619925 73.24973132370846) bank10222 +10223 POINT(40.1320907038085 74.76071062352678) bank10223 +10224 POINT(40.773525008857604 74.25513019809394) bank10224 +10225 POINT(40.01982303015436 73.60074775497984) bank10225 +10226 POINT(40.53159080360919 74.51004049767575) bank10226 +10227 POINT(39.88354416200721 73.23803536367687) bank10227 +10228 POINT(41.07842601404807 74.97018059206484) bank10228 +10229 POINT(41.54482982747953 73.80446191874894) bank10229 +10230 POINT(40.31325195443187 73.18269190369513) bank10230 +10231 POINT(39.96702528983568 73.12819300764829) bank10231 +10232 POINT(40.88315893559404 74.68525040152248) bank10232 +10233 POINT(40.760370360784506 73.63861964076685) bank10233 +10234 POINT(40.758654106749105 74.96412166607278) bank10234 +10235 POINT(40.166706112295174 74.27422418161177) bank10235 +10236 POINT(41.69737234326677 73.49719273667115) bank10236 +10237 POINT(39.819377522253916 74.31475999840077) bank10237 +10238 POINT(41.51410238577707 73.22052236646408) bank10238 +10239 POINT(39.899369633297 74.41383682454806) bank10239 +10240 POINT(41.00909338420159 73.23336107126015) bank10240 +10241 POINT(40.02341507274556 74.6770057350947) bank10241 +10242 POINT(39.893216853561945 74.24740074646971) bank10242 +10243 POINT(41.04110883844026 74.68808610012614) bank10243 +10244 POINT(39.76940740816252 74.8041815907725) bank10244 +10245 POINT(40.52193351807351 73.03244638058995) bank10245 +10246 POINT(40.97319474956128 74.06572326272118) bank10246 +10247 POINT(39.87224546996918 74.6897857872854) bank10247 +10248 POINT(39.87391617228974 74.6452541013999) bank10248 +10249 POINT(40.420706407578976 73.13616483455505) bank10249 +10250 POINT(41.133193855676986 74.89284905377778) bank10250 +10251 POINT(41.473979278594754 73.9367311109327) bank10251 +10252 POINT(39.89391481613231 73.7233370580493) bank10252 +10253 POINT(39.85714713719337 74.08533743020254) bank10253 +10254 POINT(39.76221425626256 74.62085507810801) bank10254 +10255 POINT(40.04506447884374 74.14269019494402) bank10255 +10256 POINT(40.85149499166839 74.68712578027251) bank10256 +10257 POINT(40.459026417646264 73.66775907571103) bank10257 +10258 POINT(40.32124061181136 73.89639730219855) bank10258 +10259 POINT(39.96348973028415 73.49064972905782) bank10259 +10260 POINT(40.01968350535869 73.47130637794547) bank10260 +10261 POINT(40.1298012835681 74.1068723078524) bank10261 +10262 POINT(39.909204659081624 73.96553077565055) bank10262 +10263 POINT(41.38414849684548 73.41610805601977) bank10263 +10264 POINT(40.27140991723213 73.29998867179717) bank10264 +10265 POINT(40.334410331975235 73.4165778970446) bank10265 +10266 POINT(40.65162078364871 74.2711404371995) bank10266 +10267 POINT(40.02915321729172 74.25110287270539) bank10267 +10268 POINT(41.617915492164144 74.42833060626499) bank10268 +10269 POINT(40.79110640700239 73.35306766730127) bank10269 +10270 POINT(41.518858998149895 73.44451879481073) bank10270 +10271 POINT(40.14456316180809 73.34232526010902) bank10271 +10272 POINT(41.1201081561626 73.57659923614156) bank10272 +10273 POINT(41.577064996634796 73.38778742963069) bank10273 +10274 POINT(41.2290759762836 74.85866073301533) bank10274 +10275 POINT(41.686197393882004 73.35459773591535) bank10275 +10276 POINT(40.83472644644669 73.09242092814513) bank10276 +10277 POINT(41.62523127607803 73.8657247236292) bank10277 +10278 POINT(40.37710591544375 74.55552866884375) bank10278 +10279 POINT(40.43015565320792 73.09138937732169) bank10279 +10280 POINT(41.29716879772382 73.48814109669529) bank10280 +10281 POINT(40.45821649518563 73.0672678212095) bank10281 +10282 POINT(41.59481271431985 73.72784154048922) bank10282 +10283 POINT(40.75867500767152 73.28543021714336) bank10283 +10284 POINT(41.38991445092859 73.59390583014202) bank10284 +10285 POINT(40.92000651853313 73.19300767111508) bank10285 +10286 POINT(40.39573162421079 74.32045153545756) bank10286 +10287 POINT(40.085610199211196 74.3920423940229) bank10287 +10288 POINT(40.53878748179323 74.89742908335624) bank10288 +10289 POINT(40.56815106702232 74.4509772958962) bank10289 +10290 POINT(40.19890440458357 74.0265336178731) bank10290 +10291 POINT(39.94988713204857 74.19955105171802) bank10291 +10292 POINT(41.54198894760901 74.65515545030607) bank10292 +10293 POINT(40.638050536186675 73.40480459701463) bank10293 +10294 POINT(40.68619773324895 73.50100156810004) bank10294 +10295 POINT(40.421717294686545 74.84274484369453) bank10295 +10296 POINT(39.85187711506887 74.87508673285168) bank10296 +10297 POINT(41.52247218738532 73.40769958971052) bank10297 +10298 POINT(39.75517184899366 73.98386302966685) bank10298 +10299 POINT(41.33195057822176 74.81135048195138) bank10299 +10300 POINT(41.357887935442385 74.32747190962728) bank10300 +10301 POINT(40.35015325058292 73.56668558427202) bank10301 +10302 POINT(41.33544611589741 74.85587549652021) bank10302 +10303 POINT(40.91228862797179 74.39270047632488) bank10303 +10304 POINT(41.05125286390495 73.8241208544695) bank10304 +10305 POINT(41.70926019299362 74.91078908621664) bank10305 +10306 POINT(39.88240794767136 74.28116659627426) bank10306 +10307 POINT(41.522647237339676 73.82496765676026) bank10307 +10308 POINT(39.82282828767231 74.38500129731501) bank10308 +10309 POINT(40.453442356328495 73.99704396190089) bank10309 +10310 POINT(39.807983132286815 73.5436727426808) bank10310 +10311 POINT(41.62992734701239 74.49152054850745) bank10311 +10312 POINT(40.242314690414815 73.59647786561477) bank10312 +10313 POINT(40.458733365925205 74.59072340061871) bank10313 +10314 POINT(41.20458536431682 73.65213240261572) bank10314 +10315 POINT(40.49322702661275 74.98708717720598) bank10315 +10316 POINT(40.54038672754425 74.12140694049411) bank10316 +10317 POINT(40.05443103862874 74.10799418396431) bank10317 +10318 POINT(39.96884223038756 73.16405673114832) bank10318 +10319 POINT(39.78272458108538 74.64887626189467) bank10319 +10320 POINT(40.553588722978624 73.936281916858) bank10320 +10321 POINT(41.509693783139134 74.78686519116995) bank10321 +10322 POINT(40.31955512706299 74.5221159787024) bank10322 +10323 POINT(39.7248497851314 74.58076675314557) bank10323 +10324 POINT(41.27797097970871 73.77960364301812) bank10324 +10325 POINT(39.71286596137004 73.41865621629076) bank10325 +10326 POINT(41.51955781476422 74.71776026143152) bank10326 +10327 POINT(40.41414917731106 73.67567644899667) bank10327 +10328 POINT(41.64878752592756 74.03200590509786) bank10328 +10329 POINT(40.51218417961394 73.62438901002199) bank10329 +10330 POINT(41.66510799586852 73.97743232551906) bank10330 +10331 POINT(40.34277400074166 74.25042728331773) bank10331 +10332 POINT(40.79666307260497 73.35087416026165) bank10332 +10333 POINT(39.877492636047265 73.9498449536321) bank10333 +10334 POINT(41.40415900836581 73.93213599788557) bank10334 +10335 POINT(39.72310003312754 74.52741859941366) bank10335 +10336 POINT(40.210073261330365 74.629348873551) bank10336 +10337 POINT(39.87722641148946 73.90517634713704) bank10337 +10338 POINT(40.14398113878312 74.5852920003839) bank10338 +10339 POINT(39.882976664375526 74.59066619194626) bank10339 +10340 POINT(41.44594250981747 74.6089052456344) bank10340 +10341 POINT(41.20459736983103 74.73455169254662) bank10341 +10342 POINT(41.2587350810278 74.88783346835595) bank10342 +10343 POINT(41.370404651398005 73.10055992370945) bank10343 +10344 POINT(39.82073630643841 73.44577788763078) bank10344 +10345 POINT(39.9271094776868 74.34808328681324) bank10345 +10346 POINT(41.41677328296736 73.54393592901424) bank10346 +10347 POINT(40.53004585184615 73.98625310678463) bank10347 +10348 POINT(40.52306591021891 74.45643007328977) bank10348 +10349 POINT(41.01081701199325 73.84041626891712) bank10349 +10350 POINT(41.563956820065364 74.93429086895661) bank10350 +10351 POINT(40.52615484640209 73.2079893966454) bank10351 +10352 POINT(40.545999469721494 74.27942864897902) bank10352 +10353 POINT(39.91836133667016 73.71226600447875) bank10353 +10354 POINT(41.45752560452833 73.808417091748) bank10354 +10355 POINT(40.10267112048458 73.07668584982602) bank10355 +10356 POINT(40.150737165421916 74.84026402650171) bank10356 +10357 POINT(40.211213137973004 74.81001432823206) bank10357 +10358 POINT(40.21754804533306 74.43286106790862) bank10358 +10359 POINT(40.55302126148293 73.25136997368483) bank10359 +10360 POINT(41.10464736921232 74.23877613024268) bank10360 +10361 POINT(40.07189056249541 74.92174433934326) bank10361 +10362 POINT(39.91442574544388 73.57761782188453) bank10362 +10363 POINT(39.879771596665535 74.66997971279446) bank10363 +10364 POINT(41.3152804653557 73.61494581532898) bank10364 +10365 POINT(40.08699935376996 74.85672173077452) bank10365 +10366 POINT(41.39484313299962 73.87883007250531) bank10366 +10367 POINT(39.96501720456851 73.71289530096534) bank10367 +10368 POINT(41.520182969781935 74.41244107972965) bank10368 +10369 POINT(40.20925747382706 74.82399683939153) bank10369 +10370 POINT(41.54005566790443 73.68857285713871) bank10370 +10371 POINT(41.31834507273761 74.97118561741085) bank10371 +10372 POINT(40.51770735017278 73.07602649332102) bank10372 +10373 POINT(40.55751949616548 74.52512967196475) bank10373 +10374 POINT(41.437383654796946 74.96136809770866) bank10374 +10375 POINT(41.393984922570134 74.75636555432924) bank10375 +10376 POINT(41.24275390178941 74.65081124382263) bank10376 +10377 POINT(40.88294227152808 73.58704205606932) bank10377 +10378 POINT(41.28987272123991 74.40474893759936) bank10378 +10379 POINT(40.22963379895379 74.46348506520948) bank10379 +10380 POINT(41.505373195750785 73.07438080957007) bank10380 +10381 POINT(40.16500038938437 73.69943983088821) bank10381 +10382 POINT(40.07001481629388 74.66719249605558) bank10382 +10383 POINT(41.54246893708869 74.01584573164546) bank10383 +10384 POINT(41.66628371042913 74.930606988886) bank10384 +10385 POINT(40.52024111934847 74.33823153732692) bank10385 +10386 POINT(40.63913645974144 74.82162021036478) bank10386 +10387 POINT(39.862005231886805 73.29637854772687) bank10387 +10388 POINT(40.62898943576751 73.02048404758133) bank10388 +10389 POINT(40.33153333009085 73.89496066135915) bank10389 +10390 POINT(39.892707933801034 73.26687296199636) bank10390 +10391 POINT(40.79106375473809 73.01592577199816) bank10391 +10392 POINT(41.47017404534549 74.92174450433143) bank10392 +10393 POINT(41.32129620997149 73.29473966742472) bank10393 +10394 POINT(40.33287559363648 74.76642628478628) bank10394 +10395 POINT(41.272348902967686 74.52698130351959) bank10395 +10396 POINT(39.9541183565595 74.23575190605548) bank10396 +10397 POINT(40.66219743983053 73.58239227987367) bank10397 +10398 POINT(40.118898776154865 74.80984030612665) bank10398 +10399 POINT(39.91846684149258 73.09927063134515) bank10399 +10400 POINT(41.52010334848627 74.08163126925375) bank10400 +10401 POINT(41.317516136614316 73.95493693113256) bank10401 +10402 POINT(40.22695467986768 74.51240839912931) bank10402 +10403 POINT(40.89389066537419 73.57061455943979) bank10403 +10404 POINT(41.492196474668006 73.05742062865589) bank10404 +10405 POINT(41.51327852922992 74.70660605204208) bank10405 +10406 POINT(41.63203230297236 74.54091436256111) bank10406 +10407 POINT(41.515018418026386 74.17085698051709) bank10407 +10408 POINT(40.831124736303664 74.25700120099914) bank10408 +10409 POINT(40.01570525178807 74.65692263880001) bank10409 +10410 POINT(41.63833709079183 74.68869850513025) bank10410 +10411 POINT(40.777132448475825 74.75464413601141) bank10411 +10412 POINT(40.64280870530967 74.331644774817) bank10412 +10413 POINT(39.74047948353576 74.21094464069375) bank10413 +10414 POINT(41.48975339211236 73.96174086777121) bank10414 +10415 POINT(41.13030912426157 73.80898292851984) bank10415 +10416 POINT(40.480021479888826 73.88820664549506) bank10416 +10417 POINT(40.89531479783506 73.49572922242866) bank10417 +10418 POINT(41.47236018070754 74.0984659275389) bank10418 +10419 POINT(40.934638907030774 74.56708259008012) bank10419 +10420 POINT(41.36168743719119 73.71647529414324) bank10420 +10421 POINT(40.138457115606286 73.9488366601394) bank10421 +10422 POINT(40.01717946609228 73.44978074435639) bank10422 +10423 POINT(41.16336732401415 73.81352146009428) bank10423 +10424 POINT(40.3627916708006 74.28740609871114) bank10424 +10425 POINT(40.5082830870418 73.99158587959023) bank10425 +10426 POINT(41.27913660032176 73.6091606288619) bank10426 +10427 POINT(40.42975716965317 74.27832633899249) bank10427 +10428 POINT(41.67050193053343 73.92406140787946) bank10428 +10429 POINT(41.20106751078667 74.06919921616532) bank10429 +10430 POINT(40.588348826862685 74.00329882167006) bank10430 +10431 POINT(40.57740453131439 73.86295619385581) bank10431 +10432 POINT(40.00008390455092 73.6767893713927) bank10432 +10433 POINT(40.28306879499892 73.1250072253546) bank10433 +10434 POINT(40.85133730197804 73.72122743730904) bank10434 +10435 POINT(40.24262377722898 74.23816199072405) bank10435 +10436 POINT(40.4458095165121 74.48669016754458) bank10436 +10437 POINT(40.729956777193436 74.11478338164561) bank10437 +10438 POINT(41.25202235372233 74.51849190427606) bank10438 +10439 POINT(39.86095316054109 73.08211725495767) bank10439 +10440 POINT(40.75398683594991 73.93978827452169) bank10440 +10441 POINT(40.6521932256471 73.43618242680111) bank10441 +10442 POINT(40.448376190145524 74.5638316375639) bank10442 +10443 POINT(40.2810433915749 73.19468005007417) bank10443 +10444 POINT(39.974884651088885 74.76614255636822) bank10444 +10445 POINT(40.78062342076755 74.0799129147635) bank10445 +10446 POINT(41.58739376898659 74.07773752105096) bank10446 +10447 POINT(40.71801587402475 73.0517557301128) bank10447 +10448 POINT(41.43087356041331 74.42383646459672) bank10448 +10449 POINT(40.494970800669634 74.34060507875083) bank10449 +10450 POINT(40.80260574642033 73.6137233881683) bank10450 +10451 POINT(41.647045915513786 73.02269406279494) bank10451 +10452 POINT(39.99317080996511 74.10864082459337) bank10452 +10453 POINT(41.590955188405545 74.4047078738031) bank10453 +10454 POINT(39.952446646113096 74.10896498524187) bank10454 +10455 POINT(40.566472061198006 74.23284576573464) bank10455 +10456 POINT(41.314905286129736 74.88170995854117) bank10456 +10457 POINT(41.63085081522781 74.98838662553162) bank10457 +10458 POINT(40.98045413490189 74.28427434421421) bank10458 +10459 POINT(40.72190960493424 74.82863374453142) bank10459 +10460 POINT(41.5755332849355 73.09762288631147) bank10460 +10461 POINT(40.964013365581216 74.00215436240457) bank10461 +10462 POINT(40.83675343537262 74.71540301312928) bank10462 +10463 POINT(40.54297364754708 74.49705801381054) bank10463 +10464 POINT(41.592755676693535 73.88485310698728) bank10464 +10465 POINT(39.85682528002609 73.7058421905598) bank10465 +10466 POINT(40.3214031406788 73.06500077714664) bank10466 +10467 POINT(39.99089274194409 74.83592501231728) bank10467 +10468 POINT(40.31521822671508 74.21814756483629) bank10468 +10469 POINT(41.18802800224966 74.485313563048) bank10469 +10470 POINT(40.00289977628336 73.87847800329317) bank10470 +10471 POINT(41.374351665905245 74.50581752612351) bank10471 +10472 POINT(39.74861173782553 73.33388623062224) bank10472 +10473 POINT(41.36550584467382 74.45088074337936) bank10473 +10474 POINT(41.060129808420854 74.1090789783964) bank10474 +10475 POINT(41.1930201258326 73.43483927050026) bank10475 +10476 POINT(40.811451329763834 74.62810004140279) bank10476 +10477 POINT(41.18139148939848 73.86490149583055) bank10477 +10478 POINT(40.01196361593773 74.43390742382871) bank10478 +10479 POINT(40.393652240220426 74.04246806238129) bank10479 +10480 POINT(41.14318788278413 73.52755771397104) bank10480 +10481 POINT(40.2129944558473 74.26985658584496) bank10481 +10482 POINT(40.9167112950207 74.31195129401651) bank10482 +10483 POINT(39.92270589706896 73.00660425943862) bank10483 +10484 POINT(40.732743558035004 74.39925870523511) bank10484 +10485 POINT(40.768736249429544 74.82792094000132) bank10485 +10486 POINT(39.97013348028332 73.59663659414973) bank10486 +10487 POINT(40.13580912193728 74.32660110960893) bank10487 +10488 POINT(40.447141162316775 73.3257953447682) bank10488 +10489 POINT(40.9819978061685 73.47208646331374) bank10489 +10490 POINT(41.16413342591416 73.78434497900092) bank10490 +10491 POINT(40.11449010228995 73.43339736430569) bank10491 +10492 POINT(41.18857330975685 73.86817533736311) bank10492 +10493 POINT(40.909715847342746 73.94631490435131) bank10493 +10494 POINT(40.254623078538486 73.36245888294606) bank10494 +10495 POINT(40.70051913844989 74.99564545225148) bank10495 +10496 POINT(40.3614590271047 74.81993222402059) bank10496 +10497 POINT(40.47747622356761 74.92331937883387) bank10497 +10498 POINT(40.29104581319312 73.42393797612074) bank10498 +10499 POINT(40.57091131244259 73.61813856629352) bank10499 +10500 POINT(39.927161281345406 74.84703712819528) bank10500 +10501 POINT(41.48746385465056 74.45126486960352) bank10501 +10502 POINT(41.316996348410974 74.3428224131716) bank10502 +10503 POINT(41.23809258205045 73.0263323512618) bank10503 +10504 POINT(40.36230694865341 73.84446533182908) bank10504 +10505 POINT(39.9436296773891 74.89250044840502) bank10505 +10506 POINT(41.19574201560268 74.35629831738481) bank10506 +10507 POINT(39.916772250874395 74.9000759666903) bank10507 +10508 POINT(40.574421537433224 73.46019886602198) bank10508 +10509 POINT(39.8245410672682 74.06041086101492) bank10509 +10510 POINT(40.61701167420948 73.23525042297648) bank10510 +10511 POINT(40.95586029530232 73.73420930149548) bank10511 +10512 POINT(40.58246725556447 74.76116614755051) bank10512 +10513 POINT(39.76228603095398 74.56395983581156) bank10513 +10514 POINT(41.2526789908086 74.49939661773965) bank10514 +10515 POINT(39.79592336740711 74.38671881624275) bank10515 +10516 POINT(40.5060752241711 74.9085729530721) bank10516 +10517 POINT(41.67896621159304 74.23025540827614) bank10517 +10518 POINT(40.16621153400703 73.17519566663019) bank10518 +10519 POINT(40.84918639940651 73.4680355126682) bank10519 +10520 POINT(40.946299425678646 74.7517787851973) bank10520 +10521 POINT(41.61306631957845 74.24745027815992) bank10521 +10522 POINT(40.38882139143992 74.68491805127913) bank10522 +10523 POINT(41.24851567171119 73.68983230653211) bank10523 +10524 POINT(41.480556077350094 74.53167348745876) bank10524 +10525 POINT(41.23796173177663 74.12217094402871) bank10525 +10526 POINT(39.78000046281207 74.96140522983647) bank10526 +10527 POINT(40.43503680012204 74.27003660596856) bank10527 +10528 POINT(40.58383340434706 74.80640849863713) bank10528 +10529 POINT(40.55282927279111 73.33603521901375) bank10529 +10530 POINT(40.33009131606035 74.36313000806801) bank10530 +10531 POINT(41.05726940389272 73.66911291510107) bank10531 +10532 POINT(39.82502244193748 73.36252224193775) bank10532 +10533 POINT(41.13496931930016 74.46169030300457) bank10533 +10534 POINT(41.0000188311838 74.8951454909206) bank10534 +10535 POINT(39.876031932076145 73.85085730654897) bank10535 +10536 POINT(40.98130080691655 74.83100814318618) bank10536 +10537 POINT(40.61884397913446 73.46060307850205) bank10537 +10538 POINT(40.2181540218681 74.02240317788885) bank10538 +10539 POINT(41.407258957208676 73.95564409157326) bank10539 +10540 POINT(40.90362582898404 73.42999448588989) bank10540 +10541 POINT(41.17362326777399 73.93404348718161) bank10541 +10542 POINT(39.72010659017186 74.23785991088641) bank10542 +10543 POINT(40.275874875108805 74.7249959402975) bank10543 +10544 POINT(41.36291306726298 74.8715482283876) bank10544 +10545 POINT(39.89099476171046 74.51331347139285) bank10545 +10546 POINT(41.53594860063254 74.95235049800502) bank10546 +10547 POINT(39.931854946283934 74.27749404847843) bank10547 +10548 POINT(41.27357343796201 74.62639590029055) bank10548 +10549 POINT(40.304675056402495 73.15598669967649) bank10549 +10550 POINT(41.43300013396759 73.63472160924358) bank10550 +10551 POINT(41.065878724224596 73.74237792036091) bank10551 +10552 POINT(40.514389311175265 73.2707087860163) bank10552 +10553 POINT(41.70464154303788 73.88884477007555) bank10553 +10554 POINT(40.35083235417692 73.22189187916571) bank10554 +10555 POINT(41.627751678908005 74.05254128335586) bank10555 +10556 POINT(41.27742598618442 73.64712900853277) bank10556 +10557 POINT(40.114657542023 74.27712422583264) bank10557 +10558 POINT(40.27175024654683 74.94563180458624) bank10558 +10559 POINT(40.762755732519175 73.89774746450091) bank10559 +10560 POINT(41.15717824962462 74.67928725899718) bank10560 +10561 POINT(41.53256768497834 74.4143505192014) bank10561 +10562 POINT(41.57102234737823 74.03460282289277) bank10562 +10563 POINT(40.10112648953712 73.64792276413601) bank10563 +10564 POINT(41.53122752036422 73.53706733853787) bank10564 +10565 POINT(39.84297232830238 73.2718520116716) bank10565 +10566 POINT(41.49473363537186 73.57244960577624) bank10566 +10567 POINT(40.73859696748087 74.59498006106385) bank10567 +10568 POINT(39.87387458691261 73.21515773532718) bank10568 +10569 POINT(40.84261841800489 74.63704339214846) bank10569 +10570 POINT(40.67150548712389 73.57545576548401) bank10570 +10571 POINT(40.52500224119431 73.04694672948692) bank10571 +10572 POINT(39.76874865402367 73.891518862353) bank10572 +10573 POINT(40.94249188098717 73.046992782237) bank10573 +10574 POINT(39.880691506878215 74.78277889488581) bank10574 +10575 POINT(41.293850628134386 73.49214468304032) bank10575 +10576 POINT(41.377304287148846 73.15230922773927) bank10576 +10577 POINT(40.53033350300219 74.32380136907776) bank10577 +10578 POINT(40.31904863459425 73.9817288647018) bank10578 +10579 POINT(40.7168563795569 73.5239775730679) bank10579 +10580 POINT(41.507647544183456 73.50872807904521) bank10580 +10581 POINT(41.69760314845631 74.9148533741806) bank10581 +10582 POINT(41.121131413870394 74.98791242545374) bank10582 +10583 POINT(40.90151474228411 73.10955628089073) bank10583 +10584 POINT(41.314391864777335 74.03138843733677) bank10584 +10585 POINT(39.76723241290414 73.83355085133714) bank10585 +10586 POINT(40.45926312273339 73.27124855702509) bank10586 +10587 POINT(40.69567581605312 74.426243326256) bank10587 +10588 POINT(40.728764206648066 73.05953026940847) bank10588 +10589 POINT(40.367042858457815 73.01064647472559) bank10589 +10590 POINT(40.54785031690254 73.14603465859621) bank10590 +10591 POINT(40.006849192304976 74.05802206255093) bank10591 +10592 POINT(40.15918681916311 73.65647684528162) bank10592 +10593 POINT(41.09297291920836 74.14209439294902) bank10593 +10594 POINT(41.127223525409676 73.53842005989813) bank10594 +10595 POINT(39.83296000242918 73.62179351596681) bank10595 +10596 POINT(41.32084469247976 73.17729676307933) bank10596 +10597 POINT(41.125757760482266 73.96686697312565) bank10597 +10598 POINT(40.68837879746616 74.42564685596577) bank10598 +10599 POINT(40.264457516095284 73.35067940976408) bank10599 +10600 POINT(41.36680133916516 74.04989570302482) bank10600 +10601 POINT(41.18526945882551 73.89719092019884) bank10601 +10602 POINT(40.722398060785196 74.57161005849134) bank10602 +10603 POINT(41.693437621727234 74.6173182408216) bank10603 +10604 POINT(40.668837803671074 74.59815647179006) bank10604 +10605 POINT(39.94479299540257 74.2236193297772) bank10605 +10606 POINT(40.11684842402205 73.39183539648873) bank10606 +10607 POINT(41.035396762686744 73.8971714390429) bank10607 +10608 POINT(40.849595530062324 74.23419460796717) bank10608 +10609 POINT(40.26653844866348 73.44620955726275) bank10609 +10610 POINT(40.80654513200752 74.66070212466856) bank10610 +10611 POINT(40.31842524386787 74.02705816843503) bank10611 +10612 POINT(41.13878560317541 73.15758825635089) bank10612 +10613 POINT(40.388846972190244 73.85225565978777) bank10613 +10614 POINT(40.02873463508924 73.77765958800923) bank10614 +10615 POINT(40.778394000429394 74.2976723458996) bank10615 +10616 POINT(40.53034447427099 73.82135886976765) bank10616 +10617 POINT(39.92074840129225 73.4311922648824) bank10617 +10618 POINT(41.34755068085807 73.9893300147049) bank10618 +10619 POINT(41.64173067290603 73.99396615794078) bank10619 +10620 POINT(40.953282093070854 73.14884867842316) bank10620 +10621 POINT(41.12436540853678 74.38949426768473) bank10621 +10622 POINT(41.01433197046965 74.82225105741524) bank10622 +10623 POINT(40.18913965366558 73.77213613991142) bank10623 +10624 POINT(40.87341481847129 73.80294620434768) bank10624 +10625 POINT(40.82093461193185 74.50688505961803) bank10625 +10626 POINT(40.21324890413404 73.6307639020123) bank10626 +10627 POINT(40.26043855562145 73.8796766455877) bank10627 +10628 POINT(41.5411508101638 74.22556992924422) bank10628 +10629 POINT(41.32061761040825 73.70549738055428) bank10629 +10630 POINT(40.48734332943497 74.55251704608935) bank10630 +10631 POINT(41.07796125091812 74.79137690929043) bank10631 +10632 POINT(40.80858778010808 73.81487689679953) bank10632 +10633 POINT(40.11329687750668 73.8083448740889) bank10633 +10634 POINT(41.36356969722316 73.29491373385412) bank10634 +10635 POINT(39.981303401505706 74.5296689907928) bank10635 +10636 POINT(40.46864170437825 73.92387551258878) bank10636 +10637 POINT(41.344606525002064 73.84041431840483) bank10637 +10638 POINT(41.144896699162246 73.93046741430331) bank10638 +10639 POINT(40.76501618395681 74.37620700902396) bank10639 +10640 POINT(41.30958568573465 74.2131831978734) bank10640 +10641 POINT(39.908878364763105 73.60684713410042) bank10641 +10642 POINT(40.712389985980494 73.07543904160936) bank10642 +10643 POINT(40.82871127591703 73.65145291354149) bank10643 +10644 POINT(40.5212088219656 73.18864172617603) bank10644 +10645 POINT(40.563663954266225 74.63241103839766) bank10645 +10646 POINT(41.30260681207143 73.5385766629818) bank10646 +10647 POINT(41.701582890554484 73.4415011402926) bank10647 +10648 POINT(39.74350342722418 74.05815454796709) bank10648 +10649 POINT(39.93725906077192 73.7738266152667) bank10649 +10650 POINT(40.04443987896606 74.84491537062951) bank10650 +10651 POINT(39.84091775814184 74.30902957261206) bank10651 +10652 POINT(41.05778660805999 73.24103111131198) bank10652 +10653 POINT(41.67390033247103 74.91700243048328) bank10653 +10654 POINT(40.63064914986243 73.52777256873041) bank10654 +10655 POINT(41.14687952758321 73.65643207867514) bank10655 +10656 POINT(40.30430825547428 73.34773718968077) bank10656 +10657 POINT(41.329896005936575 74.2257299421856) bank10657 +10658 POINT(40.0578893801957 74.29379809674751) bank10658 +10659 POINT(40.72410015317793 73.54075623690828) bank10659 +10660 POINT(40.936170854272184 73.33680072361948) bank10660 +10661 POINT(40.24468303619238 74.58521850131324) bank10661 +10662 POINT(40.30526215377779 73.54904864156565) bank10662 +10663 POINT(39.729201281747315 73.47862557496696) bank10663 +10664 POINT(40.73722863868766 73.09265974522064) bank10664 +10665 POINT(41.586861208251335 74.68726129768194) bank10665 +10666 POINT(40.98616536778853 73.83445355900835) bank10666 +10667 POINT(41.07671233616 73.34697617205164) bank10667 +10668 POINT(39.81724939461638 73.47260725113318) bank10668 +10669 POINT(41.01662527304555 74.1446273489311) bank10669 +10670 POINT(40.85842632713283 74.97948612557997) bank10670 +10671 POINT(41.23909628496583 74.99430631665236) bank10671 +10672 POINT(40.67388496774893 73.41984036296215) bank10672 +10673 POINT(39.90851837410241 73.48301207201186) bank10673 +10674 POINT(41.11327408828059 73.6968591754047) bank10674 +10675 POINT(39.81003412610675 74.4113710439053) bank10675 +10676 POINT(41.531667862334736 74.16077549695189) bank10676 +10677 POINT(40.60017840320985 74.36073471445852) bank10677 +10678 POINT(41.2904386364707 73.91505181841825) bank10678 +10679 POINT(40.94897944819394 73.26024519502454) bank10679 +10680 POINT(40.77144883524098 74.29383039027469) bank10680 +10681 POINT(40.801199388918405 74.70969179578015) bank10681 +10682 POINT(41.317513194513815 74.07289063283277) bank10682 +10683 POINT(40.6325686664993 74.99253186286693) bank10683 +10684 POINT(41.60536759368561 74.18290152599076) bank10684 +10685 POINT(40.568369069885094 73.64972542380328) bank10685 +10686 POINT(41.22650628218184 74.20471684563663) bank10686 +10687 POINT(41.32103867170993 74.49644002009792) bank10687 +10688 POINT(40.98151214730826 74.41529644900949) bank10688 +10689 POINT(40.18008890616803 73.99514837307875) bank10689 +10690 POINT(40.50293243574105 73.54750704776924) bank10690 +10691 POINT(40.93934850427866 74.65703143145363) bank10691 +10692 POINT(41.09749393735229 74.54491771474892) bank10692 +10693 POINT(39.79643301642163 73.98345940026327) bank10693 +10694 POINT(39.818409790875144 74.12410808271078) bank10694 +10695 POINT(39.8443231937754 73.78316503942115) bank10695 +10696 POINT(39.71416024537385 74.78558707962824) bank10696 +10697 POINT(41.33285852799117 73.89569469953406) bank10697 +10698 POINT(40.555568931803585 73.57670910906174) bank10698 +10699 POINT(40.56823654221659 74.62707273421131) bank10699 +10700 POINT(39.98296638232167 73.36653985799256) bank10700 +10701 POINT(40.01035803714359 74.92633412354326) bank10701 +10702 POINT(40.28380217440902 74.16896032420756) bank10702 +10703 POINT(40.94566198213347 74.24719219617191) bank10703 +10704 POINT(40.73403484716763 74.50591938960284) bank10704 +10705 POINT(40.653421941789034 73.77799594370285) bank10705 +10706 POINT(40.3001459003673 73.67503439074027) bank10706 +10707 POINT(41.09693889497711 73.85806096041459) bank10707 +10708 POINT(41.44194270769241 73.20683839908668) bank10708 +10709 POINT(41.110517766463296 74.03712744224232) bank10709 +10710 POINT(39.7299590040778 74.5311107663087) bank10710 +10711 POINT(39.77279468472318 73.65710655145192) bank10711 +10712 POINT(41.60420094952958 73.75331112570724) bank10712 +10713 POINT(40.79412922216876 73.1224881242615) bank10713 +10714 POINT(39.764415658581555 73.20292977156097) bank10714 +10715 POINT(41.395012298903936 74.16853231870014) bank10715 +10716 POINT(39.985037444104734 74.49201760706741) bank10716 +10717 POINT(40.348565296423125 74.06857797888264) bank10717 +10718 POINT(40.03164400984154 74.61958661984914) bank10718 +10719 POINT(41.21260633029854 73.09336448263606) bank10719 +10720 POINT(39.96190480311278 73.79414212232035) bank10720 +10721 POINT(40.89819028392074 74.689506850038) bank10721 +10722 POINT(41.56787513039003 73.7408989858624) bank10722 +10723 POINT(41.52183178287758 73.80158910339227) bank10723 +10724 POINT(41.688658053282715 74.12838654769617) bank10724 +10725 POINT(40.66690333178974 74.23985256083887) bank10725 +10726 POINT(40.37545228617679 74.66698702693996) bank10726 +10727 POINT(40.95714348019561 73.99632086178822) bank10727 +10728 POINT(40.216407712671376 73.71426587247471) bank10728 +10729 POINT(40.740236487475705 73.22334784621692) bank10729 +10730 POINT(40.63695006656493 73.23028252618214) bank10730 +10731 POINT(39.82081066032859 74.56811684788873) bank10731 +10732 POINT(41.45272330759346 73.08644375103283) bank10732 +10733 POINT(40.83631780554513 74.25369442896174) bank10733 +10734 POINT(40.269073394136754 74.27526305801682) bank10734 +10735 POINT(40.41001355964036 74.73757438644982) bank10735 +10736 POINT(40.25164397308987 73.35996540836378) bank10736 +10737 POINT(41.039810205238425 73.04219699721348) bank10737 +10738 POINT(40.26297998216965 73.56488329660105) bank10738 +10739 POINT(39.736462108574536 74.66398109241744) bank10739 +10740 POINT(39.731287669845436 73.53666890157761) bank10740 +10741 POINT(39.83444519352738 73.52308894699208) bank10741 +10742 POINT(41.519085026114794 74.74387338788169) bank10742 +10743 POINT(40.648829059453234 73.6503411965966) bank10743 +10744 POINT(41.34137278317757 73.97858323816007) bank10744 +10745 POINT(41.17814172150009 74.76861128220445) bank10745 +10746 POINT(40.38095024017511 74.91063472279755) bank10746 +10747 POINT(39.95707571704858 74.21066491626485) bank10747 +10748 POINT(39.97675314607021 73.8172348888094) bank10748 +10749 POINT(41.52698427142776 73.85758383917405) bank10749 +10750 POINT(40.63461064475623 74.53796471375846) bank10750 +10751 POINT(41.15910826384662 74.01485350917872) bank10751 +10752 POINT(39.94903600264852 73.4486463592124) bank10752 +10753 POINT(40.46278732129246 74.95722280223595) bank10753 +10754 POINT(39.78045333873965 73.08308239354498) bank10754 +10755 POINT(40.785290195309116 73.77616169720308) bank10755 +10756 POINT(41.671442195593876 73.23628912029207) bank10756 +10757 POINT(39.880509804005 73.37288232555316) bank10757 +10758 POINT(40.936212622364536 74.58815454608515) bank10758 +10759 POINT(40.89845281646851 74.24594161653378) bank10759 +10760 POINT(40.15857931592397 73.88543328262567) bank10760 +10761 POINT(40.23553855013855 74.44701888051976) bank10761 +10762 POINT(40.40203283211799 74.47770024176212) bank10762 +10763 POINT(41.61032486920124 74.26756894900689) bank10763 +10764 POINT(40.725092865781825 73.70819218937729) bank10764 +10765 POINT(41.39527391526348 73.42568538638932) bank10765 +10766 POINT(40.58454262241684 73.52166710130021) bank10766 +10767 POINT(40.02219738767988 73.83812193337042) bank10767 +10768 POINT(41.17490843447147 74.2493196862009) bank10768 +10769 POINT(39.71531423611159 73.54474430145041) bank10769 +10770 POINT(40.76050326311485 74.58198113722018) bank10770 +10771 POINT(40.37605970579544 74.35242805216365) bank10771 +10772 POINT(40.43489131695595 74.77494061709979) bank10772 +10773 POINT(40.23163792745084 73.25108309845758) bank10773 +10774 POINT(41.46910647833879 74.84638847481821) bank10774 +10775 POINT(40.27260240171139 73.32952626876487) bank10775 +10776 POINT(40.04003608397331 73.54957761476989) bank10776 +10777 POINT(39.9525915989797 74.69610224439113) bank10777 +10778 POINT(40.02176562527968 73.26416153960633) bank10778 +10779 POINT(40.871152640251005 73.53213424679488) bank10779 +10780 POINT(40.15368573974868 74.49436238727225) bank10780 +10781 POINT(39.90470509865665 73.38249290122094) bank10781 +10782 POINT(41.09917471101446 73.25802788186803) bank10782 +10783 POINT(41.0690103376096 74.30423296279525) bank10783 +10784 POINT(40.76717987381673 74.17193450677954) bank10784 +10785 POINT(41.46615363496161 73.43895641730649) bank10785 +10786 POINT(41.172789550605565 73.77291086277155) bank10786 +10787 POINT(40.15805691238052 74.38509056402637) bank10787 +10788 POINT(39.731010164917215 73.74721236015077) bank10788 +10789 POINT(40.41992478970182 74.99749401487243) bank10789 +10790 POINT(40.030607803876954 73.59010837159329) bank10790 +10791 POINT(40.265280061690035 74.97020014699598) bank10791 +10792 POINT(39.74760797637979 73.0199164389038) bank10792 +10793 POINT(40.71739983273495 73.4578601379991) bank10793 +10794 POINT(39.97266587756667 73.79811522861903) bank10794 +10795 POINT(40.25427691657345 74.86271583000817) bank10795 +10796 POINT(40.82216178688231 74.32560274789216) bank10796 +10797 POINT(40.706718090170746 74.47560408188235) bank10797 +10798 POINT(41.00996042922208 73.54156275827513) bank10798 +10799 POINT(40.10716382412513 74.32620948261554) bank10799 +10800 POINT(41.14771735147663 73.13679611838249) bank10800 +10801 POINT(41.29166507390152 73.71151633389454) bank10801 +10802 POINT(41.135360123734905 74.58919472080014) bank10802 +10803 POINT(41.495642034649066 73.50325343963883) bank10803 +10804 POINT(40.328949263326976 73.09949193910646) bank10804 +10805 POINT(40.30806134012242 73.38184141885317) bank10805 +10806 POINT(40.60733701487112 73.20328892431465) bank10806 +10807 POINT(41.19048114492279 73.62627148431112) bank10807 +10808 POINT(41.70431773553774 74.84377767608805) bank10808 +10809 POINT(40.97069301945836 74.54411052381208) bank10809 +10810 POINT(40.153269787305064 73.29473817079041) bank10810 +10811 POINT(40.004210979649784 74.81389726490417) bank10811 +10812 POINT(40.24048567944647 73.1472807182764) bank10812 +10813 POINT(39.96472790021185 73.0845217628366) bank10813 +10814 POINT(41.33984324844361 73.66576742745964) bank10814 +10815 POINT(40.336720973935364 73.21679325630278) bank10815 +10816 POINT(41.33735863963402 73.40230663019291) bank10816 +10817 POINT(41.08123407528014 73.68791413004894) bank10817 +10818 POINT(40.674066510860364 73.92272186829831) bank10818 +10819 POINT(41.33465436630525 73.75588717126328) bank10819 +10820 POINT(41.310649549463974 73.86225994197945) bank10820 +10821 POINT(39.80418457435452 74.6245841134605) bank10821 +10822 POINT(41.43987322598966 73.31659935754138) bank10822 +10823 POINT(39.808683937332546 73.86100406058192) bank10823 +10824 POINT(40.403984156576904 74.64800271410216) bank10824 +10825 POINT(39.81951035718599 74.5923355591779) bank10825 +10826 POINT(40.54567162804071 74.42092471397042) bank10826 +10827 POINT(39.94026577720529 73.0702736441592) bank10827 +10828 POINT(40.20116796199274 74.85542406972138) bank10828 +10829 POINT(40.409741378774406 74.80621034311179) bank10829 +10830 POINT(41.21522036184432 73.47078421985242) bank10830 +10831 POINT(40.608561200715634 74.58340958640926) bank10831 +10832 POINT(40.98791553024707 74.90820883969839) bank10832 +10833 POINT(40.921145220371876 73.90492593661018) bank10833 +10834 POINT(39.844664496804945 74.83247746446713) bank10834 +10835 POINT(40.09034479975014 74.35292901668569) bank10835 +10836 POINT(41.26793884754891 74.54808388609841) bank10836 +10837 POINT(40.98886560086927 73.93054104095053) bank10837 +10838 POINT(40.18792261548278 73.52906632476198) bank10838 +10839 POINT(40.202582447006925 73.50171546833386) bank10839 +10840 POINT(41.4627626736719 73.55512934081372) bank10840 +10841 POINT(40.101783428683405 74.19354797342685) bank10841 +10842 POINT(41.37692584702479 74.03221134018537) bank10842 +10843 POINT(40.644605636709315 74.28544690686526) bank10843 +10844 POINT(41.104081264778806 73.76063715175576) bank10844 +10845 POINT(41.71240891612544 74.58903828420041) bank10845 +10846 POINT(40.39738852073104 74.97187705551785) bank10846 +10847 POINT(39.744028919760275 73.18806534872039) bank10847 +10848 POINT(40.94402684134262 73.89542309266089) bank10848 +10849 POINT(39.866596852081415 74.82539385168602) bank10849 +10850 POINT(41.207384973213074 73.96070740655932) bank10850 +10851 POINT(40.167265802109966 73.07294529111512) bank10851 +10852 POINT(39.91247585678432 73.53329978493935) bank10852 +10853 POINT(41.234769507838685 73.05165434795713) bank10853 +10854 POINT(41.39446188396943 73.83592976106083) bank10854 +10855 POINT(41.62888457058513 73.67561200730617) bank10855 +10856 POINT(41.64806081690825 73.85559740976993) bank10856 +10857 POINT(41.06244111301535 74.53087051949294) bank10857 +10858 POINT(40.396938898173204 74.05489548306724) bank10858 +10859 POINT(41.07216481744486 74.90835926522996) bank10859 +10860 POINT(40.79764999949415 74.37364373045597) bank10860 +10861 POINT(40.196150742262304 74.03735389948882) bank10861 +10862 POINT(41.40427864670282 74.37930878739786) bank10862 +10863 POINT(40.151379402620194 74.92145993326272) bank10863 +10864 POINT(40.67721550323374 74.35831813374914) bank10864 +10865 POINT(41.550838303786556 73.38820929506689) bank10865 +10866 POINT(39.73016855992417 74.98064409635299) bank10866 +10867 POINT(40.503482694691556 74.78216151784427) bank10867 +10868 POINT(40.47389294118364 74.69284520143408) bank10868 +10869 POINT(39.944976176015686 73.74772469901868) bank10869 +10870 POINT(41.058735597686216 73.74048671972655) bank10870 +10871 POINT(40.785265106134915 73.27213488881559) bank10871 +10872 POINT(40.08106133039238 73.82924506105317) bank10872 +10873 POINT(40.50471563274373 73.26216609120328) bank10873 +10874 POINT(40.32436075445112 73.49048874620794) bank10874 +10875 POINT(40.09176178961736 73.77250609358691) bank10875 +10876 POINT(41.52426689708079 73.1310059963705) bank10876 +10877 POINT(41.70947485365915 73.81209099480019) bank10877 +10878 POINT(40.907336058784125 73.71722477907493) bank10878 +10879 POINT(40.47424322171042 74.37863893005569) bank10879 +10880 POINT(40.17266954625817 73.94681692295198) bank10880 +10881 POINT(39.92181970046919 73.44716215345021) bank10881 +10882 POINT(40.85144872779875 73.74800146668937) bank10882 +10883 POINT(40.93179480807843 74.11843324250157) bank10883 +10884 POINT(40.74103925109664 74.84063847538653) bank10884 +10885 POINT(40.25003097482395 74.30942194908923) bank10885 +10886 POINT(40.932717161286625 73.21957545173647) bank10886 +10887 POINT(41.1297152438129 73.47857522297858) bank10887 +10888 POINT(41.26758839295979 73.70661469575174) bank10888 +10889 POINT(40.534866064624254 73.6001963465523) bank10889 +10890 POINT(40.62012944902613 74.63612416299728) bank10890 +10891 POINT(40.697697806608254 74.91024807628244) bank10891 +10892 POINT(40.06490375006028 73.11743655821394) bank10892 +10893 POINT(40.85740374544041 73.85946930406179) bank10893 +10894 POINT(40.427532265672305 73.36434422773013) bank10894 +10895 POINT(41.50934777948495 73.03000662256228) bank10895 +10896 POINT(41.52152959783626 74.24311975490372) bank10896 +10897 POINT(40.01136485291815 74.88354612422856) bank10897 +10898 POINT(41.10548269398639 74.21406313377705) bank10898 +10899 POINT(41.62951894057381 74.70046404786387) bank10899 +10900 POINT(40.2461848877003 73.51601526828598) bank10900 +10901 POINT(41.14480212743676 73.99595527142155) bank10901 +10902 POINT(39.88041477982422 73.80608303693887) bank10902 +10903 POINT(41.371823508902835 73.94428234350636) bank10903 +10904 POINT(40.456151774318066 74.18792180953255) bank10904 +10905 POINT(41.054607321013876 73.72152127180188) bank10905 +10906 POINT(40.07377210407968 74.73264570509406) bank10906 +10907 POINT(40.35763099160545 74.38058647046523) bank10907 +10908 POINT(40.903196867581556 74.78985927032218) bank10908 +10909 POINT(40.5551342441333 74.32181762776428) bank10909 +10910 POINT(41.50407759577759 73.4540085061569) bank10910 +10911 POINT(40.706300695803144 75.00386418626273) bank10911 +10912 POINT(40.92861693399907 73.97651763565618) bank10912 +10913 POINT(40.417955217869086 74.25814106627618) bank10913 +10914 POINT(41.12163614800145 73.41000013404309) bank10914 +10915 POINT(41.0587338275114 73.91454122741773) bank10915 +10916 POINT(40.7978415486362 73.63255274629107) bank10916 +10917 POINT(40.13233689493388 73.4257867740104) bank10917 +10918 POINT(41.14730898118555 73.94689441902374) bank10918 +10919 POINT(41.54849381401942 74.93223526853961) bank10919 +10920 POINT(41.691566273577834 74.36780335117005) bank10920 +10921 POINT(40.330895361613905 74.89366150817688) bank10921 +10922 POINT(41.47264283171946 73.50565037163068) bank10922 +10923 POINT(40.532811562433025 73.39767431632501) bank10923 +10924 POINT(41.45697799037181 73.02973069218645) bank10924 +10925 POINT(40.123575749146255 73.14556609244306) bank10925 +10926 POINT(40.86459332646602 73.17663598411397) bank10926 +10927 POINT(39.72459992783693 74.30270553788364) bank10927 +10928 POINT(41.39654655416926 74.38113914153325) bank10928 +10929 POINT(40.688535270551725 73.29845834162978) bank10929 +10930 POINT(41.39665731462704 73.7786566710582) bank10930 +10931 POINT(39.732166163655066 73.77623643944135) bank10931 +10932 POINT(40.41404063293229 74.84207946631959) bank10932 +10933 POINT(40.99690894181774 74.8721474947767) bank10933 +10934 POINT(41.61763606431347 74.3560125734676) bank10934 +10935 POINT(39.84019166023543 73.38627369935756) bank10935 +10936 POINT(40.81754566104485 74.8331497442036) bank10936 +10937 POINT(40.44806849325881 74.78950814214558) bank10937 +10938 POINT(40.86138632902944 74.9356660821588) bank10938 +10939 POINT(41.05130841611998 74.19579594135563) bank10939 +10940 POINT(41.384519256455 73.92505659027468) bank10940 +10941 POINT(40.501237378605104 74.63214922168319) bank10941 +10942 POINT(40.267765408959654 74.54207693195961) bank10942 +10943 POINT(40.58860495005237 73.41616455751635) bank10943 +10944 POINT(41.21188352340739 74.91408553721882) bank10944 +10945 POINT(41.57099550178909 74.87710814414999) bank10945 +10946 POINT(41.047314284017645 73.11103841210566) bank10946 +10947 POINT(40.76115695292246 74.45175672590074) bank10947 +10948 POINT(40.26677158330857 73.36580060792262) bank10948 +10949 POINT(40.37696186801181 73.43928427625487) bank10949 +10950 POINT(41.2754960973677 73.5014961270682) bank10950 +10951 POINT(40.865404088505386 74.70400450953784) bank10951 +10952 POINT(41.006321374103884 73.65068149520941) bank10952 +10953 POINT(41.43036981093762 74.50695931184721) bank10953 +10954 POINT(40.62732893025586 73.5887480866929) bank10954 +10955 POINT(41.23593480052086 74.70433048674415) bank10955 +10956 POINT(41.16916752015396 74.25215829913616) bank10956 +10957 POINT(40.970970768281056 74.40997316330473) bank10957 +10958 POINT(41.64319249634273 73.63770956299582) bank10958 +10959 POINT(41.43431977983692 73.61114255386407) bank10959 +10960 POINT(41.63653002792945 74.65718846510296) bank10960 +10961 POINT(40.09809122665723 73.97611238742454) bank10961 +10962 POINT(40.22759142440576 74.45223293954488) bank10962 +10963 POINT(40.82865359808471 73.24104172174621) bank10963 +10964 POINT(39.71513624053354 73.88519603906039) bank10964 +10965 POINT(40.76418209932861 74.9280994711788) bank10965 +10966 POINT(41.25544764402506 74.51361455459374) bank10966 +10967 POINT(40.35983573568068 73.52471326210828) bank10967 +10968 POINT(40.369694701239865 74.47154786645166) bank10968 +10969 POINT(41.510521346700315 74.71361936820233) bank10969 +10970 POINT(39.82323450390211 74.62716371881723) bank10970 +10971 POINT(41.40711490355179 74.0215007447223) bank10971 +10972 POINT(40.98190557480507 74.74684063022177) bank10972 +10973 POINT(40.32382326210498 74.58367351740671) bank10973 +10974 POINT(40.08968381302647 73.66468284432392) bank10974 +10975 POINT(41.32149700645326 74.63759309674143) bank10975 +10976 POINT(40.50665389881475 73.66938490416531) bank10976 +10977 POINT(39.807306336296136 74.48459910920185) bank10977 +10978 POINT(40.343637191429806 74.23894513064187) bank10978 +10979 POINT(40.27863166696352 73.2673477740705) bank10979 +10980 POINT(39.79077012574758 74.93189901627379) bank10980 +10981 POINT(41.505758327725815 73.41132865910654) bank10981 +10982 POINT(39.89888514021502 74.33882884856884) bank10982 +10983 POINT(40.99259624503193 74.7660919219274) bank10983 +10984 POINT(40.70654408383928 74.77728625912944) bank10984 +10985 POINT(39.73852683432374 74.18711095554673) bank10985 +10986 POINT(40.922773191698425 73.84434023238505) bank10986 +10987 POINT(39.878483482035364 73.28278621691254) bank10987 +10988 POINT(40.78531383971976 74.64942711859496) bank10988 +10989 POINT(41.477618346186055 73.17563265742866) bank10989 +10990 POINT(40.2792632707188 74.13305287656753) bank10990 +10991 POINT(40.304671694097514 74.00891268769107) bank10991 +10992 POINT(41.52723492684406 73.44011577351473) bank10992 +10993 POINT(41.35058712463295 74.60541069733985) bank10993 +10994 POINT(39.84933886120237 74.30999907626588) bank10994 +10995 POINT(40.820390207231604 74.62677481779984) bank10995 +10996 POINT(39.99711156371441 73.26510040842413) bank10996 +10997 POINT(41.07726435719137 74.69466780794055) bank10997 +10998 POINT(41.53489373117943 73.09961358656614) bank10998 +10999 POINT(39.78276296893802 74.73526875475994) bank10999 +11000 POINT(41.32532527332686 73.28351339594549) bank11000 +11001 POINT(39.79606053418588 73.74746040259727) bank11001 +11002 POINT(41.61791564675631 74.06559838869387) bank11002 +11003 POINT(39.96825854004334 73.98775636540763) bank11003 +11004 POINT(40.96311733749884 73.63205677697954) bank11004 +11005 POINT(40.73757257548488 73.92539093136077) bank11005 +11006 POINT(41.135185596318266 73.37503964651332) bank11006 +11007 POINT(39.92775629535366 74.30517439829501) bank11007 +11008 POINT(41.29795554789508 73.02832293983019) bank11008 +11009 POINT(40.35640232851881 74.04692675925774) bank11009 +11010 POINT(40.30232919761814 73.3777345797747) bank11010 +11011 POINT(41.36411249332986 74.93933473126089) bank11011 +11012 POINT(40.63410694580255 73.2943272196535) bank11012 +11013 POINT(40.4096199959231 73.59559243484524) bank11013 +11014 POINT(41.22086240662618 74.26029805680761) bank11014 +11015 POINT(41.615354694171536 73.40822391010381) bank11015 +11016 POINT(40.78704804325029 74.37314782652966) bank11016 +11017 POINT(40.77846292524318 74.32282747869296) bank11017 +11018 POINT(41.15275086741138 74.66911747025222) bank11018 +11019 POINT(39.91230557292539 73.40883177668539) bank11019 +11020 POINT(41.67184950187014 73.62045687301553) bank11020 +11021 POINT(40.101560008414275 74.3997358841742) bank11021 +11022 POINT(41.57920576888163 74.91322754595863) bank11022 +11023 POINT(41.243972617646506 73.78623915116077) bank11023 +11024 POINT(40.29083738977708 74.54183596337404) bank11024 +11025 POINT(41.0377904575663 74.11010899747788) bank11025 +11026 POINT(40.46695966532014 74.89562743713263) bank11026 +11027 POINT(40.72361163301417 74.92320695760367) bank11027 +11028 POINT(41.18829622774382 73.70470822566477) bank11028 +11029 POINT(40.736680109120016 74.73013095305544) bank11029 +11030 POINT(40.60614564038982 73.31203246728079) bank11030 +11031 POINT(40.6249062349508 73.61151205840807) bank11031 +11032 POINT(40.899564633085696 74.58976722057145) bank11032 +11033 POINT(40.701646867251156 74.7323696632834) bank11033 +11034 POINT(41.67296817658133 73.00764701356177) bank11034 +11035 POINT(40.49245400426143 73.63174897379957) bank11035 +11036 POINT(40.02076869009859 74.07175916753908) bank11036 +11037 POINT(41.69125387388788 73.64132911472807) bank11037 +11038 POINT(40.50915944641569 73.09096776240592) bank11038 +11039 POINT(41.272783402278414 74.62503831508016) bank11039 +11040 POINT(40.85795756619972 73.51612937808498) bank11040 +11041 POINT(41.033575305834574 74.0252776664284) bank11041 +11042 POINT(40.84091544684553 73.47100908922) bank11042 +11043 POINT(40.79763558350205 73.11420443787453) bank11043 +11044 POINT(40.424319998355564 73.28979237549804) bank11044 +11045 POINT(41.50537144940262 73.59995428680793) bank11045 +11046 POINT(41.44991045432067 74.21825362655711) bank11046 +11047 POINT(41.58245912637733 73.9245554301475) bank11047 +11048 POINT(40.953731199375284 74.01860917430332) bank11048 +11049 POINT(41.121261088174144 74.40502518276432) bank11049 +11050 POINT(41.26845028653731 73.40657581883586) bank11050 +11051 POINT(40.23143668804553 73.72651932846135) bank11051 +11052 POINT(40.03161883168352 74.46765134988675) bank11052 +11053 POINT(40.79892962541764 73.95711104883338) bank11053 +11054 POINT(39.807095313692194 73.55145512233153) bank11054 +11055 POINT(41.34013658989546 73.36922390928783) bank11055 +11056 POINT(41.644624975963836 73.66627721303628) bank11056 +11057 POINT(40.088350483739355 73.21147982999305) bank11057 +11058 POINT(40.47971804202117 73.13227242125686) bank11058 +11059 POINT(39.95603467787852 74.1385159287011) bank11059 +11060 POINT(40.7891842804226 74.90458466273266) bank11060 +11061 POINT(40.06702791204606 74.75976216530943) bank11061 +11062 POINT(40.23805090712839 73.99150160357294) bank11062 +11063 POINT(40.68005835500992 74.61858278012728) bank11063 +11064 POINT(40.1522488321295 73.13905112395024) bank11064 +11065 POINT(40.465674686399524 73.1037039763259) bank11065 +11066 POINT(41.49220919041717 73.68570990830366) bank11066 +11067 POINT(39.96257764165396 73.2723564667538) bank11067 +11068 POINT(40.6957905449073 74.92589533680228) bank11068 +11069 POINT(40.82605537860057 73.30845590124979) bank11069 +11070 POINT(41.577175151512876 74.19155166555507) bank11070 +11071 POINT(40.97431723158042 74.7459688168322) bank11071 +11072 POINT(40.056201110547605 74.3414817226193) bank11072 +11073 POINT(40.416329845370214 74.18762976502926) bank11073 +11074 POINT(41.33903655025838 73.56303368431884) bank11074 +11075 POINT(41.331727670542925 74.05504574964941) bank11075 +11076 POINT(39.74326713060288 73.4398664694206) bank11076 +11077 POINT(41.59247528232636 74.47183468980184) bank11077 +11078 POINT(41.657642396073335 73.38222174682974) bank11078 +11079 POINT(40.897002725801265 73.88426875364321) bank11079 +11080 POINT(39.835142018965946 73.8828799160629) bank11080 +11081 POINT(41.42729960233774 73.93618157899525) bank11081 +11082 POINT(40.03167372846306 73.60399759480592) bank11082 +11083 POINT(40.87284122045409 74.19110034094882) bank11083 +11084 POINT(40.141016125876384 74.1349226409256) bank11084 +11085 POINT(40.4257121098427 73.96369853058329) bank11085 +11086 POINT(40.67003208117466 73.84739832992176) bank11086 +11087 POINT(41.16814943848362 73.05709722615448) bank11087 +11088 POINT(40.29177248519177 74.27516808836184) bank11088 +11089 POINT(41.41037358802257 74.4570417778294) bank11089 +11090 POINT(40.97348754180472 73.52096625147408) bank11090 +11091 POINT(39.77830849122628 73.76912183895311) bank11091 +11092 POINT(39.76833818593568 74.086341269033) bank11092 +11093 POINT(39.87728123313642 73.45438059250802) bank11093 +11094 POINT(40.15648687059809 73.79420994072522) bank11094 +11095 POINT(40.03217518451835 74.89536367718682) bank11095 +11096 POINT(41.466758627907566 74.47661856557006) bank11096 +11097 POINT(41.00840088888088 74.3847192234296) bank11097 +11098 POINT(40.24841897717743 74.67665195342937) bank11098 +11099 POINT(41.62560019113819 74.17660225188486) bank11099 +11100 POINT(39.720418716274374 73.15111185163254) bank11100 +11101 POINT(40.82713454862324 74.58273731495541) bank11101 +11102 POINT(41.55090636128166 74.03641815107002) bank11102 +11103 POINT(40.835451874134044 74.44562967606213) bank11103 +11104 POINT(40.84663099086777 74.43037421934518) bank11104 +11105 POINT(41.65079776754257 73.49268177954923) bank11105 +11106 POINT(40.388966411775066 73.75261886550679) bank11106 +11107 POINT(41.02566748459134 74.479704837499) bank11107 +11108 POINT(41.212246626199075 74.93385325413118) bank11108 +11109 POINT(39.8793422041332 73.64032536105084) bank11109 +11110 POINT(40.79631200474706 73.10222382721565) bank11110 +11111 POINT(41.175496987488295 73.03524137442038) bank11111 +11112 POINT(40.443228773634495 73.89496417263722) bank11112 +11113 POINT(40.85377499345863 73.67806637225665) bank11113 +11114 POINT(40.04850965804112 73.15979439972804) bank11114 +11115 POINT(41.26654503615415 73.99356568260043) bank11115 +11116 POINT(41.289919712717975 74.11762923500483) bank11116 +11117 POINT(41.26964581690135 73.76361759972701) bank11117 +11118 POINT(41.24631983918337 73.09690139051135) bank11118 +11119 POINT(40.36658090793284 74.1209425369181) bank11119 +11120 POINT(40.06456484578381 74.83213304213409) bank11120 +11121 POINT(40.10280110530411 73.65572305108428) bank11121 +11122 POINT(40.591849461488664 73.60409494499133) bank11122 +11123 POINT(41.48543734833429 73.9587474325843) bank11123 +11124 POINT(40.27703120595303 74.20022769600247) bank11124 +11125 POINT(40.22994712030758 74.44541910266847) bank11125 +11126 POINT(40.375189572344325 73.98110820956043) bank11126 +11127 POINT(40.76502267911531 74.27366435615559) bank11127 +11128 POINT(40.668928120635975 74.9962728897539) bank11128 +11129 POINT(40.163206011264194 73.42988820117715) bank11129 +11130 POINT(41.39123400343442 73.65834194722486) bank11130 +11131 POINT(41.42295367523248 73.36030885551234) bank11131 +11132 POINT(41.128585536921946 74.39349919995614) bank11132 +11133 POINT(40.327898452507974 73.88855171570563) bank11133 +11134 POINT(41.52817131013899 74.57097809256233) bank11134 +11135 POINT(41.42224506441861 73.42062537224253) bank11135 +11136 POINT(40.240202641127965 74.74171276801576) bank11136 +11137 POINT(41.38911006355989 74.2514221189574) bank11137 +11138 POINT(41.36165753936277 74.19407918637269) bank11138 +11139 POINT(41.11647001672284 73.52411702322365) bank11139 +11140 POINT(40.92854289706737 73.88718415266842) bank11140 +11141 POINT(41.323611103163614 74.64300985744036) bank11141 +11142 POINT(40.93411137176252 74.22014308050086) bank11142 +11143 POINT(39.83799404829466 74.21985173641897) bank11143 +11144 POINT(40.5654213348678 73.91699203711214) bank11144 +11145 POINT(41.37785030165342 73.48897817860333) bank11145 +11146 POINT(40.52005214487713 74.06980343632152) bank11146 +11147 POINT(40.810943338066075 73.9543215734923) bank11147 +11148 POINT(40.9881774356698 73.65819114101132) bank11148 +11149 POINT(40.24799557325288 73.03717763482837) bank11149 +11150 POINT(40.97431743888549 74.33815995193066) bank11150 +11151 POINT(41.373275124182626 73.11015347761594) bank11151 +11152 POINT(39.894312592679185 73.17460180432325) bank11152 +11153 POINT(41.50979954429776 73.30377172574923) bank11153 +11154 POINT(40.45465256962454 73.2670187503444) bank11154 +11155 POINT(39.90113013686511 74.65117072253707) bank11155 +11156 POINT(40.87318132418588 73.02545038922486) bank11156 +11157 POINT(40.12634645817327 74.05013761063594) bank11157 +11158 POINT(40.545548147420284 73.06438116846297) bank11158 +11159 POINT(41.1717959986716 73.24709510015052) bank11159 +11160 POINT(39.792670473876456 74.21572792655263) bank11160 +11161 POINT(40.008700223858305 73.29276713688013) bank11161 +11162 POINT(40.177312649888165 74.36380663127785) bank11162 +11163 POINT(39.79417601494351 73.79109063181737) bank11163 +11164 POINT(40.99721817808177 73.38525938353861) bank11164 +11165 POINT(40.11440770679151 74.44659871732046) bank11165 +11166 POINT(40.46209174292088 73.17409813599254) bank11166 +11167 POINT(41.38673019853245 74.38344919252843) bank11167 +11168 POINT(41.3839472271203 73.89718713675232) bank11168 +11169 POINT(40.86363360371449 74.94159053220214) bank11169 +11170 POINT(40.04945373482037 74.16031378668644) bank11170 +11171 POINT(41.10070067179649 73.2805688395335) bank11171 +11172 POINT(39.90329295689021 74.05584950559401) bank11172 +11173 POINT(40.4456542424426 74.99265443767909) bank11173 +11174 POINT(40.65171503442881 73.8753554832344) bank11174 +11175 POINT(40.12335554040125 73.86196747096196) bank11175 +11176 POINT(40.964071338886896 73.78554762661064) bank11176 +11177 POINT(40.735665662551284 74.33381130414037) bank11177 +11178 POINT(41.30987408745919 74.46262756400955) bank11178 +11179 POINT(40.42927167387482 73.44244276955774) bank11179 +11180 POINT(40.771736545567514 74.31575320897721) bank11180 +11181 POINT(40.213717584159156 74.75244156324406) bank11181 +11182 POINT(40.42583265930212 73.76606960507063) bank11182 +11183 POINT(41.668849441903845 74.50940285449497) bank11183 +11184 POINT(41.42021815751444 73.7563920466149) bank11184 +11185 POINT(40.6651138067398 73.4398266960355) bank11185 +11186 POINT(39.806902813538336 73.37983136370114) bank11186 +11187 POINT(39.998230408023204 73.85017672793423) bank11187 +11188 POINT(40.98051830430348 74.69341859733231) bank11188 +11189 POINT(40.486073902117155 73.75739438144451) bank11189 +11190 POINT(41.55637592597583 73.5473724273429) bank11190 +11191 POINT(40.361714582926275 73.12455814527931) bank11191 +11192 POINT(40.1118545571988 74.72094217567809) bank11192 +11193 POINT(39.8351553914979 74.29687295768592) bank11193 +11194 POINT(40.20615138166325 73.41944588854102) bank11194 +11195 POINT(40.70119499426995 73.56996689832185) bank11195 +11196 POINT(39.88670354651659 73.4876497481134) bank11196 +11197 POINT(41.594726148026766 74.69164541002188) bank11197 +11198 POINT(40.77153183639034 73.5009243702563) bank11198 +11199 POINT(41.04201297816773 74.43344095129522) bank11199 +11200 POINT(41.687058094202634 74.26679384119387) bank11200 +11201 POINT(41.10078087911372 74.4303528240454) bank11201 +11202 POINT(41.25738999799233 73.01413913917652) bank11202 +11203 POINT(41.14990152799832 74.68776502972489) bank11203 +11204 POINT(40.71295803632885 73.87064121008738) bank11204 +11205 POINT(39.765493898977184 74.21115725538053) bank11205 +11206 POINT(39.7452523503721 73.18652358837474) bank11206 +11207 POINT(39.76131297693341 73.12201764081226) bank11207 +11208 POINT(41.35975820731536 73.05695913049908) bank11208 +11209 POINT(40.743405724007225 73.28014723651228) bank11209 +11210 POINT(40.63664880431268 75.00168652741566) bank11210 +11211 POINT(41.53912816497371 74.88619252868313) bank11211 +11212 POINT(40.857147752432326 74.6278648022275) bank11212 +11213 POINT(40.91193897228925 74.44246470276457) bank11213 +11214 POINT(40.617428611807995 74.83230808294886) bank11214 +11215 POINT(41.31958594562821 73.0717849529172) bank11215 +11216 POINT(39.9235961613286 74.85595498607479) bank11216 +11217 POINT(40.008649471759846 74.92467959141293) bank11217 +11218 POINT(41.440782086776416 74.7660626077844) bank11218 +11219 POINT(40.26263873672462 73.69518267974462) bank11219 +11220 POINT(41.52543616712532 73.90151641569028) bank11220 +11221 POINT(41.29926346038817 74.93961654795373) bank11221 +11222 POINT(41.0509193600981 74.86595881402937) bank11222 +11223 POINT(40.79553306805693 74.50080382619609) bank11223 +11224 POINT(41.59144207915071 74.34962267078748) bank11224 +11225 POINT(41.555445351029874 74.75210041092139) bank11225 +11226 POINT(41.30503068239546 73.2445238971223) bank11226 +11227 POINT(40.65014260451959 73.93765304163051) bank11227 +11228 POINT(39.953763066888904 74.7532169580473) bank11228 +11229 POINT(40.110648992279906 73.03108764571994) bank11229 +11230 POINT(40.94594033504739 74.32737680575917) bank11230 +11231 POINT(41.10681347594672 74.13446015323362) bank11231 +11232 POINT(39.952892097151334 74.69289049803325) bank11232 +11233 POINT(39.87390221072609 73.34465388725103) bank11233 +11234 POINT(40.176701539351214 74.94037911754756) bank11234 +11235 POINT(40.61682369401128 74.78365069315139) bank11235 +11236 POINT(40.91934282796138 74.99648984404111) bank11236 +11237 POINT(41.02579040463606 74.62703671961711) bank11237 +11238 POINT(39.854830395222244 73.67384668323709) bank11238 +11239 POINT(41.4753940732082 73.87486216400306) bank11239 +11240 POINT(40.05581612624364 74.57754365916855) bank11240 +11241 POINT(39.78803046703122 73.76629295822532) bank11241 +11242 POINT(41.09567673312648 74.20957995463512) bank11242 +11243 POINT(40.895266739574886 73.83549867666171) bank11243 +11244 POINT(41.27364436158592 73.67535955641335) bank11244 +11245 POINT(39.94259318093728 73.75176404834531) bank11245 +11246 POINT(40.071068399156175 73.63061460612883) bank11246 +11247 POINT(39.80632370133571 74.85052862012243) bank11247 +11248 POINT(39.770343707461535 73.94473434592405) bank11248 +11249 POINT(40.571741644039996 73.71785602356823) bank11249 +11250 POINT(39.96704097283439 74.29937629392828) bank11250 +11251 POINT(41.31633754841936 74.70967340890309) bank11251 +11252 POINT(40.99561407218337 74.61160930890337) bank11252 +11253 POINT(40.16041263510572 73.7067851482398) bank11253 +11254 POINT(40.861573548694324 73.16556584985166) bank11254 +11255 POINT(41.20808779458629 74.78219513456881) bank11255 +11256 POINT(40.8957177349643 74.55864953986985) bank11256 +11257 POINT(41.50235675707934 74.01193074322515) bank11257 +11258 POINT(41.18879646426076 74.94505400472318) bank11258 +11259 POINT(41.405301575557765 74.19971088790803) bank11259 +11260 POINT(41.13174761329388 74.41675004874132) bank11260 +11261 POINT(41.09604845564191 73.19489357194976) bank11261 +11262 POINT(40.47082340896993 74.37401310327643) bank11262 +11263 POINT(39.74860266309062 74.93805980337446) bank11263 +11264 POINT(40.034485696217956 74.9047557872108) bank11264 +11265 POINT(39.82863550600702 74.85250945581757) bank11265 +11266 POINT(41.06269369527302 73.32224405119034) bank11266 +11267 POINT(40.71916271580118 73.8893787269643) bank11267 +11268 POINT(40.00015595832441 74.8380387655494) bank11268 +11269 POINT(41.39560512936613 73.33542255001242) bank11269 +11270 POINT(41.41826819979513 74.2679494232704) bank11270 +11271 POINT(39.8388881070293 74.36254184364918) bank11271 +11272 POINT(40.986936672944815 74.85070276174167) bank11272 +11273 POINT(41.55775985998482 74.27695687011641) bank11273 +11274 POINT(40.18654359713927 73.75614067720613) bank11274 +11275 POINT(39.92127012848191 74.51021672495402) bank11275 +11276 POINT(41.63478213412674 73.86876873412419) bank11276 +11277 POINT(41.09761764091878 74.34448098015758) bank11277 +11278 POINT(40.56029564999062 73.22139012692207) bank11278 +11279 POINT(40.36402757007503 74.02450080751996) bank11279 +11280 POINT(39.82136398855405 74.22575438030375) bank11280 +11281 POINT(40.27941158004946 74.48617973016162) bank11281 +11282 POINT(41.24667777993348 74.78078810104701) bank11282 +11283 POINT(41.46577874257062 73.42177814406654) bank11283 +11284 POINT(40.8527070334216 74.66618375340602) bank11284 +11285 POINT(40.651797681918815 73.39585393661784) bank11285 +11286 POINT(41.47365104648537 73.544788579909) bank11286 +11287 POINT(40.19461309384487 74.11420506641785) bank11287 +11288 POINT(41.419168597700306 73.77586664625981) bank11288 +11289 POINT(40.832217316005085 74.75573221818524) bank11289 +11290 POINT(39.997206023873524 73.28447806071841) bank11290 +11291 POINT(40.722200612614536 73.68528288370469) bank11291 +11292 POINT(40.267258750718305 74.76757591290284) bank11292 +11293 POINT(40.29457838753993 74.03948003043546) bank11293 +11294 POINT(41.711921054786785 74.84260578446646) bank11294 +11295 POINT(41.38372338108317 74.27182410994361) bank11295 +11296 POINT(41.28716814131378 73.49592586033322) bank11296 +11297 POINT(40.71079382859757 74.47094048701213) bank11297 +11298 POINT(41.114671567914854 74.66181087298817) bank11298 +11299 POINT(41.21336902711171 74.28481605178003) bank11299 +11300 POINT(41.348783914713174 73.29195070281152) bank11300 +11301 POINT(40.98789790027379 74.16658830917245) bank11301 +11302 POINT(40.27555911802651 73.80446747440848) bank11302 +11303 POINT(40.255119725113374 73.06832925700715) bank11303 +11304 POINT(39.86405680811443 73.66141737785456) bank11304 +11305 POINT(40.84133359254484 74.42394675819) bank11305 +11306 POINT(40.980765530381056 73.17060361126461) bank11306 +11307 POINT(39.90631319093711 73.13720785090712) bank11307 +11308 POINT(40.91718798525337 74.4911503067437) bank11308 +11309 POINT(40.24737962392316 73.33929657059508) bank11309 +11310 POINT(39.97601558544978 74.99127834717973) bank11310 +11311 POINT(40.362310371510546 73.29643476316913) bank11311 +11312 POINT(41.09157605069399 73.10576287714753) bank11312 +11313 POINT(41.39993775518244 74.06129353924798) bank11313 +11314 POINT(39.97200786726044 74.2306525401622) bank11314 +11315 POINT(41.24062191843486 73.15946746858668) bank11315 +11316 POINT(40.53561530496081 74.25345531755168) bank11316 +11317 POINT(41.449187257259325 74.89851880372196) bank11317 +11318 POINT(40.522840033618216 74.07360065699356) bank11318 +11319 POINT(40.343855745567694 74.63615103987563) bank11319 +11320 POINT(40.68058650791372 74.25769257577576) bank11320 +11321 POINT(40.08752182951136 74.24025143529728) bank11321 +11322 POINT(41.29880658668854 74.80220770806318) bank11322 +11323 POINT(39.86720911325624 73.84801587412463) bank11323 +11324 POINT(40.80295674960486 73.04967743305055) bank11324 +11325 POINT(40.14848959183519 74.1974199030494) bank11325 +11326 POINT(41.29124162539675 73.38841575800171) bank11326 +11327 POINT(41.62808673655694 74.69199203435144) bank11327 +11328 POINT(40.21261526493951 73.3027604716738) bank11328 +11329 POINT(41.50194462595702 73.84613658000421) bank11329 +11330 POINT(39.9433519460316 74.43749582770657) bank11330 +11331 POINT(40.571849043118256 74.4038657002565) bank11331 +11332 POINT(41.39682202995177 74.14742736704223) bank11332 +11333 POINT(41.18379905033379 74.41057045282749) bank11333 +11334 POINT(39.94824365712436 74.03930200852587) bank11334 +11335 POINT(40.779428006540286 73.60662749451072) bank11335 +11336 POINT(40.95647385371226 73.08543804910832) bank11336 +11337 POINT(40.973786105756844 75.0035688563996) bank11337 +11338 POINT(41.38237966327385 73.8737659129763) bank11338 +11339 POINT(40.980576179749434 73.29456771564877) bank11339 +11340 POINT(39.79685958108269 73.06135821160204) bank11340 +11341 POINT(41.33811435114678 74.75882386040729) bank11341 +11342 POINT(40.32604947924912 73.9721425431689) bank11342 +11343 POINT(40.75532431181773 73.0227942288056) bank11343 +11344 POINT(41.59439185189719 73.79671209825446) bank11344 +11345 POINT(40.16213855126279 73.24954990471046) bank11345 +11346 POINT(40.469820839688545 73.63146127437248) bank11346 +11347 POINT(40.19976017647275 73.96677955891496) bank11347 +11348 POINT(41.5539280187223 73.18211369074982) bank11348 +11349 POINT(41.042551318010055 74.79906568123056) bank11349 +11350 POINT(39.98411393821472 73.66260142866173) bank11350 +11351 POINT(41.59479992734113 74.97350842458765) bank11351 +11352 POINT(39.81058433318109 73.38883801619521) bank11352 +11353 POINT(41.369505136036665 74.55653032225928) bank11353 +11354 POINT(41.35666123323412 73.5754737456268) bank11354 +11355 POINT(41.55170851932706 74.57606915897833) bank11355 +11356 POINT(41.21507130574882 73.1751392859943) bank11356 +11357 POINT(40.709930815758845 74.75897145675799) bank11357 +11358 POINT(40.962700008108314 74.5337430378008) bank11358 +11359 POINT(40.126401009148616 74.92796595002964) bank11359 +11360 POINT(41.034836385768855 74.48568207130032) bank11360 +11361 POINT(41.357829940758464 73.02224833668598) bank11361 +11362 POINT(41.03657191737217 73.16829226936504) bank11362 +11363 POINT(39.79833873472937 74.93638178078835) bank11363 +11364 POINT(40.46844507329951 73.53992044935447) bank11364 +11365 POINT(41.119719522887245 74.27150191258804) bank11365 +11366 POINT(41.6895106059282 73.98174573198284) bank11366 +11367 POINT(40.28058837334908 74.49436108116262) bank11367 +11368 POINT(40.23566585199886 74.32893176545504) bank11368 +11369 POINT(40.4065403213985 74.16512711004502) bank11369 +11370 POINT(40.85103205338968 73.2105852171828) bank11370 +11371 POINT(39.727539479624596 74.59202922686417) bank11371 +11372 POINT(40.43293763832129 73.05888918307133) bank11372 +11373 POINT(39.96395130015836 73.1567336811753) bank11373 +11374 POINT(41.58275784117233 73.67160044408065) bank11374 +11375 POINT(40.57049474899231 75.00216033595538) bank11375 +11376 POINT(41.194158657518514 73.30114241965623) bank11376 +11377 POINT(40.95274728168825 74.89303030322026) bank11377 +11378 POINT(39.78459145445215 73.16145670534047) bank11378 +11379 POINT(40.49283572649912 74.07978195415407) bank11379 +11380 POINT(40.692556516667686 73.05849216928833) bank11380 +11381 POINT(41.670617493843096 74.27898905657489) bank11381 +11382 POINT(40.11981647375437 74.86282540414653) bank11382 +11383 POINT(40.38933743957369 74.94445977135818) bank11383 +11384 POINT(39.869986379370324 74.9263677186608) bank11384 +11385 POINT(40.830987329835374 74.14342410025726) bank11385 +11386 POINT(40.140805768320625 74.77426432578103) bank11386 +11387 POINT(41.62824135684094 73.72310894552676) bank11387 +11388 POINT(40.81765444003447 74.76494951389196) bank11388 +11389 POINT(41.175942009333966 74.70460727894998) bank11389 +11390 POINT(40.13286466840607 73.84269808954127) bank11390 +11391 POINT(40.24457406647634 74.28143534289278) bank11391 +11392 POINT(40.60528882470573 73.52444996985169) bank11392 +11393 POINT(40.614231850218644 73.2563834864937) bank11393 +11394 POINT(41.24942575506993 73.78683597529263) bank11394 +11395 POINT(40.89772985879375 73.65375090883114) bank11395 +11396 POINT(41.113614892916665 73.56004979476315) bank11396 +11397 POINT(39.9459482424083 74.63874446030157) bank11397 +11398 POINT(40.6068775342391 74.20952479290467) bank11398 +11399 POINT(40.33344799691451 73.9805790454966) bank11399 +11400 POINT(40.53666955030619 74.84979273328493) bank11400 +11401 POINT(39.8950202822634 74.75487373696261) bank11401 +11402 POINT(41.37634689536933 74.01078145694815) bank11402 +11403 POINT(41.4280177095033 74.17208264099446) bank11403 +11404 POINT(41.49700045101283 74.60305998586075) bank11404 +11405 POINT(39.84793360589961 74.83508770987602) bank11405 +11406 POINT(40.871463635387926 73.3188507920799) bank11406 +11407 POINT(40.76717270246413 73.64789536764692) bank11407 +11408 POINT(41.212103741652015 73.38653515562494) bank11408 +11409 POINT(40.45245561759114 73.29858094857852) bank11409 +11410 POINT(40.84771813259264 73.61779019808242) bank11410 +11411 POINT(40.75753845703215 74.98779579609253) bank11411 +11412 POINT(40.040341740062956 73.48844471998467) bank11412 +11413 POINT(39.8243458413798 73.60098036740925) bank11413 +11414 POINT(39.934616167958346 73.877385837736) bank11414 +11415 POINT(41.02848363285422 73.89245062819408) bank11415 +11416 POINT(40.007090069667946 74.2415001568872) bank11416 +11417 POINT(40.205872543121096 74.68272558020526) bank11417 +11418 POINT(39.984448762055024 74.90382241980863) bank11418 +11419 POINT(41.20752071182335 75.00219463217715) bank11419 +11420 POINT(40.388590015884674 73.40714704744485) bank11420 +11421 POINT(40.462360768473815 74.95624940848283) bank11421 +11422 POINT(40.71616425783254 73.94302343050698) bank11422 +11423 POINT(40.54293077015677 73.17551601370532) bank11423 +11424 POINT(40.112955461956254 73.20201216058706) bank11424 +11425 POINT(41.41485779032798 74.1329782175498) bank11425 +11426 POINT(41.06249491700435 74.95350874673719) bank11426 +11427 POINT(40.891998088274846 73.52392823547888) bank11427 +11428 POINT(40.76055344309768 74.06366342111785) bank11428 +11429 POINT(40.58517948704057 73.68291967221943) bank11429 +11430 POINT(41.34614330771642 74.62535976934268) bank11430 +11431 POINT(40.479924600586 73.97586497918246) bank11431 +11432 POINT(40.82700711198118 73.62336671547077) bank11432 +11433 POINT(41.62105674071775 73.36490282270368) bank11433 +11434 POINT(40.55874989821594 74.0864736775329) bank11434 +11435 POINT(41.66607897126191 73.37149613117386) bank11435 +11436 POINT(40.66092974930601 73.55409029587209) bank11436 +11437 POINT(40.70386901342919 73.9043060390587) bank11437 +11438 POINT(40.68623342557082 73.3168497272304) bank11438 +11439 POINT(40.80526897799619 73.62276228423069) bank11439 +11440 POINT(40.97394855030066 74.63641615131723) bank11440 +11441 POINT(41.490639548777246 74.08660133385621) bank11441 +11442 POINT(39.881710876219756 73.66237345407441) bank11442 +11443 POINT(40.32585864754686 73.49590617562305) bank11443 +11444 POINT(40.08638907366934 73.94407735226841) bank11444 +11445 POINT(40.59322187201045 74.20736314143657) bank11445 +11446 POINT(39.85428459258104 73.36345379625782) bank11446 +11447 POINT(41.580366307164375 73.78385486505098) bank11447 +11448 POINT(39.91662328445165 74.3981834011414) bank11448 +11449 POINT(40.78138209315455 74.4144489310542) bank11449 +11450 POINT(41.027560750123925 73.35507733150621) bank11450 +11451 POINT(41.30352498759675 74.29786542757674) bank11451 +11452 POINT(40.45522589827158 74.99890275580753) bank11452 +11453 POINT(40.80751287713862 74.72912632780286) bank11453 +11454 POINT(40.66932690959327 73.82490653783017) bank11454 +11455 POINT(40.208031486688405 74.63023561618854) bank11455 +11456 POINT(41.333799664315436 73.71732909178172) bank11456 +11457 POINT(41.0984425328907 74.99013986571343) bank11457 +11458 POINT(41.063608855033316 73.18465975265673) bank11458 +11459 POINT(40.92740458682248 74.9836249378221) bank11459 +11460 POINT(39.79945987710536 74.15483559022765) bank11460 +11461 POINT(40.883767564126366 74.81087934868239) bank11461 +11462 POINT(39.93291479211062 73.41075152774464) bank11462 +11463 POINT(41.66872105693924 74.623598450073) bank11463 +11464 POINT(41.26307989846236 73.73068528452573) bank11464 +11465 POINT(41.118709361276174 73.48173006845471) bank11465 +11466 POINT(40.31646164721823 74.91156762638603) bank11466 +11467 POINT(41.162206974308496 73.60237211915776) bank11467 +11468 POINT(40.89125882105533 74.51506383233999) bank11468 +11469 POINT(41.51101308493627 74.55933625152473) bank11469 +11470 POINT(41.36909250783934 73.48829143756798) bank11470 +11471 POINT(40.20471134260308 73.32610917904285) bank11471 +11472 POINT(40.650870655280336 73.94519961527106) bank11472 +11473 POINT(41.24302897332723 73.16208455462866) bank11473 +11474 POINT(41.397685564973486 74.12443765029084) bank11474 +11475 POINT(41.35548199731953 74.77127271633002) bank11475 +11476 POINT(40.16011354277491 74.3761246698505) bank11476 +11477 POINT(41.5097126590633 73.1193513890916) bank11477 +11478 POINT(40.29299297934233 73.42545107320097) bank11478 +11479 POINT(40.52699573210126 73.82871499772266) bank11479 +11480 POINT(40.399459058046794 73.55055500206971) bank11480 +11481 POINT(40.654998557045 73.8165253870672) bank11481 +11482 POINT(41.29373144347048 73.02655702006328) bank11482 +11483 POINT(40.29153479244562 74.1530728670571) bank11483 +11484 POINT(41.31842672517733 73.0821223623012) bank11484 +11485 POINT(40.41944335990328 73.55915281481118) bank11485 +11486 POINT(39.88955086834698 74.0279909954371) bank11486 +11487 POINT(39.816694105299085 74.54387650277891) bank11487 +11488 POINT(41.31129778812524 73.32260109274476) bank11488 +11489 POINT(39.78988405054987 74.70285562512886) bank11489 +11490 POINT(40.68093226422069 73.35462948651801) bank11490 +11491 POINT(40.127504291107925 73.28675934539514) bank11491 +11492 POINT(41.43300089357734 73.52062231775885) bank11492 +11493 POINT(41.29180088068632 74.91777854971507) bank11493 +11494 POINT(40.75222930907707 73.17181110039763) bank11494 +11495 POINT(39.97242328541776 73.83324423991428) bank11495 +11496 POINT(41.059321237733286 73.7681169118803) bank11496 +11497 POINT(39.813482246843954 73.87553864525289) bank11497 +11498 POINT(40.179743606009154 74.08567843156564) bank11498 +11499 POINT(40.525844315694656 73.90240370605768) bank11499 +11500 POINT(40.258265176489 73.40692958857922) bank11500 +11501 POINT(40.424585687914515 73.80694049796024) bank11501 +11502 POINT(40.856536904258625 73.36486772577878) bank11502 +11503 POINT(40.87957304535574 74.18240817675552) bank11503 +11504 POINT(40.40577424586864 73.29659386118448) bank11504 +11505 POINT(39.78914279569896 73.09679100729362) bank11505 +11506 POINT(40.68540668117317 73.84425378631242) bank11506 +11507 POINT(40.21972856901805 73.22791877928425) bank11507 +11508 POINT(40.33368370127899 73.59496135579593) bank11508 +11509 POINT(41.547430806110114 74.61816169629482) bank11509 +11510 POINT(39.770355592313564 74.44874330803273) bank11510 +11511 POINT(41.181391335949485 73.46163905305353) bank11511 +11512 POINT(41.5088184238755 73.34701020325575) bank11512 +11513 POINT(41.53411748784618 73.26691283505251) bank11513 +11514 POINT(40.73023667044949 74.7090477778195) bank11514 +11515 POINT(39.77376322213928 73.32882210208169) bank11515 +11516 POINT(40.660825353016065 74.24651470650764) bank11516 +11517 POINT(41.004177883101626 73.17704465051573) bank11517 +11518 POINT(40.59930679569765 74.03967873804744) bank11518 +11519 POINT(40.60929538813867 74.93960769968062) bank11519 +11520 POINT(41.38236193432759 73.15691335933671) bank11520 +11521 POINT(40.39724777386551 74.18507812378027) bank11521 +11522 POINT(40.63106054765315 74.43799697475036) bank11522 +11523 POINT(40.779030391764095 73.34655306978969) bank11523 +11524 POINT(40.885234477290254 74.5105317444631) bank11524 +11525 POINT(39.93322923446172 73.56381624074511) bank11525 +11526 POINT(41.2758835606446 73.68432632206756) bank11526 +11527 POINT(40.11473900252545 74.19041815148391) bank11527 +11528 POINT(40.786810288649406 74.2451671589749) bank11528 +11529 POINT(41.65209348807972 73.07303183150493) bank11529 +11530 POINT(39.80057155834773 73.80881984266006) bank11530 +11531 POINT(40.025485744875695 73.73592575145645) bank11531 +11532 POINT(41.39116824385662 73.02485770263839) bank11532 +11533 POINT(40.07496013935078 73.42157387975718) bank11533 +11534 POINT(41.413655626232874 74.72844580801167) bank11534 +11535 POINT(41.21705525635591 74.61757773723154) bank11535 +11536 POINT(40.18245322113581 74.91741410110257) bank11536 +11537 POINT(39.78144565301983 74.2161428538866) bank11537 +11538 POINT(41.07173852852477 73.59808814660235) bank11538 +11539 POINT(40.90053171968558 73.62736080044999) bank11539 +11540 POINT(40.27942573131479 74.62032829812283) bank11540 +11541 POINT(40.78172571062966 73.10661911982021) bank11541 +11542 POINT(40.68873076028321 73.39002397078971) bank11542 +11543 POINT(39.93121736377635 73.27650037029404) bank11543 +11544 POINT(39.79975422694613 73.0429985437677) bank11544 +11545 POINT(41.1608888080119 73.03972648922355) bank11545 +11546 POINT(40.57014435347878 74.85696913510485) bank11546 +11547 POINT(40.49984095244846 73.6866550100226) bank11547 +11548 POINT(40.745123672711294 73.38102984864963) bank11548 +11549 POINT(39.78529579498823 73.75590263820212) bank11549 +11550 POINT(41.03883323029248 74.07247785884354) bank11550 +11551 POINT(40.69407285483188 74.67070871860227) bank11551 +11552 POINT(41.27890265345757 73.14099935588403) bank11552 +11553 POINT(40.44749714985401 73.50054395109588) bank11553 +11554 POINT(41.053935919741164 74.40524354398617) bank11554 +11555 POINT(41.14032459065234 74.77121627857898) bank11555 +11556 POINT(40.665867805760236 74.36338320552805) bank11556 +11557 POINT(40.42729074187639 73.92153328075975) bank11557 +11558 POINT(39.77479388012855 74.6531681650255) bank11558 +11559 POINT(40.15615198545607 73.40301948964537) bank11559 +11560 POINT(39.86625843636236 73.56756795577915) bank11560 +11561 POINT(41.080973517759375 73.6354586227439) bank11561 +11562 POINT(41.103815781711184 73.8916895272939) bank11562 +11563 POINT(40.438000383966 74.82996163968578) bank11563 +11564 POINT(40.24774815771242 74.79109479828776) bank11564 +11565 POINT(41.184878048666874 74.71035762751976) bank11565 +11566 POINT(41.521732960181794 73.89932656820825) bank11566 +11567 POINT(39.837443385115456 74.42176303534113) bank11567 +11568 POINT(40.06825094908824 73.13332601123366) bank11568 +11569 POINT(41.47450079971166 73.89539770434207) bank11569 +11570 POINT(40.803553237483804 73.2310123823961) bank11570 +11571 POINT(41.700027560452575 74.17039975922064) bank11571 +11572 POINT(41.28484958197737 73.72897932466219) bank11572 +11573 POINT(40.04881557693221 74.57064870969451) bank11573 +11574 POINT(40.49952544251394 74.29607085117331) bank11574 +11575 POINT(41.47441649575499 74.58201906042224) bank11575 +11576 POINT(41.47152904997563 73.19618619162878) bank11576 +11577 POINT(40.148617521343446 73.39360875281157) bank11577 +11578 POINT(39.905999721755826 73.64750025316398) bank11578 +11579 POINT(41.55509022570067 73.90933730425155) bank11579 +11580 POINT(40.50342738509603 73.84416033750499) bank11580 +11581 POINT(41.10689435466834 74.33436460567194) bank11581 +11582 POINT(40.65425254566662 73.13109159939091) bank11582 +11583 POINT(40.37809898265023 74.60205769439631) bank11583 +11584 POINT(41.58697170464319 74.37136867523637) bank11584 +11585 POINT(39.71928436072862 74.69204114855536) bank11585 +11586 POINT(41.5422792989096 73.44844851061289) bank11586 +11587 POINT(41.36203502685124 73.17357554421714) bank11587 +11588 POINT(39.86163676518705 73.8044671930974) bank11588 +11589 POINT(41.61811544445337 74.99491514271686) bank11589 +11590 POINT(40.78619520229081 73.52184630018276) bank11590 +11591 POINT(41.02758535858112 73.35998395846042) bank11591 +11592 POINT(39.97930193171678 74.99206952858218) bank11592 +11593 POINT(41.337546883161 73.43097897721569) bank11593 +11594 POINT(40.89176631864431 73.54278994348691) bank11594 +11595 POINT(40.75591064757516 73.30581275751452) bank11595 +11596 POINT(39.826743201197395 74.80374704550361) bank11596 +11597 POINT(40.63219772424643 74.774783809498) bank11597 +11598 POINT(40.38504737683591 73.24737366732629) bank11598 +11599 POINT(40.769788458495576 73.63279431948496) bank11599 +11600 POINT(39.87536907805813 73.62328286775016) bank11600 +11601 POINT(41.534247618513824 73.37027268298822) bank11601 +11602 POINT(39.81726835073654 73.63113452398058) bank11602 +11603 POINT(40.16070148340703 73.39844768489242) bank11603 +11604 POINT(40.409506489636435 73.23567214114517) bank11604 +11605 POINT(41.06305963049381 74.92503094329102) bank11605 +11606 POINT(41.47951462569884 74.11498768221617) bank11606 +11607 POINT(40.71879749521121 73.53548766198764) bank11607 +11608 POINT(41.4418839846653 74.22149155912479) bank11608 +11609 POINT(40.83738845024684 74.03717155410962) bank11609 +11610 POINT(41.606357689989544 73.20950718803371) bank11610 +11611 POINT(40.078921375192316 73.13779464357083) bank11611 +11612 POINT(39.71440311680631 73.66788868857932) bank11612 +11613 POINT(40.858830682818585 73.87746058484605) bank11613 +11614 POINT(40.76947965635495 74.72277726088456) bank11614 +11615 POINT(41.51498806227672 74.39286967107256) bank11615 +11616 POINT(40.11774576867249 73.9082076983352) bank11616 +11617 POINT(39.905676819773284 73.23684513612443) bank11617 +11618 POINT(40.02920868829211 73.8333104785895) bank11618 +11619 POINT(40.40289536774821 74.00394507230264) bank11619 +11620 POINT(40.442885204510475 74.46065496010537) bank11620 +11621 POINT(40.79766830269969 74.58629773594711) bank11621 +11622 POINT(41.243264841763235 74.39190181500541) bank11622 +11623 POINT(40.354128122897286 74.33280369893896) bank11623 +11624 POINT(39.82428527354863 74.78051534050395) bank11624 +11625 POINT(41.394758208925715 73.64813472489962) bank11625 +11626 POINT(39.75636028433678 73.22215524987114) bank11626 +11627 POINT(40.615391954823345 73.31174532350364) bank11627 +11628 POINT(40.250961913947165 73.14597771637924) bank11628 +11629 POINT(40.17323201736033 73.92961986206623) bank11629 +11630 POINT(41.690581794923844 74.53554256214194) bank11630 +11631 POINT(39.72068700991714 73.20510792471586) bank11631 +11632 POINT(40.62487640743606 73.48082716817913) bank11632 +11633 POINT(40.9178098859021 73.57963157455521) bank11633 +11634 POINT(40.91262363757073 73.34796681472308) bank11634 +11635 POINT(39.80867519284982 73.11525358924919) bank11635 +11636 POINT(41.2599557317264 74.46423952879333) bank11636 +11637 POINT(41.07409343511299 74.32480355970306) bank11637 +11638 POINT(40.250167669789036 74.9371867882238) bank11638 +11639 POINT(40.72972551141525 74.08803878634662) bank11639 +11640 POINT(40.21411779956622 73.47730696358649) bank11640 +11641 POINT(40.46484813323188 74.53275029428126) bank11641 +11642 POINT(39.90450093021857 74.58748343253905) bank11642 +11643 POINT(39.75542226354725 73.22371356150205) bank11643 +11644 POINT(40.00201577226557 74.62180035724447) bank11644 +11645 POINT(40.26600200554968 74.97302140820169) bank11645 +11646 POINT(39.873382556154546 74.44735745327198) bank11646 +11647 POINT(40.262430321689784 73.42138705959266) bank11647 +11648 POINT(40.67564969481942 73.67570792445258) bank11648 +11649 POINT(40.67651028214425 73.89086399414238) bank11649 +11650 POINT(40.1993999849975 73.59746732635153) bank11650 +11651 POINT(41.66681821532245 74.10126542742317) bank11651 +11652 POINT(39.72613593393365 73.04892171754783) bank11652 +11653 POINT(41.320864244015496 73.14853446322272) bank11653 +11654 POINT(40.392861694578315 74.12679974928358) bank11654 +11655 POINT(39.71725098371922 74.70972523732607) bank11655 +11656 POINT(41.19060642408284 73.20623747791181) bank11656 +11657 POINT(39.715393304807016 73.79254786386365) bank11657 +11658 POINT(40.95445140557409 73.88661970221754) bank11658 +11659 POINT(40.288937274379975 73.2130914865383) bank11659 +11660 POINT(40.24092402727556 73.58192147959831) bank11660 +11661 POINT(39.93022644545734 74.0724636377617) bank11661 +11662 POINT(40.93612550357174 73.885183604707) bank11662 +11663 POINT(40.1778217318427 74.87708350630865) bank11663 +11664 POINT(40.87211533912324 74.12107024577739) bank11664 +11665 POINT(40.649834101485055 74.3271319062713) bank11665 +11666 POINT(40.84079775606336 73.15472879737958) bank11666 +11667 POINT(41.20915345205571 73.23488853168989) bank11667 +11668 POINT(41.656277668668636 73.61204848876169) bank11668 +11669 POINT(41.09096768083697 73.97530776084172) bank11669 +11670 POINT(40.29546786940945 74.21709510148172) bank11670 +11671 POINT(40.32097863971406 73.80279615413991) bank11671 +11672 POINT(39.779786161279226 74.98345181771298) bank11672 +11673 POINT(40.30175899015616 73.6755212235061) bank11673 +11674 POINT(41.18134203329679 74.95739190467552) bank11674 +11675 POINT(40.93413952182855 73.41084148206464) bank11675 +11676 POINT(41.46547931352745 74.44341724546774) bank11676 +11677 POINT(40.46839330568051 74.9912366132) bank11677 +11678 POINT(40.855833502069295 74.75946670384018) bank11678 +11679 POINT(41.55413260621786 74.3476446982235) bank11679 +11680 POINT(40.95773425676018 74.23128698178027) bank11680 +11681 POINT(39.77303687963287 73.55597622862616) bank11681 +11682 POINT(39.97555682141368 73.38278170146451) bank11682 +11683 POINT(40.9682396005818 74.32694920036525) bank11683 +11684 POINT(40.61998670741895 74.51670557179534) bank11684 +11685 POINT(41.36070690612053 73.2919256304805) bank11685 +11686 POINT(40.12086943415126 74.53246162474053) bank11686 +11687 POINT(41.11256250351394 74.56084184606122) bank11687 +11688 POINT(40.95608700065666 73.26159658933493) bank11688 +11689 POINT(40.801121719458145 74.96306226487877) bank11689 +11690 POINT(41.69182659651136 74.74169881363048) bank11690 +11691 POINT(39.830995718760356 73.506852781001) bank11691 +11692 POINT(40.98614139263719 74.91289439152649) bank11692 +11693 POINT(41.36104392843374 74.53353180534901) bank11693 +11694 POINT(40.55347750584933 74.85768616433107) bank11694 +11695 POINT(40.27046034486741 73.57379348420484) bank11695 +11696 POINT(41.55389812494403 74.63322987262681) bank11696 +11697 POINT(41.64431156957829 74.55117812388623) bank11697 +11698 POINT(40.36922655694079 73.03847522437445) bank11698 +11699 POINT(41.059715809994145 74.94248767586114) bank11699 +11700 POINT(41.444149458761025 74.88066099705757) bank11700 +11701 POINT(40.61107520431789 73.46545546845566) bank11701 +11702 POINT(39.94585518944465 73.38820594341388) bank11702 +11703 POINT(41.165140831804166 73.65368756269551) bank11703 +11704 POINT(41.651926909103146 73.83969057236143) bank11704 +11705 POINT(41.24352473165828 73.800736058855) bank11705 +11706 POINT(40.40743866186132 74.60515202961425) bank11706 +11707 POINT(41.65874364511776 74.74044073824408) bank11707 +11708 POINT(40.52197635173651 73.89711285922264) bank11708 +11709 POINT(39.814131436466994 74.60588571591731) bank11709 +11710 POINT(40.62764773854222 74.55800093195496) bank11710 +11711 POINT(40.764505159188175 73.25315020234866) bank11711 +11712 POINT(40.074906422472395 73.9058319266862) bank11712 +11713 POINT(41.20822896055561 73.75297932934693) bank11713 +11714 POINT(40.05266842784763 74.39061936165378) bank11714 +11715 POINT(40.35414663271625 74.55376414559564) bank11715 +11716 POINT(41.225694658740224 73.20419253537537) bank11716 +11717 POINT(40.11908344683628 73.3658070770543) bank11717 +11718 POINT(40.50108149373185 74.1770491163138) bank11718 +11719 POINT(41.075805898055606 73.28716040439477) bank11719 +11720 POINT(39.77568077561681 73.1269405880029) bank11720 +11721 POINT(40.44639833058596 74.91464479497938) bank11721 +11722 POINT(40.43406843573295 74.85251048634228) bank11722 +11723 POINT(41.22863766967777 73.53799281002745) bank11723 +11724 POINT(40.09886655292679 74.05804729120119) bank11724 +11725 POINT(39.735952633275474 74.57454216189444) bank11725 +11726 POINT(41.56297008937198 73.88057528502763) bank11726 +11727 POINT(41.29532446969534 73.91478766402875) bank11727 +11728 POINT(40.00978350532917 74.91119038047789) bank11728 +11729 POINT(41.19123437331907 73.3088068033992) bank11729 +11730 POINT(40.046338998530665 74.79100442306843) bank11730 +11731 POINT(39.850781098476396 74.93956716605167) bank11731 +11732 POINT(40.272436760717184 74.35740399704686) bank11732 +11733 POINT(41.010551170645485 73.24200656070522) bank11733 +11734 POINT(39.92674360000244 74.4514976317192) bank11734 +11735 POINT(41.13644012701869 74.12237230202216) bank11735 +11736 POINT(41.55244096867601 74.7297377967086) bank11736 +11737 POINT(40.84483766931327 74.01199367110482) bank11737 +11738 POINT(41.597371943338175 73.63832151901269) bank11738 +11739 POINT(40.04509020819862 74.45567577086543) bank11739 +11740 POINT(41.019958971317735 74.69883382768037) bank11740 +11741 POINT(39.904709545087286 74.79821531288829) bank11741 +11742 POINT(40.569042774727556 74.88332055040092) bank11742 +11743 POINT(40.98942377085396 74.53414213243452) bank11743 +11744 POINT(40.88610157421913 73.40533933235777) bank11744 +11745 POINT(40.7739054576582 74.67917513883202) bank11745 +11746 POINT(41.3261397023478 73.84401223267318) bank11746 +11747 POINT(41.041844059481576 73.25142767590413) bank11747 +11748 POINT(39.95991496275193 74.03860160281037) bank11748 +11749 POINT(40.65734158500152 74.13181385430445) bank11749 +11750 POINT(40.04267418834447 74.722646497107) bank11750 +11751 POINT(40.777462616970475 73.56963421850439) bank11751 +11752 POINT(41.064374541723254 73.36973067361124) bank11752 +11753 POINT(40.44054854207654 73.69444705182386) bank11753 +11754 POINT(41.505093718336674 73.22551113380338) bank11754 +11755 POINT(41.13434929376324 73.96842053661778) bank11755 +11756 POINT(40.85387225293788 73.73713385364371) bank11756 +11757 POINT(41.00094658568936 73.6980021992038) bank11757 +11758 POINT(40.08877551952294 74.03025806541054) bank11758 +11759 POINT(41.62314538600599 73.63993625472304) bank11759 +11760 POINT(41.316335127379595 74.48962623622135) bank11760 +11761 POINT(41.16121171292305 73.0933240803864) bank11761 +11762 POINT(41.6606070790042 73.81756640734692) bank11762 +11763 POINT(40.67702148881197 73.52429710107813) bank11763 +11764 POINT(40.80502539221531 74.63336254510656) bank11764 +11765 POINT(39.799432286235515 74.59969940857803) bank11765 +11766 POINT(40.061051571111044 73.8784463856482) bank11766 +11767 POINT(41.604365186966035 73.42433920055244) bank11767 +11768 POINT(40.57007185481816 74.93178127160438) bank11768 +11769 POINT(41.52728997004788 73.73794742366591) bank11769 +11770 POINT(39.89402744355809 73.19650586211782) bank11770 +11771 POINT(40.74870537523688 74.25022825913682) bank11771 +11772 POINT(40.569364862527 73.78271053234195) bank11772 +11773 POINT(41.32364123727081 73.32936892669828) bank11773 +11774 POINT(41.0171324202292 74.45352379234009) bank11774 +11775 POINT(41.31250664074197 74.10710762428462) bank11775 +11776 POINT(41.182523730180264 73.27748858070925) bank11776 +11777 POINT(41.637485201354195 74.06179688170765) bank11777 +11778 POINT(41.57928466168133 73.5138418718278) bank11778 +11779 POINT(40.321972528499806 73.65674027140959) bank11779 +11780 POINT(41.581672702545454 73.021764912238) bank11780 +11781 POINT(40.60599914466339 73.92264391679738) bank11781 +11782 POINT(40.1469004484877 74.77213434751349) bank11782 +11783 POINT(40.0342530162669 74.90513219341287) bank11783 +11784 POINT(40.485532413511294 74.38709719107102) bank11784 +11785 POINT(40.93657757981342 74.44623022066132) bank11785 +11786 POINT(40.76066957964286 74.09032738296055) bank11786 +11787 POINT(39.7267786353286 74.8791174655697) bank11787 +11788 POINT(40.59190819748895 74.01174110147302) bank11788 +11789 POINT(39.93296450072272 74.85480655664522) bank11789 +11790 POINT(41.67362034405734 74.04900141276318) bank11790 +11791 POINT(39.96228077587854 73.50938185990287) bank11791 +11792 POINT(39.934619239668955 74.65113859219572) bank11792 +11793 POINT(40.94816095152514 74.51993633787754) bank11793 +11794 POINT(41.231653335089355 73.20920717279984) bank11794 +11795 POINT(40.241259889486464 74.45017376549404) bank11795 +11796 POINT(39.861890762208105 74.83552005621694) bank11796 +11797 POINT(41.362404466581594 74.90924786452995) bank11797 +11798 POINT(41.42786168103188 74.0433940421912) bank11798 +11799 POINT(40.04393619709307 74.53779375589099) bank11799 +11800 POINT(40.237738524517596 74.53139236242805) bank11800 +11801 POINT(40.292561465123214 74.75638783169197) bank11801 +11802 POINT(39.907915773263824 74.33387814821155) bank11802 +11803 POINT(41.530035568297 73.77170021622563) bank11803 +11804 POINT(40.90456920023631 73.6030906761001) bank11804 +11805 POINT(39.85382120786913 73.69787567072518) bank11805 +11806 POINT(40.94256209703673 74.03761456472252) bank11806 +11807 POINT(40.66218857184644 74.6958893251118) bank11807 +11808 POINT(39.96060945844323 74.11817844243477) bank11808 +11809 POINT(40.79665503300467 73.85637333161489) bank11809 +11810 POINT(40.1893429441288 74.00305760551402) bank11810 +11811 POINT(40.899085976700285 73.9815062384005) bank11811 +11812 POINT(41.60241773475245 74.12138110856053) bank11812 +11813 POINT(40.928468645049136 73.71484256399935) bank11813 +11814 POINT(40.74477157776274 74.51557468539129) bank11814 +11815 POINT(40.83033518751732 74.39789550826369) bank11815 +11816 POINT(40.35818831850214 73.3381626166845) bank11816 +11817 POINT(41.124131642018035 73.94391035213604) bank11817 +11818 POINT(41.175858748913704 73.03939115977545) bank11818 +11819 POINT(40.04021563754254 74.24935724969552) bank11819 +11820 POINT(39.73059651840278 73.80552572152307) bank11820 +11821 POINT(40.88938801955086 73.58637368297718) bank11821 +11822 POINT(40.76023702078258 74.87169610716235) bank11822 +11823 POINT(40.629193325970846 74.52185945092936) bank11823 +11824 POINT(41.48880537219601 74.4351060971102) bank11824 +11825 POINT(41.452799251261354 74.14469312446339) bank11825 +11826 POINT(41.454637815208976 73.30552161186641) bank11826 +11827 POINT(39.91631732295164 73.02099736818306) bank11827 +11828 POINT(40.07737263987196 74.49877161029238) bank11828 +11829 POINT(41.28160253824693 73.37747308212276) bank11829 +11830 POINT(40.54051080348944 73.27637055728934) bank11830 +11831 POINT(39.84481562375849 74.74684059906558) bank11831 +11832 POINT(40.16469679645534 73.75592963142535) bank11832 +11833 POINT(41.40058708764986 74.31659647821806) bank11833 +11834 POINT(41.356268539364386 74.78128973599712) bank11834 +11835 POINT(40.71944244086744 74.32491456320078) bank11835 +11836 POINT(41.205931817638636 73.1544923431319) bank11836 +11837 POINT(41.11001225743781 73.09285023610951) bank11837 +11838 POINT(40.35372573844511 73.87411286892987) bank11838 +11839 POINT(40.214664604690924 73.33997314617167) bank11839 +11840 POINT(40.756200912700535 74.34924887984667) bank11840 +11841 POINT(40.16216852992856 74.7420906347585) bank11841 +11842 POINT(40.617343070152636 74.15304900921124) bank11842 +11843 POINT(39.71883547603923 73.19418157043732) bank11843 +11844 POINT(41.06370435573975 73.98753860068527) bank11844 +11845 POINT(39.76067946673181 73.70377236272043) bank11845 +11846 POINT(40.28994530420647 73.74648671567113) bank11846 +11847 POINT(41.20576830523855 73.78852204609139) bank11847 +11848 POINT(39.768576873357794 74.53586958188454) bank11848 +11849 POINT(39.89432016629969 74.33343485082874) bank11849 +11850 POINT(41.25850865212274 73.76517378917215) bank11850 +11851 POINT(40.09606769277888 73.8124141002317) bank11851 +11852 POINT(41.25419034085022 74.6293432090975) bank11852 +11853 POINT(40.070308440904334 73.95598236990249) bank11853 +11854 POINT(40.77970940852458 73.83793050049486) bank11854 +11855 POINT(41.21348328817418 73.16363096195104) bank11855 +11856 POINT(41.389857114412926 73.06646061744262) bank11856 +11857 POINT(40.657337125901925 74.86364207732093) bank11857 +11858 POINT(41.554264138170765 73.07291757589027) bank11858 +11859 POINT(41.3170761843823 73.07362978099577) bank11859 +11860 POINT(40.31600348312503 73.9861035468145) bank11860 +11861 POINT(41.63225453501909 74.09247499042928) bank11861 +11862 POINT(40.974718367933484 74.1681195806398) bank11862 +11863 POINT(41.47828738705674 74.7285092188712) bank11863 +11864 POINT(40.79665348918558 73.43815324354195) bank11864 +11865 POINT(39.77375783520077 73.45008527108995) bank11865 +11866 POINT(41.09899406521864 74.24887961270942) bank11866 +11867 POINT(39.948263243884966 73.29851947290808) bank11867 +11868 POINT(40.93368420395977 73.5725331268863) bank11868 +11869 POINT(39.900067281116655 74.93276236252032) bank11869 +11870 POINT(39.95299046017033 74.85184105574321) bank11870 +11871 POINT(40.21923792761441 74.88864941149846) bank11871 +11872 POINT(40.16221742875765 74.15241295044522) bank11872 +11873 POINT(40.33162284417767 74.30823255046211) bank11873 +11874 POINT(39.84218746212668 73.30672582735566) bank11874 +11875 POINT(41.67350793510293 74.64766674094354) bank11875 +11876 POINT(40.73614453116324 73.67128378739663) bank11876 +11877 POINT(41.409101424981515 74.56006286147208) bank11877 +11878 POINT(39.977292851380945 73.8766917113648) bank11878 +11879 POINT(40.08613271123175 74.5852215637919) bank11879 +11880 POINT(41.005414030199795 73.12289485461862) bank11880 +11881 POINT(39.80505764857487 73.6371075866971) bank11881 +11882 POINT(40.836682188218596 74.88129664465096) bank11882 +11883 POINT(39.76108313279695 74.3343594659764) bank11883 +11884 POINT(41.24982531022821 74.49805723836863) bank11884 +11885 POINT(40.38561886286079 73.73546035602097) bank11885 +11886 POINT(41.03704793254171 73.68524872519066) bank11886 +11887 POINT(41.1952428805683 73.39379337590833) bank11887 +11888 POINT(40.4310570373116 73.83890550668092) bank11888 +11889 POINT(40.32436091342817 73.86706019119418) bank11889 +11890 POINT(40.52697161402062 74.57613063528063) bank11890 +11891 POINT(41.21830470373587 73.6460482131369) bank11891 +11892 POINT(40.65256169286084 74.06728205629729) bank11892 +11893 POINT(40.93572998030141 73.88640862845023) bank11893 +11894 POINT(40.523380134484626 74.31339803626464) bank11894 +11895 POINT(40.330397776394086 73.26994618396144) bank11895 +11896 POINT(40.00707478518263 74.968201656217) bank11896 +11897 POINT(40.50453718850897 74.19860554652854) bank11897 +11898 POINT(40.468261812239625 73.2336170099131) bank11898 +11899 POINT(40.50135331576375 74.04618460808696) bank11899 +11900 POINT(40.61437630692512 74.82672072209019) bank11900 +11901 POINT(41.173652960425684 73.97817867476446) bank11901 +11902 POINT(40.76743490230347 73.23414914130699) bank11902 +11903 POINT(41.60371406939903 73.11284143405918) bank11903 +11904 POINT(41.52094846786615 73.18732834549343) bank11904 +11905 POINT(40.7879969453811 74.20138758821844) bank11905 +11906 POINT(40.58718151202311 73.92180218238487) bank11906 +11907 POINT(40.7144919799557 74.58213117272606) bank11907 +11908 POINT(41.69300875172885 74.75049220369476) bank11908 +11909 POINT(40.83951032053734 74.03014052329647) bank11909 +11910 POINT(40.510439810861506 74.75291547761132) bank11910 +11911 POINT(40.2853443157439 73.11379897717237) bank11911 +11912 POINT(41.301661091804974 74.83756067108855) bank11912 +11913 POINT(40.582502250250535 74.26255616924769) bank11913 +11914 POINT(41.639065693273714 74.20329756024327) bank11914 +11915 POINT(40.85332723828749 73.55823712526951) bank11915 +11916 POINT(40.668828470617456 74.74533963866037) bank11916 +11917 POINT(39.85004560903461 73.02206520028996) bank11917 +11918 POINT(40.77733321111975 73.35420761628713) bank11918 +11919 POINT(40.90462171808753 74.31139550171564) bank11919 +11920 POINT(40.2286254842848 74.09526609463698) bank11920 +11921 POINT(41.5306329699243 74.14945746428204) bank11921 +11922 POINT(40.843933478629936 74.67094827549249) bank11922 +11923 POINT(40.9302796041455 74.60155942261316) bank11923 +11924 POINT(41.545603747038086 73.41126649899746) bank11924 +11925 POINT(39.95735713741362 74.59415466996575) bank11925 +11926 POINT(41.388398590527906 74.25230980495152) bank11926 +11927 POINT(41.60793490405649 74.39507482679169) bank11927 +11928 POINT(41.270493800392096 73.96328476849347) bank11928 +11929 POINT(40.92855671130496 73.14870193814544) bank11929 +11930 POINT(41.438909147375796 74.64486335480706) bank11930 +11931 POINT(40.26849795309587 74.46055236615297) bank11931 +11932 POINT(40.99849058497023 74.26752226137289) bank11932 +11933 POINT(41.03163564915037 73.95983958666123) bank11933 +11934 POINT(41.537634171631616 73.84982051059916) bank11934 +11935 POINT(40.55357768835252 74.41987039854371) bank11935 +11936 POINT(39.74215069701969 74.59650262552024) bank11936 +11937 POINT(41.229497022931255 74.25531197836446) bank11937 +11938 POINT(40.94909991123393 73.21228129570777) bank11938 +11939 POINT(40.53817062103241 74.82604781849108) bank11939 +11940 POINT(39.82901878075849 73.9804828664007) bank11940 +11941 POINT(41.57925812549523 73.17048070398754) bank11941 +11942 POINT(41.11038844973772 73.06804179488952) bank11942 +11943 POINT(40.230011167687756 73.08128775528228) bank11943 +11944 POINT(41.3071883252199 74.12007292741113) bank11944 +11945 POINT(41.38110177111742 74.24029260507896) bank11945 +11946 POINT(40.776519899164725 73.05558107103174) bank11946 +11947 POINT(41.28873303464157 74.0696126499065) bank11947 +11948 POINT(41.400897251947 73.1362236956147) bank11948 +11949 POINT(39.88466373162911 73.59087940948673) bank11949 +11950 POINT(40.76706448886188 73.50156254927157) bank11950 +11951 POINT(40.96257198437482 73.13204734276002) bank11951 +11952 POINT(39.97028129073415 73.20902068512238) bank11952 +11953 POINT(41.28947172443886 74.28742637871247) bank11953 +11954 POINT(40.62055849354008 74.94976054797426) bank11954 +11955 POINT(40.39364207412139 74.6059813396282) bank11955 +11956 POINT(40.30116386057398 74.06580316456736) bank11956 +11957 POINT(41.62170450701234 73.97898889876704) bank11957 +11958 POINT(41.07693285207136 73.63133417855504) bank11958 +11959 POINT(40.848402194682635 74.51923544580772) bank11959 +11960 POINT(39.78434583984666 73.88562636487288) bank11960 +11961 POINT(41.700145351084046 74.2956385399604) bank11961 +11962 POINT(41.11287212894693 74.17405741889145) bank11962 +11963 POINT(40.031166278836615 74.74663829392708) bank11963 +11964 POINT(40.599596577435236 74.46381085148087) bank11964 +11965 POINT(40.752680640023684 73.25404526334071) bank11965 +11966 POINT(40.69634438552811 74.06870690645472) bank11966 +11967 POINT(41.4517483807443 74.87126559968739) bank11967 +11968 POINT(41.46687512585538 73.75856741219023) bank11968 +11969 POINT(39.92976271282003 73.0607224307051) bank11969 +11970 POINT(39.9371088259554 74.9147256967507) bank11970 +11971 POINT(41.31756167637636 73.712904815711) bank11971 +11972 POINT(40.747965535852195 73.98048567252155) bank11972 +11973 POINT(40.47019871284943 74.80499710381741) bank11973 +11974 POINT(39.737194127330326 74.73718663225785) bank11974 +11975 POINT(40.88908477237856 73.0463113751254) bank11975 +11976 POINT(39.98054890170022 73.90841094144938) bank11976 +11977 POINT(40.262455978524855 73.6596517457809) bank11977 +11978 POINT(40.78052800257534 74.94351191734773) bank11978 +11979 POINT(41.23548911466808 74.27214834774172) bank11979 +11980 POINT(41.33671949422961 73.86819179057282) bank11980 +11981 POINT(40.221728728379354 74.7927205846047) bank11981 +11982 POINT(39.98603734208612 73.2120826383072) bank11982 +11983 POINT(39.89925748526232 73.45896676259035) bank11983 +11984 POINT(40.198829741782994 74.56857234461812) bank11984 +11985 POINT(41.16308811999982 74.38882694316183) bank11985 +11986 POINT(41.56528457596687 73.05596139864095) bank11986 +11987 POINT(41.56255255171242 73.09393612998748) bank11987 +11988 POINT(40.13086543278952 73.37147360328294) bank11988 +11989 POINT(39.799173967924844 74.12940635556063) bank11989 +11990 POINT(40.00243143982297 73.56755116649099) bank11990 +11991 POINT(41.19330830890888 74.01469050716214) bank11991 +11992 POINT(40.664837417023485 74.68456638128033) bank11992 +11993 POINT(39.89028931197125 74.79744040787695) bank11993 +11994 POINT(40.16304543863355 73.78084367143005) bank11994 +11995 POINT(41.65343434132845 74.85511217351133) bank11995 +11996 POINT(40.136061546358256 74.03666032066161) bank11996 +11997 POINT(41.49705416162188 73.98113527824155) bank11997 +11998 POINT(40.129940500268454 73.52010560035667) bank11998 +11999 POINT(40.40611785595197 73.22995671783515) bank11999 +12000 POINT(41.12993422611788 73.80179398958711) bank12000 +12001 POINT(41.61369622356745 74.63715863118846) bank12001 +12002 POINT(41.31112317915153 74.32255121077253) bank12002 +12003 POINT(41.59965572975649 74.9907573819526) bank12003 +12004 POINT(40.97601669291362 74.57188955800214) bank12004 +12005 POINT(41.67434698950327 73.2012991065542) bank12005 +12006 POINT(40.18801071343493 73.47902411499743) bank12006 +12007 POINT(40.470496517613306 74.25885875103633) bank12007 +12008 POINT(40.511390095319086 73.86489496870682) bank12008 +12009 POINT(40.91742269871572 74.71567381427015) bank12009 +12010 POINT(40.7803864370027 74.12121568509082) bank12010 +12011 POINT(41.52613608500704 74.77822310336052) bank12011 +12012 POINT(41.13402867267809 73.6601417924888) bank12012 +12013 POINT(41.44261333867981 74.28373346085459) bank12013 +12014 POINT(40.42347129816042 74.87434335903448) bank12014 +12015 POINT(41.39935049585333 74.05877292016231) bank12015 +12016 POINT(39.8639807413378 73.0426047052658) bank12016 +12017 POINT(40.03189205382669 73.19018104173503) bank12017 +12018 POINT(40.72224466975333 73.71650395423241) bank12018 +12019 POINT(39.98510441723737 74.5787627802178) bank12019 +12020 POINT(40.57284353715608 74.67386650813017) bank12020 +12021 POINT(41.56116492628206 74.67949247694838) bank12021 +12022 POINT(39.910159337369876 73.25869894580119) bank12022 +12023 POINT(41.6561729635456 73.78647821928517) bank12023 +12024 POINT(41.57531282380935 74.7866042071125) bank12024 +12025 POINT(40.94234483973337 74.20462196093231) bank12025 +12026 POINT(40.83308849736371 73.392047875972) bank12026 +12027 POINT(39.90265481012512 73.44444017446021) bank12027 +12028 POINT(41.700572254364474 74.5120657533538) bank12028 +12029 POINT(40.738880278339664 73.37726685569312) bank12029 +12030 POINT(41.17118525163738 74.78545780206993) bank12030 +12031 POINT(41.23417267598309 73.13014488597437) bank12031 +12032 POINT(40.48743356244658 73.29891058174067) bank12032 +12033 POINT(41.57395838565196 73.76209441048789) bank12033 +12034 POINT(41.27356260584853 73.21994642989705) bank12034 +12035 POINT(41.71216176543876 73.65714497471994) bank12035 +12036 POINT(40.021255795129655 74.9765408043282) bank12036 +12037 POINT(40.93409739258532 74.80864378848348) bank12037 +12038 POINT(40.917601106163026 74.70852599695553) bank12038 +12039 POINT(40.93791736270117 74.5820023415626) bank12039 +12040 POINT(41.47394142811775 74.42245395840791) bank12040 +12041 POINT(40.96952876988516 74.59125103220936) bank12041 +12042 POINT(41.23044583849227 73.25571355034172) bank12042 +12043 POINT(40.37179220174823 74.92209577517472) bank12043 +12044 POINT(40.674999626606564 73.44781791147923) bank12044 +12045 POINT(40.35329428603657 74.7211030524881) bank12045 +12046 POINT(40.51766778185678 74.56388892597516) bank12046 +12047 POINT(40.70555647857212 74.73587361051251) bank12047 +12048 POINT(41.036049306180985 73.73222488823058) bank12048 +12049 POINT(41.21189692249819 74.77238921607635) bank12049 +12050 POINT(40.69316393068346 73.87750578497426) bank12050 +12051 POINT(41.60947003125532 73.6680349127422) bank12051 +12052 POINT(41.01841643395889 73.79797639365384) bank12052 +12053 POINT(40.572296226707024 74.00473499335097) bank12053 +12054 POINT(41.033355789838744 73.19898952763462) bank12054 +12055 POINT(41.087754835600855 74.93208181374293) bank12055 +12056 POINT(40.447755537620885 73.95263103370246) bank12056 +12057 POINT(41.54687605022043 74.47746013620048) bank12057 +12058 POINT(40.974381575299304 73.61452090393098) bank12058 +12059 POINT(40.266630150143556 74.1412679813852) bank12059 +12060 POINT(40.807610732977565 73.63292138057093) bank12060 +12061 POINT(40.622869094597185 73.07964397716106) bank12061 +12062 POINT(39.83985939662372 73.5527232723519) bank12062 +12063 POINT(41.12023691061074 74.81936036968696) bank12063 +12064 POINT(40.63791710171864 74.70391394888108) bank12064 +12065 POINT(41.71161681529327 73.26803391579945) bank12065 +12066 POINT(40.37118567202972 73.90623934443813) bank12066 +12067 POINT(41.139549102232984 74.14142803751069) bank12067 +12068 POINT(40.63518313584805 73.84787079028693) bank12068 +12069 POINT(39.892394845737755 73.24703511034997) bank12069 +12070 POINT(40.290771793393056 74.50463432249535) bank12070 +12071 POINT(39.84332621365192 73.85054921782427) bank12071 +12072 POINT(41.42411493014686 73.02272831975971) bank12072 +12073 POINT(41.22545396031628 74.15093508336959) bank12073 +12074 POINT(41.640669278940535 73.67610085098488) bank12074 +12075 POINT(41.275400600432825 74.82478073628177) bank12075 +12076 POINT(39.853711004992675 73.64621456201735) bank12076 +12077 POINT(41.275299251934904 73.83535763218849) bank12077 +12078 POINT(41.71194603570961 73.63622725462191) bank12078 +12079 POINT(40.34540359056136 74.53559965886207) bank12079 +12080 POINT(40.73273133667544 73.27233333433705) bank12080 +12081 POINT(40.42687801174734 74.78556646178916) bank12081 +12082 POINT(41.23514919613932 73.68658016477355) bank12082 +12083 POINT(41.63183644716141 73.04693365659422) bank12083 +12084 POINT(41.10383945827966 74.72597450221706) bank12084 +12085 POINT(40.28650939285735 74.04784307565401) bank12085 +12086 POINT(40.43190786612088 73.23176126316896) bank12086 +12087 POINT(40.8724161611359 73.67318901627131) bank12087 +12088 POINT(41.634472593762084 74.99416304278431) bank12088 +12089 POINT(40.232658243077196 74.85960273384514) bank12089 +12090 POINT(40.1058465399225 73.03528097709345) bank12090 +12091 POINT(41.059249406938534 73.49499608181422) bank12091 +12092 POINT(41.18290310020827 73.43261323466196) bank12092 +12093 POINT(40.34003204142444 74.72689304742582) bank12093 +12094 POINT(40.700132145398044 74.96635518523054) bank12094 +12095 POINT(40.72360078508936 74.62479621176415) bank12095 +12096 POINT(39.78207823433759 73.4983989481788) bank12096 +12097 POINT(41.12274253182199 73.27977449652633) bank12097 +12098 POINT(40.07733636414272 74.62907283493156) bank12098 +12099 POINT(40.079045009028846 73.4245934709453) bank12099 +12100 POINT(40.527122338240474 74.06324303260965) bank12100 +12101 POINT(41.30423468299916 73.91082832618132) bank12101 +12102 POINT(40.85510145827954 73.05728153403534) bank12102 +12103 POINT(39.88735426069562 73.77551029692972) bank12103 +12104 POINT(40.34080930906276 73.38473537488693) bank12104 +12105 POINT(40.44536828754931 73.90933330155482) bank12105 +12106 POINT(41.68778283034963 73.05668268967827) bank12106 +12107 POINT(40.72994675525021 73.61869037120559) bank12107 +12108 POINT(41.02834710663358 74.87820726808077) bank12108 +12109 POINT(41.50325941285284 73.20125528955506) bank12109 +12110 POINT(39.78886565990061 73.20153194179113) bank12110 +12111 POINT(41.219932869418614 74.93970756176131) bank12111 +12112 POINT(40.762636123194156 74.90185102438093) bank12112 +12113 POINT(40.918840687321826 73.54035931527547) bank12113 +12114 POINT(40.577888426996196 74.8261709675856) bank12114 +12115 POINT(41.11792400460693 74.7794569250135) bank12115 +12116 POINT(41.50432759968394 74.3379704854176) bank12116 +12117 POINT(40.721272263469636 74.67685661674943) bank12117 +12118 POINT(40.85395099623115 73.3547077074146) bank12118 +12119 POINT(41.538716670493805 73.64380880512414) bank12119 +12120 POINT(40.80278235588624 74.03693685014906) bank12120 +12121 POINT(39.77166238425401 73.22053776748984) bank12121 +12122 POINT(40.94058626105438 73.69719731878284) bank12122 +12123 POINT(39.949554690607435 73.29727128610497) bank12123 +12124 POINT(40.57690318920165 74.64866774339367) bank12124 +12125 POINT(41.29625664270085 74.02563031033245) bank12125 +12126 POINT(41.3531105173828 73.88379809331069) bank12126 +12127 POINT(40.0990857122436 74.93797277303945) bank12127 +12128 POINT(40.452799292865066 74.58061078428842) bank12128 +12129 POINT(40.23986889498768 74.11406925102173) bank12129 +12130 POINT(41.03391540270155 73.24363722756357) bank12130 +12131 POINT(41.466343821626666 74.461090540613) bank12131 +12132 POINT(41.69367437150146 74.9170835782029) bank12132 +12133 POINT(40.95956329630297 74.1993218662459) bank12133 +12134 POINT(40.722916017965446 74.17971158727576) bank12134 +12135 POINT(40.31080972656174 74.89205923833346) bank12135 +12136 POINT(41.33475301640177 73.78308774627942) bank12136 +12137 POINT(41.5736906041295 74.9516701197077) bank12137 +12138 POINT(40.40439084939233 74.60958614742741) bank12138 +12139 POINT(40.07731833080858 73.2688901309348) bank12139 +12140 POINT(41.23421452010634 74.01965429912377) bank12140 +12141 POINT(40.47822049250507 73.59350540839516) bank12141 +12142 POINT(40.58524606362805 73.49532798544591) bank12142 +12143 POINT(40.225040682122355 74.63810236363642) bank12143 +12144 POINT(40.930503023019796 74.34625725110155) bank12144 +12145 POINT(39.855755768763814 73.01412426367143) bank12145 +12146 POINT(41.67856092280267 74.67792287201573) bank12146 +12147 POINT(41.24488511699712 73.49557068162221) bank12147 +12148 POINT(41.5131023933045 73.9008096586096) bank12148 +12149 POINT(41.26943029073081 73.72711522446514) bank12149 +12150 POINT(41.33936949916038 74.72329010642464) bank12150 +12151 POINT(40.45089302527181 74.34169251880142) bank12151 +12152 POINT(40.10773759241339 73.07545581669237) bank12152 +12153 POINT(40.90989490695625 74.4514146141022) bank12153 +12154 POINT(41.5408324098857 73.06633943951167) bank12154 +12155 POINT(40.209211674082695 74.12172129284163) bank12155 +12156 POINT(40.56453117712564 73.45134708571685) bank12156 +12157 POINT(41.58784578261187 74.03884726998987) bank12157 +12158 POINT(40.734918106613534 74.82042050910376) bank12158 +12159 POINT(40.36934425275993 73.42626596834718) bank12159 +12160 POINT(39.85191235840063 74.53639628250447) bank12160 +12161 POINT(41.690328581806206 73.00960775286444) bank12161 +12162 POINT(40.710084945639345 74.77285399279808) bank12162 +12163 POINT(41.28178125430192 74.79312316429841) bank12163 +12164 POINT(39.853147070398 73.51527997326488) bank12164 +12165 POINT(41.448143737445335 74.76459782245412) bank12165 +12166 POINT(40.43043735801885 74.72382946734204) bank12166 +12167 POINT(40.850545990731696 74.89923764282284) bank12167 +12168 POINT(41.14473077667676 74.04699045181188) bank12168 +12169 POINT(40.37279319023854 74.6573420605413) bank12169 +12170 POINT(40.61571718172113 73.90212803403855) bank12170 +12171 POINT(40.6184284614601 74.65345300128385) bank12171 +12172 POINT(41.30901140801175 73.4167534902853) bank12172 +12173 POINT(39.75733999141292 74.35839038864557) bank12173 +12174 POINT(40.97446678903677 73.92036429262038) bank12174 +12175 POINT(40.640508457972494 73.23411221255171) bank12175 +12176 POINT(41.38407583603194 74.97273567208397) bank12176 +12177 POINT(39.90439368965652 74.5951696538446) bank12177 +12178 POINT(40.56590124913122 74.20122289706117) bank12178 +12179 POINT(40.18567112948408 74.84843755296043) bank12179 +12180 POINT(40.05290401102187 74.42838424070341) bank12180 +12181 POINT(40.72452500741488 73.94216831752738) bank12181 +12182 POINT(40.61127028204254 74.94771690272708) bank12182 +12183 POINT(40.09424115900153 74.38583536270532) bank12183 +12184 POINT(40.01111565202269 73.15018442504311) bank12184 +12185 POINT(39.9822203043822 73.82479684068937) bank12185 +12186 POINT(41.06873758246292 73.66131991615468) bank12186 +12187 POINT(40.27058701918584 73.12088221350625) bank12187 +12188 POINT(41.22856047640285 73.73960223202126) bank12188 +12189 POINT(40.79037492977268 73.22707683617777) bank12189 +12190 POINT(39.90399794586061 73.1439044674594) bank12190 +12191 POINT(41.275314618252 73.08169700023747) bank12191 +12192 POINT(41.69906610839695 73.94322370596704) bank12192 +12193 POINT(41.6025550715798 73.3232377252011) bank12193 +12194 POINT(41.37614487041872 74.05767660043337) bank12194 +12195 POINT(40.238902281345176 73.78333331270214) bank12195 +12196 POINT(40.4549873395525 74.90374352372233) bank12196 +12197 POINT(40.57237027927001 73.16777507497092) bank12197 +12198 POINT(39.877366467603885 74.49701422406933) bank12198 +12199 POINT(41.34592608294229 74.48464216418797) bank12199 +12200 POINT(40.11951570619398 73.63603623062832) bank12200 +12201 POINT(41.297998448150786 73.16337369965453) bank12201 +12202 POINT(41.06021108729814 74.90966662530441) bank12202 +12203 POINT(39.93266862497891 73.50854124042792) bank12203 +12204 POINT(40.595198970688664 74.9639721758873) bank12204 +12205 POINT(40.943735952918686 73.61691601336383) bank12205 +12206 POINT(40.00200671554051 74.48510462990465) bank12206 +12207 POINT(39.78661185915145 74.9913197139668) bank12207 +12208 POINT(41.67729855057115 73.8016756868209) bank12208 +12209 POINT(41.49257512140067 73.69667323498514) bank12209 +12210 POINT(40.201473489769754 74.6041808122029) bank12210 +12211 POINT(41.572681546532536 74.28643190537092) bank12211 +12212 POINT(40.32410432581916 73.49882161571627) bank12212 +12213 POINT(40.61421178301113 73.70768864573176) bank12213 +12214 POINT(39.72448682714945 74.60899642187496) bank12214 +12215 POINT(41.61490664159416 73.37010092289938) bank12215 +12216 POINT(41.60401241173567 73.78306365711191) bank12216 +12217 POINT(40.76700510273339 73.77803806369917) bank12217 +12218 POINT(40.1872467882171 74.42167025352428) bank12218 +12219 POINT(41.590962943117525 73.62193908007305) bank12219 +12220 POINT(40.70705154943625 73.84022016256486) bank12220 +12221 POINT(39.83531080286603 74.18999269426533) bank12221 +12222 POINT(40.31537498326545 73.94998154799472) bank12222 +12223 POINT(39.770589949474484 73.05122617128453) bank12223 +12224 POINT(41.37634226964891 74.0563809730341) bank12224 +12225 POINT(40.247909016940596 73.71239934878886) bank12225 +12226 POINT(40.68011109708675 73.8622869597815) bank12226 +12227 POINT(40.000117471950276 73.58346105059204) bank12227 +12228 POINT(40.23280552969689 74.23281722418805) bank12228 +12229 POINT(41.18538383190081 74.31058765128323) bank12229 +12230 POINT(40.03131042405589 74.15995225755294) bank12230 +12231 POINT(40.539933903738735 74.47679791728858) bank12231 +12232 POINT(41.5753279672284 73.65410664200297) bank12232 +12233 POINT(40.76491495985348 73.41112316016873) bank12233 +12234 POINT(41.2230915369787 74.03453904790764) bank12234 +12235 POINT(39.890953628027475 73.45605654686075) bank12235 +12236 POINT(40.59842107617627 74.21137536600244) bank12236 +12237 POINT(40.852697018358754 74.3931409306398) bank12237 +12238 POINT(40.01471903240583 73.75583599004308) bank12238 +12239 POINT(40.38917799745943 73.65815899997014) bank12239 +12240 POINT(41.50486348142079 73.47980794848385) bank12240 +12241 POINT(41.44782845292792 74.08309396451021) bank12241 +12242 POINT(41.25648196875811 73.63247779880587) bank12242 +12243 POINT(41.01252425866192 73.66000223984702) bank12243 +12244 POINT(40.620357355438905 73.61520979653285) bank12244 +12245 POINT(41.62884332785998 74.9863867031822) bank12245 +12246 POINT(40.099910099666126 74.59422208052277) bank12246 +12247 POINT(41.24092835842388 73.4151994685134) bank12247 +12248 POINT(41.25068850717619 73.98764730498164) bank12248 +12249 POINT(40.609607613991116 73.77774646814972) bank12249 +12250 POINT(40.854063710372465 73.8076870570752) bank12250 +12251 POINT(40.4650297092597 73.28323727997048) bank12251 +12252 POINT(40.027086185008095 74.85069454318908) bank12252 +12253 POINT(40.49722745425379 73.19405090616168) bank12253 +12254 POINT(41.63918669539467 73.41112779576211) bank12254 +12255 POINT(40.0601248256435 73.40890683064306) bank12255 +12256 POINT(41.147034939423705 73.98079963272535) bank12256 +12257 POINT(39.84094459905383 74.0289280538185) bank12257 +12258 POINT(40.861229499048314 73.82140815116091) bank12258 +12259 POINT(40.456863029626 74.29258740011964) bank12259 +12260 POINT(40.45198540058919 74.16790135400016) bank12260 +12261 POINT(39.92199047520515 73.31121901272435) bank12261 +12262 POINT(40.508408239204776 73.80945642939713) bank12262 +12263 POINT(40.175014732758825 74.04717270130077) bank12263 +12264 POINT(40.43111288196574 74.02652972129671) bank12264 +12265 POINT(40.96743384465306 73.78219330507747) bank12265 +12266 POINT(39.962424882549236 73.42300436278143) bank12266 +12267 POINT(40.76256936224296 74.41066666135674) bank12267 +12268 POINT(41.58013079460719 73.34754723026953) bank12268 +12269 POINT(40.957592432925516 73.74218192519007) bank12269 +12270 POINT(40.28130019934398 74.06172763931819) bank12270 +12271 POINT(40.17885237988739 75.0002272004289) bank12271 +12272 POINT(40.9636336266635 73.1325097368557) bank12272 +12273 POINT(41.70839852209689 73.97575100680909) bank12273 +12274 POINT(41.09960984410247 73.8195163950685) bank12274 +12275 POINT(40.89514610720957 74.38906552963972) bank12275 +12276 POINT(41.4290620330149 74.22264692106566) bank12276 +12277 POINT(41.151867347654786 73.32514742539895) bank12277 +12278 POINT(40.83690785348786 74.0782127176772) bank12278 +12279 POINT(40.74028667300352 74.64062928642444) bank12279 +12280 POINT(41.491291482127956 74.824256806828) bank12280 +12281 POINT(41.25363418010204 74.74596866805058) bank12281 +12282 POINT(41.00192283827424 73.69774262471178) bank12282 +12283 POINT(41.41300500367584 73.41593759140036) bank12283 +12284 POINT(40.95942244074845 73.27816269472616) bank12284 +12285 POINT(40.703938524527416 74.17244015382241) bank12285 +12286 POINT(39.828469170518666 74.40492643760173) bank12286 +12287 POINT(41.03562274776624 73.67300147781657) bank12287 +12288 POINT(41.03621094806214 74.53009483270229) bank12288 +12289 POINT(41.67424402634469 73.26909849895651) bank12289 +12290 POINT(41.40622460145491 74.72869520713489) bank12290 +12291 POINT(40.53288668809009 73.43387484021349) bank12291 +12292 POINT(40.40167117968951 73.29778510895308) bank12292 +12293 POINT(39.986093326425554 74.74002362816589) bank12293 +12294 POINT(40.716401256909805 73.46095396386679) bank12294 +12295 POINT(39.71580219014106 74.13463763941336) bank12295 +12296 POINT(40.94629381395687 73.77493867793027) bank12296 +12297 POINT(40.03462333687643 74.69980552281744) bank12297 +12298 POINT(40.087239848723186 73.09059122995603) bank12298 +12299 POINT(40.78057940780873 74.14283609234728) bank12299 +12300 POINT(41.50766154056861 74.561067779995) bank12300 +12301 POINT(40.94865493325548 74.76071609673781) bank12301 +12302 POINT(41.54994009974862 73.25698794735352) bank12302 +12303 POINT(40.244953634082165 73.61734314292835) bank12303 +12304 POINT(41.31879502102136 74.58030364279564) bank12304 +12305 POINT(39.790043700250635 73.64452567661971) bank12305 +12306 POINT(39.72092450143716 74.79752938472721) bank12306 +12307 POINT(40.31554609540903 74.63959521823062) bank12307 +12308 POINT(41.55683933435191 74.13501257687828) bank12308 +12309 POINT(41.046542402998206 73.04958009670408) bank12309 +12310 POINT(40.754947674803056 73.7076012582627) bank12310 +12311 POINT(40.78329168911801 73.4292983973587) bank12311 +12312 POINT(40.45702021680294 73.83644741258647) bank12312 +12313 POINT(40.7175864413606 73.44833336352147) bank12313 +12314 POINT(41.50378125313067 74.11417032091063) bank12314 +12315 POINT(39.753607415798925 73.67080445917478) bank12315 +12316 POINT(41.62696270954647 74.58510048527673) bank12316 +12317 POINT(41.54525861843328 73.51195329061461) bank12317 +12318 POINT(40.06287426985452 74.41537054798235) bank12318 +12319 POINT(41.37751041320035 74.59999752507024) bank12319 +12320 POINT(39.743962681965975 74.70608211281132) bank12320 +12321 POINT(40.673467177899916 73.78255701774077) bank12321 +12322 POINT(41.152728541100984 74.45524914831726) bank12322 +12323 POINT(40.50191973081184 74.3719595923282) bank12323 +12324 POINT(40.5196894524863 74.87176794797077) bank12324 +12325 POINT(41.10615889871059 73.20804043947342) bank12325 +12326 POINT(41.029391773396306 73.22149249150772) bank12326 +12327 POINT(40.38798522635772 73.16904101891451) bank12327 +12328 POINT(40.197397418730205 74.73044123751014) bank12328 +12329 POINT(41.53579771215464 73.43848921385799) bank12329 +12330 POINT(40.9075350910861 74.06013269461293) bank12330 +12331 POINT(40.42311730778644 74.39813242190382) bank12331 +12332 POINT(41.34627690311879 73.2911117486799) bank12332 +12333 POINT(40.45756743789961 74.92278200668463) bank12333 +12334 POINT(41.60839321933431 73.59929910346604) bank12334 +12335 POINT(41.10591060059819 74.3419353778561) bank12335 +12336 POINT(40.53233784430976 73.87120674974221) bank12336 +12337 POINT(40.52913393390468 73.55514030768433) bank12337 +12338 POINT(40.15875341608894 73.10756700533881) bank12338 +12339 POINT(40.83473542994879 74.08827298874611) bank12339 +12340 POINT(41.206888584148246 74.45321398815705) bank12340 +12341 POINT(40.9175570369476 73.83929127322249) bank12341 +12342 POINT(41.56387730987046 73.7766316660493) bank12342 +12343 POINT(40.183972394352615 74.68875560085962) bank12343 +12344 POINT(40.45865877385628 74.19317911363997) bank12344 +12345 POINT(41.55061271703307 74.70548878929648) bank12345 +12346 POINT(39.8619871983145 74.06841481026062) bank12346 +12347 POINT(39.78456079631914 73.94006020425293) bank12347 +12348 POINT(41.6995726275348 74.99298563511326) bank12348 +12349 POINT(40.84043503742334 74.85284076649657) bank12349 +12350 POINT(39.928080542189846 73.81176213446922) bank12350 +12351 POINT(40.135671857525 74.8053902897827) bank12351 +12352 POINT(39.718453323624914 74.09355891751404) bank12352 +12353 POINT(41.128464623958045 73.62699979811725) bank12353 +12354 POINT(40.99375066421191 73.12085975563627) bank12354 +12355 POINT(39.99003476254236 74.19518886871026) bank12355 +12356 POINT(41.70494971255888 74.83059980900336) bank12356 +12357 POINT(41.113800936509406 74.01085376614948) bank12357 +12358 POINT(40.72185831513031 73.65407070826532) bank12358 +12359 POINT(41.124073310690534 74.33619002253306) bank12359 +12360 POINT(40.72628585405717 73.65559139514069) bank12360 +12361 POINT(41.34356374881312 73.56283999106435) bank12361 +12362 POINT(40.42557366655546 73.13007211640343) bank12362 +12363 POINT(40.542511498966604 73.47463624382868) bank12363 +12364 POINT(39.909917831811974 73.04174843465233) bank12364 +12365 POINT(40.54647017755582 74.30081875002554) bank12365 +12366 POINT(41.661783897081406 73.81964204246471) bank12366 +12367 POINT(40.813776313018444 73.23358447558687) bank12367 +12368 POINT(40.17065489561304 74.93169009515864) bank12368 +12369 POINT(40.81007307632821 74.13434118284934) bank12369 +12370 POINT(40.10895420173672 74.9165472161227) bank12370 +12371 POINT(41.362484190686246 73.41974862049977) bank12371 +12372 POINT(40.77604083410414 74.61665046110944) bank12372 +12373 POINT(41.043478692491526 74.45318267225605) bank12373 +12374 POINT(41.21636020154867 74.53845763724887) bank12374 +12375 POINT(40.34578706510635 73.72180372575166) bank12375 +12376 POINT(40.64615490600212 74.42318177451068) bank12376 +12377 POINT(40.458169955044795 74.07718861129713) bank12377 +12378 POINT(40.18395683109461 73.74684712589627) bank12378 +12379 POINT(39.79626227095115 73.79672174669734) bank12379 +12380 POINT(40.1786243521362 74.451690824366) bank12380 +12381 POINT(40.4460351661499 73.99435643550125) bank12381 +12382 POINT(39.71831729118275 73.03534887747519) bank12382 +12383 POINT(41.6159091951143 74.85578427454904) bank12383 +12384 POINT(41.4735433243964 74.72816094987859) bank12384 +12385 POINT(41.47824215371915 73.76785380193239) bank12385 +12386 POINT(39.81241979503096 74.38973774750721) bank12386 +12387 POINT(40.1937626043637 73.32238615871978) bank12387 +12388 POINT(39.930617079014596 73.36180452376774) bank12388 +12389 POINT(39.76023146292308 73.54636928214592) bank12389 +12390 POINT(41.084444546195826 74.87874456464603) bank12390 +12391 POINT(41.42348735750191 74.11152300381112) bank12391 +12392 POINT(40.57121061844797 74.65445479822901) bank12392 +12393 POINT(41.26276889959137 73.185189876732) bank12393 +12394 POINT(39.980427674265215 74.36517337038876) bank12394 +12395 POINT(40.2807617902777 74.53484983862035) bank12395 +12396 POINT(41.692518493164926 73.15638175918743) bank12396 +12397 POINT(40.474076490726915 73.8720364831059) bank12397 +12398 POINT(41.63419836686616 73.36244460370342) bank12398 +12399 POINT(41.066030151659525 74.52659207967693) bank12399 +12400 POINT(40.46710241172836 73.46855501331864) bank12400 +12401 POINT(41.13137468743845 74.4152891930747) bank12401 +12402 POINT(41.59610378559118 73.1157920323212) bank12402 +12403 POINT(40.66795206988928 73.1360078754178) bank12403 +12404 POINT(41.58384800599476 74.57627389788817) bank12404 +12405 POINT(40.89752243638093 73.81011582285763) bank12405 +12406 POINT(41.055211057422405 73.28482267138237) bank12406 +12407 POINT(39.81316187816678 74.04309505743643) bank12407 +12408 POINT(40.92573475909403 73.3517202991452) bank12408 +12409 POINT(40.91351930555461 73.85742829678469) bank12409 +12410 POINT(41.15977590704444 74.22120841560448) bank12410 +12411 POINT(41.21047348572528 74.58471869380426) bank12411 +12412 POINT(39.76262958574884 73.27932376733233) bank12412 +12413 POINT(41.70336591860658 73.6765034210759) bank12413 +12414 POINT(40.22592509037944 74.93192637341582) bank12414 +12415 POINT(39.92618633271906 74.79968937677901) bank12415 +12416 POINT(40.555957408796225 74.0844029361524) bank12416 +12417 POINT(41.52176190115532 73.85289649141886) bank12417 +12418 POINT(40.20903961004516 73.23551329580654) bank12418 +12419 POINT(40.277385003752855 74.76063713129551) bank12419 +12420 POINT(40.17928884832564 74.88321274584726) bank12420 +12421 POINT(41.320502687547126 74.94879565035474) bank12421 +12422 POINT(39.745240913949374 73.265056701731) bank12422 +12423 POINT(40.89347496945736 73.15647874260613) bank12423 +12424 POINT(41.03992434994009 73.59427307029567) bank12424 +12425 POINT(39.74029025924588 74.60167174926397) bank12425 +12426 POINT(41.15386976759856 74.50607990520797) bank12426 +12427 POINT(41.60124859973784 73.00998400440541) bank12427 +12428 POINT(41.50475669376642 73.9013509623156) bank12428 +12429 POINT(40.34133624153663 73.24902446218671) bank12429 +12430 POINT(40.224345421962006 73.52361883750086) bank12430 +12431 POINT(39.83078051973536 74.36743977421902) bank12431 +12432 POINT(40.94093827571457 73.94398013839773) bank12432 +12433 POINT(40.608987013952365 74.57069035275647) bank12433 +12434 POINT(41.632990459516634 74.49020474691224) bank12434 +12435 POINT(41.698254164821975 73.42725176365458) bank12435 +12436 POINT(40.20527408863159 73.13914933361886) bank12436 +12437 POINT(41.70269975298038 73.51772152591718) bank12437 +12438 POINT(41.61292369882052 74.669151272831) bank12438 +12439 POINT(41.205167733613685 73.64857569235703) bank12439 +12440 POINT(40.25200647812349 73.8427236163553) bank12440 +12441 POINT(40.83109209122294 73.30334985324944) bank12441 +12442 POINT(40.94309339849723 74.77230185461524) bank12442 +12443 POINT(40.692805920739445 74.63317585326065) bank12443 +12444 POINT(40.796823782905115 73.139556139781) bank12444 +12445 POINT(40.99207714625737 73.6117574244138) bank12445 +12446 POINT(41.42395762806471 74.41149828939875) bank12446 +12447 POINT(39.76461509851627 74.08873485442902) bank12447 +12448 POINT(41.20931919596436 73.76381803982979) bank12448 +12449 POINT(40.018201055572945 74.45046212905449) bank12449 +12450 POINT(40.76289963285754 74.89162617919395) bank12450 +12451 POINT(41.50159136035992 73.69708476880429) bank12451 +12452 POINT(41.36380114677865 73.36069649581756) bank12452 +12453 POINT(40.41902293717159 74.4346068015532) bank12453 +12454 POINT(41.309886681132795 73.31087357826128) bank12454 +12455 POINT(41.00175047717077 74.31177486007505) bank12455 +12456 POINT(40.39639331359315 73.04969019889204) bank12456 +12457 POINT(39.935986670677984 73.77911864954875) bank12457 +12458 POINT(39.8543391344933 73.9459582772977) bank12458 +12459 POINT(40.5662364763299 74.35132860843827) bank12459 +12460 POINT(41.28763522764247 73.4506357441938) bank12460 +12461 POINT(41.670398936773694 74.38268001297979) bank12461 +12462 POINT(41.48344737280806 74.91709967784814) bank12462 +12463 POINT(40.79056366952548 74.06210582503141) bank12463 +12464 POINT(41.54253882010727 73.86893669540544) bank12464 +12465 POINT(41.476828243145306 74.99695903191618) bank12465 +12466 POINT(41.23168786842051 73.01403438871343) bank12466 +12467 POINT(41.0565243208925 73.38897009387759) bank12467 +12468 POINT(41.17088634541326 74.92341115726204) bank12468 +12469 POINT(41.66396024445178 73.80317354172539) bank12469 +12470 POINT(41.421648329728 73.3115864528238) bank12470 +12471 POINT(40.75907606831959 73.14858994205052) bank12471 +12472 POINT(40.557593884543735 73.60977811080777) bank12472 +12473 POINT(40.45988835031701 74.93976121967472) bank12473 +12474 POINT(40.67282975436619 74.12065686033412) bank12474 +12475 POINT(41.07240309751585 73.48547064750308) bank12475 +12476 POINT(41.23014136144565 74.90392570837895) bank12476 +12477 POINT(41.56833087364099 74.148066323157) bank12477 +12478 POINT(40.61875539479928 74.26998321200745) bank12478 +12479 POINT(40.54643720812481 74.18656990455885) bank12479 +12480 POINT(41.11317841825158 73.89932882252971) bank12480 +12481 POINT(41.44949632497432 74.16029564304168) bank12481 +12482 POINT(39.94303044597418 74.37717305530123) bank12482 +12483 POINT(40.39275205687971 73.07374374036087) bank12483 +12484 POINT(40.55391209076804 73.80615257369553) bank12484 +12485 POINT(41.25391679800426 74.08557541132686) bank12485 +12486 POINT(41.633926200032505 74.37449869323625) bank12486 +12487 POINT(41.70594573369059 74.99357781245043) bank12487 +12488 POINT(40.24374742969134 73.70303370878837) bank12488 +12489 POINT(40.64346461632119 73.81800305060955) bank12489 +12490 POINT(40.81709145590656 73.4570739381941) bank12490 +12491 POINT(40.33618928532023 74.63634918786445) bank12491 +12492 POINT(40.917268969523676 73.46235101177706) bank12492 +12493 POINT(41.63178027463602 73.44796197589226) bank12493 +12494 POINT(40.848851984732136 74.42372745430926) bank12494 +12495 POINT(41.52353210254876 74.71392061938144) bank12495 +12496 POINT(41.01891339191501 74.65180910067075) bank12496 +12497 POINT(41.061239777247216 74.44852952570581) bank12497 +12498 POINT(40.60122222326352 74.52647420318678) bank12498 +12499 POINT(41.13402738131351 74.3381680718724) bank12499 +12500 POINT(40.7386837178996 73.41425139325123) bank12500 +12501 POINT(40.92646763105847 74.49540540735636) bank12501 +12502 POINT(41.421146907748785 73.20060185381809) bank12502 +12503 POINT(40.34388931873251 74.31715419844919) bank12503 +12504 POINT(39.91832291850959 73.91088201459607) bank12504 +12505 POINT(40.23174276019483 74.91669936607627) bank12505 +12506 POINT(41.05414730064584 74.40461705317054) bank12506 +12507 POINT(40.432796328884876 73.8428586842039) bank12507 +12508 POINT(40.026660951851795 74.64231586385287) bank12508 +12509 POINT(39.98021689432675 73.22578472403734) bank12509 +12510 POINT(40.89075249268863 73.195669973835) bank12510 +12511 POINT(40.10700553518065 73.24304578018011) bank12511 +12512 POINT(41.41866636105564 73.01404093698262) bank12512 +12513 POINT(39.99129849356043 73.91903479494373) bank12513 +12514 POINT(40.13159440414006 74.67365264478572) bank12514 +12515 POINT(40.00126580187556 73.97753377145584) bank12515 +12516 POINT(40.85176573720577 74.81857068019502) bank12516 +12517 POINT(41.07763423413624 74.97130004432623) bank12517 +12518 POINT(40.8343353745507 73.31167874675118) bank12518 +12519 POINT(41.42615911725906 73.1958337638232) bank12519 +12520 POINT(40.872053326059984 73.14935711601503) bank12520 +12521 POINT(40.501666158686355 73.40151168836356) bank12521 +12522 POINT(40.661485354339405 74.94494523055046) bank12522 +12523 POINT(40.16995674233742 73.67733170153149) bank12523 +12524 POINT(40.35178146091887 74.78196455201204) bank12524 +12525 POINT(39.92443628274561 73.48504120845665) bank12525 +12526 POINT(40.23133865278017 74.32876289596045) bank12526 +12527 POINT(40.1861572922794 73.86574191043886) bank12527 +12528 POINT(41.52110288764639 73.57039015264581) bank12528 +12529 POINT(40.51291569318728 74.61198304816706) bank12529 +12530 POINT(40.80917855068837 74.88965930143218) bank12530 +12531 POINT(39.86083534744595 74.10727602113109) bank12531 +12532 POINT(39.966106409922034 73.11361518194028) bank12532 +12533 POINT(39.75554841136273 74.43785660889901) bank12533 +12534 POINT(40.75444145661259 74.30729432599485) bank12534 +12535 POINT(39.815807893408476 73.38086098488256) bank12535 +12536 POINT(41.51778592753283 73.75831644740745) bank12536 +12537 POINT(40.32366054973476 74.11281412754553) bank12537 +12538 POINT(41.50518140315352 74.62736314338022) bank12538 +12539 POINT(41.16289540088844 74.43980208335584) bank12539 +12540 POINT(41.70969753084085 74.98310670845785) bank12540 +12541 POINT(40.7172593755059 74.07398528819526) bank12541 +12542 POINT(41.537449528399534 73.0375413388746) bank12542 +12543 POINT(40.18687248476654 75.0006637209864) bank12543 +12544 POINT(41.61037480234202 73.42140822741442) bank12544 +12545 POINT(40.20077202873389 73.0060239147638) bank12545 +12546 POINT(40.06755809284962 73.83413024061538) bank12546 +12547 POINT(40.399564939646325 74.27535661724687) bank12547 +12548 POINT(41.25030203243619 73.85882687820009) bank12548 +12549 POINT(41.6661702419518 73.02077595929408) bank12549 +12550 POINT(40.79854093250011 73.19062394100617) bank12550 +12551 POINT(40.15092549622909 74.57414878064098) bank12551 +12552 POINT(40.99551531604031 73.8411751780846) bank12552 +12553 POINT(40.418151486919804 73.99221021889842) bank12553 +12554 POINT(40.32561138874295 73.242888630145) bank12554 +12555 POINT(40.567065746411664 74.07540249395076) bank12555 +12556 POINT(39.85120445885015 74.40824039102068) bank12556 +12557 POINT(40.889361288328665 73.91845450119393) bank12557 +12558 POINT(40.14234155137465 74.03779742849392) bank12558 +12559 POINT(41.340110990392496 73.4020778481097) bank12559 +12560 POINT(40.19073947054294 73.56248233031785) bank12560 +12561 POINT(40.25734150796602 74.72590495697003) bank12561 +12562 POINT(40.24306017726461 74.16175594238146) bank12562 +12563 POINT(41.370287808003276 73.03116028594316) bank12563 +12564 POINT(41.01207079799022 74.53957974990249) bank12564 +12565 POINT(41.32190158879063 74.205766516219) bank12565 +12566 POINT(40.135772873026 73.82025599870201) bank12566 +12567 POINT(41.07766852996688 74.7371875582033) bank12567 +12568 POINT(41.321218511531455 73.76139387378026) bank12568 +12569 POINT(40.86543916344896 73.44459893923067) bank12569 +12570 POINT(41.20285398837402 73.30126321871437) bank12570 +12571 POINT(40.05439731654486 73.90831333258086) bank12571 +12572 POINT(40.78208226800684 73.47651279846325) bank12572 +12573 POINT(41.3590248858599 74.73206602780171) bank12573 +12574 POINT(40.94593462900304 73.64854205855437) bank12574 +12575 POINT(41.084962660845996 74.70095373608008) bank12575 +12576 POINT(41.418878763638446 73.50514971396836) bank12576 +12577 POINT(40.89111404495404 73.87758843467059) bank12577 +12578 POINT(40.17482515211718 75.00501060399664) bank12578 +12579 POINT(40.24588314431446 74.41776021019284) bank12579 +12580 POINT(41.51637429789856 73.45196927640805) bank12580 +12581 POINT(41.70677346115715 74.09610157085208) bank12581 +12582 POINT(41.2665699702944 74.93827780352377) bank12582 +12583 POINT(39.9929569684905 73.95724795890456) bank12583 +12584 POINT(40.78538910428878 73.98494396214733) bank12584 +12585 POINT(39.736970838511496 73.51286875299215) bank12585 +12586 POINT(40.38682862454702 74.32951376671147) bank12586 +12587 POINT(39.89808802593958 73.24238987716679) bank12587 +12588 POINT(40.35730554118684 74.97152876861547) bank12588 +12589 POINT(40.62768449090882 73.39553795286928) bank12589 +12590 POINT(40.03770849852075 73.34977303692499) bank12590 +12591 POINT(40.840799883327435 74.7233036850146) bank12591 +12592 POINT(39.955084679480166 74.66545390596384) bank12592 +12593 POINT(40.95995195816287 74.7546854481497) bank12593 +12594 POINT(40.81171167005308 74.69055842769167) bank12594 +12595 POINT(39.889808470738565 74.52290183951456) bank12595 +12596 POINT(41.57380242249332 74.18564040454636) bank12596 +12597 POINT(40.62373417190636 73.19845965469851) bank12597 +12598 POINT(40.01603509365351 74.34136610079348) bank12598 +12599 POINT(39.71446752445239 74.80284609629459) bank12599 +12600 POINT(40.254483460343664 74.59693509290844) bank12600 +12601 POINT(40.914877379993925 74.45840571209904) bank12601 +12602 POINT(40.730768655235394 74.26856911939824) bank12602 +12603 POINT(41.256736334753946 73.38201314096773) bank12603 +12604 POINT(41.19248147489685 73.5305406030471) bank12604 +12605 POINT(40.85939189755761 73.569878922066) bank12605 +12606 POINT(41.589110788044344 74.73509349627979) bank12606 +12607 POINT(41.20289154971689 73.30036547700647) bank12607 +12608 POINT(40.777119743382784 74.97986865504713) bank12608 +12609 POINT(39.88906762663825 74.90128778297135) bank12609 +12610 POINT(41.706816431824635 74.72752080342174) bank12610 +12611 POINT(40.408305527931006 73.60723016689589) bank12611 +12612 POINT(41.30499138192886 73.54350375194804) bank12612 +12613 POINT(40.176335877533795 73.0895429340381) bank12613 +12614 POINT(41.448332752048906 74.1486926336685) bank12614 +12615 POINT(40.45213062716415 73.77919838955235) bank12615 +12616 POINT(41.13245255568078 73.46590145055987) bank12616 +12617 POINT(40.01907771545583 74.8857735458854) bank12617 +12618 POINT(41.43305737032892 73.80595048418904) bank12618 +12619 POINT(41.323585696450216 73.62327843899861) bank12619 +12620 POINT(40.50517605127423 73.36231641590572) bank12620 +12621 POINT(40.2909096742503 74.63481209901761) bank12621 +12622 POINT(40.01718844236798 73.54986272668843) bank12622 +12623 POINT(40.90669136787973 74.73427310039959) bank12623 +12624 POINT(40.251087225293055 73.43070203382749) bank12624 +12625 POINT(40.558008987237166 73.52802950041902) bank12625 +12626 POINT(41.43608606516707 73.96300161316131) bank12626 +12627 POINT(41.276329468098204 73.91645416691253) bank12627 +12628 POINT(39.79788838959635 73.9053208790074) bank12628 +12629 POINT(41.06521229777765 74.58631401689887) bank12629 +12630 POINT(40.77801056897145 74.28665277593169) bank12630 +12631 POINT(41.41117807580583 73.49920316901874) bank12631 +12632 POINT(41.66063113117584 74.93698505471015) bank12632 +12633 POINT(40.2249199259403 74.70740874526606) bank12633 +12634 POINT(40.80250674397225 73.31485169876743) bank12634 +12635 POINT(41.308821013297724 73.76686605360919) bank12635 +12636 POINT(39.9438614454917 73.27296696936465) bank12636 +12637 POINT(41.63566779779421 74.5969058807438) bank12637 +12638 POINT(40.44824061143453 73.96579299104971) bank12638 +12639 POINT(40.60842352109748 74.64794056030179) bank12639 +12640 POINT(40.97557457944693 74.57084091843632) bank12640 +12641 POINT(41.67241073047924 73.27824102616788) bank12641 +12642 POINT(39.96703651785163 74.46257034955406) bank12642 +12643 POINT(41.42808778969069 73.70996139061428) bank12643 +12644 POINT(39.999100615126686 74.45347299728566) bank12644 +12645 POINT(40.49206438663514 73.88936032069587) bank12645 +12646 POINT(40.57634935870602 74.44693155628583) bank12646 +12647 POINT(41.20433081900172 73.87408331657394) bank12647 +12648 POINT(41.63708598137081 74.05602671660793) bank12648 +12649 POINT(41.01362992482414 73.63839269813734) bank12649 +12650 POINT(39.837744377140595 73.72434144881241) bank12650 +12651 POINT(39.74856119281203 73.00676203222565) bank12651 +12652 POINT(40.12811299493806 73.23804884229632) bank12652 +12653 POINT(40.52538145753054 73.57267309720415) bank12653 +12654 POINT(40.04126593452037 73.97441087137683) bank12654 +12655 POINT(41.14769175364042 73.99753851368321) bank12655 +12656 POINT(39.933222382195346 73.82531090604854) bank12656 +12657 POINT(40.68701715056157 74.46468986096302) bank12657 +12658 POINT(40.4156065398047 73.32719343571864) bank12658 +12659 POINT(41.68174741579499 74.47146875823047) bank12659 +12660 POINT(40.88082844228081 74.45402351622121) bank12660 +12661 POINT(40.10000677832116 74.75163750156722) bank12661 +12662 POINT(40.957660606088396 74.29899862609669) bank12662 +12663 POINT(41.53748576630305 74.75013621657189) bank12663 +12664 POINT(40.08385740651525 74.24272485848925) bank12664 +12665 POINT(40.92641518941723 74.33198985644282) bank12665 +12666 POINT(40.18253900551898 73.18888902607196) bank12666 +12667 POINT(39.87916418978387 74.8227374629951) bank12667 +12668 POINT(40.988783398181894 73.6688714460364) bank12668 +12669 POINT(40.68082088215698 73.14748118299434) bank12669 +12670 POINT(41.131536281615084 74.23837394250062) bank12670 +12671 POINT(41.486964347824205 74.81979426528176) bank12671 +12672 POINT(41.12452256047552 74.99520549714998) bank12672 +12673 POINT(40.48002007582139 73.08057030417474) bank12673 +12674 POINT(40.59745519845805 73.1497886564259) bank12674 +12675 POINT(40.43222740653013 74.48272751808815) bank12675 +12676 POINT(39.784774242853885 73.79778075158367) bank12676 +12677 POINT(40.59112674764629 73.08645760369296) bank12677 +12678 POINT(41.22734987549984 73.7158784634281) bank12678 +12679 POINT(40.76770597848054 74.27547419368992) bank12679 +12680 POINT(40.39233701001958 74.83580683736007) bank12680 +12681 POINT(40.39864214139323 73.47336976124318) bank12681 +12682 POINT(41.098555449489055 74.7805448783552) bank12682 +12683 POINT(40.69515150842403 73.5826751244953) bank12683 +12684 POINT(40.58661280182043 74.42279361017958) bank12684 +12685 POINT(39.9777309021635 74.89685745607987) bank12685 +12686 POINT(40.063178670987796 74.82956505258251) bank12686 +12687 POINT(40.0132390257901 74.60290371948054) bank12687 +12688 POINT(41.4431001827903 73.15695997037953) bank12688 +12689 POINT(40.759754314473184 74.29225892528983) bank12689 +12690 POINT(40.12062034964479 73.52326990443473) bank12690 +12691 POINT(40.815670922893105 73.89400749121872) bank12691 +12692 POINT(41.57135129147705 73.90228303737595) bank12692 +12693 POINT(40.33784222959021 74.99367519721915) bank12693 +12694 POINT(40.480566212045986 73.8530670961647) bank12694 +12695 POINT(40.660887376380025 74.17098196099602) bank12695 +12696 POINT(40.19358034029754 74.045194231291) bank12696 +12697 POINT(41.51091329081823 74.32063735594052) bank12697 +12698 POINT(41.08813313779799 73.70284874624691) bank12698 +12699 POINT(40.62355334964244 74.99498953570048) bank12699 +12700 POINT(40.82484303145766 73.84028501990174) bank12700 +12701 POINT(41.62653004418148 73.90659483310574) bank12701 +12702 POINT(41.32926069068718 73.2307944145213) bank12702 +12703 POINT(39.89712851771326 74.29763947609003) bank12703 +12704 POINT(40.13439795908681 74.89662776714474) bank12704 +12705 POINT(41.1038346310124 73.04164005042387) bank12705 +12706 POINT(41.154663023216656 73.81525468466819) bank12706 +12707 POINT(40.530022805343684 73.3445400134853) bank12707 +12708 POINT(40.60041431240864 74.38684797772305) bank12708 +12709 POINT(41.28815036242674 74.51766043756999) bank12709 +12710 POINT(41.302423336473666 73.15363737947486) bank12710 +12711 POINT(40.04004554276506 74.38431803564058) bank12711 +12712 POINT(39.985596589919204 74.4399967966901) bank12712 +12713 POINT(40.63698803553876 73.0897702157756) bank12713 +12714 POINT(40.96221003041997 73.98576730921758) bank12714 +12715 POINT(39.946206099324186 73.99255021330195) bank12715 +12716 POINT(41.242941349335574 73.56614785964496) bank12716 +12717 POINT(40.78445754441949 74.49121886389784) bank12717 +12718 POINT(41.120846466107935 73.13893010052942) bank12718 +12719 POINT(39.775043020441714 74.85720155267377) bank12719 +12720 POINT(41.70814653081283 74.68022601506749) bank12720 +12721 POINT(40.42872580388077 74.32948172578817) bank12721 +12722 POINT(41.65756349896582 73.26438691057463) bank12722 +12723 POINT(41.33200349884713 73.10361548435868) bank12723 +12724 POINT(40.92217463690554 73.03635726846667) bank12724 +12725 POINT(41.390984317016034 73.96800095864566) bank12725 +12726 POINT(39.8521927319392 73.24323990255047) bank12726 +12727 POINT(41.373446159975195 74.58171629751246) bank12727 +12728 POINT(41.5649521251456 73.17204516620455) bank12728 +12729 POINT(39.810340252567244 73.75195637377985) bank12729 +12730 POINT(39.96787308470025 73.53391192351752) bank12730 +12731 POINT(40.41773151776404 74.03708556291163) bank12731 +12732 POINT(40.72019639896197 74.57646150978577) bank12732 +12733 POINT(40.20375991494092 74.14512035538115) bank12733 +12734 POINT(40.17495500505857 74.53964878804923) bank12734 +12735 POINT(40.57927629684066 73.58593214419112) bank12735 +12736 POINT(40.230064308396 74.9708145002035) bank12736 +12737 POINT(40.16740643270824 73.01553373843588) bank12737 +12738 POINT(39.785227848318414 73.06785005613077) bank12738 +12739 POINT(39.98786691370342 73.169650134774) bank12739 +12740 POINT(39.89299049502238 74.53126472850141) bank12740 +12741 POINT(40.18152028220731 73.08417664091682) bank12741 +12742 POINT(41.44922866821393 74.9607651007694) bank12742 +12743 POINT(40.57008192335018 73.46268263806971) bank12743 +12744 POINT(39.834355548775555 74.69427560911069) bank12744 +12745 POINT(40.795184002751505 73.9343395500026) bank12745 +12746 POINT(41.594544531522715 74.92745133668902) bank12746 +12747 POINT(40.88019433302945 74.95071033424367) bank12747 +12748 POINT(39.9648339568236 74.83560434626281) bank12748 +12749 POINT(39.84468382548957 74.44863587406708) bank12749 +12750 POINT(40.67168234069559 73.36387355862354) bank12750 +12751 POINT(40.74276595788924 73.90711755060242) bank12751 +12752 POINT(41.46352123549097 73.36922966506042) bank12752 +12753 POINT(41.519981169954946 73.65546493169596) bank12753 +12754 POINT(40.019580021544606 74.82084689605425) bank12754 +12755 POINT(40.75762779418936 73.99806629345103) bank12755 +12756 POINT(40.76112175785848 74.71863479289787) bank12756 +12757 POINT(40.8238547963876 73.82421061923992) bank12757 +12758 POINT(41.22675524896745 73.57828228948284) bank12758 +12759 POINT(40.610984685687384 74.14409206021257) bank12759 +12760 POINT(40.712223358713544 74.59148580410792) bank12760 +12761 POINT(39.87867151947916 73.58169128162105) bank12761 +12762 POINT(40.236058380009275 73.32377682212817) bank12762 +12763 POINT(41.40262278408811 73.9349074884378) bank12763 +12764 POINT(40.69727319145932 73.10602525050646) bank12764 +12765 POINT(41.20068880464691 73.0521039412561) bank12765 +12766 POINT(40.75063105122774 73.88715375299245) bank12766 +12767 POINT(40.48859059462486 74.56824529087848) bank12767 +12768 POINT(40.584778908162775 73.28530249452801) bank12768 +12769 POINT(39.87313995866927 73.77293626829226) bank12769 +12770 POINT(40.051928180523376 73.0881050446907) bank12770 +12771 POINT(40.3853803392576 73.65575824518925) bank12771 +12772 POINT(40.43478392041748 74.57135396532982) bank12772 +12773 POINT(41.02509802626528 73.1894615395794) bank12773 +12774 POINT(40.85899282487185 74.46679395929615) bank12774 +12775 POINT(40.16642031885563 73.98762241966485) bank12775 +12776 POINT(41.00837252922436 73.6013524788803) bank12776 +12777 POINT(40.25084895642982 73.81166915425646) bank12777 +12778 POINT(39.79399744911876 74.87440002442331) bank12778 +12779 POINT(40.479042520650154 74.68022447433593) bank12779 +12780 POINT(40.62357778316566 74.16198091874705) bank12780 +12781 POINT(41.232558700334366 74.32062361585926) bank12781 +12782 POINT(41.292660170398655 73.61656727480168) bank12782 +12783 POINT(40.68184764635634 73.85828226676105) bank12783 +12784 POINT(40.95780526144122 73.70138181493434) bank12784 +12785 POINT(40.375247503639045 73.8868967358255) bank12785 +12786 POINT(40.21769622092125 73.26294603427505) bank12786 +12787 POINT(40.6357620142552 73.6057767331773) bank12787 +12788 POINT(41.11673084115006 73.39817846150511) bank12788 +12789 POINT(39.97997751844479 73.61661592146095) bank12789 +12790 POINT(41.07210615480249 74.20234566201226) bank12790 +12791 POINT(40.96633875445714 74.13810138055932) bank12791 +12792 POINT(40.46740756166797 73.86017950526735) bank12792 +12793 POINT(40.869602384535035 74.28127545196364) bank12793 +12794 POINT(40.89433035656479 74.71255749506352) bank12794 +12795 POINT(40.21075206600781 73.50510546719367) bank12795 +12796 POINT(40.34449764320984 73.19835722209761) bank12796 +12797 POINT(41.50405812546927 73.90591187981387) bank12797 +12798 POINT(40.981769073311305 73.2152310558926) bank12798 +12799 POINT(41.44070503316216 74.65196789774693) bank12799 +12800 POINT(39.833931574057374 74.09663389425927) bank12800 +12801 POINT(40.119060499715 73.99201039534609) bank12801 +12802 POINT(39.91944517336208 74.61788686563678) bank12802 +12803 POINT(40.42851523651818 74.21122469077103) bank12803 +12804 POINT(39.745265952321525 74.47359352844721) bank12804 +12805 POINT(40.076437449688605 73.65223964427925) bank12805 +12806 POINT(41.51358924409389 74.17468597812756) bank12806 +12807 POINT(41.210890173926735 74.60589740828146) bank12807 +12808 POINT(40.254319461584515 73.64925855366708) bank12808 +12809 POINT(41.06224936710443 73.26212049061323) bank12809 +12810 POINT(40.25516624928323 74.45330884991299) bank12810 +12811 POINT(40.18210473703846 74.5808253706945) bank12811 +12812 POINT(41.0886338851657 73.40134007638598) bank12812 +12813 POINT(39.859246035714285 73.65687219807039) bank12813 +12814 POINT(40.038213597537855 73.7880588056982) bank12814 +12815 POINT(39.97015738521769 73.57564722151608) bank12815 +12816 POINT(40.46391563885478 74.48571399563441) bank12816 +12817 POINT(41.38585871223172 73.60816335123876) bank12817 +12818 POINT(39.99005455474238 73.65564871360534) bank12818 +12819 POINT(40.49663908426282 73.23514661225383) bank12819 +12820 POINT(41.64883969267735 73.1767992438121) bank12820 +12821 POINT(40.84494308774928 74.32354185887839) bank12821 +12822 POINT(41.34364166948177 73.5462817787271) bank12822 +12823 POINT(40.94724145446365 73.38214630512908) bank12823 +12824 POINT(41.39444507551466 73.74902780748886) bank12824 +12825 POINT(41.2968432680991 74.90230185726784) bank12825 +12826 POINT(41.39565462535798 73.54276888375139) bank12826 +12827 POINT(39.85424761937542 74.8074783661518) bank12827 +12828 POINT(41.08347196304896 74.4853424960994) bank12828 +12829 POINT(40.79379701440347 74.5445785103299) bank12829 +12830 POINT(40.93534406299257 73.69741235027813) bank12830 +12831 POINT(40.21291054668957 73.91709898095316) bank12831 +12832 POINT(40.12342889959862 73.15958693638991) bank12832 +12833 POINT(40.052143245631854 74.24605746115766) bank12833 +12834 POINT(40.76452939007678 73.71255816981011) bank12834 +12835 POINT(39.799841647231645 74.02907480717367) bank12835 +12836 POINT(40.55638648529233 73.8962907691863) bank12836 +12837 POINT(41.54123231206877 74.70740841884087) bank12837 +12838 POINT(40.78129907044125 74.31542916693702) bank12838 +12839 POINT(41.458638290385686 73.97060225784443) bank12839 +12840 POINT(41.40496097320588 73.88134201760435) bank12840 +12841 POINT(41.05819608305952 73.0703370214044) bank12841 +12842 POINT(40.0410044846461 73.16240497541467) bank12842 +12843 POINT(40.041530196188205 74.99943211274699) bank12843 +12844 POINT(40.54804579199257 74.9066924948413) bank12844 +12845 POINT(40.517698280613736 73.1793498555053) bank12845 +12846 POINT(41.19222494875393 74.46201365941795) bank12846 +12847 POINT(39.95237108763401 73.76204536535997) bank12847 +12848 POINT(40.02512688969418 73.16957703243611) bank12848 +12849 POINT(41.051380011521495 73.51035234840158) bank12849 +12850 POINT(40.39747975501254 73.16891544951548) bank12850 +12851 POINT(41.36618544363277 73.38890497263641) bank12851 +12852 POINT(40.10225228118896 73.44147762688554) bank12852 +12853 POINT(40.60716507992982 74.51090922492838) bank12853 +12854 POINT(41.006598880992314 73.91509818769391) bank12854 +12855 POINT(40.686272127607786 74.15088606987182) bank12855 +12856 POINT(39.79800097944176 74.49057802703608) bank12856 +12857 POINT(41.48245515941132 74.49478279819252) bank12857 +12858 POINT(41.31929743349127 74.34998459836407) bank12858 +12859 POINT(40.39680687592694 74.72131646489176) bank12859 +12860 POINT(41.699733881763734 73.18450521750196) bank12860 +12861 POINT(41.23095960227117 73.0899180635231) bank12861 +12862 POINT(40.550321789201966 73.83131838152627) bank12862 +12863 POINT(39.889367073053634 74.95370366752229) bank12863 +12864 POINT(39.809359138875294 73.3781516164749) bank12864 +12865 POINT(40.214947536935505 74.77319184234413) bank12865 +12866 POINT(40.24141054594823 74.08074525001653) bank12866 +12867 POINT(41.277148783485536 74.7029291192383) bank12867 +12868 POINT(39.91953933931848 74.38484060444586) bank12868 +12869 POINT(41.42996727036271 74.66474201322337) bank12869 +12870 POINT(41.66363261037387 74.51705284355329) bank12870 +12871 POINT(40.3203458183925 73.35204304337455) bank12871 +12872 POINT(41.39987635139227 73.50678671809477) bank12872 +12873 POINT(41.16003209657743 74.99051483903807) bank12873 +12874 POINT(41.64941090033796 73.67270635714246) bank12874 +12875 POINT(41.46272011730733 74.79432121935807) bank12875 +12876 POINT(41.095286886009276 73.29869193500494) bank12876 +12877 POINT(41.46703214306757 74.23895590460313) bank12877 +12878 POINT(40.448048167790056 73.58948264904484) bank12878 +12879 POINT(41.18313621731589 74.41664755330919) bank12879 +12880 POINT(40.17704919994511 73.41909785332939) bank12880 +12881 POINT(39.725963327251 73.73300759069384) bank12881 +12882 POINT(40.359105451838644 73.48371674978522) bank12882 +12883 POINT(40.07356916989448 74.63563807304119) bank12883 +12884 POINT(40.28386069761781 74.65992221450627) bank12884 +12885 POINT(41.0717048995167 73.2094140949111) bank12885 +12886 POINT(40.28752544126438 73.0700898819415) bank12886 +12887 POINT(41.255313117767706 74.570814942418) bank12887 +12888 POINT(41.7012193772166 74.49507406888524) bank12888 +12889 POINT(40.83353388340626 73.84261780394071) bank12889 +12890 POINT(40.75348789572692 74.22310181460583) bank12890 +12891 POINT(39.83295702505648 73.55810970671584) bank12891 +12892 POINT(39.98338148122805 73.66747812098794) bank12892 +12893 POINT(41.16682519371808 73.46821782638479) bank12893 +12894 POINT(39.731451239794666 74.84207479114195) bank12894 +12895 POINT(41.12209762114569 74.57788205980455) bank12895 +12896 POINT(39.74145816101342 74.86772453860517) bank12896 +12897 POINT(41.57368224056863 74.64707972244749) bank12897 +12898 POINT(40.13009366263799 73.73998245973124) bank12898 +12899 POINT(39.907645717069315 73.43969695387571) bank12899 +12900 POINT(41.671004596763076 74.95263260616443) bank12900 +12901 POINT(41.378168114791265 74.7972184654504) bank12901 +12902 POINT(39.861575113912956 73.81232282935554) bank12902 +12903 POINT(40.12465095679942 73.70605394280454) bank12903 +12904 POINT(40.25184046309149 74.65503495539939) bank12904 +12905 POINT(41.014653026559664 73.0833597762341) bank12905 +12906 POINT(40.66463203901577 74.421863743095) bank12906 +12907 POINT(40.65920282851585 74.88827621988976) bank12907 +12908 POINT(40.37919217325888 74.94373908317615) bank12908 +12909 POINT(39.830461701781275 74.97134780925597) bank12909 +12910 POINT(41.12782563919908 73.40972351181689) bank12910 +12911 POINT(41.658661288131114 74.45486568089004) bank12911 +12912 POINT(41.5294833866815 74.18859721385022) bank12912 +12913 POINT(40.92583634694955 74.30153909439595) bank12913 +12914 POINT(40.598076679050024 74.24395531494127) bank12914 +12915 POINT(41.245643837416935 74.91603129790768) bank12915 +12916 POINT(41.692109083435746 74.98151747308586) bank12916 +12917 POINT(39.774062389214684 74.17644217928908) bank12917 +12918 POINT(41.20185724468138 73.57409671560427) bank12918 +12919 POINT(39.98358434542721 74.80333693394832) bank12919 +12920 POINT(40.697778194215275 74.20531237722095) bank12920 +12921 POINT(40.75583949929358 73.84640591431933) bank12921 +12922 POINT(40.448507768279825 73.62571037273143) bank12922 +12923 POINT(39.99351131084165 73.707919159299) bank12923 +12924 POINT(39.79109061991071 73.55365199594668) bank12924 +12925 POINT(41.61124047626315 73.18636890300147) bank12925 +12926 POINT(41.30487503156814 73.76348403777352) bank12926 +12927 POINT(40.44642835454646 74.6663408166283) bank12927 +12928 POINT(40.077890201885126 74.91755062940888) bank12928 +12929 POINT(40.114303931107614 73.61334749572791) bank12929 +12930 POINT(41.04185512665245 74.54547723908922) bank12930 +12931 POINT(39.72207142952304 73.91129613464622) bank12931 +12932 POINT(40.51938218703738 73.80569794018237) bank12932 +12933 POINT(41.03186517162217 73.44183942010254) bank12933 +12934 POINT(40.81452560406834 73.01480884460577) bank12934 +12935 POINT(40.84931942773082 74.83537059497954) bank12935 +12936 POINT(40.72923933256162 74.56914273405393) bank12936 +12937 POINT(40.88378281195817 74.48603536623499) bank12937 +12938 POINT(40.57131459614784 73.36864726523716) bank12938 +12939 POINT(41.327889213926426 74.16519873549963) bank12939 +12940 POINT(40.99251833484786 74.10335493554913) bank12940 +12941 POINT(41.28247918400202 73.62600765844876) bank12941 +12942 POINT(39.93058727434367 73.08397035331356) bank12942 +12943 POINT(41.60767869264227 73.96838839082741) bank12943 +12944 POINT(40.50197836829218 74.3492998325008) bank12944 +12945 POINT(40.00242751370028 73.32890749117863) bank12945 +12946 POINT(40.04200713219315 74.33350575178738) bank12946 +12947 POINT(41.49879510202989 74.72008904320961) bank12947 +12948 POINT(41.198570676689194 74.7916117182565) bank12948 +12949 POINT(39.74966558271786 74.28089168189516) bank12949 +12950 POINT(40.39750108097578 73.11513300632657) bank12950 +12951 POINT(41.232026414281194 73.33363928784202) bank12951 +12952 POINT(40.270029360911074 73.02839160457732) bank12952 +12953 POINT(40.92493668803299 73.34488698438007) bank12953 +12954 POINT(41.63242870874695 74.96583259228181) bank12954 +12955 POINT(41.04635052409861 74.76525350603985) bank12955 +12956 POINT(41.25370302042268 74.4022558084879) bank12956 +12957 POINT(40.22180613317236 74.03433078587236) bank12957 +12958 POINT(40.62677919289413 74.03302561713397) bank12958 +12959 POINT(40.95078021086293 74.48878123153591) bank12959 +12960 POINT(41.51460202239006 73.6278026865515) bank12960 +12961 POINT(41.3773032604606 74.15609102075759) bank12961 +12962 POINT(39.86669183464355 74.09560681777262) bank12962 +12963 POINT(40.61338707021373 73.69886306079874) bank12963 +12964 POINT(39.820489622170065 73.86257408848466) bank12964 +12965 POINT(41.27044211291414 74.37315409906772) bank12965 +12966 POINT(39.925669580737114 74.45508914964333) bank12966 +12967 POINT(41.14317906507362 74.31959901812496) bank12967 +12968 POINT(40.04941949362544 74.94963613717935) bank12968 +12969 POINT(40.53261846067738 74.54754983437911) bank12969 +12970 POINT(40.4658881190485 74.60472712860924) bank12970 +12971 POINT(40.76149704330306 73.38266019969873) bank12971 +12972 POINT(40.2520590328252 73.73559543855801) bank12972 +12973 POINT(39.77555404305592 74.48142315515727) bank12973 +12974 POINT(40.651062274154135 73.64515803234828) bank12974 +12975 POINT(41.1900283087925 74.35296917168263) bank12975 +12976 POINT(39.99858547578296 73.30456369681357) bank12976 +12977 POINT(41.34887044196731 74.36372537162637) bank12977 +12978 POINT(39.839747786425484 74.71009159269896) bank12978 +12979 POINT(40.48157619463058 74.96511915528738) bank12979 +12980 POINT(41.29833919443844 74.9925647036905) bank12980 +12981 POINT(40.8955823033444 74.22491190973653) bank12981 +12982 POINT(40.14337996001792 74.90762262857423) bank12982 +12983 POINT(41.50495697541501 73.48298999379796) bank12983 +12984 POINT(41.21230031244887 74.75100229806162) bank12984 +12985 POINT(40.17586952626087 74.67519994674068) bank12985 +12986 POINT(40.234207139121466 74.98814388468045) bank12986 +12987 POINT(40.92556284370911 74.87582044975498) bank12987 +12988 POINT(40.36366019788853 73.27612958867104) bank12988 +12989 POINT(40.11521529644405 73.25313899723963) bank12989 +12990 POINT(41.50951612430777 73.61624692770486) bank12990 +12991 POINT(41.18853501205466 74.84677128273547) bank12991 +12992 POINT(39.71685901664829 74.38534366475612) bank12992 +12993 POINT(41.17757221440388 74.62581694725496) bank12993 +12994 POINT(41.30563625622116 73.6403585706084) bank12994 +12995 POINT(41.64430824120428 73.76197019831363) bank12995 +12996 POINT(39.82397681840199 73.86986636736037) bank12996 +12997 POINT(41.057306922858245 73.31422370851834) bank12997 +12998 POINT(40.54274600944666 74.60380841066339) bank12998 +12999 POINT(40.25542446586385 74.55896307253084) bank12999 +13000 POINT(40.181168184962246 73.95428269538196) bank13000 +13001 POINT(41.48169107333778 74.84180158461668) bank13001 +13002 POINT(41.344670000745914 74.37301478434571) bank13002 +13003 POINT(39.79531213216006 74.16152759106421) bank13003 +13004 POINT(41.07007459098453 74.51236948346141) bank13004 +13005 POINT(40.06472440121216 73.48383445067202) bank13005 +13006 POINT(40.19846098511142 73.66893198107822) bank13006 +13007 POINT(40.96735695863731 73.59419977325715) bank13007 +13008 POINT(40.5819788555982 74.2779800177403) bank13008 +13009 POINT(40.21199937974817 74.69232690331148) bank13009 +13010 POINT(40.276521809148846 74.32846602538145) bank13010 +13011 POINT(41.50342337296597 74.83700342437352) bank13011 +13012 POINT(40.42888096852686 73.80273659769946) bank13012 +13013 POINT(40.10013768309195 73.31552736085261) bank13013 +13014 POINT(41.311542966271574 73.9805152232946) bank13014 +13015 POINT(40.1888066861997 74.2202569975592) bank13015 +13016 POINT(40.41170211452775 73.88707397338959) bank13016 +13017 POINT(39.79086383420952 73.0769968491513) bank13017 +13018 POINT(41.579869801586035 73.73092543011443) bank13018 +13019 POINT(40.573775282685574 73.24254677909803) bank13019 +13020 POINT(41.706855315721114 74.35092693789726) bank13020 +13021 POINT(40.51755184894542 74.74296989454405) bank13021 +13022 POINT(40.68968350945934 73.62185329629517) bank13022 +13023 POINT(40.286426612619834 74.99596542457255) bank13023 +13024 POINT(41.65255246419421 73.61673192699493) bank13024 +13025 POINT(40.81034538618412 74.0745721421095) bank13025 +13026 POINT(40.0631924980766 74.8422538547478) bank13026 +13027 POINT(39.723009218543574 73.50238602090411) bank13027 +13028 POINT(41.15126105920967 74.07247302699871) bank13028 +13029 POINT(40.51009267911153 73.1640824619207) bank13029 +13030 POINT(41.206923664804535 74.89324327334161) bank13030 +13031 POINT(40.50713996415126 73.77418777653324) bank13031 +13032 POINT(39.82659783071312 74.74778284278986) bank13032 +13033 POINT(41.220339685999804 74.8734057929084) bank13033 +13034 POINT(41.1200687776503 74.6919142665977) bank13034 +13035 POINT(40.98343605581215 73.45937705161073) bank13035 +13036 POINT(41.21041597019151 73.28358422021657) bank13036 +13037 POINT(39.82062783413249 73.65699475572725) bank13037 +13038 POINT(40.09750045599891 73.81771091991172) bank13038 +13039 POINT(41.637304084041816 73.31306775515954) bank13039 +13040 POINT(40.29616219839399 73.18938309660675) bank13040 +13041 POINT(40.39165989356482 73.18842727778843) bank13041 +13042 POINT(41.588121179107816 74.63104167007815) bank13042 +13043 POINT(41.653047076746674 74.99715435121415) bank13043 +13044 POINT(41.20203968531994 73.80649626683496) bank13044 +13045 POINT(40.928425978773994 74.29523886043064) bank13045 +13046 POINT(40.17410810365697 74.2514919540305) bank13046 +13047 POINT(40.45107730271629 73.99532156497193) bank13047 +13048 POINT(39.84542837896525 74.36826060152396) bank13048 +13049 POINT(40.7204774234223 74.29663230075062) bank13049 +13050 POINT(40.67956966518923 73.75950690906457) bank13050 +13051 POINT(41.45196851885745 74.30538446867364) bank13051 +13052 POINT(40.664966295633796 73.26942973223075) bank13052 +13053 POINT(41.22479077135249 73.62588506155974) bank13053 +13054 POINT(40.30851664590349 74.91723864489445) bank13054 +13055 POINT(40.57350314026449 73.17970363424912) bank13055 +13056 POINT(41.26848774556412 74.64993110077282) bank13056 +13057 POINT(40.51462484885513 74.3531566094694) bank13057 +13058 POINT(40.18358066919931 73.57656444005131) bank13058 +13059 POINT(40.51719068900749 73.71102496609598) bank13059 +13060 POINT(40.7482668846562 74.58555256211764) bank13060 +13061 POINT(40.618225390649286 74.42526136587597) bank13061 +13062 POINT(40.251419380708235 74.42252895184753) bank13062 +13063 POINT(41.32613480073687 73.72595429993562) bank13063 +13064 POINT(40.170940810952565 73.20389302452188) bank13064 +13065 POINT(41.10844734661273 73.79631118649031) bank13065 +13066 POINT(41.33286196853806 74.95916847190992) bank13066 +13067 POINT(40.195846047141615 74.40925663196913) bank13067 +13068 POINT(40.59977292105948 74.48708126684743) bank13068 +13069 POINT(40.80865453933171 74.23860570301783) bank13069 +13070 POINT(40.420863540409016 74.34039813463633) bank13070 +13071 POINT(41.14650176313734 74.20012709682722) bank13071 +13072 POINT(41.375803090556246 74.05920398382617) bank13072 +13073 POINT(40.24084947074352 73.12107630315728) bank13073 +13074 POINT(41.58972916935958 73.44540915010946) bank13074 +13075 POINT(41.19220647599727 73.40569537486459) bank13075 +13076 POINT(40.94312507632501 73.16088231411388) bank13076 +13077 POINT(40.01842122334874 73.33529873910675) bank13077 +13078 POINT(41.63592604137678 74.17381046364414) bank13078 +13079 POINT(39.915877164117006 73.01349533875418) bank13079 +13080 POINT(40.8340474210864 74.28502023773778) bank13080 +13081 POINT(40.12717108177815 73.4916313504851) bank13081 +13082 POINT(41.28833208558522 74.38778888224506) bank13082 +13083 POINT(40.80766735037944 73.74198567171544) bank13083 +13084 POINT(40.886311583767046 74.59848264013509) bank13084 +13085 POINT(41.21838941090675 73.79825068558056) bank13085 +13086 POINT(41.439210020861566 73.02955409521266) bank13086 +13087 POINT(41.57360367820675 74.71410116186702) bank13087 +13088 POINT(40.86540951394796 73.39185119901556) bank13088 +13089 POINT(40.9964591612356 74.6423790863973) bank13089 +13090 POINT(40.49452758846311 73.48264631841946) bank13090 +13091 POINT(40.05091730803846 73.85712220888416) bank13091 +13092 POINT(39.74405289270195 74.323266950896) bank13092 +13093 POINT(41.064958863808535 74.49907534659378) bank13093 +13094 POINT(41.28720426452232 74.2929132807286) bank13094 +13095 POINT(41.517960383554126 74.68751093069028) bank13095 +13096 POINT(41.37808634526975 73.48238041621245) bank13096 +13097 POINT(40.31725579354771 74.0014472340455) bank13097 +13098 POINT(41.48699448252393 73.69638224015277) bank13098 +13099 POINT(40.60468134165023 74.29833652779836) bank13099 +13100 POINT(40.962078864516116 74.72908232929325) bank13100 +13101 POINT(40.8879931472688 73.55662140975696) bank13101 +13102 POINT(41.5174266216739 74.84389968649015) bank13102 +13103 POINT(41.49586840707781 73.59476049061719) bank13103 +13104 POINT(41.379427837954395 73.44305933056792) bank13104 +13105 POINT(41.05387657099943 74.34263106514153) bank13105 +13106 POINT(41.16195091467964 73.31732149027441) bank13106 +13107 POINT(40.58401496165567 73.72910469556489) bank13107 +13108 POINT(40.08405115991172 74.28537559083452) bank13108 +13109 POINT(40.04989401574794 74.07368150010444) bank13109 +13110 POINT(40.8634731029362 73.3032114649239) bank13110 +13111 POINT(40.50645764850585 73.92346936504018) bank13111 +13112 POINT(39.96569807866572 74.53673281348141) bank13112 +13113 POINT(39.95808356374685 74.17221553681118) bank13113 +13114 POINT(40.36679362937298 73.11000574616006) bank13114 +13115 POINT(40.07091659843422 74.4270570539686) bank13115 +13116 POINT(39.72441175086902 73.02878870325905) bank13116 +13117 POINT(39.985735252165796 74.58345183490073) bank13117 +13118 POINT(40.501607041214896 73.89694116399447) bank13118 +13119 POINT(41.44530036690027 73.79993202977695) bank13119 +13120 POINT(40.456029351660334 73.51882697625359) bank13120 +13121 POINT(41.37815796947531 73.29085247679568) bank13121 +13122 POINT(41.39195029966434 74.43876719016112) bank13122 +13123 POINT(40.632397480638055 74.07208425459439) bank13123 +13124 POINT(39.93818194982116 73.91688784916701) bank13124 +13125 POINT(41.35946499977486 73.3872438476298) bank13125 +13126 POINT(40.142785896383245 73.23538133566608) bank13126 +13127 POINT(39.94159223618618 73.40078439376325) bank13127 +13128 POINT(39.940758412196544 73.8126316822687) bank13128 +13129 POINT(41.07638553710069 73.11558362985716) bank13129 +13130 POINT(39.90300495745233 73.86382652706686) bank13130 +13131 POINT(40.58897512060464 73.89068236890873) bank13131 +13132 POINT(40.97237869720827 73.47247485716875) bank13132 +13133 POINT(40.72459622730142 73.43579502107302) bank13133 +13134 POINT(40.68840485254498 74.8279719132113) bank13134 +13135 POINT(41.426926327473005 74.90014020318823) bank13135 +13136 POINT(40.491247392144665 74.46719064635526) bank13136 +13137 POINT(40.33400376253601 73.62241029768474) bank13137 +13138 POINT(40.846564379227864 74.04036151688305) bank13138 +13139 POINT(40.38474928205615 74.15711008348043) bank13139 +13140 POINT(41.25709114093169 73.46390458877727) bank13140 +13141 POINT(41.71224798033111 74.11421404028717) bank13141 +13142 POINT(39.963586544497375 73.31929482927387) bank13142 +13143 POINT(41.47499991039882 73.03892259259044) bank13143 +13144 POINT(41.55476713528458 73.14239105138658) bank13144 +13145 POINT(39.85686207068626 74.80395322397453) bank13145 +13146 POINT(41.143865394889 73.17369879990292) bank13146 +13147 POINT(41.00254103798223 74.68060637515237) bank13147 +13148 POINT(40.085114244639215 73.58151112532799) bank13148 +13149 POINT(40.50938351144855 73.43486271423359) bank13149 +13150 POINT(41.23890240525719 74.68735931538264) bank13150 +13151 POINT(41.58083910481698 74.37719544268518) bank13151 +13152 POINT(41.04725051919827 74.80115850628022) bank13152 +13153 POINT(41.6521476556516 73.1372397222729) bank13153 +13154 POINT(39.94458974387878 74.13360780696073) bank13154 +13155 POINT(41.28011665894397 73.31253800614274) bank13155 +13156 POINT(39.9820797621068 74.65441896490937) bank13156 +13157 POINT(41.52155823098532 73.48456744479087) bank13157 +13158 POINT(40.40806017544235 74.82032869448365) bank13158 +13159 POINT(41.25836366377777 74.42897414819275) bank13159 +13160 POINT(39.865042650804895 73.42086044612287) bank13160 +13161 POINT(40.125831059574985 74.56065468430629) bank13161 +13162 POINT(41.4371665005505 73.19039936453552) bank13162 +13163 POINT(41.57869609961261 74.83925364149526) bank13163 +13164 POINT(40.992384725008186 74.67395914698879) bank13164 +13165 POINT(41.545222699906 74.12117127385774) bank13165 +13166 POINT(39.94093736518878 73.658838229749) bank13166 +13167 POINT(41.27147480540331 74.07566826350003) bank13167 +13168 POINT(40.232953369472014 74.75610393721158) bank13168 +13169 POINT(41.652767090241994 73.6723389714104) bank13169 +13170 POINT(41.22727885967392 74.60134949836346) bank13170 +13171 POINT(40.28618017731717 74.11906858629867) bank13171 +13172 POINT(39.79883224328993 73.6833166857787) bank13172 +13173 POINT(41.32888355507237 74.52345763207602) bank13173 +13174 POINT(40.4516519892772 73.62083169743158) bank13174 +13175 POINT(41.126430022204715 74.6697352529459) bank13175 +13176 POINT(40.732780397329755 74.54591782939721) bank13176 +13177 POINT(41.06237194250336 74.39618376385401) bank13177 +13178 POINT(41.425207420612125 73.0122135449321) bank13178 +13179 POINT(40.294773686491794 73.4746994233912) bank13179 +13180 POINT(41.40497210879785 74.64601476837322) bank13180 +13181 POINT(41.22600764550051 74.93546107565513) bank13181 +13182 POINT(40.4372166289079 74.92682641930077) bank13182 +13183 POINT(41.214022387866315 73.93256281325347) bank13183 +13184 POINT(39.90749622097956 73.98882228970571) bank13184 +13185 POINT(41.69935236461912 74.76317153643778) bank13185 +13186 POINT(40.43835207119866 73.91581170982367) bank13186 +13187 POINT(41.36208022056859 73.38335230555224) bank13187 +13188 POINT(40.771084041749006 74.06717419820112) bank13188 +13189 POINT(40.90896756335333 74.32629682888845) bank13189 +13190 POINT(41.479019563731626 74.16422093538806) bank13190 +13191 POINT(41.3227419969029 73.49347201986895) bank13191 +13192 POINT(40.53585034226408 73.42234199183758) bank13192 +13193 POINT(40.054967961840404 74.83836060104713) bank13193 +13194 POINT(41.69777819815228 73.06843076507884) bank13194 +13195 POINT(41.21651255698177 74.58251392345284) bank13195 +13196 POINT(41.21837321050558 73.58561537109594) bank13196 +13197 POINT(40.75389057286923 73.30927412042914) bank13197 +13198 POINT(39.9393172572641 74.6923225180839) bank13198 +13199 POINT(41.452750152105644 74.9099578792907) bank13199 +13200 POINT(40.66735075753047 74.13842683573685) bank13200 +13201 POINT(40.51388413293019 74.9027096394707) bank13201 +13202 POINT(41.53079367217617 74.60737838064851) bank13202 +13203 POINT(40.00841970049333 73.83040178380088) bank13203 +13204 POINT(40.473485271448574 74.47178572786665) bank13204 +13205 POINT(40.9160033937707 73.78145506963622) bank13205 +13206 POINT(39.99284041759158 74.94368239344954) bank13206 +13207 POINT(40.078629266843635 74.87065739541177) bank13207 +13208 POINT(40.59598947587055 74.25031374554237) bank13208 +13209 POINT(41.501646759614815 74.45059479849878) bank13209 +13210 POINT(41.58374546700242 73.75087817625814) bank13210 +13211 POINT(41.162978820370746 73.58303189394557) bank13211 +13212 POINT(40.416144178953616 74.05526807241787) bank13212 +13213 POINT(41.537214069184046 74.67843335774954) bank13213 +13214 POINT(40.851474235647565 74.85205495856904) bank13214 +13215 POINT(40.9150382881591 73.15690621460259) bank13215 +13216 POINT(39.73901586201035 73.39897424895152) bank13216 +13217 POINT(41.11517543397913 74.27656634199631) bank13217 +13218 POINT(39.848758448208386 74.26205159950965) bank13218 +13219 POINT(40.50926507597372 73.02784078075997) bank13219 +13220 POINT(40.29460415081165 74.52604976835636) bank13220 +13221 POINT(41.36547016777935 73.925146113359) bank13221 +13222 POINT(40.39693244402686 73.79639229303577) bank13222 +13223 POINT(39.77710728399694 74.12632326866989) bank13223 +13224 POINT(41.57758860367729 73.64746103502334) bank13224 +13225 POINT(39.79100810536128 73.92892925357984) bank13225 +13226 POINT(40.574025436317854 73.06428008299335) bank13226 +13227 POINT(41.420149927358985 74.58302436572164) bank13227 +13228 POINT(40.29309049537265 74.14007591763446) bank13228 +13229 POINT(40.88471901106665 73.6959757845863) bank13229 +13230 POINT(40.33460810763996 74.91588768815204) bank13230 +13231 POINT(40.67627293619992 73.61242592972407) bank13231 +13232 POINT(41.573949147224134 74.85695240921676) bank13232 +13233 POINT(41.49308509960381 74.29408336290214) bank13233 +13234 POINT(41.206851879645505 73.20848656371886) bank13234 +13235 POINT(41.45056653763322 73.68697786771514) bank13235 +13236 POINT(40.715909726312354 73.65059426341448) bank13236 +13237 POINT(40.538623226222505 73.83792523150908) bank13237 +13238 POINT(39.99446023643394 73.89575192702873) bank13238 +13239 POINT(40.210804087295784 74.04971540729092) bank13239 +13240 POINT(39.93581171141626 73.04903198030993) bank13240 +13241 POINT(41.60460284144789 74.30297531521686) bank13241 +13242 POINT(40.73026344461367 73.21331353151103) bank13242 +13243 POINT(40.560972610354995 74.70756167180275) bank13243 +13244 POINT(40.21326191341943 73.93427011599806) bank13244 +13245 POINT(41.560429160137154 73.10580475207833) bank13245 +13246 POINT(40.60635699255957 74.82231588452532) bank13246 +13247 POINT(39.90853853201248 73.60586543809718) bank13247 +13248 POINT(41.673470107262446 74.00434231238488) bank13248 +13249 POINT(39.75326616898863 74.99176217974349) bank13249 +13250 POINT(41.01777916871692 73.32280452490761) bank13250 +13251 POINT(41.19845982367952 74.61320421779841) bank13251 +13252 POINT(39.97875431129915 73.93677687372107) bank13252 +13253 POINT(41.40823553725662 73.43181063052741) bank13253 +13254 POINT(40.166855533021376 74.57604981286556) bank13254 +13255 POINT(40.26546976042353 73.51120059162754) bank13255 +13256 POINT(41.287108070051175 74.1059195534381) bank13256 +13257 POINT(41.22395853898364 73.59366728503252) bank13257 +13258 POINT(41.48852158474101 74.96097953601895) bank13258 +13259 POINT(39.97266899468474 73.65050720993432) bank13259 +13260 POINT(40.30235621042147 73.88574870502025) bank13260 +13261 POINT(41.08734983279609 73.9721330153571) bank13261 +13262 POINT(40.22714379836411 74.30283245818504) bank13262 +13263 POINT(40.70331536970671 74.60315642010075) bank13263 +13264 POINT(40.57811153605084 74.69436050956405) bank13264 +13265 POINT(40.055132168661224 74.52952512567359) bank13265 +13266 POINT(39.89443545175653 74.33526775246642) bank13266 +13267 POINT(41.35611181767504 73.86855014292766) bank13267 +13268 POINT(41.213810088667316 74.64570404761368) bank13268 +13269 POINT(40.939472759477724 73.00714045852172) bank13269 +13270 POINT(39.87654886009629 74.50995678591751) bank13270 +13271 POINT(40.41254128361007 74.52126324704267) bank13271 +13272 POINT(39.768603425124155 73.62124583474078) bank13272 +13273 POINT(41.30332538891106 74.3133339517625) bank13273 +13274 POINT(40.04881199291822 74.3369714162731) bank13274 +13275 POINT(41.43458670148789 73.95614562514) bank13275 +13276 POINT(40.868485865308074 74.58896547502377) bank13276 +13277 POINT(40.870176887180364 74.76090982101186) bank13277 +13278 POINT(40.606498060600785 74.0238678959459) bank13278 +13279 POINT(41.30495971833053 73.13963906407567) bank13279 +13280 POINT(41.109751666966744 73.86183136922055) bank13280 +13281 POINT(41.185588415879124 74.448344924625) bank13281 +13282 POINT(41.16387889679343 74.08182518934335) bank13282 +13283 POINT(39.819667848846585 73.11475990553458) bank13283 +13284 POINT(40.90804271288486 73.25150981699997) bank13284 +13285 POINT(41.49685718943317 74.97922958487712) bank13285 +13286 POINT(41.07615866232591 73.43153925159687) bank13286 +13287 POINT(41.056522333726015 74.76668789540105) bank13287 +13288 POINT(41.24487319975167 74.22419844706053) bank13288 +13289 POINT(40.45950545076174 73.50627320906582) bank13289 +13290 POINT(40.31346244298655 74.12743861792593) bank13290 +13291 POINT(41.28277863675037 74.8138552967402) bank13291 +13292 POINT(40.39220264582148 73.89468197055633) bank13292 +13293 POINT(41.64959530729381 73.59420156290815) bank13293 +13294 POINT(41.00769967083794 73.2942966174749) bank13294 +13295 POINT(40.6495897797688 74.20652160603376) bank13295 +13296 POINT(41.58235740956589 73.58049516563528) bank13296 +13297 POINT(41.01414408110644 74.64955458245277) bank13297 +13298 POINT(40.54665090484215 74.70861325555312) bank13298 +13299 POINT(40.16727079827523 74.73432557021864) bank13299 +13300 POINT(40.65096463238706 74.7360451433188) bank13300 +13301 POINT(40.16978893905603 74.37696033028314) bank13301 +13302 POINT(41.17026536014008 73.94449209780589) bank13302 +13303 POINT(41.422767987787566 73.52230521042155) bank13303 +13304 POINT(41.40258406355426 73.1777013599533) bank13304 +13305 POINT(40.09516722736861 73.02302505245596) bank13305 +13306 POINT(39.738535245012294 74.35521803689979) bank13306 +13307 POINT(40.93517926019541 73.38632504147839) bank13307 +13308 POINT(40.80849090985677 74.86888493087018) bank13308 +13309 POINT(40.95593528318466 74.73744324876729) bank13309 +13310 POINT(41.31434050238045 73.94714492267047) bank13310 +13311 POINT(41.1168556186342 73.92262608537082) bank13311 +13312 POINT(39.98065516256151 73.75404998062842) bank13312 +13313 POINT(39.7772065580231 73.37361153787897) bank13313 +13314 POINT(40.85766933996309 73.88412349190688) bank13314 +13315 POINT(41.667737904073476 73.74072588984424) bank13315 +13316 POINT(41.652026029589635 73.93102549112949) bank13316 +13317 POINT(41.469731535324776 74.63425888352253) bank13317 +13318 POINT(41.15126564094972 74.57324864531459) bank13318 +13319 POINT(39.92814774880333 74.71276332167217) bank13319 +13320 POINT(41.21815546186423 74.91189791762693) bank13320 +13321 POINT(41.01895989020717 73.10815929040577) bank13321 +13322 POINT(41.48170705536237 74.35646163275943) bank13322 +13323 POINT(40.23943622296577 73.22957914854437) bank13323 +13324 POINT(41.00760903918805 74.71108536103878) bank13324 +13325 POINT(41.02307811078339 73.22715507503072) bank13325 +13326 POINT(41.627636803440836 73.34655773022293) bank13326 +13327 POINT(41.658253907164486 74.7177832280236) bank13327 +13328 POINT(41.613322683518234 74.28314792946347) bank13328 +13329 POINT(40.416319689742046 73.58202124024965) bank13329 +13330 POINT(41.44111180098235 73.64721086748833) bank13330 +13331 POINT(41.297599431950495 73.87624261038584) bank13331 +13332 POINT(40.163310090544336 73.48992543939987) bank13332 +13333 POINT(41.49722554683496 74.30647184890553) bank13333 +13334 POINT(40.04291740331455 74.12972063407221) bank13334 +13335 POINT(40.902655911530445 73.30206244870834) bank13335 +13336 POINT(41.19092494692163 73.08776908223334) bank13336 +13337 POINT(40.536487360757974 73.58813030757278) bank13337 +13338 POINT(40.43424027452719 74.31143279510792) bank13338 +13339 POINT(41.66518679625523 74.46542255062387) bank13339 +13340 POINT(41.074471487669555 73.85926069549666) bank13340 +13341 POINT(41.65087097102197 73.9763817457768) bank13341 +13342 POINT(40.50248309596106 73.57644022143694) bank13342 +13343 POINT(40.362964732241885 73.20812659779399) bank13343 +13344 POINT(39.95349181400089 74.76871780075038) bank13344 +13345 POINT(39.91907780485432 73.44033761003072) bank13345 +13346 POINT(41.2459531523767 74.39331374817901) bank13346 +13347 POINT(41.1656027199699 73.5146466225805) bank13347 +13348 POINT(40.538557368775294 73.25024124677914) bank13348 +13349 POINT(39.85453050697564 73.5842767846256) bank13349 +13350 POINT(41.18253179349059 73.9503928370592) bank13350 +13351 POINT(39.900093733505194 73.57298197590671) bank13351 +13352 POINT(41.326854262459555 73.81377475486758) bank13352 +13353 POINT(40.85311362944343 73.59367892319125) bank13353 +13354 POINT(41.52616790049251 74.56057625483271) bank13354 +13355 POINT(41.59147137977429 73.49611266016156) bank13355 +13356 POINT(40.08091959360729 73.94694270072863) bank13356 +13357 POINT(40.6725353001009 73.62831257634714) bank13357 +13358 POINT(41.43279827784895 73.13999705639503) bank13358 +13359 POINT(40.275945152575815 73.85123605455914) bank13359 +13360 POINT(40.95908940030622 73.71058441608302) bank13360 +13361 POINT(40.53562808759248 74.30655849159338) bank13361 +13362 POINT(40.01957242071106 74.34184046155927) bank13362 +13363 POINT(41.536680849711125 73.15047960210278) bank13363 +13364 POINT(41.2028797953751 73.6618866447841) bank13364 +13365 POINT(40.472891172241894 74.77581993415595) bank13365 +13366 POINT(41.31374777585505 73.71877763315416) bank13366 +13367 POINT(40.722725123746905 73.41467772563871) bank13367 +13368 POINT(40.95104741417262 73.15646402097714) bank13368 +13369 POINT(40.87075625156939 74.59957627949176) bank13369 +13370 POINT(39.827313122231764 73.97285102379364) bank13370 +13371 POINT(40.30826445608652 73.74871198547334) bank13371 +13372 POINT(40.17042226754142 73.0853389221923) bank13372 +13373 POINT(41.333881035613565 74.94900044403906) bank13373 +13374 POINT(40.617363646968606 74.42133751193717) bank13374 +13375 POINT(40.1819293566912 73.92310107997733) bank13375 +13376 POINT(39.780451022886844 74.4958275836993) bank13376 +13377 POINT(40.40477916373804 74.22712901293879) bank13377 +13378 POINT(40.913388318894334 73.85774685385023) bank13378 +13379 POINT(41.236752818476376 73.26299685128026) bank13379 +13380 POINT(39.932133580166024 73.56539747613881) bank13380 +13381 POINT(40.74776502854698 73.39018134661926) bank13381 +13382 POINT(40.89704058173682 73.60133696699687) bank13382 +13383 POINT(39.94089888252275 73.04110147942406) bank13383 +13384 POINT(40.57954718218798 74.6035563083619) bank13384 +13385 POINT(41.46942525595089 73.43174428156811) bank13385 +13386 POINT(41.23941515549728 74.42939591745191) bank13386 +13387 POINT(40.9301912319016 73.75121415844214) bank13387 +13388 POINT(39.78788808907522 74.180909322247) bank13388 +13389 POINT(41.24971504565244 74.3171263962579) bank13389 +13390 POINT(40.750356883172245 73.01012978709241) bank13390 +13391 POINT(41.42840007344793 73.57220664279464) bank13391 +13392 POINT(40.046262796344905 73.01018387074704) bank13392 +13393 POINT(40.3199050004198 74.26612029744088) bank13393 +13394 POINT(40.87730081849677 74.89141379022485) bank13394 +13395 POINT(39.80141715652285 73.26588733196162) bank13395 +13396 POINT(41.45744234520702 74.21136747589816) bank13396 +13397 POINT(41.14943421878184 73.15386796538488) bank13397 +13398 POINT(39.90273372594643 74.29030197564128) bank13398 +13399 POINT(41.019724125361726 74.81974944214512) bank13399 +13400 POINT(40.40309159889918 74.40343785647468) bank13400 +13401 POINT(41.434602088410244 73.98126493969778) bank13401 +13402 POINT(40.96815772898553 74.18135201996408) bank13402 +13403 POINT(40.60535658886263 74.78111224753494) bank13403 +13404 POINT(41.13892824693238 73.3187524704423) bank13404 +13405 POINT(40.580128924366065 73.6174712489374) bank13405 +13406 POINT(40.17285833509333 73.00676402921447) bank13406 +13407 POINT(40.299221289807974 74.51271661218411) bank13407 +13408 POINT(39.886627804049006 73.5325305766667) bank13408 +13409 POINT(40.58148722173292 73.108078929713) bank13409 +13410 POINT(40.566484270655614 73.51420287530894) bank13410 +13411 POINT(39.85068928012968 74.00600374926222) bank13411 +13412 POINT(39.95134764086674 73.8531419285196) bank13412 +13413 POINT(41.4696943651692 73.84981136923804) bank13413 +13414 POINT(40.275197668349044 74.03009507965038) bank13414 +13415 POINT(41.48253114160715 73.35483991849205) bank13415 +13416 POINT(40.31702172095417 73.36628580604058) bank13416 +13417 POINT(41.31595744479096 74.79345150157434) bank13417 +13418 POINT(40.1843369366013 73.74396277254675) bank13418 +13419 POINT(41.264846568273036 74.49842826802137) bank13419 +13420 POINT(40.209120924968424 73.82536633105283) bank13420 +13421 POINT(39.87337134294282 73.62763538981446) bank13421 +13422 POINT(39.92314236306041 73.70040523527848) bank13422 +13423 POINT(40.95137186833528 73.35700039045302) bank13423 +13424 POINT(41.41977075363311 73.2377796063079) bank13424 +13425 POINT(39.87660927125101 73.42835891927285) bank13425 +13426 POINT(40.282298221360215 74.29654950176158) bank13426 +13427 POINT(41.65501827154557 73.94728918389613) bank13427 +13428 POINT(41.45061683303907 73.80245427128285) bank13428 +13429 POINT(41.32842814644166 74.58749509540581) bank13429 +13430 POINT(40.23487097628777 73.84741281875412) bank13430 +13431 POINT(41.28410059675193 73.8853884517824) bank13431 +13432 POINT(40.3887300889719 73.42314756837482) bank13432 +13433 POINT(41.57346097817932 74.82218417012363) bank13433 +13434 POINT(40.000097236555014 74.3882768605854) bank13434 +13435 POINT(40.53761717154591 73.27855117497299) bank13435 +13436 POINT(41.285526296693284 73.73128728723376) bank13436 +13437 POINT(40.3943954263125 74.37893097770018) bank13437 +13438 POINT(40.730202525728586 73.95249507662109) bank13438 +13439 POINT(39.82683060122219 74.8673642332835) bank13439 +13440 POINT(39.825281452567964 73.47638113756385) bank13440 +13441 POINT(40.3367215825803 73.2717638358577) bank13441 +13442 POINT(41.58087326283066 73.13610578362973) bank13442 +13443 POINT(41.55313099993485 73.76752285545349) bank13443 +13444 POINT(40.87919842369716 74.9409837148578) bank13444 +13445 POINT(41.036843412778 73.10066374408521) bank13445 +13446 POINT(41.70319235138119 73.87208101158151) bank13446 +13447 POINT(39.71599491947939 74.64396773688999) bank13447 +13448 POINT(39.79289008595548 74.25510942860724) bank13448 +13449 POINT(40.02088969491504 74.93026002609383) bank13449 +13450 POINT(40.571192871365085 73.09590095872596) bank13450 +13451 POINT(39.863069855754475 74.73636278888424) bank13451 +13452 POINT(41.015590196536145 74.95118850734985) bank13452 +13453 POINT(41.308014508196266 74.09759634728994) bank13453 +13454 POINT(40.694389536844554 74.21114754189799) bank13454 +13455 POINT(40.568775492937895 74.79213491200161) bank13455 +13456 POINT(41.28357154678552 74.76041825573095) bank13456 +13457 POINT(39.803714922652574 73.27565185072537) bank13457 +13458 POINT(41.3314644317024 73.22746052028845) bank13458 +13459 POINT(39.97308313936377 74.58307865771292) bank13459 +13460 POINT(40.186677074742164 74.1968101294023) bank13460 +13461 POINT(41.27981895124628 73.15273419549652) bank13461 +13462 POINT(40.508628582303594 74.46606768027335) bank13462 +13463 POINT(41.19488651751171 73.528607723836) bank13463 +13464 POINT(41.528179825233636 74.08897606181488) bank13464 +13465 POINT(41.16172863221615 74.72571898073578) bank13465 +13466 POINT(41.29860780818262 74.12659007787266) bank13466 +13467 POINT(40.655601276295144 74.32963961990757) bank13467 +13468 POINT(40.29008364711655 73.7806480614935) bank13468 +13469 POINT(40.323285885590856 74.44280236059602) bank13469 +13470 POINT(40.12649451400998 73.9547932576232) bank13470 +13471 POINT(41.22789370932537 73.85605067717154) bank13471 +13472 POINT(40.06847800691206 74.1903341935778) bank13472 +13473 POINT(41.268767019918855 74.56751417852432) bank13473 +13474 POINT(40.85504135333534 74.90332746522222) bank13474 +13475 POINT(39.714923687615496 73.68036593645036) bank13475 +13476 POINT(41.41664169439304 74.69846830085228) bank13476 +13477 POINT(40.40070341147399 74.07912885936878) bank13477 +13478 POINT(41.50185307606723 73.12782942783396) bank13478 +13479 POINT(41.54677034885221 73.13550582333822) bank13479 +13480 POINT(40.675113146374656 74.39623001322433) bank13480 +13481 POINT(39.81403101521196 74.51105482342194) bank13481 +13482 POINT(40.600660196682135 73.72975324746072) bank13482 +13483 POINT(39.749371858552536 73.36379766628797) bank13483 +13484 POINT(41.002698258082674 74.0542158712617) bank13484 +13485 POINT(40.822986822941516 74.88341969792177) bank13485 +13486 POINT(39.90039852156993 73.78098049259381) bank13486 +13487 POINT(40.292705464458585 74.31269178971388) bank13487 +13488 POINT(41.486543195491755 73.20579909178291) bank13488 +13489 POINT(41.460315141381045 74.28102280430987) bank13489 +13490 POINT(40.19885110557706 74.94929633337371) bank13490 +13491 POINT(40.016747680555 73.48077268778032) bank13491 +13492 POINT(40.54132370478542 73.35396576064272) bank13492 +13493 POINT(41.48711567719318 74.1014607269031) bank13493 +13494 POINT(40.48061244466316 74.42474348490833) bank13494 +13495 POINT(41.63843104542462 73.8046358341588) bank13495 +13496 POINT(39.78824302679453 73.31511410244606) bank13496 +13497 POINT(40.50587648878387 73.29942388035532) bank13497 +13498 POINT(41.01022657131808 73.75663404065966) bank13498 +13499 POINT(41.41770040699946 73.44524618462818) bank13499 +13500 POINT(41.21329530800015 74.1070888816342) bank13500 +13501 POINT(40.153136056007604 74.20533665785698) bank13501 +13502 POINT(41.297640347089065 73.20452146100415) bank13502 +13503 POINT(41.17458924373943 74.85376319981356) bank13503 +13504 POINT(41.022004296933986 74.93926942112009) bank13504 +13505 POINT(41.537483153634355 74.48530599970968) bank13505 +13506 POINT(40.98266286932432 73.15549590643946) bank13506 +13507 POINT(40.25711756153831 74.91139437397436) bank13507 +13508 POINT(39.93744624870141 74.74910174474003) bank13508 +13509 POINT(40.766266405897426 73.30602495808824) bank13509 +13510 POINT(39.94826157692157 73.23838031615304) bank13510 +13511 POINT(40.420034265821876 74.2951137734087) bank13511 +13512 POINT(40.816974858930756 74.63653116325821) bank13512 +13513 POINT(41.155752594406586 74.37462535041307) bank13513 +13514 POINT(41.60047383341002 73.0730344329911) bank13514 +13515 POINT(39.85689932100434 74.78885277491908) bank13515 +13516 POINT(40.12327557947553 74.12849260011866) bank13516 +13517 POINT(40.364275537808005 73.02343066410657) bank13517 +13518 POINT(41.01892435331815 73.55510822652171) bank13518 +13519 POINT(41.431193727220766 73.55973377966781) bank13519 +13520 POINT(41.43567113408683 73.08376329389503) bank13520 +13521 POINT(41.43822777335005 73.21887450745551) bank13521 +13522 POINT(40.13507363972802 74.33423195559902) bank13522 +13523 POINT(40.708532780877356 73.76838323765998) bank13523 +13524 POINT(39.731533732391306 73.55149568637243) bank13524 +13525 POINT(40.89967784299542 73.76376354748616) bank13525 +13526 POINT(40.12926678497336 74.10220766723602) bank13526 +13527 POINT(41.0801573309887 74.57147155015703) bank13527 +13528 POINT(41.61716334977967 73.37558851650043) bank13528 +13529 POINT(40.220756677030515 73.29918683550697) bank13529 +13530 POINT(39.75960216152691 74.00429279987503) bank13530 +13531 POINT(40.43316558593656 73.67029672659858) bank13531 +13532 POINT(40.233511947808616 74.96578854439585) bank13532 +13533 POINT(41.31811840290274 74.80757943924482) bank13533 +13534 POINT(41.41660874136008 73.80697265658057) bank13534 +13535 POINT(40.0000244582011 74.23411939676008) bank13535 +13536 POINT(40.31054977118283 74.33419124982663) bank13536 +13537 POINT(40.209594869947665 74.23684060485785) bank13537 +13538 POINT(40.80772996256201 73.33632089433935) bank13538 +13539 POINT(40.75091905618552 74.50226427332035) bank13539 +13540 POINT(40.25450433884181 73.88296735041507) bank13540 +13541 POINT(40.465988552829316 73.5668654782619) bank13541 +13542 POINT(41.282832584864536 74.49891635591202) bank13542 +13543 POINT(40.0723694343169 73.86704482469801) bank13543 +13544 POINT(41.158113750587525 74.95117432366253) bank13544 +13545 POINT(39.98219133743521 74.47285184809054) bank13545 +13546 POINT(39.79950110381561 73.81851116465828) bank13546 +13547 POINT(40.4184529128549 73.98429474759344) bank13547 +13548 POINT(41.12629535288372 74.49367125067008) bank13548 +13549 POINT(40.40148553947356 73.04245947202914) bank13549 +13550 POINT(39.988090708465045 73.85130295173238) bank13550 +13551 POINT(40.26377298305023 73.67881876574918) bank13551 +13552 POINT(40.013276956104306 73.09416962952939) bank13552 +13553 POINT(40.44482685962488 73.81629037021996) bank13553 +13554 POINT(40.46679778871272 74.26157637954446) bank13554 +13555 POINT(40.24733636765397 74.07586289029967) bank13555 +13556 POINT(40.97309427840795 73.52773864763556) bank13556 +13557 POINT(40.51237841024341 73.032317825775) bank13557 +13558 POINT(40.95134906907514 74.45573627697348) bank13558 +13559 POINT(40.355789322579206 74.72472688582563) bank13559 +13560 POINT(40.726011326186565 73.32235368139635) bank13560 +13561 POINT(40.456112605766855 73.27405168580134) bank13561 +13562 POINT(40.564986763383324 74.92977748005904) bank13562 +13563 POINT(41.53505300426077 73.7813200523862) bank13563 +13564 POINT(40.45468155828014 74.04957816733314) bank13564 +13565 POINT(40.10721195249339 74.96139933218936) bank13565 +13566 POINT(41.08646106647294 73.21123624736025) bank13566 +13567 POINT(41.6155238416834 74.959558575782) bank13567 +13568 POINT(39.7822088321222 73.81906127041364) bank13568 +13569 POINT(40.137905848992496 73.49577667165806) bank13569 +13570 POINT(40.068803988531215 74.5423892246062) bank13570 +13571 POINT(40.591109205479796 74.68499616612831) bank13571 +13572 POINT(40.275510782055115 74.17302364330185) bank13572 +13573 POINT(40.749493948330006 73.0408747409281) bank13573 +13574 POINT(40.17272093178609 73.20742271857299) bank13574 +13575 POINT(41.66574934097506 73.99713644024801) bank13575 +13576 POINT(41.58113667926092 73.1407515390941) bank13576 +13577 POINT(39.917808902908305 73.38107733331526) bank13577 +13578 POINT(40.02938409346849 73.78449667192692) bank13578 +13579 POINT(40.66499810526712 73.58889190673368) bank13579 +13580 POINT(41.566714539241985 73.67858570788705) bank13580 +13581 POINT(41.46346400070778 74.49728406339571) bank13581 +13582 POINT(41.62203755020339 74.26574474923285) bank13582 +13583 POINT(39.88502599074556 74.17934508569395) bank13583 +13584 POINT(39.852467292719254 74.3969002024385) bank13584 +13585 POINT(40.72853013635328 73.20403197617196) bank13585 +13586 POINT(41.08748889956118 73.25190984546231) bank13586 +13587 POINT(40.969779507158 74.13737959374916) bank13587 +13588 POINT(40.441412853417184 74.13057413548796) bank13588 +13589 POINT(40.44278777384584 73.96724992747923) bank13589 +13590 POINT(39.84532072276788 74.9411164825691) bank13590 +13591 POINT(40.28497223649967 74.76387871919707) bank13591 +13592 POINT(40.525298150519696 73.93562336141495) bank13592 +13593 POINT(40.26465310922406 73.82696739553302) bank13593 +13594 POINT(40.820365883850556 73.01312122067928) bank13594 +13595 POINT(41.20845351221805 74.62384746385402) bank13595 +13596 POINT(41.01702131677703 73.1077414882442) bank13596 +13597 POINT(40.32755769214321 73.45594770228017) bank13597 +13598 POINT(41.46934476407026 73.88566625553769) bank13598 +13599 POINT(40.084114663171675 74.4011227523671) bank13599 +13600 POINT(40.87000409721111 74.95208797593796) bank13600 +13601 POINT(40.00301850063999 73.66464811997035) bank13601 +13602 POINT(40.42746983700461 74.563589899108) bank13602 +13603 POINT(41.54748212258429 74.03673034754587) bank13603 +13604 POINT(39.83016578258806 73.23499532122119) bank13604 +13605 POINT(41.07448551051469 74.9531348163638) bank13605 +13606 POINT(40.30835111876553 74.14403057645795) bank13606 +13607 POINT(41.49706803379209 73.02447913664716) bank13607 +13608 POINT(41.451416595942796 74.7110476471932) bank13608 +13609 POINT(40.55286652236556 74.81402496866941) bank13609 +13610 POINT(39.90644480263804 74.1089105686912) bank13610 +13611 POINT(40.88709984039831 73.26038123918823) bank13611 +13612 POINT(39.72537471407514 74.8294343053379) bank13612 +13613 POINT(39.74297731287033 74.37959318867544) bank13613 +13614 POINT(40.76799916108847 74.59483118733233) bank13614 +13615 POINT(41.6615187910359 73.01123718133952) bank13615 +13616 POINT(40.691627886636184 74.26158255800965) bank13616 +13617 POINT(41.6012884660144 74.87420996747215) bank13617 +13618 POINT(40.07394223256973 73.46326730997428) bank13618 +13619 POINT(41.26547072335738 74.99673024648511) bank13619 +13620 POINT(41.634303859356876 74.63206402073405) bank13620 +13621 POINT(40.4265500248749 73.58902144997451) bank13621 +13622 POINT(40.3679222110955 74.42134416275367) bank13622 +13623 POINT(40.10524771193557 73.62682807086182) bank13623 +13624 POINT(41.587391786731075 74.87845449250253) bank13624 +13625 POINT(41.38756864721742 74.64120813140458) bank13625 +13626 POINT(40.74059163489714 74.11947160519323) bank13626 +13627 POINT(41.332155307874345 74.67228073449058) bank13627 +13628 POINT(40.17002272867433 73.87195925824777) bank13628 +13629 POINT(41.65634968016375 74.603853166086) bank13629 +13630 POINT(40.85814388162971 73.95481276864537) bank13630 +13631 POINT(40.89401612442517 73.87672611482755) bank13631 +13632 POINT(41.255543322652386 73.94787626904473) bank13632 +13633 POINT(41.192198555868075 74.11312158345453) bank13633 +13634 POINT(40.86894756740943 74.8710933647627) bank13634 +13635 POINT(40.235656804479426 74.9053060344755) bank13635 +13636 POINT(41.33236186827557 74.8245598808431) bank13636 +13637 POINT(40.17123448373617 73.28020346674813) bank13637 +13638 POINT(41.67847410453529 73.06745696232976) bank13638 +13639 POINT(39.80993073935183 73.37420227430954) bank13639 +13640 POINT(40.41008912485842 73.18739792469323) bank13640 +13641 POINT(40.30972367179383 74.82014777349688) bank13641 +13642 POINT(41.489930469688254 74.04240582085026) bank13642 +13643 POINT(41.053666569215 73.18798573148959) bank13643 +13644 POINT(40.77799857576329 74.45763787791269) bank13644 +13645 POINT(41.03233851680143 73.80946359184345) bank13645 +13646 POINT(40.65315277957466 73.92128945877478) bank13646 +13647 POINT(41.21047273741366 74.39259567407252) bank13647 +13648 POINT(40.87829557941117 74.19965277482312) bank13648 +13649 POINT(40.706916618302834 73.52812637610285) bank13649 +13650 POINT(40.678387022376974 73.78722023699376) bank13650 +13651 POINT(40.50500599411995 73.14670292984681) bank13651 +13652 POINT(40.154069994113115 74.92290543309379) bank13652 +13653 POINT(40.7819707332798 73.4348642737163) bank13653 +13654 POINT(41.136775049597496 73.26389259726028) bank13654 +13655 POINT(39.88471174828655 73.89119565653377) bank13655 +13656 POINT(41.5916765153858 73.33921403096129) bank13656 +13657 POINT(41.547214053416404 73.40012543948328) bank13657 +13658 POINT(40.03828808523505 74.21301835422632) bank13658 +13659 POINT(40.69060479110542 74.88737554709282) bank13659 +13660 POINT(40.497727712503014 74.1437866719404) bank13660 +13661 POINT(40.51185653867874 73.65598096532969) bank13661 +13662 POINT(41.157728574804615 73.80447282743225) bank13662 +13663 POINT(41.03168449209664 73.80247804840751) bank13663 +13664 POINT(40.15933734388778 74.85259122270283) bank13664 +13665 POINT(40.58938508288103 73.79930584099372) bank13665 +13666 POINT(40.62460331832449 73.40990973213707) bank13666 +13667 POINT(40.633958897851585 73.1099997392964) bank13667 +13668 POINT(40.04378525054217 74.40986137623977) bank13668 +13669 POINT(40.22842644495897 73.56251151492977) bank13669 +13670 POINT(40.702773926893734 74.12623068153165) bank13670 +13671 POINT(40.682496148385006 73.80244522493003) bank13671 +13672 POINT(41.5299499510145 73.06064668908533) bank13672 +13673 POINT(40.22416733800258 74.7478589884513) bank13673 +13674 POINT(40.94535341027399 73.88615539163544) bank13674 +13675 POINT(41.39687406280109 73.61060850477168) bank13675 +13676 POINT(39.89894579842653 73.50756627478837) bank13676 +13677 POINT(40.00214386747992 74.2689466361773) bank13677 +13678 POINT(41.16134645960487 73.55840788621813) bank13678 +13679 POINT(40.05614030559699 73.37089695831348) bank13679 +13680 POINT(40.43169252325852 73.01526124899068) bank13680 +13681 POINT(41.288222372205574 73.7605844977842) bank13681 +13682 POINT(40.27536283230378 74.89411365949958) bank13682 +13683 POINT(41.581479408444444 74.79209342922556) bank13683 +13684 POINT(40.323806390287956 73.67123260170345) bank13684 +13685 POINT(41.55557596768645 74.57978325097498) bank13685 +13686 POINT(40.47602490336061 74.28751370327885) bank13686 +13687 POINT(40.17542834105483 74.24489157718226) bank13687 +13688 POINT(41.30617555162249 74.09485509823142) bank13688 +13689 POINT(41.68214036478855 74.7627092716357) bank13689 +13690 POINT(41.42066266162622 73.20499893628629) bank13690 +13691 POINT(40.41038157837681 73.64727419272086) bank13691 +13692 POINT(40.766640007784225 73.66690693153556) bank13692 +13693 POINT(40.50954558519433 74.03610843870422) bank13693 +13694 POINT(40.78743722852893 74.9228254584259) bank13694 +13695 POINT(40.396168554456246 73.45436195574122) bank13695 +13696 POINT(40.7557069280697 74.19489452789847) bank13696 +13697 POINT(41.211674327901754 74.47155281224528) bank13697 +13698 POINT(39.80406587021397 74.6675037764157) bank13698 +13699 POINT(40.544512704356144 74.83307455422097) bank13699 +13700 POINT(39.81896989488248 73.03772968620633) bank13700 +13701 POINT(41.47296798504409 73.5260813896885) bank13701 +13702 POINT(40.30155840469585 74.32415008150164) bank13702 +13703 POINT(40.45228557051116 73.54522412595611) bank13703 +13704 POINT(40.63261022272385 73.37188361945606) bank13704 +13705 POINT(40.21459922878799 74.01183199077121) bank13705 +13706 POINT(40.29532419056372 73.71494476708604) bank13706 +13707 POINT(41.35124773152212 74.29776235229487) bank13707 +13708 POINT(40.27818602927158 74.57450819347882) bank13708 +13709 POINT(41.470716725491556 74.4936485338783) bank13709 +13710 POINT(40.16749179221673 74.35184981337042) bank13710 +13711 POINT(40.269121939770685 73.97031050346689) bank13711 +13712 POINT(40.87209692452068 73.40729016487855) bank13712 +13713 POINT(40.610461570607285 73.50469706260462) bank13713 +13714 POINT(41.33167612771976 74.55735630011235) bank13714 +13715 POINT(40.009451978215274 73.86183617846437) bank13715 +13716 POINT(40.840741027352955 74.56639969079771) bank13716 +13717 POINT(40.1558587565544 73.55116829999393) bank13717 +13718 POINT(40.82845618752809 73.7653602085436) bank13718 +13719 POINT(40.40416825601167 73.40967089663617) bank13719 +13720 POINT(41.06122332981254 73.64809178030222) bank13720 +13721 POINT(40.76448225520028 74.97171437196515) bank13721 +13722 POINT(41.384855748466244 73.83414315392737) bank13722 +13723 POINT(40.540550742103264 73.63652865607321) bank13723 +13724 POINT(41.36437886706744 73.98297225158842) bank13724 +13725 POINT(40.51513068299768 73.54299176017855) bank13725 +13726 POINT(39.88735631062818 74.75921508212959) bank13726 +13727 POINT(40.95025912012449 74.75061435884508) bank13727 +13728 POINT(41.34043087240331 73.35970406901976) bank13728 +13729 POINT(40.03062814396423 74.01423416002524) bank13729 +13730 POINT(41.132623676585794 73.35889004888539) bank13730 +13731 POINT(40.38214291967865 73.37625096090383) bank13731 +13732 POINT(40.926633331270345 74.40730823968912) bank13732 +13733 POINT(41.09881815006386 73.7170436575189) bank13733 +13734 POINT(40.595061768664515 74.44531981192742) bank13734 +13735 POINT(40.83181573384189 73.17743569663166) bank13735 +13736 POINT(40.44420611647643 73.87416652915961) bank13736 +13737 POINT(40.953401860448224 73.84385663919896) bank13737 +13738 POINT(40.51190615429497 73.3590513905019) bank13738 +13739 POINT(41.44061549844793 73.3271507540212) bank13739 +13740 POINT(40.84799882127249 73.02509969285083) bank13740 +13741 POINT(40.77158213299141 73.17452628096959) bank13741 +13742 POINT(41.63451342080822 74.78039655108442) bank13742 +13743 POINT(40.72671018240416 73.33035299183653) bank13743 +13744 POINT(41.18872512070768 74.57778565454107) bank13744 +13745 POINT(41.42390462784883 73.73015660038921) bank13745 +13746 POINT(40.252853993444184 74.23392897701339) bank13746 +13747 POINT(41.61862334539689 74.4083335677148) bank13747 +13748 POINT(40.70594376958243 73.67109254977056) bank13748 +13749 POINT(40.83271002039334 73.68563838408286) bank13749 +13750 POINT(40.68692285439835 73.48889741310587) bank13750 +13751 POINT(40.920552967827454 73.64285397117408) bank13751 +13752 POINT(40.85465149477341 73.86566192787552) bank13752 +13753 POINT(41.61169696701205 74.98328350963264) bank13753 +13754 POINT(40.77032223441684 73.64097517201517) bank13754 +13755 POINT(40.407356857700925 73.54402633936226) bank13755 +13756 POINT(40.18367601296147 73.4597193448275) bank13756 +13757 POINT(40.90767057271016 73.98650698924894) bank13757 +13758 POINT(41.59483421015995 73.19741324226543) bank13758 +13759 POINT(41.24205513945079 74.75729054839348) bank13759 +13760 POINT(39.778827894192396 73.51541670703408) bank13760 +13761 POINT(41.31141656581516 73.0183664123987) bank13761 +13762 POINT(41.33406605017812 74.70886136886573) bank13762 +13763 POINT(41.055050709934065 73.06735958789373) bank13763 +13764 POINT(40.58891298083449 73.81389679184116) bank13764 +13765 POINT(40.11751815097841 74.6627399295714) bank13765 +13766 POINT(40.7815781961565 74.60484603841014) bank13766 +13767 POINT(41.61209152729629 73.82204503055303) bank13767 +13768 POINT(40.09036182225574 74.66745348574838) bank13768 +13769 POINT(40.05797644714041 73.90925908847143) bank13769 +13770 POINT(40.616028257357584 74.68970579716938) bank13770 +13771 POINT(41.15561447440611 74.61418422580256) bank13771 +13772 POINT(41.4272804534778 75.00062321605986) bank13772 +13773 POINT(41.025001546313874 74.78461977644267) bank13773 +13774 POINT(41.0074600322202 73.5577088836713) bank13774 +13775 POINT(40.02834958548837 74.59498555623212) bank13775 +13776 POINT(40.86609085984318 74.18339525248716) bank13776 +13777 POINT(40.372484964329914 74.07463136645516) bank13777 +13778 POINT(41.61364176081049 74.58509939310477) bank13778 +13779 POINT(40.244725117759735 74.5458822719708) bank13779 +13780 POINT(40.74620998650399 73.94093842279113) bank13780 +13781 POINT(39.73657442642268 74.819665123972) bank13781 +13782 POINT(41.1337951358304 74.92709758725242) bank13782 +13783 POINT(40.366130425807775 74.92259936658193) bank13783 +13784 POINT(40.83140455245601 73.1372931918592) bank13784 +13785 POINT(39.9090922252412 73.71186474721588) bank13785 +13786 POINT(41.658941553727196 74.67588807059828) bank13786 +13787 POINT(40.90093479787946 74.46539953465808) bank13787 +13788 POINT(40.39469278885393 74.22167250447653) bank13788 +13789 POINT(41.342276680999824 73.02304624809865) bank13789 +13790 POINT(40.664948838608495 73.62808598618729) bank13790 +13791 POINT(40.171552755612566 74.57299096157736) bank13791 +13792 POINT(40.734703172127695 74.65351879757958) bank13792 +13793 POINT(40.06195182806638 73.76480338915528) bank13793 +13794 POINT(40.51008226531494 74.53814119069015) bank13794 +13795 POINT(40.765849654113936 73.04023327001863) bank13795 +13796 POINT(39.932340087269985 74.529636508101) bank13796 +13797 POINT(40.36443560872091 73.68693728745484) bank13797 +13798 POINT(40.62166465269354 74.91795835974729) bank13798 +13799 POINT(39.87871526622545 74.6760859526462) bank13799 +13800 POINT(40.588107266656955 73.24120121216934) bank13800 +13801 POINT(41.31571666355934 73.117349505185) bank13801 +13802 POINT(39.98704205785714 73.03018527880778) bank13802 +13803 POINT(40.704825435073104 73.96323251702805) bank13803 +13804 POINT(40.540194075340395 74.43115620818145) bank13804 +13805 POINT(41.62368599491198 73.6589760718364) bank13805 +13806 POINT(41.10367411187237 73.07488674532148) bank13806 +13807 POINT(41.69376865538917 73.69379977726037) bank13807 +13808 POINT(40.94573352353935 74.45305445605989) bank13808 +13809 POINT(40.186114861278476 74.28773572399487) bank13809 +13810 POINT(41.451708401199625 74.3261228715161) bank13810 +13811 POINT(40.89205939476996 73.59396782694084) bank13811 +13812 POINT(40.108476588168934 74.1520963194726) bank13812 +13813 POINT(41.25029633471388 74.57979997105431) bank13813 +13814 POINT(39.728773038711566 73.52542076976594) bank13814 +13815 POINT(40.693799892382266 74.83740091446928) bank13815 +13816 POINT(39.860971503438755 74.9045683620552) bank13816 +13817 POINT(41.50364900052216 74.8602048671328) bank13817 +13818 POINT(41.00581756449402 73.5594728663137) bank13818 +13819 POINT(41.019896325421826 74.9272834514872) bank13819 +13820 POINT(40.03583757911932 73.13358503087237) bank13820 +13821 POINT(39.960543532528526 73.12762486309771) bank13821 +13822 POINT(40.321424487584274 74.92217087547876) bank13822 +13823 POINT(41.3663186545632 74.60064400481434) bank13823 +13824 POINT(40.26723362073877 74.69252471919673) bank13824 +13825 POINT(41.627012677658506 74.94369144055095) bank13825 +13826 POINT(40.5530646795081 74.1906489688548) bank13826 +13827 POINT(40.080306312669535 74.17785333026714) bank13827 +13828 POINT(41.440565605844206 73.43934557096142) bank13828 +13829 POINT(40.28838504789313 73.41690335106169) bank13829 +13830 POINT(41.4501349566998 73.9311700256967) bank13830 +13831 POINT(40.652960524083845 74.13561594007031) bank13831 +13832 POINT(40.45026197835446 73.21073193274641) bank13832 +13833 POINT(40.36535669896713 73.27702003161501) bank13833 +13834 POINT(40.12896747411657 74.15481306604045) bank13834 +13835 POINT(40.72532537946409 73.47156497433645) bank13835 +13836 POINT(41.1336527163502 74.39199393005401) bank13836 +13837 POINT(39.93904035530459 75.00217278564794) bank13837 +13838 POINT(40.29881449753216 74.0730559638794) bank13838 +13839 POINT(39.87841092300824 74.55707644197592) bank13839 +13840 POINT(41.506348495927135 74.09197210999842) bank13840 +13841 POINT(41.40179566605818 74.31123944442535) bank13841 +13842 POINT(40.97581622889334 73.16887653317244) bank13842 +13843 POINT(40.261905975344675 73.67937312428622) bank13843 +13844 POINT(41.401784868293745 74.64077954411648) bank13844 +13845 POINT(40.45682512555236 73.79902281363476) bank13845 +13846 POINT(39.75613932321953 73.1166570350878) bank13846 +13847 POINT(41.584103135787615 74.67032294930922) bank13847 +13848 POINT(40.49304543285763 74.13281944536098) bank13848 +13849 POINT(40.18850409862642 73.00707090290662) bank13849 +13850 POINT(40.380097064892155 73.34890159481186) bank13850 +13851 POINT(39.729142581097165 73.82988000525167) bank13851 +13852 POINT(40.81829096335987 73.17150418752293) bank13852 +13853 POINT(41.37816074155761 74.89498666248386) bank13853 +13854 POINT(41.40963971448736 73.287900603592) bank13854 +13855 POINT(40.510918436139434 74.5083801783384) bank13855 +13856 POINT(40.59864395473424 74.89558825508279) bank13856 +13857 POINT(40.89681832966583 74.32437478554823) bank13857 +13858 POINT(40.12738119954384 74.80348875575426) bank13858 +13859 POINT(41.30184818995411 74.54767351635414) bank13859 +13860 POINT(40.58363194142143 74.79009408325688) bank13860 +13861 POINT(40.65338624233777 73.14051041016043) bank13861 +13862 POINT(40.26347992476149 74.10621077999382) bank13862 +13863 POINT(40.45321090826998 73.3306331953056) bank13863 +13864 POINT(39.96338225472811 74.83662443129903) bank13864 +13865 POINT(40.14207060149606 74.6549947844342) bank13865 +13866 POINT(41.22398989585284 73.16940105123498) bank13866 +13867 POINT(40.18010387578358 74.60810307151439) bank13867 +13868 POINT(40.43250623717913 74.22236771680294) bank13868 +13869 POINT(41.70034057717346 74.16500984615365) bank13869 +13870 POINT(40.734275996997845 74.69887282982897) bank13870 +13871 POINT(41.66144585040846 73.0080871738057) bank13871 +13872 POINT(40.78486136151235 73.44983768713776) bank13872 +13873 POINT(40.291034344034756 74.60278402441749) bank13873 +13874 POINT(41.63678818232742 74.7958192648496) bank13874 +13875 POINT(39.75493255647338 73.89929358581382) bank13875 +13876 POINT(41.33678307685254 74.59843933579248) bank13876 +13877 POINT(40.36838887014085 73.03427214925281) bank13877 +13878 POINT(40.509041057882065 74.69332164286021) bank13878 +13879 POINT(41.14773912947746 73.02247979874892) bank13879 +13880 POINT(40.34175557431334 74.00361009431812) bank13880 +13881 POINT(41.70030319651734 73.09669933036253) bank13881 +13882 POINT(41.31508078854261 75.00589645758348) bank13882 +13883 POINT(40.597518390339076 74.89615152666005) bank13883 +13884 POINT(41.35361513414117 74.4235255785889) bank13884 +13885 POINT(41.094849788169164 73.58347255001054) bank13885 +13886 POINT(39.96834409222628 73.52879220439651) bank13886 +13887 POINT(40.17376345879381 73.36015303578041) bank13887 +13888 POINT(39.891099750722326 74.22301452819579) bank13888 +13889 POINT(41.50126327438056 74.54611275027116) bank13889 +13890 POINT(40.56209357185101 73.41922856862695) bank13890 +13891 POINT(40.196582585736934 73.12626818723486) bank13891 +13892 POINT(40.407841211137125 73.14102445807994) bank13892 +13893 POINT(41.46414493940921 74.7606447184224) bank13893 +13894 POINT(41.30099337755445 73.0916228751599) bank13894 +13895 POINT(40.883431003404304 73.05996067745053) bank13895 +13896 POINT(40.69246662309845 73.9452276046516) bank13896 +13897 POINT(41.119795510315605 74.10805097792205) bank13897 +13898 POINT(39.72555799016862 73.28455679378213) bank13898 +13899 POINT(39.98723759385434 74.38991503747918) bank13899 +13900 POINT(41.528289860957 73.53931596047458) bank13900 +13901 POINT(41.477696159214915 73.43597121539723) bank13901 +13902 POINT(39.96471986907182 74.80931615896671) bank13902 +13903 POINT(39.87090759317065 73.60529571656124) bank13903 +13904 POINT(40.94702372242471 74.90754562807861) bank13904 +13905 POINT(40.818387035770044 74.70892101782223) bank13905 +13906 POINT(41.24828082717492 74.63751011932631) bank13906 +13907 POINT(41.5184767155601 73.9296048580521) bank13907 +13908 POINT(41.64618116013749 74.09947654625293) bank13908 +13909 POINT(39.75732433509135 74.83417238551871) bank13909 +13910 POINT(39.819689842992695 73.7404982507068) bank13910 +13911 POINT(41.48678977347002 74.71822294033076) bank13911 +13912 POINT(41.488208220948515 73.11223278124086) bank13912 +13913 POINT(41.452731882472335 73.66917921563854) bank13913 +13914 POINT(41.3208691070657 74.71763743304597) bank13914 +13915 POINT(41.01937721399588 73.14157414182056) bank13915 +13916 POINT(41.541103721421344 73.8196989444915) bank13916 +13917 POINT(40.75206981650317 74.2502620313616) bank13917 +13918 POINT(41.3149187891743 73.17327425759532) bank13918 +13919 POINT(41.464378221345 74.89799404370805) bank13919 +13920 POINT(40.793447383950294 74.35402903091425) bank13920 +13921 POINT(40.76784337425529 73.52666791859396) bank13921 +13922 POINT(40.34202576345149 73.23525010983964) bank13922 +13923 POINT(41.35917345470878 74.07911260211515) bank13923 +13924 POINT(41.44566797388014 74.43270560814528) bank13924 +13925 POINT(40.467177803658636 74.9464161201736) bank13925 +13926 POINT(40.77646372465929 73.48485022689052) bank13926 +13927 POINT(39.96269328081922 74.59295429391624) bank13927 +13928 POINT(40.34463853750484 74.69019320500065) bank13928 +13929 POINT(41.03110568562865 73.50837790216606) bank13929 +13930 POINT(39.80093644650028 73.67568144207037) bank13930 +13931 POINT(40.02939665744796 74.13283052930514) bank13931 +13932 POINT(39.92692240387051 73.77961401130554) bank13932 +13933 POINT(40.42349201468478 74.21266164849537) bank13933 +13934 POINT(41.38774673857829 73.32945467866845) bank13934 +13935 POINT(39.96695683435743 74.72828201050221) bank13935 +13936 POINT(40.94834983025244 73.20866621514583) bank13936 +13937 POINT(41.1813579827646 73.78820462815102) bank13937 +13938 POINT(41.379009013414134 73.51713969169577) bank13938 +13939 POINT(40.739890097309896 74.19423219093831) bank13939 +13940 POINT(41.316770278026695 73.86300648683383) bank13940 +13941 POINT(40.212263485786565 73.7577906105237) bank13941 +13942 POINT(40.39875003067474 73.30674527382986) bank13942 +13943 POINT(41.27699802424962 73.42962631866237) bank13943 +13944 POINT(40.098288096780294 74.66293458361427) bank13944 +13945 POINT(40.16135320790513 74.1245223583511) bank13945 +13946 POINT(40.50751528097821 73.91722804053023) bank13946 +13947 POINT(41.29073362899021 73.25091381801116) bank13947 +13948 POINT(41.438749945267446 74.44285528537628) bank13948 +13949 POINT(41.106143261705995 73.61215972444634) bank13949 +13950 POINT(41.238460125230084 74.8366703443915) bank13950 +13951 POINT(40.00227266214079 73.09487354382358) bank13951 +13952 POINT(41.606364814508666 73.80732322782545) bank13952 +13953 POINT(39.77487840607861 73.68572619990353) bank13953 +13954 POINT(40.49587733642797 73.80160969661297) bank13954 +13955 POINT(41.49662889300785 74.72641540463935) bank13955 +13956 POINT(41.28114732873016 73.37450516175512) bank13956 +13957 POINT(41.106721673179884 73.2635178425087) bank13957 +13958 POINT(40.734960391492706 74.90408571125542) bank13958 +13959 POINT(40.334996201212896 74.02489014000749) bank13959 +13960 POINT(40.80588663178691 74.52556514456217) bank13960 +13961 POINT(41.493134849199 73.58013523318529) bank13961 +13962 POINT(40.99818819958305 74.30023984065411) bank13962 +13963 POINT(40.17634444312071 73.08933161410305) bank13963 +13964 POINT(41.028010137202955 73.09430155692215) bank13964 +13965 POINT(40.92313738202352 73.27001386712678) bank13965 +13966 POINT(40.87663012573985 73.07014609638911) bank13966 +13967 POINT(40.089367959905545 74.27838753573116) bank13967 +13968 POINT(39.92736793730443 74.82672709787659) bank13968 +13969 POINT(40.48497493256772 74.7842925834999) bank13969 +13970 POINT(40.01332378102697 74.05959877217954) bank13970 +13971 POINT(39.99039802297883 74.12022536030942) bank13971 +13972 POINT(40.68970622776349 73.20488554575984) bank13972 +13973 POINT(40.14409189210909 73.12633790195096) bank13973 +13974 POINT(40.707467846185 74.708860370955) bank13974 +13975 POINT(40.86238101410648 73.51610885524543) bank13975 +13976 POINT(40.29225570009822 74.70583792039956) bank13976 +13977 POINT(39.77801147279873 74.22842216734814) bank13977 +13978 POINT(39.984171621611935 74.30129130389459) bank13978 +13979 POINT(40.64019980410862 74.28798299437895) bank13979 +13980 POINT(40.72633511731865 74.20416288496763) bank13980 +13981 POINT(41.467121678279234 73.48865352483375) bank13981 +13982 POINT(40.491230994389916 73.25485911196422) bank13982 +13983 POINT(40.215063215787524 74.559205690953) bank13983 +13984 POINT(40.033861691930454 73.06809940082178) bank13984 +13985 POINT(41.08888453491802 74.43677859352171) bank13985 +13986 POINT(41.35867378600058 73.68150406121762) bank13986 +13987 POINT(40.32255631957419 74.74899825339638) bank13987 +13988 POINT(39.96722925122096 74.78312942656336) bank13988 +13989 POINT(41.38019787418479 74.80549524473963) bank13989 +13990 POINT(41.17832482035416 74.08729169437746) bank13990 +13991 POINT(41.09382591597682 73.92215672069113) bank13991 +13992 POINT(40.46468000329851 73.81591804902402) bank13992 +13993 POINT(41.545583427658556 74.36203075125948) bank13993 +13994 POINT(40.10434076915925 73.29019765982314) bank13994 +13995 POINT(41.40648828211523 74.6770306203483) bank13995 +13996 POINT(41.43898382103958 74.09481738816689) bank13996 +13997 POINT(40.54107562900211 74.64648579430164) bank13997 +13998 POINT(40.987399440660674 73.1481479515811) bank13998 +13999 POINT(40.11801918799135 73.03286591256581) bank13999 +14000 POINT(40.747118082555794 73.43401695386459) bank14000 +14001 POINT(41.043019338464674 74.21125070211707) bank14001 +14002 POINT(41.26357730167914 73.29532770322332) bank14002 +14003 POINT(41.49018738282556 74.4929264652506) bank14003 +14004 POINT(41.40123497422488 74.76098513890445) bank14004 +14005 POINT(40.32706085025393 74.02009919426555) bank14005 +14006 POINT(40.30778748938771 73.04943769845893) bank14006 +14007 POINT(41.706744040214446 73.9229056382376) bank14007 +14008 POINT(39.93931790986348 74.15249637100426) bank14008 +14009 POINT(40.3690114051797 73.84001675075278) bank14009 +14010 POINT(40.34402767947408 73.7651675707838) bank14010 +14011 POINT(41.658325068091585 73.10222777074841) bank14011 +14012 POINT(39.753245932599775 73.52312355232132) bank14012 +14013 POINT(41.104569326753946 74.82568669978134) bank14013 +14014 POINT(39.76030704046179 73.67823879415364) bank14014 +14015 POINT(40.092074386508955 73.12958681200305) bank14015 +14016 POINT(40.38027232920182 74.69571259179388) bank14016 +14017 POINT(40.259061128997786 73.9970490503303) bank14017 +14018 POINT(40.144557965841656 73.7358655793764) bank14018 +14019 POINT(39.79720213903403 73.34378174007628) bank14019 +14020 POINT(41.35327056430565 73.2664021414439) bank14020 +14021 POINT(40.64280711489378 74.06680188866025) bank14021 +14022 POINT(40.200396811362175 73.43063826189) bank14022 +14023 POINT(40.11835793684291 73.09510028970494) bank14023 +14024 POINT(40.06105231732563 74.88710848000969) bank14024 +14025 POINT(41.11047608513478 73.05213946410042) bank14025 +14026 POINT(40.9745975634603 74.56142512248852) bank14026 +14027 POINT(39.796177801273714 74.64371590424287) bank14027 +14028 POINT(40.3307459128052 74.80223341653794) bank14028 +14029 POINT(41.27229471170979 73.78211656465395) bank14029 +14030 POINT(40.645790861116915 74.48995369886663) bank14030 +14031 POINT(39.980379338777176 73.3237567118576) bank14031 +14032 POINT(41.54549060321733 73.7703636317799) bank14032 +14033 POINT(41.64011321799989 74.58266272362897) bank14033 +14034 POINT(41.08188109709302 73.60567486124978) bank14034 +14035 POINT(40.688012095337854 73.43316245127222) bank14035 +14036 POINT(40.51634452002637 73.90492083853285) bank14036 +14037 POINT(40.63223044014257 73.76251007804564) bank14037 +14038 POINT(40.97956446856687 74.56403875203759) bank14038 +14039 POINT(41.14352322126693 74.03604880078225) bank14039 +14040 POINT(40.76657398104981 74.19234893273878) bank14040 +14041 POINT(41.04422751312804 73.89331876081606) bank14041 +14042 POINT(40.19129727993488 73.57616028448467) bank14042 +14043 POINT(41.470362566462825 73.45357836262973) bank14043 +14044 POINT(41.44200682069882 73.76810754859018) bank14044 +14045 POINT(41.23325298000518 73.33657992775393) bank14045 +14046 POINT(39.73674715637369 74.90961560040347) bank14046 +14047 POINT(39.864553849042956 74.77772870948488) bank14047 +14048 POINT(40.272491073080985 73.08990581766383) bank14048 +14049 POINT(41.07739475750475 74.32582870555677) bank14049 +14050 POINT(39.82132824216142 74.84073618199538) bank14050 +14051 POINT(40.04251303431717 74.04547054573456) bank14051 +14052 POINT(40.598966696633106 73.38670821302547) bank14052 +14053 POINT(41.64755883628397 73.27477280849537) bank14053 +14054 POINT(40.08495133976449 74.77540120511881) bank14054 +14055 POINT(41.47591795597411 73.7545252749138) bank14055 +14056 POINT(40.83722770680375 73.1168483579466) bank14056 +14057 POINT(40.427924033351914 74.25322512026538) bank14057 +14058 POINT(40.168098262941534 73.00725120442951) bank14058 +14059 POINT(40.0108261959765 73.56588100888986) bank14059 +14060 POINT(40.315064064598474 74.98516903823466) bank14060 +14061 POINT(41.61910837401774 74.39805613490348) bank14061 +14062 POINT(39.79587673446104 74.04853211130303) bank14062 +14063 POINT(40.85935335821274 73.483416451355) bank14063 +14064 POINT(40.50531903948912 73.62329171047277) bank14064 +14065 POINT(41.05840194473444 73.25803061630197) bank14065 +14066 POINT(41.36879435505713 73.72602906251898) bank14066 +14067 POINT(40.53197727015432 73.53145472588233) bank14067 +14068 POINT(41.274611451306214 73.54875064900614) bank14068 +14069 POINT(40.294037292501386 74.0232517221138) bank14069 +14070 POINT(39.834981752603085 74.13986272681642) bank14070 +14071 POINT(41.00914082056622 74.79001445331775) bank14071 +14072 POINT(40.47547987129414 74.20631199595837) bank14072 +14073 POINT(40.40487355176259 73.09605073853693) bank14073 +14074 POINT(40.15092336408166 74.80475677621196) bank14074 +14075 POINT(41.24318241163797 73.47898129858625) bank14075 +14076 POINT(40.24096060529797 74.42084689319259) bank14076 +14077 POINT(40.126477159300315 73.90184723052454) bank14077 +14078 POINT(40.458096279012004 74.95575257939721) bank14078 +14079 POINT(40.07121312001792 73.52770284198103) bank14079 +14080 POINT(41.39677222104753 73.95618988172444) bank14080 +14081 POINT(40.63081715747687 74.14344985592918) bank14081 +14082 POINT(41.483090968551785 74.94258553234162) bank14082 +14083 POINT(41.16738818881598 74.64725805664747) bank14083 +14084 POINT(40.22984674346756 74.18639464453855) bank14084 +14085 POINT(40.25937479222612 74.14079599274102) bank14085 +14086 POINT(41.204567138275905 74.70116223674404) bank14086 +14087 POINT(41.46330247189313 74.18111668902758) bank14087 +14088 POINT(39.79091346275057 74.54825360660674) bank14088 +14089 POINT(40.14160378585186 73.24025724066065) bank14089 +14090 POINT(41.32198090940629 74.72161230787094) bank14090 +14091 POINT(40.433306749755786 74.50784923319821) bank14091 +14092 POINT(39.78720753531129 74.76546446609309) bank14092 +14093 POINT(40.01462274508546 73.13000548602712) bank14093 +14094 POINT(40.5840352396433 74.01218887572081) bank14094 +14095 POINT(40.88691751521375 74.92835131919946) bank14095 +14096 POINT(41.56591524290873 73.63705003954597) bank14096 +14097 POINT(41.125232379904695 73.72616417793103) bank14097 +14098 POINT(40.9724589913413 74.42727716352424) bank14098 +14099 POINT(40.826330644768774 73.73614403524844) bank14099 +14100 POINT(40.877414583463704 74.64823353881353) bank14100 +14101 POINT(41.260801422412065 74.687137458082) bank14101 +14102 POINT(40.7350418482828 74.59594588515962) bank14102 +14103 POINT(40.32433228543993 73.42711226431902) bank14103 +14104 POINT(41.05365511424192 74.24556914431832) bank14104 +14105 POINT(40.279619664531815 73.22348657944465) bank14105 +14106 POINT(40.44032096560729 73.06700494374019) bank14106 +14107 POINT(41.04867834281705 74.41598230400531) bank14107 +14108 POINT(41.19617172906127 74.25431958098795) bank14108 +14109 POINT(40.79666037693159 73.4944645768303) bank14109 +14110 POINT(41.47505423551307 73.08151129608486) bank14110 +14111 POINT(40.64799629672198 74.03616964831504) bank14111 +14112 POINT(41.02303589786241 73.93065686191103) bank14112 +14113 POINT(40.87170583189582 74.44029031988467) bank14113 +14114 POINT(40.42858006541514 73.37871736829744) bank14114 +14115 POINT(40.64285227036615 73.39769833640199) bank14115 +14116 POINT(40.17293838136205 73.1255515967121) bank14116 +14117 POINT(40.700915951064104 74.96525608669062) bank14117 +14118 POINT(39.723562417990834 74.8484500833093) bank14118 +14119 POINT(41.38175875445039 73.08651802574775) bank14119 +14120 POINT(40.212083988935795 73.91990304699735) bank14120 +14121 POINT(40.048669108517956 74.06271077730356) bank14121 +14122 POINT(41.66436205021622 74.18653961471864) bank14122 +14123 POINT(40.60099684515193 74.08342382923004) bank14123 +14124 POINT(41.616222438240925 74.59612735039322) bank14124 +14125 POINT(40.50675926294427 74.16926650194706) bank14125 +14126 POINT(40.50650276098703 73.68935249571247) bank14126 +14127 POINT(40.35372355914474 73.69586325023452) bank14127 +14128 POINT(40.45342121897263 74.1355275080321) bank14128 +14129 POINT(40.68652573849626 73.05370838161816) bank14129 +14130 POINT(39.98988988867866 73.49344212908404) bank14130 +14131 POINT(41.01722593325746 73.64339833457231) bank14131 +14132 POINT(41.336287047961164 74.93538402413121) bank14132 +14133 POINT(40.5399075136097 74.13268446683446) bank14133 +14134 POINT(40.50316276866017 74.69733933228099) bank14134 +14135 POINT(40.32103727583421 74.69467088037625) bank14135 +14136 POINT(41.625760393020634 73.89721814361864) bank14136 +14137 POINT(40.03954266811636 74.17681793585696) bank14137 +14138 POINT(40.874272104945604 74.24949454762604) bank14138 +14139 POINT(41.553361824546585 74.53641083059574) bank14139 +14140 POINT(41.605638401552305 74.64748149786088) bank14140 +14141 POINT(40.32926451479066 74.76394983913713) bank14141 +14142 POINT(41.634362729098875 73.07806523933512) bank14142 +14143 POINT(41.67303964296767 73.2711134480274) bank14143 +14144 POINT(40.76265344062824 73.05208704311723) bank14144 +14145 POINT(39.93794059856032 74.65010243232906) bank14145 +14146 POINT(40.785127900255915 74.21904089871714) bank14146 +14147 POINT(41.275429313611205 73.05649442887787) bank14147 +14148 POINT(40.83180401402343 74.94258602820936) bank14148 +14149 POINT(40.23605087437437 73.59422379375349) bank14149 +14150 POINT(40.15500840996025 73.06564150412353) bank14150 +14151 POINT(40.3902043838137 73.22903911030903) bank14151 +14152 POINT(41.396615027533386 74.58952576724496) bank14152 +14153 POINT(40.38407548556556 73.5952584720507) bank14153 +14154 POINT(41.611201894412076 74.20045601993363) bank14154 +14155 POINT(41.133235699146745 74.25991843070818) bank14155 +14156 POINT(41.239476960618816 73.94480883118631) bank14156 +14157 POINT(41.36341623713199 74.34584344126233) bank14157 +14158 POINT(39.95186638391521 74.4619256655269) bank14158 +14159 POINT(40.618211885890005 74.45005427217966) bank14159 +14160 POINT(40.361702756419966 73.39065370844683) bank14160 +14161 POINT(40.11135039307344 74.13820874459091) bank14161 +14162 POINT(39.953367548342555 73.15730738821816) bank14162 +14163 POINT(40.0607121186461 74.74725889182653) bank14163 +14164 POINT(40.435016412021994 74.7478290946303) bank14164 +14165 POINT(41.36024932390569 74.84776782522275) bank14165 +14166 POINT(41.30847028268486 73.58541271162223) bank14166 +14167 POINT(40.12406621634516 74.7769737582331) bank14167 +14168 POINT(41.35076189314489 73.28770811990749) bank14168 +14169 POINT(41.2829515446072 74.92234459911265) bank14169 +14170 POINT(41.46630754373022 74.94948647471786) bank14170 +14171 POINT(40.11892030546031 73.84259053002751) bank14171 +14172 POINT(41.24981810805447 74.21634399373215) bank14172 +14173 POINT(40.41647603218057 73.51888695395378) bank14173 +14174 POINT(40.10930579324995 73.04610670260125) bank14174 +14175 POINT(40.295927253736544 74.55028637403063) bank14175 +14176 POINT(40.70583983555241 74.33931548760714) bank14176 +14177 POINT(41.045533286626714 73.39037385469311) bank14177 +14178 POINT(41.11016463146143 73.91474159834621) bank14178 +14179 POINT(40.66154248253415 74.98983473598054) bank14179 +14180 POINT(39.89218289677833 74.2163436221237) bank14180 +14181 POINT(40.354623160313224 73.88135781297395) bank14181 +14182 POINT(41.1394662596171 73.06660014325097) bank14182 +14183 POINT(40.586874535295344 74.73085726510239) bank14183 +14184 POINT(41.281597312594045 73.2893462798388) bank14184 +14185 POINT(40.21340718969868 74.21768785615183) bank14185 +14186 POINT(39.93442195521005 74.85429958436207) bank14186 +14187 POINT(40.81434438341773 74.39997423443535) bank14187 +14188 POINT(40.07147641766824 73.48991552083929) bank14188 +14189 POINT(41.208996695468265 74.60733304969663) bank14189 +14190 POINT(40.70829947520473 74.35058798765822) bank14190 +14191 POINT(40.05228892835423 74.60856447787106) bank14191 +14192 POINT(40.18848961577862 74.81495959133542) bank14192 +14193 POINT(39.831914275200475 73.36030053388808) bank14193 +14194 POINT(40.203260169349456 74.85275559635451) bank14194 +14195 POINT(41.53987111313199 73.65306544413458) bank14195 +14196 POINT(39.977225665500576 74.9112870230867) bank14196 +14197 POINT(40.22039801995864 74.2346772042836) bank14197 +14198 POINT(40.80479613616086 73.17407903527968) bank14198 +14199 POINT(40.76532317476124 74.96483553374692) bank14199 +14200 POINT(40.817781489307706 73.01095398132176) bank14200 +14201 POINT(41.11852334594403 73.91198694385508) bank14201 +14202 POINT(40.12344789397263 73.71509024100388) bank14202 +14203 POINT(40.64294537430565 73.74979928896337) bank14203 +14204 POINT(41.44123600162141 74.49920083955753) bank14204 +14205 POINT(41.165514230661444 74.60273775037854) bank14205 +14206 POINT(40.314621760312434 74.49981399253005) bank14206 +14207 POINT(40.51298247537708 73.11324096308064) bank14207 +14208 POINT(39.7843661199046 74.92369916929214) bank14208 +14209 POINT(40.000159329365495 74.01291917906916) bank14209 +14210 POINT(40.56580824977724 74.37057696510851) bank14210 +14211 POINT(40.15577956992572 74.12600969847001) bank14211 +14212 POINT(40.91946106312308 73.91541027396777) bank14212 +14213 POINT(40.08000351027608 74.26393266170605) bank14213 +14214 POINT(41.56210173631483 74.82718465887605) bank14214 +14215 POINT(39.99427141435952 73.2016924312692) bank14215 +14216 POINT(41.045613677170934 73.20169543484799) bank14216 +14217 POINT(41.54948937082614 74.06055927972173) bank14217 +14218 POINT(40.400308046157335 73.43386576062645) bank14218 +14219 POINT(39.764493818292074 74.01627740162978) bank14219 +14220 POINT(41.53189609735937 74.82433585049273) bank14220 +14221 POINT(41.52347444543739 74.23826126530173) bank14221 +14222 POINT(41.604981255544715 73.14431353789833) bank14222 +14223 POINT(41.572394597278084 73.64397142122063) bank14223 +14224 POINT(41.52129325446788 73.05965943650693) bank14224 +14225 POINT(40.46537633862594 74.63754205131337) bank14225 +14226 POINT(40.804900763430055 74.61107741172825) bank14226 +14227 POINT(40.51837021420864 74.1146572946318) bank14227 +14228 POINT(41.509570772857735 73.7749082445562) bank14228 +14229 POINT(40.34016115071633 74.492616235113) bank14229 +14230 POINT(41.287129300372655 75.00448465364822) bank14230 +14231 POINT(41.70575379424986 74.5062207186625) bank14231 +14232 POINT(40.37477388530397 73.65163747949539) bank14232 +14233 POINT(40.290090462626246 74.29080470088805) bank14233 +14234 POINT(39.73885660753933 73.02876700590409) bank14234 +14235 POINT(39.7878002741249 74.81963435259435) bank14235 +14236 POINT(40.38020031041477 74.97039694955538) bank14236 +14237 POINT(41.11712557952303 73.83958486730164) bank14237 +14238 POINT(41.455887642761574 74.03432724794303) bank14238 +14239 POINT(41.42271435586302 73.1344650066355) bank14239 +14240 POINT(41.62542300077431 74.58056160098661) bank14240 +14241 POINT(40.67026842345279 73.72996735085867) bank14241 +14242 POINT(40.36859712557162 73.61107552863176) bank14242 +14243 POINT(41.02930015064358 73.68824066182576) bank14243 +14244 POINT(40.284589708296686 73.99168383549332) bank14244 +14245 POINT(40.01328081285136 74.78596835650504) bank14245 +14246 POINT(41.39970291248101 73.33645881813364) bank14246 +14247 POINT(40.201141060145794 74.12495619155507) bank14247 +14248 POINT(41.22453954432476 74.63651540447697) bank14248 +14249 POINT(41.113306176365676 73.97504024745183) bank14249 +14250 POINT(41.6819166937434 74.01483274609761) bank14250 +14251 POINT(40.16808602054544 74.35236440804339) bank14251 +14252 POINT(41.567936093405784 74.66777561959105) bank14252 +14253 POINT(40.17306677705433 75.00069151332524) bank14253 +14254 POINT(40.404705911291096 73.13752807547633) bank14254 +14255 POINT(40.74009036179825 74.08594573857103) bank14255 +14256 POINT(39.98842539556794 74.11226816624003) bank14256 +14257 POINT(40.539807084790624 73.12943300240198) bank14257 +14258 POINT(40.84064703001379 74.10457728227581) bank14258 +14259 POINT(40.941650602353576 74.9985426732227) bank14259 +14260 POINT(41.41877371360941 74.95311022061054) bank14260 +14261 POINT(41.43650408405 74.94910624689152) bank14261 +14262 POINT(40.86414748144578 74.40942716257915) bank14262 +14263 POINT(41.18857963288859 73.33138003401689) bank14263 +14264 POINT(41.70431814302049 73.21318766087165) bank14264 +14265 POINT(40.90271050721575 74.29668662868139) bank14265 +14266 POINT(40.32219534087573 74.11472133539742) bank14266 +14267 POINT(39.95434552450988 73.38463476059017) bank14267 +14268 POINT(39.96990372089724 74.45587092168044) bank14268 +14269 POINT(41.063665045877734 74.95031063764728) bank14269 +14270 POINT(41.43146622945563 74.43386090390858) bank14270 +14271 POINT(39.99310225755661 73.47650323692531) bank14271 +14272 POINT(41.06793374592187 74.13281305976757) bank14272 +14273 POINT(41.33478958337642 74.82962221119737) bank14273 +14274 POINT(41.007135508807714 74.10471125698037) bank14274 +14275 POINT(40.7518934670189 73.91552371048442) bank14275 +14276 POINT(40.95506778133539 73.7269843340941) bank14276 +14277 POINT(40.165885281925576 74.46486958355688) bank14277 +14278 POINT(40.879081737707295 74.18374208635085) bank14278 +14279 POINT(41.200238596428335 74.53448885982773) bank14279 +14280 POINT(39.907717245637286 74.81259082132783) bank14280 +14281 POINT(40.376754980482225 74.74368157024905) bank14281 +14282 POINT(41.28165276995485 73.09110814854573) bank14282 +14283 POINT(41.09375047120247 73.22159997612879) bank14283 +14284 POINT(41.682776275572934 73.03642285811142) bank14284 +14285 POINT(40.785616176260035 73.74996974670039) bank14285 +14286 POINT(41.13846297416298 74.90062025953802) bank14286 +14287 POINT(40.60743222154729 74.38810106760735) bank14287 +14288 POINT(40.18418179277052 74.47982153711965) bank14288 +14289 POINT(40.16376078410199 74.36065358227476) bank14289 +14290 POINT(40.15995615852797 73.90525077067801) bank14290 +14291 POINT(39.910561379703886 74.7384877911094) bank14291 +14292 POINT(40.85296906810613 73.07564333971722) bank14292 +14293 POINT(40.955573000437546 73.9873459765021) bank14293 +14294 POINT(39.781605631815616 73.30131932202602) bank14294 +14295 POINT(40.237432685637174 73.6973806612714) bank14295 +14296 POINT(39.812212273351534 73.1411345520006) bank14296 +14297 POINT(40.129960954788785 74.08613723904242) bank14297 +14298 POINT(40.95488494415483 74.9534063875398) bank14298 +14299 POINT(40.79062612263586 73.66013686060415) bank14299 +14300 POINT(41.18644003744888 73.34148182374521) bank14300 +14301 POINT(40.06377928883403 73.4077667474024) bank14301 +14302 POINT(40.575647211087286 73.26124811352564) bank14302 +14303 POINT(39.96556880656412 74.53446598567491) bank14303 +14304 POINT(41.14695737227475 74.20358896602221) bank14304 +14305 POINT(39.92716770963986 73.76409465877674) bank14305 +14306 POINT(41.07803637124188 73.74684525690836) bank14306 +14307 POINT(41.19085403688045 73.55460816101159) bank14307 +14308 POINT(40.886198018825496 74.65513528089247) bank14308 +14309 POINT(41.40343199863027 73.33665614159791) bank14309 +14310 POINT(41.234400244805606 74.42902913469287) bank14310 +14311 POINT(40.27431413282073 74.3829443591642) bank14311 +14312 POINT(40.23119716342747 73.11610283206541) bank14312 +14313 POINT(40.60158716428841 73.57794007371729) bank14313 +14314 POINT(41.494614727530006 74.18930034562622) bank14314 +14315 POINT(39.79552018254755 74.01540978537633) bank14315 +14316 POINT(40.885127363505234 74.97933448430005) bank14316 +14317 POINT(41.32398019678236 73.984291091276) bank14317 +14318 POINT(40.937673513768324 73.7918135657773) bank14318 +14319 POINT(40.814239073263295 74.43735708830319) bank14319 +14320 POINT(40.41099462160917 74.36821842046807) bank14320 +14321 POINT(40.47136310082567 73.45045349993387) bank14321 +14322 POINT(40.19376541129233 73.26829244469262) bank14322 +14323 POINT(41.646843811398135 74.3624949263535) bank14323 +14324 POINT(41.52040046266523 74.40345989207974) bank14324 +14325 POINT(41.64038889375133 73.14783890378449) bank14325 +14326 POINT(40.303343071587236 74.85122723606308) bank14326 +14327 POINT(41.24521763601144 73.35067934602999) bank14327 +14328 POINT(40.376178761326486 73.07619681698984) bank14328 +14329 POINT(40.78483668942835 73.736613536061) bank14329 +14330 POINT(40.89723297379207 74.11025842466485) bank14330 +14331 POINT(40.37318291675406 74.8135034974158) bank14331 +14332 POINT(40.59641954119378 74.69087771300548) bank14332 +14333 POINT(40.303788092690894 74.89424577263293) bank14333 +14334 POINT(41.47043110289519 74.40433715095632) bank14334 +14335 POINT(41.16957556181011 74.64409519954111) bank14335 +14336 POINT(40.89285112904194 74.77102874982347) bank14336 +14337 POINT(39.84404043354558 73.00806539269264) bank14337 +14338 POINT(41.63348345319031 73.55341336122163) bank14338 +14339 POINT(40.02924050485945 74.60097105073325) bank14339 +14340 POINT(41.310769708451964 73.67068217874844) bank14340 +14341 POINT(40.181142257149865 74.792331865783) bank14341 +14342 POINT(41.69830207812793 74.44789123546606) bank14342 +14343 POINT(39.75882235571119 73.15844448706132) bank14343 +14344 POINT(40.72774352912416 73.48381897273934) bank14344 +14345 POINT(40.589197101516646 73.31912127349729) bank14345 +14346 POINT(40.01203157417513 74.88701109877204) bank14346 +14347 POINT(41.00677399751175 73.92234193623676) bank14347 +14348 POINT(41.09461449267507 74.06617421178304) bank14348 +14349 POINT(39.80141090398486 73.77930807982257) bank14349 +14350 POINT(41.341011512305094 74.68078546709698) bank14350 +14351 POINT(40.37723210502864 73.16150546980403) bank14351 +14352 POINT(40.420689042627295 73.72321847692241) bank14352 +14353 POINT(40.252998928571785 74.93572127701444) bank14353 +14354 POINT(39.798161354939566 74.19424181439496) bank14354 +14355 POINT(40.29242504354883 74.12537633667178) bank14355 +14356 POINT(40.720239179096865 74.23076424783338) bank14356 +14357 POINT(41.59785014647599 74.2010283671691) bank14357 +14358 POINT(41.256772779107145 73.42557943428888) bank14358 +14359 POINT(40.85663806728056 73.62995984023868) bank14359 +14360 POINT(39.90826899657437 73.19428387222719) bank14360 +14361 POINT(39.985231224313395 73.72665989804814) bank14361 +14362 POINT(41.47465843887989 73.22614487945334) bank14362 +14363 POINT(40.008287679717455 73.82825894861799) bank14363 +14364 POINT(41.43846107412949 74.80201005639265) bank14364 +14365 POINT(40.48680265932539 73.69859102911843) bank14365 +14366 POINT(39.871042766115586 74.42800850550454) bank14366 +14367 POINT(40.148599579613176 73.13073553054255) bank14367 +14368 POINT(40.658452253268294 74.27710405829467) bank14368 +14369 POINT(41.64945967554301 74.25227277467148) bank14369 +14370 POINT(40.758205130894304 74.22400237266075) bank14370 +14371 POINT(40.788423503532414 73.63152680638356) bank14371 +14372 POINT(40.6515442736643 74.06495290301687) bank14372 +14373 POINT(41.69999162570717 73.79551236026074) bank14373 +14374 POINT(41.65789893916007 74.46012735388487) bank14374 +14375 POINT(41.709985446401475 73.58290052077623) bank14375 +14376 POINT(39.9794223147541 73.53860862469533) bank14376 +14377 POINT(41.66164778401 73.94688025798416) bank14377 +14378 POINT(40.29388864598804 74.8101794375564) bank14378 +14379 POINT(40.150081053329494 74.29953495743098) bank14379 +14380 POINT(41.61296592460402 74.7183617884554) bank14380 +14381 POINT(40.94134282707244 73.43577654845556) bank14381 +14382 POINT(40.721703807905946 73.2661789342541) bank14382 +14383 POINT(40.8154937004273 74.33933811595112) bank14383 +14384 POINT(40.261847591559636 73.11793219369758) bank14384 +14385 POINT(40.610475983361034 73.66681624129195) bank14385 +14386 POINT(40.03784167972667 74.50090908742708) bank14386 +14387 POINT(40.9584367179333 73.50107458187469) bank14387 +14388 POINT(40.72779121836176 74.05275600782923) bank14388 +14389 POINT(39.94921333731818 74.43253811013923) bank14389 +14390 POINT(40.13090603257739 74.11542101975651) bank14390 +14391 POINT(40.5630590698194 73.24811167492852) bank14391 +14392 POINT(39.87262434146991 73.11489075414562) bank14392 +14393 POINT(40.35463077641067 74.98078761582182) bank14393 +14394 POINT(40.89647250414407 74.72543609294353) bank14394 +14395 POINT(40.341556721411806 75.00393341502503) bank14395 +14396 POINT(41.27556367748716 74.60237112312477) bank14396 +14397 POINT(40.34407012867757 73.18289874339732) bank14397 +14398 POINT(41.10585919304479 74.23357827233619) bank14398 +14399 POINT(41.474991967902284 74.09925184686357) bank14399 +14400 POINT(41.69184368250327 74.1512155333578) bank14400 +14401 POINT(41.44115953290747 73.1488304808171) bank14401 +14402 POINT(39.728062226796226 74.50385652562541) bank14402 +14403 POINT(40.32520684999972 74.10232874250028) bank14403 +14404 POINT(40.79173580864975 74.12134183374536) bank14404 +14405 POINT(39.78612951903994 73.46979283247003) bank14405 +14406 POINT(40.50667484784192 74.65656440376199) bank14406 +14407 POINT(40.842216068375706 73.47264912644552) bank14407 +14408 POINT(40.44012018120321 73.36659635430375) bank14408 +14409 POINT(40.3877307349423 73.55570399561913) bank14409 +14410 POINT(40.16424063464874 74.30227111411216) bank14410 +14411 POINT(40.01549012386821 74.69422912189977) bank14411 +14412 POINT(39.73198046029036 74.50395820973411) bank14412 +14413 POINT(39.97527118926911 73.96614135509132) bank14413 +14414 POINT(40.89282079051436 74.58030735824718) bank14414 +14415 POINT(39.83008669535066 74.50019097659532) bank14415 +14416 POINT(41.1066296083041 74.33640325041993) bank14416 +14417 POINT(41.699760246435154 74.06958010055129) bank14417 +14418 POINT(39.87943758281986 74.46314145087368) bank14418 +14419 POINT(40.00247190074465 73.81914020094814) bank14419 +14420 POINT(41.62922824961002 74.8574217352999) bank14420 +14421 POINT(41.688024229025736 73.13307201901831) bank14421 +14422 POINT(41.31556184232776 73.13527660665181) bank14422 +14423 POINT(41.26199252047031 74.51256723343238) bank14423 +14424 POINT(40.70735930088899 74.14621579830879) bank14424 +14425 POINT(41.1387427143362 73.25862221178643) bank14425 +14426 POINT(41.64999303539868 73.44945739082222) bank14426 +14427 POINT(40.38436553057067 73.80892777983057) bank14427 +14428 POINT(41.32226614177704 74.22975805867547) bank14428 +14429 POINT(40.626230511689805 73.1486239276445) bank14429 +14430 POINT(40.395258325315 74.60924512278602) bank14430 +14431 POINT(40.24728192949493 73.96551523846625) bank14431 +14432 POINT(41.490528914865486 74.54945871427013) bank14432 +14433 POINT(41.51632719633656 73.95946162885764) bank14433 +14434 POINT(41.43107111557916 74.5373410391191) bank14434 +14435 POINT(39.98886382352568 73.6631816451186) bank14435 +14436 POINT(40.74689445198144 74.9945635220048) bank14436 +14437 POINT(40.93745130680998 74.6490423775036) bank14437 +14438 POINT(39.79307908906334 73.26638921829091) bank14438 +14439 POINT(40.03077900165693 74.73543814835858) bank14439 +14440 POINT(41.30242867646874 73.31124143779981) bank14440 +14441 POINT(39.9650645320615 74.62944599100474) bank14441 +14442 POINT(41.544227239050315 73.29473652684842) bank14442 +14443 POINT(40.63644923722964 74.71248083317812) bank14443 +14444 POINT(41.029306249496194 73.64948756057301) bank14444 +14445 POINT(41.426102775381736 74.78282428948957) bank14445 +14446 POINT(41.10777279898562 73.2328388871181) bank14446 +14447 POINT(39.90622498855439 73.13333685034246) bank14447 +14448 POINT(40.4835457229113 73.22017274821661) bank14448 +14449 POINT(40.63469559378179 74.18202492609178) bank14449 +14450 POINT(40.07304397373876 73.45341086143986) bank14450 +14451 POINT(41.69608081123339 74.82006774907336) bank14451 +14452 POINT(41.12159256558548 73.59144394568791) bank14452 +14453 POINT(41.19988316877546 74.14087634857532) bank14453 +14454 POINT(40.958212153807864 73.11085650920913) bank14454 +14455 POINT(40.238684575647696 73.85688184424903) bank14455 +14456 POINT(41.29798941079515 74.86304679092846) bank14456 +14457 POINT(39.76076404747678 73.37003100570075) bank14457 +14458 POINT(40.986099629577986 74.86883535553261) bank14458 +14459 POINT(40.53165479913403 73.13098503932552) bank14459 +14460 POINT(40.943450832970264 74.60848896092011) bank14460 +14461 POINT(40.42849119566489 73.69613678138745) bank14461 +14462 POINT(40.50640044713007 74.17813899572965) bank14462 +14463 POINT(41.11792101407412 74.10814889386023) bank14463 +14464 POINT(39.90340643781221 74.2906119700753) bank14464 +14465 POINT(41.60186980678218 73.14625616526313) bank14465 +14466 POINT(40.50453916619868 73.5734846064401) bank14466 +14467 POINT(40.32716146560552 73.68748973169107) bank14467 +14468 POINT(41.191482455714585 73.33371730666737) bank14468 +14469 POINT(40.01791184211019 74.78424863672983) bank14469 +14470 POINT(39.74018492377678 74.20650671176021) bank14470 +14471 POINT(40.61693365503096 73.34154381891523) bank14471 +14472 POINT(40.336543002114574 74.39746305231395) bank14472 +14473 POINT(40.76302512369863 74.2771234952972) bank14473 +14474 POINT(41.45681939144549 74.82879004365545) bank14474 +14475 POINT(40.7128338865305 73.23933914837669) bank14475 +14476 POINT(41.696390253878725 74.5467892950245) bank14476 +14477 POINT(40.358813597982966 74.6813479900421) bank14477 +14478 POINT(39.74641031300876 74.97342016654324) bank14478 +14479 POINT(41.3475794706468 73.7700963909036) bank14479 +14480 POINT(41.6838724105774 73.60552405472387) bank14480 +14481 POINT(40.93966026287952 73.09559334870724) bank14481 +14482 POINT(39.97400647628915 74.95171301018581) bank14482 +14483 POINT(40.336668723512226 74.69180893016056) bank14483 +14484 POINT(39.80755867003651 73.67674370526515) bank14484 +14485 POINT(39.89248128598178 74.3262932279605) bank14485 +14486 POINT(41.685100910849684 73.24016323107593) bank14486 +14487 POINT(39.91943938288102 73.63564881706458) bank14487 +14488 POINT(40.931353553655526 73.25383133205926) bank14488 +14489 POINT(41.47208683859331 74.18531980275851) bank14489 +14490 POINT(40.682630920568876 73.50779038086908) bank14490 +14491 POINT(39.88595278032184 74.76829662514052) bank14491 +14492 POINT(41.25488782090802 73.45058145547897) bank14492 +14493 POINT(40.34215206079662 73.23642187284972) bank14493 +14494 POINT(41.26243710950929 75.00370586857798) bank14494 +14495 POINT(40.153030716815394 74.15679558451151) bank14495 +14496 POINT(41.38534127115951 73.24766244826783) bank14496 +14497 POINT(40.567661430420365 74.70509242781438) bank14497 +14498 POINT(41.25059561138841 73.7157569906325) bank14498 +14499 POINT(41.49025032671839 74.07964149736037) bank14499 +14500 POINT(41.13910141394325 73.27150192670305) bank14500 +14501 POINT(40.890564201181256 74.10804413772576) bank14501 +14502 POINT(40.68148878254046 74.58561794296594) bank14502 +14503 POINT(41.07616193266051 74.42560269869885) bank14503 +14504 POINT(41.024437073112374 73.28145999491414) bank14504 +14505 POINT(40.17413406910282 73.9998516548416) bank14505 +14506 POINT(41.24264639105803 73.20663948411732) bank14506 +14507 POINT(41.252489131706795 74.21920105390957) bank14507 +14508 POINT(40.13836358185392 73.02450023755561) bank14508 +14509 POINT(40.83155171329496 74.6599716365912) bank14509 +14510 POINT(41.533686244613705 73.71790307224485) bank14510 +14511 POINT(41.18156245797062 74.9893324997631) bank14511 +14512 POINT(40.66215563610146 73.9424943526584) bank14512 +14513 POINT(40.58991507971704 74.9080101589573) bank14513 +14514 POINT(41.62381636208642 74.3763973526557) bank14514 +14515 POINT(40.65086852671832 73.71338449698393) bank14515 +14516 POINT(40.172071434267345 73.83765370459761) bank14516 +14517 POINT(40.68716121242476 74.79644546824503) bank14517 +14518 POINT(40.96519567293724 74.1399463667704) bank14518 +14519 POINT(41.42738338200223 73.0543530193693) bank14519 +14520 POINT(40.77407813540541 74.3082223608926) bank14520 +14521 POINT(40.14976842433594 73.63004696178079) bank14521 +14522 POINT(40.48729190773059 74.80732062763151) bank14522 +14523 POINT(41.285425430720835 73.0095396670251) bank14523 +14524 POINT(41.58934192863658 73.25467113850789) bank14524 +14525 POINT(40.66116115882789 73.5394905719606) bank14525 +14526 POINT(40.86404898263968 73.15835028012512) bank14526 +14527 POINT(39.81472618143538 73.0368051805854) bank14527 +14528 POINT(40.97703022030404 73.50177086433398) bank14528 +14529 POINT(41.39237533800538 74.38905159224444) bank14529 +14530 POINT(41.2649687641725 74.12563725500573) bank14530 +14531 POINT(40.016219985306364 73.67405055276647) bank14531 +14532 POINT(39.96399666727706 74.03348132543537) bank14532 +14533 POINT(40.58750459279521 74.26856426671426) bank14533 +14534 POINT(41.21703979575246 74.74337965474106) bank14534 +14535 POINT(39.794736884647826 73.00932072847583) bank14535 +14536 POINT(41.095630219977636 73.9374754180968) bank14536 +14537 POINT(41.66137624800375 73.07252210462316) bank14537 +14538 POINT(39.820380872421175 74.19011873502048) bank14538 +14539 POINT(41.50278642824823 73.19091893784143) bank14539 +14540 POINT(40.77799237450099 73.2948659135651) bank14540 +14541 POINT(40.6020294675733 73.11097727309554) bank14541 +14542 POINT(40.90082809065771 74.41511595031214) bank14542 +14543 POINT(41.21316264596132 73.75315820819306) bank14543 +14544 POINT(41.23942646194527 73.63125207960545) bank14544 +14545 POINT(39.96117172496371 74.20706384523646) bank14545 +14546 POINT(40.71833184642615 74.7160930040766) bank14546 +14547 POINT(40.274537355848324 73.8415381280536) bank14547 +14548 POINT(41.247894502052716 73.06578978278672) bank14548 +14549 POINT(40.28469178015396 75.00244996964608) bank14549 +14550 POINT(40.40928061227015 73.07455492986655) bank14550 +14551 POINT(41.66751331316247 73.04644217784274) bank14551 +14552 POINT(40.26923998315796 74.50762365640107) bank14552 +14553 POINT(40.45066115426177 74.0356548490365) bank14553 +14554 POINT(40.45547203756676 73.20644362921998) bank14554 +14555 POINT(40.82156184755378 74.20133955612886) bank14555 +14556 POINT(39.937948942701716 74.92249066213564) bank14556 +14557 POINT(41.517499038700166 74.68324902804618) bank14557 +14558 POINT(41.38678805273684 74.84157868057466) bank14558 +14559 POINT(41.26306340433115 73.1999610541091) bank14559 +14560 POINT(41.664218946674815 73.12667668518536) bank14560 +14561 POINT(40.523116164286726 74.56852061653494) bank14561 +14562 POINT(41.21564385872697 73.59631252369967) bank14562 +14563 POINT(40.5171940329806 74.95918210231429) bank14563 +14564 POINT(40.29612334965989 73.72279866510112) bank14564 +14565 POINT(41.29429865455462 73.76936447470337) bank14565 +14566 POINT(40.12255760300623 73.44892040031954) bank14566 +14567 POINT(41.15800801429092 74.32698448922002) bank14567 +14568 POINT(40.30393830962015 74.73611263931912) bank14568 +14569 POINT(40.043314039838975 74.85505902800638) bank14569 +14570 POINT(39.899624584352246 73.86794008783107) bank14570 +14571 POINT(40.1355294837477 74.50310052558292) bank14571 +14572 POINT(39.76419791016142 73.81467689539619) bank14572 +14573 POINT(40.47634789105921 74.43037670396407) bank14573 +14574 POINT(41.60501784751445 74.64032711343849) bank14574 +14575 POINT(40.70717486667449 73.67397566761638) bank14575 +14576 POINT(40.64562625391085 74.54697587560565) bank14576 +14577 POINT(40.03058224487164 73.58753278790007) bank14577 +14578 POINT(41.38886373619123 74.8701568026155) bank14578 +14579 POINT(40.46903778878148 73.54746097365843) bank14579 +14580 POINT(41.52243759418222 74.98049595404116) bank14580 +14581 POINT(40.62924567732595 73.17957147660563) bank14581 +14582 POINT(41.093631789627985 73.18269959243764) bank14582 +14583 POINT(40.84503088301682 73.17646653664893) bank14583 +14584 POINT(39.80599350796069 73.23265230275867) bank14584 +14585 POINT(41.25352320369703 74.47912245634586) bank14585 +14586 POINT(41.45444771432695 74.87772950974296) bank14586 +14587 POINT(40.711634357381065 74.77089773160964) bank14587 +14588 POINT(39.79256607498631 74.61065647230183) bank14588 +14589 POINT(40.488163010708 73.87139825868783) bank14589 +14590 POINT(40.61894985337669 73.31369454340754) bank14590 +14591 POINT(40.61300219916192 74.52187497617147) bank14591 +14592 POINT(40.988120672002324 73.6199460818299) bank14592 +14593 POINT(39.953848374672596 73.06225049069768) bank14593 +14594 POINT(41.2347691883327 74.45616146106339) bank14594 +14595 POINT(39.95660483752307 73.28138034106186) bank14595 +14596 POINT(40.39225893610704 74.95277794427798) bank14596 +14597 POINT(40.35563593526614 74.67321760651984) bank14597 +14598 POINT(40.64687053073446 74.95677256197547) bank14598 +14599 POINT(40.54037792526036 74.7838354821877) bank14599 +14600 POINT(40.413241483058755 74.32475894173962) bank14600 +14601 POINT(40.504206192524116 74.66375545334625) bank14601 +14602 POINT(41.51901090416631 74.71149785162773) bank14602 +14603 POINT(41.45218561790098 73.85584407920351) bank14603 +14604 POINT(41.32142768356864 74.32497413888368) bank14604 +14605 POINT(41.42642664040652 74.00072644392684) bank14605 +14606 POINT(39.77377647720095 74.16341695533703) bank14606 +14607 POINT(41.30119428796033 73.51434804215471) bank14607 +14608 POINT(40.74711300876825 74.00429162260991) bank14608 +14609 POINT(41.40121261860799 74.26729110607197) bank14609 +14610 POINT(39.793638507002456 74.9825997873423) bank14610 +14611 POINT(40.214959335867775 74.31372309245513) bank14611 +14612 POINT(41.45925763584863 73.6039373481654) bank14612 +14613 POINT(40.43899295868891 73.80107495779183) bank14613 +14614 POINT(40.56498053452748 73.76058277743465) bank14614 +14615 POINT(40.2841306453953 74.55357806801919) bank14615 +14616 POINT(41.60148904873299 73.44480711145039) bank14616 +14617 POINT(40.169592642273905 74.04696189991313) bank14617 +14618 POINT(41.029914798595335 73.87989619212445) bank14618 +14619 POINT(41.02980955657293 74.83036314231504) bank14619 +14620 POINT(40.90729916654561 74.7643362996716) bank14620 +14621 POINT(40.82504612859579 73.54879167554508) bank14621 +14622 POINT(41.18107393133295 73.12857001269478) bank14622 +14623 POINT(39.89554839464503 74.45508593445237) bank14623 +14624 POINT(41.4431127919885 73.45785542047273) bank14624 +14625 POINT(41.51105775501546 73.1925713887633) bank14625 +14626 POINT(39.81873720397283 74.85847213558193) bank14626 +14627 POINT(40.70906851138281 74.84817377225502) bank14627 +14628 POINT(41.2616345489013 74.97145521755242) bank14628 +14629 POINT(40.66586792037826 74.95720634320779) bank14629 +14630 POINT(40.901344564364514 74.42996716487778) bank14630 +14631 POINT(40.42188369937276 74.16927047745213) bank14631 +14632 POINT(40.53222272225377 73.39634037787282) bank14632 +14633 POINT(39.96277837301301 74.84945685655046) bank14633 +14634 POINT(40.05979101043395 74.81128288782345) bank14634 +14635 POINT(40.087951787277426 73.87827096246208) bank14635 +14636 POINT(40.228775813998595 74.44607143557157) bank14636 +14637 POINT(41.322286903493016 74.09984614830823) bank14637 +14638 POINT(41.0872836169993 73.92786393073827) bank14638 +14639 POINT(40.68071721385238 74.74061266775377) bank14639 +14640 POINT(40.88601784222779 73.88370762004338) bank14640 +14641 POINT(41.56854802075197 74.8303055647769) bank14641 +14642 POINT(41.31587652458799 73.40396330377226) bank14642 +14643 POINT(40.96737042751294 73.21143841365426) bank14643 +14644 POINT(40.93010148139285 74.95463436930324) bank14644 +14645 POINT(40.27513957043006 73.77602457236783) bank14645 +14646 POINT(40.10612669808957 74.55669993813484) bank14646 +14647 POINT(40.70713493570508 73.65879192230534) bank14647 +14648 POINT(41.24910606852098 73.75699581083772) bank14648 +14649 POINT(41.171624273114865 73.74937847744152) bank14649 +14650 POINT(40.40020524279709 73.61224879300357) bank14650 +14651 POINT(40.41248775689567 73.11662355750845) bank14651 +14652 POINT(40.371859188136746 73.12115334540006) bank14652 +14653 POINT(39.938425516394716 74.18928997860134) bank14653 +14654 POINT(40.62241423231867 74.51077450627406) bank14654 +14655 POINT(40.22431998111005 74.01995268578283) bank14655 +14656 POINT(40.546679098527754 74.38966063885746) bank14656 +14657 POINT(40.134053456393836 73.47392777650856) bank14657 +14658 POINT(40.05995803357709 73.59767070864417) bank14658 +14659 POINT(39.730887946580786 73.15053518808408) bank14659 +14660 POINT(40.443698010047726 73.36514450546827) bank14660 +14661 POINT(40.791044562173425 73.81137493230837) bank14661 +14662 POINT(41.36141505844256 74.11293627359412) bank14662 +14663 POINT(40.97243039446953 73.15104537519733) bank14663 +14664 POINT(40.89695798146608 74.83579956398341) bank14664 +14665 POINT(41.334908602592606 73.81162393966869) bank14665 +14666 POINT(40.79000939976952 74.8545745855236) bank14666 +14667 POINT(40.83770180122802 74.41419462914605) bank14667 +14668 POINT(40.01232919947374 73.17042348362861) bank14668 +14669 POINT(40.91398327463341 74.19594830353546) bank14669 +14670 POINT(40.42089277353743 73.06925805230372) bank14670 +14671 POINT(41.67982343682871 74.0592050884453) bank14671 +14672 POINT(40.48548868524715 74.37035819100778) bank14672 +14673 POINT(40.98751801044782 74.81559993594874) bank14673 +14674 POINT(39.97968375251293 73.59774502830977) bank14674 +14675 POINT(40.27273543798363 74.24830095154446) bank14675 +14676 POINT(41.09296733018538 74.32333492450249) bank14676 +14677 POINT(40.9618730497181 74.54175619323284) bank14677 +14678 POINT(41.388968652808046 73.04431898168838) bank14678 +14679 POINT(40.87945245079284 74.3958486127823) bank14679 +14680 POINT(40.6923507443537 74.16095384290713) bank14680 +14681 POINT(39.870067628361994 73.1246071086251) bank14681 +14682 POINT(40.75498272730379 74.22280388788829) bank14682 +14683 POINT(41.600172293930996 73.24836293303008) bank14683 +14684 POINT(40.96891221280152 73.27859780752298) bank14684 +14685 POINT(39.95590762018178 74.83861113492209) bank14685 +14686 POINT(40.871925567355746 73.0510142456359) bank14686 +14687 POINT(40.80275163487032 74.55714896889518) bank14687 +14688 POINT(40.53436204846372 73.49406847102338) bank14688 +14689 POINT(41.58315022694355 73.58080119417475) bank14689 +14690 POINT(40.73567937578767 74.98045996461087) bank14690 +14691 POINT(40.10775980630497 73.54304117237696) bank14691 +14692 POINT(40.19311248784575 73.12210685638425) bank14692 +14693 POINT(41.11298520394031 73.28955924378387) bank14693 +14694 POINT(39.989976949838685 74.19081835502477) bank14694 +14695 POINT(41.407998452683714 74.96829742461611) bank14695 +14696 POINT(40.24082754027001 73.24648640122548) bank14696 +14697 POINT(40.31472645018322 74.90449404656977) bank14697 +14698 POINT(39.95177237745676 74.84622219492242) bank14698 +14699 POINT(40.94005511984022 74.91217677238878) bank14699 +14700 POINT(41.152102422626776 73.59832296314036) bank14700 +14701 POINT(41.52199857450125 74.04956914141594) bank14701 +14702 POINT(41.15158201886622 73.14988390929396) bank14702 +14703 POINT(39.741117181868866 73.42465193115) bank14703 +14704 POINT(41.367562085569936 73.83231351253937) bank14704 +14705 POINT(40.80173074617021 74.11280443187397) bank14705 +14706 POINT(40.20979478061525 74.9784978731773) bank14706 +14707 POINT(39.813896650535966 74.45211389555718) bank14707 +14708 POINT(40.485495855642625 73.84388723007504) bank14708 +14709 POINT(41.67024715817149 75.00107246888793) bank14709 +14710 POINT(40.5136615787343 74.65006548139543) bank14710 +14711 POINT(40.91383830909944 74.87160019157973) bank14711 +14712 POINT(41.65688219209786 74.87753819343011) bank14712 +14713 POINT(41.08305525855265 74.39450039673122) bank14713 +14714 POINT(41.03379683158798 73.10165946682596) bank14714 +14715 POINT(39.76424878569462 74.22881065606772) bank14715 +14716 POINT(41.44157997944784 74.14945076447727) bank14716 +14717 POINT(41.29956734067122 73.14594048870435) bank14717 +14718 POINT(40.8808383938778 73.86899909812934) bank14718 +14719 POINT(40.48494799288308 73.13202754855372) bank14719 +14720 POINT(40.110919597626705 74.76680419793631) bank14720 +14721 POINT(40.184898236952726 73.6206550330514) bank14721 +14722 POINT(39.92166257988169 74.30869654503154) bank14722 +14723 POINT(40.755367358923024 74.6917130619991) bank14723 +14724 POINT(39.76764832314823 74.07635233398754) bank14724 +14725 POINT(40.7792840232905 73.34717032455924) bank14725 +14726 POINT(40.343507952960444 74.72557724055977) bank14726 +14727 POINT(40.82751609904351 73.9731330175631) bank14727 +14728 POINT(41.11929234596749 73.56587390133237) bank14728 +14729 POINT(40.037025589715824 74.71455262435668) bank14729 +14730 POINT(39.91768570136363 74.8933128984823) bank14730 +14731 POINT(40.31198835002005 74.35492591816438) bank14731 +14732 POINT(40.090394016351034 74.4661918155988) bank14732 +14733 POINT(40.17715488630989 74.10770165916497) bank14733 +14734 POINT(40.27109973758348 73.85912853917692) bank14734 +14735 POINT(41.11377229781279 73.5469376857694) bank14735 +14736 POINT(41.544758426003234 73.85496922938805) bank14736 +14737 POINT(41.51059161951216 74.79459938057484) bank14737 +14738 POINT(41.35035103172557 73.13595938681704) bank14738 +14739 POINT(40.544148242556204 74.46213190035783) bank14739 +14740 POINT(40.9381657400256 73.61795317847796) bank14740 +14741 POINT(40.75739265082694 73.25379433445609) bank14741 +14742 POINT(40.06429256335391 74.09540681578339) bank14742 +14743 POINT(41.22766789731295 73.74130473170254) bank14743 +14744 POINT(39.96389099314249 73.22851539021805) bank14744 +14745 POINT(40.67101963828798 74.80153679100022) bank14745 +14746 POINT(41.23809015883054 74.26344609726665) bank14746 +14747 POINT(41.0368456344333 73.4966664507004) bank14747 +14748 POINT(41.36264917049691 74.48228152248902) bank14748 +14749 POINT(41.42398147810467 74.1793316763549) bank14749 +14750 POINT(41.328175323085624 73.0731528899647) bank14750 +14751 POINT(41.13525922751711 74.33322200841484) bank14751 +14752 POINT(41.30047279017258 73.02074616091413) bank14752 +14753 POINT(39.81679392965097 74.55675404393851) bank14753 +14754 POINT(40.552176477358366 74.00359951705444) bank14754 +14755 POINT(41.26828024288448 73.54850168272326) bank14755 +14756 POINT(39.72903825011308 74.11261836460254) bank14756 +14757 POINT(40.78879897385555 73.68715864846166) bank14757 +14758 POINT(40.73011936767981 73.06607905160831) bank14758 +14759 POINT(40.611232042830665 74.21939455174535) bank14759 +14760 POINT(41.30391314146089 74.98627115696613) bank14760 +14761 POINT(40.29441539276093 74.50225821588144) bank14761 +14762 POINT(41.03519337663642 74.51769061699564) bank14762 +14763 POINT(40.03965265154645 74.65488261363812) bank14763 +14764 POINT(40.19716622016591 73.40964634121605) bank14764 +14765 POINT(40.99880972352774 73.65215488489586) bank14765 +14766 POINT(41.220812723948406 74.93642697340768) bank14766 +14767 POINT(40.83076400753601 73.76871027442753) bank14767 +14768 POINT(40.36932357210954 74.14645629256226) bank14768 +14769 POINT(40.95929384455808 74.04304015278578) bank14769 +14770 POINT(41.430475388950725 74.8517268220911) bank14770 +14771 POINT(41.18787027070496 73.3724277914116) bank14771 +14772 POINT(40.884984959655725 74.40722025489417) bank14772 +14773 POINT(41.15615486858329 74.19906555501797) bank14773 +14774 POINT(39.88492010266622 74.54847872538706) bank14774 +14775 POINT(39.76803686069454 73.97367482998015) bank14775 +14776 POINT(40.90633721788692 73.26997381347765) bank14776 +14777 POINT(41.47250797842713 73.05389437420014) bank14777 +14778 POINT(40.87798527435053 73.81975757980885) bank14778 +14779 POINT(40.639312921217254 74.36981920725478) bank14779 +14780 POINT(41.393112685019126 73.8255656346149) bank14780 +14781 POINT(41.33084521700124 74.88548886266574) bank14781 +14782 POINT(41.491186593212 74.45573630251332) bank14782 +14783 POINT(41.41344345456046 73.23549644995082) bank14783 +14784 POINT(40.131287708979315 74.83062390162924) bank14784 +14785 POINT(39.9346717370717 74.00181111478851) bank14785 +14786 POINT(40.619308120215166 74.45118887592099) bank14786 +14787 POINT(41.488370139107424 74.97130171867327) bank14787 +14788 POINT(40.811029893840896 73.8928594046187) bank14788 +14789 POINT(40.58297475490723 74.37603584425517) bank14789 +14790 POINT(40.89104481813796 73.4554957935266) bank14790 +14791 POINT(39.962844491023326 73.1148666241463) bank14791 +14792 POINT(41.37990752539228 73.41704930508624) bank14792 +14793 POINT(39.882753910518396 73.67224244815075) bank14793 +14794 POINT(39.8496694072814 73.17134119197898) bank14794 +14795 POINT(41.554613875539495 74.56498836320371) bank14795 +14796 POINT(40.828189477009495 74.74864722073849) bank14796 +14797 POINT(41.38105169175927 74.13454089081368) bank14797 +14798 POINT(40.106445418746404 73.86431406450508) bank14798 +14799 POINT(41.23443868614585 74.09490902730704) bank14799 +14800 POINT(40.17643792784602 73.10257859614946) bank14800 +14801 POINT(39.80747430751066 74.39870331617789) bank14801 +14802 POINT(40.873249489510115 74.335567473473) bank14802 +14803 POINT(40.34463405727892 74.09256791237412) bank14803 +14804 POINT(40.889415775861806 73.75499105559575) bank14804 +14805 POINT(41.11964134885057 74.12094361817145) bank14805 +14806 POINT(40.92322774293771 73.25074942594227) bank14806 +14807 POINT(41.05159944412231 74.65597891787499) bank14807 +14808 POINT(39.98280947680029 73.58528900210044) bank14808 +14809 POINT(40.10394439739794 74.38834308498144) bank14809 +14810 POINT(40.04810805028029 74.58171463814818) bank14810 +14811 POINT(41.462802074048845 73.93469965025722) bank14811 +14812 POINT(40.841096440068284 73.82894413585745) bank14812 +14813 POINT(40.58411640616502 74.54483085490614) bank14813 +14814 POINT(40.46575543244498 73.9502468420414) bank14814 +14815 POINT(40.11521098103324 74.67714985862722) bank14815 +14816 POINT(40.15134104213548 73.06101709765139) bank14816 +14817 POINT(40.76484446066947 73.43786563184355) bank14817 +14818 POINT(40.610874603384886 73.13371196366762) bank14818 +14819 POINT(41.29626805455776 74.13953001060015) bank14819 +14820 POINT(40.32279096919984 74.40450223288907) bank14820 +14821 POINT(40.586677307816224 74.81229507826876) bank14821 +14822 POINT(41.24828176601519 73.11528499238337) bank14822 +14823 POINT(39.788455991068666 73.35382243108155) bank14823 +14824 POINT(40.116274486380725 74.21524730698528) bank14824 +14825 POINT(40.31883668240921 74.90927274601388) bank14825 +14826 POINT(40.589094974778675 73.07389573789813) bank14826 +14827 POINT(39.999540667272164 73.39727551188143) bank14827 +14828 POINT(40.53900670031223 74.1296062575177) bank14828 +14829 POINT(40.46410191635143 74.85023992886491) bank14829 +14830 POINT(39.84822729376106 74.48345246381471) bank14830 +14831 POINT(40.18934126186099 73.95084946238791) bank14831 +14832 POINT(41.65172459621136 74.3976539479359) bank14832 +14833 POINT(40.772412537870274 73.03361831464127) bank14833 +14834 POINT(41.2978904344579 73.1881977830809) bank14834 +14835 POINT(40.18669202906489 73.48890239586024) bank14835 +14836 POINT(41.04489318653867 73.98608983393095) bank14836 +14837 POINT(40.37301750450143 74.72274329447762) bank14837 +14838 POINT(41.51607114702599 73.45139060894692) bank14838 +14839 POINT(40.352368968931835 73.0211313711061) bank14839 +14840 POINT(40.32927614353339 73.3325291506901) bank14840 +14841 POINT(39.96592492558104 74.78470096064106) bank14841 +14842 POINT(39.857582966099606 74.65093571930989) bank14842 +14843 POINT(40.72490588304919 73.86097047397442) bank14843 +14844 POINT(40.413422024050455 74.85687066608034) bank14844 +14845 POINT(40.37703957958763 74.94856182992972) bank14845 +14846 POINT(40.84220333670953 74.3471681351609) bank14846 +14847 POINT(40.873894429788386 74.35739062650292) bank14847 +14848 POINT(40.44741148437886 74.1447240650637) bank14848 +14849 POINT(40.204309006157544 73.77778908247272) bank14849 +14850 POINT(41.17027909381085 74.68993971976012) bank14850 +14851 POINT(40.36107470874728 74.61350171102323) bank14851 +14852 POINT(41.58003677777915 74.05485081221532) bank14852 +14853 POINT(40.704087429996235 73.29359719566351) bank14853 +14854 POINT(41.68184721819556 73.46205893527816) bank14854 +14855 POINT(41.25324073621214 73.12502527988107) bank14855 +14856 POINT(40.140734472369694 73.52282680642682) bank14856 +14857 POINT(39.78400137776039 73.19087263603863) bank14857 +14858 POINT(40.97872401608759 73.84818421799461) bank14858 +14859 POINT(40.171460596923914 73.4455180938644) bank14859 +14860 POINT(39.94472931971353 73.42021586208688) bank14860 +14861 POINT(41.01193782699951 74.57044754595914) bank14861 +14862 POINT(41.69300346157883 73.33366283518694) bank14862 +14863 POINT(41.221008260360556 73.93552366201126) bank14863 +14864 POINT(41.223001768383526 73.83422749254356) bank14864 +14865 POINT(40.22802692749614 73.20222984525032) bank14865 +14866 POINT(41.230761588085166 73.4740500190393) bank14866 +14867 POINT(41.43841424011489 73.9408840213276) bank14867 +14868 POINT(40.98123497131261 74.58859473166777) bank14868 +14869 POINT(40.78537164049348 74.2124131899809) bank14869 +14870 POINT(41.45068285064398 73.37507173003506) bank14870 +14871 POINT(39.88059394027448 74.90576871854293) bank14871 +14872 POINT(41.44035798930385 73.68709386623993) bank14872 +14873 POINT(40.09483676310773 73.09395132114916) bank14873 +14874 POINT(41.6055703922343 74.63022804775362) bank14874 +14875 POINT(41.64588309633647 74.61846924166912) bank14875 +14876 POINT(40.787401884377374 74.13918958078482) bank14876 +14877 POINT(40.90753186290703 74.20231010324059) bank14877 +14878 POINT(40.1944899493318 73.77267464940202) bank14878 +14879 POINT(40.043736534429414 73.35420519050587) bank14879 +14880 POINT(40.86106914290094 73.8638064223315) bank14880 +14881 POINT(40.715469614369816 74.26098187774781) bank14881 +14882 POINT(40.749084590359914 73.47359118550486) bank14882 +14883 POINT(39.76395275040447 73.78076947879752) bank14883 +14884 POINT(40.16458251193669 74.26820503727464) bank14884 +14885 POINT(40.01080838871182 73.73769276040288) bank14885 +14886 POINT(39.92973292335943 74.6737391439621) bank14886 +14887 POINT(41.27980461995372 73.3616942487994) bank14887 +14888 POINT(39.95860347144666 74.86801232055197) bank14888 +14889 POINT(41.549424551381165 73.56432537003212) bank14889 +14890 POINT(40.03246741876343 74.7497364908078) bank14890 +14891 POINT(39.73359829221875 74.2910499914512) bank14891 +14892 POINT(40.91076145081696 74.66621036332832) bank14892 +14893 POINT(41.05414104811397 74.2099704825993) bank14893 +14894 POINT(41.22228783151568 74.32226798624902) bank14894 +14895 POINT(39.98218910992598 73.4452682624986) bank14895 +14896 POINT(41.63290656083313 73.35973923279839) bank14896 +14897 POINT(40.75899039076407 73.79765707723305) bank14897 +14898 POINT(40.73073413615003 73.828875276982) bank14898 +14899 POINT(41.34450201245414 74.91046211591825) bank14899 +14900 POINT(41.53688614845609 73.34492138238765) bank14900 +14901 POINT(41.281078921114805 73.8410152452974) bank14901 +14902 POINT(41.344621999194224 73.98510468244622) bank14902 +14903 POINT(40.20877662626729 74.83498210021766) bank14903 +14904 POINT(40.49555823633896 73.67143894736371) bank14904 +14905 POINT(41.685546368573405 74.56926850511097) bank14905 +14906 POINT(40.22861203661719 73.85261638050268) bank14906 +14907 POINT(41.173154657112654 74.2584016035199) bank14907 +14908 POINT(41.51159328811133 73.11462376634215) bank14908 +14909 POINT(39.737399109571456 74.65641005528886) bank14909 +14910 POINT(41.26094543355142 73.74646307487897) bank14910 +14911 POINT(39.91879147053269 74.79344765567967) bank14911 +14912 POINT(40.98802300081796 74.43271177272494) bank14912 +14913 POINT(40.476380206561714 74.98776525879708) bank14913 +14914 POINT(40.15511640825668 74.32319864239429) bank14914 +14915 POINT(41.273269724266484 73.83471381181404) bank14915 +14916 POINT(41.12134806455159 73.7514440947062) bank14916 +14917 POINT(39.98980278462852 74.13043106391956) bank14917 +14918 POINT(40.25584965245675 73.72125479850158) bank14918 +14919 POINT(41.06409720606342 74.98784970945988) bank14919 +14920 POINT(40.13462884890649 73.57907042869122) bank14920 +14921 POINT(41.53601094159598 73.51897092316236) bank14921 +14922 POINT(41.61124588188545 73.1947416416473) bank14922 +14923 POINT(41.28985839597029 73.25215301833526) bank14923 +14924 POINT(40.70270549396727 74.15362573496517) bank14924 +14925 POINT(39.76303874736608 73.97985257505061) bank14925 +14926 POINT(39.89006734140035 74.09528538029437) bank14926 +14927 POINT(39.854174740440854 74.83538622818983) bank14927 +14928 POINT(41.2992198028836 74.50428645809627) bank14928 +14929 POINT(40.06043539327484 73.03067666255457) bank14929 +14930 POINT(39.90154139589618 74.98008555820057) bank14930 +14931 POINT(40.91438643721905 73.47673634270214) bank14931 +14932 POINT(40.52794210637366 74.43155591516981) bank14932 +14933 POINT(40.319387702796355 74.07632041202135) bank14933 +14934 POINT(41.685358815915905 73.36197925083766) bank14934 +14935 POINT(40.544768172037266 74.54158701460123) bank14935 +14936 POINT(41.45416980137947 74.80526040912882) bank14936 +14937 POINT(40.01923479474186 74.65468703737244) bank14937 +14938 POINT(41.05622830883722 73.91988493318075) bank14938 +14939 POINT(40.25033742993965 74.93010291961767) bank14939 +14940 POINT(40.234239158521184 74.17633615800025) bank14940 +14941 POINT(41.70590827055356 73.09781160797765) bank14941 +14942 POINT(41.66128530525721 74.66134348697794) bank14942 +14943 POINT(40.25327252161636 73.69810078658725) bank14943 +14944 POINT(41.00081838535612 73.87482876192958) bank14944 +14945 POINT(41.15650141974195 74.15965547731157) bank14945 +14946 POINT(40.75545848674566 74.62235862652838) bank14946 +14947 POINT(40.3796342814811 73.28899186414066) bank14947 +14948 POINT(40.46059703239321 73.34891420125308) bank14948 +14949 POINT(40.1836379013518 73.71280645060018) bank14949 +14950 POINT(41.58415368841567 73.88955793577769) bank14950 +14951 POINT(41.43418986270746 74.16199104888022) bank14951 +14952 POINT(41.59510937219261 74.69907950103308) bank14952 +14953 POINT(40.24651972408327 74.10369110197466) bank14953 +14954 POINT(40.64187706422013 73.6783788622774) bank14954 +14955 POINT(40.302557530408954 74.47470390669038) bank14955 +14956 POINT(40.27735319802145 73.98826807023922) bank14956 +14957 POINT(40.373443648457204 73.82218817644713) bank14957 +14958 POINT(41.60691367859423 73.81255808424278) bank14958 +14959 POINT(40.264086733406906 74.78777461327506) bank14959 +14960 POINT(40.81237284129401 73.77947466510098) bank14960 +14961 POINT(41.34882654394062 74.58132955078523) bank14961 +14962 POINT(41.20082442354501 74.73756507258582) bank14962 +14963 POINT(40.793501204063034 73.49441448096518) bank14963 +14964 POINT(39.96401038677714 73.98945692898148) bank14964 +14965 POINT(40.68646987530449 74.2106072371941) bank14965 +14966 POINT(39.842012636754106 73.93851985435086) bank14966 +14967 POINT(41.42853740784636 73.79129827099531) bank14967 +14968 POINT(40.277879525117356 74.89454472308043) bank14968 +14969 POINT(40.43381478036049 74.78092124315751) bank14969 +14970 POINT(40.159737889016185 73.58203656785989) bank14970 +14971 POINT(39.93016079594911 74.78655781107186) bank14971 +14972 POINT(41.13301355603259 74.35350147932293) bank14972 +14973 POINT(40.68319048101035 74.79973213463069) bank14973 +14974 POINT(40.0445109823607 74.74042861972464) bank14974 +14975 POINT(40.08270699228925 73.56015911737042) bank14975 +14976 POINT(40.0248781581408 74.89389854784808) bank14976 +14977 POINT(41.08094019705437 73.7902105553296) bank14977 +14978 POINT(41.297536114396216 74.88455172620384) bank14978 +14979 POINT(40.723209620395814 74.97806805089874) bank14979 +14980 POINT(40.95524208526303 73.54992564214122) bank14980 +14981 POINT(41.40584324283017 73.84003344869346) bank14981 +14982 POINT(41.54538275313375 74.87705786617747) bank14982 +14983 POINT(41.36295108558355 74.60102358805774) bank14983 +14984 POINT(41.52744655189119 74.30707325872697) bank14984 +14985 POINT(40.610857845982025 73.82425977179385) bank14985 +14986 POINT(41.05243735006019 73.83584784763224) bank14986 +14987 POINT(41.156568989164256 74.80359484289232) bank14987 +14988 POINT(39.81239723953388 73.51097645186249) bank14988 +14989 POINT(40.74455574810241 74.95143206851593) bank14989 +14990 POINT(39.99893658657592 73.82444113488948) bank14990 +14991 POINT(40.276130641316705 73.53856749482141) bank14991 +14992 POINT(40.82568385703619 74.96288157292987) bank14992 +14993 POINT(40.05323170095674 73.38664138765849) bank14993 +14994 POINT(41.64982618211292 73.39083688733514) bank14994 +14995 POINT(40.28001454287926 74.78321291574063) bank14995 +14996 POINT(39.72926609709399 73.61306753123576) bank14996 +14997 POINT(40.73301890057339 74.95380672517965) bank14997 +14998 POINT(39.75204909282436 74.70683395397405) bank14998 +14999 POINT(41.56258106739456 73.61420054562383) bank14999 +15000 POINT(40.192048392409276 74.99841456622232) bank15000 +15001 POINT(39.8834377934596 73.55016986686748) bank15001 +15002 POINT(39.95241223248356 73.31850724100526) bank15002 +15003 POINT(40.73906785201427 74.81475699601478) bank15003 +15004 POINT(41.687202452936916 73.40435716781666) bank15004 +15005 POINT(40.80216813862246 74.13940692866203) bank15005 +15006 POINT(41.07606093497486 74.93966284302891) bank15006 +15007 POINT(40.22611863789276 73.1679493222146) bank15007 +15008 POINT(40.07590717823487 73.01337696734524) bank15008 +15009 POINT(41.229973146639516 74.34941109906134) bank15009 +15010 POINT(40.77247501658451 74.51604486425738) bank15010 +15011 POINT(40.76192752007364 73.17961819152887) bank15011 +15012 POINT(41.41186961894993 74.3930062526571) bank15012 +15013 POINT(40.20178215102594 73.9546728646805) bank15013 +15014 POINT(41.65350619776067 73.72150030035354) bank15014 +15015 POINT(40.64156592242087 74.87334201682702) bank15015 +15016 POINT(40.597442304474455 73.93180803875734) bank15016 +15017 POINT(40.37217061966807 73.67051301645098) bank15017 +15018 POINT(41.576470841425056 73.78976686987109) bank15018 +15019 POINT(40.64329001705059 74.15896889422467) bank15019 +15020 POINT(40.29123188885377 74.97357122606955) bank15020 +15021 POINT(40.075430264920755 73.49580237162176) bank15021 +15022 POINT(40.659097808792424 73.99678777744046) bank15022 +15023 POINT(41.33495453284627 73.53424869680354) bank15023 +15024 POINT(41.482779091454496 74.37738057888164) bank15024 +15025 POINT(39.910414545847196 74.5598494811821) bank15025 +15026 POINT(41.08411871278164 74.97696661226072) bank15026 +15027 POINT(40.35318236055919 74.25479798203517) bank15027 +15028 POINT(40.68131993563649 73.63027079911615) bank15028 +15029 POINT(39.95675578883112 73.16262347141647) bank15029 +15030 POINT(41.63771669398424 73.42603322880002) bank15030 +15031 POINT(41.40249488106493 73.03811582938117) bank15031 +15032 POINT(40.37521094064908 73.49356014217884) bank15032 +15033 POINT(39.94647917693127 74.9315298019619) bank15033 +15034 POINT(41.398903879973986 73.91265214955462) bank15034 +15035 POINT(40.592789925575964 73.99223526966401) bank15035 +15036 POINT(40.46952149578716 73.08666716515009) bank15036 +15037 POINT(40.14678128889395 73.46258539910941) bank15037 +15038 POINT(41.402045393676225 73.9299725708554) bank15038 +15039 POINT(41.096841810274206 74.39771403129053) bank15039 +15040 POINT(41.625688466051784 74.3059416696275) bank15040 +15041 POINT(41.650267305564554 73.38474035167577) bank15041 +15042 POINT(39.79572243743078 74.48341380834493) bank15042 +15043 POINT(41.29972301482805 73.50624947680804) bank15043 +15044 POINT(40.755179648300484 73.67819842947938) bank15044 +15045 POINT(41.38134317968272 73.8614550711217) bank15045 +15046 POINT(40.885827660095046 74.07113552598082) bank15046 +15047 POINT(40.84638435545337 73.51639347604225) bank15047 +15048 POINT(40.312589713857086 74.98875362767647) bank15048 +15049 POINT(40.566376889942795 73.78884818459966) bank15049 +15050 POINT(40.983486613250896 73.539355198719) bank15050 +15051 POINT(41.57431203696178 73.29135091403352) bank15051 +15052 POINT(40.173750137866534 74.70388848108637) bank15052 +15053 POINT(40.898399202579945 73.53265451860108) bank15053 +15054 POINT(41.36530936948483 74.21682787434916) bank15054 +15055 POINT(40.30862211856846 74.98633472476394) bank15055 +15056 POINT(40.79274146379459 73.5002569933226) bank15056 +15057 POINT(40.792059348189945 73.00950445135365) bank15057 +15058 POINT(41.52728463698573 73.71063644652939) bank15058 +15059 POINT(41.32258418632719 74.46542899278391) bank15059 +15060 POINT(41.11812503753145 74.43634375761285) bank15060 +15061 POINT(40.43968045386436 74.03168099529579) bank15061 +15062 POINT(41.086903614329444 73.22537461237802) bank15062 +15063 POINT(39.98119843441889 73.67454154876384) bank15063 +15064 POINT(39.82352257221053 73.85233842477643) bank15064 +15065 POINT(41.368938401325956 73.19376060583501) bank15065 +15066 POINT(41.40137625063836 74.24344048691539) bank15066 +15067 POINT(39.869850736425015 74.53360633804039) bank15067 +15068 POINT(40.40135200449778 74.52497635497855) bank15068 +15069 POINT(39.774356409427504 73.80947890152865) bank15069 +15070 POINT(39.99707177875972 73.11579053142324) bank15070 +15071 POINT(41.03207537154991 73.10591049699067) bank15071 +15072 POINT(40.353873377318195 74.92556240728811) bank15072 +15073 POINT(40.676187156734926 74.57469623039724) bank15073 +15074 POINT(40.51183223001036 74.40700248883046) bank15074 +15075 POINT(41.69365812338369 74.16775078724362) bank15075 +15076 POINT(41.048503769956575 73.47034744873139) bank15076 +15077 POINT(41.63348151259029 74.48222602277518) bank15077 +15078 POINT(41.33924434062529 74.03296419308764) bank15078 +15079 POINT(41.33527834795587 73.11012375008006) bank15079 +15080 POINT(40.70408886800078 73.1769736459958) bank15080 +15081 POINT(40.01978585810821 73.46586766489492) bank15081 +15082 POINT(41.34508413893451 74.46275667195167) bank15082 +15083 POINT(40.16184889944098 74.21664434653583) bank15083 +15084 POINT(41.2806376040665 74.85707300556389) bank15084 +15085 POINT(40.2343907736582 74.35903611436122) bank15085 +15086 POINT(39.96891087543468 73.27488422517095) bank15086 +15087 POINT(40.09997186151288 74.72855233731448) bank15087 +15088 POINT(40.31892600971103 74.67189957690375) bank15088 +15089 POINT(41.079860517629065 73.38772234817873) bank15089 +15090 POINT(39.726450353493455 74.84714270810424) bank15090 +15091 POINT(40.99364139818269 73.25065280864537) bank15091 +15092 POINT(40.3693591826026 74.31247553057646) bank15092 +15093 POINT(40.18209447588147 74.74583013506486) bank15093 +15094 POINT(39.829947606338855 73.01620747668589) bank15094 +15095 POINT(40.86092886722535 73.64054905638793) bank15095 +15096 POINT(41.288468977686634 73.23078851382574) bank15096 +15097 POINT(41.2179977824692 73.34761959099272) bank15097 +15098 POINT(40.08240696445549 73.9443380742512) bank15098 +15099 POINT(40.81754460954097 74.01412560093948) bank15099 +15100 POINT(41.09945641324629 73.92331031764576) bank15100 +15101 POINT(39.74128631661402 74.28461954430688) bank15101 +15102 POINT(40.19832817741655 73.53653886026683) bank15102 +15103 POINT(40.492885985050805 74.54652015145653) bank15103 +15104 POINT(40.645596641739786 73.44061597088216) bank15104 +15105 POINT(41.5002765193629 74.82680349329017) bank15105 +15106 POINT(39.71606638407813 74.28383294645627) bank15106 +15107 POINT(39.76540351592878 74.50847583684481) bank15107 +15108 POINT(40.42238791950464 73.57232931489658) bank15108 +15109 POINT(40.657243062507725 74.09742526980949) bank15109 +15110 POINT(40.51126716363987 73.30732101676755) bank15110 +15111 POINT(41.53289770426542 73.26176249109255) bank15111 +15112 POINT(40.05915590436686 74.51958418854802) bank15112 +15113 POINT(40.468274062489975 73.36799924641564) bank15113 +15114 POINT(41.240377621074366 73.62034193236371) bank15114 +15115 POINT(40.93306984500176 74.48774886762445) bank15115 +15116 POINT(41.136274819973245 74.61985106346793) bank15116 +15117 POINT(40.99112082601468 74.40078134163521) bank15117 +15118 POINT(39.72618898770951 74.34516760028578) bank15118 +15119 POINT(40.70410914980301 74.58120808920857) bank15119 +15120 POINT(40.538733610696504 74.3240164992084) bank15120 +15121 POINT(40.61651510801944 74.35659068951104) bank15121 +15122 POINT(41.467433686201446 73.11797706487408) bank15122 +15123 POINT(41.05506281836551 74.98220951447756) bank15123 +15124 POINT(41.36792539824411 73.68146476011144) bank15124 +15125 POINT(40.95931031131034 73.53916733089) bank15125 +15126 POINT(39.98505824548932 73.8627087846795) bank15126 +15127 POINT(41.08228359367866 74.46972223426704) bank15127 +15128 POINT(41.226940241733395 74.60235598865731) bank15128 +15129 POINT(39.93520222896116 73.95871706606621) bank15129 +15130 POINT(41.37223137486438 74.83277182588355) bank15130 +15131 POINT(41.30423972481467 73.64738486767524) bank15131 +15132 POINT(40.836701452676245 74.08597810883788) bank15132 +15133 POINT(40.582907697407876 73.22995939987551) bank15133 +15134 POINT(41.16942081568162 73.19983156277881) bank15134 +15135 POINT(41.12278054214967 74.94401620739333) bank15135 +15136 POINT(40.503854771193375 73.6224476696616) bank15136 +15137 POINT(40.38782297593451 74.88674883851924) bank15137 +15138 POINT(40.765118272767936 74.69065411750104) bank15138 +15139 POINT(41.36798368722111 73.36332767231222) bank15139 +15140 POINT(40.01159852428203 73.11197894813766) bank15140 +15141 POINT(41.552889576688905 73.21566560458486) bank15141 +15142 POINT(41.35073021586103 74.81641299961578) bank15142 +15143 POINT(40.02389858836719 73.48078173909724) bank15143 +15144 POINT(41.57838167686084 74.09501914154231) bank15144 +15145 POINT(41.40165984532503 74.60635040324422) bank15145 +15146 POINT(40.88421403858329 73.36125371602002) bank15146 +15147 POINT(39.8585320952907 74.56465792044607) bank15147 +15148 POINT(40.851097026882016 74.19928765518262) bank15148 +15149 POINT(39.97189214839969 74.55899085121371) bank15149 +15150 POINT(40.677666590594825 74.68570588055088) bank15150 +15151 POINT(41.480649090536566 73.93673884476456) bank15151 +15152 POINT(39.75750200026635 73.85179160663819) bank15152 +15153 POINT(41.691777499636956 74.17485599905581) bank15153 +15154 POINT(40.02357143240191 74.59508849742332) bank15154 +15155 POINT(40.655677864111226 73.79744401422832) bank15155 +15156 POINT(40.82450715393595 73.16796516668168) bank15156 +15157 POINT(40.49179550450705 73.14020395706693) bank15157 +15158 POINT(40.434625768546994 73.78404134224022) bank15158 +15159 POINT(40.311741005612475 74.66931154094979) bank15159 +15160 POINT(40.16487388759842 73.6320258049672) bank15160 +15161 POINT(40.73840593839074 74.56197408461689) bank15161 +15162 POINT(40.17017403276555 74.54075421171406) bank15162 +15163 POINT(41.0596561910116 73.5466897385129) bank15163 +15164 POINT(39.74897266660331 74.48895359399215) bank15164 +15165 POINT(41.041380944032284 74.66532289299172) bank15165 +15166 POINT(40.83212933001858 74.95211810388751) bank15166 +15167 POINT(41.68147577375801 74.19856066084965) bank15167 +15168 POINT(41.64757397156067 73.9422696157884) bank15168 +15169 POINT(40.997206461254976 73.25245566622996) bank15169 +15170 POINT(41.59479077080897 73.78564884671185) bank15170 +15171 POINT(41.62207788560315 73.01175492171231) bank15171 +15172 POINT(40.49204483621699 74.62395553868541) bank15172 +15173 POINT(41.30108617268118 73.46348414485838) bank15173 +15174 POINT(40.52261750698328 73.5518455207879) bank15174 +15175 POINT(40.30302729962049 73.54662866059144) bank15175 +15176 POINT(40.88520195082846 73.42154637940801) bank15176 +15177 POINT(41.21097485111665 74.73676571509151) bank15177 +15178 POINT(41.61302799446473 73.68058598793591) bank15178 +15179 POINT(39.88295911122691 74.89482717738642) bank15179 +15180 POINT(40.958084166105266 74.13352829525759) bank15180 +15181 POINT(40.423274519266286 73.63743355484931) bank15181 +15182 POINT(41.68582965471793 74.99113679205111) bank15182 +15183 POINT(39.85877422536039 74.54800305412991) bank15183 +15184 POINT(40.61674151330235 73.2656288436361) bank15184 +15185 POINT(41.1000406712562 74.27541796096745) bank15185 +15186 POINT(41.036342597496905 74.90624228438341) bank15186 +15187 POINT(41.083355449353895 73.9518882886755) bank15187 +15188 POINT(39.90553936148294 73.98468358598211) bank15188 +15189 POINT(39.79721594014314 74.50349049162321) bank15189 +15190 POINT(39.712860358619494 74.62406347101508) bank15190 +15191 POINT(40.39213484403034 74.79154576442723) bank15191 +15192 POINT(41.15196793589942 74.22114237433382) bank15192 +15193 POINT(40.96719753689081 73.71794960643857) bank15193 +15194 POINT(40.48502918839243 73.38690643955837) bank15194 +15195 POINT(40.87080389957306 73.49723702195878) bank15195 +15196 POINT(40.66050335299448 74.33428761376308) bank15196 +15197 POINT(39.840817374214964 74.64621050505316) bank15197 +15198 POINT(39.960201485887914 73.697898102456) bank15198 +15199 POINT(40.74859965335883 73.0307497382436) bank15199 +15200 POINT(39.72809751388238 74.76646317590789) bank15200 +15201 POINT(41.27707805750859 74.97585720280696) bank15201 +15202 POINT(41.33235344983349 73.36884160506912) bank15202 +15203 POINT(40.50718559785254 74.93568034020188) bank15203 +15204 POINT(39.80701838457139 74.93502428460057) bank15204 +15205 POINT(40.15418939605095 74.16265164962556) bank15205 +15206 POINT(40.93732590228866 73.92372037028005) bank15206 +15207 POINT(39.89615133216948 74.87556835532007) bank15207 +15208 POINT(41.08202798214019 74.29948939636301) bank15208 +15209 POINT(40.75812190515702 73.9566706266184) bank15209 +15210 POINT(40.9135376922463 74.10554064850194) bank15210 +15211 POINT(40.40267156141074 73.33276557699038) bank15211 +15212 POINT(40.79977444213355 73.48393332251929) bank15212 +15213 POINT(40.26054071993013 73.47410540139573) bank15213 +15214 POINT(39.973509655114526 73.61746397221732) bank15214 +15215 POINT(40.120551038748495 73.42321606629855) bank15215 +15216 POINT(40.76166160271359 73.63115980794697) bank15216 +15217 POINT(39.90862219149469 73.70520201502737) bank15217 +15218 POINT(40.55183776073451 73.30569367195397) bank15218 +15219 POINT(40.15330976959379 73.58642819972741) bank15219 +15220 POINT(40.606313582698434 74.62816222995562) bank15220 +15221 POINT(41.281572230889815 74.94070105341439) bank15221 +15222 POINT(41.705492482205415 74.12523337441593) bank15222 +15223 POINT(41.52390685504543 74.07337491623964) bank15223 +15224 POINT(40.69509743946911 74.52443614444186) bank15224 +15225 POINT(40.98755671769468 74.65631105917193) bank15225 +15226 POINT(39.915771557182744 73.85882173771166) bank15226 +15227 POINT(40.04234220138792 74.09887666771732) bank15227 +15228 POINT(40.442687589874076 73.71115575030998) bank15228 +15229 POINT(39.80565008890244 74.9925732540422) bank15229 +15230 POINT(41.45490272656291 74.89072068520812) bank15230 +15231 POINT(40.35922337584167 73.84383711736281) bank15231 +15232 POINT(40.48608919169631 74.53556896326472) bank15232 +15233 POINT(40.598336415918745 74.20167227663659) bank15233 +15234 POINT(41.436284101015765 74.44093858222229) bank15234 +15235 POINT(39.99294008479978 73.76917301059349) bank15235 +15236 POINT(40.24635655077677 74.37910893546882) bank15236 +15237 POINT(40.92590865288527 74.92955525723774) bank15237 +15238 POINT(40.750777221068255 73.84013514846772) bank15238 +15239 POINT(40.328860577151715 73.66899353167628) bank15239 +15240 POINT(40.85269923260107 74.13115757030972) bank15240 +15241 POINT(41.08678952943379 74.92028990719153) bank15241 +15242 POINT(39.95632012675167 74.17534741208286) bank15242 +15243 POINT(41.02301331113201 74.07439885115127) bank15243 +15244 POINT(39.99787162128079 73.1528765377019) bank15244 +15245 POINT(40.798226878180934 74.65725860646913) bank15245 +15246 POINT(40.289251442111066 74.12937263498544) bank15246 +15247 POINT(41.13050292689183 73.0069834453295) bank15247 +15248 POINT(39.8489845115326 73.17806665460809) bank15248 +15249 POINT(41.55318249439116 74.10197769223898) bank15249 +15250 POINT(39.940163713754174 74.80489929851353) bank15250 +15251 POINT(40.280061075916954 74.09810734655908) bank15251 +15252 POINT(41.25504411358122 73.66227284680079) bank15252 +15253 POINT(41.696994613429126 73.92345414180981) bank15253 +15254 POINT(41.09274737224609 73.83851940364998) bank15254 +15255 POINT(40.38214609752452 74.32338288781037) bank15255 +15256 POINT(40.07675556392025 73.29545286689824) bank15256 +15257 POINT(41.386388420373144 74.16417284230812) bank15257 +15258 POINT(40.7596676604894 73.15799427173113) bank15258 +15259 POINT(40.87876385145257 74.70373170381235) bank15259 +15260 POINT(39.849238614416706 73.43538311041817) bank15260 +15261 POINT(40.98341330686261 74.71182811307625) bank15261 +15262 POINT(40.39223767982653 74.13145583493993) bank15262 +15263 POINT(41.683228487672864 74.99691849560273) bank15263 +15264 POINT(40.15118775050432 74.46288719175787) bank15264 +15265 POINT(41.09296077184832 74.81388014003896) bank15265 +15266 POINT(40.419024129887624 73.46242226083046) bank15266 +15267 POINT(41.69321604276062 74.39778910332585) bank15267 +15268 POINT(40.70756882869521 73.12076346345694) bank15268 +15269 POINT(40.85988936714596 73.47155545224074) bank15269 +15270 POINT(40.83740860686031 74.10490811478093) bank15270 +15271 POINT(40.11696928166748 74.5988759200371) bank15271 +15272 POINT(40.073802991111485 74.76066763160715) bank15272 +15273 POINT(40.31194718762306 74.89851024662053) bank15273 +15274 POINT(41.253879873409545 73.6693193825894) bank15274 +15275 POINT(41.01534862096648 73.44076244643576) bank15275 +15276 POINT(40.871023663446195 74.01130330168712) bank15276 +15277 POINT(41.447321633857136 74.13069872291884) bank15277 +15278 POINT(41.41802323173822 73.6661367020851) bank15278 +15279 POINT(39.85824583533218 73.11847066430406) bank15279 +15280 POINT(40.311628038882965 74.66723332836682) bank15280 +15281 POINT(40.61815017243032 73.82552941884897) bank15281 +15282 POINT(40.08218779766674 74.48019533118263) bank15282 +15283 POINT(41.48624218497272 73.61372198161885) bank15283 +15284 POINT(40.66796184561152 74.77811987916962) bank15284 +15285 POINT(40.41098205295218 73.62686898436715) bank15285 +15286 POINT(40.61460004476519 74.2875930173119) bank15286 +15287 POINT(40.69656419896795 74.78724990463007) bank15287 +15288 POINT(40.38024664135076 74.03380896811242) bank15288 +15289 POINT(40.854789552537255 73.7388831109172) bank15289 +15290 POINT(40.682353701919354 74.67096245866857) bank15290 +15291 POINT(41.05707112981636 74.64252757625022) bank15291 +15292 POINT(41.636215396841045 73.96760655659449) bank15292 +15293 POINT(41.386498099530336 73.25291232894651) bank15293 +15294 POINT(41.216734532429705 73.07701053436611) bank15294 +15295 POINT(40.20715252975758 73.81761049165978) bank15295 +15296 POINT(41.57063490857813 73.03591478951363) bank15296 +15297 POINT(40.025193976398604 73.15481974636229) bank15297 +15298 POINT(41.09494795183465 73.02891078722482) bank15298 +15299 POINT(40.3345141961702 73.28701567479743) bank15299 +15300 POINT(41.09439612190973 73.55162485397672) bank15300 +15301 POINT(41.23020626805836 74.57198729965889) bank15301 +15302 POINT(41.63889559542973 73.19777981215739) bank15302 +15303 POINT(40.23884558210945 73.00636129765132) bank15303 +15304 POINT(40.112140905181825 74.2275992376913) bank15304 +15305 POINT(40.45740927635924 74.9327090812902) bank15305 +15306 POINT(40.01592870168263 73.80256813863735) bank15306 +15307 POINT(40.555163574684975 73.29870341620202) bank15307 +15308 POINT(41.22981321328411 74.72019095158089) bank15308 +15309 POINT(40.32684400922272 73.31550357624458) bank15309 +15310 POINT(41.093610838745136 73.10080880015279) bank15310 +15311 POINT(41.222109043574335 74.30687936045676) bank15311 +15312 POINT(40.143976679175644 73.60888900495115) bank15312 +15313 POINT(40.79835770554204 73.13283705682858) bank15313 +15314 POINT(39.98568779887203 73.15478952395472) bank15314 +15315 POINT(40.913659858249176 73.11508141726448) bank15315 +15316 POINT(40.32938658347328 74.02010278063838) bank15316 +15317 POINT(41.32841752079828 74.90761658161735) bank15317 +15318 POINT(40.705459729050084 74.43584919261735) bank15318 +15319 POINT(41.61427418159205 74.72687384752426) bank15319 +15320 POINT(40.747715078445815 74.13968481411878) bank15320 +15321 POINT(40.14460856807156 74.23760813719908) bank15321 +15322 POINT(40.514541302393816 74.28245871880489) bank15322 +15323 POINT(40.57329082035089 73.10131223085278) bank15323 +15324 POINT(41.36192963217392 73.3852507426648) bank15324 +15325 POINT(39.89822894475818 74.2971725815237) bank15325 +15326 POINT(41.0222402414578 74.24913703500424) bank15326 +15327 POINT(40.48268036239206 74.82924065427343) bank15327 +15328 POINT(41.26951497729654 74.47096081237656) bank15328 +15329 POINT(39.88320842207505 73.28916753786432) bank15329 +15330 POINT(39.74185757024393 74.77113859490203) bank15330 +15331 POINT(40.939091094442325 73.64628541059999) bank15331 +15332 POINT(41.35837575498151 73.73408789732022) bank15332 +15333 POINT(40.66777159837986 73.27660551222458) bank15333 +15334 POINT(40.53574207079885 73.19857372620669) bank15334 +15335 POINT(41.272557941939105 74.30938725380447) bank15335 +15336 POINT(41.05713096532708 74.96606231453016) bank15336 +15337 POINT(41.538745322782404 74.13007859617859) bank15337 +15338 POINT(40.84470238935535 73.21060100768027) bank15338 +15339 POINT(40.79133755928761 74.14426275901425) bank15339 +15340 POINT(40.043162505803544 74.85187610304921) bank15340 +15341 POINT(41.64633777222646 73.05983421904362) bank15341 +15342 POINT(39.87696927630635 73.22785364598104) bank15342 +15343 POINT(40.43872706563449 73.12013314026984) bank15343 +15344 POINT(40.691109511925205 74.32649313331407) bank15344 +15345 POINT(40.01506228896763 73.14252449142194) bank15345 +15346 POINT(39.71725118159741 73.47043861508338) bank15346 +15347 POINT(40.26368270884693 74.27934393606544) bank15347 +15348 POINT(40.263023942918316 73.74110191151442) bank15348 +15349 POINT(40.33706408054109 75.00266364887001) bank15349 +15350 POINT(40.875358878794 74.07795150670196) bank15350 +15351 POINT(41.35478014875338 74.67819653432342) bank15351 +15352 POINT(40.19595876719594 73.30454332401978) bank15352 +15353 POINT(40.49220560119693 73.13602789024327) bank15353 +15354 POINT(41.39334284932164 73.28954702605505) bank15354 +15355 POINT(41.603750560807676 74.93248953027674) bank15355 +15356 POINT(40.665855161733184 73.07282133879016) bank15356 +15357 POINT(40.858460202536016 73.69586106138745) bank15357 +15358 POINT(40.20100755967756 73.30247626176454) bank15358 +15359 POINT(40.12738608005248 73.4255898410173) bank15359 +15360 POINT(39.95551103103308 73.66457616156129) bank15360 +15361 POINT(39.9427540913666 74.84983380319359) bank15361 +15362 POINT(40.14089940768551 74.58243060097294) bank15362 +15363 POINT(41.11435494285848 74.99866452600325) bank15363 +15364 POINT(41.32190880162555 74.34956066207127) bank15364 +15365 POINT(41.00375233680327 74.9782752185963) bank15365 +15366 POINT(40.92628779914685 74.31280559218501) bank15366 +15367 POINT(41.1924642968488 73.9144735754156) bank15367 +15368 POINT(40.25819536511162 74.20191890213398) bank15368 +15369 POINT(41.66410822712415 73.77453317748412) bank15369 +15370 POINT(39.72454906235955 74.66805423555863) bank15370 +15371 POINT(40.413843491769626 73.47271673165702) bank15371 +15372 POINT(39.967704072749264 74.70466464584823) bank15372 +15373 POINT(41.62094931919568 73.52886197864245) bank15373 +15374 POINT(40.21094618006605 74.36168409363115) bank15374 +15375 POINT(40.298723867617824 73.82480452319722) bank15375 +15376 POINT(41.12259189534823 73.52574280806563) bank15376 +15377 POINT(41.2787213721704 73.3824529491462) bank15377 +15378 POINT(40.2315907145083 74.82208067527816) bank15378 +15379 POINT(41.03570478964371 74.76362098103593) bank15379 +15380 POINT(41.59523363910428 74.86725289141657) bank15380 +15381 POINT(41.55373115836893 73.81834103431521) bank15381 +15382 POINT(39.90802586865755 73.99099720618572) bank15382 +15383 POINT(39.909614522054135 73.1838374678946) bank15383 +15384 POINT(40.21081360658349 73.99220794601582) bank15384 +15385 POINT(39.98195377581159 74.68266803683079) bank15385 +15386 POINT(40.242365937851915 73.05437657814055) bank15386 +15387 POINT(40.30758621342204 73.83257415943976) bank15387 +15388 POINT(41.188499959616266 73.25923452270689) bank15388 +15389 POINT(41.3185690160858 74.54546635309414) bank15389 +15390 POINT(41.36833730576681 73.07769743160178) bank15390 +15391 POINT(40.03786238979299 73.32110970050991) bank15391 +15392 POINT(40.551891703405104 74.36691168317478) bank15392 +15393 POINT(40.74626653085299 74.65498419833298) bank15393 +15394 POINT(40.10105645658487 74.56421015287621) bank15394 +15395 POINT(40.6218958315647 73.18842531086455) bank15395 +15396 POINT(40.9142267311344 73.62767638080906) bank15396 +15397 POINT(40.134866489069985 74.60638135329263) bank15397 +15398 POINT(39.71392424798879 74.72613304059666) bank15398 +15399 POINT(40.36554208125461 74.72750991922881) bank15399 +15400 POINT(40.37987094556504 74.6906207855596) bank15400 +15401 POINT(39.752230201476635 73.60933982189165) bank15401 +15402 POINT(40.59056029953884 73.78695727816387) bank15402 +15403 POINT(41.12058596362223 74.00612659297006) bank15403 +15404 POINT(40.243139803933474 73.91402975014937) bank15404 +15405 POINT(40.876727184128775 74.40713891816753) bank15405 +15406 POINT(41.20550677070637 74.25222447315065) bank15406 +15407 POINT(40.81801075426014 73.80915981773775) bank15407 +15408 POINT(41.67455557921177 73.74646846962612) bank15408 +15409 POINT(40.46731688895173 73.38521524085593) bank15409 +15410 POINT(40.50395873264389 74.9111711172215) bank15410 +15411 POINT(40.27896298722662 73.68431222701514) bank15411 +15412 POINT(41.620555229070696 74.54540562202605) bank15412 +15413 POINT(40.483718919340575 74.42335088067055) bank15413 +15414 POINT(40.658576851322096 74.01908888283151) bank15414 +15415 POINT(41.422643992703335 73.79880892444072) bank15415 +15416 POINT(41.55942224125655 73.09854280722334) bank15416 +15417 POINT(40.369844516612595 73.76670419418686) bank15417 +15418 POINT(40.39418107942682 73.51251272951) bank15418 +15419 POINT(40.91628222723603 73.75556077027775) bank15419 +15420 POINT(41.59948325234457 74.62224337246997) bank15420 +15421 POINT(41.00051324126811 73.51371464227101) bank15421 +15422 POINT(40.047639295114564 73.9993092630423) bank15422 +15423 POINT(40.75999089660734 74.9785259703077) bank15423 +15424 POINT(41.12250034245183 74.41160486970342) bank15424 +15425 POINT(39.78356034381736 74.4420592952117) bank15425 +15426 POINT(41.207024136814134 74.00079780933856) bank15426 +15427 POINT(40.04662622256381 74.38683873347112) bank15427 +15428 POINT(39.99047099797756 74.61291076010505) bank15428 +15429 POINT(40.45796756837416 74.5401646934236) bank15429 +15430 POINT(40.94719003702597 73.18309308946547) bank15430 +15431 POINT(40.21296182422765 73.02165348727686) bank15431 +15432 POINT(39.98797829251429 73.51440346206405) bank15432 +15433 POINT(41.16079992477862 73.29641639123881) bank15433 +15434 POINT(41.18454262241138 73.28999997968143) bank15434 +15435 POINT(40.50676030104148 73.74065481763927) bank15435 +15436 POINT(40.4283068652739 74.36733089224623) bank15436 +15437 POINT(41.41292588774646 73.3670633502652) bank15437 +15438 POINT(41.4997297429176 74.20286502040726) bank15438 +15439 POINT(40.53830955036078 73.48288544387536) bank15439 +15440 POINT(40.425397178403955 74.26614404706905) bank15440 +15441 POINT(40.6886751716402 73.49672484649923) bank15441 +15442 POINT(41.69587664374366 74.72553671599144) bank15442 +15443 POINT(41.04776979695668 74.5939222897775) bank15443 +15444 POINT(41.61080679914186 73.24629917714189) bank15444 +15445 POINT(40.248708141232605 74.74241422009128) bank15445 +15446 POINT(41.552561011378714 74.82076975419285) bank15446 +15447 POINT(40.83412144671244 73.08320911071107) bank15447 +15448 POINT(41.01702517140725 74.93609655366184) bank15448 +15449 POINT(41.4618199002792 74.08091525122452) bank15449 +15450 POINT(40.729109372619135 74.50082833849633) bank15450 +15451 POINT(40.0747803522328 74.99234236068325) bank15451 +15452 POINT(41.56625470202952 74.44170055154113) bank15452 +15453 POINT(40.81944593883516 73.43618214294737) bank15453 +15454 POINT(40.23766139969882 73.2401792076885) bank15454 +15455 POINT(40.85395274139789 73.4702859088137) bank15455 +15456 POINT(40.23681068963655 74.0874340780245) bank15456 +15457 POINT(41.618380815653815 73.58389772667881) bank15457 +15458 POINT(41.28557956992237 73.9219560018929) bank15458 +15459 POINT(39.89706975109529 73.80048520028366) bank15459 +15460 POINT(40.469094380753944 73.26928720134634) bank15460 +15461 POINT(41.41225347990726 74.1069337939872) bank15461 +15462 POINT(40.27446795431301 74.94275639814002) bank15462 +15463 POINT(39.8007373717608 73.2630785699216) bank15463 +15464 POINT(40.00370680259765 74.62922753254941) bank15464 +15465 POINT(41.26664345167911 73.65928182201259) bank15465 +15466 POINT(41.625512452852625 73.0224293241325) bank15466 +15467 POINT(41.55980691756321 74.30547235427726) bank15467 +15468 POINT(41.65998063545572 73.19213504599641) bank15468 +15469 POINT(40.91303725624838 73.90869004997217) bank15469 +15470 POINT(40.685219068544455 73.62016980564559) bank15470 +15471 POINT(41.46356289886052 74.85406225578879) bank15471 +15472 POINT(40.30282248347406 74.37429570837764) bank15472 +15473 POINT(39.83836669098932 73.53492798403187) bank15473 +15474 POINT(40.41935365917987 74.33301802101796) bank15474 +15475 POINT(41.11965062758485 74.6005307950356) bank15475 +15476 POINT(39.81425902033772 74.111851145648) bank15476 +15477 POINT(41.534395511620325 73.32782920782893) bank15477 +15478 POINT(39.845451816581985 73.61211687730848) bank15478 +15479 POINT(40.491219955639 74.99359325645047) bank15479 +15480 POINT(39.9486379654035 73.44267695843925) bank15480 +15481 POINT(39.86356295742873 73.33258618648728) bank15481 +15482 POINT(40.28717309843506 73.03867803180033) bank15482 +15483 POINT(40.94654300294682 73.4410738375231) bank15483 +15484 POINT(40.338343354856335 74.2216165109499) bank15484 +15485 POINT(40.778467055420315 73.85670748421542) bank15485 +15486 POINT(40.98510889234152 73.04567840981042) bank15486 +15487 POINT(39.77254303406406 74.01079025053556) bank15487 +15488 POINT(40.2104145906735 74.56607074944233) bank15488 +15489 POINT(41.17872998885043 74.92436314903885) bank15489 +15490 POINT(40.00092763175193 73.63435140680481) bank15490 +15491 POINT(41.4907823343083 74.48641760226698) bank15491 +15492 POINT(39.75789330095689 73.56093046166437) bank15492 +15493 POINT(41.38841687809971 74.12086920611875) bank15493 +15494 POINT(41.46836071233134 74.80044553849184) bank15494 +15495 POINT(40.46130525819733 73.41410107639717) bank15495 +15496 POINT(41.064665688738984 73.0654068952023) bank15496 +15497 POINT(41.25430670791934 73.94203731626692) bank15497 +15498 POINT(41.6315604282558 73.2343300663291) bank15498 +15499 POINT(40.751249275748336 73.36045803535261) bank15499 +15500 POINT(40.48767551550287 73.51851709205584) bank15500 +15501 POINT(40.85561312322534 74.40728594331009) bank15501 +15502 POINT(40.93518649920774 73.77935038357842) bank15502 +15503 POINT(40.19333377933442 74.83036726385549) bank15503 +15504 POINT(41.64594326550787 74.73951575580558) bank15504 +15505 POINT(40.776984395721094 73.94926046912285) bank15505 +15506 POINT(41.685696958223815 74.96370767430535) bank15506 +15507 POINT(40.49825982091004 73.7418913767088) bank15507 +15508 POINT(40.24437261070516 74.67191216120725) bank15508 +15509 POINT(41.55204992451717 73.78407844709642) bank15509 +15510 POINT(40.855550019339496 74.77085137715069) bank15510 +15511 POINT(39.819218456477756 74.16057081092143) bank15511 +15512 POINT(41.59764813666394 73.65768954389907) bank15512 +15513 POINT(39.82624986985679 74.68025826642922) bank15513 +15514 POINT(41.44428494019592 73.39954918205207) bank15514 +15515 POINT(41.00476814207555 73.39848551631074) bank15515 +15516 POINT(41.07207808084616 73.5558116679972) bank15516 +15517 POINT(41.18122926260328 73.72356463154175) bank15517 +15518 POINT(40.188893856012015 73.02226462803343) bank15518 +15519 POINT(41.02139378026035 74.5343029851045) bank15519 +15520 POINT(39.92272136267808 74.73334959169436) bank15520 +15521 POINT(40.03234958263557 74.95122499507389) bank15521 +15522 POINT(41.50431047778806 74.58529253475822) bank15522 +15523 POINT(41.24485915990097 73.90755062883157) bank15523 +15524 POINT(41.275866875421116 73.20577815367233) bank15524 +15525 POINT(41.57276906802722 73.18420197728162) bank15525 +15526 POINT(40.24640216573462 74.79782635368699) bank15526 +15527 POINT(39.7213412252301 74.50030463525378) bank15527 +15528 POINT(41.489410803321356 73.11326596458687) bank15528 +15529 POINT(39.72151597237159 74.11853792538088) bank15529 +15530 POINT(40.663383901342705 73.97143093112558) bank15530 +15531 POINT(39.7196068734306 73.93207285261926) bank15531 +15532 POINT(41.08333759326086 73.88544742090927) bank15532 +15533 POINT(40.7703003389962 74.25380831529493) bank15533 +15534 POINT(40.906268696770425 73.42316403480827) bank15534 +15535 POINT(40.264457818145814 74.73752432601593) bank15535 +15536 POINT(40.60009226342185 74.30547825208461) bank15536 +15537 POINT(40.07798887742413 74.17572001368211) bank15537 +15538 POINT(40.89500759820049 74.13804522416004) bank15538 +15539 POINT(41.69266700490806 74.9063602454938) bank15539 +15540 POINT(39.86645230649621 73.71885659460698) bank15540 +15541 POINT(40.75021520138425 73.04635629234845) bank15541 +15542 POINT(40.28851435100526 74.42262123474731) bank15542 +15543 POINT(41.50422307779845 73.90737063561566) bank15543 +15544 POINT(40.01210188853391 73.87618706713108) bank15544 +15545 POINT(40.84860167733177 74.54470325851646) bank15545 +15546 POINT(41.04652459461267 73.55563690314585) bank15546 +15547 POINT(41.03605082369937 74.82186379716687) bank15547 +15548 POINT(39.9827613410627 74.17703738366515) bank15548 +15549 POINT(41.071888321974896 73.38318069591986) bank15549 +15550 POINT(41.23368580408394 74.24913828620573) bank15550 +15551 POINT(40.527054067765896 74.46057019524939) bank15551 +15552 POINT(40.74813192314235 73.57125306625632) bank15552 +15553 POINT(40.53094799404222 74.31787825577871) bank15553 +15554 POINT(41.29162438378752 73.27479522478912) bank15554 +15555 POINT(39.88125238464426 74.16990797485956) bank15555 +15556 POINT(41.47203605731969 73.3684237309894) bank15556 +15557 POINT(41.05873533972296 74.94131453948506) bank15557 +15558 POINT(39.97432543886733 74.39987867476223) bank15558 +15559 POINT(41.261234538393694 74.27472339181068) bank15559 +15560 POINT(39.89807256231393 74.37666951580876) bank15560 +15561 POINT(39.89035272840643 74.70841591154976) bank15561 +15562 POINT(40.55393319210595 74.86495520880807) bank15562 +15563 POINT(40.20588459331303 74.76282821418454) bank15563 +15564 POINT(40.78462697071344 74.15787497415707) bank15564 +15565 POINT(39.872165206617936 74.68741441230168) bank15565 +15566 POINT(41.61308149076195 73.95051853338542) bank15566 +15567 POINT(39.75496616045279 74.87526622083001) bank15567 +15568 POINT(40.73994687453817 74.6514936768296) bank15568 +15569 POINT(41.103052240848655 73.3160813461403) bank15569 +15570 POINT(41.44252420987295 73.70196006843851) bank15570 +15571 POINT(41.62631039348695 73.82614514404825) bank15571 +15572 POINT(39.88605029059142 74.61926935126512) bank15572 +15573 POINT(40.05277718752589 74.80590622535921) bank15573 +15574 POINT(40.56350412074891 73.41245653445051) bank15574 +15575 POINT(41.18316711386661 74.22790118766147) bank15575 +15576 POINT(41.48402475494102 73.17139266600412) bank15576 +15577 POINT(40.75762866998953 74.83346679272002) bank15577 +15578 POINT(40.12120329482309 74.5135796613569) bank15578 +15579 POINT(40.682419168007655 73.64291809644588) bank15579 +15580 POINT(41.260631117531695 74.74753346810114) bank15580 +15581 POINT(40.52799101538276 74.88790262083741) bank15581 +15582 POINT(41.66611982331798 74.20224290606998) bank15582 +15583 POINT(40.81996965025817 74.47296696586633) bank15583 +15584 POINT(41.154268631412236 74.07442291014178) bank15584 +15585 POINT(41.51358616643268 73.3953509631728) bank15585 +15586 POINT(40.24081567706489 73.49383694421161) bank15586 +15587 POINT(39.71598592527735 74.64710134791018) bank15587 +15588 POINT(40.76295142084799 74.66508128223745) bank15588 +15589 POINT(39.74512549956405 73.0179917523159) bank15589 +15590 POINT(40.15446333366131 73.944645189099) bank15590 +15591 POINT(41.25090129390835 73.71238775314124) bank15591 +15592 POINT(39.89151762405648 74.02523733719848) bank15592 +15593 POINT(40.89392706329143 73.59087489701118) bank15593 +15594 POINT(41.31386624149546 74.43427678437484) bank15594 +15595 POINT(40.368963160082906 74.28531563696058) bank15595 +15596 POINT(41.36783768391791 74.72490609257976) bank15596 +15597 POINT(40.545243947624044 73.53252199542082) bank15597 +15598 POINT(41.54005116276225 73.41849176063387) bank15598 +15599 POINT(40.23050804279478 73.97282256322856) bank15599 +15600 POINT(40.304739798369624 73.57114168539046) bank15600 +15601 POINT(40.38139675228025 74.62053191511575) bank15601 +15602 POINT(40.836843735690785 74.30032274776734) bank15602 +15603 POINT(41.42735440875404 74.49060981182711) bank15603 +15604 POINT(40.29987874151888 73.91110993853408) bank15604 +15605 POINT(40.31985870624071 74.29029744454945) bank15605 +15606 POINT(41.58745409690604 74.29053030889136) bank15606 +15607 POINT(40.0243018380361 73.91950374209766) bank15607 +15608 POINT(39.904552692478575 73.43120726448012) bank15608 +15609 POINT(41.25543212593929 73.32801610584806) bank15609 +15610 POINT(41.009299068326584 74.84796038795676) bank15610 +15611 POINT(41.38479365883788 73.97760720775848) bank15611 +15612 POINT(40.30715137481042 74.26329751837103) bank15612 +15613 POINT(41.612667973798104 74.51057475888842) bank15613 +15614 POINT(40.33950671378225 73.46265236514289) bank15614 +15615 POINT(41.63064360675564 74.16582845316591) bank15615 +15616 POINT(40.83692561829788 74.37449453804136) bank15616 +15617 POINT(39.83656120375389 73.11017704486666) bank15617 +15618 POINT(41.68541798810211 74.17955366244955) bank15618 +15619 POINT(40.34830058177217 74.1354758696671) bank15619 +15620 POINT(39.96179182226624 74.9006298569241) bank15620 +15621 POINT(41.31634279307247 73.90995518926938) bank15621 +15622 POINT(39.804753034911926 74.78784853253879) bank15622 +15623 POINT(40.64165699102635 74.09131209942367) bank15623 +15624 POINT(39.74623431112479 74.5710261436693) bank15624 +15625 POINT(41.08067964036964 74.84717152913646) bank15625 +15626 POINT(40.66574046428901 74.28008100364544) bank15626 +15627 POINT(40.80239479863791 74.99962278654854) bank15627 +15628 POINT(41.18171598141859 73.03723228878486) bank15628 +15629 POINT(40.83352364421421 73.75677585334181) bank15629 +15630 POINT(41.149072556695515 74.37928528076014) bank15630 +15631 POINT(39.71797423830822 73.3707830157284) bank15631 +15632 POINT(41.14302859838034 73.6987204542179) bank15632 +15633 POINT(40.991918267413425 74.89526946478992) bank15633 +15634 POINT(40.413456692751986 73.05037697242913) bank15634 +15635 POINT(40.22481947239053 73.92352541647844) bank15635 +15636 POINT(41.59860535945902 74.28214816113965) bank15636 +15637 POINT(40.57930179676579 74.92975118864001) bank15637 +15638 POINT(40.57561818739809 74.80967081087896) bank15638 +15639 POINT(41.02228812285043 74.77871685686694) bank15639 +15640 POINT(40.203866359213585 74.10498072363727) bank15640 +15641 POINT(41.21983237723432 73.91267241317865) bank15641 +15642 POINT(40.63760253471195 73.55346441812033) bank15642 +15643 POINT(41.49470864441549 73.9376157291659) bank15643 +15644 POINT(39.89164418559027 74.01029654193948) bank15644 +15645 POINT(40.496690960921796 74.7914336271958) bank15645 +15646 POINT(40.00829139199495 74.45443936137886) bank15646 +15647 POINT(41.153380090663845 74.67413170926862) bank15647 +15648 POINT(41.67671392346111 74.7340572844096) bank15648 +15649 POINT(41.51512185749494 73.72906846914876) bank15649 +15650 POINT(41.14019480378971 74.56581816240548) bank15650 +15651 POINT(41.0293307752701 74.04414290929694) bank15651 +15652 POINT(40.28048109968399 73.11219064489026) bank15652 +15653 POINT(40.06554162830541 73.49881196592455) bank15653 +15654 POINT(41.205639922288746 74.80689013345997) bank15654 +15655 POINT(39.99018959018872 73.32676366681218) bank15655 +15656 POINT(40.00308343742384 74.39536244930464) bank15656 +15657 POINT(40.79085099262665 73.91643814583982) bank15657 +15658 POINT(41.65417722386155 74.69799087113218) bank15658 +15659 POINT(40.74518868679897 74.43712385000626) bank15659 +15660 POINT(40.494049091713975 73.06633378831275) bank15660 +15661 POINT(40.75896620260209 73.61489891740611) bank15661 +15662 POINT(39.78284028953201 74.60624508526585) bank15662 +15663 POINT(40.01370448990605 74.35706615247662) bank15663 +15664 POINT(39.80401661984151 73.6706058225042) bank15664 +15665 POINT(40.084970939960485 74.2216767762143) bank15665 +15666 POINT(39.83339706384784 73.20153106493746) bank15666 +15667 POINT(41.17689454160963 74.72019898099212) bank15667 +15668 POINT(40.726684285199504 73.97941718948859) bank15668 +15669 POINT(40.85197287541917 74.66782989840674) bank15669 +15670 POINT(41.61325120628378 73.07215307262788) bank15670 +15671 POINT(41.08250981528642 73.49750562845536) bank15671 +15672 POINT(40.583546795803635 74.16300107236498) bank15672 +15673 POINT(40.58584690565483 74.88612045621332) bank15673 +15674 POINT(41.67537254946556 74.6722264215713) bank15674 +15675 POINT(41.0668248249177 74.9723050459423) bank15675 +15676 POINT(40.513342304633355 73.04381919700359) bank15676 +15677 POINT(40.81038091142093 74.15373905801944) bank15677 +15678 POINT(41.396718373658636 73.84421594044608) bank15678 +15679 POINT(41.328686856535036 73.06020545420402) bank15679 +15680 POINT(41.17302708026242 74.261809668477) bank15680 +15681 POINT(41.14389367995776 74.05381921106927) bank15681 +15682 POINT(40.83229390846729 73.13851661942395) bank15682 +15683 POINT(41.57473405925325 73.84155121470461) bank15683 +15684 POINT(40.59868807797583 74.10562089265571) bank15684 +15685 POINT(40.147821598704944 73.86459270722153) bank15685 +15686 POINT(39.960272468668265 73.035089400484) bank15686 +15687 POINT(40.29489403389027 73.31502442656978) bank15687 +15688 POINT(40.49932752310143 74.89869743663097) bank15688 +15689 POINT(41.00292912602925 74.49262115510716) bank15689 +15690 POINT(41.063819979799646 74.42987000619458) bank15690 +15691 POINT(40.08756810560998 74.26865889798248) bank15691 +15692 POINT(41.01401292052144 73.90241932543127) bank15692 +15693 POINT(41.46537168756708 74.64522366877264) bank15693 +15694 POINT(40.12307769102045 73.34875510622054) bank15694 +15695 POINT(40.264598469253166 74.20284026308643) bank15695 +15696 POINT(41.20248466556428 74.41514435427969) bank15696 +15697 POINT(41.52533255347218 74.06235015059491) bank15697 +15698 POINT(40.13737437575201 73.13434830054332) bank15698 +15699 POINT(39.748378564768146 74.38201823628665) bank15699 +15700 POINT(41.464300878098896 74.11715698831823) bank15700 +15701 POINT(40.627228359317606 73.69717286151594) bank15701 +15702 POINT(40.8056444197386 74.71042075989001) bank15702 +15703 POINT(40.28644210356446 74.65934665392457) bank15703 +15704 POINT(41.204549392558675 73.42114176147034) bank15704 +15705 POINT(40.62077580682739 74.05344092920674) bank15705 +15706 POINT(40.72238351900183 74.9759523951825) bank15706 +15707 POINT(39.88612773598987 73.12639388879776) bank15707 +15708 POINT(40.4644323507493 74.02847166064643) bank15708 +15709 POINT(40.18424148853137 73.3725570457843) bank15709 +15710 POINT(40.9838295086934 73.69346374043728) bank15710 +15711 POINT(39.786728866863335 74.56611126799001) bank15711 +15712 POINT(40.285629759566646 73.30011484213047) bank15712 +15713 POINT(39.80281956495246 73.43001561923849) bank15713 +15714 POINT(40.9799193724245 74.78237157115302) bank15714 +15715 POINT(40.556673021798865 73.05546961157809) bank15715 +15716 POINT(40.77574380232475 73.57237056384864) bank15716 +15717 POINT(41.381875374971756 73.77865906221778) bank15717 +15718 POINT(40.33724282431993 73.80819898976004) bank15718 +15719 POINT(40.67399725752357 73.72906682009048) bank15719 +15720 POINT(40.00093604819234 73.32398717075648) bank15720 +15721 POINT(41.53418900063071 74.40868220663224) bank15721 +15722 POINT(41.13430755760165 73.10399338985643) bank15722 +15723 POINT(41.460619185501606 74.05616284375338) bank15723 +15724 POINT(41.58708314596596 73.72110626361564) bank15724 +15725 POINT(40.09353508400138 73.31225150715127) bank15725 +15726 POINT(40.40685252490142 73.22239195870297) bank15726 +15727 POINT(40.15484629479779 74.41427543375791) bank15727 +15728 POINT(39.71495283499435 74.50851677831787) bank15728 +15729 POINT(40.68445228651695 73.44028460976088) bank15729 +15730 POINT(41.64426615054231 73.78378187025093) bank15730 +15731 POINT(41.18790182934524 73.46055027889376) bank15731 +15732 POINT(40.718517179044184 73.49991712421995) bank15732 +15733 POINT(41.219235008197664 73.75354289161689) bank15733 +15734 POINT(39.99600264601935 74.66113375342235) bank15734 +15735 POINT(39.80911040408362 74.27014157166063) bank15735 +15736 POINT(41.11453750286074 74.6878700385033) bank15736 +15737 POINT(40.829832813320984 73.28498387204937) bank15737 +15738 POINT(39.99084727390959 73.22296350928008) bank15738 +15739 POINT(41.195154170333666 74.60399001462365) bank15739 +15740 POINT(40.62386890767116 73.41832466224803) bank15740 +15741 POINT(40.50439088517565 73.44779637546293) bank15741 +15742 POINT(41.56264234227303 73.38984102698603) bank15742 +15743 POINT(41.30601434116572 74.0356180715179) bank15743 +15744 POINT(40.1355044277889 74.95659600753379) bank15744 +15745 POINT(40.656547846801566 73.53475525928451) bank15745 +15746 POINT(41.689338145000065 73.95084042986052) bank15746 +15747 POINT(39.79971876339578 73.93750073600572) bank15747 +15748 POINT(40.373709984820394 73.68771506865664) bank15748 +15749 POINT(39.859649995681735 73.54481753132791) bank15749 +15750 POINT(40.26549130774453 75.00305128876097) bank15750 +15751 POINT(40.55821996089085 74.82630432579002) bank15751 +15752 POINT(40.94301466715006 74.8760040974548) bank15752 +15753 POINT(40.874556434068005 73.72514002414977) bank15753 +15754 POINT(40.10722388120238 73.4917427587399) bank15754 +15755 POINT(41.248349374216 73.77870642639031) bank15755 +15756 POINT(40.442652881133824 73.93544696355467) bank15756 +15757 POINT(40.22805118778981 73.82060798022319) bank15757 +15758 POINT(41.062768411462436 74.42386929217871) bank15758 +15759 POINT(40.41588035975117 74.2304775596082) bank15759 +15760 POINT(39.81204556122984 73.73175308746725) bank15760 +15761 POINT(40.98277296447249 74.31707348747112) bank15761 +15762 POINT(41.018429140472044 73.6625499706398) bank15762 +15763 POINT(39.932475721838905 73.94175689476869) bank15763 +15764 POINT(39.951477104483395 74.84994821396299) bank15764 +15765 POINT(41.48015788321651 73.18369370379955) bank15765 +15766 POINT(40.85545270887986 73.33319966372841) bank15766 +15767 POINT(41.199218371613306 74.1260503794667) bank15767 +15768 POINT(41.39252855239709 74.84224660949648) bank15768 +15769 POINT(40.666403492318686 73.4762955499424) bank15769 +15770 POINT(41.445943826572304 74.41453417584536) bank15770 +15771 POINT(39.8874545692188 73.90318495721384) bank15771 +15772 POINT(40.64127455958014 74.28139514014478) bank15772 +15773 POINT(40.51978058298716 74.80341424672173) bank15773 +15774 POINT(41.33820460974328 73.93810125093579) bank15774 +15775 POINT(40.91241328630238 74.22499112167316) bank15775 +15776 POINT(40.49436647353338 74.41552712533341) bank15776 +15777 POINT(41.654459895156414 73.26801000209029) bank15777 +15778 POINT(40.29425518538425 74.81997584651279) bank15778 +15779 POINT(39.757523011380194 74.45711131264471) bank15779 +15780 POINT(40.51812524451273 74.61570762783467) bank15780 +15781 POINT(39.787826448499175 73.29280681143058) bank15781 +15782 POINT(41.43230610152486 73.0620912789103) bank15782 +15783 POINT(40.649724322335416 73.53079446144567) bank15783 +15784 POINT(39.74454574588805 74.49979965077453) bank15784 +15785 POINT(40.93769927918989 74.19801206828556) bank15785 +15786 POINT(40.07118462541018 74.97071538185016) bank15786 +15787 POINT(41.49221708638103 74.75707556598212) bank15787 +15788 POINT(40.20189419217421 73.74827523786095) bank15788 +15789 POINT(41.27596544404321 73.93448455026055) bank15789 +15790 POINT(39.767102128045174 73.91484803930835) bank15790 +15791 POINT(41.554829227551146 74.19925517298438) bank15791 +15792 POINT(39.74746763650704 73.29048514853908) bank15792 +15793 POINT(40.05121874596088 74.09210501911298) bank15793 +15794 POINT(39.87275830511593 74.53241608596623) bank15794 +15795 POINT(39.93901258579043 74.17160299746395) bank15795 +15796 POINT(39.77549253096389 73.54229517321572) bank15796 +15797 POINT(41.098176167944494 73.73985854995834) bank15797 +15798 POINT(40.64041516272708 74.23706742380958) bank15798 +15799 POINT(40.117871990948444 74.57427493469118) bank15799 +15800 POINT(40.75664662066427 74.86200398531126) bank15800 +15801 POINT(41.57529567952861 74.48842446354334) bank15801 +15802 POINT(40.520810951761945 73.52031798683397) bank15802 +15803 POINT(40.98453909521252 73.89050448286363) bank15803 +15804 POINT(40.37794811649148 74.928346987331) bank15804 +15805 POINT(41.47136284238503 73.99648963114987) bank15805 +15806 POINT(41.36457012904882 73.94558104080494) bank15806 +15807 POINT(41.4146635560345 73.38642031783684) bank15807 +15808 POINT(40.21380967306928 74.08880566032768) bank15808 +15809 POINT(41.41414969956076 74.70966957384628) bank15809 +15810 POINT(40.72825343900628 73.10475528418586) bank15810 +15811 POINT(40.246216340313936 73.8349940727948) bank15811 +15812 POINT(41.258866563491075 73.961588423905) bank15812 +15813 POINT(40.667112481631634 74.61288000111018) bank15813 +15814 POINT(41.340363200779215 73.02519310635837) bank15814 +15815 POINT(40.669096380913125 73.9155064894672) bank15815 +15816 POINT(40.54433317758284 73.62829282546622) bank15816 +15817 POINT(40.88420862617061 73.26337646376683) bank15817 +15818 POINT(39.871913380633124 74.16346548035969) bank15818 +15819 POINT(41.246602666177886 74.58626673771637) bank15819 +15820 POINT(40.88132607813961 74.62529494712243) bank15820 +15821 POINT(41.01857618420955 73.11442194539326) bank15821 +15822 POINT(39.78432772387092 74.91858600663052) bank15822 +15823 POINT(40.535134110418454 74.19414760134454) bank15823 +15824 POINT(41.64615804600079 73.11316815338569) bank15824 +15825 POINT(40.432934016915006 73.24987993814328) bank15825 +15826 POINT(39.90445713927586 73.4587533382999) bank15826 +15827 POINT(41.0820001070682 74.74129973163737) bank15827 +15828 POINT(41.37883717376093 74.20974704687058) bank15828 +15829 POINT(40.15634453795776 73.58704788090793) bank15829 +15830 POINT(40.37611139688631 74.36835091560708) bank15830 +15831 POINT(41.666794974805 74.62263534065178) bank15831 +15832 POINT(40.41643215072733 73.900930442452) bank15832 +15833 POINT(40.20550316834698 73.35397410372997) bank15833 +15834 POINT(41.37212129043307 73.95729400176695) bank15834 +15835 POINT(41.268152874695055 73.99240350905653) bank15835 +15836 POINT(40.84928185251722 73.08909801719386) bank15836 +15837 POINT(40.822759662045655 73.93956308879528) bank15837 +15838 POINT(41.67952988960602 74.8667628503006) bank15838 +15839 POINT(40.245084247340145 74.60955008409778) bank15839 +15840 POINT(39.796304528500016 73.20119090298752) bank15840 +15841 POINT(40.42734959244329 74.02738833829802) bank15841 +15842 POINT(40.443080492850825 74.70694284473764) bank15842 +15843 POINT(41.111389781527876 74.59565819781055) bank15843 +15844 POINT(40.20153640830701 73.3298536655588) bank15844 +15845 POINT(40.252622245843284 73.5389750038197) bank15845 +15846 POINT(40.19254505316843 73.46927422112974) bank15846 +15847 POINT(41.12066552200952 74.82007748645607) bank15847 +15848 POINT(40.12829094488089 73.44744575898525) bank15848 +15849 POINT(40.58885535280888 73.42206671962909) bank15849 +15850 POINT(40.47846564948444 73.65990126326294) bank15850 +15851 POINT(40.003066108336505 74.48388326902486) bank15851 +15852 POINT(40.97119068871711 73.74221241505897) bank15852 +15853 POINT(40.315737511699254 73.67762900298916) bank15853 +15854 POINT(40.86794675501928 74.44991419605593) bank15854 +15855 POINT(40.38985151173721 73.47439819761482) bank15855 +15856 POINT(40.416176332551984 74.94647126790454) bank15856 +15857 POINT(41.187358200902416 74.66945370668455) bank15857 +15858 POINT(40.16120175092547 73.76600824486248) bank15858 +15859 POINT(41.05686955465496 73.09030544290508) bank15859 +15860 POINT(41.15690323579245 73.33958334864082) bank15860 +15861 POINT(39.73737926580132 74.16963803532084) bank15861 +15862 POINT(40.317792110606526 74.08553693560118) bank15862 +15863 POINT(41.36633423658362 74.66513445364698) bank15863 +15864 POINT(40.65874928358033 74.6611089270892) bank15864 +15865 POINT(39.88167371267215 74.70795226197818) bank15865 +15866 POINT(40.09353843424043 74.83556892361042) bank15866 +15867 POINT(39.71559782443344 73.72526354501869) bank15867 +15868 POINT(41.694169488909615 73.54106404633556) bank15868 +15869 POINT(40.706264237688174 73.63134093996895) bank15869 +15870 POINT(40.11460121466163 74.33725879275875) bank15870 +15871 POINT(39.719864249799315 73.84900074219927) bank15871 +15872 POINT(39.73669741312688 74.11234427386105) bank15872 +15873 POINT(41.50709950038773 74.45177713066523) bank15873 +15874 POINT(40.1717830173738 74.8093189936268) bank15874 +15875 POINT(41.20495512357208 73.32349906578811) bank15875 +15876 POINT(39.962789192993206 74.24692099834597) bank15876 +15877 POINT(40.86167233992067 74.51488830600947) bank15877 +15878 POINT(40.15206698616889 73.26247010413692) bank15878 +15879 POINT(40.922343103016836 74.02082729489183) bank15879 +15880 POINT(39.90750215481362 74.94257795545506) bank15880 +15881 POINT(41.082944469176475 74.57002361422482) bank15881 +15882 POINT(40.451313587653075 73.44170938375997) bank15882 +15883 POINT(41.06483991689111 73.9103905389433) bank15883 +15884 POINT(39.71578369215518 74.41757348226702) bank15884 +15885 POINT(40.98927710780796 73.54706114692043) bank15885 +15886 POINT(40.72918103524771 73.25548355010102) bank15886 +15887 POINT(40.2650790351481 74.89726898308122) bank15887 +15888 POINT(40.395903445842656 74.62152579228892) bank15888 +15889 POINT(40.385074693145924 73.13878121376563) bank15889 +15890 POINT(41.52702084726225 73.99706996507359) bank15890 +15891 POINT(39.87630976176802 73.74723640886702) bank15891 +15892 POINT(41.53371151566219 74.10227811103519) bank15892 +15893 POINT(40.78634094474982 73.83993073757561) bank15893 +15894 POINT(40.774864213202584 73.52294001866092) bank15894 +15895 POINT(40.76944178763597 74.10565689067941) bank15895 +15896 POINT(41.293483635407384 74.69338794509112) bank15896 +15897 POINT(40.691625997161196 74.2843138404759) bank15897 +15898 POINT(41.07243642020798 74.4944991193712) bank15898 +15899 POINT(40.355670540630356 74.3907873565265) bank15899 +15900 POINT(40.63899346576729 73.89414707021774) bank15900 +15901 POINT(41.18022292394489 74.64751911021426) bank15901 +15902 POINT(41.022837934353134 74.15077964872576) bank15902 +15903 POINT(39.89031006856509 73.06737187665398) bank15903 +15904 POINT(39.7764727700467 73.38577277838756) bank15904 +15905 POINT(40.009750442090876 74.72361912528528) bank15905 +15906 POINT(40.70241518087407 74.1722585223886) bank15906 +15907 POINT(39.89942057635047 73.68074540688913) bank15907 +15908 POINT(40.54462422720318 73.10608216717962) bank15908 +15909 POINT(39.78725111166034 74.58313794926426) bank15909 +15910 POINT(40.5331803434736 73.0331277646972) bank15910 +15911 POINT(40.63775454322929 74.43464580678551) bank15911 +15912 POINT(41.34695399395879 73.89043137612599) bank15912 +15913 POINT(40.55899795186388 74.14844014001558) bank15913 +15914 POINT(40.95432439418153 74.56652862808343) bank15914 +15915 POINT(41.48645093724085 74.6597785001317) bank15915 +15916 POINT(40.93026584705596 73.64492880486175) bank15916 +15917 POINT(41.53167562910775 74.53028790650156) bank15917 +15918 POINT(41.30697383377502 74.4567378150023) bank15918 +15919 POINT(39.860693831233405 74.0375758228121) bank15919 +15920 POINT(40.35282853299688 73.73252214447511) bank15920 +15921 POINT(40.9689510612865 74.65716321622845) bank15921 +15922 POINT(40.26907792070843 74.6765017015658) bank15922 +15923 POINT(40.037233372499806 74.31815940412646) bank15923 +15924 POINT(40.228779585691676 74.72901432263089) bank15924 +15925 POINT(40.4617028474571 74.86912631390327) bank15925 +15926 POINT(41.20542001721984 73.18689492303963) bank15926 +15927 POINT(40.610064669211184 74.23161640050657) bank15927 +15928 POINT(41.707581326985526 74.99149776385347) bank15928 +15929 POINT(41.52074313154034 74.59377304504135) bank15929 +15930 POINT(40.14124492673294 74.54559545676204) bank15930 +15931 POINT(41.41416999771274 73.40828019858537) bank15931 +15932 POINT(40.92120786055057 74.02738829659266) bank15932 +15933 POINT(41.45277721592268 73.29487176500213) bank15933 +15934 POINT(40.682337152177155 74.51740770262515) bank15934 +15935 POINT(40.361984513618715 73.91804655244422) bank15935 +15936 POINT(40.13392838028384 74.86124716419975) bank15936 +15937 POINT(41.5944785965908 74.81170891994174) bank15937 +15938 POINT(40.85336533838698 74.84806473240795) bank15938 +15939 POINT(40.514082293229436 74.4588897602863) bank15939 +15940 POINT(40.24715807287935 74.52122065209898) bank15940 +15941 POINT(40.995577627719214 73.25757159643845) bank15941 +15942 POINT(40.23199218082708 74.92760355861034) bank15942 +15943 POINT(40.39898180189938 73.02515844583033) bank15943 +15944 POINT(40.57030298877191 73.8953397438385) bank15944 +15945 POINT(39.799515440265935 73.30689068722856) bank15945 +15946 POINT(40.76754758283289 73.78671891470061) bank15946 +15947 POINT(40.44869859557984 74.77836753104835) bank15947 +15948 POINT(40.87147088454646 74.1190357144198) bank15948 +15949 POINT(41.12501155482053 73.34082905117879) bank15949 +15950 POINT(41.58537061230884 73.4991955775111) bank15950 +15951 POINT(40.55937214389144 74.30575362417471) bank15951 +15952 POINT(41.1463080028472 73.34658777164768) bank15952 +15953 POINT(41.44885385003339 74.0685410495009) bank15953 +15954 POINT(41.571126248136714 73.34410389963442) bank15954 +15955 POINT(41.29662539569905 74.50063428411299) bank15955 +15956 POINT(39.99144735735338 73.19754961794312) bank15956 +15957 POINT(39.716656634177795 74.70700353438124) bank15957 +15958 POINT(40.339710144584025 74.83964071288439) bank15958 +15959 POINT(39.74869105880159 74.83183191171211) bank15959 +15960 POINT(41.50964017175161 74.25388431138468) bank15960 +15961 POINT(41.112534938871626 74.4333008256737) bank15961 +15962 POINT(40.632533350847886 73.70728707896949) bank15962 +15963 POINT(40.08080424259077 74.1451255939279) bank15963 +15964 POINT(40.91822274692827 73.9973345990975) bank15964 +15965 POINT(41.56904876522629 74.31292184492807) bank15965 +15966 POINT(41.29615021506593 73.25897403755893) bank15966 +15967 POINT(39.943078917154494 74.1110894570755) bank15967 +15968 POINT(40.807941091428944 74.38632624797097) bank15968 +15969 POINT(41.33967003159894 74.94543931967836) bank15969 +15970 POINT(41.48298180689436 74.42329207958703) bank15970 +15971 POINT(40.78833340734013 74.81311912131291) bank15971 +15972 POINT(41.12046364015845 74.66598133768099) bank15972 +15973 POINT(41.04010071514863 74.14987850837032) bank15973 +15974 POINT(39.83547057178303 73.50154396715212) bank15974 +15975 POINT(41.01291424641682 74.56657699472674) bank15975 +15976 POINT(40.72606467573457 73.12931236621394) bank15976 +15977 POINT(40.353043115816035 73.6313039688119) bank15977 +15978 POINT(40.16902693185642 74.58483522033984) bank15978 +15979 POINT(40.835724401250594 74.49572461395435) bank15979 +15980 POINT(41.398427329453405 73.3507305056456) bank15980 +15981 POINT(41.1521010426663 73.17371816754157) bank15981 +15982 POINT(40.762343849279 73.29130025402766) bank15982 +15983 POINT(41.601410099107696 73.18737698953262) bank15983 +15984 POINT(41.637280332897525 74.4591336480278) bank15984 +15985 POINT(41.65377590314981 73.46896577211253) bank15985 +15986 POINT(41.621232190025175 74.38837563396604) bank15986 +15987 POINT(39.87539617471697 73.90994582333713) bank15987 +15988 POINT(40.06447920501489 74.85320150622904) bank15988 +15989 POINT(40.888425237122775 73.98739370585362) bank15989 +15990 POINT(40.260367498700475 73.94067068351679) bank15990 +15991 POINT(39.90608877397965 74.27125626170054) bank15991 +15992 POINT(41.34802218046016 74.18543865529206) bank15992 +15993 POINT(41.26287555320811 73.19976318593915) bank15993 +15994 POINT(40.62305162911783 73.47053793803748) bank15994 +15995 POINT(41.23792955765753 74.20181857666077) bank15995 +15996 POINT(41.59116968705031 74.51488226525554) bank15996 +15997 POINT(41.48683767440018 74.40568519620373) bank15997 +15998 POINT(41.31399423922324 74.01582150070924) bank15998 +15999 POINT(40.14625385973668 73.25915920429722) bank15999 +16000 POINT(41.19874917538124 74.42816539573501) bank16000 +16001 POINT(40.15687746922865 73.5025179919633) bank16001 +16002 POINT(41.16095067622464 73.61147080169124) bank16002 +16003 POINT(41.35854162591719 73.44690443225015) bank16003 +16004 POINT(40.532228995470895 74.42573142270699) bank16004 +16005 POINT(40.402599924163844 73.22904446806308) bank16005 +16006 POINT(39.839316767512514 73.51541207853666) bank16006 +16007 POINT(39.89972600315051 74.47621814659709) bank16007 +16008 POINT(40.487884218355596 73.8226193806066) bank16008 +16009 POINT(40.993746152796774 74.42297862108761) bank16009 +16010 POINT(39.782184742685615 73.70953246503291) bank16010 +16011 POINT(39.86617755165683 73.11623532669402) bank16011 +16012 POINT(40.18276362743255 74.02127564108805) bank16012 +16013 POINT(41.634724929061235 73.67421555613984) bank16013 +16014 POINT(40.64753864650428 74.10229327588665) bank16014 +16015 POINT(40.19246324935545 73.43076130153779) bank16015 +16016 POINT(40.30786502765473 73.23540353868171) bank16016 +16017 POINT(40.49183624615948 73.58727490436767) bank16017 +16018 POINT(41.00320092196917 73.88298720544798) bank16018 +16019 POINT(41.161719238126544 73.5965893295871) bank16019 +16020 POINT(40.6957351901449 74.97316327448289) bank16020 +16021 POINT(41.134925692813596 74.23277186696116) bank16021 +16022 POINT(41.27765631403318 73.58423875288939) bank16022 +16023 POINT(40.218152604723834 73.37346685193089) bank16023 +16024 POINT(39.92815596505066 73.92511649104304) bank16024 +16025 POINT(40.97863970845881 73.28784232015919) bank16025 +16026 POINT(40.92039563644539 73.99350096997367) bank16026 +16027 POINT(40.554782066138955 74.47010959081715) bank16027 +16028 POINT(41.23116549840738 74.72553984331624) bank16028 +16029 POINT(40.733218693068295 73.52682490420825) bank16029 +16030 POINT(40.361852482389246 74.54680401499809) bank16030 +16031 POINT(41.63040384508497 73.20886516298117) bank16031 +16032 POINT(40.606867224478435 73.93161030206974) bank16032 +16033 POINT(40.174180307922576 74.24253903607413) bank16033 +16034 POINT(41.416475775971676 74.928069829617) bank16034 +16035 POINT(41.36772124156268 74.5171105094616) bank16035 +16036 POINT(41.099452199775534 73.08075920244491) bank16036 +16037 POINT(41.36326884135912 74.65869576764247) bank16037 +16038 POINT(39.99854130782689 73.55660351839413) bank16038 +16039 POINT(40.14051486729491 73.90499875872189) bank16039 +16040 POINT(39.85644118394205 73.4322165931399) bank16040 +16041 POINT(39.94878281315207 73.6965301838498) bank16041 +16042 POINT(41.633190101741036 74.65822120164148) bank16042 +16043 POINT(40.26712870749044 74.37362973412576) bank16043 +16044 POINT(41.05253828569263 74.61836572759613) bank16044 +16045 POINT(40.86197937109474 73.36202694757144) bank16045 +16046 POINT(41.25378891210855 73.93788776004686) bank16046 +16047 POINT(40.62125108926878 74.38556186360027) bank16047 +16048 POINT(41.05498501859748 73.69782963157321) bank16048 +16049 POINT(39.81396861366392 73.7501505319497) bank16049 +16050 POINT(41.69004730231809 74.90875343374474) bank16050 +16051 POINT(40.27175806576049 74.70687259152663) bank16051 +16052 POINT(41.537928566301176 74.93891328156178) bank16052 +16053 POINT(40.00941771438949 74.30844020999237) bank16053 +16054 POINT(40.16844645777481 74.42258182026869) bank16054 +16055 POINT(39.747590986090145 74.58280621778391) bank16055 +16056 POINT(41.13113035356432 73.32526382075169) bank16056 +16057 POINT(40.93496697573668 73.75837000138765) bank16057 +16058 POINT(40.77154418576526 74.87109272412579) bank16058 +16059 POINT(41.36506688674832 74.16043464677605) bank16059 +16060 POINT(40.99354399932126 73.64198231142909) bank16060 +16061 POINT(40.93945649713281 73.83372075221254) bank16061 +16062 POINT(39.74767582591569 74.28785832678096) bank16062 +16063 POINT(41.072577955590056 73.91259861678247) bank16063 +16064 POINT(39.936137293925576 74.28928215636783) bank16064 +16065 POINT(39.890351877717414 74.18799731326294) bank16065 +16066 POINT(40.32553980654054 73.05503449601741) bank16066 +16067 POINT(41.082734342265994 73.34935711666819) bank16067 +16068 POINT(40.09172091646888 74.33595452623541) bank16068 +16069 POINT(40.91342233911067 74.6602786518156) bank16069 +16070 POINT(39.942905805691 73.46987869892324) bank16070 +16071 POINT(41.52818638880117 74.64341040453803) bank16071 +16072 POINT(40.34700299723674 73.49878556049268) bank16072 +16073 POINT(40.468192220350275 73.65195933102972) bank16073 +16074 POINT(39.87234348260088 74.53937949192667) bank16074 +16075 POINT(41.44131222453611 73.80378865191109) bank16075 +16076 POINT(39.925424144437535 74.68239439654316) bank16076 +16077 POINT(40.43805261929861 73.9757326720282) bank16077 +16078 POINT(40.67925098007628 73.43336335995224) bank16078 +16079 POINT(40.67263536854409 73.5613848255187) bank16079 +16080 POINT(41.16989393293235 74.19398119699127) bank16080 +16081 POINT(41.34535865470835 74.7685170834584) bank16081 +16082 POINT(40.916153743688724 74.91785354688015) bank16082 +16083 POINT(40.84923299971859 74.3968345551781) bank16083 +16084 POINT(40.72314891078003 73.27188106703932) bank16084 +16085 POINT(40.06578072311129 73.49891174864293) bank16085 +16086 POINT(40.27368789558696 73.36276512197831) bank16086 +16087 POINT(41.66545862838436 74.58450777151128) bank16087 +16088 POINT(41.158031841043595 73.84913543481456) bank16088 +16089 POINT(40.2271032611723 74.34232564658168) bank16089 +16090 POINT(41.526086328055705 73.64639474322546) bank16090 +16091 POINT(40.58108599634457 73.33665239453417) bank16091 +16092 POINT(40.58637574446096 74.34973960226169) bank16092 +16093 POINT(41.53495139453008 74.59663750153773) bank16093 +16094 POINT(40.48380403498162 74.4363624019453) bank16094 +16095 POINT(40.443406322155646 73.81812820450216) bank16095 +16096 POINT(40.94563858095372 74.055718260451) bank16096 +16097 POINT(41.33843372450827 74.27678733729618) bank16097 +16098 POINT(39.73568843608026 74.00129665510558) bank16098 +16099 POINT(41.59277654686928 73.63292269596388) bank16099 +16100 POINT(40.537183703795236 74.66932204903063) bank16100 +16101 POINT(40.301395966441234 74.71409570329313) bank16101 +16102 POINT(40.89400780661801 74.95800578286415) bank16102 +16103 POINT(39.99603253053199 73.97223235839219) bank16103 +16104 POINT(41.66525828911083 74.41295450483413) bank16104 +16105 POINT(41.20819883112419 73.03663628130742) bank16105 +16106 POINT(41.38924918279087 73.0742877391856) bank16106 +16107 POINT(40.181967138419 74.98053369773375) bank16107 +16108 POINT(41.6407099757893 73.23547106668639) bank16108 +16109 POINT(41.17051378467226 73.80174124701182) bank16109 +16110 POINT(41.09291634382326 74.44518564716317) bank16110 +16111 POINT(41.12982033365467 73.24226330226989) bank16111 +16112 POINT(40.30270834076292 73.95673661160319) bank16112 +16113 POINT(41.45253640244454 74.07930920354363) bank16113 +16114 POINT(41.45776830476406 74.3645431187977) bank16114 +16115 POINT(41.54441238290292 73.26722621467533) bank16115 +16116 POINT(41.0107942771814 73.98063044319133) bank16116 +16117 POINT(41.06010402761079 73.20247350851264) bank16117 +16118 POINT(39.896687604184955 73.2901106566901) bank16118 +16119 POINT(40.92137677883931 74.27718053086367) bank16119 +16120 POINT(40.68380629692645 73.37489018738009) bank16120 +16121 POINT(40.840208277434115 74.12948317470568) bank16121 +16122 POINT(41.70313422335446 74.94121102313356) bank16122 +16123 POINT(40.9778388254584 73.50325983394801) bank16123 +16124 POINT(41.21675980515146 74.30261282704362) bank16124 +16125 POINT(41.012072520483954 73.88782560635569) bank16125 +16126 POINT(41.402564933505495 74.10295743237795) bank16126 +16127 POINT(40.82850344688909 73.99665079506636) bank16127 +16128 POINT(40.93129120377855 73.0104599085801) bank16128 +16129 POINT(41.35062032531908 74.5116018043366) bank16129 +16130 POINT(40.945924632217675 74.93814070416265) bank16130 +16131 POINT(40.51029200035081 73.39684329962897) bank16131 +16132 POINT(39.75407058210309 74.8915445402294) bank16132 +16133 POINT(40.96672876714194 73.8405179508358) bank16133 +16134 POINT(40.72553746888824 73.50791826038694) bank16134 +16135 POINT(40.94556555872633 74.84246308067753) bank16135 +16136 POINT(40.99278858126054 74.74184105189717) bank16136 +16137 POINT(40.271972890826156 73.53751825284748) bank16137 +16138 POINT(39.87645702969974 74.71056502185222) bank16138 +16139 POINT(40.69244388688148 74.62476039288174) bank16139 +16140 POINT(41.070458979458394 73.68793969472269) bank16140 +16141 POINT(41.40019399119309 74.17269665087889) bank16141 +16142 POINT(40.295889364363845 73.22268610823004) bank16142 +16143 POINT(40.45755932556742 74.78910105118652) bank16143 +16144 POINT(40.89669271805904 74.83308509528497) bank16144 +16145 POINT(39.862973858489774 74.35095724013598) bank16145 +16146 POINT(41.59997658109183 73.1945133298253) bank16146 +16147 POINT(40.84705284503396 74.8137253060912) bank16147 +16148 POINT(41.5760598565797 73.45171251396799) bank16148 +16149 POINT(41.525573502278476 74.22268854740811) bank16149 +16150 POINT(40.07524080723572 74.04121607127159) bank16150 +16151 POINT(41.03895098975516 73.64437106498676) bank16151 +16152 POINT(39.730072651114824 73.02508158708204) bank16152 +16153 POINT(40.15930375993258 73.73606856380233) bank16153 +16154 POINT(40.45386050000663 74.22389190136042) bank16154 +16155 POINT(40.88939038168214 74.48761150465077) bank16155 +16156 POINT(40.483423884171245 73.02035985641325) bank16156 +16157 POINT(41.26340082220057 73.93076274899394) bank16157 +16158 POINT(40.39854046258706 74.71463737177302) bank16158 +16159 POINT(40.781391438193964 73.99590790282559) bank16159 +16160 POINT(40.02122686261733 73.41891273471737) bank16160 +16161 POINT(41.19120981965741 74.16172879173797) bank16161 +16162 POINT(39.760295671486155 73.56162112704479) bank16162 +16163 POINT(41.54469203161198 73.30729096993005) bank16163 +16164 POINT(40.88185822466164 73.9598170472685) bank16164 +16165 POINT(41.38881917379992 73.03951922361024) bank16165 +16166 POINT(41.26026219435527 73.17160330245687) bank16166 +16167 POINT(41.61820485254163 74.49598183299857) bank16167 +16168 POINT(41.5729103000476 73.49284233627735) bank16168 +16169 POINT(40.506872900649334 74.29384557601604) bank16169 +16170 POINT(41.01914936263411 74.8959064674323) bank16170 +16171 POINT(39.82099313166176 73.63152108577074) bank16171 +16172 POINT(41.64005097561361 74.78214469280444) bank16172 +16173 POINT(41.611416034933335 73.35554075702117) bank16173 +16174 POINT(41.7126450770215 73.48874053813682) bank16174 +16175 POINT(39.75794715386662 73.12567991353914) bank16175 +16176 POINT(40.126814513565755 74.11486533503647) bank16176 +16177 POINT(41.11852287324718 73.69115316924996) bank16177 +16178 POINT(40.95993143389621 74.8478386601641) bank16178 +16179 POINT(40.671666443630706 73.9920881828664) bank16179 +16180 POINT(41.541437775978864 74.23090652548169) bank16180 +16181 POINT(41.24623996935352 74.13875992722845) bank16181 +16182 POINT(41.61998400256609 73.38369431188661) bank16182 +16183 POINT(39.83481007653623 74.59754984592728) bank16183 +16184 POINT(41.044967736217046 73.42754931574531) bank16184 +16185 POINT(40.157848579426485 74.0332463715475) bank16185 +16186 POINT(41.25535306895411 73.9756915269661) bank16186 +16187 POINT(41.53425868900942 73.91909656388667) bank16187 +16188 POINT(40.51067239380943 74.60436244459254) bank16188 +16189 POINT(40.227278107951804 73.135575136115) bank16189 +16190 POINT(40.08671635772474 74.46559436150002) bank16190 +16191 POINT(41.32359270053249 74.77068903112523) bank16191 +16192 POINT(39.82891918877151 74.80637308409578) bank16192 +16193 POINT(41.36481790635393 73.1996604379993) bank16193 +16194 POINT(41.70568554233886 74.08006024478144) bank16194 +16195 POINT(39.8812129812381 74.32739742253415) bank16195 +16196 POINT(41.07388534491785 74.1007201358828) bank16196 +16197 POINT(39.72968872818389 73.018430310725) bank16197 +16198 POINT(40.93418376766069 74.57636875594235) bank16198 +16199 POINT(40.89048272890736 73.27971416777547) bank16199 +16200 POINT(41.27195601439435 73.80508357273246) bank16200 +16201 POINT(40.4131645365117 74.23678993219617) bank16201 +16202 POINT(40.94309814138074 73.19341081590551) bank16202 +16203 POINT(41.12069203905269 73.44341633682885) bank16203 +16204 POINT(41.526517628387865 73.37636434023953) bank16204 +16205 POINT(40.13035018303591 74.77389436151341) bank16205 +16206 POINT(41.15418284751693 74.51498776101299) bank16206 +16207 POINT(40.74153771687762 73.57439726912465) bank16207 +16208 POINT(41.70997243409715 73.77989569067213) bank16208 +16209 POINT(40.74342783466048 73.65087867167695) bank16209 +16210 POINT(41.48636407416128 73.69671283455375) bank16210 +16211 POINT(40.07128186301788 73.71219024072388) bank16211 +16212 POINT(40.92267589543325 74.21438055390685) bank16212 +16213 POINT(40.881614861070375 73.51490940294464) bank16213 +16214 POINT(40.37817846436091 73.62121064864418) bank16214 +16215 POINT(41.134633051667485 73.58260627613166) bank16215 +16216 POINT(40.58001658971403 74.95140469517922) bank16216 +16217 POINT(40.889498689812875 73.44537416469959) bank16217 +16218 POINT(40.72034353680914 74.33321643702332) bank16218 +16219 POINT(41.13770457285308 74.97162577700395) bank16219 +16220 POINT(40.94379628345273 74.41873145426285) bank16220 +16221 POINT(40.757149657424605 74.23792799465724) bank16221 +16222 POINT(41.59881711904016 74.32341079607139) bank16222 +16223 POINT(40.19107111639389 73.05237814919893) bank16223 +16224 POINT(41.23185053405398 73.05729611107277) bank16224 +16225 POINT(40.88309850553061 74.06550081008335) bank16225 +16226 POINT(39.833609439857554 74.75729757317194) bank16226 +16227 POINT(40.9453967338177 74.0530009800271) bank16227 +16228 POINT(40.87853873787857 73.82852560661114) bank16228 +16229 POINT(40.57183553323585 73.00921715506398) bank16229 +16230 POINT(40.481736351740665 73.68676587800623) bank16230 +16231 POINT(41.564761000502976 73.00999631016032) bank16231 +16232 POINT(40.98284972174299 73.6475986465658) bank16232 +16233 POINT(40.41317723013495 73.0553223171775) bank16233 +16234 POINT(39.76363924751813 73.13952544502342) bank16234 +16235 POINT(40.90085525589507 74.10109835729811) bank16235 +16236 POINT(40.81954298597612 74.36703387505496) bank16236 +16237 POINT(39.984399583190566 73.08324458862829) bank16237 +16238 POINT(40.425697989560064 73.25888651544248) bank16238 +16239 POINT(40.135318274189665 73.42990285048857) bank16239 +16240 POINT(40.00417750481205 73.94253922269986) bank16240 +16241 POINT(39.74774000730268 74.48963580684834) bank16241 +16242 POINT(41.63323645708132 74.72846912721396) bank16242 +16243 POINT(40.55729881851644 73.54214536573095) bank16243 +16244 POINT(41.26547410055798 73.73368462677621) bank16244 +16245 POINT(41.651441045101514 74.82761983202661) bank16245 +16246 POINT(40.13733235852807 73.01228972591724) bank16246 +16247 POINT(41.36962749717305 74.98789593193649) bank16247 +16248 POINT(40.17068312856902 73.84069726991768) bank16248 +16249 POINT(41.32231625780339 74.5175701972783) bank16249 +16250 POINT(41.28919155013753 74.49086581816928) bank16250 +16251 POINT(41.23779067632587 74.43520615417611) bank16251 +16252 POINT(40.4762553736021 74.88689078468657) bank16252 +16253 POINT(39.94051873063525 74.16224124144432) bank16253 +16254 POINT(40.111728130479946 74.34003440009552) bank16254 +16255 POINT(40.87939335716175 73.35420589092023) bank16255 +16256 POINT(40.36662088448367 74.61912573089919) bank16256 +16257 POINT(40.845638735680105 74.97050775276031) bank16257 +16258 POINT(40.732163419136505 74.77523061075021) bank16258 +16259 POINT(41.03963303228672 73.4133311455592) bank16259 +16260 POINT(41.686447445164916 74.5166087468598) bank16260 +16261 POINT(39.85889594990895 74.14836538674739) bank16261 +16262 POINT(41.34390213071673 73.91254133896847) bank16262 +16263 POINT(40.51322348864703 74.98811746454004) bank16263 +16264 POINT(40.46185674822136 74.91256407751976) bank16264 +16265 POINT(39.81674957762204 73.45519114910846) bank16265 +16266 POINT(41.02809492559409 73.24436237306047) bank16266 +16267 POINT(39.79330478666977 74.91961568979896) bank16267 +16268 POINT(40.54122725604095 74.54753759665188) bank16268 +16269 POINT(41.180663634563665 73.89092328313168) bank16269 +16270 POINT(40.9940807091863 74.79109827379634) bank16270 +16271 POINT(39.8278749418834 74.6551318911535) bank16271 +16272 POINT(40.90787647479625 73.26911016896989) bank16272 +16273 POINT(40.23994347612493 73.04439633529994) bank16273 +16274 POINT(41.584932562992165 74.61188046498027) bank16274 +16275 POINT(41.50619940076867 74.15700555121295) bank16275 +16276 POINT(39.812279863819015 73.90797699415509) bank16276 +16277 POINT(41.08248421197609 73.76800104934479) bank16277 +16278 POINT(40.469474813488475 74.52894122447022) bank16278 +16279 POINT(41.11549654642525 73.8794577658641) bank16279 +16280 POINT(39.75709089403009 74.93999902387353) bank16280 +16281 POINT(40.890428376230666 73.7346973430234) bank16281 +16282 POINT(39.93274695083643 74.3270177893348) bank16282 +16283 POINT(41.25029756042401 74.97252683196001) bank16283 +16284 POINT(40.31592763233562 73.26253615492679) bank16284 +16285 POINT(40.45728892380588 74.51728714099703) bank16285 +16286 POINT(39.77787714236047 74.2871370839601) bank16286 +16287 POINT(41.49448807102308 73.72538010173123) bank16287 +16288 POINT(40.17515259855282 74.78444567107005) bank16288 +16289 POINT(40.857643079175894 73.22849607678081) bank16289 +16290 POINT(40.391439545993975 74.72545219927179) bank16290 +16291 POINT(41.56238452157119 74.60190865743677) bank16291 +16292 POINT(40.7522225847869 73.85668131305951) bank16292 +16293 POINT(41.5287311367254 74.49480045159147) bank16293 +16294 POINT(40.27375604161533 73.9435020391158) bank16294 +16295 POINT(39.882649943520356 74.91624945891293) bank16295 +16296 POINT(40.251328531590815 73.35681376854325) bank16296 +16297 POINT(41.20694556873102 74.19543930403896) bank16297 +16298 POINT(41.18628870138835 73.72863307696983) bank16298 +16299 POINT(41.29363476911687 74.7374109652051) bank16299 +16300 POINT(41.56760391480327 73.75967836271907) bank16300 +16301 POINT(40.772062003580636 74.35843758463598) bank16301 +16302 POINT(41.00433974143189 74.45529964513162) bank16302 +16303 POINT(40.19882400042331 74.2439420595055) bank16303 +16304 POINT(39.92331752408946 73.50019375009764) bank16304 +16305 POINT(40.126057732679755 73.43851509789171) bank16305 +16306 POINT(40.77615686837726 74.8188375960044) bank16306 +16307 POINT(39.77026374614975 74.70152897840379) bank16307 +16308 POINT(41.42129639778638 75.00353015385926) bank16308 +16309 POINT(41.26464781370647 73.66558562241491) bank16309 +16310 POINT(40.2659016620039 73.63642922436277) bank16310 +16311 POINT(40.96807976388243 73.16008571179327) bank16311 +16312 POINT(39.85567162463392 73.6340233463731) bank16312 +16313 POINT(40.23389986753777 74.56753572360499) bank16313 +16314 POINT(40.95847274204375 73.16891769283258) bank16314 +16315 POINT(40.12611797336704 74.11488624672899) bank16315 +16316 POINT(41.64194937915763 74.79567373357241) bank16316 +16317 POINT(39.863013683845224 73.70232553833866) bank16317 +16318 POINT(40.24304486154403 74.31126619828076) bank16318 +16319 POINT(41.27333929592575 74.56812201541638) bank16319 +16320 POINT(40.236057429971886 74.07373046216748) bank16320 +16321 POINT(41.09900471735034 73.25423978041735) bank16321 +16322 POINT(40.94207760550562 73.39309329160557) bank16322 +16323 POINT(40.91002843933898 73.49343222685167) bank16323 +16324 POINT(40.48994927602003 73.38312890736964) bank16324 +16325 POINT(40.8463008831375 73.27121056792723) bank16325 +16326 POINT(41.62643984243362 73.994273498329) bank16326 +16327 POINT(41.05010977528857 74.73832110072613) bank16327 +16328 POINT(40.61054158044434 73.38918801914376) bank16328 +16329 POINT(40.04801932046829 74.32404420102263) bank16329 +16330 POINT(41.05333098566531 73.87206498885851) bank16330 +16331 POINT(40.30251843961779 74.52283836824643) bank16331 +16332 POINT(41.44907976845163 74.38858519223618) bank16332 +16333 POINT(41.23148508336033 74.4461034712811) bank16333 +16334 POINT(40.46356877066062 73.9396724182282) bank16334 +16335 POINT(39.96973439916394 73.08982422294349) bank16335 +16336 POINT(41.665615949575816 74.60502320329593) bank16336 +16337 POINT(41.284226895613905 73.50214192687902) bank16337 +16338 POINT(39.95938302302201 74.84685308083186) bank16338 +16339 POINT(41.313351485924755 74.76775447230949) bank16339 +16340 POINT(39.89943790770882 73.45467937914528) bank16340 +16341 POINT(40.86557808296177 73.48494373617545) bank16341 +16342 POINT(41.01224603202052 73.05963362794841) bank16342 +16343 POINT(39.77444971942865 73.02902193637259) bank16343 +16344 POINT(41.166405631814534 73.15574340062143) bank16344 +16345 POINT(41.313260608536346 74.46814846851552) bank16345 +16346 POINT(41.61810682424702 73.69468831192135) bank16346 +16347 POINT(40.7301674674787 73.11383685142884) bank16347 +16348 POINT(41.11292520752373 74.06181876500159) bank16348 +16349 POINT(40.7410380102264 73.50798249920643) bank16349 +16350 POINT(41.65300414061757 73.30280859031117) bank16350 +16351 POINT(40.47281537888646 74.94234740004217) bank16351 +16352 POINT(41.51457707882408 73.92663077414058) bank16352 +16353 POINT(41.182206974905895 74.15422009856678) bank16353 +16354 POINT(40.10555089524335 74.0795883450711) bank16354 +16355 POINT(39.80832509761891 73.06496817023049) bank16355 +16356 POINT(40.7057880387038 74.19000458625763) bank16356 +16357 POINT(41.66614913561655 74.565214811458) bank16357 +16358 POINT(41.676648494878435 74.80877972012088) bank16358 +16359 POINT(39.82895676290457 73.78941308825684) bank16359 +16360 POINT(40.552321601299376 73.99043860886034) bank16360 +16361 POINT(40.82826530134531 74.03988509587293) bank16361 +16362 POINT(40.27191260185617 74.80468526768846) bank16362 +16363 POINT(39.83316226081867 73.37993637693206) bank16363 +16364 POINT(41.50075432670905 74.93213257061116) bank16364 +16365 POINT(40.05626264322078 73.72370005052979) bank16365 +16366 POINT(41.081334138330135 73.72102423106895) bank16366 +16367 POINT(41.22232264628187 73.70970351907884) bank16367 +16368 POINT(40.42256784728778 74.20730441349903) bank16368 +16369 POINT(41.48517809909515 74.18548123661176) bank16369 +16370 POINT(39.9423874355849 73.15059766835374) bank16370 +16371 POINT(40.186965340547545 74.9202487479359) bank16371 +16372 POINT(40.331212507884366 73.60811283693567) bank16372 +16373 POINT(40.14125322660169 74.13960200447795) bank16373 +16374 POINT(41.430468222702736 73.47178270772949) bank16374 +16375 POINT(41.49440204752932 73.71845841198531) bank16375 +16376 POINT(39.906621650828825 74.23079942557939) bank16376 +16377 POINT(40.65281337763857 74.02844962516775) bank16377 +16378 POINT(40.42267796093756 74.1770689342473) bank16378 +16379 POINT(40.93076966720149 73.72202560687045) bank16379 +16380 POINT(40.97396926607371 73.36154597307629) bank16380 +16381 POINT(40.62180364651689 73.19647398748174) bank16381 +16382 POINT(40.94675093493852 73.15484731326916) bank16382 +16383 POINT(40.407748206306245 74.29618054238038) bank16383 +16384 POINT(40.06992601116847 73.51245145274306) bank16384 +16385 POINT(40.86675342309119 73.57410192641211) bank16385 +16386 POINT(40.571029239492084 74.78312740955516) bank16386 +16387 POINT(41.10728524487145 73.59424009155367) bank16387 +16388 POINT(40.76238319766349 74.22350884343548) bank16388 +16389 POINT(40.0369987442317 73.69681502776609) bank16389 +16390 POINT(39.77154245257067 74.52022871403094) bank16390 +16391 POINT(40.158759360217786 73.59187299597362) bank16391 +16392 POINT(40.44093336179951 74.79019194072306) bank16392 +16393 POINT(40.818048865607196 74.70773342370371) bank16393 +16394 POINT(40.95335119990356 73.76442288587116) bank16394 +16395 POINT(40.58813874555617 73.54081058419813) bank16395 +16396 POINT(41.6373919920927 74.86051455761677) bank16396 +16397 POINT(41.121515730674574 74.33974625261895) bank16397 +16398 POINT(39.98547195598794 74.40751704861577) bank16398 +16399 POINT(40.48013762309766 73.61349488589356) bank16399 +16400 POINT(41.37421172938384 73.53764951635375) bank16400 +16401 POINT(40.79817832461102 74.89781620297093) bank16401 +16402 POINT(40.290877434491144 74.11462177776306) bank16402 +16403 POINT(40.54429427448282 74.61728603935848) bank16403 +16404 POINT(40.41302111132994 74.32645400469121) bank16404 +16405 POINT(40.44917678443134 73.09423030973167) bank16405 +16406 POINT(41.555446253074 73.62094082501741) bank16406 +16407 POINT(40.17416526584234 73.3001373813121) bank16407 +16408 POINT(41.16052992632281 74.42975795900045) bank16408 +16409 POINT(40.11595763805532 74.42107430454814) bank16409 +16410 POINT(40.44196419373346 74.43399040150484) bank16410 +16411 POINT(40.156882794092496 73.84324767315054) bank16411 +16412 POINT(41.695059690368026 73.07506003664875) bank16412 +16413 POINT(40.855385268153945 73.90723551428606) bank16413 +16414 POINT(41.34996846256291 74.09363435790038) bank16414 +16415 POINT(40.75356547811789 73.35650582002206) bank16415 +16416 POINT(40.99709347720386 73.32872537557324) bank16416 +16417 POINT(41.48550849465272 73.03936585173577) bank16417 +16418 POINT(40.80194111705062 74.41509570307092) bank16418 +16419 POINT(41.62429775248579 74.38045552483692) bank16419 +16420 POINT(40.520170450517654 74.13098854646285) bank16420 +16421 POINT(40.133837159780335 73.55382942208749) bank16421 +16422 POINT(41.603587794728234 74.44373774915114) bank16422 +16423 POINT(39.94378747006437 73.66744831548982) bank16423 +16424 POINT(39.75007769290641 74.56366767325048) bank16424 +16425 POINT(40.01520769498685 74.44408555876122) bank16425 +16426 POINT(40.670303995825904 73.54902105921742) bank16426 +16427 POINT(40.39717077641982 73.16871140772922) bank16427 +16428 POINT(41.00002919801521 74.59814778282393) bank16428 +16429 POINT(41.42188841426987 74.50489354727355) bank16429 +16430 POINT(40.54926530640486 74.60684287277759) bank16430 +16431 POINT(40.698133707984454 74.24326935507406) bank16431 +16432 POINT(40.85452485322674 73.04598754147348) bank16432 +16433 POINT(41.43864070550636 73.5411825525315) bank16433 +16434 POINT(41.01951009038804 73.47585338275385) bank16434 +16435 POINT(39.72271069579809 74.97824640143496) bank16435 +16436 POINT(40.67788369303268 74.85995507590872) bank16436 +16437 POINT(41.023802931383884 73.21234281921802) bank16437 +16438 POINT(41.319913618280005 73.49528000751717) bank16438 +16439 POINT(41.46710979000842 74.50860969364385) bank16439 +16440 POINT(41.367281382026235 74.30761239244919) bank16440 +16441 POINT(41.463197372282174 74.83605132217468) bank16441 +16442 POINT(41.36865749359287 74.38062721732508) bank16442 +16443 POINT(40.02489782246141 73.50632477023828) bank16443 +16444 POINT(40.36111571847452 73.33833960492866) bank16444 +16445 POINT(40.11198962508307 73.10454458342932) bank16445 +16446 POINT(41.553626364882916 73.30399504729401) bank16446 +16447 POINT(41.42599881726707 73.63416728449697) bank16447 +16448 POINT(41.4559008107819 73.39568110984153) bank16448 +16449 POINT(40.70567916615748 73.83787371010166) bank16449 +16450 POINT(40.338727698129325 73.44163097535592) bank16450 +16451 POINT(40.903416395891455 73.2443497142689) bank16451 +16452 POINT(41.03236878094029 73.707797736242) bank16452 +16453 POINT(41.537007742833 73.15651971687795) bank16453 +16454 POINT(41.388495423642 74.40940852778698) bank16454 +16455 POINT(40.971590063435166 74.20732821717799) bank16455 +16456 POINT(39.95582002610221 73.4696841346582) bank16456 +16457 POINT(41.012694248194734 73.51108914085944) bank16457 +16458 POINT(39.792704545665764 74.65859283777529) bank16458 +16459 POINT(40.54837018786054 73.65216927215381) bank16459 +16460 POINT(40.07675841475667 74.19201611052694) bank16460 +16461 POINT(39.95031154344224 73.84662715150588) bank16461 +16462 POINT(39.822819582680275 73.90768664569575) bank16462 +16463 POINT(40.87725931245732 74.14984923344772) bank16463 +16464 POINT(40.74683925153813 74.29094453950448) bank16464 +16465 POINT(40.94263691699365 73.6801426945867) bank16465 +16466 POINT(40.33314677806773 73.89701503394717) bank16466 +16467 POINT(40.44293640144672 74.51154525557975) bank16467 +16468 POINT(40.78717058025586 73.6162658507127) bank16468 +16469 POINT(39.74305484777913 73.33081217337893) bank16469 +16470 POINT(40.451970402624454 74.23474998350923) bank16470 +16471 POINT(39.90781376693141 74.42577154053208) bank16471 +16472 POINT(41.09234398366884 73.5262971181261) bank16472 +16473 POINT(40.48645161082524 74.80776671995736) bank16473 +16474 POINT(40.1827323481631 73.93212903173281) bank16474 +16475 POINT(41.120215055912674 74.47696422655103) bank16475 +16476 POINT(40.82072372449338 74.29768276720196) bank16476 +16477 POINT(40.55619832923414 73.5816705838744) bank16477 +16478 POINT(41.119204327804205 73.45140706013298) bank16478 +16479 POINT(41.6159702233726 73.37186103181052) bank16479 +16480 POINT(40.55255359653878 74.20421281947205) bank16480 +16481 POINT(41.41905485118012 74.49429041270395) bank16481 +16482 POINT(40.08996548427528 74.32930403921203) bank16482 +16483 POINT(41.08042960161138 73.98157238665652) bank16483 +16484 POINT(39.969158413696555 74.72303305110204) bank16484 +16485 POINT(41.61109206684941 74.02653493267117) bank16485 +16486 POINT(41.49777041118525 73.19475350070029) bank16486 +16487 POINT(41.640190272461616 74.1881986320945) bank16487 +16488 POINT(41.260099951146444 73.94028896628703) bank16488 +16489 POINT(41.58802594786843 74.96900755825749) bank16489 +16490 POINT(40.92048402854415 73.11872004864018) bank16490 +16491 POINT(40.32081036926871 74.64455167586631) bank16491 +16492 POINT(40.76880531059563 74.87338838080494) bank16492 +16493 POINT(41.276730119642295 74.07063733469666) bank16493 +16494 POINT(40.03406177244324 74.1364656176251) bank16494 +16495 POINT(39.90857855546969 74.30541592859942) bank16495 +16496 POINT(40.10783185845279 74.58033255643703) bank16496 +16497 POINT(40.482641539308375 74.90661847092103) bank16497 +16498 POINT(40.993286517002716 73.4271926034604) bank16498 +16499 POINT(40.52823201496364 73.84126015181782) bank16499 +16500 POINT(41.10119517044726 74.07832544200669) bank16500 +16501 POINT(40.23988795459104 73.40469954889973) bank16501 +16502 POINT(41.47717127882365 74.13718598475891) bank16502 +16503 POINT(41.32306995964796 73.40077306282949) bank16503 +16504 POINT(41.00411507019035 73.20103662124654) bank16504 +16505 POINT(41.54177678252963 73.63047591298711) bank16505 +16506 POINT(41.523784101881176 74.80719322846666) bank16506 +16507 POINT(39.851438652169 73.29519447717263) bank16507 +16508 POINT(41.167039066418724 74.17219494202809) bank16508 +16509 POINT(41.410234552633675 74.11147148564572) bank16509 +16510 POINT(40.53336980682642 73.90027921041336) bank16510 +16511 POINT(40.43773321809418 74.33495100108156) bank16511 +16512 POINT(39.89422831625296 74.89815283367365) bank16512 +16513 POINT(40.17499769125107 73.2177031610392) bank16513 +16514 POINT(40.75096765479173 73.49582796566709) bank16514 +16515 POINT(40.40529465872843 74.8086418535208) bank16515 +16516 POINT(41.523730901360125 74.86619154818088) bank16516 +16517 POINT(40.21726820115737 73.72456042828928) bank16517 +16518 POINT(39.717206801995346 73.77819935363696) bank16518 +16519 POINT(41.68574778442603 74.64335491309167) bank16519 +16520 POINT(40.76811486945813 74.43929817371938) bank16520 +16521 POINT(41.35842943491074 73.12691176954326) bank16521 +16522 POINT(41.26517370385565 74.97921859590082) bank16522 +16523 POINT(41.31787735909489 73.47268440470941) bank16523 +16524 POINT(40.25808829031642 73.64691622813743) bank16524 +16525 POINT(41.22228811199584 73.93571326901517) bank16525 +16526 POINT(39.732634926681804 74.50765080091546) bank16526 +16527 POINT(40.762355558159946 74.77705835150326) bank16527 +16528 POINT(39.922135130832075 73.92680538408084) bank16528 +16529 POINT(40.48097331811345 74.31300630113573) bank16529 +16530 POINT(40.17813309092112 74.28101095498322) bank16530 +16531 POINT(40.096146191054544 74.0479824553824) bank16531 +16532 POINT(41.60488684930804 74.11315255099781) bank16532 +16533 POINT(39.86774413347333 74.18511086096674) bank16533 +16534 POINT(39.85399336443669 73.16706116553576) bank16534 +16535 POINT(40.72254434967829 73.09157845696828) bank16535 +16536 POINT(40.598408022453434 74.29710229408437) bank16536 +16537 POINT(41.021753893414626 73.14020729040699) bank16537 +16538 POINT(39.91225377559922 74.34422190455541) bank16538 +16539 POINT(40.78758021751238 74.07956020038577) bank16539 +16540 POINT(41.3029549869057 73.94007475473579) bank16540 +16541 POINT(41.30861277089827 74.37627357436463) bank16541 +16542 POINT(40.91307797214384 74.52767388678456) bank16542 +16543 POINT(40.8782897429287 74.52335997440697) bank16543 +16544 POINT(41.712131515638404 73.89668668555264) bank16544 +16545 POINT(41.62499367023278 74.53377570918464) bank16545 +16546 POINT(39.87765791473717 74.04766527044667) bank16546 +16547 POINT(41.423378234034224 73.99899984411046) bank16547 +16548 POINT(40.10078436324383 74.18723942308768) bank16548 +16549 POINT(40.575607790196806 73.62321308561663) bank16549 +16550 POINT(39.91180231795762 73.12055738691168) bank16550 +16551 POINT(40.09386372253562 73.3440885577431) bank16551 +16552 POINT(40.78946745553414 73.09375234944609) bank16552 +16553 POINT(39.77153219336547 73.94845155420445) bank16553 +16554 POINT(41.201272272858475 74.2151188552865) bank16554 +16555 POINT(41.58459582541397 73.82263808120773) bank16555 +16556 POINT(40.60949115136608 73.21547336626807) bank16556 +16557 POINT(41.40532655296093 74.88102090599006) bank16557 +16558 POINT(40.56951563589698 74.99244041659928) bank16558 +16559 POINT(40.93873863049009 74.5925586070806) bank16559 +16560 POINT(41.295121275694946 74.1854961342319) bank16560 +16561 POINT(41.313283759634295 73.27880768482245) bank16561 +16562 POINT(39.84128910988441 74.23450027780459) bank16562 +16563 POINT(39.75638120769043 73.86514898011797) bank16563 +16564 POINT(40.482174104147205 73.33459154241561) bank16564 +16565 POINT(41.0773699559664 73.40282022923132) bank16565 +16566 POINT(39.839566735089 74.06004881913535) bank16566 +16567 POINT(40.86486948057217 74.44231687218839) bank16567 +16568 POINT(40.36854386085611 74.11984429938711) bank16568 +16569 POINT(41.198721884320236 73.66168365778468) bank16569 +16570 POINT(40.17217559795079 73.81427297999272) bank16570 +16571 POINT(40.53347319088559 74.81337786607712) bank16571 +16572 POINT(41.60281534103346 73.81908438721425) bank16572 +16573 POINT(41.11082660518002 74.36413031519196) bank16573 +16574 POINT(39.777342630674234 73.80784188651953) bank16574 +16575 POINT(40.14532396142357 73.321378848384) bank16575 +16576 POINT(40.357216286626205 74.63982067257744) bank16576 +16577 POINT(39.896935567516536 73.35535674948116) bank16577 +16578 POINT(41.030615605086055 73.27119190654345) bank16578 +16579 POINT(40.98094987579948 74.67137822368427) bank16579 +16580 POINT(40.07671609182409 74.74625827256364) bank16580 +16581 POINT(41.16251907219274 74.08017114125023) bank16581 +16582 POINT(41.32807439709121 74.32875124706148) bank16582 +16583 POINT(40.84117757092746 73.95867938345792) bank16583 +16584 POINT(40.38312561693053 74.3590830077766) bank16584 +16585 POINT(39.75945180222484 73.28356646036997) bank16585 +16586 POINT(41.1492588489629 73.35884630111322) bank16586 +16587 POINT(41.00815675568129 74.58553322853214) bank16587 +16588 POINT(41.61286856956131 74.99723492565124) bank16588 +16589 POINT(41.58642239207075 74.21911241974291) bank16589 +16590 POINT(40.40227639531865 73.96416990383852) bank16590 +16591 POINT(41.46160779638925 73.05488835782403) bank16591 +16592 POINT(40.823652102182095 73.50220347975298) bank16592 +16593 POINT(39.82685735345438 74.3812416780436) bank16593 +16594 POINT(40.332069874569555 74.32658669541192) bank16594 +16595 POINT(40.608670352410485 74.76216612864471) bank16595 +16596 POINT(41.59483603270894 73.89241638868039) bank16596 +16597 POINT(41.406100254203935 73.209197229581) bank16597 +16598 POINT(41.31221515941756 74.35039517557945) bank16598 +16599 POINT(41.298474176191654 73.39221891199222) bank16599 +16600 POINT(40.170445279453986 74.1827575055535) bank16600 +16601 POINT(40.25075641184741 74.8738088969205) bank16601 +16602 POINT(40.53218182412172 73.9158769003528) bank16602 +16603 POINT(40.960858381552995 73.65686125123997) bank16603 +16604 POINT(40.15785380425406 74.55650490832345) bank16604 +16605 POINT(41.202989791153044 74.57015361351682) bank16605 +16606 POINT(41.505252387657144 73.58254593597131) bank16606 +16607 POINT(40.61560711208916 73.43081103072939) bank16607 +16608 POINT(39.93698053675938 73.90362200174408) bank16608 +16609 POINT(40.02951993235577 74.37219858120852) bank16609 +16610 POINT(40.98193446299749 73.36391897201048) bank16610 +16611 POINT(40.58672241393844 74.6075816332832) bank16611 +16612 POINT(40.39790043945697 73.58911178580001) bank16612 +16613 POINT(40.37628031083801 73.09939942934095) bank16613 +16614 POINT(41.33643035733512 74.0374249268642) bank16614 +16615 POINT(40.96883654066197 74.48069044803323) bank16615 +16616 POINT(41.66499618043268 74.62728365295598) bank16616 +16617 POINT(41.198210019498624 73.27855881728526) bank16617 +16618 POINT(40.502645633700716 73.0120108218371) bank16618 +16619 POINT(40.94914733188201 73.52583418522703) bank16619 +16620 POINT(39.752466734855815 74.38932109618922) bank16620 +16621 POINT(40.23444655805905 74.96208683085582) bank16621 +16622 POINT(41.68016520599573 73.78090159540321) bank16622 +16623 POINT(40.258567508313924 73.98674580721436) bank16623 +16624 POINT(40.53838220340072 73.76766557517061) bank16624 +16625 POINT(40.848805266976264 73.4866846461121) bank16625 +16626 POINT(41.601176215283765 74.66543443528731) bank16626 +16627 POINT(39.930418897738846 73.29780493677336) bank16627 +16628 POINT(40.70419871455636 73.12137783047777) bank16628 +16629 POINT(39.78258672909297 74.24674128583182) bank16629 +16630 POINT(41.56676365170634 73.87547721506888) bank16630 +16631 POINT(39.86627762085635 74.14744898357772) bank16631 +16632 POINT(41.049782825297335 73.9667967478522) bank16632 +16633 POINT(39.857328014890925 74.81876576890174) bank16633 +16634 POINT(40.06293324563408 74.73737042143169) bank16634 +16635 POINT(41.31148445328263 74.26022965257631) bank16635 +16636 POINT(40.22513413825255 73.6859155646108) bank16636 +16637 POINT(41.65271497856599 74.67869546912584) bank16637 +16638 POINT(40.483023221664006 73.81084676201723) bank16638 +16639 POINT(40.22752405828092 74.37636995570823) bank16639 +16640 POINT(40.53009001149435 73.4066103812588) bank16640 +16641 POINT(41.34581583982368 73.69290558068384) bank16641 +16642 POINT(40.34799135812122 74.91371648052116) bank16642 +16643 POINT(41.07392288668385 73.57028803924815) bank16643 +16644 POINT(40.85075981722104 74.20350443762871) bank16644 +16645 POINT(40.01279750040245 73.50513992159533) bank16645 +16646 POINT(40.332151618864394 74.99506248105756) bank16646 +16647 POINT(41.315130563446004 74.90581436432957) bank16647 +16648 POINT(40.50345041794809 73.77431854648023) bank16648 +16649 POINT(41.2750679137696 74.94561265376775) bank16649 +16650 POINT(40.8432664436863 73.45671772618145) bank16650 +16651 POINT(41.254533456353315 73.83316726053474) bank16651 +16652 POINT(40.490363461601156 73.74905008754224) bank16652 +16653 POINT(41.33767734839279 73.01230243848723) bank16653 +16654 POINT(40.952833335837205 74.30364693819223) bank16654 +16655 POINT(40.69940247106509 74.36232523396541) bank16655 +16656 POINT(40.07188357404736 73.91463910076814) bank16656 +16657 POINT(41.37999497646008 74.91156566644756) bank16657 +16658 POINT(39.85186563531359 73.57214569884958) bank16658 +16659 POINT(40.784083417690475 74.08991429366823) bank16659 +16660 POINT(41.47728148251773 74.31017474905742) bank16660 +16661 POINT(41.10666931306958 73.18559875178633) bank16661 +16662 POINT(40.63875691004937 73.665864750814) bank16662 +16663 POINT(41.42271514665372 73.03896873872802) bank16663 +16664 POINT(40.07962503567892 73.61386801936013) bank16664 +16665 POINT(40.030199940891606 74.26241978906477) bank16665 +16666 POINT(41.18802190352083 73.55028171992569) bank16666 +16667 POINT(41.1955337822805 74.38061375044815) bank16667 +16668 POINT(40.811758739916364 74.5545998231053) bank16668 +16669 POINT(40.59318872170028 73.78804348358545) bank16669 +16670 POINT(39.9356121226657 73.4926510320828) bank16670 +16671 POINT(40.85145874965013 74.57200690070734) bank16671 +16672 POINT(40.67367669308617 73.79781883839551) bank16672 +16673 POINT(39.90219788968676 73.92562176534273) bank16673 +16674 POINT(40.61254456629916 73.8158094461963) bank16674 +16675 POINT(40.57040598124983 74.39201081387282) bank16675 +16676 POINT(41.03904888434006 73.56069353217718) bank16676 +16677 POINT(41.18600044321771 73.1275920539062) bank16677 +16678 POINT(41.526658961756276 74.21238152936893) bank16678 +16679 POINT(40.8162347699021 73.30257048102295) bank16679 +16680 POINT(41.4255626075217 74.93300312028069) bank16680 +16681 POINT(40.61294691206381 73.53734742435915) bank16681 +16682 POINT(39.98570470398782 74.56450807637688) bank16682 +16683 POINT(40.46826190089275 74.3667310542287) bank16683 +16684 POINT(40.33109115347142 73.1517118625487) bank16684 +16685 POINT(41.020029430852524 73.86818302962814) bank16685 +16686 POINT(40.2544342132537 73.2560484924089) bank16686 +16687 POINT(40.68021483162772 73.34782596505639) bank16687 +16688 POINT(40.14750072884455 73.29151994528208) bank16688 +16689 POINT(41.42939983282743 73.8421400059525) bank16689 +16690 POINT(39.895863002914865 73.76476134188167) bank16690 +16691 POINT(39.891954114186674 73.38806159331335) bank16691 +16692 POINT(41.65414034541568 73.37781135404033) bank16692 +16693 POINT(41.18814424707996 73.33952958079412) bank16693 +16694 POINT(39.842238976849394 73.1248006807597) bank16694 +16695 POINT(41.52830834517215 73.56588757476133) bank16695 +16696 POINT(39.871653175090174 74.10445265805349) bank16696 +16697 POINT(40.56206631648811 74.15709347971072) bank16697 +16698 POINT(40.290354963548175 73.37106495579758) bank16698 +16699 POINT(40.14965505740854 73.1314946927654) bank16699 +16700 POINT(41.68297605883845 74.5025705480715) bank16700 +16701 POINT(39.95129309502463 74.01571506079912) bank16701 +16702 POINT(39.920843056377265 73.22700527486234) bank16702 +16703 POINT(40.90174851141262 74.68158828632158) bank16703 +16704 POINT(40.32107659933399 73.70610790434853) bank16704 +16705 POINT(41.155001409671385 74.64475205405792) bank16705 +16706 POINT(39.920487538437804 73.12626853386323) bank16706 +16707 POINT(40.091155304998466 74.22919407014355) bank16707 +16708 POINT(41.26970833658827 73.64059889665516) bank16708 +16709 POINT(40.38262713572437 73.53027822814671) bank16709 +16710 POINT(41.48062481949073 73.94492454782467) bank16710 +16711 POINT(39.71373132956415 73.29151561752701) bank16711 +16712 POINT(41.22955133990071 73.06409786214006) bank16712 +16713 POINT(40.28811999653619 74.2652229631919) bank16713 +16714 POINT(41.711951949959314 74.70409613670516) bank16714 +16715 POINT(41.202550343132614 73.49759230176565) bank16715 +16716 POINT(40.451689858601604 73.6843643152547) bank16716 +16717 POINT(40.924615397553374 74.59692098317211) bank16717 +16718 POINT(41.26422775998155 74.72672718059033) bank16718 +16719 POINT(41.32362280616336 73.58452884009532) bank16719 +16720 POINT(40.1804857237297 73.55045105008463) bank16720 +16721 POINT(40.557037245942055 73.22649385255407) bank16721 +16722 POINT(39.73910721407116 74.43127042864) bank16722 +16723 POINT(41.494903537229064 74.80097136178901) bank16723 +16724 POINT(40.48655637092465 74.61227977344448) bank16724 +16725 POINT(41.353822762465946 73.0606096225406) bank16725 +16726 POINT(40.31648557686264 73.58957374035556) bank16726 +16727 POINT(40.399189302315804 74.98084242914133) bank16727 +16728 POINT(41.05011701904101 74.05265532756785) bank16728 +16729 POINT(40.12550506133668 74.10498250126837) bank16729 +16730 POINT(41.56036330483202 73.55295864958639) bank16730 +16731 POINT(41.25648077963416 74.75817074516475) bank16731 +16732 POINT(39.88566149436628 73.33175131081633) bank16732 +16733 POINT(40.05822757810238 74.4104703538636) bank16733 +16734 POINT(40.878993961552005 74.308073162477) bank16734 +16735 POINT(40.687613741380545 74.5211136209983) bank16735 +16736 POINT(41.04154099165382 74.25911492025489) bank16736 +16737 POINT(41.102894832778645 73.18580695673293) bank16737 +16738 POINT(40.307899770794705 74.3388410257783) bank16738 +16739 POINT(41.27113452235398 74.17034443536534) bank16739 +16740 POINT(40.62300867854087 74.62476684512106) bank16740 +16741 POINT(40.23058174508031 73.93981261436322) bank16741 +16742 POINT(40.74785001550982 73.94283079539463) bank16742 +16743 POINT(40.24333473966406 73.2417663777315) bank16743 +16744 POINT(40.49979660989202 74.31298470833455) bank16744 +16745 POINT(41.709112263168066 74.30133564203196) bank16745 +16746 POINT(40.6641912702568 73.53328304101602) bank16746 +16747 POINT(41.425480779917535 73.60764120516428) bank16747 +16748 POINT(41.69945006638564 73.86490861024977) bank16748 +16749 POINT(40.07192809226288 73.26565930904648) bank16749 +16750 POINT(40.68097217214477 73.75297940753072) bank16750 +16751 POINT(40.47152440674213 73.30248544783436) bank16751 +16752 POINT(41.430727438250415 73.83048279045691) bank16752 +16753 POINT(40.09505372897426 73.04145856032846) bank16753 +16754 POINT(39.77222123170943 73.13020500319567) bank16754 +16755 POINT(39.78869476865248 73.9058259849552) bank16755 +16756 POINT(40.34761041278696 74.18922526548165) bank16756 +16757 POINT(41.04837115911895 73.46211116826849) bank16757 +16758 POINT(41.43152362935583 73.76521840986383) bank16758 +16759 POINT(41.26219303743476 73.95256438516383) bank16759 +16760 POINT(40.0963976495711 74.56324473412761) bank16760 +16761 POINT(40.72739501227953 73.86277380417106) bank16761 +16762 POINT(41.125918828323094 73.18349132102998) bank16762 +16763 POINT(41.5354210194525 73.78669616594472) bank16763 +16764 POINT(40.725756229320574 73.20572490253265) bank16764 +16765 POINT(41.16924960338629 74.25063975628463) bank16765 +16766 POINT(41.709244731899 73.06537490435014) bank16766 +16767 POINT(40.29849626742489 74.79447998895253) bank16767 +16768 POINT(39.80164819308702 73.77294692663618) bank16768 +16769 POINT(41.09893534210913 73.95483499083582) bank16769 +16770 POINT(40.366235726743064 73.04384009217112) bank16770 +16771 POINT(41.46480300739584 74.61539911183297) bank16771 +16772 POINT(41.03013287544114 74.28856693612403) bank16772 +16773 POINT(39.733807848119014 74.11064017631416) bank16773 +16774 POINT(39.73912940468262 73.37272617817442) bank16774 +16775 POINT(40.34342789567647 73.13907398743363) bank16775 +16776 POINT(40.27246104074554 74.81664298014726) bank16776 +16777 POINT(40.025614600595546 73.00818813332931) bank16777 +16778 POINT(40.83817441599791 74.00314815020803) bank16778 +16779 POINT(40.97314298025428 74.237284458562) bank16779 +16780 POINT(41.50777527862419 74.62814339310567) bank16780 +16781 POINT(40.09246829717563 73.78124819568079) bank16781 +16782 POINT(40.57262823521456 73.07968031709274) bank16782 +16783 POINT(40.93336287638937 73.63338604658982) bank16783 +16784 POINT(40.73718146590292 73.76894542674825) bank16784 +16785 POINT(40.299099249396974 74.08928607233251) bank16785 +16786 POINT(40.47062942246631 73.89357920593548) bank16786 +16787 POINT(40.3674260849659 74.81167145416204) bank16787 +16788 POINT(40.95026534325184 74.97724373000077) bank16788 +16789 POINT(40.09704982786934 74.98920315140408) bank16789 +16790 POINT(39.90825374029977 74.36486644149929) bank16790 +16791 POINT(41.66638390555568 74.18486451898568) bank16791 +16792 POINT(39.83829799041911 73.21970928310952) bank16792 +16793 POINT(39.825725488755985 74.2522959806324) bank16793 +16794 POINT(40.66013379905611 74.18016313836314) bank16794 +16795 POINT(40.20112762638697 73.39258142693595) bank16795 +16796 POINT(41.699503219258915 73.87393919071043) bank16796 +16797 POINT(41.496895972714725 73.93964039424701) bank16797 +16798 POINT(41.71147608278732 73.8769582211333) bank16798 +16799 POINT(41.30143043113361 73.63294358257768) bank16799 +16800 POINT(41.170231443494416 73.35003747797194) bank16800 +16801 POINT(40.037715738348496 74.4303151237062) bank16801 +16802 POINT(40.18335871498135 74.09669965665934) bank16802 +16803 POINT(40.32741397370498 73.95586868740837) bank16803 +16804 POINT(40.27461129765585 73.70195994472176) bank16804 +16805 POINT(39.846777040433096 74.54375841878635) bank16805 +16806 POINT(41.056515678646086 74.70885254928425) bank16806 +16807 POINT(41.263472450056724 74.5839217348115) bank16807 +16808 POINT(40.08386731374817 74.79318905705617) bank16808 +16809 POINT(41.13055123304773 74.43591714659588) bank16809 +16810 POINT(39.96470767165522 74.8047282376115) bank16810 +16811 POINT(41.0846701214715 73.87087397064434) bank16811 +16812 POINT(40.144495920330485 73.08670843493783) bank16812 +16813 POINT(41.50692720720655 73.79711189437344) bank16813 +16814 POINT(40.12568934086775 73.27217406898961) bank16814 +16815 POINT(40.00093226980541 73.27304537790451) bank16815 +16816 POINT(40.39039756182703 74.15057858204108) bank16816 +16817 POINT(41.111880445958434 73.58062319476693) bank16817 +16818 POINT(41.57548081894065 73.64303455341266) bank16818 +16819 POINT(41.48700589845045 73.95464467965257) bank16819 +16820 POINT(40.65655103811084 74.64487541948986) bank16820 +16821 POINT(41.14770621987329 74.05773105256162) bank16821 +16822 POINT(40.932041714826454 73.5774139912586) bank16822 +16823 POINT(39.80037987068133 73.6870587408898) bank16823 +16824 POINT(40.33823872168116 73.93590244630106) bank16824 +16825 POINT(39.92639288622536 73.16265921316295) bank16825 +16826 POINT(41.2537106784788 74.80266204165541) bank16826 +16827 POINT(41.2273962372964 73.56303558236947) bank16827 +16828 POINT(40.79723556078966 73.22265987728294) bank16828 +16829 POINT(41.35708451354583 73.21674458249063) bank16829 +16830 POINT(41.70268419500961 74.43167993315684) bank16830 +16831 POINT(40.79524148476799 73.47213868807299) bank16831 +16832 POINT(39.80334502374831 73.34980998488658) bank16832 +16833 POINT(40.29309868778015 74.50610981541908) bank16833 +16834 POINT(41.03275062931775 74.43268359847102) bank16834 +16835 POINT(40.20899834344583 73.27258216412828) bank16835 +16836 POINT(40.14745792930609 73.98092822700798) bank16836 +16837 POINT(41.3331521016678 73.22139423522142) bank16837 +16838 POINT(41.24129114731312 73.38897420809977) bank16838 +16839 POINT(41.006213868904865 73.49053717734493) bank16839 +16840 POINT(40.01313557784261 73.67719566812913) bank16840 +16841 POINT(40.27912154969704 73.93705631140607) bank16841 +16842 POINT(40.51394062086433 74.68002560790474) bank16842 +16843 POINT(40.98785067809525 74.47116433924072) bank16843 +16844 POINT(39.79338817192762 73.383522763094) bank16844 +16845 POINT(40.776236330072805 73.38119802801836) bank16845 +16846 POINT(40.89980082334056 73.14896599845787) bank16846 +16847 POINT(41.15225170337362 73.24260460996538) bank16847 +16848 POINT(41.64210564961699 74.28165490677698) bank16848 +16849 POINT(41.346804447290346 73.14183931326887) bank16849 +16850 POINT(40.56271685327813 74.2834725435293) bank16850 +16851 POINT(40.44775167497574 73.63680407012308) bank16851 +16852 POINT(41.26054474391907 73.25671162727485) bank16852 +16853 POINT(40.60887355897496 73.77500654739617) bank16853 +16854 POINT(40.28270698465206 73.1753750491688) bank16854 +16855 POINT(41.52072914951535 73.75453921335445) bank16855 +16856 POINT(40.03412125585454 73.10977366086195) bank16856 +16857 POINT(40.8323655951383 74.82166321791632) bank16857 +16858 POINT(40.45294177961596 73.74741708769325) bank16858 +16859 POINT(41.59549745085176 74.35921052108937) bank16859 +16860 POINT(40.889248400406366 74.67837140055903) bank16860 +16861 POINT(41.449184566925965 73.48971998042619) bank16861 +16862 POINT(41.014043927405964 74.0775473548014) bank16862 +16863 POINT(41.1009733388948 74.83121532274924) bank16863 +16864 POINT(41.06295601653815 74.1589446482665) bank16864 +16865 POINT(40.93619100093752 73.28371379638571) bank16865 +16866 POINT(40.25248665707014 74.61755869890129) bank16866 +16867 POINT(40.56734793010996 73.93979011703809) bank16867 +16868 POINT(40.742609014219546 74.45408709848316) bank16868 +16869 POINT(39.80282739678435 74.30733469111713) bank16869 +16870 POINT(40.586438170804364 73.95601607860385) bank16870 +16871 POINT(40.9032599872239 73.190697161703) bank16871 +16872 POINT(41.21967641747174 74.94841029492262) bank16872 +16873 POINT(40.280766316197465 73.58034523430912) bank16873 +16874 POINT(40.68877958195293 73.26050492334832) bank16874 +16875 POINT(41.61535110512218 74.05966303014254) bank16875 +16876 POINT(40.60875288057896 74.08578452578544) bank16876 +16877 POINT(40.835159577499844 74.18416775460982) bank16877 +16878 POINT(39.76065283260455 73.66217621536217) bank16878 +16879 POINT(41.67694294213042 73.71727970318527) bank16879 +16880 POINT(41.6776638966906 73.3414031288236) bank16880 +16881 POINT(41.348821407043886 73.03515488111574) bank16881 +16882 POINT(41.687475844439284 73.77926668245881) bank16882 +16883 POINT(40.313604370644796 74.375281951891) bank16883 +16884 POINT(40.014584788113666 74.05180738368563) bank16884 +16885 POINT(39.907782524988306 73.4704956770319) bank16885 +16886 POINT(41.596444719537466 73.37096756681517) bank16886 +16887 POINT(41.63178444875217 73.93520731828083) bank16887 +16888 POINT(40.625030293075326 74.29271854299725) bank16888 +16889 POINT(40.19764742836097 74.598210686076) bank16889 +16890 POINT(40.394096256585854 74.15582202049391) bank16890 +16891 POINT(40.84993908139627 74.34607909406112) bank16891 +16892 POINT(40.183548198483805 74.10533190652092) bank16892 +16893 POINT(40.91388685408218 73.66633481693013) bank16893 +16894 POINT(40.67829862305949 73.6978593658069) bank16894 +16895 POINT(41.668562822826985 73.50693338925912) bank16895 +16896 POINT(41.64490853869845 74.49894337709425) bank16896 +16897 POINT(39.71666647704955 74.53167336569201) bank16897 +16898 POINT(40.37070326027459 74.87979967352031) bank16898 +16899 POINT(41.422123243847516 73.57382561012363) bank16899 +16900 POINT(39.8690761378494 74.48181854194729) bank16900 +16901 POINT(41.27623632116568 74.99354541462202) bank16901 +16902 POINT(40.260387022774964 74.96204808836187) bank16902 +16903 POINT(40.14094177438451 74.57472894201021) bank16903 +16904 POINT(40.17856198687979 73.90802923641773) bank16904 +16905 POINT(40.956443879595255 74.0803543241303) bank16905 +16906 POINT(40.44749185196506 73.41969390143002) bank16906 +16907 POINT(39.83276557560044 73.52107859964795) bank16907 +16908 POINT(39.7711127178624 73.4622164125915) bank16908 +16909 POINT(40.155514101112566 73.90229542229703) bank16909 +16910 POINT(40.64053400301665 74.33830648622099) bank16910 +16911 POINT(41.030201405776054 73.77527939093122) bank16911 +16912 POINT(41.20785659641645 74.14404694199592) bank16912 +16913 POINT(41.24144448722217 73.26107411802495) bank16913 +16914 POINT(41.30810504630015 73.47847503608605) bank16914 +16915 POINT(40.97787193381264 73.22226676773808) bank16915 +16916 POINT(40.920706115039394 73.26382257372778) bank16916 +16917 POINT(40.295549388021335 73.45068479266548) bank16917 +16918 POINT(39.79155693736684 73.55637388773361) bank16918 +16919 POINT(40.35380568138303 73.68799556899364) bank16919 +16920 POINT(41.006710304936234 74.14905551710201) bank16920 +16921 POINT(40.30015146792914 74.1135820228324) bank16921 +16922 POINT(40.23497612825615 73.33568986651655) bank16922 +16923 POINT(40.89569910360137 73.39332455928546) bank16923 +16924 POINT(40.563844417982686 74.23823279552454) bank16924 +16925 POINT(41.00299789977104 74.39013075115074) bank16925 +16926 POINT(40.21319253388014 74.46149262666246) bank16926 +16927 POINT(41.10893858976102 74.52191801561598) bank16927 +16928 POINT(40.64591301696821 73.44949628097018) bank16928 +16929 POINT(40.37679639906817 73.73022612015272) bank16929 +16930 POINT(40.19777048558281 74.54869308155588) bank16930 +16931 POINT(39.96348646166559 73.26839150517212) bank16931 +16932 POINT(40.62264336143253 73.30733993284626) bank16932 +16933 POINT(40.17818422231379 74.28159248348383) bank16933 +16934 POINT(40.034209805923815 73.54720809913212) bank16934 +16935 POINT(40.73159251577671 73.93932101342688) bank16935 +16936 POINT(41.02441914679483 73.46977306318504) bank16936 +16937 POINT(40.13852473410617 73.23173509413472) bank16937 +16938 POINT(40.53289922860582 74.05918043101063) bank16938 +16939 POINT(40.74169936641419 74.67052252613416) bank16939 +16940 POINT(40.489046318334076 73.3314714513785) bank16940 +16941 POINT(40.35452392134664 73.03042486409379) bank16941 +16942 POINT(40.0393850881957 74.83132925096355) bank16942 +16943 POINT(40.077939628178946 73.38040489142769) bank16943 +16944 POINT(40.84569449192663 73.12914303969131) bank16944 +16945 POINT(41.20767985781523 74.20428810075983) bank16945 +16946 POINT(41.07556901106066 74.73268449963349) bank16946 +16947 POINT(39.97625233262497 74.38414717142268) bank16947 +16948 POINT(41.07842616127106 74.61241660164299) bank16948 +16949 POINT(40.32431851029674 74.11464325555474) bank16949 +16950 POINT(40.6705014536881 73.94447779094418) bank16950 +16951 POINT(40.7596585134841 74.28031130662194) bank16951 +16952 POINT(40.810820338171006 74.23315430138189) bank16952 +16953 POINT(41.23036386355868 74.80175001247792) bank16953 +16954 POINT(40.27001844771563 73.9807377630037) bank16954 +16955 POINT(39.97973601956317 74.16521935841632) bank16955 +16956 POINT(40.69480103743014 73.37584867241826) bank16956 +16957 POINT(40.585619586910376 73.21647255307539) bank16957 +16958 POINT(40.500354356065216 73.72153913995638) bank16958 +16959 POINT(39.90256573874687 74.06540086321417) bank16959 +16960 POINT(41.691514286859075 73.76426289784118) bank16960 +16961 POINT(41.32074452468472 74.78426323223532) bank16961 +16962 POINT(40.9181871916287 74.44080307380484) bank16962 +16963 POINT(40.97600532403188 73.55525901786281) bank16963 +16964 POINT(40.10475451470592 73.86905252329487) bank16964 +16965 POINT(40.86954985058788 74.18740039572032) bank16965 +16966 POINT(41.18642919951616 74.30391475662204) bank16966 +16967 POINT(41.52333554585661 73.70047226743098) bank16967 +16968 POINT(40.968458686585215 74.89407846661322) bank16968 +16969 POINT(40.10782814926795 73.88025829044201) bank16969 +16970 POINT(40.33702660743319 73.58235275641256) bank16970 +16971 POINT(41.219538731347136 73.97881763783998) bank16971 +16972 POINT(41.711691693552396 73.67898199618645) bank16972 +16973 POINT(40.38959534175744 73.87119807803204) bank16973 +16974 POINT(41.381562900889286 73.70370785604882) bank16974 +16975 POINT(41.361074244845916 74.51056942476637) bank16975 +16976 POINT(40.07613142155524 73.94687762812673) bank16976 +16977 POINT(40.83634980965623 74.35670002163931) bank16977 +16978 POINT(40.500805380424836 73.4943724315805) bank16978 +16979 POINT(40.31580958931055 73.36354750162462) bank16979 +16980 POINT(41.43155729165762 73.38062256535001) bank16980 +16981 POINT(40.657438554145024 74.73762881176621) bank16981 +16982 POINT(41.16987188892888 74.67651023396155) bank16982 +16983 POINT(41.090676809878474 74.99042095593376) bank16983 +16984 POINT(40.4940251081453 74.67353804253565) bank16984 +16985 POINT(40.31474893605511 74.20957448042726) bank16985 +16986 POINT(41.46507454868318 73.53777924183122) bank16986 +16987 POINT(40.53971343784701 74.29144419338708) bank16987 +16988 POINT(40.21110217394403 73.21507778860327) bank16988 +16989 POINT(41.124656864850536 74.22188315761817) bank16989 +16990 POINT(40.712440354107336 73.8095696116025) bank16990 +16991 POINT(40.85700772740855 74.1731331886491) bank16991 +16992 POINT(40.66958086655575 74.33952150396499) bank16992 +16993 POINT(39.85033160572463 74.02084709820915) bank16993 +16994 POINT(40.908267180606266 73.15340612899996) bank16994 +16995 POINT(41.45950223844676 73.15824743290796) bank16995 +16996 POINT(40.39827102259387 74.97741391330861) bank16996 +16997 POINT(40.28952982375648 74.19927102853286) bank16997 +16998 POINT(41.33064335163675 73.06869769443522) bank16998 +16999 POINT(41.051745288719395 73.59649024126635) bank16999 +17000 POINT(39.97646321142786 73.80503669639705) bank17000 +17001 POINT(40.76085823690924 74.11216857852834) bank17001 +17002 POINT(41.09568038098644 73.14733584340014) bank17002 +17003 POINT(40.61813096916865 73.20617460646545) bank17003 +17004 POINT(40.49694312467737 73.70367690595026) bank17004 +17005 POINT(40.27889414229789 74.23869190075706) bank17005 +17006 POINT(40.41971367556036 74.94754578890459) bank17006 +17007 POINT(40.52184444867083 73.1682816880525) bank17007 +17008 POINT(41.29314945538171 74.69025222328737) bank17008 +17009 POINT(39.7586099850309 74.44429593380812) bank17009 +17010 POINT(41.141368938625504 74.04512045068672) bank17010 +17011 POINT(40.40302208903941 74.7057294388995) bank17011 +17012 POINT(41.00561802293011 73.33910084936502) bank17012 +17013 POINT(40.064807092261134 74.43605002273755) bank17013 +17014 POINT(40.97610426443301 73.53560232530661) bank17014 +17015 POINT(39.76634084477502 73.05350389301746) bank17015 +17016 POINT(41.53470119429578 74.1396811706543) bank17016 +17017 POINT(41.154213500390895 74.23014516966008) bank17017 +17018 POINT(41.64017497665579 74.39481225201915) bank17018 +17019 POINT(40.24884652445558 73.35697693060189) bank17019 +17020 POINT(41.15524790479724 74.03236735333601) bank17020 +17021 POINT(41.696076975591815 74.5924929964824) bank17021 +17022 POINT(40.05711909979919 74.03037761153693) bank17022 +17023 POINT(39.73084759122479 74.51636556859849) bank17023 +17024 POINT(40.131471398244514 74.1010553837005) bank17024 +17025 POINT(41.10669293598383 73.51346630043763) bank17025 +17026 POINT(39.751251561587345 73.2511534941461) bank17026 +17027 POINT(41.21528606453003 73.4181142347148) bank17027 +17028 POINT(41.20786708211948 73.0678177463018) bank17028 +17029 POINT(41.37751349716557 74.63423917359697) bank17029 +17030 POINT(40.76695656384663 73.63636694772205) bank17030 +17031 POINT(40.96454217061998 73.60562409428216) bank17031 +17032 POINT(39.72511564768929 73.55750722949875) bank17032 +17033 POINT(40.482648555454354 73.86672366034858) bank17033 +17034 POINT(40.486787078561505 73.19004853338521) bank17034 +17035 POINT(40.44508371652735 73.01500830037025) bank17035 +17036 POINT(40.68137877738438 74.80707661190453) bank17036 +17037 POINT(39.92679089345867 73.59394954760815) bank17037 +17038 POINT(41.29941504722288 73.63082343138016) bank17038 +17039 POINT(40.72056170347379 74.56243387144946) bank17039 +17040 POINT(40.85334560001898 74.93021143073558) bank17040 +17041 POINT(41.59966389843258 74.80611571900234) bank17041 +17042 POINT(41.665268844616534 74.91608501339965) bank17042 +17043 POINT(39.83711213719302 74.14708261521324) bank17043 +17044 POINT(40.15704526409513 74.99383533906656) bank17044 +17045 POINT(39.7401570429662 73.0155452547919) bank17045 +17046 POINT(40.858742644394994 74.52924453148472) bank17046 +17047 POINT(40.07960310468696 74.23244844098629) bank17047 +17048 POINT(40.159481947583636 73.67642668377721) bank17048 +17049 POINT(41.24980486857177 74.57434920459224) bank17049 +17050 POINT(41.54031871436167 73.38653365084657) bank17050 +17051 POINT(41.16305128203841 74.31342158451393) bank17051 +17052 POINT(40.609501696612526 74.31229340155997) bank17052 +17053 POINT(41.22925850679472 73.51182630731402) bank17053 +17054 POINT(41.598976528635575 73.97473256582789) bank17054 +17055 POINT(41.12329426251199 74.11204487338009) bank17055 +17056 POINT(40.542482939239065 74.6589182648671) bank17056 +17057 POINT(41.54053306610193 74.83052192724732) bank17057 +17058 POINT(40.62970024349741 73.96877171501046) bank17058 +17059 POINT(39.88341097669321 73.95388160407869) bank17059 +17060 POINT(40.418502799183464 74.20599249582101) bank17060 +17061 POINT(40.34137616953765 74.10637571985643) bank17061 +17062 POINT(40.4699859631828 74.99025166010206) bank17062 +17063 POINT(40.32398708625092 74.74897711173134) bank17063 +17064 POINT(39.82381164985984 74.18790223926874) bank17064 +17065 POINT(39.80476313044079 74.08161168748278) bank17065 +17066 POINT(39.99531582996872 74.65921118143366) bank17066 +17067 POINT(40.96710779284997 74.30636405965441) bank17067 +17068 POINT(41.35917659654818 73.73357054821457) bank17068 +17069 POINT(41.60533635795397 73.86627474216976) bank17069 +17070 POINT(40.38432333840361 74.12901911648642) bank17070 +17071 POINT(40.65523977939758 73.01863994306939) bank17071 +17072 POINT(40.84023722095672 74.79297324320838) bank17072 +17073 POINT(40.05020911335321 73.54337289039553) bank17073 +17074 POINT(40.42643322360914 74.46334696060218) bank17074 +17075 POINT(41.477833432337164 73.32152735975905) bank17075 +17076 POINT(41.37888625564415 74.08399894439964) bank17076 +17077 POINT(39.905114201081346 73.12361154552244) bank17077 +17078 POINT(40.90868803158857 74.13462820676955) bank17078 +17079 POINT(40.80632219820775 73.11722154863268) bank17079 +17080 POINT(40.274190381411344 74.65357874278436) bank17080 +17081 POINT(41.173260448079496 74.06751925608754) bank17081 +17082 POINT(40.39681339968048 73.36394526536255) bank17082 +17083 POINT(40.5510676862303 74.44535003810391) bank17083 +17084 POINT(40.68113659968391 74.43346979742681) bank17084 +17085 POINT(40.69471768240488 73.49029676627339) bank17085 +17086 POINT(39.84036770208206 74.11888831781387) bank17086 +17087 POINT(41.4038796109083 74.07740098446622) bank17087 +17088 POINT(41.16855883862995 74.43792219429693) bank17088 +17089 POINT(41.13760076122825 74.81413842308235) bank17089 +17090 POINT(40.140717615204856 74.6895109499628) bank17090 +17091 POINT(41.29433778665946 74.53661220421037) bank17091 +17092 POINT(39.85120147457146 73.096340464612) bank17092 +17093 POINT(40.33907008787963 74.83683656268711) bank17093 +17094 POINT(40.09074241115911 74.58518079810317) bank17094 +17095 POINT(40.41442731942081 73.5803590031862) bank17095 +17096 POINT(40.190925343938325 73.82555959471959) bank17096 +17097 POINT(41.45348675078859 73.19067125228794) bank17097 +17098 POINT(41.20371919853946 74.01067485397238) bank17098 +17099 POINT(39.99901705343548 73.1769966965831) bank17099 +17100 POINT(40.486509340439845 73.61963487817356) bank17100 +17101 POINT(40.02407003569944 73.75033057620401) bank17101 +17102 POINT(41.41772348028966 73.96867007532983) bank17102 +17103 POINT(40.24597238015162 73.19260615280636) bank17103 +17104 POINT(41.263268817654804 74.2712182044781) bank17104 +17105 POINT(41.09217116493924 73.12032116028765) bank17105 +17106 POINT(41.384146858574326 73.32904673725987) bank17106 +17107 POINT(41.53355286295158 73.2520916554859) bank17107 +17108 POINT(40.99896256041557 74.54092470816218) bank17108 +17109 POINT(41.523693100277264 74.72253126124446) bank17109 +17110 POINT(40.54712073692974 75.00466994697977) bank17110 +17111 POINT(40.18399350754328 73.68647665316476) bank17111 +17112 POINT(41.08551244011586 73.58668874777511) bank17112 +17113 POINT(40.61120202429446 73.58956227775604) bank17113 +17114 POINT(40.659923365361706 73.0964928994541) bank17114 +17115 POINT(39.95991836465001 74.36122874926907) bank17115 +17116 POINT(41.571250388023 73.76498651456686) bank17116 +17117 POINT(41.04446155730784 73.07722020125283) bank17117 +17118 POINT(39.76135101181341 74.04744830306264) bank17118 +17119 POINT(41.41016674176722 73.1984105939905) bank17119 +17120 POINT(41.0128987523504 74.62245887158412) bank17120 +17121 POINT(39.85325215762345 73.47746548940836) bank17121 +17122 POINT(41.709487887252564 74.65470344021207) bank17122 +17123 POINT(39.96553838288609 73.95489256428982) bank17123 +17124 POINT(39.96383438874224 74.56866494836281) bank17124 +17125 POINT(41.45344543008592 73.01600406980377) bank17125 +17126 POINT(40.71995954831796 73.09147764171774) bank17126 +17127 POINT(41.339488815582754 74.33721623909112) bank17127 +17128 POINT(39.758178350299964 73.67731570884003) bank17128 +17129 POINT(41.643392186814 73.90797580871335) bank17129 +17130 POINT(39.99775252287232 74.17814038741597) bank17130 +17131 POINT(40.04114419645887 74.0240390203809) bank17131 +17132 POINT(39.768864545730864 74.80618618253169) bank17132 +17133 POINT(40.22615314963598 74.75319151425003) bank17133 +17134 POINT(40.61820892789411 74.84933715106447) bank17134 +17135 POINT(41.21868253914106 73.56700336627704) bank17135 +17136 POINT(40.81016218639058 73.38716863137967) bank17136 +17137 POINT(41.18495906098108 73.46230486364645) bank17137 +17138 POINT(41.26210878442648 74.70299989585092) bank17138 +17139 POINT(40.5025280767747 73.39583944072979) bank17139 +17140 POINT(41.49591062444122 74.05362367477402) bank17140 +17141 POINT(40.11642793956905 74.46729510704445) bank17141 +17142 POINT(40.80685980413747 73.21838549790658) bank17142 +17143 POINT(41.081306569542576 73.28467408005719) bank17143 +17144 POINT(40.58132897999081 73.18740929204205) bank17144 +17145 POINT(40.013238787026154 74.59932017667657) bank17145 +17146 POINT(40.69910231489045 73.84894742595137) bank17146 +17147 POINT(41.1883107568056 73.22347336784023) bank17147 +17148 POINT(40.86504672848059 74.44990638979968) bank17148 +17149 POINT(40.69891884828231 74.15874653861785) bank17149 +17150 POINT(39.76604130262585 74.05036632635567) bank17150 +17151 POINT(40.98819311541504 73.51730785410173) bank17151 +17152 POINT(40.78871951926692 73.43558533615703) bank17152 +17153 POINT(40.529204250277886 73.56012808967337) bank17153 +17154 POINT(40.2226721273915 73.73165762395658) bank17154 +17155 POINT(40.34130838650361 73.9923582270217) bank17155 +17156 POINT(39.98062031128564 74.39713275353914) bank17156 +17157 POINT(40.591387519840616 73.78940672134777) bank17157 +17158 POINT(41.56316086720709 74.49556923329202) bank17158 +17159 POINT(41.44238338110272 73.27045794477735) bank17159 +17160 POINT(40.61547895500327 74.10044667629067) bank17160 +17161 POINT(41.412845930884906 74.01482677165889) bank17161 +17162 POINT(41.43011206148632 74.71238252937525) bank17162 +17163 POINT(40.54637464458445 73.7163686797889) bank17163 +17164 POINT(40.26391713624958 73.33311319104666) bank17164 +17165 POINT(40.09664385101614 73.14200845042299) bank17165 +17166 POINT(40.587034462086876 73.73797128686742) bank17166 +17167 POINT(41.54294711958969 73.71416167575559) bank17167 +17168 POINT(40.14462996475109 73.74503956062776) bank17168 +17169 POINT(40.73199641883432 74.15772033318981) bank17169 +17170 POINT(39.72502331331024 73.69487465966117) bank17170 +17171 POINT(40.68413164571374 74.17466939835465) bank17171 +17172 POINT(40.08568241973351 74.65421240915819) bank17172 +17173 POINT(40.50630366683321 73.14064814410213) bank17173 +17174 POINT(40.778187319301345 73.04825168677768) bank17174 +17175 POINT(40.54218701960473 74.67861263022566) bank17175 +17176 POINT(39.95663269536466 74.90268295469946) bank17176 +17177 POINT(40.129681986480186 73.08779270949951) bank17177 +17178 POINT(40.49715824037857 74.26912734780863) bank17178 +17179 POINT(40.56694164552106 74.23442792675024) bank17179 +17180 POINT(41.31020606254913 74.68777602224529) bank17180 +17181 POINT(40.363401313283795 73.13731128381309) bank17181 +17182 POINT(40.32488596123491 73.15102371564731) bank17182 +17183 POINT(40.22881326032976 73.74169280371076) bank17183 +17184 POINT(39.98799305176449 74.87949566799816) bank17184 +17185 POINT(41.36865506089324 74.68061982141883) bank17185 +17186 POINT(40.591122596342124 74.6736311337041) bank17186 +17187 POINT(40.9157736643147 74.65910156381487) bank17187 +17188 POINT(41.68657160527047 74.23425184875894) bank17188 +17189 POINT(40.02507786092825 73.7507736869245) bank17189 +17190 POINT(40.84487890737201 74.64733442994805) bank17190 +17191 POINT(41.15533574227092 73.74841050166198) bank17191 +17192 POINT(40.676380896038545 74.68380601195642) bank17192 +17193 POINT(40.93813925695233 73.35227977081418) bank17193 +17194 POINT(40.06480211964829 73.24287535577467) bank17194 +17195 POINT(40.869530004322996 73.69387921387734) bank17195 +17196 POINT(41.40799888316273 74.5939021400749) bank17196 +17197 POINT(41.00438485631676 73.91502559502788) bank17197 +17198 POINT(41.34208977201501 74.27007084280515) bank17198 +17199 POINT(39.974356695625126 73.39098021944821) bank17199 +17200 POINT(40.5761250412486 74.30148999855874) bank17200 +17201 POINT(40.59036879053327 74.67105118895851) bank17201 +17202 POINT(40.41775976491782 74.59311934098197) bank17202 +17203 POINT(41.20496311313436 73.99453589593574) bank17203 +17204 POINT(41.47933850982168 73.40229169862259) bank17204 +17205 POINT(40.21304491637854 74.29651372304158) bank17205 +17206 POINT(41.64878532938582 73.08446187794576) bank17206 +17207 POINT(39.796436046462325 73.93339659894843) bank17207 +17208 POINT(41.68612688501999 73.81430661416701) bank17208 +17209 POINT(40.95924907027823 74.9302413937722) bank17209 +17210 POINT(40.06906909256699 73.12421390849144) bank17210 +17211 POINT(41.703584488207845 73.41237813993921) bank17211 +17212 POINT(40.01580619481546 73.21665500840888) bank17212 +17213 POINT(40.995002653217654 74.76910758293162) bank17213 +17214 POINT(41.32796699310428 73.18316836506595) bank17214 +17215 POINT(40.88360981564239 73.44961167665713) bank17215 +17216 POINT(41.527308517445725 73.77418211890006) bank17216 +17217 POINT(39.80230656428804 73.84208167073696) bank17217 +17218 POINT(40.54672726999043 74.80211834817206) bank17218 +17219 POINT(39.99083328083552 74.79646640461576) bank17219 +17220 POINT(41.30105404770059 74.35649789753322) bank17220 +17221 POINT(41.177038006709815 73.38489616280273) bank17221 +17222 POINT(40.167652645773664 74.6155096126622) bank17222 +17223 POINT(41.43085471159734 73.6646713157903) bank17223 +17224 POINT(41.18420843636072 74.1312460788903) bank17224 +17225 POINT(39.76165951122688 74.10420267644439) bank17225 +17226 POINT(40.94343311378705 73.21277177136984) bank17226 +17227 POINT(40.80423518518622 74.03086509677794) bank17227 +17228 POINT(40.08144516068739 73.94469710291062) bank17228 +17229 POINT(39.964327732107556 74.74155342887134) bank17229 +17230 POINT(39.929293699017286 73.20024454268157) bank17230 +17231 POINT(40.563982224116366 74.31178371005231) bank17231 +17232 POINT(40.911837045237036 73.65926625161336) bank17232 +17233 POINT(41.40550424458055 74.73618745455883) bank17233 +17234 POINT(41.00226993919383 73.64832885833849) bank17234 +17235 POINT(41.581115784994914 73.68347672769663) bank17235 +17236 POINT(40.6724756833259 73.73335250049388) bank17236 +17237 POINT(41.34438810928103 74.93966781049288) bank17237 +17238 POINT(39.91486967200125 73.36431143703251) bank17238 +17239 POINT(40.100928248649524 73.29075731348142) bank17239 +17240 POINT(41.691576473491715 74.52421061228786) bank17240 +17241 POINT(41.29220015034278 74.08721128876785) bank17241 +17242 POINT(40.539486549971414 74.98132575699346) bank17242 +17243 POINT(40.68830684623892 74.70100090001179) bank17243 +17244 POINT(39.73134154296928 74.9380504222143) bank17244 +17245 POINT(40.37945798032216 73.91997558852275) bank17245 +17246 POINT(39.94280773693483 73.30179034792188) bank17246 +17247 POINT(40.82502482756013 74.08223619969465) bank17247 +17248 POINT(39.86391995833757 74.7822604559085) bank17248 +17249 POINT(40.571862787225854 74.54682625519708) bank17249 +17250 POINT(41.15959823917921 74.94469107484302) bank17250 +17251 POINT(40.52839677989789 73.03427114514376) bank17251 +17252 POINT(39.97939946387505 74.73855597540707) bank17252 +17253 POINT(40.05419336656346 73.27472750228655) bank17253 +17254 POINT(40.828989162940566 74.85976916274858) bank17254 +17255 POINT(40.107537848426944 74.46112439528221) bank17255 +17256 POINT(40.55113891655683 74.22960967202278) bank17256 +17257 POINT(39.77795292800405 73.74615662508722) bank17257 +17258 POINT(41.294845958993676 73.57798985688052) bank17258 +17259 POINT(39.870715128303054 73.15707154140445) bank17259 +17260 POINT(41.09610133248558 74.39195725934691) bank17260 +17261 POINT(40.35766654737554 74.20865183442996) bank17261 +17262 POINT(41.65348778880944 74.11212742770635) bank17262 +17263 POINT(39.999570750048235 74.17259012209806) bank17263 +17264 POINT(40.88178696914449 73.68191192495297) bank17264 +17265 POINT(40.671497534479656 74.42736725992413) bank17265 +17266 POINT(41.0268574187376 74.74764395359213) bank17266 +17267 POINT(40.78274876342961 73.25871490535171) bank17267 +17268 POINT(41.01867437149965 74.51521434121264) bank17268 +17269 POINT(40.738050494355996 73.11662957228008) bank17269 +17270 POINT(41.175051215741526 73.79267643085215) bank17270 +17271 POINT(39.73376525327802 74.55050019305908) bank17271 +17272 POINT(40.279081574446415 73.50875031554357) bank17272 +17273 POINT(40.77595521043921 73.31279264229056) bank17273 +17274 POINT(41.54196183234479 74.66085072517723) bank17274 +17275 POINT(40.77695999564797 74.20373431799148) bank17275 +17276 POINT(41.13689055556797 73.851260783198) bank17276 +17277 POINT(39.78345699591938 73.97792979107068) bank17277 +17278 POINT(40.26162348154571 74.4613468426548) bank17278 +17279 POINT(40.68519458527846 74.45387100998354) bank17279 +17280 POINT(40.50555588184816 73.78335116530857) bank17280 +17281 POINT(41.550484756465266 73.17691488539755) bank17281 +17282 POINT(40.79650300171161 74.37934862260829) bank17282 +17283 POINT(40.08693020341356 74.05110011829406) bank17283 +17284 POINT(40.09263030923313 73.93493307216418) bank17284 +17285 POINT(40.81218135375032 73.55038223603044) bank17285 +17286 POINT(40.8302704074466 74.49830348264757) bank17286 +17287 POINT(40.892052592737095 73.5998960277483) bank17287 +17288 POINT(40.50670350352875 73.19845221850476) bank17288 +17289 POINT(39.83383912994681 74.10856276439765) bank17289 +17290 POINT(40.36890048288636 73.78293679170781) bank17290 +17291 POINT(41.34447177634466 74.18403919506264) bank17291 +17292 POINT(41.473390710930424 74.40805574416916) bank17292 +17293 POINT(40.468889141687214 73.93424227445463) bank17293 +17294 POINT(40.37803077584523 74.84523708309257) bank17294 +17295 POINT(41.25961411444993 74.4170717429724) bank17295 +17296 POINT(40.668301032317665 74.3594684565105) bank17296 +17297 POINT(40.36181023765148 73.97834858246758) bank17297 +17298 POINT(40.04801272737624 74.46564085103508) bank17298 +17299 POINT(40.6327253432866 74.70454516610363) bank17299 +17300 POINT(40.227360336627534 74.22110945254617) bank17300 +17301 POINT(41.4406143206867 73.92246244769606) bank17301 +17302 POINT(39.76816450731796 74.76434197800687) bank17302 +17303 POINT(40.506960093455795 74.51245632802735) bank17303 +17304 POINT(41.517954629513454 74.93075055379123) bank17304 +17305 POINT(41.104181097321074 74.25278036927554) bank17305 +17306 POINT(41.49997811384921 73.84086658250756) bank17306 +17307 POINT(40.20844508914525 74.46127345685952) bank17307 +17308 POINT(40.07160894822511 74.19793171739144) bank17308 +17309 POINT(40.2022932619622 74.1131501727583) bank17309 +17310 POINT(39.777681853368314 73.8709595475306) bank17310 +17311 POINT(40.28832164753847 74.84719248467331) bank17311 +17312 POINT(39.75744174208255 74.61976188759466) bank17312 +17313 POINT(40.07103815590568 74.92912015954838) bank17313 +17314 POINT(41.190548405528894 74.54819491652673) bank17314 +17315 POINT(40.99738963942352 73.98554224297618) bank17315 +17316 POINT(40.791318751014025 73.37573478646081) bank17316 +17317 POINT(41.58706275299402 74.6687468901873) bank17317 +17318 POINT(40.45282290080855 74.14715359587613) bank17318 +17319 POINT(41.14353168757042 74.36592950574689) bank17319 +17320 POINT(40.19348018250853 73.25910524757808) bank17320 +17321 POINT(40.11660708906506 74.99928281325995) bank17321 +17322 POINT(40.23123906916513 73.8392465723658) bank17322 +17323 POINT(40.94780679361554 74.81358953810265) bank17323 +17324 POINT(40.01362396839124 74.19339221179018) bank17324 +17325 POINT(39.99971959378463 74.07857747060363) bank17325 +17326 POINT(41.705918471621075 74.11385431798988) bank17326 +17327 POINT(40.028771731879154 74.24360449673618) bank17327 +17328 POINT(41.16083885593952 73.53080038699372) bank17328 +17329 POINT(40.622900932760494 74.09842446327582) bank17329 +17330 POINT(40.20968573581188 74.78630499355526) bank17330 +17331 POINT(40.722845181962356 73.83615330499889) bank17331 +17332 POINT(40.37111978744393 73.95209243753479) bank17332 +17333 POINT(41.670756451590606 74.53926662412735) bank17333 +17334 POINT(39.86157452929775 73.05163974002338) bank17334 +17335 POINT(40.30789658539061 74.87830391589121) bank17335 +17336 POINT(41.56651415438836 73.81211682613973) bank17336 +17337 POINT(40.063079080707965 73.89301062789666) bank17337 +17338 POINT(40.120254753121564 74.09634669894126) bank17338 +17339 POINT(41.67004933089679 73.81099542052291) bank17339 +17340 POINT(40.441806134717964 73.49492562810251) bank17340 +17341 POINT(40.46292838376286 74.2906940724154) bank17341 +17342 POINT(41.41418204108045 74.63923714219314) bank17342 +17343 POINT(40.80584410513682 73.65863240857433) bank17343 +17344 POINT(41.043644143514385 74.12612431768322) bank17344 +17345 POINT(40.37749661860186 74.89532355906077) bank17345 +17346 POINT(41.18398925945591 74.73396864623375) bank17346 +17347 POINT(41.563052761883526 73.08514398366916) bank17347 +17348 POINT(41.12317080669716 73.5861778419046) bank17348 +17349 POINT(39.80306929144671 73.24128000333673) bank17349 +17350 POINT(41.203622647977234 74.6895297766484) bank17350 +17351 POINT(39.94782641949371 73.74772699098722) bank17351 +17352 POINT(40.70112072148571 73.83434729104155) bank17352 +17353 POINT(41.19304084682037 73.06598923357352) bank17353 +17354 POINT(41.553513905125904 73.41371979863935) bank17354 +17355 POINT(40.42219575898973 73.11153358851539) bank17355 +17356 POINT(40.92102532476943 74.9452790243905) bank17356 +17357 POINT(40.20666131423725 73.77438853997772) bank17357 +17358 POINT(41.097909382961035 74.57608097418556) bank17358 +17359 POINT(41.27959759518855 74.00763175238124) bank17359 +17360 POINT(40.68587784559255 73.99016798460711) bank17360 +17361 POINT(40.54489457929705 73.11811119563714) bank17361 +17362 POINT(41.17716561214831 73.47099437542386) bank17362 +17363 POINT(40.494350635670365 73.70930812428044) bank17363 +17364 POINT(41.597061834549294 74.13748759355157) bank17364 +17365 POINT(41.478971915151874 74.16954649815423) bank17365 +17366 POINT(41.28665220730399 73.90023751034423) bank17366 +17367 POINT(40.70680498696501 74.35999003501544) bank17367 +17368 POINT(41.01745676112096 74.33230743605374) bank17368 +17369 POINT(39.8507042386938 73.17011742158519) bank17369 +17370 POINT(39.96918042528082 74.253771145057) bank17370 +17371 POINT(40.97722219966413 74.29564010927788) bank17371 +17372 POINT(40.84182893548936 74.51085700055766) bank17372 +17373 POINT(41.108092354694826 73.34462858058633) bank17373 +17374 POINT(40.99977142309351 73.10344171534827) bank17374 +17375 POINT(40.93828730867651 73.24463058695628) bank17375 +17376 POINT(41.25681130132663 73.25884480313353) bank17376 +17377 POINT(41.25932905900331 74.74179478530672) bank17377 +17378 POINT(40.580540621690204 74.81441799160993) bank17378 +17379 POINT(40.54501307686556 74.54329425992229) bank17379 +17380 POINT(40.93580682864854 73.1146383397974) bank17380 +17381 POINT(40.912723748012574 73.22842941784748) bank17381 +17382 POINT(40.764890625425046 73.47045407506097) bank17382 +17383 POINT(40.42099372719665 73.08846198811933) bank17383 +17384 POINT(40.12439527850612 73.09716942584663) bank17384 +17385 POINT(41.71195630352412 73.18864431913484) bank17385 +17386 POINT(40.36353737906202 74.79193576732114) bank17386 +17387 POINT(39.85326097801676 74.72486386818836) bank17387 +17388 POINT(41.356919706083865 73.60709385128531) bank17388 +17389 POINT(40.480114657907805 73.71464441101317) bank17389 +17390 POINT(40.6006547257123 74.05676770261371) bank17390 +17391 POINT(40.608835134092445 73.5368647804634) bank17391 +17392 POINT(39.83029582973987 74.02038304604676) bank17392 +17393 POINT(41.49056007907687 74.11159978396785) bank17393 +17394 POINT(41.12864296033661 74.01164007278422) bank17394 +17395 POINT(41.51134867032647 73.02939649148485) bank17395 +17396 POINT(40.25680691852923 74.22667487012284) bank17396 +17397 POINT(41.48719141223867 74.91306489316722) bank17397 +17398 POINT(40.16685880120653 74.35040715461274) bank17398 +17399 POINT(39.951620056664744 74.27847783519655) bank17399 +17400 POINT(41.25000411097428 73.36506088797357) bank17400 +17401 POINT(39.77443889306159 74.05815643098543) bank17401 +17402 POINT(40.55294188630283 73.72167690791586) bank17402 +17403 POINT(41.29597791697816 74.73874846297745) bank17403 +17404 POINT(40.65012163439082 73.62255611861413) bank17404 +17405 POINT(40.21006744652289 74.33072274738959) bank17405 +17406 POINT(40.624887597233126 74.15645545934335) bank17406 +17407 POINT(41.02664093492862 74.66018824261326) bank17407 +17408 POINT(39.86265360223789 73.12070173711571) bank17408 +17409 POINT(41.383513171045266 73.03842842740501) bank17409 +17410 POINT(40.93325760662281 74.77973797701772) bank17410 +17411 POINT(40.824641025010095 73.0867168334443) bank17411 +17412 POINT(40.09368559952881 74.66967516825227) bank17412 +17413 POINT(39.87372978790383 73.97088774504397) bank17413 +17414 POINT(40.661566566994175 74.83809825902092) bank17414 +17415 POINT(40.239283876778245 73.52155635750655) bank17415 +17416 POINT(39.963990986969 73.4993422050611) bank17416 +17417 POINT(39.71832139270874 74.07433279811791) bank17417 +17418 POINT(40.98333945894278 74.16030601296434) bank17418 +17419 POINT(40.72189586700783 74.718323921958) bank17419 +17420 POINT(40.16260367368486 73.5536164207715) bank17420 +17421 POINT(40.632679480426596 74.25741438345086) bank17421 +17422 POINT(41.54029630374827 74.5839067285607) bank17422 +17423 POINT(39.84098927149811 74.72145377754583) bank17423 +17424 POINT(40.25324032494079 73.40580370337679) bank17424 +17425 POINT(41.318664697829604 74.13431873626202) bank17425 +17426 POINT(40.51244802680219 73.0257697505501) bank17426 +17427 POINT(41.10300753306084 74.20319330883902) bank17427 +17428 POINT(41.23289438363858 73.33519930758649) bank17428 +17429 POINT(39.78122263050051 74.83520207137035) bank17429 +17430 POINT(40.08457182312252 73.84791076723523) bank17430 +17431 POINT(41.52117416935017 74.5125719862279) bank17431 +17432 POINT(40.739138806340264 74.33392623149138) bank17432 +17433 POINT(41.44921308242132 74.14178978704271) bank17433 +17434 POINT(41.25000103002157 73.58547048325309) bank17434 +17435 POINT(41.605711054872046 74.88510325343252) bank17435 +17436 POINT(41.21728474444643 74.32013824429781) bank17436 +17437 POINT(40.47330569459549 74.31385127924108) bank17437 +17438 POINT(40.358025359651855 73.28808280252095) bank17438 +17439 POINT(41.6173850264054 74.06916147973354) bank17439 +17440 POINT(40.67876313875003 73.92787477679374) bank17440 +17441 POINT(40.72297626079094 74.55243352675106) bank17441 +17442 POINT(41.62794378718339 73.50894965056568) bank17442 +17443 POINT(41.54873050111774 73.40084972784976) bank17443 +17444 POINT(41.70346551729817 73.4311623968365) bank17444 +17445 POINT(41.36109220411488 74.91300452440147) bank17445 +17446 POINT(40.48904752495341 73.05530238596339) bank17446 +17447 POINT(41.303433867512794 73.03767023540355) bank17447 +17448 POINT(41.30625742885783 74.30006161273407) bank17448 +17449 POINT(40.384676120851154 73.80337980339952) bank17449 +17450 POINT(40.007159812887366 74.03610444638942) bank17450 +17451 POINT(41.205383843133134 74.92270017277315) bank17451 +17452 POINT(40.421439195023595 74.5257131125889) bank17452 +17453 POINT(40.85495882250189 74.06465473488312) bank17453 +17454 POINT(40.510773874727924 73.45534601921008) bank17454 +17455 POINT(40.66267588748318 74.77096910332024) bank17455 +17456 POINT(40.469943067970206 73.41268049340175) bank17456 +17457 POINT(40.76200054436584 74.25876517363733) bank17457 +17458 POINT(40.605484170473765 74.91715168100663) bank17458 +17459 POINT(41.55977971220632 74.63153001848244) bank17459 +17460 POINT(41.12003364456502 73.35492859013588) bank17460 +17461 POINT(41.66821412281016 74.48435344013338) bank17461 +17462 POINT(40.987736352853496 73.84440638629873) bank17462 +17463 POINT(41.42346731672592 73.59636328140938) bank17463 +17464 POINT(41.708720275823545 74.33934541809097) bank17464 +17465 POINT(41.573766715176454 73.85863374026957) bank17465 +17466 POINT(41.46938837257706 73.90768799342088) bank17466 +17467 POINT(41.58212166355423 74.8697381610861) bank17467 +17468 POINT(40.861432132922964 74.73602585414143) bank17468 +17469 POINT(40.248417952886676 73.19594884974123) bank17469 +17470 POINT(40.70899165861384 74.68502717249764) bank17470 +17471 POINT(40.19904114534939 73.4565922219373) bank17471 +17472 POINT(39.88481101599339 73.0184866638459) bank17472 +17473 POINT(41.711212098127426 74.62464543518463) bank17473 +17474 POINT(39.95865888962781 74.1819670736226) bank17474 +17475 POINT(39.86744536858882 73.24231970058764) bank17475 +17476 POINT(39.97687815685086 74.75211872490272) bank17476 +17477 POINT(40.50269973035889 73.24540973885364) bank17477 +17478 POINT(39.71852885510915 74.36921762326655) bank17478 +17479 POINT(41.39588290306182 74.63950563352431) bank17479 +17480 POINT(39.83407533307125 74.93775999277202) bank17480 +17481 POINT(41.236922363814806 74.18629870916472) bank17481 +17482 POINT(40.041948819388494 74.80248204281666) bank17482 +17483 POINT(40.069966501683275 73.88828534744255) bank17483 +17484 POINT(41.089785308080785 74.2149813389365) bank17484 +17485 POINT(40.69039608160328 73.74813656417349) bank17485 +17486 POINT(40.466406342393825 73.82100433514631) bank17486 +17487 POINT(39.94696896299514 74.26169190830531) bank17487 +17488 POINT(39.7763174565751 73.68448010867566) bank17488 +17489 POINT(40.11173914591359 74.15229387735937) bank17489 +17490 POINT(41.016549455826734 73.21746263531023) bank17490 +17491 POINT(40.3705667116515 73.31567537539354) bank17491 +17492 POINT(41.46437925363382 73.21404079998428) bank17492 +17493 POINT(40.95459061765716 74.78422208399274) bank17493 +17494 POINT(41.260262054282045 74.94912873605834) bank17494 +17495 POINT(41.68147831135238 74.05571505012483) bank17495 +17496 POINT(39.73021837781406 73.93651962734026) bank17496 +17497 POINT(40.785152087883944 74.32248070739503) bank17497 +17498 POINT(41.14137799026197 73.33529705527995) bank17498 +17499 POINT(39.944881591679746 74.94945425118969) bank17499 +17500 POINT(40.92569149894262 73.81282493722787) bank17500 +17501 POINT(41.508325403687905 73.76957421167612) bank17501 +17502 POINT(41.26392791746169 73.41229131436906) bank17502 +17503 POINT(41.266792514294934 73.52992867548826) bank17503 +17504 POINT(40.203419775315865 74.78324559228452) bank17504 +17505 POINT(40.81918893915234 74.61020082272144) bank17505 +17506 POINT(40.889226822387954 73.13229556140077) bank17506 +17507 POINT(41.57435986362852 73.882865521201) bank17507 +17508 POINT(40.748743201527994 74.8731304218411) bank17508 +17509 POINT(41.46698844515086 73.25585305792633) bank17509 +17510 POINT(40.715646080924856 74.9453523902594) bank17510 +17511 POINT(41.33077949176847 74.69840273958847) bank17511 +17512 POINT(41.51745555242249 74.03883185046503) bank17512 +17513 POINT(40.00595828605907 73.35707848667096) bank17513 +17514 POINT(41.16091244305242 73.24156637274096) bank17514 +17515 POINT(40.85123855843816 74.95303820652863) bank17515 +17516 POINT(41.168277028323864 74.90106539407414) bank17516 +17517 POINT(41.31434329091391 74.38090203499478) bank17517 +17518 POINT(40.23013708988611 74.15797400168971) bank17518 +17519 POINT(41.48225354320561 73.46664722914092) bank17519 +17520 POINT(41.23588341256342 74.75237766875249) bank17520 +17521 POINT(39.768507267544805 73.19382010116945) bank17521 +17522 POINT(40.959986973884405 74.76491749751719) bank17522 +17523 POINT(40.58317728938515 73.04001002568538) bank17523 +17524 POINT(40.49116878834689 73.01559526273729) bank17524 +17525 POINT(40.14833432842512 73.42725389873594) bank17525 +17526 POINT(40.99480777844735 73.82176153979687) bank17526 +17527 POINT(41.07078833032298 73.69311176361664) bank17527 +17528 POINT(40.11066981115597 73.06396521731236) bank17528 +17529 POINT(41.38985820663948 73.18638398747304) bank17529 +17530 POINT(41.024648870197716 74.73898368492812) bank17530 +17531 POINT(40.11270389362838 73.43200437384102) bank17531 +17532 POINT(40.77405713414607 73.53622465503265) bank17532 +17533 POINT(40.90786313212969 74.26249016255137) bank17533 +17534 POINT(40.729217931286456 73.7365836057626) bank17534 +17535 POINT(41.48762859529625 73.62117614072582) bank17535 +17536 POINT(40.39382728636995 73.83576725313281) bank17536 +17537 POINT(41.117198472137765 73.88703287506824) bank17537 +17538 POINT(40.94296024259365 74.44053067311044) bank17538 +17539 POINT(40.542396077966536 74.30530322152791) bank17539 +17540 POINT(41.49266698028006 74.06479572524113) bank17540 +17541 POINT(40.61811770418633 74.57265711713517) bank17541 +17542 POINT(39.9447242265436 73.10828531829229) bank17542 +17543 POINT(41.319219339495504 74.99912969989813) bank17543 +17544 POINT(40.11119765665661 74.3410367916367) bank17544 +17545 POINT(40.9103375007614 73.20898719948943) bank17545 +17546 POINT(40.88006091051835 73.54260926518745) bank17546 +17547 POINT(40.817852966458226 74.1114916143191) bank17547 +17548 POINT(40.799981349732995 74.48042807247754) bank17548 +17549 POINT(41.124602596872805 73.5831655814335) bank17549 +17550 POINT(41.28252208323404 73.25910557938961) bank17550 +17551 POINT(40.84939251428851 73.72468948739193) bank17551 +17552 POINT(40.97342102286237 74.34071194360732) bank17552 +17553 POINT(41.397519113036665 74.49329566000426) bank17553 +17554 POINT(40.54826400774665 74.70552078166482) bank17554 +17555 POINT(41.40909497451695 73.25165715616104) bank17555 +17556 POINT(41.33527814656098 73.29732559815744) bank17556 +17557 POINT(40.063649720141626 73.50032194215296) bank17557 +17558 POINT(40.99756249201728 73.31423710035395) bank17558 +17559 POINT(40.611132400988666 73.33718933988544) bank17559 +17560 POINT(40.958185723639325 74.98374681513677) bank17560 +17561 POINT(39.90402825109663 73.22680747649706) bank17561 +17562 POINT(40.04779528874813 74.3076654178355) bank17562 +17563 POINT(40.968330623494346 73.9034373672907) bank17563 +17564 POINT(41.541380573207256 74.5986993491396) bank17564 +17565 POINT(41.298295768236656 73.58785348010163) bank17565 +17566 POINT(40.1109021311795 74.8167517223572) bank17566 +17567 POINT(39.989683179491614 74.72034981249098) bank17567 +17568 POINT(40.462196697116816 73.2970200657707) bank17568 +17569 POINT(39.92058402536688 74.55025765443223) bank17569 +17570 POINT(40.16065731953761 74.7599313003834) bank17570 +17571 POINT(40.97044814819513 73.14238123374291) bank17571 +17572 POINT(40.09819553802728 73.73979056134498) bank17572 +17573 POINT(41.24772128955587 73.85016340799574) bank17573 +17574 POINT(41.032402935009245 73.88077219421076) bank17574 +17575 POINT(40.98242446881239 74.15810884699466) bank17575 +17576 POINT(40.81921075027986 73.32582201882313) bank17576 +17577 POINT(41.19653240264079 74.25056791252513) bank17577 +17578 POINT(40.48680756409231 73.50610994028774) bank17578 +17579 POINT(40.70872986072215 73.4922501350552) bank17579 +17580 POINT(40.93597523524681 74.26822883294527) bank17580 +17581 POINT(39.84645946576582 74.13857345647158) bank17581 +17582 POINT(40.61278609033143 73.81280093888049) bank17582 +17583 POINT(39.97882805010674 73.56866882178961) bank17583 +17584 POINT(41.68533577639475 74.24507778491925) bank17584 +17585 POINT(41.00904476139918 74.35900257711847) bank17585 +17586 POINT(40.238843470054164 74.34436884249891) bank17586 +17587 POINT(40.40525143925977 74.48560271975994) bank17587 +17588 POINT(41.475483890313114 74.08193617918964) bank17588 +17589 POINT(40.99281501721378 73.38725357619015) bank17589 +17590 POINT(40.996070515444515 73.43893485742939) bank17590 +17591 POINT(40.59873495492654 74.56784566094949) bank17591 +17592 POINT(40.59917859057822 74.85567388346995) bank17592 +17593 POINT(39.82223402883751 74.99820923211895) bank17593 +17594 POINT(41.43657877924431 73.89372145074704) bank17594 +17595 POINT(40.16907689988999 74.42932242389344) bank17595 +17596 POINT(41.290749672534716 73.57622194524262) bank17596 +17597 POINT(40.29264660212151 73.79436715574323) bank17597 +17598 POINT(41.622697795182305 73.90134412911007) bank17598 +17599 POINT(41.213826053927534 74.96189143404337) bank17599 +17600 POINT(41.20205620895755 74.5985685200567) bank17600 +17601 POINT(41.62875120160746 73.1066728394725) bank17601 +17602 POINT(39.768611994567465 74.99146595454528) bank17602 +17603 POINT(41.39524162311575 74.051772131451) bank17603 +17604 POINT(40.202396946137064 73.62413406256874) bank17604 +17605 POINT(40.427759925772335 73.2530888694415) bank17605 +17606 POINT(40.29052949909178 74.63424200664605) bank17606 +17607 POINT(39.997356671584754 74.87191571839669) bank17607 +17608 POINT(41.50750732716788 74.91272428101813) bank17608 +17609 POINT(40.660580546322954 74.52784165761025) bank17609 +17610 POINT(40.144898201153 74.40653039546879) bank17610 +17611 POINT(40.47337856484752 73.41397802368485) bank17611 +17612 POINT(41.28153271941582 73.64104237947818) bank17612 +17613 POINT(40.85673523446007 73.92740729014038) bank17613 +17614 POINT(41.00672671411843 73.74594083508167) bank17614 +17615 POINT(40.18745926274549 74.89236799732231) bank17615 +17616 POINT(40.01732022030669 74.22913868251864) bank17616 +17617 POINT(41.0646442377602 74.95594677690738) bank17617 +17618 POINT(40.703997749017425 74.8864836750657) bank17618 +17619 POINT(41.59835855103127 74.82724800145414) bank17619 +17620 POINT(41.68535199382029 74.90088705794686) bank17620 +17621 POINT(40.73478842047248 73.9275613837662) bank17621 +17622 POINT(41.65655078911531 74.44705688745883) bank17622 +17623 POINT(40.02215551869076 73.08167889673399) bank17623 +17624 POINT(40.047646335531184 74.41902644890254) bank17624 +17625 POINT(39.9361258338467 74.43118835489388) bank17625 +17626 POINT(41.07904775170941 74.80642558299911) bank17626 +17627 POINT(40.74362260836776 73.68866920551147) bank17627 +17628 POINT(40.15416188766042 73.28067529890585) bank17628 +17629 POINT(40.624126175243276 74.95444329266142) bank17629 +17630 POINT(41.651326097525065 74.1793580363936) bank17630 +17631 POINT(40.239859926738106 74.5170887959305) bank17631 +17632 POINT(40.131094512800196 74.42812293797948) bank17632 +17633 POINT(40.148007124710404 74.58526112170489) bank17633 +17634 POINT(41.41944041067192 74.82522066836806) bank17634 +17635 POINT(41.116174486887985 73.22145163506501) bank17635 +17636 POINT(41.22093276054888 73.68798296898555) bank17636 +17637 POINT(41.64537364297399 74.42653247797644) bank17637 +17638 POINT(41.60830269908601 74.12082445124108) bank17638 +17639 POINT(40.20077158122179 74.7885120332029) bank17639 +17640 POINT(41.36372678608938 73.16002880245908) bank17640 +17641 POINT(41.16790327182395 73.49732857547383) bank17641 +17642 POINT(41.18841883470878 74.92321464401117) bank17642 +17643 POINT(41.24831476192221 74.43966334115213) bank17643 +17644 POINT(40.1524266213506 74.22838665804291) bank17644 +17645 POINT(41.52908873659748 73.22093959153915) bank17645 +17646 POINT(40.91886197132294 74.74016353391076) bank17646 +17647 POINT(40.204718693201315 74.24395671098652) bank17647 +17648 POINT(40.322021410529025 74.38674999477222) bank17648 +17649 POINT(40.11218186699308 74.38542675574463) bank17649 +17650 POINT(41.15225021166901 74.79658158815286) bank17650 +17651 POINT(41.4201766011743 73.70864608050785) bank17651 +17652 POINT(40.93834497690216 74.97407384637683) bank17652 +17653 POINT(40.63734029095024 74.43839920309611) bank17653 +17654 POINT(41.69590028425145 74.22856236274849) bank17654 +17655 POINT(40.082393412995906 73.39782665341515) bank17655 +17656 POINT(41.65308489501328 74.83781806955878) bank17656 +17657 POINT(40.59631887761907 73.36524142227253) bank17657 +17658 POINT(40.05275348663889 74.30889381768559) bank17658 +17659 POINT(41.09177498481163 73.00655599233701) bank17659 +17660 POINT(40.067890684432456 73.50501393144205) bank17660 +17661 POINT(40.988727059619514 73.6636046017384) bank17661 +17662 POINT(41.17738095753112 73.75848093422556) bank17662 +17663 POINT(39.949415229830684 73.83577587418564) bank17663 +17664 POINT(40.24755981345656 73.33586732620243) bank17664 +17665 POINT(41.24348807790165 73.80228003165016) bank17665 +17666 POINT(41.11380258057893 74.52463576155898) bank17666 +17667 POINT(41.21386607746823 74.44598157069986) bank17667 +17668 POINT(40.01767201033783 74.63055921649168) bank17668 +17669 POINT(40.33522368194392 73.93901889816007) bank17669 +17670 POINT(39.836659529251364 73.42946223972768) bank17670 +17671 POINT(40.98018259682515 74.80523172040503) bank17671 +17672 POINT(39.94106420143323 74.46360186688574) bank17672 +17673 POINT(39.93776812441183 74.27492184301093) bank17673 +17674 POINT(41.265767517164484 73.20764057045486) bank17674 +17675 POINT(41.43841113631715 74.14601572074896) bank17675 +17676 POINT(41.32917915039892 73.7962805571795) bank17676 +17677 POINT(40.846508391527955 74.93252420670133) bank17677 +17678 POINT(40.2873613784231 74.7966949016084) bank17678 +17679 POINT(41.052915763811995 73.35807097835902) bank17679 +17680 POINT(41.680776826890494 73.42564989787047) bank17680 +17681 POINT(40.0702774095992 73.24190786157574) bank17681 +17682 POINT(41.30704446186299 73.02735747347134) bank17682 +17683 POINT(40.50054748475685 73.48042442916898) bank17683 +17684 POINT(39.86719426468195 73.14008775246825) bank17684 +17685 POINT(41.487307555052936 73.81913074675154) bank17685 +17686 POINT(40.37779026654996 74.13845056030205) bank17686 +17687 POINT(39.93352651826098 73.71637419073066) bank17687 +17688 POINT(40.07944590501652 74.16145403144378) bank17688 +17689 POINT(40.63817176065181 74.42380983312476) bank17689 +17690 POINT(41.20087657809012 74.76389750848969) bank17690 +17691 POINT(40.29216514791183 74.15205925830537) bank17691 +17692 POINT(40.49834275766074 74.612051344344) bank17692 +17693 POINT(40.982465905863044 73.0318591645433) bank17693 +17694 POINT(41.01793554089852 74.06221311568959) bank17694 +17695 POINT(40.20229524551137 73.83979975405425) bank17695 +17696 POINT(41.63393128103606 73.89051008637603) bank17696 +17697 POINT(40.12255049152074 74.75192128033643) bank17697 +17698 POINT(39.91597877889121 73.63205332666868) bank17698 +17699 POINT(41.09709589337379 74.19383254788188) bank17699 +17700 POINT(41.293179262849776 73.35834785724578) bank17700 +17701 POINT(40.007457045296285 74.98721544319677) bank17701 +17702 POINT(41.296639036277014 73.16562805529611) bank17702 +17703 POINT(41.71142745343956 73.00893892578144) bank17703 +17704 POINT(40.55525133325195 73.70948197143957) bank17704 +17705 POINT(41.60076274672639 73.90964927198223) bank17705 +17706 POINT(40.34434646066635 73.54127119637685) bank17706 +17707 POINT(41.712245206834304 74.0163076224652) bank17707 +17708 POINT(40.3090602754757 73.66913335457055) bank17708 +17709 POINT(41.32353377059173 74.36191851377141) bank17709 +17710 POINT(41.057023429056834 73.58889649849066) bank17710 +17711 POINT(41.37383109690878 73.46817959832471) bank17711 +17712 POINT(41.03656754560891 73.45214627501385) bank17712 +17713 POINT(40.84373207627554 74.50961672798203) bank17713 +17714 POINT(40.2780600009625 74.80675982728998) bank17714 +17715 POINT(40.448455741952934 73.49737371985279) bank17715 +17716 POINT(41.63249804944985 73.65267666190795) bank17716 +17717 POINT(41.089131534815074 73.1732900017734) bank17717 +17718 POINT(41.56291838753909 73.94194344058651) bank17718 +17719 POINT(40.48000742280642 74.59342519619192) bank17719 +17720 POINT(41.31604312168377 73.35266082796164) bank17720 +17721 POINT(40.20964916647719 73.7987989703931) bank17721 +17722 POINT(40.414445157343714 73.63125829643192) bank17722 +17723 POINT(41.63291422780218 73.62134391816468) bank17723 +17724 POINT(39.99506273081678 74.09492209650033) bank17724 +17725 POINT(40.013057690203844 74.14082610857074) bank17725 +17726 POINT(40.36413809822472 74.22151616352343) bank17726 +17727 POINT(41.12544813671624 74.00106693068227) bank17727 +17728 POINT(40.900627774179895 74.10098980193753) bank17728 +17729 POINT(40.330353071401134 74.28632578061624) bank17729 +17730 POINT(39.7182687207038 74.37435918879922) bank17730 +17731 POINT(41.47851847623628 73.64551179862534) bank17731 +17732 POINT(41.12051885628517 73.04358858957983) bank17732 +17733 POINT(41.61392034206491 73.99836982230472) bank17733 +17734 POINT(40.17660269656391 73.0348170622215) bank17734 +17735 POINT(40.740044081190064 74.69969753442157) bank17735 +17736 POINT(40.95219163705201 74.12234762283795) bank17736 +17737 POINT(41.03943992113679 73.40701977565467) bank17737 +17738 POINT(40.326266643309715 74.17363619537979) bank17738 +17739 POINT(40.539789511233955 73.31065165813463) bank17739 +17740 POINT(41.43709188059351 74.84318637771257) bank17740 +17741 POINT(40.63141647259157 74.81412519943919) bank17741 +17742 POINT(40.809840906243096 73.88759151704144) bank17742 +17743 POINT(40.35122511761937 74.98085724663059) bank17743 +17744 POINT(40.84297580191317 74.23207075414946) bank17744 +17745 POINT(40.85679573418029 73.51616702483084) bank17745 +17746 POINT(41.45865843128527 73.65738076492102) bank17746 +17747 POINT(40.373218831116795 73.49305067987979) bank17747 +17748 POINT(39.913106683868115 74.46154692908247) bank17748 +17749 POINT(41.54129673189143 73.89708684647213) bank17749 +17750 POINT(41.14023157680783 73.63054793708976) bank17750 +17751 POINT(41.06339046327313 74.25476174342931) bank17751 +17752 POINT(40.06289041152177 73.25797180575765) bank17752 +17753 POINT(40.24231384855375 74.99516042166832) bank17753 +17754 POINT(41.54978227103949 74.36409794577254) bank17754 +17755 POINT(41.63877684650792 73.37784621934037) bank17755 +17756 POINT(39.82236062812243 74.15362612029546) bank17756 +17757 POINT(39.94548297435142 74.02569943076544) bank17757 +17758 POINT(40.93805253910329 74.93501900600941) bank17758 +17759 POINT(41.29408828194457 73.7745409890128) bank17759 +17760 POINT(39.98996867868923 74.81392240625247) bank17760 +17761 POINT(40.2233609394027 74.79179367731334) bank17761 +17762 POINT(40.91107201144417 74.77493284449258) bank17762 +17763 POINT(40.30829555943278 74.29742121902393) bank17763 +17764 POINT(40.20821620560155 73.28216508940007) bank17764 +17765 POINT(41.00979079077365 73.19490544507516) bank17765 +17766 POINT(40.01014525051844 73.88919826348759) bank17766 +17767 POINT(40.75503491985133 74.55643426299056) bank17767 +17768 POINT(41.37553892318433 73.24769531503567) bank17768 +17769 POINT(41.22914652400261 73.44481044913336) bank17769 +17770 POINT(41.69050630496726 74.8960586239947) bank17770 +17771 POINT(39.96813402746365 74.28533467263578) bank17771 +17772 POINT(40.321334364591394 74.05664418684897) bank17772 +17773 POINT(41.26083920310085 73.46719840091146) bank17773 +17774 POINT(40.210886734020725 74.25776245090528) bank17774 +17775 POINT(40.937373615579595 73.50866502111542) bank17775 +17776 POINT(41.357542191368026 74.74478566523526) bank17776 +17777 POINT(40.207261062953286 74.80505273605571) bank17777 +17778 POINT(41.60614195427354 73.91972871122584) bank17778 +17779 POINT(41.58714581453923 73.68919674942003) bank17779 +17780 POINT(41.480596636185055 73.50842152497043) bank17780 +17781 POINT(41.33342987244866 73.74462592786459) bank17781 +17782 POINT(41.281029759683236 74.26494782788802) bank17782 +17783 POINT(40.296072429339645 73.74218807307314) bank17783 +17784 POINT(39.77207514446635 74.13589562503091) bank17784 +17785 POINT(41.02159985535688 73.83860346280403) bank17785 +17786 POINT(39.751095675407136 73.52647748896258) bank17786 +17787 POINT(41.005844805375894 74.18038131870695) bank17787 +17788 POINT(41.04508565821414 73.9782872189521) bank17788 +17789 POINT(41.3498669604023 74.7741106447776) bank17789 +17790 POINT(40.386239041169674 73.19942480131651) bank17790 +17791 POINT(40.33127212770952 73.70550358679228) bank17791 +17792 POINT(40.527815194135485 73.79852746195802) bank17792 +17793 POINT(41.27240852824587 74.61211809817807) bank17793 +17794 POINT(40.67449552058767 74.32337067071312) bank17794 +17795 POINT(41.04032720590249 73.58620954641908) bank17795 +17796 POINT(39.9062105698635 73.0376572147524) bank17796 +17797 POINT(41.208430843155696 75.0022256648904) bank17797 +17798 POINT(41.02416178986541 73.32010132129412) bank17798 +17799 POINT(40.81642803238222 73.95726885183791) bank17799 +17800 POINT(40.72051997797511 74.25276441318698) bank17800 +17801 POINT(41.05995553787142 73.81926346816422) bank17801 +17802 POINT(41.32412661843637 73.8383875670835) bank17802 +17803 POINT(41.25322957077331 73.57092024841809) bank17803 +17804 POINT(41.28603679373585 73.16028962559972) bank17804 +17805 POINT(40.43309465963899 73.70810876950895) bank17805 +17806 POINT(40.826524025575274 74.91186102697395) bank17806 +17807 POINT(41.04181489746246 74.79409502940177) bank17807 +17808 POINT(41.50134218175831 73.67380787161383) bank17808 +17809 POINT(39.73583152299422 74.80870760649485) bank17809 +17810 POINT(40.63086583310772 74.14066495226587) bank17810 +17811 POINT(40.00759863117186 74.42844491823371) bank17811 +17812 POINT(39.76212472324788 73.21788507205444) bank17812 +17813 POINT(41.57995080840406 74.07073866894805) bank17813 +17814 POINT(39.934700496471336 74.61734314826309) bank17814 +17815 POINT(40.41391275130876 74.03596942836349) bank17815 +17816 POINT(41.64815517400369 73.84480528894812) bank17816 +17817 POINT(40.30152246853663 74.13714342007516) bank17817 +17818 POINT(41.691370210039175 73.35172736384555) bank17818 +17819 POINT(39.772927465604084 73.9354333281916) bank17819 +17820 POINT(41.29013547415767 73.52667271509482) bank17820 +17821 POINT(41.04200250743545 73.16813270031439) bank17821 +17822 POINT(40.581230055609986 74.34394956717544) bank17822 +17823 POINT(39.82847526293743 73.06077474867875) bank17823 +17824 POINT(41.29358116635787 74.21845994423504) bank17824 +17825 POINT(40.7646435052955 73.54380207369186) bank17825 +17826 POINT(39.744494691199236 74.36783656450973) bank17826 +17827 POINT(41.05711994219474 74.45135223321407) bank17827 +17828 POINT(40.94131546526104 74.87154001201478) bank17828 +17829 POINT(40.68643136852985 73.1393423115125) bank17829 +17830 POINT(40.70654026485416 74.36776159093587) bank17830 +17831 POINT(41.66534206455917 74.60235602227901) bank17831 +17832 POINT(40.07861075952982 73.21506129673087) bank17832 +17833 POINT(40.684275425710915 73.21169175690498) bank17833 +17834 POINT(39.71663492731929 74.06747363289502) bank17834 +17835 POINT(41.53527243941669 73.49978081578492) bank17835 +17836 POINT(39.927617645694454 73.59225400665754) bank17836 +17837 POINT(41.15610012606292 73.7336768431682) bank17837 +17838 POINT(40.2926266148222 74.1223027017242) bank17838 +17839 POINT(40.10544752106091 74.10318908585342) bank17839 +17840 POINT(39.91662411387568 74.50959643748064) bank17840 +17841 POINT(40.18991633805142 73.72865430411552) bank17841 +17842 POINT(39.910486601928966 73.27600975847088) bank17842 +17843 POINT(39.71730180450637 73.80750431949875) bank17843 +17844 POINT(40.41630115530072 74.12387292475141) bank17844 +17845 POINT(41.49351790741884 74.86467589603481) bank17845 +17846 POINT(41.14445338929332 73.27170765999638) bank17846 +17847 POINT(41.037446268670344 73.37219122550631) bank17847 +17848 POINT(40.23162911093908 73.25612251686958) bank17848 +17849 POINT(40.268861855817505 74.58031151516462) bank17849 +17850 POINT(41.635675953950425 74.02168619232486) bank17850 +17851 POINT(39.80049209712314 74.08775133839232) bank17851 +17852 POINT(40.788092658405134 74.13451494471184) bank17852 +17853 POINT(40.22423870524735 74.29917174951787) bank17853 +17854 POINT(41.671032381489255 74.7684667766223) bank17854 +17855 POINT(40.55246798746437 73.51675330828975) bank17855 +17856 POINT(40.80350730648424 74.7735006852032) bank17856 +17857 POINT(40.971477211804995 73.81771647119483) bank17857 +17858 POINT(40.08952238058181 73.88726526129604) bank17858 +17859 POINT(41.05162538930153 74.71475992398126) bank17859 +17860 POINT(40.60299009925083 74.78094832804273) bank17860 +17861 POINT(41.23261736616452 73.89934142711816) bank17861 +17862 POINT(40.97286303272989 74.25128537286933) bank17862 +17863 POINT(41.31327968662216 74.35551108147558) bank17863 +17864 POINT(40.31845248910974 73.49364447487814) bank17864 +17865 POINT(41.525697168840765 74.79738030755182) bank17865 +17866 POINT(40.62908868922252 74.57109088962484) bank17866 +17867 POINT(40.515583786360715 74.96056582590461) bank17867 +17868 POINT(41.518218374952774 74.60269773575048) bank17868 +17869 POINT(40.28612868049429 73.12206003978365) bank17869 +17870 POINT(41.11195892785036 74.31599820414766) bank17870 +17871 POINT(40.82618302357106 74.11219752072961) bank17871 +17872 POINT(40.451496129935826 74.08928982625744) bank17872 +17873 POINT(39.89911075763652 74.20310236471427) bank17873 +17874 POINT(39.93974592111035 74.75382424063712) bank17874 +17875 POINT(41.3437417245154 74.70612802235635) bank17875 +17876 POINT(41.50286511046598 74.30683952694535) bank17876 +17877 POINT(41.266658129813614 73.64786905177446) bank17877 +17878 POINT(40.845634509666745 74.53007805996398) bank17878 +17879 POINT(40.786434955149936 74.47154677031604) bank17879 +17880 POINT(40.62578478497016 74.46754091992842) bank17880 +17881 POINT(40.104020685701386 73.22143501144353) bank17881 +17882 POINT(41.65083513141906 73.86608343070161) bank17882 +17883 POINT(40.376496719573396 74.37760832532494) bank17883 +17884 POINT(40.29363874501983 74.51885482623561) bank17884 +17885 POINT(40.67906201520561 74.68115429246957) bank17885 +17886 POINT(40.50511486524959 74.4246675484253) bank17886 +17887 POINT(41.28311852607898 74.39453802455081) bank17887 +17888 POINT(40.00320810143659 74.1904827248848) bank17888 +17889 POINT(39.88562360545499 74.64366571983659) bank17889 +17890 POINT(40.4776883395713 73.58172146789914) bank17890 +17891 POINT(40.8812898115218 74.17292393161236) bank17891 +17892 POINT(41.32018486936963 73.39650472519114) bank17892 +17893 POINT(41.1708931500887 73.81893130606782) bank17893 +17894 POINT(41.20150546873389 73.96604255366998) bank17894 +17895 POINT(39.934929382312795 73.56712316783518) bank17895 +17896 POINT(40.58155932670913 74.96493405614093) bank17896 +17897 POINT(41.222375852259646 73.64439272168289) bank17897 +17898 POINT(40.777672828999876 74.06560262418195) bank17898 +17899 POINT(41.51451211741649 74.2532187989251) bank17899 +17900 POINT(39.95119073366985 74.42458727826802) bank17900 +17901 POINT(41.680454293221274 73.83683749889214) bank17901 +17902 POINT(40.36006792001596 74.09489695354848) bank17902 +17903 POINT(40.866677374495666 74.47738438881397) bank17903 +17904 POINT(40.0041110893392 73.91877097788564) bank17904 +17905 POINT(41.30028074948367 74.38100635723221) bank17905 +17906 POINT(41.604494124057155 73.10867572008885) bank17906 +17907 POINT(41.17968893169382 73.73629498173236) bank17907 +17908 POINT(41.468821337526954 73.08048943101292) bank17908 +17909 POINT(40.5788373447568 73.8642557725185) bank17909 +17910 POINT(41.255322354397016 74.41758802769432) bank17910 +17911 POINT(40.71613845211466 73.2419749081722) bank17911 +17912 POINT(41.65699315323502 73.92526386672199) bank17912 +17913 POINT(40.078158183513914 74.51736112914054) bank17913 +17914 POINT(40.84356343019947 74.21433140783091) bank17914 +17915 POINT(41.0571686648522 74.01354546822778) bank17915 +17916 POINT(40.93873257994867 74.91071381702564) bank17916 +17917 POINT(41.121891025467036 74.75077446837636) bank17917 +17918 POINT(40.320489284724104 74.66105979279885) bank17918 +17919 POINT(40.91014225940504 74.49710130738062) bank17919 +17920 POINT(39.979641139254085 73.50042766815561) bank17920 +17921 POINT(40.22487058537951 74.11465178694623) bank17921 +17922 POINT(40.137836894577795 73.91421076697428) bank17922 +17923 POINT(41.084622850981205 74.31666425666243) bank17923 +17924 POINT(40.572373637385695 74.49255799046901) bank17924 +17925 POINT(40.440292620641664 73.12787962124946) bank17925 +17926 POINT(39.75100836337887 73.06525100532284) bank17926 +17927 POINT(40.528077284300736 73.86628591627566) bank17927 +17928 POINT(40.826542207685755 73.18520280166281) bank17928 +17929 POINT(40.59509680241996 73.41756600826484) bank17929 +17930 POINT(40.09832253307255 74.14043334896152) bank17930 +17931 POINT(40.009424019566616 73.78360547725063) bank17931 +17932 POINT(40.35069116823059 73.25747183848028) bank17932 +17933 POINT(40.22666534317908 74.39400263231268) bank17933 +17934 POINT(40.5211377014935 73.97928559003621) bank17934 +17935 POINT(40.466965203533455 73.29538266113933) bank17935 +17936 POINT(39.9275833820011 73.65023699373522) bank17936 +17937 POINT(39.87849231629466 73.15896813672938) bank17937 +17938 POINT(39.79802256075133 74.68678558865808) bank17938 +17939 POINT(39.843341496599244 74.07352980504237) bank17939 +17940 POINT(39.794011192389576 74.67462875409223) bank17940 +17941 POINT(41.53375803033202 73.91990177916834) bank17941 +17942 POINT(41.526762823311934 74.38237286713662) bank17942 +17943 POINT(40.13366965267021 73.11537646893905) bank17943 +17944 POINT(40.69388960448848 73.18767595100724) bank17944 +17945 POINT(41.28797722121069 73.87275764781556) bank17945 +17946 POINT(41.625930243950506 74.27234490453705) bank17946 +17947 POINT(40.41184034707746 74.81160449819703) bank17947 +17948 POINT(41.17588927826311 74.0411710153031) bank17948 +17949 POINT(41.15838003129634 74.03681379660749) bank17949 +17950 POINT(40.44081605540121 74.16781797983874) bank17950 +17951 POINT(40.18612331149423 74.33900386512902) bank17951 +17952 POINT(41.214265733476516 73.9973911243917) bank17952 +17953 POINT(40.17998332086722 74.22103670874947) bank17953 +17954 POINT(40.42435353272319 74.69289566215488) bank17954 +17955 POINT(39.92893806341771 73.46281572134542) bank17955 +17956 POINT(41.70471302062745 73.62798654569224) bank17956 +17957 POINT(40.45740520313251 74.94154407611741) bank17957 +17958 POINT(40.06700556573454 74.99178173683431) bank17958 +17959 POINT(41.50833329164456 73.5238440446667) bank17959 +17960 POINT(40.27010273367178 74.69411593364873) bank17960 +17961 POINT(41.585028370180986 73.27438847242122) bank17961 +17962 POINT(41.631387442926815 73.78564165913043) bank17962 +17963 POINT(40.387351638111504 74.64300606631863) bank17963 +17964 POINT(41.491804939606325 73.49550068064342) bank17964 +17965 POINT(41.3815279241716 74.1691611249085) bank17965 +17966 POINT(40.2652335538318 74.64024862032298) bank17966 +17967 POINT(40.509470530400385 73.99698930452755) bank17967 +17968 POINT(40.75790744934261 73.81881768966794) bank17968 +17969 POINT(40.71112836399546 74.46122912325325) bank17969 +17970 POINT(39.81520234329304 73.86634236056344) bank17970 +17971 POINT(39.916788625176835 73.03165250793062) bank17971 +17972 POINT(39.81059057308314 74.69220359671607) bank17972 +17973 POINT(40.13190065875364 73.77066102279318) bank17973 +17974 POINT(40.64558933346221 73.5846304164972) bank17974 +17975 POINT(40.09687491113095 73.42023910498322) bank17975 +17976 POINT(40.935183756261765 73.31169749173706) bank17976 +17977 POINT(41.4025582130002 73.07984589825483) bank17977 +17978 POINT(40.92119469797268 74.70958047227776) bank17978 +17979 POINT(40.06958476301179 74.03127632513262) bank17979 +17980 POINT(40.36455547487512 73.07395969413125) bank17980 +17981 POINT(40.32295578718654 74.94570691330934) bank17981 +17982 POINT(41.03474534850112 73.20904037495283) bank17982 +17983 POINT(40.227442866634284 74.22191215628875) bank17983 +17984 POINT(40.591251365307286 74.46430232699613) bank17984 +17985 POINT(41.016478371622426 74.19491177842045) bank17985 +17986 POINT(40.037255283648456 74.60370414257721) bank17986 +17987 POINT(41.27695211499343 73.92461534971471) bank17987 +17988 POINT(40.71015208247095 73.48714778329189) bank17988 +17989 POINT(40.09132921123349 74.95028554809991) bank17989 +17990 POINT(40.331073604617046 74.05565208043113) bank17990 +17991 POINT(40.09842751735173 74.89641351215737) bank17991 +17992 POINT(40.56041242070386 74.53670759670113) bank17992 +17993 POINT(41.55150813498949 74.44126675618917) bank17993 +17994 POINT(40.49504239853307 73.5561377586333) bank17994 +17995 POINT(39.851127237095255 74.29063124824751) bank17995 +17996 POINT(39.76569827298231 74.51913665189495) bank17996 +17997 POINT(40.605877539523895 73.01320619454218) bank17997 +17998 POINT(39.77076948842082 73.13551892516446) bank17998 +17999 POINT(40.558289043006376 73.45034394599406) bank17999 +18000 POINT(40.94336381036367 73.53740247261786) bank18000 +18001 POINT(40.250071238513 73.82368848428928) bank18001 +18002 POINT(40.10153208633778 74.10202445264893) bank18002 +18003 POINT(41.35492247100394 73.85188373228931) bank18003 +18004 POINT(41.358522176700596 73.78449432444859) bank18004 +18005 POINT(40.79811240892358 73.35494666494215) bank18005 +18006 POINT(40.43270762581594 74.59268232778656) bank18006 +18007 POINT(40.853575010857995 73.13830580267093) bank18007 +18008 POINT(40.55041709753341 74.46395353816501) bank18008 +18009 POINT(39.81340591449724 75.00574827147106) bank18009 +18010 POINT(41.41550483282092 74.77861256940106) bank18010 +18011 POINT(40.00466665172268 74.38585134715758) bank18011 +18012 POINT(39.82747750040295 73.43204611741915) bank18012 +18013 POINT(39.7159858520145 74.21508234030236) bank18013 +18014 POINT(40.64203985927799 74.45433567525953) bank18014 +18015 POINT(41.13710490807983 74.27264854667331) bank18015 +18016 POINT(39.91926601286038 73.24887472820627) bank18016 +18017 POINT(41.58123853085151 74.06481915623246) bank18017 +18018 POINT(39.804750761624234 73.50158562034042) bank18018 +18019 POINT(41.40294292765993 73.22534995879552) bank18019 +18020 POINT(41.15955294172305 73.04878621349833) bank18020 +18021 POINT(41.584141466583276 74.01240715451567) bank18021 +18022 POINT(39.964532454623445 73.2471768972269) bank18022 +18023 POINT(41.62089780252644 73.58700751613948) bank18023 +18024 POINT(41.433530609395824 75.00374899930024) bank18024 +18025 POINT(40.97574574586897 73.90197004685254) bank18025 +18026 POINT(41.179620250348236 73.1591740736553) bank18026 +18027 POINT(40.888850499177046 74.49305858671977) bank18027 +18028 POINT(41.07753856648394 74.89917001143102) bank18028 +18029 POINT(40.31381306961699 74.71674300479218) bank18029 +18030 POINT(40.55986819090741 74.44274973265256) bank18030 +18031 POINT(41.042419437416754 74.35099118653972) bank18031 +18032 POINT(41.70773272762277 74.25286131000115) bank18032 +18033 POINT(40.92409343347452 74.72531013520674) bank18033 +18034 POINT(41.188403223647285 74.65020066810293) bank18034 +18035 POINT(41.68555987288685 74.0133301005456) bank18035 +18036 POINT(40.6341082362155 73.20765864372979) bank18036 +18037 POINT(41.294569801018085 74.97027706793139) bank18037 +18038 POINT(40.12095594199154 74.46242005617114) bank18038 +18039 POINT(41.68885809168003 73.91822909581924) bank18039 +18040 POINT(41.46200009138542 74.2471122677357) bank18040 +18041 POINT(40.52928859302015 73.1698766647743) bank18041 +18042 POINT(40.560289424072145 73.2153467663899) bank18042 +18043 POINT(40.361597928644194 74.58272586673904) bank18043 +18044 POINT(39.88245700960729 74.2213783691646) bank18044 +18045 POINT(40.17785576875335 74.06025872996777) bank18045 +18046 POINT(41.553033125882905 73.16362175577751) bank18046 +18047 POINT(41.03383966559864 73.99085655449925) bank18047 +18048 POINT(40.78785430157871 73.19133006354633) bank18048 +18049 POINT(40.90724666807016 73.1459265450856) bank18049 +18050 POINT(41.52560911454692 73.5752147782671) bank18050 +18051 POINT(40.76286242028619 74.2259042336719) bank18051 +18052 POINT(41.33587802350083 74.5000555809845) bank18052 +18053 POINT(40.38297529151556 73.26933121857707) bank18053 +18054 POINT(41.46589786478657 74.99484943662111) bank18054 +18055 POINT(41.57181166959349 73.37417146987448) bank18055 +18056 POINT(40.914485219757516 74.35899651014516) bank18056 +18057 POINT(39.73108035070327 74.80603999197025) bank18057 +18058 POINT(40.99806525175559 73.04846927972581) bank18058 +18059 POINT(39.861169708919995 74.39668827947138) bank18059 +18060 POINT(39.953664860635584 73.12630683721736) bank18060 +18061 POINT(40.52802163640787 73.2528542442264) bank18061 +18062 POINT(40.86718512092671 74.70889716154514) bank18062 +18063 POINT(41.33695884435018 73.76327956384087) bank18063 +18064 POINT(41.13994052793624 73.56213825734255) bank18064 +18065 POINT(41.45217951790333 74.06999555665891) bank18065 +18066 POINT(40.83655936297525 74.52762977293085) bank18066 +18067 POINT(40.58559101630681 74.41347943009126) bank18067 +18068 POINT(41.17830693710872 74.30181444827284) bank18068 +18069 POINT(40.972044085604345 73.9786480660517) bank18069 +18070 POINT(41.30800403491655 73.79464400952432) bank18070 +18071 POINT(40.36444000300592 74.0717567280732) bank18071 +18072 POINT(40.311048983250245 73.4513950524686) bank18072 +18073 POINT(40.76243667160903 73.81357443952538) bank18073 +18074 POINT(41.703282826316446 73.31129943562406) bank18074 +18075 POINT(39.81192305099194 74.46092544385823) bank18075 +18076 POINT(41.228180833858225 74.56494284679461) bank18076 +18077 POINT(41.57383488521284 73.49680954222883) bank18077 +18078 POINT(40.78762065477991 74.45765396647406) bank18078 +18079 POINT(39.810017909051204 74.56313516834032) bank18079 +18080 POINT(40.115490227529534 73.9439416597823) bank18080 +18081 POINT(39.867747096520446 74.444683274518) bank18081 +18082 POINT(41.25104711398562 73.55602184705558) bank18082 +18083 POINT(40.82086151716905 74.18987523727664) bank18083 +18084 POINT(41.13966123513173 74.68218199078636) bank18084 +18085 POINT(41.65423017723272 74.06506829158741) bank18085 +18086 POINT(41.448020209133375 74.51009882485381) bank18086 +18087 POINT(41.40706259147229 73.67831010344423) bank18087 +18088 POINT(41.571888672302805 73.21421919582433) bank18088 +18089 POINT(40.937721509693 73.86172613066623) bank18089 +18090 POINT(40.896215247911414 74.60850921385872) bank18090 +18091 POINT(40.68625384917912 73.82487353876093) bank18091 +18092 POINT(40.53416128386325 74.41416878235209) bank18092 +18093 POINT(39.77843447209814 73.88888887786369) bank18093 +18094 POINT(40.15753049310554 73.56910216964793) bank18094 +18095 POINT(40.97610104426943 73.74716994687982) bank18095 +18096 POINT(39.73332228669493 74.49013263654062) bank18096 +18097 POINT(40.34510234950437 73.02608489013976) bank18097 +18098 POINT(40.31554564572927 73.43598592720879) bank18098 +18099 POINT(39.98492321961761 73.76491416048816) bank18099 +18100 POINT(41.40779051060778 74.03544758038186) bank18100 +18101 POINT(40.427420669426304 73.84721664503066) bank18101 +18102 POINT(40.098645056662875 74.99442125723073) bank18102 +18103 POINT(41.59882789717003 73.43714719151042) bank18103 +18104 POINT(40.32604008988641 73.46904872991253) bank18104 +18105 POINT(39.79522793791398 73.74665820084462) bank18105 +18106 POINT(41.37267484008882 73.35016709931202) bank18106 +18107 POINT(40.96943306499338 74.79256037548808) bank18107 +18108 POINT(40.094893039452856 73.11725186669734) bank18108 +18109 POINT(40.745547244339775 73.55562918314169) bank18109 +18110 POINT(40.92098832007071 74.09138991254203) bank18110 +18111 POINT(41.698183331774956 73.17314408916744) bank18111 +18112 POINT(40.83855613409615 73.65119877684239) bank18112 +18113 POINT(40.49631816424592 74.63555939245482) bank18113 +18114 POINT(41.023244858638016 74.32236269008365) bank18114 +18115 POINT(40.221216901834374 74.37408515824836) bank18115 +18116 POINT(40.06731622638948 73.51753607171888) bank18116 +18117 POINT(39.74752920603004 73.8551595962951) bank18117 +18118 POINT(40.65368575947454 74.22067625346764) bank18118 +18119 POINT(40.93816878757564 73.5797687075311) bank18119 +18120 POINT(40.83744972240234 73.13693563308198) bank18120 +18121 POINT(41.144939888708016 74.94040647146456) bank18121 +18122 POINT(40.5101068655976 73.68571224645143) bank18122 +18123 POINT(40.828168965494356 73.14682353245635) bank18123 +18124 POINT(40.987488594092454 73.1538385208472) bank18124 +18125 POINT(40.874455753926995 73.59304500513205) bank18125 +18126 POINT(40.59191865243517 74.76428158950925) bank18126 +18127 POINT(41.629537363419274 74.32527049250623) bank18127 +18128 POINT(41.2333299832144 74.98115351506237) bank18128 +18129 POINT(40.528014123886386 74.83660998281206) bank18129 +18130 POINT(40.35798582291965 74.81450636245505) bank18130 +18131 POINT(41.080073690696466 73.69034391132178) bank18131 +18132 POINT(40.182741495395206 73.32971240155075) bank18132 +18133 POINT(41.011863727304345 73.50085435562148) bank18133 +18134 POINT(41.454952306307945 73.33348160868614) bank18134 +18135 POINT(40.88360711788056 74.57746304451781) bank18135 +18136 POINT(41.392934207082746 74.00125452916957) bank18136 +18137 POINT(40.64055021002276 74.80657078839899) bank18137 +18138 POINT(40.74189106950921 73.81546866426798) bank18138 +18139 POINT(40.290351073386866 74.8518846599223) bank18139 +18140 POINT(41.63320538857144 73.98166563571644) bank18140 +18141 POINT(40.40784559259117 74.56166847786753) bank18141 +18142 POINT(41.64366133475465 74.3070838681008) bank18142 +18143 POINT(40.45661518199911 74.49423489043842) bank18143 +18144 POINT(40.74925389582671 73.95149812058541) bank18144 +18145 POINT(40.0410881157503 73.06274609343865) bank18145 +18146 POINT(40.23349888830629 73.17347052017519) bank18146 +18147 POINT(39.84126347149985 73.93522951361217) bank18147 +18148 POINT(41.433983191907735 74.44156022930336) bank18148 +18149 POINT(40.80187601149535 73.73228894101507) bank18149 +18150 POINT(40.66107798275985 74.7024549770183) bank18150 +18151 POINT(41.01627506048128 73.46922665804638) bank18151 +18152 POINT(41.249493109088064 74.48239847573919) bank18152 +18153 POINT(39.878206601749454 73.6700427610156) bank18153 +18154 POINT(41.47613577721987 73.09985011723836) bank18154 +18155 POINT(40.83985774316501 73.52390085631417) bank18155 +18156 POINT(40.17128378342734 74.59316612078639) bank18156 +18157 POINT(39.832386768751775 74.31176245664174) bank18157 +18158 POINT(40.71088622881953 73.9626497313323) bank18158 +18159 POINT(41.150940716507925 74.62040781499938) bank18159 +18160 POINT(40.21435606339179 73.10418993973438) bank18160 +18161 POINT(40.66542367691167 74.92101161129173) bank18161 +18162 POINT(40.543196708678835 74.27845760059752) bank18162 +18163 POINT(40.76299508517978 74.57969640988539) bank18163 +18164 POINT(40.38779858167952 74.76901101969395) bank18164 +18165 POINT(41.355421700614 73.59662022545675) bank18165 +18166 POINT(41.36276512796574 73.63006443148859) bank18166 +18167 POINT(40.57124610234735 73.98804491351743) bank18167 +18168 POINT(41.14216834266821 73.25976828384609) bank18168 +18169 POINT(41.492222463871066 73.10414429340788) bank18169 +18170 POINT(40.673265880412174 73.66737331941773) bank18170 +18171 POINT(41.29762406122095 73.92451061068078) bank18171 +18172 POINT(39.726230736826565 73.25221710535595) bank18172 +18173 POINT(40.27024160030331 74.5633465726119) bank18173 +18174 POINT(40.67025270851331 74.18969792440527) bank18174 +18175 POINT(41.56046782290016 73.9125705448484) bank18175 +18176 POINT(40.26536334457836 73.60151266638809) bank18176 +18177 POINT(40.45043213733246 74.1318830231543) bank18177 +18178 POINT(40.31104104547314 73.11030880641614) bank18178 +18179 POINT(40.39520252326514 73.32468307802809) bank18179 +18180 POINT(40.066293892584596 73.48052202986462) bank18180 +18181 POINT(40.080497939286175 74.16868348907887) bank18181 +18182 POINT(40.3781225941267 73.90552123211397) bank18182 +18183 POINT(39.904924392475436 74.53517058142661) bank18183 +18184 POINT(39.76150033034442 74.40587556909193) bank18184 +18185 POINT(40.52517742456053 73.84048359367897) bank18185 +18186 POINT(41.04238174472161 73.10018299777367) bank18186 +18187 POINT(40.842601438416224 74.1411027003178) bank18187 +18188 POINT(39.97764429123925 74.74600684064761) bank18188 +18189 POINT(41.34280843854569 73.0309401353271) bank18189 +18190 POINT(40.98176813188314 73.9472382501442) bank18190 +18191 POINT(40.727110869332904 74.3016105511285) bank18191 +18192 POINT(40.62300282862356 74.83152983276442) bank18192 +18193 POINT(40.44235526676149 74.87725404811061) bank18193 +18194 POINT(41.09286907239603 73.17684712582493) bank18194 +18195 POINT(40.97193961214907 73.18396510573753) bank18195 +18196 POINT(40.23829809317492 73.24085690502743) bank18196 +18197 POINT(39.753971524683614 74.42747208059541) bank18197 +18198 POINT(39.82549223459963 73.4448337171651) bank18198 +18199 POINT(40.23966363484538 74.25209815461496) bank18199 +18200 POINT(39.726622018625356 74.86504749929678) bank18200 +18201 POINT(40.38685708611231 74.27254652619916) bank18201 +18202 POINT(40.9094105376735 74.10466022660023) bank18202 +18203 POINT(40.778557106614755 73.84410104934204) bank18203 +18204 POINT(39.872559127358095 73.19763795404474) bank18204 +18205 POINT(40.06563765609283 74.60950133702636) bank18205 +18206 POINT(40.7676091552579 73.19661476045692) bank18206 +18207 POINT(41.2885710802011 73.49025519389413) bank18207 +18208 POINT(41.57558623645468 74.28487705336099) bank18208 +18209 POINT(40.60862118729749 74.66828346383059) bank18209 +18210 POINT(41.25226694152532 74.49552847136125) bank18210 +18211 POINT(40.47252009189579 73.40310227950177) bank18211 +18212 POINT(39.80394416712982 74.27955541030283) bank18212 +18213 POINT(39.78730539075618 74.56488389398392) bank18213 +18214 POINT(39.857336522646726 74.82007542950066) bank18214 +18215 POINT(40.645577167771684 73.734758782458) bank18215 +18216 POINT(41.706832710005415 74.01522211958877) bank18216 +18217 POINT(41.30204727833477 73.21543433278053) bank18217 +18218 POINT(41.35519098367024 74.63528041516795) bank18218 +18219 POINT(41.60802622417048 74.23547924842964) bank18219 +18220 POINT(40.013356098256686 74.50629146319524) bank18220 +18221 POINT(41.71193814104062 74.67735666699308) bank18221 +18222 POINT(40.96857705687321 73.5074713082815) bank18222 +18223 POINT(41.550197451756176 74.46747579760377) bank18223 +18224 POINT(40.89186390864211 73.30917529105025) bank18224 +18225 POINT(41.33559743626879 73.97578944595207) bank18225 +18226 POINT(41.11962421561879 73.76366980371142) bank18226 +18227 POINT(41.280242355133836 73.11562127013013) bank18227 +18228 POINT(41.3660363037289 73.61494188800206) bank18228 +18229 POINT(41.30121093530452 74.0792606741986) bank18229 +18230 POINT(39.79584997574151 73.40824161881396) bank18230 +18231 POINT(40.95999173570537 74.87092781676793) bank18231 +18232 POINT(40.492705393093544 73.05395697941174) bank18232 +18233 POINT(39.89483148610742 73.2163459246907) bank18233 +18234 POINT(40.554840497217114 73.56385570920823) bank18234 +18235 POINT(41.260783350721425 73.62439671893821) bank18235 +18236 POINT(39.945449656941086 73.36656819634815) bank18236 +18237 POINT(40.8807951847508 73.23905883859337) bank18237 +18238 POINT(40.6530690136707 74.28542350587045) bank18238 +18239 POINT(41.34987090281763 74.93582525477694) bank18239 +18240 POINT(40.41452520846238 74.17604381566228) bank18240 +18241 POINT(40.93677937898161 74.29952787154167) bank18241 +18242 POINT(40.610427979174155 74.32078547256977) bank18242 +18243 POINT(40.69233464823152 73.49743866204197) bank18243 +18244 POINT(39.95367126121278 74.66708927976441) bank18244 +18245 POINT(40.77872975150436 73.48216553098752) bank18245 +18246 POINT(41.12138259142837 74.63179171518819) bank18246 +18247 POINT(40.208394727468956 73.81009037420704) bank18247 +18248 POINT(41.41117291559797 73.96335113458856) bank18248 +18249 POINT(41.54693649412742 74.7775689657843) bank18249 +18250 POINT(40.32155517599749 74.25643654207882) bank18250 +18251 POINT(41.304898674677034 74.43256172825471) bank18251 +18252 POINT(40.318371663417295 73.04217568320004) bank18252 +18253 POINT(41.39959291066287 73.5961492649025) bank18253 +18254 POINT(40.72202643657964 74.39739465440792) bank18254 +18255 POINT(40.736256649241554 74.91671836930674) bank18255 +18256 POINT(40.527993671800964 74.0386275643618) bank18256 +18257 POINT(40.55868297379763 74.65726985902725) bank18257 +18258 POINT(40.410713912578316 74.80638918637588) bank18258 +18259 POINT(41.215472305596286 73.8122809640934) bank18259 +18260 POINT(40.6474823803854 73.11807882824809) bank18260 +18261 POINT(40.98414648556996 73.46443269665542) bank18261 +18262 POINT(39.7781098436433 74.8487391366236) bank18262 +18263 POINT(40.59425160400367 73.99267554495664) bank18263 +18264 POINT(39.801628984934396 74.0869693041828) bank18264 +18265 POINT(40.149142409396276 73.04556986554651) bank18265 +18266 POINT(41.07193523708261 73.21873259150293) bank18266 +18267 POINT(41.03296324506163 74.24129451834501) bank18267 +18268 POINT(40.919964845135055 74.73799095146039) bank18268 +18269 POINT(40.64427254014555 73.70572727980932) bank18269 +18270 POINT(40.67947195332593 74.34246936981768) bank18270 +18271 POINT(39.87992677982498 73.82250998027759) bank18271 +18272 POINT(40.6508861173732 74.90535790745892) bank18272 +18273 POINT(41.1458094850052 74.81105499277214) bank18273 +18274 POINT(40.20329950303837 73.96915671052362) bank18274 +18275 POINT(41.083301169854415 73.45488064086913) bank18275 +18276 POINT(41.21393223398647 73.28939539767858) bank18276 +18277 POINT(40.863814283878845 73.30199132115173) bank18277 +18278 POINT(41.16772828426621 74.85175423161104) bank18278 +18279 POINT(39.835063281722334 74.66012913398279) bank18279 +18280 POINT(40.48167790178189 73.30885793018226) bank18280 +18281 POINT(40.102228765003574 73.95131411440107) bank18281 +18282 POINT(39.978794880069685 74.72763659721664) bank18282 +18283 POINT(39.83301879286954 74.89305939844995) bank18283 +18284 POINT(40.428538204089485 73.57453338102361) bank18284 +18285 POINT(39.8662282029088 73.5597629120553) bank18285 +18286 POINT(39.79434044728629 74.90057001369209) bank18286 +18287 POINT(40.708733167933445 73.55965169460762) bank18287 +18288 POINT(40.74162986226194 73.82283040240902) bank18288 +18289 POINT(40.89770192968994 73.4533423421746) bank18289 +18290 POINT(41.055553269816485 74.58305723751076) bank18290 +18291 POINT(41.369863793045965 73.52590674060062) bank18291 +18292 POINT(40.5126373158817 73.90611940690086) bank18292 +18293 POINT(41.64555997226194 74.54095348460575) bank18293 +18294 POINT(39.72578916798747 74.72058062541814) bank18294 +18295 POINT(40.870524938121115 73.73940677768441) bank18295 +18296 POINT(40.69457749924815 74.59898966345754) bank18296 +18297 POINT(41.14694557982485 74.85845088254923) bank18297 +18298 POINT(40.13964219371155 73.4982849233219) bank18298 +18299 POINT(40.84429136446445 73.75069814863278) bank18299 +18300 POINT(40.504986775330465 74.93597429915029) bank18300 +18301 POINT(41.71114782931209 73.39785515850605) bank18301 +18302 POINT(39.997323057635064 74.60854685554548) bank18302 +18303 POINT(40.02277323487237 74.8645264200618) bank18303 +18304 POINT(40.995424587791824 73.67693571631774) bank18304 +18305 POINT(40.354547806031405 74.01550976508565) bank18305 +18306 POINT(41.24279367799068 73.0698933135311) bank18306 +18307 POINT(39.77784305703147 73.55290246153683) bank18307 +18308 POINT(40.34253148154825 74.37344481010993) bank18308 +18309 POINT(40.288510252163064 73.41170749842311) bank18309 +18310 POINT(40.98401507016812 74.98571701664105) bank18310 +18311 POINT(41.39956088342455 74.77730320135451) bank18311 +18312 POINT(40.48810063914051 73.95352724000674) bank18312 +18313 POINT(41.55447218616257 73.17665557943894) bank18313 +18314 POINT(40.36029173121096 74.58900716037358) bank18314 +18315 POINT(39.99537535898091 73.13944561842236) bank18315 +18316 POINT(39.926186140636666 73.88457727721676) bank18316 +18317 POINT(40.568916207065925 74.9216092180573) bank18317 +18318 POINT(39.97417027111513 73.84107499946221) bank18318 +18319 POINT(39.86221909995322 74.70452105159907) bank18319 +18320 POINT(39.714098400314626 73.86725254265343) bank18320 +18321 POINT(39.87742386224954 73.58055869557934) bank18321 +18322 POINT(41.686427658164064 73.69352079255432) bank18322 +18323 POINT(41.506333637857104 73.43884796609) bank18323 +18324 POINT(41.61617347887097 73.22105845870391) bank18324 +18325 POINT(41.694353493246 74.20514073961117) bank18325 +18326 POINT(41.17581568326194 73.19533252106542) bank18326 +18327 POINT(41.53825372468129 74.84062543827156) bank18327 +18328 POINT(40.6417578605587 74.79979772615151) bank18328 +18329 POINT(39.73688576370687 74.71099166312024) bank18329 +18330 POINT(40.10113697913024 74.81236077924636) bank18330 +18331 POINT(40.28313786542152 74.41127611169351) bank18331 +18332 POINT(41.15513207181233 73.80901378144712) bank18332 +18333 POINT(41.51794917628249 74.33957548843948) bank18333 +18334 POINT(41.589070684420186 74.24982696213831) bank18334 +18335 POINT(40.47785164785144 74.61358854326903) bank18335 +18336 POINT(39.71974242802205 73.29019274842601) bank18336 +18337 POINT(40.02109763409631 73.06216015676284) bank18337 +18338 POINT(41.028736725122705 74.26089435817558) bank18338 +18339 POINT(41.10580864809327 74.84170857094782) bank18339 +18340 POINT(40.30324893721765 74.9337198260401) bank18340 +18341 POINT(41.21557933274216 74.42316242469458) bank18341 +18342 POINT(40.45431127546181 74.51350605309263) bank18342 +18343 POINT(41.462828228518504 73.80041427497099) bank18343 +18344 POINT(40.05775309393029 73.97703813817874) bank18344 +18345 POINT(41.159797458080256 73.04803864626565) bank18345 +18346 POINT(41.19575964510645 73.98074713501947) bank18346 +18347 POINT(40.58597027827217 73.28664516754509) bank18347 +18348 POINT(41.41584332059205 74.7710758003731) bank18348 +18349 POINT(39.83918170225519 74.70807479118828) bank18349 +18350 POINT(40.9824413536152 74.17380415700033) bank18350 +18351 POINT(39.98013325454998 74.30821363990172) bank18351 +18352 POINT(40.78584410475216 73.78307070146437) bank18352 +18353 POINT(41.02434277254108 73.01557684600509) bank18353 +18354 POINT(41.291543524482606 74.27147641508593) bank18354 +18355 POINT(41.411448548279516 74.87857458979951) bank18355 +18356 POINT(41.28158892553878 74.08094955130024) bank18356 +18357 POINT(40.740941167012174 73.60893463793923) bank18357 +18358 POINT(41.709296420261815 74.68554615326678) bank18358 +18359 POINT(41.39132620704039 73.17443573245679) bank18359 +18360 POINT(40.67977166657475 73.45349784596945) bank18360 +18361 POINT(41.63688991328867 74.07823795255766) bank18361 +18362 POINT(40.8451558196578 73.25748153377835) bank18362 +18363 POINT(40.66952803078152 73.89339737056942) bank18363 +18364 POINT(39.909569371554916 74.13027219438901) bank18364 +18365 POINT(41.48929387684386 73.27106552928437) bank18365 +18366 POINT(41.22641842735102 73.47493811827444) bank18366 +18367 POINT(40.15699235701206 73.57718556460969) bank18367 +18368 POINT(40.32099091001111 74.32294617665369) bank18368 +18369 POINT(41.556530327885255 73.38839287921279) bank18369 +18370 POINT(40.85673912840921 74.0192133260565) bank18370 +18371 POINT(39.758987401065006 73.01794785005352) bank18371 +18372 POINT(40.4790234385606 73.2050939984118) bank18372 +18373 POINT(41.33305416857554 74.85817079834639) bank18373 +18374 POINT(40.42852035042105 74.43634332007562) bank18374 +18375 POINT(41.71068607631977 73.55552235802345) bank18375 +18376 POINT(41.588175392683965 73.40398051420232) bank18376 +18377 POINT(39.93761244479311 74.9521464033146) bank18377 +18378 POINT(40.231572827625556 74.8476782890497) bank18378 +18379 POINT(39.88977733387144 73.48862464858018) bank18379 +18380 POINT(41.60882575518617 73.96251477913124) bank18380 +18381 POINT(41.172230177695575 74.56653998674415) bank18381 +18382 POINT(41.219442117221384 73.71832762366306) bank18382 +18383 POINT(40.61159212806108 73.599326587979) bank18383 +18384 POINT(39.732175652524425 73.64176904268403) bank18384 +18385 POINT(41.264006423713624 73.72510621236617) bank18385 +18386 POINT(39.878817935787744 73.43482531475613) bank18386 +18387 POINT(40.437474217179066 74.70259343334799) bank18387 +18388 POINT(41.65144033483464 73.16551927501062) bank18388 +18389 POINT(41.1423093920734 74.41942863737499) bank18389 +18390 POINT(41.15327150638021 73.94050903584804) bank18390 +18391 POINT(40.55448279622703 73.36862209437764) bank18391 +18392 POINT(40.9604562333598 73.31511333884733) bank18392 +18393 POINT(40.89265242457988 74.51002724387347) bank18393 +18394 POINT(39.97492336604394 73.634734775123) bank18394 +18395 POINT(41.4647211940598 73.27020137986639) bank18395 +18396 POINT(40.239181588441156 73.98807636334709) bank18396 +18397 POINT(40.699728532220014 74.45307790018605) bank18397 +18398 POINT(40.98886987052978 73.02577504386855) bank18398 +18399 POINT(41.067903532330774 73.42529161631946) bank18399 +18400 POINT(40.63697271762231 73.17704745228141) bank18400 +18401 POINT(39.79070396302578 73.02729725511435) bank18401 +18402 POINT(40.09423614436926 73.23668996898898) bank18402 +18403 POINT(40.27356736487582 74.44317436218867) bank18403 +18404 POINT(40.17876065159736 73.32578381411004) bank18404 +18405 POINT(40.49087243514531 73.48683178798105) bank18405 +18406 POINT(41.01607141400749 73.65506082066905) bank18406 +18407 POINT(41.461090973709965 74.43691661017172) bank18407 +18408 POINT(40.60904831730962 73.41884028424963) bank18408 +18409 POINT(39.94527643974881 73.5593401470474) bank18409 +18410 POINT(41.008062776334405 73.18374438502809) bank18410 +18411 POINT(40.50056223975893 73.57993197234306) bank18411 +18412 POINT(41.47414964871786 73.06843032202141) bank18412 +18413 POINT(41.175882403252885 73.35074151572603) bank18413 +18414 POINT(41.30597195829965 74.05894814101939) bank18414 +18415 POINT(41.20020686789027 74.73927868242934) bank18415 +18416 POINT(41.44213485557856 73.41818788542928) bank18416 +18417 POINT(41.45407977987731 74.58880839471932) bank18417 +18418 POINT(40.640430444320515 73.50966437210504) bank18418 +18419 POINT(40.8485828703427 74.71191946378273) bank18419 +18420 POINT(41.43083754878855 74.95402651926203) bank18420 +18421 POINT(39.90778844943971 74.25030946410381) bank18421 +18422 POINT(40.58495436515564 74.17190732623898) bank18422 +18423 POINT(40.68108641264715 73.05532233838453) bank18423 +18424 POINT(39.91600393486906 73.21641102219706) bank18424 +18425 POINT(39.888281420286916 73.64093289558177) bank18425 +18426 POINT(41.450970003078005 73.61197087648293) bank18426 +18427 POINT(40.20945919565868 74.56973111719381) bank18427 +18428 POINT(41.64663010517626 74.20870060507791) bank18428 +18429 POINT(40.25191284402366 74.75532517076431) bank18429 +18430 POINT(39.89718042804581 73.38709796435539) bank18430 +18431 POINT(41.50948885681477 73.90148464758798) bank18431 +18432 POINT(40.29402768749723 73.82740188773273) bank18432 +18433 POINT(41.01683927354914 73.16203033804847) bank18433 +18434 POINT(40.82478635295363 73.02534663195523) bank18434 +18435 POINT(40.20023549299605 73.51573772175071) bank18435 +18436 POINT(39.9337826298551 73.34533877276853) bank18436 +18437 POINT(40.64555792504175 74.4041044123817) bank18437 +18438 POINT(41.03370487332923 73.86630024732526) bank18438 +18439 POINT(41.64297604783007 73.95945403326262) bank18439 +18440 POINT(41.12542420238 73.09659596502601) bank18440 +18441 POINT(40.29395835698234 74.34565873325425) bank18441 +18442 POINT(40.763732761830454 73.9172172248588) bank18442 +18443 POINT(40.4459985021689 73.30334359236878) bank18443 +18444 POINT(41.179256339318925 73.5523686311312) bank18444 +18445 POINT(39.71452950140039 74.66359819277687) bank18445 +18446 POINT(40.3999726453668 74.8797102170697) bank18446 +18447 POINT(40.65903809053776 73.19703238794354) bank18447 +18448 POINT(40.4686432979972 73.55284759625167) bank18448 +18449 POINT(40.32763084106286 74.27756978916861) bank18449 +18450 POINT(40.528545903685966 74.91375061070899) bank18450 +18451 POINT(40.99310559176972 74.11077313372529) bank18451 +18452 POINT(41.12449970820158 74.88688449139342) bank18452 +18453 POINT(41.61228998621536 73.27516077341156) bank18453 +18454 POINT(40.348841800608724 73.39346352066813) bank18454 +18455 POINT(40.65846070983398 74.96207919680211) bank18455 +18456 POINT(40.0874137264784 74.23766482617768) bank18456 +18457 POINT(40.235279543232814 73.61079667844557) bank18457 +18458 POINT(40.98002431523238 74.91258437339137) bank18458 +18459 POINT(40.58042964924018 74.60330399571576) bank18459 +18460 POINT(40.076579905799605 74.76556225114942) bank18460 +18461 POINT(40.049651263559056 73.59607203261587) bank18461 +18462 POINT(40.62831673095758 73.38469585243543) bank18462 +18463 POINT(41.71022480833297 74.38803044416142) bank18463 +18464 POINT(41.50908464227795 73.95465331088243) bank18464 +18465 POINT(41.48867035240649 74.74852307769021) bank18465 +18466 POINT(40.56853913344068 74.90063030135794) bank18466 +18467 POINT(41.60229735472251 74.9993055465036) bank18467 +18468 POINT(41.475087270705686 73.93130877679795) bank18468 +18469 POINT(41.04683235068298 73.40450248213045) bank18469 +18470 POINT(41.3637371935837 74.43697697050288) bank18470 +18471 POINT(40.72933041963199 74.15813833211064) bank18471 +18472 POINT(41.45214465580981 73.15297123109379) bank18472 +18473 POINT(40.162324022761865 74.46648331964012) bank18473 +18474 POINT(40.3613124257371 73.71167692300202) bank18474 +18475 POINT(41.41377000006786 73.3102213754751) bank18475 +18476 POINT(41.36060856881279 73.85349113765415) bank18476 +18477 POINT(40.50450138291034 73.41770293874002) bank18477 +18478 POINT(41.67770145017441 73.70260747903963) bank18478 +18479 POINT(41.30596738163677 74.61009583324682) bank18479 +18480 POINT(41.39255135456817 73.90518593817596) bank18480 +18481 POINT(40.4779283566344 73.85895196154773) bank18481 +18482 POINT(40.79629742852454 74.23715519795417) bank18482 +18483 POINT(41.09763097731623 74.67054560731759) bank18483 +18484 POINT(40.73316691948938 73.64891378317171) bank18484 +18485 POINT(40.49612722470705 74.50584090387534) bank18485 +18486 POINT(40.811387755010664 73.71758134118213) bank18486 +18487 POINT(40.90336188164287 73.11820509296942) bank18487 +18488 POINT(41.52513308200756 73.26902302794498) bank18488 +18489 POINT(39.71439035069824 73.77496389271194) bank18489 +18490 POINT(41.44171518433548 73.75759638161945) bank18490 +18491 POINT(40.831069377146825 73.55644360624825) bank18491 +18492 POINT(40.60394864827775 73.09798027418142) bank18492 +18493 POINT(40.41126513062256 74.35376964282683) bank18493 +18494 POINT(40.083638432558935 74.29870760984375) bank18494 +18495 POINT(40.298219777686654 73.32762422384997) bank18495 +18496 POINT(41.2934197414593 74.55146472638587) bank18496 +18497 POINT(40.47430306474574 74.39583016084755) bank18497 +18498 POINT(41.262835661341846 73.24777747996109) bank18498 +18499 POINT(40.649976172083456 73.43447606528716) bank18499 +18500 POINT(40.34109666626965 74.51454325135524) bank18500 +18501 POINT(40.663011786202915 74.99098156226643) bank18501 +18502 POINT(40.89129214613678 73.30732837323836) bank18502 +18503 POINT(41.142054473163455 74.03054321549288) bank18503 +18504 POINT(41.031918832741255 74.6192124150222) bank18504 +18505 POINT(41.305156368674886 74.98848890725321) bank18505 +18506 POINT(40.58840638465936 74.40607727536592) bank18506 +18507 POINT(41.56114629271594 74.48327971942983) bank18507 +18508 POINT(40.06544121368868 73.72006006504768) bank18508 +18509 POINT(39.78871679170859 73.68771774146128) bank18509 +18510 POINT(40.664455192515035 74.05235961184964) bank18510 +18511 POINT(39.92047696647469 73.74235177954498) bank18511 +18512 POINT(41.42976631335928 73.38405312757467) bank18512 +18513 POINT(40.65025591887275 74.35731257031752) bank18513 +18514 POINT(39.75385387732353 73.22431169809393) bank18514 +18515 POINT(41.02057829626561 74.44017740061486) bank18515 +18516 POINT(40.01745212139155 74.03996518547328) bank18516 +18517 POINT(40.98032457949865 74.49557229953773) bank18517 +18518 POINT(41.08837245214408 73.09012970207985) bank18518 +18519 POINT(40.13203474360965 73.26327899267935) bank18519 +18520 POINT(41.45581229104781 74.44160544087669) bank18520 +18521 POINT(40.96673561052749 74.49556091314423) bank18521 +18522 POINT(41.15999337546952 73.33005646889217) bank18522 +18523 POINT(40.56398951871733 73.99016702051539) bank18523 +18524 POINT(40.808514125134195 73.76948732970021) bank18524 +18525 POINT(40.43313524806967 73.02247214916578) bank18525 +18526 POINT(41.138982947656984 73.89902675510153) bank18526 +18527 POINT(41.632668424602535 74.98334910803199) bank18527 +18528 POINT(41.336780214343 74.90540134257716) bank18528 +18529 POINT(39.86030071231462 74.90094165515453) bank18529 +18530 POINT(39.90122660814795 74.59655958367162) bank18530 +18531 POINT(40.8180740388304 73.61213006559488) bank18531 +18532 POINT(41.39515949685961 75.00319743715532) bank18532 +18533 POINT(40.42825379506762 73.36716185340363) bank18533 +18534 POINT(41.30348047329566 73.4326452808657) bank18534 +18535 POINT(40.82176707422347 73.3237930278642) bank18535 +18536 POINT(41.05413012602318 74.33472325997478) bank18536 +18537 POINT(40.05218443948097 73.53174760209116) bank18537 +18538 POINT(41.63405949264639 74.37239382032857) bank18538 +18539 POINT(40.13786393025415 74.75725306768096) bank18539 +18540 POINT(40.86751595393688 74.2776356936562) bank18540 +18541 POINT(40.743933790789754 73.62217359476442) bank18541 +18542 POINT(40.39026211754934 73.85072178025027) bank18542 +18543 POINT(40.46691730914069 74.26609567067476) bank18543 +18544 POINT(41.28948916982122 74.18252485741351) bank18544 +18545 POINT(40.357060266166016 73.92834148030715) bank18545 +18546 POINT(40.13420473764823 73.191004118507) bank18546 +18547 POINT(41.59903965731631 74.0723545633649) bank18547 +18548 POINT(41.173557065209245 73.43246674465081) bank18548 +18549 POINT(40.814845403494466 73.27208608186508) bank18549 +18550 POINT(41.307431071733355 74.47852937840217) bank18550 +18551 POINT(40.66842214078076 73.20391517227718) bank18551 +18552 POINT(40.97459029814139 73.96929215965217) bank18552 +18553 POINT(41.05244336881682 74.68283531162362) bank18553 +18554 POINT(41.425894351428816 74.85221538486974) bank18554 +18555 POINT(40.221162787146746 74.0028872021483) bank18555 +18556 POINT(40.20465911011903 74.80521284885859) bank18556 +18557 POINT(41.55387447124735 74.29985565061051) bank18557 +18558 POINT(40.85467430067306 73.18307358447628) bank18558 +18559 POINT(40.72702072982589 73.74149834012624) bank18559 +18560 POINT(40.01064415941007 73.27943572956829) bank18560 +18561 POINT(41.2693147432391 73.80544079261088) bank18561 +18562 POINT(40.0759543149857 73.744044167741) bank18562 +18563 POINT(41.47617948273433 73.14134884763384) bank18563 +18564 POINT(39.88644065202456 74.92423373320712) bank18564 +18565 POINT(39.873673480502795 73.52317836672488) bank18565 +18566 POINT(40.88627122251098 74.55420474562146) bank18566 +18567 POINT(40.05922888689645 73.26177851918835) bank18567 +18568 POINT(40.410093267639496 73.78242183465258) bank18568 +18569 POINT(40.20115256398408 74.92991690692799) bank18569 +18570 POINT(40.183039998897655 74.99306987828318) bank18570 +18571 POINT(39.989295085551774 73.02780270988849) bank18571 +18572 POINT(40.25710864055397 73.06933670588543) bank18572 +18573 POINT(40.0916041917512 73.62293798509894) bank18573 +18574 POINT(41.01699886612328 74.13284669081753) bank18574 +18575 POINT(41.4246711195559 74.31694853394495) bank18575 +18576 POINT(40.525599659340266 73.210610622372) bank18576 +18577 POINT(41.1145314751799 73.5796098645194) bank18577 +18578 POINT(41.474302597584035 73.42581753375474) bank18578 +18579 POINT(39.89300052135703 73.2998865716259) bank18579 +18580 POINT(41.18609886345782 74.35034683303516) bank18580 +18581 POINT(40.068717585301485 74.3386980013854) bank18581 +18582 POINT(40.05049238613884 73.7260997177346) bank18582 +18583 POINT(41.18939655436318 74.9803574764194) bank18583 +18584 POINT(40.897225400838906 73.19719728026048) bank18584 +18585 POINT(40.21328559908066 73.80008554988692) bank18585 +18586 POINT(41.23885433388684 74.48249413473526) bank18586 +18587 POINT(40.636240529068644 74.49320642537477) bank18587 +18588 POINT(40.933516525710864 73.78355353900938) bank18588 +18589 POINT(41.52994694909279 74.77523828549093) bank18589 +18590 POINT(41.28447220641147 73.22676890325306) bank18590 +18591 POINT(41.694481735976375 74.48736458273683) bank18591 +18592 POINT(40.30422262237143 73.44520602769902) bank18592 +18593 POINT(40.853372253595005 74.21822233730892) bank18593 +18594 POINT(41.48800086757464 73.68011709970186) bank18594 +18595 POINT(41.481573877089545 74.15967910750513) bank18595 +18596 POINT(40.97099653727617 73.35461706412327) bank18596 +18597 POINT(40.86352572113846 73.0280399538543) bank18597 +18598 POINT(41.66511216797535 73.13268322047077) bank18598 +18599 POINT(41.521571092271586 73.47762666799434) bank18599 +18600 POINT(40.448637108116145 73.20161905009239) bank18600 +18601 POINT(41.25646206052229 73.35541069004995) bank18601 +18602 POINT(39.99725283348861 74.70946256490795) bank18602 +18603 POINT(39.81332227584437 73.8533960050163) bank18603 +18604 POINT(40.74818156536625 74.41369348839893) bank18604 +18605 POINT(40.94530613996586 74.95559026846209) bank18605 +18606 POINT(41.06145299267796 73.23099349813552) bank18606 +18607 POINT(41.34224639003807 73.86092131644975) bank18607 +18608 POINT(41.4222389725057 73.23718065990472) bank18608 +18609 POINT(39.725685496383896 74.05850143780131) bank18609 +18610 POINT(41.26460450034231 74.0786162615433) bank18610 +18611 POINT(40.40611384494108 73.05445095213129) bank18611 +18612 POINT(40.73684425713887 73.12300554340666) bank18612 +18613 POINT(41.160880183125485 73.77016785641308) bank18613 +18614 POINT(41.10604017681472 73.038603731019) bank18614 +18615 POINT(40.04425926939953 73.23423166980808) bank18615 +18616 POINT(41.290384415708544 73.66018271834422) bank18616 +18617 POINT(41.06324159195709 73.14204669265389) bank18617 +18618 POINT(41.00118496359776 73.36907460938662) bank18618 +18619 POINT(41.50377124191359 74.79689973720005) bank18619 +18620 POINT(41.25426440609486 74.72267257456573) bank18620 +18621 POINT(40.839428468529164 73.35822881868992) bank18621 +18622 POINT(41.013855244576554 73.87606552752742) bank18622 +18623 POINT(40.85029369427508 73.25919802469635) bank18623 +18624 POINT(41.096337073860845 73.4248843066588) bank18624 +18625 POINT(40.23416373968634 73.21604895959703) bank18625 +18626 POINT(40.247258790967976 73.60926661080188) bank18626 +18627 POINT(40.06134272746656 73.94197490824914) bank18627 +18628 POINT(40.06046074032125 74.13565550329454) bank18628 +18629 POINT(40.221059260871456 74.53353531105904) bank18629 +18630 POINT(41.36139865247069 74.88678184034583) bank18630 +18631 POINT(41.35014572872499 74.73753597647634) bank18631 +18632 POINT(40.0145936533695 74.61211944376556) bank18632 +18633 POINT(41.25404914852598 73.51369979105915) bank18633 +18634 POINT(40.36927108001411 73.19855747325695) bank18634 +18635 POINT(40.58385984939534 74.38683935117852) bank18635 +18636 POINT(41.40893695978576 74.06887736219758) bank18636 +18637 POINT(41.36546760833638 74.11341515349537) bank18637 +18638 POINT(39.7592907053607 73.42463126546804) bank18638 +18639 POINT(40.06879623349747 73.0157220614676) bank18639 +18640 POINT(40.34347306532514 73.96532207914015) bank18640 +18641 POINT(40.637809274639785 74.23402301372053) bank18641 +18642 POINT(40.74406277484904 74.52498542204039) bank18642 +18643 POINT(39.79210779815025 74.49670452691582) bank18643 +18644 POINT(41.041180969129705 73.03910340984412) bank18644 +18645 POINT(41.616799005964864 74.11882017111328) bank18645 +18646 POINT(40.50363984212416 74.22400603771618) bank18646 +18647 POINT(41.12924222977889 73.0559732262208) bank18647 +18648 POINT(40.95495614787009 73.58418156947133) bank18648 +18649 POINT(41.34742198287264 73.17544829556661) bank18649 +18650 POINT(41.50751731858335 74.35855670378139) bank18650 +18651 POINT(40.2893825599989 74.04046902598182) bank18651 +18652 POINT(41.08899277601148 73.0461192937411) bank18652 +18653 POINT(40.70514145207817 74.0064596273827) bank18653 +18654 POINT(40.78309899553874 74.98541864690937) bank18654 +18655 POINT(41.12570276185423 73.11312756348556) bank18655 +18656 POINT(41.50644019238206 73.32930288271321) bank18656 +18657 POINT(39.842854101305015 74.15421672597108) bank18657 +18658 POINT(40.02711084344528 74.77689612128933) bank18658 +18659 POINT(41.623457587268206 74.87239196807906) bank18659 +18660 POINT(40.97820672823764 73.30831176912206) bank18660 +18661 POINT(40.30618402329104 74.03006965007522) bank18661 +18662 POINT(40.86701260391765 74.05621556116307) bank18662 +18663 POINT(40.38238284118913 73.12605728685118) bank18663 +18664 POINT(40.978082141223986 74.45725078232263) bank18664 +18665 POINT(41.58166611973896 74.13428279263131) bank18665 +18666 POINT(41.383136889315125 74.20187638934895) bank18666 +18667 POINT(39.820530770389574 73.75053947473809) bank18667 +18668 POINT(40.777245251285876 73.12639222805245) bank18668 +18669 POINT(40.38583009185299 74.84811054357561) bank18669 +18670 POINT(40.124030951414646 74.19304849065786) bank18670 +18671 POINT(41.3765373469776 73.32078697942087) bank18671 +18672 POINT(40.72336226824751 74.47567325833619) bank18672 +18673 POINT(41.440642880783116 74.10861945689699) bank18673 +18674 POINT(40.52441775955234 73.74325498901482) bank18674 +18675 POINT(41.49420666725989 73.6505363107909) bank18675 +18676 POINT(40.327878945956876 74.08971366631145) bank18676 +18677 POINT(40.760853569485874 74.93364161155766) bank18677 +18678 POINT(40.097321719240966 74.7037606240696) bank18678 +18679 POINT(40.20739382079387 73.46515124636144) bank18679 +18680 POINT(41.204413701304375 74.928705152798) bank18680 +18681 POINT(41.09988478555964 73.79164518908306) bank18681 +18682 POINT(40.553298290638615 74.4363191658326) bank18682 +18683 POINT(40.548620997902965 73.26340727455701) bank18683 +18684 POINT(39.925609448018854 74.58764363682592) bank18684 +18685 POINT(41.36495963496783 73.18592652167875) bank18685 +18686 POINT(40.37351777098641 74.22446409591423) bank18686 +18687 POINT(41.005090154415775 73.87844520462923) bank18687 +18688 POINT(40.014669878028045 73.62210675766983) bank18688 +18689 POINT(39.852327785913474 74.05147567969736) bank18689 +18690 POINT(40.10048421870732 74.42109866964842) bank18690 +18691 POINT(40.39452393450883 74.49200278096986) bank18691 +18692 POINT(41.36042653885403 73.49208853293833) bank18692 +18693 POINT(41.095387306090615 74.19607008638623) bank18693 +18694 POINT(39.893650558916 74.21158070449668) bank18694 +18695 POINT(41.602878095982355 74.64404698761417) bank18695 +18696 POINT(40.00727978871766 73.52717179667187) bank18696 +18697 POINT(41.3743004913053 74.0163405986071) bank18697 +18698 POINT(40.578273935478364 74.94639274605032) bank18698 +18699 POINT(41.35583167572355 73.50191324426824) bank18699 +18700 POINT(40.861437359805365 74.2490079745238) bank18700 +18701 POINT(39.790359813874566 74.15064464100605) bank18701 +18702 POINT(41.26137673023349 73.23192816481811) bank18702 +18703 POINT(41.54670623513232 74.12445562862412) bank18703 +18704 POINT(41.070156551599155 73.37611968278362) bank18704 +18705 POINT(40.70859899065632 73.40886754345233) bank18705 +18706 POINT(41.20359452951779 74.22057541379918) bank18706 +18707 POINT(40.076919489340504 73.71056688914156) bank18707 +18708 POINT(41.18479157927839 73.92966460121114) bank18708 +18709 POINT(41.21033464150892 74.82479641517304) bank18709 +18710 POINT(41.32405664434907 74.806519863037) bank18710 +18711 POINT(40.45494712730612 74.95319823182676) bank18711 +18712 POINT(40.13946925880255 74.86401429058054) bank18712 +18713 POINT(40.1038129203294 74.95382164884113) bank18713 +18714 POINT(39.93591610562147 73.6582245203172) bank18714 +18715 POINT(40.15045835869835 73.90417085073277) bank18715 +18716 POINT(40.10344885265695 74.61288508799366) bank18716 +18717 POINT(40.41025666367495 74.39444489978902) bank18717 +18718 POINT(39.98729658363519 73.5664739919185) bank18718 +18719 POINT(40.24840662050038 74.11534086624962) bank18719 +18720 POINT(40.505856938394544 74.07061357868086) bank18720 +18721 POINT(41.308704473150144 74.20872738413803) bank18721 +18722 POINT(40.47267275478092 74.9711594959725) bank18722 +18723 POINT(41.62763849911166 73.9897909545254) bank18723 +18724 POINT(40.11983332387648 73.46432087165653) bank18724 +18725 POINT(41.6351415979432 74.89534086831078) bank18725 +18726 POINT(40.727599965155804 73.61361619765843) bank18726 +18727 POINT(40.63711650665228 73.86582973893793) bank18727 +18728 POINT(41.19234016817158 73.40431041901972) bank18728 +18729 POINT(39.94904957401152 74.5887922916719) bank18729 +18730 POINT(41.4978318189365 73.91829048963851) bank18730 +18731 POINT(40.10931737282271 74.92513355204571) bank18731 +18732 POINT(41.40720153321361 74.47097544558366) bank18732 +18733 POINT(40.91435437300991 74.06850355209885) bank18733 +18734 POINT(41.57877347359792 74.80288190069957) bank18734 +18735 POINT(40.49323570411775 74.20224561262913) bank18735 +18736 POINT(41.34603383157079 73.8478744417489) bank18736 +18737 POINT(40.78004974967294 74.22683173882939) bank18737 +18738 POINT(41.67868586406768 74.28633622583843) bank18738 +18739 POINT(40.26125054021329 73.44138915417182) bank18739 +18740 POINT(39.88860450596088 74.81303046648455) bank18740 +18741 POINT(39.97264622365279 73.23367124014723) bank18741 +18742 POINT(41.50797328424156 74.91770267577952) bank18742 +18743 POINT(41.441615172384864 74.25180265207678) bank18743 +18744 POINT(40.501281613059184 74.44156969015711) bank18744 +18745 POINT(41.59034227773037 74.4941818213147) bank18745 +18746 POINT(40.84779235036197 73.38154541896033) bank18746 +18747 POINT(40.80483801298867 74.0229786283005) bank18747 +18748 POINT(41.58561342077997 73.74767899680997) bank18748 +18749 POINT(41.639996392642026 74.47188146344433) bank18749 +18750 POINT(40.32800207980544 73.117737459128) bank18750 +18751 POINT(40.80401544412896 74.71291327507647) bank18751 +18752 POINT(39.93188726534979 74.52878759810274) bank18752 +18753 POINT(40.46147175361082 74.78969595134424) bank18753 +18754 POINT(40.6115929671348 74.69602730340884) bank18754 +18755 POINT(40.17666438064745 73.16600861367998) bank18755 +18756 POINT(41.20364081074661 74.05358544488888) bank18756 +18757 POINT(40.343448346229096 73.01869125348385) bank18757 +18758 POINT(40.495273644991016 73.42531273514774) bank18758 +18759 POINT(41.327976597507536 74.01308995240836) bank18759 +18760 POINT(39.72246281355303 74.13084919050576) bank18760 +18761 POINT(39.931700582942504 74.04072697556099) bank18761 +18762 POINT(41.49932759974848 73.17124263934053) bank18762 +18763 POINT(40.941554295646476 74.8878411194924) bank18763 +18764 POINT(40.87801450368313 74.67126363884246) bank18764 +18765 POINT(39.95023251538705 74.85441671481924) bank18765 +18766 POINT(40.5191701943291 74.90459681361241) bank18766 +18767 POINT(41.60589585293091 74.4446777404736) bank18767 +18768 POINT(40.34703038025419 74.76380587882714) bank18768 +18769 POINT(41.057351742368006 74.70790551332959) bank18769 +18770 POINT(40.48547661393417 74.55336843551923) bank18770 +18771 POINT(40.596554939176016 73.70374199369809) bank18771 +18772 POINT(40.47238273011737 73.62780840503206) bank18772 +18773 POINT(41.39122748701046 73.3849914128192) bank18773 +18774 POINT(40.30612825071023 73.17166428946338) bank18774 +18775 POINT(40.134088853502874 73.41329521265847) bank18775 +18776 POINT(40.52417849801606 74.0621441049323) bank18776 +18777 POINT(40.47006242312121 73.13927223496536) bank18777 +18778 POINT(41.049249328758016 74.28877179019483) bank18778 +18779 POINT(41.54235755370171 73.97002888143903) bank18779 +18780 POINT(41.094486628562805 74.45496906632982) bank18780 +18781 POINT(40.04648752753159 74.06693729662346) bank18781 +18782 POINT(40.86348773118806 73.27182084968004) bank18782 +18783 POINT(40.999758671301805 74.00827146218256) bank18783 +18784 POINT(41.30795487091014 74.22922434697261) bank18784 +18785 POINT(41.32477858367195 74.13039114295796) bank18785 +18786 POINT(40.10356474819128 73.62324192467408) bank18786 +18787 POINT(40.13304876873324 74.4891763552406) bank18787 +18788 POINT(39.99399303935046 74.88383632994058) bank18788 +18789 POINT(41.00257182823442 74.48069268082853) bank18789 +18790 POINT(41.564064847931306 74.52398014120809) bank18790 +18791 POINT(41.08943731869848 74.04658545025582) bank18791 +18792 POINT(41.701305010434 74.82916706374338) bank18792 +18793 POINT(41.23535924181453 74.85178346668907) bank18793 +18794 POINT(41.03424927702411 73.09628700048656) bank18794 +18795 POINT(41.19338456597201 74.14566422639035) bank18795 +18796 POINT(40.80597864750273 73.77134964902226) bank18796 +18797 POINT(41.28974938900358 74.62299797676152) bank18797 +18798 POINT(39.713003134125536 73.62436534527666) bank18798 +18799 POINT(40.476835334435165 74.95096794153596) bank18799 +18800 POINT(39.9935658872764 74.17229852603903) bank18800 +18801 POINT(40.503968455368636 74.83665799948524) bank18801 +18802 POINT(41.509306291869024 74.51760402135542) bank18802 +18803 POINT(40.54518612348741 74.42618155891948) bank18803 +18804 POINT(40.72889742571862 73.65074906489438) bank18804 +18805 POINT(39.844445265485234 74.76277555731994) bank18805 +18806 POINT(39.98002629420048 74.53004867022935) bank18806 +18807 POINT(40.70345904254804 73.03263364382418) bank18807 +18808 POINT(40.05812162382916 74.26454315554388) bank18808 +18809 POINT(40.11973135887578 74.40996126254275) bank18809 +18810 POINT(41.66906927975215 74.93910870807989) bank18810 +18811 POINT(41.570001863505894 73.15926696461457) bank18811 +18812 POINT(39.99240392302419 74.58349131427248) bank18812 +18813 POINT(40.87384189199673 74.67888747276356) bank18813 +18814 POINT(40.29311412161795 74.63089137601192) bank18814 +18815 POINT(40.630131503817545 74.58687756852319) bank18815 +18816 POINT(40.949432664390066 73.4913445914553) bank18816 +18817 POINT(40.40601375646444 73.39421483659017) bank18817 +18818 POINT(40.86537496913769 73.35188336584503) bank18818 +18819 POINT(40.50487911649619 74.15715547233386) bank18819 +18820 POINT(40.29958503735044 74.67710245628402) bank18820 +18821 POINT(41.193427800930124 73.87478163089182) bank18821 +18822 POINT(41.53550343264759 73.41296928273887) bank18822 +18823 POINT(41.0072755920026 74.10848821693342) bank18823 +18824 POINT(40.5404829487236 74.6290737480434) bank18824 +18825 POINT(39.740670615735645 74.5329218643) bank18825 +18826 POINT(40.402976763547926 73.71235086794302) bank18826 +18827 POINT(40.24486168792748 74.402746557766) bank18827 +18828 POINT(40.049941042802345 73.6007262928229) bank18828 +18829 POINT(40.54845642425074 74.68245642777212) bank18829 +18830 POINT(41.69763446304906 74.14968061578286) bank18830 +18831 POINT(41.26869268081729 73.94441197709547) bank18831 +18832 POINT(41.623346951148 74.39834527118732) bank18832 +18833 POINT(41.39149641110151 75.00333486447343) bank18833 +18834 POINT(40.554493716859504 74.9518431373977) bank18834 +18835 POINT(40.5183731706711 73.5062012998491) bank18835 +18836 POINT(40.903050455413705 74.7237755809555) bank18836 +18837 POINT(40.65056643868551 74.60513802845338) bank18837 +18838 POINT(41.2607143965333 74.89629818396001) bank18838 +18839 POINT(41.4584608674218 73.34461951188668) bank18839 +18840 POINT(40.4658575497969 73.04407874448802) bank18840 +18841 POINT(41.43447358714828 74.08641459940588) bank18841 +18842 POINT(41.14118259784526 73.21157697062833) bank18842 +18843 POINT(41.53106319860134 74.90798590775867) bank18843 +18844 POINT(41.22918099160541 74.47142628800432) bank18844 +18845 POINT(40.47307541274519 74.77351452832947) bank18845 +18846 POINT(41.27466178895311 73.01435584825587) bank18846 +18847 POINT(40.992205963656716 74.94948415458686) bank18847 +18848 POINT(41.20527281397795 74.09213078081584) bank18848 +18849 POINT(41.598721506250506 73.31423134661676) bank18849 +18850 POINT(39.82884047886182 74.55340560395724) bank18850 +18851 POINT(40.582881147569715 74.64336318709744) bank18851 +18852 POINT(40.167886269272046 73.03918660717494) bank18852 +18853 POINT(41.50042361757916 73.85046790046505) bank18853 +18854 POINT(39.94442423824771 73.31445037683862) bank18854 +18855 POINT(41.2256437209834 74.08169292654635) bank18855 +18856 POINT(40.15518471526624 73.95074571220036) bank18856 +18857 POINT(40.85330092292178 74.31260934683718) bank18857 +18858 POINT(39.86920528494921 74.90963522224052) bank18858 +18859 POINT(41.03296635593416 73.8981828746031) bank18859 +18860 POINT(40.8948117781163 74.79771779493042) bank18860 +18861 POINT(41.473184740279976 74.18288433587098) bank18861 +18862 POINT(40.6961589640382 73.56717567686007) bank18862 +18863 POINT(41.216447221060854 74.60410183404791) bank18863 +18864 POINT(41.30449043893813 73.1090452275515) bank18864 +18865 POINT(40.78500604842068 73.90532101121666) bank18865 +18866 POINT(39.772198663951364 73.32422132446503) bank18866 +18867 POINT(41.23532483814456 74.83534092538872) bank18867 +18868 POINT(39.82735560084843 73.56052667359113) bank18868 +18869 POINT(39.72977249168456 73.0425474366664) bank18869 +18870 POINT(41.48128731481201 73.29883630723279) bank18870 +18871 POINT(40.13150708405454 74.74071154021914) bank18871 +18872 POINT(39.82948783103251 74.12024083908946) bank18872 +18873 POINT(39.752609779935206 74.69698568141392) bank18873 +18874 POINT(41.21234354039037 73.23620793902519) bank18874 +18875 POINT(40.326382055170484 73.29516122624162) bank18875 +18876 POINT(40.33195092741194 73.6948072831392) bank18876 +18877 POINT(41.35743885888149 74.37903736044217) bank18877 +18878 POINT(40.9490352309819 73.99491475940579) bank18878 +18879 POINT(41.55210939736582 73.13383822715556) bank18879 +18880 POINT(41.47030332590148 74.66914542308227) bank18880 +18881 POINT(41.11730909765294 74.83262842663183) bank18881 +18882 POINT(41.64763319942661 74.07152437833963) bank18882 +18883 POINT(41.117479258559015 74.25639131375043) bank18883 +18884 POINT(40.853370133896696 74.29229793720667) bank18884 +18885 POINT(40.58398319295737 74.82305656975996) bank18885 +18886 POINT(41.60960145947702 73.95054583114761) bank18886 +18887 POINT(41.280188365290584 73.6995200622551) bank18887 +18888 POINT(40.1110326096791 73.39429754625584) bank18888 +18889 POINT(40.60195725021931 74.43025819530195) bank18889 +18890 POINT(39.87486281672241 74.5712139909768) bank18890 +18891 POINT(40.84515129055701 73.33818257747629) bank18891 +18892 POINT(40.42763987081294 75.00475053560126) bank18892 +18893 POINT(40.629293233438034 74.34760436861686) bank18893 +18894 POINT(41.22563387386463 73.89757964954582) bank18894 +18895 POINT(40.99380208788435 74.53650701799671) bank18895 +18896 POINT(40.42354347446206 73.23957747718623) bank18896 +18897 POINT(41.3079241206177 73.18357768613289) bank18897 +18898 POINT(40.00282576472591 73.25151355274032) bank18898 +18899 POINT(39.751122572802345 73.86407210481799) bank18899 +18900 POINT(40.26665219927238 74.35085881090122) bank18900 +18901 POINT(40.9466612083767 73.26759056878795) bank18901 +18902 POINT(40.55693262814808 73.0167642152509) bank18902 +18903 POINT(40.40504611435003 73.94525304313694) bank18903 +18904 POINT(40.10130073402132 74.33330670215835) bank18904 +18905 POINT(41.11182163077326 74.03444632239005) bank18905 +18906 POINT(40.06008181618485 73.31851000861693) bank18906 +18907 POINT(40.15829455308026 73.92815551220934) bank18907 +18908 POINT(41.45090480846356 73.76528723584849) bank18908 +18909 POINT(41.323709160582425 73.06662038188506) bank18909 +18910 POINT(39.961874636460365 73.92177759206137) bank18910 +18911 POINT(41.5849105404943 74.83685535795699) bank18911 +18912 POINT(39.84336447474149 74.04650147638786) bank18912 +18913 POINT(40.16919990071727 74.93317471099141) bank18913 +18914 POINT(40.0718365547499 73.2678687992747) bank18914 +18915 POINT(40.18156470891348 73.01359450555228) bank18915 +18916 POINT(40.09676785325385 74.48466599982984) bank18916 +18917 POINT(39.77537331976563 73.43520304710283) bank18917 +18918 POINT(39.88788994880457 73.60776762054466) bank18918 +18919 POINT(39.965192675624415 74.04955602833388) bank18919 +18920 POINT(39.92695687240328 74.7975903381766) bank18920 +18921 POINT(40.88295120110665 74.45205060853512) bank18921 +18922 POINT(39.757409462711024 73.46699030524115) bank18922 +18923 POINT(41.52384879738761 73.22506581174308) bank18923 +18924 POINT(40.57542215841183 74.65411490598431) bank18924 +18925 POINT(40.022776473791566 74.01536431712722) bank18925 +18926 POINT(41.41833237868232 73.0529116190984) bank18926 +18927 POINT(40.812015543403625 74.59129017581749) bank18927 +18928 POINT(40.21400132073243 74.71619689726366) bank18928 +18929 POINT(40.85548522284098 73.12100226477997) bank18929 +18930 POINT(40.55185082238301 74.54484598485507) bank18930 +18931 POINT(40.261689351339825 73.78044623405472) bank18931 +18932 POINT(40.245744928936475 73.09537898314409) bank18932 +18933 POINT(41.0746680417284 74.49537484695813) bank18933 +18934 POINT(40.55171608992159 74.69490974601668) bank18934 +18935 POINT(40.891508144524195 73.11016514631594) bank18935 +18936 POINT(40.46697528330297 74.84204297835919) bank18936 +18937 POINT(39.84869835886325 74.63494065718203) bank18937 +18938 POINT(39.846479543904245 74.33575494918875) bank18938 +18939 POINT(39.82108662659968 73.63649037604127) bank18939 +18940 POINT(40.290363416814046 74.80772157998872) bank18940 +18941 POINT(40.571768054021746 74.11682537376721) bank18941 +18942 POINT(40.17295779767029 73.58176549629715) bank18942 +18943 POINT(40.2605332215954 74.82028787299839) bank18943 +18944 POINT(39.9923588296131 73.43964701465082) bank18944 +18945 POINT(40.424283795123166 74.9873391948205) bank18945 +18946 POINT(40.680064709837445 73.33292941176182) bank18946 +18947 POINT(41.21175998641774 74.51696965192696) bank18947 +18948 POINT(40.50977426476596 74.83058514969343) bank18948 +18949 POINT(40.526801131552375 74.8979413936791) bank18949 +18950 POINT(40.23750572630292 74.8082340737935) bank18950 +18951 POINT(41.05712961267047 73.96963190076293) bank18951 +18952 POINT(39.78957405382742 74.2741470698624) bank18952 +18953 POINT(39.803265043179046 74.24680885197934) bank18953 +18954 POINT(40.45795632867319 73.48838505866614) bank18954 +18955 POINT(40.24342956042799 74.0470910727001) bank18955 +18956 POINT(41.42354016237728 74.24157338792455) bank18956 +18957 POINT(41.56857848013353 73.4840324267595) bank18957 +18958 POINT(41.55909711780476 73.82909741744771) bank18958 +18959 POINT(41.670031475080016 73.54916133607722) bank18959 +18960 POINT(39.76731919380533 74.80014423661738) bank18960 +18961 POINT(41.324215675507794 74.0463376482916) bank18961 +18962 POINT(41.15576065100342 74.38592562905579) bank18962 +18963 POINT(41.63243045237585 73.51735171809743) bank18963 +18964 POINT(41.28227400206302 73.22974061888495) bank18964 +18965 POINT(40.02955461289544 73.00783633329239) bank18965 +18966 POINT(40.713043719757586 73.52769119328244) bank18966 +18967 POINT(40.7608342847179 73.30428995431758) bank18967 +18968 POINT(41.710498286642874 74.82891129002941) bank18968 +18969 POINT(40.046754393630984 74.34754362763664) bank18969 +18970 POINT(40.73648882750153 73.18421823663509) bank18970 +18971 POINT(41.005938310173626 74.73487598586846) bank18971 +18972 POINT(41.65660954711783 74.41447541242914) bank18972 +18973 POINT(41.199367308429956 74.73837582452911) bank18973 +18974 POINT(39.743825159391406 73.01744722610385) bank18974 +18975 POINT(40.96144164712674 74.73941888950223) bank18975 +18976 POINT(40.65299996348977 74.09273217019296) bank18976 +18977 POINT(40.88322469681619 74.15472417290513) bank18977 +18978 POINT(41.60759640851116 74.90860583751093) bank18978 +18979 POINT(41.068554701410946 73.6688424384192) bank18979 +18980 POINT(39.797400721625515 74.45413937418795) bank18980 +18981 POINT(41.295343978410635 74.64789822045985) bank18981 +18982 POINT(39.71641501730984 73.48637466433424) bank18982 +18983 POINT(40.75472429381763 74.97424797435066) bank18983 +18984 POINT(40.05339618606584 74.84199734085868) bank18984 +18985 POINT(40.69534673617822 73.28661409443886) bank18985 +18986 POINT(39.949853602435844 74.08375159377988) bank18986 +18987 POINT(41.08343275458581 74.07594397663215) bank18987 +18988 POINT(40.54385450309358 73.88571372947507) bank18988 +18989 POINT(40.179018439012516 74.70706335252972) bank18989 +18990 POINT(39.74598057159167 73.50692574690463) bank18990 +18991 POINT(40.81072776500253 74.7071835609376) bank18991 +18992 POINT(40.24689058034259 73.38138722973342) bank18992 +18993 POINT(39.78181732681886 74.35609950865191) bank18993 +18994 POINT(40.6659780799415 73.43958902438236) bank18994 +18995 POINT(41.01850355964446 73.14027412197132) bank18995 +18996 POINT(41.48503338583035 74.57331158600113) bank18996 +18997 POINT(40.163697329000456 74.22897307249637) bank18997 +18998 POINT(40.65215392814832 74.14785078747693) bank18998 +18999 POINT(40.1461026059009 74.70236012076259) bank18999 +19000 POINT(41.65728503944362 73.63939301865473) bank19000 +19001 POINT(40.88149307435524 73.6280610314558) bank19001 +19002 POINT(41.682325122905695 74.93107862735829) bank19002 +19003 POINT(41.089296178613246 73.04880668989999) bank19003 +19004 POINT(39.93788359088324 74.99772377323058) bank19004 +19005 POINT(40.81092415140413 74.30599525221761) bank19005 +19006 POINT(40.978473446154624 74.12937091905304) bank19006 +19007 POINT(41.450860447743544 74.19843371382206) bank19007 +19008 POINT(41.0519453502627 74.6108625546825) bank19008 +19009 POINT(40.61419701874684 73.5835090549746) bank19009 +19010 POINT(41.68600595916412 73.30273821563208) bank19010 +19011 POINT(40.11028585826186 73.390472800306) bank19011 +19012 POINT(41.50760701833327 74.10652306181434) bank19012 +19013 POINT(40.34367460883367 73.43121092801638) bank19013 +19014 POINT(40.132293188012106 73.37604393965749) bank19014 +19015 POINT(40.642134060151356 74.33497310618401) bank19015 +19016 POINT(40.11314978069191 73.76955765001492) bank19016 +19017 POINT(40.88810388846794 74.84832345040897) bank19017 +19018 POINT(41.441649352432435 73.88505497706203) bank19018 +19019 POINT(40.27904932214617 74.21247533449854) bank19019 +19020 POINT(39.74626157912373 73.13127251306997) bank19020 +19021 POINT(40.826351334079504 74.69511836967112) bank19021 +19022 POINT(40.42447120770363 73.71555047260709) bank19022 +19023 POINT(41.1904285178215 74.82293427809923) bank19023 +19024 POINT(41.262684688603606 73.96979126884244) bank19024 +19025 POINT(40.21116733860965 74.6292220700154) bank19025 +19026 POINT(40.07400128144693 73.138265339736) bank19026 +19027 POINT(41.53329501122054 73.31137686869067) bank19027 +19028 POINT(39.98860719882654 73.12361099269842) bank19028 +19029 POINT(41.31980495473817 73.76352670096111) bank19029 +19030 POINT(40.49661214590733 73.42285552635698) bank19030 +19031 POINT(40.867828574907435 73.45643577690967) bank19031 +19032 POINT(41.25041546452167 74.38085391461799) bank19032 +19033 POINT(40.1984930165891 74.36523692083254) bank19033 +19034 POINT(40.832590882179666 74.06435152156789) bank19034 +19035 POINT(41.0706659589173 74.69799565251559) bank19035 +19036 POINT(39.80733567887503 74.555850236284) bank19036 +19037 POINT(41.391644384509995 73.69106237234656) bank19037 +19038 POINT(41.14410333302949 74.24664486734059) bank19038 +19039 POINT(40.20158586443296 74.95677108369549) bank19039 +19040 POINT(40.59452924519469 74.79452869887773) bank19040 +19041 POINT(40.39704571029705 74.6069631706091) bank19041 +19042 POINT(40.37801279128426 74.98679798852066) bank19042 +19043 POINT(39.94337388365756 74.75520295871198) bank19043 +19044 POINT(40.453027503141925 74.55016774807928) bank19044 +19045 POINT(41.404113653570946 73.52464196901327) bank19045 +19046 POINT(40.35885942272626 74.10757456730248) bank19046 +19047 POINT(40.11600895296863 74.41768570256045) bank19047 +19048 POINT(40.6014921195498 74.59021397941572) bank19048 +19049 POINT(40.93030492049659 73.51926190910022) bank19049 +19050 POINT(39.965081327917346 73.30920434629726) bank19050 +19051 POINT(41.601321437176594 74.80645132256288) bank19051 +19052 POINT(39.89893358128958 74.94590366586708) bank19052 +19053 POINT(39.830396944279244 73.62311286666315) bank19053 +19054 POINT(40.14652587842335 74.49297656741624) bank19054 +19055 POINT(40.21012676322557 73.63434510380063) bank19055 +19056 POINT(41.20156609078216 73.83699429998104) bank19056 +19057 POINT(40.52235112915696 74.4074658635911) bank19057 +19058 POINT(40.26895552144762 74.87385695682575) bank19058 +19059 POINT(41.17268517477673 74.57492165490059) bank19059 +19060 POINT(40.7065464449585 74.9983850093003) bank19060 +19061 POINT(40.976973961972185 74.95988865078287) bank19061 +19062 POINT(41.481991025103866 74.84576726888223) bank19062 +19063 POINT(41.193120348053334 73.7245521882098) bank19063 +19064 POINT(40.535731247361014 73.46451401170035) bank19064 +19065 POINT(40.7634561411213 74.84034231135973) bank19065 +19066 POINT(41.52117530208576 74.35778450110432) bank19066 +19067 POINT(40.774538264167326 74.67989093608085) bank19067 +19068 POINT(41.24452213167317 73.2786894568657) bank19068 +19069 POINT(41.21045779619451 74.75869522874244) bank19069 +19070 POINT(39.864415900478846 73.04296718837594) bank19070 +19071 POINT(41.14578638618274 73.30694015308417) bank19071 +19072 POINT(41.705899435341166 74.86089375225073) bank19072 +19073 POINT(40.909845086847035 73.33172364756895) bank19073 +19074 POINT(40.933302901504085 74.05239925232648) bank19074 +19075 POINT(39.83846993890917 74.29350001400145) bank19075 +19076 POINT(41.394713404274086 73.70464964557738) bank19076 +19077 POINT(41.44111556284707 74.4932464221946) bank19077 +19078 POINT(41.64063053128169 74.65075998448586) bank19078 +19079 POINT(41.27264986939217 73.87316696718348) bank19079 +19080 POINT(41.30312616138563 74.77505017184889) bank19080 +19081 POINT(41.054258776576184 74.03498672345344) bank19081 +19082 POINT(41.026136795786186 73.72677652015828) bank19082 +19083 POINT(40.75099288179125 73.24440568520613) bank19083 +19084 POINT(40.979765350464966 74.02275397653881) bank19084 +19085 POINT(39.88292280936106 74.97486727213277) bank19085 +19086 POINT(41.59193295736052 74.24380605673774) bank19086 +19087 POINT(40.02524969175907 73.03444919879513) bank19087 +19088 POINT(39.96452304669563 74.07929819485204) bank19088 +19089 POINT(41.32513198424591 74.64145841361402) bank19089 +19090 POINT(40.51248284825801 74.70026999388158) bank19090 +19091 POINT(41.27857749903449 73.03586317011779) bank19091 +19092 POINT(41.220442037981215 73.51016434340177) bank19092 +19093 POINT(39.83031991877232 73.71001306593767) bank19093 +19094 POINT(41.59518343394827 73.03524291577523) bank19094 +19095 POINT(40.20949774612076 73.84783907292206) bank19095 +19096 POINT(40.872398348979615 73.02152253624674) bank19096 +19097 POINT(40.555785645640775 73.0379324021887) bank19097 +19098 POINT(41.665089153574925 73.11118637413756) bank19098 +19099 POINT(40.98974594454397 74.21707112714678) bank19099 +19100 POINT(40.51132393761749 74.12063880722403) bank19100 +19101 POINT(41.56463287985255 74.84500822625614) bank19101 +19102 POINT(40.21035742084032 73.82040439423001) bank19102 +19103 POINT(41.334764411457186 74.5679478864604) bank19103 +19104 POINT(41.47967960009635 73.04166798166361) bank19104 +19105 POINT(39.866362291948484 73.61323462421427) bank19105 +19106 POINT(41.679448297933114 73.66675028464277) bank19106 +19107 POINT(41.67950200621919 74.77252701651543) bank19107 +19108 POINT(40.70277451223556 74.89182904090147) bank19108 +19109 POINT(40.57503032971034 73.90988090413862) bank19109 +19110 POINT(41.53207668963116 73.25544699142175) bank19110 +19111 POINT(40.79958648306577 73.83929487287544) bank19111 +19112 POINT(40.74824239972834 73.35450991376645) bank19112 +19113 POINT(40.47786584121165 73.3868874946431) bank19113 +19114 POINT(41.64788700693 73.14410720782617) bank19114 +19115 POINT(39.92180878279509 74.0587767690407) bank19115 +19116 POINT(39.801317247761446 73.1118604856826) bank19116 +19117 POINT(39.86056530310748 74.18691382861627) bank19117 +19118 POINT(41.3868045389036 74.30713126425862) bank19118 +19119 POINT(40.90972433137674 74.7066302777721) bank19119 +19120 POINT(39.95951464459107 74.0232373255663) bank19120 +19121 POINT(41.439183872079546 74.41260334156541) bank19121 +19122 POINT(40.28564605471229 73.2694589171236) bank19122 +19123 POINT(41.64816273093986 74.4584928585314) bank19123 +19124 POINT(40.375842642160144 73.71491061866926) bank19124 +19125 POINT(40.56671112571595 74.03818093026986) bank19125 +19126 POINT(39.835817806605675 73.09242537384334) bank19126 +19127 POINT(40.015940885140935 74.74476650919253) bank19127 +19128 POINT(40.7900424694314 73.39913244171635) bank19128 +19129 POINT(40.04985604360761 74.94849652908111) bank19129 +19130 POINT(41.46902734462632 73.19006141170478) bank19130 +19131 POINT(39.888725942430256 74.65821659077199) bank19131 +19132 POINT(39.88589474637956 74.27231610766708) bank19132 +19133 POINT(41.2806742377493 73.24342674307887) bank19133 +19134 POINT(41.659459941393656 73.29819438874705) bank19134 +19135 POINT(40.691877672873794 74.85365815017411) bank19135 +19136 POINT(40.85957488638644 73.99040711977351) bank19136 +19137 POINT(41.13773487209566 74.63066235091438) bank19137 +19138 POINT(39.79283623192785 74.9954340414678) bank19138 +19139 POINT(41.109705795409894 74.10885130893209) bank19139 +19140 POINT(41.19214768177958 73.17018181783293) bank19140 +19141 POINT(40.41614048796308 73.4650597172245) bank19141 +19142 POINT(40.83707810656556 73.8097686272752) bank19142 +19143 POINT(40.282763265723965 74.31488994374945) bank19143 +19144 POINT(41.6736060392355 74.36636850780609) bank19144 +19145 POINT(41.57411000821513 73.48927838066457) bank19145 +19146 POINT(41.151450527441 73.0100811099462) bank19146 +19147 POINT(39.71300339667319 73.56789413279901) bank19147 +19148 POINT(41.186503989346804 74.18031453580295) bank19148 +19149 POINT(40.978904550949736 74.62480650713482) bank19149 +19150 POINT(39.71827255151803 74.83768550837316) bank19150 +19151 POINT(40.33565559597926 74.1685288592859) bank19151 +19152 POINT(40.94177365461464 73.89278555042202) bank19152 +19153 POINT(40.30827307047105 74.45514183596507) bank19153 +19154 POINT(40.39388928521327 74.96788544317843) bank19154 +19155 POINT(41.57015677240619 73.02467711885105) bank19155 +19156 POINT(41.46238085594114 74.54411302754944) bank19156 +19157 POINT(41.328676013829856 73.38096318823854) bank19157 +19158 POINT(40.30437861616273 74.99450298760375) bank19158 +19159 POINT(40.31079439537697 74.91761322169434) bank19159 +19160 POINT(40.064487963745634 73.57997140582847) bank19160 +19161 POINT(41.635096898028515 73.09841417291831) bank19161 +19162 POINT(40.87972694844798 74.71906839085663) bank19162 +19163 POINT(40.50252709146169 73.09600723023084) bank19163 +19164 POINT(39.753135005358146 74.37990133619212) bank19164 +19165 POINT(40.47556058726777 74.58363609405144) bank19165 +19166 POINT(40.56789900300759 73.08250335466313) bank19166 +19167 POINT(40.44996210372078 74.37658242054849) bank19167 +19168 POINT(41.030240969682765 73.49043635747343) bank19168 +19169 POINT(41.29213966027588 74.22700735761143) bank19169 +19170 POINT(40.612885761681596 73.32703334635967) bank19170 +19171 POINT(41.17804545192316 73.27701907920247) bank19171 +19172 POINT(40.78418482003916 73.6316096877033) bank19172 +19173 POINT(40.2916883649707 74.93441398875167) bank19173 +19174 POINT(41.371646124350235 73.76652789380927) bank19174 +19175 POINT(40.45958166419077 73.91783522406243) bank19175 +19176 POINT(41.292482675003846 74.2768425637516) bank19176 +19177 POINT(41.064538331927885 74.73058542418555) bank19177 +19178 POINT(40.95632183225001 73.27811066924656) bank19178 +19179 POINT(41.62991018020078 73.2791355705918) bank19179 +19180 POINT(41.694518915225196 74.56996861593763) bank19180 +19181 POINT(41.232535908002916 74.97710065627157) bank19181 +19182 POINT(40.81873110490038 74.93355327335983) bank19182 +19183 POINT(40.70129121069085 74.75390025151273) bank19183 +19184 POINT(41.09142931703867 74.60033278125708) bank19184 +19185 POINT(41.21595121842997 74.03903947777586) bank19185 +19186 POINT(41.04198381583418 74.81077744396595) bank19186 +19187 POINT(40.55071691430062 74.26813004609555) bank19187 +19188 POINT(39.94364863297479 73.7201956198921) bank19188 +19189 POINT(41.582397127109495 74.32022595835538) bank19189 +19190 POINT(41.65178060582645 74.96428081662859) bank19190 +19191 POINT(39.94493933987197 73.97397798897185) bank19191 +19192 POINT(40.552185244967134 73.37627704461121) bank19192 +19193 POINT(41.7101013453869 73.79516965895436) bank19193 +19194 POINT(40.559946062121846 73.21613429804565) bank19194 +19195 POINT(41.14251766106672 74.88789563551236) bank19195 +19196 POINT(40.7571908090088 74.19869324639635) bank19196 +19197 POINT(39.736422886649386 74.9432609965774) bank19197 +19198 POINT(39.905663401701794 73.9734016170679) bank19198 +19199 POINT(41.1196028098986 74.50873097390665) bank19199 +19200 POINT(40.63822928778744 73.83870912541153) bank19200 +19201 POINT(40.155485628674576 74.83361623757) bank19201 +19202 POINT(41.35516096086273 73.36757615238331) bank19202 +19203 POINT(39.76075496858136 74.33254071708083) bank19203 +19204 POINT(40.3884266663761 73.23309369194361) bank19204 +19205 POINT(41.07097904615116 74.17255955932708) bank19205 +19206 POINT(39.8096446110144 73.26796636630287) bank19206 +19207 POINT(40.21893300931006 73.47086285448898) bank19207 +19208 POINT(40.953359465474534 74.09731305942782) bank19208 +19209 POINT(41.32148040792893 74.72105293195153) bank19209 +19210 POINT(39.8754523329044 74.31825455164352) bank19210 +19211 POINT(39.94287385202341 74.64333301019118) bank19211 +19212 POINT(40.270934125363794 74.89834691260347) bank19212 +19213 POINT(40.722933566905375 73.76219408157002) bank19213 +19214 POINT(40.45448086501783 74.93567717830132) bank19214 +19215 POINT(40.08900978341685 74.2432840807744) bank19215 +19216 POINT(40.68263132175183 73.35666965308634) bank19216 +19217 POINT(40.33449099476409 73.6911279490404) bank19217 +19218 POINT(40.61549717699545 74.30751193466315) bank19218 +19219 POINT(40.689624783738 74.28359329124781) bank19219 +19220 POINT(40.189886137284 74.77777133582856) bank19220 +19221 POINT(41.38419679637584 74.44705209911693) bank19221 +19222 POINT(40.63063576690937 74.35496491762738) bank19222 +19223 POINT(40.63704289983628 73.47078814471298) bank19223 +19224 POINT(40.24678552144556 73.02912234171178) bank19224 +19225 POINT(41.21870241118301 74.47898676216084) bank19225 +19226 POINT(40.14333554381311 74.09618149346136) bank19226 +19227 POINT(40.541727347378874 74.35753520241984) bank19227 +19228 POINT(41.243568528129344 73.36459992975816) bank19228 +19229 POINT(40.56173139239215 73.575125600209) bank19229 +19230 POINT(41.53020784641999 73.77506276871166) bank19230 +19231 POINT(41.55165593370333 73.59577244108783) bank19231 +19232 POINT(40.20926582548111 73.48824860977405) bank19232 +19233 POINT(39.77887605833569 73.23623063469323) bank19233 +19234 POINT(41.439147719594175 73.23565472740519) bank19234 +19235 POINT(40.03857995359136 73.35372763851174) bank19235 +19236 POINT(39.94830509071769 73.23337358669357) bank19236 +19237 POINT(40.863983720837915 74.50019702149915) bank19237 +19238 POINT(40.639886840619255 73.113878883721) bank19238 +19239 POINT(41.58129974764963 74.86888120428293) bank19239 +19240 POINT(41.28436160753419 73.35624138452938) bank19240 +19241 POINT(40.00632819582509 73.85158063245353) bank19241 +19242 POINT(40.210153848133004 74.10285893775041) bank19242 +19243 POINT(40.61921097087963 74.39778160730685) bank19243 +19244 POINT(40.56513901299898 74.34527776774118) bank19244 +19245 POINT(39.9125073309826 74.19757271850509) bank19245 +19246 POINT(41.316655690501754 73.61328511705443) bank19246 +19247 POINT(41.336887374645265 74.68908485282353) bank19247 +19248 POINT(41.50457365255264 73.42117619641833) bank19248 +19249 POINT(41.22113923936919 74.21367403663344) bank19249 +19250 POINT(41.10075874457928 73.01690729170032) bank19250 +19251 POINT(39.84869168008781 73.7609743808299) bank19251 +19252 POINT(40.73177303526596 73.61488112856176) bank19252 +19253 POINT(40.54781958295152 73.10997026527983) bank19253 +19254 POINT(40.79750048495717 73.49149980596799) bank19254 +19255 POINT(41.0463485439034 74.93794496629742) bank19255 +19256 POINT(41.50379682153423 73.9308867416342) bank19256 +19257 POINT(40.54041380421016 74.16607137644705) bank19257 +19258 POINT(41.48854651488927 74.13952327510472) bank19258 +19259 POINT(40.75593541020599 74.17229680389364) bank19259 +19260 POINT(39.89998459738643 73.02830605254786) bank19260 +19261 POINT(40.42179658137267 73.53327542442749) bank19261 +19262 POINT(41.65858818444493 73.86554800272766) bank19262 +19263 POINT(41.14912124551555 73.20167015513216) bank19263 +19264 POINT(41.411065967233256 74.62701903098797) bank19264 +19265 POINT(41.13408879099442 74.76375992915301) bank19265 +19266 POINT(41.52401579899423 73.6897473314184) bank19266 +19267 POINT(40.82082872623467 74.6208954501359) bank19267 +19268 POINT(39.85578918652204 74.079649040451) bank19268 +19269 POINT(40.60710398303927 74.57649888922577) bank19269 +19270 POINT(40.025569660381265 73.78969909652612) bank19270 +19271 POINT(41.161888056185695 73.43503284266045) bank19271 +19272 POINT(40.17952104825385 74.27135819152309) bank19272 +19273 POINT(39.94603488568747 73.8702477484008) bank19273 +19274 POINT(39.92692453589468 74.23985183956468) bank19274 +19275 POINT(40.62527150311799 73.9274206885306) bank19275 +19276 POINT(40.39117896266476 73.71082002337508) bank19276 +19277 POINT(40.77739659817914 73.63510696952308) bank19277 +19278 POINT(41.27799656748323 74.03725623682838) bank19278 +19279 POINT(40.38488166552111 73.62319020368484) bank19279 +19280 POINT(40.02080711857739 74.16305008578587) bank19280 +19281 POINT(40.892673441307764 73.27125136822092) bank19281 +19282 POINT(41.23188986381972 73.39890911465791) bank19282 +19283 POINT(40.29598056787436 73.71872975368471) bank19283 +19284 POINT(40.14325503285573 73.37215466548938) bank19284 +19285 POINT(40.37849555672796 74.90785793305685) bank19285 +19286 POINT(40.41830246983104 73.75200322018469) bank19286 +19287 POINT(40.42104042259509 74.64672771814236) bank19287 +19288 POINT(41.13811718895261 73.5749012690577) bank19288 +19289 POINT(40.30584297268539 73.76745336194563) bank19289 +19290 POINT(40.25077893237306 73.14116988021652) bank19290 +19291 POINT(40.6485864823859 74.11749433551664) bank19291 +19292 POINT(41.59571502427816 73.05602710212496) bank19292 +19293 POINT(41.23890045014289 74.74276206399283) bank19293 +19294 POINT(41.50731626023283 73.57711916593833) bank19294 +19295 POINT(40.88007433426557 74.66918529580705) bank19295 +19296 POINT(40.394486981872085 74.56184245544739) bank19296 +19297 POINT(41.21954186880239 73.40992889596598) bank19297 +19298 POINT(41.01290773017267 73.54089808706213) bank19298 +19299 POINT(40.41699925019619 74.53080414609936) bank19299 +19300 POINT(41.35221625671258 74.79551723109518) bank19300 +19301 POINT(41.01683586747316 73.55649661677786) bank19301 +19302 POINT(41.088575645797924 74.21249123146669) bank19302 +19303 POINT(40.43605744463107 74.27005711549006) bank19303 +19304 POINT(39.94498377904 74.50648575923911) bank19304 +19305 POINT(40.940143101316536 73.78622368334277) bank19305 +19306 POINT(41.224028693581445 73.26392449406511) bank19306 +19307 POINT(40.27350695234169 74.20199426056607) bank19307 +19308 POINT(41.48007954046424 74.77745971198709) bank19308 +19309 POINT(41.55347196736478 73.17456638717515) bank19309 +19310 POINT(40.6623865382487 74.25891119807518) bank19310 +19311 POINT(41.45934245579707 74.52699313162273) bank19311 +19312 POINT(40.585644238828436 74.7722569787304) bank19312 +19313 POINT(39.82055755764169 73.7583093094615) bank19313 +19314 POINT(39.851276028708746 74.50928305280308) bank19314 +19315 POINT(40.60217473728182 74.8206378515823) bank19315 +19316 POINT(40.46043492991648 73.39288419233594) bank19316 +19317 POINT(41.68833616318695 74.78263320097669) bank19317 +19318 POINT(41.11908715747664 74.52100551538217) bank19318 +19319 POINT(41.450317464231716 74.07307074100586) bank19319 +19320 POINT(39.77285321056199 73.08330177631345) bank19320 +19321 POINT(40.04691734506542 73.93303439940743) bank19321 +19322 POINT(41.051804579567715 73.87768670800465) bank19322 +19323 POINT(41.05903807631916 74.22527625012533) bank19323 +19324 POINT(41.64307004655626 73.82530758191855) bank19324 +19325 POINT(40.12233935116342 74.99575240849487) bank19325 +19326 POINT(41.01563735999867 73.8381514184677) bank19326 +19327 POINT(40.630670619386315 74.78631713344824) bank19327 +19328 POINT(41.219061266042125 73.47159534624964) bank19328 +19329 POINT(40.43688020509087 73.71502391959925) bank19329 +19330 POINT(39.8596291889814 74.46371871293832) bank19330 +19331 POINT(40.81675919974923 73.58666131388536) bank19331 +19332 POINT(41.53315248913992 74.51933998394458) bank19332 +19333 POINT(41.10091970945491 73.08278634868552) bank19333 +19334 POINT(39.95284359991804 73.78960342734379) bank19334 +19335 POINT(40.69839629485302 74.12222300805048) bank19335 +19336 POINT(40.342971853098895 74.1076706683737) bank19336 +19337 POINT(40.70195325441822 74.69438999281637) bank19337 +19338 POINT(40.95735423619579 73.50351249675975) bank19338 +19339 POINT(41.14199766709061 74.14471077898484) bank19339 +19340 POINT(40.89211010552609 73.65483012060811) bank19340 +19341 POINT(40.069469706576584 74.20419046651999) bank19341 +19342 POINT(39.8575057651744 74.86179788501505) bank19342 +19343 POINT(41.313609710576266 74.07739350558784) bank19343 +19344 POINT(41.01867110711879 74.72508983109786) bank19344 +19345 POINT(39.86778179726286 74.22997019688167) bank19345 +19346 POINT(40.10280934222141 73.340283379347) bank19346 +19347 POINT(40.444241969682 73.14299956357962) bank19347 +19348 POINT(40.734114991791685 74.43495812123851) bank19348 +19349 POINT(39.874849912278414 73.41239419372916) bank19349 +19350 POINT(40.6311057161254 74.90369042334981) bank19350 +19351 POINT(41.177322486083845 73.83810441594122) bank19351 +19352 POINT(40.56334960255581 74.25787671431259) bank19352 +19353 POINT(41.624038939102576 74.5714867710952) bank19353 +19354 POINT(40.47599303306172 73.22117240642676) bank19354 +19355 POINT(40.52802444255882 73.61158198666288) bank19355 +19356 POINT(40.7584041262003 73.60724886140548) bank19356 +19357 POINT(40.03769731293285 73.65172574597784) bank19357 +19358 POINT(40.141276137143244 74.43423017844015) bank19358 +19359 POINT(41.623312565112556 73.99485946144452) bank19359 +19360 POINT(39.9516277035489 74.31637520018106) bank19360 +19361 POINT(39.77993962253264 73.79440732050068) bank19361 +19362 POINT(40.74192995047901 73.13494177807203) bank19362 +19363 POINT(41.04983084453718 74.14196925092851) bank19363 +19364 POINT(39.864878027597356 74.80596499308693) bank19364 +19365 POINT(39.910908674816994 74.4875512436986) bank19365 +19366 POINT(40.06393020386766 74.56394038415307) bank19366 +19367 POINT(40.11497162470537 74.4098704406522) bank19367 +19368 POINT(39.81131130518658 73.08234239919967) bank19368 +19369 POINT(40.51643028007167 74.22908857196616) bank19369 +19370 POINT(40.494907186715736 73.25733576906744) bank19370 +19371 POINT(41.298057141408876 74.44210825664537) bank19371 +19372 POINT(40.16689890935249 74.37130018638454) bank19372 +19373 POINT(40.44456809084551 74.54426860004588) bank19373 +19374 POINT(40.97107347495568 73.2603931581983) bank19374 +19375 POINT(41.199197260860004 74.54918774929784) bank19375 +19376 POINT(40.877144451755115 73.80862265519985) bank19376 +19377 POINT(41.08787308326957 73.46854056567614) bank19377 +19378 POINT(40.329457668258215 74.89568033532812) bank19378 +19379 POINT(41.10801146184119 74.09513336580908) bank19379 +19380 POINT(41.49097456631833 74.33319270066464) bank19380 +19381 POINT(39.80791075736983 74.93986982781927) bank19381 +19382 POINT(40.70444552534583 74.5471035937481) bank19382 +19383 POINT(41.00502261304188 73.21477020018149) bank19383 +19384 POINT(40.83672419634453 73.91655286205717) bank19384 +19385 POINT(41.56764760656656 73.642932753548) bank19385 +19386 POINT(40.70890594875279 74.85136494835874) bank19386 +19387 POINT(40.61214021799449 74.62949422243831) bank19387 +19388 POINT(41.51451449839698 73.90880914961248) bank19388 +19389 POINT(40.9815286534484 74.78151809375251) bank19389 +19390 POINT(39.92848536756341 74.05249151533796) bank19390 +19391 POINT(41.302687889083536 73.46746732987503) bank19391 +19392 POINT(41.423894097328905 74.66487453632543) bank19392 +19393 POINT(41.46948368461732 74.43943957717877) bank19393 +19394 POINT(41.233953076705504 74.51589487007084) bank19394 +19395 POINT(41.037261210958064 74.86981284520681) bank19395 +19396 POINT(41.22943350221584 74.6315489535317) bank19396 +19397 POINT(40.65957209367608 73.19794694329136) bank19397 +19398 POINT(40.23235192176469 74.45513896796393) bank19398 +19399 POINT(41.20427936664374 74.86112817582843) bank19399 +19400 POINT(40.73847481595478 74.54771912283522) bank19400 +19401 POINT(40.20688928742051 73.22245952202661) bank19401 +19402 POINT(39.97988789984572 74.43428943580312) bank19402 +19403 POINT(40.66228532536055 73.34512468571593) bank19403 +19404 POINT(40.50313311766634 74.66848514286534) bank19404 +19405 POINT(41.5488190263201 73.38037443440736) bank19405 +19406 POINT(41.56503856846369 74.29744718501323) bank19406 +19407 POINT(40.39972240132806 73.38860618474123) bank19407 +19408 POINT(40.408816125429546 73.34686456509486) bank19408 +19409 POINT(41.65478742858567 74.00767414173013) bank19409 +19410 POINT(41.20361252179342 74.90564416823024) bank19410 +19411 POINT(40.61972646063858 74.53790443395839) bank19411 +19412 POINT(41.076431828612236 73.01713958817325) bank19412 +19413 POINT(39.74267734551636 73.02766630359292) bank19413 +19414 POINT(41.60428566117579 73.35740666051632) bank19414 +19415 POINT(40.292972787103544 74.46157946163102) bank19415 +19416 POINT(39.994684302022826 74.68380009126383) bank19416 +19417 POINT(41.305440023919 74.99637489649368) bank19417 +19418 POINT(39.771531406774905 74.43625643111336) bank19418 +19419 POINT(41.64068230061497 73.63748661448123) bank19419 +19420 POINT(41.56243634315166 74.24540116173132) bank19420 +19421 POINT(41.364818081077615 74.6752801751161) bank19421 +19422 POINT(41.465251395928505 74.67206195540855) bank19422 +19423 POINT(41.64266123421033 73.48745466496767) bank19423 +19424 POINT(41.52795603260377 73.54139087277414) bank19424 +19425 POINT(40.66735002580929 74.82146128005584) bank19425 +19426 POINT(41.694664350272106 74.77737633541946) bank19426 +19427 POINT(41.493228669127134 73.21227250583583) bank19427 +19428 POINT(40.14263310885403 73.99941972191537) bank19428 +19429 POINT(39.95326001843341 74.1557750341376) bank19429 +19430 POINT(41.59829286081399 74.9512479672425) bank19430 +19431 POINT(41.06533065111483 74.70298760786919) bank19431 +19432 POINT(40.33251062638233 73.29387489919118) bank19432 +19433 POINT(40.746831923015954 74.89958340098799) bank19433 +19434 POINT(40.35154141983748 73.5336688160143) bank19434 +19435 POINT(40.32451182273478 73.10673186444617) bank19435 +19436 POINT(40.404932656507405 73.97976327046734) bank19436 +19437 POINT(40.62300916405499 73.02143412275123) bank19437 +19438 POINT(40.43023721995511 74.28431822053948) bank19438 +19439 POINT(39.76470089070913 74.88831253811404) bank19439 +19440 POINT(40.981709341165924 74.18986401127725) bank19440 +19441 POINT(40.977975594349815 74.11174130841542) bank19441 +19442 POINT(39.9379026367715 73.69558330771736) bank19442 +19443 POINT(41.56636475760591 73.03158679959546) bank19443 +19444 POINT(40.444607647079714 73.57647009489389) bank19444 +19445 POINT(41.38195200338447 74.20502054814719) bank19445 +19446 POINT(40.45467849995491 74.56738595718006) bank19446 +19447 POINT(39.88365732507818 73.60709977215305) bank19447 +19448 POINT(41.390289262056456 73.97941254912581) bank19448 +19449 POINT(39.79029164639141 74.54912702377149) bank19449 +19450 POINT(40.85293343878265 74.70750153080911) bank19450 +19451 POINT(39.91649307833482 74.3213332316502) bank19451 +19452 POINT(41.31436870097984 74.37803171889239) bank19452 +19453 POINT(41.63038996516235 73.42754183806645) bank19453 +19454 POINT(41.510371100151744 74.36637673588643) bank19454 +19455 POINT(41.25345671659943 74.96708121697085) bank19455 +19456 POINT(40.87784404613646 73.35554774368768) bank19456 +19457 POINT(40.17832393234246 74.32658595649258) bank19457 +19458 POINT(40.58973772906331 74.47810673619861) bank19458 +19459 POINT(39.84817983614611 74.16679531208338) bank19459 +19460 POINT(39.87161736597945 74.46721793911401) bank19460 +19461 POINT(39.81800091393134 74.5212793956259) bank19461 +19462 POINT(40.51033992453585 74.43337874816274) bank19462 +19463 POINT(39.9576377628215 74.37410188948735) bank19463 +19464 POINT(40.25823943760432 74.46452576561676) bank19464 +19465 POINT(40.73438584213143 74.00002251345447) bank19465 +19466 POINT(40.02918530201615 73.50534831380192) bank19466 +19467 POINT(40.51550291065426 74.85486665375711) bank19467 +19468 POINT(40.243756779274776 73.22552973761377) bank19468 +19469 POINT(40.20167891449443 73.85600044293005) bank19469 +19470 POINT(40.42512555406485 73.35333316774752) bank19470 +19471 POINT(41.40653812966895 74.07889847602459) bank19471 +19472 POINT(39.87574839181599 73.89633402484978) bank19472 +19473 POINT(41.34013010605556 74.55824230151386) bank19473 +19474 POINT(41.33118256060061 73.80271177304516) bank19474 +19475 POINT(40.35292363815993 73.61934252018047) bank19475 +19476 POINT(40.87087029935099 73.75305417488042) bank19476 +19477 POINT(40.65382272286744 73.28245047868373) bank19477 +19478 POINT(40.70204307119727 73.0396588227429) bank19478 +19479 POINT(41.38191133511111 74.05257018329216) bank19479 +19480 POINT(40.269218752079674 74.35289417155646) bank19480 +19481 POINT(41.6837084280919 73.10762606206767) bank19481 +19482 POINT(39.908781925942996 74.18305391667994) bank19482 +19483 POINT(41.14646344827832 74.36273263655957) bank19483 +19484 POINT(41.41167725697854 73.17678515748618) bank19484 +19485 POINT(40.48543965176018 73.07725730169126) bank19485 +19486 POINT(41.565316573654314 73.9796706631645) bank19486 +19487 POINT(39.949213904957645 74.34829611080251) bank19487 +19488 POINT(39.928997245265684 73.8107747047512) bank19488 +19489 POINT(41.25109056618524 74.09053425764998) bank19489 +19490 POINT(40.62323589176584 74.80958196026823) bank19490 +19491 POINT(40.547990018339455 73.42524551388072) bank19491 +19492 POINT(41.22986726114733 73.03499221244502) bank19492 +19493 POINT(40.951989078294496 74.35665221510186) bank19493 +19494 POINT(40.058443644385434 74.00824652387749) bank19494 +19495 POINT(40.3308342866845 73.79254612863014) bank19495 +19496 POINT(40.29064496445942 73.27751953081926) bank19496 +19497 POINT(39.81708701901293 73.47105652454377) bank19497 +19498 POINT(40.52927684550827 74.63819617966016) bank19498 +19499 POINT(41.61107548314433 74.12453143566267) bank19499 +19500 POINT(39.746352298405185 74.47831436461188) bank19500 +19501 POINT(40.307273627470515 74.38506596308501) bank19501 +19502 POINT(41.21944581888587 74.59072500331916) bank19502 +19503 POINT(41.49758587435992 73.13172173836661) bank19503 +19504 POINT(40.86259234393763 74.1516406286434) bank19504 +19505 POINT(39.83425857152525 73.95302221365905) bank19505 +19506 POINT(40.3884918888699 74.1268306247402) bank19506 +19507 POINT(41.3829074294546 73.06544009774835) bank19507 +19508 POINT(40.7790225031742 73.5933062417966) bank19508 +19509 POINT(40.16990246470083 74.41726644221345) bank19509 +19510 POINT(40.45105460164638 74.90092485530596) bank19510 +19511 POINT(40.89182739847437 73.38387356972993) bank19511 +19512 POINT(41.44072533387102 73.14394266047739) bank19512 +19513 POINT(40.84904666477191 73.85044460106097) bank19513 +19514 POINT(40.8207361704522 73.44284579721592) bank19514 +19515 POINT(40.31156860142322 74.24692693176638) bank19515 +19516 POINT(41.47606485933491 73.84381706061325) bank19516 +19517 POINT(39.887484420618016 73.38812995363797) bank19517 +19518 POINT(40.905186589995985 73.20236190683455) bank19518 +19519 POINT(41.276842323816744 73.12795253634049) bank19519 +19520 POINT(40.27619617911431 73.22724045148964) bank19520 +19521 POINT(41.23915866689408 73.1218352601447) bank19521 +19522 POINT(41.181205658284036 73.87192522389417) bank19522 +19523 POINT(39.79138993764365 73.150696733745) bank19523 +19524 POINT(39.758613814152 74.59821191800113) bank19524 +19525 POINT(40.53361261496408 73.22481386650331) bank19525 +19526 POINT(41.44820762511842 73.04156455906835) bank19526 +19527 POINT(41.44181361532639 73.87194004572274) bank19527 +19528 POINT(41.20932728778915 74.95273965344714) bank19528 +19529 POINT(39.98550626675365 73.20790140025235) bank19529 +19530 POINT(40.12059710585917 74.5863391882282) bank19530 +19531 POINT(40.96287157331446 74.16012017212633) bank19531 +19532 POINT(40.324619237636675 74.96580332376756) bank19532 +19533 POINT(40.75215060203964 73.71944381769424) bank19533 +19534 POINT(39.911720372854305 74.76110309895019) bank19534 +19535 POINT(40.202439961426414 73.34825717300467) bank19535 +19536 POINT(40.11252864897983 73.59953745316542) bank19536 +19537 POINT(40.625189667903044 73.93243035420937) bank19537 +19538 POINT(40.32313338642446 73.69790606649241) bank19538 +19539 POINT(41.01380536737711 73.66598140114374) bank19539 +19540 POINT(41.37029734175049 74.46029898422641) bank19540 +19541 POINT(41.13232562830455 74.64541278215829) bank19541 +19542 POINT(40.58340909243806 73.39473655889773) bank19542 +19543 POINT(39.82967847802592 73.77183574349839) bank19543 +19544 POINT(40.48207234756248 73.26096889101417) bank19544 +19545 POINT(41.31879238107222 74.03078661139547) bank19545 +19546 POINT(40.334260382344716 73.7816747501818) bank19546 +19547 POINT(39.84724131067227 73.10433868087327) bank19547 +19548 POINT(39.726716624585684 74.11968859483012) bank19548 +19549 POINT(41.314226503969415 74.53622901412665) bank19549 +19550 POINT(41.20819386351829 74.29567033157579) bank19550 +19551 POINT(40.69716889757911 74.19637619906035) bank19551 +19552 POINT(40.86911109888986 73.01165014533152) bank19552 +19553 POINT(39.789322861696874 73.4986219414343) bank19553 +19554 POINT(40.921737063654604 74.39662793182481) bank19554 +19555 POINT(40.94708651884923 74.57944901720658) bank19555 +19556 POINT(40.607758604869844 74.92782470283169) bank19556 +19557 POINT(40.836657415498514 74.26662526329497) bank19557 +19558 POINT(41.50725361255892 74.56175676668248) bank19558 +19559 POINT(41.535707433560965 73.02132485190191) bank19559 +19560 POINT(40.001314961849815 74.63749324419163) bank19560 +19561 POINT(41.54263090937246 74.2759680852868) bank19561 +19562 POINT(41.65395285711388 74.12905208748577) bank19562 +19563 POINT(40.36008094494049 74.84675391316343) bank19563 +19564 POINT(39.731058340142994 74.4877746863816) bank19564 +19565 POINT(41.6782229334688 74.61169674797193) bank19565 +19566 POINT(40.80289826832096 73.08377679177947) bank19566 +19567 POINT(41.6696867551278 73.68480952754) bank19567 +19568 POINT(40.11810476496901 73.039624913776) bank19568 +19569 POINT(41.166358775772814 74.39733583830993) bank19569 +19570 POINT(39.88379387194117 75.00186274590494) bank19570 +19571 POINT(41.013230220366005 74.02324092212577) bank19571 +19572 POINT(40.392209764927514 73.90614336905318) bank19572 +19573 POINT(40.253507388538175 74.0504328414289) bank19573 +19574 POINT(40.65488399924386 73.7653662589445) bank19574 +19575 POINT(41.11374360981506 73.95153871667709) bank19575 +19576 POINT(40.790130125344334 73.02526415364206) bank19576 +19577 POINT(41.65229109047594 73.24284977354652) bank19577 +19578 POINT(40.513681810844254 74.40462727249593) bank19578 +19579 POINT(40.46586015836211 74.90456514573184) bank19579 +19580 POINT(40.27307740081031 74.38695581318979) bank19580 +19581 POINT(40.15462618306574 74.8116773037339) bank19581 +19582 POINT(41.31264971363687 74.82260676133164) bank19582 +19583 POINT(41.050653977576445 74.19712097859514) bank19583 +19584 POINT(41.2651092728846 73.96317609041749) bank19584 +19585 POINT(41.378452930047864 74.01723222954584) bank19585 +19586 POINT(40.259500776101945 74.65930503714023) bank19586 +19587 POINT(40.68670998124422 74.41222366870569) bank19587 +19588 POINT(39.725109312715695 73.85453162563803) bank19588 +19589 POINT(41.4006300614675 74.18058192949648) bank19589 +19590 POINT(40.079915215279364 73.20022451093146) bank19590 +19591 POINT(40.70261107377904 74.47324295908189) bank19591 +19592 POINT(41.12725376628402 74.41724408861893) bank19592 +19593 POINT(41.4616984909453 74.34433730992565) bank19593 +19594 POINT(40.434421373450135 73.10216522438712) bank19594 +19595 POINT(40.35711468382067 74.1967131280411) bank19595 +19596 POINT(40.69062319945329 73.37884439028024) bank19596 +19597 POINT(40.19974217497602 73.30509780242907) bank19597 +19598 POINT(40.51897872210665 73.35845236286767) bank19598 +19599 POINT(39.78399547336934 73.07730592397898) bank19599 +19600 POINT(40.9830000924524 73.56022660709654) bank19600 +19601 POINT(39.71451646220459 74.2634134925433) bank19601 +19602 POINT(40.66684244136286 73.62144534426177) bank19602 +19603 POINT(40.40877293563646 74.93814793700251) bank19603 +19604 POINT(40.099128428076156 73.98664754219287) bank19604 +19605 POINT(39.88666215129178 73.03029267029979) bank19605 +19606 POINT(39.725162297450254 73.13820474055487) bank19606 +19607 POINT(40.46546637857624 74.42108653102149) bank19607 +19608 POINT(41.18218839930862 74.06269287280618) bank19608 +19609 POINT(40.8852330072646 73.00674490871994) bank19609 +19610 POINT(39.98919159787609 73.22763733091894) bank19610 +19611 POINT(41.14526148691081 74.73881559373461) bank19611 +19612 POINT(41.6776786222327 74.90741418770476) bank19612 +19613 POINT(40.02402906175581 74.78040673783373) bank19613 +19614 POINT(39.7667743516432 73.28347554009244) bank19614 +19615 POINT(41.605975581772114 73.36274034117511) bank19615 +19616 POINT(39.76651111330481 74.71112983520861) bank19616 +19617 POINT(41.559205538926946 73.13816048869371) bank19617 +19618 POINT(40.22027134599084 74.64751969268539) bank19618 +19619 POINT(41.562070574576076 74.63617773089777) bank19619 +19620 POINT(40.522891673219554 74.1923497382491) bank19620 +19621 POINT(40.70047248663692 73.6561019615797) bank19621 +19622 POINT(40.95689321896265 73.59504698626664) bank19622 +19623 POINT(40.90667426119153 73.59143960348008) bank19623 +19624 POINT(41.70037384155448 73.66729433969668) bank19624 +19625 POINT(40.11542819455274 74.5910089946801) bank19625 +19626 POINT(40.240233763261294 73.42908534867203) bank19626 +19627 POINT(41.23296404223496 74.76324749660361) bank19627 +19628 POINT(41.13659610021284 73.95506318545961) bank19628 +19629 POINT(41.40942737540775 74.2563278703207) bank19629 +19630 POINT(39.91290375042827 73.40827901547635) bank19630 +19631 POINT(40.130275833935016 74.82267190192097) bank19631 +19632 POINT(41.18583078881224 73.31259929182646) bank19632 +19633 POINT(41.169894543001284 74.04198216556368) bank19633 +19634 POINT(41.115971571140804 74.32900645672711) bank19634 +19635 POINT(41.63807985735246 74.86935424428597) bank19635 +19636 POINT(40.383468821764964 74.75455430868774) bank19636 +19637 POINT(41.12053117211958 73.17109919736119) bank19637 +19638 POINT(39.98954349595887 74.15408390087644) bank19638 +19639 POINT(41.50971049418775 73.72890485842407) bank19639 +19640 POINT(40.55779558119105 74.14515674777142) bank19640 +19641 POINT(40.832459329246205 74.97523452289467) bank19641 +19642 POINT(40.051286692301936 73.26206938168521) bank19642 +19643 POINT(39.860549154563415 73.99412840154105) bank19643 +19644 POINT(41.305621267989686 73.67938102303589) bank19644 +19645 POINT(40.02029722278743 74.12885808531435) bank19645 +19646 POINT(41.493067177171696 73.41473228861345) bank19646 +19647 POINT(40.06400925772304 73.17496439698448) bank19647 +19648 POINT(40.28381857385499 73.93971347855276) bank19648 +19649 POINT(40.98396909484347 73.93129220925685) bank19649 +19650 POINT(40.445521691442764 73.25757352381233) bank19650 +19651 POINT(40.04331438591539 73.78080608063514) bank19651 +19652 POINT(41.032973224597065 74.41006385901719) bank19652 +19653 POINT(41.66020865113586 73.25608578804339) bank19653 +19654 POINT(40.65409828767694 73.04882841469978) bank19654 +19655 POINT(40.57038854343048 73.81025996301224) bank19655 +19656 POINT(41.427505910155254 74.7573648854925) bank19656 +19657 POINT(40.93028724762115 73.31936083558256) bank19657 +19658 POINT(41.59154648014981 74.66811174383444) bank19658 +19659 POINT(39.882212832831094 74.72000450590504) bank19659 +19660 POINT(41.05962140406986 74.80665075127035) bank19660 +19661 POINT(40.936526085416524 73.4210113116803) bank19661 +19662 POINT(40.77256749314324 73.16803806227882) bank19662 +19663 POINT(40.24040720006127 73.92400691840437) bank19663 +19664 POINT(41.46278395309816 73.21448957139027) bank19664 +19665 POINT(40.000181160144336 74.34607569961665) bank19665 +19666 POINT(41.23871403944612 74.7352397912971) bank19666 +19667 POINT(41.2544385263602 73.47176900426251) bank19667 +19668 POINT(41.33766907057793 73.4602144745298) bank19668 +19669 POINT(40.75062698303518 74.44460547947646) bank19669 +19670 POINT(40.096661550153996 74.62616966225504) bank19670 +19671 POINT(40.17238362852212 74.26068712661123) bank19671 +19672 POINT(39.9463654887721 73.99096117367992) bank19672 +19673 POINT(39.98839879694894 73.40057636106734) bank19673 +19674 POINT(40.92418898749632 73.7513348947017) bank19674 +19675 POINT(41.51763424076874 73.39774029118139) bank19675 +19676 POINT(40.350649438553006 73.86847986755782) bank19676 +19677 POINT(40.03158469638494 74.8860067526381) bank19677 +19678 POINT(40.65921309379132 74.12419434541665) bank19678 +19679 POINT(41.061966527930814 74.53324053234789) bank19679 +19680 POINT(40.436048591741326 74.86577468349604) bank19680 +19681 POINT(40.90190322751809 73.41416832707861) bank19681 +19682 POINT(41.31061104211149 74.74061475408844) bank19682 +19683 POINT(41.27802711984116 73.69963603722307) bank19683 +19684 POINT(40.33302656894743 74.81156801157043) bank19684 +19685 POINT(41.632510334746314 74.9046359583223) bank19685 +19686 POINT(40.95062774139597 73.03373942633989) bank19686 +19687 POINT(40.974562182221774 73.47926584485316) bank19687 +19688 POINT(39.7571865501492 74.42328513517488) bank19688 +19689 POINT(39.76471832561371 74.00791782069743) bank19689 +19690 POINT(39.94985930588497 74.12301133983789) bank19690 +19691 POINT(39.998337695169695 74.61878410967034) bank19691 +19692 POINT(40.79311850854407 74.87771956939761) bank19692 +19693 POINT(41.69844825720401 73.51676432385304) bank19693 +19694 POINT(41.04173528450819 74.9250141281036) bank19694 +19695 POINT(40.203949644383876 74.25449311439618) bank19695 +19696 POINT(40.33781888724711 73.3619047776021) bank19696 +19697 POINT(41.64474373098471 73.8901517550378) bank19697 +19698 POINT(40.815289906092815 73.00646888090529) bank19698 +19699 POINT(40.79974952887255 74.3438553710831) bank19699 +19700 POINT(40.61528131414669 74.66719680820556) bank19700 +19701 POINT(40.1795163779109 73.873467231355) bank19701 +19702 POINT(40.39858939826668 73.40915153663913) bank19702 +19703 POINT(40.2275252669121 73.33203915542428) bank19703 +19704 POINT(41.027885245704496 74.34917929597654) bank19704 +19705 POINT(41.4100565052965 74.17905817411506) bank19705 +19706 POINT(41.28982290260788 73.1098891462209) bank19706 +19707 POINT(41.62915796531588 74.19504708542094) bank19707 +19708 POINT(40.26703805781569 73.2147120264704) bank19708 +19709 POINT(40.4494256947205 73.09491012245655) bank19709 +19710 POINT(40.734990957680296 73.20530148834325) bank19710 +19711 POINT(40.73305688431661 74.2180428816194) bank19711 +19712 POINT(39.78966234288974 73.33065213270032) bank19712 +19713 POINT(41.201494646210854 73.83561946729168) bank19713 +19714 POINT(39.861664080087046 74.71867643062924) bank19714 +19715 POINT(40.07786854732574 73.63164493814234) bank19715 +19716 POINT(41.52926145269916 74.2268404111437) bank19716 +19717 POINT(40.27890342411264 73.6123908901961) bank19717 +19718 POINT(41.13610160240753 74.59565812154212) bank19718 +19719 POINT(39.92165001125501 74.39350141989148) bank19719 +19720 POINT(41.395103443698495 73.8029091494331) bank19720 +19721 POINT(40.69887943480892 73.77937612585521) bank19721 +19722 POINT(40.160484391155755 74.34047319863993) bank19722 +19723 POINT(40.474250369220016 74.81969627205282) bank19723 +19724 POINT(41.30921001545924 73.6433449469774) bank19724 +19725 POINT(40.665396030371994 73.34363206999592) bank19725 +19726 POINT(39.83417084101929 73.61583436918227) bank19726 +19727 POINT(39.777085272974894 73.5379903219426) bank19727 +19728 POINT(41.16056106101317 74.38486565148078) bank19728 +19729 POINT(40.58673305002752 74.10453525814077) bank19729 +19730 POINT(40.38078896585542 73.4080240882073) bank19730 +19731 POINT(39.890926874880535 73.03344834987105) bank19731 +19732 POINT(41.68064061699608 74.72140678358689) bank19732 +19733 POINT(40.71608305358407 73.0759100075787) bank19733 +19734 POINT(39.99273172693611 74.02077385353067) bank19734 +19735 POINT(39.7646068067431 73.91322112490538) bank19735 +19736 POINT(40.54215685419217 74.06627793900802) bank19736 +19737 POINT(40.06491569937592 73.50714790188349) bank19737 +19738 POINT(41.29796682236346 74.74922454911743) bank19738 +19739 POINT(41.43272110995817 74.90535864389015) bank19739 +19740 POINT(40.26422172240015 74.44118601081007) bank19740 +19741 POINT(40.26561028044096 74.37673523653034) bank19741 +19742 POINT(40.67770051114563 74.471377638115) bank19742 +19743 POINT(40.492110859074984 74.59970165273967) bank19743 +19744 POINT(40.06885555800263 73.13081199817354) bank19744 +19745 POINT(41.5360198962283 74.46223351182333) bank19745 +19746 POINT(40.567752672671894 74.72014931876326) bank19746 +19747 POINT(41.425085720963644 74.5155404750854) bank19747 +19748 POINT(41.363993195753274 73.56458133130933) bank19748 +19749 POINT(40.81747647487183 73.69645569746207) bank19749 +19750 POINT(40.41526605876747 74.32773573152583) bank19750 +19751 POINT(41.22209710620551 74.14049103066372) bank19751 +19752 POINT(41.49465625090149 74.63867222838897) bank19752 +19753 POINT(41.63042363824538 73.47710905536228) bank19753 +19754 POINT(41.045395235613526 73.80742903306519) bank19754 +19755 POINT(41.470543367877916 73.44692262613931) bank19755 +19756 POINT(41.6048166724371 73.39304886014632) bank19756 +19757 POINT(39.78863981917434 74.07867714925064) bank19757 +19758 POINT(40.288318201881914 73.39941567112817) bank19758 +19759 POINT(41.16469322707554 73.63174279718653) bank19759 +19760 POINT(39.73958340566753 73.0229125696177) bank19760 +19761 POINT(41.01688849581252 73.00801446334202) bank19761 +19762 POINT(40.520885706371985 73.14631785916993) bank19762 +19763 POINT(40.10106752159874 74.21309166979857) bank19763 +19764 POINT(40.83446739261214 73.89476232159711) bank19764 +19765 POINT(39.80468971392657 73.13284373478132) bank19765 +19766 POINT(39.723894788034535 74.8893151668822) bank19766 +19767 POINT(40.96099486973173 74.91345024847449) bank19767 +19768 POINT(39.877132932604795 73.51485643141805) bank19768 +19769 POINT(39.99622768859353 74.92837069666066) bank19769 +19770 POINT(40.016095899036884 73.13218807518483) bank19770 +19771 POINT(39.78601505539069 74.94948523222826) bank19771 +19772 POINT(41.118586358126684 74.27397197891206) bank19772 +19773 POINT(41.060556152662485 73.70781964826102) bank19773 +19774 POINT(41.03289002497468 73.72716227383617) bank19774 +19775 POINT(40.62731111588671 74.39092602461183) bank19775 +19776 POINT(41.68240143882552 73.70906192164357) bank19776 +19777 POINT(39.719333116241714 73.29198110857223) bank19777 +19778 POINT(40.08540842536075 74.44961685369758) bank19778 +19779 POINT(40.44002668453401 74.00218067763923) bank19779 +19780 POINT(39.800656562331845 73.65285037875928) bank19780 +19781 POINT(41.55078220202129 74.82916574442362) bank19781 +19782 POINT(40.896867008227105 73.72087828361992) bank19782 +19783 POINT(40.05791517795894 73.51998345616487) bank19783 +19784 POINT(39.73374349833092 73.47926430049046) bank19784 +19785 POINT(39.77944502009903 73.79851764045789) bank19785 +19786 POINT(40.07452016892401 74.76506549415217) bank19786 +19787 POINT(39.79177940597014 74.2484437821987) bank19787 +19788 POINT(40.34915882039722 74.96999186367856) bank19788 +19789 POINT(40.493765979913775 73.1366803905357) bank19789 +19790 POINT(39.79114975102216 73.34897682258635) bank19790 +19791 POINT(41.59696432801767 74.86861076932473) bank19791 +19792 POINT(41.35934435810006 74.03978310013787) bank19792 +19793 POINT(41.70351792083886 74.58867632433719) bank19793 +19794 POINT(40.869689500463416 74.63171255504048) bank19794 +19795 POINT(40.677545321291745 73.08646162589413) bank19795 +19796 POINT(41.03799779880332 73.29495085502455) bank19796 +19797 POINT(41.326158150257626 73.57036251310457) bank19797 +19798 POINT(41.20788126069999 73.86844267425897) bank19798 +19799 POINT(40.52934883150077 74.06399547004591) bank19799 +19800 POINT(41.11987374605902 73.50595102167141) bank19800 +19801 POINT(41.4982803595673 73.5439850618599) bank19801 +19802 POINT(39.92695995953228 74.04293734435063) bank19802 +19803 POINT(40.40806837448002 74.67622948511507) bank19803 +19804 POINT(40.87006138787434 73.18105965668666) bank19804 +19805 POINT(40.371180639356126 74.20000620945818) bank19805 +19806 POINT(40.65503369321538 73.18836263352655) bank19806 +19807 POINT(40.74163085836359 74.50393189622447) bank19807 +19808 POINT(41.69934267043877 73.42013821238665) bank19808 +19809 POINT(40.29556666557737 73.75682555466126) bank19809 +19810 POINT(40.6413639117644 74.97027487249647) bank19810 +19811 POINT(41.1144298413642 74.9950727944947) bank19811 +19812 POINT(41.220180319892734 74.46644321607208) bank19812 +19813 POINT(40.805643701122484 73.55716251837555) bank19813 +19814 POINT(41.438188346607426 73.69472519553675) bank19814 +19815 POINT(40.93880530498243 73.3368838155436) bank19815 +19816 POINT(40.55169599647667 74.99591569244403) bank19816 +19817 POINT(41.598414213953816 73.34352625726649) bank19817 +19818 POINT(40.31456591348383 74.43517509614524) bank19818 +19819 POINT(41.090748894147715 73.70484321759866) bank19819 +19820 POINT(40.217280756606506 73.66103031681473) bank19820 +19821 POINT(40.94078583983575 73.80963022592711) bank19821 +19822 POINT(41.35819527053856 73.07159061472093) bank19822 +19823 POINT(41.02142799175404 73.38347080787449) bank19823 +19824 POINT(40.95707372349075 74.97096560965878) bank19824 +19825 POINT(40.37405315503167 73.12895727329034) bank19825 +19826 POINT(40.82491128168483 74.69548562574641) bank19826 +19827 POINT(40.52145077638618 73.34029802520219) bank19827 +19828 POINT(40.310598781816076 73.89865571114393) bank19828 +19829 POINT(40.55433864120649 73.98088214750969) bank19829 +19830 POINT(41.11974062597228 73.3831709628928) bank19830 +19831 POINT(41.21264129301264 73.99423151669806) bank19831 +19832 POINT(41.118311038205675 74.8941257751037) bank19832 +19833 POINT(40.5587901530899 74.13795038552456) bank19833 +19834 POINT(40.67767985232855 73.91839414996863) bank19834 +19835 POINT(40.17893910971634 74.07578025793687) bank19835 +19836 POINT(41.44987568165268 74.1033675635311) bank19836 +19837 POINT(40.93489123915901 73.08406175299884) bank19837 +19838 POINT(41.25728344614269 73.24472081658166) bank19838 +19839 POINT(40.29303150617931 73.01991817020462) bank19839 +19840 POINT(41.16181440873647 73.60003113185134) bank19840 +19841 POINT(41.3471876579148 74.60199341929048) bank19841 +19842 POINT(40.3058063207002 74.77547550976811) bank19842 +19843 POINT(40.37610569935894 73.9330988637786) bank19843 +19844 POINT(41.362374955855465 74.39598181032757) bank19844 +19845 POINT(41.41652255351162 74.81706510943198) bank19845 +19846 POINT(41.371842067777 73.84918801162348) bank19846 +19847 POINT(40.23099930024257 74.84639092328808) bank19847 +19848 POINT(41.08768911032105 73.4791167890696) bank19848 +19849 POINT(40.86447681160643 73.26868079757783) bank19849 +19850 POINT(40.58312501635419 73.19226379642845) bank19850 +19851 POINT(39.725924020215224 74.58732249640828) bank19851 +19852 POINT(41.43887947248333 74.40643047798933) bank19852 +19853 POINT(39.86567381450151 73.14102188060319) bank19853 +19854 POINT(40.87308486612462 74.38419595152446) bank19854 +19855 POINT(40.114869166995966 73.6439681520315) bank19855 +19856 POINT(41.36797567613329 74.9635515048694) bank19856 +19857 POINT(40.31290835295279 74.87375468733318) bank19857 +19858 POINT(40.44524036199397 73.46087176885258) bank19858 +19859 POINT(39.85591911894639 73.2433997205775) bank19859 +19860 POINT(41.17601756357595 73.696649891727) bank19860 +19861 POINT(40.69245381426411 73.79844863837907) bank19861 +19862 POINT(40.25615754352903 74.51224649583904) bank19862 +19863 POINT(41.44956797155654 73.99163818496773) bank19863 +19864 POINT(41.03607267070653 74.63478697521988) bank19864 +19865 POINT(40.93594780425517 74.16092289983676) bank19865 +19866 POINT(39.97962314138144 74.76343044393637) bank19866 +19867 POINT(40.76637434236401 74.20727849504748) bank19867 +19868 POINT(39.8524603659672 74.01051041565046) bank19868 +19869 POINT(40.83876436709525 74.49337728909447) bank19869 +19870 POINT(41.245122216654515 74.01947968884429) bank19870 +19871 POINT(41.64111273770873 74.74696646033735) bank19871 +19872 POINT(40.43763337382725 73.952758770758) bank19872 +19873 POINT(41.42368406955726 74.73784147806154) bank19873 +19874 POINT(39.9108654253412 74.78924270311992) bank19874 +19875 POINT(40.922891835743314 73.7186563645588) bank19875 +19876 POINT(39.94716016848988 73.18402232666753) bank19876 +19877 POINT(41.42401083031932 74.7935910432382) bank19877 +19878 POINT(39.86418339145035 73.41741757522064) bank19878 +19879 POINT(41.598252432146076 73.88197010740174) bank19879 +19880 POINT(40.27003470890976 73.92227955220652) bank19880 +19881 POINT(40.928073386524275 73.6710400035467) bank19881 +19882 POINT(40.39983215889594 73.86585945266594) bank19882 +19883 POINT(41.017038628891065 74.00228752643072) bank19883 +19884 POINT(40.84918772130377 74.54824744331307) bank19884 +19885 POINT(40.82647442796812 73.34547414048617) bank19885 +19886 POINT(40.32537040315562 74.66646928437032) bank19886 +19887 POINT(40.970370968716665 73.4890907359743) bank19887 +19888 POINT(40.54294784018585 74.73483855649906) bank19888 +19889 POINT(40.770687971201085 74.36839348466992) bank19889 +19890 POINT(39.976004447798005 74.14492770288808) bank19890 +19891 POINT(40.017741809875524 73.80192862694042) bank19891 +19892 POINT(40.75816099074896 74.80142682329361) bank19892 +19893 POINT(41.055198970902154 73.04802448019957) bank19893 +19894 POINT(39.81073403215873 74.10708754165468) bank19894 +19895 POINT(40.902985649459396 74.84171672315402) bank19895 +19896 POINT(41.157797299627575 74.20949210566403) bank19896 +19897 POINT(41.23064107864287 74.62584061596026) bank19897 +19898 POINT(40.33757511837458 73.30426369845205) bank19898 +19899 POINT(41.29566642413876 73.55178578159365) bank19899 +19900 POINT(41.01993321534614 74.04723390837646) bank19900 +19901 POINT(41.27852119950281 74.91842773529719) bank19901 +19902 POINT(40.815592235842715 73.56557568539137) bank19902 +19903 POINT(40.95958593295026 73.23715299578109) bank19903 +19904 POINT(41.407910401070936 74.36223016007129) bank19904 +19905 POINT(40.716721327327875 74.35944671749341) bank19905 +19906 POINT(40.4636668434809 73.15395814851578) bank19906 +19907 POINT(39.933055599002714 74.28669072568606) bank19907 +19908 POINT(40.746136027530575 73.32167429758643) bank19908 +19909 POINT(41.01768078176219 74.92720045952535) bank19909 +19910 POINT(41.580929944242584 74.8509089610397) bank19910 +19911 POINT(40.872128282873454 74.61546382098474) bank19911 +19912 POINT(40.29304298097305 73.227822980309) bank19912 +19913 POINT(40.61782882071621 73.75921393020684) bank19913 +19914 POINT(40.71273196008439 73.72181484041573) bank19914 +19915 POINT(39.76974694256147 74.79152204406627) bank19915 +19916 POINT(40.117504330559484 74.97858677625011) bank19916 +19917 POINT(41.27279954274421 73.70875182941707) bank19917 +19918 POINT(41.64490567717467 74.94319325921671) bank19918 +19919 POINT(41.585161482391925 74.711538261231) bank19919 +19920 POINT(39.81189131857288 73.17776425435522) bank19920 +19921 POINT(41.04635550150306 73.41518833043004) bank19921 +19922 POINT(41.07480601813509 74.19664383640166) bank19922 +19923 POINT(40.40448476507941 73.94540573701536) bank19923 +19924 POINT(40.398823408669095 74.84785291302154) bank19924 +19925 POINT(40.369572867600844 73.1902305928756) bank19925 +19926 POINT(39.81716174115562 73.02780606025021) bank19926 +19927 POINT(40.37481216204354 74.82430695533574) bank19927 +19928 POINT(40.48577340918788 73.559201104823) bank19928 +19929 POINT(39.80216374971552 74.83721285085781) bank19929 +19930 POINT(41.54750516933753 73.46772389502537) bank19930 +19931 POINT(41.66282864953756 73.38320149201144) bank19931 +19932 POINT(40.23372805380234 73.33516261921535) bank19932 +19933 POINT(41.698967517276934 74.56027370200196) bank19933 +19934 POINT(40.04890994767687 73.51532819467305) bank19934 +19935 POINT(41.68089455046242 74.39088538144836) bank19935 +19936 POINT(40.87287027701717 74.35375126060914) bank19936 +19937 POINT(40.35992138640042 74.14727779191264) bank19937 +19938 POINT(41.62538530518874 73.63785491652465) bank19938 +19939 POINT(40.8699017567964 73.78025196728731) bank19939 +19940 POINT(41.35351296971084 73.68474400167356) bank19940 +19941 POINT(41.358575478592726 73.84210671722454) bank19941 +19942 POINT(41.007257215367524 73.29178894829366) bank19942 +19943 POINT(40.94387579810727 74.02492713912687) bank19943 +19944 POINT(41.51650328980995 73.5003685499594) bank19944 +19945 POINT(39.76201278821409 73.4956009052761) bank19945 +19946 POINT(40.98223233940998 73.41239794710023) bank19946 +19947 POINT(41.68750885115001 73.69561786488536) bank19947 +19948 POINT(41.490288909551786 74.23646594025234) bank19948 +19949 POINT(40.311730746461656 74.22110249793951) bank19949 +19950 POINT(40.90769389057934 74.74197411611438) bank19950 +19951 POINT(40.33349445949159 73.18410487981318) bank19951 +19952 POINT(40.492090668018086 74.2239141160718) bank19952 +19953 POINT(40.7052842526082 74.75968140993614) bank19953 +19954 POINT(40.87006135413851 73.48361961888251) bank19954 +19955 POINT(40.395785514954795 73.65040162294511) bank19955 +19956 POINT(40.90650698256906 74.570494831697) bank19956 +19957 POINT(40.30219439282231 74.7355344017721) bank19957 +19958 POINT(40.297997997700826 74.10675645949664) bank19958 +19959 POINT(41.54039452731112 74.9423775147971) bank19959 +19960 POINT(40.54570585496826 73.62503167955104) bank19960 +19961 POINT(40.04719632955781 74.13128942294298) bank19961 +19962 POINT(40.66764160503632 73.75681001430114) bank19962 +19963 POINT(40.501312906322624 73.70618850976086) bank19963 +19964 POINT(40.05552717195328 73.29365763449839) bank19964 +19965 POINT(41.57679071591627 73.43736926197589) bank19965 +19966 POINT(40.500241650600884 73.75826616314836) bank19966 +19967 POINT(41.23231414405476 74.2040939891) bank19967 +19968 POINT(41.068585086944424 73.73259229559959) bank19968 +19969 POINT(40.07053853425398 73.07693362002578) bank19969 +19970 POINT(41.50110960827326 73.3376072858736) bank19970 +19971 POINT(39.916472937797614 74.10046718778122) bank19971 +19972 POINT(40.59529464765061 74.16292258236311) bank19972 +19973 POINT(39.88298418926544 74.0816569437122) bank19973 +19974 POINT(41.142133290944834 73.98206347666283) bank19974 +19975 POINT(39.94058014383592 73.30219362646902) bank19975 +19976 POINT(40.47161608310165 73.66055360234765) bank19976 +19977 POINT(39.758829765791816 74.4355165716143) bank19977 +19978 POINT(39.996875134040614 74.81080415120506) bank19978 +19979 POINT(41.463838545356424 74.24094229769527) bank19979 +19980 POINT(40.163711213187185 73.49085850078635) bank19980 +19981 POINT(40.44376551900336 74.19490742949435) bank19981 +19982 POINT(41.355328730489504 73.3824575067194) bank19982 +19983 POINT(40.793665638453575 74.00843037330152) bank19983 +19984 POINT(41.62023229220496 74.42228307193096) bank19984 +19985 POINT(40.29120032736559 73.43824112483132) bank19985 +19986 POINT(41.21213945048583 74.98359323469866) bank19986 +19987 POINT(40.60070622478164 74.37104559724861) bank19987 +19988 POINT(40.43281230604313 73.79908493583461) bank19988 +19989 POINT(40.29096335459914 73.3182881247532) bank19989 +19990 POINT(39.98929710373421 74.48323943351313) bank19990 +19991 POINT(39.91102813949811 74.39498181573836) bank19991 +19992 POINT(39.71840073227552 74.5747837070502) bank19992 +19993 POINT(41.20367773976804 74.41906638497494) bank19993 +19994 POINT(40.169451212338195 74.66882127403062) bank19994 +19995 POINT(39.77908504707493 73.05933850585198) bank19995 +19996 POINT(39.93410983922905 74.15389138984078) bank19996 +19997 POINT(40.867373052072196 74.4810801917493) bank19997 +19998 POINT(41.68517941489048 73.33213628527166) bank19998 +19999 POINT(39.83130999322077 74.50682733828981) bank19999 +20000 POINT(41.462158456659246 74.64849683215041) bank20000 +20001 POINT(40.86736071712306 74.61156984322213) bank20001 +20002 POINT(40.772181757438425 73.40729637415747) bank20002 +20003 POINT(40.67606939688059 74.85384630847292) bank20003 +20004 POINT(41.057771162083526 73.06710893487093) bank20004 +20005 POINT(39.82297165698488 73.02050199509135) bank20005 +20006 POINT(41.55840675745596 74.88586821513566) bank20006 +20007 POINT(41.1941109047036 74.86147671212605) bank20007 +20008 POINT(41.17613928883852 74.90910603826825) bank20008 +20009 POINT(41.26255777715499 73.37672125402506) bank20009 +20010 POINT(40.59141346648956 74.88069050565286) bank20010 +20011 POINT(40.461928175312536 74.2083208376452) bank20011 +20012 POINT(40.74927491268099 74.361935084449) bank20012 +20013 POINT(40.35066425883833 73.03221246165333) bank20013 +20014 POINT(39.78519025862914 73.58440305996275) bank20014 +20015 POINT(39.83505209191074 74.59366663552845) bank20015 +20016 POINT(39.850273335371874 74.58583404801307) bank20016 +20017 POINT(41.46283584109466 73.93899581526482) bank20017 +20018 POINT(41.52441379494289 73.95358156790547) bank20018 +20019 POINT(40.97176489593569 74.12733900530756) bank20019 +20020 POINT(40.21213220802343 73.6280451797289) bank20020 +20021 POINT(40.213182900423526 73.73724346607379) bank20021 +20022 POINT(40.07414892114207 74.05889977464224) bank20022 +20023 POINT(41.113287224482704 73.98039513789402) bank20023 +20024 POINT(41.024648346593665 73.76742497195293) bank20024 +20025 POINT(41.16466164492159 74.85183238361361) bank20025 +20026 POINT(39.976344768459306 74.13199748903907) bank20026 +20027 POINT(41.31607829528432 74.63486921984006) bank20027 +20028 POINT(41.159597183149614 73.72563699740009) bank20028 +20029 POINT(41.20875524993003 74.94547170581617) bank20029 +20030 POINT(40.62795250990137 73.30801105627755) bank20030 +20031 POINT(40.30588560935767 74.46272628091624) bank20031 +20032 POINT(39.88877431134029 74.52274386709745) bank20032 +20033 POINT(40.83475401075017 73.10334048366397) bank20033 +20034 POINT(39.9767800760966 74.59912231654275) bank20034 +20035 POINT(40.499521649744636 74.25567925257982) bank20035 +20036 POINT(40.404443363735695 73.80317565552376) bank20036 +20037 POINT(40.2603180189223 74.72657274706505) bank20037 +20038 POINT(40.84336354899024 73.82286910132943) bank20038 +20039 POINT(41.303633148625075 73.08158118895378) bank20039 +20040 POINT(40.4957116753359 74.09563675525992) bank20040 +20041 POINT(40.92471337917951 73.96805848240062) bank20041 +20042 POINT(41.19468262283666 73.42992405196577) bank20042 +20043 POINT(40.73319141350776 73.72599089353088) bank20043 +20044 POINT(41.694914032858186 74.71961933653117) bank20044 +20045 POINT(40.99920530117903 74.4198117730954) bank20045 +20046 POINT(40.06343195396536 74.37182332531032) bank20046 +20047 POINT(41.695063323429714 73.49424755431959) bank20047 +20048 POINT(41.620875214275294 73.9171672756779) bank20048 +20049 POINT(41.47580085067442 73.32650088221395) bank20049 +20050 POINT(40.76897199582743 74.6914526145408) bank20050 +20051 POINT(41.12103955818326 73.85557260292806) bank20051 +20052 POINT(40.469601353817666 74.57958524870484) bank20052 +20053 POINT(41.11348103829657 73.24176914289235) bank20053 +20054 POINT(41.30271202748652 74.36959713492772) bank20054 +20055 POINT(41.63271978899233 74.28863315997445) bank20055 +20056 POINT(41.02012387145288 74.70247817365973) bank20056 +20057 POINT(40.26323667377321 73.82531651977183) bank20057 +20058 POINT(39.71757240267709 73.30226736085791) bank20058 +20059 POINT(41.39220931710214 73.33917305178707) bank20059 +20060 POINT(41.506794019121884 74.46250525328144) bank20060 +20061 POINT(40.777519047311536 74.61308877201215) bank20061 +20062 POINT(41.384569764739155 74.43334542735646) bank20062 +20063 POINT(41.33946731885011 73.51242599984803) bank20063 +20064 POINT(40.19890009739678 73.66862986318301) bank20064 +20065 POINT(40.57943781356715 73.45392992050884) bank20065 +20066 POINT(40.584646153188096 73.57616937059817) bank20066 +20067 POINT(40.83796684913128 73.67783326931585) bank20067 +20068 POINT(39.95564574122623 73.34673079854764) bank20068 +20069 POINT(40.93264068573126 74.52918059359243) bank20069 +20070 POINT(40.75109529657647 74.99156748292367) bank20070 +20071 POINT(40.448564717659714 73.12158218591635) bank20071 +20072 POINT(41.31248789851374 73.67595045072693) bank20072 +20073 POINT(40.55968177130295 73.2347351962644) bank20073 +20074 POINT(40.99214965012035 73.9676573373274) bank20074 +20075 POINT(40.02820580259533 73.30805741479242) bank20075 +20076 POINT(40.51106803944971 74.7670712663416) bank20076 +20077 POINT(40.91856270460256 73.91337173529779) bank20077 +20078 POINT(40.53588603425554 73.32903342614641) bank20078 +20079 POINT(40.420327455980775 74.11207036055562) bank20079 +20080 POINT(41.46501841584474 73.87479237906614) bank20080 +20081 POINT(41.22060178964434 74.45689125237163) bank20081 +20082 POINT(41.025513159591895 73.4731153848456) bank20082 +20083 POINT(40.28631871142865 73.73655529141345) bank20083 +20084 POINT(41.35044202387701 73.17161660261544) bank20084 +20085 POINT(40.404076418990066 73.83409331707757) bank20085 +20086 POINT(40.017422973762734 73.45427678141186) bank20086 +20087 POINT(40.58675658057701 74.62200005701632) bank20087 +20088 POINT(40.81735758509392 73.04296293313935) bank20088 +20089 POINT(41.42684494431698 74.92276889146873) bank20089 +20090 POINT(41.437786098415536 74.40967444734757) bank20090 +20091 POINT(40.75795390159453 73.78627121004894) bank20091 +20092 POINT(40.52895510969684 74.05077305067844) bank20092 +20093 POINT(40.26123734440475 74.47660358982951) bank20093 +20094 POINT(40.05888536256876 74.63007928024858) bank20094 +20095 POINT(39.78417125425411 73.2367002005881) bank20095 +20096 POINT(41.425860880933016 74.99404983232925) bank20096 +20097 POINT(39.965449240893044 74.64949409540067) bank20097 +20098 POINT(40.961015703892095 74.31348780466723) bank20098 +20099 POINT(40.38174238430656 73.57766843964663) bank20099 +20100 POINT(40.72385107425816 74.74260124865769) bank20100 +20101 POINT(41.13088667075585 73.55773736260657) bank20101 +20102 POINT(41.30598686624661 74.16840397778586) bank20102 +20103 POINT(40.840742596407196 73.05989828244124) bank20103 +20104 POINT(40.39783177650549 74.35791539625906) bank20104 +20105 POINT(40.309022768037025 73.09430094084348) bank20105 +20106 POINT(39.875859199818336 74.15225373153731) bank20106 +20107 POINT(41.53157571405046 73.5696757507976) bank20107 +20108 POINT(40.4428094942785 74.91029928884848) bank20108 +20109 POINT(41.37838705892179 74.56449429381281) bank20109 +20110 POINT(40.62955153600957 74.31874283758171) bank20110 +20111 POINT(40.18271057707776 73.90696982178466) bank20111 +20112 POINT(40.57824935492234 73.42733782748172) bank20112 +20113 POINT(39.82135318755031 73.92386057667416) bank20113 +20114 POINT(41.025147566582724 73.3525417786706) bank20114 +20115 POINT(41.3026761421419 74.82575607437691) bank20115 +20116 POINT(41.37475196165725 73.34997127996256) bank20116 +20117 POINT(40.09524076653856 73.33875508383372) bank20117 +20118 POINT(41.55950670181761 74.85916558928793) bank20118 +20119 POINT(40.291555611489585 73.75886647945649) bank20119 +20120 POINT(40.238385381506674 74.50614579535329) bank20120 +20121 POINT(41.18815008573678 73.8649106704011) bank20121 +20122 POINT(40.3399830939528 73.08496236034303) bank20122 +20123 POINT(41.03241952478037 73.77619527141206) bank20123 +20124 POINT(39.986088595850745 73.80574993524382) bank20124 +20125 POINT(40.7215700184868 74.07895557421655) bank20125 +20126 POINT(40.662253795698646 74.8591598828684) bank20126 +20127 POINT(41.40802243498193 73.51408546733339) bank20127 +20128 POINT(41.655024969366394 73.5801807417249) bank20128 +20129 POINT(41.203330868401586 73.4777113612748) bank20129 +20130 POINT(40.42598746951665 73.47224545513966) bank20130 +20131 POINT(41.407662065753826 74.25842782944558) bank20131 +20132 POINT(41.06256669881701 74.77001487746247) bank20132 +20133 POINT(39.73963029638941 73.10687002267811) bank20133 +20134 POINT(40.72848137388752 73.88649583250346) bank20134 +20135 POINT(39.99063248173337 73.89076258472876) bank20135 +20136 POINT(40.34416862638915 74.42045977108819) bank20136 +20137 POINT(41.549403132862224 74.29802368708394) bank20137 +20138 POINT(40.11567823212614 74.48593193274009) bank20138 +20139 POINT(40.6844375205667 73.01749717824242) bank20139 +20140 POINT(40.10319233422719 73.51948580149774) bank20140 +20141 POINT(39.813143130705875 74.31611223547289) bank20141 +20142 POINT(41.11202225210825 73.83449541903823) bank20142 +20143 POINT(40.23280935311512 73.35003557082304) bank20143 +20144 POINT(39.774966968844 74.37077868797077) bank20144 +20145 POINT(40.11459739386779 73.77421410530937) bank20145 +20146 POINT(40.16640660459483 73.9545877684523) bank20146 +20147 POINT(40.89640713788618 74.39703344774038) bank20147 +20148 POINT(40.683899431683194 73.79504093064902) bank20148 +20149 POINT(40.2685299078347 74.26548464681731) bank20149 +20150 POINT(40.979810249471974 73.44389868181557) bank20150 +20151 POINT(40.2551435497483 73.33296951558371) bank20151 +20152 POINT(40.84654434025542 73.97883837233479) bank20152 +20153 POINT(41.52033140789326 73.61642947602331) bank20153 +20154 POINT(41.55301461089579 73.4418324825028) bank20154 +20155 POINT(39.9115878312667 74.19766894716768) bank20155 +20156 POINT(39.72230801492155 74.95837142840489) bank20156 +20157 POINT(41.18324937377204 73.17126211279066) bank20157 +20158 POINT(40.943315869641054 73.79813254236012) bank20158 +20159 POINT(39.97683836834949 74.44535028562699) bank20159 +20160 POINT(40.60006014383877 73.82213006854415) bank20160 +20161 POINT(41.57815623953765 74.22458618843805) bank20161 +20162 POINT(40.30357931487221 74.58790206812864) bank20162 +20163 POINT(40.98988949827182 74.01156071559564) bank20163 +20164 POINT(40.751952267205944 73.23601908507929) bank20164 +20165 POINT(41.43447954884612 74.33320788010792) bank20165 +20166 POINT(40.75307014623752 73.39156664800554) bank20166 +20167 POINT(39.94565679183298 74.26843883999169) bank20167 +20168 POINT(40.58824902889765 74.89129946763481) bank20168 +20169 POINT(41.56253724032834 74.80360332616442) bank20169 +20170 POINT(40.86914712068392 74.39170072275354) bank20170 +20171 POINT(41.052587892334316 74.11490698826503) bank20171 +20172 POINT(40.84574971767591 73.85632221957827) bank20172 +20173 POINT(40.783911300806224 74.38107340713584) bank20173 +20174 POINT(40.89027288307865 74.31885839108409) bank20174 +20175 POINT(40.37060560979659 74.74485256378567) bank20175 +20176 POINT(40.76130876832855 73.02113695549195) bank20176 +20177 POINT(40.61578355747318 74.10113731250958) bank20177 +20178 POINT(41.41284869535758 74.73654419138272) bank20178 +20179 POINT(41.16058141847162 74.9348515984358) bank20179 +20180 POINT(40.3260036274242 73.96666221357063) bank20180 +20181 POINT(41.501979070397134 73.49516198032612) bank20181 +20182 POINT(40.05156722396651 73.64505834642499) bank20182 +20183 POINT(41.27639703626174 73.17518898410937) bank20183 +20184 POINT(39.95164208299071 74.03698284609251) bank20184 +20185 POINT(40.16309774580532 74.83716181429354) bank20185 +20186 POINT(40.60210437689477 73.27041205557592) bank20186 +20187 POINT(40.10902933599836 74.22933527741537) bank20187 +20188 POINT(39.98835393966856 73.58559073573294) bank20188 +20189 POINT(40.04910171416844 74.08960000097802) bank20189 +20190 POINT(39.74845675595198 74.60884688745278) bank20190 +20191 POINT(40.405493105853715 73.11121578083751) bank20191 +20192 POINT(40.41787982940268 74.51303161921062) bank20192 +20193 POINT(41.043494024225865 74.36308925411768) bank20193 +20194 POINT(40.74650241893725 73.731473237678) bank20194 +20195 POINT(40.06758100350677 73.19236513622363) bank20195 +20196 POINT(41.41225074603799 73.39845225887822) bank20196 +20197 POINT(41.122088175107805 73.59525791573772) bank20197 +20198 POINT(41.16128646209653 74.58142449857422) bank20198 +20199 POINT(41.09455882381088 73.89469393547074) bank20199 +20200 POINT(40.29777797673169 74.51019022684048) bank20200 +20201 POINT(41.52559629805369 74.3678325916319) bank20201 +20202 POINT(39.94442838466515 74.21160604789708) bank20202 +20203 POINT(40.458547949262076 74.18437781403107) bank20203 +20204 POINT(41.556981179622916 74.04185017209306) bank20204 +20205 POINT(40.06162937041119 74.62399370266655) bank20205 +20206 POINT(40.57545504049665 74.77436497682574) bank20206 +20207 POINT(40.01220525739509 73.11985001534802) bank20207 +20208 POINT(40.240690500746744 74.1284124438996) bank20208 +20209 POINT(40.803697748954285 73.56155788801736) bank20209 +20210 POINT(41.58695566351833 73.64255230781417) bank20210 +20211 POINT(40.712104246489524 73.29127801343583) bank20211 +20212 POINT(40.10244559563199 73.8897610206151) bank20212 +20213 POINT(41.584622086485396 73.64286366559999) bank20213 +20214 POINT(40.279613247568015 74.61140708593715) bank20214 +20215 POINT(40.20635509127853 73.1641031178561) bank20215 +20216 POINT(40.18676009348434 74.72176670859348) bank20216 +20217 POINT(40.61271040739087 74.17468405683553) bank20217 +20218 POINT(41.36075282439781 73.75386403169452) bank20218 +20219 POINT(41.15987307368818 74.75550121974369) bank20219 +20220 POINT(40.53563193096263 74.11520645567869) bank20220 +20221 POINT(41.34952808368702 74.91291272791572) bank20221 +20222 POINT(39.853983050424645 73.89238985052818) bank20222 +20223 POINT(40.02831192440304 73.13161317391703) bank20223 +20224 POINT(41.016121964318835 74.55301540294234) bank20224 +20225 POINT(41.48467610065965 73.39997252704838) bank20225 +20226 POINT(41.13709299745397 73.00889279917682) bank20226 +20227 POINT(40.131122429262376 74.28381154752616) bank20227 +20228 POINT(40.27532577980947 73.54856962149681) bank20228 +20229 POINT(40.31444199673768 74.10321687474006) bank20229 +20230 POINT(41.59284523680653 74.9713489799374) bank20230 +20231 POINT(39.921578143314626 74.1283873992924) bank20231 +20232 POINT(41.09575102100051 73.13843162322048) bank20232 +20233 POINT(41.408884405463915 74.90407254415075) bank20233 +20234 POINT(41.32483263951441 74.2175939818409) bank20234 +20235 POINT(41.34238231792592 73.54619052822673) bank20235 +20236 POINT(40.54335949697507 73.84626887648845) bank20236 +20237 POINT(40.023252592381326 74.42883133210535) bank20237 +20238 POINT(40.237911734078175 74.84316013835797) bank20238 +20239 POINT(40.917194061182535 73.39573239321011) bank20239 +20240 POINT(41.28990500358323 74.0749249716769) bank20240 +20241 POINT(40.807719313354156 74.21340402873521) bank20241 +20242 POINT(41.305078411995076 74.25391830192848) bank20242 +20243 POINT(40.63498836452454 73.90028020825743) bank20243 +20244 POINT(41.23070202752004 74.51144756621704) bank20244 +20245 POINT(41.257815728698404 73.32044477800626) bank20245 +20246 POINT(39.850586348802956 74.99048798407229) bank20246 +20247 POINT(39.828993291164686 74.62956138584016) bank20247 +20248 POINT(39.79273392492514 73.19043791158569) bank20248 +20249 POINT(40.21559777340438 73.13998265479673) bank20249 +20250 POINT(41.15098367719619 74.48598560116262) bank20250 +20251 POINT(41.68657421029308 73.73487050003067) bank20251 +20252 POINT(40.27108109074711 74.38810402652061) bank20252 +20253 POINT(40.28661800768796 74.14580554981526) bank20253 +20254 POINT(40.82884217802195 74.30614872988244) bank20254 +20255 POINT(40.237732612356055 73.74230912234833) bank20255 +20256 POINT(41.315883324365316 73.1170188331698) bank20256 +20257 POINT(41.62918720857729 73.6952615713278) bank20257 +20258 POINT(41.528569497698896 73.03791498834507) bank20258 +20259 POINT(40.25754940539203 74.60916595905803) bank20259 +20260 POINT(40.30176588268645 73.22082585901381) bank20260 +20261 POINT(39.92167125766502 73.95935911749318) bank20261 +20262 POINT(40.53138436240648 74.53756703917384) bank20262 +20263 POINT(40.72505488065475 73.40427647063399) bank20263 +20264 POINT(40.92969007500265 74.52504534985451) bank20264 +20265 POINT(39.89323734450345 74.73845248004157) bank20265 +20266 POINT(41.546361258302305 74.24957481143682) bank20266 +20267 POINT(41.24659383041397 74.1474688269848) bank20267 +20268 POINT(41.5758973085244 73.25736954037522) bank20268 +20269 POINT(41.1490417498536 74.29392147290648) bank20269 +20270 POINT(39.764339919784504 74.16803319363343) bank20270 +20271 POINT(41.10687793442455 73.50899294443735) bank20271 +20272 POINT(41.361404966058984 74.23572539569334) bank20272 +20273 POINT(41.41687530401472 73.7272874150465) bank20273 +20274 POINT(41.34165068810143 73.61326573456695) bank20274 +20275 POINT(41.48729166580406 74.08625432372773) bank20275 +20276 POINT(40.56288343702626 73.9772240000086) bank20276 +20277 POINT(41.12111474883012 74.33543916408316) bank20277 +20278 POINT(41.08868463800144 74.52675688427544) bank20278 +20279 POINT(40.140384916597185 73.57777584441118) bank20279 +20280 POINT(39.95704902908174 74.05139237133777) bank20280 +20281 POINT(39.99497850066323 74.7666220701752) bank20281 +20282 POINT(40.71417007546945 74.58528679961847) bank20282 +20283 POINT(39.939557037470635 73.29095654336021) bank20283 +20284 POINT(40.41213645526785 74.57327889795046) bank20284 +20285 POINT(40.41753819818121 73.56897473446044) bank20285 +20286 POINT(41.27560650776027 73.92601305751062) bank20286 +20287 POINT(40.95812522725546 73.10052759021912) bank20287 +20288 POINT(41.59754055470386 73.79880852796731) bank20288 +20289 POINT(40.579279010588635 73.82716174780634) bank20289 +20290 POINT(39.9932075219158 73.43420578404925) bank20290 +20291 POINT(40.78230417134369 74.39435799110262) bank20291 +20292 POINT(39.76091000958374 74.72738813628308) bank20292 +20293 POINT(39.83452366017427 73.18485520948165) bank20293 +20294 POINT(40.68366691673188 74.6429321609313) bank20294 +20295 POINT(41.61623718329546 74.62027222134176) bank20295 +20296 POINT(41.38738805958485 73.83531122388786) bank20296 +20297 POINT(40.07812162845346 74.27352821859955) bank20297 +20298 POINT(40.23441375651477 73.6402770032477) bank20298 +20299 POINT(41.2470780730451 73.79073314775619) bank20299 +20300 POINT(39.96177633189351 74.0893829720597) bank20300 +20301 POINT(40.50733789846685 74.66218033513067) bank20301 +20302 POINT(40.70807212189854 73.6836142497459) bank20302 +20303 POINT(41.26821763202496 73.69879831904615) bank20303 +20304 POINT(40.381335251514535 74.23317539339581) bank20304 +20305 POINT(41.03023865143512 73.02847534148886) bank20305 +20306 POINT(39.913821526270766 73.36324178779186) bank20306 +20307 POINT(39.91566001074839 73.48961333725683) bank20307 +20308 POINT(41.4631957729088 74.64424510299155) bank20308 +20309 POINT(39.95573778707403 74.89565599584031) bank20309 +20310 POINT(41.33272493715674 74.83116774318653) bank20310 +20311 POINT(40.400391590929615 73.76147985973329) bank20311 +20312 POINT(39.78219373234631 73.4730540078664) bank20312 +20313 POINT(39.939146822493655 73.36863710957131) bank20313 +20314 POINT(40.158236287656734 73.4489559829327) bank20314 +20315 POINT(40.29395971553937 74.78850488573327) bank20315 +20316 POINT(39.84496963290832 73.67337000638398) bank20316 +20317 POINT(41.300995873686 73.64799016715496) bank20317 +20318 POINT(40.160735416725316 74.23602359155434) bank20318 +20319 POINT(39.901638074635784 74.8522214671683) bank20319 +20320 POINT(40.683278011784054 73.7229621318454) bank20320 +20321 POINT(40.116711273409 74.61395008731093) bank20321 +20322 POINT(40.20581073720553 73.33624358422549) bank20322 +20323 POINT(41.52833904909389 74.64813922766905) bank20323 +20324 POINT(39.99038295285348 74.24160217428677) bank20324 +20325 POINT(40.28792794898367 74.22809380867935) bank20325 +20326 POINT(40.73509885597599 74.28450009932202) bank20326 +20327 POINT(41.397703810522785 74.3531618782333) bank20327 +20328 POINT(41.69514486679631 73.12393817966634) bank20328 +20329 POINT(40.26873827249184 74.35237216099777) bank20329 +20330 POINT(41.387291323075935 73.95205825723589) bank20330 +20331 POINT(41.33722579871478 74.95880666311044) bank20331 +20332 POINT(40.89161760125112 73.45307393359872) bank20332 +20333 POINT(39.93371282870488 73.45717906757531) bank20333 +20334 POINT(40.02443281052439 73.53502962291711) bank20334 +20335 POINT(40.83653419348374 73.39895871093259) bank20335 +20336 POINT(40.22572915668256 73.15569989100074) bank20336 +20337 POINT(41.647749711321595 74.33690899344337) bank20337 +20338 POINT(40.34952832259821 74.1496178997064) bank20338 +20339 POINT(40.07017077966527 73.38194369531524) bank20339 +20340 POINT(39.887332950785535 73.57638469551732) bank20340 +20341 POINT(41.1875888206799 73.31939086653587) bank20341 +20342 POINT(40.922646615191255 74.06819129480547) bank20342 +20343 POINT(41.272113445850955 73.75293984408809) bank20343 +20344 POINT(41.383208162035494 74.5740342985798) bank20344 +20345 POINT(40.2893739690956 74.66599019191479) bank20345 +20346 POINT(41.448170667113885 74.96401729511261) bank20346 +20347 POINT(40.93860157177565 73.67799833956776) bank20347 +20348 POINT(40.29306973942972 74.04658068244574) bank20348 +20349 POINT(40.873039381012816 74.56227080319003) bank20349 +20350 POINT(41.53958254757456 73.16861786082883) bank20350 +20351 POINT(41.44163591226882 73.30982667997652) bank20351 +20352 POINT(41.319315007424926 74.52507113655022) bank20352 +20353 POINT(39.9881205804563 73.78096303978829) bank20353 +20354 POINT(40.76131710678088 74.86755930553947) bank20354 +20355 POINT(40.73803259670057 73.76981851773172) bank20355 +20356 POINT(40.63763107917306 74.5793735769789) bank20356 +20357 POINT(39.814110017587325 73.64529789781982) bank20357 +20358 POINT(41.35940016568316 73.92570846195575) bank20358 +20359 POINT(40.58103367330694 73.25017967654459) bank20359 +20360 POINT(39.81791942762718 74.68349151811103) bank20360 +20361 POINT(40.745960890908606 74.94257945124772) bank20361 +20362 POINT(41.567769363973035 73.35927734222444) bank20362 +20363 POINT(41.676830142130115 73.18546303709763) bank20363 +20364 POINT(41.42819803194655 74.74315458101282) bank20364 +20365 POINT(40.72410522632963 74.4708487420705) bank20365 +20366 POINT(40.82452029377963 74.8449618196459) bank20366 +20367 POINT(39.9173303257719 73.54111854970091) bank20367 +20368 POINT(40.6311169784405 74.99529403858209) bank20368 +20369 POINT(41.07554966069991 73.80611428385714) bank20369 +20370 POINT(39.74180227963998 74.8290830015434) bank20370 +20371 POINT(41.086888153045365 73.57529420812845) bank20371 +20372 POINT(41.4418890010809 73.60865105846143) bank20372 +20373 POINT(39.92828371511687 73.91589731508766) bank20373 +20374 POINT(41.11410118282692 74.47917872894182) bank20374 +20375 POINT(40.85196617116742 74.28177541611927) bank20375 +20376 POINT(41.44985960791095 73.56300719052211) bank20376 +20377 POINT(40.34607837267969 74.53072576598598) bank20377 +20378 POINT(40.22329640428846 73.38552712612717) bank20378 +20379 POINT(39.791942739424975 73.47470180030236) bank20379 +20380 POINT(41.43116037842353 74.6571346705228) bank20380 +20381 POINT(41.64540213091398 73.32722753958798) bank20381 +20382 POINT(41.52755245873006 74.63277708528763) bank20382 +20383 POINT(40.46822384071821 74.7713832855445) bank20383 +20384 POINT(41.257674663031636 74.37884354760126) bank20384 +20385 POINT(41.193900701248985 74.10965612897101) bank20385 +20386 POINT(40.150869835411065 73.44773444392456) bank20386 +20387 POINT(39.911136934021364 73.28636352384534) bank20387 +20388 POINT(41.41920334109009 73.58800665887358) bank20388 +20389 POINT(40.590091904870036 73.18301407388005) bank20389 +20390 POINT(40.32804509654037 74.9854604575811) bank20390 +20391 POINT(40.110079794950884 73.26127901076642) bank20391 +20392 POINT(41.7102248002742 73.82671528340038) bank20392 +20393 POINT(41.26074289449651 73.49416669841114) bank20393 +20394 POINT(40.76942295522046 73.48978413620095) bank20394 +20395 POINT(41.70936648099397 73.9146705768451) bank20395 +20396 POINT(40.00440502890425 74.27828029019722) bank20396 +20397 POINT(40.80455620313337 73.08952523896052) bank20397 +20398 POINT(41.30169206389994 74.4526100646155) bank20398 +20399 POINT(40.05875078092118 73.04087963524809) bank20399 +20400 POINT(41.083523249429234 73.60311833020211) bank20400 +20401 POINT(40.110086820003254 74.36941014699843) bank20401 +20402 POINT(40.54074395309003 73.89740727936228) bank20402 +20403 POINT(40.73971113846104 74.89753262498652) bank20403 +20404 POINT(39.78849643855524 74.2427416154598) bank20404 +20405 POINT(41.39836265801429 73.00864272420004) bank20405 +20406 POINT(40.46691736999101 73.57430919107236) bank20406 +20407 POINT(40.73752478249663 74.24341302482006) bank20407 +20408 POINT(40.54824359999991 73.37340207200027) bank20408 +20409 POINT(40.9574125830313 74.21340720456001) bank20409 +20410 POINT(41.641978892645916 74.98458650436892) bank20410 +20411 POINT(39.82851954182305 74.77110763754642) bank20411 +20412 POINT(40.60641642047055 73.91195220534736) bank20412 +20413 POINT(40.72720032643129 73.1799596684429) bank20413 +20414 POINT(41.512480034612985 74.40801854966674) bank20414 +20415 POINT(39.82224465668467 74.68046560859804) bank20415 +20416 POINT(39.97843037568905 74.79693828237252) bank20416 +20417 POINT(40.40056856073659 74.22576950964714) bank20417 +20418 POINT(41.646396936292355 74.66046364552535) bank20418 +20419 POINT(41.6419978608001 73.67724116100933) bank20419 +20420 POINT(40.47425477968972 73.2785200143652) bank20420 +20421 POINT(39.86471269802993 73.33355389112565) bank20421 +20422 POINT(41.453855979004054 74.45146466138749) bank20422 +20423 POINT(41.31250940095418 73.04983180699338) bank20423 +20424 POINT(41.05408623286243 74.43912071656442) bank20424 +20425 POINT(40.07579935520063 74.14961696388406) bank20425 +20426 POINT(40.00781257269217 74.11803826532451) bank20426 +20427 POINT(41.41261338903722 74.27333591302317) bank20427 +20428 POINT(40.15943427462361 74.11593612322572) bank20428 +20429 POINT(41.02333718616405 73.67760865437509) bank20429 +20430 POINT(41.60022799149615 74.5156885792665) bank20430 +20431 POINT(40.77645778151462 73.11712837872763) bank20431 +20432 POINT(41.13485916725703 74.21065624741556) bank20432 +20433 POINT(39.92127271444738 73.75742667032739) bank20433 +20434 POINT(41.648965464942236 74.16076620044002) bank20434 +20435 POINT(41.66622155411958 73.58066528605613) bank20435 +20436 POINT(41.112399521734055 73.09390584904006) bank20436 +20437 POINT(41.56382691127333 74.21545312826176) bank20437 +20438 POINT(39.88086829585463 74.99690575619945) bank20438 +20439 POINT(41.25738297761124 74.44612652002081) bank20439 +20440 POINT(41.429424140901624 73.45354551894789) bank20440 +20441 POINT(41.161874457967606 74.29912351401612) bank20441 +20442 POINT(39.88847180518421 74.01212930376865) bank20442 +20443 POINT(41.691583896331814 74.17914623093102) bank20443 +20444 POINT(40.56535952328215 73.70548822668616) bank20444 +20445 POINT(41.268679375029905 73.28533296060247) bank20445 +20446 POINT(40.31268396265235 73.39895475266592) bank20446 +20447 POINT(41.02699966416819 74.7709166751907) bank20447 +20448 POINT(40.613377870707 73.85094967066838) bank20448 +20449 POINT(39.91007115641394 73.50322446327552) bank20449 +20450 POINT(40.78448401471184 74.80552379602234) bank20450 +20451 POINT(40.21084362457876 73.56513818218944) bank20451 +20452 POINT(40.630236303563876 73.92726044624439) bank20452 +20453 POINT(41.477606700127424 74.82584728884646) bank20453 +20454 POINT(40.07467183746454 73.94210887897908) bank20454 +20455 POINT(40.5772551683226 74.72656087792693) bank20455 +20456 POINT(41.6368314266511 73.05497718818803) bank20456 +20457 POINT(39.730174383295434 74.2143840643263) bank20457 +20458 POINT(41.60894454320638 73.24728354687666) bank20458 +20459 POINT(40.57400590084835 74.32819274378937) bank20459 +20460 POINT(40.98056929115741 73.08808232073586) bank20460 +20461 POINT(39.94955398676359 73.08878247285942) bank20461 +20462 POINT(40.87123711515747 74.317585120462) bank20462 +20463 POINT(40.23542992826365 73.14874841344266) bank20463 +20464 POINT(41.51469372144018 73.31941035508461) bank20464 +20465 POINT(40.22258630579827 75.00063209167617) bank20465 +20466 POINT(41.14951089424902 74.41297953216566) bank20466 +20467 POINT(40.51311648667627 73.01537829383427) bank20467 +20468 POINT(40.751345255128555 73.96100443178868) bank20468 +20469 POINT(40.45163423286226 74.38614279869688) bank20469 +20470 POINT(41.36632630097982 73.05472019930987) bank20470 +20471 POINT(41.236650530213296 74.09416479263045) bank20471 +20472 POINT(41.221773886208915 73.29423825071963) bank20472 +20473 POINT(40.23525176084952 73.09755835723348) bank20473 +20474 POINT(41.10699454645419 73.59470176220783) bank20474 +20475 POINT(40.06630014241852 74.3141610337361) bank20475 +20476 POINT(41.64857598601303 73.83448658582213) bank20476 +20477 POINT(40.77174384556127 74.5028755738986) bank20477 +20478 POINT(41.56070949324354 73.14632954520377) bank20478 +20479 POINT(40.15602955235999 73.6170947912749) bank20479 +20480 POINT(40.51785349038346 74.50840745857526) bank20480 +20481 POINT(41.59381218229431 73.60709666154524) bank20481 +20482 POINT(40.06428543046494 73.85736913406025) bank20482 +20483 POINT(39.72182686919581 73.61189717915613) bank20483 +20484 POINT(40.41896075775812 74.69462435570856) bank20484 +20485 POINT(40.275037567695044 73.90766327336138) bank20485 +20486 POINT(39.89169432600461 74.31296811662702) bank20486 +20487 POINT(39.88445284408971 74.60381579170023) bank20487 +20488 POINT(41.4608351542317 73.6129930903815) bank20488 +20489 POINT(40.3142619677621 73.80209983730875) bank20489 +20490 POINT(40.511904334204914 74.19109043973224) bank20490 +20491 POINT(40.106068845765925 73.43623312716201) bank20491 +20492 POINT(41.49408975656536 74.09268554646364) bank20492 +20493 POINT(40.602342041197794 74.40627100829724) bank20493 +20494 POINT(40.35423139371497 74.951775506411) bank20494 +20495 POINT(40.478901225534756 74.90405198116483) bank20495 +20496 POINT(41.296788628537655 74.38338970209566) bank20496 +20497 POINT(41.69216739058465 73.58579517109074) bank20497 +20498 POINT(41.319508115474555 73.43367036879138) bank20498 +20499 POINT(40.81347317161682 73.40718954069317) bank20499 +20500 POINT(41.10006005024827 74.26589544409447) bank20500 +20501 POINT(39.902809388088265 73.42570247142083) bank20501 +20502 POINT(41.06801995857049 73.21078967041868) bank20502 +20503 POINT(39.914944577711125 73.23867462032909) bank20503 +20504 POINT(41.00296642061511 74.68103185990601) bank20504 +20505 POINT(40.353361499881075 73.24466068085867) bank20505 +20506 POINT(40.48514497152026 73.22675798846856) bank20506 +20507 POINT(41.61334961081715 74.12622670731184) bank20507 +20508 POINT(39.79272726545366 74.29830657945448) bank20508 +20509 POINT(39.80445137817961 73.41046555374767) bank20509 +20510 POINT(40.23384342876308 73.85311796195604) bank20510 +20511 POINT(40.79698712716224 74.6716008928453) bank20511 +20512 POINT(39.99382594988415 74.7962607091887) bank20512 +20513 POINT(40.29948849789659 73.22077049053924) bank20513 +20514 POINT(41.00273943419668 74.68267751134647) bank20514 +20515 POINT(40.87739603051958 73.8924020164493) bank20515 +20516 POINT(40.33340142160017 73.76286525555052) bank20516 +20517 POINT(40.77941164749116 74.84200053864747) bank20517 +20518 POINT(40.160996852744645 74.61193339960491) bank20518 +20519 POINT(40.53652803092819 73.27065743436684) bank20519 +20520 POINT(40.48141753595396 73.14411883984454) bank20520 +20521 POINT(40.874441289864194 74.99605518817879) bank20521 +20522 POINT(39.98869972616826 74.06105148306253) bank20522 +20523 POINT(40.567814536364814 73.1393941712613) bank20523 +20524 POINT(39.880887732563686 74.33184298128151) bank20524 +20525 POINT(40.85642876071581 74.42721580150086) bank20525 +20526 POINT(40.41210394599283 74.78343280692089) bank20526 +20527 POINT(41.42268513568645 74.93501858636446) bank20527 +20528 POINT(39.969183010748544 73.76801242247211) bank20528 +20529 POINT(40.581988905889396 73.22236195823535) bank20529 +20530 POINT(39.9841346241428 73.93889839604657) bank20530 +20531 POINT(40.659192113372114 73.01242634855625) bank20531 +20532 POINT(40.21042730389316 74.44192743931345) bank20532 +20533 POINT(41.47326397246406 74.79017930586433) bank20533 +20534 POINT(40.0251446032433 74.94609280946655) bank20534 +20535 POINT(40.61383462975913 74.75977302729129) bank20535 +20536 POINT(41.5164754084796 74.59971687050157) bank20536 +20537 POINT(40.28101836518528 73.63145530613943) bank20537 +20538 POINT(41.20720800447769 73.25436455128536) bank20538 +20539 POINT(41.22925965872954 73.52888156603863) bank20539 +20540 POINT(40.647730018570606 74.31401426171682) bank20540 +20541 POINT(41.33206233003078 73.77397642051447) bank20541 +20542 POINT(41.4741987457156 74.47308671189678) bank20542 +20543 POINT(41.23576932690986 74.91660046097095) bank20543 +20544 POINT(40.68669623393979 73.65525647065085) bank20544 +20545 POINT(40.05769173293383 74.92111592817857) bank20545 +20546 POINT(40.339210723423506 74.33317346338974) bank20546 +20547 POINT(41.68901869786186 74.31591975678862) bank20547 +20548 POINT(40.99872147131795 74.7844001360464) bank20548 +20549 POINT(40.25427079768031 73.02460374359511) bank20549 +20550 POINT(40.66364957862369 73.62532906811258) bank20550 +20551 POINT(41.46303616426835 73.62187285128178) bank20551 +20552 POINT(40.212583418902646 74.80171241817997) bank20552 +20553 POINT(40.891549904548114 73.72747483162088) bank20553 +20554 POINT(40.57622072055845 73.36744678670566) bank20554 +20555 POINT(39.88444201410726 74.037051172618) bank20555 +20556 POINT(41.18537133913009 73.12053850678589) bank20556 +20557 POINT(41.53649263230925 74.11646539034061) bank20557 +20558 POINT(40.149756313840484 73.80108818123598) bank20558 +20559 POINT(41.18636127929195 74.78989329378334) bank20559 +20560 POINT(40.26793623463302 74.82310840976216) bank20560 +20561 POINT(41.48052418823587 73.17584512076482) bank20561 +20562 POINT(40.61809203840474 74.57920920484695) bank20562 +20563 POINT(41.63651461715666 73.6254676971825) bank20563 +20564 POINT(41.102850912551894 74.8515720082828) bank20564 +20565 POINT(40.93765902427952 73.66840823069197) bank20565 +20566 POINT(40.31466989575319 74.40167123362197) bank20566 +20567 POINT(39.89176897776537 73.28459247771153) bank20567 +20568 POINT(39.87753519152192 73.04420030468609) bank20568 +20569 POINT(41.529044159576316 74.35895053184198) bank20569 +20570 POINT(39.97098173083208 74.29194714104564) bank20570 +20571 POINT(41.159126993165316 73.72705854026447) bank20571 +20572 POINT(40.738817008779456 73.98508491383065) bank20572 +20573 POINT(40.397997626356776 74.66496877025148) bank20573 +20574 POINT(40.39324072790798 74.60096076984652) bank20574 +20575 POINT(40.5981202081571 74.00904988579587) bank20575 +20576 POINT(40.14280769393755 73.9293225580874) bank20576 +20577 POINT(40.427449141975714 74.17758132112026) bank20577 +20578 POINT(40.33536169019536 74.72267698686085) bank20578 +20579 POINT(40.03868381365184 74.31828868743605) bank20579 +20580 POINT(41.44910654727043 73.181290799055) bank20580 +20581 POINT(40.820125529980245 74.80010451294385) bank20581 +20582 POINT(41.67447094889936 73.51458609884897) bank20582 +20583 POINT(40.17942721297729 74.73254348809382) bank20583 +20584 POINT(40.362070839354836 74.53548926202282) bank20584 +20585 POINT(40.275385121492235 73.70813262640173) bank20585 +20586 POINT(40.16755209396762 73.84478611117576) bank20586 +20587 POINT(39.9846595278588 74.52203999005694) bank20587 +20588 POINT(40.505441885431736 73.61147228062875) bank20588 +20589 POINT(40.471022124951844 73.4933362785172) bank20589 +20590 POINT(39.86389276880564 73.29484373590891) bank20590 +20591 POINT(39.867534788527756 74.96627091837267) bank20591 +20592 POINT(40.54801462649828 73.0939261856883) bank20592 +20593 POINT(41.432670114648815 74.72943243064186) bank20593 +20594 POINT(41.663543181605256 74.54156975705608) bank20594 +20595 POINT(40.776483204413395 74.38996882923207) bank20595 +20596 POINT(40.99020178952652 74.17799935229651) bank20596 +20597 POINT(41.59458420618188 74.70261100928535) bank20597 +20598 POINT(40.323658494553754 74.42683709989595) bank20598 +20599 POINT(41.101939648948516 74.89268786144426) bank20599 +20600 POINT(39.8088226442164 74.43482982149611) bank20600 +20601 POINT(41.41743097726419 73.3857810234505) bank20601 +20602 POINT(41.50744293909791 73.10335875179072) bank20602 +20603 POINT(40.931948545044506 74.79916688969864) bank20603 +20604 POINT(39.97238173071875 74.83851951172112) bank20604 +20605 POINT(40.48009623751453 73.29667148659081) bank20605 +20606 POINT(41.460767425365184 73.25818000360697) bank20606 +20607 POINT(40.9751626051861 74.94120800650985) bank20607 +20608 POINT(41.13495850519545 74.24171856519153) bank20608 +20609 POINT(40.44976854090332 73.51141006661565) bank20609 +20610 POINT(41.68152428290043 73.76503488701819) bank20610 +20611 POINT(41.67315266773123 73.79005615898512) bank20611 +20612 POINT(40.77010039686399 74.76987119456089) bank20612 +20613 POINT(40.44151570381131 73.28044760570828) bank20613 +20614 POINT(40.725770914712825 74.6371084442153) bank20614 +20615 POINT(39.94946817849551 73.39013344347782) bank20615 +20616 POINT(39.72000419304009 73.06868156330928) bank20616 +20617 POINT(41.25169124765897 74.77575896577) bank20617 +20618 POINT(41.30086449100044 73.95452212608382) bank20618 +20619 POINT(41.09756904043065 74.15964819993675) bank20619 +20620 POINT(39.907518872677976 74.97568169630736) bank20620 +20621 POINT(40.74635730487992 73.58998123817103) bank20621 +20622 POINT(40.74660088488092 73.47555681368914) bank20622 +20623 POINT(40.529646855512304 73.7403825107141) bank20623 +20624 POINT(40.97588355557888 74.71725172193602) bank20624 +20625 POINT(41.61561207199882 74.85243574581794) bank20625 +20626 POINT(41.358946977837356 73.68990087011792) bank20626 +20627 POINT(40.249743685177044 73.95277451760371) bank20627 +20628 POINT(41.62664176668131 73.38382397202119) bank20628 +20629 POINT(40.256477552847635 74.44192431267955) bank20629 +20630 POINT(41.253284410646856 74.35478731593847) bank20630 +20631 POINT(41.24606886790457 73.94670175160985) bank20631 +20632 POINT(40.629475175493006 74.87388550745663) bank20632 +20633 POINT(39.827960272495574 73.76557956348016) bank20633 +20634 POINT(40.01140583485764 73.02687106202926) bank20634 +20635 POINT(39.87499924689644 73.8455180517932) bank20635 +20636 POINT(41.2317491567567 74.67119998411142) bank20636 +20637 POINT(40.24404874067409 73.86436228572877) bank20637 +20638 POINT(40.64541529378906 73.72818017313912) bank20638 +20639 POINT(41.21017159473632 74.01579507650948) bank20639 +20640 POINT(40.22769778878469 74.45691089871934) bank20640 +20641 POINT(41.642278560113674 74.31704875484367) bank20641 +20642 POINT(40.248538151064324 74.940299760377) bank20642 +20643 POINT(41.114006893776924 74.38363833708154) bank20643 +20644 POINT(41.026359661657885 74.73285824832672) bank20644 +20645 POINT(40.617108280532506 73.06896241928685) bank20645 +20646 POINT(41.43628524097527 74.35668045566717) bank20646 +20647 POINT(40.006998304703274 73.63761630811894) bank20647 +20648 POINT(40.072085213762456 73.37308833806543) bank20648 +20649 POINT(40.082867103420256 73.73868576822859) bank20649 +20650 POINT(41.37300762422619 74.08455424643319) bank20650 +20651 POINT(41.076224042056744 74.2888173316147) bank20651 +20652 POINT(40.12483163829204 73.46101274007373) bank20652 +20653 POINT(41.255347243167144 74.41926696570458) bank20653 +20654 POINT(41.600815315662956 74.67001037753506) bank20654 +20655 POINT(40.86748156241023 74.78697187321934) bank20655 +20656 POINT(41.374891284932204 73.94676741371455) bank20656 +20657 POINT(40.50617511400436 74.03588370225742) bank20657 +20658 POINT(39.89651763595999 74.87171786317182) bank20658 +20659 POINT(40.57479768766685 74.08288324059525) bank20659 +20660 POINT(41.216420225062876 74.98427964117286) bank20660 +20661 POINT(39.95778739218839 73.8609857472816) bank20661 +20662 POINT(41.5336226196237 73.48443745515765) bank20662 +20663 POINT(41.39126254891084 73.23702474931478) bank20663 +20664 POINT(40.89710253607381 73.61049768904923) bank20664 +20665 POINT(41.153541403503844 74.20488203914833) bank20665 +20666 POINT(41.33154451519707 73.36399579526983) bank20666 +20667 POINT(41.14893533062137 73.77976207688525) bank20667 +20668 POINT(40.09349571128578 74.42955628340093) bank20668 +20669 POINT(40.363006628604296 73.20289893112133) bank20669 +20670 POINT(40.321144620574096 73.15368734960119) bank20670 +20671 POINT(40.25965995479206 74.27775714462271) bank20671 +20672 POINT(41.365199360554534 73.23230874259532) bank20672 +20673 POINT(39.841884662219655 73.73551690191113) bank20673 +20674 POINT(41.00762341666693 73.41298219007683) bank20674 +20675 POINT(41.14320450977393 74.06384306851604) bank20675 +20676 POINT(40.67718618632386 74.10474979207505) bank20676 +20677 POINT(40.67050804088463 74.59447187125227) bank20677 +20678 POINT(41.126412191215444 73.54752830022892) bank20678 +20679 POINT(41.28862337658352 74.0377075005557) bank20679 +20680 POINT(41.01802418962752 74.64036740003357) bank20680 +20681 POINT(40.70303956927223 73.6959339061508) bank20681 +20682 POINT(41.61801068430125 73.66696086103197) bank20682 +20683 POINT(40.394402189881816 73.95964813300854) bank20683 +20684 POINT(40.27422107485487 73.8423151893762) bank20684 +20685 POINT(41.11618803635635 73.48915602879161) bank20685 +20686 POINT(40.911433286372265 74.40244589820817) bank20686 +20687 POINT(40.08562025090521 73.449445552255) bank20687 +20688 POINT(40.974325603484246 73.49655163099688) bank20688 +20689 POINT(41.378049777564826 73.52263378442859) bank20689 +20690 POINT(41.64633512702068 74.24297551483548) bank20690 +20691 POINT(40.870491412954344 74.15589332656053) bank20691 +20692 POINT(40.07771261925134 73.41786276263879) bank20692 +20693 POINT(40.43348864936599 74.41677189215484) bank20693 +20694 POINT(40.801426757942785 74.44229169707496) bank20694 +20695 POINT(40.01319481595083 73.09386610530689) bank20695 +20696 POINT(41.708771535418656 73.10278925217295) bank20696 +20697 POINT(39.71494942443578 74.37480502276526) bank20697 +20698 POINT(41.51736458115455 73.67739020123008) bank20698 +20699 POINT(40.03064093862802 73.78956374297236) bank20699 +20700 POINT(41.241385064082294 74.1735387667355) bank20700 +20701 POINT(40.10550495048612 74.63644648338413) bank20701 +20702 POINT(41.47659466948192 74.21019584859188) bank20702 +20703 POINT(41.04760933300429 74.14682713053932) bank20703 +20704 POINT(40.78479800370612 73.3911906940472) bank20704 +20705 POINT(39.81408195763793 73.69317210808583) bank20705 +20706 POINT(41.3071423923413 73.32810692020686) bank20706 +20707 POINT(40.391482449105126 74.12155880518766) bank20707 +20708 POINT(40.55987615466583 73.33354003840351) bank20708 +20709 POINT(40.68927748539095 74.44155619572308) bank20709 +20710 POINT(40.718373858322764 73.65679144929375) bank20710 +20711 POINT(40.32631895793492 74.69664560601356) bank20711 +20712 POINT(41.18472771074678 73.79577625159786) bank20712 +20713 POINT(40.151910600843735 73.73505501740857) bank20713 +20714 POINT(40.04113786216533 73.49965454070222) bank20714 +20715 POINT(40.056133051960515 74.57175942492535) bank20715 +20716 POINT(39.728911850055475 73.3132768298942) bank20716 +20717 POINT(40.89159093049199 73.9097785852549) bank20717 +20718 POINT(41.20525403438724 74.20539249789161) bank20718 +20719 POINT(40.873671752313186 73.59997604435215) bank20719 +20720 POINT(41.0924929973019 73.70734707634851) bank20720 +20721 POINT(40.58044755631855 74.50754915222369) bank20721 +20722 POINT(41.078538696228065 74.89090859899426) bank20722 +20723 POINT(40.45463941077106 74.9643726531089) bank20723 +20724 POINT(39.86284298765084 73.69566984218676) bank20724 +20725 POINT(40.751488977369576 73.19915580137294) bank20725 +20726 POINT(39.804855254354045 74.48792441236631) bank20726 +20727 POINT(40.560082180095215 74.16418215083141) bank20727 +20728 POINT(40.0803954141291 74.62251355700796) bank20728 +20729 POINT(39.91553667344881 74.97592180266327) bank20729 +20730 POINT(41.429719067382955 74.2178018301142) bank20730 +20731 POINT(40.136729456377466 74.25002553506923) bank20731 +20732 POINT(40.27618059479696 74.95883174278332) bank20732 +20733 POINT(41.102997654958536 74.97988464732398) bank20733 +20734 POINT(41.19651670748225 73.63984997298472) bank20734 +20735 POINT(40.57701720906531 73.60928498234783) bank20735 +20736 POINT(41.1727722445571 74.66938250301936) bank20736 +20737 POINT(40.47348640648684 74.48499166458474) bank20737 +20738 POINT(40.72634440358596 73.60327994412357) bank20738 +20739 POINT(41.29306207298296 73.7148880225465) bank20739 +20740 POINT(40.24812622668092 74.7561769953214) bank20740 +20741 POINT(40.45320762983731 73.81276697618488) bank20741 +20742 POINT(40.97245697592659 73.4050259323992) bank20742 +20743 POINT(40.5866239978661 73.53620970976209) bank20743 +20744 POINT(41.29306494062464 74.69427480788737) bank20744 +20745 POINT(40.78985648023808 73.26873367814218) bank20745 +20746 POINT(39.8623487366078 74.62846979465327) bank20746 +20747 POINT(40.575372924648434 73.9629691935424) bank20747 +20748 POINT(40.019506348310436 73.85780485785749) bank20748 +20749 POINT(40.69109554913612 73.94104535057129) bank20749 +20750 POINT(40.66627816231023 73.67001248839743) bank20750 +20751 POINT(39.97710657084326 73.37236804613626) bank20751 +20752 POINT(41.26519971162908 74.53174949636306) bank20752 +20753 POINT(40.040408198780355 74.14312432808907) bank20753 +20754 POINT(41.090105708227824 73.24300610531746) bank20754 +20755 POINT(41.3542439827942 73.02298949111314) bank20755 +20756 POINT(41.06379103822084 73.75664291393885) bank20756 +20757 POINT(39.7528712340407 73.33237682134514) bank20757 +20758 POINT(41.25007669241495 74.6572165610831) bank20758 +20759 POINT(40.01992507759683 73.07928166403669) bank20759 +20760 POINT(39.95383206245918 73.36916700760996) bank20760 +20761 POINT(41.18544786254975 74.1208082095721) bank20761 +20762 POINT(40.144316497283995 74.65224384082491) bank20762 +20763 POINT(40.43167970960969 73.48566484940388) bank20763 +20764 POINT(41.4402924380104 73.9330436947158) bank20764 +20765 POINT(41.55711251464656 74.4214173578982) bank20765 +20766 POINT(40.71550679138332 74.42230939570977) bank20766 +20767 POINT(41.68235850644621 73.73769532834747) bank20767 +20768 POINT(41.585188888497576 73.09474962678968) bank20768 +20769 POINT(41.69346627224769 74.06919556350444) bank20769 +20770 POINT(40.04507134951446 73.73186048232687) bank20770 +20771 POINT(39.89289695994844 74.9837520917175) bank20771 +20772 POINT(40.709254172686435 74.6850292251223) bank20772 +20773 POINT(40.483776656276035 73.4946382067392) bank20773 +20774 POINT(40.17115722722645 74.07312950191795) bank20774 +20775 POINT(41.41547445078414 74.02423946867127) bank20775 +20776 POINT(40.856277724127935 74.41677877312499) bank20776 +20777 POINT(41.2350027489906 74.32492793048488) bank20777 +20778 POINT(41.71072229899036 73.93801957301619) bank20778 +20779 POINT(40.81027100951643 73.68479978793759) bank20779 +20780 POINT(41.055794881292584 73.70549804953062) bank20780 +20781 POINT(41.69240606444692 73.88782939789073) bank20781 +20782 POINT(41.70632511062812 73.19139790135159) bank20782 +20783 POINT(40.61225411129401 74.9480199910077) bank20783 +20784 POINT(41.00054042817161 74.27235966179174) bank20784 +20785 POINT(41.6341447665922 74.37629073545358) bank20785 +20786 POINT(40.48251792076978 74.57941519588684) bank20786 +20787 POINT(40.39935982481167 73.91268542842901) bank20787 +20788 POINT(41.27042204688691 73.85796122974178) bank20788 +20789 POINT(40.85631830231518 74.74747419561439) bank20789 +20790 POINT(41.324243555485296 73.71345960444266) bank20790 +20791 POINT(41.0825922790173 73.29732018992014) bank20791 +20792 POINT(40.40168057276944 74.83452225902575) bank20792 +20793 POINT(40.360533351396334 73.61333778144032) bank20793 +20794 POINT(41.38862162418669 73.31349005960982) bank20794 +20795 POINT(39.857242459129154 73.77838002469619) bank20795 +20796 POINT(41.05304068717582 74.35312295528837) bank20796 +20797 POINT(40.38150334061028 74.98365416943277) bank20797 +20798 POINT(40.8877004711305 74.14957881443233) bank20798 +20799 POINT(40.61908267719921 73.33133506523276) bank20799 +20800 POINT(40.184882355721186 73.63333942090783) bank20800 +20801 POINT(40.7771498828468 73.56017949291785) bank20801 +20802 POINT(39.92973264964563 73.6243447359453) bank20802 +20803 POINT(41.54802996800323 73.95008411737115) bank20803 +20804 POINT(41.35656222539228 73.81853602815079) bank20804 +20805 POINT(39.742984242976156 73.735207744119) bank20805 +20806 POINT(41.5262638860307 73.62808335317834) bank20806 +20807 POINT(41.25891660450273 74.16242923158242) bank20807 +20808 POINT(40.384935803748796 73.2795942209002) bank20808 +20809 POINT(40.48365114523318 74.8796533636061) bank20809 +20810 POINT(39.82991681682213 74.47828740758393) bank20810 +20811 POINT(40.44436520822469 73.54812845398023) bank20811 +20812 POINT(41.202567332143715 73.96892382151555) bank20812 +20813 POINT(40.663375314185366 73.37338418649993) bank20813 +20814 POINT(41.39524453304812 74.20086025142167) bank20814 +20815 POINT(40.71932124698894 74.98176142081456) bank20815 +20816 POINT(40.911508952330685 74.97454280169535) bank20816 +20817 POINT(41.54377990779036 73.93912255883951) bank20817 +20818 POINT(41.248138317198716 74.68335089610738) bank20818 +20819 POINT(40.27408389007933 74.7706678708593) bank20819 +20820 POINT(41.397306026160564 73.9591952845315) bank20820 +20821 POINT(41.352688087197144 74.69456754356968) bank20821 +20822 POINT(41.2660225187884 73.4359554361106) bank20822 +20823 POINT(41.151162408354146 74.20224673650591) bank20823 +20824 POINT(40.30498979550576 74.79097583672886) bank20824 +20825 POINT(41.19570135206747 74.65545314060743) bank20825 +20826 POINT(40.962052930827866 73.8988758547793) bank20826 +20827 POINT(40.633685475094914 74.0358525087193) bank20827 +20828 POINT(41.030331058418554 74.28857864268544) bank20828 +20829 POINT(41.289812453119374 74.67079240755743) bank20829 +20830 POINT(41.147291289441256 73.11357809283534) bank20830 +20831 POINT(41.02561051525337 73.89698640694374) bank20831 +20832 POINT(41.33326262008887 73.65433482933791) bank20832 +20833 POINT(40.12303109970784 74.77756777265724) bank20833 +20834 POINT(39.95603384762845 73.86435607426489) bank20834 +20835 POINT(40.13532034679614 73.9608883934259) bank20835 +20836 POINT(41.517387508859194 74.9242424327945) bank20836 +20837 POINT(41.50028872982458 73.69144593707934) bank20837 +20838 POINT(39.984563784489154 73.09213614975398) bank20838 +20839 POINT(40.77937037530529 73.08507586756323) bank20839 +20840 POINT(40.76660469190665 73.23480798737864) bank20840 +20841 POINT(40.000521911987 74.97233855819937) bank20841 +20842 POINT(40.04272117814079 74.70428479873378) bank20842 +20843 POINT(40.290173876650236 73.50208422653108) bank20843 +20844 POINT(40.14621710431984 74.91590612806597) bank20844 +20845 POINT(39.7231063238722 73.57261092046406) bank20845 +20846 POINT(40.412350645809376 73.33687473045529) bank20846 +20847 POINT(41.419792078476824 73.41354619175355) bank20847 +20848 POINT(39.717114257409335 74.55192265194506) bank20848 +20849 POINT(40.01944826404375 74.19906084571242) bank20849 +20850 POINT(40.72584388911891 73.48275183037329) bank20850 +20851 POINT(39.90009501221323 73.42316868720853) bank20851 +20852 POINT(41.4790969349693 74.09039476873079) bank20852 +20853 POINT(40.559893013399765 73.99535417516819) bank20853 +20854 POINT(41.380715682388235 73.22424041906194) bank20854 +20855 POINT(41.120925697110835 74.10726791282579) bank20855 +20856 POINT(40.653662743904825 74.60005116235114) bank20856 +20857 POINT(40.314013528300514 74.53816593203904) bank20857 +20858 POINT(40.65916470146333 73.63481216407408) bank20858 +20859 POINT(40.835333279273925 73.33262814468013) bank20859 +20860 POINT(41.23428911476097 73.0783563932716) bank20860 +20861 POINT(41.64774234870569 74.12349541054368) bank20861 +20862 POINT(41.273319028024204 73.05275956837517) bank20862 +20863 POINT(39.881024470847215 73.43516032500862) bank20863 +20864 POINT(41.088070397453635 73.6923183794961) bank20864 +20865 POINT(41.25841960042368 74.05620781652726) bank20865 +20866 POINT(40.433915992231796 74.96038897149869) bank20866 +20867 POINT(39.93997789040079 73.2025677678953) bank20867 +20868 POINT(41.40593153514891 73.95314656296877) bank20868 +20869 POINT(40.71392007515752 73.61922708917288) bank20869 +20870 POINT(39.96890644752423 73.26058506601454) bank20870 +20871 POINT(40.942113428775265 73.94412253783857) bank20871 +20872 POINT(41.49617930060939 73.76053924938212) bank20872 +20873 POINT(40.59906918833126 74.69673947443141) bank20873 +20874 POINT(40.77903315038131 73.23586149791218) bank20874 +20875 POINT(41.67176653880176 73.23032566380911) bank20875 +20876 POINT(41.295475987258605 74.1834952638304) bank20876 +20877 POINT(39.90620798846099 74.32126665803587) bank20877 +20878 POINT(41.60205706852529 73.51691782297343) bank20878 +20879 POINT(40.00264467535084 73.2416133883569) bank20879 +20880 POINT(39.93640666984498 73.23561416016744) bank20880 +20881 POINT(39.99527733708112 74.93808158413718) bank20881 +20882 POINT(40.88142969476807 73.85930362346716) bank20882 +20883 POINT(39.94582644824545 74.77904186758077) bank20883 +20884 POINT(41.62731688866473 73.99035370464638) bank20884 +20885 POINT(41.009673391872106 73.4412918641539) bank20885 +20886 POINT(41.607401412964634 73.39350073904527) bank20886 +20887 POINT(39.76127828915998 73.4076472623349) bank20887 +20888 POINT(41.009354129297776 74.54143130669) bank20888 +20889 POINT(41.276481118643844 73.36445988251495) bank20889 +20890 POINT(41.41514576610339 74.68484727946847) bank20890 +20891 POINT(39.86487205487778 74.74577623783436) bank20891 +20892 POINT(40.12043345506756 74.99486406037524) bank20892 +20893 POINT(41.66804156887266 73.15054561646728) bank20893 +20894 POINT(39.928029512767246 74.46099368139136) bank20894 +20895 POINT(40.24757704247793 74.82474155908363) bank20895 +20896 POINT(40.12473754140312 73.0995424219667) bank20896 +20897 POINT(41.66970379213934 73.3700610612447) bank20897 +20898 POINT(40.66264068998299 74.7309947070823) bank20898 +20899 POINT(39.81062825058593 74.76379561317927) bank20899 +20900 POINT(40.241778410178384 74.84300892824292) bank20900 +20901 POINT(39.96570281743516 74.24986453597998) bank20901 +20902 POINT(41.10877424855698 73.1037910026413) bank20902 +20903 POINT(41.632026398632725 73.47070460315722) bank20903 +20904 POINT(41.24204947333065 73.45412833159877) bank20904 +20905 POINT(41.46814122970031 73.52569498557143) bank20905 +20906 POINT(39.74063318827841 73.64397950159011) bank20906 +20907 POINT(40.77531988949714 73.25845772835856) bank20907 +20908 POINT(40.30461227452223 73.89017155953414) bank20908 +20909 POINT(41.061741533582115 73.19233555825339) bank20909 +20910 POINT(40.88021661441935 74.7314090788133) bank20910 +20911 POINT(40.61364265524545 73.32992259753136) bank20911 +20912 POINT(41.679710822311236 73.78664770718682) bank20912 +20913 POINT(40.665853035863876 74.15548693999872) bank20913 +20914 POINT(40.38963964372563 73.66643755286447) bank20914 +20915 POINT(40.44861868380262 73.09576054582905) bank20915 +20916 POINT(41.17930439153838 74.22764864241036) bank20916 +20917 POINT(40.90065058101225 73.93988853792735) bank20917 +20918 POINT(41.27174607019942 73.39897836460959) bank20918 +20919 POINT(40.59219800528591 74.95163402424042) bank20919 +20920 POINT(41.515130489023576 74.59634061663093) bank20920 +20921 POINT(40.75681408288807 74.01572619034444) bank20921 +20922 POINT(40.75360237561207 73.81820124522729) bank20922 +20923 POINT(41.249678557904794 74.21758315696437) bank20923 +20924 POINT(40.357116168100355 74.6958345645916) bank20924 +20925 POINT(40.29716574401192 74.64794393799372) bank20925 +20926 POINT(41.51999640731636 73.81327090052113) bank20926 +20927 POINT(41.29671125479706 74.75508556198446) bank20927 +20928 POINT(41.67258013575732 73.41803688005078) bank20928 +20929 POINT(41.015002280001866 74.7153192670885) bank20929 +20930 POINT(41.03903074246073 73.31626949634041) bank20930 +20931 POINT(40.14404022874288 74.6188032704828) bank20931 +20932 POINT(39.85780186132476 74.52034016440129) bank20932 +20933 POINT(40.92251957199778 74.81738838576773) bank20933 +20934 POINT(40.73043817656158 73.54393954372358) bank20934 +20935 POINT(39.734936260001696 73.48493270726208) bank20935 +20936 POINT(40.54379378916359 73.92651435332384) bank20936 +20937 POINT(40.20889905490898 73.82369600618765) bank20937 +20938 POINT(40.165394835799745 73.26044482814635) bank20938 +20939 POINT(40.69035527627103 73.2364564484744) bank20939 +20940 POINT(41.37974867285943 74.50220556372153) bank20940 +20941 POINT(40.555102456408235 73.29401093241613) bank20941 +20942 POINT(40.086286088670505 73.63733724223034) bank20942 +20943 POINT(41.070797494603724 73.15360371530431) bank20943 +20944 POINT(40.14554382425212 73.16496574547034) bank20944 +20945 POINT(41.38754615900915 74.30598601059805) bank20945 +20946 POINT(41.54997141311028 74.5936432055624) bank20946 +20947 POINT(40.486994196320225 73.83513896164675) bank20947 +20948 POINT(41.591161560099046 73.3479976254405) bank20948 +20949 POINT(40.02940849366589 74.30897433327779) bank20949 +20950 POINT(40.188999377945386 74.4771325399859) bank20950 +20951 POINT(41.46849616575098 73.33267512042882) bank20951 +20952 POINT(39.7170573181612 74.80521721719184) bank20952 +20953 POINT(40.423905314048895 74.26899616681341) bank20953 +20954 POINT(39.8722472519325 74.13525996801648) bank20954 +20955 POINT(39.7263664648335 73.29486088827043) bank20955 +20956 POINT(40.26917754643955 73.77380587507217) bank20956 +20957 POINT(40.60357443143646 74.02932365310458) bank20957 +20958 POINT(40.49920502963528 73.6612960014178) bank20958 +20959 POINT(41.07807957213916 74.42228274089462) bank20959 +20960 POINT(40.63855519166507 74.24433199279176) bank20960 +20961 POINT(40.18801221425203 73.82302347712317) bank20961 +20962 POINT(40.440084910751985 74.19349569535248) bank20962 +20963 POINT(41.70052910100031 74.82476044605357) bank20963 +20964 POINT(40.8042880455973 73.73860027181593) bank20964 +20965 POINT(40.03164741818802 74.12115983743179) bank20965 +20966 POINT(41.18980901051149 74.05187136124499) bank20966 +20967 POINT(39.80843013183011 73.80873075236055) bank20967 +20968 POINT(40.44088435048366 73.97470748842711) bank20968 +20969 POINT(39.77893674511903 74.50959425460024) bank20969 +20970 POINT(40.42449298901029 74.78323191705998) bank20970 +20971 POINT(41.36480182027513 74.03821684105971) bank20971 +20972 POINT(40.18245564820914 73.75342644016206) bank20972 +20973 POINT(40.60739679440128 74.79866667927408) bank20973 +20974 POINT(40.050275015021924 74.59087135283914) bank20974 +20975 POINT(39.879545397878694 73.17273257413699) bank20975 +20976 POINT(40.378346688708405 73.21411452690144) bank20976 +20977 POINT(40.6685609459258 73.58305574171082) bank20977 +20978 POINT(40.48040801676409 74.0938548914313) bank20978 +20979 POINT(40.32627781433923 73.94407051788414) bank20979 +20980 POINT(39.84855819845067 74.51222290446849) bank20980 +20981 POINT(40.54214809039632 74.40371458139484) bank20981 +20982 POINT(41.435393471959195 74.69272217002884) bank20982 +20983 POINT(41.236754380939004 73.69621597044495) bank20983 +20984 POINT(40.8666912064916 73.09052248396418) bank20984 +20985 POINT(41.14671833852816 73.02036067453024) bank20985 +20986 POINT(40.5827345428866 74.86699897114902) bank20986 +20987 POINT(40.175935131844575 73.56991587724208) bank20987 +20988 POINT(39.75740036032579 73.3365296085403) bank20988 +20989 POINT(40.801455493775855 74.99937728798473) bank20989 +20990 POINT(41.20702482064426 73.49526205667256) bank20990 +20991 POINT(41.126926400726035 74.8219877053547) bank20991 +20992 POINT(39.788740876749 73.7031554937466) bank20992 +20993 POINT(40.21209792096674 74.7653840115403) bank20993 +20994 POINT(41.525559883290036 73.30485957834536) bank20994 +20995 POINT(40.6546051013154 73.06154892723184) bank20995 +20996 POINT(40.174974378466665 73.89930660584895) bank20996 +20997 POINT(40.05360157993676 73.23346853423415) bank20997 +20998 POINT(40.871693222712416 73.78060326001061) bank20998 +20999 POINT(40.23964049551118 74.44391958798845) bank20999 +21000 POINT(41.61948412685719 74.86750695138184) bank21000 +21001 POINT(41.60074993829528 74.89020638185416) bank21001 +21002 POINT(41.651102282111616 73.39748289094842) bank21002 +21003 POINT(41.55183066367025 73.87662961752201) bank21003 +21004 POINT(39.74398695990307 73.89256453221968) bank21004 +21005 POINT(41.302993311086084 74.54427666549998) bank21005 +21006 POINT(40.32337784586494 73.51610549778405) bank21006 +21007 POINT(39.75775237613159 74.38056938427262) bank21007 +21008 POINT(39.725690898883634 73.04798490981132) bank21008 +21009 POINT(41.07499893644586 74.82306412220187) bank21009 +21010 POINT(41.47593232458141 74.17001907875635) bank21010 +21011 POINT(41.54994484238926 73.3204398250246) bank21011 +21012 POINT(41.40152130006278 73.06115735992786) bank21012 +21013 POINT(40.74295848576768 74.81660672847843) bank21013 +21014 POINT(40.78945437087958 73.80306799854023) bank21014 +21015 POINT(41.46133672187474 73.19627435318027) bank21015 +21016 POINT(39.99612819624238 73.46745577232494) bank21016 +21017 POINT(40.5119788616671 73.93143235951689) bank21017 +21018 POINT(40.89990546062135 73.28441825071931) bank21018 +21019 POINT(39.78173794735892 73.4067086128621) bank21019 +21020 POINT(41.67962964669198 73.12312198565127) bank21020 +21021 POINT(41.638709518577244 73.04764640395088) bank21021 +21022 POINT(40.717056638697386 74.06259400893572) bank21022 +21023 POINT(40.65261247113993 73.09882957385379) bank21023 +21024 POINT(39.90345245935818 74.03608280900664) bank21024 +21025 POINT(40.774456177292116 73.16736167052397) bank21025 +21026 POINT(40.28135337453907 74.70955391324244) bank21026 +21027 POINT(41.52045021629949 73.85230031792248) bank21027 +21028 POINT(39.818309884630786 74.77017103907878) bank21028 +21029 POINT(40.87762296528799 73.46811160321255) bank21029 +21030 POINT(41.188282343479074 74.95925975676604) bank21030 +21031 POINT(40.09406086564032 74.6165161484121) bank21031 +21032 POINT(40.09208579242367 74.45617645196796) bank21032 +21033 POINT(41.04228722940383 73.15267694311423) bank21033 +21034 POINT(40.49693505190277 74.4637522798195) bank21034 +21035 POINT(41.209613044279436 73.97690313578717) bank21035 +21036 POINT(41.245869371310704 73.35380828818104) bank21036 +21037 POINT(40.58278769410028 73.63832970938299) bank21037 +21038 POINT(39.84963773785716 73.04540784103288) bank21038 +21039 POINT(41.457441725450124 74.5752045159732) bank21039 +21040 POINT(41.61315667437172 74.6847583489545) bank21040 +21041 POINT(41.38598828801194 73.35809385801126) bank21041 +21042 POINT(41.0830528972983 74.0401034562633) bank21042 +21043 POINT(40.60299032369604 73.75426695336223) bank21043 +21044 POINT(39.71964761157911 74.2661604885518) bank21044 +21045 POINT(39.9938334813107 73.14730280175127) bank21045 +21046 POINT(40.10997673030787 73.12941389638993) bank21046 +21047 POINT(41.31652429962855 73.61873357351305) bank21047 +21048 POINT(41.55502184074737 73.08776803284722) bank21048 +21049 POINT(40.37883309240432 73.7205460150717) bank21049 +21050 POINT(40.77023291798792 74.2842223802588) bank21050 +21051 POINT(41.31144367091659 74.46605871165387) bank21051 +21052 POINT(39.92720903225191 73.63852946034011) bank21052 +21053 POINT(41.35687167746432 74.6818278887614) bank21053 +21054 POINT(40.262766902414455 73.99158988204509) bank21054 +21055 POINT(41.19259029456048 74.42867689949031) bank21055 +21056 POINT(39.77163770630235 73.61015789636826) bank21056 +21057 POINT(40.85830398238765 73.81687983492127) bank21057 +21058 POINT(40.211258698690415 73.90563118835244) bank21058 +21059 POINT(40.71374012693386 74.78061676016048) bank21059 +21060 POINT(40.67883350548797 73.5269100892089) bank21060 +21061 POINT(40.26199658084578 74.79752965359846) bank21061 +21062 POINT(41.34514344918439 74.89417426320364) bank21062 +21063 POINT(39.882428547241545 74.63894896187948) bank21063 +21064 POINT(41.23883910420392 73.49421530785624) bank21064 +21065 POINT(41.48181559220897 73.36455770488435) bank21065 +21066 POINT(40.69672780836113 74.11539469176365) bank21066 +21067 POINT(40.021671656650085 73.43267634264107) bank21067 +21068 POINT(40.166321595166444 73.40823060507398) bank21068 +21069 POINT(41.535751131611455 74.22814713517178) bank21069 +21070 POINT(40.2956237632776 74.44874005608025) bank21070 +21071 POINT(41.65331469136845 74.64477284523751) bank21071 +21072 POINT(40.898942448053376 73.43278768006923) bank21072 +21073 POINT(40.61266953996406 74.30907459984593) bank21073 +21074 POINT(39.79713658594221 74.29118637183802) bank21074 +21075 POINT(40.0411925017911 73.38897262524031) bank21075 +21076 POINT(40.438086091409254 73.7972387820928) bank21076 +21077 POINT(40.7209230642393 73.66742847473891) bank21077 +21078 POINT(40.93349781914832 73.78762093572331) bank21078 +21079 POINT(40.61830111703122 74.14289208453098) bank21079 +21080 POINT(40.68567579126697 74.04981997139492) bank21080 +21081 POINT(41.202292140824404 74.51236336435443) bank21081 +21082 POINT(40.705680714985625 74.4489663362317) bank21082 +21083 POINT(41.322997015678496 74.1542632717751) bank21083 +21084 POINT(39.85201845433828 74.05120446335948) bank21084 +21085 POINT(41.64871359166894 74.49436650380659) bank21085 +21086 POINT(39.791955036476274 74.82323279657312) bank21086 +21087 POINT(40.38347258175664 73.32627973300453) bank21087 +21088 POINT(40.92595042471731 74.81339531083077) bank21088 +21089 POINT(39.91648306401031 73.45226797422194) bank21089 +21090 POINT(40.92230339070015 73.60150269281547) bank21090 +21091 POINT(40.18828829367767 73.46340457739798) bank21091 +21092 POINT(40.712599767941676 74.18406724873574) bank21092 +21093 POINT(40.747885668852554 74.16907262777386) bank21093 +21094 POINT(41.088910683657026 74.73640703341825) bank21094 +21095 POINT(40.72308020067066 73.8236789191158) bank21095 +21096 POINT(40.979390034122225 74.22919150971133) bank21096 +21097 POINT(41.051801157575206 73.16541666808756) bank21097 +21098 POINT(41.573034719406785 74.07404378620534) bank21098 +21099 POINT(41.46991014401454 73.20587769789678) bank21099 +21100 POINT(40.39632677421748 74.4994487472457) bank21100 +21101 POINT(40.5548584108781 74.33069722255863) bank21101 +21102 POINT(40.38254742389427 74.07479721910164) bank21102 +21103 POINT(41.38505120480681 74.81138308248293) bank21103 +21104 POINT(39.89450738496808 73.86800043775094) bank21104 +21105 POINT(40.32777934933732 73.709002121723) bank21105 +21106 POINT(40.59819296023844 73.93817155946233) bank21106 +21107 POINT(40.752042144752636 73.9569502415711) bank21107 +21108 POINT(40.417066281083706 73.39655821999202) bank21108 +21109 POINT(39.71328406250408 73.50223932392188) bank21109 +21110 POINT(40.260328441247104 73.23365756480902) bank21110 +21111 POINT(41.528267474641325 74.74209208468643) bank21111 +21112 POINT(41.124115768164074 74.6438943552217) bank21112 +21113 POINT(41.257537157245935 73.1700723149093) bank21113 +21114 POINT(40.06947643529949 74.02343821053353) bank21114 +21115 POINT(41.4015461263893 73.38694404326951) bank21115 +21116 POINT(41.399369175354046 74.51149490213474) bank21116 +21117 POINT(40.075721278239975 73.65738458154905) bank21117 +21118 POINT(41.24497576630985 73.07762741398724) bank21118 +21119 POINT(41.018809916879896 73.13487731498489) bank21119 +21120 POINT(41.31473854193202 74.21820443943888) bank21120 +21121 POINT(39.72641540811747 74.71381635321039) bank21121 +21122 POINT(41.69700887445577 74.63523943150821) bank21122 +21123 POINT(40.08168629021665 73.16862282134183) bank21123 +21124 POINT(41.10050556723454 73.78548692080034) bank21124 +21125 POINT(40.94132124237494 73.20626320621436) bank21125 +21126 POINT(41.20006787774873 74.16095489496198) bank21126 +21127 POINT(40.00278990824658 74.2616555183849) bank21127 +21128 POINT(40.307963332342176 73.46415597160457) bank21128 +21129 POINT(41.036284006369335 73.89510636442544) bank21129 +21130 POINT(40.82364138446025 73.81688733100918) bank21130 +21131 POINT(41.38995057043554 73.09631942758563) bank21131 +21132 POINT(41.14345836407091 74.34343769989513) bank21132 +21133 POINT(40.7470778903878 73.60470453481796) bank21133 +21134 POINT(41.117082392289085 73.51339393451038) bank21134 +21135 POINT(40.77229212516547 74.3193011299747) bank21135 +21136 POINT(40.562167264248224 74.34653183929944) bank21136 +21137 POINT(39.79521470975378 73.22225196345588) bank21137 +21138 POINT(41.02617487946163 74.1768571154153) bank21138 +21139 POINT(40.087385978446726 74.00100772719574) bank21139 +21140 POINT(41.02444497302331 73.59261082505778) bank21140 +21141 POINT(40.058243004368286 74.79432230075982) bank21141 +21142 POINT(41.333217194342254 73.86112080441829) bank21142 +21143 POINT(40.41255648540063 74.98705985704402) bank21143 +21144 POINT(40.4676056471489 73.21604922494858) bank21144 +21145 POINT(39.80341227949831 73.53772195319631) bank21145 +21146 POINT(39.79797091097554 73.28941788650538) bank21146 +21147 POINT(41.13736175143315 74.26299792160687) bank21147 +21148 POINT(41.32233774344107 74.38368014544447) bank21148 +21149 POINT(40.98790286337366 73.5063574077873) bank21149 +21150 POINT(40.11201171757875 74.63610835710844) bank21150 +21151 POINT(41.40056018425991 74.09867459348301) bank21151 +21152 POINT(40.187171785350216 74.28300016159159) bank21152 +21153 POINT(41.029590800439344 74.82811653933366) bank21153 +21154 POINT(39.98143684256721 74.86334568620477) bank21154 +21155 POINT(41.20696987101705 74.1278150598149) bank21155 +21156 POINT(41.381192692747845 74.60109521346179) bank21156 +21157 POINT(40.75961887256419 73.90348682902986) bank21157 +21158 POINT(41.11372154285581 74.43916593534126) bank21158 +21159 POINT(39.75510124265142 74.13487912775796) bank21159 +21160 POINT(40.31051407153052 73.63817584353417) bank21160 +21161 POINT(40.16232713347701 74.71408893056659) bank21161 +21162 POINT(39.838451062324104 73.43201025009546) bank21162 +21163 POINT(41.36761471158696 74.03534031931777) bank21163 +21164 POINT(40.15939056605247 73.37299252560953) bank21164 +21165 POINT(41.0610894170443 74.31684880391144) bank21165 +21166 POINT(40.11678137582443 73.15676780098909) bank21166 +21167 POINT(40.859751072357795 74.3045676665688) bank21167 +21168 POINT(41.31274532805403 73.45254170134473) bank21168 +21169 POINT(40.954256852824976 74.64231559795802) bank21169 +21170 POINT(41.339176697913395 73.26643523432948) bank21170 +21171 POINT(40.58968546070509 74.05953045638816) bank21171 +21172 POINT(40.76490308177385 74.80723172096084) bank21172 +21173 POINT(40.845022671718304 74.00503036943537) bank21173 +21174 POINT(41.324909096424456 74.08210862471132) bank21174 +21175 POINT(40.66771126289602 74.1142403477362) bank21175 +21176 POINT(40.91951843233111 73.68453750993193) bank21176 +21177 POINT(40.26594134810256 73.57755554703535) bank21177 +21178 POINT(40.20518834268344 74.6888467478118) bank21178 +21179 POINT(41.13547774146401 74.70988909880349) bank21179 +21180 POINT(41.439305880946456 73.17520937919538) bank21180 +21181 POINT(40.88328305377746 74.44399875207847) bank21181 +21182 POINT(41.6057940825323 74.37298748618197) bank21182 +21183 POINT(41.46982872326366 74.23169714072274) bank21183 +21184 POINT(40.95589714555119 74.45596168301621) bank21184 +21185 POINT(41.22863256927674 73.30737442666978) bank21185 +21186 POINT(41.13606587026578 73.07741710364903) bank21186 +21187 POINT(40.068074752481635 73.9514503813971) bank21187 +21188 POINT(41.55152756077508 73.95152333547202) bank21188 +21189 POINT(40.20814053922587 73.18664039260928) bank21189 +21190 POINT(40.660584020781336 74.72009076672225) bank21190 +21191 POINT(39.756815436680895 74.8467996958912) bank21191 +21192 POINT(41.198881735634785 74.56981248177172) bank21192 +21193 POINT(41.1423722044286 73.39348312988893) bank21193 +21194 POINT(40.132892445278216 73.49858888077321) bank21194 +21195 POINT(41.2509274197239 73.51269408214947) bank21195 +21196 POINT(40.693539935755176 73.34494208663702) bank21196 +21197 POINT(40.06029847640557 74.94906329724877) bank21197 +21198 POINT(41.556884103449896 73.3542000650125) bank21198 +21199 POINT(39.9458165897243 73.82814605694774) bank21199 +21200 POINT(40.30734955660042 73.83928877855804) bank21200 +21201 POINT(39.75432501798908 73.76866248355641) bank21201 +21202 POINT(40.87550387233726 73.09768580201413) bank21202 +21203 POINT(41.228664191401805 73.39732393856475) bank21203 +21204 POINT(40.04919287522494 73.61148765511761) bank21204 +21205 POINT(40.62776955507433 74.63401086696666) bank21205 +21206 POINT(41.099310299595594 73.36477039331682) bank21206 +21207 POINT(41.24637683955733 74.07186951599392) bank21207 +21208 POINT(39.745347380071074 73.5111600916876) bank21208 +21209 POINT(39.719088853618906 74.00559338236397) bank21209 +21210 POINT(39.894838785290794 73.39069585868008) bank21210 +21211 POINT(40.56835285612553 74.73229150252274) bank21211 +21212 POINT(40.730054157478236 73.59925511479405) bank21212 +21213 POINT(39.894447216118685 74.6257407166521) bank21213 +21214 POINT(39.77507004857533 73.3718145087229) bank21214 +21215 POINT(41.05741695571061 73.04983388980261) bank21215 +21216 POINT(39.748266369282696 74.14642210843684) bank21216 +21217 POINT(40.82164373644412 73.62206434098344) bank21217 +21218 POINT(41.139623382943036 74.33572452795804) bank21218 +21219 POINT(39.94260087752049 74.10383980207949) bank21219 +21220 POINT(40.86443803548946 73.14740551842173) bank21220 +21221 POINT(41.63786927683266 73.60679246282236) bank21221 +21222 POINT(41.64404125211291 73.64622804018626) bank21222 +21223 POINT(40.02479875801703 73.97067781980577) bank21223 +21224 POINT(41.307346589754 73.78440898188624) bank21224 +21225 POINT(40.44098713250305 74.55490482814038) bank21225 +21226 POINT(40.009895814121386 73.83435653409597) bank21226 +21227 POINT(39.841755104405216 74.81399828993632) bank21227 +21228 POINT(40.57023453923766 74.94034275094634) bank21228 +21229 POINT(40.41007049151063 74.39604895494925) bank21229 +21230 POINT(41.18422934867431 73.12332338195509) bank21230 +21231 POINT(41.28331404870961 74.04133804009018) bank21231 +21232 POINT(39.96196848657415 73.08776115093531) bank21232 +21233 POINT(40.3076165083767 74.12938454867982) bank21233 +21234 POINT(40.162785793170876 73.67549217124275) bank21234 +21235 POINT(39.81193952397607 74.45086717633387) bank21235 +21236 POINT(39.857409679047656 74.22860653240981) bank21236 +21237 POINT(41.38740407046604 73.24559051054024) bank21237 +21238 POINT(40.27855852685797 74.86064500296138) bank21238 +21239 POINT(41.259447973881976 74.78583345003737) bank21239 +21240 POINT(41.590849309155224 74.41456654193324) bank21240 +21241 POINT(40.483958651494255 74.33012061941687) bank21241 +21242 POINT(40.93920438137373 74.79921420515248) bank21242 +21243 POINT(41.12312555245284 73.12777913550373) bank21243 +21244 POINT(41.235702082831594 73.43111498855305) bank21244 +21245 POINT(41.0746166427098 74.04949484996521) bank21245 +21246 POINT(40.6487771702837 73.1113645373286) bank21246 +21247 POINT(41.2617588666926 73.76286574323625) bank21247 +21248 POINT(40.818907977750385 74.93689049629775) bank21248 +21249 POINT(40.787408925074615 73.27938991152004) bank21249 +21250 POINT(40.48007322472032 73.92513350516982) bank21250 +21251 POINT(41.46542275677404 74.547353730915) bank21251 +21252 POINT(40.19980501941019 73.98195402532404) bank21252 +21253 POINT(41.45312375918563 73.10295562212303) bank21253 +21254 POINT(40.42047881072183 73.08649798598594) bank21254 +21255 POINT(41.4058650007463 74.10743316982249) bank21255 +21256 POINT(40.31419394682549 73.59735540861894) bank21256 +21257 POINT(40.245790321613114 73.74214452376938) bank21257 +21258 POINT(40.36893097752817 74.84295868302502) bank21258 +21259 POINT(40.33972144840857 73.35223370193698) bank21259 +21260 POINT(40.291365282688886 74.88243255824483) bank21260 +21261 POINT(40.32357970677052 73.10154938453951) bank21261 +21262 POINT(40.73436967811515 74.1314190678256) bank21262 +21263 POINT(40.06476526064768 74.51726283621127) bank21263 +21264 POINT(40.376798567969 73.74157730735993) bank21264 +21265 POINT(41.454076902447994 74.38070814762614) bank21265 +21266 POINT(40.39926016678051 74.64825736740862) bank21266 +21267 POINT(40.99103920246389 74.45394594629032) bank21267 +21268 POINT(40.40938464959616 74.70088580250568) bank21268 +21269 POINT(39.829790684603374 73.24148882758809) bank21269 +21270 POINT(40.06942129543949 74.78842316698461) bank21270 +21271 POINT(40.17667967760015 74.42911093142445) bank21271 +21272 POINT(39.834897131051 73.56834138033106) bank21272 +21273 POINT(40.92859274623772 73.7345936535983) bank21273 +21274 POINT(40.45100846596562 73.28659998716925) bank21274 +21275 POINT(41.43445566888166 74.10870703593989) bank21275 +21276 POINT(41.07674818620404 73.63924492968876) bank21276 +21277 POINT(39.77716750038985 74.36927831368011) bank21277 +21278 POINT(41.64668189638097 73.58980777962964) bank21278 +21279 POINT(39.9397657089174 74.81146437320287) bank21279 +21280 POINT(40.67023214038833 74.66662229312595) bank21280 +21281 POINT(40.96451124377955 74.72682257859384) bank21281 +21282 POINT(40.75903397493788 74.47381049761326) bank21282 +21283 POINT(40.290294683076965 73.0824698272825) bank21283 +21284 POINT(41.00848210800194 74.60333296712429) bank21284 +21285 POINT(41.71041449534103 74.70920482200204) bank21285 +21286 POINT(40.88206100649644 74.2058979089922) bank21286 +21287 POINT(40.972544683441896 74.23619961186564) bank21287 +21288 POINT(40.50994333125495 74.01553393558402) bank21288 +21289 POINT(41.12598186094754 74.9663843068067) bank21289 +21290 POINT(41.350269545577675 73.62411388161009) bank21290 +21291 POINT(39.932268097067904 73.22520525698566) bank21291 +21292 POINT(40.52691214198783 73.75592319294893) bank21292 +21293 POINT(39.978398321388035 73.13102724082076) bank21293 +21294 POINT(41.17040202813758 74.03321617285502) bank21294 +21295 POINT(40.43131374760677 73.31926134340989) bank21295 +21296 POINT(41.39563456054552 74.02610784478708) bank21296 +21297 POINT(40.52934086076894 74.41491542857139) bank21297 +21298 POINT(40.08686734356534 73.776253637728) bank21298 +21299 POINT(40.20346849054251 73.45482067374219) bank21299 +21300 POINT(39.90504743249973 74.10613609811466) bank21300 +21301 POINT(41.69690833162494 74.96802906383657) bank21301 +21302 POINT(41.40085783725373 74.69798422844616) bank21302 +21303 POINT(40.34108010392948 74.67797465799086) bank21303 +21304 POINT(41.32531166680111 73.43942577752392) bank21304 +21305 POINT(40.9052395913683 74.67178987558539) bank21305 +21306 POINT(41.135156920648356 74.54822072407707) bank21306 +21307 POINT(40.33525541788441 73.76743525453277) bank21307 +21308 POINT(40.51391101844707 73.06232488620788) bank21308 +21309 POINT(40.72352772030403 74.0485307622765) bank21309 +21310 POINT(39.8478523757093 74.90456914176978) bank21310 +21311 POINT(41.174965345882725 74.30334699464593) bank21311 +21312 POINT(41.639431645278954 74.85690657568108) bank21312 +21313 POINT(40.36189102367788 73.13931548941008) bank21313 +21314 POINT(41.70561621121946 74.39281221098219) bank21314 +21315 POINT(41.08747147846887 74.2014894242406) bank21315 +21316 POINT(40.63817498180066 73.85830817259561) bank21316 +21317 POINT(41.361027647335646 73.71414185365956) bank21317 +21318 POINT(41.42615250140981 74.36861438518977) bank21318 +21319 POINT(41.63034372750183 73.93169149320197) bank21319 +21320 POINT(41.57549201383913 74.47192304023883) bank21320 +21321 POINT(39.73361912483711 73.61836947821351) bank21321 +21322 POINT(41.43808184212615 73.26247385639141) bank21322 +21323 POINT(39.772128753460755 74.59159047042958) bank21323 +21324 POINT(40.686803489088085 74.1955270811283) bank21324 +21325 POINT(41.12945542202615 74.2800258123149) bank21325 +21326 POINT(41.66157544512342 73.43199969667523) bank21326 +21327 POINT(39.96985421292281 74.55423331340155) bank21327 +21328 POINT(40.16410784524545 74.46705598527136) bank21328 +21329 POINT(41.54842770700889 74.9539015321795) bank21329 +21330 POINT(40.94082920546159 74.6648103992794) bank21330 +21331 POINT(40.86564039866247 73.63863959688146) bank21331 +21332 POINT(40.95373064589524 74.6392300680786) bank21332 +21333 POINT(40.113901095915644 74.40275111255474) bank21333 +21334 POINT(40.50462434952146 74.12948421651802) bank21334 +21335 POINT(40.771218478317216 74.09507816376701) bank21335 +21336 POINT(40.997854809103735 73.53728838402543) bank21336 +21337 POINT(40.65311571478539 74.4093524051821) bank21337 +21338 POINT(41.46254187093991 74.0106539439073) bank21338 +21339 POINT(41.626808391965035 73.08861759505331) bank21339 +21340 POINT(40.48338415858892 74.02682314073668) bank21340 +21341 POINT(40.55166893403042 73.67699220766973) bank21341 +21342 POINT(41.50869871517203 74.59193197321852) bank21342 +21343 POINT(40.31301136299525 73.20253776850373) bank21343 +21344 POINT(41.37889515435621 73.09084125075604) bank21344 +21345 POINT(41.681449802222176 73.8871615644962) bank21345 +21346 POINT(40.85010394775191 73.28832625009235) bank21346 +21347 POINT(40.42050516815453 73.89800147970942) bank21347 +21348 POINT(40.255024616740485 74.94144368008256) bank21348 +21349 POINT(40.568195942342115 74.89770189011898) bank21349 +21350 POINT(39.97665462868886 74.34910873912884) bank21350 +21351 POINT(40.752456585230725 74.21790723687882) bank21351 +21352 POINT(40.62195125056197 73.25339470468985) bank21352 +21353 POINT(40.55554914964751 73.37339491258196) bank21353 +21354 POINT(39.89249992494168 74.11222754091915) bank21354 +21355 POINT(41.178491332077435 74.52689812878829) bank21355 +21356 POINT(40.08493922002327 74.45726770719094) bank21356 +21357 POINT(40.01796798429216 74.12247345440129) bank21357 +21358 POINT(41.40879253274622 73.58788912626373) bank21358 +21359 POINT(40.973333066994606 73.796815502514) bank21359 +21360 POINT(41.125121954097594 74.60614026527155) bank21360 +21361 POINT(40.29141986245711 73.89476909694524) bank21361 +21362 POINT(41.374741629616686 74.34917322452293) bank21362 +21363 POINT(41.2609905001168 73.31645160881284) bank21363 +21364 POINT(40.14480035358365 73.63278920293203) bank21364 +21365 POINT(39.75973951442206 74.18892389651886) bank21365 +21366 POINT(40.66858670863927 73.91658343893639) bank21366 +21367 POINT(40.07085659068121 74.4539354201482) bank21367 +21368 POINT(40.171068546092435 74.67300465547977) bank21368 +21369 POINT(39.795809736635746 73.66285917069193) bank21369 +21370 POINT(40.882406472422545 73.35016709378368) bank21370 +21371 POINT(40.744932016234934 74.04632294591178) bank21371 +21372 POINT(41.57636012415047 73.56456323115425) bank21372 +21373 POINT(39.90935151157365 74.77156302578376) bank21373 +21374 POINT(41.05069287064569 73.72685864438573) bank21374 +21375 POINT(40.80621667172253 73.15585319017651) bank21375 +21376 POINT(41.13240497198813 73.91640701066366) bank21376 +21377 POINT(40.27185554129085 73.40672571762981) bank21377 +21378 POINT(40.45936794809047 73.36881823418628) bank21378 +21379 POINT(41.209051450265505 73.8268654526985) bank21379 +21380 POINT(41.55442006511211 73.17899499781143) bank21380 +21381 POINT(40.05847717116188 73.99526670225005) bank21381 +21382 POINT(40.25935684913509 74.98747846083948) bank21382 +21383 POINT(40.257664690191724 75.00221911776501) bank21383 +21384 POINT(40.73799994358094 74.11425979155396) bank21384 +21385 POINT(40.41992343286236 73.49962499421007) bank21385 +21386 POINT(41.05813209956406 73.13030130954736) bank21386 +21387 POINT(40.970970765113094 74.8824551211354) bank21387 +21388 POINT(41.17509863766213 74.36124507108977) bank21388 +21389 POINT(39.78220674596632 74.24128114523126) bank21389 +21390 POINT(40.67213478284602 74.23779880055847) bank21390 +21391 POINT(41.62036311039126 73.88346744762187) bank21391 +21392 POINT(40.59888764395205 73.18411825651259) bank21392 +21393 POINT(40.17966487302273 73.20996648422526) bank21393 +21394 POINT(40.228424039599425 74.29862226865173) bank21394 +21395 POINT(40.09458339111459 74.00919791000956) bank21395 +21396 POINT(39.95910798211489 73.88908754344703) bank21396 +21397 POINT(39.83456908623159 73.3234416758765) bank21397 +21398 POINT(40.236101700545326 73.7590375519592) bank21398 +21399 POINT(39.80333267199714 73.43111015642516) bank21399 +21400 POINT(41.05088788567181 73.05839556113645) bank21400 +21401 POINT(41.22405825203838 73.11828200947691) bank21401 +21402 POINT(39.81840022758283 73.07442429719889) bank21402 +21403 POINT(41.3062153996775 74.65279118457777) bank21403 +21404 POINT(41.20178555425959 73.91541334585666) bank21404 +21405 POINT(41.65955190188575 74.12626114360728) bank21405 +21406 POINT(40.68509258712105 74.60659192758229) bank21406 +21407 POINT(40.89436971043682 74.4864948686667) bank21407 +21408 POINT(41.309789264398454 73.11272344915078) bank21408 +21409 POINT(41.68501233359291 74.39523804283884) bank21409 +21410 POINT(41.07532851428771 73.48308412550682) bank21410 +21411 POINT(41.535271607232694 73.02701428503813) bank21411 +21412 POINT(40.1328327312142 73.53471069023158) bank21412 +21413 POINT(41.35228751635189 74.30603956629388) bank21413 +21414 POINT(40.948052440009796 73.57811180573904) bank21414 +21415 POINT(40.667438081334836 74.40962604911057) bank21415 +21416 POINT(41.03913712391755 74.63096139243599) bank21416 +21417 POINT(41.218023367158004 74.69163509140539) bank21417 +21418 POINT(40.965870582736336 73.36935927303519) bank21418 +21419 POINT(41.46323458744649 74.12893523841458) bank21419 +21420 POINT(40.67514842290726 74.43562799479257) bank21420 +21421 POINT(39.74053641012287 74.81697372657499) bank21421 +21422 POINT(40.000788436415576 74.51029843551498) bank21422 +21423 POINT(41.3747697257411 73.22204208584616) bank21423 +21424 POINT(39.75799139408352 74.45338885739795) bank21424 +21425 POINT(40.03264976165828 74.72236638319468) bank21425 +21426 POINT(39.73904260178797 73.79361360072636) bank21426 +21427 POINT(40.8818800026567 73.863357468441) bank21427 +21428 POINT(40.217192196222605 73.85999116331688) bank21428 +21429 POINT(39.92904298795744 74.35697543160977) bank21429 +21430 POINT(40.38024046479408 74.8988464225382) bank21430 +21431 POINT(41.67602641720783 74.53328778647848) bank21431 +21432 POINT(40.03942851239502 73.1955511328477) bank21432 +21433 POINT(40.858150191796355 73.08604454586863) bank21433 +21434 POINT(40.99291084515761 73.98678255732517) bank21434 +21435 POINT(40.42312351384768 73.2095191659708) bank21435 +21436 POINT(40.3218987113861 73.14915122625638) bank21436 +21437 POINT(40.95978518073747 74.01009621645372) bank21437 +21438 POINT(41.40658984261581 73.0820893856057) bank21438 +21439 POINT(41.19786375183314 74.80784657398459) bank21439 +21440 POINT(39.93764006751031 74.55088759527483) bank21440 +21441 POINT(41.295939602992895 74.33035628174234) bank21441 +21442 POINT(40.43034697336258 74.13499633657837) bank21442 +21443 POINT(41.11030603911602 74.26549581971476) bank21443 +21444 POINT(40.44023584764214 73.67943759516297) bank21444 +21445 POINT(40.22442145492143 73.16318372687748) bank21445 +21446 POINT(40.563068508449675 73.06316896290181) bank21446 +21447 POINT(40.72253192681895 74.89667589664725) bank21447 +21448 POINT(40.078608455286165 73.76250391784805) bank21448 +21449 POINT(40.767294584874016 74.14790069299995) bank21449 +21450 POINT(41.60281011780128 73.27229200881233) bank21450 +21451 POINT(40.93259704093041 74.50430588835262) bank21451 +21452 POINT(41.491634206972684 73.64180599186032) bank21452 +21453 POINT(41.406286740419596 73.48040875862004) bank21453 +21454 POINT(41.4867473837599 73.61174074112938) bank21454 +21455 POINT(41.418927849438916 73.69687630451953) bank21455 +21456 POINT(41.00030461811679 73.7836239415068) bank21456 +21457 POINT(41.56771479659888 73.73724952792881) bank21457 +21458 POINT(41.27090948437424 74.49356056848207) bank21458 +21459 POINT(41.361840333000494 74.37029544013389) bank21459 +21460 POINT(40.703245615837055 73.81865153192474) bank21460 +21461 POINT(40.78789166793973 74.83513115794251) bank21461 +21462 POINT(40.88561448275297 73.52270753831901) bank21462 +21463 POINT(39.71867764515242 73.21991145053492) bank21463 +21464 POINT(39.82096016948181 74.93691941203465) bank21464 +21465 POINT(40.561768037114405 73.95865177488056) bank21465 +21466 POINT(40.467088233698696 73.42294920377286) bank21466 +21467 POINT(40.920239313117136 73.65946697282452) bank21467 +21468 POINT(39.83921606090228 74.51399566281106) bank21468 +21469 POINT(40.373912100390136 73.47946666986815) bank21469 +21470 POINT(41.48336902975738 74.402500976703) bank21470 +21471 POINT(41.17870697660983 73.16457912503978) bank21471 +21472 POINT(41.61427205341711 74.6803134035495) bank21472 +21473 POINT(40.2544655144723 74.66930610540733) bank21473 +21474 POINT(40.297178367624404 74.5482133055535) bank21474 +21475 POINT(41.18133668855668 74.01300312616834) bank21475 +21476 POINT(41.061995605199506 73.39072157436499) bank21476 +21477 POINT(40.031002588898474 74.8015949002719) bank21477 +21478 POINT(40.391087841177914 74.50722018397742) bank21478 +21479 POINT(40.57736782378111 73.72243558240442) bank21479 +21480 POINT(41.54686301063031 73.27282615907845) bank21480 +21481 POINT(40.558334321510706 74.6958780978529) bank21481 +21482 POINT(41.28349734800262 73.05022726737883) bank21482 +21483 POINT(41.38852804456076 74.72498318612884) bank21483 +21484 POINT(40.00497704740965 73.32264169607059) bank21484 +21485 POINT(41.66824469578204 74.39140841597118) bank21485 +21486 POINT(40.787115416127634 74.22632526700728) bank21486 +21487 POINT(39.747785190948115 73.80771274537975) bank21487 +21488 POINT(41.46654435260255 74.98999450516216) bank21488 +21489 POINT(39.97207312012158 73.037296165243) bank21489 +21490 POINT(41.194883438450354 73.69016339728869) bank21490 +21491 POINT(40.05828143067773 74.9020159104416) bank21491 +21492 POINT(41.58667967934582 73.68931411040806) bank21492 +21493 POINT(41.44259521336269 74.94152655737079) bank21493 +21494 POINT(39.94860142047506 74.03040932900215) bank21494 +21495 POINT(41.14339698106643 74.54264493812954) bank21495 +21496 POINT(39.74523818428302 74.45457714269247) bank21496 +21497 POINT(40.97756471466249 74.07034268161664) bank21497 +21498 POINT(39.83907614477949 74.0838989608947) bank21498 +21499 POINT(41.43500693766422 73.5663627368784) bank21499 +21500 POINT(39.819199322205705 73.56627429896729) bank21500 +21501 POINT(39.75031494400948 73.86522269008043) bank21501 +21502 POINT(41.65177829624042 73.57957263249037) bank21502 +21503 POINT(41.37967672911279 73.9262733918217) bank21503 +21504 POINT(40.862728479722506 73.6134945519279) bank21504 +21505 POINT(40.837410824516084 74.90322473267122) bank21505 +21506 POINT(40.84824919792765 73.29856175421014) bank21506 +21507 POINT(39.871186696588616 73.48733374726235) bank21507 +21508 POINT(40.12623786042697 73.89277900885313) bank21508 +21509 POINT(41.06955897177532 74.5819354822445) bank21509 +21510 POINT(40.29604336879721 74.87097588867965) bank21510 +21511 POINT(41.24116644073942 73.1786163865305) bank21511 +21512 POINT(40.845442162610574 74.63422927270699) bank21512 +21513 POINT(40.67682403054232 74.37162979991157) bank21513 +21514 POINT(41.17005589741241 74.57269055364732) bank21514 +21515 POINT(39.88811304708727 73.7061475275184) bank21515 +21516 POINT(39.93842520612101 74.77635232791275) bank21516 +21517 POINT(41.32217152895053 73.1490945036731) bank21517 +21518 POINT(40.35420946865044 73.47981602712237) bank21518 +21519 POINT(40.24298989903596 74.10154431094391) bank21519 +21520 POINT(41.3718409709445 74.05762608391746) bank21520 +21521 POINT(40.44872698581938 74.28295943959048) bank21521 +21522 POINT(40.495283027254466 74.14331535795074) bank21522 +21523 POINT(40.66598524976791 73.52017505303682) bank21523 +21524 POINT(39.74025610840187 73.60198101344417) bank21524 +21525 POINT(40.84162485741163 74.24027594150145) bank21525 +21526 POINT(39.93182601198718 74.51609227793658) bank21526 +21527 POINT(40.57589360849962 73.89470213290407) bank21527 +21528 POINT(41.635021921210665 74.1370959119816) bank21528 +21529 POINT(41.4129878069784 74.92788780670055) bank21529 +21530 POINT(40.86804231826905 73.34466268315937) bank21530 +21531 POINT(41.540486496248064 74.67229545103324) bank21531 +21532 POINT(41.42705935406879 74.7112996083062) bank21532 +21533 POINT(40.47491574097329 74.58569565692797) bank21533 +21534 POINT(40.312585905568646 73.11963137044032) bank21534 +21535 POINT(41.07638051620568 73.47393650692756) bank21535 +21536 POINT(41.065442855862656 73.16676060975536) bank21536 +21537 POINT(40.22521886376396 74.61761029945434) bank21537 +21538 POINT(40.42531906450243 73.91632101227574) bank21538 +21539 POINT(39.750858184314865 74.12081778511107) bank21539 +21540 POINT(40.78754894576003 74.7738338840949) bank21540 +21541 POINT(40.20928851706694 73.24471866448258) bank21541 +21542 POINT(40.80491571520063 73.41999315764473) bank21542 +21543 POINT(40.04616926723397 74.3700640594046) bank21543 +21544 POINT(39.85582484653214 74.71316053104118) bank21544 +21545 POINT(40.356212529257455 73.12607537762909) bank21545 +21546 POINT(39.792339164709475 73.67547363393852) bank21546 +21547 POINT(40.421109514846144 74.82574281178387) bank21547 +21548 POINT(40.53652474856954 74.10787560312976) bank21548 +21549 POINT(41.611143017868905 74.64008530840219) bank21549 +21550 POINT(40.36245975772108 73.93000893042667) bank21550 +21551 POINT(39.824163492430976 73.1574838393754) bank21551 +21552 POINT(39.748907058499015 74.95940333749712) bank21552 +21553 POINT(40.69816575142691 74.97517921080262) bank21553 +21554 POINT(40.3929031970864 74.43310841544091) bank21554 +21555 POINT(39.83326478276233 74.93694423799182) bank21555 +21556 POINT(40.21658566132563 73.68342285517222) bank21556 +21557 POINT(39.790362681798214 74.45272072822229) bank21557 +21558 POINT(40.65848062747861 73.10390074049138) bank21558 +21559 POINT(40.17163928761885 74.05917707804673) bank21559 +21560 POINT(41.643684666001526 73.93648810122366) bank21560 +21561 POINT(41.22034455755119 74.77929903964915) bank21561 +21562 POINT(40.58270424476885 73.87098853237471) bank21562 +21563 POINT(41.68010120005916 74.61265944783287) bank21563 +21564 POINT(41.03187357219162 74.70461274448527) bank21564 +21565 POINT(40.618729808887274 74.66876910436932) bank21565 +21566 POINT(40.01697265293573 74.88876640147487) bank21566 +21567 POINT(41.30542531901661 73.46802119672688) bank21567 +21568 POINT(40.687554376956335 73.3795296260265) bank21568 +21569 POINT(41.231237447772486 74.26024629498572) bank21569 +21570 POINT(41.37392641481001 73.88550235578371) bank21570 +21571 POINT(40.42377101031374 73.10591630458396) bank21571 +21572 POINT(40.441834376600646 73.00891254544577) bank21572 +21573 POINT(40.045301170121085 73.28500258081333) bank21573 +21574 POINT(40.53543405102863 73.79657708522006) bank21574 +21575 POINT(41.53800156988928 74.21131857569837) bank21575 +21576 POINT(39.84502254658871 73.20615956320083) bank21576 +21577 POINT(41.298693602652534 73.71858558431673) bank21577 +21578 POINT(40.15771547551167 74.70769376316161) bank21578 +21579 POINT(40.62569520103392 73.01026941309024) bank21579 +21580 POINT(40.0419790896621 74.22677023465204) bank21580 +21581 POINT(41.232559266807755 73.12750810066093) bank21581 +21582 POINT(40.93549490171072 73.17811476610076) bank21582 +21583 POINT(41.326239689658294 74.59771751315081) bank21583 +21584 POINT(40.21509969025806 73.66875549646254) bank21584 +21585 POINT(40.43350123249643 73.33006486645546) bank21585 +21586 POINT(40.831437632273676 74.01432281190228) bank21586 +21587 POINT(41.52452316404597 74.93812556708221) bank21587 +21588 POINT(40.74721556528429 73.81092039245127) bank21588 +21589 POINT(40.69251151594717 74.82636335015198) bank21589 +21590 POINT(41.32449242460914 75.00072018999825) bank21590 +21591 POINT(41.054477608325385 73.94825018300196) bank21591 +21592 POINT(39.96979192794225 74.60748795114384) bank21592 +21593 POINT(41.244244689339816 74.00524003066546) bank21593 +21594 POINT(40.62093620285879 74.92735098869342) bank21594 +21595 POINT(41.51719211480068 73.08498014624337) bank21595 +21596 POINT(41.60750239927199 74.24295878065112) bank21596 +21597 POINT(40.37867781494824 73.7600922638144) bank21597 +21598 POINT(41.28036772027126 73.13925455113201) bank21598 +21599 POINT(39.867834743309665 74.1655946858645) bank21599 +21600 POINT(41.17435148726025 74.40778764628278) bank21600 +21601 POINT(41.61046857227736 74.69320405629084) bank21601 +21602 POINT(39.95095807249738 73.05287894096455) bank21602 +21603 POINT(40.08980589928644 73.89856000454282) bank21603 +21604 POINT(41.19840550670302 73.76463392256545) bank21604 +21605 POINT(40.090726112958926 73.1433633995839) bank21605 +21606 POINT(40.01547721168388 74.48689229344603) bank21606 +21607 POINT(41.014694896642254 74.33846136658781) bank21607 +21608 POINT(41.706281040366456 74.10232459708925) bank21608 +21609 POINT(41.15567181261474 73.39236591198026) bank21609 +21610 POINT(40.14870434466285 74.14927408547815) bank21610 +21611 POINT(41.119959068180606 73.41648877987961) bank21611 +21612 POINT(40.9963013698072 74.20232823831584) bank21612 +21613 POINT(40.86041950622777 74.32209440903448) bank21613 +21614 POINT(40.8637653166674 73.25208154277435) bank21614 +21615 POINT(40.98854851964125 74.38245942706791) bank21615 +21616 POINT(40.75202025785902 73.19011165328675) bank21616 +21617 POINT(41.4325048359441 73.7671247483933) bank21617 +21618 POINT(41.55102888413471 74.27073483324315) bank21618 +21619 POINT(41.393890352601595 74.17539653269363) bank21619 +21620 POINT(41.21192229049565 74.20228220957782) bank21620 +21621 POINT(41.348659164399564 73.94177755309839) bank21621 +21622 POINT(40.45995105264159 73.57808904963136) bank21622 +21623 POINT(40.86780146371164 73.63898998738173) bank21623 +21624 POINT(40.83039615428344 73.02979526707813) bank21624 +21625 POINT(40.97520967421592 73.72329729167976) bank21625 +21626 POINT(40.538360008973804 73.60357697721525) bank21626 +21627 POINT(41.687844595222735 74.44924238286798) bank21627 +21628 POINT(39.80242840301991 73.90719703687566) bank21628 +21629 POINT(41.28783071410481 73.31738629927355) bank21629 +21630 POINT(40.17757125060818 73.48481344181646) bank21630 +21631 POINT(41.14451057351562 73.20769125623897) bank21631 +21632 POINT(40.06312814345216 73.92285749271095) bank21632 +21633 POINT(41.63761485268353 73.45034403497444) bank21633 +21634 POINT(40.02480723927042 74.60612730783879) bank21634 +21635 POINT(40.464309054969476 73.416047806338) bank21635 +21636 POINT(41.003666802076964 73.85722115004) bank21636 +21637 POINT(41.1076393898606 73.80755613760803) bank21637 +21638 POINT(41.310309543748374 74.17204967761361) bank21638 +21639 POINT(41.20552765435391 74.06408842838528) bank21639 +21640 POINT(40.59554051311943 73.78581190866808) bank21640 +21641 POINT(41.55012390234752 74.5036766994529) bank21641 +21642 POINT(40.38118286410337 74.56090206947856) bank21642 +21643 POINT(40.50008850205678 74.34812613467062) bank21643 +21644 POINT(40.165756579772186 74.50418397482892) bank21644 +21645 POINT(41.676805490236 73.93030125215562) bank21645 +21646 POINT(40.08222912086661 74.13306591951064) bank21646 +21647 POINT(40.916543484879305 74.21658746694821) bank21647 +21648 POINT(40.316451224512214 74.5647645382341) bank21648 +21649 POINT(41.17063196927728 73.97321387671437) bank21649 +21650 POINT(39.89137324751938 74.31923602258068) bank21650 +21651 POINT(40.21629772850272 73.16411638572194) bank21651 +21652 POINT(40.76500168618934 74.90859323550411) bank21652 +21653 POINT(41.704571619821564 74.80358070513257) bank21653 +21654 POINT(40.30059726466603 74.42393551795008) bank21654 +21655 POINT(40.1594493689709 74.2320457851286) bank21655 +21656 POINT(41.431919770472845 74.95003867270151) bank21656 +21657 POINT(40.89031501586847 73.86125958249147) bank21657 +21658 POINT(40.65959397323051 73.91584566655588) bank21658 +21659 POINT(39.886396089016124 73.36564189247773) bank21659 +21660 POINT(40.35455114812368 74.46746322741578) bank21660 +21661 POINT(39.92567189925643 73.24034931339534) bank21661 +21662 POINT(41.54850626091304 74.20748947646858) bank21662 +21663 POINT(41.456811173467784 74.7624053507162) bank21663 +21664 POINT(41.403845490267535 74.02666348136569) bank21664 +21665 POINT(40.13820723293252 74.23192467407404) bank21665 +21666 POINT(41.71210966564407 74.78953586200666) bank21666 +21667 POINT(39.9988719134176 73.41781330238493) bank21667 +21668 POINT(39.90356695984911 74.19514777152324) bank21668 +21669 POINT(40.67938198732597 73.28304207511043) bank21669 +21670 POINT(41.00540076137408 74.15079597679069) bank21670 +21671 POINT(39.983574964445936 73.53564480421478) bank21671 +21672 POINT(41.213323499300685 73.9429733547047) bank21672 +21673 POINT(39.752576227656796 74.0637809325465) bank21673 +21674 POINT(39.79270253756288 73.97792455004686) bank21674 +21675 POINT(39.740295400628895 73.29981295053463) bank21675 +21676 POINT(39.92545679924394 73.32710253981854) bank21676 +21677 POINT(41.379740002280336 73.6184500341091) bank21677 +21678 POINT(40.61034906540326 73.77567411888363) bank21678 +21679 POINT(40.785748996686124 73.919943042334) bank21679 +21680 POINT(41.30566991728878 74.8497114520776) bank21680 +21681 POINT(41.5657603398679 74.14148520425688) bank21681 +21682 POINT(41.529947267575544 73.82577664569806) bank21682 +21683 POINT(40.701875620592645 74.57920013912853) bank21683 +21684 POINT(39.91792906069026 73.12337518620924) bank21684 +21685 POINT(39.72360934721683 73.99356070040784) bank21685 +21686 POINT(40.99071143008514 74.30635527965829) bank21686 +21687 POINT(40.03234369672909 74.5974932453917) bank21687 +21688 POINT(41.49504401677831 73.68860483312791) bank21688 +21689 POINT(41.552227260279224 73.53443779187808) bank21689 +21690 POINT(41.53608104296177 74.20958431208223) bank21690 +21691 POINT(40.8764853844066 74.08901904638972) bank21691 +21692 POINT(40.99710781545318 73.14793387766962) bank21692 +21693 POINT(39.970206905975196 74.87154209425564) bank21693 +21694 POINT(41.65121338693875 73.4166077531325) bank21694 +21695 POINT(40.966863823535505 73.20731157524597) bank21695 +21696 POINT(41.10370685847438 74.37421699126811) bank21696 +21697 POINT(41.37260207763891 73.3926110242835) bank21697 +21698 POINT(39.92091401067395 74.64322002487513) bank21698 +21699 POINT(40.20050474729719 74.05516826322459) bank21699 +21700 POINT(41.65509978507674 74.75084601113329) bank21700 +21701 POINT(40.590888344498055 74.49234867155495) bank21701 +21702 POINT(39.94887241542509 74.50995381934722) bank21702 +21703 POINT(41.326716616181244 74.06205187410089) bank21703 +21704 POINT(41.12592487965735 74.3400928136412) bank21704 +21705 POINT(40.10776189014995 73.81791818692341) bank21705 +21706 POINT(39.85424144070775 74.19589380578465) bank21706 +21707 POINT(41.26415631060893 74.89516432724335) bank21707 +21708 POINT(41.433572157319865 74.71013379288709) bank21708 +21709 POINT(40.760546916684135 73.82706307816241) bank21709 +21710 POINT(40.536481592589936 73.85998213577206) bank21710 +21711 POINT(40.26513066131814 74.62897377697911) bank21711 +21712 POINT(40.05748923968204 74.86278534792741) bank21712 +21713 POINT(40.69505144576755 74.09573867998682) bank21713 +21714 POINT(41.49918465577811 73.0255920522192) bank21714 +21715 POINT(40.83108100816008 74.77159570429652) bank21715 +21716 POINT(40.3912929427089 74.31259775925382) bank21716 +21717 POINT(40.025538242861145 74.64107925658585) bank21717 +21718 POINT(39.95919776435198 73.14618584042628) bank21718 +21719 POINT(41.50100177435848 74.22008920605859) bank21719 +21720 POINT(40.54705362218982 74.70831302347696) bank21720 +21721 POINT(41.43403490039158 73.9480201118587) bank21721 +21722 POINT(41.288989372212555 73.77317485469852) bank21722 +21723 POINT(40.5372424321401 74.79472358131069) bank21723 +21724 POINT(40.155313284647356 73.84618795838577) bank21724 +21725 POINT(39.94343765693531 73.24776847395307) bank21725 +21726 POINT(39.92020589981776 74.55065624744145) bank21726 +21727 POINT(41.29292580548735 73.29845576110074) bank21727 +21728 POINT(39.71593291278943 73.6322454182795) bank21728 +21729 POINT(40.62335442024685 74.6115084276937) bank21729 +21730 POINT(40.7819985971513 74.9543265537829) bank21730 +21731 POINT(40.12107184533129 74.49970152580882) bank21731 +21732 POINT(39.77287269078035 74.56800465130213) bank21732 +21733 POINT(41.63157406446533 74.60435168801547) bank21733 +21734 POINT(41.62134331409375 73.67694189392039) bank21734 +21735 POINT(41.39192976139418 73.37322073688293) bank21735 +21736 POINT(41.61892904448796 73.32880012209492) bank21736 +21737 POINT(41.496332393622296 74.88682164372914) bank21737 +21738 POINT(40.016445593278675 74.45140931880967) bank21738 +21739 POINT(41.048833444183856 74.31031865549464) bank21739 +21740 POINT(41.04883636725418 74.13774850958767) bank21740 +21741 POINT(40.00368538143033 74.86209317731618) bank21741 +21742 POINT(41.14170878500224 74.74925765501703) bank21742 +21743 POINT(41.6923885590086 74.09031576482343) bank21743 +21744 POINT(40.31447311151058 73.25486427950129) bank21744 +21745 POINT(41.46966948399328 73.53132836456717) bank21745 +21746 POINT(40.384149217134876 73.23044293065294) bank21746 +21747 POINT(40.85232208704398 73.58770991003587) bank21747 +21748 POINT(41.68322277086389 74.14558977425904) bank21748 +21749 POINT(40.65717099399628 74.4659781333648) bank21749 +21750 POINT(40.65278323050638 73.08929053671952) bank21750 +21751 POINT(40.12626102897281 73.27579663146871) bank21751 +21752 POINT(40.288783081212685 74.35622510464302) bank21752 +21753 POINT(39.83857509622036 73.72027941124354) bank21753 +21754 POINT(40.96921479235229 73.32010597097442) bank21754 +21755 POINT(41.26506319375054 74.96175198911158) bank21755 +21756 POINT(40.330624905690314 74.68304844257986) bank21756 +21757 POINT(40.47026532415065 73.90117269759052) bank21757 +21758 POINT(39.8893696314625 74.73461301082696) bank21758 +21759 POINT(39.927922920421956 74.33857264644917) bank21759 +21760 POINT(40.00741103236755 74.37981721438994) bank21760 +21761 POINT(40.56714910070023 73.76288175993206) bank21761 +21762 POINT(41.66906598767682 74.23157608591563) bank21762 +21763 POINT(40.15090820677852 74.57994250927024) bank21763 +21764 POINT(41.008854686821 73.09531190945725) bank21764 +21765 POINT(39.86026547121036 74.84030174971615) bank21765 +21766 POINT(40.52975336095635 73.39313781839432) bank21766 +21767 POINT(41.35630647699763 74.4464591322239) bank21767 +21768 POINT(40.85082881506875 74.51675068656485) bank21768 +21769 POINT(40.967542291551126 73.44938876723195) bank21769 +21770 POINT(40.69149847862818 74.33458008525055) bank21770 +21771 POINT(41.39493143092608 74.07950501492017) bank21771 +21772 POINT(40.850819349352356 74.75421410031012) bank21772 +21773 POINT(40.14774507191296 73.09860934837971) bank21773 +21774 POINT(41.33815719810305 74.70769741505515) bank21774 +21775 POINT(40.0563279721323 74.33195035724154) bank21775 +21776 POINT(41.402885566309436 73.6317173803805) bank21776 +21777 POINT(40.919493670630494 73.72316274453593) bank21777 +21778 POINT(40.10035921652322 73.20654444916354) bank21778 +21779 POINT(40.69833014495062 73.60993830754) bank21779 +21780 POINT(41.12316201400206 74.6573354575872) bank21780 +21781 POINT(39.775955400169764 74.71950063275551) bank21781 +21782 POINT(40.478102066474584 73.42673367062379) bank21782 +21783 POINT(40.44826770139699 74.80005853618376) bank21783 +21784 POINT(41.629054745392104 73.87996437302883) bank21784 +21785 POINT(40.069197909719406 73.03115264759305) bank21785 +21786 POINT(40.639649084926255 74.5946782374259) bank21786 +21787 POINT(40.4311376783188 73.35678221602346) bank21787 +21788 POINT(40.01383789994479 74.19845114427542) bank21788 +21789 POINT(39.870392495680406 74.24703446196457) bank21789 +21790 POINT(39.79272826953109 74.64493337189568) bank21790 +21791 POINT(40.111841311294896 73.41393903299186) bank21791 +21792 POINT(39.988959388472246 74.55837969888543) bank21792 +21793 POINT(40.1412119563232 73.43056663194373) bank21793 +21794 POINT(40.71281215161988 73.58335249256123) bank21794 +21795 POINT(40.66105042595548 73.59131846543127) bank21795 +21796 POINT(40.310146565716025 73.09730131718436) bank21796 +21797 POINT(39.83406841420616 74.96632307886246) bank21797 +21798 POINT(41.33727010710303 73.61722817598675) bank21798 +21799 POINT(40.95667936829851 73.3138856617993) bank21799 +21800 POINT(41.52081186825726 74.95030730581513) bank21800 +21801 POINT(40.6791195217923 74.67156357945616) bank21801 +21802 POINT(39.799087496635984 73.483609394648) bank21802 +21803 POINT(40.5550737802881 73.89103223669115) bank21803 +21804 POINT(40.61864780738592 73.14857543068175) bank21804 +21805 POINT(40.965182927672 73.07430227640243) bank21805 +21806 POINT(41.36851241995519 73.68223366664664) bank21806 +21807 POINT(40.14985950854255 73.40545732584778) bank21807 +21808 POINT(39.92198808736371 74.62712479855884) bank21808 +21809 POINT(40.258440693824575 73.40373390714237) bank21809 +21810 POINT(40.44918737900095 73.33732378107896) bank21810 +21811 POINT(40.32418234228546 73.71527219849689) bank21811 +21812 POINT(41.705098759981894 73.4355751763356) bank21812 +21813 POINT(39.867878731570244 73.1549860457103) bank21813 +21814 POINT(40.38866537444655 73.3890241561993) bank21814 +21815 POINT(41.11613657820289 73.08879233981138) bank21815 +21816 POINT(40.229781134975404 73.21943656569377) bank21816 +21817 POINT(41.00268304388188 73.53827989802129) bank21817 +21818 POINT(40.84775378397753 73.18691731948533) bank21818 +21819 POINT(41.054873726420226 74.81755750452665) bank21819 +21820 POINT(41.074869367386334 74.81507548567537) bank21820 +21821 POINT(40.65334020134898 74.26547217340156) bank21821 +21822 POINT(40.155170215598176 73.94241673011119) bank21822 +21823 POINT(39.93516945202527 73.69359798743902) bank21823 +21824 POINT(41.37933800476069 74.71788006568764) bank21824 +21825 POINT(40.21644201796361 74.99355528403052) bank21825 +21826 POINT(40.241038140700596 74.99587497071082) bank21826 +21827 POINT(40.45660182014595 73.24621716206829) bank21827 +21828 POINT(40.812811276939854 74.13272939486102) bank21828 +21829 POINT(40.87581679012435 74.36649189133385) bank21829 +21830 POINT(41.69865051632431 74.79470356346424) bank21830 +21831 POINT(40.60850149177369 74.63387114073323) bank21831 +21832 POINT(40.40069599088403 73.57920525604469) bank21832 +21833 POINT(40.298285181388515 73.93476648763746) bank21833 +21834 POINT(39.88649060610128 73.11939208372048) bank21834 +21835 POINT(39.99830302728869 73.43855074282826) bank21835 +21836 POINT(40.152356068060406 73.15427120350682) bank21836 +21837 POINT(39.80991428688203 74.49837509661482) bank21837 +21838 POINT(40.6572481210142 74.20396784815223) bank21838 +21839 POINT(41.35917178293685 74.00261330414811) bank21839 +21840 POINT(40.39811825987318 74.38382682934689) bank21840 +21841 POINT(40.805683212476644 74.63600952345263) bank21841 +21842 POINT(41.48970085136297 73.95566724866065) bank21842 +21843 POINT(41.627835216113894 73.14673125186376) bank21843 +21844 POINT(41.35698310817774 73.27910370308112) bank21844 +21845 POINT(40.17230977677253 74.70935801129933) bank21845 +21846 POINT(40.02374891070103 74.2817134825989) bank21846 +21847 POINT(41.544238796302736 74.0312822457848) bank21847 +21848 POINT(40.974890221503834 74.86009697000723) bank21848 +21849 POINT(40.49180894444377 74.38249519925635) bank21849 +21850 POINT(41.05115438709516 74.28735800263468) bank21850 +21851 POINT(40.28922203735927 74.866403965926) bank21851 +21852 POINT(39.795703990469114 74.22459058201613) bank21852 +21853 POINT(40.73517357772951 74.64429706125054) bank21853 +21854 POINT(41.40985299169027 73.7719460297616) bank21854 +21855 POINT(40.04497336576011 74.89898288962858) bank21855 +21856 POINT(40.303059882734175 73.50155313802344) bank21856 +21857 POINT(41.22630177767127 74.14456319942676) bank21857 +21858 POINT(39.828638805491984 73.36133158080789) bank21858 +21859 POINT(40.54839162312299 73.86275673862161) bank21859 +21860 POINT(41.08543186048066 73.46886316656169) bank21860 +21861 POINT(40.917608730327785 74.59062710991427) bank21861 +21862 POINT(39.998033003447524 74.03625183248195) bank21862 +21863 POINT(39.93167498841126 73.28674186417999) bank21863 +21864 POINT(40.37302075800329 74.7340118964853) bank21864 +21865 POINT(41.38556124695002 74.25903856592409) bank21865 +21866 POINT(39.76452786522653 73.47217783689076) bank21866 +21867 POINT(40.69865670275713 74.05081349943904) bank21867 +21868 POINT(40.101921308341005 73.76043590477795) bank21868 +21869 POINT(41.173998089726425 74.38652420970806) bank21869 +21870 POINT(39.76171957116117 74.4560502507322) bank21870 +21871 POINT(41.41827069141494 73.71077631726587) bank21871 +21872 POINT(40.414675449929746 74.3904706689694) bank21872 +21873 POINT(40.29041526354124 73.89103274737796) bank21873 +21874 POINT(41.05774135041166 74.56123010728577) bank21874 +21875 POINT(40.07188739289149 73.29332647932121) bank21875 +21876 POINT(41.51397695506184 73.97489660546738) bank21876 +21877 POINT(41.66098141593749 74.27811307842546) bank21877 +21878 POINT(40.978536479964404 73.02951064421913) bank21878 +21879 POINT(39.87893403337835 74.91408714006116) bank21879 +21880 POINT(40.86507491869174 73.61160708356333) bank21880 +21881 POINT(39.789536752186756 73.4007976391147) bank21881 +21882 POINT(40.214335926965866 73.87074007626046) bank21882 +21883 POINT(40.42084483122944 73.95062184034855) bank21883 +21884 POINT(39.95890404004041 73.48546698333907) bank21884 +21885 POINT(40.62409698754764 74.37032861404398) bank21885 +21886 POINT(40.325872140553344 73.2307024596325) bank21886 +21887 POINT(40.31217732847395 74.36331816853624) bank21887 +21888 POINT(40.51015343424712 74.80694906631231) bank21888 +21889 POINT(41.17821330838921 73.47187990162936) bank21889 +21890 POINT(41.67993809902829 74.75124278278305) bank21890 +21891 POINT(40.49759072869679 74.73794478027382) bank21891 +21892 POINT(41.552675604096194 73.78914542578586) bank21892 +21893 POINT(41.345516095339086 74.64692704037758) bank21893 +21894 POINT(40.84931343905175 74.73032754828701) bank21894 +21895 POINT(40.38769833401097 73.1921164339711) bank21895 +21896 POINT(40.34537152280228 73.67565869863847) bank21896 +21897 POINT(41.00545629519683 73.86527838736515) bank21897 +21898 POINT(39.79277528723189 73.41703368154239) bank21898 +21899 POINT(41.69492291068028 73.93217043109298) bank21899 +21900 POINT(41.27223968886923 73.26657967289317) bank21900 +21901 POINT(41.0776819745059 73.5700037457935) bank21901 +21902 POINT(41.68986761279842 73.30836556647718) bank21902 +21903 POINT(40.89246031777977 74.31877356764336) bank21903 +21904 POINT(41.343056479528 74.31841260142875) bank21904 +21905 POINT(41.65100918720332 73.33479853264379) bank21905 +21906 POINT(39.9030355845641 74.32623101519702) bank21906 +21907 POINT(40.330444114890234 73.8333835515385) bank21907 +21908 POINT(40.02855933735433 74.28333468779108) bank21908 +21909 POINT(39.7618690544967 74.25709029389257) bank21909 +21910 POINT(40.32924167806223 73.76276484888861) bank21910 +21911 POINT(41.52223174361498 74.88731520071754) bank21911 +21912 POINT(39.771883341021166 73.26266576106717) bank21912 +21913 POINT(40.674449234752295 74.0917784560734) bank21913 +21914 POINT(41.37527746554342 73.56840473200779) bank21914 +21915 POINT(41.212789947689274 74.45189601137183) bank21915 +21916 POINT(40.24718490009102 74.80314094874998) bank21916 +21917 POINT(41.45296029871768 74.8821130179727) bank21917 +21918 POINT(40.45171817191108 73.06627959099214) bank21918 +21919 POINT(39.9554503187117 73.74066821281545) bank21919 +21920 POINT(41.25017622095674 74.84049357884892) bank21920 +21921 POINT(41.622567885371055 74.37706888605365) bank21921 +21922 POINT(40.52214925880892 73.35170262132219) bank21922 +21923 POINT(40.95518223921164 74.5580457196068) bank21923 +21924 POINT(40.47794468773103 73.30270215263096) bank21924 +21925 POINT(40.574647305622385 74.7170220432456) bank21925 +21926 POINT(40.12936910366399 74.91271687455774) bank21926 +21927 POINT(40.761848502690235 74.18676163050964) bank21927 +21928 POINT(39.80629126105529 73.89609580931706) bank21928 +21929 POINT(39.97052223699921 74.54503513281139) bank21929 +21930 POINT(41.525732954449765 74.0429600961283) bank21930 +21931 POINT(39.72371156329543 73.63184064497665) bank21931 +21932 POINT(40.37024533846903 73.35268797235342) bank21932 +21933 POINT(41.214589661007366 74.37403933090208) bank21933 +21934 POINT(40.996133738943584 74.06547012476281) bank21934 +21935 POINT(40.11274744198305 74.05391878797869) bank21935 +21936 POINT(41.4570403474672 73.34032552127337) bank21936 +21937 POINT(39.92799275204397 75.00077280878062) bank21937 +21938 POINT(41.23083974903013 73.0393126418802) bank21938 +21939 POINT(39.80031127056438 73.33654970547205) bank21939 +21940 POINT(40.36289595168701 73.71646613245902) bank21940 +21941 POINT(41.1926094540621 73.15482643649163) bank21941 +21942 POINT(41.14577787787485 73.7638667093616) bank21942 +21943 POINT(41.254305429038894 74.29785145473065) bank21943 +21944 POINT(41.64180386826563 74.16407119579893) bank21944 +21945 POINT(40.97365141322883 74.25606602759956) bank21945 +21946 POINT(41.1034729696939 74.98497647766939) bank21946 +21947 POINT(41.222206448938586 74.62580651798271) bank21947 +21948 POINT(39.9892969879061 73.73886693370758) bank21948 +21949 POINT(40.214060899863114 74.97201040654369) bank21949 +21950 POINT(41.618049993301824 73.6963243732579) bank21950 +21951 POINT(39.91612337299314 73.66361917327028) bank21951 +21952 POINT(40.4120951740091 73.05973311423544) bank21952 +21953 POINT(40.01064270442357 73.37928039786368) bank21953 +21954 POINT(39.71543529293366 74.94926692607717) bank21954 +21955 POINT(39.727013295443555 74.78069275764639) bank21955 +21956 POINT(40.36237324661897 74.71086868542892) bank21956 +21957 POINT(40.18980530856467 74.60044677957292) bank21957 +21958 POINT(40.211921518230255 73.33006288537196) bank21958 +21959 POINT(40.19467979914663 74.95115540516832) bank21959 +21960 POINT(40.74307963659073 74.16049701179874) bank21960 +21961 POINT(39.90299745721795 73.59485420471866) bank21961 +21962 POINT(40.253745339543784 73.80970771423668) bank21962 +21963 POINT(40.235110690356386 73.81519457477295) bank21963 +21964 POINT(41.31664529766777 74.97461475652366) bank21964 +21965 POINT(40.05547749462432 73.83705393201863) bank21965 +21966 POINT(41.32038135390717 73.94436848405336) bank21966 +21967 POINT(40.80594523830856 73.47566436045625) bank21967 +21968 POINT(39.736085241514964 74.11458849560348) bank21968 +21969 POINT(41.18393238458063 74.56901616860624) bank21969 +21970 POINT(40.421645962676415 73.20843314552397) bank21970 +21971 POINT(41.20294449463798 73.89199046221484) bank21971 +21972 POINT(41.03744283447634 73.6351017897752) bank21972 +21973 POINT(40.357831474162474 74.95158558578575) bank21973 +21974 POINT(40.92465952236266 73.73920545242237) bank21974 +21975 POINT(40.713523255266225 75.00399031462548) bank21975 +21976 POINT(40.98418709901723 74.0092688385866) bank21976 +21977 POINT(40.83052004637493 74.01938045866636) bank21977 +21978 POINT(40.0157550929182 74.11533299527139) bank21978 +21979 POINT(40.884281278512375 73.2772634315354) bank21979 +21980 POINT(40.52485174357837 73.03874492436655) bank21980 +21981 POINT(40.996488750303286 74.70450556650306) bank21981 +21982 POINT(40.963717536031524 74.3465074735316) bank21982 +21983 POINT(39.99063818367398 74.01811835584549) bank21983 +21984 POINT(40.630722883834764 74.81520486948787) bank21984 +21985 POINT(40.63709486254376 73.54830574529721) bank21985 +21986 POINT(40.81194589103299 73.7415337518102) bank21986 +21987 POINT(39.77567036211229 74.07567412025018) bank21987 +21988 POINT(39.80038819136736 73.20380240290055) bank21988 +21989 POINT(40.27169711759878 74.9218100793899) bank21989 +21990 POINT(41.28374858019008 74.78348876216303) bank21990 +21991 POINT(40.037571997332144 74.63094430901879) bank21991 +21992 POINT(41.38816764704523 73.93342219096289) bank21992 +21993 POINT(41.3396724985614 73.28929772281082) bank21993 +21994 POINT(40.013606514221735 73.01846219761755) bank21994 +21995 POINT(40.372972219821946 73.56843860182447) bank21995 +21996 POINT(40.07941080640844 73.70549876979187) bank21996 +21997 POINT(41.323965878365065 74.23822677064015) bank21997 +21998 POINT(40.57832158166688 74.24269192130907) bank21998 +21999 POINT(40.38448974926951 73.43153840837194) bank21999 +22000 POINT(41.599376828325255 74.61635271429489) bank22000 +22001 POINT(41.67640511002938 74.159185911627) bank22001 +22002 POINT(40.5020854434561 74.32485012918683) bank22002 +22003 POINT(40.92608170329953 74.97641273701801) bank22003 +22004 POINT(39.96566254451317 74.21784713183929) bank22004 +22005 POINT(41.11877046872726 73.81558130105276) bank22005 +22006 POINT(41.058114461338555 74.41821786018428) bank22006 +22007 POINT(39.75715538030486 74.586894497608) bank22007 +22008 POINT(40.906506697010244 74.67518156709987) bank22008 +22009 POINT(40.38022665877723 73.0301559226163) bank22009 +22010 POINT(41.24942463917546 73.20882452103037) bank22010 +22011 POINT(40.86054798180615 73.25026942222418) bank22011 +22012 POINT(40.713295447209504 74.800811290484) bank22012 +22013 POINT(40.27431969312962 73.23763298514439) bank22013 +22014 POINT(40.70516599431606 74.79911523820242) bank22014 +22015 POINT(41.099847409093755 73.33446637924224) bank22015 +22016 POINT(41.54765860085992 73.94509190045164) bank22016 +22017 POINT(40.87611557495429 74.48998632687514) bank22017 +22018 POINT(40.25549586495546 74.84482434131601) bank22018 +22019 POINT(40.746648736758075 73.2738816417638) bank22019 +22020 POINT(41.537968282217776 74.74672115385158) bank22020 +22021 POINT(40.69975751849571 73.95278607973722) bank22021 +22022 POINT(40.37707144421876 73.23933158220822) bank22022 +22023 POINT(40.447753967545886 74.36697312002367) bank22023 +22024 POINT(41.20188997359167 74.29260459151816) bank22024 +22025 POINT(41.528107925406246 73.61340775424247) bank22025 +22026 POINT(40.314775757460666 73.02074610765932) bank22026 +22027 POINT(40.570257087536795 73.97649715828814) bank22027 +22028 POINT(40.71239708430499 74.43041744397374) bank22028 +22029 POINT(41.03183137250451 73.0099065321701) bank22029 +22030 POINT(40.61070264610021 73.69381293719131) bank22030 +22031 POINT(41.64644375664848 74.654036326942) bank22031 +22032 POINT(40.7535038636989 74.01160282120549) bank22032 +22033 POINT(40.66253824775302 73.15377801303744) bank22033 +22034 POINT(40.766428482528646 73.53766818831572) bank22034 +22035 POINT(41.70926789498969 73.77978442511845) bank22035 +22036 POINT(41.705387497429214 73.80279868847937) bank22036 +22037 POINT(40.83463252243776 74.42859659772635) bank22037 +22038 POINT(39.829461365830255 74.37678316869165) bank22038 +22039 POINT(40.23630418852595 74.50687639370041) bank22039 +22040 POINT(40.97369261440754 74.79474814758478) bank22040 +22041 POINT(40.28362653262529 73.62985604879185) bank22041 +22042 POINT(41.098327005528404 74.08496094806871) bank22042 +22043 POINT(39.891879705052624 73.03706892609776) bank22043 +22044 POINT(40.35914296032894 73.11945214103426) bank22044 +22045 POINT(40.112991530593014 73.60203610593182) bank22045 +22046 POINT(40.4120196285443 74.41872749677793) bank22046 +22047 POINT(40.87664816134637 74.7936375530177) bank22047 +22048 POINT(40.05433450085612 74.27498135463937) bank22048 +22049 POINT(40.789020170669914 74.03144217427948) bank22049 +22050 POINT(40.666747729782756 74.4250605301925) bank22050 +22051 POINT(40.65421852363052 74.3838295744615) bank22051 +22052 POINT(40.06197660911634 74.1433930610884) bank22052 +22053 POINT(40.53275376503106 74.4821729053793) bank22053 +22054 POINT(39.998195730276294 73.46840270491512) bank22054 +22055 POINT(41.06524541869386 74.72744035713507) bank22055 +22056 POINT(39.76385505365743 73.84162374441851) bank22056 +22057 POINT(40.14226303549448 73.67322215368625) bank22057 +22058 POINT(40.65225973498123 74.38079888276476) bank22058 +22059 POINT(39.84078671496343 73.41517296125055) bank22059 +22060 POINT(39.972342295622774 74.22663304124839) bank22060 +22061 POINT(41.707960004555865 74.25420695322924) bank22061 +22062 POINT(40.46724089573602 73.96520573708344) bank22062 +22063 POINT(40.00798019562655 74.23664471386654) bank22063 +22064 POINT(41.05966114169073 74.52118111675307) bank22064 +22065 POINT(40.54837023143436 73.2117595421181) bank22065 +22066 POINT(39.740795588049906 74.52448381882729) bank22066 +22067 POINT(40.67266106382703 74.5853004027478) bank22067 +22068 POINT(40.113645895529274 74.63065554437874) bank22068 +22069 POINT(40.85430872113427 73.69621145011745) bank22069 +22070 POINT(39.90664867462148 74.71222831938255) bank22070 +22071 POINT(41.58433526622121 74.47357411554165) bank22071 +22072 POINT(41.46552592597932 74.7955465729036) bank22072 +22073 POINT(40.39802663768679 74.75860204787023) bank22073 +22074 POINT(39.924624116149076 74.72100081649087) bank22074 +22075 POINT(41.63296145957671 73.93195365451919) bank22075 +22076 POINT(40.435564888594165 73.60470311641092) bank22076 +22077 POINT(41.066935091710235 73.71945051133444) bank22077 +22078 POINT(39.93589340673298 74.29728152490421) bank22078 +22079 POINT(41.611745592046255 74.84592272254979) bank22079 +22080 POINT(39.874175344810496 74.75736537942039) bank22080 +22081 POINT(40.3163342152574 73.31933014931364) bank22081 +22082 POINT(40.94228777272146 73.14171630417695) bank22082 +22083 POINT(39.97769860199697 73.05198237299221) bank22083 +22084 POINT(41.18161034285413 74.81734938992811) bank22084 +22085 POINT(41.28108040410716 73.89833837810656) bank22085 +22086 POINT(39.830854798625126 74.87200194217236) bank22086 +22087 POINT(40.30838505868113 74.308043172388) bank22087 +22088 POINT(39.774408195992635 73.8933587587985) bank22088 +22089 POINT(40.76468173147401 73.65765011523848) bank22089 +22090 POINT(39.720915681167234 73.81257083664799) bank22090 +22091 POINT(40.68355411982094 74.25880495787851) bank22091 +22092 POINT(40.7848028251941 73.40431450952741) bank22092 +22093 POINT(40.30679109599323 74.3710187197221) bank22093 +22094 POINT(40.827824694293895 73.01216844189685) bank22094 +22095 POINT(39.89646234177872 73.86098351044791) bank22095 +22096 POINT(40.704402053054196 74.88798335813738) bank22096 +22097 POINT(39.7837604383478 73.56934799020941) bank22097 +22098 POINT(39.88045040589044 74.20289617305053) bank22098 +22099 POINT(41.32357856808831 74.21731136143342) bank22099 +22100 POINT(40.91141000434908 73.38261944895319) bank22100 +22101 POINT(41.552148523189295 73.42345961839577) bank22101 +22102 POINT(40.184463109802046 74.00248603890282) bank22102 +22103 POINT(40.49036237031068 73.36556511267204) bank22103 +22104 POINT(41.379032528587196 74.71207991503283) bank22104 +22105 POINT(41.46728421483451 73.36551580358065) bank22105 +22106 POINT(41.048823761799405 73.4290553082663) bank22106 +22107 POINT(40.90786210483602 74.46645273156238) bank22107 +22108 POINT(40.907464554287216 73.2681349474964) bank22108 +22109 POINT(40.45002316325288 73.71024525288608) bank22109 +22110 POINT(39.91227305880931 74.15069812757388) bank22110 +22111 POINT(40.082346980682814 74.75371713846117) bank22111 +22112 POINT(40.52270919091757 73.8269735997709) bank22112 +22113 POINT(41.55787382931976 73.3183075430225) bank22113 +22114 POINT(40.360502374804305 74.61150054299512) bank22114 +22115 POINT(39.95263862794652 74.0479246235908) bank22115 +22116 POINT(40.48656928731003 73.91724575372533) bank22116 +22117 POINT(40.735275981169465 74.04178681036534) bank22117 +22118 POINT(41.659071073734694 74.00799849955301) bank22118 +22119 POINT(40.87058241306979 74.53466090158072) bank22119 +22120 POINT(40.55619774686616 73.65639151950799) bank22120 +22121 POINT(41.44840746551079 74.27018624877701) bank22121 +22122 POINT(40.45247589176736 74.63146207662301) bank22122 +22123 POINT(40.228008957745296 73.02665471332571) bank22123 +22124 POINT(40.33535086753343 74.4844632430684) bank22124 +22125 POINT(39.939431946654715 74.41664833365647) bank22125 +22126 POINT(40.00704873132036 73.30498954959234) bank22126 +22127 POINT(41.30597443289761 74.0545121972088) bank22127 +22128 POINT(40.64481510215989 74.86850989487682) bank22128 +22129 POINT(41.295301278987736 73.2805912863071) bank22129 +22130 POINT(40.22064169683781 74.0919551953309) bank22130 +22131 POINT(40.762582967981906 73.74197024863757) bank22131 +22132 POINT(39.967538110579774 74.57683162112075) bank22132 +22133 POINT(41.315812174304476 74.2463927946671) bank22133 +22134 POINT(41.26580204987977 74.69072523582614) bank22134 +22135 POINT(40.64094931129304 73.99777553099796) bank22135 +22136 POINT(40.235440635939916 73.62384306410476) bank22136 +22137 POINT(41.253330961438756 73.53183295904276) bank22137 +22138 POINT(41.17110501135754 74.44081117232622) bank22138 +22139 POINT(40.41193727554732 73.58250361788501) bank22139 +22140 POINT(40.43613737179512 73.41871244957478) bank22140 +22141 POINT(39.797850942486804 74.31253020668917) bank22141 +22142 POINT(41.07869885847516 73.75952170129156) bank22142 +22143 POINT(40.784910003330104 74.678340248335) bank22143 +22144 POINT(39.880356704478196 73.6267965073312) bank22144 +22145 POINT(40.48645447538175 73.03789789787433) bank22145 +22146 POINT(40.985029803050274 73.98594455613481) bank22146 +22147 POINT(41.35010574840205 74.30422117718093) bank22147 +22148 POINT(39.86027169465517 74.44654463427263) bank22148 +22149 POINT(41.069610198918014 74.8120195220101) bank22149 +22150 POINT(39.725081410450926 74.20506518596605) bank22150 +22151 POINT(39.992176971753764 74.50856535895623) bank22151 +22152 POINT(40.0360891129316 74.93766275992073) bank22152 +22153 POINT(40.14140436759977 73.68228515413267) bank22153 +22154 POINT(40.75505335647243 74.84013132093827) bank22154 +22155 POINT(40.10949042041774 74.14156272317351) bank22155 +22156 POINT(41.30799828575176 74.76661064430084) bank22156 +22157 POINT(40.964311477213215 73.86002862913634) bank22157 +22158 POINT(41.14673275646273 74.6982363981184) bank22158 +22159 POINT(40.13061394841535 74.03946013786702) bank22159 +22160 POINT(40.521605026878 74.67633728984234) bank22160 +22161 POINT(41.31227176317236 74.24931685127375) bank22161 +22162 POINT(40.56066889279322 73.13456055977969) bank22162 +22163 POINT(39.86093655150651 73.78465013653032) bank22163 +22164 POINT(41.27245979810249 74.24470937252435) bank22164 +22165 POINT(40.25573678978691 73.7777926255856) bank22165 +22166 POINT(40.23962572878413 73.84714982586365) bank22166 +22167 POINT(40.577485744616276 73.42813253069204) bank22167 +22168 POINT(40.122775109909185 73.25086036439379) bank22168 +22169 POINT(40.094034514907435 74.18216906878374) bank22169 +22170 POINT(40.42809659644432 74.81410243659211) bank22170 +22171 POINT(40.54228459987823 74.91434093293095) bank22171 +22172 POINT(41.05954781637378 74.02287488878208) bank22172 +22173 POINT(41.3875509021188 74.05105610483297) bank22173 +22174 POINT(40.68824422530545 74.65718378721262) bank22174 +22175 POINT(40.559967975545135 73.0407629754383) bank22175 +22176 POINT(41.682897769563425 73.62123378306607) bank22176 +22177 POINT(40.737256201051245 73.63359723407424) bank22177 +22178 POINT(41.64445870934964 74.09510129530713) bank22178 +22179 POINT(41.35180603222403 73.5679622043688) bank22179 +22180 POINT(40.990232886378024 73.62540205393597) bank22180 +22181 POINT(40.88081511838009 74.73404659960731) bank22181 +22182 POINT(39.988718011130175 74.46401754228926) bank22182 +22183 POINT(40.96062246449387 73.83485067446478) bank22183 +22184 POINT(39.77814128745812 74.97091067284228) bank22184 +22185 POINT(40.283468345384435 74.94412032422922) bank22185 +22186 POINT(39.93927226217411 74.33311964715126) bank22186 +22187 POINT(41.07961695170751 73.67185469719124) bank22187 +22188 POINT(39.773787515935936 74.513659943113) bank22188 +22189 POINT(41.500746684511846 74.72635472085994) bank22189 +22190 POINT(39.771105018677545 74.62317140590247) bank22190 +22191 POINT(40.7152468736132 74.30478557344681) bank22191 +22192 POINT(40.93696145610142 74.07288194970825) bank22192 +22193 POINT(41.65898067472823 74.45505467696894) bank22193 +22194 POINT(39.952361744689156 74.91977091993286) bank22194 +22195 POINT(40.05209358943149 73.0730547035287) bank22195 +22196 POINT(40.39681060360107 73.65526905378462) bank22196 +22197 POINT(40.17605148259381 74.50888227237469) bank22197 +22198 POINT(39.99747473523302 73.44286751412224) bank22198 +22199 POINT(40.34952791739849 74.92494427056195) bank22199 +22200 POINT(40.91136729465819 73.11762142136739) bank22200 +22201 POINT(40.23585831235848 73.31950073850157) bank22201 +22202 POINT(40.581030597428935 73.69839009461104) bank22202 +22203 POINT(41.22597474711003 73.14822392995) bank22203 +22204 POINT(40.08270112462505 73.66003009007856) bank22204 +22205 POINT(40.32729954682595 74.04631899679268) bank22205 +22206 POINT(40.30368168073141 74.89368219550214) bank22206 +22207 POINT(40.917749298358224 73.60936407321292) bank22207 +22208 POINT(39.85846747997926 74.73861131613442) bank22208 +22209 POINT(39.79900140103693 74.69033587328995) bank22209 +22210 POINT(40.048030265319305 74.4229030992704) bank22210 +22211 POINT(40.71428281484878 73.97110818952247) bank22211 +22212 POINT(40.493888157510106 73.06898984415908) bank22212 +22213 POINT(40.628479329483845 74.91730013144657) bank22213 +22214 POINT(39.731568118571886 73.8914053573318) bank22214 +22215 POINT(40.02024117243372 73.13693253882167) bank22215 +22216 POINT(40.93695564082436 74.37223402280469) bank22216 +22217 POINT(40.87814002264407 73.4251534607904) bank22217 +22218 POINT(39.824012858179735 73.24047197284766) bank22218 +22219 POINT(39.93210625928672 73.1476047081858) bank22219 +22220 POINT(41.551947087379936 73.69098396559161) bank22220 +22221 POINT(40.404338475862446 74.55656644660891) bank22221 +22222 POINT(40.58967070382647 73.11833872868559) bank22222 +22223 POINT(40.61838729086915 74.03807832589672) bank22223 +22224 POINT(40.46766131100585 73.46490100154223) bank22224 +22225 POINT(40.165655107842326 73.44541679626045) bank22225 +22226 POINT(41.49027981344908 73.35739299134853) bank22226 +22227 POINT(40.486441316227214 73.00820029766132) bank22227 +22228 POINT(41.305120239192775 73.50008541183631) bank22228 +22229 POINT(40.546652443129176 74.97637577752327) bank22229 +22230 POINT(41.406711761164274 73.52776044738538) bank22230 +22231 POINT(40.18367610636944 74.67222207025716) bank22231 +22232 POINT(41.47610118873946 74.86391718565578) bank22232 +22233 POINT(40.595415136480554 74.86755582449116) bank22233 +22234 POINT(41.1564814686162 74.05964734048518) bank22234 +22235 POINT(39.855964390465054 74.36731539635328) bank22235 +22236 POINT(41.356055404982385 74.31288334993516) bank22236 +22237 POINT(40.65977305460962 73.13786585755834) bank22237 +22238 POINT(41.3835406244339 74.28784314012799) bank22238 +22239 POINT(40.309670397275205 74.19830614886332) bank22239 +22240 POINT(40.21453697882227 73.13510633817546) bank22240 +22241 POINT(41.415387157083124 73.37322893673222) bank22241 +22242 POINT(40.879477571047246 73.4967037223479) bank22242 +22243 POINT(41.49035998541065 73.11629494842103) bank22243 +22244 POINT(40.1073385053887 74.9080917421745) bank22244 +22245 POINT(40.742829540657766 73.90489050878948) bank22245 +22246 POINT(40.351801674316185 73.87970938078801) bank22246 +22247 POINT(40.448506283615394 74.57191819879395) bank22247 +22248 POINT(39.80110178776734 74.62386308794675) bank22248 +22249 POINT(40.06836726739346 74.86316271894628) bank22249 +22250 POINT(41.54367061795991 74.24527237764659) bank22250 +22251 POINT(40.33592980304558 73.75885355412825) bank22251 +22252 POINT(40.456596414446786 74.04110726660045) bank22252 +22253 POINT(40.229047372229914 73.82649489064293) bank22253 +22254 POINT(41.33290254290939 74.34995873350283) bank22254 +22255 POINT(40.66872691771678 74.18466923515615) bank22255 +22256 POINT(41.35793155771289 74.20644963381358) bank22256 +22257 POINT(40.013505255891474 73.21471309299689) bank22257 +22258 POINT(40.83615733158919 74.72052768955675) bank22258 +22259 POINT(40.11507828810256 74.73799251412778) bank22259 +22260 POINT(40.91108850905045 74.13776703498147) bank22260 +22261 POINT(41.31275355577762 73.33347075978884) bank22261 +22262 POINT(40.23188212544876 74.15229669836987) bank22262 +22263 POINT(40.98157673800457 73.23880637280739) bank22263 +22264 POINT(40.368180006051595 73.03401398095474) bank22264 +22265 POINT(40.355732273829766 73.9487676306709) bank22265 +22266 POINT(41.54317050299814 74.9985634468157) bank22266 +22267 POINT(41.67930563323117 73.16041681586626) bank22267 +22268 POINT(40.69635849453997 74.84701117361179) bank22268 +22269 POINT(41.130691244813306 74.36690005317864) bank22269 +22270 POINT(40.905170234064485 74.78079114748921) bank22270 +22271 POINT(41.40928305265676 73.81578154797272) bank22271 +22272 POINT(40.297476894092256 74.21339086042026) bank22272 +22273 POINT(40.851641267980355 74.3347623737683) bank22273 +22274 POINT(40.36888228572462 74.36502160821172) bank22274 +22275 POINT(40.443062946638946 74.30565137461073) bank22275 +22276 POINT(39.90984049102899 74.92116025810655) bank22276 +22277 POINT(40.78449638396909 73.54423244458718) bank22277 +22278 POINT(40.645772530527516 73.46433454096059) bank22278 +22279 POINT(39.87178673211123 73.97956629715894) bank22279 +22280 POINT(41.4653616102918 73.9136946300997) bank22280 +22281 POINT(41.05529056580905 74.84117213091653) bank22281 +22282 POINT(41.7058245190462 74.83841209067124) bank22282 +22283 POINT(40.872726286403946 73.25497690925508) bank22283 +22284 POINT(40.479432456194985 73.03883147598549) bank22284 +22285 POINT(39.89326736434076 73.71237027131309) bank22285 +22286 POINT(40.95654736638673 74.89051504932173) bank22286 +22287 POINT(40.7882814974648 73.72481849320492) bank22287 +22288 POINT(40.08491097996067 75.00007915445946) bank22288 +22289 POINT(40.43454196486138 73.56941774343173) bank22289 +22290 POINT(41.619253847423806 73.56702964588105) bank22290 +22291 POINT(40.710438756505944 74.64232408149311) bank22291 +22292 POINT(40.28133166262299 73.82853168378242) bank22292 +22293 POINT(40.52063867835753 74.00430708522794) bank22293 +22294 POINT(40.14082262029698 74.96804476946392) bank22294 +22295 POINT(40.06029817221838 73.41532616449304) bank22295 +22296 POINT(41.0639077093936 74.44458197723218) bank22296 +22297 POINT(41.2395325476635 73.95791905575392) bank22297 +22298 POINT(40.711311890425385 73.72488567280912) bank22298 +22299 POINT(41.58078403817133 74.72242887970913) bank22299 +22300 POINT(41.511756774549056 74.64592126108074) bank22300 +22301 POINT(40.524474572761896 74.9657756399781) bank22301 +22302 POINT(40.374307088000684 74.38719241299128) bank22302 +22303 POINT(40.95553095547782 73.55160979475457) bank22303 +22304 POINT(40.98319662215911 74.64422707890972) bank22304 +22305 POINT(39.828032441864266 73.3411830472032) bank22305 +22306 POINT(40.22134463989133 73.77082247133711) bank22306 +22307 POINT(41.14370030035451 74.34538039752937) bank22307 +22308 POINT(39.71428258691803 74.21119099607681) bank22308 +22309 POINT(40.85548890772492 74.45742518604814) bank22309 +22310 POINT(41.29009818938772 73.09328690152473) bank22310 +22311 POINT(39.81235106136042 74.4802377779233) bank22311 +22312 POINT(40.224566390262815 74.36971440541983) bank22312 +22313 POINT(41.14051367843188 74.38910660881416) bank22313 +22314 POINT(40.807427234436354 74.77236638303609) bank22314 +22315 POINT(39.955567751515815 73.91344043647467) bank22315 +22316 POINT(40.607232070133634 73.04766313987619) bank22316 +22317 POINT(41.18729594181985 74.64451467918926) bank22317 +22318 POINT(39.764966759894115 74.11388884274326) bank22318 +22319 POINT(40.255996801627006 73.61262978559223) bank22319 +22320 POINT(41.13561239889477 74.80099146213404) bank22320 +22321 POINT(40.666432935281186 74.99027037269968) bank22321 +22322 POINT(40.381726153720564 74.1995689796111) bank22322 +22323 POINT(41.666240699853496 73.15707208631619) bank22323 +22324 POINT(40.882066272677314 73.16906612225486) bank22324 +22325 POINT(40.7750351766876 73.96198053882827) bank22325 +22326 POINT(40.38642693002875 74.63128748582133) bank22326 +22327 POINT(40.79402794882538 73.28599234657257) bank22327 +22328 POINT(40.54456936567327 74.54693900513296) bank22328 +22329 POINT(39.780522733549226 73.25597791906415) bank22329 +22330 POINT(40.3439218785851 73.6505901256968) bank22330 +22331 POINT(39.86311154472276 74.82635437575867) bank22331 +22332 POINT(40.813710563691764 73.95341085902875) bank22332 +22333 POINT(40.211252169327906 74.14824614000271) bank22333 +22334 POINT(40.00413388826443 74.93428509528196) bank22334 +22335 POINT(40.07074138527691 74.04232644908195) bank22335 +22336 POINT(41.00485182088014 73.62646157081855) bank22336 +22337 POINT(41.0173887980545 73.97024721739388) bank22337 +22338 POINT(41.30734733587008 74.70940071616424) bank22338 +22339 POINT(40.09070697004171 74.62248775317575) bank22339 +22340 POINT(41.61960059910667 74.70740427523103) bank22340 +22341 POINT(40.343157367297565 74.15149760963656) bank22341 +22342 POINT(41.48437654353184 74.05066681186294) bank22342 +22343 POINT(41.69389956096485 74.19285646644352) bank22343 +22344 POINT(40.52071380765125 73.06914005684774) bank22344 +22345 POINT(41.51571755001267 73.72903715266403) bank22345 +22346 POINT(40.44405430901933 74.16271177415294) bank22346 +22347 POINT(39.72179637425447 74.8737836380288) bank22347 +22348 POINT(40.308491119470304 74.17256841801212) bank22348 +22349 POINT(40.48592608703014 74.50875499423458) bank22349 +22350 POINT(41.273035223103044 74.11617761877825) bank22350 +22351 POINT(41.33779894574662 74.27279037616319) bank22351 +22352 POINT(39.856476203034575 73.35278905377413) bank22352 +22353 POINT(40.45396990216287 73.72231915946917) bank22353 +22354 POINT(40.76570877691647 74.82759280293284) bank22354 +22355 POINT(40.64396061688685 74.20440871394761) bank22355 +22356 POINT(40.51026340509708 74.55849207649638) bank22356 +22357 POINT(40.967139598811514 73.25492873147125) bank22357 +22358 POINT(41.17458070430113 73.26179426056291) bank22358 +22359 POINT(40.364060660396945 73.64162918384997) bank22359 +22360 POINT(40.91425913037327 73.74571542817269) bank22360 +22361 POINT(41.15765094839012 74.16060920830812) bank22361 +22362 POINT(41.66707185806059 74.92732641131174) bank22362 +22363 POINT(41.50926889884445 74.66379129468305) bank22363 +22364 POINT(40.22726542318622 73.20182840694642) bank22364 +22365 POINT(41.270756718225215 73.1432766754022) bank22365 +22366 POINT(39.846714217484205 74.84927684859102) bank22366 +22367 POINT(40.498624587068086 73.46874144892332) bank22367 +22368 POINT(41.34406009168133 73.67089901889953) bank22368 +22369 POINT(40.672836783256216 73.3777061449611) bank22369 +22370 POINT(41.22879969674819 73.26650991090922) bank22370 +22371 POINT(41.22541425292338 74.7257058468236) bank22371 +22372 POINT(41.21422131997117 73.68904006599227) bank22372 +22373 POINT(41.60641503498637 74.885263801767) bank22373 +22374 POINT(40.89571326695485 73.51113842522723) bank22374 +22375 POINT(40.953197783369795 74.51217269062391) bank22375 +22376 POINT(40.513165561983854 74.88761433981674) bank22376 +22377 POINT(40.19339902629537 73.3337270009698) bank22377 +22378 POINT(40.19644333546508 74.42388833855534) bank22378 +22379 POINT(41.34132763871478 73.4844946266587) bank22379 +22380 POINT(41.18091607266384 73.14406569046264) bank22380 +22381 POINT(40.8374560691046 74.15230239212923) bank22381 +22382 POINT(41.47015387166519 74.241295906842) bank22382 +22383 POINT(40.83518275024832 74.23665211753419) bank22383 +22384 POINT(39.86875595304866 74.91477456123052) bank22384 +22385 POINT(40.85627921276371 73.15962539130331) bank22385 +22386 POINT(41.07149630044145 74.85896442245398) bank22386 +22387 POINT(41.29401006044612 74.21624337053458) bank22387 +22388 POINT(39.93800497027044 74.85276146400471) bank22388 +22389 POINT(41.49484914920685 74.92692130706814) bank22389 +22390 POINT(40.65158626370241 73.89210330046147) bank22390 +22391 POINT(39.80774691675966 74.88605690378611) bank22391 +22392 POINT(40.129879077457225 74.91005064402388) bank22392 +22393 POINT(40.03140464195895 73.99238901764797) bank22393 +22394 POINT(40.36132393305724 74.15798453983199) bank22394 +22395 POINT(40.559774970824336 74.41670698064932) bank22395 +22396 POINT(41.117462705077216 74.93746411807564) bank22396 +22397 POINT(41.499446320168275 73.6695059080493) bank22397 +22398 POINT(40.62497935728692 74.91784867185464) bank22398 +22399 POINT(40.98847837195887 74.72350635066255) bank22399 +22400 POINT(40.41479732677706 73.47484231500474) bank22400 +22401 POINT(40.01628587404536 74.17458074581191) bank22401 +22402 POINT(41.49485223726415 73.43669128003609) bank22402 +22403 POINT(40.479223271227355 74.49872837556713) bank22403 +22404 POINT(40.03149365535098 73.08269395133836) bank22404 +22405 POINT(40.788848400340065 74.11858267591789) bank22405 +22406 POINT(41.01024001574976 74.04064449125215) bank22406 +22407 POINT(41.6654568018543 74.37420719432302) bank22407 +22408 POINT(40.76368592490207 73.78153604026548) bank22408 +22409 POINT(39.92047181428445 73.72471492285086) bank22409 +22410 POINT(39.929059327947805 74.74482582894125) bank22410 +22411 POINT(40.11833762013249 74.01165478948168) bank22411 +22412 POINT(39.72760735348856 73.20234286688323) bank22412 +22413 POINT(39.95523404220452 73.51527897868827) bank22413 +22414 POINT(40.1709596912674 73.49164746829231) bank22414 +22415 POINT(40.72817299195607 73.26290638119582) bank22415 +22416 POINT(41.120050131572995 74.76178689667944) bank22416 +22417 POINT(41.13876660791783 74.19261489327236) bank22417 +22418 POINT(40.97774929396667 74.46193936900215) bank22418 +22419 POINT(41.2298271346677 74.19550615598345) bank22419 +22420 POINT(40.02025290797123 74.35624716754994) bank22420 +22421 POINT(41.338117109322496 74.55440678847793) bank22421 +22422 POINT(40.423084016581306 73.71927406959308) bank22422 +22423 POINT(41.27021174199043 73.16638879423942) bank22423 +22424 POINT(41.47390715142857 73.18733906788843) bank22424 +22425 POINT(40.25079585276914 73.79317462775293) bank22425 +22426 POINT(40.099285219350456 74.76280772463056) bank22426 +22427 POINT(41.08388953374151 73.28035044534664) bank22427 +22428 POINT(40.45785040880321 73.09552523415843) bank22428 +22429 POINT(39.96770517422517 74.74116547765803) bank22429 +22430 POINT(40.466537914859956 74.03450660486429) bank22430 +22431 POINT(40.614140944979646 74.7553907109976) bank22431 +22432 POINT(41.452326068183034 73.73540515111533) bank22432 +22433 POINT(40.26888913024195 73.39927719019452) bank22433 +22434 POINT(40.82954680877075 73.67823844485856) bank22434 +22435 POINT(40.379970248466826 74.58708642797075) bank22435 +22436 POINT(40.81681465353234 74.54400552382563) bank22436 +22437 POINT(41.42841868855358 73.70247340558068) bank22437 +22438 POINT(40.39746442126007 73.03711212432248) bank22438 +22439 POINT(40.14951319534355 73.37280249961523) bank22439 +22440 POINT(41.70729180656746 74.64759809227878) bank22440 +22441 POINT(39.99642153623587 73.05888145603706) bank22441 +22442 POINT(41.210887845597654 74.52319093849687) bank22442 +22443 POINT(40.62101899858483 73.67184864385794) bank22443 +22444 POINT(41.58885620918749 73.57470393148935) bank22444 +22445 POINT(41.482319021648536 74.77723391666493) bank22445 +22446 POINT(41.49183400975098 74.71092169606679) bank22446 +22447 POINT(41.42045510028628 74.7562129840133) bank22447 +22448 POINT(41.33958695721905 74.64480140359431) bank22448 +22449 POINT(40.273407612407496 73.90228586967444) bank22449 +22450 POINT(40.42926552855238 73.40604480824504) bank22450 +22451 POINT(41.03924525508274 73.50395064153354) bank22451 +22452 POINT(41.42255372144506 73.7379157000929) bank22452 +22453 POINT(41.455827902321566 73.40371537996624) bank22453 +22454 POINT(41.0939750261933 73.15581012733226) bank22454 +22455 POINT(40.27668302834123 74.01328141771369) bank22455 +22456 POINT(41.36011414494836 73.56883904040758) bank22456 +22457 POINT(39.77919881605235 73.60694089891834) bank22457 +22458 POINT(41.64826921749299 74.22073161240141) bank22458 +22459 POINT(39.71499662119689 73.85747541283517) bank22459 +22460 POINT(39.739133468508925 73.43246598855134) bank22460 +22461 POINT(41.40060311065209 73.48629454718782) bank22461 +22462 POINT(41.12260948397905 74.93804699736826) bank22462 +22463 POINT(41.23692632260706 74.90568695310672) bank22463 +22464 POINT(40.30013421521573 73.09702471329608) bank22464 +22465 POINT(40.405920846365476 73.8600834328955) bank22465 +22466 POINT(41.704850476818656 74.11080385447383) bank22466 +22467 POINT(40.407847965247235 74.89186372755503) bank22467 +22468 POINT(39.909718546144504 73.73912342120806) bank22468 +22469 POINT(40.50991765157493 74.7984392432381) bank22469 +22470 POINT(40.45803002440707 73.6511632383047) bank22470 +22471 POINT(40.16642847813148 74.22195190732182) bank22471 +22472 POINT(40.00739179745621 73.20921697931952) bank22472 +22473 POINT(40.135469007545375 73.91619872317743) bank22473 +22474 POINT(40.74750271337429 73.49986718400265) bank22474 +22475 POINT(40.129378018765934 74.72325306327826) bank22475 +22476 POINT(41.37105106218591 74.98200123438633) bank22476 +22477 POINT(40.1744532800139 74.2745301391639) bank22477 +22478 POINT(40.626744139444696 73.82282734779832) bank22478 +22479 POINT(40.00896084866202 73.52996490604944) bank22479 +22480 POINT(40.82101778335118 74.28284228879302) bank22480 +22481 POINT(40.058127588711486 73.70400526575483) bank22481 +22482 POINT(41.417525471449416 73.36021921153079) bank22482 +22483 POINT(41.45311387907995 74.93038033785612) bank22483 +22484 POINT(40.047166111684106 74.64547496231594) bank22484 +22485 POINT(39.87713544268093 73.80072871659799) bank22485 +22486 POINT(39.892768540001775 74.43045945030369) bank22486 +22487 POINT(40.767843815809094 74.24531515441548) bank22487 +22488 POINT(40.4330872317424 74.56730597336475) bank22488 +22489 POINT(41.59311078618929 74.53177923557921) bank22489 +22490 POINT(40.24755951944164 74.4209653926277) bank22490 +22491 POINT(40.34664115887031 73.28865988572859) bank22491 +22492 POINT(41.52950083801264 74.60696387385795) bank22492 +22493 POINT(41.076638455332144 73.34733179060785) bank22493 +22494 POINT(39.997632731745675 73.42228677241235) bank22494 +22495 POINT(41.665534411443666 74.91110322901551) bank22495 +22496 POINT(40.316318611146684 74.02821283763501) bank22496 +22497 POINT(41.23345573231402 73.91762345966146) bank22497 +22498 POINT(40.972791636048136 73.44171705577801) bank22498 +22499 POINT(41.45063626792764 74.40203571373458) bank22499 +22500 POINT(39.98878074342554 74.53767280140221) bank22500 +22501 POINT(40.4495937976527 73.3159575288442) bank22501 +22502 POINT(41.63069647530011 73.69045906960083) bank22502 +22503 POINT(40.729312803607066 74.6262422296896) bank22503 +22504 POINT(40.85637792975489 73.94416180520753) bank22504 +22505 POINT(40.94231050378018 74.90649925379324) bank22505 +22506 POINT(41.20855732234939 73.49108949768619) bank22506 +22507 POINT(41.28632568638065 73.36003960683522) bank22507 +22508 POINT(41.08829236468793 73.98917929524835) bank22508 +22509 POINT(41.32906006767611 73.32662683439376) bank22509 +22510 POINT(40.67346655387225 74.31966900651202) bank22510 +22511 POINT(41.550099183808335 74.00811653667365) bank22511 +22512 POINT(40.8920461061437 73.76383221608722) bank22512 +22513 POINT(41.53143418927303 73.01925994202587) bank22513 +22514 POINT(40.38337057404056 74.62926441130678) bank22514 +22515 POINT(40.05757856375643 74.84160153385704) bank22515 +22516 POINT(40.27085880496523 74.8736109665579) bank22516 +22517 POINT(40.80033735288873 74.597829561841) bank22517 +22518 POINT(40.078328355412424 74.7993520886393) bank22518 +22519 POINT(41.47928305183311 73.11664616033481) bank22519 +22520 POINT(40.34375399930606 73.71193778752298) bank22520 +22521 POINT(41.430814059278305 73.55551431023521) bank22521 +22522 POINT(40.989004562056685 73.2954778360974) bank22522 +22523 POINT(41.05772718630493 74.71555457086126) bank22523 +22524 POINT(40.828647407950825 74.99431829825379) bank22524 +22525 POINT(39.96905342752783 73.08974393857828) bank22525 +22526 POINT(41.67044492498574 74.87696143642816) bank22526 +22527 POINT(40.504693609044224 73.31639390391429) bank22527 +22528 POINT(39.94220081145732 74.19241786593231) bank22528 +22529 POINT(41.40033408583856 74.77847726764921) bank22529 +22530 POINT(40.57806610610956 74.23428700140371) bank22530 +22531 POINT(39.998287508201656 74.52895738318759) bank22531 +22532 POINT(40.83451410132485 74.3466130995647) bank22532 +22533 POINT(39.80200012814258 74.88116497550703) bank22533 +22534 POINT(41.18421766974129 73.59694256561964) bank22534 +22535 POINT(39.76642874216515 74.90362561318096) bank22535 +22536 POINT(41.16453440599672 74.87868900836511) bank22536 +22537 POINT(41.159448431697655 73.92747872891204) bank22537 +22538 POINT(40.53558633529065 74.77268131306225) bank22538 +22539 POINT(40.40105373384306 73.92446714008871) bank22539 +22540 POINT(40.68927007238319 73.87170775820732) bank22540 +22541 POINT(40.57788173450533 73.58953753689934) bank22541 +22542 POINT(41.358314900403215 74.35282454032284) bank22542 +22543 POINT(40.93020089313303 74.04339819303719) bank22543 +22544 POINT(41.7083147371382 74.29905104581557) bank22544 +22545 POINT(40.0776436886273 74.50914760618579) bank22545 +22546 POINT(41.23730414876741 73.73737649346407) bank22546 +22547 POINT(40.766958292519476 73.86530398840372) bank22547 +22548 POINT(39.99553171376422 74.42287589534328) bank22548 +22549 POINT(41.60207227138972 73.80885188420734) bank22549 +22550 POINT(39.9448307420912 73.5800530310135) bank22550 +22551 POINT(40.94956712090178 74.56291568735284) bank22551 +22552 POINT(41.436650431179594 74.7775761033818) bank22552 +22553 POINT(40.16902215321954 73.70164235915026) bank22553 +22554 POINT(41.64575248402241 73.02218141566823) bank22554 +22555 POINT(41.29144874087169 73.94987494915668) bank22555 +22556 POINT(41.11367839586369 74.02459105570284) bank22556 +22557 POINT(39.8376536992496 73.04516427663297) bank22557 +22558 POINT(40.90089031436007 74.75545755932264) bank22558 +22559 POINT(40.488812959646864 73.66472312536534) bank22559 +22560 POINT(40.66056148013102 73.6327581998266) bank22560 +22561 POINT(39.86109243828098 73.49995964321499) bank22561 +22562 POINT(40.70493731118357 73.20882747861333) bank22562 +22563 POINT(39.75290087221559 74.59860280330263) bank22563 +22564 POINT(40.77040486447593 73.74465015230447) bank22564 +22565 POINT(41.07649545847131 73.46761576984042) bank22565 +22566 POINT(39.757220132153186 73.4072252710589) bank22566 +22567 POINT(40.46878833013502 73.90758114370168) bank22567 +22568 POINT(40.62242952344327 73.36721868942172) bank22568 +22569 POINT(40.18419462070023 73.36679454091795) bank22569 +22570 POINT(41.00737876972262 73.24061730343465) bank22570 +22571 POINT(40.14411323885471 74.99751062949088) bank22571 +22572 POINT(40.62337836086015 73.84970869741498) bank22572 +22573 POINT(41.536055796327716 74.91496084335829) bank22573 +22574 POINT(40.70013628152627 74.31703197424865) bank22574 +22575 POINT(40.890649622496845 73.85292663469642) bank22575 +22576 POINT(41.26288503462688 73.5903287009397) bank22576 +22577 POINT(40.8212011567148 73.40188905577158) bank22577 +22578 POINT(40.38018852407553 74.14254566393127) bank22578 +22579 POINT(41.27897189385699 73.048707057293) bank22579 +22580 POINT(41.418538975139825 74.31895508415363) bank22580 +22581 POINT(39.94955937397226 73.19810325076402) bank22581 +22582 POINT(40.437538924989454 73.50997414267567) bank22582 +22583 POINT(40.43782002930076 73.79179272254387) bank22583 +22584 POINT(41.12002556308087 73.47350140652965) bank22584 +22585 POINT(39.90505544828929 74.0534235250221) bank22585 +22586 POINT(39.77242632414086 74.52225727601076) bank22586 +22587 POINT(39.73817015745818 74.50536740745416) bank22587 +22588 POINT(40.62295600958054 74.00231421613816) bank22588 +22589 POINT(41.16694032673452 74.29003895318452) bank22589 +22590 POINT(41.10964008539921 73.01388249384786) bank22590 +22591 POINT(41.122210790973526 74.17923463236316) bank22591 +22592 POINT(40.45185245461223 74.4543978831731) bank22592 +22593 POINT(40.169386435158806 73.19781032092381) bank22593 +22594 POINT(41.6166954816322 73.51588509752669) bank22594 +22595 POINT(41.70783374890751 74.9644197327853) bank22595 +22596 POINT(41.04927457511225 74.32534051086503) bank22596 +22597 POINT(40.125523699135115 74.02616405078999) bank22597 +22598 POINT(40.97778855495397 74.34384000573506) bank22598 +22599 POINT(40.07539220546841 73.22103345423423) bank22599 +22600 POINT(41.66155340404973 74.40191069046645) bank22600 +22601 POINT(41.15728051238096 74.98972869101365) bank22601 +22602 POINT(40.178697826541296 74.3684849398667) bank22602 +22603 POINT(40.88468854418213 74.61571647674319) bank22603 +22604 POINT(40.081646389740456 73.77051649165722) bank22604 +22605 POINT(39.73511679692874 74.20117504447963) bank22605 +22606 POINT(41.218855864847264 74.9356867112214) bank22606 +22607 POINT(40.15264642880166 73.82813978457399) bank22607 +22608 POINT(40.92689984510358 73.37510171813318) bank22608 +22609 POINT(39.86952634710881 74.21773842497181) bank22609 +22610 POINT(40.707107088104614 74.95949952296395) bank22610 +22611 POINT(40.522353332783105 73.73186856362923) bank22611 +22612 POINT(39.77448922561083 73.12829872529804) bank22612 +22613 POINT(39.78662510324878 74.87837405096762) bank22613 +22614 POINT(40.01044641733077 73.2975405887846) bank22614 +22615 POINT(41.50602024428043 73.57879573853847) bank22615 +22616 POINT(40.26520716968243 73.23398241569062) bank22616 +22617 POINT(40.18053943373003 74.56943732594762) bank22617 +22618 POINT(39.804843777013716 74.26032430185603) bank22618 +22619 POINT(40.50632877962255 73.2439802222381) bank22619 +22620 POINT(40.4487163583399 73.43636178530609) bank22620 +22621 POINT(41.566714055148495 73.12049366693684) bank22621 +22622 POINT(40.05509490083649 74.19591879088298) bank22622 +22623 POINT(40.444007401471744 73.32903565554065) bank22623 +22624 POINT(40.810122399150025 73.68770083909644) bank22624 +22625 POINT(39.790307210890575 74.21338019593975) bank22625 +22626 POINT(40.489997862628044 73.07635468697735) bank22626 +22627 POINT(40.06520440335899 74.54540664011611) bank22627 +22628 POINT(40.60420493184828 74.26595914720049) bank22628 +22629 POINT(39.85643642227464 74.66625756763862) bank22629 +22630 POINT(39.847355766993104 74.84856605008909) bank22630 +22631 POINT(40.35884641446278 74.03155795002822) bank22631 +22632 POINT(40.363746026043124 74.3447644562319) bank22632 +22633 POINT(40.59864226950758 73.08173899028667) bank22633 +22634 POINT(39.83675652239897 74.9117050594791) bank22634 +22635 POINT(41.38718046243155 74.92960103617168) bank22635 +22636 POINT(41.55732594025399 73.78036027508988) bank22636 +22637 POINT(40.84515700108773 74.50559681579037) bank22637 +22638 POINT(40.898175291208666 74.06585943184368) bank22638 +22639 POINT(41.258213909935066 74.94051020444147) bank22639 +22640 POINT(39.902058244852896 74.47690012949835) bank22640 +22641 POINT(41.05559124651247 73.75742665715937) bank22641 +22642 POINT(41.53257039973019 74.26442548380713) bank22642 +22643 POINT(40.35597906363374 73.22418960104059) bank22643 +22644 POINT(40.209972897067686 73.68143979323754) bank22644 +22645 POINT(40.23495266489979 74.30869160933926) bank22645 +22646 POINT(41.53253001194574 73.60537093057509) bank22646 +22647 POINT(41.62726600251538 74.94842366920544) bank22647 +22648 POINT(41.671935954913096 73.02399420083856) bank22648 +22649 POINT(40.380518410976165 73.67616730810411) bank22649 +22650 POINT(41.683231019549545 73.86919375973314) bank22650 +22651 POINT(41.68983550581625 73.37542572882836) bank22651 +22652 POINT(41.66879799208701 74.51220888027474) bank22652 +22653 POINT(40.464714683045656 73.5428157872604) bank22653 +22654 POINT(41.020351939336315 73.26547641755499) bank22654 +22655 POINT(39.988257716975056 73.62551228719865) bank22655 +22656 POINT(40.55877610797381 73.87723042618423) bank22656 +22657 POINT(41.18408073633485 74.8036822598185) bank22657 +22658 POINT(41.14955631875061 73.89109380129412) bank22658 +22659 POINT(39.90060947784233 74.45783854707926) bank22659 +22660 POINT(41.641770777329086 73.54262730184504) bank22660 +22661 POINT(40.99735600528945 73.60314494797142) bank22661 +22662 POINT(40.95924263229332 73.71707122113838) bank22662 +22663 POINT(40.45037437600816 74.48440798507053) bank22663 +22664 POINT(40.17100455327443 73.19781449433862) bank22664 +22665 POINT(40.2362729092657 74.01946731104688) bank22665 +22666 POINT(39.83132346588302 74.94161030614048) bank22666 +22667 POINT(40.17302800257783 74.15029460216579) bank22667 +22668 POINT(41.06860516614283 74.45077839800213) bank22668 +22669 POINT(41.32867516511574 73.32474572478151) bank22669 +22670 POINT(40.60042614130202 74.94092821746528) bank22670 +22671 POINT(40.972985063687865 73.97079784166978) bank22671 +22672 POINT(39.93654559360148 74.94924192245644) bank22672 +22673 POINT(41.50860637410664 74.32241946705321) bank22673 +22674 POINT(40.97169377053387 73.29847379885345) bank22674 +22675 POINT(41.265993833130864 74.77791704709948) bank22675 +22676 POINT(41.08737567181067 73.30135888705965) bank22676 +22677 POINT(40.913702846933255 74.72298190343638) bank22677 +22678 POINT(41.45356837238707 74.51706162446752) bank22678 +22679 POINT(40.56052196526625 73.30627914811345) bank22679 +22680 POINT(41.42188169054107 73.1315627666458) bank22680 +22681 POINT(40.521358909361346 73.73028742161117) bank22681 +22682 POINT(41.540958196390804 73.51325188328249) bank22682 +22683 POINT(40.07344441700941 74.6008162236852) bank22683 +22684 POINT(39.97324467988906 74.26887414237812) bank22684 +22685 POINT(40.5289347171867 74.93284251088832) bank22685 +22686 POINT(39.85257557449689 73.20257515243333) bank22686 +22687 POINT(40.905650494337365 73.53162109428611) bank22687 +22688 POINT(40.95931711114356 73.42682177683726) bank22688 +22689 POINT(41.18729081753358 74.76934059306842) bank22689 +22690 POINT(40.3094662320667 74.2427286725939) bank22690 +22691 POINT(40.77943021865189 73.37071241119352) bank22691 +22692 POINT(39.865964921452665 74.33789338736852) bank22692 +22693 POINT(40.5915209557062 74.74861014491337) bank22693 +22694 POINT(41.258521087054824 74.23656108409631) bank22694 +22695 POINT(40.949010463947346 73.50219903749723) bank22695 +22696 POINT(40.85015112188165 73.9448505262948) bank22696 +22697 POINT(40.90716215718891 73.87305689503555) bank22697 +22698 POINT(41.67575209045024 74.22060364279865) bank22698 +22699 POINT(41.66630494797893 74.7164830392713) bank22699 +22700 POINT(40.81203876950448 73.13927200331774) bank22700 +22701 POINT(41.01595823661492 74.647975795673) bank22701 +22702 POINT(40.086399909413494 74.68132427469249) bank22702 +22703 POINT(40.76456608996166 73.021195139266) bank22703 +22704 POINT(41.10508936539688 74.65421755279627) bank22704 +22705 POINT(41.504714117174125 74.284771559073) bank22705 +22706 POINT(40.64926144006262 74.05536890429791) bank22706 +22707 POINT(41.62724444280281 73.40665063130126) bank22707 +22708 POINT(41.31537780915858 73.68231057457275) bank22708 +22709 POINT(40.16736318672959 74.3702745732742) bank22709 +22710 POINT(39.766880718248515 73.80208138610891) bank22710 +22711 POINT(40.755598236597464 73.89710874501183) bank22711 +22712 POINT(41.072414597139435 73.09449387462645) bank22712 +22713 POINT(40.24677969766069 74.8752858432873) bank22713 +22714 POINT(40.13561852184726 74.12958568584419) bank22714 +22715 POINT(40.53483784328495 74.28239377472605) bank22715 +22716 POINT(40.16124867215257 73.68208513063041) bank22716 +22717 POINT(41.70304167451426 74.63455717083266) bank22717 +22718 POINT(41.53606633055772 73.67997100886427) bank22718 +22719 POINT(40.025775418445726 73.57588556935842) bank22719 +22720 POINT(40.49314021991862 74.24854265383026) bank22720 +22721 POINT(40.1684362427261 74.85483123714206) bank22721 +22722 POINT(39.91573432035342 74.94687323694514) bank22722 +22723 POINT(41.58380756770753 74.57749515808204) bank22723 +22724 POINT(41.18945108876804 73.7257870818045) bank22724 +22725 POINT(39.812844952577464 73.96625877913067) bank22725 +22726 POINT(40.02775705801036 74.73347121320596) bank22726 +22727 POINT(40.30826711979139 74.61419466349523) bank22727 +22728 POINT(40.97586822879345 73.60005805326321) bank22728 +22729 POINT(40.646482476195324 73.04844682014328) bank22729 +22730 POINT(41.66093197728867 74.16236243980408) bank22730 +22731 POINT(40.208876534370134 74.78648784943474) bank22731 +22732 POINT(41.577458059326 73.3703360845139) bank22732 +22733 POINT(40.78068699412655 74.07619968293706) bank22733 +22734 POINT(41.01017221932421 74.81332949527163) bank22734 +22735 POINT(40.28636285314868 73.14834045465528) bank22735 +22736 POINT(40.99003830440447 74.44828455748608) bank22736 +22737 POINT(41.617937759737316 73.01157585999007) bank22737 +22738 POINT(41.25274924172137 73.93422064904885) bank22738 +22739 POINT(40.56912141888231 74.13286138299972) bank22739 +22740 POINT(40.51744278693541 73.55456680761804) bank22740 +22741 POINT(41.03692467016165 73.55333549839061) bank22741 +22742 POINT(39.8485942637001 73.13160203034437) bank22742 +22743 POINT(40.5541898030503 73.47182511354534) bank22743 +22744 POINT(40.955643510591464 74.57673730847316) bank22744 +22745 POINT(40.658234233897645 73.07969233493169) bank22745 +22746 POINT(39.74929845521146 74.83785921939048) bank22746 +22747 POINT(40.96618689217541 73.9369022227378) bank22747 +22748 POINT(39.763766189856746 74.86348546159755) bank22748 +22749 POINT(39.91625290578741 74.74196752553492) bank22749 +22750 POINT(41.44945663628003 74.26421115022998) bank22750 +22751 POINT(40.89688519705627 74.33593481599208) bank22751 +22752 POINT(39.74996838672345 74.69702359024383) bank22752 +22753 POINT(40.96358339323159 73.67678755121428) bank22753 +22754 POINT(40.161274242003664 73.38074256657596) bank22754 +22755 POINT(39.76471820968153 74.25474195901843) bank22755 +22756 POINT(40.54305942601352 73.35058254422329) bank22756 +22757 POINT(40.19685629780613 73.6409268346283) bank22757 +22758 POINT(40.65942291860001 74.26626456340655) bank22758 +22759 POINT(39.73148461543606 73.0920106048134) bank22759 +22760 POINT(41.614283077390084 74.44186792839436) bank22760 +22761 POINT(41.394558645745896 73.1611516421734) bank22761 +22762 POINT(40.725089128883795 74.94609006468897) bank22762 +22763 POINT(40.27534487725605 74.00894712493604) bank22763 +22764 POINT(39.96412515108267 73.48673524802736) bank22764 +22765 POINT(41.309165843988325 74.58317046160454) bank22765 +22766 POINT(40.4058068422766 74.58436509649638) bank22766 +22767 POINT(40.16803140295673 74.86359916711513) bank22767 +22768 POINT(41.31458102309742 73.80886601864619) bank22768 +22769 POINT(40.657168864345316 73.57425219953197) bank22769 +22770 POINT(41.222423531385296 74.3288394218574) bank22770 +22771 POINT(41.182121794592184 74.8698641827606) bank22771 +22772 POINT(40.85535518654557 73.08241897225192) bank22772 +22773 POINT(41.16832890260631 74.77018259621248) bank22773 +22774 POINT(41.507878261050216 74.34273540678069) bank22774 +22775 POINT(41.44307647760617 74.17250394786733) bank22775 +22776 POINT(41.40130498494072 74.91594727337194) bank22776 +22777 POINT(40.232299140429426 73.45996031021184) bank22777 +22778 POINT(40.27430579257792 73.80822058838662) bank22778 +22779 POINT(41.27186367777046 74.97321774354579) bank22779 +22780 POINT(40.613489720514536 74.11510364589303) bank22780 +22781 POINT(41.444780134617695 73.57935207195408) bank22781 +22782 POINT(41.17701302375234 74.22020150054547) bank22782 +22783 POINT(40.60225622619275 73.86903484532151) bank22783 +22784 POINT(40.97401313942074 73.26575939165751) bank22784 +22785 POINT(41.23120385977843 74.15022748195167) bank22785 +22786 POINT(40.38746780619966 74.91617002925385) bank22786 +22787 POINT(41.14103037470187 73.2313704187844) bank22787 +22788 POINT(41.50758530223529 74.77915051227312) bank22788 +22789 POINT(40.77969183220024 74.0835464271074) bank22789 +22790 POINT(41.50738926855488 73.69486229108246) bank22790 +22791 POINT(40.509634062904915 73.28843287894158) bank22791 +22792 POINT(41.06346229381168 74.73503409722471) bank22792 +22793 POINT(41.611318568749226 73.51183867028033) bank22793 +22794 POINT(40.7483405213225 74.94212996514662) bank22794 +22795 POINT(41.067636922464516 74.21588151821015) bank22795 +22796 POINT(41.40097275415642 74.3276419463168) bank22796 +22797 POINT(40.75812650915945 74.29502973801814) bank22797 +22798 POINT(40.6129629528395 74.33384380288467) bank22798 +22799 POINT(39.74331365502966 73.04344938300096) bank22799 +22800 POINT(40.431767491041526 73.62480194619872) bank22800 +22801 POINT(40.16947109707364 73.80339567970634) bank22801 +22802 POINT(41.21410890676654 74.85429628160539) bank22802 +22803 POINT(41.07560686204688 74.47882706212356) bank22803 +22804 POINT(41.17767906997818 73.07920938293096) bank22804 +22805 POINT(41.48952576326762 73.29615026471535) bank22805 +22806 POINT(41.65324056870421 74.57124669956626) bank22806 +22807 POINT(41.59498745175793 73.0196657116273) bank22807 +22808 POINT(40.891410903047564 74.89608459382448) bank22808 +22809 POINT(40.436080133851505 74.0759681016611) bank22809 +22810 POINT(39.94500061422433 73.10218137122365) bank22810 +22811 POINT(40.44641011453778 73.3871031788271) bank22811 +22812 POINT(40.16376263436438 73.70340480526563) bank22812 +22813 POINT(41.2939753159575 74.72696244655464) bank22813 +22814 POINT(41.26775213350261 74.94442243379524) bank22814 +22815 POINT(41.401208865629656 73.35627134604682) bank22815 +22816 POINT(40.91161240613652 74.88314928648651) bank22816 +22817 POINT(41.65671279243119 73.53644896393865) bank22817 +22818 POINT(40.174696076202196 74.5025631351891) bank22818 +22819 POINT(41.69340616131012 73.2835484410659) bank22819 +22820 POINT(39.75875927161317 73.10522052120068) bank22820 +22821 POINT(39.82949414005533 74.16331929630847) bank22821 +22822 POINT(39.761127674487796 73.66334338199383) bank22822 +22823 POINT(41.35240595246823 74.7911816012478) bank22823 +22824 POINT(41.01807354818303 74.94156151591422) bank22824 +22825 POINT(40.61938971234803 73.35019039704821) bank22825 +22826 POINT(40.13407385409045 73.3799402718376) bank22826 +22827 POINT(40.802184323256604 75.00528318168736) bank22827 +22828 POINT(40.490769050712494 73.14691621301974) bank22828 +22829 POINT(39.948927751834795 73.06364711676093) bank22829 +22830 POINT(41.15823263508557 73.84085769359706) bank22830 +22831 POINT(40.324425929720405 73.23278242424314) bank22831 +22832 POINT(40.534629946203495 74.29778872341684) bank22832 +22833 POINT(40.34271540652779 73.49433665545166) bank22833 +22834 POINT(40.28204900562648 74.78705980603476) bank22834 +22835 POINT(40.13623188922804 73.73736219525534) bank22835 +22836 POINT(40.519835018688845 73.90822768568914) bank22836 +22837 POINT(40.45443676130371 73.24510723252884) bank22837 +22838 POINT(41.0071460631584 74.63245855835967) bank22838 +22839 POINT(41.17148973714937 74.22337319835974) bank22839 +22840 POINT(40.80230451116082 73.55721096212088) bank22840 +22841 POINT(40.681970691390475 73.7625108939318) bank22841 +22842 POINT(40.03144769365547 74.20147086739469) bank22842 +22843 POINT(39.910894709931185 74.9368676281276) bank22843 +22844 POINT(39.73290355935383 74.04737842022088) bank22844 +22845 POINT(41.27096940637069 74.80220675774686) bank22845 +22846 POINT(41.15137407367274 73.62510993432767) bank22846 +22847 POINT(39.876477699251396 74.00197722523161) bank22847 +22848 POINT(39.82659035081199 73.47341944049838) bank22848 +22849 POINT(40.52281210188428 73.91067795892461) bank22849 +22850 POINT(40.33324867102385 73.7154588969266) bank22850 +22851 POINT(40.28955839093115 73.82224674444558) bank22851 +22852 POINT(41.06476683315942 73.89323235379142) bank22852 +22853 POINT(40.47421483349984 74.54277258951255) bank22853 +22854 POINT(40.382029408674185 74.2573676310402) bank22854 +22855 POINT(40.48823874077675 73.60787948665774) bank22855 +22856 POINT(41.63779289932045 73.02314764594064) bank22856 +22857 POINT(40.68678986330394 74.34347908665829) bank22857 +22858 POINT(40.19953077586795 73.7985418171826) bank22858 +22859 POINT(40.314008242243766 74.68438923793335) bank22859 +22860 POINT(40.7671534817562 73.10854317131954) bank22860 +22861 POINT(41.52192805296744 73.86711023003363) bank22861 +22862 POINT(41.26021500382266 74.22413534728062) bank22862 +22863 POINT(41.13371999613329 74.87421836264495) bank22863 +22864 POINT(40.26718828079032 73.41641424699938) bank22864 +22865 POINT(41.44024422112726 73.92460532348247) bank22865 +22866 POINT(41.05249866772573 74.861674270436) bank22866 +22867 POINT(40.395568669604266 74.20500589984086) bank22867 +22868 POINT(41.16651755364725 73.09188667933371) bank22868 +22869 POINT(41.2886075848131 74.86246104258599) bank22869 +22870 POINT(39.730441681309074 73.08430139460528) bank22870 +22871 POINT(40.7275692244314 73.42319653780895) bank22871 +22872 POINT(41.13335355332711 74.26619727611435) bank22872 +22873 POINT(40.44004536376775 74.02521081744733) bank22873 +22874 POINT(40.07892428473791 73.95017484731152) bank22874 +22875 POINT(39.967395587408795 74.6669991760114) bank22875 +22876 POINT(40.82678326281245 73.4087501538247) bank22876 +22877 POINT(40.23525276776118 73.76502935906153) bank22877 +22878 POINT(39.801442509089306 74.83633406020762) bank22878 +22879 POINT(41.185348719459284 73.98182454479975) bank22879 +22880 POINT(40.90500787271161 73.33307646501513) bank22880 +22881 POINT(39.86815835646063 74.89579474239869) bank22881 +22882 POINT(41.70010063313858 73.85434531181814) bank22882 +22883 POINT(41.20132923501471 74.74486022206925) bank22883 +22884 POINT(40.31659344975718 74.63863621607975) bank22884 +22885 POINT(40.40654346001038 74.36297698613696) bank22885 +22886 POINT(40.71698192802249 73.48304516147462) bank22886 +22887 POINT(41.20461951881633 74.73622705018877) bank22887 +22888 POINT(41.31098747031017 74.0010882465504) bank22888 +22889 POINT(40.17459912018512 74.18166778215281) bank22889 +22890 POINT(40.96450083002174 74.49050616425475) bank22890 +22891 POINT(40.7810661756159 73.21732953522105) bank22891 +22892 POINT(39.75951022865325 73.10355761727104) bank22892 +22893 POINT(41.06545331788971 73.78740300194019) bank22893 +22894 POINT(39.85147748556348 73.41566516407964) bank22894 +22895 POINT(40.3188179693514 74.89997387488685) bank22895 +22896 POINT(40.93311080625459 74.10492219615028) bank22896 +22897 POINT(40.58677183165329 73.76209422855699) bank22897 +22898 POINT(41.386377962627755 74.07941886044618) bank22898 +22899 POINT(41.56977349312095 73.22622764951916) bank22899 +22900 POINT(41.03530895715928 74.24673851572177) bank22900 +22901 POINT(39.747481786513035 74.62311512343169) bank22901 +22902 POINT(40.287429650749374 74.04351463460165) bank22902 +22903 POINT(39.79710635744667 74.85316214191772) bank22903 +22904 POINT(40.17536799878595 73.75734784812212) bank22904 +22905 POINT(40.09868029528576 73.86003057270356) bank22905 +22906 POINT(40.37522163043949 74.54277257363995) bank22906 +22907 POINT(40.11264955221086 74.96308293843242) bank22907 +22908 POINT(40.210427031496316 73.04805003444999) bank22908 +22909 POINT(39.870076303276576 74.19397326001204) bank22909 +22910 POINT(40.694959101233174 73.23369666075511) bank22910 +22911 POINT(41.59256753296363 74.18504088397984) bank22911 +22912 POINT(41.21532804040205 74.94009954429393) bank22912 +22913 POINT(40.992912104999775 74.51597646944548) bank22913 +22914 POINT(41.419187282573816 73.96669979430179) bank22914 +22915 POINT(40.451557063113185 73.86174403990306) bank22915 +22916 POINT(40.04109248836534 74.59055485681984) bank22916 +22917 POINT(40.82138457197676 74.10567734327574) bank22917 +22918 POINT(40.307847026436065 73.54946990677482) bank22918 +22919 POINT(40.859990195329054 73.10398498486244) bank22919 +22920 POINT(40.08618278963091 73.69274556209089) bank22920 +22921 POINT(39.7260231066086 73.28275663718973) bank22921 +22922 POINT(40.9241283746064 73.43172532527721) bank22922 +22923 POINT(40.718008743897016 73.13396003020748) bank22923 +22924 POINT(40.40762318205058 73.90733456413858) bank22924 +22925 POINT(41.58406848721694 74.72309596779701) bank22925 +22926 POINT(41.06710760424915 73.64302408867792) bank22926 +22927 POINT(40.8591902856375 73.10729038136236) bank22927 +22928 POINT(39.793714429543705 74.31448640716445) bank22928 +22929 POINT(40.170342631196824 73.21766448862955) bank22929 +22930 POINT(40.07925317304983 73.96020287357999) bank22930 +22931 POINT(40.44864267053958 74.46982708614192) bank22931 +22932 POINT(41.314033709444956 74.51330313199443) bank22932 +22933 POINT(41.01859901914405 73.76869260173487) bank22933 +22934 POINT(41.15381650368879 73.7577797092683) bank22934 +22935 POINT(41.00767740483006 73.67513729950842) bank22935 +22936 POINT(40.575087666662384 73.05365371757976) bank22936 +22937 POINT(41.57883886770194 73.95322872992493) bank22937 +22938 POINT(40.3220173296387 73.77235993785537) bank22938 +22939 POINT(40.753403648619155 73.28896307178071) bank22939 +22940 POINT(41.45905352623603 74.06184759973289) bank22940 +22941 POINT(39.889320397160326 73.81038105528168) bank22941 +22942 POINT(40.25168146677203 74.21175502083908) bank22942 +22943 POINT(40.71584737851793 74.51087420500293) bank22943 +22944 POINT(40.864963787962175 74.77281211132335) bank22944 +22945 POINT(40.62474164026827 74.41847528285874) bank22945 +22946 POINT(40.710601050173096 74.05416983323404) bank22946 +22947 POINT(41.684458573170645 73.36857906566965) bank22947 +22948 POINT(41.51072340181855 74.03802782431526) bank22948 +22949 POINT(40.0193022943469 74.75623244744241) bank22949 +22950 POINT(40.80180752764513 73.91576523464322) bank22950 +22951 POINT(40.54815244439579 73.01876187195784) bank22951 +22952 POINT(41.053638877895146 74.46335110631195) bank22952 +22953 POINT(40.87265487549895 73.26764444589884) bank22953 +22954 POINT(41.034454583116116 74.09311330729892) bank22954 +22955 POINT(39.86117427520978 74.06169621380405) bank22955 +22956 POINT(40.61180200842191 74.17607066136281) bank22956 +22957 POINT(41.5822103934165 73.6004412546224) bank22957 +22958 POINT(41.362608840005485 73.31213363119096) bank22958 +22959 POINT(40.18284335869955 74.04938113226802) bank22959 +22960 POINT(41.505240432376574 74.41986176308107) bank22960 +22961 POINT(41.328610151481215 74.61454346538437) bank22961 +22962 POINT(40.52867807180899 73.36765413482763) bank22962 +22963 POINT(40.13230145976509 73.90391385811567) bank22963 +22964 POINT(41.34812829548637 74.78975109825089) bank22964 +22965 POINT(41.587972819214926 73.38688053493536) bank22965 +22966 POINT(41.1046844446993 74.75305960639817) bank22966 +22967 POINT(41.47736410554974 74.71425462134901) bank22967 +22968 POINT(40.20431389910574 74.68823716775775) bank22968 +22969 POINT(41.2791485988006 74.68042645124702) bank22969 +22970 POINT(41.0957022237725 73.53890948740526) bank22970 +22971 POINT(41.29212712942219 74.59078460651911) bank22971 +22972 POINT(40.28508272596114 73.42739394145617) bank22972 +22973 POINT(40.292895085978486 73.21339509484561) bank22973 +22974 POINT(40.042234955022124 73.91866041948812) bank22974 +22975 POINT(39.72900596121971 73.1266907928352) bank22975 +22976 POINT(40.74554613629057 73.954823160636) bank22976 +22977 POINT(40.75154556040998 74.51633694597285) bank22977 +22978 POINT(41.14898154487444 74.07562833435183) bank22978 +22979 POINT(41.704198510579296 75.00426716530183) bank22979 +22980 POINT(40.68838146262768 74.73233109624884) bank22980 +22981 POINT(39.78742689158327 74.02439458897295) bank22981 +22982 POINT(39.852117508514354 73.14338294377512) bank22982 +22983 POINT(41.273419092186906 74.39108002514112) bank22983 +22984 POINT(41.62886734623523 73.22104606224498) bank22984 +22985 POINT(41.690792609685495 74.27774868445157) bank22985 +22986 POINT(41.67319758733375 74.32215683816304) bank22986 +22987 POINT(41.14676995648815 73.53308903720524) bank22987 +22988 POINT(41.45833895493745 74.51243791579458) bank22988 +22989 POINT(39.8075481599162 73.58670776982578) bank22989 +22990 POINT(39.88894200822233 73.108401669084) bank22990 +22991 POINT(40.534455235052434 73.6094979385775) bank22991 +22992 POINT(41.228224700831845 74.54485665202306) bank22992 +22993 POINT(40.01809248122696 73.36663037958073) bank22993 +22994 POINT(40.697955502598546 73.9845972696311) bank22994 +22995 POINT(40.6911809428967 73.20148656538267) bank22995 +22996 POINT(39.92104352834864 74.91852124600304) bank22996 +22997 POINT(41.61875504042287 73.52428352506338) bank22997 +22998 POINT(40.624896272133405 73.96122604556496) bank22998 +22999 POINT(39.94688202552824 74.08800643043287) bank22999 +23000 POINT(39.80911508256085 74.48952634326905) bank23000 +23001 POINT(40.490049020742696 74.21757340580945) bank23001 +23002 POINT(41.36046432535518 73.96391267049431) bank23002 +23003 POINT(40.848760879795414 74.66478401956678) bank23003 +23004 POINT(40.57589682678031 73.86264001912788) bank23004 +23005 POINT(39.79795014790246 73.46893029463989) bank23005 +23006 POINT(41.482512971161285 74.95754303722707) bank23006 +23007 POINT(39.8186371588651 74.21985793283446) bank23007 +23008 POINT(41.168110358608885 73.74213337920023) bank23008 +23009 POINT(40.99977380358184 73.52116190758542) bank23009 +23010 POINT(40.588403362869926 73.91193125116204) bank23010 +23011 POINT(40.489233929960484 74.85295190462604) bank23011 +23012 POINT(41.45833878740989 74.90014781672518) bank23012 +23013 POINT(40.587927400136955 74.14392994131661) bank23013 +23014 POINT(40.83793069156779 73.70160964896449) bank23014 +23015 POINT(40.454641555988 73.97835161339547) bank23015 +23016 POINT(41.25211902847356 74.19834104644968) bank23016 +23017 POINT(40.98952431815729 74.08331553399643) bank23017 +23018 POINT(40.69309695327169 73.3472848474905) bank23018 +23019 POINT(39.72312998115909 73.95701855708914) bank23019 +23020 POINT(40.72445690529963 73.87421235436226) bank23020 +23021 POINT(40.95748486954185 74.28631979061169) bank23021 +23022 POINT(40.146290716861444 73.4418410202232) bank23022 +23023 POINT(41.457664254359955 73.69376726980482) bank23023 +23024 POINT(40.39431923981312 73.00684822428471) bank23024 +23025 POINT(41.112838420912546 74.63430306695327) bank23025 +23026 POINT(40.7459004425468 74.00297073018359) bank23026 +23027 POINT(40.56963421922602 74.1584439410865) bank23027 +23028 POINT(41.26364135984858 74.66424683523441) bank23028 +23029 POINT(40.9542423525518 74.7419981858676) bank23029 +23030 POINT(40.60644473786522 74.85846129835863) bank23030 +23031 POINT(40.070650738846396 74.0139415351258) bank23031 +23032 POINT(41.465696543830916 74.69611474366364) bank23032 +23033 POINT(40.344962547523124 73.77415415661493) bank23033 +23034 POINT(40.512357689791614 74.30809869194641) bank23034 +23035 POINT(40.56799957319835 74.97069632760228) bank23035 +23036 POINT(39.98791896885785 73.33177806520531) bank23036 +23037 POINT(40.83324302463969 74.3131057433138) bank23037 +23038 POINT(41.57823135538071 73.64256102907693) bank23038 +23039 POINT(40.60323144168476 74.13781712543988) bank23039 +23040 POINT(41.27974512571056 74.36231808334381) bank23040 +23041 POINT(39.73351305306328 73.1425166549259) bank23041 +23042 POINT(40.39994902761163 74.71740961298184) bank23042 +23043 POINT(40.75922388597777 74.76806428391917) bank23043 +23044 POINT(41.4282792257313 74.37865790382877) bank23044 +23045 POINT(39.74630995408949 74.85857396592098) bank23045 +23046 POINT(41.10886137517942 73.41446039731007) bank23046 +23047 POINT(41.600383317380135 73.61349736696418) bank23047 +23048 POINT(39.82796258836591 73.47829188360646) bank23048 +23049 POINT(41.128290467344655 74.5180339670531) bank23049 +23050 POINT(41.42525090790004 73.12331801126658) bank23050 +23051 POINT(40.3399049967578 74.13833414222682) bank23051 +23052 POINT(41.37074041150049 73.03379734790549) bank23052 +23053 POINT(41.16910876544131 74.16292282247863) bank23053 +23054 POINT(40.07689200371937 74.17917740028777) bank23054 +23055 POINT(40.381156301077716 74.61064440816988) bank23055 +23056 POINT(41.498300406668854 74.47202421485473) bank23056 +23057 POINT(39.91609103020239 73.8206314180229) bank23057 +23058 POINT(40.1384300441097 74.62804891886401) bank23058 +23059 POINT(40.90541948771854 73.69709226355357) bank23059 +23060 POINT(41.17297168807889 74.3910675669238) bank23060 +23061 POINT(40.60492602778916 74.43126683491025) bank23061 +23062 POINT(40.387102563100484 73.53907744626014) bank23062 +23063 POINT(39.98328121197652 74.77540065775116) bank23063 +23064 POINT(39.86182309839328 73.6791546550426) bank23064 +23065 POINT(40.41467330465389 73.63309086171873) bank23065 +23066 POINT(40.40891842461565 73.47130647446048) bank23066 +23067 POINT(41.234791458022336 74.86818480949343) bank23067 +23068 POINT(41.373934622997496 74.34872426479828) bank23068 +23069 POINT(41.205829203966005 73.84848634514833) bank23069 +23070 POINT(41.02518376882377 74.32554970809045) bank23070 +23071 POINT(41.53293727090495 74.79966427454333) bank23071 +23072 POINT(41.58144111436516 73.43039552256647) bank23072 +23073 POINT(40.395446551501905 73.36923513254148) bank23073 +23074 POINT(40.72072242360562 73.56464459529424) bank23074 +23075 POINT(40.5555832065321 74.01331786712781) bank23075 +23076 POINT(41.31323736563973 73.31641948557889) bank23076 +23077 POINT(40.50510248547852 73.80382240008225) bank23077 +23078 POINT(39.88325551224328 73.48982086732187) bank23078 +23079 POINT(41.15736877405987 73.22040572791045) bank23079 +23080 POINT(39.98006756644628 74.03653274959343) bank23080 +23081 POINT(39.8166645974341 73.92950506017368) bank23081 +23082 POINT(41.10639838284971 73.4402978015797) bank23082 +23083 POINT(39.75874170849233 74.5295495360342) bank23083 +23084 POINT(40.031248095812245 74.24100426283138) bank23084 +23085 POINT(41.41239341531528 74.49323403059054) bank23085 +23086 POINT(39.71905433628107 73.73366863024961) bank23086 +23087 POINT(40.97743535499704 74.1619719670103) bank23087 +23088 POINT(39.97571852759532 74.08219681408471) bank23088 +23089 POINT(40.501559612835536 73.12345859616639) bank23089 +23090 POINT(41.11120629199387 73.05379797113059) bank23090 +23091 POINT(41.28882812184596 74.6129779508943) bank23091 +23092 POINT(40.52413801826067 74.21310208668983) bank23092 +23093 POINT(41.28215921133837 73.47807177195926) bank23093 +23094 POINT(41.04847844536367 73.056992534493) bank23094 +23095 POINT(40.326785737419684 73.97533649576447) bank23095 +23096 POINT(40.819199555778205 74.98451144456466) bank23096 +23097 POINT(40.59201825848643 74.20974576773895) bank23097 +23098 POINT(40.671598753516605 74.46842779071234) bank23098 +23099 POINT(40.67468779585686 74.66723787319138) bank23099 +23100 POINT(41.16984643293582 74.51494899140098) bank23100 +23101 POINT(40.35215674732517 73.55897671488344) bank23101 +23102 POINT(40.967425075043316 73.5667475522815) bank23102 +23103 POINT(41.58103969075768 73.36379455636569) bank23103 +23104 POINT(40.92735858845041 74.90979814728951) bank23104 +23105 POINT(40.30544020279632 73.93521853665486) bank23105 +23106 POINT(41.58191040204086 73.6880480075753) bank23106 +23107 POINT(39.72919461546608 73.54897975964303) bank23107 +23108 POINT(40.405674477348576 73.92007189686548) bank23108 +23109 POINT(39.71771598108733 73.44899570900473) bank23109 +23110 POINT(39.97015225259537 74.81812612333776) bank23110 +23111 POINT(40.57422790190455 73.1145459923186) bank23111 +23112 POINT(39.8293536205754 73.48574793974306) bank23112 +23113 POINT(40.05186729335414 74.58781339209507) bank23113 +23114 POINT(40.51109067381586 74.43027143462714) bank23114 +23115 POINT(40.35627891486237 74.49768847598753) bank23115 +23116 POINT(39.80510854946062 74.51276785189549) bank23116 +23117 POINT(39.870157247455424 74.32462987890355) bank23117 +23118 POINT(41.6964590276386 74.68862211593476) bank23118 +23119 POINT(39.99169886526619 73.52478983172132) bank23119 +23120 POINT(40.54573526785906 74.5192287609196) bank23120 +23121 POINT(39.97681391792845 74.93329593555949) bank23121 +23122 POINT(40.1399889891848 73.85407836794107) bank23122 +23123 POINT(40.577780150056505 74.55011223966702) bank23123 +23124 POINT(39.96225055177946 74.31798060308087) bank23124 +23125 POINT(41.31564109112142 74.49666826569397) bank23125 +23126 POINT(41.321189729166264 74.84269212818887) bank23126 +23127 POINT(40.6294332879547 73.98826384847534) bank23127 +23128 POINT(40.829614909334296 73.96226031233638) bank23128 +23129 POINT(40.11641655052977 74.59612339830481) bank23129 +23130 POINT(39.824458771580765 73.45935292102904) bank23130 +23131 POINT(41.696710899378544 73.49589838103186) bank23131 +23132 POINT(39.79597483828452 73.20741171830049) bank23132 +23133 POINT(40.295782334328216 74.09530433612288) bank23133 +23134 POINT(41.241992617234665 73.53051373797125) bank23134 +23135 POINT(40.23516535142863 73.46286517387614) bank23135 +23136 POINT(40.94652156097053 74.13350712467006) bank23136 +23137 POINT(39.912752087942394 74.53142717245903) bank23137 +23138 POINT(41.559987069648486 73.45720030092167) bank23138 +23139 POINT(40.8854789625671 74.90585337526228) bank23139 +23140 POINT(39.79751971778144 74.53888554477584) bank23140 +23141 POINT(40.49329662464295 73.54598322573665) bank23141 +23142 POINT(39.800932841889974 73.60372675448086) bank23142 +23143 POINT(40.42521111430656 73.88480890117567) bank23143 +23144 POINT(40.13356349742975 74.80555872052734) bank23144 +23145 POINT(40.974154787019614 73.97208886104546) bank23145 +23146 POINT(40.57653861612978 73.52257662710697) bank23146 +23147 POINT(41.253671838525065 73.21954137886318) bank23147 +23148 POINT(40.66132708460852 74.0954000940578) bank23148 +23149 POINT(40.464204379684226 74.10833295843882) bank23149 +23150 POINT(39.909895235263534 73.96201064315181) bank23150 +23151 POINT(41.590738992803765 74.21322487637916) bank23151 +23152 POINT(40.705256514589706 74.99311385682506) bank23152 +23153 POINT(39.731754262673284 73.10587869222195) bank23153 +23154 POINT(41.05263821288604 74.80902567935989) bank23154 +23155 POINT(40.23241854443922 74.66980189142316) bank23155 +23156 POINT(40.665099962649286 73.47701683999964) bank23156 +23157 POINT(39.79406592986663 74.07350037143823) bank23157 +23158 POINT(41.67059206318093 73.11788661905409) bank23158 +23159 POINT(40.38771280606251 73.19567984239151) bank23159 +23160 POINT(40.06482511030917 74.3080033953506) bank23160 +23161 POINT(40.07176974172209 74.1642854618511) bank23161 +23162 POINT(41.162007300692 73.52180648808036) bank23162 +23163 POINT(41.22930780615356 73.23878437451087) bank23163 +23164 POINT(41.34288118653764 74.63539550660246) bank23164 +23165 POINT(40.218384794551476 73.08517252596317) bank23165 +23166 POINT(41.3911471660811 74.3861164005052) bank23166 +23167 POINT(40.22992015994511 73.34586823804098) bank23167 +23168 POINT(41.47265843204123 74.83736547461868) bank23168 +23169 POINT(40.72675483709924 73.8046883899949) bank23169 +23170 POINT(40.3032391996242 73.31154609533841) bank23170 +23171 POINT(40.29514562831699 74.67004380714343) bank23171 +23172 POINT(41.51379523087246 74.24678143664846) bank23172 +23173 POINT(41.06253059621257 73.53431041642452) bank23173 +23174 POINT(41.623936235495925 74.3445131728643) bank23174 +23175 POINT(40.51021581537498 73.28471071871449) bank23175 +23176 POINT(40.81724120411706 74.82572322579357) bank23176 +23177 POINT(40.88022862476796 74.85737519419612) bank23177 +23178 POINT(41.44164228974561 73.27009914662253) bank23178 +23179 POINT(41.412864172324504 73.11834987066709) bank23179 +23180 POINT(40.31832843283378 73.56254803469005) bank23180 +23181 POINT(40.76913381559614 73.25579281764327) bank23181 +23182 POINT(41.36243549187357 73.5239373351219) bank23182 +23183 POINT(41.04333789392506 73.3178402354518) bank23183 +23184 POINT(40.00039682020724 74.93795502802699) bank23184 +23185 POINT(41.5704942485009 73.14903766924594) bank23185 +23186 POINT(41.088976334229194 74.3744886010376) bank23186 +23187 POINT(40.50325448438121 74.00209304518499) bank23187 +23188 POINT(40.26450910827135 73.61056601536258) bank23188 +23189 POINT(39.82914977231695 74.07522914081939) bank23189 +23190 POINT(40.59956549687625 74.4383048997174) bank23190 +23191 POINT(41.693201018848264 74.29473570735641) bank23191 +23192 POINT(40.91378364449973 74.56042824394068) bank23192 +23193 POINT(40.220180707577455 73.73812338137151) bank23193 +23194 POINT(40.33597519235973 73.89371011974306) bank23194 +23195 POINT(41.036453283769866 74.34379892693146) bank23195 +23196 POINT(41.39831312013894 74.20404783322611) bank23196 +23197 POINT(40.04618182586341 73.8401492120686) bank23197 +23198 POINT(41.23815099266706 74.33479564973779) bank23198 +23199 POINT(40.85972572858758 74.42110148441212) bank23199 +23200 POINT(41.17944576382447 73.20225579972292) bank23200 +23201 POINT(40.70331177338427 74.97504150210304) bank23201 +23202 POINT(40.25871151017135 75.0042862050456) bank23202 +23203 POINT(41.6925346530183 74.38562707167334) bank23203 +23204 POINT(41.68488872651244 73.98668402699884) bank23204 +23205 POINT(40.349482428587535 73.58612177844292) bank23205 +23206 POINT(40.96156904829817 73.94969694999553) bank23206 +23207 POINT(41.62630779156646 73.85825669785821) bank23207 +23208 POINT(40.73863485521624 74.02548376106307) bank23208 +23209 POINT(40.534303149637026 73.23827850010116) bank23209 +23210 POINT(40.26795389346905 73.46501406552716) bank23210 +23211 POINT(41.41964142101763 73.31340451186378) bank23211 +23212 POINT(40.4660281296401 73.06008249955433) bank23212 +23213 POINT(39.73766966206655 74.39350917480762) bank23213 +23214 POINT(40.3518260214644 73.55050260816898) bank23214 +23215 POINT(39.97641323509481 74.05719442004805) bank23215 +23216 POINT(40.30687929210179 73.40586990395666) bank23216 +23217 POINT(41.6996515468893 74.57721833183702) bank23217 +23218 POINT(40.73454477270691 74.25133140759239) bank23218 +23219 POINT(40.485506081918594 74.83445588674437) bank23219 +23220 POINT(40.802394536110185 73.32364376139992) bank23220 +23221 POINT(41.29763366274091 74.70058405573576) bank23221 +23222 POINT(41.68801812399657 73.48789067531303) bank23222 +23223 POINT(40.6241280787381 74.81618464107) bank23223 +23224 POINT(39.861432882712826 74.18121001954299) bank23224 +23225 POINT(40.35505428763307 74.08316134714333) bank23225 +23226 POINT(40.112221381882655 73.87442832044734) bank23226 +23227 POINT(40.67002883451782 73.15695377714451) bank23227 +23228 POINT(39.974902784920886 74.64930736243433) bank23228 +23229 POINT(41.08604266371437 74.06104004398154) bank23229 +23230 POINT(40.88757572717626 73.62121527388193) bank23230 +23231 POINT(39.91010523689927 74.33865762419822) bank23231 +23232 POINT(40.812401660809165 74.28872587711898) bank23232 +23233 POINT(40.52627964293986 74.98424423843537) bank23233 +23234 POINT(41.6076820543935 74.46204688750468) bank23234 +23235 POINT(41.51162355615184 73.23317492552366) bank23235 +23236 POINT(39.98543296064529 74.17166337246275) bank23236 +23237 POINT(41.42546191579288 73.9774209191173) bank23237 +23238 POINT(41.558304052925635 73.09396390240043) bank23238 +23239 POINT(41.02490869943526 73.03348219212191) bank23239 +23240 POINT(41.648031011131806 73.40578262384786) bank23240 +23241 POINT(40.92440251430374 74.08727523912208) bank23241 +23242 POINT(40.341934710842814 73.8126782880725) bank23242 +23243 POINT(40.32978673738047 74.38563674406277) bank23243 +23244 POINT(40.26241191659094 74.39843387597836) bank23244 +23245 POINT(41.09032956642311 74.23805070240391) bank23245 +23246 POINT(40.13575091210185 73.78014294321125) bank23246 +23247 POINT(41.03623604509396 74.2640567719678) bank23247 +23248 POINT(40.892628522525804 73.83215679985088) bank23248 +23249 POINT(40.21720715824859 73.72188434291608) bank23249 +23250 POINT(40.616238433668066 74.55032090481019) bank23250 +23251 POINT(40.26607863799903 74.68895917353095) bank23251 +23252 POINT(40.01801286274404 73.98102707241628) bank23252 +23253 POINT(40.5614397071053 73.2158030076462) bank23253 +23254 POINT(40.69915005753735 73.36720588553075) bank23254 +23255 POINT(40.79994220858117 73.94045339567231) bank23255 +23256 POINT(39.78089055910074 73.52828975746391) bank23256 +23257 POINT(39.83449363909075 73.65834773079384) bank23257 +23258 POINT(40.44238395429299 73.35933057595021) bank23258 +23259 POINT(41.56881214816747 74.53437540221796) bank23259 +23260 POINT(39.90657662013295 73.62882359853892) bank23260 +23261 POINT(40.82916400873184 74.01529375933286) bank23261 +23262 POINT(40.0518385439475 73.76859249110535) bank23262 +23263 POINT(39.768464346815925 74.05941277003181) bank23263 +23264 POINT(40.234058403089676 73.82599801826096) bank23264 +23265 POINT(39.75196971466647 73.22345122467037) bank23265 +23266 POINT(41.38297452331459 73.20782529993703) bank23266 +23267 POINT(41.037229695943154 74.821742889798) bank23267 +23268 POINT(40.467868403522715 74.64061350173375) bank23268 +23269 POINT(41.037103607660896 74.15924190130623) bank23269 +23270 POINT(40.01179085844613 73.76816001514506) bank23270 +23271 POINT(40.35596684728907 74.870571984457) bank23271 +23272 POINT(41.52654825484234 73.16430883016231) bank23272 +23273 POINT(41.462303138805495 73.79627606862556) bank23273 +23274 POINT(40.96451649626563 73.56976273226134) bank23274 +23275 POINT(40.01445774160883 73.99003676860646) bank23275 +23276 POINT(39.903231173393046 74.10629771028286) bank23276 +23277 POINT(41.69887563018885 73.31551508191932) bank23277 +23278 POINT(41.271271423103066 74.12990631955374) bank23278 +23279 POINT(40.75041969742445 73.45787640668424) bank23279 +23280 POINT(40.435040964716954 74.03989094756918) bank23280 +23281 POINT(41.63623971667593 74.8361516932731) bank23281 +23282 POINT(41.21030264493353 74.49512109567709) bank23282 +23283 POINT(40.82558672304535 73.8948839450767) bank23283 +23284 POINT(40.367154620787666 74.9395034893904) bank23284 +23285 POINT(40.75738934933247 74.20466165253933) bank23285 +23286 POINT(41.56380349174371 74.94128634448995) bank23286 +23287 POINT(40.001535826658966 73.12081213485095) bank23287 +23288 POINT(39.98186134462729 74.68461709416363) bank23288 +23289 POINT(41.246088450120894 74.87934234568499) bank23289 +23290 POINT(41.073252392202136 74.04668006237482) bank23290 +23291 POINT(39.805822118955284 74.44826128152974) bank23291 +23292 POINT(41.089041179979034 73.32398727548558) bank23292 +23293 POINT(39.852185114351585 74.12819391603878) bank23293 +23294 POINT(40.66423987482146 73.1425664410027) bank23294 +23295 POINT(40.08418014078463 74.01349898038792) bank23295 +23296 POINT(40.43888792089536 74.30490115387181) bank23296 +23297 POINT(41.252418034210464 73.2841142922641) bank23297 +23298 POINT(41.216612957025056 73.61598797769294) bank23298 +23299 POINT(40.25792655304691 74.5220661887482) bank23299 +23300 POINT(41.472283398797245 74.68108287132671) bank23300 +23301 POINT(41.69032800107009 73.46915241934754) bank23301 +23302 POINT(41.39121431860196 73.23909838596245) bank23302 +23303 POINT(40.84888852267351 73.48626580194107) bank23303 +23304 POINT(41.65542925984331 73.07369848640745) bank23304 +23305 POINT(39.91868549810898 74.48381047306113) bank23305 +23306 POINT(40.272767052714144 74.70088903157594) bank23306 +23307 POINT(39.802471362790214 74.09422336022813) bank23307 +23308 POINT(40.37458581388247 74.50187706246345) bank23308 +23309 POINT(40.28971538313628 74.52675385159046) bank23309 +23310 POINT(41.69763405173629 74.2085720577224) bank23310 +23311 POINT(40.11300468625434 73.83438483058225) bank23311 +23312 POINT(39.71332912149458 74.68449976693796) bank23312 +23313 POINT(40.81312297469334 73.59670097211561) bank23313 +23314 POINT(40.62745914979706 73.5484896185699) bank23314 +23315 POINT(41.70254138372769 74.17370557264427) bank23315 +23316 POINT(41.55432535450254 73.74821966504462) bank23316 +23317 POINT(41.384890338414685 73.18975790849736) bank23317 +23318 POINT(39.93803572074275 73.39953299931334) bank23318 +23319 POINT(40.44819149871457 73.2938325819812) bank23319 +23320 POINT(40.255287511862655 74.03393584452905) bank23320 +23321 POINT(39.91491885130856 73.70343520447013) bank23321 +23322 POINT(40.651038532731526 74.35160078465933) bank23322 +23323 POINT(41.110132606124694 74.02473162929121) bank23323 +23324 POINT(41.518672570530086 73.86551225182703) bank23324 +23325 POINT(39.771232008530575 73.24715939117634) bank23325 +23326 POINT(41.52591760864259 74.02948998736896) bank23326 +23327 POINT(39.76398561447286 73.10528915612612) bank23327 +23328 POINT(41.01115660679092 74.75565191993547) bank23328 +23329 POINT(40.50186977906981 74.79149704165837) bank23329 +23330 POINT(41.112903354795684 74.88900235546778) bank23330 +23331 POINT(41.646898909784966 73.82989428907456) bank23331 +23332 POINT(41.68544560700879 73.93397693715151) bank23332 +23333 POINT(41.371442576054754 74.81330870595532) bank23333 +23334 POINT(41.6704069700977 73.99894582541629) bank23334 +23335 POINT(41.20244855171351 74.34192149986606) bank23335 +23336 POINT(40.25161609700774 74.72597353766024) bank23336 +23337 POINT(41.708097012172516 73.9474020747425) bank23337 +23338 POINT(41.25434881538854 73.98137557435983) bank23338 +23339 POINT(40.37375292142306 73.53732719049164) bank23339 +23340 POINT(41.58301971236571 73.05469144308898) bank23340 +23341 POINT(41.16612701322982 73.73002024377739) bank23341 +23342 POINT(40.91194840917872 74.09063084802358) bank23342 +23343 POINT(41.23443237333622 74.89988133585804) bank23343 +23344 POINT(39.91461969744527 74.19979506137341) bank23344 +23345 POINT(41.50144898852783 74.4092540613496) bank23345 +23346 POINT(40.76759720691738 73.42047486056464) bank23346 +23347 POINT(40.58036778765354 74.22228359629736) bank23347 +23348 POINT(40.43274949392616 73.91578594966985) bank23348 +23349 POINT(39.941447066555725 74.73869174022454) bank23349 +23350 POINT(40.28821362016515 73.40531056208258) bank23350 +23351 POINT(41.16281063225879 74.53673883517315) bank23351 +23352 POINT(40.926595496456464 73.10402719413804) bank23352 +23353 POINT(41.675323822934594 73.77253455619089) bank23353 +23354 POINT(41.356841874264916 74.33298828601684) bank23354 +23355 POINT(39.79860036114295 74.7158364161616) bank23355 +23356 POINT(40.642090406409395 74.92206265506474) bank23356 +23357 POINT(41.14652081860639 74.6044592018702) bank23357 +23358 POINT(40.15090770329342 73.85036836441448) bank23358 +23359 POINT(41.06987276863599 73.08170496424496) bank23359 +23360 POINT(41.440849947580105 73.8551833550033) bank23360 +23361 POINT(41.45178427477636 73.3730810500462) bank23361 +23362 POINT(41.451797948717726 73.34133816108064) bank23362 +23363 POINT(41.633239554760664 73.14285724841001) bank23363 +23364 POINT(40.44605431819612 73.55586937826322) bank23364 +23365 POINT(40.42154229608345 74.28279295895848) bank23365 +23366 POINT(39.91722532918401 73.98180141558676) bank23366 +23367 POINT(41.14843153417305 74.8922037203579) bank23367 +23368 POINT(41.03136527743982 73.07732291809278) bank23368 +23369 POINT(40.63964456662965 74.34781517186168) bank23369 +23370 POINT(41.36468212005553 74.70591275392053) bank23370 +23371 POINT(41.259048506379564 73.62376540972897) bank23371 +23372 POINT(39.85341859937406 73.69690657936358) bank23372 +23373 POINT(41.1080189669712 74.42384297208768) bank23373 +23374 POINT(41.205703763544996 74.09091559507804) bank23374 +23375 POINT(40.66546272524708 73.71238307699575) bank23375 +23376 POINT(41.34633558906505 74.25475256702147) bank23376 +23377 POINT(41.21353267014453 74.72019175831706) bank23377 +23378 POINT(40.76830405626287 74.38909674745226) bank23378 +23379 POINT(40.3641427165668 73.64006131587045) bank23379 +23380 POINT(40.90137582944503 73.75454823068041) bank23380 +23381 POINT(39.89769953626262 74.94947310268584) bank23381 +23382 POINT(41.647434245939934 74.35843990213701) bank23382 +23383 POINT(40.16265132116094 73.1118769907003) bank23383 +23384 POINT(41.39435353746755 73.77249149160833) bank23384 +23385 POINT(41.265985228426466 74.20054351654115) bank23385 +23386 POINT(41.382184980114886 73.08204578515839) bank23386 +23387 POINT(41.49997089909166 73.25790277816515) bank23387 +23388 POINT(40.32313790292645 74.36626150651988) bank23388 +23389 POINT(41.35100788219613 73.85084108455102) bank23389 +23390 POINT(40.24393772985082 73.68159024218548) bank23390 +23391 POINT(39.792284101912394 74.28637640779041) bank23391 +23392 POINT(41.59289441268432 74.38973209401564) bank23392 +23393 POINT(39.93191811426363 74.70367269475528) bank23393 +23394 POINT(40.906102065472666 74.91209820511556) bank23394 +23395 POINT(41.45726887066161 74.55711241530479) bank23395 +23396 POINT(41.221016050048114 73.75119877052981) bank23396 +23397 POINT(41.02768006907735 73.62677426577298) bank23397 +23398 POINT(41.48575796148797 74.9889682778416) bank23398 +23399 POINT(41.51319447585301 73.17300052715912) bank23399 +23400 POINT(41.28616482988891 73.73802835488542) bank23400 +23401 POINT(39.95652390392301 74.18383629471091) bank23401 +23402 POINT(40.565401739494774 73.24218875549818) bank23402 +23403 POINT(41.21088317466975 73.86123518238563) bank23403 +23404 POINT(41.464472248306336 74.47194645636037) bank23404 +23405 POINT(40.58941142094672 73.88028670295907) bank23405 +23406 POINT(41.33769621873754 74.37817707803364) bank23406 +23407 POINT(41.508058789160536 74.19215116957008) bank23407 +23408 POINT(40.758991770573516 73.440980974003) bank23408 +23409 POINT(40.22935611322465 73.29697981113425) bank23409 +23410 POINT(41.23388804217831 73.40747522315851) bank23410 +23411 POINT(40.98131840653105 73.48309905530394) bank23411 +23412 POINT(41.270849796519286 73.25705218956918) bank23412 +23413 POINT(41.32696869770147 73.82214734008363) bank23413 +23414 POINT(41.68609351469548 74.01342248356399) bank23414 +23415 POINT(39.992571111265406 74.75770877935157) bank23415 +23416 POINT(41.107848547390354 74.04543356567395) bank23416 +23417 POINT(40.05996502431248 74.03546656832901) bank23417 +23418 POINT(40.98014712420317 74.34382977748452) bank23418 +23419 POINT(41.0254043256816 73.03531447792041) bank23419 +23420 POINT(39.73794469633662 74.21938291984183) bank23420 +23421 POINT(39.72964859436733 74.80484831194642) bank23421 +23422 POINT(41.39631872977378 73.09617475874849) bank23422 +23423 POINT(40.49547779457391 74.33751326147532) bank23423 +23424 POINT(39.920342789799896 73.74395379079859) bank23424 +23425 POINT(41.0741290831419 74.48342920688762) bank23425 +23426 POINT(40.889952320328284 74.09057557876083) bank23426 +23427 POINT(41.53631006453965 74.89058593606295) bank23427 +23428 POINT(40.55774566881655 74.16772386230222) bank23428 +23429 POINT(41.65113272419963 73.13050425348901) bank23429 +23430 POINT(40.09330421119951 73.71211174052111) bank23430 +23431 POINT(40.21139178059874 74.27714459032845) bank23431 +23432 POINT(41.686774677015535 74.71939071323081) bank23432 +23433 POINT(40.515591488649875 73.07879129859516) bank23433 +23434 POINT(40.212477236660625 73.3557666838062) bank23434 +23435 POINT(41.13654271069317 74.0305879721571) bank23435 +23436 POINT(40.740871296433504 73.98780545926132) bank23436 +23437 POINT(40.54289040552674 73.52977693948449) bank23437 +23438 POINT(39.794804375848244 74.37862802219725) bank23438 +23439 POINT(40.648270518547314 74.81187199012925) bank23439 +23440 POINT(41.68059671967248 74.18038739689942) bank23440 +23441 POINT(41.69786112244685 73.70179675289504) bank23441 +23442 POINT(41.20937666522899 74.68815878819815) bank23442 +23443 POINT(40.962292380010005 73.53408769468555) bank23443 +23444 POINT(40.68831372498273 73.50343384093308) bank23444 +23445 POINT(40.494228141608026 74.07614543669004) bank23445 +23446 POINT(41.04638213554159 73.19728270467273) bank23446 +23447 POINT(41.63480507324051 73.39777866394843) bank23447 +23448 POINT(41.42920465712032 74.90630263610831) bank23448 +23449 POINT(40.28953155327834 73.39198042464731) bank23449 +23450 POINT(40.17242802307057 73.75157846983154) bank23450 +23451 POINT(39.93744987099644 73.98074044946253) bank23451 +23452 POINT(41.35941438070343 73.69922433392618) bank23452 +23453 POINT(40.75776527271852 74.798927360241) bank23453 +23454 POINT(40.42416680421929 73.52044912605948) bank23454 +23455 POINT(40.58661079066922 74.67270452898939) bank23455 +23456 POINT(40.15327286529258 74.65335499742457) bank23456 +23457 POINT(41.532574608294105 74.6494995414814) bank23457 +23458 POINT(40.85132267014087 73.80278354466269) bank23458 +23459 POINT(40.46647399872203 74.81887614096333) bank23459 +23460 POINT(41.344077639356605 73.51300238944296) bank23460 +23461 POINT(40.460857663030694 73.41416679340986) bank23461 +23462 POINT(40.082524483620325 74.85051060640795) bank23462 +23463 POINT(41.405483989662685 74.93862110204599) bank23463 +23464 POINT(40.47187041899112 73.26795718628595) bank23464 +23465 POINT(40.03779845786742 74.99349873632066) bank23465 +23466 POINT(40.76987522579187 73.45897111001673) bank23466 +23467 POINT(41.66808954473335 73.42490522285561) bank23467 +23468 POINT(41.15843315947894 73.02652847843471) bank23468 +23469 POINT(41.01003729982955 74.55724353611356) bank23469 +23470 POINT(40.395765106912194 74.09664510276147) bank23470 +23471 POINT(41.47400955217794 73.62435158549341) bank23471 +23472 POINT(40.819372612620455 73.31165829689631) bank23472 +23473 POINT(41.110351295111634 74.4219941535449) bank23473 +23474 POINT(40.42562013072251 74.18411209401472) bank23474 +23475 POINT(41.273433293081396 73.87783840379144) bank23475 +23476 POINT(40.6117966460567 74.49554719803805) bank23476 +23477 POINT(40.6243929156371 74.42407201280388) bank23477 +23478 POINT(41.46636583084958 74.12978135309541) bank23478 +23479 POINT(41.34460605900788 73.03343746949648) bank23479 +23480 POINT(39.77803174974818 73.4017133347689) bank23480 +23481 POINT(40.86991391974847 73.23495673308226) bank23481 +23482 POINT(41.223570128285985 74.60422726682359) bank23482 +23483 POINT(39.72215324143354 73.10157196031396) bank23483 +23484 POINT(40.93403315942141 74.55243296983863) bank23484 +23485 POINT(40.01959508451875 73.95987512124417) bank23485 +23486 POINT(40.851749728427365 73.33293570030642) bank23486 +23487 POINT(40.23730959939522 74.563997891936) bank23487 +23488 POINT(40.55395019034027 73.37499599766464) bank23488 +23489 POINT(40.78555722002011 73.58926609816095) bank23489 +23490 POINT(39.84135261516424 74.6931321060064) bank23490 +23491 POINT(40.44152521083423 73.32089682354594) bank23491 +23492 POINT(40.99739551146546 74.9810039061879) bank23492 +23493 POINT(40.37424550371089 73.39540958389532) bank23493 +23494 POINT(41.0485924538405 74.36863738032987) bank23494 +23495 POINT(39.96242765913869 74.14562315828631) bank23495 +23496 POINT(40.09614560424704 74.56989611448766) bank23496 +23497 POINT(40.422886886852076 74.15185197987418) bank23497 +23498 POINT(40.920608420456325 74.93730350046681) bank23498 +23499 POINT(39.99610240665939 74.2487183533737) bank23499 +23500 POINT(40.86268397201943 74.99404409861614) bank23500 +23501 POINT(40.88618765973616 74.07636781121025) bank23501 +23502 POINT(39.949145002021034 73.5532265483025) bank23502 +23503 POINT(41.65491317639983 74.18252436458556) bank23503 +23504 POINT(40.25268809294701 74.60895304173708) bank23504 +23505 POINT(39.85859682451413 73.20089427248651) bank23505 +23506 POINT(41.18341102035832 73.82591042737178) bank23506 +23507 POINT(40.00614599783616 75.00273199723898) bank23507 +23508 POINT(39.896359348538816 73.28161868760898) bank23508 +23509 POINT(40.727245288865994 73.64971468348499) bank23509 +23510 POINT(40.322031053052285 74.52835400067013) bank23510 +23511 POINT(40.33125545595311 73.35160849958866) bank23511 +23512 POINT(41.41838960307286 73.1646664073069) bank23512 +23513 POINT(40.92405900377079 73.8549506703624) bank23513 +23514 POINT(40.34717528616231 73.16670273079247) bank23514 +23515 POINT(40.57263479318655 73.7895786130152) bank23515 +23516 POINT(40.486740172939314 73.5045174884766) bank23516 +23517 POINT(41.67642724796106 74.39252467856662) bank23517 +23518 POINT(40.00693891551229 74.27754036736073) bank23518 +23519 POINT(39.90786084660175 73.03749599611236) bank23519 +23520 POINT(41.30016412073596 73.55730465247572) bank23520 +23521 POINT(39.774815569290574 73.1499737594886) bank23521 +23522 POINT(39.885210838257564 74.76475870762525) bank23522 +23523 POINT(41.694845369512 74.71480730251288) bank23523 +23524 POINT(39.97100291820009 74.52641497695407) bank23524 +23525 POINT(40.970400747505025 74.1577212974579) bank23525 +23526 POINT(40.749217099362745 73.09410195237453) bank23526 +23527 POINT(40.78975192772033 73.39281524870816) bank23527 +23528 POINT(39.86132740172634 74.19777149472073) bank23528 +23529 POINT(41.066789624153955 73.35409713731188) bank23529 +23530 POINT(40.66057889127239 74.00403829435609) bank23530 +23531 POINT(39.93763933004423 74.56137609629566) bank23531 +23532 POINT(40.64317454430264 74.69774369979723) bank23532 +23533 POINT(39.90967750630716 74.16662008838874) bank23533 +23534 POINT(39.9436117108198 74.25446566692862) bank23534 +23535 POINT(40.46631880567954 74.91084652763978) bank23535 +23536 POINT(40.96748402614685 73.17344613115863) bank23536 +23537 POINT(41.374819992158656 74.63661243533953) bank23537 +23538 POINT(40.95937606346084 74.72538470474892) bank23538 +23539 POINT(41.00496585182564 73.47674652864973) bank23539 +23540 POINT(39.76699213474317 74.7769223704722) bank23540 +23541 POINT(40.05681724959011 73.17281443951246) bank23541 +23542 POINT(41.542253081936146 73.3621744746197) bank23542 +23543 POINT(41.32563897045164 73.13964163016354) bank23543 +23544 POINT(41.5296998249397 73.97997720598158) bank23544 +23545 POINT(40.45455076516298 73.5895146377663) bank23545 +23546 POINT(40.07717133737137 74.60375820599133) bank23546 +23547 POINT(41.35854507859998 74.4582986100308) bank23547 +23548 POINT(40.598090350669175 73.87812980997434) bank23548 +23549 POINT(40.018036734525005 74.09680592165589) bank23549 +23550 POINT(40.61010228339076 74.27477148476851) bank23550 +23551 POINT(41.44871620707384 73.85663863664483) bank23551 +23552 POINT(40.14016019062587 73.27815791037273) bank23552 +23553 POINT(41.1716717032246 74.5176885397844) bank23553 +23554 POINT(40.1520595148307 74.35069297701277) bank23554 +23555 POINT(40.87508712484873 73.79266915457879) bank23555 +23556 POINT(40.02879494840785 74.1671622575208) bank23556 +23557 POINT(39.78744622065091 73.05636853676197) bank23557 +23558 POINT(41.092775427545504 74.80337602565109) bank23558 +23559 POINT(40.54517840544398 74.33658894701323) bank23559 +23560 POINT(41.44458088428915 73.35070889197014) bank23560 +23561 POINT(41.55841834982709 73.06256842909931) bank23561 +23562 POINT(40.79426241915439 74.9322019473715) bank23562 +23563 POINT(39.77252551091808 74.70569093391865) bank23563 +23564 POINT(40.67564281693985 73.25268910307025) bank23564 +23565 POINT(41.11603191548598 73.14340841808169) bank23565 +23566 POINT(40.30571838403788 73.90053174495095) bank23566 +23567 POINT(41.41591474328873 73.49626085968167) bank23567 +23568 POINT(40.51737114192227 73.08281915844884) bank23568 +23569 POINT(40.8629162826214 74.66878123212605) bank23569 +23570 POINT(41.433416661571336 74.71203101013803) bank23570 +23571 POINT(40.05409088376217 74.76736012155806) bank23571 +23572 POINT(41.23311940027908 74.45057483226705) bank23572 +23573 POINT(40.51220837126291 73.04568649111535) bank23573 +23574 POINT(40.30101893521163 73.69570762174136) bank23574 +23575 POINT(40.50834940185901 74.42726910057952) bank23575 +23576 POINT(41.67114616878469 74.91082716905771) bank23576 +23577 POINT(41.316277380988886 74.08109203451136) bank23577 +23578 POINT(41.386642501550966 73.71438111202346) bank23578 +23579 POINT(40.9784563353338 74.53225159288519) bank23579 +23580 POINT(40.89871433915628 73.82910296742655) bank23580 +23581 POINT(41.592243534860536 74.41521267697655) bank23581 +23582 POINT(40.11316693783993 74.95409027997974) bank23582 +23583 POINT(39.71626498036921 73.93063874305531) bank23583 +23584 POINT(40.67260484402985 74.95099119227521) bank23584 +23585 POINT(41.61531631747928 74.11216235356012) bank23585 +23586 POINT(40.48995568994171 73.85869717826203) bank23586 +23587 POINT(41.26404365796434 74.09340454897021) bank23587 +23588 POINT(40.07278236634263 74.14323546173188) bank23588 +23589 POINT(39.7146514171541 74.5615423351576) bank23589 +23590 POINT(41.15943876209331 73.44828091991035) bank23590 +23591 POINT(41.45708317495023 73.60479938922785) bank23591 +23592 POINT(39.93996099877901 73.37207339339434) bank23592 +23593 POINT(39.80891880772718 74.05741871085215) bank23593 +23594 POINT(40.14533493817211 74.13960509534989) bank23594 +23595 POINT(40.251179035353985 74.41363950939042) bank23595 +23596 POINT(40.095552708780886 73.342941124099) bank23596 +23597 POINT(40.213282910344844 74.32663244958016) bank23597 +23598 POINT(40.96965176893762 73.55354866012283) bank23598 +23599 POINT(41.30536499047071 73.67747111602938) bank23599 +23600 POINT(39.77359789648878 74.7787649826089) bank23600 +23601 POINT(40.45510739174662 74.00632572425144) bank23601 +23602 POINT(41.31336151783035 74.05015197097171) bank23602 +23603 POINT(41.006159937418644 74.67130254328814) bank23603 +23604 POINT(40.89383240322379 74.32078812520044) bank23604 +23605 POINT(39.793215061891296 74.93726231923002) bank23605 +23606 POINT(41.61178551982652 74.82450018233382) bank23606 +23607 POINT(41.71121991380704 74.99404283881731) bank23607 +23608 POINT(39.75885327746598 74.44619760711043) bank23608 +23609 POINT(41.21709499084616 74.84525615427638) bank23609 +23610 POINT(40.951340383202364 73.464475583824) bank23610 +23611 POINT(41.695095340518456 73.30558954076362) bank23611 +23612 POINT(40.370518377873296 74.06957753040584) bank23612 +23613 POINT(40.16914893084655 73.4851738806256) bank23613 +23614 POINT(40.97094576680574 74.41586091608109) bank23614 +23615 POINT(41.20371663411496 74.72366290481573) bank23615 +23616 POINT(41.62017142472954 74.41652682313567) bank23616 +23617 POINT(41.04775090761976 73.72673642685798) bank23617 +23618 POINT(41.51829214844554 73.09832854391293) bank23618 +23619 POINT(41.71076603209406 73.83077173152088) bank23619 +23620 POINT(40.208152869150915 73.4940026234738) bank23620 +23621 POINT(40.87818080735506 73.95007845193751) bank23621 +23622 POINT(40.085408025579746 74.0781426778849) bank23622 +23623 POINT(40.560726125107884 73.06107068304829) bank23623 +23624 POINT(40.15837401468255 73.80872849676273) bank23624 +23625 POINT(40.77585112087952 74.64266203775573) bank23625 +23626 POINT(40.64544410546567 73.20593714210992) bank23626 +23627 POINT(41.27770667674709 73.42435753663639) bank23627 +23628 POINT(41.654560421777205 74.27321837438983) bank23628 +23629 POINT(40.904852123445714 73.45953108955423) bank23629 +23630 POINT(41.3498116632299 73.00936584840416) bank23630 +23631 POINT(41.577981675370175 73.99385464983543) bank23631 +23632 POINT(40.075773519085814 73.78051526478458) bank23632 +23633 POINT(40.52043094024321 73.31073508274866) bank23633 +23634 POINT(40.899686866535255 73.70800529407762) bank23634 +23635 POINT(40.73371950535016 74.07192903635358) bank23635 +23636 POINT(40.11127371398498 74.87487402848544) bank23636 +23637 POINT(41.26889871902314 73.38177972679443) bank23637 +23638 POINT(40.953243798742776 73.07567725523377) bank23638 +23639 POINT(40.67441212990586 73.34294597607389) bank23639 +23640 POINT(40.88445939330827 74.63150567785556) bank23640 +23641 POINT(40.50770598879987 74.76962537891103) bank23641 +23642 POINT(40.396594437701495 73.9340713941146) bank23642 +23643 POINT(40.90904577800436 74.71738871676838) bank23643 +23644 POINT(40.70134858698539 74.17569295623689) bank23644 +23645 POINT(41.65217492618487 74.6498303056843) bank23645 +23646 POINT(40.18638765939184 74.65051917668194) bank23646 +23647 POINT(40.10328913831422 74.20164694949594) bank23647 +23648 POINT(40.7053115645758 74.04025090015374) bank23648 +23649 POINT(40.71605702022438 73.95232968996335) bank23649 +23650 POINT(41.54095319643577 73.12341637555403) bank23650 +23651 POINT(41.15638215824928 73.37477341617677) bank23651 +23652 POINT(40.023721959480085 74.80070356909404) bank23652 +23653 POINT(39.95731608491899 74.70416876365574) bank23653 +23654 POINT(39.76476273721209 73.2937876341212) bank23654 +23655 POINT(41.22886516568602 74.40133296143482) bank23655 +23656 POINT(39.74538192726031 73.68832363203829) bank23656 +23657 POINT(41.64070018832022 74.99317711895205) bank23657 +23658 POINT(41.49406698677226 74.0254038905073) bank23658 +23659 POINT(41.346665042248134 74.33064100973137) bank23659 +23660 POINT(40.74349170219238 73.75087712038255) bank23660 +23661 POINT(40.12560800734475 74.94570694459343) bank23661 +23662 POINT(40.702602850641746 73.2381103057112) bank23662 +23663 POINT(40.305380132233765 74.89652136400518) bank23663 +23664 POINT(39.979416368448 73.14038669997572) bank23664 +23665 POINT(40.36161461699022 73.53816616042063) bank23665 +23666 POINT(41.34667336195304 74.55423026119448) bank23666 +23667 POINT(41.25852163991414 74.74463958008286) bank23667 +23668 POINT(40.57710997594741 73.68243499621967) bank23668 +23669 POINT(39.86513199970246 74.93535550056161) bank23669 +23670 POINT(40.61194944834846 73.2215575605544) bank23670 +23671 POINT(40.11209236635826 74.20288810582974) bank23671 +23672 POINT(40.97597451205681 73.56815620255895) bank23672 +23673 POINT(40.44177891414144 74.99671329871934) bank23673 +23674 POINT(41.616976177812525 73.72899678914095) bank23674 +23675 POINT(41.678940652216745 73.48649652820744) bank23675 +23676 POINT(39.717778185307715 73.24402379273795) bank23676 +23677 POINT(39.74143171440626 74.55805266239486) bank23677 +23678 POINT(41.349310780301124 73.8428783735458) bank23678 +23679 POINT(40.980383140706174 73.20689889709011) bank23679 +23680 POINT(40.882764023361794 74.05046679396038) bank23680 +23681 POINT(39.72555346581288 73.69574621330382) bank23681 +23682 POINT(40.7278657770513 74.05847656371246) bank23682 +23683 POINT(40.33603943016925 73.40552479694958) bank23683 +23684 POINT(41.58158392342327 73.25579447733976) bank23684 +23685 POINT(40.627900041673136 73.1832205102123) bank23685 +23686 POINT(41.44750875178894 73.9386767927299) bank23686 +23687 POINT(39.83314953933572 73.83734895591572) bank23687 +23688 POINT(40.94845385196723 73.40883311869354) bank23688 +23689 POINT(40.699019475573145 74.15033695617946) bank23689 +23690 POINT(39.93126834002273 73.0761685457011) bank23690 +23691 POINT(40.62231134623748 74.78478513651213) bank23691 +23692 POINT(41.610148034172845 73.81104947595846) bank23692 +23693 POINT(40.62137609315602 74.02537478684728) bank23693 +23694 POINT(39.99696445966275 74.95878051729736) bank23694 +23695 POINT(40.75387909296025 74.904155289006) bank23695 +23696 POINT(41.655589726646525 73.50971225482317) bank23696 +23697 POINT(41.400210813689945 74.92143895655727) bank23697 +23698 POINT(41.434109482027296 73.20620649049717) bank23698 +23699 POINT(40.20635359914282 73.615450762966) bank23699 +23700 POINT(41.63298442285964 74.86107311003552) bank23700 +23701 POINT(41.51549368231996 73.82584424379448) bank23701 +23702 POINT(39.72125906328814 73.8286810270934) bank23702 +23703 POINT(41.01694646165682 74.6760365890844) bank23703 +23704 POINT(40.60997775482066 73.888502893255) bank23704 +23705 POINT(39.715483058055675 73.77977619661633) bank23705 +23706 POINT(40.61290551450659 73.81338742028146) bank23706 +23707 POINT(41.07834152120089 73.15317937604988) bank23707 +23708 POINT(41.65446588409004 73.53383838192667) bank23708 +23709 POINT(39.934228416275936 73.59463296272625) bank23709 +23710 POINT(40.56902527870566 73.32634667986689) bank23710 +23711 POINT(39.81354167062449 74.32624755666347) bank23711 +23712 POINT(41.67713248986118 74.11044291028013) bank23712 +23713 POINT(40.1603867947179 74.89068495369168) bank23713 +23714 POINT(40.61320795046723 73.31274683826126) bank23714 +23715 POINT(41.117874690046655 73.50990548411643) bank23715 +23716 POINT(41.32061882730605 74.33050619992339) bank23716 +23717 POINT(39.92871917266954 74.52284460563789) bank23717 +23718 POINT(41.30824286084391 74.70964872754011) bank23718 +23719 POINT(41.549948769980894 73.28209129138538) bank23719 +23720 POINT(41.49619716014849 74.4750493621777) bank23720 +23721 POINT(41.05870267050961 73.28765437194265) bank23721 +23722 POINT(40.74857642647176 74.20644756126947) bank23722 +23723 POINT(40.05031417071005 74.91222339961921) bank23723 +23724 POINT(39.89655014966813 74.27111747244496) bank23724 +23725 POINT(40.1596542488477 74.00167190481152) bank23725 +23726 POINT(40.808747801556066 74.43090088620652) bank23726 +23727 POINT(39.82044705004725 74.07083475039151) bank23727 +23728 POINT(40.11762763862157 73.40408910157532) bank23728 +23729 POINT(41.30514139728602 73.57867945433925) bank23729 +23730 POINT(40.609072028928544 74.84294141714295) bank23730 +23731 POINT(41.21962312506019 74.36467820184266) bank23731 +23732 POINT(39.86053360362377 74.93581005931543) bank23732 +23733 POINT(41.25857094435178 73.96718274629923) bank23733 +23734 POINT(40.48248901020905 73.8144845656066) bank23734 +23735 POINT(40.97819540359338 73.15952272783196) bank23735 +23736 POINT(40.93318860146416 73.40380563901354) bank23736 +23737 POINT(41.258610948397 73.62265846971098) bank23737 +23738 POINT(40.156398173909416 73.01077204666765) bank23738 +23739 POINT(39.90809073498625 75.0024782259242) bank23739 +23740 POINT(40.6688342453594 73.46855328101798) bank23740 +23741 POINT(41.64264765046519 73.38220263650345) bank23741 +23742 POINT(39.88524677708297 73.33055158472658) bank23742 +23743 POINT(40.50097721939851 74.85784507176925) bank23743 +23744 POINT(40.471806160457824 73.58263167929773) bank23744 +23745 POINT(40.07300737870841 73.27902296323828) bank23745 +23746 POINT(39.76645921454148 74.05141949370369) bank23746 +23747 POINT(40.81794312604159 74.65259349656642) bank23747 +23748 POINT(40.5181044187775 74.10497536212267) bank23748 +23749 POINT(40.20744373782992 74.51298880970684) bank23749 +23750 POINT(41.36994581006364 73.22382777302202) bank23750 +23751 POINT(41.2189513566041 74.66466675189567) bank23751 +23752 POINT(41.51718352382814 74.03127415491741) bank23752 +23753 POINT(40.359801347564996 73.5895493993826) bank23753 +23754 POINT(41.32808757981751 74.49829542393618) bank23754 +23755 POINT(40.48286604313555 74.57948022845939) bank23755 +23756 POINT(41.30466655681751 74.00792172775353) bank23756 +23757 POINT(40.7091323865396 73.31117292489938) bank23757 +23758 POINT(41.63051267851827 74.8644569496649) bank23758 +23759 POINT(41.71183693666822 73.90476146081596) bank23759 +23760 POINT(39.907689109759865 74.88479846248664) bank23760 +23761 POINT(41.60639399541001 74.93471765315695) bank23761 +23762 POINT(40.9964008844352 73.40947302618703) bank23762 +23763 POINT(40.30834305314985 74.27517674308407) bank23763 +23764 POINT(40.35313083149878 73.11846173435079) bank23764 +23765 POINT(40.12184500533893 73.89879381474331) bank23765 +23766 POINT(41.577157280166716 73.2254754080464) bank23766 +23767 POINT(41.554405940643996 73.21253218455978) bank23767 +23768 POINT(41.264090125690124 73.28100948888289) bank23768 +23769 POINT(39.84921150566046 73.60160172656316) bank23769 +23770 POINT(41.64262490836518 73.20658559548) bank23770 +23771 POINT(41.349511393463295 74.1237714811006) bank23771 +23772 POINT(40.05408998838506 73.2078990084656) bank23772 +23773 POINT(41.10953237647843 74.73957095148528) bank23773 +23774 POINT(41.18647791552527 73.78207825647124) bank23774 +23775 POINT(41.51490966318371 73.72900824974138) bank23775 +23776 POINT(39.85144892120474 73.59225922583302) bank23776 +23777 POINT(40.24230165559015 73.59364271714466) bank23777 +23778 POINT(41.44998545477266 73.52040274378317) bank23778 +23779 POINT(41.442033509467436 73.6038652118025) bank23779 +23780 POINT(40.91857388932578 74.13914146215828) bank23780 +23781 POINT(40.24951466719368 73.27916206000837) bank23781 +23782 POINT(40.54697937239601 74.92624749961246) bank23782 +23783 POINT(41.64912122335873 73.82460450079944) bank23783 +23784 POINT(40.221241814055155 73.44710788427511) bank23784 +23785 POINT(40.42890484992152 73.25907869158353) bank23785 +23786 POINT(39.8245523660518 74.59607702259595) bank23786 +23787 POINT(41.479857659467406 73.74391182787929) bank23787 +23788 POINT(39.76099307922815 73.51695646202951) bank23788 +23789 POINT(40.34568057242221 74.98159206346946) bank23789 +23790 POINT(39.79943471822939 74.97529987629774) bank23790 +23791 POINT(40.46321316369479 74.8400701411797) bank23791 +23792 POINT(40.12834192152165 74.76795160646493) bank23792 +23793 POINT(41.153166723191866 74.7836754859372) bank23793 +23794 POINT(39.94841149547974 73.87881458495622) bank23794 +23795 POINT(41.180706382257355 73.53053104169062) bank23795 +23796 POINT(40.540950304181834 74.13221084469667) bank23796 +23797 POINT(41.3167334882272 73.2886963483984) bank23797 +23798 POINT(40.92181155007234 74.30605077230953) bank23798 +23799 POINT(41.087841635927774 74.28645774350339) bank23799 +23800 POINT(41.54406832793606 74.96890376872514) bank23800 +23801 POINT(41.27955210458722 74.00437728872853) bank23801 +23802 POINT(40.14986413624236 73.30199916842504) bank23802 +23803 POINT(40.251128970245965 74.00407427043369) bank23803 +23804 POINT(41.31144071142503 73.38822115033587) bank23804 +23805 POINT(40.794930321104005 74.96667172507966) bank23805 +23806 POINT(40.4229655096443 73.1828651694801) bank23806 +23807 POINT(40.45314866895044 73.67704606474574) bank23807 +23808 POINT(41.590882289886586 73.09645072838653) bank23808 +23809 POINT(40.68099697694313 73.67575220351186) bank23809 +23810 POINT(41.3447548462158 73.58854230166095) bank23810 +23811 POINT(40.513923940205075 74.40662746272666) bank23811 +23812 POINT(40.66357877122868 74.25878574035691) bank23812 +23813 POINT(40.709019136137115 73.0850108544106) bank23813 +23814 POINT(40.45382265710045 73.63893609905922) bank23814 +23815 POINT(41.42721053465538 74.21205638405787) bank23815 +23816 POINT(40.14512368307145 73.99583542209571) bank23816 +23817 POINT(40.88378532461638 74.1332700452932) bank23817 +23818 POINT(40.872027448681685 74.71292530968087) bank23818 +23819 POINT(41.27131359731023 74.0123776904762) bank23819 +23820 POINT(40.92274873038057 73.15885273439487) bank23820 +23821 POINT(40.94368477343162 74.45950755109244) bank23821 +23822 POINT(40.5667720414459 74.9851201080641) bank23822 +23823 POINT(39.76128421035577 73.76989182870373) bank23823 +23824 POINT(40.555879848593726 74.50231593643544) bank23824 +23825 POINT(39.86591294678243 73.13834884183213) bank23825 +23826 POINT(40.24518676150376 74.14565523133767) bank23826 +23827 POINT(40.06996180471818 73.84979738662042) bank23827 +23828 POINT(40.59251380038089 73.99405366830523) bank23828 +23829 POINT(41.10479815008596 74.9734165833345) bank23829 +23830 POINT(41.491420227623976 74.51169683063142) bank23830 +23831 POINT(40.87705099720987 73.89677631274864) bank23831 +23832 POINT(40.38899421318005 73.35897514885477) bank23832 +23833 POINT(40.768012293565896 73.94730907427183) bank23833 +23834 POINT(41.398433746897204 73.80816410815865) bank23834 +23835 POINT(39.71674653918157 73.92921912255058) bank23835 +23836 POINT(41.13826804956216 74.79791021687271) bank23836 +23837 POINT(41.51735980833628 74.998802061056) bank23837 +23838 POINT(40.180233375130406 74.11399435714523) bank23838 +23839 POINT(40.28507299559697 74.9268647700342) bank23839 +23840 POINT(41.637652276350465 73.3387028601537) bank23840 +23841 POINT(41.15202766522695 73.41739984691338) bank23841 +23842 POINT(40.363591736889745 73.75487932439535) bank23842 +23843 POINT(39.8588249214505 73.88603331744635) bank23843 +23844 POINT(41.214383524039164 74.79664337523431) bank23844 +23845 POINT(40.16896990260901 74.98798743406037) bank23845 +23846 POINT(40.87244931706852 73.09580327946729) bank23846 +23847 POINT(41.020749607066364 73.02832196034593) bank23847 +23848 POINT(41.395323407176754 73.93566078011632) bank23848 +23849 POINT(40.353883737087834 74.01394993468124) bank23849 +23850 POINT(40.68535424829623 74.6565840590894) bank23850 +23851 POINT(40.914547758532244 74.2228357492461) bank23851 +23852 POINT(41.02812363100521 74.59235234232845) bank23852 +23853 POINT(40.31282594219108 74.39153847393318) bank23853 +23854 POINT(41.614924532124626 73.74775949552992) bank23854 +23855 POINT(40.19089487743648 74.7639435938011) bank23855 +23856 POINT(39.74531314151573 73.27075879921537) bank23856 +23857 POINT(40.39049640451243 73.82070953392214) bank23857 +23858 POINT(39.804813668690024 73.85345489744671) bank23858 +23859 POINT(40.269625174116946 74.94256809289638) bank23859 +23860 POINT(41.06714437773006 73.51656834869041) bank23860 +23861 POINT(40.3314713319202 74.33274226631085) bank23861 +23862 POINT(39.862545615800215 73.895001562263) bank23862 +23863 POINT(41.16525287857031 74.82882787671204) bank23863 +23864 POINT(40.25811381356577 74.55835643167758) bank23864 +23865 POINT(39.98058720218077 74.98999936841952) bank23865 +23866 POINT(41.12628612150256 73.7229741717421) bank23866 +23867 POINT(40.83531193738713 74.06588476348392) bank23867 +23868 POINT(41.17114323067029 73.65431632340528) bank23868 +23869 POINT(41.18257553718096 73.71197541170065) bank23869 +23870 POINT(40.879112732498 74.73925812102061) bank23870 +23871 POINT(40.62951251969311 73.0508306731072) bank23871 +23872 POINT(40.3895751575233 73.29906136780257) bank23872 +23873 POINT(41.182107847674175 73.24804713614725) bank23873 +23874 POINT(40.95006270001264 74.03174254789428) bank23874 +23875 POINT(40.479816830818095 73.69397048678645) bank23875 +23876 POINT(40.04656298071558 74.96325927437212) bank23876 +23877 POINT(40.04307396044749 74.49259791686053) bank23877 +23878 POINT(40.07399694031719 74.13171730327163) bank23878 +23879 POINT(40.88639383677389 74.42820493846529) bank23879 +23880 POINT(41.01756874314278 74.82097064038129) bank23880 +23881 POINT(40.41339586030021 73.09920845662329) bank23881 +23882 POINT(40.40921242633516 74.5770780252512) bank23882 +23883 POINT(40.19463423098499 73.73245002844382) bank23883 +23884 POINT(40.01590766606873 74.6418349691943) bank23884 +23885 POINT(40.265587889856505 73.85974065061924) bank23885 +23886 POINT(41.34480358169056 73.66665140505611) bank23886 +23887 POINT(39.8859599936737 73.26219839230122) bank23887 +23888 POINT(39.77623920198521 74.80890978730109) bank23888 +23889 POINT(41.17885127157957 74.31235238830085) bank23889 +23890 POINT(40.3758330460251 74.87222884437296) bank23890 +23891 POINT(41.48176247601363 73.15460323629513) bank23891 +23892 POINT(40.595014332950534 74.19119549189945) bank23892 +23893 POINT(40.57285512873098 73.40105760189071) bank23893 +23894 POINT(41.41227366841073 74.00787871374901) bank23894 +23895 POINT(40.75789506947048 74.26519251232506) bank23895 +23896 POINT(40.27303730570623 74.51877560567765) bank23896 +23897 POINT(40.73974992130874 73.87872348239742) bank23897 +23898 POINT(39.790266639956734 74.92004536708079) bank23898 +23899 POINT(40.06121050107896 73.8176033120121) bank23899 +23900 POINT(41.11868714804246 73.55712422271301) bank23900 +23901 POINT(41.40489414371384 74.94364500973278) bank23901 +23902 POINT(40.16323033977195 74.97178844226715) bank23902 +23903 POINT(40.00173407811193 73.70379927676947) bank23903 +23904 POINT(40.59232818466551 73.52592477897839) bank23904 +23905 POINT(40.45904230683986 74.58972751375438) bank23905 +23906 POINT(41.21041985749292 74.4549380467528) bank23906 +23907 POINT(39.84462437699033 74.69063815785448) bank23907 +23908 POINT(41.56078369788214 74.07072588395101) bank23908 +23909 POINT(40.59434780627864 74.7341194661286) bank23909 +23910 POINT(40.59883220227276 74.69943301518002) bank23910 +23911 POINT(40.14376308624447 73.48399877234598) bank23911 +23912 POINT(40.36411430107226 73.83102270615134) bank23912 +23913 POINT(41.058151883557 73.19484787909974) bank23913 +23914 POINT(41.60997268850331 74.35069329456742) bank23914 +23915 POINT(41.12073821609226 74.5737280632303) bank23915 +23916 POINT(40.067748013434525 73.98629963294903) bank23916 +23917 POINT(41.20757129316456 74.29706590438062) bank23917 +23918 POINT(41.56167191949783 73.30101506465328) bank23918 +23919 POINT(40.28834353111668 73.9934919778526) bank23919 +23920 POINT(41.1835615027648 74.28221368054238) bank23920 +23921 POINT(41.536524285959196 74.03658094602302) bank23921 +23922 POINT(41.19824501728823 73.99416681175822) bank23922 +23923 POINT(41.341908903318796 74.67599573643724) bank23923 +23924 POINT(40.502257923517696 74.20259970098493) bank23924 +23925 POINT(39.83009644545028 73.31709398061238) bank23925 +23926 POINT(40.70869162603692 73.55782763137313) bank23926 +23927 POINT(41.66188824204147 73.07660870318814) bank23927 +23928 POINT(39.83865818689505 73.89657286467471) bank23928 +23929 POINT(40.447431701169336 74.93980338527287) bank23929 +23930 POINT(41.39761534819944 73.56034808716775) bank23930 +23931 POINT(41.330437133945196 74.86859381081223) bank23931 +23932 POINT(40.72705761019047 73.21113020894678) bank23932 +23933 POINT(40.088149671858595 74.69586940147816) bank23933 +23934 POINT(40.37289875718304 73.92213462648454) bank23934 +23935 POINT(40.069515602659166 74.67238003202657) bank23935 +23936 POINT(41.313519829577395 73.9118980718108) bank23936 +23937 POINT(40.447952930774065 74.09159561438568) bank23937 +23938 POINT(39.956233482528226 74.59717752166927) bank23938 +23939 POINT(40.43630926554036 74.88741428026471) bank23939 +23940 POINT(40.790182571943234 73.71963656068736) bank23940 +23941 POINT(40.194798399412285 74.59784981661487) bank23941 +23942 POINT(40.177538278745764 75.00515836206858) bank23942 +23943 POINT(41.696222277810136 73.78612292383843) bank23943 +23944 POINT(41.69407580350432 73.32470566272447) bank23944 +23945 POINT(41.439414419820984 74.33867577894827) bank23945 +23946 POINT(39.87069150292535 73.0250591121702) bank23946 +23947 POINT(40.91194877912635 73.42005179372947) bank23947 +23948 POINT(39.97253726076994 73.93812157005046) bank23948 +23949 POINT(40.172797758505745 74.82581179665476) bank23949 +23950 POINT(41.594630305683744 73.23058310367486) bank23950 +23951 POINT(40.70421847312851 74.41754600643674) bank23951 +23952 POINT(41.137845840705374 73.80015052063582) bank23952 +23953 POINT(41.514799426642014 74.64315970677387) bank23953 +23954 POINT(41.696663496959 74.53944617932372) bank23954 +23955 POINT(40.87893790992358 74.77350033827832) bank23955 +23956 POINT(39.91974227450446 74.66425730819252) bank23956 +23957 POINT(40.71891454531311 73.35241172848963) bank23957 +23958 POINT(40.126614211109896 73.81275488898672) bank23958 +23959 POINT(40.60558692289696 74.97617184585087) bank23959 +23960 POINT(41.23955648761927 74.70488505737949) bank23960 +23961 POINT(41.29159235906587 74.03264200226272) bank23961 +23962 POINT(39.830286333052925 74.35369990891655) bank23962 +23963 POINT(41.660278663768864 73.38938317787152) bank23963 +23964 POINT(40.57717380484594 73.40670783181217) bank23964 +23965 POINT(40.56451088246751 73.2107290922214) bank23965 +23966 POINT(40.08361772118928 73.41418676080006) bank23966 +23967 POINT(41.142603106821355 73.5672132766236) bank23967 +23968 POINT(40.27817567946549 73.82060454816252) bank23968 +23969 POINT(40.95512878348007 73.72884258703965) bank23969 +23970 POINT(39.87492117102256 74.82967544678614) bank23970 +23971 POINT(40.97855223097603 73.45625630817977) bank23971 +23972 POINT(39.858795580177635 73.2288281179143) bank23972 +23973 POINT(41.13140935672399 73.68925628133357) bank23973 +23974 POINT(40.791123556934856 74.03933437742748) bank23974 +23975 POINT(40.81380632018016 74.1581951336718) bank23975 +23976 POINT(40.10391258707088 74.17484233582958) bank23976 +23977 POINT(40.22001924378632 74.34265051185102) bank23977 +23978 POINT(39.905616123570915 74.39289391129773) bank23978 +23979 POINT(41.04929780562463 74.6264273351349) bank23979 +23980 POINT(40.96085243926134 74.3588011300458) bank23980 +23981 POINT(41.07246815023918 74.60448631129039) bank23981 +23982 POINT(40.56370222317495 74.39211852885487) bank23982 +23983 POINT(40.701754434339286 73.55148528976478) bank23983 +23984 POINT(39.751913733307994 73.99142794343446) bank23984 +23985 POINT(41.60967400405465 73.60316354546097) bank23985 +23986 POINT(39.918848993229545 74.12427109243104) bank23986 +23987 POINT(41.12083330937178 73.41245326035656) bank23987 +23988 POINT(40.65683164089595 74.54738660740587) bank23988 +23989 POINT(40.45795554676528 73.45964197328246) bank23989 +23990 POINT(40.862451855315214 74.27367079575893) bank23990 +23991 POINT(40.87163776795615 73.96880574386326) bank23991 +23992 POINT(41.452312962056105 74.14662167479456) bank23992 +23993 POINT(40.59790211726166 74.98410573569966) bank23993 +23994 POINT(39.764817024002895 74.54292970166675) bank23994 +23995 POINT(40.56096415508062 73.75181159168139) bank23995 +23996 POINT(41.084848794571506 73.97540130081086) bank23996 +23997 POINT(41.198958997513856 73.70983502838634) bank23997 +23998 POINT(40.40012547424199 74.59467758549528) bank23998 +23999 POINT(40.54752180445895 73.7389397570223) bank23999 +24000 POINT(40.89480804243526 73.21241630126845) bank24000 +24001 POINT(40.30732592339143 74.27946180784458) bank24001 +24002 POINT(40.42136799569597 73.83875101083407) bank24002 +24003 POINT(40.137930876025806 73.90849006974298) bank24003 +24004 POINT(40.716742784872636 74.12611994955621) bank24004 +24005 POINT(39.80243809813047 74.61270464190046) bank24005 +24006 POINT(41.69597155195456 74.93699623332894) bank24006 +24007 POINT(40.003872651815435 74.52553477785908) bank24007 +24008 POINT(40.94357999028079 74.86819781842316) bank24008 +24009 POINT(39.96628114070031 73.52122715887633) bank24009 +24010 POINT(40.12798772441723 74.99031287074557) bank24010 +24011 POINT(40.41358128714173 73.4417992062947) bank24011 +24012 POINT(39.749218340042255 74.52634512908557) bank24012 +24013 POINT(40.08831619795609 73.57284718826043) bank24013 +24014 POINT(40.559940203074106 74.01148606732323) bank24014 +24015 POINT(40.34141890416561 73.24005828485264) bank24015 +24016 POINT(40.968645075734386 73.67281320267622) bank24016 +24017 POINT(41.263217612590026 74.6828414282435) bank24017 +24018 POINT(39.92691888685547 73.70197855780474) bank24018 +24019 POINT(40.6400163472657 73.20468548529722) bank24019 +24020 POINT(40.98181267313122 73.47816010349216) bank24020 +24021 POINT(41.318166042538195 73.51101231208979) bank24021 +24022 POINT(40.791685592798565 74.46444175057657) bank24022 +24023 POINT(41.60793075483491 73.0489545618033) bank24023 +24024 POINT(41.53809324698377 74.3005952134208) bank24024 +24025 POINT(41.647552725438175 73.63818518119582) bank24025 +24026 POINT(40.74010390457299 74.79084904412285) bank24026 +24027 POINT(41.095282516002534 74.91767772265575) bank24027 +24028 POINT(40.736852707597315 74.019499945519) bank24028 +24029 POINT(40.30883125952466 74.79671967029435) bank24029 +24030 POINT(41.37416488807916 73.42570567997187) bank24030 +24031 POINT(41.23590197196085 73.41660265688101) bank24031 +24032 POINT(39.71329403358624 74.19188041293835) bank24032 +24033 POINT(41.69225740721053 74.29962944744952) bank24033 +24034 POINT(41.543213467378784 74.95812174461072) bank24034 +24035 POINT(40.9258850768567 74.14256136835851) bank24035 +24036 POINT(41.43851185811622 74.53128604206267) bank24036 +24037 POINT(41.025436411323184 74.7188469212852) bank24037 +24038 POINT(40.533907121285694 74.10412750470071) bank24038 +24039 POINT(40.03215171675265 74.5134719811394) bank24039 +24040 POINT(40.51242662560363 74.08485882712529) bank24040 +24041 POINT(40.27638482560061 73.21670863073156) bank24041 +24042 POINT(41.438080918622774 74.17884085665524) bank24042 +24043 POINT(40.93299255216235 74.81084494129838) bank24043 +24044 POINT(41.691202830136184 74.07287283683883) bank24044 +24045 POINT(40.3612609335077 74.38231073406514) bank24045 +24046 POINT(41.58254623971767 73.80947973625307) bank24046 +24047 POINT(41.14411636986081 74.31861506449529) bank24047 +24048 POINT(40.09247646734854 74.94134695923687) bank24048 +24049 POINT(39.776054896630924 73.21071109109377) bank24049 +24050 POINT(39.82151580177485 73.8533270960108) bank24050 +24051 POINT(41.43572174121881 73.5995681387499) bank24051 +24052 POINT(40.97587732202446 73.36347428085887) bank24052 +24053 POINT(40.945815953920146 74.98949975693391) bank24053 +24054 POINT(41.585105124369036 74.97286570059957) bank24054 +24055 POINT(39.997201090304706 73.18957103267567) bank24055 +24056 POINT(39.920007227208416 74.3252321063513) bank24056 +24057 POINT(39.87315638681561 74.2106549191793) bank24057 +24058 POINT(40.241616227228306 74.76657856532887) bank24058 +24059 POINT(39.883830028594694 73.45123229519386) bank24059 +24060 POINT(39.73205940791804 74.78873563866972) bank24060 +24061 POINT(40.52396316647754 73.3801084657935) bank24061 +24062 POINT(40.14267615574276 73.58320372805937) bank24062 +24063 POINT(41.32822619794134 74.67238196885764) bank24063 +24064 POINT(41.66870214403789 74.22289174795544) bank24064 +24065 POINT(41.15093733734002 74.37306686970366) bank24065 +24066 POINT(40.31367087567951 74.33813268270754) bank24066 +24067 POINT(40.534675568926794 74.4283694001405) bank24067 +24068 POINT(40.70883150267641 74.83729757006994) bank24068 +24069 POINT(41.16182141547817 74.0923204189257) bank24069 +24070 POINT(39.75551160133673 73.59319609759656) bank24070 +24071 POINT(41.29712534377531 74.0812983105062) bank24071 +24072 POINT(41.61862611518687 74.90717379813768) bank24072 +24073 POINT(40.4707929526251 73.99759713510973) bank24073 +24074 POINT(41.24396137857384 73.39578872360805) bank24074 +24075 POINT(40.3167496089074 75.00515905302808) bank24075 +24076 POINT(40.37095924304688 74.01689624581486) bank24076 +24077 POINT(39.90303051028767 74.23023424828952) bank24077 +24078 POINT(39.76675795325866 74.70280886895183) bank24078 +24079 POINT(40.57685662735342 74.5763098872873) bank24079 +24080 POINT(39.837441563850255 73.05130420396051) bank24080 +24081 POINT(40.43968774880005 74.17174523162268) bank24081 +24082 POINT(40.779615463708666 74.42002691567734) bank24082 +24083 POINT(39.81947703745794 73.99917555288388) bank24083 +24084 POINT(41.68963740540626 73.5115471349902) bank24084 +24085 POINT(40.510501436402954 73.48363036032096) bank24085 +24086 POINT(40.385722013627905 73.35629472730088) bank24086 +24087 POINT(41.19647196910468 73.43650190303664) bank24087 +24088 POINT(40.43540495027096 74.54570035660979) bank24088 +24089 POINT(40.74501312269773 73.09285270215325) bank24089 +24090 POINT(41.6847329407694 73.47538788401498) bank24090 +24091 POINT(41.70558790334927 74.10566154802432) bank24091 +24092 POINT(40.663711476864634 74.3808562467453) bank24092 +24093 POINT(41.344859121276365 74.44066834804086) bank24093 +24094 POINT(40.59980198266713 73.42990286475806) bank24094 +24095 POINT(41.370717589274946 74.52157469439095) bank24095 +24096 POINT(39.744895713042624 73.03047688379606) bank24096 +24097 POINT(41.635077262825426 74.63822269382331) bank24097 +24098 POINT(40.27574064805903 73.93273001022132) bank24098 +24099 POINT(39.789268378239626 74.38502456693092) bank24099 +24100 POINT(40.51927261678144 74.36848582692966) bank24100 +24101 POINT(40.311871915043895 73.40387099781525) bank24101 +24102 POINT(40.667362829750076 74.62660566057822) bank24102 +24103 POINT(40.09167853726309 74.17871448031514) bank24103 +24104 POINT(39.87262327821061 74.78077764711637) bank24104 +24105 POINT(40.76158759222652 74.6690855932409) bank24105 +24106 POINT(41.33670075698486 74.96709680685883) bank24106 +24107 POINT(40.666773716328706 73.03899110858734) bank24107 +24108 POINT(39.95208408744056 73.55497549844667) bank24108 +24109 POINT(41.552001621630055 73.38075854549712) bank24109 +24110 POINT(40.22236244550212 73.23270154146556) bank24110 +24111 POINT(41.684714330117544 74.64891361515659) bank24111 +24112 POINT(40.05402944263068 73.44714380679484) bank24112 +24113 POINT(40.29976138494508 73.39164467868498) bank24113 +24114 POINT(40.60940849207933 73.32212564446885) bank24114 +24115 POINT(40.793189460357745 73.32000594767038) bank24115 +24116 POINT(40.96756359716954 73.69995977769116) bank24116 +24117 POINT(41.02731308405333 73.40088012072792) bank24117 +24118 POINT(41.18095128165895 73.24515024323225) bank24118 +24119 POINT(41.67213720787228 73.6885401189593) bank24119 +24120 POINT(40.403946786890806 73.93161148644378) bank24120 +24121 POINT(41.2638963830808 74.1727971940258) bank24121 +24122 POINT(40.52144579038505 74.06410504315545) bank24122 +24123 POINT(41.056427356788426 74.02814007379231) bank24123 +24124 POINT(40.17496672959022 73.57348894726852) bank24124 +24125 POINT(41.35174380956324 73.35728725730301) bank24125 +24126 POINT(40.66693536712169 73.1125445345207) bank24126 +24127 POINT(41.54205530942936 74.42602010329355) bank24127 +24128 POINT(40.34101472230587 73.96419749814986) bank24128 +24129 POINT(39.76860489760423 73.72419494946524) bank24129 +24130 POINT(41.67090547543067 73.98566671136359) bank24130 +24131 POINT(41.16660736309138 73.95619597354185) bank24131 +24132 POINT(39.85082034590014 73.95798034194881) bank24132 +24133 POINT(40.94883301190273 73.48689945305622) bank24133 +24134 POINT(41.46251578799155 73.08213894227505) bank24134 +24135 POINT(40.86977492846974 74.71051846565855) bank24135 +24136 POINT(39.72294199020982 73.3313256195441) bank24136 +24137 POINT(41.69236204234224 74.37139407168796) bank24137 +24138 POINT(39.79784969891463 73.99827992233811) bank24138 +24139 POINT(40.367869932580845 74.580965568782) bank24139 +24140 POINT(41.60292422587565 73.36130066615077) bank24140 +24141 POINT(40.44589117030811 73.86618989616197) bank24141 +24142 POINT(41.53150161797746 74.10081543051682) bank24142 +24143 POINT(40.09195806445837 73.525960340203) bank24143 +24144 POINT(40.09194075837425 74.06220581728297) bank24144 +24145 POINT(39.77398127338384 73.07052299530103) bank24145 +24146 POINT(41.02092665812873 73.6722897635297) bank24146 +24147 POINT(41.089165596987 73.32231317653975) bank24147 +24148 POINT(41.70143167975412 74.73429340719795) bank24148 +24149 POINT(41.255250778228195 74.21578644423782) bank24149 +24150 POINT(41.21466136916536 74.38268249785878) bank24150 +24151 POINT(40.44919136149449 74.7221521383393) bank24151 +24152 POINT(40.46231285135947 74.52665349976587) bank24152 +24153 POINT(41.00981850060465 73.76656399392536) bank24153 +24154 POINT(41.100872556798954 74.28897033482943) bank24154 +24155 POINT(41.350307439215086 73.78354501074513) bank24155 +24156 POINT(40.16460989025871 74.73322435195958) bank24156 +24157 POINT(40.81443442456196 74.9480809748181) bank24157 +24158 POINT(40.0536212426188 73.82439618048133) bank24158 +24159 POINT(40.161034946868774 74.43292443400944) bank24159 +24160 POINT(41.20366921179526 74.67450520932677) bank24160 +24161 POINT(41.58025961624438 74.5159473428034) bank24161 +24162 POINT(41.58871984425923 73.93804722520734) bank24162 +24163 POINT(39.89348566235017 74.2041564533163) bank24163 +24164 POINT(41.00688976367954 73.35097702290457) bank24164 +24165 POINT(40.51237012379524 73.71800856093506) bank24165 +24166 POINT(41.5991183165009 74.44120805826446) bank24166 +24167 POINT(39.90679910302306 74.47609864701069) bank24167 +24168 POINT(40.18252218601626 73.05020301046773) bank24168 +24169 POINT(40.779159964813374 73.24436784335892) bank24169 +24170 POINT(40.42581840410157 73.66734959810782) bank24170 +24171 POINT(40.82948178770335 74.67921449999642) bank24171 +24172 POINT(40.76525766271678 74.94402050700215) bank24172 +24173 POINT(40.0458975373879 74.632887551762) bank24173 +24174 POINT(41.32001056267862 74.74009779736647) bank24174 +24175 POINT(40.035662839455625 73.69599593102295) bank24175 +24176 POINT(41.669932454078186 74.8429045248952) bank24176 +24177 POINT(40.91515286783995 74.9374746254829) bank24177 +24178 POINT(40.936446076462566 73.76786639327125) bank24178 +24179 POINT(41.46072715740283 73.15552743199407) bank24179 +24180 POINT(41.27392981369346 73.13365615569815) bank24180 +24181 POINT(41.29615689264429 73.60645860855007) bank24181 +24182 POINT(41.24875709245026 74.3324980422628) bank24182 +24183 POINT(39.95522858695204 74.8420785411478) bank24183 +24184 POINT(41.573809257173394 74.83697807668732) bank24184 +24185 POINT(41.50639060255959 74.92061897419711) bank24185 +24186 POINT(40.013564784540236 73.95664050698011) bank24186 +24187 POINT(41.25570159245976 73.34483853963002) bank24187 +24188 POINT(40.45521008297977 74.14385100413423) bank24188 +24189 POINT(41.424440485513024 74.58180178001521) bank24189 +24190 POINT(41.68419618706811 74.4134126163253) bank24190 +24191 POINT(41.46855466819441 74.7093105943241) bank24191 +24192 POINT(40.057774189973735 73.19088169408113) bank24192 +24193 POINT(41.08080298871493 74.66327090086219) bank24193 +24194 POINT(41.112433385506925 73.44610272216075) bank24194 +24195 POINT(40.00923596193573 73.0917262459185) bank24195 +24196 POINT(40.641503087356135 73.01671578359127) bank24196 +24197 POINT(41.42529674061934 74.36549159225407) bank24197 +24198 POINT(40.0812554983672 73.91490512231287) bank24198 +24199 POINT(41.37591950483481 73.42339481329006) bank24199 +24200 POINT(41.603248937224706 73.11994389376683) bank24200 +24201 POINT(41.05161187308154 74.98449200616169) bank24201 +24202 POINT(39.827718568182306 74.63699223533865) bank24202 +24203 POINT(39.767624374320434 74.45123179549316) bank24203 +24204 POINT(40.88968774108702 74.59436246067037) bank24204 +24205 POINT(41.293791871898 74.04324737644859) bank24205 +24206 POINT(41.46694775987162 74.24186061729894) bank24206 +24207 POINT(41.21941396100893 73.66344610702802) bank24207 +24208 POINT(39.75304003907676 73.4014262073161) bank24208 +24209 POINT(40.473014662013064 74.13859937066654) bank24209 +24210 POINT(41.29610194473126 74.96821947779286) bank24210 +24211 POINT(40.990037546085475 73.61309899079079) bank24211 +24212 POINT(39.96924293140011 73.44977321671395) bank24212 +24213 POINT(40.70931132543254 73.16632578613756) bank24213 +24214 POINT(40.86519909315092 74.18104119214844) bank24214 +24215 POINT(41.48952910198276 74.25284785988661) bank24215 +24216 POINT(40.97948984331384 73.80242161965467) bank24216 +24217 POINT(40.45644788332019 73.89801480050119) bank24217 +24218 POINT(40.26828763781086 73.06830590204409) bank24218 +24219 POINT(40.16123355251816 74.64749576899419) bank24219 +24220 POINT(40.20996011201602 73.4671955790044) bank24220 +24221 POINT(41.61083068203981 74.68882554088533) bank24221 +24222 POINT(40.352589184433675 73.13064881338263) bank24222 +24223 POINT(41.075208195334206 73.32960230523771) bank24223 +24224 POINT(39.799434497422105 74.20094627952756) bank24224 +24225 POINT(41.66064518213122 74.43420520945632) bank24225 +24226 POINT(39.99004813168125 74.68898283874235) bank24226 +24227 POINT(39.86706718907259 73.9033844453372) bank24227 +24228 POINT(40.012531319187794 73.31597391673179) bank24228 +24229 POINT(41.32524366108656 74.898002364771) bank24229 +24230 POINT(40.97027780800736 74.24476922669143) bank24230 +24231 POINT(41.02539217244684 74.01871885409889) bank24231 +24232 POINT(40.2458966602124 73.22723182048094) bank24232 +24233 POINT(40.97058282264543 73.08141447879024) bank24233 +24234 POINT(40.17931298041956 74.67620854415584) bank24234 +24235 POINT(41.1255535182506 74.34685553537784) bank24235 +24236 POINT(39.935281194097385 74.95803497305471) bank24236 +24237 POINT(40.036948948473366 74.82809724262401) bank24237 +24238 POINT(40.065352040063615 74.49748137237397) bank24238 +24239 POINT(40.82182992290884 74.1216058942049) bank24239 +24240 POINT(39.759845241433254 73.42568940963251) bank24240 +24241 POINT(41.655587721859256 73.12763831733345) bank24241 +24242 POINT(41.54807279532658 74.50593982884064) bank24242 +24243 POINT(41.452567385313834 73.95339016908738) bank24243 +24244 POINT(40.76895841003664 74.47783399504691) bank24244 +24245 POINT(39.851157006226096 74.28167517403334) bank24245 +24246 POINT(39.89815127815262 73.89520147591743) bank24246 +24247 POINT(40.649107110408295 74.84379814858453) bank24247 +24248 POINT(41.677676789923446 73.54863863401856) bank24248 +24249 POINT(40.494787234968065 73.63984660795933) bank24249 +24250 POINT(40.894159673420745 73.7427198040438) bank24250 +24251 POINT(40.22469029443767 73.72282176029302) bank24251 +24252 POINT(39.974232216488 74.48532121848508) bank24252 +24253 POINT(40.71690059771444 74.3105631696102) bank24253 +24254 POINT(41.088672756856134 73.95072663655966) bank24254 +24255 POINT(41.084070945969806 74.86609829732475) bank24255 +24256 POINT(40.560412561902666 74.72438707847904) bank24256 +24257 POINT(39.97249947435973 74.59638484592344) bank24257 +24258 POINT(40.25855560935175 74.67479433828116) bank24258 +24259 POINT(40.71137783128533 73.64785401974929) bank24259 +24260 POINT(41.13932521250825 74.27995438603884) bank24260 +24261 POINT(41.439635662212915 73.2138149026091) bank24261 +24262 POINT(39.75089470425262 74.23456647406192) bank24262 +24263 POINT(40.236933606797926 74.76622673917971) bank24263 +24264 POINT(40.290655989994804 73.15264248003425) bank24264 +24265 POINT(41.12072119327961 74.73860336991424) bank24265 +24266 POINT(39.782007537761956 74.55310017463185) bank24266 +24267 POINT(40.68324206347315 73.93896785232705) bank24267 +24268 POINT(40.831108695839724 74.46722183258991) bank24268 +24269 POINT(41.0333014778362 74.97297791817454) bank24269 +24270 POINT(41.64312124612751 73.19801937095166) bank24270 +24271 POINT(39.789166207665346 74.40988553601922) bank24271 +24272 POINT(41.01537735728593 73.37188121336989) bank24272 +24273 POINT(40.65967181482169 73.27756248799503) bank24273 +24274 POINT(39.96681182702206 74.56589130506381) bank24274 +24275 POINT(41.32541301384664 73.28648664626992) bank24275 +24276 POINT(40.07139154773894 73.72384512308571) bank24276 +24277 POINT(41.179232619202374 73.9602539171998) bank24277 +24278 POINT(40.71242592324375 73.25630402436556) bank24278 +24279 POINT(40.65733340500467 73.74275712248684) bank24279 +24280 POINT(41.091844146972164 73.27882759471794) bank24280 +24281 POINT(40.245843370009304 74.28168640025926) bank24281 +24282 POINT(40.28517769657636 74.02409462888428) bank24282 +24283 POINT(40.38479570974835 73.41596129643555) bank24283 +24284 POINT(39.92154435013323 73.8512524809441) bank24284 +24285 POINT(41.304255294469066 74.96634498203026) bank24285 +24286 POINT(41.30237910573583 73.36635232550127) bank24286 +24287 POINT(39.77301683077521 74.27243792991436) bank24287 +24288 POINT(40.09512099236651 73.72148461806988) bank24288 +24289 POINT(41.1502047288136 74.7171696097232) bank24289 +24290 POINT(41.031505296898594 74.72731797773265) bank24290 +24291 POINT(41.64234609233241 73.68840429663129) bank24291 +24292 POINT(40.04800259665869 73.64731265895449) bank24292 +24293 POINT(41.61432895811089 73.4909387834222) bank24293 +24294 POINT(40.91392416173127 73.39398595202458) bank24294 +24295 POINT(40.893605706959946 74.97489522282591) bank24295 +24296 POINT(40.111517231697526 73.78074289370451) bank24296 +24297 POINT(40.911352399511024 74.21664716198457) bank24297 +24298 POINT(40.72844042022226 73.09073339031748) bank24298 +24299 POINT(41.55713975976063 74.94174107328438) bank24299 +24300 POINT(40.58532364233047 74.99767802465212) bank24300 +24301 POINT(39.868027445318326 73.66489701827383) bank24301 +24302 POINT(41.60888358449049 73.90878814924012) bank24302 +24303 POINT(39.75539182800059 74.30312613446561) bank24303 +24304 POINT(41.172591647991425 74.80581541697934) bank24304 +24305 POINT(40.72452600920426 73.46048494140268) bank24305 +24306 POINT(40.31522307234215 73.8743391783085) bank24306 +24307 POINT(39.80326127245192 73.9494367015914) bank24307 +24308 POINT(40.56441108842558 73.02003336593918) bank24308 +24309 POINT(41.11131893862256 73.45723691632135) bank24309 +24310 POINT(40.655193536501834 74.07652161608394) bank24310 +24311 POINT(41.68279965067795 74.51262718054275) bank24311 +24312 POINT(40.63632096253039 73.03081541343073) bank24312 +24313 POINT(41.484998929056246 73.16189972856024) bank24313 +24314 POINT(40.01913130226174 73.77626117249625) bank24314 +24315 POINT(40.43276012846731 73.53947735177904) bank24315 +24316 POINT(40.286964899198466 73.86615850712343) bank24316 +24317 POINT(41.70291259149506 74.46332819807611) bank24317 +24318 POINT(41.24840823604524 73.63164708830658) bank24318 +24319 POINT(40.80636732533091 73.16349729126595) bank24319 +24320 POINT(41.099171847345545 73.04714535856215) bank24320 +24321 POINT(40.938292416926195 73.27913969871449) bank24321 +24322 POINT(41.47276701171715 74.4460045947879) bank24322 +24323 POINT(40.57959824713899 74.00540022450795) bank24323 +24324 POINT(40.60125900736087 73.03152331964597) bank24324 +24325 POINT(39.980619834880024 73.65568131068949) bank24325 +24326 POINT(40.17121024600099 74.18325021001782) bank24326 +24327 POINT(40.519688774019734 74.45330232708314) bank24327 +24328 POINT(41.63279577267822 74.55898879230429) bank24328 +24329 POINT(41.66484172058182 74.14206987631161) bank24329 +24330 POINT(41.578041144343004 73.42419197410356) bank24330 +24331 POINT(41.49602101369825 73.83104054714137) bank24331 +24332 POINT(40.49906531273978 74.96227170575857) bank24332 +24333 POINT(39.82168332458741 74.47503009246627) bank24333 +24334 POINT(41.64868445339552 74.84253605990605) bank24334 +24335 POINT(41.37823424332362 74.06629356641386) bank24335 +24336 POINT(40.767283479641904 74.16951565138966) bank24336 +24337 POINT(41.23721350528034 75.00481006039645) bank24337 +24338 POINT(39.982145240015406 74.36148841347186) bank24338 +24339 POINT(39.85489599416702 74.46710407618474) bank24339 +24340 POINT(41.18642948204668 74.50549152549047) bank24340 +24341 POINT(40.10354375062534 73.13645997403117) bank24341 +24342 POINT(39.99072537038525 73.78206936375767) bank24342 +24343 POINT(41.156345036388934 74.27354066952381) bank24343 +24344 POINT(41.35421936482954 74.27488764916525) bank24344 +24345 POINT(40.973518583617 74.76212048835993) bank24345 +24346 POINT(39.90051620075785 74.91203153199699) bank24346 +24347 POINT(40.09438541496505 73.49590278860599) bank24347 +24348 POINT(40.566871394651926 73.49389090872769) bank24348 +24349 POINT(41.367021583971265 73.984778493312) bank24349 +24350 POINT(40.34081293282049 73.91967484007506) bank24350 +24351 POINT(41.37044944553997 73.28063452534589) bank24351 +24352 POINT(41.62576592337822 74.8483403992697) bank24352 +24353 POINT(41.540949533023785 73.34474644496731) bank24353 +24354 POINT(39.72126904959748 73.54378522300811) bank24354 +24355 POINT(40.43297730351553 73.75204903808792) bank24355 +24356 POINT(41.27608132603953 74.2690479477883) bank24356 +24357 POINT(41.300602397805505 73.47335369467277) bank24357 +24358 POINT(40.30041404248261 75.00575055354761) bank24358 +24359 POINT(40.31481484305247 74.03572152255748) bank24359 +24360 POINT(39.83807043915332 74.59526213053674) bank24360 +24361 POINT(41.651095113817306 74.92778584415946) bank24361 +24362 POINT(39.88711603288524 74.82639352865785) bank24362 +24363 POINT(40.087623393538635 74.90966767862348) bank24363 +24364 POINT(41.07306966520365 73.95599017666169) bank24364 +24365 POINT(41.546514788004615 74.45996432396066) bank24365 +24366 POINT(40.69144906013362 74.05149655400393) bank24366 +24367 POINT(39.976091030573414 73.22289621924268) bank24367 +24368 POINT(41.36253785356513 73.1598572667556) bank24368 +24369 POINT(40.703677075939524 73.55698142991696) bank24369 +24370 POINT(39.782183897316685 73.0890309387769) bank24370 +24371 POINT(40.59440874580959 74.9974360212765) bank24371 +24372 POINT(40.32776908056821 74.95134558589082) bank24372 +24373 POINT(39.984105899238706 73.38527330167491) bank24373 +24374 POINT(39.92289463994477 73.30073943264118) bank24374 +24375 POINT(40.41270974224685 73.06439650854645) bank24375 +24376 POINT(40.637208656583866 74.99088822402317) bank24376 +24377 POINT(41.14828773034623 73.93281599803709) bank24377 +24378 POINT(41.306634855620906 74.87203781146063) bank24378 +24379 POINT(40.17139012047731 73.57455710614914) bank24379 +24380 POINT(39.90817828910847 74.04307853655786) bank24380 +24381 POINT(41.529981827072966 74.14857177198903) bank24381 +24382 POINT(41.40744419903142 73.14368155926489) bank24382 +24383 POINT(40.11775590868351 74.06576921289775) bank24383 +24384 POINT(41.0412491250294 74.28643429965852) bank24384 +24385 POINT(40.92430101254919 73.35018244672678) bank24385 +24386 POINT(40.26807777333359 73.80131131578942) bank24386 +24387 POINT(41.47512013420044 73.14107964122195) bank24387 +24388 POINT(40.584691698028635 74.69247302374771) bank24388 +24389 POINT(40.72956766824937 74.65342742593887) bank24389 +24390 POINT(40.12477999040123 74.88938568831192) bank24390 +24391 POINT(40.26993993735761 73.6333480696802) bank24391 +24392 POINT(40.537968293193735 74.87552822235693) bank24392 +24393 POINT(41.08392424211303 73.41425296896185) bank24393 +24394 POINT(41.464504186888036 74.64433126869245) bank24394 +24395 POINT(41.676109092806875 74.28237508345367) bank24395 +24396 POINT(40.93274112704017 74.19876804540446) bank24396 +24397 POINT(39.98078228428703 73.25357169948313) bank24397 +24398 POINT(41.2370266290531 73.63155777411964) bank24398 +24399 POINT(39.9559042060994 73.01216405679178) bank24399 +24400 POINT(40.323552601998024 73.41599805473622) bank24400 +24401 POINT(41.599489052225515 74.85958741595702) bank24401 +24402 POINT(41.21026686934731 73.59955068954041) bank24402 +24403 POINT(40.982329037250246 74.69990230242611) bank24403 +24404 POINT(39.834336829270356 74.24921335972138) bank24404 +24405 POINT(40.64861619618361 73.49582633079363) bank24405 +24406 POINT(40.4020779609052 73.81907406200193) bank24406 +24407 POINT(40.63275763205119 73.01308580118398) bank24407 +24408 POINT(40.62630077946726 73.65417256659899) bank24408 +24409 POINT(39.75729002318893 73.01294706827376) bank24409 +24410 POINT(40.93385340275703 73.56739205803761) bank24410 +24411 POINT(40.45211101595004 73.18435070676584) bank24411 +24412 POINT(40.5131300876887 73.17438808559288) bank24412 +24413 POINT(40.38838092629743 73.57291849960424) bank24413 +24414 POINT(40.98109701334047 74.66141141144413) bank24414 +24415 POINT(39.970132770851166 73.47131376711539) bank24415 +24416 POINT(40.086264313843024 73.94047923549333) bank24416 +24417 POINT(41.26103674788725 74.60162805794576) bank24417 +24418 POINT(40.3846726966413 73.48662595080746) bank24418 +24419 POINT(40.188821807116895 74.32216784703716) bank24419 +24420 POINT(40.91622085540506 74.66879255111557) bank24420 +24421 POINT(40.7352914408664 74.4535525649061) bank24421 +24422 POINT(39.91592242045574 73.51935698695439) bank24422 +24423 POINT(40.77138435852392 73.63619921315959) bank24423 +24424 POINT(41.63492872746664 73.21244237617515) bank24424 +24425 POINT(41.10849439584429 73.8681962981758) bank24425 +24426 POINT(41.55657714374819 73.19366892295805) bank24426 +24427 POINT(41.61655338767899 74.97071233418481) bank24427 +24428 POINT(40.61576980749907 74.70099944612475) bank24428 +24429 POINT(40.517064284347256 73.38948353507698) bank24429 +24430 POINT(40.70291099622022 73.53317006059467) bank24430 +24431 POINT(39.72439085267351 73.85029918819579) bank24431 +24432 POINT(40.75980065546484 73.88431597728638) bank24432 +24433 POINT(40.47690301117417 73.81394353309457) bank24433 +24434 POINT(40.449782882721536 74.04673707836369) bank24434 +24435 POINT(39.823226620387636 73.25543405674355) bank24435 +24436 POINT(39.99624671541569 74.9442821052437) bank24436 +24437 POINT(39.71704754665317 74.52084659870484) bank24437 +24438 POINT(39.96323782720616 74.83219759014443) bank24438 +24439 POINT(40.8523942484367 74.8638879208804) bank24439 +24440 POINT(41.50917803385395 73.50558936676256) bank24440 +24441 POINT(41.36271357024461 73.52538717330765) bank24441 +24442 POINT(40.84351269528307 73.059541755505) bank24442 +24443 POINT(40.739998631789355 74.02615144115752) bank24443 +24444 POINT(40.565484799633445 74.55975149715789) bank24444 +24445 POINT(39.804778541592 73.56112021285459) bank24445 +24446 POINT(39.99180164491665 74.22189336260656) bank24446 +24447 POINT(39.9336317697373 73.13929119135682) bank24447 +24448 POINT(40.43076968983527 73.16306103861201) bank24448 +24449 POINT(40.82788562699431 74.86467810950627) bank24449 +24450 POINT(40.27200879347117 74.42936949206269) bank24450 +24451 POINT(40.55806480464269 73.73424445202541) bank24451 +24452 POINT(41.68502672910431 73.01380581227895) bank24452 +24453 POINT(40.60475132171956 73.15395938246338) bank24453 +24454 POINT(40.56544820575582 73.67282831771853) bank24454 +24455 POINT(41.52732292622083 73.75927701659809) bank24455 +24456 POINT(40.66925322651286 74.82864691720596) bank24456 +24457 POINT(40.107827096898035 74.00371969399217) bank24457 +24458 POINT(39.978849542152304 74.57998309466284) bank24458 +24459 POINT(40.254357732836766 74.58076481324088) bank24459 +24460 POINT(40.746968982051726 74.17582925501344) bank24460 +24461 POINT(40.83146167679018 73.42013208935656) bank24461 +24462 POINT(40.607954161614536 74.13172787757529) bank24462 +24463 POINT(40.0096669969806 73.55404883700034) bank24463 +24464 POINT(40.310578148918275 74.10293632923481) bank24464 +24465 POINT(41.24575411034132 73.46861329329572) bank24465 +24466 POINT(41.00653973497608 73.20528848826778) bank24466 +24467 POINT(40.72124731287246 73.95993889545211) bank24467 +24468 POINT(41.09826667828937 73.29882584781899) bank24468 +24469 POINT(40.86760640757654 73.27718613870614) bank24469 +24470 POINT(41.10332079356369 74.4129663167544) bank24470 +24471 POINT(41.48099976184892 73.27916532111028) bank24471 +24472 POINT(39.99133437671277 73.61392271946576) bank24472 +24473 POINT(39.803525040467505 73.17159081997167) bank24473 +24474 POINT(41.09705196059369 74.78767413315359) bank24474 +24475 POINT(41.58326379450192 74.89841448419202) bank24475 +24476 POINT(40.02082157572893 73.53804323522024) bank24476 +24477 POINT(40.95972297783597 74.50760665901511) bank24477 +24478 POINT(41.526315335067316 74.98128164554821) bank24478 +24479 POINT(41.575581283462675 74.71677238794257) bank24479 +24480 POINT(41.48201897627338 73.09605899811814) bank24480 +24481 POINT(40.456373628528276 73.9870325808196) bank24481 +24482 POINT(40.37248403250921 73.29535363603156) bank24482 +24483 POINT(41.06607292763612 74.26640724470796) bank24483 +24484 POINT(40.27913971037651 73.29239802672733) bank24484 +24485 POINT(40.95505779873384 73.13603626743814) bank24485 +24486 POINT(40.48803944886252 74.92731466707947) bank24486 +24487 POINT(41.61259728216113 74.87192240753366) bank24487 +24488 POINT(39.98767816036553 73.21558070321473) bank24488 +24489 POINT(41.708849478985414 74.77141746080139) bank24489 +24490 POINT(39.84528592097875 74.08120386711823) bank24490 +24491 POINT(40.9480036519387 74.0446134892687) bank24491 +24492 POINT(41.62309625395478 74.16916780669388) bank24492 +24493 POINT(39.99094532224546 74.15114528966134) bank24493 +24494 POINT(39.90614999285606 73.69712991543697) bank24494 +24495 POINT(40.6774405178592 73.95616064157004) bank24495 +24496 POINT(41.44393051315372 73.17996761924549) bank24496 +24497 POINT(41.114018218825834 74.65465341386444) bank24497 +24498 POINT(40.08497829383738 73.43088653568385) bank24498 +24499 POINT(39.728675352729226 73.31812030329165) bank24499 +24500 POINT(41.55529815190044 74.06859575999825) bank24500 +24501 POINT(40.29165456281315 73.5593081820979) bank24501 +24502 POINT(40.651769962468066 73.29982803353437) bank24502 +24503 POINT(40.24856790086201 73.74681497793176) bank24503 +24504 POINT(41.44443090886347 74.08876592303638) bank24504 +24505 POINT(41.13893503047531 74.63974944123909) bank24505 +24506 POINT(40.96402055692665 73.61227370742037) bank24506 +24507 POINT(40.509697707334034 74.37397117873212) bank24507 +24508 POINT(40.860186154994246 73.26755901716338) bank24508 +24509 POINT(39.91592863928228 73.15857597783804) bank24509 +24510 POINT(41.527110515315435 74.02764822552113) bank24510 +24511 POINT(41.42295291960826 73.13731495559068) bank24511 +24512 POINT(39.721357216240726 73.91292481818543) bank24512 +24513 POINT(40.29102113245279 73.99372760989594) bank24513 +24514 POINT(40.08303951701651 74.33123589467014) bank24514 +24515 POINT(40.88483784984471 74.09423116435755) bank24515 +24516 POINT(40.01775146034168 73.15034631107476) bank24516 +24517 POINT(41.689085019600306 74.42110908869643) bank24517 +24518 POINT(40.1091867727797 74.8859223277972) bank24518 +24519 POINT(40.26291521876597 73.26491039897107) bank24519 +24520 POINT(40.08210316287258 73.22893379525155) bank24520 +24521 POINT(40.72001889085857 73.02072096852763) bank24521 +24522 POINT(41.7116855606564 73.67229630171339) bank24522 +24523 POINT(41.42169601281771 74.01797699128997) bank24523 +24524 POINT(41.1628264880972 73.11613227700089) bank24524 +24525 POINT(39.93655537191102 74.13945255032559) bank24525 +24526 POINT(40.788575051117064 74.22642664740869) bank24526 +24527 POINT(40.288893286456016 74.01186339138827) bank24527 +24528 POINT(40.07154729865665 74.35080305440405) bank24528 +24529 POINT(40.457574308814124 74.01173123718948) bank24529 +24530 POINT(41.15931832951607 74.88838055736416) bank24530 +24531 POINT(41.294355122959395 73.35639807018053) bank24531 +24532 POINT(41.605631072225314 74.15356097702445) bank24532 +24533 POINT(41.47651791704797 74.64528408925439) bank24533 +24534 POINT(41.332320078925136 74.44981623362396) bank24534 +24535 POINT(39.982527059131165 73.82137163026583) bank24535 +24536 POINT(40.85008866739617 73.21772031784235) bank24536 +24537 POINT(41.66926512343594 74.3696854577603) bank24537 +24538 POINT(40.67122031264477 73.99483464238483) bank24538 +24539 POINT(41.18087295275117 73.66347538912399) bank24539 +24540 POINT(39.966768383555504 74.53631208610936) bank24540 +24541 POINT(41.578147066344265 73.29049307203547) bank24541 +24542 POINT(41.612488919197894 74.2758696479442) bank24542 +24543 POINT(40.70607401001204 74.25596615214121) bank24543 +24544 POINT(39.877475724736904 73.48141391162802) bank24544 +24545 POINT(41.61736595660887 74.3765893120168) bank24545 +24546 POINT(40.207458898405456 74.58331730642584) bank24546 +24547 POINT(40.746614574775 74.0513116973133) bank24547 +24548 POINT(41.46075066651836 74.80231601224285) bank24548 +24549 POINT(41.18044537988203 74.57928959852995) bank24549 +24550 POINT(40.881289825653646 73.08272312320923) bank24550 +24551 POINT(41.12102379862928 74.66039095895272) bank24551 +24552 POINT(41.20278562543992 73.4024859350849) bank24552 +24553 POINT(40.12582074147795 73.87912556638528) bank24553 +24554 POINT(40.18567234954686 73.89918275298832) bank24554 +24555 POINT(40.43734873117933 74.2287288581019) bank24555 +24556 POINT(39.84600831208657 73.9362259109267) bank24556 +24557 POINT(41.48459108357166 74.55727920345382) bank24557 +24558 POINT(41.05685612125498 73.8830534890661) bank24558 +24559 POINT(40.939682770092745 73.41254355080865) bank24559 +24560 POINT(40.656160571878864 74.68401893469671) bank24560 +24561 POINT(40.99345252245063 73.30053390689113) bank24561 +24562 POINT(41.17503650088366 73.11905180094328) bank24562 +24563 POINT(41.03123157364272 73.87570866579152) bank24563 +24564 POINT(41.47090842791687 74.56379575954679) bank24564 +24565 POINT(39.78454472497582 74.3500172558603) bank24565 +24566 POINT(40.68331160877839 73.02120653006287) bank24566 +24567 POINT(41.06498201045681 73.75271019743232) bank24567 +24568 POINT(40.854075000721004 73.06081042152614) bank24568 +24569 POINT(41.32189120517312 73.03682516562266) bank24569 +24570 POINT(41.15241915355061 74.11294155025263) bank24570 +24571 POINT(40.93034971035372 73.64849905455593) bank24571 +24572 POINT(40.769058875344314 74.78004321181872) bank24572 +24573 POINT(41.190111459490105 74.62352074761006) bank24573 +24574 POINT(40.5705010460355 74.9240497039643) bank24574 +24575 POINT(40.756653048704614 74.63176320622057) bank24575 +24576 POINT(41.00165741087595 73.31370460528089) bank24576 +24577 POINT(41.647382633358376 73.82872421936422) bank24577 +24578 POINT(40.026355549197795 73.82595061354104) bank24578 +24579 POINT(39.88165141916876 74.0449595103526) bank24579 +24580 POINT(40.10290053495576 73.51801527712081) bank24580 +24581 POINT(40.52924009230642 74.2312350253783) bank24581 +24582 POINT(41.419866070633965 73.1098620764736) bank24582 +24583 POINT(40.58912809721555 73.1694928086147) bank24583 +24584 POINT(40.5184897221364 74.30114316788749) bank24584 +24585 POINT(40.42034098504377 74.029275287243) bank24585 +24586 POINT(40.0689743736698 73.58116108174882) bank24586 +24587 POINT(40.12706471017888 73.38490062789757) bank24587 +24588 POINT(40.85364370817423 73.7875210411221) bank24588 +24589 POINT(40.87034745846518 73.34034048844573) bank24589 +24590 POINT(40.160867315558846 73.06534502280739) bank24590 +24591 POINT(41.620862039697116 74.35071106951565) bank24591 +24592 POINT(39.936734824960034 73.22396433298816) bank24592 +24593 POINT(41.529280725901266 74.68268003051097) bank24593 +24594 POINT(39.82806974188727 74.84004247043997) bank24594 +24595 POINT(41.060852702737144 73.04298756276418) bank24595 +24596 POINT(40.54427498268442 74.73528406382222) bank24596 +24597 POINT(40.43831032742 73.53143996889781) bank24597 +24598 POINT(41.53811886875354 74.27138306223607) bank24598 +24599 POINT(40.617768072155485 73.87592396274482) bank24599 +24600 POINT(40.837512247571325 73.95936380215728) bank24600 +24601 POINT(41.13385637227709 74.32163806501786) bank24601 +24602 POINT(40.08797774792909 73.22982884227412) bank24602 +24603 POINT(39.99173119749202 74.3813138141531) bank24603 +24604 POINT(40.54523595399342 73.35980563403581) bank24604 +24605 POINT(40.997021509995704 73.7381307170776) bank24605 +24606 POINT(41.66437015118625 74.67082812054072) bank24606 +24607 POINT(40.35663017384296 73.39548649467446) bank24607 +24608 POINT(39.98980231842586 73.87399963253085) bank24608 +24609 POINT(41.06300115966168 73.63071662277333) bank24609 +24610 POINT(39.736495724338965 73.59548427364851) bank24610 +24611 POINT(41.50250433069458 73.71279291739287) bank24611 +24612 POINT(41.13730781596078 74.83967978431654) bank24612 +24613 POINT(41.3303167418924 73.50542725663132) bank24613 +24614 POINT(39.72576212498773 74.74770425643023) bank24614 +24615 POINT(41.671765985356686 73.72869216653378) bank24615 +24616 POINT(40.61848376163083 73.22494873790603) bank24616 +24617 POINT(41.56453215773176 74.18913134455076) bank24617 +24618 POINT(40.08858864070837 73.67714920541673) bank24618 +24619 POINT(40.32299636542138 74.73858493322777) bank24619 +24620 POINT(41.24722940581974 74.97868539846925) bank24620 +24621 POINT(39.95707394694072 73.71175643688326) bank24621 +24622 POINT(41.39523770892249 73.89796415297117) bank24622 +24623 POINT(39.87232315104293 74.16389779108086) bank24623 +24624 POINT(39.914925512526366 73.63317886803969) bank24624 +24625 POINT(39.85168106816575 74.35806504511089) bank24625 +24626 POINT(40.81984137841981 73.4031360660318) bank24626 +24627 POINT(40.933748960996965 73.57827679879719) bank24627 +24628 POINT(41.33670693277316 74.94880401838819) bank24628 +24629 POINT(41.22669682748748 73.76978713493017) bank24629 +24630 POINT(39.9743031644102 73.66820317932056) bank24630 +24631 POINT(39.9641151831958 73.98708670699551) bank24631 +24632 POINT(41.10770863753501 74.13100729657866) bank24632 +24633 POINT(40.4085137741845 74.50199827095366) bank24633 +24634 POINT(40.684466097971836 73.63376793850448) bank24634 +24635 POINT(39.860900414673644 73.84727299427682) bank24635 +24636 POINT(40.971652974520666 74.01184816248545) bank24636 +24637 POINT(40.63179861107216 74.9413691402869) bank24637 +24638 POINT(40.31132636405955 74.3593294736186) bank24638 +24639 POINT(40.07970655377852 74.96504947541266) bank24639 +24640 POINT(41.31139927430388 73.09686206888055) bank24640 +24641 POINT(41.08541246155338 73.74226547020228) bank24641 +24642 POINT(39.7826820508989 73.4196476229997) bank24642 +24643 POINT(39.962990755602846 73.13083832122996) bank24643 +24644 POINT(39.849755157625374 74.07683377437444) bank24644 +24645 POINT(40.84806862612113 74.8560390897254) bank24645 +24646 POINT(39.73620078328713 74.50915511396575) bank24646 +24647 POINT(40.45403575584943 73.0077015498453) bank24647 +24648 POINT(41.194586324709796 74.69795269425289) bank24648 +24649 POINT(40.076379933688806 73.33391012896409) bank24649 +24650 POINT(41.0286168971808 73.15916999642533) bank24650 +24651 POINT(41.43408927081695 73.23212389526124) bank24651 +24652 POINT(40.69563058298765 73.08780583998669) bank24652 +24653 POINT(40.79175498370334 73.7436116244956) bank24653 +24654 POINT(40.4773231282544 73.32919800844041) bank24654 +24655 POINT(40.30285431418753 73.38425482033591) bank24655 +24656 POINT(39.82551834011775 74.99810846965174) bank24656 +24657 POINT(40.32430345251137 73.63475390251156) bank24657 +24658 POINT(40.27535903983441 73.63023730510585) bank24658 +24659 POINT(40.32274843975286 74.22183877579648) bank24659 +24660 POINT(40.031141090418316 73.51182441648012) bank24660 +24661 POINT(40.4900528902745 73.6858259686292) bank24661 +24662 POINT(40.037312481364275 73.75089088330506) bank24662 +24663 POINT(40.23552669949051 73.65972222998991) bank24663 +24664 POINT(39.75629405819586 74.13384828367447) bank24664 +24665 POINT(40.575378421788 73.13619013466746) bank24665 +24666 POINT(41.485493922564515 74.3576801187813) bank24666 +24667 POINT(40.3206478991992 73.70720196532055) bank24667 +24668 POINT(40.75878486939048 74.98344685773547) bank24668 +24669 POINT(41.532056900584664 74.78728441510886) bank24669 +24670 POINT(41.622168976104724 74.47628829309367) bank24670 +24671 POINT(41.011349920762186 73.33902652736248) bank24671 +24672 POINT(40.41363298012358 73.94405131515238) bank24672 +24673 POINT(40.02108295866679 74.12928681546238) bank24673 +24674 POINT(39.75465800085539 74.19658967199442) bank24674 +24675 POINT(40.01108088861071 73.3080221396165) bank24675 +24676 POINT(40.89201682129177 74.26992661289506) bank24676 +24677 POINT(39.778358455345526 74.18649595952087) bank24677 +24678 POINT(40.04372716618038 73.43713271925937) bank24678 +24679 POINT(40.05499100167656 74.39124566091932) bank24679 +24680 POINT(41.13225921651092 74.24731126653816) bank24680 +24681 POINT(40.77044397748445 73.95762406800633) bank24681 +24682 POINT(41.29182764532431 74.29476521622009) bank24682 +24683 POINT(40.73746348299757 74.69252173665652) bank24683 +24684 POINT(41.32728986086431 73.71498799470761) bank24684 +24685 POINT(40.42229371551057 74.69211121159552) bank24685 +24686 POINT(41.60274187829324 73.44253318756203) bank24686 +24687 POINT(40.01534069545563 74.30127655282018) bank24687 +24688 POINT(40.55462506361826 73.24686148408574) bank24688 +24689 POINT(41.0909323937755 73.38364181715464) bank24689 +24690 POINT(41.18163258347142 74.83459769522074) bank24690 +24691 POINT(40.57938836415466 74.1492171234129) bank24691 +24692 POINT(40.803457674914114 73.81281438854744) bank24692 +24693 POINT(39.855146366758255 74.20813273063919) bank24693 +24694 POINT(41.58275696735055 74.23742490263089) bank24694 +24695 POINT(39.92142651630475 74.25484975916788) bank24695 +24696 POINT(40.2444250240761 74.68395340424159) bank24696 +24697 POINT(39.896897772282436 74.87216332169658) bank24697 +24698 POINT(40.89324823045385 74.106086848726) bank24698 +24699 POINT(41.60133339510576 74.47032153901587) bank24699 +24700 POINT(39.87236031060953 74.65862149427) bank24700 +24701 POINT(39.82558381679045 74.00864811950115) bank24701 +24702 POINT(39.8351483727199 73.47156036735097) bank24702 +24703 POINT(40.35806628115975 74.49364650080535) bank24703 +24704 POINT(40.55826649932428 74.54703236215482) bank24704 +24705 POINT(40.14299572842617 74.62104101946964) bank24705 +24706 POINT(40.71457835251106 73.28661335660236) bank24706 +24707 POINT(40.49503517358429 75.00222552490312) bank24707 +24708 POINT(40.08970441085921 73.7951723796962) bank24708 +24709 POINT(41.34834166358483 74.30641039635557) bank24709 +24710 POINT(40.68618933033039 73.28149907544906) bank24710 +24711 POINT(41.387620482178484 73.78205352336371) bank24711 +24712 POINT(41.327665208775926 73.69185067714467) bank24712 +24713 POINT(41.530635284924216 73.63371705108491) bank24713 +24714 POINT(41.197639646682546 74.7238048091958) bank24714 +24715 POINT(40.0432140491854 74.05431761681632) bank24715 +24716 POINT(40.33028005580099 73.01403864581457) bank24716 +24717 POINT(40.572201081149935 74.65030630268579) bank24717 +24718 POINT(40.40679097477535 74.07087318092107) bank24718 +24719 POINT(41.42931716758356 74.95974248865834) bank24719 +24720 POINT(40.845551487584544 73.42523239670989) bank24720 +24721 POINT(41.23884499494788 73.97975660513991) bank24721 +24722 POINT(40.88003070908787 73.56599630896692) bank24722 +24723 POINT(40.399694702559145 73.55962336115041) bank24723 +24724 POINT(40.68503851786842 74.96124711228389) bank24724 +24725 POINT(39.96301616827104 74.57405272541692) bank24725 +24726 POINT(41.54462248104397 73.4723137843893) bank24726 +24727 POINT(40.72359737851589 74.23461471749738) bank24727 +24728 POINT(40.04815858279786 73.11429154333575) bank24728 +24729 POINT(41.63737566619 73.37098529651671) bank24729 +24730 POINT(40.66961224038603 73.17522550997393) bank24730 +24731 POINT(41.561030347089954 74.60178991807449) bank24731 +24732 POINT(39.77054566733837 74.83271004424891) bank24732 +24733 POINT(39.806265070068115 74.91033772213366) bank24733 +24734 POINT(41.18384242743133 74.112731051651) bank24734 +24735 POINT(41.20693537587012 73.43988830809398) bank24735 +24736 POINT(40.45373003956157 73.67564329369148) bank24736 +24737 POINT(41.59763390053181 74.96336519980744) bank24737 +24738 POINT(40.808418827995155 74.78527972992309) bank24738 +24739 POINT(40.40463099548125 74.78390601210758) bank24739 +24740 POINT(40.277586220764995 74.5471027667727) bank24740 +24741 POINT(41.233440595945034 74.30067774209436) bank24741 +24742 POINT(41.01649825246728 74.3831104699316) bank24742 +24743 POINT(40.0479784690056 73.06335762662813) bank24743 +24744 POINT(39.881590520913925 73.85409931436538) bank24744 +24745 POINT(41.5143258695766 74.74968375893914) bank24745 +24746 POINT(40.271697790253825 73.75862840446254) bank24746 +24747 POINT(40.841166582610526 74.91082686032408) bank24747 +24748 POINT(41.05551680442987 74.85892992390065) bank24748 +24749 POINT(40.65652800187966 73.04883585651834) bank24749 +24750 POINT(40.0760717231174 74.51226689895874) bank24750 +24751 POINT(40.377038091196496 73.02416230582864) bank24751 +24752 POINT(40.21348629443998 74.9154666664581) bank24752 +24753 POINT(39.76342165892936 74.69445872333813) bank24753 +24754 POINT(40.24659929260094 73.29844983516266) bank24754 +24755 POINT(41.281062409589964 73.79579830936292) bank24755 +24756 POINT(40.420050883710374 74.04825949818913) bank24756 +24757 POINT(39.74737382005213 73.10102807037234) bank24757 +24758 POINT(40.20490115906598 73.01365605582632) bank24758 +24759 POINT(40.456142901711786 74.96921842472105) bank24759 +24760 POINT(39.86734610492674 74.40302330830347) bank24760 +24761 POINT(40.644611342623115 73.85688656515453) bank24761 +24762 POINT(41.38951345457865 74.45306237601778) bank24762 +24763 POINT(40.44293888093126 73.09360741621988) bank24763 +24764 POINT(40.470329759627376 74.34122781022582) bank24764 +24765 POINT(39.714852121318785 73.76592011383127) bank24765 +24766 POINT(41.472941749512515 73.4623930941404) bank24766 +24767 POINT(39.77607159449087 73.33701467014657) bank24767 +24768 POINT(41.39013687048128 74.32973188753935) bank24768 +24769 POINT(39.79403499168218 73.17527779716336) bank24769 +24770 POINT(40.61615236750382 74.90403147743015) bank24770 +24771 POINT(40.53733575062238 73.77329964820163) bank24771 +24772 POINT(41.1471451044124 73.43882490180954) bank24772 +24773 POINT(41.21579978034404 74.5095502563991) bank24773 +24774 POINT(41.59329669557598 73.78539603972993) bank24774 +24775 POINT(41.394628803937245 74.20459694431322) bank24775 +24776 POINT(40.03683096722964 74.75094159111862) bank24776 +24777 POINT(40.96452056536748 73.24607263027987) bank24777 +24778 POINT(40.28442464027773 74.16090477015953) bank24778 +24779 POINT(41.16711047056416 74.55855247304358) bank24779 +24780 POINT(41.620112241797365 74.14634106015009) bank24780 +24781 POINT(40.79741366753211 73.26758990748186) bank24781 +24782 POINT(41.63069654120309 74.03872723264323) bank24782 +24783 POINT(41.31439238222008 74.4859186751781) bank24783 +24784 POINT(40.8703822134129 73.77721570612186) bank24784 +24785 POINT(40.681887368498444 74.04585727825321) bank24785 +24786 POINT(40.662056813702556 74.16426857449436) bank24786 +24787 POINT(39.76491808085897 74.94819868236051) bank24787 +24788 POINT(40.591367890765106 74.89373106522702) bank24788 +24789 POINT(40.91577910093608 74.05726260197353) bank24789 +24790 POINT(40.501439979136784 74.41520830356149) bank24790 +24791 POINT(40.60119626534595 73.25778322821168) bank24791 +24792 POINT(40.05057845223671 74.70437327840854) bank24792 +24793 POINT(39.80276623453461 73.97515687720622) bank24793 +24794 POINT(40.64071087322744 74.96248764407218) bank24794 +24795 POINT(40.75872723167852 74.75185694654166) bank24795 +24796 POINT(40.57500747076298 73.49995926684912) bank24796 +24797 POINT(39.85498175837808 74.2418968418098) bank24797 +24798 POINT(41.43712230161047 74.85600287548435) bank24798 +24799 POINT(41.264827298009905 73.50431938739779) bank24799 +24800 POINT(41.55922931273111 74.43083380819145) bank24800 +24801 POINT(40.11401852554508 73.38466047324918) bank24801 +24802 POINT(41.10076685108017 74.55421675867383) bank24802 +24803 POINT(41.088654183431444 73.34931408441986) bank24803 +24804 POINT(41.503657526693075 73.44414379977626) bank24804 +24805 POINT(41.47611712247364 73.46756914134039) bank24805 +24806 POINT(40.028171040887294 73.221810323814) bank24806 +24807 POINT(41.13260340858166 74.9609109151992) bank24807 +24808 POINT(40.19686531096586 74.87922625338588) bank24808 +24809 POINT(41.64474162614618 74.12427127875887) bank24809 +24810 POINT(41.11075299945407 73.76657444610595) bank24810 +24811 POINT(41.15848983020099 74.98422791584707) bank24811 +24812 POINT(40.50242804423976 73.20620796805686) bank24812 +24813 POINT(41.39136923861868 73.51656616130776) bank24813 +24814 POINT(41.65041757389576 74.84532791587608) bank24814 +24815 POINT(39.871727645395914 73.63760256702881) bank24815 +24816 POINT(40.05251253631561 74.7639197747684) bank24816 +24817 POINT(40.9208563517334 73.28003439347319) bank24817 +24818 POINT(40.622350325606845 73.45699204682273) bank24818 +24819 POINT(41.64245308259283 74.418916102511) bank24819 +24820 POINT(40.62775096815165 74.55922258790321) bank24820 +24821 POINT(40.06639474848921 73.39106778287206) bank24821 +24822 POINT(41.080548494179624 73.35177980120262) bank24822 +24823 POINT(40.281344922219276 74.4232889585488) bank24823 +24824 POINT(40.751615144015986 74.07399400580015) bank24824 +24825 POINT(40.964802949494214 74.37827460389732) bank24825 +24826 POINT(41.61587886572107 74.62149476233996) bank24826 +24827 POINT(41.19831754019082 74.26988075519424) bank24827 +24828 POINT(39.95427865538704 73.33488674767129) bank24828 +24829 POINT(41.580166740711455 73.4844510424599) bank24829 +24830 POINT(39.76639598073188 74.70230207076834) bank24830 +24831 POINT(40.81331803839828 74.9088804597048) bank24831 +24832 POINT(40.87555953995855 74.94083500840418) bank24832 +24833 POINT(41.48729508377208 74.023955082382) bank24833 +24834 POINT(40.98279015738537 73.5584750733613) bank24834 +24835 POINT(41.52953418060589 73.37632275726797) bank24835 +24836 POINT(40.83768288424252 73.9214244681986) bank24836 +24837 POINT(40.213663944176375 73.66164395046157) bank24837 +24838 POINT(40.5305966680277 73.67895395757934) bank24838 +24839 POINT(40.52691301917675 73.90414299311178) bank24839 +24840 POINT(40.56374702080369 74.90932618176667) bank24840 +24841 POINT(39.84536650841805 74.76676855313326) bank24841 +24842 POINT(41.64793884601699 73.99249048085102) bank24842 +24843 POINT(41.46341113546595 74.49741399073659) bank24843 +24844 POINT(39.83194034537477 73.30943720268264) bank24844 +24845 POINT(41.20022908464372 73.17416683501719) bank24845 +24846 POINT(40.840394853095574 73.85902417865833) bank24846 +24847 POINT(41.04890548301921 73.06858737867987) bank24847 +24848 POINT(41.236196693329816 73.90345018152365) bank24848 +24849 POINT(41.40523925266304 73.6501158887082) bank24849 +24850 POINT(39.96322209193538 74.17854947524) bank24850 +24851 POINT(41.29885010951278 73.55174062929918) bank24851 +24852 POINT(41.6620920640174 73.2827961866971) bank24852 +24853 POINT(40.976635997629394 74.05030555017977) bank24853 +24854 POINT(40.73329481492778 74.83831307121878) bank24854 +24855 POINT(41.583098873583616 74.16574013487943) bank24855 +24856 POINT(40.76629468063335 74.72642684850592) bank24856 +24857 POINT(40.91563772459707 73.01399840849456) bank24857 +24858 POINT(39.72145627115221 74.1008844186167) bank24858 +24859 POINT(39.765811040173155 73.30794247560732) bank24859 +24860 POINT(40.78695760245811 75.0033979532477) bank24860 +24861 POINT(41.680885953082075 73.05910325367195) bank24861 +24862 POINT(40.62963726398685 74.68866523875884) bank24862 +24863 POINT(41.54233167598854 73.39421007910238) bank24863 +24864 POINT(40.396025694036446 74.9383842952683) bank24864 +24865 POINT(40.776006454738955 73.52495062767898) bank24865 +24866 POINT(39.908124717087226 73.2867867411066) bank24866 +24867 POINT(40.57798352943048 73.59000355080191) bank24867 +24868 POINT(40.74571087119447 74.82542933387646) bank24868 +24869 POINT(39.79316892736253 73.05626637815577) bank24869 +24870 POINT(40.39918409582946 74.00700932757051) bank24870 +24871 POINT(40.49508353798551 73.70570575439656) bank24871 +24872 POINT(41.666007878560926 73.50288877699656) bank24872 +24873 POINT(39.74437258543989 74.691141636857) bank24873 +24874 POINT(40.80954725134116 73.44838433735472) bank24874 +24875 POINT(41.16778986424297 74.29230466537176) bank24875 +24876 POINT(40.87217788236385 73.88123832347348) bank24876 +24877 POINT(40.4384495962994 74.66043195622083) bank24877 +24878 POINT(40.2901036665737 73.36582939471573) bank24878 +24879 POINT(40.117352203434855 74.7826428637326) bank24879 +24880 POINT(40.56472275706282 74.38296426402111) bank24880 +24881 POINT(40.432456017419476 73.77688337366325) bank24881 +24882 POINT(40.960535186228995 74.99740059324986) bank24882 +24883 POINT(40.40879882735314 74.88332639303835) bank24883 +24884 POINT(40.713129875850434 73.07806932020459) bank24884 +24885 POINT(40.53140169459226 73.73600921460665) bank24885 +24886 POINT(39.85130210785838 74.85723079319658) bank24886 +24887 POINT(39.81750259062837 74.09243342673687) bank24887 +24888 POINT(40.95121723149673 74.40048981602833) bank24888 +24889 POINT(39.766698546062734 74.9592490961912) bank24889 +24890 POINT(41.228352940369774 74.18632255911112) bank24890 +24891 POINT(40.46175650977941 74.81936621201508) bank24891 +24892 POINT(41.06479756382399 73.68182587115912) bank24892 +24893 POINT(39.751412989673625 73.79024017142773) bank24893 +24894 POINT(40.29287936049049 74.9137360888944) bank24894 +24895 POINT(41.44041321079214 74.62789684464272) bank24895 +24896 POINT(41.1726280540246 73.39825419236631) bank24896 +24897 POINT(40.09731911850316 74.57557128796765) bank24897 +24898 POINT(40.45315664759325 74.38698293605728) bank24898 +24899 POINT(40.67638040490071 73.02320288534705) bank24899 +24900 POINT(41.59627654836012 73.04943925528922) bank24900 +24901 POINT(40.15464535829448 74.04815597054177) bank24901 +24902 POINT(41.327958417424426 73.43144621290361) bank24902 +24903 POINT(41.25823543196864 74.95275220037641) bank24903 +24904 POINT(40.1455395249425 74.61099680182453) bank24904 +24905 POINT(40.412321624954636 73.04549943345458) bank24905 +24906 POINT(40.36429768940342 74.67893033618316) bank24906 +24907 POINT(39.92810306825378 73.76531947070326) bank24907 +24908 POINT(41.27753286746506 73.84268899731106) bank24908 +24909 POINT(39.766805944282545 73.75660364486156) bank24909 +24910 POINT(41.608136629237514 74.15385192663) bank24910 +24911 POINT(40.3667641173303 74.83403248487576) bank24911 +24912 POINT(40.97178879358426 73.38575301706186) bank24912 +24913 POINT(40.371486109256935 73.71627725712888) bank24913 +24914 POINT(41.08250202925595 73.21006043905182) bank24914 +24915 POINT(41.43919673562457 74.29390189840561) bank24915 +24916 POINT(41.02831506144571 73.93497411633878) bank24916 +24917 POINT(40.69032155633172 74.12886458407941) bank24917 +24918 POINT(41.47193758442764 73.19363265811658) bank24918 +24919 POINT(40.9731754147071 73.34206719581702) bank24919 +24920 POINT(41.51223600598033 73.1200050252506) bank24920 +24921 POINT(40.884749134848654 73.91739494839483) bank24921 +24922 POINT(39.75650951263006 74.22987153478668) bank24922 +24923 POINT(39.77868701851903 73.34477352318997) bank24923 +24924 POINT(40.81500867699736 74.24670477786523) bank24924 +24925 POINT(41.69641322353609 74.99558852194934) bank24925 +24926 POINT(40.53708499313363 74.84829567135364) bank24926 +24927 POINT(40.76319570800769 73.09742145044531) bank24927 +24928 POINT(39.7612112307135 73.97041786858628) bank24928 +24929 POINT(39.73879163939914 74.4015055887468) bank24929 +24930 POINT(41.68124801078493 73.69362393490799) bank24930 +24931 POINT(40.994788976834634 73.66830509065056) bank24931 +24932 POINT(40.97329818884351 74.7898153472373) bank24932 +24933 POINT(39.755534620924486 73.60956417378739) bank24933 +24934 POINT(39.9181517339064 73.54383467158208) bank24934 +24935 POINT(40.921551973354624 73.86588666677633) bank24935 +24936 POINT(41.556776895742885 73.57856237400168) bank24936 +24937 POINT(40.07998837231545 74.89172162653482) bank24937 +24938 POINT(41.71092574949496 74.71361678112896) bank24938 +24939 POINT(40.648334700915505 73.91710674880781) bank24939 +24940 POINT(41.00678874360177 73.03590817867997) bank24940 +24941 POINT(40.2313089154501 73.89177999141445) bank24941 +24942 POINT(40.18122795949538 74.66337951844716) bank24942 +24943 POINT(40.195646388017 73.64210047527388) bank24943 +24944 POINT(40.61874641469294 74.50297463266483) bank24944 +24945 POINT(40.799667114298806 73.71098903796968) bank24945 +24946 POINT(40.64356052141373 74.49905270104806) bank24946 +24947 POINT(40.43081951122849 73.91299451218242) bank24947 +24948 POINT(41.69082218349923 74.37282229277545) bank24948 +24949 POINT(40.77717795573173 73.79119094151574) bank24949 +24950 POINT(41.23000951673915 73.18570731596813) bank24950 +24951 POINT(40.93577904593971 73.7544963786896) bank24951 +24952 POINT(39.79043960548962 73.96836245314628) bank24952 +24953 POINT(39.79232834174856 73.8884372099558) bank24953 +24954 POINT(39.80771878753028 74.48335833410867) bank24954 +24955 POINT(40.81824405336381 73.69452742883551) bank24955 +24956 POINT(40.06847546078995 74.24043115968203) bank24956 +24957 POINT(40.20256455458917 74.88132548998307) bank24957 +24958 POINT(40.297763028633504 73.05751412384359) bank24958 +24959 POINT(40.751946009311794 74.41129037652605) bank24959 +24960 POINT(41.49671855649678 73.2547042898829) bank24960 +24961 POINT(40.37418229008701 74.08994735135565) bank24961 +24962 POINT(40.43572176078674 74.5368088188435) bank24962 +24963 POINT(40.272975890343844 74.65133762731716) bank24963 +24964 POINT(41.279508095351005 73.22868853524346) bank24964 +24965 POINT(41.09646326975061 73.78376319386904) bank24965 +24966 POINT(40.49304273644911 73.51389362097906) bank24966 +24967 POINT(40.6670034099536 73.5360538160427) bank24967 +24968 POINT(41.52161261769626 73.88444009548888) bank24968 +24969 POINT(39.88159843064745 74.12363044056674) bank24969 +24970 POINT(39.86913622874175 75.00066510618856) bank24970 +24971 POINT(40.40079977014155 74.88141239477443) bank24971 +24972 POINT(41.646198672439965 74.03778300500404) bank24972 +24973 POINT(41.40116522673586 74.21723108865267) bank24973 +24974 POINT(41.56560777826356 73.47314737690131) bank24974 +24975 POINT(41.32105798981468 74.51449189488933) bank24975 +24976 POINT(40.312949555842806 73.88724135140684) bank24976 +24977 POINT(40.922239173409324 74.1171568561192) bank24977 +24978 POINT(41.579961867085316 73.55023146698551) bank24978 +24979 POINT(41.695784428551164 74.16506927424501) bank24979 +24980 POINT(40.39444807911973 73.78299299597508) bank24980 +24981 POINT(40.598000157435294 73.02576785468129) bank24981 +24982 POINT(39.84092738096132 73.36322308090682) bank24982 +24983 POINT(41.15011385843825 74.57848320082162) bank24983 +24984 POINT(41.37151020917003 74.28299606724596) bank24984 +24985 POINT(40.01837428534971 74.88749455733918) bank24985 +24986 POINT(39.875240566697485 74.47300853289454) bank24986 +24987 POINT(41.64947629684841 74.70859900381109) bank24987 +24988 POINT(40.20397341460549 74.58957577805528) bank24988 +24989 POINT(40.51207406847213 73.18857892870942) bank24989 +24990 POINT(40.2923139670067 74.57322622665997) bank24990 +24991 POINT(41.08941065498513 73.97256493462876) bank24991 +24992 POINT(40.84927642233865 74.75285816781471) bank24992 +24993 POINT(40.24585251010319 73.19683044041268) bank24993 +24994 POINT(40.53932718805403 74.38816613988355) bank24994 +24995 POINT(40.266591166824405 73.313968448504) bank24995 +24996 POINT(41.68669753249893 73.05037300837606) bank24996 +24997 POINT(41.503044246472605 74.93831833525762) bank24997 +24998 POINT(39.93453641609386 73.41723992337276) bank24998 +24999 POINT(41.69648735993369 74.38924524958165) bank24999 +25000 POINT(41.30535301086746 74.70697194286285) bank25000 +25001 POINT(40.27573635732298 73.22758529210029) bank25001 +25002 POINT(40.62114389712885 74.72891862465437) bank25002 +25003 POINT(40.13748303880668 74.59600586777715) bank25003 +25004 POINT(40.57187908510533 74.57726357201732) bank25004 +25005 POINT(41.587486678147464 74.15588347623063) bank25005 +25006 POINT(41.258968574283635 74.94886954157377) bank25006 +25007 POINT(40.80429444688305 74.18068756224953) bank25007 +25008 POINT(41.131653468039396 74.30946792634498) bank25008 +25009 POINT(41.6690633790173 74.55235815619277) bank25009 +25010 POINT(40.48112109811201 74.7631827992224) bank25010 +25011 POINT(41.272755128437375 73.16393169248435) bank25011 +25012 POINT(41.46055831487352 73.6035564412378) bank25012 +25013 POINT(41.650755217182514 74.18528653614185) bank25013 +25014 POINT(41.709972889591484 73.91967296041288) bank25014 +25015 POINT(41.099086906599915 74.94335569851033) bank25015 +25016 POINT(40.55477959280519 73.1114004523314) bank25016 +25017 POINT(41.296259153047174 74.15265666635125) bank25017 +25018 POINT(41.24661377981667 73.08799993842541) bank25018 +25019 POINT(40.65752525244559 74.40996606847311) bank25019 +25020 POINT(40.14789154115464 74.19859266094298) bank25020 +25021 POINT(39.92151286901297 74.82010670873059) bank25021 +25022 POINT(41.548157048724214 74.82885802959446) bank25022 +25023 POINT(41.524462252081754 73.84601043790816) bank25023 +25024 POINT(40.115498393996624 74.89569544975893) bank25024 +25025 POINT(40.27617164303671 73.42473902944266) bank25025 +25026 POINT(40.910350611904676 74.77997151022363) bank25026 +25027 POINT(40.97415148011589 73.16634446015112) bank25027 +25028 POINT(40.09999568865381 73.05458544744327) bank25028 +25029 POINT(40.537282024033026 73.42082548350841) bank25029 +25030 POINT(41.60194844607515 73.23426638642911) bank25030 +25031 POINT(41.656344033479975 74.54818746771785) bank25031 +25032 POINT(40.12694106339784 73.02702184391556) bank25032 +25033 POINT(41.0977448133901 73.7828686660232) bank25033 +25034 POINT(40.50516900834579 74.4890623408699) bank25034 +25035 POINT(41.06623085450605 74.02289870739304) bank25035 +25036 POINT(40.38412121186029 73.83481715787073) bank25036 +25037 POINT(41.69300932652259 74.7013856803657) bank25037 +25038 POINT(40.820127588105294 73.4119539285158) bank25038 +25039 POINT(40.59541896156516 73.17148581356258) bank25039 +25040 POINT(40.07102214280607 73.50705744198218) bank25040 +25041 POINT(41.14884777290128 73.53281631197213) bank25041 +25042 POINT(41.00603975920339 73.11028627473854) bank25042 +25043 POINT(41.06471448432762 74.82625219220631) bank25043 +25044 POINT(41.059459272083664 74.73866605564227) bank25044 +25045 POINT(41.04595280883934 73.14981317658287) bank25045 +25046 POINT(41.30398969875907 74.73990975508663) bank25046 +25047 POINT(40.838160744225874 73.9546481753015) bank25047 +25048 POINT(40.78920174114462 73.27720355650767) bank25048 +25049 POINT(40.78611333184286 74.1659709254658) bank25049 +25050 POINT(39.88081167302928 73.10086233040433) bank25050 +25051 POINT(40.66667251060977 73.61933675991253) bank25051 +25052 POINT(41.63304461364708 74.73960399450992) bank25052 +25053 POINT(41.24912506021817 73.98377214048723) bank25053 +25054 POINT(41.47262390133207 73.6307642508287) bank25054 +25055 POINT(40.37429138063832 74.37277174429079) bank25055 +25056 POINT(40.28116859361739 73.58676895057172) bank25056 +25057 POINT(39.96881743558716 73.7596160866728) bank25057 +25058 POINT(40.00156286650089 74.13930004418421) bank25058 +25059 POINT(39.78337512873705 73.81707122104737) bank25059 +25060 POINT(39.78915736075897 74.52688110902554) bank25060 +25061 POINT(41.348386804117084 74.8388001567719) bank25061 +25062 POINT(41.3460172842127 73.33979409154325) bank25062 +25063 POINT(40.696754808993994 74.99453814466214) bank25063 +25064 POINT(40.686753135894065 74.67283635574152) bank25064 +25065 POINT(40.341323965340784 73.5678122073813) bank25065 +25066 POINT(41.58733208596999 74.42936116014435) bank25066 +25067 POINT(39.948913069581394 73.44545360185836) bank25067 +25068 POINT(41.57579653170016 73.22471580332449) bank25068 +25069 POINT(41.3302840874437 74.69404005206474) bank25069 +25070 POINT(40.570683354968395 74.04623060539703) bank25070 +25071 POINT(40.768313095576 74.64721963551754) bank25071 +25072 POINT(39.92523024833911 73.27530150062854) bank25072 +25073 POINT(41.61190508543304 74.16112405461209) bank25073 +25074 POINT(41.15255568442771 74.3365840520956) bank25074 +25075 POINT(41.536535404293225 74.67609635530303) bank25075 +25076 POINT(41.19990334891503 74.83515995414028) bank25076 +25077 POINT(41.4658796096339 74.32109377603793) bank25077 +25078 POINT(41.50606535441103 74.55755576508774) bank25078 +25079 POINT(40.45927879790975 74.81445572633977) bank25079 +25080 POINT(41.692553384925596 74.47230324784357) bank25080 +25081 POINT(40.914240421615276 73.64246886437942) bank25081 +25082 POINT(39.824070581798665 74.8358898472806) bank25082 +25083 POINT(40.080905766880555 73.41982506274375) bank25083 +25084 POINT(41.3629797838895 74.98738583802707) bank25084 +25085 POINT(40.282673987792236 73.29217528075968) bank25085 +25086 POINT(41.57574376954439 74.8890508424567) bank25086 +25087 POINT(40.230341082845804 73.37103659386005) bank25087 +25088 POINT(40.31245314384102 74.86369488514472) bank25088 +25089 POINT(41.316368070564266 73.54847630350598) bank25089 +25090 POINT(40.15901020360324 74.35445234645866) bank25090 +25091 POINT(40.62710804020522 73.27034180102032) bank25091 +25092 POINT(40.79146539994707 74.58780566758278) bank25092 +25093 POINT(40.11743172465463 73.79855844145948) bank25093 +25094 POINT(40.64428082339654 73.87672121061236) bank25094 +25095 POINT(40.489148304725575 73.79254828424565) bank25095 +25096 POINT(41.3493697028448 73.45813634809772) bank25096 +25097 POINT(41.58017443201477 73.83785510945577) bank25097 +25098 POINT(41.34424817043056 74.79882698445897) bank25098 +25099 POINT(40.98050711664839 73.61654050864482) bank25099 +25100 POINT(40.84792187123039 73.17533167581017) bank25100 +25101 POINT(41.22933121866374 73.5894276602737) bank25101 +25102 POINT(40.851447138050496 74.78678697896423) bank25102 +25103 POINT(41.12045638217246 73.54618691489557) bank25103 +25104 POINT(40.100950352106274 74.26747704707678) bank25104 +25105 POINT(40.71332774491915 73.6330841462154) bank25105 +25106 POINT(40.16409124582809 74.70934118527805) bank25106 +25107 POINT(40.427674056525625 73.33466039665026) bank25107 +25108 POINT(40.64519451147636 74.53539785308641) bank25108 +25109 POINT(41.555346950595506 74.29079866494808) bank25109 +25110 POINT(40.30363241575366 73.26315459468708) bank25110 +25111 POINT(40.30220004942642 73.88881973815529) bank25111 +25112 POINT(41.484943126647345 74.78837787253971) bank25112 +25113 POINT(40.69338868535618 73.00661240353747) bank25113 +25114 POINT(39.84877615706281 74.94722620483564) bank25114 +25115 POINT(41.697242162584246 73.35107742614342) bank25115 +25116 POINT(40.90710801091186 73.3159907177056) bank25116 +25117 POINT(40.38982833407204 74.1036841039516) bank25117 +25118 POINT(40.921452014502755 73.711431768913) bank25118 +25119 POINT(41.657444232053805 74.39235737131968) bank25119 +25120 POINT(41.70175343857171 74.76854980836023) bank25120 +25121 POINT(41.3610551226197 74.68389968837224) bank25121 +25122 POINT(41.31042347602209 74.51109215291984) bank25122 +25123 POINT(40.828323997690546 74.70217603096155) bank25123 +25124 POINT(41.39872945347362 74.74483978634149) bank25124 +25125 POINT(40.46365742254088 74.4266057768602) bank25125 +25126 POINT(41.221528753348835 74.61733801437109) bank25126 +25127 POINT(39.94058985485278 74.36027606206764) bank25127 +25128 POINT(40.42058902583903 73.91894313083878) bank25128 +25129 POINT(39.8033942678527 73.51681735356891) bank25129 +25130 POINT(39.96449552689943 73.91847918403035) bank25130 +25131 POINT(39.97528180153471 74.79317091521735) bank25131 +25132 POINT(40.99822888091804 74.31464819911751) bank25132 +25133 POINT(40.090055173068784 73.29057145866123) bank25133 +25134 POINT(41.13283685536318 74.87088858506584) bank25134 +25135 POINT(40.52387206005945 74.5389593864517) bank25135 +25136 POINT(41.61252438392163 74.63932165083921) bank25136 +25137 POINT(41.49416954251929 74.8184146524729) bank25137 +25138 POINT(40.608766209906406 74.21428143105375) bank25138 +25139 POINT(41.595975887970944 74.18141942433952) bank25139 +25140 POINT(39.81727482825089 74.42440712884371) bank25140 +25141 POINT(40.98798282424842 73.91536033655784) bank25141 +25142 POINT(39.726970682647234 73.86647521603929) bank25142 +25143 POINT(41.26015818011815 73.47743958961708) bank25143 +25144 POINT(39.910452177434045 73.76034353933736) bank25144 +25145 POINT(41.25603211476059 73.6965597536293) bank25145 +25146 POINT(40.97624445413326 74.56046252739817) bank25146 +25147 POINT(40.115433814411254 74.53241255833429) bank25147 +25148 POINT(41.085002956054446 73.36620825286313) bank25148 +25149 POINT(41.482404763014436 74.5814038843982) bank25149 +25150 POINT(39.79543873222298 74.81430729152233) bank25150 +25151 POINT(41.48052992283248 73.22500700163803) bank25151 +25152 POINT(40.18227519161501 73.76369866492223) bank25152 +25153 POINT(41.17697963075996 74.79948404350074) bank25153 +25154 POINT(40.8999368589167 74.7138298861077) bank25154 +25155 POINT(40.387452763583184 73.66453827745072) bank25155 +25156 POINT(41.466666782368605 73.39570744716185) bank25156 +25157 POINT(41.043408342742254 74.07705293999092) bank25157 +25158 POINT(41.38699121677748 74.65646635820863) bank25158 +25159 POINT(40.028141073931735 74.97229782247757) bank25159 +25160 POINT(39.95111518234248 74.06924413981292) bank25160 +25161 POINT(39.788331238020376 73.23052070573203) bank25161 +25162 POINT(41.67990129251721 73.17745907810173) bank25162 +25163 POINT(41.699406283375744 74.91486894312884) bank25163 +25164 POINT(40.14459167050693 74.81525857664917) bank25164 +25165 POINT(40.98029555910114 73.68688706031229) bank25165 +25166 POINT(40.9735563424175 73.75450481533973) bank25166 +25167 POINT(41.03742389641453 73.88324138764136) bank25167 +25168 POINT(40.72485670823333 74.52186570425141) bank25168 +25169 POINT(41.28866526671545 74.46346631292737) bank25169 +25170 POINT(39.912917717758795 73.89795006895034) bank25170 +25171 POINT(40.83766393383499 73.82634499892552) bank25171 +25172 POINT(41.56093906834136 74.25715956818752) bank25172 +25173 POINT(41.66206020095993 74.00399920391331) bank25173 +25174 POINT(41.52278269508636 73.81266500925688) bank25174 +25175 POINT(41.00237553963577 73.09754792029744) bank25175 +25176 POINT(39.91809953927729 74.30410360213592) bank25176 +25177 POINT(41.55372222784182 74.97201925672015) bank25177 +25178 POINT(40.51382495870338 73.66233113544531) bank25178 +25179 POINT(41.492307465997236 74.7029431404367) bank25179 +25180 POINT(39.71312260615846 74.70204415751186) bank25180 +25181 POINT(41.2002462420382 73.15462905032844) bank25181 +25182 POINT(39.86262501443692 73.1350432848533) bank25182 +25183 POINT(40.954799886287 74.50906706686249) bank25183 +25184 POINT(40.83487167358148 73.98461770152272) bank25184 +25185 POINT(41.552949567014274 74.43414842822052) bank25185 +25186 POINT(41.089219710679885 74.37470987504112) bank25186 +25187 POINT(41.39475854488007 73.32665598223912) bank25187 +25188 POINT(39.85431178159711 74.39963095711799) bank25188 +25189 POINT(40.17259469898047 73.13322178234252) bank25189 +25190 POINT(39.84685373676711 74.51654182773719) bank25190 +25191 POINT(39.74006931253452 74.0098481591981) bank25191 +25192 POINT(41.108349832652785 73.33918940391371) bank25192 +25193 POINT(41.16158252449578 74.61565596520519) bank25193 +25194 POINT(40.81587679220185 73.77811464718488) bank25194 +25195 POINT(40.04834438895763 73.4233402702697) bank25195 +25196 POINT(40.79157941105188 74.13174025010586) bank25196 +25197 POINT(40.68743325269957 73.69546703331966) bank25197 +25198 POINT(39.94531170229374 73.42218569757539) bank25198 +25199 POINT(41.38667551846699 73.86327753378866) bank25199 +25200 POINT(40.087281585889635 74.7703962868358) bank25200 +25201 POINT(40.12776496256342 74.5871666829999) bank25201 +25202 POINT(40.81645377663245 74.514441027764) bank25202 +25203 POINT(40.015130564685826 74.53777975010668) bank25203 +25204 POINT(39.85164656653002 73.41857854990265) bank25204 +25205 POINT(40.452599571375096 74.704768285993) bank25205 +25206 POINT(41.33234275784971 73.89417549993406) bank25206 +25207 POINT(40.574184452380265 73.13709375498819) bank25207 +25208 POINT(40.948171064022674 73.53085726713705) bank25208 +25209 POINT(41.447901031263015 73.2149971175458) bank25209 +25210 POINT(39.96947752989411 74.4850192743506) bank25210 +25211 POINT(40.37989166081813 73.25396359010482) bank25211 +25212 POINT(40.06556531612566 74.97210183891144) bank25212 +25213 POINT(40.82819802762524 74.09247253829943) bank25213 +25214 POINT(41.10896704153698 73.6930156974442) bank25214 +25215 POINT(39.97304688654897 73.43793200294797) bank25215 +25216 POINT(41.38245945990802 74.91338480645761) bank25216 +25217 POINT(41.209656271125546 74.8987921016937) bank25217 +25218 POINT(41.59633480830947 73.5982661822047) bank25218 +25219 POINT(40.19942745321941 74.0989112232588) bank25219 +25220 POINT(41.467811675107 74.03495760470307) bank25220 +25221 POINT(40.51133466941948 74.89600743797617) bank25221 +25222 POINT(41.45268822619402 74.82518583832426) bank25222 +25223 POINT(41.3181661466658 74.06211880938662) bank25223 +25224 POINT(40.74481695816659 73.14588071120848) bank25224 +25225 POINT(41.264842996446724 73.91920592306181) bank25225 +25226 POINT(41.388986316857924 74.01450324104496) bank25226 +25227 POINT(40.618125973582686 74.30567940978119) bank25227 +25228 POINT(40.811646425170224 74.06559136186245) bank25228 +25229 POINT(40.81972959329972 73.23477920609803) bank25229 +25230 POINT(40.56112289655356 73.64032065167632) bank25230 +25231 POINT(40.56006054627097 74.17384542534185) bank25231 +25232 POINT(41.139382660635036 74.19554609878377) bank25232 +25233 POINT(41.33155196861434 73.75746330222495) bank25233 +25234 POINT(40.708218852621 73.82285051603445) bank25234 +25235 POINT(41.629496218559616 74.6159359945948) bank25235 +25236 POINT(40.41548697496203 74.21658591655837) bank25236 +25237 POINT(40.97840592664427 74.24169797154397) bank25237 +25238 POINT(40.78439689787581 74.34237780106707) bank25238 +25239 POINT(39.833227112306105 74.90987314497067) bank25239 +25240 POINT(41.688335236641095 73.61386462768246) bank25240 +25241 POINT(41.169186272915674 74.26460368552992) bank25241 +25242 POINT(41.2192073795773 73.30951530408286) bank25242 +25243 POINT(40.7246974027347 73.07011013023036) bank25243 +25244 POINT(40.6031769635405 74.28201527346565) bank25244 +25245 POINT(40.57669497112585 74.48705935011766) bank25245 +25246 POINT(40.138835453934064 74.91136718147798) bank25246 +25247 POINT(40.121938502330735 74.3815984834713) bank25247 +25248 POINT(41.07474196174072 73.79229859694951) bank25248 +25249 POINT(40.077901235388644 74.21810733036538) bank25249 +25250 POINT(41.67983890141124 74.25763510673421) bank25250 +25251 POINT(41.28460625700481 74.50882859583379) bank25251 +25252 POINT(40.072658280067905 73.83168834721653) bank25252 +25253 POINT(41.317915441655806 74.73078568967058) bank25253 +25254 POINT(40.927164533624726 73.26226960705975) bank25254 +25255 POINT(40.45173059126954 74.00823279667549) bank25255 +25256 POINT(41.09874943861748 74.16623264293057) bank25256 +25257 POINT(40.64446318529712 73.14807270712704) bank25257 +25258 POINT(40.78044715801691 74.80803761459674) bank25258 +25259 POINT(39.942829123496786 73.65453867963346) bank25259 +25260 POINT(41.57301349635174 74.35539617557744) bank25260 +25261 POINT(40.79804719602204 73.73093079806524) bank25261 +25262 POINT(41.25999613097316 74.82296663971654) bank25262 +25263 POINT(39.90935186563176 73.81923114733979) bank25263 +25264 POINT(40.620386202132124 74.453862026399) bank25264 +25265 POINT(39.86735926151863 73.56310884635384) bank25265 +25266 POINT(41.177282498975906 74.92066951503593) bank25266 +25267 POINT(40.58026117741754 73.11294291206924) bank25267 +25268 POINT(41.20592634706738 73.37606373168582) bank25268 +25269 POINT(40.762551060221924 73.30238899867079) bank25269 +25270 POINT(40.932269157663505 73.48228907757581) bank25270 +25271 POINT(40.45311276511647 73.21136176476683) bank25271 +25272 POINT(39.883100261626176 74.41754994892142) bank25272 +25273 POINT(39.91307000399893 74.36260093404661) bank25273 +25274 POINT(39.83897490882233 74.46990900664512) bank25274 +25275 POINT(41.28950354994577 74.14731940262892) bank25275 +25276 POINT(39.9219406975166 74.16149115060341) bank25276 +25277 POINT(40.28081011090311 74.43925804886763) bank25277 +25278 POINT(40.159573276495074 73.3444223412436) bank25278 +25279 POINT(40.241254850223704 73.69018884531043) bank25279 +25280 POINT(39.75698928655173 74.85206267325529) bank25280 +25281 POINT(40.68706112699958 73.62673830312181) bank25281 +25282 POINT(40.88627538755046 74.229507035652) bank25282 +25283 POINT(41.43737819949043 73.71711582150276) bank25283 +25284 POINT(40.877934295947576 73.75003053635824) bank25284 +25285 POINT(40.17876265869214 73.43802225109084) bank25285 +25286 POINT(40.66592664793071 74.31824323305065) bank25286 +25287 POINT(41.23537681861329 74.49488074099634) bank25287 +25288 POINT(41.29022553575696 74.6905592833663) bank25288 +25289 POINT(40.25972998335141 74.91468195367555) bank25289 +25290 POINT(40.533912822273656 74.16348043568182) bank25290 +25291 POINT(41.65360592600707 73.09077681559017) bank25291 +25292 POINT(40.74273404813757 73.77219963311049) bank25292 +25293 POINT(40.486542291953455 74.69060736255375) bank25293 +25294 POINT(40.574288819392834 74.6485969547763) bank25294 +25295 POINT(41.26284035541125 73.91329576492531) bank25295 +25296 POINT(39.97635105568112 74.74246597426568) bank25296 +25297 POINT(40.56148964617627 74.35274467143871) bank25297 +25298 POINT(40.63377182281939 74.15556171179391) bank25298 +25299 POINT(41.578795224571934 74.27448270282137) bank25299 +25300 POINT(41.10766077918063 74.47482222481334) bank25300 +25301 POINT(39.97805526368142 73.57444284328173) bank25301 +25302 POINT(41.687237219734804 74.92383993058628) bank25302 +25303 POINT(41.08608194227407 74.89304819868646) bank25303 +25304 POINT(41.22903319857886 73.85206335987696) bank25304 +25305 POINT(40.649276515163265 74.60096514268136) bank25305 +25306 POINT(40.60952897251244 74.397076925262) bank25306 +25307 POINT(40.806370536764604 73.09241055891962) bank25307 +25308 POINT(41.44931343471993 73.20252270357776) bank25308 +25309 POINT(40.623741138294925 74.03176557752367) bank25309 +25310 POINT(40.18118735723404 73.82396465642628) bank25310 +25311 POINT(40.1009288214675 74.54981208661265) bank25311 +25312 POINT(41.024771131847345 73.9980726532587) bank25312 +25313 POINT(40.74730349081621 73.42793272095764) bank25313 +25314 POINT(40.08339763144887 74.19170653175942) bank25314 +25315 POINT(41.414435710226066 74.50276722195802) bank25315 +25316 POINT(41.341236095420875 74.07322280151364) bank25316 +25317 POINT(41.59064654651134 74.95579807148819) bank25317 +25318 POINT(39.77322427032958 73.60450380897552) bank25318 +25319 POINT(40.9226835845248 74.44093082252783) bank25319 +25320 POINT(41.54679438694355 74.93771161419292) bank25320 +25321 POINT(41.03269412874387 73.8232254361999) bank25321 +25322 POINT(40.21100739897273 73.92787596206153) bank25322 +25323 POINT(39.763214711000174 73.06812575276217) bank25323 +25324 POINT(40.28095610022045 74.43456852300255) bank25324 +25325 POINT(40.10635899679244 73.25186194026374) bank25325 +25326 POINT(40.02879781274075 74.19062091568017) bank25326 +25327 POINT(41.607960206887476 73.03922884713565) bank25327 +25328 POINT(39.84642403942642 73.5118952300459) bank25328 +25329 POINT(41.43445971740265 73.33253844412867) bank25329 +25330 POINT(39.8529275500632 74.62032175672734) bank25330 +25331 POINT(41.066608309577546 74.83109289720394) bank25331 +25332 POINT(41.27566375417692 73.95654908552753) bank25332 +25333 POINT(40.734680803296136 74.15324222338786) bank25333 +25334 POINT(39.76874887775891 74.17640995166455) bank25334 +25335 POINT(41.02569688859538 73.91089928033182) bank25335 +25336 POINT(40.91415201266912 74.37092375913723) bank25336 +25337 POINT(39.836987453829316 74.03483141268634) bank25337 +25338 POINT(39.72137591725502 74.54754845873204) bank25338 +25339 POINT(41.68911138314759 73.44043543055628) bank25339 +25340 POINT(41.4309231454845 73.21697104403276) bank25340 +25341 POINT(40.769890092863434 74.00403600435789) bank25341 +25342 POINT(40.310287745790696 73.22348147601959) bank25342 +25343 POINT(41.58720836764166 74.41511230083987) bank25343 +25344 POINT(41.6993634074183 74.43941314769718) bank25344 +25345 POINT(41.27175682690007 73.0665925030358) bank25345 +25346 POINT(39.97005698022786 73.28891058063165) bank25346 +25347 POINT(41.55448163154291 74.1007685267365) bank25347 +25348 POINT(41.307398166951906 74.14860975905401) bank25348 +25349 POINT(40.45947738281168 74.81400510042279) bank25349 +25350 POINT(39.95122988828525 74.60718868250225) bank25350 +25351 POINT(40.26909040417604 74.45588511182387) bank25351 +25352 POINT(41.37289715328144 73.56486680178654) bank25352 +25353 POINT(41.08041136994976 73.12520415904062) bank25353 +25354 POINT(40.91302537725521 73.30404278499307) bank25354 +25355 POINT(40.00694413506895 73.73126719373226) bank25355 +25356 POINT(40.19459675005174 74.72682435399338) bank25356 +25357 POINT(40.031093326270394 73.64928379857923) bank25357 +25358 POINT(40.00965171537649 74.50468182622694) bank25358 +25359 POINT(41.31377853956461 74.52165821243416) bank25359 +25360 POINT(41.62749225642538 74.57951064485358) bank25360 +25361 POINT(40.57450222952027 73.92715473250061) bank25361 +25362 POINT(41.416760210759506 73.18266411223777) bank25362 +25363 POINT(40.640062617611186 73.38712575581326) bank25363 +25364 POINT(41.10040356900172 74.66761102921035) bank25364 +25365 POINT(40.586616010911634 73.56490304626189) bank25365 +25366 POINT(41.43316359748276 73.84664884554007) bank25366 +25367 POINT(40.874279601873354 74.21296422593083) bank25367 +25368 POINT(39.82573826896907 74.4109891192737) bank25368 +25369 POINT(40.54724417531039 73.0816523720335) bank25369 +25370 POINT(40.73175887164614 73.1181152055116) bank25370 +25371 POINT(40.466225070198874 73.13442159556159) bank25371 +25372 POINT(41.426483783268246 74.54907050021598) bank25372 +25373 POINT(40.03921888981706 74.4591249348799) bank25373 +25374 POINT(40.63430466270331 74.17306502721136) bank25374 +25375 POINT(40.90874480921642 73.07676165766247) bank25375 +25376 POINT(41.45765751605471 73.86257687873635) bank25376 +25377 POINT(40.16463194841481 73.92258639927883) bank25377 +25378 POINT(40.899284979428614 74.70716710970112) bank25378 +25379 POINT(41.47041103943769 73.04303189541412) bank25379 +25380 POINT(40.46436812936657 73.68041034385742) bank25380 +25381 POINT(40.78461820968593 74.48197056951633) bank25381 +25382 POINT(41.54086404488343 74.72387145345154) bank25382 +25383 POINT(41.70039223649505 73.25073143340148) bank25383 +25384 POINT(40.006512453014274 73.4843625925445) bank25384 +25385 POINT(40.70238020832504 74.94857283731005) bank25385 +25386 POINT(40.35699284825814 73.69913788407648) bank25386 +25387 POINT(40.765111818782124 74.80654841269516) bank25387 +25388 POINT(40.46260252572858 74.58710235212696) bank25388 +25389 POINT(41.48447237084793 74.93999400827201) bank25389 +25390 POINT(41.434273265588494 73.95285988160418) bank25390 +25391 POINT(39.77250256144545 73.21567506781366) bank25391 +25392 POINT(40.151591866993286 74.01880919117026) bank25392 +25393 POINT(41.252475169212765 74.16074155517303) bank25393 +25394 POINT(41.42743504328286 73.55249439686814) bank25394 +25395 POINT(40.31379302212867 73.55739166100246) bank25395 +25396 POINT(39.83591615889723 73.5899384283728) bank25396 +25397 POINT(41.137985196952464 74.5681164515982) bank25397 +25398 POINT(40.98575352644822 73.26184417294603) bank25398 +25399 POINT(40.045174104449 73.3430908605707) bank25399 +25400 POINT(39.7229040829126 74.01983948028594) bank25400 +25401 POINT(40.88707909858244 73.04219836696804) bank25401 +25402 POINT(41.247396023095256 74.69136996963708) bank25402 +25403 POINT(41.33215751914767 74.22039889771209) bank25403 +25404 POINT(41.14948324027505 73.22637612625715) bank25404 +25405 POINT(40.77105480462967 74.28857840417743) bank25405 +25406 POINT(40.81332655316022 74.54504922734417) bank25406 +25407 POINT(40.08721012658382 74.43541915665776) bank25407 +25408 POINT(41.67610077643331 74.5174792637674) bank25408 +25409 POINT(40.778716387016374 73.29884954903693) bank25409 +25410 POINT(41.54934853668026 73.25095828453529) bank25410 +25411 POINT(40.048228674694 73.8652484416654) bank25411 +25412 POINT(40.27981255060502 73.56268474030239) bank25412 +25413 POINT(41.47229897438899 73.72350323434301) bank25413 +25414 POINT(41.67940389813345 74.756427839473) bank25414 +25415 POINT(40.445407230254546 73.75785449645421) bank25415 +25416 POINT(40.63803701561336 74.03616547551975) bank25416 +25417 POINT(39.94318941044974 73.09272831879264) bank25417 +25418 POINT(40.49167310491162 73.60790143374052) bank25418 +25419 POINT(40.531922539046796 74.08347916673245) bank25419 +25420 POINT(41.408032169437924 73.65999500858524) bank25420 +25421 POINT(41.02284003025284 73.18653182505457) bank25421 +25422 POINT(39.73458023750716 73.31020612461504) bank25422 +25423 POINT(40.82147481553754 74.50446219425925) bank25423 +25424 POINT(41.20635355424697 74.04976835115235) bank25424 +25425 POINT(40.070368938571484 74.43973919882545) bank25425 +25426 POINT(41.13521743320754 73.68827237438198) bank25426 +25427 POINT(40.15453053666621 73.6205458326859) bank25427 +25428 POINT(40.34362649881835 74.0495043584698) bank25428 +25429 POINT(41.62206172426273 74.83998599886267) bank25429 +25430 POINT(39.73935046901794 74.18417101798057) bank25430 +25431 POINT(41.28338381986015 73.51021627966657) bank25431 +25432 POINT(41.391327150275586 74.80888412811589) bank25432 +25433 POINT(39.83252568109901 74.44851244486665) bank25433 +25434 POINT(41.48166557408538 73.16140911813253) bank25434 +25435 POINT(40.96799497744846 74.1423316069996) bank25435 +25436 POINT(41.67275120140851 74.88715219650771) bank25436 +25437 POINT(40.89553526456742 73.75108719915036) bank25437 +25438 POINT(40.19438824296656 73.7784966109092) bank25438 +25439 POINT(39.87065239262061 73.22822049249457) bank25439 +25440 POINT(40.257678594422444 74.02721705835525) bank25440 +25441 POINT(40.30132528232609 74.45979241692262) bank25441 +25442 POINT(40.338583335918344 74.4264434691396) bank25442 +25443 POINT(41.661534605032955 74.28787459356955) bank25443 +25444 POINT(40.639055838682594 74.51979291355) bank25444 +25445 POINT(40.785227960594675 74.17313791504492) bank25445 +25446 POINT(39.808349476167166 73.71443684027112) bank25446 +25447 POINT(40.70233023600146 73.97788575578345) bank25447 +25448 POINT(40.43619313693053 73.23997065819033) bank25448 +25449 POINT(40.10542817849415 73.11533750830327) bank25449 +25450 POINT(40.6192800111695 74.48519693200092) bank25450 +25451 POINT(40.76898878649011 73.19464451190088) bank25451 +25452 POINT(40.16394392363343 73.82465537303474) bank25452 +25453 POINT(40.390722327830964 73.2486004351224) bank25453 +25454 POINT(41.11534599447232 73.91445337408796) bank25454 +25455 POINT(40.30671584566548 74.74129579197037) bank25455 +25456 POINT(40.27946262771566 73.759718184689) bank25456 +25457 POINT(40.926692705682854 73.87296408222994) bank25457 +25458 POINT(41.69347402018223 74.1854301905996) bank25458 +25459 POINT(40.97003497266224 73.64336056438536) bank25459 +25460 POINT(41.362321966062694 74.45830738358721) bank25460 +25461 POINT(41.654420877759236 73.98598391109822) bank25461 +25462 POINT(40.237198972439835 74.19468352255234) bank25462 +25463 POINT(41.118452924060335 73.81485820620192) bank25463 +25464 POINT(41.225300249002565 74.01126254527672) bank25464 +25465 POINT(41.21364401302858 73.1403272383127) bank25465 +25466 POINT(41.462435121847015 73.64647552834042) bank25466 +25467 POINT(40.154212905460675 74.17992039835312) bank25467 +25468 POINT(41.0265840007738 73.87029157837165) bank25468 +25469 POINT(40.02780477275561 74.38959272698796) bank25469 +25470 POINT(40.01904421957155 74.49372339797183) bank25470 +25471 POINT(39.90958982203075 73.72703695807832) bank25471 +25472 POINT(41.677631310580416 74.97503286614992) bank25472 +25473 POINT(40.76308562493831 74.35339139373495) bank25473 +25474 POINT(41.58102017423669 73.83993454024875) bank25474 +25475 POINT(40.061624170395866 73.52929589768088) bank25475 +25476 POINT(40.98145896660255 73.54421943255291) bank25476 +25477 POINT(40.04448474887198 73.2826725783923) bank25477 +25478 POINT(40.861676910576286 73.80654141772638) bank25478 +25479 POINT(40.89924962195837 74.01498051407589) bank25479 +25480 POINT(40.78657246091025 73.3168777826822) bank25480 +25481 POINT(40.38344644135991 73.6400110686362) bank25481 +25482 POINT(39.77786485071662 74.65031278543627) bank25482 +25483 POINT(41.47076030415181 74.58321886111602) bank25483 +25484 POINT(41.5181597276255 73.14958228676655) bank25484 +25485 POINT(40.498025330746884 74.88455786775953) bank25485 +25486 POINT(40.38437840684455 74.54021287582825) bank25486 +25487 POINT(39.784661241817915 73.72877168448832) bank25487 +25488 POINT(41.46115169654726 74.83349017599733) bank25488 +25489 POINT(41.53790147759647 73.51770970877583) bank25489 +25490 POINT(40.13339193063925 73.08121385598946) bank25490 +25491 POINT(41.10994399308086 73.56643103295957) bank25491 +25492 POINT(41.169660107947294 74.97166331738501) bank25492 +25493 POINT(40.9563405438174 73.03112946574294) bank25493 +25494 POINT(40.25950716640697 73.9801302986903) bank25494 +25495 POINT(40.840390814059575 73.68683177635103) bank25495 +25496 POINT(40.68714402433567 73.0802069539182) bank25496 +25497 POINT(39.88724423097671 73.35809443489829) bank25497 +25498 POINT(41.35323320698987 74.86277909331548) bank25498 +25499 POINT(40.263882460933786 73.16671251882794) bank25499 +25500 POINT(41.527688618531975 73.97056847267052) bank25500 +25501 POINT(40.24289878981991 74.14001171032452) bank25501 +25502 POINT(40.50953587886011 73.60712685778782) bank25502 +25503 POINT(40.09075933063447 73.9284644304287) bank25503 +25504 POINT(40.04194782926305 73.79487614187413) bank25504 +25505 POINT(40.01369479281765 73.66297423734841) bank25505 +25506 POINT(40.34626637805315 73.05584761159683) bank25506 +25507 POINT(40.950533225751116 74.92434971532025) bank25507 +25508 POINT(39.95112069269506 74.08985666533219) bank25508 +25509 POINT(40.55079039789258 74.82381783714186) bank25509 +25510 POINT(40.233936442720704 73.7204310991007) bank25510 +25511 POINT(39.82896739514502 73.7342135334342) bank25511 +25512 POINT(40.210431621176326 73.2177312927781) bank25512 +25513 POINT(41.23103729979881 73.2370094710224) bank25513 +25514 POINT(40.22868108329936 73.3519045790248) bank25514 +25515 POINT(39.7461910210598 74.69615648853296) bank25515 +25516 POINT(41.63404920026873 73.17786907413812) bank25516 +25517 POINT(40.31778992545171 74.47052232545848) bank25517 +25518 POINT(39.97512782149411 74.58775876932488) bank25518 +25519 POINT(40.28069682132506 73.2584053713637) bank25519 +25520 POINT(41.327469313684254 74.90399915530493) bank25520 +25521 POINT(41.39340705794301 74.57510116499328) bank25521 +25522 POINT(41.64478669583581 74.22282538886274) bank25522 +25523 POINT(40.73809791766972 74.69212248697012) bank25523 +25524 POINT(41.4948337766528 73.49341994022087) bank25524 +25525 POINT(40.130948987406896 74.64034915973234) bank25525 +25526 POINT(41.49534209712075 73.80032873100285) bank25526 +25527 POINT(40.14614267905987 74.03845530232498) bank25527 +25528 POINT(41.68171363705785 74.64786960031104) bank25528 +25529 POINT(40.34596398460874 74.85279149863904) bank25529 +25530 POINT(41.05682883893943 74.4954407960879) bank25530 +25531 POINT(40.8259418321148 74.27790965340384) bank25531 +25532 POINT(41.678944457186894 74.87536654200777) bank25532 +25533 POINT(41.465795627939684 73.17398619360593) bank25533 +25534 POINT(40.005107941642336 73.36791497204294) bank25534 +25535 POINT(40.610226100502615 73.32126585221906) bank25535 +25536 POINT(41.57228051449516 73.8344446784305) bank25536 +25537 POINT(40.32015485965647 73.26664948545636) bank25537 +25538 POINT(40.28927566193945 74.74016475572004) bank25538 +25539 POINT(40.563447067965384 73.31219065643636) bank25539 +25540 POINT(41.493748859829665 73.67581699481195) bank25540 +25541 POINT(41.32518650506102 74.8052828003415) bank25541 +25542 POINT(40.27232100925548 73.05743820548606) bank25542 +25543 POINT(41.10020428041333 73.07549143666627) bank25543 +25544 POINT(40.507108009870215 74.48375672423442) bank25544 +25545 POINT(39.740649775806475 73.74996264167402) bank25545 +25546 POINT(41.10238015012741 74.058546627322) bank25546 +25547 POINT(40.56473721508849 73.52673525425936) bank25547 +25548 POINT(40.626875000042716 74.17970272330021) bank25548 +25549 POINT(39.86500517245632 74.02655178065996) bank25549 +25550 POINT(40.82323049835836 74.45606654627709) bank25550 +25551 POINT(40.150211727055854 73.03834812941064) bank25551 +25552 POINT(41.39060085376389 73.326281936545) bank25552 +25553 POINT(40.5885027166467 73.90298645837889) bank25553 +25554 POINT(39.84568101494087 73.53170239227862) bank25554 +25555 POINT(41.18826319037093 73.07979483024037) bank25555 +25556 POINT(41.23374631085126 73.44215332612083) bank25556 +25557 POINT(40.06015992072247 74.10878799901808) bank25557 +25558 POINT(40.32129083612948 74.0285363889613) bank25558 +25559 POINT(40.10121436165949 74.49374735737493) bank25559 +25560 POINT(41.57777187106154 74.72141486715422) bank25560 +25561 POINT(40.01340794875844 73.95731865666289) bank25561 +25562 POINT(40.02286530094262 74.26882424269313) bank25562 +25563 POINT(39.75779685221927 74.43916913668757) bank25563 +25564 POINT(39.892311086135514 74.17255596646531) bank25564 +25565 POINT(39.93352439858258 73.52968499668027) bank25565 +25566 POINT(41.534579035211614 74.76993177267835) bank25566 +25567 POINT(39.889370638355125 74.62620479104748) bank25567 +25568 POINT(40.61149117652234 73.99719924009855) bank25568 +25569 POINT(40.53568579818895 74.00137889704725) bank25569 +25570 POINT(40.418620429789264 73.64826574893614) bank25570 +25571 POINT(40.66766070412634 73.55018235287756) bank25571 +25572 POINT(41.23767371410005 74.70207681656599) bank25572 +25573 POINT(41.46527490519966 74.65677197525892) bank25573 +25574 POINT(41.44381685446027 74.23763509394176) bank25574 +25575 POINT(41.253587819797374 73.07793483517551) bank25575 +25576 POINT(39.74540420868416 74.56784626378384) bank25576 +25577 POINT(40.22189091737431 74.88525109845096) bank25577 +25578 POINT(40.50235685657173 74.17404720039144) bank25578 +25579 POINT(40.1314747207907 74.81622609145398) bank25579 +25580 POINT(39.8256030687657 74.92056782178449) bank25580 +25581 POINT(40.50088295681053 73.7917693141883) bank25581 +25582 POINT(40.37232616096389 73.6616449871484) bank25582 +25583 POINT(40.15746205059496 74.38155528441196) bank25583 +25584 POINT(40.85946081613189 73.56264469377862) bank25584 +25585 POINT(41.161882390890355 73.27669544367483) bank25585 +25586 POINT(40.22462462228831 73.12123327349707) bank25586 +25587 POINT(40.49246625633869 73.41126432402888) bank25587 +25588 POINT(41.172652287599206 73.20836069917272) bank25588 +25589 POINT(40.34887555040316 73.96350864713243) bank25589 +25590 POINT(39.81467792865932 74.58964421665209) bank25590 +25591 POINT(41.321102737202295 74.8918288750853) bank25591 +25592 POINT(40.38179361868911 74.57325181985783) bank25592 +25593 POINT(40.21327743914856 73.95120919277596) bank25593 +25594 POINT(39.75603971574674 73.36874326388198) bank25594 +25595 POINT(41.19413096074493 74.6623549633693) bank25595 +25596 POINT(41.32143516881104 73.98380758742877) bank25596 +25597 POINT(40.042466313919526 74.81920510367277) bank25597 +25598 POINT(40.60819410356695 73.66542115428668) bank25598 +25599 POINT(40.110300074882026 74.99644791413155) bank25599 +25600 POINT(40.97001699439885 73.25180430442688) bank25600 +25601 POINT(40.679396523238246 73.0492527974185) bank25601 +25602 POINT(41.70905846357643 74.04268624247747) bank25602 +25603 POINT(40.63356926974506 73.75517977259764) bank25603 +25604 POINT(40.122161070178976 74.97270402372816) bank25604 +25605 POINT(41.28145720550593 74.87550424912283) bank25605 +25606 POINT(40.52057245610596 73.81718220460414) bank25606 +25607 POINT(39.981498551811875 73.71679595249152) bank25607 +25608 POINT(40.13395772461047 73.83186665794446) bank25608 +25609 POINT(40.83939617729243 74.918977287234) bank25609 +25610 POINT(39.7404621255303 74.51200895393322) bank25610 +25611 POINT(39.857171265558364 73.38632311826305) bank25611 +25612 POINT(40.370119110525835 74.04884072501851) bank25612 +25613 POINT(40.382350879703935 73.6790391908923) bank25613 +25614 POINT(40.01838074787741 73.29907490308203) bank25614 +25615 POINT(40.589495176968384 73.10768649368184) bank25615 +25616 POINT(40.915175999554876 73.88478098965714) bank25616 +25617 POINT(40.07360650984335 74.47463242308474) bank25617 +25618 POINT(41.17267686257156 73.01305477548142) bank25618 +25619 POINT(41.346274828204955 73.81879450750135) bank25619 +25620 POINT(39.893542532797305 73.63296290698678) bank25620 +25621 POINT(40.713193777227744 74.7478030358427) bank25621 +25622 POINT(40.8465006567318 73.26683518544525) bank25622 +25623 POINT(40.09336517031882 73.98439302092257) bank25623 +25624 POINT(39.88368875077737 73.72823640967009) bank25624 +25625 POINT(41.192371431579616 73.37364554040987) bank25625 +25626 POINT(41.58174638517766 73.472038747467) bank25626 +25627 POINT(40.398308442908466 74.4028017466908) bank25627 +25628 POINT(41.05203523660839 74.78908426270526) bank25628 +25629 POINT(41.682623563426674 74.8204173602444) bank25629 +25630 POINT(39.820584988593666 73.19269128953793) bank25630 +25631 POINT(40.34519285229804 73.39882726309017) bank25631 +25632 POINT(39.719588843310405 73.20627929239305) bank25632 +25633 POINT(39.96658359300289 74.40933502719845) bank25633 +25634 POINT(41.386801279104276 73.24445305432286) bank25634 +25635 POINT(39.85486950375242 74.01691265747559) bank25635 +25636 POINT(41.450758642706454 73.12212760146049) bank25636 +25637 POINT(40.89136096164065 74.86295007471799) bank25637 +25638 POINT(41.18871525359347 74.7774225248579) bank25638 +25639 POINT(40.46735802029521 74.17634066613171) bank25639 +25640 POINT(41.254798670742495 73.45946123373294) bank25640 +25641 POINT(41.493818068110876 73.74628270295419) bank25641 +25642 POINT(40.89586810090351 73.30374277642817) bank25642 +25643 POINT(40.15999324112943 74.05054250390491) bank25643 +25644 POINT(40.73139655292448 73.0442594883147) bank25644 +25645 POINT(41.342013168264906 74.65708516764927) bank25645 +25646 POINT(41.60795094054657 74.04593094651058) bank25646 +25647 POINT(41.50261918994454 73.74194449027337) bank25647 +25648 POINT(40.20113790553572 74.3324001686769) bank25648 +25649 POINT(40.529163668575265 73.826665133943) bank25649 +25650 POINT(40.23199422035115 73.48452202272044) bank25650 +25651 POINT(40.67153342056954 74.90438521481568) bank25651 +25652 POINT(40.265645915304994 74.10966421723742) bank25652 +25653 POINT(41.55622282086341 74.12345115736247) bank25653 +25654 POINT(41.365937085688856 73.62590200949184) bank25654 +25655 POINT(41.605365418410976 73.54413948609601) bank25655 +25656 POINT(41.086378839560815 73.47020080210548) bank25656 +25657 POINT(41.67491896023473 74.27215009139309) bank25657 +25658 POINT(40.96694872306041 74.07097582433595) bank25658 +25659 POINT(41.026104691849696 73.36078218338355) bank25659 +25660 POINT(40.64142588288033 74.35965181898698) bank25660 +25661 POINT(40.45225783946079 73.78450352131827) bank25661 +25662 POINT(41.44752701881838 73.1853323401108) bank25662 +25663 POINT(40.32137992044019 73.71826209515007) bank25663 +25664 POINT(40.66170843710588 74.83174521388295) bank25664 +25665 POINT(40.96779255681061 73.37897666765306) bank25665 +25666 POINT(40.6581879038691 73.6289933230273) bank25666 +25667 POINT(40.00372660197857 74.13287401458349) bank25667 +25668 POINT(40.127216671157726 73.65301791763227) bank25668 +25669 POINT(41.44499207026573 74.86352401115406) bank25669 +25670 POINT(41.33347446394002 73.86439706883233) bank25670 +25671 POINT(40.230897547299215 73.46158882606758) bank25671 +25672 POINT(40.34175037756388 74.4596661472236) bank25672 +25673 POINT(41.13970367628967 74.36432123451445) bank25673 +25674 POINT(40.906678752715045 74.8642263930793) bank25674 +25675 POINT(40.32375337407237 74.40290397169221) bank25675 +25676 POINT(40.988528233687525 74.2212345438448) bank25676 +25677 POINT(40.260575062520374 74.50121367354295) bank25677 +25678 POINT(41.141661289185166 73.10199808855945) bank25678 +25679 POINT(40.12401537862571 74.71461043738837) bank25679 +25680 POINT(39.73012235838276 73.36337834540994) bank25680 +25681 POINT(39.8608056074891 74.50393950965142) bank25681 +25682 POINT(41.64186468385831 74.5338826834077) bank25682 +25683 POINT(40.30212646092283 73.84163111010913) bank25683 +25684 POINT(40.952062899069475 74.65707257332711) bank25684 +25685 POINT(41.54783202915611 74.91879348829954) bank25685 +25686 POINT(41.600817042750016 74.59115079038963) bank25686 +25687 POINT(39.73757883920093 73.71983779701887) bank25687 +25688 POINT(40.99621558433735 73.09856793917778) bank25688 +25689 POINT(39.978235196954984 73.80978657628344) bank25689 +25690 POINT(40.64659526484076 74.4467610519721) bank25690 +25691 POINT(39.890624420834556 74.75564951924653) bank25691 +25692 POINT(40.61429498681402 74.20068192023722) bank25692 +25693 POINT(40.736719089553596 74.23586429511246) bank25693 +25694 POINT(40.892084803337276 73.58765389150793) bank25694 +25695 POINT(41.125655993041924 74.85872093698589) bank25695 +25696 POINT(41.4684974132886 73.71192679891485) bank25696 +25697 POINT(41.00301073125188 74.49214218559075) bank25697 +25698 POINT(40.593111448906 74.22937586419896) bank25698 +25699 POINT(40.2479255388746 73.37087715936846) bank25699 +25700 POINT(41.627197791646275 73.49630431374584) bank25700 +25701 POINT(40.426427300698414 74.89865100910383) bank25701 +25702 POINT(40.84981319168104 73.65480928274026) bank25702 +25703 POINT(41.15227335200673 73.12303045003728) bank25703 +25704 POINT(41.17945143672388 73.01289451534439) bank25704 +25705 POINT(40.63306086234753 73.41828680576761) bank25705 +25706 POINT(40.03114877500997 73.42826245586734) bank25706 +25707 POINT(40.51358178606795 73.4006959527877) bank25707 +25708 POINT(41.10338450822171 74.28431350495376) bank25708 +25709 POINT(41.07394432538977 74.32998491424136) bank25709 +25710 POINT(40.35672560708035 74.28691344405122) bank25710 +25711 POINT(41.572286830509334 73.86549014734257) bank25711 +25712 POINT(40.83672037075188 74.55605834777768) bank25712 +25713 POINT(40.02501009551557 74.5558913598005) bank25713 +25714 POINT(40.42727062393375 74.65574986481599) bank25714 +25715 POINT(39.71592413514682 74.74198977505758) bank25715 +25716 POINT(39.982959657841555 74.54689539597577) bank25716 +25717 POINT(40.6528530411982 74.72275270733952) bank25717 +25718 POINT(40.26386049728773 74.37661162515691) bank25718 +25719 POINT(40.81024140767068 74.16298935218036) bank25719 +25720 POINT(40.568537589276 73.12513372991808) bank25720 +25721 POINT(41.143150159601646 74.44392933727664) bank25721 +25722 POINT(41.27380960525302 74.18775043783451) bank25722 +25723 POINT(40.1328717660706 73.85837352657045) bank25723 +25724 POINT(41.45295702596346 74.70207425092482) bank25724 +25725 POINT(39.7481676612533 73.1688185211349) bank25725 +25726 POINT(40.877877828477274 73.78296515575647) bank25726 +25727 POINT(40.83601212136712 74.56766682688267) bank25727 +25728 POINT(40.28575912307304 74.61157221433498) bank25728 +25729 POINT(41.617015929194956 73.41854621547618) bank25729 +25730 POINT(39.76509573617539 74.65170729802614) bank25730 +25731 POINT(40.349957682815 74.85713485492371) bank25731 +25732 POINT(40.285002264384374 73.63334603381978) bank25732 +25733 POINT(40.3708063011351 73.08473468914072) bank25733 +25734 POINT(40.6623880406341 74.09276808608196) bank25734 +25735 POINT(40.16236717581478 73.03532833233149) bank25735 +25736 POINT(41.17832052540602 73.4626482303203) bank25736 +25737 POINT(41.3207013475692 74.31937540869436) bank25737 +25738 POINT(39.958421494470166 73.57588537302853) bank25738 +25739 POINT(40.348072959628404 73.47741008254127) bank25739 +25740 POINT(40.73786948129408 73.94933942119083) bank25740 +25741 POINT(39.72506097180147 74.47094426750922) bank25741 +25742 POINT(40.81273002934748 73.48394797096103) bank25742 +25743 POINT(40.45158159581696 73.1749335427266) bank25743 +25744 POINT(41.70609522995133 73.05953444455446) bank25744 +25745 POINT(40.48892109899527 74.79744521592494) bank25745 +25746 POINT(40.86765883683644 74.34170309456808) bank25746 +25747 POINT(40.43421631157998 73.11780873650045) bank25747 +25748 POINT(40.55361545663203 74.01623910969099) bank25748 +25749 POINT(41.45101453928492 73.33124030387745) bank25749 +25750 POINT(40.35506140604275 74.82243408268279) bank25750 +25751 POINT(40.06803494868073 73.36291401437533) bank25751 +25752 POINT(40.254642189000634 73.48184476238953) bank25752 +25753 POINT(39.80993380742807 73.10440076235092) bank25753 +25754 POINT(40.81990896632128 74.75274126435814) bank25754 +25755 POINT(41.2925321090911 73.02537185190903) bank25755 +25756 POINT(40.8395136702068 74.62804281844221) bank25756 +25757 POINT(41.36395876293129 73.15559030610646) bank25757 +25758 POINT(41.16070849479319 73.69867359684837) bank25758 +25759 POINT(40.8926910425483 73.65109403745946) bank25759 +25760 POINT(41.24091457026995 74.25077905625649) bank25760 +25761 POINT(41.387070205593815 73.6536487353991) bank25761 +25762 POINT(40.28654655367092 73.36725440802036) bank25762 +25763 POINT(40.80895627125594 73.52474303044951) bank25763 +25764 POINT(41.499936921530214 73.98911672869362) bank25764 +25765 POINT(39.96732029238083 73.3362391095901) bank25765 +25766 POINT(40.76596952354548 73.7727988172152) bank25766 +25767 POINT(40.44357756070788 74.97901339623033) bank25767 +25768 POINT(40.78148486051456 74.76634442865752) bank25768 +25769 POINT(40.796015894976655 73.02344673423532) bank25769 +25770 POINT(40.63460849336657 73.51325846183312) bank25770 +25771 POINT(41.43950603913987 73.20905371143311) bank25771 +25772 POINT(41.04680608294832 73.06809212796813) bank25772 +25773 POINT(40.717765296352674 73.60879177416761) bank25773 +25774 POINT(41.672852843330745 73.65280561940736) bank25774 +25775 POINT(40.73117321346921 74.5988173282395) bank25775 +25776 POINT(40.11290100993072 74.38360119718975) bank25776 +25777 POINT(39.92511312590175 73.52261226093894) bank25777 +25778 POINT(40.940897803100036 73.8037294331659) bank25778 +25779 POINT(41.198941147924934 73.222444822022) bank25779 +25780 POINT(40.071248662237394 74.55587003849881) bank25780 +25781 POINT(39.96877089534468 74.12538338494957) bank25781 +25782 POINT(40.48279725275855 74.62589352111686) bank25782 +25783 POINT(41.10728449236095 74.712106043715) bank25783 +25784 POINT(40.324490946323266 74.03288852099001) bank25784 +25785 POINT(41.55575069589813 74.62249942902977) bank25785 +25786 POINT(40.403408828030955 73.35011448961353) bank25786 +25787 POINT(41.35152159240868 74.28302484308983) bank25787 +25788 POINT(40.64873291493484 73.81423553659873) bank25788 +25789 POINT(40.18462588415634 73.38433915339186) bank25789 +25790 POINT(40.841065803584605 74.4504636799099) bank25790 +25791 POINT(41.7099659017679 73.25012693635686) bank25791 +25792 POINT(40.110581876155955 74.24778112130328) bank25792 +25793 POINT(41.07694225590202 73.07265942573397) bank25793 +25794 POINT(41.45453882241982 74.56677070629691) bank25794 +25795 POINT(41.266685163838744 74.00159580569274) bank25795 +25796 POINT(41.081018807548546 73.97665285735644) bank25796 +25797 POINT(40.83624939068467 73.94867611070609) bank25797 +25798 POINT(41.67492013940286 74.31346213785082) bank25798 +25799 POINT(40.63948609568446 74.56778937282523) bank25799 +25800 POINT(41.17728345579263 74.90924345281152) bank25800 +25801 POINT(40.37396722636717 73.27131176087954) bank25801 +25802 POINT(40.35247669383214 74.96768101020321) bank25802 +25803 POINT(40.9408670640284 73.99727392778092) bank25803 +25804 POINT(40.41095068250532 73.35459410917542) bank25804 +25805 POINT(41.42917868119692 73.02456571084804) bank25805 +25806 POINT(41.59399826143647 74.36231236521628) bank25806 +25807 POINT(41.22994814987471 73.59285872572148) bank25807 +25808 POINT(41.23665666558661 74.30947111745873) bank25808 +25809 POINT(41.57284276864813 73.63166098619023) bank25809 +25810 POINT(40.321640793727155 73.54705865624933) bank25810 +25811 POINT(40.442178562553174 73.40212927165246) bank25811 +25812 POINT(40.488984572059785 74.59490757576239) bank25812 +25813 POINT(41.104522520159534 73.42642068502315) bank25813 +25814 POINT(41.09136447970162 74.20654276748941) bank25814 +25815 POINT(41.68448461044883 74.72475318750463) bank25815 +25816 POINT(41.403647782397385 73.82221395624485) bank25816 +25817 POINT(40.31919595102497 74.5552845158262) bank25817 +25818 POINT(40.05446511506247 73.58975908474636) bank25818 +25819 POINT(41.34919755260094 74.3778820379822) bank25819 +25820 POINT(39.898652618280465 73.13470196019502) bank25820 +25821 POINT(41.115085304378205 74.21891700934854) bank25821 +25822 POINT(40.603987133634334 73.93890436432626) bank25822 +25823 POINT(39.85183440587115 74.96022608343534) bank25823 +25824 POINT(41.19354946919162 73.13842795825452) bank25824 +25825 POINT(40.77189187975753 73.07113856311025) bank25825 +25826 POINT(41.160223629334695 74.74759786928226) bank25826 +25827 POINT(40.424606866111674 73.5872082913672) bank25827 +25828 POINT(40.04080558570243 73.77361235925025) bank25828 +25829 POINT(40.59519289390124 74.52376464565722) bank25829 +25830 POINT(40.79324479073764 73.84926402584847) bank25830 +25831 POINT(41.110082060896694 73.97329830906395) bank25831 +25832 POINT(40.01538561273033 73.75907042580188) bank25832 +25833 POINT(41.3532103403385 74.09084279110195) bank25833 +25834 POINT(40.7688840586074 73.43521217145097) bank25834 +25835 POINT(39.77727035445861 73.42476065535774) bank25835 +25836 POINT(41.090596616055436 73.13759496966017) bank25836 +25837 POINT(41.67451347591917 74.25970001629645) bank25837 +25838 POINT(39.79616402369514 73.86086046074882) bank25838 +25839 POINT(40.635175278931165 73.61093441813472) bank25839 +25840 POINT(40.370812111357125 74.38291003339155) bank25840 +25841 POINT(39.79003214126405 73.55364953945217) bank25841 +25842 POINT(40.789621356564425 73.97689074389379) bank25842 +25843 POINT(41.108126209939634 73.50066822072812) bank25843 +25844 POINT(40.561712618995244 74.5922288799194) bank25844 +25845 POINT(40.40493673471046 74.59023777755094) bank25845 +25846 POINT(40.18094786161634 74.60655049197636) bank25846 +25847 POINT(40.15239256552004 74.09571271289576) bank25847 +25848 POINT(40.41637005922549 73.03898316822382) bank25848 +25849 POINT(40.073425934037125 74.79117929676293) bank25849 +25850 POINT(41.48076104110767 73.07497996008398) bank25850 +25851 POINT(41.07733996954235 74.07552378348893) bank25851 +25852 POINT(39.82494194654104 74.53312619929088) bank25852 +25853 POINT(41.21646611793173 74.02620778886582) bank25853 +25854 POINT(39.831642766454486 74.68576374926369) bank25854 +25855 POINT(41.17086150178897 73.64982551035904) bank25855 +25856 POINT(40.72292763726199 73.27463233569026) bank25856 +25857 POINT(40.54306889631724 73.86413357565387) bank25857 +25858 POINT(41.28027055882663 73.28272279725766) bank25858 +25859 POINT(40.08651687069154 73.36015917090212) bank25859 +25860 POINT(41.66857934984664 73.05920982869505) bank25860 +25861 POINT(40.00953562732984 73.26811384498012) bank25861 +25862 POINT(40.30477326595142 73.32449522919099) bank25862 +25863 POINT(41.62185591163721 73.9652305501691) bank25863 +25864 POINT(40.07562077022434 74.74639267811418) bank25864 +25865 POINT(40.92632248261842 73.14841043328903) bank25865 +25866 POINT(39.98397846527862 74.27850494363423) bank25866 +25867 POINT(40.98636795097415 74.7672045305423) bank25867 +25868 POINT(41.09221287353844 73.2483034107562) bank25868 +25869 POINT(41.590286548784 73.46114907320961) bank25869 +25870 POINT(40.606679196199615 73.97442815475715) bank25870 +25871 POINT(40.367758998992265 74.08720050932914) bank25871 +25872 POINT(41.563769624532775 73.35898350153576) bank25872 +25873 POINT(41.57894206057715 73.00824288771219) bank25873 +25874 POINT(41.259269445541115 73.83094684356907) bank25874 +25875 POINT(40.02774012328922 73.5948237840557) bank25875 +25876 POINT(40.39176182807853 74.73762711894484) bank25876 +25877 POINT(40.704862554465244 74.70555149606366) bank25877 +25878 POINT(40.39224232257601 74.02823245218077) bank25878 +25879 POINT(40.1287312174948 73.88351468663271) bank25879 +25880 POINT(40.64407660154105 74.60363941679097) bank25880 +25881 POINT(40.788903355050365 73.63745594494361) bank25881 +25882 POINT(39.78636119579104 74.33988223147583) bank25882 +25883 POINT(41.22898845086423 73.0981267156706) bank25883 +25884 POINT(40.06097002478632 74.75637067684421) bank25884 +25885 POINT(39.92352160705709 73.57647786518096) bank25885 +25886 POINT(39.96765103031724 73.28454788359058) bank25886 +25887 POINT(40.90523567572411 74.85011983246875) bank25887 +25888 POINT(39.865631911088805 73.81321499820227) bank25888 +25889 POINT(39.84204449316909 73.6622826316412) bank25889 +25890 POINT(39.75569688873948 73.4250913526146) bank25890 +25891 POINT(40.962126169912935 74.76396591628115) bank25891 +25892 POINT(40.80145804351457 74.87462868209204) bank25892 +25893 POINT(40.635213208665796 73.21905724708971) bank25893 +25894 POINT(41.58567849487321 74.80267487369548) bank25894 +25895 POINT(40.74928136945139 73.5299015183126) bank25895 +25896 POINT(40.329622156155146 73.31793944833929) bank25896 +25897 POINT(41.63799947572383 74.58439496494098) bank25897 +25898 POINT(41.06752644753566 73.42122841284578) bank25898 +25899 POINT(41.690127958665265 74.85338122444622) bank25899 +25900 POINT(41.54320302648776 74.70216271557635) bank25900 +25901 POINT(40.76756855891526 74.51283961126308) bank25901 +25902 POINT(40.062859214216815 73.71534285039803) bank25902 +25903 POINT(41.67600969890006 74.76015485832669) bank25903 +25904 POINT(40.43881873612802 73.62162289272655) bank25904 +25905 POINT(41.43060752652492 74.41049099322079) bank25905 +25906 POINT(41.08930151432558 73.62052657821334) bank25906 +25907 POINT(41.06471189641496 73.85328612382372) bank25907 +25908 POINT(41.59648106860839 74.45860693201473) bank25908 +25909 POINT(39.82857529567283 73.15966353924533) bank25909 +25910 POINT(40.436049247099035 73.43616237649087) bank25910 +25911 POINT(39.906021437225 73.30796287712792) bank25911 +25912 POINT(41.17381552132048 74.94514197111123) bank25912 +25913 POINT(41.61924795170455 73.58804924680534) bank25913 +25914 POINT(40.90627015159501 73.31975971942718) bank25914 +25915 POINT(40.77232609933568 73.11067146650686) bank25915 +25916 POINT(40.71084863816998 74.47735815886064) bank25916 +25917 POINT(39.72583801384549 73.14140109254862) bank25917 +25918 POINT(41.17374387073566 74.84489488041207) bank25918 +25919 POINT(40.84599071056045 73.04678603182501) bank25919 +25920 POINT(39.86009283431277 73.52602024866577) bank25920 +25921 POINT(40.83594163997451 73.18301159684539) bank25921 +25922 POINT(40.2903675884742 74.01225833395378) bank25922 +25923 POINT(41.339037075408534 73.86292449244544) bank25923 +25924 POINT(39.82383346664163 73.69535329046428) bank25924 +25925 POINT(39.909327435952584 73.46165961466197) bank25925 +25926 POINT(40.272506678048934 73.48295773178009) bank25926 +25927 POINT(40.413172913579004 74.19587634428568) bank25927 +25928 POINT(40.83906764956328 74.2647979222409) bank25928 +25929 POINT(40.191846244801724 73.9018370683053) bank25929 +25930 POINT(41.105472881955826 74.5907620924071) bank25930 +25931 POINT(40.9967310292064 74.72959132005285) bank25931 +25932 POINT(41.392844439571476 73.13266747267917) bank25932 +25933 POINT(40.0379476096021 74.62682387288098) bank25933 +25934 POINT(41.053520032166645 74.40835385418649) bank25934 +25935 POINT(40.17177831258945 73.24738799285988) bank25935 +25936 POINT(41.463968620442024 74.29463404268311) bank25936 +25937 POINT(40.36736212296541 73.26426091495969) bank25937 +25938 POINT(41.23457476665362 74.12528427615338) bank25938 +25939 POINT(41.581400105861846 74.19719966881983) bank25939 +25940 POINT(41.406888766047096 73.33069155394985) bank25940 +25941 POINT(39.9222850322369 73.1860274273037) bank25941 +25942 POINT(40.813081999827446 73.48103141251566) bank25942 +25943 POINT(40.105009152482026 73.30652586969448) bank25943 +25944 POINT(39.890661664837125 73.3814804340781) bank25944 +25945 POINT(41.25346774110935 74.53900960676619) bank25945 +25946 POINT(41.3411052315588 74.3992939169004) bank25946 +25947 POINT(41.11751182560736 73.50481667273249) bank25947 +25948 POINT(40.4998530273905 74.85574524795202) bank25948 +25949 POINT(39.73068402348881 73.9152831999218) bank25949 +25950 POINT(41.53793738017132 74.96418075370025) bank25950 +25951 POINT(40.566802053149736 73.03596631811641) bank25951 +25952 POINT(40.765699729346686 74.81278104666099) bank25952 +25953 POINT(40.96818470733417 73.4537423445418) bank25953 +25954 POINT(40.888996742258605 74.95404104562832) bank25954 +25955 POINT(41.36403704229539 73.45686327034635) bank25955 +25956 POINT(40.736791323224715 73.49726829574978) bank25956 +25957 POINT(41.01384116074314 73.92500627779289) bank25957 +25958 POINT(41.61465842081344 73.24837869598824) bank25958 +25959 POINT(40.17083595777234 73.78559173524668) bank25959 +25960 POINT(41.581232179904106 73.16332996081071) bank25960 +25961 POINT(40.72145402954664 74.66786024453545) bank25961 +25962 POINT(41.590034311284356 74.3563006204507) bank25962 +25963 POINT(41.507235850836246 74.22434256403061) bank25963 +25964 POINT(40.6547242273563 74.64784275749463) bank25964 +25965 POINT(41.680813059463134 74.31782832378951) bank25965 +25966 POINT(41.199531675000785 74.93702324410964) bank25966 +25967 POINT(40.50369098861229 73.81134995554997) bank25967 +25968 POINT(40.25181585510835 73.88409979608292) bank25968 +25969 POINT(40.150735281592596 74.86350392487199) bank25969 +25970 POINT(40.16583387865056 74.19712162278236) bank25970 +25971 POINT(41.15557288769669 73.47436267249269) bank25971 +25972 POINT(40.62851021853287 74.97828694437723) bank25972 +25973 POINT(40.99977015943052 74.8328300732207) bank25973 +25974 POINT(39.85953809917412 74.91279830556039) bank25974 +25975 POINT(40.91590360644528 74.05508330499619) bank25975 +25976 POINT(41.390688635783086 73.62876432681595) bank25976 +25977 POINT(39.71792495355473 74.48768459616642) bank25977 +25978 POINT(40.545267380633 73.14054673636556) bank25978 +25979 POINT(41.376419413318025 74.79255651231036) bank25979 +25980 POINT(41.160628066701044 74.32332201907452) bank25980 +25981 POINT(41.24477140512811 74.27255752129255) bank25981 +25982 POINT(40.35364590280332 74.93354074637985) bank25982 +25983 POINT(40.659863657788826 73.11129926242249) bank25983 +25984 POINT(39.72427026503599 73.29445219584935) bank25984 +25985 POINT(39.83289269886866 73.5341675355159) bank25985 +25986 POINT(40.056286683137465 74.32416242596135) bank25986 +25987 POINT(40.13357790835658 73.94267172934393) bank25987 +25988 POINT(39.978529027564015 74.0270990589897) bank25988 +25989 POINT(40.228366226561675 73.79564424242884) bank25989 +25990 POINT(41.06156529136195 74.16794597395294) bank25990 +25991 POINT(40.603247404295594 74.85316505937774) bank25991 +25992 POINT(40.3592866559797 74.23558876558518) bank25992 +25993 POINT(40.967782651964654 74.92014914197428) bank25993 +25994 POINT(39.95477009594142 73.32625532780997) bank25994 +25995 POINT(41.13107632842085 73.88444043061509) bank25995 +25996 POINT(40.119057899678765 73.69061968681446) bank25996 +25997 POINT(41.12921136322778 73.06323526520322) bank25997 +25998 POINT(41.09488479687501 73.7225989260609) bank25998 +25999 POINT(40.61665244253604 73.7452066278361) bank25999 +26000 POINT(41.35451156803361 74.70683134743926) bank26000 +26001 POINT(40.658696487796306 73.6008645303501) bank26001 +26002 POINT(41.05734420768211 74.84538628314564) bank26002 +26003 POINT(40.78953098357162 73.81297563060777) bank26003 +26004 POINT(40.07307635121741 74.7113252356192) bank26004 +26005 POINT(41.591990728979454 74.49206199767902) bank26005 +26006 POINT(40.380294737377874 73.49978041310658) bank26006 +26007 POINT(41.44087187723329 73.03983562911931) bank26007 +26008 POINT(40.122603519895854 74.40124215870284) bank26008 +26009 POINT(41.15438733775791 74.80083772033323) bank26009 +26010 POINT(41.47284552540275 74.57974601718597) bank26010 +26011 POINT(39.99406498722773 73.22851616129785) bank26011 +26012 POINT(41.540261771367376 73.34956614467175) bank26012 +26013 POINT(40.244965544490704 74.08159330266169) bank26013 +26014 POINT(41.53975372407474 74.07251678457288) bank26014 +26015 POINT(40.2717606534285 74.47277521812977) bank26015 +26016 POINT(40.437776751178916 73.90254044422785) bank26016 +26017 POINT(40.57713886147625 73.29729321574081) bank26017 +26018 POINT(40.62597906211101 74.61618445908488) bank26018 +26019 POINT(39.94854631233846 74.11395730617258) bank26019 +26020 POINT(41.701869657916404 74.68505685357287) bank26020 +26021 POINT(40.36664203799269 74.63640740248258) bank26021 +26022 POINT(40.968162853372604 73.97588233382153) bank26022 +26023 POINT(40.97905641187079 74.30991414547715) bank26023 +26024 POINT(39.83368832721071 74.03192176329678) bank26024 +26025 POINT(40.762739600691425 73.9467735832035) bank26025 +26026 POINT(40.54073419092427 74.93293313764511) bank26026 +26027 POINT(41.08612452315712 74.66540380286109) bank26027 +26028 POINT(40.25651250082372 74.73905459914431) bank26028 +26029 POINT(40.99990477960129 74.03000683261548) bank26029 +26030 POINT(41.26312739114652 73.08488043928615) bank26030 +26031 POINT(41.01720446166908 74.01286831325672) bank26031 +26032 POINT(40.48705023962438 73.76371947479004) bank26032 +26033 POINT(40.11813071864378 73.99923194428058) bank26033 +26034 POINT(41.30671083379882 73.6310487129435) bank26034 +26035 POINT(41.7025397487009 74.75057623962759) bank26035 +26036 POINT(40.21680003776195 74.87484881643115) bank26036 +26037 POINT(39.87169630246008 74.87110118867673) bank26037 +26038 POINT(40.6404133552115 74.99265298367682) bank26038 +26039 POINT(40.52790719723156 73.04417014541639) bank26039 +26040 POINT(41.31160347047166 74.93969240639626) bank26040 +26041 POINT(40.01663781033064 74.66544647919623) bank26041 +26042 POINT(40.13709523495279 74.95604550309977) bank26042 +26043 POINT(40.66695126863799 73.18312562757991) bank26043 +26044 POINT(40.99508853294246 73.76154702102447) bank26044 +26045 POINT(40.45391769007074 74.54168126161966) bank26045 +26046 POINT(40.89012470654019 73.0497137497314) bank26046 +26047 POINT(40.44012069298267 74.63857292704927) bank26047 +26048 POINT(41.28528222319745 73.88089182576236) bank26048 +26049 POINT(40.06819032056133 73.03772510196656) bank26049 +26050 POINT(40.249888479287044 74.29061232656576) bank26050 +26051 POINT(41.598521075311794 74.10168869127568) bank26051 +26052 POINT(39.830506846383116 74.64839905478422) bank26052 +26053 POINT(40.97518947232259 73.27068326934582) bank26053 +26054 POINT(41.52603260017011 73.6022716854968) bank26054 +26055 POINT(40.84889312575285 74.11958797275821) bank26055 +26056 POINT(40.35020708850477 74.49709596290694) bank26056 +26057 POINT(41.70971076241406 74.04054172470843) bank26057 +26058 POINT(40.592528407587906 74.27129906171703) bank26058 +26059 POINT(39.9997625196423 73.39646843054668) bank26059 +26060 POINT(40.81184858044511 73.70698367296383) bank26060 +26061 POINT(40.29428758289785 73.00875624331779) bank26061 +26062 POINT(41.26184326561056 74.30245322448603) bank26062 +26063 POINT(40.94236447241501 74.04047973752614) bank26063 +26064 POINT(41.64575197844574 74.11364022441973) bank26064 +26065 POINT(41.63620974448558 73.55903932898391) bank26065 +26066 POINT(41.18702058585291 74.31066768614076) bank26066 +26067 POINT(40.40155032093944 74.21786259573821) bank26067 +26068 POINT(40.08780205656256 73.73303851691497) bank26068 +26069 POINT(40.72559895737191 73.38825769541225) bank26069 +26070 POINT(39.90356014513549 74.24493487050506) bank26070 +26071 POINT(41.59083852739566 74.06341379830312) bank26071 +26072 POINT(40.51743306800093 73.22011704294668) bank26072 +26073 POINT(39.77998922584726 74.87626740598186) bank26073 +26074 POINT(40.11108525083147 73.76161823977205) bank26074 +26075 POINT(41.44620611929738 73.04861732408929) bank26075 +26076 POINT(40.677091713953736 74.91077215147692) bank26076 +26077 POINT(40.336063055248985 73.45210424166194) bank26077 +26078 POINT(40.55185240677531 74.65402627360298) bank26078 +26079 POINT(40.12739283459315 74.39035685247146) bank26079 +26080 POINT(40.74589462454699 74.43253579020025) bank26080 +26081 POINT(40.46754865886811 73.48033030232294) bank26081 +26082 POINT(40.28683334578047 73.08417708846184) bank26082 +26083 POINT(39.73017871068955 73.1862042993692) bank26083 +26084 POINT(39.980597583739026 74.00325477977633) bank26084 +26085 POINT(40.43122450424302 73.31933647393639) bank26085 +26086 POINT(40.25721726266679 74.99831523647597) bank26086 +26087 POINT(39.82665287748214 73.65125492413445) bank26087 +26088 POINT(40.560166563653674 73.37483979870755) bank26088 +26089 POINT(40.26382241164053 74.82231720599113) bank26089 +26090 POINT(40.97028911879233 73.7074852085576) bank26090 +26091 POINT(41.53230436070649 74.85226050608506) bank26091 +26092 POINT(40.87674591952358 73.83364867748257) bank26092 +26093 POINT(41.18315500856497 74.6147765992277) bank26093 +26094 POINT(40.458819890505076 74.14888998967024) bank26094 +26095 POINT(40.81554890430234 73.05526617542507) bank26095 +26096 POINT(40.70509516506273 74.92390517972872) bank26096 +26097 POINT(40.42743543883934 73.52418028226452) bank26097 +26098 POINT(40.56676503802597 74.53659837991178) bank26098 +26099 POINT(40.823996813042676 74.23467820206905) bank26099 +26100 POINT(40.860896185530436 74.90162372628608) bank26100 +26101 POINT(39.929836402942406 73.35653153773747) bank26101 +26102 POINT(39.81519112022385 73.41934119557843) bank26102 +26103 POINT(40.41437317125402 74.26021802196723) bank26103 +26104 POINT(41.29414834838828 73.45206407272589) bank26104 +26105 POINT(41.27130186502077 73.5239846376537) bank26105 +26106 POINT(41.33496019619982 74.98700032842987) bank26106 +26107 POINT(41.18939300631161 74.19208657846991) bank26107 +26108 POINT(40.005407618286206 73.96960691702425) bank26108 +26109 POINT(40.542522699630666 74.99466454468524) bank26109 +26110 POINT(40.85977496234602 74.08048124537673) bank26110 +26111 POINT(40.541973846514566 73.49168134828425) bank26111 +26112 POINT(40.60823338780168 73.8855764527766) bank26112 +26113 POINT(41.40637657969293 73.2149643162899) bank26113 +26114 POINT(41.4063287699878 74.20704256497044) bank26114 +26115 POINT(41.11848464208852 74.83483834720668) bank26115 +26116 POINT(41.23428399771121 74.77847154932597) bank26116 +26117 POINT(40.696114495059106 73.52216999106352) bank26117 +26118 POINT(40.58882327343449 74.91814565274309) bank26118 +26119 POINT(40.462599171919074 73.775083959697) bank26119 +26120 POINT(40.8493005406655 73.08432762080189) bank26120 +26121 POINT(41.63259954664317 74.44856188073703) bank26121 +26122 POINT(40.783266808617824 74.48179865463895) bank26122 +26123 POINT(41.229527655772536 74.01235943978743) bank26123 +26124 POINT(40.866260172463704 73.52807555895498) bank26124 +26125 POINT(40.917309520496836 73.91637905527436) bank26125 +26126 POINT(40.54161049261701 74.94196368091077) bank26126 +26127 POINT(40.734112110528216 73.57401129841305) bank26127 +26128 POINT(40.496349161269286 74.11892007224297) bank26128 +26129 POINT(40.593693317510876 74.91550241462917) bank26129 +26130 POINT(41.34860462821915 74.36454415708793) bank26130 +26131 POINT(39.76355775402759 73.73924201406956) bank26131 +26132 POINT(40.221824308212916 73.19295601866702) bank26132 +26133 POINT(41.341932821396355 74.84922672084699) bank26133 +26134 POINT(41.09240125976423 73.4147809655201) bank26134 +26135 POINT(41.04863517907009 74.48159990896632) bank26135 +26136 POINT(40.23891805256317 74.26458088275379) bank26136 +26137 POINT(40.867332683355684 73.80466125496793) bank26137 +26138 POINT(40.19660049350729 73.10053189905666) bank26138 +26139 POINT(40.52322079886401 73.05834291042761) bank26139 +26140 POINT(41.00854263502817 73.60351771507963) bank26140 +26141 POINT(40.36456076917339 74.27127347119679) bank26141 +26142 POINT(40.20034293021945 73.08499961294604) bank26142 +26143 POINT(40.09119512137647 73.47250221973027) bank26143 +26144 POINT(40.443272863699 73.07660914569448) bank26144 +26145 POINT(40.91056287972509 74.0381326325857) bank26145 +26146 POINT(40.44386862367688 74.4948888516622) bank26146 +26147 POINT(40.280023843588296 73.23713319859493) bank26147 +26148 POINT(40.243351048457335 74.51695672445085) bank26148 +26149 POINT(41.70394566705104 74.62895533329926) bank26149 +26150 POINT(41.69654045963887 74.97376707266149) bank26150 +26151 POINT(41.19704310257829 73.77504789468409) bank26151 +26152 POINT(39.81595473871332 74.56106728260393) bank26152 +26153 POINT(41.457754172150935 73.34614836921014) bank26153 +26154 POINT(41.605931403371834 73.73276809765274) bank26154 +26155 POINT(41.371595921304376 73.02842565350673) bank26155 +26156 POINT(41.637977703968446 73.60174977362482) bank26156 +26157 POINT(40.52804142822748 73.43641069307073) bank26157 +26158 POINT(41.256342162704776 74.34696491793463) bank26158 +26159 POINT(40.663201053300924 73.11076858691521) bank26159 +26160 POINT(40.63355454221891 74.78489464585518) bank26160 +26161 POINT(41.61154336131102 73.78942027621193) bank26161 +26162 POINT(41.05621627207505 73.94554221806924) bank26162 +26163 POINT(40.9145173951572 74.90216703103296) bank26163 +26164 POINT(40.61831672043567 73.92969572362834) bank26164 +26165 POINT(40.15804352776679 74.48330076300687) bank26165 +26166 POINT(41.44631726682469 74.2284351955738) bank26166 +26167 POINT(39.74151929394458 73.05794289704384) bank26167 +26168 POINT(40.67797228397679 74.1759061500301) bank26168 +26169 POINT(41.20474535384288 74.06129306348555) bank26169 +26170 POINT(40.86576460017618 73.52943226586699) bank26170 +26171 POINT(39.85313506376279 74.9548897670329) bank26171 +26172 POINT(41.63059260036962 73.37510865884133) bank26172 +26173 POINT(41.48670311061498 73.78596279028338) bank26173 +26174 POINT(40.610780174055925 74.96094078495317) bank26174 +26175 POINT(40.9709231379693 74.77039400780271) bank26175 +26176 POINT(41.384236394683754 74.82847151760096) bank26176 +26177 POINT(41.59606243386586 74.32747353853661) bank26177 +26178 POINT(40.697785652472284 74.6496493375478) bank26178 +26179 POINT(41.55790115705094 73.3451276842342) bank26179 +26180 POINT(40.2784989317178 73.31005981975596) bank26180 +26181 POINT(40.700170619628764 73.80472606424783) bank26181 +26182 POINT(41.05525043727949 74.43704569985093) bank26182 +26183 POINT(39.80936380954646 73.39785223006338) bank26183 +26184 POINT(41.294176847963406 73.05934801056) bank26184 +26185 POINT(41.022121721191716 74.09586583980297) bank26185 +26186 POINT(40.21232868622434 73.11976455540729) bank26186 +26187 POINT(40.532315474472405 74.6726486131744) bank26187 +26188 POINT(41.13923832530117 74.61532843981746) bank26188 +26189 POINT(41.68261433092623 73.1483834054363) bank26189 +26190 POINT(40.50030191355112 73.82886210013137) bank26190 +26191 POINT(41.44372692951209 73.68179763974089) bank26191 +26192 POINT(40.61885906950618 74.2534794376114) bank26192 +26193 POINT(40.09913832316522 74.3371891520757) bank26193 +26194 POINT(40.9117401038049 73.15716379192813) bank26194 +26195 POINT(40.86255282744979 73.33165617332448) bank26195 +26196 POINT(40.69006843434513 74.13663484924184) bank26196 +26197 POINT(41.325227780982836 73.55861543102937) bank26197 +26198 POINT(40.21154937852648 73.10151213326724) bank26198 +26199 POINT(40.68467457777652 73.99790713782727) bank26199 +26200 POINT(40.976878287503254 74.08582245140023) bank26200 +26201 POINT(41.36979299489219 74.44076950361305) bank26201 +26202 POINT(40.157337643254365 73.68312003418653) bank26202 +26203 POINT(40.73315013648761 73.70441322828702) bank26203 +26204 POINT(40.265919175789236 73.32434559584154) bank26204 +26205 POINT(41.60582370627639 74.2776489879802) bank26205 +26206 POINT(41.29898977341988 74.26599021343289) bank26206 +26207 POINT(40.58960362620572 74.71002348291661) bank26207 +26208 POINT(41.287158073771614 74.07034878638562) bank26208 +26209 POINT(40.224822487934674 74.10448816988483) bank26209 +26210 POINT(41.28590221907809 74.27530256849097) bank26210 +26211 POINT(41.49541440985579 73.8271311951243) bank26211 +26212 POINT(41.15785479136617 73.21743131134392) bank26212 +26213 POINT(40.018356695670725 73.77770075956805) bank26213 +26214 POINT(41.05163358765654 73.86504867570756) bank26214 +26215 POINT(40.799166462278826 73.75998714314689) bank26215 +26216 POINT(41.266726588604044 73.47587578191651) bank26216 +26217 POINT(40.27519389592185 73.97837265246375) bank26217 +26218 POINT(39.921411358874984 74.13898477176194) bank26218 +26219 POINT(40.65273784538585 73.08503631295862) bank26219 +26220 POINT(40.04385635984501 73.88750429840914) bank26220 +26221 POINT(40.91807566351184 73.08526781523894) bank26221 +26222 POINT(40.231158078909246 74.18190744945379) bank26222 +26223 POINT(40.325877747157406 73.49379654438643) bank26223 +26224 POINT(41.69411467977237 74.38499589078845) bank26224 +26225 POINT(41.14405195848395 74.9411531634105) bank26225 +26226 POINT(40.7975887871992 73.86270627890796) bank26226 +26227 POINT(41.31723357834229 74.92428423899564) bank26227 +26228 POINT(41.50473317236934 73.34971397638685) bank26228 +26229 POINT(40.68503926536703 73.63060613048441) bank26229 +26230 POINT(41.60786974496232 74.0984689777036) bank26230 +26231 POINT(41.379699835884075 74.99801133232926) bank26231 +26232 POINT(41.41780928124729 74.88537353251026) bank26232 +26233 POINT(40.69140071675743 73.17195545779065) bank26233 +26234 POINT(40.20891299873674 74.80051172714224) bank26234 +26235 POINT(41.543388628610586 74.12158018824401) bank26235 +26236 POINT(41.04527701749815 74.61437685038379) bank26236 +26237 POINT(40.88386601353803 73.44187955745309) bank26237 +26238 POINT(40.65086239801444 74.3764506345513) bank26238 +26239 POINT(40.62331466010117 73.27787514988859) bank26239 +26240 POINT(40.75745888670911 74.30798898235295) bank26240 +26241 POINT(41.22015643629213 73.01257220080724) bank26241 +26242 POINT(40.071061927861294 74.4401604171286) bank26242 +26243 POINT(41.11238697437605 74.3281882720014) bank26243 +26244 POINT(40.65643554527761 73.19795277425831) bank26244 +26245 POINT(41.01305988428981 73.35694678359881) bank26245 +26246 POINT(39.84169057798463 74.46818310532095) bank26246 +26247 POINT(39.8676872518207 73.45479859895137) bank26247 +26248 POINT(40.20488834491683 73.16956540092824) bank26248 +26249 POINT(40.19616709820847 74.89030061935402) bank26249 +26250 POINT(40.140136240474334 73.1128682382218) bank26250 +26251 POINT(39.94764905507276 73.86064640633968) bank26251 +26252 POINT(41.612046198173225 74.89173243858389) bank26252 +26253 POINT(40.98314183852672 73.60968613962002) bank26253 +26254 POINT(40.430943231030525 73.6049456955459) bank26254 +26255 POINT(41.311952432739666 73.94655486522846) bank26255 +26256 POINT(41.36881172296687 74.75702331340763) bank26256 +26257 POINT(39.92928794167836 73.7492842464879) bank26257 +26258 POINT(40.735361857641124 73.06313136932722) bank26258 +26259 POINT(39.84582682591727 74.43341744231205) bank26259 +26260 POINT(41.09153010079598 74.08016920386777) bank26260 +26261 POINT(41.27017238203868 74.12155881407665) bank26261 +26262 POINT(40.0035503698002 74.69239286475536) bank26262 +26263 POINT(39.839130371009446 74.3627791691309) bank26263 +26264 POINT(39.74754626534236 74.803460417375) bank26264 +26265 POINT(40.17039863940412 74.71857403628793) bank26265 +26266 POINT(40.96205080748188 73.20117439544036) bank26266 +26267 POINT(41.321791374723816 74.49431163342567) bank26267 +26268 POINT(41.61048088669808 74.99193185788876) bank26268 +26269 POINT(40.22298967337583 74.26864930232553) bank26269 +26270 POINT(39.83513345722294 74.91767059495793) bank26270 +26271 POINT(40.508854604428656 74.48635132934206) bank26271 +26272 POINT(40.373751499892016 73.30070281595344) bank26272 +26273 POINT(41.093387296068734 73.92925947072601) bank26273 +26274 POINT(40.4757475998884 73.750619661357) bank26274 +26275 POINT(41.50493320902892 74.48355352452212) bank26275 +26276 POINT(40.97412047752318 74.02347181373544) bank26276 +26277 POINT(40.80946406658446 73.70654402448687) bank26277 +26278 POINT(41.2876497060684 74.98576370137009) bank26278 +26279 POINT(40.96935569417174 73.92572383904084) bank26279 +26280 POINT(41.30016330152256 73.52348269075523) bank26280 +26281 POINT(41.25347273979796 73.61126565465791) bank26281 +26282 POINT(41.53624063124365 73.33413564522226) bank26282 +26283 POINT(41.35373227757371 73.15640870350235) bank26283 +26284 POINT(40.885745548932654 74.71387375277754) bank26284 +26285 POINT(40.933679763957215 74.26854205060259) bank26285 +26286 POINT(40.53934044721634 73.43908920419997) bank26286 +26287 POINT(40.45308334832988 73.97490551395569) bank26287 +26288 POINT(40.37618925120507 74.63091903009231) bank26288 +26289 POINT(39.83061873387822 74.718720725434) bank26289 +26290 POINT(41.661434468774665 74.29829632230233) bank26290 +26291 POINT(39.94992206271728 74.47206363089686) bank26291 +26292 POINT(41.6345172430875 74.77897159029001) bank26292 +26293 POINT(40.08637093316966 74.52048456318245) bank26293 +26294 POINT(40.982170958804744 73.04809117865386) bank26294 +26295 POINT(40.418029609173104 74.08215064722467) bank26295 +26296 POINT(40.533302667665666 74.37593432494123) bank26296 +26297 POINT(41.31143124606617 73.7726294421175) bank26297 +26298 POINT(41.394657760888734 74.1576525073371) bank26298 +26299 POINT(40.996104045790986 73.58051231683103) bank26299 +26300 POINT(40.0292626266226 73.66899366060152) bank26300 +26301 POINT(39.72065407541801 73.57912774395442) bank26301 +26302 POINT(40.54832282730452 73.51129662773535) bank26302 +26303 POINT(40.6912521526142 74.09638372878653) bank26303 +26304 POINT(40.31595112370631 74.77191794252857) bank26304 +26305 POINT(41.41213674848827 74.48468293734213) bank26305 +26306 POINT(40.05038837445912 74.05497005684762) bank26306 +26307 POINT(40.86733520255039 73.46015975389982) bank26307 +26308 POINT(40.24794000191332 74.61967427341332) bank26308 +26309 POINT(40.971418719020384 74.87945349773133) bank26309 +26310 POINT(41.15165256449487 73.83690688245764) bank26310 +26311 POINT(41.14307290920549 74.74616069668636) bank26311 +26312 POINT(41.14481122799523 73.3572231223297) bank26312 +26313 POINT(39.73850037311333 73.77547021055553) bank26313 +26314 POINT(40.64337788591027 74.57579402126912) bank26314 +26315 POINT(40.299717276424836 74.0789237773512) bank26315 +26316 POINT(41.532730494123896 73.42731575407558) bank26316 +26317 POINT(40.38196179994062 74.38227079153339) bank26317 +26318 POINT(41.400528547741175 74.9174799105556) bank26318 +26319 POINT(41.09605681971793 73.64688930774354) bank26319 +26320 POINT(40.347540278253746 74.89592396491605) bank26320 +26321 POINT(39.71743446285729 74.5454085027169) bank26321 +26322 POINT(40.43110310730841 73.25663986179866) bank26322 +26323 POINT(41.0217227642431 73.08929078178657) bank26323 +26324 POINT(40.35371495889175 74.9579313586389) bank26324 +26325 POINT(40.36657997382988 74.26766182704816) bank26325 +26326 POINT(41.06495631749621 74.5368005140476) bank26326 +26327 POINT(41.52641783810149 74.65393140553586) bank26327 +26328 POINT(41.09715497570108 74.89305875512729) bank26328 +26329 POINT(39.94168143272188 73.98397186335428) bank26329 +26330 POINT(40.24981150495506 73.0538174982879) bank26330 +26331 POINT(40.47871003343971 74.56549637886351) bank26331 +26332 POINT(40.987221601859865 73.7585542138536) bank26332 +26333 POINT(41.6675476958044 73.86902132270136) bank26333 +26334 POINT(41.64865998962692 73.97177276196675) bank26334 +26335 POINT(40.46963953218825 73.72095356548317) bank26335 +26336 POINT(40.88696086720243 74.20869487059713) bank26336 +26337 POINT(40.95915176079014 73.56617822067643) bank26337 +26338 POINT(41.018136863718645 73.4289271496866) bank26338 +26339 POINT(40.425712079734616 73.68226009058984) bank26339 +26340 POINT(40.23581506728181 73.30953018355399) bank26340 +26341 POINT(40.28390857113766 74.59616765835331) bank26341 +26342 POINT(41.237580619061774 74.99669410682137) bank26342 +26343 POINT(41.091712648160424 73.6517507648139) bank26343 +26344 POINT(40.9750287664933 73.6750668623412) bank26344 +26345 POINT(40.86192859327829 73.83379215235325) bank26345 +26346 POINT(41.481697656339755 73.39860096310355) bank26346 +26347 POINT(41.35981591338722 73.1833281555627) bank26347 +26348 POINT(41.00720586039862 74.08175409718858) bank26348 +26349 POINT(40.90965020079986 74.37801810940869) bank26349 +26350 POINT(39.98698777829569 74.11192348489142) bank26350 +26351 POINT(41.351539770037114 73.19816296794122) bank26351 +26352 POINT(40.151867379958524 73.7053938269578) bank26352 +26353 POINT(41.45566024656264 74.73343188763491) bank26353 +26354 POINT(39.721212350602606 74.13106504151735) bank26354 +26355 POINT(40.2361343111905 74.43241448710941) bank26355 +26356 POINT(41.57570434414781 73.34351477189882) bank26356 +26357 POINT(41.174506260565046 73.40503760243254) bank26357 +26358 POINT(40.440207379797215 74.90273801462976) bank26358 +26359 POINT(40.742699549937825 73.47596122953405) bank26359 +26360 POINT(40.04267766463688 74.27848246200288) bank26360 +26361 POINT(40.06218074683008 73.9617521385132) bank26361 +26362 POINT(39.85129526536372 74.78104194036095) bank26362 +26363 POINT(40.909209520647266 73.28418017981541) bank26363 +26364 POINT(40.29394870103572 73.62467999968948) bank26364 +26365 POINT(40.2628595813046 74.71567502072965) bank26365 +26366 POINT(40.65149698465719 73.32349610614497) bank26366 +26367 POINT(41.04890237451626 74.32652671986958) bank26367 +26368 POINT(39.77282376173307 74.92269347381392) bank26368 +26369 POINT(40.73607879528275 73.0063278188077) bank26369 +26370 POINT(41.579393969137726 74.17097215388036) bank26370 +26371 POINT(39.79534322180949 73.18089097499734) bank26371 +26372 POINT(39.90530355080403 73.1613204111679) bank26372 +26373 POINT(40.03648329043453 73.80346414702394) bank26373 +26374 POINT(41.532424630080214 74.80342835721656) bank26374 +26375 POINT(40.885445603830526 73.53048756098138) bank26375 +26376 POINT(40.25276636413917 74.60860604788566) bank26376 +26377 POINT(40.15019739924067 74.54934959676386) bank26377 +26378 POINT(41.04450884577337 73.56604297464384) bank26378 +26379 POINT(40.10751774002545 74.20423084108923) bank26379 +26380 POINT(39.97299310917588 73.07008966887697) bank26380 +26381 POINT(40.90100342255243 73.17897909713611) bank26381 +26382 POINT(41.46181114225523 73.58730683072793) bank26382 +26383 POINT(40.62851048058227 74.54643549764427) bank26383 +26384 POINT(41.54893021458876 74.25709181293263) bank26384 +26385 POINT(41.51231539217413 74.53444852841344) bank26385 +26386 POINT(41.50810902545687 73.11558828147871) bank26386 +26387 POINT(41.068121599046805 74.77055178005617) bank26387 +26388 POINT(41.62004153962999 73.73331015889656) bank26388 +26389 POINT(40.49231021383446 73.56074201210983) bank26389 +26390 POINT(39.875834101367474 73.01748334238451) bank26390 +26391 POINT(40.12611644583178 73.55496094632066) bank26391 +26392 POINT(40.63489283657446 74.40760936137625) bank26392 +26393 POINT(40.46389718949077 74.88004896036425) bank26393 +26394 POINT(40.55486949092923 74.00521236729169) bank26394 +26395 POINT(40.031065889568204 73.9093476800794) bank26395 +26396 POINT(40.82118085221063 73.84373565735086) bank26396 +26397 POINT(41.0675426338646 73.65083332644056) bank26397 +26398 POINT(40.75654046751623 74.8943815474385) bank26398 +26399 POINT(39.858269320261044 73.0188999471682) bank26399 +26400 POINT(41.65650601216516 73.94570402384994) bank26400 +26401 POINT(41.446828096776564 74.65627910865756) bank26401 +26402 POINT(40.95296852803781 74.12645064163738) bank26402 +26403 POINT(40.94638800917097 73.3549904816642) bank26403 +26404 POINT(40.702090317708794 74.64146153126785) bank26404 +26405 POINT(40.67088516802449 74.28919932774419) bank26405 +26406 POINT(40.1649519233264 73.41067313356479) bank26406 +26407 POINT(40.51836699571237 74.70992799916283) bank26407 +26408 POINT(41.494988218394816 74.31235887374696) bank26408 +26409 POINT(40.72282222719173 73.31183522503956) bank26409 +26410 POINT(41.191191578842044 74.50939680774087) bank26410 +26411 POINT(40.17076842460755 73.31460351417253) bank26411 +26412 POINT(40.2367498310499 74.43802677849207) bank26412 +26413 POINT(41.590737184849324 73.16477885685485) bank26413 +26414 POINT(40.21243799571101 73.34444309529749) bank26414 +26415 POINT(40.17002052413464 73.78087444892168) bank26415 +26416 POINT(41.59939042875825 74.21970647130587) bank26416 +26417 POINT(41.22284006663992 73.28845258919596) bank26417 +26418 POINT(39.85220880681132 74.83591002685692) bank26418 +26419 POINT(39.75440694190855 74.67582901758705) bank26419 +26420 POINT(41.31369037978116 74.05354927843779) bank26420 +26421 POINT(39.8877116955712 74.45358743964393) bank26421 +26422 POINT(39.83316715500075 74.3311014321217) bank26422 +26423 POINT(40.536744215524216 73.0830660460899) bank26423 +26424 POINT(40.84246692572912 73.95644896006006) bank26424 +26425 POINT(40.1683807854224 73.7158358171328) bank26425 +26426 POINT(41.684393550666165 73.16307307109749) bank26426 +26427 POINT(40.57983176550004 74.24749844228033) bank26427 +26428 POINT(39.830762445296585 74.56589681193341) bank26428 +26429 POINT(41.47189288163053 74.19658994874528) bank26429 +26430 POINT(40.71560041068025 73.42039114813772) bank26430 +26431 POINT(41.64830540022969 73.05286382515153) bank26431 +26432 POINT(40.03961559228305 74.73694788193761) bank26432 +26433 POINT(41.437721168030464 74.90208221503498) bank26433 +26434 POINT(41.51319099076549 73.54962719981266) bank26434 +26435 POINT(40.53590108166301 74.03578059626291) bank26435 +26436 POINT(41.60697479293837 74.68009636490913) bank26436 +26437 POINT(40.516954484611276 74.81544682537923) bank26437 +26438 POINT(41.07424615078831 74.75397807107579) bank26438 +26439 POINT(41.32414497119626 73.05053544759959) bank26439 +26440 POINT(40.36408720695754 73.52001729206404) bank26440 +26441 POINT(40.10881839862673 73.38478411570793) bank26441 +26442 POINT(41.66223370130348 73.9517011341021) bank26442 +26443 POINT(40.81732977609568 74.89225481165407) bank26443 +26444 POINT(39.83605025850437 73.1219852761484) bank26444 +26445 POINT(41.192129647498035 74.40131978142333) bank26445 +26446 POINT(41.557274760090415 73.11544106077973) bank26446 +26447 POINT(41.68654194557534 74.62858630214222) bank26447 +26448 POINT(40.3608318907446 73.72344465110963) bank26448 +26449 POINT(41.320901146220194 73.06151982490788) bank26449 +26450 POINT(40.36651679783082 73.53651444950863) bank26450 +26451 POINT(41.28447345406833 74.22246644884883) bank26451 +26452 POINT(40.15578345579076 74.78522459281537) bank26452 +26453 POINT(39.84538890939078 73.03864757329762) bank26453 +26454 POINT(39.85721465046799 73.35276715628677) bank26454 +26455 POINT(41.50132322373895 73.7341162918236) bank26455 +26456 POINT(41.63746552541622 74.58094108419158) bank26456 +26457 POINT(40.63004637468029 73.76843429826968) bank26457 +26458 POINT(40.117906143856956 74.525196169143) bank26458 +26459 POINT(40.20874565649784 74.34831456588478) bank26459 +26460 POINT(40.814882715625515 74.75306383830295) bank26460 +26461 POINT(41.3728651171963 73.20750629951839) bank26461 +26462 POINT(41.409139427473306 74.8781629937114) bank26462 +26463 POINT(40.166319416569046 74.99916921245519) bank26463 +26464 POINT(40.19167714616997 73.36017875329193) bank26464 +26465 POINT(40.83233087497616 74.9052055282182) bank26465 +26466 POINT(40.40360393934497 73.80585034056628) bank26466 +26467 POINT(40.74998030923377 74.05794219818536) bank26467 +26468 POINT(40.70203179837861 73.61690732299226) bank26468 +26469 POINT(41.19594902703484 73.0826975341422) bank26469 +26470 POINT(40.76439546446944 73.06222798803823) bank26470 +26471 POINT(41.453370949735444 73.48184426803861) bank26471 +26472 POINT(40.98381675253909 73.89607094056936) bank26472 +26473 POINT(40.748127508695156 74.6494943840452) bank26473 +26474 POINT(40.12375500176192 73.5029779055516) bank26474 +26475 POINT(40.59153801592821 73.00902234260684) bank26475 +26476 POINT(40.918601610951995 74.94097583205654) bank26476 +26477 POINT(41.0348551592242 73.91213207181167) bank26477 +26478 POINT(41.22380677294078 73.15486679151151) bank26478 +26479 POINT(40.0012012214975 73.08274760584999) bank26479 +26480 POINT(40.082326022696705 73.88692165239964) bank26480 +26481 POINT(40.455197278567 74.44510036975893) bank26481 +26482 POINT(41.651590660267146 73.76242241773134) bank26482 +26483 POINT(41.573443815232714 73.57002582777328) bank26483 +26484 POINT(40.526767588122745 74.03023771409299) bank26484 +26485 POINT(40.1471384176966 74.41329780772254) bank26485 +26486 POINT(41.17001946361223 74.27261970336563) bank26486 +26487 POINT(40.63793206379411 74.34630151536075) bank26487 +26488 POINT(40.35094335156292 73.76091546440034) bank26488 +26489 POINT(39.76918078428389 73.43710115782444) bank26489 +26490 POINT(40.67315779893796 74.31184222420698) bank26490 +26491 POINT(41.62962763157815 73.14960488496894) bank26491 +26492 POINT(39.98976704075571 73.06978993246557) bank26492 +26493 POINT(39.86308275771834 74.8076269732329) bank26493 +26494 POINT(41.69232275079995 74.0659226837864) bank26494 +26495 POINT(40.42774052305072 73.1941718921867) bank26495 +26496 POINT(40.052827999050905 73.86711099620663) bank26496 +26497 POINT(40.25486635292658 73.40746210618887) bank26497 +26498 POINT(39.75889959880191 73.83668995311541) bank26498 +26499 POINT(39.969946044010506 73.83807414449734) bank26499 +26500 POINT(41.68944248734668 73.90895205205841) bank26500 +26501 POINT(40.562664760022784 73.73647619435617) bank26501 +26502 POINT(39.99972234918936 74.99393356434517) bank26502 +26503 POINT(41.67439692173042 73.91542956716278) bank26503 +26504 POINT(40.56675202489699 73.91317815748577) bank26504 +26505 POINT(40.30405856589149 73.30316602709533) bank26505 +26506 POINT(40.88892421639354 74.84448009534277) bank26506 +26507 POINT(41.10464704253644 73.74401958893817) bank26507 +26508 POINT(40.57638676329423 73.55877938682691) bank26508 +26509 POINT(41.06499706336627 73.63386406923105) bank26509 +26510 POINT(40.29063574586223 74.2836388037749) bank26510 +26511 POINT(40.223168995399945 74.21268880235846) bank26511 +26512 POINT(39.91205512713382 74.55871211160513) bank26512 +26513 POINT(40.743787933442334 73.46578047563776) bank26513 +26514 POINT(39.73101449689902 73.69626021195708) bank26514 +26515 POINT(40.051866518862795 74.2863386140792) bank26515 +26516 POINT(40.02446548753489 74.82237882113702) bank26516 +26517 POINT(39.73085656204331 74.25660693080894) bank26517 +26518 POINT(41.397617841234634 73.9510160448795) bank26518 +26519 POINT(40.03457182369022 74.70630977266478) bank26519 +26520 POINT(41.04967960739379 74.86550671263215) bank26520 +26521 POINT(40.56996830649504 74.24830576884973) bank26521 +26522 POINT(41.50601271786282 73.40345165204096) bank26522 +26523 POINT(40.3939652678572 73.44571226604434) bank26523 +26524 POINT(41.28406197569814 74.56522735184588) bank26524 +26525 POINT(40.97325204657845 74.74520819564788) bank26525 +26526 POINT(40.07609800419283 74.9619916938103) bank26526 +26527 POINT(40.46290573163377 74.46300192722329) bank26527 +26528 POINT(41.478975187911075 73.49180956382503) bank26528 +26529 POINT(40.5124716595435 73.14169966320975) bank26529 +26530 POINT(41.67833834763772 74.52543253539021) bank26530 +26531 POINT(39.80609555842947 74.2913125221172) bank26531 +26532 POINT(41.39130325939668 73.32478512781532) bank26532 +26533 POINT(41.04028348942574 74.28596141428139) bank26533 +26534 POINT(40.51151311908348 73.74996551751428) bank26534 +26535 POINT(41.68213358726943 73.77500716755631) bank26535 +26536 POINT(39.739041691216315 74.55536795291636) bank26536 +26537 POINT(40.58123963309622 74.31134088322187) bank26537 +26538 POINT(40.26882256313064 74.42802874818074) bank26538 +26539 POINT(40.79715184591738 74.93831155421694) bank26539 +26540 POINT(41.503065107652425 73.3905497047199) bank26540 +26541 POINT(40.056848720406464 74.57462364791847) bank26541 +26542 POINT(41.06726800294036 73.84259348062162) bank26542 +26543 POINT(40.967859852364334 73.15906982456481) bank26543 +26544 POINT(40.45814881025829 74.27670816957972) bank26544 +26545 POINT(41.017374251490246 73.8965483527786) bank26545 +26546 POINT(41.51496408032237 73.62577828810728) bank26546 +26547 POINT(40.90548224313909 73.92042538582083) bank26547 +26548 POINT(41.05009956450358 73.23124763727593) bank26548 +26549 POINT(40.037175002893164 74.18008228026734) bank26549 +26550 POINT(41.0597085452768 74.4584033362197) bank26550 +26551 POINT(41.04549646473323 73.6690005501065) bank26551 +26552 POINT(41.46853861328705 74.01561433198758) bank26552 +26553 POINT(41.24207887006964 73.14482785261423) bank26553 +26554 POINT(40.376654442465835 74.73802322362309) bank26554 +26555 POINT(40.81806338245873 74.93980880991388) bank26555 +26556 POINT(40.634251123450134 73.74883318480097) bank26556 +26557 POINT(40.33057018300496 74.86288168526022) bank26557 +26558 POINT(41.549937497928035 74.66250827684031) bank26558 +26559 POINT(41.04345095487175 74.63291041621235) bank26559 +26560 POINT(40.28843811271844 73.73117452195237) bank26560 +26561 POINT(39.80209945625436 73.75928889729423) bank26561 +26562 POINT(40.119121187581456 73.63728135741329) bank26562 +26563 POINT(39.73728522758719 74.72709375261377) bank26563 +26564 POINT(40.74756456346869 74.8290578721177) bank26564 +26565 POINT(39.99925736234838 73.9960557749206) bank26565 +26566 POINT(40.51614016018508 74.65801918379229) bank26566 +26567 POINT(41.606849537936625 74.51943088080245) bank26567 +26568 POINT(40.727745991545014 73.64873233617136) bank26568 +26569 POINT(40.689066836298245 73.83998582662193) bank26569 +26570 POINT(40.27900337553838 73.40752952140723) bank26570 +26571 POINT(40.695986498907374 73.59829412854302) bank26571 +26572 POINT(39.840315389083514 74.83092746089494) bank26572 +26573 POINT(41.18145907316916 73.25783832042148) bank26573 +26574 POINT(41.09690034888153 73.77436482421137) bank26574 +26575 POINT(40.37304835715299 73.5438704342893) bank26575 +26576 POINT(40.94995559885395 73.02861699314799) bank26576 +26577 POINT(41.289028060845425 74.58629763271612) bank26577 +26578 POINT(41.20356260026698 74.59235105678445) bank26578 +26579 POINT(41.36355703898202 74.76521508530776) bank26579 +26580 POINT(40.752514379739324 73.0307956528588) bank26580 +26581 POINT(41.16250166667122 73.12895674117266) bank26581 +26582 POINT(40.57717248255162 73.44021926225804) bank26582 +26583 POINT(41.52929826450742 73.99849583241878) bank26583 +26584 POINT(40.445003703476644 73.03141171970533) bank26584 +26585 POINT(40.250282510514936 73.09193103139376) bank26585 +26586 POINT(40.764612836193514 73.28676279127525) bank26586 +26587 POINT(41.184068911260646 74.85449180686847) bank26587 +26588 POINT(40.72163957657157 74.79556884484799) bank26588 +26589 POINT(40.53562587992717 74.81583911393474) bank26589 +26590 POINT(41.44070414848693 74.51569438429516) bank26590 +26591 POINT(40.7994370420799 73.83873221025307) bank26591 +26592 POINT(41.030680419074585 74.6710621290953) bank26592 +26593 POINT(39.95485100762467 74.99087620410913) bank26593 +26594 POINT(41.19134259711928 73.21342370342738) bank26594 +26595 POINT(40.122145619294784 74.79883152408884) bank26595 +26596 POINT(40.89607951172884 73.70052279431826) bank26596 +26597 POINT(40.01732065822704 74.18953754333225) bank26597 +26598 POINT(40.44204413812392 74.4555998214794) bank26598 +26599 POINT(40.56151543329882 73.18055700072013) bank26599 +26600 POINT(39.71407851334257 74.13700901112888) bank26600 +26601 POINT(40.97123057623126 74.01863223217293) bank26601 +26602 POINT(41.54680949787798 73.36443582218666) bank26602 +26603 POINT(40.46440697779417 74.41351026871548) bank26603 +26604 POINT(40.25553413685842 74.81778290654746) bank26604 +26605 POINT(40.02342347531583 73.85325321446267) bank26605 +26606 POINT(41.57264771533391 73.77904901753834) bank26606 +26607 POINT(40.84353400683196 73.10437758656313) bank26607 +26608 POINT(40.42909099981432 73.16080259445793) bank26608 +26609 POINT(41.17997442542585 73.519582028422) bank26609 +26610 POINT(39.92754118270031 73.73011926415832) bank26610 +26611 POINT(39.81822876267805 74.23885013299149) bank26611 +26612 POINT(40.94308691923202 74.07414738225432) bank26612 +26613 POINT(39.878705491441515 73.26326874841229) bank26613 +26614 POINT(41.28415615679234 73.85437253175604) bank26614 +26615 POINT(40.81456746479305 73.54672103228313) bank26615 +26616 POINT(40.79496759808861 74.08899871038824) bank26616 +26617 POINT(39.952184342048746 74.75010728211363) bank26617 +26618 POINT(40.16211824830491 73.76343517361022) bank26618 +26619 POINT(40.57440067544415 74.74582630879887) bank26619 +26620 POINT(40.144677655338356 74.33245011649073) bank26620 +26621 POINT(41.61611369414922 74.51613052988108) bank26621 +26622 POINT(40.47433309525484 73.65987531999725) bank26622 +26623 POINT(40.96183291056312 74.03103144673469) bank26623 +26624 POINT(40.76897853395332 74.47192711991043) bank26624 +26625 POINT(41.39299525759363 73.61818847010633) bank26625 +26626 POINT(41.42409619125697 73.16489997983616) bank26626 +26627 POINT(40.300206864798014 73.74695222617505) bank26627 +26628 POINT(40.67808986880739 74.21908582395524) bank26628 +26629 POINT(41.42291412587658 73.72270510222306) bank26629 +26630 POINT(40.14201847803539 73.1698666741044) bank26630 +26631 POINT(41.652291626901636 73.62605533654664) bank26631 +26632 POINT(39.855936033065646 74.30941716882312) bank26632 +26633 POINT(41.4789394525121 73.00608892640133) bank26633 +26634 POINT(40.80832664661203 73.50027147710935) bank26634 +26635 POINT(41.68491194102362 74.1566895718942) bank26635 +26636 POINT(41.1389670487942 74.78615228737672) bank26636 +26637 POINT(40.49917373170128 74.29067752814389) bank26637 +26638 POINT(40.369664327437306 74.7110404797801) bank26638 +26639 POINT(41.63701752213314 74.49770935177328) bank26639 +26640 POINT(40.28164196475397 74.954149941073) bank26640 +26641 POINT(40.44374161367015 73.86757577224506) bank26641 +26642 POINT(40.59551155980388 73.95308236944624) bank26642 +26643 POINT(40.2406822728152 74.91403752625355) bank26643 +26644 POINT(41.496419032734885 74.25772354527338) bank26644 +26645 POINT(41.5812921857062 73.56603791185604) bank26645 +26646 POINT(41.68209523347961 74.16164635245269) bank26646 +26647 POINT(39.78248077092215 74.82341309692818) bank26647 +26648 POINT(39.803190124690225 74.08202254315114) bank26648 +26649 POINT(40.042463312322205 73.26983727978161) bank26649 +26650 POINT(41.06215394765977 74.37724761571454) bank26650 +26651 POINT(41.438822125457996 74.46574260106638) bank26651 +26652 POINT(40.00639270133422 73.27240244438502) bank26652 +26653 POINT(39.936698754217716 74.59049182593857) bank26653 +26654 POINT(41.23865899721453 73.65253841792924) bank26654 +26655 POINT(41.552013255665436 73.28699633991286) bank26655 +26656 POINT(40.03753785142906 74.92438637185043) bank26656 +26657 POINT(40.80242043274855 73.60231702510262) bank26657 +26658 POINT(40.184558652128906 74.56892342078302) bank26658 +26659 POINT(40.121538699818444 73.2946955014237) bank26659 +26660 POINT(40.54287584563877 74.57188202120099) bank26660 +26661 POINT(40.13323094455644 74.51025319345688) bank26661 +26662 POINT(40.27447846272007 74.48171222929356) bank26662 +26663 POINT(40.793652559415705 74.24224925268325) bank26663 +26664 POINT(40.163072185386866 73.90169661563225) bank26664 +26665 POINT(40.2330001771047 74.16388022110988) bank26665 +26666 POINT(40.6877975068231 74.14131110973783) bank26666 +26667 POINT(40.74688242614065 73.0722876163811) bank26667 +26668 POINT(41.40349742319097 74.74466275977979) bank26668 +26669 POINT(39.952371637399835 73.14065709894145) bank26669 +26670 POINT(39.88368156873182 73.41618546135555) bank26670 +26671 POINT(41.12244130537062 73.836594065734) bank26671 +26672 POINT(40.096533996653406 74.42956820159469) bank26672 +26673 POINT(40.569022034998376 73.9955049289063) bank26673 +26674 POINT(41.38061164689913 74.69879301535178) bank26674 +26675 POINT(41.070271669273204 74.89396850071148) bank26675 +26676 POINT(41.58081402482983 74.4902345828958) bank26676 +26677 POINT(40.13429455422335 73.0847002881144) bank26677 +26678 POINT(41.26620027919149 74.84976696727385) bank26678 +26679 POINT(40.69844091695846 73.53580024196239) bank26679 +26680 POINT(41.602839200306846 74.72811066118783) bank26680 +26681 POINT(41.03318667761221 74.46337867250809) bank26681 +26682 POINT(41.491565623636845 74.45794394785455) bank26682 +26683 POINT(40.06449174518218 74.5571662767534) bank26683 +26684 POINT(39.80812676560523 74.53987934508896) bank26684 +26685 POINT(40.1042819597716 73.66856367332525) bank26685 +26686 POINT(41.031570667226994 74.01714853841152) bank26686 +26687 POINT(40.82845457964441 73.9490303081015) bank26687 +26688 POINT(40.27010758199943 73.75146596950269) bank26688 +26689 POINT(40.27207923761408 73.03138892223248) bank26689 +26690 POINT(41.35444466969057 74.94423301467464) bank26690 +26691 POINT(40.701116183054744 74.30349533131175) bank26691 +26692 POINT(40.72332372378755 74.7668818717452) bank26692 +26693 POINT(39.96573146055108 73.68752097077926) bank26693 +26694 POINT(41.42358259177398 74.06919682353596) bank26694 +26695 POINT(41.49334831777254 73.89567508816928) bank26695 +26696 POINT(41.68230447030946 73.23141124867576) bank26696 +26697 POINT(40.30293647097104 74.96731927037823) bank26697 +26698 POINT(40.16379189105175 73.07277859866451) bank26698 +26699 POINT(40.64212053437557 74.41029124820115) bank26699 +26700 POINT(39.97548361006078 74.23986775750961) bank26700 +26701 POINT(40.007169261729196 73.7561556434765) bank26701 +26702 POINT(41.71117823267535 74.71425524005689) bank26702 +26703 POINT(41.514057772794175 74.93582159921993) bank26703 +26704 POINT(39.718985668941286 73.75786581531253) bank26704 +26705 POINT(41.47612012595611 73.0488113803007) bank26705 +26706 POINT(41.011600408561264 73.84197492142204) bank26706 +26707 POINT(41.27955099145591 74.83905182686915) bank26707 +26708 POINT(40.92170348251654 74.63684787641444) bank26708 +26709 POINT(40.27405181204656 74.07645621619766) bank26709 +26710 POINT(41.45676375718153 73.53322659410183) bank26710 +26711 POINT(40.66891251294832 73.70246681261698) bank26711 +26712 POINT(41.65775902525032 74.5368674493052) bank26712 +26713 POINT(39.99971626157795 74.2518394102759) bank26713 +26714 POINT(40.159682638944446 73.87445934151314) bank26714 +26715 POINT(40.41674660717598 73.1409399601825) bank26715 +26716 POINT(41.7118819115355 74.99509501988577) bank26716 +26717 POINT(41.19825755312888 73.95698321521715) bank26717 +26718 POINT(40.18091595800297 73.32961326486898) bank26718 +26719 POINT(41.633788915027665 73.375693928363) bank26719 +26720 POINT(41.245493402757994 74.88189916090279) bank26720 +26721 POINT(41.627720828706266 74.23827339907885) bank26721 +26722 POINT(41.08242057683325 73.56664796221203) bank26722 +26723 POINT(40.51890255170112 73.43513258830961) bank26723 +26724 POINT(40.17918853982355 74.42177507898636) bank26724 +26725 POINT(39.8301889438998 73.46615618386608) bank26725 +26726 POINT(41.416018543224304 73.51035467696784) bank26726 +26727 POINT(40.37076397848307 73.61235788563854) bank26727 +26728 POINT(40.41862960985958 73.96517302002154) bank26728 +26729 POINT(40.2708852373859 74.19118727410151) bank26729 +26730 POINT(40.0240816764541 73.4992594553286) bank26730 +26731 POINT(41.06807409343408 74.41885028463172) bank26731 +26732 POINT(41.22230733470197 73.3643998441217) bank26732 +26733 POINT(41.58868539295497 74.39669050484684) bank26733 +26734 POINT(39.95869054452519 73.68703782295051) bank26734 +26735 POINT(41.295508731766354 73.64815352755204) bank26735 +26736 POINT(41.20648542799754 73.9919896580119) bank26736 +26737 POINT(41.53250339102108 74.65365389963269) bank26737 +26738 POINT(40.43982720171644 73.3030877273675) bank26738 +26739 POINT(39.87919587975538 74.58418942171421) bank26739 +26740 POINT(41.37281507425015 74.0932801666307) bank26740 +26741 POINT(39.794750677390866 74.77933253152936) bank26741 +26742 POINT(40.96879163677 74.23233957578009) bank26742 +26743 POINT(40.421413766570986 74.46288976404735) bank26743 +26744 POINT(41.012330288546664 74.76149551838009) bank26744 +26745 POINT(39.90758862972951 73.55527940622227) bank26745 +26746 POINT(39.87742168994367 74.32201759968271) bank26746 +26747 POINT(40.991331200719195 73.83833897849344) bank26747 +26748 POINT(40.045187307175205 73.98507148622991) bank26748 +26749 POINT(39.83623030509488 73.02721671879944) bank26749 +26750 POINT(40.979284795091054 73.77958475602061) bank26750 +26751 POINT(41.19047807662181 74.61469858436605) bank26751 +26752 POINT(40.079734029566815 74.80247643152329) bank26752 +26753 POINT(40.41410300032434 73.16073366255893) bank26753 +26754 POINT(41.27540021149989 73.83023331253028) bank26754 +26755 POINT(41.0630751543082 74.0309910214618) bank26755 +26756 POINT(41.22858178818163 73.84771456494036) bank26756 +26757 POINT(41.71077589868348 74.0429065757683) bank26757 +26758 POINT(40.99965748120382 74.8882686365007) bank26758 +26759 POINT(41.13117577530817 73.85515500227467) bank26759 +26760 POINT(40.04959554906017 73.11652777884663) bank26760 +26761 POINT(39.85785143199682 74.39479858169935) bank26761 +26762 POINT(40.929152250874715 74.54145017101713) bank26762 +26763 POINT(40.156729252748654 74.15714690841824) bank26763 +26764 POINT(41.24938164815713 74.5696645227753) bank26764 +26765 POINT(40.441906862619554 73.50336020697155) bank26765 +26766 POINT(39.99046719149237 74.3563525820331) bank26766 +26767 POINT(40.20943486387515 74.01332452678345) bank26767 +26768 POINT(40.537976534811804 73.13501865306725) bank26768 +26769 POINT(39.81493398364075 74.33451740619263) bank26769 +26770 POINT(41.35842049163243 74.1736231804941) bank26770 +26771 POINT(41.64735645343529 73.17278119667759) bank26771 +26772 POINT(40.43937061066294 73.5624076690971) bank26772 +26773 POINT(39.75272503223715 74.94186645148795) bank26773 +26774 POINT(39.824836428808524 73.3359954872368) bank26774 +26775 POINT(41.530247642380345 73.69311707884343) bank26775 +26776 POINT(39.79955937483383 74.63598481032122) bank26776 +26777 POINT(40.46132108549953 73.41191560945342) bank26777 +26778 POINT(41.290285759899334 73.04893671292619) bank26778 +26779 POINT(41.0872958642643 73.37097918594223) bank26779 +26780 POINT(40.41518193681167 74.39981945904368) bank26780 +26781 POINT(40.14351202419603 73.73393031079127) bank26781 +26782 POINT(41.55781076303518 74.56423975909571) bank26782 +26783 POINT(39.90120712761897 73.23234505490638) bank26783 +26784 POINT(40.71228422906674 74.14979228881842) bank26784 +26785 POINT(40.24243931237589 73.50911798610898) bank26785 +26786 POINT(41.55158443795876 73.18035203406886) bank26786 +26787 POINT(39.93484868432097 74.08821131635409) bank26787 +26788 POINT(40.584604864433565 74.74990081978623) bank26788 +26789 POINT(40.61480105370106 73.54359286723991) bank26789 +26790 POINT(40.45974235069279 74.47847275102271) bank26790 +26791 POINT(40.2470235637942 74.0057105421503) bank26791 +26792 POINT(41.04059998571495 73.80755191321121) bank26792 +26793 POINT(40.7382850422981 73.73377350218574) bank26793 +26794 POINT(41.628299869117434 74.48700455343818) bank26794 +26795 POINT(39.76038348873521 74.59705768452024) bank26795 +26796 POINT(39.958488777432336 74.27969764967233) bank26796 +26797 POINT(40.72936554211236 73.60384503985037) bank26797 +26798 POINT(41.334689130922996 74.924045192667) bank26798 +26799 POINT(39.764183717567086 74.7339752416711) bank26799 +26800 POINT(40.55396804224038 74.3713188720816) bank26800 +26801 POINT(40.49794057735421 73.70801152440747) bank26801 +26802 POINT(41.60736564543531 74.82728039774234) bank26802 +26803 POINT(40.69440769838936 74.3732890073766) bank26803 +26804 POINT(41.362405322237684 73.75754858036845) bank26804 +26805 POINT(40.17618982713204 74.95126795192598) bank26805 +26806 POINT(40.41862905119027 73.40154939428969) bank26806 +26807 POINT(40.0657622656719 73.6436519630232) bank26807 +26808 POINT(40.849218858760466 74.08599762275053) bank26808 +26809 POINT(40.40231652015137 74.41714191760313) bank26809 +26810 POINT(41.22309090566591 74.88395252418567) bank26810 +26811 POINT(41.227481250045074 74.1335498107533) bank26811 +26812 POINT(39.88567655172282 73.50625936737791) bank26812 +26813 POINT(40.467636049032166 74.38918957571266) bank26813 +26814 POINT(40.08904009731845 73.73299105565015) bank26814 +26815 POINT(40.20149464384728 74.13074872043397) bank26815 +26816 POINT(40.36617187732203 73.81933024146466) bank26816 +26817 POINT(41.16222007243972 74.78561285158264) bank26817 +26818 POINT(40.57860454493743 74.61818801132011) bank26818 +26819 POINT(40.09741669716062 73.27398231823021) bank26819 +26820 POINT(40.73809945907313 73.14086535836496) bank26820 +26821 POINT(40.5105502136627 73.4733479851176) bank26821 +26822 POINT(41.495245400463084 73.2761381489043) bank26822 +26823 POINT(41.10502198577984 73.44612479336574) bank26823 +26824 POINT(41.534420663889414 74.38762302108046) bank26824 +26825 POINT(41.24526790926984 73.84832738753771) bank26825 +26826 POINT(40.000446481047085 73.05230513788092) bank26826 +26827 POINT(41.07646731284053 74.56610162700991) bank26827 +26828 POINT(40.90398685502972 73.75006778340205) bank26828 +26829 POINT(41.67103343972125 74.8038133268833) bank26829 +26830 POINT(40.79792058982304 74.5239325802587) bank26830 +26831 POINT(40.128821737789 73.06105593870856) bank26831 +26832 POINT(41.024568812845544 73.08723581824368) bank26832 +26833 POINT(41.499367152525 73.76432613225158) bank26833 +26834 POINT(41.17275468125967 73.74724392039414) bank26834 +26835 POINT(41.24085818086243 74.8510077835633) bank26835 +26836 POINT(41.516543169950324 73.25865525893892) bank26836 +26837 POINT(40.39386204833181 74.0527247941284) bank26837 +26838 POINT(40.21265846626293 73.0441116662404) bank26838 +26839 POINT(40.78270968247188 74.22723981138134) bank26839 +26840 POINT(40.5431341248242 73.17050816191761) bank26840 +26841 POINT(40.43229192452627 74.97798058188371) bank26841 +26842 POINT(39.9560057438096 74.92145677256885) bank26842 +26843 POINT(41.60546619736719 74.40800715313804) bank26843 +26844 POINT(41.36347878266161 73.23296153961712) bank26844 +26845 POINT(41.068214076322526 73.57529566103665) bank26845 +26846 POINT(41.54306536821781 73.36203005809509) bank26846 +26847 POINT(40.94178327611394 74.70801206107897) bank26847 +26848 POINT(41.17471151980029 73.91770032262664) bank26848 +26849 POINT(40.3846761917098 74.85746801056007) bank26849 +26850 POINT(40.61496757852579 74.1038713220461) bank26850 +26851 POINT(41.367781382476636 73.63920725068348) bank26851 +26852 POINT(41.60214214121616 73.55478443977627) bank26852 +26853 POINT(41.691618523085296 73.37670983037206) bank26853 +26854 POINT(41.536811093516775 73.82944652706861) bank26854 +26855 POINT(39.967425867527986 73.46259738581799) bank26855 +26856 POINT(40.30313170531455 73.41604494918553) bank26856 +26857 POINT(40.30796538406999 74.32496452815224) bank26857 +26858 POINT(40.969396190611356 73.27820553254456) bank26858 +26859 POINT(40.499617768260364 73.22233763196466) bank26859 +26860 POINT(40.84778486350756 74.6213937405707) bank26860 +26861 POINT(41.44927307310717 73.28954341144973) bank26861 +26862 POINT(39.79190776289478 73.63591151027187) bank26862 +26863 POINT(40.44734794086077 73.95973100028401) bank26863 +26864 POINT(40.366501852649165 74.74169534657592) bank26864 +26865 POINT(40.621496107099674 73.44512125000173) bank26865 +26866 POINT(40.66342189256413 74.64240445134537) bank26866 +26867 POINT(40.73411569615591 74.26721890720918) bank26867 +26868 POINT(40.502677525888686 74.95227729569486) bank26868 +26869 POINT(41.079564550428806 73.84612949102919) bank26869 +26870 POINT(40.902151706692116 73.40719605980972) bank26870 +26871 POINT(40.47506409982197 73.11447463699344) bank26871 +26872 POINT(41.32583158773537 73.35039615442689) bank26872 +26873 POINT(41.18391604533972 74.31261621401724) bank26873 +26874 POINT(40.327355265421694 74.11184808660208) bank26874 +26875 POINT(39.988408817747946 74.54528775974237) bank26875 +26876 POINT(40.27403028102789 74.94124084348802) bank26876 +26877 POINT(40.198104475227275 73.39811208231471) bank26877 +26878 POINT(41.59126871982893 74.57831771113776) bank26878 +26879 POINT(40.915139788389034 73.17629485272515) bank26879 +26880 POINT(40.670357305856946 74.65032017739875) bank26880 +26881 POINT(39.801191281065854 74.92249049661116) bank26881 +26882 POINT(41.507550001954684 74.45340430415358) bank26882 +26883 POINT(40.471128488608066 74.1675899462376) bank26883 +26884 POINT(40.28093053070269 74.02077929144625) bank26884 +26885 POINT(41.641800046150514 74.76315440862436) bank26885 +26886 POINT(40.76273595105558 74.80769526427451) bank26886 +26887 POINT(41.23448579891089 73.13897085876783) bank26887 +26888 POINT(41.13374110215903 73.80923090459365) bank26888 +26889 POINT(41.589656901936955 73.4014076986544) bank26889 +26890 POINT(40.84084365669988 74.07281058654486) bank26890 +26891 POINT(41.162624159957396 73.99468261075785) bank26891 +26892 POINT(39.78149726353327 74.8604555042183) bank26892 +26893 POINT(40.74604179347938 73.30496809698786) bank26893 +26894 POINT(41.00547651134458 73.00668902327408) bank26894 +26895 POINT(41.17069846113375 74.86944007392229) bank26895 +26896 POINT(41.71220545408823 74.26192404587995) bank26896 +26897 POINT(39.92529048081097 74.21417876882941) bank26897 +26898 POINT(41.71021719667769 73.73784969775204) bank26898 +26899 POINT(40.07163799294001 74.95695446251841) bank26899 +26900 POINT(40.34475157054619 73.66544825413096) bank26900 +26901 POINT(41.51092952122857 74.09352569519987) bank26901 +26902 POINT(40.01502917600292 74.33708564601594) bank26902 +26903 POINT(41.35243517757113 73.81737851562592) bank26903 +26904 POINT(40.90805284783364 73.71958817214077) bank26904 +26905 POINT(39.92103174761467 74.59325461210493) bank26905 +26906 POINT(41.6050051225929 73.63156385694487) bank26906 +26907 POINT(41.69127730012678 74.645975858999) bank26907 +26908 POINT(40.015752790597354 74.03182971494266) bank26908 +26909 POINT(41.5529063891921 73.40955982121417) bank26909 +26910 POINT(40.041222430052734 74.7282652365457) bank26910 +26911 POINT(41.547173099557455 74.69188148851804) bank26911 +26912 POINT(39.88965777020183 73.79838579282232) bank26912 +26913 POINT(41.14821370828519 73.47745189199019) bank26913 +26914 POINT(41.34003026109563 73.63328310750774) bank26914 +26915 POINT(41.01704325881231 73.5372988395001) bank26915 +26916 POINT(40.8893921934844 74.32616810771117) bank26916 +26917 POINT(40.38284565793099 74.02118565934249) bank26917 +26918 POINT(40.49633088075891 74.30878345531941) bank26918 +26919 POINT(40.18640331486934 74.64390091086115) bank26919 +26920 POINT(41.35276451279423 73.20596677136486) bank26920 +26921 POINT(41.13857944511869 74.20549424842066) bank26921 +26922 POINT(41.2036217295212 74.8074714764367) bank26922 +26923 POINT(40.00024271234415 73.37357768064491) bank26923 +26924 POINT(41.52451406415467 74.8652624101595) bank26924 +26925 POINT(40.116539369236854 74.68977488801404) bank26925 +26926 POINT(41.620912585215386 74.61193195444964) bank26926 +26927 POINT(39.75690399517715 73.90768430972075) bank26927 +26928 POINT(41.07069623902805 73.01663029617417) bank26928 +26929 POINT(40.78480596933914 74.00546557926596) bank26929 +26930 POINT(41.11688971660928 74.53376741669318) bank26930 +26931 POINT(40.75969480985548 73.55569270424233) bank26931 +26932 POINT(40.16130993508051 74.45878428092378) bank26932 +26933 POINT(40.94985684066693 74.07032039995083) bank26933 +26934 POINT(39.98664279708351 74.62163329767775) bank26934 +26935 POINT(40.492136326450975 74.90672727256018) bank26935 +26936 POINT(39.89932884401191 74.56249494988671) bank26936 +26937 POINT(41.38983995454166 74.63348636634394) bank26937 +26938 POINT(41.41429992364119 74.7451881522008) bank26938 +26939 POINT(39.89354059613836 74.39676304215163) bank26939 +26940 POINT(40.53556299390665 74.80955703457148) bank26940 +26941 POINT(41.70728739051814 73.75681798559619) bank26941 +26942 POINT(40.128140458714476 73.49975966662848) bank26942 +26943 POINT(41.024042631923564 73.40111236638555) bank26943 +26944 POINT(41.596255849686145 74.17064795131569) bank26944 +26945 POINT(41.58167583300984 74.92046177848104) bank26945 +26946 POINT(41.53093102415889 74.47889261012872) bank26946 +26947 POINT(40.61005468881565 73.39456598521058) bank26947 +26948 POINT(40.863470002610185 73.45600927437249) bank26948 +26949 POINT(41.00685875968627 73.83859391688509) bank26949 +26950 POINT(40.77765252535187 74.38679085207437) bank26950 +26951 POINT(40.411778603124546 73.79541755685212) bank26951 +26952 POINT(39.956961429063405 73.36399927576348) bank26952 +26953 POINT(40.016144135649725 74.15566511034311) bank26953 +26954 POINT(40.37064265687849 73.57209948013521) bank26954 +26955 POINT(40.55445319994349 74.06072639194142) bank26955 +26956 POINT(40.921204109531324 73.04001130279048) bank26956 +26957 POINT(41.16206408744876 73.08126878421677) bank26957 +26958 POINT(40.860191328865376 73.96288833118756) bank26958 +26959 POINT(40.31550798700487 73.36477946439528) bank26959 +26960 POINT(40.86733445438057 73.22550258824599) bank26960 +26961 POINT(40.0820605727316 74.15989548849082) bank26961 +26962 POINT(40.42726263483872 73.7008726570769) bank26962 +26963 POINT(41.197254400383635 74.56580285456705) bank26963 +26964 POINT(41.359300456451095 74.84998039959287) bank26964 +26965 POINT(39.89670177672194 74.1135567397186) bank26965 +26966 POINT(40.94068593049491 74.90455827686125) bank26966 +26967 POINT(39.758388260789495 73.23360929592984) bank26967 +26968 POINT(40.623950092268885 73.03195248018247) bank26968 +26969 POINT(40.33369809339984 73.36101881404868) bank26969 +26970 POINT(40.296235012400714 73.22562296241779) bank26970 +26971 POINT(41.05280117493079 73.11403709875094) bank26971 +26972 POINT(40.138255096119146 74.43278638781018) bank26972 +26973 POINT(41.41674165653049 74.04416736210595) bank26973 +26974 POINT(40.18515160841762 74.51048291562243) bank26974 +26975 POINT(39.903904908732294 74.92791290038457) bank26975 +26976 POINT(40.32041664852433 74.67675127159725) bank26976 +26977 POINT(39.93536699322476 73.10859723014622) bank26977 +26978 POINT(40.77191707684583 73.75983858235232) bank26978 +26979 POINT(40.49308562113369 74.38770272938626) bank26979 +26980 POINT(41.697642029576436 74.518516374071) bank26980 +26981 POINT(41.71071875095369 73.58435329521106) bank26981 +26982 POINT(40.298798590386596 74.47014603707855) bank26982 +26983 POINT(40.263952238844695 73.4372223598104) bank26983 +26984 POINT(41.52179945293122 73.28637136004826) bank26984 +26985 POINT(41.28076586275363 74.1257183672572) bank26985 +26986 POINT(40.70173570139193 74.65064544201334) bank26986 +26987 POINT(41.491290180967006 74.49302366661793) bank26987 +26988 POINT(40.13879856927657 73.80650685817538) bank26988 +26989 POINT(40.96313818899744 73.0115099586804) bank26989 +26990 POINT(41.64071003869803 74.07966189498462) bank26990 +26991 POINT(40.740871783339244 73.56388210708343) bank26991 +26992 POINT(40.285280284292355 73.82510418820506) bank26992 +26993 POINT(39.94364219635248 73.43363099610401) bank26993 +26994 POINT(39.79864534053376 74.62942329998091) bank26994 +26995 POINT(40.174312771611525 73.83135807529003) bank26995 +26996 POINT(40.960267959021756 74.13995294590426) bank26996 +26997 POINT(40.32482232833769 74.79395467949247) bank26997 +26998 POINT(40.96053020248983 73.39669839980098) bank26998 +26999 POINT(41.35301139903454 73.02471604178947) bank26999 +27000 POINT(40.18259825536722 73.97645755438371) bank27000 +27001 POINT(40.39672533199623 74.13714561065336) bank27001 +27002 POINT(41.365567755407305 73.43299883700644) bank27002 +27003 POINT(41.266559225074374 74.510089429505) bank27003 +27004 POINT(40.60193542372246 73.84391242103942) bank27004 +27005 POINT(40.59528134070285 74.67161336299525) bank27005 +27006 POINT(40.31507207453872 74.6245244951687) bank27006 +27007 POINT(41.3309801874402 74.16663692383177) bank27007 +27008 POINT(40.47645699252276 74.28251546791819) bank27008 +27009 POINT(41.21061057011586 74.20811175918672) bank27009 +27010 POINT(39.93820014677784 73.32758922891016) bank27010 +27011 POINT(40.57975258980122 74.85738412772878) bank27011 +27012 POINT(41.671974166753216 74.34375938108306) bank27012 +27013 POINT(40.40844253562147 73.47588386086144) bank27013 +27014 POINT(40.02246489918504 74.16215651546578) bank27014 +27015 POINT(41.15537708261138 74.28958297045644) bank27015 +27016 POINT(40.06715382732304 73.11634255298812) bank27016 +27017 POINT(41.075411180060776 74.76251710394502) bank27017 +27018 POINT(41.57743753087141 73.54367770002219) bank27018 +27019 POINT(40.01076022478781 74.25955540434046) bank27019 +27020 POINT(40.45326425883127 73.2500575363674) bank27020 +27021 POINT(39.99442258107002 73.83704325136405) bank27021 +27022 POINT(40.46313749385416 73.09466997043333) bank27022 +27023 POINT(41.219775973637674 73.53370740256779) bank27023 +27024 POINT(41.07191422157419 73.41623180608958) bank27024 +27025 POINT(40.27720732463917 74.84896854795332) bank27025 +27026 POINT(41.222541576992946 73.24180029309684) bank27026 +27027 POINT(40.97376354980737 73.4430603938221) bank27027 +27028 POINT(40.82726540400173 73.76114408843097) bank27028 +27029 POINT(41.61386298022267 74.52855619899739) bank27029 +27030 POINT(41.491917196849286 74.02904897652168) bank27030 +27031 POINT(40.31311837710621 73.30121793015641) bank27031 +27032 POINT(40.59333852668478 74.8314145255727) bank27032 +27033 POINT(41.35068297361625 73.74238215296285) bank27033 +27034 POINT(40.54946606525731 73.33286837862401) bank27034 +27035 POINT(39.824062366393136 73.67840059158412) bank27035 +27036 POINT(40.512639701480396 74.18608118622744) bank27036 +27037 POINT(41.277182646214186 73.88242773588385) bank27037 +27038 POINT(39.91648441442166 74.14800703237877) bank27038 +27039 POINT(41.54348711065905 74.94240858556238) bank27039 +27040 POINT(40.41491888095601 73.99480583511183) bank27040 +27041 POINT(39.78391325258794 73.49229488296521) bank27041 +27042 POINT(40.03096404007645 74.06034030462187) bank27042 +27043 POINT(41.06518096755528 73.25040765128516) bank27043 +27044 POINT(41.49293304021483 73.60667792666277) bank27044 +27045 POINT(41.28170021059337 73.47885690940635) bank27045 +27046 POINT(39.958729785002184 74.03069601363148) bank27046 +27047 POINT(41.11350176384779 73.84696067562663) bank27047 +27048 POINT(41.525915307779876 73.71739579747367) bank27048 +27049 POINT(40.71314379753863 73.87313172922136) bank27049 +27050 POINT(40.478773805609364 73.10815288756629) bank27050 +27051 POINT(40.08501797572891 74.08529047863921) bank27051 +27052 POINT(41.33268293883761 73.32129391186699) bank27052 +27053 POINT(39.914185609392305 73.93838839014388) bank27053 +27054 POINT(41.48092108692653 73.15982632245398) bank27054 +27055 POINT(40.944082754480846 74.37145058686114) bank27055 +27056 POINT(40.35985220503983 73.0655022217892) bank27056 +27057 POINT(41.00834132041725 74.63164906594297) bank27057 +27058 POINT(40.35937830822316 73.71021753026203) bank27058 +27059 POINT(39.904663502779854 73.19774539482587) bank27059 +27060 POINT(41.079747913963665 74.80158538920574) bank27060 +27061 POINT(40.441700872302405 73.88669220487859) bank27061 +27062 POINT(39.86075781154395 74.92059996205587) bank27062 +27063 POINT(41.44711805011062 73.69084528061197) bank27063 +27064 POINT(41.468009105046406 73.17215012833711) bank27064 +27065 POINT(41.34010417147017 73.49842278297352) bank27065 +27066 POINT(40.435650836758285 74.55199491927645) bank27066 +27067 POINT(39.8267594412175 74.02624994672375) bank27067 +27068 POINT(40.29582048223288 74.22615743301822) bank27068 +27069 POINT(40.63997574737252 73.40100115884961) bank27069 +27070 POINT(41.22968421278243 74.39587382620132) bank27070 +27071 POINT(41.1327376762661 73.83778758029005) bank27071 +27072 POINT(41.08751787767029 74.12754319953648) bank27072 +27073 POINT(39.84687626072429 74.11612645381213) bank27073 +27074 POINT(40.4244709551364 73.0848979532814) bank27074 +27075 POINT(40.25078067130716 74.08897998643027) bank27075 +27076 POINT(41.66449725311262 74.83421736249156) bank27076 +27077 POINT(41.65833647881201 73.98956722301547) bank27077 +27078 POINT(40.310368882473384 74.83344740170915) bank27078 +27079 POINT(41.176040450847495 73.08668582745536) bank27079 +27080 POINT(41.20579771187836 74.56508565662159) bank27080 +27081 POINT(40.03884811566407 74.05724955370633) bank27081 +27082 POINT(40.156327792615656 74.117557953332) bank27082 +27083 POINT(40.03874528508487 73.65137753051029) bank27083 +27084 POINT(40.594730994719406 74.57254749697717) bank27084 +27085 POINT(40.77102999726365 73.5436233301353) bank27085 +27086 POINT(40.17294457823874 74.15887196505162) bank27086 +27087 POINT(40.094934553442734 73.96673850052774) bank27087 +27088 POINT(40.445283059044684 74.97851628576237) bank27088 +27089 POINT(40.96456060985178 73.96428652314296) bank27089 +27090 POINT(40.59755685018439 73.05449277582032) bank27090 +27091 POINT(40.364252661640734 73.33545015705717) bank27091 +27092 POINT(40.10636805476393 73.70360142861443) bank27092 +27093 POINT(41.23702618659309 74.83598250066152) bank27093 +27094 POINT(39.872699493578686 73.08111363063767) bank27094 +27095 POINT(41.543030565209534 73.29550802133322) bank27095 +27096 POINT(41.17182297920887 74.12972056711709) bank27096 +27097 POINT(40.53513503683804 73.77547558765426) bank27097 +27098 POINT(40.655790475476195 74.53636070960306) bank27098 +27099 POINT(41.03791410390631 74.51140988611087) bank27099 +27100 POINT(40.20582239162031 74.52247097121338) bank27100 +27101 POINT(40.05357065406011 74.61500151004539) bank27101 +27102 POINT(39.84583834844623 73.56927682969315) bank27102 +27103 POINT(41.351651010962755 74.96827611214208) bank27103 +27104 POINT(41.10960771451228 73.02255370520076) bank27104 +27105 POINT(41.48827916211839 74.4550661239086) bank27105 +27106 POINT(41.49169932759164 74.69026816673423) bank27106 +27107 POINT(39.9101162230799 74.3178331681253) bank27107 +27108 POINT(41.17266561027815 73.19631428573034) bank27108 +27109 POINT(41.514290026997976 74.45448067120263) bank27109 +27110 POINT(39.789126950737334 73.85933255348479) bank27110 +27111 POINT(40.13368559354364 74.63180204941447) bank27111 +27112 POINT(40.50050587324083 74.53941327760477) bank27112 +27113 POINT(40.83445219113008 74.33586242057524) bank27113 +27114 POINT(40.951405888870646 74.35303649198555) bank27114 +27115 POINT(40.41986456367444 73.37699816128124) bank27115 +27116 POINT(39.98685640271839 74.6855078111967) bank27116 +27117 POINT(41.55282410336239 74.54339598949564) bank27117 +27118 POINT(40.06896769664631 74.01598350291036) bank27118 +27119 POINT(40.410348721668434 74.94606267381575) bank27119 +27120 POINT(41.57947675384806 73.11286282191666) bank27120 +27121 POINT(39.907544189428165 74.74838350396287) bank27121 +27122 POINT(40.459298706520514 73.55911453416904) bank27122 +27123 POINT(39.89267116330162 73.17896043341774) bank27123 +27124 POINT(41.251837670330104 74.45173899494614) bank27124 +27125 POINT(40.33111236870202 74.37291658334797) bank27125 +27126 POINT(41.430336810384595 73.40839443429982) bank27126 +27127 POINT(41.06158177053929 74.82136245567435) bank27127 +27128 POINT(40.693926276913594 74.29610699141061) bank27128 +27129 POINT(41.46941874787285 74.93077341049283) bank27129 +27130 POINT(40.00303619621294 74.15198991417715) bank27130 +27131 POINT(40.75598277130776 74.87710033995727) bank27131 +27132 POINT(40.50956116548617 73.68245850655175) bank27132 +27133 POINT(41.035961130364754 73.9270227059096) bank27133 +27134 POINT(41.290393032063704 74.19942234612807) bank27134 +27135 POINT(39.83414474073675 74.94216604643879) bank27135 +27136 POINT(41.666990466959746 74.77466139846206) bank27136 +27137 POINT(40.835209738003186 74.44459868641354) bank27137 +27138 POINT(40.43136632741893 74.61090139426767) bank27138 +27139 POINT(40.1652722744351 73.79789741746886) bank27139 +27140 POINT(41.58877320848825 74.41876892274811) bank27140 +27141 POINT(40.63092748468975 73.32052590467) bank27141 +27142 POINT(39.84682491822114 73.10521133169037) bank27142 +27143 POINT(39.7858740409543 74.39478799991123) bank27143 +27144 POINT(41.46540912729652 74.56854588231643) bank27144 +27145 POINT(41.194372600066515 73.75566438237708) bank27145 +27146 POINT(40.137600151980614 73.09348236180462) bank27146 +27147 POINT(41.173597361463386 74.14811810047725) bank27147 +27148 POINT(39.78152736771763 74.59340804790367) bank27148 +27149 POINT(41.465588041994536 73.70237895339733) bank27149 +27150 POINT(40.61266143938499 73.40935788650656) bank27150 +27151 POINT(41.24035792864094 74.89465495248243) bank27151 +27152 POINT(40.31489050187901 73.1529687916624) bank27152 +27153 POINT(40.74804310988488 74.53862109455964) bank27153 +27154 POINT(41.154311660610254 73.6097858340442) bank27154 +27155 POINT(41.22601816286127 73.32702437413863) bank27155 +27156 POINT(41.4731152246613 73.44336248896886) bank27156 +27157 POINT(40.90239029625407 74.62881511452613) bank27157 +27158 POINT(40.64938481196373 73.80192173189519) bank27158 +27159 POINT(40.49265089343398 74.95722486246882) bank27159 +27160 POINT(41.17676687001961 74.22721950733664) bank27160 +27161 POINT(40.17646802127591 73.57941086412274) bank27161 +27162 POINT(41.26409675613923 73.48999881114288) bank27162 +27163 POINT(40.2116561328729 73.20348732038143) bank27163 +27164 POINT(41.09408706665881 74.36591080763229) bank27164 +27165 POINT(40.19301051482709 73.49336365985273) bank27165 +27166 POINT(40.87079779727209 73.46169374564249) bank27166 +27167 POINT(41.593968300599435 74.8216875153219) bank27167 +27168 POINT(40.7036388031956 74.59929969463846) bank27168 +27169 POINT(39.88077861041431 74.42983360210164) bank27169 +27170 POINT(41.09984249668467 73.52585817282682) bank27170 +27171 POINT(40.954496322954625 74.84858239751026) bank27171 +27172 POINT(41.67121713809858 74.09987717048767) bank27172 +27173 POINT(40.25121733805514 74.26306210097745) bank27173 +27174 POINT(39.98997209579967 74.41382237093228) bank27174 +27175 POINT(41.447726564004924 73.80677862554253) bank27175 +27176 POINT(41.66142625418763 73.36976307540846) bank27176 +27177 POINT(41.70129886362125 74.51281468169968) bank27177 +27178 POINT(40.074662825550085 73.01799204563044) bank27178 +27179 POINT(40.753105525828744 74.67498312648999) bank27179 +27180 POINT(41.264460391578545 74.47965102208354) bank27180 +27181 POINT(40.78953985513289 74.1765371426775) bank27181 +27182 POINT(41.038952721211665 74.01634251106269) bank27182 +27183 POINT(40.31213179304885 73.3796191740462) bank27183 +27184 POINT(39.85493070907906 73.33931409887037) bank27184 +27185 POINT(40.86784257476806 74.39209885981917) bank27185 +27186 POINT(41.149999677438544 74.4552731796584) bank27186 +27187 POINT(40.55715995522082 74.7648769825917) bank27187 +27188 POINT(41.71023799956889 73.04628654054345) bank27188 +27189 POINT(39.84729170713711 74.98865159302542) bank27189 +27190 POINT(40.026454888425526 73.4314195970541) bank27190 +27191 POINT(40.19230310020446 74.21250617571333) bank27191 +27192 POINT(40.09343461851639 74.77716341514622) bank27192 +27193 POINT(40.192432964162464 73.24439288339929) bank27193 +27194 POINT(39.93130064893578 74.28000108318558) bank27194 +27195 POINT(41.70973129875799 74.61445554676828) bank27195 +27196 POINT(41.37466614023394 74.01366484308552) bank27196 +27197 POINT(40.58908684379886 74.12771807822472) bank27197 +27198 POINT(39.99909015690745 74.63176741801712) bank27198 +27199 POINT(41.177679287508305 74.70600877116416) bank27199 +27200 POINT(40.196982530389135 74.5697772746678) bank27200 +27201 POINT(41.51920402629779 73.46713420261185) bank27201 +27202 POINT(41.134979700448646 73.7147293695706) bank27202 +27203 POINT(40.956875238190435 73.88256835478808) bank27203 +27204 POINT(39.778013543702514 73.91850813507804) bank27204 +27205 POINT(41.01177831483672 74.90396065348773) bank27205 +27206 POINT(40.86217012591966 74.75290533207166) bank27206 +27207 POINT(40.66247589247627 73.35909266624351) bank27207 +27208 POINT(40.94360870272744 74.45732028646393) bank27208 +27209 POINT(40.583511821003896 74.67027620103198) bank27209 +27210 POINT(39.761884708374836 74.24818388508294) bank27210 +27211 POINT(40.71783332525194 74.29827954151203) bank27211 +27212 POINT(40.114244995756 74.16775270304635) bank27212 +27213 POINT(40.46728602654868 73.0397609246339) bank27213 +27214 POINT(40.5560797487066 74.73901074531035) bank27214 +27215 POINT(41.104303725702145 74.89221664281766) bank27215 +27216 POINT(39.98469838798692 73.55528365907338) bank27216 +27217 POINT(40.80897680042512 73.49049156621665) bank27217 +27218 POINT(40.35996879301073 74.33046608893166) bank27218 +27219 POINT(41.2762081956892 74.83789140399608) bank27219 +27220 POINT(40.6956700568603 74.46672996627143) bank27220 +27221 POINT(40.36773576538059 74.47696403485239) bank27221 +27222 POINT(40.86242856778984 74.63721953916736) bank27222 +27223 POINT(41.124478683314464 74.574547795423) bank27223 +27224 POINT(40.962062418869905 74.63721061817175) bank27224 +27225 POINT(41.455689186439656 73.34390189178528) bank27225 +27226 POINT(40.857684489987676 73.37821480077353) bank27226 +27227 POINT(40.02354318943118 73.436311282198) bank27227 +27228 POINT(40.324810905812555 74.19645824924075) bank27228 +27229 POINT(41.36852676454017 74.95279541284567) bank27229 +27230 POINT(40.60975021729014 74.38595669129994) bank27230 +27231 POINT(41.2627901212143 74.12274066813085) bank27231 +27232 POINT(39.859357906157 74.47066300329693) bank27232 +27233 POINT(40.1733179663918 73.60057785241156) bank27233 +27234 POINT(39.883082251530816 74.85050509788756) bank27234 +27235 POINT(41.35447254590223 74.64804880604085) bank27235 +27236 POINT(41.587296743671104 74.73360185184649) bank27236 +27237 POINT(40.37101003003261 74.37470491734263) bank27237 +27238 POINT(40.110035624508264 73.76332591104166) bank27238 +27239 POINT(41.61584719032554 73.28020023528022) bank27239 +27240 POINT(40.60778561753513 74.9180598337681) bank27240 +27241 POINT(40.21762661628769 74.13556631543207) bank27241 +27242 POINT(41.17520313203547 74.10184846649159) bank27242 +27243 POINT(40.98996550546723 73.08334987714052) bank27243 +27244 POINT(41.07260665728546 73.67905895994954) bank27244 +27245 POINT(39.89916149682581 73.81579957619884) bank27245 +27246 POINT(40.71548067923422 73.58265115926714) bank27246 +27247 POINT(39.77560544282746 74.95154396000798) bank27247 +27248 POINT(41.34434013015159 73.7715095185037) bank27248 +27249 POINT(41.26809830011541 74.81445196218593) bank27249 +27250 POINT(41.05829283751135 74.80965441208066) bank27250 +27251 POINT(41.10050910810554 74.37321175892568) bank27251 +27252 POINT(40.09115525334166 74.7876313041574) bank27252 +27253 POINT(41.538961162822346 73.9588162553205) bank27253 +27254 POINT(40.61913540571324 74.13683638639418) bank27254 +27255 POINT(40.8664433953784 74.52226663678859) bank27255 +27256 POINT(41.35079435630316 74.96704583223128) bank27256 +27257 POINT(40.38627439868019 73.7059404307152) bank27257 +27258 POINT(40.44768938330958 73.09193234327964) bank27258 +27259 POINT(40.89336833338 74.8413719675519) bank27259 +27260 POINT(40.140364733674616 74.54953685252492) bank27260 +27261 POINT(40.375299954454924 73.23066412902689) bank27261 +27262 POINT(40.37557125876724 74.79016621321713) bank27262 +27263 POINT(40.464183145822524 73.48612683498075) bank27263 +27264 POINT(40.47030282799262 74.902984225897) bank27264 +27265 POINT(40.45049582725826 74.126993482815) bank27265 +27266 POINT(41.10382450024992 73.35997014532784) bank27266 +27267 POINT(40.773725627955145 74.93466633828726) bank27267 +27268 POINT(41.2728734096535 74.22466273143085) bank27268 +27269 POINT(41.00499798479429 73.78132544652593) bank27269 +27270 POINT(39.74812218514268 74.52117810642537) bank27270 +27271 POINT(40.61892071311199 74.74099398865314) bank27271 +27272 POINT(40.32591238802458 73.57920674112829) bank27272 +27273 POINT(40.25456735795884 74.18364874092998) bank27273 +27274 POINT(40.748056939627254 73.53393189620233) bank27274 +27275 POINT(40.06367366814293 74.26940560320807) bank27275 +27276 POINT(41.12746298280925 73.37956935341634) bank27276 +27277 POINT(40.52329502670671 73.74227921509123) bank27277 +27278 POINT(41.666029043315625 74.45875348849182) bank27278 +27279 POINT(40.402512319456896 73.89091134481893) bank27279 +27280 POINT(40.56004842406884 74.15283362976822) bank27280 +27281 POINT(39.75705096491555 73.23131002597484) bank27281 +27282 POINT(40.491537087008446 73.17291664617244) bank27282 +27283 POINT(41.36044079577786 74.63012378966124) bank27283 +27284 POINT(40.04610916178977 74.96894690645324) bank27284 +27285 POINT(40.012115007122794 73.14387414883696) bank27285 +27286 POINT(40.299964588090766 73.65937393051728) bank27286 +27287 POINT(40.77029374876168 73.51828035222586) bank27287 +27288 POINT(39.92687078167871 73.61526512587368) bank27288 +27289 POINT(40.18288868631549 74.41308292306856) bank27289 +27290 POINT(39.8072802260005 74.4019251536555) bank27290 +27291 POINT(41.29553085050711 73.61584986384904) bank27291 +27292 POINT(41.05159119026746 73.88859824653804) bank27292 +27293 POINT(40.66663732729101 73.80944283884197) bank27293 +27294 POINT(39.74340809948957 74.1430649285786) bank27294 +27295 POINT(40.059418960547546 74.16009398645708) bank27295 +27296 POINT(40.314335020957614 73.2166300015552) bank27296 +27297 POINT(40.873263545699615 73.30212217614041) bank27297 +27298 POINT(41.1022685103922 73.16264516501474) bank27298 +27299 POINT(40.3360985840504 74.47539507937299) bank27299 +27300 POINT(41.55847531186037 73.9430691946806) bank27300 +27301 POINT(40.29265539779569 74.12705337871611) bank27301 +27302 POINT(40.2840018064591 74.89243792989218) bank27302 +27303 POINT(41.53743730646837 73.5447581397983) bank27303 +27304 POINT(40.843983184246966 74.32543193569161) bank27304 +27305 POINT(41.455924337509764 73.06645911920243) bank27305 +27306 POINT(41.313552291256784 74.84583804593942) bank27306 +27307 POINT(39.97572304287412 73.93999164896435) bank27307 +27308 POINT(40.553063836293475 73.24200714491592) bank27308 +27309 POINT(41.41271094599183 74.91475938690417) bank27309 +27310 POINT(39.98143366542994 73.28848262214444) bank27310 +27311 POINT(40.78205033516568 73.13150948199586) bank27311 +27312 POINT(41.317218782141275 74.84924064043044) bank27312 +27313 POINT(41.37428985475714 74.40947906765554) bank27313 +27314 POINT(40.18136022150901 73.65496030724653) bank27314 +27315 POINT(40.039108040146175 74.0108503783666) bank27315 +27316 POINT(41.69624863654732 73.22584758860923) bank27316 +27317 POINT(39.71823477559304 74.32715439881859) bank27317 +27318 POINT(40.11050468366956 73.57232705482943) bank27318 +27319 POINT(40.23481183378101 74.487429451432) bank27319 +27320 POINT(41.6975249132036 73.99982669815238) bank27320 +27321 POINT(40.43123056034548 73.92396026608874) bank27321 +27322 POINT(41.669720767169224 74.69866628450447) bank27322 +27323 POINT(40.991115127202605 73.19153113194821) bank27323 +27324 POINT(41.214387412766584 73.84953349507907) bank27324 +27325 POINT(39.774882624036785 74.30633647972716) bank27325 +27326 POINT(39.80209528412171 74.20377803447967) bank27326 +27327 POINT(41.13282551161727 74.86933196252231) bank27327 +27328 POINT(41.30635701349982 73.72832463300128) bank27328 +27329 POINT(40.88657894786017 73.5075593725603) bank27329 +27330 POINT(41.44326589420762 74.17209888103285) bank27330 +27331 POINT(41.178133699540155 73.8936221855077) bank27331 +27332 POINT(41.42980582182021 74.71640432294427) bank27332 +27333 POINT(39.735853121389965 74.0947759584497) bank27333 +27334 POINT(41.31234369658271 73.98457689081141) bank27334 +27335 POINT(40.063487025231204 74.85593291214927) bank27335 +27336 POINT(39.725472165514695 73.55361714987995) bank27336 +27337 POINT(41.30051442242999 74.57260495948154) bank27337 +27338 POINT(41.24910792374773 74.06852270819392) bank27338 +27339 POINT(41.10265816178327 73.2853207579908) bank27339 +27340 POINT(40.600925671500825 73.20369052292222) bank27340 +27341 POINT(40.76606996329758 74.71815493646031) bank27341 +27342 POINT(39.76745138626252 74.35809678600802) bank27342 +27343 POINT(41.4906607897626 73.86934330598145) bank27343 +27344 POINT(40.79000073064872 73.28672222547506) bank27344 +27345 POINT(40.636674785683965 74.13035580431871) bank27345 +27346 POINT(40.03036015865854 73.4406222994372) bank27346 +27347 POINT(40.29747582293286 74.03805080611733) bank27347 +27348 POINT(39.92529085599441 74.76497034756594) bank27348 +27349 POINT(40.77649210093803 73.62826866353979) bank27349 +27350 POINT(40.7516948612587 73.67760892983671) bank27350 +27351 POINT(41.404335635909675 73.50236402507512) bank27351 +27352 POINT(40.68342982892536 74.05359272032291) bank27352 +27353 POINT(40.646754349357295 73.13773130424738) bank27353 +27354 POINT(40.419866469013684 74.1198672724331) bank27354 +27355 POINT(39.853215646175926 74.66592090940263) bank27355 +27356 POINT(40.35223924116276 73.10791109839393) bank27356 +27357 POINT(41.66163111758507 74.28421615991276) bank27357 +27358 POINT(40.04351272249952 73.2429831160782) bank27358 +27359 POINT(39.97707205820264 74.26584960085435) bank27359 +27360 POINT(39.89518811313254 74.58930998517478) bank27360 +27361 POINT(41.468731638828 74.6991697178953) bank27361 +27362 POINT(40.24184503613032 73.81675416498015) bank27362 +27363 POINT(41.47158230303915 73.76136101978283) bank27363 +27364 POINT(39.76384782482809 74.95383552090391) bank27364 +27365 POINT(41.69612634904824 74.35964500538233) bank27365 +27366 POINT(40.77069963570942 74.06092615171238) bank27366 +27367 POINT(41.07226675471441 74.08295848782048) bank27367 +27368 POINT(41.48717049701101 73.12920421017664) bank27368 +27369 POINT(41.40036165989959 73.58279491423608) bank27369 +27370 POINT(41.41107890392853 73.90307404006522) bank27370 +27371 POINT(40.16361982037418 73.40216696305401) bank27371 +27372 POINT(39.82836947548063 73.37739219675633) bank27372 +27373 POINT(41.563998198247674 74.96142198632974) bank27373 +27374 POINT(41.05746491872414 73.15763665971224) bank27374 +27375 POINT(41.53250793061511 74.4586580163151) bank27375 +27376 POINT(40.901323374383864 74.21663395311502) bank27376 +27377 POINT(40.48320703151176 74.5999219171856) bank27377 +27378 POINT(41.00366285083966 74.05652798308435) bank27378 +27379 POINT(41.365167528740514 73.45423282492932) bank27379 +27380 POINT(40.950120708659945 74.7920431189886) bank27380 +27381 POINT(39.92329996888068 74.39242614420431) bank27381 +27382 POINT(39.90619360306237 73.55955459005237) bank27382 +27383 POINT(39.73926967318118 73.15796285502937) bank27383 +27384 POINT(41.25828488322587 73.81746297103099) bank27384 +27385 POINT(39.793505805359885 73.46970593129714) bank27385 +27386 POINT(40.256806354981876 73.72689632446703) bank27386 +27387 POINT(40.65729072190919 73.3845136779848) bank27387 +27388 POINT(40.657976049085626 74.47339391980509) bank27388 +27389 POINT(40.91754047972117 74.82077159307579) bank27389 +27390 POINT(40.22726162767929 74.77095769972519) bank27390 +27391 POINT(40.884750064919395 74.44194488175975) bank27391 +27392 POINT(40.685234677892595 73.3241797875065) bank27392 +27393 POINT(39.73447624161174 74.27617725504427) bank27393 +27394 POINT(40.364494874200915 73.31843817213745) bank27394 +27395 POINT(41.63963090143559 73.69025431903167) bank27395 +27396 POINT(41.551693575397 73.67904313506266) bank27396 +27397 POINT(41.33712002698953 73.12881354352885) bank27397 +27398 POINT(41.64104835437516 74.59204386097403) bank27398 +27399 POINT(41.30087351157264 74.30084765075195) bank27399 +27400 POINT(40.052812600193946 73.93324228919431) bank27400 +27401 POINT(40.00313346045904 73.89513283292234) bank27401 +27402 POINT(40.723771539951244 73.3579111438016) bank27402 +27403 POINT(40.278968874878686 73.0430685385727) bank27403 +27404 POINT(39.71692055917929 74.98515203432713) bank27404 +27405 POINT(41.64490627217649 74.20982673010286) bank27405 +27406 POINT(39.860799105774426 74.81710661539678) bank27406 +27407 POINT(41.69879814317326 75.00145653601906) bank27407 +27408 POINT(41.67662319791577 73.54366952826629) bank27408 +27409 POINT(40.98431637163971 73.24069225226924) bank27409 +27410 POINT(40.58674069096292 73.50788863654886) bank27410 +27411 POINT(39.80839028925781 74.2294912137601) bank27411 +27412 POINT(40.284884529388485 73.67417401354032) bank27412 +27413 POINT(41.44299497664225 74.25280394569087) bank27413 +27414 POINT(39.833367007412846 74.76577092409197) bank27414 +27415 POINT(40.18553668298856 74.51178037221727) bank27415 +27416 POINT(39.80696653339952 74.92825851366192) bank27416 +27417 POINT(40.88890388707228 74.14720253177387) bank27417 +27418 POINT(40.941360366511034 73.24976643021314) bank27418 +27419 POINT(40.84962449127136 74.24680671937618) bank27419 +27420 POINT(40.43051065998181 73.3429282405317) bank27420 +27421 POINT(40.88645568040063 74.73647396649359) bank27421 +27422 POINT(40.861744716548614 73.62138563463373) bank27422 +27423 POINT(40.48314017915831 74.28322241480626) bank27423 +27424 POINT(40.09871903541168 74.8524642914106) bank27424 +27425 POINT(40.55727082995203 74.55385859444598) bank27425 +27426 POINT(41.51868961097205 73.13688501482103) bank27426 +27427 POINT(41.45325140675862 74.47194899070138) bank27427 +27428 POINT(40.77858897977694 73.43122794151967) bank27428 +27429 POINT(40.904661076446565 73.37466427204679) bank27429 +27430 POINT(41.61773805867219 74.39845490945389) bank27430 +27431 POINT(40.48423428700005 73.92483607917721) bank27431 +27432 POINT(41.43132695020961 73.41866782023187) bank27432 +27433 POINT(41.05743953874209 73.59799845424271) bank27433 +27434 POINT(40.50109632520093 74.62467415218258) bank27434 +27435 POINT(40.31970405964483 73.49016403657691) bank27435 +27436 POINT(40.168461328259205 73.95620520620766) bank27436 +27437 POINT(41.36925738202658 73.5920722286385) bank27437 +27438 POINT(41.428059977306404 73.76598625513301) bank27438 +27439 POINT(40.40885932279623 73.72250565903204) bank27439 +27440 POINT(41.689380310355375 73.20626166180531) bank27440 +27441 POINT(39.8685516501562 73.36842798777083) bank27441 +27442 POINT(40.80249051543541 74.2871326448524) bank27442 +27443 POINT(41.51100734585925 73.74830546814005) bank27443 +27444 POINT(39.950800951521 73.96533470303079) bank27444 +27445 POINT(41.42437512028054 73.06030710202195) bank27445 +27446 POINT(40.47052066076644 74.56337448169188) bank27446 +27447 POINT(40.06249135571469 73.107777430052) bank27447 +27448 POINT(40.09869809210505 74.07343604217296) bank27448 +27449 POINT(40.27052800241387 74.21135087578159) bank27449 +27450 POINT(39.72052240396641 74.24468173153669) bank27450 +27451 POINT(40.05263967692129 74.82765804204556) bank27451 +27452 POINT(41.062976536744905 73.84806175134823) bank27452 +27453 POINT(41.02658679041001 74.17999919160964) bank27453 +27454 POINT(40.0054489433538 74.21808469752128) bank27454 +27455 POINT(40.211670583141974 74.20186614473789) bank27455 +27456 POINT(40.987387509561984 74.92338779294012) bank27456 +27457 POINT(40.81777686686662 73.21102456281588) bank27457 +27458 POINT(40.0882579387238 74.13642103326687) bank27458 +27459 POINT(39.728018404836526 74.00772162525871) bank27459 +27460 POINT(39.857376209131964 73.71711721608852) bank27460 +27461 POINT(40.97923529154999 73.07838810362333) bank27461 +27462 POINT(40.746518962611795 74.94033540847786) bank27462 +27463 POINT(41.70975751669584 73.13751371510126) bank27463 +27464 POINT(39.85431686368032 74.01952080653398) bank27464 +27465 POINT(41.14418372862744 73.27843816663888) bank27465 +27466 POINT(41.602324921060834 73.43613859183787) bank27466 +27467 POINT(41.00876551140688 73.15354371507678) bank27467 +27468 POINT(40.455020746414725 74.61156975953088) bank27468 +27469 POINT(41.63241829579722 74.24423125130757) bank27469 +27470 POINT(40.411959366097676 73.56619355897286) bank27470 +27471 POINT(40.37714601371477 74.994560831324) bank27471 +27472 POINT(40.33908476942297 74.30991979309677) bank27472 +27473 POINT(40.9461038957582 74.03027594400534) bank27473 +27474 POINT(40.24523672895862 74.84102779911147) bank27474 +27475 POINT(40.097914265344684 73.83973224139497) bank27475 +27476 POINT(40.59843352481184 74.20511896003241) bank27476 +27477 POINT(40.15365162143349 73.61410651039245) bank27477 +27478 POINT(40.322201525402704 74.8185130843531) bank27478 +27479 POINT(41.60929148441742 74.4130564143796) bank27479 +27480 POINT(41.37020307817597 73.52708639987195) bank27480 +27481 POINT(40.531372993369985 73.39790874602949) bank27481 +27482 POINT(41.309721082760504 73.65405751282914) bank27482 +27483 POINT(40.42794172478149 74.20521049216408) bank27483 +27484 POINT(40.36885307482684 73.65361579694509) bank27484 +27485 POINT(40.28658689272771 74.54639862180238) bank27485 +27486 POINT(41.454952771503585 73.1869742638792) bank27486 +27487 POINT(40.4390672310872 74.39745069523802) bank27487 +27488 POINT(41.51352938983777 74.12104145400608) bank27488 +27489 POINT(39.87625324583489 73.28960834101703) bank27489 +27490 POINT(40.64165046128127 74.07271057881326) bank27490 +27491 POINT(41.360914338743136 74.27241374892282) bank27491 +27492 POINT(41.34367676150759 74.86438066769438) bank27492 +27493 POINT(39.94007071560147 74.6489224282191) bank27493 +27494 POINT(40.574502456646705 73.64214784050782) bank27494 +27495 POINT(40.96136943496853 73.22196627915423) bank27495 +27496 POINT(41.46716499167184 73.78931117403539) bank27496 +27497 POINT(41.21702266403663 73.8166947133831) bank27497 +27498 POINT(41.227465259105585 73.46724706905948) bank27498 +27499 POINT(41.07460111708779 74.35464470467565) bank27499 +27500 POINT(39.799220489365695 73.77156719972449) bank27500 +27501 POINT(41.15514090260342 74.25532240808758) bank27501 +27502 POINT(41.508411361824066 74.89244031031149) bank27502 +27503 POINT(40.08972922853831 74.47722892245783) bank27503 +27504 POINT(40.430581636024456 73.73946360436459) bank27504 +27505 POINT(40.558399141510165 73.45382285466584) bank27505 +27506 POINT(40.89788142087011 73.84725808586478) bank27506 +27507 POINT(41.27189312426654 73.45198874727828) bank27507 +27508 POINT(41.03351228741816 74.95059525315258) bank27508 +27509 POINT(40.800953380586805 73.74701677603909) bank27509 +27510 POINT(40.69206548289278 73.2736851860869) bank27510 +27511 POINT(39.80545695736021 73.07317131138021) bank27511 +27512 POINT(40.42697883963 74.03541081578932) bank27512 +27513 POINT(41.192814331958594 73.62184907230575) bank27513 +27514 POINT(41.50988688183427 73.55411274832055) bank27514 +27515 POINT(39.8726447345277 73.12605900659325) bank27515 +27516 POINT(41.07182659206853 73.30577004504335) bank27516 +27517 POINT(40.035758637755265 74.08679008754116) bank27517 +27518 POINT(40.578014357095775 74.29499918185049) bank27518 +27519 POINT(40.963705567482776 74.56986075884905) bank27519 +27520 POINT(40.28581625463156 73.82054052022895) bank27520 +27521 POINT(41.2775072346776 74.63126927984024) bank27521 +27522 POINT(40.610769283026364 73.08798034494369) bank27522 +27523 POINT(40.74043602663843 73.46598077700291) bank27523 +27524 POINT(40.270307314318444 74.59524954918629) bank27524 +27525 POINT(41.109391672860134 74.79599331938367) bank27525 +27526 POINT(41.60077237339208 73.4983562063645) bank27526 +27527 POINT(40.42978015354406 73.37598124341741) bank27527 +27528 POINT(40.46309589516061 73.277051234092) bank27528 +27529 POINT(41.494256181144095 74.74761781137825) bank27529 +27530 POINT(41.70247102399227 74.82508374224233) bank27530 +27531 POINT(41.01966748054419 73.23337752122534) bank27531 +27532 POINT(40.89994161386082 73.08408342636532) bank27532 +27533 POINT(41.619573766347 74.55902022598471) bank27533 +27534 POINT(41.15840914861921 74.53185435229936) bank27534 +27535 POINT(40.746902038439586 73.8306239777888) bank27535 +27536 POINT(40.14707338616811 74.96803873475082) bank27536 +27537 POINT(40.68009763210388 74.57789948245038) bank27537 +27538 POINT(40.707321288594606 74.08023380173256) bank27538 +27539 POINT(41.44522609632108 73.25712384991589) bank27539 +27540 POINT(41.52902483349084 73.48570696531564) bank27540 +27541 POINT(41.296650218203936 73.81083230601492) bank27541 +27542 POINT(40.890294821816596 73.96321437054557) bank27542 +27543 POINT(40.86149115198638 73.64261638134413) bank27543 +27544 POINT(40.944824823880715 74.40193941136411) bank27544 +27545 POINT(41.16430261834828 74.32802262320989) bank27545 +27546 POINT(40.27802086239068 74.21945513187082) bank27546 +27547 POINT(41.25445840665448 73.83548529347438) bank27547 +27548 POINT(41.181318641176276 73.06716845709539) bank27548 +27549 POINT(40.605063013141795 73.32289411906757) bank27549 +27550 POINT(40.504212977705116 73.29691518641337) bank27550 +27551 POINT(41.51302268890238 73.07591269309373) bank27551 +27552 POINT(40.168950374625396 73.85819460239775) bank27552 +27553 POINT(41.02720573395845 73.34566521830398) bank27553 +27554 POINT(41.01508370136489 73.87026206667088) bank27554 +27555 POINT(41.69262338296034 74.07960776240317) bank27555 +27556 POINT(39.942747305952416 73.07199865904813) bank27556 +27557 POINT(39.79437237067174 74.59375499932952) bank27557 +27558 POINT(40.61500255883896 74.05398084151403) bank27558 +27559 POINT(39.881004907799316 74.27724909029884) bank27559 +27560 POINT(41.591928604411144 73.74473861622738) bank27560 +27561 POINT(40.5796540903053 73.62329026772565) bank27561 +27562 POINT(40.59120368876724 73.98890113219583) bank27562 +27563 POINT(41.39062051669301 73.07506612380702) bank27563 +27564 POINT(40.99525822170272 73.82480623497146) bank27564 +27565 POINT(40.48864161518382 74.67396857503797) bank27565 +27566 POINT(40.534490879934566 73.04597596912284) bank27566 +27567 POINT(40.04052010095017 73.30846506223398) bank27567 +27568 POINT(40.12323305420466 74.31798357545019) bank27568 +27569 POINT(40.328551733759944 73.85261043716397) bank27569 +27570 POINT(40.44673739113335 73.59596343650892) bank27570 +27571 POINT(40.8087941688619 74.7535047370607) bank27571 +27572 POINT(40.883457736353336 73.11344519627535) bank27572 +27573 POINT(40.84942828120794 73.7816729929229) bank27573 +27574 POINT(40.746078671344456 73.60122823438611) bank27574 +27575 POINT(40.04370768449423 74.1554533560879) bank27575 +27576 POINT(41.13607186809991 73.9519571543663) bank27576 +27577 POINT(41.660400348632066 73.45182869051324) bank27577 +27578 POINT(41.04240955560313 74.73282323445049) bank27578 +27579 POINT(41.54142286810593 73.1465513494663) bank27579 +27580 POINT(41.424352941164855 73.49452227571847) bank27580 +27581 POINT(40.257380698007175 73.02521127127972) bank27581 +27582 POINT(40.14613175694453 73.30216572514512) bank27582 +27583 POINT(41.56974338260036 73.07140208820488) bank27583 +27584 POINT(40.441760889088506 74.1925248707603) bank27584 +27585 POINT(40.03126938003118 73.56570900596874) bank27585 +27586 POINT(40.85291697177069 73.80579714327303) bank27586 +27587 POINT(40.01689302461414 73.2647525901774) bank27587 +27588 POINT(40.57598391278229 74.68361235465987) bank27588 +27589 POINT(41.25511642169153 73.8901915957307) bank27589 +27590 POINT(40.210930008227166 74.27537674421392) bank27590 +27591 POINT(40.28332916903668 73.32499737056189) bank27591 +27592 POINT(40.71461241433674 73.11602717663864) bank27592 +27593 POINT(40.27101138831625 74.46796462231275) bank27593 +27594 POINT(41.03262205893506 74.63159199111507) bank27594 +27595 POINT(39.909898441882895 73.82023857338046) bank27595 +27596 POINT(39.75984150530228 73.15455730054396) bank27596 +27597 POINT(41.401140071234636 73.16860040219407) bank27597 +27598 POINT(40.27918114459884 74.23796593330354) bank27598 +27599 POINT(40.56127081396836 73.58756958251894) bank27599 +27600 POINT(41.58923752129682 73.84903135540165) bank27600 +27601 POINT(39.76689021231646 73.4721742402654) bank27601 +27602 POINT(39.94545551894855 73.51675556741296) bank27602 +27603 POINT(39.87884241861961 74.08718803763385) bank27603 +27604 POINT(40.533041573987774 73.43295657824501) bank27604 +27605 POINT(40.78671369484084 74.05121113033204) bank27605 +27606 POINT(41.70278697769873 74.01673580123591) bank27606 +27607 POINT(41.63672871215798 73.1994753059629) bank27607 +27608 POINT(41.02055744833978 73.81083894592585) bank27608 +27609 POINT(40.288499352775624 73.2988127804863) bank27609 +27610 POINT(40.89391792018511 73.53434183042971) bank27610 +27611 POINT(41.30178010642951 74.7938674182487) bank27611 +27612 POINT(41.08988676526382 73.4698807356584) bank27612 +27613 POINT(40.766259677058855 73.05241829817591) bank27613 +27614 POINT(39.74227692403738 74.32836598807056) bank27614 +27615 POINT(40.89920942008596 73.40410494033088) bank27615 +27616 POINT(41.38877691717702 74.79209107659135) bank27616 +27617 POINT(40.07696955796867 74.92063155789722) bank27617 +27618 POINT(41.12802929431596 74.83748930332334) bank27618 +27619 POINT(40.043925164223836 74.95117613819113) bank27619 +27620 POINT(41.014411570581764 74.50043027867781) bank27620 +27621 POINT(41.204997441996504 74.27295852628869) bank27621 +27622 POINT(41.4012373516621 73.87628906027116) bank27622 +27623 POINT(40.12723040165141 73.95980938253753) bank27623 +27624 POINT(40.26229318100678 74.04343660117429) bank27624 +27625 POINT(40.759193204454434 74.8583453397234) bank27625 +27626 POINT(41.49447502695104 74.17265999556102) bank27626 +27627 POINT(41.37885216153637 74.59890505641795) bank27627 +27628 POINT(41.08373415162958 73.80757696582067) bank27628 +27629 POINT(40.78026845207159 74.10297379047144) bank27629 +27630 POINT(41.51628791869997 74.73223652638954) bank27630 +27631 POINT(40.12380694275618 73.56441229662092) bank27631 +27632 POINT(41.16181296712483 74.89157776045575) bank27632 +27633 POINT(40.813084578739506 74.03887496906206) bank27633 +27634 POINT(40.821631027268545 74.95810154925051) bank27634 +27635 POINT(41.40690242695976 74.02136887986232) bank27635 +27636 POINT(40.338500198700665 73.89710363089843) bank27636 +27637 POINT(41.31294537250433 73.42264546514886) bank27637 +27638 POINT(41.642134653805066 73.7349954035959) bank27638 +27639 POINT(40.56489102323707 73.36790251946606) bank27639 +27640 POINT(41.04024234392025 74.84713913573971) bank27640 +27641 POINT(40.63190910950294 73.3031728045232) bank27641 +27642 POINT(41.38267909009941 73.60181402272067) bank27642 +27643 POINT(39.860331961291116 74.79103211501057) bank27643 +27644 POINT(40.54370027615095 74.56931537725308) bank27644 +27645 POINT(39.84899981688938 73.72815612847442) bank27645 +27646 POINT(40.78145800578821 73.4143593875579) bank27646 +27647 POINT(40.91593143218188 74.32673957088525) bank27647 +27648 POINT(40.59012121849348 74.47390885621597) bank27648 +27649 POINT(41.3648925681425 73.22392403989234) bank27649 +27650 POINT(40.88521360948077 74.63544516612599) bank27650 +27651 POINT(39.92801190392582 73.363016718056) bank27651 +27652 POINT(40.61196642235229 74.44488151711157) bank27652 +27653 POINT(39.73255819252534 73.9163178278614) bank27653 +27654 POINT(40.61343939233682 73.93353823666574) bank27654 +27655 POINT(41.66441054298275 74.89165068187926) bank27655 +27656 POINT(40.218093481059114 73.27043606029) bank27656 +27657 POINT(41.654517539965 73.09887074550531) bank27657 +27658 POINT(40.441487923190856 73.3185084939915) bank27658 +27659 POINT(39.74149880910876 74.88501283063961) bank27659 +27660 POINT(39.96823081039568 74.70329114409574) bank27660 +27661 POINT(39.86088183405433 73.48503844085373) bank27661 +27662 POINT(41.19143734808251 73.79572438131059) bank27662 +27663 POINT(41.04933076718999 73.85801233357054) bank27663 +27664 POINT(41.49384968284524 73.39142575993388) bank27664 +27665 POINT(40.90862183712829 74.21632087491503) bank27665 +27666 POINT(40.90901424683126 74.3921983126735) bank27666 +27667 POINT(41.161686848783425 73.69570605225482) bank27667 +27668 POINT(39.983281682096155 73.6495817162784) bank27668 +27669 POINT(40.589025207056665 73.92424384022732) bank27669 +27670 POINT(41.2298551917877 73.0175469501992) bank27670 +27671 POINT(41.70437645576351 73.50333327870766) bank27671 +27672 POINT(41.08156159538872 74.42391695379851) bank27672 +27673 POINT(40.016376079057814 73.7441297905376) bank27673 +27674 POINT(39.92629635561178 74.92773863452942) bank27674 +27675 POINT(40.490613097141086 74.15291358030699) bank27675 +27676 POINT(40.62494950641637 73.78397232554155) bank27676 +27677 POINT(40.84257949286078 73.25925640309171) bank27677 +27678 POINT(40.48994707195003 74.56850758007792) bank27678 +27679 POINT(40.05474810264121 74.55520782054509) bank27679 +27680 POINT(40.20515272852698 73.5965283940651) bank27680 +27681 POINT(41.090569728946924 74.15259057940146) bank27681 +27682 POINT(40.28047948852364 74.29379922288129) bank27682 +27683 POINT(41.25627941109291 73.97037137694674) bank27683 +27684 POINT(40.70173823394869 73.97858430586692) bank27684 +27685 POINT(40.06328263026771 73.59187108203037) bank27685 +27686 POINT(41.43254470253862 73.99316572572425) bank27686 +27687 POINT(40.10019008214875 73.74225619008868) bank27687 +27688 POINT(40.20857153092745 73.34335125630228) bank27688 +27689 POINT(41.208533325378085 74.27320363002067) bank27689 +27690 POINT(41.55067119137843 73.66964236962707) bank27690 +27691 POINT(41.445580217680494 73.67919449319612) bank27691 +27692 POINT(40.49642234918952 74.10463085401258) bank27692 +27693 POINT(40.52353316972328 74.17710423515838) bank27693 +27694 POINT(40.72938258372147 74.4495695103338) bank27694 +27695 POINT(40.98906383041852 74.7616245588333) bank27695 +27696 POINT(40.857038546205175 73.82523922558832) bank27696 +27697 POINT(41.49561547846777 74.90709046147676) bank27697 +27698 POINT(40.84756924168896 74.59168484541179) bank27698 +27699 POINT(40.14907270528266 74.64251144080355) bank27699 +27700 POINT(40.87991013056762 73.4461975532532) bank27700 +27701 POINT(40.13017487274351 74.69928688709548) bank27701 +27702 POINT(40.530713642151795 73.47239658674896) bank27702 +27703 POINT(39.83974475729371 74.49311974543959) bank27703 +27704 POINT(40.4020930244716 74.32453314119321) bank27704 +27705 POINT(39.840650120754454 73.82242656025778) bank27705 +27706 POINT(40.42488133133937 73.06378174149717) bank27706 +27707 POINT(41.71085767259407 73.86522001150658) bank27707 +27708 POINT(40.8089806131551 74.1962857389096) bank27708 +27709 POINT(39.8094393408713 73.14658825655594) bank27709 +27710 POINT(39.8935760007092 74.99578397100747) bank27710 +27711 POINT(39.92469830717738 74.59610206038856) bank27711 +27712 POINT(40.16012225993005 74.27336435582093) bank27712 +27713 POINT(40.65624696595295 73.42505216376289) bank27713 +27714 POINT(40.23860058484693 73.61119059870795) bank27714 +27715 POINT(41.688549346903606 73.39444080272057) bank27715 +27716 POINT(39.90230259896141 74.77602985883894) bank27716 +27717 POINT(40.45602114990609 74.08951101743493) bank27717 +27718 POINT(41.01901161357485 73.12522309884082) bank27718 +27719 POINT(41.41314546906095 73.54393843067128) bank27719 +27720 POINT(40.10620071992163 74.34686779978307) bank27720 +27721 POINT(41.39910673538337 73.30739035016552) bank27721 +27722 POINT(39.85171864229788 74.35926929841959) bank27722 +27723 POINT(41.17507518141518 73.87477843367503) bank27723 +27724 POINT(40.50676362858807 73.5734156433853) bank27724 +27725 POINT(41.05289657846695 73.69440453683266) bank27725 +27726 POINT(40.08819599341199 73.84054161782878) bank27726 +27727 POINT(41.38554969166239 74.96942163002792) bank27727 +27728 POINT(41.06240870606442 73.34213626797239) bank27728 +27729 POINT(41.20048986461934 73.52497612145706) bank27729 +27730 POINT(40.233824886037944 74.63881990413832) bank27730 +27731 POINT(40.838772467526155 73.59458978655087) bank27731 +27732 POINT(40.42636966577473 74.21995600930333) bank27732 +27733 POINT(40.639953521371616 73.39036576394277) bank27733 +27734 POINT(40.84845214779123 73.41729447626705) bank27734 +27735 POINT(39.90014775332455 74.67642620025019) bank27735 +27736 POINT(40.55444636519072 74.0507494402631) bank27736 +27737 POINT(40.258070633494974 74.20697064752282) bank27737 +27738 POINT(40.154076063663865 74.72405405178571) bank27738 +27739 POINT(40.591524250782214 73.30815417279511) bank27739 +27740 POINT(40.08173578733064 73.94718964137311) bank27740 +27741 POINT(40.96715691786558 73.65636403293044) bank27741 +27742 POINT(39.84478003374478 74.18489692322069) bank27742 +27743 POINT(40.35096135096817 73.26670201242156) bank27743 +27744 POINT(40.635550652130156 74.41361574966334) bank27744 +27745 POINT(40.912090327628 73.14352918425861) bank27745 +27746 POINT(41.4280167353742 74.22349816886049) bank27746 +27747 POINT(40.4671186255657 73.0093589447938) bank27747 +27748 POINT(40.20799600593429 73.59448847622737) bank27748 +27749 POINT(41.67022055488862 73.94227248379576) bank27749 +27750 POINT(41.34654340025364 74.91236502114909) bank27750 +27751 POINT(41.1139401473505 73.97634395288286) bank27751 +27752 POINT(41.240031878695305 73.6723947726668) bank27752 +27753 POINT(41.36991861784117 74.1404253267031) bank27753 +27754 POINT(41.0631869368068 73.68332713867383) bank27754 +27755 POINT(39.74426777037652 73.4505208850187) bank27755 +27756 POINT(40.38319243097532 73.16056364161496) bank27756 +27757 POINT(40.20206943764143 73.96317915433201) bank27757 +27758 POINT(39.79732841967134 74.50138894901919) bank27758 +27759 POINT(40.664425249631165 73.67217496086047) bank27759 +27760 POINT(39.720868430738754 74.93253192855431) bank27760 +27761 POINT(40.15266965651624 74.59166983974784) bank27761 +27762 POINT(40.94821986360933 74.82695945093941) bank27762 +27763 POINT(40.04463750849217 74.04801045062187) bank27763 +27764 POINT(40.53862544462434 74.33601550073033) bank27764 +27765 POINT(40.51687054575148 73.2509671249799) bank27765 +27766 POINT(39.86074968503802 73.98475231279139) bank27766 +27767 POINT(40.846677791654635 74.67348757460525) bank27767 +27768 POINT(40.7964568652392 73.3774372171163) bank27768 +27769 POINT(39.79996540033603 74.05943765226282) bank27769 +27770 POINT(40.05391603529401 73.23300383819044) bank27770 +27771 POINT(40.889314881066106 74.38773228732433) bank27771 +27772 POINT(40.4684578798503 74.21624782642942) bank27772 +27773 POINT(39.878992237681345 73.79186147620014) bank27773 +27774 POINT(40.50109779816828 74.74114435407708) bank27774 +27775 POINT(40.540672830198275 73.70333789386117) bank27775 +27776 POINT(40.21095903434989 73.79751705867702) bank27776 +27777 POINT(40.476921057722 74.91072556046528) bank27777 +27778 POINT(41.13091110367527 73.09895992408066) bank27778 +27779 POINT(41.42082106733081 74.47136333560483) bank27779 +27780 POINT(39.78602847797011 74.19220626734763) bank27780 +27781 POINT(39.750706122058205 73.31314341219493) bank27781 +27782 POINT(40.8577664192903 73.87186701970371) bank27782 +27783 POINT(40.78916050292341 74.78167535372299) bank27783 +27784 POINT(41.259930583075636 74.17089746460917) bank27784 +27785 POINT(40.499798554775744 73.09384532240219) bank27785 +27786 POINT(41.39748647896805 73.92313132485747) bank27786 +27787 POINT(40.02020790610296 74.32655458143141) bank27787 +27788 POINT(41.42810641529163 73.47192879029959) bank27788 +27789 POINT(39.7190299274374 73.81658500203528) bank27789 +27790 POINT(41.20939628154195 74.21189569447785) bank27790 +27791 POINT(40.1167145546326 73.24262468909612) bank27791 +27792 POINT(41.121653114952345 73.50925939073112) bank27792 +27793 POINT(40.87551862742561 73.07063064189322) bank27793 +27794 POINT(40.90876833868784 74.17975945316417) bank27794 +27795 POINT(41.021730802166104 73.50213496420662) bank27795 +27796 POINT(39.83429040520762 73.23282170090924) bank27796 +27797 POINT(40.874933741538484 74.43120381223123) bank27797 +27798 POINT(41.2127641144929 74.00160713688557) bank27798 +27799 POINT(41.56156001152413 74.71000331646395) bank27799 +27800 POINT(40.67839340872741 73.32064794091768) bank27800 +27801 POINT(40.426703374611044 74.5763241644701) bank27801 +27802 POINT(41.429721495550304 74.43319009151998) bank27802 +27803 POINT(40.441810861340606 74.34080824618097) bank27803 +27804 POINT(40.319838535358365 74.21438707490304) bank27804 +27805 POINT(41.263826288738365 74.37453321602453) bank27805 +27806 POINT(41.51963665366365 74.00355955912326) bank27806 +27807 POINT(40.42747298379511 74.41633031989326) bank27807 +27808 POINT(41.644600956367434 74.98280250530304) bank27808 +27809 POINT(40.282939806199884 74.76562988221414) bank27809 +27810 POINT(41.43489914851248 73.03457126533445) bank27810 +27811 POINT(40.43106195029211 73.91144265320278) bank27811 +27812 POINT(39.90351362578538 73.97137420549977) bank27812 +27813 POINT(40.47146478092216 74.63768177452246) bank27813 +27814 POINT(41.46822176257648 73.31708387077886) bank27814 +27815 POINT(41.47415515696632 74.9821535871712) bank27815 +27816 POINT(40.89375626338972 73.92139045558818) bank27816 +27817 POINT(40.96723857425831 74.35049912356074) bank27817 +27818 POINT(40.83753821401626 73.2238570391803) bank27818 +27819 POINT(40.278869030390844 74.69293560798502) bank27819 +27820 POINT(41.55270522493148 73.35002512375934) bank27820 +27821 POINT(41.447042675102736 74.20219305988222) bank27821 +27822 POINT(41.42093537030118 74.12689741776984) bank27822 +27823 POINT(41.27426797721756 74.75379573100281) bank27823 +27824 POINT(39.915617465202416 73.62560986217804) bank27824 +27825 POINT(40.9764317283736 73.22555506584987) bank27825 +27826 POINT(39.80858393639094 74.79766020646908) bank27826 +27827 POINT(41.01969792296947 74.98009008072836) bank27827 +27828 POINT(41.28707897682657 74.23446122122427) bank27828 +27829 POINT(40.58931010593954 73.6072831905229) bank27829 +27830 POINT(40.984451300227555 74.2432009634726) bank27830 +27831 POINT(39.75309712958597 74.23482279590101) bank27831 +27832 POINT(41.24323798458721 73.99061449621927) bank27832 +27833 POINT(40.77744781240397 74.32216196251268) bank27833 +27834 POINT(40.66512322593792 73.72801517402279) bank27834 +27835 POINT(41.60871214787948 74.60726994717969) bank27835 +27836 POINT(40.72751787282055 73.40085251352271) bank27836 +27837 POINT(39.846301327298505 74.35406753328921) bank27837 +27838 POINT(39.99936444577354 73.52841897721703) bank27838 +27839 POINT(40.389429996983985 73.0534647784331) bank27839 +27840 POINT(40.507353155741086 73.3516542894623) bank27840 +27841 POINT(39.98736740665511 74.11951354969375) bank27841 +27842 POINT(41.49512939159437 74.68419483327722) bank27842 +27843 POINT(41.04862343110771 74.56637967745722) bank27843 +27844 POINT(40.31721416306883 74.27859338632265) bank27844 +27845 POINT(40.40492762032763 74.21011725454424) bank27845 +27846 POINT(39.82553458235405 73.49290780125558) bank27846 +27847 POINT(40.588065700731946 73.73500807985621) bank27847 +27848 POINT(40.71473482046811 74.07232126581845) bank27848 +27849 POINT(40.27557255725495 74.17644609837498) bank27849 +27850 POINT(40.85897355928785 74.67226001281908) bank27850 +27851 POINT(40.49003349652554 74.9080102445433) bank27851 +27852 POINT(40.092020308327655 74.51554583267202) bank27852 +27853 POINT(40.22514742145134 73.0513367271775) bank27853 +27854 POINT(40.42863395446831 74.83957539852783) bank27854 +27855 POINT(41.64408681659997 73.9156085514257) bank27855 +27856 POINT(39.801915945782405 74.92841878294236) bank27856 +27857 POINT(40.83078860938896 73.37033889459757) bank27857 +27858 POINT(41.475833899470864 73.93632558800915) bank27858 +27859 POINT(39.954902309157966 74.85658440299046) bank27859 +27860 POINT(39.97874055630232 74.61867148115392) bank27860 +27861 POINT(40.01252040931466 73.91847053119714) bank27861 +27862 POINT(39.8368510519822 73.53339860831366) bank27862 +27863 POINT(40.50752913476881 73.61402856445285) bank27863 +27864 POINT(41.07147223927855 73.51136097928878) bank27864 +27865 POINT(40.843523372518284 73.34276594635304) bank27865 +27866 POINT(41.4562238436622 74.20487639084061) bank27866 +27867 POINT(40.01806766611433 73.79357923195062) bank27867 +27868 POINT(39.740001667239945 74.44540450996314) bank27868 +27869 POINT(40.07320215205841 74.32872539982945) bank27869 +27870 POINT(40.2924028154384 74.7890026460828) bank27870 +27871 POINT(39.88768280891534 74.81977832767421) bank27871 +27872 POINT(40.48600079745261 74.13513160684498) bank27872 +27873 POINT(40.199675550317984 74.71312276501753) bank27873 +27874 POINT(41.084463109155216 74.92270771013818) bank27874 +27875 POINT(40.75217308389497 73.06278427942927) bank27875 +27876 POINT(41.05884672485086 73.4024785260701) bank27876 +27877 POINT(41.638815836909785 74.7577858825312) bank27877 +27878 POINT(40.43040483833149 74.33247235953449) bank27878 +27879 POINT(40.2999923227507 74.01394097654635) bank27879 +27880 POINT(41.56015721235528 73.33401504189636) bank27880 +27881 POINT(39.825651255995766 73.18199010305352) bank27881 +27882 POINT(39.8235932806802 73.1282106777616) bank27882 +27883 POINT(41.52233269874014 73.36159518685659) bank27883 +27884 POINT(40.429628992297225 74.11343018096522) bank27884 +27885 POINT(39.95876991306075 73.01753312123493) bank27885 +27886 POINT(41.4663608272242 73.4131142860359) bank27886 +27887 POINT(40.58935213530758 74.56320421019525) bank27887 +27888 POINT(40.46642677401522 74.83226279802668) bank27888 +27889 POINT(40.9605371841419 74.51233220735772) bank27889 +27890 POINT(40.90434874633955 74.83454724707714) bank27890 +27891 POINT(39.90595453366081 74.4959255403734) bank27891 +27892 POINT(41.707576279361156 74.50452206511261) bank27892 +27893 POINT(40.57402472071514 74.40904489936939) bank27893 +27894 POINT(40.51307164434012 74.78268625283914) bank27894 +27895 POINT(40.08945327447541 73.73434889760051) bank27895 +27896 POINT(41.09989495846435 74.77555636447353) bank27896 +27897 POINT(40.006984346401595 74.7560035450979) bank27897 +27898 POINT(41.43515735270534 74.89877060706227) bank27898 +27899 POINT(39.822479785538995 74.71502642801408) bank27899 +27900 POINT(40.79448753182206 74.56646831804827) bank27900 +27901 POINT(40.77178699052753 74.14310175882659) bank27901 +27902 POINT(40.74027431676635 73.42327130352601) bank27902 +27903 POINT(40.69847487617147 74.25279505998144) bank27903 +27904 POINT(40.63304071380348 74.73295194965297) bank27904 +27905 POINT(40.10465429087448 74.837731192361) bank27905 +27906 POINT(41.374953267015954 73.97969147314768) bank27906 +27907 POINT(40.42646632916739 73.07923548777234) bank27907 +27908 POINT(39.945418508009986 73.08010431818333) bank27908 +27909 POINT(41.4780560018773 74.78197934159591) bank27909 +27910 POINT(39.85505700360701 74.68033082414242) bank27910 +27911 POINT(40.73272541112805 73.63357776209293) bank27911 +27912 POINT(40.52967278652109 73.09887324837277) bank27912 +27913 POINT(41.58610548666989 73.0333953653862) bank27913 +27914 POINT(41.09813381740148 73.96315708971876) bank27914 +27915 POINT(41.522323269946064 73.25938186933729) bank27915 +27916 POINT(41.359889432013965 74.75731606121623) bank27916 +27917 POINT(40.900984234987774 74.51476325718599) bank27917 +27918 POINT(41.24592975672222 73.60486740464955) bank27918 +27919 POINT(39.97525323449927 73.20056239515931) bank27919 +27920 POINT(39.836134526383134 73.42517672019724) bank27920 +27921 POINT(41.09912241669874 74.3691268493537) bank27921 +27922 POINT(41.28545108940256 74.28842781764651) bank27922 +27923 POINT(41.590203177577706 74.36509787213936) bank27923 +27924 POINT(40.8262132143147 74.10144619023644) bank27924 +27925 POINT(39.87794710159908 73.77726294374483) bank27925 +27926 POINT(41.20929387338045 73.74434645035137) bank27926 +27927 POINT(40.79857647735413 73.90015228811077) bank27927 +27928 POINT(40.83805316549375 73.77487946133544) bank27928 +27929 POINT(40.708164919212514 74.33251037022096) bank27929 +27930 POINT(41.10525335018713 74.25467880885641) bank27930 +27931 POINT(40.44131942741075 74.95757379469954) bank27931 +27932 POINT(41.652216026690645 74.34880644423052) bank27932 +27933 POINT(40.16018874154748 75.00323517168073) bank27933 +27934 POINT(40.07292926113572 74.74650875514085) bank27934 +27935 POINT(41.18757207270806 73.80170576854218) bank27935 +27936 POINT(40.49424837558671 74.6607995109713) bank27936 +27937 POINT(41.6344459875664 74.09133291562613) bank27937 +27938 POINT(40.67985407378324 73.14759568923111) bank27938 +27939 POINT(40.92721973559419 73.33480409685438) bank27939 +27940 POINT(39.994453503438336 74.75623262471964) bank27940 +27941 POINT(41.49284028553424 73.15355784505735) bank27941 +27942 POINT(41.240287656201524 74.87312618713304) bank27942 +27943 POINT(40.063104772191814 73.76344993009921) bank27943 +27944 POINT(41.37973510111781 74.67986024133954) bank27944 +27945 POINT(39.96198230227138 74.7601874180449) bank27945 +27946 POINT(41.61040317066616 73.62616351719007) bank27946 +27947 POINT(41.71268164443309 73.28401775150863) bank27947 +27948 POINT(40.568165490620885 74.47253326682016) bank27948 +27949 POINT(40.72133318203253 73.99064811158706) bank27949 +27950 POINT(40.67816006353431 73.5110761167092) bank27950 +27951 POINT(40.948726476960275 73.39750270439752) bank27951 +27952 POINT(41.247256682650445 73.61717866485905) bank27952 +27953 POINT(40.43543260075852 74.81876606890214) bank27953 +27954 POINT(40.14555606196764 73.66911931737708) bank27954 +27955 POINT(41.08546308179521 74.24443279467536) bank27955 +27956 POINT(40.05216449166169 74.75829306621038) bank27956 +27957 POINT(41.587036569157846 73.35258390253696) bank27957 +27958 POINT(41.474170453938044 73.12579409057815) bank27958 +27959 POINT(40.88964091138812 74.45485070423464) bank27959 +27960 POINT(39.737898681690105 73.49405551757742) bank27960 +27961 POINT(41.348953074786024 73.83431275413247) bank27961 +27962 POINT(41.1045425525584 73.08349722321664) bank27962 +27963 POINT(40.93413203177228 74.28664510543105) bank27963 +27964 POINT(39.88252020406314 73.52998511861684) bank27964 +27965 POINT(41.71246525776857 74.99650554911408) bank27965 +27966 POINT(41.04568445972695 73.21742845141729) bank27966 +27967 POINT(40.147355335970836 73.61164082417251) bank27967 +27968 POINT(40.900456118090105 73.6927091787262) bank27968 +27969 POINT(40.48349908706519 74.99867212692803) bank27969 +27970 POINT(41.27908578654631 73.73134123936413) bank27970 +27971 POINT(41.44376609103291 74.4848585605296) bank27971 +27972 POINT(41.1652455070293 74.82310419779759) bank27972 +27973 POINT(40.20028341744411 74.393378646293) bank27973 +27974 POINT(39.8740352415778 73.12801620549092) bank27974 +27975 POINT(39.727111436062906 74.30404512177375) bank27975 +27976 POINT(40.02634228653664 74.19491034849813) bank27976 +27977 POINT(41.462677877701026 74.31237697178877) bank27977 +27978 POINT(41.545979769299464 74.68380630058607) bank27978 +27979 POINT(40.117561418826696 73.54415995565708) bank27979 +27980 POINT(40.852836527443706 74.525575000577) bank27980 +27981 POINT(40.00639275608375 74.96913815293682) bank27981 +27982 POINT(41.25405779858458 74.53739120858495) bank27982 +27983 POINT(40.940307569857744 74.14551782041565) bank27983 +27984 POINT(41.58656169716404 74.52051467343632) bank27984 +27985 POINT(41.664410635604916 73.7036634082758) bank27985 +27986 POINT(41.28821812505481 73.59493395054231) bank27986 +27987 POINT(39.8236255081203 73.60478825261389) bank27987 +27988 POINT(39.79186826559977 74.90456322321322) bank27988 +27989 POINT(39.952066237658734 73.70576294670893) bank27989 +27990 POINT(40.93470844582731 73.09241442305921) bank27990 +27991 POINT(41.09165309943658 73.40020975657147) bank27991 +27992 POINT(39.88170452514895 73.23318548168768) bank27992 +27993 POINT(39.90804759874923 74.15878970941134) bank27993 +27994 POINT(41.06634313819391 73.66930030241319) bank27994 +27995 POINT(40.29224942482155 73.46783142813757) bank27995 +27996 POINT(40.464187718485135 74.4890313644301) bank27996 +27997 POINT(41.099211003082 73.61162048452033) bank27997 +27998 POINT(40.70279579648734 74.64840851874678) bank27998 +27999 POINT(40.62676658609405 73.68516343759622) bank27999 +28000 POINT(39.733759132219404 74.24479127421024) bank28000 +28001 POINT(41.00971385789035 74.98835926027029) bank28001 +28002 POINT(41.54431064162945 74.43193375076764) bank28002 +28003 POINT(41.4983756365651 74.21661863160695) bank28003 +28004 POINT(41.22927725213619 74.89718152673929) bank28004 +28005 POINT(40.73463763288344 74.8100490607637) bank28005 +28006 POINT(41.04453215553676 73.06061118698366) bank28006 +28007 POINT(41.412717721690065 74.35644734909029) bank28007 +28008 POINT(41.25265915287408 74.49899001537926) bank28008 +28009 POINT(39.97209922544754 74.70225168307375) bank28009 +28010 POINT(41.471657647588614 73.42867235972753) bank28010 +28011 POINT(40.321858580209245 74.47269385253803) bank28011 +28012 POINT(40.470390781644966 73.6181471667154) bank28012 +28013 POINT(40.7793688135409 73.09209301128163) bank28013 +28014 POINT(40.01467090669213 73.82431906491102) bank28014 +28015 POINT(40.874814320508996 73.38826527914435) bank28015 +28016 POINT(40.828284035230254 74.80733669993164) bank28016 +28017 POINT(41.43096472593075 74.47173202570903) bank28017 +28018 POINT(40.39797956127543 74.96795869848329) bank28018 +28019 POINT(40.56149312714149 74.59671535586136) bank28019 +28020 POINT(40.35324655222334 73.87203699709397) bank28020 +28021 POINT(41.66671211719853 73.76155474311963) bank28021 +28022 POINT(39.91712485342087 74.41789213239733) bank28022 +28023 POINT(40.19738508285185 73.12198264543133) bank28023 +28024 POINT(40.1426441890006 73.8171243879734) bank28024 +28025 POINT(40.35669830364355 73.88154287057691) bank28025 +28026 POINT(41.46358851784951 73.14495468104224) bank28026 +28027 POINT(41.42104639636737 74.54991294581555) bank28027 +28028 POINT(39.871179360094075 74.84528056637906) bank28028 +28029 POINT(39.85667120237868 73.2883669936584) bank28029 +28030 POINT(41.50009217888939 74.66033661141734) bank28030 +28031 POINT(41.0585717382929 73.30100025418456) bank28031 +28032 POINT(40.34482151446678 74.52407752829406) bank28032 +28033 POINT(40.10762937708178 73.67934714267273) bank28033 +28034 POINT(40.71731778839296 73.85156933390577) bank28034 +28035 POINT(40.278704700642 74.70679119695443) bank28035 +28036 POINT(39.8198400421409 74.13292767689582) bank28036 +28037 POINT(41.34886550066967 73.91857055680885) bank28037 +28038 POINT(41.70723927158599 73.75828842495196) bank28038 +28039 POINT(40.4631657883494 73.19713404064139) bank28039 +28040 POINT(40.26304944960711 74.34428937191959) bank28040 +28041 POINT(39.740972193502444 74.42798126115822) bank28041 +28042 POINT(40.30720240691827 74.64429932098473) bank28042 +28043 POINT(40.99980071794678 74.03009105442682) bank28043 +28044 POINT(41.43312406350047 74.19076124620169) bank28044 +28045 POINT(40.250098341963046 73.26488002689196) bank28045 +28046 POINT(41.66472032098293 74.0834063235798) bank28046 +28047 POINT(39.91541211139419 74.5133578612381) bank28047 +28048 POINT(40.49784349379317 73.44366521653157) bank28048 +28049 POINT(40.1063133072038 73.43544207814745) bank28049 +28050 POINT(39.77628578739005 73.40546210106338) bank28050 +28051 POINT(40.886729427242585 73.32629683863473) bank28051 +28052 POINT(39.953803336892605 73.86181312976535) bank28052 +28053 POINT(41.68498241121743 74.9685843017683) bank28053 +28054 POINT(40.71266962928569 74.57845044022272) bank28054 +28055 POINT(41.563476868740544 73.15831033964193) bank28055 +28056 POINT(40.574216571282804 74.47375541476794) bank28056 +28057 POINT(39.72934500211907 74.90538973200172) bank28057 +28058 POINT(41.50794229660217 74.43881181297452) bank28058 +28059 POINT(41.21284524064013 74.65336284823836) bank28059 +28060 POINT(41.00088029944054 74.13843008209678) bank28060 +28061 POINT(40.20885397313669 73.77471312352469) bank28061 +28062 POINT(41.632253510563764 74.42512603703577) bank28062 +28063 POINT(40.93828936609177 73.9154833934789) bank28063 +28064 POINT(40.21233766005125 74.69707200853975) bank28064 +28065 POINT(41.35101485177168 74.69309750280321) bank28065 +28066 POINT(41.15509234542053 73.54527407535264) bank28066 +28067 POINT(40.05067877380957 74.5394665941353) bank28067 +28068 POINT(40.56879932149614 74.84317841264829) bank28068 +28069 POINT(39.94985805487365 73.90159011612985) bank28069 +28070 POINT(40.310789656978706 73.85360778666632) bank28070 +28071 POINT(41.03284318184011 73.85376967126447) bank28071 +28072 POINT(40.762388977555446 73.2495760400046) bank28072 +28073 POINT(40.802160624565 74.09123631472355) bank28073 +28074 POINT(41.28336931094071 74.99365856916141) bank28074 +28075 POINT(40.313464112100114 74.40493851014993) bank28075 +28076 POINT(40.37244785740501 74.73690057134847) bank28076 +28077 POINT(40.75999162868248 74.97281296755013) bank28077 +28078 POINT(40.998947913475135 73.36693945962543) bank28078 +28079 POINT(39.8610493709437 73.50698748438775) bank28079 +28080 POINT(40.64119193252488 73.98593547665222) bank28080 +28081 POINT(39.8104616049108 73.68714021342119) bank28081 +28082 POINT(41.709678392869236 73.60266813193479) bank28082 +28083 POINT(40.12635860305948 73.25229651046492) bank28083 +28084 POINT(40.15871192094541 73.39793991175905) bank28084 +28085 POINT(41.430188974411614 74.91032800932534) bank28085 +28086 POINT(41.48051777714632 73.98834431397782) bank28086 +28087 POINT(40.79230898612497 73.3065185933031) bank28087 +28088 POINT(40.06157576037047 73.86905446123784) bank28088 +28089 POINT(41.38263901416932 74.33957064467357) bank28089 +28090 POINT(39.85326057317514 74.7924728134171) bank28090 +28091 POINT(41.46578510605591 73.47204710234409) bank28091 +28092 POINT(41.57698694436325 73.04129863463682) bank28092 +28093 POINT(40.98198166693827 74.39880517680346) bank28093 +28094 POINT(41.00073528058329 74.50953618948824) bank28094 +28095 POINT(41.4149181936873 74.13982662501957) bank28095 +28096 POINT(41.0661298028695 74.2044835355782) bank28096 +28097 POINT(41.57727340339994 73.23217282876128) bank28097 +28098 POINT(41.17832104899129 73.06294084380757) bank28098 +28099 POINT(41.375377180597724 74.1505092957178) bank28099 +28100 POINT(39.836345368310525 73.90041016474115) bank28100 +28101 POINT(40.02595954325346 73.6797111229529) bank28101 +28102 POINT(40.16486491302213 74.47161390160453) bank28102 +28103 POINT(40.1937626009926 73.70093876877975) bank28103 +28104 POINT(41.49964178431185 74.95759745093729) bank28104 +28105 POINT(40.189754936256556 73.08341951085669) bank28105 +28106 POINT(40.37031747132601 74.68540376368705) bank28106 +28107 POINT(39.89209514480279 74.95066695534511) bank28107 +28108 POINT(40.48164641120537 74.10783335794355) bank28108 +28109 POINT(40.65760441707592 74.08547296174565) bank28109 +28110 POINT(40.14638818229305 74.51481170757759) bank28110 +28111 POINT(40.736831966566236 73.9829701535607) bank28111 +28112 POINT(40.0466547054282 74.00751017522586) bank28112 +28113 POINT(40.660654984658365 73.12003829496781) bank28113 +28114 POINT(41.21923055189752 73.42344210959942) bank28114 +28115 POINT(40.00845919269575 74.24349336773176) bank28115 +28116 POINT(40.43715950714701 73.96455752952313) bank28116 +28117 POINT(39.752805450873794 73.335994359448) bank28117 +28118 POINT(39.90697062559906 73.37313090226546) bank28118 +28119 POINT(40.00966919698806 73.61565150733591) bank28119 +28120 POINT(40.2886148778329 75.00492185103239) bank28120 +28121 POINT(40.63560382212925 74.14914809665545) bank28121 +28122 POINT(41.06976562426708 74.31704640559607) bank28122 +28123 POINT(41.63338389137132 73.67930960408633) bank28123 +28124 POINT(41.04148963515078 73.89822908458154) bank28124 +28125 POINT(39.777840773300944 73.16915565336198) bank28125 +28126 POINT(41.26019742201986 74.80726508829883) bank28126 +28127 POINT(41.03770814240795 74.80003369251388) bank28127 +28128 POINT(40.77867703373931 73.47332677493876) bank28128 +28129 POINT(40.49291236640642 73.41217511524206) bank28129 +28130 POINT(41.54907422135407 73.9781161564556) bank28130 +28131 POINT(40.692004375833505 74.18770231554103) bank28131 +28132 POINT(40.43522788396592 74.23178816438197) bank28132 +28133 POINT(40.89941480195603 74.49839666140271) bank28133 +28134 POINT(40.453948777237294 73.77015776656876) bank28134 +28135 POINT(40.19251909858303 74.35802143402971) bank28135 +28136 POINT(40.38729547363834 74.76297861642352) bank28136 +28137 POINT(41.08672296555875 74.71584338861986) bank28137 +28138 POINT(40.873246185191974 73.13738072997599) bank28138 +28139 POINT(39.98638888362203 74.97507602321409) bank28139 +28140 POINT(41.43810973949481 74.94437270280406) bank28140 +28141 POINT(41.671810271610475 74.58360606045547) bank28141 +28142 POINT(41.536231478042296 73.37877483048081) bank28142 +28143 POINT(41.319175787684316 74.25995874043352) bank28143 +28144 POINT(40.59248597739775 74.40540493438262) bank28144 +28145 POINT(41.1849223410918 74.4493910977936) bank28145 +28146 POINT(40.49370980180917 74.83438964443828) bank28146 +28147 POINT(40.062505022274706 74.19853042786919) bank28147 +28148 POINT(41.4685608933886 73.5378064254789) bank28148 +28149 POINT(40.81537832338923 74.1106396079629) bank28149 +28150 POINT(41.185535614873785 73.45497507061098) bank28150 +28151 POINT(41.25399608478109 73.95670163635454) bank28151 +28152 POINT(40.19342615105469 73.50950841940463) bank28152 +28153 POINT(40.37814861453171 74.55458888706019) bank28153 +28154 POINT(41.140187176595546 73.67617363700384) bank28154 +28155 POINT(40.53109498126816 73.08597127360285) bank28155 +28156 POINT(40.661637177899685 74.71043140056588) bank28156 +28157 POINT(40.507461829040324 73.84841446227446) bank28157 +28158 POINT(41.11788934868087 73.51559786293969) bank28158 +28159 POINT(39.9027606786259 73.30102647708422) bank28159 +28160 POINT(40.04719843751516 73.7112999248582) bank28160 +28161 POINT(40.026511794797656 73.53523497613338) bank28161 +28162 POINT(41.08008815068214 73.48027291475982) bank28162 +28163 POINT(40.06915316671841 74.53437760288152) bank28163 +28164 POINT(39.80343523256731 74.77665683886245) bank28164 +28165 POINT(40.32293272078445 74.87991201169359) bank28165 +28166 POINT(41.39454302192218 73.47327342640861) bank28166 +28167 POINT(40.16328016374413 73.39098605683483) bank28167 +28168 POINT(41.23918112372672 73.64047816680336) bank28168 +28169 POINT(41.226857171350225 74.02880594031576) bank28169 +28170 POINT(41.21557169118995 74.83361744653489) bank28170 +28171 POINT(41.19499642325172 74.61876082630904) bank28171 +28172 POINT(40.940061167507096 73.64232293182478) bank28172 +28173 POINT(41.34703816310094 73.16479942709347) bank28173 +28174 POINT(40.36405851212384 74.20949824547077) bank28174 +28175 POINT(40.00113905853552 74.12880456745732) bank28175 +28176 POINT(40.63192594333611 74.33492362918989) bank28176 +28177 POINT(41.57022257707004 74.84764700837249) bank28177 +28178 POINT(40.176209535109834 74.41919379329528) bank28178 +28179 POINT(41.11624376288713 74.21957109148184) bank28179 +28180 POINT(39.79038958657611 74.42977547746604) bank28180 +28181 POINT(39.801003987114214 73.31586445684215) bank28181 +28182 POINT(40.16869513183226 74.37776966014242) bank28182 +28183 POINT(41.28060750743633 74.78061392347226) bank28183 +28184 POINT(40.694886123906 73.49820221064137) bank28184 +28185 POINT(41.37558056721253 73.94290240569858) bank28185 +28186 POINT(41.04849000866157 74.87395553556888) bank28186 +28187 POINT(40.17687406011284 74.49317282218136) bank28187 +28188 POINT(40.795768789711175 73.67053420167622) bank28188 +28189 POINT(40.78572401156415 73.60949390448332) bank28189 +28190 POINT(40.27786072896334 73.44443322983462) bank28190 +28191 POINT(40.44534104908048 73.16701973402381) bank28191 +28192 POINT(40.51706160156995 74.50513850665656) bank28192 +28193 POINT(40.44909451906854 73.44765871690812) bank28193 +28194 POINT(41.618877415551125 74.62219187463484) bank28194 +28195 POINT(41.09348118472133 73.65737667973004) bank28195 +28196 POINT(40.006144539132194 74.55107422284091) bank28196 +28197 POINT(41.39917490165751 74.99839927150168) bank28197 +28198 POINT(41.39612545553834 74.63641876470781) bank28198 +28199 POINT(41.27588299171352 73.02316767943503) bank28199 +28200 POINT(41.659406732529895 74.94341993695849) bank28200 +28201 POINT(40.08313741424791 74.4593673802436) bank28201 +28202 POINT(40.368757499850894 73.69742083005632) bank28202 +28203 POINT(39.84048983350199 73.5790040517504) bank28203 +28204 POINT(39.96948110907263 74.05663321172072) bank28204 +28205 POINT(40.7577653164678 74.31850459037204) bank28205 +28206 POINT(40.84711642319533 74.75097456916602) bank28206 +28207 POINT(41.02908776649512 73.5140096377917) bank28207 +28208 POINT(40.0189469037669 74.42086135538258) bank28208 +28209 POINT(41.08416563656852 73.60844223804521) bank28209 +28210 POINT(40.78677284012834 73.40337975245372) bank28210 +28211 POINT(40.353536214936845 74.70103788508253) bank28211 +28212 POINT(40.31003034737454 74.22278873414633) bank28212 +28213 POINT(41.41817566786474 74.95901787840779) bank28213 +28214 POINT(41.64473472375958 75.00444566442903) bank28214 +28215 POINT(39.92889008812874 73.2578623107116) bank28215 +28216 POINT(40.21715330105715 74.77413102702599) bank28216 +28217 POINT(41.0267182220963 73.77735518043497) bank28217 +28218 POINT(41.13701032293819 74.20144753123317) bank28218 +28219 POINT(41.138425817435284 74.62607751361294) bank28219 +28220 POINT(41.507271022445465 74.93868581677974) bank28220 +28221 POINT(41.29828166280859 73.07784035724569) bank28221 +28222 POINT(40.6906471013916 74.67563514734941) bank28222 +28223 POINT(40.137460702546434 73.49294819771107) bank28223 +28224 POINT(41.44027781972813 73.17794411295843) bank28224 +28225 POINT(41.187562662880914 74.53486211517207) bank28225 +28226 POINT(41.46770385002096 74.99652870088043) bank28226 +28227 POINT(39.88255120618386 73.88472969001356) bank28227 +28228 POINT(40.84518115014962 73.03200089095503) bank28228 +28229 POINT(40.834347924746055 73.8060002182185) bank28229 +28230 POINT(40.50311879581955 73.62881537819237) bank28230 +28231 POINT(41.39352256715636 73.35826612879184) bank28231 +28232 POINT(39.9345131813806 74.92986373375923) bank28232 +28233 POINT(40.042729749896004 73.51622324111007) bank28233 +28234 POINT(39.89149997121655 74.2381971490472) bank28234 +28235 POINT(40.557325992395086 73.38913964515505) bank28235 +28236 POINT(41.09438823111204 73.41776547091531) bank28236 +28237 POINT(40.53718351753653 73.93148895684487) bank28237 +28238 POINT(41.20612372372185 74.7716641652681) bank28238 +28239 POINT(40.589429824878316 74.74050020123669) bank28239 +28240 POINT(40.85237967768053 73.97877625397362) bank28240 +28241 POINT(40.87149535313788 73.13059494924244) bank28241 +28242 POINT(41.67944555868324 74.9218560920716) bank28242 +28243 POINT(39.86845623613522 73.12508981377734) bank28243 +28244 POINT(41.13082968614634 74.9315801444038) bank28244 +28245 POINT(39.788314641849624 74.43837653056983) bank28245 +28246 POINT(41.684640543436316 73.27313577187755) bank28246 +28247 POINT(39.756107607411366 73.77322297918943) bank28247 +28248 POINT(41.40712355884841 73.80388700622797) bank28248 +28249 POINT(41.45717822579437 73.01505774214242) bank28249 +28250 POINT(41.39168009714921 74.04934979092668) bank28250 +28251 POINT(40.10927168432278 74.77040634555817) bank28251 +28252 POINT(40.290686723274 73.68647300720394) bank28252 +28253 POINT(41.17084510761796 74.37198528740055) bank28253 +28254 POINT(40.52937591909956 73.04817567935555) bank28254 +28255 POINT(40.740309443011554 73.11372721845981) bank28255 +28256 POINT(40.42175748528452 74.4577564361096) bank28256 +28257 POINT(41.24883017030125 74.20698223421252) bank28257 +28258 POINT(40.61709415873461 73.341628893376) bank28258 +28259 POINT(40.311951241197825 73.6256339640043) bank28259 +28260 POINT(39.80899470724571 73.4488565165155) bank28260 +28261 POINT(40.00010928494844 74.04432635243458) bank28261 +28262 POINT(41.69467617418004 73.90234792696361) bank28262 +28263 POINT(40.563882938834944 73.82952890053078) bank28263 +28264 POINT(41.296636681834855 74.63058701635387) bank28264 +28265 POINT(40.38344421719153 74.28159282039235) bank28265 +28266 POINT(41.52487476779607 74.30467615307587) bank28266 +28267 POINT(41.567370295847184 74.95110079628193) bank28267 +28268 POINT(40.441757425958606 73.79661473147526) bank28268 +28269 POINT(41.08840875619913 73.24760174647531) bank28269 +28270 POINT(40.56352211794356 73.43353672293917) bank28270 +28271 POINT(40.53543267696353 74.33242965735978) bank28271 +28272 POINT(40.38890030768917 73.10060232862291) bank28272 +28273 POINT(40.95631754581944 73.85717732143848) bank28273 +28274 POINT(41.01703607632925 73.23485308979753) bank28274 +28275 POINT(41.2391931271384 74.40127510650359) bank28275 +28276 POINT(41.30335028713211 74.82963509406744) bank28276 +28277 POINT(41.675891670088 73.29092798528487) bank28277 +28278 POINT(40.35067430708015 73.95756526314689) bank28278 +28279 POINT(41.01794337124201 73.43312819476607) bank28279 +28280 POINT(40.60004812381131 74.8659206331482) bank28280 +28281 POINT(40.031190527885656 74.14103979258353) bank28281 +28282 POINT(39.868010676632515 74.58113128869101) bank28282 +28283 POINT(41.40927563050962 74.25711336012247) bank28283 +28284 POINT(40.985527263390786 74.97964677824534) bank28284 +28285 POINT(40.01180941529863 73.3641042695167) bank28285 +28286 POINT(41.24372899294855 73.16346083215647) bank28286 +28287 POINT(41.191015106684816 74.15360923427113) bank28287 +28288 POINT(40.83526571659402 74.78283602823818) bank28288 +28289 POINT(41.40440833401608 74.53597497223221) bank28289 +28290 POINT(41.30550301892784 73.27967963296301) bank28290 +28291 POINT(39.917800507605726 74.66917957570797) bank28291 +28292 POINT(41.53581453932655 73.88482667883159) bank28292 +28293 POINT(40.20795927417447 74.51667948764307) bank28293 +28294 POINT(40.5895657356838 73.35877445729571) bank28294 +28295 POINT(40.142908638919685 73.19750058019952) bank28295 +28296 POINT(40.64058675721014 73.77943999722272) bank28296 +28297 POINT(40.32602591517358 73.87371843546185) bank28297 +28298 POINT(40.32023516137559 74.05476828085608) bank28298 +28299 POINT(41.01883140488304 74.2148036881284) bank28299 +28300 POINT(40.8599099086878 74.45535374791893) bank28300 +28301 POINT(40.20863426167445 74.47928585235576) bank28301 +28302 POINT(40.91361271104235 73.8762975751229) bank28302 +28303 POINT(39.89997977035435 73.6944470307334) bank28303 +28304 POINT(39.835711295215226 73.6124860708754) bank28304 +28305 POINT(39.87677527118188 74.86303314725934) bank28305 +28306 POINT(40.23404490614924 74.51495763020442) bank28306 +28307 POINT(41.045578544871745 73.46297611210778) bank28307 +28308 POINT(40.88517390318345 74.95093186724199) bank28308 +28309 POINT(40.73643763210989 73.70653408242421) bank28309 +28310 POINT(39.84975778414506 74.03960324739215) bank28310 +28311 POINT(40.68389003361843 74.28593821540426) bank28311 +28312 POINT(40.10898735835997 74.4718977867264) bank28312 +28313 POINT(40.48537507092878 73.49218800544116) bank28313 +28314 POINT(40.861567443701375 73.48545456688714) bank28314 +28315 POINT(40.58492902609291 74.93725537103958) bank28315 +28316 POINT(39.829612599428266 74.01823572344824) bank28316 +28317 POINT(41.58959557111222 74.48840080302406) bank28317 +28318 POINT(40.022950803222706 74.76200474745954) bank28318 +28319 POINT(39.97982452385973 74.0031289934162) bank28319 +28320 POINT(40.06256602132978 73.41401566151801) bank28320 +28321 POINT(39.75519672266463 74.91794652903513) bank28321 +28322 POINT(41.292587369519154 74.75687769464926) bank28322 +28323 POINT(41.63047994396034 74.79572903340029) bank28323 +28324 POINT(40.845211037565 74.80821566703925) bank28324 +28325 POINT(40.18029844434081 74.91391522034726) bank28325 +28326 POINT(40.7281356361756 74.25239249917519) bank28326 +28327 POINT(41.12358790533441 74.42137976997479) bank28327 +28328 POINT(39.759869561902946 74.32166155150327) bank28328 +28329 POINT(40.425236106890814 73.73712740615797) bank28329 +28330 POINT(41.049616904238306 73.75609129983731) bank28330 +28331 POINT(40.190519653234034 74.19102951541673) bank28331 +28332 POINT(40.89557601372119 74.7165359815657) bank28332 +28333 POINT(39.77849764880317 73.54491983208452) bank28333 +28334 POINT(41.075458424001916 74.60010583095169) bank28334 +28335 POINT(41.09669163294728 74.18282615725039) bank28335 +28336 POINT(40.36419056280138 74.0360115603578) bank28336 +28337 POINT(41.147036752092866 73.49150405086058) bank28337 +28338 POINT(40.37986999038295 74.18561019596379) bank28338 +28339 POINT(40.422031790078705 73.43010227075165) bank28339 +28340 POINT(41.142550052053956 73.53814752497935) bank28340 +28341 POINT(41.518562579152395 73.04546125722966) bank28341 +28342 POINT(40.15025186496351 74.46580046423726) bank28342 +28343 POINT(40.410159758150165 74.0090145940726) bank28343 +28344 POINT(40.388008167616576 73.46990950313482) bank28344 +28345 POINT(41.567825633751475 74.65516164187609) bank28345 +28346 POINT(40.534561219508674 74.89883391023862) bank28346 +28347 POINT(41.270500964698364 74.7113015290131) bank28347 +28348 POINT(40.10347955188832 73.51414237344424) bank28348 +28349 POINT(40.89399144427167 74.4608300558948) bank28349 +28350 POINT(40.736645835187055 73.2941944707855) bank28350 +28351 POINT(39.77539577476934 73.56446398066423) bank28351 +28352 POINT(40.42392522580291 73.79008037580017) bank28352 +28353 POINT(40.421370111705485 73.77822579183416) bank28353 +28354 POINT(41.26357832846176 73.6871101944838) bank28354 +28355 POINT(40.75581584048509 74.62990002398871) bank28355 +28356 POINT(40.221480668921195 74.75826339249686) bank28356 +28357 POINT(39.93646754735726 74.50532956631177) bank28357 +28358 POINT(41.27221767228442 73.83924341398414) bank28358 +28359 POINT(40.7400091914717 73.53929192749256) bank28359 +28360 POINT(41.2986678743289 73.44149022714625) bank28360 +28361 POINT(41.56794831984403 73.37998311787226) bank28361 +28362 POINT(40.43002454978312 73.18625169602201) bank28362 +28363 POINT(39.93257486863646 74.54729299750142) bank28363 +28364 POINT(41.64050248453371 74.55599470719002) bank28364 +28365 POINT(40.921346949223526 74.9681805448314) bank28365 +28366 POINT(41.07139920176761 73.62394963818878) bank28366 +28367 POINT(40.220971528646345 73.8957697963236) bank28367 +28368 POINT(40.369785770532474 74.08663999406245) bank28368 +28369 POINT(40.694048361748415 74.25418310638089) bank28369 +28370 POINT(40.78130820322601 74.65431448634867) bank28370 +28371 POINT(39.91787295309434 73.6537530817811) bank28371 +28372 POINT(41.627330885341124 74.5033111029002) bank28372 +28373 POINT(40.51802670167627 74.9073017527507) bank28373 +28374 POINT(41.179187833996885 73.71569776377316) bank28374 +28375 POINT(40.775455813365475 73.44504916235452) bank28375 +28376 POINT(40.12839107067749 74.64609055118441) bank28376 +28377 POINT(41.607834485644396 73.69839924004046) bank28377 +28378 POINT(40.60734353443277 74.4310145271793) bank28378 +28379 POINT(40.71181309569689 74.8271243646842) bank28379 +28380 POINT(41.68704109190112 74.31553034845685) bank28380 +28381 POINT(40.77544968062261 74.94028567385001) bank28381 +28382 POINT(41.54926937025974 73.91132842563265) bank28382 +28383 POINT(40.41753897642571 73.59623442881589) bank28383 +28384 POINT(41.08869173834513 73.31519211843506) bank28384 +28385 POINT(41.46635990373534 74.0089925276911) bank28385 +28386 POINT(41.254750604173545 73.5775518586181) bank28386 +28387 POINT(39.98110233160921 73.71427499222305) bank28387 +28388 POINT(41.57715493303568 73.75933184091255) bank28388 +28389 POINT(39.75191683536956 73.58437897296594) bank28389 +28390 POINT(41.4774966208662 75.00036755024198) bank28390 +28391 POINT(41.367714552791554 73.77941341705878) bank28391 +28392 POINT(39.73259859560842 74.69003442506072) bank28392 +28393 POINT(40.88098682141006 74.91518299143314) bank28393 +28394 POINT(39.73671440917484 74.99008388773247) bank28394 +28395 POINT(41.49341918598656 73.42721970172087) bank28395 +28396 POINT(41.613889126527354 74.83513430276359) bank28396 +28397 POINT(39.993915857571814 73.75237372579122) bank28397 +28398 POINT(40.583515885953 73.83954766180894) bank28398 +28399 POINT(39.95211348560238 73.64794426518158) bank28399 +28400 POINT(40.085055926178136 74.77377455074256) bank28400 +28401 POINT(40.39497854005332 74.4112517066355) bank28401 +28402 POINT(39.996636130511156 73.91755411402396) bank28402 +28403 POINT(40.09901582237998 73.6010003459813) bank28403 +28404 POINT(40.28479297823237 74.37355062203042) bank28404 +28405 POINT(40.8433713227816 73.63197367630549) bank28405 +28406 POINT(40.66212027229327 74.11188980839432) bank28406 +28407 POINT(40.32358949824495 73.9318111022453) bank28407 +28408 POINT(40.50070283493443 73.8293741116898) bank28408 +28409 POINT(41.22347914797245 73.51846381945768) bank28409 +28410 POINT(39.990172733177026 74.6110904957951) bank28410 +28411 POINT(40.09171111124562 73.8371814597043) bank28411 +28412 POINT(41.260675587092436 73.56879771999508) bank28412 +28413 POINT(41.26848710154252 73.69162394730849) bank28413 +28414 POINT(40.568806259565505 73.80172095681263) bank28414 +28415 POINT(39.8986797498242 73.9410969937949) bank28415 +28416 POINT(41.47473145763172 73.46087720082923) bank28416 +28417 POINT(40.38310770307116 74.25478955292752) bank28417 +28418 POINT(40.069730119208046 73.51237607382417) bank28418 +28419 POINT(40.5566279178932 74.76512433026147) bank28419 +28420 POINT(41.411016265909694 74.14643831804563) bank28420 +28421 POINT(39.81921443196927 74.90053801248031) bank28421 +28422 POINT(41.497139371341774 74.74779378833655) bank28422 +28423 POINT(40.05521750945091 74.13568502614228) bank28423 +28424 POINT(40.98365923626358 74.37120407027997) bank28424 +28425 POINT(41.57040359084711 74.52446565928938) bank28425 +28426 POINT(39.7276015031397 73.91374275722887) bank28426 +28427 POINT(40.71041095605462 74.41661846778337) bank28427 +28428 POINT(41.36309764448371 74.36466523293163) bank28428 +28429 POINT(41.493247907784934 73.10141392984264) bank28429 +28430 POINT(40.018525677278575 74.93701194112154) bank28430 +28431 POINT(41.29639683363456 74.43462485464146) bank28431 +28432 POINT(40.80362530373528 73.489399323863) bank28432 +28433 POINT(40.15341329084591 74.60521477310239) bank28433 +28434 POINT(39.97275055757006 73.32747207414043) bank28434 +28435 POINT(40.73220629306222 74.40234089491597) bank28435 +28436 POINT(41.66158116402025 74.258825646389) bank28436 +28437 POINT(40.217550775327524 74.20946960638001) bank28437 +28438 POINT(39.82333775384801 73.94469164058495) bank28438 +28439 POINT(40.640536297631726 73.60308429079171) bank28439 +28440 POINT(39.782751591785214 74.55619452453703) bank28440 +28441 POINT(41.160488776022504 74.53136135948199) bank28441 +28442 POINT(40.72197287942387 74.07935978243475) bank28442 +28443 POINT(40.98441307486272 73.68880900906991) bank28443 +28444 POINT(41.476743819422694 74.8266009766347) bank28444 +28445 POINT(40.020930607424624 73.78869153788224) bank28445 +28446 POINT(41.56468120992554 73.6200381505634) bank28446 +28447 POINT(41.51671417478312 74.20462857475607) bank28447 +28448 POINT(39.79652225970295 74.06616467085595) bank28448 +28449 POINT(41.556203577355426 73.72526055506017) bank28449 +28450 POINT(41.112891906260195 74.25395217340242) bank28450 +28451 POINT(40.14445748426542 73.93296308177602) bank28451 +28452 POINT(41.289781343212155 73.1181411177382) bank28452 +28453 POINT(41.00292089927835 73.27629861437117) bank28453 +28454 POINT(40.77265476661931 73.38688895910244) bank28454 +28455 POINT(41.27939763609367 73.53930398470267) bank28455 +28456 POINT(41.27092201426059 73.86707375578975) bank28456 +28457 POINT(40.30869380460802 73.60778539209987) bank28457 +28458 POINT(41.46675801774759 74.84402641247839) bank28458 +28459 POINT(41.57851591074982 73.42616941082149) bank28459 +28460 POINT(40.56412103672468 74.79415320219181) bank28460 +28461 POINT(39.773997758322196 73.3791288806184) bank28461 +28462 POINT(41.51252578634143 74.40147652319457) bank28462 +28463 POINT(40.39438009344811 74.40787679158763) bank28463 +28464 POINT(39.772415737881005 73.46125886379673) bank28464 +28465 POINT(41.26355983610063 74.69915846928987) bank28465 +28466 POINT(40.22412931117757 73.5137621479398) bank28466 +28467 POINT(39.80778166878308 74.72946367855673) bank28467 +28468 POINT(40.323041954801546 73.08237016341654) bank28468 +28469 POINT(40.05198049051254 74.43305483068248) bank28469 +28470 POINT(40.58275499693727 74.28589043168421) bank28470 +28471 POINT(41.55257615108033 74.48150551418176) bank28471 +28472 POINT(39.792099125420485 73.10617418417428) bank28472 +28473 POINT(40.058412090764016 74.78611046808618) bank28473 +28474 POINT(40.061235860623896 73.2969190392768) bank28474 +28475 POINT(40.45072630340052 73.2065230415633) bank28475 +28476 POINT(39.73140955140772 73.49862159833727) bank28476 +28477 POINT(40.78252087094016 73.04275894513641) bank28477 +28478 POINT(41.10317619443994 73.0341539481306) bank28478 +28479 POINT(40.577677746065994 73.13461352228245) bank28479 +28480 POINT(41.17851936493646 73.47727193131347) bank28480 +28481 POINT(40.42461948634011 73.0783921817947) bank28481 +28482 POINT(40.78234670207566 74.90610192009386) bank28482 +28483 POINT(40.90002008521942 74.51087059229933) bank28483 +28484 POINT(39.79092575469221 74.37286725954388) bank28484 +28485 POINT(40.688161018647286 73.80517621372582) bank28485 +28486 POINT(39.754762878976244 74.90661251687769) bank28486 +28487 POINT(40.041405501944475 73.94752360615789) bank28487 +28488 POINT(40.627454311171675 73.7294858527002) bank28488 +28489 POINT(41.12280108635272 73.08683504561141) bank28489 +28490 POINT(41.28916473595251 74.71978289006306) bank28490 +28491 POINT(40.73409179239278 73.49880254218328) bank28491 +28492 POINT(40.00181741709858 73.42236398179342) bank28492 +28493 POINT(40.8600008683823 73.97695446632996) bank28493 +28494 POINT(40.17151986687122 73.69025432881749) bank28494 +28495 POINT(40.75415246850986 74.71544226091154) bank28495 +28496 POINT(40.553020545072386 74.33480349174975) bank28496 +28497 POINT(40.272624108518265 73.25470089180322) bank28497 +28498 POINT(41.35160037333799 74.83770493334123) bank28498 +28499 POINT(41.32743019213609 74.03899984632525) bank28499 +28500 POINT(41.178763334298274 73.19123708558337) bank28500 +28501 POINT(39.775203845657686 73.39257108427068) bank28501 +28502 POINT(41.69548014926393 73.32691983040293) bank28502 +28503 POINT(40.017092398990194 73.34240672559473) bank28503 +28504 POINT(40.34877539441402 74.1660194593157) bank28504 +28505 POINT(39.95169799354785 74.61475455745902) bank28505 +28506 POINT(39.93421589920307 73.79993152423837) bank28506 +28507 POINT(41.44692666179092 74.33637146061666) bank28507 +28508 POINT(41.14975401217374 74.40176916699586) bank28508 +28509 POINT(40.51510630547706 73.26930065542939) bank28509 +28510 POINT(41.453313351168326 73.64917511381987) bank28510 +28511 POINT(41.146156299633795 73.55664674588961) bank28511 +28512 POINT(40.05115267961565 74.38828384464608) bank28512 +28513 POINT(41.126731439572836 74.14852977755051) bank28513 +28514 POINT(41.22653752466173 74.29268478843764) bank28514 +28515 POINT(39.79800365549719 74.18605867609651) bank28515 +28516 POINT(40.406671773446405 73.48131367699634) bank28516 +28517 POINT(39.87997476142994 73.43083780672183) bank28517 +28518 POINT(40.36873743501859 73.8938699290759) bank28518 +28519 POINT(41.413220372509585 74.48989609147326) bank28519 +28520 POINT(39.94128209668288 73.0908607821991) bank28520 +28521 POINT(40.16863656861596 74.35682616495025) bank28521 +28522 POINT(39.94887194138113 73.46025160559114) bank28522 +28523 POINT(40.2842260750911 74.21005891869194) bank28523 +28524 POINT(40.760960763023306 74.14480795415646) bank28524 +28525 POINT(40.72496934839094 73.6454996290549) bank28525 +28526 POINT(40.69520780882827 73.0343451532877) bank28526 +28527 POINT(39.98231015212859 73.72765391954435) bank28527 +28528 POINT(40.7581429027497 74.16586715901173) bank28528 +28529 POINT(41.469745129511494 74.61375795881666) bank28529 +28530 POINT(41.55651507680703 74.20926230653326) bank28530 +28531 POINT(40.22587859683834 74.09771406597805) bank28531 +28532 POINT(39.79270883413925 73.90999328861335) bank28532 +28533 POINT(40.78755210102048 74.65735932508724) bank28533 +28534 POINT(41.49563716917901 73.58326122388245) bank28534 +28535 POINT(41.56868809122897 73.34466020005054) bank28535 +28536 POINT(40.419500799987276 73.5924052341073) bank28536 +28537 POINT(41.515244171790386 75.00346481815355) bank28537 +28538 POINT(40.38561525219681 73.09604457598599) bank28538 +28539 POINT(40.26478503740859 74.42798428173681) bank28539 +28540 POINT(40.614673484711226 74.27371219040904) bank28540 +28541 POINT(39.90435989863999 73.15683540564525) bank28541 +28542 POINT(41.0820607131197 74.07021837582747) bank28542 +28543 POINT(40.871734903014044 73.42606272047507) bank28543 +28544 POINT(40.893101798292925 73.01469680875795) bank28544 +28545 POINT(39.78332406976756 74.27356362370388) bank28545 +28546 POINT(40.69753289642189 74.74615600598295) bank28546 +28547 POINT(41.600309385307355 73.81230357074874) bank28547 +28548 POINT(40.99050078795181 73.41165536674598) bank28548 +28549 POINT(41.664636090828566 74.01476296822132) bank28549 +28550 POINT(41.48175319794429 73.97721155021723) bank28550 +28551 POINT(40.331227478904864 73.42488269841503) bank28551 +28552 POINT(40.700103250996406 73.06288002190631) bank28552 +28553 POINT(39.76427378328775 74.26176906501603) bank28553 +28554 POINT(40.4782515377923 74.16194402111444) bank28554 +28555 POINT(40.98400297406417 74.65731332729744) bank28555 +28556 POINT(41.02715532392741 74.78361713766633) bank28556 +28557 POINT(41.68170916918942 74.57873711151603) bank28557 +28558 POINT(40.881821989865685 73.41314318081831) bank28558 +28559 POINT(40.67696970931769 73.6283066101089) bank28559 +28560 POINT(41.70429874090267 74.51717793289623) bank28560 +28561 POINT(41.5472289756587 74.51192740710971) bank28561 +28562 POINT(39.7709147146329 73.64440363806399) bank28562 +28563 POINT(41.54409177200583 73.16580545235396) bank28563 +28564 POINT(41.37714849392847 74.9306044051807) bank28564 +28565 POINT(40.97743953368815 73.06825020345028) bank28565 +28566 POINT(40.94128630961571 74.25201733822354) bank28566 +28567 POINT(40.52413192445917 74.11811511180125) bank28567 +28568 POINT(40.68745332450562 74.20855951621546) bank28568 +28569 POINT(40.838784297685194 73.75365350205213) bank28569 +28570 POINT(41.571678978209235 74.76757271052789) bank28570 +28571 POINT(40.13717193983691 73.51108971416653) bank28571 +28572 POINT(41.65080531569361 74.410379048452) bank28572 +28573 POINT(41.42631924526097 73.43446148363458) bank28573 +28574 POINT(41.65328596676823 74.47733288855314) bank28574 +28575 POINT(40.86233845865984 73.04977314441034) bank28575 +28576 POINT(40.20640035022569 74.18630025127179) bank28576 +28577 POINT(40.5978608812132 73.1284711058374) bank28577 +28578 POINT(39.9797055761927 74.50992080694392) bank28578 +28579 POINT(40.152428819303154 74.16296805341642) bank28579 +28580 POINT(40.5960293070784 73.50456230502823) bank28580 +28581 POINT(40.04732533785687 74.02450396938433) bank28581 +28582 POINT(41.65710403508664 73.6908698693629) bank28582 +28583 POINT(41.43094648312004 74.40747720665576) bank28583 +28584 POINT(40.24488896960516 74.59777062358148) bank28584 +28585 POINT(41.45323518599062 74.86710879247508) bank28585 +28586 POINT(40.70302733606052 74.84270337319383) bank28586 +28587 POINT(41.536978615990876 74.32796670621643) bank28587 +28588 POINT(41.50403031339271 73.4188025803958) bank28588 +28589 POINT(41.334685351705815 74.84647021982634) bank28589 +28590 POINT(41.55157270982885 73.07034829459849) bank28590 +28591 POINT(40.032777709224796 73.61027790297013) bank28591 +28592 POINT(40.15974162923047 73.8217590441142) bank28592 +28593 POINT(40.13242576927065 73.4962031746575) bank28593 +28594 POINT(41.19806414135888 74.94700267510606) bank28594 +28595 POINT(40.98546082975199 73.9684328378742) bank28595 +28596 POINT(40.23177350951856 74.10523532388967) bank28596 +28597 POINT(41.49893642238018 73.28192662607619) bank28597 +28598 POINT(39.961809111855516 74.88257487320605) bank28598 +28599 POINT(40.51956310877345 74.07158419815582) bank28599 +28600 POINT(40.407574726387 74.65021549630747) bank28600 +28601 POINT(39.91694003992069 73.09406891297759) bank28601 +28602 POINT(40.946297194130466 74.87751984787616) bank28602 +28603 POINT(41.269836386812194 73.43889361691318) bank28603 +28604 POINT(39.713526684122655 74.57396685812145) bank28604 +28605 POINT(41.55609122771972 74.3959039031956) bank28605 +28606 POINT(41.389511640216035 74.2526135512413) bank28606 +28607 POINT(41.709000807679956 74.4438300426174) bank28607 +28608 POINT(40.0485510475936 73.52200292028975) bank28608 +28609 POINT(41.51011110976511 74.99247858346378) bank28609 +28610 POINT(40.56814198664926 73.201587255929) bank28610 +28611 POINT(40.024763773202 74.60737688397006) bank28611 +28612 POINT(39.84641012340778 73.28271208650048) bank28612 +28613 POINT(40.031106513532976 73.54167038886806) bank28613 +28614 POINT(40.74213021302195 73.35198296079442) bank28614 +28615 POINT(41.17402266581457 74.57178356348261) bank28615 +28616 POINT(40.988613061371545 73.44315812234889) bank28616 +28617 POINT(40.44410666182058 74.64018160142003) bank28617 +28618 POINT(40.30599501266809 74.48398522425045) bank28618 +28619 POINT(39.8061257952223 73.94071613278047) bank28619 +28620 POINT(40.710616176392776 74.36326672058938) bank28620 +28621 POINT(39.82739769890217 73.23531103113719) bank28621 +28622 POINT(40.57987046515868 74.57128149049333) bank28622 +28623 POINT(41.51758459821653 73.82109428850205) bank28623 +28624 POINT(41.276108918594495 73.21276814438905) bank28624 +28625 POINT(40.927551265497954 73.375308571163) bank28625 +28626 POINT(40.79809264911414 74.0509390810078) bank28626 +28627 POINT(41.3269707595156 74.24578624979955) bank28627 +28628 POINT(41.5904819883843 73.21178838860719) bank28628 +28629 POINT(39.83930655324227 73.69359524933499) bank28629 +28630 POINT(40.26613328397146 74.52030460185891) bank28630 +28631 POINT(40.347089755074954 73.84250413152829) bank28631 +28632 POINT(40.555727939971256 74.64552825599571) bank28632 +28633 POINT(41.04328051189031 73.15262843875598) bank28633 +28634 POINT(40.888257334026605 73.73777985825568) bank28634 +28635 POINT(40.99135665164737 74.62965040741085) bank28635 +28636 POINT(40.90697884428417 73.8698620198453) bank28636 +28637 POINT(40.025499126414715 73.54064146084006) bank28637 +28638 POINT(41.30627098555011 73.61454632021929) bank28638 +28639 POINT(41.279110984394556 73.04605714094362) bank28639 +28640 POINT(40.67017234169736 74.67775767042869) bank28640 +28641 POINT(39.81601936760906 74.77528366202044) bank28641 +28642 POINT(39.7452786292279 73.52634306320157) bank28642 +28643 POINT(41.199413317461456 73.04053645237907) bank28643 +28644 POINT(40.61894771912297 74.28536635785247) bank28644 +28645 POINT(40.69768330961951 73.58530784538995) bank28645 +28646 POINT(40.60818269229255 73.24960167876264) bank28646 +28647 POINT(41.12782942217849 73.26071864719444) bank28647 +28648 POINT(40.37548739396519 73.60320891477947) bank28648 +28649 POINT(41.68476658843891 73.09733923365367) bank28649 +28650 POINT(40.097535495404394 73.13349799379283) bank28650 +28651 POINT(40.38184987430825 73.37019710536951) bank28651 +28652 POINT(41.22534204225219 74.20072719080976) bank28652 +28653 POINT(41.60576275039138 74.48799450466073) bank28653 +28654 POINT(41.277300773668635 74.79205309886991) bank28654 +28655 POINT(41.29481653782921 73.03005691731792) bank28655 +28656 POINT(41.37951249311116 73.56268765223818) bank28656 +28657 POINT(40.61726372036664 73.2670268607803) bank28657 +28658 POINT(40.675451083031064 73.12350817440196) bank28658 +28659 POINT(41.330455217279315 73.40380448656875) bank28659 +28660 POINT(40.84792013238792 74.6076750860349) bank28660 +28661 POINT(41.661926496671384 74.75602740901537) bank28661 +28662 POINT(41.654534874272045 74.61893371391118) bank28662 +28663 POINT(40.27190932134753 74.87984200475535) bank28663 +28664 POINT(40.99173732578455 73.86539650139787) bank28664 +28665 POINT(41.29650247029662 74.73893964827157) bank28665 +28666 POINT(41.252175881476056 74.94474871171798) bank28666 +28667 POINT(39.89713417141527 73.01468569033686) bank28667 +28668 POINT(41.07401195813551 73.5911633722006) bank28668 +28669 POINT(40.21390864103009 74.46190205858333) bank28669 +28670 POINT(41.57264146515388 74.25758706877575) bank28670 +28671 POINT(40.44157354789492 73.81675424509505) bank28671 +28672 POINT(40.50709138614927 73.63798385803271) bank28672 +28673 POINT(39.8469190031113 73.22386561510375) bank28673 +28674 POINT(41.563234076576215 74.5301074419239) bank28674 +28675 POINT(41.2200832825078 73.25830203788584) bank28675 +28676 POINT(40.582281906395544 74.11577627350775) bank28676 +28677 POINT(39.98370316949242 74.48862759945798) bank28677 +28678 POINT(40.169273888681204 74.73755299257894) bank28678 +28679 POINT(39.89288300769781 73.26111688604345) bank28679 +28680 POINT(40.55230901253268 73.6073673715237) bank28680 +28681 POINT(40.41854946550061 74.43339707313247) bank28681 +28682 POINT(41.620767365028755 74.54831976104798) bank28682 +28683 POINT(40.69615190002994 74.8457799981556) bank28683 +28684 POINT(40.09679964628403 73.16561346629052) bank28684 +28685 POINT(41.25737955962429 73.59884518014634) bank28685 +28686 POINT(40.05715621545761 73.85712386131195) bank28686 +28687 POINT(40.712454808238014 74.89212570389432) bank28687 +28688 POINT(41.51780525832254 74.49454150532395) bank28688 +28689 POINT(39.84365062375325 74.64006324105829) bank28689 +28690 POINT(39.71303820853708 74.5698952107532) bank28690 +28691 POINT(40.19409403991396 73.05844659608213) bank28691 +28692 POINT(40.96130591009024 74.93156035529267) bank28692 +28693 POINT(39.86427711957038 73.03394519346415) bank28693 +28694 POINT(41.618848510866975 74.18975220155967) bank28694 +28695 POINT(40.565438293500975 73.89809971377633) bank28695 +28696 POINT(40.09908195491635 73.64566325943959) bank28696 +28697 POINT(41.5483671794732 74.89058940320226) bank28697 +28698 POINT(39.894623225256076 74.38115686115471) bank28698 +28699 POINT(40.67856108092561 73.50062193902002) bank28699 +28700 POINT(39.98807596531034 73.84249922866444) bank28700 +28701 POINT(40.614284003365015 73.09795865246082) bank28701 +28702 POINT(39.93804735213321 73.80414432390454) bank28702 +28703 POINT(40.26428812583355 74.67036254774544) bank28703 +28704 POINT(41.40768429324877 74.73111688341439) bank28704 +28705 POINT(40.37169615371735 73.4006035558499) bank28705 +28706 POINT(40.99994471549301 74.31582671540039) bank28706 +28707 POINT(40.93944606395789 73.05067696185648) bank28707 +28708 POINT(41.401605597741806 74.99160083161794) bank28708 +28709 POINT(41.60673233954394 74.96702699157208) bank28709 +28710 POINT(41.54787064365231 73.87674394553618) bank28710 +28711 POINT(40.56757032150478 73.1796247977937) bank28711 +28712 POINT(39.835427185181565 74.6241511109918) bank28712 +28713 POINT(41.29735967620341 73.78922855415172) bank28713 +28714 POINT(41.47093425136074 73.79220654120084) bank28714 +28715 POINT(40.800098767847835 73.80271802820633) bank28715 +28716 POINT(40.11842398761242 73.81294976993611) bank28716 +28717 POINT(40.53962007445004 73.5720432473717) bank28717 +28718 POINT(39.722596089810864 73.70911864157225) bank28718 +28719 POINT(39.849337181560045 73.96193340344878) bank28719 +28720 POINT(40.21623219764123 73.79075308090803) bank28720 +28721 POINT(40.49465901015717 75.00039092144823) bank28721 +28722 POINT(40.896599273734296 74.69140336895754) bank28722 +28723 POINT(39.84293014253758 73.15196332707235) bank28723 +28724 POINT(40.04606365849281 73.58768156672588) bank28724 +28725 POINT(41.2171515914443 74.92312095314446) bank28725 +28726 POINT(39.72808393049708 73.94926506549753) bank28726 +28727 POINT(40.53952986569946 74.49759751888028) bank28727 +28728 POINT(40.697912475416416 73.9595679877898) bank28728 +28729 POINT(40.533385781087425 73.58906920330344) bank28729 +28730 POINT(41.39112536113612 74.5379407320809) bank28730 +28731 POINT(41.690595455020926 73.00688714126599) bank28731 +28732 POINT(41.08417346529534 74.07098074717634) bank28732 +28733 POINT(40.96028670706869 74.94564443789308) bank28733 +28734 POINT(41.11434076813102 74.6793278947122) bank28734 +28735 POINT(40.30711389584155 74.64929337470757) bank28735 +28736 POINT(40.622471269412245 74.51852915025742) bank28736 +28737 POINT(40.55068502587809 73.8948833436962) bank28737 +28738 POINT(39.998591585940474 73.38585866113041) bank28738 +28739 POINT(41.28887911136453 74.00529100440792) bank28739 +28740 POINT(40.78253218642289 73.64425117199829) bank28740 +28741 POINT(40.674379553185354 73.90945262010132) bank28741 +28742 POINT(40.96552097427747 74.43623640411369) bank28742 +28743 POINT(40.0835840706455 74.74056008194886) bank28743 +28744 POINT(40.07088720900382 74.15056185517061) bank28744 +28745 POINT(40.44083828109062 74.27010569747156) bank28745 +28746 POINT(40.46882062034802 73.74739192644162) bank28746 +28747 POINT(41.246692377873025 74.77793485143918) bank28747 +28748 POINT(39.92258669682699 74.89401636899223) bank28748 +28749 POINT(40.057114432961534 73.99714680800848) bank28749 +28750 POINT(40.853325644903805 73.09439710485718) bank28750 +28751 POINT(40.54265643397115 74.82032611981953) bank28751 +28752 POINT(40.65660678338891 73.45823891923544) bank28752 +28753 POINT(40.30939148726505 74.40820462991665) bank28753 +28754 POINT(40.33221509164994 73.43646749907336) bank28754 +28755 POINT(41.55766017250552 73.2576096944619) bank28755 +28756 POINT(39.975778620879865 74.70305847359418) bank28756 +28757 POINT(41.608179382787284 73.5443140402034) bank28757 +28758 POINT(41.42446092304379 74.79174329977587) bank28758 +28759 POINT(41.70313663742186 74.75958730073538) bank28759 +28760 POINT(40.53641963937368 74.37200021998974) bank28760 +28761 POINT(41.24276072316527 74.90567398281809) bank28761 +28762 POINT(40.28874291512159 73.75625120862671) bank28762 +28763 POINT(40.61412762927827 74.5792738589459) bank28763 +28764 POINT(40.38542892294411 73.81720564732339) bank28764 +28765 POINT(40.06637123916267 74.43256610518152) bank28765 +28766 POINT(41.435079409333056 74.46282050577777) bank28766 +28767 POINT(40.54944155081189 73.64910802913747) bank28767 +28768 POINT(40.586563151855735 73.48824489586069) bank28768 +28769 POINT(39.9369169550172 74.581079128053) bank28769 +28770 POINT(40.63651406190762 73.09992063684916) bank28770 +28771 POINT(41.46344777637306 74.34116892375887) bank28771 +28772 POINT(40.727591059995135 74.41123036491314) bank28772 +28773 POINT(40.63366698103814 73.24538597224543) bank28773 +28774 POINT(40.86709908596936 74.36143510736837) bank28774 +28775 POINT(40.11775926388634 74.2237753273821) bank28775 +28776 POINT(41.350415440878095 74.76081647074372) bank28776 +28777 POINT(40.48923237792985 74.39743255679315) bank28777 +28778 POINT(41.41111276788718 73.03733349815055) bank28778 +28779 POINT(41.26470914459667 74.76034238885129) bank28779 +28780 POINT(41.279693225862836 73.78248165416058) bank28780 +28781 POINT(40.391903994187885 74.03719943229996) bank28781 +28782 POINT(40.21388903698701 74.2492277337492) bank28782 +28783 POINT(40.62964013945426 73.84329935945632) bank28783 +28784 POINT(41.70223242625001 73.50389884878103) bank28784 +28785 POINT(41.38234066663304 74.77119251438486) bank28785 +28786 POINT(39.99284204823852 74.6690999175262) bank28786 +28787 POINT(41.393390205609755 74.20800187680761) bank28787 +28788 POINT(40.9507576820164 74.74772219879165) bank28788 +28789 POINT(41.06117291392546 74.54525244473054) bank28789 +28790 POINT(41.669268255426836 74.40985482334348) bank28790 +28791 POINT(40.900077542640176 74.37892433477796) bank28791 +28792 POINT(40.28589957315875 74.35225819816202) bank28792 +28793 POINT(41.308490468798276 74.72440217049366) bank28793 +28794 POINT(40.80003512801957 74.34941168982165) bank28794 +28795 POINT(41.59433188400805 73.04766287555267) bank28795 +28796 POINT(40.223234473554854 74.58125551808692) bank28796 +28797 POINT(40.93072426818036 73.62687176403028) bank28797 +28798 POINT(40.98218059960084 74.79816393894531) bank28798 +28799 POINT(39.82701091586574 74.02334774047746) bank28799 +28800 POINT(40.53498721118961 73.80482184550344) bank28800 +28801 POINT(41.01193716405502 73.87898962677495) bank28801 +28802 POINT(39.74385663668552 73.58341780272961) bank28802 +28803 POINT(40.628422207329386 74.88755741015879) bank28803 +28804 POINT(39.98732856991762 74.17721053377065) bank28804 +28805 POINT(40.49702110472833 73.04128452420858) bank28805 +28806 POINT(41.14992367344008 74.43669112177959) bank28806 +28807 POINT(40.551138744492405 73.31101796909721) bank28807 +28808 POINT(40.84806074107553 74.73581252861587) bank28808 +28809 POINT(40.11280915868874 74.04666497485987) bank28809 +28810 POINT(40.948193179201304 74.6613763530705) bank28810 +28811 POINT(40.828145396098186 74.65790578020955) bank28811 +28812 POINT(39.924370637146396 74.48931671920313) bank28812 +28813 POINT(40.39964701252609 74.47844402334445) bank28813 +28814 POINT(39.978137567844136 74.80047048597808) bank28814 +28815 POINT(40.85651199421184 73.9195457969467) bank28815 +28816 POINT(41.47776497006797 73.60105294075328) bank28816 +28817 POINT(40.383058238261825 74.38722474114432) bank28817 +28818 POINT(40.531400477169484 74.64139025294001) bank28818 +28819 POINT(39.71944989706063 73.67682229625163) bank28819 +28820 POINT(41.263330713718034 74.13607475247075) bank28820 +28821 POINT(39.88698778634491 73.62754879272492) bank28821 +28822 POINT(40.458175018788964 74.35956287096683) bank28822 +28823 POINT(40.09457921498692 74.98020008459338) bank28823 +28824 POINT(40.45999679003419 73.35222185109491) bank28824 +28825 POINT(40.76710478601143 74.87857708390759) bank28825 +28826 POINT(40.74879731253568 73.43678097610818) bank28826 +28827 POINT(39.968843109522304 73.20688233372584) bank28827 +28828 POINT(40.202262579035114 74.49559934893296) bank28828 +28829 POINT(41.226676817032875 74.79032451233488) bank28829 +28830 POINT(40.35720523040511 73.90578416569409) bank28830 +28831 POINT(41.47644635038697 73.14782559442511) bank28831 +28832 POINT(41.462405541250504 73.19169063727313) bank28832 +28833 POINT(39.943478959845976 73.76145050016945) bank28833 +28834 POINT(41.254916155056556 73.20543221360519) bank28834 +28835 POINT(41.229144160778056 73.58110719775367) bank28835 +28836 POINT(40.80898379823936 74.60154531716837) bank28836 +28837 POINT(40.24527481890529 74.53576956736246) bank28837 +28838 POINT(40.7423504087853 74.54864936712193) bank28838 +28839 POINT(39.930233081230554 73.04966143298451) bank28839 +28840 POINT(40.45096911855127 73.96447137060431) bank28840 +28841 POINT(40.598391256945256 74.35572226478851) bank28841 +28842 POINT(40.142078022079986 73.32359943154852) bank28842 +28843 POINT(40.4916553962722 74.98684326055064) bank28843 +28844 POINT(40.85385826837741 74.85302189712937) bank28844 +28845 POINT(41.15152931505875 74.79930990796431) bank28845 +28846 POINT(41.11723990941716 73.35643149711692) bank28846 +28847 POINT(41.393725231807174 74.96865176205483) bank28847 +28848 POINT(41.09096083890192 74.06732511853629) bank28848 +28849 POINT(40.21153132730562 73.26471143231446) bank28849 +28850 POINT(39.77483022280943 73.11041673594875) bank28850 +28851 POINT(41.26118989725518 73.22465076932416) bank28851 +28852 POINT(40.32793392781262 74.85912391299675) bank28852 +28853 POINT(41.706971298174366 73.57193960009899) bank28853 +28854 POINT(41.693554739213184 74.28052941840363) bank28854 +28855 POINT(40.867990917909154 73.8678117206582) bank28855 +28856 POINT(41.50859444177517 73.13608664977525) bank28856 +28857 POINT(39.77913183215328 74.26054168853652) bank28857 +28858 POINT(41.240329998339384 74.4769045280545) bank28858 +28859 POINT(41.5046189719184 73.09911342044664) bank28859 +28860 POINT(39.798626150914835 73.82234540833021) bank28860 +28861 POINT(40.32591771500873 73.39031874213423) bank28861 +28862 POINT(40.80763903485654 74.34576499444579) bank28862 +28863 POINT(41.39789386766187 73.34833048782133) bank28863 +28864 POINT(41.12984749269235 73.74191086563299) bank28864 +28865 POINT(40.78409097708688 73.89664423693746) bank28865 +28866 POINT(40.58619268135656 74.79272030633679) bank28866 +28867 POINT(39.757573013159416 73.59559283191649) bank28867 +28868 POINT(41.28589993974888 73.89773491857979) bank28868 +28869 POINT(39.878562480188464 74.99781691924181) bank28869 +28870 POINT(39.84755042581319 74.21482056021824) bank28870 +28871 POINT(40.07379474720839 74.7241995518671) bank28871 +28872 POINT(41.31453627802034 73.77386809150472) bank28872 +28873 POINT(40.94919740566168 73.44588294343514) bank28873 +28874 POINT(40.33374869088692 73.72256943283281) bank28874 +28875 POINT(41.635137729540844 73.19935656640067) bank28875 +28876 POINT(39.820236314177215 74.46650147128432) bank28876 +28877 POINT(40.819508931987095 73.02937465952252) bank28877 +28878 POINT(41.483085807604546 74.60024873380652) bank28878 +28879 POINT(40.04125898408178 73.5303638557513) bank28879 +28880 POINT(40.19900544425715 74.22566178259137) bank28880 +28881 POINT(41.345495333304015 74.53319832677421) bank28881 +28882 POINT(41.41106570630513 73.38900961092308) bank28882 +28883 POINT(40.74546301501773 74.15056353957381) bank28883 +28884 POINT(40.04856823323864 73.37619999935343) bank28884 +28885 POINT(41.4607151723351 73.03328053009017) bank28885 +28886 POINT(40.167613779523066 73.50050433201908) bank28886 +28887 POINT(40.70640480957215 73.45343861979015) bank28887 +28888 POINT(40.79091536107945 74.26522123038964) bank28888 +28889 POINT(41.04419114804101 74.7194420232455) bank28889 +28890 POINT(40.00737694574844 73.3546864428204) bank28890 +28891 POINT(40.67591581710913 73.17532189459446) bank28891 +28892 POINT(40.494538958011944 73.40688354542226) bank28892 +28893 POINT(39.80023930170727 73.27303362810864) bank28893 +28894 POINT(41.436106044665976 73.80236921368417) bank28894 +28895 POINT(40.78484310092137 73.50064170800594) bank28895 +28896 POINT(40.628400330184206 73.24176908941224) bank28896 +28897 POINT(40.99724358047221 74.82701816407156) bank28897 +28898 POINT(39.93582834612422 73.76888837364906) bank28898 +28899 POINT(40.53792383673194 74.62595299879126) bank28899 +28900 POINT(40.39594027443152 74.72186293037285) bank28900 +28901 POINT(40.6543939709735 74.78411616817232) bank28901 +28902 POINT(41.43417933657642 73.96055461491652) bank28902 +28903 POINT(41.01555242993963 73.18211460353972) bank28903 +28904 POINT(40.481502125473995 74.561729724633) bank28904 +28905 POINT(41.710940286604846 73.39617739524614) bank28905 +28906 POINT(40.46528685832006 74.93528311443684) bank28906 +28907 POINT(41.43376893617816 74.72122938786812) bank28907 +28908 POINT(40.95953490776069 73.8268512979248) bank28908 +28909 POINT(41.47454631641278 74.9079060862891) bank28909 +28910 POINT(41.54265718606448 73.03358158542474) bank28910 +28911 POINT(41.44783201312794 73.21534946037283) bank28911 +28912 POINT(40.067207630952296 73.32681480360223) bank28912 +28913 POINT(41.570260766233815 74.30786507615707) bank28913 +28914 POINT(41.50413915697275 73.42020368089979) bank28914 +28915 POINT(41.67182665140064 73.32188219978615) bank28915 +28916 POINT(40.443515136624875 73.38305648146027) bank28916 +28917 POINT(40.056213618143204 74.83873851981492) bank28917 +28918 POINT(40.26953445184829 74.06803712987215) bank28918 +28919 POINT(39.811784266613046 73.56279099210803) bank28919 +28920 POINT(41.57784102045681 74.33665359150841) bank28920 +28921 POINT(40.97989226606285 74.3232997019439) bank28921 +28922 POINT(41.110128170367474 73.448163422252) bank28922 +28923 POINT(39.94837657373785 73.39353959426393) bank28923 +28924 POINT(40.10196708760328 74.9989264021415) bank28924 +28925 POINT(40.261342671452624 73.899971865344) bank28925 +28926 POINT(41.69926002215291 74.89138914350386) bank28926 +28927 POINT(40.75587461116333 74.50402705988539) bank28927 +28928 POINT(40.25217944144961 73.87789498277502) bank28928 +28929 POINT(40.414940238819156 73.90846816331586) bank28929 +28930 POINT(40.58636048818908 74.23212522203232) bank28930 +28931 POINT(41.172901557401694 74.67208643575343) bank28931 +28932 POINT(41.31016931874511 73.04876699024) bank28932 +28933 POINT(41.609024169605085 74.7458684046258) bank28933 +28934 POINT(40.67664073208925 73.22392869693942) bank28934 +28935 POINT(41.70280566896042 74.0224999108819) bank28935 +28936 POINT(40.75865338498837 73.75039598541412) bank28936 +28937 POINT(41.16827463801023 73.8725191151659) bank28937 +28938 POINT(40.82476683312539 74.81235552918422) bank28938 +28939 POINT(40.843143147890515 73.56562491308016) bank28939 +28940 POINT(39.85360224884195 74.76719778162773) bank28940 +28941 POINT(40.18054483694175 74.05539671948348) bank28941 +28942 POINT(39.98322309410373 73.43201296423395) bank28942 +28943 POINT(40.832263153013905 73.52587718864125) bank28943 +28944 POINT(40.44809561156958 74.80132302287615) bank28944 +28945 POINT(40.56893083498585 73.49812454575299) bank28945 +28946 POINT(40.798696423825376 73.14291005878945) bank28946 +28947 POINT(39.81159001110212 73.10532999107977) bank28947 +28948 POINT(41.54475602564997 73.20883124354587) bank28948 +28949 POINT(41.436988530521425 74.60106067340564) bank28949 +28950 POINT(41.04775778826644 74.99910200928649) bank28950 +28951 POINT(41.17717410487714 74.0979662263778) bank28951 +28952 POINT(41.517101773670916 73.23917772946243) bank28952 +28953 POINT(40.50402257380824 73.58887879133373) bank28953 +28954 POINT(41.51255036534176 74.86989063522036) bank28954 +28955 POINT(40.86380910450697 74.32586293267029) bank28955 +28956 POINT(41.012930438718854 73.93890658561418) bank28956 +28957 POINT(39.78419601441902 73.0475242834477) bank28957 +28958 POINT(40.17198528368285 73.03433921524446) bank28958 +28959 POINT(41.492693882691576 73.33696761877717) bank28959 +28960 POINT(41.128872832594574 74.53027125152197) bank28960 +28961 POINT(40.071826520419656 73.66978742579404) bank28961 +28962 POINT(40.59748304819327 74.0452010628076) bank28962 +28963 POINT(39.86629454743826 73.84691938439113) bank28963 +28964 POINT(41.68739615035871 74.84472553911992) bank28964 +28965 POINT(39.834888083970675 73.56007100031115) bank28965 +28966 POINT(40.51916932875149 74.60015229641336) bank28966 +28967 POINT(41.405986542434476 74.51845861304402) bank28967 +28968 POINT(40.815084009422684 73.98832267591379) bank28968 +28969 POINT(41.594689022921784 73.06010575480711) bank28969 +28970 POINT(40.25498546402266 74.1496514833125) bank28970 +28971 POINT(41.474535402662454 73.52468613619988) bank28971 +28972 POINT(40.383571425718266 74.71778341254398) bank28972 +28973 POINT(40.46689222702356 73.52709442609135) bank28973 +28974 POINT(40.55042049979495 74.64776669510499) bank28974 +28975 POINT(41.05906504757834 73.48501275900999) bank28975 +28976 POINT(41.012233546961866 73.49542219757177) bank28976 +28977 POINT(40.28362679944114 74.23147457953813) bank28977 +28978 POINT(40.98978705374518 73.18435693526843) bank28978 +28979 POINT(41.27788362283598 73.55504461881958) bank28979 +28980 POINT(40.89414824699255 73.902022531386) bank28980 +28981 POINT(41.08490317177294 73.28597432237555) bank28981 +28982 POINT(39.85927679130352 74.5362355451659) bank28982 +28983 POINT(40.304933014210405 73.55877379713834) bank28983 +28984 POINT(40.05249550512047 73.53086051076963) bank28984 +28985 POINT(41.15164150262973 73.56666353629983) bank28985 +28986 POINT(40.13177565919522 73.08918413162074) bank28986 +28987 POINT(40.12906223887449 74.93075326233735) bank28987 +28988 POINT(41.180229611489686 73.18274153867463) bank28988 +28989 POINT(40.06653174482151 73.97328718073754) bank28989 +28990 POINT(41.60649573347985 74.31426610862208) bank28990 +28991 POINT(41.536749390432846 73.7921492471395) bank28991 +28992 POINT(40.59465123176353 74.91201042085655) bank28992 +28993 POINT(40.514944267078256 73.92663808123662) bank28993 +28994 POINT(39.90995815948904 74.0659261228057) bank28994 +28995 POINT(41.20297591135561 74.18316160618788) bank28995 +28996 POINT(40.74999475808165 74.45802453896428) bank28996 +28997 POINT(39.885046155883046 73.35488643561197) bank28997 +28998 POINT(40.549570474140495 73.07055952106009) bank28998 +28999 POINT(41.28603838835228 74.30854836729802) bank28999 +29000 POINT(40.19603389645196 73.12645545666983) bank29000 +29001 POINT(39.926136687877175 74.52570959856673) bank29001 +29002 POINT(40.00397256828883 74.32613136586936) bank29002 +29003 POINT(40.155116572790874 74.13937740051509) bank29003 +29004 POINT(39.73536667096078 73.91770730516579) bank29004 +29005 POINT(39.801437832908675 74.10749354736848) bank29005 +29006 POINT(40.34630441066544 73.99756675478524) bank29006 +29007 POINT(39.74378749737685 74.86058001576806) bank29007 +29008 POINT(40.56586331629282 73.6145935022115) bank29008 +29009 POINT(39.98692804152641 73.4001581948524) bank29009 +29010 POINT(40.13408565412269 74.91386611294217) bank29010 +29011 POINT(41.497917518263755 73.7175992476893) bank29011 +29012 POINT(40.2958481785796 74.55673544773725) bank29012 +29013 POINT(41.14068273535912 74.33332140948781) bank29013 +29014 POINT(40.03278329321774 74.46718455072201) bank29014 +29015 POINT(40.71445982896883 74.45207097966583) bank29015 +29016 POINT(40.686530444589906 74.47254386452587) bank29016 +29017 POINT(41.3825730222119 73.53693768438353) bank29017 +29018 POINT(40.87852463439118 73.37028182415834) bank29018 +29019 POINT(40.172933761723506 74.20247826745347) bank29019 +29020 POINT(41.20398498127225 73.98902061438962) bank29020 +29021 POINT(41.0975899753601 74.8004999427335) bank29021 +29022 POINT(40.85403260974857 74.82830880328042) bank29022 +29023 POINT(40.56282928471416 74.10560504136481) bank29023 +29024 POINT(41.620477965973144 74.16146286341126) bank29024 +29025 POINT(39.74537902609824 74.42655231352282) bank29025 +29026 POINT(41.35016966402713 74.23395488010432) bank29026 +29027 POINT(41.585112146337636 74.66666118446322) bank29027 +29028 POINT(39.80692015968853 73.3928663456417) bank29028 +29029 POINT(41.69570232343973 74.2316278098695) bank29029 +29030 POINT(41.22162572233794 74.6102510694789) bank29030 +29031 POINT(41.70213300817534 73.83019603928528) bank29031 +29032 POINT(41.10475932712291 74.04834414023328) bank29032 +29033 POINT(41.63366803211475 74.90668279011507) bank29033 +29034 POINT(39.80414591678899 73.46229647300433) bank29034 +29035 POINT(40.32659438474713 74.71655450832932) bank29035 +29036 POINT(41.041721494531515 73.08070669673401) bank29036 +29037 POINT(41.05330153021999 74.40622076901919) bank29037 +29038 POINT(41.097246632208744 73.67245471060019) bank29038 +29039 POINT(39.94862537788267 73.96088705876234) bank29039 +29040 POINT(40.31328508100373 73.16394101504362) bank29040 +29041 POINT(40.69186272649675 74.34249447513436) bank29041 +29042 POINT(40.77337934517177 73.50017665244842) bank29042 +29043 POINT(40.146563825971654 74.15905014385868) bank29043 +29044 POINT(40.583444058901904 74.11277889475528) bank29044 +29045 POINT(40.585889223052554 74.39645651910642) bank29045 +29046 POINT(40.41899433175585 73.35591486015373) bank29046 +29047 POINT(40.05142760606957 73.70923177497086) bank29047 +29048 POINT(41.38739463887069 74.39242003182734) bank29048 +29049 POINT(41.08161347694863 74.95009595546483) bank29049 +29050 POINT(41.63005098302005 74.40903212813514) bank29050 +29051 POINT(40.78960689128311 74.31768739868313) bank29051 +29052 POINT(40.04018632952728 74.73774286377359) bank29052 +29053 POINT(41.67598610010989 74.01041491785823) bank29053 +29054 POINT(41.554658015228114 73.82070928208623) bank29054 +29055 POINT(41.00833142743257 73.69789455324943) bank29055 +29056 POINT(41.40568370106092 74.8585721634175) bank29056 +29057 POINT(40.15127656324032 74.4385417123273) bank29057 +29058 POINT(40.771541693297735 73.38731283172206) bank29058 +29059 POINT(41.055960502698326 74.49426986572242) bank29059 +29060 POINT(41.40069241212925 73.23139496320495) bank29060 +29061 POINT(39.930693511469855 73.90359884594024) bank29061 +29062 POINT(41.22753407308842 73.47586056181166) bank29062 +29063 POINT(40.86833485496163 74.18750443303496) bank29063 +29064 POINT(40.57138026677909 74.19085541822722) bank29064 +29065 POINT(41.36640556060742 73.11545265662646) bank29065 +29066 POINT(40.07529267913754 73.02815192157915) bank29066 +29067 POINT(41.705385611309524 74.90001108526577) bank29067 +29068 POINT(39.71779250113718 74.57310592881824) bank29068 +29069 POINT(40.175520801155386 73.31756822350583) bank29069 +29070 POINT(41.21837897807868 74.84554276040012) bank29070 +29071 POINT(40.157750091196 74.82907285960069) bank29071 +29072 POINT(39.87766377218937 73.70357110957774) bank29072 +29073 POINT(41.04883393313287 73.76995233318262) bank29073 +29074 POINT(41.124918153008494 74.2251273353676) bank29074 +29075 POINT(41.13622519492775 73.09132382889648) bank29075 +29076 POINT(41.0869036917813 74.03222717607235) bank29076 +29077 POINT(41.499092488553764 74.1055992996709) bank29077 +29078 POINT(41.481914403137104 73.49781385417052) bank29078 +29079 POINT(40.00105469569999 74.82615737313193) bank29079 +29080 POINT(41.295477684465915 73.21383374065543) bank29080 +29081 POINT(41.62072573725972 74.74149649186313) bank29081 +29082 POINT(40.05324805625753 73.31989980881644) bank29082 +29083 POINT(41.17317136550289 73.19171373186236) bank29083 +29084 POINT(40.352142251041556 73.72483499120452) bank29084 +29085 POINT(40.15603048199672 74.43443407347064) bank29085 +29086 POINT(41.02428047894104 74.23459563076798) bank29086 +29087 POINT(41.14531043227459 73.1461125377565) bank29087 +29088 POINT(40.51445620552207 73.23852980281661) bank29088 +29089 POINT(40.21747942176855 74.52251028234451) bank29089 +29090 POINT(40.96843497840227 74.08775072650084) bank29090 +29091 POINT(40.52026331425167 73.29681802055926) bank29091 +29092 POINT(40.4632806824366 73.72530432470765) bank29092 +29093 POINT(40.28892202348561 73.92090766230116) bank29093 +29094 POINT(41.39729945088278 73.74779052144167) bank29094 +29095 POINT(40.2246186458781 73.73230035437795) bank29095 +29096 POINT(40.44290178651173 73.40429689103499) bank29096 +29097 POINT(41.08519127211364 73.72801488697988) bank29097 +29098 POINT(40.55149853268911 74.48685998486779) bank29098 +29099 POINT(40.31700995718371 74.45319204754053) bank29099 +29100 POINT(39.91319122208196 74.97306823274975) bank29100 +29101 POINT(39.90469433655896 73.01655068174341) bank29101 +29102 POINT(41.63609227921624 73.19364317581865) bank29102 +29103 POINT(40.596311549383216 73.61396906137416) bank29103 +29104 POINT(39.78870108209276 74.04932325348123) bank29104 +29105 POINT(40.86942753292075 74.49409861919334) bank29105 +29106 POINT(40.224913540934274 74.17275597943114) bank29106 +29107 POINT(41.36679376159336 73.71569676280492) bank29107 +29108 POINT(41.22396410777043 74.5740767267136) bank29108 +29109 POINT(41.41643917525254 73.41703013034923) bank29109 +29110 POINT(39.718350558399244 74.40081723388843) bank29110 +29111 POINT(41.64604929235862 73.4294241472504) bank29111 +29112 POINT(41.54183031404824 74.61664803316594) bank29112 +29113 POINT(41.57273845059763 74.05517201862851) bank29113 +29114 POINT(40.27558800956895 73.60619891041992) bank29114 +29115 POINT(40.115130646360015 74.81432522832647) bank29115 +29116 POINT(39.723938455609215 73.33130983378568) bank29116 +29117 POINT(41.098458971264876 74.07837772899111) bank29117 +29118 POINT(40.191977797305846 74.34027316828413) bank29118 +29119 POINT(41.276291835918656 74.59221455927101) bank29119 +29120 POINT(40.95127357615029 74.1457698564659) bank29120 +29121 POINT(41.15229998240349 73.0334709187416) bank29121 +29122 POINT(41.49012381048205 73.04397261609928) bank29122 +29123 POINT(41.5860227216563 73.53465782543898) bank29123 +29124 POINT(40.63289101161801 73.26238415269665) bank29124 +29125 POINT(40.89320642221325 73.20968063291487) bank29125 +29126 POINT(40.899574940292624 74.6428659620032) bank29126 +29127 POINT(41.20472719976829 74.52255885081382) bank29127 +29128 POINT(40.35138992808819 74.49369359094776) bank29128 +29129 POINT(40.08398903522825 73.70212241211388) bank29129 +29130 POINT(40.90119056293399 74.6403587743059) bank29130 +29131 POINT(41.41520088349302 74.22814117204625) bank29131 +29132 POINT(41.25551396041299 73.84421280514876) bank29132 +29133 POINT(41.23508110484537 73.63618890304755) bank29133 +29134 POINT(40.72731470258821 73.47889797207233) bank29134 +29135 POINT(41.681468844840474 73.8025316874826) bank29135 +29136 POINT(40.99321973060072 74.3284852626946) bank29136 +29137 POINT(40.6965826778204 73.82683270769884) bank29137 +29138 POINT(41.70446407555755 74.5031988530529) bank29138 +29139 POINT(40.95048779158561 73.01696637394039) bank29139 +29140 POINT(41.60290368676518 73.44943459892376) bank29140 +29141 POINT(41.31616079547743 74.9575115543602) bank29141 +29142 POINT(41.365813822788205 74.42243961052502) bank29142 +29143 POINT(40.20233894464583 74.0432145267035) bank29143 +29144 POINT(40.92455174561281 74.95255120120211) bank29144 +29145 POINT(41.39437478473639 73.20682963764146) bank29145 +29146 POINT(40.62590175748147 73.93735932837755) bank29146 +29147 POINT(41.42162437154918 73.51015169404171) bank29147 +29148 POINT(41.30190184174065 73.44696641108338) bank29148 +29149 POINT(40.05516516086378 74.9103292252319) bank29149 +29150 POINT(40.67699776715566 73.80788107862053) bank29150 +29151 POINT(39.809889319685794 73.02127250430638) bank29151 +29152 POINT(40.22113805800577 73.14609120302426) bank29152 +29153 POINT(40.673746967919506 74.19379531847902) bank29153 +29154 POINT(41.67834821342364 73.95764823940118) bank29154 +29155 POINT(41.31378343340886 74.69915488839523) bank29155 +29156 POINT(41.55156969106952 74.77236231128522) bank29156 +29157 POINT(40.889004911392895 73.24550924098627) bank29157 +29158 POINT(40.729524375705644 73.47203725160828) bank29158 +29159 POINT(40.74503244464928 74.97369881668311) bank29159 +29160 POINT(40.718341639395305 73.96927785940235) bank29160 +29161 POINT(40.413876917581845 74.12154370781144) bank29161 +29162 POINT(41.56870399849422 74.65580920676021) bank29162 +29163 POINT(39.9315341390214 73.68193954125358) bank29163 +29164 POINT(39.74440697123801 74.14820518123317) bank29164 +29165 POINT(41.41777741084423 73.31300813929457) bank29165 +29166 POINT(41.57765677042235 74.41391768798651) bank29166 +29167 POINT(41.61433108272763 73.67107814466446) bank29167 +29168 POINT(41.5991052408487 73.50408988156008) bank29168 +29169 POINT(39.78722918996107 73.63001031942225) bank29169 +29170 POINT(39.751346458331085 73.6538989971734) bank29170 +29171 POINT(40.77104564903974 74.6691419095712) bank29171 +29172 POINT(39.76119395109497 73.29497623135198) bank29172 +29173 POINT(40.75163301722062 74.41296482557927) bank29173 +29174 POINT(40.197425038766994 73.72711176948893) bank29174 +29175 POINT(41.07917681413282 73.70167343939988) bank29175 +29176 POINT(41.39031621997688 73.63627748016177) bank29176 +29177 POINT(40.695787417044635 74.29727537713141) bank29177 +29178 POINT(40.81777541752609 73.29839294283134) bank29178 +29179 POINT(39.99759671909202 74.01069388076573) bank29179 +29180 POINT(40.00400837714537 74.95164490013043) bank29180 +29181 POINT(39.824605974368005 73.65742715552066) bank29181 +29182 POINT(41.51853378833052 74.51671173162504) bank29182 +29183 POINT(40.68713690500415 74.59016555751899) bank29183 +29184 POINT(39.937582780161144 73.46957327506914) bank29184 +29185 POINT(41.357876916291545 73.01736167255731) bank29185 +29186 POINT(39.99755099222655 74.60350765361386) bank29186 +29187 POINT(39.953629777051276 74.8672673908624) bank29187 +29188 POINT(41.125334890957795 73.1356083095771) bank29188 +29189 POINT(40.22350608507297 73.29801170860102) bank29189 +29190 POINT(40.24851567057902 74.34882071824697) bank29190 +29191 POINT(41.67738890817687 74.42625590467466) bank29191 +29192 POINT(40.39485867214932 74.72265921576057) bank29192 +29193 POINT(40.254961623171695 73.87115606771432) bank29193 +29194 POINT(41.492002699005 74.37454371316886) bank29194 +29195 POINT(39.80294460474975 73.32563411013413) bank29195 +29196 POINT(41.326639460522266 73.28324297705072) bank29196 +29197 POINT(41.51094899102428 73.95287408800434) bank29197 +29198 POINT(40.50520266209607 73.63664004654792) bank29198 +29199 POINT(41.600855573089504 73.75115674983678) bank29199 +29200 POINT(40.22466187105315 74.78382676783951) bank29200 +29201 POINT(40.5482621533477 73.41952963915206) bank29201 +29202 POINT(40.20883003383992 74.8957296018343) bank29202 +29203 POINT(41.091363588714565 74.57206099936919) bank29203 +29204 POINT(41.39408255387822 74.83823730642679) bank29204 +29205 POINT(40.17566350916275 74.98909714535183) bank29205 +29206 POINT(41.51536164569656 73.8555076394071) bank29206 +29207 POINT(41.4097634690585 74.69930407877533) bank29207 +29208 POINT(41.30440322526242 73.05365729380758) bank29208 +29209 POINT(40.16121716137906 73.88514014744057) bank29209 +29210 POINT(41.439874135128825 73.15278530488803) bank29210 +29211 POINT(41.20930528923968 73.92328783834567) bank29211 +29212 POINT(41.09836808993508 74.38836695080377) bank29212 +29213 POINT(40.242820437687755 73.42544080095217) bank29213 +29214 POINT(40.9448657411769 73.38400145132691) bank29214 +29215 POINT(40.40857299177753 73.94119229767476) bank29215 +29216 POINT(41.409433052922545 74.58309078041765) bank29216 +29217 POINT(41.08506878289175 73.36931690759079) bank29217 +29218 POINT(40.36075800793594 73.57592434738113) bank29218 +29219 POINT(40.5697729123567 74.83299085934938) bank29219 +29220 POINT(40.25306033805205 74.54525127218095) bank29220 +29221 POINT(40.415038205189 74.42872165037133) bank29221 +29222 POINT(40.134176648569614 73.68887175397286) bank29222 +29223 POINT(40.25942469688356 73.0305886734064) bank29223 +29224 POINT(41.16227214516566 73.4957565543702) bank29224 +29225 POINT(40.17165189146333 74.56742521949738) bank29225 +29226 POINT(41.183192060511566 73.68529866832226) bank29226 +29227 POINT(40.77559478812847 73.1036687480314) bank29227 +29228 POINT(41.2712610410422 73.2914153539353) bank29228 +29229 POINT(40.64163408697919 73.08483965986373) bank29229 +29230 POINT(41.30480359617933 73.4786652906769) bank29230 +29231 POINT(40.110006522087644 74.19466714528686) bank29231 +29232 POINT(40.15107484053896 73.66730021704194) bank29232 +29233 POINT(40.70588419831843 74.37670991751283) bank29233 +29234 POINT(39.75438677491234 74.75985647797293) bank29234 +29235 POINT(41.132596582214696 73.58576350231289) bank29235 +29236 POINT(39.96319982726584 73.01952110580346) bank29236 +29237 POINT(40.36682694937974 73.02133076885133) bank29237 +29238 POINT(40.86754036883325 73.8283315112846) bank29238 +29239 POINT(40.181930481853065 73.99206153397523) bank29239 +29240 POINT(40.83682397718715 73.3594232719156) bank29240 +29241 POINT(39.72678430795233 73.95897746119324) bank29241 +29242 POINT(41.70827732797309 74.45493054056583) bank29242 +29243 POINT(41.62022405440435 73.34933472882169) bank29243 +29244 POINT(41.30966964316504 74.62191555521345) bank29244 +29245 POINT(40.21565336223754 74.3051531062651) bank29245 +29246 POINT(41.48101894803605 74.18508892628661) bank29246 +29247 POINT(40.21386584082985 74.71311148277731) bank29247 +29248 POINT(39.96646694175097 74.26644231009969) bank29248 +29249 POINT(41.37605001483604 74.00924678935125) bank29249 +29250 POINT(41.68035519889075 73.12061737568902) bank29250 +29251 POINT(40.19997170291261 74.4278587658938) bank29251 +29252 POINT(40.90903945425739 73.70398916430328) bank29252 +29253 POINT(39.95198610960388 74.938423474333) bank29253 +29254 POINT(41.100851411995194 73.58370430836005) bank29254 +29255 POINT(40.643444764446684 73.35765544198875) bank29255 +29256 POINT(39.87874892053807 73.88603840841095) bank29256 +29257 POINT(40.167280417256784 73.87394890365151) bank29257 +29258 POINT(40.50866420836911 73.45049589500611) bank29258 +29259 POINT(41.03407488696939 74.39346193440639) bank29259 +29260 POINT(40.36104157840522 74.57214821035141) bank29260 +29261 POINT(41.4282547777286 73.50490118856113) bank29261 +29262 POINT(41.51335587024671 73.68348141360329) bank29262 +29263 POINT(40.40703830744004 74.00825683926747) bank29263 +29264 POINT(40.529013374532845 73.02892889293886) bank29264 +29265 POINT(41.45418636731588 73.88130423025923) bank29265 +29266 POINT(40.03258586650729 74.70430787594513) bank29266 +29267 POINT(40.46422873504197 73.25314065445157) bank29267 +29268 POINT(40.64843184131109 73.08836871965828) bank29268 +29269 POINT(39.91839952165436 74.5844292073059) bank29269 +29270 POINT(40.93861545825766 73.150177347975) bank29270 +29271 POINT(40.12925664363564 74.27149516839637) bank29271 +29272 POINT(40.00046722107837 73.4859338352574) bank29272 +29273 POINT(40.770490372826536 74.38135903124372) bank29273 +29274 POINT(41.46950442731379 74.07006027841508) bank29274 +29275 POINT(40.76660974314697 73.38909058325004) bank29275 +29276 POINT(41.27664221135318 73.09775370822753) bank29276 +29277 POINT(40.6477691613384 74.28517517112536) bank29277 +29278 POINT(40.682034552199745 73.49216169486792) bank29278 +29279 POINT(40.99433065592404 73.09178428044197) bank29279 +29280 POINT(41.2075923541363 73.07178254178636) bank29280 +29281 POINT(41.68801473536919 73.82272247720725) bank29281 +29282 POINT(39.91403193601004 73.32817710677239) bank29282 +29283 POINT(40.031831832641345 74.78565140585349) bank29283 +29284 POINT(40.249136906380826 74.67145283074701) bank29284 +29285 POINT(41.56234129937936 74.5690164171809) bank29285 +29286 POINT(41.06581533665795 73.4310356714519) bank29286 +29287 POINT(39.71478505171262 73.4531463787176) bank29287 +29288 POINT(41.31712628079238 74.39528473355603) bank29288 +29289 POINT(41.30853946592356 73.0148861161691) bank29289 +29290 POINT(40.28480293622085 74.98221405925439) bank29290 +29291 POINT(41.00127802428739 74.7790443378691) bank29291 +29292 POINT(39.869609241473896 73.05132477616607) bank29292 +29293 POINT(40.60542828233035 73.66938945501964) bank29293 +29294 POINT(41.07255169510151 73.88013432793191) bank29294 +29295 POINT(40.086731522717535 73.58632741669489) bank29295 +29296 POINT(41.66151947994581 73.35501869220433) bank29296 +29297 POINT(40.93656608681907 74.90979001984321) bank29297 +29298 POINT(40.251105615893394 74.84677270422337) bank29298 +29299 POINT(41.01488301333877 73.92129969624843) bank29299 +29300 POINT(41.608328827689306 74.4984383654311) bank29300 +29301 POINT(39.839270807541055 73.43548679462295) bank29301 +29302 POINT(39.78400707263093 74.62665545067661) bank29302 +29303 POINT(41.62088609995762 74.39103508485454) bank29303 +29304 POINT(40.70067509077222 74.2752189984252) bank29304 +29305 POINT(39.9563866514886 73.800378013731) bank29305 +29306 POINT(40.0750553611097 74.98207282416696) bank29306 +29307 POINT(40.78583199812704 74.04086992345434) bank29307 +29308 POINT(40.697141477943376 73.56226201957895) bank29308 +29309 POINT(40.20584398505143 73.61734325130395) bank29309 +29310 POINT(40.220145094043026 73.67601916298166) bank29310 +29311 POINT(40.400019808607 74.60285206311197) bank29311 +29312 POINT(40.2484912376097 73.24224526665886) bank29312 +29313 POINT(39.76643420164505 74.48364184141815) bank29313 +29314 POINT(40.211828475318065 74.19563361287024) bank29314 +29315 POINT(40.0730902816683 73.80522398849436) bank29315 +29316 POINT(41.33965769816679 73.66580172163535) bank29316 +29317 POINT(41.640032828878915 74.67742245949502) bank29317 +29318 POINT(40.1093404005259 74.87020139886128) bank29318 +29319 POINT(41.52319652235017 73.85827742380663) bank29319 +29320 POINT(40.281928264391006 73.56316087738448) bank29320 +29321 POINT(41.580997029040596 73.25144512209357) bank29321 +29322 POINT(40.32773937278571 73.76093148512696) bank29322 +29323 POINT(41.02699128610591 74.03943425983934) bank29323 +29324 POINT(41.611539321932156 74.42008023159559) bank29324 +29325 POINT(41.473136958051825 74.2176120262211) bank29325 +29326 POINT(41.29086813502319 74.8125062908361) bank29326 +29327 POINT(40.714262830245495 74.38375443361346) bank29327 +29328 POINT(39.814643275220824 74.46019279634758) bank29328 +29329 POINT(39.748889261627376 74.78116985992105) bank29329 +29330 POINT(41.05367268960199 73.56761112827428) bank29330 +29331 POINT(41.48386583477356 73.21963372007062) bank29331 +29332 POINT(39.96766749884052 74.98463924798571) bank29332 +29333 POINT(40.45432081137127 73.90448322207767) bank29333 +29334 POINT(40.03139759598878 73.26028930908274) bank29334 +29335 POINT(41.17384471032235 73.97002114232924) bank29335 +29336 POINT(41.56403201685875 74.15334928868664) bank29336 +29337 POINT(40.187525591622006 74.49445940012586) bank29337 +29338 POINT(41.190967521845096 74.63254066771458) bank29338 +29339 POINT(40.93579679174723 73.99997931564938) bank29339 +29340 POINT(40.72681031270545 73.39417447458726) bank29340 +29341 POINT(40.1069235103206 74.16602327512118) bank29341 +29342 POINT(41.41002183480634 74.61264996012605) bank29342 +29343 POINT(40.33215979246484 73.6391426749155) bank29343 +29344 POINT(40.6216879398908 74.60850111255283) bank29344 +29345 POINT(41.43522318964908 73.8949814058051) bank29345 +29346 POINT(39.832209564540825 73.36098915222878) bank29346 +29347 POINT(39.995455835668864 74.13803770988694) bank29347 +29348 POINT(40.56541694927176 73.33308491922165) bank29348 +29349 POINT(41.1907671500809 74.29665786380133) bank29349 +29350 POINT(41.44148335641268 73.83633038129659) bank29350 +29351 POINT(40.7973908311242 74.97532119962376) bank29351 +29352 POINT(40.87667483386613 74.27199539279175) bank29352 +29353 POINT(39.963746292324146 73.08825735837208) bank29353 +29354 POINT(40.81653207841098 74.98484368778864) bank29354 +29355 POINT(40.21266687315457 73.09758300490144) bank29355 +29356 POINT(41.08744089680184 74.01855682201591) bank29356 +29357 POINT(40.34329945469647 74.54198194619913) bank29357 +29358 POINT(40.45449110969915 73.18489936160381) bank29358 +29359 POINT(40.92448951319322 74.02238449656168) bank29359 +29360 POINT(41.57338606784678 73.5774470571072) bank29360 +29361 POINT(40.528859107920546 73.58099337405166) bank29361 +29362 POINT(41.38087798787792 73.416442764774) bank29362 +29363 POINT(40.68849329789189 74.97222436504585) bank29363 +29364 POINT(40.56166058878282 74.02224725475914) bank29364 +29365 POINT(40.40654488601632 73.49756125049954) bank29365 +29366 POINT(39.73998040150436 73.03123342840287) bank29366 +29367 POINT(40.36403203615584 74.153548954859) bank29367 +29368 POINT(40.68080733894977 73.81276435775469) bank29368 +29369 POINT(40.59856423993076 73.48755208951953) bank29369 +29370 POINT(41.53547956585273 74.27494779506497) bank29370 +29371 POINT(41.55278726633326 74.2957843145313) bank29371 +29372 POINT(40.433848435032104 74.04615124716422) bank29372 +29373 POINT(40.977867251435335 73.61942393642077) bank29373 +29374 POINT(40.071576614849775 74.57883421979271) bank29374 +29375 POINT(40.896681895588564 74.39489072477267) bank29375 +29376 POINT(40.16532657209692 74.52694222967635) bank29376 +29377 POINT(41.70464679134135 74.50228675819442) bank29377 +29378 POINT(39.735411400214296 73.79236250713197) bank29378 +29379 POINT(40.33532932190579 74.6070019081164) bank29379 +29380 POINT(39.9013035822101 74.58648898506853) bank29380 +29381 POINT(41.04863042545035 73.9473354931785) bank29381 +29382 POINT(39.98151520679649 73.84975291822126) bank29382 +29383 POINT(40.73267258386039 74.19231926290308) bank29383 +29384 POINT(40.21366774426307 73.21118899054375) bank29384 +29385 POINT(40.615458215552245 74.48996217770322) bank29385 +29386 POINT(39.74651116059451 73.03131172743832) bank29386 +29387 POINT(40.81620067224961 74.1372180593585) bank29387 +29388 POINT(41.46268236997237 73.9205903740976) bank29388 +29389 POINT(40.846628795273766 73.45570829835695) bank29389 +29390 POINT(40.05582299114608 74.29109759646032) bank29390 +29391 POINT(41.20483248367013 74.5534353184337) bank29391 +29392 POINT(40.17177980998533 74.64544428646367) bank29392 +29393 POINT(41.31207869812697 73.12028889246024) bank29393 +29394 POINT(41.287930204690824 74.89632137273638) bank29394 +29395 POINT(41.378509339220514 73.99564941386006) bank29395 +29396 POINT(39.89772078721964 74.82826677520153) bank29396 +29397 POINT(40.52942991385638 73.47689670800997) bank29397 +29398 POINT(41.12417478992136 73.95555058783327) bank29398 +29399 POINT(41.171384979025206 74.69059160911043) bank29399 +29400 POINT(40.185279193705654 73.22045825799523) bank29400 +29401 POINT(40.01773517599623 74.24151132217096) bank29401 +29402 POINT(41.31081320120398 74.30322051062534) bank29402 +29403 POINT(40.325108278021716 73.5438592163863) bank29403 +29404 POINT(41.688920195298444 73.23802212783902) bank29404 +29405 POINT(40.467814124059224 74.7065094887167) bank29405 +29406 POINT(40.15609155215383 73.87301866442557) bank29406 +29407 POINT(40.42535144266053 73.37124470034047) bank29407 +29408 POINT(40.793612318949954 74.55727621352301) bank29408 +29409 POINT(40.63538565261337 74.13507830869636) bank29409 +29410 POINT(40.38097680158526 74.7456609880939) bank29410 +29411 POINT(41.690469381241144 73.56093778388157) bank29411 +29412 POINT(40.090345719074804 73.22064942396092) bank29412 +29413 POINT(40.89934986775715 73.04365225211126) bank29413 +29414 POINT(41.650456365353875 74.46664433206878) bank29414 +29415 POINT(41.21533987205743 73.80804174316772) bank29415 +29416 POINT(40.73056205867746 73.35294974305675) bank29416 +29417 POINT(40.59236035803033 74.07133039929289) bank29417 +29418 POINT(39.80289962345772 74.79564900813692) bank29418 +29419 POINT(41.441920683021586 74.85520965748746) bank29419 +29420 POINT(40.454421374908634 73.09830923757391) bank29420 +29421 POINT(40.89175344501555 74.09643539117823) bank29421 +29422 POINT(40.939301184139815 74.00135241322917) bank29422 +29423 POINT(41.432180708269335 73.22160031583512) bank29423 +29424 POINT(40.4049471783117 73.25259203313438) bank29424 +29425 POINT(40.79616585743873 73.61766704570715) bank29425 +29426 POINT(40.03190269340554 74.05590192669136) bank29426 +29427 POINT(39.91278595168337 74.23480380847154) bank29427 +29428 POINT(40.91679764178958 73.14340292706538) bank29428 +29429 POINT(40.68639529502433 74.78145756302531) bank29429 +29430 POINT(40.93635016169486 74.87577253376094) bank29430 +29431 POINT(40.05474526522767 73.16067208679155) bank29431 +29432 POINT(39.87988007186781 73.04844261992443) bank29432 +29433 POINT(40.41559440071537 73.22119181757694) bank29433 +29434 POINT(41.51339458556643 73.87466353078204) bank29434 +29435 POINT(39.739052639749865 73.05963091802573) bank29435 +29436 POINT(39.888315471162315 74.89300420329059) bank29436 +29437 POINT(40.495095067297925 74.87290934943047) bank29437 +29438 POINT(39.777100592472294 74.32108348151617) bank29438 +29439 POINT(41.53504818839993 74.45292165104516) bank29439 +29440 POINT(40.88956312145268 73.16212208357338) bank29440 +29441 POINT(40.54522095163882 73.74170871229315) bank29441 +29442 POINT(40.644002706584445 73.6039242348318) bank29442 +29443 POINT(39.8572315667668 74.44974271238604) bank29443 +29444 POINT(41.668870378023385 74.17758855239741) bank29444 +29445 POINT(40.91273312731732 74.34199685609289) bank29445 +29446 POINT(41.700758552681144 74.56469583869855) bank29446 +29447 POINT(40.84245967708873 74.59286352023027) bank29447 +29448 POINT(41.43870256046059 73.8211183490097) bank29448 +29449 POINT(40.745137987085336 74.11162504363674) bank29449 +29450 POINT(41.542534890029906 74.72971899256972) bank29450 +29451 POINT(40.45621524012207 73.79078429914878) bank29451 +29452 POINT(41.64571968642051 74.20561146463736) bank29452 +29453 POINT(40.137580500848465 73.64772461512786) bank29453 +29454 POINT(39.76715469853075 74.13340401682446) bank29454 +29455 POINT(41.24295719180961 74.04931927903618) bank29455 +29456 POINT(41.58766529731612 73.85409198269089) bank29456 +29457 POINT(40.707749049630706 73.27275569602948) bank29457 +29458 POINT(40.47735882583265 73.78439190684995) bank29458 +29459 POINT(40.015899681511165 73.40418305511857) bank29459 +29460 POINT(40.24063587352981 73.77442992744305) bank29460 +29461 POINT(40.31868505053737 74.1917148563834) bank29461 +29462 POINT(40.93586105441659 73.55761987295847) bank29462 +29463 POINT(39.84914911308588 74.5596395333807) bank29463 +29464 POINT(41.15423265935441 74.42876880126346) bank29464 +29465 POINT(41.46165160454996 73.78949106518877) bank29465 +29466 POINT(41.12662203165256 74.55267321147839) bank29466 +29467 POINT(40.48522041431876 74.29453549766842) bank29467 +29468 POINT(40.20679688776379 73.26208361147523) bank29468 +29469 POINT(39.88413117020178 74.282716379211) bank29469 +29470 POINT(41.59474943500547 74.83435590823288) bank29470 +29471 POINT(39.90681029328317 73.62033484067994) bank29471 +29472 POINT(40.51340761579667 74.76016763631154) bank29472 +29473 POINT(40.150140214700414 74.72064369105772) bank29473 +29474 POINT(41.53813894735458 74.14596846129918) bank29474 +29475 POINT(40.387945004236975 74.8718411635292) bank29475 +29476 POINT(40.06611944197176 73.22194078006034) bank29476 +29477 POINT(41.08896383999168 74.32299018164586) bank29477 +29478 POINT(41.0899609558836 73.85745441946634) bank29478 +29479 POINT(41.40347681023988 75.00098333650672) bank29479 +29480 POINT(40.428277093429365 73.99147927854385) bank29480 +29481 POINT(41.09982471604493 74.69561475665436) bank29481 +29482 POINT(40.34386108931812 74.34263094574462) bank29482 +29483 POINT(41.395910532727974 73.99177948909005) bank29483 +29484 POINT(41.165977520263745 74.15690707192388) bank29484 +29485 POINT(40.24412191493936 74.00922419635324) bank29485 +29486 POINT(40.86842239590589 73.47290609703096) bank29486 +29487 POINT(41.602964569525945 74.4720911572544) bank29487 +29488 POINT(40.3489991169962 74.93141747197747) bank29488 +29489 POINT(39.7643405321462 74.62244716963814) bank29489 +29490 POINT(41.62178666390703 73.33166708730485) bank29490 +29491 POINT(40.03028380434754 74.01561543337992) bank29491 +29492 POINT(40.09343635696607 74.36257825781166) bank29492 +29493 POINT(40.94153188996482 73.12585568893905) bank29493 +29494 POINT(40.51037264162756 73.33119769591497) bank29494 +29495 POINT(40.36448872086806 74.17533035674549) bank29495 +29496 POINT(40.418811361309466 74.1905865390765) bank29496 +29497 POINT(40.600415900638744 74.22757237581848) bank29497 +29498 POINT(39.93184477130409 73.6239876834543) bank29498 +29499 POINT(40.295968954519786 73.54062522729708) bank29499 +29500 POINT(40.70804616255617 73.60924744675837) bank29500 +29501 POINT(40.061581408173474 73.22070858000758) bank29501 +29502 POINT(40.3646007300678 74.70340203632509) bank29502 +29503 POINT(40.60983320704412 73.1601759394997) bank29503 +29504 POINT(39.937704913534155 73.08372599561926) bank29504 +29505 POINT(39.94424196581561 73.12732813616022) bank29505 +29506 POINT(40.62892233700947 74.97260750141184) bank29506 +29507 POINT(39.94419019615827 73.9844838342482) bank29507 +29508 POINT(40.69399521114984 74.6752367584009) bank29508 +29509 POINT(39.91872025698109 73.97690706917527) bank29509 +29510 POINT(41.427755898705854 73.66239956179258) bank29510 +29511 POINT(39.94665963312355 73.82788134322999) bank29511 +29512 POINT(41.622361653787 74.7426463084653) bank29512 +29513 POINT(40.22918590291789 74.86360490641738) bank29513 +29514 POINT(40.189542594289044 74.86063970866691) bank29514 +29515 POINT(40.57836700234214 74.7774660765535) bank29515 +29516 POINT(39.912418439917566 74.48048488145571) bank29516 +29517 POINT(40.73842538063141 73.49253133511375) bank29517 +29518 POINT(41.23500734388039 73.2432909261576) bank29518 +29519 POINT(41.18751825092579 74.56274122583986) bank29519 +29520 POINT(41.701850619421556 74.64658451649608) bank29520 +29521 POINT(39.7836052702759 73.30491523818189) bank29521 +29522 POINT(41.22895323534189 73.59952189452552) bank29522 +29523 POINT(40.67476727732853 73.33753775545058) bank29523 +29524 POINT(40.52679634070531 74.85776051116154) bank29524 +29525 POINT(40.14328733865206 73.47685462176919) bank29525 +29526 POINT(40.9455271003986 73.85450709724282) bank29526 +29527 POINT(40.583803643434436 73.90335040483914) bank29527 +29528 POINT(41.39449299495902 74.2943078679138) bank29528 +29529 POINT(41.33622490065163 73.53656886864118) bank29529 +29530 POINT(41.19134355615132 73.87399350134037) bank29530 +29531 POINT(40.40914846327206 73.67756769988695) bank29531 +29532 POINT(41.24162282667088 73.36121268205636) bank29532 +29533 POINT(41.091477766816226 73.65292730724398) bank29533 +29534 POINT(40.70392783901031 74.77586865512522) bank29534 +29535 POINT(39.75398765752113 74.00543156704383) bank29535 +29536 POINT(41.65678006633581 73.13010643160902) bank29536 +29537 POINT(41.18528898552398 74.48280752921852) bank29537 +29538 POINT(39.81780344298113 74.6158439372144) bank29538 +29539 POINT(40.75825427816835 74.18803179846539) bank29539 +29540 POINT(41.31740447178413 73.83420791069368) bank29540 +29541 POINT(41.2476557833073 74.15568682377295) bank29541 +29542 POINT(41.693187468054944 74.1303309893472) bank29542 +29543 POINT(41.61652455332053 74.15832428058152) bank29543 +29544 POINT(41.670546870480415 74.33955654997199) bank29544 +29545 POINT(40.88429227436064 73.41914144766253) bank29545 +29546 POINT(41.14441674003626 74.64171710983845) bank29546 +29547 POINT(40.45406017964111 73.4332255038684) bank29547 +29548 POINT(40.310434926644064 74.64242957856034) bank29548 +29549 POINT(40.74860747601193 74.68433915969942) bank29549 +29550 POINT(40.57541499711566 73.97993631737936) bank29550 +29551 POINT(41.14598870563603 74.53262590521233) bank29551 +29552 POINT(40.10196733123119 73.20885872350249) bank29552 +29553 POINT(41.27522338451363 73.44468640287009) bank29553 +29554 POINT(40.268943158013656 74.6209214870915) bank29554 +29555 POINT(41.231471947540676 74.93737583091249) bank29555 +29556 POINT(40.18276588797813 73.61940280052599) bank29556 +29557 POINT(40.541469466943454 73.86229528953274) bank29557 +29558 POINT(40.18649857634902 73.84381905749916) bank29558 +29559 POINT(40.97735957117139 74.89553352736533) bank29559 +29560 POINT(41.18065114325599 73.86449309768945) bank29560 +29561 POINT(39.73834263908298 73.17633975501168) bank29561 +29562 POINT(41.067078971145385 74.0795795675073) bank29562 +29563 POINT(40.80825963687097 74.43402535291112) bank29563 +29564 POINT(39.7244746960999 74.1396888812745) bank29564 +29565 POINT(41.29534253804764 74.63667808208186) bank29565 +29566 POINT(39.98684657201202 73.16542282509688) bank29566 +29567 POINT(40.341227910897246 73.49377632530442) bank29567 +29568 POINT(41.11668783224328 74.26502279070098) bank29568 +29569 POINT(40.94761236428454 74.79079525076979) bank29569 +29570 POINT(40.43230667259398 74.16202763056944) bank29570 +29571 POINT(41.6848229136384 73.85692135553418) bank29571 +29572 POINT(39.964255471000776 74.58208055907352) bank29572 +29573 POINT(40.0119736872336 74.65762309688152) bank29573 +29574 POINT(40.11208739714079 74.75772844555293) bank29574 +29575 POINT(40.31142009769388 74.48139511778328) bank29575 +29576 POINT(40.986971499548815 74.08905069480677) bank29576 +29577 POINT(40.38379411388502 74.01313097926905) bank29577 +29578 POINT(39.98214772956563 73.52258836249038) bank29578 +29579 POINT(40.37596207679482 73.34735737131824) bank29579 +29580 POINT(41.392408312390934 74.24959277762223) bank29580 +29581 POINT(41.56603503705439 74.91421952077259) bank29581 +29582 POINT(39.88967385808571 74.81335059333595) bank29582 +29583 POINT(40.85438908834764 74.87747287980051) bank29583 +29584 POINT(41.09591792565857 73.62409596702389) bank29584 +29585 POINT(40.088844175269536 73.52990046552556) bank29585 +29586 POINT(41.537706550357036 74.00320144519587) bank29586 +29587 POINT(40.21557773229229 75.00130818059992) bank29587 +29588 POINT(40.50318673396557 73.25342025241045) bank29588 +29589 POINT(39.85689951351393 74.03874770142548) bank29589 +29590 POINT(39.735454148826115 74.1853893483482) bank29590 +29591 POINT(39.74829760653593 73.3172786010633) bank29591 +29592 POINT(41.55065115519233 73.23602002108179) bank29592 +29593 POINT(40.73502028516908 74.89897775267389) bank29593 +29594 POINT(40.182082939793624 74.43838201499435) bank29594 +29595 POINT(41.00009216072317 74.11982550864458) bank29595 +29596 POINT(39.88468938854343 74.36124040601254) bank29596 +29597 POINT(40.95826912357309 73.21491331176895) bank29597 +29598 POINT(41.65708589580977 73.61615502387664) bank29598 +29599 POINT(40.60125137903053 74.39495720160488) bank29599 +29600 POINT(39.90867943264304 73.19334202919498) bank29600 +29601 POINT(39.71840891332097 74.26526987789919) bank29601 +29602 POINT(41.689116791450196 74.98700068429369) bank29602 +29603 POINT(40.76991184757708 73.58230198343261) bank29603 +29604 POINT(41.521882653891375 73.18481419862715) bank29604 +29605 POINT(40.52749246274669 73.29912607918864) bank29605 +29606 POINT(40.91177362821876 74.9028625603914) bank29606 +29607 POINT(40.56806963451891 74.44429500264214) bank29607 +29608 POINT(40.169579240573 74.0699792218782) bank29608 +29609 POINT(41.55409346337266 73.84767683986887) bank29609 +29610 POINT(40.17811494381777 73.77988921662816) bank29610 +29611 POINT(41.507630493568804 74.08170421067733) bank29611 +29612 POINT(40.44347888755745 74.18268734115546) bank29612 +29613 POINT(39.83008129419944 73.51641301390441) bank29613 +29614 POINT(39.860620487215996 73.05883591700201) bank29614 +29615 POINT(40.594329861055726 74.52517691404901) bank29615 +29616 POINT(39.87024487507565 73.59140493960031) bank29616 +29617 POINT(40.30741285691191 74.76906286107831) bank29617 +29618 POINT(39.82946181651838 73.2371541617126) bank29618 +29619 POINT(40.20042808487213 74.9761379534336) bank29619 +29620 POINT(39.71280931753697 74.47000567077559) bank29620 +29621 POINT(40.86448181262473 74.61140039168569) bank29621 +29622 POINT(40.106726710635535 73.96074698935249) bank29622 +29623 POINT(40.002528252764314 73.30085857691104) bank29623 +29624 POINT(41.253970149360015 73.01242604471315) bank29624 +29625 POINT(41.36494070388908 73.79582991441637) bank29625 +29626 POINT(40.66790036022598 73.46848771447806) bank29626 +29627 POINT(40.943111534873054 73.81445008669573) bank29627 +29628 POINT(39.903511054496754 73.0656035076553) bank29628 +29629 POINT(40.453362632494226 73.8573699549674) bank29629 +29630 POINT(40.930051968055814 73.81711300464474) bank29630 +29631 POINT(40.98333272900248 73.07457386982279) bank29631 +29632 POINT(41.26562350812914 74.15968005097598) bank29632 +29633 POINT(41.108763265442605 73.32017173404161) bank29633 +29634 POINT(40.63872360911987 73.80850937032149) bank29634 +29635 POINT(41.2747762579828 74.24577544427115) bank29635 +29636 POINT(39.7931367496425 74.91363087358556) bank29636 +29637 POINT(41.151350322955295 74.18027537830199) bank29637 +29638 POINT(39.83264912836024 74.41107963905097) bank29638 +29639 POINT(40.73651520399759 74.50847472579724) bank29639 +29640 POINT(40.62547767464047 74.64381027030356) bank29640 +29641 POINT(40.61921382420931 74.26047346199556) bank29641 +29642 POINT(40.69698720550421 74.02974936916172) bank29642 +29643 POINT(39.74603342524393 74.76508527456724) bank29643 +29644 POINT(40.44651742844501 73.54875416811089) bank29644 +29645 POINT(40.26105261884063 73.00895133919403) bank29645 +29646 POINT(40.79985228114669 74.1153045604754) bank29646 +29647 POINT(40.97894935451099 73.1088235745723) bank29647 +29648 POINT(41.202966800278034 73.51847558613602) bank29648 +29649 POINT(40.863550724725314 73.42865658949468) bank29649 +29650 POINT(40.274245905776404 73.15294019433972) bank29650 +29651 POINT(39.96409834404402 74.71299966156492) bank29651 +29652 POINT(40.1804005793808 74.23218208065393) bank29652 +29653 POINT(39.827595200811224 73.71916294841925) bank29653 +29654 POINT(41.00841288982227 74.84307070533168) bank29654 +29655 POINT(39.842124789306546 74.32571986117907) bank29655 +29656 POINT(40.19659774432791 74.82045117558854) bank29656 +29657 POINT(41.43147660328753 74.73051179792229) bank29657 +29658 POINT(40.0390747870676 74.19911332906233) bank29658 +29659 POINT(39.79284720337792 73.95906808405495) bank29659 +29660 POINT(40.43042799981709 74.64574630623149) bank29660 +29661 POINT(40.960097214735185 74.16075199843932) bank29661 +29662 POINT(39.83824811585718 73.84142721700341) bank29662 +29663 POINT(39.78013926121913 74.40027819978654) bank29663 +29664 POINT(40.63581133019608 73.77849949791263) bank29664 +29665 POINT(40.30441646394984 74.68919078146934) bank29665 +29666 POINT(40.52630508983773 73.63302157583144) bank29666 +29667 POINT(40.639123122623346 73.15341941732814) bank29667 +29668 POINT(39.7352930711226 73.58411446353425) bank29668 +29669 POINT(41.18103894016091 73.78105234322086) bank29669 +29670 POINT(40.730129338912604 74.68384416166846) bank29670 +29671 POINT(40.23449660700451 73.75648110360117) bank29671 +29672 POINT(39.78688723409678 73.31085092016754) bank29672 +29673 POINT(39.84317152336333 74.01943185856523) bank29673 +29674 POINT(40.59307273721999 74.9362348493181) bank29674 +29675 POINT(41.15552777018453 73.97092665918662) bank29675 +29676 POINT(40.225987564777334 74.8473832666504) bank29676 +29677 POINT(40.049825485825124 73.87234531476066) bank29677 +29678 POINT(40.46465906095081 73.35903814812275) bank29678 +29679 POINT(39.87260797945417 73.44720840999031) bank29679 +29680 POINT(40.21758318779705 74.19736145907895) bank29680 +29681 POINT(40.52612852225357 74.33571129049953) bank29681 +29682 POINT(40.24849430117112 73.37173970165124) bank29682 +29683 POINT(39.92938888128964 73.2235686836422) bank29683 +29684 POINT(40.18850654869016 73.33214766214849) bank29684 +29685 POINT(41.12186285994334 74.7260245728636) bank29685 +29686 POINT(40.73335267033588 74.99578926373384) bank29686 +29687 POINT(40.24949836256226 73.01188486182221) bank29687 +29688 POINT(40.31769867821959 73.98377983417424) bank29688 +29689 POINT(40.634393603574814 74.52343864786522) bank29689 +29690 POINT(41.38209936694867 73.16522463535988) bank29690 +29691 POINT(39.99410839900568 74.73591851199001) bank29691 +29692 POINT(40.02225856838932 73.42189178937775) bank29692 +29693 POINT(40.79122617273784 73.81908923266454) bank29693 +29694 POINT(41.12729786603656 73.49412770684683) bank29694 +29695 POINT(41.23844618525236 73.13605957552949) bank29695 +29696 POINT(40.37149480551511 74.37305277978619) bank29696 +29697 POINT(40.0208159846076 74.48416940504038) bank29697 +29698 POINT(41.67352832120957 74.4753157487504) bank29698 +29699 POINT(40.6843284228695 73.082803179044) bank29699 +29700 POINT(41.335484719617575 73.11551530670715) bank29700 +29701 POINT(41.37882682263008 74.59892676393798) bank29701 +29702 POINT(40.34087225444005 74.4054784261205) bank29702 +29703 POINT(41.29396931290782 74.75971028878674) bank29703 +29704 POINT(40.31646271102269 73.37013347546767) bank29704 +29705 POINT(41.456080838778725 73.27458216800699) bank29705 +29706 POINT(40.02854311391377 74.81448604021423) bank29706 +29707 POINT(40.15656684202982 74.66062572004496) bank29707 +29708 POINT(40.5227335108592 74.92081564655709) bank29708 +29709 POINT(40.466405236374825 73.85397684813425) bank29709 +29710 POINT(41.10101872484091 74.8140032518369) bank29710 +29711 POINT(39.96147482663788 74.96638123888745) bank29711 +29712 POINT(39.880016398873124 74.7003266222598) bank29712 +29713 POINT(40.87246361725046 74.00385342209356) bank29713 +29714 POINT(40.66259396401836 74.19467111910613) bank29714 +29715 POINT(40.54850317883319 74.22420268155447) bank29715 +29716 POINT(40.4285337261206 73.18819777666076) bank29716 +29717 POINT(40.78999058874912 74.28852231795565) bank29717 +29718 POINT(39.86503331746246 73.42975012978086) bank29718 +29719 POINT(41.010165244141504 73.17593786965149) bank29719 +29720 POINT(40.28299261309725 73.01143908410089) bank29720 +29721 POINT(41.39557367806471 73.89281165064565) bank29721 +29722 POINT(39.855465535256386 73.88414238906631) bank29722 +29723 POINT(40.46415220594649 74.57230127771531) bank29723 +29724 POINT(41.619289154018446 74.40697004179202) bank29724 +29725 POINT(41.59106034457143 73.05127769323798) bank29725 +29726 POINT(41.68711726689426 73.36260563533398) bank29726 +29727 POINT(41.35539555908183 73.20962404682564) bank29727 +29728 POINT(40.6247758755687 74.38522968046286) bank29728 +29729 POINT(39.87169418874221 74.4162481800615) bank29729 +29730 POINT(40.01496546540652 74.93784999939986) bank29730 +29731 POINT(41.67800868998983 74.86677606123898) bank29731 +29732 POINT(39.7402063007122 73.87953435846745) bank29732 +29733 POINT(40.86998670713511 74.75560372534302) bank29733 +29734 POINT(39.87595663698429 74.1769564953759) bank29734 +29735 POINT(41.32071606087466 74.22819410877803) bank29735 +29736 POINT(39.88203204868223 74.9680895692923) bank29736 +29737 POINT(41.28747691107103 74.78990670307319) bank29737 +29738 POINT(40.35506734405617 73.2920406936854) bank29738 +29739 POINT(41.60216180625051 73.27608092426014) bank29739 +29740 POINT(40.69317032102974 73.9669604320072) bank29740 +29741 POINT(40.51394877089725 74.62654007444182) bank29741 +29742 POINT(40.104209971739145 74.78284176542046) bank29742 +29743 POINT(41.221087870236 73.05572729367175) bank29743 +29744 POINT(40.238867508124535 74.4499722352726) bank29744 +29745 POINT(40.048777104992865 74.55645543599634) bank29745 +29746 POINT(39.793968930427155 74.52082836228938) bank29746 +29747 POINT(40.18241375985388 73.51130035895044) bank29747 +29748 POINT(41.236452396654485 73.48565806333235) bank29748 +29749 POINT(40.450484518646654 74.39545310150443) bank29749 +29750 POINT(39.770949287644136 73.73207509527177) bank29750 +29751 POINT(40.24266010356179 74.16369569642858) bank29751 +29752 POINT(40.44138231315881 74.76264383862079) bank29752 +29753 POINT(41.65325732175419 73.20608848096786) bank29753 +29754 POINT(41.35845115019685 73.0073115192595) bank29754 +29755 POINT(41.499028530616286 74.87522684354661) bank29755 +29756 POINT(41.27587568564159 74.52488484881077) bank29756 +29757 POINT(41.09330842501799 74.25617072895622) bank29757 +29758 POINT(40.93647884899408 74.80198086412004) bank29758 +29759 POINT(40.76105923290892 74.9228728375878) bank29759 +29760 POINT(39.81667105392206 73.9248442089329) bank29760 +29761 POINT(41.42009645943615 73.85870179974637) bank29761 +29762 POINT(40.83532400813807 74.43394355304548) bank29762 +29763 POINT(41.01486617205577 74.05328425157467) bank29763 +29764 POINT(40.84609072778241 74.67083846470798) bank29764 +29765 POINT(41.48664622731117 73.05300675744704) bank29765 +29766 POINT(39.957829211166704 74.99906319937621) bank29766 +29767 POINT(40.75372726911249 74.68301034780559) bank29767 +29768 POINT(40.42851520531245 73.46215141906666) bank29768 +29769 POINT(39.96971787130409 73.0093053764355) bank29769 +29770 POINT(40.22817491077576 73.69397500410138) bank29770 +29771 POINT(40.08585019569459 74.73485559703872) bank29771 +29772 POINT(40.44271696251836 74.43380032685677) bank29772 +29773 POINT(41.24035495895866 74.77010846457854) bank29773 +29774 POINT(40.44710675383594 74.84033583675385) bank29774 +29775 POINT(41.217957931073826 73.5611991494354) bank29775 +29776 POINT(40.448956090702836 74.88226422532557) bank29776 +29777 POINT(41.18440305927027 73.97515167752562) bank29777 +29778 POINT(41.3170143094895 73.12524728867615) bank29778 +29779 POINT(40.36613044239624 73.03283086397099) bank29779 +29780 POINT(41.09626979517143 73.99659429921277) bank29780 +29781 POINT(40.54728539801895 73.35959689710904) bank29781 +29782 POINT(39.989702952625265 74.313588070031) bank29782 +29783 POINT(40.78546798652255 73.7253186252978) bank29783 +29784 POINT(40.024287828450134 74.75945389722406) bank29784 +29785 POINT(41.11811651090508 74.00682868049807) bank29785 +29786 POINT(41.028930787657664 73.78600249547692) bank29786 +29787 POINT(40.1046368435959 74.4476209854304) bank29787 +29788 POINT(41.0480444869737 74.20898818960971) bank29788 +29789 POINT(40.94494736728253 74.70994611612288) bank29789 +29790 POINT(39.87250928573071 74.76650184908368) bank29790 +29791 POINT(41.23943388841866 73.37320609152037) bank29791 +29792 POINT(39.97446570105358 73.97140264855419) bank29792 +29793 POINT(40.67450999738106 74.20148641922977) bank29793 +29794 POINT(40.76650324515349 73.13319553414671) bank29794 +29795 POINT(40.303110281931374 74.74426249840542) bank29795 +29796 POINT(41.02348538977521 74.92422929677052) bank29796 +29797 POINT(40.87415768446533 73.20268432864779) bank29797 +29798 POINT(40.155587342783726 74.79121345720276) bank29798 +29799 POINT(41.219098449349836 74.76879691172107) bank29799 +29800 POINT(40.851170920347094 74.63967474924094) bank29800 +29801 POINT(40.24626076481007 73.18483710678856) bank29801 +29802 POINT(39.924512581743 74.00486568801006) bank29802 +29803 POINT(40.955456936214816 73.89143138756754) bank29803 +29804 POINT(40.334121080131844 74.01880217695533) bank29804 +29805 POINT(41.39624006545843 73.1753163292378) bank29805 +29806 POINT(40.609764597257346 74.21733896665111) bank29806 +29807 POINT(39.925245314786416 74.0344391157572) bank29807 +29808 POINT(40.014675203665256 73.12739107020197) bank29808 +29809 POINT(40.094004376804854 73.22403239714384) bank29809 +29810 POINT(41.243412529253455 74.91722188730073) bank29810 +29811 POINT(40.587491756608934 73.60197331372525) bank29811 +29812 POINT(40.524574604372575 74.58833081803385) bank29812 +29813 POINT(40.017672136160726 74.40759476179281) bank29813 +29814 POINT(40.05487407013439 74.57339678563443) bank29814 +29815 POINT(40.104331543949094 73.7490224278966) bank29815 +29816 POINT(40.80923482954488 73.44583597711282) bank29816 +29817 POINT(39.91600163139224 74.48217960694735) bank29817 +29818 POINT(40.33014224306579 73.28718085269684) bank29818 +29819 POINT(40.13448576662957 73.30790722123956) bank29819 +29820 POINT(40.79224733731223 74.40245508275649) bank29820 +29821 POINT(41.1385324677221 74.63728555350058) bank29821 +29822 POINT(40.10529284241795 73.9506159049242) bank29822 +29823 POINT(40.60160935374066 74.00630583600439) bank29823 +29824 POINT(39.87593325114793 74.3031139183485) bank29824 +29825 POINT(40.811034515426954 73.90139760657993) bank29825 +29826 POINT(40.672123978409005 74.29203834342253) bank29826 +29827 POINT(40.07547050731343 73.23050354904251) bank29827 +29828 POINT(39.937270525605456 74.84768499141182) bank29828 +29829 POINT(41.663933183055164 73.72241600185045) bank29829 +29830 POINT(41.462137507444886 74.86822257338915) bank29830 +29831 POINT(41.26978060361427 73.30104025427569) bank29831 +29832 POINT(41.526451319091834 73.96018342658867) bank29832 +29833 POINT(41.12551237124841 74.02761884247425) bank29833 +29834 POINT(40.185511730290884 73.89539297617594) bank29834 +29835 POINT(40.09532208079796 74.6767580900168) bank29835 +29836 POINT(40.57538463351789 73.83651601356554) bank29836 +29837 POINT(40.766072437984086 73.80135471011255) bank29837 +29838 POINT(41.60166330239358 73.56025785297614) bank29838 +29839 POINT(40.68257996062392 73.7817795566956) bank29839 +29840 POINT(41.07492821192493 73.09096089598656) bank29840 +29841 POINT(40.57397363432683 73.92788488295412) bank29841 +29842 POINT(41.244854251407325 73.1788064242821) bank29842 +29843 POINT(40.00106328585132 74.20713833264377) bank29843 +29844 POINT(41.6275786423048 74.03272074867043) bank29844 +29845 POINT(41.19467443235285 73.32045664418139) bank29845 +29846 POINT(40.872255022305914 74.15026598740933) bank29846 +29847 POINT(40.51341075588092 73.72498176638435) bank29847 +29848 POINT(40.189178300866715 73.29194122600028) bank29848 +29849 POINT(40.63734340309697 73.12311489303431) bank29849 +29850 POINT(40.78737032809997 74.27341938203557) bank29850 +29851 POINT(41.13045590130357 73.59899439931145) bank29851 +29852 POINT(41.477592227921534 73.72913322562486) bank29852 +29853 POINT(41.22744385613738 74.48064616986366) bank29853 +29854 POINT(40.700206789172945 73.18459828359342) bank29854 +29855 POINT(40.96126709471128 74.0620462377871) bank29855 +29856 POINT(39.89235397126738 74.76386134767436) bank29856 +29857 POINT(40.550279031694735 73.85949657710839) bank29857 +29858 POINT(40.72665633111872 73.48006608748267) bank29858 +29859 POINT(39.86562176183249 74.07848273999241) bank29859 +29860 POINT(39.801526465746164 73.36415722778686) bank29860 +29861 POINT(39.78657835529813 74.4513233615417) bank29861 +29862 POINT(40.30130318832027 74.34085375108646) bank29862 +29863 POINT(40.24493365082286 74.42016349064183) bank29863 +29864 POINT(40.421349815136054 74.0005753638892) bank29864 +29865 POINT(41.07713165978543 73.19142034088067) bank29865 +29866 POINT(41.27287611514027 73.24443121802081) bank29866 +29867 POINT(39.89052664361892 73.71629398716591) bank29867 +29868 POINT(40.56792708214525 73.88783358787026) bank29868 +29869 POINT(40.59866341761948 73.71827118595363) bank29869 +29870 POINT(40.455102357022774 73.92139499118743) bank29870 +29871 POINT(41.65736045850789 74.16824357900585) bank29871 +29872 POINT(41.03061551812258 73.20868410389127) bank29872 +29873 POINT(40.66515467453543 73.68315981554511) bank29873 +29874 POINT(39.92780095315602 73.21336065397782) bank29874 +29875 POINT(40.762857652401735 74.1318968605898) bank29875 +29876 POINT(40.51459956406572 73.67788574151116) bank29876 +29877 POINT(41.62102546444449 73.11361734120054) bank29877 +29878 POINT(41.6807886128505 74.57609848901348) bank29878 +29879 POINT(39.91115097461962 74.8946500270973) bank29879 +29880 POINT(40.84132053443351 73.61459628935818) bank29880 +29881 POINT(40.779673376149596 73.96557770141693) bank29881 +29882 POINT(39.88154005975465 73.60859921698811) bank29882 +29883 POINT(40.25273072964634 74.32212694643597) bank29883 +29884 POINT(41.16788304732253 73.75364393199683) bank29884 +29885 POINT(40.25862410689479 73.76382364644576) bank29885 +29886 POINT(40.565605403693695 73.35921795980711) bank29886 +29887 POINT(40.17598955912825 73.59320668832144) bank29887 +29888 POINT(41.564768672746126 73.92537192825729) bank29888 +29889 POINT(41.18458933314194 73.65916639106737) bank29889 +29890 POINT(41.08438220273631 73.90658904024023) bank29890 +29891 POINT(40.819793101600666 73.44401406987765) bank29891 +29892 POINT(40.347149899732685 74.8458978280559) bank29892 +29893 POINT(40.347233557829504 73.450335977765) bank29893 +29894 POINT(41.57176111818046 73.80026946374504) bank29894 +29895 POINT(41.25898951516834 74.41526247320854) bank29895 +29896 POINT(39.78540163693263 74.17032946238537) bank29896 +29897 POINT(40.196166661570885 74.78163591381859) bank29897 +29898 POINT(40.36433746841952 74.03578297045546) bank29898 +29899 POINT(40.62108844477014 74.01629844622231) bank29899 +29900 POINT(40.83416642654181 73.49096490095047) bank29900 +29901 POINT(41.30857446093803 73.57668456346182) bank29901 +29902 POINT(41.1415273091956 73.05216342906095) bank29902 +29903 POINT(41.50566506658324 74.23052867774001) bank29903 +29904 POINT(39.92392036010623 74.76399968629974) bank29904 +29905 POINT(41.27695513926485 73.6401479525847) bank29905 +29906 POINT(39.84231112575558 73.00961518308122) bank29906 +29907 POINT(40.98094520329535 73.84983644548292) bank29907 +29908 POINT(40.270135253734395 74.9110859266002) bank29908 +29909 POINT(41.62527072416509 74.98688958936893) bank29909 +29910 POINT(41.48473163904963 73.09023035638553) bank29910 +29911 POINT(40.189985234234186 73.13169988537209) bank29911 +29912 POINT(41.5031337432334 74.12602580573973) bank29912 +29913 POINT(41.11891826644037 74.47649755181205) bank29913 +29914 POINT(39.896497990411895 74.27699077506105) bank29914 +29915 POINT(40.778062959666286 73.06712317241458) bank29915 +29916 POINT(41.41684791220874 74.25806367882474) bank29916 +29917 POINT(39.967035986972526 73.67586246164036) bank29917 +29918 POINT(40.36286057838142 73.61910791543029) bank29918 +29919 POINT(40.296078813762044 74.60074787057943) bank29919 +29920 POINT(40.35600389089927 74.20249491763003) bank29920 +29921 POINT(40.86900844524346 74.31359154946564) bank29921 +29922 POINT(40.996138333437 73.74737883617851) bank29922 +29923 POINT(40.80961564832215 73.3301453704925) bank29923 +29924 POINT(40.73532306584727 73.7331029212693) bank29924 +29925 POINT(41.67901678387448 74.73024893605921) bank29925 +29926 POINT(39.886259136577486 73.89461758056439) bank29926 +29927 POINT(39.93215382290284 74.39252565885666) bank29927 +29928 POINT(40.78513674018688 74.60005731487136) bank29928 +29929 POINT(41.44047569301331 74.2106842305326) bank29929 +29930 POINT(41.19961558495781 74.69901099795776) bank29930 +29931 POINT(41.46201388676007 73.86944275886628) bank29931 +29932 POINT(40.21475419225692 73.51081510542366) bank29932 +29933 POINT(39.88965975582145 73.41912795660116) bank29933 +29934 POINT(40.43319631151493 74.1238154240722) bank29934 +29935 POINT(41.46001766260933 74.79567340292978) bank29935 +29936 POINT(40.838356747069255 73.33788041132858) bank29936 +29937 POINT(40.536336475238606 73.31959084147981) bank29937 +29938 POINT(40.777321126383825 74.92436774906514) bank29938 +29939 POINT(39.85066976725639 74.92194883384366) bank29939 +29940 POINT(41.35606291760408 73.87151086701223) bank29940 +29941 POINT(40.097228170333366 74.52974038512187) bank29941 +29942 POINT(39.79874109822904 73.66093429273465) bank29942 +29943 POINT(40.29654210310173 74.52314737902155) bank29943 +29944 POINT(40.25609886456989 73.1304623424909) bank29944 +29945 POINT(40.16722233392681 74.68005953300032) bank29945 +29946 POINT(40.125368175076275 73.55859057734976) bank29946 +29947 POINT(40.916553425080984 73.06441867533836) bank29947 +29948 POINT(41.46244478638022 74.61305612446095) bank29948 +29949 POINT(41.141497056153206 73.1709755108851) bank29949 +29950 POINT(41.47368759581886 74.15610658328185) bank29950 +29951 POINT(40.16251135361054 73.98196961200088) bank29951 +29952 POINT(40.896336613498526 73.09580259250234) bank29952 +29953 POINT(41.40000738559558 73.55773952320295) bank29953 +29954 POINT(41.01163798305354 73.81661704691392) bank29954 +29955 POINT(40.36676066879442 73.45826783902544) bank29955 +29956 POINT(40.12470893786665 74.19105042236325) bank29956 +29957 POINT(41.516834655279155 73.04096422999008) bank29957 +29958 POINT(41.577963235216835 73.9341616072517) bank29958 +29959 POINT(39.896115261228125 73.14365008012194) bank29959 +29960 POINT(40.78809136812316 73.72284631604039) bank29960 +29961 POINT(41.49699839232867 74.57502652315586) bank29961 +29962 POINT(41.23416886420522 73.49600594281272) bank29962 +29963 POINT(41.46260164161215 73.90380090033291) bank29963 +29964 POINT(40.8164291141278 73.4237849945401) bank29964 +29965 POINT(39.86005063188757 73.71704825260285) bank29965 +29966 POINT(41.34190563627719 73.90414260711907) bank29966 +29967 POINT(40.60388089909112 74.0010279252353) bank29967 +29968 POINT(40.44519600452616 74.88552066084385) bank29968 +29969 POINT(39.8886382519236 74.83486898482492) bank29969 +29970 POINT(41.71046247982556 75.00303217293073) bank29970 +29971 POINT(40.93077538380965 74.78658025876219) bank29971 +29972 POINT(40.90344647357462 73.9066182570869) bank29972 +29973 POINT(40.10329839457847 74.83994455926734) bank29973 +29974 POINT(40.39791394795368 74.79740647939093) bank29974 +29975 POINT(41.390798149484475 74.3383863385243) bank29975 +29976 POINT(40.61287381930495 73.97184084465414) bank29976 +29977 POINT(39.92150239765921 74.01935369763201) bank29977 +29978 POINT(39.94443992031584 74.13503702535739) bank29978 +29979 POINT(40.78235238587741 74.42740217755966) bank29979 +29980 POINT(40.97324917222976 73.9922035316451) bank29980 +29981 POINT(41.15496788383235 73.80691603342083) bank29981 +29982 POINT(40.77030936189441 73.99806706402272) bank29982 +29983 POINT(41.05249797325588 74.42776057691462) bank29983 +29984 POINT(40.35524483510496 73.93463093435709) bank29984 +29985 POINT(40.852820644180966 74.73603731413567) bank29985 +29986 POINT(39.930882874151514 74.5198664870732) bank29986 +29987 POINT(40.298983394503814 73.33027673684913) bank29987 +29988 POINT(41.50102181599204 73.70722013985169) bank29988 +29989 POINT(40.71157528569512 74.76470818522762) bank29989 +29990 POINT(40.5163612884489 74.46771064012047) bank29990 +29991 POINT(39.908028865636524 73.9011132475131) bank29991 +29992 POINT(41.13408877991365 73.76331981214241) bank29992 +29993 POINT(40.60470147748315 74.17002889649747) bank29993 +29994 POINT(40.273708182576804 73.79006978220205) bank29994 +29995 POINT(40.54757487308042 74.54354724832862) bank29995 +29996 POINT(40.41473229404265 74.39479691872204) bank29996 +29997 POINT(41.51171079901774 74.27826177248583) bank29997 +29998 POINT(41.329045509448065 73.98971786197575) bank29998 +29999 POINT(40.87427032643869 73.797793687616) bank29999 +30000 POINT(41.3454203088953 74.60325963984944) bank30000 +30001 POINT(41.537230543507114 74.6847002946762) bank30001 +30002 POINT(40.264928420906 74.42793544986317) bank30002 +30003 POINT(40.55964946475587 73.25145158905356) bank30003 +30004 POINT(40.81060411608431 74.357189942587) bank30004 +30005 POINT(41.3862660298047 74.65762918537449) bank30005 +30006 POINT(39.99824201312422 74.28691293874763) bank30006 +30007 POINT(41.331058639807345 73.84941526532386) bank30007 +30008 POINT(40.745996540639815 73.33152127565025) bank30008 +30009 POINT(41.177738540868056 74.17702471620579) bank30009 +30010 POINT(41.47622214112246 73.85969093751802) bank30010 +30011 POINT(40.31990212847062 73.42520087896278) bank30011 +30012 POINT(39.727712699110384 74.52102657019245) bank30012 +30013 POINT(39.98092211677168 74.75664347095187) bank30013 +30014 POINT(41.15165581372028 74.65734219390346) bank30014 +30015 POINT(41.092704170092276 73.86819803691003) bank30015 +30016 POINT(40.44146095119658 73.05923360298948) bank30016 +30017 POINT(40.030336895036484 74.83391371090964) bank30017 +30018 POINT(40.85011026138456 74.264483496488) bank30018 +30019 POINT(40.11229538121714 73.48236866037497) bank30019 +30020 POINT(40.51667984298403 74.98724106101213) bank30020 +30021 POINT(40.28578937843358 74.17405910849808) bank30021 +30022 POINT(41.692831151166175 73.45400862668525) bank30022 +30023 POINT(41.249182380610215 74.06903373483973) bank30023 +30024 POINT(40.33129175787207 73.01971928346754) bank30024 +30025 POINT(39.72389672273324 74.93762619676515) bank30025 +30026 POINT(41.138612944406816 74.74418925923895) bank30026 +30027 POINT(40.267539470294594 73.06673680955751) bank30027 +30028 POINT(40.6096573607268 74.3862264919019) bank30028 +30029 POINT(41.40305881071314 74.47271041147772) bank30029 +30030 POINT(40.849094525552495 73.5815633649645) bank30030 +30031 POINT(40.066473172597895 73.96337148122414) bank30031 +30032 POINT(39.97883127113068 74.81802870353486) bank30032 +30033 POINT(40.08636863448409 73.65615271662466) bank30033 +30034 POINT(40.91393454095918 73.25832089527714) bank30034 +30035 POINT(40.453351358884404 73.80477994575038) bank30035 +30036 POINT(40.3665514179641 73.67036702875785) bank30036 +30037 POINT(39.914566621623116 73.56150931323043) bank30037 +30038 POINT(41.669833956356364 74.54408083461097) bank30038 +30039 POINT(39.82369416227653 74.56376191679871) bank30039 +30040 POINT(41.31704449616698 74.81761584749775) bank30040 +30041 POINT(40.27604005774061 73.86918842700443) bank30041 +30042 POINT(40.622354070831534 73.96938275681288) bank30042 +30043 POINT(40.71460513651563 74.12833594911878) bank30043 +30044 POINT(39.99383100947954 73.7532361106539) bank30044 +30045 POINT(39.99368527278741 73.3069351408193) bank30045 +30046 POINT(40.039456686590206 74.82205990242612) bank30046 +30047 POINT(41.1446221845833 73.35556233652122) bank30047 +30048 POINT(39.862995901148295 73.57932291601787) bank30048 +30049 POINT(40.16199416455791 73.65597822171546) bank30049 +30050 POINT(41.45602544659812 74.78467685859043) bank30050 +30051 POINT(41.04643686541416 73.74025201881366) bank30051 +30052 POINT(40.536066751408455 73.43815054336281) bank30052 +30053 POINT(41.2824782216525 74.19174566816854) bank30053 +30054 POINT(41.35453375355451 74.93751046541001) bank30054 +30055 POINT(40.48861196235618 73.31742602221583) bank30055 +30056 POINT(41.56194559738272 74.11050601848952) bank30056 +30057 POINT(40.30801041158344 73.182107432049) bank30057 +30058 POINT(41.23309871210888 73.79029645703774) bank30058 +30059 POINT(40.84960221673523 73.51230996935429) bank30059 +30060 POINT(41.678198142143366 73.78547962489469) bank30060 +30061 POINT(39.87717276149988 74.09054225905354) bank30061 +30062 POINT(40.462690909697265 73.44785375217283) bank30062 +30063 POINT(39.82743641361126 74.79428738712154) bank30063 +30064 POINT(39.86636912669647 73.45364312811819) bank30064 +30065 POINT(41.591444733536065 73.17546936490558) bank30065 +30066 POINT(39.90974732121808 74.06604719735186) bank30066 +30067 POINT(40.61715083038085 73.8282442655655) bank30067 +30068 POINT(41.25217744743511 73.20907490894257) bank30068 +30069 POINT(41.204931690280326 73.27536634289157) bank30069 +30070 POINT(40.19663294516173 73.52765784366193) bank30070 +30071 POINT(41.081926913586294 73.53250134707483) bank30071 +30072 POINT(41.174078498843336 73.21923872919184) bank30072 +30073 POINT(40.55030278579731 73.09218078426098) bank30073 +30074 POINT(41.08367317518577 73.57566317944587) bank30074 +30075 POINT(39.970831594640856 73.63046471359438) bank30075 +30076 POINT(41.35835757339755 73.06746623869458) bank30076 +30077 POINT(40.50611952519343 73.97617225634548) bank30077 +30078 POINT(41.66184817743405 74.40706678085701) bank30078 +30079 POINT(41.157304815770125 74.93026044975343) bank30079 +30080 POINT(40.14089327561932 74.69566381969572) bank30080 +30081 POINT(41.58652541448414 74.69909693131531) bank30081 +30082 POINT(40.5026836677492 74.79668456823913) bank30082 +30083 POINT(40.98167912124218 74.83879531720096) bank30083 +30084 POINT(41.27386829334339 73.96519924068252) bank30084 +30085 POINT(40.76347679850556 74.61372347691713) bank30085 +30086 POINT(40.13556334856545 74.60375261742762) bank30086 +30087 POINT(40.32686624322897 74.6147544710188) bank30087 +30088 POINT(41.356240417138004 74.89933771413135) bank30088 +30089 POINT(39.769739219975506 74.44874401345898) bank30089 +30090 POINT(41.70340424705037 74.7233425990029) bank30090 +30091 POINT(40.23210144222266 73.46774910817997) bank30091 +30092 POINT(40.831151223157065 74.18418753254433) bank30092 +30093 POINT(41.58193260233203 73.50087220143544) bank30093 +30094 POINT(40.97902034961824 73.77294014125415) bank30094 +30095 POINT(40.12140804937778 73.68387094050075) bank30095 +30096 POINT(40.023111418314755 73.45739606466663) bank30096 +30097 POINT(41.48496528591863 73.82027265447185) bank30097 +30098 POINT(39.858793859162475 74.31530676403793) bank30098 +30099 POINT(40.359076022640004 74.1517307973616) bank30099 +30100 POINT(40.1054739171656 74.71893273258776) bank30100 +30101 POINT(41.45333898854445 73.07015658297944) bank30101 +30102 POINT(41.558263638447336 73.62510834835345) bank30102 +30103 POINT(40.74632947170137 74.07434131313659) bank30103 +30104 POINT(40.22438689595525 74.93445142013888) bank30104 +30105 POINT(40.39626130960842 74.82223635512752) bank30105 +30106 POINT(40.56489653664417 74.76713834896904) bank30106 +30107 POINT(40.91411408965614 74.81707475013864) bank30107 +30108 POINT(40.54379448772407 73.87884761927141) bank30108 +30109 POINT(40.82955153943685 73.40336082722456) bank30109 +30110 POINT(40.70150932007775 73.79832957511675) bank30110 +30111 POINT(40.04870241357295 74.90509769723754) bank30111 +30112 POINT(41.22833443105214 73.40950090341067) bank30112 +30113 POINT(39.92187551310083 74.02053084626515) bank30113 +30114 POINT(41.139088972232294 74.42103829192276) bank30114 +30115 POINT(40.66378835150763 73.61712798604778) bank30115 +30116 POINT(39.82012119077899 74.18723707437148) bank30116 +30117 POINT(41.421290162641235 74.0877180240086) bank30117 +30118 POINT(40.208784611855464 73.02161013347619) bank30118 +30119 POINT(39.888849999321664 74.98172664976998) bank30119 +30120 POINT(41.011417248253835 74.28748993328884) bank30120 +30121 POINT(39.99213714338229 73.9054045858224) bank30121 +30122 POINT(40.336719039162766 74.93599035316413) bank30122 +30123 POINT(39.72888156499737 73.66935174216982) bank30123 +30124 POINT(40.21652360902124 73.37442580407685) bank30124 +30125 POINT(40.56674364283878 73.09715473904313) bank30125 +30126 POINT(41.569921777019836 74.52976695260307) bank30126 +30127 POINT(39.87352464448686 74.35287867150227) bank30127 +30128 POINT(40.41268123880936 73.56665239250744) bank30128 +30129 POINT(40.08623206959559 73.57545176576689) bank30129 +30130 POINT(40.091954489297656 73.67930856528997) bank30130 +30131 POINT(40.525659626039726 73.69346339957985) bank30131 +30132 POINT(40.02970228152498 74.76053329411084) bank30132 +30133 POINT(41.63280262219558 73.60684778936589) bank30133 +30134 POINT(40.78423635753413 73.0727881283585) bank30134 +30135 POINT(40.77602872888275 73.15458707570654) bank30135 +30136 POINT(40.48509756085911 74.02289520285458) bank30136 +30137 POINT(41.64814548475131 74.91064995029313) bank30137 +30138 POINT(41.2290456189354 74.73392859534798) bank30138 +30139 POINT(40.267305406362006 73.73999014291456) bank30139 +30140 POINT(40.09841474975339 74.16893919301597) bank30140 +30141 POINT(40.62034975888814 74.11239492406922) bank30141 +30142 POINT(41.27942020382982 74.49515720544979) bank30142 +30143 POINT(40.75828843569847 74.47564994622145) bank30143 +30144 POINT(41.66298815831993 73.97060792568973) bank30144 +30145 POINT(41.3097266683171 74.08326544548865) bank30145 +30146 POINT(40.9577848959307 74.51351938714288) bank30146 +30147 POINT(40.202278033654835 74.60590154881402) bank30147 +30148 POINT(41.372863463002204 74.62141562066564) bank30148 +30149 POINT(40.00508623226641 73.62427756604112) bank30149 +30150 POINT(39.8319263517909 73.76158841481237) bank30150 +30151 POINT(40.68734131101293 73.71843668760759) bank30151 +30152 POINT(41.6653420849008 74.29780178400952) bank30152 +30153 POINT(41.30465562780834 74.78147175296662) bank30153 +30154 POINT(40.19309560553432 73.99346343548821) bank30154 +30155 POINT(40.93692745768482 73.09040525194547) bank30155 +30156 POINT(41.290693747536395 73.65543737606663) bank30156 +30157 POINT(40.055475801313854 74.30861638928425) bank30157 +30158 POINT(41.0961331938741 73.37959211850907) bank30158 +30159 POINT(39.84568568432101 74.58509879906954) bank30159 +30160 POINT(40.71193689746127 73.73650128219963) bank30160 +30161 POINT(40.33279875499611 73.75953053331084) bank30161 +30162 POINT(40.246043403315696 74.02833821409395) bank30162 +30163 POINT(40.5715762395435 74.83869298470258) bank30163 +30164 POINT(40.805286250834826 74.32727062562837) bank30164 +30165 POINT(41.35725563326316 73.05721204555664) bank30165 +30166 POINT(41.273897769191784 74.63156533443104) bank30166 +30167 POINT(40.90095579239417 74.72664816056822) bank30167 +30168 POINT(39.76555568927059 74.08256468568926) bank30168 +30169 POINT(40.843054797149094 74.05216917294645) bank30169 +30170 POINT(40.80330369523732 73.20438693051031) bank30170 +30171 POINT(40.885034841357445 74.74752782706068) bank30171 +30172 POINT(41.36864370822959 73.91524018198095) bank30172 +30173 POINT(40.71922813337048 73.81201744047382) bank30173 +30174 POINT(41.14132105320604 74.83796785237946) bank30174 +30175 POINT(41.265475048067294 74.37703721720668) bank30175 +30176 POINT(41.470722461320356 73.98993867926677) bank30176 +30177 POINT(40.06918536379694 74.64206230133206) bank30177 +30178 POINT(41.16417257944232 74.26824821085553) bank30178 +30179 POINT(40.74572488761104 74.64452182091178) bank30179 +30180 POINT(41.29618467743444 74.04986301553659) bank30180 +30181 POINT(40.69896448084428 73.9173880127558) bank30181 +30182 POINT(40.41516171221925 73.93352995329559) bank30182 +30183 POINT(40.895345394580055 74.30121449257493) bank30183 +30184 POINT(40.45983424134777 73.3034796202074) bank30184 +30185 POINT(39.75799426182943 73.68565130657645) bank30185 +30186 POINT(39.99682992267374 75.00000133454004) bank30186 +30187 POINT(41.512567029200355 74.04790178591779) bank30187 +30188 POINT(41.47131447070091 74.04953027855736) bank30188 +30189 POINT(40.93936574585598 73.32760290619068) bank30189 +30190 POINT(41.03711670414103 74.10627109806424) bank30190 +30191 POINT(40.25126320376365 73.8262779108931) bank30191 +30192 POINT(41.29131812416798 74.1935866940335) bank30192 +30193 POINT(40.83431967381943 74.6162412676204) bank30193 +30194 POINT(40.93858456851232 73.30399967515062) bank30194 +30195 POINT(39.97927013155544 74.764952830901) bank30195 +30196 POINT(40.07946663853298 74.66050392260395) bank30196 +30197 POINT(40.323180183306775 74.53056424382012) bank30197 +30198 POINT(41.03894114524555 73.75004950604178) bank30198 +30199 POINT(40.585012710953 73.61400256839661) bank30199 +30200 POINT(39.78172340003366 74.33422044818403) bank30200 +30201 POINT(41.49561681210234 73.4899826151636) bank30201 +30202 POINT(41.004971448327865 73.31172307698563) bank30202 +30203 POINT(39.951661612897546 74.51371593324568) bank30203 +30204 POINT(40.80280475927667 74.62756852529215) bank30204 +30205 POINT(39.78147384460177 74.51606088222445) bank30205 +30206 POINT(41.546140387463595 74.97431758100801) bank30206 +30207 POINT(40.9662234129258 74.75670324577348) bank30207 +30208 POINT(40.14288257204185 73.84339397113432) bank30208 +30209 POINT(40.14716329796046 73.0616603378675) bank30209 +30210 POINT(39.957159946132606 74.61786057105662) bank30210 +30211 POINT(40.96674413563068 73.19639521892856) bank30211 +30212 POINT(40.06785377317544 74.72597216302725) bank30212 +30213 POINT(40.07498216869064 73.75505607244703) bank30213 +30214 POINT(40.345410868593525 73.18689158007257) bank30214 +30215 POINT(39.740375633574054 74.9144343182941) bank30215 +30216 POINT(40.608833753521864 73.84445134778453) bank30216 +30217 POINT(40.961817525254496 74.23661034393221) bank30217 +30218 POINT(40.488726661325146 74.04724955707827) bank30218 +30219 POINT(40.18768103124475 73.86514960981454) bank30219 +30220 POINT(39.80109418689523 73.15768742543823) bank30220 +30221 POINT(40.777227800248795 74.37240265098823) bank30221 +30222 POINT(40.32020499108685 73.92380179663373) bank30222 +30223 POINT(40.88144596128042 74.00809241304594) bank30223 +30224 POINT(40.99525331137115 73.8557677147476) bank30224 +30225 POINT(41.309379939899266 73.6471937606568) bank30225 +30226 POINT(41.408584919635175 73.32343795099669) bank30226 +30227 POINT(41.107042320046965 74.15907455099662) bank30227 +30228 POINT(40.132918965672566 73.89447615528772) bank30228 +30229 POINT(40.71382608925978 74.78578120117605) bank30229 +30230 POINT(41.12745192679387 74.89645657386195) bank30230 +30231 POINT(40.303276169281176 73.44084577036918) bank30231 +30232 POINT(41.41723160016532 73.9939598794767) bank30232 +30233 POINT(39.85069201549148 74.86492016776381) bank30233 +30234 POINT(40.49344068798001 74.09147904461184) bank30234 +30235 POINT(41.66578430829394 74.08957119158161) bank30235 +30236 POINT(41.058069036796034 73.0697854847705) bank30236 +30237 POINT(40.72158907052379 73.02711556833763) bank30237 +30238 POINT(39.788736765107 73.06452425798138) bank30238 +30239 POINT(40.72952286486895 73.5905419364302) bank30239 +30240 POINT(39.943037898713136 74.43331784565588) bank30240 +30241 POINT(40.66418261230697 73.06332114477226) bank30241 +30242 POINT(40.131937591943554 74.79594618165912) bank30242 +30243 POINT(41.3889926940824 74.2563663641181) bank30243 +30244 POINT(40.70547703051991 73.1686970063984) bank30244 +30245 POINT(40.86739116574814 74.28049299570851) bank30245 +30246 POINT(39.91533831367399 73.72058219460628) bank30246 +30247 POINT(41.542753260565654 73.16405484695717) bank30247 +30248 POINT(41.639783897372396 73.70449164144908) bank30248 +30249 POINT(40.214352282790806 74.35966291522627) bank30249 +30250 POINT(40.87867889805232 73.72816584948104) bank30250 +30251 POINT(40.58599253032135 73.58718934602405) bank30251 +30252 POINT(40.85360209514369 74.68302085146644) bank30252 +30253 POINT(41.01698072844733 73.76243442741321) bank30253 +30254 POINT(40.05767715991363 74.2509612750269) bank30254 +30255 POINT(41.29166218353398 73.97507868910368) bank30255 +30256 POINT(39.75699749450429 74.51102542027529) bank30256 +30257 POINT(40.7307162591871 74.16509590689634) bank30257 +30258 POINT(40.90584406861073 74.15436521586913) bank30258 +30259 POINT(40.54704058183672 73.66859266928967) bank30259 +30260 POINT(41.56183645390189 74.53449446401169) bank30260 +30261 POINT(41.41531105135513 73.41176124649331) bank30261 +30262 POINT(40.92589850959035 73.69855610350331) bank30262 +30263 POINT(40.03989027449734 73.36991700005181) bank30263 +30264 POINT(41.16105008670732 74.49803399627126) bank30264 +30265 POINT(39.77613638310683 73.10064330053206) bank30265 +30266 POINT(40.371443114609946 74.11259389837116) bank30266 +30267 POINT(41.18556644142481 74.04783042274929) bank30267 +30268 POINT(40.61590844221675 73.39726207917438) bank30268 +30269 POINT(39.8833314769623 73.53604886042632) bank30269 +30270 POINT(40.01922380102563 73.1280441082327) bank30270 +30271 POINT(40.36567509943577 73.51544214024408) bank30271 +30272 POINT(40.58994888920499 73.18729292626166) bank30272 +30273 POINT(40.78879796901478 74.4915594825802) bank30273 +30274 POINT(39.82791066822349 74.86705659223296) bank30274 +30275 POINT(40.07508099753551 73.09384451762985) bank30275 +30276 POINT(40.38879919314148 74.1936008249671) bank30276 +30277 POINT(39.88020881898197 74.14973739762296) bank30277 +30278 POINT(40.190660397389756 73.43864064910213) bank30278 +30279 POINT(40.49694272983352 73.75881954726012) bank30279 +30280 POINT(41.21137340235874 73.03839247382179) bank30280 +30281 POINT(41.69738484171894 74.31032392999617) bank30281 +30282 POINT(41.28309441007528 74.82336244296954) bank30282 +30283 POINT(40.32334883674373 73.5711861696449) bank30283 +30284 POINT(40.69124653430054 74.8598125654429) bank30284 +30285 POINT(41.10170993006338 73.97992816327834) bank30285 +30286 POINT(39.867998364341574 74.94037384887139) bank30286 +30287 POINT(41.28233043082885 73.62865190353352) bank30287 +30288 POINT(39.9861763461907 73.2083933076654) bank30288 +30289 POINT(40.20462824818735 74.06265877411533) bank30289 +30290 POINT(40.26685998233305 74.61630272263824) bank30290 +30291 POINT(40.42378218212219 73.49807236514856) bank30291 +30292 POINT(39.98640758256488 73.06366696606155) bank30292 +30293 POINT(41.60962171811986 73.07777688049663) bank30293 +30294 POINT(39.72462398704951 73.77349059222814) bank30294 +30295 POINT(40.35433539011936 74.37614555509515) bank30295 +30296 POINT(40.92955558879631 74.92068685329146) bank30296 +30297 POINT(40.55411055793492 73.59643982684136) bank30297 +30298 POINT(40.73808197047958 74.57806218586919) bank30298 +30299 POINT(40.7679168601816 74.90651238572214) bank30299 +30300 POINT(40.814066954453565 73.1674111154977) bank30300 +30301 POINT(40.43740664594406 74.86504187005802) bank30301 +30302 POINT(41.66102402530303 74.96214961043239) bank30302 +30303 POINT(40.08472767656573 74.05322057771973) bank30303 +30304 POINT(40.8099030252757 73.27627063059246) bank30304 +30305 POINT(40.35762026264511 73.02633159974616) bank30305 +30306 POINT(40.97368177804694 74.7049300697613) bank30306 +30307 POINT(41.01471444000627 74.8187633430712) bank30307 +30308 POINT(40.17499775605099 74.03476708705159) bank30308 +30309 POINT(40.82254329685039 73.05957260448372) bank30309 +30310 POINT(40.19332818734945 73.47099622121176) bank30310 +30311 POINT(40.56263837793898 73.17046372347015) bank30311 +30312 POINT(40.74417065353163 73.30316838801092) bank30312 +30313 POINT(41.27181665747005 73.2020356591678) bank30313 +30314 POINT(41.42841629364931 73.93707048072899) bank30314 +30315 POINT(41.337766538182265 74.1525745499496) bank30315 +30316 POINT(40.22317129695255 73.80720292237767) bank30316 +30317 POINT(40.65050833500874 73.35619472009516) bank30317 +30318 POINT(39.81506640800558 73.16110860202657) bank30318 +30319 POINT(41.11403386824763 73.49649820820886) bank30319 +30320 POINT(39.795177388512855 74.47821128774531) bank30320 +30321 POINT(40.190605812766066 74.52637757875802) bank30321 +30322 POINT(41.575658494783966 73.42777176235634) bank30322 +30323 POINT(40.79081063584711 73.0153028685852) bank30323 +30324 POINT(39.73254638777291 74.1445357164722) bank30324 +30325 POINT(39.967608285175466 73.11184012364116) bank30325 +30326 POINT(41.59047278238837 73.5727599652726) bank30326 +30327 POINT(41.35869179509692 74.77139345135245) bank30327 +30328 POINT(41.39188858513316 74.62450440902943) bank30328 +30329 POINT(40.060569713709754 74.52328177534837) bank30329 +30330 POINT(41.230373198208355 74.37712176199267) bank30330 +30331 POINT(41.38409523787341 74.97940758015255) bank30331 +30332 POINT(40.095493573850554 73.46017052811622) bank30332 +30333 POINT(40.42032535281004 74.1331810207289) bank30333 +30334 POINT(39.72742982575984 73.79247772305138) bank30334 +30335 POINT(40.17910008095173 73.30103117662658) bank30335 +30336 POINT(39.91626566442235 74.61195232897707) bank30336 +30337 POINT(40.228074463965136 73.69490703553919) bank30337 +30338 POINT(39.94627089594374 73.18596711775244) bank30338 +30339 POINT(40.73214362384618 73.77658547512382) bank30339 +30340 POINT(40.51007292809771 74.94643286501447) bank30340 +30341 POINT(41.61919457515328 74.14610706715716) bank30341 +30342 POINT(40.6816983841997 73.59033991473532) bank30342 +30343 POINT(39.74767238579515 73.14515142286419) bank30343 +30344 POINT(40.97062936073158 73.16968095851314) bank30344 +30345 POINT(40.64773794694976 74.18455008451713) bank30345 +30346 POINT(40.798100052054664 74.44202820383352) bank30346 +30347 POINT(41.53618946333981 74.17497448856288) bank30347 +30348 POINT(39.887853346367116 73.86926103892883) bank30348 +30349 POINT(40.557210979900205 74.53525259605671) bank30349 +30350 POINT(41.0355314383932 74.08952643051218) bank30350 +30351 POINT(40.30884788986804 73.9270017485332) bank30351 +30352 POINT(41.121051614103784 74.1880434989323) bank30352 +30353 POINT(40.44534182393008 73.86753240649435) bank30353 +30354 POINT(41.3437360724872 74.18540134657134) bank30354 +30355 POINT(40.98230985260354 74.61245401257776) bank30355 +30356 POINT(40.31671404315793 74.65997409222645) bank30356 +30357 POINT(41.310555596819576 74.09710281047748) bank30357 +30358 POINT(39.97868422777168 74.14549170033418) bank30358 +30359 POINT(40.74551648922901 74.1758703395546) bank30359 +30360 POINT(41.49918328955098 74.93356288197357) bank30360 +30361 POINT(40.901315704124286 74.09092821975948) bank30361 +30362 POINT(40.25908160549894 74.82882563176301) bank30362 +30363 POINT(41.70312403976739 73.68682436765776) bank30363 +30364 POINT(40.327874935499715 74.38355316893802) bank30364 +30365 POINT(40.16044061680208 73.81183788263392) bank30365 +30366 POINT(40.00554013042586 73.39556878581301) bank30366 +30367 POINT(41.35833265177386 73.2399180111474) bank30367 +30368 POINT(41.504988339883624 73.8898618497201) bank30368 +30369 POINT(40.41835766826876 74.1831078400555) bank30369 +30370 POINT(40.28053841586808 74.73230128098828) bank30370 +30371 POINT(41.29326603426339 74.12640382006175) bank30371 +30372 POINT(40.71502112703514 73.15727283273051) bank30372 +30373 POINT(40.49222918229741 74.17073013289331) bank30373 +30374 POINT(40.54224870547083 74.37817707371086) bank30374 +30375 POINT(39.83696850854519 73.66116534248727) bank30375 +30376 POINT(40.77789413474295 73.75864934801967) bank30376 +30377 POINT(39.86544483408236 74.8761599070704) bank30377 +30378 POINT(41.404697244207725 74.02239292891683) bank30378 +30379 POINT(39.733888475515144 74.32920567731159) bank30379 +30380 POINT(40.1432784540482 74.07822939984203) bank30380 +30381 POINT(41.1367455352778 73.70772248358718) bank30381 +30382 POINT(40.0616802963861 74.77864036745741) bank30382 +30383 POINT(41.15820527824292 74.69276997008622) bank30383 +30384 POINT(40.06434585806989 73.177286772108) bank30384 +30385 POINT(41.31658820362549 73.99045957606904) bank30385 +30386 POINT(40.77416622031096 73.95437872037509) bank30386 +30387 POINT(41.622544477295335 73.20705318556341) bank30387 +30388 POINT(41.06720377137624 74.78918837522221) bank30388 +30389 POINT(40.02324313589034 73.36088801741808) bank30389 +30390 POINT(41.28567025098869 74.26439040743162) bank30390 +30391 POINT(41.397994244209755 73.86219764792159) bank30391 +30392 POINT(40.3209751651077 74.77875540163505) bank30392 +30393 POINT(41.18065837076278 73.71920476923054) bank30393 +30394 POINT(40.439582312361104 74.97054624327384) bank30394 +30395 POINT(41.27621803829134 73.4850441435659) bank30395 +30396 POINT(41.51829071968715 74.62539890910917) bank30396 +30397 POINT(41.41268556689925 73.42568491728309) bank30397 +30398 POINT(40.66993057700254 73.64870150118126) bank30398 +30399 POINT(40.24774399456033 74.11976935553926) bank30399 +30400 POINT(39.956699141563405 74.96052776254832) bank30400 +30401 POINT(40.52288813787657 74.08760532071285) bank30401 +30402 POINT(40.29907165206219 73.93116780167531) bank30402 +30403 POINT(39.896302374788526 73.95408956454702) bank30403 +30404 POINT(40.053146567666545 73.78560528861586) bank30404 +30405 POINT(40.9602496970121 73.36572471715274) bank30405 +30406 POINT(41.13756068071625 74.38468520412741) bank30406 +30407 POINT(41.219546947712466 73.6768565762028) bank30407 +30408 POINT(41.187394435916595 74.00264547227044) bank30408 +30409 POINT(40.279183513068524 73.22825781344751) bank30409 +30410 POINT(39.90607146718084 73.7970695355386) bank30410 +30411 POINT(41.22458421859326 74.11438191260443) bank30411 +30412 POINT(41.55318758112119 74.86612812280825) bank30412 +30413 POINT(41.3644645091515 73.33935593196782) bank30413 +30414 POINT(41.18629819962531 73.89890715508692) bank30414 +30415 POINT(40.479720509085936 73.89084176001052) bank30415 +30416 POINT(41.07392388670299 74.85032474908665) bank30416 +30417 POINT(41.700211502867525 74.23988987791161) bank30417 +30418 POINT(40.39474735640227 73.92245155777891) bank30418 +30419 POINT(40.239491511045586 73.90141519311953) bank30419 +30420 POINT(40.71450470490833 74.75098306710764) bank30420 +30421 POINT(40.98948356598017 75.00173277090344) bank30421 +30422 POINT(40.93553784336527 73.22376616387609) bank30422 +30423 POINT(40.45183783527357 73.8313750896068) bank30423 +30424 POINT(40.147503507328885 74.81030937953213) bank30424 +30425 POINT(40.037850191303065 74.23959746176493) bank30425 +30426 POINT(41.35040532992057 74.01625867161123) bank30426 +30427 POINT(40.508063076328604 73.54840566132042) bank30427 +30428 POINT(40.29161015807898 74.83973349470537) bank30428 +30429 POINT(40.91468519194239 73.31502367110248) bank30429 +30430 POINT(41.01500288254666 74.74946971841916) bank30430 +30431 POINT(41.37073495200908 74.37918945851035) bank30431 +30432 POINT(40.28290642844975 74.19271403193578) bank30432 +30433 POINT(40.09855171071577 73.3782429213109) bank30433 +30434 POINT(39.71839069090485 73.2055785003654) bank30434 +30435 POINT(40.79042398203427 74.69160700428814) bank30435 +30436 POINT(40.332466899272326 73.67576159290444) bank30436 +30437 POINT(40.19954759923226 73.91026032817975) bank30437 +30438 POINT(41.256665582765905 74.06954560003831) bank30438 +30439 POINT(41.702863409776796 73.4249404907494) bank30439 +30440 POINT(39.88926965769055 73.4817940034086) bank30440 +30441 POINT(41.26928214323184 73.88678121406825) bank30441 +30442 POINT(40.381560909941925 73.16741045647022) bank30442 +30443 POINT(39.77826185245072 73.38401129590729) bank30443 +30444 POINT(41.340916018524176 74.24085198486505) bank30444 +30445 POINT(40.47764939079601 73.0125076838154) bank30445 +30446 POINT(40.53615520005257 73.29606402591806) bank30446 +30447 POINT(40.21481347358835 74.08857833832027) bank30447 +30448 POINT(40.03182806488891 73.25734864841796) bank30448 +30449 POINT(40.44946176277667 73.51828008867942) bank30449 +30450 POINT(41.18288387786893 74.8215282800555) bank30450 +30451 POINT(39.98425574004085 74.2321132719377) bank30451 +30452 POINT(39.816862104004805 73.139873965371) bank30452 +30453 POINT(39.85683763086588 73.18989286865543) bank30453 +30454 POINT(41.13513673717166 74.30606789790323) bank30454 +30455 POINT(40.232260161692565 74.45407545144717) bank30455 +30456 POINT(40.62730319505473 73.40036965316551) bank30456 +30457 POINT(40.93235569536324 74.74617290246711) bank30457 +30458 POINT(39.88433884020528 73.75870389961376) bank30458 +30459 POINT(39.82345335581637 74.69143047299275) bank30459 +30460 POINT(41.33995786318559 74.01005061957648) bank30460 +30461 POINT(41.08161482575579 74.51011953802364) bank30461 +30462 POINT(40.827527390977785 73.11959619955128) bank30462 +30463 POINT(41.00152946420169 73.54729064220301) bank30463 +30464 POINT(40.42570544510303 74.9374288584934) bank30464 +30465 POINT(40.29776452054408 73.53196763094991) bank30465 +30466 POINT(40.91703681815329 74.36069312187291) bank30466 +30467 POINT(41.46226216141091 74.45515932710653) bank30467 +30468 POINT(41.65640024910376 74.48251566356855) bank30468 +30469 POINT(41.121074887798926 73.49711474171337) bank30469 +30470 POINT(41.141884973981185 73.59588850477319) bank30470 +30471 POINT(39.84633794955071 73.09045845733932) bank30471 +30472 POINT(41.548786207852444 74.57860475987103) bank30472 +30473 POINT(41.609459664905934 73.09370208384034) bank30473 +30474 POINT(40.66116307262248 73.80014047621243) bank30474 +30475 POINT(41.37662719431846 74.93515359801516) bank30475 +30476 POINT(41.5421615602264 73.74740082018302) bank30476 +30477 POINT(41.485617997257386 74.3639311741677) bank30477 +30478 POINT(41.03922605301786 73.3234974774196) bank30478 +30479 POINT(41.29658612236727 74.78384931089477) bank30479 +30480 POINT(39.79307856579754 73.19934153444802) bank30480 +30481 POINT(40.09311523049823 73.43807429456834) bank30481 +30482 POINT(41.34759340619556 73.50516242390506) bank30482 +30483 POINT(41.676529344805985 73.48386080036371) bank30483 +30484 POINT(39.98999049177994 73.17264543430959) bank30484 +30485 POINT(39.94776193456294 73.38350363170784) bank30485 +30486 POINT(40.745116122348485 73.94800490897737) bank30486 +30487 POINT(41.54460049615731 73.69918493956871) bank30487 +30488 POINT(40.853532189717626 74.57016286347579) bank30488 +30489 POINT(40.69731124035202 73.32365113090927) bank30489 +30490 POINT(40.6244015914294 73.11126093990801) bank30490 +30491 POINT(40.24302198615697 74.7863907628394) bank30491 +30492 POINT(41.65726450568018 74.22761888510848) bank30492 +30493 POINT(41.694285610782295 73.30663939627358) bank30493 +30494 POINT(41.05207339361311 73.58364519969017) bank30494 +30495 POINT(41.00149179999712 73.37069692401134) bank30495 +30496 POINT(40.16209212291417 73.1522201353921) bank30496 +30497 POINT(41.053565581320775 73.31826836297033) bank30497 +30498 POINT(40.03811646170889 73.07782974448008) bank30498 +30499 POINT(39.73021102159309 73.38234204399521) bank30499 +30500 POINT(39.95136088158568 74.3569082221838) bank30500 +30501 POINT(41.21454497593398 74.87250256076877) bank30501 +30502 POINT(41.075340492426434 73.68040965712892) bank30502 +30503 POINT(40.971868798240884 73.9654969769783) bank30503 +30504 POINT(39.74087282259591 74.50788732367576) bank30504 +30505 POINT(40.38590100824678 73.60754100895167) bank30505 +30506 POINT(40.43551468036589 74.95017334728193) bank30506 +30507 POINT(40.97163882672821 73.5859893828136) bank30507 +30508 POINT(40.515804526070255 74.85847516960993) bank30508 +30509 POINT(41.645417262320805 74.4164765086222) bank30509 +30510 POINT(40.677682523116246 73.57276541529136) bank30510 +30511 POINT(41.19172017527712 74.08121584461657) bank30511 +30512 POINT(40.02110410369637 74.04709822167744) bank30512 +30513 POINT(40.1252170885769 73.69384122416434) bank30513 +30514 POINT(40.34963891362782 74.66489163359495) bank30514 +30515 POINT(41.278874722658266 73.48384988219269) bank30515 +30516 POINT(40.09491859228663 73.5903366786639) bank30516 +30517 POINT(39.76788549740392 73.26864352647839) bank30517 +30518 POINT(39.81835832836022 74.24995409963348) bank30518 +30519 POINT(40.22301965024225 74.58096300625282) bank30519 +30520 POINT(40.12040388578602 74.08373650791435) bank30520 +30521 POINT(41.35531336378145 74.14678534961827) bank30521 +30522 POINT(40.724569840539765 74.9118423520858) bank30522 +30523 POINT(41.52696257952842 73.71604805961854) bank30523 +30524 POINT(40.47959361959758 73.61157469803123) bank30524 +30525 POINT(41.65309211347623 73.354319772992) bank30525 +30526 POINT(40.74437760418597 73.61783121542086) bank30526 +30527 POINT(40.34873504170823 74.70251223866134) bank30527 +30528 POINT(41.400008419520155 74.36435820046165) bank30528 +30529 POINT(40.71389001728432 74.15145342611333) bank30529 +30530 POINT(40.711740247969495 74.50288418731898) bank30530 +30531 POINT(40.35924687990223 73.78938004548522) bank30531 +30532 POINT(41.59789254068413 73.2185383521648) bank30532 +30533 POINT(40.98331800314327 74.08542310875288) bank30533 +30534 POINT(40.91110968264689 74.68871111840164) bank30534 +30535 POINT(41.15591540652731 74.0017439701483) bank30535 +30536 POINT(41.07156175672342 74.65248553646275) bank30536 +30537 POINT(40.109775997003304 73.399974493749) bank30537 +30538 POINT(41.00327381573226 74.90007285694449) bank30538 +30539 POINT(40.115647682107465 73.50931000181569) bank30539 +30540 POINT(41.25095778393529 74.460075034942) bank30540 +30541 POINT(40.325406090962844 74.16509879365796) bank30541 +30542 POINT(41.684870948941935 74.09459702211869) bank30542 +30543 POINT(41.37560040816485 73.24227739395337) bank30543 +30544 POINT(40.36856133823859 73.70949456373921) bank30544 +30545 POINT(39.85487584728967 73.1825129140958) bank30545 +30546 POINT(40.033942571349606 74.90632644630568) bank30546 +30547 POINT(41.073443417070074 74.51089753021161) bank30547 +30548 POINT(39.87154521330077 74.11158389864156) bank30548 +30549 POINT(41.26005848438368 73.70516972592453) bank30549 +30550 POINT(41.43563631189128 74.72306245613517) bank30550 +30551 POINT(41.161255627521115 74.43646412692473) bank30551 +30552 POINT(40.55746367398406 73.09135609828054) bank30552 +30553 POINT(40.04200113007566 73.90080656447884) bank30553 +30554 POINT(40.80643818637858 74.67291865897798) bank30554 +30555 POINT(39.821991741550626 73.17612722429355) bank30555 +30556 POINT(41.35254537970793 74.00057535074778) bank30556 +30557 POINT(41.24680121972437 74.15170275334272) bank30557 +30558 POINT(41.112098635341624 74.36353860098404) bank30558 +30559 POINT(41.04005738947757 73.3948155697058) bank30559 +30560 POINT(39.89486151019096 73.07404063605102) bank30560 +30561 POINT(41.59148012579531 74.45988020177556) bank30561 +30562 POINT(40.49904837728341 73.60538430483435) bank30562 +30563 POINT(40.340621203505705 73.77727611230455) bank30563 +30564 POINT(39.85887155398845 74.53242025914925) bank30564 +30565 POINT(40.35614477899851 74.91015815732885) bank30565 +30566 POINT(40.59405237546572 73.3298628102142) bank30566 +30567 POINT(40.523298244254335 74.7256393989389) bank30567 +30568 POINT(39.86167519173521 73.44668008994591) bank30568 +30569 POINT(41.36164390837301 74.47460376909117) bank30569 +30570 POINT(41.386586231209456 74.31806277202416) bank30570 +30571 POINT(39.790465913522596 74.6041911810066) bank30571 +30572 POINT(40.874836431954826 73.35506079808457) bank30572 +30573 POINT(41.246479110949814 73.51028468533231) bank30573 +30574 POINT(40.3398119275198 73.74531704832911) bank30574 +30575 POINT(40.421280280616855 73.89972473770001) bank30575 +30576 POINT(40.82177926756911 73.79391772272145) bank30576 +30577 POINT(40.357738939985786 74.46696042121893) bank30577 +30578 POINT(41.21922082699148 73.86699879606047) bank30578 +30579 POINT(39.7439148549689 73.08581434511443) bank30579 +30580 POINT(40.448863181918725 74.70740898750225) bank30580 +30581 POINT(41.119231753415086 74.66624826555994) bank30581 +30582 POINT(40.598956608308015 74.09193096950737) bank30582 +30583 POINT(41.149504701221744 73.02619987701806) bank30583 +30584 POINT(41.54231225786815 73.61933832777375) bank30584 +30585 POINT(40.60476008127485 73.60873289884393) bank30585 +30586 POINT(40.391960944208144 73.05091738196694) bank30586 +30587 POINT(40.10859298751409 74.96529133535724) bank30587 +30588 POINT(41.464823713720165 74.09608702107988) bank30588 +30589 POINT(41.05427581815125 73.52671694422745) bank30589 +30590 POINT(41.70076766830063 74.85869236082549) bank30590 +30591 POINT(40.50137643214579 74.40564664464537) bank30591 +30592 POINT(40.92596193958586 73.48498232568339) bank30592 +30593 POINT(41.60970568216724 73.48379913666062) bank30593 +30594 POINT(39.930985295111114 74.83182961349976) bank30594 +30595 POINT(40.96415082579551 73.153060917176) bank30595 +30596 POINT(41.18108780574533 74.66026843379588) bank30596 +30597 POINT(40.55052873409915 74.24054191210294) bank30597 +30598 POINT(41.54656807536506 74.40848453356504) bank30598 +30599 POINT(41.22331832549075 74.75330295301657) bank30599 +30600 POINT(39.74227530195448 74.0635636208132) bank30600 +30601 POINT(40.02249878605774 73.73909007212752) bank30601 +30602 POINT(40.462922220672375 73.68454803025038) bank30602 +30603 POINT(39.71303153565195 74.9352858556824) bank30603 +30604 POINT(40.348628641151166 74.8593444274217) bank30604 +30605 POINT(40.37423245415515 73.01463822275443) bank30605 +30606 POINT(41.53739314684605 74.79363426295316) bank30606 +30607 POINT(41.332745214104776 73.21137875008822) bank30607 +30608 POINT(40.27826785159459 74.00563005752359) bank30608 +30609 POINT(40.02570543896603 74.49054482363532) bank30609 +30610 POINT(41.60704033173753 74.13094411186573) bank30610 +30611 POINT(40.75517037472211 73.3323091355389) bank30611 +30612 POINT(40.417760838802884 74.18935304441862) bank30612 +30613 POINT(41.40358951000787 73.9008942322963) bank30613 +30614 POINT(41.02593542959577 74.17218231787179) bank30614 +30615 POINT(41.69818454681941 73.41382074609655) bank30615 +30616 POINT(41.6736075444534 73.07313867940557) bank30616 +30617 POINT(41.52075795647138 74.04333765380862) bank30617 +30618 POINT(41.13727364538399 73.89621147002427) bank30618 +30619 POINT(40.60900150539874 74.44502012931434) bank30619 +30620 POINT(40.143561420658564 74.14087809097124) bank30620 +30621 POINT(39.84284667730006 73.42386011733467) bank30621 +30622 POINT(41.45057798049629 74.00979894798068) bank30622 +30623 POINT(41.664688563852835 73.51868491821531) bank30623 +30624 POINT(40.656058155482945 73.18062977593837) bank30624 +30625 POINT(40.935739300702465 74.72815871128164) bank30625 +30626 POINT(41.37758457984518 74.00496981262458) bank30626 +30627 POINT(41.18783599175129 74.66031921174515) bank30627 +30628 POINT(40.96608759651823 74.9594794218663) bank30628 +30629 POINT(41.48837468156014 73.82922805335669) bank30629 +30630 POINT(40.65631596213924 75.00062159616368) bank30630 +30631 POINT(41.63201296899803 73.37825195271516) bank30631 +30632 POINT(40.94959949992717 74.1904399661087) bank30632 +30633 POINT(41.58358191315679 73.7976515100058) bank30633 +30634 POINT(40.401577190041564 74.48288932534575) bank30634 +30635 POINT(40.41755138076551 74.89468555300215) bank30635 +30636 POINT(41.56244881596804 73.89075551316063) bank30636 +30637 POINT(40.618579743153006 74.57256773831332) bank30637 +30638 POINT(40.62077432669216 73.63790154513326) bank30638 +30639 POINT(40.28162944471003 73.93411841882136) bank30639 +30640 POINT(40.84964356755461 74.40050910421083) bank30640 +30641 POINT(40.18449119508688 73.42272363758649) bank30641 +30642 POINT(40.113815705437254 74.63085954140098) bank30642 +30643 POINT(40.60955588664411 74.15236785545211) bank30643 +30644 POINT(40.871811540271814 74.32840728018515) bank30644 +30645 POINT(41.54368483166838 73.82240227413907) bank30645 +30646 POINT(41.07055790638016 74.68959229665417) bank30646 +30647 POINT(41.361526507545285 74.9264687422716) bank30647 +30648 POINT(40.980664234934295 74.19098758477513) bank30648 +30649 POINT(41.27181931040472 74.99959129032905) bank30649 +30650 POINT(40.615332816709746 73.24243924796286) bank30650 +30651 POINT(40.889537515555446 73.04580361656745) bank30651 +30652 POINT(41.289754799907314 73.78382975026113) bank30652 +30653 POINT(40.607786069645584 73.94038037684687) bank30653 +30654 POINT(40.06620729983404 73.9411060832943) bank30654 +30655 POINT(40.04429240840063 73.87704990289794) bank30655 +30656 POINT(40.55426279854917 74.44732141954792) bank30656 +30657 POINT(40.21502741535974 73.43608136241595) bank30657 +30658 POINT(41.68039204098588 73.26465150142467) bank30658 +30659 POINT(41.54221914348064 74.29443709817582) bank30659 +30660 POINT(40.707743517611725 73.26104404195327) bank30660 +30661 POINT(39.78982461424831 73.89221709113173) bank30661 +30662 POINT(40.002732397499955 74.57589649909612) bank30662 +30663 POINT(41.18094885130177 74.84055889366235) bank30663 +30664 POINT(41.112936745154116 73.15213939464155) bank30664 +30665 POINT(41.54446546015299 74.04065250956226) bank30665 +30666 POINT(41.006660261420684 73.32486610376746) bank30666 +30667 POINT(41.55915923772832 74.99115328454039) bank30667 +30668 POINT(40.468457670071835 73.83885929809023) bank30668 +30669 POINT(41.61841995289443 74.69392451767199) bank30669 +30670 POINT(41.22814933834966 73.1396990375463) bank30670 +30671 POINT(40.281001623439266 74.26417103670853) bank30671 +30672 POINT(41.04872280770838 73.41407768074919) bank30672 +30673 POINT(40.60774337190389 73.79743235058393) bank30673 +30674 POINT(41.00826351659643 74.95968154713576) bank30674 +30675 POINT(41.67263796378571 73.7669097102073) bank30675 +30676 POINT(41.23030518576952 74.6251107998482) bank30676 +30677 POINT(40.81229241486749 74.1742121857715) bank30677 +30678 POINT(40.81078313221857 74.144641419445) bank30678 +30679 POINT(40.500695878877686 74.46146154805649) bank30679 +30680 POINT(40.9415594499049 74.15534268961869) bank30680 +30681 POINT(40.12423936561953 74.84263288460215) bank30681 +30682 POINT(39.78340733429971 74.1200958248159) bank30682 +30683 POINT(40.03473017704945 73.58933548647836) bank30683 +30684 POINT(41.07730985867702 74.56608928762324) bank30684 +30685 POINT(39.98725474105803 73.81592564344274) bank30685 +30686 POINT(41.059441130133074 74.2467298717467) bank30686 +30687 POINT(41.281664064757116 74.85633554397153) bank30687 +30688 POINT(40.6900587720609 74.38777143537882) bank30688 +30689 POINT(40.981102874659115 73.76232394065016) bank30689 +30690 POINT(40.53726795968248 74.51101705861379) bank30690 +30691 POINT(40.14547377573757 73.18999635980892) bank30691 +30692 POINT(40.227626460138964 73.23038785121416) bank30692 +30693 POINT(40.322870864945735 73.62329782703515) bank30693 +30694 POINT(41.121245830504456 74.99938310362681) bank30694 +30695 POINT(39.745320830155606 73.39258147800201) bank30695 +30696 POINT(40.33565026299317 74.03827471944173) bank30696 +30697 POINT(40.99510810913649 73.1041121708691) bank30697 +30698 POINT(40.81119420836417 73.97701690555755) bank30698 +30699 POINT(40.23876419798531 73.73108997000477) bank30699 +30700 POINT(41.41856024167595 74.9770785842145) bank30700 +30701 POINT(39.733356725249045 74.64118157399187) bank30701 +30702 POINT(41.06910335782787 73.4338442664241) bank30702 +30703 POINT(41.047115795939014 74.52164352830142) bank30703 +30704 POINT(40.580584310264086 73.42202052570272) bank30704 +30705 POINT(41.33192868003004 73.53883275899241) bank30705 +30706 POINT(40.704775070630845 73.10029035975184) bank30706 +30707 POINT(41.3377427804295 73.47454347236422) bank30707 +30708 POINT(40.690848704260326 73.33015794914014) bank30708 +30709 POINT(41.48152569675291 74.93369824897532) bank30709 +30710 POINT(41.09361662664213 74.91836324057265) bank30710 +30711 POINT(40.778490258536806 74.85736065654372) bank30711 +30712 POINT(40.416169466518355 73.49398172331102) bank30712 +30713 POINT(40.0771097076769 74.53045280118643) bank30713 +30714 POINT(41.16772076285105 74.71468007059636) bank30714 +30715 POINT(41.46567353948002 74.4118502883415) bank30715 +30716 POINT(40.297316467426 73.70116343667411) bank30716 +30717 POINT(40.686384287561026 74.1057046179514) bank30717 +30718 POINT(41.23128046783701 73.11996960917436) bank30718 +30719 POINT(41.19058360520235 73.45780512410865) bank30719 +30720 POINT(40.33249176145101 73.952942508669) bank30720 +30721 POINT(39.805595475124605 73.13354175349409) bank30721 +30722 POINT(41.55484593071226 74.05474924499003) bank30722 +30723 POINT(40.73795646412017 73.96299974350393) bank30723 +30724 POINT(40.665364773181025 73.50554675004453) bank30724 +30725 POINT(40.50592535477353 74.09813104058755) bank30725 +30726 POINT(41.20166016278368 73.64401302955864) bank30726 +30727 POINT(40.36999231871343 73.63646060615136) bank30727 +30728 POINT(39.74004932770186 74.52073099835084) bank30728 +30729 POINT(40.83293406509119 74.13000798487171) bank30729 +30730 POINT(40.72209904733011 73.33768684587332) bank30730 +30731 POINT(41.44357404546618 74.00449376850244) bank30731 +30732 POINT(39.81961966138513 74.2041214088145) bank30732 +30733 POINT(41.531236277971466 73.25572714738274) bank30733 +30734 POINT(41.68893894625868 73.63086022388549) bank30734 +30735 POINT(41.598860033330325 74.63631320622886) bank30735 +30736 POINT(41.32757218641125 73.6501065539262) bank30736 +30737 POINT(40.23976273381233 73.13984109211553) bank30737 +30738 POINT(40.919165829927714 74.35806976512839) bank30738 +30739 POINT(40.96129050324444 74.89052070196533) bank30739 +30740 POINT(40.69860207068886 74.56038019479199) bank30740 +30741 POINT(41.28336954707846 73.68366276045556) bank30741 +30742 POINT(40.35220082731137 73.78961213116705) bank30742 +30743 POINT(40.283271734325744 73.74741586251366) bank30743 +30744 POINT(41.650219603912085 73.65438265596005) bank30744 +30745 POINT(40.49618094608933 73.40030525655675) bank30745 +30746 POINT(40.61276016716133 73.8760735231176) bank30746 +30747 POINT(40.66734310513441 74.16302000448871) bank30747 +30748 POINT(41.37874165561032 73.27932103764661) bank30748 +30749 POINT(40.866617014549256 73.34234742280937) bank30749 +30750 POINT(39.98327401954814 74.01320808365318) bank30750 +30751 POINT(40.08449716654182 73.56421031803177) bank30751 +30752 POINT(39.77864280134179 73.97424897699786) bank30752 +30753 POINT(41.40556391214419 73.98216122111585) bank30753 +30754 POINT(41.23905328840116 73.34302523187715) bank30754 +30755 POINT(40.535776599267436 73.27028133226294) bank30755 +30756 POINT(41.08275820518559 73.74370394124533) bank30756 +30757 POINT(41.615959004741285 74.92918897500762) bank30757 +30758 POINT(40.09609200472349 73.7274343230413) bank30758 +30759 POINT(40.36920679269761 74.53057294748916) bank30759 +30760 POINT(41.29826829901473 74.61011397403263) bank30760 +30761 POINT(41.711248996657005 74.27327496833256) bank30761 +30762 POINT(41.30384722577114 74.6835325570729) bank30762 +30763 POINT(41.07533210787577 73.80134471158726) bank30763 +30764 POINT(40.38165599274073 73.83244758978256) bank30764 +30765 POINT(39.78937574629744 74.43338890666969) bank30765 +30766 POINT(40.60712106508408 74.6832284608758) bank30766 +30767 POINT(41.616335642511785 73.41015585428298) bank30767 +30768 POINT(40.54053289659552 73.90411079523042) bank30768 +30769 POINT(39.99112534793336 74.13913575698747) bank30769 +30770 POINT(40.74000035350864 74.43971973291501) bank30770 +30771 POINT(41.13211962295736 74.27868915818866) bank30771 +30772 POINT(40.3524109828465 74.41073863528871) bank30772 +30773 POINT(40.227625055517315 73.97252431639605) bank30773 +30774 POINT(41.635023483557276 73.09613053795269) bank30774 +30775 POINT(40.039715539020364 73.15111350211521) bank30775 +30776 POINT(41.36597786440225 74.8093006381253) bank30776 +30777 POINT(41.07175233739329 74.1165466413026) bank30777 +30778 POINT(41.54051722874122 73.20581232868335) bank30778 +30779 POINT(41.01934739989666 73.5504323521465) bank30779 +30780 POINT(41.2730077518603 73.79089246638229) bank30780 +30781 POINT(41.05786879099201 74.63213094260107) bank30781 +30782 POINT(40.052513723504404 74.95399625026342) bank30782 +30783 POINT(40.03625501907087 74.08488221175362) bank30783 +30784 POINT(39.91241666587658 74.88354105846797) bank30784 +30785 POINT(40.52357477315442 74.8244335586226) bank30785 +30786 POINT(39.76360120539242 73.69254182383102) bank30786 +30787 POINT(41.266310402753 73.36231548034303) bank30787 +30788 POINT(40.95584145353136 74.68365244379281) bank30788 +30789 POINT(40.094318267315465 73.34868743427587) bank30789 +30790 POINT(40.685619419660384 73.2323759778304) bank30790 +30791 POINT(40.37659441560372 73.63275303537517) bank30791 +30792 POINT(41.70598948443867 73.11208150212079) bank30792 +30793 POINT(40.43039487326776 74.21343625287665) bank30793 +30794 POINT(40.08917712710729 74.42462873869941) bank30794 +30795 POINT(41.27942656763204 73.06918694632543) bank30795 +30796 POINT(40.09192619553154 74.07019344438093) bank30796 +30797 POINT(40.34876748629436 73.49535985145414) bank30797 +30798 POINT(41.24115768087516 74.79295297989313) bank30798 +30799 POINT(40.82406181237423 74.9911022371438) bank30799 +30800 POINT(39.72489345296506 74.94569922564882) bank30800 +30801 POINT(39.822613202685254 74.74379249342694) bank30801 +30802 POINT(41.24668025049533 74.0184126314673) bank30802 +30803 POINT(40.19246164626217 73.23153949041706) bank30803 +30804 POINT(41.50039733320279 73.07161562638836) bank30804 +30805 POINT(40.538304407830466 73.67960065145809) bank30805 +30806 POINT(39.97101655921461 73.48671164184088) bank30806 +30807 POINT(40.58767330459693 73.21642191057732) bank30807 +30808 POINT(39.84059252712868 73.53213513038688) bank30808 +30809 POINT(41.276839588237145 74.00331188911815) bank30809 +30810 POINT(39.864382387140594 74.51900844196403) bank30810 +30811 POINT(40.86809317121226 74.22059125046464) bank30811 +30812 POINT(40.38998228550563 74.14728661711143) bank30812 +30813 POINT(41.562071019836196 73.88866170400645) bank30813 +30814 POINT(40.47804755061482 74.27845173630483) bank30814 +30815 POINT(40.613035206291926 73.74391874249854) bank30815 +30816 POINT(41.596048419233206 74.13766264287364) bank30816 +30817 POINT(41.684375339396645 73.36020159835333) bank30817 +30818 POINT(39.99873672057173 74.81720817214745) bank30818 +30819 POINT(40.45901832470062 73.8605430094078) bank30819 +30820 POINT(40.55173064344547 73.36744532798342) bank30820 +30821 POINT(39.75143392372459 73.59895483885168) bank30821 +30822 POINT(40.56029454111462 74.32436898851826) bank30822 +30823 POINT(41.077482294520614 74.04619109745371) bank30823 +30824 POINT(40.32375583663113 73.05880247137365) bank30824 +30825 POINT(39.95287531504911 74.61546258188436) bank30825 +30826 POINT(41.16286476281216 74.50254593177253) bank30826 +30827 POINT(39.85795543523987 74.00515881362918) bank30827 +30828 POINT(40.5204555587312 74.34792427396451) bank30828 +30829 POINT(40.13305946819388 74.9034662892776) bank30829 +30830 POINT(39.917571481888004 74.03042618332326) bank30830 +30831 POINT(40.4840322088183 74.92729564389067) bank30831 +30832 POINT(40.6920869057879 73.2101021944583) bank30832 +30833 POINT(40.25798214126821 74.56181946988808) bank30833 +30834 POINT(39.73188248928298 73.50377672896863) bank30834 +30835 POINT(40.04995713435757 74.41443648205369) bank30835 +30836 POINT(41.22910304792927 73.3712090014962) bank30836 +30837 POINT(40.40077351423804 74.94581581868763) bank30837 +30838 POINT(40.93109387901551 74.2548636702608) bank30838 +30839 POINT(40.77407598370936 74.61490742214345) bank30839 +30840 POINT(39.971545037566685 73.69576669598482) bank30840 +30841 POINT(39.808859867980104 73.4743270814691) bank30841 +30842 POINT(40.18839755181624 74.76564776856642) bank30842 +30843 POINT(40.75448305505533 73.75007092693275) bank30843 +30844 POINT(41.07868508908776 74.81771756617908) bank30844 +30845 POINT(40.13444965920012 74.47576414572859) bank30845 +30846 POINT(40.44772409455276 73.0111704836846) bank30846 +30847 POINT(41.25371006085688 73.93655246119219) bank30847 +30848 POINT(40.56404918426681 73.41380850758493) bank30848 +30849 POINT(39.88344699510406 73.79480240536178) bank30849 +30850 POINT(40.38026211101169 74.12308892017903) bank30850 +30851 POINT(40.06615956158918 73.33539992480658) bank30851 +30852 POINT(40.195472808593514 73.18625229878236) bank30852 +30853 POINT(40.93063900442429 73.18125267562843) bank30853 +30854 POINT(40.755605844993205 73.3858488429664) bank30854 +30855 POINT(40.79386613129093 73.86446891240382) bank30855 +30856 POINT(41.46974381654266 73.72893219938011) bank30856 +30857 POINT(41.18272331998413 74.88983568532839) bank30857 +30858 POINT(41.03784672743087 73.194527683914) bank30858 +30859 POINT(40.94427469111463 73.07364442831086) bank30859 +30860 POINT(39.95283514897178 74.3754662906717) bank30860 +30861 POINT(40.244563553202134 74.69305717048825) bank30861 +30862 POINT(40.86024365054961 73.99590245029643) bank30862 +30863 POINT(40.552457756364866 73.01328147730739) bank30863 +30864 POINT(40.792865561980165 73.8092782824954) bank30864 +30865 POINT(40.974824403254246 74.81459005876452) bank30865 +30866 POINT(40.29593709293407 73.60669244439885) bank30866 +30867 POINT(41.111276767422716 73.29015543237665) bank30867 +30868 POINT(41.64551892858284 73.86628503368516) bank30868 +30869 POINT(40.357101750835426 74.91770029011934) bank30869 +30870 POINT(40.51969727388661 74.10043248978131) bank30870 +30871 POINT(41.48684006961257 74.3049641684283) bank30871 +30872 POINT(40.98962958308559 73.77409285308988) bank30872 +30873 POINT(40.68182179798651 73.85883382613144) bank30873 +30874 POINT(40.5534333055629 73.3151525779201) bank30874 +30875 POINT(41.07011078591307 74.28775165095536) bank30875 +30876 POINT(41.655216966849274 74.40336888109321) bank30876 +30877 POINT(40.40854192320947 74.1508235385984) bank30877 +30878 POINT(40.39450227623133 73.7679527559707) bank30878 +30879 POINT(40.759299524370725 73.31445731585399) bank30879 +30880 POINT(41.693404599949766 74.22241684565256) bank30880 +30881 POINT(41.00426840453199 74.5101957507856) bank30881 +30882 POINT(40.86709771064933 74.44930583501338) bank30882 +30883 POINT(41.00619463760619 74.6300461802814) bank30883 +30884 POINT(40.23418750352721 73.47141663932979) bank30884 +30885 POINT(40.82951319080647 73.68650186693797) bank30885 +30886 POINT(40.54953783841154 73.54142386430173) bank30886 +30887 POINT(40.53540547592439 74.7204826413701) bank30887 +30888 POINT(40.32868212778681 73.5317828935199) bank30888 +30889 POINT(39.91875473243053 74.76177823976549) bank30889 +30890 POINT(41.60976816003868 74.87305220083338) bank30890 +30891 POINT(40.89731495741941 73.14185175596064) bank30891 +30892 POINT(39.807260054449614 74.53268340380873) bank30892 +30893 POINT(40.854717317075874 74.20301696805997) bank30893 +30894 POINT(41.130473037395944 73.05295084869229) bank30894 +30895 POINT(40.64261031274883 73.15048447863346) bank30895 +30896 POINT(41.46995085679871 74.55529028312341) bank30896 +30897 POINT(40.529915770787866 73.59134185928156) bank30897 +30898 POINT(40.650348711819305 73.82844745498056) bank30898 +30899 POINT(41.30119241766458 74.1932943567088) bank30899 +30900 POINT(39.87210919744093 74.9737721974953) bank30900 +30901 POINT(40.34700336296601 73.15214793548711) bank30901 +30902 POINT(40.25323235370168 74.11590063593125) bank30902 +30903 POINT(40.04105472917401 73.57869040709237) bank30903 +30904 POINT(41.26137726082878 73.9005711374252) bank30904 +30905 POINT(40.90374746883835 73.51058520438194) bank30905 +30906 POINT(39.99361410092841 73.37821168850479) bank30906 +30907 POINT(41.66054609685338 73.87135249744485) bank30907 +30908 POINT(40.04486417743929 74.66451738110091) bank30908 +30909 POINT(40.03947652819576 74.48462633662683) bank30909 +30910 POINT(40.75657946275152 73.9941838223294) bank30910 +30911 POINT(41.06671245883165 74.12202866088282) bank30911 +30912 POINT(40.95917509093079 73.58161122482439) bank30912 +30913 POINT(41.16455702598997 73.85464330548399) bank30913 +30914 POINT(39.79519270700362 74.21500006259687) bank30914 +30915 POINT(39.76882714793208 74.0653831725302) bank30915 +30916 POINT(41.08480823882464 74.46989905094932) bank30916 +30917 POINT(39.812101539771085 73.5671898896704) bank30917 +30918 POINT(40.835632022611705 73.73880104610517) bank30918 +30919 POINT(40.731922637673996 73.7582418532067) bank30919 +30920 POINT(40.68240495499497 74.59241580682871) bank30920 +30921 POINT(40.05615088798497 73.89890856224638) bank30921 +30922 POINT(40.10708639530114 74.61730290591164) bank30922 +30923 POINT(40.687354761534955 73.84675230287823) bank30923 +30924 POINT(40.46615419706403 73.52564480495934) bank30924 +30925 POINT(41.15646463408299 73.37049534969627) bank30925 +30926 POINT(40.88459007159823 74.30560372624653) bank30926 +30927 POINT(40.09211225443345 74.25273218638002) bank30927 +30928 POINT(40.53429790436549 74.40945364822413) bank30928 +30929 POINT(41.41679985381576 74.26175381195483) bank30929 +30930 POINT(39.9902476137419 74.51831816774653) bank30930 +30931 POINT(41.148894512114595 73.81494779993415) bank30931 +30932 POINT(40.12461788991767 73.78032412820741) bank30932 +30933 POINT(40.03782596605621 74.44492824062303) bank30933 +30934 POINT(41.436691770633004 73.71586920089867) bank30934 +30935 POINT(40.84642716141373 73.4574323537728) bank30935 +30936 POINT(41.36884007363732 73.93051690237219) bank30936 +30937 POINT(40.25233238365609 74.93186746988837) bank30937 +30938 POINT(39.748708282994414 73.26167347833523) bank30938 +30939 POINT(41.089598436604334 74.35686383489939) bank30939 +30940 POINT(40.28472581928266 74.40585868009163) bank30940 +30941 POINT(41.572799395658 74.44676945862408) bank30941 +30942 POINT(40.76667356253994 74.43745062756469) bank30942 +30943 POINT(41.19558353842182 74.33309740312156) bank30943 +30944 POINT(40.330907571309375 73.27013298216852) bank30944 +30945 POINT(40.024125233531386 73.48896206019171) bank30945 +30946 POINT(40.8169679371639 74.65294627784078) bank30946 +30947 POINT(41.369670684563566 74.81103021409284) bank30947 +30948 POINT(41.669150106755545 74.75910183484127) bank30948 +30949 POINT(40.948927006376714 73.12282759964155) bank30949 +30950 POINT(39.77155203976813 73.72979460284148) bank30950 +30951 POINT(40.63779407861238 73.24802373650427) bank30951 +30952 POINT(41.07837335825829 74.0219285185772) bank30952 +30953 POINT(39.832156348766425 74.07991883050012) bank30953 +30954 POINT(41.375670750592995 73.36047758652629) bank30954 +30955 POINT(40.29053240005975 74.07188138333107) bank30955 +30956 POINT(41.071194556176046 74.1102119777639) bank30956 +30957 POINT(40.888159723955965 73.22226685183405) bank30957 +30958 POINT(41.53006635695047 74.18855453446385) bank30958 +30959 POINT(39.83470947088846 73.14857027689371) bank30959 +30960 POINT(41.2432778940793 74.3963776805432) bank30960 +30961 POINT(39.909607784126145 73.68749392832058) bank30961 +30962 POINT(40.88522563371019 74.98683352211927) bank30962 +30963 POINT(40.167180660961996 73.16450272724423) bank30963 +30964 POINT(39.78721225716444 73.43240177149073) bank30964 +30965 POINT(41.257121313138896 74.38214029571763) bank30965 +30966 POINT(40.80526987691684 73.8610440302682) bank30966 +30967 POINT(40.62670515100892 73.7766025650381) bank30967 +30968 POINT(40.82407047119977 74.63452933725246) bank30968 +30969 POINT(41.60287014065404 73.10017723238005) bank30969 +30970 POINT(39.78753916344485 73.63226747388607) bank30970 +30971 POINT(40.69465887883974 73.12241894562327) bank30971 +30972 POINT(41.20104322271639 74.248359098032) bank30972 +30973 POINT(41.327907022287555 73.9741272487374) bank30973 +30974 POINT(41.437421120165226 74.59256070354182) bank30974 +30975 POINT(41.46612220003492 73.79703568889732) bank30975 +30976 POINT(39.8418712974568 73.42488170010807) bank30976 +30977 POINT(41.16066740524254 74.15310237037147) bank30977 +30978 POINT(40.63022387030305 73.53206618935997) bank30978 +30979 POINT(41.25870035403349 73.98729812835651) bank30979 +30980 POINT(40.30749349843044 73.35012007013874) bank30980 +30981 POINT(41.637214629080404 74.92043608387208) bank30981 +30982 POINT(40.75775096146336 73.17651863411554) bank30982 +30983 POINT(41.056251985908 73.95374839240772) bank30983 +30984 POINT(41.4752723264544 73.24031231740379) bank30984 +30985 POINT(41.38143379362629 74.90310564269285) bank30985 +30986 POINT(39.926598535533195 73.47059280869672) bank30986 +30987 POINT(39.857747137095544 74.75011929607177) bank30987 +30988 POINT(40.14314716432038 74.46651979039767) bank30988 +30989 POINT(40.38361873878823 73.41294503944486) bank30989 +30990 POINT(40.784169999299706 74.2740544268358) bank30990 +30991 POINT(40.77578767028691 73.07503035377421) bank30991 +30992 POINT(41.33745606550954 73.39606865423927) bank30992 +30993 POINT(41.13781899968545 73.04081788129845) bank30993 +30994 POINT(41.64764226914248 74.1913425355186) bank30994 +30995 POINT(40.111609439883814 74.14438280741548) bank30995 +30996 POINT(41.55740930976098 73.17645770798279) bank30996 +30997 POINT(40.23769963378882 73.66938307353418) bank30997 +30998 POINT(40.36638778592091 74.63234576818367) bank30998 +30999 POINT(39.8694571966346 74.96326667700865) bank30999 +31000 POINT(40.342199089523774 74.41942161789014) bank31000 +31001 POINT(41.10314134806356 74.43243810999792) bank31001 +31002 POINT(40.2688090690282 74.49436223081103) bank31002 +31003 POINT(41.52216707972477 74.30007986746453) bank31003 +31004 POINT(40.23368348827458 73.04667077534322) bank31004 +31005 POINT(39.99999874818293 73.59514831557388) bank31005 +31006 POINT(39.83958384906085 74.86431863391448) bank31006 +31007 POINT(41.57341084758412 73.59620256981435) bank31007 +31008 POINT(39.75910344629214 74.0484897284545) bank31008 +31009 POINT(40.02941594888152 74.25598775699889) bank31009 +31010 POINT(39.75155763080345 73.49422362289924) bank31010 +31011 POINT(39.774165083874664 74.08239287440479) bank31011 +31012 POINT(39.81778803305831 74.77879737829787) bank31012 +31013 POINT(39.84725513186062 73.4353667149045) bank31013 +31014 POINT(40.1758702217049 73.54265593013194) bank31014 +31015 POINT(41.62978419333482 74.84709444839243) bank31015 +31016 POINT(39.847179646736365 73.64264646204876) bank31016 +31017 POINT(40.591584477011025 74.96387236286644) bank31017 +31018 POINT(41.494372545423275 74.68703659972819) bank31018 +31019 POINT(41.22299402593919 74.38103674536556) bank31019 +31020 POINT(41.10752274868476 73.9482661570933) bank31020 +31021 POINT(39.93052546850584 74.87107486633549) bank31021 +31022 POINT(40.4427639572166 74.66638743395889) bank31022 +31023 POINT(41.49443068840398 74.6621613777582) bank31023 +31024 POINT(41.1471010405592 74.12403118311646) bank31024 +31025 POINT(41.447971082619986 74.6927548701279) bank31025 +31026 POINT(40.04818475026167 74.73075142848069) bank31026 +31027 POINT(40.57369944360501 74.16192600050759) bank31027 +31028 POINT(40.37072556353977 74.61365125225927) bank31028 +31029 POINT(40.91996425215349 74.98811138892692) bank31029 +31030 POINT(41.27429826919014 74.97498081108843) bank31030 +31031 POINT(41.27073801970735 74.25202372913039) bank31031 +31032 POINT(40.770662677672235 73.8520092990152) bank31032 +31033 POINT(41.60737714378356 73.23744733350011) bank31033 +31034 POINT(40.58687971510369 73.31481959206513) bank31034 +31035 POINT(41.40652731536007 74.71260339208757) bank31035 +31036 POINT(40.471200169963154 74.55849824345987) bank31036 +31037 POINT(41.408075197831614 73.39139317816357) bank31037 +31038 POINT(40.539528283470894 74.47598351857535) bank31038 +31039 POINT(40.338468585665495 73.99844556753546) bank31039 +31040 POINT(40.559759812472485 73.66655775729471) bank31040 +31041 POINT(39.74625772530332 73.01154648742113) bank31041 +31042 POINT(41.01441072300902 74.81108107162936) bank31042 +31043 POINT(40.52061808535079 73.29221465233579) bank31043 +31044 POINT(41.45113450757365 74.34859193372222) bank31044 +31045 POINT(40.10461796814581 73.65571805324322) bank31045 +31046 POINT(39.7400008510319 73.20554428510269) bank31046 +31047 POINT(40.94272965839535 73.55547460756011) bank31047 +31048 POINT(40.76091446835566 73.37057456812141) bank31048 +31049 POINT(40.896023245146964 73.8381603446439) bank31049 +31050 POINT(39.827972399549836 73.69540307508835) bank31050 +31051 POINT(41.31836587897081 73.95808283855922) bank31051 +31052 POINT(41.690381704999126 73.27830139568478) bank31052 +31053 POINT(39.85515997577475 73.01520562922894) bank31053 +31054 POINT(41.20247123092464 74.75258676272536) bank31054 +31055 POINT(41.59753759526477 74.21920426206485) bank31055 +31056 POINT(41.06157911091327 74.12661126731743) bank31056 +31057 POINT(41.63930248106293 73.55557741492176) bank31057 +31058 POINT(40.24463459217734 73.5761527567588) bank31058 +31059 POINT(41.17229706722496 74.34556121427548) bank31059 +31060 POINT(40.010537382989405 74.27798151273358) bank31060 +31061 POINT(40.640483088544464 73.37764189899241) bank31061 +31062 POINT(41.41935758929358 73.3276900641963) bank31062 +31063 POINT(41.63083988372382 73.35841779580765) bank31063 +31064 POINT(40.89453588723276 73.05527614904533) bank31064 +31065 POINT(40.274217300752525 74.61970229593128) bank31065 +31066 POINT(41.017312604686914 73.00893942178456) bank31066 +31067 POINT(41.34236342175459 74.81290907016897) bank31067 +31068 POINT(40.20962930403397 73.14846525817256) bank31068 +31069 POINT(40.79057425433632 74.13843062530526) bank31069 +31070 POINT(40.81304851197624 73.80348111751607) bank31070 +31071 POINT(40.8494367319107 74.3116809957117) bank31071 +31072 POINT(40.4471964557731 74.71403161907777) bank31072 +31073 POINT(41.43799998825875 73.20891258673034) bank31073 +31074 POINT(40.2111684434856 73.29489986690521) bank31074 +31075 POINT(40.98450996643479 74.65224174500615) bank31075 +31076 POINT(40.571789574080135 74.48485998426885) bank31076 +31077 POINT(40.01832627402586 74.96077790616393) bank31077 +31078 POINT(40.04510960052261 73.7976697828497) bank31078 +31079 POINT(40.61208314875739 73.73550693253917) bank31079 +31080 POINT(39.82155194106845 74.9977600839878) bank31080 +31081 POINT(41.38080087815315 73.69794548030633) bank31081 +31082 POINT(40.694781057070564 73.76050687290028) bank31082 +31083 POINT(40.59301631628585 74.59472322398636) bank31083 +31084 POINT(39.8815116084935 74.2374866362891) bank31084 +31085 POINT(40.565106091793666 73.41709089754984) bank31085 +31086 POINT(41.44552130077048 73.76851161629918) bank31086 +31087 POINT(39.744815771394855 73.31824143891178) bank31087 +31088 POINT(40.96847686884931 73.99347803398362) bank31088 +31089 POINT(40.21404044290017 73.12734162891405) bank31089 +31090 POINT(40.13449084582353 73.03529632103267) bank31090 +31091 POINT(40.371624916950694 74.877687537805) bank31091 +31092 POINT(40.87989186507629 74.15791671618277) bank31092 +31093 POINT(39.93814107006305 73.79971061324429) bank31093 +31094 POINT(40.707757984648545 74.59342739123213) bank31094 +31095 POINT(40.37982075726314 73.9123495337336) bank31095 +31096 POINT(40.211422026711354 73.38737981680511) bank31096 +31097 POINT(40.50216172068521 73.40530389445172) bank31097 +31098 POINT(40.65778558347659 74.21818367085228) bank31098 +31099 POINT(40.736518166492324 73.48024823722596) bank31099 +31100 POINT(39.864323021301466 73.2930955422915) bank31100 +31101 POINT(40.04608190915156 73.71589555393922) bank31101 +31102 POINT(40.53085659339406 74.2192295928029) bank31102 +31103 POINT(40.20937044628254 74.14192684455367) bank31103 +31104 POINT(41.243359410952735 73.86366080834962) bank31104 +31105 POINT(40.68497335763341 73.2085844960255) bank31105 +31106 POINT(41.09831534069701 74.41927747790197) bank31106 +31107 POINT(40.68644314344953 73.0106401841658) bank31107 +31108 POINT(39.873740799920995 73.44095804988565) bank31108 +31109 POINT(41.62124034778368 73.34549430251096) bank31109 +31110 POINT(39.961608874154685 73.1799884742804) bank31110 +31111 POINT(40.81077413951768 73.68193490831273) bank31111 +31112 POINT(40.73116774977255 74.61505386123791) bank31112 +31113 POINT(41.61113172135921 74.99665661582408) bank31113 +31114 POINT(41.20183206034196 73.42110719200117) bank31114 +31115 POINT(40.95604176973871 73.57968218106939) bank31115 +31116 POINT(40.61210340909399 74.49783958906413) bank31116 +31117 POINT(39.935432078928244 74.0087082062473) bank31117 +31118 POINT(41.656754276406836 74.56025775396697) bank31118 +31119 POINT(41.30383455971623 73.26656179026632) bank31119 +31120 POINT(41.59573329281169 73.66700557852923) bank31120 +31121 POINT(39.89076033504634 74.87773285041132) bank31121 +31122 POINT(40.81731637796086 74.33542418721436) bank31122 +31123 POINT(40.384967677853204 74.49646375414632) bank31123 +31124 POINT(41.20292226009042 74.83281455538533) bank31124 +31125 POINT(40.74700033809214 74.94665522086765) bank31125 +31126 POINT(41.24471955689219 74.82785425643658) bank31126 +31127 POINT(41.40531487086985 74.36825423824247) bank31127 +31128 POINT(40.44736551962398 74.07978520647156) bank31128 +31129 POINT(39.97681557515656 73.04800552020323) bank31129 +31130 POINT(41.044632286623795 74.47107072949176) bank31130 +31131 POINT(41.10980793986716 74.04206012688667) bank31131 +31132 POINT(40.646942637314496 74.65186462581983) bank31132 +31133 POINT(40.17919778963066 74.85287677706134) bank31133 +31134 POINT(40.239678122525575 74.30069893522827) bank31134 +31135 POINT(41.700706030019774 73.6446292199958) bank31135 +31136 POINT(39.726630223220766 74.00174199416634) bank31136 +31137 POINT(40.94730908702766 74.43693304334246) bank31137 +31138 POINT(40.105275244705176 74.97066303788121) bank31138 +31139 POINT(41.59043222147943 73.74611074151215) bank31139 +31140 POINT(40.58058463643304 73.43946084381413) bank31140 +31141 POINT(41.28197282846142 74.9248815767114) bank31141 +31142 POINT(41.6868784456859 73.0158750907287) bank31142 +31143 POINT(40.111150170837725 73.50610958206009) bank31143 +31144 POINT(39.90325981622299 73.80010561937327) bank31144 +31145 POINT(40.34404403127782 74.55815831212854) bank31145 +31146 POINT(39.87209487167764 74.83552739319894) bank31146 +31147 POINT(40.22753214669427 73.328594691171) bank31147 +31148 POINT(40.11598018795099 73.49070442208534) bank31148 +31149 POINT(40.237905006385674 73.14812810465567) bank31149 +31150 POINT(41.094573098680925 74.63995614610027) bank31150 +31151 POINT(39.754468070104636 74.97192422290513) bank31151 +31152 POINT(40.774993194225296 73.40134376513949) bank31152 +31153 POINT(39.90380007865233 73.74682874452368) bank31153 +31154 POINT(41.50863134179756 73.77790241258384) bank31154 +31155 POINT(41.35400359847287 73.19810258331714) bank31155 +31156 POINT(40.66801863893585 73.10475337042114) bank31156 +31157 POINT(39.88354435958834 73.77825216318543) bank31157 +31158 POINT(41.597773073832364 74.50817523589383) bank31158 +31159 POINT(40.02746325959661 74.30139851033765) bank31159 +31160 POINT(40.952776429836206 73.12131851765332) bank31160 +31161 POINT(40.598922971586866 74.13116922242934) bank31161 +31162 POINT(40.80283268069897 74.15476205121654) bank31162 +31163 POINT(40.82393412867844 73.25033100600918) bank31163 +31164 POINT(41.12696297953837 74.25503815183974) bank31164 +31165 POINT(39.925626312350126 74.81750385989223) bank31165 +31166 POINT(39.94099273568711 73.65123329316822) bank31166 +31167 POINT(41.65447618539416 73.28340057303079) bank31167 +31168 POINT(39.96483799288273 74.9805726953841) bank31168 +31169 POINT(40.18500019318933 74.15076228613933) bank31169 +31170 POINT(41.58056999702036 73.25046027549288) bank31170 +31171 POINT(40.7786754106003 74.29877817851015) bank31171 +31172 POINT(41.246821314568436 74.67735599756995) bank31172 +31173 POINT(40.56723933902301 74.3144562316307) bank31173 +31174 POINT(40.12112443613465 74.24009307373323) bank31174 +31175 POINT(40.61137557604694 74.63906452813029) bank31175 +31176 POINT(40.1213391833997 73.39755208551904) bank31176 +31177 POINT(41.021121147641594 73.27760528959618) bank31177 +31178 POINT(40.616384018593855 73.82520629224601) bank31178 +31179 POINT(39.86913976963707 74.96880102566517) bank31179 +31180 POINT(40.60916804406801 74.168027218822) bank31180 +31181 POINT(40.40014742277504 74.27137978006034) bank31181 +31182 POINT(40.06553850585937 74.02704856246615) bank31182 +31183 POINT(41.478889517706804 73.1532732909974) bank31183 +31184 POINT(40.09218326195717 74.87972381355127) bank31184 +31185 POINT(40.04792365551791 75.00074146145339) bank31185 +31186 POINT(39.9747373676187 73.59605619975898) bank31186 +31187 POINT(39.98901674218991 74.53634818890205) bank31187 +31188 POINT(39.89355760034349 74.42415599370712) bank31188 +31189 POINT(40.43873098741139 73.98929212235731) bank31189 +31190 POINT(39.865261394081074 74.23552894687421) bank31190 +31191 POINT(40.820384953203664 74.08180244768549) bank31191 +31192 POINT(40.026438251206294 74.82819634908618) bank31192 +31193 POINT(40.85221233539599 74.65527382867272) bank31193 +31194 POINT(40.835988587662484 74.5476302766226) bank31194 +31195 POINT(41.35970132775115 73.43020862590942) bank31195 +31196 POINT(40.269753804138645 73.75608370339367) bank31196 +31197 POINT(41.474837073673605 74.73906913049431) bank31197 +31198 POINT(41.62261156521358 74.36078876586873) bank31198 +31199 POINT(41.398996443630544 74.13176557783459) bank31199 +31200 POINT(39.88843198875541 73.1078715504333) bank31200 +31201 POINT(40.662790682792675 73.44531132707068) bank31201 +31202 POINT(41.6940824441295 73.071288830679) bank31202 +31203 POINT(41.011499744055165 73.55087525567723) bank31203 +31204 POINT(39.95327825060231 74.58843407728774) bank31204 +31205 POINT(41.25783789248996 73.68943139577358) bank31205 +31206 POINT(40.041312418077766 73.28118899934766) bank31206 +31207 POINT(40.71837682902127 74.27233925314829) bank31207 +31208 POINT(39.939863714933075 73.72421330059203) bank31208 +31209 POINT(41.442060161805486 74.4750326560767) bank31209 +31210 POINT(40.16055810103366 74.7516785593368) bank31210 +31211 POINT(40.90333101019599 73.54999322527698) bank31211 +31212 POINT(40.70963741631099 74.07391302643678) bank31212 +31213 POINT(40.588182823640025 74.82725361854365) bank31213 +31214 POINT(40.05797916704838 74.93316356126866) bank31214 +31215 POINT(40.961503339926466 73.88723998243404) bank31215 +31216 POINT(40.02609971342541 73.93257127609014) bank31216 +31217 POINT(40.44010927579748 74.31829074446362) bank31217 +31218 POINT(40.26947681626429 74.212227941913) bank31218 +31219 POINT(40.92643379775423 74.977506435249) bank31219 +31220 POINT(40.683218985798256 73.8533877231206) bank31220 +31221 POINT(41.552340159885595 74.36648668144818) bank31221 +31222 POINT(40.49368559466616 74.83527172469057) bank31222 +31223 POINT(40.665620665150826 73.92011574050618) bank31223 +31224 POINT(41.28599931014939 74.5858176746689) bank31224 +31225 POINT(41.393981533013516 74.85193820491827) bank31225 +31226 POINT(41.07011103708332 74.0642948941662) bank31226 +31227 POINT(40.50443632561305 73.78879356079386) bank31227 +31228 POINT(41.08338513325319 73.19206682058812) bank31228 +31229 POINT(39.90729364369818 73.57191627134203) bank31229 +31230 POINT(41.64879230837344 73.67316375911221) bank31230 +31231 POINT(40.552499864691775 74.32852265570142) bank31231 +31232 POINT(41.6393849475061 74.56002816068022) bank31232 +31233 POINT(39.88416100035141 74.38749536785144) bank31233 +31234 POINT(40.43651811950447 73.40895378648825) bank31234 +31235 POINT(40.044311222893334 74.98893905289253) bank31235 +31236 POINT(40.67097713240885 73.35594826169137) bank31236 +31237 POINT(41.00038724249411 74.90480476890545) bank31237 +31238 POINT(40.277738054776364 74.012499916397) bank31238 +31239 POINT(41.230875119946134 73.60610307813035) bank31239 +31240 POINT(41.130359441566895 73.44472985681503) bank31240 +31241 POINT(41.45809961626902 75.00391142186504) bank31241 +31242 POINT(40.91763123504851 74.44981799919292) bank31242 +31243 POINT(40.48393371108094 74.32055210035264) bank31243 +31244 POINT(40.338794263741455 73.0168981065121) bank31244 +31245 POINT(40.57097084634505 73.04676873858467) bank31245 +31246 POINT(40.76070590598958 74.372103464452) bank31246 +31247 POINT(41.22191827529682 73.12853313303481) bank31247 +31248 POINT(40.23166501895832 74.02867986583101) bank31248 +31249 POINT(41.25791350771047 73.26492253991705) bank31249 +31250 POINT(41.65650965943909 74.24325129218174) bank31250 +31251 POINT(39.95709248887286 73.66869302135407) bank31251 +31252 POINT(41.3124209122137 74.44169453744317) bank31252 +31253 POINT(39.98504668482204 73.6268199587319) bank31253 +31254 POINT(40.62897079230487 74.94839348336646) bank31254 +31255 POINT(40.51485873086193 73.1764993439949) bank31255 +31256 POINT(40.370968545111054 73.05913144792963) bank31256 +31257 POINT(40.41923439559074 74.02814268132923) bank31257 +31258 POINT(39.86841076839784 73.21786165124341) bank31258 +31259 POINT(40.630392025619166 73.26639029578251) bank31259 +31260 POINT(41.098902008192496 73.09871834766945) bank31260 +31261 POINT(40.50338985121233 73.03489894068699) bank31261 +31262 POINT(41.0121447185427 74.02773211352485) bank31262 +31263 POINT(40.98328854988259 73.53635270124822) bank31263 +31264 POINT(40.114601241936626 73.61756269849145) bank31264 +31265 POINT(40.06277158969992 74.00020774525187) bank31265 +31266 POINT(39.80147394323471 74.31179440519627) bank31266 +31267 POINT(41.590986295081024 74.27096825694402) bank31267 +31268 POINT(40.45189037636085 73.52433395859939) bank31268 +31269 POINT(40.60057688218438 73.06404006906978) bank31269 +31270 POINT(41.332205688961466 74.25541849462707) bank31270 +31271 POINT(40.240131371727166 73.16233661796413) bank31271 +31272 POINT(40.06259281999445 74.85865917672805) bank31272 +31273 POINT(40.748324794925466 74.67712211759144) bank31273 +31274 POINT(41.403912225329556 74.7829877042722) bank31274 +31275 POINT(41.05819647785449 73.82355907361915) bank31275 +31276 POINT(41.71105669374216 74.4990546186069) bank31276 +31277 POINT(40.26384686555217 73.03282116129081) bank31277 +31278 POINT(40.30021173643939 74.98571898168863) bank31278 +31279 POINT(39.878187438960296 73.24816198182918) bank31279 +31280 POINT(40.02491644848933 73.89475897391992) bank31280 +31281 POINT(40.010909977749954 73.68960417596266) bank31281 +31282 POINT(40.23655267046785 74.5573704696991) bank31282 +31283 POINT(41.06312397430528 73.10681012600489) bank31283 +31284 POINT(40.4609166053733 74.81638443121945) bank31284 +31285 POINT(40.155001874253635 73.85892790129134) bank31285 +31286 POINT(40.79953617785266 74.74059709688352) bank31286 +31287 POINT(39.82205938281226 73.82911150439638) bank31287 +31288 POINT(41.23705938158333 73.95539205311448) bank31288 +31289 POINT(40.936304062396225 74.28881245673503) bank31289 +31290 POINT(41.12186016105844 74.49249678828052) bank31290 +31291 POINT(39.8513263936939 74.92403521976212) bank31291 +31292 POINT(41.53750408301859 73.78594438582238) bank31292 +31293 POINT(41.30269030635556 74.18132818429301) bank31293 +31294 POINT(41.05793731589523 73.13074196144288) bank31294 +31295 POINT(40.46253996209144 73.94985344053593) bank31295 +31296 POINT(40.17599320191898 73.65367410527806) bank31296 +31297 POINT(41.71277589609514 74.79156026071965) bank31297 +31298 POINT(41.6750833367422 74.83912379351558) bank31298 +31299 POINT(41.19594495717288 73.71925785360115) bank31299 +31300 POINT(41.60052730510207 74.93269444486134) bank31300 +31301 POINT(41.36188821394068 74.15575797425842) bank31301 +31302 POINT(39.83845293082645 74.20850780421851) bank31302 +31303 POINT(41.48526331425677 74.960010355575) bank31303 +31304 POINT(40.899629591618464 74.18769651852743) bank31304 +31305 POINT(39.91843925905451 73.7374343680454) bank31305 +31306 POINT(40.21600815459926 74.89606954448318) bank31306 +31307 POINT(39.747838167379214 74.31376705700207) bank31307 +31308 POINT(41.650140905874636 74.61978692082597) bank31308 +31309 POINT(41.05940965780351 73.77553858989012) bank31309 +31310 POINT(40.496008550110474 74.15095606448051) bank31310 +31311 POINT(39.94252607659775 73.43629914727603) bank31311 +31312 POINT(40.66676838751804 73.20118651115403) bank31312 +31313 POINT(40.18521217538173 74.97226179808708) bank31313 +31314 POINT(39.79571942679768 73.24081859485436) bank31314 +31315 POINT(41.52486231135046 73.78727602444076) bank31315 +31316 POINT(41.209599712134775 73.415541278224) bank31316 +31317 POINT(40.77665196763164 74.53316352252108) bank31317 +31318 POINT(41.033606626438456 73.65902213256192) bank31318 +31319 POINT(40.584015899113886 74.18762039036896) bank31319 +31320 POINT(40.24559572506507 74.50185013929982) bank31320 +31321 POINT(41.43474207895879 74.79808720515217) bank31321 +31322 POINT(40.29964290354341 74.94085624572392) bank31322 +31323 POINT(40.9715676169547 73.17440029278467) bank31323 +31324 POINT(41.357153267882865 73.76559923144221) bank31324 +31325 POINT(41.001492564854956 74.07839546606897) bank31325 +31326 POINT(39.9729372812635 73.12359817533267) bank31326 +31327 POINT(40.77429260345457 73.63276448899838) bank31327 +31328 POINT(40.966010364013556 73.56887236675088) bank31328 +31329 POINT(40.1962371575846 74.81951346187442) bank31329 +31330 POINT(39.884353538805065 74.98805174061313) bank31330 +31331 POINT(40.33893128314359 73.86082537554995) bank31331 +31332 POINT(41.2748105176426 73.6539159953065) bank31332 +31333 POINT(40.178903632129774 73.45415608980053) bank31333 +31334 POINT(40.51692471624643 74.10273509925779) bank31334 +31335 POINT(39.921238763250294 74.17698829086407) bank31335 +31336 POINT(40.857855635981146 74.38606280205141) bank31336 +31337 POINT(41.21393334068172 74.57242698865267) bank31337 +31338 POINT(40.65417242797418 74.49586211845731) bank31338 +31339 POINT(40.72796612793946 74.45781753759249) bank31339 +31340 POINT(40.891214564964464 74.7735155061973) bank31340 +31341 POINT(41.240556937507286 74.4208664040833) bank31341 +31342 POINT(41.56990993349245 74.47606766624651) bank31342 +31343 POINT(41.426724970656565 73.43205248372387) bank31343 +31344 POINT(41.605291799743746 73.60015833996806) bank31344 +31345 POINT(40.44555798886735 74.70219645914618) bank31345 +31346 POINT(40.128771701899 74.60811242973098) bank31346 +31347 POINT(40.02395904464266 73.96495629955322) bank31347 +31348 POINT(41.65150772912163 74.06551600823762) bank31348 +31349 POINT(40.33240516442513 73.64375165736264) bank31349 +31350 POINT(40.57898343790323 74.84949898926529) bank31350 +31351 POINT(41.66535501442021 74.6547828849782) bank31351 +31352 POINT(41.542260121952324 73.19787132493678) bank31352 +31353 POINT(41.171570643422555 73.19266515650736) bank31353 +31354 POINT(40.37608082888208 74.4428669315349) bank31354 +31355 POINT(40.82638307810782 74.98988798786164) bank31355 +31356 POINT(39.86945125206023 73.64877094592237) bank31356 +31357 POINT(40.924862512914515 73.28364978615325) bank31357 +31358 POINT(40.09593777509208 74.08500030831647) bank31358 +31359 POINT(41.56830058162159 74.74491128365337) bank31359 +31360 POINT(39.936694036038446 74.50918084976989) bank31360 +31361 POINT(41.447991380939136 74.22464252434975) bank31361 +31362 POINT(41.356420857794326 73.472598208973) bank31362 +31363 POINT(39.77658171349392 73.17715065912017) bank31363 +31364 POINT(41.36504248754605 73.80256329354935) bank31364 +31365 POINT(40.17579398597714 75.00188834827411) bank31365 +31366 POINT(39.865315576604225 73.84184484985246) bank31366 +31367 POINT(40.721907338183414 73.74925241175364) bank31367 +31368 POINT(40.668875938993594 74.96400997821749) bank31368 +31369 POINT(40.55800254935789 74.61746413845718) bank31369 +31370 POINT(40.15576890707552 73.49389314591258) bank31370 +31371 POINT(41.436285544880405 73.96812898258119) bank31371 +31372 POINT(41.453169559674514 74.0404319542409) bank31372 +31373 POINT(41.33823916503369 74.89971095605317) bank31373 +31374 POINT(40.65549522061412 73.38716784334352) bank31374 +31375 POINT(41.64473921317529 74.2291180957997) bank31375 +31376 POINT(41.51498314314218 74.529931345188) bank31376 +31377 POINT(41.62612365399698 73.08523809070411) bank31377 +31378 POINT(41.00646489694227 73.39718861319253) bank31378 +31379 POINT(40.70378394021917 74.8827125510681) bank31379 +31380 POINT(40.686139551427196 74.26015952210348) bank31380 +31381 POINT(41.063260061604154 73.4312375329062) bank31381 +31382 POINT(39.8829317760466 73.4529511494309) bank31382 +31383 POINT(41.24365616844434 73.93144988610332) bank31383 +31384 POINT(40.2388151627925 74.26536751691859) bank31384 +31385 POINT(40.110446037989355 73.62805165541545) bank31385 +31386 POINT(40.014700515696404 73.97697762517893) bank31386 +31387 POINT(41.036920613111505 73.26368757581821) bank31387 +31388 POINT(41.40582407448206 73.54763648874123) bank31388 +31389 POINT(40.49535006145534 74.54020832636535) bank31389 +31390 POINT(40.400074116182026 73.98978876821846) bank31390 +31391 POINT(40.33149030324698 73.26819690221187) bank31391 +31392 POINT(40.91396006325083 74.22847386951847) bank31392 +31393 POINT(41.47772199650333 73.41942969134197) bank31393 +31394 POINT(40.58211437485979 74.52642889751634) bank31394 +31395 POINT(40.732534191773546 74.02796451113102) bank31395 +31396 POINT(41.03898185985183 73.81731333042686) bank31396 +31397 POINT(41.32007212051369 74.43947958220612) bank31397 +31398 POINT(40.564956424428935 73.32795359852302) bank31398 +31399 POINT(40.907630823083906 74.46893556289257) bank31399 +31400 POINT(41.43144027652669 74.66454431119894) bank31400 +31401 POINT(39.811376081849886 73.21100698616803) bank31401 +31402 POINT(39.910470769561535 74.59560251211266) bank31402 +31403 POINT(41.25579555439344 74.07968799802784) bank31403 +31404 POINT(40.46429455173293 73.88370899067294) bank31404 +31405 POINT(41.458083332114846 74.03670442390988) bank31405 +31406 POINT(41.0925888964408 74.3752591351313) bank31406 +31407 POINT(40.38693655006031 74.98113004936172) bank31407 +31408 POINT(40.169107520280846 74.9495171045282) bank31408 +31409 POINT(41.266384590863325 73.69698488696739) bank31409 +31410 POINT(41.34704632736337 73.929195955339) bank31410 +31411 POINT(41.697538075738606 73.3694854319572) bank31411 +31412 POINT(40.85556085052186 73.59319366802677) bank31412 +31413 POINT(41.06144240235595 74.45074868574454) bank31413 +31414 POINT(41.22615003350275 73.1820265976319) bank31414 +31415 POINT(41.10186327829781 73.70427916464827) bank31415 +31416 POINT(39.9546533951507 74.14844084867752) bank31416 +31417 POINT(39.9447548243256 73.89046879712076) bank31417 +31418 POINT(39.7553840701042 74.82061523467954) bank31418 +31419 POINT(39.76196896580218 74.98818653270386) bank31419 +31420 POINT(41.39116471402153 74.1509334109338) bank31420 +31421 POINT(40.7784914802937 74.69593960118756) bank31421 +31422 POINT(39.82689498921571 73.52850761134579) bank31422 +31423 POINT(39.85773514874923 74.41441520271486) bank31423 +31424 POINT(40.30433170739674 73.17302386056247) bank31424 +31425 POINT(41.19092856246353 73.88543595516255) bank31425 +31426 POINT(39.74206638338723 74.09095685595219) bank31426 +31427 POINT(40.35492278747763 74.74542303880865) bank31427 +31428 POINT(40.45368852456136 74.5920954409831) bank31428 +31429 POINT(40.72465083046461 73.49259721060793) bank31429 +31430 POINT(40.93918174933096 73.57157421872094) bank31430 +31431 POINT(40.35446041820683 73.64639296198116) bank31431 +31432 POINT(39.91342570923501 73.29887780822192) bank31432 +31433 POINT(39.90989132730585 73.23582882702446) bank31433 +31434 POINT(41.0313792504436 74.88478135116142) bank31434 +31435 POINT(41.46185808903779 74.68468168364689) bank31435 +31436 POINT(40.15913255781622 74.72500752268598) bank31436 +31437 POINT(39.95039977580455 73.18074033292356) bank31437 +31438 POINT(41.23253299955284 75.00253489773506) bank31438 +31439 POINT(40.99115860391282 74.11997813941919) bank31439 +31440 POINT(40.240749882053755 74.06205109222545) bank31440 +31441 POINT(40.13141101544949 74.74234869129262) bank31441 +31442 POINT(39.96510924527183 74.72248875494637) bank31442 +31443 POINT(40.954151732789704 74.68085500800477) bank31443 +31444 POINT(40.5503041948574 73.12947135060845) bank31444 +31445 POINT(40.7572283497755 73.54437557244981) bank31445 +31446 POINT(39.873928626665844 74.77684697907358) bank31446 +31447 POINT(41.123280918349266 74.22469641241702) bank31447 +31448 POINT(41.32867026883662 73.56950715814152) bank31448 +31449 POINT(40.83408056426839 73.5802083571603) bank31449 +31450 POINT(40.63737048890777 74.27053146516809) bank31450 +31451 POINT(40.64138763111092 73.3076747302976) bank31451 +31452 POINT(40.46926445585358 74.99722169952672) bank31452 +31453 POINT(40.91113701970277 73.98834600691042) bank31453 +31454 POINT(41.17898981798236 74.85197991422658) bank31454 +31455 POINT(39.94732125657465 73.74897497536061) bank31455 +31456 POINT(41.278150064344835 73.4170922344044) bank31456 +31457 POINT(41.613254714576065 73.04489296211197) bank31457 +31458 POINT(40.62809987380007 73.26335346007833) bank31458 +31459 POINT(40.298118762191834 73.92399296946952) bank31459 +31460 POINT(39.930599465094474 73.18066645419806) bank31460 +31461 POINT(40.93404443200262 73.10752581611723) bank31461 +31462 POINT(41.09933304684359 74.99663110521573) bank31462 +31463 POINT(40.6003069150359 73.72525292828824) bank31463 +31464 POINT(40.08034205278491 73.40910671601333) bank31464 +31465 POINT(41.50513667486103 74.55599157743053) bank31465 +31466 POINT(41.38776989871749 73.57792712410998) bank31466 +31467 POINT(41.44679255723935 74.93266478817382) bank31467 +31468 POINT(41.54277123175419 74.08141270432253) bank31468 +31469 POINT(40.249588894481036 74.390488494266) bank31469 +31470 POINT(41.2055355961841 74.248925151008) bank31470 +31471 POINT(41.39033881789368 74.1104148975925) bank31471 +31472 POINT(40.20773407057573 73.23820722548436) bank31472 +31473 POINT(40.53271373060268 73.92459517413403) bank31473 +31474 POINT(41.628116251830356 74.96283125250092) bank31474 +31475 POINT(41.27272753857016 73.53497163526883) bank31475 +31476 POINT(40.63493217843844 73.45041446509336) bank31476 +31477 POINT(40.86594867039963 74.19147801450522) bank31477 +31478 POINT(41.49965410807048 74.54064213069061) bank31478 +31479 POINT(40.037431911827994 74.26168869494349) bank31479 +31480 POINT(41.65650664580157 73.86758320450956) bank31480 +31481 POINT(40.31559385815504 74.79253822758562) bank31481 +31482 POINT(41.4812178457449 74.29183627452025) bank31482 +31483 POINT(40.76888474124786 74.43585346765099) bank31483 +31484 POINT(40.60568166203415 73.59618935697291) bank31484 +31485 POINT(41.16647858648543 74.45496310899573) bank31485 +31486 POINT(40.27606437126886 74.41703701356245) bank31486 +31487 POINT(41.404962193042884 73.52874300605902) bank31487 +31488 POINT(40.500687687886625 74.7773345981259) bank31488 +31489 POINT(40.5484071289993 74.57730500185974) bank31489 +31490 POINT(41.35352780544382 73.79813170980108) bank31490 +31491 POINT(39.91865536795323 73.27867913335172) bank31491 +31492 POINT(41.485850123916514 74.40381749233649) bank31492 +31493 POINT(40.71064976573312 74.47494503909562) bank31493 +31494 POINT(40.08369759929795 73.6232665897438) bank31494 +31495 POINT(40.11031685810626 74.60772716773228) bank31495 +31496 POINT(39.81180593633461 73.09640657482137) bank31496 +31497 POINT(40.83093748034446 73.80795488509455) bank31497 +31498 POINT(40.25753803656952 73.01040247367679) bank31498 +31499 POINT(41.666767682622634 74.32980293360424) bank31499 +31500 POINT(41.240236254747735 74.33170077763558) bank31500 +31501 POINT(40.421762215209824 73.23707522021508) bank31501 +31502 POINT(40.727188410527624 74.57982911443845) bank31502 +31503 POINT(39.845756344271734 74.73219458448796) bank31503 +31504 POINT(41.01194998523859 74.7174580882308) bank31504 +31505 POINT(41.230615259415565 74.23823629686036) bank31505 +31506 POINT(39.77814387327378 73.93783522461139) bank31506 +31507 POINT(41.59428388846501 74.29782445155388) bank31507 +31508 POINT(40.84593295976024 73.62950028580593) bank31508 +31509 POINT(40.91103177815959 73.8290532558922) bank31509 +31510 POINT(40.01689221394754 73.89679755354464) bank31510 +31511 POINT(40.08941610225098 74.45488543716921) bank31511 +31512 POINT(41.64738213601057 73.2667287529275) bank31512 +31513 POINT(40.430208785968254 74.12930878118756) bank31513 +31514 POINT(40.4854076577703 73.45918733172645) bank31514 +31515 POINT(41.69801516700394 74.90634775147817) bank31515 +31516 POINT(40.54344735531301 73.05483509017193) bank31516 +31517 POINT(41.1669894690845 73.89474887032189) bank31517 +31518 POINT(41.21593268195126 74.54500375311387) bank31518 +31519 POINT(40.18266787972039 73.74263376996558) bank31519 +31520 POINT(40.33214860587651 73.95298449595732) bank31520 +31521 POINT(39.777494313156275 73.13065906768331) bank31521 +31522 POINT(41.531250045401855 74.89361306841258) bank31522 +31523 POINT(40.91855502682082 74.24928414192135) bank31523 +31524 POINT(40.97779711832523 74.73151778917455) bank31524 +31525 POINT(40.851783070576545 73.14173291512785) bank31525 +31526 POINT(40.515920459277304 73.62444783790865) bank31526 +31527 POINT(40.877932401747245 73.15073734367633) bank31527 +31528 POINT(41.20522540260546 74.61617177583919) bank31528 +31529 POINT(39.96867454892632 74.48992946332928) bank31529 +31530 POINT(39.82647894425548 73.5286571027262) bank31530 +31531 POINT(41.401991682874545 74.77405537677404) bank31531 +31532 POINT(40.93718382236019 74.38184016696901) bank31532 +31533 POINT(39.71444645382485 73.91062831774858) bank31533 +31534 POINT(40.25823127488974 74.06487975872069) bank31534 +31535 POINT(41.32089664847928 74.94409989722885) bank31535 +31536 POINT(40.331462804120335 74.79622045378753) bank31536 +31537 POINT(39.972165082117975 73.52719642906602) bank31537 +31538 POINT(40.028139956734044 74.10212594224747) bank31538 +31539 POINT(40.097669210104335 74.89881362403715) bank31539 +31540 POINT(40.83054586835032 74.24958299784922) bank31540 +31541 POINT(41.03148250859883 74.04387706504285) bank31541 +31542 POINT(40.74271564554931 74.57755000000277) bank31542 +31543 POINT(40.76368753001742 73.32278625442281) bank31543 +31544 POINT(40.070433617666666 73.96980384646403) bank31544 +31545 POINT(39.97988277474471 74.06621321143126) bank31545 +31546 POINT(40.78696903106613 74.79094357254533) bank31546 +31547 POINT(40.728566023222065 74.75397343763164) bank31547 +31548 POINT(41.4953210900932 74.19599558649803) bank31548 +31549 POINT(40.88380069309159 73.59224528904244) bank31549 +31550 POINT(41.522226659682666 74.3085158164228) bank31550 +31551 POINT(41.70308913278338 74.00336138268446) bank31551 +31552 POINT(40.73951714163944 73.17635737561324) bank31552 +31553 POINT(41.368675020941005 74.58347184401413) bank31553 +31554 POINT(41.47130647336949 74.52663987458638) bank31554 +31555 POINT(39.739236889148025 74.04712625488278) bank31555 +31556 POINT(40.0927377782599 74.58071099013043) bank31556 +31557 POINT(41.43802021236094 73.50187632999356) bank31557 +31558 POINT(41.43715057087704 74.4766800913865) bank31558 +31559 POINT(40.62595332174622 73.62713314665515) bank31559 +31560 POINT(40.92727018976583 73.0876124494295) bank31560 +31561 POINT(41.55175675001452 73.33120740922737) bank31561 +31562 POINT(39.93182648062714 74.16573659120645) bank31562 +31563 POINT(40.60085999565197 73.54504174494832) bank31563 +31564 POINT(40.57391193637124 73.422069290628) bank31564 +31565 POINT(40.539028789035044 73.28048516960436) bank31565 +31566 POINT(40.11852816387093 73.47925177549003) bank31566 +31567 POINT(39.731412760655374 73.673711398982) bank31567 +31568 POINT(40.753809217586436 73.53127845886398) bank31568 +31569 POINT(41.405022895758115 74.20979619106016) bank31569 +31570 POINT(41.483910677011174 73.5271961556311) bank31570 +31571 POINT(40.15481462706685 73.74257786056339) bank31571 +31572 POINT(40.39528507618577 73.29151929453593) bank31572 +31573 POINT(41.23961756242877 73.58081063919039) bank31573 +31574 POINT(39.99393467552168 74.91589587387023) bank31574 +31575 POINT(40.62982793607622 73.8545892046144) bank31575 +31576 POINT(40.29692319730722 74.02776712948305) bank31576 +31577 POINT(41.48318150589917 74.14947860134738) bank31577 +31578 POINT(41.112507911462714 74.28130970805441) bank31578 +31579 POINT(39.7214951974524 74.14022604979132) bank31579 +31580 POINT(40.845093401096015 74.25408109044938) bank31580 +31581 POINT(40.182980729770364 74.99123185339273) bank31581 +31582 POINT(41.192380952933746 73.48647883675892) bank31582 +31583 POINT(41.69137840022259 74.34787769031169) bank31583 +31584 POINT(41.260537728067504 73.13852700022404) bank31584 +31585 POINT(41.07796409816349 74.53864781591192) bank31585 +31586 POINT(41.53453497923277 73.9585992108532) bank31586 +31587 POINT(39.91004730170723 73.46037052374398) bank31587 +31588 POINT(41.367728265893035 74.62584302042579) bank31588 +31589 POINT(39.86096961792966 73.05406503344238) bank31589 +31590 POINT(40.70478403979861 74.0251163336837) bank31590 +31591 POINT(40.34717106573419 74.45192195180002) bank31591 +31592 POINT(41.69233836858868 73.7184716449501) bank31592 +31593 POINT(40.614070880002956 73.53107131645227) bank31593 +31594 POINT(40.196291990277146 74.66067762045687) bank31594 +31595 POINT(40.77061914468175 73.22814395465184) bank31595 +31596 POINT(41.10449058364831 74.04105713177968) bank31596 +31597 POINT(41.593646087130416 74.20786678746954) bank31597 +31598 POINT(40.59247218758908 73.94101342812306) bank31598 +31599 POINT(40.951362066241686 73.30591065428253) bank31599 +31600 POINT(41.05906699609046 74.09056416241198) bank31600 +31601 POINT(40.13352257797516 74.56389170442823) bank31601 +31602 POINT(40.22494764928118 74.7941728525347) bank31602 +31603 POINT(40.30901822725877 73.48196213108784) bank31603 +31604 POINT(41.06931088995016 74.32109296646034) bank31604 +31605 POINT(41.17942284868281 74.9999596907901) bank31605 +31606 POINT(40.61827152048843 73.2048839981949) bank31606 +31607 POINT(40.82701771786163 73.9277954659247) bank31607 +31608 POINT(41.49863622635 73.1076566665188) bank31608 +31609 POINT(40.672367659508225 73.5278623528576) bank31609 +31610 POINT(41.324255440209306 73.53138851898686) bank31610 +31611 POINT(40.93539664579277 73.60621737397055) bank31611 +31612 POINT(40.51961500604373 74.11017966575808) bank31612 +31613 POINT(39.887944994954914 73.38405099356076) bank31613 +31614 POINT(40.120441428737685 74.84907522656468) bank31614 +31615 POINT(40.72819495785473 74.50137489091841) bank31615 +31616 POINT(39.92753361855072 73.74205862643277) bank31616 +31617 POINT(41.443836810305854 74.77503545149023) bank31617 +31618 POINT(40.19563766174735 73.35553074596763) bank31618 +31619 POINT(41.709246309265914 73.18161453792894) bank31619 +31620 POINT(40.27171788743863 74.95779216544412) bank31620 +31621 POINT(40.66276625849836 74.85983034430188) bank31621 +31622 POINT(40.00171433234681 73.10710211824814) bank31622 +31623 POINT(41.066374923207164 74.62065305638819) bank31623 +31624 POINT(41.55628920916493 73.61047410247015) bank31624 +31625 POINT(40.291051781719574 73.20587703210634) bank31625 +31626 POINT(40.97252777340298 73.39164777725284) bank31626 +31627 POINT(40.070840196383685 73.70831818917499) bank31627 +31628 POINT(41.32899855922626 74.69162612363168) bank31628 +31629 POINT(40.16807440550172 74.07855824998275) bank31629 +31630 POINT(40.450934687204175 73.04921500024604) bank31630 +31631 POINT(40.595827950924225 73.8522665458249) bank31631 +31632 POINT(40.946773439164616 73.9316928442888) bank31632 +31633 POINT(40.59218117934596 73.32868357859682) bank31633 +31634 POINT(40.66803129538877 73.02480605988346) bank31634 +31635 POINT(40.3365515904935 74.90988143580256) bank31635 +31636 POINT(39.97590188189439 74.99261930637674) bank31636 +31637 POINT(40.50185720699911 73.56026260831887) bank31637 +31638 POINT(41.58753458962959 74.66077934129727) bank31638 +31639 POINT(41.34815012617069 73.70140039694267) bank31639 +31640 POINT(39.72725276503895 74.99343809825417) bank31640 +31641 POINT(41.59067563603277 73.21055024082256) bank31641 +31642 POINT(39.7890871975937 74.35262201919736) bank31642 +31643 POINT(41.69459691418623 73.43536491191246) bank31643 +31644 POINT(41.444046417439765 74.5300139100204) bank31644 +31645 POINT(41.42517159741689 74.55258235227399) bank31645 +31646 POINT(41.5332184997439 73.83815217412284) bank31646 +31647 POINT(41.27882701665084 73.33146663343453) bank31647 +31648 POINT(40.78829983611333 74.82688667282798) bank31648 +31649 POINT(41.28876746507283 74.89156041382292) bank31649 +31650 POINT(40.503591752817734 73.05737245123059) bank31650 +31651 POINT(41.06924214787039 73.86124181760016) bank31651 +31652 POINT(40.437280636992114 74.35903373506882) bank31652 +31653 POINT(40.84806749263342 73.17838838434687) bank31653 +31654 POINT(40.58063034528501 74.27548174006023) bank31654 +31655 POINT(40.72457286518008 74.40078591553254) bank31655 +31656 POINT(40.69765537547209 74.76100451796457) bank31656 +31657 POINT(41.69375481733435 73.5454322621297) bank31657 +31658 POINT(39.84201697569117 73.81347235358383) bank31658 +31659 POINT(40.52146091546268 73.8305545877731) bank31659 +31660 POINT(41.5556195064404 73.82524206693691) bank31660 +31661 POINT(40.65314620308367 74.68228159909829) bank31661 +31662 POINT(40.67902634640591 73.02141752494802) bank31662 +31663 POINT(39.987148227718784 73.27167227432585) bank31663 +31664 POINT(41.173152301264686 73.36902411331995) bank31664 +31665 POINT(40.910129508740155 73.15368400638661) bank31665 +31666 POINT(41.28842874319361 74.13165031771858) bank31666 +31667 POINT(40.32473763002896 73.68522375988839) bank31667 +31668 POINT(41.51850413076507 73.8619625022201) bank31668 +31669 POINT(40.46215446243839 74.2241196818691) bank31669 +31670 POINT(40.41359025829271 73.5806680563508) bank31670 +31671 POINT(40.26656471529445 74.46197796705934) bank31671 +31672 POINT(41.14294348862421 73.61511154995357) bank31672 +31673 POINT(41.67695523438824 73.54118344749975) bank31673 +31674 POINT(39.8502066677104 73.50321339577103) bank31674 +31675 POINT(40.99087618507691 73.64823892553412) bank31675 +31676 POINT(41.662960555859065 73.13932340377409) bank31676 +31677 POINT(41.17050241679055 73.18709222287427) bank31677 +31678 POINT(41.45331813051413 73.13876127575912) bank31678 +31679 POINT(41.36533011855586 74.70638486019524) bank31679 +31680 POINT(41.468458480944875 74.61874369486216) bank31680 +31681 POINT(39.927141345100914 73.67614027777194) bank31681 +31682 POINT(40.50215312314883 74.79539293088118) bank31682 +31683 POINT(41.22415046242592 73.59018892538946) bank31683 +31684 POINT(40.70644483330272 73.4172846443452) bank31684 +31685 POINT(40.07668059407357 74.17107956705514) bank31685 +31686 POINT(41.029504220670134 74.40420516860537) bank31686 +31687 POINT(40.82854702935067 74.22348278192544) bank31687 +31688 POINT(41.26511437437992 73.87515669691246) bank31688 +31689 POINT(40.06486069639358 74.18009666962708) bank31689 +31690 POINT(41.43448858626793 74.90643328908547) bank31690 +31691 POINT(39.827751551778874 73.82947188574772) bank31691 +31692 POINT(40.10936535974945 74.66243153509082) bank31692 +31693 POINT(41.203195413782844 73.96553190387814) bank31693 +31694 POINT(40.22487961578627 73.96023069874977) bank31694 +31695 POINT(41.36957882533198 73.05145725597478) bank31695 +31696 POINT(41.48831521169099 74.57515655128249) bank31696 +31697 POINT(40.26621383757829 74.77950586742796) bank31697 +31698 POINT(40.37122760122431 73.07861440009724) bank31698 +31699 POINT(40.902731755141396 74.02927032449615) bank31699 +31700 POINT(40.38727255612903 74.03138950167346) bank31700 +31701 POINT(40.269399481857036 74.25660538574624) bank31701 +31702 POINT(40.284525268740516 74.54375128097594) bank31702 +31703 POINT(40.20236305588788 73.3526305466709) bank31703 +31704 POINT(39.97029315469303 73.92916150075438) bank31704 +31705 POINT(40.19815248000381 74.55109562257806) bank31705 +31706 POINT(41.05250578911976 74.4634985771954) bank31706 +31707 POINT(40.67376428947209 73.33096715726556) bank31707 +31708 POINT(39.91536240046486 73.91348592411322) bank31708 +31709 POINT(41.15765615199922 73.51235570312406) bank31709 +31710 POINT(41.546392012149994 74.48470223069891) bank31710 +31711 POINT(40.92697430208567 73.60963772037849) bank31711 +31712 POINT(39.75314521768609 73.25019725955363) bank31712 +31713 POINT(41.34494045095306 73.38534893555091) bank31713 +31714 POINT(39.94795513021175 74.2872007109739) bank31714 +31715 POINT(39.92373481554802 73.29107180820428) bank31715 +31716 POINT(40.08619855538375 73.35152742267702) bank31716 +31717 POINT(40.02497359737214 74.73719715578194) bank31717 +31718 POINT(40.91327816864409 74.83575337605576) bank31718 +31719 POINT(41.643403954542606 73.83541390635507) bank31719 +31720 POINT(39.80004591635613 73.19200442387917) bank31720 +31721 POINT(40.49170457233649 74.4476796227755) bank31721 +31722 POINT(40.72092157768488 74.84704482019472) bank31722 +31723 POINT(41.41305471748438 73.31988677720078) bank31723 +31724 POINT(40.46122573097489 73.68465687943228) bank31724 +31725 POINT(39.79683298359715 73.26874666189934) bank31725 +31726 POINT(39.97360135159602 74.77614881457154) bank31726 +31727 POINT(41.62823763011191 73.53848664442363) bank31727 +31728 POINT(40.83838250510425 74.38262774311306) bank31728 +31729 POINT(39.92165005815316 73.89403832597009) bank31729 +31730 POINT(40.83504067213843 73.77978744404129) bank31730 +31731 POINT(39.93656016424471 73.98986337745258) bank31731 +31732 POINT(40.36710146743011 74.79387661667067) bank31732 +31733 POINT(40.80558319253717 73.75609642362397) bank31733 +31734 POINT(41.37798372872406 73.02094484247901) bank31734 +31735 POINT(41.52265018567937 73.37839816336711) bank31735 +31736 POINT(41.13707264998916 73.24578704193091) bank31736 +31737 POINT(40.63001611544122 73.41625081293455) bank31737 +31738 POINT(39.850792935396456 74.6708896372858) bank31738 +31739 POINT(41.44683586837548 73.01163670580092) bank31739 +31740 POINT(40.41128262149173 74.49386788545277) bank31740 +31741 POINT(40.90390444767098 73.21189386385132) bank31741 +31742 POINT(41.27269775877276 73.09130485983881) bank31742 +31743 POINT(40.45534599697495 74.68871165179239) bank31743 +31744 POINT(41.432997866091995 73.82798651696976) bank31744 +31745 POINT(41.12047171715509 74.38016415710021) bank31745 +31746 POINT(40.049505959086815 74.66836652027733) bank31746 +31747 POINT(40.63818202192904 73.06196416546513) bank31747 +31748 POINT(40.67644300162781 73.22963097114022) bank31748 +31749 POINT(40.38347434678035 73.03572846467156) bank31749 +31750 POINT(40.977725756296714 74.74787993051535) bank31750 +31751 POINT(41.43945750757094 73.9754215703039) bank31751 +31752 POINT(40.32176636210427 74.97563426075287) bank31752 +31753 POINT(41.269154892445364 74.71581203218194) bank31753 +31754 POINT(40.81560527215365 74.33046478876527) bank31754 +31755 POINT(40.8708014113994 73.52753110611111) bank31755 +31756 POINT(41.56902346928981 73.19204528714849) bank31756 +31757 POINT(40.359190676338166 73.9786086641483) bank31757 +31758 POINT(39.865282023695556 74.30113041126518) bank31758 +31759 POINT(41.05463142787427 73.94133044270217) bank31759 +31760 POINT(41.080423926504935 73.10361457927276) bank31760 +31761 POINT(40.858946282204755 74.58197140426932) bank31761 +31762 POINT(41.15013293454226 73.24449801109779) bank31762 +31763 POINT(41.1635528800904 73.91732572239978) bank31763 +31764 POINT(40.40101186964915 73.5046818582308) bank31764 +31765 POINT(40.662191426957286 74.0715560675295) bank31765 +31766 POINT(39.85394579835052 74.90715366711814) bank31766 +31767 POINT(41.39137550048795 73.91071205350545) bank31767 +31768 POINT(40.22484500896194 74.75023488277719) bank31768 +31769 POINT(40.95976382843501 73.79899302268701) bank31769 +31770 POINT(40.058678126872145 74.93468419152377) bank31770 +31771 POINT(41.66437213872095 73.57984797588152) bank31771 +31772 POINT(39.80498063805618 74.45498755088684) bank31772 +31773 POINT(40.545255450799 74.2402900709774) bank31773 +31774 POINT(40.39807765192906 73.25156488946348) bank31774 +31775 POINT(40.957937832412696 73.05525461775649) bank31775 +31776 POINT(41.06499836993669 73.99489973090834) bank31776 +31777 POINT(40.99902927868657 74.22792428019713) bank31777 +31778 POINT(40.871294864695045 74.23299032223733) bank31778 +31779 POINT(40.28257663000296 73.18192646958286) bank31779 +31780 POINT(40.004483552415245 73.5210366376779) bank31780 +31781 POINT(40.44187232623876 74.2131144361373) bank31781 +31782 POINT(39.80933362250169 73.93463351139785) bank31782 +31783 POINT(40.50665047900321 73.18915492551497) bank31783 +31784 POINT(40.61512430648313 74.26142570691258) bank31784 +31785 POINT(40.404393312309644 74.42782693694993) bank31785 +31786 POINT(40.88041432709104 73.00966632372227) bank31786 +31787 POINT(40.864281873454054 74.26395657386963) bank31787 +31788 POINT(41.62301436696743 73.53227238002869) bank31788 +31789 POINT(40.52533170034738 73.75546034948557) bank31789 +31790 POINT(40.64338567510138 73.58126017845288) bank31790 +31791 POINT(40.35786739985178 73.61308586495132) bank31791 +31792 POINT(40.99336140357353 74.51711120254193) bank31792 +31793 POINT(40.85375090858304 73.91626944664898) bank31793 +31794 POINT(41.677213164940085 74.2124841539598) bank31794 +31795 POINT(39.75360779091737 74.15132615001396) bank31795 +31796 POINT(40.21060908018156 73.5305897709517) bank31796 +31797 POINT(40.730635217077776 74.87745643659247) bank31797 +31798 POINT(40.05584835351467 74.41649739255786) bank31798 +31799 POINT(41.59889317421488 74.56074554416533) bank31799 +31800 POINT(40.14873966315942 73.36946922465255) bank31800 +31801 POINT(39.92659002518943 74.26834923655177) bank31801 +31802 POINT(40.68109630643459 73.92225503159544) bank31802 +31803 POINT(40.0725231070876 74.89904403223831) bank31803 +31804 POINT(40.32040481717475 73.47611160707969) bank31804 +31805 POINT(40.10683857387847 74.65832475805769) bank31805 +31806 POINT(41.50037592214296 73.99018703087422) bank31806 +31807 POINT(40.78274939797028 73.75372931609058) bank31807 +31808 POINT(41.10615028371626 73.70653004789017) bank31808 +31809 POINT(41.71171283743929 73.2730844773473) bank31809 +31810 POINT(41.14158152851951 74.3699995918181) bank31810 +31811 POINT(40.25935791145224 73.0090749789479) bank31811 +31812 POINT(40.67147260884581 73.13177343375222) bank31812 +31813 POINT(41.43759003895694 73.7496455151649) bank31813 +31814 POINT(41.6583766406425 73.55239049588779) bank31814 +31815 POINT(40.82712299030275 74.56480160885799) bank31815 +31816 POINT(40.30674513631499 74.88445158738956) bank31816 +31817 POINT(39.77969772548576 73.65407373495442) bank31817 +31818 POINT(41.205339868922515 73.16452733841669) bank31818 +31819 POINT(40.37658542866184 73.05214797790097) bank31819 +31820 POINT(40.63753203146008 73.42494103230173) bank31820 +31821 POINT(40.487875592078744 73.34256481394088) bank31821 +31822 POINT(40.21638930695146 74.16802289369782) bank31822 +31823 POINT(40.992100966609854 73.8793314831567) bank31823 +31824 POINT(41.674513461317076 73.34885672697624) bank31824 +31825 POINT(40.076391754294626 74.5892445549397) bank31825 +31826 POINT(40.8053726274989 73.02311075923079) bank31826 +31827 POINT(40.036328461628116 73.70349834120799) bank31827 +31828 POINT(40.38649186497625 74.7820070246302) bank31828 +31829 POINT(39.787783162602565 74.11755805744679) bank31829 +31830 POINT(40.65788135696805 73.030725933853) bank31830 +31831 POINT(41.520754562160164 73.53030732171952) bank31831 +31832 POINT(41.25826197568381 74.20368967552449) bank31832 +31833 POINT(40.49865886923394 73.34642955575436) bank31833 +31834 POINT(41.4871543184665 74.47814585897522) bank31834 +31835 POINT(41.414042905000095 73.19662186338306) bank31835 +31836 POINT(41.68469889718454 74.9974991435088) bank31836 +31837 POINT(40.1439676755348 73.74876064707595) bank31837 +31838 POINT(40.9056181856351 73.60408670042327) bank31838 +31839 POINT(40.503878393844296 74.79177190514744) bank31839 +31840 POINT(41.38520828642028 73.84605661928693) bank31840 +31841 POINT(40.386250110892284 73.75875875034447) bank31841 +31842 POINT(41.1061798354253 74.19712817377133) bank31842 +31843 POINT(41.068397069952724 74.27423238067313) bank31843 +31844 POINT(41.21470526025844 74.99444740895217) bank31844 +31845 POINT(40.49703519638086 73.9064297334781) bank31845 +31846 POINT(40.40835912209501 74.33315664394826) bank31846 +31847 POINT(41.61383603873894 74.76784779154967) bank31847 +31848 POINT(41.70511489095277 73.56139574995613) bank31848 +31849 POINT(40.7731781564772 73.80751804993194) bank31849 +31850 POINT(40.436291959695424 74.77420373432862) bank31850 +31851 POINT(41.21547108875556 73.13979962868599) bank31851 +31852 POINT(39.7145094844064 74.44244985392552) bank31852 +31853 POINT(41.36118447958225 73.78713343691695) bank31853 +31854 POINT(41.5473416133577 73.8643734395605) bank31854 +31855 POINT(41.51805369131666 74.6717490354714) bank31855 +31856 POINT(40.30150456531692 73.5617417272348) bank31856 +31857 POINT(40.39490541721164 73.45529661961463) bank31857 +31858 POINT(40.866690763546714 73.21365300461464) bank31858 +31859 POINT(40.4725372040379 73.99941671967997) bank31859 +31860 POINT(40.6485512452701 74.31456326870166) bank31860 +31861 POINT(39.954662936766134 74.55076428655252) bank31861 +31862 POINT(40.733053643834346 74.84249356752726) bank31862 +31863 POINT(40.69636683735808 73.73449995215442) bank31863 +31864 POINT(39.99313796808959 73.86551465680216) bank31864 +31865 POINT(39.804443086949334 74.60011744845147) bank31865 +31866 POINT(39.885776428879524 73.3489501325246) bank31866 +31867 POINT(41.56705259860512 74.67445300911268) bank31867 +31868 POINT(40.437423260197185 74.15029388718851) bank31868 +31869 POINT(41.58938801299616 73.76995976751617) bank31869 +31870 POINT(41.40098545721455 74.3591518578896) bank31870 +31871 POINT(39.906088808513594 74.45098482671078) bank31871 +31872 POINT(40.030831064894684 73.1391524647665) bank31872 +31873 POINT(39.74062858380523 74.14820669565601) bank31873 +31874 POINT(41.588161016555205 73.51783159709481) bank31874 +31875 POINT(39.873951587671094 73.0580341783438) bank31875 +31876 POINT(40.74420550118924 74.49565248917865) bank31876 +31877 POINT(40.3719592893669 74.02617428319087) bank31877 +31878 POINT(40.727237242829965 73.52829414917169) bank31878 +31879 POINT(40.16132886306325 73.89620277221304) bank31879 +31880 POINT(41.1945585531521 74.61771486191007) bank31880 +31881 POINT(41.342027952199814 74.19353710479682) bank31881 +31882 POINT(41.20221792589223 74.52610995126417) bank31882 +31883 POINT(40.12917019904354 74.03604094410443) bank31883 +31884 POINT(41.21965579453923 74.9508160662405) bank31884 +31885 POINT(40.942815839466654 73.52282606167924) bank31885 +31886 POINT(39.91204944330827 74.68225035089195) bank31886 +31887 POINT(40.87290553714637 73.3306560824393) bank31887 +31888 POINT(41.61588478833997 74.93779353961953) bank31888 +31889 POINT(41.39653715462107 74.34100320948187) bank31889 +31890 POINT(41.5682992234313 74.55096835026961) bank31890 +31891 POINT(40.2253859922104 73.60249074370178) bank31891 +31892 POINT(40.33431293992426 73.54626381136278) bank31892 +31893 POINT(41.542288507191344 74.12585424486589) bank31893 +31894 POINT(40.68259388240562 73.3274333953044) bank31894 +31895 POINT(39.89659760152167 73.85643023402933) bank31895 +31896 POINT(39.782451692978746 74.1440534204306) bank31896 +31897 POINT(40.9561271673825 73.1190519460911) bank31897 +31898 POINT(41.41577872033613 73.2848450306858) bank31898 +31899 POINT(39.760153310125844 74.47683650515593) bank31899 +31900 POINT(40.75671742332173 74.42831879100235) bank31900 +31901 POINT(41.25913669174965 74.2288980200899) bank31901 +31902 POINT(41.57714273619342 74.49170990521543) bank31902 +31903 POINT(40.18736978151711 74.61814098137607) bank31903 +31904 POINT(40.67011157056195 73.4473723701166) bank31904 +31905 POINT(39.88990291144975 74.90607624314198) bank31905 +31906 POINT(40.01948470624784 74.35640308461754) bank31906 +31907 POINT(40.023665063897944 73.95262185202635) bank31907 +31908 POINT(40.90550905246887 73.60647237530725) bank31908 +31909 POINT(41.33692934815759 74.7160222944928) bank31909 +31910 POINT(39.85594872724198 74.16565163076022) bank31910 +31911 POINT(40.16858139705998 73.05028318665123) bank31911 +31912 POINT(41.63760561852134 73.0124311983227) bank31912 +31913 POINT(40.45773145869478 73.84641005769605) bank31913 +31914 POINT(40.95748663530387 74.13909971918929) bank31914 +31915 POINT(40.97783487627418 74.14167366748157) bank31915 +31916 POINT(40.077391441242874 73.50804177961759) bank31916 +31917 POINT(40.110901455440896 74.56075809875533) bank31917 +31918 POINT(40.81961759657763 73.86128984778063) bank31918 +31919 POINT(39.807436099180364 74.99430622353745) bank31919 +31920 POINT(39.85442654301263 74.1472984796878) bank31920 +31921 POINT(41.638045888264664 73.15019338571281) bank31921 +31922 POINT(41.63725775601218 74.19600238671406) bank31922 +31923 POINT(41.37108025006024 73.7859022708695) bank31923 +31924 POINT(40.18693951220984 73.20725116855445) bank31924 +31925 POINT(40.110784309682295 74.66195336454417) bank31925 +31926 POINT(40.14376596779467 74.53299217567064) bank31926 +31927 POINT(41.52245153179394 74.94554423019379) bank31927 +31928 POINT(41.238144692456125 74.04886266675554) bank31928 +31929 POINT(39.943735537709394 73.81583197415787) bank31929 +31930 POINT(40.40093846579847 74.7101884878385) bank31930 +31931 POINT(41.14011783602072 73.97880373709526) bank31931 +31932 POINT(41.47610224957234 74.89202452220636) bank31932 +31933 POINT(41.021043502812276 73.26109964758126) bank31933 +31934 POINT(40.88735625210952 74.83785435142103) bank31934 +31935 POINT(41.26871798832792 74.65451052819651) bank31935 +31936 POINT(40.010410099464174 73.75283720817454) bank31936 +31937 POINT(40.29901959110142 74.50240893146677) bank31937 +31938 POINT(40.7375341389036 73.70411896164926) bank31938 +31939 POINT(39.77825445735052 73.86732726524284) bank31939 +31940 POINT(40.79302271904169 73.91548406971462) bank31940 +31941 POINT(40.879783069382874 74.76731356342725) bank31941 +31942 POINT(41.475303694222234 73.25406515495541) bank31942 +31943 POINT(40.12798891032506 74.91559653534577) bank31943 +31944 POINT(39.819282562274594 74.19750954131018) bank31944 +31945 POINT(41.11936381882357 73.68707039475231) bank31945 +31946 POINT(40.585132818020064 74.10855902481103) bank31946 +31947 POINT(40.41111472675995 73.5751492298434) bank31947 +31948 POINT(40.078929466308765 74.738374171348) bank31948 +31949 POINT(41.53430989197508 73.48326773778643) bank31949 +31950 POINT(39.96365217735425 74.03710456747893) bank31950 +31951 POINT(41.105387534725764 74.2414013005626) bank31951 +31952 POINT(40.64750249297327 74.2475288804662) bank31952 +31953 POINT(39.941351961447005 74.60271955193062) bank31953 +31954 POINT(40.758133899170616 73.6925586526496) bank31954 +31955 POINT(41.11132284589267 73.74372320531367) bank31955 +31956 POINT(40.187986700994045 73.78517255018322) bank31956 +31957 POINT(41.137404031288696 74.67534734990241) bank31957 +31958 POINT(41.38766730917963 74.25490329961313) bank31958 +31959 POINT(41.6868928807407 74.85856986806371) bank31959 +31960 POINT(41.365263606707714 74.558649448564) bank31960 +31961 POINT(41.6827449111082 74.97334044345139) bank31961 +31962 POINT(40.26931617407887 74.87415022316104) bank31962 +31963 POINT(41.31719683456813 74.02398781058608) bank31963 +31964 POINT(39.94414838445735 73.94108114124167) bank31964 +31965 POINT(41.275965474823465 74.57808128032742) bank31965 +31966 POINT(40.07501939648705 73.92175969456201) bank31966 +31967 POINT(41.27891899592642 73.04385082207337) bank31967 +31968 POINT(40.16443086968517 73.2759238317216) bank31968 +31969 POINT(41.35886715854849 73.8506015354018) bank31969 +31970 POINT(40.23791059529312 73.68916499466907) bank31970 +31971 POINT(41.22856040873625 74.10654043493165) bank31971 +31972 POINT(41.262194357046674 73.1469251099872) bank31972 +31973 POINT(41.32550757718871 73.88972514946238) bank31973 +31974 POINT(40.208143104800016 73.41393991535851) bank31974 +31975 POINT(39.91261397634822 74.50486574223882) bank31975 +31976 POINT(41.32908893205162 73.169989668477) bank31976 +31977 POINT(40.52055216455266 74.81114153603806) bank31977 +31978 POINT(41.41147699691413 73.68108986794988) bank31978 +31979 POINT(40.40023450180213 73.17845005039094) bank31979 +31980 POINT(40.83160354379932 73.30816965109611) bank31980 +31981 POINT(41.375194382840846 73.91402725075147) bank31981 +31982 POINT(40.09825622135128 74.96015978053754) bank31982 +31983 POINT(41.28713431928337 74.44143521464811) bank31983 +31984 POINT(40.13017294833824 73.72907297915862) bank31984 +31985 POINT(41.16287032456947 74.53674638094239) bank31985 +31986 POINT(41.48773029377513 74.0893697585106) bank31986 +31987 POINT(39.82448697191521 73.36463788062306) bank31987 +31988 POINT(40.57300063762031 73.42291527191402) bank31988 +31989 POINT(41.68893228056247 73.92999989289613) bank31989 +31990 POINT(41.21193653724893 74.65810537281259) bank31990 +31991 POINT(39.950276387591636 74.69441575939831) bank31991 +31992 POINT(40.80898245671525 74.56870478173745) bank31992 +31993 POINT(41.605180857572556 73.45188550005618) bank31993 +31994 POINT(40.793771433225885 74.9361278682085) bank31994 +31995 POINT(40.040077764262236 73.03839875005436) bank31995 +31996 POINT(41.51945144570339 74.53391209611635) bank31996 +31997 POINT(40.8843831674574 73.35880335260947) bank31997 +31998 POINT(41.63671039763844 73.85634832600174) bank31998 +31999 POINT(40.04478070280744 74.84046499790851) bank31999 +32000 POINT(39.922082620557994 73.94513193692055) bank32000 +32001 POINT(41.4743240738262 73.16692575399176) bank32001 +32002 POINT(40.533468002714365 73.18293872169917) bank32002 +32003 POINT(40.69792720658225 74.3941410019319) bank32003 +32004 POINT(40.15082767986159 74.55988837067723) bank32004 +32005 POINT(41.677966893302035 74.341021018094) bank32005 +32006 POINT(41.23577760751328 73.1797095051606) bank32006 +32007 POINT(39.999091476094186 74.12174828674807) bank32007 +32008 POINT(40.626500804939255 74.5218388932746) bank32008 +32009 POINT(40.96193203853966 74.62433076147448) bank32009 +32010 POINT(41.16289234926684 73.86936619454579) bank32010 +32011 POINT(39.830578275512245 74.56783381853091) bank32011 +32012 POINT(41.574391464615886 73.17538084228491) bank32012 +32013 POINT(40.513065303542135 73.03019960919458) bank32013 +32014 POINT(39.99607891342261 74.64561269516128) bank32014 +32015 POINT(40.82463618611043 73.239195161944) bank32015 +32016 POINT(40.191293547762584 74.18465009972901) bank32016 +32017 POINT(39.951063765995485 73.80013363956547) bank32017 +32018 POINT(40.54224988841174 74.45625177048372) bank32018 +32019 POINT(39.71810667429378 73.54622469703992) bank32019 +32020 POINT(40.90169777673693 74.16504310170886) bank32020 +32021 POINT(40.12760038598137 73.17714535565838) bank32021 +32022 POINT(40.8756241207541 73.85402338646765) bank32022 +32023 POINT(40.79712818415672 73.5574143853698) bank32023 +32024 POINT(41.664645028766785 73.95021351228638) bank32024 +32025 POINT(40.62065006750326 73.78832023761152) bank32025 +32026 POINT(40.14936546559771 74.88205841847949) bank32026 +32027 POINT(41.249716015182166 74.00529540797888) bank32027 +32028 POINT(40.89825376541389 74.57080973046838) bank32028 +32029 POINT(40.84362817058965 73.66780904356867) bank32029 +32030 POINT(40.36315887799327 73.85154743787336) bank32030 +32031 POINT(41.578347846338424 74.87421806049791) bank32031 +32032 POINT(41.15101525736365 74.99655136899764) bank32032 +32033 POINT(40.75612676988953 73.79037332429824) bank32033 +32034 POINT(40.759035816293235 74.26190627073728) bank32034 +32035 POINT(40.86660144884779 73.54276718789215) bank32035 +32036 POINT(39.988338870264414 74.84659044700906) bank32036 +32037 POINT(41.01285888602463 73.32950864852018) bank32037 +32038 POINT(39.885132776356464 73.13124285381592) bank32038 +32039 POINT(40.24487538390613 73.67230369673284) bank32039 +32040 POINT(40.03386386765981 73.75066331064164) bank32040 +32041 POINT(40.41745191296411 73.17873460458797) bank32041 +32042 POINT(41.02715653398852 73.21934122735001) bank32042 +32043 POINT(40.07353362214729 74.13103215335714) bank32043 +32044 POINT(41.31042575292803 73.47975186736195) bank32044 +32045 POINT(40.28463352336843 73.10116084945894) bank32045 +32046 POINT(40.18798210324022 73.87996724261991) bank32046 +32047 POINT(40.36777441682422 74.60160268345831) bank32047 +32048 POINT(40.14363456627217 74.70075775329866) bank32048 +32049 POINT(40.37767387936227 73.89367692752998) bank32049 +32050 POINT(40.04757382800139 74.15725736271206) bank32050 +32051 POINT(41.191739839868205 74.0128468413172) bank32051 +32052 POINT(39.955346022012755 74.50257084708345) bank32052 +32053 POINT(39.80910630248787 74.87419072038516) bank32053 +32054 POINT(40.32112030181374 73.5145795150858) bank32054 +32055 POINT(39.99565242265504 74.5086147284642) bank32055 +32056 POINT(39.71999083750376 73.8182441608101) bank32056 +32057 POINT(40.35038489823501 74.18584813877852) bank32057 +32058 POINT(39.82247628286149 74.22306375642957) bank32058 +32059 POINT(40.37533485128851 73.35521002461931) bank32059 +32060 POINT(41.103312386003076 74.81880303052485) bank32060 +32061 POINT(40.44024635847839 73.74169106850296) bank32061 +32062 POINT(40.46281316175953 73.56556362922716) bank32062 +32063 POINT(40.56082355541 73.61087530048489) bank32063 +32064 POINT(39.93417152922941 73.73718037133672) bank32064 +32065 POINT(41.03627699562869 74.22080918725045) bank32065 +32066 POINT(41.66091647399758 74.43628451062871) bank32066 +32067 POINT(40.90750946404041 74.02549179000108) bank32067 +32068 POINT(41.022579208127 74.36915111391484) bank32068 +32069 POINT(41.03668654818464 73.19260907981453) bank32069 +32070 POINT(40.02179823127262 74.66553726549552) bank32070 +32071 POINT(39.78511634477586 73.66993929562413) bank32071 +32072 POINT(40.78988577031615 73.39638905310768) bank32072 +32073 POINT(40.106199837274204 74.24749614104046) bank32073 +32074 POINT(40.61668693378742 74.3658752405942) bank32074 +32075 POINT(41.267290413134184 74.83549305727608) bank32075 +32076 POINT(41.45890778470551 73.50971186129043) bank32076 +32077 POINT(40.815245225876794 74.5385460155506) bank32077 +32078 POINT(40.45100397517527 73.68108289015026) bank32078 +32079 POINT(39.77294868928379 74.43388343312476) bank32079 +32080 POINT(40.18092496653606 74.23344591381377) bank32080 +32081 POINT(41.08556478927731 74.51328088065993) bank32081 +32082 POINT(40.721459064669226 73.31999472293442) bank32082 +32083 POINT(40.165798439979646 74.12230484046457) bank32083 +32084 POINT(41.21301478208575 74.3755407892586) bank32084 +32085 POINT(40.234708251577416 73.99919852293137) bank32085 +32086 POINT(41.04055231924485 73.95251770405379) bank32086 +32087 POINT(40.113140806772776 73.93215641765475) bank32087 +32088 POINT(40.97552816365472 74.052424609452) bank32088 +32089 POINT(40.01295456389583 73.89754921762122) bank32089 +32090 POINT(39.870731738634355 73.72447430641502) bank32090 +32091 POINT(41.45362553484677 74.44288548019291) bank32091 +32092 POINT(40.333708709939955 74.56147129951849) bank32092 +32093 POINT(40.709974000999466 74.4606800172246) bank32093 +32094 POINT(41.05588001463896 74.97113013805821) bank32094 +32095 POINT(40.95732118816468 73.4551164998088) bank32095 +32096 POINT(41.57385365178644 74.75527817537704) bank32096 +32097 POINT(40.9373423767054 74.20216206279598) bank32097 +32098 POINT(40.91543006393737 73.43187130325148) bank32098 +32099 POINT(41.032146408469444 73.43084016310065) bank32099 +32100 POINT(41.69836238409181 74.35740115380712) bank32100 +32101 POINT(40.34825994323343 73.40820773192029) bank32101 +32102 POINT(40.14129498919276 73.29711939335483) bank32102 +32103 POINT(41.00984911609397 73.76512850490683) bank32103 +32104 POINT(40.86219614986469 74.26665763445463) bank32104 +32105 POINT(39.86863297507071 74.4557314461627) bank32105 +32106 POINT(41.4131616098107 74.07467993688299) bank32106 +32107 POINT(39.77466713579688 73.75332188418344) bank32107 +32108 POINT(41.22869441879645 73.28920105375967) bank32108 +32109 POINT(39.77096006692103 73.20531499735948) bank32109 +32110 POINT(40.034735120294854 73.16262807860127) bank32110 +32111 POINT(40.192768526989 74.56835736529158) bank32111 +32112 POINT(41.269060732197424 73.55775906715215) bank32112 +32113 POINT(40.78693533260803 73.91130517700553) bank32113 +32114 POINT(40.859833326451 73.90857183385448) bank32114 +32115 POINT(40.78578536945327 73.83422471340607) bank32115 +32116 POINT(40.758277393398544 73.08362174527198) bank32116 +32117 POINT(39.8297121897404 74.58906555632099) bank32117 +32118 POINT(40.8601442001902 73.97896797010094) bank32118 +32119 POINT(41.41226641507849 74.30975600599332) bank32119 +32120 POINT(39.77419925649023 73.48286702507752) bank32120 +32121 POINT(40.64051515131273 73.73546656482524) bank32121 +32122 POINT(41.04605710031504 74.12331709620511) bank32122 +32123 POINT(40.70524537799987 73.7476651980642) bank32123 +32124 POINT(40.886153762126526 74.24014834923486) bank32124 +32125 POINT(40.052436150970664 74.44720250807359) bank32125 +32126 POINT(40.53563957159032 73.254027503726) bank32126 +32127 POINT(41.64048046786621 73.0944766651743) bank32127 +32128 POINT(40.02090179088673 74.64888103945442) bank32128 +32129 POINT(40.604540692763756 74.56647948031284) bank32129 +32130 POINT(40.22936545330352 73.82219507693313) bank32130 +32131 POINT(40.23053028646909 73.78283616092114) bank32131 +32132 POINT(39.74149707811528 73.1108083207195) bank32132 +32133 POINT(40.826547757691635 74.23069753643496) bank32133 +32134 POINT(41.25445983832521 73.64987915432985) bank32134 +32135 POINT(39.74568524253466 73.1656000846759) bank32135 +32136 POINT(40.12496867278985 73.50473993348811) bank32136 +32137 POINT(39.75338279693241 73.02496526252588) bank32137 +32138 POINT(39.89747118881437 74.02592248531134) bank32138 +32139 POINT(41.345804124962285 73.75095914962712) bank32139 +32140 POINT(39.77040326303042 74.87905248223271) bank32140 +32141 POINT(41.12654783544161 74.45393428560897) bank32141 +32142 POINT(41.16614345557717 73.9762242520201) bank32142 +32143 POINT(41.135275552918316 74.6347436792615) bank32143 +32144 POINT(41.69131239996241 74.61040961470181) bank32144 +32145 POINT(39.95095793092636 74.78313591992365) bank32145 +32146 POINT(40.671421023215366 73.60432074160644) bank32146 +32147 POINT(41.30155213197111 73.09096589530226) bank32147 +32148 POINT(40.51447870711808 74.56653887361064) bank32148 +32149 POINT(41.21544828018379 73.08678846804114) bank32149 +32150 POINT(41.11926781632784 73.6216511910319) bank32150 +32151 POINT(41.30938941399514 74.26802857499722) bank32151 +32152 POINT(40.92014206932222 74.43645287529685) bank32152 +32153 POINT(39.90140481626118 74.85897023907147) bank32153 +32154 POINT(40.07106896064702 74.37535419788726) bank32154 +32155 POINT(40.2336966173565 73.24804047311972) bank32155 +32156 POINT(40.8374484502344 73.05305797502557) bank32156 +32157 POINT(41.61376459271125 73.57782975674668) bank32157 +32158 POINT(40.018559555035885 73.16126670125298) bank32158 +32159 POINT(41.00410028072476 74.61147451886605) bank32159 +32160 POINT(39.80618948199603 74.92255140131111) bank32160 +32161 POINT(39.898027988251336 73.81661889402072) bank32161 +32162 POINT(41.19981789390582 73.18339582067733) bank32162 +32163 POINT(41.01935047561892 73.69268552873545) bank32163 +32164 POINT(40.45590182544588 73.42683896620758) bank32164 +32165 POINT(41.62644375349465 73.68672349365791) bank32165 +32166 POINT(39.798630939480546 73.20665763230319) bank32166 +32167 POINT(39.88266542327156 73.36612384206857) bank32167 +32168 POINT(39.85437165026641 73.68351349615192) bank32168 +32169 POINT(41.495363405135784 73.54847324474058) bank32169 +32170 POINT(41.19528883898046 73.85434225916909) bank32170 +32171 POINT(40.969463358930675 73.7293264997138) bank32171 +32172 POINT(41.42932723806263 74.96670130767848) bank32172 +32173 POINT(40.33802917065361 74.73200465937212) bank32173 +32174 POINT(41.36911491037028 74.32774632728047) bank32174 +32175 POINT(41.03627637553657 74.35185870211468) bank32175 +32176 POINT(40.30752359834693 73.02242549446841) bank32176 +32177 POINT(41.62780043276074 74.0875746177195) bank32177 +32178 POINT(40.60799994987567 74.11128996276443) bank32178 +32179 POINT(41.004265417472766 74.52879468194061) bank32179 +32180 POINT(40.41053644503779 74.10817692861491) bank32180 +32181 POINT(40.142930764089485 74.9074392357703) bank32181 +32182 POINT(40.75939150436963 73.35517673373272) bank32182 +32183 POINT(40.88845722033324 73.06833272003583) bank32183 +32184 POINT(40.535654374992106 73.17158356624417) bank32184 +32185 POINT(40.33308782036996 73.28784435023552) bank32185 +32186 POINT(40.88521040303269 74.1138615563545) bank32186 +32187 POINT(40.55777555124324 73.48197798992129) bank32187 +32188 POINT(41.09253996183144 74.26168405927778) bank32188 +32189 POINT(40.07513483002371 74.2899705344369) bank32189 +32190 POINT(41.46546195131296 74.73784098989286) bank32190 +32191 POINT(41.5131591517024 73.65798923689525) bank32191 +32192 POINT(41.67853702828854 73.21538410128325) bank32192 +32193 POINT(39.99924125701125 74.60025031052477) bank32193 +32194 POINT(39.91189565090052 74.11965375405036) bank32194 +32195 POINT(41.10261808129781 73.66789368693469) bank32195 +32196 POINT(41.01058960679797 73.8320135068193) bank32196 +32197 POINT(41.51845655191429 73.81764963257487) bank32197 +32198 POINT(40.268303782835325 73.92422568297648) bank32198 +32199 POINT(41.1390108289994 74.16959203048127) bank32199 +32200 POINT(41.239489050450565 73.15986842428948) bank32200 +32201 POINT(41.668036234213176 73.11162513673654) bank32201 +32202 POINT(41.34026779475272 73.15055987671968) bank32202 +32203 POINT(40.94733894100014 73.75149800389393) bank32203 +32204 POINT(40.35069098184495 74.26741825201746) bank32204 +32205 POINT(39.73547851187677 74.11162402810002) bank32205 +32206 POINT(39.83734942919237 74.7802905443737) bank32206 +32207 POINT(40.910304927528195 74.6974600841935) bank32207 +32208 POINT(40.96534892903294 74.54472402382993) bank32208 +32209 POINT(40.8879822060754 74.591702072038) bank32209 +32210 POINT(40.5834502014057 73.34343658507504) bank32210 +32211 POINT(41.41500347315862 73.75545636138247) bank32211 +32212 POINT(39.75657305490129 73.88946507458643) bank32212 +32213 POINT(40.72363938932138 74.48697581082307) bank32213 +32214 POINT(39.867645813767965 73.6202544787392) bank32214 +32215 POINT(40.87095874973419 74.75263792815338) bank32215 +32216 POINT(41.569639908909444 74.53670819913548) bank32216 +32217 POINT(39.84033198157995 74.93992054436377) bank32217 +32218 POINT(40.244432618262096 73.61893056542182) bank32218 +32219 POINT(41.37192334305412 74.8256131380454) bank32219 +32220 POINT(41.21359689459103 74.14212932543889) bank32220 +32221 POINT(40.38148411868762 74.77165992721086) bank32221 +32222 POINT(41.049045071592325 74.5394307106846) bank32222 +32223 POINT(40.66238120582547 74.44494199400955) bank32223 +32224 POINT(40.605669568108574 73.41801780755229) bank32224 +32225 POINT(39.92918031403405 74.45261582657328) bank32225 +32226 POINT(40.03402952395348 74.1252675046779) bank32226 +32227 POINT(41.26651954104612 74.89904492429604) bank32227 +32228 POINT(41.48852540748835 73.93436239150053) bank32228 +32229 POINT(40.1563280595902 73.14155933645912) bank32229 +32230 POINT(41.23343500293595 74.13220748513183) bank32230 +32231 POINT(41.65590874357812 73.32638791317439) bank32231 +32232 POINT(40.07443220960847 74.77107130754545) bank32232 +32233 POINT(40.31533056389004 74.47433524695352) bank32233 +32234 POINT(39.73304321326492 73.02212621055249) bank32234 +32235 POINT(39.99179514258155 74.03766334056633) bank32235 +32236 POINT(40.32437213669276 73.2431357366769) bank32236 +32237 POINT(40.975452811089 73.04703552290809) bank32237 +32238 POINT(40.009757620574355 74.04479367727119) bank32238 +32239 POINT(41.00054761344914 74.76784598070044) bank32239 +32240 POINT(41.10153648019391 73.85484028687388) bank32240 +32241 POINT(40.188693795904435 74.21358654423595) bank32241 +32242 POINT(39.866454202192386 74.15312482649892) bank32242 +32243 POINT(41.52364136214857 73.39705111055139) bank32243 +32244 POINT(41.69158036304285 74.09707691397057) bank32244 +32245 POINT(41.281578868905704 73.95517707262623) bank32245 +32246 POINT(41.577834916127294 73.95662936343547) bank32246 +32247 POINT(40.393107773399635 74.43186884337334) bank32247 +32248 POINT(41.49219684201671 73.24181696073416) bank32248 +32249 POINT(39.79310285011644 74.66464163586426) bank32249 +32250 POINT(40.534800154030336 73.2742544833321) bank32250 +32251 POINT(39.721139014492884 74.10124694770151) bank32251 +32252 POINT(41.40746114143655 73.30784534083064) bank32252 +32253 POINT(40.82129851912118 73.1160844394155) bank32253 +32254 POINT(40.897029854881005 74.43977735320651) bank32254 +32255 POINT(39.954229466035876 73.56518377830062) bank32255 +32256 POINT(40.33288010366397 73.7847095283209) bank32256 +32257 POINT(40.35333695865432 74.53994394256793) bank32257 +32258 POINT(40.713836229933314 73.18319261119294) bank32258 +32259 POINT(40.22897123965482 74.25985964020241) bank32259 +32260 POINT(41.42335267740783 74.63951871025728) bank32260 +32261 POINT(41.09873470754147 74.62999920336043) bank32261 +32262 POINT(41.58631195587394 73.25551644734409) bank32262 +32263 POINT(39.84970095839922 74.90511529501033) bank32263 +32264 POINT(40.27199519606907 74.24420591195451) bank32264 +32265 POINT(41.338804302655674 73.52407162970827) bank32265 +32266 POINT(41.290178383956935 74.14467693467667) bank32266 +32267 POINT(40.569542727432555 74.59123714276859) bank32267 +32268 POINT(41.64379993996505 73.11851551286244) bank32268 +32269 POINT(40.01195307908 74.45984601516544) bank32269 +32270 POINT(41.40387176125739 74.73317433729473) bank32270 +32271 POINT(40.412314795848594 73.14209581337704) bank32271 +32272 POINT(40.80592230267683 73.90904500236519) bank32272 +32273 POINT(41.56778859603284 73.37083153246449) bank32273 +32274 POINT(41.01856726767795 74.11738528057045) bank32274 +32275 POINT(40.87045828749375 74.5324637406732) bank32275 +32276 POINT(40.92552836756305 73.70433456639711) bank32276 +32277 POINT(40.049415754992914 74.1423686517654) bank32277 +32278 POINT(40.996983044175465 74.47774803636972) bank32278 +32279 POINT(41.70185554386866 74.02833656805797) bank32279 +32280 POINT(39.89875462395897 74.26341930602591) bank32280 +32281 POINT(39.98805971053985 73.3480693544378) bank32281 +32282 POINT(40.12354002633055 74.0149146846167) bank32282 +32283 POINT(40.982510350002094 74.08249290150023) bank32283 +32284 POINT(41.03964286076874 73.47384206979534) bank32284 +32285 POINT(40.55015272121981 74.14366116982868) bank32285 +32286 POINT(41.17807148221829 74.29516230124528) bank32286 +32287 POINT(41.3266102077384 73.15278149154587) bank32287 +32288 POINT(39.731116134653426 74.46204329279162) bank32288 +32289 POINT(39.88184462137756 73.2755968210138) bank32289 +32290 POINT(40.99004751979659 73.78876743315712) bank32290 +32291 POINT(41.264900241397996 74.04110943069138) bank32291 +32292 POINT(41.472282402095956 73.42228576197542) bank32292 +32293 POINT(39.97616752242107 73.51392090572479) bank32293 +32294 POINT(41.08950323184254 74.59351915852785) bank32294 +32295 POINT(41.20540789371895 73.08695914362549) bank32295 +32296 POINT(40.81912651508035 74.95480791495206) bank32296 +32297 POINT(39.78127423473057 73.38842813417348) bank32297 +32298 POINT(39.92803885721163 74.41081848238001) bank32298 +32299 POINT(41.203932894225375 74.89372624388272) bank32299 +32300 POINT(41.06899115239852 74.52980287273749) bank32300 +32301 POINT(41.31903396037289 74.53261651588619) bank32301 +32302 POINT(41.038158080460576 73.66112940878935) bank32302 +32303 POINT(40.06384507935911 73.73275763818064) bank32303 +32304 POINT(40.523152529802566 74.66558547404688) bank32304 +32305 POINT(40.93543211503964 74.45206307245589) bank32305 +32306 POINT(40.20681909545461 74.80262811961124) bank32306 +32307 POINT(40.85934699136858 73.5224923845819) bank32307 +32308 POINT(40.36563955886931 74.45807331469805) bank32308 +32309 POINT(41.34784550051829 73.74989324410588) bank32309 +32310 POINT(40.83595973916512 73.85934401362202) bank32310 +32311 POINT(40.9190286138655 73.53395447423812) bank32311 +32312 POINT(40.798755665358875 74.98092807367878) bank32312 +32313 POINT(40.510201382962656 73.57055278641722) bank32313 +32314 POINT(41.067771676685595 73.43108426005938) bank32314 +32315 POINT(41.540438061027416 73.19724583527002) bank32315 +32316 POINT(40.56704354098363 73.52004872238895) bank32316 +32317 POINT(41.57092777077705 73.31558124217426) bank32317 +32318 POINT(40.56839132043313 74.73754707189603) bank32318 +32319 POINT(39.89508942495479 73.46048156635193) bank32319 +32320 POINT(39.94946701414485 74.7635281333533) bank32320 +32321 POINT(40.610252749636885 73.07237719741016) bank32321 +32322 POINT(41.37628447342232 73.18697500632351) bank32322 +32323 POINT(40.77273701173462 74.27060192245044) bank32323 +32324 POINT(40.746146256351324 74.93708488084214) bank32324 +32325 POINT(41.04697460953847 73.35722177284524) bank32325 +32326 POINT(39.846476482257955 73.05404925568804) bank32326 +32327 POINT(39.73066547913243 74.23717392748105) bank32327 +32328 POINT(40.872417611777884 73.58902038660767) bank32328 +32329 POINT(40.677153838778295 73.52177030955374) bank32329 +32330 POINT(41.35117809303259 73.83231060063535) bank32330 +32331 POINT(40.24048270287143 74.56174897557857) bank32331 +32332 POINT(40.429230060440304 74.07713412463198) bank32332 +32333 POINT(40.29404444638066 73.69055263691507) bank32333 +32334 POINT(41.64125729971986 73.31250092912816) bank32334 +32335 POINT(40.769687932304315 73.27385679675469) bank32335 +32336 POINT(40.50047361256597 74.65164781239106) bank32336 +32337 POINT(39.861626238155864 73.44150955354766) bank32337 +32338 POINT(41.3773424087375 73.2430741930106) bank32338 +32339 POINT(41.48725280247294 73.21399895725386) bank32339 +32340 POINT(40.3225155892063 73.74617572491498) bank32340 +32341 POINT(40.39368918080923 73.88068617330723) bank32341 +32342 POINT(41.298730626251526 73.53757903933592) bank32342 +32343 POINT(41.34272861153643 74.12230148101389) bank32343 +32344 POINT(41.222241233493804 73.56177344389172) bank32344 +32345 POINT(40.28683408078242 74.2318156506937) bank32345 +32346 POINT(41.0953679707169 73.79823090279064) bank32346 +32347 POINT(41.50773087325923 74.94836213612048) bank32347 +32348 POINT(41.47583200551625 73.51901833205241) bank32348 +32349 POINT(39.903514411231455 74.83358662509934) bank32349 +32350 POINT(41.70577150345367 73.26843427590696) bank32350 +32351 POINT(39.96518691182658 73.43537623752151) bank32351 +32352 POINT(40.80665358659183 74.61877693109683) bank32352 +32353 POINT(40.88504557667301 73.87426335199696) bank32353 +32354 POINT(40.48485652277204 73.1478730050105) bank32354 +32355 POINT(40.23445393057509 73.74294509831041) bank32355 +32356 POINT(40.97282719582502 73.6858394005916) bank32356 +32357 POINT(40.5960293162694 73.32714899931437) bank32357 +32358 POINT(40.16749762377403 74.57360353711667) bank32358 +32359 POINT(41.415586537045044 74.38045848477309) bank32359 +32360 POINT(41.1614815065873 74.26160176793042) bank32360 +32361 POINT(41.70927873442438 73.4265574251469) bank32361 +32362 POINT(40.62934403530919 73.7136697478896) bank32362 +32363 POINT(40.19312774189268 73.01428343457707) bank32363 +32364 POINT(40.072954071269216 73.37917534890927) bank32364 +32365 POINT(41.51315946358653 74.11474134252835) bank32365 +32366 POINT(41.587325007342734 73.28043171534921) bank32366 +32367 POINT(40.24022341804097 73.92101668452175) bank32367 +32368 POINT(41.178015838244505 73.27147165255681) bank32368 +32369 POINT(40.52091383885479 73.12388574001096) bank32369 +32370 POINT(40.64158499221712 73.39825505599005) bank32370 +32371 POINT(41.193222223428315 74.47902052590341) bank32371 +32372 POINT(39.742900981408674 74.900071626198) bank32372 +32373 POINT(39.89203132211291 74.0360736186356) bank32373 +32374 POINT(41.49271857233052 74.99377013786982) bank32374 +32375 POINT(40.63604660557424 74.87548468353751) bank32375 +32376 POINT(39.784902168276 73.26632350776839) bank32376 +32377 POINT(40.7711207629488 74.60429592769609) bank32377 +32378 POINT(41.523986379393946 73.46375748115561) bank32378 +32379 POINT(40.26532102613699 73.56469599633705) bank32379 +32380 POINT(40.692358870610114 73.02598607350717) bank32380 +32381 POINT(39.956958096615125 74.06550406737313) bank32381 +32382 POINT(41.094191625094304 74.3272794059557) bank32382 +32383 POINT(40.326948499461935 73.34074899250359) bank32383 +32384 POINT(40.913120496878015 73.90964844331178) bank32384 +32385 POINT(41.147743651523406 74.89768609956145) bank32385 +32386 POINT(40.404944303055665 73.73820936197099) bank32386 +32387 POINT(40.82043609783189 73.5036607008328) bank32387 +32388 POINT(40.32931798560313 74.20673864672358) bank32388 +32389 POINT(40.00133854809887 73.55805064634248) bank32389 +32390 POINT(41.2595596139052 74.11525848988713) bank32390 +32391 POINT(41.11528725374418 74.7783988068218) bank32391 +32392 POINT(40.334892299072074 74.10963637440433) bank32392 +32393 POINT(41.170077569954955 73.38312691947405) bank32393 +32394 POINT(40.87525906604767 74.72663176492085) bank32394 +32395 POINT(41.045009183912256 73.30532386180373) bank32395 +32396 POINT(40.55698963368457 73.87569878571715) bank32396 +32397 POINT(40.70983742197544 74.3781715878787) bank32397 +32398 POINT(40.657981566800984 74.18224613788676) bank32398 +32399 POINT(40.59276069566106 73.00704371032612) bank32399 +32400 POINT(41.31995753119398 74.46231552246672) bank32400 +32401 POINT(41.61004781588029 74.7995114275701) bank32401 +32402 POINT(40.20087929429998 74.3950353229976) bank32402 +32403 POINT(39.869782640915254 73.09826416911382) bank32403 +32404 POINT(39.8891946572429 73.9394905195571) bank32404 +32405 POINT(40.300641677864185 73.27362704450297) bank32405 +32406 POINT(41.026280969536884 73.40918568369099) bank32406 +32407 POINT(40.12160490658171 74.43684864119471) bank32407 +32408 POINT(41.54832027531295 73.1080210964407) bank32408 +32409 POINT(40.783793156455864 74.45572153008165) bank32409 +32410 POINT(39.76327670386971 73.72929300864982) bank32410 +32411 POINT(40.382789438130565 73.14967122174424) bank32411 +32412 POINT(40.37239998228362 73.41529390459229) bank32412 +32413 POINT(41.23013278606402 74.90638680375791) bank32413 +32414 POINT(41.61192190710073 73.00768214477395) bank32414 +32415 POINT(41.12053412952428 74.99436625591748) bank32415 +32416 POINT(41.43411279769545 74.97584095504435) bank32416 +32417 POINT(41.51201689214859 73.42783658181379) bank32417 +32418 POINT(40.37338536349627 73.93105752138017) bank32418 +32419 POINT(39.76483676629788 74.1837510865198) bank32419 +32420 POINT(41.492487239235224 73.62494290857984) bank32420 +32421 POINT(40.19958480869618 74.0364292021613) bank32421 +32422 POINT(40.376090354352264 74.91318199907244) bank32422 +32423 POINT(41.267757025820664 74.74455652322771) bank32423 +32424 POINT(40.77466038089019 74.07471739795417) bank32424 +32425 POINT(40.58267468443914 74.79769332738387) bank32425 +32426 POINT(41.384917120775626 74.31344554422652) bank32426 +32427 POINT(41.426222504294216 73.77320434536105) bank32427 +32428 POINT(40.48631616088382 73.62196111680957) bank32428 +32429 POINT(40.22956515609706 73.5751336226549) bank32429 +32430 POINT(41.606179053707656 74.36525937839252) bank32430 +32431 POINT(41.549779833640386 74.75880201455495) bank32431 +32432 POINT(40.70471284537759 73.39862670324823) bank32432 +32433 POINT(41.483301265117966 73.44522565618792) bank32433 +32434 POINT(41.62157951007789 74.49892841669387) bank32434 +32435 POINT(40.98446752495398 74.7824681909541) bank32435 +32436 POINT(40.90964324428213 73.48666541588936) bank32436 +32437 POINT(41.066220847069445 74.1047433781913) bank32437 +32438 POINT(41.25998114841658 73.50565790145569) bank32438 +32439 POINT(41.61462114537405 74.31928619637873) bank32439 +32440 POINT(40.63904783128751 73.85403299708511) bank32440 +32441 POINT(41.17743562799707 74.64509708353887) bank32441 +32442 POINT(41.60864698223429 74.26669489802025) bank32442 +32443 POINT(39.75866948687669 73.72303319435622) bank32443 +32444 POINT(40.0942048371181 73.22852842988414) bank32444 +32445 POINT(40.91979735630484 73.1279457613184) bank32445 +32446 POINT(41.343660868700745 73.55480122125718) bank32446 +32447 POINT(40.60492165813708 74.36325641567507) bank32447 +32448 POINT(39.93892784685546 74.13825297720273) bank32448 +32449 POINT(41.48846790628714 74.3207438945465) bank32449 +32450 POINT(39.90198762831431 74.9378162198267) bank32450 +32451 POINT(39.93298253222128 73.03251760186491) bank32451 +32452 POINT(41.60000958346584 74.87718432875586) bank32452 +32453 POINT(40.70133652098192 73.25940867777608) bank32453 +32454 POINT(40.7003968392913 74.16210489964524) bank32454 +32455 POINT(40.226853732086305 73.26054780020971) bank32455 +32456 POINT(40.7334204461285 73.40650699430843) bank32456 +32457 POINT(41.66258848312597 73.56471397433063) bank32457 +32458 POINT(41.230744528528476 73.37786870607272) bank32458 +32459 POINT(40.533135253417754 74.95388823880636) bank32459 +32460 POINT(40.25994879682044 73.22138743944629) bank32460 +32461 POINT(40.59696716688194 73.58228295913905) bank32461 +32462 POINT(40.84586726339773 73.52928059868418) bank32462 +32463 POINT(40.701032923540346 74.92604971838487) bank32463 +32464 POINT(39.78594685691891 73.96753910284684) bank32464 +32465 POINT(41.5918885935381 73.03920961089325) bank32465 +32466 POINT(40.69413771174206 74.82127771920617) bank32466 +32467 POINT(40.16240133688991 74.62612045900893) bank32467 +32468 POINT(40.06107743274926 73.24526274582747) bank32468 +32469 POINT(41.6586810488745 73.01262728519194) bank32469 +32470 POINT(40.45265658809905 74.44644546392844) bank32470 +32471 POINT(40.50018211997107 74.60916965974704) bank32471 +32472 POINT(40.20393288223367 74.8375467279746) bank32472 +32473 POINT(40.06603798356159 74.63635720989004) bank32473 +32474 POINT(40.92028522306332 74.62863664167446) bank32474 +32475 POINT(39.766445656587834 74.14074490952109) bank32475 +32476 POINT(40.14033999119492 74.50027380409799) bank32476 +32477 POINT(41.625257795320735 74.83975815969832) bank32477 +32478 POINT(41.479244396700466 73.24249078834953) bank32478 +32479 POINT(41.33543050403744 73.91642556807109) bank32479 +32480 POINT(39.730743925893734 74.3267135783924) bank32480 +32481 POINT(39.91541105922325 73.54642940494917) bank32481 +32482 POINT(40.60165285682879 74.17331254230824) bank32482 +32483 POINT(40.325815329815185 73.26300125768061) bank32483 +32484 POINT(40.88778101459987 73.47638741452946) bank32484 +32485 POINT(39.77949408396754 74.34217174188086) bank32485 +32486 POINT(41.33757944537711 73.6878060596199) bank32486 +32487 POINT(41.00421760454016 73.2871680079951) bank32487 +32488 POINT(41.47721328687345 73.6339146793874) bank32488 +32489 POINT(40.20432032361877 74.11837747888647) bank32489 +32490 POINT(41.021346293037254 74.11335307590116) bank32490 +32491 POINT(41.25331499695842 74.04088641751189) bank32491 +32492 POINT(40.08088227194519 74.20183165866891) bank32492 +32493 POINT(40.6776684982654 74.77060196561771) bank32493 +32494 POINT(41.55667636863696 74.26651585239942) bank32494 +32495 POINT(41.0173103496171 74.06905527978626) bank32495 +32496 POINT(40.528763559748626 74.65938803917332) bank32496 +32497 POINT(40.0716585693495 73.73261862081719) bank32497 +32498 POINT(40.7708816527729 74.84823823152301) bank32498 +32499 POINT(41.071576824770545 73.51825967087642) bank32499 +32500 POINT(41.355117266055565 74.38416284312618) bank32500 +32501 POINT(41.19437892666228 74.78466123517805) bank32501 +32502 POINT(39.796364860504895 73.54974750061193) bank32502 +32503 POINT(40.07802799812164 74.91360628085613) bank32503 +32504 POINT(40.128153157159524 74.42377440687574) bank32504 +32505 POINT(40.62471301657636 73.03677971990487) bank32505 +32506 POINT(41.37087865747779 73.84244343075518) bank32506 +32507 POINT(39.93135511853134 73.60375166905344) bank32507 +32508 POINT(40.15359270945268 73.38238372733511) bank32508 +32509 POINT(41.28565675541038 74.30926807070135) bank32509 +32510 POINT(41.41401628394747 73.80768881900649) bank32510 +32511 POINT(40.4634198115468 73.9952367942112) bank32511 +32512 POINT(40.705878515915835 74.17765358869164) bank32512 +32513 POINT(40.25481334595949 74.93978857213558) bank32513 +32514 POINT(40.321504029343416 74.80376200303685) bank32514 +32515 POINT(40.09052892578929 73.58496357302869) bank32515 +32516 POINT(40.938445787613794 74.38018187233644) bank32516 +32517 POINT(41.449295416425755 73.65146557569436) bank32517 +32518 POINT(40.86542893108842 73.22231913349273) bank32518 +32519 POINT(40.64460820929268 74.03654122173528) bank32519 +32520 POINT(40.29332584964842 74.18923079315923) bank32520 +32521 POINT(40.97983892330564 74.39720808107668) bank32521 +32522 POINT(40.26489718576532 73.34477591732329) bank32522 +32523 POINT(40.607746778183376 74.32972956618157) bank32523 +32524 POINT(40.18288696439143 73.07003263869721) bank32524 +32525 POINT(40.38945662891261 73.7699457413906) bank32525 +32526 POINT(40.183361391372905 73.31249415999672) bank32526 +32527 POINT(40.36788426591633 74.77756178407918) bank32527 +32528 POINT(40.29892839779173 74.18336211790519) bank32528 +32529 POINT(40.58179749252592 73.88232521141147) bank32529 +32530 POINT(40.0587907756127 74.72546585847482) bank32530 +32531 POINT(41.48727833034479 73.66464013725022) bank32531 +32532 POINT(40.64471246975883 73.70865046983704) bank32532 +32533 POINT(40.333967128206424 74.00326661404515) bank32533 +32534 POINT(39.81306763458898 73.79051610683513) bank32534 +32535 POINT(40.811489200721034 74.61946115821635) bank32535 +32536 POINT(40.28947796213745 73.52766372459465) bank32536 +32537 POINT(41.6025144902206 74.23666722036688) bank32537 +32538 POINT(41.70522373562679 74.71273828759283) bank32538 +32539 POINT(41.496985833877275 73.99739176282863) bank32539 +32540 POINT(41.06213009754586 74.10090492282106) bank32540 +32541 POINT(40.136437441179076 74.97761056388676) bank32541 +32542 POINT(39.85172323911024 73.48011187581308) bank32542 +32543 POINT(40.628411707172134 73.38799359306374) bank32543 +32544 POINT(40.39797278176381 74.57736005614218) bank32544 +32545 POINT(40.07900313975981 74.50299161045247) bank32545 +32546 POINT(41.15513490337452 73.73840289656714) bank32546 +32547 POINT(40.89358698284412 73.0968353795638) bank32547 +32548 POINT(41.26598028782918 74.48256732201737) bank32548 +32549 POINT(41.691244894962594 73.64720407448416) bank32549 +32550 POINT(40.21471493287127 73.27810119204686) bank32550 +32551 POINT(41.62690435063174 74.88586334140307) bank32551 +32552 POINT(41.63251004338591 74.70165475788421) bank32552 +32553 POINT(40.79460568132698 74.14886280469113) bank32553 +32554 POINT(41.37665903776217 73.18818635864359) bank32554 +32555 POINT(40.211103326929546 74.57750076602072) bank32555 +32556 POINT(41.39735189600283 74.36069920733661) bank32556 +32557 POINT(40.46686002668706 73.32533021715693) bank32557 +32558 POINT(41.691764070661044 74.76493186226715) bank32558 +32559 POINT(40.28043585077249 73.62747945323639) bank32559 +32560 POINT(40.52112893542576 74.02136986643778) bank32560 +32561 POINT(40.667671141442405 73.41697352629429) bank32561 +32562 POINT(39.740785317081034 73.68115529266855) bank32562 +32563 POINT(40.892283946946435 74.31112315120662) bank32563 +32564 POINT(39.76059153335566 73.63175403099078) bank32564 +32565 POINT(41.04168909780649 74.08512221194998) bank32565 +32566 POINT(40.54566642941671 74.07286882487635) bank32566 +32567 POINT(41.2597589562144 74.95844694298219) bank32567 +32568 POINT(41.57779946027949 74.99977248991739) bank32568 +32569 POINT(40.87083934755017 74.99185536094282) bank32569 +32570 POINT(39.7936267715577 74.18868745095098) bank32570 +32571 POINT(41.12662910555754 74.80118348025252) bank32571 +32572 POINT(41.486271770519885 74.51310568309941) bank32572 +32573 POINT(39.99939251649662 73.60209137639804) bank32573 +32574 POINT(41.23670921465943 73.86726869679615) bank32574 +32575 POINT(41.231629498082434 73.88422780339067) bank32575 +32576 POINT(41.51144826335326 74.93998726413523) bank32576 +32577 POINT(39.94989482654849 73.26676618086071) bank32577 +32578 POINT(40.15281511321341 73.4080577836407) bank32578 +32579 POINT(41.291681675170516 74.54759590433702) bank32579 +32580 POINT(39.91069304819451 73.58188631161003) bank32580 +32581 POINT(40.5058381462919 73.12876593551029) bank32581 +32582 POINT(40.00164584773101 74.99608308703212) bank32582 +32583 POINT(41.51630411287737 74.1006083665292) bank32583 +32584 POINT(41.15510386227105 73.2969127863643) bank32584 +32585 POINT(41.08931136784765 73.42849237699629) bank32585 +32586 POINT(41.229405859541004 73.99520423959699) bank32586 +32587 POINT(40.513639993873085 74.33556217594436) bank32587 +32588 POINT(40.68305259561129 74.49682642682117) bank32588 +32589 POINT(41.41601393732716 74.70112464048424) bank32589 +32590 POINT(40.51595335231798 73.67665757739292) bank32590 +32591 POINT(41.704887020376624 73.77325540045659) bank32591 +32592 POINT(40.586664533635975 73.0445825115105) bank32592 +32593 POINT(40.5228954144406 73.80667076288617) bank32593 +32594 POINT(41.550742839178 73.28072110995856) bank32594 +32595 POINT(40.9796535532207 74.52356213632773) bank32595 +32596 POINT(39.81871418236427 74.04393081272887) bank32596 +32597 POINT(40.49634718625627 73.76178808116912) bank32597 +32598 POINT(40.85317503976716 74.709539609393) bank32598 +32599 POINT(41.08713934789636 75.00394280731855) bank32599 +32600 POINT(39.87950469014293 73.84170127978747) bank32600 +32601 POINT(39.78888385088079 74.401819825054) bank32601 +32602 POINT(40.247261179573414 73.38558407108246) bank32602 +32603 POINT(39.75962406901399 74.68023274716546) bank32603 +32604 POINT(41.14433513774282 73.65527652558524) bank32604 +32605 POINT(40.83529311195313 73.60282204231078) bank32605 +32606 POINT(40.81022412755793 73.91662822447307) bank32606 +32607 POINT(40.48870105988834 74.8044694115926) bank32607 +32608 POINT(41.238646200964865 74.86645446952875) bank32608 +32609 POINT(40.93390370814721 73.30315942593931) bank32609 +32610 POINT(40.09318884164324 73.14607666991816) bank32610 +32611 POINT(40.994309011075565 73.28682109903255) bank32611 +32612 POINT(41.12098856213149 73.583601104421) bank32612 +32613 POINT(41.36078126763016 74.73535269779933) bank32613 +32614 POINT(41.09771441675341 73.70303848401058) bank32614 +32615 POINT(41.28983393991671 74.12629598423543) bank32615 +32616 POINT(40.083240343901366 74.0936464391793) bank32616 +32617 POINT(40.60925864379709 74.32436501554045) bank32617 +32618 POINT(41.152950655580035 74.58744777765175) bank32618 +32619 POINT(40.91346639184439 74.37057821563498) bank32619 +32620 POINT(40.28141061451364 73.27109095879956) bank32620 +32621 POINT(40.687877120582236 74.2063599027098) bank32621 +32622 POINT(40.440546479594204 73.3749454655533) bank32622 +32623 POINT(41.01927943025384 74.45152115397308) bank32623 +32624 POINT(41.205874928935174 74.00993503099855) bank32624 +32625 POINT(41.49824299170422 73.23746334909572) bank32625 +32626 POINT(39.90890866090258 74.41772819476876) bank32626 +32627 POINT(40.07489025464682 73.45296582926082) bank32627 +32628 POINT(40.70433090024975 73.16923072626895) bank32628 +32629 POINT(41.56443255525555 73.77583138668555) bank32629 +32630 POINT(40.16910952425647 74.21536367652004) bank32630 +32631 POINT(39.89359446928407 74.86524843277739) bank32631 +32632 POINT(40.25333338280215 74.70646991672892) bank32632 +32633 POINT(41.29532972472194 73.19493411989457) bank32633 +32634 POINT(41.601311684505426 74.89581976168758) bank32634 +32635 POINT(40.91062347068281 73.20598088461598) bank32635 +32636 POINT(41.22443621107215 74.86766127930032) bank32636 +32637 POINT(40.180527185826946 74.45040270045254) bank32637 +32638 POINT(39.878128249284174 73.54387203287114) bank32638 +32639 POINT(40.119290564428695 73.30334957537853) bank32639 +32640 POINT(41.49301443333226 73.7729425279719) bank32640 +32641 POINT(41.271644517238705 74.14460961938303) bank32641 +32642 POINT(40.159913247000866 73.32529899486502) bank32642 +32643 POINT(40.031953658410764 74.95165983105817) bank32643 +32644 POINT(41.64186958724437 73.19268017383133) bank32644 +32645 POINT(40.617181709343846 74.77895614453847) bank32645 +32646 POINT(41.36974486099339 74.85766340972776) bank32646 +32647 POINT(40.12703748306396 74.17555504576433) bank32647 +32648 POINT(39.73941058380561 74.30560441726895) bank32648 +32649 POINT(41.12505556302323 74.01907754290762) bank32649 +32650 POINT(40.56600695421014 74.31854211959919) bank32650 +32651 POINT(40.818350883187286 74.49841418262365) bank32651 +32652 POINT(41.021878217476456 73.14701002191386) bank32652 +32653 POINT(40.2618439204901 73.52768504952591) bank32653 +32654 POINT(41.48770903061295 73.45725962517713) bank32654 +32655 POINT(41.612459606880265 74.17429057761916) bank32655 +32656 POINT(39.77505800195558 74.91668565734324) bank32656 +32657 POINT(41.40094654618487 73.30702849118433) bank32657 +32658 POINT(40.98340496297228 74.42282532588412) bank32658 +32659 POINT(40.926816080878346 74.16964107278768) bank32659 +32660 POINT(40.33824538848819 74.06682187672676) bank32660 +32661 POINT(41.12622304261126 73.5489022359162) bank32661 +32662 POINT(39.76774979578184 74.92653936165874) bank32662 +32663 POINT(41.22275514491199 74.26349983150146) bank32663 +32664 POINT(40.853342199719194 73.25364105632887) bank32664 +32665 POINT(41.42341254764381 73.07213347212546) bank32665 +32666 POINT(41.09807078500291 74.29536685279317) bank32666 +32667 POINT(41.60067953162422 74.29522999459412) bank32667 +32668 POINT(41.076201130090986 74.72149732536211) bank32668 +32669 POINT(40.07293048399795 73.27750387872003) bank32669 +32670 POINT(40.10890193315061 73.88902240005919) bank32670 +32671 POINT(41.55892133360242 73.67615906259202) bank32671 +32672 POINT(41.16694843530768 74.25609326105509) bank32672 +32673 POINT(40.363415416059986 74.90088699235284) bank32673 +32674 POINT(41.19369831112746 73.25433540337325) bank32674 +32675 POINT(41.08422815560689 74.35723735239463) bank32675 +32676 POINT(40.24754959830003 73.5057842691439) bank32676 +32677 POINT(39.88803114341329 73.129543608121) bank32677 +32678 POINT(40.253206934098245 74.56902626406308) bank32678 +32679 POINT(40.21167403517655 74.69303410181243) bank32679 +32680 POINT(41.64644070564752 73.1050418461298) bank32680 +32681 POINT(40.45905677035097 74.31321321089507) bank32681 +32682 POINT(40.7260037577018 73.20393093882147) bank32682 +32683 POINT(40.76509131721779 74.15844083364895) bank32683 +32684 POINT(40.989736317906065 74.4835712633857) bank32684 +32685 POINT(40.12206031245934 73.55961100948572) bank32685 +32686 POINT(40.59067374713585 74.68956369091426) bank32686 +32687 POINT(41.479793391111635 74.30025164585962) bank32687 +32688 POINT(41.62570963188052 74.64606214308822) bank32688 +32689 POINT(40.55525299053973 74.99286610941849) bank32689 +32690 POINT(40.89419129317207 73.74608526026043) bank32690 +32691 POINT(41.00904084995646 74.36735124837517) bank32691 +32692 POINT(39.79129269483743 74.9725820839488) bank32692 +32693 POINT(40.94757756402506 73.27257353411376) bank32693 +32694 POINT(41.02400042431389 73.10442236195851) bank32694 +32695 POINT(39.864108601897094 73.1011975686859) bank32695 +32696 POINT(41.242690244433554 73.24996550178389) bank32696 +32697 POINT(41.688432046797615 73.85136864915683) bank32697 +32698 POINT(39.745985805323286 74.6374498602536) bank32698 +32699 POINT(41.224583510641075 74.9300867264132) bank32699 +32700 POINT(41.69699033346777 73.79556010351007) bank32700 +32701 POINT(40.73386146716629 73.45509619550597) bank32701 +32702 POINT(40.71273285389059 74.70301429898346) bank32702 +32703 POINT(41.63152195679473 74.35941590535349) bank32703 +32704 POINT(41.61778422928437 73.89838983754957) bank32704 +32705 POINT(40.13189041551104 74.11532638227091) bank32705 +32706 POINT(40.75744289403005 73.5816838248127) bank32706 +32707 POINT(40.35178100904706 73.56297875927235) bank32707 +32708 POINT(41.33695256312001 74.45341717215749) bank32708 +32709 POINT(40.43887457266809 74.27007630518686) bank32709 +32710 POINT(40.59732263522883 74.11379130622832) bank32710 +32711 POINT(41.13434069965726 73.830307724905) bank32711 +32712 POINT(41.44976626440197 74.53754244939583) bank32712 +32713 POINT(40.34550870446572 74.66262090331634) bank32713 +32714 POINT(40.24376712897714 73.28546636449609) bank32714 +32715 POINT(40.73930952225649 73.65780629125393) bank32715 +32716 POINT(41.096077054326884 74.16014047242881) bank32716 +32717 POINT(41.59640999028592 73.92321250606793) bank32717 +32718 POINT(41.69214560537135 73.15769616058748) bank32718 +32719 POINT(41.26767322698594 74.47020241135601) bank32719 +32720 POINT(40.383849551128385 74.71913738185344) bank32720 +32721 POINT(41.24225620323555 73.75803005355019) bank32721 +32722 POINT(41.28056936363251 74.10015229333094) bank32722 +32723 POINT(41.578005699654966 74.42493787335862) bank32723 +32724 POINT(40.62564611513596 73.40764029202062) bank32724 +32725 POINT(40.16377510332508 74.5770403370898) bank32725 +32726 POINT(39.9134467624036 73.76108897761915) bank32726 +32727 POINT(41.12753448136316 74.15946018739628) bank32727 +32728 POINT(40.478117698500554 74.4602026263453) bank32728 +32729 POINT(40.722762850767346 73.22599563809132) bank32729 +32730 POINT(41.55385408388006 73.26158417291927) bank32730 +32731 POINT(40.40829482339797 74.07463999369642) bank32731 +32732 POINT(41.014462179115704 74.7245854559991) bank32732 +32733 POINT(40.64142319113511 74.91108758212901) bank32733 +32734 POINT(41.453204389093855 73.90220187463441) bank32734 +32735 POINT(40.78063213569895 73.23176243718886) bank32735 +32736 POINT(39.8742351921235 74.66037942768338) bank32736 +32737 POINT(41.14771622725284 74.18863768316737) bank32737 +32738 POINT(41.50827773630638 73.682247875492) bank32738 +32739 POINT(40.86426196157719 74.03873963963359) bank32739 +32740 POINT(41.094611215987385 73.0769816663494) bank32740 +32741 POINT(40.151932258362955 74.08112995236165) bank32741 +32742 POINT(40.32534027001636 74.48699774037375) bank32742 +32743 POINT(40.88456174591895 74.19402282002228) bank32743 +32744 POINT(41.50583784284072 73.09613876910458) bank32744 +32745 POINT(39.78704951645554 74.94914564221986) bank32745 +32746 POINT(41.51009173175961 74.64938689926319) bank32746 +32747 POINT(39.800375930852304 73.72782988077277) bank32747 +32748 POINT(39.76141159656823 74.2015699499813) bank32748 +32749 POINT(40.74540778503183 73.32153356503086) bank32749 +32750 POINT(41.56798164400232 74.10795766740117) bank32750 +32751 POINT(40.305145381783866 73.95847530081684) bank32751 +32752 POINT(41.70393487788973 73.47887676370873) bank32752 +32753 POINT(40.04062637558719 73.36008490638228) bank32753 +32754 POINT(40.58518967977885 74.73638970518296) bank32754 +32755 POINT(41.48325748055886 74.31255806119127) bank32755 +32756 POINT(41.23923104860946 74.99113980580168) bank32756 +32757 POINT(41.59221286667295 73.42834035160926) bank32757 +32758 POINT(41.4885515432783 73.3293285687987) bank32758 +32759 POINT(40.75788229020217 74.87644406324485) bank32759 +32760 POINT(40.467631019008856 73.90522872540065) bank32760 +32761 POINT(41.283068695688186 74.67121686028605) bank32761 +32762 POINT(40.636962903275844 74.29024512684522) bank32762 +32763 POINT(40.85848989850055 74.10493130342901) bank32763 +32764 POINT(40.52613695705137 74.89502878351469) bank32764 +32765 POINT(39.79170195101406 73.9130653562225) bank32765 +32766 POINT(40.820771030968984 73.16936618371257) bank32766 +32767 POINT(41.43088243625218 73.7735551544655) bank32767 +32768 POINT(40.99809864691224 73.51854447771849) bank32768 +32769 POINT(41.12704942043453 73.8943758224624) bank32769 +32770 POINT(40.0195315128145 73.51833574430779) bank32770 +32771 POINT(40.74580191594612 73.58082851820174) bank32771 +32772 POINT(41.079467347675255 74.01794537923988) bank32772 +32773 POINT(40.3569561948474 73.2904117029199) bank32773 +32774 POINT(41.31499985865665 73.8775044432129) bank32774 +32775 POINT(41.59295714023728 73.79052964937614) bank32775 +32776 POINT(40.155136134056775 73.29270488638211) bank32776 +32777 POINT(41.02133291067739 73.19918907813309) bank32777 +32778 POINT(41.22054739924007 74.29072584298532) bank32778 +32779 POINT(41.19898070497045 74.3456321380742) bank32779 +32780 POINT(41.707650027398614 74.68846297352479) bank32780 +32781 POINT(39.908862788167525 73.1394359285143) bank32781 +32782 POINT(39.77774044211005 74.36093567842272) bank32782 +32783 POINT(41.33447340000253 73.83662430410392) bank32783 +32784 POINT(40.07811660025396 74.60744095751713) bank32784 +32785 POINT(40.43935224804796 73.39250520908249) bank32785 +32786 POINT(40.4477026086935 74.74971703427082) bank32786 +32787 POINT(40.58150640863399 74.46296644554701) bank32787 +32788 POINT(41.376176211295544 73.07781908520704) bank32788 +32789 POINT(40.76048153982881 73.78845736070026) bank32789 +32790 POINT(39.96997120266031 74.51999264643605) bank32790 +32791 POINT(40.80656063509955 73.48022687139768) bank32791 +32792 POINT(41.15452448616477 73.64682683083988) bank32792 +32793 POINT(41.268274532826176 74.45815360013162) bank32793 +32794 POINT(41.21728149345715 74.17369862138547) bank32794 +32795 POINT(41.471696774367594 74.33804055930598) bank32795 +32796 POINT(40.140434419015605 74.23874963575065) bank32796 +32797 POINT(40.53041767897492 73.4612379688142) bank32797 +32798 POINT(39.75837010361839 73.47287821177473) bank32798 +32799 POINT(40.24227922311425 73.77378255031671) bank32799 +32800 POINT(41.27229294686071 73.77630700729581) bank32800 +32801 POINT(39.95657973418779 74.90296282550521) bank32801 +32802 POINT(40.80846195976372 74.96946123725) bank32802 +32803 POINT(40.416889865132745 74.99984332303818) bank32803 +32804 POINT(41.1280366819885 73.6452971981092) bank32804 +32805 POINT(39.782086362615935 74.28935521639156) bank32805 +32806 POINT(40.936901479902495 73.69185312887255) bank32806 +32807 POINT(41.653326484559464 74.32877636163538) bank32807 +32808 POINT(39.99431825297969 73.40973470616835) bank32808 +32809 POINT(40.32146861705929 74.41123252238968) bank32809 +32810 POINT(40.79612765393466 74.51615692550553) bank32810 +32811 POINT(41.51110491584454 74.48958241023158) bank32811 +32812 POINT(40.19417709831771 74.02423011180836) bank32812 +32813 POINT(40.382684105922706 73.9046155414753) bank32813 +32814 POINT(40.12476182980569 73.52078295144284) bank32814 +32815 POINT(40.70251323949384 73.34795071208224) bank32815 +32816 POINT(41.193381143165034 74.10043280450613) bank32816 +32817 POINT(40.967707090205835 74.28619450272662) bank32817 +32818 POINT(40.36755136171 74.0499552262886) bank32818 +32819 POINT(41.50799374849945 73.77177023105705) bank32819 +32820 POINT(41.41497741607199 73.51301861315456) bank32820 +32821 POINT(39.72803812690487 73.94313590799084) bank32821 +32822 POINT(40.792696248741144 73.86475950964906) bank32822 +32823 POINT(41.145067127904504 73.08870473985546) bank32823 +32824 POINT(40.16999338318817 74.45550811189754) bank32824 +32825 POINT(41.19477392112773 73.3810855273736) bank32825 +32826 POINT(40.85207822973125 74.37586989366582) bank32826 +32827 POINT(40.27616243528951 73.05354010015634) bank32827 +32828 POINT(41.234975582407856 73.38238270583327) bank32828 +32829 POINT(40.38309874875364 74.85241880544399) bank32829 +32830 POINT(40.05011082586344 74.74615501779408) bank32830 +32831 POINT(41.1191857515251 73.19811325100702) bank32831 +32832 POINT(40.08260553683602 74.34436565326631) bank32832 +32833 POINT(40.53406207301002 73.21591406492814) bank32833 +32834 POINT(41.13870609701055 74.39127797682194) bank32834 +32835 POINT(40.789109812935436 73.36140572123685) bank32835 +32836 POINT(40.957068722089964 74.64087794182113) bank32836 +32837 POINT(40.080744010832966 74.81466533229468) bank32837 +32838 POINT(40.845522546173285 73.54224472382822) bank32838 +32839 POINT(41.477311085470106 73.43620873619587) bank32839 +32840 POINT(39.83163172282318 73.22177332160594) bank32840 +32841 POINT(40.24290527495363 73.96786587110462) bank32841 +32842 POINT(39.798055828064534 74.29189392842851) bank32842 +32843 POINT(40.394296450212316 74.679248626324) bank32843 +32844 POINT(41.16441813802563 74.46528255017603) bank32844 +32845 POINT(40.55810174288013 73.77607417567641) bank32845 +32846 POINT(40.997470690500194 73.13269780088004) bank32846 +32847 POINT(40.13847080734097 73.58228635813045) bank32847 +32848 POINT(40.99095778475812 74.40153827936759) bank32848 +32849 POINT(41.692652254640244 73.04974993440415) bank32849 +32850 POINT(40.47956452853122 74.08741629122426) bank32850 +32851 POINT(40.573362060939516 73.60905443644926) bank32851 +32852 POINT(40.5535497046574 73.42677947655574) bank32852 +32853 POINT(41.32282897756919 74.53790558028848) bank32853 +32854 POINT(41.423133804831856 74.29868560465441) bank32854 +32855 POINT(39.88075133227668 74.53618152286712) bank32855 +32856 POINT(40.534643779214704 74.54330605006594) bank32856 +32857 POINT(40.669030960752245 73.73102590629313) bank32857 +32858 POINT(40.13648238071773 73.04950406205538) bank32858 +32859 POINT(40.51271511883713 73.08687784205294) bank32859 +32860 POINT(40.61745227385228 74.75869737413855) bank32860 +32861 POINT(40.37973943741733 73.45230722259488) bank32861 +32862 POINT(40.19001931013493 73.44369937353973) bank32862 +32863 POINT(40.32499014325592 73.20562032244052) bank32863 +32864 POINT(40.08017373355174 73.55527252003347) bank32864 +32865 POINT(40.93758718992808 74.63628830980134) bank32865 +32866 POINT(40.66030547969116 74.14831441070616) bank32866 +32867 POINT(40.56399142882339 74.36071555035299) bank32867 +32868 POINT(39.811674208896584 74.614195516211) bank32868 +32869 POINT(40.165682652975065 74.66191899167366) bank32869 +32870 POINT(41.102134620302046 74.79395712795862) bank32870 +32871 POINT(40.424447569039515 73.79013027814747) bank32871 +32872 POINT(39.721204986178805 73.94029971186357) bank32872 +32873 POINT(41.071833447177724 74.20632189663085) bank32873 +32874 POINT(39.74272066004765 73.66330718530614) bank32874 +32875 POINT(40.282371282559986 74.60346071148886) bank32875 +32876 POINT(40.86659662149518 74.92279880922392) bank32876 +32877 POINT(41.66321558207025 74.37837102591938) bank32877 +32878 POINT(40.414492231437116 73.18315607682824) bank32878 +32879 POINT(40.49145853603948 74.1618976093347) bank32879 +32880 POINT(39.76220891448414 73.9565828134854) bank32880 +32881 POINT(39.81587400331708 73.6654064189282) bank32881 +32882 POINT(40.96823072429718 73.34976715418661) bank32882 +32883 POINT(40.76483435756312 73.97576994008797) bank32883 +32884 POINT(39.88347751927612 73.68792467139774) bank32884 +32885 POINT(41.70562534894946 73.95644034574313) bank32885 +32886 POINT(41.55573324780895 74.89730513784822) bank32886 +32887 POINT(40.61290714993989 73.71330018525182) bank32887 +32888 POINT(39.758174762718006 74.81575943519553) bank32888 +32889 POINT(39.72235264024725 73.23954194800196) bank32889 +32890 POINT(41.555326126873 73.74731326861266) bank32890 +32891 POINT(41.079875153172075 73.82162200683877) bank32891 +32892 POINT(39.72996387101063 73.52819693114505) bank32892 +32893 POINT(39.991154059109704 74.74468847670936) bank32893 +32894 POINT(41.120452208779525 74.9110957408999) bank32894 +32895 POINT(41.56561567983012 73.82883469677077) bank32895 +32896 POINT(40.55710310039881 74.14442050717915) bank32896 +32897 POINT(41.32319998775806 73.61152159246642) bank32897 +32898 POINT(41.58205617646721 73.6197319258652) bank32898 +32899 POINT(41.552467351724715 74.01957964637839) bank32899 +32900 POINT(40.040299253535046 73.74305869099722) bank32900 +32901 POINT(39.77271262636556 73.3753924757401) bank32901 +32902 POINT(41.3409326370041 73.04815177433339) bank32902 +32903 POINT(40.475132216927506 73.98258147193309) bank32903 +32904 POINT(40.89867848044277 73.85263829685734) bank32904 +32905 POINT(40.43573088192436 73.18198697981785) bank32905 +32906 POINT(40.846219337002054 73.25291005002201) bank32906 +32907 POINT(39.91831419821778 73.9650705777905) bank32907 +32908 POINT(40.98028215030638 74.24831723028205) bank32908 +32909 POINT(41.63670369293973 73.35963086297964) bank32909 +32910 POINT(40.330456828710474 74.53444304822467) bank32910 +32911 POINT(41.04948261413457 74.84116335835839) bank32911 +32912 POINT(40.811276301344236 73.42437863437672) bank32912 +32913 POINT(40.27138279663133 73.54069951962975) bank32913 +32914 POINT(40.722836232730316 74.9460360412079) bank32914 +32915 POINT(39.78380234775142 74.78800098181496) bank32915 +32916 POINT(41.4246659672718 73.69183458147944) bank32916 +32917 POINT(41.47196100025523 74.71769733461046) bank32917 +32918 POINT(40.02053020937044 75.00544192317211) bank32918 +32919 POINT(40.335335820775356 73.2474093204669) bank32919 +32920 POINT(40.22409432362367 74.05836705657926) bank32920 +32921 POINT(40.699260110452975 74.70362000263982) bank32921 +32922 POINT(40.82876612993042 73.22945298370152) bank32922 +32923 POINT(40.7304864205163 74.531736577122) bank32923 +32924 POINT(41.62209046396053 73.18107127666397) bank32924 +32925 POINT(40.35727482706999 73.5728086507555) bank32925 +32926 POINT(40.0468320995086 74.01347575205075) bank32926 +32927 POINT(39.807470838952064 74.57988304302675) bank32927 +32928 POINT(40.198962984690624 74.55425151165191) bank32928 +32929 POINT(40.78955595055351 74.23808185111702) bank32929 +32930 POINT(41.50098193552692 74.9172703012196) bank32930 +32931 POINT(39.941188011259136 73.22368672356559) bank32931 +32932 POINT(39.963467010850366 74.09221633180057) bank32932 +32933 POINT(41.48763333554121 74.91084874429451) bank32933 +32934 POINT(40.78928308641592 73.21484547607473) bank32934 +32935 POINT(39.866139704285494 74.24429963920208) bank32935 +32936 POINT(41.404308080057824 74.96301919110847) bank32936 +32937 POINT(41.64737397568496 74.9412336559573) bank32937 +32938 POINT(41.517834380735955 74.24237147051002) bank32938 +32939 POINT(40.35402208150142 74.34855350000912) bank32939 +32940 POINT(40.179236942197335 74.4622523329515) bank32940 +32941 POINT(41.375298593730285 74.91852600330918) bank32941 +32942 POINT(40.71997595502908 74.16368823064693) bank32942 +32943 POINT(41.28830853777491 74.25437338163778) bank32943 +32944 POINT(41.29760414010919 73.7157173881782) bank32944 +32945 POINT(41.05438754031624 73.59335606207672) bank32945 +32946 POINT(41.60176590914331 73.95409248742946) bank32946 +32947 POINT(40.100725678366835 73.6759890294437) bank32947 +32948 POINT(41.19396888791252 74.13892768614744) bank32948 +32949 POINT(40.46453045557838 74.43350248633345) bank32949 +32950 POINT(40.23359591256979 73.98552019745716) bank32950 +32951 POINT(39.85419149574521 74.09849439497165) bank32951 +32952 POINT(40.38437665713493 73.03784114630496) bank32952 +32953 POINT(41.6156660711073 74.54775142650732) bank32953 +32954 POINT(41.498923959172274 73.7409438528394) bank32954 +32955 POINT(40.69165868074295 74.0983986022861) bank32955 +32956 POINT(41.037468702429585 74.10733563747132) bank32956 +32957 POINT(40.41525292902297 74.20836940029916) bank32957 +32958 POINT(41.549656026670384 74.98457108613923) bank32958 +32959 POINT(41.44549028052791 73.57954174523472) bank32959 +32960 POINT(41.59398025861716 74.5247119747502) bank32960 +32961 POINT(40.39018250859789 73.89991306417052) bank32961 +32962 POINT(40.35618991110216 74.05657411278975) bank32962 +32963 POINT(40.299660882570635 74.35971445778546) bank32963 +32964 POINT(39.80365900309019 74.66790693345375) bank32964 +32965 POINT(40.28689711317883 74.27726544479351) bank32965 +32966 POINT(41.39968501035156 74.22972828362978) bank32966 +32967 POINT(40.85704360215099 73.28804515888422) bank32967 +32968 POINT(40.605708616120786 73.31249415011807) bank32968 +32969 POINT(40.94212359881383 74.62652346363983) bank32969 +32970 POINT(41.40616892931396 73.30555160767959) bank32970 +32971 POINT(39.96453029925227 74.18055811198255) bank32971 +32972 POINT(40.35530227979081 73.51182231386686) bank32972 +32973 POINT(39.84951916922038 74.78959738803816) bank32973 +32974 POINT(41.04533228584959 74.83654788521288) bank32974 +32975 POINT(40.32886562099362 74.18399934823377) bank32975 +32976 POINT(39.914237573861385 73.66948065233062) bank32976 +32977 POINT(41.030285381951934 73.76717345802359) bank32977 +32978 POINT(41.49557648760655 74.28864715292325) bank32978 +32979 POINT(41.1923950916347 74.94096965419996) bank32979 +32980 POINT(40.03021977004812 74.53104278976987) bank32980 +32981 POINT(40.24045814526758 74.810049045789) bank32981 +32982 POINT(40.56466955967837 74.5855232251384) bank32982 +32983 POINT(41.150346380562056 73.5702070339615) bank32983 +32984 POINT(41.03015278823896 73.11327096956008) bank32984 +32985 POINT(41.076019568605396 74.7942319845493) bank32985 +32986 POINT(40.8824402336743 74.09921333730117) bank32986 +32987 POINT(39.94501634561745 73.1689414517201) bank32987 +32988 POINT(39.89302923812398 73.56244312989736) bank32988 +32989 POINT(40.67366850055921 73.13732309666943) bank32989 +32990 POINT(40.529308217538734 73.53063764358774) bank32990 +32991 POINT(40.10200945688131 73.7910330754354) bank32991 +32992 POINT(40.92937215537779 74.26583217917913) bank32992 +32993 POINT(41.317951751821724 73.34518555734796) bank32993 +32994 POINT(40.62972682535949 73.32905411392201) bank32994 +32995 POINT(40.366455166970226 74.26298888137073) bank32995 +32996 POINT(40.56996572961303 74.47615066616147) bank32996 +32997 POINT(41.16523038164064 73.42607674406717) bank32997 +32998 POINT(41.14569712061281 74.07948013530488) bank32998 +32999 POINT(41.03584290865111 73.68408613102999) bank32999 +33000 POINT(41.05392302645917 74.69910118121851) bank33000 +33001 POINT(39.837690053470666 74.63502967073856) bank33001 +33002 POINT(41.29956530741361 74.99784253975872) bank33002 +33003 POINT(40.04458003313328 74.25467040029794) bank33003 +33004 POINT(41.338983319919144 73.73608449882165) bank33004 +33005 POINT(39.74132765688836 73.67985080542923) bank33005 +33006 POINT(41.08857594028518 73.15942026732536) bank33006 +33007 POINT(41.480213538051586 74.84440825369259) bank33007 +33008 POINT(41.27302478026793 73.73331082266189) bank33008 +33009 POINT(40.9622605626041 74.08379522755158) bank33009 +33010 POINT(40.57653589616796 74.22099853149504) bank33010 +33011 POINT(40.88115397449993 74.47566973841101) bank33011 +33012 POINT(41.22557001138572 74.89583609389211) bank33012 +33013 POINT(41.13490439388307 73.64774953605686) bank33013 +33014 POINT(40.98723428516252 73.58058055693031) bank33014 +33015 POINT(40.13865418539034 74.61442432030168) bank33015 +33016 POINT(39.80200523750507 73.4328236872041) bank33016 +33017 POINT(41.37292555143071 73.05893403293072) bank33017 +33018 POINT(40.100973779518384 73.5193722048546) bank33018 +33019 POINT(41.3635127080291 74.94319424886356) bank33019 +33020 POINT(40.68761665863935 73.64231632713432) bank33020 +33021 POINT(39.94227784043374 74.4645443810786) bank33021 +33022 POINT(40.675030810570895 74.31951424597419) bank33022 +33023 POINT(41.026187451500505 74.59729099866078) bank33023 +33024 POINT(41.23067352406324 73.9930732028891) bank33024 +33025 POINT(40.42417454474116 73.3789028452356) bank33025 +33026 POINT(40.22409365719507 73.08936817545775) bank33026 +33027 POINT(40.43088971555027 73.22404535074578) bank33027 +33028 POINT(40.02617933061465 74.12924963473922) bank33028 +33029 POINT(39.74421507550353 73.03777789343661) bank33029 +33030 POINT(40.247141713936955 74.17045908599789) bank33030 +33031 POINT(39.95644242367476 74.86499944091094) bank33031 +33032 POINT(39.80393501630611 73.95177823080988) bank33032 +33033 POINT(40.57279230977914 74.54690464470225) bank33033 +33034 POINT(40.2299608585879 73.47273089500695) bank33034 +33035 POINT(39.757054082434315 73.7287786390666) bank33035 +33036 POINT(41.363846907203936 74.46990926585336) bank33036 +33037 POINT(40.00483330378913 73.27405095752604) bank33037 +33038 POINT(41.5075961812253 74.99307027458556) bank33038 +33039 POINT(39.80402819734431 73.8177352911435) bank33039 +33040 POINT(41.20663956544782 74.28954109949983) bank33040 +33041 POINT(39.74801604202803 74.53698327688788) bank33041 +33042 POINT(41.65824837279442 73.96262668744544) bank33042 +33043 POINT(40.98163929073 74.89551973372403) bank33043 +33044 POINT(40.937354191195695 73.78905374032271) bank33044 +33045 POINT(39.88521007490224 74.06340394069312) bank33045 +33046 POINT(41.01954124905112 74.9272760172457) bank33046 +33047 POINT(39.95211999726367 73.92681225829595) bank33047 +33048 POINT(41.18520338494379 73.63645419448726) bank33048 +33049 POINT(40.732259383978715 73.5363395867325) bank33049 +33050 POINT(40.7382157065781 73.98603096612615) bank33050 +33051 POINT(40.11962044622187 74.98554625326588) bank33051 +33052 POINT(40.97625265175256 73.55140339567075) bank33052 +33053 POINT(39.78991224377776 73.85645855364459) bank33053 +33054 POINT(41.23824794448161 73.52883448371627) bank33054 +33055 POINT(40.289873382278365 74.4691692561975) bank33055 +33056 POINT(41.28855366184389 73.26149883288849) bank33056 +33057 POINT(40.53488103091743 73.23882651569734) bank33057 +33058 POINT(40.15603463434517 73.40470945417347) bank33058 +33059 POINT(41.3990541967032 74.9892781944664) bank33059 +33060 POINT(41.40796627779984 73.2718103930231) bank33060 +33061 POINT(39.94274121804591 74.13779223607534) bank33061 +33062 POINT(39.91948993321099 73.46545122701264) bank33062 +33063 POINT(40.15518165239667 73.1573007789674) bank33063 +33064 POINT(40.07282092119537 73.66252642491285) bank33064 +33065 POINT(41.181035962728835 73.15293292232438) bank33065 +33066 POINT(40.25822068462966 73.5800035361068) bank33066 +33067 POINT(39.98039788657216 74.65340720444455) bank33067 +33068 POINT(40.24381070050849 74.00576503821702) bank33068 +33069 POINT(41.4907383554851 73.72656222642229) bank33069 +33070 POINT(40.863930001438774 74.430433362437) bank33070 +33071 POINT(41.48962062806041 73.95644518305947) bank33071 +33072 POINT(39.880123574597654 74.40525241558599) bank33072 +33073 POINT(40.35810122165849 73.40106586424332) bank33073 +33074 POINT(40.008623157015336 73.62518746110376) bank33074 +33075 POINT(40.263004767792175 73.69824673355554) bank33075 +33076 POINT(40.12223768589196 74.75927211509948) bank33076 +33077 POINT(40.33979154741588 73.7132419583524) bank33077 +33078 POINT(41.13328827530462 74.16952831198418) bank33078 +33079 POINT(41.47200204664984 73.80961321652978) bank33079 +33080 POINT(39.88831480079974 74.87581381932158) bank33080 +33081 POINT(39.90703984910426 73.99067080320587) bank33081 +33082 POINT(39.935515220539315 73.6636333531997) bank33082 +33083 POINT(40.62192853706844 74.1654537811915) bank33083 +33084 POINT(40.401381192823806 74.50060312369162) bank33084 +33085 POINT(41.56210029863976 73.38563014190454) bank33085 +33086 POINT(39.943987038165446 73.5565190738348) bank33086 +33087 POINT(39.76325677050122 74.14711728685994) bank33087 +33088 POINT(41.154054397044604 74.36190543572884) bank33088 +33089 POINT(40.038593021504944 74.29779146736195) bank33089 +33090 POINT(41.16116950128821 73.07591383340342) bank33090 +33091 POINT(40.21944964285721 74.4599905123496) bank33091 +33092 POINT(39.78654167866057 73.88737192564459) bank33092 +33093 POINT(41.27974821753446 74.87629951893643) bank33093 +33094 POINT(39.94881627489259 73.59176639050945) bank33094 +33095 POINT(40.290065629631194 74.17432171481757) bank33095 +33096 POINT(41.05889350661611 73.91376387762014) bank33096 +33097 POINT(41.18562466734311 74.98151302387126) bank33097 +33098 POINT(41.561119676560644 74.98881258377753) bank33098 +33099 POINT(40.990692604495386 74.33638165718467) bank33099 +33100 POINT(39.83861785789619 73.79333087825621) bank33100 +33101 POINT(41.06784426463028 74.18478320222225) bank33101 +33102 POINT(41.138147862409205 74.81661664625264) bank33102 +33103 POINT(41.52923149634062 74.87478165380367) bank33103 +33104 POINT(41.43044657482933 73.67682753044652) bank33104 +33105 POINT(41.479579697536984 74.53465920496606) bank33105 +33106 POINT(41.69681428385772 74.56843572167772) bank33106 +33107 POINT(40.94596503226272 73.99292854874447) bank33107 +33108 POINT(40.910815605377636 74.94046234285939) bank33108 +33109 POINT(40.446434891159114 74.80000254540109) bank33109 +33110 POINT(40.6468053200686 73.63268310552965) bank33110 +33111 POINT(41.12098811209267 73.27392388627315) bank33111 +33112 POINT(41.64261127845633 74.73439888444052) bank33112 +33113 POINT(40.105148758666715 74.56651002166836) bank33113 +33114 POINT(41.43844424016298 73.49500054351891) bank33114 +33115 POINT(41.547023292881924 74.14533429813186) bank33115 +33116 POINT(40.30812233386201 73.9357496503458) bank33116 +33117 POINT(39.98330655148376 74.55580836528593) bank33117 +33118 POINT(40.76088305936316 73.11277501487328) bank33118 +33119 POINT(41.66112606809429 73.98012657557891) bank33119 +33120 POINT(40.31937149897618 74.67939162015526) bank33120 +33121 POINT(40.54035943983538 74.10688988731314) bank33121 +33122 POINT(40.3740059339499 73.75195170250917) bank33122 +33123 POINT(41.465377371019855 74.64093112112151) bank33123 +33124 POINT(41.2765514649703 74.99005154211902) bank33124 +33125 POINT(39.748650415479844 74.84234030815679) bank33125 +33126 POINT(40.982727341773774 74.41703293280266) bank33126 +33127 POINT(41.29006259497739 74.38277250945843) bank33127 +33128 POINT(41.24972611204046 73.0274328522155) bank33128 +33129 POINT(40.129292824157524 74.10986219043113) bank33129 +33130 POINT(41.69326236864414 74.72955877315422) bank33130 +33131 POINT(40.26127592491124 73.01498809753505) bank33131 +33132 POINT(40.99199697046223 73.25801888430072) bank33132 +33133 POINT(39.80238536737153 74.6716247701927) bank33133 +33134 POINT(39.71486421639729 74.9556157514181) bank33134 +33135 POINT(40.49799186601054 74.44265526396357) bank33135 +33136 POINT(41.18828594431114 74.99628970778933) bank33136 +33137 POINT(40.19580813840113 74.82325606687402) bank33137 +33138 POINT(40.373585258501365 74.31509634116455) bank33138 +33139 POINT(40.62148038535387 74.36456640811952) bank33139 +33140 POINT(40.582459912170826 73.9374918305461) bank33140 +33141 POINT(41.26180510573216 73.92745012306439) bank33141 +33142 POINT(41.65002315015378 74.13940040770598) bank33142 +33143 POINT(41.02233130738544 73.39074202433417) bank33143 +33144 POINT(40.66210732091711 73.26426061780951) bank33144 +33145 POINT(40.123050775318305 74.76288064649572) bank33145 +33146 POINT(40.03757223894176 74.13060008686085) bank33146 +33147 POINT(40.543789319471806 74.69776321057223) bank33147 +33148 POINT(40.116247601908825 73.49254041794681) bank33148 +33149 POINT(41.566100011229466 73.03385345671622) bank33149 +33150 POINT(40.5364299787375 74.54483360796533) bank33150 +33151 POINT(41.047091038841856 74.51350172512652) bank33151 +33152 POINT(40.50206811666262 74.48108070928753) bank33152 +33153 POINT(39.83845324755933 74.79604741543223) bank33153 +33154 POINT(40.62849277295424 74.27491554913686) bank33154 +33155 POINT(40.93486270142248 73.95793229120164) bank33155 +33156 POINT(39.718263122572765 73.9355668964119) bank33156 +33157 POINT(39.943125030722776 74.48174535514985) bank33157 +33158 POINT(40.61498893441994 73.14180448779912) bank33158 +33159 POINT(39.764206942685206 74.10370157809095) bank33159 +33160 POINT(41.56475021418442 74.06922599978037) bank33160 +33161 POINT(40.64035343943818 74.06319441086416) bank33161 +33162 POINT(40.31458556356065 74.68607699032908) bank33162 +33163 POINT(40.93518782666926 74.63161481553522) bank33163 +33164 POINT(41.246506864820745 74.70921888094894) bank33164 +33165 POINT(41.59233961078067 73.49048721702339) bank33165 +33166 POINT(41.04509842702247 74.84551399047261) bank33166 +33167 POINT(40.89963214892601 74.86966994826552) bank33167 +33168 POINT(39.7616433279814 73.34818551386506) bank33168 +33169 POINT(40.29203570110387 74.37338174688314) bank33169 +33170 POINT(41.66692076092079 74.34919169564057) bank33170 +33171 POINT(41.10331010985458 74.11679882252102) bank33171 +33172 POINT(40.63146956817245 74.71329617081692) bank33172 +33173 POINT(39.984791622491954 73.43426009236283) bank33173 +33174 POINT(39.98108466884919 74.0795354001907) bank33174 +33175 POINT(41.59520996255447 73.57445882131933) bank33175 +33176 POINT(39.777487890852015 73.25997814896742) bank33176 +33177 POINT(40.85220831324474 73.65135634646421) bank33177 +33178 POINT(40.694231605614405 73.16713798366673) bank33178 +33179 POINT(40.18658225669972 73.47227797213232) bank33179 +33180 POINT(41.3636027526322 74.23651036390078) bank33180 +33181 POINT(40.44758973676743 73.4786423543731) bank33181 +33182 POINT(40.38560453492826 73.51443171113353) bank33182 +33183 POINT(41.015046288140184 73.86742160366585) bank33183 +33184 POINT(41.22488099723796 74.55080697756638) bank33184 +33185 POINT(40.32052518461326 73.41841000854119) bank33185 +33186 POINT(41.01334056784332 74.40219902363158) bank33186 +33187 POINT(40.27664711169323 74.09117026602246) bank33187 +33188 POINT(41.47170402474084 74.34121769365075) bank33188 +33189 POINT(41.16513478659542 74.65255662880882) bank33189 +33190 POINT(41.177243241393334 74.80252957359556) bank33190 +33191 POINT(41.0772642733947 74.70833446547343) bank33191 +33192 POINT(40.76026442532883 73.63803786894331) bank33192 +33193 POINT(40.61588556371213 74.44499064554735) bank33193 +33194 POINT(41.4095382974915 73.9331401550367) bank33194 +33195 POINT(40.368285425055035 74.38763400811119) bank33195 +33196 POINT(40.151269348767066 74.3503918655128) bank33196 +33197 POINT(40.08713061127479 73.05228529965213) bank33197 +33198 POINT(41.69021540591831 73.38390862745895) bank33198 +33199 POINT(41.53034166438562 74.59152137274924) bank33199 +33200 POINT(41.384921588432 73.42567972476994) bank33200 +33201 POINT(39.8353050146072 74.67468311795525) bank33201 +33202 POINT(39.95833523456269 73.70542717422214) bank33202 +33203 POINT(40.90085611338881 74.5378032561435) bank33203 +33204 POINT(39.96604944241841 74.21579642610558) bank33204 +33205 POINT(41.431253843028124 74.90430045105629) bank33205 +33206 POINT(41.55155787598173 73.95664275670023) bank33206 +33207 POINT(41.52603263319057 74.92680682198595) bank33207 +33208 POINT(40.49153524119496 73.85955292409125) bank33208 +33209 POINT(41.4049078975487 74.44808488529345) bank33209 +33210 POINT(40.9000051319415 73.50085621646195) bank33210 +33211 POINT(40.676554875669105 74.39852274228055) bank33211 +33212 POINT(40.7899774825483 73.40884236649418) bank33212 +33213 POINT(40.155026363074704 73.2780014950719) bank33213 +33214 POINT(41.55022948670738 74.23008548540248) bank33214 +33215 POINT(40.26906285760721 74.76440603076962) bank33215 +33216 POINT(40.199586869455274 73.36157046178624) bank33216 +33217 POINT(40.3237964930379 74.25459556632205) bank33217 +33218 POINT(41.577217136623275 73.79605904096272) bank33218 +33219 POINT(40.39305984814742 74.05633283340873) bank33219 +33220 POINT(41.18388124575788 73.93549645715329) bank33220 +33221 POINT(41.09365824542728 74.27752549418214) bank33221 +33222 POINT(41.492722377943615 73.69732767903217) bank33222 +33223 POINT(40.91991073769592 74.17907212797846) bank33223 +33224 POINT(41.02449310665841 73.56348885844385) bank33224 +33225 POINT(40.217852610780795 73.37203602348751) bank33225 +33226 POINT(40.799194942187455 73.14133336543846) bank33226 +33227 POINT(41.17726731585467 74.43343497981321) bank33227 +33228 POINT(40.958489202637764 73.90586166168853) bank33228 +33229 POINT(40.75744007164292 73.46306090545828) bank33229 +33230 POINT(41.04993666906202 74.37727094938722) bank33230 +33231 POINT(39.80038844926288 74.26088345702378) bank33231 +33232 POINT(40.95223381936447 74.5219066532165) bank33232 +33233 POINT(40.17267974169648 73.48084701991414) bank33233 +33234 POINT(41.654607931207565 74.9770840917359) bank33234 +33235 POINT(40.26929011780085 73.11957149249353) bank33235 +33236 POINT(41.20164752709868 74.05275087544945) bank33236 +33237 POINT(41.582534462894785 74.74025956092476) bank33237 +33238 POINT(41.18613676254252 74.05924887586227) bank33238 +33239 POINT(40.249489415548815 73.62136372258004) bank33239 +33240 POINT(41.11603613469984 74.8597738083908) bank33240 +33241 POINT(41.2671738549234 74.74127354720748) bank33241 +33242 POINT(40.49914535177452 74.4717129878317) bank33242 +33243 POINT(40.830003412271694 74.64730957889364) bank33243 +33244 POINT(40.26211498778613 74.93789649265847) bank33244 +33245 POINT(40.45332475269203 74.89345610403647) bank33245 +33246 POINT(41.04832807707262 74.41550015268405) bank33246 +33247 POINT(41.129648268346166 73.05431358562093) bank33247 +33248 POINT(40.40473577290615 73.31624912664238) bank33248 +33249 POINT(39.76337722191678 73.46423179364488) bank33249 +33250 POINT(40.828066996044534 73.71458171471488) bank33250 +33251 POINT(41.30006976044544 74.1054612171794) bank33251 +33252 POINT(40.37012802055715 74.90573964056244) bank33252 +33253 POINT(40.19872563157047 74.41042503032814) bank33253 +33254 POINT(40.38337331914925 74.99000937177956) bank33254 +33255 POINT(40.54603483971786 73.83728865640319) bank33255 +33256 POINT(40.68451012910537 73.66266316349005) bank33256 +33257 POINT(41.30789488238616 74.54419918423515) bank33257 +33258 POINT(41.6238490119536 74.78487757447962) bank33258 +33259 POINT(40.89656817218463 73.04869750189808) bank33259 +33260 POINT(41.42058019684688 74.95927005654589) bank33260 +33261 POINT(40.35574435384274 74.68506302794742) bank33261 +33262 POINT(40.21687562319928 73.76399639868656) bank33262 +33263 POINT(41.39021041669725 73.30763346808943) bank33263 +33264 POINT(41.497497470703415 74.87025324963523) bank33264 +33265 POINT(39.82527100888855 74.06966457110113) bank33265 +33266 POINT(39.737992100680565 74.63522117337139) bank33266 +33267 POINT(39.944940126799494 74.47061164243846) bank33267 +33268 POINT(41.36743201892805 74.02532339523938) bank33268 +33269 POINT(41.506074705716536 73.96662007064005) bank33269 +33270 POINT(39.849615800703795 74.63624265680063) bank33270 +33271 POINT(40.715609948219694 74.0396133012266) bank33271 +33272 POINT(40.5750625163829 73.22894978373982) bank33272 +33273 POINT(41.712498519836764 74.32938945870585) bank33273 +33274 POINT(39.754150585570116 74.86441926517902) bank33274 +33275 POINT(41.694102593665676 73.50686338864197) bank33275 +33276 POINT(41.44395835587285 74.45396699265443) bank33276 +33277 POINT(39.9088676328778 74.61648015531223) bank33277 +33278 POINT(40.82428175603113 74.42859652195236) bank33278 +33279 POINT(40.70876125347653 73.97736149472937) bank33279 +33280 POINT(41.55792786699063 73.83385462336851) bank33280 +33281 POINT(41.0626754284814 74.07318793872734) bank33281 +33282 POINT(39.876362523986856 73.2276043289203) bank33282 +33283 POINT(41.05395523243798 73.5822696262928) bank33283 +33284 POINT(39.78625334356925 73.2784026042393) bank33284 +33285 POINT(40.84108857028765 73.39778863295102) bank33285 +33286 POINT(41.31015827727478 73.29791120086314) bank33286 +33287 POINT(41.54974343739442 73.269253409782) bank33287 +33288 POINT(41.29607404212705 73.42411896448483) bank33288 +33289 POINT(40.41971270850382 73.40444329801734) bank33289 +33290 POINT(40.27924385739202 74.5791626901202) bank33290 +33291 POINT(39.953959831373716 73.20744074712418) bank33291 +33292 POINT(40.93147001510581 74.76752130684181) bank33292 +33293 POINT(41.43429203436172 73.53984803515256) bank33293 +33294 POINT(40.48992294308305 74.34758492917678) bank33294 +33295 POINT(41.42458463487039 74.6773047342166) bank33295 +33296 POINT(39.9753365076452 74.48317269868224) bank33296 +33297 POINT(40.94980854655598 73.7051986848688) bank33297 +33298 POINT(40.57063593437722 74.58437937880275) bank33298 +33299 POINT(40.23370622301691 73.15595054566231) bank33299 +33300 POINT(41.69545279272015 74.00462947516682) bank33300 +33301 POINT(40.626416722372426 73.79856387663642) bank33301 +33302 POINT(40.377519758826416 73.28174455697165) bank33302 +33303 POINT(40.359140640727546 73.30014122950699) bank33303 +33304 POINT(41.460799809710075 74.79403887251432) bank33304 +33305 POINT(41.16740072529972 74.5770120049288) bank33305 +33306 POINT(40.467313606699314 73.31747673211576) bank33306 +33307 POINT(40.930994643919476 74.76188402922782) bank33307 +33308 POINT(40.21005845018939 74.91806415008203) bank33308 +33309 POINT(39.951679247624455 74.47089616115683) bank33309 +33310 POINT(41.35760455427391 73.62295956835467) bank33310 +33311 POINT(41.142882703408446 73.12325315357681) bank33311 +33312 POINT(41.21450203259878 73.61851199356208) bank33312 +33313 POINT(41.699002073658 73.87301944032845) bank33313 +33314 POINT(40.73754308415219 73.03751246027518) bank33314 +33315 POINT(39.900469002725124 73.2938740019367) bank33315 +33316 POINT(40.775153571867754 73.74427254122875) bank33316 +33317 POINT(41.3785198219393 73.96502687200686) bank33317 +33318 POINT(40.68832200464795 74.07555497661954) bank33318 +33319 POINT(40.50505193120027 74.39188504090173) bank33319 +33320 POINT(41.441796047623896 73.19824439032267) bank33320 +33321 POINT(41.07937656127431 73.31022912234772) bank33321 +33322 POINT(41.11219158849756 73.22846186269553) bank33322 +33323 POINT(40.33628442503743 74.4611866676934) bank33323 +33324 POINT(41.46000161514528 73.49624711026195) bank33324 +33325 POINT(40.08500391411553 75.00384123050655) bank33325 +33326 POINT(41.12774342393038 73.2656972297272) bank33326 +33327 POINT(41.44273344965596 74.12307030353526) bank33327 +33328 POINT(40.38275890140717 74.81747989061344) bank33328 +33329 POINT(41.189675810858546 74.17536849035086) bank33329 +33330 POINT(41.57419495962232 74.07216035264574) bank33330 +33331 POINT(39.92005243827082 73.47082632971653) bank33331 +33332 POINT(40.60247976498107 73.05665790926436) bank33332 +33333 POINT(40.11245332541942 74.04164450012148) bank33333 +33334 POINT(40.91573724163684 74.27797002106425) bank33334 +33335 POINT(40.30805837688836 73.00781379258055) bank33335 +33336 POINT(41.62065983356864 74.51280116835702) bank33336 +33337 POINT(40.832412642620056 73.77285831120237) bank33337 +33338 POINT(41.476236347020055 74.57725657517045) bank33338 +33339 POINT(39.92247086336637 74.9038594946902) bank33339 +33340 POINT(41.54694525513962 74.5607319203128) bank33340 +33341 POINT(40.20144156918536 73.45704393423728) bank33341 +33342 POINT(40.206828638389524 74.52803082425055) bank33342 +33343 POINT(40.40480379074674 74.82672644688633) bank33343 +33344 POINT(41.69317110790189 74.33737916991923) bank33344 +33345 POINT(40.14866377047841 73.48298955119031) bank33345 +33346 POINT(40.797253433157735 73.21843306162035) bank33346 +33347 POINT(39.90351334005515 74.59414359461493) bank33347 +33348 POINT(41.42533616960873 74.24098279177271) bank33348 +33349 POINT(41.492386627109184 74.15131388054857) bank33349 +33350 POINT(39.883203899863304 73.43770453106026) bank33350 +33351 POINT(40.4762039438889 74.49856164966134) bank33351 +33352 POINT(40.91571983900361 73.10789946034404) bank33352 +33353 POINT(40.17960970725893 74.33499169541525) bank33353 +33354 POINT(40.45779180434867 73.01389932465068) bank33354 +33355 POINT(41.64202330125611 73.58461895091779) bank33355 +33356 POINT(40.43287302249159 73.43434045942006) bank33356 +33357 POINT(40.82251827464371 74.714421893168) bank33357 +33358 POINT(40.11594686902416 74.10215886252413) bank33358 +33359 POINT(40.85387472810435 74.524186949546) bank33359 +33360 POINT(40.52784270259294 74.07634915953996) bank33360 +33361 POINT(41.077931238643316 74.9016989392429) bank33361 +33362 POINT(40.07880709908433 74.72654431372895) bank33362 +33363 POINT(40.545332191799126 74.61628799190922) bank33363 +33364 POINT(41.6657808869884 73.78318742909977) bank33364 +33365 POINT(40.48818986193086 73.77956692526134) bank33365 +33366 POINT(40.100503489001916 73.73841115049898) bank33366 +33367 POINT(40.53455823557438 74.85957403078362) bank33367 +33368 POINT(39.92419292368183 74.29219930873693) bank33368 +33369 POINT(39.96368254028994 74.36327407716466) bank33369 +33370 POINT(41.586923592618824 74.6165294484369) bank33370 +33371 POINT(40.45932997489506 73.65143603941718) bank33371 +33372 POINT(41.1181889016431 73.59969588835536) bank33372 +33373 POINT(40.916315761147814 74.55408446773238) bank33373 +33374 POINT(40.4559161871966 74.99536629133874) bank33374 +33375 POINT(40.81603919007695 74.39072380644781) bank33375 +33376 POINT(41.06698526508177 74.42314895555093) bank33376 +33377 POINT(40.0468867166969 73.2472138634205) bank33377 +33378 POINT(40.369076034553835 73.37901014021155) bank33378 +33379 POINT(41.20502522798584 74.40121152776055) bank33379 +33380 POINT(40.97840230631577 73.70436531651283) bank33380 +33381 POINT(41.173416213238426 73.41933388848028) bank33381 +33382 POINT(39.90302130623327 74.13005224339295) bank33382 +33383 POINT(40.579334482518185 74.00618273380526) bank33383 +33384 POINT(41.161480937875396 73.52650502402695) bank33384 +33385 POINT(40.09145298354248 74.62722663847715) bank33385 +33386 POINT(40.188545922446174 73.28842479762012) bank33386 +33387 POINT(40.195360362823834 73.61671407876499) bank33387 +33388 POINT(39.93335151899417 73.99798746540642) bank33388 +33389 POINT(39.739983174177475 73.49139084594881) bank33389 +33390 POINT(41.239487221863946 73.8876702639039) bank33390 +33391 POINT(40.99601880446532 73.33613617460924) bank33391 +33392 POINT(41.32976390498196 73.8418413156033) bank33392 +33393 POINT(41.25064476058011 73.85075560145528) bank33393 +33394 POINT(41.31818787233031 73.56366676684759) bank33394 +33395 POINT(40.13176395173353 73.21383534818186) bank33395 +33396 POINT(40.25769993031689 73.90180425903065) bank33396 +33397 POINT(41.51875940395781 73.14908972925001) bank33397 +33398 POINT(40.307542844283795 73.73223255324679) bank33398 +33399 POINT(40.67046907885739 74.3284236032108) bank33399 +33400 POINT(40.42361153888917 74.7457468357904) bank33400 +33401 POINT(40.81895496029665 73.37498228689745) bank33401 +33402 POINT(40.46081639210612 73.87663243825345) bank33402 +33403 POINT(39.89931259586487 73.46026897178032) bank33403 +33404 POINT(41.47999686967936 73.11998068666509) bank33404 +33405 POINT(40.725741183945644 74.18871873512659) bank33405 +33406 POINT(41.616847388351445 73.18780104580475) bank33406 +33407 POINT(41.488782965502736 74.68884145510974) bank33407 +33408 POINT(40.97447963771728 73.1788313554674) bank33408 +33409 POINT(41.0771240892126 74.23512673898773) bank33409 +33410 POINT(40.984078969639086 73.20059613666626) bank33410 +33411 POINT(40.18477609271398 74.06867707503176) bank33411 +33412 POINT(39.95167405555026 73.47502658225) bank33412 +33413 POINT(41.5656864811649 74.33014100553675) bank33413 +33414 POINT(40.14823554807515 73.11012002209742) bank33414 +33415 POINT(41.11542145325436 73.95648453962608) bank33415 +33416 POINT(40.159026405808945 73.18381471841053) bank33416 +33417 POINT(41.50876147354826 74.34590378064604) bank33417 +33418 POINT(40.12286877473173 74.15648498028439) bank33418 +33419 POINT(41.65139335767374 73.83221979948406) bank33419 +33420 POINT(40.122184718057895 73.90424787734705) bank33420 +33421 POINT(41.03642125058637 74.89093627747437) bank33421 +33422 POINT(41.551058368338424 74.37730877273324) bank33422 +33423 POINT(40.858480665889665 74.36367117056011) bank33423 +33424 POINT(41.26578246236876 74.85961107224745) bank33424 +33425 POINT(41.28044798629011 73.53793243718343) bank33425 +33426 POINT(40.72944313350213 73.49468092129007) bank33426 +33427 POINT(41.57075987702365 74.63727662898901) bank33427 +33428 POINT(40.63438806987189 74.91859008336806) bank33428 +33429 POINT(40.785194282323545 74.32181411552877) bank33429 +33430 POINT(39.87894813928308 74.42628469910804) bank33430 +33431 POINT(41.0050604401402 74.52186238813977) bank33431 +33432 POINT(40.812608115043396 73.99257577808254) bank33432 +33433 POINT(40.52587604256871 74.98462836934665) bank33433 +33434 POINT(41.691026391836985 74.92129058451225) bank33434 +33435 POINT(40.71466275720365 73.79472603361721) bank33435 +33436 POINT(41.004642610764726 74.64924405770822) bank33436 +33437 POINT(40.17860162972343 74.90390825554758) bank33437 +33438 POINT(39.96058113821954 74.69011745939916) bank33438 +33439 POINT(41.51162406099687 73.26347280951498) bank33439 +33440 POINT(41.406575214582716 73.6779637613641) bank33440 +33441 POINT(41.17831322313833 73.25670471450663) bank33441 +33442 POINT(40.87072684038763 73.58628053943106) bank33442 +33443 POINT(40.01984562946833 73.4195660546038) bank33443 +33444 POINT(41.125579418538344 74.70140817247085) bank33444 +33445 POINT(40.856386529281615 74.09930808379616) bank33445 +33446 POINT(40.88962779279232 73.9898417670647) bank33446 +33447 POINT(41.33040709213667 73.78325379307854) bank33447 +33448 POINT(41.571435060909764 74.0900945870592) bank33448 +33449 POINT(40.29970011451146 73.54075481062814) bank33449 +33450 POINT(40.51950570314817 74.80421041359864) bank33450 +33451 POINT(40.21015276762492 73.65360475945954) bank33451 +33452 POINT(41.57059301337875 73.05894870508206) bank33452 +33453 POINT(40.74065548731879 74.8038788393451) bank33453 +33454 POINT(40.69300483546483 73.46393873416646) bank33454 +33455 POINT(40.275774020978105 73.82534804980574) bank33455 +33456 POINT(41.594661554052095 74.81247032521601) bank33456 +33457 POINT(40.936242563050996 73.37445092265264) bank33457 +33458 POINT(40.37745697540228 74.27217304598328) bank33458 +33459 POINT(39.79606821229793 74.98319917003647) bank33459 +33460 POINT(41.44453010808029 74.3810934280906) bank33460 +33461 POINT(41.01319823875081 73.40430752388912) bank33461 +33462 POINT(40.4392188187037 74.98098061844557) bank33462 +33463 POINT(40.992787164696615 73.29477050352071) bank33463 +33464 POINT(40.280237251368185 74.51846704429828) bank33464 +33465 POINT(39.81992129804125 74.16866780994776) bank33465 +33466 POINT(41.17168782825628 73.95566247361936) bank33466 +33467 POINT(41.279238593225735 74.1278524396631) bank33467 +33468 POINT(41.25827063608531 74.39231795922906) bank33468 +33469 POINT(40.268097519535175 73.77698660045797) bank33469 +33470 POINT(41.635283268725544 74.23000673926605) bank33470 +33471 POINT(41.156536671247984 74.93811523566569) bank33471 +33472 POINT(39.98596807026635 74.76074951501619) bank33472 +33473 POINT(40.959120862444905 73.29620709882663) bank33473 +33474 POINT(41.251740987534234 74.07039266140065) bank33474 +33475 POINT(40.5857228698218 73.39859889986134) bank33475 +33476 POINT(41.00780048409404 73.25848801643153) bank33476 +33477 POINT(41.48900767836424 73.33614902641517) bank33477 +33478 POINT(40.68521055345754 73.97004783098858) bank33478 +33479 POINT(40.86115137385396 73.77179313667698) bank33479 +33480 POINT(41.61498279265568 73.73382945421194) bank33480 +33481 POINT(41.30322355890282 73.6109918133185) bank33481 +33482 POINT(41.43162176384071 73.11525904571069) bank33482 +33483 POINT(40.2177332674961 74.11505937328184) bank33483 +33484 POINT(40.055894459328194 74.7697863366522) bank33484 +33485 POINT(39.743154633132235 74.4225499986922) bank33485 +33486 POINT(40.89881021583691 74.201890876404) bank33486 +33487 POINT(40.642886873817396 73.16344919492823) bank33487 +33488 POINT(40.31128692135548 74.59734303013411) bank33488 +33489 POINT(40.12748262685776 74.41906528406773) bank33489 +33490 POINT(40.725739374795396 74.17818799152239) bank33490 +33491 POINT(40.14642046315201 74.51503423059275) bank33491 +33492 POINT(41.656883392251345 73.1292375888502) bank33492 +33493 POINT(40.00992156099399 73.85301521124643) bank33493 +33494 POINT(41.59352626861743 74.0518216913252) bank33494 +33495 POINT(39.721501066725764 73.22072538223544) bank33495 +33496 POINT(41.69160420056142 73.01757018600311) bank33496 +33497 POINT(40.81939246222578 73.85537435551765) bank33497 +33498 POINT(40.16748143275312 73.55362204861272) bank33498 +33499 POINT(40.63344803768777 73.63331493833198) bank33499 +33500 POINT(41.26345436081931 73.22337828977498) bank33500 +33501 POINT(40.59169773802895 73.29209232405725) bank33501 +33502 POINT(41.159368126392174 73.61306504137725) bank33502 +33503 POINT(41.16314709512822 73.4499096226012) bank33503 +33504 POINT(40.08144198575867 73.39461514881424) bank33504 +33505 POINT(40.61158263743708 74.43682509739527) bank33505 +33506 POINT(41.00261306844195 73.63767150487895) bank33506 +33507 POINT(40.748995683723294 74.47103399193008) bank33507 +33508 POINT(41.22834510125895 73.32776351310963) bank33508 +33509 POINT(39.89546108048151 74.08852081842386) bank33509 +33510 POINT(39.96203458264828 73.744710261431) bank33510 +33511 POINT(40.26932903068705 73.8257043143783) bank33511 +33512 POINT(41.42801941534663 73.02880348619244) bank33512 +33513 POINT(40.3581967556452 73.63890132098965) bank33513 +33514 POINT(41.6145547714436 74.31283107513637) bank33514 +33515 POINT(40.73464823089747 74.79848512597206) bank33515 +33516 POINT(40.892614774394744 73.66596231374957) bank33516 +33517 POINT(41.17034223943899 74.59540948932415) bank33517 +33518 POINT(41.71010555108678 74.24543427008854) bank33518 +33519 POINT(40.751793747039606 73.34661026158099) bank33519 +33520 POINT(40.90428011774541 73.38736978740215) bank33520 +33521 POINT(41.18850152680419 74.73269154929558) bank33521 +33522 POINT(39.94298735467625 73.37793625611671) bank33522 +33523 POINT(40.841365149828476 73.92766988086198) bank33523 +33524 POINT(41.19105515244839 73.19090214448984) bank33524 +33525 POINT(41.708449833764945 74.95862767399393) bank33525 +33526 POINT(40.75273409611186 73.27840586855356) bank33526 +33527 POINT(39.958118082498636 73.92933600816038) bank33527 +33528 POINT(40.533244938966504 74.75568958429832) bank33528 +33529 POINT(40.99577851316993 73.72576454779588) bank33529 +33530 POINT(41.278251553461274 74.67614017869208) bank33530 +33531 POINT(40.655289660992125 74.80136554563016) bank33531 +33532 POINT(41.60804693367915 74.25732457074736) bank33532 +33533 POINT(40.4050609150722 73.39533287876556) bank33533 +33534 POINT(41.239674708603204 74.13837060508206) bank33534 +33535 POINT(40.3616322911307 73.05981867613798) bank33535 +33536 POINT(40.54041569784556 74.48148103127492) bank33536 +33537 POINT(41.384833585122166 73.82883482095839) bank33537 +33538 POINT(39.92668666666005 74.64312549260032) bank33538 +33539 POINT(40.693248846062694 73.06789755311844) bank33539 +33540 POINT(40.86641078168364 73.30026262175255) bank33540 +33541 POINT(40.00071429399364 74.55486513429061) bank33541 +33542 POINT(40.51704101979343 74.3351604704417) bank33542 +33543 POINT(40.45965296700458 74.09263392295696) bank33543 +33544 POINT(39.95374455443334 73.6224685348507) bank33544 +33545 POINT(40.0266074505663 73.959617522608) bank33545 +33546 POINT(40.007908283102964 74.1509057290742) bank33546 +33547 POINT(41.37586573151775 73.85078100354019) bank33547 +33548 POINT(39.73163438205322 73.23414630001773) bank33548 +33549 POINT(40.95929276141108 74.97467665522028) bank33549 +33550 POINT(40.01161369881084 74.48327430322506) bank33550 +33551 POINT(40.372302209859285 74.12829609650791) bank33551 +33552 POINT(39.863300873299025 73.07663358260768) bank33552 +33553 POINT(41.58138195921764 73.52176737146597) bank33553 +33554 POINT(41.71015408240662 73.2680968670513) bank33554 +33555 POINT(39.86394771687792 73.72327625475279) bank33555 +33556 POINT(40.510681864375016 74.55521881035256) bank33556 +33557 POINT(40.34808170231336 73.80104157710825) bank33557 +33558 POINT(40.16579374138758 74.19307349987132) bank33558 +33559 POINT(41.33188800766474 74.66730900124284) bank33559 +33560 POINT(40.97777405050409 74.19789433021788) bank33560 +33561 POINT(39.984391742363925 73.85484563339344) bank33561 +33562 POINT(40.813484933904896 73.81575068833163) bank33562 +33563 POINT(41.07614011494657 73.15961574754134) bank33563 +33564 POINT(41.55663968471849 73.52369158841664) bank33564 +33565 POINT(39.750753452781495 74.80276608629148) bank33565 +33566 POINT(39.72349254395853 73.24874981075692) bank33566 +33567 POINT(40.17977733434383 73.2457074511961) bank33567 +33568 POINT(41.06504757955806 74.33136173382405) bank33568 +33569 POINT(40.76738563356025 74.78882495749271) bank33569 +33570 POINT(41.31511760170967 74.12788542317958) bank33570 +33571 POINT(41.18372009026133 74.01943927135937) bank33571 +33572 POINT(39.817181821809825 75.0052812991446) bank33572 +33573 POINT(41.21254596125641 73.77119360910913) bank33573 +33574 POINT(41.46837004001528 74.28297504553095) bank33574 +33575 POINT(40.285602786637305 74.03814283139256) bank33575 +33576 POINT(40.427215409903795 73.21428171815562) bank33576 +33577 POINT(40.15012302497487 74.67355675760737) bank33577 +33578 POINT(41.19438582623303 73.6392583053778) bank33578 +33579 POINT(41.62800276864677 74.80003368451472) bank33579 +33580 POINT(39.92200706063032 74.7833473424054) bank33580 +33581 POINT(40.51051757971474 74.88481926273502) bank33581 +33582 POINT(41.35893889564779 74.33640679115842) bank33582 +33583 POINT(40.538489641836385 74.09666120172135) bank33583 +33584 POINT(41.326117521914355 73.84575844291624) bank33584 +33585 POINT(41.3500950666767 73.81958530433414) bank33585 +33586 POINT(41.10085740468251 73.42625176738224) bank33586 +33587 POINT(40.45753005402876 73.58335586739408) bank33587 +33588 POINT(41.36889477334357 74.3997365464626) bank33588 +33589 POINT(40.79748803615011 74.03837488942125) bank33589 +33590 POINT(41.08256698217806 74.56939925602414) bank33590 +33591 POINT(41.1495463813408 73.41093758916122) bank33591 +33592 POINT(40.013013999496295 74.70616006194086) bank33592 +33593 POINT(40.07216465565349 73.10395911735688) bank33593 +33594 POINT(40.56250649898691 73.64249611523084) bank33594 +33595 POINT(40.24613322033589 73.32735114003438) bank33595 +33596 POINT(40.47732571396672 74.63002093811349) bank33596 +33597 POINT(40.7341831064503 74.8183455037709) bank33597 +33598 POINT(41.667078283916666 74.96210643734885) bank33598 +33599 POINT(39.81193468735108 73.7381746913457) bank33599 +33600 POINT(40.0036782177232 73.21565054990884) bank33600 +33601 POINT(40.69341430559951 73.02084576489676) bank33601 +33602 POINT(40.439892268259236 74.76344465358459) bank33602 +33603 POINT(41.5722197478679 74.54156024127197) bank33603 +33604 POINT(39.86507515919247 74.9058081295933) bank33604 +33605 POINT(41.28692828608218 73.34517920231355) bank33605 +33606 POINT(39.917638811328516 74.09253554704594) bank33606 +33607 POINT(41.59537146147769 74.0337429490296) bank33607 +33608 POINT(40.218956110500756 74.85073101708504) bank33608 +33609 POINT(39.99821732951982 74.34646179806239) bank33609 +33610 POINT(41.440461551219045 73.50322815560276) bank33610 +33611 POINT(41.30716284638992 73.3332836434029) bank33611 +33612 POINT(40.339895518075586 73.14221083392816) bank33612 +33613 POINT(40.72617567344198 73.55257084625661) bank33613 +33614 POINT(40.38146188644878 74.51933106113957) bank33614 +33615 POINT(39.90779895759059 73.13584824873087) bank33615 +33616 POINT(39.871769708297265 73.14681106895925) bank33616 +33617 POINT(41.419811733248125 73.73811038852071) bank33617 +33618 POINT(40.25294435191364 74.01008397310578) bank33618 +33619 POINT(41.331375650499304 73.73429078199494) bank33619 +33620 POINT(40.037902896852664 74.53664606913343) bank33620 +33621 POINT(40.953792831645096 74.16789173345786) bank33621 +33622 POINT(40.5731577156386 74.98658159661053) bank33622 +33623 POINT(40.52179671531181 74.47296317043478) bank33623 +33624 POINT(40.01643291888087 74.03960222952949) bank33624 +33625 POINT(40.12004545037042 73.48284008319906) bank33625 +33626 POINT(39.75717376584015 73.2764076508743) bank33626 +33627 POINT(41.35897265760218 73.6063496366861) bank33627 +33628 POINT(41.47695467557059 74.1799722966959) bank33628 +33629 POINT(40.123421930550265 73.16810863798692) bank33629 +33630 POINT(40.17314688936312 73.80533801372574) bank33630 +33631 POINT(41.47744506576544 74.00759572835103) bank33631 +33632 POINT(39.94374123003526 73.59958902673733) bank33632 +33633 POINT(40.2329013196993 73.16225085263119) bank33633 +33634 POINT(41.28357677786747 73.42086253573989) bank33634 +33635 POINT(41.57773258139451 74.82333524082837) bank33635 +33636 POINT(41.03117114867497 74.79313547351974) bank33636 +33637 POINT(40.29091140358609 74.62830173437237) bank33637 +33638 POINT(40.311505311676164 73.68893294862089) bank33638 +33639 POINT(40.121906766608475 73.64618842324224) bank33639 +33640 POINT(40.956657647919 73.51896266281608) bank33640 +33641 POINT(41.169818015081205 73.8463643806794) bank33641 +33642 POINT(39.893809754231455 74.87155469499352) bank33642 +33643 POINT(41.362559620543976 73.26889469987624) bank33643 +33644 POINT(40.73265447213284 73.53083917327201) bank33644 +33645 POINT(40.06623438650031 74.35599272978905) bank33645 +33646 POINT(40.87651962899253 74.6534964680577) bank33646 +33647 POINT(40.98786440531253 73.2031496748422) bank33647 +33648 POINT(40.872909003394156 73.95026461757989) bank33648 +33649 POINT(41.336974478341965 73.48931677579696) bank33649 +33650 POINT(41.13956332799513 74.67433152812546) bank33650 +33651 POINT(40.38129479163171 74.60433661444196) bank33651 +33652 POINT(40.573006473699444 74.23976070531667) bank33652 +33653 POINT(40.68645508072775 74.8364991909752) bank33653 +33654 POINT(41.080733582761745 73.03290217644934) bank33654 +33655 POINT(40.10730712207802 73.2431215209354) bank33655 +33656 POINT(41.561092990935315 73.90902121907519) bank33656 +33657 POINT(40.72421250614915 73.47372818319016) bank33657 +33658 POINT(41.21464712509912 73.82652936791881) bank33658 +33659 POINT(40.391868950134395 73.64118757618263) bank33659 +33660 POINT(40.5601212741671 73.13058033839448) bank33660 +33661 POINT(39.77862611535514 74.52670361789603) bank33661 +33662 POINT(41.22213996969372 73.87175140239417) bank33662 +33663 POINT(40.45052897497495 73.0812184892574) bank33663 +33664 POINT(40.020422449638694 73.33163039671177) bank33664 +33665 POINT(39.84476631274987 74.73416052324997) bank33665 +33666 POINT(41.51251680692871 75.00002227346691) bank33666 +33667 POINT(39.72888060871508 74.56808125593396) bank33667 +33668 POINT(41.70352177613936 73.5919548179682) bank33668 +33669 POINT(40.154362476845016 74.52130488782934) bank33669 +33670 POINT(41.18720268562064 74.22384022420697) bank33670 +33671 POINT(40.14640249451081 74.83915308612862) bank33671 +33672 POINT(41.30861926312855 73.32568002516722) bank33672 +33673 POINT(41.45999424156894 73.9345313757128) bank33673 +33674 POINT(41.627704963305746 74.77436983639703) bank33674 +33675 POINT(40.9867926903751 73.94802007561739) bank33675 +33676 POINT(40.50635041070138 74.36710212042327) bank33676 +33677 POINT(39.916059399735225 74.31002059600021) bank33677 +33678 POINT(41.19016094421814 74.98923272930263) bank33678 +33679 POINT(39.84313796855147 74.75957293921073) bank33679 +33680 POINT(41.57832532509038 73.84840824933652) bank33680 +33681 POINT(40.396520739680426 73.1088157953892) bank33681 +33682 POINT(41.31256483289389 74.44343117149431) bank33682 +33683 POINT(40.40630582764113 73.68563195742735) bank33683 +33684 POINT(39.82410827043485 74.21054960937914) bank33684 +33685 POINT(40.9609179344997 74.7137794346291) bank33685 +33686 POINT(41.424297450861424 73.13816113214769) bank33686 +33687 POINT(40.04161732454618 73.89468978013437) bank33687 +33688 POINT(39.77178316321789 74.76669466931315) bank33688 +33689 POINT(39.83611827944184 74.91839875947657) bank33689 +33690 POINT(40.35352722406416 74.4597536264629) bank33690 +33691 POINT(40.33793166045235 73.42791709396997) bank33691 +33692 POINT(39.77160840040484 74.95917907633886) bank33692 +33693 POINT(40.93276776603315 73.14645835909715) bank33693 +33694 POINT(41.30725103930038 73.85026854935441) bank33694 +33695 POINT(41.0295884777783 74.48733068161557) bank33695 +33696 POINT(40.40340128469059 73.14245435679295) bank33696 +33697 POINT(40.1882638627346 73.18084632384786) bank33697 +33698 POINT(41.71192527246284 74.00065782968659) bank33698 +33699 POINT(40.982876284861035 73.80104970042832) bank33699 +33700 POINT(40.59151127750715 73.95967066842226) bank33700 +33701 POINT(39.73452259714069 74.92268688218522) bank33701 +33702 POINT(41.702470950539265 74.18048044042109) bank33702 +33703 POINT(41.315529264190715 74.37983695199331) bank33703 +33704 POINT(41.17313743255116 74.29548523024748) bank33704 +33705 POINT(40.12914442544221 73.66136387727835) bank33705 +33706 POINT(41.58150958952067 74.63842938209682) bank33706 +33707 POINT(40.4704742916083 73.06834648153435) bank33707 +33708 POINT(40.89549276857483 73.64947617201318) bank33708 +33709 POINT(41.446626869179276 74.79561036840488) bank33709 +33710 POINT(40.89749893658075 74.8988924071526) bank33710 +33711 POINT(40.61949443816521 74.99455508393676) bank33711 +33712 POINT(41.188858202514105 74.57238620609192) bank33712 +33713 POINT(41.71195799352098 74.81802765085935) bank33713 +33714 POINT(40.69631746981643 74.21634386512359) bank33714 +33715 POINT(40.98617954168684 74.73964636762595) bank33715 +33716 POINT(40.317474677969074 74.35202761150629) bank33716 +33717 POINT(39.86724469685198 74.62768408107101) bank33717 +33718 POINT(39.72778551976254 74.00029149250747) bank33718 +33719 POINT(39.92875640820285 74.62238523386655) bank33719 +33720 POINT(41.62201915542179 74.59941574763162) bank33720 +33721 POINT(40.50128091545543 74.71053994964011) bank33721 +33722 POINT(41.58687294554953 74.69466942942776) bank33722 +33723 POINT(41.39097527002273 73.03013102153031) bank33723 +33724 POINT(40.70159125630746 73.5299831538037) bank33724 +33725 POINT(39.83290160985382 74.44348087086642) bank33725 +33726 POINT(40.19589254586151 74.36225601005633) bank33726 +33727 POINT(40.04249015676666 74.93011603523354) bank33727 +33728 POINT(40.78132525407886 73.96444236202841) bank33728 +33729 POINT(41.30436378290089 74.20589435378525) bank33729 +33730 POINT(41.38384844208377 73.14852489008291) bank33730 +33731 POINT(40.82620404777041 74.97896656298931) bank33731 +33732 POINT(40.57446451195623 74.6392601712493) bank33732 +33733 POINT(40.72417497514634 74.76756745560472) bank33733 +33734 POINT(41.23133754058284 73.34543443565462) bank33734 +33735 POINT(40.238310145286235 74.58746720079533) bank33735 +33736 POINT(40.57101607332672 74.41913576811962) bank33736 +33737 POINT(41.0520013282077 73.94478714229115) bank33737 +33738 POINT(40.82726648999523 73.01181249709126) bank33738 +33739 POINT(41.466700771886856 73.57766178545617) bank33739 +33740 POINT(40.23569803193153 74.0735739316225) bank33740 +33741 POINT(39.81731358794249 74.99968472018588) bank33741 +33742 POINT(41.70141167985087 73.62239241855947) bank33742 +33743 POINT(41.31838803025289 73.84813420520865) bank33743 +33744 POINT(40.691236936203524 74.10438919144508) bank33744 +33745 POINT(41.63218728199654 74.06407469858928) bank33745 +33746 POINT(40.96818148075915 73.31434049347789) bank33746 +33747 POINT(40.6237699368767 73.83582854095191) bank33747 +33748 POINT(40.14038326959548 74.62243504695034) bank33748 +33749 POINT(40.323453896831424 73.39937451393614) bank33749 +33750 POINT(40.33762259979543 74.73876600062603) bank33750 +33751 POINT(41.06333183660196 74.5688328516999) bank33751 +33752 POINT(40.44673756545123 73.93597578173025) bank33752 +33753 POINT(39.94575317429578 74.5658172360242) bank33753 +33754 POINT(40.47680603083957 74.82556777360257) bank33754 +33755 POINT(41.24185502433702 74.65789683680848) bank33755 +33756 POINT(41.574592404695075 74.69919315381786) bank33756 +33757 POINT(39.89543159224477 74.9923150072494) bank33757 +33758 POINT(40.63843835107211 73.84549267141247) bank33758 +33759 POINT(40.241506997104224 73.97457060436425) bank33759 +33760 POINT(40.241164968657394 73.48269549153045) bank33760 +33761 POINT(40.98252757432794 73.30201500447775) bank33761 +33762 POINT(39.810670981120886 74.77178978496053) bank33762 +33763 POINT(41.61326745445466 74.91386368735664) bank33763 +33764 POINT(41.02587952820358 73.33047317392794) bank33764 +33765 POINT(40.59435483350297 73.39065760487385) bank33765 +33766 POINT(41.62808800434799 74.74933742990181) bank33766 +33767 POINT(41.338055277423194 74.93817561069648) bank33767 +33768 POINT(40.57684486500583 73.45732004425633) bank33768 +33769 POINT(40.61797176690133 74.45394543080263) bank33769 +33770 POINT(39.797526027231804 74.17195031938518) bank33770 +33771 POINT(41.62875748813616 74.55871493215491) bank33771 +33772 POINT(40.405732764400845 73.78001596729366) bank33772 +33773 POINT(40.24263505875309 73.27568641726364) bank33773 +33774 POINT(39.90638527609695 74.64312131079075) bank33774 +33775 POINT(40.289148350199106 73.25957317521363) bank33775 +33776 POINT(41.63674818151748 73.85585117194208) bank33776 +33777 POINT(40.682719211379 74.28004619182448) bank33777 +33778 POINT(41.31202242385577 73.81483276689936) bank33778 +33779 POINT(41.70173448677428 74.227943978903) bank33779 +33780 POINT(41.33324940125064 74.31918362007174) bank33780 +33781 POINT(40.026833599959694 73.97336053965974) bank33781 +33782 POINT(40.21752395538378 74.77860455470523) bank33782 +33783 POINT(41.5671214487916 74.459199794569) bank33783 +33784 POINT(40.94108854688281 74.32989458399992) bank33784 +33785 POINT(39.96383339094285 73.99530229730799) bank33785 +33786 POINT(40.9965281277297 74.27305717891097) bank33786 +33787 POINT(41.34198970073247 73.62122080966355) bank33787 +33788 POINT(41.272852504202525 73.38086952434816) bank33788 +33789 POINT(41.0034479671802 74.06670265507562) bank33789 +33790 POINT(40.6273595173411 74.41702115403369) bank33790 +33791 POINT(40.18365401902282 73.13774204208575) bank33791 +33792 POINT(40.362453949559615 74.13325457305358) bank33792 +33793 POINT(40.165313645779065 74.83933940353863) bank33793 +33794 POINT(40.53599003634811 73.5700816974893) bank33794 +33795 POINT(39.80270923232823 73.37589125102832) bank33795 +33796 POINT(39.7335729135094 74.82927478073388) bank33796 +33797 POINT(41.68344738436978 74.70192991221752) bank33797 +33798 POINT(39.80967652234884 73.26130882235384) bank33798 +33799 POINT(40.81328074231539 73.89468617647974) bank33799 +33800 POINT(41.297573241752396 74.1405241739769) bank33800 +33801 POINT(40.203294249820935 73.60066280911144) bank33801 +33802 POINT(41.16989738641739 74.2631020818933) bank33802 +33803 POINT(39.88576618996938 74.10304919398195) bank33803 +33804 POINT(40.42628220839091 75.00512746913076) bank33804 +33805 POINT(40.39338940576072 74.31890033585499) bank33805 +33806 POINT(41.05018933874181 74.30931305932803) bank33806 +33807 POINT(41.7005615188767 73.77521393705555) bank33807 +33808 POINT(41.20745218899001 73.5051386021651) bank33808 +33809 POINT(41.61682123174419 74.30299588847032) bank33809 +33810 POINT(40.859122944196606 74.16886350390531) bank33810 +33811 POINT(40.07692721203364 74.09209444450319) bank33811 +33812 POINT(40.21421631970772 74.65250400706876) bank33812 +33813 POINT(40.46023035445021 73.60152339304064) bank33813 +33814 POINT(40.51457067419219 74.94453353089108) bank33814 +33815 POINT(40.488444004029986 73.85835665004264) bank33815 +33816 POINT(39.85965911424906 73.1916332286349) bank33816 +33817 POINT(40.32049158889154 74.34176030188308) bank33817 +33818 POINT(40.06816364385058 74.96888780130159) bank33818 +33819 POINT(41.661510601232294 74.77293636528334) bank33819 +33820 POINT(41.02875910753927 73.63540825759402) bank33820 +33821 POINT(41.38820331301651 74.6695294193056) bank33821 +33822 POINT(40.2547368723247 74.62321886288677) bank33822 +33823 POINT(40.15500831780587 73.40935330824199) bank33823 +33824 POINT(41.699741566739384 74.47070485109482) bank33824 +33825 POINT(39.881377571945706 74.9920695905403) bank33825 +33826 POINT(40.53061682772856 73.96742236628687) bank33826 +33827 POINT(41.12537658595273 73.69743541865391) bank33827 +33828 POINT(41.08404550874382 73.27964726451782) bank33828 +33829 POINT(39.78138988571364 73.99287892060538) bank33829 +33830 POINT(40.30703596691148 73.45417998591887) bank33830 +33831 POINT(39.765066660512204 74.12695575743302) bank33831 +33832 POINT(40.723869617504036 74.41825409666134) bank33832 +33833 POINT(40.22222401106223 74.8241532763998) bank33833 +33834 POINT(41.3282996838997 74.40290789414749) bank33834 +33835 POINT(41.23731633229045 74.99687771447424) bank33835 +33836 POINT(40.26311757007772 74.04578348896456) bank33836 +33837 POINT(41.334503774787706 73.33083009706569) bank33837 +33838 POINT(40.90892140721095 74.97667998385599) bank33838 +33839 POINT(41.209746102094805 74.6737432528756) bank33839 +33840 POINT(40.356358424957946 73.19490825094289) bank33840 +33841 POINT(39.73549255944495 73.60603502568239) bank33841 +33842 POINT(40.894273196778215 73.16477034315555) bank33842 +33843 POINT(41.20830076563526 73.17753991869094) bank33843 +33844 POINT(40.799452642408156 73.77057003359494) bank33844 +33845 POINT(40.04092415446055 73.70082183804122) bank33845 +33846 POINT(39.824096529620434 74.08932249113589) bank33846 +33847 POINT(40.63522457948823 73.28268134755157) bank33847 +33848 POINT(40.43748458603258 74.12481287062717) bank33848 +33849 POINT(40.660875320489865 74.89192864918792) bank33849 +33850 POINT(41.120087018009485 74.80229497929344) bank33850 +33851 POINT(40.93387296492194 74.39363425298991) bank33851 +33852 POINT(40.87604843960523 73.20812491293215) bank33852 +33853 POINT(40.03923755149539 74.13150996204614) bank33853 +33854 POINT(41.25716301072001 73.36259079503796) bank33854 +33855 POINT(40.2609451576803 73.8941888625516) bank33855 +33856 POINT(41.12554438932399 73.14401534965205) bank33856 +33857 POINT(41.26861402604468 73.02117232600496) bank33857 +33858 POINT(41.671618088408884 73.4193108444034) bank33858 +33859 POINT(40.32244094809571 74.50664915944743) bank33859 +33860 POINT(41.21019741409474 74.63034609864552) bank33860 +33861 POINT(40.95705082899525 73.72725041169403) bank33861 +33862 POINT(39.90474543773338 73.88182506224634) bank33862 +33863 POINT(40.519037438094706 73.18184568309434) bank33863 +33864 POINT(39.72245942108535 74.95544779365454) bank33864 +33865 POINT(41.211848820925944 74.13292363700637) bank33865 +33866 POINT(41.53899789523713 74.73816710564188) bank33866 +33867 POINT(40.719703254604866 74.58924304384651) bank33867 +33868 POINT(41.55965193578961 73.45170395981071) bank33868 +33869 POINT(41.30745669176033 73.45691662657987) bank33869 +33870 POINT(41.25291803557078 74.06919702416799) bank33870 +33871 POINT(41.436232036552376 73.20573431513833) bank33871 +33872 POINT(41.61517836618024 73.44659754504995) bank33872 +33873 POINT(41.50842403026652 73.69186380602646) bank33873 +33874 POINT(41.3091166741063 73.15918756653569) bank33874 +33875 POINT(41.11849444778422 74.61525671374484) bank33875 +33876 POINT(40.95454609819313 74.24898031786478) bank33876 +33877 POINT(39.73830714634073 73.00917632158583) bank33877 +33878 POINT(40.509248377619045 74.98215888824394) bank33878 +33879 POINT(40.0255987370945 74.80169924609983) bank33879 +33880 POINT(40.74826784492014 74.89776809586988) bank33880 +33881 POINT(41.08827008498361 73.5102232042012) bank33881 +33882 POINT(40.122909134464145 74.54235642788299) bank33882 +33883 POINT(40.686430123773405 73.44097780493533) bank33883 +33884 POINT(39.97007651173504 74.51971664372556) bank33884 +33885 POINT(40.74820003761317 73.08901248766962) bank33885 +33886 POINT(40.149571010673625 74.66474241757666) bank33886 +33887 POINT(40.52606363562111 73.64252976464577) bank33887 +33888 POINT(41.43400215406526 73.86518732584017) bank33888 +33889 POINT(40.81068927514482 74.47228924514346) bank33889 +33890 POINT(40.33848954770121 74.71493659932192) bank33890 +33891 POINT(40.198712616154104 73.75584447579732) bank33891 +33892 POINT(40.4027143443462 74.43950518762517) bank33892 +33893 POINT(40.299297342083676 74.66109963321914) bank33893 +33894 POINT(40.021978044494226 74.86283838454781) bank33894 +33895 POINT(40.69014348183445 74.54487915020026) bank33895 +33896 POINT(40.20737673220307 73.24880446645837) bank33896 +33897 POINT(40.87603417218256 73.77057372581486) bank33897 +33898 POINT(39.88564149533783 74.2892325117909) bank33898 +33899 POINT(41.25952904866549 73.41994555104803) bank33899 +33900 POINT(41.27500254604529 73.82111460973374) bank33900 +33901 POINT(40.873118558751884 74.88446086165115) bank33901 +33902 POINT(40.769855806679764 73.68591648858597) bank33902 +33903 POINT(39.7194487160924 74.84656775146291) bank33903 +33904 POINT(40.458896808236474 73.31791480262122) bank33904 +33905 POINT(41.54400465299005 74.35561507512381) bank33905 +33906 POINT(41.47395743700549 73.4621603752417) bank33906 +33907 POINT(40.539663052995 74.58257042638918) bank33907 +33908 POINT(40.403928397168045 74.26959337345373) bank33908 +33909 POINT(39.73982919177722 73.51247012120191) bank33909 +33910 POINT(41.01501490000092 74.28655412049203) bank33910 +33911 POINT(39.73743242338378 73.56481468618655) bank33911 +33912 POINT(40.45363162260328 73.14857425704032) bank33912 +33913 POINT(40.44120046022669 73.12301598982822) bank33913 +33914 POINT(41.6648324063346 74.43096164159739) bank33914 +33915 POINT(40.351539334808216 74.75847436762106) bank33915 +33916 POINT(40.94039550558987 74.7478513864228) bank33916 +33917 POINT(40.03402532598774 74.32683141322752) bank33917 +33918 POINT(41.3361457456107 73.58772096064328) bank33918 +33919 POINT(40.665016735648564 73.27413041864416) bank33919 +33920 POINT(41.44404493786625 73.98501018302159) bank33920 +33921 POINT(41.223745179060344 74.46586490169634) bank33921 +33922 POINT(41.6519068099352 73.47230378137465) bank33922 +33923 POINT(40.33334481745566 74.75959579093893) bank33923 +33924 POINT(40.64700591975287 73.65974593381956) bank33924 +33925 POINT(40.24076264658193 74.95090733792463) bank33925 +33926 POINT(40.841936737742124 73.88449890926594) bank33926 +33927 POINT(39.82279231983736 73.61782136614235) bank33927 +33928 POINT(40.2682880577514 73.8542164888906) bank33928 +33929 POINT(39.72965008083231 74.52203530673962) bank33929 +33930 POINT(41.18117165075111 73.09335836403822) bank33930 +33931 POINT(40.23158800853212 73.65015390220621) bank33931 +33932 POINT(41.020783307136874 74.89912461656732) bank33932 +33933 POINT(41.09858256243321 74.65014390338108) bank33933 +33934 POINT(41.03957883242304 74.98121330714277) bank33934 +33935 POINT(40.185804548209596 73.3526719364858) bank33935 +33936 POINT(40.834298268836726 73.08618288938823) bank33936 +33937 POINT(40.25341314362562 73.96171804300661) bank33937 +33938 POINT(39.714354606649515 73.80252464603252) bank33938 +33939 POINT(40.171745275272244 74.81663628633171) bank33939 +33940 POINT(41.55180386835056 73.30755846898137) bank33940 +33941 POINT(40.35421204119182 74.78477954961973) bank33941 +33942 POINT(39.856864480672385 73.45212044468822) bank33942 +33943 POINT(39.95781059597901 74.7695674841089) bank33943 +33944 POINT(41.30412168886415 74.53703601619809) bank33944 +33945 POINT(39.851611572025554 73.01385255265204) bank33945 +33946 POINT(41.50948920181043 73.38119873454012) bank33946 +33947 POINT(40.143427477757754 74.41366861188814) bank33947 +33948 POINT(41.223368842630904 73.52495972032328) bank33948 +33949 POINT(40.99896922720201 73.37648680529222) bank33949 +33950 POINT(40.114687128507896 73.18640374090093) bank33950 +33951 POINT(40.87732056534719 74.64505895332421) bank33951 +33952 POINT(41.66645484726813 74.36658138807833) bank33952 +33953 POINT(40.75033844448087 73.18018202972137) bank33953 +33954 POINT(40.89842597614412 74.17998575902853) bank33954 +33955 POINT(40.03014685409638 73.92082857722724) bank33955 +33956 POINT(40.01254097252128 74.91464867683463) bank33956 +33957 POINT(39.999129002130836 73.35221126990633) bank33957 +33958 POINT(41.2103303172287 73.2405799300623) bank33958 +33959 POINT(41.28704248224297 74.1113776193575) bank33959 +33960 POINT(41.192245576255644 73.70938340093184) bank33960 +33961 POINT(40.09934602271966 73.02261467862145) bank33961 +33962 POINT(41.22477554197079 73.98628052492525) bank33962 +33963 POINT(41.119128595681424 74.30631987017132) bank33963 +33964 POINT(39.813261748801594 74.50069756876506) bank33964 +33965 POINT(41.17625056457835 73.42875186463775) bank33965 +33966 POINT(41.5697539521423 73.0131180089091) bank33966 +33967 POINT(40.758678338593526 74.47583538014003) bank33967 +33968 POINT(41.423674718037645 74.68413152923398) bank33968 +33969 POINT(39.90298826483677 74.76432000751504) bank33969 +33970 POINT(41.19823025420463 74.61315753600257) bank33970 +33971 POINT(40.12442347887328 74.2888687096732) bank33971 +33972 POINT(41.635977541827764 73.25863350850146) bank33972 +33973 POINT(39.969573333824556 73.98109125297684) bank33973 +33974 POINT(40.635876574825886 74.30554178476802) bank33974 +33975 POINT(40.69894006251371 74.01052025951019) bank33975 +33976 POINT(40.61722180918931 73.98868360547254) bank33976 +33977 POINT(41.41490667603833 74.28545703509126) bank33977 +33978 POINT(41.61326283542932 74.46620365204603) bank33978 +33979 POINT(39.89216508908465 74.11688398291399) bank33979 +33980 POINT(39.90671251738706 73.32342273893494) bank33980 +33981 POINT(41.52481992961792 73.25645975459979) bank33981 +33982 POINT(40.572216173143005 73.8124150068904) bank33982 +33983 POINT(39.910416646815975 73.51513601862123) bank33983 +33984 POINT(40.725009082031214 74.71773069419395) bank33984 +33985 POINT(40.49785514179645 73.28035890131966) bank33985 +33986 POINT(39.98934928136704 73.14792146946554) bank33986 +33987 POINT(40.843395695397156 73.77439337946531) bank33987 +33988 POINT(40.78172787661617 74.39876192227258) bank33988 +33989 POINT(41.19346754822865 73.74949137360605) bank33989 +33990 POINT(41.51919805970015 74.2372776975611) bank33990 +33991 POINT(39.82516925788289 73.7302864386889) bank33991 +33992 POINT(40.488913912052766 74.79764663578892) bank33992 +33993 POINT(41.41419277546049 73.51198925363583) bank33993 +33994 POINT(41.24677733171835 73.7022781069234) bank33994 +33995 POINT(40.06111622914558 74.70754107766078) bank33995 +33996 POINT(41.18047293969567 74.54815507659805) bank33996 +33997 POINT(40.09101872893071 73.825233791622) bank33997 +33998 POINT(39.836631836113625 74.65823979757269) bank33998 +33999 POINT(40.713964858604655 74.70216224308236) bank33999 +34000 POINT(41.319136756060495 73.58111520724927) bank34000 +34001 POINT(41.18341589517302 73.27969329866913) bank34001 +34002 POINT(41.64124462352926 73.03118065531666) bank34002 +34003 POINT(41.17065615698399 74.64518728108465) bank34003 +34004 POINT(40.50933133035104 74.18417486661333) bank34004 +34005 POINT(40.79467093476759 74.50987693698627) bank34005 +34006 POINT(40.78676617898373 73.77149444281311) bank34006 +34007 POINT(39.95879882865843 73.73650996182391) bank34007 +34008 POINT(39.771358410325305 74.092219868447) bank34008 +34009 POINT(39.953134777374636 74.7901996877839) bank34009 +34010 POINT(40.37481908065671 73.86345509870597) bank34010 +34011 POINT(40.1300373887168 73.66272337291153) bank34011 +34012 POINT(40.837107170358905 74.71826023006044) bank34012 +34013 POINT(40.84182028539222 74.31176743720702) bank34013 +34014 POINT(41.220727249724554 73.38235661625497) bank34014 +34015 POINT(40.34668782978102 74.80872042654845) bank34015 +34016 POINT(40.244558861948214 74.0503361906196) bank34016 +34017 POINT(40.1154545550123 74.33672116486761) bank34017 +34018 POINT(39.782823952074374 74.21963471618014) bank34018 +34019 POINT(39.75477498536986 73.59461367175669) bank34019 +34020 POINT(41.687752775228404 74.86901533917643) bank34020 +34021 POINT(41.59578765333918 73.19516928378734) bank34021 +34022 POINT(39.90894489874512 73.21886963905676) bank34022 +34023 POINT(41.31725740679107 74.30946303991583) bank34023 +34024 POINT(41.48454000469306 73.88347840030038) bank34024 +34025 POINT(40.377550720142835 74.5990363362059) bank34025 +34026 POINT(40.623739773845784 74.76566070433876) bank34026 +34027 POINT(41.64972309727652 74.59073256009427) bank34027 +34028 POINT(41.14131359868249 74.89121421018359) bank34028 +34029 POINT(40.519249280338016 74.97906070691242) bank34029 +34030 POINT(41.159352046938864 74.38341906868132) bank34030 +34031 POINT(40.63109980918721 74.76007611640021) bank34031 +34032 POINT(40.07595948940692 74.10072591385644) bank34032 +34033 POINT(40.638390790314716 73.2849501293688) bank34033 +34034 POINT(39.89395137894638 73.09942429732105) bank34034 +34035 POINT(41.27449844071208 74.97811629212336) bank34035 +34036 POINT(41.21547968091957 73.41789000386477) bank34036 +34037 POINT(40.11752819228844 74.72272084844406) bank34037 +34038 POINT(41.32930718024807 74.9048718202139) bank34038 +34039 POINT(41.49657257113372 73.47249061541352) bank34039 +34040 POINT(40.1588128530703 74.83887825902465) bank34040 +34041 POINT(40.047415824316495 74.37255221954102) bank34041 +34042 POINT(40.53081968821627 74.10455146200225) bank34042 +34043 POINT(41.56483254068749 73.07054293307861) bank34043 +34044 POINT(41.22330736067793 73.53352837961361) bank34044 +34045 POINT(41.06320481163149 73.13064245568572) bank34045 +34046 POINT(41.30456193424289 73.8178996673446) bank34046 +34047 POINT(40.7982178244424 73.53709190004317) bank34047 +34048 POINT(41.5605313668836 73.63806580256505) bank34048 +34049 POINT(40.90519929886234 73.05943760073978) bank34049 +34050 POINT(40.84224951071399 73.06169415870733) bank34050 +34051 POINT(41.21490947009153 73.82088744146577) bank34051 +34052 POINT(41.493866510298105 74.40468101553702) bank34052 +34053 POINT(40.498537793994245 74.17232428716201) bank34053 +34054 POINT(40.20313441676245 74.56247442698617) bank34054 +34055 POINT(40.078482328946585 74.69702721337374) bank34055 +34056 POINT(41.15282724535741 73.00793068441216) bank34056 +34057 POINT(40.41008358898071 73.254208362579) bank34057 +34058 POINT(40.301259731524404 73.05462144048323) bank34058 +34059 POINT(39.949378538879564 73.48208422678871) bank34059 +34060 POINT(40.42934808673898 74.5704546900621) bank34060 +34061 POINT(40.81363429837427 74.3645138102044) bank34061 +34062 POINT(40.823082486774716 74.51168939647812) bank34062 +34063 POINT(41.56471385476003 73.69031054210826) bank34063 +34064 POINT(40.61050926177458 73.91645306729839) bank34064 +34065 POINT(41.4361196508728 73.75426956733916) bank34065 +34066 POINT(41.32298309841753 73.01419977674996) bank34066 +34067 POINT(40.10184678329492 74.87351136175498) bank34067 +34068 POINT(39.975782347132906 73.5927796498361) bank34068 +34069 POINT(39.91383177738147 73.69887576347496) bank34069 +34070 POINT(40.214294102780876 74.78939897058716) bank34070 +34071 POINT(39.83624270637069 74.00042705179213) bank34071 +34072 POINT(40.662643750064575 74.11977833690112) bank34072 +34073 POINT(40.22141708529519 74.84948930797974) bank34073 +34074 POINT(41.33260816353241 73.98718446775977) bank34074 +34075 POINT(40.8009480488712 74.90213407153914) bank34075 +34076 POINT(39.76642317980831 74.5098975047021) bank34076 +34077 POINT(40.54184345367936 74.5507951840195) bank34077 +34078 POINT(40.498939332022 73.97154322106144) bank34078 +34079 POINT(40.62335659826639 73.97003868339117) bank34079 +34080 POINT(41.072514716357304 74.63527156713381) bank34080 +34081 POINT(40.30225098940398 74.47173258215493) bank34081 +34082 POINT(40.35182618685733 73.18558441348563) bank34082 +34083 POINT(40.39727259965331 74.8743021715831) bank34083 +34084 POINT(39.740861740066165 74.36957199785186) bank34084 +34085 POINT(40.5669484501385 74.49772041952774) bank34085 +34086 POINT(41.312438029584534 74.46042893049173) bank34086 +34087 POINT(40.26945599888045 74.26668254188907) bank34087 +34088 POINT(41.67544986739799 74.37826837281116) bank34088 +34089 POINT(39.80817881488912 73.3080883813177) bank34089 +34090 POINT(40.77586813354398 74.84711662186241) bank34090 +34091 POINT(40.259840965074275 74.6466761451124) bank34091 +34092 POINT(40.67041287182198 73.36490642651304) bank34092 +34093 POINT(40.24395884111969 73.50300513165078) bank34093 +34094 POINT(41.488351951515355 74.49695493093706) bank34094 +34095 POINT(41.56641410350761 73.55250749787847) bank34095 +34096 POINT(40.84388484074162 73.90473153230809) bank34096 +34097 POINT(41.41344409453618 74.65259150110563) bank34097 +34098 POINT(40.392439575106735 74.5235043017301) bank34098 +34099 POINT(41.08968919466385 74.05314592537857) bank34099 +34100 POINT(40.46645792680588 74.44987709626959) bank34100 +34101 POINT(40.623511547166835 73.6804157500977) bank34101 +34102 POINT(40.818567761156025 74.52204876851565) bank34102 +34103 POINT(41.37779724931279 74.39062520516956) bank34103 +34104 POINT(40.13808694466724 74.88213746240417) bank34104 +34105 POINT(40.89017976111017 73.21563216928077) bank34105 +34106 POINT(39.94836379388629 74.86180213102752) bank34106 +34107 POINT(41.217236449549965 74.42782764977441) bank34107 +34108 POINT(40.00759949523438 74.01066398613072) bank34108 +34109 POINT(40.946649931537486 73.62503708386272) bank34109 +34110 POINT(39.7919422507337 74.8417137486978) bank34110 +34111 POINT(41.12852765088963 73.37865913997786) bank34111 +34112 POINT(40.19454689065628 74.81805852944932) bank34112 +34113 POINT(41.51462602584624 74.86480054033109) bank34113 +34114 POINT(41.454238835074825 73.4008326912008) bank34114 +34115 POINT(41.50493781238292 74.86940404021338) bank34115 +34116 POINT(41.55125973877283 73.48638483155672) bank34116 +34117 POINT(40.47572691008603 73.49875273399941) bank34117 +34118 POINT(41.38926403085013 74.38567710317115) bank34118 +34119 POINT(40.37799945213647 74.33440037729264) bank34119 +34120 POINT(41.2275071014461 74.14433463012537) bank34120 +34121 POINT(41.26484821157556 74.12039608244437) bank34121 +34122 POINT(40.8777515331348 73.87985207028798) bank34122 +34123 POINT(40.48456760982656 73.93536044027618) bank34123 +34124 POINT(40.59428297143565 73.26819719435598) bank34124 +34125 POINT(40.347934885460155 73.52671322125028) bank34125 +34126 POINT(40.03560854322562 73.02440220053901) bank34126 +34127 POINT(41.61122044995402 74.1555678920896) bank34127 +34128 POINT(39.954586662278594 74.6792154077555) bank34128 +34129 POINT(41.01667463568237 73.50369668542055) bank34129 +34130 POINT(40.74888610638424 73.64436568824026) bank34130 +34131 POINT(41.01910442340849 73.94563091428202) bank34131 +34132 POINT(40.28005500509882 73.84921166225051) bank34132 +34133 POINT(41.154846125611016 73.12713809265169) bank34133 +34134 POINT(41.54658729684753 73.14720811949523) bank34134 +34135 POINT(41.69719048132895 74.61153648539486) bank34135 +34136 POINT(40.21188543183457 73.84051190663038) bank34136 +34137 POINT(39.72853141397751 74.70919107828404) bank34137 +34138 POINT(39.731371553577304 74.7194361629261) bank34138 +34139 POINT(40.80382539486158 74.6149824630104) bank34139 +34140 POINT(40.03003909381576 74.96084050411216) bank34140 +34141 POINT(41.525102898655824 74.43354279637913) bank34141 +34142 POINT(40.46218821704746 73.88657189976652) bank34142 +34143 POINT(40.40023715628217 74.9100755549345) bank34143 +34144 POINT(40.20327334056871 73.61531933727342) bank34144 +34145 POINT(41.40461899746883 74.8880207541145) bank34145 +34146 POINT(41.39813527779407 74.08249711693662) bank34146 +34147 POINT(41.10737855603514 74.28840825417127) bank34147 +34148 POINT(40.071042250156744 73.71678991405258) bank34148 +34149 POINT(40.91786597665275 73.40729330982202) bank34149 +34150 POINT(41.16763808329917 73.23670042859558) bank34150 +34151 POINT(40.7463005015359 73.89654376851854) bank34151 +34152 POINT(39.75602324994582 73.10098401626871) bank34152 +34153 POINT(40.76002460736489 73.62186839088136) bank34153 +34154 POINT(41.45645972330439 73.2385157762893) bank34154 +34155 POINT(40.37780961392518 73.88702951290144) bank34155 +34156 POINT(41.593174281774125 73.21865915315367) bank34156 +34157 POINT(41.069547648553296 74.00452219897126) bank34157 +34158 POINT(40.00035348403371 73.63256379170173) bank34158 +34159 POINT(41.064382138413976 74.69447259582964) bank34159 +34160 POINT(40.5669642901208 74.87694204739405) bank34160 +34161 POINT(41.285017705896514 74.67225464128066) bank34161 +34162 POINT(40.05178764334684 74.7259434813952) bank34162 +34163 POINT(41.598656776550904 73.58346876725605) bank34163 +34164 POINT(40.81712009565739 73.87817246570162) bank34164 +34165 POINT(40.16438457004588 73.75777560402045) bank34165 +34166 POINT(40.615713847050166 73.34227057518945) bank34166 +34167 POINT(41.25964642358573 73.74508061513606) bank34167 +34168 POINT(39.95908123642202 73.42177350537276) bank34168 +34169 POINT(40.045440949755346 73.05857021308353) bank34169 +34170 POINT(41.39573564288979 74.6698880723217) bank34170 +34171 POINT(40.60118454516229 74.49628597421354) bank34171 +34172 POINT(41.394956864746554 74.95623280035637) bank34172 +34173 POINT(41.31978914318234 74.41725561852722) bank34173 +34174 POINT(40.27442849575022 74.52986824402609) bank34174 +34175 POINT(40.567557733272835 74.93698710375338) bank34175 +34176 POINT(40.82483233588426 74.05020949371833) bank34176 +34177 POINT(40.439673126355636 74.21439460983306) bank34177 +34178 POINT(41.19223009141993 73.66175572753798) bank34178 +34179 POINT(41.08411848945496 73.59783992875352) bank34179 +34180 POINT(40.77116944107889 73.30810792377355) bank34180 +34181 POINT(41.12829450422319 74.63327206827626) bank34181 +34182 POINT(41.372995871429666 73.67637096401258) bank34182 +34183 POINT(41.344701314955806 73.24211260343382) bank34183 +34184 POINT(41.389187450579534 73.78532383468072) bank34184 +34185 POINT(40.44448854734419 74.78240548699587) bank34185 +34186 POINT(41.42330177132863 74.33485943544582) bank34186 +34187 POINT(40.38081547577056 73.36718563643268) bank34187 +34188 POINT(40.48179897858868 73.11854508340761) bank34188 +34189 POINT(41.02827589631725 73.29127970505708) bank34189 +34190 POINT(40.974740990906696 73.85276171287833) bank34190 +34191 POINT(41.652148605679805 73.95810896896339) bank34191 +34192 POINT(40.732937449409825 73.91004545717392) bank34192 +34193 POINT(41.31769297012364 73.17403095272404) bank34193 +34194 POINT(40.76192220963945 74.57478479730803) bank34194 +34195 POINT(40.16636899333428 74.7848358760547) bank34195 +34196 POINT(40.40155336181061 74.43858670581191) bank34196 +34197 POINT(40.46365339208159 73.9753461856786) bank34197 +34198 POINT(40.36863925736541 73.93060392501701) bank34198 +34199 POINT(40.406998469570745 74.96326502387154) bank34199 +34200 POINT(39.98537343130935 74.94432667904266) bank34200 +34201 POINT(41.54448109716517 74.52886987172353) bank34201 +34202 POINT(40.89907140682033 73.73887397381239) bank34202 +34203 POINT(40.5357501961968 73.24094375942576) bank34203 +34204 POINT(40.2036171504203 74.74382685967981) bank34204 +34205 POINT(41.34762398743177 74.32955987177748) bank34205 +34206 POINT(40.56752265456369 74.33471556724291) bank34206 +34207 POINT(40.03208777145872 74.4439680448104) bank34207 +34208 POINT(40.53864108784824 73.4691957546522) bank34208 +34209 POINT(41.01173250874066 74.9492379782727) bank34209 +34210 POINT(41.21723902013319 74.24760442993941) bank34210 +34211 POINT(40.693394703741234 73.10112772083791) bank34211 +34212 POINT(39.87206373568927 73.07385864269652) bank34212 +34213 POINT(41.3600461334519 74.60776106250327) bank34213 +34214 POINT(39.81200172769772 74.83653517846473) bank34214 +34215 POINT(40.311392119814144 74.87128968679072) bank34215 +34216 POINT(41.12254026944554 74.72357014520598) bank34216 +34217 POINT(41.04267119437058 74.98747585729025) bank34217 +34218 POINT(40.53820730983985 74.70207674074749) bank34218 +34219 POINT(40.257904176512724 74.5223573862518) bank34219 +34220 POINT(40.519586505469164 73.23766777025806) bank34220 +34221 POINT(41.370582688603115 74.59340667064836) bank34221 +34222 POINT(40.596633584053656 74.90820801170618) bank34222 +34223 POINT(40.990393694358744 74.66512877739308) bank34223 +34224 POINT(40.78206318736943 73.3236037724301) bank34224 +34225 POINT(40.917852304207194 73.34069215778831) bank34225 +34226 POINT(41.03069546142427 73.14891592041913) bank34226 +34227 POINT(41.35972447759732 74.15533946363739) bank34227 +34228 POINT(39.80841374888322 74.32933925740899) bank34228 +34229 POINT(41.34112144589105 74.95799950143501) bank34229 +34230 POINT(41.35942999246736 74.64236002369182) bank34230 +34231 POINT(40.23386682510972 74.3982283976572) bank34231 +34232 POINT(41.02164563783447 73.61887550275095) bank34232 +34233 POINT(39.766535733923924 74.00415084135783) bank34233 +34234 POINT(40.88392076894075 74.63091120519483) bank34234 +34235 POINT(41.13272666641817 73.18677168995345) bank34235 +34236 POINT(41.31613238342934 74.66931952757893) bank34236 +34237 POINT(41.037722779826666 74.01054337936596) bank34237 +34238 POINT(40.83228887440649 74.60247851968664) bank34238 +34239 POINT(39.74740531315946 73.96384998937008) bank34239 +34240 POINT(40.65967966395703 73.32450480302204) bank34240 +34241 POINT(40.78505155250341 73.69250724503911) bank34241 +34242 POINT(41.15804541220744 73.35829840572734) bank34242 +34243 POINT(40.59273274464966 74.78147001752775) bank34243 +34244 POINT(41.42664812180591 74.12690205753297) bank34244 +34245 POINT(39.99192438467176 73.83715010546986) bank34245 +34246 POINT(41.544742774203215 73.91393080368766) bank34246 +34247 POINT(40.43208777236915 74.33779714233486) bank34247 +34248 POINT(40.1943685012461 74.12040715578307) bank34248 +34249 POINT(40.717927920080015 74.03791378000405) bank34249 +34250 POINT(39.77321351229889 74.80352170228917) bank34250 +34251 POINT(41.52919452923744 74.8707291902704) bank34251 +34252 POINT(40.665978761922794 74.71413240736409) bank34252 +34253 POINT(40.786438803792215 74.3020829204058) bank34253 +34254 POINT(39.85016043242816 73.99239923973907) bank34254 +34255 POINT(40.60781076966653 73.8846232128281) bank34255 +34256 POINT(39.83703354350673 73.27538764670693) bank34256 +34257 POINT(39.85018555896806 73.43570175772531) bank34257 +34258 POINT(40.69861844570193 73.37410439903621) bank34258 +34259 POINT(41.53411250408262 74.59003319982938) bank34259 +34260 POINT(41.667197000964336 74.13644141605158) bank34260 +34261 POINT(40.53637955499826 74.01861507750317) bank34261 +34262 POINT(41.70789649444203 74.4794369421964) bank34262 +34263 POINT(41.41997762424263 74.98292828044761) bank34263 +34264 POINT(41.268694004847475 74.5216835380304) bank34264 +34265 POINT(40.405357644667816 74.15578905076777) bank34265 +34266 POINT(39.74084209055191 73.41672495742802) bank34266 +34267 POINT(41.42512399258945 73.57348126721412) bank34267 +34268 POINT(40.79648143885596 74.50699046803928) bank34268 +34269 POINT(39.74369725448923 73.74089337454647) bank34269 +34270 POINT(40.16655091522318 74.5884994309126) bank34270 +34271 POINT(41.46116453224734 73.48451636694375) bank34271 +34272 POINT(40.06536094266813 74.00799015711777) bank34272 +34273 POINT(40.07503385923543 74.35731510186872) bank34273 +34274 POINT(41.29203571847943 74.37457777207489) bank34274 +34275 POINT(40.76431986578979 74.65327667621855) bank34275 +34276 POINT(40.76268459939172 74.07925292389388) bank34276 +34277 POINT(40.81722115554632 73.60276571253306) bank34277 +34278 POINT(41.55994899204312 74.80658185989574) bank34278 +34279 POINT(40.01788532832838 74.60858154246107) bank34279 +34280 POINT(41.272000116897765 74.46325299494141) bank34280 +34281 POINT(40.22130043354923 74.78471351386393) bank34281 +34282 POINT(39.8611322202633 74.99728799677811) bank34282 +34283 POINT(40.088977858103 74.91290114184936) bank34283 +34284 POINT(41.43911354784316 74.00713679466637) bank34284 +34285 POINT(40.187177538683486 74.39374751670447) bank34285 +34286 POINT(40.05626379389865 74.95570564031277) bank34286 +34287 POINT(41.099842823942076 74.87946213738712) bank34287 +34288 POINT(41.65428737017086 74.10871245097252) bank34288 +34289 POINT(40.14044992163074 73.36944084800591) bank34289 +34290 POINT(41.327540284792654 74.1251941103377) bank34290 +34291 POINT(40.69330852507965 73.96300373526788) bank34291 +34292 POINT(40.506593792190074 73.70704662208102) bank34292 +34293 POINT(40.33571503498689 73.34277798873006) bank34293 +34294 POINT(41.70636773853492 73.53066314468121) bank34294 +34295 POINT(40.428576047929695 74.64961875591393) bank34295 +34296 POINT(41.2131550908346 73.46907897525546) bank34296 +34297 POINT(40.53358178426006 74.1112698555537) bank34297 +34298 POINT(40.527785359449155 74.6754660265279) bank34298 +34299 POINT(40.925750742035234 74.71662490042759) bank34299 +34300 POINT(40.74683063247939 73.57251760350982) bank34300 +34301 POINT(40.10450638290458 74.89153619218504) bank34301 +34302 POINT(41.491027262033725 73.20047307219271) bank34302 +34303 POINT(40.9658497734578 74.54970674592336) bank34303 +34304 POINT(41.57387536494191 74.30330301597142) bank34304 +34305 POINT(40.111257457720754 73.22010132243413) bank34305 +34306 POINT(41.61778524236296 73.36921231203897) bank34306 +34307 POINT(40.08899089712769 73.23207310040621) bank34307 +34308 POINT(40.95058372795518 73.15057746131387) bank34308 +34309 POINT(39.98265551272286 73.0096454145829) bank34309 +34310 POINT(40.893400667325345 74.97704775091476) bank34310 +34311 POINT(40.190831248657595 74.17869642027978) bank34311 +34312 POINT(41.2793757126864 74.013662048623) bank34312 +34313 POINT(40.2695254819817 73.29195482709524) bank34313 +34314 POINT(40.45970665808222 74.3104975074557) bank34314 +34315 POINT(40.67643680883223 73.7621093477024) bank34315 +34316 POINT(41.29025038060712 73.18796425628882) bank34316 +34317 POINT(40.21900685537873 74.14970125749177) bank34317 +34318 POINT(39.72051364633584 73.10914148448335) bank34318 +34319 POINT(40.9927343095503 74.59990594752007) bank34319 +34320 POINT(41.14083290086387 74.37596465950524) bank34320 +34321 POINT(40.62329212730904 74.82524758442206) bank34321 +34322 POINT(40.17152234179091 73.51641013977368) bank34322 +34323 POINT(40.49835639195081 73.30381491276825) bank34323 +34324 POINT(40.90662928106814 74.62533128132958) bank34324 +34325 POINT(41.14571733885433 74.53165361195333) bank34325 +34326 POINT(40.66972447940117 74.38770351788554) bank34326 +34327 POINT(39.767625450206836 73.11112637939944) bank34327 +34328 POINT(41.39705173679641 73.18238035513673) bank34328 +34329 POINT(41.6647055681036 73.11086037763009) bank34329 +34330 POINT(40.16393880843735 73.8555722783771) bank34330 +34331 POINT(40.61497961963509 73.4291837617289) bank34331 +34332 POINT(40.6767102856718 73.22620979269824) bank34332 +34333 POINT(40.168196257271546 73.17110384311559) bank34333 +34334 POINT(40.06048092320389 73.08130754368457) bank34334 +34335 POINT(39.80109322901903 73.80935816752202) bank34335 +34336 POINT(41.502903384361474 73.01404866916364) bank34336 +34337 POINT(41.139409944111556 73.6434371261242) bank34337 +34338 POINT(40.95480869185582 73.42155897333402) bank34338 +34339 POINT(41.68131869890187 74.79168665058455) bank34339 +34340 POINT(40.01832351366368 73.4316045414248) bank34340 +34341 POINT(40.69806466711399 74.61269248103666) bank34341 +34342 POINT(41.41669542882831 73.69024462410546) bank34342 +34343 POINT(41.52532144796965 73.4446877021902) bank34343 +34344 POINT(41.432795014681915 74.53178871702922) bank34344 +34345 POINT(40.245365619012844 74.9328466457096) bank34345 +34346 POINT(40.23930376583118 73.22936990240756) bank34346 +34347 POINT(41.08166872366682 74.85615852103582) bank34347 +34348 POINT(40.10816437257175 73.93565748500063) bank34348 +34349 POINT(40.07601718273621 73.22168883661115) bank34349 +34350 POINT(41.1287819017642 74.5758082078817) bank34350 +34351 POINT(41.53494727404463 74.00196257938158) bank34351 +34352 POINT(41.64725872670695 73.72107612166474) bank34352 +34353 POINT(41.516075704291126 73.17874937363887) bank34353 +34354 POINT(40.09751232263644 73.1641932927532) bank34354 +34355 POINT(40.43951111520147 74.27367119836919) bank34355 +34356 POINT(40.38569204483845 74.19379909004064) bank34356 +34357 POINT(41.08440802928339 74.61986925115319) bank34357 +34358 POINT(41.28705625593583 74.10995862539527) bank34358 +34359 POINT(40.180674792251935 73.1395732595221) bank34359 +34360 POINT(40.897263854256366 73.25592084110366) bank34360 +34361 POINT(39.914777202746556 74.87515667964915) bank34361 +34362 POINT(40.907145563204 74.49109227224665) bank34362 +34363 POINT(41.23831297588682 74.89610392344467) bank34363 +34364 POINT(39.75405823258904 74.19321381483236) bank34364 +34365 POINT(40.73593675629172 74.47250469983418) bank34365 +34366 POINT(41.240943031212424 74.59452448171348) bank34366 +34367 POINT(39.78087943358534 73.29738057991173) bank34367 +34368 POINT(39.97413064693193 73.17134681191408) bank34368 +34369 POINT(40.996851030114215 73.37255473863291) bank34369 +34370 POINT(41.15175342164331 74.63477480293112) bank34370 +34371 POINT(40.87564239195828 73.83259683526732) bank34371 +34372 POINT(40.975812754769834 73.57342481972408) bank34372 +34373 POINT(40.25332277935595 73.29586203552314) bank34373 +34374 POINT(40.808816908449984 74.20600593117948) bank34374 +34375 POINT(41.56758328317531 74.28326202286506) bank34375 +34376 POINT(40.89015206713182 73.6193058496814) bank34376 +34377 POINT(40.87666086672333 74.70865420222748) bank34377 +34378 POINT(41.19870871711882 73.01316834341443) bank34378 +34379 POINT(40.36784528388975 73.70116676355435) bank34379 +34380 POINT(39.762520902675526 73.45665605034831) bank34380 +34381 POINT(39.932530566753215 74.20387363782585) bank34381 +34382 POINT(40.88914996400706 73.16287013813367) bank34382 +34383 POINT(41.441514333352764 74.54089498324585) bank34383 +34384 POINT(40.721545510659865 74.56236318797797) bank34384 +34385 POINT(39.72371335847785 73.2413988628667) bank34385 +34386 POINT(40.45207553885545 74.06084321786786) bank34386 +34387 POINT(40.381401279681405 74.83545986650833) bank34387 +34388 POINT(40.78248652029327 73.27108571632994) bank34388 +34389 POINT(40.75150417271699 73.89836959741787) bank34389 +34390 POINT(40.00026318017707 74.84208948871381) bank34390 +34391 POINT(41.342838643290165 74.78052772201367) bank34391 +34392 POINT(40.957001871500175 74.23021702689374) bank34392 +34393 POINT(40.90184706499982 74.21697310466887) bank34393 +34394 POINT(41.44277990768124 74.8462810212661) bank34394 +34395 POINT(39.730992433443674 73.0958558521417) bank34395 +34396 POINT(40.17943103166304 74.07277584130895) bank34396 +34397 POINT(41.58277937916865 74.40381937285473) bank34397 +34398 POINT(39.81050341839677 73.01788815103366) bank34398 +34399 POINT(40.83309772436685 73.26527750606317) bank34399 +34400 POINT(41.6308198098406 74.13608904272321) bank34400 +34401 POINT(39.844667267852714 73.67470011034504) bank34401 +34402 POINT(41.664895621804654 73.49641688717774) bank34402 +34403 POINT(41.05948558973166 74.31565638588165) bank34403 +34404 POINT(40.87914775830403 74.669687259388) bank34404 +34405 POINT(39.9763783060847 74.68206914124434) bank34405 +34406 POINT(40.50843209373862 74.18244759522713) bank34406 +34407 POINT(40.87123539542888 74.43721725273346) bank34407 +34408 POINT(40.8806085152833 74.47054636571244) bank34408 +34409 POINT(40.09201166340276 74.6221314095505) bank34409 +34410 POINT(39.737957178875845 74.61603036072692) bank34410 +34411 POINT(41.10386193920418 73.69644802934306) bank34411 +34412 POINT(41.486664618756556 74.80128606257392) bank34412 +34413 POINT(40.03191981983621 73.65818587064717) bank34413 +34414 POINT(39.81869136067453 73.37213511290291) bank34414 +34415 POINT(41.506070697019624 73.54922485421737) bank34415 +34416 POINT(40.22949739180388 73.51997925219344) bank34416 +34417 POINT(40.12181561210034 73.25560262494363) bank34417 +34418 POINT(39.751717750183616 73.32915489677728) bank34418 +34419 POINT(41.33882341627119 73.53660352145477) bank34419 +34420 POINT(40.85542341412057 74.49109590757224) bank34420 +34421 POINT(39.83123920028423 73.54278334509762) bank34421 +34422 POINT(41.216397134455356 73.93047910169913) bank34422 +34423 POINT(40.8366114330186 73.08470155898199) bank34423 +34424 POINT(40.17718102412074 73.10015304777365) bank34424 +34425 POINT(40.293520717357225 74.22782549117944) bank34425 +34426 POINT(40.36136168021345 74.67386429637364) bank34426 +34427 POINT(40.8664656088785 74.8277663866575) bank34427 +34428 POINT(41.56582952811333 74.37854299370383) bank34428 +34429 POINT(40.01614717138934 74.64633399196445) bank34429 +34430 POINT(41.14600142930352 74.08570643781587) bank34430 +34431 POINT(41.29398011317612 74.95443396245132) bank34431 +34432 POINT(39.759859079913724 74.34194031398752) bank34432 +34433 POINT(41.29527176168985 73.80819786060894) bank34433 +34434 POINT(41.03420562972613 74.26146674106572) bank34434 +34435 POINT(40.11398795763898 73.32635580238103) bank34435 +34436 POINT(41.1959409709062 74.29799750658974) bank34436 +34437 POINT(40.352354720865684 74.90349468935531) bank34437 +34438 POINT(40.70329717887355 74.83948941288183) bank34438 +34439 POINT(40.62899737545729 73.0720444286447) bank34439 +34440 POINT(41.145365080464444 74.25518818075423) bank34440 +34441 POINT(41.183543312366794 73.37650357843901) bank34441 +34442 POINT(41.68982710049965 73.45839265371691) bank34442 +34443 POINT(40.254089836603306 74.67414887357505) bank34443 +34444 POINT(40.696513588986065 74.5592187127794) bank34444 +34445 POINT(41.67434039851733 73.7804982893135) bank34445 +34446 POINT(39.84625713059023 73.371029531516) bank34446 +34447 POINT(41.51852164725418 74.73918935076851) bank34447 +34448 POINT(39.79273209063851 73.91678163382205) bank34448 +34449 POINT(41.69075233170287 73.05219135460354) bank34449 +34450 POINT(40.71803125346298 74.72984198781592) bank34450 +34451 POINT(40.97899430841402 73.55851336547963) bank34451 +34452 POINT(40.78389030087144 73.8076878748229) bank34452 +34453 POINT(40.20551907522176 73.81548968629149) bank34453 +34454 POINT(41.13468689134347 73.62986698170793) bank34454 +34455 POINT(40.05027864531284 74.4024086098362) bank34455 +34456 POINT(40.751671404515 73.50793697126332) bank34456 +34457 POINT(41.660953234492084 73.12329329122434) bank34457 +34458 POINT(41.27719699444792 73.352944708057) bank34458 +34459 POINT(40.71860854083484 74.87341377596944) bank34459 +34460 POINT(41.20173849202946 73.93415548104812) bank34460 +34461 POINT(41.061505036494374 73.10837385937977) bank34461 +34462 POINT(40.081448888107296 73.41911844436616) bank34462 +34463 POINT(41.5944408313249 73.97505843595683) bank34463 +34464 POINT(40.93976631632251 73.52967191949071) bank34464 +34465 POINT(40.11712145136168 74.27234981949124) bank34465 +34466 POINT(40.616665575431696 73.61485722726037) bank34466 +34467 POINT(41.61350052120371 74.60438502739758) bank34467 +34468 POINT(41.407396003982875 73.56401139772133) bank34468 +34469 POINT(40.68461731878175 74.92836621935861) bank34469 +34470 POINT(41.04191811747574 74.91198972592026) bank34470 +34471 POINT(40.66553652485565 74.72956230900554) bank34471 +34472 POINT(41.12518743817887 74.42766743735346) bank34472 +34473 POINT(41.404465550476004 74.82781969891208) bank34473 +34474 POINT(40.42736759657731 74.11094627167748) bank34474 +34475 POINT(39.79790327562018 74.0416839253393) bank34475 +34476 POINT(40.972263178037785 73.95764763530165) bank34476 +34477 POINT(40.51216579661745 73.9102894895631) bank34477 +34478 POINT(40.92060689509184 74.20050950647915) bank34478 +34479 POINT(41.70204122492517 73.9613148899331) bank34479 +34480 POINT(40.51420372503165 73.00888251143998) bank34480 +34481 POINT(39.81824849288377 73.84611124095825) bank34481 +34482 POINT(41.23270222630946 73.2796064237857) bank34482 +34483 POINT(39.73452083813818 74.9505887020035) bank34483 +34484 POINT(39.92839435004371 74.93641822453415) bank34484 +34485 POINT(41.16784359520782 74.24423993272242) bank34485 +34486 POINT(40.707427080274684 73.14067227524032) bank34486 +34487 POINT(41.067845324745555 74.57755088827786) bank34487 +34488 POINT(40.88588500859587 73.18468441108524) bank34488 +34489 POINT(41.4307455031908 73.52965047437216) bank34489 +34490 POINT(40.02505220395075 74.72661126684733) bank34490 +34491 POINT(39.84662575223009 73.69603614149712) bank34491 +34492 POINT(40.39040123744411 73.10257826044894) bank34492 +34493 POINT(40.830660097123875 73.36365375930932) bank34493 +34494 POINT(41.21855539945277 74.16476861902915) bank34494 +34495 POINT(40.047431059212045 73.81891642632391) bank34495 +34496 POINT(40.61155742861823 74.26571034778199) bank34496 +34497 POINT(40.57189347660449 73.48762171441463) bank34497 +34498 POINT(39.805239291356465 74.22922854685267) bank34498 +34499 POINT(39.99313688839314 73.76722006304342) bank34499 +34500 POINT(40.60547858288171 74.51000201581228) bank34500 +34501 POINT(40.952659919882215 73.64001240475642) bank34501 +34502 POINT(41.56182955602346 74.30592454365022) bank34502 +34503 POINT(39.800615011436285 74.71883977746414) bank34503 +34504 POINT(41.35068154720125 74.7536993270374) bank34504 +34505 POINT(41.16876075126943 73.90438451546393) bank34505 +34506 POINT(40.052521071530116 73.38752963912373) bank34506 +34507 POINT(40.205947864179294 73.85483900708893) bank34507 +34508 POINT(41.29848900991883 74.24408786626752) bank34508 +34509 POINT(40.408771742895084 74.57227850812959) bank34509 +34510 POINT(41.456515763824925 73.34148096993913) bank34510 +34511 POINT(41.374767556264636 74.48910336944684) bank34511 +34512 POINT(41.04231289429743 73.91839786464796) bank34512 +34513 POINT(41.220488353255675 75.00011239142648) bank34513 +34514 POINT(41.16770189263918 73.3548260402941) bank34514 +34515 POINT(41.5354760812501 73.98102047724781) bank34515 +34516 POINT(41.49553116187193 74.95041448502018) bank34516 +34517 POINT(40.52828909767547 73.32844575353943) bank34517 +34518 POINT(40.87615685233434 73.6849470799842) bank34518 +34519 POINT(40.1839555558359 73.30801910759625) bank34519 +34520 POINT(41.096931901300074 74.887822373415) bank34520 +34521 POINT(41.4903616042868 74.55361413617528) bank34521 +34522 POINT(39.996439746933866 73.73191716328795) bank34522 +34523 POINT(41.07194488015968 74.96822169541548) bank34523 +34524 POINT(41.58983830556049 74.47457330997841) bank34524 +34525 POINT(41.11686974197707 73.43872327422427) bank34525 +34526 POINT(40.124479023837516 74.99989601169919) bank34526 +34527 POINT(40.96492988059972 73.16167021568194) bank34527 +34528 POINT(41.52082756052185 74.22182539534505) bank34528 +34529 POINT(40.25869512414843 74.69742894165746) bank34529 +34530 POINT(41.26692040444911 73.66187434379862) bank34530 +34531 POINT(40.8139865995337 74.26199735991011) bank34531 +34532 POINT(40.57860580787825 73.30611626546279) bank34532 +34533 POINT(41.19896506042371 74.21448028640093) bank34533 +34534 POINT(40.962043375839194 74.3272660479437) bank34534 +34535 POINT(39.77386109935247 74.09780314216475) bank34535 +34536 POINT(39.79651204086539 74.82427861899609) bank34536 +34537 POINT(40.74381470769836 74.35673908614015) bank34537 +34538 POINT(41.327471381735506 74.10147099444761) bank34538 +34539 POINT(39.94449735575941 73.20379410446111) bank34539 +34540 POINT(39.82106701102509 73.21746616343661) bank34540 +34541 POINT(41.67227722978387 74.74112797734632) bank34541 +34542 POINT(40.76811228901492 74.56234155438473) bank34542 +34543 POINT(41.41447754238657 73.75939102823415) bank34543 +34544 POINT(39.99804852065638 74.99934383751939) bank34544 +34545 POINT(40.0099538957526 74.29722730736128) bank34545 +34546 POINT(41.30082099729327 74.67219069276062) bank34546 +34547 POINT(40.407985111727434 73.38873689157109) bank34547 +34548 POINT(40.412144807968396 73.10438664807529) bank34548 +34549 POINT(40.40754220962843 73.71492268239366) bank34549 +34550 POINT(40.81006525692629 74.81048740088765) bank34550 +34551 POINT(40.91009294053223 74.31117211119569) bank34551 +34552 POINT(39.84400495784772 74.5629961814845) bank34552 +34553 POINT(41.681141192330394 74.31267684196035) bank34553 +34554 POINT(41.6155520021983 74.43089452676053) bank34554 +34555 POINT(40.716896572729254 74.82872924851135) bank34555 +34556 POINT(40.75185702835928 73.05536167647273) bank34556 +34557 POINT(41.09826714557491 73.31384913861945) bank34557 +34558 POINT(41.58491648102892 74.25704926146449) bank34558 +34559 POINT(40.33134828244962 73.58656844661624) bank34559 +34560 POINT(39.98535775098453 74.57785163794469) bank34560 +34561 POINT(41.49542670663256 73.0142555908749) bank34561 +34562 POINT(39.9677943797507 73.99381844462444) bank34562 +34563 POINT(41.28614041061138 73.59698694417597) bank34563 +34564 POINT(40.912447634647 73.10768850163208) bank34564 +34565 POINT(41.25691791377751 73.44189240493024) bank34565 +34566 POINT(40.467072073898755 74.22896132751931) bank34566 +34567 POINT(40.41478494810348 73.01601636986724) bank34567 +34568 POINT(41.630432110450435 74.38924188345965) bank34568 +34569 POINT(41.6677573617306 73.89405391821806) bank34569 +34570 POINT(41.63895823127477 74.25663066830273) bank34570 +34571 POINT(40.25373339742234 73.67090902864423) bank34571 +34572 POINT(39.738167327834006 74.65073011318701) bank34572 +34573 POINT(41.63170552508591 73.60392254432804) bank34573 +34574 POINT(41.663325885915306 73.09924746830717) bank34574 +34575 POINT(40.530356745719 74.24953438384358) bank34575 +34576 POINT(39.82075436301269 73.34292506018696) bank34576 +34577 POINT(41.60880274715422 73.577323829495) bank34577 +34578 POINT(40.102123492964864 73.24092776159418) bank34578 +34579 POINT(40.93284763421854 74.30672776033576) bank34579 +34580 POINT(40.621561639886124 73.89871037833701) bank34580 +34581 POINT(39.8032282111102 73.99386489608727) bank34581 +34582 POINT(40.43696478533796 73.97769519475946) bank34582 +34583 POINT(41.167729314312226 73.27586042415984) bank34583 +34584 POINT(40.60696595622848 74.64827437607889) bank34584 +34585 POINT(40.66801979944552 73.53866538798353) bank34585 +34586 POINT(39.821851083275604 73.12319794470662) bank34586 +34587 POINT(40.50328605275656 73.7933238440225) bank34587 +34588 POINT(41.196409004691986 73.30953082556208) bank34588 +34589 POINT(40.93556813640478 74.89095202722785) bank34589 +34590 POINT(41.0206824131333 73.23935423590102) bank34590 +34591 POINT(40.63269377191998 73.9795624963981) bank34591 +34592 POINT(41.04741748137838 74.58898626316766) bank34592 +34593 POINT(41.077775176765996 73.0756293273723) bank34593 +34594 POINT(40.33759592992857 74.69751648151937) bank34594 +34595 POINT(40.213282912508625 74.35516827950151) bank34595 +34596 POINT(40.90300656222319 74.78996827505642) bank34596 +34597 POINT(41.55710432675819 73.97778405177412) bank34597 +34598 POINT(40.86639006132705 73.21934368304828) bank34598 +34599 POINT(40.721362362892755 74.29817192516619) bank34599 +34600 POINT(40.863706653739484 73.33159525878074) bank34600 +34601 POINT(40.99038033839143 74.82900167789664) bank34601 +34602 POINT(40.48188244419756 73.5609962861315) bank34602 +34603 POINT(40.08237949054915 74.39593397869747) bank34603 +34604 POINT(41.51690808609486 73.31663166780763) bank34604 +34605 POINT(41.42129017687192 74.3438535853771) bank34605 +34606 POINT(40.974696596766066 73.29909984266186) bank34606 +34607 POINT(40.540124209689026 74.6481350771951) bank34607 +34608 POINT(41.6549186896548 74.28304945290861) bank34608 +34609 POINT(39.85823569928059 73.17746083675847) bank34609 +34610 POINT(41.10684726714306 73.04214486784724) bank34610 +34611 POINT(40.33209163665223 73.91049743080227) bank34611 +34612 POINT(40.323146840890196 73.35780696354337) bank34612 +34613 POINT(41.296787292492795 74.36558410142311) bank34613 +34614 POINT(40.20179401089448 73.51461958038904) bank34614 +34615 POINT(40.340198930321904 73.28938089571514) bank34615 +34616 POINT(41.39932284092612 73.37340580870858) bank34616 +34617 POINT(41.557463766961625 74.08521548883269) bank34617 +34618 POINT(40.05109665539572 73.12232103752054) bank34618 +34619 POINT(40.750863102269236 74.35623971297498) bank34619 +34620 POINT(39.74653762154514 74.04791401460632) bank34620 +34621 POINT(41.70423908034854 73.2611441720134) bank34621 +34622 POINT(41.36982032640335 73.57465089205817) bank34622 +34623 POINT(39.79457301827181 73.16261754669813) bank34623 +34624 POINT(41.02678752640598 73.77737245494231) bank34624 +34625 POINT(40.47139027529184 73.15429664189904) bank34625 +34626 POINT(40.370585620326324 73.65056225046422) bank34626 +34627 POINT(40.10619928706362 73.44066856279285) bank34627 +34628 POINT(40.94824502370592 74.81874196663924) bank34628 +34629 POINT(39.87346742060445 74.80754346832893) bank34629 +34630 POINT(40.39789996672677 74.4105159486917) bank34630 +34631 POINT(40.05289353443262 74.35996094454555) bank34631 +34632 POINT(40.15362591115493 74.06492751132565) bank34632 +34633 POINT(40.85114337687825 74.73387880024308) bank34633 +34634 POINT(41.01134120047273 73.35483184780995) bank34634 +34635 POINT(40.81810410387701 73.52597747334381) bank34635 +34636 POINT(40.121576788497826 74.20059891528264) bank34636 +34637 POINT(40.2015644805953 73.02919672296986) bank34637 +34638 POINT(40.77638368195335 73.67394590062415) bank34638 +34639 POINT(41.15416038759357 74.28518047231714) bank34639 +34640 POINT(40.43716498161597 74.07740539107567) bank34640 +34641 POINT(40.46038435315007 74.31709169335471) bank34641 +34642 POINT(40.67263123745164 74.5038937301232) bank34642 +34643 POINT(40.852825010067775 73.74176418212718) bank34643 +34644 POINT(39.88624971477436 73.63845634672248) bank34644 +34645 POINT(39.718885535028264 73.44574667318359) bank34645 +34646 POINT(41.585568726344086 73.18317297134128) bank34646 +34647 POINT(40.143696527428816 74.64193204294196) bank34647 +34648 POINT(40.634894350236436 73.56752409893868) bank34648 +34649 POINT(40.48776022024127 73.67702253828017) bank34649 +34650 POINT(40.52497593385871 74.68100403208167) bank34650 +34651 POINT(40.900942921876116 73.33476211434625) bank34651 +34652 POINT(39.794122692202905 73.64792415719286) bank34652 +34653 POINT(40.25400672901789 73.27765114843108) bank34653 +34654 POINT(41.68221323332144 74.85880643027731) bank34654 +34655 POINT(40.079285161105624 73.8303559151879) bank34655 +34656 POINT(39.980005207588896 74.78152167999926) bank34656 +34657 POINT(41.09843136631544 73.0800686915122) bank34657 +34658 POINT(40.066769516007746 74.27413786414485) bank34658 +34659 POINT(39.90022051350279 73.12796169444417) bank34659 +34660 POINT(40.619869932322985 74.45465612177871) bank34660 +34661 POINT(41.306789274955385 73.9452423296081) bank34661 +34662 POINT(40.94805258386558 73.88435070268173) bank34662 +34663 POINT(41.675842379668886 74.94462834033764) bank34663 +34664 POINT(40.855900130875696 73.0489969933251) bank34664 +34665 POINT(41.169819325806536 74.07839352241888) bank34665 +34666 POINT(41.523592212315336 73.55015471580546) bank34666 +34667 POINT(39.76048614674506 73.72656752733208) bank34667 +34668 POINT(40.38982480981582 73.84404112649274) bank34668 +34669 POINT(41.181882053730625 74.66577684867418) bank34669 +34670 POINT(41.06374398743029 74.13322645143697) bank34670 +34671 POINT(40.14286749077686 73.81344256936843) bank34671 +34672 POINT(40.52382056652738 73.13369582665804) bank34672 +34673 POINT(40.18468958735762 73.34439854492864) bank34673 +34674 POINT(39.71929063783636 73.1324548417148) bank34674 +34675 POINT(40.188538124084225 74.02961761536514) bank34675 +34676 POINT(39.983563815093106 74.36834811701736) bank34676 +34677 POINT(40.806163290603344 73.02118999982609) bank34677 +34678 POINT(41.02308412639074 73.80298060252623) bank34678 +34679 POINT(40.27207018383308 74.95467298498052) bank34679 +34680 POINT(39.88154235865236 73.15803340298056) bank34680 +34681 POINT(40.090695891186634 73.92137357869795) bank34681 +34682 POINT(41.62617388709942 73.45900142979899) bank34682 +34683 POINT(41.33956270524509 74.95137407483472) bank34683 +34684 POINT(40.0104865051544 73.56879746957729) bank34684 +34685 POINT(41.188233961326574 74.77335157730376) bank34685 +34686 POINT(40.98352563898883 73.50927761929934) bank34686 +34687 POINT(40.260109416761416 73.21427132091573) bank34687 +34688 POINT(40.46765958019194 73.13284485442536) bank34688 +34689 POINT(41.59123692824798 73.3018794098879) bank34689 +34690 POINT(40.23396081673137 73.288928349967) bank34690 +34691 POINT(41.346537377302774 73.42874692571786) bank34691 +34692 POINT(41.30860777622859 74.18243984921281) bank34692 +34693 POINT(40.20705239629447 73.02216507476275) bank34693 +34694 POINT(40.74422248308666 73.31465659494334) bank34694 +34695 POINT(41.62528170723367 73.80723392158141) bank34695 +34696 POINT(40.05376435131034 73.21954042237839) bank34696 +34697 POINT(40.973887717095025 73.39239720634477) bank34697 +34698 POINT(41.33929760966454 74.47514625586498) bank34698 +34699 POINT(40.9579769317323 73.72342166401046) bank34699 +34700 POINT(41.0040187060413 73.09551245159093) bank34700 +34701 POINT(41.24771102386622 73.87891372920639) bank34701 +34702 POINT(41.57116324425624 73.19017504320028) bank34702 +34703 POINT(41.47150501813648 74.35938508622542) bank34703 +34704 POINT(41.34177961902514 74.98496798535346) bank34704 +34705 POINT(41.464764059860954 73.01297009775308) bank34705 +34706 POINT(40.14319327107173 74.47841457193591) bank34706 +34707 POINT(41.09840927967086 74.28207877993927) bank34707 +34708 POINT(40.804101654394266 73.12539591320296) bank34708 +34709 POINT(40.78826677916868 74.3328446117884) bank34709 +34710 POINT(39.829144220893326 74.12495278946996) bank34710 +34711 POINT(41.493129248187 74.76654303758211) bank34711 +34712 POINT(41.4413542538414 73.84291859111985) bank34712 +34713 POINT(41.32216239463728 74.99611017940178) bank34713 +34714 POINT(40.91353277214025 74.15896512353856) bank34714 +34715 POINT(41.48105607016646 74.09234192189442) bank34715 +34716 POINT(39.856667400623564 74.59564302046752) bank34716 +34717 POINT(40.23567726351096 73.1602355480345) bank34717 +34718 POINT(40.4518957253063 73.85977519345369) bank34718 +34719 POINT(40.68253123518569 73.66038222164642) bank34719 +34720 POINT(40.90051037877094 74.18223726796869) bank34720 +34721 POINT(40.102212182554695 74.86982523460762) bank34721 +34722 POINT(41.11093474294711 74.9471566896427) bank34722 +34723 POINT(40.014245082156826 74.75838279005897) bank34723 +34724 POINT(40.33511707906242 74.80017950412342) bank34724 +34725 POINT(40.89561733292453 73.07506084879286) bank34725 +34726 POINT(40.31808211603333 73.63617651732373) bank34726 +34727 POINT(41.505352009122575 73.5129051726919) bank34727 +34728 POINT(41.63651481542678 74.8382547235567) bank34728 +34729 POINT(41.40587810218216 74.29567921727347) bank34729 +34730 POINT(41.03289153556363 73.77470541051747) bank34730 +34731 POINT(40.358101544286264 73.69034225042714) bank34731 +34732 POINT(41.2188829272627 74.74243744803229) bank34732 +34733 POINT(40.83276439101459 74.4494847368889) bank34733 +34734 POINT(40.742436604736874 73.95254941449862) bank34734 +34735 POINT(40.05254424396267 74.64984543546157) bank34735 +34736 POINT(41.05074794950322 73.51426398477469) bank34736 +34737 POINT(41.13206860347415 73.11014378264545) bank34737 +34738 POINT(40.50378467869288 73.38163206982746) bank34738 +34739 POINT(41.51460994445616 74.77479809270268) bank34739 +34740 POINT(40.45542316836469 73.62990566069851) bank34740 +34741 POINT(39.806143030051686 73.65466418018957) bank34741 +34742 POINT(41.050082940432645 74.81513403639885) bank34742 +34743 POINT(41.6521457452109 74.16595455894499) bank34743 +34744 POINT(40.60393202372225 73.5939044926731) bank34744 +34745 POINT(41.391549279069764 73.7467841992487) bank34745 +34746 POINT(41.6594831691138 74.07533587962558) bank34746 +34747 POINT(39.90192871640479 73.85838079150443) bank34747 +34748 POINT(41.13415563517507 73.63079653821637) bank34748 +34749 POINT(40.416132083025495 74.3252860648461) bank34749 +34750 POINT(41.038541246999706 73.06302597741622) bank34750 +34751 POINT(41.68591872424876 74.04221957285678) bank34751 +34752 POINT(41.05171825724279 74.29602235795764) bank34752 +34753 POINT(40.267829099643016 74.72867271998906) bank34753 +34754 POINT(40.642745449353946 74.52463442533715) bank34754 +34755 POINT(41.5429817397517 73.59898573709027) bank34755 +34756 POINT(41.57013348650296 74.5176573744113) bank34756 +34757 POINT(40.10574244923804 74.57241199463914) bank34757 +34758 POINT(40.35643314552016 74.79174327960357) bank34758 +34759 POINT(41.00756886458905 74.11989027589247) bank34759 +34760 POINT(40.53432898867229 73.99925454618166) bank34760 +34761 POINT(41.523777663598125 74.67008111338433) bank34761 +34762 POINT(40.88587660531475 74.86421983263935) bank34762 +34763 POINT(39.99405622162775 73.63354590624783) bank34763 +34764 POINT(40.45601986307113 74.27930216772283) bank34764 +34765 POINT(40.667349147544115 73.70516845377546) bank34765 +34766 POINT(40.05064193990001 74.38258764428836) bank34766 +34767 POINT(41.50046398050755 73.87406127662874) bank34767 +34768 POINT(41.45447810100495 74.47528008618475) bank34768 +34769 POINT(41.659220895834764 73.72619425886109) bank34769 +34770 POINT(40.63071107144103 74.96233816830234) bank34770 +34771 POINT(40.10119257863907 73.85946947111486) bank34771 +34772 POINT(41.34389400521851 74.50262076223626) bank34772 +34773 POINT(41.08808883049369 73.94732046491339) bank34773 +34774 POINT(40.373053219447236 74.14001029597783) bank34774 +34775 POINT(39.90163049088639 73.72818943381922) bank34775 +34776 POINT(41.16273835641827 74.65291448313526) bank34776 +34777 POINT(41.13402267858469 74.20874857813905) bank34777 +34778 POINT(40.78105603622235 73.78974480024657) bank34778 +34779 POINT(39.932891938398356 74.9608090810675) bank34779 +34780 POINT(40.51504299867258 74.03072761111505) bank34780 +34781 POINT(39.866663657422556 73.65615417723143) bank34781 +34782 POINT(41.290476561500256 74.90970787341557) bank34782 +34783 POINT(40.934875618678305 73.24281784673053) bank34783 +34784 POINT(39.719227014580866 74.20984561222606) bank34784 +34785 POINT(40.08648040154841 73.07699661647305) bank34785 +34786 POINT(41.63244190465928 74.29694339906135) bank34786 +34787 POINT(41.3937567244076 73.52967247660841) bank34787 +34788 POINT(41.63128682579808 74.23641169417019) bank34788 +34789 POINT(39.84629780585527 74.69285328608301) bank34789 +34790 POINT(39.724469185452676 73.94120012244514) bank34790 +34791 POINT(40.838834027894244 73.327991522853) bank34791 +34792 POINT(40.109685946158635 73.04266334813993) bank34792 +34793 POINT(40.789627299103095 74.0051238885676) bank34793 +34794 POINT(40.787816010619785 74.33022519606499) bank34794 +34795 POINT(40.06381168694741 74.20517398603764) bank34795 +34796 POINT(40.354152967036555 74.04639490066396) bank34796 +34797 POINT(41.54000134941028 73.84853449731676) bank34797 +34798 POINT(41.53253730321668 74.93841424529353) bank34798 +34799 POINT(40.75630199428375 73.09840544154174) bank34799 +34800 POINT(39.81729032168563 73.77074985293345) bank34800 +34801 POINT(40.499530051818404 74.63214115867936) bank34801 +34802 POINT(39.83129438717754 73.93943035694613) bank34802 +34803 POINT(40.33974514429491 74.4536832689544) bank34803 +34804 POINT(40.23131685070137 73.93430281011167) bank34804 +34805 POINT(40.799524937125966 73.94657139499199) bank34805 +34806 POINT(41.46761075660616 73.08658658896924) bank34806 +34807 POINT(39.98841993075667 74.18942359631276) bank34807 +34808 POINT(41.64689458581127 74.16584426828858) bank34808 +34809 POINT(41.42300601757574 74.68936545440388) bank34809 +34810 POINT(40.15315907865748 73.20239246031593) bank34810 +34811 POINT(39.911615495494836 74.75287418405411) bank34811 +34812 POINT(41.28429215011361 73.62913514207787) bank34812 +34813 POINT(40.55705960836898 74.7989209620739) bank34813 +34814 POINT(39.8117181486422 74.74829043040853) bank34814 +34815 POINT(41.434426455067666 73.40169699138795) bank34815 +34816 POINT(40.56828199275418 74.23923267451384) bank34816 +34817 POINT(40.50351908381778 74.51941581651498) bank34817 +34818 POINT(40.85946614492268 74.63560914992468) bank34818 +34819 POINT(40.86790452685701 74.99916897961569) bank34819 +34820 POINT(41.20265195529903 73.88559820344427) bank34820 +34821 POINT(40.134217819331155 74.30769765007759) bank34821 +34822 POINT(40.033753539146495 73.29469152712117) bank34822 +34823 POINT(41.49887259041663 73.55026599305344) bank34823 +34824 POINT(40.17588673578912 74.11926403688874) bank34824 +34825 POINT(41.62521251338705 73.51214135067997) bank34825 +34826 POINT(40.13998317298612 74.63474843523468) bank34826 +34827 POINT(41.507558888525566 73.3878224428371) bank34827 +34828 POINT(40.10356442843802 73.22374267028006) bank34828 +34829 POINT(40.02687269879783 74.27201333188876) bank34829 +34830 POINT(39.76188612369602 73.66895017603197) bank34830 +34831 POINT(41.25596697968607 73.36872256399043) bank34831 +34832 POINT(41.416627150960856 73.86404458611759) bank34832 +34833 POINT(40.66176712007042 74.50068057548488) bank34833 +34834 POINT(41.07469597834555 73.35743506279792) bank34834 +34835 POINT(40.379699326381065 74.49115916258756) bank34835 +34836 POINT(40.90145362200522 74.92902039271212) bank34836 +34837 POINT(41.45019834945179 74.68989197225308) bank34837 +34838 POINT(40.34450359862643 74.66473537107508) bank34838 +34839 POINT(41.233613384956115 74.50138748952737) bank34839 +34840 POINT(40.6921779680273 74.91473591975745) bank34840 +34841 POINT(40.53373760389888 73.09713229420994) bank34841 +34842 POINT(40.58027129321362 73.93518583876175) bank34842 +34843 POINT(40.68720940859834 73.30931778599873) bank34843 +34844 POINT(40.51729634286331 73.1558732650139) bank34844 +34845 POINT(40.26607931584932 74.39319266628763) bank34845 +34846 POINT(40.71396965337941 74.05030496258996) bank34846 +34847 POINT(40.98777419864371 74.36729534001172) bank34847 +34848 POINT(40.521709031887745 74.05810151136767) bank34848 +34849 POINT(41.22994404551565 74.87339458455664) bank34849 +34850 POINT(41.43528814566465 73.85708622802252) bank34850 +34851 POINT(39.79263576381404 74.79955361473824) bank34851 +34852 POINT(40.33034445839484 74.13376883443199) bank34852 +34853 POINT(39.97267180391872 73.08643105258915) bank34853 +34854 POINT(40.54838289355351 73.4142492139489) bank34854 +34855 POINT(40.01132908577864 73.30893939435528) bank34855 +34856 POINT(40.597859953926715 74.62050047918616) bank34856 +34857 POINT(40.10581153249977 74.05774832548788) bank34857 +34858 POINT(40.455050570726456 74.78762875304689) bank34858 +34859 POINT(40.70643627013603 73.42776076034478) bank34859 +34860 POINT(40.562268998114774 73.15254032946295) bank34860 +34861 POINT(41.544396867526515 74.82608849968491) bank34861 +34862 POINT(39.95626987103946 73.39661357497715) bank34862 +34863 POINT(40.94140660863989 73.22629405300758) bank34863 +34864 POINT(41.210806886107676 74.93799966399601) bank34864 +34865 POINT(41.17782077252384 73.2000183925718) bank34865 +34866 POINT(41.09564334877565 74.5084138673008) bank34866 +34867 POINT(40.216797291860395 74.42903219323044) bank34867 +34868 POINT(40.6045504545864 73.37206471483147) bank34868 +34869 POINT(40.952629691622995 73.71206533734085) bank34869 +34870 POINT(39.87636669951946 74.2401426741702) bank34870 +34871 POINT(39.8980283806873 73.91970474532874) bank34871 +34872 POINT(40.450152471197335 74.05120117198594) bank34872 +34873 POINT(40.81726957677366 74.44806240767093) bank34873 +34874 POINT(40.22565891752788 73.26809036090836) bank34874 +34875 POINT(39.97867331139396 73.48208930137537) bank34875 +34876 POINT(41.318038243965134 74.80395630389971) bank34876 +34877 POINT(41.370243210197636 74.37009135300505) bank34877 +34878 POINT(39.92113953477177 73.38550156340725) bank34878 +34879 POINT(40.0013611357694 74.02615421183967) bank34879 +34880 POINT(39.75618540797287 74.9784577337825) bank34880 +34881 POINT(40.13758974976485 74.12399940128356) bank34881 +34882 POINT(40.739919902343345 74.302738443452) bank34882 +34883 POINT(39.929701012705685 74.44150042980262) bank34883 +34884 POINT(41.236067971253995 74.99490125618209) bank34884 +34885 POINT(39.89738009079339 74.76839977022988) bank34885 +34886 POINT(39.733743607766456 74.01744487025647) bank34886 +34887 POINT(41.66163321817736 74.84967496857212) bank34887 +34888 POINT(41.27628935576203 74.78367171228517) bank34888 +34889 POINT(41.11744323536576 73.09319511401482) bank34889 +34890 POINT(39.9040834762519 73.95489296470461) bank34890 +34891 POINT(40.03906503237177 74.21118009398234) bank34891 +34892 POINT(40.77683633813572 73.4364415789021) bank34892 +34893 POINT(40.94230585411851 73.50073369996309) bank34893 +34894 POINT(40.05472992359735 74.86591348313667) bank34894 +34895 POINT(40.94118856188175 74.8469605869937) bank34895 +34896 POINT(40.332628164836095 74.15465288520603) bank34896 +34897 POINT(41.080589776100716 75.00526499498393) bank34897 +34898 POINT(40.99256463314446 73.62323976438364) bank34898 +34899 POINT(41.647968793169234 73.02676303167098) bank34899 +34900 POINT(41.209683692676876 74.1014279157898) bank34900 +34901 POINT(40.58134383854243 74.34950960919883) bank34901 +34902 POINT(40.37003901235876 74.12316654980135) bank34902 +34903 POINT(40.00738124895226 74.21171000534527) bank34903 +34904 POINT(40.376434660314004 73.97713531209598) bank34904 +34905 POINT(41.397469984979345 74.78549773184821) bank34905 +34906 POINT(40.94734173075807 73.01980745638988) bank34906 +34907 POINT(41.199625241429 73.88116107565119) bank34907 +34908 POINT(39.93230590322135 74.54549884433085) bank34908 +34909 POINT(41.54602351291333 74.8322032681833) bank34909 +34910 POINT(40.6795779053909 74.77125128163816) bank34910 +34911 POINT(40.396205546187474 73.18464069184039) bank34911 +34912 POINT(40.96716538194499 74.3250397442951) bank34912 +34913 POINT(39.97701820101135 74.28023264657608) bank34913 +34914 POINT(40.134753315371086 73.39008013460253) bank34914 +34915 POINT(40.82842296177654 73.0857686800592) bank34915 +34916 POINT(40.94987360085315 74.37352011192212) bank34916 +34917 POINT(40.69190289662764 73.57225112577665) bank34917 +34918 POINT(39.91352167199225 74.1956282866647) bank34918 +34919 POINT(41.56244370373476 73.15353854429056) bank34919 +34920 POINT(40.63349072083844 74.52598491869364) bank34920 +34921 POINT(41.50157861187605 73.20902677338364) bank34921 +34922 POINT(41.20244259829348 73.770307063013) bank34922 +34923 POINT(40.55351449171445 74.469444802132) bank34923 +34924 POINT(39.797376226850254 73.96588764014858) bank34924 +34925 POINT(41.156508365395794 73.19528621564423) bank34925 +34926 POINT(40.74291651689141 74.95286012779383) bank34926 +34927 POINT(41.04461094760199 74.20332229509839) bank34927 +34928 POINT(40.48161960501206 74.6261366590396) bank34928 +34929 POINT(40.795463302512495 74.5597137965464) bank34929 +34930 POINT(40.28565548347855 74.38391264907801) bank34930 +34931 POINT(41.061299816683174 74.74791488606104) bank34931 +34932 POINT(40.57640761009466 74.0896921241616) bank34932 +34933 POINT(40.81800530123801 74.28963032231715) bank34933 +34934 POINT(41.205380709842395 73.4809220232182) bank34934 +34935 POINT(41.49290817995153 74.34308303389518) bank34935 +34936 POINT(40.38495896029426 74.52072482504158) bank34936 +34937 POINT(41.285758600473 73.29092380209991) bank34937 +34938 POINT(39.980815895829096 74.63196257423522) bank34938 +34939 POINT(41.42900985443274 74.43208873580583) bank34939 +34940 POINT(40.02355368459494 74.90128722502548) bank34940 +34941 POINT(40.09778534935693 73.98039697829302) bank34941 +34942 POINT(40.96669923753328 73.42686904475573) bank34942 +34943 POINT(40.98717622597623 73.60380159265988) bank34943 +34944 POINT(40.04529468669845 73.68980313816995) bank34944 +34945 POINT(40.57337799888693 73.8321874909361) bank34945 +34946 POINT(40.54485748117501 74.55397520496396) bank34946 +34947 POINT(40.61363700715991 73.32557360109467) bank34947 +34948 POINT(41.41544604206289 74.53760700167156) bank34948 +34949 POINT(39.77900330543495 73.55946917210277) bank34949 +34950 POINT(40.97942045513867 73.34269216161175) bank34950 +34951 POINT(40.01055027920664 74.11094194759372) bank34951 +34952 POINT(40.994981256969616 74.41252878432455) bank34952 +34953 POINT(41.70686350134527 73.9769838825527) bank34953 +34954 POINT(41.30317806061715 73.53549921001004) bank34954 +34955 POINT(41.00736863168893 73.2043553228335) bank34955 +34956 POINT(40.02444600973074 73.38321954769052) bank34956 +34957 POINT(41.37616902009053 73.37270244293268) bank34957 +34958 POINT(39.980771834519025 73.20869451760623) bank34958 +34959 POINT(40.95779828145377 74.49727975149885) bank34959 +34960 POINT(40.56500980249512 74.80301166053368) bank34960 +34961 POINT(41.491254589886815 74.24738546099012) bank34961 +34962 POINT(41.105801944110596 73.8189401721121) bank34962 +34963 POINT(40.571835333221514 73.61898117783365) bank34963 +34964 POINT(40.95776344767705 73.65964995244832) bank34964 +34965 POINT(41.63020897548259 74.67627989178781) bank34965 +34966 POINT(41.405862035369175 73.55059594677526) bank34966 +34967 POINT(40.05730838137504 74.79658685946322) bank34967 +34968 POINT(39.9120357738489 74.24311542632854) bank34968 +34969 POINT(41.4282178011778 73.13461704262485) bank34969 +34970 POINT(41.12047419340881 74.54937577666811) bank34970 +34971 POINT(41.41323086136177 73.03750735554561) bank34971 +34972 POINT(39.814734848565955 74.30380985635121) bank34972 +34973 POINT(39.79854990473471 74.7088793988488) bank34973 +34974 POINT(41.542996348987195 73.92453526122425) bank34974 +34975 POINT(40.898022283440106 73.09032315414683) bank34975 +34976 POINT(41.5638260612322 73.31270708856178) bank34976 +34977 POINT(40.96285320044721 74.20328021086083) bank34977 +34978 POINT(40.2007487999809 74.61553441306495) bank34978 +34979 POINT(41.46972314754339 73.50928316120822) bank34979 +34980 POINT(40.08623934159312 74.97313719803577) bank34980 +34981 POINT(40.228704578145255 74.2076419067274) bank34981 +34982 POINT(40.17371813602403 73.36479843709765) bank34982 +34983 POINT(39.71871322648786 74.50659617031589) bank34983 +34984 POINT(40.07010171182263 74.94400562125246) bank34984 +34985 POINT(39.97661336943025 74.99934013533898) bank34985 +34986 POINT(39.84692201399724 73.18819305893511) bank34986 +34987 POINT(41.567078450854396 73.90029047473757) bank34987 +34988 POINT(41.00529025157625 74.93579538854206) bank34988 +34989 POINT(40.502313821470416 73.06959157183258) bank34989 +34990 POINT(39.75639735841913 73.91341454316809) bank34990 +34991 POINT(40.216606344077576 74.94123103203229) bank34991 +34992 POINT(40.08226604962466 74.58251860114993) bank34992 +34993 POINT(39.93147245028977 74.4722193764934) bank34993 +34994 POINT(41.63548647717614 74.23560185710527) bank34994 +34995 POINT(40.18398316326848 73.33657156493142) bank34995 +34996 POINT(40.040134686893275 73.66819765660676) bank34996 +34997 POINT(40.63473491801904 74.94724415143571) bank34997 +34998 POINT(39.797720121136706 74.05206478000909) bank34998 +34999 POINT(40.380715422645814 74.23687826056612) bank34999 +35000 POINT(40.49681808291771 73.37234375630656) bank35000 +35001 POINT(41.60723519558592 73.30242521869727) bank35001 +35002 POINT(40.44338119375986 74.51233538722578) bank35002 +35003 POINT(41.14662777764097 73.89375580283276) bank35003 +35004 POINT(41.092934659194135 73.01114934010322) bank35004 +35005 POINT(40.92786357650537 73.2187541266647) bank35005 +35006 POINT(40.90611498003809 73.53362052679543) bank35006 +35007 POINT(40.79137247522583 74.10070399423142) bank35007 +35008 POINT(41.26490068817612 74.13345782887802) bank35008 +35009 POINT(41.66852913958088 73.42147590851977) bank35009 +35010 POINT(40.98985005947146 73.05805426691062) bank35010 +35011 POINT(40.631688032672606 74.25422203203533) bank35011 +35012 POINT(40.013204376104845 74.50957454628363) bank35012 +35013 POINT(40.39321973481326 74.93521661048796) bank35013 +35014 POINT(39.94478138611909 73.20767314463052) bank35014 +35015 POINT(39.84742857332751 73.02069931558106) bank35015 +35016 POINT(40.88159818651428 74.32291541554551) bank35016 +35017 POINT(41.568899052685516 74.27979086954723) bank35017 +35018 POINT(41.17242054143904 74.34338472008949) bank35018 +35019 POINT(39.87545905580887 73.36167382500832) bank35019 +35020 POINT(40.344956133880515 74.94262728958016) bank35020 +35021 POINT(39.79387332066857 73.7139696798942) bank35021 +35022 POINT(39.81336315311777 74.56936013128647) bank35022 +35023 POINT(40.32604740754085 74.86617105764351) bank35023 +35024 POINT(41.331783809687195 74.76959170592427) bank35024 +35025 POINT(40.14676262884804 74.90865360968682) bank35025 +35026 POINT(41.112406893554905 74.37408387867923) bank35026 +35027 POINT(40.004683465007496 73.78061342977179) bank35027 +35028 POINT(39.95663082405765 74.26277589109637) bank35028 +35029 POINT(39.764982790760925 74.07714658086637) bank35029 +35030 POINT(40.03067795740508 73.3153841004634) bank35030 +35031 POINT(41.25612420836822 74.80893798350795) bank35031 +35032 POINT(40.62357907814359 73.7680507142813) bank35032 +35033 POINT(39.762228910447554 74.36610202062427) bank35033 +35034 POINT(40.801826428400524 74.75083657010876) bank35034 +35035 POINT(41.6363744097544 73.54555814018443) bank35035 +35036 POINT(40.92619478042395 73.53644647356735) bank35036 +35037 POINT(40.38967787639288 73.19085333471891) bank35037 +35038 POINT(40.28862489317287 73.26983545129177) bank35038 +35039 POINT(41.18934379038467 73.63103652222208) bank35039 +35040 POINT(41.607252753684165 74.33404753553287) bank35040 +35041 POINT(40.810077227803184 74.97711219416733) bank35041 +35042 POINT(40.436339617007334 73.47753568404296) bank35042 +35043 POINT(41.250272902905564 73.38338912459815) bank35043 +35044 POINT(40.3138724153878 74.61339259069318) bank35044 +35045 POINT(40.57940433818672 73.05477995552788) bank35045 +35046 POINT(41.24004729666279 73.55050512487611) bank35046 +35047 POINT(40.140823577370355 74.61320972208149) bank35047 +35048 POINT(40.81961605813741 74.2042091870216) bank35048 +35049 POINT(40.30947985107993 74.8549946005715) bank35049 +35050 POINT(40.94954431600349 74.02625801387005) bank35050 +35051 POINT(40.088068892521264 73.4618187130237) bank35051 +35052 POINT(40.21836858354127 73.2891985718468) bank35052 +35053 POINT(41.14585476927829 73.30150271027048) bank35053 +35054 POINT(41.049286460246734 74.81215278780307) bank35054 +35055 POINT(41.48728998235776 74.85044847281381) bank35055 +35056 POINT(40.69076052352014 73.92071832669198) bank35056 +35057 POINT(40.51620123679531 73.96961847467294) bank35057 +35058 POINT(41.69019566595822 73.45736260395918) bank35058 +35059 POINT(39.873321337883915 73.16245510966775) bank35059 +35060 POINT(40.189894510169964 73.17798953029622) bank35060 +35061 POINT(39.97198831173497 73.68954265955485) bank35061 +35062 POINT(40.360273479960576 73.88864737652031) bank35062 +35063 POINT(41.13439354601201 73.06540257642116) bank35063 +35064 POINT(41.39149183668922 73.08229778227442) bank35064 +35065 POINT(40.34457971391945 73.19405554728942) bank35065 +35066 POINT(40.40685205726299 74.87816171009997) bank35066 +35067 POINT(41.4462047177426 74.76170164250483) bank35067 +35068 POINT(41.24125759432991 73.03252643743538) bank35068 +35069 POINT(39.948561594033 73.98157805326798) bank35069 +35070 POINT(40.54904896128482 74.38950368211063) bank35070 +35071 POINT(41.476640706815395 73.00769750821387) bank35071 +35072 POINT(40.00947743296349 73.21349414443773) bank35072 +35073 POINT(40.07321519512671 73.42983397819374) bank35073 +35074 POINT(40.04703090981114 73.22118958799456) bank35074 +35075 POINT(41.55677836373562 73.5811320695759) bank35075 +35076 POINT(40.58982384510421 73.69767905631588) bank35076 +35077 POINT(40.79047860890616 74.95072153234982) bank35077 +35078 POINT(40.90333535782101 74.3086190762794) bank35078 +35079 POINT(40.66229315858953 73.27488972600102) bank35079 +35080 POINT(39.839145846302124 74.9169488150405) bank35080 +35081 POINT(40.3949304362201 73.30108377733127) bank35081 +35082 POINT(41.43061424863008 73.21629733519691) bank35082 +35083 POINT(39.85160507556315 74.83302727000041) bank35083 +35084 POINT(39.81578880198636 73.95113163633961) bank35084 +35085 POINT(39.88147779323252 73.90625869506704) bank35085 +35086 POINT(41.16887686699976 74.88730228872191) bank35086 +35087 POINT(39.92934943029428 73.6982083981915) bank35087 +35088 POINT(40.824575680168365 74.003919126367) bank35088 +35089 POINT(41.015054693796664 73.73783985408643) bank35089 +35090 POINT(40.61306943577697 74.37581230295514) bank35090 +35091 POINT(39.93533483857103 74.26245699735165) bank35091 +35092 POINT(41.6936724400619 74.58021636600397) bank35092 +35093 POINT(41.03477345816649 73.27323796734179) bank35093 +35094 POINT(40.95354166084681 74.27555243666173) bank35094 +35095 POINT(40.77931981297126 73.93030548794401) bank35095 +35096 POINT(41.15139237333117 74.70717266616434) bank35096 +35097 POINT(41.27951334064831 73.4022605379904) bank35097 +35098 POINT(40.10525924663282 73.90744450406626) bank35098 +35099 POINT(40.98393255793913 74.27218066062105) bank35099 +35100 POINT(39.996766436694415 74.02179985428785) bank35100 +35101 POINT(41.4544929970551 74.34718753851921) bank35101 +35102 POINT(41.50475320601034 74.17542659125183) bank35102 +35103 POINT(39.807077190044694 73.89395730625483) bank35103 +35104 POINT(41.065242046478396 74.53005556311265) bank35104 +35105 POINT(39.968113748297604 74.19747062592371) bank35105 +35106 POINT(41.13858503162268 74.53288336387119) bank35106 +35107 POINT(40.24166272775783 74.37068969740015) bank35107 +35108 POINT(41.546219467186205 74.68786898982509) bank35108 +35109 POINT(41.46721763898279 74.4586573492404) bank35109 +35110 POINT(41.27193563924875 74.76031133991418) bank35110 +35111 POINT(39.88668559726404 73.20130087515844) bank35111 +35112 POINT(41.39999877744305 74.24576236450748) bank35112 +35113 POINT(40.75471991847591 73.70823560171272) bank35113 +35114 POINT(40.42058365544764 74.79677810418602) bank35114 +35115 POINT(40.48061197958945 74.89944626270653) bank35115 +35116 POINT(40.07924257309603 73.40653390625133) bank35116 +35117 POINT(40.21341310817329 74.36386334817217) bank35117 +35118 POINT(40.67462557250057 73.46948558579156) bank35118 +35119 POINT(39.971871463895205 73.25780043583444) bank35119 +35120 POINT(40.497459388943376 74.39322720178268) bank35120 +35121 POINT(41.60974723433122 73.13040920348702) bank35121 +35122 POINT(40.188751098341115 74.35324129013284) bank35122 +35123 POINT(40.80454655832546 73.31734621811985) bank35123 +35124 POINT(40.45782764413899 74.96067415803721) bank35124 +35125 POINT(39.782687751539875 73.97518456259805) bank35125 +35126 POINT(40.19757266139696 74.1727715378782) bank35126 +35127 POINT(40.093948646527224 73.18758216160678) bank35127 +35128 POINT(41.63107793254277 74.24609480736953) bank35128 +35129 POINT(40.57088842704537 74.73656924047006) bank35129 +35130 POINT(41.088271523233296 74.88720694608173) bank35130 +35131 POINT(40.327352406342975 74.54266752957813) bank35131 +35132 POINT(40.51012956810971 74.2688698731815) bank35132 +35133 POINT(40.199927161523206 73.18369542090353) bank35133 +35134 POINT(41.002732839912824 74.7042115085895) bank35134 +35135 POINT(40.50694827309407 74.6884022580647) bank35135 +35136 POINT(41.19888285867149 74.56222618637179) bank35136 +35137 POINT(40.79044628400661 73.28998281218753) bank35137 +35138 POINT(40.379495469968596 73.84787930314643) bank35138 +35139 POINT(39.74092763036438 74.55173048527449) bank35139 +35140 POINT(40.11531239672471 73.39017940696213) bank35140 +35141 POINT(41.384363628924845 74.17319535250982) bank35141 +35142 POINT(40.42181714170717 74.21368020311559) bank35142 +35143 POINT(41.355147048095255 73.92034121033596) bank35143 +35144 POINT(39.869316384863005 73.39441138029646) bank35144 +35145 POINT(40.415985141593346 73.4328067565733) bank35145 +35146 POINT(40.262256060868516 74.3294908475158) bank35146 +35147 POINT(41.62073392470925 73.6551535501776) bank35147 +35148 POINT(40.38337320223925 74.32264891624493) bank35148 +35149 POINT(40.87358827085999 74.06462066945272) bank35149 +35150 POINT(40.197988612966576 73.36991129562418) bank35150 +35151 POINT(40.82120971138393 73.1387172040268) bank35151 +35152 POINT(39.898967763206265 74.80365995958867) bank35152 +35153 POINT(40.72680357552132 74.02856609923026) bank35153 +35154 POINT(40.18466946564806 73.23594070326499) bank35154 +35155 POINT(41.39578837329457 74.10387390948428) bank35155 +35156 POINT(40.37335415178188 74.56925705348914) bank35156 +35157 POINT(41.610963607287786 74.3355775846543) bank35157 +35158 POINT(40.5737414429606 74.51493610131573) bank35158 +35159 POINT(41.58349614355445 73.5109878636008) bank35159 +35160 POINT(40.684504360440435 74.11995705073446) bank35160 +35161 POINT(41.59032367999308 73.30544836218095) bank35161 +35162 POINT(40.680573767568376 73.19108944920808) bank35162 +35163 POINT(40.35458983593886 74.76650827894345) bank35163 +35164 POINT(41.043967422557245 73.28019571336723) bank35164 +35165 POINT(40.42039457573675 73.20672021145421) bank35165 +35166 POINT(41.315911309802495 74.42484654569576) bank35166 +35167 POINT(41.704644701314606 73.46608085057099) bank35167 +35168 POINT(40.87679947948784 73.32970151446804) bank35168 +35169 POINT(41.539507311857776 74.80840346447776) bank35169 +35170 POINT(41.10356386837062 73.83695731713827) bank35170 +35171 POINT(40.92499311909314 73.76541365629635) bank35171 +35172 POINT(39.780534345500755 74.06025939917612) bank35172 +35173 POINT(40.064623050620774 73.1529542580358) bank35173 +35174 POINT(40.87663598419865 74.04703907747411) bank35174 +35175 POINT(40.3193135855876 74.39054084497229) bank35175 +35176 POINT(41.339191738462766 74.49478937110337) bank35176 +35177 POINT(40.12664555552574 74.19278356794698) bank35177 +35178 POINT(39.76352004308731 74.06234806822175) bank35178 +35179 POINT(39.778313947623076 73.92449587671793) bank35179 +35180 POINT(40.0360363472269 74.42757969914143) bank35180 +35181 POINT(39.93779624546902 73.613052204908) bank35181 +35182 POINT(40.42930435606546 73.21622086164297) bank35182 +35183 POINT(40.19138216699694 74.590111388269) bank35183 +35184 POINT(41.04331069715631 74.9840824002052) bank35184 +35185 POINT(40.53183731174198 73.46220026846261) bank35185 +35186 POINT(39.85451513815641 74.48507127481076) bank35186 +35187 POINT(40.220421569767915 74.97867894646761) bank35187 +35188 POINT(40.64664084396325 74.31316512055525) bank35188 +35189 POINT(41.67205583748707 73.44001112482796) bank35189 +35190 POINT(39.800347890335715 73.81751825040561) bank35190 +35191 POINT(40.22028583561969 74.20747028625787) bank35191 +35192 POINT(41.370198615192585 73.78744229719332) bank35192 +35193 POINT(41.6510463859566 74.95303177766003) bank35193 +35194 POINT(39.770773701881794 73.55369490212314) bank35194 +35195 POINT(40.15148458604398 73.51672613328489) bank35195 +35196 POINT(41.49047855887695 73.69903155104994) bank35196 +35197 POINT(39.87257353778574 73.58052077011412) bank35197 +35198 POINT(40.47372762983298 73.31272393819738) bank35198 +35199 POINT(40.27434779111827 74.78403579355708) bank35199 +35200 POINT(40.05070613370503 74.15312052956149) bank35200 +35201 POINT(41.090819643635 74.15618654686546) bank35201 +35202 POINT(41.506315206326725 74.75393530815019) bank35202 +35203 POINT(41.07934830198107 73.34219071686856) bank35203 +35204 POINT(40.835165466553285 73.42712232095649) bank35204 +35205 POINT(40.36361745376284 74.56682839030606) bank35205 +35206 POINT(39.77570375666547 73.82028923054874) bank35206 +35207 POINT(40.02042454955709 73.57695670182486) bank35207 +35208 POINT(40.943931825054236 73.16159281894545) bank35208 +35209 POINT(41.43476871446077 74.02662494263359) bank35209 +35210 POINT(40.23623200571422 73.89200950534895) bank35210 +35211 POINT(40.6826223246239 73.13913153044243) bank35211 +35212 POINT(40.020445046376686 74.24022775147776) bank35212 +35213 POINT(39.825076351187924 73.51378273319258) bank35213 +35214 POINT(41.47958433039271 74.6120754254513) bank35214 +35215 POINT(41.566473943099936 73.05082910203187) bank35215 +35216 POINT(41.01156005689522 73.62484685189122) bank35216 +35217 POINT(41.254695386415214 73.20987777089994) bank35217 +35218 POINT(41.07628017191197 74.19485641028619) bank35218 +35219 POINT(40.056135348448834 73.25589380296475) bank35219 +35220 POINT(40.901343110497095 74.13996853678923) bank35220 +35221 POINT(39.76877082431559 74.1943987934934) bank35221 +35222 POINT(40.123479788668185 74.479806582913) bank35222 +35223 POINT(40.51580022702582 74.34145877984591) bank35223 +35224 POINT(40.456200469831856 74.70837549356072) bank35224 +35225 POINT(40.95250691979547 73.20984850185665) bank35225 +35226 POINT(40.222131896071986 73.08413225125538) bank35226 +35227 POINT(40.42963423066014 73.15455663288206) bank35227 +35228 POINT(40.49124220651751 74.98850048863301) bank35228 +35229 POINT(40.896907544784696 73.84205326684659) bank35229 +35230 POINT(41.40805039510032 74.88296510403796) bank35230 +35231 POINT(40.48029145768583 74.16567376641157) bank35231 +35232 POINT(40.97080207929706 74.63384051545908) bank35232 +35233 POINT(40.35981143264715 74.63553739303671) bank35233 +35234 POINT(41.51689394694654 74.97799557222076) bank35234 +35235 POINT(40.121274548102704 74.54083781212947) bank35235 +35236 POINT(40.22118455278499 74.2703588825291) bank35236 +35237 POINT(41.57834630170955 73.74697350112335) bank35237 +35238 POINT(40.860771421587714 73.40539010205966) bank35238 +35239 POINT(39.815432746981216 73.4419028383658) bank35239 +35240 POINT(41.28708364050319 73.20799764109445) bank35240 +35241 POINT(40.96123550135473 73.64696347939437) bank35241 +35242 POINT(40.10884248155392 73.41289779948576) bank35242 +35243 POINT(40.72918041970548 74.37746091339343) bank35243 +35244 POINT(40.89153344771719 73.57088696578442) bank35244 +35245 POINT(41.24229973853709 74.2985996228374) bank35245 +35246 POINT(40.871574115889594 73.28006568056723) bank35246 +35247 POINT(41.587326136444155 73.35674349353009) bank35247 +35248 POINT(41.636088240129325 74.03534011807857) bank35248 +35249 POINT(41.67664183466829 74.0706328562702) bank35249 +35250 POINT(41.613743461269145 74.38053149763799) bank35250 +35251 POINT(41.12536667895998 73.03925276683646) bank35251 +35252 POINT(41.42761856459571 74.93397459980764) bank35252 +35253 POINT(40.73264859756343 73.62632870546197) bank35253 +35254 POINT(40.40750268015008 74.83478524978399) bank35254 +35255 POINT(39.72980213934163 74.08676884574272) bank35255 +35256 POINT(41.363591623943805 73.96254548322266) bank35256 +35257 POINT(40.59363097680647 74.9565050460471) bank35257 +35258 POINT(40.03609533591504 74.47240636955546) bank35258 +35259 POINT(40.135384424609484 74.92163308550414) bank35259 +35260 POINT(41.55378086684259 74.01443781630726) bank35260 +35261 POINT(41.43376407717558 73.5020838595891) bank35261 +35262 POINT(41.58180351530729 73.36005730652857) bank35262 +35263 POINT(40.174520428208574 74.61256447427779) bank35263 +35264 POINT(40.49951977188191 73.44530765522902) bank35264 +35265 POINT(41.69702589960245 73.33529620400448) bank35265 +35266 POINT(40.24400880673685 74.55363402735155) bank35266 +35267 POINT(40.776162681843466 74.11707819362586) bank35267 +35268 POINT(41.00407980499825 74.9557425203197) bank35268 +35269 POINT(41.05534082130698 73.72098570278212) bank35269 +35270 POINT(40.614604858285674 74.34509685266386) bank35270 +35271 POINT(40.07479055736054 73.77048169749214) bank35271 +35272 POINT(40.74498291910819 74.33909623083699) bank35272 +35273 POINT(39.809092925781734 74.66356489983274) bank35273 +35274 POINT(39.79222451856701 73.56791660491272) bank35274 +35275 POINT(40.37295141770488 73.75328839286965) bank35275 +35276 POINT(41.46734807142357 74.94870180112486) bank35276 +35277 POINT(41.51415762240656 73.83124249942416) bank35277 +35278 POINT(41.28030994101222 74.67653872409461) bank35278 +35279 POINT(41.3660880164247 73.56096226962494) bank35279 +35280 POINT(41.09033973235719 73.39698559801153) bank35280 +35281 POINT(40.15530736206148 74.351489974276) bank35281 +35282 POINT(41.190656884424946 73.02658638630649) bank35282 +35283 POINT(40.72055599413394 74.63822533465544) bank35283 +35284 POINT(40.29021989761292 74.54560537799544) bank35284 +35285 POINT(40.250711947897486 74.00831386635758) bank35285 +35286 POINT(41.35676975742965 74.86277053193716) bank35286 +35287 POINT(40.26189465007005 73.48918885410785) bank35287 +35288 POINT(40.477035335942695 73.33563733290346) bank35288 +35289 POINT(40.97528142828051 73.02227305518204) bank35289 +35290 POINT(41.24705507850761 74.31467666152662) bank35290 +35291 POINT(40.12238701394466 73.52676101525554) bank35291 +35292 POINT(41.47064094040965 73.47096006880776) bank35292 +35293 POINT(41.65273018482021 74.18170155563647) bank35293 +35294 POINT(40.152999974530914 73.44301949392857) bank35294 +35295 POINT(41.4680549331688 73.37618356638595) bank35295 +35296 POINT(40.40868949503232 74.11899112053813) bank35296 +35297 POINT(39.86320352713682 74.10271253606149) bank35297 +35298 POINT(40.80042518592011 73.4032176415791) bank35298 +35299 POINT(39.920640049050824 74.26836099457407) bank35299 +35300 POINT(40.152547065303104 73.22053213170807) bank35300 +35301 POINT(40.41036031484218 74.09872938268785) bank35301 +35302 POINT(40.69489421835181 74.14916867225902) bank35302 +35303 POINT(40.269962338751846 74.19481339438423) bank35303 +35304 POINT(39.929734768274 74.96708053852576) bank35304 +35305 POINT(40.368149222256534 73.1068502914261) bank35305 +35306 POINT(40.288895469433676 74.93817797542992) bank35306 +35307 POINT(40.75859174866532 74.16288470431626) bank35307 +35308 POINT(41.682587684293146 74.2906088303662) bank35308 +35309 POINT(40.92348986649806 73.46848599366405) bank35309 +35310 POINT(40.92825933532656 74.5783271321679) bank35310 +35311 POINT(40.9705211179728 73.18640934882369) bank35311 +35312 POINT(41.64791452303716 74.35021024726741) bank35312 +35313 POINT(41.31485638299074 73.94664726546861) bank35313 +35314 POINT(39.92904976252499 73.17063882018994) bank35314 +35315 POINT(41.17634339105777 74.66365336466738) bank35315 +35316 POINT(40.73150377214016 74.91772374430653) bank35316 +35317 POINT(40.99776082465521 74.07846562041101) bank35317 +35318 POINT(41.42137656886399 73.61612545906551) bank35318 +35319 POINT(41.204511421028414 74.41498278949516) bank35319 +35320 POINT(41.63317133354106 73.13940450135856) bank35320 +35321 POINT(41.446638406837955 74.90403043232452) bank35321 +35322 POINT(40.18443454164135 74.24190700035008) bank35322 +35323 POINT(40.68106493770791 73.4666766000119) bank35323 +35324 POINT(39.71794378626308 74.52638357472314) bank35324 +35325 POINT(40.689447704661205 73.28947312973055) bank35325 +35326 POINT(41.069076785659846 74.02295305123239) bank35326 +35327 POINT(39.99764976065834 73.05325761652057) bank35327 +35328 POINT(40.130475803594415 73.62784727851786) bank35328 +35329 POINT(40.57587711454078 74.5216746729019) bank35329 +35330 POINT(39.86503952173297 73.76251451022785) bank35330 +35331 POINT(39.917200299322595 73.94473581484927) bank35331 +35332 POINT(40.129009157742686 74.73613920966008) bank35332 +35333 POINT(39.94014502230027 73.3092920612215) bank35333 +35334 POINT(40.87632812603857 73.83398440274895) bank35334 +35335 POINT(40.22132386154263 73.00869445882228) bank35335 +35336 POINT(40.010867813727934 74.78425362132495) bank35336 +35337 POINT(40.74980352463345 73.30810565528614) bank35337 +35338 POINT(40.90122289115522 73.16335047799947) bank35338 +35339 POINT(40.003802328514894 73.69756555493922) bank35339 +35340 POINT(40.94006570652892 73.92746670027476) bank35340 +35341 POINT(40.717882299393324 74.70450274573983) bank35341 +35342 POINT(41.612448696119394 73.37531970157964) bank35342 +35343 POINT(39.97286170020569 73.09627950236073) bank35343 +35344 POINT(41.21651817862484 73.03138581469737) bank35344 +35345 POINT(40.48755431769524 74.25263503015167) bank35345 +35346 POINT(41.50451659077966 73.70182305960003) bank35346 +35347 POINT(41.55299226556436 74.48115580921747) bank35347 +35348 POINT(41.5413417279393 74.67832389875004) bank35348 +35349 POINT(41.28048328555955 74.46929410664268) bank35349 +35350 POINT(41.60248991220849 74.52103538965578) bank35350 +35351 POINT(40.34989998099 74.16935852903202) bank35351 +35352 POINT(39.944860333141264 74.65402154610864) bank35352 +35353 POINT(40.60897837890847 73.1911083085267) bank35353 +35354 POINT(40.11700865818914 74.05668452860421) bank35354 +35355 POINT(39.73291237010286 74.87538093950339) bank35355 +35356 POINT(40.47545644294827 73.41043504301496) bank35356 +35357 POINT(39.78022623940897 73.06305815824238) bank35357 +35358 POINT(40.74956898758026 73.09704267098869) bank35358 +35359 POINT(39.8424925649589 73.90103648178648) bank35359 +35360 POINT(40.577155887245425 74.63510965957933) bank35360 +35361 POINT(40.57855358027178 74.00585526119987) bank35361 +35362 POINT(41.17920255177328 73.7544791727223) bank35362 +35363 POINT(41.64394858353088 74.81043067242955) bank35363 +35364 POINT(40.01316152273612 73.26650929154168) bank35364 +35365 POINT(39.743939478278676 73.99745341793438) bank35365 +35366 POINT(40.36283079155533 73.21618335789694) bank35366 +35367 POINT(41.274498118915226 73.85154908102461) bank35367 +35368 POINT(40.29319439057788 73.57874668099923) bank35368 +35369 POINT(41.18091729022845 73.1412844589318) bank35369 +35370 POINT(40.04426204084182 73.35376212008818) bank35370 +35371 POINT(41.04394829112908 73.95095212851453) bank35371 +35372 POINT(41.20557872855373 74.34751527988695) bank35372 +35373 POINT(40.23907777017437 73.70949624169504) bank35373 +35374 POINT(41.19812235785675 73.27686861904631) bank35374 +35375 POINT(40.86456213820637 73.75911965210074) bank35375 +35376 POINT(39.86630270036245 73.92116055681387) bank35376 +35377 POINT(40.030543334970695 73.8372108343709) bank35377 +35378 POINT(39.85685101189863 74.8434131971483) bank35378 +35379 POINT(40.48659988407275 74.18360512974814) bank35379 +35380 POINT(41.199396800905724 73.67398834745559) bank35380 +35381 POINT(40.886281445080236 73.24349060987637) bank35381 +35382 POINT(39.90633926342411 73.50237335677733) bank35382 +35383 POINT(40.87450230302415 74.93538534201316) bank35383 +35384 POINT(39.970183914346336 74.46046310391266) bank35384 +35385 POINT(41.108521210750446 74.5402659000987) bank35385 +35386 POINT(41.7015601391073 74.29093498093444) bank35386 +35387 POINT(41.099791941244824 73.34059655279643) bank35387 +35388 POINT(41.05643330963095 73.03718363580357) bank35388 +35389 POINT(40.225903034057474 74.67462691029868) bank35389 +35390 POINT(41.061094737895615 73.25395293476899) bank35390 +35391 POINT(41.23467576334494 73.62724187780344) bank35391 +35392 POINT(41.066982170620946 74.31122608214672) bank35392 +35393 POINT(39.98971770893977 74.67849970861418) bank35393 +35394 POINT(39.73821575665956 73.83631019273828) bank35394 +35395 POINT(39.88226051762338 74.90106806024207) bank35395 +35396 POINT(39.84456737601653 73.79288109307902) bank35396 +35397 POINT(41.01042776388711 74.90572480126811) bank35397 +35398 POINT(39.95100871012398 73.35963537227774) bank35398 +35399 POINT(40.0721117922196 73.88715897538128) bank35399 +35400 POINT(40.27798396509456 73.75855310748892) bank35400 +35401 POINT(39.9364283452273 73.97538236232784) bank35401 +35402 POINT(40.05881053752302 73.76185424598313) bank35402 +35403 POINT(40.554226633361885 74.99765884186402) bank35403 +35404 POINT(41.05029416184872 73.9305698385561) bank35404 +35405 POINT(39.9397731333438 74.80132939155303) bank35405 +35406 POINT(40.73723293818931 73.65721265447772) bank35406 +35407 POINT(40.053375649792315 73.1914802951986) bank35407 +35408 POINT(40.051820565706265 73.69726459014936) bank35408 +35409 POINT(40.86152730243116 73.65017143894325) bank35409 +35410 POINT(40.07903992996588 73.85576912484939) bank35410 +35411 POINT(41.43039831914429 73.33141629469252) bank35411 +35412 POINT(41.65437025126892 74.97080683424907) bank35412 +35413 POINT(40.96161223181735 73.19399665998408) bank35413 +35414 POINT(39.72062692078136 73.60760044233417) bank35414 +35415 POINT(41.0771186714448 73.34322379096605) bank35415 +35416 POINT(41.213755620617086 74.93111602634757) bank35416 +35417 POINT(40.623869068427084 73.53053948311445) bank35417 +35418 POINT(40.98675178959949 74.94281174486446) bank35418 +35419 POINT(41.019642065505366 74.99763363018869) bank35419 +35420 POINT(41.292108309011674 74.99022776534254) bank35420 +35421 POINT(39.97381233979228 74.12264987249742) bank35421 +35422 POINT(40.53350073726465 74.46659611994384) bank35422 +35423 POINT(41.566831050982564 73.48528197796553) bank35423 +35424 POINT(40.33532880732507 73.3559245766853) bank35424 +35425 POINT(40.52115789937701 74.11097899300714) bank35425 +35426 POINT(40.219815558647085 74.02542059692446) bank35426 +35427 POINT(40.18228804577444 73.84469634215984) bank35427 +35428 POINT(40.08368275908945 73.95280245126523) bank35428 +35429 POINT(39.90272019937308 74.09421056499772) bank35429 +35430 POINT(41.120264123461496 74.54818400083711) bank35430 +35431 POINT(41.51490597708982 73.33924305824323) bank35431 +35432 POINT(41.13331839366111 73.75157799613352) bank35432 +35433 POINT(40.26827624508851 73.90246105445813) bank35433 +35434 POINT(41.57390011968338 73.55636221580997) bank35434 +35435 POINT(40.355617640124 74.14301437095591) bank35435 +35436 POINT(40.07746143020337 74.91646657086626) bank35436 +35437 POINT(41.21443343670332 74.34506509264239) bank35437 +35438 POINT(40.26927946644986 74.99997206603558) bank35438 +35439 POINT(41.518859467808284 73.57675864182355) bank35439 +35440 POINT(41.50016878021917 73.42024112123796) bank35440 +35441 POINT(41.52142070165382 73.82780482013665) bank35441 +35442 POINT(41.06479962759293 73.92045118481424) bank35442 +35443 POINT(41.26456568825614 73.65742745995304) bank35443 +35444 POINT(41.64183085627877 74.64189475462072) bank35444 +35445 POINT(40.313395871642015 74.91180924660073) bank35445 +35446 POINT(39.9817638180097 74.56026619888883) bank35446 +35447 POINT(40.718263279653435 73.69358819230891) bank35447 +35448 POINT(40.64102041577626 74.19664804181077) bank35448 +35449 POINT(41.18339001337177 74.2708768174766) bank35449 +35450 POINT(40.13961134619451 73.32158290532227) bank35450 +35451 POINT(41.071147498297655 73.88753520355202) bank35451 +35452 POINT(40.06145400226063 73.42382827756991) bank35452 +35453 POINT(40.57821283599098 73.83312968138486) bank35453 +35454 POINT(39.82259641282244 74.47527696333161) bank35454 +35455 POINT(40.1388301880083 74.9646035520398) bank35455 +35456 POINT(40.49045738072129 74.45495879249543) bank35456 +35457 POINT(41.6116045949784 74.87709813517888) bank35457 +35458 POINT(40.519573333366786 73.91680173730846) bank35458 +35459 POINT(41.13791655869571 74.81451600346418) bank35459 +35460 POINT(40.57547891724233 74.30277177736257) bank35460 +35461 POINT(41.685121271686995 73.7429499624241) bank35461 +35462 POINT(40.61306739551524 73.16691147575656) bank35462 +35463 POINT(39.740112738354625 74.57595674344928) bank35463 +35464 POINT(41.68105410757265 74.13787982728319) bank35464 +35465 POINT(40.244113891000396 73.11149577324494) bank35465 +35466 POINT(39.74267906532569 73.64457692937386) bank35466 +35467 POINT(41.400262780553454 73.54867385030366) bank35467 +35468 POINT(40.586046577223634 73.90655256572536) bank35468 +35469 POINT(40.36426072232184 74.7177637355345) bank35469 +35470 POINT(39.944775615696585 74.48703916275326) bank35470 +35471 POINT(40.14375335540095 74.21772834661873) bank35471 +35472 POINT(41.033378363369266 74.61399486547967) bank35472 +35473 POINT(41.67945000660942 73.60820280793398) bank35473 +35474 POINT(40.36615968893113 73.522838997359) bank35474 +35475 POINT(40.33875587412834 74.95372286101018) bank35475 +35476 POINT(40.60667065742275 74.55320093450545) bank35476 +35477 POINT(41.2605545251122 73.5704547998028) bank35477 +35478 POINT(41.129130265161756 74.67308348659583) bank35478 +35479 POINT(40.11462758601262 73.95445001460612) bank35479 +35480 POINT(40.895821070158576 74.88677824731568) bank35480 +35481 POINT(40.94913671592038 74.0691810275763) bank35481 +35482 POINT(40.87221647449635 73.20848278032857) bank35482 +35483 POINT(41.18029613103749 74.00129978153251) bank35483 +35484 POINT(41.064504078092966 74.30880256417008) bank35484 +35485 POINT(40.62825679944744 73.27780026203945) bank35485 +35486 POINT(40.87620624554768 73.23163424325918) bank35486 +35487 POINT(40.923461548832684 73.32195056485513) bank35487 +35488 POINT(40.13936505941185 74.00867294516155) bank35488 +35489 POINT(41.291595134944885 73.69415768121453) bank35489 +35490 POINT(41.03586105410421 74.53774112354222) bank35490 +35491 POINT(40.64772434033455 74.15846684927156) bank35491 +35492 POINT(41.70523334400754 74.93497096615523) bank35492 +35493 POINT(40.99959616095048 73.57751704582927) bank35493 +35494 POINT(39.81225748254137 73.52015625500204) bank35494 +35495 POINT(41.355986437599995 75.00108277461361) bank35495 +35496 POINT(40.17165709934406 73.308760470899) bank35496 +35497 POINT(41.01293873093053 73.88185636982138) bank35497 +35498 POINT(41.007884625384705 73.61135162171485) bank35498 +35499 POINT(40.3697396156826 73.45197123372556) bank35499 +35500 POINT(39.99223033116792 73.79373968895578) bank35500 +35501 POINT(40.30468056081543 74.26508020769764) bank35501 +35502 POINT(40.56890872509365 73.61395184098164) bank35502 +35503 POINT(39.830118044943696 73.94085010993057) bank35503 +35504 POINT(41.51746469431904 74.36967834959557) bank35504 +35505 POINT(41.11031102174812 74.72041455942917) bank35505 +35506 POINT(39.71668810216593 74.91174709388837) bank35506 +35507 POINT(40.16206772509847 74.99182822985343) bank35507 +35508 POINT(39.87349857161897 73.86351220554158) bank35508 +35509 POINT(41.018431774472134 73.56178621595446) bank35509 +35510 POINT(40.46084444944575 74.55452666360202) bank35510 +35511 POINT(40.60242394706877 74.74244279274009) bank35511 +35512 POINT(41.08208317783679 74.88553077209316) bank35512 +35513 POINT(40.425431802437735 74.3115387678378) bank35513 +35514 POINT(41.65953304930398 73.32272102190841) bank35514 +35515 POINT(40.974192699297284 73.12726064400469) bank35515 +35516 POINT(41.50389600531405 74.36921432279848) bank35516 +35517 POINT(39.776734890195506 73.74807304840671) bank35517 +35518 POINT(40.59393190814846 73.45770434307819) bank35518 +35519 POINT(40.77984728295276 73.93292143327353) bank35519 +35520 POINT(40.786455290373006 73.40378561087358) bank35520 +35521 POINT(39.80868307270426 73.76078818766634) bank35521 +35522 POINT(40.435500816982035 73.61272814945727) bank35522 +35523 POINT(41.056576194386814 74.26788329625727) bank35523 +35524 POINT(41.366807591175785 74.5946613491196) bank35524 +35525 POINT(40.65715049500084 74.60227959802694) bank35525 +35526 POINT(40.29658835854933 73.98497114749716) bank35526 +35527 POINT(41.059230998180325 74.58165017402267) bank35527 +35528 POINT(39.981252155588614 74.441259364354) bank35528 +35529 POINT(41.41814271010029 73.26419667078721) bank35529 +35530 POINT(39.83745853933394 73.12664752920251) bank35530 +35531 POINT(40.74322881022849 73.54094954635545) bank35531 +35532 POINT(40.17333915003195 73.66526107421531) bank35532 +35533 POINT(40.912227943076154 73.05776264705497) bank35533 +35534 POINT(40.25630028399009 73.36384639356801) bank35534 +35535 POINT(40.317826707724194 74.27620952735035) bank35535 +35536 POINT(41.659655787377034 74.81468898060714) bank35536 +35537 POINT(41.62327668732591 74.62950056352446) bank35537 +35538 POINT(40.095324517704675 73.42393084684686) bank35538 +35539 POINT(41.250672670357936 73.53720575443542) bank35539 +35540 POINT(41.54826070814282 74.49010011135576) bank35540 +35541 POINT(41.01531866153243 73.22919004901387) bank35541 +35542 POINT(41.31164392801314 74.3138363932676) bank35542 +35543 POINT(39.77134940268792 74.95460436173515) bank35543 +35544 POINT(41.404878711829795 74.92304223766487) bank35544 +35545 POINT(40.65462673604419 73.13831933373842) bank35545 +35546 POINT(40.99809811587467 73.1392540801052) bank35546 +35547 POINT(40.42676769390891 73.51703272758921) bank35547 +35548 POINT(40.87323118352381 74.51640603015639) bank35548 +35549 POINT(39.921335467173314 74.57551893721745) bank35549 +35550 POINT(41.51636982587195 73.63347770603039) bank35550 +35551 POINT(40.8178818864379 73.71818753955301) bank35551 +35552 POINT(40.10759162485587 73.5984747617883) bank35552 +35553 POINT(39.746828769678515 73.36355523481092) bank35553 +35554 POINT(40.84438936319685 73.64386879456613) bank35554 +35555 POINT(40.308783667977714 74.86551818764269) bank35555 +35556 POINT(40.22162491035919 73.37759823319456) bank35556 +35557 POINT(40.5635419993329 74.30105378294857) bank35557 +35558 POINT(40.700391942639804 74.77383460476783) bank35558 +35559 POINT(40.284681737296985 74.40014448585744) bank35559 +35560 POINT(40.75465440014563 73.5790106243283) bank35560 +35561 POINT(39.925789977629805 74.88741204176218) bank35561 +35562 POINT(40.69395507390345 74.65698298118531) bank35562 +35563 POINT(40.899621982376075 74.87192680521358) bank35563 +35564 POINT(40.62522039369166 74.63406227866916) bank35564 +35565 POINT(41.4896782232532 73.0313491763271) bank35565 +35566 POINT(40.656741465628315 73.77536557538069) bank35566 +35567 POINT(40.94574812662082 73.31126776769766) bank35567 +35568 POINT(40.83707091976731 73.25762315306801) bank35568 +35569 POINT(41.57914204717181 73.89812440080007) bank35569 +35570 POINT(40.280244273636036 73.16784831640858) bank35570 +35571 POINT(40.12592417122656 73.17963235979614) bank35571 +35572 POINT(40.794283768325485 73.28070107804139) bank35572 +35573 POINT(41.49544131281009 73.31093910799774) bank35573 +35574 POINT(40.48449307050437 74.41874696564294) bank35574 +35575 POINT(40.211437255502794 74.17610851810171) bank35575 +35576 POINT(41.5078530435861 74.02958349160862) bank35576 +35577 POINT(39.73099305837559 74.9327341553804) bank35577 +35578 POINT(40.42643456819031 73.07492732210643) bank35578 +35579 POINT(41.34068286065699 73.48132485860523) bank35579 +35580 POINT(40.4177458189429 73.1942764051035) bank35580 +35581 POINT(40.615111264183795 74.81369297922022) bank35581 +35582 POINT(40.69732481129665 74.60996080252976) bank35582 +35583 POINT(40.937482375918925 74.44815872987392) bank35583 +35584 POINT(41.513369317913096 73.42822388956813) bank35584 +35585 POINT(40.71799283615285 73.52810078961957) bank35585 +35586 POINT(41.11440873934943 74.41929447098536) bank35586 +35587 POINT(40.51305933074606 74.74082340705817) bank35587 +35588 POINT(40.573282644858736 74.74292369823254) bank35588 +35589 POINT(41.50396313116266 74.9254306732978) bank35589 +35590 POINT(41.585805888575464 74.35184751126498) bank35590 +35591 POINT(40.64993477004041 73.26646404396085) bank35591 +35592 POINT(39.85812247755254 73.47182390186443) bank35592 +35593 POINT(41.442099345232876 74.85797279016084) bank35593 +35594 POINT(41.612152649207395 74.72523373696436) bank35594 +35595 POINT(40.94191690230461 74.9997028221218) bank35595 +35596 POINT(40.64615102596388 74.7649172094571) bank35596 +35597 POINT(41.48485317332265 74.37721917158183) bank35597 +35598 POINT(39.868820165440965 74.94434318564139) bank35598 +35599 POINT(41.22409605378953 73.2900685956938) bank35599 +35600 POINT(41.531671525582155 73.36470930119883) bank35600 +35601 POINT(40.27254941861685 74.79115228328378) bank35601 +35602 POINT(41.06025716548327 73.18153862972187) bank35602 +35603 POINT(40.29242941198314 74.40894539174661) bank35603 +35604 POINT(40.9970382018236 73.93064267713075) bank35604 +35605 POINT(40.048680527011086 74.89205551809714) bank35605 +35606 POINT(40.14308098204017 73.83996448979376) bank35606 +35607 POINT(40.9596726838673 73.15878461293045) bank35607 +35608 POINT(40.6203705343947 73.40297418224515) bank35608 +35609 POINT(41.23068540692552 73.81570334590985) bank35609 +35610 POINT(39.814829900941774 74.77288700867626) bank35610 +35611 POINT(40.05707995449591 73.9141120627966) bank35611 +35612 POINT(41.42452185728081 74.79384463936279) bank35612 +35613 POINT(40.94200406624445 74.66958863564693) bank35613 +35614 POINT(40.2099986059674 73.65532778127971) bank35614 +35615 POINT(41.43115686002437 73.70021720275966) bank35615 +35616 POINT(41.612712634381616 73.43565811108569) bank35616 +35617 POINT(40.79771260286659 73.18260997195782) bank35617 +35618 POINT(40.307390403322074 74.02750229870095) bank35618 +35619 POINT(40.69505101396721 74.98600657210736) bank35619 +35620 POINT(41.467340161761946 74.58437692336378) bank35620 +35621 POINT(40.867514003930104 73.167011592486) bank35621 +35622 POINT(41.33862685513134 74.00718072558203) bank35622 +35623 POINT(40.40511221950349 74.42863489013601) bank35623 +35624 POINT(40.60072373313403 73.98567408222137) bank35624 +35625 POINT(41.274507354731725 73.67450395864785) bank35625 +35626 POINT(39.92767189280329 74.42487303196746) bank35626 +35627 POINT(40.56452871256912 73.01251090093702) bank35627 +35628 POINT(39.76585504389146 73.12300706658279) bank35628 +35629 POINT(40.06972158501598 74.57620851139276) bank35629 +35630 POINT(40.98237599879106 74.5470687189618) bank35630 +35631 POINT(40.68777616630879 73.37225954911307) bank35631 +35632 POINT(40.95104319974391 74.72332363609635) bank35632 +35633 POINT(41.02325090700623 74.48239937972207) bank35633 +35634 POINT(40.69785954744055 74.41832428611033) bank35634 +35635 POINT(39.89365323153703 73.1474051435598) bank35635 +35636 POINT(40.84734548358468 74.4744345858061) bank35636 +35637 POINT(40.61497444835509 73.2642286802598) bank35637 +35638 POINT(40.2513625289393 73.83114340437953) bank35638 +35639 POINT(40.16485647191768 73.3865861648056) bank35639 +35640 POINT(40.16805657320265 74.48581378400178) bank35640 +35641 POINT(40.80668933010226 73.43254599564537) bank35641 +35642 POINT(39.898368301187006 74.56841042110631) bank35642 +35643 POINT(39.820391723779764 74.16578749318474) bank35643 +35644 POINT(40.455665548225475 73.77476150266409) bank35644 +35645 POINT(40.54263032353106 74.7010706697419) bank35645 +35646 POINT(40.27520541527563 74.71310371686565) bank35646 +35647 POINT(41.42836887549362 74.09387739706615) bank35647 +35648 POINT(40.562166901481156 74.72361595546872) bank35648 +35649 POINT(39.7489487828763 73.45981002468093) bank35649 +35650 POINT(41.65098840876952 74.00860688265848) bank35650 +35651 POINT(40.10712996916989 74.58316048433569) bank35651 +35652 POINT(40.22695077579427 73.66705802253082) bank35652 +35653 POINT(41.33705393432461 74.41469911681016) bank35653 +35654 POINT(40.56280463090543 74.91457989299721) bank35654 +35655 POINT(41.144502191885486 74.98590629062781) bank35655 +35656 POINT(41.201716751366604 73.94104353663562) bank35656 +35657 POINT(41.55788195800913 74.64158859255615) bank35657 +35658 POINT(41.522655585561594 73.56557409947824) bank35658 +35659 POINT(40.235024195275294 74.25330175498407) bank35659 +35660 POINT(41.5506496427854 73.96718786592541) bank35660 +35661 POINT(39.766398810751284 73.60776743017705) bank35661 +35662 POINT(40.21471126563577 73.70604456447319) bank35662 +35663 POINT(39.939702056095435 74.08024903605049) bank35663 +35664 POINT(40.212012738954805 73.38963860296654) bank35664 +35665 POINT(40.5560089616282 74.2027588701939) bank35665 +35666 POINT(40.50016411994115 73.31884558451469) bank35666 +35667 POINT(41.29360383315571 73.42792595213437) bank35667 +35668 POINT(40.52655144757385 73.50444992246238) bank35668 +35669 POINT(40.65756900290431 73.54609313016329) bank35669 +35670 POINT(40.95362849717488 74.09299319054006) bank35670 +35671 POINT(41.06927980586219 73.74987050265585) bank35671 +35672 POINT(40.02819907836427 73.71446523172983) bank35672 +35673 POINT(41.36624576611961 73.83951218314016) bank35673 +35674 POINT(40.22563782493777 73.6323094331722) bank35674 +35675 POINT(41.232452429110616 73.64846020814907) bank35675 +35676 POINT(41.58258625668663 74.92793967092204) bank35676 +35677 POINT(40.41728883393756 75.0036225641997) bank35677 +35678 POINT(39.949116779685276 73.65909656865689) bank35678 +35679 POINT(40.2626370085202 74.2519176621654) bank35679 +35680 POINT(41.49075513069279 74.73071273296662) bank35680 +35681 POINT(40.56763742850247 74.66340538405215) bank35681 +35682 POINT(40.29672946172264 73.59055283041549) bank35682 +35683 POINT(40.25906954338703 74.01014831647079) bank35683 +35684 POINT(41.0698142546738 73.24577796203947) bank35684 +35685 POINT(41.114136341682666 74.12479348555071) bank35685 +35686 POINT(40.83858431732467 73.26791152709696) bank35686 +35687 POINT(40.0861191772475 73.42802439240137) bank35687 +35688 POINT(41.01376709257297 73.67194465976222) bank35688 +35689 POINT(40.35700967229624 73.93687686086757) bank35689 +35690 POINT(41.49260244292115 74.45978026200191) bank35690 +35691 POINT(41.13328857415201 74.68651675064172) bank35691 +35692 POINT(40.63968370070293 74.56725107817998) bank35692 +35693 POINT(39.78558215616903 74.29231469570294) bank35693 +35694 POINT(41.266245429951496 73.39542336675025) bank35694 +35695 POINT(40.61602895161941 73.13335894225534) bank35695 +35696 POINT(40.113665155995214 73.05327885541841) bank35696 +35697 POINT(40.12795865989182 74.09255480329539) bank35697 +35698 POINT(40.6156567311313 73.14029930412488) bank35698 +35699 POINT(40.32314971884011 74.504052746718) bank35699 +35700 POINT(39.98527201556864 73.45279158706265) bank35700 +35701 POINT(39.85712954862068 73.93276538047115) bank35701 +35702 POINT(39.97485602621186 74.99500568985621) bank35702 +35703 POINT(40.066009200984595 73.04107629734648) bank35703 +35704 POINT(41.31325047691798 74.70008340627244) bank35704 +35705 POINT(40.08840194508739 73.63553522753213) bank35705 +35706 POINT(40.4807212189105 73.9039031626808) bank35706 +35707 POINT(40.133834964416586 74.89629187308759) bank35707 +35708 POINT(40.68622973243592 73.15924155521405) bank35708 +35709 POINT(41.1513573190145 74.45553252323151) bank35709 +35710 POINT(40.332130798070956 73.62412409548782) bank35710 +35711 POINT(41.37689270564783 74.18057227027155) bank35711 +35712 POINT(40.55091707700805 74.05772991231336) bank35712 +35713 POINT(40.8468285004692 74.43948986136537) bank35713 +35714 POINT(40.67355573966067 74.33357588335397) bank35714 +35715 POINT(40.42248148025254 73.081317553008) bank35715 +35716 POINT(40.22484166502935 74.92136629105225) bank35716 +35717 POINT(41.69124445105539 73.03340896662039) bank35717 +35718 POINT(41.56875549061542 74.68177274592324) bank35718 +35719 POINT(40.199695853761284 73.8879316100344) bank35719 +35720 POINT(40.05337640487614 73.0578546462792) bank35720 +35721 POINT(40.01966126906977 74.28790901678431) bank35721 +35722 POINT(40.494066494398595 73.86428988254401) bank35722 +35723 POINT(40.746618931179235 73.51038183266839) bank35723 +35724 POINT(41.04897207679812 73.60420014836826) bank35724 +35725 POINT(41.55266632708119 73.40259367611253) bank35725 +35726 POINT(40.15717077335395 73.30954468843638) bank35726 +35727 POINT(40.78559720578649 74.90823164773478) bank35727 +35728 POINT(40.25971075908482 74.50958258283792) bank35728 +35729 POINT(40.72362484900787 74.29911068500004) bank35729 +35730 POINT(40.08819069192539 73.03791986798777) bank35730 +35731 POINT(40.93024256681294 74.20417624840069) bank35731 +35732 POINT(40.96565584457005 74.43349673225916) bank35732 +35733 POINT(41.47493862802536 74.28571648726397) bank35733 +35734 POINT(41.23731319836776 73.22227485476094) bank35734 +35735 POINT(41.36225967709535 74.6744667573806) bank35735 +35736 POINT(41.15156637629063 74.63381676119698) bank35736 +35737 POINT(41.223346489441916 74.54239028347212) bank35737 +35738 POINT(40.01220030806641 73.9278430604009) bank35738 +35739 POINT(40.75027200408184 73.783785101493) bank35739 +35740 POINT(39.7936537215551 73.2819983651673) bank35740 +35741 POINT(40.976408952172214 74.55349436298977) bank35741 +35742 POINT(40.56605691374491 73.1422032868854) bank35742 +35743 POINT(39.75770727612029 73.15327455658549) bank35743 +35744 POINT(40.25020157128605 73.48062296322044) bank35744 +35745 POINT(40.8103560413598 74.4986299326263) bank35745 +35746 POINT(40.34803659774924 73.65422673493838) bank35746 +35747 POINT(41.20393673630205 73.5397733571141) bank35747 +35748 POINT(40.989061114553145 73.83770503995977) bank35748 +35749 POINT(40.73557140010697 73.89241616132665) bank35749 +35750 POINT(40.67003844994311 73.64402427375808) bank35750 +35751 POINT(40.54181201544663 74.2435891632537) bank35751 +35752 POINT(40.74584612205291 74.9545818842599) bank35752 +35753 POINT(40.015678750453766 74.65459597279633) bank35753 +35754 POINT(41.06120829164782 74.67157342847187) bank35754 +35755 POINT(40.042173663225185 74.83045753456533) bank35755 +35756 POINT(40.81555225619358 73.23162609807594) bank35756 +35757 POINT(40.981478016467676 74.1784251940546) bank35757 +35758 POINT(40.809105649615866 74.39696192645117) bank35758 +35759 POINT(40.44503887959499 74.7224991680046) bank35759 +35760 POINT(41.14984358407956 73.40039313390207) bank35760 +35761 POINT(41.43861082848699 74.44119942072425) bank35761 +35762 POINT(40.63928031947882 73.10507080761484) bank35762 +35763 POINT(41.63531151086194 74.47203221575782) bank35763 +35764 POINT(40.88404630935995 74.90071918572421) bank35764 +35765 POINT(41.50201996881346 74.99090765842811) bank35765 +35766 POINT(39.814243723768804 73.14566037162488) bank35766 +35767 POINT(39.88312980296504 73.50178555181044) bank35767 +35768 POINT(40.20909692217136 74.35489533421776) bank35768 +35769 POINT(40.02366674229904 74.48879086162835) bank35769 +35770 POINT(40.244797997927655 73.09470848905151) bank35770 +35771 POINT(40.68896126507221 73.45552800984518) bank35771 +35772 POINT(40.84364505711783 74.50200398160338) bank35772 +35773 POINT(40.197201810662804 74.18130098946793) bank35773 +35774 POINT(41.66238889158339 73.94110897522775) bank35774 +35775 POINT(41.43451233690542 73.96942602949143) bank35775 +35776 POINT(39.92540911178875 74.75404759818602) bank35776 +35777 POINT(40.863048785308855 73.63174565054496) bank35777 +35778 POINT(40.41612966656311 74.39246944248895) bank35778 +35779 POINT(41.6501538032707 74.54572163698076) bank35779 +35780 POINT(40.62813113832209 74.40562336730537) bank35780 +35781 POINT(40.40686752253577 74.30270567660831) bank35781 +35782 POINT(40.1006973677127 73.04144340350577) bank35782 +35783 POINT(41.66453949806016 74.25873494151863) bank35783 +35784 POINT(40.54332196919445 74.91430057742737) bank35784 +35785 POINT(40.42718971079367 73.79431129125621) bank35785 +35786 POINT(40.59244044008131 74.50106420463344) bank35786 +35787 POINT(40.86960041116452 74.62417326706822) bank35787 +35788 POINT(40.649675794518195 73.54908704958756) bank35788 +35789 POINT(41.03634272656099 74.87814347715567) bank35789 +35790 POINT(41.389588164408366 73.4367698814783) bank35790 +35791 POINT(41.30755646851341 74.4123266052759) bank35791 +35792 POINT(39.78965073862346 73.63910434427984) bank35792 +35793 POINT(41.645576970462805 73.62632326553444) bank35793 +35794 POINT(39.7136816828966 73.03634758679542) bank35794 +35795 POINT(41.136541270941386 74.56540242237013) bank35795 +35796 POINT(39.90566345727706 74.47785480944414) bank35796 +35797 POINT(41.469159563498465 74.57235785464658) bank35797 +35798 POINT(41.55131219083401 73.47793581176083) bank35798 +35799 POINT(40.25151877241533 74.89889096925934) bank35799 +35800 POINT(40.42163564409867 73.57854220426415) bank35800 +35801 POINT(40.411005752407405 74.37836045594305) bank35801 +35802 POINT(40.841332433708594 73.16420413282032) bank35802 +35803 POINT(40.0185707496736 74.4659542047034) bank35803 +35804 POINT(41.272067045802956 74.70800730481925) bank35804 +35805 POINT(39.725999135055496 74.62795087706971) bank35805 +35806 POINT(41.09625166002121 73.20582383863955) bank35806 +35807 POINT(41.14911050961912 74.30987815172625) bank35807 +35808 POINT(41.25886780802459 74.04132482620494) bank35808 +35809 POINT(41.02670985639708 73.6978368345352) bank35809 +35810 POINT(39.998999981958036 73.4270441801754) bank35810 +35811 POINT(40.60298804101431 74.65656454365777) bank35811 +35812 POINT(40.66272384481477 74.24009049904336) bank35812 +35813 POINT(41.26068264322371 73.53036501794803) bank35813 +35814 POINT(41.4432496720208 74.4855417619793) bank35814 +35815 POINT(41.32125297041415 73.18730835813201) bank35815 +35816 POINT(40.777410555244394 74.71913173424208) bank35816 +35817 POINT(41.239088070911755 74.76315075331144) bank35817 +35818 POINT(40.054746311273945 74.4085954728045) bank35818 +35819 POINT(40.04765149461556 74.25374907851814) bank35819 +35820 POINT(41.10916155825247 74.52255331493494) bank35820 +35821 POINT(40.30367202741756 74.18312632315184) bank35821 +35822 POINT(41.646365890270125 74.85626445997632) bank35822 +35823 POINT(40.53513596780394 74.77456954611988) bank35823 +35824 POINT(39.80663119124534 74.88715826985842) bank35824 +35825 POINT(40.41267139809624 74.80097213816764) bank35825 +35826 POINT(40.93722695732659 74.76278415837155) bank35826 +35827 POINT(39.98773834367096 74.3845497770425) bank35827 +35828 POINT(41.646641921439326 73.61722206754777) bank35828 +35829 POINT(40.47725989346265 74.41377088239747) bank35829 +35830 POINT(41.65000078957285 73.8179440087573) bank35830 +35831 POINT(40.00628615006195 74.98755406530765) bank35831 +35832 POINT(41.52446423024587 74.18283776795768) bank35832 +35833 POINT(40.333040219203966 74.7412887730381) bank35833 +35834 POINT(39.77443648797008 74.38353715700025) bank35834 +35835 POINT(41.18410740150366 73.09555849005636) bank35835 +35836 POINT(40.277077556926514 74.45605667832706) bank35836 +35837 POINT(40.96616500467308 74.16605981286584) bank35837 +35838 POINT(39.878623561744924 73.68976792959074) bank35838 +35839 POINT(41.16289630175207 73.30130703546045) bank35839 +35840 POINT(41.629923575502254 74.93186616746448) bank35840 +35841 POINT(40.94011426340198 74.12806117074385) bank35841 +35842 POINT(40.65736674834733 74.00796788932888) bank35842 +35843 POINT(40.323725475971976 74.70732683747525) bank35843 +35844 POINT(41.339353961229776 73.64569872095889) bank35844 +35845 POINT(40.41194895281833 74.76965081190394) bank35845 +35846 POINT(39.83079393442851 73.54942899828072) bank35846 +35847 POINT(40.36181207147781 73.91698687501055) bank35847 +35848 POINT(39.76287785767496 73.7696692774037) bank35848 +35849 POINT(40.87267183500481 74.24107900997765) bank35849 +35850 POINT(41.28272888520928 74.66000278824721) bank35850 +35851 POINT(41.47178148542066 73.36808195921218) bank35851 +35852 POINT(41.67453635457038 73.83237002161951) bank35852 +35853 POINT(41.70461634063201 74.21968174186622) bank35853 +35854 POINT(41.52791073148403 73.46512702836544) bank35854 +35855 POINT(40.20178327525981 73.31003323259395) bank35855 +35856 POINT(41.490688248448784 73.63691861547662) bank35856 +35857 POINT(41.54433334483905 73.97806545891055) bank35857 +35858 POINT(40.66325140778062 73.33960404121909) bank35858 +35859 POINT(40.232163227049426 73.13083690983352) bank35859 +35860 POINT(40.77327915009221 74.94831491876225) bank35860 +35861 POINT(40.07351857732328 74.45808133734583) bank35861 +35862 POINT(40.23945751958177 73.39892989494514) bank35862 +35863 POINT(40.757562235542785 73.5502261182484) bank35863 +35864 POINT(39.989506650492835 73.17013269880462) bank35864 +35865 POINT(39.79478409446764 73.11371513688835) bank35865 +35866 POINT(40.915026584844526 74.94500383867907) bank35866 +35867 POINT(40.342421867302505 74.83899919371204) bank35867 +35868 POINT(39.90690930453472 73.35254062301095) bank35868 +35869 POINT(40.16947144216453 75.00582913846353) bank35869 +35870 POINT(40.9824301793041 73.8729708682995) bank35870 +35871 POINT(40.50032568180635 73.33276948938155) bank35871 +35872 POINT(41.392757160340835 73.4184076109637) bank35872 +35873 POINT(41.69173502821869 73.27084606672277) bank35873 +35874 POINT(39.849263873245754 74.07730163688133) bank35874 +35875 POINT(40.353756785595266 74.82593329298216) bank35875 +35876 POINT(40.80577069293783 74.6267039373551) bank35876 +35877 POINT(41.66926269959656 73.56439595708169) bank35877 +35878 POINT(39.74277219830673 74.52075197868261) bank35878 +35879 POINT(41.350453929995425 73.66248815583808) bank35879 +35880 POINT(41.68161391286913 74.43269507945872) bank35880 +35881 POINT(41.129603788889625 74.75763282638366) bank35881 +35882 POINT(39.878766644794446 74.67762354965825) bank35882 +35883 POINT(40.073413786089915 73.8189671182359) bank35883 +35884 POINT(39.86115487756841 74.08424271533443) bank35884 +35885 POINT(40.20757787842153 73.50208803545809) bank35885 +35886 POINT(39.810472531297066 74.68800201936732) bank35886 +35887 POINT(40.44254992261377 73.90579000805664) bank35887 +35888 POINT(39.736942310308386 74.46311806415116) bank35888 +35889 POINT(40.328471564824866 73.22638427031087) bank35889 +35890 POINT(40.6052190930244 74.73090242033042) bank35890 +35891 POINT(40.64186074037594 74.07557185907798) bank35891 +35892 POINT(40.98501996348118 73.36409313429849) bank35892 +35893 POINT(41.59580413020927 73.42740774626729) bank35893 +35894 POINT(39.78013290188663 73.35928547588658) bank35894 +35895 POINT(40.86033917355469 74.79480918995299) bank35895 +35896 POINT(39.772092508137916 73.24713248703891) bank35896 +35897 POINT(40.33027646010551 73.60158561837879) bank35897 +35898 POINT(41.119836384116894 74.21107916625989) bank35898 +35899 POINT(40.770676001916016 74.61921051747069) bank35899 +35900 POINT(40.59798663598147 74.63212067836432) bank35900 +35901 POINT(40.75020381530217 74.44206636800605) bank35901 +35902 POINT(39.86590143987124 73.57302917896445) bank35902 +35903 POINT(41.08882374989579 73.82607452078514) bank35903 +35904 POINT(39.998496664978056 74.63852553052459) bank35904 +35905 POINT(41.66406530588581 73.53020112733259) bank35905 +35906 POINT(41.50035312193746 74.70706843948548) bank35906 +35907 POINT(41.303544695432414 73.15078768362886) bank35907 +35908 POINT(40.53213977036066 73.94893810217087) bank35908 +35909 POINT(40.46067460725253 73.89350243024835) bank35909 +35910 POINT(39.74436819561505 73.71825195223767) bank35910 +35911 POINT(39.92895068239795 73.24051282891146) bank35911 +35912 POINT(39.73630263797946 74.69018871138142) bank35912 +35913 POINT(40.06847656471679 74.6028131807122) bank35913 +35914 POINT(41.637220043115114 74.06374345456214) bank35914 +35915 POINT(39.96599750918159 74.55917448568229) bank35915 +35916 POINT(39.76993644102145 74.54309454700307) bank35916 +35917 POINT(41.59834719061508 73.12499402757916) bank35917 +35918 POINT(40.619044159385254 73.52238587971416) bank35918 +35919 POINT(39.885030503520376 73.69260848768215) bank35919 +35920 POINT(40.123821020549215 74.24158113126163) bank35920 +35921 POINT(41.12840784006415 73.49382825380515) bank35921 +35922 POINT(40.10143191034706 73.40942871131591) bank35922 +35923 POINT(40.36742025300006 73.42033114138304) bank35923 +35924 POINT(40.221361062402906 74.93051976833932) bank35924 +35925 POINT(41.67941055589938 74.04878237259044) bank35925 +35926 POINT(39.93634368816834 73.80917679464268) bank35926 +35927 POINT(40.607054725406186 74.3791231860915) bank35927 +35928 POINT(41.6842663298358 74.66965909548286) bank35928 +35929 POINT(40.9525348908475 73.76529885987588) bank35929 +35930 POINT(41.22603244336751 74.60730231077783) bank35930 +35931 POINT(40.01033295222123 73.68223375519398) bank35931 +35932 POINT(40.93345481333267 74.54412849181506) bank35932 +35933 POINT(40.783676577859026 73.2091571800785) bank35933 +35934 POINT(39.99471788198189 73.52716467644592) bank35934 +35935 POINT(40.33132577886665 74.54261608310267) bank35935 +35936 POINT(39.93449193178088 74.36746252263043) bank35936 +35937 POINT(40.72964066694363 73.0313318998594) bank35937 +35938 POINT(40.783368475557715 74.10116159905218) bank35938 +35939 POINT(40.30051053833476 74.09482001421384) bank35939 +35940 POINT(40.224105983247235 73.35437566216478) bank35940 +35941 POINT(39.85015323232709 73.57176724175538) bank35941 +35942 POINT(40.01893408454675 73.99326109836264) bank35942 +35943 POINT(40.365168687702436 74.02042731986081) bank35943 +35944 POINT(41.34405496479169 74.76097309568262) bank35944 +35945 POINT(40.12807113071535 73.2289894240296) bank35945 +35946 POINT(40.966357728102814 73.50053292069008) bank35946 +35947 POINT(40.70782403989723 73.75073718477714) bank35947 +35948 POINT(40.77153414392463 74.99261405993904) bank35948 +35949 POINT(41.40036754903346 73.5677383172482) bank35949 +35950 POINT(41.233930563749155 73.081509994711) bank35950 +35951 POINT(41.05516170464883 74.48923810232385) bank35951 +35952 POINT(39.86489997174646 73.83215088619114) bank35952 +35953 POINT(39.8157925125746 73.0629773044425) bank35953 +35954 POINT(40.031051212301364 73.42604805764937) bank35954 +35955 POINT(41.03068832747062 74.89737138022146) bank35955 +35956 POINT(40.21844261076492 74.70954621997663) bank35956 +35957 POINT(40.42190406198857 74.70918772635703) bank35957 +35958 POINT(40.732554607024326 74.68509989169101) bank35958 +35959 POINT(40.114678766323046 74.37319829782913) bank35959 +35960 POINT(41.22869997031953 73.55274914275124) bank35960 +35961 POINT(40.305525809292064 73.06342692994025) bank35961 +35962 POINT(39.80197942751083 73.70904319839674) bank35962 +35963 POINT(40.681518017192104 74.24407136300616) bank35963 +35964 POINT(40.48483410517092 74.22439107097485) bank35964 +35965 POINT(39.847303251576676 74.26939756751196) bank35965 +35966 POINT(39.96295042828217 74.23582136514557) bank35966 +35967 POINT(40.8471926583615 74.17393686441117) bank35967 +35968 POINT(41.15706682615194 74.6442557606755) bank35968 +35969 POINT(39.82509325298366 73.9453278270892) bank35969 +35970 POINT(40.5970386213372 73.32832177780604) bank35970 +35971 POINT(40.89203547907615 74.49032741499883) bank35971 +35972 POINT(40.765106828671875 73.87835794523639) bank35972 +35973 POINT(40.03878179418566 73.68325490048191) bank35973 +35974 POINT(40.27692281855267 73.06812046758914) bank35974 +35975 POINT(40.758171134711276 74.91893735049273) bank35975 +35976 POINT(41.25946597237552 73.99887276016166) bank35976 +35977 POINT(40.81019761439248 74.76046934464877) bank35977 +35978 POINT(40.39886025005569 74.9114400256033) bank35978 +35979 POINT(40.50985463353356 74.53957130875385) bank35979 +35980 POINT(40.24366560695613 74.74683743841905) bank35980 +35981 POINT(41.195812556534854 73.01097749247349) bank35981 +35982 POINT(40.08275007167288 73.74860375047817) bank35982 +35983 POINT(40.315271560575184 74.31612771352242) bank35983 +35984 POINT(40.5000779933583 74.11021940402192) bank35984 +35985 POINT(39.87452771090358 74.68600708835919) bank35985 +35986 POINT(40.83380536984827 74.68829148783452) bank35986 +35987 POINT(40.777850328561975 74.7589991206654) bank35987 +35988 POINT(41.539604578982555 73.67227364468263) bank35988 +35989 POINT(40.07731696380124 73.88946652463865) bank35989 +35990 POINT(39.95377228489248 73.92453124708699) bank35990 +35991 POINT(41.17191370797687 74.93495591646916) bank35991 +35992 POINT(41.29186861088875 74.83294032859193) bank35992 +35993 POINT(40.17451476421704 74.5486260092328) bank35993 +35994 POINT(41.08313379444389 74.06711522877097) bank35994 +35995 POINT(41.53414851814016 73.54136347791703) bank35995 +35996 POINT(41.635491936447494 74.26712685890091) bank35996 +35997 POINT(41.68351162195433 73.37910598812564) bank35997 +35998 POINT(40.52477889458904 74.24237835563302) bank35998 +35999 POINT(40.620726276632574 74.49388121878621) bank35999 +36000 POINT(40.3850199807112 73.88646816139872) bank36000 +36001 POINT(40.648222365854465 73.36016325093938) bank36001 +36002 POINT(41.53866839739427 73.29734349167052) bank36002 +36003 POINT(40.85949648442772 74.44728075664123) bank36003 +36004 POINT(41.46839462267727 73.95184663228835) bank36004 +36005 POINT(40.807109780618234 74.44370131133701) bank36005 +36006 POINT(40.10903332481148 74.87130647256367) bank36006 +36007 POINT(40.43386874767092 74.56225448589758) bank36007 +36008 POINT(40.031041520915615 74.8845324000982) bank36008 +36009 POINT(40.53239047117577 73.0416238531385) bank36009 +36010 POINT(40.3585171808503 73.28900779841335) bank36010 +36011 POINT(40.14603228581166 73.54430500203713) bank36011 +36012 POINT(40.44402140332801 73.29323261351584) bank36012 +36013 POINT(40.553042301874456 74.47972371676327) bank36013 +36014 POINT(40.50735436518617 73.91694941153376) bank36014 +36015 POINT(41.69134870998206 74.98181608866271) bank36015 +36016 POINT(40.64671538036832 73.71609411216262) bank36016 +36017 POINT(41.25775007579468 74.95476212579884) bank36017 +36018 POINT(40.17913688375442 73.86116223504398) bank36018 +36019 POINT(39.909809210563736 73.03016787182233) bank36019 +36020 POINT(40.06021846552463 73.01950663177645) bank36020 +36021 POINT(41.56641048045152 73.45955670107718) bank36021 +36022 POINT(41.60850508678462 73.7992483956461) bank36022 +36023 POINT(40.991632863148766 74.97800820344813) bank36023 +36024 POINT(41.56477165273991 74.45470857720572) bank36024 +36025 POINT(41.403919648173975 73.24956755915947) bank36025 +36026 POINT(41.19684591651298 74.7881520899386) bank36026 +36027 POINT(41.32633391004596 74.66401532340265) bank36027 +36028 POINT(41.310929957916024 74.36669753759732) bank36028 +36029 POINT(39.79374978348235 74.22918016055657) bank36029 +36030 POINT(41.423981512126595 73.23826933892622) bank36030 +36031 POINT(41.267168577040195 73.50373024819856) bank36031 +36032 POINT(40.84623400531588 74.56324710670198) bank36032 +36033 POINT(40.9090992670799 73.11324601039895) bank36033 +36034 POINT(39.95575796224304 74.60997169604371) bank36034 +36035 POINT(41.17952786163565 73.74783766098703) bank36035 +36036 POINT(41.6693534219366 73.58419095331357) bank36036 +36037 POINT(40.82936319751625 74.18223079049017) bank36037 +36038 POINT(41.438958188398715 74.27288791834707) bank36038 +36039 POINT(41.24148546853143 73.0743323038678) bank36039 +36040 POINT(40.84509655139571 73.74336666898688) bank36040 +36041 POINT(40.72765154142947 73.62852924737606) bank36041 +36042 POINT(40.48597967603804 74.66884938121923) bank36042 +36043 POINT(39.83701792623626 73.29437586797565) bank36043 +36044 POINT(39.774843719704165 73.7480948771097) bank36044 +36045 POINT(41.47901281829413 73.94734727973405) bank36045 +36046 POINT(41.66086735039035 73.31763293229923) bank36046 +36047 POINT(41.34332049609542 74.501759461977) bank36047 +36048 POINT(39.797204902258635 74.00254895908104) bank36048 +36049 POINT(41.39683086377884 74.64019879571939) bank36049 +36050 POINT(40.61550984614539 74.78143311419284) bank36050 +36051 POINT(41.23539486763848 74.1543509505982) bank36051 +36052 POINT(39.77888825119965 73.26045357286884) bank36052 +36053 POINT(40.9087206368156 73.76904008148506) bank36053 +36054 POINT(41.512922648473634 75.00022691022713) bank36054 +36055 POINT(41.51024778651184 74.49218700294844) bank36055 +36056 POINT(41.226109861996605 74.89379676107212) bank36056 +36057 POINT(40.228536319916756 74.53923997503595) bank36057 +36058 POINT(39.95773507147087 74.63666247493649) bank36058 +36059 POINT(41.310796592847744 74.717905944712) bank36059 +36060 POINT(40.17847708788269 73.88593290220822) bank36060 +36061 POINT(40.34639311564122 73.84522622271705) bank36061 +36062 POINT(40.11384998485913 74.74692008601829) bank36062 +36063 POINT(41.588890302450814 73.07735734769557) bank36063 +36064 POINT(40.437380221487146 73.81509052573693) bank36064 +36065 POINT(41.32798622473323 74.52204621765257) bank36065 +36066 POINT(41.10357138932434 74.15159246936616) bank36066 +36067 POINT(40.60688030580336 74.13324998099114) bank36067 +36068 POINT(40.8935208231262 74.82340678991714) bank36068 +36069 POINT(41.68459851541382 73.8644911405106) bank36069 +36070 POINT(40.17372185699238 74.28394928267197) bank36070 +36071 POINT(40.95528594944356 73.56736275064232) bank36071 +36072 POINT(41.148214872815856 74.26749445580616) bank36072 +36073 POINT(40.30827607069924 73.30049843619554) bank36073 +36074 POINT(41.182789474869026 73.6126459014222) bank36074 +36075 POINT(40.15971904019135 74.53526675654909) bank36075 +36076 POINT(41.347098228523464 73.49440326001445) bank36076 +36077 POINT(39.999793664617556 73.61172089742657) bank36077 +36078 POINT(40.15490382821996 74.97398288789769) bank36078 +36079 POINT(40.003071232898805 74.32482289603936) bank36079 +36080 POINT(40.65117186899224 74.39762059831867) bank36080 +36081 POINT(41.02272662122869 73.7284796862546) bank36081 +36082 POINT(40.81764240931192 74.61872367343636) bank36082 +36083 POINT(40.01219940850938 74.42514156502087) bank36083 +36084 POINT(41.51014729856244 74.48519477821056) bank36084 +36085 POINT(41.00119045274318 74.492052566416) bank36085 +36086 POINT(39.95578958199312 74.00300490283679) bank36086 +36087 POINT(40.63099510865307 74.11700696520658) bank36087 +36088 POINT(40.625583650282 74.17537902993091) bank36088 +36089 POINT(41.38034076435244 73.47585319058288) bank36089 +36090 POINT(39.95215762072097 74.84533599524886) bank36090 +36091 POINT(41.22926564038225 73.34179921425053) bank36091 +36092 POINT(40.1800770117864 74.42639608277966) bank36092 +36093 POINT(41.22309334550284 73.31016124697992) bank36093 +36094 POINT(41.56502921148071 73.64361234564718) bank36094 +36095 POINT(41.644952920266526 73.14337347404856) bank36095 +36096 POINT(40.826685449371 74.18477804202985) bank36096 +36097 POINT(40.61052624328857 74.97882903392481) bank36097 +36098 POINT(40.78538876271997 74.1873073884633) bank36098 +36099 POINT(41.61461359700273 73.54563508699442) bank36099 +36100 POINT(41.31259699673486 73.88026930447268) bank36100 +36101 POINT(41.123858743483645 73.60740143911374) bank36101 +36102 POINT(41.235844049476945 74.92756608204837) bank36102 +36103 POINT(41.05241170047588 74.89196507777571) bank36103 +36104 POINT(40.053865923224244 73.4114191157882) bank36104 +36105 POINT(40.81686267852413 74.20300974906903) bank36105 +36106 POINT(40.26110270293061 74.47487249308253) bank36106 +36107 POINT(40.158181054398234 74.92588290323299) bank36107 +36108 POINT(41.33817266726303 74.57335646411309) bank36108 +36109 POINT(39.94305713881774 73.4262646536217) bank36109 +36110 POINT(40.509262971514524 73.54580517880149) bank36110 +36111 POINT(41.064274603719156 73.6693752222299) bank36111 +36112 POINT(39.97431372813632 73.19860425567634) bank36112 +36113 POINT(39.79190783601196 74.92823564920086) bank36113 +36114 POINT(39.858818273173455 73.59843435360173) bank36114 +36115 POINT(40.269381158811065 73.5700709830647) bank36115 +36116 POINT(40.56990538336188 73.12980922968927) bank36116 +36117 POINT(41.216727224142 74.5549021386056) bank36117 +36118 POINT(40.22101331054275 74.7193189465016) bank36118 +36119 POINT(39.93910017376148 74.07755741880233) bank36119 +36120 POINT(40.2034953377045 73.95990184387885) bank36120 +36121 POINT(40.19839328305311 73.62157413656067) bank36121 +36122 POINT(41.030757881576484 74.60339017485995) bank36122 +36123 POINT(40.058158977042034 74.1794673375838) bank36123 +36124 POINT(40.56661051133121 74.41207567911971) bank36124 +36125 POINT(41.53489939622909 74.93028848703894) bank36125 +36126 POINT(40.50046566391173 74.5921528712975) bank36126 +36127 POINT(39.798053047775355 73.69651247706146) bank36127 +36128 POINT(40.25267244800772 74.66397788514827) bank36128 +36129 POINT(40.34011869039278 75.0025751837434) bank36129 +36130 POINT(40.78368119502108 74.74048619357207) bank36130 +36131 POINT(41.136629278990775 74.76838817420102) bank36131 +36132 POINT(41.18721409288657 73.21840729612892) bank36132 +36133 POINT(41.67313825100569 74.56808866246651) bank36133 +36134 POINT(39.958827726797736 73.21008455257514) bank36134 +36135 POINT(39.83816379371667 74.26424168692435) bank36135 +36136 POINT(40.92018654793325 73.3410742809892) bank36136 +36137 POINT(39.758088383066266 74.38564428005411) bank36137 +36138 POINT(41.235292509630696 73.03403798297433) bank36138 +36139 POINT(41.69989220820334 73.59825780283164) bank36139 +36140 POINT(40.990699553018906 74.55484750251095) bank36140 +36141 POINT(40.71047464560042 74.61301880130361) bank36141 +36142 POINT(39.946163507508906 74.6729112139461) bank36142 +36143 POINT(39.85409548610066 74.70319696177157) bank36143 +36144 POINT(41.64088065140684 73.54688441727025) bank36144 +36145 POINT(39.81393312379403 73.52034696008701) bank36145 +36146 POINT(40.84277668792913 74.35749294726274) bank36146 +36147 POINT(41.39449297846717 73.25187112426309) bank36147 +36148 POINT(41.42918391627037 74.31230400248316) bank36148 +36149 POINT(39.95949363460052 74.13721743032117) bank36149 +36150 POINT(40.64323697939213 74.32532699840448) bank36150 +36151 POINT(39.872542456830935 73.78226493199202) bank36151 +36152 POINT(40.27204985238996 74.98617397817803) bank36152 +36153 POINT(40.75286524335547 74.81132685443092) bank36153 +36154 POINT(40.633471056969384 74.94426469815467) bank36154 +36155 POINT(40.40133372619902 74.69753836737017) bank36155 +36156 POINT(40.55171050180082 74.45843349862812) bank36156 +36157 POINT(41.706259930758065 74.07367950534423) bank36157 +36158 POINT(39.74117286657457 73.87559444817965) bank36158 +36159 POINT(41.004727478855884 73.71450713020826) bank36159 +36160 POINT(40.48097964252939 73.97384837428187) bank36160 +36161 POINT(40.69771562073002 73.06649445131949) bank36161 +36162 POINT(41.419878800526455 74.12545634495237) bank36162 +36163 POINT(41.175005739929 74.8237121125931) bank36163 +36164 POINT(40.75051191312219 73.33447988959944) bank36164 +36165 POINT(41.68756454561867 74.67979043294469) bank36165 +36166 POINT(40.19588732250007 74.93937962248359) bank36166 +36167 POINT(41.177005353434616 73.33264884728351) bank36167 +36168 POINT(40.34328738676281 74.27286049460083) bank36168 +36169 POINT(40.57043490788906 73.84087955608055) bank36169 +36170 POINT(40.79021416276786 74.06668724154966) bank36170 +36171 POINT(40.11430317012662 73.44411715028963) bank36171 +36172 POINT(41.55900351817176 73.82588838233694) bank36172 +36173 POINT(41.3815560318433 74.36720091073357) bank36173 +36174 POINT(40.92380355404994 73.88560626756389) bank36174 +36175 POINT(41.68983186459848 73.19891252814885) bank36175 +36176 POINT(41.01780566631944 74.13065453576344) bank36176 +36177 POINT(39.893338004753296 74.37876077091455) bank36177 +36178 POINT(40.33176656281319 74.19805096687622) bank36178 +36179 POINT(39.87766612482367 73.65568341064456) bank36179 +36180 POINT(40.27531446860582 73.45290462726963) bank36180 +36181 POINT(41.424819220940336 73.49446256963024) bank36181 +36182 POINT(40.071148835254014 74.98351757525431) bank36182 +36183 POINT(40.15997831248102 73.0550657318527) bank36183 +36184 POINT(40.634152328506715 74.33006445238345) bank36184 +36185 POINT(40.59598545947627 73.84509304909429) bank36185 +36186 POINT(40.564517444739224 73.63081532121045) bank36186 +36187 POINT(41.68301490345591 74.9573873435736) bank36187 +36188 POINT(40.893217269790775 74.34711045572038) bank36188 +36189 POINT(40.681910491415294 74.10511804057543) bank36189 +36190 POINT(40.701423890479624 73.74645751202593) bank36190 +36191 POINT(40.198448710623936 74.15596261536217) bank36191 +36192 POINT(41.376057721054 74.2566276417898) bank36192 +36193 POINT(41.28563223121914 74.45977293300886) bank36193 +36194 POINT(39.86908168140586 74.05367014385526) bank36194 +36195 POINT(41.404514229119954 74.07575524253059) bank36195 +36196 POINT(40.21663452305099 73.6887782799064) bank36196 +36197 POINT(41.01041707186205 74.06380968924924) bank36197 +36198 POINT(41.62603613950154 73.14187986214135) bank36198 +36199 POINT(40.64219769171062 73.38117921933777) bank36199 +36200 POINT(40.51414369169546 73.08088005502434) bank36200 +36201 POINT(40.94972670509967 74.87376165063354) bank36201 +36202 POINT(39.948044623254525 73.3622735541061) bank36202 +36203 POINT(39.93666227055518 74.40583142099257) bank36203 +36204 POINT(39.85355083534528 73.70089751435414) bank36204 +36205 POINT(40.44351631718817 74.46238548559712) bank36205 +36206 POINT(40.514076064788945 74.80061464399657) bank36206 +36207 POINT(41.555764755742445 73.5906807887539) bank36207 +36208 POINT(40.650119468339135 73.75407617856645) bank36208 +36209 POINT(39.82629334662281 73.94241997677042) bank36209 +36210 POINT(41.2727511203633 74.4359792903011) bank36210 +36211 POINT(41.672115415523585 73.34085757795013) bank36211 +36212 POINT(41.19917845629524 73.94066613203398) bank36212 +36213 POINT(40.64156768789556 73.74769688022148) bank36213 +36214 POINT(41.282156800539155 74.37799162854448) bank36214 +36215 POINT(40.08206707814528 73.94181634931877) bank36215 +36216 POINT(40.98106828097396 73.12151942964176) bank36216 +36217 POINT(40.889834748204734 74.13326627240156) bank36217 +36218 POINT(39.98751736984058 74.92803718573717) bank36218 +36219 POINT(41.07863032583653 73.74729632705807) bank36219 +36220 POINT(40.58264872614375 74.11080066775288) bank36220 +36221 POINT(40.03583513837067 74.16843734464352) bank36221 +36222 POINT(40.7777946172037 74.56603798076473) bank36222 +36223 POINT(40.5003904414654 73.82348860439907) bank36223 +36224 POINT(41.03718473172813 73.567480210344) bank36224 +36225 POINT(41.26000685033816 74.21846081298128) bank36225 +36226 POINT(41.26336390246223 74.15090435551026) bank36226 +36227 POINT(39.92541567968314 73.1366827106805) bank36227 +36228 POINT(40.596714407417416 74.72161710872285) bank36228 +36229 POINT(39.848614820431266 74.27642068977077) bank36229 +36230 POINT(41.387097881133315 73.39885060894676) bank36230 +36231 POINT(41.062343469604485 73.37660410877922) bank36231 +36232 POINT(40.813735511723394 74.19858732286164) bank36232 +36233 POINT(40.033621062480556 73.30714727222964) bank36233 +36234 POINT(40.62975710850199 73.5644655027884) bank36234 +36235 POINT(40.46151330092551 74.01702366764151) bank36235 +36236 POINT(41.05272420676059 73.60953334949217) bank36236 +36237 POINT(40.13367319411564 73.07286935993955) bank36237 +36238 POINT(40.17052400593399 73.52603570131672) bank36238 +36239 POINT(41.61287624165813 73.74714012077364) bank36239 +36240 POINT(41.271306881380475 73.09214051398544) bank36240 +36241 POINT(41.13703236953415 73.66777214126986) bank36241 +36242 POINT(40.15068681820987 74.97510840606027) bank36242 +36243 POINT(41.32917047788403 74.58487037392409) bank36243 +36244 POINT(40.83869809190006 73.92235520266937) bank36244 +36245 POINT(40.386280434412136 74.54204730589345) bank36245 +36246 POINT(39.96699154055262 74.75017635393675) bank36246 +36247 POINT(41.6769933837785 73.65660556054121) bank36247 +36248 POINT(41.58523112201441 73.37782799618726) bank36248 +36249 POINT(40.97792920722279 73.32477916941059) bank36249 +36250 POINT(40.02650328423907 73.35427614113722) bank36250 +36251 POINT(40.09937391972189 74.31908856248742) bank36251 +36252 POINT(41.308060556890744 73.53058228573317) bank36252 +36253 POINT(40.64700488965232 74.01122348508095) bank36253 +36254 POINT(41.2233376849211 74.68990822934306) bank36254 +36255 POINT(41.385023879743294 73.5040381258148) bank36255 +36256 POINT(41.4835339175105 74.90370500028749) bank36256 +36257 POINT(40.692196577087856 74.29000832213909) bank36257 +36258 POINT(40.90194770212683 74.30468891864973) bank36258 +36259 POINT(41.078472024068915 73.62511859217011) bank36259 +36260 POINT(41.32343317285293 73.11145639871906) bank36260 +36261 POINT(41.26965516283729 74.37012571998243) bank36261 +36262 POINT(40.30682589116323 73.82657405467452) bank36262 +36263 POINT(40.346244668057466 73.75947235459145) bank36263 +36264 POINT(41.15138783400873 73.87657019953316) bank36264 +36265 POINT(39.78908351637678 74.35443698007872) bank36265 +36266 POINT(40.94133461525163 74.04076508713388) bank36266 +36267 POINT(41.42545356414589 73.46407574107054) bank36267 +36268 POINT(40.19030356013184 73.3399887216611) bank36268 +36269 POINT(40.24082272838905 73.9200319209183) bank36269 +36270 POINT(39.875528080579464 74.41462512162323) bank36270 +36271 POINT(40.063935617575936 74.85648208946841) bank36271 +36272 POINT(41.00371860167419 73.99956393488453) bank36272 +36273 POINT(41.52252575739565 74.6640761886659) bank36273 +36274 POINT(40.08873470321783 73.80150516333819) bank36274 +36275 POINT(39.735101494696345 73.24184610892583) bank36275 +36276 POINT(40.57645185641091 73.60990309861053) bank36276 +36277 POINT(41.197878528301764 74.7545387905027) bank36277 +36278 POINT(40.58297898068459 73.7887997012021) bank36278 +36279 POINT(41.29431999011274 73.94741314830561) bank36279 +36280 POINT(40.21130070728959 73.74455173457388) bank36280 +36281 POINT(40.645634324832756 73.11103148677316) bank36281 +36282 POINT(40.969584815290396 73.41611002948353) bank36282 +36283 POINT(39.90148455181935 74.73461772927281) bank36283 +36284 POINT(40.1953802929397 74.14033484339389) bank36284 +36285 POINT(39.755941936168 74.85530259077696) bank36285 +36286 POINT(40.462545769120254 73.50661965828066) bank36286 +36287 POINT(41.42687638383026 74.81888658646398) bank36287 +36288 POINT(40.0544720968611 74.83936655130373) bank36288 +36289 POINT(40.619717426243106 74.83212865592229) bank36289 +36290 POINT(41.2813418627391 74.63297660291917) bank36290 +36291 POINT(41.05040981492676 73.23527899718799) bank36291 +36292 POINT(41.624360680510094 74.81561272324181) bank36292 +36293 POINT(41.409985888319376 73.74184089382673) bank36293 +36294 POINT(41.47196268187553 74.67871516248958) bank36294 +36295 POINT(40.667601175610784 73.39757715886819) bank36295 +36296 POINT(41.66489064264838 73.25587578974606) bank36296 +36297 POINT(40.23840661366931 73.35069204341981) bank36297 +36298 POINT(41.11641659678027 74.41846486541142) bank36298 +36299 POINT(40.504935104706504 73.47049800320552) bank36299 +36300 POINT(41.493175765646 73.81040908308634) bank36300 +36301 POINT(41.43802988446401 73.90922406993974) bank36301 +36302 POINT(40.186101072457674 73.62172287240087) bank36302 +36303 POINT(39.99917219997014 74.16571026769871) bank36303 +36304 POINT(41.256195331683365 74.01032143223284) bank36304 +36305 POINT(40.4997074120375 73.79099007995892) bank36305 +36306 POINT(41.09328325549159 73.09120707735279) bank36306 +36307 POINT(40.957727035205394 73.88010466092089) bank36307 +36308 POINT(39.96504676869365 74.14476528590797) bank36308 +36309 POINT(41.07689271579374 74.21752895155797) bank36309 +36310 POINT(40.175574566055175 74.75207839879441) bank36310 +36311 POINT(41.32420243100056 73.30234300452686) bank36311 +36312 POINT(40.74886099941557 74.83367053123388) bank36312 +36313 POINT(40.286303984655206 74.67031444440653) bank36313 +36314 POINT(40.285395174545414 74.86915684733081) bank36314 +36315 POINT(40.292254602231935 74.01146296553827) bank36315 +36316 POINT(39.867645542396374 73.3392827179102) bank36316 +36317 POINT(41.47059240042966 73.88021391665569) bank36317 +36318 POINT(39.870773558687894 74.67867960862884) bank36318 +36319 POINT(41.480074591381026 73.66553022496869) bank36319 +36320 POINT(40.95653073050368 74.30521845417613) bank36320 +36321 POINT(39.90118108817126 73.12648725456475) bank36321 +36322 POINT(40.96596418954799 74.85454914751445) bank36322 +36323 POINT(41.07041878110919 74.54703878043867) bank36323 +36324 POINT(41.553791461068776 74.07232245042795) bank36324 +36325 POINT(40.982845994291175 74.46077528078914) bank36325 +36326 POINT(40.245586157189656 74.10666078556581) bank36326 +36327 POINT(41.383720621533804 74.3976704831872) bank36327 +36328 POINT(40.99059954009581 73.16709564551623) bank36328 +36329 POINT(40.49487525796268 73.107395084688) bank36329 +36330 POINT(40.53193413534343 73.42917555648926) bank36330 +36331 POINT(40.483491607781566 74.53134553567429) bank36331 +36332 POINT(40.812247687040724 73.38266842147304) bank36332 +36333 POINT(41.35841952768213 74.06325903525402) bank36333 +36334 POINT(41.0599870294489 74.3628691395171) bank36334 +36335 POINT(41.200121886117344 73.06356488673403) bank36335 +36336 POINT(40.757371397257266 73.87606047561148) bank36336 +36337 POINT(41.25760821955435 74.56087760231605) bank36337 +36338 POINT(41.28582111403553 74.65544681591865) bank36338 +36339 POINT(40.74717314786484 73.70363033082238) bank36339 +36340 POINT(41.67476571427179 73.88437465739831) bank36340 +36341 POINT(39.73893505907912 74.57006517170949) bank36341 +36342 POINT(39.714895801386206 74.26895252705181) bank36342 +36343 POINT(40.12145168335594 73.32387923343877) bank36343 +36344 POINT(41.15785202122287 73.6055481517931) bank36344 +36345 POINT(41.192499329839904 74.2929084511733) bank36345 +36346 POINT(41.47252081425498 73.81610545459836) bank36346 +36347 POINT(41.646728178855106 73.34553871110748) bank36347 +36348 POINT(41.617775378633695 74.96563596744654) bank36348 +36349 POINT(40.54048243167387 73.52447510729864) bank36349 +36350 POINT(39.859562659862014 73.44968796362171) bank36350 +36351 POINT(39.83310459061752 73.51695462134978) bank36351 +36352 POINT(40.941566023314046 73.55730440970783) bank36352 +36353 POINT(39.93888578997104 74.355136723334) bank36353 +36354 POINT(39.890296878835095 74.04513068835264) bank36354 +36355 POINT(41.47760553438639 74.47395074984148) bank36355 +36356 POINT(40.14535530909005 73.38068936793776) bank36356 +36357 POINT(41.29418077676676 73.41730390966195) bank36357 +36358 POINT(40.456318181321215 74.86781027145531) bank36358 +36359 POINT(40.2588618574877 73.29043112163048) bank36359 +36360 POINT(40.3449693408178 74.63641581830119) bank36360 +36361 POINT(39.787892138812836 74.63246222811033) bank36361 +36362 POINT(41.64824570279145 74.83488144162222) bank36362 +36363 POINT(39.7292510343825 73.8696947460136) bank36363 +36364 POINT(40.635503699045074 74.38364077738876) bank36364 +36365 POINT(39.74864934373767 74.37133319817039) bank36365 +36366 POINT(40.2002022263967 74.60358271249113) bank36366 +36367 POINT(41.54604336155212 74.40006004851591) bank36367 +36368 POINT(40.59223883740269 73.31578828209636) bank36368 +36369 POINT(40.19090287456975 73.43189156933478) bank36369 +36370 POINT(39.7248304141929 74.74817710090636) bank36370 +36371 POINT(40.100385457831436 73.71127099782106) bank36371 +36372 POINT(40.24632110346722 74.64916999873941) bank36372 +36373 POINT(41.160403727829376 74.54898479509508) bank36373 +36374 POINT(40.65613921106072 73.16905116633056) bank36374 +36375 POINT(40.67076264676019 73.43738482848454) bank36375 +36376 POINT(39.83349311461445 74.65902538903018) bank36376 +36377 POINT(41.27121944736642 73.66172188037048) bank36377 +36378 POINT(40.65526421614723 74.7020374839817) bank36378 +36379 POINT(40.30529174084257 73.09809665771077) bank36379 +36380 POINT(41.3495198954077 74.79668758936982) bank36380 +36381 POINT(41.339250158502914 74.35093875768496) bank36381 +36382 POINT(41.35093476055714 74.09612546794942) bank36382 +36383 POINT(41.05689112566172 73.37668661614947) bank36383 +36384 POINT(40.932838399700664 74.43396482522374) bank36384 +36385 POINT(39.95716109838349 74.01694457743984) bank36385 +36386 POINT(41.32839617088339 74.57373004901551) bank36386 +36387 POINT(40.330338236440646 74.31816058825457) bank36387 +36388 POINT(40.26965928499913 73.05620513927569) bank36388 +36389 POINT(41.159838713940424 73.27086370633981) bank36389 +36390 POINT(39.957551772025944 73.83626216462147) bank36390 +36391 POINT(41.068345570149404 73.03922329470326) bank36391 +36392 POINT(41.53668526516424 73.90456543870803) bank36392 +36393 POINT(39.981886875203415 73.05471591047534) bank36393 +36394 POINT(40.005409408302455 73.45631659370576) bank36394 +36395 POINT(41.20835992207979 74.72833176793947) bank36395 +36396 POINT(40.32954287656346 73.93224385816312) bank36396 +36397 POINT(39.98645449429068 73.29001906854427) bank36397 +36398 POINT(39.92440880521072 73.34684853588534) bank36398 +36399 POINT(41.02590929238237 73.43520353490028) bank36399 +36400 POINT(40.72424830451509 73.06323673563404) bank36400 +36401 POINT(40.05722497212559 74.84731986051104) bank36401 +36402 POINT(39.7392335880108 74.47497404821557) bank36402 +36403 POINT(39.81254484336119 74.51897029391239) bank36403 +36404 POINT(41.516235233542986 74.55705195201398) bank36404 +36405 POINT(41.32746420829263 73.96127849505645) bank36405 +36406 POINT(41.43136494763775 74.65817204991016) bank36406 +36407 POINT(40.79081463921328 74.44508825822147) bank36407 +36408 POINT(40.75732840810398 73.51140511365861) bank36408 +36409 POINT(41.142575696696206 74.38129602177594) bank36409 +36410 POINT(40.61530298606836 73.4846527188237) bank36410 +36411 POINT(40.576973084827664 74.38135681534277) bank36411 +36412 POINT(41.33592277440575 74.33221449032435) bank36412 +36413 POINT(40.44812127471237 73.88496625835538) bank36413 +36414 POINT(41.05721072049065 73.75223592382515) bank36414 +36415 POINT(39.95782169299528 74.64542973849923) bank36415 +36416 POINT(39.73120135207511 73.97527626271403) bank36416 +36417 POINT(40.40537332300876 74.31687805296049) bank36417 +36418 POINT(39.93388859473353 73.76522804286974) bank36418 +36419 POINT(40.748017342796395 74.51570305432487) bank36419 +36420 POINT(40.520000120441054 73.16826085333304) bank36420 +36421 POINT(40.21480313941884 74.12433730080113) bank36421 +36422 POINT(40.356375631900825 73.70353016378864) bank36422 +36423 POINT(41.580544016630455 73.20437035145731) bank36423 +36424 POINT(41.050522594119116 73.64912177960892) bank36424 +36425 POINT(41.220270598544275 73.38758175230454) bank36425 +36426 POINT(40.717176332856596 73.65595845072467) bank36426 +36427 POINT(40.97262279984134 74.59149674879299) bank36427 +36428 POINT(40.882180308460974 74.28316233537986) bank36428 +36429 POINT(41.14609114997719 73.73204221909309) bank36429 +36430 POINT(40.64910383782775 74.20542579206554) bank36430 +36431 POINT(41.526657615753805 73.90782473408888) bank36431 +36432 POINT(40.58170916637716 74.02370728993718) bank36432 +36433 POINT(39.979797295365124 74.1804515066193) bank36433 +36434 POINT(40.53783223683897 73.76164473101949) bank36434 +36435 POINT(40.46352115049775 73.13847312698365) bank36435 +36436 POINT(39.84139081260986 74.95895847443316) bank36436 +36437 POINT(39.79692054812806 73.98804670609927) bank36437 +36438 POINT(41.44987898230974 74.999412154198) bank36438 +36439 POINT(41.623975855833535 74.6804792894033) bank36439 +36440 POINT(40.81355648945817 74.3843114232641) bank36440 +36441 POINT(40.5240370355873 73.25818115407829) bank36441 +36442 POINT(41.665066116622114 74.53756005857811) bank36442 +36443 POINT(40.83418612004852 74.18827052998802) bank36443 +36444 POINT(40.9342405454666 74.98960371347206) bank36444 +36445 POINT(40.0804076773129 73.07991055647251) bank36445 +36446 POINT(41.14205304752354 74.05013048552654) bank36446 +36447 POINT(40.096914949954574 73.3158182178074) bank36447 +36448 POINT(40.630874464096614 73.60980958786027) bank36448 +36449 POINT(41.27999893166506 74.1095563964193) bank36449 +36450 POINT(41.2363583756772 74.33899323605534) bank36450 +36451 POINT(41.35352164643544 74.54776445396257) bank36451 +36452 POINT(40.80624432317445 74.8124555844322) bank36452 +36453 POINT(41.2929007432719 73.87925214543723) bank36453 +36454 POINT(41.39166796655426 74.478738244046) bank36454 +36455 POINT(41.289749052491764 74.16805817485827) bank36455 +36456 POINT(41.52141934934279 74.17798736075535) bank36456 +36457 POINT(39.860452747147264 73.699286296315) bank36457 +36458 POINT(41.087210600505216 74.19755506952812) bank36458 +36459 POINT(41.46650175451611 73.63680480132471) bank36459 +36460 POINT(40.2129359756257 74.41617947585682) bank36460 +36461 POINT(40.70263493777774 74.12988127387148) bank36461 +36462 POINT(41.335173746456405 74.99979317491541) bank36462 +36463 POINT(39.865625572969456 73.91850220882144) bank36463 +36464 POINT(40.063274526200736 74.15914189278766) bank36464 +36465 POINT(40.654573338755604 73.19172827717772) bank36465 +36466 POINT(41.06803285329779 74.44712957929994) bank36466 +36467 POINT(39.87815895655878 73.20769673489572) bank36467 +36468 POINT(40.99078309334632 74.89543733577939) bank36468 +36469 POINT(40.84939315770716 73.6827135447027) bank36469 +36470 POINT(41.0144152751064 74.81400195666258) bank36470 +36471 POINT(40.83813197527017 73.8611478823242) bank36471 +36472 POINT(40.14971900234024 73.05066940702466) bank36472 +36473 POINT(41.05650573858832 74.81090741274834) bank36473 +36474 POINT(40.73124779520111 74.57755835164876) bank36474 +36475 POINT(41.6324798658919 74.66126865119827) bank36475 +36476 POINT(40.611047735319545 73.85039114712532) bank36476 +36477 POINT(39.80857697872091 73.27976882193748) bank36477 +36478 POINT(40.31013680293181 73.3101058792434) bank36478 +36479 POINT(39.94310779292655 74.62251473151468) bank36479 +36480 POINT(41.52960523895825 73.4503046296248) bank36480 +36481 POINT(41.119157807752664 74.13543197658737) bank36481 +36482 POINT(40.9119475012911 74.30943043176174) bank36482 +36483 POINT(41.37196879057659 73.23164690081713) bank36483 +36484 POINT(39.97311498157604 74.86383773867325) bank36484 +36485 POINT(39.77819699972692 73.63508854456465) bank36485 +36486 POINT(40.52784541707539 74.05419635164239) bank36486 +36487 POINT(40.38787580571489 74.96228841568964) bank36487 +36488 POINT(39.96034157919756 74.44599496770056) bank36488 +36489 POINT(40.6996736860321 74.00185987185235) bank36489 +36490 POINT(39.73795356742551 74.21514533240452) bank36490 +36491 POINT(41.63105065833975 73.29499129107846) bank36491 +36492 POINT(39.82138117858734 73.09679855433538) bank36492 +36493 POINT(40.53504823227013 73.10207542481567) bank36493 +36494 POINT(40.59298424639744 74.96251453260594) bank36494 +36495 POINT(41.22894509141147 73.56882634622127) bank36495 +36496 POINT(40.41288717601897 73.45679365342177) bank36496 +36497 POINT(41.66428492234002 74.61624895890839) bank36497 +36498 POINT(40.46498384166714 73.79518928688081) bank36498 +36499 POINT(39.85900441286025 74.55567170286027) bank36499 +36500 POINT(41.287670731328674 74.67616321325663) bank36500 +36501 POINT(41.59044805239893 74.34113786108085) bank36501 +36502 POINT(40.608338603734815 74.5826277208458) bank36502 +36503 POINT(40.52461837977656 74.45866016645905) bank36503 +36504 POINT(40.74812538926901 74.67771317831898) bank36504 +36505 POINT(40.801980141527785 73.74412698663515) bank36505 +36506 POINT(40.027014354503955 73.2261527968751) bank36506 +36507 POINT(40.66248909081988 73.12033458956671) bank36507 +36508 POINT(41.357158962126405 73.29622643766095) bank36508 +36509 POINT(40.931278407499676 74.91109662706484) bank36509 +36510 POINT(41.01203965798594 73.25956709988996) bank36510 +36511 POINT(40.64878316458425 73.17879679727405) bank36511 +36512 POINT(41.00174555211288 73.22765468823758) bank36512 +36513 POINT(40.188021049023845 73.09809559243797) bank36513 +36514 POINT(40.42851545311578 74.1556983802875) bank36514 +36515 POINT(41.04389052730607 74.39808572514308) bank36515 +36516 POINT(40.51615824483887 74.82961691223127) bank36516 +36517 POINT(41.27957060256975 73.70004797018171) bank36517 +36518 POINT(40.222322908936206 74.36125675192456) bank36518 +36519 POINT(41.561609244947725 74.0338755371731) bank36519 +36520 POINT(40.0354273803199 73.7052079617483) bank36520 +36521 POINT(40.39311408617023 74.0350641144859) bank36521 +36522 POINT(40.522566857579974 74.21112767827807) bank36522 +36523 POINT(41.12952671211202 74.2840674472598) bank36523 +36524 POINT(40.8811074976138 73.45354235092641) bank36524 +36525 POINT(41.35944608898464 73.71041246257931) bank36525 +36526 POINT(41.335469507225994 74.43161810619958) bank36526 +36527 POINT(40.87071032616293 74.03113923797068) bank36527 +36528 POINT(39.900227642890286 74.7343837122819) bank36528 +36529 POINT(41.074464342561264 73.60639826261418) bank36529 +36530 POINT(40.54779715505403 74.84539142958944) bank36530 +36531 POINT(40.675555738421785 74.10708347486569) bank36531 +36532 POINT(41.48357645064474 74.33750578817637) bank36532 +36533 POINT(40.314818896220935 74.03814325603658) bank36533 +36534 POINT(41.69389691338769 73.11748272191078) bank36534 +36535 POINT(40.60175522614799 74.40714769715042) bank36535 +36536 POINT(41.34235925691884 74.37115216777391) bank36536 +36537 POINT(40.72030702840128 73.38498027229167) bank36537 +36538 POINT(39.78119240097475 74.96798897590992) bank36538 +36539 POINT(40.43479741343961 74.33054574129372) bank36539 +36540 POINT(40.12265412983583 74.55413498069993) bank36540 +36541 POINT(41.02551617911434 74.95502185027817) bank36541 +36542 POINT(40.90280050793177 74.26682533347528) bank36542 +36543 POINT(41.70490832322856 74.78386096400426) bank36543 +36544 POINT(40.76928648003757 73.34580394018884) bank36544 +36545 POINT(40.74260779052378 73.96992155147227) bank36545 +36546 POINT(40.59099416005279 73.55712801496625) bank36546 +36547 POINT(41.63339763481867 73.72510200603165) bank36547 +36548 POINT(40.11691706165764 74.6961865668562) bank36548 +36549 POINT(40.875703558162954 74.49586737815443) bank36549 +36550 POINT(40.472757001773616 74.26597601030316) bank36550 +36551 POINT(41.179184473854036 74.35717360757884) bank36551 +36552 POINT(39.79584531703827 73.48009122249314) bank36552 +36553 POINT(40.423989400099444 73.9707751766631) bank36553 +36554 POINT(40.934555877469585 74.37431608766144) bank36554 +36555 POINT(40.6222661164319 73.94764736349738) bank36555 +36556 POINT(41.391234893435275 74.2425692999488) bank36556 +36557 POINT(41.03050065109622 74.38260868583902) bank36557 +36558 POINT(40.27032781996245 74.55137552727818) bank36558 +36559 POINT(40.069845214797056 74.30516137827222) bank36559 +36560 POINT(40.58140339993291 74.48848056197987) bank36560 +36561 POINT(39.87381498833153 74.0612085296465) bank36561 +36562 POINT(39.9196919304646 73.04990897425996) bank36562 +36563 POINT(41.07663264368912 73.61530087114755) bank36563 +36564 POINT(41.04090010032732 73.85171084904684) bank36564 +36565 POINT(41.61184163612534 73.77166432843715) bank36565 +36566 POINT(41.71221885520759 73.847227881853) bank36566 +36567 POINT(41.38506794181727 74.4689042075512) bank36567 +36568 POINT(41.07509206028793 73.7585001389862) bank36568 +36569 POINT(40.9332223860168 74.39197791113975) bank36569 +36570 POINT(41.17697226290077 73.58256881671647) bank36570 +36571 POINT(41.16772467925454 74.00340015693484) bank36571 +36572 POINT(40.08416068641982 73.3054737516911) bank36572 +36573 POINT(39.98838223108976 74.98608156499404) bank36573 +36574 POINT(40.45117595744765 73.8271755028279) bank36574 +36575 POINT(40.146297448224544 74.4628969784423) bank36575 +36576 POINT(40.47214037851366 74.87864463850933) bank36576 +36577 POINT(41.18752393156078 73.402536227681) bank36577 +36578 POINT(41.42213181956646 74.89463819622517) bank36578 +36579 POINT(40.51034522650362 74.55059997807624) bank36579 +36580 POINT(39.76231596344981 73.80498721331232) bank36580 +36581 POINT(40.37977976304393 74.40806640827063) bank36581 +36582 POINT(40.618313337554966 74.54746473716969) bank36582 +36583 POINT(41.22172032990934 73.35837318338305) bank36583 +36584 POINT(41.2222242679736 74.528160714466) bank36584 +36585 POINT(39.85283246768198 73.79976415672922) bank36585 +36586 POINT(40.495421350829155 73.15743188243637) bank36586 +36587 POINT(40.6055237463535 73.53568842005497) bank36587 +36588 POINT(39.856952350481876 74.30493038122599) bank36588 +36589 POINT(40.944710772895625 73.98628818934867) bank36589 +36590 POINT(39.87155979892152 74.16873816055946) bank36590 +36591 POINT(41.348433235074445 74.81746509772113) bank36591 +36592 POINT(40.67537028311739 74.71546077295672) bank36592 +36593 POINT(40.25435466204676 73.34753452023591) bank36593 +36594 POINT(40.08423626329807 74.62874097831946) bank36594 +36595 POINT(40.60802114618297 73.4391385837011) bank36595 +36596 POINT(40.04883889011074 74.60568031532746) bank36596 +36597 POINT(40.2308146584132 73.16965766473484) bank36597 +36598 POINT(40.17674661275617 74.03348753523035) bank36598 +36599 POINT(39.811563428953484 73.0681068401932) bank36599 +36600 POINT(40.50668527379879 74.46403819732504) bank36600 +36601 POINT(40.78498286767774 74.60990272224794) bank36601 +36602 POINT(40.98957776986069 73.13749525016833) bank36602 +36603 POINT(40.7521371561846 74.67512251196867) bank36603 +36604 POINT(40.286596503897194 74.61687536316192) bank36604 +36605 POINT(41.428064294322446 74.14565922561334) bank36605 +36606 POINT(40.38897013670718 74.04693653793504) bank36606 +36607 POINT(40.44795814143464 74.50060083810575) bank36607 +36608 POINT(41.11965843724797 73.62053065437108) bank36608 +36609 POINT(39.959820444669624 74.58060573347659) bank36609 +36610 POINT(40.88931443564395 74.54997999295965) bank36610 +36611 POINT(40.852175963106326 73.1512979004247) bank36611 +36612 POINT(41.08390631429519 73.05145911499656) bank36612 +36613 POINT(41.01499567836459 74.64439856780669) bank36613 +36614 POINT(40.51288940541396 74.57529466758731) bank36614 +36615 POINT(40.69464681118142 74.60435928333762) bank36615 +36616 POINT(40.00373583842358 74.0477964929031) bank36616 +36617 POINT(41.32527687616657 74.97912552790775) bank36617 +36618 POINT(41.077162979701185 73.9789244394309) bank36618 +36619 POINT(40.72013771607778 74.18566295359294) bank36619 +36620 POINT(40.98716886060311 74.77783305081564) bank36620 +36621 POINT(40.81548825059969 73.6476459768948) bank36621 +36622 POINT(40.84894727389918 73.11244363487944) bank36622 +36623 POINT(39.7906954128759 73.19275968157237) bank36623 +36624 POINT(40.65978797989356 74.03355012561155) bank36624 +36625 POINT(41.444469419643156 73.87257523256717) bank36625 +36626 POINT(40.67183257406755 74.11825217165988) bank36626 +36627 POINT(40.0872818990517 73.62513768844197) bank36627 +36628 POINT(39.95716911722371 73.2767550348871) bank36628 +36629 POINT(41.62073986166156 73.83450303696385) bank36629 +36630 POINT(40.70611615383331 74.95390635586311) bank36630 +36631 POINT(39.939110824151626 73.4276376899601) bank36631 +36632 POINT(41.505318330544576 74.32144393330996) bank36632 +36633 POINT(40.57327496448073 74.41076329111502) bank36633 +36634 POINT(40.21338719474372 74.61513032442956) bank36634 +36635 POINT(41.199530334482795 74.65027877954915) bank36635 +36636 POINT(41.260600105164364 74.61922644354323) bank36636 +36637 POINT(40.203713884240514 73.07113639258863) bank36637 +36638 POINT(40.55285106608359 73.25099096302893) bank36638 +36639 POINT(39.96519487316948 73.49437913603518) bank36639 +36640 POINT(40.2224485396202 73.07888478006444) bank36640 +36641 POINT(40.0133089187901 73.450838888895) bank36641 +36642 POINT(39.931059647234626 74.36032679947229) bank36642 +36643 POINT(40.180182900353394 74.98694109875949) bank36643 +36644 POINT(40.038351513892046 73.6781109547377) bank36644 +36645 POINT(41.01210168249217 74.00309415235311) bank36645 +36646 POINT(41.427542920024386 73.7078377689665) bank36646 +36647 POINT(40.24366842107544 73.82888626314812) bank36647 +36648 POINT(40.49312737938273 74.3389510632582) bank36648 +36649 POINT(40.36796470346287 73.96016722766987) bank36649 +36650 POINT(40.710447452193556 74.67089267652341) bank36650 +36651 POINT(40.299333597611884 74.7135504597964) bank36651 +36652 POINT(40.355520218443935 73.56874341513846) bank36652 +36653 POINT(41.45049313533713 73.96428871119485) bank36653 +36654 POINT(39.93182884615733 74.25164018144373) bank36654 +36655 POINT(41.429993853995946 74.75856059831843) bank36655 +36656 POINT(41.25721093940964 73.71517427163002) bank36656 +36657 POINT(40.22582444850949 74.16066585859308) bank36657 +36658 POINT(40.239152239989025 73.18296902077405) bank36658 +36659 POINT(41.009705344261775 74.52812882358378) bank36659 +36660 POINT(41.52848848977942 74.55113263711976) bank36660 +36661 POINT(40.05587981062072 73.44676652984207) bank36661 +36662 POINT(40.951529885498616 73.48068321307353) bank36662 +36663 POINT(40.78662516882542 74.49775923355422) bank36663 +36664 POINT(41.61349292055777 74.70766539822259) bank36664 +36665 POINT(41.25059453941666 73.97982179316503) bank36665 +36666 POINT(40.5389822827357 74.69731288440207) bank36666 +36667 POINT(39.92715310915112 74.29550507437399) bank36667 +36668 POINT(40.784937118535275 74.70675619789496) bank36668 +36669 POINT(39.82969961134307 74.54765885828273) bank36669 +36670 POINT(41.3906066864399 74.48285963114917) bank36670 +36671 POINT(40.29565661357882 73.16211176365748) bank36671 +36672 POINT(39.88704433425171 73.99518317481261) bank36672 +36673 POINT(40.73549221156897 73.22743890502113) bank36673 +36674 POINT(40.36263360961312 74.12853901563346) bank36674 +36675 POINT(40.868077739178815 74.01652929541528) bank36675 +36676 POINT(40.60756702507037 74.2393965814795) bank36676 +36677 POINT(41.60034076204989 73.35763293730678) bank36677 +36678 POINT(40.75537437363531 74.27464382541707) bank36678 +36679 POINT(40.61580499875611 74.61421180555008) bank36679 +36680 POINT(40.82671184445538 73.19128235894622) bank36680 +36681 POINT(40.418715060577405 73.01790935884252) bank36681 +36682 POINT(39.865029663310075 73.78724850289451) bank36682 +36683 POINT(40.465884463101695 74.31449894724108) bank36683 +36684 POINT(40.275707470074344 73.19574671635132) bank36684 +36685 POINT(40.11200554951033 73.59442340054156) bank36685 +36686 POINT(40.15616951305341 74.05553908074137) bank36686 +36687 POINT(40.97013088143783 74.17055982492067) bank36687 +36688 POINT(41.66070548070395 74.6129253179396) bank36688 +36689 POINT(39.78891208461121 74.16064788102628) bank36689 +36690 POINT(40.1426099135598 74.9927754490603) bank36690 +36691 POINT(40.6654461155171 74.55930406852475) bank36691 +36692 POINT(41.50114272722445 74.59408647011483) bank36692 +36693 POINT(41.34109753279516 74.45620749736715) bank36693 +36694 POINT(40.31094876754162 73.08012994268157) bank36694 +36695 POINT(41.139820212267516 73.78461143977711) bank36695 +36696 POINT(41.60823233967486 74.02664652050787) bank36696 +36697 POINT(40.36808036015698 74.8404460490942) bank36697 +36698 POINT(41.527663996402055 74.26606861510012) bank36698 +36699 POINT(39.897603519788376 73.96166278409369) bank36699 +36700 POINT(40.39564069858529 74.56807627950982) bank36700 +36701 POINT(39.84450036366334 73.33642441936442) bank36701 +36702 POINT(40.22433997104142 73.22951153852541) bank36702 +36703 POINT(41.50584549048759 74.67814330746378) bank36703 +36704 POINT(39.84498520286736 73.76775900671399) bank36704 +36705 POINT(40.84537963573207 73.39521410550664) bank36705 +36706 POINT(41.44969796068759 73.18414143383228) bank36706 +36707 POINT(40.94320861842024 74.46404621694339) bank36707 +36708 POINT(40.063737008384635 74.3572721667641) bank36708 +36709 POINT(41.52440760346618 73.45765968813275) bank36709 +36710 POINT(41.439339118997495 73.14418752202768) bank36710 +36711 POINT(40.430176078303916 73.7229358739282) bank36711 +36712 POINT(40.81907232908064 74.13728374865498) bank36712 +36713 POINT(40.092698981352726 74.34707659749564) bank36713 +36714 POINT(39.845789315603795 73.47636507326587) bank36714 +36715 POINT(40.995969056980215 74.73961250983527) bank36715 +36716 POINT(40.33421984691643 73.38130892374379) bank36716 +36717 POINT(41.069433024086685 74.7589581983358) bank36717 +36718 POINT(40.47181821044899 73.69676923111932) bank36718 +36719 POINT(40.279683975643124 74.49773797107281) bank36719 +36720 POINT(40.118005640236696 73.72516354747312) bank36720 +36721 POINT(40.114542180330304 73.82230934630516) bank36721 +36722 POINT(40.93622230587207 73.2424374751552) bank36722 +36723 POINT(40.17176082885256 73.60120635521257) bank36723 +36724 POINT(41.52494528246402 74.01069343204983) bank36724 +36725 POINT(40.67253659242393 74.63071124776108) bank36725 +36726 POINT(39.764159716338135 73.42134497351108) bank36726 +36727 POINT(40.45155561051829 74.65359952114166) bank36727 +36728 POINT(39.90018840411703 73.21344251378844) bank36728 +36729 POINT(40.079148689479865 73.35370273292513) bank36729 +36730 POINT(41.01185966451708 74.42757454470464) bank36730 +36731 POINT(41.13247934491555 73.67413745282872) bank36731 +36732 POINT(40.35619181288559 74.64165295038073) bank36732 +36733 POINT(40.30862197791681 73.21799858360053) bank36733 +36734 POINT(41.44682576695405 73.67914445487806) bank36734 +36735 POINT(41.617326562068314 73.86951435530978) bank36735 +36736 POINT(40.020639683850696 73.61990862393672) bank36736 +36737 POINT(41.03843430028959 74.19610322466252) bank36737 +36738 POINT(41.702728682181856 73.59961598602207) bank36738 +36739 POINT(40.38934683881412 74.12422333392153) bank36739 +36740 POINT(40.921197678249776 73.07620605029572) bank36740 +36741 POINT(40.48937639656417 74.43100205047494) bank36741 +36742 POINT(40.39080654668203 74.73417444954751) bank36742 +36743 POINT(40.481476916929765 74.00270858829556) bank36743 +36744 POINT(41.477532148670605 73.83423834102439) bank36744 +36745 POINT(40.354299931275044 73.8137845692881) bank36745 +36746 POINT(41.69479633953766 73.66840539890373) bank36746 +36747 POINT(40.36636497822446 73.0544960825599) bank36747 +36748 POINT(39.86708033224853 73.61446116098665) bank36748 +36749 POINT(39.781706903541895 74.79558121035537) bank36749 +36750 POINT(39.98910664857056 74.41220180502218) bank36750 +36751 POINT(40.09413659246153 74.23875759114358) bank36751 +36752 POINT(40.085930264955635 73.96527691438781) bank36752 +36753 POINT(41.14577677949155 73.19435647690929) bank36753 +36754 POINT(41.475646782221794 74.63449724426033) bank36754 +36755 POINT(39.91871948249731 73.5097058220434) bank36755 +36756 POINT(40.54039324824824 73.89227737429222) bank36756 +36757 POINT(41.396951231848604 74.39300093121453) bank36757 +36758 POINT(40.43334714558204 73.73185390157116) bank36758 +36759 POINT(40.94399135524994 74.17101392328537) bank36759 +36760 POINT(41.22264452992504 74.02610342149104) bank36760 +36761 POINT(40.30839821866462 73.58108697376066) bank36761 +36762 POINT(39.818327790550676 73.47588365636238) bank36762 +36763 POINT(41.26628699174158 74.97823476004179) bank36763 +36764 POINT(40.25918528544604 73.8822176591344) bank36764 +36765 POINT(40.6078595042577 73.83203843670557) bank36765 +36766 POINT(41.57678059635982 73.20161915415653) bank36766 +36767 POINT(41.04818959285945 74.88084475917427) bank36767 +36768 POINT(40.47489091825189 73.27116381041262) bank36768 +36769 POINT(41.40821272416313 73.4495285144804) bank36769 +36770 POINT(39.73089165476024 73.512546470516) bank36770 +36771 POINT(41.08454574097489 73.46115534756466) bank36771 +36772 POINT(41.68486850934791 74.6550944135248) bank36772 +36773 POINT(41.289035382622814 74.19149603810747) bank36773 +36774 POINT(40.75065155116785 74.75046919974247) bank36774 +36775 POINT(40.64359082373472 74.83967833627284) bank36775 +36776 POINT(41.03060268594368 74.06963035553537) bank36776 +36777 POINT(40.567077038281 74.82557891223215) bank36777 +36778 POINT(40.02197854084216 73.46725871108828) bank36778 +36779 POINT(39.782452724420104 73.49221934933945) bank36779 +36780 POINT(39.82339079578027 73.79772540116087) bank36780 +36781 POINT(41.437179189866335 74.66880630312107) bank36781 +36782 POINT(40.94063976946032 74.82420080836208) bank36782 +36783 POINT(41.4242519940444 74.36869064555891) bank36783 +36784 POINT(41.295992659047016 73.11551744577687) bank36784 +36785 POINT(40.47364732273763 74.01794126614514) bank36785 +36786 POINT(40.54871430608912 74.74465872002898) bank36786 +36787 POINT(40.637021820334 73.15106026038646) bank36787 +36788 POINT(41.48643161086514 74.37602559304112) bank36788 +36789 POINT(41.171476620544205 73.58505794950487) bank36789 +36790 POINT(41.103021411785 73.51258944477823) bank36790 +36791 POINT(40.60040084294147 74.22527074395309) bank36791 +36792 POINT(41.36194915894272 74.01332265470973) bank36792 +36793 POINT(41.25719286458818 74.90193766576974) bank36793 +36794 POINT(40.174558653845274 74.4105915464218) bank36794 +36795 POINT(39.83742910340516 73.12694350586663) bank36795 +36796 POINT(39.85655368891057 74.51132835550976) bank36796 +36797 POINT(40.232056268106625 74.38219694799035) bank36797 +36798 POINT(41.24567224818239 73.69674543620125) bank36798 +36799 POINT(40.56756971536427 73.95040722551988) bank36799 +36800 POINT(40.737486180244765 74.05679069116941) bank36800 +36801 POINT(41.01496231231811 74.0550794920479) bank36801 +36802 POINT(40.21667090836103 74.48202065156066) bank36802 +36803 POINT(41.36469932443865 74.01601875790982) bank36803 +36804 POINT(40.36390147025066 73.26430900402961) bank36804 +36805 POINT(40.278378430158 73.87251276065629) bank36805 +36806 POINT(41.19998011931214 73.52662207210312) bank36806 +36807 POINT(41.68481741953092 73.24615757540985) bank36807 +36808 POINT(40.295889372596136 73.66698769315312) bank36808 +36809 POINT(41.34192159919121 73.22945122746755) bank36809 +36810 POINT(40.56853416797419 74.76484669198092) bank36810 +36811 POINT(41.23879663057754 74.55832855253962) bank36811 +36812 POINT(40.611706939160655 74.26275080252111) bank36812 +36813 POINT(40.62958060487293 74.1552915885032) bank36813 +36814 POINT(40.746670811005146 73.56003526094048) bank36814 +36815 POINT(40.99115118819501 74.78017570304708) bank36815 +36816 POINT(40.19911920818641 74.40486750422161) bank36816 +36817 POINT(40.67368455634084 74.44342731443314) bank36817 +36818 POINT(40.71485532555816 73.4785462112241) bank36818 +36819 POINT(41.13158736786043 74.96479973307046) bank36819 +36820 POINT(40.02961892945025 74.58853621229277) bank36820 +36821 POINT(41.18510056989999 73.17633422545612) bank36821 +36822 POINT(41.3461940493498 74.73166484725256) bank36822 +36823 POINT(40.12077894432721 73.61484227250767) bank36823 +36824 POINT(41.56077050835856 74.14617097073534) bank36824 +36825 POINT(39.71546098674534 73.09049103876657) bank36825 +36826 POINT(40.56132655462143 74.60800300022775) bank36826 +36827 POINT(40.38188876483374 74.75130430329709) bank36827 +36828 POINT(40.707089592732764 73.8972657431803) bank36828 +36829 POINT(39.97224585835923 73.515648704948) bank36829 +36830 POINT(40.40644431072091 74.23936794556771) bank36830 +36831 POINT(39.78679190899595 73.54560172209494) bank36831 +36832 POINT(39.81953080052914 74.19137793695285) bank36832 +36833 POINT(40.594938300910734 73.08794096141223) bank36833 +36834 POINT(39.866690067680295 74.62846324968712) bank36834 +36835 POINT(40.69162073709097 74.21394711277046) bank36835 +36836 POINT(40.52781187339539 73.68196908896515) bank36836 +36837 POINT(40.31473628088305 73.42663616486887) bank36837 +36838 POINT(41.475723849439945 73.58955583198802) bank36838 +36839 POINT(40.257852922198275 73.5458956862667) bank36839 +36840 POINT(40.54285462074982 73.80200034886364) bank36840 +36841 POINT(40.439876464903186 73.46183434183669) bank36841 +36842 POINT(41.251092773082256 73.11883693594518) bank36842 +36843 POINT(40.261416835579055 73.64436210843007) bank36843 +36844 POINT(40.79480296809581 73.64428311265583) bank36844 +36845 POINT(40.11716892451025 74.88073075452233) bank36845 +36846 POINT(41.55096838786439 73.78955338949379) bank36846 +36847 POINT(40.89744789330505 74.02168779941059) bank36847 +36848 POINT(41.35894050050389 74.53835618863305) bank36848 +36849 POINT(41.6978625432851 74.11507720432475) bank36849 +36850 POINT(40.436615933642265 74.57070216093418) bank36850 +36851 POINT(41.407546819868756 74.0911046770436) bank36851 +36852 POINT(40.13467816843649 74.18312072658453) bank36852 +36853 POINT(41.32398579410251 73.04567634408932) bank36853 +36854 POINT(40.854373493939086 74.22886969715323) bank36854 +36855 POINT(40.89736590366143 73.3901178472424) bank36855 +36856 POINT(39.967502656944475 73.73237673559306) bank36856 +36857 POINT(40.092487430739524 74.36362325860891) bank36857 +36858 POINT(40.994693808854365 73.90186396331414) bank36858 +36859 POINT(41.65275356051896 74.8539386878406) bank36859 +36860 POINT(39.922706993748626 73.80614285048462) bank36860 +36861 POINT(41.26853587645821 74.35558674717953) bank36861 +36862 POINT(41.30520927207023 73.92285118950116) bank36862 +36863 POINT(39.93319059451068 74.87356661927772) bank36863 +36864 POINT(40.062908038410114 74.9723323394945) bank36864 +36865 POINT(41.271414452359956 74.79383789356324) bank36865 +36866 POINT(40.52974488919446 74.7203190764973) bank36866 +36867 POINT(40.584113139943035 73.27308102490548) bank36867 +36868 POINT(39.911045044017364 73.76052514566307) bank36868 +36869 POINT(41.38776271059176 74.12330787949311) bank36869 +36870 POINT(40.66811904262084 74.0898923327903) bank36870 +36871 POINT(40.499493838913104 73.67066367061312) bank36871 +36872 POINT(40.0844711946978 74.73923770717887) bank36872 +36873 POINT(40.696673894559595 73.09656584136066) bank36873 +36874 POINT(40.91827540911254 73.06645892099174) bank36874 +36875 POINT(40.49535763621995 73.61227459569713) bank36875 +36876 POINT(40.64464302489705 74.9872576082607) bank36876 +36877 POINT(40.80565541083122 73.21117999125433) bank36877 +36878 POINT(40.71352562068575 73.34618571502463) bank36878 +36879 POINT(40.930682537001545 73.31653725170064) bank36879 +36880 POINT(40.13591007149786 74.03203844721511) bank36880 +36881 POINT(39.835357266537194 74.88156358756642) bank36881 +36882 POINT(40.532493859729605 74.30127688882293) bank36882 +36883 POINT(40.040885691018254 73.24902543144233) bank36883 +36884 POINT(41.47885930347706 74.1930195378498) bank36884 +36885 POINT(39.961398627688595 73.97993244012048) bank36885 +36886 POINT(40.92530200022156 73.2351673918647) bank36886 +36887 POINT(39.78562476093355 73.07888072113907) bank36887 +36888 POINT(39.9118003871273 74.9001402824092) bank36888 +36889 POINT(40.05167162006852 73.45460520376784) bank36889 +36890 POINT(40.76590524189948 73.30264104006848) bank36890 +36891 POINT(40.99387314677344 74.55155237830624) bank36891 +36892 POINT(40.25091253852699 73.39254365894344) bank36892 +36893 POINT(40.076113499922755 74.82389530626816) bank36893 +36894 POINT(41.36865333340571 74.73439174431381) bank36894 +36895 POINT(39.829859766881505 73.25256235489468) bank36895 +36896 POINT(40.60472738434 74.45895472625922) bank36896 +36897 POINT(41.37619807567475 73.62896512999062) bank36897 +36898 POINT(40.19467054107171 74.37447600192354) bank36898 +36899 POINT(41.43928429408707 74.01708193272042) bank36899 +36900 POINT(41.169727468639564 74.2834055813727) bank36900 +36901 POINT(41.410971680276916 74.01969595002943) bank36901 +36902 POINT(40.99148244246499 73.54229342226242) bank36902 +36903 POINT(41.01802822551153 73.53380247837687) bank36903 +36904 POINT(40.7845767718273 74.81974295421053) bank36904 +36905 POINT(39.77216841526212 74.26613216520853) bank36905 +36906 POINT(39.84271039088559 74.44563186471352) bank36906 +36907 POINT(41.62946628743063 73.58426765820096) bank36907 +36908 POINT(41.66451243622574 74.31668861213811) bank36908 +36909 POINT(40.307188597500456 74.10404129883814) bank36909 +36910 POINT(41.0669228311371 74.53689150626504) bank36910 +36911 POINT(41.62517271753283 74.2459208012512) bank36911 +36912 POINT(41.321095395832636 73.61755005369334) bank36912 +36913 POINT(40.55964413431362 73.11578292796649) bank36913 +36914 POINT(40.63979668370004 74.76316987681565) bank36914 +36915 POINT(41.40492666213524 73.46715324584673) bank36915 +36916 POINT(41.36773661466113 74.59611099297315) bank36916 +36917 POINT(39.73665639346573 73.74851960384133) bank36917 +36918 POINT(41.70030216667329 74.99058228120361) bank36918 +36919 POINT(41.19595444516654 74.82757305821653) bank36919 +36920 POINT(40.43034604544512 73.81887022309773) bank36920 +36921 POINT(39.85761493651348 74.05085585534447) bank36921 +36922 POINT(41.63750125753743 74.64222441587901) bank36922 +36923 POINT(39.78382975809419 74.7970250779218) bank36923 +36924 POINT(39.7889624592509 73.41147867404794) bank36924 +36925 POINT(40.085069819834985 74.5447061644797) bank36925 +36926 POINT(40.175931140638156 74.37577214591579) bank36926 +36927 POINT(39.81440111769553 73.90279011559198) bank36927 +36928 POINT(39.897806785903256 73.84057471739104) bank36928 +36929 POINT(41.57771986782166 73.95658958717833) bank36929 +36930 POINT(40.7081088700976 74.88797196273109) bank36930 +36931 POINT(40.33934060363251 73.70949191617355) bank36931 +36932 POINT(40.15435611876974 73.81529822057492) bank36932 +36933 POINT(40.59311066628723 74.04595284999222) bank36933 +36934 POINT(40.23648403162081 73.9696041911666) bank36934 +36935 POINT(40.51297047263574 74.4190921266494) bank36935 +36936 POINT(39.99599013820402 73.54700697043091) bank36936 +36937 POINT(40.21163846268204 73.32116406891508) bank36937 +36938 POINT(40.632347604068755 73.19166030620552) bank36938 +36939 POINT(41.01500212556564 74.4399959320917) bank36939 +36940 POINT(41.46621736502884 74.73366964382252) bank36940 +36941 POINT(40.01955751079205 73.4263805967156) bank36941 +36942 POINT(41.50507883290163 73.34815946212554) bank36942 +36943 POINT(41.617128549022326 74.77660646874482) bank36943 +36944 POINT(40.492964858971284 73.76947654247111) bank36944 +36945 POINT(40.32599711472344 73.70329710759785) bank36945 +36946 POINT(40.265189213886686 74.94330375684596) bank36946 +36947 POINT(39.81784747246084 74.98571574896037) bank36947 +36948 POINT(39.82520487703944 73.8701353953888) bank36948 +36949 POINT(40.8528544914188 73.6254135584009) bank36949 +36950 POINT(40.51003439756874 74.92709832543052) bank36950 +36951 POINT(40.23850235387754 73.13733542478793) bank36951 +36952 POINT(40.954798085599876 74.52231807706445) bank36952 +36953 POINT(41.67331613688247 74.05120809470982) bank36953 +36954 POINT(40.53424556133391 73.0614354479555) bank36954 +36955 POINT(41.70403962728781 73.08653659604092) bank36955 +36956 POINT(40.15484717957657 74.88999509538455) bank36956 +36957 POINT(41.055262876267804 73.34953179577252) bank36957 +36958 POINT(41.35131296202036 73.80228982124449) bank36958 +36959 POINT(40.21924995409627 74.89679001842866) bank36959 +36960 POINT(41.355205661749515 74.10325391649744) bank36960 +36961 POINT(40.7289921546347 74.05595061889115) bank36961 +36962 POINT(40.68604747454267 73.32991401685126) bank36962 +36963 POINT(41.60285177275034 73.96596592255669) bank36963 +36964 POINT(41.05752745441315 73.02435266984041) bank36964 +36965 POINT(40.39194928734377 74.88906374738227) bank36965 +36966 POINT(40.88599471301513 74.82303320271018) bank36966 +36967 POINT(40.994525471856626 74.74052300878225) bank36967 +36968 POINT(40.03514476726109 74.14335594119912) bank36968 +36969 POINT(41.582740037308724 73.12809186448489) bank36969 +36970 POINT(40.44185950386676 73.52467367462549) bank36970 +36971 POINT(41.04406174321318 74.71751988061787) bank36971 +36972 POINT(40.95861619961684 73.22608628802644) bank36972 +36973 POINT(40.60441679165697 74.37956563137914) bank36973 +36974 POINT(40.62565064493726 74.89192010300208) bank36974 +36975 POINT(40.45112775856532 73.30270154555318) bank36975 +36976 POINT(40.475415485880454 74.71000636462178) bank36976 +36977 POINT(40.705805324522785 74.75520311997201) bank36977 +36978 POINT(41.00099931051532 74.64275814603359) bank36978 +36979 POINT(40.47566559666765 74.78379972562144) bank36979 +36980 POINT(40.297071176397225 74.39356344942853) bank36980 +36981 POINT(41.57680161244384 73.96343850880228) bank36981 +36982 POINT(40.743317861507755 74.70681800444821) bank36982 +36983 POINT(39.90919057117213 74.87462672799091) bank36983 +36984 POINT(41.38842988611179 73.76070242995262) bank36984 +36985 POINT(39.885236886500905 74.85917185510024) bank36985 +36986 POINT(40.798406649303566 73.93522497563524) bank36986 +36987 POINT(40.70205615994624 74.57290845579139) bank36987 +36988 POINT(40.22514198515946 74.47714067111848) bank36988 +36989 POINT(40.49666501671987 74.87607199267832) bank36989 +36990 POINT(40.66150098375285 74.64190486259042) bank36990 +36991 POINT(40.2903188548786 74.78916478342539) bank36991 +36992 POINT(40.828457391879034 74.45763159328519) bank36992 +36993 POINT(41.22228593730072 73.23337803220528) bank36993 +36994 POINT(40.56810782316889 74.71393928082465) bank36994 +36995 POINT(41.56125117297758 73.47118689897225) bank36995 +36996 POINT(40.637664291652484 73.52941459502978) bank36996 +36997 POINT(40.002303959514514 73.7232342268792) bank36997 +36998 POINT(41.233219487072994 74.84040312706581) bank36998 +36999 POINT(40.52656942913342 74.33460564962184) bank36999 +37000 POINT(40.1359400197904 74.73712459386547) bank37000 +37001 POINT(41.61734531973003 73.54875776072919) bank37001 +37002 POINT(40.39080651162838 73.47538735455446) bank37002 +37003 POINT(41.526517231845766 73.10992812831329) bank37003 +37004 POINT(41.501592186425434 74.70552976666333) bank37004 +37005 POINT(40.69967923575993 73.32006626681424) bank37005 +37006 POINT(40.640130134387284 74.92211777821247) bank37006 +37007 POINT(41.51934838596236 73.60950913563292) bank37007 +37008 POINT(40.534599057844645 73.3417250939352) bank37008 +37009 POINT(41.40574636722852 73.0825728762039) bank37009 +37010 POINT(41.58526263296591 73.60033628041965) bank37010 +37011 POINT(39.78359625872581 73.09826202607124) bank37011 +37012 POINT(39.95608818912494 73.77073861522375) bank37012 +37013 POINT(39.73608662578372 73.8011499709157) bank37013 +37014 POINT(41.04472781363495 74.18143335517335) bank37014 +37015 POINT(39.798623244888304 73.20703116962653) bank37015 +37016 POINT(40.62650450616399 74.87134757740537) bank37016 +37017 POINT(41.39989786066804 74.56433445505614) bank37017 +37018 POINT(40.96108052103644 73.48584818082134) bank37018 +37019 POINT(39.78637780658972 73.37127347779227) bank37019 +37020 POINT(40.80124256942254 73.22927691728952) bank37020 +37021 POINT(41.14054134447144 74.99025094252629) bank37021 +37022 POINT(40.6843273135381 74.36716721644676) bank37022 +37023 POINT(40.707994313629904 73.5376088319414) bank37023 +37024 POINT(41.56288786335893 73.50956174389849) bank37024 +37025 POINT(40.07922685169798 74.11216260118327) bank37025 +37026 POINT(40.51697494055758 73.06097877058376) bank37026 +37027 POINT(41.36868130901945 73.66167697784596) bank37027 +37028 POINT(41.513198904945625 73.50850737131374) bank37028 +37029 POINT(39.846641859043174 73.75454028051024) bank37029 +37030 POINT(41.59336954114201 73.47461455088646) bank37030 +37031 POINT(41.2675750502372 74.71888460864383) bank37031 +37032 POINT(39.93673319028463 74.40915619671617) bank37032 +37033 POINT(41.68539171043366 73.2972695591106) bank37033 +37034 POINT(40.72249434226645 73.20661623001178) bank37034 +37035 POINT(41.174735089833476 73.44181178401158) bank37035 +37036 POINT(41.262022951394 73.83747064524562) bank37036 +37037 POINT(39.731577528018086 73.7558811303593) bank37037 +37038 POINT(40.043920250197026 74.84125213313652) bank37038 +37039 POINT(41.18685189385086 73.70958681454972) bank37039 +37040 POINT(39.89090244265043 73.36715744347184) bank37040 +37041 POINT(41.09249464463408 74.96933341084149) bank37041 +37042 POINT(40.083111560445815 73.02837951663115) bank37042 +37043 POINT(41.57150086553827 74.22388358704872) bank37043 +37044 POINT(41.42441422912155 73.25760475377976) bank37044 +37045 POINT(40.626381680070374 73.11670302273161) bank37045 +37046 POINT(41.52630238267427 73.99554741554829) bank37046 +37047 POINT(41.61442726374642 73.45920459939207) bank37047 +37048 POINT(40.46065856258356 74.9507042014235) bank37048 +37049 POINT(41.282088245745626 74.23534968406219) bank37049 +37050 POINT(39.810586580640575 74.04897695050063) bank37050 +37051 POINT(41.03756266619686 74.75251145818896) bank37051 +37052 POINT(41.631894441756366 73.28512495077734) bank37052 +37053 POINT(41.60998383519336 73.42600390335654) bank37053 +37054 POINT(41.6459659248908 74.92063217943038) bank37054 +37055 POINT(39.94468206473689 73.09592526420109) bank37055 +37056 POINT(40.75820181132815 74.60054184660473) bank37056 +37057 POINT(40.38435771173895 73.00829463954257) bank37057 +37058 POINT(40.240730233842505 73.0570606873417) bank37058 +37059 POINT(40.352982774522246 74.57653532929137) bank37059 +37060 POINT(40.89239148172649 74.59673377339634) bank37060 +37061 POINT(39.81036598633389 73.2194476494662) bank37061 +37062 POINT(40.614911536456276 74.49560515746468) bank37062 +37063 POINT(39.92503392935414 73.00685946069747) bank37063 +37064 POINT(40.81339342422659 73.66484769468978) bank37064 +37065 POINT(40.4363126383036 73.41299092028186) bank37065 +37066 POINT(39.943412227036994 73.16168327900323) bank37066 +37067 POINT(39.89684414842282 73.91640175532845) bank37067 +37068 POINT(40.805544966498545 73.97394425934914) bank37068 +37069 POINT(41.15321063624499 74.87006306026306) bank37069 +37070 POINT(41.01975008136166 74.81774381987144) bank37070 +37071 POINT(41.358686470950175 74.25855547975985) bank37071 +37072 POINT(39.818898447347514 73.93758679236012) bank37072 +37073 POINT(40.91075949551822 73.62235805601019) bank37073 +37074 POINT(40.55568734123569 74.8665499220217) bank37074 +37075 POINT(40.74285355729923 73.40465512948255) bank37075 +37076 POINT(40.46224832324374 74.06244700365407) bank37076 +37077 POINT(40.053224644540684 73.95963714817884) bank37077 +37078 POINT(40.20500978420599 73.12235853246045) bank37078 +37079 POINT(40.47012938079827 74.45343927769463) bank37079 +37080 POINT(40.92571905382583 74.25724722282564) bank37080 +37081 POINT(41.12727835901178 73.52318840993735) bank37081 +37082 POINT(40.39815394235276 73.79793420857187) bank37082 +37083 POINT(40.614518636228084 73.85118278569499) bank37083 +37084 POINT(39.9687835918897 74.40805485989655) bank37084 +37085 POINT(39.788811657609976 74.93863030539913) bank37085 +37086 POINT(41.13336409310259 74.31807323372675) bank37086 +37087 POINT(40.64731597548642 73.96652292884066) bank37087 +37088 POINT(41.2878280821168 73.50203231233289) bank37088 +37089 POINT(40.11256970082642 74.47426243807229) bank37089 +37090 POINT(39.95224090729998 73.12090707108442) bank37090 +37091 POINT(40.0659133823092 74.9793232009038) bank37091 +37092 POINT(41.613681650068536 74.01892886414454) bank37092 +37093 POINT(40.53787055648201 74.96330220419254) bank37093 +37094 POINT(40.55432735063453 74.75574619745346) bank37094 +37095 POINT(41.31419614695144 73.78843821208058) bank37095 +37096 POINT(40.42656601855449 73.91760794452966) bank37096 +37097 POINT(39.85924047683615 74.84835559359382) bank37097 +37098 POINT(40.96524651314729 73.2010075978576) bank37098 +37099 POINT(40.554186796164586 73.74344257402397) bank37099 +37100 POINT(40.26372572000848 74.30047119198477) bank37100 +37101 POINT(41.703585130755776 73.98741718296708) bank37101 +37102 POINT(40.73609231555509 74.48149221077078) bank37102 +37103 POINT(39.930320604975805 73.96037838459674) bank37103 +37104 POINT(41.125984448806 74.41110424727479) bank37104 +37105 POINT(40.686144301696785 74.04226764168149) bank37105 +37106 POINT(40.39548172372963 73.78297814195935) bank37106 +37107 POINT(40.046498835835834 73.3191113238762) bank37107 +37108 POINT(41.422621151011626 73.79720193571576) bank37108 +37109 POINT(40.97660354669311 74.09071729837417) bank37109 +37110 POINT(41.12838297273999 73.97506921897494) bank37110 +37111 POINT(41.187042814621066 74.57176234572911) bank37111 +37112 POINT(41.61676550559399 74.31782541160378) bank37112 +37113 POINT(41.636190942657045 73.74319172611258) bank37113 +37114 POINT(39.80799971514383 73.98019116808358) bank37114 +37115 POINT(40.01153222185083 74.39495508872638) bank37115 +37116 POINT(39.72618772133425 74.46135340872375) bank37116 +37117 POINT(41.70391255949267 74.95378659884483) bank37117 +37118 POINT(40.21219517135845 73.15820779922689) bank37118 +37119 POINT(40.230388839589935 73.57279938686344) bank37119 +37120 POINT(40.296600897502245 74.28098311464572) bank37120 +37121 POINT(41.44987987976166 74.05746020203094) bank37121 +37122 POINT(41.05872651019608 74.44599005533233) bank37122 +37123 POINT(40.91888993014498 74.81406806837408) bank37123 +37124 POINT(40.15139006423393 74.4979955719215) bank37124 +37125 POINT(41.41224472356761 73.22970537442204) bank37125 +37126 POINT(40.526897701218935 74.73525248281386) bank37126 +37127 POINT(40.63530033775303 73.24551007293222) bank37127 +37128 POINT(40.4798882480189 73.9741102672807) bank37128 +37129 POINT(40.157495894967624 74.58331532709838) bank37129 +37130 POINT(40.28386140246493 73.31303048359352) bank37130 +37131 POINT(39.736573615025044 73.93226783373079) bank37131 +37132 POINT(41.40550316052122 74.702024477814) bank37132 +37133 POINT(40.94250698021854 73.97951831685367) bank37133 +37134 POINT(41.0940287739903 74.43144156233694) bank37134 +37135 POINT(40.90582720084197 74.55342247063246) bank37135 +37136 POINT(39.75872190054454 74.07653170675536) bank37136 +37137 POINT(39.71395824371861 74.9415803284014) bank37137 +37138 POINT(41.064372978896834 74.55257148130725) bank37138 +37139 POINT(40.139620729533924 73.04108203502827) bank37139 +37140 POINT(40.39995411380119 73.17505902056301) bank37140 +37141 POINT(39.83771384259335 73.18830000326317) bank37141 +37142 POINT(41.143092237714576 73.95219798883576) bank37142 +37143 POINT(40.731725473999404 74.50422294747517) bank37143 +37144 POINT(40.40108894973631 74.43941524062768) bank37144 +37145 POINT(40.535010266266866 73.31431728544608) bank37145 +37146 POINT(41.15237250969257 74.72292875046746) bank37146 +37147 POINT(40.55162190546243 73.79517771197341) bank37147 +37148 POINT(40.987775698248534 73.4591505812236) bank37148 +37149 POINT(41.534198944337014 73.04496387783334) bank37149 +37150 POINT(40.90632536283749 73.59558630235868) bank37150 +37151 POINT(40.35568662048016 74.57902122132236) bank37151 +37152 POINT(39.782043393292284 73.97770170542736) bank37152 +37153 POINT(41.68028734037114 74.37393573136782) bank37153 +37154 POINT(41.03492833596152 74.31359539478753) bank37154 +37155 POINT(40.51931884724324 74.22997444239417) bank37155 +37156 POINT(40.529484234700014 73.36620975864325) bank37156 +37157 POINT(40.61459844185261 73.26376195849292) bank37157 +37158 POINT(40.71267219325569 73.46833592171548) bank37158 +37159 POINT(40.125666208210426 74.23851075432366) bank37159 +37160 POINT(40.302123345995604 73.5050495114612) bank37160 +37161 POINT(41.300091770666086 74.65878590465654) bank37161 +37162 POINT(41.310923390613304 74.63061540780711) bank37162 +37163 POINT(41.48963903944448 73.41446508520575) bank37163 +37164 POINT(40.4119731783702 74.9608743450621) bank37164 +37165 POINT(41.487052020453305 73.6489144282333) bank37165 +37166 POINT(39.73599999784637 74.9897970483274) bank37166 +37167 POINT(40.10489148667002 75.00194697103362) bank37167 +37168 POINT(39.97115065451284 73.97651192401969) bank37168 +37169 POINT(40.11525902339016 74.35392848825444) bank37169 +37170 POINT(40.996555951329384 73.03464321807807) bank37170 +37171 POINT(41.08710922566921 74.23273361548966) bank37171 +37172 POINT(40.73894714844283 73.05643802707982) bank37172 +37173 POINT(41.179371414843814 73.80439114274587) bank37173 +37174 POINT(40.440394816711816 73.65573355702497) bank37174 +37175 POINT(40.16217968553729 74.53185948649296) bank37175 +37176 POINT(39.96286746337534 73.59619592390648) bank37176 +37177 POINT(40.36521312088202 73.62275134052844) bank37177 +37178 POINT(40.24849619530866 73.67184243476004) bank37178 +37179 POINT(39.91689202441104 74.87616419108046) bank37179 +37180 POINT(40.67102788611606 74.09348257404527) bank37180 +37181 POINT(39.94084091352792 74.43368760709906) bank37181 +37182 POINT(41.34913160346888 73.44744554817834) bank37182 +37183 POINT(41.5787403124043 74.62432463171925) bank37183 +37184 POINT(40.89767887877636 73.92647542008456) bank37184 +37185 POINT(40.99275603668447 73.70675735303759) bank37185 +37186 POINT(41.512537283502105 73.6736221268207) bank37186 +37187 POINT(41.40555343812197 74.36726387893519) bank37187 +37188 POINT(40.28071598079392 73.56624424611068) bank37188 +37189 POINT(40.73541538645716 73.54165424050703) bank37189 +37190 POINT(40.65603987947477 74.70319995603592) bank37190 +37191 POINT(41.658242256432644 74.20017617538923) bank37191 +37192 POINT(40.890335307617406 73.40890109040056) bank37192 +37193 POINT(41.2115787218535 74.82502619965342) bank37193 +37194 POINT(41.34263364028647 73.16799159794549) bank37194 +37195 POINT(41.08631574323543 74.93594755369868) bank37195 +37196 POINT(41.447341718227335 74.18794997502408) bank37196 +37197 POINT(41.649694329412995 74.64200694469929) bank37197 +37198 POINT(40.47510579423877 75.00485127020491) bank37198 +37199 POINT(40.77165246842178 73.1067320123738) bank37199 +37200 POINT(41.16199889618316 73.35739158065988) bank37200 +37201 POINT(40.18555838910024 73.70597337732474) bank37201 +37202 POINT(40.118734111919885 73.6514752005279) bank37202 +37203 POINT(41.218811819215816 74.31633276228646) bank37203 +37204 POINT(41.4790195879376 73.05826943233771) bank37204 +37205 POINT(39.79296376210619 73.78101519839241) bank37205 +37206 POINT(40.91369947665215 74.80585549376664) bank37206 +37207 POINT(41.27012601332677 74.00860685806985) bank37207 +37208 POINT(40.89311947955513 74.38512125211638) bank37208 +37209 POINT(41.68523546800258 73.87628763967975) bank37209 +37210 POINT(40.07239646993967 74.42841408044545) bank37210 +37211 POINT(41.00701432371177 73.07235810627287) bank37211 +37212 POINT(40.41610203341281 73.67724028582985) bank37212 +37213 POINT(40.091696861148584 73.77037401049252) bank37213 +37214 POINT(41.21720781730971 73.96892964706616) bank37214 +37215 POINT(41.09372451806943 73.7414404976688) bank37215 +37216 POINT(40.47419691261 73.42399909472734) bank37216 +37217 POINT(41.11045686926308 74.51693636643382) bank37217 +37218 POINT(41.12803941836615 73.74256030163666) bank37218 +37219 POINT(41.52173786327235 74.13371619531763) bank37219 +37220 POINT(40.616307289151365 74.07990585421207) bank37220 +37221 POINT(39.97749156125569 73.52724749502995) bank37221 +37222 POINT(40.8693156027873 74.66676505349541) bank37222 +37223 POINT(40.57765294905626 74.75116066785966) bank37223 +37224 POINT(40.15456468367935 74.59826975931442) bank37224 +37225 POINT(41.08199903404462 74.62106196413619) bank37225 +37226 POINT(40.183907369238064 74.57148378624895) bank37226 +37227 POINT(39.728319099212584 73.44113161335488) bank37227 +37228 POINT(40.49902869617271 73.94840451234364) bank37228 +37229 POINT(40.78383373605147 73.05433908197844) bank37229 +37230 POINT(40.71172479869445 73.84407951931952) bank37230 +37231 POINT(41.653637041922465 74.13639941113266) bank37231 +37232 POINT(40.378122922385614 73.70186209865577) bank37232 +37233 POINT(41.41733592329429 74.9225720248067) bank37233 +37234 POINT(41.20609719321424 73.23616025539205) bank37234 +37235 POINT(40.568019449939484 74.53201355281583) bank37235 +37236 POINT(40.35688915014107 74.84441111510183) bank37236 +37237 POINT(40.09681522741721 73.28738831546485) bank37237 +37238 POINT(40.5755058682542 73.57865057941696) bank37238 +37239 POINT(41.18919733309737 74.64280722802819) bank37239 +37240 POINT(40.54230101450187 73.17696072164071) bank37240 +37241 POINT(41.3518096388819 73.64367072358515) bank37241 +37242 POINT(41.073729218378254 73.16081099365287) bank37242 +37243 POINT(40.8490505589297 74.93139309150291) bank37243 +37244 POINT(39.857133450831384 73.0094497613781) bank37244 +37245 POINT(41.118410857743065 73.79660592272843) bank37245 +37246 POINT(40.18479735141209 74.04606835666482) bank37246 +37247 POINT(39.99872559707869 74.64351719000223) bank37247 +37248 POINT(40.34555390918046 74.04600534650116) bank37248 +37249 POINT(41.00039539362794 73.41394139367944) bank37249 +37250 POINT(41.457139064463846 73.21240821240877) bank37250 +37251 POINT(40.298873580377915 73.71876266000756) bank37251 +37252 POINT(41.54028564920702 74.94009920470126) bank37252 +37253 POINT(41.672447289428874 74.30195469806986) bank37253 +37254 POINT(40.65947311760885 74.4996244306658) bank37254 +37255 POINT(41.57794758142957 73.14882854432324) bank37255 +37256 POINT(39.85784676476586 73.56074147064757) bank37256 +37257 POINT(39.72619495897595 74.04452972765598) bank37257 +37258 POINT(40.97295697252234 74.17407935570596) bank37258 +37259 POINT(41.60066372189943 74.92769634109544) bank37259 +37260 POINT(40.596598473473975 74.40258877647094) bank37260 +37261 POINT(40.13276726285447 73.96570906130016) bank37261 +37262 POINT(40.645598434114625 73.34488121295985) bank37262 +37263 POINT(40.237755342766526 73.69492295519036) bank37263 +37264 POINT(39.73272668712096 74.42584348243734) bank37264 +37265 POINT(40.53115120523994 74.76440161752024) bank37265 +37266 POINT(40.821523829108465 74.47896179268608) bank37266 +37267 POINT(40.87725290660478 73.73202815866493) bank37267 +37268 POINT(40.05547665455095 74.308798241447) bank37268 +37269 POINT(40.614217795479725 73.86160351681569) bank37269 +37270 POINT(40.25754345109903 73.98329222059978) bank37270 +37271 POINT(40.294819118787785 74.3920167192894) bank37271 +37272 POINT(40.678945141947914 73.60299772975826) bank37272 +37273 POINT(40.1825336836645 73.84359028830897) bank37273 +37274 POINT(41.43422687722179 74.46182544776471) bank37274 +37275 POINT(41.707931289852745 73.2341491147131) bank37275 +37276 POINT(40.68251256154621 73.27776145483504) bank37276 +37277 POINT(40.13324998571678 73.32224914177267) bank37277 +37278 POINT(41.640335642107104 74.28056141527667) bank37278 +37279 POINT(41.12997906451754 73.90355266535246) bank37279 +37280 POINT(40.877828595903694 73.31205448440349) bank37280 +37281 POINT(40.85660877586812 73.27326131911252) bank37281 +37282 POINT(41.65978264804957 74.00023136018253) bank37282 +37283 POINT(40.458940593301406 73.4818244318954) bank37283 +37284 POINT(41.511957525637314 74.97563586000918) bank37284 +37285 POINT(40.403070399018475 73.92037916249178) bank37285 +37286 POINT(41.44594230707723 74.58498530840274) bank37286 +37287 POINT(40.538643207927336 74.86209980430965) bank37287 +37288 POINT(40.846067406681975 74.33206766863273) bank37288 +37289 POINT(41.55527449062218 74.29084829066902) bank37289 +37290 POINT(40.34137412333689 74.3800921163431) bank37290 +37291 POINT(41.12197068418855 73.3006285737146) bank37291 +37292 POINT(40.41069555568041 74.86363451521744) bank37292 +37293 POINT(40.31125458140856 73.92741579586554) bank37293 +37294 POINT(40.61342935597106 73.78480026803058) bank37294 +37295 POINT(41.17426059536987 74.31866181904715) bank37295 +37296 POINT(41.24417131789231 73.9725245062053) bank37296 +37297 POINT(40.63791995723184 74.38296367814641) bank37297 +37298 POINT(40.49534269578645 74.7071313196106) bank37298 +37299 POINT(41.12326172155893 74.26369103048256) bank37299 +37300 POINT(40.94030453406754 73.16436178416888) bank37300 +37301 POINT(40.670235361996475 74.40690952459617) bank37301 +37302 POINT(40.10599838487139 74.23817847601404) bank37302 +37303 POINT(41.203822817356844 73.31000551249622) bank37303 +37304 POINT(41.63255508090298 73.06257275226996) bank37304 +37305 POINT(40.30724740482427 74.31604427898182) bank37305 +37306 POINT(41.22738913324097 73.19588159750778) bank37306 +37307 POINT(41.64920344145656 74.62682811365949) bank37307 +37308 POINT(40.33436606914501 74.9834324167407) bank37308 +37309 POINT(39.913771417327006 73.97114446515161) bank37309 +37310 POINT(40.45616209729661 74.67653404821624) bank37310 +37311 POINT(40.929088929169716 73.57421162808753) bank37311 +37312 POINT(40.81526307645524 74.14627936156562) bank37312 +37313 POINT(39.99362315744355 73.16328151444111) bank37313 +37314 POINT(41.025353345683385 73.25931074000448) bank37314 +37315 POINT(40.8221484426376 73.46158051290212) bank37315 +37316 POINT(40.77288301700854 74.94375829793854) bank37316 +37317 POINT(41.237661841790455 73.06920824496822) bank37317 +37318 POINT(40.29863970934837 74.781536587413) bank37318 +37319 POINT(41.35300992918105 73.59135329215111) bank37319 +37320 POINT(41.25534788865824 74.35990996903759) bank37320 +37321 POINT(39.73998369256138 74.9713885076759) bank37321 +37322 POINT(40.10183457551036 73.41525721103692) bank37322 +37323 POINT(41.47506731770708 73.21603805422194) bank37323 +37324 POINT(40.933344810542636 74.65976468235424) bank37324 +37325 POINT(41.28806462470543 74.56590172110991) bank37325 +37326 POINT(40.58892933873539 73.5632341226292) bank37326 +37327 POINT(40.14341847202673 74.00746192539299) bank37327 +37328 POINT(41.55849093804481 73.29437395247616) bank37328 +37329 POINT(39.72632138392148 74.12538348580628) bank37329 +37330 POINT(41.39325571806096 73.06562253540002) bank37330 +37331 POINT(39.79097838988752 73.88594944147721) bank37331 +37332 POINT(40.36387878154415 74.87537754043969) bank37332 +37333 POINT(40.46268679278723 73.58738863497622) bank37333 +37334 POINT(41.400277504947546 73.10377282757909) bank37334 +37335 POINT(41.140121344412464 73.17169476849811) bank37335 +37336 POINT(41.49789069148206 74.8923947044266) bank37336 +37337 POINT(40.07437021057426 74.74705102647883) bank37337 +37338 POINT(40.392051933021385 74.67882282286624) bank37338 +37339 POINT(40.712537312858075 74.51676086848657) bank37339 +37340 POINT(40.65032143737291 74.0233347570202) bank37340 +37341 POINT(41.045222776884636 74.39809437766507) bank37341 +37342 POINT(40.81232398976236 74.67501551431558) bank37342 +37343 POINT(40.857674038379216 73.14285334446502) bank37343 +37344 POINT(39.988651620180825 74.04511315758631) bank37344 +37345 POINT(41.5081994286312 73.81274398515828) bank37345 +37346 POINT(40.03686245195574 73.43611973744095) bank37346 +37347 POINT(40.27816075585209 74.62528411792258) bank37347 +37348 POINT(41.186858726602615 73.9784350778107) bank37348 +37349 POINT(40.474428647811386 73.51494435458297) bank37349 +37350 POINT(41.52394833910569 73.10376495344988) bank37350 +37351 POINT(39.9856078255006 73.11390007338825) bank37351 +37352 POINT(41.490170554675956 74.41476000788195) bank37352 +37353 POINT(41.219494102136956 74.98839130225188) bank37353 +37354 POINT(41.33168622478808 73.21509293761545) bank37354 +37355 POINT(40.78827445733232 74.97237683690531) bank37355 +37356 POINT(39.83083297895509 74.58602361004262) bank37356 +37357 POINT(40.18199423293535 74.37991254493399) bank37357 +37358 POINT(40.772429889139026 73.56154544992108) bank37358 +37359 POINT(41.379962766509216 74.37143023282202) bank37359 +37360 POINT(39.789271537701346 73.37333140664133) bank37360 +37361 POINT(41.67580230260249 73.19447962638584) bank37361 +37362 POINT(40.6413609542887 73.26865530354152) bank37362 +37363 POINT(40.69637539094718 74.26973287152602) bank37363 +37364 POINT(41.34467247017608 74.71164119725168) bank37364 +37365 POINT(40.29811626108775 74.02202648672746) bank37365 +37366 POINT(40.70559326085855 73.24683283787705) bank37366 +37367 POINT(39.862414358610934 74.06066760598216) bank37367 +37368 POINT(40.87456536149571 74.54248631741994) bank37368 +37369 POINT(40.11242610618324 74.0024590591838) bank37369 +37370 POINT(41.30419060745454 73.29379011492922) bank37370 +37371 POINT(39.74139389636145 75.0010477522137) bank37371 +37372 POINT(41.440867640302535 73.24614776166763) bank37372 +37373 POINT(40.262493055684956 73.41998904581034) bank37373 +37374 POINT(41.1459880220018 73.6221393826301) bank37374 +37375 POINT(40.00186921428214 74.16347260218619) bank37375 +37376 POINT(39.83855458805044 74.16904326582329) bank37376 +37377 POINT(40.30804934040562 74.93808049129133) bank37377 +37378 POINT(41.51936207733505 73.38709666069917) bank37378 +37379 POINT(41.367288838439976 74.33152202888924) bank37379 +37380 POINT(41.59207686013112 73.33511311909152) bank37380 +37381 POINT(41.26884549228117 74.342424424509) bank37381 +37382 POINT(41.6119502648931 73.01608492155368) bank37382 +37383 POINT(40.29050764730437 73.66373897032241) bank37383 +37384 POINT(40.20914295651357 73.75928475074303) bank37384 +37385 POINT(39.819854698752266 73.63125228912524) bank37385 +37386 POINT(40.11344977750621 73.31653227601838) bank37386 +37387 POINT(41.14914951227916 73.49306577388587) bank37387 +37388 POINT(41.39738776019656 74.03456945430445) bank37388 +37389 POINT(41.15141801941235 73.07088660396775) bank37389 +37390 POINT(41.63410424638039 73.02782164215331) bank37390 +37391 POINT(40.88549349268888 73.96910068364896) bank37391 +37392 POINT(41.67086085905608 74.7034177366035) bank37392 +37393 POINT(40.212877179726 74.27626509280392) bank37393 +37394 POINT(41.657280631754006 73.74745121132743) bank37394 +37395 POINT(41.310246179275346 73.35001845598745) bank37395 +37396 POINT(41.31124195443015 73.27743515532872) bank37396 +37397 POINT(40.84203196134646 74.37235187160043) bank37397 +37398 POINT(41.648311511585156 73.37332700538579) bank37398 +37399 POINT(40.73012921107077 74.05688688867178) bank37399 +37400 POINT(41.38233352355998 74.10183123828848) bank37400 +37401 POINT(41.09417556981542 74.93616203929959) bank37401 +37402 POINT(41.706494155099364 74.79793377631685) bank37402 +37403 POINT(40.21501763650091 74.96753247159151) bank37403 +37404 POINT(41.20640622938433 73.95125845246294) bank37404 +37405 POINT(40.968010386003954 73.43834246586101) bank37405 +37406 POINT(40.46367490335396 74.07212167055047) bank37406 +37407 POINT(39.90594827374009 73.48403426644165) bank37407 +37408 POINT(41.28915451149341 73.35460623983444) bank37408 +37409 POINT(41.10852175580382 73.0970892747996) bank37409 +37410 POINT(40.23448635373475 73.28024243150399) bank37410 +37411 POINT(40.59899536104339 74.11719143650164) bank37411 +37412 POINT(41.26530269689744 73.85209921603503) bank37412 +37413 POINT(41.41251771814904 74.83592157430182) bank37413 +37414 POINT(39.93963260571276 74.11843208399813) bank37414 +37415 POINT(40.17468896790663 74.11119187407057) bank37415 +37416 POINT(41.12878791093052 74.09128899196335) bank37416 +37417 POINT(40.02684788061282 74.25424127945813) bank37417 +37418 POINT(41.614180590018194 73.75049868619512) bank37418 +37419 POINT(40.549895896526586 74.91509761094102) bank37419 +37420 POINT(41.54449941664032 74.39098906410618) bank37420 +37421 POINT(41.66855097911305 73.93811705619544) bank37421 +37422 POINT(39.88427629197865 74.92664598008163) bank37422 +37423 POINT(39.744049542770924 74.73715568818075) bank37423 +37424 POINT(40.97872135037018 74.46657537903504) bank37424 +37425 POINT(40.5328355198856 74.7600728918241) bank37425 +37426 POINT(40.10068728271808 73.77965592385522) bank37426 +37427 POINT(41.51552365241447 73.58609937081357) bank37427 +37428 POINT(40.50173127240561 74.32746696603803) bank37428 +37429 POINT(40.33415254468972 74.15485117538786) bank37429 +37430 POINT(40.287115933546524 74.92246273771424) bank37430 +37431 POINT(41.40591348872142 74.00811515021724) bank37431 +37432 POINT(41.1187957853534 73.96410377828921) bank37432 +37433 POINT(39.8371058625571 73.67407084459155) bank37433 +37434 POINT(40.68326471191288 73.55463061783308) bank37434 +37435 POINT(40.48379625593569 73.3439155986313) bank37435 +37436 POINT(41.351451571593294 74.71734895763673) bank37436 +37437 POINT(40.881322498059845 73.49568912379355) bank37437 +37438 POINT(39.80239938959072 73.07915477731078) bank37438 +37439 POINT(40.39465131747333 74.22801072591014) bank37439 +37440 POINT(40.62207037446121 74.10735996988814) bank37440 +37441 POINT(40.13595706052654 73.4066356817959) bank37441 +37442 POINT(40.52933068936486 74.59301988504555) bank37442 +37443 POINT(41.11554088020377 73.4662370019531) bank37443 +37444 POINT(40.45661643064108 74.60667068638038) bank37444 +37445 POINT(39.88258149340583 73.42593191512087) bank37445 +37446 POINT(40.70075492104757 73.15001400928222) bank37446 +37447 POINT(39.76343692366592 73.98283215786164) bank37447 +37448 POINT(40.733289761705976 74.78626931221368) bank37448 +37449 POINT(40.16758657241136 74.73138823707548) bank37449 +37450 POINT(39.78456580235229 73.35456049725857) bank37450 +37451 POINT(41.70097369702651 74.58753969664886) bank37451 +37452 POINT(41.67013379439484 74.39584051110829) bank37452 +37453 POINT(41.38577055358212 74.15075870999021) bank37453 +37454 POINT(40.669477318226505 73.58709141211882) bank37454 +37455 POINT(39.73571570618076 73.67933921094934) bank37455 +37456 POINT(41.510331469539665 73.60390191720626) bank37456 +37457 POINT(40.319461685547495 73.74441510483467) bank37457 +37458 POINT(41.444029113769595 74.35217066166) bank37458 +37459 POINT(39.92890462682543 74.71120172287446) bank37459 +37460 POINT(40.06742971366315 73.99927123738553) bank37460 +37461 POINT(39.91110090706023 73.72515695850741) bank37461 +37462 POINT(41.485498668324695 74.11192518494049) bank37462 +37463 POINT(41.63808736831859 74.8576120895563) bank37463 +37464 POINT(40.14004355630585 73.47294174250969) bank37464 +37465 POINT(40.24517378536872 74.21207269727452) bank37465 +37466 POINT(40.47378719565216 74.81004166616553) bank37466 +37467 POINT(41.19376151048912 74.2770109096325) bank37467 +37468 POINT(40.57496805581621 73.42791544865682) bank37468 +37469 POINT(41.53742844938631 74.21620471885043) bank37469 +37470 POINT(41.63043550199858 74.47763489853176) bank37470 +37471 POINT(40.53691539183319 74.35811474104406) bank37471 +37472 POINT(39.841797505152066 74.63816653905565) bank37472 +37473 POINT(41.53943703914509 73.5748270802756) bank37473 +37474 POINT(40.93857714124878 74.78074932135837) bank37474 +37475 POINT(39.80720248439943 73.0281162963503) bank37475 +37476 POINT(40.68725667617692 73.93668467633681) bank37476 +37477 POINT(41.4942684688171 74.74408243435222) bank37477 +37478 POINT(40.09169457632527 73.11726617408615) bank37478 +37479 POINT(40.48617739089126 74.74439058680082) bank37479 +37480 POINT(41.51156905049417 73.47216085217418) bank37480 +37481 POINT(40.17466295126386 73.36509426809644) bank37481 +37482 POINT(41.424344251223495 74.15521687684927) bank37482 +37483 POINT(39.81965083272996 73.53348823158153) bank37483 +37484 POINT(39.87173527437208 73.59291615442821) bank37484 +37485 POINT(40.644165106138914 74.39711243065788) bank37485 +37486 POINT(40.234627408656465 73.94429467255142) bank37486 +37487 POINT(41.335187533815414 73.98226585275826) bank37487 +37488 POINT(40.59763572145911 74.83883349018689) bank37488 +37489 POINT(40.99746526527198 74.02413405337339) bank37489 +37490 POINT(40.235875801804795 74.68213603406225) bank37490 +37491 POINT(41.039057716319654 74.77003800356697) bank37491 +37492 POINT(40.48842688052488 74.27077871692393) bank37492 +37493 POINT(41.03674190732401 74.66229486865636) bank37493 +37494 POINT(40.28404089542597 73.98359258423939) bank37494 +37495 POINT(40.307619279964754 73.04387194038753) bank37495 +37496 POINT(40.74093763621607 74.44218011997938) bank37496 +37497 POINT(40.94062097022593 74.19179624027628) bank37497 +37498 POINT(39.97694623764673 73.51976124086438) bank37498 +37499 POINT(41.4608499251183 73.77080096516664) bank37499 +37500 POINT(40.28146053800069 74.26670582028162) bank37500 +37501 POINT(41.51077691585525 73.02925915688294) bank37501 +37502 POINT(41.113229593444245 74.99004636611555) bank37502 +37503 POINT(40.15268814683501 73.76313944757806) bank37503 +37504 POINT(39.97719918290456 74.00894258560238) bank37504 +37505 POINT(41.28377577890971 73.04765792519706) bank37505 +37506 POINT(40.49889637935171 73.49443472126865) bank37506 +37507 POINT(41.65863600704852 74.26635957236786) bank37507 +37508 POINT(40.20540324921398 73.46630157178558) bank37508 +37509 POINT(40.06864051139812 73.98294550398184) bank37509 +37510 POINT(41.65195594674998 74.8051086543647) bank37510 +37511 POINT(41.16629107495437 73.47563391466889) bank37511 +37512 POINT(41.32774943051202 73.25921023407386) bank37512 +37513 POINT(41.14945544037666 74.49601182096373) bank37513 +37514 POINT(41.5284662412074 74.25326917045919) bank37514 +37515 POINT(39.74377765020513 73.89164171527237) bank37515 +37516 POINT(40.40039219169956 74.6896316701659) bank37516 +37517 POINT(40.62583783290228 73.70047690622668) bank37517 +37518 POINT(39.72511680854044 74.1818572530517) bank37518 +37519 POINT(40.667882973096226 74.46903474498848) bank37519 +37520 POINT(40.146908462534654 74.17384004063219) bank37520 +37521 POINT(40.72137947513745 74.98420159821796) bank37521 +37522 POINT(41.07909420922163 73.48942267136938) bank37522 +37523 POINT(41.27384676808917 73.95308985951479) bank37523 +37524 POINT(41.687647227626144 74.57464437581162) bank37524 +37525 POINT(41.04127956722758 74.8923813418946) bank37525 +37526 POINT(41.240556335462415 74.65398911176885) bank37526 +37527 POINT(41.375380018997625 73.3445875963705) bank37527 +37528 POINT(40.960399256221365 73.0124740374469) bank37528 +37529 POINT(40.23530895272887 74.1402137114886) bank37529 +37530 POINT(40.466431305085585 73.37539095885485) bank37530 +37531 POINT(40.70145433053359 73.39948745395834) bank37531 +37532 POINT(41.113182780096714 74.94926105906944) bank37532 +37533 POINT(40.03843465004304 73.84272766910712) bank37533 +37534 POINT(40.73660147885703 74.2909934549985) bank37534 +37535 POINT(40.51167716076004 74.65293181219948) bank37535 +37536 POINT(40.44722098692507 73.36829941050306) bank37536 +37537 POINT(41.01776333840892 73.75243829321701) bank37537 +37538 POINT(40.57012751132246 74.4988858703645) bank37538 +37539 POINT(40.83212997770356 73.6892907209491) bank37539 +37540 POINT(41.40933855474335 74.33242406456966) bank37540 +37541 POINT(40.94850181407399 73.71449958029169) bank37541 +37542 POINT(41.00509838636343 74.15857266591793) bank37542 +37543 POINT(40.27703393333735 74.22505000711415) bank37543 +37544 POINT(39.74522542436051 73.03925096544107) bank37544 +37545 POINT(41.14319742123135 73.64975332346208) bank37545 +37546 POINT(41.07054680418148 73.32991883221494) bank37546 +37547 POINT(41.61269180455174 73.84640209845644) bank37547 +37548 POINT(41.49438472458415 73.50554382837619) bank37548 +37549 POINT(41.662767747083116 73.92441560592655) bank37549 +37550 POINT(39.99756400648938 74.74379547476669) bank37550 +37551 POINT(41.2952255407328 74.11575779127226) bank37551 +37552 POINT(41.69992259571297 74.14372713745914) bank37552 +37553 POINT(40.49084341968927 73.07554432158031) bank37553 +37554 POINT(40.510835790131225 73.74385208414026) bank37554 +37555 POINT(40.41721701436033 73.31992569758783) bank37555 +37556 POINT(40.526704546666814 73.87657937422782) bank37556 +37557 POINT(41.34090716402338 73.76630589404014) bank37557 +37558 POINT(41.58100010134176 73.31663056631791) bank37558 +37559 POINT(41.352735472510055 73.56597637153847) bank37559 +37560 POINT(40.97596284505628 74.28443046642734) bank37560 +37561 POINT(40.97879045073131 73.70655231187045) bank37561 +37562 POINT(40.40018079872434 74.80717904693492) bank37562 +37563 POINT(39.91565819867248 74.49013597913458) bank37563 +37564 POINT(41.623556510056936 73.64522072077122) bank37564 +37565 POINT(40.163703254641696 73.93506144608571) bank37565 +37566 POINT(40.680990883965094 73.76664303672467) bank37566 +37567 POINT(40.175295033299584 73.91752256474614) bank37567 +37568 POINT(40.17583626958617 74.1342186729291) bank37568 +37569 POINT(41.26688373479472 74.77090631666981) bank37569 +37570 POINT(40.75667094147429 74.77297607037387) bank37570 +37571 POINT(40.44485232830519 74.85409692501523) bank37571 +37572 POINT(40.61518862180709 74.07193764303383) bank37572 +37573 POINT(41.617141350716174 74.13705157549497) bank37573 +37574 POINT(39.89859324486963 73.9631110396651) bank37574 +37575 POINT(41.5578577023263 73.29538401831385) bank37575 +37576 POINT(41.65997197390495 73.51379251843414) bank37576 +37577 POINT(40.1258928953315 74.14302796778634) bank37577 +37578 POINT(40.1688080766346 74.85620448970576) bank37578 +37579 POINT(40.16525333387353 73.32179498946186) bank37579 +37580 POINT(41.618882673602904 74.0274634103726) bank37580 +37581 POINT(40.79535452429276 74.93388325478922) bank37581 +37582 POINT(41.207011828369026 74.35494902259411) bank37582 +37583 POINT(41.229420572339016 73.62887298951506) bank37583 +37584 POINT(41.08173597783044 74.38194345181172) bank37584 +37585 POINT(41.59255327835234 74.95212872968459) bank37585 +37586 POINT(40.58293153689875 74.19892894294479) bank37586 +37587 POINT(41.16515651757543 74.65099600575903) bank37587 +37588 POINT(41.463416680174774 74.8725859936229) bank37588 +37589 POINT(41.058370559087514 74.02916403792884) bank37589 +37590 POINT(40.72919418133868 74.93912613364209) bank37590 +37591 POINT(39.98289633223057 74.07577892781303) bank37591 +37592 POINT(39.95015235599736 73.0815937600792) bank37592 +37593 POINT(41.31227838696792 74.87403924980131) bank37593 +37594 POINT(41.304237621849346 73.73744420434903) bank37594 +37595 POINT(40.136174761067174 73.39847893118521) bank37595 +37596 POINT(40.55255932171363 73.73662630602713) bank37596 +37597 POINT(41.134341132521975 73.32872648190587) bank37597 +37598 POINT(40.670971773439526 73.32298341671105) bank37598 +37599 POINT(40.43387426367608 74.15560099079214) bank37599 +37600 POINT(39.786351460255254 73.22930535495334) bank37600 +37601 POINT(41.3667874346045 73.15749867785969) bank37601 +37602 POINT(40.46764311238337 74.81816037182415) bank37602 +37603 POINT(41.12330656611761 74.13879457932684) bank37603 +37604 POINT(40.110181512011415 74.38191858178962) bank37604 +37605 POINT(41.22242397249549 73.23477852344963) bank37605 +37606 POINT(39.881694952611994 74.8315509038584) bank37606 +37607 POINT(41.57445418067773 73.00922097615512) bank37607 +37608 POINT(39.8947472716424 74.41043231982346) bank37608 +37609 POINT(41.31998637478447 73.86313661254646) bank37609 +37610 POINT(39.9173238963575 73.34959906076622) bank37610 +37611 POINT(39.833957026966914 73.46356507231933) bank37611 +37612 POINT(41.0879865916049 73.7549239874534) bank37612 +37613 POINT(40.9433410126831 73.33007471072206) bank37613 +37614 POINT(40.99784540962152 73.3093312551568) bank37614 +37615 POINT(41.58071024928345 74.09218020958232) bank37615 +37616 POINT(40.42244765373597 73.89827957988842) bank37616 +37617 POINT(40.35998195040016 73.98200883699332) bank37617 +37618 POINT(40.12520103172988 74.27316365572722) bank37618 +37619 POINT(40.16333472451509 74.52020424473255) bank37619 +37620 POINT(41.53274090638435 73.18113382454453) bank37620 +37621 POINT(40.70848654679436 73.9986302959083) bank37621 +37622 POINT(41.377422913130275 73.19369682609589) bank37622 +37623 POINT(40.84815230934162 74.09147600841506) bank37623 +37624 POINT(41.66768739587071 73.61784934619776) bank37624 +37625 POINT(39.80059965811975 73.72011118075724) bank37625 +37626 POINT(40.61000779334115 73.89418218638882) bank37626 +37627 POINT(40.63843929366858 74.74952931967334) bank37627 +37628 POINT(40.53961874390038 73.64245798520857) bank37628 +37629 POINT(40.752626140018556 73.62557186728961) bank37629 +37630 POINT(40.67581759214859 73.51557142167778) bank37630 +37631 POINT(40.178107091614265 73.15714956669373) bank37631 +37632 POINT(39.89629853839016 74.23745713254579) bank37632 +37633 POINT(39.814079562483414 73.46386755125364) bank37633 +37634 POINT(40.805519235173115 73.69900660407569) bank37634 +37635 POINT(40.14251496802772 73.90314580361071) bank37635 +37636 POINT(40.157232254094495 74.16278158784249) bank37636 +37637 POINT(40.80617378881967 73.27182895098167) bank37637 +37638 POINT(40.158597682825146 74.17325664221782) bank37638 +37639 POINT(39.74162881617295 74.97503787686219) bank37639 +37640 POINT(41.0688823649275 74.53770405426332) bank37640 +37641 POINT(41.024371874315385 73.67245741963265) bank37641 +37642 POINT(40.24098556518982 74.50643545632971) bank37642 +37643 POINT(41.12564714336215 74.92489016650501) bank37643 +37644 POINT(40.7020144295214 74.66292353266358) bank37644 +37645 POINT(41.22725243533639 73.37821752879015) bank37645 +37646 POINT(40.395641440704644 75.00461207882363) bank37646 +37647 POINT(40.740528019265824 74.89356681469683) bank37647 +37648 POINT(40.08805435907748 73.05179863003718) bank37648 +37649 POINT(41.456461722423946 73.77546095038755) bank37649 +37650 POINT(41.36853688272246 73.25519069237178) bank37650 +37651 POINT(41.24845427532385 74.007845226766) bank37651 +37652 POINT(39.79926626603498 74.15275178984726) bank37652 +37653 POINT(40.847035400950084 73.22304373329078) bank37653 +37654 POINT(40.55100272955677 74.01545147096482) bank37654 +37655 POINT(41.44912508212318 74.9769702612226) bank37655 +37656 POINT(40.22188184971228 74.68742281405653) bank37656 +37657 POINT(41.70383129145742 74.05850684873604) bank37657 +37658 POINT(40.06447921027667 73.03481210517992) bank37658 +37659 POINT(40.771488471340426 73.26315307095858) bank37659 +37660 POINT(41.35300528890949 74.46036852990065) bank37660 +37661 POINT(41.11150889909502 73.27044077043892) bank37661 +37662 POINT(40.83388863360457 73.48689500839964) bank37662 +37663 POINT(40.92857357837489 74.22631848858121) bank37663 +37664 POINT(40.15825575595937 73.74444150892603) bank37664 +37665 POINT(40.74410221412303 74.29452639431011) bank37665 +37666 POINT(40.96476588142562 74.9415122798334) bank37666 +37667 POINT(40.60465424271251 73.62797770062772) bank37667 +37668 POINT(41.07792217831537 74.89584731301159) bank37668 +37669 POINT(41.44415653259483 73.35986600979989) bank37669 +37670 POINT(41.08601014460064 74.22345756978555) bank37670 +37671 POINT(40.68700293710265 74.45747872466366) bank37671 +37672 POINT(40.3231523799675 73.82854697347348) bank37672 +37673 POINT(41.18250228245647 73.03202773913807) bank37673 +37674 POINT(41.527524424776864 74.80058970822235) bank37674 +37675 POINT(41.20490178975223 74.4671123050607) bank37675 +37676 POINT(40.65096992492675 73.22742442151852) bank37676 +37677 POINT(40.104679864394924 73.37960769740131) bank37677 +37678 POINT(40.96551599893222 74.50808116903909) bank37678 +37679 POINT(41.1997956958396 73.18748115554182) bank37679 +37680 POINT(40.85238244034137 74.45346711465658) bank37680 +37681 POINT(40.362890754787976 74.79211028722652) bank37681 +37682 POINT(41.03624673528983 73.20031782846695) bank37682 +37683 POINT(40.208135900543134 73.97283129320834) bank37683 +37684 POINT(40.670447774170036 73.0259578691452) bank37684 +37685 POINT(41.68512396879041 73.17093563072754) bank37685 +37686 POINT(39.795553076999994 74.67494523498537) bank37686 +37687 POINT(40.23341438495405 74.97943787563068) bank37687 +37688 POINT(40.4171824362556 74.52734137223351) bank37688 +37689 POINT(40.84154337838282 73.092128825845) bank37689 +37690 POINT(40.7560587253637 73.12788677441095) bank37690 +37691 POINT(41.03926249094062 74.25149360329752) bank37691 +37692 POINT(40.93885253562769 73.06952644970444) bank37692 +37693 POINT(40.634876399190084 73.92321298580923) bank37693 +37694 POINT(41.09634681634709 74.66859571012674) bank37694 +37695 POINT(41.21508123874056 73.4762496563765) bank37695 +37696 POINT(39.77972827349453 74.11611250318006) bank37696 +37697 POINT(40.251200395301204 73.51368700357841) bank37697 +37698 POINT(40.457793668284566 73.34375135708504) bank37698 +37699 POINT(40.19045707373584 74.52429934227455) bank37699 +37700 POINT(39.76780976250979 73.88335919393751) bank37700 +37701 POINT(40.27257223004113 74.6712112252887) bank37701 +37702 POINT(41.09440844643451 74.02467812853317) bank37702 +37703 POINT(40.77680962942395 73.52280473600659) bank37703 +37704 POINT(39.93975119065351 73.60378493920142) bank37704 +37705 POINT(40.65414958953638 73.66527096062843) bank37705 +37706 POINT(40.86507803979009 73.98528855911704) bank37706 +37707 POINT(41.264070462307664 74.70716642115472) bank37707 +37708 POINT(39.72057311485516 74.15990651206181) bank37708 +37709 POINT(40.10655375087944 73.77650295838049) bank37709 +37710 POINT(40.703717190372515 74.60603308590473) bank37710 +37711 POINT(39.998622694566215 73.18155500329495) bank37711 +37712 POINT(39.758879966564216 73.51688070513596) bank37712 +37713 POINT(40.84930078764339 74.28950181137745) bank37713 +37714 POINT(40.930678236713824 74.57844784804627) bank37714 +37715 POINT(41.383468951965625 74.95203208455219) bank37715 +37716 POINT(41.05131694842211 74.97095389254629) bank37716 +37717 POINT(40.60162848975452 73.57601018198714) bank37717 +37718 POINT(41.45126303611092 74.75971706412354) bank37718 +37719 POINT(40.25492237146713 74.86585088090484) bank37719 +37720 POINT(39.84575086538244 73.6838195715399) bank37720 +37721 POINT(41.128839111943606 73.53530869895631) bank37721 +37722 POINT(40.33320242079751 74.77530957405982) bank37722 +37723 POINT(41.04848427218717 74.57049687045611) bank37723 +37724 POINT(40.59519987533364 73.1146964795248) bank37724 +37725 POINT(39.89725006637236 74.38498230538111) bank37725 +37726 POINT(40.74521072767511 73.85817032132843) bank37726 +37727 POINT(41.627323235621844 74.22659835204327) bank37727 +37728 POINT(39.941709167534086 73.50665610639167) bank37728 +37729 POINT(41.006665683019484 74.44387432554637) bank37729 +37730 POINT(40.99201741482643 73.4967815509818) bank37730 +37731 POINT(41.13091766493439 74.82087737702957) bank37731 +37732 POINT(41.44195615984881 73.50938172032079) bank37732 +37733 POINT(41.3061568119629 74.80486352140217) bank37733 +37734 POINT(41.13340101632363 74.27854367534799) bank37734 +37735 POINT(40.67771015071428 74.80326814917525) bank37735 +37736 POINT(39.96978068545087 74.82597987561613) bank37736 +37737 POINT(39.977213750678516 74.81327634834668) bank37737 +37738 POINT(41.221580131171265 73.68637835493526) bank37738 +37739 POINT(39.932710167342904 73.46571997635216) bank37739 +37740 POINT(40.30402810428069 73.80143667727384) bank37740 +37741 POINT(40.19345015421182 73.7119835262438) bank37741 +37742 POINT(39.858515495716155 73.49743380531828) bank37742 +37743 POINT(41.39693895018095 74.54925087230012) bank37743 +37744 POINT(40.35754236284426 74.65243200992849) bank37744 +37745 POINT(40.51736750538268 74.54252415002988) bank37745 +37746 POINT(39.860871020194516 73.79663996762008) bank37746 +37747 POINT(41.05569201986969 74.5907624242373) bank37747 +37748 POINT(41.5080730785996 74.46285785681881) bank37748 +37749 POINT(39.7915944153171 74.35021513579683) bank37749 +37750 POINT(41.11986476241536 73.66619200587924) bank37750 +37751 POINT(40.252885894375375 74.54155212486569) bank37751 +37752 POINT(41.36257099749596 73.54319125641429) bank37752 +37753 POINT(40.55806934485171 74.04984699731948) bank37753 +37754 POINT(41.270651269879664 74.20737113105675) bank37754 +37755 POINT(40.532590165755046 73.6627566231258) bank37755 +37756 POINT(41.48955350218641 73.3215293924687) bank37756 +37757 POINT(39.80033467006414 74.49849077422404) bank37757 +37758 POINT(41.42088772793556 74.52515002524825) bank37758 +37759 POINT(41.575603992009285 74.7931470676506) bank37759 +37760 POINT(39.80492657668305 73.81556844087464) bank37760 +37761 POINT(41.71115055931491 74.90851173411414) bank37761 +37762 POINT(41.03972640579902 73.4180466390776) bank37762 +37763 POINT(41.39691217109771 74.52244996087055) bank37763 +37764 POINT(40.24714412023696 73.20462506715658) bank37764 +37765 POINT(40.15730530636083 74.78233464530243) bank37765 +37766 POINT(40.672379005385636 73.19638305237042) bank37766 +37767 POINT(39.80185635202161 74.84768950206995) bank37767 +37768 POINT(41.45623571966031 74.53444110467042) bank37768 +37769 POINT(40.37161740170414 74.5139980762997) bank37769 +37770 POINT(40.90699296367202 73.17050237407348) bank37770 +37771 POINT(41.48716294434045 74.18909841740779) bank37771 +37772 POINT(40.25216504677951 74.63784259629868) bank37772 +37773 POINT(41.50683782750531 73.33365156123459) bank37773 +37774 POINT(40.45734480400125 74.79674447352264) bank37774 +37775 POINT(40.90409607754988 73.08035932213038) bank37775 +37776 POINT(40.99036467509204 74.31308537323007) bank37776 +37777 POINT(40.91936780381185 73.72742660551165) bank37777 +37778 POINT(40.86357315591748 74.37333101996892) bank37778 +37779 POINT(40.31086305355828 74.75755252361861) bank37779 +37780 POINT(40.06912845051295 74.64639537982374) bank37780 +37781 POINT(39.94785445675616 74.80057977901602) bank37781 +37782 POINT(41.04137299067687 73.64445206892923) bank37782 +37783 POINT(41.414790139929686 74.93235962218024) bank37783 +37784 POINT(40.38683963767883 74.49536071224661) bank37784 +37785 POINT(41.4920292715272 74.80728890986231) bank37785 +37786 POINT(40.28464145211596 74.72937128677127) bank37786 +37787 POINT(40.14911900300799 73.890792844037) bank37787 +37788 POINT(40.65871771096818 74.22197242800164) bank37788 +37789 POINT(40.25158342696503 73.87854547836908) bank37789 +37790 POINT(41.081726528704344 73.29505831692472) bank37790 +37791 POINT(40.30520034377319 74.95096028483765) bank37791 +37792 POINT(41.16835544587895 73.83725263949309) bank37792 +37793 POINT(39.86643836700829 73.24156706949253) bank37793 +37794 POINT(40.19937459241898 74.32276006832078) bank37794 +37795 POINT(40.224316015841524 73.76868490060012) bank37795 +37796 POINT(41.22775478616904 74.74382816225506) bank37796 +37797 POINT(40.2657556509806 73.77387274266404) bank37797 +37798 POINT(40.748483919100956 74.36504895361334) bank37798 +37799 POINT(40.43229789718094 74.79297265849354) bank37799 +37800 POINT(40.32089380939609 74.60683411121596) bank37800 +37801 POINT(41.096874691104276 73.53790304076941) bank37801 +37802 POINT(40.08340064732275 73.40352665776041) bank37802 +37803 POINT(41.55324146241183 74.369577430515) bank37803 +37804 POINT(41.28337504934807 73.96979996452579) bank37804 +37805 POINT(39.95128704567567 73.17259414921746) bank37805 +37806 POINT(40.75388372394791 73.89887010018136) bank37806 +37807 POINT(41.48529617318053 74.33052646980467) bank37807 +37808 POINT(41.3055824908381 74.25558153677908) bank37808 +37809 POINT(41.703660150792196 74.78038539852531) bank37809 +37810 POINT(41.32871397012013 73.67114774406909) bank37810 +37811 POINT(41.70981630807197 74.97162918243646) bank37811 +37812 POINT(40.584910657563725 74.79608870072501) bank37812 +37813 POINT(41.31664121209424 74.03386793179176) bank37813 +37814 POINT(40.82942271711401 74.89427446511793) bank37814 +37815 POINT(41.19742057016658 74.20196005637253) bank37815 +37816 POINT(40.07277518162735 74.36699338976271) bank37816 +37817 POINT(40.185241504118004 74.85716501648665) bank37817 +37818 POINT(40.464049438969454 73.17280554556959) bank37818 +37819 POINT(40.527304792797295 74.10441097917663) bank37819 +37820 POINT(39.8941585384912 73.71292911533189) bank37820 +37821 POINT(40.12856424674559 73.18199430021275) bank37821 +37822 POINT(40.825003605733514 74.39728167126916) bank37822 +37823 POINT(41.48766518025555 73.38813320557988) bank37823 +37824 POINT(40.73671537743919 74.20571449257864) bank37824 +37825 POINT(41.603426279144145 74.72702134709589) bank37825 +37826 POINT(40.217910395138674 73.4099895586926) bank37826 +37827 POINT(41.06150013429953 73.56554948678918) bank37827 +37828 POINT(40.293909990326846 74.94921827139815) bank37828 +37829 POINT(40.352892036259526 73.263919474366) bank37829 +37830 POINT(41.04530443582352 73.27333056069482) bank37830 +37831 POINT(40.47714976709317 73.13063783036085) bank37831 +37832 POINT(41.57547735174484 74.96982737171949) bank37832 +37833 POINT(40.57681553936338 74.7871979894445) bank37833 +37834 POINT(41.29883863550057 73.15764896767274) bank37834 +37835 POINT(41.27392136690407 73.0248792488644) bank37835 +37836 POINT(41.50475395725597 74.95355651096718) bank37836 +37837 POINT(41.54068879031383 73.81147384797761) bank37837 +37838 POINT(40.94266350035633 73.37629506268333) bank37838 +37839 POINT(40.67903118254425 74.531681641159) bank37839 +37840 POINT(41.526102653651165 74.39762147086064) bank37840 +37841 POINT(40.560357953548106 74.22626564724877) bank37841 +37842 POINT(40.68933024789098 74.3493205019478) bank37842 +37843 POINT(41.02168080722589 73.87109658722983) bank37843 +37844 POINT(40.289432970239304 74.71340163423142) bank37844 +37845 POINT(40.561391058658764 74.836774885941) bank37845 +37846 POINT(40.265456703113884 73.2016519941801) bank37846 +37847 POINT(40.98729849115644 73.35485687628298) bank37847 +37848 POINT(41.04040790363032 74.32096768638569) bank37848 +37849 POINT(41.01988054651169 74.89833792516784) bank37849 +37850 POINT(40.108943300915094 73.85786268184349) bank37850 +37851 POINT(40.970399456107856 74.55175059755791) bank37851 +37852 POINT(41.68716632971357 73.5254330063171) bank37852 +37853 POINT(41.44498333915757 74.81899317858309) bank37853 +37854 POINT(41.53152835394743 74.68446539037912) bank37854 +37855 POINT(40.348902301028275 73.92795765618845) bank37855 +37856 POINT(39.76611869521345 74.59260297728747) bank37856 +37857 POINT(41.49627222063539 74.72506853814076) bank37857 +37858 POINT(40.36557035349778 74.5142037406132) bank37858 +37859 POINT(40.768123824659185 74.1091490182214) bank37859 +37860 POINT(41.43991237251851 73.95912299192703) bank37860 +37861 POINT(41.687685389717124 74.36780915909281) bank37861 +37862 POINT(41.33911464756999 74.43870025588713) bank37862 +37863 POINT(40.81760728325677 73.59669981497443) bank37863 +37864 POINT(40.53069286165253 74.23861430585983) bank37864 +37865 POINT(41.589830942848224 74.22963872165431) bank37865 +37866 POINT(40.564144319763635 73.36321958420321) bank37866 +37867 POINT(39.73253941577341 73.77628029447881) bank37867 +37868 POINT(41.160998229533426 73.3608832883249) bank37868 +37869 POINT(40.780138154914695 73.49841118516113) bank37869 +37870 POINT(39.878034540003014 74.99223328521849) bank37870 +37871 POINT(41.01995122456832 74.00590683931111) bank37871 +37872 POINT(39.94201256064584 73.06141333207604) bank37872 +37873 POINT(41.44895303818898 73.52322157884993) bank37873 +37874 POINT(40.72755749089444 73.02145826464107) bank37874 +37875 POINT(40.67472081858342 73.61258770418786) bank37875 +37876 POINT(40.07461137344185 74.30715291184792) bank37876 +37877 POINT(40.7151896985927 74.02318603415617) bank37877 +37878 POINT(41.49429569655457 73.24205549133475) bank37878 +37879 POINT(40.752184748293004 73.98660105726317) bank37879 +37880 POINT(41.049265424085604 73.377146237172) bank37880 +37881 POINT(40.257266776242794 74.43853407667555) bank37881 +37882 POINT(40.33131702283157 74.58731193391492) bank37882 +37883 POINT(40.567303967524 74.06346747450768) bank37883 +37884 POINT(41.193863235446635 73.27101150283212) bank37884 +37885 POINT(40.601790666294235 73.18026843382951) bank37885 +37886 POINT(41.563807137137665 73.56419262703359) bank37886 +37887 POINT(40.33409337428251 74.93403610462614) bank37887 +37888 POINT(41.09755968048922 74.69763951978312) bank37888 +37889 POINT(40.46663315517122 73.56535966529889) bank37889 +37890 POINT(41.15344898730995 73.24756202028964) bank37890 +37891 POINT(41.334998067139935 74.3529288540547) bank37891 +37892 POINT(41.627494836648495 73.41025464980646) bank37892 +37893 POINT(40.423274030993845 73.946847137714) bank37893 +37894 POINT(41.06131728278234 74.31776045478753) bank37894 +37895 POINT(40.81562469695497 74.34671155579274) bank37895 +37896 POINT(40.87594252958013 74.58221467025353) bank37896 +37897 POINT(41.62013409000947 74.53493920035567) bank37897 +37898 POINT(40.37379561771603 74.98427739410789) bank37898 +37899 POINT(41.01146514665039 73.52703455155913) bank37899 +37900 POINT(40.164979855078066 73.06384238796858) bank37900 +37901 POINT(39.91471582843974 73.3252074327153) bank37901 +37902 POINT(41.59868721132357 74.52191020282406) bank37902 +37903 POINT(41.21819496862477 73.92531002058331) bank37903 +37904 POINT(40.649911977133044 74.26481433406298) bank37904 +37905 POINT(40.46724000806313 74.68257146529258) bank37905 +37906 POINT(39.90738281928507 74.65437687650001) bank37906 +37907 POINT(39.85253941437308 74.49424729152987) bank37907 +37908 POINT(41.16013506844137 74.07918668296539) bank37908 +37909 POINT(40.227644151461455 73.35118974823983) bank37909 +37910 POINT(41.340716572836726 73.45910327196796) bank37910 +37911 POINT(40.54376759996456 73.83839671954212) bank37911 +37912 POINT(40.93239933774995 73.77301125064454) bank37912 +37913 POINT(39.7912292294162 74.75983973680627) bank37913 +37914 POINT(40.84101499576502 74.64437566844232) bank37914 +37915 POINT(40.4416585418858 74.93687664739896) bank37915 +37916 POINT(40.699148682284736 73.47064438814454) bank37916 +37917 POINT(40.82344394488659 73.56975042904101) bank37917 +37918 POINT(39.89316323774824 73.16330310653638) bank37918 +37919 POINT(41.33107621268826 73.13198768488394) bank37919 +37920 POINT(39.850466849413756 74.9074201310682) bank37920 +37921 POINT(41.08775966387665 73.05339700153047) bank37921 +37922 POINT(39.72047147578705 74.48119190927322) bank37922 +37923 POINT(40.31718314114363 74.8608230139943) bank37923 +37924 POINT(40.040852742497904 74.27796211287921) bank37924 +37925 POINT(39.986766888417826 74.27847444399583) bank37925 +37926 POINT(40.36903378721273 74.88405485321154) bank37926 +37927 POINT(41.428964102742 74.48844650352682) bank37927 +37928 POINT(41.704664409063454 73.67562730944535) bank37928 +37929 POINT(41.160442263246956 74.93763252070106) bank37929 +37930 POINT(41.18198074086262 74.11456821341395) bank37930 +37931 POINT(40.000231665843856 73.31059021184899) bank37931 +37932 POINT(40.42907205401601 74.66919104334713) bank37932 +37933 POINT(40.22124224067979 73.0514337535498) bank37933 +37934 POINT(39.861481449471405 74.17561591895777) bank37934 +37935 POINT(40.46304743395583 74.62817304126027) bank37935 +37936 POINT(40.31849558870659 73.95019019580853) bank37936 +37937 POINT(39.9291953461523 74.06437430061075) bank37937 +37938 POINT(40.66700679782656 74.64935215999316) bank37938 +37939 POINT(40.20243640460128 74.46115599052972) bank37939 +37940 POINT(39.7523882810532 74.225825633989) bank37940 +37941 POINT(39.98691872573934 74.78967335563988) bank37941 +37942 POINT(40.14748914844866 74.59242558627624) bank37942 +37943 POINT(40.05939240826646 74.55938908819309) bank37943 +37944 POINT(40.07920929727961 73.42582532569818) bank37944 +37945 POINT(41.121972435474284 74.597938289025) bank37945 +37946 POINT(41.60942780379341 73.8687156510499) bank37946 +37947 POINT(41.18518761135674 74.50624452674803) bank37947 +37948 POINT(41.301446932290624 73.80469879058364) bank37948 +37949 POINT(40.72536995969608 74.46066812454626) bank37949 +37950 POINT(41.6314918750237 73.54929865322231) bank37950 +37951 POINT(41.5087034964103 74.79122416558961) bank37951 +37952 POINT(40.63917528089636 74.7027847717992) bank37952 +37953 POINT(40.541775847058304 73.38070621618135) bank37953 +37954 POINT(41.3752354320637 73.06819063839495) bank37954 +37955 POINT(40.31634722301869 74.78679617906386) bank37955 +37956 POINT(41.615415134907025 74.51801637897667) bank37956 +37957 POINT(40.38419500587436 74.77526306308371) bank37957 +37958 POINT(40.231817426967204 73.42594776692096) bank37958 +37959 POINT(40.59630076290706 74.69156264677491) bank37959 +37960 POINT(41.088755483558494 74.7720535715217) bank37960 +37961 POINT(40.94693465670512 73.87542832261903) bank37961 +37962 POINT(41.00915019205798 73.42910361378466) bank37962 +37963 POINT(41.189766445343956 74.45468304053841) bank37963 +37964 POINT(40.15917706192276 74.63976277665724) bank37964 +37965 POINT(39.97424583369616 74.73680537453707) bank37965 +37966 POINT(40.988970348858444 73.24091360330556) bank37966 +37967 POINT(40.56358056848408 73.82249932308211) bank37967 +37968 POINT(40.96435090537314 74.0684845378257) bank37968 +37969 POINT(40.23637079281353 74.95221142091975) bank37969 +37970 POINT(40.916650998649224 74.75645065907183) bank37970 +37971 POINT(40.31334300967937 74.6033975004384) bank37971 +37972 POINT(41.18484561721421 74.29133809090196) bank37972 +37973 POINT(41.23001893834743 74.68218446052771) bank37973 +37974 POINT(40.50965977752556 73.4467033744541) bank37974 +37975 POINT(40.102394172250555 73.18926020044374) bank37975 +37976 POINT(40.06619561583703 73.19221429988166) bank37976 +37977 POINT(40.68861059905658 74.03217785780075) bank37977 +37978 POINT(40.904118879757824 73.95641589080844) bank37978 +37979 POINT(41.16926842868731 74.21732561961021) bank37979 +37980 POINT(40.02038368966528 74.73501827271329) bank37980 +37981 POINT(39.72764865708189 73.84207917266515) bank37981 +37982 POINT(39.82130439560809 73.36372853269465) bank37982 +37983 POINT(41.31108882570228 74.93799456610589) bank37983 +37984 POINT(40.64439497755316 73.49516944235285) bank37984 +37985 POINT(40.47478075935847 74.53709248138681) bank37985 +37986 POINT(41.458656311619315 73.52620934169148) bank37986 +37987 POINT(40.337791523902624 74.87127990087478) bank37987 +37988 POINT(40.22141106698434 74.68707032124703) bank37988 +37989 POINT(41.3605129646585 74.14807786051766) bank37989 +37990 POINT(40.210696261564586 74.16681774049405) bank37990 +37991 POINT(40.365103576000664 73.16344784144643) bank37991 +37992 POINT(39.892294226232316 74.68959632917047) bank37992 +37993 POINT(40.63698772215425 74.96032005743685) bank37993 +37994 POINT(40.13525182146567 73.36314941723408) bank37994 +37995 POINT(41.12042559187466 73.33202958100117) bank37995 +37996 POINT(40.634207102535186 73.79846260569568) bank37996 +37997 POINT(41.223350037812956 73.25573850547981) bank37997 +37998 POINT(40.3584160402001 74.71468931621177) bank37998 +37999 POINT(40.10884371670565 74.57733522016244) bank37999 +38000 POINT(39.81734758089075 73.33348989030169) bank38000 +38001 POINT(40.58558297947448 74.53911573778043) bank38001 +38002 POINT(40.42885476282692 74.75293655052849) bank38002 +38003 POINT(40.64519191162089 73.93434525991165) bank38003 +38004 POINT(41.366901104825814 73.84246348761036) bank38004 +38005 POINT(40.64569194904095 74.204921192111) bank38005 +38006 POINT(41.521305703233736 74.90194340519422) bank38006 +38007 POINT(40.997424362456805 73.85354962216269) bank38007 +38008 POINT(41.07104817554838 74.60954150203614) bank38008 +38009 POINT(39.97691916244128 73.03532202994825) bank38009 +38010 POINT(41.08253408682889 73.30549236554545) bank38010 +38011 POINT(40.179964094526106 73.6483622011172) bank38011 +38012 POINT(40.789367773546424 73.66105869348148) bank38012 +38013 POINT(41.64521351629345 73.31114348544823) bank38013 +38014 POINT(40.31320085881517 74.47070853865559) bank38014 +38015 POINT(40.45589344389586 74.01701660098288) bank38015 +38016 POINT(40.55839546501969 74.73988714514452) bank38016 +38017 POINT(40.100940962717495 73.91202059657353) bank38017 +38018 POINT(40.23587755469785 73.97990374333739) bank38018 +38019 POINT(41.165018240932824 73.66791063341118) bank38019 +38020 POINT(41.66433223598523 73.1365241546564) bank38020 +38021 POINT(40.693524592295475 74.27278528949844) bank38021 +38022 POINT(41.19498541084622 73.91079477323602) bank38022 +38023 POINT(39.792026834079586 73.46422918336387) bank38023 +38024 POINT(41.36835106158246 74.86536645463676) bank38024 +38025 POINT(40.817016158512246 73.55093707381464) bank38025 +38026 POINT(40.25683079551513 73.69657900797094) bank38026 +38027 POINT(40.88902188868701 74.14534147148923) bank38027 +38028 POINT(40.746205460989565 73.25991274420231) bank38028 +38029 POINT(40.15899849171034 73.23872294623004) bank38029 +38030 POINT(41.13038547153264 73.31419539512927) bank38030 +38031 POINT(40.26391167892799 73.34389162091968) bank38031 +38032 POINT(40.94315700223063 73.28849267311213) bank38032 +38033 POINT(39.996226811687 73.24476204386295) bank38033 +38034 POINT(41.02528049267902 74.21428585879724) bank38034 +38035 POINT(40.32443548020778 74.84342816732985) bank38035 +38036 POINT(41.15705699540054 74.82795493432258) bank38036 +38037 POINT(40.06683188969247 73.11806710739205) bank38037 +38038 POINT(40.8971746319592 74.20020363736114) bank38038 +38039 POINT(40.027884570061325 74.90983732992098) bank38039 +38040 POINT(40.41667855794613 74.17595251862939) bank38040 +38041 POINT(41.140552129887254 74.95332585049327) bank38041 +38042 POINT(40.36418045339392 74.42049696308796) bank38042 +38043 POINT(40.686399783007744 73.83522622917243) bank38043 +38044 POINT(40.6818046927136 74.78672772880522) bank38044 +38045 POINT(40.444840411747556 73.90643009565223) bank38045 +38046 POINT(41.51213342487728 73.93209805147956) bank38046 +38047 POINT(40.64688108585463 74.1125484575862) bank38047 +38048 POINT(40.31200072800567 73.47714670381669) bank38048 +38049 POINT(39.98367608410048 73.04298948256996) bank38049 +38050 POINT(41.39555914859613 73.35063066946842) bank38050 +38051 POINT(40.820176713228314 73.31346373185109) bank38051 +38052 POINT(40.85174617772625 74.45688883650121) bank38052 +38053 POINT(41.59743570478259 73.40500317293095) bank38053 +38054 POINT(40.62431473667202 74.20720287217071) bank38054 +38055 POINT(41.19704482447267 74.9557754242144) bank38055 +38056 POINT(40.35827831400383 73.2281368018732) bank38056 +38057 POINT(41.344174266326974 74.87420200828169) bank38057 +38058 POINT(41.41378318682573 73.10162154611083) bank38058 +38059 POINT(40.18753476452531 74.82545195520555) bank38059 +38060 POINT(39.78677460629346 73.6645745578149) bank38060 +38061 POINT(41.263200402784214 74.32068757931091) bank38061 +38062 POINT(40.87545088561239 73.48291132797306) bank38062 +38063 POINT(41.11132825590168 73.81645178799795) bank38063 +38064 POINT(39.752013605683075 73.57352544564903) bank38064 +38065 POINT(41.27175438623649 74.72104617357029) bank38065 +38066 POINT(39.93404969054837 74.72965424437805) bank38066 +38067 POINT(39.95664782988173 73.16873427653283) bank38067 +38068 POINT(41.171629842402865 73.25503169268823) bank38068 +38069 POINT(40.46600316572284 73.11222906504075) bank38069 +38070 POINT(40.66170099742737 73.27273269665189) bank38070 +38071 POINT(41.03776459401867 73.80517525885178) bank38071 +38072 POINT(40.27856257241566 74.36697164431959) bank38072 +38073 POINT(41.56558914787455 73.55104652704188) bank38073 +38074 POINT(41.49687951296311 74.49403392808524) bank38074 +38075 POINT(40.59544164265761 74.69411217844942) bank38075 +38076 POINT(40.333495217009975 73.1140671855185) bank38076 +38077 POINT(40.65687295435005 73.37574948768071) bank38077 +38078 POINT(39.8695706716119 73.63040789745769) bank38078 +38079 POINT(41.5389568956582 74.39156370216757) bank38079 +38080 POINT(40.858413232302574 74.92276700021851) bank38080 +38081 POINT(39.84536861260612 73.346183287024) bank38081 +38082 POINT(40.299838671164714 74.99650572045462) bank38082 +38083 POINT(41.33985320579929 74.43876207644959) bank38083 +38084 POINT(39.87889141438716 74.12808548110785) bank38084 +38085 POINT(40.544835562233544 74.29961678440564) bank38085 +38086 POINT(41.70717550576128 74.06892501639241) bank38086 +38087 POINT(40.99940836854541 73.79944226368175) bank38087 +38088 POINT(39.764690508304845 73.25328423250555) bank38088 +38089 POINT(41.43290417221572 73.41564497409034) bank38089 +38090 POINT(41.6085001161448 73.53427484736076) bank38090 +38091 POINT(41.096993254850204 73.32059868320535) bank38091 +38092 POINT(39.84545669564226 74.74174896811341) bank38092 +38093 POINT(41.331744357882286 74.88315736026023) bank38093 +38094 POINT(40.37515760829298 73.03469357684124) bank38094 +38095 POINT(41.63185814281985 74.56146222856371) bank38095 +38096 POINT(41.17861923935318 74.08036376187812) bank38096 +38097 POINT(40.49205364188768 74.51835139621369) bank38097 +38098 POINT(40.30162253464502 74.756944375988) bank38098 +38099 POINT(40.43818767909444 73.96050497538859) bank38099 +38100 POINT(39.89008599963178 74.93106230515247) bank38100 +38101 POINT(40.323875517085185 73.49287127650726) bank38101 +38102 POINT(40.817504835628775 74.72848469627326) bank38102 +38103 POINT(40.51222101238955 74.50533225987371) bank38103 +38104 POINT(39.90195128313361 74.33706425582433) bank38104 +38105 POINT(41.10380367332186 74.87704125260171) bank38105 +38106 POINT(41.230254479535844 74.77366718809317) bank38106 +38107 POINT(40.27611462506158 74.98589150096603) bank38107 +38108 POINT(40.48526201099862 73.72269593494298) bank38108 +38109 POINT(40.793248021779235 73.90708331975239) bank38109 +38110 POINT(41.395938844998014 74.83126413806845) bank38110 +38111 POINT(40.0033746664689 74.80327863312696) bank38111 +38112 POINT(40.22957618762463 73.72265417218516) bank38112 +38113 POINT(40.293188059941116 73.29137674189373) bank38113 +38114 POINT(40.72257437217891 74.06060561980703) bank38114 +38115 POINT(40.456924776917155 74.06256897067706) bank38115 +38116 POINT(40.385165302281045 73.41214114763729) bank38116 +38117 POINT(41.56555344581405 74.0518385125414) bank38117 +38118 POINT(39.86316317846917 73.95626507888986) bank38118 +38119 POINT(41.06871717197636 74.72711787258122) bank38119 +38120 POINT(41.079745297484976 73.53719776910317) bank38120 +38121 POINT(39.76030229051659 74.62692892741882) bank38121 +38122 POINT(41.475340609650615 73.2861817183246) bank38122 +38123 POINT(40.724171449146034 73.98880517146661) bank38123 +38124 POINT(41.19798448854687 74.16575538393009) bank38124 +38125 POINT(40.66531452285789 73.63187997175729) bank38125 +38126 POINT(41.53190539150597 74.69498886321703) bank38126 +38127 POINT(40.9846368816367 73.97002827476629) bank38127 +38128 POINT(41.54723697817977 73.08037263585625) bank38128 +38129 POINT(41.267222397507624 74.86737274010969) bank38129 +38130 POINT(40.36458999404472 74.56054437507713) bank38130 +38131 POINT(41.53127181402947 73.92574750684692) bank38131 +38132 POINT(40.05871972394485 73.98620238444794) bank38132 +38133 POINT(40.9803763758337 74.52517847584883) bank38133 +38134 POINT(40.09313834625195 73.48238948729374) bank38134 +38135 POINT(39.72622192112105 73.32681692171971) bank38135 +38136 POINT(39.84334180676468 74.39776848946174) bank38136 +38137 POINT(40.428821833737814 74.70583188944389) bank38137 +38138 POINT(39.87666912460245 74.6629806856912) bank38138 +38139 POINT(40.57379962495675 73.4606403555291) bank38139 +38140 POINT(40.82284008637284 73.05014323105206) bank38140 +38141 POINT(40.50971220933181 73.1619119607886) bank38141 +38142 POINT(41.683699576053854 74.40891521096468) bank38142 +38143 POINT(40.45511076682526 74.05115575803035) bank38143 +38144 POINT(41.597929009435816 73.12951564028054) bank38144 +38145 POINT(41.24050934800547 73.02676555886775) bank38145 +38146 POINT(41.69391274474529 74.15954170989076) bank38146 +38147 POINT(41.232004072934984 74.24970247128411) bank38147 +38148 POINT(41.1832171083463 74.77954393082817) bank38148 +38149 POINT(41.020918303376504 74.59372176373861) bank38149 +38150 POINT(41.016195785815675 73.47970093384524) bank38150 +38151 POINT(40.44870344602409 73.46382420238896) bank38151 +38152 POINT(40.15455626236514 74.56875914999678) bank38152 +38153 POINT(40.779481053900504 74.62557435135834) bank38153 +38154 POINT(41.39206686601085 73.35800307988471) bank38154 +38155 POINT(40.03570055384002 73.47072866633275) bank38155 +38156 POINT(40.59086011071743 74.40016130232539) bank38156 +38157 POINT(41.62438648850222 73.77212430961283) bank38157 +38158 POINT(39.74157914759438 73.40205614387918) bank38158 +38159 POINT(41.21952743048857 73.14208193206282) bank38159 +38160 POINT(40.49087944154786 74.66585349055295) bank38160 +38161 POINT(40.879073612293425 73.03704813446498) bank38161 +38162 POINT(39.76113021236248 73.77722855802982) bank38162 +38163 POINT(40.493754140046875 73.99271734936097) bank38163 +38164 POINT(41.39856184803975 73.10848949548661) bank38164 +38165 POINT(41.12887921047347 73.55462516138296) bank38165 +38166 POINT(40.467688893346576 74.42062244746366) bank38166 +38167 POINT(40.865320319523946 73.09966848239173) bank38167 +38168 POINT(39.794016041612004 74.03689520877846) bank38168 +38169 POINT(41.43151649016907 73.205286303095) bank38169 +38170 POINT(40.46030860375191 73.52843119732806) bank38170 +38171 POINT(41.275155111722015 74.13223110497157) bank38171 +38172 POINT(41.38559970913607 73.74097168039795) bank38172 +38173 POINT(40.322062811851325 73.8169809258485) bank38173 +38174 POINT(39.833919235802355 73.44039983158314) bank38174 +38175 POINT(41.33005327799343 73.76796899925175) bank38175 +38176 POINT(41.56144193305601 74.5214959315961) bank38176 +38177 POINT(40.943592434104126 73.73888151812626) bank38177 +38178 POINT(40.60191996824287 73.33961410460776) bank38178 +38179 POINT(40.482796619086784 73.19588326640415) bank38179 +38180 POINT(41.03189350275468 74.62896963887582) bank38180 +38181 POINT(40.152151111783766 73.3581917199301) bank38181 +38182 POINT(41.34579825642926 73.00861737862711) bank38182 +38183 POINT(40.60703380063706 74.88424553583616) bank38183 +38184 POINT(41.209506550358014 73.92037062202144) bank38184 +38185 POINT(41.35832197397186 74.69153023507307) bank38185 +38186 POINT(39.828601743793236 73.58169184267035) bank38186 +38187 POINT(39.98074090742637 74.43209588666961) bank38187 +38188 POINT(40.16307258976406 73.05450648218353) bank38188 +38189 POINT(40.67161348428806 73.43568401248248) bank38189 +38190 POINT(40.269704370131 74.61215242973715) bank38190 +38191 POINT(39.7881988749989 74.2827616190376) bank38191 +38192 POINT(41.402033060563575 73.4930824172578) bank38192 +38193 POINT(39.946076534490054 73.97249879297502) bank38193 +38194 POINT(39.727127231437194 73.0502376769113) bank38194 +38195 POINT(40.01620485630089 74.34856686361728) bank38195 +38196 POINT(41.6550401732766 74.62145076371662) bank38196 +38197 POINT(40.99641286702783 74.4542997551242) bank38197 +38198 POINT(41.27734533634467 74.57446353643606) bank38198 +38199 POINT(39.91254225854573 74.4639874570915) bank38199 +38200 POINT(40.86078255582261 73.34423645430569) bank38200 +38201 POINT(41.64095332408861 73.56156579291354) bank38201 +38202 POINT(41.18458783082438 74.79878116463966) bank38202 +38203 POINT(41.41108911556005 73.33354557535047) bank38203 +38204 POINT(41.02172307202909 74.51909333132573) bank38204 +38205 POINT(40.815273131591745 73.37403066762437) bank38205 +38206 POINT(39.94201316923288 73.95384453358056) bank38206 +38207 POINT(41.65524332164293 74.29624897925474) bank38207 +38208 POINT(40.41663345900363 74.8323626539975) bank38208 +38209 POINT(40.34433371336004 73.93391616283887) bank38209 +38210 POINT(40.682480750143775 73.60256512059073) bank38210 +38211 POINT(40.234279313720386 73.04476063949576) bank38211 +38212 POINT(41.17964106513934 73.3602594902827) bank38212 +38213 POINT(40.496346701774186 74.46744872764478) bank38213 +38214 POINT(40.34080766757367 73.70316939577182) bank38214 +38215 POINT(40.99280849972489 73.9508485115686) bank38215 +38216 POINT(41.433865283163826 74.48141360262989) bank38216 +38217 POINT(40.157535020675766 73.13332047205915) bank38217 +38218 POINT(40.4829151494062 73.14222857619947) bank38218 +38219 POINT(41.14921583983177 73.10621253508879) bank38219 +38220 POINT(40.55034939810807 74.28157412854252) bank38220 +38221 POINT(41.38604246827598 74.14381341093784) bank38221 +38222 POINT(39.896979741038884 73.27283472391848) bank38222 +38223 POINT(40.445911382778135 74.00035746641505) bank38223 +38224 POINT(41.08789253079306 73.76371864436004) bank38224 +38225 POINT(40.9653495471195 73.54740208345345) bank38225 +38226 POINT(40.3685497130516 73.09997872138803) bank38226 +38227 POINT(41.218557439779595 74.01753406668138) bank38227 +38228 POINT(41.109638502928505 73.73139710938608) bank38228 +38229 POINT(39.79997186311004 73.65603355593191) bank38229 +38230 POINT(40.56561635226693 73.49951763846246) bank38230 +38231 POINT(40.29523982865088 74.8733162197149) bank38231 +38232 POINT(41.32876150303075 74.15836866642736) bank38232 +38233 POINT(41.34251552730023 74.26186077203846) bank38233 +38234 POINT(40.872437328746045 73.76572054476225) bank38234 +38235 POINT(41.35916807191508 73.73124070912195) bank38235 +38236 POINT(41.066152860137905 73.84146204582576) bank38236 +38237 POINT(39.870121062844596 74.75299514228428) bank38237 +38238 POINT(40.2715453379518 73.08721925595606) bank38238 +38239 POINT(40.294730233031835 73.5698985950594) bank38239 +38240 POINT(41.25000976418743 74.13678687477585) bank38240 +38241 POINT(41.61851063441277 74.22247762191647) bank38241 +38242 POINT(41.02958054003439 74.67338819167713) bank38242 +38243 POINT(39.78953356543979 74.28344645042914) bank38243 +38244 POINT(40.1634060881655 73.19753355602577) bank38244 +38245 POINT(40.39533943692735 73.84215029351947) bank38245 +38246 POINT(40.70452822613568 73.27585367403337) bank38246 +38247 POINT(40.92892277290881 74.61712490922066) bank38247 +38248 POINT(40.13735419880335 73.8403562037514) bank38248 +38249 POINT(41.22674850711715 74.84960208745791) bank38249 +38250 POINT(40.799702949484676 74.71368610917547) bank38250 +38251 POINT(40.036289188199284 73.66707853036014) bank38251 +38252 POINT(40.61690992040035 74.99357502393698) bank38252 +38253 POINT(40.55655498046231 74.44236023059199) bank38253 +38254 POINT(41.62433689682733 74.9225376976999) bank38254 +38255 POINT(40.46430204550189 74.0926918316033) bank38255 +38256 POINT(41.53579358872784 73.65737920251199) bank38256 +38257 POINT(40.78273410069181 74.19547062831896) bank38257 +38258 POINT(40.88213856651411 74.68335684502368) bank38258 +38259 POINT(40.269865556864104 74.06370773912128) bank38259 +38260 POINT(39.832368692021134 73.79102272346528) bank38260 +38261 POINT(40.01052945192393 74.15321059337812) bank38261 +38262 POINT(41.638942546098775 73.83106018481696) bank38262 +38263 POINT(40.62185915918223 73.97344024630894) bank38263 +38264 POINT(41.38836269906145 74.12054383362116) bank38264 +38265 POINT(40.29886473377179 74.76040064668958) bank38265 +38266 POINT(41.394312404094 73.64851542283752) bank38266 +38267 POINT(39.872117705202676 74.84639462515601) bank38267 +38268 POINT(41.33724018908708 73.63634262021851) bank38268 +38269 POINT(40.17604686523985 74.48335235375873) bank38269 +38270 POINT(41.0662965274308 73.45133132001178) bank38270 +38271 POINT(41.51537536060877 74.56632435798765) bank38271 +38272 POINT(40.77782552094844 73.835544331573) bank38272 +38273 POINT(40.593527010179585 73.12079867364069) bank38273 +38274 POINT(41.255937052463025 74.56127954671119) bank38274 +38275 POINT(40.25783607522902 74.0567240631573) bank38275 +38276 POINT(39.95152485826693 74.58116921104366) bank38276 +38277 POINT(40.836998664748386 74.97117633187263) bank38277 +38278 POINT(40.475687071959435 74.63560905328109) bank38278 +38279 POINT(40.96684328401674 73.77912709813288) bank38279 +38280 POINT(41.11195725535055 74.37402169208868) bank38280 +38281 POINT(39.97433572032767 73.3575753905041) bank38281 +38282 POINT(40.041705148604564 74.64186568339682) bank38282 +38283 POINT(41.04411058932088 74.54737583795317) bank38283 +38284 POINT(41.35157883082197 74.88959335684015) bank38284 +38285 POINT(41.604729810271735 74.42402732823507) bank38285 +38286 POINT(40.0883780851188 73.68352872130754) bank38286 +38287 POINT(41.538494641777845 73.31797388556608) bank38287 +38288 POINT(41.61683668894107 74.34164700135554) bank38288 +38289 POINT(41.622090683248686 73.36564034488657) bank38289 +38290 POINT(39.84103289560322 73.62682002178757) bank38290 +38291 POINT(39.99978378932625 74.10578949094194) bank38291 +38292 POINT(41.55651658730328 73.19299199233313) bank38292 +38293 POINT(39.74410026414741 74.37367694150919) bank38293 +38294 POINT(41.56373512128097 73.45219223124461) bank38294 +38295 POINT(40.008360662039 73.40801035523056) bank38295 +38296 POINT(40.86135685755849 73.85714876503496) bank38296 +38297 POINT(40.306392289753184 73.29546667437654) bank38297 +38298 POINT(40.719711473120086 73.54026244150313) bank38298 +38299 POINT(41.125364017927296 73.14276787245379) bank38299 +38300 POINT(39.90504724660092 73.80909943935087) bank38300 +38301 POINT(40.25135332113021 73.08606462391437) bank38301 +38302 POINT(40.012148104389304 74.09661362110836) bank38302 +38303 POINT(39.981100544397066 74.69528258196117) bank38303 +38304 POINT(40.10552659758967 73.69825530970442) bank38304 +38305 POINT(40.913338432916866 74.93438224036434) bank38305 +38306 POINT(40.14021969148714 73.84104573407124) bank38306 +38307 POINT(41.26516077024328 73.92445312063981) bank38307 +38308 POINT(39.71535033561451 74.42977951591217) bank38308 +38309 POINT(40.82503432460917 74.24000673952891) bank38309 +38310 POINT(40.59363523406602 73.15706903636098) bank38310 +38311 POINT(41.621547002496285 74.92001490710747) bank38311 +38312 POINT(40.95088791214002 74.56882359181645) bank38312 +38313 POINT(39.80357953989335 74.24273906837776) bank38313 +38314 POINT(39.96837527651613 74.40298851166114) bank38314 +38315 POINT(40.659991797673605 73.96030114527007) bank38315 +38316 POINT(39.84104873825759 74.73352633039761) bank38316 +38317 POINT(41.165252773185955 73.30838994234716) bank38317 +38318 POINT(40.25176140599706 74.36464866307313) bank38318 +38319 POINT(40.68489534375932 73.21832169772505) bank38319 +38320 POINT(39.787576872238695 73.78435022404085) bank38320 +38321 POINT(40.58187050916311 73.53689892999793) bank38321 +38322 POINT(41.2305681380901 73.58686467203808) bank38322 +38323 POINT(40.633990365518834 73.80605910710375) bank38323 +38324 POINT(40.31212163340194 74.48618362892529) bank38324 +38325 POINT(41.30536299901172 74.97508677272562) bank38325 +38326 POINT(41.526714731823034 74.26497553797671) bank38326 +38327 POINT(40.464015897195395 74.34029706853799) bank38327 +38328 POINT(40.01575173017257 74.8902074818682) bank38328 +38329 POINT(39.82329074077021 73.57867724890244) bank38329 +38330 POINT(40.21106114196536 73.18125489430489) bank38330 +38331 POINT(41.32495969278535 73.05988005595314) bank38331 +38332 POINT(39.962423619741145 74.56996880194146) bank38332 +38333 POINT(41.44531037159018 74.2851894976581) bank38333 +38334 POINT(41.461129513961446 74.6372963320601) bank38334 +38335 POINT(41.34360717249181 73.74574421039468) bank38335 +38336 POINT(39.98900736263884 73.31388087965153) bank38336 +38337 POINT(40.06063471146998 74.04121453513234) bank38337 +38338 POINT(41.43266895295015 74.67866329194429) bank38338 +38339 POINT(40.74240732683924 74.38203268502977) bank38339 +38340 POINT(40.871264286399374 74.35849150323249) bank38340 +38341 POINT(41.22997622573652 74.10484186449007) bank38341 +38342 POINT(40.353933463883685 74.47225398273619) bank38342 +38343 POINT(40.44401909856865 74.56842921979236) bank38343 +38344 POINT(39.89957266765092 74.71876957972634) bank38344 +38345 POINT(39.97701259496144 73.49063407933684) bank38345 +38346 POINT(40.54081747210584 73.19124027410038) bank38346 +38347 POINT(40.35829576983984 73.28307831307707) bank38347 +38348 POINT(41.296464244821514 73.03746873500602) bank38348 +38349 POINT(41.2598139825745 73.26521391779197) bank38349 +38350 POINT(40.99249755098718 74.68641341063548) bank38350 +38351 POINT(40.9841598947451 73.21394151076483) bank38351 +38352 POINT(39.85504740292014 73.46330890741864) bank38352 +38353 POINT(41.21407648388153 74.51190673063674) bank38353 +38354 POINT(41.46886603139565 74.43529833398563) bank38354 +38355 POINT(40.075399491345976 74.12002134564878) bank38355 +38356 POINT(40.70526796550103 73.35167384732867) bank38356 +38357 POINT(40.82907506612803 73.62702107690167) bank38357 +38358 POINT(40.37031337889398 74.82903375501314) bank38358 +38359 POINT(39.73299199104121 74.53927757144383) bank38359 +38360 POINT(40.87478054159028 74.9704536244376) bank38360 +38361 POINT(39.81688718679107 74.91701281596491) bank38361 +38362 POINT(40.91783971646114 74.97045095038445) bank38362 +38363 POINT(40.31955526939938 74.82622129246558) bank38363 +38364 POINT(41.37652282116182 74.80995105670834) bank38364 +38365 POINT(41.64257652212151 74.78667109325698) bank38365 +38366 POINT(39.964353124180064 73.704062145782) bank38366 +38367 POINT(41.687233353028475 73.43960335922024) bank38367 +38368 POINT(40.39237470269009 74.87262679250169) bank38368 +38369 POINT(40.88625396205517 74.15348894107699) bank38369 +38370 POINT(40.21868624462059 73.39836933857467) bank38370 +38371 POINT(41.505883049889775 73.38706334978714) bank38371 +38372 POINT(40.23124232525064 74.46498181773367) bank38372 +38373 POINT(41.01239210323313 73.3086198269841) bank38373 +38374 POINT(41.46511661174155 73.06173293110989) bank38374 +38375 POINT(40.33266043275142 73.42623645045067) bank38375 +38376 POINT(40.24824534147201 73.08250513709834) bank38376 +38377 POINT(40.24903929136371 74.14923898617114) bank38377 +38378 POINT(40.34025796223116 73.86933395657177) bank38378 +38379 POINT(41.294904213951 73.13825121230875) bank38379 +38380 POINT(41.45428727531901 73.88680292361745) bank38380 +38381 POINT(40.40860335675595 74.42510826772153) bank38381 +38382 POINT(41.59708238527011 73.66656627514787) bank38382 +38383 POINT(41.28429296573126 74.3168514751943) bank38383 +38384 POINT(40.881040821131585 74.66931555242009) bank38384 +38385 POINT(41.43763525825116 73.4506717116174) bank38385 +38386 POINT(41.566113199073186 73.51900679517975) bank38386 +38387 POINT(41.127177076462445 74.53542516851658) bank38387 +38388 POINT(40.417886770190435 74.97678211366811) bank38388 +38389 POINT(39.810347810023636 73.67560443791503) bank38389 +38390 POINT(40.75405537120821 74.5828375123149) bank38390 +38391 POINT(40.80259699024994 73.30488951041646) bank38391 +38392 POINT(40.07990583423418 74.22336635807457) bank38392 +38393 POINT(40.8799258759733 73.03688904349411) bank38393 +38394 POINT(39.975373584243364 73.29285357259722) bank38394 +38395 POINT(40.33663346896764 74.83893114673823) bank38395 +38396 POINT(40.09074209703439 74.9068460146465) bank38396 +38397 POINT(40.335165149988825 74.5865478723095) bank38397 +38398 POINT(41.14321088735112 74.7702715932493) bank38398 +38399 POINT(41.260426072524254 73.57867402353509) bank38399 +38400 POINT(41.66510681485695 73.35538472819583) bank38400 +38401 POINT(39.94653592928005 73.28558797744826) bank38401 +38402 POINT(41.15979105371203 73.66488398956643) bank38402 +38403 POINT(39.86750933079389 74.92703495202079) bank38403 +38404 POINT(39.768505044268785 74.27115812660482) bank38404 +38405 POINT(41.2414712887657 73.10351211497824) bank38405 +38406 POINT(41.20778457645446 73.69347042237813) bank38406 +38407 POINT(41.07169507807866 74.74807672675118) bank38407 +38408 POINT(41.41957399126012 73.06488420250633) bank38408 +38409 POINT(39.87179320817225 73.20407076345555) bank38409 +38410 POINT(40.92997034509971 73.37494071660106) bank38410 +38411 POINT(40.13908202688393 74.80433712002656) bank38411 +38412 POINT(41.18455886590586 74.01196633343653) bank38412 +38413 POINT(41.51246798612495 73.83470916077239) bank38413 +38414 POINT(40.40802790540224 73.01155635352725) bank38414 +38415 POINT(40.537179438668396 74.03807916684273) bank38415 +38416 POINT(40.72830280657183 73.09677327905148) bank38416 +38417 POINT(41.1393864279986 74.64407190577766) bank38417 +38418 POINT(40.5093749243676 74.21243149772559) bank38418 +38419 POINT(40.315103272643775 74.09945532645195) bank38419 +38420 POINT(39.742956853583024 73.40428189533766) bank38420 +38421 POINT(40.86773156743208 73.76337275115387) bank38421 +38422 POINT(41.31818984441725 73.94339728827872) bank38422 +38423 POINT(41.24461370530898 74.86665547996004) bank38423 +38424 POINT(40.027655862124114 74.53873084768482) bank38424 +38425 POINT(40.16746138857086 73.28080374939474) bank38425 +38426 POINT(40.697202749245825 74.17922799372799) bank38426 +38427 POINT(41.687487136326695 73.56314135031452) bank38427 +38428 POINT(40.71315200528603 74.18239839291427) bank38428 +38429 POINT(40.99860339021368 73.45483514007208) bank38429 +38430 POINT(40.2388514593863 74.96685634886572) bank38430 +38431 POINT(41.02854725105834 73.10359343626686) bank38431 +38432 POINT(40.321603756784775 73.6924614310381) bank38432 +38433 POINT(41.22413920974986 73.93939191768968) bank38433 +38434 POINT(39.92585240712498 74.80390977965939) bank38434 +38435 POINT(41.37415110992369 73.02607965814755) bank38435 +38436 POINT(41.5845503760813 73.62327888500673) bank38436 +38437 POINT(41.41199043508126 73.82482586807542) bank38437 +38438 POINT(40.28264524745739 73.44144933383619) bank38438 +38439 POINT(39.77610765607428 73.28930307354847) bank38439 +38440 POINT(41.70379961058395 74.60033597461334) bank38440 +38441 POINT(39.91847586497703 74.09677039660345) bank38441 +38442 POINT(40.290461761279204 74.82804543625078) bank38442 +38443 POINT(39.93440163484982 74.50209174849427) bank38443 +38444 POINT(39.82934793698641 73.10389960163957) bank38444 +38445 POINT(41.51376221973916 74.70076955577282) bank38445 +38446 POINT(41.5066352060078 74.87451787902297) bank38446 +38447 POINT(40.553488009702164 74.69650585006916) bank38447 +38448 POINT(41.07264304944098 73.42184394205422) bank38448 +38449 POINT(40.372626219845515 73.47323856606877) bank38449 +38450 POINT(40.71621475692066 74.18403681294005) bank38450 +38451 POINT(41.1110471953601 74.74256554426053) bank38451 +38452 POINT(41.28541184834299 74.98366734798519) bank38452 +38453 POINT(41.64162212325757 74.38445950192056) bank38453 +38454 POINT(40.13619042476977 73.07578542297543) bank38454 +38455 POINT(41.618689531955035 74.40278914357695) bank38455 +38456 POINT(40.229104972942906 73.8269583696437) bank38456 +38457 POINT(39.758273947020015 74.4139217363375) bank38457 +38458 POINT(40.06392424533831 73.69976791463718) bank38458 +38459 POINT(41.454323552412205 74.86452441269535) bank38459 +38460 POINT(40.34864818281159 74.39272490347386) bank38460 +38461 POINT(40.33410823636886 73.41409721285062) bank38461 +38462 POINT(39.974380810276756 73.07597506625831) bank38462 +38463 POINT(40.68255269222781 73.98161842599265) bank38463 +38464 POINT(39.77425884324339 73.27123876966724) bank38464 +38465 POINT(40.76776124564545 73.91611551818886) bank38465 +38466 POINT(40.43233136150532 73.1698134607098) bank38466 +38467 POINT(41.172877525434586 74.92685507734011) bank38467 +38468 POINT(40.74813196080333 74.72068215199242) bank38468 +38469 POINT(40.32923001952444 73.4179701256629) bank38469 +38470 POINT(40.818537047720795 73.55864852591746) bank38470 +38471 POINT(40.21493187458064 74.73164165106147) bank38471 +38472 POINT(40.52175549163556 74.53806801106037) bank38472 +38473 POINT(40.467768237681526 74.36083494505161) bank38473 +38474 POINT(40.35689398110402 74.19681463380061) bank38474 +38475 POINT(40.2027614670907 74.04923484864695) bank38475 +38476 POINT(40.83342822466328 73.57016778283783) bank38476 +38477 POINT(39.85449188389141 73.44287378687663) bank38477 +38478 POINT(40.610014158268655 73.1276495457654) bank38478 +38479 POINT(40.10558385835883 74.01839987418549) bank38479 +38480 POINT(41.31237654686247 73.21886692676435) bank38480 +38481 POINT(41.64677153251863 74.47012468897165) bank38481 +38482 POINT(40.74594965958883 74.863597324058) bank38482 +38483 POINT(41.27441412053649 73.00626622198374) bank38483 +38484 POINT(40.66984246775811 74.62676313078795) bank38484 +38485 POINT(39.9113832776583 74.26266899247841) bank38485 +38486 POINT(40.724332386235886 73.16396613272461) bank38486 +38487 POINT(40.169784027705894 73.56860784443063) bank38487 +38488 POINT(40.61537128021987 73.1544491336256) bank38488 +38489 POINT(40.56526392544461 74.47892929146064) bank38489 +38490 POINT(39.806406423561846 74.19565957034806) bank38490 +38491 POINT(39.90942775252905 74.29131338954717) bank38491 +38492 POINT(39.81460636071344 74.10547413336538) bank38492 +38493 POINT(41.68547244594724 73.3342284082566) bank38493 +38494 POINT(41.54917841438577 74.92170723728734) bank38494 +38495 POINT(40.76037168611963 74.48245471035379) bank38495 +38496 POINT(41.35224510621115 73.16024415487732) bank38496 +38497 POINT(41.243558522713656 74.92336856878757) bank38497 +38498 POINT(40.24935909433131 74.0848609867364) bank38498 +38499 POINT(40.529963375630736 73.53688559052459) bank38499 +38500 POINT(40.49353068463505 74.6303012071662) bank38500 +38501 POINT(41.624650361995926 73.67485131724494) bank38501 +38502 POINT(41.15858684815079 74.15850837816932) bank38502 +38503 POINT(39.9994047950653 74.3087690347598) bank38503 +38504 POINT(40.298044846766814 73.82031617276348) bank38504 +38505 POINT(40.97440108738665 74.44293225731772) bank38505 +38506 POINT(40.23853782732422 74.86763363486762) bank38506 +38507 POINT(41.30513419773058 73.19424160368034) bank38507 +38508 POINT(40.02122112327314 74.14251939422111) bank38508 +38509 POINT(41.339745527527505 73.91070693239142) bank38509 +38510 POINT(39.75889756347764 74.98955865951183) bank38510 +38511 POINT(41.13012632606704 74.81321668057413) bank38511 +38512 POINT(40.46179668909297 74.63894087229193) bank38512 +38513 POINT(40.05111213521973 73.4155657217086) bank38513 +38514 POINT(41.00646931053774 73.1374215711358) bank38514 +38515 POINT(39.97523817144225 73.99553602841694) bank38515 +38516 POINT(40.73499794271513 73.54713726253863) bank38516 +38517 POINT(40.420103650172166 73.00830938329534) bank38517 +38518 POINT(40.962977452559414 73.47106061249171) bank38518 +38519 POINT(41.31636960465462 74.37302465504682) bank38519 +38520 POINT(40.58279536277281 74.37015956244512) bank38520 +38521 POINT(39.92962987979301 73.39448571305272) bank38521 +38522 POINT(40.513636413411355 73.40526933974752) bank38522 +38523 POINT(40.84504987238679 74.9018434032274) bank38523 +38524 POINT(39.933561234518955 74.43575228150634) bank38524 +38525 POINT(39.81939006085148 74.14167902203958) bank38525 +38526 POINT(40.35373248704308 73.96453906056676) bank38526 +38527 POINT(41.19627214031081 74.2799150770661) bank38527 +38528 POINT(41.67664394796243 74.23610788176796) bank38528 +38529 POINT(41.07556889273313 74.4531638433555) bank38529 +38530 POINT(41.06990019703697 73.65059239086477) bank38530 +38531 POINT(40.794568584315385 74.81181534694392) bank38531 +38532 POINT(39.77280310632165 74.01907109442881) bank38532 +38533 POINT(39.7744995924881 73.64164171940392) bank38533 +38534 POINT(40.16678231625614 74.51451966204547) bank38534 +38535 POINT(41.193168288460285 73.54894118358656) bank38535 +38536 POINT(41.14954120633371 73.4549861263765) bank38536 +38537 POINT(41.13061925881699 73.90494985467832) bank38537 +38538 POINT(40.218581085987516 74.04715531398944) bank38538 +38539 POINT(40.84389966890775 73.01869782311576) bank38539 +38540 POINT(39.761738331399584 74.78995106739691) bank38540 +38541 POINT(41.633227678858766 74.42160499899889) bank38541 +38542 POINT(41.54532224519134 74.29468073309474) bank38542 +38543 POINT(40.94150000426476 74.51828829950591) bank38543 +38544 POINT(41.47981119336502 74.64853385869715) bank38544 +38545 POINT(41.25938610319292 74.40142643491738) bank38545 +38546 POINT(40.96719341521965 74.12734582418574) bank38546 +38547 POINT(40.251429851704316 74.74877588525936) bank38547 +38548 POINT(39.89403763099067 73.86497232548433) bank38548 +38549 POINT(40.95237563279774 73.46840360267747) bank38549 +38550 POINT(40.221550496009684 74.38815288190531) bank38550 +38551 POINT(41.59169239332763 74.11631440905236) bank38551 +38552 POINT(40.89325405844787 74.33753639459404) bank38552 +38553 POINT(40.32474557408866 73.03386639141974) bank38553 +38554 POINT(40.389014389753925 73.21584059458624) bank38554 +38555 POINT(40.942309587345136 73.58842204191821) bank38555 +38556 POINT(39.98856932603474 74.30813214540676) bank38556 +38557 POINT(40.53239459288908 74.08092129864548) bank38557 +38558 POINT(39.75188621994358 74.49006316678006) bank38558 +38559 POINT(41.49668611382809 74.23837672379376) bank38559 +38560 POINT(40.12407414432122 74.9142663880252) bank38560 +38561 POINT(40.70786464338071 73.25146349608744) bank38561 +38562 POINT(40.11684549165491 73.38520817046775) bank38562 +38563 POINT(41.007311565757945 73.17485840401616) bank38563 +38564 POINT(39.79897819338494 73.99204338293566) bank38564 +38565 POINT(41.13858615421371 74.93409330721931) bank38565 +38566 POINT(40.8477774430094 73.6076847277345) bank38566 +38567 POINT(41.100235355000606 73.26329672483456) bank38567 +38568 POINT(41.59811530124637 73.53006928200658) bank38568 +38569 POINT(39.75592877778677 74.22384489325282) bank38569 +38570 POINT(40.613397744400665 74.69946977678337) bank38570 +38571 POINT(40.13812419754641 73.5768675278262) bank38571 +38572 POINT(40.02713222948324 74.49418448472231) bank38572 +38573 POINT(39.856183847785914 73.69653189450761) bank38573 +38574 POINT(40.89297736142236 73.00910726800434) bank38574 +38575 POINT(41.38281040409486 73.78971502136129) bank38575 +38576 POINT(40.02724360694106 74.9364567097094) bank38576 +38577 POINT(40.010881139729065 74.20757538116675) bank38577 +38578 POINT(41.38466928316439 73.68205365211155) bank38578 +38579 POINT(39.931143044791774 74.20384875454543) bank38579 +38580 POINT(40.67779183106724 74.24680359236834) bank38580 +38581 POINT(41.21713200527692 75.004483627647) bank38581 +38582 POINT(39.94168578833068 74.90361081859301) bank38582 +38583 POINT(40.82907897103044 74.57564505941014) bank38583 +38584 POINT(41.45983611787482 74.70039760040521) bank38584 +38585 POINT(39.79893652004023 74.26392895819738) bank38585 +38586 POINT(41.59737798493466 74.09929332403944) bank38586 +38587 POINT(40.97223645250526 74.7025392009496) bank38587 +38588 POINT(40.31926716509 73.62862345932872) bank38588 +38589 POINT(41.300111347440755 74.83908903592024) bank38589 +38590 POINT(41.63708651373912 74.68745224347549) bank38590 +38591 POINT(41.3855139983921 74.95295265555767) bank38591 +38592 POINT(41.52426349521375 73.97596182231071) bank38592 +38593 POINT(40.751217511741835 74.56127098124716) bank38593 +38594 POINT(40.17228762455292 73.13667818093325) bank38594 +38595 POINT(41.41602696468507 73.8214431419401) bank38595 +38596 POINT(40.4077060491632 74.2407817116801) bank38596 +38597 POINT(40.569643360552554 74.8860969100568) bank38597 +38598 POINT(40.440964796367254 74.05808941444167) bank38598 +38599 POINT(39.7996620738064 74.8971106831351) bank38599 +38600 POINT(41.158036771350346 74.99546376207277) bank38600 +38601 POINT(39.809475905988194 74.54730442048907) bank38601 +38602 POINT(40.2928385354555 74.79862626521857) bank38602 +38603 POINT(39.87741633154761 74.42266781106802) bank38603 +38604 POINT(41.58949528984352 73.0468683971126) bank38604 +38605 POINT(40.31278603921798 73.0945749719732) bank38605 +38606 POINT(40.81062111122417 73.56505352623606) bank38606 +38607 POINT(40.37337528295952 73.77625243101613) bank38607 +38608 POINT(39.87404333975847 74.48947147721837) bank38608 +38609 POINT(41.20042297165087 74.36756566624649) bank38609 +38610 POINT(41.120996602911944 73.63175227539146) bank38610 +38611 POINT(40.04837365800471 74.43609658566788) bank38611 +38612 POINT(39.931635499510364 74.94806626673905) bank38612 +38613 POINT(40.318611475961674 74.79968127190477) bank38613 +38614 POINT(40.15010582355009 73.11630824487342) bank38614 +38615 POINT(40.37974313546047 74.664607351331) bank38615 +38616 POINT(41.22977095454638 74.18992258791164) bank38616 +38617 POINT(40.337061420005135 73.56558333543184) bank38617 +38618 POINT(40.76162227585615 73.65716866838605) bank38618 +38619 POINT(41.34815342993398 74.74115199775893) bank38619 +38620 POINT(41.51465794022131 74.6012961941962) bank38620 +38621 POINT(41.03783652496886 74.02557365224767) bank38621 +38622 POINT(40.17890480956962 74.47646551602516) bank38622 +38623 POINT(41.52665834815113 75.00109669412637) bank38623 +38624 POINT(41.00152420213986 73.46849946978963) bank38624 +38625 POINT(41.20017996240314 73.03128236288482) bank38625 +38626 POINT(41.09153176832704 74.86791651930095) bank38626 +38627 POINT(40.28435744659357 73.69322075592285) bank38627 +38628 POINT(39.82813505357997 73.08856335402045) bank38628 +38629 POINT(41.24642362063864 73.89229766465193) bank38629 +38630 POINT(40.812638817971084 73.46077626212325) bank38630 +38631 POINT(40.96495991326537 73.35165648281497) bank38631 +38632 POINT(40.23828650251388 73.74714317980178) bank38632 +38633 POINT(40.15474215163112 73.7843970125847) bank38633 +38634 POINT(41.621092038157826 73.07226583680537) bank38634 +38635 POINT(40.935212972883754 74.2283471235994) bank38635 +38636 POINT(40.86156791984538 74.30682264862226) bank38636 +38637 POINT(41.56751124727804 74.24972457423439) bank38637 +38638 POINT(40.715410434689936 73.80979501481616) bank38638 +38639 POINT(40.25025370585699 73.89436141763701) bank38639 +38640 POINT(40.12069949632745 74.84487848891962) bank38640 +38641 POINT(40.34089794611645 74.04493967952092) bank38641 +38642 POINT(40.05531859812848 73.68253765070246) bank38642 +38643 POINT(41.43133253979744 74.14282781082441) bank38643 +38644 POINT(40.07005565792825 74.21566652998672) bank38644 +38645 POINT(41.50179040195915 73.53565422178471) bank38645 +38646 POINT(39.902014271859244 73.01879477495723) bank38646 +38647 POINT(41.49849555481342 74.39641204259145) bank38647 +38648 POINT(41.11059089592875 73.69051329227713) bank38648 +38649 POINT(41.65049906889075 74.72142464813115) bank38649 +38650 POINT(41.12327551313452 74.51149653271679) bank38650 +38651 POINT(41.16060066855148 73.96408465040345) bank38651 +38652 POINT(40.562454914004306 73.07327860895901) bank38652 +38653 POINT(39.869232581493286 73.97638037403486) bank38653 +38654 POINT(40.593115358473014 73.17194891017498) bank38654 +38655 POINT(41.06732701440893 73.4557218786718) bank38655 +38656 POINT(40.05200378533826 73.55321932774368) bank38656 +38657 POINT(41.18696388733472 74.75650280323005) bank38657 +38658 POINT(40.03317534766882 73.86256219653468) bank38658 +38659 POINT(40.27691401517514 74.12102429572145) bank38659 +38660 POINT(40.855358081175574 73.01925009171988) bank38660 +38661 POINT(41.701133231856 73.9909033866562) bank38661 +38662 POINT(40.91272392964015 73.72145372885251) bank38662 +38663 POINT(40.0947335326722 73.40414569102226) bank38663 +38664 POINT(41.38561327123135 74.17878754813162) bank38664 +38665 POINT(40.913619687309016 74.00992767504248) bank38665 +38666 POINT(40.29349550708041 74.86149874426626) bank38666 +38667 POINT(40.084090313932464 74.69355665264924) bank38667 +38668 POINT(40.69643047172721 74.02943002807346) bank38668 +38669 POINT(41.2166452530593 74.21557213184991) bank38669 +38670 POINT(40.30091592521157 73.33998724419625) bank38670 +38671 POINT(40.887735307516714 73.52570464405697) bank38671 +38672 POINT(40.34946358753557 74.75121130737735) bank38672 +38673 POINT(41.144436621778155 73.88811234155051) bank38673 +38674 POINT(40.727260036927525 74.57290659932025) bank38674 +38675 POINT(41.60347739765415 73.165681451441) bank38675 +38676 POINT(41.23008722222952 74.70187408345883) bank38676 +38677 POINT(40.47170971403089 73.5636408679297) bank38677 +38678 POINT(40.21504154199632 74.7921007734117) bank38678 +38679 POINT(41.46106416533297 74.72059857225487) bank38679 +38680 POINT(41.020901419475315 73.16688780417837) bank38680 +38681 POINT(41.29083467647586 73.28208868350268) bank38681 +38682 POINT(41.61515465737757 73.76268706660525) bank38682 +38683 POINT(40.37902169470349 74.16840634239371) bank38683 +38684 POINT(40.346591767213866 73.9938092776834) bank38684 +38685 POINT(39.95049268224123 73.23859473021275) bank38685 +38686 POINT(40.71886590023151 73.44364046521365) bank38686 +38687 POINT(39.79552785649312 74.91098510710619) bank38687 +38688 POINT(40.7818847977227 73.84161322413678) bank38688 +38689 POINT(41.34147062569578 74.41728990776134) bank38689 +38690 POINT(41.29150091050904 73.07531638013982) bank38690 +38691 POINT(41.3114872123259 74.70231421480354) bank38691 +38692 POINT(41.23595746754079 74.43689593615501) bank38692 +38693 POINT(39.770338660852595 73.0376547599184) bank38693 +38694 POINT(41.02596026035372 73.48992159792576) bank38694 +38695 POINT(40.14580854091459 73.15109636061142) bank38695 +38696 POINT(40.17060821518229 74.95630722169483) bank38696 +38697 POINT(41.13470241402502 73.13565475984551) bank38697 +38698 POINT(40.10105009371169 74.04914999626243) bank38698 +38699 POINT(41.247400678207455 73.59109475240572) bank38699 +38700 POINT(40.55289707663242 74.4829954484381) bank38700 +38701 POINT(39.834981875528776 74.38250851996833) bank38701 +38702 POINT(40.294496127169424 74.35175262959932) bank38702 +38703 POINT(40.94814258412632 74.15337147334627) bank38703 +38704 POINT(41.65726346780141 73.04868914309031) bank38704 +38705 POINT(39.84782058752166 74.1267662808422) bank38705 +38706 POINT(40.26394340539698 73.03849597274025) bank38706 +38707 POINT(40.074859390548035 73.30080516630757) bank38707 +38708 POINT(40.72980839894869 73.96825323057355) bank38708 +38709 POINT(40.892403935698894 74.2939630972241) bank38709 +38710 POINT(40.71282905343383 73.16211709969001) bank38710 +38711 POINT(40.39466714179762 74.83331568393096) bank38711 +38712 POINT(39.90565196392749 73.17690089504075) bank38712 +38713 POINT(41.18381084004967 73.20133973016577) bank38713 +38714 POINT(41.316597560981585 73.16812062298816) bank38714 +38715 POINT(41.456946196002086 73.1430435723637) bank38715 +38716 POINT(39.929127598215565 73.08346399491644) bank38716 +38717 POINT(41.672105905969296 74.2233578301354) bank38717 +38718 POINT(40.600319235626934 74.35945987739332) bank38718 +38719 POINT(41.05829184643853 73.89434888998774) bank38719 +38720 POINT(41.20292934946974 73.70408355132126) bank38720 +38721 POINT(41.137447048788395 74.16230886748096) bank38721 +38722 POINT(39.72945031308956 74.346766465047) bank38722 +38723 POINT(40.63183281988427 74.61515259786428) bank38723 +38724 POINT(41.360257459179095 73.30153963000838) bank38724 +38725 POINT(41.65235866937961 74.8752222490706) bank38725 +38726 POINT(40.32864667741352 74.46261383188913) bank38726 +38727 POINT(40.83604844997681 74.09412607492222) bank38727 +38728 POINT(40.40092676997352 74.0730571720177) bank38728 +38729 POINT(41.24909240939649 73.47886072148154) bank38729 +38730 POINT(40.24037187549046 74.92479675068732) bank38730 +38731 POINT(40.64755049187092 73.91871136600182) bank38731 +38732 POINT(40.83648756932473 74.65499070015791) bank38732 +38733 POINT(40.48985182639301 73.63228690293052) bank38733 +38734 POINT(40.683606972992585 73.95378098467165) bank38734 +38735 POINT(41.67193702564279 74.27487373900452) bank38735 +38736 POINT(40.825861458944004 74.5840750956668) bank38736 +38737 POINT(41.268049241251475 74.74983745229125) bank38737 +38738 POINT(40.69783057479183 73.06400265099747) bank38738 +38739 POINT(40.961018168503976 74.19303902564255) bank38739 +38740 POINT(40.77697130981007 74.16759722546902) bank38740 +38741 POINT(40.8870309090474 74.48489894528815) bank38741 +38742 POINT(41.13827165275676 74.51432687259518) bank38742 +38743 POINT(41.61594398808496 73.57379351662742) bank38743 +38744 POINT(40.67951001098908 73.59250393544204) bank38744 +38745 POINT(39.79616410442524 73.13623300682583) bank38745 +38746 POINT(40.79803131232091 74.99020375191107) bank38746 +38747 POINT(41.31502043595046 74.89033151678778) bank38747 +38748 POINT(40.22103524935575 74.25285212600019) bank38748 +38749 POINT(41.05408153776662 74.1168625187892) bank38749 +38750 POINT(41.113889151030726 73.56618339261382) bank38750 +38751 POINT(40.58703528131198 74.85255319606611) bank38751 +38752 POINT(40.20667549613518 73.3759943052522) bank38752 +38753 POINT(40.985938017146616 73.83226827567276) bank38753 +38754 POINT(41.12610373216921 73.05883691012197) bank38754 +38755 POINT(40.26478569228332 74.29217176514585) bank38755 +38756 POINT(40.339567746584656 73.47225624526067) bank38756 +38757 POINT(40.798312070205355 73.59998820947521) bank38757 +38758 POINT(39.81091639819081 73.37020264658653) bank38758 +38759 POINT(40.88187642831356 73.84682601156271) bank38759 +38760 POINT(39.76254554723316 73.10681658437525) bank38760 +38761 POINT(40.677549667429716 74.5039018867592) bank38761 +38762 POINT(40.11784559561546 74.17430954909963) bank38762 +38763 POINT(40.33352844947885 74.61795120808794) bank38763 +38764 POINT(40.349776085558844 74.03525764300007) bank38764 +38765 POINT(41.23645512735179 74.38422531743437) bank38765 +38766 POINT(41.596804925527415 74.22314791422693) bank38766 +38767 POINT(40.943558084507664 74.83891674467456) bank38767 +38768 POINT(40.35089355748267 74.82249466226357) bank38768 +38769 POINT(40.79920784109328 73.28364767213063) bank38769 +38770 POINT(40.91357918470726 73.84962225848797) bank38770 +38771 POINT(40.71280587976175 74.7975391242878) bank38771 +38772 POINT(40.36057494883099 74.73712662808099) bank38772 +38773 POINT(40.44471211121292 74.97929161269268) bank38773 +38774 POINT(39.944530251871754 73.12393281905435) bank38774 +38775 POINT(40.95241601948567 73.89090559606785) bank38775 +38776 POINT(40.15022351617076 73.05993311775276) bank38776 +38777 POINT(40.453997646991574 73.34140252878544) bank38777 +38778 POINT(41.00953362845815 73.44303080816502) bank38778 +38779 POINT(41.16538643319879 73.56280104729025) bank38779 +38780 POINT(41.10502756190704 74.25450513531918) bank38780 +38781 POINT(41.621357770386886 73.45841252887094) bank38781 +38782 POINT(40.426764087886795 73.06056048037087) bank38782 +38783 POINT(40.46312889244679 74.81794973268188) bank38783 +38784 POINT(40.92374870618588 74.52518011870895) bank38784 +38785 POINT(41.22822788262507 74.58608987577897) bank38785 +38786 POINT(41.11989346591687 74.55001553470034) bank38786 +38787 POINT(41.67820922590032 74.94121858565337) bank38787 +38788 POINT(41.53673199373391 73.57778334885712) bank38788 +38789 POINT(41.59275315953502 74.96873160026594) bank38789 +38790 POINT(40.84895076788658 74.29683080865696) bank38790 +38791 POINT(40.10670778500538 74.82207691853237) bank38791 +38792 POINT(40.02315644130838 74.96365214133841) bank38792 +38793 POINT(40.5810051427116 74.72426384390806) bank38793 +38794 POINT(40.129281573953826 74.75555496888458) bank38794 +38795 POINT(41.65241435671308 73.36776152116873) bank38795 +38796 POINT(40.11014967936466 74.84454364018596) bank38796 +38797 POINT(40.58840231220007 73.05587067680534) bank38797 +38798 POINT(40.8515524963659 73.1320880113677) bank38798 +38799 POINT(41.56570215786517 73.6581803327634) bank38799 +38800 POINT(40.60827302089334 74.64454474790473) bank38800 +38801 POINT(41.10635931601474 74.00742922598259) bank38801 +38802 POINT(40.02584825453209 74.77927884949284) bank38802 +38803 POINT(41.233000476118946 73.3702529389773) bank38803 +38804 POINT(41.244134797369064 74.50164771273302) bank38804 +38805 POINT(40.825718656626904 74.7767936314345) bank38805 +38806 POINT(41.404578873542334 74.32183343012677) bank38806 +38807 POINT(41.51040388267902 74.28248581257675) bank38807 +38808 POINT(40.00117272801753 74.90864400050785) bank38808 +38809 POINT(41.44711780228449 74.78378559714449) bank38809 +38810 POINT(40.03434951506056 73.49446287810576) bank38810 +38811 POINT(39.7733275869735 74.62765524189624) bank38811 +38812 POINT(41.07002820580243 74.02028912001143) bank38812 +38813 POINT(40.844297582381785 73.49806525762348) bank38813 +38814 POINT(41.30384805723371 73.15488447679851) bank38814 +38815 POINT(40.07962183141616 74.39311356136909) bank38815 +38816 POINT(41.601645124558985 73.34303106593894) bank38816 +38817 POINT(39.802643293655514 74.28971891911254) bank38817 +38818 POINT(41.26589018566045 73.87976054927992) bank38818 +38819 POINT(40.2218456147168 73.74257537567556) bank38819 +38820 POINT(41.30997193954846 73.85461029988089) bank38820 +38821 POINT(40.075707097409655 73.7611683008007) bank38821 +38822 POINT(40.31624179274171 73.25208784695062) bank38822 +38823 POINT(40.96502032573222 74.44296425011355) bank38823 +38824 POINT(39.82344544833236 74.89646254236158) bank38824 +38825 POINT(41.35552446501274 73.92332487792147) bank38825 +38826 POINT(40.589951575833325 74.21257010909169) bank38826 +38827 POINT(40.06587019610037 73.06152759642471) bank38827 +38828 POINT(39.77598297414341 73.62158943783943) bank38828 +38829 POINT(39.75309989059772 74.8433877619751) bank38829 +38830 POINT(39.84003570186258 74.91123180386457) bank38830 +38831 POINT(41.675037738911506 73.19541959816787) bank38831 +38832 POINT(40.4947562236231 74.07606197927865) bank38832 +38833 POINT(41.4737750413818 74.93069537816837) bank38833 +38834 POINT(41.4898510233124 74.24237992202325) bank38834 +38835 POINT(40.577467682764336 73.94610118202138) bank38835 +38836 POINT(40.845298098917034 74.19908071640265) bank38836 +38837 POINT(41.525951714874935 74.4699276841465) bank38837 +38838 POINT(39.716764640045824 74.99994180913895) bank38838 +38839 POINT(40.75423059410492 74.98946553845994) bank38839 +38840 POINT(40.095108548484816 74.63315271755731) bank38840 +38841 POINT(41.28888154068181 74.10584304719389) bank38841 +38842 POINT(41.03984208221711 74.90392375995131) bank38842 +38843 POINT(40.41161446792631 74.53446628959108) bank38843 +38844 POINT(40.22321510433631 74.22352829698445) bank38844 +38845 POINT(41.086835530336586 74.17874156257544) bank38845 +38846 POINT(40.21924988922632 74.60431796544883) bank38846 +38847 POINT(41.39287100773425 74.70708194161708) bank38847 +38848 POINT(40.13146046396417 74.67701365293735) bank38848 +38849 POINT(40.4338616753316 74.77189548479946) bank38849 +38850 POINT(40.76267231084122 74.8588681069034) bank38850 +38851 POINT(40.218145002424535 73.04736175874929) bank38851 +38852 POINT(41.62645569121964 73.88077168381024) bank38852 +38853 POINT(40.114653907638726 74.01668229495561) bank38853 +38854 POINT(41.32427100046686 74.55125633653466) bank38854 +38855 POINT(41.61681107773339 74.73703371925087) bank38855 +38856 POINT(40.6581525151686 73.12366037368965) bank38856 +38857 POINT(41.10252038283478 73.2881285234978) bank38857 +38858 POINT(41.531102978249 73.8588705944913) bank38858 +38859 POINT(41.463215268181315 74.34504880399284) bank38859 +38860 POINT(41.26943489605786 74.47668173961405) bank38860 +38861 POINT(39.91069573422199 73.05806994292229) bank38861 +38862 POINT(40.934567130054134 74.12566577530202) bank38862 +38863 POINT(40.72847134995222 73.10594538203102) bank38863 +38864 POINT(41.10289989268976 74.4012432316985) bank38864 +38865 POINT(39.8556962228911 74.10544624454621) bank38865 +38866 POINT(41.45842909545234 73.64646461948318) bank38866 +38867 POINT(41.53536152682309 74.16594107224257) bank38867 +38868 POINT(39.74380953388388 73.03698016058144) bank38868 +38869 POINT(40.75222161166476 73.7752601953735) bank38869 +38870 POINT(40.62606707406659 74.4760803292678) bank38870 +38871 POINT(39.896996095917245 74.86395959565304) bank38871 +38872 POINT(41.237702413088805 73.56968763590434) bank38872 +38873 POINT(41.275469833135176 74.63791339109225) bank38873 +38874 POINT(41.11626687021034 74.30157591354413) bank38874 +38875 POINT(41.083580675669964 73.94760856694216) bank38875 +38876 POINT(41.332254287928194 74.75242569387937) bank38876 +38877 POINT(41.07335923009358 73.97160732069125) bank38877 +38878 POINT(41.32444636377599 74.83070395368621) bank38878 +38879 POINT(40.102006998965116 74.66192354172487) bank38879 +38880 POINT(40.559726639371284 73.0368691504657) bank38880 +38881 POINT(40.111709613309706 73.60692595783145) bank38881 +38882 POINT(40.814484756220665 73.64783909894534) bank38882 +38883 POINT(40.33561431172102 74.07381753846313) bank38883 +38884 POINT(41.19638412583462 73.67108701188825) bank38884 +38885 POINT(40.871758276892045 73.20798942963263) bank38885 +38886 POINT(41.26069575530294 73.56536314385885) bank38886 +38887 POINT(40.59032671983125 73.00994413860165) bank38887 +38888 POINT(39.7486863738341 74.0540272904552) bank38888 +38889 POINT(40.74062202152538 73.94875058543215) bank38889 +38890 POINT(41.507082631151086 73.21781311109554) bank38890 +38891 POINT(40.17804531254824 74.58270226891098) bank38891 +38892 POINT(40.58632139576696 74.4497075503711) bank38892 +38893 POINT(39.72544475876344 74.80535650644683) bank38893 +38894 POINT(40.35356998722943 74.54347592211758) bank38894 +38895 POINT(41.524286953187065 73.0680295646525) bank38895 +38896 POINT(40.41426448772463 74.4902231380804) bank38896 +38897 POINT(40.52533654825333 74.17923060286788) bank38897 +38898 POINT(39.93841819272574 74.20506132411091) bank38898 +38899 POINT(41.195641184998955 74.98571493534514) bank38899 +38900 POINT(41.51108994440955 74.68785576097825) bank38900 +38901 POINT(40.07853435020882 73.59030196771539) bank38901 +38902 POINT(40.88311605675325 73.96591954007455) bank38902 +38903 POINT(40.930906613606496 74.04948709641734) bank38903 +38904 POINT(40.51554711562799 74.54533040896035) bank38904 +38905 POINT(41.082488955973076 73.06195663672135) bank38905 +38906 POINT(40.14939119477191 73.01848200464998) bank38906 +38907 POINT(40.75764901079728 73.36492267102537) bank38907 +38908 POINT(41.24036209256342 73.68772923665183) bank38908 +38909 POINT(40.95367626343836 73.90535855522114) bank38909 +38910 POINT(41.11914936544283 74.22230124091756) bank38910 +38911 POINT(39.993655025336935 73.47485379620164) bank38911 +38912 POINT(40.625604790923646 74.99671757254346) bank38912 +38913 POINT(40.11509592274021 74.72502684399616) bank38913 +38914 POINT(40.314220528190354 73.95002392837448) bank38914 +38915 POINT(39.95206322978361 73.82144575470033) bank38915 +38916 POINT(40.44436471188496 74.03815690612996) bank38916 +38917 POINT(39.98952055134466 73.8407214967385) bank38917 +38918 POINT(40.793844953686175 73.24791508264762) bank38918 +38919 POINT(40.89503485393569 73.55368795036276) bank38919 +38920 POINT(40.23661266349232 73.71471952523434) bank38920 +38921 POINT(40.75454145357408 73.50120908589416) bank38921 +38922 POINT(41.43812509377365 73.30367883648748) bank38922 +38923 POINT(40.57837237393827 73.99365368927802) bank38923 +38924 POINT(41.34414716869963 74.98571155879942) bank38924 +38925 POINT(41.00571513975371 74.70333027335116) bank38925 +38926 POINT(39.71534507753311 74.18468608404426) bank38926 +38927 POINT(39.74941781741897 73.00782633371732) bank38927 +38928 POINT(40.3753472264889 74.90180664507464) bank38928 +38929 POINT(40.934102316502546 74.82807096349254) bank38929 +38930 POINT(40.342318869391626 74.98322197204911) bank38930 +38931 POINT(40.54619926208004 73.92457325744934) bank38931 +38932 POINT(39.973041738607016 74.11654107809449) bank38932 +38933 POINT(41.67385114157235 74.35111509022968) bank38933 +38934 POINT(39.76899941665989 74.68453188277454) bank38934 +38935 POINT(41.27850912921809 74.72423155656251) bank38935 +38936 POINT(41.35430883627999 73.12909987778136) bank38936 +38937 POINT(40.50220125753722 74.07128475110508) bank38937 +38938 POINT(40.70359188646796 74.25933039906504) bank38938 +38939 POINT(41.30333712126396 73.22234024778074) bank38939 +38940 POINT(41.24541993388197 74.54156461675677) bank38940 +38941 POINT(40.030033781447074 74.79355156863338) bank38941 +38942 POINT(40.606504424539665 73.56241038940522) bank38942 +38943 POINT(41.46987519096136 73.54837664838534) bank38943 +38944 POINT(40.54108337795649 74.98317841876512) bank38944 +38945 POINT(41.6542484349649 74.94174150926553) bank38945 +38946 POINT(41.17061845027345 73.89971799341204) bank38946 +38947 POINT(41.53803706450108 73.69484158023717) bank38947 +38948 POINT(40.37507089311919 73.67322079922548) bank38948 +38949 POINT(41.51590286340909 73.8993935930551) bank38949 +38950 POINT(41.45249167192838 73.63217118959662) bank38950 +38951 POINT(40.16058919514893 73.89814207575286) bank38951 +38952 POINT(39.89501516811257 73.97763398924872) bank38952 +38953 POINT(41.471537238531745 73.2821570147601) bank38953 +38954 POINT(41.5880131359055 74.5089964472612) bank38954 +38955 POINT(41.50934743280089 73.32636884171701) bank38955 +38956 POINT(41.40777530838516 74.86950478059224) bank38956 +38957 POINT(41.55359491235777 73.54279194653164) bank38957 +38958 POINT(40.079714931669415 74.71265190438295) bank38958 +38959 POINT(41.51111400623346 74.62408261521644) bank38959 +38960 POINT(39.80549843732568 73.28999955152968) bank38960 +38961 POINT(39.76971434153601 74.73823467205945) bank38961 +38962 POINT(41.14174152711552 74.91172103719173) bank38962 +38963 POINT(41.415412719306374 74.34532220926747) bank38963 +38964 POINT(41.092343338092604 73.44639245075906) bank38964 +38965 POINT(39.718470903234675 74.98527547466036) bank38965 +38966 POINT(40.615432883950675 74.50507747864141) bank38966 +38967 POINT(40.55610688685359 74.80385672626785) bank38967 +38968 POINT(40.56425775027996 73.91143498757623) bank38968 +38969 POINT(40.72500816685725 73.77858432378967) bank38969 +38970 POINT(41.47282281595104 74.25477072770465) bank38970 +38971 POINT(40.80310837228327 73.76429598585197) bank38971 +38972 POINT(39.946270802804285 73.2015885507209) bank38972 +38973 POINT(40.0748483737844 74.52220529578422) bank38973 +38974 POINT(39.89171798883669 73.26359107605259) bank38974 +38975 POINT(40.40952124203893 74.87641419131712) bank38975 +38976 POINT(40.8015310856535 74.1260200739362) bank38976 +38977 POINT(40.378782575684454 74.60067384025817) bank38977 +38978 POINT(40.74844295519217 73.55548243775388) bank38978 +38979 POINT(40.25112944486605 74.69477402729893) bank38979 +38980 POINT(40.98116302148254 73.8185238506836) bank38980 +38981 POINT(41.211952116755775 74.64741772114623) bank38981 +38982 POINT(39.911628189987155 74.43597063236797) bank38982 +38983 POINT(40.14410205148669 73.71651648649143) bank38983 +38984 POINT(40.52197856881736 73.38067199938241) bank38984 +38985 POINT(41.21119339240869 74.62947150020631) bank38985 +38986 POINT(40.66310787229139 73.84454840106805) bank38986 +38987 POINT(40.42745198984306 74.37605040509118) bank38987 +38988 POINT(41.664030690629424 74.81119349282731) bank38988 +38989 POINT(40.964634975635825 73.78625697403507) bank38989 +38990 POINT(41.45563940812076 74.16589876362929) bank38990 +38991 POINT(41.043671976485854 73.56297478767432) bank38991 +38992 POINT(40.53027976354983 73.06511626851065) bank38992 +38993 POINT(40.12033491446621 73.72939196833842) bank38993 +38994 POINT(40.07281121470697 74.23127054432986) bank38994 +38995 POINT(41.17087363070946 73.33616187028468) bank38995 +38996 POINT(40.2426281963766 74.20966434154191) bank38996 +38997 POINT(41.60537259591566 73.1463539488963) bank38997 +38998 POINT(41.329295472672065 73.0449927660934) bank38998 +38999 POINT(39.85808365354639 74.29514711339765) bank38999 +39000 POINT(41.56142554302597 73.7163763730736) bank39000 +39001 POINT(41.38356237867472 73.95326794091245) bank39001 +39002 POINT(40.19111595570164 74.51411298166023) bank39002 +39003 POINT(41.3193081406303 74.96453900919143) bank39003 +39004 POINT(41.3514633947224 74.44458568652094) bank39004 +39005 POINT(40.88765660222025 73.25396136601225) bank39005 +39006 POINT(40.5446254432373 74.21852641702863) bank39006 +39007 POINT(41.53217263243408 74.992585261319) bank39007 +39008 POINT(41.60782457119846 73.25461941139311) bank39008 +39009 POINT(40.533209720711454 74.46649474094394) bank39009 +39010 POINT(40.53757633486077 74.5745652423514) bank39010 +39011 POINT(41.480698557274195 73.09952464170674) bank39011 +39012 POINT(41.20232096801101 74.46809121933026) bank39012 +39013 POINT(40.566169425666 73.74984072754533) bank39013 +39014 POINT(40.958089536823515 73.88082622492769) bank39014 +39015 POINT(41.17002458659675 74.17075620261188) bank39015 +39016 POINT(41.300458938003125 74.22400201937194) bank39016 +39017 POINT(40.434709818454216 74.16061415310813) bank39017 +39018 POINT(40.18799834170443 74.4748979977835) bank39018 +39019 POINT(39.93005733696064 74.10167308585793) bank39019 +39020 POINT(39.90095067658724 73.65931771476663) bank39020 +39021 POINT(40.73809729264105 73.88908565909716) bank39021 +39022 POINT(40.95452723777327 74.75177706876163) bank39022 +39023 POINT(40.5224990430709 73.97458680281856) bank39023 +39024 POINT(41.20695848278855 73.24187346174581) bank39024 +39025 POINT(40.125848900483106 74.89145703154618) bank39025 +39026 POINT(40.85432094505998 73.28316470511035) bank39026 +39027 POINT(41.30000133383255 74.88998195563623) bank39027 +39028 POINT(40.10666292982786 74.63628550722098) bank39028 +39029 POINT(40.17087932094411 74.46692248507755) bank39029 +39030 POINT(40.23154753154474 73.70338993598189) bank39030 +39031 POINT(40.60521603687915 73.43367940737717) bank39031 +39032 POINT(40.195900242758306 74.24814281584739) bank39032 +39033 POINT(40.475017141537116 74.68015079414921) bank39033 +39034 POINT(39.7988139933199 73.61301186029243) bank39034 +39035 POINT(39.819458032927606 74.23514891751117) bank39035 +39036 POINT(41.47582192284587 74.06946811634333) bank39036 +39037 POINT(40.38487611737171 73.24416003036562) bank39037 +39038 POINT(40.85264537775926 74.78919037675512) bank39038 +39039 POINT(39.95535511018521 73.78696937266969) bank39039 +39040 POINT(40.41220466995569 74.98894355405804) bank39040 +39041 POINT(41.59877947831314 74.53929461328546) bank39041 +39042 POINT(41.08403383747078 74.93832957931549) bank39042 +39043 POINT(39.8979971766256 73.82925600521081) bank39043 +39044 POINT(40.63191967446906 73.04915453919546) bank39044 +39045 POINT(40.04611305116369 74.15049062929073) bank39045 +39046 POINT(41.669687999993045 74.55491000354014) bank39046 +39047 POINT(40.715195628622645 74.0235639038818) bank39047 +39048 POINT(41.542128275345135 73.03715337764699) bank39048 +39049 POINT(40.509413000000464 74.55993696385462) bank39049 +39050 POINT(40.53920390997479 74.27008503481737) bank39050 +39051 POINT(41.09381788201994 73.82508284267865) bank39051 +39052 POINT(40.19284946093527 73.66151773352122) bank39052 +39053 POINT(41.50846263815235 74.28510706963367) bank39053 +39054 POINT(41.548502094417465 73.5586283231996) bank39054 +39055 POINT(41.07173639841577 74.97711710706606) bank39055 +39056 POINT(40.056614655813526 73.23816493443795) bank39056 +39057 POINT(41.06198675794043 74.43422905047285) bank39057 +39058 POINT(41.492735901840256 74.44982538590958) bank39058 +39059 POINT(40.41027703142275 74.47468842249553) bank39059 +39060 POINT(39.86325155554592 73.87212524146778) bank39060 +39061 POINT(41.3511868746484 74.353802599827) bank39061 +39062 POINT(40.484365474707246 73.95247389168385) bank39062 +39063 POINT(39.8347290129797 74.35709584918997) bank39063 +39064 POINT(40.78633050563776 74.74346620469119) bank39064 +39065 POINT(40.6958517466746 74.5433888277762) bank39065 +39066 POINT(40.02150969239673 73.542785652831) bank39066 +39067 POINT(40.26849029603214 74.759968884158) bank39067 +39068 POINT(40.25649624877442 74.21010238105151) bank39068 +39069 POINT(40.4180368066152 74.53844007428853) bank39069 +39070 POINT(40.18114300761267 74.19293308501689) bank39070 +39071 POINT(40.920570729918325 74.46359506260801) bank39071 +39072 POINT(40.55726299796917 73.95626157173895) bank39072 +39073 POINT(40.307533781305644 74.35792436970841) bank39073 +39074 POINT(41.30670960471695 73.26972943685655) bank39074 +39075 POINT(40.84179323319305 73.46978141766179) bank39075 +39076 POINT(41.62528487579754 73.06641910360679) bank39076 +39077 POINT(41.661653593298766 73.04214127205024) bank39077 +39078 POINT(41.0633439144146 73.32370630036486) bank39078 +39079 POINT(41.51440439865971 74.04930713221077) bank39079 +39080 POINT(39.946895830879946 73.30311019082794) bank39080 +39081 POINT(40.52856481323969 73.97169861034203) bank39081 +39082 POINT(41.622774821930015 73.91787126568184) bank39082 +39083 POINT(41.546223737745336 73.21668027784013) bank39083 +39084 POINT(40.61648454762243 74.25138101251173) bank39084 +39085 POINT(41.379398633066664 73.51646885610774) bank39085 +39086 POINT(40.92887669236773 74.7351749750488) bank39086 +39087 POINT(40.4469547198659 74.4494886796241) bank39087 +39088 POINT(40.165037184923676 74.16600020817563) bank39088 +39089 POINT(39.82471664543325 74.8518661660813) bank39089 +39090 POINT(41.12756340000743 74.060909166296) bank39090 +39091 POINT(40.62223104046592 74.27938620861353) bank39091 +39092 POINT(41.50729663802021 74.4796764847053) bank39092 +39093 POINT(39.84316184769479 74.95343284301987) bank39093 +39094 POINT(40.97801633242535 74.71181859147704) bank39094 +39095 POINT(40.16749483009889 74.11507327468286) bank39095 +39096 POINT(40.21304918432214 73.17964017597878) bank39096 +39097 POINT(39.954740040271204 74.41363850014434) bank39097 +39098 POINT(41.326696363444505 73.44091535318182) bank39098 +39099 POINT(40.15378177086656 73.1625353488343) bank39099 +39100 POINT(40.47390205917528 73.39830852781004) bank39100 +39101 POINT(40.87385395422838 74.76641360272636) bank39101 +39102 POINT(41.574659977115864 73.74797915418908) bank39102 +39103 POINT(40.93898942284448 73.01234768848713) bank39103 +39104 POINT(40.37400533649176 74.46057757502494) bank39104 +39105 POINT(41.46050417270373 74.14204741046188) bank39105 +39106 POINT(41.496948821681876 73.38377928469474) bank39106 +39107 POINT(41.007718943461036 74.7793052043485) bank39107 +39108 POINT(40.73596171601676 74.03367467572625) bank39108 +39109 POINT(40.191319919070615 73.64126165540786) bank39109 +39110 POINT(40.2395697116276 74.87763401421988) bank39110 +39111 POINT(40.162222532527316 74.94637431384791) bank39111 +39112 POINT(40.62574945749609 74.3249352583591) bank39112 +39113 POINT(40.710674669967275 74.24169253321477) bank39113 +39114 POINT(40.84404225166794 73.69328212958166) bank39114 +39115 POINT(39.762657022669266 73.5541417258584) bank39115 +39116 POINT(41.26355151808876 74.30763000334711) bank39116 +39117 POINT(40.44329352647018 74.88379985505466) bank39117 +39118 POINT(40.8906294899722 74.71223801280753) bank39118 +39119 POINT(40.56577495744937 74.10800295173604) bank39119 +39120 POINT(39.944279592261026 73.14043104920665) bank39120 +39121 POINT(40.17435048082907 74.96271897130676) bank39121 +39122 POINT(40.55088787124148 73.23091175479041) bank39122 +39123 POINT(40.94196948303674 73.25531961040313) bank39123 +39124 POINT(39.87357755914718 74.37077888095563) bank39124 +39125 POINT(39.974462100347694 74.42758690909429) bank39125 +39126 POINT(40.75738289381313 73.36057293586674) bank39126 +39127 POINT(41.08707921639341 74.7538496150432) bank39127 +39128 POINT(40.85888213229426 74.36758641573533) bank39128 +39129 POINT(40.21296596189566 74.5337497855804) bank39129 +39130 POINT(41.076582812705134 73.61062403622012) bank39130 +39131 POINT(41.07582923506472 74.97068226423283) bank39131 +39132 POINT(40.35079058126424 73.80783854989181) bank39132 +39133 POINT(40.81440243642306 73.76419942843727) bank39133 +39134 POINT(39.89411477724827 73.9812469031784) bank39134 +39135 POINT(39.95547404036168 74.00580060684246) bank39135 +39136 POINT(41.27040771232561 74.54926507474975) bank39136 +39137 POINT(41.663459614806435 74.11219869084124) bank39137 +39138 POINT(40.025817332908275 73.03368160839526) bank39138 +39139 POINT(41.54890846323244 74.30367128688007) bank39139 +39140 POINT(41.203064570361086 73.57951657709135) bank39140 +39141 POINT(41.29601966198127 73.75834327603496) bank39141 +39142 POINT(40.687559560224926 73.90462760014104) bank39142 +39143 POINT(40.23742857938774 73.89717781402992) bank39143 +39144 POINT(40.849160531029334 73.2717503890541) bank39144 +39145 POINT(41.540881317887624 73.78487366827567) bank39145 +39146 POINT(40.63387672087451 74.15054431082832) bank39146 +39147 POINT(40.65956680421299 73.65145883414372) bank39147 +39148 POINT(40.14068388535016 74.97654372674066) bank39148 +39149 POINT(40.564831267834165 74.7266606941398) bank39149 +39150 POINT(40.64040774545919 73.41439934846858) bank39150 +39151 POINT(40.001817898482656 73.82767327894024) bank39151 +39152 POINT(40.0705292242503 75.00136566035918) bank39152 +39153 POINT(39.76603436802487 74.47343214322687) bank39153 +39154 POINT(41.602246201699614 73.41848614234702) bank39154 +39155 POINT(40.343015219040126 73.3390371652628) bank39155 +39156 POINT(41.091542417557214 73.18391891962669) bank39156 +39157 POINT(40.7439693360921 74.78560985293973) bank39157 +39158 POINT(40.28873929056563 73.98887019807995) bank39158 +39159 POINT(40.382267294258156 74.5814826605091) bank39159 +39160 POINT(41.082804850918954 73.44715229197098) bank39160 +39161 POINT(41.57996968292249 74.12712913566438) bank39161 +39162 POINT(39.98540356282746 73.8450474728249) bank39162 +39163 POINT(41.44024536625872 73.24517338290514) bank39163 +39164 POINT(41.4988715793907 74.00780841265666) bank39164 +39165 POINT(40.917779307367056 74.69992499671253) bank39165 +39166 POINT(41.18331322486276 74.24609699559437) bank39166 +39167 POINT(41.14401972550883 73.11097759921144) bank39167 +39168 POINT(39.825359957392294 73.46146987389231) bank39168 +39169 POINT(40.805321224548464 73.65870329093568) bank39169 +39170 POINT(41.565356029741565 74.28510943001989) bank39170 +39171 POINT(40.79470933504451 74.01923183822925) bank39171 +39172 POINT(40.648559842185065 74.44942508655303) bank39172 +39173 POINT(40.368494555109315 74.02106805351737) bank39173 +39174 POINT(40.088156275859085 74.21706037843845) bank39174 +39175 POINT(39.79425540531809 73.8274757290818) bank39175 +39176 POINT(40.65987549202112 74.09065276737952) bank39176 +39177 POINT(40.755204991739916 74.23594437584968) bank39177 +39178 POINT(40.35624301652794 74.28995763308905) bank39178 +39179 POINT(41.52175386298559 74.06270722430523) bank39179 +39180 POINT(41.27846155860686 74.28047871483147) bank39180 +39181 POINT(41.19732791145634 73.92151935431103) bank39181 +39182 POINT(39.81019956033614 73.85316271348) bank39182 +39183 POINT(41.53219024069137 74.14896723858726) bank39183 +39184 POINT(40.40978724825484 74.63609658162795) bank39184 +39185 POINT(41.70565943820711 73.95136246313916) bank39185 +39186 POINT(40.50697260261092 73.19994087799427) bank39186 +39187 POINT(39.99209958755859 74.29383173588016) bank39187 +39188 POINT(40.69216515924931 73.08256218305391) bank39188 +39189 POINT(40.08033602789949 73.11621886544192) bank39189 +39190 POINT(40.77447833497457 73.69011571435111) bank39190 +39191 POINT(41.6400381051333 73.35305195727497) bank39191 +39192 POINT(41.22536060018316 73.78056753371078) bank39192 +39193 POINT(39.924413423205856 74.18788986200288) bank39193 +39194 POINT(40.02442281427482 73.5127881968891) bank39194 +39195 POINT(39.723591252328724 74.39283374595233) bank39195 +39196 POINT(41.21540600340787 73.06868268336927) bank39196 +39197 POINT(41.11063699261596 73.40746417118132) bank39197 +39198 POINT(40.42592076479954 73.89591775515977) bank39198 +39199 POINT(41.61566689649283 74.55283895334176) bank39199 +39200 POINT(40.12990699284062 74.1178336287326) bank39200 +39201 POINT(41.26405903286965 74.20779255648256) bank39201 +39202 POINT(40.74385982047936 74.58817289227339) bank39202 +39203 POINT(40.6314477534782 74.89981441054421) bank39203 +39204 POINT(41.664966280150864 73.99611742487936) bank39204 +39205 POINT(41.03377063320948 74.10488417705055) bank39205 +39206 POINT(40.61117734581849 73.55459175097612) bank39206 +39207 POINT(39.856201809497506 73.32154590120281) bank39207 +39208 POINT(40.17587095660795 73.6550162163734) bank39208 +39209 POINT(41.31391594841088 74.23152701430755) bank39209 +39210 POINT(41.21528085525708 74.44355196360327) bank39210 +39211 POINT(40.16630970525346 74.47017167892955) bank39211 +39212 POINT(39.91749800950324 74.36870138416734) bank39212 +39213 POINT(40.10868338670116 73.39280933834527) bank39213 +39214 POINT(39.762079319803064 73.79081878255379) bank39214 +39215 POINT(41.361848132268285 74.74458638726296) bank39215 +39216 POINT(41.45751777369991 74.40320698326536) bank39216 +39217 POINT(41.1127792285133 74.06530581871125) bank39217 +39218 POINT(39.9562585772057 74.72045154578745) bank39218 +39219 POINT(41.64627240041119 73.33842880169195) bank39219 +39220 POINT(41.59941270461744 74.94315556197172) bank39220 +39221 POINT(40.08176556894461 73.20891983426581) bank39221 +39222 POINT(41.11832920778708 73.37118031814924) bank39222 +39223 POINT(39.763065246039595 73.5631464263438) bank39223 +39224 POINT(41.62056272125161 73.49109098909206) bank39224 +39225 POINT(41.16686206394399 73.52222956866767) bank39225 +39226 POINT(41.546980361257084 73.5059484245834) bank39226 +39227 POINT(41.12311946679152 73.8871600441198) bank39227 +39228 POINT(40.790118847456334 74.71323428109378) bank39228 +39229 POINT(40.45303659763496 73.1007985082375) bank39229 +39230 POINT(40.05386169960356 73.2733147608301) bank39230 +39231 POINT(39.86769703513906 74.75565217860378) bank39231 +39232 POINT(40.2765856867762 74.83743080118947) bank39232 +39233 POINT(41.201985509881844 73.31026265339788) bank39233 +39234 POINT(40.61428228054573 73.64133944850497) bank39234 +39235 POINT(41.11665528919968 73.88258951404072) bank39235 +39236 POINT(40.20883980117456 74.37350009770883) bank39236 +39237 POINT(40.4540405551478 74.20665096407204) bank39237 +39238 POINT(41.58987112786442 73.80462234061743) bank39238 +39239 POINT(41.41147501508428 74.67182107959667) bank39239 +39240 POINT(41.09964537130359 74.27828097524379) bank39240 +39241 POINT(40.493800030099635 74.02049768753558) bank39241 +39242 POINT(40.57175181851932 73.92772249555904) bank39242 +39243 POINT(41.173186369524984 74.06029372622795) bank39243 +39244 POINT(41.35145363097442 74.55840086581787) bank39244 +39245 POINT(39.754790881385475 73.82217214891641) bank39245 +39246 POINT(41.05839738994879 73.93858462582195) bank39246 +39247 POINT(40.54455574891821 74.28526199167497) bank39247 +39248 POINT(41.01002992407503 73.84166058349504) bank39248 +39249 POINT(41.14456612957459 73.6489903635712) bank39249 +39250 POINT(40.87254344997258 73.7967612939875) bank39250 +39251 POINT(40.79073912763322 74.29427652551803) bank39251 +39252 POINT(40.63207578387839 74.37101473512358) bank39252 +39253 POINT(41.655931259346445 74.18115797729008) bank39253 +39254 POINT(41.381503737817 73.86714444288143) bank39254 +39255 POINT(41.27690539157826 73.02666802721808) bank39255 +39256 POINT(41.14702187792172 73.86570468048821) bank39256 +39257 POINT(40.65979147966586 73.60265084883709) bank39257 +39258 POINT(41.18255347152591 73.5614118093941) bank39258 +39259 POINT(41.27021078681295 73.49392626227788) bank39259 +39260 POINT(40.75973322520911 74.36881064271378) bank39260 +39261 POINT(40.043367171061355 74.69426369756793) bank39261 +39262 POINT(39.75483504772308 73.32195690595286) bank39262 +39263 POINT(40.92406227146835 74.43459875915782) bank39263 +39264 POINT(40.28041430450673 73.36886340895035) bank39264 +39265 POINT(41.29863462489816 73.7312011859) bank39265 +39266 POINT(40.52386884646721 73.63324900313984) bank39266 +39267 POINT(39.80981674022538 73.05764402464547) bank39267 +39268 POINT(40.609216902469505 74.30572270303219) bank39268 +39269 POINT(41.48104672735822 74.21676153073466) bank39269 +39270 POINT(40.32311196031563 74.33002346566276) bank39270 +39271 POINT(39.94197000874631 74.29799908612281) bank39271 +39272 POINT(40.69572484324883 74.59512811878562) bank39272 +39273 POINT(41.100154493323636 73.36389757459794) bank39273 +39274 POINT(40.78910615107794 73.32196684475059) bank39274 +39275 POINT(40.25968219687397 74.60869909121357) bank39275 +39276 POINT(40.19506203743571 74.85159169477289) bank39276 +39277 POINT(40.33832841926329 74.92631783836175) bank39277 +39278 POINT(41.59995093321209 73.74926818526436) bank39278 +39279 POINT(40.141301199337995 73.85664834330245) bank39279 +39280 POINT(41.2379129684898 73.98492236818173) bank39280 +39281 POINT(40.88377535323869 74.63925996766451) bank39281 +39282 POINT(40.80084386420601 73.32297186512784) bank39282 +39283 POINT(41.18387974865992 73.93639664968971) bank39283 +39284 POINT(41.69351559914265 73.54093796589964) bank39284 +39285 POINT(41.41202673363927 73.92536578557645) bank39285 +39286 POINT(39.820737854167454 74.59378116691362) bank39286 +39287 POINT(41.10037378635717 73.5385120255483) bank39287 +39288 POINT(40.33018734676143 74.38546166264867) bank39288 +39289 POINT(41.542982677465574 75.00107214069894) bank39289 +39290 POINT(41.677222165419074 74.21301483203109) bank39290 +39291 POINT(40.33320803296794 74.86308223749725) bank39291 +39292 POINT(40.85000224830963 74.27732592847185) bank39292 +39293 POINT(40.08557918028966 74.69482946867574) bank39293 +39294 POINT(40.90748779076141 73.04836007211449) bank39294 +39295 POINT(40.82979738329831 74.06737485202895) bank39295 +39296 POINT(39.809177633065765 73.13816142429174) bank39296 +39297 POINT(40.87891281325176 73.02238871114793) bank39297 +39298 POINT(41.16721176755866 73.39216416267317) bank39298 +39299 POINT(41.03197470756633 74.02711486973179) bank39299 +39300 POINT(40.26607322354213 74.04173598344754) bank39300 +39301 POINT(40.85445466134584 74.53184698960663) bank39301 +39302 POINT(40.51341774056026 74.61240201947447) bank39302 +39303 POINT(39.877123561615306 74.30864584648833) bank39303 +39304 POINT(41.365728758364845 73.97782716236391) bank39304 +39305 POINT(41.7043327791293 73.1820664558908) bank39305 +39306 POINT(40.49822144389303 73.9304826277761) bank39306 +39307 POINT(40.9179873170488 74.41937989424127) bank39307 +39308 POINT(40.190630924542 74.49837342642563) bank39308 +39309 POINT(41.581759102462044 74.96707993225301) bank39309 +39310 POINT(40.40454416739476 74.54817979214091) bank39310 +39311 POINT(40.15717420831795 73.69797476223296) bank39311 +39312 POINT(40.827700277457886 74.17284064563712) bank39312 +39313 POINT(40.76763286310246 74.7803604769414) bank39313 +39314 POINT(41.52264344119363 73.71578550517134) bank39314 +39315 POINT(39.741949201424596 74.81345532105476) bank39315 +39316 POINT(40.24613782496789 74.75692780326446) bank39316 +39317 POINT(39.7858863207813 74.27180357878956) bank39317 +39318 POINT(40.42452973585934 74.16017064990004) bank39318 +39319 POINT(39.89458833832693 73.42032393604188) bank39319 +39320 POINT(41.42603583371716 74.13307713742186) bank39320 +39321 POINT(41.06719972984502 74.52757618425593) bank39321 +39322 POINT(40.46090339512009 73.57279342858678) bank39322 +39323 POINT(39.94900167447245 73.62485120093238) bank39323 +39324 POINT(41.59578311376615 73.16551696716935) bank39324 +39325 POINT(41.034409333339134 74.53753603543936) bank39325 +39326 POINT(40.53375617449348 73.48865951223267) bank39326 +39327 POINT(40.7297492290094 73.95987272356145) bank39327 +39328 POINT(40.40434319116311 74.90713011587971) bank39328 +39329 POINT(40.397922406973315 74.06992574402521) bank39329 +39330 POINT(40.78276219467069 73.11366490677213) bank39330 +39331 POINT(40.360166738836085 74.75233057460385) bank39331 +39332 POINT(40.259056769473716 73.31835646356699) bank39332 +39333 POINT(40.78669863412715 74.89446379578582) bank39333 +39334 POINT(40.892467975569055 73.27520553687987) bank39334 +39335 POINT(41.042231774081024 73.79959058794273) bank39335 +39336 POINT(40.535893595251984 73.79866825381771) bank39336 +39337 POINT(40.941209137481096 73.47794397506586) bank39337 +39338 POINT(40.597564395474 74.4578318954873) bank39338 +39339 POINT(40.70273720473319 73.00827716710879) bank39339 +39340 POINT(39.92722495970698 74.72018946070851) bank39340 +39341 POINT(41.68293232311511 74.21224715555132) bank39341 +39342 POINT(40.85587802787845 74.30291553055844) bank39342 +39343 POINT(40.93228229748805 74.47626314307308) bank39343 +39344 POINT(40.66123095485679 73.16565214683497) bank39344 +39345 POINT(40.16153113429373 74.0972679341568) bank39345 +39346 POINT(41.684674251657135 74.9376512001564) bank39346 +39347 POINT(41.30389679873728 73.15829029699849) bank39347 +39348 POINT(41.381540664833395 74.51430698334245) bank39348 +39349 POINT(40.79118892381244 73.48226128336508) bank39349 +39350 POINT(41.25623665741545 73.31234134181858) bank39350 +39351 POINT(39.951878538456576 74.89090435073398) bank39351 +39352 POINT(41.064955531302765 73.17534915807794) bank39352 +39353 POINT(40.686790555651285 74.85394078183973) bank39353 +39354 POINT(40.038950399899925 74.84516002775784) bank39354 +39355 POINT(41.05146969515355 74.82929405514513) bank39355 +39356 POINT(40.40623852393108 73.25263691354944) bank39356 +39357 POINT(39.91890257078607 74.42897133682486) bank39357 +39358 POINT(40.499774373697726 73.62330167283274) bank39358 +39359 POINT(40.49746624506361 73.72795310203544) bank39359 +39360 POINT(39.9237401079007 74.69652354777882) bank39360 +39361 POINT(40.063435818587145 73.38931302016651) bank39361 +39362 POINT(41.290066954650364 73.51860885215386) bank39362 +39363 POINT(40.33618041532699 74.07806593793092) bank39363 +39364 POINT(40.572568357919366 74.69482550968704) bank39364 +39365 POINT(41.66920698411809 73.57264625042582) bank39365 +39366 POINT(40.16945302323187 73.55071041934747) bank39366 +39367 POINT(40.31180831151021 73.37593272680684) bank39367 +39368 POINT(40.10846254937428 73.86446519854182) bank39368 +39369 POINT(41.38344824507451 74.50822411278035) bank39369 +39370 POINT(40.363460467821184 74.45318253724746) bank39370 +39371 POINT(40.09920298547342 73.55867209166988) bank39371 +39372 POINT(40.09881332809955 74.506529310239) bank39372 +39373 POINT(40.487195608671115 74.2141760709794) bank39373 +39374 POINT(40.04890517148307 74.84168992064154) bank39374 +39375 POINT(40.90228102047723 73.02466646572816) bank39375 +39376 POINT(40.26563660139672 73.812730238026) bank39376 +39377 POINT(41.33037112363207 73.5956961411466) bank39377 +39378 POINT(41.69282608131255 73.86122342950715) bank39378 +39379 POINT(41.17791921116743 73.18453481729978) bank39379 +39380 POINT(39.780052211627556 73.11778902654459) bank39380 +39381 POINT(40.73134193037032 73.05610953472234) bank39381 +39382 POINT(40.18448587077355 74.34038086015204) bank39382 +39383 POINT(41.55765339388156 73.43847673068505) bank39383 +39384 POINT(40.46026420770323 73.90980502154224) bank39384 +39385 POINT(40.61376882660216 73.16159304318273) bank39385 +39386 POINT(41.24228530831239 74.68728196391278) bank39386 +39387 POINT(40.39171126576952 73.53957147565418) bank39387 +39388 POINT(41.038697651625725 74.7465498961278) bank39388 +39389 POINT(41.00854385918019 74.80361668624016) bank39389 +39390 POINT(41.45117780037189 74.95716789505502) bank39390 +39391 POINT(40.51192902620644 74.60332169661632) bank39391 +39392 POINT(39.818997014856706 73.97271652485085) bank39392 +39393 POINT(40.77144623215359 74.02105664596965) bank39393 +39394 POINT(40.83165746096908 74.15493751061052) bank39394 +39395 POINT(41.30217913561566 74.91658084992584) bank39395 +39396 POINT(39.747114720837864 74.96026449313727) bank39396 +39397 POINT(40.69541948097538 73.367584842529) bank39397 +39398 POINT(40.306876630510594 74.34525332087495) bank39398 +39399 POINT(41.080100530137 73.6294236626032) bank39399 +39400 POINT(40.306166517234765 74.40121835333399) bank39400 +39401 POINT(40.97407498062859 74.02207106079142) bank39401 +39402 POINT(39.900661716368504 73.0675347666153) bank39402 +39403 POINT(40.271909001449906 74.43816628551934) bank39403 +39404 POINT(41.33044991462501 73.62134701976942) bank39404 +39405 POINT(39.86289029715317 74.08941670405488) bank39405 +39406 POINT(41.40736502398897 74.78405612874619) bank39406 +39407 POINT(40.65839266563475 73.35304269324175) bank39407 +39408 POINT(41.42131381857433 73.50458291180524) bank39408 +39409 POINT(40.153033990864216 73.82125765745386) bank39409 +39410 POINT(40.33324153602854 73.21522141297287) bank39410 +39411 POINT(41.63324451538311 74.39865019248869) bank39411 +39412 POINT(39.79019566587364 73.38517494581824) bank39412 +39413 POINT(40.53382257819341 74.74402491269821) bank39413 +39414 POINT(41.63056055322084 74.14375747263963) bank39414 +39415 POINT(40.65306201039764 74.83131160910305) bank39415 +39416 POINT(40.87669868660912 74.43510930514564) bank39416 +39417 POINT(40.36996937259558 74.91547859317842) bank39417 +39418 POINT(41.469098183889834 74.41430803795933) bank39418 +39419 POINT(40.6384412314968 74.36883774201608) bank39419 +39420 POINT(41.17285241264715 73.85850057394414) bank39420 +39421 POINT(40.227685694822306 74.9604571907228) bank39421 +39422 POINT(41.67268427121866 73.11547636673765) bank39422 +39423 POINT(40.21972261307288 73.12992066470304) bank39423 +39424 POINT(40.03725176382317 73.77307567601446) bank39424 +39425 POINT(41.156047488092895 74.28770401518256) bank39425 +39426 POINT(40.06339659562456 73.36047821154489) bank39426 +39427 POINT(40.99471735834057 73.3544140442523) bank39427 +39428 POINT(41.09910234435178 73.30646402513244) bank39428 +39429 POINT(40.319954039409154 74.69424815170676) bank39429 +39430 POINT(40.622539894479914 73.92174786987393) bank39430 +39431 POINT(40.764758923574114 74.7106773597373) bank39431 +39432 POINT(40.25635455006079 73.40988510281423) bank39432 +39433 POINT(40.43470746680318 74.5799298549051) bank39433 +39434 POINT(40.486285005982225 73.72532493302089) bank39434 +39435 POINT(41.5999606401117 73.96132327058952) bank39435 +39436 POINT(41.44002681653429 73.33653789161777) bank39436 +39437 POINT(40.060714391786405 74.78171498697571) bank39437 +39438 POINT(41.4395028209118 73.04616991892996) bank39438 +39439 POINT(39.7601431461873 74.47474329238761) bank39439 +39440 POINT(40.09673537459765 73.93415403256667) bank39440 +39441 POINT(40.856598490192965 73.9087075677455) bank39441 +39442 POINT(40.732182852635724 73.19428455047222) bank39442 +39443 POINT(40.54834648328659 74.67420237751752) bank39443 +39444 POINT(40.01497051902435 73.63449251289781) bank39444 +39445 POINT(41.12207533416418 74.18601643942657) bank39445 +39446 POINT(41.47774729098698 73.93986107945457) bank39446 +39447 POINT(39.97873183174234 73.10017738049967) bank39447 +39448 POINT(40.74762216150577 74.4094348816012) bank39448 +39449 POINT(40.75344757855707 74.40443729683406) bank39449 +39450 POINT(40.443336010442586 73.74247611860373) bank39450 +39451 POINT(40.32197455648498 74.77015103288687) bank39451 +39452 POINT(40.29940114801631 74.0967211580379) bank39452 +39453 POINT(40.069374510780335 73.46149485547157) bank39453 +39454 POINT(40.771448113608315 73.92648545907204) bank39454 +39455 POINT(40.690878575478216 74.35575241681511) bank39455 +39456 POINT(39.723280931358396 73.58430691295338) bank39456 +39457 POINT(41.69165961441766 73.9950003870101) bank39457 +39458 POINT(41.554116296007656 74.38261360622533) bank39458 +39459 POINT(41.57652011888763 73.00836500620163) bank39459 +39460 POINT(41.49169644059732 74.99126376314369) bank39460 +39461 POINT(40.274293308585236 73.4201106772686) bank39461 +39462 POINT(41.08992591883506 73.99478364826625) bank39462 +39463 POINT(41.01930311028385 73.94864049088173) bank39463 +39464 POINT(41.062159095837885 74.31301481875452) bank39464 +39465 POINT(40.571451466928444 74.06958066992772) bank39465 +39466 POINT(40.26833304050805 74.93648683089758) bank39466 +39467 POINT(40.549350684246235 74.70136614227116) bank39467 +39468 POINT(41.540993941520455 73.10026624327843) bank39468 +39469 POINT(40.348939186995985 74.17291241682743) bank39469 +39470 POINT(40.326444565440966 73.75415636140347) bank39470 +39471 POINT(40.576622962218345 74.50206891810336) bank39471 +39472 POINT(40.480564614140086 73.84218014455533) bank39472 +39473 POINT(41.20823278795137 73.35124230638188) bank39473 +39474 POINT(41.569876299913005 73.02727267115515) bank39474 +39475 POINT(40.47420300415592 73.55432182750526) bank39475 +39476 POINT(39.73715997822964 73.55554683847195) bank39476 +39477 POINT(40.01582141278075 74.91060660579964) bank39477 +39478 POINT(41.07134011299458 73.31693497174508) bank39478 +39479 POINT(40.218626973477356 73.27464980362565) bank39479 +39480 POINT(41.466783019074896 73.08324668748563) bank39480 +39481 POINT(41.28173685206186 74.68728331445097) bank39481 +39482 POINT(40.80828129486485 73.05146845509368) bank39482 +39483 POINT(39.88012679738332 73.75525119805216) bank39483 +39484 POINT(41.66611744052342 74.6019611712916) bank39484 +39485 POINT(40.10613515760477 73.24161266033748) bank39485 +39486 POINT(40.72908132440742 74.41971964630673) bank39486 +39487 POINT(41.0177805898492 73.43801630055147) bank39487 +39488 POINT(40.6831564574764 73.02552110245334) bank39488 +39489 POINT(40.90221313517599 73.22264540719452) bank39489 +39490 POINT(41.05075724029603 74.31669740609836) bank39490 +39491 POINT(41.60480932394953 73.73340005878003) bank39491 +39492 POINT(40.27661056204307 74.29429417952755) bank39492 +39493 POINT(40.52358511744326 73.32389574694106) bank39493 +39494 POINT(41.249080867244736 73.91052115067288) bank39494 +39495 POINT(41.36422658519541 73.42180367900212) bank39495 +39496 POINT(40.66696325666928 74.75486914277283) bank39496 +39497 POINT(39.868540621756395 73.87015082755511) bank39497 +39498 POINT(40.349030216028375 74.03157575866749) bank39498 +39499 POINT(41.687576704633614 73.11486799655113) bank39499 +39500 POINT(40.19447981388797 74.98580444022325) bank39500 +39501 POINT(40.431083035915684 74.77494493868383) bank39501 +39502 POINT(40.069257821450314 74.43284537555235) bank39502 +39503 POINT(40.19514415851419 73.1224513762789) bank39503 +39504 POINT(40.73314106679143 74.39394497284475) bank39504 +39505 POINT(40.12030677590227 74.9455430428348) bank39505 +39506 POINT(41.52789352116541 73.73710154919651) bank39506 +39507 POINT(40.49561785202554 73.44633841474612) bank39507 +39508 POINT(39.71557361986155 74.79971725734626) bank39508 +39509 POINT(39.71954029969153 73.33956021128019) bank39509 +39510 POINT(40.24712831360321 73.51914498078939) bank39510 +39511 POINT(40.2401168944139 74.27985595037883) bank39511 +39512 POINT(41.370624590877235 74.76925387847169) bank39512 +39513 POINT(39.95345530465549 74.74091630480578) bank39513 +39514 POINT(40.149836548838806 73.03356766616147) bank39514 +39515 POINT(40.76334993903068 74.1822488114353) bank39515 +39516 POINT(40.70547097786179 73.43510227212482) bank39516 +39517 POINT(40.27307374954963 74.84366607432673) bank39517 +39518 POINT(40.575261477980376 73.27747926163703) bank39518 +39519 POINT(41.374210506769444 73.63084825349337) bank39519 +39520 POINT(40.458903895812995 74.30243265462514) bank39520 +39521 POINT(41.66408975767279 74.35722852900663) bank39521 +39522 POINT(40.65525885652717 73.51358282347933) bank39522 +39523 POINT(40.0715092050297 73.23943697345771) bank39523 +39524 POINT(41.56004702857503 74.82629591520121) bank39524 +39525 POINT(40.151758819480655 73.0456321502424) bank39525 +39526 POINT(39.81870135341302 73.15061779370703) bank39526 +39527 POINT(40.39358191014562 74.41386878437879) bank39527 +39528 POINT(41.15721005991861 74.73956791145552) bank39528 +39529 POINT(40.32069359083389 74.17942510441017) bank39529 +39530 POINT(41.245255663012216 74.50621422435567) bank39530 +39531 POINT(41.29603001095472 74.70657865648485) bank39531 +39532 POINT(39.88811211371011 73.04433600620239) bank39532 +39533 POINT(41.70261315832853 73.7547887867943) bank39533 +39534 POINT(41.47621361759286 74.74131153891074) bank39534 +39535 POINT(40.75148147098815 74.30942284588707) bank39535 +39536 POINT(41.417638264233375 73.78006964128471) bank39536 +39537 POINT(40.57380791870691 73.02571138027437) bank39537 +39538 POINT(40.753622537031454 73.46425179516957) bank39538 +39539 POINT(40.57260891338311 73.25755581677677) bank39539 +39540 POINT(40.165632007837644 73.9947671379838) bank39540 +39541 POINT(40.44600137286264 74.7981183376743) bank39541 +39542 POINT(40.06716424397777 74.55489435683101) bank39542 +39543 POINT(41.44032106023293 74.08044759099786) bank39543 +39544 POINT(40.46785559138496 73.72585868504137) bank39544 +39545 POINT(41.3425531141248 74.81585645207306) bank39545 +39546 POINT(39.92382630195355 74.78144223006032) bank39546 +39547 POINT(40.62943952457648 74.89633229413656) bank39547 +39548 POINT(40.09907215335746 73.39966629708013) bank39548 +39549 POINT(40.26455708383366 74.24765758038936) bank39549 +39550 POINT(40.345849121362306 73.00977522209291) bank39550 +39551 POINT(40.85078568020008 74.77070441977392) bank39551 +39552 POINT(40.70642221941883 74.10676908127763) bank39552 +39553 POINT(40.878262389439456 73.54988462901461) bank39553 +39554 POINT(41.32156963783114 73.3674016959163) bank39554 +39555 POINT(41.690972791173046 74.82287043478078) bank39555 +39556 POINT(39.91158108117442 73.08872893648326) bank39556 +39557 POINT(41.67986909874523 73.04091036675939) bank39557 +39558 POINT(41.401115510686836 74.39644494085586) bank39558 +39559 POINT(39.99970489690743 74.16587162653462) bank39559 +39560 POINT(41.221559325323426 74.60688877460555) bank39560 +39561 POINT(40.61375398249225 74.8713290337985) bank39561 +39562 POINT(40.257670651735225 74.29794877560772) bank39562 +39563 POINT(41.71264466264725 74.38237018038477) bank39563 +39564 POINT(40.866496486543305 74.08625050572817) bank39564 +39565 POINT(41.665462921837666 74.50897807000565) bank39565 +39566 POINT(40.62969396303972 74.81638459386082) bank39566 +39567 POINT(41.41929923861666 73.77048123339735) bank39567 +39568 POINT(39.749209734491686 75.00345265635241) bank39568 +39569 POINT(41.20341517914646 74.60534951462571) bank39569 +39570 POINT(40.7893369496864 74.08048741725214) bank39570 +39571 POINT(39.73997156570025 74.05431428487455) bank39571 +39572 POINT(40.43436622050666 73.72534826228373) bank39572 +39573 POINT(41.3419694598155 73.31532740255433) bank39573 +39574 POINT(41.39014615241608 74.27803534517194) bank39574 +39575 POINT(41.39597910218716 74.74689190678869) bank39575 +39576 POINT(41.37355170410303 74.02829407049042) bank39576 +39577 POINT(39.92960091964461 73.55088309901359) bank39577 +39578 POINT(40.3309269400988 73.31223637474477) bank39578 +39579 POINT(40.063389623232524 73.18681676465043) bank39579 +39580 POINT(41.39866557138561 74.40325041787823) bank39580 +39581 POINT(39.796120355493166 73.09612058189913) bank39581 +39582 POINT(40.21451812716414 73.02751850008237) bank39582 +39583 POINT(40.405131862885575 74.63000902566058) bank39583 +39584 POINT(41.592494132247175 74.14649080243629) bank39584 +39585 POINT(40.39201424288527 73.2372098106166) bank39585 +39586 POINT(39.99918695838264 73.32551766647165) bank39586 +39587 POINT(39.8750054000512 73.49257983632592) bank39587 +39588 POINT(39.948801575031936 73.71328250679696) bank39588 +39589 POINT(40.447193822368575 74.99127554404274) bank39589 +39590 POINT(41.685720389513186 74.74922588913529) bank39590 +39591 POINT(41.381590860640586 73.34798483736868) bank39591 +39592 POINT(39.996594963263504 73.57947901690673) bank39592 +39593 POINT(40.424695794103826 74.85350422796223) bank39593 +39594 POINT(40.28762251217807 73.66285300196107) bank39594 +39595 POINT(40.18993883993207 74.25062655520891) bank39595 +39596 POINT(39.8618254572869 74.0373751585854) bank39596 +39597 POINT(41.01860104495821 73.98197070778112) bank39597 +39598 POINT(41.285545543673734 74.96275301802001) bank39598 +39599 POINT(40.86506740049621 73.27443271781881) bank39599 +39600 POINT(39.95979270738054 73.66022055477771) bank39600 +39601 POINT(41.32454103076313 74.13286971620906) bank39601 +39602 POINT(40.51038915899 74.82722381985019) bank39602 +39603 POINT(39.759518826358395 74.8547115674311) bank39603 +39604 POINT(40.31264484234476 74.32166725458762) bank39604 +39605 POINT(41.46042651757068 73.5485593227043) bank39605 +39606 POINT(40.52441264567773 74.05985108669) bank39606 +39607 POINT(40.109515488660904 73.27966483943182) bank39607 +39608 POINT(40.7066260224434 73.71194245639948) bank39608 +39609 POINT(41.168937488124016 74.20183589258708) bank39609 +39610 POINT(39.78936571020288 74.88637190076962) bank39610 +39611 POINT(40.00887481200946 73.60726034461842) bank39611 +39612 POINT(40.42838738119999 73.89562525942334) bank39612 +39613 POINT(40.896995740723625 74.95784018659826) bank39613 +39614 POINT(40.63323046687994 74.27692537119272) bank39614 +39615 POINT(39.85232262116161 74.35857752938307) bank39615 +39616 POINT(40.88438183800895 73.11546915076134) bank39616 +39617 POINT(40.989740006535 74.52164603427072) bank39617 +39618 POINT(40.7212445742638 73.01329686454639) bank39618 +39619 POINT(40.30444252585723 74.19777443926705) bank39619 +39620 POINT(39.894028044870815 74.63811854730253) bank39620 +39621 POINT(40.031532158185556 73.84004451537996) bank39621 +39622 POINT(41.592469970619625 74.27697022012677) bank39622 +39623 POINT(40.83382870859375 73.95804207400238) bank39623 +39624 POINT(40.562398926060425 73.9690111453015) bank39624 +39625 POINT(40.79401340728744 74.01291664176574) bank39625 +39626 POINT(41.70149428016773 73.8884191071054) bank39626 +39627 POINT(40.41998201684381 74.3046196430535) bank39627 +39628 POINT(40.10004677802554 73.49726399844371) bank39628 +39629 POINT(40.2430659533576 73.91567979552711) bank39629 +39630 POINT(39.85562056736907 73.33062090107859) bank39630 +39631 POINT(39.88642930945905 74.5387480517015) bank39631 +39632 POINT(41.09723512075002 74.37335226609291) bank39632 +39633 POINT(41.1301471116445 73.27633871773646) bank39633 +39634 POINT(40.8784102356064 73.22029503650704) bank39634 +39635 POINT(40.047672891186515 74.75798864319599) bank39635 +39636 POINT(41.06542596197307 73.38397305446811) bank39636 +39637 POINT(40.558786080402264 73.60770458176061) bank39637 +39638 POINT(39.84738043289215 74.62436042317819) bank39638 +39639 POINT(40.71295060919217 73.95270846870183) bank39639 +39640 POINT(40.48585189004005 74.77475279188843) bank39640 +39641 POINT(41.16434953859817 74.97031774608662) bank39641 +39642 POINT(41.596571496778495 73.98474666872708) bank39642 +39643 POINT(41.58044496049088 74.58755811974154) bank39643 +39644 POINT(40.55692883470167 74.46511262189819) bank39644 +39645 POINT(41.45863573120706 74.38946363395922) bank39645 +39646 POINT(41.615085635563815 74.4376263892283) bank39646 +39647 POINT(40.67876807214421 73.07123608911694) bank39647 +39648 POINT(40.85823863693822 73.68829809786843) bank39648 +39649 POINT(39.79970939336554 73.11092495040018) bank39649 +39650 POINT(40.0776410908902 73.41316827351889) bank39650 +39651 POINT(40.746123819271155 73.3255139298528) bank39651 +39652 POINT(40.10987275494795 73.52367521141036) bank39652 +39653 POINT(40.30195626690451 74.87467847242728) bank39653 +39654 POINT(41.38675838494004 73.33973794870633) bank39654 +39655 POINT(40.160286110347236 74.92277776261449) bank39655 +39656 POINT(39.85274082761631 73.21349918114505) bank39656 +39657 POINT(41.630763859213 74.45659267372855) bank39657 +39658 POINT(40.0226180597489 74.37730861819665) bank39658 +39659 POINT(40.984304464676946 73.02452861747507) bank39659 +39660 POINT(40.86940645520013 73.36287471814033) bank39660 +39661 POINT(40.921424402188954 74.89367031326977) bank39661 +39662 POINT(41.291240498285404 74.61961136183773) bank39662 +39663 POINT(41.486262675236894 73.29895629045649) bank39663 +39664 POINT(39.73799034481347 73.57169110660385) bank39664 +39665 POINT(41.035227099338705 73.76012030741032) bank39665 +39666 POINT(41.29854315172126 74.19684787422311) bank39666 +39667 POINT(39.820137854709344 73.86658454134839) bank39667 +39668 POINT(41.56154691641597 73.60359080370155) bank39668 +39669 POINT(41.216205149191566 74.64042783434274) bank39669 +39670 POINT(39.82176968488709 73.32583007042135) bank39670 +39671 POINT(40.23098127539119 73.3750494256423) bank39671 +39672 POINT(40.449416348960234 74.58296651192904) bank39672 +39673 POINT(40.81138661263085 73.01463921388721) bank39673 +39674 POINT(40.491968500833345 73.07754367257996) bank39674 +39675 POINT(40.98776718479317 73.83880174572637) bank39675 +39676 POINT(41.491610125839046 74.46998936905433) bank39676 +39677 POINT(40.140762142322856 74.99350871511099) bank39677 +39678 POINT(41.012479079114954 74.44162674086537) bank39678 +39679 POINT(40.20057805011656 74.72470155468937) bank39679 +39680 POINT(41.079602507202026 74.8025509672686) bank39680 +39681 POINT(39.9868456133672 74.73516679908455) bank39681 +39682 POINT(39.91059683363265 73.5560833998944) bank39682 +39683 POINT(40.96494595235152 74.56569475998869) bank39683 +39684 POINT(39.97835368607074 73.5036386513881) bank39684 +39685 POINT(41.4090075266765 73.3628076000684) bank39685 +39686 POINT(39.80403032944362 73.62510460879217) bank39686 +39687 POINT(40.13333816387785 74.9915458006486) bank39687 +39688 POINT(41.266684946462235 74.6475584023029) bank39688 +39689 POINT(40.61647752777716 74.20696896639438) bank39689 +39690 POINT(41.64066937292107 74.99970950303926) bank39690 +39691 POINT(41.34556736735829 73.88219476119555) bank39691 +39692 POINT(39.80467341289281 73.10864445936036) bank39692 +39693 POINT(39.98265008100592 73.7347101566853) bank39693 +39694 POINT(40.75686653803912 74.56172090438592) bank39694 +39695 POINT(41.068931958122974 74.1657234099274) bank39695 +39696 POINT(39.88722761251653 73.39845372381154) bank39696 +39697 POINT(39.879272760384026 73.8165054782607) bank39697 +39698 POINT(41.12769771822258 74.90329079227226) bank39698 +39699 POINT(41.05826624619776 73.0380301805604) bank39699 +39700 POINT(39.78239382975721 73.10794763715606) bank39700 +39701 POINT(41.45882011470296 73.28360432738361) bank39701 +39702 POINT(39.83273667718009 73.70399709908713) bank39702 +39703 POINT(39.840240546939775 73.70626789801952) bank39703 +39704 POINT(40.73366196667792 74.08328602966755) bank39704 +39705 POINT(41.53997961552256 74.50649228419199) bank39705 +39706 POINT(41.155227755087864 73.14998424875404) bank39706 +39707 POINT(40.68938684177131 74.5365756580445) bank39707 +39708 POINT(40.83661617176389 74.19303682834926) bank39708 +39709 POINT(40.018855604067014 74.80657870490087) bank39709 +39710 POINT(39.83966287019325 74.71115551201002) bank39710 +39711 POINT(40.243776256415224 74.566316250499) bank39711 +39712 POINT(41.1701541006388 73.26000911637905) bank39712 +39713 POINT(40.937616990789955 73.83402196198723) bank39713 +39714 POINT(41.02023688006908 74.21510549273275) bank39714 +39715 POINT(40.059204211995095 73.20760363748029) bank39715 +39716 POINT(40.493297192847024 73.03323448761319) bank39716 +39717 POINT(40.60597271785603 74.89070112104312) bank39717 +39718 POINT(40.559440524596646 74.74373135504543) bank39718 +39719 POINT(39.77687739631621 74.18303656641305) bank39719 +39720 POINT(41.062552148746434 73.05635312097236) bank39720 +39721 POINT(41.17868471827632 73.04820107210894) bank39721 +39722 POINT(41.02253277223327 74.93122208067103) bank39722 +39723 POINT(41.48893657511635 74.9217126886455) bank39723 +39724 POINT(41.252026676636895 74.38369965387338) bank39724 +39725 POINT(40.47469333773412 73.00884816324088) bank39725 +39726 POINT(40.38353821698366 73.81583376679507) bank39726 +39727 POINT(39.71366731956761 74.85496770402709) bank39727 +39728 POINT(40.39052967439066 73.33322349098877) bank39728 +39729 POINT(40.28073578600319 74.03058142717474) bank39729 +39730 POINT(41.476934907769554 74.52670783790784) bank39730 +39731 POINT(41.48422111634915 74.76027393991298) bank39731 +39732 POINT(40.77828525555638 74.44288148497671) bank39732 +39733 POINT(40.587978402763845 74.82229219161898) bank39733 +39734 POINT(41.37049213981197 74.25216210003217) bank39734 +39735 POINT(41.02110045998178 74.90464913004013) bank39735 +39736 POINT(39.892302331354976 73.56219651770093) bank39736 +39737 POINT(39.73683659791663 74.7595573543357) bank39737 +39738 POINT(39.790726718750825 74.99111841226276) bank39738 +39739 POINT(40.04325854997702 74.62206756076273) bank39739 +39740 POINT(41.02979461007885 74.0467640332123) bank39740 +39741 POINT(41.677645910257766 73.06112457321596) bank39741 +39742 POINT(39.88477912198979 73.66604146553726) bank39742 +39743 POINT(40.16116537473025 74.55190773153227) bank39743 +39744 POINT(41.145606071359396 73.33719166439272) bank39744 +39745 POINT(41.67678562350238 74.06179555654349) bank39745 +39746 POINT(40.64035520855489 73.87344142080684) bank39746 +39747 POINT(40.482818271654374 74.57868611320895) bank39747 +39748 POINT(40.66005844322646 74.372232581064) bank39748 +39749 POINT(41.176035670058006 74.11741614522863) bank39749 +39750 POINT(40.770821593152505 74.01726602322145) bank39750 +39751 POINT(40.274332434038875 74.02275282690238) bank39751 +39752 POINT(39.76383040528819 73.26198729705922) bank39752 +39753 POINT(41.0881112151809 73.87450077771928) bank39753 +39754 POINT(41.18510754478822 73.58016740571799) bank39754 +39755 POINT(40.23483580029563 73.5139414382807) bank39755 +39756 POINT(40.559574255087526 74.00200236792745) bank39756 +39757 POINT(41.607455523080255 74.59968732428803) bank39757 +39758 POINT(41.31708780412517 74.95527492242587) bank39758 +39759 POINT(40.888360778826865 74.06536740977997) bank39759 +39760 POINT(40.24346528391279 73.1444348506943) bank39760 +39761 POINT(40.69468802226875 74.71702002428026) bank39761 +39762 POINT(40.83594137590019 73.4608430212991) bank39762 +39763 POINT(41.11756875577674 74.89686454851615) bank39763 +39764 POINT(40.98979626106524 73.15930569102717) bank39764 +39765 POINT(41.59294556075331 74.89331403040924) bank39765 +39766 POINT(41.027393038685574 73.66436428267743) bank39766 +39767 POINT(40.43059176368463 73.7028833638869) bank39767 +39768 POINT(39.903965170118724 73.48125536944076) bank39768 +39769 POINT(41.15899982074703 73.02518047073467) bank39769 +39770 POINT(40.682386491556464 73.81938338141993) bank39770 +39771 POINT(40.50173850474374 73.19877924025973) bank39771 +39772 POINT(41.21379346146515 74.80027018044322) bank39772 +39773 POINT(40.32893691976503 73.62737400366188) bank39773 +39774 POINT(39.98465906908806 74.39927671120223) bank39774 +39775 POINT(40.161928509050725 74.5055407123084) bank39775 +39776 POINT(40.21157732076801 74.47678901892704) bank39776 +39777 POINT(39.86204070443344 73.41592097717242) bank39777 +39778 POINT(40.43115603008471 73.29456380064326) bank39778 +39779 POINT(41.22289385021606 74.80926747960618) bank39779 +39780 POINT(40.08836152767121 73.33699033814698) bank39780 +39781 POINT(41.51115604144111 74.04958183532511) bank39781 +39782 POINT(40.09075007727179 73.21704443605469) bank39782 +39783 POINT(40.8266894539663 74.4187963008976) bank39783 +39784 POINT(39.745906334132336 74.18616023612958) bank39784 +39785 POINT(40.72891552230671 73.23628764273987) bank39785 +39786 POINT(40.31589235399025 73.71803571975589) bank39786 +39787 POINT(41.17587960340996 73.18345395408089) bank39787 +39788 POINT(40.08211542470957 74.39988288564605) bank39788 +39789 POINT(41.23286663553147 73.92114703605041) bank39789 +39790 POINT(41.17647412150611 73.4665362292624) bank39790 +39791 POINT(40.04457553833799 73.48441333811553) bank39791 +39792 POINT(40.29072534087596 73.7868829440693) bank39792 +39793 POINT(41.40183914804062 73.01415239307947) bank39793 +39794 POINT(40.37153660814016 73.23297661533752) bank39794 +39795 POINT(41.23061666695927 73.34884608647266) bank39795 +39796 POINT(41.169762988850174 74.80087607595439) bank39796 +39797 POINT(41.03301171031893 73.4526805337864) bank39797 +39798 POINT(40.32903261519957 73.64331681394464) bank39798 +39799 POINT(40.169354182251105 74.87410728115161) bank39799 +39800 POINT(40.62916524892486 73.32076022386046) bank39800 +39801 POINT(40.647003972830376 73.05244790568318) bank39801 +39802 POINT(41.444182813091565 73.34487903666691) bank39802 +39803 POINT(40.00588860814234 74.18897142215108) bank39803 +39804 POINT(41.2622830889295 74.8823927695741) bank39804 +39805 POINT(41.34742507489521 74.43951985473906) bank39805 +39806 POINT(40.5995409801812 74.06670526553548) bank39806 +39807 POINT(41.52345973576746 74.3393643748255) bank39807 +39808 POINT(39.95091387093959 73.8270832202557) bank39808 +39809 POINT(40.39563627442873 73.1996197756541) bank39809 +39810 POINT(41.41047159849423 74.25307268531544) bank39810 +39811 POINT(39.95707672417258 73.46497834405945) bank39811 +39812 POINT(39.92531243205799 74.84546353334007) bank39812 +39813 POINT(41.37050266875602 74.39583364165696) bank39813 +39814 POINT(41.554111005707405 73.10211990224312) bank39814 +39815 POINT(40.40120672234866 74.58856060248897) bank39815 +39816 POINT(40.223150138145215 73.04583179732224) bank39816 +39817 POINT(39.95357151852459 74.28034466066178) bank39817 +39818 POINT(39.89903362179305 73.44734460187343) bank39818 +39819 POINT(39.92837175786762 74.87528732379276) bank39819 +39820 POINT(41.0991397040189 73.62715368779583) bank39820 +39821 POINT(39.842579058153326 73.51676154941323) bank39821 +39822 POINT(40.196652208770864 73.0639552042057) bank39822 +39823 POINT(41.281934316116576 74.91785950734425) bank39823 +39824 POINT(39.82197607341975 73.75697341742067) bank39824 +39825 POINT(40.363051947541685 73.79625426077543) bank39825 +39826 POINT(40.739492877058844 73.18467261196717) bank39826 +39827 POINT(41.00090619785938 73.92927168905867) bank39827 +39828 POINT(40.85094563069507 75.0010964823322) bank39828 +39829 POINT(40.48181473938911 73.53419891783932) bank39829 +39830 POINT(40.33635368451433 73.14853844865563) bank39830 +39831 POINT(41.298042920305264 74.4539863902322) bank39831 +39832 POINT(40.241406775932035 73.08351842900365) bank39832 +39833 POINT(40.77959474825872 73.51701692019003) bank39833 +39834 POINT(41.02217244295938 74.66642604314231) bank39834 +39835 POINT(41.468781033989835 74.950342890074) bank39835 +39836 POINT(41.49774812144146 74.83097430983038) bank39836 +39837 POINT(40.44753584018686 74.67501020810677) bank39837 +39838 POINT(41.69831524505408 74.09560721785797) bank39838 +39839 POINT(40.12015707365469 73.2572813700438) bank39839 +39840 POINT(40.59601161074537 73.8916194316548) bank39840 +39841 POINT(40.64025678664863 73.99279324356776) bank39841 +39842 POINT(40.14144806693131 73.51714822034693) bank39842 +39843 POINT(40.38790110653669 73.63324741245914) bank39843 +39844 POINT(39.78776348165063 73.22169164978507) bank39844 +39845 POINT(41.31824026768754 73.38437575782758) bank39845 +39846 POINT(40.827578138531734 74.8513703730817) bank39846 +39847 POINT(40.44637515397687 73.83998139287628) bank39847 +39848 POINT(40.87337099726452 73.89124339721954) bank39848 +39849 POINT(41.381114298988926 73.37832492475073) bank39849 +39850 POINT(41.28664253815429 74.73611111063292) bank39850 +39851 POINT(40.55335477171875 74.45540075265879) bank39851 +39852 POINT(40.46630528870575 74.41291285587967) bank39852 +39853 POINT(40.84242574198026 73.49535281055574) bank39853 +39854 POINT(39.981964499042036 74.57641526061084) bank39854 +39855 POINT(41.63972353038433 73.29400261167909) bank39855 +39856 POINT(40.137466327994375 74.41377603092484) bank39856 +39857 POINT(40.09226031837346 74.75191119441241) bank39857 +39858 POINT(40.37803913926605 73.71783360753187) bank39858 +39859 POINT(41.31247181411222 74.11028063343801) bank39859 +39860 POINT(40.89843422606732 73.59873723012609) bank39860 +39861 POINT(40.64208506271208 74.14097314978677) bank39861 +39862 POINT(39.768270174830626 74.53309122460202) bank39862 +39863 POINT(39.73660007713201 73.62332888743263) bank39863 +39864 POINT(39.83854142198657 74.6092059143726) bank39864 +39865 POINT(39.77147360113933 73.7127772076297) bank39865 +39866 POINT(41.30604022559661 74.66027422163513) bank39866 +39867 POINT(40.18828252693562 74.85344889540089) bank39867 +39868 POINT(41.10752369325889 73.01129247844159) bank39868 +39869 POINT(41.204773311297316 74.04406591453956) bank39869 +39870 POINT(40.872520047223546 73.67371457265517) bank39870 +39871 POINT(39.749153404537964 73.35936068759804) bank39871 +39872 POINT(41.143209620942606 74.67626688319855) bank39872 +39873 POINT(41.23926909984491 73.92626728769046) bank39873 +39874 POINT(41.590108109888796 73.55684109881972) bank39874 +39875 POINT(41.211793844683626 74.36581167789929) bank39875 +39876 POINT(39.78391038281034 74.99165753262967) bank39876 +39877 POINT(40.02469342108105 74.58568899584385) bank39877 +39878 POINT(41.51954123103486 74.70608574782553) bank39878 +39879 POINT(40.865212209850455 73.74257648579932) bank39879 +39880 POINT(40.904689451485226 74.92601313589292) bank39880 +39881 POINT(41.56395421774239 73.18784525083444) bank39881 +39882 POINT(40.94687635290101 73.94427765536936) bank39882 +39883 POINT(40.744018878485306 74.3366051755114) bank39883 +39884 POINT(39.85051337507211 74.82339499251327) bank39884 +39885 POINT(41.70079205868374 73.34951106804878) bank39885 +39886 POINT(40.99299393628007 74.41367305583492) bank39886 +39887 POINT(40.536439841747374 73.1411670358122) bank39887 +39888 POINT(40.38135384008509 73.82500323902566) bank39888 +39889 POINT(41.12982562438324 73.84370163614422) bank39889 +39890 POINT(40.3546030727114 74.70875841282235) bank39890 +39891 POINT(40.928061134383626 74.2217732288791) bank39891 +39892 POINT(41.67689807221175 73.94892250649994) bank39892 +39893 POINT(41.41139793278939 74.12537291439357) bank39893 +39894 POINT(39.72660146254463 73.08709159904326) bank39894 +39895 POINT(41.342406198702385 74.53972873657041) bank39895 +39896 POINT(39.80567717569005 74.23343842205043) bank39896 +39897 POINT(40.79609220365942 73.66927050679331) bank39897 +39898 POINT(41.628724467859 74.57186551491303) bank39898 +39899 POINT(40.41810198544397 74.53336277768801) bank39899 +39900 POINT(40.135205500306256 73.78288354035944) bank39900 +39901 POINT(39.73161121268794 74.45008814360845) bank39901 +39902 POINT(41.0958428929509 74.79074915581882) bank39902 +39903 POINT(40.937483432240114 73.51257759562901) bank39903 +39904 POINT(40.690003276963 74.7233000026699) bank39904 +39905 POINT(40.620880159999686 74.14511965746085) bank39905 +39906 POINT(40.123475194383005 73.06821573041525) bank39906 +39907 POINT(39.728564262781475 73.03873516399712) bank39907 +39908 POINT(40.84246126313417 73.73197267010268) bank39908 +39909 POINT(40.09568949916288 73.43497317659764) bank39909 +39910 POINT(40.39166964939599 73.1094044505083) bank39910 +39911 POINT(40.74648188635453 73.57541981094207) bank39911 +39912 POINT(40.90138793201844 73.63320236018517) bank39912 +39913 POINT(41.364392363517226 73.5094993741441) bank39913 +39914 POINT(41.03670736908487 74.08433733336874) bank39914 +39915 POINT(41.40485670763409 74.28560289014963) bank39915 +39916 POINT(41.12146828284819 74.2233251877272) bank39916 +39917 POINT(39.99321849729568 74.86887254855236) bank39917 +39918 POINT(39.990271362744714 73.83779412863957) bank39918 +39919 POINT(41.40786159975758 74.67973536817689) bank39919 +39920 POINT(39.75580168608049 73.13967188951116) bank39920 +39921 POINT(41.02523033376324 73.93707492874795) bank39921 +39922 POINT(40.85243286085164 73.72398305411234) bank39922 +39923 POINT(40.84429872411451 73.66406578580208) bank39923 +39924 POINT(41.17527460970522 74.81525691125353) bank39924 +39925 POINT(40.07930830257204 74.31608326812479) bank39925 +39926 POINT(40.44069237850409 74.18695233361385) bank39926 +39927 POINT(39.768288170578714 74.89797820293609) bank39927 +39928 POINT(39.82931408950806 73.42943424894432) bank39928 +39929 POINT(41.01499213483253 73.18352240816436) bank39929 +39930 POINT(41.10338759018149 74.10346647894978) bank39930 +39931 POINT(40.163356277930525 73.44646179242882) bank39931 +39932 POINT(40.73556952112141 74.77722169273375) bank39932 +39933 POINT(40.33071366283834 74.43310399809017) bank39933 +39934 POINT(40.55823443766998 74.89170767756315) bank39934 +39935 POINT(41.53144041680058 74.41753071221383) bank39935 +39936 POINT(41.181009604540584 73.21848054385926) bank39936 +39937 POINT(40.378597763165246 73.31356738490791) bank39937 +39938 POINT(41.204692190723364 74.24570625531256) bank39938 +39939 POINT(40.15056889986869 74.0793155153916) bank39939 +39940 POINT(41.18401364638455 74.95584024902297) bank39940 +39941 POINT(41.2156740764174 73.72934545223077) bank39941 +39942 POINT(40.04694082338547 73.95089406570666) bank39942 +39943 POINT(41.109001812797956 74.94776357106069) bank39943 +39944 POINT(40.50497488253994 73.01399487457046) bank39944 +39945 POINT(41.31472216107822 74.89017532790257) bank39945 +39946 POINT(39.97308456682455 74.29422195709756) bank39946 +39947 POINT(40.78729126819765 74.7428824579586) bank39947 +39948 POINT(39.77572641004342 73.7917214909595) bank39948 +39949 POINT(40.888239829194745 73.8244550742489) bank39949 +39950 POINT(41.104404674092436 73.78795596687884) bank39950 +39951 POINT(41.22147749584074 73.26776375159264) bank39951 +39952 POINT(41.346419003576536 74.81343831997684) bank39952 +39953 POINT(40.28349572207166 73.5301263772568) bank39953 +39954 POINT(40.52184768152464 73.14210256136776) bank39954 +39955 POINT(41.22217156843171 73.21263949265612) bank39955 +39956 POINT(40.742741580994384 74.72375757056658) bank39956 +39957 POINT(40.951167247764666 74.49552023849265) bank39957 +39958 POINT(40.63357558990162 74.3926015929076) bank39958 +39959 POINT(39.872437164232764 73.56074269681315) bank39959 +39960 POINT(40.29730561893964 74.89923442158555) bank39960 +39961 POINT(39.87699831862047 74.47479614501664) bank39961 +39962 POINT(41.4611074451283 74.19418541284975) bank39962 +39963 POINT(40.91472116667314 73.84239804026484) bank39963 +39964 POINT(39.77587961752203 73.5860257756294) bank39964 +39965 POINT(41.653825626824485 73.18431562701541) bank39965 +39966 POINT(39.73633613513569 74.09933841697955) bank39966 +39967 POINT(39.945035554677816 74.31644736104585) bank39967 +39968 POINT(41.163201625170196 74.08221227352628) bank39968 +39969 POINT(40.574762026456554 74.35837896519777) bank39969 +39970 POINT(41.288891285705624 74.6160680636932) bank39970 +39971 POINT(41.03878308158382 74.62672330318239) bank39971 +39972 POINT(41.04231100789311 73.36763263188934) bank39972 +39973 POINT(40.73813144229915 73.70999867559719) bank39973 +39974 POINT(39.830497535390414 74.02374118082106) bank39974 +39975 POINT(40.51018799117077 73.8792433446942) bank39975 +39976 POINT(41.22340490252319 73.94014021295665) bank39976 +39977 POINT(40.92848405727673 73.03410346741529) bank39977 +39978 POINT(40.106792573891006 74.81904887492233) bank39978 +39979 POINT(39.80697889795075 74.4232428303999) bank39979 +39980 POINT(40.65154502475775 73.62662049988549) bank39980 +39981 POINT(41.58763236422166 74.07412234363248) bank39981 +39982 POINT(40.72438918907115 73.79531076066627) bank39982 +39983 POINT(41.19105024979784 74.41991500940514) bank39983 +39984 POINT(40.464407221258924 73.97248861253858) bank39984 +39985 POINT(39.991156865468525 73.13598088450819) bank39985 +39986 POINT(41.079545917426245 74.4475673986184) bank39986 +39987 POINT(40.949545558427374 74.76252759109157) bank39987 +39988 POINT(41.20122129249178 73.5006025513171) bank39988 +39989 POINT(40.072100863826286 74.72519917047572) bank39989 +39990 POINT(41.67040079079469 73.5106307920775) bank39990 +39991 POINT(41.6496635554995 73.59587156947407) bank39991 +39992 POINT(40.61475638036093 73.3895734899812) bank39992 +39993 POINT(40.61865048982117 73.86774009560415) bank39993 +39994 POINT(41.33474546433903 73.0922407675024) bank39994 +39995 POINT(41.137790326726964 73.48192848461865) bank39995 +39996 POINT(40.548808924373915 73.90712677577739) bank39996 +39997 POINT(41.447533039550756 73.2519285901726) bank39997 +39998 POINT(40.137602615566024 74.97001283492936) bank39998 +39999 POINT(40.02195816349543 74.00888689207422) bank39999 +40000 POINT(40.9110065676383 74.37488995145509) bank40000 +40001 POINT(41.01654170781009 74.90470543514913) bank40001 +40002 POINT(39.785550173658415 73.78823682844455) bank40002 +40003 POINT(40.2804277205386 74.8496388627235) bank40003 +40004 POINT(41.54355585536162 73.9454032210093) bank40004 +40005 POINT(39.89564734152542 74.77237660964732) bank40005 +40006 POINT(40.35860654934136 73.88914031434615) bank40006 +40007 POINT(40.82520263608486 73.88536764562645) bank40007 +40008 POINT(40.52376655383046 73.08940270621353) bank40008 +40009 POINT(41.10266285643573 73.41452902705595) bank40009 +40010 POINT(39.964421239809276 74.04332320258132) bank40010 +40011 POINT(41.24739495292047 73.51557946790082) bank40011 +40012 POINT(40.44550614851028 74.09043590127955) bank40012 +40013 POINT(41.228339619736474 74.21338394674387) bank40013 +40014 POINT(41.1274253130187 74.35842907821291) bank40014 +40015 POINT(41.363632769464274 73.16745523346204) bank40015 +40016 POINT(40.79710046661698 73.444134108116) bank40016 +40017 POINT(41.30580760336353 73.40725664798904) bank40017 +40018 POINT(41.11926900042224 74.12684718320331) bank40018 +40019 POINT(41.34775067414595 73.257757911338) bank40019 +40020 POINT(41.02192640681257 73.70246129667818) bank40020 +40021 POINT(40.76657407697405 74.29693600059316) bank40021 +40022 POINT(40.88139900217078 73.14574337685704) bank40022 +40023 POINT(40.182176502641276 73.14654380692237) bank40023 +40024 POINT(40.01528490625128 73.95409632046174) bank40024 +40025 POINT(39.95965724303357 74.40969836703697) bank40025 +40026 POINT(40.10932318447062 73.7064215332237) bank40026 +40027 POINT(39.90370656720651 73.93267777438032) bank40027 +40028 POINT(40.73030859605268 73.97103094601803) bank40028 +40029 POINT(40.92288785513331 73.1755220108153) bank40029 +40030 POINT(40.48472855128828 74.36119449479607) bank40030 +40031 POINT(39.77354908209702 74.67857668836754) bank40031 +40032 POINT(40.72398462163503 74.17187342781455) bank40032 +40033 POINT(39.918265630212055 73.93952512044336) bank40033 +40034 POINT(41.228515295942024 73.41077246659098) bank40034 +40035 POINT(40.83426998239081 74.96608776056752) bank40035 +40036 POINT(41.2286348169235 74.29138777789358) bank40036 +40037 POINT(40.71367876320424 73.08285412127124) bank40037 +40038 POINT(41.25986149588352 73.3410038675108) bank40038 +40039 POINT(41.47888971211654 74.28871704619311) bank40039 +40040 POINT(40.93188437351156 74.58926146811362) bank40040 +40041 POINT(41.68213156409718 74.89393989727786) bank40041 +40042 POINT(40.28999244099424 74.21226981912424) bank40042 +40043 POINT(41.2197113961484 74.8702642559472) bank40043 +40044 POINT(41.10926772120117 73.4834058781865) bank40044 +40045 POINT(41.69499974453602 74.01234280445208) bank40045 +40046 POINT(40.77216989717328 73.10885421130861) bank40046 +40047 POINT(40.16512410514361 74.15447445610185) bank40047 +40048 POINT(40.39703246335816 74.38185064743583) bank40048 +40049 POINT(41.15063449405267 73.72282116357817) bank40049 +40050 POINT(40.25342822330942 73.62172358714243) bank40050 +40051 POINT(40.732222560605194 74.50789525846203) bank40051 +40052 POINT(40.06952722533758 74.74695594248651) bank40052 +40053 POINT(40.35317683425009 74.41925778767398) bank40053 +40054 POINT(41.44214840699004 74.68959987383724) bank40054 +40055 POINT(40.98645028889224 73.93903493302443) bank40055 +40056 POINT(41.165335006242294 74.2608847196964) bank40056 +40057 POINT(40.38329462887699 73.45070706681751) bank40057 +40058 POINT(41.18270414083429 73.68107218592547) bank40058 +40059 POINT(39.831854175231776 73.84775592137565) bank40059 +40060 POINT(40.99174730422339 73.45085631066881) bank40060 +40061 POINT(40.157020548086585 74.47681167175082) bank40061 +40062 POINT(40.13149697084876 74.16088719986266) bank40062 +40063 POINT(40.65649731589762 73.51008599973888) bank40063 +40064 POINT(39.726501224735586 74.50466739741056) bank40064 +40065 POINT(41.62281959443232 73.97746423361357) bank40065 +40066 POINT(40.35299913085638 73.85587854447546) bank40066 +40067 POINT(41.127622670863886 73.139723265673) bank40067 +40068 POINT(41.39247855395996 73.7555489015985) bank40068 +40069 POINT(40.57755428670984 73.85689304573368) bank40069 +40070 POINT(41.12454618530734 73.13854655074299) bank40070 +40071 POINT(40.818400746746406 73.59796690472582) bank40071 +40072 POINT(41.44862447783398 73.27213876081089) bank40072 +40073 POINT(40.61054864052637 74.75385355807474) bank40073 +40074 POINT(41.62198790058362 73.18627306734149) bank40074 +40075 POINT(40.8673000075322 73.3786899506007) bank40075 +40076 POINT(40.162604548981975 73.28905935619922) bank40076 +40077 POINT(39.85446834074118 73.48680659415272) bank40077 +40078 POINT(40.749984780120066 73.18792360218846) bank40078 +40079 POINT(41.63857945857882 73.94293930867914) bank40079 +40080 POINT(41.65559805222204 73.54204602205958) bank40080 +40081 POINT(40.67099527403888 74.05882177623323) bank40081 +40082 POINT(40.676316370759345 74.14452299618087) bank40082 +40083 POINT(40.21383893135817 74.38225696161133) bank40083 +40084 POINT(41.53740303453899 73.27562639477541) bank40084 +40085 POINT(41.018678590604296 74.48247578991806) bank40085 +40086 POINT(40.59980456586675 73.33077292985276) bank40086 +40087 POINT(39.81856815590774 74.98393535356946) bank40087 +40088 POINT(40.36590870967563 74.3826306098629) bank40088 +40089 POINT(40.74021496857893 73.37541422859917) bank40089 +40090 POINT(40.350499095684256 74.20600368233372) bank40090 +40091 POINT(40.432251270573715 74.55664901137446) bank40091 +40092 POINT(41.082325024187874 73.02963212634899) bank40092 +40093 POINT(41.477523912667465 73.55491182525887) bank40093 +40094 POINT(40.20269366663929 74.17404914583142) bank40094 +40095 POINT(40.70993680138575 74.95164773313759) bank40095 +40096 POINT(41.45576053829641 74.97038047451616) bank40096 +40097 POINT(41.166382263392315 73.66593235535794) bank40097 +40098 POINT(40.9129158153122 73.02996118709672) bank40098 +40099 POINT(40.11439858830807 74.05188244457115) bank40099 +40100 POINT(40.624087128932956 74.38392397301239) bank40100 +40101 POINT(39.96157002038876 73.47938419141745) bank40101 +40102 POINT(41.55331158147696 73.53601332119968) bank40102 +40103 POINT(40.81602008632005 74.31115265110508) bank40103 +40104 POINT(40.8189594292747 73.58298163246604) bank40104 +40105 POINT(41.46444103169511 73.90049606474864) bank40105 +40106 POINT(41.09362874192762 73.44955160808222) bank40106 +40107 POINT(40.20165028446377 74.06807576169848) bank40107 +40108 POINT(41.287730669148395 74.29850240798508) bank40108 +40109 POINT(41.42735921297894 74.38940221256945) bank40109 +40110 POINT(39.7841342628374 74.35128342956101) bank40110 +40111 POINT(40.48683514260213 74.74718208246902) bank40111 +40112 POINT(39.98665626550557 74.39760814968525) bank40112 +40113 POINT(40.97238313698299 73.79845006507476) bank40113 +40114 POINT(40.60025556197011 73.16960495334432) bank40114 +40115 POINT(41.12205366828147 73.12426825903025) bank40115 +40116 POINT(39.90367779066902 73.07850467372808) bank40116 +40117 POINT(39.93085175637015 74.89729679189333) bank40117 +40118 POINT(41.38179627561833 74.68526810540622) bank40118 +40119 POINT(41.55349970381884 73.39376842715048) bank40119 +40120 POINT(39.771309242173196 73.08263131124549) bank40120 +40121 POINT(39.964256811004304 74.01789897882034) bank40121 +40122 POINT(41.27485013903441 74.24326751342994) bank40122 +40123 POINT(41.47117547476505 73.91215775693952) bank40123 +40124 POINT(41.22827239639843 73.33021336837642) bank40124 +40125 POINT(41.31623291788701 73.55483795415012) bank40125 +40126 POINT(40.975073278323265 74.96223345693481) bank40126 +40127 POINT(41.468629583348296 73.50784036119117) bank40127 +40128 POINT(40.63006019327503 74.29234775078827) bank40128 +40129 POINT(40.22740065367941 74.37040246978061) bank40129 +40130 POINT(40.21221024954604 74.39050068891596) bank40130 +40131 POINT(41.006047667316665 73.52517249459567) bank40131 +40132 POINT(40.63583850789308 74.91956477186709) bank40132 +40133 POINT(40.13758928006443 73.03538831663045) bank40133 +40134 POINT(40.901590438172846 73.06268625053605) bank40134 +40135 POINT(40.02787757507349 74.90107930572206) bank40135 +40136 POINT(40.51018284846368 73.67411497406076) bank40136 +40137 POINT(40.32004248798137 73.18464662965863) bank40137 +40138 POINT(40.66057702192312 73.65536607578306) bank40138 +40139 POINT(40.90232397847602 73.83796093232678) bank40139 +40140 POINT(40.2976077230162 73.61891912212913) bank40140 +40141 POINT(40.45412958508413 74.57654486022227) bank40141 +40142 POINT(39.80175968655132 74.6933386736423) bank40142 +40143 POINT(39.9641540080287 74.51688481834158) bank40143 +40144 POINT(40.091071723007005 73.07024795851086) bank40144 +40145 POINT(40.68854685877428 74.65215931159175) bank40145 +40146 POINT(39.72801496433006 73.30797387780723) bank40146 +40147 POINT(40.58549732976652 73.54756701157133) bank40147 +40148 POINT(41.626390926684344 73.6958365901939) bank40148 +40149 POINT(40.130319115648064 74.41767916370937) bank40149 +40150 POINT(39.913651105584634 74.37398466965726) bank40150 +40151 POINT(39.94269299302186 73.532008224204) bank40151 +40152 POINT(40.85375816956696 74.11551440519614) bank40152 +40153 POINT(41.641750029598626 74.86815376571668) bank40153 +40154 POINT(40.14127910850821 73.25069136489188) bank40154 +40155 POINT(40.87805822102748 73.07373307387681) bank40155 +40156 POINT(40.201205221410305 74.23539755694463) bank40156 +40157 POINT(40.26958499453323 74.17940628729896) bank40157 +40158 POINT(40.73806976147821 74.3602190264141) bank40158 +40159 POINT(41.65661612073266 73.7811199879399) bank40159 +40160 POINT(40.83256819699551 74.99935186579401) bank40160 +40161 POINT(41.31345350995155 73.50791030720036) bank40161 +40162 POINT(41.20610303899411 74.23803120208187) bank40162 +40163 POINT(41.15907429632438 74.6973163720917) bank40163 +40164 POINT(40.92514457349426 73.06244072825183) bank40164 +40165 POINT(40.08895737342435 74.19925950424799) bank40165 +40166 POINT(41.279904771372664 73.47368273814526) bank40166 +40167 POINT(41.65632508475271 73.08261984416323) bank40167 +40168 POINT(41.231184023760115 73.75865516537532) bank40168 +40169 POINT(40.456663641318926 73.40311697623356) bank40169 +40170 POINT(41.39389401869374 74.66181238475987) bank40170 +40171 POINT(40.29219813388745 73.54182883811693) bank40171 +40172 POINT(41.52284392859375 73.26156402686146) bank40172 +40173 POINT(40.75797799003016 73.91673875086302) bank40173 +40174 POINT(41.656779725289006 74.95180676970203) bank40174 +40175 POINT(40.16232561041456 73.80863066031122) bank40175 +40176 POINT(40.965550736165014 74.52219968792508) bank40176 +40177 POINT(40.11033776943602 73.04217077097479) bank40177 +40178 POINT(41.03270227642981 73.35226919499809) bank40178 +40179 POINT(40.3396734593911 74.79257967971535) bank40179 +40180 POINT(40.07422911881427 74.23916336641237) bank40180 +40181 POINT(41.505522346974516 73.92177832705016) bank40181 +40182 POINT(40.683443076448015 74.91842153566368) bank40182 +40183 POINT(40.18138044336119 74.0944160674805) bank40183 +40184 POINT(40.643998442778916 74.34119988025748) bank40184 +40185 POINT(40.93822199771267 73.56769858264454) bank40185 +40186 POINT(41.34626435728205 74.27927161444433) bank40186 +40187 POINT(41.01435581940004 73.96399968791115) bank40187 +40188 POINT(41.519594757372545 74.94661051514024) bank40188 +40189 POINT(41.54944398227442 73.90312344819039) bank40189 +40190 POINT(39.789218654528796 74.09269516785108) bank40190 +40191 POINT(41.60028114921236 73.69075181896265) bank40191 +40192 POINT(41.382563813999155 74.77368861047857) bank40192 +40193 POINT(39.97584149610169 73.2046663773727) bank40193 +40194 POINT(40.89445171325392 74.3906658755466) bank40194 +40195 POINT(40.72720626176733 74.20604368604454) bank40195 +40196 POINT(39.7180685623177 74.22969499216325) bank40196 +40197 POINT(39.803711980273754 74.20679965713853) bank40197 +40198 POINT(40.711324939218244 73.18152885460822) bank40198 +40199 POINT(40.47077437028499 74.44808290137502) bank40199 +40200 POINT(40.65725782684259 74.7864325736784) bank40200 +40201 POINT(39.797467554922555 73.54682434909022) bank40201 +40202 POINT(41.70111977484594 74.06793969921809) bank40202 +40203 POINT(39.99208841180993 74.67844241550019) bank40203 +40204 POINT(41.38265406034116 74.18788336144944) bank40204 +40205 POINT(40.57135668970004 74.98779047794467) bank40205 +40206 POINT(40.51657908113609 73.11992019000218) bank40206 +40207 POINT(39.94098504887935 74.08412142460125) bank40207 +40208 POINT(41.39953996752741 74.95555657885336) bank40208 +40209 POINT(41.41802009687314 74.2494498373162) bank40209 +40210 POINT(40.19407699145352 73.37332019935408) bank40210 +40211 POINT(40.796836838838914 73.09709503719436) bank40211 +40212 POINT(40.19610730460889 74.37608413778014) bank40212 +40213 POINT(39.87798964670501 73.24563755499278) bank40213 +40214 POINT(41.526084442695854 73.75708317793345) bank40214 +40215 POINT(40.29283914356266 73.20555264712603) bank40215 +40216 POINT(40.07814630453871 73.9393673859904) bank40216 +40217 POINT(40.09774776865202 73.8336238992104) bank40217 +40218 POINT(40.12527379626473 73.56040855455122) bank40218 +40219 POINT(40.83433912161883 73.11969886951135) bank40219 +40220 POINT(40.358215639366804 73.24159739676892) bank40220 +40221 POINT(40.18722983127491 73.61119114949997) bank40221 +40222 POINT(40.757527931877505 74.3233476723462) bank40222 +40223 POINT(40.53453181818305 73.27138438955569) bank40223 +40224 POINT(40.190929222824046 73.26653193619032) bank40224 +40225 POINT(40.77452879595755 74.59828462323767) bank40225 +40226 POINT(41.44114057563035 73.74073131576313) bank40226 +40227 POINT(39.78154221637589 73.9958660648084) bank40227 +40228 POINT(39.765468048472954 74.63014741398113) bank40228 +40229 POINT(41.55550105112159 74.58185249375848) bank40229 +40230 POINT(40.73747959697153 73.3217235084949) bank40230 +40231 POINT(41.29724727180613 73.39515512935377) bank40231 +40232 POINT(39.75454009664352 74.82556336960916) bank40232 +40233 POINT(39.76453480430794 74.82571889319247) bank40233 +40234 POINT(41.41730721758351 74.51025693408714) bank40234 +40235 POINT(40.99636845522813 74.73414329262746) bank40235 +40236 POINT(40.62283625730968 73.72429072240521) bank40236 +40237 POINT(40.784905329289295 73.64131275152313) bank40237 +40238 POINT(40.094910773061194 74.53869333386207) bank40238 +40239 POINT(40.1535885385477 73.06158718730354) bank40239 +40240 POINT(40.62523704689151 74.59681614570275) bank40240 +40241 POINT(39.78029824911279 74.16016190851246) bank40241 +40242 POINT(41.28209600301594 73.35643485318977) bank40242 +40243 POINT(41.27262798831689 74.31262109396346) bank40243 +40244 POINT(40.103737412053704 73.66167008720053) bank40244 +40245 POINT(40.98659924835109 73.75656842349171) bank40245 +40246 POINT(39.95740319840745 73.85069441388778) bank40246 +40247 POINT(40.35828901679738 74.14121633786237) bank40247 +40248 POINT(40.01278296837953 74.00835386999124) bank40248 +40249 POINT(40.11248164514737 73.23996778321013) bank40249 +40250 POINT(39.81907646060475 73.8722334035665) bank40250 +40251 POINT(40.80657427304892 73.3449327875504) bank40251 +40252 POINT(40.40279876881235 73.54869266728772) bank40252 +40253 POINT(41.47499619153555 73.12733672489794) bank40253 +40254 POINT(39.82069987942642 73.20982256885617) bank40254 +40255 POINT(40.01266645280788 73.32131474600969) bank40255 +40256 POINT(41.46182472518599 74.78154515851635) bank40256 +40257 POINT(40.14898683118903 73.4820552712468) bank40257 +40258 POINT(41.26507432700111 74.50963414636398) bank40258 +40259 POINT(40.07160818315107 74.99931779712355) bank40259 +40260 POINT(40.6215415929227 73.86993709458542) bank40260 +40261 POINT(41.601743577643134 73.09481972004251) bank40261 +40262 POINT(41.31779459941152 73.23467125880688) bank40262 +40263 POINT(40.33340679425671 73.29744110291517) bank40263 +40264 POINT(41.489976400659046 73.15021901239967) bank40264 +40265 POINT(41.417350327219125 74.16810325936038) bank40265 +40266 POINT(41.185730233719745 74.21164488277995) bank40266 +40267 POINT(40.28085825484824 74.54095813426903) bank40267 +40268 POINT(41.36421906832545 74.00180999413514) bank40268 +40269 POINT(39.88272255706572 74.36021981235295) bank40269 +40270 POINT(39.79988987944783 73.08010730388142) bank40270 +40271 POINT(41.4997045584565 73.4189154456323) bank40271 +40272 POINT(41.104299407562465 74.5230949509759) bank40272 +40273 POINT(41.65873596822253 74.58868392461699) bank40273 +40274 POINT(40.26241776342324 73.06356001399438) bank40274 +40275 POINT(39.74155630349066 74.14061842333092) bank40275 +40276 POINT(40.27582771105589 74.08125617792591) bank40276 +40277 POINT(40.98145578858915 73.19393437846192) bank40277 +40278 POINT(41.05753118932627 74.34557371824967) bank40278 +40279 POINT(40.420875210881746 73.91788477745308) bank40279 +40280 POINT(41.04055572540798 73.22857940572432) bank40280 +40281 POINT(41.04381401859939 74.25022439052519) bank40281 +40282 POINT(41.16626087031785 74.32654553390479) bank40282 +40283 POINT(41.15758386171059 73.53595203245443) bank40283 +40284 POINT(40.73829156762811 74.48354230737372) bank40284 +40285 POINT(40.24458115204887 73.08495884218246) bank40285 +40286 POINT(40.18170466222465 73.44867108200116) bank40286 +40287 POINT(39.83670367052936 74.36415871826564) bank40287 +40288 POINT(41.44856446821686 74.21616243102861) bank40288 +40289 POINT(40.9975530264477 74.0216311660196) bank40289 +40290 POINT(41.59702391887356 73.48345085218622) bank40290 +40291 POINT(39.73091770584254 73.20178846675613) bank40291 +40292 POINT(40.30803163550969 73.24617536804496) bank40292 +40293 POINT(41.07086791855145 74.01083297229509) bank40293 +40294 POINT(40.23584476008964 74.29727210783528) bank40294 +40295 POINT(39.72688230100507 74.43263749429485) bank40295 +40296 POINT(39.95835037152299 74.78020246152582) bank40296 +40297 POINT(40.790466170089665 73.9388359026563) bank40297 +40298 POINT(40.8675038700947 74.98537294152996) bank40298 +40299 POINT(40.453495288844266 74.83965315030521) bank40299 +40300 POINT(40.79258144562088 73.4088238477785) bank40300 +40301 POINT(41.30257451812005 74.34202847824172) bank40301 +40302 POINT(39.78522364506041 74.02797957304915) bank40302 +40303 POINT(40.690456150508 73.29248740557205) bank40303 +40304 POINT(39.810549729160215 73.78148160501692) bank40304 +40305 POINT(39.93093848176342 74.62221649644621) bank40305 +40306 POINT(40.41938960422439 73.82548216323313) bank40306 +40307 POINT(40.07780252299071 73.89452593520488) bank40307 +40308 POINT(39.96715756508646 74.09604724132386) bank40308 +40309 POINT(40.551297085280666 74.48591563651016) bank40309 +40310 POINT(41.58178617703333 73.51611137448961) bank40310 +40311 POINT(40.06916456270357 73.36362973434818) bank40311 +40312 POINT(40.48846886990689 73.83219844704469) bank40312 +40313 POINT(41.450276258651016 74.3722770445806) bank40313 +40314 POINT(39.883471314523256 74.40650590157215) bank40314 +40315 POINT(41.12142339858536 74.14807027521465) bank40315 +40316 POINT(40.80480096744643 74.13172427429241) bank40316 +40317 POINT(40.69386682797357 74.8825529371861) bank40317 +40318 POINT(40.43786332714232 74.22497150314376) bank40318 +40319 POINT(40.300323343767666 74.30540030718545) bank40319 +40320 POINT(40.3939317778419 74.00216101418899) bank40320 +40321 POINT(41.34259194350815 74.49047529015304) bank40321 +40322 POINT(41.39696644421654 73.8557676070352) bank40322 +40323 POINT(40.48713151086561 74.13403885995005) bank40323 +40324 POINT(40.82434513410714 73.03481776052514) bank40324 +40325 POINT(41.067376468635544 73.8501035285424) bank40325 +40326 POINT(41.18265437839964 73.17805661250692) bank40326 +40327 POINT(41.514510259746196 73.23124300633044) bank40327 +40328 POINT(41.52871701111923 73.97558301079351) bank40328 +40329 POINT(41.19778535781461 74.0611481590689) bank40329 +40330 POINT(40.378666981859446 73.66065393561182) bank40330 +40331 POINT(40.3927466910851 73.90604615643656) bank40331 +40332 POINT(41.52725624538321 74.92874709011005) bank40332 +40333 POINT(39.79247232776972 74.39944661754166) bank40333 +40334 POINT(39.94431010891775 73.04091055086553) bank40334 +40335 POINT(40.67845706390197 73.81662149794516) bank40335 +40336 POINT(39.9114617982784 73.90146400040311) bank40336 +40337 POINT(41.674438136340605 74.27641499690728) bank40337 +40338 POINT(40.67592868239051 74.03382516167726) bank40338 +40339 POINT(40.370065073400724 74.16946648821668) bank40339 +40340 POINT(40.23754830674799 73.65425052609179) bank40340 +40341 POINT(41.303549675091695 73.46261555210899) bank40341 +40342 POINT(40.57861202784959 73.26799882826944) bank40342 +40343 POINT(40.77983175802522 74.28745542344492) bank40343 +40344 POINT(41.34556821444193 73.89550109476473) bank40344 +40345 POINT(40.85203999886807 74.70845303567545) bank40345 +40346 POINT(40.76306289101716 73.90106262499143) bank40346 +40347 POINT(40.2463136075136 74.44009709102693) bank40347 +40348 POINT(40.01606511340482 73.89383649381651) bank40348 +40349 POINT(41.35081107786437 74.02706192043107) bank40349 +40350 POINT(41.66735430168197 74.80865566107168) bank40350 +40351 POINT(40.29535737544728 73.97449044849493) bank40351 +40352 POINT(40.28874701804667 73.48405968452295) bank40352 +40353 POINT(41.53279224862709 73.3482899598098) bank40353 +40354 POINT(40.671553179145874 74.8587214415516) bank40354 +40355 POINT(40.70178785708957 73.51160875393359) bank40355 +40356 POINT(41.55983220000831 74.28483283369732) bank40356 +40357 POINT(40.21918001552589 74.37011380373903) bank40357 +40358 POINT(40.51665426423166 74.05572643159915) bank40358 +40359 POINT(40.064459455496596 73.65617154084035) bank40359 +40360 POINT(41.62833202757679 74.34769730186373) bank40360 +40361 POINT(41.66759435815867 73.29260629011871) bank40361 +40362 POINT(39.89783059650622 73.91755284549161) bank40362 +40363 POINT(40.767521446326086 73.42446415679281) bank40363 +40364 POINT(39.90453649631575 74.3549987428837) bank40364 +40365 POINT(40.687020984527436 73.82010798081437) bank40365 +40366 POINT(40.563756705884586 74.46800033180322) bank40366 +40367 POINT(41.03389481163817 74.19820378637398) bank40367 +40368 POINT(40.449563408046004 73.54155870088161) bank40368 +40369 POINT(40.87060838993878 73.35841064617534) bank40369 +40370 POINT(41.54641796377983 74.30635585821754) bank40370 +40371 POINT(40.02991691679752 74.06310171266794) bank40371 +40372 POINT(40.45452195216465 74.2729465455827) bank40372 +40373 POINT(40.59004637153207 74.23028868065641) bank40373 +40374 POINT(40.76489589229344 74.01274988813016) bank40374 +40375 POINT(40.51384979126838 73.74288431915718) bank40375 +40376 POINT(41.42522162756721 73.79778400080637) bank40376 +40377 POINT(40.273027041354986 73.38566702720908) bank40377 +40378 POINT(41.56318887599937 74.5307438076322) bank40378 +40379 POINT(40.911231776197205 73.9080740751242) bank40379 +40380 POINT(40.3113786404695 74.34896708858575) bank40380 +40381 POINT(39.85970967954133 73.08153319612659) bank40381 +40382 POINT(41.34511871418066 73.45612249605611) bank40382 +40383 POINT(41.429359336436235 74.09138019729676) bank40383 +40384 POINT(40.483176990047674 73.70012180466458) bank40384 +40385 POINT(41.6867455803113 73.00639987174657) bank40385 +40386 POINT(40.68196422429442 74.03843392208178) bank40386 +40387 POINT(39.86702889591672 73.99900725313024) bank40387 +40388 POINT(40.42330861809371 73.34358717099157) bank40388 +40389 POINT(39.92049615950549 73.41161854776185) bank40389 +40390 POINT(40.45146441121057 74.3933289151385) bank40390 +40391 POINT(40.944999427405314 74.73989022654222) bank40391 +40392 POINT(41.20787543502489 73.6693187013079) bank40392 +40393 POINT(41.08524877122727 73.86879296667844) bank40393 +40394 POINT(41.24735520870385 73.88089680442583) bank40394 +40395 POINT(40.11360395695477 73.77387666197096) bank40395 +40396 POINT(40.27259364628048 73.41730436795991) bank40396 +40397 POINT(40.54644062465347 73.19112662702629) bank40397 +40398 POINT(40.63684069459908 73.64428628646182) bank40398 +40399 POINT(41.08873355896746 73.04562057237037) bank40399 +40400 POINT(39.96582601919327 73.45673878874389) bank40400 +40401 POINT(40.03032680200738 74.56149154485124) bank40401 +40402 POINT(40.21328920719069 74.77801701220876) bank40402 +40403 POINT(40.23142096239303 73.90494685019888) bank40403 +40404 POINT(39.93694680816573 73.85833159879336) bank40404 +40405 POINT(40.313642240626955 74.55159613254004) bank40405 +40406 POINT(40.09774011288752 74.03303646638166) bank40406 +40407 POINT(41.35290508371137 74.52324572043699) bank40407 +40408 POINT(40.88238814500951 73.12266959048463) bank40408 +40409 POINT(40.84929565183126 73.51733668597926) bank40409 +40410 POINT(40.442572300473124 74.79491099929251) bank40410 +40411 POINT(40.67234743944357 73.70429453649609) bank40411 +40412 POINT(40.04404154044852 74.17366568724542) bank40412 +40413 POINT(39.89989453104299 74.761805922609) bank40413 +40414 POINT(41.52791833907058 73.57194213768268) bank40414 +40415 POINT(40.165712897236325 73.97423202166054) bank40415 +40416 POINT(40.394384711952235 74.62584587314583) bank40416 +40417 POINT(39.73791265913809 73.82503573340604) bank40417 +40418 POINT(40.48683712832485 74.4909307427064) bank40418 +40419 POINT(39.93655666505807 73.28958610646373) bank40419 +40420 POINT(40.383117388418874 73.26233515332827) bank40420 +40421 POINT(40.89932441678241 74.82192287661772) bank40421 +40422 POINT(40.66170167382457 73.47511369282729) bank40422 +40423 POINT(41.38159836668577 74.85105680108693) bank40423 +40424 POINT(39.8019062587781 74.81890985456765) bank40424 +40425 POINT(40.4716958886996 74.29659238765917) bank40425 +40426 POINT(40.926849540736285 73.70495589502139) bank40426 +40427 POINT(40.85032459226593 73.3064058595556) bank40427 +40428 POINT(40.664498453312014 74.67410008532168) bank40428 +40429 POINT(40.623671220322926 74.0785860953602) bank40429 +40430 POINT(40.219257951584 74.2387180301432) bank40430 +40431 POINT(40.682069072975956 74.24457174011314) bank40431 +40432 POINT(39.72926883315928 74.25692202931188) bank40432 +40433 POINT(40.20910578018704 73.71970264987401) bank40433 +40434 POINT(39.75462036665828 74.97945069491327) bank40434 +40435 POINT(40.92160670003693 73.08749125497627) bank40435 +40436 POINT(41.33553881184652 74.196602142365) bank40436 +40437 POINT(41.38411180918015 74.40648286290818) bank40437 +40438 POINT(39.846082773703024 73.2382909615848) bank40438 +40439 POINT(40.70128814077684 74.93027844290879) bank40439 +40440 POINT(40.37391709084051 73.23067998845313) bank40440 +40441 POINT(40.71031270147855 73.95967066285631) bank40441 +40442 POINT(39.848102737478584 73.6202774002559) bank40442 +40443 POINT(41.48069448128168 73.67510428504572) bank40443 +40444 POINT(39.720286457990966 74.25125049056695) bank40444 +40445 POINT(40.41213492975193 74.05547209484352) bank40445 +40446 POINT(41.349214647343736 73.59324196461404) bank40446 +40447 POINT(41.326671300857235 74.91327318484178) bank40447 +40448 POINT(40.277407363369335 73.79237579081443) bank40448 +40449 POINT(40.77268011570416 74.51002799517376) bank40449 +40450 POINT(40.76649562510004 74.34175279495113) bank40450 +40451 POINT(41.592785654418066 74.56040086686113) bank40451 +40452 POINT(40.26963526602584 73.70418711165759) bank40452 +40453 POINT(40.69032848998139 73.27290645809667) bank40453 +40454 POINT(40.02034656104382 74.809595029633) bank40454 +40455 POINT(41.14459313277382 74.8015655370868) bank40455 +40456 POINT(41.39366488023026 73.19979758574758) bank40456 +40457 POINT(40.46170965190917 74.01274163124575) bank40457 +40458 POINT(40.58548785785212 73.65360164208921) bank40458 +40459 POINT(39.91414789131631 74.51751253064045) bank40459 +40460 POINT(40.33125508156525 74.40304899845307) bank40460 +40461 POINT(39.91394859172542 74.47586498271944) bank40461 +40462 POINT(40.83424807500124 73.33469619372461) bank40462 +40463 POINT(41.52223167195755 73.12939029724282) bank40463 +40464 POINT(41.26346220051327 74.55282638972679) bank40464 +40465 POINT(40.12086357962 74.81859244649421) bank40465 +40466 POINT(40.383089715727465 74.94607463522304) bank40466 +40467 POINT(41.4334594150385 73.51642585409871) bank40467 +40468 POINT(40.249710441125465 73.96960165543254) bank40468 +40469 POINT(41.30174079490516 74.9884691231405) bank40469 +40470 POINT(40.975665594706975 73.77601067000194) bank40470 +40471 POINT(40.49696385802331 73.31098681152004) bank40471 +40472 POINT(40.33165350600358 73.78991769316389) bank40472 +40473 POINT(39.768965848280125 73.84263779853647) bank40473 +40474 POINT(40.4441406707419 73.31352330089285) bank40474 +40475 POINT(41.23248766315583 74.356295010068) bank40475 +40476 POINT(39.75483200936625 74.32463804188703) bank40476 +40477 POINT(41.58421894209932 73.83622117458762) bank40477 +40478 POINT(40.46792935906871 74.71419762645588) bank40478 +40479 POINT(41.70763936513697 73.9601537269315) bank40479 +40480 POINT(40.01958551656961 74.14478745372531) bank40480 +40481 POINT(40.92407565629004 74.0023104817534) bank40481 +40482 POINT(40.23050199962973 73.95818907052323) bank40482 +40483 POINT(41.25780976775098 73.31637838947395) bank40483 +40484 POINT(40.90880926591397 73.46687120685475) bank40484 +40485 POINT(40.65463327496024 74.56724405791525) bank40485 +40486 POINT(41.17872852126792 73.8478054391484) bank40486 +40487 POINT(41.43847788147877 74.5775034557295) bank40487 +40488 POINT(41.439525342448135 73.05772520718216) bank40488 +40489 POINT(39.9585625942787 74.66198512547948) bank40489 +40490 POINT(40.07006515073477 74.39856130932402) bank40490 +40491 POINT(41.13413165930228 73.3085029007989) bank40491 +40492 POINT(40.37044300396836 73.03966664550816) bank40492 +40493 POINT(40.81586135012993 74.248325329362) bank40493 +40494 POINT(41.11250739902238 74.08160255634779) bank40494 +40495 POINT(40.8813458276243 74.17670530501431) bank40495 +40496 POINT(41.57048924697246 73.13621660242698) bank40496 +40497 POINT(41.425133962621516 74.87455609317124) bank40497 +40498 POINT(39.785097848855 74.19919579234272) bank40498 +40499 POINT(41.56930759402806 74.19531536399468) bank40499 +40500 POINT(40.8359763627561 74.81282370981152) bank40500 +40501 POINT(40.13821152784062 74.95991965658229) bank40501 +40502 POINT(40.32756714696116 73.13955941476704) bank40502 +40503 POINT(41.557239099809024 74.28682337437394) bank40503 +40504 POINT(39.88085721802166 73.5329271311404) bank40504 +40505 POINT(40.938972745956825 73.21266674612474) bank40505 +40506 POINT(39.72032613101385 73.41296900986573) bank40506 +40507 POINT(41.009417741421984 73.0257863530619) bank40507 +40508 POINT(41.325842955821415 74.25071920881453) bank40508 +40509 POINT(41.50486910718058 73.82270270317984) bank40509 +40510 POINT(39.719107999439004 74.41566938350988) bank40510 +40511 POINT(40.37798092366586 73.74331333555776) bank40511 +40512 POINT(40.878913624253784 74.14944923285728) bank40512 +40513 POINT(40.6553175095259 73.45060002649578) bank40513 +40514 POINT(40.33365393821333 74.222436091096) bank40514 +40515 POINT(40.00957811179995 73.93635775982206) bank40515 +40516 POINT(41.660136526837874 74.69022879376288) bank40516 +40517 POINT(41.44101390217689 74.19480152048985) bank40517 +40518 POINT(41.390821895094966 73.67162391618925) bank40518 +40519 POINT(40.784162002519565 73.01905232861387) bank40519 +40520 POINT(40.82875745188324 73.51423154984631) bank40520 +40521 POINT(41.306665116212734 73.4531708048966) bank40521 +40522 POINT(41.390753168777145 73.07931530553981) bank40522 +40523 POINT(41.10722495935449 73.75628137355926) bank40523 +40524 POINT(41.21144135741232 73.83605798522495) bank40524 +40525 POINT(41.632677770504415 73.73524701426713) bank40525 +40526 POINT(40.892864790862916 74.1204753484458) bank40526 +40527 POINT(39.87288612184168 74.79103990861746) bank40527 +40528 POINT(40.93358169921888 73.16575885985384) bank40528 +40529 POINT(40.80833093308477 73.096454288077) bank40529 +40530 POINT(41.53380366016546 74.0338517102223) bank40530 +40531 POINT(40.014673666289745 74.34821292364174) bank40531 +40532 POINT(41.50302601471117 74.82518209393795) bank40532 +40533 POINT(40.53964754334974 74.11528121211822) bank40533 +40534 POINT(41.27294604811781 73.95534228433843) bank40534 +40535 POINT(40.99398868755863 74.61189572770736) bank40535 +40536 POINT(41.460734847340625 74.83054748414484) bank40536 +40537 POINT(40.73982730274798 73.58225846560369) bank40537 +40538 POINT(39.95618983425276 73.4035970394946) bank40538 +40539 POINT(39.96493596040567 74.73468423520347) bank40539 +40540 POINT(40.28581322807382 74.03732701747657) bank40540 +40541 POINT(41.50281272380571 73.87991430543055) bank40541 +40542 POINT(41.10304095852103 73.72523595653851) bank40542 +40543 POINT(40.55346798421315 73.2172116939582) bank40543 +40544 POINT(41.6654800224167 73.72234726081581) bank40544 +40545 POINT(41.02694482909401 73.63848218560597) bank40545 +40546 POINT(41.231032462783936 73.10838995329712) bank40546 +40547 POINT(40.68804946954497 73.57808265399952) bank40547 +40548 POINT(40.985372779959775 73.5480822538267) bank40548 +40549 POINT(40.184601507683716 73.87911704113937) bank40549 +40550 POINT(40.95703760261188 73.95754663995119) bank40550 +40551 POINT(41.48249944831273 74.45651389513202) bank40551 +40552 POINT(39.852771484293335 74.31043748023238) bank40552 +40553 POINT(40.924263157109955 73.97606393199935) bank40553 +40554 POINT(39.8470720768855 73.80176860290766) bank40554 +40555 POINT(39.78530616644146 74.78364860339843) bank40555 +40556 POINT(40.12453807524933 74.26951641383558) bank40556 +40557 POINT(40.44942608905792 74.15502950725072) bank40557 +40558 POINT(40.40989329166252 73.82674874456252) bank40558 +40559 POINT(39.781322074458494 74.00739451628363) bank40559 +40560 POINT(40.42796826377881 74.8589702332738) bank40560 +40561 POINT(40.884766056023366 73.89173543154628) bank40561 +40562 POINT(40.2952220896952 73.03345656980841) bank40562 +40563 POINT(41.33966156125359 74.25530189897069) bank40563 +40564 POINT(39.87027053224273 73.5254961881408) bank40564 +40565 POINT(41.2340451643447 73.46279766731064) bank40565 +40566 POINT(41.68925353155646 74.04378113874814) bank40566 +40567 POINT(41.463595974220794 74.30867490924601) bank40567 +40568 POINT(40.45073310204326 73.3577732235521) bank40568 +40569 POINT(39.85551356038107 74.58780831378004) bank40569 +40570 POINT(40.784095786082425 73.64749605620827) bank40570 +40571 POINT(40.961654928185965 73.08433193424412) bank40571 +40572 POINT(39.74974237675974 74.27362532876793) bank40572 +40573 POINT(39.99108384250521 74.11554000046445) bank40573 +40574 POINT(41.05189854356551 74.90087610964991) bank40574 +40575 POINT(40.35484444157142 74.71382295277058) bank40575 +40576 POINT(40.79855389366301 74.95938503837546) bank40576 +40577 POINT(40.30071793536079 73.56597004838987) bank40577 +40578 POINT(41.09521902038345 73.62733512303225) bank40578 +40579 POINT(40.85782598684852 73.04931127143855) bank40579 +40580 POINT(40.24174853457806 74.32330071038744) bank40580 +40581 POINT(40.67623773014602 74.85203754909453) bank40581 +40582 POINT(40.524532818569966 74.43679330390653) bank40582 +40583 POINT(40.266693524076004 74.22607637551165) bank40583 +40584 POINT(40.55452056192503 73.28237308210042) bank40584 +40585 POINT(40.16858087445331 73.2573093888833) bank40585 +40586 POINT(40.399284270642596 73.62458942764403) bank40586 +40587 POINT(39.940313966456124 74.00995685673337) bank40587 +40588 POINT(40.14429027361945 74.8784735221924) bank40588 +40589 POINT(41.39352188183436 74.14824069053573) bank40589 +40590 POINT(40.55732179926795 74.08529155713609) bank40590 +40591 POINT(40.40795418513153 74.26936163026986) bank40591 +40592 POINT(40.398850892088625 74.36938272402722) bank40592 +40593 POINT(39.870738309298766 73.62745936470706) bank40593 +40594 POINT(40.77133831946163 73.65155461793698) bank40594 +40595 POINT(40.66924626412742 73.16955387542147) bank40595 +40596 POINT(39.785677340436706 73.51320331388568) bank40596 +40597 POINT(40.27136124512751 73.69208024300154) bank40597 +40598 POINT(40.209888886479206 73.90307828869672) bank40598 +40599 POINT(41.55826712250119 73.47682595065892) bank40599 +40600 POINT(40.640327530572804 74.03073992699753) bank40600 +40601 POINT(41.234278637278436 73.31794272349381) bank40601 +40602 POINT(40.722691275961694 73.08376856012376) bank40602 +40603 POINT(40.05871023601772 74.13320736921615) bank40603 +40604 POINT(41.33846287849393 74.97217791445053) bank40604 +40605 POINT(40.03346106984337 74.3150085689386) bank40605 +40606 POINT(40.789139929933185 74.73144281066155) bank40606 +40607 POINT(41.33813166629575 73.19697075500122) bank40607 +40608 POINT(40.862230926885786 74.90340206105255) bank40608 +40609 POINT(41.29923054286209 73.77830581824892) bank40609 +40610 POINT(40.03142447625049 74.18131191729489) bank40610 +40611 POINT(39.95752456845775 74.61928619687792) bank40611 +40612 POINT(40.060582382063224 73.80540096240858) bank40612 +40613 POINT(41.50966269550015 73.7792011239831) bank40613 +40614 POINT(41.296437759606505 73.43119458842108) bank40614 +40615 POINT(40.870656236165146 73.66141822886948) bank40615 +40616 POINT(41.14946798097733 73.83568352757737) bank40616 +40617 POINT(40.731961022985914 74.52098981817446) bank40617 +40618 POINT(41.264802302438966 74.40309442057959) bank40618 +40619 POINT(40.03454916723501 74.67332357729978) bank40619 +40620 POINT(41.088497287559605 73.4880107873482) bank40620 +40621 POINT(41.50338183242954 73.25519680313812) bank40621 +40622 POINT(40.74238623665908 74.51357609070732) bank40622 +40623 POINT(40.687190589152344 73.72871694433749) bank40623 +40624 POINT(41.08164784092254 73.65740811649059) bank40624 +40625 POINT(40.49136681607171 73.69728919329873) bank40625 +40626 POINT(40.45665359384426 75.00564900770587) bank40626 +40627 POINT(40.36691248738497 73.11027300902691) bank40627 +40628 POINT(39.87412876008786 74.44447048354559) bank40628 +40629 POINT(41.131918076747034 73.15601850397626) bank40629 +40630 POINT(41.70929169865811 74.7630849451595) bank40630 +40631 POINT(40.088821982146456 74.11727261065155) bank40631 +40632 POINT(39.772460712887124 74.08061487493372) bank40632 +40633 POINT(40.52287863542405 74.75589569520582) bank40633 +40634 POINT(40.690353195744166 74.31084111784683) bank40634 +40635 POINT(41.25986491730183 73.25467292417493) bank40635 +40636 POINT(39.71829263886917 74.29234558900103) bank40636 +40637 POINT(40.409104549547756 73.74369571592077) bank40637 +40638 POINT(41.637782939119546 74.36117948906147) bank40638 +40639 POINT(41.619308193767075 74.65992900957893) bank40639 +40640 POINT(40.42342028918097 74.0405008974863) bank40640 +40641 POINT(40.22994333112366 74.67646365879045) bank40641 +40642 POINT(39.83039209691593 73.50900320850471) bank40642 +40643 POINT(41.64236101329711 73.50111738488381) bank40643 +40644 POINT(41.60958283923771 74.81461149499692) bank40644 +40645 POINT(41.573752789690914 74.47258075483342) bank40645 +40646 POINT(40.58251014244378 73.36930635184576) bank40646 +40647 POINT(40.111887756584615 74.76045959421698) bank40647 +40648 POINT(40.3024801714291 73.10027486567297) bank40648 +40649 POINT(41.22692541662619 74.27713167253621) bank40649 +40650 POINT(40.4713256382337 74.10186756194494) bank40650 +40651 POINT(41.00767250563434 73.59318310586733) bank40651 +40652 POINT(40.65387199466249 73.15578806436403) bank40652 +40653 POINT(41.523538459042335 73.69761597139271) bank40653 +40654 POINT(40.38815967602757 74.04780674457001) bank40654 +40655 POINT(41.276709219985804 73.05345228233425) bank40655 +40656 POINT(41.274705439084755 73.56594163839011) bank40656 +40657 POINT(40.60338171761404 74.87565751958113) bank40657 +40658 POINT(41.056951367042046 74.60620263567641) bank40658 +40659 POINT(40.57005830159977 73.15360706330412) bank40659 +40660 POINT(39.802780977910494 73.38056133586383) bank40660 +40661 POINT(39.79083711254352 74.31158872202386) bank40661 +40662 POINT(40.09455868546644 73.26340539895189) bank40662 +40663 POINT(39.883821306993696 73.85016720693211) bank40663 +40664 POINT(41.079931037476115 74.3352464083904) bank40664 +40665 POINT(40.677476617197044 73.20831863518733) bank40665 +40666 POINT(40.965833310486246 73.75347816617162) bank40666 +40667 POINT(39.887296045110524 73.43247440243466) bank40667 +40668 POINT(40.86322826368249 73.15406794422488) bank40668 +40669 POINT(40.169947746741414 74.56114704160765) bank40669 +40670 POINT(40.795258451636094 74.66540601628749) bank40670 +40671 POINT(39.97922376568899 73.55644879312726) bank40671 +40672 POINT(41.256245110491825 74.17333012158126) bank40672 +40673 POINT(41.70605885774356 73.27454613961919) bank40673 +40674 POINT(40.87711647825154 73.80613495162723) bank40674 +40675 POINT(41.40968302533224 73.11090148546232) bank40675 +40676 POINT(41.45923405281662 74.13239561455899) bank40676 +40677 POINT(40.416561929409006 74.03513918746012) bank40677 +40678 POINT(40.03187608271851 73.40088880860087) bank40678 +40679 POINT(40.38086283504065 74.645675956123) bank40679 +40680 POINT(39.80205602651927 74.9966574443622) bank40680 +40681 POINT(40.41102860588653 74.39645086035515) bank40681 +40682 POINT(41.32762419769982 74.68550641589837) bank40682 +40683 POINT(41.26894148470611 74.2436101801284) bank40683 +40684 POINT(41.36952009980249 73.97706029995878) bank40684 +40685 POINT(41.139496467174965 73.90043498177445) bank40685 +40686 POINT(41.43446727583525 74.16544233445974) bank40686 +40687 POINT(41.031235251611385 73.97089777259762) bank40687 +40688 POINT(39.86788128428361 74.68668912029493) bank40688 +40689 POINT(39.77762657505183 73.03496645176341) bank40689 +40690 POINT(41.53030342806965 73.994417993852) bank40690 +40691 POINT(41.225485418975246 74.53692964297106) bank40691 +40692 POINT(40.8761226433113 73.91669720889581) bank40692 +40693 POINT(39.809385326181356 73.20445470472822) bank40693 +40694 POINT(40.88062854595097 74.55805077519567) bank40694 +40695 POINT(40.6740862501456 73.98401034182903) bank40695 +40696 POINT(39.884719453836354 74.71223994404191) bank40696 +40697 POINT(39.939647005106906 73.77444243119045) bank40697 +40698 POINT(41.56492609219626 73.86030816345436) bank40698 +40699 POINT(40.52209024218768 74.02536733079604) bank40699 +40700 POINT(40.08543215819271 74.15162740860097) bank40700 +40701 POINT(40.87573289969024 74.80362415989921) bank40701 +40702 POINT(40.008925453788976 73.71198254265292) bank40702 +40703 POINT(40.95903030973144 73.1948711509493) bank40703 +40704 POINT(39.843254606049555 73.10466700353707) bank40704 +40705 POINT(40.65503205491751 73.4722478877973) bank40705 +40706 POINT(39.732380156944316 74.69501078662937) bank40706 +40707 POINT(40.48840761222287 73.38441886976584) bank40707 +40708 POINT(39.751517263173 73.36696770088716) bank40708 +40709 POINT(41.1464293496291 73.00853900437453) bank40709 +40710 POINT(41.02696858394875 74.89696005529078) bank40710 +40711 POINT(41.30176657582985 73.52085465896846) bank40711 +40712 POINT(40.34103817071622 73.49948044153493) bank40712 +40713 POINT(40.25643438838378 74.56518968903478) bank40713 +40714 POINT(41.103928094388785 73.09711517948794) bank40714 +40715 POINT(41.64507955264941 73.86335538326657) bank40715 +40716 POINT(40.22912266388384 73.57736547339248) bank40716 +40717 POINT(40.58385471338961 74.0597737943286) bank40717 +40718 POINT(41.31858260948176 74.06049296472952) bank40718 +40719 POINT(41.204524717621126 73.62723568654) bank40719 +40720 POINT(41.123238923384775 74.73543665034046) bank40720 +40721 POINT(40.61242682787644 73.4617950369159) bank40721 +40722 POINT(40.582792737038886 74.17861993481681) bank40722 +40723 POINT(41.412100577126296 74.15662327065182) bank40723 +40724 POINT(40.62583846099591 73.72163075817562) bank40724 +40725 POINT(39.939164848129366 74.9039259092741) bank40725 +40726 POINT(41.446391457768094 74.20885886243015) bank40726 +40727 POINT(40.00783777357414 74.90367968880992) bank40727 +40728 POINT(41.536642989102184 73.87916110169148) bank40728 +40729 POINT(40.84128923671183 74.39341537878917) bank40729 +40730 POINT(40.440760563397795 73.07339015562067) bank40730 +40731 POINT(41.64817292467646 73.18704820631241) bank40731 +40732 POINT(40.40221369694909 74.40422128733178) bank40732 +40733 POINT(40.35047995807825 73.37179356762807) bank40733 +40734 POINT(41.17319951075105 73.56536406229415) bank40734 +40735 POINT(41.454939086483805 74.24786031764748) bank40735 +40736 POINT(40.71298508816375 73.41379024064253) bank40736 +40737 POINT(40.753015246356064 74.46265004279834) bank40737 +40738 POINT(41.46242330503487 73.57014162543425) bank40738 +40739 POINT(39.92704354080113 74.6454187257348) bank40739 +40740 POINT(40.3974943222755 74.59246410804371) bank40740 +40741 POINT(41.61460581898872 74.2579665446531) bank40741 +40742 POINT(41.51777454391738 73.77435910376559) bank40742 +40743 POINT(39.95024695243605 73.35864100012283) bank40743 +40744 POINT(40.01615321231659 74.09790241688783) bank40744 +40745 POINT(40.14662662212481 73.33972966919411) bank40745 +40746 POINT(40.836667144092104 73.7802292988273) bank40746 +40747 POINT(41.65600831542356 73.40430829930207) bank40747 +40748 POINT(41.302167842699575 74.36303286860485) bank40748 +40749 POINT(40.84035541226715 74.83135566755132) bank40749 +40750 POINT(41.61051553178702 73.06759922269329) bank40750 +40751 POINT(39.92333268014611 74.96075788483765) bank40751 +40752 POINT(39.932067089791595 73.65629205638825) bank40752 +40753 POINT(40.40891773246817 73.30918245998933) bank40753 +40754 POINT(40.57989069431734 73.44496625659812) bank40754 +40755 POINT(40.693532054301755 73.20359603208833) bank40755 +40756 POINT(41.13483622544835 73.14911521488958) bank40756 +40757 POINT(41.313697267275266 74.27790606309041) bank40757 +40758 POINT(40.67325949170573 74.89003992716783) bank40758 +40759 POINT(40.05853233139639 73.90550230563899) bank40759 +40760 POINT(40.01658453462692 74.11175754430664) bank40760 +40761 POINT(40.18000647682881 73.64112135137975) bank40761 +40762 POINT(40.81089552014198 73.2501834106942) bank40762 +40763 POINT(41.48415530213479 73.40790906175238) bank40763 +40764 POINT(40.39615843848608 74.07395851922492) bank40764 +40765 POINT(40.68348682101696 74.3791327499979) bank40765 +40766 POINT(41.374617070053326 74.76084933782317) bank40766 +40767 POINT(40.250337283365525 74.98368802144539) bank40767 +40768 POINT(40.40047096594781 74.36281074550791) bank40768 +40769 POINT(39.715493277196835 73.17982737994227) bank40769 +40770 POINT(40.0659684692899 73.91192632251497) bank40770 +40771 POINT(40.80338835823069 74.70718912206455) bank40771 +40772 POINT(41.65830625899843 74.67710483469062) bank40772 +40773 POINT(41.57143108176082 74.63863405595448) bank40773 +40774 POINT(39.73579866273887 74.60505053478094) bank40774 +40775 POINT(41.11198858119198 73.9571941622343) bank40775 +40776 POINT(41.676035921576634 73.71123218303683) bank40776 +40777 POINT(40.48767142386337 74.41940367095629) bank40777 +40778 POINT(40.93811116836907 74.21862627397866) bank40778 +40779 POINT(40.81180162597449 74.56108421927914) bank40779 +40780 POINT(41.045745785608354 73.11947956059515) bank40780 +40781 POINT(40.627078312406624 73.80239121456293) bank40781 +40782 POINT(40.42750656271144 73.6919397017916) bank40782 +40783 POINT(39.83031726074523 74.53405494349441) bank40783 +40784 POINT(41.39440899221093 74.07661329779692) bank40784 +40785 POINT(39.870228836905746 74.5452911935691) bank40785 +40786 POINT(40.71722440131768 74.82928223075541) bank40786 +40787 POINT(40.718729164008266 74.61215782999814) bank40787 +40788 POINT(40.03917791073146 73.28386028936309) bank40788 +40789 POINT(39.91883213240128 74.65766946818839) bank40789 +40790 POINT(40.424486281317414 73.86385505234307) bank40790 +40791 POINT(40.24247874094941 74.6697482987533) bank40791 +40792 POINT(40.442081580016676 73.65873296280746) bank40792 +40793 POINT(40.48218851652145 73.33656465302249) bank40793 +40794 POINT(40.625341971820504 73.05009576072536) bank40794 +40795 POINT(41.21039953342191 73.90874133657613) bank40795 +40796 POINT(40.69088871762785 73.98787104771345) bank40796 +40797 POINT(41.65739954849805 73.38347141518722) bank40797 +40798 POINT(41.58175340458485 74.78011466054417) bank40798 +40799 POINT(40.49943731159337 74.2184795511188) bank40799 +40800 POINT(39.805538985118645 74.81160156271177) bank40800 +40801 POINT(41.025091030221915 74.74119637717732) bank40801 +40802 POINT(41.015476017648865 74.65846114182821) bank40802 +40803 POINT(40.97377468932191 73.35090826171307) bank40803 +40804 POINT(41.684604771403635 73.24085872544163) bank40804 +40805 POINT(40.317667936389206 74.90908800316691) bank40805 +40806 POINT(39.98813440506134 73.40860575870788) bank40806 +40807 POINT(40.205217273100004 73.99191273728103) bank40807 +40808 POINT(39.91272125279444 74.38935356483023) bank40808 +40809 POINT(40.231518566693474 74.72931589951763) bank40809 +40810 POINT(40.26746716728367 74.80916557550069) bank40810 +40811 POINT(40.696650985929864 73.67630441682067) bank40811 +40812 POINT(41.36409577097917 73.75277849869204) bank40812 +40813 POINT(40.811208202125364 74.34680434400406) bank40813 +40814 POINT(40.283753567265606 74.4666335172835) bank40814 +40815 POINT(40.78858361163388 74.37504113293765) bank40815 +40816 POINT(41.19401866556692 74.04110826357315) bank40816 +40817 POINT(40.27120832498251 73.44354165717762) bank40817 +40818 POINT(40.55715095225516 74.61474957320898) bank40818 +40819 POINT(40.961906906137905 74.08169750623259) bank40819 +40820 POINT(41.508934317927704 74.19388453417994) bank40820 +40821 POINT(39.7367252733503 74.73478833531844) bank40821 +40822 POINT(40.179938383242984 73.80711455215638) bank40822 +40823 POINT(41.37055384476796 74.33562026126327) bank40823 +40824 POINT(40.34313221158333 73.5276580127922) bank40824 +40825 POINT(41.314744452427746 73.16535868467072) bank40825 +40826 POINT(40.47629222591349 73.4579398017432) bank40826 +40827 POINT(40.24132849770828 74.9376978796071) bank40827 +40828 POINT(40.12461185494033 73.0397090578666) bank40828 +40829 POINT(40.43807801286679 74.64618661119125) bank40829 +40830 POINT(40.54329370178922 74.09648518313624) bank40830 +40831 POINT(41.552635606591146 73.01919190807983) bank40831 +40832 POINT(40.36116120387921 74.39081648188758) bank40832 +40833 POINT(41.425305575475676 74.62797293731415) bank40833 +40834 POINT(41.15415200831696 74.10871981718398) bank40834 +40835 POINT(39.94659270505437 73.62927320049853) bank40835 +40836 POINT(41.64073699512627 73.19165979412624) bank40836 +40837 POINT(41.1999279233623 74.54579807049049) bank40837 +40838 POINT(40.73961460985848 73.59584057823635) bank40838 +40839 POINT(40.05408641151816 74.95238448562576) bank40839 +40840 POINT(40.78677294379601 73.64679602808695) bank40840 +40841 POINT(41.35355400186873 74.7281411015265) bank40841 +40842 POINT(39.80380472294727 74.4803323862917) bank40842 +40843 POINT(40.46312277364126 73.66818695358667) bank40843 +40844 POINT(40.278927702846545 74.87858202355707) bank40844 +40845 POINT(40.7169452976421 73.37084622983105) bank40845 +40846 POINT(40.157657305329366 73.56494745756243) bank40846 +40847 POINT(39.87702827351493 74.80770560533263) bank40847 +40848 POINT(39.91662714645249 73.48210284364967) bank40848 +40849 POINT(40.25777559301339 73.9823681034631) bank40849 +40850 POINT(41.32471035844331 73.06870599388908) bank40850 +40851 POINT(41.20068369593936 74.89772017565177) bank40851 +40852 POINT(39.85733539735752 75.00385280419954) bank40852 +40853 POINT(39.71289352085019 73.72397450922924) bank40853 +40854 POINT(41.43835488843466 73.32826988529709) bank40854 +40855 POINT(40.92047218577894 73.60378868691332) bank40855 +40856 POINT(40.37379097148553 74.74910888193072) bank40856 +40857 POINT(39.81926413421324 73.05671395492068) bank40857 +40858 POINT(41.5945865492035 73.34423456174267) bank40858 +40859 POINT(41.102602317860224 74.44370169533015) bank40859 +40860 POINT(40.23405097189927 74.73452854165335) bank40860 +40861 POINT(40.47000290058532 74.7413197682668) bank40861 +40862 POINT(41.52948190260324 73.85921705959672) bank40862 +40863 POINT(40.611289136864286 73.53186429605279) bank40863 +40864 POINT(41.077436015524 74.37476160525385) bank40864 +40865 POINT(40.05006191148251 74.163264665416) bank40865 +40866 POINT(40.988565247143875 74.00484254108758) bank40866 +40867 POINT(41.194342353240906 73.21049634350265) bank40867 +40868 POINT(41.32531899412595 73.90918450029704) bank40868 +40869 POINT(41.38739608857978 74.44990268784827) bank40869 +40870 POINT(40.660562981650614 73.54475165834722) bank40870 +40871 POINT(41.636924724348916 74.74681509417604) bank40871 +40872 POINT(41.36097603502416 74.52256895495118) bank40872 +40873 POINT(40.71551486074411 74.64599096368902) bank40873 +40874 POINT(40.206253408047374 74.47191727142072) bank40874 +40875 POINT(40.84404873547477 74.49067276457173) bank40875 +40876 POINT(39.95025008041986 74.56806860776132) bank40876 +40877 POINT(40.38990296256322 74.24795719102353) bank40877 +40878 POINT(40.134826800882934 73.9344057592723) bank40878 +40879 POINT(40.985331425580746 73.78398206306065) bank40879 +40880 POINT(41.64485558202265 73.2419394515396) bank40880 +40881 POINT(40.92808011623307 73.78818935220444) bank40881 +40882 POINT(40.54709597227001 74.51147747499958) bank40882 +40883 POINT(40.29623722476756 74.62606435291629) bank40883 +40884 POINT(40.537507770865034 74.68546143754875) bank40884 +40885 POINT(40.68557048646981 73.33775671192379) bank40885 +40886 POINT(41.65854898730933 74.16477225381044) bank40886 +40887 POINT(40.73913148373618 73.22322195076617) bank40887 +40888 POINT(40.00778146849893 73.46335318529442) bank40888 +40889 POINT(41.57548815348564 73.48147796753585) bank40889 +40890 POINT(39.90467089943483 74.98057170715502) bank40890 +40891 POINT(40.17431065963188 73.03839321671144) bank40891 +40892 POINT(39.8079735281726 73.38056782879883) bank40892 +40893 POINT(39.71837390624824 73.0653696383368) bank40893 +40894 POINT(41.703648184665774 74.40743869500004) bank40894 +40895 POINT(40.81765417073065 73.69556947046325) bank40895 +40896 POINT(40.83514187989001 74.33181809410554) bank40896 +40897 POINT(40.185192876872556 73.24452445864931) bank40897 +40898 POINT(41.108069092363735 74.41828500397808) bank40898 +40899 POINT(40.06005163202701 73.50190436518281) bank40899 +40900 POINT(41.67071727077485 74.35843032198267) bank40900 +40901 POINT(40.22872553850432 74.29244354120817) bank40901 +40902 POINT(41.29701698065542 73.28947579643338) bank40902 +40903 POINT(41.033586278094546 74.28652103025925) bank40903 +40904 POINT(39.95075828796194 73.27358143012306) bank40904 +40905 POINT(41.28433378077145 73.2649887762814) bank40905 +40906 POINT(39.93933132678906 73.04132794404045) bank40906 +40907 POINT(40.36885590684156 74.4057125701802) bank40907 +40908 POINT(41.08063771667412 74.02478975267394) bank40908 +40909 POINT(40.24812527577324 74.66213838262254) bank40909 +40910 POINT(40.532132372475424 73.12103775985376) bank40910 +40911 POINT(41.65765814305793 73.1435629400072) bank40911 +40912 POINT(40.24397313106095 74.4869292487809) bank40912 +40913 POINT(41.281447846675135 73.31497836925064) bank40913 +40914 POINT(40.41304924378553 73.11891224036545) bank40914 +40915 POINT(41.69812080778752 73.18078325521735) bank40915 +40916 POINT(40.41308986908332 74.8393301439917) bank40916 +40917 POINT(40.22068403830878 73.58526248688126) bank40917 +40918 POINT(40.79030495560999 73.32054946476704) bank40918 +40919 POINT(41.182611508195954 73.0938917106311) bank40919 +40920 POINT(41.48198748482115 74.962441934321) bank40920 +40921 POINT(41.287215233059136 73.45044033067468) bank40921 +40922 POINT(40.49973261271173 74.10904848087722) bank40922 +40923 POINT(41.10380972256239 73.15560851025923) bank40923 +40924 POINT(40.650025209943564 73.40270371152916) bank40924 +40925 POINT(41.57922202121779 74.12161606561062) bank40925 +40926 POINT(40.92989169393549 73.91335380783337) bank40926 +40927 POINT(40.313039732741885 73.20822212814248) bank40927 +40928 POINT(41.111002408019964 74.51808493688993) bank40928 +40929 POINT(40.925612235897525 74.08191363578153) bank40929 +40930 POINT(40.7036570883903 73.92017689784494) bank40930 +40931 POINT(40.1643809534974 73.23490087094751) bank40931 +40932 POINT(40.526934845774484 74.87118311148426) bank40932 +40933 POINT(40.247364973044775 74.26124616034501) bank40933 +40934 POINT(40.17692459189716 73.28231417434829) bank40934 +40935 POINT(40.050810143123584 73.10713939302639) bank40935 +40936 POINT(39.9349803849727 73.90581785668745) bank40936 +40937 POINT(41.11648525067871 74.65562665550023) bank40937 +40938 POINT(39.771163171595674 74.27448569515555) bank40938 +40939 POINT(41.280933955849704 74.02446824735031) bank40939 +40940 POINT(40.1323189318245 73.14248803260325) bank40940 +40941 POINT(41.63795634707309 73.19420382634314) bank40941 +40942 POINT(40.14167232965611 73.01726015375483) bank40942 +40943 POINT(40.14526627061908 73.1891097914409) bank40943 +40944 POINT(41.22173797839537 73.90412157849717) bank40944 +40945 POINT(41.586114127215545 73.6734190383045) bank40945 +40946 POINT(40.14718973255219 74.77260688080024) bank40946 +40947 POINT(40.737679356334084 74.97661473903713) bank40947 +40948 POINT(40.734706785788084 73.06838006029257) bank40948 +40949 POINT(41.49282154773634 75.0005910588378) bank40949 +40950 POINT(40.09320506393766 74.42470697352815) bank40950 +40951 POINT(40.477222249765916 73.2797163451271) bank40951 +40952 POINT(40.29956321797048 73.07739213326107) bank40952 +40953 POINT(41.25729805163757 73.9114678638605) bank40953 +40954 POINT(40.259942222550094 74.09115211054444) bank40954 +40955 POINT(40.66651034928828 73.83662533150559) bank40955 +40956 POINT(40.132604128663345 73.51063512733646) bank40956 +40957 POINT(41.25146663728461 73.01465857161276) bank40957 +40958 POINT(39.79102490757144 74.63447967466435) bank40958 +40959 POINT(40.15746542901962 74.54349178730723) bank40959 +40960 POINT(40.4637877862926 73.26980163156274) bank40960 +40961 POINT(39.963118262122684 74.05091322473513) bank40961 +40962 POINT(41.11548753625189 73.91087752982496) bank40962 +40963 POINT(40.16670625313788 73.99433136308386) bank40963 +40964 POINT(41.37767084328676 74.40904254651667) bank40964 +40965 POINT(41.08272744059183 74.1928974681458) bank40965 +40966 POINT(40.5995930304535 73.30586177846722) bank40966 +40967 POINT(41.18177383376801 74.56269454089323) bank40967 +40968 POINT(40.52384763398658 73.75862510211816) bank40968 +40969 POINT(40.32878707883774 74.45316437260708) bank40969 +40970 POINT(40.301205507676414 73.28539391608717) bank40970 +40971 POINT(39.95240090502273 74.53880620751525) bank40971 +40972 POINT(41.537715034329764 73.22242149050187) bank40972 +40973 POINT(40.74156266148778 73.07910738427609) bank40973 +40974 POINT(40.367508756660634 74.03941357292554) bank40974 +40975 POINT(40.24162140661902 73.24589773354235) bank40975 +40976 POINT(41.59147688733061 74.02812337160655) bank40976 +40977 POINT(40.55236888638773 73.01675117347814) bank40977 +40978 POINT(40.25396516358457 74.79964638672097) bank40978 +40979 POINT(40.29221124260636 73.71852913281425) bank40979 +40980 POINT(41.71175571653798 73.37601768418803) bank40980 +40981 POINT(39.97249646634713 73.03687785766482) bank40981 +40982 POINT(41.03803755732807 74.81100699971186) bank40982 +40983 POINT(40.30066628755291 74.45715141787619) bank40983 +40984 POINT(39.82427328602888 74.80066890650181) bank40984 +40985 POINT(40.503595290706436 74.11661078956452) bank40985 +40986 POINT(39.72688789248644 73.03583231186592) bank40986 +40987 POINT(40.45559826736282 73.1076446272189) bank40987 +40988 POINT(40.46865677171728 74.62485553157214) bank40988 +40989 POINT(39.995881390790814 73.8425812944793) bank40989 +40990 POINT(41.526916435988056 73.67886829993124) bank40990 +40991 POINT(41.10075530628035 74.31595614803496) bank40991 +40992 POINT(40.8779077126041 74.8350856402158) bank40992 +40993 POINT(41.00090422526697 73.58766045450157) bank40993 +40994 POINT(41.20567964721399 73.39252955903021) bank40994 +40995 POINT(40.50373500764111 73.75784496085457) bank40995 +40996 POINT(41.035836224750945 73.43866069940158) bank40996 +40997 POINT(40.91636653740797 73.94268363299464) bank40997 +40998 POINT(40.87490924871643 73.88401331367463) bank40998 +40999 POINT(40.0354304277398 73.51333951070883) bank40999 +41000 POINT(39.865019222075965 73.26146763530876) bank41000 +41001 POINT(40.22527232771847 73.06446502149456) bank41001 +41002 POINT(41.411940248042335 73.07362202291887) bank41002 +41003 POINT(41.43597400243575 73.70035253522845) bank41003 +41004 POINT(41.04110458183576 73.61130769971328) bank41004 +41005 POINT(39.86537537258342 73.09888561222076) bank41005 +41006 POINT(40.0071622271843 73.11618037088947) bank41006 +41007 POINT(40.94187392562139 73.63198102303181) bank41007 +41008 POINT(40.258794194254165 73.82838761907237) bank41008 +41009 POINT(40.43617437797337 73.99516825629101) bank41009 +41010 POINT(39.76254267974554 74.8648653205519) bank41010 +41011 POINT(40.002643201235244 73.53661016812606) bank41011 +41012 POINT(41.284895842827346 74.35921039539016) bank41012 +41013 POINT(41.05759833916504 73.99252431788817) bank41013 +41014 POINT(39.75605688732355 74.63992393878941) bank41014 +41015 POINT(40.88858911440704 73.71911520567403) bank41015 +41016 POINT(41.075822889416244 74.64286429030288) bank41016 +41017 POINT(39.76141369728063 74.32794410364966) bank41017 +41018 POINT(41.52978817766948 74.14320177102869) bank41018 +41019 POINT(41.225377649694884 74.00939421060949) bank41019 +41020 POINT(41.269350274786966 74.20966102093392) bank41020 +41021 POINT(40.55442116181588 73.20369474863324) bank41021 +41022 POINT(41.58503377588226 74.62075936300491) bank41022 +41023 POINT(39.85600486504993 74.42829020238437) bank41023 +41024 POINT(40.72026855070795 74.28546903076798) bank41024 +41025 POINT(41.340519908777466 73.47452545467809) bank41025 +41026 POINT(40.33728138828876 74.13223964515718) bank41026 +41027 POINT(41.04623641370204 74.9433280824396) bank41027 +41028 POINT(41.258100026493686 73.9178006354392) bank41028 +41029 POINT(41.01827942589659 74.05452118146935) bank41029 +41030 POINT(41.24189580390514 73.29168722411052) bank41030 +41031 POINT(40.86094863293849 74.37797721681989) bank41031 +41032 POINT(40.97729069648927 73.16944018910701) bank41032 +41033 POINT(41.20980079526777 73.47171899136612) bank41033 +41034 POINT(40.06855376491067 73.51927899244332) bank41034 +41035 POINT(41.078108344068504 73.38141727062164) bank41035 +41036 POINT(41.53863693442523 73.33270691823796) bank41036 +41037 POINT(39.719546822247075 73.95055693081913) bank41037 +41038 POINT(41.001130834149166 73.72359683413879) bank41038 +41039 POINT(41.320720932417316 73.52377565905081) bank41039 +41040 POINT(40.21566392812194 74.55111528719094) bank41040 +41041 POINT(40.30523304914992 74.23745864564991) bank41041 +41042 POINT(40.08802447429216 73.46134025319945) bank41042 +41043 POINT(39.91686696766626 73.42406246616126) bank41043 +41044 POINT(41.22208381946388 73.63745168977854) bank41044 +41045 POINT(40.15189745572089 73.06509900160734) bank41045 +41046 POINT(40.565083944747784 74.85942466841014) bank41046 +41047 POINT(40.33539879979269 74.15509559544141) bank41047 +41048 POINT(40.66267708172289 73.10485587861055) bank41048 +41049 POINT(40.37383729387891 74.45598628034739) bank41049 +41050 POINT(40.71270350375191 73.9922934718837) bank41050 +41051 POINT(40.358516091210845 73.71116956885663) bank41051 +41052 POINT(40.16543014897881 73.90432019429046) bank41052 +41053 POINT(40.62139474476906 74.68986694433451) bank41053 +41054 POINT(41.384116886886844 73.80195309870629) bank41054 +41055 POINT(40.36210871110832 73.39813048668996) bank41055 +41056 POINT(39.946286930810714 73.86412828062363) bank41056 +41057 POINT(39.99252437026658 74.03477321750293) bank41057 +41058 POINT(41.33188717329544 73.59762111209585) bank41058 +41059 POINT(41.008225310601816 73.2684294005764) bank41059 +41060 POINT(40.769439540685404 74.03563403379403) bank41060 +41061 POINT(40.6864453112872 74.69245047192064) bank41061 +41062 POINT(41.42284114554673 74.05534153763469) bank41062 +41063 POINT(41.40962793788493 74.18425406275477) bank41063 +41064 POINT(40.38323119534508 73.94773620300536) bank41064 +41065 POINT(40.778578331393255 73.74370056062602) bank41065 +41066 POINT(41.40579499895563 74.20008912503067) bank41066 +41067 POINT(41.38089106371916 74.32275590278331) bank41067 +41068 POINT(41.60690192732403 74.27379460043994) bank41068 +41069 POINT(40.56656349781136 73.95478347462722) bank41069 +41070 POINT(40.94108088935404 74.35709574873499) bank41070 +41071 POINT(40.67852274100306 73.85335308862592) bank41071 +41072 POINT(40.72349080017389 73.00719223173365) bank41072 +41073 POINT(39.817109123740266 73.39122602687753) bank41073 +41074 POINT(40.669422025332985 73.81333796011806) bank41074 +41075 POINT(40.76101858582874 73.09262290469199) bank41075 +41076 POINT(41.233996194843826 73.56159600886208) bank41076 +41077 POINT(40.97074760107271 74.25255569210609) bank41077 +41078 POINT(41.0127145969219 73.37590245799136) bank41078 +41079 POINT(41.075140581737 74.93585407621114) bank41079 +41080 POINT(41.35419091390518 73.79101105402724) bank41080 +41081 POINT(40.12717878153124 74.35614095842591) bank41081 +41082 POINT(40.99346847488994 73.15077407063906) bank41082 +41083 POINT(41.57567034828631 74.96740758826172) bank41083 +41084 POINT(39.91779000765068 74.22471592921934) bank41084 +41085 POINT(40.71925914256132 73.23720487362827) bank41085 +41086 POINT(40.97859010228849 73.9477103180389) bank41086 +41087 POINT(39.89601175673896 74.01768253040525) bank41087 +41088 POINT(41.3574666353736 73.64303471067811) bank41088 +41089 POINT(40.619947022440776 73.5342960256771) bank41089 +41090 POINT(41.05395280032212 74.64000346414949) bank41090 +41091 POINT(39.95371943616587 73.55669810598165) bank41091 +41092 POINT(40.608897130418505 74.73218350495299) bank41092 +41093 POINT(39.841321391662035 74.12003624388487) bank41093 +41094 POINT(40.95753489810005 73.13365713136761) bank41094 +41095 POINT(40.37269297233873 73.46320570810224) bank41095 +41096 POINT(40.996158166347335 74.31488017159352) bank41096 +41097 POINT(40.738206803048826 74.54262928715752) bank41097 +41098 POINT(41.12538088539524 73.40711612124483) bank41098 +41099 POINT(41.09817086385125 74.69651199045157) bank41099 +41100 POINT(41.62213477468655 73.60632191531666) bank41100 +41101 POINT(41.04679911520182 73.05749853977599) bank41101 +41102 POINT(40.95810714461054 73.76256686222568) bank41102 +41103 POINT(41.207490236957504 74.66426335170252) bank41103 +41104 POINT(40.51877495261315 74.44389700232571) bank41104 +41105 POINT(40.64341493441914 74.64814206974006) bank41105 +41106 POINT(41.508052704945214 73.12722739837386) bank41106 +41107 POINT(41.22817156620388 74.76435243959985) bank41107 +41108 POINT(40.04256353322063 73.80915862489843) bank41108 +41109 POINT(40.052475933795485 74.32084619504573) bank41109 +41110 POINT(40.92569928993668 74.16852362732575) bank41110 +41111 POINT(41.38593053809784 74.23093111138874) bank41111 +41112 POINT(40.71213112757581 73.18029272934116) bank41112 +41113 POINT(41.01548862095168 73.92510543883468) bank41113 +41114 POINT(41.371749621044174 73.38825586606202) bank41114 +41115 POINT(41.32195452111033 74.89250424054171) bank41115 +41116 POINT(39.71535458504524 73.10535063957809) bank41116 +41117 POINT(40.378236971481336 74.10359463549895) bank41117 +41118 POINT(39.79799277369164 73.46421139758071) bank41118 +41119 POINT(40.71476417486848 74.54810012157792) bank41119 +41120 POINT(41.190420207192325 73.26387151223477) bank41120 +41121 POINT(40.031816843130585 73.47936967302503) bank41121 +41122 POINT(40.91777331197806 74.29884764880724) bank41122 +41123 POINT(41.44883881138768 73.43452228132928) bank41123 +41124 POINT(41.5771646474844 74.15814718923954) bank41124 +41125 POINT(41.139175763160324 73.26514204954323) bank41125 +41126 POINT(40.08002606287463 73.18752262158938) bank41126 +41127 POINT(39.87100378357933 74.63736043427714) bank41127 +41128 POINT(39.9526999397793 74.41038493521795) bank41128 +41129 POINT(40.168595429325634 73.30871469193666) bank41129 +41130 POINT(40.48639866765972 73.1680208330576) bank41130 +41131 POINT(40.09888892609777 74.7523437577234) bank41131 +41132 POINT(39.71378598126923 73.01251279559021) bank41132 +41133 POINT(40.27364150639063 74.83593621242402) bank41133 +41134 POINT(41.153373115403696 74.77676880465023) bank41134 +41135 POINT(39.936207780476316 74.78288436783389) bank41135 +41136 POINT(41.394733735524774 74.99760108006896) bank41136 +41137 POINT(40.488411933150005 74.48646044934698) bank41137 +41138 POINT(39.791012437112364 73.65134730736321) bank41138 +41139 POINT(40.43465880817922 73.22277610021726) bank41139 +41140 POINT(41.49533663115986 73.64620632016253) bank41140 +41141 POINT(41.71085354445616 74.11677379561355) bank41141 +41142 POINT(39.717238201028884 74.11132633394982) bank41142 +41143 POINT(41.613967236298734 73.03603963869547) bank41143 +41144 POINT(39.945208029331944 74.85282737695034) bank41144 +41145 POINT(41.03665584819876 73.06942221597417) bank41145 +41146 POINT(39.819035935068044 74.34735634923189) bank41146 +41147 POINT(41.477678007511024 74.06310080604938) bank41147 +41148 POINT(41.68083905832377 74.78047132683255) bank41148 +41149 POINT(40.894824793364016 73.83297356417984) bank41149 +41150 POINT(40.685348850137274 73.37592927805095) bank41150 +41151 POINT(41.01985435093185 74.20130161396212) bank41151 +41152 POINT(39.948109553266235 74.4145700203633) bank41152 +41153 POINT(41.41838203398167 73.93843568983326) bank41153 +41154 POINT(40.26138527443603 73.75888367813468) bank41154 +41155 POINT(39.93771283277131 73.8222229361024) bank41155 +41156 POINT(39.88476286812268 73.39499670887808) bank41156 +41157 POINT(40.33831907745225 73.18471999140479) bank41157 +41158 POINT(41.396070138054775 74.01195741879458) bank41158 +41159 POINT(41.59082775221798 74.72256358427481) bank41159 +41160 POINT(41.41542776903805 73.58487718580676) bank41160 +41161 POINT(41.15879578685692 73.02177963793157) bank41161 +41162 POINT(41.21818932119874 73.32205338836087) bank41162 +41163 POINT(41.54107118745326 74.62693864735522) bank41163 +41164 POINT(39.71749807159176 74.22433785776194) bank41164 +41165 POINT(40.69730024768342 73.69019530879817) bank41165 +41166 POINT(41.36854006811878 74.71085636205464) bank41166 +41167 POINT(41.14174639193807 74.78653897742106) bank41167 +41168 POINT(40.9412230219833 73.03126590725356) bank41168 +41169 POINT(41.165942541148794 73.73614694166359) bank41169 +41170 POINT(40.0675415348695 74.71988784713226) bank41170 +41171 POINT(40.755467746738475 74.76215367477911) bank41171 +41172 POINT(41.1775476611814 74.47267997473905) bank41172 +41173 POINT(41.14422174337163 73.14763321094024) bank41173 +41174 POINT(40.919246870080606 73.08388443203192) bank41174 +41175 POINT(41.23972060550561 74.33580266454489) bank41175 +41176 POINT(40.13006272401333 74.9886191945213) bank41176 +41177 POINT(40.947284818198455 74.6350828826302) bank41177 +41178 POINT(40.39072705211774 73.23826225704221) bank41178 +41179 POINT(40.74271147175554 74.8109713651587) bank41179 +41180 POINT(40.73895468710154 74.18596495085161) bank41180 +41181 POINT(40.12121508678633 73.2837963309907) bank41181 +41182 POINT(40.57080321405328 74.8455940295284) bank41182 +41183 POINT(39.773564392912725 73.06946746011278) bank41183 +41184 POINT(41.158685876902844 74.28629950323892) bank41184 +41185 POINT(40.09120621965279 73.47838825646825) bank41185 +41186 POINT(41.69427977093741 74.26883500367543) bank41186 +41187 POINT(39.826103621135935 73.74282040726179) bank41187 +41188 POINT(41.131427275792696 74.39301820725845) bank41188 +41189 POINT(39.84495375270121 74.35907266638431) bank41189 +41190 POINT(41.075701139808864 73.57338532805525) bank41190 +41191 POINT(39.74419910058378 74.9352038328777) bank41191 +41192 POINT(40.827914191206645 74.31156900860371) bank41192 +41193 POINT(41.62757209136994 74.29434812012727) bank41193 +41194 POINT(40.27091022233859 74.03849815080002) bank41194 +41195 POINT(41.47368401368246 73.13012312203197) bank41195 +41196 POINT(41.063085650322805 73.76183174360274) bank41196 +41197 POINT(41.27074364023619 74.47574758170103) bank41197 +41198 POINT(39.791900124008 74.90634833103377) bank41198 +41199 POINT(40.51492308049537 73.64619466426147) bank41199 +41200 POINT(41.64770492839437 73.54431057845544) bank41200 +41201 POINT(41.47595561482439 73.06177880131797) bank41201 +41202 POINT(40.011260714276084 73.74620605793012) bank41202 +41203 POINT(41.598146820654925 74.4866388407793) bank41203 +41204 POINT(41.4782464444976 73.57331587582378) bank41204 +41205 POINT(41.21762919975743 73.76020629972706) bank41205 +41206 POINT(40.14507084534402 74.89725049949179) bank41206 +41207 POINT(40.511086161947304 73.02083012989034) bank41207 +41208 POINT(40.31503551339122 73.82918467862294) bank41208 +41209 POINT(41.447281595569486 74.70260247769771) bank41209 +41210 POINT(40.24066668016928 74.96116065088518) bank41210 +41211 POINT(40.69399673678936 73.08449945470613) bank41211 +41212 POINT(39.86634991942688 74.1079717970487) bank41212 +41213 POINT(41.40707219164402 74.3976233744146) bank41213 +41214 POINT(40.41499373943857 73.39203847560088) bank41214 +41215 POINT(40.057459214988704 74.06063841564885) bank41215 +41216 POINT(40.378255496825666 73.79919670818401) bank41216 +41217 POINT(40.77388135326132 73.8326941984671) bank41217 +41218 POINT(41.32155588443375 73.5221454516418) bank41218 +41219 POINT(41.01171432276894 74.12734595246384) bank41219 +41220 POINT(41.654539963174756 74.97500219364686) bank41220 +41221 POINT(41.452194856018934 73.79798686084777) bank41221 +41222 POINT(39.863640879456376 74.30465761276625) bank41222 +41223 POINT(41.02637174397217 74.10402392405929) bank41223 +41224 POINT(40.79042318513426 74.29441975324161) bank41224 +41225 POINT(41.28481034945823 73.32843882702939) bank41225 +41226 POINT(40.42531776319862 74.93186986850753) bank41226 +41227 POINT(41.482340838751526 74.47498298870296) bank41227 +41228 POINT(40.473090637865724 73.05582590000252) bank41228 +41229 POINT(40.54040992621949 73.05938559182177) bank41229 +41230 POINT(40.83618827814855 73.32261916491876) bank41230 +41231 POINT(40.497218042336115 74.22054760933409) bank41231 +41232 POINT(40.08460371231355 73.97632635407132) bank41232 +41233 POINT(40.800797854826236 73.44271750115975) bank41233 +41234 POINT(41.34299361552687 74.68038755174227) bank41234 +41235 POINT(40.67032468888934 73.28218539221443) bank41235 +41236 POINT(41.69779313997415 74.56233955467903) bank41236 +41237 POINT(41.1566401013557 74.77444651620195) bank41237 +41238 POINT(39.85272944759772 74.06046148196383) bank41238 +41239 POINT(40.463251565347676 73.85258161582951) bank41239 +41240 POINT(41.50729866113572 73.30627179877068) bank41240 +41241 POINT(40.808916809246035 74.96826360310966) bank41241 +41242 POINT(40.857967797148625 74.14045898062734) bank41242 +41243 POINT(40.2988036744113 73.34031022486346) bank41243 +41244 POINT(40.196039250510275 73.10323356497) bank41244 +41245 POINT(41.5447259539798 74.2240226157194) bank41245 +41246 POINT(40.716976025338624 74.39541131617162) bank41246 +41247 POINT(41.59481904966721 74.7427905644815) bank41247 +41248 POINT(40.41539059186212 74.4004523146151) bank41248 +41249 POINT(40.72239286054752 74.95187880228983) bank41249 +41250 POINT(40.881079376358315 73.45511772609716) bank41250 +41251 POINT(40.28308319067981 74.60448875742492) bank41251 +41252 POINT(39.82913993907267 73.31462358064334) bank41252 +41253 POINT(41.45107357093145 73.66992780199273) bank41253 +41254 POINT(40.45633444184815 74.01677120806589) bank41254 +41255 POINT(40.665367297804345 74.39645575291445) bank41255 +41256 POINT(40.602015525752336 73.28955802956787) bank41256 +41257 POINT(41.63642322013207 73.3491886126025) bank41257 +41258 POINT(39.83458475357189 73.1181591913803) bank41258 +41259 POINT(41.53621966477772 74.53243509132257) bank41259 +41260 POINT(41.37600272010241 74.11151308430102) bank41260 +41261 POINT(40.30015839949831 73.44096220244892) bank41261 +41262 POINT(41.21845132699351 73.18228048065475) bank41262 +41263 POINT(39.774026027198225 73.90283292517354) bank41263 +41264 POINT(40.25313155843748 73.02981070528092) bank41264 +41265 POINT(40.512379608181554 73.54411487062188) bank41265 +41266 POINT(40.918498887917465 74.1255603104356) bank41266 +41267 POINT(40.1667787519981 73.40659094812345) bank41267 +41268 POINT(39.890367470470025 74.10481111603582) bank41268 +41269 POINT(41.09980590725627 74.98601215362287) bank41269 +41270 POINT(41.387904427809055 74.58679787845537) bank41270 +41271 POINT(39.91482468094746 73.59351914865928) bank41271 +41272 POINT(40.092295826573874 73.24956408677707) bank41272 +41273 POINT(41.48476470456088 73.88908014132996) bank41273 +41274 POINT(40.317680363980344 73.09883915048275) bank41274 +41275 POINT(40.755642746101366 73.59224761893981) bank41275 +41276 POINT(40.78123473133318 73.94422496527544) bank41276 +41277 POINT(41.148505173087244 73.95617084583174) bank41277 +41278 POINT(40.92224449693927 73.52456581695498) bank41278 +41279 POINT(40.15163099562719 74.00224486125974) bank41279 +41280 POINT(41.25940706010813 74.85631145166957) bank41280 +41281 POINT(39.898994343306015 74.07784906525524) bank41281 +41282 POINT(40.69908190063486 73.64254786604025) bank41282 +41283 POINT(41.336743344192925 74.95802317764377) bank41283 +41284 POINT(40.40143466655989 74.16301913672056) bank41284 +41285 POINT(40.03198944531429 73.1685059348978) bank41285 +41286 POINT(40.432371959609725 73.58031548014948) bank41286 +41287 POINT(39.79578247832495 74.13222302268376) bank41287 +41288 POINT(41.54177414642625 74.11808568359424) bank41288 +41289 POINT(40.38667634816766 74.15835500188221) bank41289 +41290 POINT(41.43621636658473 73.44492626850105) bank41290 +41291 POINT(41.04942659610492 73.2733464702747) bank41291 +41292 POINT(41.669857702893516 73.80231603318849) bank41292 +41293 POINT(41.428197572813495 74.77395450833905) bank41293 +41294 POINT(40.10800162931477 74.42220171866195) bank41294 +41295 POINT(39.947580107578816 74.89799092107178) bank41295 +41296 POINT(40.94039789892019 73.49546628046454) bank41296 +41297 POINT(40.502021168249094 74.36412334201478) bank41297 +41298 POINT(40.4323364837892 74.64807359231008) bank41298 +41299 POINT(39.81295873342261 74.70191810052343) bank41299 +41300 POINT(40.19385826731633 73.27393634494683) bank41300 +41301 POINT(39.94157494952578 74.75996465900644) bank41301 +41302 POINT(41.53798704523774 73.38246684229284) bank41302 +41303 POINT(40.32322629007992 73.27189835070547) bank41303 +41304 POINT(40.483750679688946 73.24375844096791) bank41304 +41305 POINT(41.33338545177891 73.1630853507302) bank41305 +41306 POINT(41.68528915056237 74.44621012197973) bank41306 +41307 POINT(39.78245483779383 74.70307004749363) bank41307 +41308 POINT(40.53258975420462 73.6558496191534) bank41308 +41309 POINT(40.57979173793433 74.16171024003972) bank41309 +41310 POINT(39.771508165880135 74.4178208690648) bank41310 +41311 POINT(40.69625692139716 74.65479336542458) bank41311 +41312 POINT(40.0959984868059 73.78835076602655) bank41312 +41313 POINT(40.061894191997375 73.71954911053845) bank41313 +41314 POINT(41.091202829728395 74.3899239694817) bank41314 +41315 POINT(39.8207558577282 73.42506272282407) bank41315 +41316 POINT(40.78503800948355 74.84883885454381) bank41316 +41317 POINT(40.11110021174083 73.65645690931723) bank41317 +41318 POINT(39.78863224665458 73.08155565694419) bank41318 +41319 POINT(40.67841645069311 73.5042656454168) bank41319 +41320 POINT(40.972880420364746 73.22666632053698) bank41320 +41321 POINT(41.01251437687565 74.07228657031224) bank41321 +41322 POINT(41.25658960733084 73.4792631408191) bank41322 +41323 POINT(40.12973505684543 73.12228395793272) bank41323 +41324 POINT(40.0987834515576 73.535282931771) bank41324 +41325 POINT(41.66481556840453 74.3049711410649) bank41325 +41326 POINT(41.14017282884032 73.67829037311613) bank41326 +41327 POINT(41.48441360453406 73.98919329424773) bank41327 +41328 POINT(40.31492975593565 73.3319422622014) bank41328 +41329 POINT(40.91077179573039 74.74362547539296) bank41329 +41330 POINT(40.761053314093836 74.34986715030503) bank41330 +41331 POINT(41.06700065310086 74.23265825335785) bank41331 +41332 POINT(40.965678699187485 74.8152291037357) bank41332 +41333 POINT(40.233922570797034 73.99686259460016) bank41333 +41334 POINT(40.847330899201445 74.0244437913832) bank41334 +41335 POINT(40.568135987455186 73.38401137842386) bank41335 +41336 POINT(41.12104922311973 73.89849277435994) bank41336 +41337 POINT(40.252887541795054 73.44394001935889) bank41337 +41338 POINT(41.41088521168851 73.14455083736543) bank41338 +41339 POINT(39.825600133543524 73.80331128345918) bank41339 +41340 POINT(41.016221384599355 74.91304258524865) bank41340 +41341 POINT(40.74277592761187 74.01253335292719) bank41341 +41342 POINT(41.35518937193522 74.03495623345974) bank41342 +41343 POINT(40.739640392425365 73.74504171127074) bank41343 +41344 POINT(41.64211813896844 73.31139502310947) bank41344 +41345 POINT(41.216652309549744 74.90299341378046) bank41345 +41346 POINT(39.9708551702313 73.75464018027735) bank41346 +41347 POINT(41.08194310677598 73.73952829096835) bank41347 +41348 POINT(41.57003734708707 73.49106665992497) bank41348 +41349 POINT(40.734555809254466 73.75654875929912) bank41349 +41350 POINT(41.578168143735454 73.74729257448004) bank41350 +41351 POINT(39.735751948491284 73.29925462276627) bank41351 +41352 POINT(40.75954022317327 73.73714517129052) bank41352 +41353 POINT(40.56008398104268 74.02875033169673) bank41353 +41354 POINT(39.84851923806873 73.61462859310112) bank41354 +41355 POINT(40.88506894192567 73.7179720949804) bank41355 +41356 POINT(40.31534096126072 74.22826010036442) bank41356 +41357 POINT(39.832301549663455 73.56540145815573) bank41357 +41358 POINT(41.60244361605534 74.53452133525616) bank41358 +41359 POINT(41.28541895994598 73.19741604973181) bank41359 +41360 POINT(40.992773907012406 74.2130135316436) bank41360 +41361 POINT(40.337490493317745 73.70402339521301) bank41361 +41362 POINT(41.558483287052425 73.60578674329669) bank41362 +41363 POINT(41.329903073660965 73.7426463993889) bank41363 +41364 POINT(40.79686178665969 73.19109622483113) bank41364 +41365 POINT(41.661934285892315 73.72616917854192) bank41365 +41366 POINT(40.3494276912186 73.87699107927403) bank41366 +41367 POINT(41.248983438209066 73.7984907291449) bank41367 +41368 POINT(40.89334854830686 74.74421607885952) bank41368 +41369 POINT(40.355123396463185 73.26363318689329) bank41369 +41370 POINT(41.33154753939698 74.5101144183691) bank41370 +41371 POINT(40.68022375475831 74.92811859404989) bank41371 +41372 POINT(40.95082335931149 74.51290677185388) bank41372 +41373 POINT(40.93329006889303 73.41835677695339) bank41373 +41374 POINT(41.1505918146849 73.11554232544637) bank41374 +41375 POINT(40.2178586516279 74.17295002122368) bank41375 +41376 POINT(40.58999621073729 73.47125003156295) bank41376 +41377 POINT(40.532809889977464 74.1761115621231) bank41377 +41378 POINT(41.03974045013316 73.9287242720548) bank41378 +41379 POINT(39.823340287977636 74.524492874895) bank41379 +41380 POINT(41.50412849024126 73.1786653268986) bank41380 +41381 POINT(39.75166823893401 73.39030320659508) bank41381 +41382 POINT(40.87813554236417 74.45248837257975) bank41382 +41383 POINT(40.55018339123164 73.88636048082807) bank41383 +41384 POINT(40.24443602309995 74.01524158128178) bank41384 +41385 POINT(41.37577602630889 73.1906291305252) bank41385 +41386 POINT(40.44124601933706 74.83192428850788) bank41386 +41387 POINT(41.21390527441606 73.46806887595147) bank41387 +41388 POINT(40.455096380772844 73.7353157899849) bank41388 +41389 POINT(40.59340751917599 73.70382181510307) bank41389 +41390 POINT(41.43459653444296 74.31705480890713) bank41390 +41391 POINT(41.139195375364395 74.15805536544602) bank41391 +41392 POINT(39.75377813217069 73.641797913401) bank41392 +41393 POINT(39.80056525748882 73.6742265804121) bank41393 +41394 POINT(41.277029669557685 74.76876886455202) bank41394 +41395 POINT(40.59829773159174 73.71759182357113) bank41395 +41396 POINT(41.340575783786846 74.09174757904695) bank41396 +41397 POINT(41.64749892067526 73.6772921754337) bank41397 +41398 POINT(39.739318800797776 73.06400333660035) bank41398 +41399 POINT(40.79161594877338 74.15754322049268) bank41399 +41400 POINT(39.815397345770734 73.36451729812511) bank41400 +41401 POINT(40.67231760431063 74.95041958099519) bank41401 +41402 POINT(41.03675957176402 73.29536009449355) bank41402 +41403 POINT(40.26501394001564 74.30460331292085) bank41403 +41404 POINT(41.49295032303659 74.22731379416238) bank41404 +41405 POINT(41.36987085738354 74.99335156896534) bank41405 +41406 POINT(40.39362902539315 74.00714067841561) bank41406 +41407 POINT(41.30480272056432 74.34972034918516) bank41407 +41408 POINT(40.15362375869576 74.68444443019801) bank41408 +41409 POINT(41.308162067199255 73.11138816559243) bank41409 +41410 POINT(40.77144027719878 74.70242445503192) bank41410 +41411 POINT(41.520473339937915 73.08919341484867) bank41411 +41412 POINT(39.71885380639335 74.1738693112052) bank41412 +41413 POINT(41.10541292390934 73.01599638942523) bank41413 +41414 POINT(40.389251828701106 74.21182119565864) bank41414 +41415 POINT(40.605498379108184 74.62639769465369) bank41415 +41416 POINT(41.25600356447398 73.37401834847525) bank41416 +41417 POINT(40.2889353333448 74.60268322191884) bank41417 +41418 POINT(41.000623968166764 74.84505481846804) bank41418 +41419 POINT(39.79937750758992 74.31802413062714) bank41419 +41420 POINT(40.49380816687639 73.42625043229503) bank41420 +41421 POINT(41.42519435361442 74.31141769824015) bank41421 +41422 POINT(40.25021853309774 73.0562254593308) bank41422 +41423 POINT(40.97003931102261 73.36150464298787) bank41423 +41424 POINT(41.08520774430361 73.12290106678942) bank41424 +41425 POINT(41.00546973324223 73.54020973639234) bank41425 +41426 POINT(40.280584350213864 74.2240741693511) bank41426 +41427 POINT(40.79902010191408 74.24318240071086) bank41427 +41428 POINT(39.77088091075216 74.84160441450392) bank41428 +41429 POINT(41.36639794390778 73.98798219038753) bank41429 +41430 POINT(39.82976285868069 73.85380183791082) bank41430 +41431 POINT(41.494099865453755 73.13675887758626) bank41431 +41432 POINT(39.96138081598287 74.42588638415847) bank41432 +41433 POINT(40.47659118315236 74.16252315938542) bank41433 +41434 POINT(41.63333747622537 74.12523762162782) bank41434 +41435 POINT(40.656580979446595 73.56796315812761) bank41435 +41436 POINT(41.50698002221762 73.73154724137189) bank41436 +41437 POINT(40.76096840303111 73.1598971129667) bank41437 +41438 POINT(40.1632173502744 74.20488015184172) bank41438 +41439 POINT(41.16557849465243 74.07350169806122) bank41439 +41440 POINT(41.11317032419401 73.12825321442669) bank41440 +41441 POINT(41.53706362191316 73.75689380678809) bank41441 +41442 POINT(39.995328971947394 73.08636382733832) bank41442 +41443 POINT(40.090706311137254 74.8200544719435) bank41443 +41444 POINT(41.481019797095264 73.84563403858684) bank41444 +41445 POINT(41.082867851879826 74.78570042615821) bank41445 +41446 POINT(39.73173214476324 73.43657353102442) bank41446 +41447 POINT(40.57620977493472 73.89809851242168) bank41447 +41448 POINT(41.44970415860561 73.34231760302514) bank41448 +41449 POINT(40.073253835278194 74.04354292696851) bank41449 +41450 POINT(39.882302927001426 73.07978789642992) bank41450 +41451 POINT(41.15455804682638 74.72411116770735) bank41451 +41452 POINT(40.23617772026123 74.15131891951826) bank41452 +41453 POINT(40.54956070750158 73.12050185066266) bank41453 +41454 POINT(40.41012183661177 74.92310072340211) bank41454 +41455 POINT(40.503470305260926 74.04590696671073) bank41455 +41456 POINT(40.23356159532317 73.02137726143555) bank41456 +41457 POINT(41.4546322980658 74.6319206113212) bank41457 +41458 POINT(41.3257571234789 74.68995310280813) bank41458 +41459 POINT(40.98800548692313 73.24365921561518) bank41459 +41460 POINT(40.459179851152484 73.52746187879357) bank41460 +41461 POINT(41.06392486109269 74.80462147773545) bank41461 +41462 POINT(40.58490196023424 74.98844567214353) bank41462 +41463 POINT(41.459968842926116 73.74464496650346) bank41463 +41464 POINT(40.06999851802847 73.5816654566783) bank41464 +41465 POINT(39.92707324855186 73.78215072204524) bank41465 +41466 POINT(39.845412142384674 74.8718002531583) bank41466 +41467 POINT(41.268968687324055 73.82657517434939) bank41467 +41468 POINT(41.22671645323311 73.29714843543228) bank41468 +41469 POINT(39.762402866676474 73.50956531785552) bank41469 +41470 POINT(40.4480061784279 74.4073898576455) bank41470 +41471 POINT(41.468233520323274 73.48486863371369) bank41471 +41472 POINT(40.77597891306139 74.73728096859554) bank41472 +41473 POINT(39.95321779362966 74.48876843444397) bank41473 +41474 POINT(41.013219437928235 73.90713508957906) bank41474 +41475 POINT(40.25910754606898 74.84664462478544) bank41475 +41476 POINT(40.084231303067725 74.30162775544373) bank41476 +41477 POINT(40.34127204785112 74.91802616658002) bank41477 +41478 POINT(39.94174456437521 73.12842008668599) bank41478 +41479 POINT(41.15417122593163 73.79949992440292) bank41479 +41480 POINT(41.069243603424496 74.2812311140013) bank41480 +41481 POINT(41.146009911505615 73.20381866927819) bank41481 +41482 POINT(41.14356125404235 73.27220341441372) bank41482 +41483 POINT(40.692854466873186 73.86864251982362) bank41483 +41484 POINT(40.41634861546191 73.01665560906024) bank41484 +41485 POINT(39.82769485521088 73.63806781244799) bank41485 +41486 POINT(41.462647937737756 74.89472010650971) bank41486 +41487 POINT(40.79581035562372 74.1033488195879) bank41487 +41488 POINT(41.641655988448505 73.51819102367774) bank41488 +41489 POINT(40.196497178496266 74.30006868846723) bank41489 +41490 POINT(41.50665669925051 73.45513759850999) bank41490 +41491 POINT(39.726206304803334 74.74412522808554) bank41491 +41492 POINT(40.72349377925974 73.63508364427545) bank41492 +41493 POINT(41.329599998868794 73.69941120196451) bank41493 +41494 POINT(40.55329249110552 74.58257912642661) bank41494 +41495 POINT(40.2703452600706 74.36736893717482) bank41495 +41496 POINT(40.94222963339743 74.09759806988862) bank41496 +41497 POINT(41.52105061868865 74.58717220010497) bank41497 +41498 POINT(41.624463604221965 74.35218692195596) bank41498 +41499 POINT(41.600687718987956 73.69385898829009) bank41499 +41500 POINT(41.167317318828125 73.05808027539231) bank41500 +41501 POINT(41.69222274187732 73.27874776401985) bank41501 +41502 POINT(41.621263192859416 74.77909186500719) bank41502 +41503 POINT(40.37649456982174 74.61254480961779) bank41503 +41504 POINT(40.28896980650271 74.1991934980588) bank41504 +41505 POINT(41.355437619092754 73.74280958283894) bank41505 +41506 POINT(40.34392303754862 73.1540903321257) bank41506 +41507 POINT(41.28360979059623 73.02417440810792) bank41507 +41508 POINT(41.048118593787756 74.22389375005328) bank41508 +41509 POINT(40.8359513665666 73.61837761159714) bank41509 +41510 POINT(40.19175067201677 73.07162769246398) bank41510 +41511 POINT(40.054376315056416 74.09390563911741) bank41511 +41512 POINT(40.75897622215793 73.820822484312) bank41512 +41513 POINT(41.672737072413554 73.85943186304384) bank41513 +41514 POINT(40.295072625427636 73.52650574160442) bank41514 +41515 POINT(39.82405823768455 74.92092522613761) bank41515 +41516 POINT(40.34178745624404 74.47262153973116) bank41516 +41517 POINT(41.117106259681286 73.76417568126193) bank41517 +41518 POINT(41.70724776547915 74.18297762049387) bank41518 +41519 POINT(40.60915734012698 73.63897061644062) bank41519 +41520 POINT(39.86006455621743 74.49085250187892) bank41520 +41521 POINT(40.891613373334486 74.01272990732889) bank41521 +41522 POINT(41.01164894423731 74.11484607182862) bank41522 +41523 POINT(39.77704737258585 73.13805880572579) bank41523 +41524 POINT(40.74755662227073 73.62001482168557) bank41524 +41525 POINT(40.86046184035516 73.12570585231727) bank41525 +41526 POINT(41.08572581425697 73.62643532391694) bank41526 +41527 POINT(41.19118583578247 73.40305942695825) bank41527 +41528 POINT(41.48392914743783 73.95806005572439) bank41528 +41529 POINT(40.027116598402834 73.39113601247097) bank41529 +41530 POINT(40.8885973374943 73.88457789242783) bank41530 +41531 POINT(40.45742513627723 74.09930155937506) bank41531 +41532 POINT(41.513717293612366 74.35544194570306) bank41532 +41533 POINT(39.95971424802157 73.55474236459752) bank41533 +41534 POINT(40.00952606552776 74.88420827915266) bank41534 +41535 POINT(39.95415641242017 74.11712595689353) bank41535 +41536 POINT(40.80887159033076 74.44271726213198) bank41536 +41537 POINT(40.61632858640665 73.98945197783884) bank41537 +41538 POINT(40.16897645254651 73.99399345420075) bank41538 +41539 POINT(41.37749463264225 73.0452884606882) bank41539 +41540 POINT(39.933849655054324 73.48053846672123) bank41540 +41541 POINT(41.09184306473776 73.80892975938062) bank41541 +41542 POINT(39.71952381216235 74.15397649989585) bank41542 +41543 POINT(41.17762667548391 73.65108014480109) bank41543 +41544 POINT(40.96693557733198 73.11918689474827) bank41544 +41545 POINT(41.244212892946955 73.09079293482345) bank41545 +41546 POINT(40.70732192335816 74.3714176026801) bank41546 +41547 POINT(40.285342638671395 74.76801506199843) bank41547 +41548 POINT(40.999730502734856 73.68576042686945) bank41548 +41549 POINT(41.076113606937106 74.35054893877873) bank41549 +41550 POINT(41.05411433950878 73.62654116692617) bank41550 +41551 POINT(41.27813531840884 74.71397638685033) bank41551 +41552 POINT(39.75253526978086 74.15974834101308) bank41552 +41553 POINT(40.92520721123504 73.50888537237336) bank41553 +41554 POINT(40.673847432968934 74.27935069172523) bank41554 +41555 POINT(41.45288444544442 74.20749786215693) bank41555 +41556 POINT(41.02534420132795 74.41436982095327) bank41556 +41557 POINT(41.4616369024976 74.61386911880861) bank41557 +41558 POINT(40.130705241221925 73.33450930016313) bank41558 +41559 POINT(41.07758365167394 74.53176391212884) bank41559 +41560 POINT(40.218675357687474 74.30251673780873) bank41560 +41561 POINT(40.968019136849 74.4462451719092) bank41561 +41562 POINT(41.611265754139005 74.77996291334429) bank41562 +41563 POINT(40.36231521808152 73.43524524604533) bank41563 +41564 POINT(40.76828673285316 74.6696501651168) bank41564 +41565 POINT(40.29596328285018 73.76156439448312) bank41565 +41566 POINT(40.66690927785885 73.74152839669954) bank41566 +41567 POINT(41.26584901653895 74.63789035370502) bank41567 +41568 POINT(41.00087652392266 73.83408668331136) bank41568 +41569 POINT(40.49243142754184 74.93797793681429) bank41569 +41570 POINT(40.41806664207268 73.63661864003858) bank41570 +41571 POINT(40.94648759200584 73.70972467089315) bank41571 +41572 POINT(40.10678665124354 74.22615721720172) bank41572 +41573 POINT(40.00099780669678 74.42649059343928) bank41573 +41574 POINT(39.82824998169058 74.26597732017586) bank41574 +41575 POINT(41.558164741759875 73.95157124604138) bank41575 +41576 POINT(41.479351083207355 74.70792724437158) bank41576 +41577 POINT(41.250630737804066 73.28180711768603) bank41577 +41578 POINT(41.40934623067262 73.18964445989216) bank41578 +41579 POINT(41.2039644854387 73.15626365988487) bank41579 +41580 POINT(40.219143661608484 73.69097433703398) bank41580 +41581 POINT(41.07855838584807 74.91646023671132) bank41581 +41582 POINT(41.40093510638222 74.60186739122042) bank41582 +41583 POINT(39.882323614493714 74.80575229650877) bank41583 +41584 POINT(40.303561362845834 73.32118089411813) bank41584 +41585 POINT(40.17921394676953 74.40161752517963) bank41585 +41586 POINT(40.91653629603355 74.01165632340793) bank41586 +41587 POINT(41.28918601092355 74.44883069314074) bank41587 +41588 POINT(40.3838080852678 74.38861737743886) bank41588 +41589 POINT(41.130697965687176 73.96726107302136) bank41589 +41590 POINT(39.97622225264517 73.30014884091416) bank41590 +41591 POINT(41.57974748004326 73.77642523928168) bank41591 +41592 POINT(41.12537808818745 73.66353875574042) bank41592 +41593 POINT(40.73694084504089 73.33257845850024) bank41593 +41594 POINT(41.072253863688296 73.47516381378807) bank41594 +41595 POINT(40.5135621841157 73.29159190385847) bank41595 +41596 POINT(40.89032217596538 73.30653660260882) bank41596 +41597 POINT(39.95928804804073 74.79893799065744) bank41597 +41598 POINT(40.88638398980156 73.78151424480374) bank41598 +41599 POINT(40.223863870636876 74.09336934635725) bank41599 +41600 POINT(40.51983769211945 74.44122589354501) bank41600 +41601 POINT(40.63382314427867 74.1920340478164) bank41601 +41602 POINT(40.509371928912415 73.00727897711019) bank41602 +41603 POINT(40.57715355969522 74.17211958042657) bank41603 +41604 POINT(39.7903668455912 74.8784487024584) bank41604 +41605 POINT(39.896014299196395 74.7649524115847) bank41605 +41606 POINT(40.13064245347149 74.0976472836424) bank41606 +41607 POINT(39.93519750965034 74.64377495933077) bank41607 +41608 POINT(40.36558758935884 74.75996865231618) bank41608 +41609 POINT(40.97373076400799 74.59733209586875) bank41609 +41610 POINT(40.38022127028516 74.42185584071908) bank41610 +41611 POINT(40.54106357407093 73.36047282204292) bank41611 +41612 POINT(39.77279134700622 74.21066755628996) bank41612 +41613 POINT(40.445898571069925 74.87577970876872) bank41613 +41614 POINT(41.460036984786655 73.26038258714372) bank41614 +41615 POINT(40.73287108167693 74.3964087331154) bank41615 +41616 POINT(40.75370439591889 74.8265272705761) bank41616 +41617 POINT(40.291083901553165 74.56034973928452) bank41617 +41618 POINT(40.77954068318542 73.77597638176859) bank41618 +41619 POINT(40.1297809895584 74.13187527066768) bank41619 +41620 POINT(40.63855968789589 73.02807358353705) bank41620 +41621 POINT(41.53228782578206 73.87546095757683) bank41621 +41622 POINT(41.65992384646925 73.76001161931991) bank41622 +41623 POINT(40.61289409989874 73.48828194521334) bank41623 +41624 POINT(40.16486194627362 73.78903667872892) bank41624 +41625 POINT(41.64507416935748 73.03982251446574) bank41625 +41626 POINT(41.02464184742945 74.5236284248426) bank41626 +41627 POINT(40.1917240488742 73.13369274068592) bank41627 +41628 POINT(41.256175268688395 74.71241219329843) bank41628 +41629 POINT(40.4280467818966 73.28349028894141) bank41629 +41630 POINT(40.62137257341896 73.33735142268186) bank41630 +41631 POINT(41.42696043911606 73.15668690765412) bank41631 +41632 POINT(41.48070704557083 74.40960208556649) bank41632 +41633 POINT(41.48075497690354 74.57967161714981) bank41633 +41634 POINT(40.371214356534296 74.1786312011013) bank41634 +41635 POINT(40.58509113886909 74.17448943602841) bank41635 +41636 POINT(41.161120064970895 73.93630325908858) bank41636 +41637 POINT(39.923187455926005 74.40287093506798) bank41637 +41638 POINT(40.31033858549897 73.24466158107117) bank41638 +41639 POINT(41.04891970214402 74.46372318868119) bank41639 +41640 POINT(40.84000586403606 74.15270505135435) bank41640 +41641 POINT(40.14427845648052 74.46928180320991) bank41641 +41642 POINT(39.74463553805987 74.77719980134982) bank41642 +41643 POINT(40.8198511933336 73.12132349982055) bank41643 +41644 POINT(40.593814086424466 74.81128449412346) bank41644 +41645 POINT(39.751817199044154 73.10947172321451) bank41645 +41646 POINT(40.81042112823283 73.09488464413083) bank41646 +41647 POINT(40.15089738201372 73.36532032991867) bank41647 +41648 POINT(40.733054896963765 73.20079206539248) bank41648 +41649 POINT(40.090267010015566 74.5933315523771) bank41649 +41650 POINT(40.35542258249506 74.98806161727258) bank41650 +41651 POINT(40.930127443772406 74.38635375519955) bank41651 +41652 POINT(41.52825341203964 74.48168161947031) bank41652 +41653 POINT(41.27277495891523 73.78805697573067) bank41653 +41654 POINT(40.28489804929146 74.98346309592691) bank41654 +41655 POINT(40.25120731161184 73.40860799509203) bank41655 +41656 POINT(40.87309928452764 73.48045025368229) bank41656 +41657 POINT(41.10456805653526 74.28990472744685) bank41657 +41658 POINT(40.600876122358514 73.3972886294732) bank41658 +41659 POINT(40.995149075979 73.8337701765319) bank41659 +41660 POINT(40.34386838244745 74.99462463847215) bank41660 +41661 POINT(41.204346803615415 74.27014394959292) bank41661 +41662 POINT(40.91267448824222 74.3482247312757) bank41662 +41663 POINT(39.750864093079656 74.01490596297505) bank41663 +41664 POINT(41.42584574335794 74.40931453831179) bank41664 +41665 POINT(41.53776415369425 74.99451689656091) bank41665 +41666 POINT(41.54175420645783 73.55135607915686) bank41666 +41667 POINT(40.1894046530305 74.51664128946642) bank41667 +41668 POINT(40.3913595414655 73.04160923412715) bank41668 +41669 POINT(41.22645545778612 74.20902810027805) bank41669 +41670 POINT(41.28306683677036 74.33293753643646) bank41670 +41671 POINT(41.17401878980458 74.49213871621828) bank41671 +41672 POINT(41.69923314544704 73.68320652165472) bank41672 +41673 POINT(41.23919224227712 74.13739931699168) bank41673 +41674 POINT(41.03024797570884 73.19253867043462) bank41674 +41675 POINT(40.77145922582557 75.00402511045968) bank41675 +41676 POINT(40.12170646789314 74.68057336092612) bank41676 +41677 POINT(40.13552003815039 74.02545916886193) bank41677 +41678 POINT(40.176376898783865 74.05437318631895) bank41678 +41679 POINT(40.9460131025481 73.60969977212716) bank41679 +41680 POINT(41.42280040844262 73.0246041606471) bank41680 +41681 POINT(39.97610274511021 73.96140184359749) bank41681 +41682 POINT(40.018489956990585 74.65316789822513) bank41682 +41683 POINT(41.527496230639244 73.10986743055626) bank41683 +41684 POINT(41.431707119867156 73.50887234000311) bank41684 +41685 POINT(40.591278891305315 73.34397527766117) bank41685 +41686 POINT(41.20403594860752 73.00706994613755) bank41686 +41687 POINT(40.41725907096614 73.75367755957362) bank41687 +41688 POINT(41.14309748791353 73.84814852512808) bank41688 +41689 POINT(41.497936398290896 74.48643351473608) bank41689 +41690 POINT(40.804807350335615 73.67187852459429) bank41690 +41691 POINT(41.32990779455983 74.91848242845171) bank41691 +41692 POINT(40.49991956376424 74.05020809711058) bank41692 +41693 POINT(40.223044205745 73.3758937365617) bank41693 +41694 POINT(40.87841880069479 73.60863329588403) bank41694 +41695 POINT(40.75546716495295 74.3241377968917) bank41695 +41696 POINT(40.69140559725823 74.84987352941741) bank41696 +41697 POINT(39.88446946459427 73.78746793858147) bank41697 +41698 POINT(40.71376480220131 74.57109705491622) bank41698 +41699 POINT(41.27209057588097 73.3808358882984) bank41699 +41700 POINT(41.148227088548104 73.48633201469596) bank41700 +41701 POINT(40.902640311921594 73.17402032871765) bank41701 +41702 POINT(40.01556574706375 74.64122874986988) bank41702 +41703 POINT(39.96443225088682 74.57642976811968) bank41703 +41704 POINT(40.161945863249635 74.85613756500688) bank41704 +41705 POINT(41.70992134253214 73.14763217731043) bank41705 +41706 POINT(41.120539532060796 73.63999042939155) bank41706 +41707 POINT(41.432723539436104 73.85498540411824) bank41707 +41708 POINT(40.38259119562613 74.00187699594187) bank41708 +41709 POINT(40.746988333472615 74.96708048551457) bank41709 +41710 POINT(40.61407940388208 73.2859077158421) bank41710 +41711 POINT(41.51447620585121 74.81317983685094) bank41711 +41712 POINT(41.20316661568732 73.30656820243283) bank41712 +41713 POINT(41.16942901364615 73.34006739801109) bank41713 +41714 POINT(41.705704892868205 74.80806672173931) bank41714 +41715 POINT(41.594630844319894 74.8840832971937) bank41715 +41716 POINT(40.35279713501743 73.51534956772991) bank41716 +41717 POINT(39.71359876037891 73.44850462330473) bank41717 +41718 POINT(41.68594128058213 74.07076105987278) bank41718 +41719 POINT(41.39500636484043 73.4647779445981) bank41719 +41720 POINT(41.26793765408301 74.99307045350744) bank41720 +41721 POINT(40.27907423149887 74.5975307883419) bank41721 +41722 POINT(41.29644386340544 74.43723215225957) bank41722 +41723 POINT(39.93706410261182 74.45479575427458) bank41723 +41724 POINT(40.41335506057907 73.79646690427192) bank41724 +41725 POINT(39.79051870240612 74.48038419540275) bank41725 +41726 POINT(40.09690764413651 74.47867782829113) bank41726 +41727 POINT(39.98670433943082 73.81027461664578) bank41727 +41728 POINT(39.86220144419106 73.28712651189619) bank41728 +41729 POINT(40.48131825527032 73.20015952933788) bank41729 +41730 POINT(41.293621565525086 73.5032473378454) bank41730 +41731 POINT(40.03072524696296 73.43209705884112) bank41731 +41732 POINT(39.86764525116174 74.33159821738846) bank41732 +41733 POINT(41.156681384844156 74.21826950598292) bank41733 +41734 POINT(40.70801301429997 74.51625253726922) bank41734 +41735 POINT(40.33365546773578 74.89854116729876) bank41735 +41736 POINT(40.46176517319724 74.95794129839982) bank41736 +41737 POINT(40.96690345108068 74.46000323527909) bank41737 +41738 POINT(41.33294358899731 73.3374736524197) bank41738 +41739 POINT(40.994800374715894 73.91334413606003) bank41739 +41740 POINT(41.144270922740965 74.47865760562004) bank41740 +41741 POINT(40.95583173730822 73.10937187864724) bank41741 +41742 POINT(41.4491147532136 74.20483491115304) bank41742 +41743 POINT(41.031151924730445 73.25619302281922) bank41743 +41744 POINT(41.12915821056789 74.9842555240744) bank41744 +41745 POINT(41.69396828155229 73.8403189048618) bank41745 +41746 POINT(39.91583445826321 73.8093941618618) bank41746 +41747 POINT(41.13335618478056 74.37245131154937) bank41747 +41748 POINT(40.852383709424736 74.59788457531603) bank41748 +41749 POINT(39.77673509784452 74.88136653324563) bank41749 +41750 POINT(40.82048688113344 74.39182308006858) bank41750 +41751 POINT(41.70850075035678 73.16018046051963) bank41751 +41752 POINT(39.94718342981795 73.04921316042758) bank41752 +41753 POINT(39.76619854534978 73.12171509548475) bank41753 +41754 POINT(40.68436701276077 74.10557309358309) bank41754 +41755 POINT(41.5930377636766 74.61304855991862) bank41755 +41756 POINT(41.31703277399213 73.17655287286591) bank41756 +41757 POINT(39.81512981842936 74.88336205956516) bank41757 +41758 POINT(41.230838119150484 73.82016390914244) bank41758 +41759 POINT(41.52761711881995 73.2816064043563) bank41759 +41760 POINT(41.47589102131827 73.14749760063351) bank41760 +41761 POINT(40.85465459923982 73.20241132503324) bank41761 +41762 POINT(41.26647096713229 73.37499487717027) bank41762 +41763 POINT(40.19267395535304 73.48119494517216) bank41763 +41764 POINT(40.76665006209775 73.06127168688292) bank41764 +41765 POINT(40.70761147863682 74.2901465436283) bank41765 +41766 POINT(40.070204002940706 73.15630069502919) bank41766 +41767 POINT(41.17591923997323 73.42803840895073) bank41767 +41768 POINT(40.83922572162249 73.62297493777619) bank41768 +41769 POINT(40.807450737026905 73.51501962035825) bank41769 +41770 POINT(40.34481763517305 74.15308247925536) bank41770 +41771 POINT(40.19834974498145 74.18320992293826) bank41771 +41772 POINT(40.36877200840696 74.5235386973076) bank41772 +41773 POINT(39.90916025272406 74.82256320361849) bank41773 +41774 POINT(40.955633422219435 74.1814235895647) bank41774 +41775 POINT(40.72270597609654 73.26832074243393) bank41775 +41776 POINT(40.29980432581999 74.02700303836579) bank41776 +41777 POINT(40.87174738781725 73.38050880246273) bank41777 +41778 POINT(39.78972267887468 74.30396706240204) bank41778 +41779 POINT(40.0029741394588 73.9724566204594) bank41779 +41780 POINT(41.57728862950385 74.72000495950006) bank41780 +41781 POINT(40.04987093824022 74.54850840852426) bank41781 +41782 POINT(39.91008734196659 74.82980365902469) bank41782 +41783 POINT(40.13129636517207 74.55426458117752) bank41783 +41784 POINT(40.439892212630276 73.8753033001976) bank41784 +41785 POINT(40.82159084573522 73.8575421182899) bank41785 +41786 POINT(41.165094743139555 74.80710528512346) bank41786 +41787 POINT(40.37813698289105 74.824333258926) bank41787 +41788 POINT(39.99561976680469 74.41825051379692) bank41788 +41789 POINT(41.65651214624202 73.33415766742925) bank41789 +41790 POINT(40.91502924406581 73.19602751955595) bank41790 +41791 POINT(40.50504457718314 73.00675732050418) bank41791 +41792 POINT(40.79074821146561 74.72490938707573) bank41792 +41793 POINT(40.07719529137184 73.04034606838006) bank41793 +41794 POINT(40.34170501798724 73.93020146769909) bank41794 +41795 POINT(39.88779033100558 73.84262359976157) bank41795 +41796 POINT(40.203278606598815 73.83344980110377) bank41796 +41797 POINT(40.756504816734186 74.60201452884883) bank41797 +41798 POINT(41.189951118918025 73.8015694365407) bank41798 +41799 POINT(40.362105379899454 73.01118301142387) bank41799 +41800 POINT(41.65788216321324 73.6850249538268) bank41800 +41801 POINT(40.000649234852546 74.99681925347045) bank41801 +41802 POINT(39.91680392900926 74.19315162785125) bank41802 +41803 POINT(40.65094143674681 73.95735718824604) bank41803 +41804 POINT(40.254438990933636 73.14312402678661) bank41804 +41805 POINT(41.48640804459156 74.54660184771053) bank41805 +41806 POINT(41.33248381113121 73.25732263516667) bank41806 +41807 POINT(40.60028639685493 73.1824144510064) bank41807 +41808 POINT(39.7941530991393 74.55303999809466) bank41808 +41809 POINT(40.11753885311966 73.63686357137544) bank41809 +41810 POINT(41.09507581202785 73.47534010706804) bank41810 +41811 POINT(41.56019672663829 73.41813327414724) bank41811 +41812 POINT(39.97094427839021 74.52870383355997) bank41812 +41813 POINT(41.280630365740244 74.99320450231286) bank41813 +41814 POINT(40.34809772782485 74.4138009506) bank41814 +41815 POINT(41.119440058421965 74.24786894535339) bank41815 +41816 POINT(39.762662252710626 74.11583021567193) bank41816 +41817 POINT(40.145944398569114 73.69815248283689) bank41817 +41818 POINT(40.115536282641024 73.94039679607687) bank41818 +41819 POINT(40.740587375103544 74.27091457887357) bank41819 +41820 POINT(39.74327582008987 73.81948467917125) bank41820 +41821 POINT(40.607958366806216 73.69073515105512) bank41821 +41822 POINT(41.02060319873825 74.28704076974685) bank41822 +41823 POINT(40.41922181671021 74.50960605810013) bank41823 +41824 POINT(41.124290697272286 73.15772366599563) bank41824 +41825 POINT(41.00387314938348 74.36183260556267) bank41825 +41826 POINT(39.84147640498282 74.37699096372324) bank41826 +41827 POINT(40.79963340285331 74.25555540794092) bank41827 +41828 POINT(40.27100260582319 74.75732868996309) bank41828 +41829 POINT(39.79604492630577 73.9973317951444) bank41829 +41830 POINT(41.07923122377499 74.10948356821939) bank41830 +41831 POINT(41.5628907887546 74.64758752448256) bank41831 +41832 POINT(39.78092333411759 73.70558158168274) bank41832 +41833 POINT(41.02141341112603 74.98647266925148) bank41833 +41834 POINT(40.63080823289357 74.82983907177622) bank41834 +41835 POINT(41.04639029254039 73.78044756330138) bank41835 +41836 POINT(41.283175802229295 73.61531641062686) bank41836 +41837 POINT(40.103781231975624 74.55513848194188) bank41837 +41838 POINT(39.804218657835364 74.61756262412688) bank41838 +41839 POINT(40.08197387371983 74.6591774811502) bank41839 +41840 POINT(40.08455829843257 73.68660824713344) bank41840 +41841 POINT(41.3848312327809 74.19755982401057) bank41841 +41842 POINT(41.6493345501832 74.87263427441543) bank41842 +41843 POINT(41.648783625699785 73.37616190212157) bank41843 +41844 POINT(41.67252723615507 73.27883988423146) bank41844 +41845 POINT(41.598594792214335 74.28337789697439) bank41845 +41846 POINT(40.848904847236994 73.1400348630165) bank41846 +41847 POINT(39.90060817480979 73.44318565397064) bank41847 +41848 POINT(41.07325541378304 73.34504585348958) bank41848 +41849 POINT(40.2106895619478 74.64355910091729) bank41849 +41850 POINT(40.69528957472894 74.46780829961857) bank41850 +41851 POINT(41.469087402939024 74.95343686733158) bank41851 +41852 POINT(40.98937131037749 73.49711756135623) bank41852 +41853 POINT(39.94459703486611 74.56026594436851) bank41853 +41854 POINT(41.289075907646726 74.46122093506796) bank41854 +41855 POINT(40.607467170240916 74.596114990115) bank41855 +41856 POINT(39.72978263716281 73.43966838101696) bank41856 +41857 POINT(39.772853298957195 74.07867382061977) bank41857 +41858 POINT(41.52037773433891 73.74692181197724) bank41858 +41859 POINT(41.546592367186385 73.36567328889458) bank41859 +41860 POINT(41.06372153424923 73.48417758863968) bank41860 +41861 POINT(39.87622463577667 74.26796139856691) bank41861 +41862 POINT(39.79507625497022 73.30143701035901) bank41862 +41863 POINT(41.0375218944126 74.27798486380874) bank41863 +41864 POINT(41.70682440608968 74.30033037289313) bank41864 +41865 POINT(41.272337998591105 74.27775201012521) bank41865 +41866 POINT(40.7390672781707 74.04070584760606) bank41866 +41867 POINT(40.28257690051361 74.29565391391138) bank41867 +41868 POINT(40.40744665451941 73.26600130361558) bank41868 +41869 POINT(41.21389656404523 73.7626263340597) bank41869 +41870 POINT(39.97075875700066 73.48360658394276) bank41870 +41871 POINT(41.102874584571886 73.80038134304128) bank41871 +41872 POINT(40.18174903692034 74.0161518149595) bank41872 +41873 POINT(41.059748747292474 74.64428563039671) bank41873 +41874 POINT(41.58765181340252 73.9757100371583) bank41874 +41875 POINT(41.6928517294006 73.98786817016412) bank41875 +41876 POINT(40.20065813498975 73.40532578182895) bank41876 +41877 POINT(40.87932277799647 73.53385198870522) bank41877 +41878 POINT(41.564368688986896 73.50197061499038) bank41878 +41879 POINT(41.693459908217086 74.88684972768836) bank41879 +41880 POINT(40.556462541655094 73.9902037809063) bank41880 +41881 POINT(39.93377241499345 74.85948997859339) bank41881 +41882 POINT(41.207608428585075 74.79714847924417) bank41882 +41883 POINT(40.06911480306026 74.54183085329288) bank41883 +41884 POINT(40.3541241718759 74.8659892343097) bank41884 +41885 POINT(41.147361551201875 73.23087476131622) bank41885 +41886 POINT(40.347671265548705 73.67022765181638) bank41886 +41887 POINT(41.49417730934005 73.96239105446482) bank41887 +41888 POINT(41.32524843461906 74.99814873188748) bank41888 +41889 POINT(40.70583643399329 74.6058689162639) bank41889 +41890 POINT(41.04795458214307 73.02857108816514) bank41890 +41891 POINT(40.16314197497226 74.17126296790859) bank41891 +41892 POINT(40.07985433869983 74.83823191591986) bank41892 +41893 POINT(40.52843223941838 74.68718225462372) bank41893 +41894 POINT(40.318709553243195 73.81995237362737) bank41894 +41895 POINT(40.267802878735864 74.21731172509887) bank41895 +41896 POINT(41.55595841930873 74.2401763508089) bank41896 +41897 POINT(41.19649373122276 73.82620911201401) bank41897 +41898 POINT(41.052740881422196 74.5621682947871) bank41898 +41899 POINT(40.60596304707115 74.68437497349436) bank41899 +41900 POINT(40.79483569566386 73.82089194154875) bank41900 +41901 POINT(40.87449162894112 73.84926439627971) bank41901 +41902 POINT(41.43945284382614 73.93983021568592) bank41902 +41903 POINT(40.97970502121493 73.26202911741898) bank41903 +41904 POINT(41.10284359347189 74.119844746636) bank41904 +41905 POINT(39.718882967568746 74.84985105794006) bank41905 +41906 POINT(41.02173315757043 74.30194383463308) bank41906 +41907 POINT(39.71474207865276 74.30677368926764) bank41907 +41908 POINT(40.72209971566279 74.12494282287116) bank41908 +41909 POINT(41.24962631056979 74.99475320882803) bank41909 +41910 POINT(41.68502927868475 74.85765061429369) bank41910 +41911 POINT(40.908230470698065 73.53294683860094) bank41911 +41912 POINT(40.36653169140535 74.81362532664852) bank41912 +41913 POINT(40.23863003824109 74.94243022672433) bank41913 +41914 POINT(41.577836938065616 73.2254566986207) bank41914 +41915 POINT(40.02182281431715 74.83304248784519) bank41915 +41916 POINT(40.00710167318175 74.3262393166862) bank41916 +41917 POINT(41.673592136236365 74.37484212824015) bank41917 +41918 POINT(41.02480999495779 73.65252284573555) bank41918 +41919 POINT(41.380624637070426 73.05810909411198) bank41919 +41920 POINT(39.765123247895026 73.93959876793512) bank41920 +41921 POINT(40.96524851163102 73.90915334669592) bank41921 +41922 POINT(40.34624459992396 73.63041398159523) bank41922 +41923 POINT(40.6877725257388 73.08597557158245) bank41923 +41924 POINT(40.65979657681793 73.78628185219755) bank41924 +41925 POINT(40.99380057744528 73.29437135969455) bank41925 +41926 POINT(40.78740281324069 73.22519141876506) bank41926 +41927 POINT(40.25855861188889 74.0930673513181) bank41927 +41928 POINT(39.885016393388135 73.81083049957817) bank41928 +41929 POINT(40.293832125104664 73.42323178832513) bank41929 +41930 POINT(41.00293306899439 73.34145678737342) bank41930 +41931 POINT(41.330757060288036 74.86360929769968) bank41931 +41932 POINT(40.30694852629194 74.37529308216149) bank41932 +41933 POINT(41.20137864171945 73.35692125902062) bank41933 +41934 POINT(39.909058621273026 73.57018786663879) bank41934 +41935 POINT(41.6737555896361 74.82642876194527) bank41935 +41936 POINT(41.251224675468904 73.3300962611742) bank41936 +41937 POINT(40.15466544029741 74.82462635559025) bank41937 +41938 POINT(40.5929199915885 73.57750426708571) bank41938 +41939 POINT(39.93176660454016 74.76440227625733) bank41939 +41940 POINT(40.952691299827606 73.60235557514876) bank41940 +41941 POINT(41.12095777121785 73.39671416101329) bank41941 +41942 POINT(40.298167156862824 73.391528018796) bank41942 +41943 POINT(39.9540186418183 74.16038895380699) bank41943 +41944 POINT(41.70310260886491 73.72333060500745) bank41944 +41945 POINT(41.135493171031825 74.15564483550106) bank41945 +41946 POINT(40.44554391469095 73.8469664025411) bank41946 +41947 POINT(40.51230114770209 73.19147765134416) bank41947 +41948 POINT(41.45727874679457 73.14007477963366) bank41948 +41949 POINT(41.23214173401036 73.10309735960222) bank41949 +41950 POINT(39.989321857612076 74.49564496654018) bank41950 +41951 POINT(40.26430437659563 73.73806347777955) bank41951 +41952 POINT(41.345589031794674 73.42049689683874) bank41952 +41953 POINT(41.23928906836138 74.76186403320608) bank41953 +41954 POINT(40.61319197980136 73.20574893987794) bank41954 +41955 POINT(40.89703953871178 74.64639926895119) bank41955 +41956 POINT(40.74803768533243 74.59243135169307) bank41956 +41957 POINT(40.13641669973492 73.45103284623147) bank41957 +41958 POINT(40.096780755329846 74.48367033090295) bank41958 +41959 POINT(41.1157405300999 73.0386449465715) bank41959 +41960 POINT(41.50578695514369 74.69544765652513) bank41960 +41961 POINT(40.74394959316419 74.18636489176261) bank41961 +41962 POINT(40.32502078199063 73.7284817118299) bank41962 +41963 POINT(39.97485131614487 74.37279821121057) bank41963 +41964 POINT(40.32449377924535 73.98800178184679) bank41964 +41965 POINT(40.00980298063244 73.23157702505716) bank41965 +41966 POINT(40.58325520289524 74.68562224468263) bank41966 +41967 POINT(41.62440662625289 74.25881628915194) bank41967 +41968 POINT(39.82172925048585 73.47107007259152) bank41968 +41969 POINT(40.01011658934295 74.87154325215512) bank41969 +41970 POINT(41.395208718719914 73.45266312543274) bank41970 +41971 POINT(40.593828670512046 74.83652909780864) bank41971 +41972 POINT(40.708253531037 74.62516281783493) bank41972 +41973 POINT(40.69796918186236 73.09294733988101) bank41973 +41974 POINT(41.563386611100285 74.64968820357502) bank41974 +41975 POINT(41.15570282276898 73.82673465212065) bank41975 +41976 POINT(40.74445417395704 74.19112570465666) bank41976 +41977 POINT(41.364405552368865 74.36065104670568) bank41977 +41978 POINT(39.93994617172637 74.46182493467347) bank41978 +41979 POINT(39.879793566045144 73.86278464210615) bank41979 +41980 POINT(40.397344789708875 74.91875788580349) bank41980 +41981 POINT(39.90044317087196 73.85963921350778) bank41981 +41982 POINT(40.90268350934428 74.24939246938608) bank41982 +41983 POINT(40.37031929438845 73.27831905087851) bank41983 +41984 POINT(40.384477354212606 73.16231065577652) bank41984 +41985 POINT(40.74453111917824 73.34550809796329) bank41985 +41986 POINT(40.52252564098388 73.6326467305085) bank41986 +41987 POINT(41.32400913392713 74.95839676006835) bank41987 +41988 POINT(41.58248317095753 73.93917737433084) bank41988 +41989 POINT(40.24582430553313 73.50749402193283) bank41989 +41990 POINT(41.10342361325396 73.98402127286482) bank41990 +41991 POINT(40.21085273497046 74.99572825305845) bank41991 +41992 POINT(39.75428325466747 74.8829747696576) bank41992 +41993 POINT(40.48828701288334 73.97200548086606) bank41993 +41994 POINT(39.766638022892806 74.58755989385072) bank41994 +41995 POINT(41.04075981147252 73.52299345985988) bank41995 +41996 POINT(39.8645407482899 73.24445089443233) bank41996 +41997 POINT(41.66775600244753 74.3407138447219) bank41997 +41998 POINT(41.25203797064634 73.9264777324811) bank41998 +41999 POINT(40.03549332610702 73.20956198338273) bank41999 +42000 POINT(39.777362730765844 74.07849130221672) bank42000 +42001 POINT(39.74910560531615 73.09310887002528) bank42001 +42002 POINT(40.34664782988738 74.21784532785972) bank42002 +42003 POINT(41.10351097293794 73.62201071716916) bank42003 +42004 POINT(39.86340995111109 74.5328883650604) bank42004 +42005 POINT(40.97718630237499 74.52878816781293) bank42005 +42006 POINT(41.31840176304987 74.18172124875343) bank42006 +42007 POINT(40.29349272853014 74.32902061572769) bank42007 +42008 POINT(40.48825457959816 74.42227438740589) bank42008 +42009 POINT(39.896742366607924 74.43738699261056) bank42009 +42010 POINT(40.44284990487859 74.19356212666176) bank42010 +42011 POINT(40.748742266158516 74.77264439893187) bank42011 +42012 POINT(39.8800675361681 74.17072374822055) bank42012 +42013 POINT(40.37098489333956 74.36508169365494) bank42013 +42014 POINT(41.19667353645719 74.49690101573601) bank42014 +42015 POINT(40.122162290892206 73.01416340923691) bank42015 +42016 POINT(41.302692176738496 74.51652243247663) bank42016 +42017 POINT(40.69789464084703 74.39152914919768) bank42017 +42018 POINT(40.50509643108818 73.44820005616864) bank42018 +42019 POINT(40.22001163552401 74.78257794972285) bank42019 +42020 POINT(40.202117150948396 74.13381894733709) bank42020 +42021 POINT(41.324050204124454 74.24017379100403) bank42021 +42022 POINT(41.13454729945119 73.78055260462496) bank42022 +42023 POINT(40.58357755275696 73.8064494392875) bank42023 +42024 POINT(40.668496668652075 74.2806345428274) bank42024 +42025 POINT(41.03307290962326 73.72033409367906) bank42025 +42026 POINT(40.901493834894055 73.25531695463553) bank42026 +42027 POINT(40.379062995816234 74.15003896775045) bank42027 +42028 POINT(40.613319765459224 73.74512196799309) bank42028 +42029 POINT(40.69751890341414 73.06042173811892) bank42029 +42030 POINT(40.11628558691892 73.16723712738906) bank42030 +42031 POINT(39.93072145529077 73.00716463536834) bank42031 +42032 POINT(39.83969286887397 73.37232072605738) bank42032 +42033 POINT(40.10539560862936 74.56516844591067) bank42033 +42034 POINT(41.41636669866642 74.32851861157398) bank42034 +42035 POINT(41.055502375516895 74.0175091564943) bank42035 +42036 POINT(39.82471869877347 74.68934692485176) bank42036 +42037 POINT(41.415374703370084 73.87853961706908) bank42037 +42038 POINT(41.64235123567079 73.69736665106153) bank42038 +42039 POINT(39.75729833985524 74.32759199428484) bank42039 +42040 POINT(40.78298677589394 74.12898451931662) bank42040 +42041 POINT(41.636576498201926 74.30604127055365) bank42041 +42042 POINT(41.68942771264471 73.53600460639649) bank42042 +42043 POINT(40.25468775042854 74.81565365555169) bank42043 +42044 POINT(41.63557854280247 73.18073017306948) bank42044 +42045 POINT(40.34230860047618 74.99395123042511) bank42045 +42046 POINT(39.82720714402143 74.89354088440768) bank42046 +42047 POINT(40.78075933772148 73.95495501252005) bank42047 +42048 POINT(40.28737236962763 73.7061080767433) bank42048 +42049 POINT(41.38683274650649 74.05417016831589) bank42049 +42050 POINT(41.45896789295637 73.01715408149535) bank42050 +42051 POINT(40.13187891820571 74.06549189588736) bank42051 +42052 POINT(40.86815452468104 73.1088976338368) bank42052 +42053 POINT(41.33532857423367 74.99312802574808) bank42053 +42054 POINT(39.726768720619546 73.04836780248148) bank42054 +42055 POINT(41.12938249653194 74.06007362301065) bank42055 +42056 POINT(39.91200300348673 73.57623521227383) bank42056 +42057 POINT(41.319296092054735 73.44554789912044) bank42057 +42058 POINT(40.94568631152682 73.3228967371231) bank42058 +42059 POINT(41.03829004801917 73.28328685699904) bank42059 +42060 POINT(41.462137482441 73.71328481028662) bank42060 +42061 POINT(41.15416908679761 74.0973072990507) bank42061 +42062 POINT(41.038287313862014 73.71376059587303) bank42062 +42063 POINT(40.26612733809801 74.62410844480847) bank42063 +42064 POINT(40.2573182317887 73.04757474942089) bank42064 +42065 POINT(40.912654758256586 73.06071991923355) bank42065 +42066 POINT(39.91260063086152 73.80565047108679) bank42066 +42067 POINT(40.954570157235466 73.30724925504846) bank42067 +42068 POINT(41.36144706148415 74.58427650935562) bank42068 +42069 POINT(40.20564768208888 74.27688156565824) bank42069 +42070 POINT(41.42029948972269 73.21670790287958) bank42070 +42071 POINT(39.824205539927085 73.05362193423018) bank42071 +42072 POINT(40.25334587267715 74.60879174305651) bank42072 +42073 POINT(41.49618911516345 73.63397834165904) bank42073 +42074 POINT(41.69637559593465 74.42828112044211) bank42074 +42075 POINT(40.525843566924856 73.62400231318713) bank42075 +42076 POINT(39.86941073730797 74.02043268438597) bank42076 +42077 POINT(41.59439204481361 73.28146328945527) bank42077 +42078 POINT(41.3277770915262 74.41305882488118) bank42078 +42079 POINT(39.92195105489811 73.56653607086558) bank42079 +42080 POINT(41.10217650422173 73.33019598540989) bank42080 +42081 POINT(41.503700749858325 73.38110792445103) bank42081 +42082 POINT(40.03415789539277 73.09118140785402) bank42082 +42083 POINT(41.2517678806094 73.80044834030132) bank42083 +42084 POINT(40.763477007093485 74.48600298172214) bank42084 +42085 POINT(41.63997111581631 73.28317805433545) bank42085 +42086 POINT(40.428972237487116 73.11027034872605) bank42086 +42087 POINT(40.65701822410661 74.65923455975101) bank42087 +42088 POINT(41.28192614334694 73.07986430586271) bank42088 +42089 POINT(39.88204273918842 74.15578169055438) bank42089 +42090 POINT(39.75025686016387 74.0477037715942) bank42090 +42091 POINT(40.219963548870716 73.55090789384221) bank42091 +42092 POINT(41.66867269741666 74.92085469149359) bank42092 +42093 POINT(40.362142167817176 73.43248455874321) bank42093 +42094 POINT(40.93828914205495 74.41445075983884) bank42094 +42095 POINT(40.01339772849028 74.27024786650664) bank42095 +42096 POINT(41.415726218424545 74.10326142938591) bank42096 +42097 POINT(40.980505136607036 73.72945575369225) bank42097 +42098 POINT(40.54811250162287 73.97264009735069) bank42098 +42099 POINT(40.07616854816284 74.45860584622498) bank42099 +42100 POINT(40.9334098706219 74.83223282932265) bank42100 +42101 POINT(40.69869695158993 74.74008361631077) bank42101 +42102 POINT(41.33037204924426 73.37414542962135) bank42102 +42103 POINT(40.50530715133505 74.24635410898087) bank42103 +42104 POINT(41.33597288347075 74.02285818585369) bank42104 +42105 POINT(40.47250048206659 74.40574582018192) bank42105 +42106 POINT(40.42212016752335 74.23793699524855) bank42106 +42107 POINT(41.46672062393203 73.21525525317473) bank42107 +42108 POINT(39.94171586302978 73.81308132470468) bank42108 +42109 POINT(41.526245813286785 74.91206007116602) bank42109 +42110 POINT(40.89358426440229 73.20972224029894) bank42110 +42111 POINT(39.84486311019667 74.5634560656176) bank42111 +42112 POINT(41.494775443442634 74.624924670072) bank42112 +42113 POINT(41.48332031594053 73.44393210667384) bank42113 +42114 POINT(39.945592318392016 73.38026502022112) bank42114 +42115 POINT(39.971976540292125 74.34971355440375) bank42115 +42116 POINT(40.38885663573179 73.54694888109114) bank42116 +42117 POINT(40.72201854897318 74.66565047971685) bank42117 +42118 POINT(41.66867367158414 73.99212441025954) bank42118 +42119 POINT(40.953800169950476 74.87054686230977) bank42119 +42120 POINT(39.71842825160306 73.34107533899137) bank42120 +42121 POINT(40.82790370436318 73.83423292301364) bank42121 +42122 POINT(40.78141528751403 74.22410219240973) bank42122 +42123 POINT(40.73338570557394 73.84790121833952) bank42123 +42124 POINT(39.792108691120326 73.10011729309052) bank42124 +42125 POINT(40.41511627974791 74.43600534759581) bank42125 +42126 POINT(41.17617953417212 73.05986405373169) bank42126 +42127 POINT(41.70954458304244 74.3743389881553) bank42127 +42128 POINT(40.513850525795824 74.48985229391234) bank42128 +42129 POINT(41.24222693923927 74.20474225366608) bank42129 +42130 POINT(40.36332704760173 73.05903579813095) bank42130 +42131 POINT(40.69128747382116 73.9451961243109) bank42131 +42132 POINT(40.689380081915886 74.26272993588033) bank42132 +42133 POINT(41.10962396478203 74.47955776435093) bank42133 +42134 POINT(41.65796893487292 73.76397641217252) bank42134 +42135 POINT(39.71298672263453 74.1780171827221) bank42135 +42136 POINT(41.39759399256502 74.30642685310492) bank42136 +42137 POINT(40.23060104684504 74.36001801723155) bank42137 +42138 POINT(39.868048275942705 74.88987989159318) bank42138 +42139 POINT(40.71722461369702 73.57087796710252) bank42139 +42140 POINT(40.599881679638834 74.36601000538128) bank42140 +42141 POINT(40.95207898457164 73.74852700950788) bank42141 +42142 POINT(41.080690012727 74.70699872338585) bank42142 +42143 POINT(40.85886624899259 73.26480953872641) bank42143 +42144 POINT(39.71686731339259 74.48505313636234) bank42144 +42145 POINT(39.85536772496446 74.95104286808235) bank42145 +42146 POINT(40.698781068999494 73.80367722837612) bank42146 +42147 POINT(40.41292360325075 74.1966168357342) bank42147 +42148 POINT(40.98800069652328 74.2452670788194) bank42148 +42149 POINT(39.87752155996494 73.6061016272496) bank42149 +42150 POINT(40.054632047288315 73.66562310262299) bank42150 +42151 POINT(39.982328083721725 74.16541389116195) bank42151 +42152 POINT(40.328648783040315 74.01270388986305) bank42152 +42153 POINT(40.754597639120064 73.51745395733168) bank42153 +42154 POINT(40.29327308710834 73.60227289097439) bank42154 +42155 POINT(41.618768598767296 73.6497487700943) bank42155 +42156 POINT(39.88997719954297 74.91844863511045) bank42156 +42157 POINT(40.111850989384756 73.88778347261736) bank42157 +42158 POINT(39.84646733285628 74.7783405768353) bank42158 +42159 POINT(40.87574878676844 74.18184309110613) bank42159 +42160 POINT(40.94189760804983 73.06810797333281) bank42160 +42161 POINT(41.49710512542162 74.19803635627173) bank42161 +42162 POINT(39.754497800428595 73.2685055951031) bank42162 +42163 POINT(40.91626967165455 74.5630084799457) bank42163 +42164 POINT(41.3396603424691 74.30409530790563) bank42164 +42165 POINT(39.89821275395611 73.37964532982708) bank42165 +42166 POINT(40.13317266237701 74.89602574755958) bank42166 +42167 POINT(39.79662259310049 74.05198581276606) bank42167 +42168 POINT(39.91830850374324 73.34046746961899) bank42168 +42169 POINT(40.64152659135736 73.35298497713217) bank42169 +42170 POINT(40.428539673332416 73.91163691516464) bank42170 +42171 POINT(41.2247707910496 74.04746700248624) bank42171 +42172 POINT(40.74462550709047 74.17163836077566) bank42172 +42173 POINT(41.00904097519156 73.88642114926458) bank42173 +42174 POINT(40.40475345885699 73.18950142348771) bank42174 +42175 POINT(40.82604791143256 73.31407961661206) bank42175 +42176 POINT(40.99358990106955 73.50450719875182) bank42176 +42177 POINT(41.649074540511805 73.36556518096529) bank42177 +42178 POINT(40.98296119992691 73.7060554884788) bank42178 +42179 POINT(40.788422259601454 73.68276722216824) bank42179 +42180 POINT(40.9362561156121 74.61217521121846) bank42180 +42181 POINT(40.15561342498301 73.55867558571495) bank42181 +42182 POINT(41.12500068692608 73.40723266297195) bank42182 +42183 POINT(41.24068388893553 73.93182798106217) bank42183 +42184 POINT(40.71379779672425 73.48714706431856) bank42184 +42185 POINT(40.01244125008289 73.08785624809606) bank42185 +42186 POINT(40.62335898431816 73.22809636938683) bank42186 +42187 POINT(40.1002966839611 74.43588607133252) bank42187 +42188 POINT(40.16793121280008 73.70412403178636) bank42188 +42189 POINT(40.448084556078 73.5537913241144) bank42189 +42190 POINT(41.24334332252612 73.64795139126262) bank42190 +42191 POINT(39.78328298089662 74.55449594914845) bank42191 +42192 POINT(41.18791375564078 73.69963498765345) bank42192 +42193 POINT(39.76474685847657 74.76815882135334) bank42193 +42194 POINT(40.13930738560553 73.18113184065226) bank42194 +42195 POINT(41.1014936441417 73.68356583413788) bank42195 +42196 POINT(40.157898586012884 75.00475825637938) bank42196 +42197 POINT(39.74781462436442 74.05065523848019) bank42197 +42198 POINT(40.855469251435245 73.20262871532992) bank42198 +42199 POINT(41.24915206381458 73.62534318790699) bank42199 +42200 POINT(41.63730545858747 73.78215693589115) bank42200 +42201 POINT(40.647908585789025 74.1504752241572) bank42201 +42202 POINT(40.398831290153645 74.25988110540145) bank42202 +42203 POINT(40.01778909390061 74.87129500825839) bank42203 +42204 POINT(40.73546654443054 74.35489661673697) bank42204 +42205 POINT(40.820377806869836 74.09763963157909) bank42205 +42206 POINT(40.37732530669399 74.03029233667327) bank42206 +42207 POINT(39.86670164368963 74.40608327960223) bank42207 +42208 POINT(41.67744033371893 73.8003360634335) bank42208 +42209 POINT(40.40982926819943 73.44665324106866) bank42209 +42210 POINT(41.576798229832534 74.58834255700002) bank42210 +42211 POINT(39.739498029769976 74.76661223426626) bank42211 +42212 POINT(40.51025699543528 73.17197855246201) bank42212 +42213 POINT(40.83882801792486 74.33385332975833) bank42213 +42214 POINT(41.54657403842489 74.10182059442761) bank42214 +42215 POINT(39.72052843368216 73.81602872836551) bank42215 +42216 POINT(41.3382535257861 74.75430447133144) bank42216 +42217 POINT(40.78563548394987 74.02105107202946) bank42217 +42218 POINT(41.084197938623376 74.03431520294617) bank42218 +42219 POINT(39.881458109872725 74.38367843490433) bank42219 +42220 POINT(39.98045717716706 73.10840714144557) bank42220 +42221 POINT(40.15860776081156 73.47823978193884) bank42221 +42222 POINT(39.81274090991075 74.1979719596087) bank42222 +42223 POINT(41.32643178880927 74.60419632991942) bank42223 +42224 POINT(41.315573398166265 74.26615802917152) bank42224 +42225 POINT(41.648799132529874 73.77178861572251) bank42225 +42226 POINT(41.499099720828625 74.1884174013016) bank42226 +42227 POINT(41.18442325065601 73.8737293565222) bank42227 +42228 POINT(40.09378322803487 73.97103947978897) bank42228 +42229 POINT(40.67689290025888 73.9014155520749) bank42229 +42230 POINT(41.61567465770404 74.75482064036532) bank42230 +42231 POINT(41.417580647717735 73.84854765219649) bank42231 +42232 POINT(40.845427873664526 74.29110626810564) bank42232 +42233 POINT(41.314936313171216 73.5273571994627) bank42233 +42234 POINT(40.38944645969214 73.15834023279372) bank42234 +42235 POINT(41.32641145050744 74.51972511878041) bank42235 +42236 POINT(39.95378015451652 74.92086123635156) bank42236 +42237 POINT(39.883994840452104 74.66676792933663) bank42237 +42238 POINT(41.275663799229434 74.68621679668547) bank42238 +42239 POINT(40.69963245129327 74.2244226273139) bank42239 +42240 POINT(41.543871947535216 74.03085186463932) bank42240 +42241 POINT(40.807483027356376 74.4049917984191) bank42241 +42242 POINT(40.85151783569451 73.4027178574537) bank42242 +42243 POINT(40.300836809095514 73.21877560284355) bank42243 +42244 POINT(40.72809565312753 73.31118747562252) bank42244 +42245 POINT(40.72896819985057 73.13558757473555) bank42245 +42246 POINT(40.43386815328567 74.35565772155253) bank42246 +42247 POINT(41.48710388421177 74.81293945540929) bank42247 +42248 POINT(41.300114312422686 74.98880131626218) bank42248 +42249 POINT(39.765507846613055 74.15732737702123) bank42249 +42250 POINT(40.92697975331002 74.54802752243403) bank42250 +42251 POINT(41.26444209341812 73.0980124858762) bank42251 +42252 POINT(40.854445432498174 74.15479824187125) bank42252 +42253 POINT(39.753834166975295 74.64302674839061) bank42253 +42254 POINT(41.282492077863466 73.38291615385548) bank42254 +42255 POINT(39.835225777078875 73.12225291102344) bank42255 +42256 POINT(40.788751268521565 73.051220504798) bank42256 +42257 POINT(41.467994104133126 73.79540444292996) bank42257 +42258 POINT(40.97735633106336 74.96954999308791) bank42258 +42259 POINT(39.80831798661519 73.98124109391658) bank42259 +42260 POINT(41.344414513818045 74.74985754233934) bank42260 +42261 POINT(40.576377409504104 73.24621695967404) bank42261 +42262 POINT(41.30619004467582 73.26561804570349) bank42262 +42263 POINT(40.428375756837916 74.38950996314335) bank42263 +42264 POINT(40.29814560604508 73.56251347934551) bank42264 +42265 POINT(39.967541494742854 74.73556338576888) bank42265 +42266 POINT(40.89804666678546 73.75214996253821) bank42266 +42267 POINT(41.169178496376354 74.09234878120228) bank42267 +42268 POINT(41.43211572404321 73.2526036087349) bank42268 +42269 POINT(40.39911462897379 74.79366990608676) bank42269 +42270 POINT(39.976074865349396 73.34073621619223) bank42270 +42271 POINT(40.1443851179998 74.38901855511168) bank42271 +42272 POINT(41.40368514498564 74.59626155309172) bank42272 +42273 POINT(39.77784815484419 73.31015791720993) bank42273 +42274 POINT(41.53822020819314 74.67286830592367) bank42274 +42275 POINT(41.122602667845314 74.20365580544619) bank42275 +42276 POINT(40.51568765063339 74.60232202367303) bank42276 +42277 POINT(39.826831087276524 73.3941686044985) bank42277 +42278 POINT(40.60483613795291 73.57305374247414) bank42278 +42279 POINT(40.9701542619549 73.09658721337225) bank42279 +42280 POINT(40.12486765851673 73.00977315481468) bank42280 +42281 POINT(40.33220448928778 74.42838525325993) bank42281 +42282 POINT(39.94399563669083 74.55476831860341) bank42282 +42283 POINT(41.13976389800709 74.24456744479875) bank42283 +42284 POINT(41.67054034130078 74.75309569840356) bank42284 +42285 POINT(39.71498882933239 73.41947304675985) bank42285 +42286 POINT(39.92493539128564 74.23950506125804) bank42286 +42287 POINT(40.10000128890091 73.18667823172922) bank42287 +42288 POINT(41.5913100615782 73.68096890378749) bank42288 +42289 POINT(41.23000594044786 73.57959413344557) bank42289 +42290 POINT(39.75717289299686 73.60698181096153) bank42290 +42291 POINT(39.74152136200471 73.76837041572341) bank42291 +42292 POINT(41.596301171601525 74.17667002618911) bank42292 +42293 POINT(40.015453375674376 74.68030608318293) bank42293 +42294 POINT(40.566226865656496 74.88293265925388) bank42294 +42295 POINT(40.171771641660854 73.53753749341216) bank42295 +42296 POINT(40.95898552616527 73.24237058436678) bank42296 +42297 POINT(39.9985306621466 73.53114047860515) bank42297 +42298 POINT(40.90796426536359 74.85573271594747) bank42298 +42299 POINT(40.73982149206479 74.31346317242284) bank42299 +42300 POINT(40.72857133561854 73.17592089441148) bank42300 +42301 POINT(41.41184267472042 73.52190741550763) bank42301 +42302 POINT(41.369010660352885 73.49436777006451) bank42302 +42303 POINT(41.599683708693235 74.18183651214417) bank42303 +42304 POINT(40.369675732652205 73.19522012312356) bank42304 +42305 POINT(41.34400294650313 73.05333147783298) bank42305 +42306 POINT(41.17202325538341 74.81713869337726) bank42306 +42307 POINT(41.48265407141015 73.0842875969475) bank42307 +42308 POINT(41.52463460833582 74.11475905062085) bank42308 +42309 POINT(40.82604770519416 74.13517973694253) bank42309 +42310 POINT(40.49555246217678 74.51024168951768) bank42310 +42311 POINT(40.390510604836386 74.9863649260971) bank42311 +42312 POINT(41.01115857028887 73.46273918382792) bank42312 +42313 POINT(41.50528921976743 73.26854137638986) bank42313 +42314 POINT(40.126469617210205 74.6124195597737) bank42314 +42315 POINT(40.76645631086578 74.11810265445874) bank42315 +42316 POINT(40.29523638360703 73.29795127797716) bank42316 +42317 POINT(40.59365817391074 73.93498040390467) bank42317 +42318 POINT(41.64505780510137 73.7182251488688) bank42318 +42319 POINT(40.30017444326673 74.8638267520554) bank42319 +42320 POINT(40.25685044649223 74.50245983609797) bank42320 +42321 POINT(41.35268686746069 73.92889679526877) bank42321 +42322 POINT(40.535944949627876 74.3977240638706) bank42322 +42323 POINT(41.43901125893174 74.08899976924255) bank42323 +42324 POINT(40.180053751962355 74.87270640012945) bank42324 +42325 POINT(41.23361899927156 74.29141732565532) bank42325 +42326 POINT(40.13584810780635 73.31876679613352) bank42326 +42327 POINT(41.37886915785853 74.87581159100826) bank42327 +42328 POINT(40.880362454907754 73.27977620907046) bank42328 +42329 POINT(41.39510319032209 73.71793830387858) bank42329 +42330 POINT(39.84277704115266 74.80606216187084) bank42330 +42331 POINT(40.10830494743471 73.2195300010138) bank42331 +42332 POINT(40.40742576127347 73.4789249317849) bank42332 +42333 POINT(41.238133299328325 73.69591627927194) bank42333 +42334 POINT(41.56156757222165 74.27639336790554) bank42334 +42335 POINT(40.22132297444556 74.81308017887469) bank42335 +42336 POINT(40.955402646298175 73.4348250270285) bank42336 +42337 POINT(40.61059025752362 73.78688577006909) bank42337 +42338 POINT(39.79060474672403 73.11699112021327) bank42338 +42339 POINT(41.387690288105105 74.58155011954321) bank42339 +42340 POINT(40.5008443989732 73.0745895739941) bank42340 +42341 POINT(39.953759443725716 74.45936551421985) bank42341 +42342 POINT(40.87774353375453 73.22131761702923) bank42342 +42343 POINT(40.05291815871649 73.69562251327878) bank42343 +42344 POINT(40.56247924703818 74.63836068207242) bank42344 +42345 POINT(41.27872769662757 74.75252671894886) bank42345 +42346 POINT(40.51478540923764 73.7731072004034) bank42346 +42347 POINT(39.792346240809486 74.52909995843841) bank42347 +42348 POINT(40.792905593730914 73.94116845308739) bank42348 +42349 POINT(40.676821310787716 73.03392628414582) bank42349 +42350 POINT(41.253366589812295 74.78071431747031) bank42350 +42351 POINT(41.47587585179527 74.67376638322548) bank42351 +42352 POINT(40.91533469956633 73.61837348249634) bank42352 +42353 POINT(40.92385628814348 74.09246419352645) bank42353 +42354 POINT(41.30839637181046 74.36920891228199) bank42354 +42355 POINT(41.40966943590857 73.68452032415163) bank42355 +42356 POINT(41.2716584369694 73.64228377916776) bank42356 +42357 POINT(40.0912248213712 73.91198909602795) bank42357 +42358 POINT(40.08166685860678 73.19759973558851) bank42358 +42359 POINT(41.48568665887578 73.37258709610217) bank42359 +42360 POINT(41.388452887470955 73.43457921656098) bank42360 +42361 POINT(40.318097476404986 74.0831560990455) bank42361 +42362 POINT(40.45710237044107 74.20677629002036) bank42362 +42363 POINT(41.261487215079136 74.25231879444503) bank42363 +42364 POINT(40.25904676288565 73.9877074913821) bank42364 +42365 POINT(40.76655488434562 73.99870607254675) bank42365 +42366 POINT(40.158005495292734 73.61097778925033) bank42366 +42367 POINT(40.92741149569625 74.1995187591618) bank42367 +42368 POINT(40.92186631034794 73.8877373980987) bank42368 +42369 POINT(39.9116644020222 74.79126587774867) bank42369 +42370 POINT(41.37763733237031 74.59411234365028) bank42370 +42371 POINT(41.02429463119471 74.6114811810289) bank42371 +42372 POINT(40.20964622850231 74.80963863545287) bank42372 +42373 POINT(41.230544138160106 74.8879441977733) bank42373 +42374 POINT(40.938198323104075 73.52315742708392) bank42374 +42375 POINT(39.77359730804477 73.14106183736925) bank42375 +42376 POINT(39.75361635740194 73.12217560401177) bank42376 +42377 POINT(40.33926869510205 74.88871285326952) bank42377 +42378 POINT(39.76401368210311 74.86659264837678) bank42378 +42379 POINT(40.213288029860585 74.02341951580007) bank42379 +42380 POINT(40.76735378191129 74.50473155441823) bank42380 +42381 POINT(40.0968825364706 74.63709961880652) bank42381 +42382 POINT(41.60753945551767 74.215287282946) bank42382 +42383 POINT(40.6654038111513 74.69455907425014) bank42383 +42384 POINT(41.37116374323761 73.10738372954627) bank42384 +42385 POINT(40.40877628819371 74.24226263312067) bank42385 +42386 POINT(39.88487711334572 74.02455432427942) bank42386 +42387 POINT(40.3277240159267 74.35953478601867) bank42387 +42388 POINT(39.96452142371603 73.31914338962106) bank42388 +42389 POINT(41.6156020423749 74.11524852319246) bank42389 +42390 POINT(41.30290958227696 73.73540358710612) bank42390 +42391 POINT(41.29472654036765 74.39723706818442) bank42391 +42392 POINT(41.488880239522146 73.91661759833333) bank42392 +42393 POINT(39.936584399450716 74.16857628086251) bank42393 +42394 POINT(39.74097341190982 74.00435347713027) bank42394 +42395 POINT(41.546475460148216 73.30600061617453) bank42395 +42396 POINT(40.54749469261462 73.97773424567532) bank42396 +42397 POINT(41.25206308534784 73.46589210959775) bank42397 +42398 POINT(41.021238278810344 73.17847769017725) bank42398 +42399 POINT(40.30841588472115 74.90705913101452) bank42399 +42400 POINT(40.68275129943505 74.58168706856648) bank42400 +42401 POINT(41.4369287622679 73.16684746340876) bank42401 +42402 POINT(41.54571468007863 75.00257184787291) bank42402 +42403 POINT(39.93958159954347 73.56960716299346) bank42403 +42404 POINT(40.91567921225767 73.4989298476032) bank42404 +42405 POINT(41.669667714253144 74.83139951049438) bank42405 +42406 POINT(41.07484009007797 74.54631717780539) bank42406 +42407 POINT(41.39540224134939 73.54944451928388) bank42407 +42408 POINT(40.310530311036345 74.02729788222557) bank42408 +42409 POINT(40.73375386504109 73.89281094188168) bank42409 +42410 POINT(40.26459626129886 73.65883150217408) bank42410 +42411 POINT(40.54955372103756 73.5423354778306) bank42411 +42412 POINT(41.59077280571387 74.97640795364707) bank42412 +42413 POINT(41.48096340097155 74.54613540940603) bank42413 +42414 POINT(40.77172541889921 73.68161824300576) bank42414 +42415 POINT(39.83350343644901 74.1638285054596) bank42415 +42416 POINT(39.716630651644394 74.5661352204331) bank42416 +42417 POINT(39.798892757601855 74.26358981943008) bank42417 +42418 POINT(40.37534740059754 74.11765727819898) bank42418 +42419 POINT(39.72659665727621 73.34015970966891) bank42419 +42420 POINT(41.51690761788001 74.42584549644049) bank42420 +42421 POINT(40.478589286863 73.37017932457091) bank42421 +42422 POINT(41.055770014196256 73.84074013742489) bank42422 +42423 POINT(40.434393000374754 73.89026357034203) bank42423 +42424 POINT(41.26760268616643 73.25462644645685) bank42424 +42425 POINT(40.468617497800466 74.34216676292118) bank42425 +42426 POINT(41.280280435435536 74.81398948098688) bank42426 +42427 POINT(40.07079674570558 74.65126490714015) bank42427 +42428 POINT(41.25422794636813 74.84391746742993) bank42428 +42429 POINT(41.10559602708139 74.92331130771954) bank42429 +42430 POINT(39.83553873294227 73.43356641878901) bank42430 +42431 POINT(40.52566633648758 73.4158292863599) bank42431 +42432 POINT(41.618271249079626 73.62165117104085) bank42432 +42433 POINT(41.18113584963726 73.85936010680645) bank42433 +42434 POINT(41.22341554352433 74.61130809187128) bank42434 +42435 POINT(41.14593144098964 74.62892611682645) bank42435 +42436 POINT(41.6017094675152 73.13095527909984) bank42436 +42437 POINT(41.42574795154958 73.47864551279443) bank42437 +42438 POINT(41.54152754000226 73.13948581465677) bank42438 +42439 POINT(41.13735165349116 74.29298548092696) bank42439 +42440 POINT(41.62492410799959 73.71474614882948) bank42440 +42441 POINT(41.338942147968574 73.03797600627425) bank42441 +42442 POINT(41.7126040303106 74.69790148165767) bank42442 +42443 POINT(40.31886301119417 74.53899226946147) bank42443 +42444 POINT(40.779629559114504 74.02357917864866) bank42444 +42445 POINT(40.275072799915016 74.7886182471302) bank42445 +42446 POINT(40.85479480227214 74.77351934163599) bank42446 +42447 POINT(40.48801011387154 73.84222068411951) bank42447 +42448 POINT(40.06435727802419 73.62127586704203) bank42448 +42449 POINT(41.56442322324591 74.55089505561135) bank42449 +42450 POINT(41.686782248229484 73.38536940177254) bank42450 +42451 POINT(41.35039941854718 73.92581896663253) bank42451 +42452 POINT(40.97912377039013 74.6622929282719) bank42452 +42453 POINT(40.9043499364296 73.95968577446988) bank42453 +42454 POINT(40.681696678112246 73.4426843080069) bank42454 +42455 POINT(40.17050856021553 74.09516814935678) bank42455 +42456 POINT(39.86345512892624 73.94558591055623) bank42456 +42457 POINT(39.98075232718106 73.14181021531587) bank42457 +42458 POINT(40.59901828598732 74.23739789260861) bank42458 +42459 POINT(40.135301216388896 74.94680173894358) bank42459 +42460 POINT(40.29892300833668 74.00220492771503) bank42460 +42461 POINT(40.6267441023322 73.74811267743542) bank42461 +42462 POINT(41.06950064707607 74.37780507887545) bank42462 +42463 POINT(40.43369270147199 74.87904693093957) bank42463 +42464 POINT(40.11819035923431 73.86597431468118) bank42464 +42465 POINT(40.432912206546746 73.67176176745686) bank42465 +42466 POINT(39.82459789928769 73.89004762006498) bank42466 +42467 POINT(39.952033892674585 73.42599256421906) bank42467 +42468 POINT(39.99167862166014 74.21627611475955) bank42468 +42469 POINT(41.532073981058396 73.32034022928488) bank42469 +42470 POINT(40.45355167195534 74.71320484273464) bank42470 +42471 POINT(40.06746699259745 74.44162208889813) bank42471 +42472 POINT(40.35334875567997 74.22523670010774) bank42472 +42473 POINT(40.613157223585716 74.04402613843301) bank42473 +42474 POINT(40.877135840506746 74.68517976039882) bank42474 +42475 POINT(40.336249365212254 73.22192628677249) bank42475 +42476 POINT(41.598320595955066 74.33544722044017) bank42476 +42477 POINT(39.90476598542249 73.46224644075657) bank42477 +42478 POINT(40.27621844969235 74.31177267225651) bank42478 +42479 POINT(41.08734029413609 73.01909591127588) bank42479 +42480 POINT(41.66293411937723 74.30531764652336) bank42480 +42481 POINT(39.993320083671925 73.12332514188978) bank42481 +42482 POINT(39.93728373294289 73.78437146973735) bank42482 +42483 POINT(41.108786391380285 73.38448397479304) bank42483 +42484 POINT(41.48600066533641 74.61141895060737) bank42484 +42485 POINT(40.80027376332006 74.33773867759372) bank42485 +42486 POINT(39.93857597637957 74.20278518722378) bank42486 +42487 POINT(41.51899237186843 73.10409154742926) bank42487 +42488 POINT(40.2401510830411 73.08317289567557) bank42488 +42489 POINT(40.38648910621288 73.97732328613858) bank42489 +42490 POINT(40.748060190616265 73.23872175732842) bank42490 +42491 POINT(41.19765455860887 74.00715513779078) bank42491 +42492 POINT(40.728042157408005 73.01928257381564) bank42492 +42493 POINT(40.88464546757321 73.11691072568836) bank42493 +42494 POINT(41.26912339723967 74.46674726048157) bank42494 +42495 POINT(40.797836736660436 74.05686267304164) bank42495 +42496 POINT(40.565195179678 74.03828595878588) bank42496 +42497 POINT(40.97819252997115 74.8557348275217) bank42497 +42498 POINT(40.886640735914085 73.55057264631021) bank42498 +42499 POINT(39.913285714096894 73.45470407349322) bank42499 +42500 POINT(40.06774482854555 73.88521845649944) bank42500 +42501 POINT(41.32322746624441 74.60472180268917) bank42501 +42502 POINT(40.15121760927259 74.49796295451713) bank42502 +42503 POINT(40.20285557810654 73.09462937948184) bank42503 +42504 POINT(40.030893860971744 74.04242290831078) bank42504 +42505 POINT(41.659062147379665 73.80103760664997) bank42505 +42506 POINT(39.8419869298325 74.83604381228976) bank42506 +42507 POINT(40.344694209126146 73.79612375690294) bank42507 +42508 POINT(39.78874357405526 73.10040762411538) bank42508 +42509 POINT(41.34010662421251 73.46286309996752) bank42509 +42510 POINT(41.0458293609709 73.74880236655422) bank42510 +42511 POINT(40.65971797057785 74.88226457312011) bank42511 +42512 POINT(41.36810702197868 74.01249156790516) bank42512 +42513 POINT(40.59616250586288 73.0518803234153) bank42513 +42514 POINT(40.205062984625414 73.54597374628831) bank42514 +42515 POINT(40.5721021211765 74.70610661372984) bank42515 +42516 POINT(41.36724424475555 74.82266037987293) bank42516 +42517 POINT(40.89298647376186 74.03368573192932) bank42517 +42518 POINT(41.51596113467247 73.18518508612681) bank42518 +42519 POINT(41.48234182617833 74.42072235176698) bank42519 +42520 POINT(40.85224222298908 73.08255864081448) bank42520 +42521 POINT(40.80656209661727 73.20324768058667) bank42521 +42522 POINT(41.27317590448198 73.74010242181693) bank42522 +42523 POINT(40.620527695100066 74.52162127102805) bank42523 +42524 POINT(41.47596070408834 73.31752628533278) bank42524 +42525 POINT(40.22344807841444 74.58046069233617) bank42525 +42526 POINT(41.491317043338874 73.37820380998176) bank42526 +42527 POINT(39.91813994892438 73.45804541456208) bank42527 +42528 POINT(40.81059597751305 73.17788404180838) bank42528 +42529 POINT(39.8512056360259 73.86695933887731) bank42529 +42530 POINT(40.18478040983111 74.3866615574123) bank42530 +42531 POINT(40.56321538389181 73.73590390939958) bank42531 +42532 POINT(40.5076700395436 74.01827517621156) bank42532 +42533 POINT(40.676198821836834 73.8970821032188) bank42533 +42534 POINT(39.86118924401216 74.1457949687595) bank42534 +42535 POINT(40.73150416110266 73.05162367582167) bank42535 +42536 POINT(39.76648780855748 74.14835580616847) bank42536 +42537 POINT(41.140485144226766 73.96507993999124) bank42537 +42538 POINT(41.38577629709953 74.23927990435384) bank42538 +42539 POINT(40.09488856493767 73.8406040939975) bank42539 +42540 POINT(40.69064534880303 73.81047394577091) bank42540 +42541 POINT(41.467942010474616 73.0164877031943) bank42541 +42542 POINT(40.313493685667574 73.47668231116428) bank42542 +42543 POINT(40.0932368358438 73.87792138470607) bank42543 +42544 POINT(40.03471435584503 73.65126835614912) bank42544 +42545 POINT(40.21550655429065 74.64418290335274) bank42545 +42546 POINT(40.02424776657273 74.10316813867534) bank42546 +42547 POINT(40.91564948606695 73.04770874375792) bank42547 +42548 POINT(39.989045301947904 74.41772711158288) bank42548 +42549 POINT(40.10771840531539 73.61712336743871) bank42549 +42550 POINT(41.25519308978426 73.3829971040914) bank42550 +42551 POINT(40.63376575945079 74.44308945405406) bank42551 +42552 POINT(40.79527359729701 73.7553861057946) bank42552 +42553 POINT(40.50900542752448 73.0433143115689) bank42553 +42554 POINT(39.71536349397906 74.85430017941353) bank42554 +42555 POINT(40.767035657716136 74.92514489809416) bank42555 +42556 POINT(40.852173589678756 73.51118300849319) bank42556 +42557 POINT(40.73617102416912 74.85399693385484) bank42557 +42558 POINT(41.64817791967093 74.60941304785895) bank42558 +42559 POINT(41.59835011489983 74.8484826527003) bank42559 +42560 POINT(40.932328528816406 73.27329879639086) bank42560 +42561 POINT(41.267144990822544 74.18893474057307) bank42561 +42562 POINT(41.68645489986268 74.31723221200355) bank42562 +42563 POINT(40.21858090546146 74.66983989000491) bank42563 +42564 POINT(41.06890738124848 74.72387072483116) bank42564 +42565 POINT(40.11506514168914 73.74084629043706) bank42565 +42566 POINT(40.52938112859254 73.46792717449004) bank42566 +42567 POINT(40.664487080009394 74.0688140822747) bank42567 +42568 POINT(39.71582693901624 74.05649189310238) bank42568 +42569 POINT(41.52381491742409 73.97003619539976) bank42569 +42570 POINT(41.24645690354357 73.59865554135351) bank42570 +42571 POINT(40.7279805056632 74.81000139053154) bank42571 +42572 POINT(41.128106148507634 74.4273044719159) bank42572 +42573 POINT(39.78032257671138 73.59518244270328) bank42573 +42574 POINT(40.33092359659223 74.29395727440506) bank42574 +42575 POINT(40.613982672896434 73.40824474920656) bank42575 +42576 POINT(40.464969161619166 73.83893717093862) bank42576 +42577 POINT(40.576170522454234 74.25277797635493) bank42577 +42578 POINT(41.68234976749264 73.95780622402141) bank42578 +42579 POINT(41.17208212469446 73.19775262386376) bank42579 +42580 POINT(41.40702125849124 73.40874809236364) bank42580 +42581 POINT(40.69536078372326 73.84473469351248) bank42581 +42582 POINT(39.95874155760772 73.24949372187781) bank42582 +42583 POINT(40.18562912815573 74.94727518070904) bank42583 +42584 POINT(40.89887577251591 74.63647331296063) bank42584 +42585 POINT(40.53034296863935 73.82505659027365) bank42585 +42586 POINT(41.11983959054851 74.70422021441306) bank42586 +42587 POINT(40.79778216301433 73.87617105696096) bank42587 +42588 POINT(41.407824984397166 74.6846560172775) bank42588 +42589 POINT(40.28692275057223 73.09767093881882) bank42589 +42590 POINT(40.36671521333984 74.71155999504973) bank42590 +42591 POINT(40.330414912059425 74.16199410033143) bank42591 +42592 POINT(40.455821539793504 73.68969132532642) bank42592 +42593 POINT(41.43753611969474 74.892910824073) bank42593 +42594 POINT(39.81552018579441 74.0457348814617) bank42594 +42595 POINT(39.80988626304938 74.98767727676001) bank42595 +42596 POINT(40.2479644669163 74.71079153247928) bank42596 +42597 POINT(41.62492311119685 73.75339994068567) bank42597 +42598 POINT(40.93214605950302 73.50023346302592) bank42598 +42599 POINT(39.96297556425179 74.80708006618592) bank42599 +42600 POINT(39.974799222958374 74.63034182145955) bank42600 +42601 POINT(40.73034089337316 73.84862720661353) bank42601 +42602 POINT(40.29778086213628 74.38292499232642) bank42602 +42603 POINT(41.70212584335786 74.06712838026968) bank42603 +42604 POINT(40.965633583814544 74.8892294381098) bank42604 +42605 POINT(41.20584633846262 74.48121877401579) bank42605 +42606 POINT(41.46487654785397 74.55232979184106) bank42606 +42607 POINT(39.81787219277777 73.8686761969806) bank42607 +42608 POINT(41.113231064334876 74.47647466903555) bank42608 +42609 POINT(41.32286474621789 73.48437119768452) bank42609 +42610 POINT(41.71035499825023 74.17758368531373) bank42610 +42611 POINT(41.49249170985034 73.9016577865309) bank42611 +42612 POINT(40.8444311539051 74.31981706953296) bank42612 +42613 POINT(40.103408421830665 73.02711013053596) bank42613 +42614 POINT(39.750592185734064 73.71951016304975) bank42614 +42615 POINT(40.36877185158715 74.28730402344334) bank42615 +42616 POINT(41.62986644808784 74.43202566209555) bank42616 +42617 POINT(40.891005424614264 73.01164448693258) bank42617 +42618 POINT(39.783559054350874 73.89952718201461) bank42618 +42619 POINT(39.78271711268051 73.19017797431113) bank42619 +42620 POINT(40.461800239368436 73.52867690823095) bank42620 +42621 POINT(40.45843431555031 74.2443014570577) bank42621 +42622 POINT(41.50104962404488 74.81603807411778) bank42622 +42623 POINT(40.96358556213384 74.20381649044178) bank42623 +42624 POINT(40.88040575545525 73.37743050478562) bank42624 +42625 POINT(40.64130297848431 73.08514447945454) bank42625 +42626 POINT(40.11203996103918 74.89448001639474) bank42626 +42627 POINT(39.953063334514404 74.28304711620474) bank42627 +42628 POINT(41.04172112861442 73.886491032693) bank42628 +42629 POINT(40.84745646257164 73.91093515363522) bank42629 +42630 POINT(40.87545220161614 74.42373321840779) bank42630 +42631 POINT(41.04911702428202 73.09167085561164) bank42631 +42632 POINT(40.80951024089993 73.79646808292866) bank42632 +42633 POINT(39.91863748371578 74.08862969694597) bank42633 +42634 POINT(40.32965575339987 74.32269283425748) bank42634 +42635 POINT(40.415860018891465 73.05021808139088) bank42635 +42636 POINT(40.45414887625839 73.73297091270076) bank42636 +42637 POINT(40.93333135760733 73.98966896004512) bank42637 +42638 POINT(40.51616441221921 74.83064306011457) bank42638 +42639 POINT(39.82480640984374 73.37368119262838) bank42639 +42640 POINT(40.96137131070829 73.90130858858575) bank42640 +42641 POINT(40.014578366719135 74.4449502345633) bank42641 +42642 POINT(40.270349853647204 74.42709225504726) bank42642 +42643 POINT(40.38026035379228 73.70824273895771) bank42643 +42644 POINT(40.0620847192204 74.5590339347283) bank42644 +42645 POINT(40.52673909800148 74.9657749005628) bank42645 +42646 POINT(41.57694798524292 73.66854355186814) bank42646 +42647 POINT(40.121893282888166 74.55375225311401) bank42647 +42648 POINT(41.49178916802769 74.93743746892167) bank42648 +42649 POINT(40.9922585806149 73.11173008929403) bank42649 +42650 POINT(40.81183752365191 74.94354865776475) bank42650 +42651 POINT(40.86096889052044 73.8529407093482) bank42651 +42652 POINT(40.74034613723706 74.01392243909542) bank42652 +42653 POINT(40.71548647977234 73.68451213058421) bank42653 +42654 POINT(40.903489886383596 74.01663781043248) bank42654 +42655 POINT(41.2390421237777 73.50262288994513) bank42655 +42656 POINT(41.09307187389778 73.8933421987724) bank42656 +42657 POINT(40.34018234971412 74.65522600925904) bank42657 +42658 POINT(40.79064854068513 73.51262234670376) bank42658 +42659 POINT(41.48335694060383 73.67477479373979) bank42659 +42660 POINT(41.618032226491565 74.92643903274096) bank42660 +42661 POINT(39.7851845368436 74.6328734247534) bank42661 +42662 POINT(40.0227108594334 74.2494186692214) bank42662 +42663 POINT(41.221006669407636 74.34547023911658) bank42663 +42664 POINT(39.98566471941387 73.59649351630488) bank42664 +42665 POINT(39.95817371597603 74.51450058842865) bank42665 +42666 POINT(40.40980800416382 74.78646223517981) bank42666 +42667 POINT(41.32092820107958 73.40247346358063) bank42667 +42668 POINT(40.640964239692714 73.36790077506167) bank42668 +42669 POINT(40.05187312593306 73.86714775110258) bank42669 +42670 POINT(41.665825156855334 74.0944362406055) bank42670 +42671 POINT(39.787645562758435 74.63373905561166) bank42671 +42672 POINT(39.945364994112516 73.92968373021021) bank42672 +42673 POINT(40.54764837471118 74.10941987875434) bank42673 +42674 POINT(41.59725764364042 74.83445493555129) bank42674 +42675 POINT(41.50024468583116 73.45660230608723) bank42675 +42676 POINT(41.2303784557688 74.6147191436702) bank42676 +42677 POINT(41.171573825796976 73.20818229551047) bank42677 +42678 POINT(39.98803782336516 74.92390783238014) bank42678 +42679 POINT(40.840725513113384 74.00760015560243) bank42679 +42680 POINT(41.2628624944352 73.60323554369894) bank42680 +42681 POINT(40.19412483806976 73.04258175086818) bank42681 +42682 POINT(40.89598316774989 73.93463357388323) bank42682 +42683 POINT(40.358287979769926 74.38658301510321) bank42683 +42684 POINT(41.374603491922606 74.73980860428604) bank42684 +42685 POINT(40.814697737258314 73.68867157645049) bank42685 +42686 POINT(40.65998562385652 73.96224780427697) bank42686 +42687 POINT(40.25455370537043 74.53566386836656) bank42687 +42688 POINT(40.35611241011764 74.96240050109382) bank42688 +42689 POINT(39.961770007332994 74.91421472426536) bank42689 +42690 POINT(40.267086667880044 74.11153660029332) bank42690 +42691 POINT(41.07605204118359 74.39611717534497) bank42691 +42692 POINT(41.56539226338328 74.27082923713697) bank42692 +42693 POINT(41.12261404634014 74.76747577501405) bank42693 +42694 POINT(40.66260225642959 73.36106160034788) bank42694 +42695 POINT(40.06113847135607 74.44540671617858) bank42695 +42696 POINT(40.56158696599505 73.78525469791055) bank42696 +42697 POINT(40.44113676980351 73.88583305296861) bank42697 +42698 POINT(41.59750192235991 74.57760938194042) bank42698 +42699 POINT(40.491661126842395 74.22244863445022) bank42699 +42700 POINT(41.00828245652621 73.58656900019555) bank42700 +42701 POINT(41.68868495077552 75.00488311757536) bank42701 +42702 POINT(40.14011706680065 74.1383204009368) bank42702 +42703 POINT(41.038414759120705 73.67202494023086) bank42703 +42704 POINT(40.66281013554246 73.46705002852349) bank42704 +42705 POINT(40.886387649889336 73.53248327304242) bank42705 +42706 POINT(40.29705860862601 74.21983168056238) bank42706 +42707 POINT(41.598223865178554 73.82817029116829) bank42707 +42708 POINT(39.79269862161621 74.67159098011417) bank42708 +42709 POINT(40.19296426561644 73.569819384894) bank42709 +42710 POINT(40.36554089557051 73.42614488487888) bank42710 +42711 POINT(40.63494180698982 73.05636116708774) bank42711 +42712 POINT(41.29818515781208 73.39408462566992) bank42712 +42713 POINT(41.34877002815961 74.23960978924494) bank42713 +42714 POINT(40.66664820302124 74.40301347724464) bank42714 +42715 POINT(40.477452557740605 74.02995565996355) bank42715 +42716 POINT(39.82860312783032 74.43275961867225) bank42716 +42717 POINT(41.54949222489196 74.77870807732938) bank42717 +42718 POINT(40.210717929398065 74.72468505063233) bank42718 +42719 POINT(40.310006043881586 73.80590284237799) bank42719 +42720 POINT(41.3170715044279 73.95613176658935) bank42720 +42721 POINT(41.11839478625825 74.31186265769188) bank42721 +42722 POINT(40.6861896648322 73.94683620046366) bank42722 +42723 POINT(40.99160403909056 73.1869863569538) bank42723 +42724 POINT(39.95359897952979 73.7228901182947) bank42724 +42725 POINT(41.61849450220675 73.68129439103137) bank42725 +42726 POINT(40.2814770999831 73.47589106960737) bank42726 +42727 POINT(41.37662478392477 74.95462547648549) bank42727 +42728 POINT(39.7618536528339 74.0613368042283) bank42728 +42729 POINT(39.93252947786266 74.10637941591501) bank42729 +42730 POINT(41.71068356390888 74.27762910824812) bank42730 +42731 POINT(41.01009940202589 74.06284630199757) bank42731 +42732 POINT(40.39675446595511 74.23553549778144) bank42732 +42733 POINT(39.87998893678137 74.41474711805681) bank42733 +42734 POINT(40.162307437642056 73.53952230137838) bank42734 +42735 POINT(41.14740915399598 74.81280628368077) bank42735 +42736 POINT(41.63283614739036 74.82308685059533) bank42736 +42737 POINT(41.50852080309476 74.04081399349936) bank42737 +42738 POINT(41.03461218713184 74.28335680935896) bank42738 +42739 POINT(40.200269463720204 73.48475055283377) bank42739 +42740 POINT(40.29763666284495 74.14610959819387) bank42740 +42741 POINT(40.091141276577176 74.94596198701413) bank42741 +42742 POINT(40.23763078388486 73.26389868040647) bank42742 +42743 POINT(40.05259725918781 74.14761161042937) bank42743 +42744 POINT(41.53418817568874 73.38497681339939) bank42744 +42745 POINT(41.26246997001996 74.04751819309628) bank42745 +42746 POINT(40.18318682504973 73.90342478267564) bank42746 +42747 POINT(41.23545966182734 74.37925742273518) bank42747 +42748 POINT(40.303083795823675 73.40461892413235) bank42748 +42749 POINT(41.01160509409115 74.21976317124302) bank42749 +42750 POINT(40.75475466961658 74.06711907637592) bank42750 +42751 POINT(40.844738691579 74.36921326493021) bank42751 +42752 POINT(40.01407836071567 74.88072690396385) bank42752 +42753 POINT(40.109446223505564 74.16199888899614) bank42753 +42754 POINT(40.56557310548707 73.73895278282417) bank42754 +42755 POINT(40.34711491714979 74.8498526217089) bank42755 +42756 POINT(40.65581266773866 74.68675789340821) bank42756 +42757 POINT(40.23371200784318 73.77867978398848) bank42757 +42758 POINT(41.66502656308576 74.52858587159623) bank42758 +42759 POINT(41.62259509506152 73.46056400828252) bank42759 +42760 POINT(40.22952152165243 74.5510566160954) bank42760 +42761 POINT(41.44829614116542 73.8337007837467) bank42761 +42762 POINT(40.22785263771363 74.57212296575173) bank42762 +42763 POINT(40.32670494266232 74.2814507566884) bank42763 +42764 POINT(40.7711683265553 74.71912840037947) bank42764 +42765 POINT(39.902828187043525 73.20449971431606) bank42765 +42766 POINT(41.493908750007364 74.07527610227734) bank42766 +42767 POINT(40.53040565891614 74.74947445048552) bank42767 +42768 POINT(40.82589103067732 73.9697683510306) bank42768 +42769 POINT(40.50680661566229 74.93268948988938) bank42769 +42770 POINT(40.546035245794535 73.55947312861936) bank42770 +42771 POINT(40.27905853860106 74.57911019567452) bank42771 +42772 POINT(41.3365742588943 73.95330791669507) bank42772 +42773 POINT(40.29519030871484 74.26509189991026) bank42773 +42774 POINT(40.95956977396026 73.773287153315) bank42774 +42775 POINT(40.04352781562716 74.7384729759699) bank42775 +42776 POINT(40.49843156807155 74.813099831483) bank42776 +42777 POINT(41.68227863253688 73.95886429752493) bank42777 +42778 POINT(41.46636958368364 74.70850082290191) bank42778 +42779 POINT(41.13600384041032 73.77031627206115) bank42779 +42780 POINT(41.36334171710941 73.98073859042859) bank42780 +42781 POINT(40.027166114425356 74.22627590601321) bank42781 +42782 POINT(40.622427486672436 73.90871894041635) bank42782 +42783 POINT(40.09332389709717 73.80800130025466) bank42783 +42784 POINT(40.96179385724042 74.35098737156224) bank42784 +42785 POINT(41.22255081888877 74.67182369343638) bank42785 +42786 POINT(39.87683314546452 74.55693034462442) bank42786 +42787 POINT(41.44864994538978 73.48050899444107) bank42787 +42788 POINT(41.07991280836471 73.76211991091891) bank42788 +42789 POINT(41.255232405169075 74.9617512290425) bank42789 +42790 POINT(40.7636170431616 74.29275483520314) bank42790 +42791 POINT(39.89012256704056 73.85828922024706) bank42791 +42792 POINT(40.33986613570329 74.23644612452615) bank42792 +42793 POINT(40.98596620324885 73.35158855183407) bank42793 +42794 POINT(41.06156184905318 74.13268196091934) bank42794 +42795 POINT(41.467257688491415 74.43478205017314) bank42795 +42796 POINT(39.955427973251915 74.84241640220598) bank42796 +42797 POINT(39.96806781271303 73.22204996971121) bank42797 +42798 POINT(40.223444834374874 74.68399932028521) bank42798 +42799 POINT(41.061109491681165 73.81904801842416) bank42799 +42800 POINT(40.423971140147806 74.10015539989807) bank42800 +42801 POINT(41.524270870769676 74.33457157325519) bank42801 +42802 POINT(40.162997246912745 73.41379240033936) bank42802 +42803 POINT(39.90893342718434 73.69549410891317) bank42803 +42804 POINT(40.977995851159726 74.79725444921603) bank42804 +42805 POINT(41.707606009179 74.5163169936845) bank42805 +42806 POINT(41.28058137637239 74.22105916696142) bank42806 +42807 POINT(39.85490071503138 73.26541842414892) bank42807 +42808 POINT(40.28564147183465 74.55768749792328) bank42808 +42809 POINT(40.33333853591665 74.54789080655786) bank42809 +42810 POINT(41.4692850868304 73.00728199828797) bank42810 +42811 POINT(41.16647430155247 74.9473766932783) bank42811 +42812 POINT(40.27707158051425 73.7425933010397) bank42812 +42813 POINT(41.323948949129296 73.48444680295965) bank42813 +42814 POINT(40.28052519810136 74.95320551641912) bank42814 +42815 POINT(41.07282787685764 73.27463943713096) bank42815 +42816 POINT(39.912844206341774 73.09048124762838) bank42816 +42817 POINT(41.02018184858944 73.29515276631264) bank42817 +42818 POINT(40.009059335836824 74.95246217930867) bank42818 +42819 POINT(40.467229771491176 74.45453857959158) bank42819 +42820 POINT(40.061365755165625 74.69121082379799) bank42820 +42821 POINT(40.541387915353354 74.26443243071402) bank42821 +42822 POINT(41.42484538512524 73.89910086686076) bank42822 +42823 POINT(41.301312230960725 73.37698205750424) bank42823 +42824 POINT(40.34465853070139 73.42655349189928) bank42824 +42825 POINT(39.895057456973184 74.82321988987728) bank42825 +42826 POINT(40.43427051152505 74.52723627967757) bank42826 +42827 POINT(40.17675130119122 73.1732141610643) bank42827 +42828 POINT(41.43108769143271 73.23697755496306) bank42828 +42829 POINT(41.05776247654072 73.74544442190356) bank42829 +42830 POINT(40.820997978499655 73.73277636997634) bank42830 +42831 POINT(41.41328173369626 74.49909306822663) bank42831 +42832 POINT(40.54467368899252 74.17421528040546) bank42832 +42833 POINT(40.36712681208729 73.38989511160983) bank42833 +42834 POINT(41.549274502978285 74.29977036955223) bank42834 +42835 POINT(41.07969388124436 73.43266894992459) bank42835 +42836 POINT(41.0366440846538 73.2556752781171) bank42836 +42837 POINT(40.15583624067667 73.96714542008891) bank42837 +42838 POINT(40.00296192252667 74.24976391885184) bank42838 +42839 POINT(39.80233795642094 73.80243277573797) bank42839 +42840 POINT(40.91202649799197 73.16688402075978) bank42840 +42841 POINT(39.89763339585126 74.90114885009322) bank42841 +42842 POINT(40.428614426271025 73.03067519313088) bank42842 +42843 POINT(40.20960568214963 73.59202254462878) bank42843 +42844 POINT(40.86252616373373 73.5206007649511) bank42844 +42845 POINT(40.66413771639514 74.09651995150543) bank42845 +42846 POINT(41.21738105714799 73.70103922927403) bank42846 +42847 POINT(40.2234692426498 74.05849186958847) bank42847 +42848 POINT(40.37296674195337 73.26852566940502) bank42848 +42849 POINT(41.04409577689053 74.20389775030773) bank42849 +42850 POINT(39.93525001218499 74.9918464385248) bank42850 +42851 POINT(39.717110249471666 73.57868026597754) bank42851 +42852 POINT(40.554668316841955 73.60131863973604) bank42852 +42853 POINT(40.78055606659164 74.95887294996777) bank42853 +42854 POINT(40.97910621732034 73.53990202748024) bank42854 +42855 POINT(39.71625813343216 73.46147177396406) bank42855 +42856 POINT(40.92138515735941 73.24399243222084) bank42856 +42857 POINT(40.76303609605168 74.70414669255844) bank42857 +42858 POINT(41.408760539053404 74.27803793372402) bank42858 +42859 POINT(40.04889514749222 73.08934976971875) bank42859 +42860 POINT(40.89763560671554 74.53784984964567) bank42860 +42861 POINT(39.72913777684797 74.07772489593597) bank42861 +42862 POINT(39.72239661420977 73.01393658090754) bank42862 +42863 POINT(40.215456086376946 73.40201105869643) bank42863 +42864 POINT(40.46187348540986 74.12636362012623) bank42864 +42865 POINT(41.266561587665215 74.32176363633323) bank42865 +42866 POINT(40.57351454553606 74.65933525260745) bank42866 +42867 POINT(40.49923820335589 74.48171268456593) bank42867 +42868 POINT(39.78659459594861 73.68252483693492) bank42868 +42869 POINT(40.632193857687824 73.4469016370794) bank42869 +42870 POINT(39.74811830700695 73.56861052147421) bank42870 +42871 POINT(40.76145593280476 74.37918372868407) bank42871 +42872 POINT(40.51529980664661 73.79696680973319) bank42872 +42873 POINT(40.640165155168724 74.95827741279814) bank42873 +42874 POINT(39.968409672146066 73.54080112276552) bank42874 +42875 POINT(40.291525009543136 73.43271282616966) bank42875 +42876 POINT(40.8712374340131 73.60205202033823) bank42876 +42877 POINT(40.152920685035774 74.9269261727375) bank42877 +42878 POINT(41.45872662449827 73.34240893102137) bank42878 +42879 POINT(41.568462807019934 74.83022968922123) bank42879 +42880 POINT(39.986659056992764 74.39116627641036) bank42880 +42881 POINT(40.06427675484255 74.72176451751243) bank42881 +42882 POINT(39.882775459033404 73.84341886508328) bank42882 +42883 POINT(40.97695882246289 74.98830372410393) bank42883 +42884 POINT(39.91360356773981 74.6279386568254) bank42884 +42885 POINT(40.57192630183539 74.75103775452632) bank42885 +42886 POINT(40.76476051001835 74.22144704211009) bank42886 +42887 POINT(41.436907726006105 74.87906597253323) bank42887 +42888 POINT(40.904753732849045 74.50439843111312) bank42888 +42889 POINT(41.276336987632426 74.97097296638925) bank42889 +42890 POINT(39.73223533745648 74.83745516252606) bank42890 +42891 POINT(41.28757124554815 74.9088900149724) bank42891 +42892 POINT(40.047323600381034 73.76952722724154) bank42892 +42893 POINT(41.00883590330805 74.3641126271131) bank42893 +42894 POINT(40.43777821473015 73.67855626247169) bank42894 +42895 POINT(40.294071497595084 74.7874029787218) bank42895 +42896 POINT(41.220616833212645 73.38426587819134) bank42896 +42897 POINT(39.95795706915728 74.39749435859002) bank42897 +42898 POINT(40.31864129251004 74.81738400582202) bank42898 +42899 POINT(41.353700035333674 73.11065977441464) bank42899 +42900 POINT(41.44978934160227 73.79801104674942) bank42900 +42901 POINT(41.392843964346774 74.03726582273535) bank42901 +42902 POINT(39.986488989925036 73.472532465621) bank42902 +42903 POINT(40.80811362438016 73.36732209299697) bank42903 +42904 POINT(41.55084125489369 74.77925334291403) bank42904 +42905 POINT(41.38326771557456 73.70404392955156) bank42905 +42906 POINT(41.34077051991792 73.83532819607802) bank42906 +42907 POINT(39.75860134699042 74.89364592952462) bank42907 +42908 POINT(40.928680785225126 74.99228458276704) bank42908 +42909 POINT(40.141878666690154 73.22294269321573) bank42909 +42910 POINT(40.438252161974525 74.8666358269239) bank42910 +42911 POINT(41.105723379980134 73.12278649450089) bank42911 +42912 POINT(41.111396398617266 73.0928582881807) bank42912 +42913 POINT(40.047958603636864 74.45637603013998) bank42913 +42914 POINT(41.405870280671465 73.235255074809) bank42914 +42915 POINT(39.862047554028045 74.85748126954122) bank42915 +42916 POINT(40.67944182906337 74.39850506197585) bank42916 +42917 POINT(41.10653194389557 75.00284896516827) bank42917 +42918 POINT(41.2686567672303 73.69083402281359) bank42918 +42919 POINT(40.34303149604416 73.6751881272521) bank42919 +42920 POINT(41.60258400232048 73.59609255594158) bank42920 +42921 POINT(41.23072822468749 74.60372825580119) bank42921 +42922 POINT(39.784128804728454 73.40555846762491) bank42922 +42923 POINT(41.29942785152645 73.71620270762195) bank42923 +42924 POINT(40.88343719777514 74.04275729315515) bank42924 +42925 POINT(41.46781123244995 74.55230915878478) bank42925 +42926 POINT(41.24776806962123 73.4212442647089) bank42926 +42927 POINT(41.25740542545175 74.262258590132) bank42927 +42928 POINT(41.18259862379183 73.7633025229605) bank42928 +42929 POINT(39.833593184721956 73.68702908279815) bank42929 +42930 POINT(40.97539239906052 73.42292764498745) bank42930 +42931 POINT(41.51080565947488 74.46968239496326) bank42931 +42932 POINT(40.97431681105115 73.57182829891138) bank42932 +42933 POINT(40.502471619739126 73.10211123354848) bank42933 +42934 POINT(40.170716953542325 74.16134192507934) bank42934 +42935 POINT(40.6663973437361 73.15111555122924) bank42935 +42936 POINT(41.65866513069786 74.36476663756982) bank42936 +42937 POINT(41.69963722936124 73.08851139701814) bank42937 +42938 POINT(40.69519317885226 74.93762508982098) bank42938 +42939 POINT(41.50455549568528 73.02324832055184) bank42939 +42940 POINT(40.408993504898774 73.2833058303222) bank42940 +42941 POINT(41.375828419804655 73.30495224536484) bank42941 +42942 POINT(41.60220451590603 74.18006132330714) bank42942 +42943 POINT(40.212084691124645 73.61395603039276) bank42943 +42944 POINT(41.229712967342095 73.58915098955775) bank42944 +42945 POINT(40.77588285877964 73.37877359205946) bank42945 +42946 POINT(40.54984767981185 74.44542990979134) bank42946 +42947 POINT(40.754218439070335 74.56557429137406) bank42947 +42948 POINT(41.23597347600929 73.63819727270845) bank42948 +42949 POINT(39.965924311395334 73.08039141915441) bank42949 +42950 POINT(41.29973156755833 73.37882458858392) bank42950 +42951 POINT(40.00615311998059 74.41270682473215) bank42951 +42952 POINT(40.09836833770817 73.74063034809467) bank42952 +42953 POINT(40.80623947911525 74.2457373490509) bank42953 +42954 POINT(40.39738572533943 74.14501443273934) bank42954 +42955 POINT(41.387291049037714 74.70935816830186) bank42955 +42956 POINT(40.86908129099982 74.26864653020243) bank42956 +42957 POINT(41.23062318352849 74.42781999232328) bank42957 +42958 POINT(41.51438309153138 73.77091437027205) bank42958 +42959 POINT(39.73360495154829 73.96190288078392) bank42959 +42960 POINT(41.66213844457457 74.44907333849633) bank42960 +42961 POINT(40.59039566975109 74.29163544778895) bank42961 +42962 POINT(40.796614816427216 74.61949384869953) bank42962 +42963 POINT(40.065604375687194 73.40764962375425) bank42963 +42964 POINT(41.580952575297744 74.40369287283063) bank42964 +42965 POINT(41.227943452930994 73.69961706832055) bank42965 +42966 POINT(41.39784886549595 74.56029147215382) bank42966 +42967 POINT(40.29786912065874 73.82716153541092) bank42967 +42968 POINT(41.026804432556375 73.84396746687553) bank42968 +42969 POINT(40.880500583147395 73.26687949382286) bank42969 +42970 POINT(40.602809630472 74.17866359901856) bank42970 +42971 POINT(39.859594788763914 73.5632020989948) bank42971 +42972 POINT(39.718956310257774 73.90797086522583) bank42972 +42973 POINT(41.03541889892657 74.95919776013861) bank42973 +42974 POINT(40.49073336589315 73.36961404927848) bank42974 +42975 POINT(40.43304125821045 74.57427014539778) bank42975 +42976 POINT(40.67063404207487 73.62619490415935) bank42976 +42977 POINT(40.37450635444358 73.40822240468503) bank42977 +42978 POINT(40.449760442722585 74.59963341770849) bank42978 +42979 POINT(40.379828836930365 74.14576533273818) bank42979 +42980 POINT(41.23688798378904 73.81263962204108) bank42980 +42981 POINT(40.33069857069919 73.56433135400336) bank42981 +42982 POINT(41.37248632617244 73.5898871977005) bank42982 +42983 POINT(39.716386975280386 73.32302420829703) bank42983 +42984 POINT(40.7630662889474 73.40756944951576) bank42984 +42985 POINT(40.15828521066021 74.86231627706199) bank42985 +42986 POINT(39.96490926588143 73.96291961796207) bank42986 +42987 POINT(41.514936661677794 74.7800824848899) bank42987 +42988 POINT(41.32906639095413 74.78431522344933) bank42988 +42989 POINT(39.81177130671329 74.031232201978) bank42989 +42990 POINT(41.69131109337517 73.08680157267916) bank42990 +42991 POINT(41.65994223085315 74.82013551140935) bank42991 +42992 POINT(41.33944237828423 74.09364906327015) bank42992 +42993 POINT(40.69725507729108 74.73245925100298) bank42993 +42994 POINT(39.994844708393586 73.2487345730391) bank42994 +42995 POINT(41.63945119173194 73.85244704532755) bank42995 +42996 POINT(40.910726269448176 73.77398761576174) bank42996 +42997 POINT(40.64362849038114 73.15300762701733) bank42997 +42998 POINT(41.06478183847582 73.61490294566323) bank42998 +42999 POINT(40.14355788138993 73.60950175318315) bank42999 +43000 POINT(41.02666001663932 73.51433591261518) bank43000 +43001 POINT(40.44894545021481 73.78563704332932) bank43001 +43002 POINT(40.60437823647408 74.03378410666375) bank43002 +43003 POINT(40.87636510985976 74.2797464883272) bank43003 +43004 POINT(40.99233802961615 74.46244088173016) bank43004 +43005 POINT(40.94416618988777 74.14637443205365) bank43005 +43006 POINT(39.81499410929888 74.49367170479188) bank43006 +43007 POINT(40.61290218091957 74.16075848835743) bank43007 +43008 POINT(41.19784707229878 74.1048980855229) bank43008 +43009 POINT(40.189382768831585 74.11166227669867) bank43009 +43010 POINT(41.027222231633374 73.79901094711745) bank43010 +43011 POINT(41.11773906415902 74.83702418140453) bank43011 +43012 POINT(39.8822343601403 73.1384706648667) bank43012 +43013 POINT(40.509786104230024 73.33156926289307) bank43013 +43014 POINT(41.08165451047117 73.34789239758257) bank43014 +43015 POINT(40.14272248998439 74.15041564526311) bank43015 +43016 POINT(40.80105254817813 74.96468541905065) bank43016 +43017 POINT(40.28656726599695 73.48754850856254) bank43017 +43018 POINT(41.11392360997121 74.37273123991908) bank43018 +43019 POINT(40.37365794290336 73.786362500136) bank43019 +43020 POINT(40.32652025262547 74.36046859964937) bank43020 +43021 POINT(41.465377900416165 73.44449492046871) bank43021 +43022 POINT(39.858070640900195 73.02711669271888) bank43022 +43023 POINT(41.150008670369 73.81285091118885) bank43023 +43024 POINT(40.6989307056049 74.50273469549354) bank43024 +43025 POINT(41.51002146907457 74.72290705366116) bank43025 +43026 POINT(41.304691141818424 73.73125756538066) bank43026 +43027 POINT(39.8056250293992 73.02667075000112) bank43027 +43028 POINT(40.565573358183 73.22663445252952) bank43028 +43029 POINT(39.84712736062973 73.67782494295183) bank43029 +43030 POINT(41.32031830996357 74.1260726885158) bank43030 +43031 POINT(40.93159218051528 73.07821429904739) bank43031 +43032 POINT(41.156447072951494 74.7191628113413) bank43032 +43033 POINT(40.28142655957898 73.17864575324222) bank43033 +43034 POINT(40.26391609835863 74.7532612728831) bank43034 +43035 POINT(40.024279707542604 73.20359967934372) bank43035 +43036 POINT(40.627051814926645 74.92947606626608) bank43036 +43037 POINT(41.19726240917354 73.37846239964401) bank43037 +43038 POINT(41.434210378611624 74.77763599674775) bank43038 +43039 POINT(39.81363446999999 73.31993958126239) bank43039 +43040 POINT(40.988339383068706 73.40334212853979) bank43040 +43041 POINT(41.32995015866804 73.55508872013613) bank43041 +43042 POINT(39.865727905142464 74.415932779339) bank43042 +43043 POINT(40.18987427525131 73.59124035775721) bank43043 +43044 POINT(40.212294579022235 73.75481723009254) bank43044 +43045 POINT(39.790250029891105 73.52894525804022) bank43045 +43046 POINT(40.60015202062276 73.5666824637976) bank43046 +43047 POINT(39.90770974449392 73.6835431790164) bank43047 +43048 POINT(41.07414230610467 74.34355867694815) bank43048 +43049 POINT(41.546234042868925 74.16473253140289) bank43049 +43050 POINT(40.99824157282886 73.0203674710517) bank43050 +43051 POINT(41.42436951674777 74.41341266811361) bank43051 +43052 POINT(41.35001353724694 74.64684244934126) bank43052 +43053 POINT(40.82756038629788 74.89780278537589) bank43053 +43054 POINT(40.112578754615434 73.94506251675195) bank43054 +43055 POINT(41.380834503815535 74.71862584765769) bank43055 +43056 POINT(41.0276631113703 73.88042733686248) bank43056 +43057 POINT(40.05440126773779 73.48362163994764) bank43057 +43058 POINT(40.403169450174296 74.32118515227023) bank43058 +43059 POINT(40.590583646970636 74.63965279421178) bank43059 +43060 POINT(40.336333292100704 74.77030435805543) bank43060 +43061 POINT(39.73563936996959 74.98369724070095) bank43061 +43062 POINT(40.24724602045512 73.01320765332336) bank43062 +43063 POINT(40.90677538033273 74.44075477584683) bank43063 +43064 POINT(40.99266183402733 73.22091375098312) bank43064 +43065 POINT(41.59485016793047 74.24187923303022) bank43065 +43066 POINT(40.99564307181329 73.11507665007916) bank43066 +43067 POINT(40.69398764045116 74.94035286965011) bank43067 +43068 POINT(40.52894857579776 74.15233874405313) bank43068 +43069 POINT(41.543594790043166 74.82855972376838) bank43069 +43070 POINT(41.19841829021086 74.717152268626) bank43070 +43071 POINT(40.26467311253666 73.3915366168581) bank43071 +43072 POINT(40.02274127375841 73.91429489647034) bank43072 +43073 POINT(39.793961932740174 74.2861252959727) bank43073 +43074 POINT(41.10853085158176 73.67257265836885) bank43074 +43075 POINT(39.93884095066857 74.95119820065733) bank43075 +43076 POINT(41.21463246375068 73.67959408136555) bank43076 +43077 POINT(40.0205103824956 74.92492542799353) bank43077 +43078 POINT(41.581463570097945 74.2438578856012) bank43078 +43079 POINT(40.62002416775336 73.69185744515998) bank43079 +43080 POINT(40.265917055252956 74.3506419628879) bank43080 +43081 POINT(41.02297641924731 74.16181551909082) bank43081 +43082 POINT(41.11777446885021 73.04569156260195) bank43082 +43083 POINT(40.19977015716349 74.40880369830134) bank43083 +43084 POINT(41.11562628615837 74.32132106329037) bank43084 +43085 POINT(40.66936891783868 73.79738612592445) bank43085 +43086 POINT(41.372426119276305 74.80995888274344) bank43086 +43087 POINT(41.44245399631421 73.2631237013262) bank43087 +43088 POINT(40.262777682890736 73.71823449654741) bank43088 +43089 POINT(40.410078832230155 74.972149762191) bank43089 +43090 POINT(41.097133953286466 73.96970661959834) bank43090 +43091 POINT(41.531913080899194 74.6312563879077) bank43091 +43092 POINT(40.200549906815844 73.73267625880126) bank43092 +43093 POINT(40.45878316950162 73.43582030666106) bank43093 +43094 POINT(40.561339258608236 74.94332320721772) bank43094 +43095 POINT(40.83640169256702 73.32767619753736) bank43095 +43096 POINT(40.02508540666911 73.51171060936109) bank43096 +43097 POINT(40.51790961490478 74.34479541748172) bank43097 +43098 POINT(40.057118290618014 73.48858720605114) bank43098 +43099 POINT(39.816438871677526 74.18443050854003) bank43099 +43100 POINT(41.11199596676171 73.20496104384459) bank43100 +43101 POINT(41.45108067246548 73.30840470357103) bank43101 +43102 POINT(39.831518505836094 74.66678462957412) bank43102 +43103 POINT(40.221607566595964 73.380650232246) bank43103 +43104 POINT(41.610536689113076 73.47699376688408) bank43104 +43105 POINT(40.15917959674899 74.76788157851615) bank43105 +43106 POINT(41.690289420729215 73.08272253150162) bank43106 +43107 POINT(41.446343741059735 74.82953854933855) bank43107 +43108 POINT(40.07032909997549 74.98040098058262) bank43108 +43109 POINT(40.32985983057382 74.06692230294206) bank43109 +43110 POINT(41.052381412404486 73.55746464880437) bank43110 +43111 POINT(40.13408288018976 73.79277130899726) bank43111 +43112 POINT(40.639812945850196 74.23005238880101) bank43112 +43113 POINT(41.459247203664795 74.83370656755469) bank43113 +43114 POINT(40.41568349359562 73.91519940063013) bank43114 +43115 POINT(40.21688162912747 73.38218291992867) bank43115 +43116 POINT(39.91324418641648 73.12035403078025) bank43116 +43117 POINT(40.026847732830355 73.12181579526798) bank43117 +43118 POINT(40.55178437151092 74.06302599923433) bank43118 +43119 POINT(40.17070184847398 74.11393691264918) bank43119 +43120 POINT(39.829118420692 73.44235245134232) bank43120 +43121 POINT(40.8297831164041 73.97029596599053) bank43121 +43122 POINT(39.81447829797551 74.89190879282485) bank43122 +43123 POINT(40.23379601741472 74.65772320526179) bank43123 +43124 POINT(41.53656831614794 73.1117871306465) bank43124 +43125 POINT(41.62556511521676 74.40929701638056) bank43125 +43126 POINT(40.29446028692076 73.2566238486828) bank43126 +43127 POINT(40.02445763311652 73.49896195019451) bank43127 +43128 POINT(39.74958695602426 74.1232049350941) bank43128 +43129 POINT(41.5788522628725 73.66322856600951) bank43129 +43130 POINT(40.593283144121074 74.76042581047449) bank43130 +43131 POINT(40.34287585798928 73.26697466904686) bank43131 +43132 POINT(41.613416179345414 74.90067942684207) bank43132 +43133 POINT(41.0586056564351 73.1864320315728) bank43133 +43134 POINT(39.91233299100107 73.86665065666159) bank43134 +43135 POINT(40.08471087387687 73.39828067518131) bank43135 +43136 POINT(41.60053735289399 73.25874242717174) bank43136 +43137 POINT(40.823224192703876 74.56209092544312) bank43137 +43138 POINT(41.56495565054993 74.5041365624632) bank43138 +43139 POINT(40.65267864069462 74.45824727472484) bank43139 +43140 POINT(40.85735250681877 74.85321904940763) bank43140 +43141 POINT(40.568202751059346 73.22000216219739) bank43141 +43142 POINT(40.955659333426254 73.0156405483948) bank43142 +43143 POINT(40.04340248392266 73.71871370140822) bank43143 +43144 POINT(41.091058302934016 74.56532934638625) bank43144 +43145 POINT(41.69267189545522 73.92244064977635) bank43145 +43146 POINT(40.58445192683434 74.8327205700045) bank43146 +43147 POINT(41.68708891527624 74.02823743421179) bank43147 +43148 POINT(40.96342137896473 74.36887421759583) bank43148 +43149 POINT(40.20286888332795 74.80267101635756) bank43149 +43150 POINT(41.49386809414756 74.94858580874377) bank43150 +43151 POINT(40.81404382889161 74.77503471572896) bank43151 +43152 POINT(41.22853254691393 74.69666490113703) bank43152 +43153 POINT(40.932277211554535 73.39996184432687) bank43153 +43154 POINT(40.5153576108051 74.66612109204894) bank43154 +43155 POINT(41.51847216471861 74.2946811269433) bank43155 +43156 POINT(39.721132999551926 73.12537859499044) bank43156 +43157 POINT(41.52650549134311 74.91973658470212) bank43157 +43158 POINT(41.551371143667716 74.98294141049985) bank43158 +43159 POINT(41.578924347788856 73.25453121624558) bank43159 +43160 POINT(40.34338955574123 73.70146417236617) bank43160 +43161 POINT(41.41397811778831 74.8667686073524) bank43161 +43162 POINT(39.756279141705605 74.32754845546961) bank43162 +43163 POINT(40.218349632175084 73.5175581528048) bank43163 +43164 POINT(40.528339000417425 73.23817329111861) bank43164 +43165 POINT(39.98202057411368 74.93962815944107) bank43165 +43166 POINT(40.75100173431605 74.69155113912734) bank43166 +43167 POINT(41.30982723060579 73.42039977087109) bank43167 +43168 POINT(40.613629145975345 73.44161761810062) bank43168 +43169 POINT(39.72187256772081 73.56808052892038) bank43169 +43170 POINT(40.85570022987931 74.36391011766698) bank43170 +43171 POINT(41.595272515553084 73.17224930362572) bank43171 +43172 POINT(39.76881346347252 73.64231902631475) bank43172 +43173 POINT(41.06483851276935 74.16211544441293) bank43173 +43174 POINT(41.14014240551205 74.67217852075773) bank43174 +43175 POINT(40.059000045385034 73.24304090863181) bank43175 +43176 POINT(39.94122230507631 73.3951062381104) bank43176 +43177 POINT(41.55403697021405 74.53265465208717) bank43177 +43178 POINT(40.94253507309567 74.62837566577177) bank43178 +43179 POINT(41.278711072602015 74.37629173165774) bank43179 +43180 POINT(40.874001680995725 74.45826470739198) bank43180 +43181 POINT(40.4529242719578 74.28373709687904) bank43181 +43182 POINT(39.75546354533196 74.10151694887203) bank43182 +43183 POINT(40.27051729021787 73.83197672378202) bank43183 +43184 POINT(40.23367846956225 74.1107996208452) bank43184 +43185 POINT(41.081467141079145 74.69882408325425) bank43185 +43186 POINT(41.13601396445483 73.91215667858421) bank43186 +43187 POINT(41.189128929221 73.98537897822789) bank43187 +43188 POINT(41.062491961199676 73.62704406551393) bank43188 +43189 POINT(41.34441916024265 74.34240768752564) bank43189 +43190 POINT(40.272675657787026 74.2094510153267) bank43190 +43191 POINT(41.15879236278266 73.0497630294206) bank43191 +43192 POINT(40.27859918409605 73.48451460633501) bank43192 +43193 POINT(41.26951859147484 73.17040518480644) bank43193 +43194 POINT(40.000618478520884 73.16457804884328) bank43194 +43195 POINT(41.24038162262732 74.58319006770702) bank43195 +43196 POINT(39.79586761768166 74.47593522837045) bank43196 +43197 POINT(41.05434557569771 73.62510372165501) bank43197 +43198 POINT(40.28973316605104 73.49707076421014) bank43198 +43199 POINT(41.5726145199705 74.97542218399384) bank43199 +43200 POINT(41.28899933898739 73.39906720260755) bank43200 +43201 POINT(41.00237395157089 74.95988504368289) bank43201 +43202 POINT(41.306215134559054 73.42255424184039) bank43202 +43203 POINT(41.11753600145596 73.1120673745556) bank43203 +43204 POINT(41.01118031262003 74.44345745366371) bank43204 +43205 POINT(41.67686599407231 74.50236773380351) bank43205 +43206 POINT(41.20540828221365 74.79021288211325) bank43206 +43207 POINT(40.13440837237173 73.03173091886407) bank43207 +43208 POINT(40.56583840946498 73.83757450515378) bank43208 +43209 POINT(41.442920163064485 74.46706888586664) bank43209 +43210 POINT(40.00417335239911 73.08475840462526) bank43210 +43211 POINT(41.06115473242577 74.08593691526998) bank43211 +43212 POINT(40.88469567978211 74.87326390293127) bank43212 +43213 POINT(41.42765666706899 74.14008792008597) bank43213 +43214 POINT(40.82218004352549 73.73635530487523) bank43214 +43215 POINT(41.31403337005469 74.97593165510875) bank43215 +43216 POINT(40.5029419219099 73.27790293537777) bank43216 +43217 POINT(41.631748096630425 74.72935985323824) bank43217 +43218 POINT(39.74416450119733 73.79476789748011) bank43218 +43219 POINT(40.320587830090396 74.05914419122173) bank43219 +43220 POINT(41.587176127472524 73.4443356818331) bank43220 +43221 POINT(40.219418923828925 73.27446973038168) bank43221 +43222 POINT(41.06438261751053 74.05459613794571) bank43222 +43223 POINT(39.776858889459035 73.55940100294428) bank43223 +43224 POINT(41.29391829096363 73.87562346869026) bank43224 +43225 POINT(41.29047409841724 74.89181458471775) bank43225 +43226 POINT(40.89121770936128 74.4148428198482) bank43226 +43227 POINT(40.33218242531844 73.78135077316544) bank43227 +43228 POINT(40.31631661816847 74.66869616493926) bank43228 +43229 POINT(39.758360644936545 74.72965837368422) bank43229 +43230 POINT(41.48987482865575 74.79899173568826) bank43230 +43231 POINT(40.98141157666467 73.26893989660643) bank43231 +43232 POINT(40.94681486831835 73.27598279062417) bank43232 +43233 POINT(40.44994618902011 73.44659312004792) bank43233 +43234 POINT(40.81920459550267 74.6828922969631) bank43234 +43235 POINT(40.39313478680742 73.87638475599752) bank43235 +43236 POINT(40.96287553287503 74.03792914548866) bank43236 +43237 POINT(40.46323798894824 73.73616057243336) bank43237 +43238 POINT(40.62695010498007 74.02226658812424) bank43238 +43239 POINT(40.37113630334569 74.61670757527233) bank43239 +43240 POINT(40.717012943991776 74.7954223018793) bank43240 +43241 POINT(39.90571260169961 74.03068756457222) bank43241 +43242 POINT(40.20871184042436 74.98273238189948) bank43242 +43243 POINT(41.074206931168874 74.83279097983198) bank43243 +43244 POINT(41.01804680500228 73.10491328764608) bank43244 +43245 POINT(40.0472968407686 73.79504977787872) bank43245 +43246 POINT(40.13189730379083 73.29053663049883) bank43246 +43247 POINT(39.76421903758059 74.73620046427193) bank43247 +43248 POINT(41.67362881448985 73.97417647511323) bank43248 +43249 POINT(41.41615411733247 73.73977390710496) bank43249 +43250 POINT(41.00233181348254 73.32822184471054) bank43250 +43251 POINT(40.75971103188492 73.65668510356663) bank43251 +43252 POINT(41.23279138084448 74.30270253719348) bank43252 +43253 POINT(40.634065972977545 73.43231534854138) bank43253 +43254 POINT(39.86541821950569 73.38486591698701) bank43254 +43255 POINT(41.50249204775196 74.4550670823918) bank43255 +43256 POINT(40.34318017502186 73.26091886511307) bank43256 +43257 POINT(39.900618471886645 74.87123680698855) bank43257 +43258 POINT(39.75698604218417 74.11840062465753) bank43258 +43259 POINT(40.16292303014364 74.80514068583648) bank43259 +43260 POINT(41.4536762521582 73.84699845557867) bank43260 +43261 POINT(41.463459718125314 73.48554908388) bank43261 +43262 POINT(41.58394223854667 74.52821146694345) bank43262 +43263 POINT(39.8613919215329 74.02695957420477) bank43263 +43264 POINT(41.527425596991094 74.477259897446) bank43264 +43265 POINT(41.3288359241421 73.21552782789904) bank43265 +43266 POINT(40.94544695946251 74.21467025684477) bank43266 +43267 POINT(40.72472534740922 73.15109524860394) bank43267 +43268 POINT(41.24311415232965 74.61908653169084) bank43268 +43269 POINT(40.58553285710039 73.90858969090851) bank43269 +43270 POINT(40.55845462577672 73.44217524561483) bank43270 +43271 POINT(40.47766726032462 74.92172265004095) bank43271 +43272 POINT(41.261690160227964 74.94967805615468) bank43272 +43273 POINT(40.208860692981915 73.34508712094296) bank43273 +43274 POINT(41.44243704041016 73.38538558567106) bank43274 +43275 POINT(40.31490030484829 73.85261338312478) bank43275 +43276 POINT(41.23862900071608 74.92097439304908) bank43276 +43277 POINT(41.034174354023065 74.97025179525775) bank43277 +43278 POINT(39.8867898601868 73.18260319668998) bank43278 +43279 POINT(40.96001629287531 73.3285396528326) bank43279 +43280 POINT(40.04988845564899 73.43255229194094) bank43280 +43281 POINT(40.27647654720394 74.00654544958861) bank43281 +43282 POINT(40.3633426595918 74.32424988158755) bank43282 +43283 POINT(39.944643175168956 73.1032607822094) bank43283 +43284 POINT(39.76306655079376 74.8514628270292) bank43284 +43285 POINT(41.02212851915133 74.55070263840845) bank43285 +43286 POINT(40.704676573606704 73.24247536628083) bank43286 +43287 POINT(39.93390505501246 73.14763243783453) bank43287 +43288 POINT(41.66951503308914 74.71487773640132) bank43288 +43289 POINT(41.61954237868344 74.61047477627028) bank43289 +43290 POINT(40.552590143858914 74.07282229605157) bank43290 +43291 POINT(39.985342505101805 74.3803963367391) bank43291 +43292 POINT(40.67754748308049 73.18654706738114) bank43292 +43293 POINT(40.672906491004845 74.10517389791922) bank43293 +43294 POINT(40.35411396693121 74.98406332771522) bank43294 +43295 POINT(41.18253890956611 74.3762700622389) bank43295 +43296 POINT(39.76236474528674 73.89563729253977) bank43296 +43297 POINT(41.237198975205274 74.24831118061) bank43297 +43298 POINT(40.11355617975798 74.75773740397683) bank43298 +43299 POINT(41.185422837980276 74.71976221515717) bank43299 +43300 POINT(41.3703537188048 74.09369432700902) bank43300 +43301 POINT(40.566810723364796 74.83180138401917) bank43301 +43302 POINT(41.141551845281874 74.16924689284635) bank43302 +43303 POINT(40.96589158116684 73.41519969802054) bank43303 +43304 POINT(39.85014608702142 73.18626918305664) bank43304 +43305 POINT(40.54879773744568 73.60442076632874) bank43305 +43306 POINT(40.68143792858013 73.05856713158296) bank43306 +43307 POINT(40.55069688020996 74.71917151174432) bank43307 +43308 POINT(40.65166831350412 74.60410203683367) bank43308 +43309 POINT(41.05369912428711 73.5560275620159) bank43309 +43310 POINT(40.87579060619215 74.92167569607655) bank43310 +43311 POINT(39.772595945709966 74.25951434358389) bank43311 +43312 POINT(41.16740721879558 74.5155716400723) bank43312 +43313 POINT(40.61025042531446 73.2559650632036) bank43313 +43314 POINT(41.52766393148898 74.25437471006899) bank43314 +43315 POINT(40.022256681918094 74.09104177955612) bank43315 +43316 POINT(40.78180460529995 73.6889083509586) bank43316 +43317 POINT(41.41085983650245 73.98563767250718) bank43317 +43318 POINT(41.70979640926027 74.23253916579267) bank43318 +43319 POINT(40.338466482890794 74.08296063350795) bank43319 +43320 POINT(40.274751020453685 73.29551185812251) bank43320 +43321 POINT(39.84468915439402 74.52434262841074) bank43321 +43322 POINT(40.3311586061166 73.18917228110391) bank43322 +43323 POINT(40.903752210830746 73.95799494354279) bank43323 +43324 POINT(40.17373488628809 73.1974942466244) bank43324 +43325 POINT(41.26332414599852 74.34821828861818) bank43325 +43326 POINT(41.67746379264283 74.59445092210319) bank43326 +43327 POINT(41.11740577551104 73.47514310157533) bank43327 +43328 POINT(41.236020364124215 73.69835558925128) bank43328 +43329 POINT(39.84797249941133 74.95245388187077) bank43329 +43330 POINT(40.41010158629295 73.57976358447327) bank43330 +43331 POINT(40.37365263154535 74.6536500793185) bank43331 +43332 POINT(41.54829225815266 74.90545537540217) bank43332 +43333 POINT(41.427928036382156 74.36643153529684) bank43333 +43334 POINT(41.55457112908634 74.38894576522196) bank43334 +43335 POINT(41.50874202602887 74.97437772276355) bank43335 +43336 POINT(40.293469170397074 74.40417676102487) bank43336 +43337 POINT(41.29800091172038 74.71226996961333) bank43337 +43338 POINT(40.018373579547536 73.73716262993071) bank43338 +43339 POINT(41.16659907590801 74.80811687020085) bank43339 +43340 POINT(41.28921216958251 74.26938659951746) bank43340 +43341 POINT(41.18664895181147 73.25128790045763) bank43341 +43342 POINT(39.91502881500872 73.58416944788686) bank43342 +43343 POINT(40.896997202378444 73.49901191993595) bank43343 +43344 POINT(40.89485868574636 73.32989939804072) bank43344 +43345 POINT(40.685091026208255 74.52202399013908) bank43345 +43346 POINT(41.169744248720846 74.61657605557848) bank43346 +43347 POINT(40.26844544364054 74.24721672136037) bank43347 +43348 POINT(40.69152355314621 74.49706305803251) bank43348 +43349 POINT(40.52806002403763 73.4626753336162) bank43349 +43350 POINT(40.50883784179418 73.49029203553657) bank43350 +43351 POINT(41.61892365684162 73.08930281860272) bank43351 +43352 POINT(40.86825750386599 74.97932101441461) bank43352 +43353 POINT(39.75920815105821 73.7138842206301) bank43353 +43354 POINT(41.530561171612604 73.46716387123958) bank43354 +43355 POINT(39.73153278457893 74.33624284800707) bank43355 +43356 POINT(39.88656576109189 73.23470194574406) bank43356 +43357 POINT(39.832759719294764 74.47957091950481) bank43357 +43358 POINT(41.39919509423257 73.32422836737113) bank43358 +43359 POINT(39.841231026820985 74.1955697757484) bank43359 +43360 POINT(40.85002344354183 74.76014896561429) bank43360 +43361 POINT(40.080500268881394 74.5309599393417) bank43361 +43362 POINT(39.92963736804158 73.64045528773404) bank43362 +43363 POINT(40.87805095945709 73.61888746111124) bank43363 +43364 POINT(41.1194984652974 74.41151640988329) bank43364 +43365 POINT(39.80273569506505 74.27231785424726) bank43365 +43366 POINT(40.60655883478774 74.22915094078371) bank43366 +43367 POINT(40.43956994778295 74.50452184085442) bank43367 +43368 POINT(41.077785488066205 74.67621411777358) bank43368 +43369 POINT(40.75153952896595 74.20926878182205) bank43369 +43370 POINT(40.26814618247961 74.55782119267828) bank43370 +43371 POINT(40.701373414184815 74.52672746248398) bank43371 +43372 POINT(41.25112519131764 73.72240656097364) bank43372 +43373 POINT(40.7231645789635 73.20687519257747) bank43373 +43374 POINT(41.3077166677746 74.11446304876955) bank43374 +43375 POINT(41.14350897557815 74.13979603506989) bank43375 +43376 POINT(40.89635879234864 74.78316030403394) bank43376 +43377 POINT(41.17525105825085 73.13602214897291) bank43377 +43378 POINT(40.430123946162425 74.31216722135828) bank43378 +43379 POINT(39.91626759634039 74.45300780489518) bank43379 +43380 POINT(40.41678889897789 74.77133003614983) bank43380 +43381 POINT(41.42551914524193 74.12329212509613) bank43381 +43382 POINT(41.0720848242893 74.83397128116395) bank43382 +43383 POINT(41.28035564577034 74.12628775915013) bank43383 +43384 POINT(39.806233047346964 73.37191373494004) bank43384 +43385 POINT(41.25137122701943 74.0088252799871) bank43385 +43386 POINT(40.421924503215564 74.39781213386328) bank43386 +43387 POINT(40.21044049285283 73.06949315515092) bank43387 +43388 POINT(40.53763654803152 73.07835211851719) bank43388 +43389 POINT(40.77036188270974 74.0341391143775) bank43389 +43390 POINT(40.287250077230965 74.1363162606772) bank43390 +43391 POINT(40.44919631524822 73.83375020706146) bank43391 +43392 POINT(39.88178059696132 73.86408686432125) bank43392 +43393 POINT(40.7175119503756 74.8872184347168) bank43393 +43394 POINT(39.767052486793666 73.40075306933609) bank43394 +43395 POINT(40.54266142150663 73.16274171804429) bank43395 +43396 POINT(40.430576002630296 74.90764354943236) bank43396 +43397 POINT(40.10000937272408 73.68407963027043) bank43397 +43398 POINT(40.5676528046421 73.70839837857774) bank43398 +43399 POINT(40.06120986383773 73.96302310134107) bank43399 +43400 POINT(40.9889507772757 74.91469474333931) bank43400 +43401 POINT(39.939512141573516 74.35865551930735) bank43401 +43402 POINT(39.73710319644181 73.60415867742286) bank43402 +43403 POINT(41.3731694082894 74.35084684639705) bank43403 +43404 POINT(40.42111754425709 74.03082610864568) bank43404 +43405 POINT(41.46867216309686 73.20453048979655) bank43405 +43406 POINT(40.67060657925666 73.74654198110264) bank43406 +43407 POINT(40.874691004217595 74.59385627862287) bank43407 +43408 POINT(41.428572704606424 74.24710648092913) bank43408 +43409 POINT(41.54185318419231 74.20981584761711) bank43409 +43410 POINT(40.92646447332257 74.71898451826831) bank43410 +43411 POINT(41.63046149899069 74.01173549001997) bank43411 +43412 POINT(41.22906023292694 73.48351497151904) bank43412 +43413 POINT(39.823506508641756 73.48249508230006) bank43413 +43414 POINT(41.230591729405866 73.77779226274947) bank43414 +43415 POINT(41.38554941496823 74.63489271773419) bank43415 +43416 POINT(40.819373869669064 73.1364799068304) bank43416 +43417 POINT(40.99686055288018 74.51775313372423) bank43417 +43418 POINT(39.92418176580135 73.58344100889174) bank43418 +43419 POINT(39.84914276873272 74.22967194191956) bank43419 +43420 POINT(40.665766161676316 73.21729176874358) bank43420 +43421 POINT(40.04105609243838 73.24291755267807) bank43421 +43422 POINT(39.97698791549384 74.1840975941) bank43422 +43423 POINT(41.29029392102181 74.25894959367281) bank43423 +43424 POINT(40.71192595567664 74.22244281353431) bank43424 +43425 POINT(40.00990681481101 74.9075760966857) bank43425 +43426 POINT(40.71493660958301 74.33251393351662) bank43426 +43427 POINT(40.859796539651434 73.23365499753591) bank43427 +43428 POINT(39.95605093390332 74.37299794996531) bank43428 +43429 POINT(40.669900329441006 74.18833707256356) bank43429 +43430 POINT(39.75392742562838 74.07836303501715) bank43430 +43431 POINT(40.256497403398534 73.38227023699774) bank43431 +43432 POINT(39.845522349614185 73.35302403515075) bank43432 +43433 POINT(40.22116877392919 73.12957195973173) bank43433 +43434 POINT(41.52640670682723 74.47749451978534) bank43434 +43435 POINT(39.74087631869538 73.87981158809416) bank43435 +43436 POINT(41.020143553661924 74.69645458412899) bank43436 +43437 POINT(41.43545653392082 73.05012386266982) bank43437 +43438 POINT(41.663546808371784 74.94164656360027) bank43438 +43439 POINT(41.60568361298595 73.32229049876517) bank43439 +43440 POINT(40.200939184650245 73.3276032923764) bank43440 +43441 POINT(40.825130676685944 73.47165715278548) bank43441 +43442 POINT(39.86280489158928 74.24308167889251) bank43442 +43443 POINT(41.14967773070041 74.78822942221488) bank43443 +43444 POINT(40.828051055844156 74.19722146093598) bank43444 +43445 POINT(40.296596970318745 73.06292571735408) bank43445 +43446 POINT(40.8598101661105 74.28383187740265) bank43446 +43447 POINT(41.174325698522225 74.34429400021627) bank43447 +43448 POINT(40.386392876901006 73.8813382950198) bank43448 +43449 POINT(40.299707579467295 74.75685234885583) bank43449 +43450 POINT(41.312951816555284 74.95377500034311) bank43450 +43451 POINT(41.518579677645576 73.39441970086764) bank43451 +43452 POINT(40.60627933737536 73.1334596526149) bank43452 +43453 POINT(41.19814427582949 74.65148553969921) bank43453 +43454 POINT(40.70249882340659 74.97939786740606) bank43454 +43455 POINT(40.21470068561916 73.714614202871) bank43455 +43456 POINT(40.98207312087372 74.05009920061471) bank43456 +43457 POINT(41.47829141238156 74.05472824820943) bank43457 +43458 POINT(39.803868837257546 73.72977245297953) bank43458 +43459 POINT(39.793460688365435 74.70340201710276) bank43459 +43460 POINT(41.67736048167452 73.13717558503141) bank43460 +43461 POINT(40.713240405563944 73.8526060493459) bank43461 +43462 POINT(41.566683680870256 73.7313203950724) bank43462 +43463 POINT(40.453131623810876 73.57103170757952) bank43463 +43464 POINT(40.65907253048481 74.06971060020219) bank43464 +43465 POINT(40.31728862217713 73.34622454993472) bank43465 +43466 POINT(40.06411178940213 74.27184733161747) bank43466 +43467 POINT(41.47334183531591 73.39040307530189) bank43467 +43468 POINT(41.483162268867346 73.15146707198853) bank43468 +43469 POINT(39.980616973216705 73.27891540468872) bank43469 +43470 POINT(41.203850666176024 74.1646952453669) bank43470 +43471 POINT(40.989587935068414 74.69473183929877) bank43471 +43472 POINT(39.830609003569805 74.71730727646772) bank43472 +43473 POINT(40.839795510537506 74.07258700350194) bank43473 +43474 POINT(40.783256224695776 74.21183698853815) bank43474 +43475 POINT(41.22608391758359 73.13421029118527) bank43475 +43476 POINT(40.60918229842582 74.60563585645166) bank43476 +43477 POINT(41.559639162260886 73.02676438583399) bank43477 +43478 POINT(40.24739977684953 73.92552645149446) bank43478 +43479 POINT(39.871432999737976 74.20229831678157) bank43479 +43480 POINT(40.745809395592225 74.31608785765279) bank43480 +43481 POINT(41.65059138803777 73.7178706803768) bank43481 +43482 POINT(39.950502909681326 73.60137269542267) bank43482 +43483 POINT(41.04022655657186 74.43040663737996) bank43483 +43484 POINT(39.95260872288346 74.79607263816494) bank43484 +43485 POINT(40.33185509422707 74.49641199019469) bank43485 +43486 POINT(41.06309952343713 74.84284224908072) bank43486 +43487 POINT(41.48724089293747 74.47995611909586) bank43487 +43488 POINT(40.50094415039235 73.87817379541252) bank43488 +43489 POINT(40.84897487835691 74.3186108286475) bank43489 +43490 POINT(40.10889627032188 73.13799110675353) bank43490 +43491 POINT(40.00380573143744 73.80289166619454) bank43491 +43492 POINT(40.59873017337359 74.47789357491145) bank43492 +43493 POINT(40.221111394155244 73.90173431861963) bank43493 +43494 POINT(40.45145588170656 73.535420855361) bank43494 +43495 POINT(39.92525401109482 73.58886875660916) bank43495 +43496 POINT(41.468863093604945 74.70805609229382) bank43496 +43497 POINT(39.84457432206404 74.339107838578) bank43497 +43498 POINT(39.91831691124295 73.86462938714155) bank43498 +43499 POINT(40.50895702036286 73.2974026224598) bank43499 +43500 POINT(40.40865456979263 73.1950737458404) bank43500 +43501 POINT(41.57991832890926 73.26129971597634) bank43501 +43502 POINT(40.75165213416541 73.03733454664302) bank43502 +43503 POINT(41.36674087669362 73.89428427602459) bank43503 +43504 POINT(41.115130201354816 73.677478915198) bank43504 +43505 POINT(41.04371873588596 73.59993044453634) bank43505 +43506 POINT(39.75408269978743 73.1523367475617) bank43506 +43507 POINT(40.79564289707066 74.61773693956336) bank43507 +43508 POINT(41.020305048796885 74.33328233191261) bank43508 +43509 POINT(39.78323327629863 73.44361849997911) bank43509 +43510 POINT(40.17511072888213 73.50495829972118) bank43510 +43511 POINT(40.7786872950518 73.15898396681419) bank43511 +43512 POINT(39.960683162619155 74.03620396825008) bank43512 +43513 POINT(40.153567501332056 74.84718440885604) bank43513 +43514 POINT(41.059332985804716 74.05396180502414) bank43514 +43515 POINT(40.2122283497671 73.51424081957643) bank43515 +43516 POINT(40.91133904484791 74.98962140108114) bank43516 +43517 POINT(41.507946425854904 73.04329381932688) bank43517 +43518 POINT(41.6838953152011 73.69149500154167) bank43518 +43519 POINT(40.073234606126405 74.99463375023154) bank43519 +43520 POINT(39.83559599989304 73.58705683256632) bank43520 +43521 POINT(40.60685047830796 74.93175256649009) bank43521 +43522 POINT(41.64277392637546 74.27844769406931) bank43522 +43523 POINT(39.77174770607934 73.55421279482351) bank43523 +43524 POINT(40.96411637390631 74.24472766896176) bank43524 +43525 POINT(39.93964276763608 74.28330940007316) bank43525 +43526 POINT(41.0845660946695 74.0545459166484) bank43526 +43527 POINT(41.49069645185261 73.03231739986602) bank43527 +43528 POINT(41.22141717246776 73.50362234250572) bank43528 +43529 POINT(39.75041937194252 73.24559239533968) bank43529 +43530 POINT(41.083624616610415 73.18916152476717) bank43530 +43531 POINT(41.269483137027244 73.81498893416102) bank43531 +43532 POINT(40.007861880408086 74.18055016328596) bank43532 +43533 POINT(41.51394766431993 73.54010928413705) bank43533 +43534 POINT(41.01705740977081 74.05939787633805) bank43534 +43535 POINT(41.437037509642245 73.12540613581443) bank43535 +43536 POINT(40.118827013606904 74.69456981904376) bank43536 +43537 POINT(40.49947736034333 73.22701719926606) bank43537 +43538 POINT(41.26731320909155 73.88980174297582) bank43538 +43539 POINT(40.14208468153545 73.09633964203937) bank43539 +43540 POINT(40.821340772834525 73.10096485105808) bank43540 +43541 POINT(40.74458012755091 73.26501198332164) bank43541 +43542 POINT(39.842327037256425 74.32331794591536) bank43542 +43543 POINT(40.06507704717189 74.7500211344444) bank43543 +43544 POINT(41.3378558757919 73.11642219829383) bank43544 +43545 POINT(40.7041713744315 74.80053527807662) bank43545 +43546 POINT(41.08411339881139 73.44231603868317) bank43546 +43547 POINT(40.90337161804136 74.19996338693625) bank43547 +43548 POINT(41.44982004504211 74.7657728103199) bank43548 +43549 POINT(39.71285295208894 74.43110940064857) bank43549 +43550 POINT(40.7774390946574 74.9672365723365) bank43550 +43551 POINT(41.462587590508825 73.76233969478601) bank43551 +43552 POINT(40.37930474860695 73.7018361528812) bank43552 +43553 POINT(40.83345101805552 74.98374189599163) bank43553 +43554 POINT(40.161741685252444 74.38789603920002) bank43554 +43555 POINT(40.34553416700604 74.22735085639704) bank43555 +43556 POINT(41.119421558537326 74.40206376274182) bank43556 +43557 POINT(40.36861799888702 73.27009264500812) bank43557 +43558 POINT(40.403555511509 73.59568234286995) bank43558 +43559 POINT(40.452037975545 73.71959259789726) bank43559 +43560 POINT(40.420101759446496 73.43379908232454) bank43560 +43561 POINT(41.55856146044304 73.77958393521618) bank43561 +43562 POINT(40.2070005732802 73.6547368799274) bank43562 +43563 POINT(41.177753883800484 74.48939177655105) bank43563 +43564 POINT(40.79211328079321 74.95817959072795) bank43564 +43565 POINT(41.69710280647375 73.5182215920094) bank43565 +43566 POINT(41.45753875244125 74.61943187875667) bank43566 +43567 POINT(41.327998340899704 73.47500808135239) bank43567 +43568 POINT(40.247771517606544 74.71051131349563) bank43568 +43569 POINT(41.448468466077635 73.0360067173498) bank43569 +43570 POINT(40.95193824635493 73.61807677665963) bank43570 +43571 POINT(41.316915165519575 74.96140898837207) bank43571 +43572 POINT(41.29554424309393 73.0515749424703) bank43572 +43573 POINT(41.34787092194778 73.33431545418391) bank43573 +43574 POINT(40.605420689428435 73.27972206187668) bank43574 +43575 POINT(40.40785998689545 73.78442841772426) bank43575 +43576 POINT(40.458993564277506 74.32769655970336) bank43576 +43577 POINT(40.93012772916815 73.86008570593344) bank43577 +43578 POINT(40.38832722881111 73.9861042737668) bank43578 +43579 POINT(40.67282951679907 74.17308805759804) bank43579 +43580 POINT(41.4669776519867 74.22739294560782) bank43580 +43581 POINT(40.96807144265335 74.99871033900536) bank43581 +43582 POINT(39.93837328999909 73.72090046676655) bank43582 +43583 POINT(40.52578483680773 73.51398079113247) bank43583 +43584 POINT(40.40512434139203 74.27702395039216) bank43584 +43585 POINT(39.748267330413675 73.25917926240626) bank43585 +43586 POINT(39.72275076989551 74.61606458774787) bank43586 +43587 POINT(40.468819544303535 73.0777487237897) bank43587 +43588 POINT(41.400510016899446 74.08774060365641) bank43588 +43589 POINT(39.96745694296473 73.38036339216772) bank43589 +43590 POINT(40.173020762920245 73.7705622287451) bank43590 +43591 POINT(41.66397777573881 74.76086203686495) bank43591 +43592 POINT(40.59697985974156 74.88807596543157) bank43592 +43593 POINT(40.526386266303675 74.34232679523454) bank43593 +43594 POINT(41.26968921486229 74.52257908583609) bank43594 +43595 POINT(40.343051908781916 74.86779002103337) bank43595 +43596 POINT(39.75561249643705 74.18218911623474) bank43596 +43597 POINT(40.69429568714033 74.41902840453317) bank43597 +43598 POINT(40.88980772527812 73.10359337350121) bank43598 +43599 POINT(39.93675946767875 73.12363818228062) bank43599 +43600 POINT(40.406050083511744 74.90010200047003) bank43600 +43601 POINT(40.396969903063095 73.97775303266093) bank43601 +43602 POINT(41.61759833360733 73.9836581855198) bank43602 +43603 POINT(39.993458542105415 74.9695105503078) bank43603 +43604 POINT(40.19583989180725 74.67917934310672) bank43604 +43605 POINT(40.465108331661376 73.58997246488673) bank43605 +43606 POINT(40.030990733974974 74.30512410865263) bank43606 +43607 POINT(40.7783244560662 74.48269112116104) bank43607 +43608 POINT(39.99393294572121 74.9913490558133) bank43608 +43609 POINT(41.31109255403864 73.78030703412671) bank43609 +43610 POINT(40.07035604653778 74.43892764855896) bank43610 +43611 POINT(40.25047485206894 74.09327740283724) bank43611 +43612 POINT(40.70736652089178 73.67753925811218) bank43612 +43613 POINT(40.90820319609021 73.21597629173127) bank43613 +43614 POINT(40.55230330326902 74.34216447836315) bank43614 +43615 POINT(41.693533533683066 74.23477226672551) bank43615 +43616 POINT(40.27086234877699 73.97142499982975) bank43616 +43617 POINT(40.072352416937626 73.2827477183052) bank43617 +43618 POINT(40.66991311160744 74.87628882685522) bank43618 +43619 POINT(40.92380688635082 74.84292360662155) bank43619 +43620 POINT(39.81171127156023 73.73757029906963) bank43620 +43621 POINT(41.10722233964543 74.19586295629341) bank43621 +43622 POINT(41.36357290657416 74.85106465919263) bank43622 +43623 POINT(41.70656892008692 73.9369953791722) bank43623 +43624 POINT(41.05623073250032 74.64384141417486) bank43624 +43625 POINT(41.58436508669455 74.81958606880326) bank43625 +43626 POINT(40.6334888562135 74.60154009579922) bank43626 +43627 POINT(40.856340295570384 74.78178608581672) bank43627 +43628 POINT(41.529845118911055 73.44964247400328) bank43628 +43629 POINT(41.03908767422244 73.5610670688836) bank43629 +43630 POINT(40.10049454118298 74.63440855930345) bank43630 +43631 POINT(41.00033253018803 74.46017047951773) bank43631 +43632 POINT(40.12231316195204 74.22583132718124) bank43632 +43633 POINT(40.76139353485933 73.70871327949915) bank43633 +43634 POINT(40.44148637584413 73.39032195510588) bank43634 +43635 POINT(40.74020963432022 74.7584806376738) bank43635 +43636 POINT(40.25432046123352 73.99984235801308) bank43636 +43637 POINT(41.024270200515495 73.06136042371084) bank43637 +43638 POINT(40.132687433780056 74.36796428521164) bank43638 +43639 POINT(40.32504588625732 74.88500372987019) bank43639 +43640 POINT(39.97627025636523 74.46145547636286) bank43640 +43641 POINT(40.33286652024205 74.19469127001845) bank43641 +43642 POINT(41.00484998579266 73.47394998045156) bank43642 +43643 POINT(41.03455701476344 74.91186714020823) bank43643 +43644 POINT(41.628903651592914 74.51319919088269) bank43644 +43645 POINT(41.35822383938644 73.49145402598123) bank43645 +43646 POINT(40.915669418742745 73.3846601636428) bank43646 +43647 POINT(41.30718219845116 74.89933892663197) bank43647 +43648 POINT(41.38390893850108 74.75622700192523) bank43648 +43649 POINT(41.009325832454884 74.6603130510243) bank43649 +43650 POINT(40.55596599731759 73.85132498135322) bank43650 +43651 POINT(40.246373272058094 73.05043665719569) bank43651 +43652 POINT(41.30789768867086 74.68586949721818) bank43652 +43653 POINT(39.89858437837947 74.55669839145372) bank43653 +43654 POINT(40.916982879272595 74.96914630490748) bank43654 +43655 POINT(41.56939482885535 74.30677556455807) bank43655 +43656 POINT(40.03227410024223 73.6837310715664) bank43656 +43657 POINT(41.34666752527899 74.19364834450452) bank43657 +43658 POINT(40.62848143794345 74.86481439108158) bank43658 +43659 POINT(40.63062038587681 73.315607160105) bank43659 +43660 POINT(40.30046780035403 73.72045781612671) bank43660 +43661 POINT(40.33404614373406 74.62448573413128) bank43661 +43662 POINT(40.45989573858117 74.71350269143997) bank43662 +43663 POINT(41.1204947853186 73.47688322082686) bank43663 +43664 POINT(40.03673882781041 73.02880000865139) bank43664 +43665 POINT(41.61322676019771 74.56302071675138) bank43665 +43666 POINT(40.817347732050465 74.97527943469991) bank43666 +43667 POINT(40.054429386953565 74.49978836794826) bank43667 +43668 POINT(39.90275690944195 73.49564966150267) bank43668 +43669 POINT(41.649006119293276 74.20223724468248) bank43669 +43670 POINT(40.54633324218639 74.75989723664948) bank43670 +43671 POINT(40.065117238596486 73.27452430157703) bank43671 +43672 POINT(39.86209329898956 74.66958995977109) bank43672 +43673 POINT(39.92249229817051 73.39521456032526) bank43673 +43674 POINT(40.162971336484354 74.3965968594187) bank43674 +43675 POINT(41.08147993613305 73.3882058029426) bank43675 +43676 POINT(40.00655268089944 73.14131219777657) bank43676 +43677 POINT(40.079341052479656 73.6199817172189) bank43677 +43678 POINT(40.006492207374855 73.83487845054772) bank43678 +43679 POINT(40.000942436440766 73.33840521663454) bank43679 +43680 POINT(41.22133927317237 73.92919916039816) bank43680 +43681 POINT(41.49392754491939 73.88379343550962) bank43681 +43682 POINT(40.42988748763519 74.34536510835858) bank43682 +43683 POINT(40.73511232291807 74.48501737901377) bank43683 +43684 POINT(41.00017542574862 73.9289633447571) bank43684 +43685 POINT(39.75564565204045 74.75094440625249) bank43685 +43686 POINT(41.536142047091936 73.15596080068387) bank43686 +43687 POINT(40.796015115369066 73.27763713438515) bank43687 +43688 POINT(40.01410640480702 74.96316591101974) bank43688 +43689 POINT(41.07986630777772 74.64162006280746) bank43689 +43690 POINT(40.10545165840007 73.18679428562642) bank43690 +43691 POINT(40.41830125822984 74.73367814305433) bank43691 +43692 POINT(39.90631222080032 73.50041456038296) bank43692 +43693 POINT(40.819021184579526 73.48269541835593) bank43693 +43694 POINT(41.20521435320185 74.12694257058263) bank43694 +43695 POINT(40.60282096623978 73.109262466141) bank43695 +43696 POINT(40.17737033763515 73.15036700922393) bank43696 +43697 POINT(40.448790832787424 74.7423068481291) bank43697 +43698 POINT(39.896688976809486 74.80048327788505) bank43698 +43699 POINT(39.772633151358356 74.18745487014874) bank43699 +43700 POINT(41.13991331704005 73.60318019558257) bank43700 +43701 POINT(41.100446643464636 74.44397450635206) bank43701 +43702 POINT(39.72476388150366 73.80832251177061) bank43702 +43703 POINT(40.749945418538736 73.70375002796072) bank43703 +43704 POINT(40.25889914593604 74.81263756441777) bank43704 +43705 POINT(40.34983179053549 74.64485111401724) bank43705 +43706 POINT(39.819312534853275 73.85950480087752) bank43706 +43707 POINT(40.30405241411219 74.05397309341662) bank43707 +43708 POINT(39.94992540434017 74.57280922139705) bank43708 +43709 POINT(40.419285490220425 74.55874608294575) bank43709 +43710 POINT(40.351883719340165 73.47959757312442) bank43710 +43711 POINT(39.99621867464819 74.77019917584433) bank43711 +43712 POINT(40.35701614476267 74.83622824857092) bank43712 +43713 POINT(40.81513603824315 73.13198084668402) bank43713 +43714 POINT(41.04809320380408 73.15807201523285) bank43714 +43715 POINT(40.936074065433075 73.73663871676251) bank43715 +43716 POINT(40.67669797343461 74.82753098979151) bank43716 +43717 POINT(40.12790759910607 74.71500456652855) bank43717 +43718 POINT(40.16939078691921 73.86311136223189) bank43718 +43719 POINT(41.67548662980838 73.89893410941178) bank43719 +43720 POINT(41.57294068469813 73.30309362379356) bank43720 +43721 POINT(41.12581895392345 74.5428838076038) bank43721 +43722 POINT(40.591736783215964 74.68722713704574) bank43722 +43723 POINT(40.64221305818363 73.12452418381953) bank43723 +43724 POINT(40.645373437182705 73.10196094304271) bank43724 +43725 POINT(39.9953375580725 73.05464813563319) bank43725 +43726 POINT(40.08477778922504 73.1088508899713) bank43726 +43727 POINT(41.34650129229006 73.1129574114491) bank43727 +43728 POINT(40.711828760384634 74.69411472452298) bank43728 +43729 POINT(39.98227933681184 74.63245943646221) bank43729 +43730 POINT(40.45204134240719 73.84426112820867) bank43730 +43731 POINT(39.74822078276248 74.40348192179981) bank43731 +43732 POINT(41.391052373161806 73.01492470517042) bank43732 +43733 POINT(40.498892224386765 74.9893924478778) bank43733 +43734 POINT(40.45810502604177 74.44984529085714) bank43734 +43735 POINT(41.25910325583085 74.17272580425315) bank43735 +43736 POINT(40.85517382547583 73.8628123923864) bank43736 +43737 POINT(40.45315498786251 73.89899635061003) bank43737 +43738 POINT(40.89235831551508 73.90042394933847) bank43738 +43739 POINT(41.16426138878745 73.11250244918239) bank43739 +43740 POINT(40.115013367107494 74.54638610641979) bank43740 +43741 POINT(40.214047584101536 73.47924895812305) bank43741 +43742 POINT(40.493914483786384 74.61846298196619) bank43742 +43743 POINT(40.99362147047639 73.35019104059613) bank43743 +43744 POINT(39.850330933581645 73.92805461302159) bank43744 +43745 POINT(40.112143099647255 73.61352037162477) bank43745 +43746 POINT(40.88316486166844 73.85411248299496) bank43746 +43747 POINT(40.8861542896269 74.92317394433718) bank43747 +43748 POINT(40.63623477705846 73.59564077601308) bank43748 +43749 POINT(40.39138955786003 73.82497930560955) bank43749 +43750 POINT(41.26211747542206 74.18120442740519) bank43750 +43751 POINT(39.86560518244642 74.60154154413863) bank43751 +43752 POINT(40.02191693707931 73.19276552231575) bank43752 +43753 POINT(40.673186971262645 74.59045112149809) bank43753 +43754 POINT(39.87639293316916 73.90433711753856) bank43754 +43755 POINT(39.81903920912837 74.38427385486119) bank43755 +43756 POINT(41.45148937234842 73.6533407313449) bank43756 +43757 POINT(40.978371114321035 74.84537772750402) bank43757 +43758 POINT(40.387512484366056 73.89986147526525) bank43758 +43759 POINT(40.55711655697003 73.9922775505161) bank43759 +43760 POINT(40.37312174163446 73.05808571040046) bank43760 +43761 POINT(39.74117189167277 74.31370999486009) bank43761 +43762 POINT(41.06160356153297 74.92825483120744) bank43762 +43763 POINT(40.831397037684866 73.22646330401892) bank43763 +43764 POINT(39.76387582727754 74.10502043553923) bank43764 +43765 POINT(40.007676273895605 74.64092966265626) bank43765 +43766 POINT(40.677132956835926 74.0189130103851) bank43766 +43767 POINT(39.85887429424307 74.15296751926938) bank43767 +43768 POINT(40.36103928671533 73.62777745072134) bank43768 +43769 POINT(41.564374584888874 74.76172037155008) bank43769 +43770 POINT(39.922234516192084 74.52596679799355) bank43770 +43771 POINT(41.151422767610775 74.50599449580858) bank43771 +43772 POINT(41.0203314384204 73.58931781271629) bank43772 +43773 POINT(41.032300184601056 74.52241275754317) bank43773 +43774 POINT(40.759735453446694 73.2130980413433) bank43774 +43775 POINT(40.251858672294276 74.64788955545772) bank43775 +43776 POINT(41.44169925941131 74.07906638966911) bank43776 +43777 POINT(41.10086342056337 73.17059444150867) bank43777 +43778 POINT(40.86285140927484 73.6354576069022) bank43778 +43779 POINT(40.726986891623284 74.7158244224743) bank43779 +43780 POINT(40.02593455145455 74.9258941020054) bank43780 +43781 POINT(40.86212299562778 74.81257346666922) bank43781 +43782 POINT(40.746651123726615 74.7689217247257) bank43782 +43783 POINT(41.447949104778885 73.4259518496994) bank43783 +43784 POINT(40.911124005478214 74.73700566530083) bank43784 +43785 POINT(41.46855575450251 74.5565843331737) bank43785 +43786 POINT(40.413430585671804 73.26194273480198) bank43786 +43787 POINT(40.47638139113268 74.85871841841244) bank43787 +43788 POINT(41.25245700323844 73.67671848406235) bank43788 +43789 POINT(39.8535194702585 74.71402107125131) bank43789 +43790 POINT(41.1869309950624 73.83136429297511) bank43790 +43791 POINT(40.09765482158539 73.88961637668605) bank43791 +43792 POINT(39.820581340994416 74.83731766856998) bank43792 +43793 POINT(40.23206070400526 73.54284981360206) bank43793 +43794 POINT(40.868152159282616 74.25633221401388) bank43794 +43795 POINT(40.38803071529366 74.45726931639786) bank43795 +43796 POINT(41.70817921911808 73.69363852056071) bank43796 +43797 POINT(41.669191276335454 74.99362078373979) bank43797 +43798 POINT(40.29898495045979 73.63136136124331) bank43798 +43799 POINT(40.67005338384449 74.6509015660846) bank43799 +43800 POINT(41.008167481876384 74.29070033966133) bank43800 +43801 POINT(39.965344656471316 73.34654378580767) bank43801 +43802 POINT(40.25242857894306 73.84044321930509) bank43802 +43803 POINT(41.693980444805355 73.88735147204183) bank43803 +43804 POINT(39.959130684889175 74.56745178605608) bank43804 +43805 POINT(40.93528100982173 73.36837336151903) bank43805 +43806 POINT(40.74473205969285 73.26037575221639) bank43806 +43807 POINT(41.6432118335142 73.77600122457847) bank43807 +43808 POINT(40.23855144737849 73.1256619958429) bank43808 +43809 POINT(40.05001905321345 73.38802132818) bank43809 +43810 POINT(40.001227595673676 73.7958235600034) bank43810 +43811 POINT(40.954237745589836 73.95679228419591) bank43811 +43812 POINT(41.61270496281327 74.07456009286642) bank43812 +43813 POINT(41.37699282635442 73.68590449622792) bank43813 +43814 POINT(39.75998905969179 74.55947838700544) bank43814 +43815 POINT(39.96233977449501 75.00435297081972) bank43815 +43816 POINT(39.85784524495666 73.24139508669917) bank43816 +43817 POINT(40.7825022653714 73.68615863866619) bank43817 +43818 POINT(41.70249707466186 73.62876429348975) bank43818 +43819 POINT(40.652998635841406 74.27868310365932) bank43819 +43820 POINT(40.84808513650442 74.93818884889745) bank43820 +43821 POINT(40.52432146864837 74.19583900580695) bank43821 +43822 POINT(41.11073598188152 74.90345478965928) bank43822 +43823 POINT(41.68024875509849 74.71391245344545) bank43823 +43824 POINT(40.12592902702076 74.26239767626353) bank43824 +43825 POINT(41.287649725064455 73.18809784568363) bank43825 +43826 POINT(39.82254862877052 73.63678034224611) bank43826 +43827 POINT(40.79368962424096 73.67748212788426) bank43827 +43828 POINT(39.91740644127003 73.14235606567814) bank43828 +43829 POINT(40.95818287030832 74.12958955572422) bank43829 +43830 POINT(41.70049375768093 73.26823008236079) bank43830 +43831 POINT(40.581543483079024 74.51981913514466) bank43831 +43832 POINT(40.6569683110705 74.81960241230165) bank43832 +43833 POINT(40.286100514741875 73.9544150096076) bank43833 +43834 POINT(41.45998356454152 74.48786334758252) bank43834 +43835 POINT(40.15074429089588 74.87246858778953) bank43835 +43836 POINT(39.91824975536292 73.14346097514007) bank43836 +43837 POINT(40.819186378854376 73.43588719354878) bank43837 +43838 POINT(39.86972929959033 73.3162077184772) bank43838 +43839 POINT(40.418292750370384 73.45408683942803) bank43839 +43840 POINT(40.96986038245581 74.83619215112759) bank43840 +43841 POINT(39.97786517690681 73.7342089164915) bank43841 +43842 POINT(41.46319780890241 74.63445447203036) bank43842 +43843 POINT(40.19836102228901 74.11964413594139) bank43843 +43844 POINT(41.17217817711034 74.69042122738867) bank43844 +43845 POINT(41.40700631952615 73.68770604160312) bank43845 +43846 POINT(41.29341943631173 74.18860599926198) bank43846 +43847 POINT(40.595828567530184 73.46399613647736) bank43847 +43848 POINT(39.874630579896106 74.81970333026564) bank43848 +43849 POINT(41.49166302499154 74.44690275586545) bank43849 +43850 POINT(39.801234213629144 73.35623835129564) bank43850 +43851 POINT(41.02722275355162 73.66470159444798) bank43851 +43852 POINT(41.5113377082062 74.4976499768168) bank43852 +43853 POINT(40.51366080618372 74.48582180257203) bank43853 +43854 POINT(41.29192232735045 74.23672320501845) bank43854 +43855 POINT(40.53574346306271 74.39441416212426) bank43855 +43856 POINT(40.400326419254824 73.76298321588612) bank43856 +43857 POINT(41.1979636329604 74.96398413975163) bank43857 +43858 POINT(39.76059034546315 74.21122123621588) bank43858 +43859 POINT(40.31745547632332 74.9596424476528) bank43859 +43860 POINT(40.738529970951554 74.48466947096105) bank43860 +43861 POINT(40.97996618069167 73.94366124539359) bank43861 +43862 POINT(41.221475372257416 73.48085715334838) bank43862 +43863 POINT(41.31631688511479 73.28716532677204) bank43863 +43864 POINT(40.554213893664546 73.98514176911391) bank43864 +43865 POINT(41.40354266606629 73.88422330594636) bank43865 +43866 POINT(41.39114286635476 73.5621813671666) bank43866 +43867 POINT(40.58612872676072 74.74695064277722) bank43867 +43868 POINT(41.40124609393691 74.5141915206429) bank43868 +43869 POINT(41.38646450592722 73.22189926593386) bank43869 +43870 POINT(41.52823036184838 73.52031400312553) bank43870 +43871 POINT(41.63972990645092 73.80460793276349) bank43871 +43872 POINT(40.89044119432454 74.39440556587874) bank43872 +43873 POINT(41.18073576127934 74.81961231627608) bank43873 +43874 POINT(41.45064617750436 74.78514159587706) bank43874 +43875 POINT(40.35925657678336 74.86069566522357) bank43875 +43876 POINT(41.708334014593014 74.62685300857696) bank43876 +43877 POINT(41.21786653660694 73.1943092760412) bank43877 +43878 POINT(40.46222500941128 73.67576918786946) bank43878 +43879 POINT(41.232033018068854 74.27059021949782) bank43879 +43880 POINT(40.0483366121437 74.56824482784565) bank43880 +43881 POINT(40.1161549527358 74.59519363885002) bank43881 +43882 POINT(39.75518955741855 74.50693297024294) bank43882 +43883 POINT(40.25916912303861 74.70843792996826) bank43883 +43884 POINT(41.142550165879605 74.63246542427257) bank43884 +43885 POINT(40.463537420676154 73.91622638774075) bank43885 +43886 POINT(40.214926890936994 73.43759144670581) bank43886 +43887 POINT(40.75885612539318 74.93081197185718) bank43887 +43888 POINT(39.82476588858282 74.04948710427307) bank43888 +43889 POINT(40.18257123539824 73.71432416868058) bank43889 +43890 POINT(40.70734631882745 74.12952952938728) bank43890 +43891 POINT(39.815398611559395 73.05904564924154) bank43891 +43892 POINT(39.83404301310806 73.39974529516795) bank43892 +43893 POINT(41.69342183946257 73.90466414218967) bank43893 +43894 POINT(41.31958564890693 73.57018888374874) bank43894 +43895 POINT(40.72040351057535 73.70236816974624) bank43895 +43896 POINT(40.67020211807613 74.23444949281885) bank43896 +43897 POINT(41.031542587463505 73.24626066815958) bank43897 +43898 POINT(39.78570887249352 73.52384311183411) bank43898 +43899 POINT(40.43173120275127 73.78306539116463) bank43899 +43900 POINT(41.261997393841646 73.21970444607824) bank43900 +43901 POINT(41.17063694123223 74.9096269431558) bank43901 +43902 POINT(40.42163360524853 73.74159632188328) bank43902 +43903 POINT(40.112450780001836 74.88038643279866) bank43903 +43904 POINT(40.20201042428167 74.72217019406537) bank43904 +43905 POINT(40.998995603626625 74.52337930921777) bank43905 +43906 POINT(41.25850233380924 74.98844785546677) bank43906 +43907 POINT(40.43621733802804 74.35551444605537) bank43907 +43908 POINT(41.6578742074591 74.14055053465106) bank43908 +43909 POINT(41.07690156533901 74.15501954291926) bank43909 +43910 POINT(40.733195868455404 73.4314090792943) bank43910 +43911 POINT(40.90146456558377 73.30066480619735) bank43911 +43912 POINT(40.83304864287109 73.75319696749698) bank43912 +43913 POINT(41.17310162335539 73.48207417809245) bank43913 +43914 POINT(41.090184538906485 73.60033797346927) bank43914 +43915 POINT(40.28830045850041 74.2439879967415) bank43915 +43916 POINT(40.62975315138985 74.01643353935353) bank43916 +43917 POINT(40.10901349942922 73.93571091107874) bank43917 +43918 POINT(40.66384613016181 73.55100374220717) bank43918 +43919 POINT(41.687322599942576 74.00809386776393) bank43919 +43920 POINT(41.40909485076789 74.38471587312065) bank43920 +43921 POINT(41.05210993669354 74.70348546012222) bank43921 +43922 POINT(40.11590681968964 73.31742611859809) bank43922 +43923 POINT(41.539684596867076 73.8596786017161) bank43923 +43924 POINT(41.05682339858216 73.76250017337917) bank43924 +43925 POINT(41.36420879750502 73.99985157277399) bank43925 +43926 POINT(40.12514285030736 74.38574407372583) bank43926 +43927 POINT(40.47361085816873 73.82206003111268) bank43927 +43928 POINT(39.744442117643715 74.52505043398075) bank43928 +43929 POINT(41.28014478689973 74.14785521343546) bank43929 +43930 POINT(39.90707579688084 74.77433671489284) bank43930 +43931 POINT(40.92499886170999 74.51756390889926) bank43931 +43932 POINT(40.97859399247091 74.69212915427974) bank43932 +43933 POINT(40.83372700673318 74.53331910810265) bank43933 +43934 POINT(41.68599017242172 74.42634707003799) bank43934 +43935 POINT(40.9148975287107 74.77336330762172) bank43935 +43936 POINT(41.44944012364194 73.63003647659076) bank43936 +43937 POINT(40.689567547783554 74.26344880766052) bank43937 +43938 POINT(41.0468464168669 73.09847450822353) bank43938 +43939 POINT(40.08513805908728 73.45409162450541) bank43939 +43940 POINT(41.082905522462056 73.73193254867176) bank43940 +43941 POINT(40.23415292959032 74.44114484524061) bank43941 +43942 POINT(40.55684663836778 73.36234370313935) bank43942 +43943 POINT(40.037064044636914 73.14193011684945) bank43943 +43944 POINT(40.45835472554117 74.09934068923987) bank43944 +43945 POINT(40.44408612773928 73.03440865358267) bank43945 +43946 POINT(39.99059971313115 74.98479880477623) bank43946 +43947 POINT(41.49731596165802 74.10365673518474) bank43947 +43948 POINT(39.970857575798725 74.70119740830893) bank43948 +43949 POINT(40.76722508296255 73.8851610668953) bank43949 +43950 POINT(40.6509539215565 73.54441049800234) bank43950 +43951 POINT(41.6575838007953 74.02386002691438) bank43951 +43952 POINT(40.93608422674002 74.61097949640258) bank43952 +43953 POINT(41.333734418200926 73.85378611235834) bank43953 +43954 POINT(40.64755791859883 73.7614700580927) bank43954 +43955 POINT(40.05994248720584 73.8763242910751) bank43955 +43956 POINT(40.73676455447564 74.4352202311889) bank43956 +43957 POINT(41.1667017264508 74.44111560716672) bank43957 +43958 POINT(40.238292427942916 74.44924904440144) bank43958 +43959 POINT(40.729262612449475 74.24312995069647) bank43959 +43960 POINT(41.33075805062768 74.66864359452195) bank43960 +43961 POINT(40.56320880743596 73.99508029990204) bank43961 +43962 POINT(40.61522743158751 73.5458373119144) bank43962 +43963 POINT(41.19249944314871 73.81698236151497) bank43963 +43964 POINT(41.441309112612366 74.02987022039127) bank43964 +43965 POINT(40.95304147870971 74.77994381157656) bank43965 +43966 POINT(41.35542054384651 73.6021806276013) bank43966 +43967 POINT(40.860473681721636 73.79721076373333) bank43967 +43968 POINT(40.78903688178331 73.93401084009936) bank43968 +43969 POINT(40.436649955835975 74.72243470653574) bank43969 +43970 POINT(40.55886079670748 73.65306965767991) bank43970 +43971 POINT(40.82723920248562 73.19668496964381) bank43971 +43972 POINT(41.51741081692263 73.27524805801954) bank43972 +43973 POINT(40.96980550988656 73.95222926481877) bank43973 +43974 POINT(39.995742215499 73.69706101539788) bank43974 +43975 POINT(40.860461536474396 74.68570181055011) bank43975 +43976 POINT(40.310662789777936 73.7610200032017) bank43976 +43977 POINT(40.68188324650758 73.73634960997718) bank43977 +43978 POINT(40.66703989803706 73.93783211710758) bank43978 +43979 POINT(41.54701713806736 74.88762380913423) bank43979 +43980 POINT(40.458880139386075 73.71297171817251) bank43980 +43981 POINT(40.57278802187637 73.74461399046197) bank43981 +43982 POINT(41.275224751647826 74.12565190898363) bank43982 +43983 POINT(41.20370431455775 74.66309752035913) bank43983 +43984 POINT(41.35310617028203 74.4528511635525) bank43984 +43985 POINT(40.211796879879 73.77479123360195) bank43985 +43986 POINT(40.92759364903659 74.61592887020689) bank43986 +43987 POINT(40.0958263812172 73.51045420599243) bank43987 +43988 POINT(39.84266525065536 74.8137469741697) bank43988 +43989 POINT(40.232223208257516 73.61507573099504) bank43989 +43990 POINT(41.32729292901662 74.65213768474759) bank43990 +43991 POINT(40.17500449918751 74.12052238326383) bank43991 +43992 POINT(40.03014451971283 73.09448474498765) bank43992 +43993 POINT(41.282949309358564 74.5073612018297) bank43993 +43994 POINT(40.22886838357291 73.11895724706677) bank43994 +43995 POINT(41.323082589489424 73.40524540983144) bank43995 +43996 POINT(39.89639460453301 73.29236218000112) bank43996 +43997 POINT(41.57539131317055 74.7267425406931) bank43997 +43998 POINT(41.30506174206323 73.19361459447255) bank43998 +43999 POINT(39.741845648335094 74.10489246610425) bank43999 +44000 POINT(41.568981362444276 73.23667531237292) bank44000 +44001 POINT(40.42471156077994 74.37525024728797) bank44001 +44002 POINT(41.24071441795612 74.0969246841762) bank44002 +44003 POINT(40.43084790268864 73.79188294698095) bank44003 +44004 POINT(41.65096191175178 73.25780817434303) bank44004 +44005 POINT(40.13442831607239 73.89370879252974) bank44005 +44006 POINT(41.20572262505731 73.13393616220476) bank44006 +44007 POINT(41.28385162191226 74.2571796342395) bank44007 +44008 POINT(41.17930112330836 73.88563491981574) bank44008 +44009 POINT(40.81442344445264 73.59139609638487) bank44009 +44010 POINT(41.20406856037406 74.73291972072082) bank44010 +44011 POINT(41.367732074179216 74.193738126081) bank44011 +44012 POINT(40.01767671443003 74.71665087256888) bank44012 +44013 POINT(41.26077157594714 73.40054632482898) bank44013 +44014 POINT(40.645966581187515 74.65296940088474) bank44014 +44015 POINT(40.85671915241668 74.67009603401513) bank44015 +44016 POINT(41.035953479244355 74.61977540907904) bank44016 +44017 POINT(41.32753229775654 74.25805266982289) bank44017 +44018 POINT(40.893664483279565 74.28668101484817) bank44018 +44019 POINT(39.99944358553908 74.92280299347084) bank44019 +44020 POINT(41.17595869939516 73.82021421370952) bank44020 +44021 POINT(41.486486553453545 74.52173678057488) bank44021 +44022 POINT(40.863481667936156 74.02152215281679) bank44022 +44023 POINT(40.9578108358522 73.90047243129995) bank44023 +44024 POINT(41.10463329730034 74.50879166197872) bank44024 +44025 POINT(40.73507470408676 74.8958052390999) bank44025 +44026 POINT(40.30071924698614 74.36458130958135) bank44026 +44027 POINT(40.91209444200334 74.37423668195188) bank44027 +44028 POINT(40.81540943206828 73.9470600232142) bank44028 +44029 POINT(40.516752419898346 73.28474081428199) bank44029 +44030 POINT(40.66574809592251 73.83990937735003) bank44030 +44031 POINT(40.245864159716646 74.60493411520922) bank44031 +44032 POINT(40.02921070516998 73.16571044425518) bank44032 +44033 POINT(40.427331048690576 73.92126373658915) bank44033 +44034 POINT(41.33594852872841 73.93701946482562) bank44034 +44035 POINT(40.00769032215547 73.99871380090623) bank44035 +44036 POINT(40.625599695026246 73.05728128494653) bank44036 +44037 POINT(40.537327677532545 73.74813035769928) bank44037 +44038 POINT(41.07834764971775 73.73639870262977) bank44038 +44039 POINT(40.81990959339924 73.20363056336205) bank44039 +44040 POINT(40.61608481311368 73.56525003170636) bank44040 +44041 POINT(41.502083678251594 74.00562839396282) bank44041 +44042 POINT(39.9467636058814 74.92892635123398) bank44042 +44043 POINT(40.198276855994614 74.87119742067678) bank44043 +44044 POINT(41.49482197774709 73.35939544536703) bank44044 +44045 POINT(41.4902686320188 74.52652522309235) bank44045 +44046 POINT(40.84954040659574 74.12295343333905) bank44046 +44047 POINT(40.02849033630731 73.19202805773764) bank44047 +44048 POINT(41.45695321266894 74.97763783827308) bank44048 +44049 POINT(40.34571603073177 74.1087780307926) bank44049 +44050 POINT(40.816242894115305 74.94619648359992) bank44050 +44051 POINT(40.73795935310756 73.17706736747658) bank44051 +44052 POINT(40.42521675813382 74.26070159139913) bank44052 +44053 POINT(40.06841559797171 74.55163817066072) bank44053 +44054 POINT(40.18668577705766 73.77880457144353) bank44054 +44055 POINT(39.837333409051176 74.0031084780188) bank44055 +44056 POINT(41.33688093892344 73.83105218878883) bank44056 +44057 POINT(40.04772824857084 73.44490234010152) bank44057 +44058 POINT(40.07944682738768 73.58113592605442) bank44058 +44059 POINT(40.21668562471278 74.9927195619303) bank44059 +44060 POINT(41.69043221113947 74.08509685235838) bank44060 +44061 POINT(40.527235798532764 73.87693651676007) bank44061 +44062 POINT(40.40687159951791 74.8509686670599) bank44062 +44063 POINT(40.81519028154925 74.57635145910945) bank44063 +44064 POINT(41.191930524917375 74.74608267944669) bank44064 +44065 POINT(39.88003684624974 74.66747328737617) bank44065 +44066 POINT(41.5534159785281 73.51095897724926) bank44066 +44067 POINT(41.1625428895294 74.3719979360647) bank44067 +44068 POINT(41.00476238833249 74.196920781557) bank44068 +44069 POINT(41.352778719906766 74.9534099676809) bank44069 +44070 POINT(41.05749304720422 73.13163947178604) bank44070 +44071 POINT(41.099580188702554 74.90746322259278) bank44071 +44072 POINT(40.64182867083676 74.4446706811374) bank44072 +44073 POINT(41.292934493420525 74.67393341194627) bank44073 +44074 POINT(39.833563781373506 74.68966252578258) bank44074 +44075 POINT(40.552632584360026 73.76190585667162) bank44075 +44076 POINT(41.37865419717523 74.57433568822111) bank44076 +44077 POINT(40.9648884741408 73.10958888994692) bank44077 +44078 POINT(39.89283945195432 74.32970940906162) bank44078 +44079 POINT(41.1206426728984 73.92730056864512) bank44079 +44080 POINT(40.71649365351439 73.32258486778511) bank44080 +44081 POINT(40.81457624838872 74.91805131342566) bank44081 +44082 POINT(41.01176239235979 74.02263424161329) bank44082 +44083 POINT(40.41831011639187 73.04032209744115) bank44083 +44084 POINT(41.696596575985346 73.5139598078197) bank44084 +44085 POINT(39.892258502602076 73.26270395731648) bank44085 +44086 POINT(40.76131727968686 73.69165678093458) bank44086 +44087 POINT(40.48536272903205 74.96539428381624) bank44087 +44088 POINT(41.5011338695028 74.57545332737502) bank44088 +44089 POINT(41.42529943104258 74.15139379325316) bank44089 +44090 POINT(41.646401563769544 74.40794447152582) bank44090 +44091 POINT(40.69985136386538 74.10918118262404) bank44091 +44092 POINT(41.51425237578062 73.99646338256213) bank44092 +44093 POINT(40.80348085153649 73.9902177815206) bank44093 +44094 POINT(40.4828909959612 73.50175347714351) bank44094 +44095 POINT(40.96515505789038 73.27448187310806) bank44095 +44096 POINT(40.567186115609815 74.07075236092305) bank44096 +44097 POINT(41.66246005619311 74.31056637653182) bank44097 +44098 POINT(40.815409008956166 74.23750878599229) bank44098 +44099 POINT(40.337156335456086 73.40455830173214) bank44099 +44100 POINT(41.04956634892561 74.56389731287821) bank44100 +44101 POINT(40.16995301159003 73.103586600615) bank44101 +44102 POINT(41.45995886328085 74.32882934325319) bank44102 +44103 POINT(41.01926527728823 73.2121890470423) bank44103 +44104 POINT(40.41068214570604 73.8605310051553) bank44104 +44105 POINT(40.348207266723904 74.93474720540257) bank44105 +44106 POINT(39.7484847942191 73.7410440919436) bank44106 +44107 POINT(40.52132309426543 73.700447717163) bank44107 +44108 POINT(41.33248659023768 73.06879036257479) bank44108 +44109 POINT(40.21868952458001 73.3473122948127) bank44109 +44110 POINT(41.624230448512236 74.80816188864296) bank44110 +44111 POINT(41.57324584548629 74.74114021789049) bank44111 +44112 POINT(40.31309832354176 74.46549244702254) bank44112 +44113 POINT(39.88610452224264 74.85642421638829) bank44113 +44114 POINT(41.44713369047124 74.23970629772398) bank44114 +44115 POINT(40.854478046483436 73.0467457857441) bank44115 +44116 POINT(40.46206637887738 74.49649464661577) bank44116 +44117 POINT(41.45592330141097 73.82989920537192) bank44117 +44118 POINT(40.65967067276495 74.95401977325794) bank44118 +44119 POINT(39.850416928796854 74.39668598646163) bank44119 +44120 POINT(39.98278628050929 74.49830199119437) bank44120 +44121 POINT(41.17693233674711 74.24566423144266) bank44121 +44122 POINT(41.37522937782024 74.06423289781252) bank44122 +44123 POINT(40.05435722285184 73.34483216305851) bank44123 +44124 POINT(41.32431483048358 73.98175881905352) bank44124 +44125 POINT(40.620527981808564 74.18684892730968) bank44125 +44126 POINT(40.23178647733729 74.45021724257427) bank44126 +44127 POINT(40.57903956326923 74.24722908909291) bank44127 +44128 POINT(39.72874610687143 74.71260857909556) bank44128 +44129 POINT(41.437897950862 74.14853511600413) bank44129 +44130 POINT(39.96287423921849 73.78139739193055) bank44130 +44131 POINT(40.24978371830551 73.21061800074175) bank44131 +44132 POINT(41.62569959224732 74.22441237681088) bank44132 +44133 POINT(40.2279772991165 73.26222837949004) bank44133 +44134 POINT(40.86806424341732 74.84826080478528) bank44134 +44135 POINT(39.924704605527424 74.98165546319899) bank44135 +44136 POINT(41.674664021394165 73.46162927708913) bank44136 +44137 POINT(40.182951166597924 74.64760324140458) bank44137 +44138 POINT(40.03824046053661 74.45173329337265) bank44138 +44139 POINT(41.3326390147003 73.72677466970194) bank44139 +44140 POINT(41.68104503113092 74.82951164701282) bank44140 +44141 POINT(41.46766329682095 74.92033703495231) bank44141 +44142 POINT(40.65445342220821 74.2113581800373) bank44142 +44143 POINT(40.37304563051598 74.37732022007695) bank44143 +44144 POINT(41.39441421773652 73.77058113025582) bank44144 +44145 POINT(41.64455199871639 73.30557169273423) bank44145 +44146 POINT(41.26364855597994 73.18269644930348) bank44146 +44147 POINT(41.43059613070192 73.44586419917786) bank44147 +44148 POINT(40.00393438903506 73.81835942701983) bank44148 +44149 POINT(40.34151077287192 74.06745978837263) bank44149 +44150 POINT(41.66378107262493 74.89263639643679) bank44150 +44151 POINT(41.657445501731786 74.4359576473504) bank44151 +44152 POINT(40.71114793296666 74.37364362981843) bank44152 +44153 POINT(41.489075316650016 74.21455664633046) bank44153 +44154 POINT(41.516507973508745 74.68933788706639) bank44154 +44155 POINT(39.71996833848078 73.78091486889438) bank44155 +44156 POINT(41.44574572372609 74.87238111454307) bank44156 +44157 POINT(40.73058022931219 73.10309397521577) bank44157 +44158 POINT(40.342908487654455 73.94313959191626) bank44158 +44159 POINT(41.26815657829039 74.90054681847741) bank44159 +44160 POINT(41.60236788735026 73.4407824533853) bank44160 +44161 POINT(40.20042354479961 73.86177036693724) bank44161 +44162 POINT(40.06562619059033 74.10212606219996) bank44162 +44163 POINT(41.52858942236885 73.07832319001079) bank44163 +44164 POINT(41.416345021775065 74.85452657347572) bank44164 +44165 POINT(40.76061502905018 73.13722069068344) bank44165 +44166 POINT(39.80109408860714 73.44715775063989) bank44166 +44167 POINT(40.623167276224706 73.81616420451466) bank44167 +44168 POINT(40.983910574775756 74.32637212137493) bank44168 +44169 POINT(40.11925662897022 74.76242570203254) bank44169 +44170 POINT(41.281924914783374 73.78556015117684) bank44170 +44171 POINT(41.57951839677421 74.30634600477084) bank44171 +44172 POINT(41.564804382925054 73.41055627388305) bank44172 +44173 POINT(40.064086374277714 73.86661741493307) bank44173 +44174 POINT(41.47127707372473 74.91914044867725) bank44174 +44175 POINT(40.11406313672044 74.06949097172618) bank44175 +44176 POINT(41.18428492118515 73.20385815711015) bank44176 +44177 POINT(40.26851844460579 73.29898387690824) bank44177 +44178 POINT(40.80055314797291 73.8442512480087) bank44178 +44179 POINT(39.96891946080153 73.7438518630916) bank44179 +44180 POINT(40.47653700108828 74.93890724791919) bank44180 +44181 POINT(40.672702842345075 73.62310931375937) bank44181 +44182 POINT(41.17823159810051 73.68947202151539) bank44182 +44183 POINT(40.23170961272489 74.35220718046577) bank44183 +44184 POINT(40.57568174534695 73.36290867534836) bank44184 +44185 POINT(41.702489291811695 73.5285836156244) bank44185 +44186 POINT(40.42550227310137 74.32935977201268) bank44186 +44187 POINT(40.09268353585271 73.31050580335385) bank44187 +44188 POINT(40.67265325477067 73.04551180524336) bank44188 +44189 POINT(40.40498199769195 73.36656852728179) bank44189 +44190 POINT(40.25306556018105 74.32597326183404) bank44190 +44191 POINT(39.99024721641739 74.66611855459547) bank44191 +44192 POINT(41.2084032562757 73.1597785888691) bank44192 +44193 POINT(41.56921192368135 74.89944065364824) bank44193 +44194 POINT(40.84913944866493 74.34412240002165) bank44194 +44195 POINT(40.71035605506736 73.86835125035938) bank44195 +44196 POINT(39.73745535795079 73.06560604124405) bank44196 +44197 POINT(41.47917373819139 74.86264840732275) bank44197 +44198 POINT(39.99204483682612 74.82248493115563) bank44198 +44199 POINT(40.665890958256746 74.46718143370964) bank44199 +44200 POINT(41.50689334940888 73.08531601245231) bank44200 +44201 POINT(41.36173726805206 73.04596739134993) bank44201 +44202 POINT(41.61884237414093 73.51780170053263) bank44202 +44203 POINT(41.52510394648277 73.85943994544286) bank44203 +44204 POINT(40.5376783412065 74.2897850237471) bank44204 +44205 POINT(40.14227179807885 74.94357318402469) bank44205 +44206 POINT(40.25834090270982 74.45678217546235) bank44206 +44207 POINT(41.51978762686565 74.63067155186368) bank44207 +44208 POINT(40.350935927578846 73.342020893206) bank44208 +44209 POINT(41.187739145661894 74.9756011586413) bank44209 +44210 POINT(41.5065766028257 74.67784551820297) bank44210 +44211 POINT(41.70168820057739 73.59533711701538) bank44211 +44212 POINT(41.68077796899788 73.82485959339544) bank44212 +44213 POINT(40.53707626298043 73.60635240327015) bank44213 +44214 POINT(41.23568012780312 73.89875695648254) bank44214 +44215 POINT(40.608079437530996 73.72466938391791) bank44215 +44216 POINT(39.86343081015425 73.95418697568346) bank44216 +44217 POINT(39.72041166400199 73.25601631723102) bank44217 +44218 POINT(40.64658372919976 73.17388356695061) bank44218 +44219 POINT(40.8569729049817 74.58678634066887) bank44219 +44220 POINT(40.60173988681715 74.23697862082871) bank44220 +44221 POINT(41.25544983708542 73.53573370196034) bank44221 +44222 POINT(40.992496670168755 73.08001207043915) bank44222 +44223 POINT(41.14325235611466 73.42205373599352) bank44223 +44224 POINT(40.26195121970276 73.6919568640949) bank44224 +44225 POINT(40.3352221987205 75.00328245368237) bank44225 +44226 POINT(39.97382754089083 73.31092291617259) bank44226 +44227 POINT(39.94603906439834 73.23085066944098) bank44227 +44228 POINT(41.12834648334447 73.23289503720139) bank44228 +44229 POINT(39.830690805817795 73.35617401062784) bank44229 +44230 POINT(41.70500966186617 73.47496679102247) bank44230 +44231 POINT(40.94099011386435 74.06601875619722) bank44231 +44232 POINT(41.70933019745422 74.42619560536941) bank44232 +44233 POINT(41.140998161381745 74.96523053829719) bank44233 +44234 POINT(39.91392162636146 74.65159000714465) bank44234 +44235 POINT(41.535618600320795 74.11964356670565) bank44235 +44236 POINT(40.96970841407283 74.84647975219451) bank44236 +44237 POINT(40.119887147431655 73.67989025652922) bank44237 +44238 POINT(41.41095396993403 73.80117823669654) bank44238 +44239 POINT(40.57230062085223 73.42983849318952) bank44239 +44240 POINT(40.73041834537674 73.84796740032303) bank44240 +44241 POINT(40.370567958494064 73.03640688284925) bank44241 +44242 POINT(41.03638081260008 74.55422562588856) bank44242 +44243 POINT(41.36318295727655 73.43747441766021) bank44243 +44244 POINT(40.94025774832332 73.41599858064971) bank44244 +44245 POINT(41.65122459923212 73.2903174627265) bank44245 +44246 POINT(40.010314114585654 74.83928026996693) bank44246 +44247 POINT(40.67483434432368 74.54296397004877) bank44247 +44248 POINT(39.787356165556005 74.24555313424936) bank44248 +44249 POINT(40.00520910177566 73.29096173458994) bank44249 +44250 POINT(40.80162764529284 74.64474582050636) bank44250 +44251 POINT(40.19740991539284 73.97361310536385) bank44251 +44252 POINT(40.9989033580831 74.55012942554485) bank44252 +44253 POINT(40.356374603126596 74.21183441167736) bank44253 +44254 POINT(40.635814424172196 74.05412153637984) bank44254 +44255 POINT(40.43806890193881 73.54404881760023) bank44255 +44256 POINT(40.703152697173586 73.55308215747003) bank44256 +44257 POINT(40.63733467092175 74.78768580622574) bank44257 +44258 POINT(39.88866724838878 73.50126224486502) bank44258 +44259 POINT(39.99565407190968 74.39931658016701) bank44259 +44260 POINT(40.820270441664704 73.34683904604674) bank44260 +44261 POINT(40.63574967890686 73.15986124685945) bank44261 +44262 POINT(40.155649596767745 73.37090582275898) bank44262 +44263 POINT(39.84249283381441 74.17515353348334) bank44263 +44264 POINT(40.79041153175526 73.95844511784864) bank44264 +44265 POINT(41.52104455947461 73.82202513215869) bank44265 +44266 POINT(41.698704698145605 73.87202820174413) bank44266 +44267 POINT(39.869552494750764 74.02411826271492) bank44267 +44268 POINT(39.780985449943536 74.81220153805877) bank44268 +44269 POINT(41.30510538811743 74.62894616285533) bank44269 +44270 POINT(40.060341397834044 74.82787104771276) bank44270 +44271 POINT(40.65342959627238 73.51576971432411) bank44271 +44272 POINT(41.35886864027682 74.35134284097487) bank44272 +44273 POINT(40.852319132412646 73.06504731266836) bank44273 +44274 POINT(41.56419466024797 73.66534884031768) bank44274 +44275 POINT(40.16969256442748 73.06880117718585) bank44275 +44276 POINT(40.483160771395774 74.43749586545414) bank44276 +44277 POINT(41.22563067202576 73.69038624972168) bank44277 +44278 POINT(41.30342879475597 73.62367030635734) bank44278 +44279 POINT(40.73894996251179 73.5417714255136) bank44279 +44280 POINT(40.323482134329275 74.24583118718202) bank44280 +44281 POINT(40.89887579465789 74.92694391368447) bank44281 +44282 POINT(41.60700123462858 73.07618106681319) bank44282 +44283 POINT(40.965676859885086 73.99961149199622) bank44283 +44284 POINT(41.69234401724901 74.62337393827976) bank44284 +44285 POINT(40.477810753144034 74.47925855698291) bank44285 +44286 POINT(40.01887861629261 73.7197782184908) bank44286 +44287 POINT(39.72935546609882 74.58512063812734) bank44287 +44288 POINT(40.83628465363209 74.80901023500151) bank44288 +44289 POINT(41.5391896086068 73.76181373031838) bank44289 +44290 POINT(40.83419397179526 73.61574531133942) bank44290 +44291 POINT(40.58205904656566 73.02208217992936) bank44291 +44292 POINT(39.785190854261984 73.50239295784631) bank44292 +44293 POINT(40.458714316065915 73.8950693915042) bank44293 +44294 POINT(41.461371621593706 74.26270032310045) bank44294 +44295 POINT(41.20161225539461 73.62705621330885) bank44295 +44296 POINT(41.62216908559689 74.21532855688965) bank44296 +44297 POINT(40.96738248617945 74.43798426441471) bank44297 +44298 POINT(39.99888428827292 74.57603405338975) bank44298 +44299 POINT(40.93699876232474 73.50739773931086) bank44299 +44300 POINT(40.11449286886317 74.01703169129054) bank44300 +44301 POINT(40.41037197030386 74.88299509772166) bank44301 +44302 POINT(40.26781714517224 74.41397123002506) bank44302 +44303 POINT(39.93002878202982 73.96726875656597) bank44303 +44304 POINT(39.7933847696531 73.08602736731349) bank44304 +44305 POINT(40.51515487779005 73.69979128142283) bank44305 +44306 POINT(40.91467951998413 74.99184233943566) bank44306 +44307 POINT(40.9660896683602 73.58161484834227) bank44307 +44308 POINT(40.56720125980503 74.34649308096289) bank44308 +44309 POINT(41.283369247491976 74.95678433222722) bank44309 +44310 POINT(41.25321812755283 73.54592688705216) bank44310 +44311 POINT(40.969355697592945 73.81113286975983) bank44311 +44312 POINT(40.28023660685923 74.47050733196613) bank44312 +44313 POINT(41.14872428568923 73.63993407113362) bank44313 +44314 POINT(41.32139663919605 73.50544176613084) bank44314 +44315 POINT(40.35756868061079 73.35418081654205) bank44315 +44316 POINT(39.82323418221529 74.81582650150897) bank44316 +44317 POINT(40.30378848562442 73.39028583767468) bank44317 +44318 POINT(41.149959482332314 73.98072991731043) bank44318 +44319 POINT(41.04453451108949 73.28950196956657) bank44319 +44320 POINT(40.595463940136526 74.09669016854042) bank44320 +44321 POINT(41.53866594177916 73.75210453199328) bank44321 +44322 POINT(40.619661843172025 74.38761383939045) bank44322 +44323 POINT(40.02716614737607 74.76602374313333) bank44323 +44324 POINT(41.52041371129133 73.9201962781909) bank44324 +44325 POINT(40.44775445917272 73.47764976675604) bank44325 +44326 POINT(41.073013031292945 73.10216236539611) bank44326 +44327 POINT(39.98655849005933 73.26171059693581) bank44327 +44328 POINT(40.498073377326925 73.23864797995874) bank44328 +44329 POINT(41.32544503188109 74.6370605268704) bank44329 +44330 POINT(40.37451709033894 73.15083181617594) bank44330 +44331 POINT(40.003471380388966 74.49160309227469) bank44331 +44332 POINT(40.33804157645333 73.57510411885602) bank44332 +44333 POINT(41.50693878114655 74.88084079347516) bank44333 +44334 POINT(41.44091243106352 74.77509558895419) bank44334 +44335 POINT(41.60128586166143 73.6317741350385) bank44335 +44336 POINT(41.13881064921014 74.90241264612737) bank44336 +44337 POINT(40.00835184038422 74.79716883263882) bank44337 +44338 POINT(41.19959884288977 73.59509193141776) bank44338 +44339 POINT(40.483760236709834 74.57695128297819) bank44339 +44340 POINT(40.67773046636614 74.6108456280358) bank44340 +44341 POINT(40.8673628004684 73.12270046999787) bank44341 +44342 POINT(41.24607808014052 74.91043366760941) bank44342 +44343 POINT(40.38665554937729 74.42527247942378) bank44343 +44344 POINT(41.16144441526215 74.37008929790521) bank44344 +44345 POINT(40.98198529015961 73.75985132084831) bank44345 +44346 POINT(40.09949457301085 73.55842627320922) bank44346 +44347 POINT(41.64550136357621 74.5814881929597) bank44347 +44348 POINT(41.129797944598145 73.82663814615151) bank44348 +44349 POINT(40.8157166906723 74.68719833989299) bank44349 +44350 POINT(40.60577016804294 74.76045469355242) bank44350 +44351 POINT(41.437136592568095 73.63375487061334) bank44351 +44352 POINT(39.7548328646988 73.11220214425087) bank44352 +44353 POINT(39.85520832944303 73.6111959755247) bank44353 +44354 POINT(41.24607876043935 73.42902359543756) bank44354 +44355 POINT(40.762463009962104 73.97052084756989) bank44355 +44356 POINT(40.40827015716347 74.41620609593183) bank44356 +44357 POINT(41.62356707816205 74.51127906558054) bank44357 +44358 POINT(40.16399692149513 73.88666894920306) bank44358 +44359 POINT(40.598732718046364 73.70532475356667) bank44359 +44360 POINT(40.60928989959856 74.34590785495631) bank44360 +44361 POINT(40.04504128041906 73.1407318782148) bank44361 +44362 POINT(40.35442824419296 73.69253855507452) bank44362 +44363 POINT(39.81375339540767 73.50579722681331) bank44363 +44364 POINT(39.85724156721601 73.80817015107887) bank44364 +44365 POINT(41.60345596289399 73.1353910652748) bank44365 +44366 POINT(39.72125981305693 74.26507837309236) bank44366 +44367 POINT(39.894323542272744 74.1287819490762) bank44367 +44368 POINT(41.22342136687427 74.07747357023723) bank44368 +44369 POINT(39.76468734984616 73.0618056294811) bank44369 +44370 POINT(40.46385407044135 74.54110603410332) bank44370 +44371 POINT(40.73676822393522 74.90426806460606) bank44371 +44372 POINT(39.944397336029795 74.59836887589276) bank44372 +44373 POINT(41.00784628243348 73.68054229722476) bank44373 +44374 POINT(40.76970929896512 74.79118817051402) bank44374 +44375 POINT(40.78846403641078 73.90683483389438) bank44375 +44376 POINT(41.409939536635285 74.57871296519474) bank44376 +44377 POINT(41.1715196782153 73.84756863461321) bank44377 +44378 POINT(39.85332972494032 74.86408567847903) bank44378 +44379 POINT(40.82247999220887 74.58319126196847) bank44379 +44380 POINT(40.198818458021535 73.00663542239808) bank44380 +44381 POINT(40.86812979476669 73.85759755804104) bank44381 +44382 POINT(40.5774036986692 74.4507614054141) bank44382 +44383 POINT(41.29778278164633 74.50141591500876) bank44383 +44384 POINT(41.44659789399804 73.07537120641344) bank44384 +44385 POINT(40.24051298116757 74.94864050880507) bank44385 +44386 POINT(39.93402504539686 74.45906852803049) bank44386 +44387 POINT(41.59247036566746 74.20121847946486) bank44387 +44388 POINT(41.20650542971273 73.16415393664111) bank44388 +44389 POINT(40.49943772137809 74.59202613936512) bank44389 +44390 POINT(41.505045421116584 73.72293569898439) bank44390 +44391 POINT(41.06120599972574 74.64979528289591) bank44391 +44392 POINT(40.957444924901544 73.96444298216923) bank44392 +44393 POINT(40.866335501871056 73.4339893547258) bank44393 +44394 POINT(41.11150598343908 73.52507297954668) bank44394 +44395 POINT(39.871993884367846 73.47536164534014) bank44395 +44396 POINT(41.34790761001023 73.15701413140953) bank44396 +44397 POINT(40.852798778696275 74.57552186256325) bank44397 +44398 POINT(40.64718205837156 73.91409091917826) bank44398 +44399 POINT(40.460344611670756 74.63392845713481) bank44399 +44400 POINT(41.10945246917281 73.9732010930797) bank44400 +44401 POINT(41.55888180083996 74.209074184127) bank44401 +44402 POINT(41.34730402183321 74.23921810102898) bank44402 +44403 POINT(40.050792311233856 73.17044442111123) bank44403 +44404 POINT(41.127873861930816 73.15129218138702) bank44404 +44405 POINT(41.361115134622594 73.96460465178383) bank44405 +44406 POINT(41.33916735689133 73.69868900561666) bank44406 +44407 POINT(39.997924898328634 74.63357419011035) bank44407 +44408 POINT(41.45915141110462 74.90930332554196) bank44408 +44409 POINT(40.9832660540998 73.16260105723137) bank44409 +44410 POINT(40.232093229170474 74.84404375979074) bank44410 +44411 POINT(40.14725932689718 74.18488851702496) bank44411 +44412 POINT(40.04445931706394 73.36552329371817) bank44412 +44413 POINT(40.08773793285975 73.83465173668152) bank44413 +44414 POINT(41.39037159494411 73.04357677310666) bank44414 +44415 POINT(41.151993621399946 73.78438279041865) bank44415 +44416 POINT(40.84184159290119 74.51043443525735) bank44416 +44417 POINT(40.48179323655669 74.09469099716891) bank44417 +44418 POINT(40.25234977501042 74.19183157634566) bank44418 +44419 POINT(40.16691889590634 74.33878708139513) bank44419 +44420 POINT(41.574086747843744 73.88709655474227) bank44420 +44421 POINT(40.13678577709502 74.57288832840234) bank44421 +44422 POINT(40.23907561823689 74.73829694627207) bank44422 +44423 POINT(40.00718516856112 73.47767845295267) bank44423 +44424 POINT(41.64197772523739 73.59521275854344) bank44424 +44425 POINT(41.06248620077082 74.34434961099767) bank44425 +44426 POINT(40.52271625356163 73.40088401131759) bank44426 +44427 POINT(40.48255610448028 74.57860998713164) bank44427 +44428 POINT(40.08637478145518 73.58501730374795) bank44428 +44429 POINT(41.443624641823256 74.02074338517276) bank44429 +44430 POINT(40.1956486578147 74.5544214079437) bank44430 +44431 POINT(40.229515053100585 74.230264417594) bank44431 +44432 POINT(40.214770496367876 73.84613732231135) bank44432 +44433 POINT(40.7502194423331 74.07940394595889) bank44433 +44434 POINT(41.048688156211014 74.0134971997497) bank44434 +44435 POINT(41.41251019306117 73.82770365627405) bank44435 +44436 POINT(39.818809403861614 74.52429555084073) bank44436 +44437 POINT(41.102529686281386 74.32770682256445) bank44437 +44438 POINT(41.203471513282146 74.42457111209303) bank44438 +44439 POINT(39.799703339075116 73.16337718076689) bank44439 +44440 POINT(41.400505547967626 74.76209354967112) bank44440 +44441 POINT(39.83959130018917 73.9972571921834) bank44441 +44442 POINT(39.8878237803554 73.67102809879361) bank44442 +44443 POINT(41.36808530677771 73.5520847348458) bank44443 +44444 POINT(41.10069526011719 74.67456691444238) bank44444 +44445 POINT(40.93129636364447 73.02065084858617) bank44445 +44446 POINT(39.723243800676904 74.38427941829858) bank44446 +44447 POINT(40.604115967399224 74.6481744526004) bank44447 +44448 POINT(40.00625305491593 74.70213688451858) bank44448 +44449 POINT(40.35518349631548 74.07747314164114) bank44449 +44450 POINT(40.04244288764029 73.95658499641455) bank44450 +44451 POINT(40.84689778448193 73.92535242296131) bank44451 +44452 POINT(41.07204892964635 73.18143820021821) bank44452 +44453 POINT(40.73585148794552 73.24323132363217) bank44453 +44454 POINT(40.75476268583323 73.38793370229136) bank44454 +44455 POINT(40.5527829550201 73.37859890929451) bank44455 +44456 POINT(41.08019590322273 73.37744835996514) bank44456 +44457 POINT(40.99559082859777 73.15018231997314) bank44457 +44458 POINT(40.715869421669616 74.37325996469153) bank44458 +44459 POINT(40.43730032409108 74.3840191195213) bank44459 +44460 POINT(39.84592313443264 74.06304991321826) bank44460 +44461 POINT(41.00568466287167 74.78594739625471) bank44461 +44462 POINT(39.825950330359746 73.47741686869843) bank44462 +44463 POINT(41.10976948697559 73.4090814325441) bank44463 +44464 POINT(40.234375523341384 74.20426128486197) bank44464 +44465 POINT(41.22535956311533 74.75950364339252) bank44465 +44466 POINT(40.27242988238584 73.99738133991251) bank44466 +44467 POINT(40.87471606806005 74.45884691960615) bank44467 +44468 POINT(40.42583162483402 74.45819968617785) bank44468 +44469 POINT(40.97500641046158 73.35388340281777) bank44469 +44470 POINT(39.99173309344254 73.94756655745068) bank44470 +44471 POINT(40.517455876554195 74.01806784176839) bank44471 +44472 POINT(39.76277052031768 73.0641100401317) bank44472 +44473 POINT(40.058413407511836 74.74061281110285) bank44473 +44474 POINT(40.65838830962571 74.3777167575905) bank44474 +44475 POINT(40.770195141492124 74.42886095922113) bank44475 +44476 POINT(39.88842869942341 73.2919866424078) bank44476 +44477 POINT(41.497370466881264 73.92004739795706) bank44477 +44478 POINT(41.267376819586964 73.71712301844427) bank44478 +44479 POINT(40.422208800149136 73.03150670440336) bank44479 +44480 POINT(40.6790441231682 73.11783981836382) bank44480 +44481 POINT(41.59807330372292 73.87227187021195) bank44481 +44482 POINT(40.93504358963349 74.45742306424526) bank44482 +44483 POINT(41.45409924732513 74.9828039554208) bank44483 +44484 POINT(39.74648174347641 73.59259217560962) bank44484 +44485 POINT(39.75308295987537 73.47434047418771) bank44485 +44486 POINT(40.24883993520172 74.89150352274456) bank44486 +44487 POINT(39.98622629154276 74.98375085299614) bank44487 +44488 POINT(40.50391987184004 74.34913801127227) bank44488 +44489 POINT(41.31153782684656 74.06654813251019) bank44489 +44490 POINT(41.03188659913624 74.20567551178118) bank44490 +44491 POINT(40.66751182482068 74.14187458896072) bank44491 +44492 POINT(40.42305371114584 74.62760353752515) bank44492 +44493 POINT(39.90232317718908 73.76339611619872) bank44493 +44494 POINT(40.63137233683607 73.31692851498997) bank44494 +44495 POINT(40.28858372014265 73.5113382490354) bank44495 +44496 POINT(40.723717032225395 74.79437981887567) bank44496 +44497 POINT(40.87177473761783 74.86090936711382) bank44497 +44498 POINT(41.50006823685547 74.51342882464283) bank44498 +44499 POINT(41.095038102257696 74.44398665204945) bank44499 +44500 POINT(41.31431924542685 73.24861138781259) bank44500 +44501 POINT(40.597489411606844 74.50993501752421) bank44501 +44502 POINT(40.044528397223694 74.93199109609664) bank44502 +44503 POINT(40.806665863058065 73.96798293338983) bank44503 +44504 POINT(40.41429532083996 74.31807859401182) bank44504 +44505 POINT(40.23905448666667 73.2993842564507) bank44505 +44506 POINT(41.712449174667896 73.30865255839001) bank44506 +44507 POINT(40.67575101008728 73.59583720279402) bank44507 +44508 POINT(40.112852586306204 74.79078077578842) bank44508 +44509 POINT(40.83335220505195 73.437435260444) bank44509 +44510 POINT(40.639872023918635 74.99155191544608) bank44510 +44511 POINT(39.92154976303957 73.28045160836189) bank44511 +44512 POINT(40.78417709597369 74.50977586827112) bank44512 +44513 POINT(40.21551393570987 73.52733763534317) bank44513 +44514 POINT(40.9001690845767 74.4321883143369) bank44514 +44515 POINT(40.0452750109319 73.69995221922143) bank44515 +44516 POINT(40.198302335396725 73.0619816816462) bank44516 +44517 POINT(41.36783649058769 73.31673901348351) bank44517 +44518 POINT(41.188266476619184 74.32824831270558) bank44518 +44519 POINT(41.410781093139136 73.12173669630025) bank44519 +44520 POINT(39.8968729815123 73.15942111781224) bank44520 +44521 POINT(40.85931212727598 73.10550401621606) bank44521 +44522 POINT(40.75959505230719 74.39964736167357) bank44522 +44523 POINT(41.41393287973364 73.0472553673294) bank44523 +44524 POINT(41.337413191531446 73.93900609571656) bank44524 +44525 POINT(41.6941485333473 74.49886039044031) bank44525 +44526 POINT(40.77079132952395 74.2752410450112) bank44526 +44527 POINT(39.867259524587205 74.87435595524612) bank44527 +44528 POINT(41.69262787177171 73.11317082264627) bank44528 +44529 POINT(40.81047440998172 73.4573040575795) bank44529 +44530 POINT(41.07526039727978 74.2807871157333) bank44530 +44531 POINT(41.538711440999116 73.42372063380068) bank44531 +44532 POINT(40.02992567268162 73.40235983191718) bank44532 +44533 POINT(40.57576526187585 74.58653214167501) bank44533 +44534 POINT(40.44021248728609 73.95929681288875) bank44534 +44535 POINT(40.00894210611812 74.15618190910669) bank44535 +44536 POINT(41.237577346901055 73.93577186287477) bank44536 +44537 POINT(40.98402916858974 74.16597531770293) bank44537 +44538 POINT(40.690854999105454 73.46431389155464) bank44538 +44539 POINT(40.05171389256295 74.75216418203368) bank44539 +44540 POINT(41.560177164551504 74.06698145361437) bank44540 +44541 POINT(40.799360814285336 73.74830666068821) bank44541 +44542 POINT(41.26800050652568 74.19981683003967) bank44542 +44543 POINT(41.17153954626248 74.99712803243963) bank44543 +44544 POINT(40.31584829687105 73.19461100963524) bank44544 +44545 POINT(39.974140276299075 74.08602094296404) bank44545 +44546 POINT(41.18767991594456 73.56632936647092) bank44546 +44547 POINT(40.481204229512315 74.95429515523814) bank44547 +44548 POINT(41.461725356287225 73.42164845068535) bank44548 +44549 POINT(39.93475607454323 73.06942297392825) bank44549 +44550 POINT(40.2047271570944 74.33619151460937) bank44550 +44551 POINT(39.76376734732318 73.87133302648785) bank44551 +44552 POINT(41.58456614740633 74.29824082265185) bank44552 +44553 POINT(41.59095864617821 74.43594510172683) bank44553 +44554 POINT(41.587936167659365 74.81509613711393) bank44554 +44555 POINT(40.70107963909369 73.05030306470073) bank44555 +44556 POINT(39.937082855529674 74.12637387037394) bank44556 +44557 POINT(39.73898115227136 74.81725594984253) bank44557 +44558 POINT(41.460624469944634 73.17263078232745) bank44558 +44559 POINT(40.60381536812372 73.40207023426734) bank44559 +44560 POINT(41.46583891094075 74.48426908633786) bank44560 +44561 POINT(40.79169437815421 73.34772678429762) bank44561 +44562 POINT(41.32470405296919 73.05417037427735) bank44562 +44563 POINT(40.71866628504268 74.10776436724981) bank44563 +44564 POINT(40.10700475372708 73.05130578444385) bank44564 +44565 POINT(40.47270872720695 74.66007613368963) bank44565 +44566 POINT(40.785067369752305 73.52928693709545) bank44566 +44567 POINT(40.686317108254215 74.19427015752898) bank44567 +44568 POINT(40.07767805573268 74.18187798054089) bank44568 +44569 POINT(41.41790580608701 74.8163848838066) bank44569 +44570 POINT(40.60384360686679 74.18221739198178) bank44570 +44571 POINT(40.879019292572266 74.11588251330969) bank44571 +44572 POINT(40.51370689944085 74.60870239422128) bank44572 +44573 POINT(41.440104144815834 74.33321041799299) bank44573 +44574 POINT(40.995556934105316 73.23678687310698) bank44574 +44575 POINT(40.2585252184988 73.06109895791707) bank44575 +44576 POINT(41.4345788063658 73.54074321700165) bank44576 +44577 POINT(40.227784478543704 73.20470684065016) bank44577 +44578 POINT(40.84390021663629 73.78510263289814) bank44578 +44579 POINT(41.191359919187555 73.78627836342761) bank44579 +44580 POINT(41.421479484084465 73.28025180111973) bank44580 +44581 POINT(40.75003299623787 74.06503611163683) bank44581 +44582 POINT(41.50810153303229 74.84338947961477) bank44582 +44583 POINT(41.094303438018144 74.29308582089244) bank44583 +44584 POINT(40.90319884300435 73.94955115791046) bank44584 +44585 POINT(41.54538158840396 73.73478582493308) bank44585 +44586 POINT(41.11006117017693 74.40014154852577) bank44586 +44587 POINT(40.03453182028733 73.43107328352131) bank44587 +44588 POINT(40.967344500776115 73.10399517347672) bank44588 +44589 POINT(39.7539074561706 74.76072394880201) bank44589 +44590 POINT(41.66892835184705 74.86719511577496) bank44590 +44591 POINT(40.4327881522395 74.68937227896171) bank44591 +44592 POINT(41.22111693825629 74.37733860307317) bank44592 +44593 POINT(41.51355800559389 73.9870102599474) bank44593 +44594 POINT(41.147470209505045 74.75180907179778) bank44594 +44595 POINT(41.36056359701371 74.3103466389106) bank44595 +44596 POINT(40.82643193850375 74.47233429843594) bank44596 +44597 POINT(41.529902437837215 74.48374964817978) bank44597 +44598 POINT(41.689265380630445 73.46762939795778) bank44598 +44599 POINT(41.12021178939777 74.78330108125569) bank44599 +44600 POINT(39.85665372484856 74.25101872403849) bank44600 +44601 POINT(40.53291854297989 73.21262808183903) bank44601 +44602 POINT(40.50265782607694 73.77370493624608) bank44602 +44603 POINT(39.82298536441366 74.5246252728189) bank44603 +44604 POINT(39.891808017781706 74.90460133149328) bank44604 +44605 POINT(40.73880397966944 73.10575591621132) bank44605 +44606 POINT(40.608251158810944 74.23113999816515) bank44606 +44607 POINT(40.70057762709254 74.98501129710107) bank44607 +44608 POINT(40.7552406100346 74.27040926191256) bank44608 +44609 POINT(40.87251381741945 74.23974015582759) bank44609 +44610 POINT(41.66288286510621 74.48555819847594) bank44610 +44611 POINT(40.161466801887265 73.56140874181138) bank44611 +44612 POINT(40.16901589527837 73.65453115871311) bank44612 +44613 POINT(41.3142854295447 73.014042411651) bank44613 +44614 POINT(41.06034537548016 73.90729286914778) bank44614 +44615 POINT(39.84025287704578 74.55207687494661) bank44615 +44616 POINT(40.7132726215796 73.19635363793311) bank44616 +44617 POINT(40.667324487636144 74.49344720414788) bank44617 +44618 POINT(39.80333254562855 73.86886627029092) bank44618 +44619 POINT(40.65991166912965 73.16898389687005) bank44619 +44620 POINT(40.715850742412584 73.42179436877689) bank44620 +44621 POINT(40.5673477103151 73.39236543464318) bank44621 +44622 POINT(39.80987052068706 74.94431555672942) bank44622 +44623 POINT(40.238350260905015 74.13232357315295) bank44623 +44624 POINT(39.96785440852145 74.84610869968247) bank44624 +44625 POINT(40.91001092377188 74.22025529024579) bank44625 +44626 POINT(39.84783790904855 74.25949854049624) bank44626 +44627 POINT(40.35276776531533 73.83139044037141) bank44627 +44628 POINT(40.39809424585429 73.47601492957375) bank44628 +44629 POINT(40.48596108964761 74.27192231019238) bank44629 +44630 POINT(40.54522089923342 74.61426749953372) bank44630 +44631 POINT(41.133531778708196 74.95346462404632) bank44631 +44632 POINT(41.45790484588789 74.69866538732268) bank44632 +44633 POINT(40.833620434541494 73.12544496382806) bank44633 +44634 POINT(39.9433730756489 74.75878327519169) bank44634 +44635 POINT(40.315707876782604 73.11308270075357) bank44635 +44636 POINT(41.47346088819338 74.82333886787546) bank44636 +44637 POINT(41.21344671127767 73.69750400372163) bank44637 +44638 POINT(41.062319053064336 73.22420092406877) bank44638 +44639 POINT(40.16518232259555 73.95846890288894) bank44639 +44640 POINT(41.47411240305726 74.96069658354999) bank44640 +44641 POINT(41.016050539967175 74.25322770470545) bank44641 +44642 POINT(40.29855226693082 74.27384961804538) bank44642 +44643 POINT(41.646817276573984 74.14431006437448) bank44643 +44644 POINT(40.70775062317915 73.53023197892287) bank44644 +44645 POINT(41.40808498150946 74.46160821031268) bank44645 +44646 POINT(41.02596999300979 74.65596950493787) bank44646 +44647 POINT(40.475017893653 74.33045033928592) bank44647 +44648 POINT(41.034605227227615 74.97302516437263) bank44648 +44649 POINT(39.946710932814156 73.8442514832815) bank44649 +44650 POINT(41.54663089424707 74.55402047548863) bank44650 +44651 POINT(40.186709159193725 73.04473796914196) bank44651 +44652 POINT(40.539220237175456 74.6936231326822) bank44652 +44653 POINT(40.24471354088402 73.57808957525154) bank44653 +44654 POINT(39.917300298279784 73.74121396858041) bank44654 +44655 POINT(40.49334323083613 73.25109296658958) bank44655 +44656 POINT(40.76393910216088 74.31436710010226) bank44656 +44657 POINT(41.43969252383884 73.9926016629471) bank44657 +44658 POINT(41.446366220105105 74.22735265751348) bank44658 +44659 POINT(39.79545380406268 74.68297416512681) bank44659 +44660 POINT(41.33464208070448 73.3973513088824) bank44660 +44661 POINT(40.75769921802005 73.9693122976457) bank44661 +44662 POINT(41.40488360164278 73.84018506459998) bank44662 +44663 POINT(41.5531635215186 73.86013128615959) bank44663 +44664 POINT(40.29699977796519 73.5938838125049) bank44664 +44665 POINT(40.96760658895132 74.94994909916304) bank44665 +44666 POINT(39.83224434714786 74.21283651325926) bank44666 +44667 POINT(41.469389679104694 73.41827779806755) bank44667 +44668 POINT(40.22579028129224 73.28707145298827) bank44668 +44669 POINT(39.79442776690575 74.01666730268589) bank44669 +44670 POINT(39.760755820398664 74.32710799482916) bank44670 +44671 POINT(41.58416559129756 73.10663179605271) bank44671 +44672 POINT(41.4951970679655 73.57553134871004) bank44672 +44673 POINT(40.02865911946543 73.14094415387096) bank44673 +44674 POINT(40.472683509339674 74.00991267474963) bank44674 +44675 POINT(41.36021607112604 73.88828363875321) bank44675 +44676 POINT(40.07621436725378 74.4294454926744) bank44676 +44677 POINT(40.948561600362694 74.84694146875451) bank44677 +44678 POINT(41.46758774253663 74.77585497857899) bank44678 +44679 POINT(41.23324728827641 74.64845762596526) bank44679 +44680 POINT(40.73385415212458 74.94399896675395) bank44680 +44681 POINT(40.614407794020465 73.9478619203069) bank44681 +44682 POINT(40.577968796216766 73.09528229745703) bank44682 +44683 POINT(40.6756610794285 73.64191007894522) bank44683 +44684 POINT(39.979577591241814 73.15142574127232) bank44684 +44685 POINT(40.199017402763616 73.74225766642914) bank44685 +44686 POINT(41.1387878209241 73.82921020121732) bank44686 +44687 POINT(40.247158282598754 74.21331375039071) bank44687 +44688 POINT(41.43076062999216 73.16047479755095) bank44688 +44689 POINT(40.19555072544515 73.76766994723246) bank44689 +44690 POINT(41.49644991404316 74.91759981612103) bank44690 +44691 POINT(40.93366664799685 73.36843262231471) bank44691 +44692 POINT(41.32546829311051 74.42470292631712) bank44692 +44693 POINT(40.35992999992212 73.8429648451661) bank44693 +44694 POINT(39.86925778773076 73.5353920851337) bank44694 +44695 POINT(41.09255531374981 73.69072554123439) bank44695 +44696 POINT(41.610889119831945 74.91115508544749) bank44696 +44697 POINT(41.557159305122134 73.49291627689219) bank44697 +44698 POINT(40.86559383546774 73.81802516096324) bank44698 +44699 POINT(39.96729247963186 73.02775598507195) bank44699 +44700 POINT(40.38440362736456 74.652640001612) bank44700 +44701 POINT(39.927521398509924 73.79243851377875) bank44701 +44702 POINT(40.53031845576287 73.10886485173627) bank44702 +44703 POINT(40.71468146190979 74.70928647869972) bank44703 +44704 POINT(41.057004250800105 73.17162088549152) bank44704 +44705 POINT(40.77590088231583 73.16554018785439) bank44705 +44706 POINT(40.62879387880037 74.91596402167482) bank44706 +44707 POINT(41.284198006130936 73.03533988413241) bank44707 +44708 POINT(40.72949901419779 74.30602990908952) bank44708 +44709 POINT(41.62248422897355 74.2709847812901) bank44709 +44710 POINT(41.186703210910665 73.34162528574893) bank44710 +44711 POINT(41.631387723599715 73.84619679212908) bank44711 +44712 POINT(40.401410814603 74.9682515666219) bank44712 +44713 POINT(40.77330477167612 73.70307610497967) bank44713 +44714 POINT(41.23238531456496 74.6892312053941) bank44714 +44715 POINT(40.65568344622059 74.24680639526223) bank44715 +44716 POINT(40.402812302927465 73.72461989969176) bank44716 +44717 POINT(40.17289185104668 74.11044389272223) bank44717 +44718 POINT(41.445079220778645 74.61693851990147) bank44718 +44719 POINT(41.13455841479662 73.44662494447486) bank44719 +44720 POINT(40.740820543732546 74.54745540874819) bank44720 +44721 POINT(40.34873822216798 74.56306510481042) bank44721 +44722 POINT(40.53115212917759 73.7489808124787) bank44722 +44723 POINT(41.05378308148008 73.26648395340455) bank44723 +44724 POINT(41.18831159009028 74.09840600486127) bank44724 +44725 POINT(41.46978228248651 74.20143071242775) bank44725 +44726 POINT(40.06215944852301 73.63845512934589) bank44726 +44727 POINT(41.70732194260486 73.82337219779068) bank44727 +44728 POINT(41.47924100840796 74.53939452920261) bank44728 +44729 POINT(40.170355711879616 73.10347745710422) bank44729 +44730 POINT(40.98249901061593 73.44914448224037) bank44730 +44731 POINT(40.998193094077465 73.57460086165399) bank44731 +44732 POINT(39.74080093041798 74.46509118949248) bank44732 +44733 POINT(41.46190433850379 74.01209120950149) bank44733 +44734 POINT(40.62972663173467 73.77266349760819) bank44734 +44735 POINT(41.39549785869168 74.65645648063304) bank44735 +44736 POINT(41.44003181102417 74.59292899563701) bank44736 +44737 POINT(41.170953453384534 73.17869306938096) bank44737 +44738 POINT(41.49817786979219 73.73754705851188) bank44738 +44739 POINT(41.312661127234485 73.81158572872069) bank44739 +44740 POINT(41.63558875098629 74.457049482079) bank44740 +44741 POINT(39.84129314840708 73.54810975816501) bank44741 +44742 POINT(41.6965160576926 73.6317862574067) bank44742 +44743 POINT(41.31811008795835 74.00115182884909) bank44743 +44744 POINT(40.63552859628817 74.93637674067273) bank44744 +44745 POINT(39.92091905451362 73.8171493042062) bank44745 +44746 POINT(40.19042577078399 73.52968777431488) bank44746 +44747 POINT(39.76788581095305 74.23774260861096) bank44747 +44748 POINT(40.730961679849784 73.13508852485421) bank44748 +44749 POINT(41.536452479687924 73.27977257019006) bank44749 +44750 POINT(40.17683872844308 74.53402555312582) bank44750 +44751 POINT(41.53742727680217 74.33499302500763) bank44751 +44752 POINT(41.57468514491896 74.97701540309772) bank44752 +44753 POINT(40.32103020676885 73.4356299429155) bank44753 +44754 POINT(41.28573775361173 74.04875458874542) bank44754 +44755 POINT(40.791991083573464 74.89820157893323) bank44755 +44756 POINT(40.693847708141874 74.35710822167417) bank44756 +44757 POINT(41.56653687440048 73.2011074518772) bank44757 +44758 POINT(40.7502212727767 73.05167895269925) bank44758 +44759 POINT(40.39882252067405 74.44566273412167) bank44759 +44760 POINT(40.307528433610756 74.24261852339924) bank44760 +44761 POINT(40.6205422529764 73.83551286122362) bank44761 +44762 POINT(40.31922932924673 73.01595674879978) bank44762 +44763 POINT(41.23570190092061 73.02016533794306) bank44763 +44764 POINT(40.22187897987689 73.08167067587145) bank44764 +44765 POINT(41.65001245576337 74.9651920231798) bank44765 +44766 POINT(40.28880311609732 73.38699123354678) bank44766 +44767 POINT(40.61123537292943 74.1950436903343) bank44767 +44768 POINT(41.251663679947846 73.40619242171616) bank44768 +44769 POINT(40.50513833949207 74.51624083837746) bank44769 +44770 POINT(39.898611519925666 73.27617947613108) bank44770 +44771 POINT(41.3690796714205 73.2475249683443) bank44771 +44772 POINT(40.15694238212804 74.38860160617894) bank44772 +44773 POINT(39.880692222450485 73.07138638265393) bank44773 +44774 POINT(40.35554077690784 74.27167602360869) bank44774 +44775 POINT(41.31306348760513 73.96277065696131) bank44775 +44776 POINT(40.074249040181044 73.60719778508441) bank44776 +44777 POINT(39.73033439532905 74.75976359539418) bank44777 +44778 POINT(40.79893143747381 74.1901725660186) bank44778 +44779 POINT(40.31466956353589 74.72400725804157) bank44779 +44780 POINT(41.46469135046031 74.96395541812637) bank44780 +44781 POINT(40.11974583570276 73.67962064026808) bank44781 +44782 POINT(40.716721812064165 74.8244394751518) bank44782 +44783 POINT(40.63238548613267 74.02987443957063) bank44783 +44784 POINT(41.4944475878025 74.73059242706725) bank44784 +44785 POINT(39.971344113357944 73.30580407462979) bank44785 +44786 POINT(39.785675829661784 74.50513999138309) bank44786 +44787 POINT(40.028490767382685 74.82029138998794) bank44787 +44788 POINT(41.35809946516282 73.28269611267838) bank44788 +44789 POINT(40.23712545149894 73.75888573985104) bank44789 +44790 POINT(41.2880767752277 73.68142928046194) bank44790 +44791 POINT(40.689565055961594 73.28061050608343) bank44791 +44792 POINT(39.93731812034072 74.58087905183055) bank44792 +44793 POINT(40.77711216881493 74.51858717031965) bank44793 +44794 POINT(41.42516608828422 74.27874210510245) bank44794 +44795 POINT(40.06215291178203 74.3669162577632) bank44795 +44796 POINT(40.58234541788066 74.4509629089712) bank44796 +44797 POINT(41.352029636279624 73.65160587194083) bank44797 +44798 POINT(39.76964662494558 73.36368741667832) bank44798 +44799 POINT(41.36408946602208 74.32188949221808) bank44799 +44800 POINT(40.90743233028253 73.80196114335635) bank44800 +44801 POINT(41.636522734760796 74.24497830276395) bank44801 +44802 POINT(41.106469869672246 73.35330928852363) bank44802 +44803 POINT(39.757542823001295 74.5944879301014) bank44803 +44804 POINT(40.93633131125745 74.63042386910524) bank44804 +44805 POINT(41.48949583073009 73.36728077192095) bank44805 +44806 POINT(39.72867829911489 74.9268780415641) bank44806 +44807 POINT(39.96842413270023 74.78865637327625) bank44807 +44808 POINT(41.564767376781724 74.98787541066247) bank44808 +44809 POINT(39.93682871749643 74.13626449141329) bank44809 +44810 POINT(40.030243457038424 74.22303770340511) bank44810 +44811 POINT(41.246951761970664 74.14187973277505) bank44811 +44812 POINT(40.833389779218244 73.48133703088352) bank44812 +44813 POINT(39.738490712043074 73.32497754963698) bank44813 +44814 POINT(41.057930709066206 73.25063090700672) bank44814 +44815 POINT(40.09656053137543 74.06138787583731) bank44815 +44816 POINT(40.57673494088862 73.04212111872833) bank44816 +44817 POINT(39.964093009292334 74.42697609685065) bank44817 +44818 POINT(41.33379302294577 74.34894930593569) bank44818 +44819 POINT(40.96082619108934 73.52992254068401) bank44819 +44820 POINT(41.19521693214917 73.58532764588522) bank44820 +44821 POINT(40.49792228082191 74.11962144397492) bank44821 +44822 POINT(40.26508389711943 74.12544703418484) bank44822 +44823 POINT(41.31643230864047 73.66800281160555) bank44823 +44824 POINT(40.72008859686282 74.96798805788605) bank44824 +44825 POINT(41.3265158961265 73.11245332722781) bank44825 +44826 POINT(40.361610486547946 74.6926653379031) bank44826 +44827 POINT(39.97681659862167 73.93067794175322) bank44827 +44828 POINT(41.346240310984676 73.96876839146852) bank44828 +44829 POINT(41.303037062354015 74.54206055248147) bank44829 +44830 POINT(41.67489303612497 74.80181874619691) bank44830 +44831 POINT(41.37110921333229 74.46693491335482) bank44831 +44832 POINT(40.585437998096126 73.28302966270392) bank44832 +44833 POINT(40.36585524311905 74.23664338141471) bank44833 +44834 POINT(39.77328935236011 74.83369694554071) bank44834 +44835 POINT(40.75005459784379 74.8202463257126) bank44835 +44836 POINT(39.985041662623914 73.56683098055757) bank44836 +44837 POINT(40.01652866143716 74.72370071029849) bank44837 +44838 POINT(39.943424904873844 74.085752134491) bank44838 +44839 POINT(41.65787871329676 74.1994544846165) bank44839 +44840 POINT(39.918076317793194 73.11072001091196) bank44840 +44841 POINT(40.14335693310705 73.271810731069) bank44841 +44842 POINT(40.29318887971992 73.23778522731615) bank44842 +44843 POINT(41.016415655848974 73.05662586471529) bank44843 +44844 POINT(41.13326266906224 73.57951949504707) bank44844 +44845 POINT(41.51920949688221 74.61283532786726) bank44845 +44846 POINT(40.06131973602622 74.09953888258757) bank44846 +44847 POINT(41.00192407823 73.82870547884035) bank44847 +44848 POINT(40.845241852867986 74.96820419148723) bank44848 +44849 POINT(41.06687660771461 73.19560212433672) bank44849 +44850 POINT(40.751250099655174 74.94426860316135) bank44850 +44851 POINT(39.95804293032751 74.30870358452609) bank44851 +44852 POINT(40.956251789730494 73.93342329183413) bank44852 +44853 POINT(40.99583947024891 74.45256440291908) bank44853 +44854 POINT(40.93320052771817 73.38056059575193) bank44854 +44855 POINT(40.08040836810158 74.89342442336135) bank44855 +44856 POINT(41.622556491319926 74.58633603133866) bank44856 +44857 POINT(41.14344984684398 74.12677889818484) bank44857 +44858 POINT(40.86884126685429 74.27980449938846) bank44858 +44859 POINT(40.744202474602794 73.93351975409665) bank44859 +44860 POINT(40.846341412935445 73.84566393348751) bank44860 +44861 POINT(40.65262904977488 74.58878789115418) bank44861 +44862 POINT(39.74754699764877 73.09292192828173) bank44862 +44863 POINT(39.959883935983754 74.34853476040756) bank44863 +44864 POINT(40.87126500060675 73.27082391152481) bank44864 +44865 POINT(39.78446616882705 74.3306789605766) bank44865 +44866 POINT(40.05020299517616 74.37066071652526) bank44866 +44867 POINT(41.52859827129998 74.79061069177595) bank44867 +44868 POINT(40.484087927438544 73.65698379353314) bank44868 +44869 POINT(41.107657205924646 73.08869697215117) bank44869 +44870 POINT(40.78266438498616 73.01496950795831) bank44870 +44871 POINT(40.811621628416546 73.79497612095828) bank44871 +44872 POINT(40.07561971672881 73.75733618168329) bank44872 +44873 POINT(41.65484352713195 73.43336009769764) bank44873 +44874 POINT(39.91043822652138 73.88778948780666) bank44874 +44875 POINT(40.77206924530089 74.65997153693155) bank44875 +44876 POINT(40.90476334734165 74.98758271051123) bank44876 +44877 POINT(41.25372240654886 73.81703067635571) bank44877 +44878 POINT(40.016328136031554 74.00571396113757) bank44878 +44879 POINT(40.1431096462232 74.28798391378318) bank44879 +44880 POINT(41.635078207288366 74.39534179451117) bank44880 +44881 POINT(40.29522820309029 73.7068269432938) bank44881 +44882 POINT(41.10515639418173 73.82412838436727) bank44882 +44883 POINT(39.73347751723856 73.53593725054294) bank44883 +44884 POINT(41.22802909041503 74.27104181474469) bank44884 +44885 POINT(40.13313578067578 74.68564982318175) bank44885 +44886 POINT(41.167238888550905 74.73848509622583) bank44886 +44887 POINT(40.81630541069948 73.85354513704495) bank44887 +44888 POINT(40.02237320951348 74.60553146058191) bank44888 +44889 POINT(41.36147019455932 74.23749539120438) bank44889 +44890 POINT(39.781296666195665 74.44161531121598) bank44890 +44891 POINT(40.94576685190688 74.48186207432403) bank44891 +44892 POINT(41.03285452477042 73.84588999902205) bank44892 +44893 POINT(39.95643871842833 73.45693117972623) bank44893 +44894 POINT(40.760481987052145 74.91559897186512) bank44894 +44895 POINT(40.01285313478853 74.17932133611453) bank44895 +44896 POINT(39.91468063471424 73.44600718834816) bank44896 +44897 POINT(41.627347487389 73.75123606408185) bank44897 +44898 POINT(40.67862882226296 73.99403222317738) bank44898 +44899 POINT(40.42539467788831 74.77611661648055) bank44899 +44900 POINT(39.89533936158807 73.89641442838884) bank44900 +44901 POINT(40.26927735201565 73.92403869094615) bank44901 +44902 POINT(40.0615462336201 73.64167050908337) bank44902 +44903 POINT(41.11709227906588 73.60684251761346) bank44903 +44904 POINT(40.78563903455546 73.30507751928732) bank44904 +44905 POINT(40.77224604139021 73.26074708747458) bank44905 +44906 POINT(40.22557122545809 73.88134150538083) bank44906 +44907 POINT(40.14556762398102 74.85674872874628) bank44907 +44908 POINT(40.7185932384464 74.66410453134824) bank44908 +44909 POINT(40.14052243419127 73.26593779064757) bank44909 +44910 POINT(39.783229358874365 74.39014133344675) bank44910 +44911 POINT(40.73525029573687 74.30029804626015) bank44911 +44912 POINT(40.32157430092577 74.14096771474739) bank44912 +44913 POINT(41.426840823723346 74.88807239341521) bank44913 +44914 POINT(39.958003368438625 74.77813176070828) bank44914 +44915 POINT(39.83734818195092 73.0608482193362) bank44915 +44916 POINT(41.44641566777871 74.07066214609385) bank44916 +44917 POINT(41.45952444671835 74.93063323735186) bank44917 +44918 POINT(41.033115485089255 74.27244443605055) bank44918 +44919 POINT(39.78613182609222 73.16134833146975) bank44919 +44920 POINT(41.245174200792825 73.28009314072224) bank44920 +44921 POINT(40.665916781962 73.15379558870866) bank44921 +44922 POINT(40.307143982111924 73.60056161334248) bank44922 +44923 POINT(40.099962424369075 73.74006204815609) bank44923 +44924 POINT(41.20904412431955 74.44137418454025) bank44924 +44925 POINT(41.39347167974676 74.27849666138286) bank44925 +44926 POINT(40.97537548032004 73.1131270711253) bank44926 +44927 POINT(40.56728294788664 73.80264632267333) bank44927 +44928 POINT(40.4863585199217 74.89002908300421) bank44928 +44929 POINT(41.58484342517894 74.7352686695795) bank44929 +44930 POINT(41.472638343552816 73.71444838759457) bank44930 +44931 POINT(39.791751719171415 74.95199997439411) bank44931 +44932 POINT(41.1639005105842 73.6195331293641) bank44932 +44933 POINT(40.44732543706264 74.28866706087398) bank44933 +44934 POINT(40.82212041840633 74.3005818592086) bank44934 +44935 POINT(39.742913684975335 74.74993896891438) bank44935 +44936 POINT(40.822835134676026 74.95255931716135) bank44936 +44937 POINT(39.721456184620926 73.73029945560147) bank44937 +44938 POINT(40.3338688210125 73.13742736636299) bank44938 +44939 POINT(39.87030366081335 73.17230088940838) bank44939 +44940 POINT(39.92612604581779 74.09623595045633) bank44940 +44941 POINT(41.5120708098696 74.13816828666684) bank44941 +44942 POINT(40.10069688142358 73.26421424315667) bank44942 +44943 POINT(41.529993217499154 74.09692435223556) bank44943 +44944 POINT(41.41310165878945 73.63235253832326) bank44944 +44945 POINT(40.830668952434394 74.69823656641718) bank44945 +44946 POINT(41.31823209088617 73.43176647491319) bank44946 +44947 POINT(40.71116521812783 74.13440197838842) bank44947 +44948 POINT(39.92671267396516 73.45575989389772) bank44948 +44949 POINT(40.83438805203097 74.36790428537168) bank44949 +44950 POINT(40.336556524192076 74.81721388974363) bank44950 +44951 POINT(39.82028554785938 74.84935393601594) bank44951 +44952 POINT(40.04787264793448 74.49730794169308) bank44952 +44953 POINT(40.61177120166636 74.6628650411141) bank44953 +44954 POINT(40.680102040009984 74.51888245169418) bank44954 +44955 POINT(41.488546556524135 74.01036288179269) bank44955 +44956 POINT(39.8254827883414 74.01829634410313) bank44956 +44957 POINT(40.195906139837774 73.39671278251197) bank44957 +44958 POINT(40.183811872628695 73.31081725101552) bank44958 +44959 POINT(41.37438969138712 74.83351571702215) bank44959 +44960 POINT(39.797126195136094 73.10514743848292) bank44960 +44961 POINT(41.28761860738214 73.32627012337178) bank44961 +44962 POINT(41.045247299327364 74.98576348415264) bank44962 +44963 POINT(41.044243986165355 73.18041903791885) bank44963 +44964 POINT(39.730671699413875 73.15425094233804) bank44964 +44965 POINT(41.4375768740735 74.81140889016976) bank44965 +44966 POINT(40.6818628266213 73.36414396468771) bank44966 +44967 POINT(40.41712720585411 74.7361906078852) bank44967 +44968 POINT(41.02968929152033 74.56233162679551) bank44968 +44969 POINT(41.68406522368309 74.04592143750602) bank44969 +44970 POINT(40.42697488962921 74.37228218937148) bank44970 +44971 POINT(41.451819683242846 73.16285096180306) bank44971 +44972 POINT(40.773039309378994 74.26303057361675) bank44972 +44973 POINT(40.63893327213797 74.8341310581936) bank44973 +44974 POINT(40.56054445221322 73.95490683232528) bank44974 +44975 POINT(40.27432299610674 73.89516051828541) bank44975 +44976 POINT(41.32922816931996 73.02191701774501) bank44976 +44977 POINT(40.77303074430827 73.86996951950674) bank44977 +44978 POINT(41.34592312998081 73.74090534083858) bank44978 +44979 POINT(40.579600703043596 73.74909338408527) bank44979 +44980 POINT(40.30051689622938 74.36386270111551) bank44980 +44981 POINT(39.91547075067307 73.0814095771235) bank44981 +44982 POINT(41.38215989499579 73.09789243197629) bank44982 +44983 POINT(39.72952815685559 73.164997404576) bank44983 +44984 POINT(40.197934214720675 73.84645552694303) bank44984 +44985 POINT(40.171815630893434 74.47512020782966) bank44985 +44986 POINT(40.531061955763725 74.41048342496941) bank44986 +44987 POINT(41.00300334547876 73.17807748050099) bank44987 +44988 POINT(39.74339109989612 73.74567209571904) bank44988 +44989 POINT(41.50683433667065 73.9866672527422) bank44989 +44990 POINT(41.62617311609547 74.12819101015768) bank44990 +44991 POINT(41.61534945247788 74.02315257649231) bank44991 +44992 POINT(39.99593078109387 74.46065054987646) bank44992 +44993 POINT(40.71909296391739 73.59825197720704) bank44993 +44994 POINT(41.23901660729961 74.9726036572428) bank44994 +44995 POINT(40.291802871766855 74.84065944471364) bank44995 +44996 POINT(39.84218594557452 73.73956927864249) bank44996 +44997 POINT(41.53233291271859 73.73340939765427) bank44997 +44998 POINT(41.18190451083372 74.18414234988792) bank44998 +44999 POINT(41.051120138937186 73.05961032339955) bank44999 +45000 POINT(40.82714891159391 74.41119874369838) bank45000 +45001 POINT(41.6463518660744 74.17576214609177) bank45001 +45002 POINT(41.53639767371145 74.41272313151356) bank45002 +45003 POINT(41.52797725037381 73.50755881953157) bank45003 +45004 POINT(40.588571306224594 74.45194047749489) bank45004 +45005 POINT(41.480551548292894 73.27171148542088) bank45005 +45006 POINT(41.657423255734955 74.88593752905075) bank45006 +45007 POINT(40.418482823812745 74.51015928062846) bank45007 +45008 POINT(40.32252637274189 73.17268124129093) bank45008 +45009 POINT(40.311877115681035 74.32614965851938) bank45009 +45010 POINT(40.848481464321715 73.07846705204355) bank45010 +45011 POINT(41.70918232130428 73.23396229384663) bank45011 +45012 POINT(40.58256945579376 73.31912202315986) bank45012 +45013 POINT(40.501314024872556 74.0119832540879) bank45013 +45014 POINT(40.252775171870574 73.48467886475447) bank45014 +45015 POINT(40.15080466861212 73.88986141406836) bank45015 +45016 POINT(41.07513591479264 74.14046734689785) bank45016 +45017 POINT(39.71449528653872 74.49171238071538) bank45017 +45018 POINT(41.169281794301945 74.7014445833051) bank45018 +45019 POINT(41.239911055699444 74.05726282926796) bank45019 +45020 POINT(41.651277014277404 74.00324152235125) bank45020 +45021 POINT(41.36233227946868 74.01898497974726) bank45021 +45022 POINT(41.62709590466509 74.32788039134009) bank45022 +45023 POINT(39.842568996417484 73.77033648180283) bank45023 +45024 POINT(39.81051438174694 73.88282893377645) bank45024 +45025 POINT(39.922516841107566 73.40511345895965) bank45025 +45026 POINT(40.24445963481332 74.62466521134245) bank45026 +45027 POINT(41.417535425171046 74.42063909263034) bank45027 +45028 POINT(40.20609191615508 74.17057443800505) bank45028 +45029 POINT(40.639643054126076 73.03431259972824) bank45029 +45030 POINT(40.65558400661593 74.52917435474302) bank45030 +45031 POINT(39.85940057105251 74.80959787837453) bank45031 +45032 POINT(40.72718359900658 73.62060773367553) bank45032 +45033 POINT(40.12900686303285 74.32343287500987) bank45033 +45034 POINT(41.388562666610284 74.22220996951319) bank45034 +45035 POINT(41.26447957706387 74.1938423330512) bank45035 +45036 POINT(40.822833232727554 74.3037303898632) bank45036 +45037 POINT(40.81813545649859 74.58098451187008) bank45037 +45038 POINT(40.26022192719437 74.6282350864136) bank45038 +45039 POINT(39.86697228267215 73.36535280598147) bank45039 +45040 POINT(41.27144610457371 73.99234152660485) bank45040 +45041 POINT(39.934901088424446 73.30365813750379) bank45041 +45042 POINT(40.354035918953166 73.25218570184094) bank45042 +45043 POINT(40.680562385995515 73.22529648172778) bank45043 +45044 POINT(40.05606354474102 74.53843568199534) bank45044 +45045 POINT(39.942842483751015 73.84794849631975) bank45045 +45046 POINT(41.23861784043836 74.78605972270704) bank45046 +45047 POINT(40.85272778003941 74.8912469514316) bank45047 +45048 POINT(39.87637929392089 73.0288032856611) bank45048 +45049 POINT(41.300381279906965 73.02719526581211) bank45049 +45050 POINT(39.78217821772591 73.44855034647416) bank45050 +45051 POINT(39.866824984992775 73.10280436029618) bank45051 +45052 POINT(40.38686335591326 74.9348617945672) bank45052 +45053 POINT(41.30998390390216 73.10424629076076) bank45053 +45054 POINT(41.57915506585135 74.87294800539048) bank45054 +45055 POINT(40.31144198054993 73.53582020075581) bank45055 +45056 POINT(41.338483930822676 74.61115575605491) bank45056 +45057 POINT(41.518747890062095 74.03633812503384) bank45057 +45058 POINT(40.654841758888054 74.8119022857742) bank45058 +45059 POINT(40.77785786335996 73.97080359410656) bank45059 +45060 POINT(41.48110463822522 74.13146591214601) bank45060 +45061 POINT(39.80686215737933 74.33281144472707) bank45061 +45062 POINT(41.18533565191665 73.33158690281962) bank45062 +45063 POINT(39.86698561001746 74.42646906992285) bank45063 +45064 POINT(40.733607105423424 74.44848884358845) bank45064 +45065 POINT(41.629556158146215 73.86085489105265) bank45065 +45066 POINT(41.71090874209179 74.04165046235093) bank45066 +45067 POINT(40.91163344881832 74.05055618806924) bank45067 +45068 POINT(40.54334471255532 74.53497542938184) bank45068 +45069 POINT(40.95575610857345 73.81633308518799) bank45069 +45070 POINT(41.057552436339904 73.19517913579197) bank45070 +45071 POINT(40.46637259948116 74.55342125691824) bank45071 +45072 POINT(39.71898379301124 74.43929230559813) bank45072 +45073 POINT(39.95552696147183 74.2375599463152) bank45073 +45074 POINT(41.32790290032676 74.7401054494643) bank45074 +45075 POINT(41.434133500891264 74.3479072042658) bank45075 +45076 POINT(40.912203275783874 74.52884998910211) bank45076 +45077 POINT(41.573007945345 74.95574379764403) bank45077 +45078 POINT(41.66474531894723 74.52655727500135) bank45078 +45079 POINT(41.020265371923415 73.98389853510268) bank45079 +45080 POINT(40.60316770843878 73.95881066523062) bank45080 +45081 POINT(39.830519018313254 74.27905440497912) bank45081 +45082 POINT(40.603773168096275 74.75903235739054) bank45082 +45083 POINT(41.020256676515785 74.79925482167694) bank45083 +45084 POINT(41.381069899322405 74.04912067169123) bank45084 +45085 POINT(40.99947836090451 73.58344656080449) bank45085 +45086 POINT(41.325244827770824 74.50106146881521) bank45086 +45087 POINT(40.37140973095594 74.5257434676496) bank45087 +45088 POINT(39.85451005820731 74.08247553790484) bank45088 +45089 POINT(41.563009339924626 74.35888555629813) bank45089 +45090 POINT(40.319098346183395 73.30953628533833) bank45090 +45091 POINT(41.61035535763264 74.94489907031544) bank45091 +45092 POINT(39.72115849223335 73.91789287008886) bank45092 +45093 POINT(41.49813383785669 73.70910760177489) bank45093 +45094 POINT(40.76372814936884 73.32249463791177) bank45094 +45095 POINT(39.83626052062579 74.90608894216545) bank45095 +45096 POINT(39.938415532427804 74.85090611650129) bank45096 +45097 POINT(40.031758691626806 73.91836137717979) bank45097 +45098 POINT(41.02964943984558 73.10917041055728) bank45098 +45099 POINT(39.918523759009204 74.78241011115338) bank45099 +45100 POINT(40.423152709237336 74.93571678805785) bank45100 +45101 POINT(41.20532732769714 74.54635168034848) bank45101 +45102 POINT(40.53944095362758 73.12433146255357) bank45102 +45103 POINT(39.95776625102553 74.03377502345568) bank45103 +45104 POINT(40.12680735820334 73.16652280332593) bank45104 +45105 POINT(41.61606167250703 74.15228352549832) bank45105 +45106 POINT(40.30497418105131 74.87522533762598) bank45106 +45107 POINT(39.7197691277958 73.37897579009135) bank45107 +45108 POINT(40.054724611074164 73.40445388308201) bank45108 +45109 POINT(40.09562666961643 73.58481530305494) bank45109 +45110 POINT(40.6152885352801 74.21838365887231) bank45110 +45111 POINT(41.54196170965073 73.85759634435406) bank45111 +45112 POINT(40.51625602676839 74.24093654385646) bank45112 +45113 POINT(41.04599568382285 73.64765976726297) bank45113 +45114 POINT(39.74054166979619 73.17094120944431) bank45114 +45115 POINT(41.43462149229976 73.34823875664226) bank45115 +45116 POINT(41.3908339863691 74.86185857238691) bank45116 +45117 POINT(40.26479481208825 74.88033593271116) bank45117 +45118 POINT(40.01047507657751 73.84399214314449) bank45118 +45119 POINT(41.31738088956169 74.35724094001195) bank45119 +45120 POINT(41.1051352581575 74.3062098116468) bank45120 +45121 POINT(40.90288504779654 73.18197930841069) bank45121 +45122 POINT(39.895218672451655 73.05071133667097) bank45122 +45123 POINT(41.145624364012264 73.23080421758256) bank45123 +45124 POINT(40.76675828943032 73.99666573503258) bank45124 +45125 POINT(40.54175264404927 74.39240428186582) bank45125 +45126 POINT(41.591128446073014 73.48007861815064) bank45126 +45127 POINT(39.967095935872564 74.35139301024304) bank45127 +45128 POINT(40.89890665602794 73.23458711580327) bank45128 +45129 POINT(41.61961720584079 73.34799874345357) bank45129 +45130 POINT(40.341156609080755 73.86362156682955) bank45130 +45131 POINT(40.47618478731133 73.66333202353506) bank45131 +45132 POINT(41.54778426227997 73.46656243610305) bank45132 +45133 POINT(41.384004845552646 73.35964355463663) bank45133 +45134 POINT(41.030411144639636 74.24601716550715) bank45134 +45135 POINT(41.69227044661498 73.35026650600079) bank45135 +45136 POINT(41.02954898438818 74.84143801970056) bank45136 +45137 POINT(40.362168444816376 73.28113610558701) bank45137 +45138 POINT(40.84938756630351 73.15138710096525) bank45138 +45139 POINT(39.976924099572216 74.6991589128652) bank45139 +45140 POINT(40.0182705738912 74.42732289614543) bank45140 +45141 POINT(39.822226274208816 74.19276641410194) bank45141 +45142 POINT(41.50721861350579 73.70703053540485) bank45142 +45143 POINT(40.093768807247 74.5568466002155) bank45143 +45144 POINT(40.1115956316175 74.2041276642112) bank45144 +45145 POINT(41.27020839256017 73.55076775485954) bank45145 +45146 POINT(41.59568868910901 73.68776102866593) bank45146 +45147 POINT(41.21992504949417 74.08802475580102) bank45147 +45148 POINT(41.22133085549472 74.6084358638639) bank45148 +45149 POINT(41.60146485583297 74.88063425316665) bank45149 +45150 POINT(41.70541642779648 74.5087058818129) bank45150 +45151 POINT(40.32570688157667 73.4434969145461) bank45151 +45152 POINT(40.754347146039144 73.38327831887739) bank45152 +45153 POINT(39.817697770337055 73.8167840361519) bank45153 +45154 POINT(40.614058275542604 74.12213510343973) bank45154 +45155 POINT(40.367845007890246 74.65462208426419) bank45155 +45156 POINT(40.506738296388 74.94217273295011) bank45156 +45157 POINT(41.195096958093266 73.32084352554747) bank45157 +45158 POINT(40.248945942967474 73.69114626435974) bank45158 +45159 POINT(39.925389552900896 74.29838133802245) bank45159 +45160 POINT(40.44694321027531 73.78688749410452) bank45160 +45161 POINT(41.6348626676711 74.01838204446723) bank45161 +45162 POINT(40.242622363268204 73.87187983151715) bank45162 +45163 POINT(40.63513952176339 74.83419493308799) bank45163 +45164 POINT(39.74818727060607 74.10629739933916) bank45164 +45165 POINT(40.19794681372798 73.96762254779952) bank45165 +45166 POINT(41.13331894410991 73.86656119112216) bank45166 +45167 POINT(40.21103883323376 74.17713599302586) bank45167 +45168 POINT(41.394591151273175 74.97755070697106) bank45168 +45169 POINT(40.09809352278823 74.9423241569691) bank45169 +45170 POINT(39.74480117046084 74.60416707961093) bank45170 +45171 POINT(39.74358587220667 74.81678703492635) bank45171 +45172 POINT(41.181567135006794 73.12532291370916) bank45172 +45173 POINT(41.03678802434622 74.51273602660937) bank45173 +45174 POINT(41.43419108663183 74.23978159656004) bank45174 +45175 POINT(41.13824515084373 73.28085018569938) bank45175 +45176 POINT(40.585774996508036 74.70959539502613) bank45176 +45177 POINT(40.48935184230307 74.44036467720574) bank45177 +45178 POINT(40.93827827161069 73.97049063100154) bank45178 +45179 POINT(39.8830785209292 73.65193518175774) bank45179 +45180 POINT(40.34127419861038 73.02230578591362) bank45180 +45181 POINT(41.63758604613239 74.79103330807806) bank45181 +45182 POINT(40.07354704711126 73.50394242102577) bank45182 +45183 POINT(41.417818050745474 74.99523888204752) bank45183 +45184 POINT(40.273244949501986 74.7572672905648) bank45184 +45185 POINT(41.37712453497444 74.07293292174023) bank45185 +45186 POINT(40.80411685703783 73.418251403932) bank45186 +45187 POINT(41.216587067945426 74.14016772216988) bank45187 +45188 POINT(39.72252429613827 74.87113811922201) bank45188 +45189 POINT(40.268916611913475 74.05098593947139) bank45189 +45190 POINT(41.5135311155899 73.89424359482805) bank45190 +45191 POINT(40.33096639636955 73.0332192685192) bank45191 +45192 POINT(40.479914352989454 74.1316587069438) bank45192 +45193 POINT(41.006290573371494 73.01223314275921) bank45193 +45194 POINT(41.625689419972176 74.49898829965244) bank45194 +45195 POINT(40.359006120804544 74.09946270357565) bank45195 +45196 POINT(41.19043762129325 74.91161067934205) bank45196 +45197 POINT(40.21887392262827 74.89246916799222) bank45197 +45198 POINT(39.755131419203195 74.21830610904011) bank45198 +45199 POINT(40.374179293674636 74.88422091909881) bank45199 +45200 POINT(40.79656129553332 73.74482851378335) bank45200 +45201 POINT(40.072835102585 73.41230355195152) bank45201 +45202 POINT(41.210494577416306 73.91141811235934) bank45202 +45203 POINT(40.102455569305064 74.87240092872992) bank45203 +45204 POINT(40.649449743896156 74.78862767552513) bank45204 +45205 POINT(40.061079111241675 73.66765648507754) bank45205 +45206 POINT(41.29966795865465 73.55558317725395) bank45206 +45207 POINT(39.80809629066665 73.30985489619526) bank45207 +45208 POINT(40.556539704321615 73.44177474453008) bank45208 +45209 POINT(40.838645938089286 74.63410627349569) bank45209 +45210 POINT(40.93116856144351 74.21198849620139) bank45210 +45211 POINT(40.6090284540691 73.86757065306108) bank45211 +45212 POINT(41.32055997510782 74.47460385211019) bank45212 +45213 POINT(40.03710290663129 73.67615496352387) bank45213 +45214 POINT(40.55315333166011 73.11398389592802) bank45214 +45215 POINT(41.246100095154304 74.36741995009484) bank45215 +45216 POINT(40.568993042927616 74.20877458967779) bank45216 +45217 POINT(40.94246839899463 73.94559718026862) bank45217 +45218 POINT(39.745943610926766 74.4889379295852) bank45218 +45219 POINT(41.40116127031514 73.55656625122617) bank45219 +45220 POINT(39.82240568023469 74.3819569473755) bank45220 +45221 POINT(41.674048940974636 74.81620322843438) bank45221 +45222 POINT(41.66617116160757 74.29714236314952) bank45222 +45223 POINT(41.180628042659556 74.54137031494535) bank45223 +45224 POINT(40.856333443007074 74.21252122921238) bank45224 +45225 POINT(40.756844520966304 74.15803784764007) bank45225 +45226 POINT(41.15371657242884 74.65701395917797) bank45226 +45227 POINT(40.59119970324689 73.78084283138912) bank45227 +45228 POINT(40.452298619749435 73.2793968542954) bank45228 +45229 POINT(40.49242977458201 73.69348993258816) bank45229 +45230 POINT(41.15676609140865 73.2098494536353) bank45230 +45231 POINT(41.215439402024195 74.93756361993276) bank45231 +45232 POINT(40.796508840799376 73.89990960159741) bank45232 +45233 POINT(40.73368411684011 73.29273947812155) bank45233 +45234 POINT(40.54085966346919 73.65496215689795) bank45234 +45235 POINT(41.617111145236855 74.34358431296107) bank45235 +45236 POINT(39.77579024662812 73.47752389440434) bank45236 +45237 POINT(41.49354766236253 74.4196582750946) bank45237 +45238 POINT(41.34128799255313 73.5475624507421) bank45238 +45239 POINT(40.13018826324758 73.26057847446506) bank45239 +45240 POINT(40.0078437561072 73.59817404816059) bank45240 +45241 POINT(40.18731956815156 73.98235508561666) bank45241 +45242 POINT(40.07932743562555 74.18460790889142) bank45242 +45243 POINT(40.79631145411871 74.84977903348381) bank45243 +45244 POINT(41.17705456837026 73.78105251552955) bank45244 +45245 POINT(41.51418481230003 73.23397947205196) bank45245 +45246 POINT(40.99388631785684 74.91620923614154) bank45246 +45247 POINT(40.50736854383582 73.0341262899848) bank45247 +45248 POINT(39.718610614959594 74.020715602794) bank45248 +45249 POINT(41.36819425651943 74.55045550947567) bank45249 +45250 POINT(41.60536462311506 73.4858756890565) bank45250 +45251 POINT(40.614127185005444 74.75429593319444) bank45251 +45252 POINT(40.26817219414036 73.01443688261504) bank45252 +45253 POINT(40.88687234494593 73.1973553353002) bank45253 +45254 POINT(40.93741417815271 74.38238581912913) bank45254 +45255 POINT(40.42939121001598 74.72626941377303) bank45255 +45256 POINT(40.87505413708955 73.90444003680068) bank45256 +45257 POINT(40.778206301689046 74.13124112746704) bank45257 +45258 POINT(40.56858042041565 73.81464077970095) bank45258 +45259 POINT(40.58309976502135 74.83785551647695) bank45259 +45260 POINT(40.33403890375468 74.30007945047299) bank45260 +45261 POINT(41.031174176065576 74.47711815145789) bank45261 +45262 POINT(41.61505073356899 73.97031372099252) bank45262 +45263 POINT(40.2012790730677 73.24491652653626) bank45263 +45264 POINT(41.13587823596219 73.18464137882692) bank45264 +45265 POINT(40.175848831865096 73.79254264192707) bank45265 +45266 POINT(41.02687505263704 73.22948613249036) bank45266 +45267 POINT(40.384130481416875 73.7839965569338) bank45267 +45268 POINT(40.26186747746948 73.89793100395148) bank45268 +45269 POINT(40.943442398100096 74.66550410375486) bank45269 +45270 POINT(40.93916795026054 74.84973768386654) bank45270 +45271 POINT(39.985513421449845 74.77569275950523) bank45271 +45272 POINT(41.62833777362744 74.90543372337919) bank45272 +45273 POINT(40.389521392952624 73.51067830852473) bank45273 +45274 POINT(40.557418229267874 74.66589935480688) bank45274 +45275 POINT(39.75357207117344 73.25451421722545) bank45275 +45276 POINT(41.02140558381938 73.73893708183205) bank45276 +45277 POINT(41.116492022225536 74.093086306052) bank45277 +45278 POINT(39.8989517012201 73.58205089132774) bank45278 +45279 POINT(40.047403700287674 74.25397015917021) bank45279 +45280 POINT(40.61476014176048 74.58860753632197) bank45280 +45281 POINT(41.06622606014123 74.59640509521249) bank45281 +45282 POINT(41.12649986347278 74.18956061148677) bank45282 +45283 POINT(40.752803200340416 74.5255241825457) bank45283 +45284 POINT(41.552092715618095 74.6169701925345) bank45284 +45285 POINT(41.48674984025197 74.91332452769696) bank45285 +45286 POINT(40.227534309540935 74.62985755494239) bank45286 +45287 POINT(39.894746849884676 74.6170620218161) bank45287 +45288 POINT(40.23614724471072 74.42855526941166) bank45288 +45289 POINT(40.45906002389587 73.98729125449519) bank45289 +45290 POINT(40.176936681859495 74.88450755097759) bank45290 +45291 POINT(40.46020077740329 74.60727677058553) bank45291 +45292 POINT(40.45029537579961 73.5529116505181) bank45292 +45293 POINT(40.8375549146444 74.83160931130246) bank45293 +45294 POINT(41.35997860179614 73.95368614734052) bank45294 +45295 POINT(39.862062140913 74.73601077707491) bank45295 +45296 POINT(40.23976331933164 73.24245682489463) bank45296 +45297 POINT(40.98973098776011 74.82976036207262) bank45297 +45298 POINT(40.59932663277057 74.1501069415046) bank45298 +45299 POINT(41.02695962024745 74.35618179604805) bank45299 +45300 POINT(40.37238616756766 73.2488768486021) bank45300 +45301 POINT(41.60329832776673 74.76019817674477) bank45301 +45302 POINT(40.3760644052888 73.14344880440109) bank45302 +45303 POINT(39.822905769653545 74.44401555766787) bank45303 +45304 POINT(40.62825899873983 73.23971398540868) bank45304 +45305 POINT(41.28036280981746 74.03683638299957) bank45305 +45306 POINT(40.62826758861545 73.43662164192989) bank45306 +45307 POINT(41.706006010607815 73.36275334507313) bank45307 +45308 POINT(40.002197539158445 74.34455765465553) bank45308 +45309 POINT(40.679501296793724 74.92732699198781) bank45309 +45310 POINT(40.56525177611834 74.50958557310277) bank45310 +45311 POINT(40.002827219823374 73.30229375089246) bank45311 +45312 POINT(40.45877327384145 74.8036258204974) bank45312 +45313 POINT(40.764029983359784 74.0745299848855) bank45313 +45314 POINT(39.927943792422695 74.89342733355235) bank45314 +45315 POINT(40.74326212130215 73.74438950207534) bank45315 +45316 POINT(40.11087431861109 73.11645432777502) bank45316 +45317 POINT(40.63317190983373 73.13585093950076) bank45317 +45318 POINT(40.08338231336563 74.51191212731959) bank45318 +45319 POINT(41.021961758891926 74.93908270914929) bank45319 +45320 POINT(41.375495066202234 74.03725670950128) bank45320 +45321 POINT(39.92487750979158 74.90453032347371) bank45321 +45322 POINT(40.79847528380955 73.83777463702832) bank45322 +45323 POINT(40.61722073458609 73.07366723937339) bank45323 +45324 POINT(41.37746737629073 73.6876504954699) bank45324 +45325 POINT(40.901575263589834 74.7693036124007) bank45325 +45326 POINT(40.982313290992835 73.22059123592462) bank45326 +45327 POINT(40.168023016467245 73.6010833352635) bank45327 +45328 POINT(41.392526078434365 73.56292062496276) bank45328 +45329 POINT(41.455512516120606 74.00511622918889) bank45329 +45330 POINT(41.031010370984745 74.39661897721292) bank45330 +45331 POINT(39.94428361595319 74.02497382166935) bank45331 +45332 POINT(40.17809239113686 74.31919036296497) bank45332 +45333 POINT(40.13070583748445 74.58864194017094) bank45333 +45334 POINT(39.94030011258719 74.23578645822148) bank45334 +45335 POINT(41.428149412147754 73.86649693384436) bank45335 +45336 POINT(40.322449024713386 74.63360245880233) bank45336 +45337 POINT(41.156182665027586 74.90807901061815) bank45337 +45338 POINT(40.54778020605275 74.77601791861318) bank45338 +45339 POINT(40.34704310183329 74.23508492696028) bank45339 +45340 POINT(41.338358519936065 73.24069156974213) bank45340 +45341 POINT(40.24155215802625 73.97080335009909) bank45341 +45342 POINT(41.10724914800021 73.88245928921638) bank45342 +45343 POINT(41.597897092761556 74.77973227829163) bank45343 +45344 POINT(39.8304896281467 73.84457082958231) bank45344 +45345 POINT(41.50186595456345 73.19057398925221) bank45345 +45346 POINT(40.08548844946366 74.7500948648577) bank45346 +45347 POINT(41.45924474478404 74.45789672492049) bank45347 +45348 POINT(40.766907952745896 73.45385823657828) bank45348 +45349 POINT(40.565265980391516 74.88056789442128) bank45349 +45350 POINT(39.989425465233474 73.50359149120528) bank45350 +45351 POINT(40.36163899313378 73.68527134759857) bank45351 +45352 POINT(41.34573020045088 74.58113348814528) bank45352 +45353 POINT(40.80748980408697 74.86510855371507) bank45353 +45354 POINT(41.13461393178231 73.84522524324122) bank45354 +45355 POINT(41.40084122236809 73.7057619580827) bank45355 +45356 POINT(40.4906932756743 74.15057023610719) bank45356 +45357 POINT(39.876083756932225 73.45627049535868) bank45357 +45358 POINT(40.77982368567122 74.42687896695793) bank45358 +45359 POINT(40.92158297502036 73.7423999916658) bank45359 +45360 POINT(41.69584475018544 73.81554312907728) bank45360 +45361 POINT(40.45609454783819 73.16104858362158) bank45361 +45362 POINT(40.08001099470474 74.22860205794018) bank45362 +45363 POINT(41.661497339848204 74.35409582336338) bank45363 +45364 POINT(39.922715116974416 74.68912634042928) bank45364 +45365 POINT(39.91435657611454 74.05992971293622) bank45365 +45366 POINT(40.80428785554182 73.66031827092868) bank45366 +45367 POINT(41.16581799502474 73.55240569841544) bank45367 +45368 POINT(40.06631336796629 73.4082531200433) bank45368 +45369 POINT(40.95776482187843 74.45042366527757) bank45369 +45370 POINT(41.27424043038484 74.73431919634598) bank45370 +45371 POINT(41.11869385497626 74.48347640955954) bank45371 +45372 POINT(40.18931888192303 73.6462550361764) bank45372 +45373 POINT(39.71635206202856 74.71557061719032) bank45373 +45374 POINT(40.664305873744524 74.01378827658017) bank45374 +45375 POINT(39.87239427622094 74.18721529450593) bank45375 +45376 POINT(40.600905204765326 73.81106132340882) bank45376 +45377 POINT(40.003463885219 73.00684166812108) bank45377 +45378 POINT(40.315596022280424 74.5864690308902) bank45378 +45379 POINT(40.693945792792675 74.14820786609482) bank45379 +45380 POINT(41.04671724038223 74.59635048453873) bank45380 +45381 POINT(40.649463441056454 74.53597964558924) bank45381 +45382 POINT(41.12227514826347 73.29604862966825) bank45382 +45383 POINT(41.34984971351545 74.21254719315715) bank45383 +45384 POINT(39.95511418311213 74.00703097559051) bank45384 +45385 POINT(39.89483299110381 74.92847721482421) bank45385 +45386 POINT(40.988395390147716 73.94324010617967) bank45386 +45387 POINT(39.88509246126112 73.80913825174376) bank45387 +45388 POINT(40.732215630922084 73.6171915912048) bank45388 +45389 POINT(41.364532569103744 73.2279803996712) bank45389 +45390 POINT(40.39284948001135 73.60889291360117) bank45390 +45391 POINT(41.45653726485545 74.55724774449052) bank45391 +45392 POINT(41.611138682818286 73.49101849178456) bank45392 +45393 POINT(40.669880501675124 73.59535997915515) bank45393 +45394 POINT(41.631322213871385 73.75245459298938) bank45394 +45395 POINT(41.13248314485431 73.33712504958315) bank45395 +45396 POINT(40.69231319627096 73.79047119480656) bank45396 +45397 POINT(40.54127101834871 73.01759301061527) bank45397 +45398 POINT(39.976264449642656 74.06715747908372) bank45398 +45399 POINT(40.60929786396682 74.20478733427709) bank45399 +45400 POINT(40.04654155867943 73.26178171808444) bank45400 +45401 POINT(40.43960360292929 74.06065588898134) bank45401 +45402 POINT(40.09020276744063 74.60046827815303) bank45402 +45403 POINT(40.809872731426296 74.08221002387519) bank45403 +45404 POINT(41.151941783190395 74.12372929495675) bank45404 +45405 POINT(39.74628062254532 73.19775314674416) bank45405 +45406 POINT(41.24093705688016 73.01611323816873) bank45406 +45407 POINT(41.55514154851355 74.31972883008162) bank45407 +45408 POINT(41.693895266268974 73.69542013979508) bank45408 +45409 POINT(40.34816932531975 73.16345408044056) bank45409 +45410 POINT(40.9542160259499 74.47393259751313) bank45410 +45411 POINT(39.82748799039442 73.69760752882347) bank45411 +45412 POINT(41.25916134461458 73.08123712095761) bank45412 +45413 POINT(40.608647787969296 73.79660684460272) bank45413 +45414 POINT(40.95201935446922 73.56609590286729) bank45414 +45415 POINT(40.472899753838625 73.27820268335282) bank45415 +45416 POINT(40.490827442681926 73.18954523459773) bank45416 +45417 POINT(40.64959890527301 74.48816903519246) bank45417 +45418 POINT(40.12146737914839 74.95932176185616) bank45418 +45419 POINT(40.33827722222468 73.86278311910661) bank45419 +45420 POINT(40.61262383654141 74.96479402154566) bank45420 +45421 POINT(41.28803499250761 74.70426270338409) bank45421 +45422 POINT(40.549506091957134 74.11943439048544) bank45422 +45423 POINT(41.16240208765014 73.49917199618723) bank45423 +45424 POINT(41.051872693727006 73.87581348137297) bank45424 +45425 POINT(41.02607562757146 73.04414963487984) bank45425 +45426 POINT(40.87254982627813 73.75366294292789) bank45426 +45427 POINT(39.9595560769096 73.34616833353543) bank45427 +45428 POINT(39.748297475211125 74.86790806350747) bank45428 +45429 POINT(40.09859062880209 74.22741727164846) bank45429 +45430 POINT(41.06019372645483 74.34373008810806) bank45430 +45431 POINT(39.7326413055461 74.04019084825448) bank45431 +45432 POINT(40.90248838232596 74.650058209586) bank45432 +45433 POINT(41.09117248124991 73.91706614278111) bank45433 +45434 POINT(41.451895191652326 74.88063725178878) bank45434 +45435 POINT(41.40194804444562 73.56113146032362) bank45435 +45436 POINT(41.162063243972995 73.63868489300911) bank45436 +45437 POINT(40.43146529843971 74.86699530479291) bank45437 +45438 POINT(40.33136551155156 73.73795089577895) bank45438 +45439 POINT(40.92447011484733 73.2840000894883) bank45439 +45440 POINT(39.735291320798986 73.24576278041486) bank45440 +45441 POINT(41.28487146111054 73.62899433039591) bank45441 +45442 POINT(40.344264690371396 73.69471090358465) bank45442 +45443 POINT(40.64142816569638 74.41798564194023) bank45443 +45444 POINT(41.71143879796505 74.49335655958625) bank45444 +45445 POINT(39.99987501958634 73.37190361555282) bank45445 +45446 POINT(39.89355802457908 74.6509968041879) bank45446 +45447 POINT(40.34427188697732 73.93566901272577) bank45447 +45448 POINT(40.912684191978755 74.04231976298227) bank45448 +45449 POINT(41.083720864155964 73.68717791960013) bank45449 +45450 POINT(41.460878002712484 73.00709147176138) bank45450 +45451 POINT(40.14047231720835 74.53745523290303) bank45451 +45452 POINT(41.179926432418235 73.24312166931293) bank45452 +45453 POINT(40.10840471808981 74.80878407128152) bank45453 +45454 POINT(39.97451278696092 73.46029283149231) bank45454 +45455 POINT(40.51371750267786 74.31278237612729) bank45455 +45456 POINT(40.28861371079621 74.71768747908851) bank45456 +45457 POINT(41.57489471514584 74.9885387938638) bank45457 +45458 POINT(40.707732895284146 74.3089194901947) bank45458 +45459 POINT(41.257035917135234 73.68491672097804) bank45459 +45460 POINT(40.80457073178428 73.39212420807257) bank45460 +45461 POINT(41.02298431691407 74.7909976232823) bank45461 +45462 POINT(40.969025168706516 74.24495281782313) bank45462 +45463 POINT(41.07436198718043 74.84047584287524) bank45463 +45464 POINT(39.78215065284957 74.07008245916073) bank45464 +45465 POINT(41.339753623590326 73.86681539102487) bank45465 +45466 POINT(41.19969567730975 73.81773127583402) bank45466 +45467 POINT(40.3318784207529 73.91428567526405) bank45467 +45468 POINT(40.20197539984027 74.2929284033732) bank45468 +45469 POINT(40.98483371809291 74.93758362187774) bank45469 +45470 POINT(40.85773503665808 73.31184440804856) bank45470 +45471 POINT(40.19596383214416 73.42821908920385) bank45471 +45472 POINT(41.57567121506059 73.46223249508456) bank45472 +45473 POINT(41.2692751798763 73.96763972839994) bank45473 +45474 POINT(40.45354580743572 73.43488137025194) bank45474 +45475 POINT(40.64366736767209 73.2072629410285) bank45475 +45476 POINT(40.742203895604774 73.18277556626296) bank45476 +45477 POINT(41.374290753262954 73.29068659774154) bank45477 +45478 POINT(40.2168049108559 74.82824962117031) bank45478 +45479 POINT(40.39186948716862 73.3642868331857) bank45479 +45480 POINT(39.909358129698994 73.03876249103901) bank45480 +45481 POINT(41.41644113672165 73.30004059774276) bank45481 +45482 POINT(41.01128773701536 74.41333272719251) bank45482 +45483 POINT(41.48343728414218 74.89387769674919) bank45483 +45484 POINT(41.09181706335188 73.04055261028469) bank45484 +45485 POINT(40.18027404996649 74.3398869536492) bank45485 +45486 POINT(40.33213418099994 74.95795704757886) bank45486 +45487 POINT(41.500679889081994 74.65751158182519) bank45487 +45488 POINT(40.72993985594791 73.79772783096278) bank45488 +45489 POINT(40.22010113594715 73.89078509409947) bank45489 +45490 POINT(39.74403403465369 73.29372871038753) bank45490 +45491 POINT(41.659568483636015 73.27154359469898) bank45491 +45492 POINT(40.11662926581704 73.41373759737675) bank45492 +45493 POINT(41.38889999874505 73.83901785094271) bank45493 +45494 POINT(41.234024825481626 74.61911980062708) bank45494 +45495 POINT(40.01479161749944 73.60024404884963) bank45495 +45496 POINT(41.67966900056838 74.9665233761361) bank45496 +45497 POINT(40.54379731474763 73.07056541787351) bank45497 +45498 POINT(40.43003954330265 73.69176289316086) bank45498 +45499 POINT(40.14608130896008 73.75560214166818) bank45499 +45500 POINT(40.676556082962534 73.08598951954977) bank45500 +45501 POINT(40.7444090384977 74.0600153751653) bank45501 +45502 POINT(40.97349707883475 74.75181686028375) bank45502 +45503 POINT(39.81740510889959 73.65019552972593) bank45503 +45504 POINT(40.66537335070991 74.96525923240848) bank45504 +45505 POINT(41.29270611510518 74.7592930488875) bank45505 +45506 POINT(41.27948775957936 74.95261275083762) bank45506 +45507 POINT(41.36339553112272 73.02979907899521) bank45507 +45508 POINT(40.74829185527005 74.79001946735005) bank45508 +45509 POINT(40.28201149180991 73.72608976035119) bank45509 +45510 POINT(41.392625423244205 74.19754801285367) bank45510 +45511 POINT(41.12430315598512 73.99140676297013) bank45511 +45512 POINT(41.187866929996446 73.69460714870233) bank45512 +45513 POINT(40.170862440284196 74.52431769966711) bank45513 +45514 POINT(40.50504581894412 73.83607541098675) bank45514 +45515 POINT(39.71412132122045 74.36841966944573) bank45515 +45516 POINT(41.56484392133892 74.57043534559737) bank45516 +45517 POINT(40.86008742047609 74.63499846960423) bank45517 +45518 POINT(40.71679494970161 73.33701274543643) bank45518 +45519 POINT(40.16103119676332 73.55491693542236) bank45519 +45520 POINT(40.13435975564002 73.20121343313141) bank45520 +45521 POINT(39.836071465978975 74.78473673025152) bank45521 +45522 POINT(40.218095767018234 74.07516256016905) bank45522 +45523 POINT(41.101462805400054 75.00216213095437) bank45523 +45524 POINT(41.65505114321905 74.4528007783673) bank45524 +45525 POINT(41.448353142150374 73.06635579733417) bank45525 +45526 POINT(39.858920838399506 74.64274455252567) bank45526 +45527 POINT(40.49711769160953 73.37726078704083) bank45527 +45528 POINT(41.18165608970198 74.64019069305691) bank45528 +45529 POINT(41.45069448154918 73.72773818176235) bank45529 +45530 POINT(40.54774941595278 74.38881921477461) bank45530 +45531 POINT(39.79189054961467 73.13000084624272) bank45531 +45532 POINT(41.00318659710244 73.17158818234026) bank45532 +45533 POINT(40.46680242026229 73.18364018940724) bank45533 +45534 POINT(40.47711797776619 74.03465295772999) bank45534 +45535 POINT(41.34360261920677 74.094503891052) bank45535 +45536 POINT(41.09416505710329 74.56244596767272) bank45536 +45537 POINT(40.33340505304134 74.78668015885916) bank45537 +45538 POINT(41.553526751658545 73.16609091336099) bank45538 +45539 POINT(40.44549269348435 73.78782901885211) bank45539 +45540 POINT(41.46366112143327 74.84204202311629) bank45540 +45541 POINT(41.37176447033583 74.53677628359387) bank45541 +45542 POINT(41.152762247360734 74.48796258059573) bank45542 +45543 POINT(41.20809735888953 74.36522346789856) bank45543 +45544 POINT(40.63955764974252 74.56903534933657) bank45544 +45545 POINT(41.549222649820145 74.61726766090744) bank45545 +45546 POINT(40.438716524047756 73.49935147445558) bank45546 +45547 POINT(40.703954860163535 74.99619606042145) bank45547 +45548 POINT(41.711857956826975 74.96349393471878) bank45548 +45549 POINT(41.24846158067059 73.65814745545835) bank45549 +45550 POINT(41.46554138813824 73.90890219135349) bank45550 +45551 POINT(41.70632382931981 74.76987483511478) bank45551 +45552 POINT(40.65959618441154 73.49145557853119) bank45552 +45553 POINT(39.88678436906746 74.51466730135638) bank45553 +45554 POINT(40.2915398937589 73.65260585814676) bank45554 +45555 POINT(40.3851937983635 74.57911582470796) bank45555 +45556 POINT(41.655936770234916 74.5461767671726) bank45556 +45557 POINT(39.94266013251146 73.48478821151902) bank45557 +45558 POINT(41.280930535632 74.75095371971038) bank45558 +45559 POINT(40.17941557834017 73.68710850049165) bank45559 +45560 POINT(40.72183700184618 73.03431175874995) bank45560 +45561 POINT(40.31054898901645 73.15228641367638) bank45561 +45562 POINT(41.06962223573571 74.53998879528127) bank45562 +45563 POINT(40.07904180439427 74.10014843067223) bank45563 +45564 POINT(41.465864540853275 73.92650117480451) bank45564 +45565 POINT(40.956733245812856 74.59045063372461) bank45565 +45566 POINT(39.73890262980987 73.8630701474532) bank45566 +45567 POINT(41.586785864321904 74.52822052301512) bank45567 +45568 POINT(41.17547658928006 73.1809863944281) bank45568 +45569 POINT(40.7452643984185 74.3580049279731) bank45569 +45570 POINT(41.56575645060616 74.32444864709134) bank45570 +45571 POINT(41.48271610516567 74.57491520416298) bank45571 +45572 POINT(41.202424965318116 73.15907306707747) bank45572 +45573 POINT(39.74557944187509 74.31052951384328) bank45573 +45574 POINT(41.187428269399334 74.88534391910191) bank45574 +45575 POINT(40.06639330144847 74.26861759064535) bank45575 +45576 POINT(40.778136304575476 73.88226825437796) bank45576 +45577 POINT(41.131478776747684 73.59092495821733) bank45577 +45578 POINT(40.09288809525726 74.82705317466912) bank45578 +45579 POINT(39.89877366906762 73.40487461684414) bank45579 +45580 POINT(41.21936178126889 73.88818745647666) bank45580 +45581 POINT(40.433083575811146 74.0503880550221) bank45581 +45582 POINT(41.05257281082793 73.72864495596696) bank45582 +45583 POINT(39.756182220826 74.19106398814304) bank45583 +45584 POINT(40.585862734703205 74.65193054277648) bank45584 +45585 POINT(41.30446344373921 74.67861117823087) bank45585 +45586 POINT(40.29192160785008 74.03600867837051) bank45586 +45587 POINT(40.18306152283297 74.09630115977019) bank45587 +45588 POINT(40.15385972813581 73.95837443105431) bank45588 +45589 POINT(40.37091530430678 74.15755021438626) bank45589 +45590 POINT(40.2640199163347 74.48593308482688) bank45590 +45591 POINT(40.31053475465588 74.04780454142515) bank45591 +45592 POINT(40.586677553919436 73.80562631624954) bank45592 +45593 POINT(41.35375905127813 73.31777619236213) bank45593 +45594 POINT(41.16510467209124 73.8203733142393) bank45594 +45595 POINT(40.59873993567641 73.88318285457262) bank45595 +45596 POINT(41.39776461356481 73.20351723235264) bank45596 +45597 POINT(40.92401862117063 74.90457670676187) bank45597 +45598 POINT(41.455770811594 73.56372959439443) bank45598 +45599 POINT(39.955597696439874 74.25210726069585) bank45599 +45600 POINT(40.4939134822989 73.82961816367994) bank45600 +45601 POINT(41.15255245042033 74.66259621555777) bank45601 +45602 POINT(40.56525288179543 74.91740613907572) bank45602 +45603 POINT(40.86233347142816 74.56723537372152) bank45603 +45604 POINT(40.16025224240933 74.90324990967311) bank45604 +45605 POINT(39.86443255722429 74.73864085248388) bank45605 +45606 POINT(40.82292266570379 74.7631149877631) bank45606 +45607 POINT(39.78938898459409 74.32755914033693) bank45607 +45608 POINT(40.42170937935952 73.78222152267641) bank45608 +45609 POINT(41.454244635561714 73.78043748419985) bank45609 +45610 POINT(40.846893514303176 73.01552158193589) bank45610 +45611 POINT(41.008751497707976 74.63026655596583) bank45611 +45612 POINT(41.14667062331502 73.62193476617722) bank45612 +45613 POINT(41.61962897746551 73.9567873962638) bank45613 +45614 POINT(40.0619490058021 73.936086863208) bank45614 +45615 POINT(40.81078484445028 74.34169289666268) bank45615 +45616 POINT(40.005998201853274 74.7367023861498) bank45616 +45617 POINT(41.19862040461927 74.55813307908821) bank45617 +45618 POINT(41.664581978318544 73.42290776341116) bank45618 +45619 POINT(40.78262368706016 73.61528785354298) bank45619 +45620 POINT(40.8702782619616 73.19485462602327) bank45620 +45621 POINT(39.75340683999664 73.38373974162089) bank45621 +45622 POINT(40.15637477006202 73.34527267445525) bank45622 +45623 POINT(40.85864314972776 74.06772342234758) bank45623 +45624 POINT(41.0977041573687 73.94271827598766) bank45624 +45625 POINT(40.74990223116696 74.01546930774492) bank45625 +45626 POINT(41.57116477574393 74.02842741213846) bank45626 +45627 POINT(40.31274919366208 74.84407964939993) bank45627 +45628 POINT(40.87050755641792 73.34889202705105) bank45628 +45629 POINT(41.22048995058038 73.91230735237917) bank45629 +45630 POINT(40.22863568249205 73.93001341176107) bank45630 +45631 POINT(39.82126481457583 74.09582695389342) bank45631 +45632 POINT(39.965773861593924 73.30786945452272) bank45632 +45633 POINT(40.91640693109288 73.51636810250099) bank45633 +45634 POINT(41.46426836228399 73.57629052482936) bank45634 +45635 POINT(40.52697200985231 73.57095636546742) bank45635 +45636 POINT(40.92273929951613 73.04865409558047) bank45636 +45637 POINT(40.81120665216034 73.7850454732795) bank45637 +45638 POINT(40.12793935443125 74.12706088973991) bank45638 +45639 POINT(40.49305857350144 73.05460175292248) bank45639 +45640 POINT(40.19104761441909 74.93782148643984) bank45640 +45641 POINT(40.12801369051827 73.76623885388425) bank45641 +45642 POINT(41.5193954648278 74.17817631227419) bank45642 +45643 POINT(40.05603212966738 74.22498812285666) bank45643 +45644 POINT(40.228971248816315 74.15426827558954) bank45644 +45645 POINT(41.073659405638544 74.87376946855987) bank45645 +45646 POINT(41.6092279025559 74.0404600227566) bank45646 +45647 POINT(40.0809140206321 73.35412334808966) bank45647 +45648 POINT(40.85846524063757 74.03171374757143) bank45648 +45649 POINT(40.418103280479706 74.44290922666076) bank45649 +45650 POINT(40.82076196273803 74.47715931688772) bank45650 +45651 POINT(41.568871469664806 73.68906103345232) bank45651 +45652 POINT(40.81072762393039 74.44949501976531) bank45652 +45653 POINT(40.43542130287314 73.01718013146015) bank45653 +45654 POINT(40.565886910392074 73.52384460068748) bank45654 +45655 POINT(41.57506932261739 74.76410976310152) bank45655 +45656 POINT(40.7329720352363 74.29822574258908) bank45656 +45657 POINT(41.07101019636418 74.52306733839143) bank45657 +45658 POINT(41.58089949330218 74.24669507132346) bank45658 +45659 POINT(40.845421482024236 73.98683741003914) bank45659 +45660 POINT(40.381239317783276 74.64163435821412) bank45660 +45661 POINT(40.40260070569143 74.03764475563685) bank45661 +45662 POINT(40.18634992422195 73.7544877414401) bank45662 +45663 POINT(40.077319319854794 74.66482522376681) bank45663 +45664 POINT(41.03455798084738 74.31135865867188) bank45664 +45665 POINT(41.68283164311542 74.19038860944423) bank45665 +45666 POINT(40.06506045997269 74.19105471917354) bank45666 +45667 POINT(40.437716495353214 73.67493453314964) bank45667 +45668 POINT(40.721027424703884 74.1725226389161) bank45668 +45669 POINT(41.25154419896441 73.55621764915823) bank45669 +45670 POINT(41.044277800367595 74.93049699314201) bank45670 +45671 POINT(40.05700645321637 73.32938586783693) bank45671 +45672 POINT(40.32629726822063 73.4561601311225) bank45672 +45673 POINT(41.56716427669771 74.85203830933611) bank45673 +45674 POINT(40.622801618878576 74.41837146728885) bank45674 +45675 POINT(41.10853572515182 73.46920912004921) bank45675 +45676 POINT(40.01833841397598 73.39421797210838) bank45676 +45677 POINT(41.41242617143164 73.64585108012652) bank45677 +45678 POINT(39.824573821896216 74.66974924668557) bank45678 +45679 POINT(41.38076519008684 74.24123947215577) bank45679 +45680 POINT(39.95423653553488 74.52468510999483) bank45680 +45681 POINT(40.41459079660115 74.12969961937979) bank45681 +45682 POINT(41.40991995426592 74.01087524082739) bank45682 +45683 POINT(39.9557259914453 74.31466506171637) bank45683 +45684 POINT(39.91629077270683 74.04708806026318) bank45684 +45685 POINT(40.07407802374713 74.88702608759249) bank45685 +45686 POINT(40.50725321585973 73.36779793743456) bank45686 +45687 POINT(40.90353624572798 73.92723931331972) bank45687 +45688 POINT(41.358063813074246 73.98339669748142) bank45688 +45689 POINT(39.78720073417459 73.07995975247161) bank45689 +45690 POINT(40.52181220843131 74.30628838237978) bank45690 +45691 POINT(40.59465537235366 74.72802977419467) bank45691 +45692 POINT(40.51674299708778 74.79959306952821) bank45692 +45693 POINT(41.422719588893784 74.08368049954383) bank45693 +45694 POINT(41.06576596153763 74.94187565726826) bank45694 +45695 POINT(41.35151247151467 73.4559666781151) bank45695 +45696 POINT(41.613561475782674 73.5886718196811) bank45696 +45697 POINT(40.437261413338994 74.13900518852047) bank45697 +45698 POINT(40.55204789586717 75.00078216718634) bank45698 +45699 POINT(40.89267137426185 73.35776641597887) bank45699 +45700 POINT(40.80689736727505 74.61393536240641) bank45700 +45701 POINT(40.27968626009483 74.14963605562015) bank45701 +45702 POINT(40.869895286861876 74.60173431441122) bank45702 +45703 POINT(39.74631691204279 74.33153876116667) bank45703 +45704 POINT(39.860468286018566 73.0876159213029) bank45704 +45705 POINT(40.39802512602115 74.27907903888104) bank45705 +45706 POINT(41.66695958275849 74.17973531442438) bank45706 +45707 POINT(41.35705496018877 73.59479219581473) bank45707 +45708 POINT(40.64114755088137 74.67564880449446) bank45708 +45709 POINT(40.7579227756361 74.09717886653004) bank45709 +45710 POINT(41.00735076254656 73.52406165917158) bank45710 +45711 POINT(41.256033143119794 73.54683259417293) bank45711 +45712 POINT(40.04097519423774 74.51622236209326) bank45712 +45713 POINT(40.46448258983497 73.87377986265258) bank45713 +45714 POINT(40.11903943603515 73.73549345345165) bank45714 +45715 POINT(40.62824045557934 73.41870747155527) bank45715 +45716 POINT(40.54847009399776 73.9283581913661) bank45716 +45717 POINT(41.36108615833562 73.5872579135921) bank45717 +45718 POINT(39.90948499755232 73.03892866184496) bank45718 +45719 POINT(41.28947995285381 73.74573545773215) bank45719 +45720 POINT(39.828635714866095 73.22976429405053) bank45720 +45721 POINT(40.54415620202324 74.14684059831055) bank45721 +45722 POINT(41.55709995353705 73.4838240293256) bank45722 +45723 POINT(40.08779147408506 73.526953996219) bank45723 +45724 POINT(39.7486305039645 73.73686714578668) bank45724 +45725 POINT(41.1891587387869 74.21777349183095) bank45725 +45726 POINT(40.49027332613379 73.2068175783666) bank45726 +45727 POINT(39.80169053261514 73.01766260161128) bank45727 +45728 POINT(41.59993629685459 74.47667410019709) bank45728 +45729 POINT(41.16524726202892 73.11580483514254) bank45729 +45730 POINT(40.93613381856248 74.42057324211669) bank45730 +45731 POINT(40.20621231244123 73.1997059109653) bank45731 +45732 POINT(40.34408734210304 74.02938107993022) bank45732 +45733 POINT(40.4090218135499 73.09001013479963) bank45733 +45734 POINT(41.449573518345396 73.6006635410651) bank45734 +45735 POINT(39.87817593907524 74.99879791613016) bank45735 +45736 POINT(40.150837515640035 73.25127224016512) bank45736 +45737 POINT(40.57786487643006 73.90366230610795) bank45737 +45738 POINT(41.174258879937625 73.28510012398236) bank45738 +45739 POINT(41.37341822185182 74.15937082164153) bank45739 +45740 POINT(41.264806225863914 73.22661362807088) bank45740 +45741 POINT(41.17314018906818 74.52996592616813) bank45741 +45742 POINT(41.33459553542399 73.8355544375986) bank45742 +45743 POINT(41.38392144813307 74.85920897439527) bank45743 +45744 POINT(39.90602529121655 73.77503676165003) bank45744 +45745 POINT(40.64411839872236 73.14413782031) bank45745 +45746 POINT(39.852058583117255 73.94972305962628) bank45746 +45747 POINT(41.26724135889951 73.84369827935257) bank45747 +45748 POINT(41.542840778195654 74.31291987577144) bank45748 +45749 POINT(40.287193011995456 73.35418841882766) bank45749 +45750 POINT(41.294089256508606 73.94614690888974) bank45750 +45751 POINT(40.660604432741735 73.95896433345266) bank45751 +45752 POINT(39.852796449584865 73.60269904947315) bank45752 +45753 POINT(41.13059712375044 73.24717426466873) bank45753 +45754 POINT(41.05841333120425 74.57092973625491) bank45754 +45755 POINT(40.47222611776803 73.6158748661675) bank45755 +45756 POINT(40.28995790848604 74.44058113460898) bank45756 +45757 POINT(40.489457992236105 73.79855381631236) bank45757 +45758 POINT(41.33204700092382 74.21001727941585) bank45758 +45759 POINT(41.48614887921727 74.57838341754261) bank45759 +45760 POINT(41.211638143307546 74.17056537485799) bank45760 +45761 POINT(40.00736062201404 74.62186733816911) bank45761 +45762 POINT(41.54554068489276 73.98706669314778) bank45762 +45763 POINT(41.24901847042675 73.38737577931454) bank45763 +45764 POINT(40.99458214277526 74.3903985898497) bank45764 +45765 POINT(40.60797641992363 74.0636950485302) bank45765 +45766 POINT(40.204960102446286 73.75333227611932) bank45766 +45767 POINT(40.19622032432056 74.09469408470255) bank45767 +45768 POINT(41.67077418879068 73.09234644709528) bank45768 +45769 POINT(40.298124839394994 74.49914242535019) bank45769 +45770 POINT(41.006667415439 73.84568434146723) bank45770 +45771 POINT(39.739368496678246 73.49102595249632) bank45771 +45772 POINT(41.155179872378866 73.74282690442135) bank45772 +45773 POINT(40.45778392292752 73.20229164251734) bank45773 +45774 POINT(39.954303873707225 73.37282471365285) bank45774 +45775 POINT(39.713581835055244 74.83394716863019) bank45775 +45776 POINT(41.43260236257585 74.85194365244682) bank45776 +45777 POINT(41.226555055977606 74.93601373120607) bank45777 +45778 POINT(40.74701284330194 73.79681358792817) bank45778 +45779 POINT(40.651119084683565 74.771235795066) bank45779 +45780 POINT(40.72225450394169 73.05590541889597) bank45780 +45781 POINT(41.32798438920397 73.69263772824556) bank45781 +45782 POINT(41.01698825755327 73.72756965462433) bank45782 +45783 POINT(40.026332603196124 73.36027893143294) bank45783 +45784 POINT(41.30518489882977 74.83158362324944) bank45784 +45785 POINT(40.04452562192298 73.3838429662022) bank45785 +45786 POINT(40.91038091709935 74.43547308773336) bank45786 +45787 POINT(41.46020061488608 74.65883883312954) bank45787 +45788 POINT(40.18133990269053 73.51908124853973) bank45788 +45789 POINT(41.68928552003814 74.4712048224758) bank45789 +45790 POINT(41.044449912845224 74.99233366851153) bank45790 +45791 POINT(39.78516883713188 73.0773172959796) bank45791 +45792 POINT(40.804796230393 74.24751715571301) bank45792 +45793 POINT(40.02005770426089 74.08658682259917) bank45793 +45794 POINT(40.778682567462425 73.23570859444342) bank45794 +45795 POINT(40.54711056229966 73.18044573583732) bank45795 +45796 POINT(40.69151132142755 73.59502471730735) bank45796 +45797 POINT(41.3530711829065 73.7515807572212) bank45797 +45798 POINT(40.44459059749063 73.76928384639982) bank45798 +45799 POINT(40.20081259222892 74.77807174974146) bank45799 +45800 POINT(41.36256549923495 74.8488816427173) bank45800 +45801 POINT(40.25372602236536 74.4915967882089) bank45801 +45802 POINT(40.546833050688996 73.54624646154168) bank45802 +45803 POINT(41.086957604781716 73.37354136471168) bank45803 +45804 POINT(39.95425769689788 73.87311062417422) bank45804 +45805 POINT(40.96037768335828 73.79343187140115) bank45805 +45806 POINT(40.13828154606692 73.249199062934) bank45806 +45807 POINT(40.30821490137899 74.43894936026707) bank45807 +45808 POINT(41.22440571902596 73.29775287496999) bank45808 +45809 POINT(41.6704982269005 74.33985025725252) bank45809 +45810 POINT(41.708643941112285 73.31457305290107) bank45810 +45811 POINT(39.98158455626178 74.16924145435506) bank45811 +45812 POINT(41.17340377023437 74.42214460049277) bank45812 +45813 POINT(40.173235983188576 73.06479813746381) bank45813 +45814 POINT(41.03309395689359 73.18236429748335) bank45814 +45815 POINT(40.313957920311466 73.80403982144144) bank45815 +45816 POINT(40.42556625622104 74.18286118678101) bank45816 +45817 POINT(40.03464001852512 74.04930997726534) bank45817 +45818 POINT(39.79591534499017 74.29371663830837) bank45818 +45819 POINT(40.08858943925249 73.46514103374801) bank45819 +45820 POINT(40.161537990868595 74.34909586204417) bank45820 +45821 POINT(39.800565842117244 73.9840251196375) bank45821 +45822 POINT(39.936510242964076 73.29964736486164) bank45822 +45823 POINT(39.78977013619471 73.85912747096401) bank45823 +45824 POINT(41.20592552190351 73.28194391412097) bank45824 +45825 POINT(40.64432025277758 73.33383470793827) bank45825 +45826 POINT(40.630310805120224 74.28773835846089) bank45826 +45827 POINT(41.14733820394034 73.42793864760344) bank45827 +45828 POINT(39.97022638338175 74.56190990473092) bank45828 +45829 POINT(41.10490910015274 73.84544944016723) bank45829 +45830 POINT(40.33292997371788 74.54345760435292) bank45830 +45831 POINT(40.860074335078984 73.5821360646719) bank45831 +45832 POINT(40.49467276083813 73.87200503942563) bank45832 +45833 POINT(41.220854745726015 74.72056122020203) bank45833 +45834 POINT(40.43188890187159 74.19538875168907) bank45834 +45835 POINT(40.68261532051798 74.49051901553354) bank45835 +45836 POINT(41.28211441879257 73.77865319408585) bank45836 +45837 POINT(40.38800554058181 73.49250502277832) bank45837 +45838 POINT(40.11114538865226 73.87460132592274) bank45838 +45839 POINT(41.60634602387037 73.99848919454978) bank45839 +45840 POINT(40.990457453002755 74.66012795980893) bank45840 +45841 POINT(39.78942523010363 74.70564183376487) bank45841 +45842 POINT(40.80461344465667 73.95740197835852) bank45842 +45843 POINT(40.349359403405614 73.14029297448702) bank45843 +45844 POINT(40.39714062093426 74.25183505054211) bank45844 +45845 POINT(41.41604072175735 73.61956601044139) bank45845 +45846 POINT(40.62750296121109 74.19399305599114) bank45846 +45847 POINT(41.0903230031045 73.46300120089491) bank45847 +45848 POINT(41.410963656176406 73.88229678114573) bank45848 +45849 POINT(39.85008099703599 73.77259149778246) bank45849 +45850 POINT(40.78546290840206 74.5287018267862) bank45850 +45851 POINT(40.49182045546793 74.55360057854006) bank45851 +45852 POINT(41.51307472338886 74.952916164931) bank45852 +45853 POINT(41.62981380733725 74.76834504221137) bank45853 +45854 POINT(40.043335624659264 73.5142437423289) bank45854 +45855 POINT(39.80128403795771 74.54954471630623) bank45855 +45856 POINT(40.49266069306295 73.59141459331097) bank45856 +45857 POINT(39.815937581736925 74.5220711065386) bank45857 +45858 POINT(40.07328466084698 74.90293024429991) bank45858 +45859 POINT(39.738912964374755 74.35910728266184) bank45859 +45860 POINT(40.0818940024799 74.38249631422669) bank45860 +45861 POINT(41.43905696266181 74.36089555506537) bank45861 +45862 POINT(40.89142528337315 73.4854725146233) bank45862 +45863 POINT(41.045911095048695 73.23124912060733) bank45863 +45864 POINT(40.99204502541591 74.24992200470307) bank45864 +45865 POINT(39.9341944498941 73.18870106981925) bank45865 +45866 POINT(41.6491747963822 74.60375650576205) bank45866 +45867 POINT(41.08780105813023 74.70722361495756) bank45867 +45868 POINT(39.821422926006726 74.64828335529891) bank45868 +45869 POINT(41.2583059331567 73.19548990919289) bank45869 +45870 POINT(39.80512547482638 74.7072219568708) bank45870 +45871 POINT(41.262367652416906 74.6883325666683) bank45871 +45872 POINT(40.879245021673086 74.36446875931873) bank45872 +45873 POINT(39.86873855995232 74.9293844337309) bank45873 +45874 POINT(39.910231194362815 74.02362416135055) bank45874 +45875 POINT(39.77445421117743 73.88651140932284) bank45875 +45876 POINT(40.2670488783677 73.7302182870287) bank45876 +45877 POINT(39.729382952341155 74.84423250208707) bank45877 +45878 POINT(41.6539289705161 74.57782224643653) bank45878 +45879 POINT(40.044777241408696 74.55769777841417) bank45879 +45880 POINT(40.755762201915275 74.07172634177122) bank45880 +45881 POINT(40.008238572607674 74.38017619969116) bank45881 +45882 POINT(39.92815866256909 74.80108128642514) bank45882 +45883 POINT(39.93029052283122 74.72256467665615) bank45883 +45884 POINT(41.2737667203382 74.64667533856874) bank45884 +45885 POINT(41.64732602615547 73.08542930005456) bank45885 +45886 POINT(39.73227167399721 74.21039135163954) bank45886 +45887 POINT(40.228449742365676 74.84492293164897) bank45887 +45888 POINT(40.38075829107943 73.58507267542711) bank45888 +45889 POINT(40.196692660945004 73.1656676815609) bank45889 +45890 POINT(40.19871215673458 74.639812147768) bank45890 +45891 POINT(40.91730288445153 74.87600584962073) bank45891 +45892 POINT(41.69213875317554 74.88720983034126) bank45892 +45893 POINT(40.29015906535829 73.5886658260039) bank45893 +45894 POINT(40.82891943670135 73.26321882006981) bank45894 +45895 POINT(41.36615310163783 73.45444028212545) bank45895 +45896 POINT(41.508886665976355 74.72781875663888) bank45896 +45897 POINT(41.41847195220156 73.96397933434488) bank45897 +45898 POINT(40.629520260558195 73.26855914470188) bank45898 +45899 POINT(40.34516365447612 73.23736997202988) bank45899 +45900 POINT(40.86433894510377 74.39942860003089) bank45900 +45901 POINT(41.113864870764644 73.461842767481) bank45901 +45902 POINT(40.210067534763155 74.33405960297813) bank45902 +45903 POINT(41.28906124754276 74.57255968169254) bank45903 +45904 POINT(41.28180581082141 73.7239500422981) bank45904 +45905 POINT(40.66763269311823 74.37686508819172) bank45905 +45906 POINT(39.73841794613632 73.5181742654407) bank45906 +45907 POINT(40.95805679158605 73.05479924291322) bank45907 +45908 POINT(41.17706210659137 73.7920621309089) bank45908 +45909 POINT(40.102671549489216 73.0753829366988) bank45909 +45910 POINT(40.606889927551364 73.89050475460529) bank45910 +45911 POINT(40.50543131826836 73.88604063759557) bank45911 +45912 POINT(41.577595950813546 73.79560021516218) bank45912 +45913 POINT(41.154576864828655 73.59697926516176) bank45913 +45914 POINT(40.79457575425949 73.96918438250641) bank45914 +45915 POINT(40.9781952474851 74.06003756251702) bank45915 +45916 POINT(41.02634383494041 74.50773251424043) bank45916 +45917 POINT(41.48689770010684 73.7706047563383) bank45917 +45918 POINT(41.16515819707003 73.52973333054179) bank45918 +45919 POINT(40.68853850198859 73.47502523111656) bank45919 +45920 POINT(40.462943285664394 74.3868823261816) bank45920 +45921 POINT(41.35823436636808 74.94145201584465) bank45921 +45922 POINT(41.38065729362551 74.50954743639042) bank45922 +45923 POINT(40.662721583371585 74.4353313219766) bank45923 +45924 POINT(40.10725397494746 74.59404596738682) bank45924 +45925 POINT(41.335858317302055 74.02219682116055) bank45925 +45926 POINT(41.29328076439047 73.69129203299867) bank45926 +45927 POINT(41.099325043112344 74.02323518075359) bank45927 +45928 POINT(41.61692261024401 73.40416265350773) bank45928 +45929 POINT(40.19414508164869 74.18349666163651) bank45929 +45930 POINT(41.414414065653745 73.72062235687605) bank45930 +45931 POINT(40.0736737152907 73.1745269940843) bank45931 +45932 POINT(41.137471557966876 73.24850539804747) bank45932 +45933 POINT(41.219100923285424 74.78610002503073) bank45933 +45934 POINT(40.841853634173425 74.60746248030577) bank45934 +45935 POINT(40.85120182511595 73.43105643174925) bank45935 +45936 POINT(41.48845819494775 74.72072629009357) bank45936 +45937 POINT(40.04667943825578 74.80754485758553) bank45937 +45938 POINT(41.06075103311847 74.27143202431053) bank45938 +45939 POINT(40.6275564629603 74.45441656058158) bank45939 +45940 POINT(40.69607189749474 74.62559300489934) bank45940 +45941 POINT(40.92986694323005 73.72489448025344) bank45941 +45942 POINT(39.794271208953866 73.42674053988425) bank45942 +45943 POINT(39.94782441925555 73.52786461697488) bank45943 +45944 POINT(40.65622668884681 74.72085964274447) bank45944 +45945 POINT(39.74406969627782 73.1961306841631) bank45945 +45946 POINT(40.70752177964828 73.14653940069871) bank45946 +45947 POINT(39.89350375709603 73.10687729747657) bank45947 +45948 POINT(40.81034524185954 73.49825909602033) bank45948 +45949 POINT(41.158824647415415 73.71520388605506) bank45949 +45950 POINT(40.49055553871327 73.78709820928954) bank45950 +45951 POINT(41.00172750470843 73.36115572105261) bank45951 +45952 POINT(40.521450977021836 74.93794905691747) bank45952 +45953 POINT(41.01493643635565 74.51373343443062) bank45953 +45954 POINT(41.446504175070785 74.34262906007042) bank45954 +45955 POINT(41.27491497589888 74.21315304965673) bank45955 +45956 POINT(39.8304560462675 73.86698955829146) bank45956 +45957 POINT(41.13700068090204 74.17529031218203) bank45957 +45958 POINT(41.21506662130618 73.37992375242287) bank45958 +45959 POINT(40.964005123216914 74.54857338722101) bank45959 +45960 POINT(40.16663362613879 73.87171117128436) bank45960 +45961 POINT(40.40130498189103 73.90448855203813) bank45961 +45962 POINT(40.75160248756558 74.97274256072636) bank45962 +45963 POINT(41.405363680084065 73.79496362319554) bank45963 +45964 POINT(41.317435666469734 73.90710079296316) bank45964 +45965 POINT(41.38741550299198 74.77230017835211) bank45965 +45966 POINT(40.10742314669222 74.98381554106551) bank45966 +45967 POINT(39.88401888958361 74.84592530389) bank45967 +45968 POINT(40.87284840761567 73.85699059966987) bank45968 +45969 POINT(40.64239225180641 74.40272786549872) bank45969 +45970 POINT(41.69743930327789 75.00281941677007) bank45970 +45971 POINT(41.549004865362456 73.91395944791502) bank45971 +45972 POINT(41.51077704570301 73.47420695998682) bank45972 +45973 POINT(40.413905480658265 73.59824547006613) bank45973 +45974 POINT(40.479033534800706 74.19145727094825) bank45974 +45975 POINT(41.351081336550756 73.21396536519137) bank45975 +45976 POINT(40.969463799740794 74.76387371793449) bank45976 +45977 POINT(40.55437966960224 74.8176896814793) bank45977 +45978 POINT(40.13777300458938 73.77654259252951) bank45978 +45979 POINT(39.88999347488612 73.58554065208861) bank45979 +45980 POINT(40.03898833040447 73.52560365563193) bank45980 +45981 POINT(40.364883575412 74.94405172111743) bank45981 +45982 POINT(39.72491733131891 73.24312672401938) bank45982 +45983 POINT(41.180235054127785 73.09962086643876) bank45983 +45984 POINT(40.770892521248825 74.6980642363299) bank45984 +45985 POINT(41.696856018446915 73.8845633932056) bank45985 +45986 POINT(40.87495379993113 73.26914896601376) bank45986 +45987 POINT(39.74419733777727 74.94907648579303) bank45987 +45988 POINT(41.653483157119965 74.95324149179332) bank45988 +45989 POINT(40.584259570091774 73.18665725752801) bank45989 +45990 POINT(39.790469836121055 74.19480216423759) bank45990 +45991 POINT(40.085911480407795 73.24381796645467) bank45991 +45992 POINT(40.15882225257176 74.79427005314287) bank45992 +45993 POINT(41.1796052241004 74.39326116693609) bank45993 +45994 POINT(41.3784038309978 73.09508067999462) bank45994 +45995 POINT(39.88282852031552 73.51678694443773) bank45995 +45996 POINT(41.50515033651659 73.60592005572829) bank45996 +45997 POINT(41.67574859308854 73.04991051436265) bank45997 +45998 POINT(40.493737895286486 74.0408540358365) bank45998 +45999 POINT(40.047322494473036 73.2781105656816) bank45999 +46000 POINT(39.87514883549723 74.15655282353823) bank46000 +46001 POINT(40.91506979317461 74.5846267682339) bank46001 +46002 POINT(40.469344667282456 74.52543096175289) bank46002 +46003 POINT(40.807123139013974 73.46613313639963) bank46003 +46004 POINT(39.75160064463959 74.99372745420627) bank46004 +46005 POINT(40.48079604456192 74.55359531443729) bank46005 +46006 POINT(40.18665178473288 73.82637140109324) bank46006 +46007 POINT(39.986174807235635 73.46347124015685) bank46007 +46008 POINT(41.558381419861185 74.58885820548912) bank46008 +46009 POINT(40.514152713860284 74.20337166072457) bank46009 +46010 POINT(41.618280197519184 73.45107829754818) bank46010 +46011 POINT(40.65520508157739 74.79096438013602) bank46011 +46012 POINT(40.68186259085877 74.41698266559382) bank46012 +46013 POINT(40.174545449569585 74.06001863777789) bank46013 +46014 POINT(40.224150533405904 73.16375968983809) bank46014 +46015 POINT(40.596531565746375 74.67578960064623) bank46015 +46016 POINT(40.79638064425241 74.5145211186052) bank46016 +46017 POINT(39.90605707192587 73.22887380837821) bank46017 +46018 POINT(40.70320134459313 73.52497810209107) bank46018 +46019 POINT(41.27365943343896 74.04751286291113) bank46019 +46020 POINT(40.91005366245235 73.10666489365823) bank46020 +46021 POINT(41.59439458957671 73.57432636357905) bank46021 +46022 POINT(40.68095652231894 73.6488647148794) bank46022 +46023 POINT(40.909101332762035 74.34453468482408) bank46023 +46024 POINT(40.063784567843136 74.40081463098807) bank46024 +46025 POINT(40.169096240183066 74.9228028970872) bank46025 +46026 POINT(40.01232442275386 74.21204471927403) bank46026 +46027 POINT(39.78888065464435 74.88863367614093) bank46027 +46028 POINT(40.19907499151369 73.98948491087643) bank46028 +46029 POINT(41.525914052425385 74.42576955617008) bank46029 +46030 POINT(41.0954474989884 73.55051703689689) bank46030 +46031 POINT(41.19134679518833 74.90786111572086) bank46031 +46032 POINT(41.53202245718673 73.48528988312509) bank46032 +46033 POINT(41.69173182739409 73.28009301432965) bank46033 +46034 POINT(40.58860699758042 73.12774469315787) bank46034 +46035 POINT(41.23510950371093 74.60276521896687) bank46035 +46036 POINT(40.54824362418471 73.11579580783494) bank46036 +46037 POINT(40.857912601768895 74.21646610562064) bank46037 +46038 POINT(40.43473575162405 74.53817204471305) bank46038 +46039 POINT(40.207003545094715 73.50937148202063) bank46039 +46040 POINT(41.514825162002296 74.77104126922447) bank46040 +46041 POINT(40.25673560707825 74.08923010413041) bank46041 +46042 POINT(40.520829635236936 73.17776449344751) bank46042 +46043 POINT(39.923876629285054 73.62631662586863) bank46043 +46044 POINT(40.00020341950524 74.35906904937661) bank46044 +46045 POINT(41.61594831088904 74.50407823113466) bank46045 +46046 POINT(39.75462011068745 73.65257971639338) bank46046 +46047 POINT(41.33755093676109 73.70645769777221) bank46047 +46048 POINT(40.80719512571927 73.26203583857539) bank46048 +46049 POINT(40.19853657712473 74.49452145845434) bank46049 +46050 POINT(41.46835644638186 73.58573683941432) bank46050 +46051 POINT(41.643333753707616 74.1570522593026) bank46051 +46052 POINT(40.21212869895352 74.79734712054696) bank46052 +46053 POINT(40.12088909745762 73.49693507373229) bank46053 +46054 POINT(40.74518414465847 74.86585590261211) bank46054 +46055 POINT(40.52338642119711 74.51257350254862) bank46055 +46056 POINT(41.49899492560111 74.16998306871601) bank46056 +46057 POINT(41.016691859822025 73.70925989892143) bank46057 +46058 POINT(40.71233805939252 74.35421115093168) bank46058 +46059 POINT(41.26322971731526 74.82157839226262) bank46059 +46060 POINT(41.67328327349425 74.0657245181079) bank46060 +46061 POINT(40.46158911975151 74.60156564797204) bank46061 +46062 POINT(40.984606278292965 73.49825025165953) bank46062 +46063 POINT(40.08151429431204 73.08961803390913) bank46063 +46064 POINT(41.445375327488506 74.364964812676) bank46064 +46065 POINT(41.479727042365255 74.65134045212241) bank46065 +46066 POINT(39.94671865957699 74.07553019945021) bank46066 +46067 POINT(41.41203909091481 73.48089598350562) bank46067 +46068 POINT(41.69276328877341 75.00396425584503) bank46068 +46069 POINT(41.07207464938942 74.44321486136671) bank46069 +46070 POINT(41.37214792188978 73.9199733881827) bank46070 +46071 POINT(41.420590893210864 74.77116400170476) bank46071 +46072 POINT(40.76687169111244 74.90416990328181) bank46072 +46073 POINT(40.22892430669544 74.67564204536717) bank46073 +46074 POINT(39.99536627012986 73.16987960045225) bank46074 +46075 POINT(40.790084307793386 73.48544837803264) bank46075 +46076 POINT(40.37643965707865 74.7135494963717) bank46076 +46077 POINT(40.79820407473481 73.76707110273527) bank46077 +46078 POINT(41.21107116691603 73.53257533382363) bank46078 +46079 POINT(39.9771123634207 73.7615136814075) bank46079 +46080 POINT(40.78418154934 74.59640271881304) bank46080 +46081 POINT(40.996047344151734 74.33391546633692) bank46081 +46082 POINT(40.059995355805135 73.33929976017112) bank46082 +46083 POINT(40.33112556178689 74.7325480354608) bank46083 +46084 POINT(39.7239920199273 73.79972719075782) bank46084 +46085 POINT(39.832378834564665 73.2500433778644) bank46085 +46086 POINT(39.97674574308567 74.30311250089385) bank46086 +46087 POINT(40.12302192442757 74.9533349061252) bank46087 +46088 POINT(41.164694677650154 74.66406816201572) bank46088 +46089 POINT(41.151635904498065 73.20554677858728) bank46089 +46090 POINT(41.176767709753065 74.08518271250077) bank46090 +46091 POINT(39.80306672279163 73.5093946157654) bank46091 +46092 POINT(39.714948594695855 74.47719812116263) bank46092 +46093 POINT(40.68173541452807 73.16323874084149) bank46093 +46094 POINT(41.49018786804211 74.1023816172184) bank46094 +46095 POINT(41.268191834578545 74.06904005187974) bank46095 +46096 POINT(41.1024068362891 73.7037350664429) bank46096 +46097 POINT(40.47635156481271 74.04651390571105) bank46097 +46098 POINT(40.315561690909824 74.97335932723047) bank46098 +46099 POINT(41.62482424651833 73.87354622930485) bank46099 +46100 POINT(41.476947890986416 74.0853294152058) bank46100 +46101 POINT(40.14245589454271 73.63138412618046) bank46101 +46102 POINT(40.24571537435172 73.28400406173863) bank46102 +46103 POINT(40.60423971524557 74.0908651564437) bank46103 +46104 POINT(40.146158461099034 73.88306371790843) bank46104 +46105 POINT(40.43141950570702 74.3222248993074) bank46105 +46106 POINT(39.79860542121319 74.5020853981634) bank46106 +46107 POINT(40.57745437574824 73.16370628461151) bank46107 +46108 POINT(41.0401344767107 74.4528858063793) bank46108 +46109 POINT(40.337403179344946 74.14971985909753) bank46109 +46110 POINT(40.348708578686654 73.2058224123649) bank46110 +46111 POINT(39.74909433969917 74.41799715797448) bank46111 +46112 POINT(41.50971090216078 74.90354055093498) bank46112 +46113 POINT(40.97005449504619 74.91322136641385) bank46113 +46114 POINT(40.72570656488635 74.2019075917241) bank46114 +46115 POINT(40.35732331764421 73.76861863565466) bank46115 +46116 POINT(41.375196031501424 74.16338615404152) bank46116 +46117 POINT(40.851783731837266 73.20617725390022) bank46117 +46118 POINT(40.025289638034 74.07426726702138) bank46118 +46119 POINT(41.041692760323 74.76939850196894) bank46119 +46120 POINT(40.897373509186345 73.27500705601723) bank46120 +46121 POINT(41.04065567259744 74.13470464546698) bank46121 +46122 POINT(39.80005573339487 74.76722072829432) bank46122 +46123 POINT(40.022197579993374 73.8703548095687) bank46123 +46124 POINT(41.58847884089674 73.76331826383121) bank46124 +46125 POINT(40.09156662425639 74.60515463265155) bank46125 +46126 POINT(41.564691569747914 73.60732642860305) bank46126 +46127 POINT(41.11474949640824 73.49853408589271) bank46127 +46128 POINT(40.364721891740814 73.75796121727852) bank46128 +46129 POINT(41.0480808684306 74.46471699039378) bank46129 +46130 POINT(40.85649574723601 74.95269052186485) bank46130 +46131 POINT(41.659817298007454 74.67420734850086) bank46131 +46132 POINT(41.42683808475085 74.34448907615533) bank46132 +46133 POINT(40.82404293542166 74.35927846092548) bank46133 +46134 POINT(40.544526270854504 73.181359856572) bank46134 +46135 POINT(40.518974666070015 73.07327228552373) bank46135 +46136 POINT(41.31567601597457 73.21472820382215) bank46136 +46137 POINT(41.399247210437395 73.99788140575954) bank46137 +46138 POINT(41.508540046886246 73.2622638333241) bank46138 +46139 POINT(41.15203970831169 73.76578261606696) bank46139 +46140 POINT(41.38764135462606 74.78509134998731) bank46140 +46141 POINT(40.64066653920255 74.65573287208879) bank46141 +46142 POINT(40.88142138900436 74.53266457641195) bank46142 +46143 POINT(39.91246898130146 73.79120689222091) bank46143 +46144 POINT(41.35178601015749 73.73453334589448) bank46144 +46145 POINT(41.63140489649389 73.03534095051893) bank46145 +46146 POINT(40.62635599442084 74.6398963282811) bank46146 +46147 POINT(41.42046221763618 73.40229104300575) bank46147 +46148 POINT(41.090497210412096 74.83706490162655) bank46148 +46149 POINT(39.717121244324304 74.66655628615608) bank46149 +46150 POINT(40.2791080938032 73.43985511436546) bank46150 +46151 POINT(39.96351807216836 73.88079845092993) bank46151 +46152 POINT(40.44753380638456 73.51552909282233) bank46152 +46153 POINT(40.989438502671476 74.60913338126942) bank46153 +46154 POINT(41.53163902996108 74.78891765506184) bank46154 +46155 POINT(40.066560191142294 73.66073979764047) bank46155 +46156 POINT(41.280439136243665 73.70733129325707) bank46156 +46157 POINT(41.22914773670849 73.10464121827737) bank46157 +46158 POINT(41.35575365871016 73.72469976492208) bank46158 +46159 POINT(41.55352308046137 73.67862373168748) bank46159 +46160 POINT(40.832068790114675 74.4899489282881) bank46160 +46161 POINT(40.02360467768013 74.63097638854472) bank46161 +46162 POINT(41.24026045531444 74.20773907252901) bank46162 +46163 POINT(39.81753918091991 75.00585784802375) bank46163 +46164 POINT(40.803995929291354 74.66046695867342) bank46164 +46165 POINT(40.07785517113447 74.20519743287745) bank46165 +46166 POINT(40.55914889493783 73.57377199725661) bank46166 +46167 POINT(41.17466490458794 73.82754436291123) bank46167 +46168 POINT(41.49475026599396 73.39008558903697) bank46168 +46169 POINT(40.27577864014838 74.25541573463114) bank46169 +46170 POINT(41.27780561571858 74.71986236329306) bank46170 +46171 POINT(39.74412233674094 73.70626483299435) bank46171 +46172 POINT(41.4504900162949 74.97597176239181) bank46172 +46173 POINT(41.2416021985695 74.60336146783256) bank46173 +46174 POINT(41.21369329715892 74.35386219041217) bank46174 +46175 POINT(41.66064046999417 74.55251905088906) bank46175 +46176 POINT(39.783583751142245 73.57692207820449) bank46176 +46177 POINT(40.60870613885202 73.83119460361736) bank46177 +46178 POINT(40.13088226171928 74.69848874901838) bank46178 +46179 POINT(40.11881574534167 74.78447676349253) bank46179 +46180 POINT(41.662089118555464 73.84992811372054) bank46180 +46181 POINT(40.95337314883486 74.14836346209707) bank46181 +46182 POINT(41.46998444644416 73.43516339261508) bank46182 +46183 POINT(40.25765035835709 74.4045065372268) bank46183 +46184 POINT(40.622109793511314 73.18595207065484) bank46184 +46185 POINT(39.76008199051272 73.03764400064694) bank46185 +46186 POINT(40.15334182640349 73.82844207960434) bank46186 +46187 POINT(40.96969682884958 73.59544115412716) bank46187 +46188 POINT(41.34540610355135 75.00309982366073) bank46188 +46189 POINT(41.18060899331588 73.4269989964924) bank46189 +46190 POINT(40.76170895768103 74.92035539788544) bank46190 +46191 POINT(41.222068523045564 73.0437755744767) bank46191 +46192 POINT(39.79928525070713 74.9058388456799) bank46192 +46193 POINT(40.969918440226934 73.38583751767774) bank46193 +46194 POINT(40.62852403046728 74.18317802767194) bank46194 +46195 POINT(41.17640470786054 74.1398109753444) bank46195 +46196 POINT(41.23000695232904 74.60223318941013) bank46196 +46197 POINT(41.61320977327933 73.56398135953035) bank46197 +46198 POINT(40.41337179566729 73.2180637908555) bank46198 +46199 POINT(41.38711911350854 74.9694747109464) bank46199 +46200 POINT(40.93641246874329 73.31852916084254) bank46200 +46201 POINT(39.78065658725173 73.21433666323226) bank46201 +46202 POINT(40.529641210219864 74.55167138408238) bank46202 +46203 POINT(40.099708424351704 73.19598031026834) bank46203 +46204 POINT(40.530710843787254 73.98358456198291) bank46204 +46205 POINT(41.56603741995572 73.2440599472746) bank46205 +46206 POINT(40.418235446116526 73.25860416305508) bank46206 +46207 POINT(40.82538001195811 73.75211233505928) bank46207 +46208 POINT(39.96305764259558 74.06252002792976) bank46208 +46209 POINT(40.90263238022695 74.00594773071548) bank46209 +46210 POINT(40.07774055896577 74.03115041354691) bank46210 +46211 POINT(41.39875760941112 73.58728809073763) bank46211 +46212 POINT(40.971180637130374 73.69432449415427) bank46212 +46213 POINT(40.65878484484809 74.55959073548185) bank46213 +46214 POINT(39.754029089533766 74.12725788448826) bank46214 +46215 POINT(40.90210783958996 73.7966150208403) bank46215 +46216 POINT(39.958079279864236 73.70607562739495) bank46216 +46217 POINT(41.00263804119659 73.97013006542936) bank46217 +46218 POINT(40.08678916335189 73.10553316841118) bank46218 +46219 POINT(40.628996768063274 73.69772951592562) bank46219 +46220 POINT(40.003401838582434 74.69631793052244) bank46220 +46221 POINT(39.749269856411 74.01504507039688) bank46221 +46222 POINT(40.4761959484497 74.35379789112704) bank46222 +46223 POINT(40.655337602173134 73.16878133045198) bank46223 +46224 POINT(40.80186845629998 74.11056522605453) bank46224 +46225 POINT(39.793890577066485 73.01526626778107) bank46225 +46226 POINT(39.73220225292397 73.25057738066901) bank46226 +46227 POINT(41.44043008228447 73.38640372704359) bank46227 +46228 POINT(40.58382498494838 73.08670665678076) bank46228 +46229 POINT(41.12630226029639 74.64140827790249) bank46229 +46230 POINT(40.22470139710905 73.8470640577063) bank46230 +46231 POINT(41.2999281696019 74.93101613585533) bank46231 +46232 POINT(40.507405355882206 73.90661861028576) bank46232 +46233 POINT(40.40888653602222 74.8211634887287) bank46233 +46234 POINT(40.98173298592365 74.70162434043701) bank46234 +46235 POINT(40.67921545882876 74.53654935920426) bank46235 +46236 POINT(40.90997937541802 74.17962835871388) bank46236 +46237 POINT(41.02128487734202 73.75059346143188) bank46237 +46238 POINT(41.09013468346216 74.57779718893721) bank46238 +46239 POINT(40.48376782566636 74.09860815785963) bank46239 +46240 POINT(41.65421475566369 74.99231101039459) bank46240 +46241 POINT(40.10515035046579 73.60369854658256) bank46241 +46242 POINT(40.03692354712636 73.7176350745262) bank46242 +46243 POINT(41.3770968385276 74.06893407759144) bank46243 +46244 POINT(41.37860884074987 74.39896154481688) bank46244 +46245 POINT(41.33139377496848 74.05151404074162) bank46245 +46246 POINT(41.59179748550286 73.12478251333894) bank46246 +46247 POINT(40.5625580952953 74.3344404765098) bank46247 +46248 POINT(40.82512974906385 73.50499552793299) bank46248 +46249 POINT(40.68359904369807 74.97440634377686) bank46249 +46250 POINT(41.53747707308054 73.31772502143414) bank46250 +46251 POINT(41.46126270836138 73.0752859574821) bank46251 +46252 POINT(40.9729420022358 74.13695018787476) bank46252 +46253 POINT(39.836754426917565 73.72059859041718) bank46253 +46254 POINT(40.72267050837585 74.3601793635545) bank46254 +46255 POINT(40.62791820473152 74.22464838997398) bank46255 +46256 POINT(40.057242776081914 74.826429681106) bank46256 +46257 POINT(40.87110246412305 74.43911333608887) bank46257 +46258 POINT(41.16603893455171 74.96220781165052) bank46258 +46259 POINT(40.98437494126209 73.76977076472649) bank46259 +46260 POINT(41.056056402712336 73.84885039572663) bank46260 +46261 POINT(39.750555889610624 73.13690710926943) bank46261 +46262 POINT(41.39184773982916 74.17181324990891) bank46262 +46263 POINT(41.34028586048678 73.10299216303984) bank46263 +46264 POINT(41.30476922529738 73.45965216340953) bank46264 +46265 POINT(41.04071171083632 73.69228241179036) bank46265 +46266 POINT(39.717127463750096 74.6199059324199) bank46266 +46267 POINT(39.727767180105545 73.32935998425806) bank46267 +46268 POINT(40.113248222101554 74.51132998185513) bank46268 +46269 POINT(41.14004229521892 73.92944957039549) bank46269 +46270 POINT(40.44391900061307 73.5110992547316) bank46270 +46271 POINT(41.040182662315395 74.40623338008547) bank46271 +46272 POINT(41.556727326194206 74.32506851315595) bank46272 +46273 POINT(41.11723496681097 74.18827916641273) bank46273 +46274 POINT(40.44437985105975 73.02071876902922) bank46274 +46275 POINT(40.67767971197574 73.9705789142836) bank46275 +46276 POINT(40.13918151653253 73.66293414166002) bank46276 +46277 POINT(40.293080948897305 73.4322229031064) bank46277 +46278 POINT(41.309482576704454 74.82714370924664) bank46278 +46279 POINT(41.07907180601411 73.80308066338259) bank46279 +46280 POINT(41.5378859916757 74.04895541414356) bank46280 +46281 POINT(40.03373440235846 73.31110402867007) bank46281 +46282 POINT(39.97732144527742 74.5226982892994) bank46282 +46283 POINT(40.06608516389993 74.00863538095132) bank46283 +46284 POINT(41.30875562480771 73.02102011825617) bank46284 +46285 POINT(41.505702494078236 74.91728647519952) bank46285 +46286 POINT(40.061897972138425 73.69436766197747) bank46286 +46287 POINT(40.76967475965287 74.60514574301827) bank46287 +46288 POINT(40.53869753871909 73.89241572751135) bank46288 +46289 POINT(40.841807412422874 73.28151540149697) bank46289 +46290 POINT(41.5913736757765 73.57979235022954) bank46290 +46291 POINT(41.19316280066029 74.1524676572549) bank46291 +46292 POINT(39.71938483069476 73.86558305362972) bank46292 +46293 POINT(40.50919042321742 73.89621106956288) bank46293 +46294 POINT(40.30764290608672 74.48235414929741) bank46294 +46295 POINT(39.998472047074344 73.8366946586973) bank46295 +46296 POINT(40.419464916058736 74.02104304857943) bank46296 +46297 POINT(41.39221985924463 74.62773995105019) bank46297 +46298 POINT(40.77501422851824 74.29920897727567) bank46298 +46299 POINT(41.54433245426925 74.42526630050854) bank46299 +46300 POINT(40.27875149710282 74.61736992888665) bank46300 +46301 POINT(40.86129931478338 74.09820050690088) bank46301 +46302 POINT(40.18550657229907 74.36030625954169) bank46302 +46303 POINT(40.11028891152806 74.33055937507996) bank46303 +46304 POINT(39.72404426671145 73.74544285036588) bank46304 +46305 POINT(40.12830972610308 73.21269817985147) bank46305 +46306 POINT(41.612412558434144 73.75303842685663) bank46306 +46307 POINT(39.94154708397374 73.54843071138345) bank46307 +46308 POINT(41.03486782221305 74.13477963988609) bank46308 +46309 POINT(40.27069080243186 74.71654584001712) bank46309 +46310 POINT(41.45945986571074 73.76935568190905) bank46310 +46311 POINT(41.497918098852665 73.60009774075296) bank46311 +46312 POINT(41.065486428427086 74.94390176177195) bank46312 +46313 POINT(41.35643028542165 74.82155248983827) bank46313 +46314 POINT(40.18394627920848 74.12822418583585) bank46314 +46315 POINT(39.98661681665925 74.43921860844156) bank46315 +46316 POINT(40.6784403875274 74.96406516121064) bank46316 +46317 POINT(40.19304480610047 74.37728310308032) bank46317 +46318 POINT(40.78498744557433 74.22903191311381) bank46318 +46319 POINT(39.928008252284634 73.44604697409433) bank46319 +46320 POINT(40.443470343833916 74.09058649287331) bank46320 +46321 POINT(40.1781245681537 73.70633498849182) bank46321 +46322 POINT(40.231534964635415 73.33761651483039) bank46322 +46323 POINT(40.53435381777257 74.00140137210225) bank46323 +46324 POINT(40.812925847240706 74.8744509507724) bank46324 +46325 POINT(41.171962579843296 73.2218913927944) bank46325 +46326 POINT(40.55323393653546 73.76931652793873) bank46326 +46327 POINT(40.95519953505039 73.20405162299613) bank46327 +46328 POINT(39.904690351520074 73.2972865918518) bank46328 +46329 POINT(39.82146112666105 73.66031442839314) bank46329 +46330 POINT(39.948845083689626 74.19211640715773) bank46330 +46331 POINT(39.824176695413165 73.20871480523564) bank46331 +46332 POINT(40.52580192688398 73.31518704652149) bank46332 +46333 POINT(40.69347891534485 74.1970681827235) bank46333 +46334 POINT(40.85837032118096 73.67314006186541) bank46334 +46335 POINT(41.22705928318252 73.74380758015295) bank46335 +46336 POINT(40.36227757664272 73.56763767260747) bank46336 +46337 POINT(40.16059665207815 73.10156091812863) bank46337 +46338 POINT(39.735106872306694 74.60892178566796) bank46338 +46339 POINT(40.675163522762546 74.68963618308344) bank46339 +46340 POINT(39.858694453949276 74.1369855831493) bank46340 +46341 POINT(40.977538502346185 74.33614479208877) bank46341 +46342 POINT(41.339006245995456 74.83156772373665) bank46342 +46343 POINT(40.70197319298857 74.71906738006382) bank46343 +46344 POINT(40.314345964514175 73.40250060540492) bank46344 +46345 POINT(41.40535484949999 73.72668731183971) bank46345 +46346 POINT(41.354623398030995 74.70409720067313) bank46346 +46347 POINT(40.19094962101909 73.68431406552038) bank46347 +46348 POINT(41.600083439905134 74.07500552435305) bank46348 +46349 POINT(40.179158109825615 74.18399978083556) bank46349 +46350 POINT(41.270004005262585 73.22233433497858) bank46350 +46351 POINT(40.945440486641964 73.45660037084204) bank46351 +46352 POINT(41.52939855027401 73.60263266324591) bank46352 +46353 POINT(41.65791523172749 73.72926112366594) bank46353 +46354 POINT(41.44994340086977 74.48197710283506) bank46354 +46355 POINT(40.60038447943174 74.59515671269394) bank46355 +46356 POINT(40.246683617443736 73.0660997172605) bank46356 +46357 POINT(41.147428183553934 73.53012529143614) bank46357 +46358 POINT(41.551319819708525 74.19768620517738) bank46358 +46359 POINT(41.2429157296745 73.48059558023117) bank46359 +46360 POINT(41.38346310985689 74.10265936182402) bank46360 +46361 POINT(41.36726094136799 73.19007936792032) bank46361 +46362 POINT(39.76569289142893 74.30815932698297) bank46362 +46363 POINT(40.107559503130275 74.1340084288032) bank46363 +46364 POINT(41.46669469402084 73.12575295404024) bank46364 +46365 POINT(41.52575035567273 74.18969535556236) bank46365 +46366 POINT(40.41412836479164 74.67747274500867) bank46366 +46367 POINT(40.154728968763465 74.11368058085444) bank46367 +46368 POINT(39.97275191687257 74.39042478861175) bank46368 +46369 POINT(40.51600319903944 73.0795952949679) bank46369 +46370 POINT(40.3142449617624 73.02577401037865) bank46370 +46371 POINT(41.41651884231547 74.84967052678884) bank46371 +46372 POINT(41.591656005857075 73.6839509544364) bank46372 +46373 POINT(40.853517430740744 73.23116641269691) bank46373 +46374 POINT(41.234405828782016 74.3544274610738) bank46374 +46375 POINT(40.62330616951876 74.8147392726562) bank46375 +46376 POINT(40.09482748884859 74.17699318186293) bank46376 +46377 POINT(41.64720793288479 74.71910828681365) bank46377 +46378 POINT(40.2657274465558 74.32226642422187) bank46378 +46379 POINT(41.080550205776525 73.74841142935824) bank46379 +46380 POINT(41.14425693118213 74.37622143425068) bank46380 +46381 POINT(40.64588757167349 73.9978171079404) bank46381 +46382 POINT(39.82152216767656 73.87378650885417) bank46382 +46383 POINT(41.313370771208675 74.35136912546447) bank46383 +46384 POINT(40.91599758443305 73.4365421712974) bank46384 +46385 POINT(39.911272041244374 73.1861289205908) bank46385 +46386 POINT(40.22289505910846 74.63338918261648) bank46386 +46387 POINT(41.63893351524281 74.84406079964049) bank46387 +46388 POINT(40.290815986844315 73.36468943848608) bank46388 +46389 POINT(41.292330976774544 73.86199148542781) bank46389 +46390 POINT(40.33947126216198 74.03942271224645) bank46390 +46391 POINT(39.88724055709908 74.81968400440724) bank46391 +46392 POINT(41.287620253493614 74.35247827121633) bank46392 +46393 POINT(40.60413384033047 73.3817979896173) bank46393 +46394 POINT(40.1155208660868 73.58606936716667) bank46394 +46395 POINT(40.16167110247291 73.15315513494636) bank46395 +46396 POINT(41.61010371194258 74.8082105884871) bank46396 +46397 POINT(40.14711265171835 74.94824432715443) bank46397 +46398 POINT(39.71333549152996 73.9930135323998) bank46398 +46399 POINT(40.38727644441534 73.23813314157975) bank46399 +46400 POINT(41.06204096198808 74.52547988662168) bank46400 +46401 POINT(41.03272888166468 73.19313442821795) bank46401 +46402 POINT(40.48990595279942 74.47329837679895) bank46402 +46403 POINT(40.472947172039746 73.75550106815983) bank46403 +46404 POINT(40.52129539309328 73.77630696929985) bank46404 +46405 POINT(40.44053986129274 73.04338679879487) bank46405 +46406 POINT(40.07910602238686 74.63190678489549) bank46406 +46407 POINT(41.248899071884146 74.79517049095605) bank46407 +46408 POINT(41.67038742772627 74.0551584712517) bank46408 +46409 POINT(40.99717777773999 74.96764188248241) bank46409 +46410 POINT(40.136203785122035 73.14410839559807) bank46410 +46411 POINT(41.53447264015675 74.69626303858306) bank46411 +46412 POINT(39.71993130521316 73.03467784780095) bank46412 +46413 POINT(41.12598207358524 73.21491475274756) bank46413 +46414 POINT(39.9807447903663 74.41599581870172) bank46414 +46415 POINT(40.407582971658734 74.28561836276232) bank46415 +46416 POINT(40.09939553283274 74.90383212538151) bank46416 +46417 POINT(40.20684255836515 73.22215729081233) bank46417 +46418 POINT(40.35253937338571 73.2552863916583) bank46418 +46419 POINT(41.22162291448143 74.93526053150245) bank46419 +46420 POINT(40.72309952697016 73.92961843960745) bank46420 +46421 POINT(39.979859714452054 74.41161850842971) bank46421 +46422 POINT(41.14405723399389 73.88993113237485) bank46422 +46423 POINT(41.49361315292409 73.1183730640912) bank46423 +46424 POINT(40.948987432613016 73.87219374323949) bank46424 +46425 POINT(41.388962101920164 74.82059669186286) bank46425 +46426 POINT(41.50559719132219 73.01024158716211) bank46426 +46427 POINT(39.824677381518825 73.82269222761424) bank46427 +46428 POINT(40.61025089072226 74.26421779863833) bank46428 +46429 POINT(41.578168137846205 73.3746342403515) bank46429 +46430 POINT(41.05500420332707 74.48591385101592) bank46430 +46431 POINT(41.106566526298536 73.41055996582807) bank46431 +46432 POINT(40.11816840128459 74.28549882379465) bank46432 +46433 POINT(39.975745715802454 73.19318010465858) bank46433 +46434 POINT(41.122766192522825 74.65839361371553) bank46434 +46435 POINT(40.78107535876009 73.89464602749155) bank46435 +46436 POINT(40.168160481996836 73.81289898778583) bank46436 +46437 POINT(41.1709209712308 74.65407190350723) bank46437 +46438 POINT(40.59046279803301 73.94626012259427) bank46438 +46439 POINT(39.76582658710202 73.43027068932469) bank46439 +46440 POINT(41.644508935935015 74.02387457931123) bank46440 +46441 POINT(41.025254025816864 74.98679891662717) bank46441 +46442 POINT(39.762373624013115 74.68887959436601) bank46442 +46443 POINT(40.11906222172398 74.26009365606338) bank46443 +46444 POINT(41.58091874121409 73.28001749853966) bank46444 +46445 POINT(40.52856728631952 73.57764000987761) bank46445 +46446 POINT(39.97426407886577 73.26782677824573) bank46446 +46447 POINT(41.42234721498475 73.69989097955707) bank46447 +46448 POINT(40.81395274553564 73.87547280357428) bank46448 +46449 POINT(39.78544756983233 74.25236997052716) bank46449 +46450 POINT(41.47505379624495 73.89712591001378) bank46450 +46451 POINT(40.45127706188582 74.74838163874306) bank46451 +46452 POINT(40.565917095689976 73.41922894939682) bank46452 +46453 POINT(40.294741458235436 74.31283166144316) bank46453 +46454 POINT(40.44480052597816 74.3448063563777) bank46454 +46455 POINT(40.07462328430674 74.82877833742475) bank46455 +46456 POINT(41.2015238952239 74.61835448920156) bank46456 +46457 POINT(40.11696490214254 73.82670094124487) bank46457 +46458 POINT(41.37332343508509 74.35954935320547) bank46458 +46459 POINT(40.929089657537226 73.06984956229022) bank46459 +46460 POINT(40.91932673363363 74.82717687118944) bank46460 +46461 POINT(40.70808977534347 74.11571818367703) bank46461 +46462 POINT(40.479062987555615 73.72895364489177) bank46462 +46463 POINT(40.22820620696732 74.96542718522993) bank46463 +46464 POINT(40.122248293598 73.69891416247947) bank46464 +46465 POINT(41.17998997179314 73.72728484982325) bank46465 +46466 POINT(40.65435988462956 73.4364161177203) bank46466 +46467 POINT(40.551428322974445 74.47530352890182) bank46467 +46468 POINT(40.99258889457153 73.61086252200705) bank46468 +46469 POINT(41.67702477924204 73.62931465351888) bank46469 +46470 POINT(40.848175273166476 73.28205378164064) bank46470 +46471 POINT(40.62750261146702 74.40283819088593) bank46471 +46472 POINT(40.47179823730861 73.95017385445314) bank46472 +46473 POINT(41.36915480249046 73.55243413235264) bank46473 +46474 POINT(40.228779572733345 74.83857286386304) bank46474 +46475 POINT(39.89998450178688 73.92119789583063) bank46475 +46476 POINT(40.70303324371822 73.51475007767789) bank46476 +46477 POINT(41.3188032718165 73.16353810590898) bank46477 +46478 POINT(39.87632485330285 74.58503560413301) bank46478 +46479 POINT(39.83608125112046 73.23219143964732) bank46479 +46480 POINT(41.1852776965145 73.11598573152148) bank46480 +46481 POINT(40.03622955697058 74.08477876921133) bank46481 +46482 POINT(40.02825375452266 73.96360630189339) bank46482 +46483 POINT(40.20903336000093 75.00296900745403) bank46483 +46484 POINT(41.668024167937766 74.02764042379604) bank46484 +46485 POINT(41.52680034746901 73.49946805633832) bank46485 +46486 POINT(41.043116118730744 74.90119172345065) bank46486 +46487 POINT(39.98648448374633 74.34646675028569) bank46487 +46488 POINT(41.59197812005901 74.79875004713713) bank46488 +46489 POINT(40.721556677608966 73.85397652055617) bank46489 +46490 POINT(41.711519478200536 73.29337120308462) bank46490 +46491 POINT(41.00612497246952 74.32769239380981) bank46491 +46492 POINT(40.5254649709919 73.91422697491063) bank46492 +46493 POINT(39.84456472402233 73.383184915293) bank46493 +46494 POINT(40.0108622503367 73.4175085070234) bank46494 +46495 POINT(40.83933937228785 73.98141923844602) bank46495 +46496 POINT(40.61163606511905 73.14490497211446) bank46496 +46497 POINT(40.73277961542431 74.46461305431333) bank46497 +46498 POINT(41.55263879536571 74.64314878659509) bank46498 +46499 POINT(40.223621453978076 74.80974077445593) bank46499 +46500 POINT(39.79123297721357 74.44215880408832) bank46500 +46501 POINT(41.60547021256302 74.18126127284043) bank46501 +46502 POINT(40.712547638017135 74.37561090351208) bank46502 +46503 POINT(40.643626409563815 74.26371646597488) bank46503 +46504 POINT(39.79595014472415 73.00806794366586) bank46504 +46505 POINT(39.93437375060422 73.9943121523679) bank46505 +46506 POINT(40.962635850244375 74.03768788302423) bank46506 +46507 POINT(40.188424191948165 74.47621521522375) bank46507 +46508 POINT(41.12895671101376 74.05373306613437) bank46508 +46509 POINT(40.127361373463316 73.17354770082747) bank46509 +46510 POINT(40.943792500016535 74.03580493059454) bank46510 +46511 POINT(39.77295018708202 73.91694383366783) bank46511 +46512 POINT(41.27839734969212 73.38737909991755) bank46512 +46513 POINT(39.74742792547781 74.58445730732082) bank46513 +46514 POINT(41.49012795337742 74.99040482801446) bank46514 +46515 POINT(40.006449396244356 74.53731477661272) bank46515 +46516 POINT(41.60989969666969 73.80539967997343) bank46516 +46517 POINT(39.94472017084367 74.88933359521424) bank46517 +46518 POINT(39.88179042083011 74.6021814916535) bank46518 +46519 POINT(41.13670483436751 74.39879626618226) bank46519 +46520 POINT(40.10297700067526 73.96335937155565) bank46520 +46521 POINT(39.75842374544949 73.38753274120562) bank46521 +46522 POINT(39.965218002196984 74.01382728267865) bank46522 +46523 POINT(40.17162497171494 73.33781758299291) bank46523 +46524 POINT(40.96580539378179 73.89377006062107) bank46524 +46525 POINT(40.61134330811865 73.55120454070351) bank46525 +46526 POINT(41.70090065106062 74.00207911011893) bank46526 +46527 POINT(40.04712077420724 73.64771154674847) bank46527 +46528 POINT(39.83327195266571 73.71003936721004) bank46528 +46529 POINT(40.15741225273632 73.07894755764191) bank46529 +46530 POINT(41.29447619988441 73.60465254778131) bank46530 +46531 POINT(41.29125502970888 73.77510978792863) bank46531 +46532 POINT(41.62834762169558 74.12012885769582) bank46532 +46533 POINT(39.85039577487842 73.6309597841642) bank46533 +46534 POINT(40.30737679317688 73.6066654298717) bank46534 +46535 POINT(41.32373198127809 74.46600912925304) bank46535 +46536 POINT(40.660748742230915 73.39630662088825) bank46536 +46537 POINT(40.80307659737657 73.09456885333637) bank46537 +46538 POINT(41.45875214682018 73.06666632136012) bank46538 +46539 POINT(41.34996743541867 73.65489372983232) bank46539 +46540 POINT(40.93806520960839 74.7939687647403) bank46540 +46541 POINT(41.46939178522511 74.4807914512052) bank46541 +46542 POINT(40.924163064034786 74.36579296719918) bank46542 +46543 POINT(41.326828648391455 74.65839418369644) bank46543 +46544 POINT(40.41774071995848 74.68471806480864) bank46544 +46545 POINT(40.829336704338274 73.09133803424913) bank46545 +46546 POINT(39.815948089457265 73.92466671280994) bank46546 +46547 POINT(41.50260198184829 73.63458293269548) bank46547 +46548 POINT(40.60955271647045 73.74670267298616) bank46548 +46549 POINT(40.5419864897683 74.5219174924429) bank46549 +46550 POINT(41.6950989009402 73.11047954890263) bank46550 +46551 POINT(40.09618084608911 74.9240728780376) bank46551 +46552 POINT(39.742734974578866 74.79005230907987) bank46552 +46553 POINT(41.559134625883146 73.85201385876607) bank46553 +46554 POINT(39.86417135669988 74.6155355803231) bank46554 +46555 POINT(40.83875399560983 73.58654948459211) bank46555 +46556 POINT(40.59258557866875 74.15761985677449) bank46556 +46557 POINT(39.92324932314931 73.4759793853118) bank46557 +46558 POINT(40.743878648507874 74.18764075908595) bank46558 +46559 POINT(41.07896550437136 74.28387702016244) bank46559 +46560 POINT(40.573565792782304 74.42681122449972) bank46560 +46561 POINT(40.37482740094453 73.39461513312172) bank46561 +46562 POINT(41.51086202603507 73.9236843551908) bank46562 +46563 POINT(40.29493664332571 74.91538012842705) bank46563 +46564 POINT(40.76702154234078 73.54968098013846) bank46564 +46565 POINT(41.32838181717114 73.02465944168362) bank46565 +46566 POINT(40.03537234393099 73.13157094001997) bank46566 +46567 POINT(40.20071090088076 74.24927763670665) bank46567 +46568 POINT(39.856274027473205 74.47855924460852) bank46568 +46569 POINT(41.33588076583677 74.14406025946177) bank46569 +46570 POINT(40.86943060776023 74.97548710201073) bank46570 +46571 POINT(39.873237424644984 73.14064561393374) bank46571 +46572 POINT(41.08573676216443 73.47319517697434) bank46572 +46573 POINT(41.25566724934902 74.9718394632745) bank46573 +46574 POINT(39.72187059629559 74.39844212494066) bank46574 +46575 POINT(40.20405146751501 74.16023298373185) bank46575 +46576 POINT(41.15414780636057 73.95793594144699) bank46576 +46577 POINT(39.78704885571293 74.30121479893319) bank46577 +46578 POINT(40.96044623550117 74.50659370636396) bank46578 +46579 POINT(41.12539917785539 73.92524030592493) bank46579 +46580 POINT(41.52573156964888 73.25947431085038) bank46580 +46581 POINT(41.09331017601831 74.00859558037422) bank46581 +46582 POINT(40.103307307363245 73.38793224994889) bank46582 +46583 POINT(40.825921595353535 73.32017048842431) bank46583 +46584 POINT(41.47578563659946 73.56408387992056) bank46584 +46585 POINT(40.07368123275701 74.24793521388509) bank46585 +46586 POINT(40.91166037214482 73.66992798157598) bank46586 +46587 POINT(41.1648989246807 73.01668569649433) bank46587 +46588 POINT(40.72761561300476 74.29689951089364) bank46588 +46589 POINT(40.4104920252849 73.3486129063122) bank46589 +46590 POINT(41.54846342208253 74.26083337302987) bank46590 +46591 POINT(41.57735399676272 74.66993616676473) bank46591 +46592 POINT(39.99405039215834 74.83662920622598) bank46592 +46593 POINT(41.51032512192951 74.93951762127507) bank46593 +46594 POINT(40.52480461778273 74.81273509328341) bank46594 +46595 POINT(41.30413026432689 74.02352922482235) bank46595 +46596 POINT(40.461788144746194 74.1092521759512) bank46596 +46597 POINT(40.30796183491304 73.88104542135251) bank46597 +46598 POINT(41.288830547555385 73.15475707891254) bank46598 +46599 POINT(41.02561675756821 74.13896733134027) bank46599 +46600 POINT(39.951799452978705 74.76189531293365) bank46600 +46601 POINT(39.823475709124466 74.98951909555505) bank46601 +46602 POINT(40.36468408161577 74.63204806039448) bank46602 +46603 POINT(40.08033469319411 74.44091329166567) bank46603 +46604 POINT(41.69349833572041 73.77543558182298) bank46604 +46605 POINT(40.3957552650581 73.06609796151507) bank46605 +46606 POINT(40.59935968431449 73.77763123975144) bank46606 +46607 POINT(40.875516253518995 73.9153787050095) bank46607 +46608 POINT(41.69099766225555 74.33728817738464) bank46608 +46609 POINT(40.810286545530865 74.94063912180033) bank46609 +46610 POINT(39.910527189015376 74.17003331430683) bank46610 +46611 POINT(40.29236700185056 74.97629962877039) bank46611 +46612 POINT(41.252893246165854 74.50702844675078) bank46612 +46613 POINT(41.63892240735936 73.62119570431308) bank46613 +46614 POINT(41.28081409297477 73.82156869102208) bank46614 +46615 POINT(40.26717641597476 73.07512901317476) bank46615 +46616 POINT(41.63443519420036 74.64348725813356) bank46616 +46617 POINT(40.45146898052985 73.5071169587143) bank46617 +46618 POINT(41.70550837589131 73.59085180325046) bank46618 +46619 POINT(40.9307920362817 74.14458732466835) bank46619 +46620 POINT(40.692209989942654 73.06044723431158) bank46620 +46621 POINT(40.0749344661426 74.3872084247709) bank46621 +46622 POINT(40.99923735532392 74.49599062427009) bank46622 +46623 POINT(41.0115932416612 74.43244564853518) bank46623 +46624 POINT(41.271690869703846 73.58336456871744) bank46624 +46625 POINT(40.47954232165486 74.7178456356175) bank46625 +46626 POINT(41.064420594327636 74.53687892253056) bank46626 +46627 POINT(40.79828021217139 73.14320161354452) bank46627 +46628 POINT(39.9814442808746 74.88802769587043) bank46628 +46629 POINT(41.23036331822135 74.6920866137098) bank46629 +46630 POINT(41.60046831856107 74.16964651578371) bank46630 +46631 POINT(41.61654013561487 74.45899503973071) bank46631 +46632 POINT(40.704199275992245 74.54990617855525) bank46632 +46633 POINT(41.68825887442815 74.619343166817) bank46633 +46634 POINT(41.697693078344976 73.94611236707554) bank46634 +46635 POINT(41.18612491331609 73.22919081634795) bank46635 +46636 POINT(39.85298258989571 74.73670857188525) bank46636 +46637 POINT(41.090151371751595 73.15247112209255) bank46637 +46638 POINT(41.33412520250137 73.3788642121672) bank46638 +46639 POINT(41.11422868756908 73.14005162019585) bank46639 +46640 POINT(41.63363204191259 73.68241299100626) bank46640 +46641 POINT(40.64064637928453 74.50125824691716) bank46641 +46642 POINT(41.575819838574105 73.66984855539378) bank46642 +46643 POINT(41.338015350739255 74.6147209056306) bank46643 +46644 POINT(40.23141233617011 73.92912044386156) bank46644 +46645 POINT(40.919052578573286 74.32640288636117) bank46645 +46646 POINT(41.03599450789457 74.48011391781162) bank46646 +46647 POINT(41.4432419079786 73.536635330508) bank46647 +46648 POINT(41.366680394149625 73.23003416937823) bank46648 +46649 POINT(41.504617181818055 73.72846853345494) bank46649 +46650 POINT(41.04288869989856 74.6887351249452) bank46650 +46651 POINT(41.582546075651 73.20395356336441) bank46651 +46652 POINT(41.60733300897049 74.10700668805973) bank46652 +46653 POINT(40.859797172570765 73.08992725367219) bank46653 +46654 POINT(40.594707935677675 74.84185780386262) bank46654 +46655 POINT(40.02846538147709 73.15337835464003) bank46655 +46656 POINT(40.652952414934845 73.14895617712676) bank46656 +46657 POINT(40.24056384184763 74.63615132346673) bank46657 +46658 POINT(40.80804469760343 74.08484847743419) bank46658 +46659 POINT(40.93071273716186 73.05284703105302) bank46659 +46660 POINT(41.10395092773146 74.84873734926518) bank46660 +46661 POINT(39.95530381517307 73.93125727865622) bank46661 +46662 POINT(41.5896246454389 73.70561926963784) bank46662 +46663 POINT(41.17578488168577 73.78034096797593) bank46663 +46664 POINT(41.15411383723115 74.1352039083639) bank46664 +46665 POINT(40.826685330932065 74.23186484799126) bank46665 +46666 POINT(40.90665496935739 74.77604176872929) bank46666 +46667 POINT(40.86198041308704 74.73247458859412) bank46667 +46668 POINT(41.6091027007533 73.16823378480396) bank46668 +46669 POINT(40.26636918137622 74.52973549767691) bank46669 +46670 POINT(39.71394869222407 74.69817131033719) bank46670 +46671 POINT(40.95450854004902 73.54271521625196) bank46671 +46672 POINT(41.500903710598955 73.63313576861076) bank46672 +46673 POINT(40.22902702588974 74.08231427306193) bank46673 +46674 POINT(40.29448214643981 74.90374145009855) bank46674 +46675 POINT(40.95316173999759 73.9265993032333) bank46675 +46676 POINT(40.63559805804409 73.30507575761237) bank46676 +46677 POINT(39.740277809763704 74.75538093985877) bank46677 +46678 POINT(40.36273012190401 74.49620158294967) bank46678 +46679 POINT(40.071196027624886 74.9923188276376) bank46679 +46680 POINT(41.274442229253864 74.9140715410979) bank46680 +46681 POINT(41.4560175890302 74.12618307183405) bank46681 +46682 POINT(40.09019808742029 73.67465531738658) bank46682 +46683 POINT(39.782215825366386 73.88639646820921) bank46683 +46684 POINT(41.425384569633025 73.60359349257388) bank46684 +46685 POINT(40.37956455441406 73.60338832273095) bank46685 +46686 POINT(41.185020896614304 74.40389563666584) bank46686 +46687 POINT(40.637810293504174 74.31644173835062) bank46687 +46688 POINT(40.10601600533112 74.01089076361536) bank46688 +46689 POINT(40.191586415205244 74.34458734693732) bank46689 +46690 POINT(40.145077679149345 74.80983359305048) bank46690 +46691 POINT(39.83878151491799 73.85178971503248) bank46691 +46692 POINT(41.50624094149112 74.7099038973977) bank46692 +46693 POINT(41.542250619350725 73.59497144113352) bank46693 +46694 POINT(40.6628419960751 73.62946766010506) bank46694 +46695 POINT(40.480703657419724 74.10965576542941) bank46695 +46696 POINT(39.92223314122967 74.84464308927095) bank46696 +46697 POINT(41.273447778418905 73.95023127096954) bank46697 +46698 POINT(41.1869332328385 73.46626438695814) bank46698 +46699 POINT(40.02553336666043 73.65875959105585) bank46699 +46700 POINT(40.858598284623056 74.95005934127163) bank46700 +46701 POINT(40.884853483106674 73.53747622514638) bank46701 +46702 POINT(41.49205729299587 73.49452970335376) bank46702 +46703 POINT(40.75844420331395 73.98216733491711) bank46703 +46704 POINT(40.334805176932825 73.70104645593155) bank46704 +46705 POINT(40.49998460837457 73.96053043030037) bank46705 +46706 POINT(41.3837592886932 73.4155010989277) bank46706 +46707 POINT(41.52399434142384 74.15340167039025) bank46707 +46708 POINT(41.660216702797754 74.87160166813585) bank46708 +46709 POINT(41.12698333859326 74.10048563349655) bank46709 +46710 POINT(41.525278123553015 74.13566742439998) bank46710 +46711 POINT(40.77244542051905 73.1493817827528) bank46711 +46712 POINT(40.098875165465586 74.76590978489946) bank46712 +46713 POINT(40.19591499700599 73.38473939722773) bank46713 +46714 POINT(41.591644430581425 74.28355838745956) bank46714 +46715 POINT(40.030591171553986 74.96353406026051) bank46715 +46716 POINT(41.243831149386736 73.03608331140704) bank46716 +46717 POINT(41.347045678000974 73.13533901532189) bank46717 +46718 POINT(41.534188830179346 73.23441475374425) bank46718 +46719 POINT(40.15405047411096 73.92986713562708) bank46719 +46720 POINT(40.11461720158745 73.02209862559964) bank46720 +46721 POINT(40.77966222027509 73.09137170494793) bank46721 +46722 POINT(40.74573654377989 73.71988871405676) bank46722 +46723 POINT(41.25618635882304 73.35582718320731) bank46723 +46724 POINT(40.540093811538824 73.88793541782564) bank46724 +46725 POINT(41.06027788198764 74.16863796975501) bank46725 +46726 POINT(41.191486909559636 74.89203521821011) bank46726 +46727 POINT(41.54823806839095 74.7191603869496) bank46727 +46728 POINT(41.41272789717208 73.36223666155735) bank46728 +46729 POINT(40.205543125238925 74.04771024569888) bank46729 +46730 POINT(40.251670834793394 73.90788405455096) bank46730 +46731 POINT(39.94726215752986 73.55915089866537) bank46731 +46732 POINT(41.191598807011395 73.38700613335396) bank46732 +46733 POINT(40.95950947145799 74.8276224605778) bank46733 +46734 POINT(41.56824515579952 74.00152285397097) bank46734 +46735 POINT(39.92854644067222 74.71192907548753) bank46735 +46736 POINT(40.928080529432115 74.10593907422162) bank46736 +46737 POINT(40.30426383367455 73.1617628471899) bank46737 +46738 POINT(41.16433995307718 73.64901071147575) bank46738 +46739 POINT(40.01051350421186 74.08690516794071) bank46739 +46740 POINT(40.53963322139681 74.1154163698939) bank46740 +46741 POINT(40.9387871502981 73.8677149208903) bank46741 +46742 POINT(40.65533093529941 73.09384818999304) bank46742 +46743 POINT(41.036889156615686 73.3000278543567) bank46743 +46744 POINT(41.66448060640972 74.2329821690807) bank46744 +46745 POINT(41.021633260066736 74.3004334614868) bank46745 +46746 POINT(41.51060728000872 74.0542361928854) bank46746 +46747 POINT(40.35051584804214 74.31125048155644) bank46747 +46748 POINT(40.20348527589953 73.41508437307232) bank46748 +46749 POINT(40.88099730306028 73.90762828060052) bank46749 +46750 POINT(41.34009950721918 74.83960113937918) bank46750 +46751 POINT(39.95693027540497 73.90753900609603) bank46751 +46752 POINT(40.03592630493075 73.78692055945476) bank46752 +46753 POINT(41.24185460715017 74.54987895396899) bank46753 +46754 POINT(39.86095755839631 73.97717582188957) bank46754 +46755 POINT(41.66055696740873 74.55963664036555) bank46755 +46756 POINT(41.65315643928479 73.95940365196961) bank46756 +46757 POINT(41.18665411212541 73.92938003118982) bank46757 +46758 POINT(41.25836434526829 74.88749962023333) bank46758 +46759 POINT(40.84337374974398 74.30624935468384) bank46759 +46760 POINT(40.82770390887839 74.04333058221859) bank46760 +46761 POINT(41.154150878205066 73.17834154988417) bank46761 +46762 POINT(41.11730517633504 74.86362893931009) bank46762 +46763 POINT(40.71403526420707 73.46562103552743) bank46763 +46764 POINT(40.17964430273555 74.39969269208363) bank46764 +46765 POINT(41.604935570654526 73.08090396702016) bank46765 +46766 POINT(39.934148697691484 73.40566400394447) bank46766 +46767 POINT(41.6526781500301 74.6491382781384) bank46767 +46768 POINT(41.16675727935256 73.56174153753045) bank46768 +46769 POINT(40.009910955660885 74.43394818307891) bank46769 +46770 POINT(40.57052536188556 73.56923373942368) bank46770 +46771 POINT(40.245314889166664 73.82606114168917) bank46771 +46772 POINT(40.7556813167912 74.98798673540058) bank46772 +46773 POINT(41.025558079677005 73.86100156050634) bank46773 +46774 POINT(41.68110480509335 74.65005933253063) bank46774 +46775 POINT(39.95214700797423 73.50104140600406) bank46775 +46776 POINT(41.645154818326525 74.15737147742762) bank46776 +46777 POINT(40.90009816966215 73.76653673394277) bank46777 +46778 POINT(40.366862695760716 73.94328283163841) bank46778 +46779 POINT(40.546305071882486 73.10412938917345) bank46779 +46780 POINT(40.003579839612236 73.2174960041262) bank46780 +46781 POINT(39.751070311551494 74.4371566900337) bank46781 +46782 POINT(41.23103657722872 73.74878842332689) bank46782 +46783 POINT(40.35004113910267 73.75143224920953) bank46783 +46784 POINT(41.14726898813067 74.91519646425316) bank46784 +46785 POINT(40.0615700014584 74.44101816769685) bank46785 +46786 POINT(40.32728843143862 74.5378665199918) bank46786 +46787 POINT(39.92130488219671 74.77540216562845) bank46787 +46788 POINT(40.235608480647414 74.1978515811467) bank46788 +46789 POINT(40.82239875993487 73.30943352643457) bank46789 +46790 POINT(39.91070328637281 74.6421228767651) bank46790 +46791 POINT(41.111726155744265 74.94740006128774) bank46791 +46792 POINT(41.5709148051082 73.65706665659415) bank46792 +46793 POINT(41.359449669212566 74.1494429448621) bank46793 +46794 POINT(40.69281516547488 74.02201841152241) bank46794 +46795 POINT(40.876042133416966 73.67670001908145) bank46795 +46796 POINT(40.13269624585371 73.4293245918603) bank46796 +46797 POINT(40.389752390373296 74.74568693658765) bank46797 +46798 POINT(41.495779126832524 74.64995083386346) bank46798 +46799 POINT(41.51924371016754 74.94307496500507) bank46799 +46800 POINT(40.12445614219968 74.04802919225426) bank46800 +46801 POINT(40.57785715306942 74.42695279309248) bank46801 +46802 POINT(40.884663641484885 74.0415082025661) bank46802 +46803 POINT(40.87265202781527 73.34875279997753) bank46803 +46804 POINT(40.28633484472841 73.40844486084568) bank46804 +46805 POINT(40.986590263941515 74.37034518992212) bank46805 +46806 POINT(41.089267894938814 73.45644844897872) bank46806 +46807 POINT(41.31716163984514 74.52640918737173) bank46807 +46808 POINT(39.85796981444005 73.74203358913148) bank46808 +46809 POINT(39.77350783946374 73.61536129459391) bank46809 +46810 POINT(41.17652853524096 73.03346933331665) bank46810 +46811 POINT(41.334916118907685 73.83990209742733) bank46811 +46812 POINT(39.74619243321262 73.13596359436579) bank46812 +46813 POINT(39.809443298934276 74.63050359415955) bank46813 +46814 POINT(41.339607567042385 74.41803557875167) bank46814 +46815 POINT(41.433762350864406 73.58726512016733) bank46815 +46816 POINT(41.31351251811216 74.30359488238003) bank46816 +46817 POINT(40.24564672489322 74.67638291891187) bank46817 +46818 POINT(40.651109262534966 73.87327893322978) bank46818 +46819 POINT(40.04068169610348 73.15114715943774) bank46819 +46820 POINT(41.64837526253057 73.09566236938466) bank46820 +46821 POINT(41.36981505586434 73.74452685111935) bank46821 +46822 POINT(40.326320286977094 74.79126831675684) bank46822 +46823 POINT(40.12271477972169 74.70648679585983) bank46823 +46824 POINT(41.70877681478495 74.17724383235463) bank46824 +46825 POINT(39.74852818475753 73.00680049510079) bank46825 +46826 POINT(39.77729112556036 74.12431008426329) bank46826 +46827 POINT(41.220329832185705 74.73111228047145) bank46827 +46828 POINT(39.82258777241894 74.92486061416216) bank46828 +46829 POINT(40.55095015902432 73.28200680331571) bank46829 +46830 POINT(40.34248326429329 73.9059063983049) bank46830 +46831 POINT(40.70030236346628 73.56301352989263) bank46831 +46832 POINT(41.300948462079916 73.74816767925911) bank46832 +46833 POINT(41.14608549766227 74.3866387950264) bank46833 +46834 POINT(40.738638899961984 74.49126035466702) bank46834 +46835 POINT(41.62733183043367 73.91449819363346) bank46835 +46836 POINT(41.685443005539526 74.56856730947567) bank46836 +46837 POINT(41.4763303936513 73.87414974066546) bank46837 +46838 POINT(40.933987249087295 73.1293919361985) bank46838 +46839 POINT(40.54590261879867 73.78094994017839) bank46839 +46840 POINT(41.48435121643358 74.22036124787334) bank46840 +46841 POINT(41.53821660767282 73.6625284321558) bank46841 +46842 POINT(41.141805015235974 73.87082518972039) bank46842 +46843 POINT(41.61561168566575 74.4990079092835) bank46843 +46844 POINT(40.124681798170975 74.08055679732377) bank46844 +46845 POINT(40.798443838586294 74.24631865091978) bank46845 +46846 POINT(40.39944327776976 74.05217796226927) bank46846 +46847 POINT(40.44182341492238 73.23499637902688) bank46847 +46848 POINT(39.777800831226436 74.18985993535932) bank46848 +46849 POINT(40.08661865364786 74.01370943273199) bank46849 +46850 POINT(40.19948828688333 73.97748758389237) bank46850 +46851 POINT(41.32349823046188 74.44205958296068) bank46851 +46852 POINT(40.35318575538957 73.90094534475074) bank46852 +46853 POINT(39.779222945313784 73.63413949215737) bank46853 +46854 POINT(40.04091132924107 73.55846140520744) bank46854 +46855 POINT(41.18189146897833 74.66588680880481) bank46855 +46856 POINT(40.689227074555184 73.35908331196956) bank46856 +46857 POINT(41.49129038553828 74.37580685256113) bank46857 +46858 POINT(41.44964315870095 74.81260332334664) bank46858 +46859 POINT(40.6519176900774 74.41357386854173) bank46859 +46860 POINT(40.59009547083837 74.10393485269337) bank46860 +46861 POINT(40.81652974234999 74.36024420485198) bank46861 +46862 POINT(39.74871385119397 73.86051347386348) bank46862 +46863 POINT(40.005481804162685 73.76711897590604) bank46863 +46864 POINT(41.137575577646615 73.7209200771857) bank46864 +46865 POINT(40.73095845948568 74.54153046595161) bank46865 +46866 POINT(40.988878940152794 74.65611197796827) bank46866 +46867 POINT(40.287183368245366 73.57917989261028) bank46867 +46868 POINT(39.96354510534693 75.00571173887316) bank46868 +46869 POINT(41.298709408875276 74.5257103449037) bank46869 +46870 POINT(40.975221851564136 73.92804559341285) bank46870 +46871 POINT(41.554335289677766 73.21179062675756) bank46871 +46872 POINT(40.10856841830043 73.97775609380079) bank46872 +46873 POINT(40.93569558760707 73.5890820654376) bank46873 +46874 POINT(40.8933819687034 74.6566106928322) bank46874 +46875 POINT(41.638199955732354 73.63244353273134) bank46875 +46876 POINT(41.63628084021441 73.79535269275655) bank46876 +46877 POINT(41.61324957897184 73.3938735516456) bank46877 +46878 POINT(40.2966503002336 73.88326555939135) bank46878 +46879 POINT(41.0415347642258 74.39606284442353) bank46879 +46880 POINT(40.355892121180915 73.1254843712542) bank46880 +46881 POINT(41.01341956422463 73.99002291221367) bank46881 +46882 POINT(41.67488605419629 74.91553317339655) bank46882 +46883 POINT(40.43112773698513 74.91031438757678) bank46883 +46884 POINT(40.01442162142958 74.82290711164903) bank46884 +46885 POINT(40.56843502517454 73.03116474499818) bank46885 +46886 POINT(41.25245278637798 73.50620773917319) bank46886 +46887 POINT(40.99235361414993 74.80744153849433) bank46887 +46888 POINT(40.9352982604689 74.9237536532917) bank46888 +46889 POINT(40.97211554514661 74.01083814250451) bank46889 +46890 POINT(41.00310307644416 74.45588952364778) bank46890 +46891 POINT(39.96499354085606 73.07226912975095) bank46891 +46892 POINT(40.85651128138396 73.6559218008257) bank46892 +46893 POINT(41.142227740824055 73.71278807631516) bank46893 +46894 POINT(41.53135651255985 73.86511391514505) bank46894 +46895 POINT(41.21583219946557 73.3721230126949) bank46895 +46896 POINT(41.061060707169645 74.83582453223953) bank46896 +46897 POINT(40.393461813424004 74.81615443315182) bank46897 +46898 POINT(41.49251472278327 74.09473223542506) bank46898 +46899 POINT(40.73530651023442 74.60564621228521) bank46899 +46900 POINT(39.76793826070375 74.02792692624013) bank46900 +46901 POINT(40.99719093043914 74.4543571376667) bank46901 +46902 POINT(39.82116682178348 74.28752926564724) bank46902 +46903 POINT(40.77537293237451 73.62918101916284) bank46903 +46904 POINT(40.50852304149589 73.90955590567185) bank46904 +46905 POINT(40.1471739694536 73.74530292581132) bank46905 +46906 POINT(40.00809254978465 73.89696491183179) bank46906 +46907 POINT(40.458005397305676 74.72165781239178) bank46907 +46908 POINT(39.89549440833036 73.97539837311025) bank46908 +46909 POINT(41.69204958240662 74.89322549945638) bank46909 +46910 POINT(40.540745542524405 74.65791998029424) bank46910 +46911 POINT(41.28729552669602 73.41285988327971) bank46911 +46912 POINT(40.43279554379111 73.53309451596472) bank46912 +46913 POINT(41.09020143666463 73.35291778801947) bank46913 +46914 POINT(41.27894377638936 73.23861066428255) bank46914 +46915 POINT(40.46946599498231 74.19638413780451) bank46915 +46916 POINT(40.727081126898774 74.39346156232386) bank46916 +46917 POINT(41.21484124200136 74.31920351999376) bank46917 +46918 POINT(40.82142050777219 74.33692320848844) bank46918 +46919 POINT(40.61938602168812 73.25832847853525) bank46919 +46920 POINT(40.369130749392184 73.41726482538398) bank46920 +46921 POINT(41.660831537968875 74.4602708825864) bank46921 +46922 POINT(40.000263033834806 73.23641014814677) bank46922 +46923 POINT(41.70437854510325 74.48444781836045) bank46923 +46924 POINT(40.85770079031352 74.28240884911317) bank46924 +46925 POINT(39.9584554474679 74.93587750999197) bank46925 +46926 POINT(39.86599389131731 74.08980444449635) bank46926 +46927 POINT(40.78716863569004 73.70677652466678) bank46927 +46928 POINT(39.75277619910239 74.01264797319193) bank46928 +46929 POINT(41.15004477114159 73.31691331453858) bank46929 +46930 POINT(41.585231896181895 74.628151449806) bank46930 +46931 POINT(40.12319874510359 73.2783409002433) bank46931 +46932 POINT(41.21030675521756 74.39435551767448) bank46932 +46933 POINT(40.90766505250511 74.22584354208833) bank46933 +46934 POINT(41.712232046696045 73.94822452358282) bank46934 +46935 POINT(39.728813030531356 74.98974364552743) bank46935 +46936 POINT(41.71276676398195 74.98804887202895) bank46936 +46937 POINT(40.81749505333069 73.96930064012844) bank46937 +46938 POINT(41.30824962265178 73.85081619467218) bank46938 +46939 POINT(40.03322012719371 73.32306510156502) bank46939 +46940 POINT(41.18595388736767 73.10878191171186) bank46940 +46941 POINT(40.49021849171785 73.06669972979661) bank46941 +46942 POINT(41.17899987022853 73.98523937454625) bank46942 +46943 POINT(40.97379786586849 73.36212975111019) bank46943 +46944 POINT(41.03317246463472 74.21985112284372) bank46944 +46945 POINT(40.14977349718082 73.77271433387529) bank46945 +46946 POINT(39.925010074910844 73.76038485012727) bank46946 +46947 POINT(40.952252526616796 73.03830801241718) bank46947 +46948 POINT(39.91785897361805 73.0303102576122) bank46948 +46949 POINT(40.540864774184826 73.5889638294676) bank46949 +46950 POINT(40.619134312526825 74.62865229098472) bank46950 +46951 POINT(41.50303927331811 74.14106908924117) bank46951 +46952 POINT(40.85005595150629 73.9715201966951) bank46952 +46953 POINT(41.26384918377411 74.01040830223694) bank46953 +46954 POINT(40.2141930088367 73.27942077160303) bank46954 +46955 POINT(39.88292547293735 73.8692707848982) bank46955 +46956 POINT(39.993903294235615 73.38631504817859) bank46956 +46957 POINT(40.701163508436416 74.89573036384044) bank46957 +46958 POINT(41.125068103601556 73.17721426352765) bank46958 +46959 POINT(41.02755639405729 74.98440947019913) bank46959 +46960 POINT(40.075747785801184 74.77720328361951) bank46960 +46961 POINT(41.10317650320286 73.33831812074561) bank46961 +46962 POINT(40.82194078050241 73.46816387443688) bank46962 +46963 POINT(41.34621365694818 74.15542665097045) bank46963 +46964 POINT(40.005008403693054 74.5913408855413) bank46964 +46965 POINT(39.93295295164649 74.1566200822717) bank46965 +46966 POINT(40.2948485506651 74.09948630590382) bank46966 +46967 POINT(40.863556822808455 73.41086089937478) bank46967 +46968 POINT(40.899932988227846 74.47813812692623) bank46968 +46969 POINT(39.930083412008145 74.27767800114788) bank46969 +46970 POINT(40.908052213998964 73.64857874462807) bank46970 +46971 POINT(40.870408811232465 73.52018308247693) bank46971 +46972 POINT(40.80054278194576 74.32105371051514) bank46972 +46973 POINT(39.976149478879016 73.96510837395732) bank46973 +46974 POINT(41.14318837090062 73.87684813614392) bank46974 +46975 POINT(40.40821344528563 73.95491777564209) bank46975 +46976 POINT(40.21760200561884 73.68032242871638) bank46976 +46977 POINT(40.63669745440527 73.76043815235094) bank46977 +46978 POINT(39.9774864963635 74.18520919095904) bank46978 +46979 POINT(40.37799690804706 73.26316792095842) bank46979 +46980 POINT(40.41023918592791 73.65893627528209) bank46980 +46981 POINT(41.683464764899504 73.81884531365942) bank46981 +46982 POINT(40.152966774291556 73.67104641649438) bank46982 +46983 POINT(40.30899902578659 74.8805161082708) bank46983 +46984 POINT(40.222678070893686 73.75840802608941) bank46984 +46985 POINT(40.41067430710634 73.58760908541372) bank46985 +46986 POINT(39.92934007530761 74.74853911279047) bank46986 +46987 POINT(41.620439693959916 73.2580040284396) bank46987 +46988 POINT(40.00832756184409 74.78501540493166) bank46988 +46989 POINT(40.947743132522234 74.8983621379397) bank46989 +46990 POINT(40.73601708040543 73.5429710645835) bank46990 +46991 POINT(41.41730150419512 74.33859072378834) bank46991 +46992 POINT(40.54606768809745 73.00926336877767) bank46992 +46993 POINT(41.55228086895111 73.83977543413134) bank46993 +46994 POINT(40.57151917873434 73.60864464085358) bank46994 +46995 POINT(39.87279801586089 74.01669195631966) bank46995 +46996 POINT(40.04954793504853 74.29808129504929) bank46996 +46997 POINT(41.17720324243297 74.04321757988794) bank46997 +46998 POINT(41.682996738267605 73.77873704738957) bank46998 +46999 POINT(41.348993208985014 74.4096075250591) bank46999 +47000 POINT(41.51361786072666 74.19648448161536) bank47000 +47001 POINT(40.810121790196845 74.129267561096) bank47001 +47002 POINT(40.226610399759586 73.4488472100901) bank47002 +47003 POINT(40.85368874088341 73.83322551580181) bank47003 +47004 POINT(41.06708085420282 73.40736485636899) bank47004 +47005 POINT(40.10566235645921 73.85078611236345) bank47005 +47006 POINT(41.531369483233675 73.10415053635639) bank47006 +47007 POINT(41.033699985829074 74.77409721837871) bank47007 +47008 POINT(41.447471364828424 74.39308786427262) bank47008 +47009 POINT(40.768115586575945 74.06045873132011) bank47009 +47010 POINT(41.19057562115499 73.37008423944964) bank47010 +47011 POINT(40.9732621461881 74.95126117835939) bank47011 +47012 POINT(40.02510406722273 73.25297580672088) bank47012 +47013 POINT(41.3272177767825 74.90008355709145) bank47013 +47014 POINT(40.7156009708167 73.83494597653934) bank47014 +47015 POINT(40.151008756050004 73.45486250510305) bank47015 +47016 POINT(40.068262152306524 74.65564995678834) bank47016 +47017 POINT(39.96813214862394 73.68930116373758) bank47017 +47018 POINT(41.531584203669716 73.7000471754125) bank47018 +47019 POINT(40.20335453522286 74.65428228391184) bank47019 +47020 POINT(41.436746378620896 74.17114311464479) bank47020 +47021 POINT(40.76581430751348 74.57186321726269) bank47021 +47022 POINT(40.84495590738222 74.58678023240934) bank47022 +47023 POINT(40.83207138281723 74.89746689584986) bank47023 +47024 POINT(41.482459305711416 73.05625492701645) bank47024 +47025 POINT(40.97872166917728 73.17507932902055) bank47025 +47026 POINT(40.257051530611605 73.60001999169152) bank47026 +47027 POINT(41.08068782742186 74.24319925238174) bank47027 +47028 POINT(41.376724677917764 74.99180811105377) bank47028 +47029 POINT(39.786795245855096 73.14873349356498) bank47029 +47030 POINT(41.35816167859472 73.46797335875628) bank47030 +47031 POINT(40.077391226515104 74.52989387373103) bank47031 +47032 POINT(41.1578103630392 73.26807515282006) bank47032 +47033 POINT(39.87520905527518 73.00806462045094) bank47033 +47034 POINT(40.70690762840364 73.40352973859103) bank47034 +47035 POINT(40.06497217857293 74.9514422942454) bank47035 +47036 POINT(41.34331457431283 74.68087353987778) bank47036 +47037 POINT(40.28794007431543 74.95273790438253) bank47037 +47038 POINT(41.4139870297032 73.01430388538813) bank47038 +47039 POINT(40.51211484096852 73.7995658600443) bank47039 +47040 POINT(40.968417018922466 73.16253526994939) bank47040 +47041 POINT(40.69977609621852 74.62073573711906) bank47041 +47042 POINT(40.61420785761141 73.32065672185705) bank47042 +47043 POINT(40.165649965190774 74.55124370639831) bank47043 +47044 POINT(40.70768600700739 74.32990116629897) bank47044 +47045 POINT(41.25601661999069 73.01098852789056) bank47045 +47046 POINT(40.27896562103753 73.63097909035487) bank47046 +47047 POINT(39.94168222085387 73.55833011205841) bank47047 +47048 POINT(41.24405883339379 73.87693253226304) bank47048 +47049 POINT(40.21768058593764 73.09237439961228) bank47049 +47050 POINT(40.92050624668366 73.6954454992381) bank47050 +47051 POINT(40.478971097140764 73.04014714958659) bank47051 +47052 POINT(41.69139201628154 73.95920351796256) bank47052 +47053 POINT(41.700228248984644 74.64525691845975) bank47053 +47054 POINT(40.34124167234157 73.01282439804386) bank47054 +47055 POINT(39.960867063373485 73.65642595803685) bank47055 +47056 POINT(39.96293130966404 73.21538870181652) bank47056 +47057 POINT(40.485085866647715 74.07505742062959) bank47057 +47058 POINT(40.96149694295901 74.5298854090116) bank47058 +47059 POINT(39.93380014965028 73.46970028913046) bank47059 +47060 POINT(41.5199952731683 73.62154247988799) bank47060 +47061 POINT(41.28043420593963 73.54551601354665) bank47061 +47062 POINT(41.286863786833194 73.88204364831795) bank47062 +47063 POINT(40.865848163230254 74.18445073249696) bank47063 +47064 POINT(40.92776623627622 73.85942309704579) bank47064 +47065 POINT(41.18210196751957 74.63180603750267) bank47065 +47066 POINT(41.312707740334204 73.16679609237403) bank47066 +47067 POINT(41.32423726800948 74.80964724536592) bank47067 +47068 POINT(41.288896239429235 73.74460109966016) bank47068 +47069 POINT(40.44081007310341 74.68443802163286) bank47069 +47070 POINT(40.84165246019605 73.0359533389762) bank47070 +47071 POINT(41.695573054168136 73.21008014849723) bank47071 +47072 POINT(40.67351831241425 73.63818872126777) bank47072 +47073 POINT(41.43887506768154 74.60704388960251) bank47073 +47074 POINT(41.637975262839 74.59302161421888) bank47074 +47075 POINT(41.63294831833456 74.54200201650552) bank47075 +47076 POINT(40.72188372431153 73.99124603867254) bank47076 +47077 POINT(40.480722675191046 74.11116883857358) bank47077 +47078 POINT(40.40896215980393 73.63696851423357) bank47078 +47079 POINT(40.390548736221426 73.43701685266444) bank47079 +47080 POINT(40.59085455723885 73.30961650288391) bank47080 +47081 POINT(41.229549121037905 73.5827955283886) bank47081 +47082 POINT(40.69357209830337 73.68728591311853) bank47082 +47083 POINT(40.765066010979595 73.41827366085639) bank47083 +47084 POINT(41.163409654689914 74.35463404594422) bank47084 +47085 POINT(41.48173892102972 74.07650037380769) bank47085 +47086 POINT(41.16878948899329 73.01051107746574) bank47086 +47087 POINT(41.068155561268135 73.15279017377921) bank47087 +47088 POINT(41.27985641129044 73.85669346671837) bank47088 +47089 POINT(41.687600363075354 74.18326596284079) bank47089 +47090 POINT(39.91357602826699 74.06257855786448) bank47090 +47091 POINT(40.22237680964983 74.96918352294513) bank47091 +47092 POINT(41.07441988465583 73.08595382960354) bank47092 +47093 POINT(41.03215286648073 73.58840532330133) bank47093 +47094 POINT(40.26845697040195 73.74200244695687) bank47094 +47095 POINT(39.903314874529265 74.585388146134) bank47095 +47096 POINT(41.635252603246386 74.3933911617527) bank47096 +47097 POINT(40.1073221861867 74.03067295428995) bank47097 +47098 POINT(40.2483445156685 73.9488770213364) bank47098 +47099 POINT(41.10303316656155 74.1211340874695) bank47099 +47100 POINT(41.42323850908283 73.71488190825572) bank47100 +47101 POINT(40.647992022450595 73.65496215469993) bank47101 +47102 POINT(40.84070276271295 73.32366457195661) bank47102 +47103 POINT(40.03584955723978 73.84478415831605) bank47103 +47104 POINT(40.25057980405239 74.1551172267309) bank47104 +47105 POINT(40.94983204442666 74.75838717174051) bank47105 +47106 POINT(41.689114680949096 74.47849000660298) bank47106 +47107 POINT(40.39296031117293 74.9280873991821) bank47107 +47108 POINT(41.390999486499126 73.65694193401184) bank47108 +47109 POINT(40.404646775280305 73.27254504606785) bank47109 +47110 POINT(41.08482623722076 73.40830665238492) bank47110 +47111 POINT(41.30950613138339 73.15875097791725) bank47111 +47112 POINT(40.931121751874485 74.61867931567858) bank47112 +47113 POINT(41.46473243328521 74.66241832717661) bank47113 +47114 POINT(41.31455186399848 73.87412257071315) bank47114 +47115 POINT(40.198803230182826 73.47940345037804) bank47115 +47116 POINT(40.31222585930527 74.66099494833131) bank47116 +47117 POINT(40.81881196275966 74.05265387864775) bank47117 +47118 POINT(41.65211819913665 74.94704951712983) bank47118 +47119 POINT(40.93064695945001 73.37651145405316) bank47119 +47120 POINT(39.87981486750687 74.4367023329962) bank47120 +47121 POINT(39.927148133705444 73.26779706556083) bank47121 +47122 POINT(40.662870887466525 73.33250093897828) bank47122 +47123 POINT(41.252215788433745 74.26154879518411) bank47123 +47124 POINT(41.60376278976862 74.71339601577698) bank47124 +47125 POINT(40.76090890725945 74.55547578140212) bank47125 +47126 POINT(40.53056673489423 74.83040643163409) bank47126 +47127 POINT(39.727031616884204 73.4508076431494) bank47127 +47128 POINT(40.4182923123818 73.74197551970286) bank47128 +47129 POINT(40.91370331999388 74.44148177950774) bank47129 +47130 POINT(41.3992343845755 74.75131015150176) bank47130 +47131 POINT(39.72044239188471 73.46396162962803) bank47131 +47132 POINT(41.23246016618283 74.26545500303817) bank47132 +47133 POINT(40.031244645506455 73.77897537121254) bank47133 +47134 POINT(40.237945485519006 74.62907610991795) bank47134 +47135 POINT(40.578490576532666 73.94326608028719) bank47135 +47136 POINT(40.533658015259284 73.96340773037552) bank47136 +47137 POINT(40.91617095366124 73.75403211202021) bank47137 +47138 POINT(40.717888855393454 74.13123065680827) bank47138 +47139 POINT(41.46052879011333 74.45222340972867) bank47139 +47140 POINT(40.163507458582565 74.88389452927403) bank47140 +47141 POINT(41.48971129861931 74.18372529593411) bank47141 +47142 POINT(40.039821157612494 74.48755484194325) bank47142 +47143 POINT(41.31809322414602 73.8614929263553) bank47143 +47144 POINT(41.649111018082415 73.04259340323902) bank47144 +47145 POINT(40.08119207537859 73.15182854863492) bank47145 +47146 POINT(41.443416903883 73.9404276852634) bank47146 +47147 POINT(40.69928392446043 74.28413494406837) bank47147 +47148 POINT(41.28759544867481 73.91018121065406) bank47148 +47149 POINT(41.09173442147838 74.33709395627191) bank47149 +47150 POINT(41.66087720428116 74.71706882018921) bank47150 +47151 POINT(41.34790400003877 74.27300468180327) bank47151 +47152 POINT(41.22309762050831 74.89836509350397) bank47152 +47153 POINT(41.41204623564251 74.1332548475087) bank47153 +47154 POINT(41.56740727226063 74.6681054557812) bank47154 +47155 POINT(40.04565909120291 74.76933084941219) bank47155 +47156 POINT(40.74286262684881 74.22334829365337) bank47156 +47157 POINT(41.454977588281295 73.67143086942507) bank47157 +47158 POINT(40.10447772653929 74.24096077245323) bank47158 +47159 POINT(41.52252277510393 74.34181036195346) bank47159 +47160 POINT(40.18135664001243 73.36979458794693) bank47160 +47161 POINT(39.73201599627726 74.543794724043) bank47161 +47162 POINT(40.638386115205215 73.83652039476578) bank47162 +47163 POINT(41.40612190164888 73.72247439200383) bank47163 +47164 POINT(41.37878069119435 74.54098635095524) bank47164 +47165 POINT(40.62273521188449 74.65293120482842) bank47165 +47166 POINT(40.16792530466796 73.50553250439532) bank47166 +47167 POINT(40.73099230481183 74.81918178736964) bank47167 +47168 POINT(41.24333426913499 74.98336920122307) bank47168 +47169 POINT(40.11596933987987 73.08673382208531) bank47169 +47170 POINT(40.47957152218355 74.04250397385373) bank47170 +47171 POINT(39.751677189966976 74.90819928081547) bank47171 +47172 POINT(41.632764247775555 73.97899922291137) bank47172 +47173 POINT(40.76143233864284 73.25887439755613) bank47173 +47174 POINT(40.58256507005553 73.22819503197078) bank47174 +47175 POINT(40.240779914097 74.64126784998999) bank47175 +47176 POINT(41.300496470559885 73.625488538223) bank47176 +47177 POINT(39.77871288492984 73.21850241369324) bank47177 +47178 POINT(40.07276334606748 74.39860961353476) bank47178 +47179 POINT(40.86003462197159 73.23635105173759) bank47179 +47180 POINT(40.74629736038373 74.33544821386808) bank47180 +47181 POINT(41.25380291592803 73.42865035381188) bank47181 +47182 POINT(41.70370330380525 74.25828960513884) bank47182 +47183 POINT(40.22341862665176 73.92023500993297) bank47183 +47184 POINT(41.015595631525244 73.36259104095787) bank47184 +47185 POINT(41.490163125345234 73.37109137055592) bank47185 +47186 POINT(40.2035761963219 74.11165675728077) bank47186 +47187 POINT(41.61012622737638 73.68168716526692) bank47187 +47188 POINT(41.561974079984275 73.83455121150395) bank47188 +47189 POINT(39.943229322902944 73.89985890713619) bank47189 +47190 POINT(40.19231098596685 74.81629565741612) bank47190 +47191 POINT(39.89022008352848 74.06005646480105) bank47191 +47192 POINT(40.00207842031468 74.79397739202585) bank47192 +47193 POINT(41.196637221772455 74.71893945393391) bank47193 +47194 POINT(41.40390550258013 74.04345343033751) bank47194 +47195 POINT(40.91497717676583 74.43908947769687) bank47195 +47196 POINT(39.98661710887711 74.09576585958027) bank47196 +47197 POINT(40.00685491009352 74.23851593683268) bank47197 +47198 POINT(39.71953093795363 74.03314110259748) bank47198 +47199 POINT(40.49217703367317 73.76798889046452) bank47199 +47200 POINT(40.076018701268964 74.97971034170027) bank47200 +47201 POINT(39.71303399295031 74.1105301438053) bank47201 +47202 POINT(40.39565878975132 73.71670931048513) bank47202 +47203 POINT(39.78929780334016 74.79734761331392) bank47203 +47204 POINT(40.938623364919145 73.73351095745426) bank47204 +47205 POINT(41.077250479268756 74.87083002925414) bank47205 +47206 POINT(40.66502454770108 73.10707396030588) bank47206 +47207 POINT(41.43912483863464 73.93883698337682) bank47207 +47208 POINT(40.29042107847353 73.10188324320748) bank47208 +47209 POINT(40.21576058437288 74.76774228882806) bank47209 +47210 POINT(41.154691392781444 74.19572807499891) bank47210 +47211 POINT(40.038890302827596 73.53470242123834) bank47211 +47212 POINT(39.90985512133718 74.58961950096254) bank47212 +47213 POINT(40.68253899979801 74.67065236644744) bank47213 +47214 POINT(41.59222535039885 74.5708159839055) bank47214 +47215 POINT(39.83426616790387 74.87090023621157) bank47215 +47216 POINT(39.76077523497072 73.10869274566139) bank47216 +47217 POINT(41.58536378722624 73.12380960244892) bank47217 +47218 POINT(39.8918676872903 74.61801268884686) bank47218 +47219 POINT(41.2938043470037 73.79033677044008) bank47219 +47220 POINT(40.23999672918647 74.75533098433563) bank47220 +47221 POINT(39.748542801603456 73.78800872422069) bank47221 +47222 POINT(40.33224787295715 73.05772466849574) bank47222 +47223 POINT(39.86241534055745 73.49013384086109) bank47223 +47224 POINT(40.66918998164203 74.42325618028245) bank47224 +47225 POINT(40.451164146102975 73.056843234189) bank47225 +47226 POINT(39.76605817839358 73.95107282698837) bank47226 +47227 POINT(41.086762107670744 74.3364551762597) bank47227 +47228 POINT(41.47519391952712 73.7354121945109) bank47228 +47229 POINT(40.14517776054719 74.64602408537225) bank47229 +47230 POINT(40.95778489392981 74.57848985856768) bank47230 +47231 POINT(39.858731679433376 74.94434352039366) bank47231 +47232 POINT(39.72410671841369 74.2755442864485) bank47232 +47233 POINT(41.657134532394124 73.18563670667528) bank47233 +47234 POINT(40.089774159127664 73.06480333947603) bank47234 +47235 POINT(40.27512132007809 74.80616466133543) bank47235 +47236 POINT(40.58805957395964 73.62212961170128) bank47236 +47237 POINT(40.76685586940246 73.26973401801929) bank47237 +47238 POINT(40.8727343037658 74.62483832630939) bank47238 +47239 POINT(40.95615238580469 74.47987069653323) bank47239 +47240 POINT(40.38532906783373 73.64670052261815) bank47240 +47241 POINT(40.388969650312546 74.8502617057694) bank47241 +47242 POINT(41.52056021941897 74.00313073529807) bank47242 +47243 POINT(40.911623934296465 74.7451818343866) bank47243 +47244 POINT(41.32984661204784 74.88270279332136) bank47244 +47245 POINT(40.360628612896996 74.74247296480624) bank47245 +47246 POINT(41.43793550820509 73.99467792601563) bank47246 +47247 POINT(39.78287215911458 74.21956096158263) bank47247 +47248 POINT(40.038895225418834 74.9466867236836) bank47248 +47249 POINT(41.44504285268424 74.32998826811416) bank47249 +47250 POINT(40.6898865227416 73.07606951814975) bank47250 +47251 POINT(40.16693129585545 73.41017870260615) bank47251 +47252 POINT(41.43224235610348 74.85776300176637) bank47252 +47253 POINT(41.446893022439255 74.10580722092521) bank47253 +47254 POINT(41.37888853458696 73.0869449981022) bank47254 +47255 POINT(41.654895609134535 73.02259140039767) bank47255 +47256 POINT(41.275400096767115 73.2617813661472) bank47256 +47257 POINT(40.55326035104335 73.02997324791896) bank47257 +47258 POINT(40.296849722535576 74.11761282537319) bank47258 +47259 POINT(39.98960651110434 75.00537542125767) bank47259 +47260 POINT(40.43483028669826 74.3650848764352) bank47260 +47261 POINT(40.04521786231096 74.25446861630151) bank47261 +47262 POINT(40.359214338113276 73.28134981514357) bank47262 +47263 POINT(41.545272430514885 73.30031295545474) bank47263 +47264 POINT(41.22688524817796 73.00807608436926) bank47264 +47265 POINT(40.029941913015435 74.38356701072317) bank47265 +47266 POINT(39.841992650394836 73.5535967445189) bank47266 +47267 POINT(41.13758385044692 73.05783911755032) bank47267 +47268 POINT(40.63091822067211 74.43560006780828) bank47268 +47269 POINT(40.25029146380108 74.47157188965518) bank47269 +47270 POINT(40.93715225662091 74.33230986559167) bank47270 +47271 POINT(41.57935470103025 74.44360927355318) bank47271 +47272 POINT(41.61378102260622 73.64046693613729) bank47272 +47273 POINT(41.65633509781476 73.2334297683964) bank47273 +47274 POINT(40.0451240236218 73.65606460718304) bank47274 +47275 POINT(39.79690948492742 74.25085472571028) bank47275 +47276 POINT(41.19163136611683 73.81945664900485) bank47276 +47277 POINT(41.489173034371234 73.29077683419625) bank47277 +47278 POINT(40.30891744514298 74.7839271004847) bank47278 +47279 POINT(40.52117575922714 73.44568191781924) bank47279 +47280 POINT(40.344300133592334 73.84420070362451) bank47280 +47281 POINT(40.55579440795773 73.52111874447353) bank47281 +47282 POINT(40.6851367884912 73.96113723962053) bank47282 +47283 POINT(41.60265319492131 74.91072213862219) bank47283 +47284 POINT(40.30001039199234 74.23511899570542) bank47284 +47285 POINT(39.94521113728373 74.20760979223961) bank47285 +47286 POINT(41.682518569969 73.80569595445323) bank47286 +47287 POINT(40.45962403653299 74.6851921175369) bank47287 +47288 POINT(39.862494678090215 73.36146430200213) bank47288 +47289 POINT(40.48766480872691 74.6395368728168) bank47289 +47290 POINT(39.85243214558935 73.19224309609602) bank47290 +47291 POINT(40.894205036967776 74.2030709732452) bank47291 +47292 POINT(41.23175226354924 73.89192819894254) bank47292 +47293 POINT(40.37854592309461 73.65826324746918) bank47293 +47294 POINT(41.113802686010494 73.66961940278223) bank47294 +47295 POINT(41.68717510917592 74.69439553569082) bank47295 +47296 POINT(40.76223052422529 74.39251077418733) bank47296 +47297 POINT(40.04081531371404 74.4338251830154) bank47297 +47298 POINT(40.60212546678518 74.94750601043377) bank47298 +47299 POINT(40.0792465669258 74.00046602900031) bank47299 +47300 POINT(40.36168677794855 73.81608992621824) bank47300 +47301 POINT(41.29753068433784 74.5665483117797) bank47301 +47302 POINT(41.25239716821878 73.1645440020371) bank47302 +47303 POINT(41.11760085864634 74.60038431987287) bank47303 +47304 POINT(40.37566538573916 73.3506236984892) bank47304 +47305 POINT(40.14330998252731 73.0852511743713) bank47305 +47306 POINT(40.1690335354983 73.53993537124326) bank47306 +47307 POINT(40.657508496213445 74.85593944669) bank47307 +47308 POINT(41.29488314266481 73.60029669019305) bank47308 +47309 POINT(40.781817435978354 74.85672499734635) bank47309 +47310 POINT(40.9365897027059 74.27302175151016) bank47310 +47311 POINT(40.978539939666355 73.22050888655922) bank47311 +47312 POINT(41.476105537368944 74.96081354580922) bank47312 +47313 POINT(40.89446776915054 74.32240551084116) bank47313 +47314 POINT(41.50293214677073 74.45401041033857) bank47314 +47315 POINT(41.60349421683238 73.68057257965816) bank47315 +47316 POINT(39.960240603440866 74.46494653376972) bank47316 +47317 POINT(41.41526156256206 73.52783605964427) bank47317 +47318 POINT(41.65998678611729 74.77964670729645) bank47318 +47319 POINT(40.34337634847834 73.56416883841715) bank47319 +47320 POINT(41.574734333758265 74.9978376562791) bank47320 +47321 POINT(40.31209911213036 74.98229525828404) bank47321 +47322 POINT(41.12199815745721 73.5470130907152) bank47322 +47323 POINT(41.08414092374431 74.8101256023225) bank47323 +47324 POINT(41.294075189030124 74.66636650921369) bank47324 +47325 POINT(40.93197471142241 73.80612942959303) bank47325 +47326 POINT(41.5103205096879 74.99995216211988) bank47326 +47327 POINT(40.64781264989191 74.5571600647653) bank47327 +47328 POINT(41.067302924705004 73.10716852040724) bank47328 +47329 POINT(40.58357786960206 74.42883646057041) bank47329 +47330 POINT(41.28228429865797 73.80369146692121) bank47330 +47331 POINT(41.465208628554734 73.28922536693993) bank47331 +47332 POINT(40.14270918616091 74.88703053975628) bank47332 +47333 POINT(39.819891084368834 73.76831065728999) bank47333 +47334 POINT(41.64793728219433 73.9709931935759) bank47334 +47335 POINT(40.19189341686125 73.86907817951537) bank47335 +47336 POINT(40.37294426802789 74.0959918486198) bank47336 +47337 POINT(40.27375317227032 74.8144523221501) bank47337 +47338 POINT(40.099850872561525 74.61129157197625) bank47338 +47339 POINT(40.00613322486399 74.80154018321727) bank47339 +47340 POINT(40.56374517588682 74.72643966562183) bank47340 +47341 POINT(41.70254072395779 74.67493094290631) bank47341 +47342 POINT(39.960864006232335 74.15687510327402) bank47342 +47343 POINT(40.66294345543255 74.46053515665814) bank47343 +47344 POINT(40.991326265503936 73.68899185081052) bank47344 +47345 POINT(40.33457094530301 74.24816283041703) bank47345 +47346 POINT(39.796809780483436 74.92584784499219) bank47346 +47347 POINT(40.420642452365385 73.711682181195) bank47347 +47348 POINT(40.9528509768364 74.62613524122507) bank47348 +47349 POINT(41.66301682865211 73.38329923878881) bank47349 +47350 POINT(40.31623281544049 74.69204984037094) bank47350 +47351 POINT(40.158559241441914 74.48707702135812) bank47351 +47352 POINT(40.58907951704331 74.92627068747747) bank47352 +47353 POINT(41.164303870355596 74.71590369649196) bank47353 +47354 POINT(40.26063933038096 73.9594937670676) bank47354 +47355 POINT(40.26870300900733 74.6993871010268) bank47355 +47356 POINT(40.56530444881814 74.27861659836694) bank47356 +47357 POINT(39.90307660542813 73.48350978143226) bank47357 +47358 POINT(39.8884339048473 74.80289584026866) bank47358 +47359 POINT(41.51227115215878 74.06657646708592) bank47359 +47360 POINT(40.96813838138403 74.38238117247222) bank47360 +47361 POINT(40.267518255109614 74.17316072310557) bank47361 +47362 POINT(41.40556539050223 74.56971588126278) bank47362 +47363 POINT(40.56602360501704 73.80635433591854) bank47363 +47364 POINT(40.522276325150465 73.30515306522899) bank47364 +47365 POINT(40.00474305509642 73.2487646638016) bank47365 +47366 POINT(40.565705563228896 74.25719732507137) bank47366 +47367 POINT(41.47908525237747 73.7977183770779) bank47367 +47368 POINT(41.2587623860282 74.97785667475871) bank47368 +47369 POINT(41.03924369637577 73.95346214059506) bank47369 +47370 POINT(40.6883965649734 74.73152292695096) bank47370 +47371 POINT(41.26752631881364 73.93382983803303) bank47371 +47372 POINT(40.33493647936599 73.6024735992275) bank47372 +47373 POINT(40.32747150675473 74.8697059504635) bank47373 +47374 POINT(40.93768813138352 74.71796143003247) bank47374 +47375 POINT(39.893340399950716 74.94681396071309) bank47375 +47376 POINT(41.660559102280345 73.86419515907656) bank47376 +47377 POINT(40.5476248837536 73.06094280661016) bank47377 +47378 POINT(41.018817183731585 73.94831147062582) bank47378 +47379 POINT(40.51206669935689 73.48214339921468) bank47379 +47380 POINT(39.88747210925396 73.80149827491624) bank47380 +47381 POINT(41.65596120297573 73.92993113756755) bank47381 +47382 POINT(40.7612879450582 73.2646105581179) bank47382 +47383 POINT(41.27582235750358 73.87743929592303) bank47383 +47384 POINT(40.42988135033435 74.28219527853913) bank47384 +47385 POINT(39.75253805922294 73.3223067517494) bank47385 +47386 POINT(40.226237603363096 73.41442909024235) bank47386 +47387 POINT(41.30918823771306 74.73864666527629) bank47387 +47388 POINT(41.58997556433633 73.57704715641444) bank47388 +47389 POINT(41.02281003391969 73.33056360717825) bank47389 +47390 POINT(41.03479231477478 74.82913993707056) bank47390 +47391 POINT(41.29739047599603 74.68967685871708) bank47391 +47392 POINT(40.94989726376271 74.08070105623024) bank47392 +47393 POINT(41.03547836351765 73.90943620231268) bank47393 +47394 POINT(41.198176887770344 74.47182491709165) bank47394 +47395 POINT(40.95824666515198 74.84791914960843) bank47395 +47396 POINT(41.12430841564145 73.64756731970186) bank47396 +47397 POINT(41.571882414295395 73.98536346211246) bank47397 +47398 POINT(41.16271166779022 73.0391157741682) bank47398 +47399 POINT(41.486162354675635 73.29215048057752) bank47399 +47400 POINT(39.74804145626896 74.82173539595351) bank47400 +47401 POINT(41.17162140884595 73.90765525483266) bank47401 +47402 POINT(41.142542187116135 74.04826177772733) bank47402 +47403 POINT(39.87495356480509 73.34745592547873) bank47403 +47404 POINT(41.666392629792455 73.36788776538636) bank47404 +47405 POINT(41.3305223514468 74.59641568472637) bank47405 +47406 POINT(41.40972632296123 74.98205606095598) bank47406 +47407 POINT(39.85907346846589 73.08574535662443) bank47407 +47408 POINT(39.912597513606485 73.22125169048049) bank47408 +47409 POINT(40.569145182466244 73.69237034583666) bank47409 +47410 POINT(40.67847089836579 74.64433106941067) bank47410 +47411 POINT(41.15211778179719 74.47488490480299) bank47411 +47412 POINT(40.44950344195579 74.687102264886) bank47412 +47413 POINT(39.883522287045444 74.06493896950877) bank47413 +47414 POINT(40.50389325974881 73.28975790272288) bank47414 +47415 POINT(40.19175641886981 74.14907898239106) bank47415 +47416 POINT(41.01893290589492 73.58803710542219) bank47416 +47417 POINT(40.784504178344484 74.25991548453626) bank47417 +47418 POINT(40.185564792335434 74.74956190401261) bank47418 +47419 POINT(41.688352442887805 74.58221564958113) bank47419 +47420 POINT(39.947561261919695 74.91707247134315) bank47420 +47421 POINT(40.7877450224168 74.74290381129177) bank47421 +47422 POINT(41.17499633438879 73.59164769582688) bank47422 +47423 POINT(41.119983451265185 73.97085888085456) bank47423 +47424 POINT(39.77808284342067 74.28623725050743) bank47424 +47425 POINT(41.044412466206516 74.89559282386028) bank47425 +47426 POINT(40.11011827453293 74.83541263534579) bank47426 +47427 POINT(40.764404298773584 73.32236400531852) bank47427 +47428 POINT(41.36754031620217 74.58497694079658) bank47428 +47429 POINT(40.69649111390431 73.40394446480349) bank47429 +47430 POINT(41.0634954896982 73.19761106959048) bank47430 +47431 POINT(40.188606636656836 73.39290058469993) bank47431 +47432 POINT(40.235259085801715 74.78106254043091) bank47432 +47433 POINT(40.243345743957086 73.44301327499994) bank47433 +47434 POINT(40.196322771897535 74.74106333663524) bank47434 +47435 POINT(41.45912496722567 73.31848130866682) bank47435 +47436 POINT(39.732279864418146 74.96057075809549) bank47436 +47437 POINT(40.559286730064 73.06398039639122) bank47437 +47438 POINT(41.670203973722664 74.07364875962176) bank47438 +47439 POINT(39.98481545730061 74.51451688524706) bank47439 +47440 POINT(41.28663838796556 73.85430991257938) bank47440 +47441 POINT(40.53825932935571 73.86376377870151) bank47441 +47442 POINT(41.65991286399078 73.96170274126608) bank47442 +47443 POINT(40.57501921717354 73.66299387475803) bank47443 +47444 POINT(39.934379666539094 73.4304586181065) bank47444 +47445 POINT(40.83327511139444 74.74802150439747) bank47445 +47446 POINT(41.43890357289853 73.87733994404503) bank47446 +47447 POINT(40.1617964504721 74.51608442054992) bank47447 +47448 POINT(40.41087134188467 73.81578990503228) bank47448 +47449 POINT(41.19009085252886 73.59813874096703) bank47449 +47450 POINT(40.60688273890414 74.43378826649496) bank47450 +47451 POINT(40.12979987682763 74.5320357511604) bank47451 +47452 POINT(40.97451472705889 73.59310692224422) bank47452 +47453 POINT(40.65243099687871 73.65003258979796) bank47453 +47454 POINT(40.043009255377974 74.64426160990654) bank47454 +47455 POINT(40.61615099905124 73.21958643866661) bank47455 +47456 POINT(39.826913320356546 74.31523136677392) bank47456 +47457 POINT(40.19739492134323 73.1043952043936) bank47457 +47458 POINT(41.41345938403514 74.57641387447774) bank47458 +47459 POINT(41.56660919977836 74.48959710126076) bank47459 +47460 POINT(40.056387639788134 74.66183092958083) bank47460 +47461 POINT(41.670195512543955 73.2544645462049) bank47461 +47462 POINT(40.780564728192275 74.51340543587277) bank47462 +47463 POINT(41.529447306191614 73.99409931850559) bank47463 +47464 POINT(40.6116439617378 73.22688829010475) bank47464 +47465 POINT(40.872294377390844 73.15332473720673) bank47465 +47466 POINT(40.51956878134108 73.35365111470773) bank47466 +47467 POINT(41.046020089833405 74.44243658655913) bank47467 +47468 POINT(40.53518138268367 74.58991666622151) bank47468 +47469 POINT(41.219239867943756 73.96507051304681) bank47469 +47470 POINT(41.44025799422131 73.59132156799825) bank47470 +47471 POINT(41.215296724291065 73.02396795046354) bank47471 +47472 POINT(41.36005774802535 74.30243105118691) bank47472 +47473 POINT(41.144873080038 73.88641575643227) bank47473 +47474 POINT(40.58334251965202 73.12305655405338) bank47474 +47475 POINT(40.375380528807405 74.55771098660279) bank47475 +47476 POINT(41.55312556112788 74.3044442137184) bank47476 +47477 POINT(40.07138101787701 74.99812291529604) bank47477 +47478 POINT(39.999665545757026 74.5368536956875) bank47478 +47479 POINT(39.86284562736777 73.52536819918994) bank47479 +47480 POINT(40.732885549322624 74.18311397483608) bank47480 +47481 POINT(40.96200493026324 74.31222755077762) bank47481 +47482 POINT(40.99499920834948 74.42384984630519) bank47482 +47483 POINT(40.73173850213084 74.4469404478875) bank47483 +47484 POINT(41.39519260129254 73.39600976471178) bank47484 +47485 POINT(41.04996096711779 73.23993753514439) bank47485 +47486 POINT(40.545444570801806 74.53513875905125) bank47486 +47487 POINT(40.24366450838339 73.05062369512105) bank47487 +47488 POINT(40.08516729750219 73.45848653006712) bank47488 +47489 POINT(40.755321900235394 73.81233645815489) bank47489 +47490 POINT(41.21432475397745 74.42212471806185) bank47490 +47491 POINT(40.27784952956641 74.87979403894502) bank47491 +47492 POINT(40.515257166237674 73.24970528271226) bank47492 +47493 POINT(40.602128331242476 73.16351993135105) bank47493 +47494 POINT(40.643669253828826 73.05313035451277) bank47494 +47495 POINT(40.45901918142404 74.82520215072599) bank47495 +47496 POINT(40.49615010763144 74.63747409250131) bank47496 +47497 POINT(40.12616152256948 74.65562294365255) bank47497 +47498 POINT(40.619544398158 73.97744623064517) bank47498 +47499 POINT(41.13513730654429 73.86048399063375) bank47499 +47500 POINT(40.23299516199059 73.13601089696601) bank47500 +47501 POINT(41.310340948893064 74.91685758789623) bank47501 +47502 POINT(39.98896662384865 74.00843628798292) bank47502 +47503 POINT(40.287272772961295 74.79397526106207) bank47503 +47504 POINT(40.37768132348829 73.04879764546457) bank47504 +47505 POINT(40.28794815648647 73.32236920130714) bank47505 +47506 POINT(39.99590376023643 73.44107538511159) bank47506 +47507 POINT(40.859890956456766 74.01308218330738) bank47507 +47508 POINT(41.53030974680157 73.42799325072869) bank47508 +47509 POINT(41.08101969854755 74.88615464672) bank47509 +47510 POINT(40.60028704015387 74.11942717586736) bank47510 +47511 POINT(40.47315004237362 74.75767428611634) bank47511 +47512 POINT(40.58635951942847 74.26361391159206) bank47512 +47513 POINT(39.74698113090976 73.305993777499) bank47513 +47514 POINT(41.464559001211455 74.33881943167916) bank47514 +47515 POINT(40.09811505393316 73.1198622369775) bank47515 +47516 POINT(40.505981337006475 73.15335207046013) bank47516 +47517 POINT(41.29995334273363 74.3048758637018) bank47517 +47518 POINT(40.88590434838894 73.1007077686369) bank47518 +47519 POINT(40.248859550156595 73.03836712569054) bank47519 +47520 POINT(41.25366374504357 74.29073410793838) bank47520 +47521 POINT(41.232150759433736 73.17476356685842) bank47521 +47522 POINT(39.84350963734293 74.5814403408175) bank47522 +47523 POINT(41.25755289991077 74.05026460428554) bank47523 +47524 POINT(39.780241730973685 73.86165725276037) bank47524 +47525 POINT(39.75948629175631 74.06472761427126) bank47525 +47526 POINT(40.223558640221455 73.724231053515) bank47526 +47527 POINT(41.183635786067 73.10052286787317) bank47527 +47528 POINT(41.658435523249274 73.59351671325312) bank47528 +47529 POINT(40.91517440090841 73.74683137127973) bank47529 +47530 POINT(41.6593521554205 73.22413700069407) bank47530 +47531 POINT(41.02196001423787 73.56025627163349) bank47531 +47532 POINT(41.29499353127369 74.7408779583037) bank47532 +47533 POINT(40.505928367395526 74.86281315231483) bank47533 +47534 POINT(40.39507562669091 73.08594892111904) bank47534 +47535 POINT(39.844327629307934 73.23625321898754) bank47535 +47536 POINT(39.990692165798 74.08612140929749) bank47536 +47537 POINT(40.34255993113408 73.82917247590912) bank47537 +47538 POINT(40.27611480882483 73.39356850287085) bank47538 +47539 POINT(41.065931766968504 73.79742909094871) bank47539 +47540 POINT(40.84329762348643 74.96185582516439) bank47540 +47541 POINT(41.050513946625834 74.35624047219846) bank47541 +47542 POINT(41.671691028431084 73.8198485319034) bank47542 +47543 POINT(39.73779962616379 74.55074222216125) bank47543 +47544 POINT(41.653718758064784 73.18199663147911) bank47544 +47545 POINT(41.35077929391777 74.57208405713095) bank47545 +47546 POINT(39.758324358851105 74.24713505272867) bank47546 +47547 POINT(40.3877935150212 73.28849087717579) bank47547 +47548 POINT(41.02979615699997 73.5883265028725) bank47548 +47549 POINT(40.74325014045953 73.15502200624141) bank47549 +47550 POINT(40.28021978617691 74.59342134736762) bank47550 +47551 POINT(40.71029263714666 73.22887211359992) bank47551 +47552 POINT(40.80952420216631 74.14252581265163) bank47552 +47553 POINT(40.480379841911926 74.07073975948491) bank47553 +47554 POINT(40.99266062153459 73.97027574017895) bank47554 +47555 POINT(41.60920567505246 73.65077781411729) bank47555 +47556 POINT(40.323436110776676 74.9201346563214) bank47556 +47557 POINT(40.28032904819942 73.23412318647917) bank47557 +47558 POINT(41.03495765095399 73.64230282010725) bank47558 +47559 POINT(41.50915267564842 74.34900919814494) bank47559 +47560 POINT(40.18329089054826 74.94300677421037) bank47560 +47561 POINT(40.85633641943497 73.68850749917175) bank47561 +47562 POINT(40.6618778666933 73.71012001187829) bank47562 +47563 POINT(41.633968134328555 73.01747117559894) bank47563 +47564 POINT(40.87272698804128 73.09575651129931) bank47564 +47565 POINT(40.37874149787108 73.8854144778588) bank47565 +47566 POINT(40.397044102450046 74.15189141224042) bank47566 +47567 POINT(40.9954050619017 74.26081258643009) bank47567 +47568 POINT(41.532885735243084 73.15328755710448) bank47568 +47569 POINT(41.499415692107505 73.68211179385776) bank47569 +47570 POINT(41.082999788955235 74.22566690815329) bank47570 +47571 POINT(40.79998047519809 73.4835955508289) bank47571 +47572 POINT(41.635903970926464 73.54633856982593) bank47572 +47573 POINT(39.81562197671859 74.1457983448916) bank47573 +47574 POINT(40.73278708277163 74.17128009489254) bank47574 +47575 POINT(40.279890917489745 73.32253659093882) bank47575 +47576 POINT(40.93783658145942 74.21589735993886) bank47576 +47577 POINT(40.723213228186665 73.58495712945448) bank47577 +47578 POINT(40.23397739856964 73.54274833607825) bank47578 +47579 POINT(40.32237225286895 73.3597693186672) bank47579 +47580 POINT(41.091538242280826 74.57568767750617) bank47580 +47581 POINT(40.03599355220268 73.60479967420014) bank47581 +47582 POINT(40.59708666130726 74.56798914202984) bank47582 +47583 POINT(41.06804115583527 74.83827073870273) bank47583 +47584 POINT(41.26333517553974 74.65361202260422) bank47584 +47585 POINT(40.77448802078415 74.67916121115113) bank47585 +47586 POINT(39.8510402160782 74.63693454692061) bank47586 +47587 POINT(39.99462699509657 74.3927424343331) bank47587 +47588 POINT(39.74763332822612 74.78587876826367) bank47588 +47589 POINT(41.37003688848714 73.36167519270138) bank47589 +47590 POINT(39.87810856506118 73.31110929555433) bank47590 +47591 POINT(41.14708463536731 74.61865610389093) bank47591 +47592 POINT(39.784863936528325 73.0482560308368) bank47592 +47593 POINT(40.47018361699514 73.93119137499102) bank47593 +47594 POINT(40.72309323127432 73.57490611566185) bank47594 +47595 POINT(40.58914961020136 74.56979376065959) bank47595 +47596 POINT(41.05796116262832 73.88429040804303) bank47596 +47597 POINT(41.67729108401401 73.45585678218434) bank47597 +47598 POINT(39.93750252924015 74.70522300383011) bank47598 +47599 POINT(39.90092076365805 74.75895956293554) bank47599 +47600 POINT(41.122168212211164 73.17809604583519) bank47600 +47601 POINT(40.76149492911445 74.46832772431914) bank47601 +47602 POINT(41.63330513800076 74.734648605293) bank47602 +47603 POINT(40.164307236880255 74.76744983476345) bank47603 +47604 POINT(40.67196636832491 73.69155009724527) bank47604 +47605 POINT(39.802461219255825 74.9312112225279) bank47605 +47606 POINT(41.12331742119802 74.01113413042961) bank47606 +47607 POINT(39.99440230579929 73.46458773070978) bank47607 +47608 POINT(41.29917952713246 73.6044523191433) bank47608 +47609 POINT(40.05792659659494 74.62046371127641) bank47609 +47610 POINT(40.285936689617564 73.52994757518651) bank47610 +47611 POINT(41.13353725451795 73.42459555892779) bank47611 +47612 POINT(39.90819781331854 73.08392068557808) bank47612 +47613 POINT(41.01395733321304 74.75050775165184) bank47613 +47614 POINT(40.19317461776008 73.29750865591006) bank47614 +47615 POINT(40.58592973033242 74.89147407523159) bank47615 +47616 POINT(40.23127379237402 74.5045451443941) bank47616 +47617 POINT(41.546791707688676 73.84650632406786) bank47617 +47618 POINT(41.608177439186484 74.751407560977) bank47618 +47619 POINT(40.90065291907479 73.11573617129706) bank47619 +47620 POINT(39.971478509462486 74.05276135195712) bank47620 +47621 POINT(40.79530874416746 74.64150415081993) bank47621 +47622 POINT(40.04513573430247 74.74916270394323) bank47622 +47623 POINT(39.77270161423055 73.56500776040882) bank47623 +47624 POINT(41.28113916850394 74.03092926101792) bank47624 +47625 POINT(40.86827605339244 74.88250804954109) bank47625 +47626 POINT(39.768640865153024 74.22769365375416) bank47626 +47627 POINT(41.393933095711056 73.61965063914862) bank47627 +47628 POINT(41.48489882052065 73.67947027303606) bank47628 +47629 POINT(40.97751058578102 74.3755712678006) bank47629 +47630 POINT(40.71891585437701 74.25878441176758) bank47630 +47631 POINT(41.69315035216003 74.47115587626601) bank47631 +47632 POINT(40.123455262837716 73.17273249527095) bank47632 +47633 POINT(40.80368756282026 74.82615939609495) bank47633 +47634 POINT(40.71725027417835 74.66265816694948) bank47634 +47635 POINT(40.2433521998706 73.71913047562306) bank47635 +47636 POINT(41.59517445215971 74.58931372507728) bank47636 +47637 POINT(39.91319045688322 73.5238032666874) bank47637 +47638 POINT(39.8161797642817 74.23707016520031) bank47638 +47639 POINT(40.65590358711256 73.85860235995688) bank47639 +47640 POINT(41.43547433806096 74.0432004117431) bank47640 +47641 POINT(41.50061292376442 74.89145629375732) bank47641 +47642 POINT(40.03619701073993 73.86520598888895) bank47642 +47643 POINT(41.14518850114855 74.6645625420152) bank47643 +47644 POINT(40.616372548774166 73.60986389226163) bank47644 +47645 POINT(41.18234325403964 74.43113031570236) bank47645 +47646 POINT(40.63426167755825 73.07955817369096) bank47646 +47647 POINT(40.56977154534387 74.7859395213784) bank47647 +47648 POINT(41.47761952801537 73.46724872126201) bank47648 +47649 POINT(39.883420281767 73.91908597820907) bank47649 +47650 POINT(40.92802316572555 73.20496098730537) bank47650 +47651 POINT(41.56457978407218 73.64819972819139) bank47651 +47652 POINT(40.3929783681564 74.53096549527757) bank47652 +47653 POINT(40.02708506402577 73.25116264001034) bank47653 +47654 POINT(40.0891939375969 74.17740606345376) bank47654 +47655 POINT(40.70811913891197 74.98267256612046) bank47655 +47656 POINT(41.00747178410866 74.35310005644577) bank47656 +47657 POINT(41.59231813595769 74.64591348392199) bank47657 +47658 POINT(40.83607753654286 74.51692378161573) bank47658 +47659 POINT(40.52083615879435 73.78970867604825) bank47659 +47660 POINT(40.665499103461414 73.18763460111032) bank47660 +47661 POINT(40.32646007292647 73.80516380695273) bank47661 +47662 POINT(40.061773010644266 74.7521360000836) bank47662 +47663 POINT(41.234665342812704 74.12407121259298) bank47663 +47664 POINT(39.74971732166253 73.3347323834518) bank47664 +47665 POINT(41.20645233765198 74.02747855518237) bank47665 +47666 POINT(41.63498657003113 73.2197108317836) bank47666 +47667 POINT(40.721356154461084 74.48210507464348) bank47667 +47668 POINT(39.918830371053446 74.44014399645953) bank47668 +47669 POINT(41.34265655717215 73.25146071366126) bank47669 +47670 POINT(40.06753032203654 74.06659020168237) bank47670 +47671 POINT(40.10752435206978 73.81128943244643) bank47671 +47672 POINT(41.406995300291115 73.64077436823092) bank47672 +47673 POINT(41.25668765905997 74.79446064010995) bank47673 +47674 POINT(40.3887805693908 74.01299334007514) bank47674 +47675 POINT(41.37894095280555 73.8036408507497) bank47675 +47676 POINT(41.08262205568468 74.00861271420342) bank47676 +47677 POINT(41.092276207491444 74.7546468924613) bank47677 +47678 POINT(39.72732431995255 74.25631644338026) bank47678 +47679 POINT(40.98343254135003 73.87427291742519) bank47679 +47680 POINT(40.307850037023705 74.78593838123336) bank47680 +47681 POINT(41.391630769519985 74.39134585141844) bank47681 +47682 POINT(41.074526253277746 73.92125865580017) bank47682 +47683 POINT(41.01240589503707 74.00341380143558) bank47683 +47684 POINT(39.721937125019764 73.50510725045149) bank47684 +47685 POINT(40.14659749949681 73.20111858388304) bank47685 +47686 POINT(41.703539305387615 73.13042710103673) bank47686 +47687 POINT(40.58206239506727 73.42751083801608) bank47687 +47688 POINT(40.314370452457695 74.81810875869114) bank47688 +47689 POINT(40.387226522815276 73.30896808884081) bank47689 +47690 POINT(39.9663777698499 73.45778293142465) bank47690 +47691 POINT(40.32333614322147 73.94465361207143) bank47691 +47692 POINT(40.569773903914914 74.1731593025308) bank47692 +47693 POINT(40.67656200079583 74.46735753876533) bank47693 +47694 POINT(40.82587877683399 74.46383428676553) bank47694 +47695 POINT(40.3616840858965 73.36275387228135) bank47695 +47696 POINT(39.79303772365476 74.92132160222573) bank47696 +47697 POINT(41.59114632231302 74.43147402557994) bank47697 +47698 POINT(41.25646211611062 74.96966706822678) bank47698 +47699 POINT(40.79369918211706 73.81091145656592) bank47699 +47700 POINT(40.30946642113419 73.67789642124866) bank47700 +47701 POINT(40.98497315512501 74.77376402737973) bank47701 +47702 POINT(40.903054199921904 73.75572840089936) bank47702 +47703 POINT(41.30018888550288 73.25641265906211) bank47703 +47704 POINT(40.48641930933758 73.71790634348213) bank47704 +47705 POINT(40.70391963026578 74.92448606988894) bank47705 +47706 POINT(41.465624000047036 73.29579750197098) bank47706 +47707 POINT(41.67383772780597 73.79412986811676) bank47707 +47708 POINT(40.79154772277161 74.70053013956289) bank47708 +47709 POINT(40.69611886956719 73.10820161788716) bank47709 +47710 POINT(40.60477566601394 74.28317678913434) bank47710 +47711 POINT(41.166054101890175 74.04851875133959) bank47711 +47712 POINT(41.050397889991544 73.16406445867355) bank47712 +47713 POINT(40.52031313234585 73.78114508002966) bank47713 +47714 POINT(40.080530757741734 73.73960505500834) bank47714 +47715 POINT(40.74318671727099 73.1208369472264) bank47715 +47716 POINT(40.36143226236043 73.44127571891069) bank47716 +47717 POINT(41.47616907907398 74.44784093598807) bank47717 +47718 POINT(41.01357337877575 73.19434285399774) bank47718 +47719 POINT(40.13574265659139 74.50572971812579) bank47719 +47720 POINT(40.68023941771898 74.69341845325002) bank47720 +47721 POINT(41.25510846028915 73.19641745415035) bank47721 +47722 POINT(41.54168091609078 74.52963891617775) bank47722 +47723 POINT(40.039492369884584 74.63615556206773) bank47723 +47724 POINT(41.55690332260096 74.71406814733251) bank47724 +47725 POINT(40.450143648196146 73.0357621456129) bank47725 +47726 POINT(39.98680485352442 73.85404508711493) bank47726 +47727 POINT(40.28144676814041 73.51735767344123) bank47727 +47728 POINT(41.17179330169459 74.55573576909475) bank47728 +47729 POINT(41.4020621688106 74.41642688516467) bank47729 +47730 POINT(39.726591226848356 74.75707937274362) bank47730 +47731 POINT(40.323761853230124 74.94379096308464) bank47731 +47732 POINT(41.524393684091784 74.70053850030283) bank47732 +47733 POINT(40.62639687696419 74.97076108547326) bank47733 +47734 POINT(40.264505066293 74.47136381682417) bank47734 +47735 POINT(40.46395396617754 73.28286977146163) bank47735 +47736 POINT(41.300690990343874 73.78677222993542) bank47736 +47737 POINT(41.546789322692916 74.05952248946359) bank47737 +47738 POINT(40.928351537266536 74.32603331321904) bank47738 +47739 POINT(40.91081613952546 73.91660504261154) bank47739 +47740 POINT(41.43562639028355 73.54980832725276) bank47740 +47741 POINT(40.74321149202736 73.72853819153806) bank47741 +47742 POINT(41.344599062522825 74.82353569350492) bank47742 +47743 POINT(40.05809358254428 74.82221647481626) bank47743 +47744 POINT(40.576998583119966 73.85639611822475) bank47744 +47745 POINT(40.24800553105007 73.50706990317514) bank47745 +47746 POINT(40.808577204827934 73.81416463684013) bank47746 +47747 POINT(41.109153329637856 74.58748346690824) bank47747 +47748 POINT(41.503625540252976 74.53492161348348) bank47748 +47749 POINT(41.137494016629645 73.50271453943586) bank47749 +47750 POINT(40.72807908292105 73.65084964434251) bank47750 +47751 POINT(40.90983262577005 73.9262161403947) bank47751 +47752 POINT(40.348209501527286 74.59294544577914) bank47752 +47753 POINT(39.920518595662784 74.52361500365846) bank47753 +47754 POINT(39.96222212326994 74.05875939102516) bank47754 +47755 POINT(41.063553640646894 74.6883602645452) bank47755 +47756 POINT(40.09459365740359 74.12853263840876) bank47756 +47757 POINT(41.061996093411054 74.71683376999206) bank47757 +47758 POINT(40.79944710261606 73.48552107916949) bank47758 +47759 POINT(40.291439742468185 73.47184881871098) bank47759 +47760 POINT(41.51695037233897 74.69773235911286) bank47760 +47761 POINT(41.68274059803924 74.20076393731567) bank47761 +47762 POINT(41.2675539211609 73.68697567968641) bank47762 +47763 POINT(40.21982249222801 73.36504681176227) bank47763 +47764 POINT(41.2571162752644 73.08661387083667) bank47764 +47765 POINT(41.00357214639325 73.06295406490729) bank47765 +47766 POINT(41.092526475263114 73.52512490776199) bank47766 +47767 POINT(40.66535305503778 74.46853332293765) bank47767 +47768 POINT(40.30361989211494 74.47439040814507) bank47768 +47769 POINT(40.094281693477924 73.14245826406892) bank47769 +47770 POINT(41.42996316431278 74.59272074821976) bank47770 +47771 POINT(40.10228674292843 73.05874574253289) bank47771 +47772 POINT(41.34283976043772 73.93366633388128) bank47772 +47773 POINT(39.80019397041336 74.27364859295507) bank47773 +47774 POINT(41.44642262234397 73.13606790485035) bank47774 +47775 POINT(40.556670104095794 74.79704983969864) bank47775 +47776 POINT(40.59323473210623 74.73796925058262) bank47776 +47777 POINT(41.68205549188901 74.81007925926222) bank47777 +47778 POINT(41.26802675373776 73.34459158512759) bank47778 +47779 POINT(40.01276333654825 73.2202549811743) bank47779 +47780 POINT(39.91356226641091 73.9021006048516) bank47780 +47781 POINT(39.849483132929365 73.22750064134493) bank47781 +47782 POINT(41.626851645243974 73.24363005390592) bank47782 +47783 POINT(41.57363404464308 73.5744540686788) bank47783 +47784 POINT(41.51603623196985 74.57444277948582) bank47784 +47785 POINT(40.199696973426434 73.75942195213287) bank47785 +47786 POINT(41.236575959282185 73.69553640286287) bank47786 +47787 POINT(41.55321076268788 73.44748153617708) bank47787 +47788 POINT(40.88157471449584 74.75025219775384) bank47788 +47789 POINT(41.62199076442504 73.92950547204363) bank47789 +47790 POINT(41.47626204168588 73.56868165128857) bank47790 +47791 POINT(41.436036634534744 73.30846730335003) bank47791 +47792 POINT(41.42146919475004 74.91752721210639) bank47792 +47793 POINT(40.32465786824018 74.2925667524248) bank47793 +47794 POINT(39.760040886397704 73.21393006082494) bank47794 +47795 POINT(40.03364678598665 74.48453705562419) bank47795 +47796 POINT(41.418022280722596 73.1763114794405) bank47796 +47797 POINT(40.982125914643674 73.82863139144648) bank47797 +47798 POINT(39.73383034868706 73.9253257004707) bank47798 +47799 POINT(41.031885150312995 73.20849256259719) bank47799 +47800 POINT(41.40298024992224 74.93555980535398) bank47800 +47801 POINT(40.88884025302572 73.78202784690704) bank47801 +47802 POINT(40.60973127036638 74.09519732370727) bank47802 +47803 POINT(40.237152081631706 73.59829319846715) bank47803 +47804 POINT(41.08664340297947 73.88805401378646) bank47804 +47805 POINT(41.3817406519461 74.90148687743685) bank47805 +47806 POINT(40.81013399544665 73.04347757195795) bank47806 +47807 POINT(39.8928490195124 74.5825585193418) bank47807 +47808 POINT(41.01769701404369 73.41996973454474) bank47808 +47809 POINT(40.70640366309146 73.09876251216576) bank47809 +47810 POINT(40.80761977323771 74.49156146082731) bank47810 +47811 POINT(41.11126130917065 73.12536656699984) bank47811 +47812 POINT(40.80029018013369 73.63223662708104) bank47812 +47813 POINT(41.40540523706631 73.9673884787823) bank47813 +47814 POINT(40.4679898939357 74.92943921654377) bank47814 +47815 POINT(40.80757511979816 74.20516864075442) bank47815 +47816 POINT(39.86325984533022 73.66973442972952) bank47816 +47817 POINT(40.97215638791998 74.82925654805862) bank47817 +47818 POINT(41.16807486596798 73.27664236314419) bank47818 +47819 POINT(39.80189759472506 74.45609100527055) bank47819 +47820 POINT(41.685946266055055 73.63608183296255) bank47820 +47821 POINT(40.01336305437781 74.15036725737548) bank47821 +47822 POINT(39.731180467963206 73.67782031459157) bank47822 +47823 POINT(39.83935646327072 74.71639199789536) bank47823 +47824 POINT(41.6006564153061 74.23970803722172) bank47824 +47825 POINT(40.65991043647877 73.73751994862714) bank47825 +47826 POINT(39.89877746968841 74.43707385594399) bank47826 +47827 POINT(40.53357725086685 73.44130889038497) bank47827 +47828 POINT(40.23310508911363 74.4460055572472) bank47828 +47829 POINT(40.24277207361246 73.4882300994559) bank47829 +47830 POINT(41.444395263217295 74.40009305270546) bank47830 +47831 POINT(41.02921754668176 73.02724302080944) bank47831 +47832 POINT(39.73296483049387 73.55800635551975) bank47832 +47833 POINT(41.57005120903706 73.12078445512383) bank47833 +47834 POINT(39.72405074584703 73.04843213640946) bank47834 +47835 POINT(40.521524172592564 74.1110791902197) bank47835 +47836 POINT(41.611843026928355 74.80306679468463) bank47836 +47837 POINT(39.795343275265 73.50771752821404) bank47837 +47838 POINT(41.257298104463224 74.25832395343471) bank47838 +47839 POINT(41.187327524920676 74.42439608963507) bank47839 +47840 POINT(41.590354119950774 74.9220006765373) bank47840 +47841 POINT(40.49170252760174 74.56498046316295) bank47841 +47842 POINT(41.48583642023758 73.89453952421779) bank47842 +47843 POINT(40.22465181941885 74.01007780930614) bank47843 +47844 POINT(40.728664772883036 73.96122983611262) bank47844 +47845 POINT(41.24625558720167 74.48729822080435) bank47845 +47846 POINT(40.24952569917193 74.23011179129938) bank47846 +47847 POINT(40.379801411181575 73.71559231890767) bank47847 +47848 POINT(40.36744683259427 74.72733828518426) bank47848 +47849 POINT(40.89359205438238 74.80268303995157) bank47849 +47850 POINT(40.36027515274307 74.62174308539286) bank47850 +47851 POINT(40.85542204882313 73.06473897717969) bank47851 +47852 POINT(40.90009638764252 74.0031158232914) bank47852 +47853 POINT(39.9492683317932 74.49999354898411) bank47853 +47854 POINT(41.25402891791184 74.60650521587047) bank47854 +47855 POINT(39.91181845567362 73.93299058587738) bank47855 +47856 POINT(40.89908812933508 74.40189605480984) bank47856 +47857 POINT(41.67358599795632 74.67420811767278) bank47857 +47858 POINT(41.28367039068177 73.79544464914224) bank47858 +47859 POINT(41.40328344595455 74.17527128547803) bank47859 +47860 POINT(41.58489815267259 73.08133928681342) bank47860 +47861 POINT(41.574586075367776 74.06147489702005) bank47861 +47862 POINT(41.46126189087142 73.44790475211731) bank47862 +47863 POINT(40.602677718496764 74.0864973105331) bank47863 +47864 POINT(41.10181406202804 74.77140457558184) bank47864 +47865 POINT(39.9234833543224 75.00200196660762) bank47865 +47866 POINT(40.97488059928725 74.49545383214956) bank47866 +47867 POINT(40.2927816030823 73.27620310761444) bank47867 +47868 POINT(40.210022700922146 74.60369495281795) bank47868 +47869 POINT(41.67689092633885 74.78279253345933) bank47869 +47870 POINT(40.76129648256407 74.58322028551686) bank47870 +47871 POINT(40.06187823277961 73.6255577809849) bank47871 +47872 POINT(40.51671931279468 73.50489750954257) bank47872 +47873 POINT(41.704204484592054 73.43525511578352) bank47873 +47874 POINT(40.679975421311475 74.23441285456083) bank47874 +47875 POINT(41.3920503781957 74.88631667812815) bank47875 +47876 POINT(41.646877204856104 73.57519678501295) bank47876 +47877 POINT(41.43293797411226 74.77492764138509) bank47877 +47878 POINT(39.97654815144018 74.1525176671702) bank47878 +47879 POINT(39.97524107885211 74.94922985871601) bank47879 +47880 POINT(39.81555780349775 74.98894398627041) bank47880 +47881 POINT(41.2541031542436 74.90719863155319) bank47881 +47882 POINT(40.53006144635008 73.23480166232734) bank47882 +47883 POINT(41.16580767345882 74.12996341629979) bank47883 +47884 POINT(41.21275779539841 73.5824450555699) bank47884 +47885 POINT(41.06714934237177 73.46203565707587) bank47885 +47886 POINT(41.35319056173262 73.78986532849942) bank47886 +47887 POINT(41.61597152650089 74.79181015397965) bank47887 +47888 POINT(40.77278434297981 73.98085747917999) bank47888 +47889 POINT(41.68411610327696 74.98267708413344) bank47889 +47890 POINT(40.50128343822414 73.76909897954555) bank47890 +47891 POINT(41.344194501631165 73.17918784672901) bank47891 +47892 POINT(40.664635697803085 73.81155366815688) bank47892 +47893 POINT(41.29782354103991 74.56261604630537) bank47893 +47894 POINT(40.35133312868237 74.4501249648673) bank47894 +47895 POINT(39.7613464090643 73.22028505168689) bank47895 +47896 POINT(41.42102638480142 74.2754043972556) bank47896 +47897 POINT(40.73220578099999 73.69250774048349) bank47897 +47898 POINT(39.867709405364664 73.92276527648471) bank47898 +47899 POINT(41.67076682261462 73.32024611279714) bank47899 +47900 POINT(40.231613049214644 74.06085693871442) bank47900 +47901 POINT(40.46417386860624 74.52349754015835) bank47901 +47902 POINT(40.75130679173576 74.75049784821391) bank47902 +47903 POINT(41.48785213130562 73.99768945458192) bank47903 +47904 POINT(41.33597446405672 74.34522496411464) bank47904 +47905 POINT(40.51189824808946 74.39874227183765) bank47905 +47906 POINT(41.13504291944703 73.55672981720018) bank47906 +47907 POINT(41.453604073128446 73.49455991672498) bank47907 +47908 POINT(40.66302103753375 73.10301066515498) bank47908 +47909 POINT(40.629951485539635 74.00059920017635) bank47909 +47910 POINT(39.846750803760315 74.11057867499439) bank47910 +47911 POINT(39.92451619142202 74.98116434260781) bank47911 +47912 POINT(40.78361186754538 73.17035478125953) bank47912 +47913 POINT(41.30622990927404 73.52622260917316) bank47913 +47914 POINT(40.00474354108899 74.25418661926598) bank47914 +47915 POINT(40.09788393459945 73.52154975131202) bank47915 +47916 POINT(41.047913917140406 74.87066324585093) bank47916 +47917 POINT(39.984279873481746 73.08911742284997) bank47917 +47918 POINT(40.007728213198035 74.66060090636536) bank47918 +47919 POINT(40.623380530329854 74.34471174224586) bank47919 +47920 POINT(41.13674102431069 73.3410685281857) bank47920 +47921 POINT(41.394381244480876 73.46786126422204) bank47921 +47922 POINT(40.11302654856267 73.13621111593919) bank47922 +47923 POINT(41.55400011403493 74.69787914357347) bank47923 +47924 POINT(40.901694033672875 73.76149700300917) bank47924 +47925 POINT(41.33719212779919 73.88936007707956) bank47925 +47926 POINT(39.95151701457779 74.22659589236464) bank47926 +47927 POINT(40.21298508238703 73.74035278358103) bank47927 +47928 POINT(40.76759476319273 74.26694945083914) bank47928 +47929 POINT(41.14007436402253 74.99083385727164) bank47929 +47930 POINT(40.709270584143916 73.31973111166445) bank47930 +47931 POINT(40.199470642832246 74.52281468151767) bank47931 +47932 POINT(40.92841653067473 73.48401117499715) bank47932 +47933 POINT(41.63901957693374 73.55241942012594) bank47933 +47934 POINT(41.17627835472296 73.3363889071961) bank47934 +47935 POINT(40.141119357736834 74.81627033730416) bank47935 +47936 POINT(40.05893194170963 74.19322042540135) bank47936 +47937 POINT(41.21800577730922 73.85145271597399) bank47937 +47938 POINT(40.53055959184894 74.24033064888313) bank47938 +47939 POINT(40.791172509767456 73.95738857930193) bank47939 +47940 POINT(41.011663447502244 73.82420395656973) bank47940 +47941 POINT(40.39964874407732 73.77740370405512) bank47941 +47942 POINT(41.080379525406386 73.87570596754306) bank47942 +47943 POINT(40.586907131093945 74.68810272298809) bank47943 +47944 POINT(41.13449606895443 73.68038473111224) bank47944 +47945 POINT(40.83769394866398 74.22491802253303) bank47945 +47946 POINT(39.95397783089304 74.31998673061678) bank47946 +47947 POINT(39.812185633627784 73.31796942314968) bank47947 +47948 POINT(39.826841256690095 74.20006825790941) bank47948 +47949 POINT(40.87831957790557 74.353112443904) bank47949 +47950 POINT(40.159831759127464 73.68468616712187) bank47950 +47951 POINT(39.902233613920835 73.496441795125) bank47951 +47952 POINT(39.72776044205425 73.57159636614185) bank47952 +47953 POINT(40.13150237722125 74.26593875368208) bank47953 +47954 POINT(40.21247148119455 73.86993632686581) bank47954 +47955 POINT(40.09989704054172 74.28938058785877) bank47955 +47956 POINT(41.09833049469017 74.58262296707936) bank47956 +47957 POINT(41.54127433593651 74.28303914476577) bank47957 +47958 POINT(39.90756875996075 74.70235151147033) bank47958 +47959 POINT(41.47116573478592 74.25498208030554) bank47959 +47960 POINT(40.64376997148575 74.2323893602962) bank47960 +47961 POINT(40.85331291444368 74.5455233621161) bank47961 +47962 POINT(40.98040018127171 73.34534804393367) bank47962 +47963 POINT(40.28156218558789 73.25624989764776) bank47963 +47964 POINT(41.395812393080064 74.30162280837513) bank47964 +47965 POINT(40.677493751440124 74.78765933715593) bank47965 +47966 POINT(40.21119826128708 74.97044182095354) bank47966 +47967 POINT(40.58107047797205 74.00023810768057) bank47967 +47968 POINT(40.62452027740729 74.03896029571555) bank47968 +47969 POINT(41.46873357562449 73.69038336264852) bank47969 +47970 POINT(39.79733368538448 74.89623476650627) bank47970 +47971 POINT(40.55594253357639 74.94461639550177) bank47971 +47972 POINT(40.916445778203936 74.51431786685879) bank47972 +47973 POINT(40.62737989663916 73.03632106017938) bank47973 +47974 POINT(39.77545148866131 74.81887473517706) bank47974 +47975 POINT(41.671804373446555 74.99045711718124) bank47975 +47976 POINT(40.237763338455366 74.86575846468384) bank47976 +47977 POINT(40.55173305946735 73.32743517077267) bank47977 +47978 POINT(40.351283320142855 73.47541027151703) bank47978 +47979 POINT(41.35426399935261 73.79624207092533) bank47979 +47980 POINT(40.20390623906819 74.14384785635724) bank47980 +47981 POINT(40.569596335020705 74.75809725387016) bank47981 +47982 POINT(40.735803955042606 73.40288617387631) bank47982 +47983 POINT(41.42220425581573 73.3945669601836) bank47983 +47984 POINT(40.22061112329043 74.4624807029483) bank47984 +47985 POINT(41.556149333316895 73.12596354761558) bank47985 +47986 POINT(40.733587382933315 74.20535707264456) bank47986 +47987 POINT(41.37800456105875 73.95800236330498) bank47987 +47988 POINT(40.81805118536075 73.82096359191559) bank47988 +47989 POINT(40.44985913665894 74.88712446881703) bank47989 +47990 POINT(40.53805618796343 73.94268579462745) bank47990 +47991 POINT(41.475109404887455 73.34875412553362) bank47991 +47992 POINT(40.283576606292726 74.39635458046983) bank47992 +47993 POINT(40.322274082570736 74.58667537692244) bank47993 +47994 POINT(40.08304426494831 73.79758804737156) bank47994 +47995 POINT(41.33093234909187 74.09848956803175) bank47995 +47996 POINT(41.31421223343395 73.06838351445339) bank47996 +47997 POINT(39.80934403933534 74.14067274723365) bank47997 +47998 POINT(40.32220240910913 74.23107976951945) bank47998 +47999 POINT(40.39127286549823 74.21924402903839) bank47999 +48000 POINT(40.20865351880584 73.31042385794369) bank48000 +48001 POINT(41.08540469278314 73.36084103868131) bank48001 +48002 POINT(40.326055002527326 74.32621552610622) bank48002 +48003 POINT(41.524549877083864 73.30669090783117) bank48003 +48004 POINT(41.0069217401172 74.9615868142797) bank48004 +48005 POINT(39.733283879236524 74.5937959650218) bank48005 +48006 POINT(41.3899641003437 74.5644680760632) bank48006 +48007 POINT(40.783056105856986 74.05257787269474) bank48007 +48008 POINT(40.08701817539431 74.37175429407962) bank48008 +48009 POINT(39.90506367141969 74.78705377617126) bank48009 +48010 POINT(40.58677534348394 74.63267204020275) bank48010 +48011 POINT(39.92032691483222 73.23651159745158) bank48011 +48012 POINT(40.12375464608195 74.11080502405477) bank48012 +48013 POINT(40.47602806750678 74.93028922869821) bank48013 +48014 POINT(41.626355105137286 73.29544258568325) bank48014 +48015 POINT(40.907692469170804 73.37162622767619) bank48015 +48016 POINT(41.19133316973591 73.79153323664653) bank48016 +48017 POINT(41.35345954986941 73.44499497457262) bank48017 +48018 POINT(41.10459754656709 74.20633949888969) bank48018 +48019 POINT(39.851259099649056 73.31506304017994) bank48019 +48020 POINT(40.91542129920087 73.72890363787666) bank48020 +48021 POINT(39.87085352629902 73.90775134445954) bank48021 +48022 POINT(40.43696426143193 74.9651757533642) bank48022 +48023 POINT(41.123694971453595 73.41285460905196) bank48023 +48024 POINT(40.858043517426665 74.89458876727163) bank48024 +48025 POINT(40.715310902653535 74.315147995961) bank48025 +48026 POINT(41.542583516447586 74.61901089915857) bank48026 +48027 POINT(39.75538794892641 74.289023105418) bank48027 +48028 POINT(40.35543051255724 73.31530714422448) bank48028 +48029 POINT(41.22594178160486 73.9443793838082) bank48029 +48030 POINT(41.468381048797454 73.03453286078448) bank48030 +48031 POINT(40.09603391820635 74.4383503959121) bank48031 +48032 POINT(39.91055765032751 73.07387786961043) bank48032 +48033 POINT(41.260958570304005 74.2286729210475) bank48033 +48034 POINT(40.50066970687751 74.31101664317109) bank48034 +48035 POINT(40.38777865979676 73.88597408470227) bank48035 +48036 POINT(39.84877573201442 74.79568788002638) bank48036 +48037 POINT(41.46093556583398 73.21678306667071) bank48037 +48038 POINT(40.810219271941826 74.8507419594691) bank48038 +48039 POINT(40.959202367741135 74.18983629084572) bank48039 +48040 POINT(41.645771650753105 74.24672379592137) bank48040 +48041 POINT(40.18099956920728 74.76505469936001) bank48041 +48042 POINT(41.51423995063409 74.46729863972192) bank48042 +48043 POINT(41.432388040525 74.1111310110043) bank48043 +48044 POINT(41.438755287138264 73.7263314932329) bank48044 +48045 POINT(41.357300035980735 73.80412392095616) bank48045 +48046 POINT(39.929388586544476 74.71635454020803) bank48046 +48047 POINT(40.39445697423375 74.22628404821445) bank48047 +48048 POINT(39.717872296079165 73.53846816214768) bank48048 +48049 POINT(40.213493873730954 74.05159246133644) bank48049 +48050 POINT(40.36645626579725 73.50928847134442) bank48050 +48051 POINT(40.149292379710154 74.28812119256682) bank48051 +48052 POINT(40.64096424031366 73.86558339582787) bank48052 +48053 POINT(40.565454884301516 73.16533934932637) bank48053 +48054 POINT(41.41121353255378 74.1493333510814) bank48054 +48055 POINT(41.5185284663117 74.09936360548633) bank48055 +48056 POINT(40.57164086747536 74.50213797155924) bank48056 +48057 POINT(39.806529705130444 74.30256634395244) bank48057 +48058 POINT(40.81207869182022 73.2240881558216) bank48058 +48059 POINT(40.49402458188279 73.92584745321207) bank48059 +48060 POINT(40.86348847472608 74.17059230167206) bank48060 +48061 POINT(40.21636667230694 74.72341475431111) bank48061 +48062 POINT(40.412676632032394 73.28941084332388) bank48062 +48063 POINT(40.05744674347841 74.9114916885289) bank48063 +48064 POINT(41.23025102066738 74.67992010728017) bank48064 +48065 POINT(40.75177578465439 73.47554054491147) bank48065 +48066 POINT(39.92675329089479 73.71391977276268) bank48066 +48067 POINT(40.25202778078413 74.93915877647404) bank48067 +48068 POINT(41.42847055216511 73.11384604352993) bank48068 +48069 POINT(39.858669646166724 74.88170217140488) bank48069 +48070 POINT(39.82402986044033 74.7513555775584) bank48070 +48071 POINT(40.5513393734783 73.95993158996926) bank48071 +48072 POINT(41.290146426474465 73.50262048630478) bank48072 +48073 POINT(41.50193632216174 73.62698074062286) bank48073 +48074 POINT(40.123061422501486 74.34437260547749) bank48074 +48075 POINT(41.36013481292365 73.41467883994726) bank48075 +48076 POINT(40.71915887538933 74.1399572478251) bank48076 +48077 POINT(40.20285619342944 74.83611926170892) bank48077 +48078 POINT(41.28721897204554 74.0232017423339) bank48078 +48079 POINT(39.807765228964655 73.73400983526373) bank48079 +48080 POINT(40.47045641105499 73.2312418167752) bank48080 +48081 POINT(41.4812943520205 73.49452136347968) bank48081 +48082 POINT(40.7803649406798 73.21503768203367) bank48082 +48083 POINT(40.264491629029706 74.48128864732082) bank48083 +48084 POINT(40.19394751080146 73.82472329920904) bank48084 +48085 POINT(39.97588838682526 74.61508122441455) bank48085 +48086 POINT(40.52286511561288 74.5821472330736) bank48086 +48087 POINT(41.11006001544203 73.01310988689787) bank48087 +48088 POINT(40.0364514566426 73.2767281350018) bank48088 +48089 POINT(39.9982008655481 74.7415254710557) bank48089 +48090 POINT(40.8663998262448 73.03735503776355) bank48090 +48091 POINT(40.882593121071714 73.07968852296226) bank48091 +48092 POINT(40.526525391895376 74.03189626753586) bank48092 +48093 POINT(40.18399830456916 74.58551706907943) bank48093 +48094 POINT(40.76282042866576 73.62685520700224) bank48094 +48095 POINT(39.833706234601685 74.50281892399465) bank48095 +48096 POINT(40.3429509743717 74.62084419722987) bank48096 +48097 POINT(41.39554316941005 73.57830043998248) bank48097 +48098 POINT(40.41236166217244 73.24623569434924) bank48098 +48099 POINT(39.862144683428326 74.61325898098765) bank48099 +48100 POINT(41.07050225040323 73.53707060412324) bank48100 +48101 POINT(41.046040448198866 74.25149226579462) bank48101 +48102 POINT(41.05849535000927 73.69798511436676) bank48102 +48103 POINT(40.83396200069552 73.168913984595) bank48103 +48104 POINT(39.758605779326615 74.81035518139859) bank48104 +48105 POINT(39.9433300699114 73.0933963683433) bank48105 +48106 POINT(40.57906524864167 74.84276034605006) bank48106 +48107 POINT(40.12562005876644 74.48771862151555) bank48107 +48108 POINT(40.94069244773825 74.06221912367097) bank48108 +48109 POINT(39.781477910842554 74.31083394355812) bank48109 +48110 POINT(40.14252216142684 74.15005402210268) bank48110 +48111 POINT(40.44072859371932 74.4683940041061) bank48111 +48112 POINT(40.9669221542144 74.60113255450523) bank48112 +48113 POINT(39.92360361745832 74.13183760197924) bank48113 +48114 POINT(40.71369373764572 73.24485387819843) bank48114 +48115 POINT(39.98899040579152 74.74350515435556) bank48115 +48116 POINT(40.73907581163853 74.82120295656021) bank48116 +48117 POINT(40.49941366301325 74.39474218697451) bank48117 +48118 POINT(41.49508466072177 74.31842201486981) bank48118 +48119 POINT(40.46507960289868 74.91796285362257) bank48119 +48120 POINT(40.14375926620953 74.93359374932395) bank48120 +48121 POINT(41.211951465933055 74.24843371166655) bank48121 +48122 POINT(39.71958394587824 74.66589127903191) bank48122 +48123 POINT(41.66930520098379 74.80037090010228) bank48123 +48124 POINT(41.14598844199229 74.4646980768338) bank48124 +48125 POINT(40.05618731610547 73.96366966488453) bank48125 +48126 POINT(40.11266922451734 73.33154692851957) bank48126 +48127 POINT(41.019678662872835 73.56305034095855) bank48127 +48128 POINT(40.44303027132978 73.07482132601669) bank48128 +48129 POINT(39.941004911250005 74.35828984285281) bank48129 +48130 POINT(40.90243296225931 74.89878493370813) bank48130 +48131 POINT(41.209137566273924 73.95363554258128) bank48131 +48132 POINT(40.7932624135623 74.9590675992131) bank48132 +48133 POINT(41.29475369560474 73.07847227761245) bank48133 +48134 POINT(41.56761764728939 73.71216368798824) bank48134 +48135 POINT(41.11251847412844 74.54787983659621) bank48135 +48136 POINT(40.925005041024264 74.30622452883412) bank48136 +48137 POINT(41.41441350982556 74.58322602955995) bank48137 +48138 POINT(39.9408490695667 73.82298516407072) bank48138 +48139 POINT(41.49255386307537 73.09869385600103) bank48139 +48140 POINT(39.798076749382496 74.7463921448584) bank48140 +48141 POINT(41.52906592551278 74.93585201877328) bank48141 +48142 POINT(40.188989572830216 74.60372983575522) bank48142 +48143 POINT(40.604639894565366 73.19968682862829) bank48143 +48144 POINT(40.11726250648311 74.60606400605408) bank48144 +48145 POINT(40.26141330328932 73.80518386147027) bank48145 +48146 POINT(41.41562930732361 74.07468798155135) bank48146 +48147 POINT(40.46010507943162 74.79135146157077) bank48147 +48148 POINT(40.82923590740173 74.87584141639972) bank48148 +48149 POINT(41.20158811436105 73.86415099535597) bank48149 +48150 POINT(41.149793328148 73.98608327855956) bank48150 +48151 POINT(39.96202117082511 73.10506509122) bank48151 +48152 POINT(41.700064815857694 73.99507958126112) bank48152 +48153 POINT(40.01659407888933 74.44842721200104) bank48153 +48154 POINT(39.90734256672802 74.29886368553441) bank48154 +48155 POINT(41.293809771767826 74.42064731421272) bank48155 +48156 POINT(40.766905310185265 74.65836645016678) bank48156 +48157 POINT(41.477180298904145 73.50876718856249) bank48157 +48158 POINT(40.64561528471769 74.6164709184156) bank48158 +48159 POINT(39.88386635390341 74.22667573559275) bank48159 +48160 POINT(41.23610803813986 73.614325582461) bank48160 +48161 POINT(40.272264662412944 74.33869637666837) bank48161 +48162 POINT(40.293483757941395 73.55058366881137) bank48162 +48163 POINT(41.524186856732 74.35362959384224) bank48163 +48164 POINT(40.350474429539325 73.34493750607157) bank48164 +48165 POINT(39.80649399088077 73.26527728966784) bank48165 +48166 POINT(40.72823055347872 74.69356572688822) bank48166 +48167 POINT(41.155232970255554 73.29959319071976) bank48167 +48168 POINT(39.971791458475046 74.29869572899607) bank48168 +48169 POINT(39.78395174644793 73.71312203133121) bank48169 +48170 POINT(40.0002039743681 73.65185700551325) bank48170 +48171 POINT(40.12256397176143 74.15109893840847) bank48171 +48172 POINT(40.007399874802346 74.58986260974432) bank48172 +48173 POINT(41.0741513042724 73.52105355864468) bank48173 +48174 POINT(40.54466280685404 74.99105442555755) bank48174 +48175 POINT(40.03072919695713 74.42878725974285) bank48175 +48176 POINT(40.20709857140968 73.08392977964964) bank48176 +48177 POINT(40.82942497129399 73.2155525474867) bank48177 +48178 POINT(40.792063070066526 74.90707869894837) bank48178 +48179 POINT(41.570276145080584 74.67825096578329) bank48179 +48180 POINT(41.200659589343644 73.60972143820669) bank48180 +48181 POINT(41.22786198577988 74.02794374426857) bank48181 +48182 POINT(40.7998853137517 73.93234308725472) bank48182 +48183 POINT(41.53816292545351 73.79399364464014) bank48183 +48184 POINT(40.17346314370053 74.9031466965472) bank48184 +48185 POINT(40.318308368159315 74.60730173528414) bank48185 +48186 POINT(39.98453284497879 74.65978894266871) bank48186 +48187 POINT(40.43061286826975 74.60465333877099) bank48187 +48188 POINT(41.62346207466901 73.2676790719293) bank48188 +48189 POINT(39.71853757454702 74.43222687767869) bank48189 +48190 POINT(40.074760252463115 73.80163394580451) bank48190 +48191 POINT(40.62729913309054 74.10773077455731) bank48191 +48192 POINT(41.33284579512037 74.28407547823659) bank48192 +48193 POINT(40.40386037339667 74.67495414765014) bank48193 +48194 POINT(40.63986201164534 74.1772442595251) bank48194 +48195 POINT(41.04032963304971 74.43710955981935) bank48195 +48196 POINT(40.266496161119534 73.88692139493703) bank48196 +48197 POINT(39.85147019244425 73.19525134731431) bank48197 +48198 POINT(41.616287336198624 73.25329141783827) bank48198 +48199 POINT(40.76516344440632 73.87342441080091) bank48199 +48200 POINT(41.29224189514161 74.65864526298778) bank48200 +48201 POINT(40.34825281473007 74.99031409075822) bank48201 +48202 POINT(40.82441080231811 74.9351917907301) bank48202 +48203 POINT(40.585878244397136 73.41865027808934) bank48203 +48204 POINT(41.225351548852515 73.03006934711162) bank48204 +48205 POINT(41.17775229644806 73.12546073632254) bank48205 +48206 POINT(40.808185063840156 73.99610016829192) bank48206 +48207 POINT(41.03321006509524 73.6230017269347) bank48207 +48208 POINT(40.53512671818336 74.65743701104313) bank48208 +48209 POINT(40.14866611229024 73.48020831851149) bank48209 +48210 POINT(41.053073945070416 74.47781904591626) bank48210 +48211 POINT(41.044674394849366 74.06457164349634) bank48211 +48212 POINT(40.51822463232105 73.97915697176164) bank48212 +48213 POINT(39.89681397851354 73.84619222471972) bank48213 +48214 POINT(40.192616713387565 73.51916293518404) bank48214 +48215 POINT(40.125132174455466 74.76318652622444) bank48215 +48216 POINT(40.03718947396851 74.6360535133921) bank48216 +48217 POINT(39.83072849321913 74.87476137129941) bank48217 +48218 POINT(40.14408082783848 74.34708808030233) bank48218 +48219 POINT(41.340040228338786 74.19100113406915) bank48219 +48220 POINT(41.247377799716205 74.36707360990816) bank48220 +48221 POINT(39.749414645047075 73.41301842581635) bank48221 +48222 POINT(41.582324956321806 74.89324375341343) bank48222 +48223 POINT(41.1584177067141 73.29622841984593) bank48223 +48224 POINT(41.592246842013495 74.46720771061058) bank48224 +48225 POINT(40.51623427997346 73.83632906619951) bank48225 +48226 POINT(40.66701789182052 73.30189129529627) bank48226 +48227 POINT(40.66226973999765 74.36999281066815) bank48227 +48228 POINT(39.98282040354062 74.50262624304922) bank48228 +48229 POINT(40.250659233792845 73.11317708966921) bank48229 +48230 POINT(41.23867117780337 73.52433249780684) bank48230 +48231 POINT(41.532256338735166 74.73982848303598) bank48231 +48232 POINT(39.7140427813786 73.93792200080351) bank48232 +48233 POINT(40.713104833286536 74.45728383277859) bank48233 +48234 POINT(40.36186307824646 74.80876904608853) bank48234 +48235 POINT(40.37446441088922 74.92853262159204) bank48235 +48236 POINT(40.95765203551283 73.50869300466591) bank48236 +48237 POINT(40.99574542716072 74.35898017691142) bank48237 +48238 POINT(41.629901835609935 73.57821418010138) bank48238 +48239 POINT(40.97384804538368 74.56581878686033) bank48239 +48240 POINT(40.07889499560773 74.01750396742786) bank48240 +48241 POINT(40.550057046084994 73.02117179749845) bank48241 +48242 POINT(40.10935722610115 73.41215991888066) bank48242 +48243 POINT(41.685339996794454 74.6783784556883) bank48243 +48244 POINT(40.777658377100344 74.48724538013789) bank48244 +48245 POINT(41.275237446774966 74.2029718038439) bank48245 +48246 POINT(40.98596606297223 74.09298199436903) bank48246 +48247 POINT(40.3267724297754 74.63239434434891) bank48247 +48248 POINT(39.95139216247018 74.40383711990715) bank48248 +48249 POINT(41.39474372107939 73.53560146889765) bank48249 +48250 POINT(40.512635339849595 74.07190806937385) bank48250 +48251 POINT(40.63573527282469 74.22293711929845) bank48251 +48252 POINT(40.34040379868049 74.37513667055998) bank48252 +48253 POINT(40.0150890981939 74.75564477257336) bank48253 +48254 POINT(40.774494559727216 73.733010538029) bank48254 +48255 POINT(40.06008553694061 74.14321409167248) bank48255 +48256 POINT(40.429864807256216 73.1971978697461) bank48256 +48257 POINT(39.84036727102112 73.05414178849031) bank48257 +48258 POINT(41.142749130056735 73.02914235485986) bank48258 +48259 POINT(40.125720734640645 74.13430319397817) bank48259 +48260 POINT(40.685890959325555 73.11473668174354) bank48260 +48261 POINT(40.05857468690445 74.7739990522628) bank48261 +48262 POINT(40.27472060772014 73.9957762872686) bank48262 +48263 POINT(41.078400287313634 74.54517643752185) bank48263 +48264 POINT(40.65276624353245 74.14369852212182) bank48264 +48265 POINT(39.7706240690635 73.08043392319432) bank48265 +48266 POINT(41.22616104403026 73.37337009433627) bank48266 +48267 POINT(41.0779688021763 74.26909722897211) bank48267 +48268 POINT(41.419227120868314 74.7781274762471) bank48268 +48269 POINT(40.36647920793124 73.34647260768514) bank48269 +48270 POINT(40.909886661427905 74.85235600129828) bank48270 +48271 POINT(41.223243329296324 73.40425291032737) bank48271 +48272 POINT(40.24954604653019 73.8310827767225) bank48272 +48273 POINT(39.9528663054682 73.36883963714331) bank48273 +48274 POINT(40.08597557223154 73.20461118936988) bank48274 +48275 POINT(41.696461544829795 74.86615365225326) bank48275 +48276 POINT(41.073807612568274 74.97727773153416) bank48276 +48277 POINT(40.763304851655796 74.68510895470224) bank48277 +48278 POINT(40.68643056770035 74.05065716426108) bank48278 +48279 POINT(41.308849558774966 74.7590254564874) bank48279 +48280 POINT(40.203357587341166 74.31370779198495) bank48280 +48281 POINT(39.85990335227212 74.78454581195116) bank48281 +48282 POINT(39.765595063882216 74.20373580878602) bank48282 +48283 POINT(41.703918422369064 74.46790675774392) bank48283 +48284 POINT(40.97665880000739 74.13701593244208) bank48284 +48285 POINT(39.73619516160751 73.26944615408397) bank48285 +48286 POINT(40.569563831655046 73.15676019797205) bank48286 +48287 POINT(41.67381489634954 74.98705611034794) bank48287 +48288 POINT(40.45807223028423 73.01951819364828) bank48288 +48289 POINT(39.85078506607013 73.65020434153222) bank48289 +48290 POINT(39.8598788064997 74.8858557636617) bank48290 +48291 POINT(41.37160739210338 74.89296001936286) bank48291 +48292 POINT(40.83853856302182 74.39315281576206) bank48292 +48293 POINT(41.48589388941725 74.6919083333126) bank48293 +48294 POINT(40.23637877698364 74.15875531915735) bank48294 +48295 POINT(39.800853769096 74.87625939433995) bank48295 +48296 POINT(40.04343848463086 73.70585701864638) bank48296 +48297 POINT(40.79701804622478 73.93474278308936) bank48297 +48298 POINT(40.34710638657559 74.4434898841647) bank48298 +48299 POINT(41.47438797016652 73.70673908756334) bank48299 +48300 POINT(41.63934819211517 74.84402102461895) bank48300 +48301 POINT(39.72648335472534 74.03377743147398) bank48301 +48302 POINT(41.25494789373923 73.97625498111493) bank48302 +48303 POINT(41.64035493901518 73.40194898856723) bank48303 +48304 POINT(40.04860867886291 74.41355088287717) bank48304 +48305 POINT(39.86215311043225 73.67296515320754) bank48305 +48306 POINT(41.641574473822175 74.72848757700534) bank48306 +48307 POINT(40.524706604208106 73.47435859306887) bank48307 +48308 POINT(40.227760944392045 74.51850829214047) bank48308 +48309 POINT(40.31419667368998 74.04871133959391) bank48309 +48310 POINT(40.26281982044556 74.09629439523115) bank48310 +48311 POINT(40.52356265386421 73.48090934802744) bank48311 +48312 POINT(41.12716804206789 74.99788123073021) bank48312 +48313 POINT(41.66194680360345 74.91208947167743) bank48313 +48314 POINT(40.67710069033238 74.66439200959617) bank48314 +48315 POINT(40.91115835873922 74.1732119764) bank48315 +48316 POINT(39.85118429237132 74.7403798962662) bank48316 +48317 POINT(40.66714705883908 74.73090081442705) bank48317 +48318 POINT(40.53344820962071 74.64523938789537) bank48318 +48319 POINT(40.94264354475983 74.959379623632) bank48319 +48320 POINT(40.98127057135916 73.67655728698644) bank48320 +48321 POINT(41.362044653289 73.81107770880757) bank48321 +48322 POINT(40.83813534229482 73.10624673820448) bank48322 +48323 POINT(41.624258001052034 73.61732020908961) bank48323 +48324 POINT(40.5142445746618 73.91570337519089) bank48324 +48325 POINT(41.322961355725305 74.86608804056475) bank48325 +48326 POINT(39.950218266609085 73.80710218107019) bank48326 +48327 POINT(41.70008046568021 73.88613594940617) bank48327 +48328 POINT(40.2291438412646 74.11706617161043) bank48328 +48329 POINT(41.12961859724644 73.94178469277745) bank48329 +48330 POINT(40.07697654666493 73.4278758203778) bank48330 +48331 POINT(40.243682846043086 73.04286286729331) bank48331 +48332 POINT(40.03507023708439 74.68481559437039) bank48332 +48333 POINT(40.77849466301623 74.09839414469046) bank48333 +48334 POINT(40.05168246364699 73.53603783481923) bank48334 +48335 POINT(41.481710527356334 73.05345215828984) bank48335 +48336 POINT(40.89057078247443 74.01262240661656) bank48336 +48337 POINT(40.569494437527496 73.86335557472788) bank48337 +48338 POINT(40.252487105591676 73.8364295104919) bank48338 +48339 POINT(40.86812779298819 73.83414741077175) bank48339 +48340 POINT(40.41066248733584 74.804407111814) bank48340 +48341 POINT(40.79498904931348 73.86306247623385) bank48341 +48342 POINT(40.17267568896332 73.94190451746766) bank48342 +48343 POINT(40.373898790480084 73.25487354246837) bank48343 +48344 POINT(40.16817391515995 73.36656061585056) bank48344 +48345 POINT(40.782491246753104 74.48372965977099) bank48345 +48346 POINT(40.16407173657491 74.96302711483679) bank48346 +48347 POINT(41.663830099902206 74.7674658214265) bank48347 +48348 POINT(40.527735026115316 74.12947896843282) bank48348 +48349 POINT(41.16613801850672 74.98015230474222) bank48349 +48350 POINT(41.58784320005628 73.23596470203334) bank48350 +48351 POINT(41.24072506827045 74.47714332113324) bank48351 +48352 POINT(39.87311033240613 73.95840329146739) bank48352 +48353 POINT(39.85201349665152 73.9276029999449) bank48353 +48354 POINT(40.49342249107046 73.40981256580548) bank48354 +48355 POINT(40.710160836552745 73.16130175049983) bank48355 +48356 POINT(40.089452802087564 74.04706373088926) bank48356 +48357 POINT(41.55454506725469 74.78331625866952) bank48357 +48358 POINT(41.5280471824899 74.77631014151757) bank48358 +48359 POINT(40.46508044081618 73.34684201967801) bank48359 +48360 POINT(39.86098195014476 73.47452491862158) bank48360 +48361 POINT(41.1754803782854 73.34988511980413) bank48361 +48362 POINT(39.73434756499082 74.2052982980877) bank48362 +48363 POINT(39.89320585213646 74.70746853148219) bank48363 +48364 POINT(39.96051536941255 73.87521217406511) bank48364 +48365 POINT(40.39207988415285 74.0952283897685) bank48365 +48366 POINT(40.6065914113867 73.03665305282199) bank48366 +48367 POINT(41.483713756333636 74.26066293367933) bank48367 +48368 POINT(40.31240665980896 74.51503109786167) bank48368 +48369 POINT(40.545767634298265 74.97282622508804) bank48369 +48370 POINT(39.80157487264363 73.90172359423023) bank48370 +48371 POINT(40.332008555239256 73.33962751486244) bank48371 +48372 POINT(41.1218191358395 73.62826023940538) bank48372 +48373 POINT(40.18046211009014 74.7484744466507) bank48373 +48374 POINT(40.52888261842395 74.1208495777005) bank48374 +48375 POINT(39.93776503826197 74.88396271903495) bank48375 +48376 POINT(41.15491842868932 73.9288143157554) bank48376 +48377 POINT(40.05377826700966 73.92113546533919) bank48377 +48378 POINT(40.64185990651926 74.78546406249927) bank48378 +48379 POINT(40.660851222270416 74.34720354315192) bank48379 +48380 POINT(39.798046920471954 74.18408164586045) bank48380 +48381 POINT(40.20314141114105 73.6338277827499) bank48381 +48382 POINT(40.362758207201985 74.1223016774326) bank48382 +48383 POINT(39.7175334133074 73.50300469790871) bank48383 +48384 POINT(41.467207791867686 73.22942182525891) bank48384 +48385 POINT(40.89242928000815 73.65742997706216) bank48385 +48386 POINT(40.40555537685658 74.6266435612951) bank48386 +48387 POINT(40.520345915445695 74.08634705860112) bank48387 +48388 POINT(40.74685328372307 73.35909522023537) bank48388 +48389 POINT(41.461258982051746 73.74403621295745) bank48389 +48390 POINT(41.09104830757555 73.24334158040094) bank48390 +48391 POINT(41.175225662812174 74.80566519149387) bank48391 +48392 POINT(40.11485000066413 73.3399821045808) bank48392 +48393 POINT(41.36202652942338 74.22206028674252) bank48393 +48394 POINT(40.879815315556435 74.87223589601597) bank48394 +48395 POINT(40.77313754306256 74.35003486588461) bank48395 +48396 POINT(41.51403251054334 73.30850957783649) bank48396 +48397 POINT(41.19973098848678 74.50235117983273) bank48397 +48398 POINT(41.698150455360874 74.6198290042746) bank48398 +48399 POINT(40.39629327616503 74.77009453763819) bank48399 +48400 POINT(41.660096136637996 73.41623123189774) bank48400 +48401 POINT(40.07776573449213 73.62537849287267) bank48401 +48402 POINT(40.71527331128969 74.24906480417735) bank48402 +48403 POINT(41.11199552910665 74.30488399383756) bank48403 +48404 POINT(41.22254027711283 73.81008137529531) bank48404 +48405 POINT(40.761966193128416 73.94769215563065) bank48405 +48406 POINT(41.160064469636595 74.82253192945515) bank48406 +48407 POINT(40.78456150553531 74.844607364593) bank48407 +48408 POINT(41.687125624550326 74.45059215179134) bank48408 +48409 POINT(40.5500814485717 74.2227574040756) bank48409 +48410 POINT(41.298555935958134 74.63740986731563) bank48410 +48411 POINT(39.88371283469419 74.09734759386835) bank48411 +48412 POINT(40.966597419745064 73.68092445453367) bank48412 +48413 POINT(41.34372778096054 73.60054311701386) bank48413 +48414 POINT(40.41033329383845 73.30912149253432) bank48414 +48415 POINT(39.84861899365293 74.1791094248342) bank48415 +48416 POINT(40.442700005771265 73.42746102223066) bank48416 +48417 POINT(41.04873612599234 73.73091348922512) bank48417 +48418 POINT(41.15127866828153 74.11232958571229) bank48418 +48419 POINT(41.15396557878956 73.16456484160778) bank48419 +48420 POINT(41.518568408705846 73.94623399712967) bank48420 +48421 POINT(40.948285703165915 73.53489648740171) bank48421 +48422 POINT(41.3321850665213 73.20033586750775) bank48422 +48423 POINT(39.81321827100585 73.66380774183425) bank48423 +48424 POINT(40.38328546219114 73.64177135292095) bank48424 +48425 POINT(41.66629606612172 73.22763074055598) bank48425 +48426 POINT(40.71327472118356 74.06728556378917) bank48426 +48427 POINT(41.52287463122152 74.83228648029693) bank48427 +48428 POINT(41.334880099043374 74.64778445659718) bank48428 +48429 POINT(40.69149205572737 73.8159735004035) bank48429 +48430 POINT(41.275840917342016 74.68300337598217) bank48430 +48431 POINT(40.795449406164465 74.65677421023553) bank48431 +48432 POINT(40.01493761850449 73.63819381676184) bank48432 +48433 POINT(39.80271078758433 73.46894744632112) bank48433 +48434 POINT(41.54190608982782 74.9087872160428) bank48434 +48435 POINT(40.32427024747327 74.76247113205561) bank48435 +48436 POINT(41.026441510632424 73.58515806583794) bank48436 +48437 POINT(40.83875318112752 73.85728925571831) bank48437 +48438 POINT(40.905049839276224 73.16595086561578) bank48438 +48439 POINT(39.765999056856785 74.81841723092943) bank48439 +48440 POINT(41.64568415618961 74.56007100216117) bank48440 +48441 POINT(41.13519596440261 74.13181871721295) bank48441 +48442 POINT(39.80199410150935 74.91059929906778) bank48442 +48443 POINT(40.04704595343163 73.72055193760573) bank48443 +48444 POINT(40.13856012753181 73.979997284521) bank48444 +48445 POINT(40.158227654113034 74.22271837209287) bank48445 +48446 POINT(40.13273606911184 73.79724379772608) bank48446 +48447 POINT(39.77261909513517 74.27999787319078) bank48447 +48448 POINT(39.77893101927182 73.96591634641443) bank48448 +48449 POINT(41.12068018779306 74.61104514379771) bank48449 +48450 POINT(40.66462393217835 73.71093340646871) bank48450 +48451 POINT(41.023635408811444 74.27230730277182) bank48451 +48452 POINT(41.53041166580809 73.37420686442256) bank48452 +48453 POINT(40.806505674777746 73.88024467792603) bank48453 +48454 POINT(41.46013353283583 73.47345874810861) bank48454 +48455 POINT(41.6282964205846 73.15130161494187) bank48455 +48456 POINT(41.462897341876264 74.56881028972576) bank48456 +48457 POINT(41.68774023900253 74.38182704791748) bank48457 +48458 POINT(41.31081788907292 73.15593047529485) bank48458 +48459 POINT(40.1870147089006 73.13570801586616) bank48459 +48460 POINT(40.60107531859056 73.62930024347104) bank48460 +48461 POINT(41.223949683475546 74.95794198124108) bank48461 +48462 POINT(40.59784220664035 73.83804744202472) bank48462 +48463 POINT(40.225669438600455 74.33530969333935) bank48463 +48464 POINT(41.00811903192331 73.13083409872776) bank48464 +48465 POINT(39.73150873793111 73.34756518057675) bank48465 +48466 POINT(41.56621210393491 74.26165305976306) bank48466 +48467 POINT(39.815904581940764 73.60519331545473) bank48467 +48468 POINT(40.0500149596082 74.67634027285433) bank48468 +48469 POINT(39.87514882097548 73.29218523337791) bank48469 +48470 POINT(40.34506367353522 73.50282177199564) bank48470 +48471 POINT(39.77576936189891 74.27780314972026) bank48471 +48472 POINT(40.5103499619684 74.82006913488539) bank48472 +48473 POINT(41.46295770685131 73.10474269841578) bank48473 +48474 POINT(40.04353925919226 73.17387753766016) bank48474 +48475 POINT(39.93333437297942 73.61002917663492) bank48475 +48476 POINT(40.82266573370018 74.20346171837309) bank48476 +48477 POINT(39.98233670679453 73.94993933362464) bank48477 +48478 POINT(41.05882254276141 74.8484423722901) bank48478 +48479 POINT(40.70845522093071 74.07677827379207) bank48479 +48480 POINT(41.570267741330845 74.07698775985948) bank48480 +48481 POINT(39.762211034244785 74.40403831301825) bank48481 +48482 POINT(40.76004711385159 74.9353086157364) bank48482 +48483 POINT(41.55115795159788 73.429652219566) bank48483 +48484 POINT(40.111422089536305 74.25077303686663) bank48484 +48485 POINT(39.911667843043055 73.5396866487448) bank48485 +48486 POINT(41.5072216514578 73.44854206130917) bank48486 +48487 POINT(41.3458592546348 74.04960522915442) bank48487 +48488 POINT(39.9458524169336 74.21503511593477) bank48488 +48489 POINT(40.10172381053992 74.51115071024381) bank48489 +48490 POINT(40.951618865861114 74.48909854007692) bank48490 +48491 POINT(40.67622428527306 73.48857168442731) bank48491 +48492 POINT(39.912795564460005 73.8376578711573) bank48492 +48493 POINT(40.793979399186085 74.73179527964355) bank48493 +48494 POINT(41.548217312611605 73.66934005832614) bank48494 +48495 POINT(39.79445044679927 74.27262931829307) bank48495 +48496 POINT(40.11460898187575 74.32456260324697) bank48496 +48497 POINT(40.33246115310362 74.02961910034882) bank48497 +48498 POINT(41.68029814647284 74.9533191133558) bank48498 +48499 POINT(41.54832770734177 73.27500798351129) bank48499 +48500 POINT(40.11082177838827 73.13868902979698) bank48500 +48501 POINT(40.72669176506849 73.03738138945265) bank48501 +48502 POINT(40.554944134151846 73.49498175445672) bank48502 +48503 POINT(39.95548585083054 73.86732614303911) bank48503 +48504 POINT(40.15709988584011 73.21393512022813) bank48504 +48505 POINT(40.78720144075072 73.88211018232609) bank48505 +48506 POINT(41.593156969834894 73.52383718502907) bank48506 +48507 POINT(41.05242228841487 74.57106793381065) bank48507 +48508 POINT(40.590101038650516 73.31058754160806) bank48508 +48509 POINT(40.94437815451046 73.94039181994286) bank48509 +48510 POINT(41.469502273546254 73.78931220195936) bank48510 +48511 POINT(40.740604472662795 73.44282379564278) bank48511 +48512 POINT(39.72864401857186 74.14644590682228) bank48512 +48513 POINT(40.75261279107441 73.29687919048612) bank48513 +48514 POINT(41.70423123410962 73.03987881720508) bank48514 +48515 POINT(40.114204234126966 73.31227907526336) bank48515 +48516 POINT(40.92334418742083 74.4763798025172) bank48516 +48517 POINT(39.92155305767239 74.0566624129763) bank48517 +48518 POINT(40.15072770619098 73.41222519740968) bank48518 +48519 POINT(40.466614355320004 74.14235808832574) bank48519 +48520 POINT(40.22739331623605 73.20015299134658) bank48520 +48521 POINT(40.62658734798804 74.78592962569336) bank48521 +48522 POINT(41.379968245454734 73.11138395826517) bank48522 +48523 POINT(39.72192591593194 74.33836190995702) bank48523 +48524 POINT(40.282751273675615 73.06770977055507) bank48524 +48525 POINT(41.15840525860172 74.94726485218926) bank48525 +48526 POINT(39.925284047706555 74.78484545414604) bank48526 +48527 POINT(39.961180626025396 74.90079925088244) bank48527 +48528 POINT(41.134109142091035 73.16564759667254) bank48528 +48529 POINT(40.302825882354036 74.4097297443097) bank48529 +48530 POINT(40.07211764658324 73.8340062926725) bank48530 +48531 POINT(41.31054111310613 73.43232816148301) bank48531 +48532 POINT(40.254476262131284 74.16743501663014) bank48532 +48533 POINT(41.60239487994446 73.70822375859669) bank48533 +48534 POINT(40.36872247082516 74.87100257270376) bank48534 +48535 POINT(41.70107666478625 74.55819803297) bank48535 +48536 POINT(40.415089192474646 73.11192471674464) bank48536 +48537 POINT(41.17330665673811 74.3756438417219) bank48537 +48538 POINT(41.27516178962432 73.99220772117069) bank48538 +48539 POINT(41.1138550087875 74.97607349391767) bank48539 +48540 POINT(40.351657947441225 74.3596202925124) bank48540 +48541 POINT(39.79214792471599 73.75111979986957) bank48541 +48542 POINT(40.82983006527124 74.85625217806532) bank48542 +48543 POINT(40.10555339057853 73.1798564900413) bank48543 +48544 POINT(40.453315990535216 74.3759031463119) bank48544 +48545 POINT(40.170228481076926 74.9949235155575) bank48545 +48546 POINT(41.418917863893164 74.40894905076196) bank48546 +48547 POINT(40.45062792646587 73.55862936638107) bank48547 +48548 POINT(40.507164907248075 73.74128524237764) bank48548 +48549 POINT(40.72666378731112 73.22043706525132) bank48549 +48550 POINT(41.20025590112304 74.3533662818864) bank48550 +48551 POINT(40.87634221919277 74.0390104418331) bank48551 +48552 POINT(39.830214206203 74.75229436013257) bank48552 +48553 POINT(40.93278190816967 74.21110544662777) bank48553 +48554 POINT(40.659108335186005 74.05227326265943) bank48554 +48555 POINT(40.1023794713266 74.98006217973031) bank48555 +48556 POINT(41.11566115755209 74.55837096045047) bank48556 +48557 POINT(40.847688728674235 73.45477185045567) bank48557 +48558 POINT(40.84805708212362 73.13778668298768) bank48558 +48559 POINT(40.53475238012597 74.75262820439593) bank48559 +48560 POINT(39.71623015932117 74.22069749373006) bank48560 +48561 POINT(40.94649573097712 74.8460136907433) bank48561 +48562 POINT(40.89950743329418 73.51401081635275) bank48562 +48563 POINT(40.5727656652668 73.95582932915102) bank48563 +48564 POINT(39.86112871757842 73.35791599070613) bank48564 +48565 POINT(41.03233577944749 73.64751578274856) bank48565 +48566 POINT(41.40271717674823 74.11571474007805) bank48566 +48567 POINT(41.5704890601629 73.09196572350952) bank48567 +48568 POINT(39.85264637030367 74.77293396264699) bank48568 +48569 POINT(40.06646883257645 74.61185948915508) bank48569 +48570 POINT(41.678847425507676 73.66663855784395) bank48570 +48571 POINT(41.64618299862667 73.71270110467782) bank48571 +48572 POINT(40.886463538677816 74.41313320572583) bank48572 +48573 POINT(40.55049817875564 73.28108631341928) bank48573 +48574 POINT(40.7493317344833 73.86933057449757) bank48574 +48575 POINT(40.55328338346034 73.16306550944526) bank48575 +48576 POINT(41.390142352031724 73.89509385716916) bank48576 +48577 POINT(40.366051073226295 73.01422543908915) bank48577 +48578 POINT(40.57344073158198 74.92077221365872) bank48578 +48579 POINT(41.337099100366125 73.24610917933673) bank48579 +48580 POINT(40.914718485324606 73.20011239342257) bank48580 +48581 POINT(41.65459682384489 74.02731515298798) bank48581 +48582 POINT(40.01811429520016 74.77302842432825) bank48582 +48583 POINT(41.0566871021575 74.78258599763151) bank48583 +48584 POINT(41.134564528502274 73.71142107690862) bank48584 +48585 POINT(39.92295120808591 74.16558182729486) bank48585 +48586 POINT(41.35455343341359 74.90341094845735) bank48586 +48587 POINT(39.75129591749276 74.17472007093849) bank48587 +48588 POINT(40.448652159298895 74.34482823811709) bank48588 +48589 POINT(40.08058380014155 74.35102972178333) bank48589 +48590 POINT(39.84931632470561 74.39974890718806) bank48590 +48591 POINT(39.93366714593067 74.30688902932968) bank48591 +48592 POINT(41.0102577003517 74.67807519233224) bank48592 +48593 POINT(40.33788312521285 73.32637508264779) bank48593 +48594 POINT(39.9097074774756 73.68886878886254) bank48594 +48595 POINT(39.83280241181584 73.6531899120799) bank48595 +48596 POINT(40.474320351094505 73.29775906693652) bank48596 +48597 POINT(40.305132940088626 74.6423654135229) bank48597 +48598 POINT(39.91658462083427 73.09586389887886) bank48598 +48599 POINT(39.859981856329746 74.41420346372169) bank48599 +48600 POINT(41.540305309277926 74.32697880372172) bank48600 +48601 POINT(41.30597102646396 74.88317007251692) bank48601 +48602 POINT(41.00318775576713 74.53369489881746) bank48602 +48603 POINT(41.4202169475536 73.78862038611773) bank48603 +48604 POINT(40.078136835581425 74.80480381612274) bank48604 +48605 POINT(40.408404449182704 73.50174349139868) bank48605 +48606 POINT(40.82216669548827 74.26057883032395) bank48606 +48607 POINT(39.8032097782725 73.17735732324283) bank48607 +48608 POINT(41.25226827976526 73.930932162726) bank48608 +48609 POINT(39.87751244855619 74.40489426456526) bank48609 +48610 POINT(40.441922198850726 73.50962166872412) bank48610 +48611 POINT(39.800297772668074 73.09889730606106) bank48611 +48612 POINT(40.69781257604701 73.114174062669) bank48612 +48613 POINT(40.23080907523267 73.24957224974818) bank48613 +48614 POINT(40.799334234826496 74.76937943625362) bank48614 +48615 POINT(40.48320150717386 74.18499872058241) bank48615 +48616 POINT(39.75166603069318 73.77986846523564) bank48616 +48617 POINT(41.37295636894908 73.58138214334771) bank48617 +48618 POINT(40.69609164158722 74.21938791630065) bank48618 +48619 POINT(40.00937736251701 74.32910657593578) bank48619 +48620 POINT(41.427505323826786 73.38714031211556) bank48620 +48621 POINT(40.83381757586888 74.23640799473365) bank48621 +48622 POINT(40.64713426794992 74.51318200661399) bank48622 +48623 POINT(40.92619103783091 73.17925284991517) bank48623 +48624 POINT(39.916929399393645 74.97894402249672) bank48624 +48625 POINT(39.9831318793162 73.53011279339272) bank48625 +48626 POINT(39.76723674864799 73.40144974213445) bank48626 +48627 POINT(39.99583094319403 73.76858637063216) bank48627 +48628 POINT(39.929639355903696 74.34224276935639) bank48628 +48629 POINT(41.462265157572965 74.96142424840956) bank48629 +48630 POINT(40.30554890222764 73.2498677922311) bank48630 +48631 POINT(40.82203294540726 74.0769424911837) bank48631 +48632 POINT(40.023058542016194 73.15214677954268) bank48632 +48633 POINT(40.54380614729616 74.75651752784503) bank48633 +48634 POINT(41.570027044029054 73.96273565388016) bank48634 +48635 POINT(40.40856089820623 73.40121268116702) bank48635 +48636 POINT(39.9173871629948 73.80895944586128) bank48636 +48637 POINT(41.679248903368084 73.98431335620498) bank48637 +48638 POINT(40.40618693655648 73.19994652893511) bank48638 +48639 POINT(41.22249747887955 73.63790506548033) bank48639 +48640 POINT(40.86502089505021 73.63065436482714) bank48640 +48641 POINT(41.28062961480198 73.09325767907113) bank48641 +48642 POINT(40.10991857632374 74.19644911834007) bank48642 +48643 POINT(41.21711218187305 74.0342411137678) bank48643 +48644 POINT(41.4213022506415 74.96895667591056) bank48644 +48645 POINT(41.53729047636845 74.5200669546605) bank48645 +48646 POINT(39.846096490010865 74.35255948846898) bank48646 +48647 POINT(40.840415223130236 73.94463296452541) bank48647 +48648 POINT(40.270816121666684 73.95622346703993) bank48648 +48649 POINT(41.56285521132613 74.05629914142116) bank48649 +48650 POINT(41.214185965624225 74.53271391256308) bank48650 +48651 POINT(41.332116281460074 73.98873901309653) bank48651 +48652 POINT(40.77527681095017 73.62926689057738) bank48652 +48653 POINT(41.37816013747286 74.74831356723337) bank48653 +48654 POINT(41.22196665444981 74.0034640993271) bank48654 +48655 POINT(41.135626143305686 73.29391938008949) bank48655 +48656 POINT(40.366656720474495 73.8356971341454) bank48656 +48657 POINT(40.470382546857905 73.88527946889062) bank48657 +48658 POINT(39.809911076670254 73.95969863940122) bank48658 +48659 POINT(40.27485942041114 74.98353692114163) bank48659 +48660 POINT(40.018155560308 74.99434866439584) bank48660 +48661 POINT(39.728876361427425 74.69382117011551) bank48661 +48662 POINT(39.9921082355872 74.58980304169921) bank48662 +48663 POINT(39.923971489467135 74.19055474187948) bank48663 +48664 POINT(40.029374543860136 74.83573029789446) bank48664 +48665 POINT(40.05727684278634 73.4804325714804) bank48665 +48666 POINT(40.76470294096428 73.09444069154436) bank48666 +48667 POINT(41.15817365203461 73.25904898232729) bank48667 +48668 POINT(41.09214609372528 74.99406343446552) bank48668 +48669 POINT(40.28591192872697 73.35836558117057) bank48669 +48670 POINT(39.88582723393843 73.6530297173885) bank48670 +48671 POINT(41.2145629622806 74.58123824739516) bank48671 +48672 POINT(40.92243876655746 74.92874478910986) bank48672 +48673 POINT(39.87377736609643 74.59319238187528) bank48673 +48674 POINT(39.820951325042444 73.17384518787216) bank48674 +48675 POINT(40.93565244017564 74.4124267929295) bank48675 +48676 POINT(40.79097905613866 73.25463417002783) bank48676 +48677 POINT(40.61623932726481 73.78176059302574) bank48677 +48678 POINT(41.4043340163489 73.78554728952196) bank48678 +48679 POINT(40.16676505468001 73.61349607002253) bank48679 +48680 POINT(41.543155467242094 74.62101167299582) bank48680 +48681 POINT(41.37009260730418 74.77958749991188) bank48681 +48682 POINT(40.98406214201922 74.6905153601276) bank48682 +48683 POINT(40.23928677324817 74.9322575871296) bank48683 +48684 POINT(40.92033264931593 73.61884562566925) bank48684 +48685 POINT(40.758876374975564 73.10234203557924) bank48685 +48686 POINT(40.18172855190611 73.86508069204122) bank48686 +48687 POINT(40.99156046931512 74.2792259760092) bank48687 +48688 POINT(39.75326348738182 73.07062851423619) bank48688 +48689 POINT(40.092118188783935 73.26804806818237) bank48689 +48690 POINT(40.93260712203637 74.40978320798496) bank48690 +48691 POINT(40.37783804384665 74.41572437312007) bank48691 +48692 POINT(41.472079182546054 74.15948574272652) bank48692 +48693 POINT(41.45757930566706 74.78957981843106) bank48693 +48694 POINT(40.42431964678863 74.06519161105487) bank48694 +48695 POINT(41.58464934940006 73.95618894322632) bank48695 +48696 POINT(40.43601324925186 74.8595211633207) bank48696 +48697 POINT(41.1782048614089 74.23582679508759) bank48697 +48698 POINT(40.27138537173579 74.54719095774584) bank48698 +48699 POINT(39.95559689145347 73.79129856795001) bank48699 +48700 POINT(40.97164346129838 73.41138195784382) bank48700 +48701 POINT(40.20639899314057 74.58121855030284) bank48701 +48702 POINT(41.60385024828392 74.98261700929453) bank48702 +48703 POINT(41.199086690421666 74.06188391824988) bank48703 +48704 POINT(40.49461048886265 74.7212116890921) bank48704 +48705 POINT(40.26858146130494 74.39357808454952) bank48705 +48706 POINT(40.308572933050684 74.27294101654583) bank48706 +48707 POINT(40.55066032834375 74.32181249667572) bank48707 +48708 POINT(40.502679110025966 74.60073838942535) bank48708 +48709 POINT(41.21262076439006 74.44693258522739) bank48709 +48710 POINT(40.20797797365287 73.3582533602336) bank48710 +48711 POINT(40.906109204013035 74.20917849392188) bank48711 +48712 POINT(40.29835590975068 73.43420827137332) bank48712 +48713 POINT(40.386737631902285 73.04179135366854) bank48713 +48714 POINT(39.84702485343817 73.16637231743826) bank48714 +48715 POINT(41.372627158705875 74.44796726360299) bank48715 +48716 POINT(40.07907834471393 73.94141784489528) bank48716 +48717 POINT(40.07220515130431 73.85935279986752) bank48717 +48718 POINT(40.28107717122363 73.83624712159151) bank48718 +48719 POINT(39.74304617930476 74.19947717761619) bank48719 +48720 POINT(41.404986664886444 73.51864222440933) bank48720 +48721 POINT(41.59765202446869 73.93085045663769) bank48721 +48722 POINT(40.25501877886295 73.135442579664) bank48722 +48723 POINT(39.74634044014063 74.87946718443747) bank48723 +48724 POINT(40.98306054079726 74.00807704693463) bank48724 +48725 POINT(41.62488730872001 73.79635429474077) bank48725 +48726 POINT(40.25834804877983 73.51596132287426) bank48726 +48727 POINT(41.35982214448394 73.3688962395279) bank48727 +48728 POINT(41.11645566337543 74.58572720573639) bank48728 +48729 POINT(41.033738238713305 74.64900401561259) bank48729 +48730 POINT(40.89847625724469 73.74158758523384) bank48730 +48731 POINT(41.430675751529584 73.67873948575712) bank48731 +48732 POINT(41.2231667411425 74.1374943368537) bank48732 +48733 POINT(41.266555051052734 74.13216547419239) bank48733 +48734 POINT(40.63805649612835 74.38535579306641) bank48734 +48735 POINT(41.5958958159313 74.66896890590209) bank48735 +48736 POINT(40.508083988388336 73.03969194281966) bank48736 +48737 POINT(41.4999490382296 74.87533868717632) bank48737 +48738 POINT(40.85603267412945 73.90366804157621) bank48738 +48739 POINT(40.711666753855056 74.9506463941868) bank48739 +48740 POINT(40.424055521462854 74.8604866459716) bank48740 +48741 POINT(40.0858279298061 73.56050561148591) bank48741 +48742 POINT(40.84655275929748 73.06956835811368) bank48742 +48743 POINT(40.021884254135685 74.67390066738268) bank48743 +48744 POINT(40.74221456322353 73.90384828322776) bank48744 +48745 POINT(41.22025632193418 73.65471260972166) bank48745 +48746 POINT(40.51210116583078 74.28279074124848) bank48746 +48747 POINT(40.66961846000215 74.76245471725126) bank48747 +48748 POINT(41.61812189693059 74.7248112149017) bank48748 +48749 POINT(40.71300355243936 74.58065213537117) bank48749 +48750 POINT(41.3082238886611 74.0435324875836) bank48750 +48751 POINT(39.95438048430874 74.41452340792661) bank48751 +48752 POINT(40.48562094908819 73.60654461394805) bank48752 +48753 POINT(39.721990182478365 73.25868963814956) bank48753 +48754 POINT(40.960295257658366 73.53077969618752) bank48754 +48755 POINT(39.94747306043673 73.72450754656398) bank48755 +48756 POINT(40.66264980017334 74.31159982872659) bank48756 +48757 POINT(39.795099924411126 74.37636615896248) bank48757 +48758 POINT(40.99641337166428 73.10388023897633) bank48758 +48759 POINT(39.846354373304315 74.31096243855255) bank48759 +48760 POINT(40.64150434724436 74.2442395396826) bank48760 +48761 POINT(39.77307314178039 73.04581860791023) bank48761 +48762 POINT(41.469242605114474 74.52071029701412) bank48762 +48763 POINT(41.23261096368976 73.36170039999374) bank48763 +48764 POINT(39.751456180051164 73.66501211178175) bank48764 +48765 POINT(40.254889084805725 74.40534851198534) bank48765 +48766 POINT(41.485708630214006 73.50115533847229) bank48766 +48767 POINT(40.91698368696276 74.6736561310443) bank48767 +48768 POINT(41.22028595333077 74.53965606520657) bank48768 +48769 POINT(40.56982065965749 73.69149326813056) bank48769 +48770 POINT(40.13795235111751 73.5226072019188) bank48770 +48771 POINT(40.473037888494446 73.07419629845627) bank48771 +48772 POINT(40.51085133558392 73.80268613433192) bank48772 +48773 POINT(41.66055195714982 74.6140037531651) bank48773 +48774 POINT(41.12996334973398 73.46383763818946) bank48774 +48775 POINT(39.86638069353291 73.25325354728518) bank48775 +48776 POINT(40.72093166710024 73.9992539296989) bank48776 +48777 POINT(41.240163803789486 73.57886675484173) bank48777 +48778 POINT(40.24766409396682 73.16456125088115) bank48778 +48779 POINT(41.595981223682955 74.11739327593106) bank48779 +48780 POINT(40.96048370592576 73.86347055601676) bank48780 +48781 POINT(41.680069596865195 73.23312280100613) bank48781 +48782 POINT(40.16916317802661 74.7480535971022) bank48782 +48783 POINT(40.09916782178185 74.15403217021563) bank48783 +48784 POINT(40.25986851850906 73.08762680710117) bank48784 +48785 POINT(40.69560097143322 74.5012405846002) bank48785 +48786 POINT(40.253669322380865 73.27056873484946) bank48786 +48787 POINT(40.5839125252974 73.77211289160543) bank48787 +48788 POINT(39.77631292015961 74.59966605852627) bank48788 +48789 POINT(40.70117887345691 74.39790911290629) bank48789 +48790 POINT(41.42505184153675 74.55072860756579) bank48790 +48791 POINT(40.82575927911937 74.57008911933231) bank48791 +48792 POINT(41.48666725129621 74.13148646222352) bank48792 +48793 POINT(39.895602313611185 73.38164698392875) bank48793 +48794 POINT(39.933601198019375 74.66387458538884) bank48794 +48795 POINT(41.32927715214108 74.53144265653553) bank48795 +48796 POINT(40.159215023295666 74.96557162590368) bank48796 +48797 POINT(41.08902722105592 74.8696428731888) bank48797 +48798 POINT(41.642009406973486 74.5820982264808) bank48798 +48799 POINT(40.09302724840195 74.92174860119971) bank48799 +48800 POINT(40.30726416195058 73.52818968775792) bank48800 +48801 POINT(41.661246021890726 74.79336583413192) bank48801 +48802 POINT(41.03599162869366 73.02041022048576) bank48802 +48803 POINT(41.20596574483673 73.3304944758971) bank48803 +48804 POINT(40.7406713247965 74.33256301910542) bank48804 +48805 POINT(39.99756163347246 74.29554990563832) bank48805 +48806 POINT(40.6034864387762 74.40354640412683) bank48806 +48807 POINT(40.76450953881688 74.69029350251829) bank48807 +48808 POINT(41.03294444837091 73.53347883624883) bank48808 +48809 POINT(40.59892815250725 74.4314722165025) bank48809 +48810 POINT(40.94226934671279 74.95160501111322) bank48810 +48811 POINT(41.55704648053115 74.63127518158852) bank48811 +48812 POINT(40.05129256102781 74.11784174786897) bank48812 +48813 POINT(40.843072526830554 73.49625576682686) bank48813 +48814 POINT(39.739830774432505 73.81315962606479) bank48814 +48815 POINT(40.531650363747104 74.87445451451198) bank48815 +48816 POINT(40.13265375596099 74.91649500570018) bank48816 +48817 POINT(41.53826681885283 74.72463654779591) bank48817 +48818 POINT(41.69849086389465 74.99670839052274) bank48818 +48819 POINT(41.488746292270484 73.33673461907576) bank48819 +48820 POINT(39.78554918597755 73.32020307636274) bank48820 +48821 POINT(41.55740762498134 73.72174845139791) bank48821 +48822 POINT(40.61427066047983 73.38068854489896) bank48822 +48823 POINT(39.93290493573126 74.22980927057243) bank48823 +48824 POINT(40.830477061930004 73.30627324527055) bank48824 +48825 POINT(40.60744850740644 73.15718110499228) bank48825 +48826 POINT(39.8967766392457 73.15601037954983) bank48826 +48827 POINT(40.182423144917344 73.648954756739) bank48827 +48828 POINT(41.55069703978447 74.42490521449422) bank48828 +48829 POINT(40.96518763882604 73.69791498670266) bank48829 +48830 POINT(41.04828602173932 74.45326209252566) bank48830 +48831 POINT(39.84304528740231 73.79718394563372) bank48831 +48832 POINT(40.65139116157139 74.3078226577755) bank48832 +48833 POINT(40.45585906205411 73.32018706338403) bank48833 +48834 POINT(40.82749104242553 74.35041422063117) bank48834 +48835 POINT(40.39319022232985 74.13129141651712) bank48835 +48836 POINT(40.95244325889792 74.28583200788383) bank48836 +48837 POINT(39.93630212363104 74.17585581420424) bank48837 +48838 POINT(39.77538802616381 73.04278303473967) bank48838 +48839 POINT(41.58003692191875 73.12795091716883) bank48839 +48840 POINT(41.56332660000962 73.59802750080914) bank48840 +48841 POINT(41.28214717184231 73.70918925269173) bank48841 +48842 POINT(40.35313241173114 73.40282796478581) bank48842 +48843 POINT(39.93449928687133 74.36794069553946) bank48843 +48844 POINT(41.22125732032134 74.87901187262828) bank48844 +48845 POINT(39.75740210971132 74.0605173298225) bank48845 +48846 POINT(40.23339871424398 74.77716685611371) bank48846 +48847 POINT(40.47783320314343 74.29532503713443) bank48847 +48848 POINT(41.29842500409203 74.50054170058596) bank48848 +48849 POINT(41.517629514766405 73.26891008610863) bank48849 +48850 POINT(41.241158036787006 74.14432535951313) bank48850 +48851 POINT(39.977541464018415 74.6060493672905) bank48851 +48852 POINT(40.00270424008114 74.22629048256374) bank48852 +48853 POINT(41.43194413586774 74.51642163366017) bank48853 +48854 POINT(41.68940119723386 74.32089491603917) bank48854 +48855 POINT(39.75342516876453 73.44842788462967) bank48855 +48856 POINT(40.918626311335544 74.6458836779049) bank48856 +48857 POINT(40.26553826281116 74.19089867982562) bank48857 +48858 POINT(41.56635887189289 73.8745988122514) bank48858 +48859 POINT(41.04060753005858 73.5699092519896) bank48859 +48860 POINT(40.26783583908737 73.05203052981174) bank48860 +48861 POINT(40.34740912303163 74.05457452860776) bank48861 +48862 POINT(40.54462414799175 74.97176073927577) bank48862 +48863 POINT(40.571417732323646 74.89207811642406) bank48863 +48864 POINT(40.161922296755364 73.88774498326497) bank48864 +48865 POINT(39.79058828681883 73.3885122512481) bank48865 +48866 POINT(41.450832006821265 73.16203885153583) bank48866 +48867 POINT(41.41394169384659 74.70216832291554) bank48867 +48868 POINT(40.89589893442416 74.49766102790443) bank48868 +48869 POINT(40.10009241803647 73.26905313692218) bank48869 +48870 POINT(40.3727734274718 73.45182059514191) bank48870 +48871 POINT(40.89429668102362 73.39603028386034) bank48871 +48872 POINT(41.412412173575945 74.607845810656) bank48872 +48873 POINT(40.106930068173156 74.45538651706426) bank48873 +48874 POINT(41.562322808588746 74.39183903212137) bank48874 +48875 POINT(40.58914370837472 74.05644637353237) bank48875 +48876 POINT(40.500561256093405 73.49655306474942) bank48876 +48877 POINT(41.64500345426004 74.44035691967005) bank48877 +48878 POINT(41.06053868021056 74.61346470021826) bank48878 +48879 POINT(41.2487647583321 73.79706941477565) bank48879 +48880 POINT(40.9243944506899 74.73243947698404) bank48880 +48881 POINT(40.835265411198336 73.57226712193203) bank48881 +48882 POINT(39.74019775882388 73.80655449025029) bank48882 +48883 POINT(41.305994912592766 74.93984435447311) bank48883 +48884 POINT(40.920857468558296 73.8086997021661) bank48884 +48885 POINT(40.69951606716297 73.9451802647595) bank48885 +48886 POINT(40.71988033963645 74.8669045763852) bank48886 +48887 POINT(39.76069897733353 74.76093984645233) bank48887 +48888 POINT(41.36094444366928 74.5669516640232) bank48888 +48889 POINT(41.311890852240786 74.24822380788221) bank48889 +48890 POINT(41.53402914608596 74.91935037257967) bank48890 +48891 POINT(41.647848345199385 74.42844819561644) bank48891 +48892 POINT(40.34093660216171 74.38452281938164) bank48892 +48893 POINT(39.833146132452214 73.46716247252097) bank48893 +48894 POINT(40.42547517932785 73.4723179370994) bank48894 +48895 POINT(41.18479753844295 74.08411700525392) bank48895 +48896 POINT(40.91873492409698 73.19561568567656) bank48896 +48897 POINT(41.45920351621675 73.18190829760945) bank48897 +48898 POINT(39.86011467714986 74.9952610114412) bank48898 +48899 POINT(40.46750100565823 73.28498875497773) bank48899 +48900 POINT(41.355384431356505 74.58995536477634) bank48900 +48901 POINT(41.15721128348699 73.52161397431033) bank48901 +48902 POINT(41.390534165356875 73.19941738723448) bank48902 +48903 POINT(40.16138813209924 74.40199074117689) bank48903 +48904 POINT(40.78813508270754 73.71685996549982) bank48904 +48905 POINT(41.125565536735756 73.2459587785962) bank48905 +48906 POINT(41.021484589792145 73.57238692961747) bank48906 +48907 POINT(41.11904367084527 73.18091770540012) bank48907 +48908 POINT(41.1539542344951 74.07280352583797) bank48908 +48909 POINT(40.42751618628704 74.38472017069829) bank48909 +48910 POINT(40.46612399421522 73.04530607285793) bank48910 +48911 POINT(40.56974584559558 73.44909591593449) bank48911 +48912 POINT(40.960042936726815 73.19890955539175) bank48912 +48913 POINT(41.23135687817557 73.50757818743732) bank48913 +48914 POINT(40.15035006757263 73.3254401985127) bank48914 +48915 POINT(40.67211734500463 73.71434051123275) bank48915 +48916 POINT(40.91197624638975 74.61377040155496) bank48916 +48917 POINT(41.664776464229945 74.5066165088133) bank48917 +48918 POINT(41.063967521221144 73.83058918412169) bank48918 +48919 POINT(40.27045684885498 73.37086172125052) bank48919 +48920 POINT(41.634614480416474 73.6224709209777) bank48920 +48921 POINT(39.9173949054461 74.13094256649735) bank48921 +48922 POINT(41.503310567469526 74.41540173491495) bank48922 +48923 POINT(39.87084927675589 73.57052406766645) bank48923 +48924 POINT(39.94079771872709 73.14765279404705) bank48924 +48925 POINT(41.042560777329946 74.86895998670533) bank48925 +48926 POINT(41.06859134583786 73.27250412888394) bank48926 +48927 POINT(40.21734529808327 74.0214341135801) bank48927 +48928 POINT(40.982771234570315 74.61608132065165) bank48928 +48929 POINT(39.93643959268527 74.59341900127528) bank48929 +48930 POINT(40.06228610304573 74.54744911122913) bank48930 +48931 POINT(41.04490745231824 73.48072888586243) bank48931 +48932 POINT(41.15787912751837 73.11358707134045) bank48932 +48933 POINT(40.56904207394389 73.73524284438962) bank48933 +48934 POINT(40.08069704512093 74.62893482106632) bank48934 +48935 POINT(40.0178137571536 74.84455954628196) bank48935 +48936 POINT(39.826127396996895 73.09664256163207) bank48936 +48937 POINT(41.540986074838806 74.63122791423712) bank48937 +48938 POINT(39.98485788905468 74.67349407326991) bank48938 +48939 POINT(40.48261073005132 73.99500203194741) bank48939 +48940 POINT(40.65322216749019 74.0500890637558) bank48940 +48941 POINT(41.2615714596643 73.09236534098704) bank48941 +48942 POINT(40.761283506183375 73.94295861997155) bank48942 +48943 POINT(41.465935308712865 74.932623099781) bank48943 +48944 POINT(39.89976369304208 74.25851125862324) bank48944 +48945 POINT(41.500798020359554 73.0789271604271) bank48945 +48946 POINT(39.92508422160709 73.90867578850252) bank48946 +48947 POINT(41.508552889025026 73.57280685593597) bank48947 +48948 POINT(41.63253254787538 73.65061284375594) bank48948 +48949 POINT(41.56827961384411 74.6173821925081) bank48949 +48950 POINT(40.634313330745364 74.22330621173931) bank48950 +48951 POINT(41.004741260587174 74.39350994666965) bank48951 +48952 POINT(41.52952673351702 73.20630303909016) bank48952 +48953 POINT(41.477852431174696 73.78622075160852) bank48953 +48954 POINT(40.161070316261814 74.94392110389428) bank48954 +48955 POINT(39.98413193689181 74.27025314110726) bank48955 +48956 POINT(40.06968328992447 73.07881039229588) bank48956 +48957 POINT(41.276816330086135 74.13778559519179) bank48957 +48958 POINT(40.993958724487804 73.2141599425586) bank48958 +48959 POINT(41.50904057774589 73.89571376115697) bank48959 +48960 POINT(39.94673260322309 73.71804852910637) bank48960 +48961 POINT(39.80256551681646 73.79164214278138) bank48961 +48962 POINT(41.6604740923234 74.92790017322088) bank48962 +48963 POINT(40.60796001252146 73.13615158736653) bank48963 +48964 POINT(41.17790276702246 74.46357425367002) bank48964 +48965 POINT(40.727148195668384 74.24934576314804) bank48965 +48966 POINT(41.37067031207051 73.81774048475998) bank48966 +48967 POINT(41.211770353424896 73.29880162712097) bank48967 +48968 POINT(40.78659423729291 74.69079477737888) bank48968 +48969 POINT(41.27154181181369 74.93222685324434) bank48969 +48970 POINT(40.12837227328672 74.29740875203638) bank48970 +48971 POINT(40.82618535180445 74.66820230663733) bank48971 +48972 POINT(39.71908372670613 74.66462831511929) bank48972 +48973 POINT(41.3806601060859 74.38486199287632) bank48973 +48974 POINT(40.965155614598515 74.95717312627221) bank48974 +48975 POINT(39.98892344326476 74.23940463636724) bank48975 +48976 POINT(40.00962532120633 73.84643299198575) bank48976 +48977 POINT(41.54418627801253 74.13223171946751) bank48977 +48978 POINT(41.3027884336554 73.41489706214698) bank48978 +48979 POINT(40.285768963335606 74.54666716205203) bank48979 +48980 POINT(40.229642067430724 73.49478083107515) bank48980 +48981 POINT(41.31453070531502 74.8131183168809) bank48981 +48982 POINT(40.947595636626296 74.21110881243791) bank48982 +48983 POINT(40.93609137547519 73.5978509994336) bank48983 +48984 POINT(40.63138170948098 73.79111992068282) bank48984 +48985 POINT(40.907644817582906 73.42485810166886) bank48985 +48986 POINT(40.81580295055015 73.11347610509519) bank48986 +48987 POINT(39.754180308039125 74.56668420879869) bank48987 +48988 POINT(41.43336218002878 74.46054008961133) bank48988 +48989 POINT(40.993465106535055 74.50548778161686) bank48989 +48990 POINT(40.754215599062526 73.61710665438864) bank48990 +48991 POINT(40.55969909792485 73.79510131430146) bank48991 +48992 POINT(41.02188431657859 73.50654959011534) bank48992 +48993 POINT(40.45927298527108 74.46807876173061) bank48993 +48994 POINT(40.80845378858619 73.21616440615894) bank48994 +48995 POINT(40.1264968049199 73.2528119835652) bank48995 +48996 POINT(41.32242896045034 73.30283768648718) bank48996 +48997 POINT(40.22981266091215 74.65447782102737) bank48997 +48998 POINT(41.15378522930781 73.3758246258833) bank48998 +48999 POINT(40.19954222994373 74.2583778257258) bank48999 +49000 POINT(39.9688380872932 73.2259738504549) bank49000 +49001 POINT(40.419169012927334 73.1824672044188) bank49001 +49002 POINT(39.73205482496391 74.66867902779762) bank49002 +49003 POINT(41.061313786574345 73.77729045427482) bank49003 +49004 POINT(41.34286306880635 74.79553834137846) bank49004 +49005 POINT(40.84586597211076 73.4521525062295) bank49005 +49006 POINT(40.45019558552827 73.92015964373155) bank49006 +49007 POINT(40.84962053939696 74.39318621037826) bank49007 +49008 POINT(41.43863521338827 74.01135387577143) bank49008 +49009 POINT(41.57855523282184 73.62597230312043) bank49009 +49010 POINT(41.289109806679164 74.3876392128977) bank49010 +49011 POINT(41.68255423416272 73.66949134790912) bank49011 +49012 POINT(41.3896705173637 74.11321480893558) bank49012 +49013 POINT(41.44373090511812 74.20038541223342) bank49013 +49014 POINT(41.220472359150364 73.73743418514889) bank49014 +49015 POINT(41.161236129754435 73.53854802347715) bank49015 +49016 POINT(40.56828345518245 74.65432415143691) bank49016 +49017 POINT(40.204147856898786 74.54955221011907) bank49017 +49018 POINT(40.27100431604883 74.45083535658516) bank49018 +49019 POINT(40.5193873032381 73.22782882210154) bank49019 +49020 POINT(40.506755782052124 74.26590320630478) bank49020 +49021 POINT(40.259442603976 73.1356230747053) bank49021 +49022 POINT(40.82967859509008 73.97349477683707) bank49022 +49023 POINT(39.98622652290747 74.82021132939917) bank49023 +49024 POINT(41.31573405409675 73.144779156027) bank49024 +49025 POINT(41.034837537245714 73.70529850579604) bank49025 +49026 POINT(40.63384391993547 73.2256625799191) bank49026 +49027 POINT(41.18509804864446 73.69832891184036) bank49027 +49028 POINT(39.80718634925282 73.88493828597554) bank49028 +49029 POINT(41.482709702667506 73.70187392699373) bank49029 +49030 POINT(40.66741957196448 74.295383055666) bank49030 +49031 POINT(41.55277401910191 73.66342785478956) bank49031 +49032 POINT(39.98699251340216 74.646229028629) bank49032 +49033 POINT(40.756445716913944 74.64049302030912) bank49033 +49034 POINT(40.19640898686355 73.54524309332984) bank49034 +49035 POINT(41.155462886463646 73.77141669609811) bank49035 +49036 POINT(39.72260022486823 74.81677080795556) bank49036 +49037 POINT(41.00590272574618 74.00244857089541) bank49037 +49038 POINT(40.54057069957726 73.57994613469849) bank49038 +49039 POINT(40.88832165097515 73.45789695673703) bank49039 +49040 POINT(39.845303871211456 74.18998666636426) bank49040 +49041 POINT(40.891056515829845 73.15167611383168) bank49041 +49042 POINT(40.071127759262474 74.79565212332041) bank49042 +49043 POINT(40.978434170597026 73.42602169364318) bank49043 +49044 POINT(40.043494051395946 74.78897927765598) bank49044 +49045 POINT(41.59359897855155 73.05141789760071) bank49045 +49046 POINT(40.131211499989625 74.9563153563334) bank49046 +49047 POINT(40.94184846521179 73.11079619287734) bank49047 +49048 POINT(40.0166981738117 73.79580858786021) bank49048 +49049 POINT(39.85294810777076 73.49247583119875) bank49049 +49050 POINT(40.48416529790157 74.76791706540959) bank49050 +49051 POINT(40.82022695823811 74.24723189821825) bank49051 +49052 POINT(40.44316781467515 73.07936111420595) bank49052 +49053 POINT(40.35963589593254 73.01187948575851) bank49053 +49054 POINT(40.52081650810915 74.08380285443687) bank49054 +49055 POINT(40.43766643606749 73.65124064899875) bank49055 +49056 POINT(40.428975538183344 73.08679902428699) bank49056 +49057 POINT(41.635386453630076 73.3867172713557) bank49057 +49058 POINT(41.36722479784442 73.92142792969177) bank49058 +49059 POINT(40.426928317386675 73.14213465989384) bank49059 +49060 POINT(40.56302486171732 73.41206596285518) bank49060 +49061 POINT(41.16738276896268 73.02549618120024) bank49061 +49062 POINT(41.67258491612787 74.85976210429057) bank49062 +49063 POINT(41.564898969200534 74.39568601672377) bank49063 +49064 POINT(39.7404223247279 73.69114874097271) bank49064 +49065 POINT(41.38703981421822 73.22025037635547) bank49065 +49066 POINT(40.67606928010866 73.63516337059662) bank49066 +49067 POINT(41.4435399546227 74.4903262551386) bank49067 +49068 POINT(40.322725272177415 74.59788919214132) bank49068 +49069 POINT(39.78765680664359 74.6586152818464) bank49069 +49070 POINT(40.406092347157944 73.99302437387435) bank49070 +49071 POINT(40.50134233227112 74.79648887357804) bank49071 +49072 POINT(41.25435718491353 73.68587072125196) bank49072 +49073 POINT(40.7088264380693 73.51953580173817) bank49073 +49074 POINT(39.72737710734538 73.05687109952683) bank49074 +49075 POINT(40.29098656105391 74.79105264432306) bank49075 +49076 POINT(40.38676752206922 74.09995574008781) bank49076 +49077 POINT(40.939660768734484 73.47249562371796) bank49077 +49078 POINT(41.673166047122905 73.07274885619564) bank49078 +49079 POINT(40.51018272004518 73.08322175382541) bank49079 +49080 POINT(41.15597705191945 74.40336767151423) bank49080 +49081 POINT(39.74967486140256 74.58425672473889) bank49081 +49082 POINT(40.20658859089985 74.16481369077857) bank49082 +49083 POINT(41.32083056611146 73.03173007268971) bank49083 +49084 POINT(40.75653301434626 74.37868387475062) bank49084 +49085 POINT(40.98941350444414 73.81133974335877) bank49085 +49086 POINT(40.5008711174621 73.30848652838341) bank49086 +49087 POINT(41.30010449120885 73.6372487018428) bank49087 +49088 POINT(41.14658082949447 73.51638949857178) bank49088 +49089 POINT(39.77494723954132 73.49570872554952) bank49089 +49090 POINT(40.02183454246352 73.45784187552471) bank49090 +49091 POINT(41.082606632248975 73.25513836005162) bank49091 +49092 POINT(40.309259905554676 73.40243097443262) bank49092 +49093 POINT(40.59816609115528 74.17372603265686) bank49093 +49094 POINT(41.14670828760767 74.50902657936524) bank49094 +49095 POINT(40.41063560107736 73.0546953080409) bank49095 +49096 POINT(40.460255273857314 73.4277897642303) bank49096 +49097 POINT(41.206894559329584 73.75855734077848) bank49097 +49098 POINT(41.02151272828043 73.13284261297932) bank49098 +49099 POINT(41.481362176784295 73.5281194065535) bank49099 +49100 POINT(40.5860549798735 74.77059681483696) bank49100 +49101 POINT(40.45052187015853 73.32095976370995) bank49101 +49102 POINT(41.62293146685587 73.70215666051367) bank49102 +49103 POINT(40.58869291125467 74.25542811833547) bank49103 +49104 POINT(40.057321510588174 73.67661562428485) bank49104 +49105 POINT(40.928084914599744 74.41289404828574) bank49105 +49106 POINT(40.77517519656849 74.18448028185442) bank49106 +49107 POINT(40.453438706844935 74.11734157374742) bank49107 +49108 POINT(40.05936272586719 74.63526891179649) bank49108 +49109 POINT(41.2459915937422 74.00642736853322) bank49109 +49110 POINT(40.334973457100745 74.9257019008879) bank49110 +49111 POINT(41.02514346414703 73.3347737899959) bank49111 +49112 POINT(40.67327784581005 74.53751923030416) bank49112 +49113 POINT(40.501543861080826 74.98562762178217) bank49113 +49114 POINT(40.42418335107613 74.5740003033892) bank49114 +49115 POINT(40.25295977465047 74.76080890208564) bank49115 +49116 POINT(41.51971858341317 73.70545327241885) bank49116 +49117 POINT(40.43252086639397 73.87737120598948) bank49117 +49118 POINT(41.50735371871804 73.86602301652174) bank49118 +49119 POINT(40.07488121304443 73.6682152759666) bank49119 +49120 POINT(40.60958647335546 74.43255156133158) bank49120 +49121 POINT(40.737714189430186 73.26540854224622) bank49121 +49122 POINT(41.14321448050719 73.58611595552333) bank49122 +49123 POINT(39.755961712651626 74.78745987677584) bank49123 +49124 POINT(40.76222155161725 73.15139805851341) bank49124 +49125 POINT(40.719499274420805 73.82609622451463) bank49125 +49126 POINT(40.92000551044892 74.16935959236713) bank49126 +49127 POINT(41.339095631610746 73.65684838650553) bank49127 +49128 POINT(41.15142271613743 73.74980738023677) bank49128 +49129 POINT(40.949118234626305 74.63714274987866) bank49129 +49130 POINT(41.234496990373756 73.88140201905907) bank49130 +49131 POINT(41.03782438919778 74.95067145792169) bank49131 +49132 POINT(40.41690446085593 74.38402704479547) bank49132 +49133 POINT(40.882346897164375 74.55275686616842) bank49133 +49134 POINT(40.60982827861211 73.42844425010045) bank49134 +49135 POINT(39.904444116281155 74.51700786395524) bank49135 +49136 POINT(40.042238106169144 74.66633861982993) bank49136 +49137 POINT(40.37320921126813 74.68751412615111) bank49137 +49138 POINT(41.38387440357193 74.02127245050146) bank49138 +49139 POINT(40.7291408020737 74.40245154952451) bank49139 +49140 POINT(40.74597160586166 74.1406987896066) bank49140 +49141 POINT(39.88079998086229 73.31222408022275) bank49141 +49142 POINT(41.55978793165331 73.63078437464289) bank49142 +49143 POINT(41.37477603728575 73.99440370541794) bank49143 +49144 POINT(40.4397258860481 74.81707746095294) bank49144 +49145 POINT(40.73817374074796 73.85395883053852) bank49145 +49146 POINT(41.00206495259704 74.52702316504704) bank49146 +49147 POINT(41.03225796330965 73.17943384677402) bank49147 +49148 POINT(41.478348694087515 73.38683284230238) bank49148 +49149 POINT(39.87843736031449 74.89983664014554) bank49149 +49150 POINT(41.45369888913576 73.24174038698055) bank49150 +49151 POINT(41.56341806177544 74.81351080970455) bank49151 +49152 POINT(40.56294161655876 74.20461570563648) bank49152 +49153 POINT(40.092859911159685 73.48491159792373) bank49153 +49154 POINT(40.29316124001844 73.52439630139948) bank49154 +49155 POINT(41.26216154837657 74.66813265900358) bank49155 +49156 POINT(41.473028199728866 73.9055766818796) bank49156 +49157 POINT(40.72104516273816 73.99216795443346) bank49157 +49158 POINT(40.3217475245372 73.80392277115239) bank49158 +49159 POINT(40.566169364463164 74.92907296719136) bank49159 +49160 POINT(39.73460621748029 74.44687265780384) bank49160 +49161 POINT(40.13898367368021 73.36730948932431) bank49161 +49162 POINT(40.3114354416948 73.2902274740567) bank49162 +49163 POINT(40.06744052046078 74.44825438470149) bank49163 +49164 POINT(40.19311196115754 74.25253656238489) bank49164 +49165 POINT(40.64540401265667 74.99570107265139) bank49165 +49166 POINT(41.57597566809529 74.9065952664034) bank49166 +49167 POINT(40.72407186999477 73.68506039315267) bank49167 +49168 POINT(40.97655148975129 73.26185625533745) bank49168 +49169 POINT(41.510343547543556 73.13519577269851) bank49169 +49170 POINT(40.92709145534849 73.21879480477659) bank49170 +49171 POINT(40.92700004816558 74.05117903062327) bank49171 +49172 POINT(41.696886316548905 74.68252229848062) bank49172 +49173 POINT(40.03199812508964 74.31730556372327) bank49173 +49174 POINT(41.39663177096105 73.17603568533316) bank49174 +49175 POINT(40.35271799089272 73.27349843564933) bank49175 +49176 POINT(41.495339577576054 73.92516176488087) bank49176 +49177 POINT(40.027370739986445 73.73719371986468) bank49177 +49178 POINT(40.85406001736874 74.00614325104839) bank49178 +49179 POINT(40.20022909325951 74.39153422409586) bank49179 +49180 POINT(40.82150871989428 74.90784327615499) bank49180 +49181 POINT(39.896063474603224 73.51115905242911) bank49181 +49182 POINT(41.56702333269118 75.00565147251497) bank49182 +49183 POINT(40.77207504521139 73.91921560195101) bank49183 +49184 POINT(40.296732615419714 73.59988811373496) bank49184 +49185 POINT(40.82915318269296 73.54740000810446) bank49185 +49186 POINT(40.39705276512952 74.99859730904306) bank49186 +49187 POINT(41.21932881818127 74.35538904795746) bank49187 +49188 POINT(40.684577274696416 73.59243370023344) bank49188 +49189 POINT(41.710842002491994 73.56354073214365) bank49189 +49190 POINT(40.43587723821485 73.58512372824018) bank49190 +49191 POINT(41.17298790482007 73.63653992812353) bank49191 +49192 POINT(40.44946584568797 74.45740465702893) bank49192 +49193 POINT(41.511362382687665 73.01620380317732) bank49193 +49194 POINT(40.32871290362959 74.36976238771001) bank49194 +49195 POINT(40.95961942726201 74.77767314476066) bank49195 +49196 POINT(40.40712979550784 73.82734713772277) bank49196 +49197 POINT(41.555984691551345 73.45642188714889) bank49197 +49198 POINT(41.430926045364075 73.15736810639208) bank49198 +49199 POINT(40.97666285823572 74.71613266719855) bank49199 +49200 POINT(40.38331458968399 73.98262078549944) bank49200 +49201 POINT(40.68210234021143 74.4652119500452) bank49201 +49202 POINT(39.72656434144451 73.23146465347403) bank49202 +49203 POINT(40.413605520210155 73.40560496506673) bank49203 +49204 POINT(40.65741768147369 74.29941728145502) bank49204 +49205 POINT(40.1483953668424 73.68984998029995) bank49205 +49206 POINT(40.90570805481965 73.3041311908719) bank49206 +49207 POINT(40.62169930288182 73.44640484134948) bank49207 +49208 POINT(40.065157556483214 73.62767151431018) bank49208 +49209 POINT(39.88548871786668 74.7339940082732) bank49209 +49210 POINT(40.54371587551844 74.20433103580325) bank49210 +49211 POINT(40.038764370003165 74.7539439864648) bank49211 +49212 POINT(41.30376716684358 73.92686953250119) bank49212 +49213 POINT(40.755534269435536 73.50530954336459) bank49213 +49214 POINT(40.51021909580616 73.2536571095104) bank49214 +49215 POINT(40.769436159564904 73.44043089325109) bank49215 +49216 POINT(41.216852646274134 74.23432560875251) bank49216 +49217 POINT(40.68194770213664 73.73393480530403) bank49217 +49218 POINT(40.541411480390366 73.53463979124928) bank49218 +49219 POINT(40.87372644959431 74.66528058135218) bank49219 +49220 POINT(41.35507738527436 73.69495746449797) bank49220 +49221 POINT(40.78196078830207 73.42097179767573) bank49221 +49222 POINT(40.458418124640176 73.42331982509604) bank49222 +49223 POINT(41.44958776801837 74.88372908547916) bank49223 +49224 POINT(40.56671679822106 74.33263266334177) bank49224 +49225 POINT(40.19777432431213 73.70438151226699) bank49225 +49226 POINT(41.467471837251985 73.58489182737704) bank49226 +49227 POINT(40.82641156909186 74.83348539974517) bank49227 +49228 POINT(41.54453063854409 73.41671829329337) bank49228 +49229 POINT(41.29626101393279 74.62852377051935) bank49229 +49230 POINT(40.65862323319211 74.37108039489155) bank49230 +49231 POINT(40.155762493959536 73.08404259597278) bank49231 +49232 POINT(41.226524081345524 74.55688380141112) bank49232 +49233 POINT(40.79844163170488 73.63179950671574) bank49233 +49234 POINT(41.587880344553454 74.13955255905509) bank49234 +49235 POINT(41.30574874804298 74.33705485105241) bank49235 +49236 POINT(40.7913415438161 74.60088199114328) bank49236 +49237 POINT(40.66025822630412 73.72647247441847) bank49237 +49238 POINT(41.55805879162406 74.711198762624) bank49238 +49239 POINT(41.589211122908026 74.49006168825792) bank49239 +49240 POINT(41.03758288844792 74.46266169872412) bank49240 +49241 POINT(40.66463071134351 74.47947812337296) bank49241 +49242 POINT(41.35473527560564 73.95631018677595) bank49242 +49243 POINT(40.34045107029865 74.35205377999127) bank49243 +49244 POINT(41.2374434160114 73.72192021619075) bank49244 +49245 POINT(40.71724932458861 73.8482888626662) bank49245 +49246 POINT(41.11070338896624 73.81369099883503) bank49246 +49247 POINT(40.43619367833412 74.7793793338545) bank49247 +49248 POINT(40.479904220183826 74.92337950705796) bank49248 +49249 POINT(40.398478165307935 73.58405498618971) bank49249 +49250 POINT(40.96295205392943 74.91865641735455) bank49250 +49251 POINT(41.56807736952622 73.19447968623473) bank49251 +49252 POINT(40.14627796159444 73.254484996649) bank49252 +49253 POINT(40.09932990688934 73.46932997962236) bank49253 +49254 POINT(39.953811488472454 73.9483871457869) bank49254 +49255 POINT(40.51686091735246 73.20547800573529) bank49255 +49256 POINT(41.38241765573968 74.22564811001173) bank49256 +49257 POINT(40.03437749676644 73.73873231775599) bank49257 +49258 POINT(40.65280770158688 74.76035481067836) bank49258 +49259 POINT(40.51608801193718 74.40499734753082) bank49259 +49260 POINT(40.49666506985569 74.45487344399268) bank49260 +49261 POINT(40.97930469027943 73.10905371974495) bank49261 +49262 POINT(40.155209280165195 73.14220891569424) bank49262 +49263 POINT(40.28130085867365 73.50030442191313) bank49263 +49264 POINT(40.72030380799009 73.25609628671893) bank49264 +49265 POINT(41.038682861453815 73.31448983749202) bank49265 +49266 POINT(41.283631492261556 74.86174912031174) bank49266 +49267 POINT(39.80349779496658 73.2628839609063) bank49267 +49268 POINT(40.760217708660896 73.55256072239824) bank49268 +49269 POINT(40.64953504666041 73.52129782568859) bank49269 +49270 POINT(41.41001517546009 74.62235304798794) bank49270 +49271 POINT(40.5161620469895 74.05712379233526) bank49271 +49272 POINT(41.421575956831504 73.92136442360471) bank49272 +49273 POINT(40.695384439131274 74.13457740745264) bank49273 +49274 POINT(40.69280431566505 74.78774247566211) bank49274 +49275 POINT(40.686183056980184 73.86205973087719) bank49275 +49276 POINT(40.089583881708435 74.28323664341431) bank49276 +49277 POINT(40.0633568686119 74.54642814714855) bank49277 +49278 POINT(40.58537348323215 74.10633804251292) bank49278 +49279 POINT(39.91192013215924 74.09541985795104) bank49279 +49280 POINT(39.731692528658805 73.4415423823421) bank49280 +49281 POINT(41.09838861834329 73.68248259354978) bank49281 +49282 POINT(41.490377706035545 74.27706143606581) bank49282 +49283 POINT(40.30127625470748 74.1725050008538) bank49283 +49284 POINT(39.84881663239769 73.77614380604481) bank49284 +49285 POINT(40.74688266676238 74.77998285197937) bank49285 +49286 POINT(41.59663669094391 74.89932846831954) bank49286 +49287 POINT(40.331183895972934 73.76828133876137) bank49287 +49288 POINT(40.48283160002119 74.53035403194887) bank49288 +49289 POINT(39.990667198393936 74.30323101403958) bank49289 +49290 POINT(40.76223461831257 74.42684127697626) bank49290 +49291 POINT(41.414468590175616 74.81731527686595) bank49291 +49292 POINT(40.39239210341564 74.03132545367365) bank49292 +49293 POINT(39.99026245827083 74.14349225513723) bank49293 +49294 POINT(41.21617595154506 73.22220352939772) bank49294 +49295 POINT(40.35664422217739 74.86780633613103) bank49295 +49296 POINT(41.51999486315685 74.97612236286423) bank49296 +49297 POINT(40.97502477148809 74.4086913421168) bank49297 +49298 POINT(40.9182016225858 73.82856999316945) bank49298 +49299 POINT(40.00686641923286 74.14145829088191) bank49299 +49300 POINT(40.82981611427762 73.88692779710195) bank49300 +49301 POINT(39.86763746545874 74.34422984904099) bank49301 +49302 POINT(40.30724860623991 74.66992030809152) bank49302 +49303 POINT(40.28684282763304 73.12680596239234) bank49303 +49304 POINT(41.019145043482894 74.59035750215659) bank49304 +49305 POINT(41.12014002133436 73.16402830778765) bank49305 +49306 POINT(41.3098764862193 73.28845474006135) bank49306 +49307 POINT(39.87327345367211 73.46083884856695) bank49307 +49308 POINT(40.37317122313435 74.3767677655165) bank49308 +49309 POINT(39.918383808022355 74.77314049066422) bank49309 +49310 POINT(40.10935743699282 74.03168271557895) bank49310 +49311 POINT(40.92163123088576 74.42636328126274) bank49311 +49312 POINT(40.75907986762785 74.54619166973755) bank49312 +49313 POINT(39.83295895115054 74.40635745744055) bank49313 +49314 POINT(40.67085622104604 74.81958179881352) bank49314 +49315 POINT(40.73376571933047 73.33602339816112) bank49315 +49316 POINT(40.15562223359244 73.06384304027453) bank49316 +49317 POINT(39.872830969093854 73.56648098011875) bank49317 +49318 POINT(40.45898036013559 73.51879488656742) bank49318 +49319 POINT(41.50532671100828 73.4075964436954) bank49319 +49320 POINT(40.085545869109716 74.22849289577668) bank49320 +49321 POINT(40.074934178285694 74.94505953329961) bank49321 +49322 POINT(40.0527679997951 74.90776215226253) bank49322 +49323 POINT(39.81468401016658 74.68390218490066) bank49323 +49324 POINT(40.38940750950382 73.40448925416338) bank49324 +49325 POINT(41.00012518860926 73.79085692317742) bank49325 +49326 POINT(40.73018839259635 74.58992676782775) bank49326 +49327 POINT(40.18495391323689 73.35359101045086) bank49327 +49328 POINT(40.146413269964455 73.01616965102235) bank49328 +49329 POINT(40.72818206053636 74.10908077785385) bank49329 +49330 POINT(39.769361496391696 73.82015701616263) bank49330 +49331 POINT(41.486119292043696 74.87798197557358) bank49331 +49332 POINT(40.559594340263175 73.48366196029657) bank49332 +49333 POINT(40.89271673755922 74.87129075426952) bank49333 +49334 POINT(39.955588876605724 74.04788973658921) bank49334 +49335 POINT(40.597696399594746 74.05892895776313) bank49335 +49336 POINT(39.951038427771515 74.92220545473202) bank49336 +49337 POINT(41.070445026930244 73.93472159433843) bank49337 +49338 POINT(40.73195727969933 73.27147087181748) bank49338 +49339 POINT(40.4987895764241 74.49180559823546) bank49339 +49340 POINT(40.79102662963483 73.23323938803102) bank49340 +49341 POINT(40.72441438529744 74.29779704429492) bank49341 +49342 POINT(41.644528995990335 73.2207901035257) bank49342 +49343 POINT(41.5305562743952 73.18331582201353) bank49343 +49344 POINT(40.42639385203014 73.65544919748918) bank49344 +49345 POINT(41.3770454114872 73.68044364635483) bank49345 +49346 POINT(41.25593542476469 73.26053618837518) bank49346 +49347 POINT(40.500387153619826 73.98617979462884) bank49347 +49348 POINT(40.316191631737624 73.22331879451222) bank49348 +49349 POINT(40.14069838107602 74.14794198218503) bank49349 +49350 POINT(39.99977729735472 74.17556875848369) bank49350 +49351 POINT(40.016383146777535 73.69085228765061) bank49351 +49352 POINT(41.44363770386981 74.64784968107335) bank49352 +49353 POINT(41.567867247775496 74.40203783615456) bank49353 +49354 POINT(40.298738550350386 74.3439344089916) bank49354 +49355 POINT(40.1443970176854 73.07365802832183) bank49355 +49356 POINT(41.190224744851804 73.64174594254884) bank49356 +49357 POINT(41.35138008603376 74.00847922941495) bank49357 +49358 POINT(41.33343168320279 73.50039959491431) bank49358 +49359 POINT(40.971742845147915 74.18130804126662) bank49359 +49360 POINT(40.50073146272807 74.79604285919332) bank49360 +49361 POINT(41.01471519994486 74.92700896358647) bank49361 +49362 POINT(39.96831112675048 74.92504405036222) bank49362 +49363 POINT(40.16159210092529 74.10427734173777) bank49363 +49364 POINT(41.61198803915996 73.14416470494012) bank49364 +49365 POINT(40.94443743011871 74.20476320671378) bank49365 +49366 POINT(39.998865890553006 74.89042126212121) bank49366 +49367 POINT(40.90357649278857 74.27471882025783) bank49367 +49368 POINT(40.90737937767359 73.96820393118355) bank49368 +49369 POINT(40.42939000714959 73.26059849292385) bank49369 +49370 POINT(41.5073511753842 74.02247634724846) bank49370 +49371 POINT(41.20213952877098 74.53383995842361) bank49371 +49372 POINT(39.80280280248431 74.77974845534597) bank49372 +49373 POINT(39.73400448777896 73.19516206504873) bank49373 +49374 POINT(40.02190780076775 74.24324241052447) bank49374 +49375 POINT(41.136589259803145 74.4424389553812) bank49375 +49376 POINT(39.95104901941598 73.02517125352993) bank49376 +49377 POINT(41.071374081963626 73.2548100798418) bank49377 +49378 POINT(40.606727736540236 74.96672382213004) bank49378 +49379 POINT(40.364359417224215 74.00182645145215) bank49379 +49380 POINT(41.298140723146034 74.59162336635188) bank49380 +49381 POINT(40.752886499379905 73.6719275276657) bank49381 +49382 POINT(40.80450501738631 73.22121240897872) bank49382 +49383 POINT(40.02821881318653 73.63706798017988) bank49383 +49384 POINT(40.58621357497224 73.67335801966534) bank49384 +49385 POINT(41.40399461605089 74.94165714278253) bank49385 +49386 POINT(39.80523352800695 73.02538080892366) bank49386 +49387 POINT(41.377055693029845 74.28068227416298) bank49387 +49388 POINT(41.70997268084286 74.95992729277384) bank49388 +49389 POINT(41.59622008440145 74.79523243411408) bank49389 +49390 POINT(39.78801069473822 74.1678959995123) bank49390 +49391 POINT(40.042393583070165 74.75880790937843) bank49391 +49392 POINT(40.69949188764914 74.33397761104007) bank49392 +49393 POINT(41.66572187463493 74.29861507859235) bank49393 +49394 POINT(41.37293080186549 74.73263967917791) bank49394 +49395 POINT(41.69171999796318 74.41235548366465) bank49395 +49396 POINT(41.30521997898158 74.80364822137993) bank49396 +49397 POINT(40.73857303742349 74.87238335843905) bank49397 +49398 POINT(40.410698768807364 74.29460284738524) bank49398 +49399 POINT(40.98415103003812 73.97096489801584) bank49399 +49400 POINT(40.26518227226533 74.79559943670577) bank49400 +49401 POINT(41.43123450816017 73.75710411377507) bank49401 +49402 POINT(40.94710416415974 74.02210387905058) bank49402 +49403 POINT(41.33740573914694 74.22130816766692) bank49403 +49404 POINT(40.48729107374679 74.867744421805) bank49404 +49405 POINT(40.94947686014936 73.74512427595369) bank49405 +49406 POINT(39.89536369505261 75.0047988206756) bank49406 +49407 POINT(40.94081646768913 73.7198398879709) bank49407 +49408 POINT(40.10167887104398 73.74533141003074) bank49408 +49409 POINT(40.25417989235997 73.14798004051943) bank49409 +49410 POINT(40.90000159386936 73.33966312445622) bank49410 +49411 POINT(40.34017833082353 74.69082733895968) bank49411 +49412 POINT(41.683139720053184 73.59247171701443) bank49412 +49413 POINT(39.73475692375261 74.57746286907681) bank49413 +49414 POINT(41.471594387002696 73.68711098454241) bank49414 +49415 POINT(40.00963366918168 73.6904555185831) bank49415 +49416 POINT(40.664315641133655 74.21742928198154) bank49416 +49417 POINT(41.65746029942072 74.2094952206483) bank49417 +49418 POINT(39.742877216495195 74.79346491318721) bank49418 +49419 POINT(39.83426277460236 73.1269933408695) bank49419 +49420 POINT(40.88396939089746 74.86216260883774) bank49420 +49421 POINT(40.09016396306842 73.71288053645485) bank49421 +49422 POINT(39.8562114496573 74.11185139874411) bank49422 +49423 POINT(40.520669126133875 74.42574377415367) bank49423 +49424 POINT(41.07169315982211 74.40245418074103) bank49424 +49425 POINT(41.56847433784704 73.63806470749176) bank49425 +49426 POINT(40.691107278370566 74.40526383416507) bank49426 +49427 POINT(41.29178082807757 73.64291725895484) bank49427 +49428 POINT(41.54782255296778 74.15348274667984) bank49428 +49429 POINT(41.05760540331813 74.14210607875577) bank49429 +49430 POINT(39.75336937121739 74.1574425434206) bank49430 +49431 POINT(41.20832637550401 74.27676882499011) bank49431 +49432 POINT(40.9290536091151 73.46708694766279) bank49432 +49433 POINT(41.129492071432985 74.93642429370064) bank49433 +49434 POINT(41.712363763470776 73.05819048413541) bank49434 +49435 POINT(41.642003163421094 73.72615410789389) bank49435 +49436 POINT(40.234438158781025 73.26529250759287) bank49436 +49437 POINT(40.92025280559509 74.25986932496839) bank49437 +49438 POINT(40.56789577638409 74.7795677768663) bank49438 +49439 POINT(40.690502054944965 73.56069699767805) bank49439 +49440 POINT(40.35062306009371 74.4943382764678) bank49440 +49441 POINT(40.58547655436041 74.04244371673778) bank49441 +49442 POINT(41.1855101954293 73.19818414835159) bank49442 +49443 POINT(41.39314821801994 73.23880965590132) bank49443 +49444 POINT(41.4600811251551 73.16474409906029) bank49444 +49445 POINT(40.46715314194238 74.72447124750006) bank49445 +49446 POINT(41.052545851802726 73.54257609166878) bank49446 +49447 POINT(40.34056984732827 74.5774066631766) bank49447 +49448 POINT(41.43764598020803 73.81015518480905) bank49448 +49449 POINT(40.10259133061857 74.10555769828491) bank49449 +49450 POINT(41.16387014707725 73.33839387021497) bank49450 +49451 POINT(41.41421622037623 74.02523718375211) bank49451 +49452 POINT(39.77517798912486 74.53584476082517) bank49452 +49453 POINT(41.63077633041949 73.14087946673114) bank49453 +49454 POINT(40.10650151953363 74.85414148010936) bank49454 +49455 POINT(41.512682648353874 74.2548584239575) bank49455 +49456 POINT(40.64730620932377 74.72969883328263) bank49456 +49457 POINT(39.90980426589012 74.1125305539399) bank49457 +49458 POINT(40.02272449442226 73.12561035606724) bank49458 +49459 POINT(41.447635894138536 75.00447991319402) bank49459 +49460 POINT(40.07957321014827 73.33770430926214) bank49460 +49461 POINT(41.5431035701634 73.44272974278513) bank49461 +49462 POINT(40.321693842120666 74.42173838517925) bank49462 +49463 POINT(40.87967204492598 74.0551191409785) bank49463 +49464 POINT(41.292379132148014 73.35532596241556) bank49464 +49465 POINT(40.11500816615814 74.53686908746495) bank49465 +49466 POINT(41.20301913960147 73.28282372617319) bank49466 +49467 POINT(39.84346155087561 73.04423189156581) bank49467 +49468 POINT(41.711692017886364 74.53293814114895) bank49468 +49469 POINT(41.406078973994816 73.63596675569194) bank49469 +49470 POINT(40.50340392005231 74.65857685825443) bank49470 +49471 POINT(39.84503317605581 73.10148553620095) bank49471 +49472 POINT(41.544372077623365 73.45250740149417) bank49472 +49473 POINT(39.95650942949166 73.58127824397192) bank49473 +49474 POINT(39.787143734262756 74.46722163574312) bank49474 +49475 POINT(39.88239740336325 73.438713891657) bank49475 +49476 POINT(41.659768134265946 74.85197850674209) bank49476 +49477 POINT(40.0438706677032 73.88316195685431) bank49477 +49478 POINT(41.09325988820179 74.98581550396976) bank49478 +49479 POINT(41.49036048899004 73.70715362624546) bank49479 +49480 POINT(41.65706263852276 73.92230919337827) bank49480 +49481 POINT(39.923403928588364 74.50790068131533) bank49481 +49482 POINT(41.4090953740523 74.11299458525018) bank49482 +49483 POINT(40.48421513178851 74.99672986732894) bank49483 +49484 POINT(40.74168382083149 74.40073557499053) bank49484 +49485 POINT(40.41292218348283 74.87639797683624) bank49485 +49486 POINT(39.7797605575915 74.42751258096519) bank49486 +49487 POINT(41.18166552182915 73.10251490055028) bank49487 +49488 POINT(40.17864398995265 74.14982282985312) bank49488 +49489 POINT(39.93778897738303 73.3642415861277) bank49489 +49490 POINT(40.05466090307301 73.83605422622043) bank49490 +49491 POINT(40.28152047144377 74.639829571505) bank49491 +49492 POINT(41.04691719183539 73.87590723312384) bank49492 +49493 POINT(41.080736949939514 74.19701535570167) bank49493 +49494 POINT(40.2710043479067 73.75648035207945) bank49494 +49495 POINT(39.73625557207907 74.49416370416736) bank49495 +49496 POINT(40.173820397740016 74.17020430566672) bank49496 +49497 POINT(40.53857442045181 74.55152736142377) bank49497 +49498 POINT(39.87098650192941 73.80038097298957) bank49498 +49499 POINT(41.31484790782954 74.50336660246607) bank49499 +49500 POINT(40.96429486654004 74.25860769539584) bank49500 +49501 POINT(40.821152940929416 73.51830928714351) bank49501 +49502 POINT(39.86531411441052 74.07818025811626) bank49502 +49503 POINT(40.135883509983024 74.49537102823557) bank49503 +49504 POINT(40.671589120612815 74.99633274787752) bank49504 +49505 POINT(39.80482203615571 74.60782966953103) bank49505 +49506 POINT(41.24470222713184 74.39219044257347) bank49506 +49507 POINT(39.829913167328506 73.65712141061472) bank49507 +49508 POINT(41.411696950782414 73.63797111437891) bank49508 +49509 POINT(40.65264354787797 74.71765628081305) bank49509 +49510 POINT(41.66794791542079 73.15399428240305) bank49510 +49511 POINT(40.042104516750584 73.52652767336987) bank49511 +49512 POINT(40.8082914722613 73.54431663330004) bank49512 +49513 POINT(40.83273004594406 73.95620960720807) bank49513 +49514 POINT(41.70019088193693 73.05777761522559) bank49514 +49515 POINT(39.881642553978075 74.34335488022494) bank49515 +49516 POINT(40.317754912018415 73.5475657234433) bank49516 +49517 POINT(39.7852946149989 74.10621108266893) bank49517 +49518 POINT(40.51404924609827 73.8463761376847) bank49518 +49519 POINT(39.91239451560622 74.95967497940171) bank49519 +49520 POINT(40.97364953656703 74.3734804719839) bank49520 +49521 POINT(41.28972187501793 74.6236750477828) bank49521 +49522 POINT(41.62835530009341 74.10007906086553) bank49522 +49523 POINT(39.82106222265599 73.08906785154824) bank49523 +49524 POINT(40.95336620831445 73.47160396532614) bank49524 +49525 POINT(41.41345100073387 73.42287324527246) bank49525 +49526 POINT(39.77434239712227 73.03826764329223) bank49526 +49527 POINT(39.84703367228764 73.12594127524765) bank49527 +49528 POINT(39.89364898864709 74.87177743771824) bank49528 +49529 POINT(40.525224486163744 73.58096566515461) bank49529 +49530 POINT(40.305274069907526 74.90499128128162) bank49530 +49531 POINT(40.61778401009723 74.32669096579696) bank49531 +49532 POINT(41.433061585310895 74.54210073227617) bank49532 +49533 POINT(40.51838114561834 73.6730937917591) bank49533 +49534 POINT(41.312668944562304 74.88198790095153) bank49534 +49535 POINT(41.229376806595326 73.50701750582424) bank49535 +49536 POINT(41.32955546521761 73.11169461658143) bank49536 +49537 POINT(40.05655389398691 73.4866313737738) bank49537 +49538 POINT(40.49076094452573 73.03885111914938) bank49538 +49539 POINT(41.224465345816895 73.26178286533667) bank49539 +49540 POINT(41.573022942123 73.60131504091595) bank49540 +49541 POINT(41.479404420676666 74.70701056117053) bank49541 +49542 POINT(41.40041780349645 73.10050547040103) bank49542 +49543 POINT(40.46042272694347 73.49275077493176) bank49543 +49544 POINT(39.87230942051867 74.1668637793242) bank49544 +49545 POINT(41.235460299669086 73.36624425460738) bank49545 +49546 POINT(41.27091478488748 74.63041420582634) bank49546 +49547 POINT(40.79061051051754 74.76122715797028) bank49547 +49548 POINT(39.7657616796513 73.7402458343614) bank49548 +49549 POINT(41.02567357523595 74.69654231974854) bank49549 +49550 POINT(40.206186369232555 73.07112357860814) bank49550 +49551 POINT(39.78976888723248 74.91679613040905) bank49551 +49552 POINT(40.90837909024937 74.99889993585755) bank49552 +49553 POINT(39.95234598309868 74.43103155469358) bank49553 +49554 POINT(40.76009218349392 74.32488640342962) bank49554 +49555 POINT(40.62559028454907 73.07957393283127) bank49555 +49556 POINT(40.536590860508745 73.9102778041797) bank49556 +49557 POINT(41.2472267173455 75.00079454011777) bank49557 +49558 POINT(40.5369006404776 73.65445026250342) bank49558 +49559 POINT(40.797099875567675 73.1657898135162) bank49559 +49560 POINT(40.245003615995266 73.638326969157) bank49560 +49561 POINT(40.02297658329597 73.31207958830764) bank49561 +49562 POINT(41.315378261957804 73.85471977578044) bank49562 +49563 POINT(41.33564415509493 73.92366647358416) bank49563 +49564 POINT(39.94277225813307 73.11513965350657) bank49564 +49565 POINT(39.872284225623986 74.63487244281465) bank49565 +49566 POINT(41.03111044892815 74.36972846779602) bank49566 +49567 POINT(41.3480187755121 74.22142813550342) bank49567 +49568 POINT(40.42483762050847 73.89347175512994) bank49568 +49569 POINT(40.9566382702078 73.19817389646947) bank49569 +49570 POINT(41.28694493009654 73.55924350732379) bank49570 +49571 POINT(40.30157544137601 74.47677363472236) bank49571 +49572 POINT(40.83813334449659 73.5218328078008) bank49572 +49573 POINT(40.130653719341176 74.03883439184156) bank49573 +49574 POINT(40.391900576428924 73.48368317059862) bank49574 +49575 POINT(41.61495130139364 74.41329168969722) bank49575 +49576 POINT(40.72466994401092 73.54943587114255) bank49576 +49577 POINT(41.358809232090906 73.0330875333843) bank49577 +49578 POINT(39.8251647852716 74.506954070535) bank49578 +49579 POINT(40.330692485472994 74.9580095974956) bank49579 +49580 POINT(41.37607309460816 74.65869824752049) bank49580 +49581 POINT(41.19896550113093 73.52389168718055) bank49581 +49582 POINT(41.256265759796136 73.0613980339953) bank49582 +49583 POINT(41.1058570180977 73.1197401491034) bank49583 +49584 POINT(40.963434169416495 73.24461874305392) bank49584 +49585 POINT(39.81313670470805 74.24270564560477) bank49585 +49586 POINT(41.433153631656864 74.04465901035489) bank49586 +49587 POINT(41.14557154040287 74.22841756307122) bank49587 +49588 POINT(41.143686823473594 73.13440348057375) bank49588 +49589 POINT(41.46710129590875 73.96425731917472) bank49589 +49590 POINT(40.93684978195378 73.04171909978194) bank49590 +49591 POINT(40.38106858482591 74.12546746867703) bank49591 +49592 POINT(40.55324770359038 73.07170686010807) bank49592 +49593 POINT(39.74352560799023 73.75784660575243) bank49593 +49594 POINT(40.90229607854311 74.80875433186323) bank49594 +49595 POINT(41.36981408294155 74.51369985249531) bank49595 +49596 POINT(41.54515643649166 74.83334641812525) bank49596 +49597 POINT(40.29217580819589 74.6548598855318) bank49597 +49598 POINT(40.269117625419646 74.83088727781573) bank49598 +49599 POINT(40.375970249558804 74.49840414688525) bank49599 +49600 POINT(41.580418388816625 73.29898914920913) bank49600 +49601 POINT(41.149822457209645 73.80160631666321) bank49601 +49602 POINT(39.935943038873035 74.27920381329386) bank49602 +49603 POINT(40.583224558436264 73.61648587167693) bank49603 +49604 POINT(41.67933685879835 73.67433334197258) bank49604 +49605 POINT(40.55621851571266 74.25441778967662) bank49605 +49606 POINT(39.89743021914156 74.35842757484915) bank49606 +49607 POINT(39.744634186002806 74.52462494172208) bank49607 +49608 POINT(40.41909885736558 73.74867222757824) bank49608 +49609 POINT(41.561919082265874 73.33437134008643) bank49609 +49610 POINT(41.69150865461639 73.35405474983368) bank49610 +49611 POINT(40.38843780025012 74.42455874823914) bank49611 +49612 POINT(40.27189870928779 74.44969511329595) bank49612 +49613 POINT(41.61134886901115 73.11669895249675) bank49613 +49614 POINT(39.87473509066063 73.34764974921784) bank49614 +49615 POINT(41.0201905198346 74.26924903295266) bank49615 +49616 POINT(41.37731879630857 73.27257009491468) bank49616 +49617 POINT(40.836324239632425 73.5650233996713) bank49617 +49618 POINT(41.19314590919511 74.23367110494057) bank49618 +49619 POINT(40.204956489082875 74.54114137835818) bank49619 +49620 POINT(40.4346948881905 74.47947406451613) bank49620 +49621 POINT(40.98238914466211 73.40624858797091) bank49621 +49622 POINT(39.92048816253687 73.75142942571469) bank49622 +49623 POINT(41.56369423496115 74.89030123862119) bank49623 +49624 POINT(40.81545408488845 74.53111200491261) bank49624 +49625 POINT(40.860023308099926 74.75872050191875) bank49625 +49626 POINT(41.31654166466402 74.69833846833825) bank49626 +49627 POINT(40.841861790637864 73.97261076442715) bank49627 +49628 POINT(40.497260187957735 73.84720229727422) bank49628 +49629 POINT(40.63465854302783 73.84728790656568) bank49629 +49630 POINT(40.84333562386704 74.70210857778598) bank49630 +49631 POINT(40.203367856986624 73.12547700876898) bank49631 +49632 POINT(39.997042414378015 73.55895682448002) bank49632 +49633 POINT(39.764295958305965 73.43450939550954) bank49633 +49634 POINT(41.35349169448329 74.49598901350127) bank49634 +49635 POINT(41.0152798499942 74.50333682674068) bank49635 +49636 POINT(40.9393591333071 73.82759777583956) bank49636 +49637 POINT(41.684299771975745 73.96342227181098) bank49637 +49638 POINT(40.714768857633246 74.49391779147295) bank49638 +49639 POINT(39.93895020827177 74.8355161742369) bank49639 +49640 POINT(40.65188032781369 74.13596552744968) bank49640 +49641 POINT(39.87894011559355 74.15663413950986) bank49641 +49642 POINT(41.27959579918584 74.8272093322824) bank49642 +49643 POINT(39.8349379187542 74.00347817915781) bank49643 +49644 POINT(41.172630509735896 73.90104772656161) bank49644 +49645 POINT(41.30030221006167 73.6983635286556) bank49645 +49646 POINT(41.48007594639687 74.48243240344338) bank49646 +49647 POINT(40.05122720828884 74.2911179294964) bank49647 +49648 POINT(40.874561947848925 74.0540489607856) bank49648 +49649 POINT(41.07162245085137 73.13690596449811) bank49649 +49650 POINT(41.462318052330566 74.42892236827177) bank49650 +49651 POINT(40.31624477969777 73.29462644793253) bank49651 +49652 POINT(41.54308609950199 73.10216302780353) bank49652 +49653 POINT(41.03107534683664 73.69549196735304) bank49653 +49654 POINT(40.101574302392564 73.26759636886736) bank49654 +49655 POINT(39.713807983322084 73.20469064427233) bank49655 +49656 POINT(40.414903064094545 73.91629122708184) bank49656 +49657 POINT(40.780217206527894 75.00565401732699) bank49657 +49658 POINT(41.680314521089095 73.79778652149744) bank49658 +49659 POINT(40.453403547709115 73.16407441291481) bank49659 +49660 POINT(41.208979324123455 74.45174576935379) bank49660 +49661 POINT(40.079870978580075 73.9437707220241) bank49661 +49662 POINT(40.90798023952666 73.1799729419509) bank49662 +49663 POINT(40.54992810120515 73.02999749386186) bank49663 +49664 POINT(40.7128085397867 74.24943274876762) bank49664 +49665 POINT(39.9341369952802 74.83437357973851) bank49665 +49666 POINT(40.21453141389829 74.28279303497531) bank49666 +49667 POINT(40.52789408399718 73.09313659533686) bank49667 +49668 POINT(40.50023518522482 73.02442405956337) bank49668 +49669 POINT(40.11119413859285 74.75818129764335) bank49669 +49670 POINT(39.887114719332125 73.85988099806646) bank49670 +49671 POINT(41.06787728250705 74.09115578898394) bank49671 +49672 POINT(41.2379042012611 74.8931541744002) bank49672 +49673 POINT(39.95261515664813 74.53660087705693) bank49673 +49674 POINT(41.14460035871068 74.35072402811862) bank49674 +49675 POINT(40.18447307604061 74.1539013013904) bank49675 +49676 POINT(40.16910954675185 73.14507501430317) bank49676 +49677 POINT(40.08829665148117 73.42208698613727) bank49677 +49678 POINT(41.28940083816693 74.84476805665605) bank49678 +49679 POINT(40.29849685557535 74.81269733784121) bank49679 +49680 POINT(39.75487025560887 73.62275197357188) bank49680 +49681 POINT(41.43367150976732 73.86882211390075) bank49681 +49682 POINT(41.06570996729898 73.72884028657971) bank49682 +49683 POINT(39.81292655989449 74.58304405938627) bank49683 +49684 POINT(40.067227385796684 74.53762222119576) bank49684 +49685 POINT(39.77114032697115 74.55930844635327) bank49685 +49686 POINT(40.71047934738271 73.8348213065898) bank49686 +49687 POINT(40.507915773507456 74.34473361746004) bank49687 +49688 POINT(41.21932373603136 73.38089836343389) bank49688 +49689 POINT(41.578542184745324 74.82086470575469) bank49689 +49690 POINT(40.92210081664572 73.84601311193461) bank49690 +49691 POINT(41.360520293809884 74.628087356105) bank49691 +49692 POINT(40.6839166491826 73.32779226643635) bank49692 +49693 POINT(41.108780141134766 74.93292707171948) bank49693 +49694 POINT(40.70369013925099 74.28037651368373) bank49694 +49695 POINT(41.658938222335415 73.51486157383128) bank49695 +49696 POINT(41.38936032498727 74.55519389141689) bank49696 +49697 POINT(40.57683312411762 74.08219881417055) bank49697 +49698 POINT(39.91133308886984 73.75589481127862) bank49698 +49699 POINT(40.53583116957821 73.28679072548456) bank49699 +49700 POINT(39.83667730090924 74.17771235962479) bank49700 +49701 POINT(40.15455501988776 73.78871039764236) bank49701 +49702 POINT(40.893928457900614 74.5894097192184) bank49702 +49703 POINT(40.07098262039335 74.07129325671416) bank49703 +49704 POINT(41.55836708606506 74.42055693885449) bank49704 +49705 POINT(41.537957089188815 74.11123020832123) bank49705 +49706 POINT(40.32627036951709 73.04772774814322) bank49706 +49707 POINT(41.08745508607551 74.87031023110592) bank49707 +49708 POINT(40.673190687573126 73.02706493934917) bank49708 +49709 POINT(40.87115322461966 73.8578642049919) bank49709 +49710 POINT(40.419599445540456 73.55594409263904) bank49710 +49711 POINT(40.0004454898429 73.02508012637206) bank49711 +49712 POINT(40.5597292689893 74.87983617067441) bank49712 +49713 POINT(40.22354979256735 73.12932239056227) bank49713 +49714 POINT(40.13610982111794 74.09716437206073) bank49714 +49715 POINT(40.74357904835485 74.80321047151345) bank49715 +49716 POINT(40.18993641012281 74.11443318732303) bank49716 +49717 POINT(40.423740493545644 73.56465738354689) bank49717 +49718 POINT(40.46660993547202 74.24685085031678) bank49718 +49719 POINT(39.90043852289337 74.61349500302904) bank49719 +49720 POINT(40.26268821727269 74.21625609175076) bank49720 +49721 POINT(40.90079779974154 73.66683133853171) bank49721 +49722 POINT(40.258909680463724 74.19209156802592) bank49722 +49723 POINT(39.78803362641252 74.4076958596742) bank49723 +49724 POINT(41.551172914465766 73.81616895268823) bank49724 +49725 POINT(41.59453515104191 73.182219070611) bank49725 +49726 POINT(40.086010546301274 74.8753625526788) bank49726 +49727 POINT(41.50377869319417 73.92639689408954) bank49727 +49728 POINT(40.7778822589918 73.2787130920752) bank49728 +49729 POINT(40.897077729615575 73.92303540573629) bank49729 +49730 POINT(41.43237966880214 74.02942885307) bank49730 +49731 POINT(40.45195848652943 74.85052884898418) bank49731 +49732 POINT(41.0247711316059 73.96119893356489) bank49732 +49733 POINT(40.78419077330504 74.20923743583619) bank49733 +49734 POINT(41.066307025492385 74.50670157890754) bank49734 +49735 POINT(40.252099727063936 73.75129366707289) bank49735 +49736 POINT(41.198060756159215 74.36814275417022) bank49736 +49737 POINT(40.80656716305052 73.76680145569236) bank49737 +49738 POINT(41.43720585915256 73.54168291393748) bank49738 +49739 POINT(41.50357469752983 73.11895562218992) bank49739 +49740 POINT(41.305135102404655 73.17060458776123) bank49740 +49741 POINT(40.55066871778823 73.26223213129428) bank49741 +49742 POINT(41.34221587306257 73.9561316043382) bank49742 +49743 POINT(40.06708321576929 74.55381967451292) bank49743 +49744 POINT(40.6062057991916 74.35728904705088) bank49744 +49745 POINT(40.45287458120785 74.96206492491028) bank49745 +49746 POINT(40.60675309435744 74.43124218810979) bank49746 +49747 POINT(40.022077757875124 73.355139665907) bank49747 +49748 POINT(40.929702621020446 74.34977457166313) bank49748 +49749 POINT(41.28180085403045 74.48661712037381) bank49749 +49750 POINT(39.874264680615376 74.2335549055876) bank49750 +49751 POINT(40.37370719144207 74.70812083101316) bank49751 +49752 POINT(40.610774958670014 74.74724660975573) bank49752 +49753 POINT(40.830462758133834 74.11964133122703) bank49753 +49754 POINT(41.09292634808255 73.0226756547389) bank49754 +49755 POINT(40.45270365310462 74.69849709332273) bank49755 +49756 POINT(40.27438418489704 73.79904778004203) bank49756 +49757 POINT(41.53320590881556 73.42474567563067) bank49757 +49758 POINT(40.069712983115885 73.37597569378411) bank49758 +49759 POINT(40.069068747543625 74.04590137916496) bank49759 +49760 POINT(39.93371420564857 73.28511361383319) bank49760 +49761 POINT(41.53415661275383 74.89312784748216) bank49761 +49762 POINT(41.440774064037754 73.9936211835178) bank49762 +49763 POINT(41.41096260149429 73.28089807462385) bank49763 +49764 POINT(40.690020626588435 74.65395759532788) bank49764 +49765 POINT(40.87068808539525 73.09458525496797) bank49765 +49766 POINT(40.433232855211365 73.178666798133) bank49766 +49767 POINT(40.35011221513789 73.82467206873262) bank49767 +49768 POINT(40.62387117476977 73.73207563352723) bank49768 +49769 POINT(40.45801655948475 73.98593339240794) bank49769 +49770 POINT(40.41188753871779 74.30948905687724) bank49770 +49771 POINT(40.04017314379585 73.4621753630348) bank49771 +49772 POINT(40.52459421036322 73.42542006525105) bank49772 +49773 POINT(40.72762759541394 74.80266222068612) bank49773 +49774 POINT(40.784671888614646 73.16262173156639) bank49774 +49775 POINT(40.386152258023685 74.93796372828857) bank49775 +49776 POINT(40.519953449899845 74.55490820528246) bank49776 +49777 POINT(41.663980709781505 73.66665248219627) bank49777 +49778 POINT(40.5039788744241 73.61038243227063) bank49778 +49779 POINT(39.78629802978556 74.39472457453708) bank49779 +49780 POINT(40.7655813479038 73.1230300027552) bank49780 +49781 POINT(40.74556299508351 74.2181797908131) bank49781 +49782 POINT(39.874605640176455 73.3354090450061) bank49782 +49783 POINT(40.39772005680728 73.50951277414516) bank49783 +49784 POINT(40.21152293451581 73.57136815938766) bank49784 +49785 POINT(39.745855477824044 74.86552078487458) bank49785 +49786 POINT(40.293311109656955 73.77178876485004) bank49786 +49787 POINT(41.372551304748875 74.63566667473343) bank49787 +49788 POINT(40.59917914489783 73.19743568868772) bank49788 +49789 POINT(41.59852924877222 74.2804828630141) bank49789 +49790 POINT(39.98788778319692 74.67517945335689) bank49790 +49791 POINT(40.447137859956825 73.08118175354564) bank49791 +49792 POINT(39.89621878914217 73.48732540426917) bank49792 +49793 POINT(41.054195634159456 74.29045248816465) bank49793 +49794 POINT(40.729584372810656 74.46382996961968) bank49794 +49795 POINT(40.05952394511912 73.98387712410788) bank49795 +49796 POINT(40.9410736301667 73.16557829300436) bank49796 +49797 POINT(41.563593180305126 74.29524662505811) bank49797 +49798 POINT(40.95504683773403 74.70502513965678) bank49798 +49799 POINT(41.64211354022531 73.3715097290344) bank49799 +49800 POINT(40.80639934366816 73.38413169348192) bank49800 +49801 POINT(40.96733160448333 74.34557681698925) bank49801 +49802 POINT(41.43860856819767 74.55210764177173) bank49802 +49803 POINT(40.25963935295938 73.79252145635398) bank49803 +49804 POINT(41.34390999477462 73.99565921952431) bank49804 +49805 POINT(40.051093491408906 74.86760965527803) bank49805 +49806 POINT(40.57840190009823 73.15685477958974) bank49806 +49807 POINT(41.02828413198904 74.98669484291915) bank49807 +49808 POINT(41.66908864818848 73.20858600280299) bank49808 +49809 POINT(40.20168001911999 73.82080309004398) bank49809 +49810 POINT(40.8128039942363 74.04254272888504) bank49810 +49811 POINT(41.58774707317213 73.01216999346994) bank49811 +49812 POINT(41.350987558839705 73.83785397204156) bank49812 +49813 POINT(39.97922490153748 74.93587699912372) bank49813 +49814 POINT(41.279561426471105 74.01228619648633) bank49814 +49815 POINT(41.58233808138286 73.56259146884302) bank49815 +49816 POINT(40.96152434147605 74.15091580874493) bank49816 +49817 POINT(41.080901453955065 73.72165980677177) bank49817 +49818 POINT(40.173370552315504 74.69422646731962) bank49818 +49819 POINT(40.085759874942404 73.74045870073553) bank49819 +49820 POINT(39.84614834491655 74.53123236712108) bank49820 +49821 POINT(39.99981765211331 73.42214067937759) bank49821 +49822 POINT(40.80382380946405 74.84228425208727) bank49822 +49823 POINT(40.405411701235025 74.48434089375446) bank49823 +49824 POINT(40.00692785718957 74.71534377015132) bank49824 +49825 POINT(40.82886627221493 73.70186460402243) bank49825 +49826 POINT(40.85462119358497 73.63529629957071) bank49826 +49827 POINT(40.10271920482105 73.28754137654255) bank49827 +49828 POINT(41.031442985326464 73.87319126481371) bank49828 +49829 POINT(41.09604290921102 74.38476592665779) bank49829 +49830 POINT(40.373415472874576 73.8237036512105) bank49830 +49831 POINT(40.38256296225315 73.22821252531838) bank49831 +49832 POINT(40.92510446037217 73.4274463072785) bank49832 +49833 POINT(41.08595625510273 74.92644503964453) bank49833 +49834 POINT(41.4151572994006 74.92727787792398) bank49834 +49835 POINT(41.26310905419463 73.88885939646252) bank49835 +49836 POINT(40.992415039753965 73.93366420216938) bank49836 +49837 POINT(40.42499508814668 74.63176775516014) bank49837 +49838 POINT(39.72538440804882 73.2809696331437) bank49838 +49839 POINT(39.90782302535391 73.5612757296536) bank49839 +49840 POINT(40.721907186748815 74.03284578127679) bank49840 +49841 POINT(41.354506125520025 74.26705838664934) bank49841 +49842 POINT(41.42016480495655 74.94159589267747) bank49842 +49843 POINT(40.31575532661847 74.35324520709891) bank49843 +49844 POINT(41.53449679613151 73.31620263741054) bank49844 +49845 POINT(41.709458883976 73.06492866056308) bank49845 +49846 POINT(40.52777629208373 74.57473549020845) bank49846 +49847 POINT(40.17507424788401 73.1320033538737) bank49847 +49848 POINT(40.37571232505048 73.400052646418) bank49848 +49849 POINT(40.29655065800919 74.9059428493159) bank49849 +49850 POINT(40.399334753632075 74.65138639171396) bank49850 +49851 POINT(41.24662234676136 73.4536872894971) bank49851 +49852 POINT(40.31955962044445 74.82699378537548) bank49852 +49853 POINT(40.61367619192646 73.61227366137967) bank49853 +49854 POINT(40.24671737990643 73.33691292454674) bank49854 +49855 POINT(41.465599618223365 73.06328061761525) bank49855 +49856 POINT(40.594017626052064 73.45375639470117) bank49856 +49857 POINT(41.554466476898405 73.62348540609888) bank49857 +49858 POINT(40.62832251152887 73.62565562244882) bank49858 +49859 POINT(40.70091066024504 73.04916273350067) bank49859 +49860 POINT(39.71421917331289 74.83961162205568) bank49860 +49861 POINT(40.555395299429016 74.4998545784624) bank49861 +49862 POINT(40.82913079950123 74.86127246066157) bank49862 +49863 POINT(41.55527213069034 73.05786437017814) bank49863 +49864 POINT(41.01588130673115 74.3792749328744) bank49864 +49865 POINT(40.124533936395444 74.53293188026295) bank49865 +49866 POINT(41.208815820283654 73.79612285200841) bank49866 +49867 POINT(39.983883477900534 73.81224314496514) bank49867 +49868 POINT(40.9743768219057 74.3259560551822) bank49868 +49869 POINT(40.75193848254443 74.30350159315611) bank49869 +49870 POINT(40.139826103549574 74.10305694192265) bank49870 +49871 POINT(40.50175024861123 73.90817850755953) bank49871 +49872 POINT(40.62620086536278 74.80442102096217) bank49872 +49873 POINT(41.65411544693831 73.31865820343462) bank49873 +49874 POINT(40.092791909404575 73.75748198704177) bank49874 +49875 POINT(41.14007558320371 74.44658166145281) bank49875 +49876 POINT(39.875703019278 74.70388667533406) bank49876 +49877 POINT(41.53017519918107 73.06711250231288) bank49877 +49878 POINT(39.98359294883673 74.43157752898682) bank49878 +49879 POINT(40.80750229998543 73.30588074252717) bank49879 +49880 POINT(39.721805758432374 74.39459898825817) bank49880 +49881 POINT(40.812534248465276 74.3674932817565) bank49881 +49882 POINT(40.60553696284516 74.96318481492374) bank49882 +49883 POINT(40.41643584132578 73.07594128455771) bank49883 +49884 POINT(40.244751006298465 74.64196250898345) bank49884 +49885 POINT(41.58964514208055 74.13227442911602) bank49885 +49886 POINT(40.61416551658311 73.81991353009914) bank49886 +49887 POINT(41.41076857795551 74.93385319090592) bank49887 +49888 POINT(41.2188756590401 74.035880250346) bank49888 +49889 POINT(39.85752271424202 74.47186747230516) bank49889 +49890 POINT(40.6978455832202 73.11278753229777) bank49890 +49891 POINT(41.676452199775866 73.79138572542392) bank49891 +49892 POINT(40.779851828492404 74.09362227667107) bank49892 +49893 POINT(41.22162462977966 73.71290016211324) bank49893 +49894 POINT(39.84472751976877 74.94067689601945) bank49894 +49895 POINT(40.56459340674384 73.10223391681104) bank49895 +49896 POINT(41.32923196457078 73.47231162982108) bank49896 +49897 POINT(40.7024718757317 73.63809608744711) bank49897 +49898 POINT(40.90672309331779 73.37303460110098) bank49898 +49899 POINT(41.35020144470314 74.08299737227692) bank49899 +49900 POINT(40.203940674645644 74.86812283097875) bank49900 +49901 POINT(41.523756108884015 73.992714184608) bank49901 +49902 POINT(41.65531306553168 74.9904805578345) bank49902 +49903 POINT(41.40187873322837 74.68798422998236) bank49903 +49904 POINT(40.85090499718734 73.85047999734533) bank49904 +49905 POINT(41.0134088478529 73.95166492418726) bank49905 +49906 POINT(39.96633083341328 74.12669161910658) bank49906 +49907 POINT(40.42828587646629 73.58217805769348) bank49907 +49908 POINT(41.351224555355536 74.62227772620777) bank49908 +49909 POINT(40.45034747114305 73.1495907095729) bank49909 +49910 POINT(40.56368777970004 74.67064814349094) bank49910 +49911 POINT(40.44354969139234 73.09071876026974) bank49911 +49912 POINT(41.08205563196111 73.48547026855414) bank49912 +49913 POINT(40.00786729880052 74.3330414413659) bank49913 +49914 POINT(40.640383813306805 73.17149762547206) bank49914 +49915 POINT(41.53280979129732 74.88136721023282) bank49915 +49916 POINT(41.499083817397704 73.20885289614131) bank49916 +49917 POINT(40.148550224369686 73.49186743569894) bank49917 +49918 POINT(40.623335709191245 73.12144490436798) bank49918 +49919 POINT(40.82545563501989 74.69564804191172) bank49919 +49920 POINT(40.363345075461936 73.83081400448279) bank49920 +49921 POINT(41.00807067503904 73.07116538444164) bank49921 +49922 POINT(40.722654154977576 74.38636940943361) bank49922 +49923 POINT(41.64451863163909 74.04669123423929) bank49923 +49924 POINT(39.898103199682275 74.3743904785466) bank49924 +49925 POINT(40.71772423149091 73.6706275533892) bank49925 +49926 POINT(41.184876438339714 74.71692804444181) bank49926 +49927 POINT(40.67124300119156 74.22305085020307) bank49927 +49928 POINT(40.278542476764144 73.56626024474804) bank49928 +49929 POINT(41.4209839513446 74.73401223947289) bank49929 +49930 POINT(39.98851080238582 73.8943798239017) bank49930 +49931 POINT(40.12101290096608 73.7169118403454) bank49931 +49932 POINT(39.7162593372478 74.18370788927483) bank49932 +49933 POINT(40.35186706763005 74.08619631537748) bank49933 +49934 POINT(41.59141903867321 73.3232066714654) bank49934 +49935 POINT(41.46846481107114 74.50417851374735) bank49935 +49936 POINT(40.529618895777176 73.2319093685552) bank49936 +49937 POINT(40.50549828622256 74.59567842317445) bank49937 +49938 POINT(41.38266059370761 74.9073758857944) bank49938 +49939 POINT(41.46020664817821 73.20751708249415) bank49939 +49940 POINT(40.90559564748393 74.55062219008353) bank49940 +49941 POINT(41.23131351090905 74.54693348042406) bank49941 +49942 POINT(40.15226994658879 73.74925879376755) bank49942 +49943 POINT(39.71343658435465 73.49748572700803) bank49943 +49944 POINT(41.18551391707808 73.13358787717333) bank49944 +49945 POINT(40.40341012242553 74.65113735577971) bank49945 +49946 POINT(40.936073269964865 74.8738868271441) bank49946 +49947 POINT(41.42166365787463 74.47997427962514) bank49947 +49948 POINT(40.866012393495154 74.13400092038322) bank49948 +49949 POINT(41.40245507390293 74.98327574871224) bank49949 +49950 POINT(40.68138204780834 73.03299735037562) bank49950 +49951 POINT(40.315183924939696 74.2699953977378) bank49951 +49952 POINT(41.41986803370667 74.04901840263065) bank49952 +49953 POINT(39.97245084696162 74.12505900935258) bank49953 +49954 POINT(41.09620685362411 73.36244147354901) bank49954 +49955 POINT(41.237032706338 74.17222442643533) bank49955 +49956 POINT(40.06839194293099 73.90422858575708) bank49956 +49957 POINT(40.39949085284556 74.84958375010359) bank49957 +49958 POINT(41.477467829567 74.86089396271153) bank49958 +49959 POINT(40.11080328538598 74.52889237189694) bank49959 +49960 POINT(40.31951985602793 73.0494019576022) bank49960 +49961 POINT(40.554201307996784 74.16408834773696) bank49961 +49962 POINT(41.67166208453124 73.5548730584774) bank49962 +49963 POINT(40.468769276681506 74.1651204785282) bank49963 +49964 POINT(41.54198080471223 73.33376537692318) bank49964 +49965 POINT(41.20052428528153 74.9410121477028) bank49965 +49966 POINT(40.042950202490154 74.59837000974133) bank49966 +49967 POINT(40.35059815650844 73.61371984368976) bank49967 +49968 POINT(41.28718411839535 74.05154585170949) bank49968 +49969 POINT(40.78760444699754 74.10774356132747) bank49969 +49970 POINT(41.63950260944868 74.89093019889327) bank49970 +49971 POINT(40.2831415856199 73.76851180347302) bank49971 +49972 POINT(41.5331295154378 73.76283328483218) bank49972 +49973 POINT(39.905515120941345 74.74808082192352) bank49973 +49974 POINT(41.15703343161907 74.57736138049337) bank49974 +49975 POINT(40.063014111070984 74.15019308780285) bank49975 +49976 POINT(40.71399468649993 73.8524551504444) bank49976 +49977 POINT(40.20156588924928 74.46684593420062) bank49977 +49978 POINT(41.06889461186842 74.81937001641708) bank49978 +49979 POINT(40.97043153856391 73.0904177153732) bank49979 +49980 POINT(39.779107683547934 73.18667097075756) bank49980 +49981 POINT(41.61353538777578 74.56748287031552) bank49981 +49982 POINT(40.35910384506054 74.63455339434378) bank49982 +49983 POINT(39.911170547446304 73.43976814545091) bank49983 +49984 POINT(41.55944471217789 74.73695465815345) bank49984 +49985 POINT(40.05470003947791 73.3917275297769) bank49985 +49986 POINT(41.15087991920115 73.91505945501245) bank49986 +49987 POINT(41.663088286996256 74.69384053838955) bank49987 +49988 POINT(40.96836699040002 74.48623655630647) bank49988 +49989 POINT(41.70882322238187 73.75859771228647) bank49989 +49990 POINT(40.34053971677794 73.55976874132878) bank49990 +49991 POINT(40.18517143591148 73.47933979317759) bank49991 +49992 POINT(39.95534823601421 73.71755325318895) bank49992 +49993 POINT(41.70706859270601 73.91891940532861) bank49993 +49994 POINT(41.501572612887806 73.58458432824169) bank49994 +49995 POINT(40.397168666937716 73.85804662994506) bank49995 +49996 POINT(40.246526907981305 73.98242560810242) bank49996 +49997 POINT(40.81415723122535 73.14998959912451) bank49997 +49998 POINT(40.67990063896955 73.3202570034032) bank49998 +49999 POINT(40.876799428015886 73.03606317330242) bank49999 +50000 POINT(40.41654708757828 74.65034735251237) bank50000 +50001 POINT(40.13662246102269 74.0443162555023) bank50001 +50002 POINT(40.935414767012766 74.1701201320525) bank50002 +50003 POINT(40.40627440697937 73.80273046817476) bank50003 +50004 POINT(41.55057217090566 73.38835812441899) bank50004 +50005 POINT(41.65619912636462 73.81518981783319) bank50005 +50006 POINT(40.858827383959095 74.41995210060047) bank50006 +50007 POINT(41.54461814531663 74.64303352176125) bank50007 +50008 POINT(40.934421775725944 73.23335500610008) bank50008 +50009 POINT(41.17363462857179 73.62929621040921) bank50009 +50010 POINT(41.15116346874719 74.94194112934815) bank50010 +50011 POINT(40.09723645165622 73.45020713176083) bank50011 +50012 POINT(39.78865321789018 74.57038723173463) bank50012 +50013 POINT(39.989152153633704 74.00728364154456) bank50013 +50014 POINT(40.0779247505956 74.87534015133667) bank50014 +50015 POINT(40.71330784993348 74.15576970588857) bank50015 +50016 POINT(41.44521078676834 74.66711016215889) bank50016 +50017 POINT(40.48955414328726 73.16737134333671) bank50017 +50018 POINT(41.55359834410817 74.07885222840343) bank50018 +50019 POINT(39.73047548270318 73.9999084984075) bank50019 +50020 POINT(40.28185851315894 73.2354463970393) bank50020 +50021 POINT(40.20461481692144 73.88197123275292) bank50021 +50022 POINT(41.1313259493235 73.94915625932553) bank50022 +50023 POINT(41.384869347214924 74.67922588612103) bank50023 +50024 POINT(41.28666265593244 74.48491611793614) bank50024 +50025 POINT(40.15988304609609 74.46680332770043) bank50025 +50026 POINT(39.986370019172824 74.69073956490688) bank50026 +50027 POINT(39.98095424515294 74.76984177187266) bank50027 +50028 POINT(40.65297985803134 73.56569695331962) bank50028 +50029 POINT(41.58462653471887 73.45259837105603) bank50029 +50030 POINT(41.28973949812818 74.25260356827557) bank50030 +50031 POINT(40.552290176023035 74.93380810055044) bank50031 +50032 POINT(41.541167461195904 73.15900010777781) bank50032 +50033 POINT(39.94726034086221 73.46062647693782) bank50033 +50034 POINT(40.340184714417404 73.06075012232441) bank50034 +50035 POINT(40.20044661326407 73.07945458969208) bank50035 +50036 POINT(40.342458501395896 73.43050928707875) bank50036 +50037 POINT(41.42213005761701 73.91648620321335) bank50037 +50038 POINT(41.27359013809903 73.24648641285548) bank50038 +50039 POINT(41.66076270365771 74.52324982186077) bank50039 +50040 POINT(41.1491903925962 74.94318699739904) bank50040 +50041 POINT(41.35936421720045 73.12045317781704) bank50041 +50042 POINT(40.63171335133578 73.02048317441586) bank50042 +50043 POINT(40.05513142304732 74.9836227171429) bank50043 +50044 POINT(40.87808244793294 73.95387726813257) bank50044 +50045 POINT(41.37126071775308 74.41929334290224) bank50045 +50046 POINT(40.75357297952082 73.22056467907714) bank50046 +50047 POINT(39.910128255406384 74.53771670738801) bank50047 +50048 POINT(41.66152108832646 73.7966876765074) bank50048 +50049 POINT(40.305568554179935 73.08731632006085) bank50049 +50050 POINT(40.88857893880351 74.274913733127) bank50050 +50051 POINT(40.70584855564215 74.3245077148688) bank50051 +50052 POINT(41.17141525085397 73.64282260519138) bank50052 +50053 POINT(40.2158542844158 73.50349148055783) bank50053 +50054 POINT(40.17626965333709 74.27185940483396) bank50054 +50055 POINT(40.31932376184389 74.40182712137604) bank50055 +50056 POINT(40.74547401964795 74.59319808906525) bank50056 +50057 POINT(40.68617463610025 74.75396015389467) bank50057 +50058 POINT(40.06210793882814 74.28477314915898) bank50058 +50059 POINT(40.537670489655326 73.22350934241403) bank50059 +50060 POINT(40.978089736628036 73.48894558735529) bank50060 +50061 POINT(41.55613341630395 73.46637797514862) bank50061 +50062 POINT(39.75117096313084 73.88257951874411) bank50062 +50063 POINT(41.097841199910775 73.95178799549063) bank50063 +50064 POINT(40.49891385998514 74.87478830399981) bank50064 +50065 POINT(40.27814838449609 74.9734238398436) bank50065 +50066 POINT(40.6309403092887 73.25024595928448) bank50066 +50067 POINT(40.547548092679754 73.657181820352) bank50067 +50068 POINT(41.14213171690519 74.25366067114479) bank50068 +50069 POINT(40.2952202688279 74.44277927132201) bank50069 +50070 POINT(40.50913276250912 74.8267876932526) bank50070 +50071 POINT(40.53005246214879 74.55321273001778) bank50071 +50072 POINT(40.74325040133649 74.51327775494622) bank50072 +50073 POINT(40.86668420527047 73.02020570495095) bank50073 +50074 POINT(41.51822004364856 74.35052004875999) bank50074 +50075 POINT(40.22281720147754 74.58916276125788) bank50075 +50076 POINT(40.71573216577944 74.14871899013589) bank50076 +50077 POINT(41.118843186382286 74.28712876928037) bank50077 +50078 POINT(40.2333632782892 73.03235270954814) bank50078 +50079 POINT(40.667016865182774 73.60652943745347) bank50079 +50080 POINT(40.29542553840332 73.62596635044663) bank50080 +50081 POINT(41.45181341064907 73.89550875926997) bank50081 +50082 POINT(40.536549795482316 74.35403719125135) bank50082 +50083 POINT(41.05811585458342 73.64293666664804) bank50083 +50084 POINT(39.75868742863579 73.07156795771677) bank50084 +50085 POINT(39.73558733834827 74.23677731705786) bank50085 +50086 POINT(41.10028345245406 74.82178246829447) bank50086 +50087 POINT(39.76342940151647 73.75846254644782) bank50087 +50088 POINT(40.95737337278404 74.84220153575653) bank50088 +50089 POINT(39.78735451937172 74.09040018712912) bank50089 +50090 POINT(41.12414138913181 74.87520480689817) bank50090 +50091 POINT(41.427845189825646 73.80776726696357) bank50091 +50092 POINT(41.34282427766255 73.2542322148219) bank50092 +50093 POINT(41.67776261359373 74.91270678820457) bank50093 +50094 POINT(41.67504308751429 73.11317441834079) bank50094 +50095 POINT(40.14421481883474 73.38216070579193) bank50095 +50096 POINT(40.66987469494309 73.5215227517703) bank50096 +50097 POINT(41.10812903999557 73.97358911049108) bank50097 +50098 POINT(41.100617208517065 74.01822735088835) bank50098 +50099 POINT(40.36961891743201 73.12249667896963) bank50099 +50100 POINT(39.72055087479723 74.63340765168617) bank50100 +50101 POINT(40.88513822608681 73.3002202614737) bank50101 +50102 POINT(39.99142303913342 73.40248963693556) bank50102 +50103 POINT(40.42791526661861 73.72911719799903) bank50103 +50104 POINT(40.86954896822615 73.55382451230244) bank50104 +50105 POINT(40.19366668876409 74.54893435423763) bank50105 +50106 POINT(41.092395835257285 73.81588669891322) bank50106 +50107 POINT(40.90313784188349 73.89632572981289) bank50107 +50108 POINT(40.13067768025996 73.92328335158548) bank50108 +50109 POINT(39.761872527977374 73.85583371926181) bank50109 +50110 POINT(40.36161547287149 74.67662067409414) bank50110 +50111 POINT(40.70830030359406 74.14424636703961) bank50111 +50112 POINT(41.3293040215345 74.9371256830077) bank50112 +50113 POINT(40.01364480965495 73.70314789845806) bank50113 +50114 POINT(39.86688170082747 73.87842534120665) bank50114 +50115 POINT(41.30862300652583 73.42498585690836) bank50115 +50116 POINT(41.358976025293224 74.98818315839331) bank50116 +50117 POINT(40.85657296626415 73.51959308008898) bank50117 +50118 POINT(41.13731431235911 73.20111718533391) bank50118 +50119 POINT(40.54752670714336 73.91955920296186) bank50119 +50120 POINT(41.01638711360868 74.40799071706556) bank50120 +50121 POINT(39.75540856210638 74.4452833614814) bank50121 +50122 POINT(40.847972273489965 74.69848784087336) bank50122 +50123 POINT(40.37760864787802 73.92437814150904) bank50123 +50124 POINT(40.9414128222014 74.61047056362968) bank50124 +50125 POINT(41.25568147423018 73.6461349470436) bank50125 +50126 POINT(39.956735077376194 74.8200455160005) bank50126 +50127 POINT(40.610602086728406 73.21977947270952) bank50127 +50128 POINT(40.70415893498429 74.659764145532) bank50128 +50129 POINT(39.77894639755638 74.43164516659927) bank50129 +50130 POINT(41.30500022195279 73.02356738205938) bank50130 +50131 POINT(40.80633703816037 73.17532994492343) bank50131 +50132 POINT(40.2372801757449 73.64628946087043) bank50132 +50133 POINT(41.47329160061546 73.10067504876555) bank50133 +50134 POINT(40.18476119866807 73.81932794525942) bank50134 +50135 POINT(39.89838077412622 74.73895292093624) bank50135 +50136 POINT(40.41271717569727 74.1121665214856) bank50136 +50137 POINT(41.289699167884564 74.01576590575071) bank50137 +50138 POINT(39.822428895611345 74.39809795642697) bank50138 +50139 POINT(40.391071768844554 74.25257207535795) bank50139 +50140 POINT(41.562557768617225 73.45700083839803) bank50140 +50141 POINT(40.372505345965216 73.82024142767172) bank50141 +50142 POINT(40.11611467613399 73.26105435412214) bank50142 +50143 POINT(40.98317975065988 73.29530509569173) bank50143 +50144 POINT(41.58111206595315 73.10184099754224) bank50144 +50145 POINT(40.28724312183284 73.6456100867435) bank50145 +50146 POINT(40.053196053584955 73.2987067085136) bank50146 +50147 POINT(39.90925030484682 74.29176870647292) bank50147 +50148 POINT(41.58838295894178 73.92756749463918) bank50148 +50149 POINT(39.751625881658505 73.74926038688623) bank50149 +50150 POINT(40.351892302124305 73.89339182391828) bank50150 +50151 POINT(40.251102494418795 74.14013079941239) bank50151 +50152 POINT(40.41207202233925 74.62193941341215) bank50152 +50153 POINT(41.02178707465792 73.07951617723954) bank50153 +50154 POINT(40.08252737401764 73.53675430532239) bank50154 +50155 POINT(40.54065037485017 74.61726799853393) bank50155 +50156 POINT(40.89641275058394 74.70063666443549) bank50156 +50157 POINT(41.30198005504944 73.44221203913158) bank50157 +50158 POINT(41.48654126756808 74.61452931698986) bank50158 +50159 POINT(41.03479749165951 73.20137106958096) bank50159 +50160 POINT(41.21635931576006 74.51084946796442) bank50160 +50161 POINT(40.380768059937566 73.01083753302214) bank50161 +50162 POINT(41.47702500642067 73.99773936664128) bank50162 +50163 POINT(41.27981052824738 74.57657827089633) bank50163 +50164 POINT(40.83513831071754 73.52704553869958) bank50164 +50165 POINT(39.73207622800842 73.6584148199295) bank50165 +50166 POINT(41.3421956758091 73.8744887941883) bank50166 +50167 POINT(40.332222094899485 73.79363678061083) bank50167 +50168 POINT(40.33005063181605 73.73921327691606) bank50168 +50169 POINT(41.19399358379615 73.90925778991416) bank50169 +50170 POINT(40.96945245600534 74.56613252470332) bank50170 +50171 POINT(41.295380023895184 74.13697495474261) bank50171 +50172 POINT(41.68481371394204 73.55659285983324) bank50172 +50173 POINT(40.995743827753024 74.22899812640472) bank50173 +50174 POINT(41.444313220479664 73.23670421336975) bank50174 +50175 POINT(40.63259074247591 73.6816458525142) bank50175 +50176 POINT(41.598828321867266 73.02517321933504) bank50176 +50177 POINT(41.036171891694984 74.02596142664538) bank50177 +50178 POINT(40.423101509632716 74.49367923730522) bank50178 +50179 POINT(40.37392822838504 73.53220123829128) bank50179 +50180 POINT(40.065192857418005 73.08802591545799) bank50180 +50181 POINT(39.80152578901022 73.86788642443346) bank50181 +50182 POINT(39.86532528773381 73.2787270209532) bank50182 +50183 POINT(39.997887512296266 73.82161896991931) bank50183 +50184 POINT(40.39184968225827 74.69751321940416) bank50184 +50185 POINT(40.152578310872876 74.25795097748686) bank50185 +50186 POINT(40.60990085328926 74.1713850773769) bank50186 +50187 POINT(41.1999034433136 73.36679861241836) bank50187 +50188 POINT(41.36452313493033 74.23282855163443) bank50188 +50189 POINT(40.799109008421304 74.61449899576823) bank50189 +50190 POINT(41.659465989504845 74.53131862248479) bank50190 +50191 POINT(41.68726735445712 74.21865057498535) bank50191 +50192 POINT(40.47367711453238 73.57857161166963) bank50192 +50193 POINT(41.40954773131889 74.93559180819473) bank50193 +50194 POINT(40.89405840986803 73.80302387748597) bank50194 +50195 POINT(41.661229528476845 73.72271463014003) bank50195 +50196 POINT(41.37669161384294 74.85502483900144) bank50196 +50197 POINT(40.608544691777475 73.333042428541) bank50197 +50198 POINT(40.798613212706584 74.07793969322269) bank50198 +50199 POINT(41.26101498352908 73.28313576183652) bank50199 +50200 POINT(40.49375163971426 73.51820906903642) bank50200 +50201 POINT(39.956628397634155 73.71176455027188) bank50201 +50202 POINT(40.46463981500346 73.36516565982886) bank50202 +50203 POINT(40.87810105638958 74.27857882176237) bank50203 +50204 POINT(41.33127959544299 74.08553739436577) bank50204 +50205 POINT(40.65803393154633 73.57884917551385) bank50205 +50206 POINT(40.32377216848626 73.85455295746502) bank50206 +50207 POINT(40.96906278455177 74.68140127160156) bank50207 +50208 POINT(41.03536521000723 74.51979729114339) bank50208 +50209 POINT(40.640638431328796 73.13657579558105) bank50209 +50210 POINT(41.61236579745538 74.94767204784189) bank50210 +50211 POINT(40.58464849819204 74.76247789580204) bank50211 +50212 POINT(41.38799852132968 74.74313256044128) bank50212 +50213 POINT(41.23466857755599 73.74010766072598) bank50213 +50214 POINT(41.23390109023613 73.0333791318073) bank50214 +50215 POINT(41.46577342539744 73.79151949026065) bank50215 +50216 POINT(39.788210208211595 73.95025776902841) bank50216 +50217 POINT(40.64742263986178 74.0976809863197) bank50217 +50218 POINT(41.29694000549509 73.55960466472193) bank50218 +50219 POINT(40.52543587457382 73.82561998127046) bank50219 +50220 POINT(41.58577578568217 74.5679444052222) bank50220 +50221 POINT(39.82830754781813 74.83116809103652) bank50221 +50222 POINT(40.44971084807274 73.70187003493153) bank50222 +50223 POINT(39.73989400353043 74.33744136163175) bank50223 +50224 POINT(40.0021232330192 73.07717325209087) bank50224 +50225 POINT(40.20941689952217 74.67037900559522) bank50225 +50226 POINT(41.323119979337854 74.87710795376496) bank50226 +50227 POINT(41.20250887422065 73.40843002866833) bank50227 +50228 POINT(41.184643323751004 73.33207245377424) bank50228 +50229 POINT(41.21728747336046 74.55198828003589) bank50229 +50230 POINT(40.29274007548616 74.94931949402323) bank50230 +50231 POINT(40.584205361358414 74.3023926239864) bank50231 +50232 POINT(40.84742922911089 74.07830946044352) bank50232 +50233 POINT(41.333442577014885 73.07581360366217) bank50233 +50234 POINT(40.73742150356569 73.95595156284091) bank50234 +50235 POINT(40.77449115158831 73.03276262234832) bank50235 +50236 POINT(40.430550640976804 74.87628014821382) bank50236 +50237 POINT(40.68719867223658 74.66240671415898) bank50237 +50238 POINT(40.59095459785189 73.08862696737157) bank50238 +50239 POINT(40.401868064787124 74.69597134418387) bank50239 +50240 POINT(41.47363479418931 74.66258481608546) bank50240 +50241 POINT(41.66554997099949 74.37793910026194) bank50241 +50242 POINT(40.664208045161224 74.97877329228152) bank50242 +50243 POINT(40.207024240504175 74.62420756862808) bank50243 +50244 POINT(41.325577790423516 74.98438456645223) bank50244 +50245 POINT(40.30744218625802 74.86445693399428) bank50245 +50246 POINT(39.790711684782714 73.94427937564484) bank50246 +50247 POINT(40.770392846285105 74.79915945485152) bank50247 +50248 POINT(39.90578716867486 73.25542857687165) bank50248 +50249 POINT(40.640905722868034 74.28648582547687) bank50249 +50250 POINT(40.017102714668816 74.9557760752713) bank50250 +50251 POINT(40.66357459631762 74.47478967843793) bank50251 +50252 POINT(41.464709898971954 74.48360384659553) bank50252 +50253 POINT(40.87076396098162 74.91636888010645) bank50253 +50254 POINT(41.2125030224292 73.20448009571888) bank50254 +50255 POINT(40.78709961294687 74.63608911792151) bank50255 +50256 POINT(41.56035069156187 74.6452570421546) bank50256 +50257 POINT(41.21289623112325 74.83064735411952) bank50257 +50258 POINT(40.524811528165756 74.92643248900343) bank50258 +50259 POINT(40.67545221809314 74.22722904024116) bank50259 +50260 POINT(40.57608697849893 73.15884644398542) bank50260 +50261 POINT(41.2304580639049 73.65745541712697) bank50261 +50262 POINT(40.48814489835475 73.06344418964845) bank50262 +50263 POINT(40.51106452670748 73.19002371599174) bank50263 +50264 POINT(40.85437414642929 74.00964403745867) bank50264 +50265 POINT(40.23042571082683 74.88513795109937) bank50265 +50266 POINT(41.544212879232454 74.84648205770857) bank50266 +50267 POINT(40.415194710803064 74.67646724561332) bank50267 +50268 POINT(40.05409637964258 73.4464161508771) bank50268 +50269 POINT(41.43546808926512 73.33916286770537) bank50269 +50270 POINT(41.42040411211597 73.73894347466722) bank50270 +50271 POINT(41.01354126832796 73.91434654334475) bank50271 +50272 POINT(41.2832382383167 74.88956464167784) bank50272 +50273 POINT(41.213686856186435 73.66844283489571) bank50273 +50274 POINT(41.27844854628242 74.3079609646652) bank50274 +50275 POINT(40.37878789315387 73.6259993064928) bank50275 +50276 POINT(39.966290142077305 74.50889590579568) bank50276 +50277 POINT(41.0099394290524 73.11323830579947) bank50277 +50278 POINT(40.09143675796664 74.58924493987601) bank50278 +50279 POINT(41.214784351752215 73.27235688725612) bank50279 +50280 POINT(40.959092390161835 73.14445051797624) bank50280 +50281 POINT(41.580971867158375 73.16265498242485) bank50281 +50282 POINT(41.41598886209979 73.53350757369952) bank50282 +50283 POINT(40.71581893199891 74.48698764768346) bank50283 +50284 POINT(41.526765021444305 74.89307147627683) bank50284 +50285 POINT(39.72283569837941 74.23895735041899) bank50285 +50286 POINT(41.513114521786584 73.45030758822007) bank50286 +50287 POINT(40.67684218613713 73.76350228669558) bank50287 +50288 POINT(41.28662760578576 73.10728060563122) bank50288 +50289 POINT(39.93501035160358 74.94028863205702) bank50289 +50290 POINT(41.07861723252096 74.5324001850674) bank50290 +50291 POINT(39.75608801591125 74.10661276484107) bank50291 +50292 POINT(41.25571549862085 74.076036371893) bank50292 +50293 POINT(40.482946907210504 74.04702450755518) bank50293 +50294 POINT(39.82892889519574 73.858418057919) bank50294 +50295 POINT(40.35635283921921 73.59235786407797) bank50295 +50296 POINT(39.96478522868131 73.61992200745206) bank50296 +50297 POINT(40.841939324345375 74.58156848635993) bank50297 +50298 POINT(40.373206193160584 73.5916160137323) bank50298 +50299 POINT(41.501495331366186 73.23992912168514) bank50299 +50300 POINT(39.96542940410849 73.95080063077226) bank50300 +50301 POINT(40.6116417491603 73.2828302992011) bank50301 +50302 POINT(41.54704697225237 73.5353862813131) bank50302 +50303 POINT(39.989255414345585 73.11147232698389) bank50303 +50304 POINT(41.125860027355095 73.22798358906057) bank50304 +50305 POINT(40.88627827136302 74.05867303891918) bank50305 +50306 POINT(40.904401159713906 73.2309067926262) bank50306 +50307 POINT(40.990492531535665 73.29402040514323) bank50307 +50308 POINT(41.70709720663724 74.17475109360146) bank50308 +50309 POINT(40.22354383335921 73.34264609713854) bank50309 +50310 POINT(40.90609767733043 73.96293176756387) bank50310 +50311 POINT(40.46590090480931 74.90960036310885) bank50311 +50312 POINT(41.41754879602511 74.49922759660194) bank50312 +50313 POINT(40.754791691429546 73.84202439942794) bank50313 +50314 POINT(41.2999302385365 74.60841284355595) bank50314 +50315 POINT(41.579264029762435 73.12004684537183) bank50315 +50316 POINT(41.677881633956005 74.27813381021426) bank50316 +50317 POINT(41.495993929685916 73.89161739296203) bank50317 +50318 POINT(40.12687642849841 73.96695154420279) bank50318 +50319 POINT(41.64465856593355 73.83380029269327) bank50319 +50320 POINT(40.81914297145091 74.7790340725607) bank50320 +50321 POINT(40.01020641793085 73.89416064325508) bank50321 +50322 POINT(41.481948479419906 73.33536527127002) bank50322 +50323 POINT(41.50782775116932 73.76595480558146) bank50323 +50324 POINT(40.80961047497215 74.68142182939401) bank50324 +50325 POINT(41.023019879419984 74.68660778585745) bank50325 +50326 POINT(40.40951746896996 73.66719523822736) bank50326 +50327 POINT(41.29244111607759 73.13478222776324) bank50327 +50328 POINT(41.27876755331618 73.36374754476812) bank50328 +50329 POINT(40.15393236464905 74.3770234083244) bank50329 +50330 POINT(40.40113615109334 73.46179286178187) bank50330 +50331 POINT(39.73877964071973 73.76824545506273) bank50331 +50332 POINT(41.26434329603424 74.40879779534431) bank50332 +50333 POINT(40.562949377250355 74.21252680664941) bank50333 +50334 POINT(40.12562385863409 73.80465089993045) bank50334 +50335 POINT(40.47565254420581 73.0995609530551) bank50335 +50336 POINT(41.53142356321119 73.67313947422642) bank50336 +50337 POINT(39.73043776669071 73.77434911833238) bank50337 +50338 POINT(41.401939332034736 73.40536936871143) bank50338 +50339 POINT(40.36967738081402 73.63260113313753) bank50339 +50340 POINT(40.396968812468316 74.21204738839705) bank50340 +50341 POINT(40.13135924634915 73.49176985135317) bank50341 +50342 POINT(41.023452277545246 74.77968272927124) bank50342 +50343 POINT(41.26024448760743 73.8468762417352) bank50343 +50344 POINT(41.05523105392567 74.31633257611988) bank50344 +50345 POINT(40.09572719059874 73.57580116377852) bank50345 +50346 POINT(40.459888276542785 74.38360144810996) bank50346 +50347 POINT(40.103084874976034 73.64174385346627) bank50347 +50348 POINT(40.90068545099889 73.9511297020747) bank50348 +50349 POINT(40.06638005248456 73.64696559654088) bank50349 +50350 POINT(41.28998823459521 74.0429193423394) bank50350 +50351 POINT(39.8089993552854 74.43054796708229) bank50351 +50352 POINT(40.47811196663659 73.12470683771109) bank50352 +50353 POINT(40.42145166424193 74.58158343823919) bank50353 +50354 POINT(40.93316007190879 73.63157651512901) bank50354 +50355 POINT(41.358295516014834 73.01923200449188) bank50355 +50356 POINT(39.92561904632539 74.29963301180744) bank50356 +50357 POINT(41.521801502304264 74.67287931087255) bank50357 +50358 POINT(40.506119270274716 73.58277281235145) bank50358 +50359 POINT(40.66026615259483 73.01355182253002) bank50359 +50360 POINT(41.46012765530434 74.7278936924481) bank50360 +50361 POINT(41.42347983242502 73.08409245114005) bank50361 +50362 POINT(39.94909739084547 74.70461689382876) bank50362 +50363 POINT(40.499455672540066 74.76479385433605) bank50363 +50364 POINT(40.290442912959854 74.36623242918103) bank50364 +50365 POINT(41.70359690960217 73.14679300793259) bank50365 +50366 POINT(40.3910019563113 74.35300223358065) bank50366 +50367 POINT(41.11858932794702 74.78258686721229) bank50367 +50368 POINT(40.34250932344468 74.02007179358115) bank50368 +50369 POINT(40.2065347292495 73.38864876584307) bank50369 +50370 POINT(39.97988908893388 73.4446760574695) bank50370 +50371 POINT(41.29614668162292 74.42934917854393) bank50371 +50372 POINT(41.67733855469771 74.02441121413791) bank50372 +50373 POINT(40.61087713018874 73.62359821334647) bank50373 +50374 POINT(40.715941610013004 74.47002190433219) bank50374 +50375 POINT(40.84753319494329 73.08082507389662) bank50375 +50376 POINT(40.32970775548673 73.77763698940919) bank50376 +50377 POINT(40.42178960670997 73.37212208436864) bank50377 +50378 POINT(40.30169381071296 73.2355934430689) bank50378 +50379 POINT(41.020681335881655 74.92043312123809) bank50379 +50380 POINT(40.93820715013085 73.07564033976429) bank50380 +50381 POINT(40.91492902605952 74.33877719076) bank50381 +50382 POINT(41.53345195856289 73.61134548620413) bank50382 +50383 POINT(40.27553869398956 74.7214996198013) bank50383 +50384 POINT(41.33880392133163 74.97901311167017) bank50384 +50385 POINT(40.098967799178766 73.7478647753316) bank50385 +50386 POINT(41.59600095862295 73.06512427403712) bank50386 +50387 POINT(40.01192954026085 74.82242796156238) bank50387 +50388 POINT(40.59568567043681 73.04819945359215) bank50388 +50389 POINT(39.75360613421292 73.89733225640533) bank50389 +50390 POINT(40.70712948192293 74.38007894527122) bank50390 +50391 POINT(39.79145096003231 73.85090948628867) bank50391 +50392 POINT(40.19532715777049 73.58861915320317) bank50392 +50393 POINT(40.209411549607395 74.76401547505111) bank50393 +50394 POINT(40.25975891326124 74.70686233043283) bank50394 +50395 POINT(41.1212588584375 73.283093501111) bank50395 +50396 POINT(40.955773876824715 73.54251153362075) bank50396 +50397 POINT(39.716895697675476 73.05244463920177) bank50397 +50398 POINT(40.26903899788393 73.8270932307834) bank50398 +50399 POINT(39.82349564472355 73.9237640096505) bank50399 +50400 POINT(39.86707839851544 74.28913620540186) bank50400 +50401 POINT(39.91330002128401 73.35271597535078) bank50401 +50402 POINT(40.41137261273872 74.7894542906125) bank50402 +50403 POINT(41.61036212958236 73.10832862604047) bank50403 +50404 POINT(41.56599056800563 74.23769560112555) bank50404 +50405 POINT(40.2806749975891 74.53575137569612) bank50405 +50406 POINT(41.40584724219829 73.52679189630159) bank50406 +50407 POINT(41.00355935886223 73.93240209749368) bank50407 +50408 POINT(41.56211403197821 74.24988388141654) bank50408 +50409 POINT(40.6320780806317 73.76537490064035) bank50409 +50410 POINT(41.387923066120884 73.03456203722095) bank50410 +50411 POINT(40.47993163694164 74.63240309992878) bank50411 +50412 POINT(41.37634189622344 73.50023497754917) bank50412 +50413 POINT(41.05103724488874 73.09563160121944) bank50413 +50414 POINT(39.92542806280202 74.12595429497249) bank50414 +50415 POINT(39.76671434563326 74.54568227225096) bank50415 +50416 POINT(39.732688171064005 74.10767084483741) bank50416 +50417 POINT(40.94062315886239 73.36115530053729) bank50417 +50418 POINT(40.1437762765816 74.07385450684409) bank50418 +50419 POINT(40.48050062057264 73.41813560791257) bank50419 +50420 POINT(39.78859893712284 73.85065256757419) bank50420 +50421 POINT(39.80326186761715 73.3353785541961) bank50421 +50422 POINT(41.48713222550918 74.03109629512839) bank50422 +50423 POINT(40.59993799824926 74.32501686924118) bank50423 +50424 POINT(41.67596559087197 73.89524778082843) bank50424 +50425 POINT(40.019777788743845 73.23507951229907) bank50425 +50426 POINT(39.944780550440534 74.28496963478973) bank50426 +50427 POINT(40.40605532599151 74.04404598066581) bank50427 +50428 POINT(41.6720314759244 75.00241673593787) bank50428 +50429 POINT(40.02258639665592 74.50924452299768) bank50429 +50430 POINT(40.808561204873115 74.98438279869424) bank50430 +50431 POINT(40.79358738869445 74.21927714120336) bank50431 +50432 POINT(41.194179132511785 73.4911941007489) bank50432 +50433 POINT(40.1573576529512 74.64490640489562) bank50433 +50434 POINT(41.588450839943704 74.92157661148482) bank50434 +50435 POINT(41.05871922836829 74.07269409680298) bank50435 +50436 POINT(41.59454466644181 73.66050416337471) bank50436 +50437 POINT(40.362452874821585 73.44246653881706) bank50437 +50438 POINT(41.362453919767155 74.58108972683893) bank50438 +50439 POINT(40.80674718054951 74.81166564378606) bank50439 +50440 POINT(40.491585567373626 74.45706172466213) bank50440 +50441 POINT(41.251111626379355 74.85327800254765) bank50441 +50442 POINT(40.54484084540813 73.8727401964517) bank50442 +50443 POINT(40.137098468930425 74.60719872527515) bank50443 +50444 POINT(40.45368996480689 74.97831548085098) bank50444 +50445 POINT(41.0685472753383 73.71942834944528) bank50445 +50446 POINT(41.2516179316068 74.57792599353326) bank50446 +50447 POINT(40.62650691609171 73.34985043210253) bank50447 +50448 POINT(40.990998076054396 74.64311291950528) bank50448 +50449 POINT(40.91100961311235 74.62048477522488) bank50449 +50450 POINT(40.11000291429859 74.02568022057109) bank50450 +50451 POINT(40.440649919400094 74.90338061064888) bank50451 +50452 POINT(41.416517674346146 74.47773805482325) bank50452 +50453 POINT(41.64956978510545 74.80392823538754) bank50453 +50454 POINT(41.45739697088742 74.09258102775125) bank50454 +50455 POINT(40.7308024424217 73.0727465314382) bank50455 +50456 POINT(40.224949180001246 73.29665228802882) bank50456 +50457 POINT(40.34129945453361 73.86976779514752) bank50457 +50458 POINT(40.86073046792197 73.0509314209576) bank50458 +50459 POINT(41.210773047885205 73.46844174554347) bank50459 +50460 POINT(41.14155675820917 73.3866715754611) bank50460 +50461 POINT(40.72275547060981 74.40527784434362) bank50461 +50462 POINT(40.36867128586057 73.52844679227742) bank50462 +50463 POINT(40.97274490457555 74.57267044052189) bank50463 +50464 POINT(40.803750130518196 73.10863649501847) bank50464 +50465 POINT(41.062453216025496 74.73795248298408) bank50465 +50466 POINT(41.518811147466074 73.97018249572925) bank50466 +50467 POINT(41.26160302379498 73.09342591450877) bank50467 +50468 POINT(40.01519419106024 74.43751313849786) bank50468 +50469 POINT(41.18423618615468 73.47318937854091) bank50469 +50470 POINT(40.405356582425156 74.8131293427421) bank50470 +50471 POINT(40.8885733105613 74.15176432182186) bank50471 +50472 POINT(40.45966759625198 73.22610769014173) bank50472 +50473 POINT(41.153587194808054 74.53380865639507) bank50473 +50474 POINT(40.858439041947975 74.04930264964183) bank50474 +50475 POINT(39.96689926542925 73.51873575022388) bank50475 +50476 POINT(40.078309369907416 74.26336888040697) bank50476 +50477 POINT(40.812133826598405 74.82230750439187) bank50477 +50478 POINT(41.442663470399665 73.17799156022905) bank50478 +50479 POINT(40.75949782852384 74.74109677517865) bank50479 +50480 POINT(39.89017768578394 73.14412964523434) bank50480 +50481 POINT(40.3911201709381 74.71388657964351) bank50481 +50482 POINT(40.22003855430019 74.42790172055858) bank50482 +50483 POINT(40.7004105909552 74.37853249986821) bank50483 +50484 POINT(41.31077128084301 74.10615042099724) bank50484 +50485 POINT(40.0707088975452 73.91252080738768) bank50485 +50486 POINT(39.714695326236466 74.51510838990765) bank50486 +50487 POINT(41.437705108959335 73.88645529484198) bank50487 +50488 POINT(41.05178373449707 73.35360433362544) bank50488 +50489 POINT(39.903078976027764 73.04546160125835) bank50489 +50490 POINT(41.46030440280499 74.79077259502546) bank50490 +50491 POINT(39.87330424680714 73.08877912004668) bank50491 +50492 POINT(40.26899512454681 73.22035905108159) bank50492 +50493 POINT(41.28124325752016 74.73930055839134) bank50493 +50494 POINT(39.75871514668213 74.66307962968644) bank50494 +50495 POINT(41.505400126403124 73.25448073400652) bank50495 +50496 POINT(40.128015929294634 73.20033219831092) bank50496 +50497 POINT(40.15648628740431 73.3479813415663) bank50497 +50498 POINT(40.46747018162276 73.84133420927827) bank50498 +50499 POINT(40.44038848140888 73.61500629734405) bank50499 +50500 POINT(40.68484595036967 74.35554365500008) bank50500 +50501 POINT(40.12459856165756 74.57166667672904) bank50501 +50502 POINT(40.31307164983649 73.63134882429542) bank50502 +50503 POINT(39.96557474511867 74.15378383706873) bank50503 +50504 POINT(39.940483204187714 74.54806374994132) bank50504 +50505 POINT(40.0709164476311 74.0065119964965) bank50505 +50506 POINT(40.81677359287445 74.41387711388089) bank50506 +50507 POINT(40.82011344997053 74.62201620160987) bank50507 +50508 POINT(41.21127814455582 73.62918894242105) bank50508 +50509 POINT(39.87990469232595 74.58600577675176) bank50509 +50510 POINT(41.263675802808116 73.84828667923688) bank50510 +50511 POINT(39.95365810046398 74.27577258678846) bank50511 +50512 POINT(39.962397555575464 74.39264388858842) bank50512 +50513 POINT(39.72815197397177 73.48142483133643) bank50513 +50514 POINT(40.73080569506996 73.03908373365982) bank50514 +50515 POINT(41.563123284056395 74.01840288960437) bank50515 +50516 POINT(40.59114283766892 73.29387093989293) bank50516 +50517 POINT(40.17176465208096 74.22270083175448) bank50517 +50518 POINT(40.67731125737331 73.98981564100815) bank50518 +50519 POINT(41.238590762491015 73.99348253178331) bank50519 +50520 POINT(41.622129076125724 74.6769900891037) bank50520 +50521 POINT(41.46629535839234 74.47103356064378) bank50521 +50522 POINT(41.24294172222268 74.537925561745) bank50522 +50523 POINT(41.580569105466324 74.21744317332328) bank50523 +50524 POINT(40.268913671740926 74.71252793958712) bank50524 +50525 POINT(40.66545531210373 73.90572057676452) bank50525 +50526 POINT(40.130496387229584 73.10531763204305) bank50526 +50527 POINT(40.820461822866136 73.50381742630212) bank50527 +50528 POINT(41.3446190880238 73.87148921177075) bank50528 +50529 POINT(41.22622245852447 73.56282064673434) bank50529 +50530 POINT(39.985146774491774 74.29433266679165) bank50530 +50531 POINT(40.1915414313544 73.13542831810481) bank50531 +50532 POINT(41.49267243147208 74.3513921308484) bank50532 +50533 POINT(41.66826477327973 74.43310861782484) bank50533 +50534 POINT(41.63193550178569 73.66928070500288) bank50534 +50535 POINT(41.3897563385002 73.76594203615407) bank50535 +50536 POINT(41.55503431536161 73.84008312127344) bank50536 +50537 POINT(41.485712604110994 74.07738461871028) bank50537 +50538 POINT(40.88613862739498 74.1096384364777) bank50538 +50539 POINT(40.70118846509238 73.058629926236) bank50539 +50540 POINT(41.11128992960175 73.27946088906228) bank50540 +50541 POINT(41.339565106321146 74.03917452918041) bank50541 +50542 POINT(41.14776671858547 74.90234404559925) bank50542 +50543 POINT(41.7034191460659 74.05869879166255) bank50543 +50544 POINT(40.12706194367888 73.05218205690824) bank50544 +50545 POINT(40.48581640321051 73.79337195971699) bank50545 +50546 POINT(41.40138772405775 74.8736669004248) bank50546 +50547 POINT(40.55653971993478 74.30763668926933) bank50547 +50548 POINT(40.12143458275424 74.84252607058005) bank50548 +50549 POINT(40.47604569874184 74.93509325781217) bank50549 +50550 POINT(40.12980445564685 74.00170531193505) bank50550 +50551 POINT(40.32528928711593 73.38470495181492) bank50551 +50552 POINT(40.32763916468335 74.06581737151971) bank50552 +50553 POINT(41.015289642250785 73.26465263977296) bank50553 +50554 POINT(41.62484432485266 74.77114691440096) bank50554 +50555 POINT(41.08924346528285 74.87324205370696) bank50555 +50556 POINT(40.46727766194509 73.68096128911813) bank50556 +50557 POINT(41.264447246684796 73.91639723287444) bank50557 +50558 POINT(41.029434806553446 73.97875115463576) bank50558 +50559 POINT(40.44949526115998 73.72497622214543) bank50559 +50560 POINT(41.157963663176005 74.20508024929845) bank50560 +50561 POINT(39.73365818086265 74.57423707094613) bank50561 +50562 POINT(41.38548792892923 73.8873913567259) bank50562 +50563 POINT(40.762060557673934 73.93675260828026) bank50563 +50564 POINT(39.790218178117584 73.32558445750811) bank50564 +50565 POINT(41.66395590391244 74.25015220819373) bank50565 +50566 POINT(41.244821682258404 73.84853702848339) bank50566 +50567 POINT(39.91664236900393 74.19427869821196) bank50567 +50568 POINT(40.565909370539735 73.7401341798503) bank50568 +50569 POINT(40.636137611597654 73.93564885976693) bank50569 +50570 POINT(41.537975976649456 74.71374257703688) bank50570 +50571 POINT(40.79715305292069 74.2104749417031) bank50571 +50572 POINT(40.641528219413466 73.40961370425559) bank50572 +50573 POINT(41.509120588644976 73.92666340575042) bank50573 +50574 POINT(41.04434112728258 73.62159241642671) bank50574 +50575 POINT(41.63653705250231 74.77452828746165) bank50575 +50576 POINT(40.28920865793378 74.34859590685967) bank50576 +50577 POINT(41.559999796054356 74.85042838105007) bank50577 +50578 POINT(40.33173765419376 73.26844678869979) bank50578 +50579 POINT(40.74536204945723 74.31649811435318) bank50579 +50580 POINT(40.57890253472994 74.22292589379124) bank50580 +50581 POINT(40.79102953077924 74.52643238504744) bank50581 +50582 POINT(41.25863746543962 73.25762736738523) bank50582 +50583 POINT(40.99538853226915 73.05336970884373) bank50583 +50584 POINT(40.77252012306532 73.1387078552346) bank50584 +50585 POINT(40.589820625046464 74.02668781436468) bank50585 +50586 POINT(40.61623699866257 73.09413794939381) bank50586 +50587 POINT(40.06583544119447 74.04362389782423) bank50587 +50588 POINT(40.49025774711836 73.99550042545425) bank50588 +50589 POINT(39.826582328128694 74.9881438950982) bank50589 +50590 POINT(40.87151644722039 74.83377198800795) bank50590 +50591 POINT(39.93266666086213 74.05661579103) bank50591 +50592 POINT(40.99737003388448 73.1032091316075) bank50592 +50593 POINT(41.66655494993132 73.3478921836997) bank50593 +50594 POINT(41.696719480659475 73.81444398769806) bank50594 +50595 POINT(39.957013735975146 74.07873280574437) bank50595 +50596 POINT(41.23912378824774 73.46709426882735) bank50596 +50597 POINT(40.8474233964602 74.2226258384893) bank50597 +50598 POINT(40.157312528009754 73.49869398157853) bank50598 +50599 POINT(40.422066615993096 73.41788166251776) bank50599 +50600 POINT(41.365535411695774 74.06150236965784) bank50600 +50601 POINT(40.65402907546273 74.14791654427519) bank50601 +50602 POINT(40.326715641461476 74.52307975726899) bank50602 +50603 POINT(40.0907747821869 73.26541775890662) bank50603 +50604 POINT(40.30389944069217 74.53909754095896) bank50604 +50605 POINT(39.9253280190551 74.21034612829686) bank50605 +50606 POINT(40.958340619237234 73.11866222152553) bank50606 +50607 POINT(41.28979471113982 74.41590855641621) bank50607 +50608 POINT(41.395366650054626 73.38937696674138) bank50608 +50609 POINT(40.76845260476595 74.86392328369848) bank50609 +50610 POINT(40.31989948440131 74.52556860653301) bank50610 +50611 POINT(40.00464802063921 74.82626558304568) bank50611 +50612 POINT(40.53187406717444 74.72822006466093) bank50612 +50613 POINT(41.570253731436 74.86591834027939) bank50613 +50614 POINT(40.41752885087168 74.11315567186095) bank50614 +50615 POINT(41.103235259688866 73.97495455017206) bank50615 +50616 POINT(40.840216267790375 75.0043197384995) bank50616 +50617 POINT(40.02254908156804 74.21304881445388) bank50617 +50618 POINT(41.56316959879527 73.24165788188562) bank50618 +50619 POINT(41.03885518499238 74.01493622462534) bank50619 +50620 POINT(40.63051855171703 74.60124531301042) bank50620 +50621 POINT(39.823300675846625 73.7938025155514) bank50621 +50622 POINT(41.5271903002145 74.0922118302607) bank50622 +50623 POINT(41.476033206115396 74.63992872561636) bank50623 +50624 POINT(41.09951762817262 73.36293483998) bank50624 +50625 POINT(40.645947680726145 73.16540393724132) bank50625 +50626 POINT(40.292214950105645 74.6210124719082) bank50626 +50627 POINT(40.04266238481549 73.86259862894103) bank50627 +50628 POINT(39.912716436611326 73.55165338223473) bank50628 +50629 POINT(41.27704086085382 74.21607961078227) bank50629 +50630 POINT(41.29643581203057 74.83353721196251) bank50630 +50631 POINT(41.16795743897897 73.10807183505194) bank50631 +50632 POINT(39.9709423095175 73.83033961939292) bank50632 +50633 POINT(39.97948808457706 74.46601884441185) bank50633 +50634 POINT(41.23899064563985 73.6723346359245) bank50634 +50635 POINT(40.690311199684245 74.18354141136179) bank50635 +50636 POINT(40.553987912531674 74.26944385503383) bank50636 +50637 POINT(41.634937670911846 74.681828845947) bank50637 +50638 POINT(41.27531771716587 73.59841164688493) bank50638 +50639 POINT(40.07375248259496 73.80255232559064) bank50639 +50640 POINT(40.939459020058 74.15349327526052) bank50640 +50641 POINT(40.093203863842035 74.99942179429694) bank50641 +50642 POINT(39.95589142063681 73.43562625339389) bank50642 +50643 POINT(39.80856801086845 74.50794779998496) bank50643 +50644 POINT(40.27763814087724 74.22468653156223) bank50644 +50645 POINT(41.39244892581394 74.56083571740965) bank50645 +50646 POINT(41.45122650510359 74.7955037271531) bank50646 +50647 POINT(39.88773128726337 73.45234633534072) bank50647 +50648 POINT(39.798342880225505 74.73121730681483) bank50648 +50649 POINT(39.91247214930703 73.07944077536403) bank50649 +50650 POINT(40.02714032244543 74.55499458046185) bank50650 +50651 POINT(40.543260017467624 73.51117025297738) bank50651 +50652 POINT(41.381491752108445 74.53853579591409) bank50652 +50653 POINT(41.312991765005414 73.04566564782267) bank50653 +50654 POINT(40.510201897840496 74.51538057251202) bank50654 +50655 POINT(39.934290939865 74.77745579991257) bank50655 +50656 POINT(39.759070231800436 73.05387722134581) bank50656 +50657 POINT(40.008589423034934 73.60357179264902) bank50657 +50658 POINT(41.5413435942621 74.1596853304029) bank50658 +50659 POINT(41.00219704104601 74.29498883996676) bank50659 +50660 POINT(40.99214187518891 74.30852300990077) bank50660 +50661 POINT(41.1244364237502 74.70107061848334) bank50661 +50662 POINT(40.958447763730774 73.62141831604981) bank50662 +50663 POINT(41.01904124477645 73.31421368840341) bank50663 +50664 POINT(40.43701101519319 73.58328589935287) bank50664 +50665 POINT(41.17392813241915 73.37199540318369) bank50665 +50666 POINT(40.16975069516238 73.67363863705165) bank50666 +50667 POINT(40.93553566744817 73.89348330888768) bank50667 +50668 POINT(40.20754648957961 73.51352092509663) bank50668 +50669 POINT(41.08028718233396 74.48419023282453) bank50669 +50670 POINT(40.965161364401915 73.25745374427201) bank50670 +50671 POINT(41.42162648144418 73.46812268474193) bank50671 +50672 POINT(39.946323987894075 73.7013505672435) bank50672 +50673 POINT(41.265477318342974 74.4242323594601) bank50673 +50674 POINT(39.77721517064456 73.70178194799881) bank50674 +50675 POINT(39.799180536618856 74.77414725524827) bank50675 +50676 POINT(40.78598948952319 74.36287930858396) bank50676 +50677 POINT(40.969025336929626 74.52828936889163) bank50677 +50678 POINT(41.36363075279069 73.86752540209761) bank50678 +50679 POINT(41.58809646870937 74.58931619598037) bank50679 +50680 POINT(40.996107429908335 73.62144724446682) bank50680 +50681 POINT(40.42581324801031 75.00156703085567) bank50681 +50682 POINT(40.31055258240072 74.91933101029618) bank50682 +50683 POINT(39.75697122777214 73.47574218131746) bank50683 +50684 POINT(39.774672960812275 74.82294412064728) bank50684 +50685 POINT(41.50268592965369 73.10032785561603) bank50685 +50686 POINT(40.92307945310724 74.1623773783576) bank50686 +50687 POINT(40.83588710883155 73.75540937899646) bank50687 +50688 POINT(40.269806485963585 73.10605715042595) bank50688 +50689 POINT(40.48948053336994 73.4693317535031) bank50689 +50690 POINT(39.95280577664988 74.98678022853781) bank50690 +50691 POINT(41.39807692425323 74.6980453098666) bank50691 +50692 POINT(41.04040092075775 73.40574246320968) bank50692 +50693 POINT(40.490337559829726 73.65602370462076) bank50693 +50694 POINT(39.95920574524506 74.18536118929957) bank50694 +50695 POINT(40.10986246966204 74.80352954197674) bank50695 +50696 POINT(40.89261779537196 73.33418081539966) bank50696 +50697 POINT(39.96024192094454 74.44582032795286) bank50697 +50698 POINT(40.929823042830975 74.56745355821444) bank50698 +50699 POINT(41.017197865043656 73.15496650441432) bank50699 +50700 POINT(40.32655067681901 73.35448115265771) bank50700 +50701 POINT(41.672656922798105 73.16019731971869) bank50701 +50702 POINT(41.195760424863686 74.69720086115603) bank50702 +50703 POINT(40.18591791462395 74.22569906029726) bank50703 +50704 POINT(40.9391562026555 74.9240092164775) bank50704 +50705 POINT(41.01406611638849 73.29513418529864) bank50705 +50706 POINT(41.14648096320841 74.04386463302365) bank50706 +50707 POINT(40.770663444201844 73.45693060402942) bank50707 +50708 POINT(40.72532733197201 74.21504665081558) bank50708 +50709 POINT(41.62958602363196 74.34807053477968) bank50709 +50710 POINT(40.692860212159125 74.92451247037721) bank50710 +50711 POINT(41.369659869459056 74.82151083600719) bank50711 +50712 POINT(39.97998660718218 74.05839053581303) bank50712 +50713 POINT(41.485365905610216 73.50285126595604) bank50713 +50714 POINT(40.043557373111916 73.45237984990469) bank50714 +50715 POINT(40.04758937372288 74.32362682561518) bank50715 +50716 POINT(41.23733276339226 74.06287872398963) bank50716 +50717 POINT(40.51310674007345 73.94870645790564) bank50717 +50718 POINT(41.69048511215366 74.16601334356375) bank50718 +50719 POINT(41.567249249190084 73.95597564666903) bank50719 +50720 POINT(40.15327566091026 74.58449940298061) bank50720 +50721 POINT(40.79173836657616 73.7534878922243) bank50721 +50722 POINT(40.49593680915082 73.05318407754875) bank50722 +50723 POINT(40.145836605570636 73.35524395681705) bank50723 +50724 POINT(40.81942621857886 74.40972165332363) bank50724 +50725 POINT(40.02790994112978 73.63924736340725) bank50725 +50726 POINT(40.20882437756078 73.7830859930448) bank50726 +50727 POINT(40.01270195455561 74.08077879308425) bank50727 +50728 POINT(40.240281706562705 74.31884347404333) bank50728 +50729 POINT(41.12887448827918 73.62811930514326) bank50729 +50730 POINT(41.25143340001667 73.4839384628253) bank50730 +50731 POINT(40.554375939392926 74.87402172934502) bank50731 +50732 POINT(39.819974376383826 74.21646905434048) bank50732 +50733 POINT(40.20793179561831 73.49622002854137) bank50733 +50734 POINT(40.63115031042299 73.71204778732866) bank50734 +50735 POINT(41.33559923172125 73.29100370807684) bank50735 +50736 POINT(41.38966703403182 73.78560387236382) bank50736 +50737 POINT(40.30731813039879 74.2791252369023) bank50737 +50738 POINT(41.31774968030126 74.9403255373193) bank50738 +50739 POINT(41.23953442377522 74.68121366803578) bank50739 +50740 POINT(39.98521473458003 74.7056080740818) bank50740 +50741 POINT(41.50930565505785 74.47730683371141) bank50741 +50742 POINT(40.24720469642874 74.62587351324144) bank50742 +50743 POINT(40.65020613369098 73.04366812897864) bank50743 +50744 POINT(40.182447378834134 73.1645378977866) bank50744 +50745 POINT(39.918804630261626 73.81217106748886) bank50745 +50746 POINT(40.90487527013251 73.20843334121275) bank50746 +50747 POINT(40.91914217293401 74.04702052233911) bank50747 +50748 POINT(40.636964506574365 73.00605290217551) bank50748 +50749 POINT(41.12530477128031 73.78594376843606) bank50749 +50750 POINT(40.591394357625745 73.25080366426317) bank50750 +50751 POINT(40.16203935137307 74.29647734539493) bank50751 +50752 POINT(40.79194314866539 73.28631632032818) bank50752 +50753 POINT(41.219916985284854 73.7438398003194) bank50753 +50754 POINT(40.10617726620667 73.84856081225848) bank50754 +50755 POINT(41.441361036246725 73.01331084303118) bank50755 +50756 POINT(40.27878810277983 74.27792924064613) bank50756 +50757 POINT(41.061744155802685 74.20557996820158) bank50757 +50758 POINT(39.786557330987705 74.59954108735896) bank50758 +50759 POINT(41.48876525518623 74.92707867990494) bank50759 +50760 POINT(40.90898275192515 74.56099940076453) bank50760 +50761 POINT(39.97117705025973 73.2799387032913) bank50761 +50762 POINT(40.39487351252964 74.95256038169649) bank50762 +50763 POINT(41.16196916629067 73.61523787719739) bank50763 +50764 POINT(41.68663216636958 73.22036462603164) bank50764 +50765 POINT(41.173950046623354 74.02703671678458) bank50765 +50766 POINT(40.384991481498666 74.3262940621783) bank50766 +50767 POINT(41.55670718207346 74.37712163340201) bank50767 +50768 POINT(39.96396162126857 74.09279978875749) bank50768 +50769 POINT(40.32271839849492 74.73146624986829) bank50769 +50770 POINT(40.624353467418565 73.95222987845278) bank50770 +50771 POINT(40.710890768944225 74.82417755571655) bank50771 +50772 POINT(40.54498970837829 73.97472085011456) bank50772 +50773 POINT(41.022636924282565 73.71009112147085) bank50773 +50774 POINT(41.504728092957635 74.69509923369928) bank50774 +50775 POINT(41.12526737093749 74.36103173388166) bank50775 +50776 POINT(40.148301286819155 73.30810278621317) bank50776 +50777 POINT(40.516363245384184 73.49270639916976) bank50777 +50778 POINT(40.00502451454089 74.46034344765566) bank50778 +50779 POINT(40.123355387285734 74.93146997522103) bank50779 +50780 POINT(40.4489243911836 74.98119048633103) bank50780 +50781 POINT(40.56004903522949 73.8709884139078) bank50781 +50782 POINT(39.81418616615676 73.19763559222874) bank50782 +50783 POINT(40.23923351616244 74.57289973026603) bank50783 +50784 POINT(41.4320779596969 73.30248798336058) bank50784 +50785 POINT(40.131333690541105 73.82943338743439) bank50785 +50786 POINT(41.47315774930996 74.64493102492285) bank50786 +50787 POINT(40.72476757997304 73.17374845178118) bank50787 +50788 POINT(41.18993003984899 73.3266305929794) bank50788 +50789 POINT(39.74916796648584 74.37367780707697) bank50789 +50790 POINT(41.65252313326116 73.14671078734345) bank50790 +50791 POINT(39.889334953795185 73.46555006271431) bank50791 +50792 POINT(41.581208507438916 73.4537693882463) bank50792 +50793 POINT(40.321446420576585 74.30428527644409) bank50793 +50794 POINT(39.83382826089006 74.89296141181929) bank50794 +50795 POINT(41.03789426393848 74.42867775028117) bank50795 +50796 POINT(40.71254999231688 74.9847187815223) bank50796 +50797 POINT(40.97976804977622 73.89495981044685) bank50797 +50798 POINT(41.29846420734284 74.06525284402335) bank50798 +50799 POINT(41.0636574532897 73.30704669643748) bank50799 +50800 POINT(40.25945471464155 74.51173915964478) bank50800 +50801 POINT(40.31552731655562 74.3258553683184) bank50801 +50802 POINT(41.56902558384729 73.39786958930718) bank50802 +50803 POINT(41.33936795379546 74.77686543031493) bank50803 +50804 POINT(39.97919025278358 74.37929383880774) bank50804 +50805 POINT(41.154758142437295 74.33535015494411) bank50805 +50806 POINT(40.450831711348144 73.91092712621763) bank50806 +50807 POINT(41.17507295702725 74.3960744184079) bank50807 +50808 POINT(41.03818397407666 74.58985338292902) bank50808 +50809 POINT(40.64873608493154 74.30859729301457) bank50809 +50810 POINT(39.851473869664744 73.71718361077728) bank50810 +50811 POINT(41.42785689555203 74.72648926671167) bank50811 +50812 POINT(40.39282986904415 73.1740939192444) bank50812 +50813 POINT(40.944704336184486 73.96832180146004) bank50813 +50814 POINT(40.16176304804918 73.40051038591874) bank50814 +50815 POINT(39.755359714777654 74.63622956296439) bank50815 +50816 POINT(41.1251467802658 74.60161014557323) bank50816 +50817 POINT(41.23467233303407 74.04348769380844) bank50817 +50818 POINT(41.46032864959298 74.0268861665243) bank50818 +50819 POINT(40.50720691366561 74.03817472306928) bank50819 +50820 POINT(40.44820186982101 74.65188057851033) bank50820 +50821 POINT(40.47747654530828 74.07456437000204) bank50821 +50822 POINT(41.67717331289907 74.67291308049337) bank50822 +50823 POINT(40.58785807972678 73.40651126811305) bank50823 +50824 POINT(40.91581705191196 73.14929730104129) bank50824 +50825 POINT(41.589839973933124 74.83834512522184) bank50825 +50826 POINT(41.12865281700082 74.99180105992075) bank50826 +50827 POINT(39.754230267113186 74.65691238292885) bank50827 +50828 POINT(40.12417897107589 73.35316724480514) bank50828 +50829 POINT(40.461218254647676 74.89727895248468) bank50829 +50830 POINT(39.910134995842554 74.00871849284029) bank50830 +50831 POINT(40.418736820368586 74.05319310552402) bank50831 +50832 POINT(40.75741681501013 73.5428583931033) bank50832 +50833 POINT(41.13238410989938 73.00775153579202) bank50833 +50834 POINT(40.94430790354523 73.1948962160291) bank50834 +50835 POINT(40.24652425790763 74.2474266929431) bank50835 +50836 POINT(40.4903118892392 73.21796225017995) bank50836 +50837 POINT(40.39241328282248 74.36605829506946) bank50837 +50838 POINT(40.18034499623349 74.24496949576454) bank50838 +50839 POINT(40.20717617544795 73.77945861942379) bank50839 +50840 POINT(41.166870863304354 73.19286172369846) bank50840 +50841 POINT(40.77090262951583 74.30337486057047) bank50841 +50842 POINT(40.59592170022032 74.55941082298831) bank50842 +50843 POINT(39.81483703963224 73.2583694328672) bank50843 +50844 POINT(40.50618006196416 74.37907018780743) bank50844 +50845 POINT(41.68860176097619 74.00977201945108) bank50845 +50846 POINT(40.71881715065736 73.48072420321392) bank50846 +50847 POINT(40.19525992592146 73.28508590736946) bank50847 +50848 POINT(40.58207290545391 74.46914270542497) bank50848 +50849 POINT(41.25950519499892 73.77594493351121) bank50849 +50850 POINT(40.23953640870828 73.25227975172304) bank50850 +50851 POINT(39.95525013733387 74.7650581608104) bank50851 +50852 POINT(41.18636559179675 74.65284715260711) bank50852 +50853 POINT(40.25715913931062 74.69417058684131) bank50853 +50854 POINT(40.1445451714215 73.1489731175742) bank50854 +50855 POINT(41.41773887277655 73.23300321828471) bank50855 +50856 POINT(40.39750544105774 74.99982975181607) bank50856 +50857 POINT(40.36028260420135 74.47237365070981) bank50857 +50858 POINT(41.172835605854004 74.79884958601784) bank50858 +50859 POINT(41.40575127666199 74.23494762418832) bank50859 +50860 POINT(41.01753841024929 74.30874080850948) bank50860 +50861 POINT(40.99099844217835 73.84525156246787) bank50861 +50862 POINT(39.82822909460203 74.79151215655536) bank50862 +50863 POINT(40.57780222090529 73.21409428762321) bank50863 +50864 POINT(41.072078283673754 73.5377100848752) bank50864 +50865 POINT(40.99065182086024 73.80009875992266) bank50865 +50866 POINT(41.15630456740784 74.7664721643441) bank50866 +50867 POINT(40.090437482010074 73.28362851669671) bank50867 +50868 POINT(41.539762878713105 73.59622702044679) bank50868 +50869 POINT(41.37151787505897 73.80759987696138) bank50869 +50870 POINT(40.74193846045786 74.43036730736642) bank50870 +50871 POINT(41.485110094048935 74.831601311012) bank50871 +50872 POINT(41.25980324006686 74.89072312241493) bank50872 +50873 POINT(39.714866069656225 74.68895776357623) bank50873 +50874 POINT(40.07058420438827 73.57161277579854) bank50874 +50875 POINT(41.002297091660154 73.91335061676658) bank50875 +50876 POINT(39.94209444180103 73.16289105445237) bank50876 +50877 POINT(40.12811662765208 73.14558661907027) bank50877 +50878 POINT(41.1698675099249 73.46139718585931) bank50878 +50879 POINT(41.290227044015175 74.67058433089889) bank50879 +50880 POINT(40.118074992404125 74.93290402093967) bank50880 +50881 POINT(39.93731562432276 73.12936981360143) bank50881 +50882 POINT(41.55842892343224 73.13397038545685) bank50882 +50883 POINT(40.31110905241073 74.88267188908624) bank50883 +50884 POINT(41.2845628942257 73.62189901926506) bank50884 +50885 POINT(41.45740739534704 73.81014794630322) bank50885 +50886 POINT(40.362465488868665 73.75532428751713) bank50886 +50887 POINT(41.03937740745066 74.22102075462648) bank50887 +50888 POINT(39.808779988207085 73.69854585767537) bank50888 +50889 POINT(41.6403422336839 73.11835864105018) bank50889 +50890 POINT(40.16319809910518 73.52351193779741) bank50890 +50891 POINT(41.37591165376484 73.13346657700392) bank50891 +50892 POINT(40.14228206446834 74.86505153822252) bank50892 +50893 POINT(41.02402246365217 73.35626084726887) bank50893 +50894 POINT(40.27024000776777 73.25777816356168) bank50894 +50895 POINT(40.06908386363724 74.0285599329455) bank50895 +50896 POINT(40.78929394660628 73.66092177997565) bank50896 +50897 POINT(41.662813673481075 73.019567425801) bank50897 +50898 POINT(41.63059983577281 74.92050144231567) bank50898 +50899 POINT(39.88631192210753 74.64213448042523) bank50899 +50900 POINT(41.19243933944095 73.2856129542588) bank50900 +50901 POINT(41.192179549933975 74.42900128771144) bank50901 +50902 POINT(40.748285360343615 74.43884576871547) bank50902 +50903 POINT(40.3120514364993 73.9822728385195) bank50903 +50904 POINT(39.74641740456115 74.07016297792597) bank50904 +50905 POINT(40.66117555824164 74.82799189147559) bank50905 +50906 POINT(41.373916205944965 74.67167337486313) bank50906 +50907 POINT(41.42835012347478 74.14311046349567) bank50907 +50908 POINT(41.484917511516414 74.98613838349222) bank50908 +50909 POINT(40.50048131908633 73.19345144284117) bank50909 +50910 POINT(40.847727301638606 74.95124472833125) bank50910 +50911 POINT(40.11398456667977 73.31506136641384) bank50911 +50912 POINT(41.344533983728354 74.00729670773718) bank50912 +50913 POINT(41.3752611600003 73.64434016419112) bank50913 +50914 POINT(41.40036754552732 73.68281109435647) bank50914 +50915 POINT(41.58119034321727 74.15456808525455) bank50915 +50916 POINT(40.220878862880326 74.98574951400606) bank50916 +50917 POINT(40.57472438736753 74.75344611194794) bank50917 +50918 POINT(39.771984800365686 73.04154754888465) bank50918 +50919 POINT(39.80815182949222 74.19825243553426) bank50919 +50920 POINT(39.80734333881285 74.18051322226368) bank50920 +50921 POINT(41.47446429504576 73.9614989143562) bank50921 +50922 POINT(40.31707105262697 73.07737564589807) bank50922 +50923 POINT(40.65097757624074 73.79568920320601) bank50923 +50924 POINT(40.61904917997 73.73609641694586) bank50924 +50925 POINT(40.34755705481426 74.54031648677012) bank50925 +50926 POINT(40.26588841689713 74.01098612284288) bank50926 +50927 POINT(40.932262328745445 74.72433618606158) bank50927 +50928 POINT(41.29268863887508 73.40387935832106) bank50928 +50929 POINT(40.37527892665836 74.50149536521184) bank50929 +50930 POINT(41.33765400228717 74.39564810044085) bank50930 +50931 POINT(39.804526931273195 74.73735213851569) bank50931 +50932 POINT(41.3691966085454 73.94425666463842) bank50932 +50933 POINT(41.23851763575547 74.01519114468039) bank50933 +50934 POINT(40.48055281251063 74.02022029389454) bank50934 +50935 POINT(40.790934832910295 74.51256686156749) bank50935 +50936 POINT(40.654018689303655 74.76739261990738) bank50936 +50937 POINT(40.22303675472637 73.88229125284309) bank50937 +50938 POINT(41.288019385415694 74.72117911523766) bank50938 +50939 POINT(41.63207437093384 73.95163102339097) bank50939 +50940 POINT(41.125506525738864 74.76387749667347) bank50940 +50941 POINT(41.02823593797673 74.83366210296464) bank50941 +50942 POINT(41.29283635360818 74.11593925429227) bank50942 +50943 POINT(41.19722195322054 73.89986382032816) bank50943 +50944 POINT(41.52074386806409 73.92881435483083) bank50944 +50945 POINT(41.30192213699405 73.97499904539879) bank50945 +50946 POINT(40.8933037699978 73.035733532895) bank50946 +50947 POINT(41.63352942557712 73.02179922057438) bank50947 +50948 POINT(40.44424391435014 74.21564383891689) bank50948 +50949 POINT(41.02503107544453 74.57998528517577) bank50949 +50950 POINT(40.08845965614864 74.99892899788412) bank50950 +50951 POINT(39.89442120015211 74.92227233759483) bank50951 +50952 POINT(41.20196142417596 74.60337674571164) bank50952 +50953 POINT(40.259603712435464 73.30099574784249) bank50953 +50954 POINT(41.108957850849656 74.03646983169496) bank50954 +50955 POINT(40.125488766227 73.06468618357444) bank50955 +50956 POINT(40.930987033476924 74.84801303327771) bank50956 +50957 POINT(39.81239584434962 74.09340704291601) bank50957 +50958 POINT(39.78166829624493 74.59619029303015) bank50958 +50959 POINT(41.17824452650437 73.15662761519395) bank50959 +50960 POINT(40.31274247532942 73.63478771668807) bank50960 +50961 POINT(41.003802245641076 73.76912476722222) bank50961 +50962 POINT(40.205199439956765 73.436136369681) bank50962 +50963 POINT(40.038493645878006 73.43866040208552) bank50963 +50964 POINT(40.756461773341556 74.92948302488983) bank50964 +50965 POINT(41.47663861766871 74.98077772642932) bank50965 +50966 POINT(40.914617470747864 74.008382655519) bank50966 +50967 POINT(39.72626713824025 73.06904892269479) bank50967 +50968 POINT(41.350830242217086 75.00126813355617) bank50968 +50969 POINT(40.77640225366593 73.75355402232586) bank50969 +50970 POINT(41.1330616919998 73.03883990130718) bank50970 +50971 POINT(40.069563865745515 73.85807432575231) bank50971 +50972 POINT(39.84315279137731 73.63936624108885) bank50972 +50973 POINT(40.489862079644176 73.76014655867087) bank50973 +50974 POINT(39.99997167120662 74.73235539861868) bank50974 +50975 POINT(39.73183322123894 73.17758739787872) bank50975 +50976 POINT(41.5747326088541 73.56643673261115) bank50976 +50977 POINT(40.6668699044529 73.514460427511) bank50977 +50978 POINT(41.51184832006713 73.8364727441702) bank50978 +50979 POINT(41.14918514346726 74.67318937746923) bank50979 +50980 POINT(40.3833207741212 74.33702204384855) bank50980 +50981 POINT(39.74952206072434 74.46424383822189) bank50981 +50982 POINT(41.422588584941266 74.55632912726618) bank50982 +50983 POINT(41.38123543877119 73.98751180938177) bank50983 +50984 POINT(41.04296821950321 73.15536010038622) bank50984 +50985 POINT(39.719767299339885 73.64608160586226) bank50985 +50986 POINT(39.90828513641916 73.26415748384832) bank50986 +50987 POINT(41.48325602901347 74.77805596787346) bank50987 +50988 POINT(41.346712214914575 73.95544466425022) bank50988 +50989 POINT(40.70660834391747 73.40450019711697) bank50989 +50990 POINT(40.858352809104666 73.62461155207319) bank50990 +50991 POINT(40.5609410199589 73.72531626368917) bank50991 +50992 POINT(40.498711860514994 73.07664343084264) bank50992 +50993 POINT(41.06716827785239 73.49326150942356) bank50993 +50994 POINT(41.000505210065946 73.43571640756511) bank50994 +50995 POINT(40.582626478654845 73.10344050542797) bank50995 +50996 POINT(41.1003378799937 73.52341956489923) bank50996 +50997 POINT(40.20439515741661 74.48585420509069) bank50997 +50998 POINT(40.817845406523816 74.7897706739042) bank50998 +50999 POINT(40.24327091489599 74.68801114170901) bank50999 +51000 POINT(40.75148945458082 73.30987730264138) bank51000 +51001 POINT(40.383175925659174 73.84612586502053) bank51001 +51002 POINT(40.55049520941836 74.2841268932672) bank51002 +51003 POINT(39.77860269727715 73.53248222013538) bank51003 +51004 POINT(41.454705255067346 73.43449588175137) bank51004 +51005 POINT(41.66923080396152 73.60596301128815) bank51005 +51006 POINT(41.684103508546954 73.06410473638397) bank51006 +51007 POINT(40.65722194749117 73.64908791521808) bank51007 +51008 POINT(41.11161374443286 73.43778205362166) bank51008 +51009 POINT(41.50175358564244 73.39250178962384) bank51009 +51010 POINT(41.04003229562664 74.70819040122986) bank51010 +51011 POINT(39.866822460436055 73.44322835133855) bank51011 +51012 POINT(41.68088386385046 73.49500923165763) bank51012 +51013 POINT(41.14672722080203 73.11804739731252) bank51013 +51014 POINT(40.84786298479151 74.25529993307411) bank51014 +51015 POINT(39.8886884783693 74.97659759434215) bank51015 +51016 POINT(41.50953447606206 74.60472684244318) bank51016 +51017 POINT(41.08340445777647 74.83136575259276) bank51017 +51018 POINT(40.62310716547811 73.36742107414014) bank51018 +51019 POINT(40.82621248556335 73.08727852848672) bank51019 +51020 POINT(40.93770022872416 74.34119605256288) bank51020 +51021 POINT(40.651508227650076 74.0519158862474) bank51021 +51022 POINT(40.76232436754166 74.38962783448815) bank51022 +51023 POINT(41.48933938137665 73.88868126081542) bank51023 +51024 POINT(41.43016622342857 74.16131634293208) bank51024 +51025 POINT(40.45810412336357 74.04801281186892) bank51025 +51026 POINT(40.2713377516394 73.77155469052973) bank51026 +51027 POINT(40.57096465845437 74.15102771221338) bank51027 +51028 POINT(40.517893001671595 73.98129763877994) bank51028 +51029 POINT(40.9771950030408 74.86965283935315) bank51029 +51030 POINT(40.147050122905384 74.72772567496752) bank51030 +51031 POINT(41.059470967850594 74.65358385602559) bank51031 +51032 POINT(41.10876493846783 73.24903624905816) bank51032 +51033 POINT(41.70888776495351 74.37723841340515) bank51033 +51034 POINT(41.683737642458226 74.3338136215467) bank51034 +51035 POINT(40.97485171352797 73.48927543214582) bank51035 +51036 POINT(40.09964880823199 73.87126499454496) bank51036 +51037 POINT(40.785336597169035 73.46261235501657) bank51037 +51038 POINT(41.59151810944359 73.5905912765959) bank51038 +51039 POINT(41.28074444130937 73.17603622153892) bank51039 +51040 POINT(40.72322064300005 74.71347285379782) bank51040 +51041 POINT(40.914849416516454 74.24575010375106) bank51041 +51042 POINT(40.0566929273475 74.94132820706865) bank51042 +51043 POINT(40.25328428631598 73.74001612194346) bank51043 +51044 POINT(41.32252754917508 74.98966730489316) bank51044 +51045 POINT(41.54510696089417 74.33936585779425) bank51045 +51046 POINT(40.267624610030865 74.48803404093901) bank51046 +51047 POINT(40.812512530290746 74.6277388524465) bank51047 +51048 POINT(40.4117539218668 73.40828104622639) bank51048 +51049 POINT(39.76495413350497 74.25296445655015) bank51049 +51050 POINT(40.58068957624375 74.27576212339945) bank51050 +51051 POINT(41.69102492228272 73.31903345867225) bank51051 +51052 POINT(40.29797970938717 73.84225056982393) bank51052 +51053 POINT(40.26593260941561 73.99833206659879) bank51053 +51054 POINT(40.90689792885847 74.66367673539379) bank51054 +51055 POINT(40.862405159820284 73.84220691006504) bank51055 +51056 POINT(41.273424201537104 73.7446979366022) bank51056 +51057 POINT(39.84927007404258 73.94699322757738) bank51057 +51058 POINT(41.32723810318368 73.9265169143789) bank51058 +51059 POINT(41.62957968307511 73.95777620509391) bank51059 +51060 POINT(41.58479724327188 73.85416433936864) bank51060 +51061 POINT(40.679702707018535 73.73466343410475) bank51061 +51062 POINT(41.41103944628901 74.25925819333021) bank51062 +51063 POINT(41.518398383609124 74.90898187837973) bank51063 +51064 POINT(40.96845604856038 73.30339597094729) bank51064 +51065 POINT(40.36351683572912 73.41218652751752) bank51065 +51066 POINT(39.75137825221926 73.73185024298327) bank51066 +51067 POINT(40.59766547323661 74.61645138157064) bank51067 +51068 POINT(41.28624142969022 73.56210364955457) bank51068 +51069 POINT(41.430324550507756 73.10885528453274) bank51069 +51070 POINT(41.35911240808805 74.7308933233717) bank51070 +51071 POINT(40.025680113931365 73.69704878560592) bank51071 +51072 POINT(40.232155181178506 73.68268994594226) bank51072 +51073 POINT(40.58227146484733 74.56622011135067) bank51073 +51074 POINT(41.44021315610431 74.74383360327634) bank51074 +51075 POINT(41.601473060992724 74.37088208799992) bank51075 +51076 POINT(41.66653816896912 73.36377854680619) bank51076 +51077 POINT(40.46804525697348 74.24341790746897) bank51077 +51078 POINT(41.64785006007144 74.9836144580349) bank51078 +51079 POINT(40.39253417478648 73.66978891606857) bank51079 +51080 POINT(40.93234586877083 74.23957701219885) bank51080 +51081 POINT(40.775927318448396 73.67155681288646) bank51081 +51082 POINT(41.421506591540535 73.82317442944574) bank51082 +51083 POINT(40.213052377577576 74.56807918805075) bank51083 +51084 POINT(41.26686586094051 74.63812337728393) bank51084 +51085 POINT(40.950099977618976 73.48785142496958) bank51085 +51086 POINT(40.59146152299292 73.77285723952002) bank51086 +51087 POINT(40.92781399920486 73.64236404662128) bank51087 +51088 POINT(41.42868744835001 73.90423631954485) bank51088 +51089 POINT(40.24541683085214 73.53587692637602) bank51089 +51090 POINT(39.93052185207179 74.33160177554599) bank51090 +51091 POINT(39.815202098474074 74.48538990135977) bank51091 +51092 POINT(40.81421155958683 73.30326883522562) bank51092 +51093 POINT(40.26831899612479 74.09297224056864) bank51093 +51094 POINT(40.87129605187859 74.9265750815074) bank51094 +51095 POINT(41.23496271144458 73.60158676928157) bank51095 +51096 POINT(40.12520300695741 74.4721465801568) bank51096 +51097 POINT(39.864265530514785 74.35635380196705) bank51097 +51098 POINT(41.51924390625047 73.49788517841132) bank51098 +51099 POINT(39.9207181948585 74.52677177652077) bank51099 +51100 POINT(40.08719144430998 74.51256602029122) bank51100 +51101 POINT(40.22261346568318 73.2996936758675) bank51101 +51102 POINT(39.90552789081639 73.65362669558586) bank51102 +51103 POINT(40.33296819124661 74.73787209105296) bank51103 +51104 POINT(41.68627851144842 73.85093646585237) bank51104 +51105 POINT(40.78420321515879 73.60664364096216) bank51105 +51106 POINT(40.363501634232556 74.24938709305002) bank51106 +51107 POINT(40.58004243527997 73.01129769251943) bank51107 +51108 POINT(40.69922830127436 73.08836415088145) bank51108 +51109 POINT(41.52498036498545 74.53988591166066) bank51109 +51110 POINT(41.35098724885933 74.91323572858158) bank51110 +51111 POINT(41.70887728758055 74.95307842260414) bank51111 +51112 POINT(41.70603392979869 74.83685063604379) bank51112 +51113 POINT(39.89539502820565 74.99480511017293) bank51113 +51114 POINT(41.526441246295576 73.37198787848347) bank51114 +51115 POINT(40.44684761258023 73.80664515432008) bank51115 +51116 POINT(40.37815494788302 73.68018455923651) bank51116 +51117 POINT(40.8426353668097 73.05591973534517) bank51117 +51118 POINT(41.429457815666275 75.00554686164055) bank51118 +51119 POINT(39.76756339694434 74.86662358724787) bank51119 +51120 POINT(41.05354248957753 73.20311288016313) bank51120 +51121 POINT(40.46286971296247 74.2390882817695) bank51121 +51122 POINT(40.71685265088254 73.03331022557606) bank51122 +51123 POINT(40.71861012552007 73.97507836657797) bank51123 +51124 POINT(40.50726648135634 74.78110020420058) bank51124 +51125 POINT(40.19986406671106 73.29842450495708) bank51125 +51126 POINT(41.48485542427978 74.08960482085457) bank51126 +51127 POINT(40.90511360989446 73.64462395740586) bank51127 +51128 POINT(40.327144016732525 74.45094200450411) bank51128 +51129 POINT(41.0260186651363 74.07349704250969) bank51129 +51130 POINT(41.569643813349614 73.43502460939713) bank51130 +51131 POINT(41.37488147215318 73.84445070599764) bank51131 +51132 POINT(41.33302214996023 73.02483840526618) bank51132 +51133 POINT(41.51324169597987 73.32997490644615) bank51133 +51134 POINT(41.50443840043546 74.96391692766008) bank51134 +51135 POINT(39.91150217844823 73.99159507387849) bank51135 +51136 POINT(40.83128785232716 73.87832201599709) bank51136 +51137 POINT(40.70883173010641 73.12616817914414) bank51137 +51138 POINT(41.14495487509845 73.6862172750802) bank51138 +51139 POINT(41.67855852935623 74.7253992694734) bank51139 +51140 POINT(39.74544181770228 73.92517522275732) bank51140 +51141 POINT(41.409862703104565 73.42696863210664) bank51141 +51142 POINT(40.360468267604666 73.95957586039808) bank51142 +51143 POINT(41.116621010650306 73.23665219209843) bank51143 +51144 POINT(41.36185451019729 74.22186837337162) bank51144 +51145 POINT(41.44602674682646 73.46181118924677) bank51145 +51146 POINT(40.28556749724238 73.56920580831768) bank51146 +51147 POINT(39.793299296808 74.32416924577797) bank51147 +51148 POINT(41.284927675651204 74.9018974165593) bank51148 +51149 POINT(39.843300024506924 74.29869751640811) bank51149 +51150 POINT(39.981213845196585 74.30130473079953) bank51150 +51151 POINT(40.32414915982672 74.50363923686962) bank51151 +51152 POINT(40.10354647659728 74.36018959204728) bank51152 +51153 POINT(40.03411739127158 73.96587817287835) bank51153 +51154 POINT(40.02769563382081 74.66158385139519) bank51154 +51155 POINT(41.37063172878845 74.39566563896483) bank51155 +51156 POINT(39.850870696720435 74.91216357301819) bank51156 +51157 POINT(40.4486646660335 73.89652475641066) bank51157 +51158 POINT(41.47366950711632 73.70419367139169) bank51158 +51159 POINT(40.74335968212594 73.33710742889888) bank51159 +51160 POINT(41.25758011320942 73.11583428339262) bank51160 +51161 POINT(41.708925581664126 74.23306225223172) bank51161 +51162 POINT(41.15226265047429 73.30322929962254) bank51162 +51163 POINT(40.19344701942329 73.06950567125737) bank51163 +51164 POINT(40.27335532331919 74.39981710216978) bank51164 +51165 POINT(40.266941787129475 74.00092359135056) bank51165 +51166 POINT(40.33699777533368 73.22727821702782) bank51166 +51167 POINT(40.90654392026926 73.98724085071191) bank51167 +51168 POINT(39.720453778392134 73.80026605449207) bank51168 +51169 POINT(40.438295778201635 74.20274778418938) bank51169 +51170 POINT(40.900880366847375 74.94107495938103) bank51170 +51171 POINT(41.44776307939179 74.97157673231774) bank51171 +51172 POINT(40.22191612861198 73.46971288313264) bank51172 +51173 POINT(40.03526943033738 74.4753849808004) bank51173 +51174 POINT(41.66053558859509 73.22530379657606) bank51174 +51175 POINT(40.91204239249758 74.2578254919408) bank51175 +51176 POINT(39.83477083548314 73.50369887655222) bank51176 +51177 POINT(40.709383895596865 73.51983522588125) bank51177 +51178 POINT(40.41575063020848 73.80857173558975) bank51178 +51179 POINT(40.83786435423862 74.26710530070397) bank51179 +51180 POINT(41.31147847472175 74.67110010405897) bank51180 +51181 POINT(40.259965240794415 74.9030081350427) bank51181 +51182 POINT(40.21744228141757 73.28580027148489) bank51182 +51183 POINT(39.75196444417782 73.45815444582645) bank51183 +51184 POINT(41.15103070684474 73.00955668355266) bank51184 +51185 POINT(41.331597133098995 74.24992476366556) bank51185 +51186 POINT(40.449209486856596 74.37216172771222) bank51186 +51187 POINT(39.906690818300795 73.74127744235514) bank51187 +51188 POINT(41.32377316008652 74.19222629390744) bank51188 +51189 POINT(40.054120669703686 73.33049804292608) bank51189 +51190 POINT(39.991277929299805 73.04702071286877) bank51190 +51191 POINT(40.60785226540871 74.30258434936609) bank51191 +51192 POINT(40.41567878524959 74.2941723136162) bank51192 +51193 POINT(41.39688362741455 74.80830610383188) bank51193 +51194 POINT(40.46470566973564 73.05703286628439) bank51194 +51195 POINT(41.646850839980665 74.59287924004826) bank51195 +51196 POINT(39.734385825521564 74.04047679607173) bank51196 +51197 POINT(40.743292331584975 73.16452496427794) bank51197 +51198 POINT(40.1339793229103 74.66785989972702) bank51198 +51199 POINT(39.92482822676338 74.33907776191973) bank51199 +51200 POINT(40.664794589490434 74.23343751666675) bank51200 +51201 POINT(41.708981608214046 73.24409813873822) bank51201 +51202 POINT(40.54326853935092 73.14313515957211) bank51202 +51203 POINT(41.317363544014476 73.2810555820238) bank51203 +51204 POINT(40.57579842823429 74.15493302388388) bank51204 +51205 POINT(41.10099708621287 73.0692299601144) bank51205 +51206 POINT(41.284389064100424 74.72646306073602) bank51206 +51207 POINT(39.725007106525226 73.02969942861483) bank51207 +51208 POINT(41.60628302777838 74.83562774636256) bank51208 +51209 POINT(40.86734555723408 73.48942161055771) bank51209 +51210 POINT(40.85837511186067 73.68806300177145) bank51210 +51211 POINT(40.81100833192617 74.74425660368082) bank51211 +51212 POINT(40.81075035341397 74.28814083872578) bank51212 +51213 POINT(40.17516926592939 74.36186863130004) bank51213 +51214 POINT(41.52776072377167 73.01037312177982) bank51214 +51215 POINT(39.721923529864426 74.71608552020803) bank51215 +51216 POINT(41.55741527971865 73.09065242511137) bank51216 +51217 POINT(40.690286084662 73.73138834425009) bank51217 +51218 POINT(40.818450256604876 74.3460542294163) bank51218 +51219 POINT(41.63133877985417 74.77919179884597) bank51219 +51220 POINT(41.36162693187018 73.87671844744786) bank51220 +51221 POINT(40.123032608016295 74.65434504943747) bank51221 +51222 POINT(40.083691180779795 74.43285028098587) bank51222 +51223 POINT(41.61807545641508 73.45542759358914) bank51223 +51224 POINT(41.38588408344196 73.03740873469465) bank51224 +51225 POINT(40.576980338386406 74.54105787862666) bank51225 +51226 POINT(40.79153445214993 74.63244140074235) bank51226 +51227 POINT(40.27350020000807 73.58168033525813) bank51227 +51228 POINT(41.001687524536514 73.30364314139133) bank51228 +51229 POINT(40.66144145452488 73.29524625810788) bank51229 +51230 POINT(39.730402711182265 74.41283417884087) bank51230 +51231 POINT(39.992331665933435 73.6750256921789) bank51231 +51232 POINT(40.65390328720146 74.53909793884557) bank51232 +51233 POINT(41.270806412012554 74.01202728818343) bank51233 +51234 POINT(39.79835750125469 74.74645950915561) bank51234 +51235 POINT(41.39102658087468 73.63159661931469) bank51235 +51236 POINT(40.11783012435531 74.05765201375141) bank51236 +51237 POINT(41.31838288810955 73.97591629416459) bank51237 +51238 POINT(41.25553110687568 73.25640549645372) bank51238 +51239 POINT(41.285668487401765 73.87648065796996) bank51239 +51240 POINT(40.20020515053554 73.79836204878036) bank51240 +51241 POINT(40.852555520248956 73.14556562986441) bank51241 +51242 POINT(41.33943255274789 74.8409071000424) bank51242 +51243 POINT(39.89139357098033 74.25118526649146) bank51243 +51244 POINT(40.586953080571206 73.71438488483425) bank51244 +51245 POINT(39.99257222960445 73.85040592028318) bank51245 +51246 POINT(40.88797241622973 74.6534079178599) bank51246 +51247 POINT(41.62127745315872 74.40462836082799) bank51247 +51248 POINT(40.1363985254265 74.79408485579822) bank51248 +51249 POINT(41.45469359563393 73.71839514645211) bank51249 +51250 POINT(41.208342278603936 74.58335252668171) bank51250 +51251 POINT(41.058183285713106 74.16910583096262) bank51251 +51252 POINT(39.919649473452964 74.94859323448642) bank51252 +51253 POINT(39.976342355966146 74.09153523947764) bank51253 +51254 POINT(40.660860029299705 74.44661715032233) bank51254 +51255 POINT(40.94258025550482 73.64334411092482) bank51255 +51256 POINT(41.054422507399856 74.4313431643896) bank51256 +51257 POINT(40.43833388503465 73.6589892789241) bank51257 +51258 POINT(41.521386867159336 73.13351521787638) bank51258 +51259 POINT(40.52806684425991 74.43284560948081) bank51259 +51260 POINT(40.04773361947608 73.04442293364686) bank51260 +51261 POINT(41.03388648323776 73.67567546895073) bank51261 +51262 POINT(40.28235634073546 73.74167673843961) bank51262 +51263 POINT(40.351675777458965 73.02874592212758) bank51263 +51264 POINT(39.77873336742233 74.81836940674782) bank51264 +51265 POINT(39.935471539634854 74.1923589572263) bank51265 +51266 POINT(40.5375746550334 74.50192474982124) bank51266 +51267 POINT(39.98570403402155 74.26457615946627) bank51267 +51268 POINT(39.81288489729953 74.22213795639462) bank51268 +51269 POINT(40.81306137003977 74.82778021646926) bank51269 +51270 POINT(40.38924674050442 73.4488296404983) bank51270 +51271 POINT(40.000834612425045 74.6217228265697) bank51271 +51272 POINT(40.00703155076076 74.9225599427468) bank51272 +51273 POINT(40.825624441466076 74.71845202715788) bank51273 +51274 POINT(41.220059215093094 74.45023710189595) bank51274 +51275 POINT(40.02934604570477 73.86166938610205) bank51275 +51276 POINT(40.75570952410624 73.11904552724981) bank51276 +51277 POINT(40.100902534895525 73.19631904154122) bank51277 +51278 POINT(41.041514805097975 73.98712029729799) bank51278 +51279 POINT(41.196172268667816 73.98086953331287) bank51279 +51280 POINT(41.188655477343985 73.0489457484182) bank51280 +51281 POINT(40.124938984162455 73.43678782298605) bank51281 +51282 POINT(41.694128623799266 73.4375595017475) bank51282 +51283 POINT(41.010968684018295 73.15162831612595) bank51283 +51284 POINT(41.09054180847759 74.25863848252949) bank51284 +51285 POINT(41.678970007098556 73.15787710944707) bank51285 +51286 POINT(41.35162515421146 73.97503344575262) bank51286 +51287 POINT(40.78184323156555 74.3624314982403) bank51287 +51288 POINT(40.51181568176965 73.23622515340688) bank51288 +51289 POINT(39.73392412858779 73.94244652700698) bank51289 +51290 POINT(41.572209094836644 73.9410611747629) bank51290 +51291 POINT(40.35676227725573 73.16433905141811) bank51291 +51292 POINT(40.70964216062786 74.49687063513213) bank51292 +51293 POINT(40.67599446960246 73.93662933693362) bank51293 +51294 POINT(41.484461091715744 73.76896256865399) bank51294 +51295 POINT(40.8411872979899 74.53069764658554) bank51295 +51296 POINT(39.94564606882598 74.24195967547392) bank51296 +51297 POINT(40.560615218773286 74.63922639004339) bank51297 +51298 POINT(40.795661534150675 74.39635753920882) bank51298 +51299 POINT(40.39886328466367 74.05255508172272) bank51299 +51300 POINT(40.33689440221232 73.41317744605972) bank51300 +51301 POINT(40.905242837166504 73.7769803396953) bank51301 +51302 POINT(40.88030987722453 73.93168979958612) bank51302 +51303 POINT(39.910182744483365 73.71523030993292) bank51303 +51304 POINT(40.39449505167204 73.25104067755245) bank51304 +51305 POINT(39.82758815845269 74.39620357260829) bank51305 +51306 POINT(41.45335894182531 73.13083131931347) bank51306 +51307 POINT(40.38354449924274 73.5196959492741) bank51307 +51308 POINT(40.34988576923329 74.2793111705591) bank51308 +51309 POINT(41.49879333598781 73.6607201145703) bank51309 +51310 POINT(41.66718581601922 73.66824154482858) bank51310 +51311 POINT(40.601931441652674 74.70708715943056) bank51311 +51312 POINT(39.92346015571431 74.79085623166256) bank51312 +51313 POINT(40.95774947891125 73.84231477077084) bank51313 +51314 POINT(39.80368632664558 74.50775896071008) bank51314 +51315 POINT(41.645490896282716 73.37256904383167) bank51315 +51316 POINT(40.390222800245496 74.45022996036528) bank51316 +51317 POINT(41.66833661572688 74.71208334243758) bank51317 +51318 POINT(41.576341173835864 73.72154967381422) bank51318 +51319 POINT(40.0415595438321 74.33214356386671) bank51319 +51320 POINT(40.60008744187474 74.83373218750252) bank51320 +51321 POINT(41.0800352394175 73.22083498166687) bank51321 +51322 POINT(40.74434492422218 74.56927670023569) bank51322 +51323 POINT(40.48569028263669 73.02241849112798) bank51323 +51324 POINT(41.08950314078101 74.83143175864139) bank51324 +51325 POINT(41.233331783528264 74.34496961437716) bank51325 +51326 POINT(41.038001519526745 73.14928306976736) bank51326 +51327 POINT(39.73429408317579 74.37034917216266) bank51327 +51328 POINT(40.45375271604936 73.82267477776213) bank51328 +51329 POINT(41.657706197459255 73.53624366188384) bank51329 +51330 POINT(39.85633722643011 74.50605751358533) bank51330 +51331 POINT(41.51178037559985 73.3413627521424) bank51331 +51332 POINT(41.422502639824 73.66843573710243) bank51332 +51333 POINT(40.91595109676586 74.7826921480368) bank51333 +51334 POINT(41.32069453780643 73.98009683728891) bank51334 +51335 POINT(40.5524057206756 73.25554018053666) bank51335 +51336 POINT(40.67840372363204 74.88915197121385) bank51336 +51337 POINT(40.42382846736904 73.59451269458344) bank51337 +51338 POINT(40.30977038418721 74.93169275430394) bank51338 +51339 POINT(41.64141395779017 74.16196509903477) bank51339 +51340 POINT(41.52740046324966 74.25536236833723) bank51340 +51341 POINT(40.0854771379613 74.40100061897878) bank51341 +51342 POINT(41.217572840615794 73.86717977379482) bank51342 +51343 POINT(39.87826980128709 74.12152444907606) bank51343 +51344 POINT(40.45016519017673 74.56525547165374) bank51344 +51345 POINT(41.509503973249934 73.29989055048625) bank51345 +51346 POINT(39.762826991584234 73.7705211768607) bank51346 +51347 POINT(40.60296737367774 73.06518970066853) bank51347 +51348 POINT(39.85546525244914 74.03004163861766) bank51348 +51349 POINT(41.6616962142643 74.94171781862974) bank51349 +51350 POINT(39.94661108247992 74.31721527405436) bank51350 +51351 POINT(41.5536939395038 74.03482126559734) bank51351 +51352 POINT(41.044548202892656 73.41349696916161) bank51352 +51353 POINT(40.44761438108537 74.5638251037077) bank51353 +51354 POINT(40.45991440345765 74.601273514932) bank51354 +51355 POINT(41.62379205918435 73.4933562118312) bank51355 +51356 POINT(40.41318372432701 74.45510762844116) bank51356 +51357 POINT(41.00507556247877 74.30367871863714) bank51357 +51358 POINT(41.705831755919654 74.77171346050389) bank51358 +51359 POINT(40.78582880545263 73.66630792967975) bank51359 +51360 POINT(41.649548818513296 73.7923919293951) bank51360 +51361 POINT(39.87806198144089 74.53253059840449) bank51361 +51362 POINT(39.788925298011165 74.48398188799939) bank51362 +51363 POINT(41.681250476399214 74.15429722863873) bank51363 +51364 POINT(41.17246722659772 73.14200018524969) bank51364 +51365 POINT(40.052017191474235 74.40950046514062) bank51365 +51366 POINT(41.39149635142645 73.56817352223062) bank51366 +51367 POINT(40.87183143263158 73.62775587239591) bank51367 +51368 POINT(39.93948997654165 74.7237915002727) bank51368 +51369 POINT(40.33765366120788 73.2176858665593) bank51369 +51370 POINT(41.65908044264437 73.34597373548823) bank51370 +51371 POINT(39.884700793617206 73.32887065260972) bank51371 +51372 POINT(41.1880936917094 74.97217380725228) bank51372 +51373 POINT(41.08153667786284 73.61007682038804) bank51373 +51374 POINT(41.31659043306128 73.84102439069206) bank51374 +51375 POINT(41.53581271922641 73.42816875175261) bank51375 +51376 POINT(40.05936231535653 74.3495579853989) bank51376 +51377 POINT(40.30488422383329 73.21858784217874) bank51377 +51378 POINT(39.99277402559986 74.62273534108435) bank51378 +51379 POINT(40.0321949988271 74.60330960235186) bank51379 +51380 POINT(41.092130357941045 74.8209567069845) bank51380 +51381 POINT(41.169040553647015 74.3273091350713) bank51381 +51382 POINT(40.73778333274831 74.84590015301877) bank51382 +51383 POINT(40.71281043774521 74.84310448476673) bank51383 +51384 POINT(39.92688100492607 74.64401809123181) bank51384 +51385 POINT(40.74213506275277 73.0729727688514) bank51385 +51386 POINT(40.404241244462554 73.15729779583796) bank51386 +51387 POINT(40.39512141795834 73.11454307779553) bank51387 +51388 POINT(40.63837557774978 73.63594161866253) bank51388 +51389 POINT(40.782253677292296 73.21388166810007) bank51389 +51390 POINT(39.91723318665193 73.6239034018983) bank51390 +51391 POINT(40.03476343257856 73.5059002269537) bank51391 +51392 POINT(39.745154942168654 73.10529571823936) bank51392 +51393 POINT(40.70898507682388 73.2034968250467) bank51393 +51394 POINT(40.88195317651757 73.87009642594667) bank51394 +51395 POINT(41.657750715888334 73.19299611653186) bank51395 +51396 POINT(41.052686493208284 74.95227389300449) bank51396 +51397 POINT(41.344347936228345 73.47543965852768) bank51397 +51398 POINT(41.687284202167625 73.66773376968148) bank51398 +51399 POINT(40.95348514734402 73.42708738853194) bank51399 +51400 POINT(41.557370848869375 73.01623838619048) bank51400 +51401 POINT(40.77711857858858 74.28543472673496) bank51401 +51402 POINT(39.75055159699482 73.43266209106007) bank51402 +51403 POINT(41.62918752235819 73.79794296566016) bank51403 +51404 POINT(40.08078523431348 73.9396182486667) bank51404 +51405 POINT(41.29252527916527 73.0590832506444) bank51405 +51406 POINT(40.65714805150674 73.02625314817182) bank51406 +51407 POINT(40.12353872867539 74.0544138946458) bank51407 +51408 POINT(40.0811295443337 73.97653748245042) bank51408 +51409 POINT(40.553053094034986 73.40055705494309) bank51409 +51410 POINT(41.37766891174123 74.12333611995204) bank51410 +51411 POINT(41.06616166297787 73.38847970949898) bank51411 +51412 POINT(39.74265632977393 74.88447673510763) bank51412 +51413 POINT(40.29676979953101 73.36856579791714) bank51413 +51414 POINT(40.32944370470252 74.60856212477808) bank51414 +51415 POINT(40.209665528308086 73.71391288431371) bank51415 +51416 POINT(40.17850491807252 73.91204257440072) bank51416 +51417 POINT(39.79292870175138 73.39391150299885) bank51417 +51418 POINT(41.32174145929318 73.62358528691753) bank51418 +51419 POINT(40.04306105503703 74.86710554916144) bank51419 +51420 POINT(40.8199467381817 73.41052100524105) bank51420 +51421 POINT(40.64449746169757 73.72442371219736) bank51421 +51422 POINT(40.910801353007976 73.13492445724351) bank51422 +51423 POINT(41.434496349612104 74.20594614658356) bank51423 +51424 POINT(41.64319432689322 73.48119992873808) bank51424 +51425 POINT(40.639085098144164 73.87548936936814) bank51425 +51426 POINT(41.12623160709721 74.50497368784339) bank51426 +51427 POINT(40.1283108286838 73.4100440293364) bank51427 +51428 POINT(40.1442342022875 73.72651522250798) bank51428 +51429 POINT(40.4770912516498 74.83643536407736) bank51429 +51430 POINT(41.1394450102825 73.41878521398554) bank51430 +51431 POINT(41.58574361110673 73.33671076163643) bank51431 +51432 POINT(40.25978513720157 74.20932943909095) bank51432 +51433 POINT(41.57607976933699 73.91782267520425) bank51433 +51434 POINT(41.27811889469044 73.13363081796311) bank51434 +51435 POINT(41.03422524135798 74.70824610344427) bank51435 +51436 POINT(40.66436100927944 74.97679432319316) bank51436 +51437 POINT(39.961467609583174 73.66720455735086) bank51437 +51438 POINT(41.69727723030854 74.65673884969773) bank51438 +51439 POINT(41.31440398912169 74.68327698146975) bank51439 +51440 POINT(40.36251716875752 74.93244073461197) bank51440 +51441 POINT(40.14692596634212 73.32854809495235) bank51441 +51442 POINT(40.12594817528067 74.39792080918953) bank51442 +51443 POINT(39.7148542907725 73.91698484915544) bank51443 +51444 POINT(41.48239757375379 74.54961872489989) bank51444 +51445 POINT(39.82989233037418 74.52743252283037) bank51445 +51446 POINT(40.395918096383845 74.80341064439732) bank51446 +51447 POINT(41.40227728087091 74.9439525120837) bank51447 +51448 POINT(41.45965148438399 73.83900982739907) bank51448 +51449 POINT(41.00095302555733 74.58259789746296) bank51449 +51450 POINT(40.949122258181085 74.81999360588999) bank51450 +51451 POINT(41.584712499133296 74.9516574226787) bank51451 +51452 POINT(40.394771149080874 74.99155225088563) bank51452 +51453 POINT(40.95970660584333 74.16220295098464) bank51453 +51454 POINT(41.1948142394744 74.94777313663401) bank51454 +51455 POINT(40.987562264364655 73.28210094205676) bank51455 +51456 POINT(41.29639271161363 74.86590688906749) bank51456 +51457 POINT(41.27684953799467 73.02520184094605) bank51457 +51458 POINT(39.93069806868553 73.2858232779364) bank51458 +51459 POINT(39.76563453314759 74.79799763830285) bank51459 +51460 POINT(41.581509286591086 74.04487621526249) bank51460 +51461 POINT(41.011076739769756 73.63383148770782) bank51461 +51462 POINT(41.548545836203296 73.91556609847756) bank51462 +51463 POINT(39.93961712987339 74.77489252615884) bank51463 +51464 POINT(41.37403868545481 74.65477450718589) bank51464 +51465 POINT(39.80745146446098 74.03323016321941) bank51465 +51466 POINT(40.10975259278443 74.04081874743842) bank51466 +51467 POINT(39.76639033111084 74.69565402852724) bank51467 +51468 POINT(40.57705794874182 74.14947132741729) bank51468 +51469 POINT(41.51949060216343 73.94091249390193) bank51469 +51470 POINT(40.59893698745663 74.46162863311937) bank51470 +51471 POINT(39.73769521040381 73.9618215213196) bank51471 +51472 POINT(40.82676510137692 73.13014424819036) bank51472 +51473 POINT(40.74642361697024 74.5689489689815) bank51473 +51474 POINT(41.44817065082313 73.25241844279601) bank51474 +51475 POINT(40.84094921814046 73.34150849185022) bank51475 +51476 POINT(41.342955494547105 74.339800548988) bank51476 +51477 POINT(40.06428338277474 74.27501545494599) bank51477 +51478 POINT(40.49334258237118 74.05752728953846) bank51478 +51479 POINT(40.02092216497347 73.4207134535975) bank51479 +51480 POINT(40.908961568973695 74.86768426997054) bank51480 +51481 POINT(39.888813966835514 73.05860330191878) bank51481 +51482 POINT(40.35361725202654 74.65559192556441) bank51482 +51483 POINT(39.7900432762502 74.03728373590344) bank51483 +51484 POINT(41.34195205164796 73.27544457088301) bank51484 +51485 POINT(41.60823359068411 74.86571714555545) bank51485 +51486 POINT(39.78121809925248 73.58011518577848) bank51486 +51487 POINT(40.51820743488823 73.11149986998856) bank51487 +51488 POINT(41.617304664747245 74.72323129369772) bank51488 +51489 POINT(41.22787589788968 74.0849124336245) bank51489 +51490 POINT(40.08332052186132 74.36369099124042) bank51490 +51491 POINT(41.300012844973736 73.21509803271348) bank51491 +51492 POINT(41.373069082684594 73.52246787750657) bank51492 +51493 POINT(40.25337474846395 73.36042542061763) bank51493 +51494 POINT(39.71351471634219 73.74862951986711) bank51494 +51495 POINT(40.977453417208366 74.80586713778352) bank51495 +51496 POINT(41.03642107009347 73.63923503595605) bank51496 +51497 POINT(40.31246799586157 73.22416651013869) bank51497 +51498 POINT(41.61664765461452 74.21172259285154) bank51498 +51499 POINT(39.86910962039365 74.67580322635264) bank51499 +51500 POINT(41.67586857516934 73.10159790290585) bank51500 +51501 POINT(41.46083960121855 74.30703013519535) bank51501 +51502 POINT(39.959767329753184 73.4853847789416) bank51502 +51503 POINT(40.71639110184209 73.36963103440551) bank51503 +51504 POINT(40.5636699313464 74.77177436729819) bank51504 +51505 POINT(41.36371659051295 73.53724686113893) bank51505 +51506 POINT(40.775866217893814 73.75921265576265) bank51506 +51507 POINT(40.55851630637244 73.42408155491121) bank51507 +51508 POINT(40.151956805988746 73.32907547436247) bank51508 +51509 POINT(39.99351041300919 73.09968930631035) bank51509 +51510 POINT(40.10195589456266 74.42945410818118) bank51510 +51511 POINT(40.69073654290974 73.47891457047851) bank51511 +51512 POINT(40.38257814201048 73.35223022219087) bank51512 +51513 POINT(41.28178323556997 74.82071588179998) bank51513 +51514 POINT(41.33172536365157 74.59958287290448) bank51514 +51515 POINT(40.58515933560959 74.51124242828519) bank51515 +51516 POINT(39.82647040302344 73.00988455930955) bank51516 +51517 POINT(40.1786511068898 74.5138344551581) bank51517 +51518 POINT(40.88478663193857 73.12970564126402) bank51518 +51519 POINT(40.349787745919535 73.99773286655069) bank51519 +51520 POINT(41.050685833320394 73.0162844410697) bank51520 +51521 POINT(39.79457700051249 73.96631648328075) bank51521 +51522 POINT(40.44398336559565 74.78886200720967) bank51522 +51523 POINT(41.69198411946592 74.33167179984679) bank51523 +51524 POINT(41.022831462609595 73.18815884817757) bank51524 +51525 POINT(40.508037198352056 73.22622180824212) bank51525 +51526 POINT(40.42203923878808 73.94407614499565) bank51526 +51527 POINT(41.1507420161457 73.70008867579801) bank51527 +51528 POINT(39.77835302085241 73.13746240869625) bank51528 +51529 POINT(40.857420664615134 74.7641443969794) bank51529 +51530 POINT(39.86716799375237 73.74001173932098) bank51530 +51531 POINT(40.26518268510494 74.93304572766087) bank51531 +51532 POINT(40.29831449107732 74.79163607007946) bank51532 +51533 POINT(40.01068891982872 73.52928593229696) bank51533 +51534 POINT(41.62963834777893 74.36073490410269) bank51534 +51535 POINT(39.77961723616732 74.08382538779158) bank51535 +51536 POINT(40.48623801886812 73.18097023436648) bank51536 +51537 POINT(40.880834151572145 74.08925988207466) bank51537 +51538 POINT(40.157906848077985 74.70234161161615) bank51538 +51539 POINT(41.09826460624539 74.10378706764406) bank51539 +51540 POINT(40.58227412646781 74.86447721336211) bank51540 +51541 POINT(40.24717011824795 74.20840593905085) bank51541 +51542 POINT(40.01128590320065 74.11904573698527) bank51542 +51543 POINT(41.59347833982634 74.89214866886718) bank51543 +51544 POINT(41.250668520853736 74.37696259655291) bank51544 +51545 POINT(41.344769562683595 74.45595329822018) bank51545 +51546 POINT(40.74947862856779 74.67498250693052) bank51546 +51547 POINT(40.67492458300042 73.71521725006303) bank51547 +51548 POINT(41.092320525516 73.54091150524161) bank51548 +51549 POINT(40.87618329734758 73.51174822812129) bank51549 +51550 POINT(40.8861391711769 73.5354490972678) bank51550 +51551 POINT(41.223201278970635 74.90943548253682) bank51551 +51552 POINT(40.46360987740901 73.76441888214816) bank51552 +51553 POINT(40.265266604635904 73.86307610072114) bank51553 +51554 POINT(39.92952508537354 74.99355240714382) bank51554 +51555 POINT(40.849134183489404 74.67944287298388) bank51555 +51556 POINT(39.78972912837429 74.1262396135596) bank51556 +51557 POINT(40.26694540222278 74.24972022603787) bank51557 +51558 POINT(41.36898285915674 74.6224211680035) bank51558 +51559 POINT(39.783789940803956 73.4872304436285) bank51559 +51560 POINT(41.61822395466629 73.01864650654268) bank51560 +51561 POINT(41.37272369422697 73.69120565568645) bank51561 +51562 POINT(41.2939945151109 73.2704159354558) bank51562 +51563 POINT(41.19052438981829 73.89222960593116) bank51563 +51564 POINT(40.934626420973785 74.98853573418437) bank51564 +51565 POINT(39.9715364320516 74.00416895478787) bank51565 +51566 POINT(40.37881954789641 74.54349485056093) bank51566 +51567 POINT(41.644368545266346 73.22979577525946) bank51567 +51568 POINT(41.406651618299826 73.63911256568836) bank51568 +51569 POINT(39.933804406572015 74.09431123350049) bank51569 +51570 POINT(41.087660900173006 74.92469818765647) bank51570 +51571 POINT(41.31107374114217 73.09447959512913) bank51571 +51572 POINT(41.691796772613806 74.0684914102905) bank51572 +51573 POINT(40.36920017595881 73.57722509352664) bank51573 +51574 POINT(39.97937809662597 73.49802713447455) bank51574 +51575 POINT(40.77927248819294 73.5582282151138) bank51575 +51576 POINT(41.07153256624869 74.87811040698419) bank51576 +51577 POINT(40.802199426169985 74.40618438743977) bank51577 +51578 POINT(39.882553389330745 73.19140115713466) bank51578 +51579 POINT(40.39557598519501 73.76804672711735) bank51579 +51580 POINT(40.56460836066938 74.96821022643024) bank51580 +51581 POINT(40.17847374456875 73.56869030813088) bank51581 +51582 POINT(41.265624938688525 74.11585219278199) bank51582 +51583 POINT(41.538630920146076 74.36452835910585) bank51583 +51584 POINT(41.63916004432803 74.57387589893082) bank51584 +51585 POINT(40.24435452092141 73.04064474260203) bank51585 +51586 POINT(39.76674267154857 73.45651870627228) bank51586 +51587 POINT(40.06821451162941 73.87757576291189) bank51587 +51588 POINT(39.76031364971183 73.11291082145515) bank51588 +51589 POINT(39.74729634275591 74.8510315205689) bank51589 +51590 POINT(41.082419581188034 74.32378911210316) bank51590 +51591 POINT(41.427123687615264 73.29505956141985) bank51591 +51592 POINT(40.782204140928265 74.57487889214045) bank51592 +51593 POINT(41.00689629059212 74.594319432548) bank51593 +51594 POINT(40.0318640639347 74.94490747320067) bank51594 +51595 POINT(41.445670307785534 73.30833429086576) bank51595 +51596 POINT(40.21352164001178 73.7931481774664) bank51596 +51597 POINT(40.82602653709733 74.26426348864976) bank51597 +51598 POINT(41.555355325176706 74.78522125859104) bank51598 +51599 POINT(40.38876187824045 73.5915175875817) bank51599 +51600 POINT(41.234465142100404 73.28192542828882) bank51600 +51601 POINT(41.21975740663011 74.88586081923317) bank51601 +51602 POINT(40.87609727839019 74.01614280246604) bank51602 +51603 POINT(40.10780879820181 74.13904710792389) bank51603 +51604 POINT(40.45274513020681 74.60814809616457) bank51604 +51605 POINT(40.168566500707904 74.82749822499837) bank51605 +51606 POINT(40.48045833844874 74.91859927277183) bank51606 +51607 POINT(40.18536733454465 74.03884124282665) bank51607 +51608 POINT(40.44635480435832 73.034215553626) bank51608 +51609 POINT(39.92859749712312 73.73564713540227) bank51609 +51610 POINT(41.44660898800931 73.65829271207949) bank51610 +51611 POINT(41.4612034763082 73.62051582144055) bank51611 +51612 POINT(41.27016620380348 74.8408435201304) bank51612 +51613 POINT(40.28461484140199 73.27355549260481) bank51613 +51614 POINT(40.83857480910119 74.50493886396194) bank51614 +51615 POINT(41.60186757086165 73.5559144670318) bank51615 +51616 POINT(40.78948441603979 74.7163020823284) bank51616 +51617 POINT(40.554542296762705 74.53228625701514) bank51617 +51618 POINT(41.109336540616674 74.09520645325779) bank51618 +51619 POINT(39.7312127405458 73.79129103308323) bank51619 +51620 POINT(41.08503928931571 74.6319797889276) bank51620 +51621 POINT(39.99451618702365 73.09625180912552) bank51621 +51622 POINT(41.51458529419711 74.35765066617202) bank51622 +51623 POINT(39.75325273132727 74.71881631799312) bank51623 +51624 POINT(41.42847318387926 74.82355954965773) bank51624 +51625 POINT(39.77070547755328 73.93372047396096) bank51625 +51626 POINT(40.25681274940741 74.71138896530267) bank51626 +51627 POINT(40.35370794865292 74.97170191517012) bank51627 +51628 POINT(40.19263136638785 73.38069984214205) bank51628 +51629 POINT(40.938229838762865 74.82903886553947) bank51629 +51630 POINT(40.54684650141695 74.95891418502579) bank51630 +51631 POINT(41.32517266757804 74.7200582520499) bank51631 +51632 POINT(41.513564088579884 73.36162016243627) bank51632 +51633 POINT(41.093095923613056 73.42719667473604) bank51633 +51634 POINT(40.922169874300046 74.51154842759979) bank51634 +51635 POINT(40.83869335248598 73.62056548943306) bank51635 +51636 POINT(41.328918494850384 74.12209118635565) bank51636 +51637 POINT(41.56611773810231 74.77903374895516) bank51637 +51638 POINT(41.61214733302244 74.79049187688699) bank51638 +51639 POINT(41.07355981745391 74.47591828149599) bank51639 +51640 POINT(40.625637414756724 74.37109975027171) bank51640 +51641 POINT(41.22189537903564 73.40316430917706) bank51641 +51642 POINT(40.32807777985477 74.99570400713475) bank51642 +51643 POINT(41.36884370776559 73.10909322356346) bank51643 +51644 POINT(40.86342205567728 74.34696275322977) bank51644 +51645 POINT(39.761343640843066 74.03670253346142) bank51645 +51646 POINT(39.783242681320914 74.39536622435415) bank51646 +51647 POINT(41.20251734914931 73.33879927432174) bank51647 +51648 POINT(40.032879639556995 74.90143660661296) bank51648 +51649 POINT(40.022821913840744 73.62219556217282) bank51649 +51650 POINT(41.44831477123894 73.21146682003277) bank51650 +51651 POINT(41.11276730758909 73.7006388488426) bank51651 +51652 POINT(41.088528739823204 74.5070605265953) bank51652 +51653 POINT(40.85769186071791 74.44550832301398) bank51653 +51654 POINT(41.38077923480264 73.53935196368693) bank51654 +51655 POINT(40.20725090091154 73.4081906758196) bank51655 +51656 POINT(41.612518806312956 73.85921589445279) bank51656 +51657 POINT(40.277664807110185 74.94381272734645) bank51657 +51658 POINT(41.274920394045715 73.03496984385599) bank51658 +51659 POINT(41.11853597301674 74.03018081184209) bank51659 +51660 POINT(40.85835551830082 74.34153689300247) bank51660 +51661 POINT(39.841052684592384 74.69134854104146) bank51661 +51662 POINT(39.81325782362524 73.13361760380909) bank51662 +51663 POINT(41.24567955016517 74.90619574403276) bank51663 +51664 POINT(40.541932530722114 73.85855972498301) bank51664 +51665 POINT(40.010050163797466 73.31741684171585) bank51665 +51666 POINT(40.07648369487613 73.57308409657075) bank51666 +51667 POINT(40.414239757996675 73.13315655599308) bank51667 +51668 POINT(40.76664893020957 73.64388853285897) bank51668 +51669 POINT(40.947777114990316 73.73488591793884) bank51669 +51670 POINT(40.5937668053341 73.84277635480775) bank51670 +51671 POINT(40.72275755844818 73.81184501331488) bank51671 +51672 POINT(41.09185732960146 73.4717405318838) bank51672 +51673 POINT(39.99842778686013 73.89861758146571) bank51673 +51674 POINT(41.183174082237 73.57681136201165) bank51674 +51675 POINT(40.57439055533794 74.85266009871032) bank51675 +51676 POINT(41.61021123068888 74.69270815197832) bank51676 +51677 POINT(41.228287647833405 73.28018014876557) bank51677 +51678 POINT(41.47450786791558 74.22401597990792) bank51678 +51679 POINT(40.32028795223398 73.65671604165716) bank51679 +51680 POINT(40.08781150746694 73.69668312504717) bank51680 +51681 POINT(39.764195900336844 74.43840546080666) bank51681 +51682 POINT(40.94569415188303 74.66473728978326) bank51682 +51683 POINT(40.829541851430925 74.41701581129344) bank51683 +51684 POINT(41.31692784467873 73.97966227150755) bank51684 +51685 POINT(41.505139868574034 73.80639004813924) bank51685 +51686 POINT(40.06682283314334 74.66476799231643) bank51686 +51687 POINT(41.52946999313726 74.02859859546359) bank51687 +51688 POINT(41.48708202969933 73.17673126280167) bank51688 +51689 POINT(41.30517945600749 73.82114308173614) bank51689 +51690 POINT(40.32243140749267 73.45621791846114) bank51690 +51691 POINT(41.304765333145234 74.74916205505616) bank51691 +51692 POINT(39.81824643060801 74.44018385492794) bank51692 +51693 POINT(40.448496195514664 74.80935504305228) bank51693 +51694 POINT(39.73947977119265 73.30318614085887) bank51694 +51695 POINT(40.1340022431834 74.69538924902706) bank51695 +51696 POINT(39.7302925195512 74.53436849503458) bank51696 +51697 POINT(40.11279561858996 74.85472401381443) bank51697 +51698 POINT(40.66494531194411 73.1683983298498) bank51698 +51699 POINT(40.37789382455312 73.7567937299497) bank51699 +51700 POINT(40.515152774111364 74.0462642244497) bank51700 +51701 POINT(41.1982292727748 74.0485482186233) bank51701 +51702 POINT(40.132464110850826 74.70565564009628) bank51702 +51703 POINT(39.72002359833284 74.35874246372089) bank51703 +51704 POINT(39.8314175124445 74.52571516590223) bank51704 +51705 POINT(41.0790758684297 74.26863580842112) bank51705 +51706 POINT(40.53835746876509 74.62734214177192) bank51706 +51707 POINT(40.179686183428736 73.68267710360414) bank51707 +51708 POINT(40.52276995244553 74.47981097204001) bank51708 +51709 POINT(40.67618772021902 74.67039705411693) bank51709 +51710 POINT(40.35449252153206 73.18004137667971) bank51710 +51711 POINT(41.371820986795726 74.62325328081559) bank51711 +51712 POINT(41.5673622905123 74.85252681039627) bank51712 +51713 POINT(40.90929289864813 74.22758745798681) bank51713 +51714 POINT(40.2077717238697 73.27602063910565) bank51714 +51715 POINT(41.68812800060934 74.72892878833798) bank51715 +51716 POINT(40.68864730418256 73.87122610528283) bank51716 +51717 POINT(40.273669445971 74.29776321814269) bank51717 +51718 POINT(40.919512644047906 74.1434657473049) bank51718 +51719 POINT(39.93961357040276 74.92487244971318) bank51719 +51720 POINT(40.03017543778613 74.81121381960742) bank51720 +51721 POINT(39.71787851670522 74.32086152289011) bank51721 +51722 POINT(41.008572195263206 74.40808032745018) bank51722 +51723 POINT(41.09567739931925 74.78778876610455) bank51723 +51724 POINT(39.93303581336474 74.55266440924977) bank51724 +51725 POINT(41.477933646044555 73.6455166392658) bank51725 +51726 POINT(40.29070882543612 73.1530233728593) bank51726 +51727 POINT(40.48910494745468 73.00829869642408) bank51727 +51728 POINT(39.85702755248485 74.97819955330586) bank51728 +51729 POINT(41.40833936500778 73.63882784151069) bank51729 +51730 POINT(41.482962089220266 74.70731379417) bank51730 +51731 POINT(39.75773771088398 73.01013140234956) bank51731 +51732 POINT(41.0689569419777 73.93797505204444) bank51732 +51733 POINT(39.809057033594826 73.5041272469477) bank51733 +51734 POINT(41.4932715942948 74.80994789711671) bank51734 +51735 POINT(40.106216334047254 74.66800134311191) bank51735 +51736 POINT(41.580489248350545 73.41603239076143) bank51736 +51737 POINT(40.197201826133664 74.81440774792152) bank51737 +51738 POINT(40.6105479026181 74.92297010027279) bank51738 +51739 POINT(39.7392032945753 73.64714067709052) bank51739 +51740 POINT(41.63816431774211 74.44348933864812) bank51740 +51741 POINT(39.77233393348781 74.95805334976421) bank51741 +51742 POINT(41.264384008041226 73.96531960391715) bank51742 +51743 POINT(40.65440797668462 73.92064638200728) bank51743 +51744 POINT(40.59604322001775 74.67322131547702) bank51744 +51745 POINT(41.51168555878716 73.61183680586271) bank51745 +51746 POINT(41.54629037781911 73.53969711439598) bank51746 +51747 POINT(39.791559232051775 73.4279190982379) bank51747 +51748 POINT(40.40681820148996 73.01532937995862) bank51748 +51749 POINT(41.18956639559318 74.62127055550062) bank51749 +51750 POINT(40.32657587913756 74.17718341282723) bank51750 +51751 POINT(40.137435199799626 74.15612258926765) bank51751 +51752 POINT(41.26727410623572 74.54633635518518) bank51752 +51753 POINT(40.433486769317966 74.13229618267654) bank51753 +51754 POINT(41.3008130190231 74.49906755717248) bank51754 +51755 POINT(40.25122434346443 73.63897670093647) bank51755 +51756 POINT(41.27858336705438 74.27039071611564) bank51756 +51757 POINT(40.31230851756814 74.10338415025828) bank51757 +51758 POINT(40.807245951277395 73.54851865213446) bank51758 +51759 POINT(40.29599201482973 74.0428812593724) bank51759 +51760 POINT(40.99453170128943 74.42791516191707) bank51760 +51761 POINT(41.28500086394419 74.65578963082365) bank51761 +51762 POINT(40.873996424337705 73.77240307908271) bank51762 +51763 POINT(39.950180432289294 73.78692573369923) bank51763 +51764 POINT(39.98846679068568 74.19857724711255) bank51764 +51765 POINT(41.24884351834804 73.29347246781406) bank51765 +51766 POINT(41.312462905848946 74.21118814389678) bank51766 +51767 POINT(39.75003122292435 74.31655737528975) bank51767 +51768 POINT(40.73108886353232 74.30059355977069) bank51768 +51769 POINT(39.729979980620186 74.63614059050292) bank51769 +51770 POINT(40.30564784637003 74.66086958780228) bank51770 +51771 POINT(39.916720919478486 73.36978638879536) bank51771 +51772 POINT(41.53944784819419 73.80263027667525) bank51772 +51773 POINT(41.63828353269503 74.89537077025365) bank51773 +51774 POINT(41.58196302731912 74.51018804378855) bank51774 +51775 POINT(41.56701099421587 74.03652254129305) bank51775 +51776 POINT(41.19263461317838 75.0024758955871) bank51776 +51777 POINT(39.95922702474172 73.37681510222117) bank51777 +51778 POINT(40.57640464553837 74.26593830770003) bank51778 +51779 POINT(41.24402764248419 73.57809594562804) bank51779 +51780 POINT(41.68500891494894 74.09553134201428) bank51780 +51781 POINT(40.257293323490984 73.49458159665907) bank51781 +51782 POINT(41.22208125453223 73.02670388709421) bank51782 +51783 POINT(40.41847751075402 73.59872149746533) bank51783 +51784 POINT(41.06603074024952 74.98801556374858) bank51784 +51785 POINT(40.96410452920107 73.88623206871408) bank51785 +51786 POINT(40.74230039786007 73.17723048265212) bank51786 +51787 POINT(41.07737860824783 73.53643860537846) bank51787 +51788 POINT(40.04691352560015 74.339692987736) bank51788 +51789 POINT(40.08309761808756 74.82728117988917) bank51789 +51790 POINT(41.52370869832154 73.8760474545962) bank51790 +51791 POINT(41.08886032812316 73.7760936799985) bank51791 +51792 POINT(40.93637445445362 74.41684738044898) bank51792 +51793 POINT(39.88606361723056 74.5191322259415) bank51793 +51794 POINT(39.87089608997241 73.50520586630141) bank51794 +51795 POINT(39.80198640202981 74.011521211795) bank51795 +51796 POINT(41.03463084382192 73.85374676120286) bank51796 +51797 POINT(39.811030436927204 73.88331327662281) bank51797 +51798 POINT(41.42016414734898 74.95074194894688) bank51798 +51799 POINT(41.62448314126282 73.46220458021581) bank51799 +51800 POINT(40.02858572297541 73.27419037689293) bank51800 +51801 POINT(39.8035594765213 74.14865295492719) bank51801 +51802 POINT(41.004200138634666 74.96962787596925) bank51802 +51803 POINT(40.46519498343502 73.2251958608899) bank51803 +51804 POINT(40.24863026622922 73.721822261821) bank51804 +51805 POINT(40.266385141643504 74.49485059942995) bank51805 +51806 POINT(40.22056386819864 73.36220414195594) bank51806 +51807 POINT(39.78089033579402 73.9624559549751) bank51807 +51808 POINT(41.684265373009254 73.23038293917561) bank51808 +51809 POINT(40.12509717742864 73.28175492872111) bank51809 +51810 POINT(41.4388917952527 74.82072891443639) bank51810 +51811 POINT(41.361889850967536 74.00867584654924) bank51811 +51812 POINT(40.97488413089548 73.83982276516168) bank51812 +51813 POINT(41.164824259537475 73.26627051329639) bank51813 +51814 POINT(40.660378518600595 73.49825222868617) bank51814 +51815 POINT(41.18216752753797 74.53062001722238) bank51815 +51816 POINT(40.04343667813586 73.90467962261177) bank51816 +51817 POINT(40.732110733453716 74.30922567406574) bank51817 +51818 POINT(39.833996838048435 74.97176653371487) bank51818 +51819 POINT(41.68516506149188 73.74132289641334) bank51819 +51820 POINT(40.940980253588386 74.54931417474823) bank51820 +51821 POINT(40.3370408213968 74.77435155230027) bank51821 +51822 POINT(41.557378980498044 73.66298294058026) bank51822 +51823 POINT(40.03443531543761 73.74286143587254) bank51823 +51824 POINT(40.43090877639783 74.85199522726101) bank51824 +51825 POINT(39.88881577910098 73.38261978670542) bank51825 +51826 POINT(40.16551012201534 74.4444050585251) bank51826 +51827 POINT(39.95262718620117 73.51695752372709) bank51827 +51828 POINT(40.817466138945775 74.38758526333561) bank51828 +51829 POINT(39.76522050570685 73.38189204633507) bank51829 +51830 POINT(40.468097043546535 73.24576763615681) bank51830 +51831 POINT(40.76371802162071 74.81991834947122) bank51831 +51832 POINT(41.17700881335209 73.06027086976894) bank51832 +51833 POINT(40.740622298655055 74.84656473704935) bank51833 +51834 POINT(39.730536582967076 74.82171951833868) bank51834 +51835 POINT(40.06222535619568 74.74099336712818) bank51835 +51836 POINT(40.42300697601126 74.42582094161196) bank51836 +51837 POINT(41.32611702819947 74.56586891452208) bank51837 +51838 POINT(39.90677208395741 74.82267035431161) bank51838 +51839 POINT(40.04989256473317 74.64498504878208) bank51839 +51840 POINT(41.07930017169633 73.3847542299657) bank51840 +51841 POINT(40.64827719951208 74.87630003351347) bank51841 +51842 POINT(41.52710962579803 74.99173279021075) bank51842 +51843 POINT(41.602036219438475 73.54949055073669) bank51843 +51844 POINT(41.70180164647029 74.78983368003941) bank51844 +51845 POINT(40.35151451918446 74.63285168494991) bank51845 +51846 POINT(40.35327454264193 74.03535826561578) bank51846 +51847 POINT(39.878487426042426 73.07085129556526) bank51847 +51848 POINT(41.22581635201994 74.47009758865825) bank51848 +51849 POINT(40.68860091164192 73.5735206790446) bank51849 +51850 POINT(40.71858152502471 74.78480715614198) bank51850 +51851 POINT(40.58513945723528 73.59121315708316) bank51851 +51852 POINT(41.21423214016496 73.26015438635024) bank51852 +51853 POINT(40.13912744016862 73.79344816693545) bank51853 +51854 POINT(41.01139565684768 73.2089757368707) bank51854 +51855 POINT(41.55387018242881 73.1920459306038) bank51855 +51856 POINT(41.58277684689454 74.72514588234355) bank51856 +51857 POINT(41.37882397454752 73.66815230439028) bank51857 +51858 POINT(41.50498562383778 74.70345757028554) bank51858 +51859 POINT(41.305772048173175 74.9766590502872) bank51859 +51860 POINT(41.315590225488776 73.18985901858858) bank51860 +51861 POINT(40.559478445874376 74.58987891328844) bank51861 +51862 POINT(41.154675928321616 74.61439937726759) bank51862 +51863 POINT(40.996075589090864 73.56262588324394) bank51863 +51864 POINT(39.78149769551707 73.34424659726469) bank51864 +51865 POINT(41.0533712472626 73.18130880569211) bank51865 +51866 POINT(41.623854323606785 74.33101367933824) bank51866 +51867 POINT(41.50114236093102 73.06246575254747) bank51867 +51868 POINT(40.46240904771953 73.6960941595984) bank51868 +51869 POINT(41.2656134672019 73.43649787767215) bank51869 +51870 POINT(41.25571406339004 74.99210735394189) bank51870 +51871 POINT(41.526586627357084 74.69780945090943) bank51871 +51872 POINT(40.420055987690546 73.14321693125486) bank51872 +51873 POINT(41.525718910606 73.92010093146318) bank51873 +51874 POINT(41.44550646810831 74.78797297610289) bank51874 +51875 POINT(41.44845480468678 74.93116109004562) bank51875 +51876 POINT(40.62916227124888 73.95309624047566) bank51876 +51877 POINT(40.9912728232858 74.9539209020576) bank51877 +51878 POINT(39.78718270941475 73.70109184663302) bank51878 +51879 POINT(41.30909400194651 74.41611834436658) bank51879 +51880 POINT(41.300008418069226 74.2450200473239) bank51880 +51881 POINT(41.58159407077698 73.21487391871061) bank51881 +51882 POINT(40.23244412074364 74.41883569706957) bank51882 +51883 POINT(39.7619256732985 73.95190288927057) bank51883 +51884 POINT(40.65464997108456 73.35638589459882) bank51884 +51885 POINT(39.72693508174747 74.00820015457843) bank51885 +51886 POINT(41.37518068916718 73.948828931284) bank51886 +51887 POINT(40.113923894891 73.37677930297836) bank51887 +51888 POINT(39.89699029251349 73.20996218959613) bank51888 +51889 POINT(41.194470789121446 73.60304775155623) bank51889 +51890 POINT(40.64630686291967 73.04094747261513) bank51890 +51891 POINT(40.261412485683785 73.24588625521913) bank51891 +51892 POINT(41.19332984784564 74.3998166157699) bank51892 +51893 POINT(40.668455585099686 74.96846857654815) bank51893 +51894 POINT(40.533822194807755 74.29020877804011) bank51894 +51895 POINT(40.667711659688514 73.43876085729427) bank51895 +51896 POINT(40.243367407075034 73.4459509711101) bank51896 +51897 POINT(41.157827812408094 73.24499638296649) bank51897 +51898 POINT(41.244830484931846 73.79164743794054) bank51898 +51899 POINT(39.73814784946059 73.05514538379583) bank51899 +51900 POINT(41.180553431009514 74.07277301615429) bank51900 +51901 POINT(40.44054843891983 74.70105720700988) bank51901 +51902 POINT(41.55388484972907 73.94591275672721) bank51902 +51903 POINT(40.046255841073936 73.98892969932665) bank51903 +51904 POINT(39.76271862015482 73.39907871809017) bank51904 +51905 POINT(40.2186489189269 73.03610481591737) bank51905 +51906 POINT(41.22948273572802 74.50004227168405) bank51906 +51907 POINT(39.84337941176314 73.17382443803358) bank51907 +51908 POINT(40.75513797278023 74.41274056399706) bank51908 +51909 POINT(39.81292670375443 74.38585951105267) bank51909 +51910 POINT(40.65766337415715 73.09162088124432) bank51910 +51911 POINT(40.38708901703605 73.58943163576704) bank51911 +51912 POINT(40.014710531751334 74.25525118492801) bank51912 +51913 POINT(40.386764594354965 74.31360081953115) bank51913 +51914 POINT(41.03230077543973 73.43040025934977) bank51914 +51915 POINT(40.51653127059776 73.59766155766977) bank51915 +51916 POINT(40.72663610703933 74.89435585710419) bank51916 +51917 POINT(41.70773014721989 74.26304500830948) bank51917 +51918 POINT(41.29662292224389 73.5753191496549) bank51918 +51919 POINT(41.147416012054634 73.52922108198037) bank51919 +51920 POINT(40.31894609410641 73.7080626344549) bank51920 +51921 POINT(41.2893851522573 73.71864208085557) bank51921 +51922 POINT(41.28894120015766 74.05525344353528) bank51922 +51923 POINT(41.414874694553774 73.31103961116283) bank51923 +51924 POINT(41.305201564153485 73.41680821183043) bank51924 +51925 POINT(40.02252575901703 74.70982561622533) bank51925 +51926 POINT(41.07251093513698 74.23386873143875) bank51926 +51927 POINT(40.49172949517775 73.11368590227617) bank51927 +51928 POINT(41.253522805066886 74.06539370298708) bank51928 +51929 POINT(39.96510502691764 73.01843380760529) bank51929 +51930 POINT(41.06222781313667 74.5958911492857) bank51930 +51931 POINT(40.53888744647175 74.10437836094444) bank51931 +51932 POINT(40.72675084783846 73.36268419181462) bank51932 +51933 POINT(40.32876122731284 73.83933170404102) bank51933 +51934 POINT(40.78555103563196 73.59495853971536) bank51934 +51935 POINT(40.663378540641354 73.95021623721296) bank51935 +51936 POINT(41.65889133079998 74.23181208154034) bank51936 +51937 POINT(40.56066444959873 73.36085544265391) bank51937 +51938 POINT(40.99094818989382 74.44179295879823) bank51938 +51939 POINT(39.941301595282106 74.04657345866725) bank51939 +51940 POINT(40.87954543741211 73.6250123936082) bank51940 +51941 POINT(40.04985930356054 73.10643623479713) bank51941 +51942 POINT(41.1073329782936 74.04594246341125) bank51942 +51943 POINT(40.60044573606258 74.13392816756989) bank51943 +51944 POINT(39.94826178942662 74.97185266249974) bank51944 +51945 POINT(39.92721310484515 74.96918968192273) bank51945 +51946 POINT(40.567449406293704 74.94622130558194) bank51946 +51947 POINT(40.98048969767972 73.42515935413095) bank51947 +51948 POINT(41.23404671166147 74.88643627029364) bank51948 +51949 POINT(39.849353456709686 74.19632341157468) bank51949 +51950 POINT(40.27077618793474 73.66646468901205) bank51950 +51951 POINT(41.26449738224433 73.39427963747471) bank51951 +51952 POINT(40.580815819551795 73.03090291924785) bank51952 +51953 POINT(41.03163536048261 73.98452532957445) bank51953 +51954 POINT(41.46158104518705 74.00759691457837) bank51954 +51955 POINT(40.86503549393205 73.07322279037567) bank51955 +51956 POINT(39.83467097343571 74.5231661512038) bank51956 +51957 POINT(40.39829667465585 74.06214022961996) bank51957 +51958 POINT(41.55252403062249 74.25512612532434) bank51958 +51959 POINT(40.389681895618686 74.1557946740951) bank51959 +51960 POINT(40.94627380756865 73.54929686903702) bank51960 +51961 POINT(40.39709811855105 74.0170703770405) bank51961 +51962 POINT(41.69920417506866 74.35154005515514) bank51962 +51963 POINT(39.739057951651226 73.3711554589596) bank51963 +51964 POINT(41.267636043367816 73.65312914146624) bank51964 +51965 POINT(41.10461719284898 74.58400096562042) bank51965 +51966 POINT(40.588899922469885 74.50588886463844) bank51966 +51967 POINT(41.5112929834539 74.23019199703738) bank51967 +51968 POINT(40.562487925431576 74.97020987721059) bank51968 +51969 POINT(41.69615187867506 74.42439952463229) bank51969 +51970 POINT(40.83118207222762 73.57275366586883) bank51970 +51971 POINT(41.599932793518434 74.32116863489672) bank51971 +51972 POINT(40.715958075030365 73.81730378879362) bank51972 +51973 POINT(41.2877199555374 73.50115959204444) bank51973 +51974 POINT(40.536785951029046 73.43673920548078) bank51974 +51975 POINT(41.18487626138904 73.43021796221578) bank51975 +51976 POINT(40.764633233011224 73.75354288343947) bank51976 +51977 POINT(40.919590626551944 74.35221053412707) bank51977 +51978 POINT(41.55089028395361 73.75822243609313) bank51978 +51979 POINT(40.522100647756695 74.02843324373255) bank51979 +51980 POINT(40.49426241749618 73.04040789393899) bank51980 +51981 POINT(39.78721816989685 74.4965448516003) bank51981 +51982 POINT(40.750640605026774 74.11719926456014) bank51982 +51983 POINT(41.19809940573591 74.77857109395237) bank51983 +51984 POINT(41.18263658548582 73.1940512737056) bank51984 +51985 POINT(39.899324324605466 74.30199577246778) bank51985 +51986 POINT(40.641199825624035 74.44693324413478) bank51986 +51987 POINT(40.525422639344285 74.69926835830182) bank51987 +51988 POINT(39.72700956306321 74.68850752336958) bank51988 +51989 POINT(40.38532439650161 73.7881644782243) bank51989 +51990 POINT(40.53458777167077 73.49011805086073) bank51990 +51991 POINT(40.67556501073335 73.66158234477608) bank51991 +51992 POINT(40.17643928358011 73.81584196684729) bank51992 +51993 POINT(41.06488435721123 73.27925791206971) bank51993 +51994 POINT(39.85954154719046 74.07707952515666) bank51994 +51995 POINT(39.99305478116826 74.61037910704803) bank51995 +51996 POINT(39.82595480866641 73.64440654356821) bank51996 +51997 POINT(40.60995093766608 73.13620221556788) bank51997 +51998 POINT(41.618238871432766 73.51820983601064) bank51998 +51999 POINT(41.53297263497994 73.8991415687031) bank51999 +52000 POINT(41.667001236252084 73.03211097294277) bank52000 +52001 POINT(40.611226049348886 74.92588121447142) bank52001 +52002 POINT(41.028396497006 74.74373356003477) bank52002 +52003 POINT(40.952378551172956 74.23593016820409) bank52003 +52004 POINT(40.932268008751194 74.96731463909721) bank52004 +52005 POINT(41.02630370450916 74.98842546397185) bank52005 +52006 POINT(39.913483204610856 73.55540873507313) bank52006 +52007 POINT(40.71403755834594 74.28923853413215) bank52007 +52008 POINT(40.23049733634402 73.02428006103392) bank52008 +52009 POINT(40.565584383685675 73.62237207669864) bank52009 +52010 POINT(40.64956231937286 73.48054531543029) bank52010 +52011 POINT(40.91900797939116 73.79316054989575) bank52011 +52012 POINT(40.405293073139575 73.03469108717759) bank52012 +52013 POINT(40.47958231723935 74.73179863548093) bank52013 +52014 POINT(39.91027064382507 74.39930349064144) bank52014 +52015 POINT(41.068636287319215 73.19853932336926) bank52015 +52016 POINT(40.91398349853989 74.85853198761225) bank52016 +52017 POINT(41.07643291637196 73.2920340901964) bank52017 +52018 POINT(41.086894056876595 73.78767201235318) bank52018 +52019 POINT(40.80009158754793 74.1814960393821) bank52019 +52020 POINT(41.53343854088586 73.11807073913259) bank52020 +52021 POINT(41.41791843794223 74.28444755634541) bank52021 +52022 POINT(40.010583779822845 74.49381231425781) bank52022 +52023 POINT(41.01802876523092 74.50046551700765) bank52023 +52024 POINT(40.01545339891815 74.9823435716707) bank52024 +52025 POINT(41.26837136776353 73.06802277220866) bank52025 +52026 POINT(39.73743594681836 73.85999143578123) bank52026 +52027 POINT(40.92293679975496 73.31680615878031) bank52027 +52028 POINT(41.20178873248419 74.29444317134902) bank52028 +52029 POINT(41.14131157016918 73.47058920952514) bank52029 +52030 POINT(41.01819394105876 74.42996715442183) bank52030 +52031 POINT(40.48874762494397 74.78428479314492) bank52031 +52032 POINT(41.002770904870985 73.47161812665793) bank52032 +52033 POINT(40.192040353299646 73.84097199540989) bank52033 +52034 POINT(40.284355720340145 73.99483657819904) bank52034 +52035 POINT(40.33693681933281 73.37478137310971) bank52035 +52036 POINT(41.06809217595359 74.10105239183824) bank52036 +52037 POINT(41.68720470490503 74.00725636269097) bank52037 +52038 POINT(41.26607058700747 73.8393865003364) bank52038 +52039 POINT(41.169540569267475 73.10984127514958) bank52039 +52040 POINT(41.343043657388115 74.49174071560468) bank52040 +52041 POINT(41.494774155701414 74.62471348496327) bank52041 +52042 POINT(40.83133088539562 74.87002159555777) bank52042 +52043 POINT(41.15740341632904 74.06536842981109) bank52043 +52044 POINT(40.012018766812055 74.23938481331388) bank52044 +52045 POINT(41.062870661925274 73.91119017116506) bank52045 +52046 POINT(39.98744206949188 74.59208158285664) bank52046 +52047 POINT(40.2494779152393 73.96943315657744) bank52047 +52048 POINT(40.758788286238804 73.3869408754003) bank52048 +52049 POINT(40.42384303445879 73.7512286868127) bank52049 +52050 POINT(39.89752540986655 73.49654730022644) bank52050 +52051 POINT(41.17768174904727 74.21390913305973) bank52051 +52052 POINT(40.747734806282004 74.84440548989744) bank52052 +52053 POINT(40.794685522827756 73.1638246598635) bank52053 +52054 POINT(41.57322240198865 73.92592811949908) bank52054 +52055 POINT(40.20533120179376 74.75961821649057) bank52055 +52056 POINT(40.74182118970713 74.12120345330594) bank52056 +52057 POINT(40.648066125284316 74.86212502306407) bank52057 +52058 POINT(40.062559674173905 74.53448529188344) bank52058 +52059 POINT(41.277957385036984 73.27629748231865) bank52059 +52060 POINT(40.69033623049019 73.93156439347315) bank52060 +52061 POINT(40.72361846660921 73.89266136583532) bank52061 +52062 POINT(40.348453404183594 73.3958025888061) bank52062 +52063 POINT(39.840872519577374 73.68589410908162) bank52063 +52064 POINT(40.2323677736669 73.17773433857617) bank52064 +52065 POINT(39.715355564610135 74.38517682353105) bank52065 +52066 POINT(40.98533092033885 74.55538651288938) bank52066 +52067 POINT(40.44425636958196 73.4786291247385) bank52067 +52068 POINT(39.93926429588295 73.6336245016843) bank52068 +52069 POINT(39.74910221642903 73.3797677832193) bank52069 +52070 POINT(40.36078439678808 74.93269616160865) bank52070 +52071 POINT(40.13315744948427 73.1236005776954) bank52071 +52072 POINT(40.44029295151169 73.7680620681871) bank52072 +52073 POINT(40.09884303771789 73.26932103334123) bank52073 +52074 POINT(39.98523082872565 73.88757694494299) bank52074 +52075 POINT(40.63549684687477 73.97065666151522) bank52075 +52076 POINT(41.04762159423964 74.56225202119012) bank52076 +52077 POINT(40.68972220338032 74.55982326532464) bank52077 +52078 POINT(40.81281766185107 74.34575035998772) bank52078 +52079 POINT(41.70909032394196 73.18873393437822) bank52079 +52080 POINT(41.64621396283723 74.23269192695794) bank52080 +52081 POINT(40.2650070237012 74.34044631229958) bank52081 +52082 POINT(40.38852442196572 73.97629951295995) bank52082 +52083 POINT(40.83413742091472 73.0878704822598) bank52083 +52084 POINT(41.4207768750075 74.93626110505032) bank52084 +52085 POINT(40.55976602784566 74.5358408034523) bank52085 +52086 POINT(41.41612266396235 74.41087841647223) bank52086 +52087 POINT(41.35892872728787 73.20956366411876) bank52087 +52088 POINT(40.63943917772211 74.46502326130955) bank52088 +52089 POINT(41.279420275516706 74.40417073486935) bank52089 +52090 POINT(41.01482406776021 74.20419959823413) bank52090 +52091 POINT(41.37005698220636 73.54148332126864) bank52091 +52092 POINT(41.256984921402264 74.892663381199) bank52092 +52093 POINT(41.226698888830164 74.74417524053712) bank52093 +52094 POINT(39.797393594912954 74.82161638161537) bank52094 +52095 POINT(41.24668700398158 73.93485548077905) bank52095 +52096 POINT(41.42093523889838 74.19128446902869) bank52096 +52097 POINT(40.14510260591419 74.82466321828973) bank52097 +52098 POINT(41.29547443181437 73.6293115323648) bank52098 +52099 POINT(40.682201683459446 73.78268103095802) bank52099 +52100 POINT(40.58876089021407 73.47345124718078) bank52100 +52101 POINT(41.36373729866618 74.0649008398283) bank52101 +52102 POINT(41.25416002203686 73.5627518839623) bank52102 +52103 POINT(40.2655796453926 73.65485260528578) bank52103 +52104 POINT(40.749530463293944 73.78758667721004) bank52104 +52105 POINT(40.41067707445309 73.65869499592877) bank52105 +52106 POINT(40.22902087882409 74.65863392907237) bank52106 +52107 POINT(41.05208230507551 74.30447666680782) bank52107 +52108 POINT(40.61665113106692 74.50753014686784) bank52108 +52109 POINT(41.68649119403652 73.22539580611553) bank52109 +52110 POINT(39.82663903325395 74.35233427134214) bank52110 +52111 POINT(40.96765133129771 73.60214360845288) bank52111 +52112 POINT(40.56695849629021 73.68407119195129) bank52112 +52113 POINT(39.82616965397442 74.02481267961478) bank52113 +52114 POINT(40.147600621419706 73.42508273607903) bank52114 +52115 POINT(41.36207753580319 74.51184362717594) bank52115 +52116 POINT(40.37676991843886 73.91374149468596) bank52116 +52117 POINT(40.38808108987103 74.07793113178381) bank52117 +52118 POINT(39.72061914580461 74.41217759574766) bank52118 +52119 POINT(40.27067643714785 74.33281701838138) bank52119 +52120 POINT(39.96442366211466 73.34607171363996) bank52120 +52121 POINT(40.767061677830704 74.00721783687281) bank52121 +52122 POINT(40.00889293381416 74.67838439985695) bank52122 +52123 POINT(40.65456980752471 73.87333633610416) bank52123 +52124 POINT(40.85816139544354 74.797424462898) bank52124 +52125 POINT(41.66107241255552 74.45766739597165) bank52125 +52126 POINT(41.4900471474771 73.79683691213182) bank52126 +52127 POINT(41.27737888119042 73.79065968069474) bank52127 +52128 POINT(39.90900580113448 74.25687448005544) bank52128 +52129 POINT(41.01850541349231 73.22308716202359) bank52129 +52130 POINT(39.96122704596879 74.01122082231689) bank52130 +52131 POINT(40.3303025128269 74.32961029078366) bank52131 +52132 POINT(41.148718535239844 73.3008213010605) bank52132 +52133 POINT(41.57954390233776 73.2381726968265) bank52133 +52134 POINT(40.35748485250762 74.273801907743) bank52134 +52135 POINT(40.20482149864916 73.21311979358885) bank52135 +52136 POINT(40.856333293756684 74.3452428216401) bank52136 +52137 POINT(40.03962874153301 74.34659839134804) bank52137 +52138 POINT(40.42962899649109 74.71544535240538) bank52138 +52139 POINT(40.91890505353356 73.80522251587088) bank52139 +52140 POINT(40.97293381121323 73.67571163918038) bank52140 +52141 POINT(40.151366196041366 73.1274945595596) bank52141 +52142 POINT(41.34121120743626 74.88373884163417) bank52142 +52143 POINT(41.190400140077266 74.84731615610374) bank52143 +52144 POINT(39.92861251976435 74.84590501187812) bank52144 +52145 POINT(40.77993070084009 74.66383103986821) bank52145 +52146 POINT(41.26516032982405 74.50333594554422) bank52146 +52147 POINT(40.357065880067196 74.43148446733181) bank52147 +52148 POINT(40.88431445954769 74.64367485686107) bank52148 +52149 POINT(40.38405953978792 74.8241421604577) bank52149 +52150 POINT(40.23254122004686 73.25852597613192) bank52150 +52151 POINT(40.94488459120974 73.58078301426144) bank52151 +52152 POINT(39.71408660685729 74.04459312439259) bank52152 +52153 POINT(41.549621698756 73.15899724906087) bank52153 +52154 POINT(41.46887640754114 74.97923235021229) bank52154 +52155 POINT(40.03595451647346 74.75736979246875) bank52155 +52156 POINT(41.116109494424656 73.11452541846339) bank52156 +52157 POINT(41.62771704753011 73.17329775689538) bank52157 +52158 POINT(40.24907590169934 73.43335078400877) bank52158 +52159 POINT(40.407419984652535 73.77852276227965) bank52159 +52160 POINT(39.761946396857134 73.23824205808528) bank52160 +52161 POINT(40.68030428623288 73.46464974234569) bank52161 +52162 POINT(40.76638032167677 74.79517535238557) bank52162 +52163 POINT(40.20507615720879 73.37805272034717) bank52163 +52164 POINT(40.712342343411784 73.59863416267721) bank52164 +52165 POINT(40.306890340281214 74.09279858578736) bank52165 +52166 POINT(39.82980866473494 74.08132120820463) bank52166 +52167 POINT(39.890352741546906 73.59366713088427) bank52167 +52168 POINT(39.79546467875904 73.11054620207538) bank52168 +52169 POINT(41.017191825659395 74.58759328388844) bank52169 +52170 POINT(41.04971078236405 73.97377908304244) bank52170 +52171 POINT(41.30584651483358 74.70665399472213) bank52171 +52172 POINT(40.04933865143218 74.54464769972483) bank52172 +52173 POINT(39.8709332487089 74.24436058369075) bank52173 +52174 POINT(40.11993445014882 73.69048179026477) bank52174 +52175 POINT(41.324572178923916 73.1213572742193) bank52175 +52176 POINT(39.84433770281494 73.94785864293752) bank52176 +52177 POINT(41.27366909446118 73.24377005304716) bank52177 +52178 POINT(40.56207845317654 74.5040412203822) bank52178 +52179 POINT(40.45381880105196 73.10262617197677) bank52179 +52180 POINT(41.37345502649952 74.68147918015289) bank52180 +52181 POINT(41.693688548608364 74.03674440889922) bank52181 +52182 POINT(40.75871994193428 73.13782361316483) bank52182 +52183 POINT(41.13879290383759 73.32461620413864) bank52183 +52184 POINT(41.5944431302748 74.0477029260373) bank52184 +52185 POINT(41.160565948314435 74.60330694899137) bank52185 +52186 POINT(40.27926007440402 73.0691016011504) bank52186 +52187 POINT(40.19864541625571 74.7225351531539) bank52187 +52188 POINT(41.157027677056455 74.5675821227159) bank52188 +52189 POINT(40.90054816169789 73.67819684018235) bank52189 +52190 POINT(41.33586716054755 74.48670162612817) bank52190 +52191 POINT(41.17709519795884 73.18718954770809) bank52191 +52192 POINT(39.81296747326279 74.20407447815124) bank52192 +52193 POINT(40.255855023221386 74.03022983773637) bank52193 +52194 POINT(40.40969591206256 74.80713922226556) bank52194 +52195 POINT(40.45103722940354 73.8252835966405) bank52195 +52196 POINT(40.67111468482306 73.12441088002407) bank52196 +52197 POINT(40.493154978161165 74.88343543704549) bank52197 +52198 POINT(40.39583202310394 74.4405796111018) bank52198 +52199 POINT(40.32781396687134 74.21116285336207) bank52199 +52200 POINT(40.898159531742074 73.68775962596708) bank52200 +52201 POINT(39.78330657640384 73.5810586235133) bank52201 +52202 POINT(39.89892521872427 73.55470661686773) bank52202 +52203 POINT(40.85393378529711 73.49251443125213) bank52203 +52204 POINT(40.10310823115954 73.31736142714311) bank52204 +52205 POINT(40.88718647459303 74.9424945927137) bank52205 +52206 POINT(39.80448731620204 73.45821448497964) bank52206 +52207 POINT(40.741875605986806 74.95712435727397) bank52207 +52208 POINT(39.85931437927884 73.78532632047283) bank52208 +52209 POINT(41.287739303992204 74.94678326521561) bank52209 +52210 POINT(40.32076496520602 74.47612218388444) bank52210 +52211 POINT(40.77841065728371 73.59982571568722) bank52211 +52212 POINT(40.092602891510964 74.63416107975635) bank52212 +52213 POINT(40.206627175558864 73.82126322546097) bank52213 +52214 POINT(41.32252202608032 73.09871949120685) bank52214 +52215 POINT(40.6678886901549 74.20197367075404) bank52215 +52216 POINT(39.76538103436362 74.34358927192113) bank52216 +52217 POINT(40.37536041702243 73.76923807876493) bank52217 +52218 POINT(41.144320899660414 73.67219631538948) bank52218 +52219 POINT(39.73136463366836 73.3295793866587) bank52219 +52220 POINT(39.76536525794822 73.84018549501613) bank52220 +52221 POINT(40.046678722333 74.32337616486937) bank52221 +52222 POINT(41.67212869108236 74.8053740144476) bank52222 +52223 POINT(40.60961522427755 74.54867815129283) bank52223 +52224 POINT(41.57269193151359 73.15996370305002) bank52224 +52225 POINT(39.747022164721095 73.82768761116796) bank52225 +52226 POINT(41.36200245393587 74.33598604133991) bank52226 +52227 POINT(39.890981687631175 74.86331894067015) bank52227 +52228 POINT(39.94727219390136 73.8156094363507) bank52228 +52229 POINT(41.51783855998944 74.84312568784208) bank52229 +52230 POINT(41.35807070753795 73.95168056838874) bank52230 +52231 POINT(40.39177322630185 74.43516197607755) bank52231 +52232 POINT(39.957157456259644 74.66816069472584) bank52232 +52233 POINT(40.58789593665626 74.4497981022496) bank52233 +52234 POINT(41.38655047872354 74.0981595526125) bank52234 +52235 POINT(40.76121720544553 73.06762202344983) bank52235 +52236 POINT(40.61557398583881 74.74879256495782) bank52236 +52237 POINT(39.88936206931676 73.50366903618207) bank52237 +52238 POINT(40.02587127661865 73.86619816367089) bank52238 +52239 POINT(41.62244629078385 74.53033105686389) bank52239 +52240 POINT(40.07351620866819 74.99193869324166) bank52240 +52241 POINT(41.649385205423954 74.05863986914098) bank52241 +52242 POINT(41.664370710075445 74.41204582794589) bank52242 +52243 POINT(40.3087512032614 74.81833452873116) bank52243 +52244 POINT(40.68358190912224 73.50887492578022) bank52244 +52245 POINT(41.30923177351276 74.84384573242708) bank52245 +52246 POINT(41.44638583975 74.2858566529057) bank52246 +52247 POINT(41.419127475900936 73.38034041028249) bank52247 +52248 POINT(41.015559271199294 73.19372678711613) bank52248 +52249 POINT(41.11556265943924 74.25269235624859) bank52249 +52250 POINT(41.196169165126584 73.32973805756825) bank52250 +52251 POINT(41.223039960905375 73.34700116818384) bank52251 +52252 POINT(40.936553222646474 73.6899952786856) bank52252 +52253 POINT(41.333695495265594 73.63185431758079) bank52253 +52254 POINT(41.07810381158245 73.65548103516338) bank52254 +52255 POINT(40.22839037778164 74.56922970941423) bank52255 +52256 POINT(40.65982222949806 74.06337247513788) bank52256 +52257 POINT(40.86602978602068 74.5979518087296) bank52257 +52258 POINT(40.943118748867086 73.85898876025728) bank52258 +52259 POINT(40.47383595211427 74.40526778409578) bank52259 +52260 POINT(41.43328781671218 74.22809213463498) bank52260 +52261 POINT(40.042383385717194 73.53031402061148) bank52261 +52262 POINT(40.04510264165464 73.26099843326485) bank52262 +52263 POINT(41.304837372999316 73.31231042488292) bank52263 +52264 POINT(40.44468263365662 73.18856049881171) bank52264 +52265 POINT(40.995952700539114 73.56388828378202) bank52265 +52266 POINT(40.180083192015985 74.62304872863466) bank52266 +52267 POINT(40.79499879749758 74.33576874144087) bank52267 +52268 POINT(41.36561729534354 74.46680187073218) bank52268 +52269 POINT(40.13988392794801 74.19176035468492) bank52269 +52270 POINT(41.703364793309305 73.96054960538393) bank52270 +52271 POINT(41.5777593440862 74.83792625393514) bank52271 +52272 POINT(40.56558627460459 74.09697702428753) bank52272 +52273 POINT(39.87898208108105 74.88282654481192) bank52273 +52274 POINT(41.33332171865984 74.50102662869566) bank52274 +52275 POINT(40.96027099152677 74.04890283738422) bank52275 +52276 POINT(40.79674619357241 74.18598055237959) bank52276 +52277 POINT(40.598652076043166 74.12634515835065) bank52277 +52278 POINT(40.505476863527875 73.20019610237402) bank52278 +52279 POINT(39.769762372412075 74.49635287175704) bank52279 +52280 POINT(40.20386025212128 73.86045669794791) bank52280 +52281 POINT(40.67986940215675 74.7766590654806) bank52281 +52282 POINT(41.05178741545557 73.63123796345913) bank52282 +52283 POINT(40.79581460115597 73.49716000956445) bank52283 +52284 POINT(40.201687601103124 73.97221577364928) bank52284 +52285 POINT(41.66946665936601 74.62913107418741) bank52285 +52286 POINT(41.61940007609356 74.99211080553648) bank52286 +52287 POINT(40.14869319627827 74.17161480878235) bank52287 +52288 POINT(39.86076537952861 74.19109177449829) bank52288 +52289 POINT(39.837415239738114 74.9017584336077) bank52289 +52290 POINT(41.31016014044695 74.86458377063866) bank52290 +52291 POINT(41.32841803253212 74.36279285579246) bank52291 +52292 POINT(41.682223549393804 74.58792907619011) bank52292 +52293 POINT(40.49187536406484 73.11740461064211) bank52293 +52294 POINT(39.879785340553354 74.43863580729229) bank52294 +52295 POINT(39.9503300553533 74.31361090068712) bank52295 +52296 POINT(39.87575195679897 74.56775834429081) bank52296 +52297 POINT(40.22271612724323 74.79332513292637) bank52297 +52298 POINT(41.132993177220506 74.4215402361661) bank52298 +52299 POINT(41.64945611463345 74.7286816752504) bank52299 +52300 POINT(41.13004728006418 74.11523946065194) bank52300 +52301 POINT(41.631420907092554 74.63462342692743) bank52301 +52302 POINT(39.965444315593125 74.95692068801844) bank52302 +52303 POINT(40.49214341571245 74.86009062550525) bank52303 +52304 POINT(40.084685021738494 73.2127484186173) bank52304 +52305 POINT(41.336488552462114 73.6860331182215) bank52305 +52306 POINT(41.444911106001115 73.08517316115773) bank52306 +52307 POINT(41.24859475798795 74.90164370615227) bank52307 +52308 POINT(40.96784732531201 73.97963818778699) bank52308 +52309 POINT(40.36458922379027 74.37482656724679) bank52309 +52310 POINT(40.381740755600966 73.34726021673909) bank52310 +52311 POINT(40.15035915557214 73.96808367012956) bank52311 +52312 POINT(39.86240625185085 73.65624364993593) bank52312 +52313 POINT(40.932339714945044 74.4960921870826) bank52313 +52314 POINT(40.362478144320626 74.221212342793) bank52314 +52315 POINT(40.77605232316647 74.02243071519145) bank52315 +52316 POINT(40.896553939112614 73.34362817675016) bank52316 +52317 POINT(41.57186203042805 74.74451017998884) bank52317 +52318 POINT(41.037112873709276 74.90619940798415) bank52318 +52319 POINT(40.03982520315502 74.42109876413085) bank52319 +52320 POINT(39.929784236672695 74.39481445741905) bank52320 +52321 POINT(40.60358897036772 73.72770497151765) bank52321 +52322 POINT(41.389259536697374 73.90101110890198) bank52322 +52323 POINT(40.35488315839288 73.36422738715558) bank52323 +52324 POINT(41.2059928388908 73.14691882771534) bank52324 +52325 POINT(41.41305266118305 74.43622808952823) bank52325 +52326 POINT(40.90068627148031 73.25331304999013) bank52326 +52327 POINT(39.812907004434585 74.32267594538908) bank52327 +52328 POINT(41.46129552136488 73.67700156917519) bank52328 +52329 POINT(41.20038139715886 73.61128440584021) bank52329 +52330 POINT(40.556761612251016 74.90059040238118) bank52330 +52331 POINT(40.81749876194238 74.82198602685268) bank52331 +52332 POINT(40.38387843656924 74.88499672107493) bank52332 +52333 POINT(41.3897606852886 73.33797048953873) bank52333 +52334 POINT(40.76880920108943 73.91005082124927) bank52334 +52335 POINT(40.034471411217204 73.1762727106629) bank52335 +52336 POINT(40.36621808661954 74.9340635121583) bank52336 +52337 POINT(40.29302570743846 73.4399319980807) bank52337 +52338 POINT(41.09086011247656 74.83402317695733) bank52338 +52339 POINT(40.823766641714144 73.50492237871678) bank52339 +52340 POINT(41.3903695288697 74.55241184501256) bank52340 +52341 POINT(41.412263510643115 73.3562619376033) bank52341 +52342 POINT(41.34534565255485 73.97797471577036) bank52342 +52343 POINT(40.903040304258525 74.69403426858105) bank52343 +52344 POINT(39.827555678651734 73.23964011677724) bank52344 +52345 POINT(40.58025334128697 74.91680577714286) bank52345 +52346 POINT(41.179693612638566 73.73224317765752) bank52346 +52347 POINT(40.28092513496042 74.34179898850084) bank52347 +52348 POINT(40.935700887470006 73.59736671666295) bank52348 +52349 POINT(41.14490756537345 74.09760908736169) bank52349 +52350 POINT(40.09439740848055 73.80320377864147) bank52350 +52351 POINT(39.782653981833995 74.13914604700265) bank52351 +52352 POINT(41.528637898329734 73.28298126222737) bank52352 +52353 POINT(40.76915373589916 74.79095917859273) bank52353 +52354 POINT(40.8008020573639 73.93826835755996) bank52354 +52355 POINT(40.86296926410162 73.19929375902996) bank52355 +52356 POINT(40.85423245748195 73.93192252155822) bank52356 +52357 POINT(41.07302967530044 73.3365914276098) bank52357 +52358 POINT(41.1602333648925 73.05312611409398) bank52358 +52359 POINT(40.649249033980745 73.70337840860779) bank52359 +52360 POINT(40.24623397759293 74.92647795504527) bank52360 +52361 POINT(40.525677551092556 73.72366407543602) bank52361 +52362 POINT(40.40975197609702 74.60796324044018) bank52362 +52363 POINT(39.8975466115564 74.36515373864191) bank52363 +52364 POINT(41.03154814566217 74.45687553882951) bank52364 +52365 POINT(41.687661120656664 74.26400935332357) bank52365 +52366 POINT(39.73988219464251 74.09991808990637) bank52366 +52367 POINT(40.30246346947486 73.89424828792099) bank52367 +52368 POINT(41.49155233937286 73.6286398135375) bank52368 +52369 POINT(39.82357041206641 74.3134288262416) bank52369 +52370 POINT(41.57128618563454 74.40790315888427) bank52370 +52371 POINT(40.245452680833225 73.21925760821854) bank52371 +52372 POINT(40.468768705429596 73.11696618062055) bank52372 +52373 POINT(40.55706136133399 74.9116083863821) bank52373 +52374 POINT(41.26767099146977 73.397295979459) bank52374 +52375 POINT(41.051870799714415 73.73454325077671) bank52375 +52376 POINT(40.87428984633239 73.68951430914365) bank52376 +52377 POINT(40.85960933218657 73.64385725637102) bank52377 +52378 POINT(41.098360086990866 74.21214700279266) bank52378 +52379 POINT(40.15007800683181 73.0313306260233) bank52379 +52380 POINT(40.19018446103915 74.63448739508539) bank52380 +52381 POINT(40.49167105205884 73.71481623446864) bank52381 +52382 POINT(40.38257269711573 74.88269403550859) bank52382 +52383 POINT(40.60834410303599 74.37947760915324) bank52383 +52384 POINT(41.33825315977399 73.48976985223122) bank52384 +52385 POINT(41.563788916261416 74.45209777076846) bank52385 +52386 POINT(41.281069532849294 74.53323670979091) bank52386 +52387 POINT(40.33674186332145 74.0354574751929) bank52387 +52388 POINT(41.087924910765025 73.42118840581506) bank52388 +52389 POINT(39.953347472593144 74.43021616923332) bank52389 +52390 POINT(40.16310342393688 73.59905650544675) bank52390 +52391 POINT(40.44730382518155 74.36101903428033) bank52391 +52392 POINT(41.03678375007132 73.09940591114685) bank52392 +52393 POINT(40.804199463189306 73.11105192978219) bank52393 +52394 POINT(40.34817465828068 73.248555534238) bank52394 +52395 POINT(40.40303519361107 73.67843175841635) bank52395 +52396 POINT(40.968522285272265 74.84772674583789) bank52396 +52397 POINT(40.1781847702847 74.26792207604055) bank52397 +52398 POINT(41.427163280087655 73.6962846348556) bank52398 +52399 POINT(41.272060695165266 73.8752511613038) bank52399 +52400 POINT(41.70991380098679 74.70964767723069) bank52400 +52401 POINT(41.12800255323395 74.5434528504578) bank52401 +52402 POINT(40.73409340866689 74.63110462931628) bank52402 +52403 POINT(41.30529324144004 73.58875903375177) bank52403 +52404 POINT(39.906746511119096 74.98657089960118) bank52404 +52405 POINT(41.57837896223786 73.91468521232676) bank52405 +52406 POINT(40.94334134134107 74.50252228340362) bank52406 +52407 POINT(39.87101744747758 73.9089846498642) bank52407 +52408 POINT(41.04941231072302 73.76418100862168) bank52408 +52409 POINT(41.43327397878187 74.82657010614538) bank52409 +52410 POINT(41.470812669243024 74.2965613211272) bank52410 +52411 POINT(40.83597755988328 74.206497222144) bank52411 +52412 POINT(39.73343198358699 73.65485081187826) bank52412 +52413 POINT(40.603056235082356 73.15461226646953) bank52413 +52414 POINT(40.063876443427965 73.23675833049833) bank52414 +52415 POINT(40.986827315803936 74.31068563231244) bank52415 +52416 POINT(41.380647701779786 73.86606451636175) bank52416 +52417 POINT(41.58635495819923 73.86421538989856) bank52417 +52418 POINT(41.51888113669423 74.91884166336803) bank52418 +52419 POINT(39.73000024138649 74.42965549376801) bank52419 +52420 POINT(41.67570472448983 74.1122049547064) bank52420 +52421 POINT(39.86828250720951 73.1990010097448) bank52421 +52422 POINT(41.43696682592002 74.46027891507701) bank52422 +52423 POINT(39.88303825647955 74.55495528427689) bank52423 +52424 POINT(41.07251326620137 74.03805587653028) bank52424 +52425 POINT(41.70319093923974 73.49755125426232) bank52425 +52426 POINT(40.696092736611156 74.74990791195239) bank52426 +52427 POINT(40.97005121315183 74.40281849419753) bank52427 +52428 POINT(40.516940289364925 73.30710089120998) bank52428 +52429 POINT(40.23990326524249 73.12394435863078) bank52429 +52430 POINT(41.38626733395128 73.0452720055221) bank52430 +52431 POINT(40.78962329132649 74.31486852053085) bank52431 +52432 POINT(40.46249768625141 73.07306910294068) bank52432 +52433 POINT(39.98477056797544 74.80306784468527) bank52433 +52434 POINT(41.464762319742 73.33655033540396) bank52434 +52435 POINT(40.65997969284569 73.42085966283658) bank52435 +52436 POINT(40.37478558466068 74.10481882609292) bank52436 +52437 POINT(40.55091911552176 74.93121177562455) bank52437 +52438 POINT(41.041064018948795 73.9217739224586) bank52438 +52439 POINT(39.90577711535365 73.49721447640721) bank52439 +52440 POINT(40.160992525981726 73.13574055601273) bank52440 +52441 POINT(40.72149593225372 73.64941757377179) bank52441 +52442 POINT(41.37016667395133 74.98188468658694) bank52442 +52443 POINT(40.036306400771075 73.30496391265078) bank52443 +52444 POINT(40.279365011218545 73.03068807332225) bank52444 +52445 POINT(40.42652022207737 73.92434093388071) bank52445 +52446 POINT(41.610799489553976 74.04696562775192) bank52446 +52447 POINT(41.287949618830204 74.36333106794153) bank52447 +52448 POINT(40.38675517112114 73.65944044116507) bank52448 +52449 POINT(39.84829088844797 75.00095858319762) bank52449 +52450 POINT(39.7714520438859 73.34041586490675) bank52450 +52451 POINT(40.093292755235304 73.44572633609384) bank52451 +52452 POINT(41.46041174915226 73.90537483761153) bank52452 +52453 POINT(41.377892551685385 73.64539042086011) bank52453 +52454 POINT(41.067940362301414 73.70020279494281) bank52454 +52455 POINT(40.40143388674266 73.34521135918587) bank52455 +52456 POINT(39.73182705793697 74.42914760914185) bank52456 +52457 POINT(39.730806769279084 75.00466038567686) bank52457 +52458 POINT(40.32365811563467 74.25313349274394) bank52458 +52459 POINT(41.3357970130141 73.3618072640542) bank52459 +52460 POINT(39.92161663047022 74.38433762187115) bank52460 +52461 POINT(41.090715716583674 73.08534001343787) bank52461 +52462 POINT(41.648856083102984 73.84544195451602) bank52462 +52463 POINT(40.465164065852065 74.95067340510663) bank52463 +52464 POINT(40.58635769764755 74.7832822444181) bank52464 +52465 POINT(40.39164892033343 73.71478035259247) bank52465 +52466 POINT(40.43244961657941 73.37799315613056) bank52466 +52467 POINT(40.99415558360529 73.98999270257292) bank52467 +52468 POINT(40.3549516347484 74.03552230816263) bank52468 +52469 POINT(39.85095623121578 73.57740809104958) bank52469 +52470 POINT(40.37490774770483 74.93278227714953) bank52470 +52471 POINT(40.54608322320441 73.3342801521148) bank52471 +52472 POINT(40.70789367347053 74.7450645326442) bank52472 +52473 POINT(41.25127684639097 74.3584093747072) bank52473 +52474 POINT(40.00389732891821 74.82114957845398) bank52474 +52475 POINT(40.4297391521166 73.40323033452398) bank52475 +52476 POINT(41.374634061677824 73.20673821637098) bank52476 +52477 POINT(39.79464394589984 73.13180120500748) bank52477 +52478 POINT(41.58636939426925 74.58249680658297) bank52478 +52479 POINT(40.7296460854545 73.94229000636332) bank52479 +52480 POINT(40.13856970238084 73.05853039636034) bank52480 +52481 POINT(40.82255864492726 73.82152015239056) bank52481 +52482 POINT(41.04712232490359 73.38955319460268) bank52482 +52483 POINT(41.482857709979676 73.6179976776647) bank52483 +52484 POINT(40.81534047221393 74.99586613456495) bank52484 +52485 POINT(41.37156040721164 73.05153672185519) bank52485 +52486 POINT(41.671610274174746 74.83095911552019) bank52486 +52487 POINT(39.93809931708571 74.98414967833428) bank52487 +52488 POINT(41.46179223551676 74.72697236702452) bank52488 +52489 POINT(41.36279498686082 73.32642142578491) bank52489 +52490 POINT(41.04664667137195 73.16060080045985) bank52490 +52491 POINT(40.90891853913993 74.9445772679121) bank52491 +52492 POINT(39.98204672597524 73.79827660365413) bank52492 +52493 POINT(40.16791050938888 73.9999142989989) bank52493 +52494 POINT(41.605154247875475 73.69708486645958) bank52494 +52495 POINT(40.11202053321037 74.09094809154858) bank52495 +52496 POINT(40.11992228188199 73.98979312875136) bank52496 +52497 POINT(40.04679433377174 74.96878833849871) bank52497 +52498 POINT(39.966503475607425 74.13335379137868) bank52498 +52499 POINT(41.26091820234414 74.76257782715575) bank52499 +52500 POINT(40.438489308473976 73.37988705516472) bank52500 +52501 POINT(39.737545869311575 74.18995905790652) bank52501 +52502 POINT(40.35889903513836 73.92694521737864) bank52502 +52503 POINT(40.63792593051602 74.61284189087326) bank52503 +52504 POINT(40.483289603185 74.27535447947015) bank52504 +52505 POINT(39.95548668903738 74.54943933663777) bank52505 +52506 POINT(41.2665084480453 73.82802663231233) bank52506 +52507 POINT(41.16615296556028 74.85323769220439) bank52507 +52508 POINT(39.759310546934145 73.25433907227685) bank52508 +52509 POINT(41.556510863772196 74.9078625954759) bank52509 +52510 POINT(41.544096339303415 74.23936511623032) bank52510 +52511 POINT(41.20773670511783 73.47343983378481) bank52511 +52512 POINT(41.113264571531715 74.30053424967903) bank52512 +52513 POINT(40.62467039065692 74.06164416283664) bank52513 +52514 POINT(40.49126908501977 73.0856916018691) bank52514 +52515 POINT(40.82167947137144 73.07326479451982) bank52515 +52516 POINT(39.985146172486445 73.46778822393311) bank52516 +52517 POINT(40.335099423504666 74.41152835943947) bank52517 +52518 POINT(41.346050049461226 74.70765678539956) bank52518 +52519 POINT(40.79002405675089 74.4728583723456) bank52519 +52520 POINT(41.398481593394024 74.54648110349724) bank52520 +52521 POINT(40.07400348438128 73.04625444040774) bank52521 +52522 POINT(40.41504717578637 74.31666360305613) bank52522 +52523 POINT(39.758271993151375 73.23366864762465) bank52523 +52524 POINT(40.152452669837174 74.81790941286968) bank52524 +52525 POINT(41.42231919398015 73.02465447764433) bank52525 +52526 POINT(40.354596090688354 74.02490633783418) bank52526 +52527 POINT(41.012847695591255 74.42053154861767) bank52527 +52528 POINT(41.01991429457507 74.40327732082854) bank52528 +52529 POINT(40.035810082940365 74.50798541881558) bank52529 +52530 POINT(40.33898235259378 73.04120281964504) bank52530 +52531 POINT(41.15926092396426 74.28578335185405) bank52531 +52532 POINT(41.19782529931619 73.74491260289722) bank52532 +52533 POINT(40.25615370138333 74.13118146309635) bank52533 +52534 POINT(39.73583456336745 73.04016078055133) bank52534 +52535 POINT(40.56414143030399 74.61150023195324) bank52535 +52536 POINT(40.54328757312222 74.08278633750399) bank52536 +52537 POINT(40.31470763419887 74.73676959880892) bank52537 +52538 POINT(41.106484943071784 74.86839885957404) bank52538 +52539 POINT(41.209419423255795 74.43713562468977) bank52539 +52540 POINT(41.095143045324924 74.94352956794127) bank52540 +52541 POINT(39.721282061508056 73.3139705768971) bank52541 +52542 POINT(41.678844910418476 73.95641335984278) bank52542 +52543 POINT(40.80200532244327 73.65821765175463) bank52543 +52544 POINT(40.170578069431656 73.56718492293687) bank52544 +52545 POINT(40.594627998974744 74.05900743359297) bank52545 +52546 POINT(40.59590247276714 73.29704187599637) bank52546 +52547 POINT(40.0966577133309 73.67553978115777) bank52547 +52548 POINT(40.374712177322714 73.79605759398042) bank52548 +52549 POINT(40.97278742705848 73.07519334171948) bank52549 +52550 POINT(41.116424823421 73.41956589939907) bank52550 +52551 POINT(41.304309115157174 74.24874925385235) bank52551 +52552 POINT(41.17652796718493 73.4986209419117) bank52552 +52553 POINT(39.803023812727304 74.70738932579566) bank52553 +52554 POINT(40.463078466574075 73.2406374549875) bank52554 +52555 POINT(41.24428890545477 73.8390308130174) bank52555 +52556 POINT(40.13444316688148 74.8687082239477) bank52556 +52557 POINT(40.61796886925393 74.9306553825315) bank52557 +52558 POINT(41.44719366193792 73.45400651716439) bank52558 +52559 POINT(41.10888017808675 73.44705519267386) bank52559 +52560 POINT(40.16541062536701 73.10493973878722) bank52560 +52561 POINT(39.95466914371911 73.68325037336031) bank52561 +52562 POINT(41.07352629636357 73.82267140351274) bank52562 +52563 POINT(40.36603863962944 74.16827356074039) bank52563 +52564 POINT(40.39027318391886 74.66207427647092) bank52564 +52565 POINT(39.86106999135737 73.42185898405594) bank52565 +52566 POINT(41.03439000493487 74.61397803007799) bank52566 +52567 POINT(41.42322825452496 73.13573962181505) bank52567 +52568 POINT(40.345775871444744 74.37950591120695) bank52568 +52569 POINT(41.55400783960256 74.52769096514592) bank52569 +52570 POINT(39.8939827964455 74.45723360266051) bank52570 +52571 POINT(41.29625874595848 73.47568028285421) bank52571 +52572 POINT(41.06480397970536 73.74552846613173) bank52572 +52573 POINT(40.597303268169846 74.348247066443) bank52573 +52574 POINT(39.72715102985238 74.10704051997875) bank52574 +52575 POINT(40.935273027800136 74.69331603183336) bank52575 +52576 POINT(40.014710871100476 74.66986516389599) bank52576 +52577 POINT(40.86577669731266 73.7007722103824) bank52577 +52578 POINT(41.29800201787214 74.53345444325075) bank52578 +52579 POINT(40.65638644172546 74.99778929774115) bank52579 +52580 POINT(40.40834437603149 74.18341487476765) bank52580 +52581 POINT(39.99438574990071 73.04259667732696) bank52581 +52582 POINT(41.67392125059669 73.73003644277186) bank52582 +52583 POINT(41.580696058988195 73.73925313281906) bank52583 +52584 POINT(40.94868601268278 74.32345874347494) bank52584 +52585 POINT(40.953951541164315 73.10052467253134) bank52585 +52586 POINT(39.73186773204253 74.24515135680406) bank52586 +52587 POINT(40.7859958409377 73.63300895944498) bank52587 +52588 POINT(40.42176513185059 73.0257330337075) bank52588 +52589 POINT(41.503763585566844 73.1940677472602) bank52589 +52590 POINT(39.887841862126095 74.51581140318073) bank52590 +52591 POINT(39.798484670750355 73.25441691493815) bank52591 +52592 POINT(39.76499095699694 74.0626235610856) bank52592 +52593 POINT(41.62624039773458 74.74717372537486) bank52593 +52594 POINT(40.90431557256001 73.23750473405018) bank52594 +52595 POINT(41.146839408707166 73.99509109102975) bank52595 +52596 POINT(41.64194840176578 74.53065901827394) bank52596 +52597 POINT(40.78950558130972 74.69074906544238) bank52597 +52598 POINT(39.77193237391953 73.581749556964) bank52598 +52599 POINT(40.16789593638346 74.26806677248457) bank52599 +52600 POINT(41.48443940262457 74.28042096594798) bank52600 +52601 POINT(40.91426485246539 73.4349817620604) bank52601 +52602 POINT(41.487360792763084 73.34302946039857) bank52602 +52603 POINT(41.06917544322886 73.06709200802216) bank52603 +52604 POINT(41.71266879606443 74.18423042825916) bank52604 +52605 POINT(39.93653999448086 74.01763352132647) bank52605 +52606 POINT(40.82143832902346 74.24176180451462) bank52606 +52607 POINT(41.429690657005416 74.18042333470524) bank52607 +52608 POINT(39.84901740276773 73.39812949265954) bank52608 +52609 POINT(40.774189614283436 73.5591144806685) bank52609 +52610 POINT(39.91479417646301 73.46880184457945) bank52610 +52611 POINT(41.56003680168384 74.36042306531407) bank52611 +52612 POINT(41.26593217160506 73.16792747632861) bank52612 +52613 POINT(41.534416665931424 74.12593868585618) bank52613 +52614 POINT(40.17224953423188 74.75520079376025) bank52614 +52615 POINT(39.88617749595304 73.76489571089857) bank52615 +52616 POINT(39.83556329225463 74.84288139342829) bank52616 +52617 POINT(41.08957496679997 73.9078862357711) bank52617 +52618 POINT(39.75759317878532 73.28649245793059) bank52618 +52619 POINT(41.41306871313285 74.51789227963395) bank52619 +52620 POINT(40.73986228525281 73.7665495669186) bank52620 +52621 POINT(40.240053156030676 74.00241332653795) bank52621 +52622 POINT(40.316281242632144 73.11861748163305) bank52622 +52623 POINT(41.4847514326099 74.72887667438545) bank52623 +52624 POINT(40.64938504759316 73.16174147348728) bank52624 +52625 POINT(40.58704474088157 74.91648037942053) bank52625 +52626 POINT(40.99111557817999 73.08028917651538) bank52626 +52627 POINT(40.476141062287255 74.32495149201524) bank52627 +52628 POINT(41.08957534247106 74.24230263658174) bank52628 +52629 POINT(39.741435024663225 73.02088404320163) bank52629 +52630 POINT(41.46648281239016 73.89590464225012) bank52630 +52631 POINT(40.645931955919266 73.76985205874297) bank52631 +52632 POINT(40.192212362039314 74.32807139327808) bank52632 +52633 POINT(40.50652779168067 74.08590003927512) bank52633 +52634 POINT(40.680976779428704 74.75928736303061) bank52634 +52635 POINT(40.29910573008424 73.12220414077372) bank52635 +52636 POINT(40.64362594893035 73.73008908377187) bank52636 +52637 POINT(41.13610767636693 73.34224799346745) bank52637 +52638 POINT(40.185073611850754 74.67931084899394) bank52638 +52639 POINT(39.892486014579305 74.93887982136708) bank52639 +52640 POINT(41.30231696155139 73.67950904676776) bank52640 +52641 POINT(41.57570426437274 73.15312527912782) bank52641 +52642 POINT(40.65808848274681 74.89274612412103) bank52642 +52643 POINT(41.1876021289602 74.04817423627675) bank52643 +52644 POINT(40.595258806046274 73.18731646114453) bank52644 +52645 POINT(40.34343818340444 73.90184035557165) bank52645 +52646 POINT(40.826224830958324 74.34534079918419) bank52646 +52647 POINT(40.39947814155445 73.83639094445591) bank52647 +52648 POINT(41.15086953381984 73.12014920817218) bank52648 +52649 POINT(40.011838006921366 73.925081842415) bank52649 +52650 POINT(40.97632183484312 74.44326338086145) bank52650 +52651 POINT(40.96984812737655 74.58107151100255) bank52651 +52652 POINT(39.78148916372552 74.3650441006299) bank52652 +52653 POINT(41.39805450508128 74.23380673944375) bank52653 +52654 POINT(40.446127066008756 74.93728156618246) bank52654 +52655 POINT(41.24507511001976 73.83122523942977) bank52655 +52656 POINT(41.63924167515677 73.72749036797552) bank52656 +52657 POINT(39.92928785772627 73.16592571048952) bank52657 +52658 POINT(39.920200396301695 74.01527871667747) bank52658 +52659 POINT(39.721217395917584 73.35051432594476) bank52659 +52660 POINT(39.71813033248401 74.50755196974758) bank52660 +52661 POINT(41.632467892288574 74.8212659562953) bank52661 +52662 POINT(40.107863898243 74.12965397296335) bank52662 +52663 POINT(40.173622282064585 74.00657638866217) bank52663 +52664 POINT(41.091235538361914 73.88060356048483) bank52664 +52665 POINT(41.422916542460406 74.77112237620315) bank52665 +52666 POINT(40.40502367312333 74.80058335231577) bank52666 +52667 POINT(39.99486471542082 74.24817845332151) bank52667 +52668 POINT(40.07702364778164 74.10880849386463) bank52668 +52669 POINT(41.59977143878935 74.47750798383738) bank52669 +52670 POINT(41.64019607001448 73.99575297393119) bank52670 +52671 POINT(40.763130509781895 73.87034293383701) bank52671 +52672 POINT(40.739859932059126 74.7818430417474) bank52672 +52673 POINT(41.598718681816564 74.41514712108045) bank52673 +52674 POINT(39.89529676966382 73.1256410396119) bank52674 +52675 POINT(40.72692669559241 74.7324708533587) bank52675 +52676 POINT(39.79174104184062 73.4863021481188) bank52676 +52677 POINT(40.0654719527127 74.93269859080404) bank52677 +52678 POINT(40.630661830425105 74.01165859098616) bank52678 +52679 POINT(41.280059639119095 74.69167890557179) bank52679 +52680 POINT(39.84023297551476 73.57266864254538) bank52680 +52681 POINT(40.40410566522571 74.85781688495187) bank52681 +52682 POINT(41.09511294988104 73.8627958777108) bank52682 +52683 POINT(39.95004058347361 73.32196713910908) bank52683 +52684 POINT(40.72931280097549 73.13297699330874) bank52684 +52685 POINT(40.73213785884299 74.87153183295766) bank52685 +52686 POINT(41.452547881112416 73.37670656541317) bank52686 +52687 POINT(41.14626284489949 73.52121103301172) bank52687 +52688 POINT(40.14708310308417 73.75586192721829) bank52688 +52689 POINT(39.86484218695168 74.71988660436058) bank52689 +52690 POINT(39.81570447334992 74.84348159041026) bank52690 +52691 POINT(40.6786511074503 74.90839833204457) bank52691 +52692 POINT(39.98045669723003 74.05303660966429) bank52692 +52693 POINT(39.852876369782905 74.81437142256704) bank52693 +52694 POINT(40.09756592364547 74.03784155853253) bank52694 +52695 POINT(40.71199039926216 73.63181959604536) bank52695 +52696 POINT(41.02244945047662 73.44450729527922) bank52696 +52697 POINT(40.848501464563476 74.71191029056753) bank52697 +52698 POINT(41.213363719227274 74.40354633529506) bank52698 +52699 POINT(41.616130229568164 74.36219328968953) bank52699 +52700 POINT(40.92026259218826 74.59835297927103) bank52700 +52701 POINT(39.86369312511382 73.47877830059696) bank52701 +52702 POINT(40.88609802274822 74.56151279100453) bank52702 +52703 POINT(40.8806433242402 73.4605773469945) bank52703 +52704 POINT(40.736547807440715 73.27982771288264) bank52704 +52705 POINT(40.60420120210285 73.0139283383115) bank52705 +52706 POINT(40.36664883671777 73.78474309448171) bank52706 +52707 POINT(41.04461837759168 74.08311573115367) bank52707 +52708 POINT(39.87929081012475 73.60873794134213) bank52708 +52709 POINT(40.82254360461124 73.75814310549846) bank52709 +52710 POINT(41.29951085282519 74.08686762541663) bank52710 +52711 POINT(40.730904015311474 74.4762563791244) bank52711 +52712 POINT(40.15614152528011 73.68683425818328) bank52712 +52713 POINT(40.38682123641965 74.82518910356855) bank52713 +52714 POINT(41.32472883014318 73.91997104201882) bank52714 +52715 POINT(41.22778775860987 74.91535639905686) bank52715 +52716 POINT(39.937348933850956 74.97027895546373) bank52716 +52717 POINT(40.28015028068491 74.87705391717463) bank52717 +52718 POINT(41.236475127124585 73.75690031649077) bank52718 +52719 POINT(41.40203033849509 74.49683180520326) bank52719 +52720 POINT(41.13807394538788 74.02798295063818) bank52720 +52721 POINT(41.69988678725618 73.65233581922794) bank52721 +52722 POINT(40.440334575857015 73.96228992209247) bank52722 +52723 POINT(41.10584880959884 74.95287894514098) bank52723 +52724 POINT(39.90213668345843 74.32675906362967) bank52724 +52725 POINT(41.54577895400391 74.3403534127996) bank52725 +52726 POINT(39.941334015976906 73.17824977621359) bank52726 +52727 POINT(40.13437702731836 74.44363103752012) bank52727 +52728 POINT(41.07365256290837 74.50467048811134) bank52728 +52729 POINT(40.846127449234395 74.10278331530056) bank52729 +52730 POINT(39.96105843080328 74.38181198521822) bank52730 +52731 POINT(40.011979721470006 74.26208797972669) bank52731 +52732 POINT(41.50913670466216 74.27123017405421) bank52732 +52733 POINT(41.56121813696972 73.62457293449278) bank52733 +52734 POINT(40.871098784502166 74.62835246743735) bank52734 +52735 POINT(39.85507740817072 73.48300064343721) bank52735 +52736 POINT(41.26801530112227 74.18718603036065) bank52736 +52737 POINT(40.41936787224765 73.48270418547972) bank52737 +52738 POINT(39.76201920704528 73.90280216806072) bank52738 +52739 POINT(41.303633896221015 73.95502737773552) bank52739 +52740 POINT(40.00999789758799 74.29389766780115) bank52740 +52741 POINT(41.56135260147382 73.33649282997553) bank52741 +52742 POINT(40.63188105050254 73.74510116120616) bank52742 +52743 POINT(41.60803643556128 74.34295721541173) bank52743 +52744 POINT(40.05535416353952 74.99996569504903) bank52744 +52745 POINT(41.69148638821362 74.02908745888402) bank52745 +52746 POINT(40.859785458849814 73.3251378472123) bank52746 +52747 POINT(40.70499874338472 73.50028007493064) bank52747 +52748 POINT(40.99249343999646 73.48828704247217) bank52748 +52749 POINT(40.12293682521753 73.7938713863492) bank52749 +52750 POINT(41.289734819508126 74.68090149187802) bank52750 +52751 POINT(40.51682934714787 73.07650097295875) bank52751 +52752 POINT(39.81211754338527 73.81970850581757) bank52752 +52753 POINT(40.565868566197004 74.87474117797487) bank52753 +52754 POINT(40.543852693356236 74.47596612932496) bank52754 +52755 POINT(40.92556538194346 73.23018717298861) bank52755 +52756 POINT(41.505090682120446 73.80663314649617) bank52756 +52757 POINT(40.910405142604866 73.32523820078173) bank52757 +52758 POINT(40.89712721276207 74.15441664685521) bank52758 +52759 POINT(40.95605720001126 74.43619737194419) bank52759 +52760 POINT(40.54684640969126 74.12385442805557) bank52760 +52761 POINT(41.0767010847041 73.16429416523545) bank52761 +52762 POINT(40.640523906943436 74.77034116550331) bank52762 +52763 POINT(40.248859866177014 73.91293990055111) bank52763 +52764 POINT(41.2336091402645 73.05906729081359) bank52764 +52765 POINT(39.834725230999084 74.824951617832) bank52765 +52766 POINT(40.710770145756655 73.02649612969847) bank52766 +52767 POINT(40.19612066674139 74.17341251460873) bank52767 +52768 POINT(40.269469632684306 74.97636017845966) bank52768 +52769 POINT(40.162643964138816 74.75348470637515) bank52769 +52770 POINT(40.08655682502812 73.26817243061546) bank52770 +52771 POINT(41.23627101689444 75.00427891710274) bank52771 +52772 POINT(40.45680868919291 73.74119796791138) bank52772 +52773 POINT(40.86445948804976 74.43846117620916) bank52773 +52774 POINT(40.02374440178647 74.8437805165895) bank52774 +52775 POINT(39.98304425205489 73.58793990315498) bank52775 +52776 POINT(41.49778237372523 74.38897032739295) bank52776 +52777 POINT(41.25169399432233 74.06525100929824) bank52777 +52778 POINT(41.19035880610452 74.72039632866282) bank52778 +52779 POINT(40.49771955886097 73.56470929689014) bank52779 +52780 POINT(41.49764762332708 73.83833488871814) bank52780 +52781 POINT(39.90197660555087 74.61105310759093) bank52781 +52782 POINT(41.022810561104244 73.09986940590004) bank52782 +52783 POINT(41.24277657109484 73.42900621465573) bank52783 +52784 POINT(41.221089992601534 73.71335756110732) bank52784 +52785 POINT(40.26490792479936 73.61531383850266) bank52785 +52786 POINT(41.19344182055647 74.0057945893101) bank52786 +52787 POINT(39.81409764545291 74.2261696830285) bank52787 +52788 POINT(39.83785429363489 74.27404633779669) bank52788 +52789 POINT(40.136805073310306 74.2316582233101) bank52789 +52790 POINT(39.75285310141145 74.92058571399924) bank52790 +52791 POINT(40.88294671279623 74.16829148176309) bank52791 +52792 POINT(41.43560880084965 74.98705945445688) bank52792 +52793 POINT(40.01335552384534 73.52904007982136) bank52793 +52794 POINT(41.2839943513624 75.000464040185) bank52794 +52795 POINT(40.332316704780396 73.4657189437941) bank52795 +52796 POINT(39.94777011573464 74.70097025713616) bank52796 +52797 POINT(41.70374819107322 73.96570774977691) bank52797 +52798 POINT(40.549870286683564 73.45173833273807) bank52798 +52799 POINT(40.150911669314816 73.26904139784075) bank52799 +52800 POINT(40.68209141639785 73.81342669883071) bank52800 +52801 POINT(40.18605436763852 73.12885112240686) bank52801 +52802 POINT(40.73737943879825 74.11657868496275) bank52802 +52803 POINT(40.788954951487916 74.78155592981018) bank52803 +52804 POINT(40.648055904587366 73.6830818397101) bank52804 +52805 POINT(39.89329052895929 73.8036050122799) bank52805 +52806 POINT(40.50483780460085 73.34625231911525) bank52806 +52807 POINT(40.98019610701439 74.58748999752235) bank52807 +52808 POINT(41.32292443708912 73.52175257843417) bank52808 +52809 POINT(40.57092105206669 73.13446157471279) bank52809 +52810 POINT(40.471468569259706 73.04210869083077) bank52810 +52811 POINT(40.46337298354028 74.86753178412575) bank52811 +52812 POINT(40.540400105476664 73.4939470338982) bank52812 +52813 POINT(39.83068722097312 73.07959580435856) bank52813 +52814 POINT(41.382372388490154 73.31393375217077) bank52814 +52815 POINT(41.06244739026874 74.25147410618123) bank52815 +52816 POINT(40.40698149910392 74.14451830833228) bank52816 +52817 POINT(40.11796986716498 74.12591237102205) bank52817 +52818 POINT(41.37761649891662 73.61991358141415) bank52818 +52819 POINT(41.0656440418408 74.13175464422217) bank52819 +52820 POINT(41.221812823724264 73.34194126328376) bank52820 +52821 POINT(40.29610148699715 74.42381650228701) bank52821 +52822 POINT(40.623723140893944 73.8552875913229) bank52822 +52823 POINT(40.96128342922384 73.47475547874176) bank52823 +52824 POINT(41.46370108280731 74.9389147210329) bank52824 +52825 POINT(40.02911205935101 73.37028968466485) bank52825 +52826 POINT(40.69083040713175 74.80511250013552) bank52826 +52827 POINT(41.42948193207304 73.75848824571509) bank52827 +52828 POINT(41.452786110244794 73.61098994654483) bank52828 +52829 POINT(39.89605444313522 73.43095007117408) bank52829 +52830 POINT(41.43592184772431 74.64975670609593) bank52830 +52831 POINT(41.03725369675252 73.6808927844486) bank52831 +52832 POINT(41.25282041538205 74.7821165153558) bank52832 +52833 POINT(41.519807675389984 74.45912238139915) bank52833 +52834 POINT(39.88103867144663 74.27691267911386) bank52834 +52835 POINT(40.84230190601025 73.35000658843373) bank52835 +52836 POINT(41.05537199216314 74.27382289768478) bank52836 +52837 POINT(41.648744927356965 74.4581131180032) bank52837 +52838 POINT(41.22151472171791 74.5636078051683) bank52838 +52839 POINT(40.179450832068106 73.97355688726225) bank52839 +52840 POINT(40.42295802438719 74.77400877158918) bank52840 +52841 POINT(41.03639349123488 73.84249188714664) bank52841 +52842 POINT(41.48176196801456 73.58404189922884) bank52842 +52843 POINT(40.38693821882583 73.74194365276534) bank52843 +52844 POINT(40.488704081909816 73.32621172551735) bank52844 +52845 POINT(41.30226294186466 73.51095346906023) bank52845 +52846 POINT(41.10006873120004 73.09206586705741) bank52846 +52847 POINT(41.021176156267394 74.98179004470877) bank52847 +52848 POINT(39.890490907617604 74.51145075090635) bank52848 +52849 POINT(41.436626016899076 74.48216588631324) bank52849 +52850 POINT(41.17045854819387 74.2271812077972) bank52850 +52851 POINT(41.04443495976911 74.22679018031236) bank52851 +52852 POINT(40.240937484315 73.88685090698247) bank52852 +52853 POINT(41.59725245502765 74.21793479611125) bank52853 +52854 POINT(39.92365998508354 74.089703290196) bank52854 +52855 POINT(40.03250655548638 73.82563189561475) bank52855 +52856 POINT(41.444836073840456 73.86791849336755) bank52856 +52857 POINT(40.278601683378874 73.47579964329677) bank52857 +52858 POINT(40.952544343097735 73.19163491297196) bank52858 +52859 POINT(40.482046446356186 73.6871658955724) bank52859 +52860 POINT(41.69822331274608 73.60528500173707) bank52860 +52861 POINT(40.42220472165993 74.45444055215769) bank52861 +52862 POINT(41.28450473485806 74.31963806822628) bank52862 +52863 POINT(39.73589860056026 73.7584688417867) bank52863 +52864 POINT(41.18371333893116 73.5591062900964) bank52864 +52865 POINT(41.51119429096809 74.65504299955427) bank52865 +52866 POINT(40.25766479142576 73.45396541764568) bank52866 +52867 POINT(41.29424667857349 73.98476475507353) bank52867 +52868 POINT(41.16188389518726 74.11094562895087) bank52868 +52869 POINT(41.30292880071929 74.10145848637843) bank52869 +52870 POINT(40.0317450708998 73.09163665766586) bank52870 +52871 POINT(41.238037772669976 74.8751188111257) bank52871 +52872 POINT(39.911078609648015 74.74938483804519) bank52872 +52873 POINT(39.87776406413709 74.68482379901766) bank52873 +52874 POINT(41.0883765227796 73.94273405637273) bank52874 +52875 POINT(40.6406685420921 73.9698976487001) bank52875 +52876 POINT(39.71360133254795 74.63570211399222) bank52876 +52877 POINT(39.96682691418139 74.4826920630239) bank52877 +52878 POINT(40.599869036676004 74.90943723010089) bank52878 +52879 POINT(40.2846799002009 74.92043921288028) bank52879 +52880 POINT(40.24939905321307 74.27443042358034) bank52880 +52881 POINT(41.5891240375281 74.48938142176998) bank52881 +52882 POINT(39.88538962575468 74.1250543234617) bank52882 +52883 POINT(39.95087434352621 73.53971616452853) bank52883 +52884 POINT(41.13211967216653 73.18959366376464) bank52884 +52885 POINT(41.63020068387527 74.63997584093897) bank52885 +52886 POINT(39.79163795654972 73.84579799989393) bank52886 +52887 POINT(39.73322209298372 74.04489850835205) bank52887 +52888 POINT(40.03186068080134 74.89165987541759) bank52888 +52889 POINT(39.77163921489964 74.65519845509256) bank52889 +52890 POINT(40.74831489155299 74.89394647667815) bank52890 +52891 POINT(40.36447394374788 73.3371426247728) bank52891 +52892 POINT(41.01205317282605 73.53475146201492) bank52892 +52893 POINT(40.80829661295749 74.53182254157684) bank52893 +52894 POINT(41.42016729209321 74.26733796698508) bank52894 +52895 POINT(41.20474747207965 74.9648408014849) bank52895 +52896 POINT(40.45378018288699 73.03990351009128) bank52896 +52897 POINT(41.43318720037536 73.1461051014445) bank52897 +52898 POINT(40.095148546640125 74.29604964285839) bank52898 +52899 POINT(40.754941458615484 73.69689071170686) bank52899 +52900 POINT(40.20596261951362 73.90717759447057) bank52900 +52901 POINT(41.7014005123241 73.93375361627767) bank52901 +52902 POINT(39.848939049391966 73.28285254324192) bank52902 +52903 POINT(39.999754741767575 74.70505329453188) bank52903 +52904 POINT(41.44537221889249 74.54193856023686) bank52904 +52905 POINT(41.61048283636598 73.17389892952326) bank52905 +52906 POINT(40.57583753085008 73.65852531809085) bank52906 +52907 POINT(41.60478020134543 73.16111402215655) bank52907 +52908 POINT(39.79389892715953 74.4555530357784) bank52908 +52909 POINT(41.43697053489906 73.22362526698014) bank52909 +52910 POINT(41.28956225585858 73.42669782395042) bank52910 +52911 POINT(40.6049476395188 74.02685661501224) bank52911 +52912 POINT(41.2129932236526 73.59364577954116) bank52912 +52913 POINT(40.7282592136166 74.87367431552276) bank52913 +52914 POINT(39.84219318723365 73.56537819462625) bank52914 +52915 POINT(40.088973569694936 74.7161552770128) bank52915 +52916 POINT(41.4015253872335 74.14580904503781) bank52916 +52917 POINT(40.74330177758494 74.57866667666211) bank52917 +52918 POINT(40.51173705683491 73.95833476604814) bank52918 +52919 POINT(41.61473693531951 74.78352494583802) bank52919 +52920 POINT(40.450996424339806 74.79698401472469) bank52920 +52921 POINT(41.365869195158304 73.51025788067989) bank52921 +52922 POINT(40.027608789720865 73.06356311575158) bank52922 +52923 POINT(40.12629816817975 73.69938474133164) bank52923 +52924 POINT(40.403578726622804 74.3938056914274) bank52924 +52925 POINT(41.56180616592231 74.58382717336232) bank52925 +52926 POINT(40.250274527223105 74.654814629541) bank52926 +52927 POINT(40.738311875657516 73.27216006671816) bank52927 +52928 POINT(40.67006002920287 73.30041209260482) bank52928 +52929 POINT(40.46042060025081 74.16238729154063) bank52929 +52930 POINT(41.48053692102083 73.35077452533223) bank52930 +52931 POINT(40.93990885288141 74.88781538549368) bank52931 +52932 POINT(40.647913238929675 74.25650881557131) bank52932 +52933 POINT(40.3751617119637 73.93609235158695) bank52933 +52934 POINT(39.93091141218962 73.26800287854178) bank52934 +52935 POINT(40.78899486120076 73.97376413128731) bank52935 +52936 POINT(41.239193615801945 74.24622839393207) bank52936 +52937 POINT(40.44285732146888 74.4792887384461) bank52937 +52938 POINT(41.54066412846066 74.84708235586514) bank52938 +52939 POINT(40.11934896856594 74.46534898558235) bank52939 +52940 POINT(41.204838620612655 74.79219964972229) bank52940 +52941 POINT(39.78316575624659 73.30186848990162) bank52941 +52942 POINT(41.52306245709035 73.43411744535847) bank52942 +52943 POINT(41.367616416167486 73.43569385842585) bank52943 +52944 POINT(41.1828139200603 74.6023237924119) bank52944 +52945 POINT(40.22100665606944 73.53958543599822) bank52945 +52946 POINT(40.94318134882715 73.62867687437979) bank52946 +52947 POINT(40.45091138663016 73.67001721137832) bank52947 +52948 POINT(41.36932893893807 74.11989855877002) bank52948 +52949 POINT(40.742349609913944 74.71028428443192) bank52949 +52950 POINT(39.94093134283606 74.027537622897) bank52950 +52951 POINT(41.04524127039197 73.45758882091089) bank52951 +52952 POINT(41.561779477158915 73.26566934213263) bank52952 +52953 POINT(40.89653203722735 73.57209137581337) bank52953 +52954 POINT(40.9918031786449 74.70573177756683) bank52954 +52955 POINT(40.111651748015994 74.89623614488087) bank52955 +52956 POINT(41.21417061859265 73.4349494964019) bank52956 +52957 POINT(40.36489495217391 74.02631263885023) bank52957 +52958 POINT(41.497793120392025 73.27666025451616) bank52958 +52959 POINT(41.501388815922 73.67212311656542) bank52959 +52960 POINT(41.582019068885415 73.0239916254261) bank52960 +52961 POINT(40.13300743166527 73.52207088654113) bank52961 +52962 POINT(40.99516557802675 74.21397710483105) bank52962 +52963 POINT(41.190893454947485 74.83781719124346) bank52963 +52964 POINT(41.16098807654698 74.54740648639755) bank52964 +52965 POINT(41.63338347547054 73.74043361335036) bank52965 +52966 POINT(40.7946717219801 73.64626350111025) bank52966 +52967 POINT(40.84973425156399 73.06324341888153) bank52967 +52968 POINT(41.54179930257592 73.78486376046213) bank52968 +52969 POINT(41.311601823069616 73.59646083695091) bank52969 +52970 POINT(40.83584658877772 73.2114615716086) bank52970 +52971 POINT(41.44297937177453 74.76767423127704) bank52971 +52972 POINT(40.68807324733855 73.73044180287839) bank52972 +52973 POINT(40.17451062642408 73.19292540582813) bank52973 +52974 POINT(41.37995438840702 73.8251931504075) bank52974 +52975 POINT(41.175772670815256 73.63091706553523) bank52975 +52976 POINT(41.30423376192134 74.13770758031436) bank52976 +52977 POINT(40.61739485108594 74.27827860108601) bank52977 +52978 POINT(40.281643128462676 73.64785391230264) bank52978 +52979 POINT(39.74168554647343 73.96375616788855) bank52979 +52980 POINT(39.71605126646452 74.70991409816997) bank52980 +52981 POINT(41.04768718004518 73.25764247226051) bank52981 +52982 POINT(41.21621340218607 73.67295264654149) bank52982 +52983 POINT(40.58618813261738 74.13102174190512) bank52983 +52984 POINT(40.60827747560488 73.3861558718178) bank52984 +52985 POINT(40.93073106951576 73.61080330851183) bank52985 +52986 POINT(41.1462863739857 74.31953946245984) bank52986 +52987 POINT(40.631248956382635 74.06252062539212) bank52987 +52988 POINT(41.280187335655945 75.00215084733296) bank52988 +52989 POINT(41.019442920582065 73.36300575015964) bank52989 +52990 POINT(40.461205320393 73.1949824675118) bank52990 +52991 POINT(40.352235708805004 74.02771322348804) bank52991 +52992 POINT(41.131226330207284 73.99707260629498) bank52992 +52993 POINT(39.77002775386245 73.11951425123947) bank52993 +52994 POINT(41.085614467489805 74.63243643433556) bank52994 +52995 POINT(40.393224073235665 73.41007961873791) bank52995 +52996 POINT(41.53186991569799 74.41522374046978) bank52996 +52997 POINT(41.42935471719851 74.54041122857889) bank52997 +52998 POINT(40.32951446374488 74.06545898155602) bank52998 +52999 POINT(41.12296661496786 74.8490118168724) bank52999 +53000 POINT(40.009997363192134 74.2618143352728) bank53000 +53001 POINT(41.29306302357021 74.26319261806684) bank53001 +53002 POINT(39.94292624595351 74.76933397390766) bank53002 +53003 POINT(40.207610313111644 74.65245845729227) bank53003 +53004 POINT(41.079666476607805 74.14801348526188) bank53004 +53005 POINT(40.74031116367469 74.92080785976472) bank53005 +53006 POINT(39.869148855665706 74.75856134245306) bank53006 +53007 POINT(41.3840666069133 74.07508312961816) bank53007 +53008 POINT(41.364368728182754 74.79528540287271) bank53008 +53009 POINT(41.44694044752032 73.05042399366036) bank53009 +53010 POINT(40.56518790473776 74.35500308466914) bank53010 +53011 POINT(40.619563432208416 74.16504050145694) bank53011 +53012 POINT(40.02611563768355 74.90590204825423) bank53012 +53013 POINT(41.47372115361502 74.80421735488052) bank53013 +53014 POINT(41.560852146487306 74.35592322331368) bank53014 +53015 POINT(40.082254292866764 74.15837319770667) bank53015 +53016 POINT(41.15033513124543 73.70321164076658) bank53016 +53017 POINT(41.10269995119164 74.88659620876682) bank53017 +53018 POINT(41.49031307833723 74.01282923028135) bank53018 +53019 POINT(39.99580616749328 73.7057345809454) bank53019 +53020 POINT(41.42240915247947 74.94080413085314) bank53020 +53021 POINT(41.501954049321384 73.52340185302418) bank53021 +53022 POINT(39.951147575594085 74.89175808304351) bank53022 +53023 POINT(39.90349548890642 73.41970081518816) bank53023 +53024 POINT(41.07511971267073 74.12923997052894) bank53024 +53025 POINT(40.431842339748485 73.9929407400873) bank53025 +53026 POINT(40.8807402706266 73.15302433906545) bank53026 +53027 POINT(40.00517976778991 74.5669996287507) bank53027 +53028 POINT(40.94317182960211 74.63519367117988) bank53028 +53029 POINT(40.50510583153508 73.51435104587875) bank53029 +53030 POINT(40.72835405952242 73.36113271015779) bank53030 +53031 POINT(40.77608892994731 74.00328740880376) bank53031 +53032 POINT(40.629687505691294 73.0925228776281) bank53032 +53033 POINT(40.091128833903255 73.84773673122618) bank53033 +53034 POINT(40.45159733349299 73.04826549746082) bank53034 +53035 POINT(39.94560567844496 74.19557044843256) bank53035 +53036 POINT(40.742445466207165 74.68936399733988) bank53036 +53037 POINT(41.70073553598362 73.8991427282286) bank53037 +53038 POINT(41.55414062546037 74.61782093151571) bank53038 +53039 POINT(41.0384665970346 73.28295214051853) bank53039 +53040 POINT(40.32437848729211 73.83374309527694) bank53040 +53041 POINT(40.64719485886527 74.54991237058113) bank53041 +53042 POINT(40.08121536239795 74.19355514461189) bank53042 +53043 POINT(41.65162582600799 73.43857429358572) bank53043 +53044 POINT(39.98964064657659 73.29286469533548) bank53044 +53045 POINT(41.15147616665646 74.62637356869119) bank53045 +53046 POINT(41.358228698743304 73.6310381645564) bank53046 +53047 POINT(40.493671690977926 73.43729076481722) bank53047 +53048 POINT(40.31798887406729 74.52125826899967) bank53048 +53049 POINT(40.27418939345079 73.12317264615157) bank53049 +53050 POINT(41.04789405389278 73.39406474759964) bank53050 +53051 POINT(40.84317438570635 74.9992974568582) bank53051 +53052 POINT(41.339757728961224 74.94680283709796) bank53052 +53053 POINT(40.64576632008132 74.42572571577358) bank53053 +53054 POINT(40.07659716925447 73.19198664407442) bank53054 +53055 POINT(39.72665597005044 73.44597873048066) bank53055 +53056 POINT(40.25866074528509 74.96791247905539) bank53056 +53057 POINT(40.39281007143609 73.0401601112264) bank53057 +53058 POINT(41.36286980361298 73.70259178964429) bank53058 +53059 POINT(40.55995701063137 74.44556341021328) bank53059 +53060 POINT(40.25380929349501 74.44853067084229) bank53060 +53061 POINT(41.31533412485686 73.27952962478274) bank53061 +53062 POINT(41.62813464155663 73.38006894048767) bank53062 +53063 POINT(39.77852275430417 74.71658061452624) bank53063 +53064 POINT(40.324409461846045 74.71745578126858) bank53064 +53065 POINT(41.69119260040753 74.62298873997644) bank53065 +53066 POINT(40.27809758716372 74.50699594926925) bank53066 +53067 POINT(40.111259269256834 73.11904241179523) bank53067 +53068 POINT(40.803829062301155 74.02985554367093) bank53068 +53069 POINT(40.33549040427344 73.97654609448102) bank53069 +53070 POINT(41.38123313021779 74.30849436917023) bank53070 +53071 POINT(40.5946925260574 74.34559700665422) bank53071 +53072 POINT(40.2636121708383 73.9755290653506) bank53072 +53073 POINT(40.973812670875574 74.16162322515596) bank53073 +53074 POINT(40.36961988422051 74.43892566755139) bank53074 +53075 POINT(40.636238940437416 74.96171030425403) bank53075 +53076 POINT(39.73545083491619 73.73254012435035) bank53076 +53077 POINT(41.096129591038775 73.43044771700691) bank53077 +53078 POINT(39.889016901497705 73.44202993657136) bank53078 +53079 POINT(41.375435769681005 74.63134015293528) bank53079 +53080 POINT(41.65399879838634 73.76545072278303) bank53080 +53081 POINT(41.147860768872505 74.91203393364533) bank53081 +53082 POINT(40.02706217589424 74.50511595996298) bank53082 +53083 POINT(41.385208474963875 73.5646970143846) bank53083 +53084 POINT(40.96545709460029 74.61602006436108) bank53084 +53085 POINT(40.75002928669109 73.02718061968368) bank53085 +53086 POINT(40.777702436263034 74.72872134070558) bank53086 +53087 POINT(39.712995191441884 74.22797861837677) bank53087 +53088 POINT(40.40715956096474 73.26272977611838) bank53088 +53089 POINT(41.00680968036433 74.56424842187191) bank53089 +53090 POINT(40.1091755065734 74.62157492277588) bank53090 +53091 POINT(41.00082950308667 73.37687874695415) bank53091 +53092 POINT(39.88454393644242 73.39847791803257) bank53092 +53093 POINT(41.441153599375134 73.87048058407322) bank53093 +53094 POINT(40.477549312125305 74.59384100773613) bank53094 +53095 POINT(41.45461808432519 73.42506005935294) bank53095 +53096 POINT(40.744652544932855 73.50046048420603) bank53096 +53097 POINT(41.60158029801352 74.09971416257761) bank53097 +53098 POINT(40.99660625208281 74.68053530465573) bank53098 +53099 POINT(40.66639960781781 74.58135569399253) bank53099 +53100 POINT(40.52070578836353 74.03590424454039) bank53100 +53101 POINT(41.10700581717644 73.49059765548859) bank53101 +53102 POINT(40.668769474666306 74.17130124663095) bank53102 +53103 POINT(41.03458638210788 73.55059481960608) bank53103 +53104 POINT(41.12083191361443 74.46372569026529) bank53104 +53105 POINT(41.347566829159355 73.18493853459454) bank53105 +53106 POINT(40.88458299050769 73.89722731165695) bank53106 +53107 POINT(41.1447477260354 74.98447763974231) bank53107 +53108 POINT(40.771606076398974 74.41153560745866) bank53108 +53109 POINT(40.074619232690104 73.40812983488857) bank53109 +53110 POINT(40.34221400324043 73.02722455159676) bank53110 +53111 POINT(41.36509939780832 74.83540346736471) bank53111 +53112 POINT(41.41562382838126 74.47718977551007) bank53112 +53113 POINT(40.58942035955355 74.04349062346145) bank53113 +53114 POINT(40.327445477381346 73.51920545836644) bank53114 +53115 POINT(41.58436298399232 73.76168562789847) bank53115 +53116 POINT(41.03414625074326 74.25996283864926) bank53116 +53117 POINT(41.51374913374146 74.77924533411064) bank53117 +53118 POINT(40.79123665202415 73.3834875565054) bank53118 +53119 POINT(40.205303486287164 74.54535225763841) bank53119 +53120 POINT(40.66230172708203 73.29671929624182) bank53120 +53121 POINT(41.466733348850475 73.22996741266972) bank53121 +53122 POINT(40.965005595242665 74.00974317164153) bank53122 +53123 POINT(41.47644863778232 74.94316739750339) bank53123 +53124 POINT(39.800503625662905 73.74921544171241) bank53124 +53125 POINT(41.04433091199904 73.11611885357094) bank53125 +53126 POINT(41.613822301866385 74.84393933878196) bank53126 +53127 POINT(39.95459665883262 73.59037022276297) bank53127 +53128 POINT(40.03740154125942 73.59450652396363) bank53128 +53129 POINT(40.50621490324687 74.96935105968824) bank53129 +53130 POINT(41.57369054712417 73.26072655940142) bank53130 +53131 POINT(40.663984513331755 73.45119572748357) bank53131 +53132 POINT(40.9205823567847 73.06517005003488) bank53132 +53133 POINT(41.43980817936614 73.8781722351645) bank53133 +53134 POINT(41.274846684562206 74.8176758799875) bank53134 +53135 POINT(40.41644287734522 73.48529596453812) bank53135 +53136 POINT(41.05011817223009 74.12197414638491) bank53136 +53137 POINT(40.94685778207478 73.7448107078485) bank53137 +53138 POINT(40.428334934623116 73.11082331874996) bank53138 +53139 POINT(39.878409129181044 74.58257328712247) bank53139 +53140 POINT(40.017127047319654 73.4629793542624) bank53140 +53141 POINT(41.39432541101764 74.18897003889579) bank53141 +53142 POINT(40.93335190681077 74.06830455840068) bank53142 +53143 POINT(41.22136721420043 74.4633048032603) bank53143 +53144 POINT(40.05864231903583 73.39028152360824) bank53144 +53145 POINT(41.007984025573805 73.46223695051033) bank53145 +53146 POINT(40.72058494717855 73.04535223868083) bank53146 +53147 POINT(41.05309316626975 73.19949657072897) bank53147 +53148 POINT(40.019340956796576 74.63123428974669) bank53148 +53149 POINT(40.9295406100123 74.49195763120262) bank53149 +53150 POINT(40.374760810267766 73.97591921744048) bank53150 +53151 POINT(41.70102089442063 74.80623209985309) bank53151 +53152 POINT(39.74663005741701 74.68290652485354) bank53152 +53153 POINT(41.21785164388122 74.26522400011824) bank53153 +53154 POINT(40.31014762971162 73.7297712646294) bank53154 +53155 POINT(40.46015641466745 73.63296291320742) bank53155 +53156 POINT(40.59682852889092 74.71016282246327) bank53156 +53157 POINT(40.70505663752074 74.46023053118013) bank53157 +53158 POINT(40.486916132052855 74.96653277806232) bank53158 +53159 POINT(41.11258935562208 73.69572643327629) bank53159 +53160 POINT(40.18515539731684 74.69581795326266) bank53160 +53161 POINT(40.249994648287434 74.41192361421083) bank53161 +53162 POINT(40.8050109652093 74.18126226900182) bank53162 +53163 POINT(41.22262079339429 73.00656297409772) bank53163 +53164 POINT(41.02025118136233 74.82526558173828) bank53164 +53165 POINT(40.52314370512034 73.28047723892675) bank53165 +53166 POINT(39.849431079090365 74.32358380347304) bank53166 +53167 POINT(40.38077803898596 74.56585634451461) bank53167 +53168 POINT(41.50339729237801 74.60812403567232) bank53168 +53169 POINT(40.110112411689585 73.34700998293577) bank53169 +53170 POINT(40.69024721756431 74.89343841620392) bank53170 +53171 POINT(41.028234375363624 74.9418868470265) bank53171 +53172 POINT(40.998704699381925 73.29391231902945) bank53172 +53173 POINT(39.79964067814318 73.6678322351434) bank53173 +53174 POINT(40.745088069246705 73.50006572762203) bank53174 +53175 POINT(41.4276914608657 73.02441645860121) bank53175 +53176 POINT(41.22915090908524 74.57488885490957) bank53176 +53177 POINT(40.91965417269555 73.10879522628106) bank53177 +53178 POINT(39.82448536713852 74.46655204758397) bank53178 +53179 POINT(41.573302059085854 74.64338178838128) bank53179 +53180 POINT(41.399806987710654 74.60512326488815) bank53180 +53181 POINT(41.11253984498787 73.12763458482584) bank53181 +53182 POINT(40.790579475803725 74.95288685269173) bank53182 +53183 POINT(40.64028391641702 73.53473304010542) bank53183 +53184 POINT(39.84823849339604 73.53865800064666) bank53184 +53185 POINT(39.9419394493721 74.06516110606462) bank53185 +53186 POINT(40.232991249732905 73.6589207313337) bank53186 +53187 POINT(40.21098229680511 74.9082843953062) bank53187 +53188 POINT(39.799665540415525 74.51518531229492) bank53188 +53189 POINT(40.89135536177931 74.08901585902537) bank53189 +53190 POINT(40.43061645904791 75.00060555588409) bank53190 +53191 POINT(40.421671470775706 73.14903030453591) bank53191 +53192 POINT(41.369999818653916 73.5161041450211) bank53192 +53193 POINT(41.208969605173365 74.586940061478) bank53193 +53194 POINT(40.73396408057495 74.77699724477795) bank53194 +53195 POINT(41.08853482853128 74.76919399328466) bank53195 +53196 POINT(41.30898119842814 73.5169182079381) bank53196 +53197 POINT(40.205924927274445 74.24148461779167) bank53197 +53198 POINT(40.47900288684636 73.43606313664975) bank53198 +53199 POINT(41.68545813955284 73.72451045139195) bank53199 +53200 POINT(41.53379584511181 73.70298913072172) bank53200 +53201 POINT(40.689646989667075 74.24551726741028) bank53201 +53202 POINT(40.472011484586616 74.51527320624695) bank53202 +53203 POINT(40.964145507965654 74.27190653723397) bank53203 +53204 POINT(39.84848457989614 74.61164841929352) bank53204 +53205 POINT(41.34598196495981 74.5757738598119) bank53205 +53206 POINT(39.901186854402916 74.34144089063678) bank53206 +53207 POINT(41.55813653072954 73.08877149375701) bank53207 +53208 POINT(41.344655559493745 74.65766022047706) bank53208 +53209 POINT(40.07852821982752 73.0298023590525) bank53209 +53210 POINT(40.8420648429336 74.80628299045033) bank53210 +53211 POINT(40.20790022627388 74.48319226148931) bank53211 +53212 POINT(40.471594575254265 74.9280327602833) bank53212 +53213 POINT(41.38006524734903 74.51025945605635) bank53213 +53214 POINT(39.840109744484685 73.24384096060503) bank53214 +53215 POINT(41.53535565570925 74.007022245063) bank53215 +53216 POINT(41.109451671276126 73.5871505132984) bank53216 +53217 POINT(41.380329101883646 74.96079214525905) bank53217 +53218 POINT(41.39590186782265 74.75657415033353) bank53218 +53219 POINT(41.44020047593406 74.78405007921617) bank53219 +53220 POINT(41.02188033356455 74.26101762893862) bank53220 +53221 POINT(40.018523623456424 74.37805190415965) bank53221 +53222 POINT(40.95329494309218 74.74541826006936) bank53222 +53223 POINT(41.313610837580136 74.31895078200091) bank53223 +53224 POINT(40.05266079717843 73.60770290392041) bank53224 +53225 POINT(40.71815104900048 73.59870340621734) bank53225 +53226 POINT(40.82070574365242 73.61907876319397) bank53226 +53227 POINT(40.15195168226037 74.82752612264636) bank53227 +53228 POINT(40.063566900779065 74.33642864262262) bank53228 +53229 POINT(41.42322590166595 73.4339420469685) bank53229 +53230 POINT(40.90008038823585 74.2278545723627) bank53230 +53231 POINT(40.63307910953751 74.84098363623079) bank53231 +53232 POINT(39.99144708051076 74.37534834417893) bank53232 +53233 POINT(41.385820540456635 74.07333757862284) bank53233 +53234 POINT(40.60494443552693 74.13645866221702) bank53234 +53235 POINT(40.32332824150728 73.19623535752494) bank53235 +53236 POINT(40.33145901702108 74.20415688212347) bank53236 +53237 POINT(40.48500590582174 74.32905168854454) bank53237 +53238 POINT(41.19816186232043 73.38545648452134) bank53238 +53239 POINT(39.76808869519471 74.29838107162252) bank53239 +53240 POINT(40.00217315891635 73.0307862293277) bank53240 +53241 POINT(40.94056930679614 73.91725062243886) bank53241 +53242 POINT(40.44854831636988 74.13128362404736) bank53242 +53243 POINT(40.166510842714814 73.36058237556188) bank53243 +53244 POINT(41.005148825922774 74.17864756940912) bank53244 +53245 POINT(40.240934389525044 74.31300983973607) bank53245 +53246 POINT(41.07859919031798 74.26142953335024) bank53246 +53247 POINT(40.54177929212074 73.51529858098174) bank53247 +53248 POINT(41.1608014185249 73.37798937968263) bank53248 +53249 POINT(39.94343356451275 74.86412131679201) bank53249 +53250 POINT(40.403470394353334 73.07440684665409) bank53250 +53251 POINT(41.19058478527751 73.03316539776915) bank53251 +53252 POINT(40.242158735816794 73.17169670420611) bank53252 +53253 POINT(39.98851159272726 73.46687716265437) bank53253 +53254 POINT(40.90008998075712 74.17201236345733) bank53254 +53255 POINT(40.13558877763725 74.1407519402286) bank53255 +53256 POINT(39.7312823963195 74.24930517386923) bank53256 +53257 POINT(41.03711835481118 74.98365008249839) bank53257 +53258 POINT(41.65332759778382 73.08473424726277) bank53258 +53259 POINT(41.37876690662642 73.70437807202313) bank53259 +53260 POINT(40.399480289219774 74.01557454107964) bank53260 +53261 POINT(40.50641314533721 73.57849308328234) bank53261 +53262 POINT(41.00989649797829 74.84877842379302) bank53262 +53263 POINT(41.46022543760471 74.62988976502841) bank53263 +53264 POINT(40.51695007174464 74.73414790038717) bank53264 +53265 POINT(41.43707896149549 74.86122320336806) bank53265 +53266 POINT(40.50516158724159 73.70656383357563) bank53266 +53267 POINT(40.988900907770415 74.92346913987234) bank53267 +53268 POINT(41.04454390395404 73.0854437049259) bank53268 +53269 POINT(41.64075813579538 73.02137945544197) bank53269 +53270 POINT(41.06796919480536 74.36027541740847) bank53270 +53271 POINT(41.257700720201065 73.30324255726639) bank53271 +53272 POINT(41.19690805463153 74.7085628567103) bank53272 +53273 POINT(40.79995604238283 73.41672233455286) bank53273 +53274 POINT(40.10201785238105 74.64450967505724) bank53274 +53275 POINT(40.16877148923687 73.13725276892845) bank53275 +53276 POINT(39.99062224033483 74.71545514247748) bank53276 +53277 POINT(40.43162869736452 73.09794236246923) bank53277 +53278 POINT(40.006875242882536 73.42461313944277) bank53278 +53279 POINT(40.27589932838549 73.62982859610011) bank53279 +53280 POINT(41.60472070938706 74.38402394238076) bank53280 +53281 POINT(40.436254040666114 74.99256359206575) bank53281 +53282 POINT(41.534959605990295 73.31354319601861) bank53282 +53283 POINT(41.25722979824167 73.17462615218751) bank53283 +53284 POINT(39.82433733873174 73.98908170056747) bank53284 +53285 POINT(40.08351294532691 73.24669688007782) bank53285 +53286 POINT(39.746379429038775 73.11743231156784) bank53286 +53287 POINT(39.92206584811483 73.34062324060088) bank53287 +53288 POINT(41.17354289784664 74.96490578214102) bank53288 +53289 POINT(40.77433926901453 73.70245765501967) bank53289 +53290 POINT(40.446028185739 74.67896094288702) bank53290 +53291 POINT(39.776248122925104 74.11845615774308) bank53291 +53292 POINT(40.18695366071677 74.32620920763937) bank53292 +53293 POINT(39.7936147717465 74.7254421835209) bank53293 +53294 POINT(39.86053821887239 73.99540650236948) bank53294 +53295 POINT(41.70684056928485 74.14737053948163) bank53295 +53296 POINT(39.731983192523195 74.74473693505065) bank53296 +53297 POINT(40.86911528843301 75.00200130584076) bank53297 +53298 POINT(41.53337469364313 74.34005211320907) bank53298 +53299 POINT(40.855905802673306 73.23259126181233) bank53299 +53300 POINT(40.406885399279695 73.06941256763832) bank53300 +53301 POINT(40.8808857459888 73.54037238546181) bank53301 +53302 POINT(40.35546731595396 74.2567848046918) bank53302 +53303 POINT(40.72996033735505 73.24026389233683) bank53303 +53304 POINT(41.41750328278343 73.95683093012974) bank53304 +53305 POINT(40.175657065814015 73.1710892930615) bank53305 +53306 POINT(40.35964454903302 74.44788410770471) bank53306 +53307 POINT(40.82728975692187 74.98234390105021) bank53307 +53308 POINT(41.584850995480615 73.09134709181707) bank53308 +53309 POINT(40.481303513100386 74.0021185633362) bank53309 +53310 POINT(41.547735350421966 73.21111246947729) bank53310 +53311 POINT(40.262997098073974 74.52183804118077) bank53311 +53312 POINT(40.73415069494606 73.50249554593391) bank53312 +53313 POINT(41.67970193803597 74.39899958065625) bank53313 +53314 POINT(41.319579782108825 73.42800323757518) bank53314 +53315 POINT(40.155394099568774 73.31606672990176) bank53315 +53316 POINT(40.773822698903416 74.77270122475282) bank53316 +53317 POINT(39.996959629962426 74.34880146025766) bank53317 +53318 POINT(40.871332909622026 74.40877455169341) bank53318 +53319 POINT(39.85042004901154 73.34089942895905) bank53319 +53320 POINT(39.930094144106924 73.12088471465218) bank53320 +53321 POINT(41.04655858790454 73.54881964413448) bank53321 +53322 POINT(39.73412336861389 74.94449705973189) bank53322 +53323 POINT(40.15388472328566 73.07839644611967) bank53323 +53324 POINT(41.670826312688206 74.92057915671008) bank53324 +53325 POINT(39.811826466255624 73.75061594170583) bank53325 +53326 POINT(40.7350237051441 73.25450570007645) bank53326 +53327 POINT(40.98164306576917 74.15596521262394) bank53327 +53328 POINT(40.429309143154356 73.21068804354675) bank53328 +53329 POINT(41.2921297988168 74.02548589315458) bank53329 +53330 POINT(40.9364928174324 73.81779058598745) bank53330 +53331 POINT(40.17829129121582 73.98350330718351) bank53331 +53332 POINT(41.57474247672869 73.81717977156434) bank53332 +53333 POINT(40.787176723970816 74.41078247161762) bank53333 +53334 POINT(40.63325450051651 74.507159344066) bank53334 +53335 POINT(41.12366394449199 74.04819368241026) bank53335 +53336 POINT(40.92502700877188 74.26854756632788) bank53336 +53337 POINT(40.10967144977949 73.97102229148655) bank53337 +53338 POINT(41.11214015137435 73.38962287862185) bank53338 +53339 POINT(40.1418490092104 73.4336767988504) bank53339 +53340 POINT(41.50327200093952 74.82777693407742) bank53340 +53341 POINT(40.806916101367264 73.11976527501587) bank53341 +53342 POINT(40.71967478162641 74.66959857935912) bank53342 +53343 POINT(41.546988315509054 73.81599809847137) bank53343 +53344 POINT(40.93778894744917 74.38005591972869) bank53344 +53345 POINT(40.674669088850116 74.22178688762432) bank53345 +53346 POINT(41.082014147881026 74.62873868307254) bank53346 +53347 POINT(40.31625806428829 74.48171528667497) bank53347 +53348 POINT(40.200524021147224 73.42441081538891) bank53348 +53349 POINT(40.817457776986764 74.31722894154456) bank53349 +53350 POINT(39.976940844040655 73.10142571792417) bank53350 +53351 POINT(40.55410023466366 73.78707899269892) bank53351 +53352 POINT(41.57676763225832 73.16369866327332) bank53352 +53353 POINT(40.919346616128735 74.65219801394221) bank53353 +53354 POINT(39.93422197055104 73.50908150094718) bank53354 +53355 POINT(41.676690441817755 73.01676155476683) bank53355 +53356 POINT(41.42909305127469 73.26558328084927) bank53356 +53357 POINT(40.911332399475455 73.1329640072204) bank53357 +53358 POINT(40.19644930451891 73.13051450645673) bank53358 +53359 POINT(41.34820347429018 74.85318943682418) bank53359 +53360 POINT(41.43784659592294 73.61732374566931) bank53360 +53361 POINT(40.47246179923815 74.62733093030613) bank53361 +53362 POINT(40.18564106542235 73.94327614067176) bank53362 +53363 POINT(41.242657910474904 73.86363361858088) bank53363 +53364 POINT(40.24415794283501 74.78239278774906) bank53364 +53365 POINT(40.72969600383302 73.37428290228272) bank53365 +53366 POINT(39.75154326319384 73.2186703738856) bank53366 +53367 POINT(39.801037591735344 73.47420998492825) bank53367 +53368 POINT(40.49032666318877 74.60602048550844) bank53368 +53369 POINT(41.30793109502183 74.09882774332182) bank53369 +53370 POINT(40.06679122476781 73.75626838561732) bank53370 +53371 POINT(40.49827374159527 74.30831750942338) bank53371 +53372 POINT(40.95049312064525 74.9109331158836) bank53372 +53373 POINT(40.077603637022186 74.24044225685778) bank53373 +53374 POINT(41.31912277569957 74.01181604848307) bank53374 +53375 POINT(40.56845125911648 74.24782241655934) bank53375 +53376 POINT(40.245023586542644 74.99217572134654) bank53376 +53377 POINT(41.603794530401515 74.14805000627867) bank53377 +53378 POINT(41.04187170848704 73.29557834992326) bank53378 +53379 POINT(40.24228786117615 74.7147185394906) bank53379 +53380 POINT(39.81259649603251 74.98789820580609) bank53380 +53381 POINT(41.459101204287926 74.52290066913521) bank53381 +53382 POINT(39.71677271829091 74.87726552382064) bank53382 +53383 POINT(41.18910138560452 75.00392973155675) bank53383 +53384 POINT(39.99469756534804 74.20195293289837) bank53384 +53385 POINT(39.896838347658345 74.81565084501983) bank53385 +53386 POINT(40.663626982250356 73.02653959695657) bank53386 +53387 POINT(40.02981483397189 73.43915030051467) bank53387 +53388 POINT(39.77962753066413 74.28618493316469) bank53388 +53389 POINT(40.1128669826698 74.53339025331665) bank53389 +53390 POINT(40.443908427794064 73.79478284907884) bank53390 +53391 POINT(41.07818451033208 73.82969429055007) bank53391 +53392 POINT(41.67851663200146 74.38263613921441) bank53392 +53393 POINT(39.86010553637088 73.36844856657748) bank53393 +53394 POINT(39.725179581629035 73.77590923123145) bank53394 +53395 POINT(39.980935813914456 73.29480843849176) bank53395 +53396 POINT(40.68065765727021 74.21789236828988) bank53396 +53397 POINT(40.64133778878872 74.96896107892616) bank53397 +53398 POINT(40.05027353752578 75.00495976903345) bank53398 +53399 POINT(41.10395063893045 74.01621747236685) bank53399 +53400 POINT(41.51389855699878 73.83989802338216) bank53400 +53401 POINT(41.318808585934626 74.28804353916149) bank53401 +53402 POINT(40.42431236996186 74.5746279664866) bank53402 +53403 POINT(40.578526490711155 73.62874946576709) bank53403 +53404 POINT(39.82530327676506 73.34253139563478) bank53404 +53405 POINT(39.84174522754048 73.28023261530744) bank53405 +53406 POINT(40.885044672321165 74.22258068205956) bank53406 +53407 POINT(40.88739875648423 74.31096407456546) bank53407 +53408 POINT(40.59702411918606 73.07477118335892) bank53408 +53409 POINT(41.40718329783424 73.33582341128148) bank53409 +53410 POINT(39.73089044627315 74.47767991860496) bank53410 +53411 POINT(40.040506481112146 73.94195097934156) bank53411 +53412 POINT(40.99681306180014 74.77556240696148) bank53412 +53413 POINT(40.11623975807373 73.66695001428751) bank53413 +53414 POINT(39.7433793053845 73.24845107945973) bank53414 +53415 POINT(40.64666983358609 73.13465314828794) bank53415 +53416 POINT(39.883918993214074 74.01279169670322) bank53416 +53417 POINT(40.03777597749748 74.29638596711263) bank53417 +53418 POINT(40.88955056969901 74.23392984979577) bank53418 +53419 POINT(41.024823294693505 74.26306715133548) bank53419 +53420 POINT(40.50899357981683 74.14484996291279) bank53420 +53421 POINT(41.167287118428995 73.92795905193083) bank53421 +53422 POINT(40.62748004675503 73.20035545858443) bank53422 +53423 POINT(41.59481991132822 73.01005149710915) bank53423 +53424 POINT(41.436032982262994 74.64136799362107) bank53424 +53425 POINT(40.55336182628128 73.39247505628171) bank53425 +53426 POINT(40.27505023460108 74.42903091936407) bank53426 +53427 POINT(40.834095335551574 73.89215257958016) bank53427 +53428 POINT(39.865663180902246 74.05725364781178) bank53428 +53429 POINT(41.198756126930945 74.06914319305547) bank53429 +53430 POINT(41.47732612615868 73.22270687926269) bank53430 +53431 POINT(40.771216470782996 74.31722764080045) bank53431 +53432 POINT(40.5706912428891 73.03353240918995) bank53432 +53433 POINT(39.87336225957339 74.02632251876999) bank53433 +53434 POINT(40.06219509384371 74.33748432581797) bank53434 +53435 POINT(40.19712498986085 74.1755247425666) bank53435 +53436 POINT(39.862160430320905 74.08915844125337) bank53436 +53437 POINT(41.36216822687972 74.45500254866198) bank53437 +53438 POINT(40.03100388004757 73.23362107631698) bank53438 +53439 POINT(40.41798318766584 73.95814527203936) bank53439 +53440 POINT(41.187782315625036 73.52930517521662) bank53440 +53441 POINT(40.60862664548092 74.72131937434656) bank53441 +53442 POINT(40.66703912069664 73.07395690048253) bank53442 +53443 POINT(41.134950398229456 73.23768604529073) bank53443 +53444 POINT(40.77173889235151 73.99780908037235) bank53444 +53445 POINT(40.19078922219344 74.8299133019191) bank53445 +53446 POINT(39.73253970762436 74.2104942260905) bank53446 +53447 POINT(40.54884837253594 73.83670991624975) bank53447 +53448 POINT(40.43833634135833 73.7531753752941) bank53448 +53449 POINT(40.56338334758118 73.98969134056466) bank53449 +53450 POINT(41.358814573441165 74.89482981873333) bank53450 +53451 POINT(40.459041607829846 74.56794485099167) bank53451 +53452 POINT(40.54257354937527 73.05216892486345) bank53452 +53453 POINT(41.57320812449967 75.00388436901359) bank53453 +53454 POINT(40.86015557257352 74.26172247488235) bank53454 +53455 POINT(41.21997460835095 73.04281417320375) bank53455 +53456 POINT(41.381898906377145 73.139855223187) bank53456 +53457 POINT(41.349383002284945 73.83534133729691) bank53457 +53458 POINT(40.14227123397136 73.50882856450559) bank53458 +53459 POINT(40.78875790289951 74.62406620079334) bank53459 +53460 POINT(40.50482261101013 73.36902290344429) bank53460 +53461 POINT(40.48477180193218 73.89640021200826) bank53461 +53462 POINT(39.95349174615417 74.44516890942862) bank53462 +53463 POINT(40.97761921620026 74.03860356953365) bank53463 +53464 POINT(40.830640737304066 74.01159178005538) bank53464 +53465 POINT(40.942270780513454 73.30700730024799) bank53465 +53466 POINT(41.26112275350916 73.19609291999566) bank53466 +53467 POINT(41.382664170671646 74.27211674771121) bank53467 +53468 POINT(41.48951889540493 73.54121372263195) bank53468 +53469 POINT(40.211210235300165 73.71780262560203) bank53469 +53470 POINT(40.58769234261849 74.53272119751847) bank53470 +53471 POINT(40.78975994277038 73.85526595396972) bank53471 +53472 POINT(40.50894693198644 73.82149053775774) bank53472 +53473 POINT(41.359871226161545 74.487893573017) bank53473 +53474 POINT(41.0552402426424 74.24835693751282) bank53474 +53475 POINT(39.71924407526173 73.2302242677996) bank53475 +53476 POINT(40.960607883585354 74.15740275398734) bank53476 +53477 POINT(41.07957763499016 74.48700860486252) bank53477 +53478 POINT(39.857210391094696 74.3813253292572) bank53478 +53479 POINT(39.765052437881614 74.52480617742134) bank53479 +53480 POINT(40.82009862369661 73.42417792304164) bank53480 +53481 POINT(40.73142255563257 73.477713731851) bank53481 +53482 POINT(41.42724102441136 73.22264966081062) bank53482 +53483 POINT(39.830688796814734 73.40769170175027) bank53483 +53484 POINT(40.22607609432099 73.19594224988622) bank53484 +53485 POINT(41.62197195029367 74.10771457862221) bank53485 +53486 POINT(40.87899658672409 74.84163798548333) bank53486 +53487 POINT(41.4900257497463 73.58175797861935) bank53487 +53488 POINT(41.07321371625494 74.99649524577943) bank53488 +53489 POINT(41.49796533715524 74.8801218753603) bank53489 +53490 POINT(40.58012139654817 74.97132075118763) bank53490 +53491 POINT(41.02454674909331 73.86903863794163) bank53491 +53492 POINT(39.79591181725653 73.72285469362389) bank53492 +53493 POINT(40.38018254861063 73.77634159963178) bank53493 +53494 POINT(40.989362477460645 73.82333955167958) bank53494 +53495 POINT(41.495147242180195 73.67519581185749) bank53495 +53496 POINT(40.671137888169824 74.83564883630663) bank53496 +53497 POINT(39.79923146340678 73.05148002362947) bank53497 +53498 POINT(41.48254563197366 74.56617169861275) bank53498 +53499 POINT(40.0191215342391 74.77952181252705) bank53499 +53500 POINT(41.006051966691714 73.07099859609133) bank53500 +53501 POINT(40.49171173672414 74.02427821579437) bank53501 +53502 POINT(40.55931690786098 73.56986231500345) bank53502 +53503 POINT(39.93923517195073 74.81189497593216) bank53503 +53504 POINT(40.460307032709096 74.15081067540787) bank53504 +53505 POINT(41.1962869735517 73.79609506038436) bank53505 +53506 POINT(39.79749102385957 74.53339766898158) bank53506 +53507 POINT(40.54360119260158 73.54186628236347) bank53507 +53508 POINT(40.18794504578153 73.49427241316309) bank53508 +53509 POINT(40.25349994336402 74.21667159382174) bank53509 +53510 POINT(40.50638523147522 74.42539345233402) bank53510 +53511 POINT(40.563643503678314 74.18372471648522) bank53511 +53512 POINT(41.02594311061478 73.98306912086163) bank53512 +53513 POINT(40.75689517639246 73.60205905838933) bank53513 +53514 POINT(40.21674723818471 74.33251219932347) bank53514 +53515 POINT(39.93632347524671 73.45899723712286) bank53515 +53516 POINT(41.21336224106966 73.67527186729023) bank53516 +53517 POINT(41.40506525881126 73.76068691544953) bank53517 +53518 POINT(39.97144628231389 74.51282844476431) bank53518 +53519 POINT(40.90927206612892 73.729374255719) bank53519 +53520 POINT(41.169574037469765 73.65864041527624) bank53520 +53521 POINT(41.276951947796384 73.87698447337057) bank53521 +53522 POINT(40.80556373568629 73.28169489606616) bank53522 +53523 POINT(40.87680588126476 73.00839764761812) bank53523 +53524 POINT(41.61646673897443 73.78458422425224) bank53524 +53525 POINT(39.79278095620668 74.55087256150948) bank53525 +53526 POINT(41.089264260067516 74.72982825192125) bank53526 +53527 POINT(39.9277602121252 73.38963752792834) bank53527 +53528 POINT(41.29809156980882 73.167254352953) bank53528 +53529 POINT(41.41868551953841 73.70402453460231) bank53529 +53530 POINT(40.348978517916485 74.45687892480719) bank53530 +53531 POINT(41.285327782299554 73.04034349636657) bank53531 +53532 POINT(40.13605544118424 74.76205335828561) bank53532 +53533 POINT(41.18475495588475 73.91930821174107) bank53533 +53534 POINT(41.147411707298005 73.32870405941765) bank53534 +53535 POINT(39.90735340603488 74.19993731942569) bank53535 +53536 POINT(40.86288194486076 74.28669326745477) bank53536 +53537 POINT(39.732544645723536 73.91346769328656) bank53537 +53538 POINT(41.44747657827847 73.8537875764485) bank53538 +53539 POINT(40.06511582499067 73.61732517885535) bank53539 +53540 POINT(40.152629793993604 74.1551101863848) bank53540 +53541 POINT(40.06575373041078 73.73864956720891) bank53541 +53542 POINT(40.76951133933432 73.94023727485842) bank53542 +53543 POINT(41.3969886493739 73.12983226354397) bank53543 +53544 POINT(39.80396351092432 73.6414900835185) bank53544 +53545 POINT(41.3333592295625 73.65648548241444) bank53545 +53546 POINT(40.35938498719975 73.12777234337423) bank53546 +53547 POINT(41.51812434343692 73.22658836757266) bank53547 +53548 POINT(40.4374771287282 73.89035220830822) bank53548 +53549 POINT(40.408796966349 73.0147938271637) bank53549 +53550 POINT(41.117147971873806 73.10496528508276) bank53550 +53551 POINT(40.740947501914405 74.33126573840241) bank53551 +53552 POINT(41.03631189677307 73.58157292245274) bank53552 +53553 POINT(41.08135981270389 73.57671167925514) bank53553 +53554 POINT(40.80988653808738 74.85586298877202) bank53554 +53555 POINT(41.62506101609234 74.02193355547689) bank53555 +53556 POINT(41.70677843743643 74.85326599064763) bank53556 +53557 POINT(41.33793040599237 74.78807131045221) bank53557 +53558 POINT(41.09129756259085 73.55327753325147) bank53558 +53559 POINT(41.54739652810478 74.44671128922118) bank53559 +53560 POINT(40.586569494917384 74.46731918349852) bank53560 +53561 POINT(41.25053914130703 73.97758049171048) bank53561 +53562 POINT(41.09657203325024 74.31410652769219) bank53562 +53563 POINT(40.931778745496516 73.74844687679378) bank53563 +53564 POINT(41.502923600361775 73.7274791759348) bank53564 +53565 POINT(39.736768363105774 74.18409708951347) bank53565 +53566 POINT(40.29743755613403 74.78540970459288) bank53566 +53567 POINT(39.94493174465215 74.56238869853134) bank53567 +53568 POINT(41.55860052522109 73.27602645368013) bank53568 +53569 POINT(40.793681623883444 74.65243208972159) bank53569 +53570 POINT(39.74894155328589 74.783232080847) bank53570 +53571 POINT(41.368726102782766 74.51137283884098) bank53571 +53572 POINT(40.94425881256542 73.34034451279479) bank53572 +53573 POINT(40.82975355461672 74.13250604386185) bank53573 +53574 POINT(40.164012176726736 74.97935735086257) bank53574 +53575 POINT(39.771023157969985 73.25924199916237) bank53575 +53576 POINT(40.774357489065544 74.84081894450694) bank53576 +53577 POINT(41.59552755465277 74.22690701965493) bank53577 +53578 POINT(41.28148429140477 74.85893181642984) bank53578 +53579 POINT(40.423998926201506 74.70661767405046) bank53579 +53580 POINT(41.11133810264596 74.778799665855) bank53580 +53581 POINT(40.2890970234643 73.19577977773086) bank53581 +53582 POINT(40.49064872308436 73.25125935365736) bank53582 +53583 POINT(41.25079939585862 73.19858730671686) bank53583 +53584 POINT(41.152808643378904 74.63823194448237) bank53584 +53585 POINT(40.673905622897536 73.99572348270578) bank53585 +53586 POINT(40.10144441854757 73.26969792805399) bank53586 +53587 POINT(40.34322938263217 74.18237350869764) bank53587 +53588 POINT(41.531323124063505 73.33044189058253) bank53588 +53589 POINT(41.66966157696288 74.38546016554176) bank53589 +53590 POINT(40.42103782171519 73.86403256843563) bank53590 +53591 POINT(41.179287430624825 73.91173453870405) bank53591 +53592 POINT(40.155267288485476 74.07152279727373) bank53592 +53593 POINT(41.63723155463791 73.65635960099227) bank53593 +53594 POINT(41.045297021250136 73.78563118581441) bank53594 +53595 POINT(41.188395574583566 74.60063962475456) bank53595 +53596 POINT(40.87657957698579 73.79575837477317) bank53596 +53597 POINT(41.06595877328337 73.64050007172818) bank53597 +53598 POINT(40.07101121093259 74.1531844516178) bank53598 +53599 POINT(40.297696734042304 73.54303166357852) bank53599 +53600 POINT(40.40942491008597 74.29494221681256) bank53600 +53601 POINT(40.1028022972312 74.1183969237433) bank53601 +53602 POINT(39.965366860506926 74.87806606861326) bank53602 +53603 POINT(39.95499213340824 73.85344761621025) bank53603 +53604 POINT(40.510886304616484 73.57430364016138) bank53604 +53605 POINT(40.244430325085055 73.54378694511028) bank53605 +53606 POINT(40.884817563206376 74.43715583772318) bank53606 +53607 POINT(40.86311205523812 73.09462002537123) bank53607 +53608 POINT(40.12593874547841 74.67988044294363) bank53608 +53609 POINT(40.74696261204104 74.51393724398149) bank53609 +53610 POINT(41.339470380722624 74.66058102259099) bank53610 +53611 POINT(40.39989478430702 73.5415773637439) bank53611 +53612 POINT(40.98090624190968 74.52927145894583) bank53612 +53613 POINT(41.55443117491591 74.24186186481772) bank53613 +53614 POINT(41.65941086924139 73.22807628884969) bank53614 +53615 POINT(41.03631604312416 74.532708556792) bank53615 +53616 POINT(40.85556642999367 73.35374745775879) bank53616 +53617 POINT(40.53408296263207 74.49919753579091) bank53617 +53618 POINT(41.08003688404952 74.30084860428636) bank53618 +53619 POINT(40.257718634887006 74.0621529368028) bank53619 +53620 POINT(40.836454776036184 73.975067528704) bank53620 +53621 POINT(40.24317819491203 74.47095511285875) bank53621 +53622 POINT(41.40392743900964 73.60117931956968) bank53622 +53623 POINT(39.94104358660362 74.55549071805626) bank53623 +53624 POINT(40.72255474513121 74.18584386886414) bank53624 +53625 POINT(41.172486973770596 73.93471378991613) bank53625 +53626 POINT(40.10130767905643 74.41372478100628) bank53626 +53627 POINT(39.84280894156286 74.61018999542388) bank53627 +53628 POINT(41.02480972139182 74.18349378358798) bank53628 +53629 POINT(41.411402226777405 74.82351627082737) bank53629 +53630 POINT(40.48708426969817 73.35815531309976) bank53630 +53631 POINT(40.77980349455753 74.49342393158727) bank53631 +53632 POINT(41.62375511124822 74.24578475852269) bank53632 +53633 POINT(41.266896285852624 74.49770414004195) bank53633 +53634 POINT(40.95727278859782 74.49070190264824) bank53634 +53635 POINT(41.60171211148224 73.43235343073498) bank53635 +53636 POINT(40.27004646642858 73.45765187716152) bank53636 +53637 POINT(40.65515616585531 73.22930442804417) bank53637 +53638 POINT(39.81916121195505 73.32287201044187) bank53638 +53639 POINT(40.97015466897096 73.64965173372109) bank53639 +53640 POINT(40.63634080307151 73.15108987326028) bank53640 +53641 POINT(41.246898135651996 74.85188466739324) bank53641 +53642 POINT(39.80883233809499 73.60217085323723) bank53642 +53643 POINT(39.71742751719069 73.74181267988105) bank53643 +53644 POINT(41.16215141579013 74.59614281434676) bank53644 +53645 POINT(40.18884282870635 74.55349535085792) bank53645 +53646 POINT(40.381886311306566 73.51790666383671) bank53646 +53647 POINT(39.989357827030894 73.93978922014487) bank53647 +53648 POINT(39.92563060786855 74.96871967989946) bank53648 +53649 POINT(39.81483302078438 73.28273961921093) bank53649 +53650 POINT(41.20221532544333 74.32191376450939) bank53650 +53651 POINT(41.19455396373337 73.34083438761908) bank53651 +53652 POINT(40.554169626737824 74.2670374905157) bank53652 +53653 POINT(40.58151581470274 73.23083434075622) bank53653 +53654 POINT(40.51930403102113 74.25297493859406) bank53654 +53655 POINT(39.92937353546994 73.6799469530113) bank53655 +53656 POINT(39.9784326858562 74.01415125613599) bank53656 +53657 POINT(41.70550441486117 74.70222333840046) bank53657 +53658 POINT(40.379405433550744 74.75942943574245) bank53658 +53659 POINT(39.74486862205712 74.00255539853757) bank53659 +53660 POINT(39.73717210520769 73.40344427829592) bank53660 +53661 POINT(41.09813394474758 73.61612709431509) bank53661 +53662 POINT(39.74203164040905 73.69232114821259) bank53662 +53663 POINT(41.69859459808503 74.54474084286586) bank53663 +53664 POINT(40.240644164127794 73.62359703517248) bank53664 +53665 POINT(41.37767765945487 73.22224182140098) bank53665 +53666 POINT(40.66146766956381 73.66729847396084) bank53666 +53667 POINT(41.04310923156984 73.50441033414351) bank53667 +53668 POINT(41.016565239732586 74.88827897161502) bank53668 +53669 POINT(40.45089011080419 74.20758790632702) bank53669 +53670 POINT(39.769627176855685 73.89692562838137) bank53670 +53671 POINT(40.32411176068334 74.82279581717621) bank53671 +53672 POINT(41.37343423622784 74.39656343611001) bank53672 +53673 POINT(40.87443954449164 73.00652540623193) bank53673 +53674 POINT(40.32870637218059 74.7754244180002) bank53674 +53675 POINT(40.45707198534627 73.67154022376731) bank53675 +53676 POINT(39.978853306568155 74.05841640511268) bank53676 +53677 POINT(41.05064385266054 73.788296439875) bank53677 +53678 POINT(41.457586471879175 73.43338068459386) bank53678 +53679 POINT(40.89120552662308 74.29001006975484) bank53679 +53680 POINT(39.849611246862324 73.87387379374962) bank53680 +53681 POINT(40.18849691762679 73.94064487300534) bank53681 +53682 POINT(40.66957494638381 74.313406558585) bank53682 +53683 POINT(40.503056857424944 74.97568155879519) bank53683 +53684 POINT(40.88494940539474 73.64016954499735) bank53684 +53685 POINT(41.672143899954015 74.27992890294477) bank53685 +53686 POINT(40.45238317554669 74.5747253780344) bank53686 +53687 POINT(41.2273805473406 73.21329810635541) bank53687 +53688 POINT(39.95699445691803 73.88795806819736) bank53688 +53689 POINT(39.996075434248596 74.25989218176382) bank53689 +53690 POINT(40.87179657280366 74.3783672139685) bank53690 +53691 POINT(40.5762216458461 74.80156798384448) bank53691 +53692 POINT(40.38769094600588 73.5901827197774) bank53692 +53693 POINT(40.00147580208298 73.75712101752194) bank53693 +53694 POINT(40.13643037637119 74.923334994822) bank53694 +53695 POINT(41.15711332042208 73.73016952680979) bank53695 +53696 POINT(41.352254011818786 74.5025168605608) bank53696 +53697 POINT(40.96504994658324 73.60382348106609) bank53697 +53698 POINT(40.72180807756691 74.52492292162671) bank53698 +53699 POINT(39.99514237634157 73.42687543331397) bank53699 +53700 POINT(40.651500526978104 74.19519621988906) bank53700 +53701 POINT(40.3842777893726 74.35900210146374) bank53701 +53702 POINT(40.30162614931648 73.84403386210464) bank53702 +53703 POINT(40.72661851290667 74.16421165994811) bank53703 +53704 POINT(41.46857876861493 73.27194055572592) bank53704 +53705 POINT(41.35124382412554 73.11838767137304) bank53705 +53706 POINT(40.92441654202633 74.73608804105274) bank53706 +53707 POINT(39.858500248963246 74.87023988789527) bank53707 +53708 POINT(39.84764936058526 73.71845998010558) bank53708 +53709 POINT(41.43365407562779 74.75611258420344) bank53709 +53710 POINT(41.59080661981819 74.79953576199975) bank53710 +53711 POINT(41.64129165289314 74.54297026941022) bank53711 +53712 POINT(40.04150615435173 74.09954281186505) bank53712 +53713 POINT(40.88864334764177 73.12725127826263) bank53713 +53714 POINT(41.05093954782749 74.82441184635712) bank53714 +53715 POINT(40.233006456225 74.27130661668336) bank53715 +53716 POINT(40.64795745623021 74.27623627934507) bank53716 +53717 POINT(40.31069144168644 73.86994965010351) bank53717 +53718 POINT(41.49616196535159 73.21450794599963) bank53718 +53719 POINT(41.33343822728448 74.18340603688114) bank53719 +53720 POINT(41.36792369963927 74.48489825546773) bank53720 +53721 POINT(39.79880006038867 73.21495719498334) bank53721 +53722 POINT(41.16112603164995 73.74697487666572) bank53722 +53723 POINT(41.49984249713565 74.8250781654227) bank53723 +53724 POINT(41.236866581560825 74.8700647939978) bank53724 +53725 POINT(41.470740943824644 74.16893606130796) bank53725 +53726 POINT(40.52528786547875 73.2742795720899) bank53726 +53727 POINT(41.533647899255335 74.66790624031593) bank53727 +53728 POINT(40.45707880643078 74.24802877878005) bank53728 +53729 POINT(40.47425321255968 73.81764203603431) bank53729 +53730 POINT(40.847504267659424 74.14715820200567) bank53730 +53731 POINT(39.908831066267965 74.93006138106797) bank53731 +53732 POINT(40.074221650675284 74.0751139900437) bank53732 +53733 POINT(40.90608694710492 74.46500391681116) bank53733 +53734 POINT(41.57218769128651 74.15228867488165) bank53734 +53735 POINT(41.50239054372236 74.7641834764596) bank53735 +53736 POINT(40.52826509548814 74.37559210646765) bank53736 +53737 POINT(40.93304195664603 74.3656608774315) bank53737 +53738 POINT(41.00086930100821 73.39500208220468) bank53738 +53739 POINT(39.803751080501534 74.99179298468026) bank53739 +53740 POINT(39.85326348533787 74.29377934269486) bank53740 +53741 POINT(39.73761595013939 74.07826812261615) bank53741 +53742 POINT(40.4635999901976 74.4537089746907) bank53742 +53743 POINT(40.847804700782476 73.2246414106011) bank53743 +53744 POINT(41.27061657740422 73.08677905840223) bank53744 +53745 POINT(39.78055467051626 74.82000453222226) bank53745 +53746 POINT(40.876512535590095 74.31550808231637) bank53746 +53747 POINT(40.00887318867251 73.88650996822766) bank53747 +53748 POINT(40.10696250396675 74.68573682183859) bank53748 +53749 POINT(41.69608904995117 73.32792889397689) bank53749 +53750 POINT(41.68176988478515 74.00178519392436) bank53750 +53751 POINT(40.991653656462894 74.87438067962631) bank53751 +53752 POINT(41.51710982992581 73.07511656777017) bank53752 +53753 POINT(39.86951804569603 74.1053220028619) bank53753 +53754 POINT(40.5980599672839 73.14565484380772) bank53754 +53755 POINT(39.883450573774795 74.13581309095562) bank53755 +53756 POINT(41.23161779739422 74.83768508942741) bank53756 +53757 POINT(41.48360583232401 73.19850567809054) bank53757 +53758 POINT(40.391231938836555 74.71984208910395) bank53758 +53759 POINT(41.284026710776466 74.58889329608878) bank53759 +53760 POINT(40.88411927926864 74.03125908204142) bank53760 +53761 POINT(40.24563144940278 73.16466022050868) bank53761 +53762 POINT(41.38758784861133 73.22517842660606) bank53762 +53763 POINT(40.603891247872376 73.75577256474291) bank53763 +53764 POINT(41.465431631156 73.84949298209591) bank53764 +53765 POINT(40.99295492940633 73.90483513294092) bank53765 +53766 POINT(40.61122724693706 73.64279853260676) bank53766 +53767 POINT(40.432688305121644 74.07830050476582) bank53767 +53768 POINT(40.609587297740376 74.24198361453799) bank53768 +53769 POINT(40.71627063861089 73.21802496960792) bank53769 +53770 POINT(41.353645465746006 74.34934686220126) bank53770 +53771 POINT(40.88123315894934 73.13434580222155) bank53771 +53772 POINT(39.92442254117625 73.0926164785198) bank53772 +53773 POINT(40.36133008215389 74.80461963723873) bank53773 +53774 POINT(41.618322144524 73.22566099989542) bank53774 +53775 POINT(40.60461409839576 73.8133300062849) bank53775 +53776 POINT(40.541057566513935 74.19576696910926) bank53776 +53777 POINT(41.502126370030865 74.81809638776049) bank53777 +53778 POINT(40.47742669729159 73.70949212922798) bank53778 +53779 POINT(39.81121207252704 73.4192716804702) bank53779 +53780 POINT(41.599563600256204 73.13495562755541) bank53780 +53781 POINT(41.14514263648117 73.60006022517092) bank53781 +53782 POINT(40.359430542151166 73.75466795766333) bank53782 +53783 POINT(41.20382934924299 74.0114548735435) bank53783 +53784 POINT(41.083268364736384 73.21276296111252) bank53784 +53785 POINT(40.863556351973926 73.46409379052636) bank53785 +53786 POINT(39.836037335798686 74.72475172702761) bank53786 +53787 POINT(41.580338627945366 73.89806065120166) bank53787 +53788 POINT(41.41401442325112 73.99888769105779) bank53788 +53789 POINT(41.46873276155233 74.22432783362099) bank53789 +53790 POINT(40.83915765420178 73.9143724588718) bank53790 +53791 POINT(40.86168429588387 74.4153344252863) bank53791 +53792 POINT(39.98837389784191 73.53202659772457) bank53792 +53793 POINT(41.097900995922956 74.18212724851823) bank53793 +53794 POINT(39.763999716196956 74.02731623195675) bank53794 +53795 POINT(39.80090758869088 73.75589650791463) bank53795 +53796 POINT(41.16931581202664 74.23270484144848) bank53796 +53797 POINT(40.40615460395897 74.7895700388145) bank53797 +53798 POINT(39.8186192439535 74.04652034552798) bank53798 +53799 POINT(41.45933952772029 74.74016901720294) bank53799 +53800 POINT(40.48461186412569 74.06579981125455) bank53800 +53801 POINT(40.40177882430107 73.15874153011731) bank53801 +53802 POINT(40.80990953714503 73.55857697983957) bank53802 +53803 POINT(40.75327745540853 74.41783101604945) bank53803 +53804 POINT(39.85906785009923 74.53815149580444) bank53804 +53805 POINT(40.34959396760327 74.493708453504) bank53805 +53806 POINT(40.40303297507395 73.72566214628377) bank53806 +53807 POINT(39.751128192521534 74.36525431824965) bank53807 +53808 POINT(40.69623531037278 73.14987414785304) bank53808 +53809 POINT(40.563325740555996 74.41757306950552) bank53809 +53810 POINT(41.013373130858305 73.31878109374998) bank53810 +53811 POINT(40.43642815033631 73.33413475350619) bank53811 +53812 POINT(40.76965963268492 73.93036986532324) bank53812 +53813 POINT(41.37458106561563 74.78826547809851) bank53813 +53814 POINT(41.154053321741486 74.42655285314027) bank53814 +53815 POINT(41.365481305801985 74.25733637169016) bank53815 +53816 POINT(41.553935203693605 74.2879328874173) bank53816 +53817 POINT(39.79710070096442 74.15524908297006) bank53817 +53818 POINT(40.95782638525692 73.24393906202482) bank53818 +53819 POINT(40.90161452707083 74.65633451947168) bank53819 +53820 POINT(41.061781532449444 73.56173203608088) bank53820 +53821 POINT(40.130252188706415 74.71448932833579) bank53821 +53822 POINT(40.781499161447144 74.03372503380804) bank53822 +53823 POINT(39.77998148491975 73.38777422018757) bank53823 +53824 POINT(40.70946657524754 73.4052170148041) bank53824 +53825 POINT(40.194632061851685 73.86308194901733) bank53825 +53826 POINT(39.839602815521125 74.01204252057637) bank53826 +53827 POINT(40.04771012090057 74.45452094587075) bank53827 +53828 POINT(39.943450570772455 73.46719615251438) bank53828 +53829 POINT(41.35359143600819 73.98123249981832) bank53829 +53830 POINT(41.05774459210315 74.13007416302148) bank53830 +53831 POINT(41.49703104394767 74.28724580657052) bank53831 +53832 POINT(41.54632534182749 74.47628414167862) bank53832 +53833 POINT(40.12336151520952 73.03918959642844) bank53833 +53834 POINT(41.501976379045594 73.09422672263342) bank53834 +53835 POINT(40.099548622491255 73.45325892977685) bank53835 +53836 POINT(41.316540468451166 74.38969618037335) bank53836 +53837 POINT(40.99433574691977 74.84927628884769) bank53837 +53838 POINT(40.856841137704244 73.78379010477667) bank53838 +53839 POINT(41.28665534433171 74.32351478937524) bank53839 +53840 POINT(40.793295306179914 73.84544606331083) bank53840 +53841 POINT(39.90072128746787 74.56713315646088) bank53841 +53842 POINT(40.08863545110553 73.96248109373197) bank53842 +53843 POINT(39.718945646576074 73.48238731205656) bank53843 +53844 POINT(41.241583627389936 74.49627066264559) bank53844 +53845 POINT(41.04227979547076 73.76129685309321) bank53845 +53846 POINT(40.31854544064241 73.77380124896546) bank53846 +53847 POINT(39.78710899877041 73.48175179486464) bank53847 +53848 POINT(40.774405882602544 74.14931453183947) bank53848 +53849 POINT(41.123911522264095 73.14230846499295) bank53849 +53850 POINT(41.10778391193385 74.04598826608498) bank53850 +53851 POINT(41.26437886784792 73.49972956714768) bank53851 +53852 POINT(41.11544842588694 73.76410813556647) bank53852 +53853 POINT(40.92992144645616 74.08645987118065) bank53853 +53854 POINT(40.13691155010223 74.48135208803589) bank53854 +53855 POINT(41.41327533582335 73.6930555704802) bank53855 +53856 POINT(39.9122419357437 74.18867878994307) bank53856 +53857 POINT(40.615340217382986 73.76324711565582) bank53857 +53858 POINT(40.182317101781116 73.0441730875796) bank53858 +53859 POINT(40.778214613764966 73.63350961896616) bank53859 +53860 POINT(39.97859594582069 74.41040640778603) bank53860 +53861 POINT(39.89321367355421 73.35441927346032) bank53861 +53862 POINT(41.61243185145085 74.98156223907654) bank53862 +53863 POINT(40.996563322602306 74.17719210340854) bank53863 +53864 POINT(41.005189674824486 74.77270182105913) bank53864 +53865 POINT(40.37660079332533 74.61567333054936) bank53865 +53866 POINT(39.87940608090017 74.39052430751343) bank53866 +53867 POINT(40.673644365518065 73.62864524778514) bank53867 +53868 POINT(41.38440104780723 73.81159309688937) bank53868 +53869 POINT(40.14549182487977 73.6040625742611) bank53869 +53870 POINT(41.12337495052377 73.46800501150865) bank53870 +53871 POINT(40.417604887472415 74.73117442397484) bank53871 +53872 POINT(41.10335206731489 73.92732398476811) bank53872 +53873 POINT(41.015270168241564 74.42780651044336) bank53873 +53874 POINT(41.60281459260117 73.71409184794528) bank53874 +53875 POINT(39.99213283784377 74.46565071111817) bank53875 +53876 POINT(41.31273666713184 74.43200191250506) bank53876 +53877 POINT(41.65778736619062 74.62559500029526) bank53877 +53878 POINT(40.61279286085668 74.80479881596155) bank53878 +53879 POINT(41.71045120257413 74.47244839068185) bank53879 +53880 POINT(39.73815312121869 74.7062394264163) bank53880 +53881 POINT(40.03920165713464 74.98322057026157) bank53881 +53882 POINT(41.32939102298118 74.88564604353063) bank53882 +53883 POINT(39.80508474409005 73.21683182151351) bank53883 +53884 POINT(41.589271387654826 73.44154831734893) bank53884 +53885 POINT(41.62997755582251 74.43455378502051) bank53885 +53886 POINT(40.392147073037215 73.24408957765588) bank53886 +53887 POINT(40.71620127207896 73.84964465961983) bank53887 +53888 POINT(39.76249995771389 73.26548276098939) bank53888 +53889 POINT(41.307896407003504 74.56869328392109) bank53889 +53890 POINT(40.55454490172261 74.25172322272871) bank53890 +53891 POINT(41.25279991790588 73.23031444396622) bank53891 +53892 POINT(40.51861706848862 74.29820991422471) bank53892 +53893 POINT(39.83802877061111 73.20552913559716) bank53893 +53894 POINT(40.2138365922097 73.68658606938396) bank53894 +53895 POINT(39.95015932061959 73.32156303122188) bank53895 +53896 POINT(40.25602761459954 74.8038191604149) bank53896 +53897 POINT(40.36223749988017 73.89902512266778) bank53897 +53898 POINT(39.80866525813616 74.16064229929555) bank53898 +53899 POINT(41.58333617111781 73.88889298356416) bank53899 +53900 POINT(41.44983842407584 73.54042271267329) bank53900 +53901 POINT(40.22707535426787 73.41182949918453) bank53901 +53902 POINT(41.41322620388506 73.5516143979484) bank53902 +53903 POINT(40.62649989133701 73.86380028856831) bank53903 +53904 POINT(40.627131636194314 74.62448041268492) bank53904 +53905 POINT(40.65214899237681 73.54282720260615) bank53905 +53906 POINT(40.92849086333277 74.51951740823417) bank53906 +53907 POINT(41.2141895125965 73.84574533745432) bank53907 +53908 POINT(41.56402883689639 73.76413487881277) bank53908 +53909 POINT(40.644549650870445 74.24674488246771) bank53909 +53910 POINT(40.285329630895994 74.81173397088396) bank53910 +53911 POINT(40.367223902477036 74.84311535375117) bank53911 +53912 POINT(40.82929504866137 73.24421508295032) bank53912 +53913 POINT(39.812925388016104 74.01861195723009) bank53913 +53914 POINT(39.738766623181256 73.64112158987764) bank53914 +53915 POINT(41.375869749069 74.81922930881878) bank53915 +53916 POINT(41.24078405521959 74.21863269784026) bank53916 +53917 POINT(39.857934432682505 74.81356004713227) bank53917 +53918 POINT(39.911659877484254 74.68008358559236) bank53918 +53919 POINT(40.4102845825553 73.45949016730327) bank53919 +53920 POINT(39.71604422976431 74.40490497092546) bank53920 +53921 POINT(41.015993985949905 73.51355481990637) bank53921 +53922 POINT(41.60163301243118 74.05991837182933) bank53922 +53923 POINT(41.19110974681293 74.68022470820507) bank53923 +53924 POINT(41.07221250930177 74.32348376047733) bank53924 +53925 POINT(39.8302440041669 73.9198412014508) bank53925 +53926 POINT(41.32531908083611 74.33393734040162) bank53926 +53927 POINT(41.03274850543655 74.54546336372793) bank53927 +53928 POINT(40.50772701198683 74.93629545373747) bank53928 +53929 POINT(40.21453674272568 74.65469660303009) bank53929 +53930 POINT(41.586338914447545 74.0428425082898) bank53930 +53931 POINT(41.21954014722881 74.99311933070122) bank53931 +53932 POINT(41.026626588435406 74.13034759567755) bank53932 +53933 POINT(41.10169431979051 73.42074759526125) bank53933 +53934 POINT(41.543715976244435 74.03370002262146) bank53934 +53935 POINT(41.52968506472203 73.19340541695306) bank53935 +53936 POINT(39.77163157553112 73.90795147469646) bank53936 +53937 POINT(41.12896063340603 73.48456638471082) bank53937 +53938 POINT(41.43630863272932 74.93076658791207) bank53938 +53939 POINT(41.08881095672892 73.95185802017966) bank53939 +53940 POINT(39.78277622849289 73.21104597755793) bank53940 +53941 POINT(40.63214553847064 73.16229682132911) bank53941 +53942 POINT(41.59407992648804 73.86332952915009) bank53942 +53943 POINT(40.89592000001641 73.48122968830187) bank53943 +53944 POINT(41.02034774042593 74.10850951682998) bank53944 +53945 POINT(40.21239953420617 73.14871311012975) bank53945 +53946 POINT(40.21217749593556 74.59805808121358) bank53946 +53947 POINT(40.79840495100615 74.41159481157658) bank53947 +53948 POINT(40.70593056863115 74.78721329997727) bank53948 +53949 POINT(39.80921622081833 74.74642809209048) bank53949 +53950 POINT(40.70659983304964 74.83030801835349) bank53950 +53951 POINT(40.92955687530934 73.77413125666524) bank53951 +53952 POINT(41.652834843127145 74.74395546939348) bank53952 +53953 POINT(41.38496160266979 74.81047538413479) bank53953 +53954 POINT(40.9798774473599 73.25343624797496) bank53954 +53955 POINT(41.025778631335996 74.77594533883845) bank53955 +53956 POINT(40.397983817382425 74.78986532150444) bank53956 +53957 POINT(41.07376147540921 74.88979950440665) bank53957 +53958 POINT(39.71767855407498 74.40488530386692) bank53958 +53959 POINT(41.058556377343294 73.14757332458831) bank53959 +53960 POINT(41.28540561469462 73.41504007758927) bank53960 +53961 POINT(40.62013175233691 73.37360994000953) bank53961 +53962 POINT(40.06308596758907 74.13106178690828) bank53962 +53963 POINT(40.93330025585901 73.21447408644222) bank53963 +53964 POINT(41.208793569045454 74.81280664326638) bank53964 +53965 POINT(39.92164108412737 73.2163646883214) bank53965 +53966 POINT(40.75499577814714 73.14832361254447) bank53966 +53967 POINT(39.894166160999504 73.57074512671763) bank53967 +53968 POINT(40.673406119250096 73.0749520466283) bank53968 +53969 POINT(40.675953632265426 73.09730181321548) bank53969 +53970 POINT(40.22812440296479 74.5109497605669) bank53970 +53971 POINT(41.21437374023834 73.02889159528455) bank53971 +53972 POINT(40.4949827750514 74.86549482595053) bank53972 +53973 POINT(40.65169634347051 74.26700407247121) bank53973 +53974 POINT(40.37979674106091 73.287515300255) bank53974 +53975 POINT(40.05534650344857 73.49904965022141) bank53975 +53976 POINT(40.443249101793185 73.15997161236405) bank53976 +53977 POINT(41.23533365592678 74.86471063885554) bank53977 +53978 POINT(40.05159812843923 74.48610150045357) bank53978 +53979 POINT(40.26279664227423 74.6653334792683) bank53979 +53980 POINT(40.697803969795174 74.28354102876882) bank53980 +53981 POINT(40.48567147313623 73.08047704440402) bank53981 +53982 POINT(40.71974794352179 74.07108732557137) bank53982 +53983 POINT(40.88216776935578 73.60031866245204) bank53983 +53984 POINT(40.41975027981503 74.51522427265083) bank53984 +53985 POINT(39.8470758872465 74.34389937315213) bank53985 +53986 POINT(40.78901122585889 73.0360489635109) bank53986 +53987 POINT(41.06168705795509 74.52819364274175) bank53987 +53988 POINT(40.895159042681776 73.25123042961688) bank53988 +53989 POINT(40.39888522027425 73.9234900579336) bank53989 +53990 POINT(40.707012853417915 74.58607730981667) bank53990 +53991 POINT(40.53854545554919 73.24838354990445) bank53991 +53992 POINT(41.10771621854304 73.72322754921176) bank53992 +53993 POINT(39.918302508218595 74.57686498992204) bank53993 +53994 POINT(40.88860949042853 73.82313405043787) bank53994 +53995 POINT(40.718625085160404 73.7919542746675) bank53995 +53996 POINT(41.411116143815015 73.43705622650816) bank53996 +53997 POINT(40.98623729238284 73.84237220030745) bank53997 +53998 POINT(40.03114830204537 73.42121586203568) bank53998 +53999 POINT(40.59914544470286 74.15031283644677) bank53999 +54000 POINT(40.64916015710136 73.56735591252298) bank54000 +54001 POINT(40.41704641088168 74.38773872884195) bank54001 +54002 POINT(40.92666563366865 74.82610559428352) bank54002 +54003 POINT(40.938898349985905 74.23641800157566) bank54003 +54004 POINT(41.25039166112106 74.31273643288982) bank54004 +54005 POINT(41.648332622035156 73.123482777311) bank54005 +54006 POINT(40.40260747583473 73.97323671322965) bank54006 +54007 POINT(41.68368364441167 73.89291828685162) bank54007 +54008 POINT(41.077553582867814 73.10733358933687) bank54008 +54009 POINT(41.13724246054289 73.66909064195299) bank54009 +54010 POINT(40.209192784204404 74.81416684143188) bank54010 +54011 POINT(41.254457071780614 74.99297369211547) bank54011 +54012 POINT(40.84798345602896 73.94583136750924) bank54012 +54013 POINT(39.79508266244548 74.20016704737681) bank54013 +54014 POINT(41.64912346818911 74.43955049704486) bank54014 +54015 POINT(40.03542898614242 74.67119256214951) bank54015 +54016 POINT(41.09404862076574 73.16963992017529) bank54016 +54017 POINT(40.58137726920714 74.48434431079806) bank54017 +54018 POINT(41.31271427490344 74.46862294845519) bank54018 +54019 POINT(40.606585424913604 74.78377702313362) bank54019 +54020 POINT(41.661240946990475 73.293279965862) bank54020 +54021 POINT(40.77016915349414 73.77724229233617) bank54021 +54022 POINT(40.29588137879212 74.11966319606412) bank54022 +54023 POINT(41.15101344854139 74.28065258589366) bank54023 +54024 POINT(41.37597140473305 73.57615713567502) bank54024 +54025 POINT(40.57453246908802 73.46984967048243) bank54025 +54026 POINT(40.27828713067888 73.02751489222639) bank54026 +54027 POINT(40.03457995656742 74.49415013153705) bank54027 +54028 POINT(41.48786480257214 74.46427585340956) bank54028 +54029 POINT(39.95430675503509 73.48012233037733) bank54029 +54030 POINT(39.88675550139504 74.54560156884982) bank54030 +54031 POINT(40.63268681890505 74.5649068973625) bank54031 +54032 POINT(40.31949820093431 74.79753696071737) bank54032 +54033 POINT(39.91928876578749 74.98476023397536) bank54033 +54034 POINT(40.00201309669981 75.0050521262406) bank54034 +54035 POINT(40.615173445807926 74.89584603213034) bank54035 +54036 POINT(40.35205282487907 73.86947581817151) bank54036 +54037 POINT(40.32474178028195 74.79554509287584) bank54037 +54038 POINT(40.142497010152006 74.57003991197242) bank54038 +54039 POINT(39.95108582183663 73.32935979059525) bank54039 +54040 POINT(39.79367997853177 73.0696180756949) bank54040 +54041 POINT(40.953058110141924 73.94415771502226) bank54041 +54042 POINT(40.123775449557286 73.69316729885489) bank54042 +54043 POINT(41.18748048259634 73.95658894676116) bank54043 +54044 POINT(41.60784034160561 73.25974187021774) bank54044 +54045 POINT(40.1375790605555 74.41548938685877) bank54045 +54046 POINT(40.71753530913558 74.30840092027904) bank54046 +54047 POINT(40.331887199860404 74.82377583679498) bank54047 +54048 POINT(39.909162733797196 73.56258422412067) bank54048 +54049 POINT(39.727928763434726 73.99763321535467) bank54049 +54050 POINT(41.13398728082568 74.44624265366298) bank54050 +54051 POINT(40.607741517460006 73.9824146486773) bank54051 +54052 POINT(39.977555700573426 74.85449984807241) bank54052 +54053 POINT(41.462460126043155 73.71915749327889) bank54053 +54054 POINT(40.99359902010517 74.13396345920562) bank54054 +54055 POINT(41.62726476562696 74.36409780392353) bank54055 +54056 POINT(39.948130417230594 74.38879633487889) bank54056 +54057 POINT(40.28044448732098 74.72969624169332) bank54057 +54058 POINT(41.43646796524409 74.93232758889211) bank54058 +54059 POINT(41.70908420460919 74.8590672500329) bank54059 +54060 POINT(41.621908099580125 74.79397472344952) bank54060 +54061 POINT(41.70419976618034 73.5221284100688) bank54061 +54062 POINT(40.5247212343829 73.4728434331205) bank54062 +54063 POINT(39.859480356652284 73.11258671594834) bank54063 +54064 POINT(39.98991888008117 74.09057865289196) bank54064 +54065 POINT(40.16048032544302 74.27525027863229) bank54065 +54066 POINT(40.677207450828654 74.71397931991837) bank54066 +54067 POINT(40.08708542704068 73.3998998082661) bank54067 +54068 POINT(41.028798392080375 73.51508383463624) bank54068 +54069 POINT(41.489335817317404 73.8226245384102) bank54069 +54070 POINT(40.22221559074983 73.56179904085008) bank54070 +54071 POINT(41.31807360174226 74.45334396650505) bank54071 +54072 POINT(39.74433221719045 73.0116967582756) bank54072 +54073 POINT(41.47327867730576 73.73462335279915) bank54073 +54074 POINT(40.21351384808961 73.0991755507069) bank54074 +54075 POINT(39.86965020455511 73.9505774570591) bank54075 +54076 POINT(41.08130696941289 73.29207808693437) bank54076 +54077 POINT(40.56205189817266 73.70857504198841) bank54077 +54078 POINT(41.01484301480832 74.13722204364191) bank54078 +54079 POINT(39.94546513570601 73.07284002563121) bank54079 +54080 POINT(40.96941353305952 74.88155002463317) bank54080 +54081 POINT(40.650161422918586 74.65971109042074) bank54081 +54082 POINT(40.7488842184819 73.70356365100128) bank54082 +54083 POINT(41.17391340251242 73.79102465361177) bank54083 +54084 POINT(40.57487593717138 73.36567165951497) bank54084 +54085 POINT(40.32827426250067 74.10749908482424) bank54085 +54086 POINT(40.87060774979644 74.67239069181831) bank54086 +54087 POINT(40.141093575881 73.45347557182393) bank54087 +54088 POINT(40.98526327287841 74.12139818485012) bank54088 +54089 POINT(41.54626641198875 73.80767421628397) bank54089 +54090 POINT(40.58963296525206 73.94102791191526) bank54090 +54091 POINT(41.617423531879204 73.20459846300123) bank54091 +54092 POINT(39.760801459275044 73.11772841718103) bank54092 +54093 POINT(40.557053089058535 74.17016548721885) bank54093 +54094 POINT(40.56510259089347 74.53707671856765) bank54094 +54095 POINT(39.91334642623911 74.41186341225112) bank54095 +54096 POINT(39.76406035750058 73.0324990299636) bank54096 +54097 POINT(41.25186178826472 73.96975398212389) bank54097 +54098 POINT(40.23652529248283 73.3567728095845) bank54098 +54099 POINT(40.871996606321254 73.83941188974568) bank54099 +54100 POINT(39.80452911371325 73.14085607217228) bank54100 +54101 POINT(39.714588399773355 73.54100842684085) bank54101 +54102 POINT(41.55317666075528 73.65210296129892) bank54102 +54103 POINT(41.6996451285097 73.19791095283567) bank54103 +54104 POINT(40.74993261409273 73.16080713537364) bank54104 +54105 POINT(41.36685290886045 73.11713330056587) bank54105 +54106 POINT(40.273041714743066 73.23082967511698) bank54106 +54107 POINT(40.55053878917169 74.0253387558158) bank54107 +54108 POINT(40.54572782984138 73.42965851383369) bank54108 +54109 POINT(40.261909308279954 74.24644459407175) bank54109 +54110 POINT(40.23338089522906 74.48177001265644) bank54110 +54111 POINT(40.27770017912383 74.95958882341893) bank54111 +54112 POINT(40.49028777640125 73.61584864639345) bank54112 +54113 POINT(41.16145529473557 73.84136162322172) bank54113 +54114 POINT(40.582313335255435 73.90590740524641) bank54114 +54115 POINT(40.93827897648244 73.32594593921982) bank54115 +54116 POINT(39.97301293534685 73.92309736030353) bank54116 +54117 POINT(39.89421688528414 73.88343464640423) bank54117 +54118 POINT(41.62341482509361 74.61761750061572) bank54118 +54119 POINT(40.785860376187486 73.54820161836496) bank54119 +54120 POINT(39.933591245727584 74.28628678247537) bank54120 +54121 POINT(40.05984776376843 73.1640223442259) bank54121 +54122 POINT(39.94823424264446 75.00450751878007) bank54122 +54123 POINT(41.090412698870175 74.99072294788915) bank54123 +54124 POINT(40.52246996865585 74.11845893806422) bank54124 +54125 POINT(40.30691648155561 73.34296736596391) bank54125 +54126 POINT(41.20101155009333 74.47015300284929) bank54126 +54127 POINT(39.92002611495114 74.54451299653944) bank54127 +54128 POINT(40.02690137248697 74.94743270287766) bank54128 +54129 POINT(41.13218458985603 73.53828564579261) bank54129 +54130 POINT(40.31650195592966 74.02713095036297) bank54130 +54131 POINT(40.11814591635522 74.85341719011781) bank54131 +54132 POINT(40.30713280427767 73.62452348402022) bank54132 +54133 POINT(40.13478803937103 73.9874976554716) bank54133 +54134 POINT(40.98934922031321 73.9777233860244) bank54134 +54135 POINT(39.72000077354271 74.71396879276216) bank54135 +54136 POINT(40.61131511200159 73.77729035498174) bank54136 +54137 POINT(41.08011647021427 73.70384984830966) bank54137 +54138 POINT(40.60071526319672 73.58464001392373) bank54138 +54139 POINT(40.99262385613128 73.66870696225699) bank54139 +54140 POINT(41.18479411619348 74.6653285083413) bank54140 +54141 POINT(41.67189823449782 73.08673126952247) bank54141 +54142 POINT(41.30001575987275 73.03380271519447) bank54142 +54143 POINT(41.67113682217365 74.93373420805314) bank54143 +54144 POINT(40.27775140594824 73.97140156865032) bank54144 +54145 POINT(40.89471936855918 73.5474782549956) bank54145 +54146 POINT(40.13829838702162 74.54243797965692) bank54146 +54147 POINT(39.9924959177894 73.41246029424026) bank54147 +54148 POINT(40.56430532267463 74.7711547030633) bank54148 +54149 POINT(40.65545951255461 73.400316048856) bank54149 +54150 POINT(40.24411975502723 73.87272543687615) bank54150 +54151 POINT(41.544855902844084 73.20799277552014) bank54151 +54152 POINT(41.434855818135496 73.1842259934668) bank54152 +54153 POINT(40.709368300524254 73.24569425612324) bank54153 +54154 POINT(40.33995457717415 74.32958619977038) bank54154 +54155 POINT(40.98426603878418 73.42356581183533) bank54155 +54156 POINT(40.93922383270069 74.83011541604645) bank54156 +54157 POINT(39.82664009067512 74.41603100639236) bank54157 +54158 POINT(41.2249282738443 73.99079683656856) bank54158 +54159 POINT(40.268154218319935 73.15816048407702) bank54159 +54160 POINT(40.800128835111494 73.12143216168354) bank54160 +54161 POINT(39.886269170588584 73.15583371654583) bank54161 +54162 POINT(40.290390386069475 73.83091839579401) bank54162 +54163 POINT(40.66383084432984 73.93659252108753) bank54163 +54164 POINT(39.894054209032824 73.31466377759082) bank54164 +54165 POINT(40.07559620680358 74.52158497155604) bank54165 +54166 POINT(40.55690404288575 73.18212434606161) bank54166 +54167 POINT(39.878660350267715 74.6436390682469) bank54167 +54168 POINT(41.6360411547196 74.14624119625964) bank54168 +54169 POINT(41.18557473890676 73.0988831538084) bank54169 +54170 POINT(40.405211323024005 73.15130574189895) bank54170 +54171 POINT(40.265090988070924 73.13924475765752) bank54171 +54172 POINT(41.02793339275875 73.6903007291952) bank54172 +54173 POINT(41.27757420824036 73.65184698732105) bank54173 +54174 POINT(40.96403802001228 74.72947259517407) bank54174 +54175 POINT(40.87978288637388 73.90590295174516) bank54175 +54176 POINT(40.4222881048326 74.49612172942206) bank54176 +54177 POINT(39.718629987118426 74.68424156361543) bank54177 +54178 POINT(40.77130648372838 74.61130302362422) bank54178 +54179 POINT(40.49289270186727 73.70600962669869) bank54179 +54180 POINT(40.19718487224781 73.93877068035108) bank54180 +54181 POINT(40.25013079309338 73.671691549001) bank54181 +54182 POINT(41.17387339570669 73.31487330473954) bank54182 +54183 POINT(41.47930693674101 74.06379577114211) bank54183 +54184 POINT(40.42879810398272 73.94146032309662) bank54184 +54185 POINT(40.418734879170295 74.35501864866109) bank54185 +54186 POINT(40.785208006329 73.79525861710863) bank54186 +54187 POINT(40.53003593030709 74.85189031297755) bank54187 +54188 POINT(41.007898982422226 73.02390268132767) bank54188 +54189 POINT(40.361933335503835 74.06790512310259) bank54189 +54190 POINT(41.433191460863604 73.14617522342283) bank54190 +54191 POINT(39.80257664974807 74.442817636396) bank54191 +54192 POINT(41.30943978182314 74.72745523533426) bank54192 +54193 POINT(39.88097790008621 74.69333628247418) bank54193 +54194 POINT(41.44114795147071 73.95159675392831) bank54194 +54195 POINT(40.79580415407093 74.65648170790224) bank54195 +54196 POINT(40.305446420068236 74.6074405440808) bank54196 +54197 POINT(40.69180750853582 74.07302542047495) bank54197 +54198 POINT(39.82610002415361 74.00547729388566) bank54198 +54199 POINT(40.33620333193227 74.23291747178386) bank54199 +54200 POINT(40.22146103794783 74.84209418748777) bank54200 +54201 POINT(40.502827910351364 73.23983800323053) bank54201 +54202 POINT(40.5306577379449 73.74084101716834) bank54202 +54203 POINT(40.42382923547425 73.15581877062914) bank54203 +54204 POINT(41.01517675467547 73.45191680047787) bank54204 +54205 POINT(41.61490309843499 74.01097868924347) bank54205 +54206 POINT(40.49193042909867 74.8728122330006) bank54206 +54207 POINT(40.74553786074113 73.30111476452043) bank54207 +54208 POINT(41.5818915984222 74.97582743795697) bank54208 +54209 POINT(39.771663559130374 74.20876187711856) bank54209 +54210 POINT(40.8382194993884 74.82211718163356) bank54210 +54211 POINT(39.93185374118272 73.7767562425764) bank54211 +54212 POINT(40.65838475367386 74.92240275932255) bank54212 +54213 POINT(40.43343886263229 74.09564102943021) bank54213 +54214 POINT(40.80262778263359 73.85254407155504) bank54214 +54215 POINT(40.816311656030855 74.14240850552994) bank54215 +54216 POINT(41.49676388625518 74.82543844890203) bank54216 +54217 POINT(39.887616723651334 74.60318132258085) bank54217 +54218 POINT(41.204467429633205 74.318672489919) bank54218 +54219 POINT(39.94667051060238 74.44992311010809) bank54219 +54220 POINT(41.08563247008132 74.51076108919692) bank54220 +54221 POINT(40.527714990941654 73.92984699617224) bank54221 +54222 POINT(40.35446046955348 74.76994728369525) bank54222 +54223 POINT(40.29354695650911 74.08508674284552) bank54223 +54224 POINT(41.337157859084016 74.81415061428869) bank54224 +54225 POINT(41.180938360019695 73.82487339261398) bank54225 +54226 POINT(41.4830437975167 73.22686801480765) bank54226 +54227 POINT(41.70175002858429 73.5123645247662) bank54227 +54228 POINT(39.87178303511522 73.67234208967243) bank54228 +54229 POINT(40.08719373957298 74.1428887977208) bank54229 +54230 POINT(41.66386702958559 74.7492434834598) bank54230 +54231 POINT(40.06659709536399 74.59551809881806) bank54231 +54232 POINT(40.67619539304964 73.38544567624442) bank54232 +54233 POINT(41.6781640054816 73.95968695182421) bank54233 +54234 POINT(40.02701304593489 73.98117713546458) bank54234 +54235 POINT(40.53634435485392 74.04117301640143) bank54235 +54236 POINT(41.03374830742507 73.65925130107071) bank54236 +54237 POINT(39.92525996834213 74.54431950924499) bank54237 +54238 POINT(39.81025360082563 74.88888793076342) bank54238 +54239 POINT(40.01551834740862 74.50503527139041) bank54239 +54240 POINT(39.881104651801095 73.38844055198872) bank54240 +54241 POINT(40.93129695857856 73.91178974526456) bank54241 +54242 POINT(41.30173797289468 74.16945503162118) bank54242 +54243 POINT(39.886147593989854 74.5628910684314) bank54243 +54244 POINT(40.20802861707833 74.31449867961142) bank54244 +54245 POINT(41.45523852141134 73.84109332416386) bank54245 +54246 POINT(39.878568622668396 74.26763966110191) bank54246 +54247 POINT(40.77332237597948 74.16532050022313) bank54247 +54248 POINT(40.38612212532645 74.51269064384508) bank54248 +54249 POINT(39.759389715375875 73.32007164313951) bank54249 +54250 POINT(40.24014449008941 73.64485616057136) bank54250 +54251 POINT(41.5860140463993 74.43783785185529) bank54251 +54252 POINT(40.05832098864299 74.34146818857472) bank54252 +54253 POINT(40.68195420349753 73.89586526947578) bank54253 +54254 POINT(40.53051625430463 73.88442052780549) bank54254 +54255 POINT(41.10814820425429 74.84460321394694) bank54255 +54256 POINT(41.033920869244874 74.63826722227176) bank54256 +54257 POINT(40.286526315664695 74.57189529405419) bank54257 +54258 POINT(40.409942773641056 74.95599356982301) bank54258 +54259 POINT(39.83239907646006 74.68774428624769) bank54259 +54260 POINT(41.46949462353339 73.02788365961932) bank54260 +54261 POINT(40.7826479334963 73.44019750877676) bank54261 +54262 POINT(39.78602838448704 74.84292549136138) bank54262 +54263 POINT(40.53570468805015 73.11353109910729) bank54263 +54264 POINT(40.18114747212223 73.80830371331172) bank54264 +54265 POINT(40.59516669331678 74.32700313588343) bank54265 +54266 POINT(40.794253070196575 74.13110936204305) bank54266 +54267 POINT(40.7799338571803 73.91621792988074) bank54267 +54268 POINT(40.066233639752205 74.23522725458933) bank54268 +54269 POINT(39.80147010955116 73.76700681492619) bank54269 +54270 POINT(40.73704373738479 73.54762519849386) bank54270 +54271 POINT(40.84039842975671 73.25709884907799) bank54271 +54272 POINT(41.47752351357076 74.3474882198159) bank54272 +54273 POINT(40.606503140053455 74.15743617427876) bank54273 +54274 POINT(40.603146989610416 74.2366153542399) bank54274 +54275 POINT(41.01788904142529 73.2020471475118) bank54275 +54276 POINT(41.365025307227576 74.02964645901163) bank54276 +54277 POINT(40.14623213748635 73.48463590960297) bank54277 +54278 POINT(41.07662443189007 73.07821354633364) bank54278 +54279 POINT(40.49599345958903 74.97038553453831) bank54279 +54280 POINT(40.14160014176643 74.51103282665922) bank54280 +54281 POINT(39.83325845406118 74.31384062401308) bank54281 +54282 POINT(40.3579820632449 73.1222563531433) bank54282 +54283 POINT(40.86107392499297 74.92311172287465) bank54283 +54284 POINT(39.876094786503145 73.28539148511672) bank54284 +54285 POINT(40.780449096024135 73.969394147607) bank54285 +54286 POINT(40.59300335852115 74.74424920867975) bank54286 +54287 POINT(41.49071256144242 74.48501977179365) bank54287 +54288 POINT(40.397684423140674 74.89661695151234) bank54288 +54289 POINT(40.10453143039702 73.1738363764917) bank54289 +54290 POINT(41.37551087559388 74.24740819630296) bank54290 +54291 POINT(41.46474768732101 73.44059950289272) bank54291 +54292 POINT(41.6247515140663 74.10843214264297) bank54292 +54293 POINT(40.90299433572313 73.66958184027101) bank54293 +54294 POINT(39.875059516491795 73.60167702001732) bank54294 +54295 POINT(41.22501217126268 73.55573673219062) bank54295 +54296 POINT(40.41762447379473 73.86663018375452) bank54296 +54297 POINT(40.901425321510644 73.71808525328879) bank54297 +54298 POINT(41.652531744235944 73.30382919539767) bank54298 +54299 POINT(41.10703293789508 73.95369036686826) bank54299 +54300 POINT(41.42815372221333 74.36436486621595) bank54300 +54301 POINT(40.869465920248125 73.90303867279955) bank54301 +54302 POINT(41.67050815024835 74.32704410762004) bank54302 +54303 POINT(41.097758811195575 74.1144045991464) bank54303 +54304 POINT(40.208902691823596 74.79416372960449) bank54304 +54305 POINT(40.80339894268657 73.72918713659222) bank54305 +54306 POINT(41.600198277768214 73.42911495733077) bank54306 +54307 POINT(41.14875062017393 73.80813857492765) bank54307 +54308 POINT(40.450035305579476 73.51348428267904) bank54308 +54309 POINT(41.14230104343123 74.44127268984441) bank54309 +54310 POINT(40.6296965871024 73.70959733299651) bank54310 +54311 POINT(41.48863934772784 74.76212010820035) bank54311 +54312 POINT(39.82919247699815 73.63576375653389) bank54312 +54313 POINT(40.10947315769069 74.98579226502827) bank54313 +54314 POINT(40.79385380808712 74.6968626250428) bank54314 +54315 POINT(39.96963456568061 73.44746719498181) bank54315 +54316 POINT(40.172341960812346 74.81103240300892) bank54316 +54317 POINT(40.8043633917041 74.99137255840202) bank54317 +54318 POINT(41.132378359239965 74.9638903690371) bank54318 +54319 POINT(40.47819556477603 73.92615904407161) bank54319 +54320 POINT(40.750165896320695 74.61726178769062) bank54320 +54321 POINT(41.53662756693948 73.48461908926227) bank54321 +54322 POINT(41.00915179191984 73.46956991163829) bank54322 +54323 POINT(40.46014889262516 74.49023908111278) bank54323 +54324 POINT(41.03448193131782 74.40146486377124) bank54324 +54325 POINT(40.5950755048251 74.34150416898841) bank54325 +54326 POINT(41.07532779118879 73.12229678489602) bank54326 +54327 POINT(40.89476332171283 74.5479882946706) bank54327 +54328 POINT(41.284630270038654 74.29778600077026) bank54328 +54329 POINT(41.20954545662806 73.69236939715448) bank54329 +54330 POINT(40.971652881990344 73.69853635003501) bank54330 +54331 POINT(40.99764922066394 74.08297555427576) bank54331 +54332 POINT(41.02688246168436 74.67182419309675) bank54332 +54333 POINT(41.55001801501887 74.33502258457492) bank54333 +54334 POINT(41.27232309710242 74.82007377621264) bank54334 +54335 POINT(40.34613597036178 73.1780231397046) bank54335 +54336 POINT(40.85269696427696 74.69396133034822) bank54336 +54337 POINT(40.86113124285414 73.55741119885488) bank54337 +54338 POINT(40.49961037365666 73.63330783042576) bank54338 +54339 POINT(41.00275541826635 73.80500923902123) bank54339 +54340 POINT(40.49405907850306 74.95089633896542) bank54340 +54341 POINT(40.06919471904276 73.58331860706602) bank54341 +54342 POINT(41.26396863000344 74.23932950752486) bank54342 +54343 POINT(39.71799397276336 74.74221653580454) bank54343 +54344 POINT(41.610637491473675 73.45945042908815) bank54344 +54345 POINT(39.82723317956237 74.52834229168803) bank54345 +54346 POINT(40.569841004451845 73.21802332629443) bank54346 +54347 POINT(40.734895640392196 74.69004694405957) bank54347 +54348 POINT(41.62054273444351 74.8784311883173) bank54348 +54349 POINT(39.94455160379204 74.51771182085113) bank54349 +54350 POINT(40.75739537545122 73.13501879098338) bank54350 +54351 POINT(41.1669268350901 74.68679146641344) bank54351 +54352 POINT(40.56503317493121 74.47311583662844) bank54352 +54353 POINT(40.44183295919574 73.68622125611394) bank54353 +54354 POINT(41.10706975680334 74.20726161306203) bank54354 +54355 POINT(41.60364269038623 74.37008168011235) bank54355 +54356 POINT(40.13812505311798 73.30319775869376) bank54356 +54357 POINT(40.04983112880946 74.50154637286062) bank54357 +54358 POINT(40.03579186124473 73.14449052083063) bank54358 +54359 POINT(40.2688729248879 73.89550162819883) bank54359 +54360 POINT(40.26720586337329 73.01767869814009) bank54360 +54361 POINT(41.25064716618083 73.24761403074103) bank54361 +54362 POINT(41.211687158169305 73.23127992624923) bank54362 +54363 POINT(41.223384167369915 73.97018520821742) bank54363 +54364 POINT(40.594185749201046 74.94689615166149) bank54364 +54365 POINT(40.22961324651502 74.46439425007911) bank54365 +54366 POINT(41.604715302339415 73.9171827085747) bank54366 +54367 POINT(40.79724725259664 74.30631901662375) bank54367 +54368 POINT(40.73926619561708 74.16989335285216) bank54368 +54369 POINT(39.90495923800128 73.32507149663645) bank54369 +54370 POINT(40.74265707374767 74.2272521707144) bank54370 +54371 POINT(40.030981004577654 73.77139985106493) bank54371 +54372 POINT(39.95741656468769 73.6398667096948) bank54372 +54373 POINT(41.24453755282131 73.34869388327697) bank54373 +54374 POINT(40.22614350998325 74.19614272609283) bank54374 +54375 POINT(41.449526738720756 74.63268698231325) bank54375 +54376 POINT(39.82636192459991 73.32342309240094) bank54376 +54377 POINT(39.764601329072484 73.82166603552568) bank54377 +54378 POINT(40.01316866053112 73.82404029588187) bank54378 +54379 POINT(40.30405574557294 73.32908300341984) bank54379 +54380 POINT(41.18917766455915 73.91115999046637) bank54380 +54381 POINT(40.849431507859066 74.18001424566093) bank54381 +54382 POINT(40.33577583495767 73.80122471801849) bank54382 +54383 POINT(40.650462613549244 74.7277835315839) bank54383 +54384 POINT(39.97124940861435 73.2581367101385) bank54384 +54385 POINT(41.15964770528669 74.35830812011079) bank54385 +54386 POINT(41.565315305585955 73.98449910869907) bank54386 +54387 POINT(40.728378533420624 73.12098176839393) bank54387 +54388 POINT(41.4700880277569 73.88563172376217) bank54388 +54389 POINT(41.58805245386568 73.07504990921151) bank54389 +54390 POINT(40.32856380284885 74.75525985793868) bank54390 +54391 POINT(39.911358959000225 74.40021766036386) bank54391 +54392 POINT(40.58020455849857 73.03187771336596) bank54392 +54393 POINT(41.26566597302115 74.86719452409054) bank54393 +54394 POINT(39.8398522694489 73.75173766715109) bank54394 +54395 POINT(40.33723798601508 73.72235238907471) bank54395 +54396 POINT(40.95667741801627 73.41390967709653) bank54396 +54397 POINT(40.56432147387285 74.18415384449978) bank54397 +54398 POINT(40.04553671586771 74.9231644887043) bank54398 +54399 POINT(41.68868049544312 74.33816717591552) bank54399 +54400 POINT(41.41195644861401 73.39374411511061) bank54400 +54401 POINT(39.94257980917068 73.69221878010946) bank54401 +54402 POINT(39.734588225234496 73.11751664180368) bank54402 +54403 POINT(40.68492490902162 73.82093030349715) bank54403 +54404 POINT(40.883254379050165 73.90275049306814) bank54404 +54405 POINT(41.6726876477717 74.81361876864037) bank54405 +54406 POINT(41.12025528107794 73.37865093295413) bank54406 +54407 POINT(41.50688304751788 73.06559507364388) bank54407 +54408 POINT(40.806261881057026 73.17182437111065) bank54408 +54409 POINT(40.20559183926543 73.81287377131635) bank54409 +54410 POINT(41.087650391156764 74.35186753740962) bank54410 +54411 POINT(40.673701514563405 74.43635282718925) bank54411 +54412 POINT(41.61093484119471 73.46217407177396) bank54412 +54413 POINT(41.57920973683561 74.24281518424507) bank54413 +54414 POINT(41.28096347190782 73.20923159923342) bank54414 +54415 POINT(40.291307068833376 73.59172023984239) bank54415 +54416 POINT(40.2924258353825 74.66746268791697) bank54416 +54417 POINT(39.9377204241266 73.47225529067366) bank54417 +54418 POINT(40.12728621729244 73.29954467142008) bank54418 +54419 POINT(41.207733596026465 74.96944251083313) bank54419 +54420 POINT(40.84061691486873 73.79065393851853) bank54420 +54421 POINT(40.976868112613666 74.19431640159395) bank54421 +54422 POINT(40.196157145087845 74.20840867356146) bank54422 +54423 POINT(41.30835887401562 73.26614659049424) bank54423 +54424 POINT(41.59348283676496 73.28914111260515) bank54424 +54425 POINT(40.929955691838884 74.18781749730022) bank54425 +54426 POINT(40.019754767163576 74.3613946765131) bank54426 +54427 POINT(41.27896191475622 74.03780783858727) bank54427 +54428 POINT(41.655808338672294 73.99567964789773) bank54428 +54429 POINT(40.316875374190055 74.89449416015154) bank54429 +54430 POINT(39.84965526818583 74.14015534199731) bank54430 +54431 POINT(41.58855801525863 73.61852506412647) bank54431 +54432 POINT(39.968630430214255 73.63722506446891) bank54432 +54433 POINT(40.21475600394881 74.15475968415386) bank54433 +54434 POINT(39.845275624366664 74.83246462496791) bank54434 +54435 POINT(40.39582033156531 73.90219221736903) bank54435 +54436 POINT(39.845852053667684 73.5360357305403) bank54436 +54437 POINT(39.75921315533703 74.34941320282084) bank54437 +54438 POINT(40.0060564722953 73.52877875316067) bank54438 +54439 POINT(40.73161978480298 73.3338117393336) bank54439 +54440 POINT(40.56986928068621 73.06893333594593) bank54440 +54441 POINT(40.90002952245492 74.07033080682383) bank54441 +54442 POINT(41.08607363066541 74.50327901897307) bank54442 +54443 POINT(40.20661048895705 73.58371832351409) bank54443 +54444 POINT(40.063649321284224 74.96947872441866) bank54444 +54445 POINT(41.107855905067 73.74373261281545) bank54445 +54446 POINT(40.25620517364581 74.73302891501156) bank54446 +54447 POINT(41.4292712806745 73.07797401103437) bank54447 +54448 POINT(39.95056274660447 73.26022424682496) bank54448 +54449 POINT(41.440306085186364 73.93732347675083) bank54449 +54450 POINT(40.34905644919469 73.05040229553994) bank54450 +54451 POINT(40.95219184783186 73.88711519414507) bank54451 +54452 POINT(39.8149161757001 73.88695400054188) bank54452 +54453 POINT(41.55123421541412 73.45115935512452) bank54453 +54454 POINT(40.41667976713565 73.27533883106965) bank54454 +54455 POINT(39.939999648711435 74.47725099525252) bank54455 +54456 POINT(40.54023245163085 74.41954725630403) bank54456 +54457 POINT(41.41965895205658 74.06958812390084) bank54457 +54458 POINT(41.18755203513372 73.20671251380908) bank54458 +54459 POINT(41.07957548995272 73.95261115203695) bank54459 +54460 POINT(40.47750106036199 73.78663705796431) bank54460 +54461 POINT(40.000408378193264 74.47189582959712) bank54461 +54462 POINT(41.20371224308597 73.8028798486032) bank54462 +54463 POINT(40.10689812984371 74.74747068881673) bank54463 +54464 POINT(40.59568526733225 74.89121231512742) bank54464 +54465 POINT(40.28873565092182 74.06558445477134) bank54465 +54466 POINT(41.24621538486302 73.12753491130889) bank54466 +54467 POINT(40.55825466892278 74.90703659953012) bank54467 +54468 POINT(40.31428411554217 74.45415629472154) bank54468 +54469 POINT(41.43192551667899 74.81718544070212) bank54469 +54470 POINT(40.15163482978558 73.55938127024416) bank54470 +54471 POINT(40.45330914840857 73.28149395467601) bank54471 +54472 POINT(41.26336721626996 74.08868888092644) bank54472 +54473 POINT(40.025728771346785 74.20852740211367) bank54473 +54474 POINT(40.031773232874535 74.46473649891003) bank54474 +54475 POINT(40.37911427363198 74.82208011002373) bank54475 +54476 POINT(39.8982934618591 73.15080483267631) bank54476 +54477 POINT(41.42119465955212 73.58017347714825) bank54477 +54478 POINT(40.30093471441929 74.58666476146942) bank54478 +54479 POINT(41.48908132632789 73.98171057088491) bank54479 +54480 POINT(40.38213233785323 74.8298250588676) bank54480 +54481 POINT(40.45018375740592 74.72155642454524) bank54481 +54482 POINT(40.94625864773474 73.12489493713456) bank54482 +54483 POINT(40.02973626530854 74.05894801856202) bank54483 +54484 POINT(41.336495301336406 74.65164288945407) bank54484 +54485 POINT(41.153655087793226 74.83197719431605) bank54485 +54486 POINT(40.10450791361795 73.18089365977305) bank54486 +54487 POINT(39.78243074832503 73.89910024784025) bank54487 +54488 POINT(41.529481769531365 74.05473846739191) bank54488 +54489 POINT(39.71564852571962 73.04848621607849) bank54489 +54490 POINT(40.81662185978348 73.59016560494216) bank54490 +54491 POINT(40.662251352203015 73.56776668227425) bank54491 +54492 POINT(39.906298947980815 74.52728443140843) bank54492 +54493 POINT(41.460745195884485 74.19747423467577) bank54493 +54494 POINT(41.683096639917096 74.36333901625267) bank54494 +54495 POINT(41.31521366409391 73.01775115244087) bank54495 +54496 POINT(41.07916488878398 74.25430562172188) bank54496 +54497 POINT(39.84804558574356 73.79849195079743) bank54497 +54498 POINT(41.24163689120475 74.40301352081119) bank54498 +54499 POINT(41.073522556806836 74.28976243982797) bank54499 +54500 POINT(39.994235166241936 73.02148395010076) bank54500 +54501 POINT(40.23667281499626 73.35642939256277) bank54501 +54502 POINT(41.6787181253 74.13072225390682) bank54502 +54503 POINT(41.03898455044191 73.23727664136342) bank54503 +54504 POINT(41.38935183035151 74.32847559999242) bank54504 +54505 POINT(41.48026748386012 74.64345233325008) bank54505 +54506 POINT(40.23807261063196 74.72276581707386) bank54506 +54507 POINT(41.33439715616952 74.15477236102373) bank54507 +54508 POINT(40.8364103626402 74.1634763610102) bank54508 +54509 POINT(40.277878721405514 73.36622933075023) bank54509 +54510 POINT(41.07681259374239 73.91843288949254) bank54510 +54511 POINT(40.038786935892425 74.18590251216354) bank54511 +54512 POINT(40.02485154901785 73.77594947426655) bank54512 +54513 POINT(41.30371568400583 73.50150442769993) bank54513 +54514 POINT(41.07372231997633 74.01050152183822) bank54514 +54515 POINT(39.78135161150921 74.53774197951829) bank54515 +54516 POINT(40.68605375927379 74.66061627638649) bank54516 +54517 POINT(41.40340778748341 73.24999343501456) bank54517 +54518 POINT(41.203299768687856 74.86002367910514) bank54518 +54519 POINT(41.15825024921362 74.73393338237713) bank54519 +54520 POINT(40.20506575396958 74.18698899983906) bank54520 +54521 POINT(40.0237991182693 73.8777662285451) bank54521 +54522 POINT(41.68048117290869 73.46265426334821) bank54522 +54523 POINT(39.96825349806399 73.70247989334982) bank54523 +54524 POINT(41.42648576245182 73.73325433317419) bank54524 +54525 POINT(41.48706998461532 74.65910768616364) bank54525 +54526 POINT(40.32890554529289 74.44563335562256) bank54526 +54527 POINT(41.35128125159414 73.47604079518281) bank54527 +54528 POINT(40.49636552743306 73.10482672036115) bank54528 +54529 POINT(40.518864868558644 73.00759682530284) bank54529 +54530 POINT(39.94234761331519 74.58127346438707) bank54530 +54531 POINT(41.516845902767976 74.08924209035501) bank54531 +54532 POINT(39.88029702299728 73.83162271337117) bank54532 +54533 POINT(41.153281302381345 74.27930537398593) bank54533 +54534 POINT(40.745537473755306 73.3309281899773) bank54534 +54535 POINT(41.15379342470858 74.63887742136713) bank54535 +54536 POINT(40.68122723464378 74.40983041650128) bank54536 +54537 POINT(40.11477009074976 73.78647386132968) bank54537 +54538 POINT(40.54598788215272 74.26511478018305) bank54538 +54539 POINT(41.59783902574318 73.85332472185304) bank54539 +54540 POINT(40.0224341376017 74.27717688999354) bank54540 +54541 POINT(40.923904796301436 73.26569604333253) bank54541 +54542 POINT(40.206235391086935 74.16688506583804) bank54542 +54543 POINT(40.209323183712385 74.48459086507542) bank54543 +54544 POINT(40.59915602852844 74.62512863348802) bank54544 +54545 POINT(40.15767634724789 74.19084455999236) bank54545 +54546 POINT(40.91506875865915 74.17739494546802) bank54546 +54547 POINT(41.35080763307684 73.34147341173166) bank54547 +54548 POINT(41.01513543873663 74.47265855133105) bank54548 +54549 POINT(39.87490261577558 74.8504501268792) bank54549 +54550 POINT(40.3594868694976 73.75553703003764) bank54550 +54551 POINT(41.37293759483878 74.19734846060476) bank54551 +54552 POINT(41.525078297836096 74.68782724505193) bank54552 +54553 POINT(41.16221441070184 73.0229347500811) bank54553 +54554 POINT(40.676924322558904 74.80960827462702) bank54554 +54555 POINT(39.74509585480415 74.82008838768918) bank54555 +54556 POINT(41.13018861260118 73.43670893163097) bank54556 +54557 POINT(39.792335084900955 74.9343806608882) bank54557 +54558 POINT(41.622676429698906 73.06554220622195) bank54558 +54559 POINT(41.43874789455453 74.44921215395813) bank54559 +54560 POINT(41.41430833419349 73.18178167124312) bank54560 +54561 POINT(40.2632886047575 73.56980831995163) bank54561 +54562 POINT(40.82485509211118 74.95267499914512) bank54562 +54563 POINT(41.0959991661296 73.90356360763015) bank54563 +54564 POINT(40.265078949954585 73.9530239759049) bank54564 +54565 POINT(41.02411026934213 74.95431583872673) bank54565 +54566 POINT(40.533397104506136 74.56516344735155) bank54566 +54567 POINT(41.655200305086346 73.056315100477) bank54567 +54568 POINT(40.03805266693838 73.95812929990606) bank54568 +54569 POINT(41.70369796601258 73.23769428549252) bank54569 +54570 POINT(39.91609908990472 74.8770435475479) bank54570 +54571 POINT(40.58031168703872 73.9301264751798) bank54571 +54572 POINT(39.98271062895885 73.39100098700196) bank54572 +54573 POINT(41.157024430501096 74.28662915042696) bank54573 +54574 POINT(39.862744630513205 74.57583968436734) bank54574 +54575 POINT(40.42414041844898 73.55337760084801) bank54575 +54576 POINT(40.54991551014236 74.6298137874547) bank54576 +54577 POINT(41.35947928510519 73.63427103916146) bank54577 +54578 POINT(39.82984232660829 74.87080253343899) bank54578 +54579 POINT(40.37852698343699 73.23888214917939) bank54579 +54580 POINT(40.04992425136554 73.14210795906072) bank54580 +54581 POINT(41.57410565288786 74.33112122518291) bank54581 +54582 POINT(40.6408927611502 74.81427658795596) bank54582 +54583 POINT(41.56800090429552 73.75566820955007) bank54583 +54584 POINT(40.552210786492445 74.78158185691373) bank54584 +54585 POINT(40.0889273902041 74.82234626010386) bank54585 +54586 POINT(41.6667380765665 74.81801016650546) bank54586 +54587 POINT(40.634631717137694 74.46774443265501) bank54587 +54588 POINT(40.08882716158173 73.29017869880552) bank54588 +54589 POINT(41.44952106348848 73.09285338914715) bank54589 +54590 POINT(40.47858145112489 74.188025805949) bank54590 +54591 POINT(40.25350932445475 73.65277447678335) bank54591 +54592 POINT(40.226621095983845 73.97484180857572) bank54592 +54593 POINT(41.496154596698496 73.4253777794145) bank54593 +54594 POINT(39.73041596441285 73.71597283872207) bank54594 +54595 POINT(39.94348337722372 73.2858524378353) bank54595 +54596 POINT(41.101092946095655 74.90784269692091) bank54596 +54597 POINT(41.658342362252675 74.77133598917224) bank54597 +54598 POINT(41.2791162210414 74.32296699148982) bank54598 +54599 POINT(39.74175649450729 73.2795585272964) bank54599 +54600 POINT(40.17486944090005 74.95115594607489) bank54600 +54601 POINT(41.25843973009262 75.00243358617732) bank54601 +54602 POINT(40.64788001955641 73.09665886021915) bank54602 +54603 POINT(39.90386389185304 73.53765585995389) bank54603 +54604 POINT(40.32936161153512 73.33236370324062) bank54604 +54605 POINT(39.95571081441646 74.58540038730655) bank54605 +54606 POINT(41.53903693627275 74.90968492973431) bank54606 +54607 POINT(40.13857061196418 73.12577913036343) bank54607 +54608 POINT(39.72634644893853 74.31758469596907) bank54608 +54609 POINT(39.882924647504495 73.15274928242044) bank54609 +54610 POINT(41.36843827182512 73.27645922570362) bank54610 +54611 POINT(39.84566831370572 73.94843481063225) bank54611 +54612 POINT(40.65698350662279 73.30943455328263) bank54612 +54613 POINT(39.81826045063276 73.74827202391715) bank54613 +54614 POINT(40.53590759310227 74.13555451133918) bank54614 +54615 POINT(39.95176952065458 73.93503260277842) bank54615 +54616 POINT(40.194571672903066 74.16897929860845) bank54616 +54617 POINT(41.16078465350604 74.9864105541682) bank54617 +54618 POINT(40.33279947679186 73.48021670977403) bank54618 +54619 POINT(40.71405465753914 73.42484641598163) bank54619 +54620 POINT(39.76597786281725 74.10760443994187) bank54620 +54621 POINT(40.03511512662832 73.02637535534829) bank54621 +54622 POINT(40.58666424928201 73.04601005109143) bank54622 +54623 POINT(40.590289271188375 74.8750279350866) bank54623 +54624 POINT(40.690203593427334 74.11449194621171) bank54624 +54625 POINT(40.88361967535154 74.50029687106269) bank54625 +54626 POINT(40.9165430556738 73.32318397962332) bank54626 +54627 POINT(39.71685892820303 74.06343998896122) bank54627 +54628 POINT(41.133898809483746 73.7653178461342) bank54628 +54629 POINT(39.85661533382577 73.02245424456406) bank54629 +54630 POINT(41.47239510085992 74.38133873272507) bank54630 +54631 POINT(40.01802328687038 74.59406303692201) bank54631 +54632 POINT(40.862300498338996 73.9675778581598) bank54632 +54633 POINT(40.77912710199198 74.79741494901457) bank54633 +54634 POINT(39.72805964856754 73.09839505573605) bank54634 +54635 POINT(40.77311821156812 73.28393845608238) bank54635 +54636 POINT(40.720357606931415 74.62395341510248) bank54636 +54637 POINT(41.50991439919098 74.73955347012682) bank54637 +54638 POINT(41.543532784051926 74.16896325948908) bank54638 +54639 POINT(41.65038796235109 73.11142964866757) bank54639 +54640 POINT(41.61926831438272 74.30914126400296) bank54640 +54641 POINT(41.19634716908041 73.45037334838099) bank54641 +54642 POINT(41.35723035543652 73.29762206508047) bank54642 +54643 POINT(41.46653730222617 73.17926756443923) bank54643 +54644 POINT(40.716474902231106 74.20258945635055) bank54644 +54645 POINT(39.95767342960407 74.90924806742288) bank54645 +54646 POINT(40.48548971842972 74.2645818649467) bank54646 +54647 POINT(41.48988695889488 73.8945224274439) bank54647 +54648 POINT(39.93503546111827 73.35067703408167) bank54648 +54649 POINT(39.921583027096595 73.20424656470084) bank54649 +54650 POINT(40.33304449999488 73.35416029839487) bank54650 +54651 POINT(41.1198908156772 73.48410109617473) bank54651 +54652 POINT(40.64013949429392 73.4699442625174) bank54652 +54653 POINT(39.773823163523005 74.32583473871458) bank54653 +54654 POINT(40.51520549537189 73.0979431546532) bank54654 +54655 POINT(41.32325250903436 74.97270036932704) bank54655 +54656 POINT(40.47471661149404 74.14421556562874) bank54656 +54657 POINT(40.35403120418416 74.10814593544336) bank54657 +54658 POINT(40.436405282386175 73.61225260160685) bank54658 +54659 POINT(41.66492690777209 74.2651206119141) bank54659 +54660 POINT(41.68246276741903 73.78461202599347) bank54660 +54661 POINT(41.1952161970446 74.37367490462277) bank54661 +54662 POINT(39.950614053599146 74.06089430608725) bank54662 +54663 POINT(40.63601094364958 73.84924043729221) bank54663 +54664 POINT(41.40538521036524 73.87815054276122) bank54664 +54665 POINT(40.192123993659884 73.10648929991217) bank54665 +54666 POINT(39.882206110725726 74.1574339525277) bank54666 +54667 POINT(40.15039712104109 73.83950810774505) bank54667 +54668 POINT(41.13508154277664 73.06978912186021) bank54668 +54669 POINT(41.63927625984022 73.49925987811989) bank54669 +54670 POINT(41.31784503350365 74.00668240998402) bank54670 +54671 POINT(39.79631385251465 73.31645683775663) bank54671 +54672 POINT(41.51610157381131 73.40015672249734) bank54672 +54673 POINT(41.29338629519333 73.42829771164408) bank54673 +54674 POINT(41.655879180206625 74.18244990459945) bank54674 +54675 POINT(41.59791592272934 74.37942407417555) bank54675 +54676 POINT(40.44670548317898 73.51908158232702) bank54676 +54677 POINT(41.04016317223424 74.63732448947906) bank54677 +54678 POINT(41.487994415620705 74.03032592440029) bank54678 +54679 POINT(41.67143485836771 74.24730241407359) bank54679 +54680 POINT(41.24064541516551 73.42197195938549) bank54680 +54681 POINT(40.83639035382463 74.62047203401374) bank54681 +54682 POINT(41.147440424163385 73.01054309545367) bank54682 +54683 POINT(40.50607713315166 74.9797989847682) bank54683 +54684 POINT(40.16859653452535 74.56634904295107) bank54684 +54685 POINT(40.19471074377657 73.13240103194829) bank54685 +54686 POINT(40.666828761920854 74.20383251539822) bank54686 +54687 POINT(40.22555679394923 74.27905019946229) bank54687 +54688 POINT(39.876654336149805 74.27342724715247) bank54688 +54689 POINT(41.60735992913745 73.72559041178324) bank54689 +54690 POINT(40.19370220005723 74.66929073948897) bank54690 +54691 POINT(40.13656161030288 74.4245918479566) bank54691 +54692 POINT(40.50665918402756 73.26823027634684) bank54692 +54693 POINT(40.27189297444482 74.2316599224916) bank54693 +54694 POINT(41.33840097934906 74.17543620204997) bank54694 +54695 POINT(40.64456791776499 74.00307866827814) bank54695 +54696 POINT(40.160657192940036 74.49734475888786) bank54696 +54697 POINT(41.374897767168314 73.67349680817607) bank54697 +54698 POINT(40.007598903454905 74.14174986566141) bank54698 +54699 POINT(41.4429392250458 73.33844924514136) bank54699 +54700 POINT(40.4257591689521 74.84767105818872) bank54700 +54701 POINT(40.18108371547598 73.83796200893221) bank54701 +54702 POINT(41.02659605835045 74.01850325911765) bank54702 +54703 POINT(41.225352834773794 74.67937062589402) bank54703 +54704 POINT(41.46975611781752 73.10118093293283) bank54704 +54705 POINT(41.530383244814864 74.11782519295248) bank54705 +54706 POINT(39.87933637253788 73.40846235661934) bank54706 +54707 POINT(41.067812209978186 74.71068144368957) bank54707 +54708 POINT(40.00501333366656 73.72048917130411) bank54708 +54709 POINT(41.705071223474306 73.95868181657963) bank54709 +54710 POINT(40.865965089357665 73.18026448655743) bank54710 +54711 POINT(40.11517875661595 74.14415800731844) bank54711 +54712 POINT(41.04139691304207 73.62636204818709) bank54712 +54713 POINT(41.324060050859586 73.61486807867608) bank54713 +54714 POINT(41.42041959718721 73.05120703225455) bank54714 +54715 POINT(39.88099567175149 73.84133246392769) bank54715 +54716 POINT(40.64415115246247 73.16698437064444) bank54716 +54717 POINT(40.77093574341052 74.29804641262609) bank54717 +54718 POINT(41.179252789094825 73.53414683690387) bank54718 +54719 POINT(40.60324504988648 74.2281312683683) bank54719 +54720 POINT(41.320169575838996 73.87473158942036) bank54720 +54721 POINT(40.3484157832779 73.76218630604492) bank54721 +54722 POINT(41.2072308689092 74.86347351562435) bank54722 +54723 POINT(40.710117343177096 74.06970292459778) bank54723 +54724 POINT(39.77985021916975 74.69780367126405) bank54724 +54725 POINT(40.52309123900373 73.01706029114905) bank54725 +54726 POINT(40.53634716411404 73.82127920915339) bank54726 +54727 POINT(40.2507230209687 74.90559563119999) bank54727 +54728 POINT(40.9647244565754 74.96862885805352) bank54728 +54729 POINT(40.069667539458976 73.84969774035234) bank54729 +54730 POINT(40.21412257470385 74.58155827087893) bank54730 +54731 POINT(40.150305645863064 74.45376470432235) bank54731 +54732 POINT(39.88915392824693 74.69868024693523) bank54732 +54733 POINT(40.47600543364855 73.67740604562815) bank54733 +54734 POINT(39.85789812528856 74.38655870632286) bank54734 +54735 POINT(39.96420115741819 74.63359097385438) bank54735 +54736 POINT(41.03881349583392 73.4206260810617) bank54736 +54737 POINT(41.604643159516755 74.66993810754556) bank54737 +54738 POINT(41.698966897288464 73.23070496004121) bank54738 +54739 POINT(40.73128937748269 74.52318511863982) bank54739 +54740 POINT(40.189836425914706 73.17032447178006) bank54740 +54741 POINT(41.23941624838824 73.92217290294926) bank54741 +54742 POINT(39.8874454913167 73.5842369690513) bank54742 +54743 POINT(40.81067837651419 73.89304224887351) bank54743 +54744 POINT(40.875090965250905 73.27033091147142) bank54744 +54745 POINT(41.201526084997646 74.48208096392644) bank54745 +54746 POINT(40.574527970214845 74.66363639647355) bank54746 +54747 POINT(40.559842719905625 74.10778514831088) bank54747 +54748 POINT(40.84022476784086 74.01474190996665) bank54748 +54749 POINT(41.186314863896534 73.87994840435385) bank54749 +54750 POINT(40.383826774949995 74.88331777159515) bank54750 +54751 POINT(41.49126942934848 74.57086403967813) bank54751 +54752 POINT(41.488111505070385 74.16275878928977) bank54752 +54753 POINT(41.24524594620438 74.5440465492449) bank54753 +54754 POINT(41.4414975068002 74.24107911371696) bank54754 +54755 POINT(40.28708075098965 73.6230180484985) bank54755 +54756 POINT(40.81201077568993 73.48698952752164) bank54756 +54757 POINT(39.77693668464911 74.22678334516033) bank54757 +54758 POINT(39.890058845889044 73.50699217290014) bank54758 +54759 POINT(40.05214140563261 73.66445205519616) bank54759 +54760 POINT(40.678583925192356 73.2764632814875) bank54760 +54761 POINT(40.70743587541598 74.33489281988315) bank54761 +54762 POINT(41.60865093878224 74.61813070569157) bank54762 +54763 POINT(40.7571379582984 74.81979775733436) bank54763 +54764 POINT(40.08302620710282 73.21944145056288) bank54764 +54765 POINT(40.86411972132668 73.6708858040411) bank54765 +54766 POINT(40.188193640475504 74.9143732213869) bank54766 +54767 POINT(40.32358930441781 73.79339038396064) bank54767 +54768 POINT(40.38780406788747 73.18854773045919) bank54768 +54769 POINT(41.094823797646896 73.54303688095762) bank54769 +54770 POINT(40.48001993433448 73.80495398274992) bank54770 +54771 POINT(40.450319762474116 73.06824654799735) bank54771 +54772 POINT(41.32326560942317 73.94651103960439) bank54772 +54773 POINT(40.017210340369765 74.12053982337875) bank54773 +54774 POINT(41.33046558256527 73.36877387757355) bank54774 +54775 POINT(41.67392340826714 74.0651683151765) bank54775 +54776 POINT(41.57135661843571 74.90182577905826) bank54776 +54777 POINT(40.58241392199952 74.68445981123268) bank54777 +54778 POINT(40.859058210964726 73.52521159239444) bank54778 +54779 POINT(40.200321686928916 73.43474185951362) bank54779 +54780 POINT(40.74291853677468 74.22103459290503) bank54780 +54781 POINT(41.53528407906351 74.80676301523681) bank54781 +54782 POINT(41.39150338928815 73.86092386351999) bank54782 +54783 POINT(40.92724803900723 73.68794715606718) bank54783 +54784 POINT(40.92197954453532 73.65339405183363) bank54784 +54785 POINT(40.09430316262981 73.31186162532693) bank54785 +54786 POINT(41.709790338970954 74.73430203546889) bank54786 +54787 POINT(41.24259717540891 74.0848576142383) bank54787 +54788 POINT(40.36693377746483 74.00063640765784) bank54788 +54789 POINT(41.404319035283855 74.81288682741761) bank54789 +54790 POINT(39.94795163949723 73.62011853366944) bank54790 +54791 POINT(41.44919372355767 73.63469315290625) bank54791 +54792 POINT(41.41558340361326 73.20193966086784) bank54792 +54793 POINT(40.11017983089217 74.93137763526184) bank54793 +54794 POINT(40.55889162285149 73.68618880237455) bank54794 +54795 POINT(41.29879997196478 73.44171352634251) bank54795 +54796 POINT(41.40832567168587 73.71704359911756) bank54796 +54797 POINT(41.278055769982885 74.79781470930892) bank54797 +54798 POINT(40.23899511736048 74.19385342531118) bank54798 +54799 POINT(40.31651978263454 73.04878753906695) bank54799 +54800 POINT(41.68023340918651 74.79818032820043) bank54800 +54801 POINT(41.520909111931374 73.72517372478316) bank54801 +54802 POINT(40.09914696460513 74.0329808677898) bank54802 +54803 POINT(41.2317899713991 74.26050824779031) bank54803 +54804 POINT(40.53011475090377 74.09791445205968) bank54804 +54805 POINT(39.73685841721963 73.94509641396868) bank54805 +54806 POINT(40.69866565360408 73.8975370077752) bank54806 +54807 POINT(40.102433898574105 74.88828593958743) bank54807 +54808 POINT(41.45831606603723 74.09765888101268) bank54808 +54809 POINT(41.25993098782977 74.9397280626392) bank54809 +54810 POINT(39.82077239068203 73.777764873662) bank54810 +54811 POINT(41.45286514369139 73.68863884004782) bank54811 +54812 POINT(41.2363620844758 74.20700206533705) bank54812 +54813 POINT(40.54048562586329 73.1645288734286) bank54813 +54814 POINT(41.255824652431016 74.69009060697435) bank54814 +54815 POINT(39.97357220414428 74.10696405414447) bank54815 +54816 POINT(40.00980339046175 73.84181218323565) bank54816 +54817 POINT(40.563542693140754 74.51312990742426) bank54817 +54818 POINT(41.45709830407687 73.30541535708113) bank54818 +54819 POINT(41.16280418955408 73.44983409737178) bank54819 +54820 POINT(40.63794538067326 73.57389238842643) bank54820 +54821 POINT(41.11604384849112 74.2848513100734) bank54821 +54822 POINT(39.985051107984624 73.49292925636378) bank54822 +54823 POINT(39.90673642855454 73.5676791686652) bank54823 +54824 POINT(40.65820927689546 74.20803527053978) bank54824 +54825 POINT(41.45328855942125 73.10234951629805) bank54825 +54826 POINT(40.577405986642724 74.72668907925953) bank54826 +54827 POINT(40.17415514786053 73.90717158723386) bank54827 +54828 POINT(40.544498967776114 73.74323214927031) bank54828 +54829 POINT(41.16531827702022 74.65468204983301) bank54829 +54830 POINT(39.766657601853964 74.29902587828916) bank54830 +54831 POINT(41.59860164240167 73.98266713971319) bank54831 +54832 POINT(41.61696026959163 73.08767255274022) bank54832 +54833 POINT(40.68934477833902 74.12943569546839) bank54833 +54834 POINT(41.57255749523082 74.34973702914336) bank54834 +54835 POINT(41.223687743416534 73.4138789818928) bank54835 +54836 POINT(40.46034298028267 73.9880106462955) bank54836 +54837 POINT(41.401616627312706 74.6674389555545) bank54837 +54838 POINT(40.27811607783582 73.52736201309267) bank54838 +54839 POINT(40.68683614070763 73.48730226053925) bank54839 +54840 POINT(39.76990936422739 74.99283539116348) bank54840 +54841 POINT(40.25382447749847 74.53707113461495) bank54841 +54842 POINT(40.349753500170976 74.78353460987154) bank54842 +54843 POINT(40.884104370547156 73.73243243457723) bank54843 +54844 POINT(41.3199766657946 74.45113675210311) bank54844 +54845 POINT(41.32513958154026 73.90929478504279) bank54845 +54846 POINT(40.08998313185977 73.57602005515167) bank54846 +54847 POINT(40.387350209829016 73.21198003597857) bank54847 +54848 POINT(41.44092820237288 74.7832189725638) bank54848 +54849 POINT(40.772679136741345 73.07162976848836) bank54849 +54850 POINT(40.894529612421834 73.23005050638038) bank54850 +54851 POINT(40.509759042383806 74.91888433041937) bank54851 +54852 POINT(40.81787144084319 73.47907264754015) bank54852 +54853 POINT(40.651849294907855 73.9958735974118) bank54853 +54854 POINT(41.16787318345473 74.04792799164248) bank54854 +54855 POINT(40.86375648695463 73.37284651634289) bank54855 +54856 POINT(40.16535396388842 74.06468206224712) bank54856 +54857 POINT(40.96904401142875 74.88697598195597) bank54857 +54858 POINT(40.754758802077056 74.9777421921103) bank54858 +54859 POINT(41.4890690792474 74.08305820636528) bank54859 +54860 POINT(39.738099166423346 74.12250639361707) bank54860 +54861 POINT(40.18027635310516 74.36301235385986) bank54861 +54862 POINT(40.20736743874719 74.91621474187895) bank54862 +54863 POINT(41.28472912618636 74.50544834731109) bank54863 +54864 POINT(40.94792126856662 74.61353266997408) bank54864 +54865 POINT(41.43838799406516 74.9407987177247) bank54865 +54866 POINT(41.134728562097216 74.46494356386586) bank54866 +54867 POINT(39.729722608805716 73.57804428891264) bank54867 +54868 POINT(40.554824168068336 74.92551442026163) bank54868 +54869 POINT(39.96764369789119 73.82266246234332) bank54869 +54870 POINT(40.55747461296241 74.9459701475007) bank54870 +54871 POINT(40.38375390720183 74.7036032171384) bank54871 +54872 POINT(39.843065772954425 73.05575364141882) bank54872 +54873 POINT(41.454503200772166 73.72051965143915) bank54873 +54874 POINT(41.65303167426222 74.54476901728837) bank54874 +54875 POINT(40.392236653581605 74.33782509953677) bank54875 +54876 POINT(41.21371098728908 73.15034655984897) bank54876 +54877 POINT(41.41255383854579 74.75593628446074) bank54877 +54878 POINT(40.26467042485216 73.46580812049824) bank54878 +54879 POINT(41.33723269836828 74.48075881315803) bank54879 +54880 POINT(41.471127027960414 74.63496078831953) bank54880 +54881 POINT(41.10070208003721 73.36315116745683) bank54881 +54882 POINT(40.104643018811146 74.07288767068592) bank54882 +54883 POINT(41.09504305521836 73.19172302284575) bank54883 +54884 POINT(40.104225803054966 73.56380173479594) bank54884 +54885 POINT(41.35448507614067 74.15810686814827) bank54885 +54886 POINT(39.736086513650605 74.49163577967447) bank54886 +54887 POINT(39.747547922067746 74.84345086046508) bank54887 +54888 POINT(40.793936875708134 74.14041711092999) bank54888 +54889 POINT(40.801790082087784 74.69048548464208) bank54889 +54890 POINT(40.68975895902163 73.89044903302437) bank54890 +54891 POINT(41.02747788189866 73.12987427565636) bank54891 +54892 POINT(41.17218964421371 73.08798043381033) bank54892 +54893 POINT(40.037503814380614 74.91838656090272) bank54893 +54894 POINT(40.08357560813933 74.0582354258666) bank54894 +54895 POINT(41.49483549034356 74.00361526666117) bank54895 +54896 POINT(41.303280567353994 74.68521427982851) bank54896 +54897 POINT(40.32634859007292 74.18865094348149) bank54897 +54898 POINT(39.88982271139768 74.96452892863904) bank54898 +54899 POINT(41.64536708087903 74.97519325499941) bank54899 +54900 POINT(40.40920557292736 74.91878889238616) bank54900 +54901 POINT(41.572039392388945 74.94736197295731) bank54901 +54902 POINT(41.29870373517282 73.70173198647424) bank54902 +54903 POINT(40.59386898878078 73.72181607071795) bank54903 +54904 POINT(40.75632880688648 73.56363098654504) bank54904 +54905 POINT(40.675646117654736 73.43205452298479) bank54905 +54906 POINT(40.73665615444323 73.36945252814871) bank54906 +54907 POINT(40.56318560773283 74.08169821316734) bank54907 +54908 POINT(40.06986493487863 74.55039201905342) bank54908 +54909 POINT(40.94625678130685 73.82155532750579) bank54909 +54910 POINT(40.174569023749136 74.62986716430389) bank54910 +54911 POINT(41.667537844960485 74.67501857648797) bank54911 +54912 POINT(39.720721341312306 73.491713083588) bank54912 +54913 POINT(40.68153628148663 73.71709291339663) bank54913 +54914 POINT(41.57968462731208 73.54585691969707) bank54914 +54915 POINT(40.84338053562702 74.21026576502553) bank54915 +54916 POINT(41.680401027328934 74.81999278812238) bank54916 +54917 POINT(39.94034656255189 74.65033929092385) bank54917 +54918 POINT(41.61617934170517 74.0519146616266) bank54918 +54919 POINT(40.99224454924083 74.99524508530205) bank54919 +54920 POINT(40.98854979981093 74.78785583228837) bank54920 +54921 POINT(40.12106098470513 74.91488672138742) bank54921 +54922 POINT(40.1640271745628 74.11026452709284) bank54922 +54923 POINT(40.38628992302865 74.06628688177877) bank54923 +54924 POINT(40.719336093227 73.82608766930768) bank54924 +54925 POINT(40.52355316967158 74.41474151548726) bank54925 +54926 POINT(40.82306644728425 73.40740173021186) bank54926 +54927 POINT(40.68883708705497 74.77312652543317) bank54927 +54928 POINT(41.15061065628278 73.9284636465554) bank54928 +54929 POINT(39.83562911656578 73.58876272638649) bank54929 +54930 POINT(41.51050082939672 74.79676641201438) bank54930 +54931 POINT(41.20737802428838 73.89440255851274) bank54931 +54932 POINT(41.109210386885074 74.54497965078008) bank54932 +54933 POINT(39.72142840467429 74.91101853318031) bank54933 +54934 POINT(41.034082045231045 74.82295664245679) bank54934 +54935 POINT(41.2983476397995 74.31499849539631) bank54935 +54936 POINT(40.377037658309234 74.7275534840226) bank54936 +54937 POINT(41.3352865211713 74.23393812254746) bank54937 +54938 POINT(39.97619351460298 73.98557446535808) bank54938 +54939 POINT(41.266747815718354 74.15767922834482) bank54939 +54940 POINT(40.91833023652362 73.92673953215356) bank54940 +54941 POINT(40.523898652355186 73.10766395685752) bank54941 +54942 POINT(41.47901983434213 73.39797127371567) bank54942 +54943 POINT(40.59799550598906 74.12504283198452) bank54943 +54944 POINT(39.76746008051119 73.06789931352891) bank54944 +54945 POINT(40.17172744789115 74.98387538234734) bank54945 +54946 POINT(40.61332849696589 74.79027635842615) bank54946 +54947 POINT(39.87500920907386 73.31058242874266) bank54947 +54948 POINT(40.04441851517182 73.09014484942968) bank54948 +54949 POINT(40.65762299364343 73.5082400881976) bank54949 +54950 POINT(40.575049901555914 73.84931196015692) bank54950 +54951 POINT(39.974572359303195 73.91888252803365) bank54951 +54952 POINT(41.709834682854186 74.94161287238357) bank54952 +54953 POINT(40.060816005317676 74.56656985411489) bank54953 +54954 POINT(41.310692351779096 74.01667310847166) bank54954 +54955 POINT(40.63951666654816 74.63063431178811) bank54955 +54956 POINT(40.148800350244684 74.62785145696334) bank54956 +54957 POINT(40.60814963195501 73.6318711235107) bank54957 +54958 POINT(39.87585879919942 73.06199724691349) bank54958 +54959 POINT(40.69593663460615 74.61011495431109) bank54959 +54960 POINT(41.48295077698912 74.81048969666615) bank54960 +54961 POINT(41.01968434295231 74.27686257872533) bank54961 +54962 POINT(39.9295806150103 73.20338223732738) bank54962 +54963 POINT(40.25992140383349 74.44679565088477) bank54963 +54964 POINT(40.37216418554434 74.81274828634379) bank54964 +54965 POINT(41.50323567217986 73.42100326054039) bank54965 +54966 POINT(40.543107369077575 74.25637807618824) bank54966 +54967 POINT(40.575540791830825 74.8848462381823) bank54967 +54968 POINT(40.56368678372656 73.57483386687257) bank54968 +54969 POINT(40.66952735759655 74.0555196649668) bank54969 +54970 POINT(40.61615621803113 74.87441622719791) bank54970 +54971 POINT(40.76317397315209 74.14118776526594) bank54971 +54972 POINT(41.15180683770487 74.5431689261264) bank54972 +54973 POINT(41.16311705708911 73.42748388499008) bank54973 +54974 POINT(40.10120352507637 73.30571914688487) bank54974 +54975 POINT(40.27870171530699 73.09825486273621) bank54975 +54976 POINT(39.980201173917955 74.58240832497245) bank54976 +54977 POINT(40.35985307463911 74.12492492166861) bank54977 +54978 POINT(40.027592542256784 73.46578173716509) bank54978 +54979 POINT(40.18092538869844 74.81706696183902) bank54979 +54980 POINT(40.670418954105216 73.2505049420552) bank54980 +54981 POINT(40.41044413138081 74.72293696952046) bank54981 +54982 POINT(40.608944266528624 74.4604106190341) bank54982 +54983 POINT(40.272826021835876 73.30834319916319) bank54983 +54984 POINT(41.43590925066856 74.22527998630667) bank54984 +54985 POINT(40.75234972847484 74.37634794754119) bank54985 +54986 POINT(40.61292946303929 74.19683188842323) bank54986 +54987 POINT(40.83222376473068 73.76698707063686) bank54987 +54988 POINT(40.87442132273246 74.14208222027182) bank54988 +54989 POINT(40.21905074881737 74.05406945302501) bank54989 +54990 POINT(40.55299346940636 74.61346692243335) bank54990 +54991 POINT(40.26750847985571 73.18985739770143) bank54991 +54992 POINT(39.86390851110822 73.64332090126359) bank54992 +54993 POINT(41.613103149791 74.75007261538335) bank54993 +54994 POINT(40.3870562464418 73.5065304835633) bank54994 +54995 POINT(40.631613122443504 74.17193731197324) bank54995 +54996 POINT(41.686757235756495 74.04303383897316) bank54996 +54997 POINT(40.62004768371541 73.95709560838098) bank54997 +54998 POINT(41.62331040063981 73.8748455944787) bank54998 +54999 POINT(40.187906827979575 73.59233727658453) bank54999 +55000 POINT(40.76175054472982 74.95689831093107) bank55000 +55001 POINT(41.26073213168916 73.99984576636146) bank55001 +55002 POINT(41.68882640775607 73.11034845997348) bank55002 +55003 POINT(40.050357795263245 74.76598710410737) bank55003 +55004 POINT(40.39273487253448 73.08435359423184) bank55004 +55005 POINT(39.81611965727463 73.15130476532448) bank55005 +55006 POINT(40.8429009486953 74.06193259114177) bank55006 +55007 POINT(41.02314271319696 73.12830977669508) bank55007 +55008 POINT(41.702072667517555 73.6632099051944) bank55008 +55009 POINT(41.37024299072505 74.34514722640314) bank55009 +55010 POINT(40.50743552330212 73.25144034351197) bank55010 +55011 POINT(41.622786356832215 73.19259057247018) bank55011 +55012 POINT(41.5152792407971 74.60447156834219) bank55012 +55013 POINT(40.97355406399034 74.40667750734838) bank55013 +55014 POINT(40.928016753596395 74.0658008545337) bank55014 +55015 POINT(41.112437534737644 73.72001806519157) bank55015 +55016 POINT(40.93486808856027 73.04948750959093) bank55016 +55017 POINT(40.314982991479255 74.75533841417955) bank55017 +55018 POINT(40.88071268446577 74.3452625894368) bank55018 +55019 POINT(39.79361615024791 74.20631496012177) bank55019 +55020 POINT(39.73782540044788 73.24532078737181) bank55020 +55021 POINT(40.77754223078899 73.7603453459182) bank55021 +55022 POINT(41.58910151154805 73.91833377002126) bank55022 +55023 POINT(41.1450584891721 73.41883229511916) bank55023 +55024 POINT(39.77623225228582 74.41440832234103) bank55024 +55025 POINT(40.392632550773 73.4853732875484) bank55025 +55026 POINT(39.86185616414254 73.78086651977215) bank55026 +55027 POINT(41.50979657760706 74.4908231427286) bank55027 +55028 POINT(40.32642027537753 74.13754285582944) bank55028 +55029 POINT(41.310975228651166 74.2841423892466) bank55029 +55030 POINT(41.33579064998266 74.71787768730562) bank55030 +55031 POINT(39.984192556096 74.10922961139448) bank55031 +55032 POINT(40.43484678928094 74.17401936737758) bank55032 +55033 POINT(40.57914382496487 74.50049920439976) bank55033 +55034 POINT(40.065121448872866 74.7832838496622) bank55034 +55035 POINT(41.461125877424024 73.10890863535825) bank55035 +55036 POINT(39.813085535217446 73.68875052273707) bank55036 +55037 POINT(40.33851280147023 74.23322964879382) bank55037 +55038 POINT(39.97644748607428 73.28473751914562) bank55038 +55039 POINT(41.21609809679148 74.9455335101227) bank55039 +55040 POINT(40.01935391378566 74.21002864658573) bank55040 +55041 POINT(41.53265302674807 74.75215229548937) bank55041 +55042 POINT(41.562839934686814 73.11940368785288) bank55042 +55043 POINT(40.511398153779446 73.7695230961829) bank55043 +55044 POINT(40.90331835105727 73.84176263534431) bank55044 +55045 POINT(40.4601002268141 74.64856226993884) bank55045 +55046 POINT(41.66579749850533 73.03748738125543) bank55046 +55047 POINT(41.12842943593555 73.92059029863118) bank55047 +55048 POINT(41.54300258747029 74.70665382434177) bank55048 +55049 POINT(40.502262976824085 74.44591531667547) bank55049 +55050 POINT(41.0807590449323 73.06545529200407) bank55050 +55051 POINT(40.569388374449694 73.35363493270135) bank55051 +55052 POINT(39.951112050508506 74.97622324168061) bank55052 +55053 POINT(39.98026399026411 73.8835257615426) bank55053 +55054 POINT(40.55486819072059 74.59822232619813) bank55054 +55055 POINT(40.03183746439222 74.34225343300739) bank55055 +55056 POINT(40.88128114294349 74.50423204007552) bank55056 +55057 POINT(41.65760978041708 74.40197748115125) bank55057 +55058 POINT(40.70770667071322 74.56732398971593) bank55058 +55059 POINT(40.636312036058044 73.79052338842675) bank55059 +55060 POINT(41.43333977755073 74.38442439338836) bank55060 +55061 POINT(41.22860260039547 74.72064699766219) bank55061 +55062 POINT(40.72970131527385 73.71092626763857) bank55062 +55063 POINT(41.58443910178612 74.52911313605553) bank55063 +55064 POINT(40.05804705808832 73.2224384047025) bank55064 +55065 POINT(41.65845909267725 74.73421594092515) bank55065 +55066 POINT(40.77160689365049 73.37167953015661) bank55066 +55067 POINT(41.22748177738117 73.73694703098923) bank55067 +55068 POINT(40.17657115334336 73.03774686347273) bank55068 +55069 POINT(40.89940689193836 74.84460475844804) bank55069 +55070 POINT(41.26378542965989 74.32387189333922) bank55070 +55071 POINT(41.41344255387469 73.67833149822387) bank55071 +55072 POINT(40.46193236791613 73.1406277655566) bank55072 +55073 POINT(40.16892449013237 74.56111558216631) bank55073 +55074 POINT(41.052927133822244 74.39284945472996) bank55074 +55075 POINT(41.284574630894625 73.77340597312228) bank55075 +55076 POINT(40.85319364585789 74.06006617192148) bank55076 +55077 POINT(41.12272862440691 74.9306704782941) bank55077 +55078 POINT(41.25574347278839 73.37178738924244) bank55078 +55079 POINT(40.08776383470441 74.16095346564123) bank55079 +55080 POINT(40.066170019955585 73.02661684707707) bank55080 +55081 POINT(41.54689014466656 74.1378434275543) bank55081 +55082 POINT(40.94595354270021 73.53358979569772) bank55082 +55083 POINT(41.63541677144716 73.2661549212205) bank55083 +55084 POINT(39.89082031139536 73.43318464580352) bank55084 +55085 POINT(40.69682097415776 74.05423430907561) bank55085 +55086 POINT(41.44151176215358 73.35908451557236) bank55086 +55087 POINT(40.7757244705288 74.89381905610546) bank55087 +55088 POINT(40.309141179556875 73.65191470863513) bank55088 +55089 POINT(40.654507619669666 73.50115044373698) bank55089 +55090 POINT(39.9121372797476 73.90867890307) bank55090 +55091 POINT(40.610904011988744 73.95834466701773) bank55091 +55092 POINT(41.558107944560724 73.99612209111564) bank55092 +55093 POINT(40.93233897228281 74.88055170306545) bank55093 +55094 POINT(41.10388283320278 74.39139296175028) bank55094 +55095 POINT(40.33022300667331 73.16860663781864) bank55095 +55096 POINT(41.20620058548932 74.08610363813798) bank55096 +55097 POINT(41.236506696841815 73.77770460143707) bank55097 +55098 POINT(40.441131748582926 73.0935855156996) bank55098 +55099 POINT(40.994563380566106 74.13588267050558) bank55099 +55100 POINT(39.76205790755899 74.78481450686591) bank55100 +55101 POINT(39.946498023421725 73.27052364782132) bank55101 +55102 POINT(41.37826360341916 73.48404597748254) bank55102 +55103 POINT(39.76984712645233 74.18173450040095) bank55103 +55104 POINT(39.86005915894567 73.96038780914553) bank55104 +55105 POINT(39.89048197011926 73.39665309318714) bank55105 +55106 POINT(39.83712579506806 73.11179061376421) bank55106 +55107 POINT(41.346553595016374 74.62316835092697) bank55107 +55108 POINT(40.916103276723256 74.78094098265606) bank55108 +55109 POINT(39.91281305328052 73.20354845210308) bank55109 +55110 POINT(40.994544916119786 73.22819995857134) bank55110 +55111 POINT(41.17143192553489 73.42177243896361) bank55111 +55112 POINT(40.54890761030977 73.52135214188853) bank55112 +55113 POINT(40.98874859293407 74.46699623965111) bank55113 +55114 POINT(40.47847495150532 74.45106610709314) bank55114 +55115 POINT(39.81264364466332 73.58593224133051) bank55115 +55116 POINT(41.04169921086868 74.43969729092655) bank55116 +55117 POINT(39.95499223829331 73.61872209914794) bank55117 +55118 POINT(40.540668553227654 74.08643057187246) bank55118 +55119 POINT(40.62528083485778 74.68481753630587) bank55119 +55120 POINT(40.00425797016742 74.0176887356615) bank55120 +55121 POINT(40.04069051075788 74.18244490167159) bank55121 +55122 POINT(40.62118519250104 73.40658732351974) bank55122 +55123 POINT(41.4364877323013 74.57062833505542) bank55123 +55124 POINT(41.063477380704676 74.9734457926128) bank55124 +55125 POINT(40.81805337622315 74.615743556218) bank55125 +55126 POINT(41.62113787120542 74.33325592969024) bank55126 +55127 POINT(41.177419033460296 73.26619468749287) bank55127 +55128 POINT(40.60714970703722 73.40715863150318) bank55128 +55129 POINT(39.92398996444215 73.57837299153688) bank55129 +55130 POINT(40.21612435156609 73.07633163590397) bank55130 +55131 POINT(40.43599902134579 73.54517927732492) bank55131 +55132 POINT(40.64703974458596 74.09419729655104) bank55132 +55133 POINT(40.84370381805233 74.01101901545856) bank55133 +55134 POINT(40.74440360081473 74.81876744389375) bank55134 +55135 POINT(41.54472697763661 73.40535345811399) bank55135 +55136 POINT(39.92818795053335 73.46406993367121) bank55136 +55137 POINT(40.36690414398154 74.44016293052002) bank55137 +55138 POINT(41.459613964705795 73.09396438962938) bank55138 +55139 POINT(41.48894770122889 74.69988502232135) bank55139 +55140 POINT(40.04933645842477 73.8753344884799) bank55140 +55141 POINT(40.06260151033283 74.99919810995202) bank55141 +55142 POINT(40.94473387860182 74.04767111893304) bank55142 +55143 POINT(40.596096195520154 74.70657707781712) bank55143 +55144 POINT(41.0926843869901 73.4521796925386) bank55144 +55145 POINT(41.235704997644845 74.7344554693792) bank55145 +55146 POINT(39.857009082540415 74.57749514487462) bank55146 +55147 POINT(41.11626255109097 73.48238103649909) bank55147 +55148 POINT(41.62334994294955 74.46817114416332) bank55148 +55149 POINT(41.69525116532608 74.98313498421945) bank55149 +55150 POINT(40.125982047366264 73.718868961675) bank55150 +55151 POINT(41.60433336965048 74.62149609167174) bank55151 +55152 POINT(41.591328994066075 73.5731507747346) bank55152 +55153 POINT(41.2621593472568 74.05673383513273) bank55153 +55154 POINT(40.07942946498495 73.12782997213388) bank55154 +55155 POINT(40.67572712867584 73.56556510590651) bank55155 +55156 POINT(40.656832450136775 74.69053440911085) bank55156 +55157 POINT(41.50721823285932 74.98424454144308) bank55157 +55158 POINT(40.0737489017927 74.05566801589018) bank55158 +55159 POINT(40.50250281865226 74.23727923233294) bank55159 +55160 POINT(39.953521407038 73.63179227955797) bank55160 +55161 POINT(40.04883666950875 73.84433686460497) bank55161 +55162 POINT(41.31355347495061 73.03119700935558) bank55162 +55163 POINT(40.69483555494435 74.65276288667299) bank55163 +55164 POINT(41.382988733515184 74.7432098913894) bank55164 +55165 POINT(40.21111593633063 73.03537722169838) bank55165 +55166 POINT(40.90292356020003 73.28584319791806) bank55166 +55167 POINT(39.842086578722665 73.2979960877533) bank55167 +55168 POINT(40.17210444636146 73.97285217291821) bank55168 +55169 POINT(41.49550697476077 73.80435125898575) bank55169 +55170 POINT(39.979742921334754 74.70112916979785) bank55170 +55171 POINT(40.713171978247644 74.06402742830325) bank55171 +55172 POINT(40.08521279052625 74.40941238188753) bank55172 +55173 POINT(41.303003718403474 74.65105018808204) bank55173 +55174 POINT(41.62016518943909 74.6006632809635) bank55174 +55175 POINT(40.67290882533094 73.75106951335303) bank55175 +55176 POINT(40.50192217332022 74.044430899701) bank55176 +55177 POINT(41.618191197389166 74.56747333362553) bank55177 +55178 POINT(40.27029318396359 73.91495123976745) bank55178 +55179 POINT(41.609432412623185 74.96351690249487) bank55179 +55180 POINT(41.20362978007669 74.40035287993234) bank55180 +55181 POINT(39.98170453779538 74.93042765051389) bank55181 +55182 POINT(40.13027980424288 74.09665460587988) bank55182 +55183 POINT(41.27991998106785 73.92734576914712) bank55183 +55184 POINT(41.23934372550757 73.56548099925412) bank55184 +55185 POINT(41.61166507653651 74.82304011865892) bank55185 +55186 POINT(41.133424826448184 73.49234942832358) bank55186 +55187 POINT(41.15144568608742 73.13855384699576) bank55187 +55188 POINT(41.60249237682553 73.63487224756616) bank55188 +55189 POINT(40.45808147989262 73.89169560405965) bank55189 +55190 POINT(41.08726776036865 73.60212804743469) bank55190 +55191 POINT(40.732475616221464 74.89390256431801) bank55191 +55192 POINT(39.94607292757117 74.44956843739564) bank55192 +55193 POINT(41.34216565975644 74.6018139090359) bank55193 +55194 POINT(41.464068625909704 74.58798174185038) bank55194 +55195 POINT(41.68410751710296 74.05537467597594) bank55195 +55196 POINT(39.99129051340631 74.69252036282325) bank55196 +55197 POINT(40.29168394130152 74.16525567804418) bank55197 +55198 POINT(39.724463271698596 74.27095282936538) bank55198 +55199 POINT(40.58626957659829 74.97105281705862) bank55199 +55200 POINT(41.473002055704164 73.73304357718028) bank55200 +55201 POINT(40.337935577709004 74.12130724979448) bank55201 +55202 POINT(40.26908795531882 74.41917026597893) bank55202 +55203 POINT(40.02844621803151 74.74606781447258) bank55203 +55204 POINT(41.10191659421358 74.11401938205402) bank55204 +55205 POINT(40.69423754418068 74.67675784416828) bank55205 +55206 POINT(40.628197242957654 74.15315982414832) bank55206 +55207 POINT(40.17702687980656 74.37200938836277) bank55207 +55208 POINT(40.68254605312157 73.91193760398849) bank55208 +55209 POINT(41.12098582812372 74.9477179282044) bank55209 +55210 POINT(40.9992840718383 73.88383506674565) bank55210 +55211 POINT(39.733279032836855 74.37314835780204) bank55211 +55212 POINT(41.43290554139408 74.99713176017367) bank55212 +55213 POINT(40.94220278481244 73.76295575716954) bank55213 +55214 POINT(41.05993965995409 73.35427231810483) bank55214 +55215 POINT(41.654410199180056 73.57113564838967) bank55215 +55216 POINT(40.470956918981095 73.38572823210117) bank55216 +55217 POINT(41.65319496727093 73.07651351357337) bank55217 +55218 POINT(41.20693981421052 74.20996193046281) bank55218 +55219 POINT(41.62140767144261 73.29638213629396) bank55219 +55220 POINT(41.4394847570461 73.46919168530255) bank55220 +55221 POINT(40.75731270626108 74.37890353800525) bank55221 +55222 POINT(40.31443262791471 74.96141496768959) bank55222 +55223 POINT(41.47254149529893 73.5014446127469) bank55223 +55224 POINT(40.11017438667003 74.92446592153712) bank55224 +55225 POINT(41.27647115666004 73.70419962333855) bank55225 +55226 POINT(39.9113800029414 73.41089555449769) bank55226 +55227 POINT(41.00755820494583 73.85875169500032) bank55227 +55228 POINT(40.37267816344537 74.32578179385368) bank55228 +55229 POINT(40.078798856447456 74.61655592831437) bank55229 +55230 POINT(41.10247142152477 73.95985309050192) bank55230 +55231 POINT(39.99412858508993 73.27218664588233) bank55231 +55232 POINT(40.33053125450788 73.01637832858172) bank55232 +55233 POINT(39.82420556148008 73.49234575192084) bank55233 +55234 POINT(40.31755237784142 74.7214484498378) bank55234 +55235 POINT(39.81273888475905 74.98528160939529) bank55235 +55236 POINT(41.25724669216126 74.5296721070406) bank55236 +55237 POINT(40.07272861601217 73.38662647053822) bank55237 +55238 POINT(39.827162633832785 73.28303633789945) bank55238 +55239 POINT(40.203926208067436 74.19393440012888) bank55239 +55240 POINT(40.81315093243756 73.32723263017533) bank55240 +55241 POINT(40.521590263964505 74.40280944543012) bank55241 +55242 POINT(41.263008172838234 73.17121477771133) bank55242 +55243 POINT(41.291799817084296 74.65705897221348) bank55243 +55244 POINT(39.98597579968561 73.39310536572206) bank55244 +55245 POINT(40.73770296959963 73.49325132963283) bank55245 +55246 POINT(40.85357043687772 74.41151489764475) bank55246 +55247 POINT(39.84364544906358 74.29847907207089) bank55247 +55248 POINT(40.501384919084614 73.83499091564279) bank55248 +55249 POINT(40.1828790951682 74.06811428686775) bank55249 +55250 POINT(40.35846705680572 74.72997297204718) bank55250 +55251 POINT(39.87475278555336 73.21065406800359) bank55251 +55252 POINT(40.74311659662896 74.63125944031658) bank55252 +55253 POINT(40.17637885022691 74.969274060096) bank55253 +55254 POINT(40.75802434793431 74.69697671289428) bank55254 +55255 POINT(39.84611063260079 74.97835205957712) bank55255 +55256 POINT(40.84145194871951 73.25329505756812) bank55256 +55257 POINT(39.91293171690865 74.16476720723112) bank55257 +55258 POINT(40.07911478313631 74.2884578096816) bank55258 +55259 POINT(41.08590432727625 74.27262746698393) bank55259 +55260 POINT(40.42001241520559 74.27620286502193) bank55260 +55261 POINT(40.09003681449563 74.47495273793626) bank55261 +55262 POINT(40.81804776615789 73.3967540659764) bank55262 +55263 POINT(41.114871966052654 74.7142111361927) bank55263 +55264 POINT(40.210344950334516 74.88535457603527) bank55264 +55265 POINT(41.44656169968008 73.20922324409871) bank55265 +55266 POINT(40.72465599640012 74.60317161849068) bank55266 +55267 POINT(40.855079822091795 73.90373830690953) bank55267 +55268 POINT(40.15676194518812 74.56299861722194) bank55268 +55269 POINT(41.437441643865306 73.32098942767219) bank55269 +55270 POINT(41.3599769451301 73.93921441179003) bank55270 +55271 POINT(40.26683467775401 73.0448217836355) bank55271 +55272 POINT(40.71590997039885 73.85724464839572) bank55272 +55273 POINT(41.44737293275311 73.24039600408805) bank55273 +55274 POINT(40.62934275843765 74.34698399756603) bank55274 +55275 POINT(40.25015008846563 74.2839289138127) bank55275 +55276 POINT(40.57392353268309 74.91981890514475) bank55276 +55277 POINT(41.24576763237969 73.88575011058018) bank55277 +55278 POINT(41.11968719026565 73.1721324543149) bank55278 +55279 POINT(41.69546068400055 74.06139893802245) bank55279 +55280 POINT(40.52387380069859 73.29558847056573) bank55280 +55281 POINT(40.887648495247184 74.62411328837966) bank55281 +55282 POINT(39.89936974796131 74.21787916501125) bank55282 +55283 POINT(40.845270524019035 73.86838753434489) bank55283 +55284 POINT(41.31885838381603 73.49627106644131) bank55284 +55285 POINT(40.801764064114934 74.69734619338631) bank55285 +55286 POINT(40.61288317499478 74.14732928412026) bank55286 +55287 POINT(40.98813863962234 74.61836790994087) bank55287 +55288 POINT(39.98153659646601 73.20945989810416) bank55288 +55289 POINT(39.92846886119835 73.60507401144528) bank55289 +55290 POINT(40.59437585601643 73.56237396280456) bank55290 +55291 POINT(40.333722473567896 73.27676653659385) bank55291 +55292 POINT(40.11190681275919 74.56776360419772) bank55292 +55293 POINT(39.900734066486514 73.03877941056484) bank55293 +55294 POINT(40.40802461293818 74.27881701858357) bank55294 +55295 POINT(40.74037979252007 73.600286237395) bank55295 +55296 POINT(41.69501642262259 74.94604063230037) bank55296 +55297 POINT(40.160539149285654 74.02147278861817) bank55297 +55298 POINT(40.575114816090625 73.84676528013942) bank55298 +55299 POINT(40.52236165174243 73.33912678747625) bank55299 +55300 POINT(40.78040530970742 74.67591831483062) bank55300 +55301 POINT(41.12348039341676 73.92190974080734) bank55301 +55302 POINT(41.39515115493669 74.00159861733633) bank55302 +55303 POINT(41.54989352625856 73.47673472739311) bank55303 +55304 POINT(40.54288089431046 73.14335993560222) bank55304 +55305 POINT(40.654009522190684 74.11309682324516) bank55305 +55306 POINT(40.60306274625238 73.59511496178384) bank55306 +55307 POINT(40.542255420742656 74.21449439991555) bank55307 +55308 POINT(40.516844338802116 74.38449974362749) bank55308 +55309 POINT(41.00811297188626 74.98662464046367) bank55309 +55310 POINT(40.94313627108535 74.98360567840614) bank55310 +55311 POINT(40.81140088853369 74.28423931749428) bank55311 +55312 POINT(41.52201761955256 73.34528580227052) bank55312 +55313 POINT(40.11037452940812 73.76150718021657) bank55313 +55314 POINT(41.42794596480386 74.36856779365525) bank55314 +55315 POINT(39.99112193183426 74.79574392361324) bank55315 +55316 POINT(41.65983980057056 74.14820493144357) bank55316 +55317 POINT(41.3545554123855 73.97628764512875) bank55317 +55318 POINT(40.63892045023704 74.66389040005971) bank55318 +55319 POINT(39.76720994588284 73.63048511574553) bank55319 +55320 POINT(40.05507756218311 73.23362390993626) bank55320 +55321 POINT(40.62652047667852 73.13322959626737) bank55321 +55322 POINT(41.23897233629758 74.24809337855116) bank55322 +55323 POINT(41.61412765058168 73.22323403358577) bank55323 +55324 POINT(41.21112823604249 73.76421044396646) bank55324 +55325 POINT(40.089480256204126 74.09339133384844) bank55325 +55326 POINT(41.1763502806406 73.0872529529552) bank55326 +55327 POINT(40.17758579792193 73.55663874684505) bank55327 +55328 POINT(40.9409619873162 73.15616283330846) bank55328 +55329 POINT(41.458580117393524 74.52586358925012) bank55329 +55330 POINT(40.52363846733541 74.60245106186643) bank55330 +55331 POINT(40.02578477509134 74.17308189349457) bank55331 +55332 POINT(41.1201553243405 73.9222179640509) bank55332 +55333 POINT(40.72914279367955 73.41035991918918) bank55333 +55334 POINT(40.12592719939674 73.16346727452529) bank55334 +55335 POINT(41.49197460987888 74.78600844077914) bank55335 +55336 POINT(41.5541677466233 73.66995431452752) bank55336 +55337 POINT(39.73665554629832 73.04690003067905) bank55337 +55338 POINT(41.50812298273882 73.54350442720646) bank55338 +55339 POINT(40.29892793492601 74.9594211117546) bank55339 +55340 POINT(41.23483803018561 74.93750097095517) bank55340 +55341 POINT(40.29951262133519 73.35217245555046) bank55341 +55342 POINT(40.05857497868756 74.59618322362294) bank55342 +55343 POINT(41.490764699728324 73.89514435378365) bank55343 +55344 POINT(40.68522535201742 73.60584006358427) bank55344 +55345 POINT(41.29354853167256 73.11929420328397) bank55345 +55346 POINT(39.94090525188557 73.56219172778239) bank55346 +55347 POINT(41.284300051602976 74.78792900853023) bank55347 +55348 POINT(39.995260446964416 74.66613018915872) bank55348 +55349 POINT(41.15980403894079 73.92538049491344) bank55349 +55350 POINT(40.71227260509386 74.60938925126312) bank55350 +55351 POINT(41.64088735812767 74.14114246770848) bank55351 +55352 POINT(40.206219202332214 74.40540400294698) bank55352 +55353 POINT(41.15945718049361 74.95544242714692) bank55353 +55354 POINT(41.06421225617015 74.40316557245569) bank55354 +55355 POINT(41.495940604409476 74.7120427102226) bank55355 +55356 POINT(40.392146495000844 73.9340525732608) bank55356 +55357 POINT(41.449256976169345 74.36416144582942) bank55357 +55358 POINT(41.42632179089177 73.69523744710465) bank55358 +55359 POINT(40.45931349629876 73.15317816267591) bank55359 +55360 POINT(41.12595153625889 74.93017578272664) bank55360 +55361 POINT(41.641052305679516 74.08681688050332) bank55361 +55362 POINT(40.7937831473385 73.58201732226937) bank55362 +55363 POINT(41.02359195097834 74.95023282648789) bank55363 +55364 POINT(39.84293695062767 73.49972913793222) bank55364 +55365 POINT(39.72826301107776 74.04653758876543) bank55365 +55366 POINT(40.08892205338327 74.37132573185902) bank55366 +55367 POINT(40.98749908117149 73.39965095223248) bank55367 +55368 POINT(40.90658342211117 74.86690381962337) bank55368 +55369 POINT(41.58130189105981 73.7726016176792) bank55369 +55370 POINT(40.92765840014598 73.61566695145392) bank55370 +55371 POINT(40.9964755015064 74.80636767070055) bank55371 +55372 POINT(41.6982386790483 74.04181287511587) bank55372 +55373 POINT(39.97257633703418 73.91283997777093) bank55373 +55374 POINT(40.75741234666651 73.73490294480467) bank55374 +55375 POINT(40.551150346079424 74.09299283744998) bank55375 +55376 POINT(40.411794485241536 74.06377264378199) bank55376 +55377 POINT(41.206251865109465 73.30239281509695) bank55377 +55378 POINT(41.29919066178581 74.36280144233349) bank55378 +55379 POINT(40.769056037857155 73.23946586679268) bank55379 +55380 POINT(39.993545264563394 73.70856725684357) bank55380 +55381 POINT(40.44275827791816 74.70279252646009) bank55381 +55382 POINT(41.10490312671585 74.85205259616754) bank55382 +55383 POINT(39.719719280145156 74.06955636337806) bank55383 +55384 POINT(41.5321715742657 74.87819029963619) bank55384 +55385 POINT(40.48102746487721 74.97806533549459) bank55385 +55386 POINT(39.7921182285687 74.35098785150849) bank55386 +55387 POINT(40.09464872985491 74.55285860935525) bank55387 +55388 POINT(40.685838315580334 73.87268421486243) bank55388 +55389 POINT(40.15973663050002 74.97833414729989) bank55389 +55390 POINT(41.14661550113284 73.3882099505622) bank55390 +55391 POINT(41.55193181599734 73.70963625120243) bank55391 +55392 POINT(40.30258790646309 73.77291126113465) bank55392 +55393 POINT(41.64304596066961 74.5606725092829) bank55393 +55394 POINT(40.62364834370427 73.84403863887813) bank55394 +55395 POINT(40.915364795041526 74.26311678037344) bank55395 +55396 POINT(41.66130645254758 73.0101707590275) bank55396 +55397 POINT(40.60306909004313 73.68219734054749) bank55397 +55398 POINT(41.08176939736582 73.17188214689314) bank55398 +55399 POINT(41.626705062802344 73.37621242838276) bank55399 +55400 POINT(39.835126712178564 73.81213912278228) bank55400 +55401 POINT(40.51599806663377 74.51580948573883) bank55401 +55402 POINT(40.71983188012151 74.33617542296625) bank55402 +55403 POINT(40.85170935550126 74.34858521877746) bank55403 +55404 POINT(40.85583564011234 73.61809938391893) bank55404 +55405 POINT(40.33117721490541 73.72351097641626) bank55405 +55406 POINT(41.3735333505817 73.7628208680958) bank55406 +55407 POINT(41.70249010556062 73.79071328954062) bank55407 +55408 POINT(41.624184899077406 73.8495867206468) bank55408 +55409 POINT(41.048247847680614 74.87998313329044) bank55409 +55410 POINT(40.90309543089947 74.96552069117831) bank55410 +55411 POINT(40.36953999844515 73.14625430979405) bank55411 +55412 POINT(40.08605911868033 74.10984313532182) bank55412 +55413 POINT(40.773747829733466 74.58239087089105) bank55413 +55414 POINT(41.40681617834754 73.2268262788503) bank55414 +55415 POINT(39.96950761489568 73.66469662108241) bank55415 +55416 POINT(40.51300241631515 73.52117804477199) bank55416 +55417 POINT(40.803347879784575 74.9878430039479) bank55417 +55418 POINT(40.75355085909204 73.9638224842169) bank55418 +55419 POINT(40.11626462981349 74.76758679205103) bank55419 +55420 POINT(41.48923297024906 73.8896538122967) bank55420 +55421 POINT(39.93867885204216 74.54389808520922) bank55421 +55422 POINT(41.56869987466806 74.00519100249205) bank55422 +55423 POINT(39.9086748470185 74.10368971729994) bank55423 +55424 POINT(41.405646166290566 74.67379207982636) bank55424 +55425 POINT(40.432459357527414 73.60338861834646) bank55425 +55426 POINT(40.244379363275385 73.95801401509097) bank55426 +55427 POINT(40.518156090283604 73.33101342622285) bank55427 +55428 POINT(40.51855274197519 73.5138285817605) bank55428 +55429 POINT(40.292369038665676 73.50502700546139) bank55429 +55430 POINT(40.060471672322215 74.94874461464242) bank55430 +55431 POINT(40.5339121819943 73.34222923107974) bank55431 +55432 POINT(39.972099678623586 73.7611619024307) bank55432 +55433 POINT(39.97151349399675 73.62195849033685) bank55433 +55434 POINT(40.799460541315334 74.79762659161851) bank55434 +55435 POINT(41.68435042804065 74.47159822467303) bank55435 +55436 POINT(40.541063291326985 74.63677599230249) bank55436 +55437 POINT(39.85516698099463 74.29651162727593) bank55437 +55438 POINT(40.71920380386714 73.76302573530371) bank55438 +55439 POINT(41.467925778644535 74.67474260359161) bank55439 +55440 POINT(41.359599668903726 73.68883444696056) bank55440 +55441 POINT(40.45522796539871 74.97524846587086) bank55441 +55442 POINT(41.10823239160469 73.09261275544303) bank55442 +55443 POINT(40.29851472877024 74.69552059407683) bank55443 +55444 POINT(39.984755729680195 73.03322024694124) bank55444 +55445 POINT(40.02594949190061 74.58687228266658) bank55445 +55446 POINT(40.07933266230743 73.00926101125992) bank55446 +55447 POINT(40.897239507895755 74.08077602210368) bank55447 +55448 POINT(41.569521265641626 74.10609103032046) bank55448 +55449 POINT(40.65318642285496 73.97040860398042) bank55449 +55450 POINT(40.47225916834869 74.76628334467391) bank55450 +55451 POINT(40.74180990658359 74.92652684711815) bank55451 +55452 POINT(39.795828917744664 73.1625899101207) bank55452 +55453 POINT(39.75027166036546 74.81336987048702) bank55453 +55454 POINT(41.25470611075241 73.70914230054821) bank55454 +55455 POINT(39.99874240453974 74.01112330444585) bank55455 +55456 POINT(40.80352382768735 74.03097748770372) bank55456 +55457 POINT(40.977649749039706 74.58979742573405) bank55457 +55458 POINT(41.26609337243489 73.05156121954762) bank55458 +55459 POINT(39.85388871824019 74.9465520221934) bank55459 +55460 POINT(41.3889304216181 74.33369940481519) bank55460 +55461 POINT(39.9070793625944 74.2414204845514) bank55461 +55462 POINT(40.36696290197879 74.31842631782081) bank55462 +55463 POINT(39.76853055622454 73.85152286095146) bank55463 +55464 POINT(40.574917941045314 73.42049638658982) bank55464 +55465 POINT(41.52380602484849 74.58377660403107) bank55465 +55466 POINT(39.76511498006717 74.19712422187244) bank55466 +55467 POINT(41.138534393502546 74.19142752611124) bank55467 +55468 POINT(40.5942878604113 74.65077171165547) bank55468 +55469 POINT(41.30281346931862 73.53943179813838) bank55469 +55470 POINT(41.31215493154154 74.09090777970331) bank55470 +55471 POINT(40.87039362461023 73.39386164767342) bank55471 +55472 POINT(40.919663284176366 74.25496730480631) bank55472 +55473 POINT(40.558313013381415 73.76716442547723) bank55473 +55474 POINT(41.574810814446955 73.25803805467231) bank55474 +55475 POINT(40.8390581445033 74.83875963046586) bank55475 +55476 POINT(41.334456148371544 73.60384153287795) bank55476 +55477 POINT(40.96379856713283 74.15770409280269) bank55477 +55478 POINT(40.015280646318836 73.92259374277047) bank55478 +55479 POINT(40.156966406299006 73.19869937141773) bank55479 +55480 POINT(40.03756630754891 73.45563145417437) bank55480 +55481 POINT(40.143349122555776 74.99503276486375) bank55481 +55482 POINT(41.02790051395084 74.88466263200209) bank55482 +55483 POINT(39.93831318980879 74.37958732851884) bank55483 +55484 POINT(40.49628600723892 73.02196708600714) bank55484 +55485 POINT(41.16052853852022 73.17692150173784) bank55485 +55486 POINT(40.509409373936954 73.57471766534465) bank55486 +55487 POINT(41.329398618017365 74.3747055022782) bank55487 +55488 POINT(41.53616006734248 74.66924807937617) bank55488 +55489 POINT(41.667978183563704 74.32434134607476) bank55489 +55490 POINT(39.8902080677259 73.2421293321661) bank55490 +55491 POINT(40.55413310825766 73.26689096053394) bank55491 +55492 POINT(40.43030892746084 74.70864820957783) bank55492 +55493 POINT(40.11722219015061 73.95367191726396) bank55493 +55494 POINT(40.89989412843031 73.90536236864816) bank55494 +55495 POINT(40.694335923556146 74.25682405202194) bank55495 +55496 POINT(41.470563643454746 73.19849577498431) bank55496 +55497 POINT(41.420364671441035 74.05327878331407) bank55497 +55498 POINT(41.07172624403813 73.63436105539526) bank55498 +55499 POINT(40.39726202046338 74.20303580090015) bank55499 +55500 POINT(40.37282180693702 74.40193765014725) bank55500 +55501 POINT(40.24315255885074 74.57619324183469) bank55501 +55502 POINT(41.69041058870968 73.5167576561396) bank55502 +55503 POINT(41.07057579501469 73.99986094644623) bank55503 +55504 POINT(41.37188160387812 74.18144188871761) bank55504 +55505 POINT(40.32470588232157 74.52356962191026) bank55505 +55506 POINT(39.93130362172297 74.28072663566411) bank55506 +55507 POINT(39.85727249155674 74.15393104633891) bank55507 +55508 POINT(41.5354956241818 74.66490111515638) bank55508 +55509 POINT(41.28682021122229 74.00766073622432) bank55509 +55510 POINT(40.35007774203565 74.80518992071497) bank55510 +55511 POINT(39.90331573121579 73.88933476971737) bank55511 +55512 POINT(40.25461945918475 74.58536253960051) bank55512 +55513 POINT(40.77019444592643 73.69464956381302) bank55513 +55514 POINT(41.5112260411689 73.2966998372933) bank55514 +55515 POINT(40.494169302093695 74.40244683945417) bank55515 +55516 POINT(40.115013517225115 73.17375540335297) bank55516 +55517 POINT(41.265582975714395 73.98355097800238) bank55517 +55518 POINT(40.163868347361465 73.7583151576808) bank55518 +55519 POINT(41.26048594531227 73.51222432522124) bank55519 +55520 POINT(41.053585775987834 74.6848147441734) bank55520 +55521 POINT(40.51794489107878 74.24052445350705) bank55521 +55522 POINT(39.88758288937804 74.28683453393855) bank55522 +55523 POINT(39.873863272376 74.42230004861368) bank55523 +55524 POINT(40.49695549240264 74.0824300086783) bank55524 +55525 POINT(40.10060102878218 74.12512876491412) bank55525 +55526 POINT(41.402112820236475 73.67641518786425) bank55526 +55527 POINT(40.4481530506046 74.11538466855446) bank55527 +55528 POINT(40.44022365156709 74.432170695599) bank55528 +55529 POINT(40.40635306120167 73.17974363092044) bank55529 +55530 POINT(40.69951932711667 74.6126754446131) bank55530 +55531 POINT(41.40674068169398 74.54295992398018) bank55531 +55532 POINT(40.376818028142445 74.44409377695824) bank55532 +55533 POINT(39.733463290561005 73.75564350558878) bank55533 +55534 POINT(40.27689791825627 74.96782172835343) bank55534 +55535 POINT(41.69025402524534 74.63421779395391) bank55535 +55536 POINT(41.56982598011904 74.42732930696279) bank55536 +55537 POINT(41.02738897733477 73.86739279263364) bank55537 +55538 POINT(40.22067268407406 73.16107956105489) bank55538 +55539 POINT(40.10202336559923 74.89481867187587) bank55539 +55540 POINT(41.4503372814856 74.31362033703634) bank55540 +55541 POINT(39.89486763524323 74.13482789819653) bank55541 +55542 POINT(39.767484852697756 74.07112556785351) bank55542 +55543 POINT(41.28384440290153 74.21811343203306) bank55543 +55544 POINT(40.07823779422961 73.10147465583826) bank55544 +55545 POINT(40.26666339070135 73.60995282165759) bank55545 +55546 POINT(40.878975128520814 74.01842955285002) bank55546 +55547 POINT(41.02628765068467 74.46950549912366) bank55547 +55548 POINT(40.11574230308308 73.43563179458775) bank55548 +55549 POINT(40.250528364496894 73.37232699050931) bank55549 +55550 POINT(40.20168000313388 73.17623718859706) bank55550 +55551 POINT(40.518686507907546 73.74959031356595) bank55551 +55552 POINT(41.22341268968979 74.41277293915776) bank55552 +55553 POINT(40.88086623721849 73.47157402482736) bank55553 +55554 POINT(40.3876942225673 74.18911856432302) bank55554 +55555 POINT(40.38500539541604 74.30005992346256) bank55555 +55556 POINT(41.39243076807596 73.88333324965556) bank55556 +55557 POINT(40.46309216907316 73.55226051411587) bank55557 +55558 POINT(39.89641754892672 74.04117295140529) bank55558 +55559 POINT(40.579301005589784 74.24419640579075) bank55559 +55560 POINT(40.89433777829828 74.86231029929552) bank55560 +55561 POINT(40.80043495989604 74.94669251112308) bank55561 +55562 POINT(40.82255065966115 74.37486786893646) bank55562 +55563 POINT(41.71165587812246 74.38026760455412) bank55563 +55564 POINT(40.94925238707079 73.89092783360721) bank55564 +55565 POINT(40.69869330588727 74.7848459450294) bank55565 +55566 POINT(41.59777058424363 74.41545175156594) bank55566 +55567 POINT(40.91571522619753 74.3567236682553) bank55567 +55568 POINT(40.2518200484606 73.65725884470925) bank55568 +55569 POINT(40.478900468859486 73.37752719472326) bank55569 +55570 POINT(40.036789625774496 74.53578998082524) bank55570 +55571 POINT(40.34029688725914 74.46891348303231) bank55571 +55572 POINT(40.823101212246065 73.58815989614811) bank55572 +55573 POINT(39.99435535895594 73.97048935469003) bank55573 +55574 POINT(39.96625995067419 74.32383497495056) bank55574 +55575 POINT(41.183623957120794 73.98600967899361) bank55575 +55576 POINT(40.88486498067814 74.7835094850983) bank55576 +55577 POINT(39.99030511250584 73.12985344249044) bank55577 +55578 POINT(39.85062391122293 73.84577456907905) bank55578 +55579 POINT(41.3206736946087 74.59774688711934) bank55579 +55580 POINT(41.65784205351485 73.50425472723717) bank55580 +55581 POINT(39.826854274633575 73.4662264354261) bank55581 +55582 POINT(40.04707270286069 73.8304337048534) bank55582 +55583 POINT(40.94082766487099 73.91761111054333) bank55583 +55584 POINT(40.56747015581136 74.46788724545932) bank55584 +55585 POINT(41.245138906291935 73.75750446232624) bank55585 +55586 POINT(41.023849255081906 73.86918007915241) bank55586 +55587 POINT(41.255613380882934 73.69247735421342) bank55587 +55588 POINT(40.97702728386954 74.71496674390096) bank55588 +55589 POINT(40.19371214865939 74.61852051143032) bank55589 +55590 POINT(40.74793512360616 73.15696933524283) bank55590 +55591 POINT(41.55059157101307 74.77862202137302) bank55591 +55592 POINT(41.0739702762591 74.80702852997847) bank55592 +55593 POINT(41.59809161887208 73.28373634363483) bank55593 +55594 POINT(41.54884434543853 73.43655966698469) bank55594 +55595 POINT(40.64221262235145 73.95208347688016) bank55595 +55596 POINT(39.73378724788511 73.86426265360478) bank55596 +55597 POINT(40.61339042296565 74.8874182650013) bank55597 +55598 POINT(40.33095600163375 73.77687486490841) bank55598 +55599 POINT(40.778905362830606 73.22231602208217) bank55599 +55600 POINT(40.74091175127751 73.8712380609626) bank55600 +55601 POINT(39.7809711751706 74.08578435743598) bank55601 +55602 POINT(40.29339324165726 73.42269439467151) bank55602 +55603 POINT(40.182687812076736 74.60812740877472) bank55603 +55604 POINT(41.277023846597835 74.72845662680331) bank55604 +55605 POINT(40.847725287664446 73.27015609870357) bank55605 +55606 POINT(39.96230380576033 74.25186664483626) bank55606 +55607 POINT(39.77960072332943 74.52668724686924) bank55607 +55608 POINT(40.187312750887635 73.63329724846004) bank55608 +55609 POINT(39.79122958601715 74.80420818911146) bank55609 +55610 POINT(41.517848001659125 74.6551402631594) bank55610 +55611 POINT(40.009574292924704 74.76498263242397) bank55611 +55612 POINT(40.606652712297354 74.66431135541328) bank55612 +55613 POINT(40.91717221423957 74.68275734087528) bank55613 +55614 POINT(41.705368612281845 74.86765686346087) bank55614 +55615 POINT(40.823628084664605 73.40972289717861) bank55615 +55616 POINT(39.96133696818143 74.61795227029539) bank55616 +55617 POINT(41.60254757811142 73.70778060212866) bank55617 +55618 POINT(41.476105398853264 73.93434636183682) bank55618 +55619 POINT(41.05276596954809 74.2947700627865) bank55619 +55620 POINT(40.106849370608366 74.63138364918066) bank55620 +55621 POINT(41.345398402650375 74.11356479734528) bank55621 +55622 POINT(41.13585015207368 73.53644875678735) bank55622 +55623 POINT(40.9284880485821 73.90737597263146) bank55623 +55624 POINT(40.45320410072165 74.98911213396774) bank55624 +55625 POINT(41.243155704593754 74.62525042920083) bank55625 +55626 POINT(40.10088047105135 73.16646910979986) bank55626 +55627 POINT(41.05740008678665 73.15357909892785) bank55627 +55628 POINT(40.32449798710295 73.59870132958261) bank55628 +55629 POINT(40.56892413955901 74.1554847005625) bank55629 +55630 POINT(41.623903719507986 73.10730686943012) bank55630 +55631 POINT(39.77005700170776 74.14951179420332) bank55631 +55632 POINT(40.459584460642965 73.57794868734872) bank55632 +55633 POINT(40.23369627925967 74.78167904030629) bank55633 +55634 POINT(40.01184776658263 74.32672214016792) bank55634 +55635 POINT(40.413212144967474 73.52474715730933) bank55635 +55636 POINT(41.70967019887158 73.12707519169417) bank55636 +55637 POINT(41.0287552173351 74.24389738653267) bank55637 +55638 POINT(41.17794110878987 74.96250653009321) bank55638 +55639 POINT(41.25104985237174 74.48033264459713) bank55639 +55640 POINT(40.922230729528195 74.76637089815164) bank55640 +55641 POINT(39.88170209846459 73.37813431121134) bank55641 +55642 POINT(40.850501406133596 73.39566929398575) bank55642 +55643 POINT(40.36183629956909 74.55709787623452) bank55643 +55644 POINT(41.317868960612834 73.93099821524255) bank55644 +55645 POINT(40.04633165163547 74.87378695919732) bank55645 +55646 POINT(39.861318824941264 73.59295764157697) bank55646 +55647 POINT(41.67018861651032 74.803791716162) bank55647 +55648 POINT(41.2589564592152 73.0819552773769) bank55648 +55649 POINT(40.23573436490854 74.34790446094485) bank55649 +55650 POINT(41.42533817746495 73.94367558579144) bank55650 +55651 POINT(41.37822923854576 74.20032879247118) bank55651 +55652 POINT(40.595907579432854 74.82656054503022) bank55652 +55653 POINT(40.31364330655152 73.26231617824952) bank55653 +55654 POINT(41.59765020894319 74.4816487540604) bank55654 +55655 POINT(41.69902528149051 74.61237028648684) bank55655 +55656 POINT(39.81491501776905 73.1448698687121) bank55656 +55657 POINT(40.52529939797287 74.82447180088278) bank55657 +55658 POINT(40.18522151142942 73.28070014684776) bank55658 +55659 POINT(41.20583203251711 73.49172308388131) bank55659 +55660 POINT(40.202313547400294 73.49025814120144) bank55660 +55661 POINT(41.10328745067467 74.7184590227278) bank55661 +55662 POINT(41.2272423273577 74.98601347058039) bank55662 +55663 POINT(40.399050522008636 73.69325651199398) bank55663 +55664 POINT(39.916267039024746 74.87626760725382) bank55664 +55665 POINT(40.79758056508901 73.44639749958466) bank55665 +55666 POINT(40.79363086164834 74.3586871265606) bank55666 +55667 POINT(39.72854073349849 73.79216449298663) bank55667 +55668 POINT(40.46886826698062 73.2253555931266) bank55668 +55669 POINT(41.43161415458827 74.74234647730557) bank55669 +55670 POINT(41.024474348726095 73.21272263819732) bank55670 +55671 POINT(39.92733538003452 74.85885822512725) bank55671 +55672 POINT(41.310778033722485 74.90746673300589) bank55672 +55673 POINT(40.35182569170349 73.46617948655576) bank55673 +55674 POINT(40.47758849031037 73.01841685702549) bank55674 +55675 POINT(39.76208669383557 74.23064125302616) bank55675 +55676 POINT(40.299098836527854 74.96383151668802) bank55676 +55677 POINT(40.17126321499173 74.28176221098906) bank55677 +55678 POINT(40.322569231126 74.08043099103557) bank55678 +55679 POINT(41.28478111756454 74.42139036805402) bank55679 +55680 POINT(41.523961315667044 74.44946522592109) bank55680 +55681 POINT(41.23182134128299 74.14947760949238) bank55681 +55682 POINT(40.44341512302551 73.76442534200909) bank55682 +55683 POINT(41.04088728695079 74.08244590957946) bank55683 +55684 POINT(40.324583846778715 74.92987783564203) bank55684 +55685 POINT(40.50195427042742 73.35056033876694) bank55685 +55686 POINT(41.507597493422274 74.79177407383781) bank55686 +55687 POINT(41.531798531158614 73.21317789277414) bank55687 +55688 POINT(40.74183757239796 74.37957726253612) bank55688 +55689 POINT(40.575322905112536 74.92999881400083) bank55689 +55690 POINT(40.78066505859676 73.76817113210987) bank55690 +55691 POINT(40.194807963631554 73.63842757789268) bank55691 +55692 POINT(40.42953783833349 74.25251596133575) bank55692 +55693 POINT(39.83055957677354 73.66492183074718) bank55693 +55694 POINT(41.668561013606855 74.44291442577354) bank55694 +55695 POINT(40.4365888028692 73.99393591774978) bank55695 +55696 POINT(41.440197535185135 74.43227768034869) bank55696 +55697 POINT(41.03630652691584 73.04963318265573) bank55697 +55698 POINT(40.39908244335698 74.7521090068611) bank55698 +55699 POINT(40.10533460787201 74.7226824543461) bank55699 +55700 POINT(40.17179324542531 73.65644652216261) bank55700 +55701 POINT(39.781051186184484 73.24785417811304) bank55701 +55702 POINT(41.50752813142478 74.09667038033479) bank55702 +55703 POINT(40.88345412349139 74.40622323064001) bank55703 +55704 POINT(41.681107956847534 74.75983845258779) bank55704 +55705 POINT(41.4717324176476 74.31624756322172) bank55705 +55706 POINT(41.333152576235655 74.64921401378392) bank55706 +55707 POINT(41.581236057160154 73.05038096285173) bank55707 +55708 POINT(40.45504057740508 74.17398667878362) bank55708 +55709 POINT(41.59705632868835 73.11957095101285) bank55709 +55710 POINT(41.08346839367119 74.51764531783425) bank55710 +55711 POINT(41.240490050079906 74.43389639450888) bank55711 +55712 POINT(40.27321582058389 73.57576650437244) bank55712 +55713 POINT(41.17715554385321 73.06055043459455) bank55713 +55714 POINT(40.254893340045385 73.04513782877713) bank55714 +55715 POINT(41.52745693149755 74.19609963264418) bank55715 +55716 POINT(40.9002645538392 73.53523458397778) bank55716 +55717 POINT(40.616180970827024 73.00727292133963) bank55717 +55718 POINT(40.61565442056104 74.44488006261591) bank55718 +55719 POINT(40.76324433165548 74.89571134298633) bank55719 +55720 POINT(40.76347413993779 74.81101215450099) bank55720 +55721 POINT(41.05948930873731 74.45393713799069) bank55721 +55722 POINT(40.747689276636976 74.07051496362568) bank55722 +55723 POINT(39.886701742990354 74.97487685081683) bank55723 +55724 POINT(41.149069681939984 73.66890939488425) bank55724 +55725 POINT(39.997183631938015 73.90775478850651) bank55725 +55726 POINT(40.95875683066917 73.54920078202736) bank55726 +55727 POINT(41.41823383842756 74.0770129367621) bank55727 +55728 POINT(41.066346853602035 73.49840587367801) bank55728 +55729 POINT(40.61219064123112 74.00299756057338) bank55729 +55730 POINT(41.588442345365856 74.22966567592474) bank55730 +55731 POINT(41.34610462986014 73.01405690070361) bank55731 +55732 POINT(41.56822523219041 74.24918377854588) bank55732 +55733 POINT(40.050696636841195 74.50064006613239) bank55733 +55734 POINT(39.81748115826677 74.30066634333151) bank55734 +55735 POINT(40.23783403660741 74.45613034281453) bank55735 +55736 POINT(41.63124281734098 73.34851244267266) bank55736 +55737 POINT(40.786978117645425 73.80925090337058) bank55737 +55738 POINT(39.811878237173055 74.19778775175206) bank55738 +55739 POINT(41.141572497989586 74.2267589023961) bank55739 +55740 POINT(41.411570081777654 74.56280299161388) bank55740 +55741 POINT(40.75762728382598 74.50369081952863) bank55741 +55742 POINT(40.701137577799244 74.1729098470495) bank55742 +55743 POINT(40.58845849599378 73.01134992632889) bank55743 +55744 POINT(40.60037487667144 74.05824300030572) bank55744 +55745 POINT(39.78056459269631 74.85634910011174) bank55745 +55746 POINT(40.42505987028768 73.12031281844786) bank55746 +55747 POINT(39.838552059701414 73.71604537141194) bank55747 +55748 POINT(40.98691494667874 73.76665880364574) bank55748 +55749 POINT(40.724259265985694 74.74810580600051) bank55749 +55750 POINT(40.8484467788434 74.42551175059666) bank55750 +55751 POINT(40.62890185775406 74.82456490666239) bank55751 +55752 POINT(40.30689136818678 74.86592933912709) bank55752 +55753 POINT(39.97342149500028 73.18711414388048) bank55753 +55754 POINT(41.39718769843361 74.84144305328326) bank55754 +55755 POINT(40.30688615246578 74.40011773746726) bank55755 +55756 POINT(39.9311563177832 73.76024360449759) bank55756 +55757 POINT(41.12428807593605 74.75044109068159) bank55757 +55758 POINT(40.41664724519298 73.3168702391515) bank55758 +55759 POINT(39.85830791583722 73.01534229850444) bank55759 +55760 POINT(41.54389885927277 73.59456603089104) bank55760 +55761 POINT(41.11463812787822 73.96310695508667) bank55761 +55762 POINT(39.73113494130551 74.51997877998748) bank55762 +55763 POINT(39.96284667911695 74.4716536700564) bank55763 +55764 POINT(39.93972226657298 73.23152731540306) bank55764 +55765 POINT(40.27742200185284 73.737404529575) bank55765 +55766 POINT(40.35980177053863 73.50293308515559) bank55766 +55767 POINT(40.38006229521939 74.38224880937109) bank55767 +55768 POINT(40.38750456085869 74.14819089524023) bank55768 +55769 POINT(40.07003218335887 74.59928286281603) bank55769 +55770 POINT(40.42550965451165 73.07083122940001) bank55770 +55771 POINT(41.16782238443726 73.9231539086696) bank55771 +55772 POINT(40.33410234436901 73.57317850302698) bank55772 +55773 POINT(40.70444324841446 74.49151834557466) bank55773 +55774 POINT(40.20048666897843 73.16280761832553) bank55774 +55775 POINT(41.339274241272626 73.10560792752128) bank55775 +55776 POINT(41.68552525137245 73.57822867251932) bank55776 +55777 POINT(40.700551718459835 74.57841167560458) bank55777 +55778 POINT(41.3087190884323 74.2307123070218) bank55778 +55779 POINT(40.46600120336061 74.44371720888675) bank55779 +55780 POINT(41.47028920028082 74.96163021118782) bank55780 +55781 POINT(41.530837393952616 73.28348804777951) bank55781 +55782 POINT(40.5651491287326 73.78604170678092) bank55782 +55783 POINT(41.63888128528419 73.41055672086252) bank55783 +55784 POINT(39.84468714981818 74.57258782575471) bank55784 +55785 POINT(41.34999390494451 73.90169604254197) bank55785 +55786 POINT(41.154463652769124 74.67322864661426) bank55786 +55787 POINT(40.62033455601234 74.80227164770882) bank55787 +55788 POINT(41.071252428519216 73.1852142614795) bank55788 +55789 POINT(41.18966998342292 73.46759240797238) bank55789 +55790 POINT(39.836263992017344 74.4427392928544) bank55790 +55791 POINT(40.204768464475485 73.85551115809169) bank55791 +55792 POINT(40.4145962968832 74.94976768165313) bank55792 +55793 POINT(40.59543182452447 73.17538179772075) bank55793 +55794 POINT(40.79474512819621 74.06202232892326) bank55794 +55795 POINT(41.03764487598603 74.37706116551688) bank55795 +55796 POINT(40.369693726022774 73.75439479743103) bank55796 +55797 POINT(41.24762359946637 74.99508246903049) bank55797 +55798 POINT(39.837346526601124 74.98315967390636) bank55798 +55799 POINT(40.322663047702015 74.84839467520221) bank55799 +55800 POINT(40.905289011456766 73.7819341070572) bank55800 +55801 POINT(41.59475534583619 73.99637839943662) bank55801 +55802 POINT(40.85449041850283 73.02261030890664) bank55802 +55803 POINT(40.00542550509219 74.98009348666723) bank55803 +55804 POINT(41.13982161759436 74.84628091399328) bank55804 +55805 POINT(41.16372073421881 74.20399897510441) bank55805 +55806 POINT(40.1064888822756 73.51731623498387) bank55806 +55807 POINT(41.07093023584788 73.78293271087776) bank55807 +55808 POINT(41.030991354117006 74.78413517124986) bank55808 +55809 POINT(39.777844793441474 74.0030820857354) bank55809 +55810 POINT(40.48286494491978 73.41599164876588) bank55810 +55811 POINT(40.67095304340539 73.82045977437885) bank55811 +55812 POINT(40.86628748696451 74.7092186691605) bank55812 +55813 POINT(40.373380286690086 73.13244506142173) bank55813 +55814 POINT(41.40101308725458 74.93357378454986) bank55814 +55815 POINT(40.85528662910926 73.16877790864824) bank55815 +55816 POINT(39.98044217212439 73.93444266788552) bank55816 +55817 POINT(41.096211452962876 73.92237598096337) bank55817 +55818 POINT(41.47560983060605 74.36020813277322) bank55818 +55819 POINT(41.352094749841164 73.11234761554854) bank55819 +55820 POINT(39.93473967609219 73.51533417390907) bank55820 +55821 POINT(39.8925888012041 74.91207592838451) bank55821 +55822 POINT(40.1673375067077 73.07801502780411) bank55822 +55823 POINT(40.91942210010913 74.34547862957635) bank55823 +55824 POINT(41.4979390993779 74.70803325248708) bank55824 +55825 POINT(39.96139195455635 74.21971348542485) bank55825 +55826 POINT(40.0504819863222 74.87419218691416) bank55826 +55827 POINT(40.61921751380879 74.55458763665956) bank55827 +55828 POINT(41.70368406552221 73.38773487423293) bank55828 +55829 POINT(40.377681645822456 74.52855686300158) bank55829 +55830 POINT(40.08540515375918 74.99132524353216) bank55830 +55831 POINT(41.387664701210745 73.1859094959083) bank55831 +55832 POINT(39.929820097804665 74.49319047870142) bank55832 +55833 POINT(40.65961683633424 73.47010532918095) bank55833 +55834 POINT(40.054812322026784 73.1921896672151) bank55834 +55835 POINT(40.595266964716984 74.36215018808458) bank55835 +55836 POINT(40.18854459182597 74.5673022255489) bank55836 +55837 POINT(41.41434744440694 74.57362448318294) bank55837 +55838 POINT(41.591465857101795 73.63121009844183) bank55838 +55839 POINT(39.77245858755483 74.73908817748853) bank55839 +55840 POINT(40.19529013293608 73.70528148437474) bank55840 +55841 POINT(41.01276380905673 73.8357385541687) bank55841 +55842 POINT(41.691541042855896 74.27116880138651) bank55842 +55843 POINT(40.354860457024856 73.35573284640617) bank55843 +55844 POINT(39.935272892415746 74.659836643243) bank55844 +55845 POINT(41.53974082337443 73.98606883698568) bank55845 +55846 POINT(41.57506796050895 74.13805175648397) bank55846 +55847 POINT(40.541798829300596 73.69624409113078) bank55847 +55848 POINT(41.67264540501514 73.1959588397872) bank55848 +55849 POINT(40.11602587293717 74.06002284215305) bank55849 +55850 POINT(41.302619732977945 74.26371811864118) bank55850 +55851 POINT(40.49347142944433 74.59231938801899) bank55851 +55852 POINT(41.212649335541634 73.16175634611677) bank55852 +55853 POINT(40.96921665724508 73.57027852512465) bank55853 +55854 POINT(40.7659706772089 74.3181347594609) bank55854 +55855 POINT(41.24444621338929 73.79824203224358) bank55855 +55856 POINT(40.92002912644111 73.31424867206219) bank55856 +55857 POINT(40.217746709607255 73.18725502099579) bank55857 +55858 POINT(41.010070787729745 74.68392234385419) bank55858 +55859 POINT(41.152587325189565 74.32894619242764) bank55859 +55860 POINT(41.10571067540069 74.8922542122313) bank55860 +55861 POINT(41.65712922633985 73.50611095600901) bank55861 +55862 POINT(40.98406945451784 74.75923329637563) bank55862 +55863 POINT(39.765810451510475 73.7269099296322) bank55863 +55864 POINT(39.77073427648687 73.07506225106417) bank55864 +55865 POINT(39.80153602312766 74.68751037020847) bank55865 +55866 POINT(41.2010396130255 73.26048542288022) bank55866 +55867 POINT(40.918122165957094 74.02023919175099) bank55867 +55868 POINT(40.78367710353365 74.36420052169363) bank55868 +55869 POINT(41.08605663522103 74.71783338670265) bank55869 +55870 POINT(40.57768849498449 73.87145359285893) bank55870 +55871 POINT(40.54399704206712 74.96390741937117) bank55871 +55872 POINT(39.96839221438599 73.20981788187088) bank55872 +55873 POINT(41.59375792747751 74.28776509015529) bank55873 +55874 POINT(41.27331750395215 74.49243807168915) bank55874 +55875 POINT(40.2916931518246 74.84202035498768) bank55875 +55876 POINT(41.43674857666324 73.8432552661397) bank55876 +55877 POINT(39.88629141596771 74.10040096894068) bank55877 +55878 POINT(40.15229009810655 73.88445215242544) bank55878 +55879 POINT(40.93064787159288 73.5377775940363) bank55879 +55880 POINT(40.73662607563144 74.02550856911267) bank55880 +55881 POINT(40.09755909739711 73.16728922793452) bank55881 +55882 POINT(40.58586601010767 74.68292630385191) bank55882 +55883 POINT(40.64820080643806 73.2107460204488) bank55883 +55884 POINT(41.617809085889846 74.21057128961185) bank55884 +55885 POINT(39.77399861412697 74.61394718318255) bank55885 +55886 POINT(40.56087367389174 73.2548881348225) bank55886 +55887 POINT(40.22301574010228 73.47305959436575) bank55887 +55888 POINT(41.59497744655279 74.81579501480792) bank55888 +55889 POINT(40.595863217725174 74.4993483574752) bank55889 +55890 POINT(40.65055044694794 73.58516283476878) bank55890 +55891 POINT(41.396892611045374 73.80316348330678) bank55891 +55892 POINT(41.248130746085224 74.50188908599978) bank55892 +55893 POINT(40.185253693746986 74.43577044679932) bank55893 +55894 POINT(40.46720753655872 73.91226888429289) bank55894 +55895 POINT(41.32632404862578 73.68443563636771) bank55895 +55896 POINT(40.807329379702004 73.24021529034592) bank55896 +55897 POINT(40.79003906234351 73.04463240266801) bank55897 +55898 POINT(40.70160443970878 73.80424963617035) bank55898 +55899 POINT(40.79263413349832 74.72267224272215) bank55899 +55900 POINT(40.411920897305045 74.26533996533453) bank55900 +55901 POINT(40.690075297105295 73.972107455247) bank55901 +55902 POINT(40.39130090277442 73.3248499356835) bank55902 +55903 POINT(40.43363250264847 73.90154818751782) bank55903 +55904 POINT(39.910553708038684 73.40896264927002) bank55904 +55905 POINT(40.18977293184487 74.78809595062845) bank55905 +55906 POINT(41.20882015035599 74.72335745730338) bank55906 +55907 POINT(41.015446685749716 74.26003191319917) bank55907 +55908 POINT(40.59150513561532 73.29471464369365) bank55908 +55909 POINT(40.059840014154744 73.40930334712012) bank55909 +55910 POINT(41.0853745292469 73.87915801486965) bank55910 +55911 POINT(40.65568586152544 73.12898441586586) bank55911 +55912 POINT(40.3059747529708 74.51642647487355) bank55912 +55913 POINT(39.92629098088241 73.4375684196824) bank55913 +55914 POINT(40.27580472824188 73.95664717313336) bank55914 +55915 POINT(41.01833374135049 74.67052461930213) bank55915 +55916 POINT(41.70769291746616 74.31720816083883) bank55916 +55917 POINT(40.71154443707957 73.05623766683082) bank55917 +55918 POINT(40.01343748943409 74.84288197280038) bank55918 +55919 POINT(41.08540180223658 73.71999886590551) bank55919 +55920 POINT(40.303737370169614 73.2775273588286) bank55920 +55921 POINT(40.08785768612131 74.94554599981949) bank55921 +55922 POINT(40.61374974899061 74.4244777999512) bank55922 +55923 POINT(39.71296613165464 74.39399780014323) bank55923 +55924 POINT(40.846317977419545 73.02054073178006) bank55924 +55925 POINT(41.238127495093195 74.75256573830644) bank55925 +55926 POINT(40.15998340308967 73.63127514723762) bank55926 +55927 POINT(40.622783238794206 74.17932018740787) bank55927 +55928 POINT(40.292972539273464 73.6742250684383) bank55928 +55929 POINT(40.05675958452045 73.26867841412957) bank55929 +55930 POINT(39.795727732604924 74.15973181804924) bank55930 +55931 POINT(40.2518432927558 74.07150921380351) bank55931 +55932 POINT(40.115354790138326 74.37073164787412) bank55932 +55933 POINT(39.968008251311346 74.26028332729786) bank55933 +55934 POINT(41.01522449406981 74.00200794228488) bank55934 +55935 POINT(41.52781174882312 74.76537189264612) bank55935 +55936 POINT(40.77836725009111 74.58782474606028) bank55936 +55937 POINT(40.39087871418785 73.74294400956326) bank55937 +55938 POINT(40.15296479040939 73.55656656384433) bank55938 +55939 POINT(40.33453993214162 73.74610401494307) bank55939 +55940 POINT(40.040939048713305 73.66693999715528) bank55940 +55941 POINT(40.91018327149815 73.5565872684325) bank55941 +55942 POINT(40.878728664249095 74.87030232585857) bank55942 +55943 POINT(41.05048552911877 73.30497122339322) bank55943 +55944 POINT(40.59583247747649 73.20200597623257) bank55944 +55945 POINT(40.248759996261526 73.80408341423063) bank55945 +55946 POINT(40.35099605242314 73.07523991484156) bank55946 +55947 POINT(39.74398373009964 73.67724341822398) bank55947 +55948 POINT(39.94665689790287 73.37979173790298) bank55948 +55949 POINT(41.24408111323469 74.54996818642398) bank55949 +55950 POINT(40.14382429628478 73.8804801396776) bank55950 +55951 POINT(40.72882258094144 74.89823176605545) bank55951 +55952 POINT(40.19450623338664 73.02082710457917) bank55952 +55953 POINT(39.7833932351773 73.30966251149293) bank55953 +55954 POINT(40.5562064542091 74.28458008123131) bank55954 +55955 POINT(41.13594901215627 74.21046325150614) bank55955 +55956 POINT(40.012663109923906 73.22799341179261) bank55956 +55957 POINT(41.32502110377522 74.30034134216021) bank55957 +55958 POINT(41.60020217191343 73.20753428225515) bank55958 +55959 POINT(41.44429385956797 73.91070086175945) bank55959 +55960 POINT(41.42959493230593 73.92203273521274) bank55960 +55961 POINT(40.03738617436687 73.43491950829285) bank55961 +55962 POINT(39.72615772655839 74.18663959744623) bank55962 +55963 POINT(40.79937376656255 73.3627189792132) bank55963 +55964 POINT(41.6185455320558 74.37783872845218) bank55964 +55965 POINT(41.64767306840986 74.06255582161687) bank55965 +55966 POINT(39.93284228117971 74.50162633584127) bank55966 +55967 POINT(41.421639774440564 73.75736988174948) bank55967 +55968 POINT(41.27940790760116 74.73367833991814) bank55968 +55969 POINT(39.90349224367563 73.02369536613952) bank55969 +55970 POINT(41.700938409433476 74.01766637727374) bank55970 +55971 POINT(39.815310465307746 74.62644294705447) bank55971 +55972 POINT(41.348372030537945 73.11999977119675) bank55972 +55973 POINT(41.58080989397972 74.93603758466631) bank55973 +55974 POINT(39.76886534752209 74.78160785632846) bank55974 +55975 POINT(40.68516617847854 74.97889926614698) bank55975 +55976 POINT(40.19425726756627 74.0502029657257) bank55976 +55977 POINT(39.83184854392626 74.60916159138526) bank55977 +55978 POINT(40.69251299465978 73.54354776051625) bank55978 +55979 POINT(40.11153094441036 73.6376600415095) bank55979 +55980 POINT(41.46361618140018 74.05448158465288) bank55980 +55981 POINT(40.5783023161045 73.8316316260183) bank55981 +55982 POINT(40.31097527580234 73.67649559346535) bank55982 +55983 POINT(40.746807950134006 74.02391104958517) bank55983 +55984 POINT(41.18418426347506 73.31859396375131) bank55984 +55985 POINT(40.8414361904729 74.06191512344347) bank55985 +55986 POINT(41.184105382332255 74.33088013413457) bank55986 +55987 POINT(39.72426878752144 73.44555575285307) bank55987 +55988 POINT(40.37998099170737 73.60593116194092) bank55988 +55989 POINT(41.28205023926191 74.93034592306051) bank55989 +55990 POINT(40.90016745045106 74.78209059700268) bank55990 +55991 POINT(41.561913189920304 73.73030113825513) bank55991 +55992 POINT(40.67843069557892 73.44479050479049) bank55992 +55993 POINT(41.55850715690136 74.52400745540118) bank55993 +55994 POINT(41.370067464056675 74.58305183014019) bank55994 +55995 POINT(39.810261307578465 73.83639321424798) bank55995 +55996 POINT(41.70835800795058 73.0285968473509) bank55996 +55997 POINT(41.623522094900274 73.6192308307483) bank55997 +55998 POINT(41.111256851068994 73.38198927274468) bank55998 +55999 POINT(41.435953881014335 74.62059992539979) bank55999 +56000 POINT(40.76934901377154 73.31225007180316) bank56000 +56001 POINT(40.99831794312603 74.56277101709287) bank56001 +56002 POINT(41.03508573796145 74.2551307035186) bank56002 +56003 POINT(41.24675184937276 74.94868289740153) bank56003 +56004 POINT(39.80979057724921 74.33144697114231) bank56004 +56005 POINT(39.794373631965975 74.27829839941) bank56005 +56006 POINT(41.70403313062496 73.30814718176434) bank56006 +56007 POINT(41.251428889844604 74.6009866878845) bank56007 +56008 POINT(41.635183249134705 73.68869031484367) bank56008 +56009 POINT(40.73686068701673 73.5472956654461) bank56009 +56010 POINT(40.43131697074337 73.65370248583066) bank56010 +56011 POINT(41.317600725525665 74.02134380374737) bank56011 +56012 POINT(41.35157878321302 73.96886717066774) bank56012 +56013 POINT(40.98263370731453 73.00983652039558) bank56013 +56014 POINT(40.27232315112778 73.33571100515088) bank56014 +56015 POINT(41.62042274244768 74.15880471543981) bank56015 +56016 POINT(40.66901880023262 74.95481981464316) bank56016 +56017 POINT(40.722556293412794 74.48475612819139) bank56017 +56018 POINT(40.88354103057008 74.437645384064) bank56018 +56019 POINT(41.53355247197282 73.11699587712172) bank56019 +56020 POINT(41.343132593269175 73.55408111121993) bank56020 +56021 POINT(40.12722605371969 73.42215776499246) bank56021 +56022 POINT(41.300966598434414 73.85767289349644) bank56022 +56023 POINT(40.133977626561546 73.65069992935891) bank56023 +56024 POINT(41.18573291571602 73.44883693483298) bank56024 +56025 POINT(41.69669275504574 74.92645092968957) bank56025 +56026 POINT(40.606960669572565 74.54008500960019) bank56026 +56027 POINT(41.29419386879483 74.92794670730859) bank56027 +56028 POINT(41.32162839062266 74.50795458802203) bank56028 +56029 POINT(41.22168205603968 73.41983055542894) bank56029 +56030 POINT(40.72395876150922 73.5139692705606) bank56030 +56031 POINT(40.74498292695902 73.54654317326695) bank56031 +56032 POINT(40.88446952578998 74.64258736220762) bank56032 +56033 POINT(40.59310614231563 73.59430299553613) bank56033 +56034 POINT(41.00000095714126 73.8038863698195) bank56034 +56035 POINT(40.13488558451469 74.56586217237438) bank56035 +56036 POINT(40.956198569752154 74.32447864415346) bank56036 +56037 POINT(41.321466109048124 73.2927385905568) bank56037 +56038 POINT(41.42585497461925 74.5243874207035) bank56038 +56039 POINT(40.56359879399253 73.82969593308435) bank56039 +56040 POINT(41.17400215986877 74.998993054507) bank56040 +56041 POINT(40.350191084636236 74.91270422083119) bank56041 +56042 POINT(39.87515913342081 73.21613050098543) bank56042 +56043 POINT(41.45721054828693 73.44304285182655) bank56043 +56044 POINT(41.03499979967487 74.59759662117297) bank56044 +56045 POINT(41.31225161118944 74.27298470689875) bank56045 +56046 POINT(40.34887066869882 73.85240971488128) bank56046 +56047 POINT(40.94846866251955 74.53284646358432) bank56047 +56048 POINT(41.12471990395152 73.51406782938986) bank56048 +56049 POINT(39.72879935653576 73.51929042583903) bank56049 +56050 POINT(41.63892556118801 73.46526222982959) bank56050 +56051 POINT(41.10552783637127 73.05103951069674) bank56051 +56052 POINT(40.93289446485468 73.84658404428193) bank56052 +56053 POINT(41.62287256011287 73.96469522061648) bank56053 +56054 POINT(40.47247176852596 73.64190318338467) bank56054 +56055 POINT(40.03516899788229 74.20100624161819) bank56055 +56056 POINT(41.11283323631882 74.99592876834575) bank56056 +56057 POINT(40.023885879307116 74.53436721520156) bank56057 +56058 POINT(40.060236333155686 74.26465369189056) bank56058 +56059 POINT(41.54941942135612 74.9735021694421) bank56059 +56060 POINT(39.78425246672293 73.7860853627878) bank56060 +56061 POINT(41.35361205519402 74.54757547854459) bank56061 +56062 POINT(41.148245849601985 74.76354982370118) bank56062 +56063 POINT(40.643058262983715 74.4301016964178) bank56063 +56064 POINT(41.325921383825936 74.73562403413295) bank56064 +56065 POINT(40.690697487924886 73.22780517867629) bank56065 +56066 POINT(40.46653570400794 73.57552521586692) bank56066 +56067 POINT(40.01869011242692 74.48911915440098) bank56067 +56068 POINT(41.030306194871464 73.87744972572776) bank56068 +56069 POINT(41.388433141406864 73.06284833563805) bank56069 +56070 POINT(39.88970934709963 73.97344411121132) bank56070 +56071 POINT(41.07392830426277 74.9528346343865) bank56071 +56072 POINT(40.593230903371435 73.31682681477024) bank56072 +56073 POINT(39.95446684322534 73.51819593550998) bank56073 +56074 POINT(41.04481999414549 74.32815323872224) bank56074 +56075 POINT(41.60829405741783 73.50853321648803) bank56075 +56076 POINT(41.466230177522334 73.10250492177325) bank56076 +56077 POINT(39.80400535878684 74.24316892826583) bank56077 +56078 POINT(40.605194762654826 73.60127242784654) bank56078 +56079 POINT(41.26480241888319 73.12347633705568) bank56079 +56080 POINT(40.38868806306268 74.4147752380487) bank56080 +56081 POINT(41.47758981724824 73.94213376040403) bank56081 +56082 POINT(41.32441219172164 74.23902844192753) bank56082 +56083 POINT(41.12742643400303 73.9117024356531) bank56083 +56084 POINT(39.97049673896664 74.9052121080862) bank56084 +56085 POINT(40.06413692607781 74.75622298908804) bank56085 +56086 POINT(41.2521343684349 73.61614771978839) bank56086 +56087 POINT(40.700867533841084 74.12333494230568) bank56087 +56088 POINT(39.93349838576704 74.51864033410655) bank56088 +56089 POINT(39.99078895199686 74.38568772108745) bank56089 +56090 POINT(41.46175883958802 73.33144731201094) bank56090 +56091 POINT(39.93869311674299 74.98911984404769) bank56091 +56092 POINT(40.7772986455778 74.39455672008422) bank56092 +56093 POINT(40.33053212063423 73.39407361441835) bank56093 +56094 POINT(41.35482507410697 74.45889197713025) bank56094 +56095 POINT(40.16447197118982 74.2391776851146) bank56095 +56096 POINT(40.534108918761056 73.82351396014863) bank56096 +56097 POINT(39.877367718811406 74.400229387553) bank56097 +56098 POINT(41.390118364992354 74.76190166742742) bank56098 +56099 POINT(40.75067119432111 74.0681118761237) bank56099 +56100 POINT(40.602678547576446 74.13235612789963) bank56100 +56101 POINT(41.17559672404062 73.60609934181518) bank56101 +56102 POINT(40.368190501581495 73.9806446975248) bank56102 +56103 POINT(40.00289938725211 74.99764410799422) bank56103 +56104 POINT(41.37883615975354 73.58854506635878) bank56104 +56105 POINT(41.207866865740705 73.52967455435963) bank56105 +56106 POINT(40.977182464759416 74.03755632627802) bank56106 +56107 POINT(41.65469086222771 73.67202749161162) bank56107 +56108 POINT(40.11060308567465 74.49508151358786) bank56108 +56109 POINT(41.58062707253552 73.30758500044927) bank56109 +56110 POINT(40.55568361106714 74.13874840895066) bank56110 +56111 POINT(40.17323959019852 73.15274499985904) bank56111 +56112 POINT(40.37166818982177 74.83430741204604) bank56112 +56113 POINT(41.25229058327181 74.38192583372098) bank56113 +56114 POINT(41.42493608889291 73.66791888267532) bank56114 +56115 POINT(39.73214541378588 73.0937357456718) bank56115 +56116 POINT(40.32000588584915 73.27666406704746) bank56116 +56117 POINT(39.82137339913962 74.94732428587777) bank56117 +56118 POINT(40.52653479331489 73.33913996136545) bank56118 +56119 POINT(41.597596668595294 74.14122762940582) bank56119 +56120 POINT(39.941762300403006 73.97500607520779) bank56120 +56121 POINT(41.611483368266036 73.55603714871192) bank56121 +56122 POINT(40.20721019760314 73.872834840204) bank56122 +56123 POINT(40.322380920054705 73.98828719838917) bank56123 +56124 POINT(40.54475423711075 74.69840240374242) bank56124 +56125 POINT(39.927908433178565 74.20284801344269) bank56125 +56126 POINT(41.297803285480796 74.33179445796252) bank56126 +56127 POINT(40.50082516879068 74.74214745423505) bank56127 +56128 POINT(41.36904527430348 73.83644431773969) bank56128 +56129 POINT(40.88326283709789 73.16912605003282) bank56129 +56130 POINT(40.3920589763272 73.5389439589791) bank56130 +56131 POINT(40.79025426604974 73.93552152579055) bank56131 +56132 POINT(40.90667125694585 73.18719324300216) bank56132 +56133 POINT(40.74110290535894 73.74158225544078) bank56133 +56134 POINT(40.83172785793396 73.95303957465038) bank56134 +56135 POINT(40.88755973370164 74.15471514403066) bank56135 +56136 POINT(41.489943927734366 74.47242277908501) bank56136 +56137 POINT(41.30048646583863 73.50709087147757) bank56137 +56138 POINT(41.56003544657208 74.95914003464333) bank56138 +56139 POINT(40.30699227272323 73.90656209652099) bank56139 +56140 POINT(41.44587063245594 74.36386973421466) bank56140 +56141 POINT(40.958889483681546 74.62495691939114) bank56141 +56142 POINT(40.59356464451399 73.26399652315793) bank56142 +56143 POINT(40.31046987051121 73.50296418074788) bank56143 +56144 POINT(40.45423062962713 73.1499948932118) bank56144 +56145 POINT(39.73274300664312 73.55108644309965) bank56145 +56146 POINT(40.6394621188918 74.40704215974971) bank56146 +56147 POINT(39.97847155559315 73.14453340211523) bank56147 +56148 POINT(40.746262267134696 74.620121697277) bank56148 +56149 POINT(41.252004552038265 73.200957367907) bank56149 +56150 POINT(39.86100743206826 74.03318279956493) bank56150 +56151 POINT(40.44037701957858 73.16362380855166) bank56151 +56152 POINT(41.22418307909487 73.3078183247697) bank56152 +56153 POINT(39.93923013446247 73.88789140538543) bank56153 +56154 POINT(40.01711523762382 73.27335161544545) bank56154 +56155 POINT(41.1342753215967 74.516207662078) bank56155 +56156 POINT(40.826304411737986 74.26894057297261) bank56156 +56157 POINT(41.63698135701393 74.86078138329982) bank56157 +56158 POINT(41.357464766503895 74.18655388434864) bank56158 +56159 POINT(40.431368344221546 74.17587341122872) bank56159 +56160 POINT(41.37425951258493 74.29742128252909) bank56160 +56161 POINT(40.19675107001251 73.33148405430757) bank56161 +56162 POINT(40.981952548008586 73.47966366934035) bank56162 +56163 POINT(39.86806508085731 73.47030075135015) bank56163 +56164 POINT(41.114986257507 74.43544826486306) bank56164 +56165 POINT(39.96784360803407 73.7160346793667) bank56165 +56166 POINT(39.97436652572907 73.45827064562295) bank56166 +56167 POINT(39.93166026403758 73.23844970085631) bank56167 +56168 POINT(40.04886491472109 74.07960948094113) bank56168 +56169 POINT(40.615383484774235 73.90745355512169) bank56169 +56170 POINT(40.438046100196196 74.7477667207365) bank56170 +56171 POINT(41.60426040874253 74.81984476347523) bank56171 +56172 POINT(41.567907532241826 73.06148152783844) bank56172 +56173 POINT(40.88573269194379 74.76204535770572) bank56173 +56174 POINT(39.84689199998402 73.14218086827644) bank56174 +56175 POINT(41.03824331986952 73.53975545181436) bank56175 +56176 POINT(40.41401234973721 73.5456583418113) bank56176 +56177 POINT(40.15081642793472 73.63378681077266) bank56177 +56178 POINT(41.03016429216338 74.75512647805147) bank56178 +56179 POINT(41.50352580577181 73.61951859010452) bank56179 +56180 POINT(40.21599205856076 74.25952240011416) bank56180 +56181 POINT(40.79750449265392 73.75569671069746) bank56181 +56182 POINT(39.99312339959943 74.77497643585075) bank56182 +56183 POINT(41.04692240506248 74.7588724501214) bank56183 +56184 POINT(40.510911590459514 74.45234651091283) bank56184 +56185 POINT(41.50451033328329 74.3285479607121) bank56185 +56186 POINT(41.61016570656109 73.21525614256512) bank56186 +56187 POINT(39.895991792334264 74.63494416868922) bank56187 +56188 POINT(41.47322502414769 74.4361821204421) bank56188 +56189 POINT(40.691500718161244 74.55427025051002) bank56189 +56190 POINT(40.63840909184547 73.44781113181395) bank56190 +56191 POINT(40.99632525989792 73.67946321383062) bank56191 +56192 POINT(40.53364615648658 73.43543894577186) bank56192 +56193 POINT(40.29614920694908 74.91397015374602) bank56193 +56194 POINT(41.31473019414057 74.04687802756263) bank56194 +56195 POINT(39.957960001537664 73.74598234335383) bank56195 +56196 POINT(40.39936961331083 73.97994036863759) bank56196 +56197 POINT(40.93338018779566 73.64955549303967) bank56197 +56198 POINT(40.82243483082078 74.02166487766166) bank56198 +56199 POINT(40.06189311730424 74.68146473001785) bank56199 +56200 POINT(41.02570391951737 74.60469646015751) bank56200 +56201 POINT(40.721597178277726 74.63043226101362) bank56201 +56202 POINT(40.40468294703399 74.58541541306977) bank56202 +56203 POINT(40.53362920054209 74.2263865001511) bank56203 +56204 POINT(41.67459892233981 74.51890743119704) bank56204 +56205 POINT(40.99484133202291 74.82105355439539) bank56205 +56206 POINT(41.39854012603537 73.69082889816582) bank56206 +56207 POINT(40.248255632023785 74.62122197372291) bank56207 +56208 POINT(39.97777002056587 73.64884774157403) bank56208 +56209 POINT(41.4060901594047 74.59661701125526) bank56209 +56210 POINT(40.15321807202705 73.93742831836605) bank56210 +56211 POINT(40.19331459789201 74.42667427319813) bank56211 +56212 POINT(41.70936577148148 73.448308816126) bank56212 +56213 POINT(40.39835226527308 73.77696802213576) bank56213 +56214 POINT(41.3313826986545 74.03933100938163) bank56214 +56215 POINT(41.20349074726438 73.89693674984461) bank56215 +56216 POINT(39.7898397997116 73.57436540304559) bank56216 +56217 POINT(41.420696168080696 74.59806627362394) bank56217 +56218 POINT(40.6580006711028 73.7005841621511) bank56218 +56219 POINT(40.22326037759676 73.21594197104488) bank56219 +56220 POINT(40.347315357036145 73.104243124996) bank56220 +56221 POINT(40.215278500413724 73.29974849887061) bank56221 +56222 POINT(40.88243542412423 74.2426392749007) bank56222 +56223 POINT(41.66701203049395 74.93715644691605) bank56223 +56224 POINT(41.44455128737119 74.6243666079243) bank56224 +56225 POINT(40.82432848994877 74.90985659713061) bank56225 +56226 POINT(41.32861558905455 73.83909387152514) bank56226 +56227 POINT(40.66501621805875 74.41647471945285) bank56227 +56228 POINT(41.199009162784904 74.57418068668007) bank56228 +56229 POINT(39.756118431989776 74.60491626918171) bank56229 +56230 POINT(41.35031586923134 73.93394424062822) bank56230 +56231 POINT(39.91265082158045 74.92157099933786) bank56231 +56232 POINT(40.25116088799885 73.67100361244903) bank56232 +56233 POINT(40.818581145450565 74.80301384587554) bank56233 +56234 POINT(41.15878791250421 74.41442695601242) bank56234 +56235 POINT(40.25860604129176 74.21147153787518) bank56235 +56236 POINT(41.34183596809004 74.46240189705993) bank56236 +56237 POINT(39.81821679869579 73.30552942281041) bank56237 +56238 POINT(41.360831157395126 73.83468753471776) bank56238 +56239 POINT(41.38187591386074 73.73792199327592) bank56239 +56240 POINT(40.6557637667249 74.34831918379535) bank56240 +56241 POINT(41.67196168808331 73.17567440273447) bank56241 +56242 POINT(41.59518082316744 73.70061216165757) bank56242 +56243 POINT(40.425620893398836 73.67893037579239) bank56243 +56244 POINT(40.453688373875565 73.03439230853165) bank56244 +56245 POINT(39.99625624417072 74.78206184135477) bank56245 +56246 POINT(39.96716743809338 73.832035478603) bank56246 +56247 POINT(40.133778378207666 74.3613159856007) bank56247 +56248 POINT(40.37622508748511 73.41397977594504) bank56248 +56249 POINT(40.35576720649207 74.02098149779383) bank56249 +56250 POINT(39.77152797174649 73.49125199347765) bank56250 +56251 POINT(40.893927173156165 73.68256032356243) bank56251 +56252 POINT(41.37178903086612 73.54071652728935) bank56252 +56253 POINT(39.777769637667056 74.31181742627182) bank56253 +56254 POINT(41.09172691415335 74.9836395078305) bank56254 +56255 POINT(41.48420880325523 73.55335350992563) bank56255 +56256 POINT(41.47468667403523 73.26441303968733) bank56256 +56257 POINT(39.77288733993904 74.91406194319717) bank56257 +56258 POINT(40.81238476160484 75.0057476225506) bank56258 +56259 POINT(39.948142240052746 74.29776807468231) bank56259 +56260 POINT(39.74928780647946 73.09077818828808) bank56260 +56261 POINT(39.85616400815593 74.95737922581141) bank56261 +56262 POINT(40.84620748762893 73.4510391435173) bank56262 +56263 POINT(41.36871569136549 74.26329278002672) bank56263 +56264 POINT(40.912294056526314 74.57867799435819) bank56264 +56265 POINT(40.98701386817301 73.80692891482211) bank56265 +56266 POINT(40.309918330102505 73.8599296830122) bank56266 +56267 POINT(41.0535930962177 73.36990459791987) bank56267 +56268 POINT(41.67381284030156 74.73578627868082) bank56268 +56269 POINT(39.744003142386745 73.43569975189396) bank56269 +56270 POINT(40.58783173221228 73.54517977895144) bank56270 +56271 POINT(40.14448018238444 73.32999735485906) bank56271 +56272 POINT(39.749220711656186 73.04186123948679) bank56272 +56273 POINT(40.08643131634754 73.25107733351638) bank56273 +56274 POINT(39.8546907932862 73.61988191005062) bank56274 +56275 POINT(41.08737886697389 73.92316439077187) bank56275 +56276 POINT(41.64283713872805 73.84659350488377) bank56276 +56277 POINT(41.14600546371401 74.25775049477129) bank56277 +56278 POINT(40.04936211738449 74.2716761915086) bank56278 +56279 POINT(41.261908780800304 74.41492836200207) bank56279 +56280 POINT(40.60297360563131 74.22714749577183) bank56280 +56281 POINT(39.90836780980555 74.74867775423154) bank56281 +56282 POINT(40.04321000630805 74.27172927525268) bank56282 +56283 POINT(39.789923627799084 73.13831917379112) bank56283 +56284 POINT(40.52992235354894 73.59860108232294) bank56284 +56285 POINT(40.964975473842976 73.86910072615531) bank56285 +56286 POINT(40.449616634059396 74.04164870630869) bank56286 +56287 POINT(39.985085582935504 73.21904113211188) bank56287 +56288 POINT(39.99638539638115 73.02709167399685) bank56288 +56289 POINT(39.97229350106335 74.54741749301661) bank56289 +56290 POINT(41.1127403217337 74.92077871689426) bank56290 +56291 POINT(40.95023163885904 73.13938423637974) bank56291 +56292 POINT(41.70714744741019 73.65379710216601) bank56292 +56293 POINT(41.25515680338262 74.33105021926835) bank56293 +56294 POINT(40.23316955044178 73.75395526709815) bank56294 +56295 POINT(40.273473124124024 73.39156560360334) bank56295 +56296 POINT(40.64729835740117 73.58240532810899) bank56296 +56297 POINT(40.52399410304532 73.03219567190122) bank56297 +56298 POINT(41.550469461881804 73.3726715980445) bank56298 +56299 POINT(40.77081765181961 74.73469695701878) bank56299 +56300 POINT(39.95919284868528 73.39420098839324) bank56300 +56301 POINT(41.14223761065896 73.6609700881192) bank56301 +56302 POINT(39.713188126593366 73.42507281501292) bank56302 +56303 POINT(40.98759611709423 73.35539446767076) bank56303 +56304 POINT(40.57470328665008 74.25691166661181) bank56304 +56305 POINT(40.31644759908173 74.69537020707274) bank56305 +56306 POINT(40.364822536203256 73.62155704877777) bank56306 +56307 POINT(40.16740185205214 73.4724527829187) bank56307 +56308 POINT(40.38157230824222 73.30621869151896) bank56308 +56309 POINT(40.13377864419318 73.47488608061518) bank56309 +56310 POINT(40.0247596222537 73.61431576473966) bank56310 +56311 POINT(40.04242966964001 74.488451535166) bank56311 +56312 POINT(39.80879225980224 73.13622581954542) bank56312 +56313 POINT(41.073567480264735 73.15776891777264) bank56313 +56314 POINT(40.69533203674892 74.43179349023642) bank56314 +56315 POINT(41.462642418511145 73.545826389782) bank56315 +56316 POINT(40.199707604577405 73.9521645580262) bank56316 +56317 POINT(40.69912899725832 74.84392210608374) bank56317 +56318 POINT(41.07352709572208 74.27612771963811) bank56318 +56319 POINT(40.168425389195335 73.73788924438419) bank56319 +56320 POINT(41.51175981399384 74.65834787013908) bank56320 +56321 POINT(40.39252560091033 73.87297005533817) bank56321 +56322 POINT(41.485287621389304 73.82303316795648) bank56322 +56323 POINT(41.09781589485443 74.9103758021037) bank56323 +56324 POINT(40.28763678157832 73.43318950778958) bank56324 +56325 POINT(41.44957296821071 74.01069781209127) bank56325 +56326 POINT(40.56751257218405 74.21726324291002) bank56326 +56327 POINT(40.28669921957993 73.23228872111258) bank56327 +56328 POINT(41.573603003711945 74.83910802154357) bank56328 +56329 POINT(40.96530767308986 73.39377403283906) bank56329 +56330 POINT(41.283014608309706 74.82869436132069) bank56330 +56331 POINT(41.42674490754585 73.31097537929844) bank56331 +56332 POINT(40.435845981605574 74.09580868298364) bank56332 +56333 POINT(41.49490362185167 74.64837606707891) bank56333 +56334 POINT(40.104305110668896 74.78928123165497) bank56334 +56335 POINT(41.09248633342506 74.35558090519336) bank56335 +56336 POINT(41.503038966710044 73.48319974988698) bank56336 +56337 POINT(41.002043487626075 73.8101260862228) bank56337 +56338 POINT(41.27869793491405 73.43275416598263) bank56338 +56339 POINT(40.84496473462334 74.95317691642894) bank56339 +56340 POINT(40.3941605439808 74.4101605809575) bank56340 +56341 POINT(40.147098222312664 73.95477695035864) bank56341 +56342 POINT(39.863452136638394 73.78761101159093) bank56342 +56343 POINT(41.06092579547145 73.29459517216938) bank56343 +56344 POINT(39.95673724830475 73.92674634574492) bank56344 +56345 POINT(40.4082781735086 74.25450210426774) bank56345 +56346 POINT(41.1395945924677 73.96492130444642) bank56346 +56347 POINT(40.053507576854166 74.06298665875062) bank56347 +56348 POINT(40.49487460588846 74.27152319928396) bank56348 +56349 POINT(40.73954436014923 73.34485003713378) bank56349 +56350 POINT(40.36998638360902 73.86454746932522) bank56350 +56351 POINT(39.95648406396211 74.96158267219963) bank56351 +56352 POINT(41.176066019985235 73.84456823868744) bank56352 +56353 POINT(41.16936922089467 73.22359980042324) bank56353 +56354 POINT(41.64743574491511 74.18728914435229) bank56354 +56355 POINT(41.332575232301984 73.66754631819073) bank56355 +56356 POINT(40.2961983917535 74.90783721033172) bank56356 +56357 POINT(41.16020414607301 74.11599193779794) bank56357 +56358 POINT(40.526391306827215 74.96754648904987) bank56358 +56359 POINT(39.91405347631547 74.85388822787561) bank56359 +56360 POINT(41.70982899905084 74.20706191117554) bank56360 +56361 POINT(41.2590596364193 73.24139311501386) bank56361 +56362 POINT(40.403775791818894 73.10084288980215) bank56362 +56363 POINT(40.1250145760042 73.62178944704259) bank56363 +56364 POINT(40.494566996688775 73.07086750663048) bank56364 +56365 POINT(41.15801518694543 73.82329218496791) bank56365 +56366 POINT(39.823521217656214 74.24824296621058) bank56366 +56367 POINT(40.414490854109765 73.15832943425981) bank56367 +56368 POINT(40.262032703618445 74.85642470124918) bank56368 +56369 POINT(40.0504188560175 73.09009477094288) bank56369 +56370 POINT(40.92558155557123 73.07619772166555) bank56370 +56371 POINT(40.932391840546636 74.78261592601835) bank56371 +56372 POINT(40.68062650305457 74.32399477058719) bank56372 +56373 POINT(39.804012629607044 73.7552849077856) bank56373 +56374 POINT(40.00277386749691 74.95409150709858) bank56374 +56375 POINT(39.77263226884472 73.07059783994407) bank56375 +56376 POINT(41.566122324486074 74.7641316982365) bank56376 +56377 POINT(40.033647711811014 73.50614899842274) bank56377 +56378 POINT(41.197299518069414 73.44252606626411) bank56378 +56379 POINT(40.48801269247495 74.30910697985979) bank56379 +56380 POINT(41.599551846110415 74.17506964783574) bank56380 +56381 POINT(41.57269250290749 73.93537945776092) bank56381 +56382 POINT(39.71791996754386 73.16628407855649) bank56382 +56383 POINT(41.363058109073464 74.93427716212197) bank56383 +56384 POINT(40.53606015598106 73.54252799417394) bank56384 +56385 POINT(41.33254028411035 73.7102787281052) bank56385 +56386 POINT(40.16590334484921 74.1225061556427) bank56386 +56387 POINT(41.498362657883035 73.04007883600734) bank56387 +56388 POINT(40.57268475949018 74.99380245474892) bank56388 +56389 POINT(39.72304819260711 73.40204787084238) bank56389 +56390 POINT(41.09668210637843 74.3204789734319) bank56390 +56391 POINT(39.85491833139035 74.20864912803002) bank56391 +56392 POINT(39.87075893663457 74.36756631337143) bank56392 +56393 POINT(40.209615790414986 73.96830060027963) bank56393 +56394 POINT(40.25630482343527 74.34748118503899) bank56394 +56395 POINT(41.50375649177053 74.67019351108158) bank56395 +56396 POINT(41.56448230543078 74.96344259140942) bank56396 +56397 POINT(39.95310684362972 74.14377914110075) bank56397 +56398 POINT(40.904913397322964 73.93457585938215) bank56398 +56399 POINT(40.353446001400556 73.55034924574625) bank56399 +56400 POINT(41.33019107095514 74.63604856165657) bank56400 +56401 POINT(41.225532140114886 74.72333113172624) bank56401 +56402 POINT(39.97290525797177 74.5270010149604) bank56402 +56403 POINT(39.81270311510269 74.01831350645016) bank56403 +56404 POINT(40.83244354655047 74.38345274227) bank56404 +56405 POINT(41.41279186213601 73.50028882276553) bank56405 +56406 POINT(39.86388748260272 74.7443234094936) bank56406 +56407 POINT(40.32604921443243 74.99717043662787) bank56407 +56408 POINT(40.594434213399 74.7479266615457) bank56408 +56409 POINT(40.4834277051393 74.32275819372249) bank56409 +56410 POINT(39.97459251436983 73.6446059429451) bank56410 +56411 POINT(40.222456064890075 74.05075927085396) bank56411 +56412 POINT(41.33622628891268 74.18449545471184) bank56412 +56413 POINT(39.929690421102066 74.6726673770664) bank56413 +56414 POINT(41.437109302669406 73.54634742169011) bank56414 +56415 POINT(41.12817373041774 74.05892494643764) bank56415 +56416 POINT(41.31313819745776 73.80631312764561) bank56416 +56417 POINT(40.53560961115573 73.61853422078904) bank56417 +56418 POINT(39.80687766841014 74.06398607670543) bank56418 +56419 POINT(41.52899594870389 73.91641392572873) bank56419 +56420 POINT(41.21248458158022 73.97098473243979) bank56420 +56421 POINT(41.43687802260857 73.33792267936808) bank56421 +56422 POINT(40.11061000377007 74.38029701871201) bank56422 +56423 POINT(40.48511742822888 74.11712237852261) bank56423 +56424 POINT(41.32941072590337 74.72368655546421) bank56424 +56425 POINT(40.059451706297544 74.94309106276158) bank56425 +56426 POINT(41.066275029338215 74.67937249257658) bank56426 +56427 POINT(40.236706535459824 73.47499253856185) bank56427 +56428 POINT(40.33327655149091 73.23825528167754) bank56428 +56429 POINT(40.83374815531728 73.26655573951852) bank56429 +56430 POINT(41.2407478233201 74.16952542173995) bank56430 +56431 POINT(41.19926911410593 74.31971271891967) bank56431 +56432 POINT(41.60683796654995 73.8837052818672) bank56432 +56433 POINT(41.071353794361166 74.53134913322981) bank56433 +56434 POINT(40.290231901198304 73.45377062473212) bank56434 +56435 POINT(41.696307729328716 74.31551841376576) bank56435 +56436 POINT(41.66852877880032 73.46004455676375) bank56436 +56437 POINT(40.722672721181965 73.96392413283934) bank56437 +56438 POINT(41.4406504141387 74.3988937962139) bank56438 +56439 POINT(41.61791042385978 73.21700698929406) bank56439 +56440 POINT(40.27977500405544 73.67135017146286) bank56440 +56441 POINT(40.08569659133871 73.22790253403997) bank56441 +56442 POINT(40.92282948497407 74.58819070957757) bank56442 +56443 POINT(40.91149485529442 73.7737941444518) bank56443 +56444 POINT(40.93772324735838 73.05437433834167) bank56444 +56445 POINT(41.277891591494104 73.12914081301663) bank56445 +56446 POINT(41.677915372989354 73.58709197361694) bank56446 +56447 POINT(41.134457784096774 73.92228538825375) bank56447 +56448 POINT(40.44135752658927 73.78305141948796) bank56448 +56449 POINT(40.42367771917051 73.79682351650709) bank56449 +56450 POINT(39.88439152059002 73.56195586994909) bank56450 +56451 POINT(40.85305228529985 73.66587280937438) bank56451 +56452 POINT(41.03751171887786 73.67129959567067) bank56452 +56453 POINT(41.104871031544256 74.20575371259902) bank56453 +56454 POINT(40.269462497280664 74.51634396609239) bank56454 +56455 POINT(41.624531478994946 74.89708038046277) bank56455 +56456 POINT(40.145869723319485 73.86972669517374) bank56456 +56457 POINT(40.9387911878734 73.40897610985958) bank56457 +56458 POINT(40.093164241814335 74.01594499814453) bank56458 +56459 POINT(41.52947064823175 73.35452725883283) bank56459 +56460 POINT(41.6351291429387 74.02782024893176) bank56460 +56461 POINT(40.35852028489823 73.1020101342827) bank56461 +56462 POINT(40.55472655382918 74.21876113881638) bank56462 +56463 POINT(40.25605062602545 73.38578260514511) bank56463 +56464 POINT(39.74995851662928 74.72247115758255) bank56464 +56465 POINT(40.93060018421522 74.66546187341304) bank56465 +56466 POINT(40.17490712689042 73.23679756220587) bank56466 +56467 POINT(40.415627624633956 73.52689999974126) bank56467 +56468 POINT(41.42062404862864 73.55636969902949) bank56468 +56469 POINT(40.54136186324559 74.7819763094715) bank56469 +56470 POINT(40.83796681815716 74.78515353300364) bank56470 +56471 POINT(40.01871894601974 74.50945740396338) bank56471 +56472 POINT(39.93684778176104 74.76785756219323) bank56472 +56473 POINT(39.81714460502502 73.3403172844455) bank56473 +56474 POINT(41.50867360783242 74.04320064704282) bank56474 +56475 POINT(39.98272997171502 73.34120594169394) bank56475 +56476 POINT(41.05938642224088 73.49744533412871) bank56476 +56477 POINT(39.919760469372264 74.07122719873131) bank56477 +56478 POINT(40.831564612837866 73.68211802138076) bank56478 +56479 POINT(40.440193831069394 73.75198961451656) bank56479 +56480 POINT(40.723535247186796 74.56550491114331) bank56480 +56481 POINT(40.040465212996644 75.00272006075502) bank56481 +56482 POINT(41.66567068111109 74.76731289250472) bank56482 +56483 POINT(39.72881454984403 73.739545217715) bank56483 +56484 POINT(41.43142225728145 73.49031763361342) bank56484 +56485 POINT(40.68465599414552 73.01334899580715) bank56485 +56486 POINT(40.9835620514432 73.62576197097326) bank56486 +56487 POINT(40.46439044965162 74.68824825260069) bank56487 +56488 POINT(41.629223868397624 73.2455908403261) bank56488 +56489 POINT(40.18491219025293 74.02246203783736) bank56489 +56490 POINT(40.530673762304794 73.46419498790067) bank56490 +56491 POINT(39.93986024916809 74.79754087544623) bank56491 +56492 POINT(41.599196206571186 74.37950261456713) bank56492 +56493 POINT(39.836632836958046 74.82356522249486) bank56493 +56494 POINT(40.20048114821504 74.06133837751058) bank56494 +56495 POINT(40.76953280270429 74.51669020700938) bank56495 +56496 POINT(40.89791112052932 74.27397570430949) bank56496 +56497 POINT(41.34283089186221 74.22953590861187) bank56497 +56498 POINT(41.701866741212406 74.86013365249433) bank56498 +56499 POINT(41.70599169827056 74.42163664019654) bank56499 +56500 POINT(40.7810269169149 73.86933985808135) bank56500 +56501 POINT(39.74181076369222 73.45058531294933) bank56501 +56502 POINT(40.45222187588137 73.29798207064314) bank56502 +56503 POINT(40.62183164337416 73.53797983196164) bank56503 +56504 POINT(41.42638980980828 74.99725948087955) bank56504 +56505 POINT(41.120985741233824 74.32073651197554) bank56505 +56506 POINT(41.45025195993874 73.7266163649747) bank56506 +56507 POINT(41.13242120691418 74.09296858961669) bank56507 +56508 POINT(40.63717061872594 73.35402468772529) bank56508 +56509 POINT(41.05805432203125 74.27045775376607) bank56509 +56510 POINT(39.92872265318383 74.54245957226259) bank56510 +56511 POINT(40.09023374161726 74.10961928649306) bank56511 +56512 POINT(39.7621786068929 73.14970356833913) bank56512 +56513 POINT(40.40873447092559 73.37417121604335) bank56513 +56514 POINT(40.41473816086754 74.41101701104547) bank56514 +56515 POINT(40.185395597082355 73.83448805515505) bank56515 +56516 POINT(40.900491171549646 73.97623814869903) bank56516 +56517 POINT(41.067064664963446 73.59673734930838) bank56517 +56518 POINT(40.489667587155225 73.58199224280962) bank56518 +56519 POINT(40.541356833782096 74.3888822399532) bank56519 +56520 POINT(41.512414688586944 73.36906213757425) bank56520 +56521 POINT(41.417836836328746 73.0789679197479) bank56521 +56522 POINT(41.06931236303765 74.37355090182116) bank56522 +56523 POINT(41.48141518463109 73.61557754456425) bank56523 +56524 POINT(39.82118014742337 74.04860411217695) bank56524 +56525 POINT(41.04090625456303 73.92964381707445) bank56525 +56526 POINT(41.01336705478238 73.26805204730249) bank56526 +56527 POINT(40.081924702617805 74.57101606459233) bank56527 +56528 POINT(41.01761457997922 74.57300652896616) bank56528 +56529 POINT(41.115566804380464 74.33826761548626) bank56529 +56530 POINT(39.81868047636873 74.52557888808269) bank56530 +56531 POINT(41.566726961799574 73.53010918561223) bank56531 +56532 POINT(41.06222720906148 74.20798069501897) bank56532 +56533 POINT(40.25538924338152 73.5219356679003) bank56533 +56534 POINT(41.105080328818545 74.93281657185581) bank56534 +56535 POINT(41.53741621112678 74.36426139650496) bank56535 +56536 POINT(40.4169468385954 74.35795290588706) bank56536 +56537 POINT(39.833210755297586 74.19841588456902) bank56537 +56538 POINT(40.3529805294779 73.56711488079445) bank56538 +56539 POINT(41.15120214567004 74.27900743385847) bank56539 +56540 POINT(40.519356040814834 73.32904565423817) bank56540 +56541 POINT(39.80986683553846 74.49332757389813) bank56541 +56542 POINT(41.58306585904967 74.58479709892697) bank56542 +56543 POINT(41.54268312751284 73.96351097547463) bank56543 +56544 POINT(41.04185134747302 74.1716430098234) bank56544 +56545 POINT(41.303265177936524 73.78302074783873) bank56545 +56546 POINT(40.91388806378065 74.42354866451096) bank56546 +56547 POINT(39.88368308541958 73.60367431579587) bank56547 +56548 POINT(41.4298332441584 74.30169471261375) bank56548 +56549 POINT(40.9972143797816 73.55592191280245) bank56549 +56550 POINT(40.23289235469642 74.57222465816837) bank56550 +56551 POINT(41.600968648298135 73.49100354318733) bank56551 +56552 POINT(41.45839184284817 74.30511208417738) bank56552 +56553 POINT(40.53831968450865 73.5695707473305) bank56553 +56554 POINT(41.32975995028129 73.67893255082424) bank56554 +56555 POINT(40.084242335184044 73.84305960841832) bank56555 +56556 POINT(40.62076820839254 74.4540846109825) bank56556 +56557 POINT(40.21351596043557 74.39687904079412) bank56557 +56558 POINT(41.19298352846103 74.64519166639656) bank56558 +56559 POINT(40.76490151686728 73.66396212967787) bank56559 +56560 POINT(39.967027836381064 74.82846734292383) bank56560 +56561 POINT(40.71407436986195 73.6137529924793) bank56561 +56562 POINT(40.79925607271091 74.58689340956062) bank56562 +56563 POINT(40.45324892874574 74.28124254846121) bank56563 +56564 POINT(41.066905829181046 73.04071619238313) bank56564 +56565 POINT(40.64195765920087 73.11973813003807) bank56565 +56566 POINT(40.35421998896212 74.22381368987688) bank56566 +56567 POINT(39.755854153612034 74.35473737679933) bank56567 +56568 POINT(40.481672403053494 74.0075243689953) bank56568 +56569 POINT(40.96003752137319 74.87389673273965) bank56569 +56570 POINT(40.80319596008513 73.40929223017889) bank56570 +56571 POINT(41.119389727788096 74.23405082410082) bank56571 +56572 POINT(40.27448430162607 74.99063495758715) bank56572 +56573 POINT(40.871492901174214 74.51212591469245) bank56573 +56574 POINT(40.17900812789954 74.8514074829177) bank56574 +56575 POINT(39.85364449806887 74.1912069295945) bank56575 +56576 POINT(41.25211092076045 74.89065354891554) bank56576 +56577 POINT(41.24221669150879 73.39761209191572) bank56577 +56578 POINT(41.65341674165007 73.10063879158344) bank56578 +56579 POINT(39.96745697794741 73.68221122514986) bank56579 +56580 POINT(41.35788169053252 74.55425221003352) bank56580 +56581 POINT(41.474126602322094 74.72888369356832) bank56581 +56582 POINT(39.795132851768024 74.83951554197537) bank56582 +56583 POINT(41.201006951691774 74.96480041867504) bank56583 +56584 POINT(40.65448718913435 73.27432020492115) bank56584 +56585 POINT(41.43392973185149 73.87251118435917) bank56585 +56586 POINT(40.75397225240607 73.39155839556453) bank56586 +56587 POINT(40.55804843188359 73.32236326675093) bank56587 +56588 POINT(40.9218622884543 74.38448533455812) bank56588 +56589 POINT(40.16742257228171 73.63857049649646) bank56589 +56590 POINT(39.97564403364557 73.96490676786401) bank56590 +56591 POINT(41.53800054036002 73.06718976409782) bank56591 +56592 POINT(41.494425346414964 74.44411386854067) bank56592 +56593 POINT(41.42240221322538 74.71277797444424) bank56593 +56594 POINT(39.87984311417742 74.08019833451426) bank56594 +56595 POINT(40.54567991422013 75.0045165888187) bank56595 +56596 POINT(40.210094819698085 74.82364001182827) bank56596 +56597 POINT(41.27680033188376 73.81403596025841) bank56597 +56598 POINT(41.08374500279485 73.90865043002368) bank56598 +56599 POINT(41.456292141792844 74.63735500528307) bank56599 +56600 POINT(41.53976963718607 73.2894632740509) bank56600 +56601 POINT(39.85104574309862 74.54861748229283) bank56601 +56602 POINT(39.82692661988374 74.69862999053586) bank56602 +56603 POINT(40.371497869206685 73.71890263566496) bank56603 +56604 POINT(41.19436077926681 74.4337878009426) bank56604 +56605 POINT(40.04940374565511 74.00509493508602) bank56605 +56606 POINT(41.169283883611996 74.8777021099508) bank56606 +56607 POINT(40.414897831772066 74.03283548294519) bank56607 +56608 POINT(41.66305559067041 74.57047182612332) bank56608 +56609 POINT(40.81344967136826 74.99775635693705) bank56609 +56610 POINT(40.88525487430156 73.13697843343718) bank56610 +56611 POINT(40.79065650448706 73.67019666547917) bank56611 +56612 POINT(40.17616586743084 73.98835294988467) bank56612 +56613 POINT(41.65838113972 74.54741490690012) bank56613 +56614 POINT(41.21702759988118 73.010393890005) bank56614 +56615 POINT(40.3411593931068 73.59190118725552) bank56615 +56616 POINT(39.74355499559377 74.12250303684633) bank56616 +56617 POINT(41.305962690137974 74.28537985639976) bank56617 +56618 POINT(41.58616725754123 73.27852261900432) bank56618 +56619 POINT(40.30545259939514 73.51832863082453) bank56619 +56620 POINT(40.77509934122305 74.1508682331488) bank56620 +56621 POINT(41.39657857285896 74.43497007379791) bank56621 +56622 POINT(41.08853158813396 74.36192940651434) bank56622 +56623 POINT(39.71901146326457 74.36683662735565) bank56623 +56624 POINT(41.253049089065804 74.75145268686158) bank56624 +56625 POINT(39.84432452361626 73.72079537617132) bank56625 +56626 POINT(40.45597554492819 73.94720139165678) bank56626 +56627 POINT(39.87155903171165 74.77040592156021) bank56627 +56628 POINT(41.686169618984785 73.77649366298186) bank56628 +56629 POINT(40.84370686302238 74.37016776225117) bank56629 +56630 POINT(40.65071197506474 73.20147350196723) bank56630 +56631 POINT(40.130880741652135 74.47952174546944) bank56631 +56632 POINT(40.30936856844127 74.24724383530467) bank56632 +56633 POINT(41.203468090671635 74.56191361219739) bank56633 +56634 POINT(40.95336539367914 73.16285340408902) bank56634 +56635 POINT(40.8280429209749 73.06894400102296) bank56635 +56636 POINT(40.47079934203681 73.43490753019327) bank56636 +56637 POINT(41.61479944453243 73.44483318346957) bank56637 +56638 POINT(41.13848592547184 74.72945748261274) bank56638 +56639 POINT(41.32849990559634 74.4754904006625) bank56639 +56640 POINT(40.379692778593956 73.6432189672628) bank56640 +56641 POINT(40.672042379743395 74.67788496019395) bank56641 +56642 POINT(40.399837722191975 74.93869240147275) bank56642 +56643 POINT(41.56225583686249 74.24631690270525) bank56643 +56644 POINT(40.82792413961754 73.29619825157158) bank56644 +56645 POINT(40.540032927039874 73.11480450496869) bank56645 +56646 POINT(41.18496758638845 74.69162663020026) bank56646 +56647 POINT(40.166306081793316 74.72546304780947) bank56647 +56648 POINT(39.91511542066615 73.76315023265585) bank56648 +56649 POINT(39.86574690255271 73.20630036874022) bank56649 +56650 POINT(41.052198689461605 74.31422250863729) bank56650 +56651 POINT(39.7313753043268 74.72888993303563) bank56651 +56652 POINT(40.029806721976506 73.02218623784862) bank56652 +56653 POINT(40.41646388178515 73.66403114896355) bank56653 +56654 POINT(40.08517394943463 73.61715153110917) bank56654 +56655 POINT(40.87692693394538 73.92286945399485) bank56655 +56656 POINT(39.932586196693755 74.74225512353942) bank56656 +56657 POINT(39.92215108291587 74.22507420985212) bank56657 +56658 POINT(41.03577153400727 74.21311558992994) bank56658 +56659 POINT(41.00558770768981 74.20246565964989) bank56659 +56660 POINT(40.88895787451823 74.40226686403882) bank56660 +56661 POINT(40.448787897216526 74.78897028080296) bank56661 +56662 POINT(39.90609256930509 74.70507159413577) bank56662 +56663 POINT(40.877678920145684 74.31656377366212) bank56663 +56664 POINT(41.62466620096092 74.6484294269633) bank56664 +56665 POINT(40.19895480075244 73.94542920462595) bank56665 +56666 POINT(39.96784729308924 74.10498081952325) bank56666 +56667 POINT(40.5037892676554 74.14503526352536) bank56667 +56668 POINT(40.403397406042075 74.21789551670089) bank56668 +56669 POINT(41.16033216738217 74.90778695630438) bank56669 +56670 POINT(41.304029964242034 73.80756876076886) bank56670 +56671 POINT(41.002754942252814 74.08122096065019) bank56671 +56672 POINT(41.43189629459187 74.65840365941766) bank56672 +56673 POINT(41.40161139376144 73.28069577448562) bank56673 +56674 POINT(40.034196990608685 74.5165683425203) bank56674 +56675 POINT(39.979122948620265 74.9516028043281) bank56675 +56676 POINT(41.33567956648393 73.23794514640258) bank56676 +56677 POINT(40.85486566063291 73.54223189200962) bank56677 +56678 POINT(41.41729118312972 74.97748339033683) bank56678 +56679 POINT(40.918287371597295 73.16700392029519) bank56679 +56680 POINT(40.79844557470431 74.99158430336759) bank56680 +56681 POINT(40.54615813049691 73.49687297044352) bank56681 +56682 POINT(39.85719538660387 74.81418400748892) bank56682 +56683 POINT(40.674951436736926 73.96422473990634) bank56683 +56684 POINT(40.22078276135479 73.3319018508917) bank56684 +56685 POINT(40.09481948278369 73.53689892295978) bank56685 +56686 POINT(41.54551251038997 73.69856886538652) bank56686 +56687 POINT(40.73042728335039 73.20010182815876) bank56687 +56688 POINT(40.06938910535308 73.68545390371318) bank56688 +56689 POINT(41.34063512245228 73.5495412816513) bank56689 +56690 POINT(40.44678544626396 74.7453858182242) bank56690 +56691 POINT(39.73385816431078 74.1780472263015) bank56691 +56692 POINT(40.99024166125776 74.01017342242857) bank56692 +56693 POINT(41.16692025402553 73.80933101637059) bank56693 +56694 POINT(40.43709994998275 74.5871635618997) bank56694 +56695 POINT(41.691301702218055 74.01969880097317) bank56695 +56696 POINT(40.00390683327941 74.41637666615154) bank56696 +56697 POINT(41.0789669059568 73.93019283611302) bank56697 +56698 POINT(40.586997712884276 74.42600531404575) bank56698 +56699 POINT(40.43159350498347 73.6316074394276) bank56699 +56700 POINT(41.176608520449044 74.63138944264716) bank56700 +56701 POINT(40.57936683604665 74.76297508591466) bank56701 +56702 POINT(41.13399211574327 74.15055600009097) bank56702 +56703 POINT(41.02967884619328 74.21494887291327) bank56703 +56704 POINT(40.30111935053624 74.13904498150369) bank56704 +56705 POINT(40.36659768395262 74.63616396966279) bank56705 +56706 POINT(40.42481531651572 73.40138181158135) bank56706 +56707 POINT(41.084768424111886 73.26734064404693) bank56707 +56708 POINT(40.0241546191047 74.13502799986242) bank56708 +56709 POINT(39.80572293559856 74.82470981765158) bank56709 +56710 POINT(41.55226507890796 74.54050806991101) bank56710 +56711 POINT(40.068946620019126 74.58086014905064) bank56711 +56712 POINT(40.617489100847976 74.68918214429456) bank56712 +56713 POINT(40.77199015917427 73.5573252964366) bank56713 +56714 POINT(39.92944667773534 74.66982949574867) bank56714 +56715 POINT(41.43590579762035 74.68322513571746) bank56715 +56716 POINT(40.54660708798259 74.24766978467163) bank56716 +56717 POINT(40.506959592057335 73.35081322903288) bank56717 +56718 POINT(40.32859095407919 73.8691331314262) bank56718 +56719 POINT(40.030004759729536 73.49993860223064) bank56719 +56720 POINT(40.021571069984645 73.87250665789958) bank56720 +56721 POINT(40.784331568835306 74.32908818251083) bank56721 +56722 POINT(40.85402711360713 73.07636298408767) bank56722 +56723 POINT(41.07861417101139 73.44239888727415) bank56723 +56724 POINT(41.239525644749456 74.40281650449312) bank56724 +56725 POINT(41.60053353020904 74.48885330265877) bank56725 +56726 POINT(40.090048440162406 73.21658074306734) bank56726 +56727 POINT(40.11055500460037 74.0915615152856) bank56727 +56728 POINT(40.04419894751936 73.46111459201472) bank56728 +56729 POINT(39.81728397840965 74.43102707332643) bank56729 +56730 POINT(39.99767331286328 73.22947872823694) bank56730 +56731 POINT(40.76834780949352 74.93399383144208) bank56731 +56732 POINT(40.233817302828804 74.51195865152833) bank56732 +56733 POINT(41.15477920394715 73.71679344451357) bank56733 +56734 POINT(39.744808430866065 74.803691509282) bank56734 +56735 POINT(40.101884909537155 74.03872820604298) bank56735 +56736 POINT(41.40462916902473 74.28884122297299) bank56736 +56737 POINT(41.43604592447598 73.05826569849758) bank56737 +56738 POINT(39.85693554619297 73.24250625066804) bank56738 +56739 POINT(41.40262461124582 73.71819676551723) bank56739 +56740 POINT(40.2191397906621 73.66028703503584) bank56740 +56741 POINT(39.833789326751315 74.43898508277583) bank56741 +56742 POINT(40.2389631924199 74.07968279202935) bank56742 +56743 POINT(41.678832456050515 74.12388060839321) bank56743 +56744 POINT(41.2422731492108 73.70231026259759) bank56744 +56745 POINT(40.59465487567965 74.45995389334955) bank56745 +56746 POINT(41.07592084165078 74.37794314817533) bank56746 +56747 POINT(41.06730524715065 73.78238992070578) bank56747 +56748 POINT(41.38806002176871 73.07377511399578) bank56748 +56749 POINT(39.78275419767385 74.01233853522578) bank56749 +56750 POINT(40.95325929739236 74.24330074182994) bank56750 +56751 POINT(39.949393105508854 73.40436647913576) bank56751 +56752 POINT(41.173268071287865 74.72074786938956) bank56752 +56753 POINT(41.059124432409725 74.07760995631632) bank56753 +56754 POINT(41.692363917836715 73.88149052913762) bank56754 +56755 POINT(41.07418419096961 73.62163995401396) bank56755 +56756 POINT(41.23055824233959 74.53665610025722) bank56756 +56757 POINT(40.65056176047068 74.59688261752514) bank56757 +56758 POINT(39.96673425784774 74.77757848541216) bank56758 +56759 POINT(41.09982602219607 73.20628820386841) bank56759 +56760 POINT(41.54993848144148 74.93817685751304) bank56760 +56761 POINT(40.68185617319332 73.77933198932858) bank56761 +56762 POINT(39.931312715812936 74.42871795023497) bank56762 +56763 POINT(41.15921101856918 74.95843549746256) bank56763 +56764 POINT(40.83588908245773 73.9748708191004) bank56764 +56765 POINT(40.42868034146113 74.57614923256128) bank56765 +56766 POINT(41.01798176790277 74.82672981439495) bank56766 +56767 POINT(40.338118590069996 73.9900735581859) bank56767 +56768 POINT(40.25369612397705 73.92315087100377) bank56768 +56769 POINT(40.41852739222823 74.20313280905091) bank56769 +56770 POINT(39.82690887486156 73.73898491984039) bank56770 +56771 POINT(39.790498025188434 74.25773331869061) bank56771 +56772 POINT(41.27530980170535 73.97592556285205) bank56772 +56773 POINT(40.111816760745896 73.80154389957208) bank56773 +56774 POINT(39.80657343104243 74.23607771853595) bank56774 +56775 POINT(40.25298571795751 74.28924186730401) bank56775 +56776 POINT(41.3877031089073 74.40693616686899) bank56776 +56777 POINT(40.7226887085236 73.4260354124404) bank56777 +56778 POINT(41.50851602369012 74.23696103167065) bank56778 +56779 POINT(41.19029121692769 74.52263161752332) bank56779 +56780 POINT(40.51217392580683 74.01800925958004) bank56780 +56781 POINT(39.96741018548322 73.15793732404715) bank56781 +56782 POINT(40.74667882849072 74.92907016152806) bank56782 +56783 POINT(40.93023973025269 73.33062032099588) bank56783 +56784 POINT(41.42745485946014 74.2152275239078) bank56784 +56785 POINT(41.37624336451192 73.3553480658154) bank56785 +56786 POINT(41.53006107812552 74.9686275790273) bank56786 +56787 POINT(40.104270278716434 73.38504756482777) bank56787 +56788 POINT(40.71071055420798 73.57850602097812) bank56788 +56789 POINT(41.50759253585403 74.75553388923336) bank56789 +56790 POINT(41.381999808321645 74.39484380040396) bank56790 +56791 POINT(39.87900006103778 74.522409719133) bank56791 +56792 POINT(41.195197805519825 74.37080778566737) bank56792 +56793 POINT(39.81323199404791 73.90123861967281) bank56793 +56794 POINT(39.86453987013338 74.17585264296493) bank56794 +56795 POINT(40.42228939944567 74.06750189397428) bank56795 +56796 POINT(40.29281684961898 73.53773535796651) bank56796 +56797 POINT(39.7854297326283 74.94555229400731) bank56797 +56798 POINT(40.56412557760751 74.81868711056154) bank56798 +56799 POINT(40.45238613627985 73.95245475287827) bank56799 +56800 POINT(41.467220561067286 74.63030794300786) bank56800 +56801 POINT(40.094659037881726 74.28855626051799) bank56801 +56802 POINT(39.93014134615338 73.6203300186608) bank56802 +56803 POINT(41.523405525028615 74.1631595261562) bank56803 +56804 POINT(41.33290393814502 73.53093005400736) bank56804 +56805 POINT(41.530353024749694 74.72209345339111) bank56805 +56806 POINT(40.484368293118365 74.50523101359009) bank56806 +56807 POINT(39.9522819835714 73.37681885562958) bank56807 +56808 POINT(41.10570956806743 73.64869412349186) bank56808 +56809 POINT(40.75308689389334 74.88939753056326) bank56809 +56810 POINT(41.07716355091888 73.71622928914232) bank56810 +56811 POINT(40.802741879726995 73.95566498207457) bank56811 +56812 POINT(40.830294019427264 73.39649911067382) bank56812 +56813 POINT(41.449770598805806 73.25911778138017) bank56813 +56814 POINT(40.68851827644195 74.83090154966666) bank56814 +56815 POINT(41.36170737553527 74.54222231252304) bank56815 +56816 POINT(40.63653886155271 74.69163692863081) bank56816 +56817 POINT(41.529405998274314 73.89952206618689) bank56817 +56818 POINT(40.041639691684665 73.21988184953689) bank56818 +56819 POINT(40.97768583913603 74.3532170132994) bank56819 +56820 POINT(40.054148969797296 73.3135344147187) bank56820 +56821 POINT(40.730815528055196 73.28514207304029) bank56821 +56822 POINT(40.714161296642736 74.05312525119228) bank56822 +56823 POINT(40.094371242941925 73.26094856260839) bank56823 +56824 POINT(41.43794776972989 73.60123591411103) bank56824 +56825 POINT(41.255512789859786 73.7244232921798) bank56825 +56826 POINT(40.5522341907492 73.47843243433103) bank56826 +56827 POINT(40.90668899086208 73.77005570669816) bank56827 +56828 POINT(40.68739439434437 74.35334604614411) bank56828 +56829 POINT(41.70099884092211 73.09999297102466) bank56829 +56830 POINT(41.09918571967253 74.61361025198453) bank56830 +56831 POINT(40.55124874068849 73.67182060239143) bank56831 +56832 POINT(40.42900542888895 73.69536379130558) bank56832 +56833 POINT(39.982574975231365 74.40626505783331) bank56833 +56834 POINT(40.886801907508776 73.84150787194989) bank56834 +56835 POINT(41.66080892498717 73.68340841863947) bank56835 +56836 POINT(41.1914175710723 74.64820169254084) bank56836 +56837 POINT(39.76679419018167 74.9868156766601) bank56837 +56838 POINT(40.93262930832911 73.79886146058016) bank56838 +56839 POINT(41.45224526669017 74.78784536461777) bank56839 +56840 POINT(40.14773576523506 74.72417487687297) bank56840 +56841 POINT(39.994850297168114 74.91614647019043) bank56841 +56842 POINT(40.008594305585206 73.73182098231672) bank56842 +56843 POINT(41.20286979466193 73.42022152187161) bank56843 +56844 POINT(40.521593498104274 74.4515830082636) bank56844 +56845 POINT(39.837522875719934 73.48245413545524) bank56845 +56846 POINT(40.77748170177875 74.82945814891814) bank56846 +56847 POINT(40.100331681287514 73.78516759173645) bank56847 +56848 POINT(40.256930421790365 73.5170697947852) bank56848 +56849 POINT(40.764420183505365 74.38035468470524) bank56849 +56850 POINT(41.09240980168675 73.31057422406926) bank56850 +56851 POINT(40.11745063787252 74.14087915395419) bank56851 +56852 POINT(40.96578563949547 73.38837619187024) bank56852 +56853 POINT(41.14197733899872 73.6195110499665) bank56853 +56854 POINT(41.278558915819836 73.14560057282046) bank56854 +56855 POINT(41.22804282880442 73.64013296621086) bank56855 +56856 POINT(40.301727435265974 73.03904580320767) bank56856 +56857 POINT(39.84456997092779 75.00558156287997) bank56857 +56858 POINT(40.32495024913094 73.42500697971971) bank56858 +56859 POINT(41.44893823140565 73.84443656644102) bank56859 +56860 POINT(40.181708095772315 73.79945360380927) bank56860 +56861 POINT(40.15102629355449 74.92028337464673) bank56861 +56862 POINT(40.734825382275496 73.81147243512481) bank56862 +56863 POINT(41.39699073965401 74.83940382585207) bank56863 +56864 POINT(40.9861025533452 74.08490146794318) bank56864 +56865 POINT(40.17988598606366 73.1652125983658) bank56865 +56866 POINT(41.066968432629665 74.09700482888994) bank56866 +56867 POINT(40.27367089783262 73.31664246485346) bank56867 +56868 POINT(41.4566669847695 73.46526660090252) bank56868 +56869 POINT(40.99805970194075 73.68653074493294) bank56869 +56870 POINT(40.527220335518926 74.99957295313436) bank56870 +56871 POINT(41.278315687150645 74.66697160647139) bank56871 +56872 POINT(40.705408655373105 74.9714758028975) bank56872 +56873 POINT(40.080291607624424 73.82559856718781) bank56873 +56874 POINT(40.02307310730344 74.10795606531359) bank56874 +56875 POINT(39.992549781800754 73.05458788176804) bank56875 +56876 POINT(41.38032569221129 74.0401881550757) bank56876 +56877 POINT(41.04697068449598 74.28330209719229) bank56877 +56878 POINT(41.320261397426826 74.59001411520862) bank56878 +56879 POINT(41.285354960243474 74.67036747844469) bank56879 +56880 POINT(40.58875849233154 73.43585998305883) bank56880 +56881 POINT(39.858019843244016 73.97974136122514) bank56881 +56882 POINT(41.24079241878989 73.01640245365118) bank56882 +56883 POINT(40.295776895882526 74.86594548494304) bank56883 +56884 POINT(40.01808553278018 74.4961042106932) bank56884 +56885 POINT(40.21656403311642 74.7207029418628) bank56885 +56886 POINT(41.11711680432459 74.36763547022132) bank56886 +56887 POINT(40.705303543772054 74.39355589590583) bank56887 +56888 POINT(40.67099366155592 73.51037092893361) bank56888 +56889 POINT(40.72343104592554 73.38578445250498) bank56889 +56890 POINT(41.47534240807265 74.65216800409308) bank56890 +56891 POINT(40.97374893199922 73.79072582644262) bank56891 +56892 POINT(40.478838706626256 74.64241052297307) bank56892 +56893 POINT(41.5572744777743 74.5705961814507) bank56893 +56894 POINT(39.924959350610344 74.50207332147129) bank56894 +56895 POINT(41.04243400451065 74.02677387457581) bank56895 +56896 POINT(40.892378109940445 73.64976415096945) bank56896 +56897 POINT(40.33072297890667 74.29241083504556) bank56897 +56898 POINT(41.36455809678034 73.50336805114247) bank56898 +56899 POINT(40.060731941690904 73.86046539340471) bank56899 +56900 POINT(40.365090420160826 73.21282508626119) bank56900 +56901 POINT(41.02264092318336 73.86737808417696) bank56901 +56902 POINT(41.49411322753021 74.33282345400566) bank56902 +56903 POINT(40.84203995348062 74.84201611710698) bank56903 +56904 POINT(41.501648123261134 73.31478543915335) bank56904 +56905 POINT(39.780370805546255 74.54917493267594) bank56905 +56906 POINT(41.12427080046031 74.43814042403973) bank56906 +56907 POINT(41.53093413708651 73.76080347718528) bank56907 +56908 POINT(41.400985108886836 74.3278070171379) bank56908 +56909 POINT(40.86124179955993 73.15594530931097) bank56909 +56910 POINT(41.60998667324558 74.89908041808992) bank56910 +56911 POINT(41.38072981675951 74.68232314830202) bank56911 +56912 POINT(40.57207226339619 73.05636189466682) bank56912 +56913 POINT(40.48213194909315 74.11345765234117) bank56913 +56914 POINT(40.24064633516797 73.27004871642663) bank56914 +56915 POINT(39.71813061600637 74.10287165920278) bank56915 +56916 POINT(39.86643805315871 73.97311392564004) bank56916 +56917 POINT(41.13993417060788 73.01264314266139) bank56917 +56918 POINT(40.643802068612516 74.87936134735804) bank56918 +56919 POINT(40.21428059686189 73.79972646396621) bank56919 +56920 POINT(40.490993981935844 74.22777012269682) bank56920 +56921 POINT(41.5286992110896 74.32220899682346) bank56921 +56922 POINT(41.53299324192786 73.01114616164799) bank56922 +56923 POINT(39.92010878997736 73.47657873049913) bank56923 +56924 POINT(41.09112050704954 74.80885253279673) bank56924 +56925 POINT(39.95900061700044 73.4887002881701) bank56925 +56926 POINT(40.90036317674606 74.59972174346116) bank56926 +56927 POINT(41.217041871780594 73.76334616150932) bank56927 +56928 POINT(40.95787781307212 73.34449933400433) bank56928 +56929 POINT(40.80662161971045 74.1369654382844) bank56929 +56930 POINT(39.92290895250787 74.60799345663044) bank56930 +56931 POINT(41.698257888459274 73.24896496601652) bank56931 +56932 POINT(41.29973746425164 73.56169916660019) bank56932 +56933 POINT(40.16039119524632 74.29576781534037) bank56933 +56934 POINT(41.36338802529994 73.90182947616648) bank56934 +56935 POINT(41.62817292396913 74.07486954504358) bank56935 +56936 POINT(40.88125275777812 73.20149998635384) bank56936 +56937 POINT(41.55427228196431 74.57228165573923) bank56937 +56938 POINT(41.70436532821874 74.12426335815658) bank56938 +56939 POINT(40.67280196017703 74.85988808521104) bank56939 +56940 POINT(41.44749444396895 73.34860686545736) bank56940 +56941 POINT(41.06181145220551 74.50539451203761) bank56941 +56942 POINT(39.90219653411656 74.43454855800175) bank56942 +56943 POINT(41.2539482615423 73.24151128752932) bank56943 +56944 POINT(40.64130670574954 74.2627211968169) bank56944 +56945 POINT(40.13661556699306 74.25350602640985) bank56945 +56946 POINT(40.766772230226444 73.2079791275491) bank56946 +56947 POINT(39.857831920740686 73.75654932290685) bank56947 +56948 POINT(41.06756868958789 74.66401686998368) bank56948 +56949 POINT(41.5485318197957 74.6103995494005) bank56949 +56950 POINT(40.559719033061235 73.7902423985738) bank56950 +56951 POINT(39.85772643452464 74.7378854148332) bank56951 +56952 POINT(39.774315606377364 73.39394326260663) bank56952 +56953 POINT(40.60328473541111 73.67731255666514) bank56953 +56954 POINT(40.178408389631294 73.60627510890406) bank56954 +56955 POINT(40.7711063152619 74.48851278750773) bank56955 +56956 POINT(40.62107813128478 73.01964433112457) bank56956 +56957 POINT(40.70693182566029 74.68942945906655) bank56957 +56958 POINT(41.14046196108217 73.54669202508612) bank56958 +56959 POINT(41.618335462186124 74.79301654523965) bank56959 +56960 POINT(40.60936722872088 74.05576329026893) bank56960 +56961 POINT(41.22424203647086 74.72210021297039) bank56961 +56962 POINT(41.11303487774201 73.04848726874879) bank56962 +56963 POINT(40.52576702603376 74.93899516851171) bank56963 +56964 POINT(40.2835570168834 73.22068186605404) bank56964 +56965 POINT(41.23040848376465 73.98566116307728) bank56965 +56966 POINT(41.60313724432086 73.4124314259874) bank56966 +56967 POINT(41.194958106925576 73.92069094835391) bank56967 +56968 POINT(39.798122245819336 73.11278925715612) bank56968 +56969 POINT(41.12343298238285 74.40752808989393) bank56969 +56970 POINT(40.179603209606825 73.93004962085392) bank56970 +56971 POINT(39.77064147489485 73.93112917281641) bank56971 +56972 POINT(41.24170837651234 73.2155067852134) bank56972 +56973 POINT(40.53425607233633 74.51758536250745) bank56973 +56974 POINT(41.668834340093255 74.9312171858913) bank56974 +56975 POINT(39.879623295353504 73.02461877795615) bank56975 +56976 POINT(40.835032910302175 74.56755771529973) bank56976 +56977 POINT(40.153032316099996 73.53183914302872) bank56977 +56978 POINT(39.90876156404362 74.20117598321043) bank56978 +56979 POINT(40.77435641869128 74.97057429658715) bank56979 +56980 POINT(40.151496882247315 74.61354396055438) bank56980 +56981 POINT(41.49768563400081 73.14489947770754) bank56981 +56982 POINT(40.16089955627869 73.96935804019039) bank56982 +56983 POINT(40.67990998416911 74.31189351915918) bank56983 +56984 POINT(40.846223039856184 73.52215447502324) bank56984 +56985 POINT(40.750326891894304 74.17397918156769) bank56985 +56986 POINT(40.16592044843061 74.39086147379751) bank56986 +56987 POINT(40.86072000587585 73.19654997935311) bank56987 +56988 POINT(41.71029548182623 74.54155268381706) bank56988 +56989 POINT(41.68269486330517 74.79773366462592) bank56989 +56990 POINT(40.971456241548985 73.72980594914513) bank56990 +56991 POINT(40.99196300005223 73.65637381380321) bank56991 +56992 POINT(41.233668539858755 74.22555502935515) bank56992 +56993 POINT(41.009614115083814 74.72898966149108) bank56993 +56994 POINT(39.75256756108273 73.79259479576662) bank56994 +56995 POINT(41.65189113653828 74.48573377563278) bank56995 +56996 POINT(40.38010808100967 73.19353797652752) bank56996 +56997 POINT(39.97489284919342 74.58761101952317) bank56997 +56998 POINT(40.05359893488045 74.38845725741636) bank56998 +56999 POINT(41.01785101680484 74.42840481095651) bank56999 +57000 POINT(41.234080980951134 73.20356961513308) bank57000 +57001 POINT(39.74485520348647 73.23576025367355) bank57001 +57002 POINT(41.158685707389544 74.19049472701951) bank57002 +57003 POINT(40.634619596569145 74.46071126614736) bank57003 +57004 POINT(41.482158572724835 73.55171895442503) bank57004 +57005 POINT(40.98949253027983 74.69702528703449) bank57005 +57006 POINT(41.437440178255905 73.29690524537185) bank57006 +57007 POINT(41.40530066177687 73.37686433335337) bank57007 +57008 POINT(40.99988214898369 74.01641762397205) bank57008 +57009 POINT(41.59417146363557 73.96575355598024) bank57009 +57010 POINT(39.713295250066345 74.49402072417705) bank57010 +57011 POINT(41.221915079591604 73.44950204772881) bank57011 +57012 POINT(40.03161598346698 74.60040736971152) bank57012 +57013 POINT(39.8990098246358 73.50742629207889) bank57013 +57014 POINT(40.55049550127804 74.9847604707511) bank57014 +57015 POINT(40.94341137411486 73.8604274528487) bank57015 +57016 POINT(39.74061873191178 73.10845206451904) bank57016 +57017 POINT(40.23493985171633 73.12442544317727) bank57017 +57018 POINT(40.32723923093808 74.82389906973674) bank57018 +57019 POINT(41.39141124568676 73.2784354460699) bank57019 +57020 POINT(41.189579209107166 73.09079669498355) bank57020 +57021 POINT(40.17657493614925 73.75422043627141) bank57021 +57022 POINT(40.265714946088806 73.74143571723585) bank57022 +57023 POINT(41.61804352747471 74.86685388858858) bank57023 +57024 POINT(40.59251431837656 74.80268541885795) bank57024 +57025 POINT(40.451386904802916 73.17432357284167) bank57025 +57026 POINT(41.397079573692594 74.37826545416732) bank57026 +57027 POINT(39.8462392660379 74.88821058570207) bank57027 +57028 POINT(40.526217983646774 73.04287766809851) bank57028 +57029 POINT(40.14194019754057 74.11632438777582) bank57029 +57030 POINT(39.760833171727384 73.37132534138804) bank57030 +57031 POINT(40.929513728482114 73.69229277523678) bank57031 +57032 POINT(40.31218227073124 74.44205217627683) bank57032 +57033 POINT(40.87046203833969 74.39332429164563) bank57033 +57034 POINT(41.43079624130508 73.82900458597604) bank57034 +57035 POINT(41.15431250647635 74.74292105950423) bank57035 +57036 POINT(39.88635663403818 74.46173213595534) bank57036 +57037 POINT(41.3768047058519 73.67264940767942) bank57037 +57038 POINT(41.259828219712254 73.7440409647878) bank57038 +57039 POINT(40.933627829962475 73.67536473143234) bank57039 +57040 POINT(40.77129770903253 73.7757971360927) bank57040 +57041 POINT(41.42228325230875 73.08740281903131) bank57041 +57042 POINT(40.82300656560464 74.01934520882094) bank57042 +57043 POINT(39.76798809694848 74.23988137857651) bank57043 +57044 POINT(41.27665345455522 74.939807448335) bank57044 +57045 POINT(40.66080480941392 74.82746857688062) bank57045 +57046 POINT(40.92713926574619 74.8689055549291) bank57046 +57047 POINT(40.27953421061092 74.56087871249997) bank57047 +57048 POINT(39.77735475462409 74.60838337177974) bank57048 +57049 POINT(40.36321068085705 74.79604184494137) bank57049 +57050 POINT(41.221182985274226 74.42203337832984) bank57050 +57051 POINT(40.19613137194987 73.0593314427849) bank57051 +57052 POINT(40.25661134567912 73.74254064895197) bank57052 +57053 POINT(41.10096568075569 73.74729723369819) bank57053 +57054 POINT(41.20375690313011 74.05418171330851) bank57054 +57055 POINT(40.53051716996783 73.96594733256319) bank57055 +57056 POINT(40.38753420328471 73.45274004334945) bank57056 +57057 POINT(39.832700295257595 73.49190034853507) bank57057 +57058 POINT(40.42839636224428 73.98714934514028) bank57058 +57059 POINT(40.4710507858621 74.59243183758537) bank57059 +57060 POINT(40.39194824169166 74.13248035663682) bank57060 +57061 POINT(40.44658474460452 74.28394845330232) bank57061 +57062 POINT(41.18375811620439 73.79231039785222) bank57062 +57063 POINT(39.732238717266696 73.53959805793676) bank57063 +57064 POINT(41.43723422328794 74.79068657205377) bank57064 +57065 POINT(40.02945521892931 73.94831322304859) bank57065 +57066 POINT(41.44236682956607 73.70975192906373) bank57066 +57067 POINT(41.49119480358006 74.65775649535094) bank57067 +57068 POINT(39.90418850702848 73.91455520600287) bank57068 +57069 POINT(40.20020654006347 74.14325829808644) bank57069 +57070 POINT(40.77145186061461 74.03886191938015) bank57070 +57071 POINT(41.299021450918964 73.15761495686063) bank57071 +57072 POINT(40.919693023780226 73.73874926200966) bank57072 +57073 POINT(40.86657317710774 73.22371639714943) bank57073 +57074 POINT(40.30249666384121 74.09736309745863) bank57074 +57075 POINT(40.66285977326132 73.91584678815664) bank57075 +57076 POINT(40.154888307718934 73.16489435405909) bank57076 +57077 POINT(40.328741395347556 73.9628743467684) bank57077 +57078 POINT(40.15065443986221 73.96176591933589) bank57078 +57079 POINT(39.99233454735129 74.8674516319095) bank57079 +57080 POINT(40.17015827829582 74.15864340781715) bank57080 +57081 POINT(39.991781944069245 74.39624208697978) bank57081 +57082 POINT(39.7468003308474 73.08202530404925) bank57082 +57083 POINT(39.821452357064594 73.43176989539066) bank57083 +57084 POINT(40.941479477388974 73.74658204940144) bank57084 +57085 POINT(40.90256669017444 73.57250043851596) bank57085 +57086 POINT(40.471087216073364 73.98377007330627) bank57086 +57087 POINT(41.16161895934299 74.53076800324708) bank57087 +57088 POINT(40.13648457400206 74.81555753284238) bank57088 +57089 POINT(41.06244086241744 74.25375211709557) bank57089 +57090 POINT(40.41745489363603 73.6891921965485) bank57090 +57091 POINT(40.62416970678845 73.63017460210604) bank57091 +57092 POINT(40.591753464209816 73.07386094193153) bank57092 +57093 POINT(40.20281766403661 74.5624002187729) bank57093 +57094 POINT(40.839076298153266 73.45274969354415) bank57094 +57095 POINT(41.446442971150304 74.58365295586823) bank57095 +57096 POINT(40.753620373148095 73.82653855532863) bank57096 +57097 POINT(40.560735161720714 73.37111124384036) bank57097 +57098 POINT(41.490262483380555 73.49363840789782) bank57098 +57099 POINT(41.1433306816939 73.20755227456401) bank57099 +57100 POINT(41.38671523422573 73.15709547917517) bank57100 +57101 POINT(39.752826284895875 73.52728137539316) bank57101 +57102 POINT(41.25354915776035 74.67488653287467) bank57102 +57103 POINT(41.4840828122241 73.40461401206467) bank57103 +57104 POINT(40.726817292504904 74.82674704108055) bank57104 +57105 POINT(41.615159338840634 73.94729613745675) bank57105 +57106 POINT(41.45096083603112 73.37353287722387) bank57106 +57107 POINT(40.344247085717264 73.86680048863246) bank57107 +57108 POINT(41.67821374651863 74.23837031961425) bank57108 +57109 POINT(41.32187406295271 74.31956612203211) bank57109 +57110 POINT(41.68022561899018 73.63611642922938) bank57110 +57111 POINT(40.645273035458345 74.16795987378028) bank57111 +57112 POINT(41.30365875621304 74.26177799426442) bank57112 +57113 POINT(40.77446670027401 73.90104393109753) bank57113 +57114 POINT(41.50822029682384 73.91659971390611) bank57114 +57115 POINT(41.443317147712996 73.04315705432852) bank57115 +57116 POINT(40.26715193341896 73.4340499875611) bank57116 +57117 POINT(41.133136233814334 74.77659157402067) bank57117 +57118 POINT(40.351606979487954 73.33113790471297) bank57118 +57119 POINT(40.33395580903187 73.04129469584453) bank57119 +57120 POINT(40.87557740943236 73.54004895230115) bank57120 +57121 POINT(41.111743514270806 74.3747848781764) bank57121 +57122 POINT(41.65273032558421 73.01756339604515) bank57122 +57123 POINT(40.04238372381429 73.38024116312411) bank57123 +57124 POINT(39.975255773097246 73.17109432414674) bank57124 +57125 POINT(40.69884969612772 73.7447766358236) bank57125 +57126 POINT(41.6642118785911 73.82477589087767) bank57126 +57127 POINT(40.8083117244833 73.77906644909149) bank57127 +57128 POINT(41.38746569413401 73.26806153479004) bank57128 +57129 POINT(41.52410968389628 74.68057572224998) bank57129 +57130 POINT(40.900820135750884 73.43894897265716) bank57130 +57131 POINT(39.929219610953304 74.35910474259771) bank57131 +57132 POINT(39.87637554139051 74.69658813941966) bank57132 +57133 POINT(41.34596126164247 73.97970201294741) bank57133 +57134 POINT(39.944177865571405 74.08548588288387) bank57134 +57135 POINT(39.96401944065871 74.58769346962528) bank57135 +57136 POINT(40.7601820160982 74.01959284999897) bank57136 +57137 POINT(40.92174317986273 74.95779656186755) bank57137 +57138 POINT(41.19908935143852 74.16979214536026) bank57138 +57139 POINT(40.91310121936575 74.47082923717255) bank57139 +57140 POINT(40.78376928891919 74.33953307614793) bank57140 +57141 POINT(40.14372481898005 73.34312110004537) bank57141 +57142 POINT(39.722488816464285 74.93058965653822) bank57142 +57143 POINT(39.73469755957671 73.3671631210851) bank57143 +57144 POINT(39.78223986372085 73.60170335432187) bank57144 +57145 POINT(40.13472332017764 74.66534254670528) bank57145 +57146 POINT(41.62925712285145 73.11753205484239) bank57146 +57147 POINT(41.02763704417415 73.48830567720978) bank57147 +57148 POINT(39.75969985413144 73.31350315655095) bank57148 +57149 POINT(40.46019082097459 74.23397383927784) bank57149 +57150 POINT(41.66728792785154 74.63871447251542) bank57150 +57151 POINT(41.33475775460636 74.84208842422223) bank57151 +57152 POINT(40.67947131947812 74.86002131899754) bank57152 +57153 POINT(40.99845817189487 74.61899843872901) bank57153 +57154 POINT(41.58788308516548 74.97417740377858) bank57154 +57155 POINT(39.87146753056363 73.10969008244454) bank57155 +57156 POINT(40.33679351094868 73.52357205259935) bank57156 +57157 POINT(40.29447038288774 74.66014568451158) bank57157 +57158 POINT(40.44338705045664 73.98596137411236) bank57158 +57159 POINT(40.59864108052304 73.48561516535703) bank57159 +57160 POINT(40.19507605384465 74.15044034215596) bank57160 +57161 POINT(39.82842708496149 73.17117040246859) bank57161 +57162 POINT(40.16636951453892 73.19466102750505) bank57162 +57163 POINT(40.192262429458964 74.72458274000302) bank57163 +57164 POINT(40.04194339349956 74.67165467752699) bank57164 +57165 POINT(41.57341861152076 74.40697135392197) bank57165 +57166 POINT(40.19945544890042 73.26999453034328) bank57166 +57167 POINT(41.44167996292089 74.8330036039062) bank57167 +57168 POINT(40.497132831479 74.91680987410156) bank57168 +57169 POINT(41.17960366071812 74.52845982453863) bank57169 +57170 POINT(39.96405440251393 74.30973226391013) bank57170 +57171 POINT(41.12837887580547 74.29285081078024) bank57171 +57172 POINT(40.249867892395116 74.59514408960925) bank57172 +57173 POINT(40.54510161064385 73.83501577661308) bank57173 +57174 POINT(40.84289596736328 73.13177352034141) bank57174 +57175 POINT(40.04148355425595 74.58967638950116) bank57175 +57176 POINT(41.159880619321285 74.43890829476365) bank57176 +57177 POINT(40.481665886426846 74.32940396578542) bank57177 +57178 POINT(41.23314068291347 74.40808818212143) bank57178 +57179 POINT(40.567747878870414 74.72515880031042) bank57179 +57180 POINT(41.1209418433109 74.65893759436403) bank57180 +57181 POINT(40.11928890963653 74.19880142890598) bank57181 +57182 POINT(40.78642899576005 73.04178581206082) bank57182 +57183 POINT(40.091239396563935 73.77347599871433) bank57183 +57184 POINT(41.05539585543677 74.94416075550554) bank57184 +57185 POINT(41.054951245492724 74.07257084292125) bank57185 +57186 POINT(41.34063789406085 73.66888357956098) bank57186 +57187 POINT(40.53211851109766 74.09360096551681) bank57187 +57188 POINT(39.993301433219735 73.3897390579103) bank57188 +57189 POINT(40.97960876170205 73.51676000271465) bank57189 +57190 POINT(41.64310351019992 74.05112124901794) bank57190 +57191 POINT(40.37805919973129 73.44503590659485) bank57191 +57192 POINT(40.85996665453997 74.60134707979051) bank57192 +57193 POINT(41.064094991565604 73.24352480111554) bank57193 +57194 POINT(40.36224012386515 74.33317841911698) bank57194 +57195 POINT(41.69709123579203 73.11712635053256) bank57195 +57196 POINT(40.99272004015846 74.04895885748626) bank57196 +57197 POINT(40.83951101053787 74.51207034990043) bank57197 +57198 POINT(41.227724534339835 73.66829844757063) bank57198 +57199 POINT(41.06141514333398 73.44317195776007) bank57199 +57200 POINT(41.27010347387101 74.09463688948519) bank57200 +57201 POINT(39.975032078361046 74.05875275763508) bank57201 +57202 POINT(40.651301056971924 74.7051791795418) bank57202 +57203 POINT(40.48191492356111 73.74665458805188) bank57203 +57204 POINT(40.41235080936135 74.7047019916932) bank57204 +57205 POINT(40.72823130954647 73.44625615635647) bank57205 +57206 POINT(40.48259446511814 74.47420207099555) bank57206 +57207 POINT(40.63624163178709 74.83178417631908) bank57207 +57208 POINT(40.558525549604134 73.36085074748185) bank57208 +57209 POINT(40.97329378977266 73.13173273476654) bank57209 +57210 POINT(40.19007509483687 74.39005625505379) bank57210 +57211 POINT(40.67181122661383 74.45476079260445) bank57211 +57212 POINT(40.1081669378969 74.45116976441608) bank57212 +57213 POINT(40.5360282066215 74.96905880134925) bank57213 +57214 POINT(41.24728961357384 73.38285975256181) bank57214 +57215 POINT(39.82672846158779 73.22249479298496) bank57215 +57216 POINT(41.60070338441715 74.06447840703841) bank57216 +57217 POINT(40.09660842382775 74.23298522894028) bank57217 +57218 POINT(41.19897490565797 73.7037479440728) bank57218 +57219 POINT(40.61587281822966 73.53759885482195) bank57219 +57220 POINT(40.96775252513488 73.03003268072622) bank57220 +57221 POINT(41.22534356081068 73.24141421178976) bank57221 +57222 POINT(40.000714843750735 73.8060387535327) bank57222 +57223 POINT(41.46657167561039 73.0476751953539) bank57223 +57224 POINT(40.68440842696183 73.58373033942246) bank57224 +57225 POINT(41.429720129165084 73.80948949174163) bank57225 +57226 POINT(39.95298800219788 73.23444515332757) bank57226 +57227 POINT(39.85535828154103 74.07339234333463) bank57227 +57228 POINT(41.401633468214946 73.93676982601407) bank57228 +57229 POINT(41.656062927791154 74.10148944904616) bank57229 +57230 POINT(40.792106044546586 74.66059244359394) bank57230 +57231 POINT(40.28521820253235 74.4584892280854) bank57231 +57232 POINT(40.596701058881706 73.62089206663825) bank57232 +57233 POINT(40.69855443249064 74.89671325308841) bank57233 +57234 POINT(41.13064579081028 74.35376575522116) bank57234 +57235 POINT(40.32949426909867 73.85830219331837) bank57235 +57236 POINT(40.69012791072923 74.92589174787207) bank57236 +57237 POINT(39.934447553136266 73.96629658372187) bank57237 +57238 POINT(41.262100902738915 73.77079375955063) bank57238 +57239 POINT(40.309718829995944 74.49660027804816) bank57239 +57240 POINT(41.30647804461213 73.4086959480245) bank57240 +57241 POINT(40.409256944327346 73.15616103271806) bank57241 +57242 POINT(40.77604189338522 74.27201256127984) bank57242 +57243 POINT(40.89532281884428 73.91130675896112) bank57243 +57244 POINT(41.498091143926366 74.7063597511547) bank57244 +57245 POINT(39.97444070963356 73.20232759855841) bank57245 +57246 POINT(40.616469575662435 74.54369723476808) bank57246 +57247 POINT(39.77951593083541 73.15876245513999) bank57247 +57248 POINT(40.46750698518409 74.08557586464212) bank57248 +57249 POINT(40.60159797554348 74.54051994729946) bank57249 +57250 POINT(39.89981449126653 74.34119536776365) bank57250 +57251 POINT(40.167926246786095 73.3649494619) bank57251 +57252 POINT(40.11921546435044 74.10635328889467) bank57252 +57253 POINT(40.408529007968724 74.19425755537519) bank57253 +57254 POINT(41.39140011065198 73.6143785037624) bank57254 +57255 POINT(41.31945579135306 74.68125509989385) bank57255 +57256 POINT(40.120311720338805 74.14723137245002) bank57256 +57257 POINT(41.574088220586766 73.54582152636834) bank57257 +57258 POINT(40.36465794497122 74.12503158075303) bank57258 +57259 POINT(41.12469926291016 74.35481562876299) bank57259 +57260 POINT(40.500446625147426 73.83005471917956) bank57260 +57261 POINT(40.14174166523256 74.61479714401487) bank57261 +57262 POINT(41.665548106498626 74.81546487235934) bank57262 +57263 POINT(41.060561295069554 73.66019402002443) bank57263 +57264 POINT(41.60190280357287 73.56357900356905) bank57264 +57265 POINT(41.42777679829221 73.1425180189856) bank57265 +57266 POINT(41.110272418801145 73.8684629585785) bank57266 +57267 POINT(40.131673881239195 74.76545275388321) bank57267 +57268 POINT(40.454164190246985 74.0465999738905) bank57268 +57269 POINT(40.16539473717506 74.59909599764842) bank57269 +57270 POINT(40.37950486916874 73.32542980040485) bank57270 +57271 POINT(40.60466435188986 73.38526709143352) bank57271 +57272 POINT(41.317402359238045 73.6810414132271) bank57272 +57273 POINT(40.595280148884434 73.24063443715333) bank57273 +57274 POINT(40.40546448481361 73.3642002365387) bank57274 +57275 POINT(41.30271192909736 73.59604920546364) bank57275 +57276 POINT(41.309703119838424 74.59959049515203) bank57276 +57277 POINT(40.431626499695454 74.30431265332716) bank57277 +57278 POINT(40.46895685046398 73.16166675172596) bank57278 +57279 POINT(40.34311066764673 73.08533967717925) bank57279 +57280 POINT(41.36341147121209 73.42050175819303) bank57280 +57281 POINT(40.23418508354985 74.14426386207268) bank57281 +57282 POINT(41.26572212904716 73.71527600999543) bank57282 +57283 POINT(41.440913627661956 74.26480160464936) bank57283 +57284 POINT(41.285003788318285 73.73919335704731) bank57284 +57285 POINT(40.668085560600126 73.15481571569318) bank57285 +57286 POINT(40.35537837009846 74.78888248154836) bank57286 +57287 POINT(41.06599718408346 73.97638429863191) bank57287 +57288 POINT(40.97913937137394 74.24037786153582) bank57288 +57289 POINT(39.84866812384357 73.08984739159607) bank57289 +57290 POINT(41.3935091022049 74.92053736149145) bank57290 +57291 POINT(40.81525056583853 74.82780052727847) bank57291 +57292 POINT(40.55851108360455 74.43297986860352) bank57292 +57293 POINT(40.597155592633214 74.35122016542383) bank57293 +57294 POINT(40.49604319751276 73.17876231408007) bank57294 +57295 POINT(40.3518674567856 74.10698009526709) bank57295 +57296 POINT(41.6888286585769 74.10884887584069) bank57296 +57297 POINT(39.75067865538816 73.42842885394501) bank57297 +57298 POINT(40.445461181772174 74.56976651249283) bank57298 +57299 POINT(40.41419256151221 73.2467572026884) bank57299 +57300 POINT(41.10625865511764 74.34601683583972) bank57300 +57301 POINT(41.109030614238534 73.21826508542499) bank57301 +57302 POINT(39.89144648471742 74.41014300283054) bank57302 +57303 POINT(41.357816920759745 73.14136014981632) bank57303 +57304 POINT(41.251154924579495 73.27345518126204) bank57304 +57305 POINT(41.64849300269347 73.79687741708571) bank57305 +57306 POINT(40.994610073795194 74.54086614435653) bank57306 +57307 POINT(40.83759283766492 73.80900476965124) bank57307 +57308 POINT(40.4829803905636 73.0066160198695) bank57308 +57309 POINT(39.85967341472267 74.72059162986395) bank57309 +57310 POINT(39.85476958405901 74.28749210152631) bank57310 +57311 POINT(39.88848539944514 74.14892188559644) bank57311 +57312 POINT(41.54373182490627 73.82192815387387) bank57312 +57313 POINT(39.84742455412533 74.01784906049708) bank57313 +57314 POINT(41.605502950763906 74.56284636221709) bank57314 +57315 POINT(40.50559346619537 73.07323389505486) bank57315 +57316 POINT(41.51178473649047 73.84756444823968) bank57316 +57317 POINT(41.14789888116559 74.77253609256479) bank57317 +57318 POINT(41.23454521494645 74.24950299210724) bank57318 +57319 POINT(40.486895526494 73.08419008619326) bank57319 +57320 POINT(39.91815874644571 73.70112577450104) bank57320 +57321 POINT(40.46735399328974 73.62352991490835) bank57321 +57322 POINT(39.73167406349783 74.59989909359086) bank57322 +57323 POINT(40.706618752807785 73.96992717387418) bank57323 +57324 POINT(41.45854696965335 73.94856072733671) bank57324 +57325 POINT(40.29773507078389 74.55904694523923) bank57325 +57326 POINT(40.77495905574104 73.25189324493532) bank57326 +57327 POINT(41.687293193574504 73.93294034384193) bank57327 +57328 POINT(40.72599368136629 74.47275257715683) bank57328 +57329 POINT(40.688123437835706 73.02265747274903) bank57329 +57330 POINT(41.28963688939938 74.17538487999606) bank57330 +57331 POINT(41.51372028511143 74.73388121617985) bank57331 +57332 POINT(41.40267470197577 74.12305725803554) bank57332 +57333 POINT(41.005987652164045 73.07224698094855) bank57333 +57334 POINT(40.50468342839645 73.29447971728655) bank57334 +57335 POINT(40.91572419759478 74.55432516650751) bank57335 +57336 POINT(40.49076491407186 74.72198400518418) bank57336 +57337 POINT(40.874499385036835 73.61137338616338) bank57337 +57338 POINT(40.18971537271615 74.08811341578566) bank57338 +57339 POINT(40.54576134911017 73.63682106322236) bank57339 +57340 POINT(41.00396192069851 74.65475677901154) bank57340 +57341 POINT(40.830309419164244 73.29773126714791) bank57341 +57342 POINT(41.26605928093126 73.3190229812814) bank57342 +57343 POINT(40.053255931416906 73.47815842416679) bank57343 +57344 POINT(40.715701489439525 73.36454768795974) bank57344 +57345 POINT(41.377216582121726 74.72950413020598) bank57345 +57346 POINT(39.80851453135858 74.42501576056748) bank57346 +57347 POINT(41.37821842744379 73.31541773145067) bank57347 +57348 POINT(40.24923771052675 74.46502141489292) bank57348 +57349 POINT(40.95081282945366 74.760216078381) bank57349 +57350 POINT(39.838638264592596 74.72639322298447) bank57350 +57351 POINT(41.46473754346326 73.28847745386336) bank57351 +57352 POINT(40.79407186185358 74.04656227383815) bank57352 +57353 POINT(41.65522757300481 74.8918073308601) bank57353 +57354 POINT(39.73735258204473 74.32458541109102) bank57354 +57355 POINT(40.91502169771395 73.47956273415303) bank57355 +57356 POINT(40.453060187279945 74.58481790792209) bank57356 +57357 POINT(40.92182499187893 73.02299134635227) bank57357 +57358 POINT(40.50875729066344 74.05654305886294) bank57358 +57359 POINT(39.79095737333141 74.88635195091378) bank57359 +57360 POINT(41.07125820168908 73.42863998496735) bank57360 +57361 POINT(39.880570006653684 74.79687584248158) bank57361 +57362 POINT(40.9374074447521 73.63473861519941) bank57362 +57363 POINT(40.42049972111877 74.33706515551658) bank57363 +57364 POINT(41.588410619867034 73.03823719800728) bank57364 +57365 POINT(40.80729044817005 73.94563202110585) bank57365 +57366 POINT(41.2666788401769 74.37969480231835) bank57366 +57367 POINT(41.16499377807036 73.9102457262023) bank57367 +57368 POINT(41.40068162491164 74.96926536958166) bank57368 +57369 POINT(40.08002565877099 73.4862616501378) bank57369 +57370 POINT(41.468700404262975 73.75535331465682) bank57370 +57371 POINT(40.32914281403101 74.97691340867483) bank57371 +57372 POINT(39.73154373611863 74.8511676174114) bank57372 +57373 POINT(41.44278161477659 74.98138658384781) bank57373 +57374 POINT(40.510181937062136 74.30423627312084) bank57374 +57375 POINT(40.91344236659662 73.54913086434271) bank57375 +57376 POINT(41.29286249019918 74.76311810075227) bank57376 +57377 POINT(39.826745501722854 74.7916072520082) bank57377 +57378 POINT(41.24484233169636 73.3643601931157) bank57378 +57379 POINT(39.817713546466706 74.13319716832697) bank57379 +57380 POINT(40.58095771959935 73.91638459825158) bank57380 +57381 POINT(41.02308144178416 74.82049391642295) bank57381 +57382 POINT(40.92807102136173 73.46615214893123) bank57382 +57383 POINT(41.62382099462316 74.17658321592943) bank57383 +57384 POINT(40.480925566892324 73.82997945591339) bank57384 +57385 POINT(40.44673565453764 74.05473841621333) bank57385 +57386 POINT(39.82678090105 73.2350306266321) bank57386 +57387 POINT(40.39270757805276 73.53277138612461) bank57387 +57388 POINT(41.48534038368714 74.0803918093366) bank57388 +57389 POINT(39.96271203457305 73.98811833073343) bank57389 +57390 POINT(40.47236812318425 73.43614211598464) bank57390 +57391 POINT(41.026736154142675 74.65036774926558) bank57391 +57392 POINT(41.6951962569164 74.82750515589966) bank57392 +57393 POINT(41.2074405448403 73.06756929949387) bank57393 +57394 POINT(40.16881622525691 74.40619253298647) bank57394 +57395 POINT(41.53176505042931 74.98703218590988) bank57395 +57396 POINT(39.72253563430182 73.9398371275642) bank57396 +57397 POINT(40.741345567748645 73.24658513675139) bank57397 +57398 POINT(40.17195652401198 73.91241520883425) bank57398 +57399 POINT(39.845745430766364 74.4465949167954) bank57399 +57400 POINT(40.8366193123091 74.34604533250025) bank57400 +57401 POINT(40.977532337131116 73.95345218889703) bank57401 +57402 POINT(41.629340555609204 74.60880261153208) bank57402 +57403 POINT(41.39391707865127 74.66485383568342) bank57403 +57404 POINT(39.75202427314567 73.73063508789261) bank57404 +57405 POINT(39.81603041844256 74.03717608761977) bank57405 +57406 POINT(41.01056219483958 74.22228537729539) bank57406 +57407 POINT(40.34701329149112 74.23349576686655) bank57407 +57408 POINT(40.113397703294204 74.91002354113158) bank57408 +57409 POINT(40.36553070407726 74.07782807635193) bank57409 +57410 POINT(40.11833897453328 74.66163616284022) bank57410 +57411 POINT(40.43160733295976 73.01768746554538) bank57411 +57412 POINT(40.655668105106415 74.15693799623847) bank57412 +57413 POINT(39.7472377836143 74.27424443270047) bank57413 +57414 POINT(40.33700900866486 74.54509013452235) bank57414 +57415 POINT(40.93402276674505 74.58326653187804) bank57415 +57416 POINT(39.77278473884937 73.12922714584592) bank57416 +57417 POINT(41.45772674763956 74.23803392672525) bank57417 +57418 POINT(40.20293980344716 74.44796483303345) bank57418 +57419 POINT(40.05126596205931 73.01295967916086) bank57419 +57420 POINT(41.231299797262814 73.14092619435145) bank57420 +57421 POINT(40.31256174619899 74.74525347567196) bank57421 +57422 POINT(41.56052117909856 74.33649978906165) bank57422 +57423 POINT(40.67624252286681 74.95901578196968) bank57423 +57424 POINT(39.829021693896856 74.5478355818059) bank57424 +57425 POINT(40.85434758870552 73.94030443122433) bank57425 +57426 POINT(41.247623561669386 74.63952103118606) bank57426 +57427 POINT(40.94981335809346 74.89507859308463) bank57427 +57428 POINT(40.36485541854719 74.94080276176808) bank57428 +57429 POINT(41.54715956930826 74.76291064668985) bank57429 +57430 POINT(41.11702160138283 74.79278928116372) bank57430 +57431 POINT(41.48308299467694 74.223636076825) bank57431 +57432 POINT(40.13427098538801 74.45686844697333) bank57432 +57433 POINT(41.27166241506784 73.82497078544998) bank57433 +57434 POINT(41.37320672863846 74.00477419408759) bank57434 +57435 POINT(41.4276056548196 73.09521640350587) bank57435 +57436 POINT(40.36964291914898 73.92161127901427) bank57436 +57437 POINT(40.71358863837955 73.55856497531443) bank57437 +57438 POINT(39.86169171784574 74.37700610912728) bank57438 +57439 POINT(39.86767551925726 73.72120942678987) bank57439 +57440 POINT(39.816779354423005 74.51565449157171) bank57440 +57441 POINT(40.6678319732609 74.36752202334398) bank57441 +57442 POINT(41.23913632722149 73.75369821631331) bank57442 +57443 POINT(40.99749876825035 74.60622041707192) bank57443 +57444 POINT(40.96272801936408 73.83757012891063) bank57444 +57445 POINT(39.771860655618966 73.1099752459116) bank57445 +57446 POINT(40.39076199092772 73.85467796213348) bank57446 +57447 POINT(41.366572389760506 73.74039208573735) bank57447 +57448 POINT(40.19025944236011 74.12602116029058) bank57448 +57449 POINT(40.77110749941183 73.62105767605073) bank57449 +57450 POINT(40.38260422924208 74.06136953161048) bank57450 +57451 POINT(41.301717626183354 74.58432555336891) bank57451 +57452 POINT(40.503243673231 74.76541334149897) bank57452 +57453 POINT(40.83285259358521 73.77082110002702) bank57453 +57454 POINT(40.31965368608294 74.16393307966115) bank57454 +57455 POINT(39.72968590914678 73.50440111065215) bank57455 +57456 POINT(41.456727448130316 74.99644096229868) bank57456 +57457 POINT(41.08859533942096 73.55585045310235) bank57457 +57458 POINT(40.014414347240475 74.16793904911732) bank57458 +57459 POINT(41.67491571041762 74.25546920078587) bank57459 +57460 POINT(40.971294233958 74.83408148050458) bank57460 +57461 POINT(41.20215484214805 74.65611289924743) bank57461 +57462 POINT(41.409410900796985 73.55005403430388) bank57462 +57463 POINT(40.150306588161826 73.35151610471709) bank57463 +57464 POINT(39.809925748416134 74.8428054183838) bank57464 +57465 POINT(41.283882253766066 74.59859682765081) bank57465 +57466 POINT(41.66264328801405 73.45047518092893) bank57466 +57467 POINT(40.099721898142874 74.03830682640573) bank57467 +57468 POINT(40.56679538063682 74.00328462915245) bank57468 +57469 POINT(41.30556160744988 73.06064613939084) bank57469 +57470 POINT(41.07707817593122 74.35644158453208) bank57470 +57471 POINT(40.355877401077635 74.00656743695934) bank57471 +57472 POINT(41.544059698539506 73.44350660084896) bank57472 +57473 POINT(40.76502398297293 73.68892002330318) bank57473 +57474 POINT(40.2080171259697 74.76606606869451) bank57474 +57475 POINT(41.68854167466478 74.89857724692575) bank57475 +57476 POINT(41.640227864538794 73.07639885170482) bank57476 +57477 POINT(41.414570096242734 74.2738213684038) bank57477 +57478 POINT(40.30758621406065 73.43280439670892) bank57478 +57479 POINT(40.532109069174425 74.62934020979206) bank57479 +57480 POINT(41.65927124571236 73.39178299031003) bank57480 +57481 POINT(40.10591178043809 73.94516866700063) bank57481 +57482 POINT(41.32802554293711 73.03830910071396) bank57482 +57483 POINT(40.53843394146969 73.69832334774202) bank57483 +57484 POINT(40.47603893989938 73.26995071323574) bank57484 +57485 POINT(41.27895892522271 74.95518711636218) bank57485 +57486 POINT(39.99285458067734 74.34980425087592) bank57486 +57487 POINT(40.210671310982775 74.21475047588703) bank57487 +57488 POINT(41.11212898904515 74.60628495239385) bank57488 +57489 POINT(40.82153144871089 74.63297110140536) bank57489 +57490 POINT(40.47025981206153 74.41295347414388) bank57490 +57491 POINT(39.81620081879379 73.57040637188754) bank57491 +57492 POINT(39.97560131889996 73.2923220582541) bank57492 +57493 POINT(40.12598976928099 73.2340631130787) bank57493 +57494 POINT(41.534027133185475 73.35150030052037) bank57494 +57495 POINT(41.20580274824478 73.28576129539442) bank57495 +57496 POINT(41.22359055618629 74.68674519497625) bank57496 +57497 POINT(40.30584289768971 74.16375293258224) bank57497 +57498 POINT(40.057198434249884 74.71240279122705) bank57498 +57499 POINT(39.95541043973512 73.2623533770765) bank57499 +57500 POINT(39.758413202809386 74.06613147314889) bank57500 +57501 POINT(40.95593841287182 73.17660803369452) bank57501 +57502 POINT(41.55462919943887 74.11241454077651) bank57502 +57503 POINT(40.1197313671552 74.8063075418958) bank57503 +57504 POINT(39.73641142889593 74.17731866346458) bank57504 +57505 POINT(40.698230279878814 74.05503552046781) bank57505 +57506 POINT(39.89853853114835 74.72088619804953) bank57506 +57507 POINT(40.131036186141664 74.16979926317683) bank57507 +57508 POINT(40.035112087261304 73.48242294163042) bank57508 +57509 POINT(41.26003662439226 73.1145597674666) bank57509 +57510 POINT(41.32671808906067 73.85077606271471) bank57510 +57511 POINT(41.59478348344658 74.18043790601024) bank57511 +57512 POINT(40.57985676662269 73.51465032754301) bank57512 +57513 POINT(40.307586978511196 74.34913656529024) bank57513 +57514 POINT(40.69054546379794 73.70538712894657) bank57514 +57515 POINT(41.55300412199592 74.5498489679319) bank57515 +57516 POINT(40.26604018643749 74.6849998026287) bank57516 +57517 POINT(41.07567952645085 74.33933189782476) bank57517 +57518 POINT(40.699122360656254 74.0814121661933) bank57518 +57519 POINT(39.86744540894044 73.61628458929381) bank57519 +57520 POINT(40.49916813551593 73.45363443071503) bank57520 +57521 POINT(40.316145494541914 74.11276245452297) bank57521 +57522 POINT(40.49918769791194 74.27527592088254) bank57522 +57523 POINT(40.65231347202475 74.0205450345982) bank57523 +57524 POINT(40.59846136216676 74.34851304854963) bank57524 +57525 POINT(41.48860059734122 74.4402457174348) bank57525 +57526 POINT(39.96989995207329 73.8140049896216) bank57526 +57527 POINT(40.985849760079155 73.27264634642367) bank57527 +57528 POINT(40.64873107166211 73.25851813266016) bank57528 +57529 POINT(40.5885328083487 73.86574064756931) bank57529 +57530 POINT(40.443620948432084 74.12535104095457) bank57530 +57531 POINT(41.155674800308056 73.62446829174883) bank57531 +57532 POINT(40.372178382417694 74.4753271642262) bank57532 +57533 POINT(40.72853847115712 73.65581091983209) bank57533 +57534 POINT(39.74185996678404 74.75371629592769) bank57534 +57535 POINT(40.65879392858592 73.23121064921953) bank57535 +57536 POINT(39.775580957464896 73.02067478468896) bank57536 +57537 POINT(41.63755415442455 73.36861170095385) bank57537 +57538 POINT(40.89541543753485 74.03795676277366) bank57538 +57539 POINT(41.52130916712478 74.01433585020685) bank57539 +57540 POINT(39.72109361889029 74.6637845647124) bank57540 +57541 POINT(40.36834093491792 73.31836191854707) bank57541 +57542 POINT(40.60945525917097 74.20534738824595) bank57542 +57543 POINT(40.92002064965197 73.14107859722397) bank57543 +57544 POINT(41.208160208595686 74.53363151646697) bank57544 +57545 POINT(39.97153416194376 74.54940411462302) bank57545 +57546 POINT(41.08011427352953 73.5009814578967) bank57546 +57547 POINT(40.78050267949938 73.59900584680261) bank57547 +57548 POINT(41.08700285355343 73.43774791800986) bank57548 +57549 POINT(40.11062156213995 74.68753516622748) bank57549 +57550 POINT(41.58878676835389 74.96212761322985) bank57550 +57551 POINT(41.28392078547331 74.70600851169218) bank57551 +57552 POINT(40.58509745153987 73.39617046220562) bank57552 +57553 POINT(41.55488267826428 74.20073446003484) bank57553 +57554 POINT(40.006878549836664 73.55881340932552) bank57554 +57555 POINT(40.720638856418404 73.05991238089592) bank57555 +57556 POINT(40.31897100445856 73.14788976149278) bank57556 +57557 POINT(40.49744342985894 73.19952839580057) bank57557 +57558 POINT(39.75678793389792 74.11008482174448) bank57558 +57559 POINT(39.727010169716564 73.82544017980098) bank57559 +57560 POINT(39.93004108809551 74.1737584636239) bank57560 +57561 POINT(39.866654491838354 73.1251081090257) bank57561 +57562 POINT(40.2853951040611 74.05028783031781) bank57562 +57563 POINT(39.87369413213248 73.82502836147516) bank57563 +57564 POINT(40.38361193396133 73.839357452201) bank57564 +57565 POINT(39.88942352294386 74.64859884210925) bank57565 +57566 POINT(41.59036756857342 74.914212739642) bank57566 +57567 POINT(41.60281218443774 73.39084776441719) bank57567 +57568 POINT(40.731281791500464 73.78293685345267) bank57568 +57569 POINT(41.5171389306074 73.50181704908822) bank57569 +57570 POINT(40.10498136230113 74.84162656600355) bank57570 +57571 POINT(41.6790914624279 73.57311341611401) bank57571 +57572 POINT(39.801512307998394 73.4378946501271) bank57572 +57573 POINT(39.888245242281975 73.37585863071727) bank57573 +57574 POINT(40.02662031507905 74.04233020802945) bank57574 +57575 POINT(41.23093528872073 74.42218638483916) bank57575 +57576 POINT(40.32312493916407 74.35765478532021) bank57576 +57577 POINT(40.185274652539455 73.62672950681615) bank57577 +57578 POINT(40.15951750299107 74.24331298896061) bank57578 +57579 POINT(40.834010454667784 74.11017089658124) bank57579 +57580 POINT(40.20710384543931 73.87835285291784) bank57580 +57581 POINT(40.55049052039364 74.42479954385401) bank57581 +57582 POINT(40.193071814842035 74.89046044017587) bank57582 +57583 POINT(41.22874487053748 74.377507267048) bank57583 +57584 POINT(39.99450130514617 74.38704697640094) bank57584 +57585 POINT(40.34717185763074 74.79668723203908) bank57585 +57586 POINT(41.18252546256604 73.59864292196033) bank57586 +57587 POINT(40.52074137240106 73.3916208893483) bank57587 +57588 POINT(41.11536182091912 73.57465280689223) bank57588 +57589 POINT(40.95069094790252 73.08314709168425) bank57589 +57590 POINT(41.404613919115 73.07881247603845) bank57590 +57591 POINT(40.114293234596495 73.38149309921106) bank57591 +57592 POINT(40.912076356007795 73.7845927560973) bank57592 +57593 POINT(40.015146258243796 73.4944111613649) bank57593 +57594 POINT(41.23181431059081 73.97415669981595) bank57594 +57595 POINT(40.00143364267402 73.96096882747376) bank57595 +57596 POINT(41.22068220797501 73.19441459856817) bank57596 +57597 POINT(41.020894664236046 74.86571755613396) bank57597 +57598 POINT(41.102410360312646 73.36483186281367) bank57598 +57599 POINT(40.342506532014276 74.18702996785655) bank57599 +57600 POINT(41.06950051332444 74.4058055794032) bank57600 +57601 POINT(40.004984390680846 74.98394743405068) bank57601 +57602 POINT(40.45235505513974 73.96507005594944) bank57602 +57603 POINT(41.269109066176725 74.63425840897163) bank57603 +57604 POINT(40.91469731624065 74.7721742967464) bank57604 +57605 POINT(40.48718074544475 73.99546869215844) bank57605 +57606 POINT(41.30626258858986 73.35414759312071) bank57606 +57607 POINT(40.94008560183197 74.1530434481486) bank57607 +57608 POINT(40.64738819279679 74.79224603785332) bank57608 +57609 POINT(39.82649262805782 73.72460506840403) bank57609 +57610 POINT(41.33806032500957 73.59683016266452) bank57610 +57611 POINT(41.608917430519384 74.89758491528968) bank57611 +57612 POINT(40.250968696597155 74.42994636744521) bank57612 +57613 POINT(39.78943091943207 73.72602471259904) bank57613 +57614 POINT(40.13097110428104 74.20581509590329) bank57614 +57615 POINT(40.13264959254596 73.97769854314377) bank57615 +57616 POINT(40.63548812151837 73.96668327957488) bank57616 +57617 POINT(41.17341350367346 74.22577329961199) bank57617 +57618 POINT(40.00117478893016 74.71401970804003) bank57618 +57619 POINT(40.04901417709517 73.04073164710968) bank57619 +57620 POINT(40.212281517520985 74.6917147760775) bank57620 +57621 POINT(39.72150557199125 74.84136166420431) bank57621 +57622 POINT(41.664107782807356 74.23331707131385) bank57622 +57623 POINT(40.85039415238471 73.57145292984531) bank57623 +57624 POINT(40.853795883106876 73.05026555718749) bank57624 +57625 POINT(40.561656008485386 73.53396051434703) bank57625 +57626 POINT(40.88751214919592 73.10529414142286) bank57626 +57627 POINT(41.6053538532809 73.6793097091918) bank57627 +57628 POINT(40.27350152006751 73.92528641545175) bank57628 +57629 POINT(40.6263384953067 74.18951869679327) bank57629 +57630 POINT(39.990191446090954 74.4139270715334) bank57630 +57631 POINT(41.412126695339396 74.9765499186457) bank57631 +57632 POINT(41.3609780649624 74.05172673954479) bank57632 +57633 POINT(41.008726091868354 73.33772486114847) bank57633 +57634 POINT(40.01821202836807 74.6445053476371) bank57634 +57635 POINT(40.03467310527048 74.95863681373852) bank57635 +57636 POINT(40.44111502660005 74.0786797308982) bank57636 +57637 POINT(40.07265194313836 74.47218913831601) bank57637 +57638 POINT(40.42269426361716 73.875303274623) bank57638 +57639 POINT(41.65886416670575 73.24577445951876) bank57639 +57640 POINT(41.689619790296874 73.37024794515793) bank57640 +57641 POINT(40.829740709494324 73.20399496054718) bank57641 +57642 POINT(40.11983749535144 74.6625633897207) bank57642 +57643 POINT(40.10912065653045 74.40988588363497) bank57643 +57644 POINT(40.57654609883223 73.41791560279651) bank57644 +57645 POINT(40.049029378529596 73.63673838784969) bank57645 +57646 POINT(41.15844181725251 73.05908884227273) bank57646 +57647 POINT(41.44518679108719 73.16038049399606) bank57647 +57648 POINT(40.16975578321369 73.99023658015493) bank57648 +57649 POINT(41.24422210889975 74.22721053529882) bank57649 +57650 POINT(40.93124080399986 73.82200795353707) bank57650 +57651 POINT(39.79597190139711 74.95869355405262) bank57651 +57652 POINT(40.910133812819296 73.08262680077031) bank57652 +57653 POINT(40.72884863999469 74.30883101360781) bank57653 +57654 POINT(41.134845665738176 74.36172644225412) bank57654 +57655 POINT(41.18421914586048 73.93496569066572) bank57655 +57656 POINT(41.29346858594545 74.59601295686028) bank57656 +57657 POINT(41.15787096914386 73.97578966607945) bank57657 +57658 POINT(40.41524934456032 74.74455864958281) bank57658 +57659 POINT(40.51344183881438 74.65729105062631) bank57659 +57660 POINT(41.061629778098606 73.05377764699632) bank57660 +57661 POINT(40.103642302208655 73.67067614659585) bank57661 +57662 POINT(40.66102623757281 74.55542152295938) bank57662 +57663 POINT(41.692864638475456 73.11915593252672) bank57663 +57664 POINT(40.93549344779123 73.6912427304697) bank57664 +57665 POINT(41.56359002936544 73.41290084145253) bank57665 +57666 POINT(40.401188741503 74.95358811126812) bank57666 +57667 POINT(40.969000546913875 73.84442921666113) bank57667 +57668 POINT(40.49136886350768 73.49807944022484) bank57668 +57669 POINT(40.97961810260465 74.64169211373161) bank57669 +57670 POINT(40.66554916343759 73.83114163309386) bank57670 +57671 POINT(40.99427589364512 73.22790871724321) bank57671 +57672 POINT(40.58873295071408 73.36078883772014) bank57672 +57673 POINT(40.615547170929865 74.6572924869804) bank57673 +57674 POINT(39.73581045870567 74.57084578829654) bank57674 +57675 POINT(40.94562107922823 73.10256565215134) bank57675 +57676 POINT(41.41658738610893 74.3429625476961) bank57676 +57677 POINT(40.43778653992241 74.59531633855198) bank57677 +57678 POINT(40.368355669368185 73.29925140198) bank57678 +57679 POINT(41.41508560124794 74.53683043381518) bank57679 +57680 POINT(41.14516600302653 74.2299141111495) bank57680 +57681 POINT(40.13748009284426 73.0600997131236) bank57681 +57682 POINT(41.58399028786485 74.88637932224873) bank57682 +57683 POINT(40.574128127249054 74.8675805845179) bank57683 +57684 POINT(41.15854185690913 74.3804555544876) bank57684 +57685 POINT(40.672768231369254 74.48279076367756) bank57685 +57686 POINT(40.434499733990805 74.42751444897831) bank57686 +57687 POINT(40.16546289720521 73.70038432010936) bank57687 +57688 POINT(40.13927831695655 73.50143169906815) bank57688 +57689 POINT(40.31758542265776 73.59582440668635) bank57689 +57690 POINT(40.039492687622456 74.46298886778578) bank57690 +57691 POINT(40.155097594724694 73.47339785388316) bank57691 +57692 POINT(41.49924058346639 73.53358595174075) bank57692 +57693 POINT(41.251493302357765 73.4947351126166) bank57693 +57694 POINT(41.310333669457556 73.87639908831461) bank57694 +57695 POINT(41.39985295567956 74.35889304916044) bank57695 +57696 POINT(41.11746873595173 73.56765223868821) bank57696 +57697 POINT(41.49592940531258 73.07033039681208) bank57697 +57698 POINT(39.762647442736636 74.95735902931361) bank57698 +57699 POINT(41.440480258417 74.05181591888541) bank57699 +57700 POINT(40.59605657901902 74.45127093826545) bank57700 +57701 POINT(41.346992580395366 73.44039387253405) bank57701 +57702 POINT(41.098885628824334 74.03886674182674) bank57702 +57703 POINT(41.37045100128695 73.3484366539173) bank57703 +57704 POINT(40.168898341975805 73.58033747734945) bank57704 +57705 POINT(41.140427556678596 74.62877401707068) bank57705 +57706 POINT(41.050010819471666 73.53733067389417) bank57706 +57707 POINT(40.743116010190676 74.29247803958101) bank57707 +57708 POINT(41.65192538323552 73.49825456004385) bank57708 +57709 POINT(40.68419643499355 73.92369454790821) bank57709 +57710 POINT(40.86271231967592 74.93671740520429) bank57710 +57711 POINT(40.386629455558676 74.93153042506655) bank57711 +57712 POINT(41.242360729638314 73.90287352660611) bank57712 +57713 POINT(40.120646285591434 74.24598383422908) bank57713 +57714 POINT(40.636813743225474 73.27773263556995) bank57714 +57715 POINT(41.174787950939944 74.79067732179594) bank57715 +57716 POINT(41.109841683116 73.77021887954324) bank57716 +57717 POINT(41.3771752415871 73.84916036705812) bank57717 +57718 POINT(40.88622202595871 74.62069226108959) bank57718 +57719 POINT(40.35821783164854 73.7832015939183) bank57719 +57720 POINT(40.95051582041179 74.49712988953611) bank57720 +57721 POINT(41.680877022364015 74.6357795093434) bank57721 +57722 POINT(41.44395025900206 74.83931128601316) bank57722 +57723 POINT(40.66197847115799 74.44983890180387) bank57723 +57724 POINT(40.92574711228027 73.21406467164773) bank57724 +57725 POINT(40.942073296099906 74.50496452180337) bank57725 +57726 POINT(40.25127793789786 73.49074167693693) bank57726 +57727 POINT(40.096487358350174 74.68290773566487) bank57727 +57728 POINT(40.82570618179788 73.71245373205298) bank57728 +57729 POINT(41.668295301112344 74.8868795662753) bank57729 +57730 POINT(39.96211823991636 73.02331015369717) bank57730 +57731 POINT(41.246952687765685 73.1401005651081) bank57731 +57732 POINT(41.47523339805288 73.75017506713463) bank57732 +57733 POINT(40.13328558267017 73.90391146733353) bank57733 +57734 POINT(41.23896689665985 74.47158025981959) bank57734 +57735 POINT(40.82199898500119 73.76248705507776) bank57735 +57736 POINT(40.933429459336644 74.8754513974576) bank57736 +57737 POINT(41.18825304986978 73.17832557268154) bank57737 +57738 POINT(41.383999670390025 73.55635419167838) bank57738 +57739 POINT(41.171382204596235 73.04101754702886) bank57739 +57740 POINT(39.82523693271537 74.61336536242752) bank57740 +57741 POINT(40.200183185766555 73.3560863977766) bank57741 +57742 POINT(41.143225670442234 73.706590996787) bank57742 +57743 POINT(41.00853116791202 73.56122699910574) bank57743 +57744 POINT(39.755213219658984 74.36248831521137) bank57744 +57745 POINT(40.08598775325673 73.15713244240669) bank57745 +57746 POINT(41.41094968678882 73.01986780860832) bank57746 +57747 POINT(39.98174359441856 74.23767589962722) bank57747 +57748 POINT(40.097567902596715 73.98638099610335) bank57748 +57749 POINT(39.79737899792325 73.51889605118066) bank57749 +57750 POINT(40.90861842697106 73.58331719354128) bank57750 +57751 POINT(40.15658820815758 73.71671092009298) bank57751 +57752 POINT(39.93075737714418 73.24819766111878) bank57752 +57753 POINT(41.13722487856481 73.41774998272938) bank57753 +57754 POINT(41.530181286201184 74.0537199187706) bank57754 +57755 POINT(41.19576645714942 73.75115181937342) bank57755 +57756 POINT(41.456947175110834 73.18249790981959) bank57756 +57757 POINT(40.35020371083301 74.88780775830615) bank57757 +57758 POINT(40.15115957418638 73.9385455053453) bank57758 +57759 POINT(39.957418068289606 74.72235512950239) bank57759 +57760 POINT(39.904821636083966 74.6535121321926) bank57760 +57761 POINT(41.07074328490264 73.05982672408123) bank57761 +57762 POINT(41.61170487399362 74.42818341221071) bank57762 +57763 POINT(40.60736793802038 74.93342394999506) bank57763 +57764 POINT(41.45016720857738 73.37344698428048) bank57764 +57765 POINT(40.02632324477475 74.4365774025616) bank57765 +57766 POINT(40.63614553680807 74.96551603112121) bank57766 +57767 POINT(40.99697363722448 73.47726658570865) bank57767 +57768 POINT(41.295421543491315 74.12911275240945) bank57768 +57769 POINT(41.334829992819714 74.21828902258946) bank57769 +57770 POINT(41.26505560791949 73.97485988445499) bank57770 +57771 POINT(40.75729625560847 74.7117029848723) bank57771 +57772 POINT(40.44003610657497 74.95085613368215) bank57772 +57773 POINT(40.48097498371824 74.15985618357217) bank57773 +57774 POINT(41.23504259048989 73.15149494179121) bank57774 +57775 POINT(40.55735027031127 74.67513509817987) bank57775 +57776 POINT(40.681476549865955 73.2312314434773) bank57776 +57777 POINT(40.59599176082916 73.74284114507752) bank57777 +57778 POINT(41.632921156206706 74.00645051109501) bank57778 +57779 POINT(41.45801696074663 73.38468906593167) bank57779 +57780 POINT(40.63491787039308 74.45866281324959) bank57780 +57781 POINT(41.10669564970921 74.94847859094311) bank57781 +57782 POINT(41.10206701089598 73.66911548103212) bank57782 +57783 POINT(40.47903094810848 73.16902491254483) bank57783 +57784 POINT(40.06468846467457 74.26815834008787) bank57784 +57785 POINT(41.28032875817564 73.88218433978567) bank57785 +57786 POINT(40.894745149814426 74.26649413931997) bank57786 +57787 POINT(40.48235791459133 73.91163619967428) bank57787 +57788 POINT(41.70411740214643 74.50327568626922) bank57788 +57789 POINT(41.63173994176969 74.32001954710472) bank57789 +57790 POINT(41.11044928444056 74.09083064902946) bank57790 +57791 POINT(40.46634552701402 74.20645153011816) bank57791 +57792 POINT(40.46443363345976 74.53725615287266) bank57792 +57793 POINT(41.29067748238809 74.77510124977137) bank57793 +57794 POINT(39.945640864532116 74.325133422737) bank57794 +57795 POINT(41.14744159361621 73.14565579788298) bank57795 +57796 POINT(40.08182507452681 73.68938398644885) bank57796 +57797 POINT(39.98936928465716 74.17808000896684) bank57797 +57798 POINT(40.869129633361844 73.41269880226523) bank57798 +57799 POINT(39.91124338521722 74.51131055388065) bank57799 +57800 POINT(40.24380124891281 74.72374525222988) bank57800 +57801 POINT(40.694709537405714 74.22238225680225) bank57801 +57802 POINT(41.55159530642217 73.13740897389023) bank57802 +57803 POINT(41.4460749496815 74.79500122421298) bank57803 +57804 POINT(39.804082663787156 73.4090982470595) bank57804 +57805 POINT(41.045765835093995 73.82349338870334) bank57805 +57806 POINT(40.526405966049396 73.40001086243583) bank57806 +57807 POINT(40.46397724898667 73.02491196253905) bank57807 +57808 POINT(40.67874301790941 74.99533942515954) bank57808 +57809 POINT(40.243782814668656 73.25057020891315) bank57809 +57810 POINT(40.37802056742224 73.0725225816142) bank57810 +57811 POINT(40.98594283457427 74.57206760849604) bank57811 +57812 POINT(40.412189446931635 73.9411462662809) bank57812 +57813 POINT(40.359021345718084 74.3773510292186) bank57813 +57814 POINT(41.12207279709285 74.84998431155861) bank57814 +57815 POINT(40.68414162786166 74.20006099535077) bank57815 +57816 POINT(41.64309251935369 74.72676031884836) bank57816 +57817 POINT(39.86166480046529 74.28217189398636) bank57817 +57818 POINT(40.16010366663484 73.21981070195903) bank57818 +57819 POINT(40.19219968005005 73.0122230322567) bank57819 +57820 POINT(39.883550788745985 74.81169051789595) bank57820 +57821 POINT(40.70168972102641 73.91231104127928) bank57821 +57822 POINT(40.575786039827044 73.34923392154349) bank57822 +57823 POINT(41.51549735732774 73.63086429338402) bank57823 +57824 POINT(40.452569281059525 74.65760722340187) bank57824 +57825 POINT(41.69020708808542 73.31977224572245) bank57825 +57826 POINT(41.5031017788201 74.43210399776383) bank57826 +57827 POINT(39.722674967700776 73.27004494463546) bank57827 +57828 POINT(40.904787502424576 74.07530687195305) bank57828 +57829 POINT(41.34877496992462 73.2642738969931) bank57829 +57830 POINT(41.26540577441657 73.27778463904488) bank57830 +57831 POINT(39.787127075881756 73.96962707225921) bank57831 +57832 POINT(41.33082453192695 73.01648406671397) bank57832 +57833 POINT(41.209046329573454 73.3175853527673) bank57833 +57834 POINT(40.429674754774084 74.05623508146668) bank57834 +57835 POINT(41.529601295069135 73.90411581632233) bank57835 +57836 POINT(39.82969843368517 74.37749092646801) bank57836 +57837 POINT(40.57662628223729 73.1546217453306) bank57837 +57838 POINT(40.861338145876104 73.15015502384438) bank57838 +57839 POINT(40.68762034349128 74.2406806148217) bank57839 +57840 POINT(41.63009187310256 74.24903048465276) bank57840 +57841 POINT(40.57912361842099 73.60148874050896) bank57841 +57842 POINT(40.118750181376996 74.07709609942968) bank57842 +57843 POINT(41.122095705885954 73.37444205533416) bank57843 +57844 POINT(40.537648162326875 74.26729863953103) bank57844 +57845 POINT(40.41098813809267 74.32950583962955) bank57845 +57846 POINT(41.601093560003754 74.64898304888875) bank57846 +57847 POINT(40.60579637812476 74.56874357789286) bank57847 +57848 POINT(39.739301174492766 73.56251228854643) bank57848 +57849 POINT(40.59381233804549 74.87955802163714) bank57849 +57850 POINT(40.102115626165684 74.32053595350578) bank57850 +57851 POINT(41.04173760586065 73.56477771656314) bank57851 +57852 POINT(41.14804718443073 73.91144402270302) bank57852 +57853 POINT(41.03792839917372 74.20808067152143) bank57853 +57854 POINT(40.55555269070463 74.86383769046144) bank57854 +57855 POINT(40.585212488800586 73.37777158825564) bank57855 +57856 POINT(41.61765758124778 74.71676488127201) bank57856 +57857 POINT(41.27762021951821 74.23657189957272) bank57857 +57858 POINT(41.10251969614925 74.43930014170441) bank57858 +57859 POINT(41.18477792005555 74.32874649471276) bank57859 +57860 POINT(40.36681694278502 73.55085693871877) bank57860 +57861 POINT(40.64035438755533 74.92739354768963) bank57861 +57862 POINT(39.725442379043365 74.2892433368289) bank57862 +57863 POINT(41.46799057875786 73.04089367786804) bank57863 +57864 POINT(40.87102248494028 73.9821677941723) bank57864 +57865 POINT(39.87929746527927 74.84089796502964) bank57865 +57866 POINT(41.39116078316488 74.43560973808204) bank57866 +57867 POINT(41.194646527169375 74.671198447054) bank57867 +57868 POINT(41.64985621612154 74.80968996197035) bank57868 +57869 POINT(40.31484383913567 73.23694273627689) bank57869 +57870 POINT(41.06276743654044 74.20048055912669) bank57870 +57871 POINT(40.11066994106455 73.50195003364234) bank57871 +57872 POINT(41.053720601269546 74.75316569674234) bank57872 +57873 POINT(39.77183060974564 73.09161240841408) bank57873 +57874 POINT(39.771539643963926 73.44174173519826) bank57874 +57875 POINT(39.878571156736086 73.18019189440147) bank57875 +57876 POINT(40.485120030215946 73.9879736427928) bank57876 +57877 POINT(41.376547985881665 74.82894057696333) bank57877 +57878 POINT(40.441919417197205 73.35643171819348) bank57878 +57879 POINT(39.783971320118944 73.688170838549) bank57879 +57880 POINT(39.740896964890815 74.16150736758922) bank57880 +57881 POINT(40.902397728301125 74.04558550788745) bank57881 +57882 POINT(40.055333606583304 73.43813129246738) bank57882 +57883 POINT(41.46471668875022 74.01950245761621) bank57883 +57884 POINT(40.037514975258816 74.07567612098573) bank57884 +57885 POINT(41.41331742406536 73.08809491320005) bank57885 +57886 POINT(41.248042457257554 74.68566220283087) bank57886 +57887 POINT(40.316845330625554 74.3076451041039) bank57887 +57888 POINT(40.115586667219944 74.66495224547802) bank57888 +57889 POINT(41.18338049983629 74.0929521795671) bank57889 +57890 POINT(40.78101649052203 73.22914546549644) bank57890 +57891 POINT(40.35164925163484 73.72087085465574) bank57891 +57892 POINT(41.56244013046015 74.75462002138948) bank57892 +57893 POINT(39.835783195408226 73.33925961594517) bank57893 +57894 POINT(40.54881455164192 73.0425713828987) bank57894 +57895 POINT(39.734687689061104 74.76501779001964) bank57895 +57896 POINT(40.18489723804254 73.05436528041665) bank57896 +57897 POINT(41.32876445693923 73.39772581906996) bank57897 +57898 POINT(40.97465864708415 73.7421099824245) bank57898 +57899 POINT(41.56761231907835 73.18032432473336) bank57899 +57900 POINT(41.32839269306826 74.09257750212504) bank57900 +57901 POINT(40.89976304297195 74.0045294862143) bank57901 +57902 POINT(40.78259693313728 73.3582785105083) bank57902 +57903 POINT(40.51064127010002 73.79042284733502) bank57903 +57904 POINT(41.09926783010395 73.28083252981526) bank57904 +57905 POINT(41.491164495441076 73.57155214514911) bank57905 +57906 POINT(41.38159342317112 74.50338636477028) bank57906 +57907 POINT(41.43718711547414 73.96119758846476) bank57907 +57908 POINT(41.40602950763876 74.637705575019) bank57908 +57909 POINT(40.88135757824516 73.99541317712763) bank57909 +57910 POINT(40.64854346381757 73.66547865546882) bank57910 +57911 POINT(41.578637630505604 73.68531123736125) bank57911 +57912 POINT(40.725385353796 74.19297113626327) bank57912 +57913 POINT(40.14800254353614 73.72573276422814) bank57913 +57914 POINT(41.14805432552538 74.55274669691796) bank57914 +57915 POINT(41.585429322063504 73.10158706018906) bank57915 +57916 POINT(41.08577136463045 73.90537253351522) bank57916 +57917 POINT(41.33644413808198 74.61951443604501) bank57917 +57918 POINT(39.740408211389685 74.43126225961281) bank57918 +57919 POINT(39.93946360383924 73.83485842890931) bank57919 +57920 POINT(40.40071253127155 74.77484356219689) bank57920 +57921 POINT(41.26278978933614 73.53481601555332) bank57921 +57922 POINT(40.18592320353767 74.6026546540336) bank57922 +57923 POINT(41.142957558128074 74.72735554843155) bank57923 +57924 POINT(40.19712913146203 74.86948780885052) bank57924 +57925 POINT(41.633799546033764 73.34773464791618) bank57925 +57926 POINT(41.17989043164463 73.33571236525266) bank57926 +57927 POINT(40.26070132378437 73.87323574047502) bank57927 +57928 POINT(41.25718884425225 73.52153101124313) bank57928 +57929 POINT(40.46157639693515 73.03674398198534) bank57929 +57930 POINT(39.838377551382706 73.27506941496762) bank57930 +57931 POINT(41.446518938355894 73.24905523036415) bank57931 +57932 POINT(40.222567923119556 73.0364088985595) bank57932 +57933 POINT(41.14732016510989 74.9066608252728) bank57933 +57934 POINT(40.97588401782541 74.85945995207246) bank57934 +57935 POINT(40.38596883050481 74.33117111235049) bank57935 +57936 POINT(40.38073844367997 73.23214274394138) bank57936 +57937 POINT(40.68515992142821 74.00553213137802) bank57937 +57938 POINT(40.70482321441713 74.3666177601772) bank57938 +57939 POINT(40.073501891790436 73.75476341803429) bank57939 +57940 POINT(40.49987556768905 74.79305677606443) bank57940 +57941 POINT(40.91293177806108 73.7764246122591) bank57941 +57942 POINT(39.74190144625654 74.95975913391666) bank57942 +57943 POINT(40.8445400428592 74.73879931988554) bank57943 +57944 POINT(40.58802973698868 73.21257903475649) bank57944 +57945 POINT(39.952198558047755 73.4187244594041) bank57945 +57946 POINT(39.904372724639316 73.30306012983355) bank57946 +57947 POINT(40.87737188335162 73.87612836897668) bank57947 +57948 POINT(41.45281955118113 74.30514220375639) bank57948 +57949 POINT(40.40604313268881 73.82722432992574) bank57949 +57950 POINT(41.257006650157685 73.649308165097) bank57950 +57951 POINT(41.38977934164329 74.06833340178882) bank57951 +57952 POINT(40.11249004867732 74.63156290709826) bank57952 +57953 POINT(39.77118392588165 74.28415142508938) bank57953 +57954 POINT(40.81488989835432 73.21678623539566) bank57954 +57955 POINT(41.62674653236149 73.37217078218998) bank57955 +57956 POINT(41.20331125275705 73.48671667430068) bank57956 +57957 POINT(40.945669266825824 73.79573205079159) bank57957 +57958 POINT(40.250748188414974 73.74779333913587) bank57958 +57959 POINT(39.74067527789367 74.01102764795512) bank57959 +57960 POINT(39.83122080136763 73.97364501523903) bank57960 +57961 POINT(39.9754028155203 74.556856509599) bank57961 +57962 POINT(40.300406718450944 74.66538556265384) bank57962 +57963 POINT(39.82122734067411 74.25326640681166) bank57963 +57964 POINT(40.99009183954763 74.61607034541713) bank57964 +57965 POINT(40.771049220361334 74.62577726418245) bank57965 +57966 POINT(40.45021241645978 73.78128056841982) bank57966 +57967 POINT(40.377457393918135 74.87587806398732) bank57967 +57968 POINT(40.73224250359889 74.81090139403656) bank57968 +57969 POINT(39.83237296985746 73.44225740425064) bank57969 +57970 POINT(39.812994111824096 74.66449432929673) bank57970 +57971 POINT(41.35201481020341 73.79108402246972) bank57971 +57972 POINT(41.005466885423125 74.83964408150743) bank57972 +57973 POINT(39.85603195665402 73.22725414585351) bank57973 +57974 POINT(40.459223117590746 73.02880088118053) bank57974 +57975 POINT(40.93356452246252 74.05773445036922) bank57975 +57976 POINT(40.140715643520394 74.02417954026207) bank57976 +57977 POINT(41.16570580385748 74.09481206137035) bank57977 +57978 POINT(40.548208147513044 74.77532180997481) bank57978 +57979 POINT(41.51539736186435 74.0454614921059) bank57979 +57980 POINT(40.90586661814952 73.30495318613578) bank57980 +57981 POINT(41.64060032735849 73.15542083333081) bank57981 +57982 POINT(40.02797018572131 73.03359776992544) bank57982 +57983 POINT(41.15452066238556 74.03263531546422) bank57983 +57984 POINT(41.173382177615196 73.93728163538123) bank57984 +57985 POINT(39.777884832327 74.00334227657885) bank57985 +57986 POINT(40.39900082191734 74.55042391467636) bank57986 +57987 POINT(41.409064343616926 73.84460465336329) bank57987 +57988 POINT(39.809533256066615 74.84984206217909) bank57988 +57989 POINT(41.611275305392475 74.26274714919015) bank57989 +57990 POINT(41.25052482545863 73.40725576035265) bank57990 +57991 POINT(39.80325799079834 74.22120245545253) bank57991 +57992 POINT(40.79068306767263 73.86947249541261) bank57992 +57993 POINT(40.56577883700186 74.86118008821914) bank57993 +57994 POINT(40.276681143804325 74.90154255236403) bank57994 +57995 POINT(41.52918086103465 74.22675234793024) bank57995 +57996 POINT(40.82384466456357 73.76513831003801) bank57996 +57997 POINT(40.64675255076369 73.57113900882787) bank57997 +57998 POINT(40.17751609956095 74.27097333372474) bank57998 +57999 POINT(41.28721305996355 73.52219337776174) bank57999 +58000 POINT(41.028840022795336 74.39405592639265) bank58000 +58001 POINT(41.298046866833396 74.63544548440858) bank58001 +58002 POINT(41.36537919442714 74.58938601532236) bank58002 +58003 POINT(41.3613452020673 73.85673000524034) bank58003 +58004 POINT(41.5155889884105 73.40197735656233) bank58004 +58005 POINT(40.513949915572944 74.64849171145191) bank58005 +58006 POINT(41.52032068331629 73.21376307855526) bank58006 +58007 POINT(40.04649103574364 73.06474985887745) bank58007 +58008 POINT(41.13210316898664 73.71988616956872) bank58008 +58009 POINT(41.35151928026473 74.79104971679811) bank58009 +58010 POINT(41.65480012722737 73.45421699746257) bank58010 +58011 POINT(40.75013821920729 74.27613380690767) bank58011 +58012 POINT(41.54894537304245 73.4140910449091) bank58012 +58013 POINT(40.604539202261876 73.5240393714556) bank58013 +58014 POINT(40.61727985953254 74.5552358684704) bank58014 +58015 POINT(40.210572027389446 74.94820858283748) bank58015 +58016 POINT(41.22866652247868 74.44610357581195) bank58016 +58017 POINT(41.135110226726916 74.93669308217139) bank58017 +58018 POINT(40.21713620541045 74.27974124494011) bank58018 +58019 POINT(41.68829978018561 74.04807036297593) bank58019 +58020 POINT(41.67660081342051 74.41509926976211) bank58020 +58021 POINT(40.74732494147322 73.09645655179494) bank58021 +58022 POINT(40.630796819011614 73.74839826778963) bank58022 +58023 POINT(40.02051473035521 73.66169373923654) bank58023 +58024 POINT(41.31836322867389 74.23850790162633) bank58024 +58025 POINT(41.098404552631195 73.23729314785096) bank58025 +58026 POINT(39.915079820413546 74.9094303861693) bank58026 +58027 POINT(40.92742273269565 74.48794714129136) bank58027 +58028 POINT(40.21286407075859 73.01030746944917) bank58028 +58029 POINT(40.08431487313501 74.08883689646555) bank58029 +58030 POINT(40.13636081128932 74.89474852748859) bank58030 +58031 POINT(39.78563832464491 74.48017576702374) bank58031 +58032 POINT(40.45240142994235 74.36997718867414) bank58032 +58033 POINT(40.31005025491459 74.96612049399984) bank58033 +58034 POINT(40.142151459813036 73.70470913947727) bank58034 +58035 POINT(40.31992001007261 74.85757214896097) bank58035 +58036 POINT(41.32405194359598 73.77380501211745) bank58036 +58037 POINT(41.13742866967565 73.66733394638173) bank58037 +58038 POINT(40.94784804606672 73.20240633886958) bank58038 +58039 POINT(41.42465291604373 73.94483890131144) bank58039 +58040 POINT(40.525789583145034 74.88005302118691) bank58040 +58041 POINT(40.101859448895084 73.2458697464907) bank58041 +58042 POINT(40.874775681017724 74.36426253437858) bank58042 +58043 POINT(41.27903400511924 73.61101487466152) bank58043 +58044 POINT(41.1421483865971 73.89874392971491) bank58044 +58045 POINT(41.36391289591141 74.12404842982404) bank58045 +58046 POINT(41.5370984680255 73.39343439843111) bank58046 +58047 POINT(40.594448450554474 74.25995315513593) bank58047 +58048 POINT(41.34251337509491 73.65127464793487) bank58048 +58049 POINT(39.76460765777088 73.33433678697197) bank58049 +58050 POINT(39.76193801299701 74.02542515969851) bank58050 +58051 POINT(40.951000590874756 73.78440738579704) bank58051 +58052 POINT(40.236202857332756 73.51600301035242) bank58052 +58053 POINT(40.12684190857688 74.08176237507355) bank58053 +58054 POINT(40.44255250664142 73.31834598208148) bank58054 +58055 POINT(40.50958844770728 73.1077683663553) bank58055 +58056 POINT(41.243654759865194 74.18004607543855) bank58056 +58057 POINT(40.52542106051561 74.06498643193828) bank58057 +58058 POINT(40.20564062492947 73.97649688432283) bank58058 +58059 POINT(39.902719522921785 74.33108900629435) bank58059 +58060 POINT(41.22119669401001 73.9614674705783) bank58060 +58061 POINT(41.682574188519524 74.88238944875847) bank58061 +58062 POINT(41.16956381081895 73.24271643977325) bank58062 +58063 POINT(41.528172798690484 73.04809682825434) bank58063 +58064 POINT(40.51145206089951 74.50978275124992) bank58064 +58065 POINT(39.77460297566179 73.76739453439792) bank58065 +58066 POINT(41.0193811892935 73.33389315272518) bank58066 +58067 POINT(40.57154407433082 73.31192748774443) bank58067 +58068 POINT(39.807092971757285 73.1255314257599) bank58068 +58069 POINT(41.5952104661574 73.71646391903234) bank58069 +58070 POINT(41.551179717672525 73.86870864606104) bank58070 +58071 POINT(39.91512706690649 74.6689346082838) bank58071 +58072 POINT(40.17947293636086 74.99884665989161) bank58072 +58073 POINT(40.50585051763734 73.15842644522273) bank58073 +58074 POINT(41.46921496948511 74.80334200557049) bank58074 +58075 POINT(40.90927249868631 73.61110729579622) bank58075 +58076 POINT(40.71445634313573 73.89513006533284) bank58076 +58077 POINT(41.65484583075666 73.52197462315065) bank58077 +58078 POINT(40.67488809119637 74.8680379956198) bank58078 +58079 POINT(39.886306333698904 73.66669126908198) bank58079 +58080 POINT(41.10403902646063 73.53981411835422) bank58080 +58081 POINT(40.602710490081826 73.05251451134151) bank58081 +58082 POINT(39.973108457996716 74.33454523617388) bank58082 +58083 POINT(40.998458099854986 73.72365711849686) bank58083 +58084 POINT(40.02859502041122 74.66135525736368) bank58084 +58085 POINT(39.92009343136325 74.07472521896852) bank58085 +58086 POINT(40.91754876409899 74.61707742086864) bank58086 +58087 POINT(40.070878044119226 74.25633101661492) bank58087 +58088 POINT(41.077632384810784 73.88076216908989) bank58088 +58089 POINT(39.84322080526694 74.37732662912653) bank58089 +58090 POINT(41.41504909508345 74.07421690334303) bank58090 +58091 POINT(41.229258286254705 73.36795545064204) bank58091 +58092 POINT(41.579820346375755 73.42513518803082) bank58092 +58093 POINT(41.4772424606058 73.51184047762808) bank58093 +58094 POINT(40.81398441181476 74.96700768788138) bank58094 +58095 POINT(41.523592212976425 73.48685397682172) bank58095 +58096 POINT(41.15630357453287 73.88553333339664) bank58096 +58097 POINT(40.260265222073635 74.11229055682071) bank58097 +58098 POINT(41.348585209441545 74.7096098443799) bank58098 +58099 POINT(40.855795810318746 73.77375791982232) bank58099 +58100 POINT(39.725486254064336 73.79595272711339) bank58100 +58101 POINT(40.9562655077687 74.40365111783458) bank58101 +58102 POINT(40.01730498264701 73.97405169684735) bank58102 +58103 POINT(41.19356139782994 73.65165103479217) bank58103 +58104 POINT(41.09808239898232 73.1287977077882) bank58104 +58105 POINT(41.259894085696544 74.47274090948983) bank58105 +58106 POINT(40.8411401679968 73.1740883273473) bank58106 +58107 POINT(40.13681093867617 73.83565191066721) bank58107 +58108 POINT(41.323126612220605 74.760518123899) bank58108 +58109 POINT(39.86511599951966 73.87190999006526) bank58109 +58110 POINT(40.22150010815097 73.12283436041258) bank58110 +58111 POINT(39.781723525732176 73.5569388889805) bank58111 +58112 POINT(40.41198965598652 74.57105912264788) bank58112 +58113 POINT(40.91038386521729 73.62029476603853) bank58113 +58114 POINT(41.34690234791362 74.532029234582) bank58114 +58115 POINT(41.37233326002628 74.35067744546997) bank58115 +58116 POINT(40.93399208276117 74.34708518041734) bank58116 +58117 POINT(40.62116462280804 74.00676324891208) bank58117 +58118 POINT(40.43275767630728 74.97028895473267) bank58118 +58119 POINT(40.112042984049815 73.66939584023378) bank58119 +58120 POINT(40.801963566489725 73.76983600101477) bank58120 +58121 POINT(40.36161465922582 73.437981805868) bank58121 +58122 POINT(40.21098040882598 74.6327938272924) bank58122 +58123 POINT(40.43847008344963 74.86146203467166) bank58123 +58124 POINT(40.6217436431895 74.69001872417401) bank58124 +58125 POINT(41.213027616723565 73.9223417730791) bank58125 +58126 POINT(41.63802013697179 73.97676173195245) bank58126 +58127 POINT(39.86820757925085 74.41301071962664) bank58127 +58128 POINT(40.72760752957922 73.36198652444801) bank58128 +58129 POINT(40.172861828747976 73.09234978497173) bank58129 +58130 POINT(41.441296051343286 74.38042425924543) bank58130 +58131 POINT(41.10559216053212 74.2648626469059) bank58131 +58132 POINT(41.38270957291376 73.56430908208524) bank58132 +58133 POINT(40.279335335600145 73.91392637557773) bank58133 +58134 POINT(40.511710139933335 73.61054021998477) bank58134 +58135 POINT(40.35390248439698 73.83861259967905) bank58135 +58136 POINT(41.43858396658619 74.82577657521537) bank58136 +58137 POINT(41.02773650416773 74.40160344873811) bank58137 +58138 POINT(40.40420045588937 74.30675291078796) bank58138 +58139 POINT(41.07245589594955 73.09925556896853) bank58139 +58140 POINT(40.620275281259765 73.47076233885662) bank58140 +58141 POINT(41.6245962889968 73.03926809836832) bank58141 +58142 POINT(40.6807949785094 73.7930528784616) bank58142 +58143 POINT(41.58295377159835 73.47718377525666) bank58143 +58144 POINT(40.442033526412786 73.4456573169408) bank58144 +58145 POINT(40.95318970425914 74.03404916910662) bank58145 +58146 POINT(41.4594287279999 73.84225253485721) bank58146 +58147 POINT(41.14578301632791 74.83261421613611) bank58147 +58148 POINT(39.733523558363174 73.1550759804861) bank58148 +58149 POINT(39.91216268207742 74.77554101493092) bank58149 +58150 POINT(41.10850584847076 73.40575593670383) bank58150 +58151 POINT(40.3685348938897 74.32370086401211) bank58151 +58152 POINT(41.213320551021866 74.7877873123808) bank58152 +58153 POINT(40.7949865822811 74.22132892653997) bank58153 +58154 POINT(40.374760994184975 73.7940486897341) bank58154 +58155 POINT(40.803054946567066 73.84871424978037) bank58155 +58156 POINT(40.28034684869939 73.2777101773287) bank58156 +58157 POINT(40.43169696077697 74.86520310459535) bank58157 +58158 POINT(39.84995809912078 74.58300489746448) bank58158 +58159 POINT(40.74354023647374 74.86845964922631) bank58159 +58160 POINT(41.218267997702945 74.50716561158539) bank58160 +58161 POINT(41.62966688994147 74.02850609603357) bank58161 +58162 POINT(40.135070709328296 74.518245161364) bank58162 +58163 POINT(41.51649924178361 75.00317959019925) bank58163 +58164 POINT(40.35759941033919 74.45944384398697) bank58164 +58165 POINT(40.03497362332517 74.35558561465444) bank58165 +58166 POINT(39.915219395836324 74.0257590264028) bank58166 +58167 POINT(40.80968887573936 73.83770528320798) bank58167 +58168 POINT(41.244930600007834 74.58647633058538) bank58168 +58169 POINT(41.172809847453465 74.64383298681769) bank58169 +58170 POINT(40.488690263044674 73.1006154132938) bank58170 +58171 POINT(41.110814793858665 73.78444401774433) bank58171 +58172 POINT(41.04771148713232 74.12576402673888) bank58172 +58173 POINT(39.825216969043765 73.38030848186666) bank58173 +58174 POINT(41.25095899965169 73.90463309217982) bank58174 +58175 POINT(41.709614288815686 73.33187302880188) bank58175 +58176 POINT(39.94396193643716 73.45581354334357) bank58176 +58177 POINT(41.477391408065074 74.33037871314689) bank58177 +58178 POINT(41.39846128583963 74.38561939270129) bank58178 +58179 POINT(41.093302677239414 73.32205501895322) bank58179 +58180 POINT(41.07137615172148 74.81145131423861) bank58180 +58181 POINT(40.098988085957345 74.47613252232213) bank58181 +58182 POINT(40.29998730080614 73.7622777052652) bank58182 +58183 POINT(39.990789941588865 73.60489101395567) bank58183 +58184 POINT(40.09419877523536 74.20546156575232) bank58184 +58185 POINT(40.11083040232091 73.80120429397215) bank58185 +58186 POINT(40.20130855609875 73.07238693532886) bank58186 +58187 POINT(40.09601794048043 74.86146872819418) bank58187 +58188 POINT(39.77507030380063 74.42066992307332) bank58188 +58189 POINT(40.75766905129829 73.62456343902596) bank58189 +58190 POINT(40.72315343770174 74.02861920917236) bank58190 +58191 POINT(40.44214642560991 74.91959332711366) bank58191 +58192 POINT(40.54965338979359 74.3093498818251) bank58192 +58193 POINT(40.89802551992372 74.8829771247944) bank58193 +58194 POINT(41.457878710760845 73.49917854043316) bank58194 +58195 POINT(39.71558314928884 74.38063155743511) bank58195 +58196 POINT(39.78317255932391 73.69136009410607) bank58196 +58197 POINT(40.92145847307654 73.94950707733629) bank58197 +58198 POINT(41.186353990726666 73.53526862308411) bank58198 +58199 POINT(40.024806740260566 74.89625429525746) bank58199 +58200 POINT(41.58478585608775 74.38764660886599) bank58200 +58201 POINT(40.644827030353795 73.64099923421603) bank58201 +58202 POINT(41.5287667975306 73.66686659224337) bank58202 +58203 POINT(40.45928161013556 73.90492418476455) bank58203 +58204 POINT(41.33291496355478 74.95538445466423) bank58204 +58205 POINT(39.90436750147235 74.83321331367468) bank58205 +58206 POINT(40.01360108994549 74.99738227241224) bank58206 +58207 POINT(41.59766093663557 74.73648948436959) bank58207 +58208 POINT(40.71670622866124 74.19639722274138) bank58208 +58209 POINT(41.27143046640406 74.1869410738282) bank58209 +58210 POINT(41.07052904339815 74.12371781479145) bank58210 +58211 POINT(41.17565098953011 74.67119512460646) bank58211 +58212 POINT(41.34134332170035 74.17132659046908) bank58212 +58213 POINT(40.27848509327295 74.39230713535892) bank58213 +58214 POINT(40.01592931943899 74.92024050424757) bank58214 +58215 POINT(41.21946928493181 74.24231350352757) bank58215 +58216 POINT(40.60252467960352 73.87793745564646) bank58216 +58217 POINT(39.939931597386085 74.256738508705) bank58217 +58218 POINT(40.98019289904248 74.00184622251933) bank58218 +58219 POINT(41.323594706509404 73.10724103292138) bank58219 +58220 POINT(40.804260091962824 73.76801472180809) bank58220 +58221 POINT(39.8098325617473 74.053820737587) bank58221 +58222 POINT(39.71438696425875 73.25497501718387) bank58222 +58223 POINT(40.371468561075254 73.41889695158402) bank58223 +58224 POINT(40.48070130672668 73.98393149008677) bank58224 +58225 POINT(40.96241096314943 74.30641478845222) bank58225 +58226 POINT(40.92836664830144 74.27770422928558) bank58226 +58227 POINT(40.66718937543503 74.33340528649768) bank58227 +58228 POINT(40.23652532717072 73.42506310995228) bank58228 +58229 POINT(40.98433376267202 74.36794408342415) bank58229 +58230 POINT(39.94019825012038 73.27375466380612) bank58230 +58231 POINT(41.32390985801618 73.07576122642405) bank58231 +58232 POINT(40.03875458240911 74.7552625388976) bank58232 +58233 POINT(40.209823326071685 73.5917853413173) bank58233 +58234 POINT(41.53562336785054 73.08255528368457) bank58234 +58235 POINT(40.39543152235452 73.87214055694137) bank58235 +58236 POINT(39.96267932040207 73.60050884894986) bank58236 +58237 POINT(41.111168022374756 73.93695276286523) bank58237 +58238 POINT(41.046264007892376 73.40863844770465) bank58238 +58239 POINT(40.57735764338362 74.049102634249) bank58239 +58240 POINT(41.704077339275976 73.346333655876) bank58240 +58241 POINT(40.33301997230116 73.79725894146516) bank58241 +58242 POINT(41.58969627770901 74.56150579562538) bank58242 +58243 POINT(41.492012201269034 73.6227813880131) bank58243 +58244 POINT(41.21115150830115 73.21329772521057) bank58244 +58245 POINT(40.956056830646915 73.12796720251035) bank58245 +58246 POINT(39.913733568689274 74.17552625798585) bank58246 +58247 POINT(40.965203866069274 73.33680938788893) bank58247 +58248 POINT(40.64221369019014 74.94129801254788) bank58248 +58249 POINT(41.54015682737806 73.60813398710133) bank58249 +58250 POINT(41.476764896153284 74.17985174638277) bank58250 +58251 POINT(41.374938920651594 73.14937078893092) bank58251 +58252 POINT(40.12715821954531 74.00122225677804) bank58252 +58253 POINT(41.06101806574463 73.40675818628156) bank58253 +58254 POINT(41.23028083817583 74.27649424195542) bank58254 +58255 POINT(40.9188202126613 73.27492684992345) bank58255 +58256 POINT(40.05873941833613 73.78590635302429) bank58256 +58257 POINT(40.260931219729045 73.14083786432907) bank58257 +58258 POINT(41.49739961107133 73.89949741946698) bank58258 +58259 POINT(40.57503267848447 74.08691683181047) bank58259 +58260 POINT(40.25307062817022 73.79932216982168) bank58260 +58261 POINT(40.13678756108505 74.05913913164657) bank58261 +58262 POINT(40.84621303453509 74.25442902734251) bank58262 +58263 POINT(40.38303754038255 73.7375464512267) bank58263 +58264 POINT(41.2027547585029 73.77680079764347) bank58264 +58265 POINT(39.87036995113471 73.88569916924614) bank58265 +58266 POINT(39.72339670532713 73.16168239364069) bank58266 +58267 POINT(40.559204629055394 73.11117023955634) bank58267 +58268 POINT(41.56420893481467 73.45637624763194) bank58268 +58269 POINT(40.686742876674806 74.40539174006062) bank58269 +58270 POINT(40.55121573310555 73.33517204427574) bank58270 +58271 POINT(40.71995556107278 74.82111939344696) bank58271 +58272 POINT(41.232306887593786 74.90226742557047) bank58272 +58273 POINT(41.66303466564955 74.10898413821239) bank58273 +58274 POINT(40.639677222050075 73.34786318344811) bank58274 +58275 POINT(40.45364684080936 73.64337364494666) bank58275 +58276 POINT(40.38145526357419 74.97587601990949) bank58276 +58277 POINT(39.938518879947964 73.95032032706298) bank58277 +58278 POINT(41.46522544334383 73.6771557763366) bank58278 +58279 POINT(39.746890627114304 73.2319591823446) bank58279 +58280 POINT(41.68668950116573 73.40995816769714) bank58280 +58281 POINT(40.24554746541099 73.97693171727335) bank58281 +58282 POINT(41.49872673334585 73.75326233361052) bank58282 +58283 POINT(41.26446437275705 73.33083700608529) bank58283 +58284 POINT(39.85521507847466 73.05834117399033) bank58284 +58285 POINT(41.25704785991064 74.15279133920825) bank58285 +58286 POINT(40.65573381794907 74.741832998928) bank58286 +58287 POINT(40.4231502225835 74.88975363341737) bank58287 +58288 POINT(40.34577322472997 74.87259956908485) bank58288 +58289 POINT(40.33636533228303 74.68638691771218) bank58289 +58290 POINT(39.855261132788996 74.42538176673051) bank58290 +58291 POINT(41.62075119494009 73.53577953484778) bank58291 +58292 POINT(40.9125472295621 73.78449420723263) bank58292 +58293 POINT(40.636038794580735 73.23043030838595) bank58293 +58294 POINT(41.275729942767235 73.54399716411413) bank58294 +58295 POINT(39.93184351758085 74.98225194413708) bank58295 +58296 POINT(41.046878455635245 74.09141590858589) bank58296 +58297 POINT(40.744299397819965 74.04077673761813) bank58297 +58298 POINT(41.622484345128065 73.92392964267711) bank58298 +58299 POINT(41.45728062684132 74.4664326718912) bank58299 +58300 POINT(40.51835916826967 74.41986269367598) bank58300 +58301 POINT(41.59519978858937 74.9140126294432) bank58301 +58302 POINT(40.79452593790193 74.89462725164873) bank58302 +58303 POINT(40.87084338114389 74.34121502143871) bank58303 +58304 POINT(41.630559618095496 73.2337443844866) bank58304 +58305 POINT(41.277027313296976 73.27085034551335) bank58305 +58306 POINT(40.283824969660955 73.3552589838122) bank58306 +58307 POINT(39.894741789837305 73.10948459078172) bank58307 +58308 POINT(41.67064863234142 73.56771726135005) bank58308 +58309 POINT(39.80520324047691 73.54540798300324) bank58309 +58310 POINT(41.265686715871766 73.49805643825279) bank58310 +58311 POINT(39.88340484648683 74.34396084895502) bank58311 +58312 POINT(40.97494608049159 74.4028247128136) bank58312 +58313 POINT(41.61298913840565 73.95105808399829) bank58313 +58314 POINT(41.27569076124665 73.41813262146744) bank58314 +58315 POINT(40.259347525702104 74.74767338633217) bank58315 +58316 POINT(40.396478979644314 74.27668536404686) bank58316 +58317 POINT(40.68315747642372 74.63672412350549) bank58317 +58318 POINT(40.22212303705245 74.02884155483532) bank58318 +58319 POINT(40.19452963393993 74.98983238028661) bank58319 +58320 POINT(41.53743763020778 74.88073716192848) bank58320 +58321 POINT(41.076585531567034 73.5879723869561) bank58321 +58322 POINT(40.55189768423693 74.67794760283518) bank58322 +58323 POINT(41.662177570405106 74.898940734772) bank58323 +58324 POINT(41.52755626924375 74.90921729644029) bank58324 +58325 POINT(40.548284483130956 74.76485539913308) bank58325 +58326 POINT(40.323107090680566 74.732372365739) bank58326 +58327 POINT(40.490448902713744 74.13618028989538) bank58327 +58328 POINT(41.10137585197688 73.0590756686735) bank58328 +58329 POINT(41.47644103708982 74.97568440900335) bank58329 +58330 POINT(40.27510946468658 73.23098705302945) bank58330 +58331 POINT(41.62450018016457 73.50280064358513) bank58331 +58332 POINT(40.484038836491095 73.36255076367354) bank58332 +58333 POINT(41.51304027625204 74.94229721066907) bank58333 +58334 POINT(40.70273825838283 74.41848836932479) bank58334 +58335 POINT(41.629490267635425 73.76642253768368) bank58335 +58336 POINT(40.663554153620105 74.7357737454141) bank58336 +58337 POINT(39.81742488204927 74.2843625365834) bank58337 +58338 POINT(41.19382594247146 73.19100563421361) bank58338 +58339 POINT(41.059380337734105 73.29411025046981) bank58339 +58340 POINT(40.80455745164981 74.26127798904507) bank58340 +58341 POINT(41.57440233205021 73.5764720308643) bank58341 +58342 POINT(40.32739295725463 74.25100698370001) bank58342 +58343 POINT(39.916134878683586 74.1983174181086) bank58343 +58344 POINT(41.04139154380211 74.32612632877303) bank58344 +58345 POINT(40.0157172376324 74.03489029057981) bank58345 +58346 POINT(40.08515775372503 74.35654382477131) bank58346 +58347 POINT(40.78315548329978 73.52599911711324) bank58347 +58348 POINT(41.219777702046976 73.70717639304243) bank58348 +58349 POINT(40.24810360350432 73.99524274463406) bank58349 +58350 POINT(41.23821448608691 74.89061469469381) bank58350 +58351 POINT(41.472302418451335 73.71966315383783) bank58351 +58352 POINT(41.519966523006104 74.4960294454948) bank58352 +58353 POINT(41.0158211901466 73.36280949408497) bank58353 +58354 POINT(41.40230798564827 74.05636711515703) bank58354 +58355 POINT(39.972905963777436 74.49292323430794) bank58355 +58356 POINT(41.35950781309634 73.86483014410022) bank58356 +58357 POINT(41.56222842647644 73.32805084847286) bank58357 +58358 POINT(41.34995525431347 74.4479614322739) bank58358 +58359 POINT(41.14593889782294 74.86391113805458) bank58359 +58360 POINT(40.75430522046011 74.81098320190817) bank58360 +58361 POINT(40.63245542287416 73.99958521132224) bank58361 +58362 POINT(39.97046838570658 74.07365884655937) bank58362 +58363 POINT(40.015245151967676 74.0640951136168) bank58363 +58364 POINT(40.587436587528835 74.05549092718692) bank58364 +58365 POINT(41.71201187302722 74.85611192781366) bank58365 +58366 POINT(41.22192888755927 74.06617207810741) bank58366 +58367 POINT(40.512799856825296 73.50355594232174) bank58367 +58368 POINT(39.78151647246248 73.435207089717) bank58368 +58369 POINT(40.44991535905748 74.09549292459943) bank58369 +58370 POINT(40.31616074498389 73.57411917654716) bank58370 +58371 POINT(39.9393754440646 73.29035444918549) bank58371 +58372 POINT(39.988676474689356 74.65983790064145) bank58372 +58373 POINT(40.00354682771137 73.50252395270738) bank58373 +58374 POINT(40.03098830711963 74.13268174210577) bank58374 +58375 POINT(40.657018676646395 73.0325535613166) bank58375 +58376 POINT(41.04213601234848 73.03167382009963) bank58376 +58377 POINT(40.88024051662358 74.55663904272052) bank58377 +58378 POINT(40.69645252776645 74.86193038145717) bank58378 +58379 POINT(41.069840263750606 73.90202965615016) bank58379 +58380 POINT(40.04545862528227 73.14159132077927) bank58380 +58381 POINT(40.51768522991105 73.28192524143716) bank58381 +58382 POINT(39.94792888341188 74.07757757597109) bank58382 +58383 POINT(41.16698900128679 73.5932530458251) bank58383 +58384 POINT(40.33805742182396 73.83442943279164) bank58384 +58385 POINT(40.655786099696726 73.3321051813329) bank58385 +58386 POINT(41.62806667990922 74.36085118391381) bank58386 +58387 POINT(40.76130938318897 74.08766054518716) bank58387 +58388 POINT(41.27681113889611 74.0805926818315) bank58388 +58389 POINT(40.95832289028554 73.92694625514882) bank58389 +58390 POINT(41.19131308633174 73.84786582850455) bank58390 +58391 POINT(40.20693425270462 74.62039955082507) bank58391 +58392 POINT(41.30028044421741 74.12193917412868) bank58392 +58393 POINT(41.277628499960336 73.15758483725553) bank58393 +58394 POINT(41.59686156887671 74.90666269498692) bank58394 +58395 POINT(41.4326826230707 74.46177390258347) bank58395 +58396 POINT(40.62763911731834 73.55558725858621) bank58396 +58397 POINT(41.082820562181304 73.36946369673524) bank58397 +58398 POINT(40.89904496044408 73.41796998310537) bank58398 +58399 POINT(40.26162419074081 74.36851952223626) bank58399 +58400 POINT(40.8366022951008 74.18520188126375) bank58400 +58401 POINT(40.9176709883189 73.22181380517229) bank58401 +58402 POINT(40.063386574494615 74.16843916874835) bank58402 +58403 POINT(40.97158912319006 74.31087553083324) bank58403 +58404 POINT(40.0887376768892 74.35329421067044) bank58404 +58405 POINT(40.051342247152334 73.01199688105855) bank58405 +58406 POINT(41.653398829907985 74.36749792429372) bank58406 +58407 POINT(39.95598287959833 74.67913161068299) bank58407 +58408 POINT(40.92070446937286 73.48650813190123) bank58408 +58409 POINT(40.63079201853681 73.25884339027652) bank58409 +58410 POINT(40.35302541201479 74.60433108139434) bank58410 +58411 POINT(41.636754801563235 73.68551367503649) bank58411 +58412 POINT(40.44326862039143 73.27923616361765) bank58412 +58413 POINT(40.3152783525339 74.71256401150873) bank58413 +58414 POINT(41.45963997933159 73.70600952932068) bank58414 +58415 POINT(41.2397148147152 74.85973949736103) bank58415 +58416 POINT(39.88347336099181 74.51532423176953) bank58416 +58417 POINT(41.6165588803787 74.65326914843276) bank58417 +58418 POINT(40.767432460870076 74.53103725347573) bank58418 +58419 POINT(40.01049413034547 74.94256158228411) bank58419 +58420 POINT(39.839571218075626 74.50080127177185) bank58420 +58421 POINT(40.70539929406463 74.87110484119279) bank58421 +58422 POINT(40.00414518119982 73.39697091503544) bank58422 +58423 POINT(41.23632761190325 73.52307498271091) bank58423 +58424 POINT(41.40478546498585 73.77427869976509) bank58424 +58425 POINT(40.569640003105796 74.5888342656458) bank58425 +58426 POINT(40.43106894939972 74.333395173582) bank58426 +58427 POINT(41.660264381571444 73.44037003400655) bank58427 +58428 POINT(39.84930181321732 74.18769917960844) bank58428 +58429 POINT(41.04111284060006 74.83118841364703) bank58429 +58430 POINT(41.53436793300945 73.18340403385203) bank58430 +58431 POINT(40.09528419988716 74.94020311811191) bank58431 +58432 POINT(41.29193850631957 74.98172045255393) bank58432 +58433 POINT(39.717844760787635 73.30888133226011) bank58433 +58434 POINT(41.3257409207178 74.98407770865266) bank58434 +58435 POINT(41.02092259392086 74.16620168650485) bank58435 +58436 POINT(40.103599120404766 73.4829007248214) bank58436 +58437 POINT(40.42125766064994 73.99485880624789) bank58437 +58438 POINT(40.022736724210695 73.41263594214253) bank58438 +58439 POINT(40.66832485716678 73.24484134536807) bank58439 +58440 POINT(40.87359451591253 73.97965413658923) bank58440 +58441 POINT(41.09486052086289 73.61381821212187) bank58441 +58442 POINT(41.36674743919424 73.81799673548424) bank58442 +58443 POINT(40.50051028515628 74.8508668570711) bank58443 +58444 POINT(41.04977922362515 74.2151148870561) bank58444 +58445 POINT(41.45014190072351 73.81467009769406) bank58445 +58446 POINT(39.97625844957752 73.49210290824887) bank58446 +58447 POINT(39.97379919826227 74.6731117956237) bank58447 +58448 POINT(40.783271808039046 73.1760307022228) bank58448 +58449 POINT(40.82775033747044 74.75819023534154) bank58449 +58450 POINT(40.23037910745019 73.04050130468067) bank58450 +58451 POINT(40.22595023045266 73.38444640608158) bank58451 +58452 POINT(40.30384960425461 73.26271898448692) bank58452 +58453 POINT(39.81524958792925 73.7627065097602) bank58453 +58454 POINT(40.72310659448238 74.36510694185026) bank58454 +58455 POINT(40.31293913149183 73.65326302485427) bank58455 +58456 POINT(39.95800510693619 74.33830772667154) bank58456 +58457 POINT(40.540871476238316 73.01730723829354) bank58457 +58458 POINT(41.63531516029005 74.52453021045997) bank58458 +58459 POINT(41.5307775968325 73.64344040831634) bank58459 +58460 POINT(40.07698499028649 73.5194652122284) bank58460 +58461 POINT(41.257474145981114 74.04692813662942) bank58461 +58462 POINT(39.94906571747005 73.07814714619457) bank58462 +58463 POINT(40.00922543821522 74.98517387614712) bank58463 +58464 POINT(40.66957547987882 74.62340587139697) bank58464 +58465 POINT(39.890732297962266 73.7614568290865) bank58465 +58466 POINT(40.501796003219916 73.40285375399515) bank58466 +58467 POINT(40.577780221846425 74.46516092446619) bank58467 +58468 POINT(40.210750804673786 73.71273488333134) bank58468 +58469 POINT(40.34048418345265 73.04130643195568) bank58469 +58470 POINT(40.26648514688691 73.51577726977916) bank58470 +58471 POINT(41.46732738650018 73.74806383374008) bank58471 +58472 POINT(41.04144299625296 74.07002992756404) bank58472 +58473 POINT(39.843846648139746 74.62402487948339) bank58473 +58474 POINT(41.273506114102155 73.22926602390478) bank58474 +58475 POINT(41.04843600437398 74.65339231956273) bank58475 +58476 POINT(39.98143742847727 73.79455091367741) bank58476 +58477 POINT(40.443628775214925 74.32302769833258) bank58477 +58478 POINT(40.059762511351444 74.48558666642683) bank58478 +58479 POINT(40.6121020973762 73.272643619932) bank58479 +58480 POINT(40.87971551235508 74.93135320842039) bank58480 +58481 POINT(40.76235020066886 73.06268572869536) bank58481 +58482 POINT(40.219171288258046 74.26834214102455) bank58482 +58483 POINT(40.636283856535 74.21740372892982) bank58483 +58484 POINT(40.86123867733717 73.56621352390808) bank58484 +58485 POINT(39.955406521461946 73.07128858880274) bank58485 +58486 POINT(41.05424494998292 73.96277980281508) bank58486 +58487 POINT(41.14769610754305 74.88138579865506) bank58487 +58488 POINT(41.164530579174425 73.45821541785602) bank58488 +58489 POINT(40.75467001520911 74.85108645322983) bank58489 +58490 POINT(41.65063280996372 73.57449120820853) bank58490 +58491 POINT(39.99700257564415 73.45790935998791) bank58491 +58492 POINT(40.47375135904807 73.30377414688776) bank58492 +58493 POINT(40.437982594351844 74.00379278123948) bank58493 +58494 POINT(40.58984928473176 74.48088490744513) bank58494 +58495 POINT(41.14903239960089 74.58449853556255) bank58495 +58496 POINT(40.484814681398575 74.50835584653166) bank58496 +58497 POINT(40.850538758972746 74.49920349943811) bank58497 +58498 POINT(41.67778220739276 74.96808599561969) bank58498 +58499 POINT(41.20910194367223 74.5540956405548) bank58499 +58500 POINT(40.140697238093814 74.5726921174929) bank58500 +58501 POINT(40.4567416659194 73.88118009226383) bank58501 +58502 POINT(41.506653854321634 74.31240663074578) bank58502 +58503 POINT(40.55703788297187 74.94645725756843) bank58503 +58504 POINT(41.29765866573516 74.16256926635576) bank58504 +58505 POINT(40.7129639711232 74.91934414526091) bank58505 +58506 POINT(40.14315125175061 74.89355042592476) bank58506 +58507 POINT(41.466415609875426 74.99702134899987) bank58507 +58508 POINT(40.31412234769023 74.96633833812338) bank58508 +58509 POINT(40.02411159296581 74.38727259303478) bank58509 +58510 POINT(41.24264025089223 73.90540826797692) bank58510 +58511 POINT(41.016668805949735 73.0425056611108) bank58511 +58512 POINT(41.37997930496919 74.92708291376901) bank58512 +58513 POINT(41.4673769084362 74.68564619550108) bank58513 +58514 POINT(40.5341164471037 73.89666931247726) bank58514 +58515 POINT(41.287103962737596 73.57669154784139) bank58515 +58516 POINT(39.759673253312066 73.0464947256574) bank58516 +58517 POINT(40.72613325043892 74.51105707870593) bank58517 +58518 POINT(41.35599790494479 74.73655960942915) bank58518 +58519 POINT(39.969757240638465 73.74841055419554) bank58519 +58520 POINT(41.294843150120975 74.22115366045465) bank58520 +58521 POINT(41.608725798560464 73.61945488816454) bank58521 +58522 POINT(40.03057275908414 73.78588211120484) bank58522 +58523 POINT(41.08307951265439 74.86173108090031) bank58523 +58524 POINT(40.74698331087061 73.33370831726563) bank58524 +58525 POINT(41.63196144619276 73.3429317002105) bank58525 +58526 POINT(40.09660685511782 73.11228325011874) bank58526 +58527 POINT(41.238522424385074 74.28235062211606) bank58527 +58528 POINT(40.72036558367563 74.54871871985627) bank58528 +58529 POINT(40.88567634256183 73.22854876413967) bank58529 +58530 POINT(40.001125177233064 74.31298130912472) bank58530 +58531 POINT(40.83402312908619 74.7773096770617) bank58531 +58532 POINT(40.0396727252559 74.3587460872891) bank58532 +58533 POINT(39.778204766951035 74.323838846146) bank58533 +58534 POINT(41.43838343770051 74.43327681834717) bank58534 +58535 POINT(40.12071678205761 73.72506802315746) bank58535 +58536 POINT(41.60327390640304 74.3728424477512) bank58536 +58537 POINT(41.375804137879804 73.37501631505106) bank58537 +58538 POINT(40.175248040375216 75.00361375028673) bank58538 +58539 POINT(40.34821366295801 74.53187189784113) bank58539 +58540 POINT(40.845084885065894 73.41229181160213) bank58540 +58541 POINT(40.17724584973 74.69139757947202) bank58541 +58542 POINT(40.013144985986315 74.48756230758174) bank58542 +58543 POINT(41.25211365421638 73.59243302063126) bank58543 +58544 POINT(41.071487822019705 73.19167612646869) bank58544 +58545 POINT(40.12166077492704 73.83086111744099) bank58545 +58546 POINT(39.78320106593581 73.00820775451058) bank58546 +58547 POINT(40.30863172180228 74.18592559132844) bank58547 +58548 POINT(40.34013164620821 73.0355133168686) bank58548 +58549 POINT(41.39395659932452 73.52814279385615) bank58549 +58550 POINT(41.239436032945186 73.64800160757505) bank58550 +58551 POINT(40.62235847971819 74.82048256838159) bank58551 +58552 POINT(41.44068295616374 73.23688571562585) bank58552 +58553 POINT(40.70429993491785 73.51599714917023) bank58553 +58554 POINT(41.49379700126352 73.08843419408788) bank58554 +58555 POINT(40.51903733206484 73.55623297198959) bank58555 +58556 POINT(40.241477155606894 74.26176832311052) bank58556 +58557 POINT(41.41714532479231 73.18330268740019) bank58557 +58558 POINT(40.614662509155124 74.20984436289878) bank58558 +58559 POINT(41.25832786214403 74.92629532411998) bank58559 +58560 POINT(40.01740147168068 74.69214731716339) bank58560 +58561 POINT(40.91074573891529 73.28341039143612) bank58561 +58562 POINT(40.12892957438538 74.04422481230417) bank58562 +58563 POINT(39.94190527598127 73.55671992686992) bank58563 +58564 POINT(40.72411435594294 73.51176694934522) bank58564 +58565 POINT(40.287803036085265 73.56139463510613) bank58565 +58566 POINT(41.407897202446264 74.37080046498686) bank58566 +58567 POINT(40.628924670486896 74.48525731603898) bank58567 +58568 POINT(41.16705516460824 74.8150042588302) bank58568 +58569 POINT(39.775239306882106 73.69499092218201) bank58569 +58570 POINT(39.8887543624346 74.85292879269947) bank58570 +58571 POINT(40.404361708669505 74.37456320871068) bank58571 +58572 POINT(40.40486928446697 74.24833936004403) bank58572 +58573 POINT(41.2123050546928 73.46948348872259) bank58573 +58574 POINT(41.08830294198375 73.74930442329524) bank58574 +58575 POINT(41.36682181496861 74.42572216334557) bank58575 +58576 POINT(40.495797125283076 73.95660736154763) bank58576 +58577 POINT(40.39601258694575 74.35632628259994) bank58577 +58578 POINT(41.03549139755239 73.16286172612867) bank58578 +58579 POINT(40.50178632944181 73.45515564289225) bank58579 +58580 POINT(40.91630705613006 73.46428950573241) bank58580 +58581 POINT(39.83346365369554 73.09025518952417) bank58581 +58582 POINT(41.63937945935117 74.69983262910795) bank58582 +58583 POINT(39.87964239285176 74.60189456326316) bank58583 +58584 POINT(40.86878716869898 74.64469827185016) bank58584 +58585 POINT(39.97763163798256 73.85008981368463) bank58585 +58586 POINT(39.775456058999126 74.48909527116605) bank58586 +58587 POINT(41.591759306131245 73.45309114023519) bank58587 +58588 POINT(40.50763393518882 73.12962538017003) bank58588 +58589 POINT(40.65671627503162 74.82273447346735) bank58589 +58590 POINT(41.04552914284652 73.62452539562675) bank58590 +58591 POINT(40.78365692771753 73.45502339360866) bank58591 +58592 POINT(40.85638367333403 74.85608763200993) bank58592 +58593 POINT(40.232222525337036 74.54219833969145) bank58593 +58594 POINT(41.42872534207947 73.86245226963068) bank58594 +58595 POINT(40.4163261338791 73.01152195957884) bank58595 +58596 POINT(40.025410202309985 73.62155800224477) bank58596 +58597 POINT(40.24501670730213 74.0204355046549) bank58597 +58598 POINT(41.045307162996174 73.16884346650849) bank58598 +58599 POINT(40.1305582409879 73.78854303145674) bank58599 +58600 POINT(40.125547763125034 73.20483577323012) bank58600 +58601 POINT(40.675237854487484 73.2159359796674) bank58601 +58602 POINT(40.72595641505973 74.93414660646096) bank58602 +58603 POINT(41.090476616190585 74.0684144866944) bank58603 +58604 POINT(40.07862846178211 73.13152959380851) bank58604 +58605 POINT(40.22552582081316 73.00909343694173) bank58605 +58606 POINT(41.35811293564775 73.54920489529748) bank58606 +58607 POINT(40.55772106996814 73.24927679916924) bank58607 +58608 POINT(40.385211769591876 73.0303761669048) bank58608 +58609 POINT(39.96372048365799 73.51613998511466) bank58609 +58610 POINT(41.332241404879994 74.85852287078193) bank58610 +58611 POINT(40.59402342744362 73.92899225822528) bank58611 +58612 POINT(40.69883067366976 74.54129737844724) bank58612 +58613 POINT(41.16323545592334 73.1259174636408) bank58613 +58614 POINT(39.843118733355695 73.49703050649542) bank58614 +58615 POINT(39.92083017549521 74.50490274930411) bank58615 +58616 POINT(40.7185272034491 74.63951092075418) bank58616 +58617 POINT(41.197941530910086 74.15389231957346) bank58617 +58618 POINT(40.67770424400014 73.96561013947188) bank58618 +58619 POINT(41.318139924805166 74.41360682648178) bank58619 +58620 POINT(40.90828265859872 74.90931334646298) bank58620 +58621 POINT(40.9670524936784 73.83485904386517) bank58621 +58622 POINT(41.3701323264999 73.94839044839418) bank58622 +58623 POINT(41.582402146926924 73.9068237089612) bank58623 +58624 POINT(40.99028797059691 74.18850200966027) bank58624 +58625 POINT(40.144222254172526 73.59833008658354) bank58625 +58626 POINT(40.30205936692489 73.91123737986757) bank58626 +58627 POINT(41.59435971575168 73.69250275254184) bank58627 +58628 POINT(41.1505655696354 74.66832973974829) bank58628 +58629 POINT(41.574400985983125 74.69580638021218) bank58629 +58630 POINT(40.93227827682004 74.04327675897024) bank58630 +58631 POINT(41.604069953616005 73.1246998924546) bank58631 +58632 POINT(40.82765915019633 73.12158054462267) bank58632 +58633 POINT(40.349312772281756 74.19434907659578) bank58633 +58634 POINT(39.94430225235833 74.45044685417766) bank58634 +58635 POINT(40.2209885394208 74.80697981378722) bank58635 +58636 POINT(40.39935443367907 74.80125280564178) bank58636 +58637 POINT(41.09495841016699 73.88808377103715) bank58637 +58638 POINT(41.044451872509526 74.94548908208768) bank58638 +58639 POINT(39.95528409698878 74.93533701280012) bank58639 +58640 POINT(41.248956656167366 74.93528861501485) bank58640 +58641 POINT(40.671659705389 74.7453961030301) bank58641 +58642 POINT(41.084110121089935 74.83133672911609) bank58642 +58643 POINT(40.49687944652294 73.56959513186963) bank58643 +58644 POINT(41.17110202535695 74.5494464875086) bank58644 +58645 POINT(41.3603514227264 74.01439469454334) bank58645 +58646 POINT(41.48460858026335 73.9983775825861) bank58646 +58647 POINT(40.93724420966104 74.57737691790912) bank58647 +58648 POINT(40.462150300799884 73.99559670152365) bank58648 +58649 POINT(41.277093467104734 74.20023919261254) bank58649 +58650 POINT(41.45946567467089 74.20003796400498) bank58650 +58651 POINT(39.736178159200875 73.78476432389722) bank58651 +58652 POINT(41.25242373403226 74.3852486348593) bank58652 +58653 POINT(40.343726276282325 73.09670348377658) bank58653 +58654 POINT(40.76462417740387 73.17807460664218) bank58654 +58655 POINT(40.515323061897355 74.55989414925094) bank58655 +58656 POINT(41.156776992967245 73.2151489225948) bank58656 +58657 POINT(40.85098176216986 73.30196219666155) bank58657 +58658 POINT(41.58860846911528 73.25961813814277) bank58658 +58659 POINT(40.88470411148742 74.25450313974595) bank58659 +58660 POINT(40.570208396502345 74.34993285408902) bank58660 +58661 POINT(39.836191970101005 74.86693072113292) bank58661 +58662 POINT(41.248662952138794 73.43297947268672) bank58662 +58663 POINT(40.8590621198594 74.11583390157674) bank58663 +58664 POINT(40.96665857026252 74.50662720395371) bank58664 +58665 POINT(40.95083904599001 74.65019201269043) bank58665 +58666 POINT(39.81577983354307 74.9057586338523) bank58666 +58667 POINT(40.31858116824938 73.23949079609324) bank58667 +58668 POINT(40.78484751059513 73.75453783628696) bank58668 +58669 POINT(40.47256277138553 74.71576479219026) bank58669 +58670 POINT(41.05835077126889 73.49816828129624) bank58670 +58671 POINT(40.69684989759444 74.06451302049689) bank58671 +58672 POINT(41.69529026416513 73.10049624196479) bank58672 +58673 POINT(40.84069980368903 74.0038070617802) bank58673 +58674 POINT(40.481937926603706 74.68823230040137) bank58674 +58675 POINT(41.040557927079796 74.03233236822375) bank58675 +58676 POINT(40.95846537030837 74.67484456179136) bank58676 +58677 POINT(41.30033443868597 73.95363245395497) bank58677 +58678 POINT(40.431758563302516 73.89159343309406) bank58678 +58679 POINT(40.8557961290959 74.8434418855638) bank58679 +58680 POINT(40.99649171922368 73.97571120864603) bank58680 +58681 POINT(40.6577413141097 74.06054666086645) bank58681 +58682 POINT(40.36535955636229 73.30337914105384) bank58682 +58683 POINT(39.767211744393784 73.68839673416201) bank58683 +58684 POINT(40.59246005499519 74.68014657243351) bank58684 +58685 POINT(41.17727036085772 73.39751626797224) bank58685 +58686 POINT(39.894016431837315 73.01429306922428) bank58686 +58687 POINT(41.54866477740468 73.63394384329823) bank58687 +58688 POINT(41.654417834860084 73.82774611020055) bank58688 +58689 POINT(41.40265393120747 74.23121275636535) bank58689 +58690 POINT(40.6940566333493 74.32811778149608) bank58690 +58691 POINT(40.395031156710786 75.00588023602126) bank58691 +58692 POINT(40.246729891519394 73.3262228684494) bank58692 +58693 POINT(40.09710822563694 73.18538340490477) bank58693 +58694 POINT(41.44648222593866 74.31869050646232) bank58694 +58695 POINT(41.38618265214797 74.1816782532211) bank58695 +58696 POINT(40.26192169202239 73.80133349405632) bank58696 +58697 POINT(39.85768430916107 74.86808146522411) bank58697 +58698 POINT(41.41143724893899 74.92019429714543) bank58698 +58699 POINT(41.65518968517795 73.16207084571097) bank58699 +58700 POINT(40.02383409273318 74.18474104839981) bank58700 +58701 POINT(41.64713647338768 74.7954160691761) bank58701 +58702 POINT(41.458268464016236 74.14032091374713) bank58702 +58703 POINT(41.587196991190275 74.18019145862145) bank58703 +58704 POINT(41.582922327589365 73.99401225078601) bank58704 +58705 POINT(39.95851377769934 73.0329919541359) bank58705 +58706 POINT(40.97394927772621 74.38162892583277) bank58706 +58707 POINT(40.32247089302326 74.97669193009911) bank58707 +58708 POINT(40.201358291483594 73.45512137149541) bank58708 +58709 POINT(40.53860889481162 74.02905725384637) bank58709 +58710 POINT(40.786956155211 73.82556792272428) bank58710 +58711 POINT(41.227557536581536 74.93756223034985) bank58711 +58712 POINT(41.286132799593055 73.03312620722811) bank58712 +58713 POINT(41.020831453107746 73.06703488010749) bank58713 +58714 POINT(41.64465395756952 74.27161739076206) bank58714 +58715 POINT(41.70221200540363 73.44859077489592) bank58715 +58716 POINT(40.80114170932946 74.31275249878757) bank58716 +58717 POINT(40.00927613009287 74.2435454735287) bank58717 +58718 POINT(39.959548883860805 74.93503775455152) bank58718 +58719 POINT(41.15084818558595 73.14621467414628) bank58719 +58720 POINT(40.09353405094368 74.61989445294235) bank58720 +58721 POINT(41.493547798550644 74.91010895596231) bank58721 +58722 POINT(40.175691335261995 73.79447461762919) bank58722 +58723 POINT(40.3752024689605 74.2513439920686) bank58723 +58724 POINT(41.19329859272935 74.2380731730916) bank58724 +58725 POINT(41.416704897384214 74.54122982571562) bank58725 +58726 POINT(40.032006581865375 73.69858129597608) bank58726 +58727 POINT(41.088634585464796 74.94092296568932) bank58727 +58728 POINT(41.5700720419527 74.57291011890906) bank58728 +58729 POINT(40.68509308382413 74.73754994514383) bank58729 +58730 POINT(40.39805189949866 73.41207540923628) bank58730 +58731 POINT(40.52537858334758 74.03518377840561) bank58731 +58732 POINT(40.58510391196905 74.6667691018256) bank58732 +58733 POINT(39.758507573900026 74.63219594342273) bank58733 +58734 POINT(40.598992186239904 73.83569076934772) bank58734 +58735 POINT(41.033084426076066 73.68087824512769) bank58735 +58736 POINT(39.75897918720978 73.83085967002694) bank58736 +58737 POINT(39.7150499809162 73.8727835005565) bank58737 +58738 POINT(41.445168047438784 73.50161226053415) bank58738 +58739 POINT(41.33700844757133 73.13363749133656) bank58739 +58740 POINT(41.273310164767054 73.62115415628843) bank58740 +58741 POINT(41.34956745252308 73.40250989922957) bank58741 +58742 POINT(41.106397755384364 73.70312189643717) bank58742 +58743 POINT(39.93416779436833 73.50294705739317) bank58743 +58744 POINT(40.641147520916675 74.63263900720203) bank58744 +58745 POINT(40.4610671130451 74.6636860031126) bank58745 +58746 POINT(40.7887465537174 74.56081992350906) bank58746 +58747 POINT(40.529808444093206 74.28846380504646) bank58747 +58748 POINT(41.36638446208276 74.48814079794465) bank58748 +58749 POINT(40.825295017779375 74.61682381455869) bank58749 +58750 POINT(40.047214199599985 73.42487764578286) bank58750 +58751 POINT(40.26961729296678 74.45311764585317) bank58751 +58752 POINT(40.252175051055076 74.43182956116407) bank58752 +58753 POINT(39.88256176521676 73.65062312139372) bank58753 +58754 POINT(40.42716232974184 73.66806552808583) bank58754 +58755 POINT(39.81100059462411 73.21399054322666) bank58755 +58756 POINT(40.36583300995036 74.31455903085327) bank58756 +58757 POINT(40.312234844789046 73.44993240227436) bank58757 +58758 POINT(41.107195168312956 73.09210850650639) bank58758 +58759 POINT(39.81379102833731 74.51914018013736) bank58759 +58760 POINT(40.551853260448894 74.6191565293149) bank58760 +58761 POINT(39.77382377851408 74.8715020482781) bank58761 +58762 POINT(41.05393472859686 74.78345523050709) bank58762 +58763 POINT(39.73989399870433 74.33794956929334) bank58763 +58764 POINT(41.48603654981636 74.27648119264565) bank58764 +58765 POINT(39.84068344354133 73.7773840662475) bank58765 +58766 POINT(40.17705071426537 74.0977082319257) bank58766 +58767 POINT(41.42461388412981 73.97648683471508) bank58767 +58768 POINT(41.58118271097103 73.18797974140706) bank58768 +58769 POINT(39.97458310263396 73.41996253396437) bank58769 +58770 POINT(40.46823157823594 74.19208721611285) bank58770 +58771 POINT(40.87731310840944 73.37794010886206) bank58771 +58772 POINT(40.68846318248494 73.18090954971487) bank58772 +58773 POINT(41.567489337233305 74.25101812780508) bank58773 +58774 POINT(41.5351239823289 74.01723013274734) bank58774 +58775 POINT(40.404447907564744 73.61402275886557) bank58775 +58776 POINT(40.43031504557737 73.65343842045081) bank58776 +58777 POINT(40.726929245951766 74.70617361471714) bank58777 +58778 POINT(41.305192625544805 73.04659120426054) bank58778 +58779 POINT(39.78459161896225 73.05392256825942) bank58779 +58780 POINT(40.74631869472303 73.14066722599713) bank58780 +58781 POINT(41.35011544402575 73.94528515766) bank58781 +58782 POINT(39.92247109363589 73.62806198694973) bank58782 +58783 POINT(40.78784125279381 73.4333641160274) bank58783 +58784 POINT(41.271504655523415 74.44100309485052) bank58784 +58785 POINT(40.552870120031436 73.8658571631921) bank58785 +58786 POINT(40.81229376037268 73.52090760529461) bank58786 +58787 POINT(40.115594706569254 74.99697609243907) bank58787 +58788 POINT(40.54761187077129 73.88219647471506) bank58788 +58789 POINT(40.37703510281359 73.4303853578963) bank58789 +58790 POINT(40.09864239133274 73.92260134505595) bank58790 +58791 POINT(40.63373994359602 74.4275456443474) bank58791 +58792 POINT(41.704604473286565 74.00834392100245) bank58792 +58793 POINT(41.704569792268586 73.51633804074227) bank58793 +58794 POINT(41.28354993548622 74.33929216173553) bank58794 +58795 POINT(41.65830311446885 74.06398864846867) bank58795 +58796 POINT(40.47799175466255 73.73470816204483) bank58796 +58797 POINT(39.755596952969796 73.19806228577178) bank58797 +58798 POINT(39.82043182881819 73.45365633859083) bank58798 +58799 POINT(41.46413267353422 74.13985554153189) bank58799 +58800 POINT(39.82255472267299 74.34056184205103) bank58800 +58801 POINT(40.684680880351735 74.92541231996293) bank58801 +58802 POINT(40.92602778168145 73.21961291968826) bank58802 +58803 POINT(40.607925707190326 73.1376214522431) bank58803 +58804 POINT(40.61173584375132 74.57181928554283) bank58804 +58805 POINT(40.80994304739975 73.30326541150436) bank58805 +58806 POINT(40.205041757398156 73.11420263129953) bank58806 +58807 POINT(39.73298293291173 73.7914445410905) bank58807 +58808 POINT(40.344287881982055 74.95340884848454) bank58808 +58809 POINT(41.10653482176255 73.50413789043165) bank58809 +58810 POINT(41.405515143660686 73.71402906620463) bank58810 +58811 POINT(41.442089602494846 74.83588411758583) bank58811 +58812 POINT(41.05806294447078 73.46322261173557) bank58812 +58813 POINT(40.1146344272589 74.63774789227395) bank58813 +58814 POINT(40.80271724056815 74.29229717078736) bank58814 +58815 POINT(41.53059379130961 73.5853180549508) bank58815 +58816 POINT(40.72347348003297 73.46335322984528) bank58816 +58817 POINT(40.93655407315763 74.09747848897956) bank58817 +58818 POINT(40.15558346041202 73.67079664312539) bank58818 +58819 POINT(39.812053057040664 73.43806519747648) bank58819 +58820 POINT(40.19601414504874 73.1054073712484) bank58820 +58821 POINT(39.72928637411776 73.93549521476555) bank58821 +58822 POINT(39.826384256636146 73.71403382060444) bank58822 +58823 POINT(40.419027688161094 74.03250870037516) bank58823 +58824 POINT(40.61145923382683 73.2849995922665) bank58824 +58825 POINT(40.616089461252876 74.0471835520737) bank58825 +58826 POINT(40.11949341385624 74.03060976575871) bank58826 +58827 POINT(41.06015228150376 73.4831710800035) bank58827 +58828 POINT(39.715548808526925 73.33321846904943) bank58828 +58829 POINT(40.13343720012169 74.94344877775471) bank58829 +58830 POINT(40.47269276728786 73.7767076232152) bank58830 +58831 POINT(40.43104753914552 73.4360457931948) bank58831 +58832 POINT(41.586754425558496 74.89742071844037) bank58832 +58833 POINT(41.643200675976196 74.62012638918122) bank58833 +58834 POINT(40.126775369737786 74.4592880194393) bank58834 +58835 POINT(41.29187076184255 73.03488572521309) bank58835 +58836 POINT(41.51934987535441 73.98264212501022) bank58836 +58837 POINT(41.55339161745401 73.87219274591436) bank58837 +58838 POINT(41.20959484245458 74.91534506397947) bank58838 +58839 POINT(40.6993208612379 74.34071112463506) bank58839 +58840 POINT(40.33223482988451 74.28999345207461) bank58840 +58841 POINT(40.370399136661774 73.16962849343652) bank58841 +58842 POINT(40.64693945991543 73.52891230560297) bank58842 +58843 POINT(40.89655797504025 74.55335045915585) bank58843 +58844 POINT(41.13099336777929 74.10189302304848) bank58844 +58845 POINT(40.77064342235827 73.15005237932117) bank58845 +58846 POINT(41.046482362993785 73.15359908875999) bank58846 +58847 POINT(39.84018238625975 73.55153070831871) bank58847 +58848 POINT(40.65033003624481 73.87683407054689) bank58848 +58849 POINT(39.91022084723039 74.62652270176227) bank58849 +58850 POINT(41.22043981159841 73.25616661900125) bank58850 +58851 POINT(40.463271152805284 73.84287975382834) bank58851 +58852 POINT(40.10237607796666 73.94100888265375) bank58852 +58853 POINT(40.81385323580454 74.32196286754393) bank58853 +58854 POINT(40.22643783131697 74.23917178342916) bank58854 +58855 POINT(39.735689302346046 74.07763091148425) bank58855 +58856 POINT(41.23537426807142 73.32967409567406) bank58856 +58857 POINT(41.00156815690091 74.93532175304443) bank58857 +58858 POINT(40.5056876423947 73.44453461309601) bank58858 +58859 POINT(40.38780617894241 74.99996706419782) bank58859 +58860 POINT(39.80258666627611 73.50387749001074) bank58860 +58861 POINT(39.91466996419558 73.52288858839962) bank58861 +58862 POINT(41.48150064760469 74.35341204387944) bank58862 +58863 POINT(41.248354991129226 74.05272187440251) bank58863 +58864 POINT(40.949518184233185 74.47991307045834) bank58864 +58865 POINT(41.59599648410548 74.98979193814482) bank58865 +58866 POINT(41.3964347301439 74.59463843182317) bank58866 +58867 POINT(41.206620145639924 74.90348006742272) bank58867 +58868 POINT(41.05237385393194 73.59859605375799) bank58868 +58869 POINT(40.56016722342891 74.55713885143668) bank58869 +58870 POINT(41.706284389087365 74.61333145322547) bank58870 +58871 POINT(40.70726895690749 73.97637253873295) bank58871 +58872 POINT(39.91701383130422 73.5973013969874) bank58872 +58873 POINT(40.22221397633021 73.35530136048743) bank58873 +58874 POINT(40.47007063239419 73.01873296956357) bank58874 +58875 POINT(41.47017749813784 73.40726338004112) bank58875 +58876 POINT(40.159597697383816 73.97804315384205) bank58876 +58877 POINT(40.90785441196428 74.48287563394481) bank58877 +58878 POINT(41.56824202566902 73.82794045915833) bank58878 +58879 POINT(41.62826231514162 74.84681181807088) bank58879 +58880 POINT(40.98223977501437 74.63801670810318) bank58880 +58881 POINT(40.08991528080314 74.33032648495369) bank58881 +58882 POINT(40.55770574431868 74.47566270254543) bank58882 +58883 POINT(41.163826987035456 74.61388653601452) bank58883 +58884 POINT(39.72521750691283 74.09711019964986) bank58884 +58885 POINT(40.644301001692796 73.26914776434842) bank58885 +58886 POINT(40.66192228919319 74.99347562529434) bank58886 +58887 POINT(39.73798918790861 74.7469883950348) bank58887 +58888 POINT(40.57054769135742 74.73849755392152) bank58888 +58889 POINT(41.58240158369092 74.61735245095352) bank58889 +58890 POINT(40.16418864831057 73.60677337976016) bank58890 +58891 POINT(40.921058540341896 74.78557753243352) bank58891 +58892 POINT(39.760021332172535 74.9028503622154) bank58892 +58893 POINT(39.988194821226834 74.5893587854035) bank58893 +58894 POINT(40.611181396485264 73.5153072278635) bank58894 +58895 POINT(40.09688343364092 74.77978470484601) bank58895 +58896 POINT(41.161933071256385 74.8854617844598) bank58896 +58897 POINT(40.590760545047345 73.58756684487541) bank58897 +58898 POINT(39.91603396739864 73.52488320334406) bank58898 +58899 POINT(40.73744266898063 74.91118666968447) bank58899 +58900 POINT(40.48462097802018 74.91702327159409) bank58900 +58901 POINT(39.85042030659133 74.13467861656895) bank58901 +58902 POINT(40.21149038245429 74.4338693849895) bank58902 +58903 POINT(40.28175226212967 73.8957538870638) bank58903 +58904 POINT(40.62914405900345 73.96295064132138) bank58904 +58905 POINT(41.23506944846371 73.49994570261492) bank58905 +58906 POINT(41.424067558745946 74.17834265238979) bank58906 +58907 POINT(41.17677719316272 73.1829108903235) bank58907 +58908 POINT(41.21962680115526 74.25028028011256) bank58908 +58909 POINT(40.67340425557673 73.31475115942696) bank58909 +58910 POINT(40.038311743955234 74.39779885039641) bank58910 +58911 POINT(39.90073344881545 74.89083156579785) bank58911 +58912 POINT(40.746742899679475 73.43819350146103) bank58912 +58913 POINT(41.6464761953945 73.74264598566063) bank58913 +58914 POINT(39.98205322458606 73.25633324246935) bank58914 +58915 POINT(40.91880349680055 73.89609552529352) bank58915 +58916 POINT(41.597958525124234 74.00886586603932) bank58916 +58917 POINT(40.99966848822964 73.28613320631516) bank58917 +58918 POINT(40.672206462103965 73.25173841146574) bank58918 +58919 POINT(39.792442638337256 74.85783148587096) bank58919 +58920 POINT(40.36144185552381 74.14204605743552) bank58920 +58921 POINT(39.73848551912747 73.43835425736876) bank58921 +58922 POINT(40.05535246295772 73.7686028981689) bank58922 +58923 POINT(40.342769328357676 73.8445863982722) bank58923 +58924 POINT(41.435889005625654 73.86286495439258) bank58924 +58925 POINT(41.14384352408401 74.39304730672653) bank58925 +58926 POINT(40.641649556124285 73.99693591834844) bank58926 +58927 POINT(40.947923685449936 73.7820157742706) bank58927 +58928 POINT(41.09501817120857 73.95711448363909) bank58928 +58929 POINT(40.539216851330025 74.98039763241484) bank58929 +58930 POINT(40.03195222154756 74.58433182141086) bank58930 +58931 POINT(41.3795135057547 74.25218526615483) bank58931 +58932 POINT(40.17337628572924 74.95816962153812) bank58932 +58933 POINT(40.21955179420936 73.32509108608264) bank58933 +58934 POINT(40.7379254939923 73.34365214963154) bank58934 +58935 POINT(41.01978660046496 73.32422353967027) bank58935 +58936 POINT(39.84166971469947 74.80307028538145) bank58936 +58937 POINT(40.59678174099278 74.31931190738075) bank58937 +58938 POINT(41.22288033782227 73.65778916194122) bank58938 +58939 POINT(40.318831056337714 74.33126599511037) bank58939 +58940 POINT(39.77781346251449 74.78914643830859) bank58940 +58941 POINT(40.768515449989266 74.34106989360738) bank58941 +58942 POINT(40.25082578854154 73.03241231877092) bank58942 +58943 POINT(40.11213345868681 74.72644766326181) bank58943 +58944 POINT(41.52043772074354 74.0436942068548) bank58944 +58945 POINT(41.34989136126883 73.7100237794775) bank58945 +58946 POINT(40.3441124014197 73.20329255036486) bank58946 +58947 POINT(40.55879978902215 74.00755920078421) bank58947 +58948 POINT(39.962834230506004 73.71717266618398) bank58948 +58949 POINT(40.596071963985914 73.99001958858233) bank58949 +58950 POINT(41.11029790728809 74.74620191842827) bank58950 +58951 POINT(40.02219747653172 74.29734640060262) bank58951 +58952 POINT(40.68961146507679 73.92600500912933) bank58952 +58953 POINT(40.9568795226924 73.48706985218641) bank58953 +58954 POINT(40.2473439636655 74.64847101536347) bank58954 +58955 POINT(40.50073944326789 74.67607953643761) bank58955 +58956 POINT(41.21784602040689 74.19162512297414) bank58956 +58957 POINT(41.40788455373279 74.91696193960807) bank58957 +58958 POINT(41.52358941420651 73.60924493624086) bank58958 +58959 POINT(40.19152667251481 73.41724146482608) bank58959 +58960 POINT(41.55783600229772 73.78547697270929) bank58960 +58961 POINT(41.39145428225994 73.85178019474414) bank58961 +58962 POINT(40.10452264834117 74.16414609163847) bank58962 +58963 POINT(41.35683357480697 74.00963975845902) bank58963 +58964 POINT(40.772143101405845 73.52168292590531) bank58964 +58965 POINT(41.5881008639847 73.37727527590133) bank58965 +58966 POINT(40.679570743649585 74.38899748008818) bank58966 +58967 POINT(39.884006914138695 73.20686396546274) bank58967 +58968 POINT(39.83764626461117 74.14574177591741) bank58968 +58969 POINT(40.21300217776089 74.75537227296361) bank58969 +58970 POINT(40.80521286167315 74.83143624857547) bank58970 +58971 POINT(41.291342678175056 73.0063790379408) bank58971 +58972 POINT(40.04925148760141 73.94728706028205) bank58972 +58973 POINT(41.59925711628617 74.28250485655506) bank58973 +58974 POINT(40.08232963580734 74.08468549330571) bank58974 +58975 POINT(41.46871369763328 73.78873009797753) bank58975 +58976 POINT(39.97770758775883 73.08970135707513) bank58976 +58977 POINT(40.157516317257965 74.78702200757002) bank58977 +58978 POINT(40.067934589204704 74.64396523349508) bank58978 +58979 POINT(40.673791642771945 73.94611179180399) bank58979 +58980 POINT(40.6733775879106 73.3848523111786) bank58980 +58981 POINT(40.092063560728 73.8915759647986) bank58981 +58982 POINT(41.29232451619292 74.09412654603571) bank58982 +58983 POINT(41.23944636395302 73.67106373320787) bank58983 +58984 POINT(41.421350089121 73.50768970384136) bank58984 +58985 POINT(40.110229477114956 74.39675705200207) bank58985 +58986 POINT(39.93292640744845 73.40463199120272) bank58986 +58987 POINT(40.26973984144779 75.00004068529228) bank58987 +58988 POINT(41.58999473788891 73.4769560355068) bank58988 +58989 POINT(39.87718480325785 74.94173609394244) bank58989 +58990 POINT(41.552349290953266 74.14683007230161) bank58990 +58991 POINT(40.08063401200589 73.35608814854146) bank58991 +58992 POINT(40.84420096189246 74.04993487762768) bank58992 +58993 POINT(41.42844993961354 73.3371655727703) bank58993 +58994 POINT(40.150688490439684 74.63324339344963) bank58994 +58995 POINT(40.10916475861061 74.81567963417415) bank58995 +58996 POINT(41.31490842744692 74.20431704553633) bank58996 +58997 POINT(40.79346532556521 73.78309600955367) bank58997 +58998 POINT(40.82456442603757 73.90072273635562) bank58998 +58999 POINT(40.590954364525196 74.63571877114971) bank58999 +59000 POINT(40.066035021782945 74.77650163384565) bank59000 +59001 POINT(40.50483604884967 74.88224754077707) bank59001 +59002 POINT(40.53963046446276 73.3658543429905) bank59002 +59003 POINT(39.978536437477274 74.8765221928741) bank59003 +59004 POINT(41.68524451871882 74.52002207919283) bank59004 +59005 POINT(40.09119575521705 74.63719990691291) bank59005 +59006 POINT(41.514395013798 74.19421735922538) bank59006 +59007 POINT(40.11191787330866 74.84817691468707) bank59007 +59008 POINT(41.4418504900176 74.22068806875181) bank59008 +59009 POINT(41.33694042230354 73.33282524025527) bank59009 +59010 POINT(41.49932021407711 74.5604566782322) bank59010 +59011 POINT(41.538870288894856 74.6906093226086) bank59011 +59012 POINT(40.169736907808684 73.67394034994129) bank59012 +59013 POINT(41.05909350136484 73.47570360039244) bank59013 +59014 POINT(40.02521851055789 73.39970006597865) bank59014 +59015 POINT(40.356205562822325 73.49842454211397) bank59015 +59016 POINT(41.37055850653098 74.9987051115126) bank59016 +59017 POINT(39.96179028227269 74.81714824548028) bank59017 +59018 POINT(40.2693937502751 74.73190788659274) bank59018 +59019 POINT(41.025250240364954 73.97474812094474) bank59019 +59020 POINT(41.03945680899216 73.49036527881874) bank59020 +59021 POINT(40.3527455410969 73.33741809735784) bank59021 +59022 POINT(39.89944897129765 74.8300993135085) bank59022 +59023 POINT(40.25148063203059 73.8174975275573) bank59023 +59024 POINT(40.33276188403976 73.75380645781235) bank59024 +59025 POINT(41.01688648395836 74.97435707964338) bank59025 +59026 POINT(40.36911830759959 74.84806191365543) bank59026 +59027 POINT(40.44502710779125 73.18758262052364) bank59027 +59028 POINT(41.361958725071844 74.57753778299306) bank59028 +59029 POINT(40.575830611238516 73.50390189535004) bank59029 +59030 POINT(40.631881664122524 73.62116886838457) bank59030 +59031 POINT(40.587200544718186 73.02992288401424) bank59031 +59032 POINT(40.99521544487737 74.7790929652294) bank59032 +59033 POINT(40.94509633618436 74.5024454392996) bank59033 +59034 POINT(41.594502047216594 73.73313686099418) bank59034 +59035 POINT(41.27868259071713 73.59221888162368) bank59035 +59036 POINT(40.76605985173557 73.44604460519096) bank59036 +59037 POINT(40.82976277050357 74.83687215812327) bank59037 +59038 POINT(41.10240641504012 73.07447468638843) bank59038 +59039 POINT(40.86978913988441 73.79892492009448) bank59039 +59040 POINT(39.90594160665039 74.56159916571389) bank59040 +59041 POINT(39.85433702406077 73.17793577256342) bank59041 +59042 POINT(39.81179977369322 74.6538664519092) bank59042 +59043 POINT(40.364273613132845 73.3578983036103) bank59043 +59044 POINT(40.869839063030454 74.19962426905538) bank59044 +59045 POINT(41.377606431537366 73.35589277083905) bank59045 +59046 POINT(41.494244764019506 73.5857805211334) bank59046 +59047 POINT(39.76781493225251 74.29865172667502) bank59047 +59048 POINT(40.2381585840772 74.27076052191539) bank59048 +59049 POINT(40.63174303215723 74.74369887182088) bank59049 +59050 POINT(40.61583319004142 74.72239039058256) bank59050 +59051 POINT(39.8036178280692 74.84583601830623) bank59051 +59052 POINT(40.709719641066975 74.42511698509752) bank59052 +59053 POINT(40.25877541105159 73.6385153086145) bank59053 +59054 POINT(40.871954803632214 73.67917803869801) bank59054 +59055 POINT(41.699093963891066 74.84874946224548) bank59055 +59056 POINT(40.63572121822887 73.17465829078091) bank59056 +59057 POINT(41.011864505921565 73.74286102513892) bank59057 +59058 POINT(40.92509977466312 73.42673788000162) bank59058 +59059 POINT(40.115939479371924 74.38925583094098) bank59059 +59060 POINT(41.46072953217373 73.40721273198716) bank59060 +59061 POINT(41.28610654906396 74.78602668219096) bank59061 +59062 POINT(41.00593285544453 74.5531221226666) bank59062 +59063 POINT(40.761533806421625 74.59664418176742) bank59063 +59064 POINT(40.447479519553106 74.4751696906466) bank59064 +59065 POINT(40.50786338316879 73.88050257602384) bank59065 +59066 POINT(40.8220960523268 74.42494699482843) bank59066 +59067 POINT(40.588284552019765 73.55504458038536) bank59067 +59068 POINT(40.119262976277106 74.65997257788747) bank59068 +59069 POINT(41.69353712454913 73.79501162514248) bank59069 +59070 POINT(40.46705926739314 73.43415961155232) bank59070 +59071 POINT(39.99110288813568 74.14042115352947) bank59071 +59072 POINT(40.92058542144238 74.2978852306108) bank59072 +59073 POINT(40.052764655177505 73.70508905915528) bank59073 +59074 POINT(39.74378585008927 73.82179580889189) bank59074 +59075 POINT(40.027223630641274 73.27230921548234) bank59075 +59076 POINT(41.35076669414466 74.98806041510467) bank59076 +59077 POINT(39.97497854250936 73.82002781350333) bank59077 +59078 POINT(39.75189084694022 74.01600179143036) bank59078 +59079 POINT(40.840140974850435 73.55337502271765) bank59079 +59080 POINT(40.11222645161858 73.16701895479079) bank59080 +59081 POINT(41.41926429669174 75.00128481241796) bank59081 +59082 POINT(39.80651457448198 73.553970822915) bank59082 +59083 POINT(41.28377721934013 74.63842715695601) bank59083 +59084 POINT(41.24759542267607 74.16554404504666) bank59084 +59085 POINT(40.04345222415226 73.63019019743491) bank59085 +59086 POINT(41.094875830743504 74.10273343985801) bank59086 +59087 POINT(40.43928922887302 73.75230032890268) bank59087 +59088 POINT(39.767873489547995 74.4997877788646) bank59088 +59089 POINT(40.185638301371156 74.33271964688893) bank59089 +59090 POINT(39.972929743143766 73.04262840421055) bank59090 +59091 POINT(40.91243883013566 74.00184433145355) bank59091 +59092 POINT(39.87809996921044 74.73384444825355) bank59092 +59093 POINT(39.86177940309788 73.92992233493305) bank59093 +59094 POINT(40.944501297775666 74.14968881987797) bank59094 +59095 POINT(41.66270788657541 74.77439330247593) bank59095 +59096 POINT(40.09508902214878 74.08041551599139) bank59096 +59097 POINT(40.32323512674459 73.756164213279) bank59097 +59098 POINT(40.355143143490864 74.25392134600007) bank59098 +59099 POINT(40.618592165278336 74.65470444254794) bank59099 +59100 POINT(41.26902585313121 73.13144187908922) bank59100 +59101 POINT(40.66690248172098 74.0302739501264) bank59101 +59102 POINT(40.88079598741213 73.51158611034472) bank59102 +59103 POINT(41.246666017228954 73.96412882195077) bank59103 +59104 POINT(40.29174775571856 74.68951940455334) bank59104 +59105 POINT(40.929824119887776 73.65238023343478) bank59105 +59106 POINT(40.74020284990082 74.58124771566365) bank59106 +59107 POINT(40.50266835069409 74.15739476791079) bank59107 +59108 POINT(40.049467317308874 73.25850302700628) bank59108 +59109 POINT(39.770552076150295 73.95794918723762) bank59109 +59110 POINT(41.53371657199139 73.18333465868254) bank59110 +59111 POINT(40.67976145363528 73.46225935645829) bank59111 +59112 POINT(40.41709966164393 74.46130223140887) bank59112 +59113 POINT(40.30555643853428 73.89192901119159) bank59113 +59114 POINT(40.7748313764848 74.625148751117) bank59114 +59115 POINT(39.80731930758417 74.78902508441082) bank59115 +59116 POINT(40.216348968021656 74.17696146208118) bank59116 +59117 POINT(40.45480744030485 74.11941515284455) bank59117 +59118 POINT(40.81485985934177 73.06580265008927) bank59118 +59119 POINT(41.39596350678641 73.95264546053718) bank59119 +59120 POINT(40.334933796608915 73.24224135940017) bank59120 +59121 POINT(41.21323797677276 74.43376232475053) bank59121 +59122 POINT(39.96886964877969 74.84036191517905) bank59122 +59123 POINT(40.24753926654884 73.28627278138237) bank59123 +59124 POINT(40.49819431475327 74.59604965307189) bank59124 +59125 POINT(40.60544527737094 73.04326835022425) bank59125 +59126 POINT(41.58679230424739 73.71227482040366) bank59126 +59127 POINT(40.76379614182366 73.46544921761053) bank59127 +59128 POINT(39.73006192252416 73.87657214631442) bank59128 +59129 POINT(40.273856061329795 73.3168143588757) bank59129 +59130 POINT(40.406731796602166 73.57876146808228) bank59130 +59131 POINT(41.53274188379369 74.58108044717076) bank59131 +59132 POINT(41.1082086560681 74.62558677905983) bank59132 +59133 POINT(40.23892347366862 74.34698077328348) bank59133 +59134 POINT(40.478555733228 74.06293810344636) bank59134 +59135 POINT(40.543946325843706 74.48070745441152) bank59135 +59136 POINT(41.41613998710449 74.02581898719193) bank59136 +59137 POINT(40.44256862677874 74.05175138829343) bank59137 +59138 POINT(40.87427436083931 73.1541152721003) bank59138 +59139 POINT(40.98885311860566 73.15880668150781) bank59139 +59140 POINT(40.636683332532286 74.0708919677091) bank59140 +59141 POINT(40.31212950444149 74.11605352822194) bank59141 +59142 POINT(40.27054455654178 74.82810799787471) bank59142 +59143 POINT(41.02033231028837 73.90606275105988) bank59143 +59144 POINT(40.55107486506382 73.78753522206982) bank59144 +59145 POINT(41.38979952071152 74.42959887296122) bank59145 +59146 POINT(40.90861396543149 73.32995624878033) bank59146 +59147 POINT(41.170835490081046 74.01779054237203) bank59147 +59148 POINT(41.70924001377135 74.78764411097087) bank59148 +59149 POINT(40.235436040877644 74.02654018379395) bank59149 +59150 POINT(40.14120494791381 74.14919998003423) bank59150 +59151 POINT(41.696227070775294 74.21218364078396) bank59151 +59152 POINT(40.14282741929739 73.22285874383724) bank59152 +59153 POINT(39.83862209459194 74.95621237123426) bank59153 +59154 POINT(40.27745200473788 74.48177186666213) bank59154 +59155 POINT(40.08670458581721 73.51594973852903) bank59155 +59156 POINT(40.69901116301017 73.50359461494527) bank59156 +59157 POINT(40.90557769517783 74.40569750092386) bank59157 +59158 POINT(41.10044549882096 74.61865336702895) bank59158 +59159 POINT(41.03325815397272 74.18524561397102) bank59159 +59160 POINT(41.711252587387 73.10821550826716) bank59160 +59161 POINT(41.669283497792335 73.55865008397767) bank59161 +59162 POINT(40.12643001760074 73.3330712123625) bank59162 +59163 POINT(41.670762630678944 73.53850727277982) bank59163 +59164 POINT(40.271259736338244 73.1961919556638) bank59164 +59165 POINT(40.6855355455308 74.35634985583839) bank59165 +59166 POINT(40.7107755449893 74.74688855731064) bank59166 +59167 POINT(40.12086708866798 74.32374426368634) bank59167 +59168 POINT(40.72620756080388 74.92285182464578) bank59168 +59169 POINT(40.3158143335542 73.69325194507566) bank59169 +59170 POINT(41.523630022378015 73.58179643734685) bank59170 +59171 POINT(41.45544779262054 73.13007965358825) bank59171 +59172 POINT(41.20297590592759 73.81546926047204) bank59172 +59173 POINT(41.09797310656074 74.65883970518514) bank59173 +59174 POINT(41.405082081360504 73.41291147784716) bank59174 +59175 POINT(39.88932658271604 74.52681341759269) bank59175 +59176 POINT(41.16818447616463 74.71940464433575) bank59176 +59177 POINT(39.9179293213406 73.75440958022506) bank59177 +59178 POINT(41.28047097228675 74.35761196829333) bank59178 +59179 POINT(41.315054000807635 73.06632147333075) bank59179 +59180 POINT(41.60468467452023 73.5923453374397) bank59180 +59181 POINT(39.89256798061447 73.73195968373201) bank59181 +59182 POINT(41.28741835692537 73.56459474332564) bank59182 +59183 POINT(39.92724163825183 74.69829137062324) bank59183 +59184 POINT(40.54651272511931 74.12546976669437) bank59184 +59185 POINT(40.77887258366093 73.26742959606038) bank59185 +59186 POINT(39.80880087006851 74.61727858604637) bank59186 +59187 POINT(41.28554868701181 74.18397275149415) bank59187 +59188 POINT(41.53300445578496 74.14388884349187) bank59188 +59189 POINT(40.83542139263269 73.82372926053368) bank59189 +59190 POINT(40.25848782489962 74.89329474558916) bank59190 +59191 POINT(41.57223958157216 73.53314434573912) bank59191 +59192 POINT(40.23551907244701 73.41895092769774) bank59192 +59193 POINT(40.82861321246302 73.56256486740784) bank59193 +59194 POINT(40.261107302355995 74.58474803925766) bank59194 +59195 POINT(40.52226589158966 74.08683360801622) bank59195 +59196 POINT(40.94869391429692 73.02642086988496) bank59196 +59197 POINT(40.571855065807675 73.82324320127395) bank59197 +59198 POINT(41.10425127631151 73.21683999970877) bank59198 +59199 POINT(40.96744544023635 74.55544411615448) bank59199 +59200 POINT(40.50641430453606 74.00512559342786) bank59200 +59201 POINT(40.28432516013181 74.89352322751795) bank59201 +59202 POINT(40.503000404458405 74.31964009502487) bank59202 +59203 POINT(39.96193222187529 74.33009207107679) bank59203 +59204 POINT(40.683268978472434 73.2640030980214) bank59204 +59205 POINT(41.45683410896825 74.38463157628345) bank59205 +59206 POINT(40.394245353497226 73.47376206292991) bank59206 +59207 POINT(39.795144840744584 73.43965520245283) bank59207 +59208 POINT(40.02422772846359 73.7152881448274) bank59208 +59209 POINT(39.90553991411779 74.64113416073383) bank59209 +59210 POINT(41.55081499743225 74.5116338793117) bank59210 +59211 POINT(40.05201583525345 73.69910047132298) bank59211 +59212 POINT(39.876505461758796 73.78211021948553) bank59212 +59213 POINT(40.88569090315127 74.13909242157807) bank59213 +59214 POINT(41.05591029697504 74.69549565028929) bank59214 +59215 POINT(40.32590126314907 74.1718412246825) bank59215 +59216 POINT(39.949677554128115 74.88251857273136) bank59216 +59217 POINT(41.31793811703763 74.8503342295532) bank59217 +59218 POINT(39.743713669182625 74.35618258231887) bank59218 +59219 POINT(40.17022215981759 74.3970233782872) bank59219 +59220 POINT(40.89247372451207 74.75085381173139) bank59220 +59221 POINT(40.28312189289272 73.33629758761656) bank59221 +59222 POINT(40.27783407149534 74.96259042583186) bank59222 +59223 POINT(40.933949241908955 73.89410407367016) bank59223 +59224 POINT(40.82771626794741 73.30600499056995) bank59224 +59225 POINT(40.43789576013506 73.22754516220775) bank59225 +59226 POINT(41.38588666796448 74.69615109252898) bank59226 +59227 POINT(40.71211317933331 74.74386362176487) bank59227 +59228 POINT(40.189380077135915 73.570793524693) bank59228 +59229 POINT(40.94219212636298 73.60649329583329) bank59229 +59230 POINT(41.08371438176909 73.12295044679183) bank59230 +59231 POINT(41.05150271015777 74.2441877779899) bank59231 +59232 POINT(41.3724082698257 73.80253029473347) bank59232 +59233 POINT(41.49257121153339 74.5667306483907) bank59233 +59234 POINT(40.57031531089828 74.65645843071779) bank59234 +59235 POINT(41.31764489191413 73.07395883673706) bank59235 +59236 POINT(40.74937345705299 74.25821697919963) bank59236 +59237 POINT(39.79952555829009 73.21226745495923) bank59237 +59238 POINT(39.80509907947877 73.29801033365897) bank59238 +59239 POINT(41.25649493312128 74.66982703454653) bank59239 +59240 POINT(39.97585853956965 73.67467007965492) bank59240 +59241 POINT(40.22486945668585 73.84278033087293) bank59241 +59242 POINT(40.111872514208 74.2054926667585) bank59242 +59243 POINT(40.69905085596208 73.48080966502513) bank59243 +59244 POINT(41.484977370336686 73.951823001592) bank59244 +59245 POINT(39.82161526276192 74.265644138524) bank59245 +59246 POINT(41.05817795800963 74.47155714910542) bank59246 +59247 POINT(41.0873692375567 73.99100639025833) bank59247 +59248 POINT(41.0242008201816 73.32316342992806) bank59248 +59249 POINT(40.526855366854846 73.23394116565481) bank59249 +59250 POINT(40.672620188926885 74.2455616381224) bank59250 +59251 POINT(41.557210449265106 74.45768186621845) bank59251 +59252 POINT(41.2660175149938 73.50423500815913) bank59252 +59253 POINT(41.45036034888879 73.30411965133843) bank59253 +59254 POINT(39.95156476020061 74.92968116887111) bank59254 +59255 POINT(41.38206930068554 74.55045340146972) bank59255 +59256 POINT(41.429378484322804 73.24629098041488) bank59256 +59257 POINT(39.81053357743103 74.40025092473235) bank59257 +59258 POINT(40.99313680669311 74.4403059824785) bank59258 +59259 POINT(40.386288315075426 74.5468980279238) bank59259 +59260 POINT(39.850817514043804 73.44898573361998) bank59260 +59261 POINT(40.9338785422413 74.45025495836843) bank59261 +59262 POINT(40.5966415656306 73.01244318896876) bank59262 +59263 POINT(41.62375680000823 73.53350065364363) bank59263 +59264 POINT(40.15904878349718 73.46995780668333) bank59264 +59265 POINT(40.98731339438508 74.8502658993931) bank59265 +59266 POINT(41.13059317211605 73.32529442769538) bank59266 +59267 POINT(40.396185603896065 73.69350810558939) bank59267 +59268 POINT(40.64142019520781 74.66744655221075) bank59268 +59269 POINT(41.03996872320249 74.25340525477219) bank59269 +59270 POINT(40.0976789143557 74.79048776284777) bank59270 +59271 POINT(40.951095864756134 74.94729264067715) bank59271 +59272 POINT(41.45757552669073 73.15731905088035) bank59272 +59273 POINT(40.02606470106519 73.69268454472174) bank59273 +59274 POINT(40.73648226961524 74.13456717727472) bank59274 +59275 POINT(40.17597075779632 73.12256027749254) bank59275 +59276 POINT(40.982703298156125 73.87924076043811) bank59276 +59277 POINT(41.42207155796327 73.86987691151968) bank59277 +59278 POINT(41.02952024563986 73.9078640586393) bank59278 +59279 POINT(41.68798797280591 73.13928562713257) bank59279 +59280 POINT(40.8033211543692 73.89746196376286) bank59280 +59281 POINT(40.50274482653436 73.74638467860719) bank59281 +59282 POINT(41.499261154577646 74.51669953306993) bank59282 +59283 POINT(40.5557511912579 73.67554154393719) bank59283 +59284 POINT(40.73313405867618 74.31272586062583) bank59284 +59285 POINT(40.21978012312174 74.96710204516808) bank59285 +59286 POINT(41.106263267869366 74.27648295785275) bank59286 +59287 POINT(41.226620568996424 74.66874040182634) bank59287 +59288 POINT(41.46015157475498 73.11196267822929) bank59288 +59289 POINT(41.638511292135284 73.53550946587187) bank59289 +59290 POINT(41.57581083820177 73.27866706981493) bank59290 +59291 POINT(41.54854567598703 73.15285344970272) bank59291 +59292 POINT(40.36126674383169 74.37162546666899) bank59292 +59293 POINT(39.9361349729291 73.72332633198559) bank59293 +59294 POINT(41.63657027757715 73.46069569797551) bank59294 +59295 POINT(41.20999197185607 74.20822954723869) bank59295 +59296 POINT(39.825634784805985 73.58931155816407) bank59296 +59297 POINT(40.16908212616546 74.49018182944309) bank59297 +59298 POINT(40.77973502904256 74.73333822767266) bank59298 +59299 POINT(40.538665725106014 75.00499369320254) bank59299 +59300 POINT(40.4278755088925 74.84137900271142) bank59300 +59301 POINT(41.178643973278625 74.43072632277526) bank59301 +59302 POINT(41.14281418276725 74.88274208650061) bank59302 +59303 POINT(40.91653669910268 73.25121352521944) bank59303 +59304 POINT(40.188450149945645 73.37163533412341) bank59304 +59305 POINT(40.40218462103393 73.34605901114803) bank59305 +59306 POINT(40.8029449429159 74.37357546182193) bank59306 +59307 POINT(40.14098963346273 73.62687256191552) bank59307 +59308 POINT(40.66360005598028 73.50533970815498) bank59308 +59309 POINT(40.160464374483325 73.38218149616036) bank59309 +59310 POINT(39.944882438377604 73.84017553055658) bank59310 +59311 POINT(40.77255797029514 73.5472238044924) bank59311 +59312 POINT(39.898972346470245 74.69628872728782) bank59312 +59313 POINT(40.705526997285844 73.31210190749955) bank59313 +59314 POINT(40.11157917238324 74.37689946714102) bank59314 +59315 POINT(41.296709533319365 74.19464104454094) bank59315 +59316 POINT(40.76423254311759 74.30792358789878) bank59316 +59317 POINT(41.39748722660102 73.26944922022211) bank59317 +59318 POINT(39.78473051151802 73.4316028503055) bank59318 +59319 POINT(41.501967201927975 74.73158619872186) bank59319 +59320 POINT(41.338819221851175 74.31739387530791) bank59320 +59321 POINT(40.38344951665743 74.40490300656168) bank59321 +59322 POINT(41.34904588785326 74.05396280958576) bank59322 +59323 POINT(39.94111425978656 74.22267278698433) bank59323 +59324 POINT(40.35919688074083 73.04411441947062) bank59324 +59325 POINT(41.55296235349019 73.76511084486637) bank59325 +59326 POINT(40.22843896848235 74.8919122400409) bank59326 +59327 POINT(40.02949294580665 74.32365365516267) bank59327 +59328 POINT(40.56199667189603 74.48503539651962) bank59328 +59329 POINT(40.809013494377645 73.05508200911855) bank59329 +59330 POINT(41.39984658760843 74.14148064856933) bank59330 +59331 POINT(40.85218167517594 73.55782912701012) bank59331 +59332 POINT(40.27406682136604 73.73955715658647) bank59332 +59333 POINT(40.47540689455379 74.52519609312805) bank59333 +59334 POINT(40.29527229837119 74.20928433374345) bank59334 +59335 POINT(40.24020201822344 73.69102017735034) bank59335 +59336 POINT(41.1486634181359 74.11988650662214) bank59336 +59337 POINT(40.456182781725055 73.54349521433353) bank59337 +59338 POINT(39.8856274494232 74.47608243470913) bank59338 +59339 POINT(40.278978705948894 73.80299432710622) bank59339 +59340 POINT(41.55458326295157 73.90585880603254) bank59340 +59341 POINT(39.94153317226556 74.95488244464612) bank59341 +59342 POINT(40.71743830241077 74.93924102205925) bank59342 +59343 POINT(41.6447371820184 73.16011148265632) bank59343 +59344 POINT(40.026606722398746 74.93770568665174) bank59344 +59345 POINT(40.355331404990594 73.16765237882959) bank59345 +59346 POINT(40.3473084692366 73.52659322115062) bank59346 +59347 POINT(40.73944307910569 73.36951794747516) bank59347 +59348 POINT(40.29394987712265 74.8351228240436) bank59348 +59349 POINT(40.73478809768689 73.94147467210958) bank59349 +59350 POINT(40.96967424772818 74.06166794768087) bank59350 +59351 POINT(39.78878194407859 74.50087854284254) bank59351 +59352 POINT(40.48920256394195 74.60897063338709) bank59352 +59353 POINT(40.42801002050247 74.8111010097896) bank59353 +59354 POINT(40.81674425459303 73.52403309870441) bank59354 +59355 POINT(40.2653392489432 74.79307190716102) bank59355 +59356 POINT(41.36666310396753 74.84547114540463) bank59356 +59357 POINT(41.5554119491613 74.43903475383019) bank59357 +59358 POINT(40.48177817858632 73.8575811305877) bank59358 +59359 POINT(40.841711190573854 73.18979360323502) bank59359 +59360 POINT(40.67504710079083 73.11246600139356) bank59360 +59361 POINT(41.00597952404063 74.26126459450111) bank59361 +59362 POINT(41.08813837031854 73.14609195774774) bank59362 +59363 POINT(39.95777991438339 73.37152995105274) bank59363 +59364 POINT(41.69278090109376 74.27404698186713) bank59364 +59365 POINT(40.669360175006446 74.57224113192419) bank59365 +59366 POINT(40.792113100432466 74.51703276707147) bank59366 +59367 POINT(41.26160367562746 74.74510240911243) bank59367 +59368 POINT(41.418116159010644 74.92084591418256) bank59368 +59369 POINT(40.004498185860506 73.52561616544428) bank59369 +59370 POINT(40.691932504925205 73.3406109174935) bank59370 +59371 POINT(40.192866220459464 74.23413698632172) bank59371 +59372 POINT(41.27799178949809 74.63379887631368) bank59372 +59373 POINT(39.93012321856356 73.86864989271231) bank59373 +59374 POINT(40.497044387118514 73.26144049534852) bank59374 +59375 POINT(41.441010679661744 73.4210702139011) bank59375 +59376 POINT(41.68904708597756 74.71652656404403) bank59376 +59377 POINT(41.241986332362025 73.36230734714165) bank59377 +59378 POINT(40.55734830343871 74.97492547122674) bank59378 +59379 POINT(40.54448230625458 74.39377459299408) bank59379 +59380 POINT(40.495102649679225 74.93290135297008) bank59380 +59381 POINT(40.462428077294774 74.00830664066434) bank59381 +59382 POINT(41.435888221267746 74.0646159139346) bank59382 +59383 POINT(41.648927983520956 73.48115236133631) bank59383 +59384 POINT(40.97249903926938 74.7402547990178) bank59384 +59385 POINT(40.59395887608436 73.17995007491687) bank59385 +59386 POINT(41.70204177498623 73.60793024482842) bank59386 +59387 POINT(41.56474981232489 74.24785814515785) bank59387 +59388 POINT(39.83459842284204 73.11029138576247) bank59388 +59389 POINT(39.862693364453214 74.71163017839129) bank59389 +59390 POINT(39.945663183896166 74.49067404114352) bank59390 +59391 POINT(41.504285216028315 73.87580217650279) bank59391 +59392 POINT(40.15757221340015 73.87380882737033) bank59392 +59393 POINT(41.204152405405374 74.0618324316833) bank59393 +59394 POINT(40.55116624672509 73.50870143952189) bank59394 +59395 POINT(40.31755754588717 73.71282261979603) bank59395 +59396 POINT(40.80753106791453 74.41527434132472) bank59396 +59397 POINT(40.12973479763797 73.31019189994916) bank59397 +59398 POINT(40.45386887591101 73.23830901970493) bank59398 +59399 POINT(40.193410268434874 74.81052429637973) bank59399 +59400 POINT(40.780805280429654 73.09585352459914) bank59400 +59401 POINT(40.25305224982931 74.13054259683376) bank59401 +59402 POINT(41.68142411070433 74.89933643275135) bank59402 +59403 POINT(40.93469043908995 74.6989782445106) bank59403 +59404 POINT(40.01210151410367 73.86326610509275) bank59404 +59405 POINT(40.67972676684508 73.25275679134283) bank59405 +59406 POINT(40.45335910563403 73.43958737519105) bank59406 +59407 POINT(40.88349345300401 73.70983508445059) bank59407 +59408 POINT(41.45860635910713 73.18555608775799) bank59408 +59409 POINT(40.48829030036718 73.78334710194457) bank59409 +59410 POINT(40.42407334145399 74.17314141849764) bank59410 +59411 POINT(40.20091794021209 74.09869307451518) bank59411 +59412 POINT(40.08802289130323 74.45555645694672) bank59412 +59413 POINT(40.76957127233259 74.42480866203653) bank59413 +59414 POINT(41.078192839750635 73.07249222243811) bank59414 +59415 POINT(40.33580073471641 74.73338977962815) bank59415 +59416 POINT(40.49264641973124 73.82611354752856) bank59416 +59417 POINT(40.25834343515561 74.64820040906771) bank59417 +59418 POINT(41.22639510302849 74.42462977145817) bank59418 +59419 POINT(40.843484466774015 73.5048884147135) bank59419 +59420 POINT(41.428505523106914 73.72042256159567) bank59420 +59421 POINT(41.48499874919865 74.21502585960573) bank59421 +59422 POINT(40.61092133813906 74.75274126529833) bank59422 +59423 POINT(41.01707505405242 73.51030993743608) bank59423 +59424 POINT(41.69023919200371 73.86497169358479) bank59424 +59425 POINT(40.221391940622 73.43348587378138) bank59425 +59426 POINT(39.71705790238147 73.99328104681398) bank59426 +59427 POINT(41.53748945132094 73.30223338765752) bank59427 +59428 POINT(40.586343456455 74.23237423092658) bank59428 +59429 POINT(39.99070417036181 74.91001671238641) bank59429 +59430 POINT(40.86620762468229 74.92823044085245) bank59430 +59431 POINT(40.277248091063576 74.70993861489066) bank59431 +59432 POINT(41.12196473355499 74.38318625513037) bank59432 +59433 POINT(41.68045024194881 74.86263836014311) bank59433 +59434 POINT(41.232448349874275 73.04575256537647) bank59434 +59435 POINT(40.024951549355926 74.35261707999922) bank59435 +59436 POINT(40.1152549327719 74.8433980604586) bank59436 +59437 POINT(40.55037365874759 73.35258078261711) bank59437 +59438 POINT(40.67242433466315 73.64550660942162) bank59438 +59439 POINT(41.5932681009507 73.49397779244951) bank59439 +59440 POINT(40.086224031259526 74.75969060513958) bank59440 +59441 POINT(40.143823483721754 73.5729589937045) bank59441 +59442 POINT(40.09568803593815 74.90107509085722) bank59442 +59443 POINT(41.02265181451494 74.63142298558039) bank59443 +59444 POINT(41.62955792019806 73.2936707394828) bank59444 +59445 POINT(41.38235326232398 74.29643465402334) bank59445 +59446 POINT(40.565469131039364 73.32229477872555) bank59446 +59447 POINT(39.8595249064888 73.75548781180825) bank59447 +59448 POINT(40.211238992770774 74.54829234262687) bank59448 +59449 POINT(40.69607529376721 73.48321667856965) bank59449 +59450 POINT(40.376165146829045 73.83467601082714) bank59450 +59451 POINT(40.39655839077791 74.25700657882302) bank59451 +59452 POINT(41.43370792222137 73.85439356233759) bank59452 +59453 POINT(40.66549450385489 74.16266308823737) bank59453 +59454 POINT(40.99401255174836 74.04261601244065) bank59454 +59455 POINT(40.635585363480104 73.51527658309764) bank59455 +59456 POINT(41.11356966130472 73.77469936078927) bank59456 +59457 POINT(40.603461803023336 73.75965658575792) bank59457 +59458 POINT(40.82999943272145 74.42219722539356) bank59458 +59459 POINT(40.205201777098374 73.3474094913388) bank59459 +59460 POINT(41.47135315178322 74.97374753114282) bank59460 +59461 POINT(40.59523969433796 74.79257651945461) bank59461 +59462 POINT(40.13750848154533 74.96785986853521) bank59462 +59463 POINT(40.06091214744266 73.5096212270824) bank59463 +59464 POINT(40.40864920843899 74.97472796140045) bank59464 +59465 POINT(39.85335457522688 74.6123153775764) bank59465 +59466 POINT(40.137751226447676 73.5454618492702) bank59466 +59467 POINT(40.71902611741076 74.76995490230243) bank59467 +59468 POINT(40.6924669792899 73.65127129077182) bank59468 +59469 POINT(39.74219364097266 73.35643049923755) bank59469 +59470 POINT(40.69911829306483 74.57049467802344) bank59470 +59471 POINT(40.872126453994206 74.90993990943589) bank59471 +59472 POINT(40.245798990593805 73.014470356173) bank59472 +59473 POINT(40.2200350282339 73.61209641782771) bank59473 +59474 POINT(39.9039229334297 74.5348403629189) bank59474 +59475 POINT(41.5378353052406 74.04214610541891) bank59475 +59476 POINT(41.67525412485615 74.75682492235927) bank59476 +59477 POINT(39.786756699705094 74.01140748898298) bank59477 +59478 POINT(40.62311371357219 73.22513002181566) bank59478 +59479 POINT(41.40431721852692 74.59921453300326) bank59479 +59480 POINT(41.33523126789075 74.45067331191837) bank59480 +59481 POINT(41.06811233213622 73.44272011847545) bank59481 +59482 POINT(40.112142306424666 73.80272471690989) bank59482 +59483 POINT(40.141974299633326 74.25228630843122) bank59483 +59484 POINT(40.11388485219256 74.77602552978816) bank59484 +59485 POINT(41.14944094960558 74.86520032561634) bank59485 +59486 POINT(39.818219655158074 73.09871126463219) bank59486 +59487 POINT(39.713717922060546 74.57247207484666) bank59487 +59488 POINT(40.43769551320728 74.26389555880327) bank59488 +59489 POINT(40.42372152648639 74.12792965047217) bank59489 +59490 POINT(41.661505250745414 74.7231099484512) bank59490 +59491 POINT(41.48068994560656 73.58615287892718) bank59491 +59492 POINT(41.35416964921179 73.15753193833322) bank59492 +59493 POINT(40.99746354465472 74.78551238169) bank59493 +59494 POINT(41.38401696117985 74.80590498461237) bank59494 +59495 POINT(40.81988638901296 74.61306712570585) bank59495 +59496 POINT(40.801524855339586 74.8694874865283) bank59496 +59497 POINT(40.124463889378674 73.10315354808716) bank59497 +59498 POINT(40.33291451525065 73.97058151447824) bank59498 +59499 POINT(40.70257727243114 73.47713219163859) bank59499 +59500 POINT(40.566410053203704 73.06856129490238) bank59500 +59501 POINT(40.05408167140049 74.29330773293837) bank59501 +59502 POINT(40.10233508884779 74.35789156820702) bank59502 +59503 POINT(40.2357001814072 74.81870074288302) bank59503 +59504 POINT(41.023704522393395 73.64803435961572) bank59504 +59505 POINT(41.133954652061036 73.87355900544655) bank59505 +59506 POINT(40.30745284749062 74.30410544322461) bank59506 +59507 POINT(41.16383438402603 74.26069424176134) bank59507 +59508 POINT(40.060624534821144 73.43513106273807) bank59508 +59509 POINT(41.5055110053698 73.71773157913854) bank59509 +59510 POINT(41.12010867503539 74.5247422460279) bank59510 +59511 POINT(39.79558663428487 74.37357645063373) bank59511 +59512 POINT(39.835343689961725 73.07158450155337) bank59512 +59513 POINT(41.393509365022716 73.16391466878927) bank59513 +59514 POINT(40.164696809818466 74.79941600582801) bank59514 +59515 POINT(39.83940399145488 74.64420839249436) bank59515 +59516 POINT(40.037940606218385 74.24825225955355) bank59516 +59517 POINT(39.89539510721382 74.04445340139458) bank59517 +59518 POINT(39.811717664732676 74.69018777881956) bank59518 +59519 POINT(40.44138112787802 73.17235442774671) bank59519 +59520 POINT(40.7174162628988 74.62339865810523) bank59520 +59521 POINT(41.67340613377312 74.62617345409691) bank59521 +59522 POINT(39.99482333727634 73.5420507559667) bank59522 +59523 POINT(39.85846792508704 73.48554001032353) bank59523 +59524 POINT(40.41635034907935 74.48918678196671) bank59524 +59525 POINT(40.826882984238715 73.66191808284934) bank59525 +59526 POINT(41.124537605642246 74.70401671990172) bank59526 +59527 POINT(41.31078676396037 74.40007987760885) bank59527 +59528 POINT(40.88012391533477 73.9479091923027) bank59528 +59529 POINT(40.73809196070982 73.60879712186465) bank59529 +59530 POINT(40.23715369126275 74.69003766039756) bank59530 +59531 POINT(39.801285209313455 73.9306745458874) bank59531 +59532 POINT(40.73382494280507 73.11946216882421) bank59532 +59533 POINT(40.02226628134915 73.82300149222871) bank59533 +59534 POINT(39.89606058381197 74.65443327005352) bank59534 +59535 POINT(41.70664385197306 74.10370951212155) bank59535 +59536 POINT(40.635625120178574 74.18458500184215) bank59536 +59537 POINT(41.547562009477666 73.30024948731976) bank59537 +59538 POINT(40.033070718009824 73.92812761588746) bank59538 +59539 POINT(40.003427945483395 73.9986600513621) bank59539 +59540 POINT(40.33573890050804 73.19012724527418) bank59540 +59541 POINT(40.973935406122834 74.69455774366683) bank59541 +59542 POINT(41.64954622837128 73.04051430141598) bank59542 +59543 POINT(40.04592470121434 74.67034631327556) bank59543 +59544 POINT(40.82422809908035 73.76149827384964) bank59544 +59545 POINT(40.63787317079881 73.03844449863078) bank59545 +59546 POINT(41.3653323416372 73.64869721135942) bank59546 +59547 POINT(39.89818963608404 73.45392672929714) bank59547 +59548 POINT(41.21412902849809 74.85300364622643) bank59548 +59549 POINT(41.531801218868814 73.36885358524113) bank59549 +59550 POINT(41.078977562483935 74.63921958833039) bank59550 +59551 POINT(41.70169668612463 73.50039449001785) bank59551 +59552 POINT(40.91111140099709 73.68552188431926) bank59552 +59553 POINT(39.93084020363522 74.62797184729334) bank59553 +59554 POINT(41.45916242270926 74.41699280962402) bank59554 +59555 POINT(41.70212806479086 73.46407692626033) bank59555 +59556 POINT(40.917587478758456 73.63199832496154) bank59556 +59557 POINT(40.13630663931694 74.19162339992634) bank59557 +59558 POINT(41.34868142994179 73.50275610812231) bank59558 +59559 POINT(41.34626905127964 74.30790340696127) bank59559 +59560 POINT(41.161295857428925 74.6234687674923) bank59560 +59561 POINT(39.736878095803156 73.23116635500202) bank59561 +59562 POINT(40.03005478331524 74.54777810517524) bank59562 +59563 POINT(41.26608294823816 74.44935157538356) bank59563 +59564 POINT(41.21525461533544 74.69833083336914) bank59564 +59565 POINT(40.311514290405526 74.05242034602132) bank59565 +59566 POINT(40.49655628557992 73.04517154318009) bank59566 +59567 POINT(41.39356777358485 74.96899461573085) bank59567 +59568 POINT(40.97565264170886 74.18833803651115) bank59568 +59569 POINT(40.41595105425677 74.61389282667471) bank59569 +59570 POINT(41.28819562306225 74.20592786091704) bank59570 +59571 POINT(40.960111865836346 74.80639584129193) bank59571 +59572 POINT(40.475409154670196 73.67365960883151) bank59572 +59573 POINT(40.60604562963486 74.85456291379617) bank59573 +59574 POINT(41.62519820153072 73.15781076906752) bank59574 +59575 POINT(41.35547816829436 74.92612301547115) bank59575 +59576 POINT(39.71431452578709 73.77992381860386) bank59576 +59577 POINT(41.24531614223696 74.51865381364975) bank59577 +59578 POINT(41.44232098937201 73.39059688445734) bank59578 +59579 POINT(39.940219362885266 74.26249787832046) bank59579 +59580 POINT(39.784005347723806 73.73371507165379) bank59580 +59581 POINT(40.65292766583908 73.4016081428096) bank59581 +59582 POINT(41.52783578720741 73.54902767383464) bank59582 +59583 POINT(40.13426821213423 74.90548097302005) bank59583 +59584 POINT(40.544867266124 74.38048291210336) bank59584 +59585 POINT(40.63844300446984 73.98356413039558) bank59585 +59586 POINT(40.683707132307426 73.0783045812678) bank59586 +59587 POINT(41.62908797045647 73.75671507328384) bank59587 +59588 POINT(40.27241373902079 74.7621219701834) bank59588 +59589 POINT(39.86488053752555 73.95470878402463) bank59589 +59590 POINT(41.458597133599866 74.87362864817231) bank59590 +59591 POINT(41.4405715166864 73.65548883795533) bank59591 +59592 POINT(41.66380413286321 73.18960674394702) bank59592 +59593 POINT(40.248679367337004 73.5560810666401) bank59593 +59594 POINT(40.06936256159308 73.05359267224527) bank59594 +59595 POINT(40.11144651885163 73.39955318916759) bank59595 +59596 POINT(40.186017656173014 73.58099180928728) bank59596 +59597 POINT(40.39071429059787 74.6026730175756) bank59597 +59598 POINT(39.89363484580795 73.71174995800388) bank59598 +59599 POINT(40.2905824106795 74.19544790822242) bank59599 +59600 POINT(40.25103790252267 73.89286847543173) bank59600 +59601 POINT(39.81048316557282 73.16141467630604) bank59601 +59602 POINT(40.65751446140805 74.17652129196452) bank59602 +59603 POINT(40.712981128012316 74.22283034899398) bank59603 +59604 POINT(40.64071015261322 73.0435840921542) bank59604 +59605 POINT(40.950183469113455 73.5559378484224) bank59605 +59606 POINT(41.63063388240562 73.75342997802326) bank59606 +59607 POINT(40.38589074738302 74.60235627788782) bank59607 +59608 POINT(39.987285450238325 74.27156404998715) bank59608 +59609 POINT(40.257443523713036 73.44427546564883) bank59609 +59610 POINT(40.94951958077124 74.09556109616852) bank59610 +59611 POINT(40.09479206743228 74.34932257828189) bank59611 +59612 POINT(40.56056545983397 73.9436523633932) bank59612 +59613 POINT(40.376360153015185 74.14903746613138) bank59613 +59614 POINT(40.84172855756907 73.61282682173952) bank59614 +59615 POINT(40.139993934678195 74.76685554195015) bank59615 +59616 POINT(41.63484542478622 74.58178729700577) bank59616 +59617 POINT(40.414809412370204 73.28926868346386) bank59617 +59618 POINT(41.631405807016996 74.04402587392994) bank59618 +59619 POINT(40.78900032319737 73.06313302103416) bank59619 +59620 POINT(41.68875315516552 73.1018408150234) bank59620 +59621 POINT(41.53950466113794 74.1442538071098) bank59621 +59622 POINT(40.70128874046773 74.9042121455147) bank59622 +59623 POINT(40.18440739015003 73.76580660937218) bank59623 +59624 POINT(41.16173553760359 74.3122752982814) bank59624 +59625 POINT(41.39359276105348 73.88812156930037) bank59625 +59626 POINT(40.697965118638365 73.84759810159395) bank59626 +59627 POINT(39.82810344820484 73.48241962729188) bank59627 +59628 POINT(40.10412862733337 73.08624296661124) bank59628 +59629 POINT(40.24711186375827 74.96495649040544) bank59629 +59630 POINT(40.33741639461638 74.0209068994953) bank59630 +59631 POINT(40.00410972418952 74.76344566328976) bank59631 +59632 POINT(41.01945676742289 73.17176484967358) bank59632 +59633 POINT(40.944707256536496 74.21243411127193) bank59633 +59634 POINT(41.06261145802194 73.02493630132992) bank59634 +59635 POINT(41.031187296596784 73.63004961305997) bank59635 +59636 POINT(40.05213676196982 74.06124045950254) bank59636 +59637 POINT(41.71141336432673 74.0816544926701) bank59637 +59638 POINT(39.87451495376897 74.5625417071016) bank59638 +59639 POINT(39.92269216898508 74.83774335011555) bank59639 +59640 POINT(40.27968501495867 74.13014351844154) bank59640 +59641 POINT(41.25149370092518 73.13613340661023) bank59641 +59642 POINT(41.069011034684564 74.95558825000516) bank59642 +59643 POINT(40.620558037303724 74.64562586213192) bank59643 +59644 POINT(41.51581224554227 73.5430237907414) bank59644 +59645 POINT(40.78418224021614 74.12746467759659) bank59645 +59646 POINT(41.319515425029635 74.15439541339373) bank59646 +59647 POINT(40.19683920550801 74.41264529979163) bank59647 +59648 POINT(39.91793259191537 73.72851519755986) bank59648 +59649 POINT(40.859165606909 73.41615597462646) bank59649 +59650 POINT(41.03188920599303 74.15461604548705) bank59650 +59651 POINT(41.51063518790418 73.2189467633809) bank59651 +59652 POINT(40.000417244764364 73.6128438774915) bank59652 +59653 POINT(41.55808046639015 74.04047543154901) bank59653 +59654 POINT(40.27620785883467 73.05091269318851) bank59654 +59655 POINT(41.42988797129005 74.83931904947706) bank59655 +59656 POINT(40.79990567839239 73.11768802614687) bank59656 +59657 POINT(40.89780896131626 73.04764483732225) bank59657 +59658 POINT(40.445361316535646 73.4065151863013) bank59658 +59659 POINT(40.853162707217635 74.60322624765425) bank59659 +59660 POINT(40.32744100296442 73.61250337970132) bank59660 +59661 POINT(41.67323285443123 73.53788991305882) bank59661 +59662 POINT(41.40265984020165 73.04997889071652) bank59662 +59663 POINT(41.14634065109728 74.97400070872926) bank59663 +59664 POINT(41.13797983817216 73.13214685448663) bank59664 +59665 POINT(41.26853560207796 74.02117104569686) bank59665 +59666 POINT(41.00570844043822 74.2929398289475) bank59666 +59667 POINT(41.26720320116017 74.91481008770513) bank59667 +59668 POINT(39.9808661204635 73.58175465630848) bank59668 +59669 POINT(41.45589582479578 73.27059494746742) bank59669 +59670 POINT(40.825554228703865 74.91090340376748) bank59670 +59671 POINT(40.8380505997122 73.41695768989021) bank59671 +59672 POINT(40.587620436837696 73.18093624430351) bank59672 +59673 POINT(40.23514922576054 74.13821191403746) bank59673 +59674 POINT(40.836216491287274 74.2762775265154) bank59674 +59675 POINT(41.4002164835431 74.92783489216146) bank59675 +59676 POINT(41.48825899697533 73.66761945518903) bank59676 +59677 POINT(41.01386707491002 73.07345556864647) bank59677 +59678 POINT(41.53014973517244 74.41879925498546) bank59678 +59679 POINT(41.06862699719399 74.62523774551644) bank59679 +59680 POINT(41.408595023494534 74.48270574093414) bank59680 +59681 POINT(41.43661614089176 74.08337688149061) bank59681 +59682 POINT(39.7376189881702 74.62601228018266) bank59682 +59683 POINT(40.871698834300716 74.38451102201542) bank59683 +59684 POINT(40.46288919080133 74.24247003618622) bank59684 +59685 POINT(40.54675856251562 74.68868088924692) bank59685 +59686 POINT(40.574944684289505 73.34524453088785) bank59686 +59687 POINT(40.430034595131964 73.41516363684279) bank59687 +59688 POINT(41.542512155904895 74.37500653871228) bank59688 +59689 POINT(41.292185954378155 73.03730139233929) bank59689 +59690 POINT(39.92017045257753 74.25051540771109) bank59690 +59691 POINT(40.48084173338934 73.17210245132846) bank59691 +59692 POINT(41.30328220964795 73.56772597240301) bank59692 +59693 POINT(39.75551541798017 74.05815923116577) bank59693 +59694 POINT(40.18650576029363 73.34782294785941) bank59694 +59695 POINT(39.79315158523199 74.98371872031353) bank59695 +59696 POINT(41.49736988829203 74.24592586923917) bank59696 +59697 POINT(41.666598904805014 74.38218334480858) bank59697 +59698 POINT(40.95125119929041 74.52398921719785) bank59698 +59699 POINT(41.1186822397276 74.21416378488128) bank59699 +59700 POINT(40.637369905641535 73.45968931573442) bank59700 +59701 POINT(40.15228188911056 74.34540951192128) bank59701 +59702 POINT(41.598067513107644 74.97535570924416) bank59702 +59703 POINT(40.585570013828836 73.14347380673722) bank59703 +59704 POINT(39.831722985241235 74.3700131420833) bank59704 +59705 POINT(39.723812300793206 73.42772305979288) bank59705 +59706 POINT(40.88607109106039 74.72901973349242) bank59706 +59707 POINT(41.54662923483103 73.72769866036307) bank59707 +59708 POINT(40.63807180876783 73.82178962003013) bank59708 +59709 POINT(41.43504204550406 74.14697817037634) bank59709 +59710 POINT(40.524624275362356 74.45256240656134) bank59710 +59711 POINT(40.83106799244732 74.53934643583297) bank59711 +59712 POINT(39.98084687261651 74.70385624194383) bank59712 +59713 POINT(41.311038347321166 73.79309631166817) bank59713 +59714 POINT(41.04797971837435 74.47218341695425) bank59714 +59715 POINT(40.63315220041669 74.49412054408383) bank59715 +59716 POINT(40.279127758430704 73.5755504065105) bank59716 +59717 POINT(40.67671541618857 74.87301284353997) bank59717 +59718 POINT(41.518365730409364 73.34800950414322) bank59718 +59719 POINT(40.53291600295659 74.3764144737661) bank59719 +59720 POINT(40.59543498296282 73.1203282191062) bank59720 +59721 POINT(40.81646590113531 74.57613462946934) bank59721 +59722 POINT(39.877351601075425 73.65668348678861) bank59722 +59723 POINT(40.55654099661247 74.59282499237185) bank59723 +59724 POINT(40.987982377674314 74.48324136190082) bank59724 +59725 POINT(41.56090217201713 73.40478909712668) bank59725 +59726 POINT(41.33329827233172 73.75571084820434) bank59726 +59727 POINT(40.85067313719103 74.81827809138478) bank59727 +59728 POINT(41.51180807336957 74.35738668983252) bank59728 +59729 POINT(40.42259800166845 73.07505749422764) bank59729 +59730 POINT(40.76906920561588 74.06404818715913) bank59730 +59731 POINT(40.783239420588615 73.85018565073717) bank59731 +59732 POINT(40.769404192317495 73.09540172072181) bank59732 +59733 POINT(40.56457607094575 73.7570467852835) bank59733 +59734 POINT(41.53217813139267 74.53215839316834) bank59734 +59735 POINT(40.338751400425245 73.44837372687724) bank59735 +59736 POINT(40.71256470063409 73.02015334288276) bank59736 +59737 POINT(39.91055141280903 73.99791738323904) bank59737 +59738 POINT(40.560400997675224 73.63040698068595) bank59738 +59739 POINT(40.90952424700912 74.51472316238943) bank59739 +59740 POINT(41.70393907724631 74.57821938042991) bank59740 +59741 POINT(41.061764138566154 73.548777392834) bank59741 +59742 POINT(40.657823775153155 73.06188192759022) bank59742 +59743 POINT(40.43612157492861 73.26668934550622) bank59743 +59744 POINT(40.56570167144172 73.39359090074319) bank59744 +59745 POINT(41.100776793397706 74.959648892628) bank59745 +59746 POINT(40.45612123309452 74.08988525342458) bank59746 +59747 POINT(39.79067814166755 73.72249245909148) bank59747 +59748 POINT(40.46230031369484 73.55432432837085) bank59748 +59749 POINT(40.806443550701125 73.90958225402662) bank59749 +59750 POINT(40.28808381382476 74.43551397214883) bank59750 +59751 POINT(40.62902050113721 74.44411726931109) bank59751 +59752 POINT(40.54830479045366 74.62139868383422) bank59752 +59753 POINT(39.7699186605745 73.31689701698244) bank59753 +59754 POINT(39.73632074059707 74.88796948233625) bank59754 +59755 POINT(40.38078010978679 73.3143544798058) bank59755 +59756 POINT(41.0532864974082 74.33555651301022) bank59756 +59757 POINT(40.98954086268717 73.027109858172) bank59757 +59758 POINT(40.312116208544126 74.11742377230814) bank59758 +59759 POINT(39.77663032240965 73.05592000669323) bank59759 +59760 POINT(40.64200745314562 73.42560486321133) bank59760 +59761 POINT(41.305829814253904 74.5303353377517) bank59761 +59762 POINT(41.39512841320469 73.96348607953445) bank59762 +59763 POINT(41.095960855602115 74.38105668874002) bank59763 +59764 POINT(41.25563416207074 74.79346388443845) bank59764 +59765 POINT(39.822575698858536 74.57212130533877) bank59765 +59766 POINT(40.27552648268015 74.11161393740755) bank59766 +59767 POINT(40.15575175936374 73.06528638032178) bank59767 +59768 POINT(40.55549968569883 73.4451762574569) bank59768 +59769 POINT(41.13217510260837 73.58840348698043) bank59769 +59770 POINT(41.16787282918296 74.644537204041) bank59770 +59771 POINT(40.04765634167183 73.90161545888046) bank59771 +59772 POINT(40.064843174531376 73.12218597658513) bank59772 +59773 POINT(40.17388107867689 74.14979163868018) bank59773 +59774 POINT(39.82210420364204 73.26195534212482) bank59774 +59775 POINT(40.72707917013469 73.31432223380759) bank59775 +59776 POINT(39.905384328745654 73.43647663419767) bank59776 +59777 POINT(41.50537490026581 73.86775800843765) bank59777 +59778 POINT(41.03952058470557 74.49908194924362) bank59778 +59779 POINT(41.25884547705588 73.9087099659557) bank59779 +59780 POINT(41.110779717756515 73.84599034006729) bank59780 +59781 POINT(41.21231995562393 73.59701241085597) bank59781 +59782 POINT(39.95357132841994 74.56461617762284) bank59782 +59783 POINT(41.45691838058401 73.62347845893856) bank59783 +59784 POINT(40.25552663061379 73.25603850804103) bank59784 +59785 POINT(39.95013514607436 74.47658245487996) bank59785 +59786 POINT(41.664761640528866 73.05625887073042) bank59786 +59787 POINT(39.85680249216642 73.41823525352835) bank59787 +59788 POINT(39.96046863783499 74.56351100940081) bank59788 +59789 POINT(40.818661845575946 73.93665811144258) bank59789 +59790 POINT(40.95874457821819 73.78152396822462) bank59790 +59791 POINT(41.13151022698394 74.26693452355934) bank59791 +59792 POINT(40.77459095221363 73.37091934000986) bank59792 +59793 POINT(41.16641950623305 74.67432490853825) bank59793 +59794 POINT(41.70981318128726 73.22865276985613) bank59794 +59795 POINT(41.609924489112494 74.95922432440625) bank59795 +59796 POINT(40.03231793039891 73.5888267156953) bank59796 +59797 POINT(41.53772415891685 73.5229652751669) bank59797 +59798 POINT(40.20343723482505 73.1791462622252) bank59798 +59799 POINT(41.13091481840904 74.41747345491726) bank59799 +59800 POINT(41.46882019067551 74.99162602850326) bank59800 +59801 POINT(40.68475358241726 74.70234457153386) bank59801 +59802 POINT(41.38566954667552 73.17048055613624) bank59802 +59803 POINT(41.15219366404465 74.58120007021253) bank59803 +59804 POINT(40.71090774990627 74.84330965523137) bank59804 +59805 POINT(39.95805481086184 74.59249917569564) bank59805 +59806 POINT(39.72112285641966 73.48550631332922) bank59806 +59807 POINT(41.14564594283465 74.02826305888519) bank59807 +59808 POINT(41.14184842043134 73.1606141998252) bank59808 +59809 POINT(41.15448162914252 73.15748245367823) bank59809 +59810 POINT(40.90456261709595 73.61249726062704) bank59810 +59811 POINT(40.60427733458245 74.4520479839859) bank59811 +59812 POINT(41.24816902307019 73.04030484540903) bank59812 +59813 POINT(40.769923141851706 73.88909815191936) bank59813 +59814 POINT(40.79099062892637 73.89134223251298) bank59814 +59815 POINT(40.656421737017 74.56901890071669) bank59815 +59816 POINT(40.311432280540366 74.75323152738311) bank59816 +59817 POINT(40.11612741759808 74.73990269034454) bank59817 +59818 POINT(39.93879883911583 74.88050829010024) bank59818 +59819 POINT(40.95656874887172 74.39129365077441) bank59819 +59820 POINT(40.97364187122412 74.29167769752699) bank59820 +59821 POINT(40.85197286003297 74.58253280214902) bank59821 +59822 POINT(41.0332190115504 74.95706688419475) bank59822 +59823 POINT(40.65069953551844 74.87398281980559) bank59823 +59824 POINT(41.46014325736551 74.03313652829294) bank59824 +59825 POINT(40.978739959617 74.85174340335269) bank59825 +59826 POINT(40.13637091410771 73.34180130625045) bank59826 +59827 POINT(40.35429187336825 74.69812981323598) bank59827 +59828 POINT(40.88227595768205 73.88059819799574) bank59828 +59829 POINT(41.002988278103714 74.32430876376941) bank59829 +59830 POINT(41.68681974391 74.7183230066429) bank59830 +59831 POINT(39.896619619160305 74.58532602168428) bank59831 +59832 POINT(40.21187804988323 73.74752588203992) bank59832 +59833 POINT(39.72125103880287 73.75373161015209) bank59833 +59834 POINT(40.47629674666371 73.8006806280363) bank59834 +59835 POINT(40.22022745214474 74.25851349372564) bank59835 +59836 POINT(39.95610215968773 74.60616840382374) bank59836 +59837 POINT(41.589107948487985 73.43099198902627) bank59837 +59838 POINT(40.65690399783597 73.22994072620966) bank59838 +59839 POINT(41.17181382475885 74.63348165380941) bank59839 +59840 POINT(40.46825873219295 73.37299752553635) bank59840 +59841 POINT(40.36218662767532 73.30098433890666) bank59841 +59842 POINT(40.984681850023286 74.23222041705372) bank59842 +59843 POINT(39.80754265009619 73.42940112442697) bank59843 +59844 POINT(39.99526463540428 73.34066317833376) bank59844 +59845 POINT(40.9285768152323 74.97230417937968) bank59845 +59846 POINT(40.052228502539876 74.40783271513978) bank59846 +59847 POINT(39.968969929218375 73.59460387475409) bank59847 +59848 POINT(39.86195338547933 73.18305457407503) bank59848 +59849 POINT(41.01337207169236 74.70072114968404) bank59849 +59850 POINT(40.085947570470545 73.50843763469025) bank59850 +59851 POINT(40.243787355771275 73.8678287860742) bank59851 +59852 POINT(39.80399516733468 74.71883588662286) bank59852 +59853 POINT(40.82419312587527 73.84431858019111) bank59853 +59854 POINT(40.071927286936344 73.53470419437573) bank59854 +59855 POINT(41.13719532195836 73.73531188989075) bank59855 +59856 POINT(41.418918559856245 73.54760772458128) bank59856 +59857 POINT(41.33136770279752 74.74573131471419) bank59857 +59858 POINT(40.68261739650774 73.81186933052317) bank59858 +59859 POINT(41.35605061592205 73.362738256269) bank59859 +59860 POINT(41.00646245209982 74.57179100647106) bank59860 +59861 POINT(40.88842208853464 74.81782071117806) bank59861 +59862 POINT(40.25008302645033 73.43940055836457) bank59862 +59863 POINT(41.37178739944965 73.3943348139192) bank59863 +59864 POINT(40.662075338806076 74.62122452035747) bank59864 +59865 POINT(41.55217443699084 74.5667169343173) bank59865 +59866 POINT(40.73764663639567 74.49651159314766) bank59866 +59867 POINT(41.534104338052764 74.0761476343146) bank59867 +59868 POINT(40.90606244081725 73.18070655089208) bank59868 +59869 POINT(41.10318608422912 74.97846521747817) bank59869 +59870 POINT(39.75216672748997 74.56196519199484) bank59870 +59871 POINT(39.91490491924476 73.4325940538643) bank59871 +59872 POINT(40.29507132428052 73.8745355137725) bank59872 +59873 POINT(41.37504439371168 75.00084403444068) bank59873 +59874 POINT(41.576213949813436 73.63778761600621) bank59874 +59875 POINT(40.16255715256667 74.51082474415553) bank59875 +59876 POINT(41.49199835184203 74.77975165179986) bank59876 +59877 POINT(40.967122039505135 73.9835048481061) bank59877 +59878 POINT(39.81415946785365 74.76952249352033) bank59878 +59879 POINT(41.10433714205669 73.15447390829296) bank59879 +59880 POINT(40.59921635011151 74.74691229310642) bank59880 +59881 POINT(41.58143804506348 74.23522047819387) bank59881 +59882 POINT(40.52910704708472 73.42116990371407) bank59882 +59883 POINT(40.385983895696974 73.79302374319312) bank59883 +59884 POINT(40.43885365735702 74.12221189023244) bank59884 +59885 POINT(41.24080955780442 73.41833208824575) bank59885 +59886 POINT(40.54439124271532 73.96615587244311) bank59886 +59887 POINT(40.23969417016606 73.3620746050715) bank59887 +59888 POINT(40.73972409585322 74.42070748491528) bank59888 +59889 POINT(41.39615232340492 74.2141309995742) bank59889 +59890 POINT(40.55127880135679 74.9014949731294) bank59890 +59891 POINT(40.321936194771425 73.16841544100215) bank59891 +59892 POINT(40.069233433056624 74.14330198437342) bank59892 +59893 POINT(39.916361899895136 74.5463745446939) bank59893 +59894 POINT(40.63052875067785 74.88021312113275) bank59894 +59895 POINT(40.325877484904495 73.81808103935388) bank59895 +59896 POINT(39.89243651320396 73.58614557671082) bank59896 +59897 POINT(40.845798416069435 73.9891233617045) bank59897 +59898 POINT(41.27463347620376 74.48521651429515) bank59898 +59899 POINT(39.76315035960788 74.49817927032247) bank59899 +59900 POINT(40.45695934266393 73.58793777405491) bank59900 +59901 POINT(41.66510672014175 74.97532057338682) bank59901 +59902 POINT(41.53669428648691 73.48885397191647) bank59902 +59903 POINT(40.18093260319313 73.42572604230288) bank59903 +59904 POINT(41.20670992162808 74.09716263741423) bank59904 +59905 POINT(41.70485124593037 73.7219751038044) bank59905 +59906 POINT(40.45872639586655 74.467390971304) bank59906 +59907 POINT(40.45916744181272 73.81536525197996) bank59907 +59908 POINT(40.59271064487697 74.20560142400261) bank59908 +59909 POINT(41.41120939497011 73.43639270601489) bank59909 +59910 POINT(40.19675909510945 73.58661387862581) bank59910 +59911 POINT(40.78987812005606 74.54469541040908) bank59911 +59912 POINT(41.561585046029556 73.00726004079603) bank59912 +59913 POINT(41.69005878014574 74.16010951484265) bank59913 +59914 POINT(41.005480867638546 73.53135485764308) bank59914 +59915 POINT(40.690579686267085 73.8596255959095) bank59915 +59916 POINT(41.60383382170792 73.07728439033872) bank59916 +59917 POINT(41.27018745514212 74.96580476486051) bank59917 +59918 POINT(41.084256078858864 74.98186417224923) bank59918 +59919 POINT(41.57181408684122 74.3631506120634) bank59919 +59920 POINT(39.93528788513023 74.9625539521365) bank59920 +59921 POINT(39.96039308573568 74.11044320471257) bank59921 +59922 POINT(41.58176188135712 73.11725881209672) bank59922 +59923 POINT(39.89991697979802 73.90680030642453) bank59923 +59924 POINT(40.51754396951156 74.54699889937535) bank59924 +59925 POINT(41.28597530776165 74.88113959885145) bank59925 +59926 POINT(40.98043336880238 73.9488305703481) bank59926 +59927 POINT(39.99091903598977 73.52866652586646) bank59927 +59928 POINT(41.31643864604716 74.82679309887565) bank59928 +59929 POINT(40.74761329294745 73.77924256083081) bank59929 +59930 POINT(40.12379179795088 73.67632063644835) bank59930 +59931 POINT(40.77284348604292 73.64486505692719) bank59931 +59932 POINT(40.62972346303263 74.84822026022967) bank59932 +59933 POINT(41.61992089373895 74.6748883621206) bank59933 +59934 POINT(40.48233301370116 73.82310511596748) bank59934 +59935 POINT(39.745199401055466 74.73829565471615) bank59935 +59936 POINT(40.45806778037831 74.6815964903123) bank59936 +59937 POINT(39.834241690550805 74.39673442583477) bank59937 +59938 POINT(41.59286288590286 73.08958744088535) bank59938 +59939 POINT(40.15209012155355 73.89572007603655) bank59939 +59940 POINT(41.207616251328936 73.42968861876638) bank59940 +59941 POINT(40.23682677809711 74.50335165872298) bank59941 +59942 POINT(40.56879036564759 73.67765181588055) bank59942 +59943 POINT(40.640414250409435 74.50200255377128) bank59943 +59944 POINT(40.73098260576003 74.49902229607648) bank59944 +59945 POINT(40.00475903793182 74.02773950654223) bank59945 +59946 POINT(40.21732844251551 73.80332111098761) bank59946 +59947 POINT(40.2207456537298 74.49598026900705) bank59947 +59948 POINT(40.564820713271985 73.10054378675872) bank59948 +59949 POINT(41.33239348466393 73.58251669614076) bank59949 +59950 POINT(41.13962458820224 74.01714500075275) bank59950 +59951 POINT(40.0607693566937 74.87686318440942) bank59951 +59952 POINT(40.49591700813139 74.62556979786591) bank59952 +59953 POINT(40.451522104249804 73.28822887878958) bank59953 +59954 POINT(40.91416942806799 74.06474329456795) bank59954 +59955 POINT(40.52339346637073 74.68652793818268) bank59955 +59956 POINT(41.047754885078426 74.64855458126426) bank59956 +59957 POINT(40.924726084417514 73.5110010783944) bank59957 +59958 POINT(40.75286877971462 73.8025477570911) bank59958 +59959 POINT(41.1074531764226 74.12201734726953) bank59959 +59960 POINT(40.61128984431158 73.69915153125223) bank59960 +59961 POINT(40.62350415939322 74.43507604745518) bank59961 +59962 POINT(40.367174784631594 74.6123830272632) bank59962 +59963 POINT(41.01335012475056 74.20385420278014) bank59963 +59964 POINT(41.65513299012988 74.09769131347049) bank59964 +59965 POINT(41.29549996299743 74.85574166037073) bank59965 +59966 POINT(40.500952590962406 73.7092688861093) bank59966 +59967 POINT(39.961513421168036 74.6897621666538) bank59967 +59968 POINT(41.08481815017947 73.23370557303734) bank59968 +59969 POINT(39.77618221348394 73.65316374880598) bank59969 +59970 POINT(41.17273915517704 74.91114224158139) bank59970 +59971 POINT(39.80394164395357 73.58353844667857) bank59971 +59972 POINT(41.088117690106515 74.9272123721848) bank59972 +59973 POINT(39.90131385371646 74.98890635581188) bank59973 +59974 POINT(41.333116186394804 73.51036076430414) bank59974 +59975 POINT(39.8571335419774 73.83451189014525) bank59975 +59976 POINT(40.042902226547554 73.27933872335379) bank59976 +59977 POINT(40.85785214490437 74.64405928112903) bank59977 +59978 POINT(41.15859786994621 74.78161733641895) bank59978 +59979 POINT(40.36402398731013 73.7770318134409) bank59979 +59980 POINT(41.449326976347336 73.8578713970352) bank59980 +59981 POINT(40.128340317083875 74.13137305673058) bank59981 +59982 POINT(40.07774187025621 74.92188585101215) bank59982 +59983 POINT(40.96434675710264 74.21812118151722) bank59983 +59984 POINT(41.35078134986743 74.65957026585058) bank59984 +59985 POINT(41.28724271009083 74.9591755842269) bank59985 +59986 POINT(41.6902200165907 74.43892047025426) bank59986 +59987 POINT(39.926790135256084 74.84274656394962) bank59987 +59988 POINT(39.83687885548251 74.5477180410291) bank59988 +59989 POINT(41.35616735986849 74.65968322698636) bank59989 +59990 POINT(41.17185257035395 74.86741715119192) bank59990 +59991 POINT(41.31293193853019 74.18605688128719) bank59991 +59992 POINT(41.105384014891335 74.05632221237366) bank59992 +59993 POINT(40.648794457287735 74.88277726594657) bank59993 +59994 POINT(41.118244428120335 74.24238848413796) bank59994 +59995 POINT(40.862941122079235 74.2454993488905) bank59995 +59996 POINT(41.36812822341125 74.95552596063473) bank59996 +59997 POINT(40.715577396202534 73.99023532881654) bank59997 +59998 POINT(40.84761424886531 74.27813457868834) bank59998 +59999 POINT(40.06370651397264 74.49954321932246) bank59999 +60000 POINT(39.84692002664319 74.7900022297228) bank60000 +60001 POINT(40.66072152776683 74.94133824832097) bank60001 +60002 POINT(40.70522534600632 74.51292167005879) bank60002 +60003 POINT(40.81387278846057 73.12411585147743) bank60003 +60004 POINT(41.501655040125726 74.38326447055127) bank60004 +60005 POINT(41.29075446254043 74.76937077518949) bank60005 +60006 POINT(40.06592125190453 74.41690005492566) bank60006 +60007 POINT(40.580426377067134 74.88415091438044) bank60007 +60008 POINT(41.61789079749716 73.6185488979681) bank60008 +60009 POINT(41.03984939387064 74.63365738100576) bank60009 +60010 POINT(40.72531539031764 73.6156436144296) bank60010 +60011 POINT(41.52473692799497 73.03988367243389) bank60011 +60012 POINT(41.207288672242235 74.50191820633887) bank60012 +60013 POINT(41.02665241066943 73.16153488791397) bank60013 +60014 POINT(41.51120548005255 74.50906772509991) bank60014 +60015 POINT(41.516534104280176 74.53925308225689) bank60015 +60016 POINT(41.045691009595494 74.96221776790391) bank60016 +60017 POINT(41.26199279666342 73.56200580965478) bank60017 +60018 POINT(39.988957943270364 74.23686387778721) bank60018 +60019 POINT(41.07526017266477 73.65006807475586) bank60019 +60020 POINT(41.30743314174257 74.5349189378813) bank60020 +60021 POINT(40.61385326706111 73.24664026434412) bank60021 +60022 POINT(39.79559733672837 73.71331489104432) bank60022 +60023 POINT(40.878587408031386 74.59898646588802) bank60023 +60024 POINT(40.64197955738615 74.09282112147636) bank60024 +60025 POINT(41.542574277404256 73.98800507872036) bank60025 +60026 POINT(40.705045647112705 74.41601935989449) bank60026 +60027 POINT(39.99246496207306 74.76866565309452) bank60027 +60028 POINT(41.48700553874885 74.04741756009432) bank60028 +60029 POINT(40.41194460200253 73.14537786016818) bank60029 +60030 POINT(40.964654342181746 74.36594293024663) bank60030 +60031 POINT(41.366036699274446 74.46570453188382) bank60031 +60032 POINT(40.209012651004315 74.66647736917983) bank60032 +60033 POINT(40.27058381673718 74.45582234122747) bank60033 +60034 POINT(41.03756777546124 74.69747367526227) bank60034 +60035 POINT(41.156018702983665 74.55658965989934) bank60035 +60036 POINT(41.427919685238145 74.14584593168814) bank60036 +60037 POINT(41.643263645901875 73.15780208726507) bank60037 +60038 POINT(40.458908874948314 74.11690716447944) bank60038 +60039 POINT(40.954154407778276 74.72676909817017) bank60039 +60040 POINT(40.604741729207525 74.55834286312493) bank60040 +60041 POINT(41.5928627197782 73.97710029505565) bank60041 +60042 POINT(40.56344037157602 74.63936508211417) bank60042 +60043 POINT(41.473929854878094 74.5447785039258) bank60043 +60044 POINT(39.91282406785478 74.96317365831696) bank60044 +60045 POINT(40.681780837583865 74.11598686371951) bank60045 +60046 POINT(39.826483503235956 73.39465298319804) bank60046 +60047 POINT(41.46972014769834 74.50784590785294) bank60047 +60048 POINT(40.385798908206276 73.92151501845717) bank60048 +60049 POINT(41.215579489645975 73.1848233692264) bank60049 +60050 POINT(39.72644118654742 73.54948530169897) bank60050 +60051 POINT(40.2078667900957 74.92973943177077) bank60051 +60052 POINT(40.461415670475446 74.85688146239566) bank60052 +60053 POINT(41.48839808161759 73.3845308772166) bank60053 +60054 POINT(40.807626364505204 73.15236864527628) bank60054 +60055 POINT(40.125247217479014 73.38269124405038) bank60055 +60056 POINT(41.42938178035463 73.2558193465656) bank60056 +60057 POINT(39.98990630290043 74.19241897974183) bank60057 +60058 POINT(40.47672656478328 74.3064958271367) bank60058 +60059 POINT(41.076700656786166 74.44457274083635) bank60059 +60060 POINT(40.78979430578366 74.589773837737) bank60060 +60061 POINT(40.42792384632623 73.7926339676024) bank60061 +60062 POINT(40.52486879002613 74.83393842826516) bank60062 +60063 POINT(40.67166358743154 73.58809680582134) bank60063 +60064 POINT(41.564208183710555 73.40412104017469) bank60064 +60065 POINT(40.148052426288864 73.20271242293543) bank60065 +60066 POINT(40.79647411115324 74.46818405878227) bank60066 +60067 POINT(40.61705386839381 74.92317283063984) bank60067 +60068 POINT(41.05601801503538 74.97573920487272) bank60068 +60069 POINT(41.193726286364125 74.10357516056953) bank60069 +60070 POINT(39.96944109709153 74.13967019442732) bank60070 +60071 POINT(40.33671748442682 74.66403426786536) bank60071 +60072 POINT(41.57114012937456 74.52179979201064) bank60072 +60073 POINT(39.77887134139437 74.80696164193445) bank60073 +60074 POINT(41.6673105393927 73.44891575789899) bank60074 +60075 POINT(41.53628771967237 73.71187006998012) bank60075 +60076 POINT(41.38047124036728 73.90996467451585) bank60076 +60077 POINT(41.39861970465713 74.15938152384267) bank60077 +60078 POINT(41.22756236509442 74.09811071307962) bank60078 +60079 POINT(40.11489476116614 74.35288270455906) bank60079 +60080 POINT(41.06119618024132 74.70654608665639) bank60080 +60081 POINT(41.599575685758765 74.9810812657939) bank60081 +60082 POINT(40.378544080430125 73.94136463179271) bank60082 +60083 POINT(40.35464257816826 73.88467973046879) bank60083 +60084 POINT(39.99943929291981 73.22931120914932) bank60084 +60085 POINT(40.71830259463804 73.61219020978402) bank60085 +60086 POINT(40.05384062766129 74.85272930485293) bank60086 +60087 POINT(39.94218663329735 74.92875934746462) bank60087 +60088 POINT(41.250443269843736 73.12493831794677) bank60088 +60089 POINT(39.94910194930913 73.95124839127016) bank60089 +60090 POINT(40.23181538309688 74.88761452529921) bank60090 +60091 POINT(41.38057646333458 74.66858600797008) bank60091 +60092 POINT(40.14309693732657 74.3671195599562) bank60092 +60093 POINT(41.154536644911744 74.09212613320642) bank60093 +60094 POINT(40.976992280474434 73.68734764418025) bank60094 +60095 POINT(40.76472589565782 74.86228067128835) bank60095 +60096 POINT(40.81034774162328 73.01436824608403) bank60096 +60097 POINT(40.423682319212936 74.16303520691466) bank60097 +60098 POINT(41.59759802546484 74.81839267184249) bank60098 +60099 POINT(40.19206530158356 73.38605313454633) bank60099 +60100 POINT(40.09695524198405 73.17384471542422) bank60100 +60101 POINT(41.43874095228726 74.32478607617408) bank60101 +60102 POINT(40.63096638065712 73.21556345917536) bank60102 +60103 POINT(41.233261494664035 73.11574625469447) bank60103 +60104 POINT(40.602674600303025 73.6040749369728) bank60104 +60105 POINT(40.8344650051911 73.61756589408172) bank60105 +60106 POINT(40.77526343901268 74.41584926675938) bank60106 +60107 POINT(40.11442859948512 74.03523548327222) bank60107 +60108 POINT(41.001713746170154 73.08487250770959) bank60108 +60109 POINT(41.183772291363695 74.60802323247732) bank60109 +60110 POINT(39.80485075853531 73.35337616095703) bank60110 +60111 POINT(41.569658651468714 73.76753429396109) bank60111 +60112 POINT(40.09578835439154 74.55297715493734) bank60112 +60113 POINT(39.747482454561094 74.76980333020865) bank60113 +60114 POINT(39.78314991799818 74.21810790955193) bank60114 +60115 POINT(39.89728939161748 73.9454058599592) bank60115 +60116 POINT(40.26327960681157 73.87345996493492) bank60116 +60117 POINT(39.88372659340331 73.20342463976178) bank60117 +60118 POINT(41.52329268404231 73.2384393146635) bank60118 +60119 POINT(41.49009936371981 73.4140310151862) bank60119 +60120 POINT(41.69465916125671 73.367913405421) bank60120 +60121 POINT(41.2857525293956 73.54405816337159) bank60121 +60122 POINT(40.33939991827265 73.56955412198558) bank60122 +60123 POINT(39.92744478962466 73.18466054854109) bank60123 +60124 POINT(41.08686682705003 73.75570290279722) bank60124 +60125 POINT(41.15479886364601 73.28115568340029) bank60125 +60126 POINT(41.09925969579012 73.08305858772843) bank60126 +60127 POINT(40.30140905406662 74.04667422619248) bank60127 +60128 POINT(39.927558405885605 73.09693377432109) bank60128 +60129 POINT(41.339793345902464 73.65162958665725) bank60129 +60130 POINT(41.70478835639083 73.14373235288002) bank60130 +60131 POINT(40.08494348838842 74.7352573091435) bank60131 +60132 POINT(40.857686899054855 74.52922726610686) bank60132 +60133 POINT(40.58435756495791 74.32949244272511) bank60133 +60134 POINT(41.593665421527426 73.17114450836254) bank60134 +60135 POINT(40.94608260069653 73.95421724991493) bank60135 +60136 POINT(41.167313025553675 74.09753089830585) bank60136 +60137 POINT(40.043551024620655 73.67570695930004) bank60137 +60138 POINT(41.49955378109266 74.42899800140628) bank60138 +60139 POINT(40.882091096413184 74.11453769963137) bank60139 +60140 POINT(41.39473789403686 73.82074240451685) bank60140 +60141 POINT(40.14972402150748 74.6751056481348) bank60141 +60142 POINT(41.498167051922586 74.47638785727172) bank60142 +60143 POINT(41.41540230918095 73.99199618671875) bank60143 +60144 POINT(41.25579338397009 73.29685884840026) bank60144 +60145 POINT(40.15586471371952 73.48246917033926) bank60145 +60146 POINT(41.37971026692411 73.59055628420793) bank60146 +60147 POINT(40.14318092377768 73.48507917362775) bank60147 +60148 POINT(39.83610119758346 73.60394320914125) bank60148 +60149 POINT(40.37537005631749 74.96776961942885) bank60149 +60150 POINT(41.41847211910278 73.32228292856261) bank60150 +60151 POINT(39.82297817639642 73.59072779327208) bank60151 +60152 POINT(41.56244860657216 74.030237634673) bank60152 +60153 POINT(40.669528699537075 73.8372655687624) bank60153 +60154 POINT(39.964600353441874 74.74786183795212) bank60154 +60155 POINT(40.04164395842504 74.87682932350381) bank60155 +60156 POINT(40.56584169143401 74.02759236055746) bank60156 +60157 POINT(40.970890773040004 74.47777246337891) bank60157 +60158 POINT(40.65378245904119 73.45378565038585) bank60158 +60159 POINT(40.302566078212706 73.64236757858676) bank60159 +60160 POINT(40.73717371746406 73.12242077245298) bank60160 +60161 POINT(41.34655438690143 74.63045617027693) bank60161 +60162 POINT(41.191035170251524 73.62045466276086) bank60162 +60163 POINT(41.687562459893286 73.58162096881432) bank60163 +60164 POINT(40.59951031703909 73.77399340445311) bank60164 +60165 POINT(41.0844754944329 74.9189331682325) bank60165 +60166 POINT(39.88422230829872 74.5952990669675) bank60166 +60167 POINT(40.99962960668168 74.30859301099227) bank60167 +60168 POINT(40.829029243430284 73.7720330290644) bank60168 +60169 POINT(40.75950417823233 74.45039923115424) bank60169 +60170 POINT(41.11194088093352 74.00431248766755) bank60170 +60171 POINT(39.73436140180878 73.83443479783097) bank60171 +60172 POINT(41.45725587966468 74.28765934920774) bank60172 +60173 POINT(39.886032453370746 74.90276000655219) bank60173 +60174 POINT(40.54426952446852 73.93620058538009) bank60174 +60175 POINT(41.24133104893307 74.91561555346749) bank60175 +60176 POINT(39.83363180920057 74.30841542422581) bank60176 +60177 POINT(39.99296085633952 73.07815998571697) bank60177 +60178 POINT(41.38761240225472 73.53435823159782) bank60178 +60179 POINT(40.70351202664629 73.78166733151251) bank60179 +60180 POINT(40.554057097755894 74.30433852721328) bank60180 +60181 POINT(39.945250817901425 73.11963595866388) bank60181 +60182 POINT(41.00842739949384 74.02523412991209) bank60182 +60183 POINT(40.46941571166965 73.52724690299974) bank60183 +60184 POINT(41.6194194063197 73.75277555369003) bank60184 +60185 POINT(41.336386664795725 73.84816942696268) bank60185 +60186 POINT(40.96977284530334 74.71533008229687) bank60186 +60187 POINT(39.8556473759487 74.58867503256012) bank60187 +60188 POINT(39.97998536652726 74.98239391160558) bank60188 +60189 POINT(39.797158496816905 73.5894424434633) bank60189 +60190 POINT(41.34285817379229 74.30574477336421) bank60190 +60191 POINT(39.844144127570196 73.06750161559445) bank60191 +60192 POINT(40.52773812679383 73.26303423032103) bank60192 +60193 POINT(40.51748144117426 74.0973019327919) bank60193 +60194 POINT(39.911938249685704 73.91423381393946) bank60194 +60195 POINT(41.25291515945689 74.35395305865904) bank60195 +60196 POINT(40.64991097011036 73.55523412766833) bank60196 +60197 POINT(39.835505557675944 73.25277689685306) bank60197 +60198 POINT(39.98970855275294 73.24530805468784) bank60198 +60199 POINT(40.85238698716945 74.87256191455852) bank60199 +60200 POINT(41.246061102047264 74.82787368214062) bank60200 +60201 POINT(40.402721747406844 74.24411177604047) bank60201 +60202 POINT(39.98111639603281 74.1382516515686) bank60202 +60203 POINT(40.74540081023684 73.99733022505508) bank60203 +60204 POINT(41.57904703229133 74.91124185528697) bank60204 +60205 POINT(39.974847906374094 74.27685968587824) bank60205 +60206 POINT(40.95333038607121 74.08849952039303) bank60206 +60207 POINT(40.649676314499544 73.56143931453808) bank60207 +60208 POINT(39.74393449109408 74.47255580018022) bank60208 +60209 POINT(41.366112178474275 74.59650511348808) bank60209 +60210 POINT(40.33188339109997 74.65057660110654) bank60210 +60211 POINT(41.22659621062129 73.18473542719182) bank60211 +60212 POINT(40.455251997914466 74.95472874430249) bank60212 +60213 POINT(40.35407257613763 73.29088063926766) bank60213 +60214 POINT(41.707742571775775 74.04125662799979) bank60214 +60215 POINT(40.195167229020484 74.6125296139826) bank60215 +60216 POINT(40.5187756506399 74.21940187557837) bank60216 +60217 POINT(40.505712882188156 74.97463196119611) bank60217 +60218 POINT(40.99226906652009 74.27958301434603) bank60218 +60219 POINT(40.68029162993565 73.67527537842417) bank60219 +60220 POINT(40.89793606068797 73.2203712524097) bank60220 +60221 POINT(41.02677004582619 74.12963941502355) bank60221 +60222 POINT(40.05359863210929 73.86854272129438) bank60222 +60223 POINT(40.29320079180389 73.17131044761608) bank60223 +60224 POINT(39.832643403444564 74.28558553912292) bank60224 +60225 POINT(39.79389513004098 73.20566701098201) bank60225 +60226 POINT(40.07669536656934 74.26324704046468) bank60226 +60227 POINT(40.677835709475715 74.82723835166905) bank60227 +60228 POINT(39.76043149588482 74.77839927063098) bank60228 +60229 POINT(41.4407183891272 73.76391761257113) bank60229 +60230 POINT(41.48058544633825 73.26626131583373) bank60230 +60231 POINT(41.1282149530058 73.77445929760529) bank60231 +60232 POINT(40.701806687093566 74.00060601129849) bank60232 +60233 POINT(40.763267406839034 74.3789804058185) bank60233 +60234 POINT(41.58347166270602 74.2373315238878) bank60234 +60235 POINT(40.7966525293938 74.3049492167196) bank60235 +60236 POINT(41.070472478885506 74.79317581505494) bank60236 +60237 POINT(40.377089089675145 74.05802046985752) bank60237 +60238 POINT(40.47616587331831 74.93717155847483) bank60238 +60239 POINT(39.922056452799595 73.69842489003544) bank60239 +60240 POINT(41.25361924561679 73.33792157443075) bank60240 +60241 POINT(41.277114497820776 74.5157307791389) bank60241 +60242 POINT(40.90822273979923 74.49016087744742) bank60242 +60243 POINT(40.608559162482464 73.03138706704895) bank60243 +60244 POINT(41.64410298363668 74.15645659820545) bank60244 +60245 POINT(41.296725884784216 74.97014883072112) bank60245 +60246 POINT(41.61144112744414 74.39915692566811) bank60246 +60247 POINT(40.264099881371145 73.82683113278) bank60247 +60248 POINT(40.6941786545373 73.71681135650529) bank60248 +60249 POINT(40.8322429948581 74.12764623601757) bank60249 +60250 POINT(41.51190070162566 73.75760147797182) bank60250 +60251 POINT(40.27245960808745 73.75365415265401) bank60251 +60252 POINT(40.655388330574326 74.75938083448608) bank60252 +60253 POINT(41.58061494009964 74.69044796847687) bank60253 +60254 POINT(40.99954129563448 74.22305707419312) bank60254 +60255 POINT(40.25949721652401 73.02000423689219) bank60255 +60256 POINT(41.20823423115399 73.27183115549971) bank60256 +60257 POINT(40.59215919439535 74.42819705393413) bank60257 +60258 POINT(40.88112962301457 73.27084100031308) bank60258 +60259 POINT(41.563721517331196 73.95015868254681) bank60259 +60260 POINT(41.05915292430667 73.10097242927274) bank60260 +60261 POINT(41.49639996063283 73.88139979034833) bank60261 +60262 POINT(40.574608553777125 74.18265774257699) bank60262 +60263 POINT(40.79213262890339 74.81962788129275) bank60263 +60264 POINT(39.881760524552774 73.5858520227414) bank60264 +60265 POINT(39.73687192584681 73.06240398728247) bank60265 +60266 POINT(41.22510802444286 74.2524304856571) bank60266 +60267 POINT(41.38367374176084 73.78362618127635) bank60267 +60268 POINT(40.06407112624059 74.93590106759193) bank60268 +60269 POINT(40.42362396966216 73.47559732981787) bank60269 +60270 POINT(41.1517214093278 74.6523005242392) bank60270 +60271 POINT(41.36470885867673 73.90919006204713) bank60271 +60272 POINT(40.98049867666585 74.2427838650914) bank60272 +60273 POINT(41.21385910355504 73.08413007025405) bank60273 +60274 POINT(40.865089375528186 74.7652562041976) bank60274 +60275 POINT(39.86277147853335 73.62654138735613) bank60275 +60276 POINT(40.19256042127159 73.30370778322785) bank60276 +60277 POINT(39.94588306819264 73.37409410861434) bank60277 +60278 POINT(41.195797887234875 74.64409838633237) bank60278 +60279 POINT(40.12662994172295 73.37963277171839) bank60279 +60280 POINT(40.12866393450391 73.94659296153476) bank60280 +60281 POINT(40.37059012272286 74.03377102070816) bank60281 +60282 POINT(41.60819583700703 74.35880897703116) bank60282 +60283 POINT(40.20326704439401 74.14899288080707) bank60283 +60284 POINT(39.95075131222322 73.68025730156359) bank60284 +60285 POINT(40.10958307313482 73.6974473583731) bank60285 +60286 POINT(41.48450681545949 73.30499166279078) bank60286 +60287 POINT(39.75667700389131 74.62806283167669) bank60287 +60288 POINT(40.81715072718598 73.26977333108711) bank60288 +60289 POINT(40.131991906605805 74.44222522406523) bank60289 +60290 POINT(40.493368517517844 74.95154024677296) bank60290 +60291 POINT(40.889210781892594 74.4989317628824) bank60291 +60292 POINT(40.74201596203525 74.69097473260305) bank60292 +60293 POINT(39.8150110338316 74.38221819212231) bank60293 +60294 POINT(41.147884882930505 73.39678548264303) bank60294 +60295 POINT(41.477288871979766 74.67291592983516) bank60295 +60296 POINT(41.19445165705319 74.8410041851742) bank60296 +60297 POINT(39.758141576134605 73.57642647192641) bank60297 +60298 POINT(40.037084897433296 74.69838083582935) bank60298 +60299 POINT(40.406960530651425 73.99481483128523) bank60299 +60300 POINT(40.576100581038894 73.52274389454816) bank60300 +60301 POINT(40.78301146665177 74.70204779945551) bank60301 +60302 POINT(40.34259721798226 74.91280333623658) bank60302 +60303 POINT(39.74780153308576 74.90788589059511) bank60303 +60304 POINT(40.37740437432499 74.88473607777809) bank60304 +60305 POINT(40.582635798766574 73.51483785150229) bank60305 +60306 POINT(40.509782088865514 73.57968387210994) bank60306 +60307 POINT(39.827434356266075 74.13219351229127) bank60307 +60308 POINT(39.82512158585609 73.1460546209303) bank60308 +60309 POINT(41.18807252238481 73.07264993107208) bank60309 +60310 POINT(41.19232848332356 74.39945650723922) bank60310 +60311 POINT(41.449603007845724 73.8197224904325) bank60311 +60312 POINT(40.35841146553197 73.50595749285945) bank60312 +60313 POINT(41.25803744370949 73.1958702193751) bank60313 +60314 POINT(40.96143514714181 73.73754387128679) bank60314 +60315 POINT(41.16785170613033 73.179078670694) bank60315 +60316 POINT(41.31870526865534 74.44428624045757) bank60316 +60317 POINT(40.87924572871096 74.16916151072611) bank60317 +60318 POINT(40.53294624666179 74.24659978633287) bank60318 +60319 POINT(40.05485612837638 74.17904079519363) bank60319 +60320 POINT(40.92125980697543 73.70544535244336) bank60320 +60321 POINT(41.00950085981873 74.47616294057829) bank60321 +60322 POINT(39.8250759551276 73.18588542555713) bank60322 +60323 POINT(40.52044065149924 73.57863367915733) bank60323 +60324 POINT(39.959345080886784 73.45848391320808) bank60324 +60325 POINT(40.488508993028766 74.58768377564769) bank60325 +60326 POINT(41.321428401981585 73.26527242679454) bank60326 +60327 POINT(41.0373009018801 73.91471178218166) bank60327 +60328 POINT(40.197606640577554 73.95199414417102) bank60328 +60329 POINT(40.31159811980423 74.03982170875419) bank60329 +60330 POINT(40.92242153196205 74.74604475435413) bank60330 +60331 POINT(40.80138511599076 74.57813894226054) bank60331 +60332 POINT(40.4658169538044 74.73967382568661) bank60332 +60333 POINT(40.95956365881676 74.0864847908526) bank60333 +60334 POINT(39.72202552680872 73.58087384304605) bank60334 +60335 POINT(40.8083586293239 73.52241868708603) bank60335 +60336 POINT(40.56470476146423 73.11702472633367) bank60336 +60337 POINT(40.33695106229864 73.04275790119648) bank60337 +60338 POINT(41.52108387016358 74.13240015902825) bank60338 +60339 POINT(40.01991851401211 73.58627836397447) bank60339 +60340 POINT(41.220123474156104 74.1786850036217) bank60340 +60341 POINT(40.286583244880724 73.60844679568682) bank60341 +60342 POINT(39.93319777526566 73.03878572619045) bank60342 +60343 POINT(40.77288741004775 74.24929022463597) bank60343 +60344 POINT(40.768965454359 74.90099065041862) bank60344 +60345 POINT(41.42550108049931 74.66591393890259) bank60345 +60346 POINT(41.49562192192479 73.02488620738085) bank60346 +60347 POINT(39.843995930326294 73.67820362430446) bank60347 +60348 POINT(39.86446909533624 74.30271335064894) bank60348 +60349 POINT(39.76657472302476 74.7131591071194) bank60349 +60350 POINT(39.831575146635856 73.54446284930029) bank60350 +60351 POINT(41.01286452232319 74.8311206568521) bank60351 +60352 POINT(39.88464796573833 73.93184596875854) bank60352 +60353 POINT(40.91268297023388 73.1427881449777) bank60353 +60354 POINT(41.393826072420175 73.64141264433115) bank60354 +60355 POINT(39.86081308114752 74.06516874380573) bank60355 +60356 POINT(39.768134139225815 74.08040859268293) bank60356 +60357 POINT(40.036314684393076 74.98098776444631) bank60357 +60358 POINT(41.093452656592085 73.72412680725132) bank60358 +60359 POINT(40.713461449041546 74.37218265954122) bank60359 +60360 POINT(39.73157997757714 73.4752289533357) bank60360 +60361 POINT(41.640890053965 74.11110325594096) bank60361 +60362 POINT(40.89564953842729 74.48548909981342) bank60362 +60363 POINT(40.05506890716089 74.94089167563548) bank60363 +60364 POINT(40.85958113256753 73.89171358485865) bank60364 +60365 POINT(40.98811864618181 73.7715847423486) bank60365 +60366 POINT(40.38186727011712 74.3440511230411) bank60366 +60367 POINT(40.034997323924806 74.9976385132666) bank60367 +60368 POINT(39.81171260139085 74.14108131479193) bank60368 +60369 POINT(40.590897727526276 73.75088969999119) bank60369 +60370 POINT(40.37740400972608 73.89941744682987) bank60370 +60371 POINT(41.340456935701106 73.66076257380149) bank60371 +60372 POINT(40.33457412773004 73.46814674267151) bank60372 +60373 POINT(40.95516448920771 73.37072626380534) bank60373 +60374 POINT(40.1210393210863 73.68725133526362) bank60374 +60375 POINT(40.74116457015836 73.35559546960677) bank60375 +60376 POINT(41.637997561883225 73.17619280046871) bank60376 +60377 POINT(40.10340617699471 74.32042915312213) bank60377 +60378 POINT(40.62776275174591 73.79251156294147) bank60378 +60379 POINT(40.768759775623565 73.54706385818108) bank60379 +60380 POINT(40.130126488887996 74.88110976035769) bank60380 +60381 POINT(41.01180237140959 74.8332905899797) bank60381 +60382 POINT(41.23244306880363 74.12093079286302) bank60382 +60383 POINT(40.26875641097265 73.76677064197503) bank60383 +60384 POINT(40.00263124337301 74.1592344645306) bank60384 +60385 POINT(40.40577136625222 73.22806257291292) bank60385 +60386 POINT(40.79666763939877 74.87952572847267) bank60386 +60387 POINT(41.422852255336466 73.55445648239146) bank60387 +60388 POINT(40.79014982427942 74.99152045843374) bank60388 +60389 POINT(40.942736474933874 73.30474057949127) bank60389 +60390 POINT(41.01438218921003 73.90045073293136) bank60390 +60391 POINT(40.48011216653177 74.73271527574651) bank60391 +60392 POINT(41.20165048442254 73.62584453821064) bank60392 +60393 POINT(40.82315509922601 73.60175935889816) bank60393 +60394 POINT(41.171363377772614 74.53897211244546) bank60394 +60395 POINT(40.69636967013692 73.46191230389822) bank60395 +60396 POINT(41.60807211382691 73.9556793648803) bank60396 +60397 POINT(40.88499927409416 74.10278952226582) bank60397 +60398 POINT(41.25550314434883 74.35424406660069) bank60398 +60399 POINT(41.577638555162906 73.91654968692067) bank60399 +60400 POINT(40.39189375864256 74.2868205136682) bank60400 +60401 POINT(41.455192786903936 73.1721417032716) bank60401 +60402 POINT(40.22424394367497 74.52567065057359) bank60402 +60403 POINT(40.96084703950788 74.00050286536187) bank60403 +60404 POINT(40.5300324101485 74.50742132271519) bank60404 +60405 POINT(40.32657075703807 73.71292930266726) bank60405 +60406 POINT(40.262014888831075 73.90254969296916) bank60406 +60407 POINT(41.572475721685166 73.65610667477898) bank60407 +60408 POINT(40.606470463776255 73.87324799532351) bank60408 +60409 POINT(40.75959073004128 73.62803204693935) bank60409 +60410 POINT(41.608542769563414 73.15889848019658) bank60410 +60411 POINT(40.647380932013824 73.64678897992135) bank60411 +60412 POINT(41.374070230969394 73.19071409578973) bank60412 +60413 POINT(41.64050364348907 74.67729536768218) bank60413 +60414 POINT(41.20980768205772 73.86535360528757) bank60414 +60415 POINT(40.51487033618373 73.01097347303242) bank60415 +60416 POINT(41.0684997361656 73.55650076048985) bank60416 +60417 POINT(40.54009885648142 74.65817877649226) bank60417 +60418 POINT(41.18133742175426 73.54093175300386) bank60418 +60419 POINT(39.95454906020474 74.29504577139465) bank60419 +60420 POINT(41.35532754207603 73.88604168695913) bank60420 +60421 POINT(41.00423736715893 74.25872875370696) bank60421 +60422 POINT(40.04824090481254 73.06710288352465) bank60422 +60423 POINT(41.261123336503864 73.63798804606377) bank60423 +60424 POINT(41.311539886224715 73.23883773616224) bank60424 +60425 POINT(40.968853972771576 73.68412823761402) bank60425 +60426 POINT(40.55038606762768 74.76402223447094) bank60426 +60427 POINT(41.23259203454316 73.29735626554051) bank60427 +60428 POINT(40.318819717246235 73.5550871610029) bank60428 +60429 POINT(41.25285565952489 74.5977605556994) bank60429 +60430 POINT(40.69093734572377 73.81143128488362) bank60430 +60431 POINT(40.85217291204833 73.01768613576547) bank60431 +60432 POINT(40.75319410696148 73.73470184464767) bank60432 +60433 POINT(41.66004762677628 74.78605528136258) bank60433 +60434 POINT(40.60535845055821 73.15020965463628) bank60434 +60435 POINT(41.331128035431576 73.23275645363373) bank60435 +60436 POINT(40.71234811555458 74.34600106230879) bank60436 +60437 POINT(39.755845109279264 74.03944318262276) bank60437 +60438 POINT(40.46564403863936 74.42054907505783) bank60438 +60439 POINT(40.06430170965074 73.32749950907767) bank60439 +60440 POINT(41.315120411371076 74.79574790340416) bank60440 +60441 POINT(41.58606740552005 74.6081686723174) bank60441 +60442 POINT(40.71312505741045 73.18093396882095) bank60442 +60443 POINT(41.091631546016664 73.98338533960847) bank60443 +60444 POINT(40.69303708941921 74.64358931950837) bank60444 +60445 POINT(40.972459826767086 73.93859574451209) bank60445 +60446 POINT(40.42588772568486 73.07522232765287) bank60446 +60447 POINT(40.53654405226814 73.76542483842766) bank60447 +60448 POINT(41.4471235206078 74.6931417731164) bank60448 +60449 POINT(40.0893534736353 73.30803974050707) bank60449 +60450 POINT(41.294923468262255 74.48067355253346) bank60450 +60451 POINT(39.965441776049076 73.67654754187066) bank60451 +60452 POINT(41.62258786357637 73.62921119822418) bank60452 +60453 POINT(41.06580439006683 73.57959872983248) bank60453 +60454 POINT(40.37399529535555 74.282648812299) bank60454 +60455 POINT(40.15232173367903 74.58888506938851) bank60455 +60456 POINT(40.444901109716064 73.71968623204192) bank60456 +60457 POINT(41.711747970039134 74.77150843896338) bank60457 +60458 POINT(40.24510360482008 74.89167127771675) bank60458 +60459 POINT(41.3438666387422 73.18230495464728) bank60459 +60460 POINT(40.95854123976282 74.82592250043875) bank60460 +60461 POINT(41.47898001598671 74.1643254488095) bank60461 +60462 POINT(41.13881102022081 73.73045706719124) bank60462 +60463 POINT(40.22146060764451 74.6784435456196) bank60463 +60464 POINT(40.87607651176067 75.00147826841346) bank60464 +60465 POINT(40.02942173251334 74.14613170353786) bank60465 +60466 POINT(40.94669167715455 74.31024928328611) bank60466 +60467 POINT(39.996805295057364 73.79664644686387) bank60467 +60468 POINT(39.80672027473426 73.29929701938228) bank60468 +60469 POINT(40.62620578564511 74.76690298336003) bank60469 +60470 POINT(40.53589261699473 73.55571228957459) bank60470 +60471 POINT(40.68197078917994 74.32935894480772) bank60471 +60472 POINT(40.362785411222276 74.40803381222467) bank60472 +60473 POINT(39.91085055239357 73.74547309523234) bank60473 +60474 POINT(40.56446753415308 73.4731353977367) bank60474 +60475 POINT(41.60664887984521 73.72130292044932) bank60475 +60476 POINT(39.85518401505733 73.92614697574255) bank60476 +60477 POINT(41.32836875340521 73.13892819416714) bank60477 +60478 POINT(39.79403221608975 74.94200004504985) bank60478 +60479 POINT(39.90668491050347 73.54450857396525) bank60479 +60480 POINT(40.86814567404927 73.5810996913729) bank60480 +60481 POINT(39.720037854078825 73.91336206150119) bank60481 +60482 POINT(41.16623700018451 74.13997248722947) bank60482 +60483 POINT(40.46962942938665 73.21534824634375) bank60483 +60484 POINT(41.38571972004954 74.44480659735778) bank60484 +60485 POINT(40.95414519372508 73.95051132374074) bank60485 +60486 POINT(40.385956188898525 74.50044395060655) bank60486 +60487 POINT(41.325765371905625 73.57908717251009) bank60487 +60488 POINT(40.2106869683444 74.76825697122996) bank60488 +60489 POINT(41.2440522191673 73.51889781099467) bank60489 +60490 POINT(40.015624083935094 73.43278324084226) bank60490 +60491 POINT(41.2591998113546 74.90864380728789) bank60491 +60492 POINT(39.82142791590917 73.33594686672497) bank60492 +60493 POINT(41.183434633630604 73.60016052586285) bank60493 +60494 POINT(41.095951840937566 74.0270149244379) bank60494 +60495 POINT(40.92970821121227 74.52997864012717) bank60495 +60496 POINT(39.90795875753721 74.2341127657136) bank60496 +60497 POINT(40.000732816368824 74.348446654516) bank60497 +60498 POINT(40.59545483548284 73.26803824369618) bank60498 +60499 POINT(41.05571015803102 73.5260987140655) bank60499 +60500 POINT(39.740174454793724 73.07646805208935) bank60500 +60501 POINT(41.57555945348746 74.16026453184661) bank60501 +60502 POINT(41.38406585530425 73.84908239636897) bank60502 +60503 POINT(41.03713512196718 74.9734102790974) bank60503 +60504 POINT(41.06134821572873 73.95712553068351) bank60504 +60505 POINT(40.39693084753901 73.47083086342194) bank60505 +60506 POINT(41.473510218914065 74.41959120596158) bank60506 +60507 POINT(41.37521481903451 73.10210134815304) bank60507 +60508 POINT(39.90912990666145 73.39743262334409) bank60508 +60509 POINT(40.430537762671854 74.99005298228732) bank60509 +60510 POINT(40.391479194358084 73.12013256589746) bank60510 +60511 POINT(41.191123848436646 74.06145438497957) bank60511 +60512 POINT(41.35199865597134 74.54438709797303) bank60512 +60513 POINT(41.45860570330075 73.82172955325345) bank60513 +60514 POINT(39.74583963994293 74.71045944252076) bank60514 +60515 POINT(41.55658607835872 74.11434870401507) bank60515 +60516 POINT(40.88942476546066 74.06176131191458) bank60516 +60517 POINT(40.912835272623305 74.92503264924717) bank60517 +60518 POINT(40.97204721510533 74.44760008831894) bank60518 +60519 POINT(41.10376562061361 74.50717789042827) bank60519 +60520 POINT(40.7202804095197 74.0697323578051) bank60520 +60521 POINT(40.8874795340081 73.45868424716434) bank60521 +60522 POINT(40.43743082682098 73.81147354053203) bank60522 +60523 POINT(41.54250099644494 73.74094350410279) bank60523 +60524 POINT(41.558380528272416 73.35718462760578) bank60524 +60525 POINT(41.12187308347581 73.6512238237232) bank60525 +60526 POINT(41.03489181504266 74.56378934964384) bank60526 +60527 POINT(41.273765924027025 73.45751704354291) bank60527 +60528 POINT(40.334586265625525 73.42772691319634) bank60528 +60529 POINT(40.29318717431155 74.55001975036397) bank60529 +60530 POINT(41.17724803165055 74.12693156284658) bank60530 +60531 POINT(40.16723189740624 74.30580958600575) bank60531 +60532 POINT(41.55393237822885 74.41960536884869) bank60532 +60533 POINT(41.6051639986976 73.94615142305405) bank60533 +60534 POINT(40.65432233348313 74.42711876827227) bank60534 +60535 POINT(40.06365131213405 74.3722415434124) bank60535 +60536 POINT(40.324674440547525 73.91716491815379) bank60536 +60537 POINT(41.55901955178441 73.37995223890597) bank60537 +60538 POINT(41.252076530690275 73.48389816452828) bank60538 +60539 POINT(41.692775781510456 74.73920024106332) bank60539 +60540 POINT(40.64628152097909 73.49128818471685) bank60540 +60541 POINT(39.92001193393337 73.67772123986107) bank60541 +60542 POINT(40.41601297688834 74.69311184448611) bank60542 +60543 POINT(40.73977384807277 74.11292815195381) bank60543 +60544 POINT(40.12325693551943 73.28322020981554) bank60544 +60545 POINT(41.04440284869273 74.02711671472184) bank60545 +60546 POINT(40.28251823833622 73.41159265816692) bank60546 +60547 POINT(40.38974880736275 73.67408469504258) bank60547 +60548 POINT(40.57092304011485 73.05348277742365) bank60548 +60549 POINT(41.30166998004723 73.34005423901418) bank60549 +60550 POINT(40.13780810478445 74.45780798730947) bank60550 +60551 POINT(40.54354899751553 73.05281542201485) bank60551 +60552 POINT(41.473127238119 74.18515833904344) bank60552 +60553 POINT(40.817306875703686 74.90853211036034) bank60553 +60554 POINT(40.80997881488623 73.13435219563397) bank60554 +60555 POINT(40.281832015796084 74.84137521867193) bank60555 +60556 POINT(40.982432582839444 73.66091341174156) bank60556 +60557 POINT(39.82558254932382 73.44676144587629) bank60557 +60558 POINT(41.255775619114964 73.984593216699) bank60558 +60559 POINT(40.8685260710794 74.54242895681504) bank60559 +60560 POINT(39.99616797564695 73.39113513939195) bank60560 +60561 POINT(40.92286602790402 73.07925114952968) bank60561 +60562 POINT(41.31886446808945 73.8204699757044) bank60562 +60563 POINT(39.7733509215763 74.66825282335573) bank60563 +60564 POINT(40.74227234664177 74.7558721665084) bank60564 +60565 POINT(41.194307163467954 74.3789044894715) bank60565 +60566 POINT(40.709284131523916 73.21212692317626) bank60566 +60567 POINT(40.7260671869276 73.31951736434944) bank60567 +60568 POINT(41.23698722205089 73.06012462429906) bank60568 +60569 POINT(41.56765341498147 73.81290008804) bank60569 +60570 POINT(41.172841197013405 73.61365082353798) bank60570 +60571 POINT(39.79239082860563 73.98976671103253) bank60571 +60572 POINT(40.48089160060312 74.95268844523439) bank60572 +60573 POINT(41.070719798516095 73.79006495437632) bank60573 +60574 POINT(41.006203719218114 74.03510780514078) bank60574 +60575 POINT(39.83859694399146 74.84448838726622) bank60575 +60576 POINT(40.47680558048153 74.55452613889688) bank60576 +60577 POINT(41.451375353274905 74.23598591111988) bank60577 +60578 POINT(40.91221767127424 74.74153279551763) bank60578 +60579 POINT(40.7093035094185 74.72182526572686) bank60579 +60580 POINT(40.35280652858884 73.19649200118779) bank60580 +60581 POINT(39.88203747617555 73.88916105701671) bank60581 +60582 POINT(40.49308516326647 74.09928138984134) bank60582 +60583 POINT(40.69537808683203 73.5457231012893) bank60583 +60584 POINT(41.29698268330323 74.97584303125605) bank60584 +60585 POINT(40.129714314231194 74.36193456816208) bank60585 +60586 POINT(41.2620051586383 73.6732573005215) bank60586 +60587 POINT(40.561669208128244 74.59072267380255) bank60587 +60588 POINT(41.37983011037622 73.92734546624929) bank60588 +60589 POINT(40.84022937991789 74.04258159174563) bank60589 +60590 POINT(41.4253204709158 73.77963681841764) bank60590 +60591 POINT(41.33532487797421 73.32894789921116) bank60591 +60592 POINT(41.656274932209726 73.39485475001695) bank60592 +60593 POINT(40.76399187523705 74.69468432994803) bank60593 +60594 POINT(41.01800635834939 73.77413659646413) bank60594 +60595 POINT(41.62010253111454 74.16580361294177) bank60595 +60596 POINT(41.62600079374291 74.37385179127921) bank60596 +60597 POINT(40.042260061217654 73.53010906925813) bank60597 +60598 POINT(40.65797324535273 73.96314494511809) bank60598 +60599 POINT(41.000502380453916 74.01401151531968) bank60599 +60600 POINT(40.17248974731785 73.16808669095585) bank60600 +60601 POINT(40.77417465474484 73.18359691206298) bank60601 +60602 POINT(39.96481584577593 74.58107002018426) bank60602 +60603 POINT(40.213567736830825 73.18791303124152) bank60603 +60604 POINT(40.26395784219978 73.33530228094554) bank60604 +60605 POINT(40.913007988747495 73.86412185347658) bank60605 +60606 POINT(40.1033611189255 74.2493611105525) bank60606 +60607 POINT(40.53341813200131 74.9767530670494) bank60607 +60608 POINT(41.0049755517877 73.87052971087672) bank60608 +60609 POINT(41.497555125211036 73.38859813888769) bank60609 +60610 POINT(41.01662473621457 73.29577829218819) bank60610 +60611 POINT(40.250547043745655 73.64672899606536) bank60611 +60612 POINT(40.29555630416523 73.44954019896284) bank60612 +60613 POINT(41.35704698456377 74.23536912264109) bank60613 +60614 POINT(40.92662826208529 73.01538404031298) bank60614 +60615 POINT(41.28310634101581 73.76391996298827) bank60615 +60616 POINT(41.4458398274068 73.43585807004492) bank60616 +60617 POINT(41.32974114630189 73.98877782779554) bank60617 +60618 POINT(40.80744405190026 74.59907543548888) bank60618 +60619 POINT(40.696811763883034 73.01856540147544) bank60619 +60620 POINT(40.2722623241655 74.65696787935964) bank60620 +60621 POINT(40.0912772990802 74.88425768445234) bank60621 +60622 POINT(41.14869697224718 73.87510650770842) bank60622 +60623 POINT(41.05174002787511 74.73089260608992) bank60623 +60624 POINT(39.76840634358809 73.8395217899104) bank60624 +60625 POINT(39.88169077303701 73.81033298400426) bank60625 +60626 POINT(41.249248793419206 73.47213078993624) bank60626 +60627 POINT(40.717323076701604 74.84316119941359) bank60627 +60628 POINT(40.88475268266887 74.36171138648491) bank60628 +60629 POINT(39.9894405552619 73.41064559627434) bank60629 +60630 POINT(40.50220511848308 73.37639017090095) bank60630 +60631 POINT(40.43797152583989 74.18107075468407) bank60631 +60632 POINT(41.11764671995439 74.46025864206004) bank60632 +60633 POINT(40.64509457749044 73.31624574005258) bank60633 +60634 POINT(40.55362923795049 73.20497174640414) bank60634 +60635 POINT(39.890938761905986 73.05381582057532) bank60635 +60636 POINT(40.74802107608637 74.00513391463322) bank60636 +60637 POINT(41.5418327782644 74.15453132548205) bank60637 +60638 POINT(40.651357206375046 73.97721846314283) bank60638 +60639 POINT(41.28857450324683 73.35592816409842) bank60639 +60640 POINT(40.25441393745993 73.70869022834354) bank60640 +60641 POINT(41.13758807665764 73.30334816553595) bank60641 +60642 POINT(40.31554580868689 74.45655988430431) bank60642 +60643 POINT(40.09206244771339 73.79404781237416) bank60643 +60644 POINT(39.843934734578404 73.47094289169011) bank60644 +60645 POINT(41.32579827345596 73.71227166235242) bank60645 +60646 POINT(40.45620539563625 73.7881064042062) bank60646 +60647 POINT(40.99997414631687 74.52197434901822) bank60647 +60648 POINT(41.67358162298166 73.93753901446063) bank60648 +60649 POINT(41.04793839394266 73.47984188781915) bank60649 +60650 POINT(39.793585647198164 74.82749100094392) bank60650 +60651 POINT(41.083625954307784 74.01779417974753) bank60651 +60652 POINT(40.500487014637976 74.70625292049385) bank60652 +60653 POINT(40.78690969579347 73.99646210920659) bank60653 +60654 POINT(41.71171715125753 73.69475278273748) bank60654 +60655 POINT(41.12356975399162 74.47788024642482) bank60655 +60656 POINT(39.80258716164481 74.65050188503947) bank60656 +60657 POINT(41.14315229312138 74.37167118232065) bank60657 +60658 POINT(41.67863396362782 73.4441568975609) bank60658 +60659 POINT(41.60438936186192 73.16809996848892) bank60659 +60660 POINT(40.40343775809022 73.03453514048581) bank60660 +60661 POINT(41.117254741391626 73.92429350735348) bank60661 +60662 POINT(40.98575139799302 74.24913189569656) bank60662 +60663 POINT(40.82678589962815 74.68224220731621) bank60663 +60664 POINT(39.923887015007324 73.94200837884416) bank60664 +60665 POINT(41.59938824803885 74.8725489214744) bank60665 +60666 POINT(41.292645738227016 74.03221896164995) bank60666 +60667 POINT(40.48243727139533 73.40554286384915) bank60667 +60668 POINT(40.84675514965194 73.03014647034462) bank60668 +60669 POINT(41.18726667189382 73.21145839280395) bank60669 +60670 POINT(40.67709692384089 74.3724072884534) bank60670 +60671 POINT(41.665580846414045 74.95607895972911) bank60671 +60672 POINT(41.0776396412907 74.69973085846928) bank60672 +60673 POINT(40.15973314038327 74.11988944177926) bank60673 +60674 POINT(41.62291064088355 73.56310791927127) bank60674 +60675 POINT(41.61912197660451 73.02894241037531) bank60675 +60676 POINT(41.672736398023225 73.92683055536074) bank60676 +60677 POINT(40.488089543437354 74.91054427745301) bank60677 +60678 POINT(41.094401021771596 74.61989635733448) bank60678 +60679 POINT(40.441551865817964 73.60195645008989) bank60679 +60680 POINT(40.56926919500253 73.80500135005222) bank60680 +60681 POINT(39.97546228527426 74.20662918696125) bank60681 +60682 POINT(40.45332167042743 73.63185356727038) bank60682 +60683 POINT(41.32462361441234 74.1511414144163) bank60683 +60684 POINT(39.91174367063502 74.65084538055505) bank60684 +60685 POINT(41.022452365540985 73.84445525310143) bank60685 +60686 POINT(41.610821490151196 73.51874087497562) bank60686 +60687 POINT(41.39092806352797 74.00533628799299) bank60687 +60688 POINT(40.61306995399077 73.05824175804823) bank60688 +60689 POINT(41.09718648351375 73.82741526579179) bank60689 +60690 POINT(41.59579170825192 73.96733973819033) bank60690 +60691 POINT(41.391977693443344 74.06199714718437) bank60691 +60692 POINT(40.975564953631086 73.87511158870518) bank60692 +60693 POINT(39.77703605074515 74.3650053725979) bank60693 +60694 POINT(41.204030874859306 73.65352853369914) bank60694 +60695 POINT(40.74261564110273 73.92546931757077) bank60695 +60696 POINT(39.8553962236583 73.40606440212729) bank60696 +60697 POINT(40.60432662324662 73.54683365334603) bank60697 +60698 POINT(41.66782192167396 74.77604038102298) bank60698 +60699 POINT(40.0695082587134 73.30405773736665) bank60699 +60700 POINT(40.87374969098747 74.6648797812083) bank60700 +60701 POINT(40.72828488853827 74.0133846557414) bank60701 +60702 POINT(40.48970620852958 74.34503536337438) bank60702 +60703 POINT(41.25292749573114 73.75965812404932) bank60703 +60704 POINT(41.5434555434567 73.9063701130568) bank60704 +60705 POINT(40.54143277137142 74.93492756407763) bank60705 +60706 POINT(41.362620601766245 75.00425378411751) bank60706 +60707 POINT(41.03102252883651 73.60625023226659) bank60707 +60708 POINT(40.55117932260596 73.5675383881782) bank60708 +60709 POINT(40.10308105115335 74.45869050037203) bank60709 +60710 POINT(41.05386019126368 74.31577095076301) bank60710 +60711 POINT(40.12707471829759 74.14960778877504) bank60711 +60712 POINT(40.96165547291234 73.80058333498827) bank60712 +60713 POINT(40.50023602948101 74.74017718428753) bank60713 +60714 POINT(40.01630231686304 73.6184706751257) bank60714 +60715 POINT(41.708699286014976 74.85702558034136) bank60715 +60716 POINT(40.171494363153414 74.44981171411698) bank60716 +60717 POINT(40.21576327728154 74.82682726932298) bank60717 +60718 POINT(40.208843469327256 73.73538306638457) bank60718 +60719 POINT(40.90112939741278 74.57518255628659) bank60719 +60720 POINT(40.519620597196955 73.15217192401269) bank60720 +60721 POINT(40.49693261028269 73.53481770543462) bank60721 +60722 POINT(40.50563550800876 73.21961930523547) bank60722 +60723 POINT(41.122736193901 74.63433752941552) bank60723 +60724 POINT(41.514259899337326 74.31288251195052) bank60724 +60725 POINT(39.85478857860694 73.57255827217136) bank60725 +60726 POINT(40.46515626970059 73.79711087121669) bank60726 +60727 POINT(41.5966599470231 73.5632803684933) bank60727 +60728 POINT(39.908135395145614 74.19452052813843) bank60728 +60729 POINT(41.52250264756386 73.18523858731831) bank60729 +60730 POINT(40.963220539722975 74.26720214301903) bank60730 +60731 POINT(39.772415403014676 74.12475353259151) bank60731 +60732 POINT(39.882938060691316 74.99242961353926) bank60732 +60733 POINT(41.68234720895247 73.82011864935848) bank60733 +60734 POINT(41.297881092243756 74.01370847097216) bank60734 +60735 POINT(40.80446873529119 74.12084701723317) bank60735 +60736 POINT(40.87031447127552 74.80194971339732) bank60736 +60737 POINT(40.25183560049265 73.55304555408308) bank60737 +60738 POINT(40.32154654278589 73.09690657029816) bank60738 +60739 POINT(41.26684208746468 73.68384688542623) bank60739 +60740 POINT(40.80672721125302 73.0527873103287) bank60740 +60741 POINT(41.274024539279445 74.5593978734647) bank60741 +60742 POINT(40.71389530520674 73.28750963694667) bank60742 +60743 POINT(40.058138341430116 73.47053127663678) bank60743 +60744 POINT(40.22349393674374 74.13867764042992) bank60744 +60745 POINT(39.88419975778858 74.15182191811074) bank60745 +60746 POINT(41.24265205493441 73.97187795170731) bank60746 +60747 POINT(40.50476889994671 74.72951456006732) bank60747 +60748 POINT(41.31165005832827 73.66919268689874) bank60748 +60749 POINT(39.86217492043138 73.42363965967537) bank60749 +60750 POINT(41.3421742667228 73.34779851614482) bank60750 +60751 POINT(41.264266349233075 73.12363163250595) bank60751 +60752 POINT(41.0868575271073 74.1187829259119) bank60752 +60753 POINT(40.125513747309505 74.232602144295) bank60753 +60754 POINT(40.27901016819928 74.13926537998064) bank60754 +60755 POINT(40.42983302413455 73.96867243586733) bank60755 +60756 POINT(41.26609936104198 73.55740228128258) bank60756 +60757 POINT(41.32963321872511 74.15087902421587) bank60757 +60758 POINT(41.57361230054944 74.55934898884371) bank60758 +60759 POINT(41.416592005243245 73.98180483004658) bank60759 +60760 POINT(40.0036956084038 74.62363448688978) bank60760 +60761 POINT(39.88458707422021 74.3210249862552) bank60761 +60762 POINT(41.302426264562975 74.12867035346805) bank60762 +60763 POINT(40.53355945926165 73.21037732635858) bank60763 +60764 POINT(41.20517335051094 74.27137011808584) bank60764 +60765 POINT(40.4448390407616 74.14052319299968) bank60765 +60766 POINT(40.842078980869466 74.1834211418628) bank60766 +60767 POINT(41.48329665952111 74.01106742171066) bank60767 +60768 POINT(40.64598375035705 73.92511380536803) bank60768 +60769 POINT(41.18107089696697 74.72145310741448) bank60769 +60770 POINT(40.746396218978695 73.2290934274617) bank60770 +60771 POINT(40.08847122884961 73.27798227225665) bank60771 +60772 POINT(40.76744624133766 74.44618819898275) bank60772 +60773 POINT(41.032328118530344 74.44512515069077) bank60773 +60774 POINT(40.892797615829664 73.87095159709006) bank60774 +60775 POINT(39.97368284841596 74.80365695139244) bank60775 +60776 POINT(40.88737198061386 74.86448385460228) bank60776 +60777 POINT(41.6842852023885 74.61970945186088) bank60777 +60778 POINT(41.290925822343524 73.8336395916032) bank60778 +60779 POINT(39.71498441113828 74.90848938225813) bank60779 +60780 POINT(40.18836183972286 74.70624423340996) bank60780 +60781 POINT(40.82844005028223 73.3365531271953) bank60781 +60782 POINT(39.99280720269982 74.25425092169542) bank60782 +60783 POINT(40.27069416510274 73.70531586547135) bank60783 +60784 POINT(39.87320774415586 74.00893185932601) bank60784 +60785 POINT(39.72179802904641 74.88165978544482) bank60785 +60786 POINT(39.92044949462179 73.43229067348727) bank60786 +60787 POINT(40.1871660771575 73.61095969389271) bank60787 +60788 POINT(40.30769515355077 73.53766406066987) bank60788 +60789 POINT(41.66652133501445 73.48525438959416) bank60789 +60790 POINT(40.86252880712718 73.15051239708521) bank60790 +60791 POINT(40.34785825194632 73.48578729184383) bank60791 +60792 POINT(39.90771339417919 74.80469269098131) bank60792 +60793 POINT(40.39089657211445 74.74405173830448) bank60793 +60794 POINT(39.83817924526111 73.2275155900601) bank60794 +60795 POINT(40.47031924566393 73.08426978384034) bank60795 +60796 POINT(40.73450397853129 74.68218517891628) bank60796 +60797 POINT(39.83679277481947 73.16243377334007) bank60797 +60798 POINT(40.1639803533052 74.90460865737248) bank60798 +60799 POINT(41.17562317778095 73.09264829597892) bank60799 +60800 POINT(40.78738961753285 74.25702666060671) bank60800 +60801 POINT(40.164442182008294 73.5246987692964) bank60801 +60802 POINT(40.02429730480481 73.07155644759874) bank60802 +60803 POINT(40.617571107581526 73.04054759018439) bank60803 +60804 POINT(39.76167456229465 74.91993945115483) bank60804 +60805 POINT(40.40793287438195 73.46919475811302) bank60805 +60806 POINT(40.16060519620254 74.85810202840315) bank60806 +60807 POINT(40.611502596615495 74.32764197276644) bank60807 +60808 POINT(39.9943788993983 73.9282930598945) bank60808 +60809 POINT(41.677476428190076 74.92095112552364) bank60809 +60810 POINT(41.12923984714904 74.2428518044789) bank60810 +60811 POINT(40.971483196826256 73.30130446663722) bank60811 +60812 POINT(41.10821456982346 73.03866685570516) bank60812 +60813 POINT(41.474924000760204 74.39830242273389) bank60813 +60814 POINT(40.43614614051114 74.87702659046793) bank60814 +60815 POINT(39.82870148177764 74.33480017017311) bank60815 +60816 POINT(40.07112865700661 73.40450255840578) bank60816 +60817 POINT(40.70954816595453 73.67785086046223) bank60817 +60818 POINT(40.94783278113058 74.6073118898605) bank60818 +60819 POINT(41.24704325964772 74.19738518513209) bank60819 +60820 POINT(39.88194312362306 73.91753280498394) bank60820 +60821 POINT(40.84931595851907 74.86992493646162) bank60821 +60822 POINT(41.09599287898973 73.18580986485459) bank60822 +60823 POINT(40.38890858869798 74.15816385436986) bank60823 +60824 POINT(40.10058720770459 74.80304996857949) bank60824 +60825 POINT(41.45731684790372 73.08704702565592) bank60825 +60826 POINT(41.05213655329791 74.22983128896942) bank60826 +60827 POINT(41.44278343513703 73.54069980789437) bank60827 +60828 POINT(40.349380539733 74.59047442491976) bank60828 +60829 POINT(40.39666041592538 74.20159713417785) bank60829 +60830 POINT(40.76072044480608 73.61759033962464) bank60830 +60831 POINT(39.99741034385866 73.9598587835375) bank60831 +60832 POINT(39.99353535262496 74.0460646099506) bank60832 +60833 POINT(41.4986450899797 73.3482948310892) bank60833 +60834 POINT(41.692065122079626 74.03733230097116) bank60834 +60835 POINT(41.31729380712839 73.03728933923465) bank60835 +60836 POINT(40.081056916099904 73.12219824017099) bank60836 +60837 POINT(41.64272108822119 74.66055480851716) bank60837 +60838 POINT(41.35851532836334 73.52300915265796) bank60838 +60839 POINT(41.131092512768014 74.27302816425683) bank60839 +60840 POINT(41.69760823060716 74.7048298192332) bank60840 +60841 POINT(41.28532729380083 73.42027927577479) bank60841 +60842 POINT(39.86779075851698 73.02151821386613) bank60842 +60843 POINT(40.21256363807676 74.04951334142197) bank60843 +60844 POINT(39.85980845823184 73.31838739781159) bank60844 +60845 POINT(40.36207197835019 74.320781222866) bank60845 +60846 POINT(40.96481585046246 73.67758029357235) bank60846 +60847 POINT(40.30074513305049 74.35804397993323) bank60847 +60848 POINT(41.39283966697704 73.74137698962144) bank60848 +60849 POINT(41.21858259957489 74.75205524242764) bank60849 +60850 POINT(41.13254240692439 74.4929923371192) bank60850 +60851 POINT(41.24640912635571 73.63083616034491) bank60851 +60852 POINT(41.57406163180836 74.37080570196328) bank60852 +60853 POINT(41.19315300616945 73.98877589238157) bank60853 +60854 POINT(40.62287863259763 74.29783535627749) bank60854 +60855 POINT(40.055716313033564 74.73565613052492) bank60855 +60856 POINT(40.521293277667795 73.34530059784426) bank60856 +60857 POINT(41.06498269416001 74.07671513294405) bank60857 +60858 POINT(41.601694130226036 74.3217758493688) bank60858 +60859 POINT(40.17824155951892 74.74196381984218) bank60859 +60860 POINT(40.884350988625094 74.0290653513762) bank60860 +60861 POINT(40.43266439842851 73.53755799964485) bank60861 +60862 POINT(39.832398795096076 73.02856208445615) bank60862 +60863 POINT(39.998474674286555 74.66201384865568) bank60863 +60864 POINT(40.34832855069441 73.61473772174105) bank60864 +60865 POINT(40.588860620622874 74.71742167328519) bank60865 +60866 POINT(40.56641316880601 73.58706245636938) bank60866 +60867 POINT(41.64661949548446 74.47515915887861) bank60867 +60868 POINT(40.82364031113757 73.62141821828567) bank60868 +60869 POINT(39.961273196538045 73.34318051229921) bank60869 +60870 POINT(41.44500374607894 74.45597628919401) bank60870 +60871 POINT(40.02703926344678 73.40274935463398) bank60871 +60872 POINT(40.66745946481416 74.599555080154) bank60872 +60873 POINT(41.51443871003554 73.64066306612959) bank60873 +60874 POINT(40.90591029895617 73.60476546988706) bank60874 +60875 POINT(40.05032157532722 74.18277938270118) bank60875 +60876 POINT(40.11222869220014 74.6289968394253) bank60876 +60877 POINT(41.691238668890286 73.63496272343092) bank60877 +60878 POINT(41.19951613443484 74.12107858707743) bank60878 +60879 POINT(41.70981736017189 74.42603168964808) bank60879 +60880 POINT(40.23388578547311 73.17416388489693) bank60880 +60881 POINT(41.29612714749406 73.6012909768408) bank60881 +60882 POINT(40.35770226136802 73.15280466907622) bank60882 +60883 POINT(40.65823532099562 74.49762648905136) bank60883 +60884 POINT(40.4362579283903 74.73802560087105) bank60884 +60885 POINT(39.902155601344056 74.7857392691161) bank60885 +60886 POINT(41.29283053908618 73.26449145082327) bank60886 +60887 POINT(40.62364586133537 73.82650314877885) bank60887 +60888 POINT(41.28884218785298 73.49791200928951) bank60888 +60889 POINT(39.91341285512672 73.20435171005165) bank60889 +60890 POINT(40.784209488101524 73.04465977571988) bank60890 +60891 POINT(39.94644662343222 73.59527339044442) bank60891 +60892 POINT(41.13080327524998 74.56772434304996) bank60892 +60893 POINT(40.61877172161167 73.66317038064788) bank60893 +60894 POINT(40.14630463627697 74.31698325849793) bank60894 +60895 POINT(39.97700303788646 74.52737412659117) bank60895 +60896 POINT(41.13018482390083 74.23198210205284) bank60896 +60897 POINT(40.59961516231471 74.69718002635666) bank60897 +60898 POINT(39.99993408438975 74.21899131365427) bank60898 +60899 POINT(41.6701679773572 74.33105707466382) bank60899 +60900 POINT(40.5770066159646 73.8350998851712) bank60900 +60901 POINT(40.33873584163747 74.61380697918425) bank60901 +60902 POINT(40.21836725777177 74.75382214263287) bank60902 +60903 POINT(41.5293301024383 74.62443004499703) bank60903 +60904 POINT(40.532787132863284 74.12294721471044) bank60904 +60905 POINT(41.60622471777391 73.86742349909254) bank60905 +60906 POINT(40.12616613532807 73.61659793723776) bank60906 +60907 POINT(39.98663772730066 75.00224591017364) bank60907 +60908 POINT(40.653570068273716 73.98521009497594) bank60908 +60909 POINT(40.03237846865995 75.0011841994187) bank60909 +60910 POINT(41.26040313757271 74.11465946518531) bank60910 +60911 POINT(39.830510778224166 73.12296927680666) bank60911 +60912 POINT(40.308350154782254 74.76588124397739) bank60912 +60913 POINT(39.80909981304801 73.6614066728714) bank60913 +60914 POINT(41.40882445598797 73.5950267723019) bank60914 +60915 POINT(40.864364589267 73.84487915836233) bank60915 +60916 POINT(40.51954290004134 74.05105125536875) bank60916 +60917 POINT(40.4819999800536 73.03819559584602) bank60917 +60918 POINT(41.270361871256625 73.02026259317414) bank60918 +60919 POINT(39.71687027127387 73.41124014208724) bank60919 +60920 POINT(40.271039149382865 74.67173776508244) bank60920 +60921 POINT(41.07189193464159 74.88578518284419) bank60921 +60922 POINT(40.45140752336636 73.46191321633283) bank60922 +60923 POINT(40.105192153388465 74.83383846249174) bank60923 +60924 POINT(40.63240722169811 73.35500905013274) bank60924 +60925 POINT(41.12208175128404 73.44331550046716) bank60925 +60926 POINT(41.63691325086041 74.95111859796556) bank60926 +60927 POINT(40.545395669848396 74.44894384073777) bank60927 +60928 POINT(41.01024348977229 73.80009763357027) bank60928 +60929 POINT(41.32827045909111 74.0133175166433) bank60929 +60930 POINT(40.32490698835359 74.80450947219961) bank60930 +60931 POINT(40.1479275799216 74.46202540136359) bank60931 +60932 POINT(39.84019616890449 74.41277589309077) bank60932 +60933 POINT(40.53159646797078 74.67784506904967) bank60933 +60934 POINT(41.686883513073994 74.87007725980476) bank60934 +60935 POINT(40.11636269857758 74.93374601113769) bank60935 +60936 POINT(40.42392477891012 73.83899785268412) bank60936 +60937 POINT(40.13790130000567 74.68090250147627) bank60937 +60938 POINT(40.55397198142923 74.10397958961477) bank60938 +60939 POINT(40.288419409657976 73.24830923739357) bank60939 +60940 POINT(40.809273275192176 74.62551331938555) bank60940 +60941 POINT(40.80999010445388 74.05989606113204) bank60941 +60942 POINT(41.15131365421715 74.43111302740547) bank60942 +60943 POINT(41.180800428344725 73.69588767941053) bank60943 +60944 POINT(40.6233944406988 73.15701772008327) bank60944 +60945 POINT(40.72345532416985 73.18139836689126) bank60945 +60946 POINT(41.67646836426509 73.75672406474604) bank60946 +60947 POINT(40.918656991603044 73.29810527015117) bank60947 +60948 POINT(39.735044509185094 74.76407649842832) bank60948 +60949 POINT(41.18910121851718 74.21678853997376) bank60949 +60950 POINT(41.28146696467469 73.12405564842388) bank60950 +60951 POINT(41.09135312341569 73.02686900019276) bank60951 +60952 POINT(41.061374827091285 73.02249860855697) bank60952 +60953 POINT(39.89493520950027 74.3548331735983) bank60953 +60954 POINT(40.53149753901856 74.00392029675386) bank60954 +60955 POINT(41.11796836580419 73.60016648626362) bank60955 +60956 POINT(41.075722228173774 74.09281633078814) bank60956 +60957 POINT(41.070553311460934 74.61245253979432) bank60957 +60958 POINT(41.388715371651415 74.47734828243253) bank60958 +60959 POINT(40.76064676670097 73.31431314965823) bank60959 +60960 POINT(41.00385495032981 74.16075723806755) bank60960 +60961 POINT(40.7924807678729 73.69497011590927) bank60961 +60962 POINT(41.561990816836875 74.2479326692925) bank60962 +60963 POINT(41.26107013745739 73.08895589495755) bank60963 +60964 POINT(41.02071170743404 74.70121619455564) bank60964 +60965 POINT(40.63455096899793 74.98962496559895) bank60965 +60966 POINT(40.060000793157954 74.10576103261761) bank60966 +60967 POINT(40.72372892963784 73.3008058457835) bank60967 +60968 POINT(41.32712862869134 74.53772945258622) bank60968 +60969 POINT(40.89966880321637 73.81900816502674) bank60969 +60970 POINT(41.53993943318074 73.85686109369487) bank60970 +60971 POINT(41.125428970453484 73.11622527679056) bank60971 +60972 POINT(41.133684353472304 74.06709535601173) bank60972 +60973 POINT(41.605184597276846 74.12382572894461) bank60973 +60974 POINT(39.8311092410933 73.71429376265147) bank60974 +60975 POINT(40.88014803584141 74.08733943871334) bank60975 +60976 POINT(40.34722426147312 74.69266776899681) bank60976 +60977 POINT(39.763953275078705 73.01919575365447) bank60977 +60978 POINT(41.68172989102154 73.0475963164394) bank60978 +60979 POINT(39.920323701124616 74.475849649866) bank60979 +60980 POINT(40.34545340173448 74.31892363131647) bank60980 +60981 POINT(40.65858688330532 73.22978228468924) bank60981 +60982 POINT(40.056507452812205 73.2549731156161) bank60982 +60983 POINT(40.67345236192994 73.16795658325655) bank60983 +60984 POINT(41.38794953681521 73.6125134238819) bank60984 +60985 POINT(41.30927232370412 73.06543928833851) bank60985 +60986 POINT(40.35391101711488 73.11409897978193) bank60986 +60987 POINT(40.68781405861821 74.17065915746838) bank60987 +60988 POINT(40.18521170737723 73.58320501732425) bank60988 +60989 POINT(39.79511387088187 73.3724382693) bank60989 +60990 POINT(41.33482004742583 74.04214069381439) bank60990 +60991 POINT(40.58917431776015 74.84654591890595) bank60991 +60992 POINT(40.40242621339242 73.55459189970736) bank60992 +60993 POINT(40.92949186789884 73.36651124786101) bank60993 +60994 POINT(41.39465325604945 73.66896455531456) bank60994 +60995 POINT(40.018771354998364 73.3347899345809) bank60995 +60996 POINT(40.131150176723416 74.94096847336343) bank60996 +60997 POINT(40.53790378137769 73.32252234382617) bank60997 +60998 POINT(40.267775787637575 73.53368354574364) bank60998 +60999 POINT(41.26531848996144 73.76475435313651) bank60999 +61000 POINT(40.25735699549368 74.65856080164563) bank61000 +61001 POINT(40.474797945754 74.34198863011348) bank61001 +61002 POINT(41.56338895761275 74.20529645001982) bank61002 +61003 POINT(41.62746096269705 73.91653274602768) bank61003 +61004 POINT(39.92253985263122 74.45736078882958) bank61004 +61005 POINT(40.1280437146444 73.99147952868415) bank61005 +61006 POINT(40.645804561966024 74.32055175187776) bank61006 +61007 POINT(40.27677791972876 74.74724349922404) bank61007 +61008 POINT(40.373883654396785 73.0739024135827) bank61008 +61009 POINT(39.9455054781453 73.90230121547653) bank61009 +61010 POINT(40.938059256659145 73.60510087034373) bank61010 +61011 POINT(40.20013648476462 74.64004943025265) bank61011 +61012 POINT(40.55480195848684 74.43915211905212) bank61012 +61013 POINT(40.335730367271196 73.2724863518977) bank61013 +61014 POINT(40.011402783274235 73.48490521866619) bank61014 +61015 POINT(40.662615670717486 74.42424814075069) bank61015 +61016 POINT(41.0452474095473 74.02284361162081) bank61016 +61017 POINT(41.40785877966772 73.14213256392357) bank61017 +61018 POINT(41.27387632103807 74.22061767913696) bank61018 +61019 POINT(41.49948772776589 74.4495360668912) bank61019 +61020 POINT(41.052825040215424 74.33324991694602) bank61020 +61021 POINT(39.8779202214631 73.76192248167045) bank61021 +61022 POINT(40.41240760077765 74.03945920459708) bank61022 +61023 POINT(40.92025358723024 73.4395240456705) bank61023 +61024 POINT(40.439852135403804 74.59108007520044) bank61024 +61025 POINT(39.85259337890129 73.50321241652597) bank61025 +61026 POINT(41.40999142182137 73.48855619967432) bank61026 +61027 POINT(40.44444592898604 73.97694372950775) bank61027 +61028 POINT(41.5279683094186 74.1499580857464) bank61028 +61029 POINT(40.23458356985951 74.72128535621458) bank61029 +61030 POINT(41.25496583464166 74.44128956405767) bank61030 +61031 POINT(40.86900869417458 73.77983842168734) bank61031 +61032 POINT(40.77036515262235 73.85213138989954) bank61032 +61033 POINT(40.23602250754015 73.58970514044056) bank61033 +61034 POINT(39.790435109060745 74.42739543402436) bank61034 +61035 POINT(41.40932652416059 74.76973665452857) bank61035 +61036 POINT(39.946456147818814 73.95765445541875) bank61036 +61037 POINT(40.88207116518134 74.77969142710688) bank61037 +61038 POINT(40.70939117187939 74.54111422983928) bank61038 +61039 POINT(41.29028900830157 74.24219673744722) bank61039 +61040 POINT(40.11849614770556 74.79068435320012) bank61040 +61041 POINT(41.712768644457405 73.20308960064492) bank61041 +61042 POINT(41.465533376012466 73.03618296744612) bank61042 +61043 POINT(41.371097295221624 74.43589347956194) bank61043 +61044 POINT(39.765922940089006 74.41654248205747) bank61044 +61045 POINT(41.69546042819915 73.46256494292199) bank61045 +61046 POINT(41.62078330266378 74.13570920585683) bank61046 +61047 POINT(41.32223760822753 73.77648202521625) bank61047 +61048 POINT(41.57810819924981 73.15954026065586) bank61048 +61049 POINT(40.15297478216938 73.2125346072158) bank61049 +61050 POINT(40.51760349666359 73.67592361553272) bank61050 +61051 POINT(41.38020603110483 74.00767161698363) bank61051 +61052 POINT(40.24079918234048 73.6089266803327) bank61052 +61053 POINT(39.72445812914437 74.0629305637392) bank61053 +61054 POINT(41.507867993763874 73.43151439338563) bank61054 +61055 POINT(41.31973355286539 73.5753542141493) bank61055 +61056 POINT(41.5817640328052 74.54668557407398) bank61056 +61057 POINT(40.35619172773293 73.55611926087565) bank61057 +61058 POINT(40.02382605390801 74.11611290090526) bank61058 +61059 POINT(41.27127236353174 73.6070851499373) bank61059 +61060 POINT(40.61677394664227 74.78635371241357) bank61060 +61061 POINT(41.43853196291685 73.78036585181874) bank61061 +61062 POINT(41.232261322471544 73.07783841122802) bank61062 +61063 POINT(39.86528334840449 73.76189427720122) bank61063 +61064 POINT(41.08568461998735 73.26775558915483) bank61064 +61065 POINT(39.91102896793686 74.8809435779348) bank61065 +61066 POINT(39.73516228493017 74.86344105114262) bank61066 +61067 POINT(40.05616260981306 73.96797473251917) bank61067 +61068 POINT(39.74149719857967 74.81445899041955) bank61068 +61069 POINT(41.16905653177969 74.02043942226854) bank61069 +61070 POINT(40.668579111619295 73.17780847459598) bank61070 +61071 POINT(40.4690655593965 74.97808253387163) bank61071 +61072 POINT(39.995322072703594 74.6805389221659) bank61072 +61073 POINT(41.518471937403476 73.70120757618731) bank61073 +61074 POINT(40.56892864848118 74.34692947213375) bank61074 +61075 POINT(41.15510967752633 74.62920981773605) bank61075 +61076 POINT(41.666177901992405 73.01290659780584) bank61076 +61077 POINT(40.603731802601764 73.46201740657064) bank61077 +61078 POINT(40.4796470295184 74.69532200284256) bank61078 +61079 POINT(40.96218040843768 73.80070633259365) bank61079 +61080 POINT(40.800235172370755 73.41572730410543) bank61080 +61081 POINT(40.003467896423544 74.46718635204472) bank61081 +61082 POINT(40.78280405702071 74.54701201952525) bank61082 +61083 POINT(40.44943777192215 74.03453546129192) bank61083 +61084 POINT(41.28863253480165 74.70184023383437) bank61084 +61085 POINT(41.55001879698612 74.31751881760526) bank61085 +61086 POINT(39.92366772599322 73.25205333576461) bank61086 +61087 POINT(39.898977489735174 74.76554149465652) bank61087 +61088 POINT(41.05520180500745 73.43663089416084) bank61088 +61089 POINT(40.79221592989737 73.31038263200068) bank61089 +61090 POINT(40.03061891119547 73.7640722049726) bank61090 +61091 POINT(40.03660131712453 74.75284026478043) bank61091 +61092 POINT(40.05663202008075 74.18219475604488) bank61092 +61093 POINT(41.37147248730064 73.02794483586472) bank61093 +61094 POINT(39.73783686726972 73.10300292359707) bank61094 +61095 POINT(40.859132788556394 74.90862998694726) bank61095 +61096 POINT(40.57085865978982 73.22661381171021) bank61096 +61097 POINT(40.069896559211685 73.84951080245574) bank61097 +61098 POINT(40.30547698557649 74.08928823544976) bank61098 +61099 POINT(40.08713529836985 73.79745116590284) bank61099 +61100 POINT(40.140955511538515 73.03278371313881) bank61100 +61101 POINT(41.36905237708934 73.84679692731045) bank61101 +61102 POINT(39.817659188224695 74.38290277017455) bank61102 +61103 POINT(39.87329472073177 74.52500109086671) bank61103 +61104 POINT(40.29720769012731 74.84872109089369) bank61104 +61105 POINT(41.69386699217002 74.4422188541508) bank61105 +61106 POINT(40.4493028421323 74.02521377344493) bank61106 +61107 POINT(40.41414193925592 73.72756602726324) bank61107 +61108 POINT(40.115786879096596 74.54344076283577) bank61108 +61109 POINT(41.4383138259283 74.85608406592368) bank61109 +61110 POINT(41.33722527100512 73.31025234982097) bank61110 +61111 POINT(40.140192623835304 74.24983646326311) bank61111 +61112 POINT(40.21861644163366 74.76001275494208) bank61112 +61113 POINT(41.13753843910957 74.05999803856004) bank61113 +61114 POINT(41.06727917381682 74.78121307315067) bank61114 +61115 POINT(41.30786639261344 73.12362485941163) bank61115 +61116 POINT(40.73142806055909 74.65117976487713) bank61116 +61117 POINT(41.67882070660068 73.61943067948509) bank61117 +61118 POINT(40.48563379391747 73.44338809988315) bank61118 +61119 POINT(40.75398001816481 73.08631519456338) bank61119 +61120 POINT(41.586391069353176 75.00354245250516) bank61120 +61121 POINT(41.620181098423416 73.06680841764293) bank61121 +61122 POINT(40.83578417023907 74.05138862423256) bank61122 +61123 POINT(39.86157400872823 74.72675564184817) bank61123 +61124 POINT(41.56676549524062 74.27178358373007) bank61124 +61125 POINT(40.31858316485903 74.29701557871803) bank61125 +61126 POINT(39.7655972091468 74.24714753383678) bank61126 +61127 POINT(40.47185457317222 74.00175012610727) bank61127 +61128 POINT(40.92360600270755 74.91805399658436) bank61128 +61129 POINT(40.06044164335367 74.2643931950671) bank61129 +61130 POINT(39.863348631100905 73.50994764902066) bank61130 +61131 POINT(41.28773568008722 74.19336469030841) bank61131 +61132 POINT(40.77923276056922 74.98951200143425) bank61132 +61133 POINT(41.095895192544155 73.67530632433807) bank61133 +61134 POINT(40.090514744093156 74.79031408234766) bank61134 +61135 POINT(39.84482838639225 73.90555812527623) bank61135 +61136 POINT(41.286113585146616 74.01761394535039) bank61136 +61137 POINT(41.25553818051996 74.47582375311647) bank61137 +61138 POINT(40.34853272073427 73.84852892056875) bank61138 +61139 POINT(40.9958448936616 74.39300937601357) bank61139 +61140 POINT(40.291206778421554 74.78482389661045) bank61140 +61141 POINT(39.945487023162535 73.1529687870212) bank61141 +61142 POINT(39.76011718208337 74.75970684362869) bank61142 +61143 POINT(41.45203191154824 74.08622016207792) bank61143 +61144 POINT(41.57050184740811 74.78658686171711) bank61144 +61145 POINT(41.34422577158532 73.8180047707066) bank61145 +61146 POINT(39.86025300430225 73.37159863349657) bank61146 +61147 POINT(40.74818411830845 73.85918723471237) bank61147 +61148 POINT(41.598401798727274 73.30313581032507) bank61148 +61149 POINT(41.68701491591994 73.68254544028032) bank61149 +61150 POINT(40.0831416493293 73.66696314675464) bank61150 +61151 POINT(40.42865948992678 74.95641433266647) bank61151 +61152 POINT(39.90029640493177 74.90690361118375) bank61152 +61153 POINT(41.15329595070361 74.77866850887135) bank61153 +61154 POINT(40.84754221270477 73.97773381726527) bank61154 +61155 POINT(40.05413380632518 73.15132942727402) bank61155 +61156 POINT(40.15577306597971 74.41334238581247) bank61156 +61157 POINT(41.203051573284604 73.81317960322659) bank61157 +61158 POINT(40.358347481618196 73.91808781898752) bank61158 +61159 POINT(41.44525645998351 74.90150521743712) bank61159 +61160 POINT(40.505656187875836 73.59905310016639) bank61160 +61161 POINT(41.42120172510347 73.85155410119359) bank61161 +61162 POINT(40.66053136999969 74.68308853165004) bank61162 +61163 POINT(41.65927612566766 73.57257132522203) bank61163 +61164 POINT(39.97703628682956 73.16690794343751) bank61164 +61165 POINT(39.9207805474437 73.83116967808054) bank61165 +61166 POINT(41.57666574126905 73.73236254712528) bank61166 +61167 POINT(41.15302514972972 74.12424462641712) bank61167 +61168 POINT(40.2479891025355 74.69730664338896) bank61168 +61169 POINT(40.113865079686946 73.66972134009286) bank61169 +61170 POINT(40.71476633436836 73.80016044894468) bank61170 +61171 POINT(40.28750876520057 74.16466273534452) bank61171 +61172 POINT(41.609753232139916 74.59037244157905) bank61172 +61173 POINT(40.37884668306289 74.62822417413227) bank61173 +61174 POINT(40.81416339496585 74.61621182790014) bank61174 +61175 POINT(41.54802362838055 74.16288636547763) bank61175 +61176 POINT(40.07004695326218 74.84294020435082) bank61176 +61177 POINT(40.14374063640632 74.85966616258034) bank61177 +61178 POINT(39.73651794181404 73.66591362050451) bank61178 +61179 POINT(40.10036122179339 73.48574562658484) bank61179 +61180 POINT(41.42014136959007 74.70779915478902) bank61180 +61181 POINT(40.48759865351711 73.73915369226366) bank61181 +61182 POINT(40.23541651706967 73.25205016637905) bank61182 +61183 POINT(41.42530307193266 73.87659620425899) bank61183 +61184 POINT(40.39026710469292 74.29653448758596) bank61184 +61185 POINT(41.40323279656689 74.18692064756576) bank61185 +61186 POINT(40.533294614712766 73.68326292310046) bank61186 +61187 POINT(40.81884930261487 74.51156799138083) bank61187 +61188 POINT(40.36773542738158 73.87367335046042) bank61188 +61189 POINT(40.48193617442886 74.27689491837768) bank61189 +61190 POINT(41.14770605873101 73.29095983350862) bank61190 +61191 POINT(40.252660004010735 74.6972161633367) bank61191 +61192 POINT(40.59395271900974 74.53203531298725) bank61192 +61193 POINT(40.57147850435069 73.23500921446461) bank61193 +61194 POINT(40.02674778661326 73.99405354980955) bank61194 +61195 POINT(40.480688086565834 73.87104740417247) bank61195 +61196 POINT(40.38752142828181 73.18456188001062) bank61196 +61197 POINT(41.21863203554444 73.52656937057982) bank61197 +61198 POINT(39.91027860732717 73.1509947257825) bank61198 +61199 POINT(40.11515053360061 73.37467574207048) bank61199 +61200 POINT(41.64395824761368 74.0370873228787) bank61200 +61201 POINT(40.11800362219941 74.95013490474742) bank61201 +61202 POINT(39.82730980616918 73.02924391059882) bank61202 +61203 POINT(41.428917595023655 73.73835561205713) bank61203 +61204 POINT(40.58920941017916 73.61947761016802) bank61204 +61205 POINT(41.370369257666134 73.57001465054029) bank61205 +61206 POINT(39.86510338529129 74.25856849053345) bank61206 +61207 POINT(40.91068820839861 73.55137337983535) bank61207 +61208 POINT(41.22995192643866 74.94267961852532) bank61208 +61209 POINT(40.678358410789606 74.1797527529923) bank61209 +61210 POINT(41.6182003231102 73.20348999896724) bank61210 +61211 POINT(40.80794085011168 73.6078242098208) bank61211 +61212 POINT(41.03207664869443 73.90421894853891) bank61212 +61213 POINT(41.333615228803474 74.32502173519768) bank61213 +61214 POINT(39.985605596602355 74.85974059670737) bank61214 +61215 POINT(41.2227242290277 74.01130121507013) bank61215 +61216 POINT(40.32628453732666 73.53315218815835) bank61216 +61217 POINT(40.06461587487513 74.91245489193034) bank61217 +61218 POINT(41.03853260984807 73.40798911261145) bank61218 +61219 POINT(40.62436341703856 74.20887666019382) bank61219 +61220 POINT(39.83452948579912 74.79799399317604) bank61220 +61221 POINT(41.64623902212975 74.16078786849174) bank61221 +61222 POINT(39.76012824524284 74.70598015328103) bank61222 +61223 POINT(41.39924633531662 74.32277587346466) bank61223 +61224 POINT(41.66849397647334 73.09039298805885) bank61224 +61225 POINT(41.01771782220995 73.42265091949008) bank61225 +61226 POINT(40.22477262419873 73.3391574854602) bank61226 +61227 POINT(40.41518533220649 73.95237167328726) bank61227 +61228 POINT(40.001788703395256 74.31505685029327) bank61228 +61229 POINT(39.78845796679124 73.5578960504287) bank61229 +61230 POINT(40.01222789692754 74.16176233101173) bank61230 +61231 POINT(39.78597621679651 74.64493504358316) bank61231 +61232 POINT(41.088298827096224 73.75754101179052) bank61232 +61233 POINT(40.470677379020806 73.16048242505197) bank61233 +61234 POINT(41.512784942798525 75.0024127082159) bank61234 +61235 POINT(41.364725614609696 74.04462780752623) bank61235 +61236 POINT(41.08847194157049 73.20554851288304) bank61236 +61237 POINT(41.15513136899218 73.52292507352693) bank61237 +61238 POINT(40.91128777016586 74.79315945486782) bank61238 +61239 POINT(41.19309178102823 74.92968400164294) bank61239 +61240 POINT(40.87258251312026 74.47656332399035) bank61240 +61241 POINT(41.129277473689804 73.35059479615242) bank61241 +61242 POINT(40.34301813874155 73.7249641072504) bank61242 +61243 POINT(39.853842276092884 74.481576637386) bank61243 +61244 POINT(41.02110838521248 73.0281411104361) bank61244 +61245 POINT(40.16891518187466 74.42340840567037) bank61245 +61246 POINT(41.481666714916564 73.29975469476594) bank61246 +61247 POINT(40.87737954989295 73.26153629555631) bank61247 +61248 POINT(40.05390061099877 73.30459077285329) bank61248 +61249 POINT(40.20001927638241 74.21202414941004) bank61249 +61250 POINT(40.81653528508347 73.67751316331321) bank61250 +61251 POINT(41.23224215719794 73.57170158474977) bank61251 +61252 POINT(39.81523986384145 74.12734364468744) bank61252 +61253 POINT(41.48347015260958 73.39022307961183) bank61253 +61254 POINT(40.61168519606657 73.44528161371274) bank61254 +61255 POINT(40.65497944890665 73.06433966226905) bank61255 +61256 POINT(40.28635980303644 73.26696373714059) bank61256 +61257 POINT(40.49071085316847 73.87984988903496) bank61257 +61258 POINT(40.58202488434131 73.44447707202853) bank61258 +61259 POINT(39.88235949689026 74.87418484130045) bank61259 +61260 POINT(40.329927655413215 74.2101302041501) bank61260 +61261 POINT(40.024512945385354 74.74752840487228) bank61261 +61262 POINT(41.06530780195441 73.27830703031849) bank61262 +61263 POINT(40.195661932691905 73.71467393576302) bank61263 +61264 POINT(40.852941423279965 74.01539675019909) bank61264 +61265 POINT(41.3527879452556 73.95490510796573) bank61265 +61266 POINT(40.38601278302924 73.248063882754) bank61266 +61267 POINT(40.815839205994216 73.18489619510635) bank61267 +61268 POINT(40.93086551504931 74.5319812142985) bank61268 +61269 POINT(40.63830513228053 74.91914442530381) bank61269 +61270 POINT(39.988164021305764 74.89839224480576) bank61270 +61271 POINT(40.03365632018129 74.9934388087686) bank61271 +61272 POINT(41.437819013775226 74.81968286513437) bank61272 +61273 POINT(41.16528338912003 73.0937188163565) bank61273 +61274 POINT(41.70805445668032 74.65988721059279) bank61274 +61275 POINT(40.800590618750554 73.53361004025933) bank61275 +61276 POINT(39.82477983646642 74.52493083511641) bank61276 +61277 POINT(40.67913731870621 74.24214527115228) bank61277 +61278 POINT(39.870739171397986 74.54543496497516) bank61278 +61279 POINT(40.37679821749344 73.1382891733096) bank61279 +61280 POINT(41.33803502275746 73.6819563547962) bank61280 +61281 POINT(40.04262345027327 73.37065920317447) bank61281 +61282 POINT(40.09743741470484 73.55985915349491) bank61282 +61283 POINT(41.046969963651286 74.34306914710139) bank61283 +61284 POINT(40.24645025612547 73.69668833309129) bank61284 +61285 POINT(40.16880140731381 73.2707554289224) bank61285 +61286 POINT(41.57122019090512 73.68117698764958) bank61286 +61287 POINT(40.93058280803082 73.64221246218003) bank61287 +61288 POINT(40.99038257486131 73.61775218127423) bank61288 +61289 POINT(40.77203813920954 73.2013630996553) bank61289 +61290 POINT(40.45912651848808 74.05429210945336) bank61290 +61291 POINT(40.24997064687927 73.861126095173) bank61291 +61292 POINT(39.78975479751297 74.67115490757725) bank61292 +61293 POINT(41.314823695369626 74.19433278258549) bank61293 +61294 POINT(39.71422478926169 74.51067791929191) bank61294 +61295 POINT(40.35087780288807 74.5956040355233) bank61295 +61296 POINT(40.51850337362247 73.74699595969375) bank61296 +61297 POINT(41.48395463581476 73.13553550682305) bank61297 +61298 POINT(39.80599565495832 73.23730529894299) bank61298 +61299 POINT(40.50615760619052 74.75483100338674) bank61299 +61300 POINT(40.500692639338176 73.5522694872223) bank61300 +61301 POINT(41.19678568510735 73.41600014682061) bank61301 +61302 POINT(41.383348758332254 74.32219165485324) bank61302 +61303 POINT(40.76257245447062 74.54747092007898) bank61303 +61304 POINT(40.33509248849804 73.8184559724471) bank61304 +61305 POINT(40.77747559858265 73.70788609191403) bank61305 +61306 POINT(41.50922278875134 74.25624436048155) bank61306 +61307 POINT(40.58429294522165 73.53042875100977) bank61307 +61308 POINT(40.34045042850674 74.14242475350065) bank61308 +61309 POINT(39.98801367708368 73.29974956974388) bank61309 +61310 POINT(40.49483128381175 73.80260896173102) bank61310 +61311 POINT(40.96428163569815 74.50118865908738) bank61311 +61312 POINT(41.28849474096508 74.97325526772399) bank61312 +61313 POINT(40.34938554624079 73.67596985800623) bank61313 +61314 POINT(41.65256641781552 73.67928023201776) bank61314 +61315 POINT(41.45309215807964 73.97309342009238) bank61315 +61316 POINT(39.76374847408106 74.26135595310237) bank61316 +61317 POINT(39.876758613593125 74.61929338242317) bank61317 +61318 POINT(41.271778957367644 74.02492821157921) bank61318 +61319 POINT(41.55355466528624 73.77326928005535) bank61319 +61320 POINT(40.18550359151038 73.98626311958027) bank61320 +61321 POINT(40.05212209489801 73.32869722273452) bank61321 +61322 POINT(39.74532164509681 74.57450525729637) bank61322 +61323 POINT(39.89107801611663 74.68647685953644) bank61323 +61324 POINT(40.77349335483988 73.78135088724535) bank61324 +61325 POINT(41.10974917509305 74.64720377090352) bank61325 +61326 POINT(40.54753828422261 73.7770431161425) bank61326 +61327 POINT(39.73195964590868 73.4365844257804) bank61327 +61328 POINT(41.0549536280988 74.797481611804) bank61328 +61329 POINT(41.129877093030274 73.57929314801072) bank61329 +61330 POINT(40.3764313283366 74.7689381522059) bank61330 +61331 POINT(40.38528241766908 73.9448904228135) bank61331 +61332 POINT(41.55345582744659 73.7352018133715) bank61332 +61333 POINT(40.789056200496944 74.54528538807455) bank61333 +61334 POINT(41.48242410042629 74.34825566863192) bank61334 +61335 POINT(40.31207541473 73.57800663544306) bank61335 +61336 POINT(40.19903628273465 74.78227131733408) bank61336 +61337 POINT(39.9569331561843 74.20483164282676) bank61337 +61338 POINT(39.971053339999806 74.65549795616401) bank61338 +61339 POINT(41.1588100102336 74.00154538322474) bank61339 +61340 POINT(39.894255379981374 74.86649914665584) bank61340 +61341 POINT(40.85720241532085 74.89451187886394) bank61341 +61342 POINT(39.81346056067517 74.60879608597224) bank61342 +61343 POINT(39.90844259561737 73.25421948581328) bank61343 +61344 POINT(41.44616985988786 73.10638392137523) bank61344 +61345 POINT(40.09174703609734 73.99078982738678) bank61345 +61346 POINT(41.367380370339724 74.00609757078905) bank61346 +61347 POINT(40.688699842171886 73.37278421539455) bank61347 +61348 POINT(41.36039884435132 73.19248867454587) bank61348 +61349 POINT(40.12229905248378 74.65348470268287) bank61349 +61350 POINT(41.201227642942264 74.54425430608264) bank61350 +61351 POINT(40.10372675383983 74.57717991485737) bank61351 +61352 POINT(40.48512881148494 74.30538611945589) bank61352 +61353 POINT(40.160216054078994 73.03632070055149) bank61353 +61354 POINT(40.460649732362796 73.72354828287139) bank61354 +61355 POINT(40.84065794859707 73.2769045919338) bank61355 +61356 POINT(40.91060900689753 74.7141503244116) bank61356 +61357 POINT(41.22826023500141 74.74967760677382) bank61357 +61358 POINT(40.39515371996708 73.04091119148316) bank61358 +61359 POINT(41.09910307105303 74.90705394385637) bank61359 +61360 POINT(41.27770087987394 74.94437007980636) bank61360 +61361 POINT(40.34421900597341 73.84369436656111) bank61361 +61362 POINT(40.110081857390156 74.96383889209162) bank61362 +61363 POINT(41.672273993504305 73.28404609087211) bank61363 +61364 POINT(40.24783770926244 73.78019785127319) bank61364 +61365 POINT(41.56188050949395 74.62646493739582) bank61365 +61366 POINT(41.0012493491529 74.52476038830453) bank61366 +61367 POINT(39.974849034490504 74.71521651676538) bank61367 +61368 POINT(40.34986706452099 73.57307386561634) bank61368 +61369 POINT(40.544097972196866 73.60554925195109) bank61369 +61370 POINT(41.679417727864816 74.06221131951577) bank61370 +61371 POINT(41.553558437783884 73.56178617126284) bank61371 +61372 POINT(40.372055039176495 74.65120930540071) bank61372 +61373 POINT(41.22369117482399 74.23407674315393) bank61373 +61374 POINT(40.92712334377033 73.30104733378894) bank61374 +61375 POINT(41.36835301403445 73.1406184332288) bank61375 +61376 POINT(40.123114981221015 73.376369751368) bank61376 +61377 POINT(41.304990751285615 74.85182131436828) bank61377 +61378 POINT(41.130884721777235 74.36281353456678) bank61378 +61379 POINT(40.065586291297 73.85753516252625) bank61379 +61380 POINT(40.48997721440377 73.69868674952052) bank61380 +61381 POINT(40.457713019786524 73.64705249206152) bank61381 +61382 POINT(41.67710761785725 74.81760266052959) bank61382 +61383 POINT(41.19266498809429 73.52681495122079) bank61383 +61384 POINT(40.65547556780353 73.82553805254436) bank61384 +61385 POINT(40.71329034904465 74.40079834000026) bank61385 +61386 POINT(41.11436948376696 74.91531757231601) bank61386 +61387 POINT(41.078766143566945 74.8129441265409) bank61387 +61388 POINT(39.93681369648546 73.69881842031694) bank61388 +61389 POINT(39.8322536081563 74.08814268518725) bank61389 +61390 POINT(41.4391709212308 73.90101220101579) bank61390 +61391 POINT(40.27543829454775 74.89821874706254) bank61391 +61392 POINT(40.677144845172 73.64191247742824) bank61392 +61393 POINT(40.62548398737725 74.33676583944833) bank61393 +61394 POINT(40.49047279598498 74.0966084652655) bank61394 +61395 POINT(40.41714392420152 74.63647073053333) bank61395 +61396 POINT(41.50320933737417 73.31936547455405) bank61396 +61397 POINT(41.21258585546534 74.41039240226387) bank61397 +61398 POINT(41.59711190344268 73.92502089010918) bank61398 +61399 POINT(41.55799524699483 73.61428312647689) bank61399 +61400 POINT(40.277400859023345 73.79689201939578) bank61400 +61401 POINT(41.14929390699519 73.95071640982529) bank61401 +61402 POINT(40.619243755392695 73.95903267313217) bank61402 +61403 POINT(39.750634084906736 73.66847814774789) bank61403 +61404 POINT(40.032880025248055 73.70085686591517) bank61404 +61405 POINT(39.82305505048927 74.70668419171695) bank61405 +61406 POINT(41.23447366236578 74.82146976371776) bank61406 +61407 POINT(40.364034007603934 74.73172942686851) bank61407 +61408 POINT(39.98781583842696 74.60790990747937) bank61408 +61409 POINT(40.812782230073076 74.71002773626239) bank61409 +61410 POINT(40.67837404719234 74.42810097894602) bank61410 +61411 POINT(41.568075890600774 74.2107758178645) bank61411 +61412 POINT(41.069182847359635 73.0677260250447) bank61412 +61413 POINT(39.739047872823015 74.57947359239806) bank61413 +61414 POINT(40.14299586731501 73.68778731594593) bank61414 +61415 POINT(41.159528583593264 74.10395012239391) bank61415 +61416 POINT(40.5293849849524 74.67596773391718) bank61416 +61417 POINT(41.154585123362374 74.21809222843869) bank61417 +61418 POINT(40.26915512969065 73.80168567640376) bank61418 +61419 POINT(41.69195659360134 74.49999561788563) bank61419 +61420 POINT(41.325691026409835 74.65145807268016) bank61420 +61421 POINT(41.66429646970048 74.42488949459299) bank61421 +61422 POINT(41.64770353406344 73.04900022388975) bank61422 +61423 POINT(40.12166732811687 73.08931243051588) bank61423 +61424 POINT(40.43890623855662 73.84550821821256) bank61424 +61425 POINT(40.173948236218926 73.70934192278877) bank61425 +61426 POINT(39.960380210030685 73.00697644382677) bank61426 +61427 POINT(40.765570060935055 74.11083690616937) bank61427 +61428 POINT(41.02055426371718 74.22863410660487) bank61428 +61429 POINT(41.392219055465304 74.61276733652961) bank61429 +61430 POINT(40.92026939678282 74.23294912716818) bank61430 +61431 POINT(41.59170193870089 73.05269509045606) bank61431 +61432 POINT(40.47892899824425 73.06150123123938) bank61432 +61433 POINT(39.846552414525874 73.0223765281174) bank61433 +61434 POINT(40.45486592235973 74.16762215782708) bank61434 +61435 POINT(41.199467940591695 73.38037450956381) bank61435 +61436 POINT(41.06718582068341 73.9486534564043) bank61436 +61437 POINT(41.46415491907926 74.57110367002866) bank61437 +61438 POINT(41.64831618336131 74.61835234898334) bank61438 +61439 POINT(39.75116507700504 74.77593401324468) bank61439 +61440 POINT(41.566253377968124 73.81910243267144) bank61440 +61441 POINT(41.31785096145056 74.29895979095491) bank61441 +61442 POINT(41.600543252287046 74.63132348894766) bank61442 +61443 POINT(41.061774570641994 74.83260212234569) bank61443 +61444 POINT(40.84061880216135 74.3787217003459) bank61444 +61445 POINT(40.15267058256372 73.62940156765293) bank61445 +61446 POINT(40.309921024945716 73.895242899353) bank61446 +61447 POINT(40.16927692503586 73.42543332676595) bank61447 +61448 POINT(40.516450091031395 73.9636884489578) bank61448 +61449 POINT(39.74077160305127 74.30886118772982) bank61449 +61450 POINT(41.53975633247739 73.72600004627657) bank61450 +61451 POINT(40.536061153073284 73.78454741027947) bank61451 +61452 POINT(41.46100786848145 74.99010528541062) bank61452 +61453 POINT(39.7408969186285 73.55072205261975) bank61453 +61454 POINT(39.78365130526918 74.2307952129081) bank61454 +61455 POINT(39.78917929916231 73.65969344607396) bank61455 +61456 POINT(40.87622629557867 74.15141222803567) bank61456 +61457 POINT(40.09109095841749 73.12501219228227) bank61457 +61458 POINT(40.545152032274515 73.04846783332383) bank61458 +61459 POINT(41.484403068519285 73.50036701468193) bank61459 +61460 POINT(39.90037533487266 73.16724272918405) bank61460 +61461 POINT(40.341708445933975 73.27151597320149) bank61461 +61462 POINT(40.64702073111343 74.08100461275683) bank61462 +61463 POINT(40.80875741810921 74.58746376278084) bank61463 +61464 POINT(41.224082739251834 73.15803048917152) bank61464 +61465 POINT(39.75536377426477 73.78293589979046) bank61465 +61466 POINT(40.337755233836624 73.11105083604956) bank61466 +61467 POINT(40.96887188183848 73.5698150212044) bank61467 +61468 POINT(40.37850879939422 73.62956544997579) bank61468 +61469 POINT(41.14885378605432 73.8029265646348) bank61469 +61470 POINT(40.33066926816598 74.5467508682304) bank61470 +61471 POINT(40.118913749503164 74.94068270167946) bank61471 +61472 POINT(40.963320088319094 73.86967456288961) bank61472 +61473 POINT(39.761798994014995 74.82139899847753) bank61473 +61474 POINT(40.055530845759776 73.7662257154749) bank61474 +61475 POINT(40.45169314753015 74.58090080017014) bank61475 +61476 POINT(39.79450945133844 73.36339693341105) bank61476 +61477 POINT(41.52258106680153 74.76001787457855) bank61477 +61478 POINT(40.526532831562704 74.18473765567568) bank61478 +61479 POINT(40.492688335942844 73.32693282956448) bank61479 +61480 POINT(41.05525678397059 73.94137179202482) bank61480 +61481 POINT(39.849137352751974 74.6313997349071) bank61481 +61482 POINT(41.444428452230376 73.30369670123085) bank61482 +61483 POINT(40.67355620920472 73.52251253247286) bank61483 +61484 POINT(40.508345421846045 74.87208097499636) bank61484 +61485 POINT(40.32294856791937 73.00915779460428) bank61485 +61486 POINT(41.01444790547001 74.06630559837076) bank61486 +61487 POINT(40.9486923653148 73.51316434146842) bank61487 +61488 POINT(40.38632995699967 73.73295184943261) bank61488 +61489 POINT(40.55458498947295 74.62180819039084) bank61489 +61490 POINT(40.6041960422738 74.1217478137097) bank61490 +61491 POINT(40.91871083419579 73.84838405937039) bank61491 +61492 POINT(41.05006357748112 74.10631256230144) bank61492 +61493 POINT(41.63488291370264 74.57860793261968) bank61493 +61494 POINT(40.98106383167294 74.3690350508915) bank61494 +61495 POINT(40.06454447519732 74.9867432728966) bank61495 +61496 POINT(40.211914550843794 73.2852009104893) bank61496 +61497 POINT(41.169629901887305 74.39120245080252) bank61497 +61498 POINT(39.86483838770182 73.22185604538868) bank61498 +61499 POINT(40.56136889796115 73.44168939343709) bank61499 +61500 POINT(40.96944470108525 73.18040862385202) bank61500 +61501 POINT(40.78625666915549 73.94719865798366) bank61501 +61502 POINT(40.416427828087954 74.17762144248542) bank61502 +61503 POINT(40.560928308479546 73.76282616985696) bank61503 +61504 POINT(40.47386265627341 74.42512725873048) bank61504 +61505 POINT(41.6854685788993 74.98893602147906) bank61505 +61506 POINT(40.30312355669666 73.15186794615592) bank61506 +61507 POINT(40.069629355274564 73.95489363495076) bank61507 +61508 POINT(40.085471063043975 74.34707404970032) bank61508 +61509 POINT(40.9444378769199 74.10417622695802) bank61509 +61510 POINT(39.78097995335418 74.784922255122) bank61510 +61511 POINT(41.28229915129428 73.34060401364366) bank61511 +61512 POINT(40.01106519994272 74.4503337486928) bank61512 +61513 POINT(40.83709161209383 74.75853717047545) bank61513 +61514 POINT(41.182754502970866 74.13162471790487) bank61514 +61515 POINT(40.01352383245617 74.80074395015957) bank61515 +61516 POINT(40.81591366127867 74.90897952090754) bank61516 +61517 POINT(39.81720597393795 73.33965867637062) bank61517 +61518 POINT(41.4296629727659 73.09074044709894) bank61518 +61519 POINT(40.677570048144645 74.635050311797) bank61519 +61520 POINT(40.54946006180278 74.23055908920685) bank61520 +61521 POINT(39.80572484934392 74.17223757925316) bank61521 +61522 POINT(41.27306350252221 73.78971989985823) bank61522 +61523 POINT(39.762035094622156 73.48100530062166) bank61523 +61524 POINT(40.2033541655385 73.49978095713583) bank61524 +61525 POINT(39.74650161015031 73.2122872953523) bank61525 +61526 POINT(40.37767669398724 74.01921057466897) bank61526 +61527 POINT(40.13326526933155 73.82448250386838) bank61527 +61528 POINT(40.21405072456579 73.32993708500193) bank61528 +61529 POINT(40.41080357834281 73.91562806341487) bank61529 +61530 POINT(41.12639197423425 73.79413250523277) bank61530 +61531 POINT(40.667247071564155 73.10794550261818) bank61531 +61532 POINT(40.08977207470507 74.72702176079599) bank61532 +61533 POINT(41.0012881549096 73.03182385874082) bank61533 +61534 POINT(41.39921433203513 73.04752972115034) bank61534 +61535 POINT(40.51373495512141 74.34133420769294) bank61535 +61536 POINT(39.87992189936224 73.38608690369567) bank61536 +61537 POINT(41.19207015941959 73.4596297192118) bank61537 +61538 POINT(41.621866098715806 73.59423432121955) bank61538 +61539 POINT(41.231556216329146 73.97109138881616) bank61539 +61540 POINT(41.317035387658464 73.01018400719808) bank61540 +61541 POINT(40.191346282957895 73.77734449743672) bank61541 +61542 POINT(40.959537181199174 73.31705798215776) bank61542 +61543 POINT(40.3629719250155 73.92107426962978) bank61543 +61544 POINT(40.042413309203944 74.57875395480956) bank61544 +61545 POINT(40.163301588576395 74.25269752405583) bank61545 +61546 POINT(41.146869933077305 74.03208383335698) bank61546 +61547 POINT(41.428662584165416 74.52337934333835) bank61547 +61548 POINT(41.391432057132555 74.48210196797096) bank61548 +61549 POINT(40.45991319376698 74.81759250006688) bank61549 +61550 POINT(40.692784365292326 74.76564006211647) bank61550 +61551 POINT(41.49482177420238 74.41588421516146) bank61551 +61552 POINT(40.761797733557124 73.422593904445) bank61552 +61553 POINT(41.43944659247449 73.13913877139319) bank61553 +61554 POINT(40.298597689995034 74.62308570011835) bank61554 +61555 POINT(40.13884770671012 74.8008906996965) bank61555 +61556 POINT(40.486681782950974 74.70300679084181) bank61556 +61557 POINT(41.19573167079788 74.6979124436217) bank61557 +61558 POINT(39.73780342263392 73.88183502295166) bank61558 +61559 POINT(41.10470471361705 74.87701906251614) bank61559 +61560 POINT(41.62512580165394 74.00893671663972) bank61560 +61561 POINT(41.18403478766173 73.66790561075699) bank61561 +61562 POINT(40.890974239058174 73.19955105229337) bank61562 +61563 POINT(40.95978654603151 73.11789926544978) bank61563 +61564 POINT(39.782576722383375 74.4136358015553) bank61564 +61565 POINT(40.68286758726175 74.69988246763187) bank61565 +61566 POINT(39.99877341465193 74.63124789958565) bank61566 +61567 POINT(40.876457370392394 73.64976691498981) bank61567 +61568 POINT(40.93619221280666 73.16529799832432) bank61568 +61569 POINT(41.38157076904865 74.31155315178499) bank61569 +61570 POINT(41.186976259990416 73.64858915374246) bank61570 +61571 POINT(41.439426447035295 73.52226081655867) bank61571 +61572 POINT(40.45092750988954 74.93369369459327) bank61572 +61573 POINT(39.7732600136116 73.10840247083482) bank61573 +61574 POINT(40.137427032558406 74.52813145708198) bank61574 +61575 POINT(40.223390089065866 73.0605759525024) bank61575 +61576 POINT(40.193781281725826 73.53102375357997) bank61576 +61577 POINT(41.442964109459055 73.55932694148105) bank61577 +61578 POINT(40.14014627811898 73.86468691212285) bank61578 +61579 POINT(40.525105975446536 73.18415369816333) bank61579 +61580 POINT(40.75880470752123 74.06234654128097) bank61580 +61581 POINT(40.22298913029973 74.26383733924864) bank61581 +61582 POINT(41.69847786299354 74.85651466756701) bank61582 +61583 POINT(40.91529177647734 73.3786061154975) bank61583 +61584 POINT(41.65096160700159 74.19103589996158) bank61584 +61585 POINT(40.39869892263494 74.45979250196824) bank61585 +61586 POINT(39.96551144472094 73.37412391525154) bank61586 +61587 POINT(40.55737265217493 74.40297023314248) bank61587 +61588 POINT(40.711785453978685 73.52314139711338) bank61588 +61589 POINT(41.38000068518908 73.97453486685137) bank61589 +61590 POINT(39.822067744769875 74.24301675947711) bank61590 +61591 POINT(40.606680630065924 73.16897954602699) bank61591 +61592 POINT(40.06415391651502 73.11633864001095) bank61592 +61593 POINT(40.47539958713612 74.84729094665411) bank61593 +61594 POINT(40.49404022806895 73.78281867378531) bank61594 +61595 POINT(40.97405688061141 73.60215430909308) bank61595 +61596 POINT(40.95932383012417 74.77608008735683) bank61596 +61597 POINT(40.71305301729095 73.31308674324706) bank61597 +61598 POINT(40.20658512489318 73.36724591609047) bank61598 +61599 POINT(41.58464264687881 73.68384073811562) bank61599 +61600 POINT(41.25626349106526 73.10067434746368) bank61600 +61601 POINT(40.6366326387419 74.34391316977849) bank61601 +61602 POINT(41.515303925288734 74.42129480401599) bank61602 +61603 POINT(40.5154976422913 74.3115882491489) bank61603 +61604 POINT(40.261872782819104 74.16646166539198) bank61604 +61605 POINT(41.06795921984054 74.98104743027336) bank61605 +61606 POINT(39.751960761431796 74.13883175170186) bank61606 +61607 POINT(41.605276446172404 74.72367183665057) bank61607 +61608 POINT(40.01280964772587 73.1486432031919) bank61608 +61609 POINT(39.81082766362207 73.22034177254596) bank61609 +61610 POINT(41.3619746493453 74.16687569276314) bank61610 +61611 POINT(41.572022065038155 74.64222210312491) bank61611 +61612 POINT(39.880492305692655 74.10294964344826) bank61612 +61613 POINT(40.443667510356754 73.03528603770425) bank61613 +61614 POINT(41.24896561149032 74.44047582898828) bank61614 +61615 POINT(40.299333233353515 73.03042048291358) bank61615 +61616 POINT(40.03207385135123 74.69455542424971) bank61616 +61617 POINT(40.47993270658054 73.74670112302928) bank61617 +61618 POINT(39.92013662381864 74.70672848199395) bank61618 +61619 POINT(40.08087956342469 73.66798139162441) bank61619 +61620 POINT(39.9314620979557 74.57045092482831) bank61620 +61621 POINT(40.1808152114447 73.64723598496961) bank61621 +61622 POINT(39.98845592530243 74.63692739285922) bank61622 +61623 POINT(41.0833452517659 74.62798582958783) bank61623 +61624 POINT(41.48865266519395 74.63025787496245) bank61624 +61625 POINT(41.5747928293832 74.17591385114001) bank61625 +61626 POINT(40.812985873561075 74.7171508989109) bank61626 +61627 POINT(40.68414811012056 73.45150403422373) bank61627 +61628 POINT(40.8305524598084 73.92003754179902) bank61628 +61629 POINT(41.155484798374815 74.41756483355296) bank61629 +61630 POINT(40.40927748116823 73.65697988717262) bank61630 +61631 POINT(40.41318442884623 73.14590445735804) bank61631 +61632 POINT(41.678069683658954 73.46309835209219) bank61632 +61633 POINT(41.461851563681165 73.26504121236967) bank61633 +61634 POINT(40.00504758208954 74.2761945655304) bank61634 +61635 POINT(40.86348490679708 74.46404833255056) bank61635 +61636 POINT(41.55358648821407 73.39634410237576) bank61636 +61637 POINT(40.91413761724803 73.94893473631056) bank61637 +61638 POINT(41.570549831148995 74.75235458568241) bank61638 +61639 POINT(40.13306698974877 73.08305784387537) bank61639 +61640 POINT(39.78110206849179 74.59371893950824) bank61640 +61641 POINT(40.14919223912908 74.51969163240791) bank61641 +61642 POINT(40.2523790756999 74.66159224501868) bank61642 +61643 POINT(40.488787083975716 74.75268704921069) bank61643 +61644 POINT(40.31492981901485 73.24532341307386) bank61644 +61645 POINT(41.12458853703126 73.81262645655043) bank61645 +61646 POINT(41.341826314351366 74.28357226619781) bank61646 +61647 POINT(39.89189357838709 74.39178873245304) bank61647 +61648 POINT(39.78778727446899 74.26381305883513) bank61648 +61649 POINT(41.38226301012069 73.28528237060107) bank61649 +61650 POINT(40.7256217320448 73.09535521637483) bank61650 +61651 POINT(40.898840971791074 74.54212227749747) bank61651 +61652 POINT(40.2737117737846 73.7452130874736) bank61652 +61653 POINT(41.48562094287015 73.76967721804894) bank61653 +61654 POINT(41.165885436605706 73.24280590154108) bank61654 +61655 POINT(40.307672842014874 73.23542956788276) bank61655 +61656 POINT(40.198555131375024 74.14006837275917) bank61656 +61657 POINT(39.96549382474972 74.36624124074191) bank61657 +61658 POINT(39.890717186057685 73.00897145793552) bank61658 +61659 POINT(40.64915090217747 73.16366239916542) bank61659 +61660 POINT(40.09070447542834 73.04245701189888) bank61660 +61661 POINT(39.739867645213266 74.54436975288871) bank61661 +61662 POINT(41.57590190820172 74.03963481100257) bank61662 +61663 POINT(41.63962107981042 74.50055235864309) bank61663 +61664 POINT(41.316017994839825 73.8931871299022) bank61664 +61665 POINT(40.127627287631064 74.09382720098061) bank61665 +61666 POINT(40.298111984974604 73.55572420101655) bank61666 +61667 POINT(41.70520446339375 73.64800780628079) bank61667 +61668 POINT(40.1707464118897 73.12000737356998) bank61668 +61669 POINT(39.72842421441559 74.81273830886529) bank61669 +61670 POINT(41.14090747096216 74.13018123313779) bank61670 +61671 POINT(40.11403263327479 74.9072787231305) bank61671 +61672 POINT(40.59311523103864 73.92339913111748) bank61672 +61673 POINT(40.059770572570926 74.178440105072) bank61673 +61674 POINT(41.35699374263687 73.97302589277257) bank61674 +61675 POINT(40.42833999214752 73.46461987595585) bank61675 +61676 POINT(40.22166947128368 74.93169475692436) bank61676 +61677 POINT(40.41252784635128 73.38408479014032) bank61677 +61678 POINT(40.36520145721063 74.20880381154639) bank61678 +61679 POINT(40.95868354383113 74.2310268998192) bank61679 +61680 POINT(40.677438508468256 73.35163095550602) bank61680 +61681 POINT(41.22757351290355 73.1910726736323) bank61681 +61682 POINT(40.32078745643838 73.0827900132843) bank61682 +61683 POINT(40.58492109385946 73.92714996926325) bank61683 +61684 POINT(40.22105829044553 73.68423797452469) bank61684 +61685 POINT(40.94262808670517 73.74296326069154) bank61685 +61686 POINT(41.143723303130486 74.32188035583887) bank61686 +61687 POINT(40.71380333446229 74.95020707146165) bank61687 +61688 POINT(40.827969886523555 73.61388680756448) bank61688 +61689 POINT(40.994097068845605 73.43556992511594) bank61689 +61690 POINT(41.4718138971174 74.46011219897663) bank61690 +61691 POINT(40.44995856785474 74.16460339344272) bank61691 +61692 POINT(39.77303507331064 73.65144656672058) bank61692 +61693 POINT(41.54275974988923 74.18537952528801) bank61693 +61694 POINT(41.16460383880262 73.48481887581659) bank61694 +61695 POINT(40.141957724090354 73.80985517145405) bank61695 +61696 POINT(41.34724646400784 74.13283112842622) bank61696 +61697 POINT(41.27387503731261 74.18329483365773) bank61697 +61698 POINT(41.32912925132065 74.23755504551364) bank61698 +61699 POINT(40.96552053280201 74.64724695452595) bank61699 +61700 POINT(40.61834729620012 74.88535890302586) bank61700 +61701 POINT(40.58808157476042 74.03927380189849) bank61701 +61702 POINT(41.0042639143581 74.61043784481427) bank61702 +61703 POINT(39.9066144969288 74.67682036331553) bank61703 +61704 POINT(40.02976529799176 73.41786699736099) bank61704 +61705 POINT(40.41744957930544 73.8141363632897) bank61705 +61706 POINT(41.2025591266573 74.04662098551344) bank61706 +61707 POINT(41.33600464730831 73.42517264285819) bank61707 +61708 POINT(40.87663223462018 74.53948361032667) bank61708 +61709 POINT(41.218107273000605 73.529817397883) bank61709 +61710 POINT(41.5185978882398 74.83024218188677) bank61710 +61711 POINT(41.13663393799026 73.72239979246778) bank61711 +61712 POINT(39.723122456934284 74.78999186318875) bank61712 +61713 POINT(41.27543742195922 73.9528406242398) bank61713 +61714 POINT(40.72249035064351 74.81566600656697) bank61714 +61715 POINT(41.155203387339355 74.14592733526266) bank61715 +61716 POINT(40.19968748337636 74.16571537225839) bank61716 +61717 POINT(40.03020256095041 74.25003860747431) bank61717 +61718 POINT(40.90172848063275 74.53363908615741) bank61718 +61719 POINT(40.875639653928474 73.40725287609376) bank61719 +61720 POINT(40.13589632613038 73.25350063276129) bank61720 +61721 POINT(41.17161180552439 74.1793596629106) bank61721 +61722 POINT(41.5065625855421 73.19827717029986) bank61722 +61723 POINT(40.35901959211471 73.63297566025486) bank61723 +61724 POINT(40.587229728587985 74.49794991038124) bank61724 +61725 POINT(40.351585901582 73.43967993051636) bank61725 +61726 POINT(40.84657858249872 74.75299826822925) bank61726 +61727 POINT(41.4179696159522 74.12143869506197) bank61727 +61728 POINT(41.10886610412938 73.64162143856632) bank61728 +61729 POINT(40.60085540479421 74.7798848713551) bank61729 +61730 POINT(40.41939453363739 73.3385463188233) bank61730 +61731 POINT(41.53487380567253 74.68660918429578) bank61731 +61732 POINT(40.470966348075024 73.61771865349913) bank61732 +61733 POINT(40.69499397593011 74.16703715932412) bank61733 +61734 POINT(41.104431955159605 73.25186010636938) bank61734 +61735 POINT(40.60611813438809 74.43085262288241) bank61735 +61736 POINT(40.240687323885666 74.99388910783028) bank61736 +61737 POINT(40.57079618212719 74.83479898960128) bank61737 +61738 POINT(41.28612823924115 74.761082283398) bank61738 +61739 POINT(41.52571729634773 74.56867954166853) bank61739 +61740 POINT(40.52494374777583 73.08589458944402) bank61740 +61741 POINT(40.075392319770124 73.25393886880386) bank61741 +61742 POINT(41.413115775514456 74.70023787092947) bank61742 +61743 POINT(40.145541067323954 74.45025634163002) bank61743 +61744 POINT(39.73387690212255 74.94464434358814) bank61744 +61745 POINT(40.35274013565393 73.64532964628788) bank61745 +61746 POINT(40.34838607546314 73.60542786883107) bank61746 +61747 POINT(39.84056583342 73.19473538111558) bank61747 +61748 POINT(39.98071011071351 73.07564901925801) bank61748 +61749 POINT(41.03214688496901 73.50242834270281) bank61749 +61750 POINT(40.66903582738421 73.40135831464593) bank61750 +61751 POINT(40.337038724121136 73.40170417207906) bank61751 +61752 POINT(40.324830344949085 73.35745754349296) bank61752 +61753 POINT(40.43569827614577 73.4171216298969) bank61753 +61754 POINT(40.6047450258605 73.0847953069173) bank61754 +61755 POINT(40.13821628639232 74.571575364037) bank61755 +61756 POINT(40.56477728349853 73.40187323523391) bank61756 +61757 POINT(41.559124808138264 74.95675084247657) bank61757 +61758 POINT(40.5289708981973 73.90378838610145) bank61758 +61759 POINT(41.4420504520455 73.56889680186396) bank61759 +61760 POINT(40.20893946224058 74.17042294491719) bank61760 +61761 POINT(39.97023774273617 73.16672941880843) bank61761 +61762 POINT(39.81814688519446 74.56693826595328) bank61762 +61763 POINT(40.89836787562426 73.76022156009616) bank61763 +61764 POINT(40.57856007620032 73.5370815841395) bank61764 +61765 POINT(41.07537926208727 74.8766721402954) bank61765 +61766 POINT(41.44503086802419 73.92694021443089) bank61766 +61767 POINT(39.88985319974597 73.99133747335439) bank61767 +61768 POINT(41.35299496947386 74.16617278882569) bank61768 +61769 POINT(41.18694224481611 74.74401181249104) bank61769 +61770 POINT(41.68435767142122 74.29224946754758) bank61770 +61771 POINT(40.49583649973294 74.11105726183938) bank61771 +61772 POINT(41.083607622047325 74.82154908975107) bank61772 +61773 POINT(40.79021180653921 74.22917286616585) bank61773 +61774 POINT(39.97771311570975 74.10566531917064) bank61774 +61775 POINT(40.04459116881841 74.54918902268315) bank61775 +61776 POINT(41.27596622795158 74.38086740539745) bank61776 +61777 POINT(40.741572591979434 73.86041342167735) bank61777 +61778 POINT(41.07096651974083 74.05698275890656) bank61778 +61779 POINT(41.613548389608724 74.22898709412728) bank61779 +61780 POINT(41.521022663362594 74.25009186323156) bank61780 +61781 POINT(40.097562779387 73.20491841624022) bank61781 +61782 POINT(40.193940927850804 74.72275756616499) bank61782 +61783 POINT(41.581965346663196 74.38760431669087) bank61783 +61784 POINT(40.51265457065329 73.23393359104388) bank61784 +61785 POINT(40.710475319278125 73.67558199379137) bank61785 +61786 POINT(40.48230131808339 73.28298045636531) bank61786 +61787 POINT(39.93267742320963 74.79461776032802) bank61787 +61788 POINT(39.718057000130145 74.05533280205523) bank61788 +61789 POINT(41.19135438834004 74.80481470359436) bank61789 +61790 POINT(40.24372080192184 74.84608844020471) bank61790 +61791 POINT(40.75575991424724 74.98980942656158) bank61791 +61792 POINT(41.41708934220774 73.4007984844072) bank61792 +61793 POINT(40.90030442944896 74.8230537539144) bank61793 +61794 POINT(39.987117871125534 74.5643048744458) bank61794 +61795 POINT(40.68761432945774 73.64726055670364) bank61795 +61796 POINT(40.90336288425631 74.15635625092301) bank61796 +61797 POINT(40.257200722226045 73.4495046153544) bank61797 +61798 POINT(39.91053459214133 73.48402129698212) bank61798 +61799 POINT(41.346057758093394 73.9143715104379) bank61799 +61800 POINT(40.20135109103508 74.6725895656396) bank61800 +61801 POINT(40.75708567470848 74.29370144963664) bank61801 +61802 POINT(40.561517844870835 73.37001843614509) bank61802 +61803 POINT(40.440114102864115 73.50516073749253) bank61803 +61804 POINT(40.97809228910514 74.4128061398466) bank61804 +61805 POINT(40.9728412191613 74.57416632025415) bank61805 +61806 POINT(41.61132608974723 73.17940955153158) bank61806 +61807 POINT(39.841817174556084 73.80350803869106) bank61807 +61808 POINT(41.36553965963949 73.18657960459836) bank61808 +61809 POINT(41.24290040261882 73.98682104513681) bank61809 +61810 POINT(40.526588590460605 73.07329390085592) bank61810 +61811 POINT(39.86166666031053 74.67319350172383) bank61811 +61812 POINT(41.14145984758583 73.97124356413077) bank61812 +61813 POINT(39.83063680806881 74.37955204431843) bank61813 +61814 POINT(40.4963085844459 74.60870737494986) bank61814 +61815 POINT(40.64926272360428 73.01259906439348) bank61815 +61816 POINT(41.380548788491204 74.32822829656476) bank61816 +61817 POINT(41.23178975296418 74.74482384544966) bank61817 +61818 POINT(41.28917798015981 74.78301394428424) bank61818 +61819 POINT(40.92354571656247 74.21642803613834) bank61819 +61820 POINT(40.72365695098223 73.81628903544862) bank61820 +61821 POINT(41.32010794162489 73.36068386733074) bank61821 +61822 POINT(40.41008009960205 73.22278536253549) bank61822 +61823 POINT(40.68387644569692 74.74941861330763) bank61823 +61824 POINT(40.340984448904784 74.31393324821309) bank61824 +61825 POINT(40.9151100092654 74.45978161942121) bank61825 +61826 POINT(41.33014098234128 73.07205940686029) bank61826 +61827 POINT(41.38634253853224 74.74952803856509) bank61827 +61828 POINT(40.74956641840066 73.10660218431931) bank61828 +61829 POINT(40.650530499714 74.31768021137795) bank61829 +61830 POINT(41.08225784516621 73.02686170625175) bank61830 +61831 POINT(40.37815045133608 73.60946477296355) bank61831 +61832 POINT(39.91082331304568 74.57163763012156) bank61832 +61833 POINT(41.38538378847363 73.62018024336719) bank61833 +61834 POINT(39.90604942455947 74.68184482705199) bank61834 +61835 POINT(40.834627225654216 74.98318979622182) bank61835 +61836 POINT(40.728108491001485 73.62126673718105) bank61836 +61837 POINT(40.28795090272297 73.5420847769466) bank61837 +61838 POINT(41.009967424189526 73.73686559045872) bank61838 +61839 POINT(40.02320263827095 73.57657918669743) bank61839 +61840 POINT(40.99880324745175 73.22484401778063) bank61840 +61841 POINT(40.34009573578545 73.90657681666684) bank61841 +61842 POINT(41.484545290429516 74.70432517394285) bank61842 +61843 POINT(40.35654389211365 73.53039233067759) bank61843 +61844 POINT(40.560185882854874 74.81852184687177) bank61844 +61845 POINT(41.36798930168296 73.49208411689929) bank61845 +61846 POINT(40.033226667842435 73.20240625717283) bank61846 +61847 POINT(40.951595572092955 74.08569597500822) bank61847 +61848 POINT(41.36503287941734 73.69734891639737) bank61848 +61849 POINT(39.952523226057 74.87976928617802) bank61849 +61850 POINT(39.76587790168398 73.01805839328995) bank61850 +61851 POINT(39.90260128883911 74.43946886667443) bank61851 +61852 POINT(40.28362824366264 73.66820267651796) bank61852 +61853 POINT(39.7412986049882 74.99933133745421) bank61853 +61854 POINT(40.001671963029914 74.59414842327668) bank61854 +61855 POINT(40.94274488143706 74.25918494715549) bank61855 +61856 POINT(40.677579322000014 74.53189397554706) bank61856 +61857 POINT(40.031074581694014 73.96358307360143) bank61857 +61858 POINT(40.46664331459328 74.1839309373278) bank61858 +61859 POINT(41.30489068918212 74.52712023415697) bank61859 +61860 POINT(40.124737782897434 74.35285097806492) bank61860 +61861 POINT(41.373187585887145 73.91769521510281) bank61861 +61862 POINT(40.551268441550775 73.72939506344215) bank61862 +61863 POINT(40.38710772794075 73.61473769548014) bank61863 +61864 POINT(39.91187162837708 73.10178613487797) bank61864 +61865 POINT(41.60086589953382 74.00604242673748) bank61865 +61866 POINT(41.5368538454707 74.94732721115905) bank61866 +61867 POINT(40.75803979165381 73.24722615002224) bank61867 +61868 POINT(40.446107004333975 73.20319542938881) bank61868 +61869 POINT(40.15036318964802 73.24926333591748) bank61869 +61870 POINT(40.71205872394524 73.83242515646019) bank61870 +61871 POINT(40.33931563174942 74.26561129902986) bank61871 +61872 POINT(40.0378624089499 74.9215441597525) bank61872 +61873 POINT(41.61114557618993 74.77811045783517) bank61873 +61874 POINT(40.55735774641574 73.45249216383422) bank61874 +61875 POINT(39.75565342647229 74.14493782611294) bank61875 +61876 POINT(41.03731053599212 74.38546190095927) bank61876 +61877 POINT(40.396471253277795 74.23649817241112) bank61877 +61878 POINT(40.84321050555892 74.66026011878394) bank61878 +61879 POINT(41.37217982965207 74.04059140237942) bank61879 +61880 POINT(41.153911255588305 73.449027630493) bank61880 +61881 POINT(41.3017040238111 74.76233581907113) bank61881 +61882 POINT(40.14172417088735 73.49295673228646) bank61882 +61883 POINT(41.02819989513841 74.41129628730289) bank61883 +61884 POINT(39.83594668955547 73.26172262260452) bank61884 +61885 POINT(40.30905398737328 74.10956207991168) bank61885 +61886 POINT(40.37837510457899 73.9071645160516) bank61886 +61887 POINT(41.50111999186774 74.1621582512534) bank61887 +61888 POINT(40.19261522046829 74.06015713844782) bank61888 +61889 POINT(39.72397879087735 73.42652353275048) bank61889 +61890 POINT(41.22025087155265 73.10279467012872) bank61890 +61891 POINT(41.45454930705052 73.83572877252627) bank61891 +61892 POINT(40.868174118374704 74.35718933155991) bank61892 +61893 POINT(40.71259759933774 73.05192300092472) bank61893 +61894 POINT(40.45349069431437 74.5929540016291) bank61894 +61895 POINT(40.92883095283905 74.66216851863837) bank61895 +61896 POINT(39.87121879268478 74.50692208128697) bank61896 +61897 POINT(41.05510861756232 74.99358668756818) bank61897 +61898 POINT(41.31689287372589 73.6658413841738) bank61898 +61899 POINT(41.18133186845462 73.51607639924296) bank61899 +61900 POINT(39.77645590917189 73.5943505308031) bank61900 +61901 POINT(40.88779910490756 73.43891539037266) bank61901 +61902 POINT(40.34036722672042 73.12958301270154) bank61902 +61903 POINT(39.727298493298704 73.19725038318666) bank61903 +61904 POINT(40.467272598101886 73.82415159985244) bank61904 +61905 POINT(39.90191231270272 73.04539528688967) bank61905 +61906 POINT(40.84167689279784 74.90367502338356) bank61906 +61907 POINT(40.93124073836918 73.16390865885795) bank61907 +61908 POINT(41.36528597579886 73.0274157080448) bank61908 +61909 POINT(40.71318296878972 73.37086663105258) bank61909 +61910 POINT(39.770795912769316 73.46406248365182) bank61910 +61911 POINT(40.358080370408224 74.13154672106259) bank61911 +61912 POINT(40.88164410580204 74.46521607316062) bank61912 +61913 POINT(41.67225298974846 73.96364961743596) bank61913 +61914 POINT(39.914770768532044 73.99332621631876) bank61914 +61915 POINT(40.238372320892424 74.35269206061454) bank61915 +61916 POINT(40.051237828654855 74.44204767037262) bank61916 +61917 POINT(40.17526920251328 74.14652885599342) bank61917 +61918 POINT(39.80193826939965 73.43428705822035) bank61918 +61919 POINT(40.675498155800874 73.37148611579677) bank61919 +61920 POINT(41.08494517771928 74.1754142494394) bank61920 +61921 POINT(40.94940943287551 74.38788626641268) bank61921 +61922 POINT(41.29617186866048 73.33395060614204) bank61922 +61923 POINT(40.071622568617435 74.57972191616065) bank61923 +61924 POINT(41.592698154321525 74.84074668413548) bank61924 +61925 POINT(41.33347218980587 74.43763338614598) bank61925 +61926 POINT(40.696943467779604 73.36040048712766) bank61926 +61927 POINT(40.71692055682834 73.3794094336378) bank61927 +61928 POINT(40.887467348637216 74.27202917744101) bank61928 +61929 POINT(39.91230019583852 73.08134509492542) bank61929 +61930 POINT(40.00398915466015 74.27992825563119) bank61930 +61931 POINT(40.2736781270282 74.34334367139734) bank61931 +61932 POINT(41.422213794559276 73.13261518780293) bank61932 +61933 POINT(41.5992149085413 74.96321868092801) bank61933 +61934 POINT(40.23012984076666 74.0107923367676) bank61934 +61935 POINT(41.17688737007218 74.53379547068468) bank61935 +61936 POINT(41.34964960832437 73.44834637101557) bank61936 +61937 POINT(40.17030010298347 73.5800488369262) bank61937 +61938 POINT(41.06802209696206 74.8766911742355) bank61938 +61939 POINT(39.912392325719814 74.66192043857816) bank61939 +61940 POINT(40.027411531953696 74.85623034738386) bank61940 +61941 POINT(40.03982245052031 74.01852786961425) bank61941 +61942 POINT(41.61958182055279 73.29388366670592) bank61942 +61943 POINT(40.98149876376756 74.92397650866046) bank61943 +61944 POINT(41.020322862360494 73.12199993530241) bank61944 +61945 POINT(41.095380070142525 73.90829925755851) bank61945 +61946 POINT(39.874941788679806 73.62067738810917) bank61946 +61947 POINT(40.17775747328315 74.33258353828077) bank61947 +61948 POINT(40.56154200435691 74.12795760425783) bank61948 +61949 POINT(40.11471493438355 74.6565764017571) bank61949 +61950 POINT(40.465627665663774 73.19653928976071) bank61950 +61951 POINT(41.492037012552956 74.06171006538312) bank61951 +61952 POINT(41.48907487753066 73.4347710889889) bank61952 +61953 POINT(40.560618634343776 74.56803450134734) bank61953 +61954 POINT(41.40062576993 74.63249374119015) bank61954 +61955 POINT(39.901811532281215 73.20084290053325) bank61955 +61956 POINT(40.38532486008238 74.425884476794) bank61956 +61957 POINT(41.009013871790536 74.6576405718896) bank61957 +61958 POINT(41.087532514417944 73.29983637427897) bank61958 +61959 POINT(40.345268784497485 74.33814582654068) bank61959 +61960 POINT(40.271745536902664 73.88140651253048) bank61960 +61961 POINT(40.28829799761727 73.31243369542423) bank61961 +61962 POINT(40.561276455100874 73.3885776314027) bank61962 +61963 POINT(40.87314213015068 73.16558907302093) bank61963 +61964 POINT(41.6677471943856 74.27951238069275) bank61964 +61965 POINT(39.92058813225199 74.8559054190668) bank61965 +61966 POINT(40.33262651665868 74.79901498213574) bank61966 +61967 POINT(40.20611525387929 73.44801236077296) bank61967 +61968 POINT(39.99693007500585 73.38524123711417) bank61968 +61969 POINT(41.23688631160874 74.1025019272501) bank61969 +61970 POINT(41.30109334040723 74.72298162499396) bank61970 +61971 POINT(40.58185694017362 73.36887527268878) bank61971 +61972 POINT(40.21514751826679 74.78788472383077) bank61972 +61973 POINT(39.8095468173423 74.96575740254298) bank61973 +61974 POINT(40.256424806793284 73.928558011793) bank61974 +61975 POINT(40.565863886451616 74.5100688040614) bank61975 +61976 POINT(40.75152905277631 74.09897436905827) bank61976 +61977 POINT(41.28542699415954 73.72721699349044) bank61977 +61978 POINT(40.991863369288225 74.10982042429065) bank61978 +61979 POINT(40.918683842622436 73.49755516730418) bank61979 +61980 POINT(41.12610860575262 74.76714949700519) bank61980 +61981 POINT(41.27343203389955 74.18955952079416) bank61981 +61982 POINT(39.915838131410354 74.3191186467595) bank61982 +61983 POINT(40.94590671336478 74.23698151609398) bank61983 +61984 POINT(40.09196006405954 74.94271823479669) bank61984 +61985 POINT(40.30103784076127 74.47555574068741) bank61985 +61986 POINT(40.71716042326579 74.03094940790939) bank61986 +61987 POINT(41.13513456575783 74.49400305634515) bank61987 +61988 POINT(40.27720487808038 73.77337987192418) bank61988 +61989 POINT(40.96208268722203 73.34010695082388) bank61989 +61990 POINT(41.101193480470315 74.88523341878413) bank61990 +61991 POINT(40.73012092130948 74.18088297466645) bank61991 +61992 POINT(39.89164261279021 74.90068402904194) bank61992 +61993 POINT(40.802548814896625 74.33588375366234) bank61993 +61994 POINT(40.80166191117073 74.6286748090858) bank61994 +61995 POINT(41.622721133137475 74.14179478380625) bank61995 +61996 POINT(40.491391950705534 74.59946274815735) bank61996 +61997 POINT(40.507040412909134 73.43873516754711) bank61997 +61998 POINT(40.52579233112241 74.09876893793523) bank61998 +61999 POINT(40.87883823732853 74.8044344000552) bank61999 +62000 POINT(41.615117798637876 73.50420625364818) bank62000 +62001 POINT(41.47553942718077 74.25177823786169) bank62001 +62002 POINT(41.34929141043949 74.2672129449867) bank62002 +62003 POINT(41.34444061680939 73.04304591068889) bank62003 +62004 POINT(41.11313184669756 74.20624629308399) bank62004 +62005 POINT(40.573744955282336 73.9897958223285) bank62005 +62006 POINT(41.52925141485361 74.99339768487809) bank62006 +62007 POINT(40.8265334936986 74.59000592720281) bank62007 +62008 POINT(40.67770176829214 73.20917996498139) bank62008 +62009 POINT(41.49971649822835 73.44182784478542) bank62009 +62010 POINT(41.507363914303966 73.45439429263469) bank62010 +62011 POINT(40.3235569128952 74.51345528959514) bank62011 +62012 POINT(40.153217052705074 73.53414264826185) bank62012 +62013 POINT(40.57870455805903 74.2925667179222) bank62013 +62014 POINT(41.27484069315925 74.018989493722) bank62014 +62015 POINT(40.733534383836066 74.08501517489856) bank62015 +62016 POINT(40.81934960717723 73.98596236123441) bank62016 +62017 POINT(39.763474400434106 74.1360012215365) bank62017 +62018 POINT(40.52356798339865 74.04912060857342) bank62018 +62019 POINT(40.746636600596936 74.72420237789161) bank62019 +62020 POINT(39.865262257460316 74.28275303271586) bank62020 +62021 POINT(40.454248360770244 74.301758833599) bank62021 +62022 POINT(40.37430809164315 73.3845520535191) bank62022 +62023 POINT(41.26090188887529 73.98703243421558) bank62023 +62024 POINT(41.39980003362878 74.94563756368825) bank62024 +62025 POINT(41.676435153429324 74.09799851269946) bank62025 +62026 POINT(40.41469346177218 74.80625982002005) bank62026 +62027 POINT(40.50774601898292 73.26146122201197) bank62027 +62028 POINT(40.526314801281615 73.12424813930919) bank62028 +62029 POINT(40.418036302811714 73.09157350211933) bank62029 +62030 POINT(41.197276828122156 73.7680649016769) bank62030 +62031 POINT(40.1308057643478 74.33880401752235) bank62031 +62032 POINT(40.46962315597578 74.91470957180915) bank62032 +62033 POINT(39.75667819924009 73.02068026324808) bank62033 +62034 POINT(40.78403967626355 74.81810228319114) bank62034 +62035 POINT(39.77779139449685 74.84072066550006) bank62035 +62036 POINT(40.367794881720165 74.47904347191775) bank62036 +62037 POINT(39.825826996993435 73.30297695081066) bank62037 +62038 POINT(41.50208471307411 73.30878608834996) bank62038 +62039 POINT(41.23799087792431 73.72217800651192) bank62039 +62040 POINT(41.15091271577347 74.59828386312475) bank62040 +62041 POINT(40.232380486527674 73.00747707542689) bank62041 +62042 POINT(40.29982058130926 74.31090791191072) bank62042 +62043 POINT(41.008585481290766 73.15305810461568) bank62043 +62044 POINT(40.28124618471467 74.62930239199622) bank62044 +62045 POINT(39.916979308275174 73.70593033858358) bank62045 +62046 POINT(41.452234098387756 73.14679953020375) bank62046 +62047 POINT(40.64946650342326 73.21330312847262) bank62047 +62048 POINT(41.391917718904224 73.52475711108447) bank62048 +62049 POINT(40.04980521070892 73.98512107928764) bank62049 +62050 POINT(41.3027439479577 73.56214289169678) bank62050 +62051 POINT(41.37431277694197 73.17558926523601) bank62051 +62052 POINT(40.81809772314574 73.12368001345455) bank62052 +62053 POINT(41.43478886571774 73.11155647467787) bank62053 +62054 POINT(40.92327101451095 73.99658902850912) bank62054 +62055 POINT(41.05245585183593 74.55552358472708) bank62055 +62056 POINT(41.11908790934244 74.39959593952157) bank62056 +62057 POINT(39.78879020982524 74.90234119269616) bank62057 +62058 POINT(39.82471662585867 74.2232245489073) bank62058 +62059 POINT(39.92796738187025 74.30079258123818) bank62059 +62060 POINT(40.04591404066311 74.67233215468457) bank62060 +62061 POINT(41.27774319667287 74.26812529759084) bank62061 +62062 POINT(41.4564079540419 73.44383370417277) bank62062 +62063 POINT(41.61497190643692 73.91185770672445) bank62063 +62064 POINT(40.30023083984332 74.74660591505932) bank62064 +62065 POINT(41.233932469124916 74.9776260776836) bank62065 +62066 POINT(39.90796021732984 74.14884416504546) bank62066 +62067 POINT(41.10805806719076 74.14687756437156) bank62067 +62068 POINT(41.258769197537895 74.411872486231) bank62068 +62069 POINT(40.84437606683453 74.04033982248346) bank62069 +62070 POINT(39.726930113921334 73.68472425844375) bank62070 +62071 POINT(40.03399933602612 73.5348253286432) bank62071 +62072 POINT(40.68968585071259 74.90453318557388) bank62072 +62073 POINT(40.44135552442511 73.27878222386244) bank62073 +62074 POINT(40.16872387266507 74.53415657325952) bank62074 +62075 POINT(39.753044046873775 74.40787295853198) bank62075 +62076 POINT(39.986071868608484 73.92262767575744) bank62076 +62077 POINT(41.6117441814448 74.81375645761273) bank62077 +62078 POINT(39.86175094370223 74.72347855238493) bank62078 +62079 POINT(40.422439310832104 73.78451976230926) bank62079 +62080 POINT(39.913927636425974 74.08175044131006) bank62080 +62081 POINT(41.496475140312874 73.80036510212268) bank62081 +62082 POINT(40.41709337540606 74.05317953659987) bank62082 +62083 POINT(41.69780324917379 74.29057740839546) bank62083 +62084 POINT(40.2491394294017 73.36615285955091) bank62084 +62085 POINT(41.442946593068754 74.96620492614386) bank62085 +62086 POINT(40.298885571712354 73.96756054270077) bank62086 +62087 POINT(41.35480660774186 73.65147117906436) bank62087 +62088 POINT(40.64099984407842 74.527634421136) bank62088 +62089 POINT(40.85318419449427 74.32129432169928) bank62089 +62090 POINT(40.55797178074635 74.59083514173977) bank62090 +62091 POINT(41.20959690724524 73.60029470439363) bank62091 +62092 POINT(40.77398412561214 74.01156474316704) bank62092 +62093 POINT(39.96622929312352 74.59045281046652) bank62093 +62094 POINT(40.98732526208784 74.15327681629283) bank62094 +62095 POINT(39.826923506938634 74.54575361077461) bank62095 +62096 POINT(39.86448096678356 74.18544498757986) bank62096 +62097 POINT(39.85973794196687 74.81548558355796) bank62097 +62098 POINT(40.767195345059875 73.59821772566825) bank62098 +62099 POINT(40.19729906397039 73.7344704504183) bank62099 +62100 POINT(40.98873687493221 74.87434998159085) bank62100 +62101 POINT(40.90269625718722 73.66544488544002) bank62101 +62102 POINT(40.10663981826187 73.11260229379111) bank62102 +62103 POINT(41.50128248537617 73.75683241398586) bank62103 +62104 POINT(40.89220879107042 74.39645006428991) bank62104 +62105 POINT(39.90820099607236 73.3954860748079) bank62105 +62106 POINT(41.611699332204445 73.03292081132345) bank62106 +62107 POINT(41.60051409963813 74.63220088157824) bank62107 +62108 POINT(39.85953043194633 73.36545293389966) bank62108 +62109 POINT(40.063696342011596 74.33702566478) bank62109 +62110 POINT(41.08293834344643 73.64291123373701) bank62110 +62111 POINT(41.62933501195784 73.48979789730018) bank62111 +62112 POINT(40.99124915743635 73.22473677157768) bank62112 +62113 POINT(40.605575731675444 74.35751266252545) bank62113 +62114 POINT(40.269698853620625 73.45182879045133) bank62114 +62115 POINT(40.811704645059145 73.15583104016834) bank62115 +62116 POINT(40.049890562242496 73.93966787250012) bank62116 +62117 POINT(40.651759366989076 74.45709309306899) bank62117 +62118 POINT(40.88227808871012 74.73334778627145) bank62118 +62119 POINT(39.79167859844184 73.71413484761237) bank62119 +62120 POINT(40.6826631662804 73.81985626416821) bank62120 +62121 POINT(41.33572826836707 74.16451775731602) bank62121 +62122 POINT(39.83631399030122 74.32366427205866) bank62122 +62123 POINT(41.696172059996414 74.01481338134583) bank62123 +62124 POINT(40.7676959550931 74.26661037463315) bank62124 +62125 POINT(39.74717910083248 74.73944905712453) bank62125 +62126 POINT(41.06548318753636 74.30727233096233) bank62126 +62127 POINT(40.82203732276531 73.6452644946637) bank62127 +62128 POINT(40.34062935091095 74.64405135487323) bank62128 +62129 POINT(40.924601767551536 73.79773961140667) bank62129 +62130 POINT(40.775806584016536 74.90504470395109) bank62130 +62131 POINT(40.480847119850665 73.15678014329637) bank62131 +62132 POINT(41.02663062893117 74.87068602612018) bank62132 +62133 POINT(41.59681276522018 74.03572606927212) bank62133 +62134 POINT(41.24740187717379 73.51125034487922) bank62134 +62135 POINT(39.95896543980501 73.52299334282812) bank62135 +62136 POINT(41.02256985313187 74.86013747685392) bank62136 +62137 POINT(40.540424110890655 74.5102061965734) bank62137 +62138 POINT(41.41737149831276 74.26711081452952) bank62138 +62139 POINT(40.67420651669167 73.76410590286555) bank62139 +62140 POINT(41.51190334343084 73.44349655214208) bank62140 +62141 POINT(40.730387864994846 73.52847190186829) bank62141 +62142 POINT(40.33728778838911 74.72213405197434) bank62142 +62143 POINT(40.42274720459264 74.79477647976086) bank62143 +62144 POINT(41.594819617899226 73.77854885937968) bank62144 +62145 POINT(40.730454879719446 73.44314407505024) bank62145 +62146 POINT(40.442535759898824 73.74657823077568) bank62146 +62147 POINT(40.01609074354541 74.14590916781141) bank62147 +62148 POINT(40.71109390425568 73.158712407307) bank62148 +62149 POINT(39.88975117470842 73.52694902823984) bank62149 +62150 POINT(40.1522562458306 73.85198198906942) bank62150 +62151 POINT(39.86937538385478 73.13866036797313) bank62151 +62152 POINT(41.41364607717379 74.50231139878116) bank62152 +62153 POINT(39.72398120889578 74.9356332683729) bank62153 +62154 POINT(40.20427468456845 73.97746115458332) bank62154 +62155 POINT(39.94431801326901 73.39685171385175) bank62155 +62156 POINT(41.651522409557096 74.70436763614457) bank62156 +62157 POINT(40.060000316905445 73.34671912245396) bank62157 +62158 POINT(40.39884582278038 74.03007238545194) bank62158 +62159 POINT(40.05628474410813 73.08642120884802) bank62159 +62160 POINT(40.255035596635494 73.78634714605849) bank62160 +62161 POINT(41.313270341926405 73.89308891198492) bank62161 +62162 POINT(41.44010586838328 73.33023861976474) bank62162 +62163 POINT(40.10728316013647 73.3952706991029) bank62163 +62164 POINT(40.35369812359524 73.02089746987988) bank62164 +62165 POINT(40.43818945754962 73.9789737833539) bank62165 +62166 POINT(40.04163235134889 73.72615135901701) bank62166 +62167 POINT(41.403029084855476 74.92570884835473) bank62167 +62168 POINT(40.776418775266684 74.50409344348411) bank62168 +62169 POINT(41.53592292133073 73.32243482006511) bank62169 +62170 POINT(39.71787230110177 74.94177003210382) bank62170 +62171 POINT(40.116658393003895 73.57439005915913) bank62171 +62172 POINT(40.590182945858544 74.8255679285129) bank62172 +62173 POINT(40.76085938519077 74.82105996785356) bank62173 +62174 POINT(40.43540164076307 74.08185809408084) bank62174 +62175 POINT(41.27359224210485 73.0952930552385) bank62175 +62176 POINT(39.77865262449365 73.85930036896568) bank62176 +62177 POINT(40.377547341805744 73.28201675946416) bank62177 +62178 POINT(40.90595299939248 73.06343557366546) bank62178 +62179 POINT(40.27344885784059 73.33684929814571) bank62179 +62180 POINT(40.31095121419352 74.20422289692692) bank62180 +62181 POINT(41.46619712487525 74.35268602258598) bank62181 +62182 POINT(40.40372806284512 74.93133242928347) bank62182 +62183 POINT(40.805628755479916 73.98887759637196) bank62183 +62184 POINT(39.9042076669241 74.2113691643377) bank62184 +62185 POINT(41.25698207750444 73.57025759469737) bank62185 +62186 POINT(40.6031934358071 74.67678165695477) bank62186 +62187 POINT(39.914917329405085 73.11277540470502) bank62187 +62188 POINT(40.53949241134608 74.81997750534285) bank62188 +62189 POINT(39.78537314129567 74.47992067131747) bank62189 +62190 POINT(40.27863981942398 74.77335505743916) bank62190 +62191 POINT(41.24116092219011 73.56476161540695) bank62191 +62192 POINT(39.77939399666067 74.73222076902684) bank62192 +62193 POINT(41.50364263028563 73.58237003567368) bank62193 +62194 POINT(40.102353274553 74.54567082854227) bank62194 +62195 POINT(40.35204983827775 74.07636455598262) bank62195 +62196 POINT(41.16893421082868 74.24789124715424) bank62196 +62197 POINT(41.63483419992165 73.3023381288467) bank62197 +62198 POINT(39.807739774989955 73.93821094756186) bank62198 +62199 POINT(39.77484151171281 73.64388997037116) bank62199 +62200 POINT(40.24507608443597 74.46366695077336) bank62200 +62201 POINT(40.34001652821077 74.31206533763876) bank62201 +62202 POINT(41.17445662844888 74.87460457058906) bank62202 +62203 POINT(40.9034729592699 73.38370880378976) bank62203 +62204 POINT(39.93676483611464 73.69747296957827) bank62204 +62205 POINT(40.08061764632641 73.56166110270756) bank62205 +62206 POINT(40.37836217492849 73.72775567518701) bank62206 +62207 POINT(40.39899367433203 74.33688346694389) bank62207 +62208 POINT(40.88649786040081 74.6174667923381) bank62208 +62209 POINT(40.3171025702121 74.62862959684469) bank62209 +62210 POINT(41.17868855525106 74.7457408449431) bank62210 +62211 POINT(41.35712920327831 73.91470381328148) bank62211 +62212 POINT(40.42083838172739 74.31158596872515) bank62212 +62213 POINT(40.617986996397434 74.40595440931283) bank62213 +62214 POINT(40.87473312912466 73.52929809797126) bank62214 +62215 POINT(40.63705692147736 74.47293508462856) bank62215 +62216 POINT(41.30127175023044 74.25572867775375) bank62216 +62217 POINT(40.26315320795405 74.29016456508909) bank62217 +62218 POINT(39.72749415962949 74.29668642892764) bank62218 +62219 POINT(40.71837243540621 74.54037878109374) bank62219 +62220 POINT(40.37867908330191 74.0821309556859) bank62220 +62221 POINT(40.85001253746723 73.88987380657245) bank62221 +62222 POINT(40.25198136294543 74.57745631461688) bank62222 +62223 POINT(40.77450308898123 74.07652139070863) bank62223 +62224 POINT(39.90348584947887 73.34175922222255) bank62224 +62225 POINT(39.85659456481857 74.28534024045253) bank62225 +62226 POINT(39.89700214883556 74.92885543730736) bank62226 +62227 POINT(41.159100364670365 74.54894524217056) bank62227 +62228 POINT(40.207183931222126 74.47252190578446) bank62228 +62229 POINT(40.03270378913893 74.20950782949576) bank62229 +62230 POINT(41.412232516431146 73.61318423198828) bank62230 +62231 POINT(41.04311117573589 73.65743787878753) bank62231 +62232 POINT(41.25491336377639 73.25902840490765) bank62232 +62233 POINT(40.50593795930964 73.46318761839349) bank62233 +62234 POINT(41.3683662614045 74.05269833681596) bank62234 +62235 POINT(40.50025620786509 73.99873368547459) bank62235 +62236 POINT(41.23539435160696 74.96648461523823) bank62236 +62237 POINT(40.608133677961455 73.8705461064336) bank62237 +62238 POINT(39.811306992511504 73.363301859414) bank62238 +62239 POINT(39.81486335889747 73.44485115164755) bank62239 +62240 POINT(40.730851398465454 74.33303283172131) bank62240 +62241 POINT(40.033494366199115 74.14795169748011) bank62241 +62242 POINT(40.47155141061527 73.10848780524024) bank62242 +62243 POINT(39.926497439783645 74.41914075167774) bank62243 +62244 POINT(41.5833993507173 74.46243443932967) bank62244 +62245 POINT(40.62872761756117 74.3645737158594) bank62245 +62246 POINT(41.53448880329925 74.41766185306277) bank62246 +62247 POINT(41.05230767134085 73.2314147598828) bank62247 +62248 POINT(40.058888555994514 73.02191555735581) bank62248 +62249 POINT(40.13876585942461 73.89642215867502) bank62249 +62250 POINT(41.190553885112216 74.47597772799273) bank62250 +62251 POINT(40.14353012474622 74.13201134936949) bank62251 +62252 POINT(40.21631493243841 74.17728062240681) bank62252 +62253 POINT(41.06600417633148 73.80474883768272) bank62253 +62254 POINT(41.477501978558045 74.69219954495708) bank62254 +62255 POINT(40.090617054303905 73.32003434283996) bank62255 +62256 POINT(40.48697654313665 73.15801120781323) bank62256 +62257 POINT(40.44228896481601 74.54708666099802) bank62257 +62258 POINT(40.084621100320646 74.8112208216128) bank62258 +62259 POINT(40.3033009443263 73.88093404638435) bank62259 +62260 POINT(39.955210593059775 74.62664082226873) bank62260 +62261 POINT(39.717102627018825 73.48125487419074) bank62261 +62262 POINT(40.02172677611698 74.07570940003131) bank62262 +62263 POINT(41.03170690152443 73.88424023292366) bank62263 +62264 POINT(40.408991266672466 74.34898957523413) bank62264 +62265 POINT(41.48500533089671 73.43769675189341) bank62265 +62266 POINT(41.261529020161646 73.93993942474677) bank62266 +62267 POINT(41.535326836709125 73.9591465363496) bank62267 +62268 POINT(40.15030518069945 73.9821169259444) bank62268 +62269 POINT(41.407981289222214 74.16457930665506) bank62269 +62270 POINT(39.78372261882689 74.74128597396424) bank62270 +62271 POINT(40.943654100044746 73.42281504992584) bank62271 +62272 POINT(40.9266139847579 73.42890417685098) bank62272 +62273 POINT(40.490380201915784 74.93736571869977) bank62273 +62274 POINT(39.81969183629771 74.98144635359674) bank62274 +62275 POINT(39.86057942180776 74.72120767013827) bank62275 +62276 POINT(41.322070135928506 73.6629303814655) bank62276 +62277 POINT(41.190027356257225 74.12899431645128) bank62277 +62278 POINT(41.343207802772255 73.49733809789943) bank62278 +62279 POINT(41.545407621580985 73.25759851751955) bank62279 +62280 POINT(40.05454413281272 73.91721087884615) bank62280 +62281 POINT(40.24586418398517 73.80255029715391) bank62281 +62282 POINT(41.42771291887598 73.56951297131648) bank62282 +62283 POINT(40.46776112273323 73.16139837672218) bank62283 +62284 POINT(40.14613694991859 73.20334167626142) bank62284 +62285 POINT(40.14846363881995 73.15462334606809) bank62285 +62286 POINT(40.1457207963981 73.53026691697035) bank62286 +62287 POINT(40.80119191020505 74.96547500609134) bank62287 +62288 POINT(40.61334288722511 73.22545270911164) bank62288 +62289 POINT(40.42586825929509 73.09875397081987) bank62289 +62290 POINT(40.59977959824947 74.0582646328337) bank62290 +62291 POINT(40.29368223772839 73.45942293761382) bank62291 +62292 POINT(40.67623450355549 73.70618690335509) bank62292 +62293 POINT(40.95299801637266 73.51487108001689) bank62293 +62294 POINT(40.52838938179417 74.72073292551795) bank62294 +62295 POINT(41.57122218317483 73.41291389119853) bank62295 +62296 POINT(39.86944420349809 73.89076321576333) bank62296 +62297 POINT(41.32310641748051 74.78914546085603) bank62297 +62298 POINT(41.27947884819902 73.21016841758338) bank62298 +62299 POINT(40.45246773009542 73.3839035078652) bank62299 +62300 POINT(40.726619204158865 74.93767363319508) bank62300 +62301 POINT(41.13604539384051 74.33910882907084) bank62301 +62302 POINT(40.88338840745751 73.35978965659586) bank62302 +62303 POINT(40.622579806412965 73.50351051100195) bank62303 +62304 POINT(40.92651876271306 74.59082046062522) bank62304 +62305 POINT(39.73773025684841 73.74956853931343) bank62305 +62306 POINT(40.52470686042096 74.00365548607404) bank62306 +62307 POINT(41.24478926355482 73.32915087184018) bank62307 +62308 POINT(41.33422288993035 74.32310464739784) bank62308 +62309 POINT(40.196075232233675 74.46007365511463) bank62309 +62310 POINT(40.996070830478246 74.70814269841517) bank62310 +62311 POINT(41.04024764942628 73.14905836777235) bank62311 +62312 POINT(40.87506541056273 74.37006263807736) bank62312 +62313 POINT(40.923554056079624 74.06105686055749) bank62313 +62314 POINT(41.634533088364684 74.33183283224935) bank62314 +62315 POINT(40.14060929694466 73.62546005680971) bank62315 +62316 POINT(40.134625665755514 74.25473252592445) bank62316 +62317 POINT(39.998294799618094 73.71063961568927) bank62317 +62318 POINT(40.354835749190094 73.33179340340135) bank62318 +62319 POINT(40.482105298494275 74.92631856755857) bank62319 +62320 POINT(40.36761850159024 74.47667064414978) bank62320 +62321 POINT(39.883616371001146 73.7799121186952) bank62321 +62322 POINT(40.465893459423604 74.0538655853092) bank62322 +62323 POINT(41.44289597184364 74.38403149503037) bank62323 +62324 POINT(41.63639959602003 73.0609610864863) bank62324 +62325 POINT(40.209540561089945 74.4012137886085) bank62325 +62326 POINT(39.92900918221771 73.73271963731221) bank62326 +62327 POINT(39.79106258279239 73.74446000067827) bank62327 +62328 POINT(39.99918183130916 74.03378574639818) bank62328 +62329 POINT(40.871711147365524 74.24238328252416) bank62329 +62330 POINT(41.636657539682 74.6423187622878) bank62330 +62331 POINT(40.662246657754764 74.6576464826743) bank62331 +62332 POINT(40.05444839361482 74.80162295289541) bank62332 +62333 POINT(40.5849281659939 74.54796730011371) bank62333 +62334 POINT(41.65358750246874 73.96999253810172) bank62334 +62335 POINT(40.455987306374226 73.94411050598907) bank62335 +62336 POINT(40.83577046418934 74.56242655205735) bank62336 +62337 POINT(40.75174744867816 73.67562452221976) bank62337 +62338 POINT(40.095507308709095 73.25546340866006) bank62338 +62339 POINT(41.36802830643821 73.50709670413642) bank62339 +62340 POINT(41.36267327116607 74.53302599930211) bank62340 +62341 POINT(40.73110964142961 73.579058365333) bank62341 +62342 POINT(40.375436172938485 73.79974776176473) bank62342 +62343 POINT(40.12022329654914 74.19625100460641) bank62343 +62344 POINT(40.91602168475011 73.43431059336288) bank62344 +62345 POINT(39.90295679714876 74.73816899853705) bank62345 +62346 POINT(40.14504470234079 74.15672011904249) bank62346 +62347 POINT(40.1370176839314 73.64571844128912) bank62347 +62348 POINT(40.46788081409554 73.67436213097315) bank62348 +62349 POINT(40.97500824149974 73.13563776730162) bank62349 +62350 POINT(41.217347938807166 74.33125438130972) bank62350 +62351 POINT(40.229107759868654 73.98877002407168) bank62351 +62352 POINT(41.576831197750444 74.49512543510953) bank62352 +62353 POINT(40.363008360547205 74.79427972179249) bank62353 +62354 POINT(41.598539125768625 73.12204534076683) bank62354 +62355 POINT(41.23665809972661 74.51058721761419) bank62355 +62356 POINT(40.546710099535034 74.29333816760287) bank62356 +62357 POINT(41.25221409811631 73.68859545128782) bank62357 +62358 POINT(41.483148613369664 73.31100566521341) bank62358 +62359 POINT(40.331159151053754 74.48678419110061) bank62359 +62360 POINT(41.075361671649276 74.89039451143748) bank62360 +62361 POINT(40.257267193209 74.63461776673135) bank62361 +62362 POINT(41.02694140097619 74.3603767236593) bank62362 +62363 POINT(41.57328074119034 73.41994719714687) bank62363 +62364 POINT(41.487921744747915 74.03940465070986) bank62364 +62365 POINT(40.67836652938698 74.15830406136132) bank62365 +62366 POINT(39.869989384550486 73.18750520303583) bank62366 +62367 POINT(39.81939814563344 74.53393919144155) bank62367 +62368 POINT(40.45100104004936 75.00321243904781) bank62368 +62369 POINT(41.681843976655315 73.80188159558598) bank62369 +62370 POINT(41.52306393040764 74.2620104423834) bank62370 +62371 POINT(41.37195600429231 73.65803121746298) bank62371 +62372 POINT(41.47517364628444 73.03076745860385) bank62372 +62373 POINT(40.863997432354886 73.98301976726401) bank62373 +62374 POINT(39.917991106190506 74.47388236131748) bank62374 +62375 POINT(40.01454145774724 73.50392692891238) bank62375 +62376 POINT(40.5394542914502 73.4918433315565) bank62376 +62377 POINT(41.596270207140435 73.8291209046879) bank62377 +62378 POINT(40.23121086490261 74.90001737970631) bank62378 +62379 POINT(41.566283715807714 73.55755893780037) bank62379 +62380 POINT(40.36407423920097 74.60674220102102) bank62380 +62381 POINT(41.02843018333232 74.48673770326914) bank62381 +62382 POINT(39.80241971743543 74.27060165974406) bank62382 +62383 POINT(41.043698737146045 73.1601779633211) bank62383 +62384 POINT(41.01604859341208 73.02031363868417) bank62384 +62385 POINT(40.060160297794305 73.63018682702038) bank62385 +62386 POINT(41.392875022711976 73.95965397984281) bank62386 +62387 POINT(41.509052108928785 73.63491474153324) bank62387 +62388 POINT(41.46432510937317 73.88388478803778) bank62388 +62389 POINT(40.491369640612426 73.80482973384903) bank62389 +62390 POINT(40.54590039370227 73.2538691563479) bank62390 +62391 POINT(40.88445953538124 73.95230836721205) bank62391 +62392 POINT(40.19426455906534 74.68176567273044) bank62392 +62393 POINT(40.94392878834966 73.2592591048899) bank62393 +62394 POINT(41.1007557132424 74.87092886486424) bank62394 +62395 POINT(40.796303229310574 73.89325521634561) bank62395 +62396 POINT(41.28697540549252 74.39195365957717) bank62396 +62397 POINT(40.708970012909575 73.23374272304277) bank62397 +62398 POINT(40.56529292385077 73.40057099816893) bank62398 +62399 POINT(40.42537491286938 73.31529991075978) bank62399 +62400 POINT(40.17311275576061 74.39777358522844) bank62400 +62401 POINT(40.579235074615866 73.05158865308672) bank62401 +62402 POINT(40.22577745892979 74.9083641028485) bank62402 +62403 POINT(39.86932912560755 73.96986402516858) bank62403 +62404 POINT(41.5548039055866 73.04098156193538) bank62404 +62405 POINT(41.17214918407608 74.23290633838732) bank62405 +62406 POINT(40.62689435076246 73.68502669209644) bank62406 +62407 POINT(40.41102271012956 74.90038955354177) bank62407 +62408 POINT(40.676626540910284 74.99045553849082) bank62408 +62409 POINT(40.60665345924855 73.80892494628355) bank62409 +62410 POINT(39.73413979385219 73.91582618620856) bank62410 +62411 POINT(40.43096283201271 74.90772788616442) bank62411 +62412 POINT(40.99640160505753 73.3462878385591) bank62412 +62413 POINT(40.74581645256706 74.09242050717083) bank62413 +62414 POINT(40.68419432717841 73.40722017618907) bank62414 +62415 POINT(40.238654815045074 74.03002284388127) bank62415 +62416 POINT(40.317290904515296 74.23186027046934) bank62416 +62417 POINT(41.239593441791776 73.13368663053059) bank62417 +62418 POINT(40.103540247612564 73.05682247256678) bank62418 +62419 POINT(41.646948001251154 73.91064988961323) bank62419 +62420 POINT(40.671914141305265 74.72247544353034) bank62420 +62421 POINT(40.80976603940935 73.87597437308168) bank62421 +62422 POINT(40.97592880287604 74.83162288135618) bank62422 +62423 POINT(39.87552387448636 73.05830129190993) bank62423 +62424 POINT(40.14870894185482 74.86020294766439) bank62424 +62425 POINT(41.05298225700885 73.131099179607) bank62425 +62426 POINT(41.4654976072549 73.82406770203966) bank62426 +62427 POINT(40.6528512641397 73.64077569774511) bank62427 +62428 POINT(41.4752338732642 74.53094590308898) bank62428 +62429 POINT(40.77176904583047 74.1986509362838) bank62429 +62430 POINT(40.21162976011742 73.99619539013875) bank62430 +62431 POINT(40.109337050200715 73.50218391665456) bank62431 +62432 POINT(40.94440974615684 74.41511697013935) bank62432 +62433 POINT(39.73960469269993 74.1708386176629) bank62433 +62434 POINT(40.03521348099808 73.07427730413053) bank62434 +62435 POINT(41.6164572823501 74.99903651192928) bank62435 +62436 POINT(41.631433922994894 73.95571472764675) bank62436 +62437 POINT(40.61615392467753 73.35912087853312) bank62437 +62438 POINT(41.048802011265884 73.2756114797579) bank62438 +62439 POINT(40.558821322957876 73.62846989269644) bank62439 +62440 POINT(40.223306460832234 73.77212141490679) bank62440 +62441 POINT(41.414537794391414 73.5722467420884) bank62441 +62442 POINT(41.16203428875032 74.4055204480352) bank62442 +62443 POINT(41.282524887504465 74.43947509776086) bank62443 +62444 POINT(40.66292066297395 74.07319331551678) bank62444 +62445 POINT(40.15212707268184 73.4126021415077) bank62445 +62446 POINT(40.84434991820391 73.08548756281772) bank62446 +62447 POINT(41.27657496081238 74.12695652103336) bank62447 +62448 POINT(40.756805919530706 74.35552140646013) bank62448 +62449 POINT(41.11143929232729 74.12976384104283) bank62449 +62450 POINT(40.661836716389914 74.40090204669279) bank62450 +62451 POINT(40.109367171111174 73.61609442654566) bank62451 +62452 POINT(40.097947245707736 74.66475524270741) bank62452 +62453 POINT(40.41277796269257 73.3910526116733) bank62453 +62454 POINT(40.63440504426273 73.96933738393506) bank62454 +62455 POINT(40.024095397138076 74.03559339151006) bank62455 +62456 POINT(41.67170256917983 74.20293507555776) bank62456 +62457 POINT(41.031233004957706 73.41395822304169) bank62457 +62458 POINT(40.96635754961201 73.80848640477005) bank62458 +62459 POINT(40.715845210616884 73.83288511830617) bank62459 +62460 POINT(40.734385501363725 74.7431499640325) bank62460 +62461 POINT(40.85954904408264 73.55844258658689) bank62461 +62462 POINT(40.48027525418372 74.97997417411179) bank62462 +62463 POINT(41.06454851305339 73.11860144402995) bank62463 +62464 POINT(41.02972134766262 73.944934904468) bank62464 +62465 POINT(39.817009296132596 73.33919929513864) bank62465 +62466 POINT(41.493937050026794 74.3314312214442) bank62466 +62467 POINT(41.662411225948325 74.0925236302006) bank62467 +62468 POINT(40.88092093196822 73.02410081072534) bank62468 +62469 POINT(40.62600954129066 74.86137670083531) bank62469 +62470 POINT(39.76296094777125 74.39554275599392) bank62470 +62471 POINT(40.2863274752049 73.06224520012643) bank62471 +62472 POINT(41.54098149000988 74.6542700784494) bank62472 +62473 POINT(40.50707989367589 73.32564951315736) bank62473 +62474 POINT(40.85674366123799 74.48217828726683) bank62474 +62475 POINT(40.73663400679855 73.87477627080894) bank62475 +62476 POINT(41.67652186152781 73.71402158764059) bank62476 +62477 POINT(40.53600080482167 74.04434414519942) bank62477 +62478 POINT(40.65396365531733 74.82486426785864) bank62478 +62479 POINT(40.90969914543432 74.87914884824401) bank62479 +62480 POINT(40.00508830294924 74.75699669947429) bank62480 +62481 POINT(41.05317660513715 73.43510877902033) bank62481 +62482 POINT(41.42838848189442 73.45392063991757) bank62482 +62483 POINT(40.46464255529884 74.83861029952969) bank62483 +62484 POINT(41.1415069537832 74.6620533673705) bank62484 +62485 POINT(39.96719452540746 74.62930255489103) bank62485 +62486 POINT(41.21409872677537 74.8175232054664) bank62486 +62487 POINT(41.54394250047015 73.2780984149732) bank62487 +62488 POINT(41.66288624304462 73.61277972614675) bank62488 +62489 POINT(39.95708456336055 74.91965682513033) bank62489 +62490 POINT(41.67730469894396 73.61940036532098) bank62490 +62491 POINT(41.37362671078789 73.81972909625321) bank62491 +62492 POINT(40.47466429396559 73.07833605370999) bank62492 +62493 POINT(40.466208348114016 73.95527156641228) bank62493 +62494 POINT(41.13606427660425 73.86778637038368) bank62494 +62495 POINT(40.89482661163374 74.72457579909079) bank62495 +62496 POINT(40.23320572449434 74.36985404913345) bank62496 +62497 POINT(40.340581551340634 73.83507516213638) bank62497 +62498 POINT(39.965244862469646 74.63628099280793) bank62498 +62499 POINT(40.28902308575672 74.29673035625841) bank62499 +62500 POINT(41.688818464595506 74.15446205738284) bank62500 +62501 POINT(41.13839597111825 73.28977586309689) bank62501 +62502 POINT(40.4651942189486 73.48348288094776) bank62502 +62503 POINT(41.352488555620475 74.13583693591495) bank62503 +62504 POINT(41.15387709565335 73.83006167212662) bank62504 +62505 POINT(39.893162869858784 73.52775500981046) bank62505 +62506 POINT(40.0869293096725 74.20280185307915) bank62506 +62507 POINT(41.71052691163139 73.2179930364181) bank62507 +62508 POINT(41.69376253964299 75.00525958193256) bank62508 +62509 POINT(39.8408380446424 74.48725099945261) bank62509 +62510 POINT(41.58523017995577 74.33822108820242) bank62510 +62511 POINT(41.35198344378855 73.92920416652008) bank62511 +62512 POINT(40.426725806843876 74.12009719179902) bank62512 +62513 POINT(40.30719646797672 73.1694731222564) bank62513 +62514 POINT(40.21025955476343 74.72986491007366) bank62514 +62515 POINT(41.240205113561494 73.5256237014652) bank62515 +62516 POINT(40.313295463240685 74.3924791793835) bank62516 +62517 POINT(41.32675932089552 74.35871639374949) bank62517 +62518 POINT(41.4590446954656 73.91681399369736) bank62518 +62519 POINT(41.56426432737626 73.87587124318905) bank62519 +62520 POINT(40.59098874203421 73.38819278098056) bank62520 +62521 POINT(41.06367150175307 73.85762196597926) bank62521 +62522 POINT(41.06669129634145 74.6653236518774) bank62522 +62523 POINT(41.50446746093154 74.14916356905479) bank62523 +62524 POINT(41.30641536885264 74.77276081557216) bank62524 +62525 POINT(39.78727925923388 74.29617228579302) bank62525 +62526 POINT(41.673957167790185 73.69640058866135) bank62526 +62527 POINT(40.13196846029417 74.4745534922641) bank62527 +62528 POINT(40.6275003894495 73.28833614527386) bank62528 +62529 POINT(41.33826648640594 73.17794982063592) bank62529 +62530 POINT(39.85161164426811 74.10827323743644) bank62530 +62531 POINT(41.6364768650991 73.1595919813573) bank62531 +62532 POINT(41.37003735706912 73.71237796961783) bank62532 +62533 POINT(41.02649394587355 73.0183575730205) bank62533 +62534 POINT(39.77480486677183 74.56988100635714) bank62534 +62535 POINT(40.189896057037856 73.28334631395185) bank62535 +62536 POINT(41.577836639318456 74.71029077624213) bank62536 +62537 POINT(41.15181798811577 73.14083955360742) bank62537 +62538 POINT(40.928789256466786 74.82694885557412) bank62538 +62539 POINT(41.39448030871161 73.01196418950319) bank62539 +62540 POINT(40.98522989441217 74.10975245172644) bank62540 +62541 POINT(40.75890043549302 74.94047106584355) bank62541 +62542 POINT(41.50060761896238 73.17577831189342) bank62542 +62543 POINT(39.90546135609622 73.66553867928526) bank62543 +62544 POINT(40.95921603391933 74.41880109939085) bank62544 +62545 POINT(41.57657081942801 74.8540700041034) bank62545 +62546 POINT(40.95619760740112 73.77347506026325) bank62546 +62547 POINT(40.44409198639212 74.7289763157947) bank62547 +62548 POINT(41.03633595856107 74.26833902409163) bank62548 +62549 POINT(41.53722738049472 73.10395626695748) bank62549 +62550 POINT(40.000406325670475 74.04146751889645) bank62550 +62551 POINT(40.482203858136266 74.00028848440944) bank62551 +62552 POINT(41.61575523860722 73.4531602862064) bank62552 +62553 POINT(41.27661120245982 74.1276739422044) bank62553 +62554 POINT(41.03055540661485 74.70286306402771) bank62554 +62555 POINT(41.00173332913764 74.62007892801634) bank62555 +62556 POINT(41.49785380764496 74.49694023974591) bank62556 +62557 POINT(40.955892240101996 73.43324765165494) bank62557 +62558 POINT(41.169849702698556 74.57640568714815) bank62558 +62559 POINT(41.018501292944734 73.72669449185119) bank62559 +62560 POINT(39.76234106893589 73.7994252922571) bank62560 +62561 POINT(41.030384464045255 73.25794657428521) bank62561 +62562 POINT(40.19952449151705 73.81433852864116) bank62562 +62563 POINT(40.820014285550734 73.85254396207799) bank62563 +62564 POINT(40.32088516847356 74.36976026833912) bank62564 +62565 POINT(40.1758895882185 73.64463827714535) bank62565 +62566 POINT(40.09443615992944 74.13665580779711) bank62566 +62567 POINT(40.035078477007275 74.74325944628022) bank62567 +62568 POINT(39.73011648781531 74.10467438060213) bank62568 +62569 POINT(41.53095250915957 74.01931113944428) bank62569 +62570 POINT(39.74365402536641 73.02748164254848) bank62570 +62571 POINT(41.0796560942378 74.59241374678811) bank62571 +62572 POINT(40.58953318002061 74.3682875731529) bank62572 +62573 POINT(41.248861257651974 73.42477854298055) bank62573 +62574 POINT(41.1055687525649 74.49056770843082) bank62574 +62575 POINT(40.242249398614405 73.99265063690333) bank62575 +62576 POINT(41.53301830237223 73.37589709869354) bank62576 +62577 POINT(39.883111569898304 74.97239364182948) bank62577 +62578 POINT(40.08456692918952 74.20933594426141) bank62578 +62579 POINT(40.20371785513029 73.02657481746228) bank62579 +62580 POINT(40.15599453264702 73.11761974835967) bank62580 +62581 POINT(40.12204205114881 73.51007185617773) bank62581 +62582 POINT(40.907621817463664 73.1742120398033) bank62582 +62583 POINT(39.92482305787046 74.24509227515813) bank62583 +62584 POINT(41.25518836090446 73.51976185694099) bank62584 +62585 POINT(41.040751827106895 73.7644927528029) bank62585 +62586 POINT(40.05819793347261 73.40841878820623) bank62586 +62587 POINT(40.70330110945915 73.54435927067654) bank62587 +62588 POINT(40.09795671244386 73.96734595645513) bank62588 +62589 POINT(41.28557397426003 74.46167738680614) bank62589 +62590 POINT(41.02172291507191 73.09213669135698) bank62590 +62591 POINT(40.180759453417934 74.91886157754266) bank62591 +62592 POINT(40.11789952356802 74.72136053024094) bank62592 +62593 POINT(41.66447831267598 73.63919101020676) bank62593 +62594 POINT(40.65300983533584 73.78914059393925) bank62594 +62595 POINT(39.740204799419104 73.93058817108187) bank62595 +62596 POINT(41.13392327078384 74.9377854224848) bank62596 +62597 POINT(41.29661428444252 73.93782578427376) bank62597 +62598 POINT(39.88072109708759 74.12903360225145) bank62598 +62599 POINT(40.22495712320686 74.23453501638285) bank62599 +62600 POINT(39.997872580054725 73.83258237192949) bank62600 +62601 POINT(41.46652937750142 73.49139539258543) bank62601 +62602 POINT(40.59354019113988 74.74818910561513) bank62602 +62603 POINT(40.14578213410613 73.604126814857) bank62603 +62604 POINT(41.34612816031937 73.034144277403) bank62604 +62605 POINT(41.147289141060625 74.61975872159532) bank62605 +62606 POINT(41.269316143810336 74.65441153244605) bank62606 +62607 POINT(41.51769222026829 74.7470932622845) bank62607 +62608 POINT(41.23038653307865 73.79727165138277) bank62608 +62609 POINT(40.60049285822091 73.56830254946098) bank62609 +62610 POINT(40.105713106428034 73.27704634739302) bank62610 +62611 POINT(41.5344870167684 74.09786907585098) bank62611 +62612 POINT(39.80280350666523 74.84672505717381) bank62612 +62613 POINT(40.48163180549135 73.4394377058778) bank62613 +62614 POINT(40.93370630753494 73.25109142438524) bank62614 +62615 POINT(41.57997102553736 73.58280949114291) bank62615 +62616 POINT(40.47175444666169 73.83768873261823) bank62616 +62617 POINT(40.034181697191634 73.66777912258138) bank62617 +62618 POINT(41.08505826514153 73.96415142573099) bank62618 +62619 POINT(40.464366989755995 74.92492853246681) bank62619 +62620 POINT(39.90523682456476 73.64322234817135) bank62620 +62621 POINT(40.68595394454829 73.59782958998971) bank62621 +62622 POINT(40.718203871382414 74.63292363080825) bank62622 +62623 POINT(41.474541661582585 73.38036477038378) bank62623 +62624 POINT(40.57215792319056 74.29692892874696) bank62624 +62625 POINT(40.87629650425288 73.66909170684791) bank62625 +62626 POINT(41.17401984886959 73.2209869286288) bank62626 +62627 POINT(41.32759989976119 74.81602744009386) bank62627 +62628 POINT(40.92909566920576 74.89718290481046) bank62628 +62629 POINT(40.11755333933016 73.09830854475956) bank62629 +62630 POINT(41.610722149845216 73.15616565447499) bank62630 +62631 POINT(40.37231069482714 73.79973986759731) bank62631 +62632 POINT(41.67911857885496 73.71107673299873) bank62632 +62633 POINT(41.4474185216255 74.06428189642106) bank62633 +62634 POINT(40.8238722501253 74.55572655239055) bank62634 +62635 POINT(41.38572270878413 73.11131234158483) bank62635 +62636 POINT(40.57096926609316 73.45321144658948) bank62636 +62637 POINT(40.15671941113323 74.21985871248492) bank62637 +62638 POINT(41.419728723439206 73.85882110431515) bank62638 +62639 POINT(41.4700683761155 73.47139423347869) bank62639 +62640 POINT(41.186162473466226 74.26024138545297) bank62640 +62641 POINT(41.29143285026717 73.71379653509065) bank62641 +62642 POINT(40.40083997241638 73.95041750086564) bank62642 +62643 POINT(39.87824730187078 74.71056351746456) bank62643 +62644 POINT(39.957009933998165 74.74871993263177) bank62644 +62645 POINT(39.892967092517495 73.89048386027305) bank62645 +62646 POINT(39.90996985013239 74.74064826847415) bank62646 +62647 POINT(40.28093734619134 74.00459874256927) bank62647 +62648 POINT(40.10888272390787 74.55202977353129) bank62648 +62649 POINT(40.56221331266389 74.11559452992279) bank62649 +62650 POINT(40.624619681236794 73.54318222535429) bank62650 +62651 POINT(40.03087033793752 73.55555129634958) bank62651 +62652 POINT(41.40203670173967 74.25838775731661) bank62652 +62653 POINT(41.58386711355383 74.10297715822713) bank62653 +62654 POINT(40.07673992856217 73.52938382858558) bank62654 +62655 POINT(40.545098645076735 73.76271654207261) bank62655 +62656 POINT(40.117157215929886 73.77353815183031) bank62656 +62657 POINT(41.4391598228527 73.96750309758538) bank62657 +62658 POINT(40.4838096379037 74.76948758542206) bank62658 +62659 POINT(41.08851134581291 73.99185862488551) bank62659 +62660 POINT(40.448257207984405 74.69554833248878) bank62660 +62661 POINT(40.50338493732797 73.02605361863408) bank62661 +62662 POINT(39.812352089150906 74.56350555988149) bank62662 +62663 POINT(40.24419756687945 73.67727988032277) bank62663 +62664 POINT(40.146277866410685 73.36357523741367) bank62664 +62665 POINT(41.60683189990016 73.37013667358137) bank62665 +62666 POINT(40.21264739655912 73.00849409723916) bank62666 +62667 POINT(40.14552023031215 74.92128348467175) bank62667 +62668 POINT(39.78134892383701 74.76379387168448) bank62668 +62669 POINT(39.82586215248767 73.99483796430829) bank62669 +62670 POINT(40.48988615244557 74.5548627583269) bank62670 +62671 POINT(40.65802892911396 74.72937537071573) bank62671 +62672 POINT(39.83655241153854 73.83379822049453) bank62672 +62673 POINT(39.85383934939816 74.83505951651533) bank62673 +62674 POINT(41.453096393716805 74.66054454005223) bank62674 +62675 POINT(40.078869445341695 73.94187150350847) bank62675 +62676 POINT(41.511420732941176 73.90818126451157) bank62676 +62677 POINT(41.62493264633959 74.1729976802042) bank62677 +62678 POINT(40.99395927581547 74.31453129424202) bank62678 +62679 POINT(40.232913546399885 73.71125205254697) bank62679 +62680 POINT(40.86648465991444 73.82505479669571) bank62680 +62681 POINT(39.95542660298564 73.30002208926261) bank62681 +62682 POINT(41.48238422617077 74.43022283254113) bank62682 +62683 POINT(40.265433681320644 74.08899651598315) bank62683 +62684 POINT(41.6491081797207 73.38055367367562) bank62684 +62685 POINT(40.35642452319087 74.51839829643492) bank62685 +62686 POINT(40.27506200647503 74.16659526156995) bank62686 +62687 POINT(41.53379561360386 74.68635187080827) bank62687 +62688 POINT(40.66872952233403 73.19649443173788) bank62688 +62689 POINT(40.66579829290878 74.31385922785131) bank62689 +62690 POINT(40.401661834369705 74.90102755779651) bank62690 +62691 POINT(40.43608601405069 73.82226856773332) bank62691 +62692 POINT(41.53817069807753 73.42072858265047) bank62692 +62693 POINT(40.15225616782278 73.24322738443004) bank62693 +62694 POINT(41.210426707081965 73.41579406019581) bank62694 +62695 POINT(39.72009177479322 74.76991946641643) bank62695 +62696 POINT(41.10389159956462 73.7886014048337) bank62696 +62697 POINT(40.45678483407562 74.14114096234894) bank62697 +62698 POINT(41.49672926435765 74.75143520016114) bank62698 +62699 POINT(41.32244621067632 73.70917981583271) bank62699 +62700 POINT(41.006323775304445 73.27868965245507) bank62700 +62701 POINT(41.44268410824287 75.00535893335254) bank62701 +62702 POINT(39.741630163523496 73.75386022673369) bank62702 +62703 POINT(40.070388081106856 73.91186111244478) bank62703 +62704 POINT(40.93572814563102 74.5351553076601) bank62704 +62705 POINT(41.59699621993377 73.99132649416076) bank62705 +62706 POINT(40.793154091525615 74.56202599634314) bank62706 +62707 POINT(41.34355017117958 74.37592663454765) bank62707 +62708 POINT(39.86254419604691 73.85594829353448) bank62708 +62709 POINT(41.599340950155096 74.59035967623325) bank62709 +62710 POINT(41.56945846510449 74.2432223794002) bank62710 +62711 POINT(41.06574245138949 73.88621909452483) bank62711 +62712 POINT(39.83227759659217 73.46130826507508) bank62712 +62713 POINT(40.11005474134588 74.3647542771929) bank62713 +62714 POINT(41.33151088530015 74.79957102153894) bank62714 +62715 POINT(41.43326621610344 73.30336152171273) bank62715 +62716 POINT(40.81657739023282 74.98463503257112) bank62716 +62717 POINT(40.420073986550555 73.35921392674699) bank62717 +62718 POINT(41.18090493197174 73.44553429478432) bank62718 +62719 POINT(40.45685947943445 73.27102067102494) bank62719 +62720 POINT(40.19405863419906 74.67667740820195) bank62720 +62721 POINT(39.795284627908515 74.07709431738289) bank62721 +62722 POINT(40.60433897809283 74.2043699959525) bank62722 +62723 POINT(40.14167540590312 74.86964839411512) bank62723 +62724 POINT(41.54283642556009 73.38772155352218) bank62724 +62725 POINT(40.611895012434594 73.54139420990057) bank62725 +62726 POINT(40.90009806827073 74.1770269921473) bank62726 +62727 POINT(41.01520002748124 74.34331178102397) bank62727 +62728 POINT(41.295782123684276 74.85444304675568) bank62728 +62729 POINT(39.85662768430366 73.55765938153343) bank62729 +62730 POINT(40.24419757314718 73.14288141071106) bank62730 +62731 POINT(41.16275006359583 74.66167549521266) bank62731 +62732 POINT(41.09931580901939 74.24532505914564) bank62732 +62733 POINT(40.24631831658768 73.46939006337321) bank62733 +62734 POINT(40.63808589560332 74.26111926657805) bank62734 +62735 POINT(40.75824274092789 74.68300624492886) bank62735 +62736 POINT(39.771527081290756 73.13354782087357) bank62736 +62737 POINT(40.31827502415147 74.13871608459102) bank62737 +62738 POINT(40.521532803511874 73.35075645814898) bank62738 +62739 POINT(41.10966063421127 73.20910657287811) bank62739 +62740 POINT(41.054661631238304 74.03450465774674) bank62740 +62741 POINT(41.37010460428481 74.43188543933418) bank62741 +62742 POINT(41.24722797361553 73.52334234191432) bank62742 +62743 POINT(40.51425892879099 73.5413464643351) bank62743 +62744 POINT(41.10662005703949 74.1691847889995) bank62744 +62745 POINT(40.97132776102893 74.19208688823214) bank62745 +62746 POINT(40.8071993307754 74.90935559441462) bank62746 +62747 POINT(41.56426929656682 74.56557018920725) bank62747 +62748 POINT(41.19122212860898 73.64246635731652) bank62748 +62749 POINT(39.7357663466752 73.53994194412611) bank62749 +62750 POINT(41.359215632269624 74.38262231774429) bank62750 +62751 POINT(39.983745791076345 74.07919561800249) bank62751 +62752 POINT(40.49797618618825 73.90561902875777) bank62752 +62753 POINT(41.33439263100338 73.94127219438913) bank62753 +62754 POINT(40.42982183697013 73.23035612162272) bank62754 +62755 POINT(39.81378377256339 73.20462449576064) bank62755 +62756 POINT(40.91531230518328 74.55080653166934) bank62756 +62757 POINT(41.093005871079 74.45433538142733) bank62757 +62758 POINT(40.41664756789171 73.05116791958905) bank62758 +62759 POINT(41.2293954628729 73.43678421544207) bank62759 +62760 POINT(41.36620369244519 74.17741356311427) bank62760 +62761 POINT(40.97475469454832 74.33950241960545) bank62761 +62762 POINT(40.374716924164346 73.18138248608442) bank62762 +62763 POINT(39.969274996601875 74.59575266256705) bank62763 +62764 POINT(40.453719013071755 73.83976024786081) bank62764 +62765 POINT(40.565291871456544 73.46712145038623) bank62765 +62766 POINT(41.14116937361726 73.9652421341646) bank62766 +62767 POINT(39.986053977742785 73.68446779165107) bank62767 +62768 POINT(39.78716799501137 73.49507693360198) bank62768 +62769 POINT(40.89881410532824 74.69031782391079) bank62769 +62770 POINT(40.401972353949965 73.34368596927249) bank62770 +62771 POINT(41.30474720431495 73.5793777308608) bank62771 +62772 POINT(41.62484425923967 74.33716137321456) bank62772 +62773 POINT(40.46306233422378 73.10806027153562) bank62773 +62774 POINT(41.50393026889223 74.26135798799085) bank62774 +62775 POINT(41.12082345956263 74.42936463717815) bank62775 +62776 POINT(40.049665383463086 74.01873486863153) bank62776 +62777 POINT(40.90639695229093 74.13210982137252) bank62777 +62778 POINT(41.62983329572833 73.68155989566549) bank62778 +62779 POINT(41.552291031694544 73.02585539573968) bank62779 +62780 POINT(40.04523957938857 74.83851329853613) bank62780 +62781 POINT(40.16143932336552 74.26920454875939) bank62781 +62782 POINT(40.61170157849052 74.46666196955456) bank62782 +62783 POINT(40.106656091718115 73.4502743031258) bank62783 +62784 POINT(41.56803598826358 74.22688180622855) bank62784 +62785 POINT(39.87548121755128 73.90473216710124) bank62785 +62786 POINT(40.95228424321989 73.3777395159717) bank62786 +62787 POINT(41.285379358261146 74.58479877453082) bank62787 +62788 POINT(41.42360112480128 73.30539862137462) bank62788 +62789 POINT(40.00701017567667 74.26998271397255) bank62789 +62790 POINT(41.39883342733574 74.23805426452758) bank62790 +62791 POINT(40.70936158637772 74.01297102920526) bank62791 +62792 POINT(40.15483870356849 74.8444536452941) bank62792 +62793 POINT(41.43500959455293 73.61620764192058) bank62793 +62794 POINT(39.97419151721228 74.99592296743681) bank62794 +62795 POINT(39.91796833540283 73.39145008826367) bank62795 +62796 POINT(40.786906755975515 74.16174264699704) bank62796 +62797 POINT(39.88077101818635 73.76767164333536) bank62797 +62798 POINT(41.703493927817625 73.38038088159522) bank62798 +62799 POINT(40.3583733830677 74.49885908403253) bank62799 +62800 POINT(41.237444828866295 73.71176138916036) bank62800 +62801 POINT(41.51601482940587 73.84136021931124) bank62801 +62802 POINT(40.01621987803915 73.29120697306921) bank62802 +62803 POINT(39.93493238798711 74.15156395456286) bank62803 +62804 POINT(39.84631198606479 73.17977675672853) bank62804 +62805 POINT(39.94450563459005 73.57710287844213) bank62805 +62806 POINT(40.96582094699755 74.29567666492656) bank62806 +62807 POINT(40.5308810740597 73.73720772387182) bank62807 +62808 POINT(40.79047358349545 74.87678420309815) bank62808 +62809 POINT(40.43382051130546 74.31456393653392) bank62809 +62810 POINT(41.69811119370081 73.28046275898717) bank62810 +62811 POINT(40.14532148767886 74.07307046899966) bank62811 +62812 POINT(41.16230927994931 74.27890706182984) bank62812 +62813 POINT(40.01847784529295 73.64474934973067) bank62813 +62814 POINT(39.85892204816471 73.55478211394126) bank62814 +62815 POINT(41.012587554538236 74.63908943780358) bank62815 +62816 POINT(40.67503000670821 74.19557544209084) bank62816 +62817 POINT(40.65743325656627 73.90554565368193) bank62817 +62818 POINT(41.29777609131538 74.8283996833042) bank62818 +62819 POINT(40.94975813705645 73.85616254937479) bank62819 +62820 POINT(40.427127508063926 74.74961474781044) bank62820 +62821 POINT(40.13369316709517 73.23172984178484) bank62821 +62822 POINT(40.168376414228526 73.99641465428833) bank62822 +62823 POINT(40.65189677599932 73.94333654497372) bank62823 +62824 POINT(39.79158260579902 74.54675248930782) bank62824 +62825 POINT(40.588965926803354 73.319978412109) bank62825 +62826 POINT(40.62251952321305 73.10084591007401) bank62826 +62827 POINT(39.99186825948344 74.05977947271698) bank62827 +62828 POINT(41.50989592590918 74.96300460942055) bank62828 +62829 POINT(41.05487628400409 74.72997558662006) bank62829 +62830 POINT(40.306244210939695 74.01506976092861) bank62830 +62831 POINT(40.91009806648979 74.46955504870665) bank62831 +62832 POINT(41.39878017351052 73.94301435449722) bank62832 +62833 POINT(40.63750620279118 73.75116979722816) bank62833 +62834 POINT(41.620948031798655 74.59172837848189) bank62834 +62835 POINT(40.666531936697105 74.12725101841033) bank62835 +62836 POINT(41.600828375797704 74.21167023655131) bank62836 +62837 POINT(40.79155835048052 73.63152816321367) bank62837 +62838 POINT(41.573189246247 73.78060536770296) bank62838 +62839 POINT(39.84036070411606 74.94502662739033) bank62839 +62840 POINT(41.14593121166345 74.2003683524346) bank62840 +62841 POINT(41.619860457092486 73.3111658926105) bank62841 +62842 POINT(41.156139893475235 73.79386090002711) bank62842 +62843 POINT(39.74301113548517 74.18788373616196) bank62843 +62844 POINT(41.32896814803925 73.6737108117099) bank62844 +62845 POINT(39.73662545916109 73.181393722173) bank62845 +62846 POINT(41.21957478344817 73.29776432197147) bank62846 +62847 POINT(39.78060547628339 73.19767317655156) bank62847 +62848 POINT(41.62981175373887 74.53920775964082) bank62848 +62849 POINT(41.16953802024658 73.7788800315844) bank62849 +62850 POINT(39.73929115574657 74.76477425549344) bank62850 +62851 POINT(41.657191157500144 73.41485691808587) bank62851 +62852 POINT(40.8303369917958 74.36699701920371) bank62852 +62853 POINT(40.342553456421534 74.48819855681515) bank62853 +62854 POINT(40.73266933095546 73.8951564325821) bank62854 +62855 POINT(39.962821521193 74.19535601651943) bank62855 +62856 POINT(39.740417817345204 73.25292247433038) bank62856 +62857 POINT(41.194481463374636 74.10523629709802) bank62857 +62858 POINT(41.51552968358843 73.30435488386345) bank62858 +62859 POINT(40.30983984447206 74.29365520487815) bank62859 +62860 POINT(40.93660456700067 74.83029099356632) bank62860 +62861 POINT(40.48506828947962 73.40694846577337) bank62861 +62862 POINT(41.429850044959416 73.81795680965095) bank62862 +62863 POINT(39.7483792000646 73.89974715073453) bank62863 +62864 POINT(40.03768881023294 73.33975988540955) bank62864 +62865 POINT(40.120302027195 74.27998135027845) bank62865 +62866 POINT(41.68330921438828 73.98466318767547) bank62866 +62867 POINT(41.4947914220165 74.77013119277075) bank62867 +62868 POINT(40.42598583555898 73.94978564719776) bank62868 +62869 POINT(41.37733973300082 74.8535136429603) bank62869 +62870 POINT(40.34123389861133 74.75130385486158) bank62870 +62871 POINT(40.58868018324271 73.50718712211548) bank62871 +62872 POINT(40.6245556335355 74.0896560968129) bank62872 +62873 POINT(41.490979887615985 73.14135179818904) bank62873 +62874 POINT(41.21870701678112 74.8651287674356) bank62874 +62875 POINT(41.04496317000181 74.44878652877146) bank62875 +62876 POINT(40.494144296699346 73.42438136301867) bank62876 +62877 POINT(40.743542864028946 73.94239912242945) bank62877 +62878 POINT(40.76371255223912 73.10300420993478) bank62878 +62879 POINT(40.075377397934226 73.36162711782106) bank62879 +62880 POINT(41.34752058454035 73.05331534994471) bank62880 +62881 POINT(39.7635606962392 73.74739181926142) bank62881 +62882 POINT(40.452726048719825 73.9789045548616) bank62882 +62883 POINT(40.368024781197754 73.7230146818635) bank62883 +62884 POINT(39.85434951968915 74.34336016281108) bank62884 +62885 POINT(40.27259268936158 73.15254421773429) bank62885 +62886 POINT(41.34661660400608 74.47451660535044) bank62886 +62887 POINT(40.5193848692248 74.6473782545758) bank62887 +62888 POINT(39.84447052689352 73.74360228356294) bank62888 +62889 POINT(41.06799834266262 74.01344153423062) bank62889 +62890 POINT(40.74334432596874 73.99788955383208) bank62890 +62891 POINT(40.39448542807838 74.14727102668856) bank62891 +62892 POINT(40.35107866840226 74.09487989285171) bank62892 +62893 POINT(39.8672914024861 73.77316869268171) bank62893 +62894 POINT(39.88713371172856 74.20043530446398) bank62894 +62895 POINT(41.2860502041854 73.77005591311172) bank62895 +62896 POINT(40.34952406443656 73.33229281050687) bank62896 +62897 POINT(41.268153464551034 73.44361719766763) bank62897 +62898 POINT(40.48246660506939 74.78183890912544) bank62898 +62899 POINT(40.38478466728935 73.28145664988669) bank62899 +62900 POINT(40.48970711917026 74.46498950061132) bank62900 +62901 POINT(40.96435554389636 73.26863756528861) bank62901 +62902 POINT(41.58965370164162 74.50337813972004) bank62902 +62903 POINT(41.1660041223128 73.31533478695201) bank62903 +62904 POINT(41.332439761439765 74.87981745161719) bank62904 +62905 POINT(41.70175194140432 74.26278108376296) bank62905 +62906 POINT(40.21731593271937 74.25105933097936) bank62906 +62907 POINT(40.80995610376859 74.24817145845282) bank62907 +62908 POINT(41.38143117447691 73.31610341001277) bank62908 +62909 POINT(40.28406471086446 73.1920024038338) bank62909 +62910 POINT(40.69617173724903 73.71960857228625) bank62910 +62911 POINT(41.43667579425667 74.83215683348683) bank62911 +62912 POINT(40.18897950080655 74.97089725156707) bank62912 +62913 POINT(40.0510933339638 73.86188507517191) bank62913 +62914 POINT(40.931468092637466 73.5429109598321) bank62914 +62915 POINT(41.61313632355805 74.73284169453784) bank62915 +62916 POINT(40.328880766545005 73.41241208536022) bank62916 +62917 POINT(40.10471592049156 74.6602186415048) bank62917 +62918 POINT(40.37082052412354 73.0230582746197) bank62918 +62919 POINT(41.594699531962085 73.20711994377427) bank62919 +62920 POINT(40.49046127679564 73.5614299739359) bank62920 +62921 POINT(40.273441147692196 73.92000776514801) bank62921 +62922 POINT(40.632296461662 73.05722041890905) bank62922 +62923 POINT(40.281486727830135 73.2903566792009) bank62923 +62924 POINT(40.72828147004029 74.056008948403) bank62924 +62925 POINT(39.790111107187634 73.35479588688455) bank62925 +62926 POINT(40.133973001485906 74.6939753202671) bank62926 +62927 POINT(40.91654396288268 73.54919454307961) bank62927 +62928 POINT(40.1626550274628 73.53439749540543) bank62928 +62929 POINT(40.00308125522635 74.41561936840411) bank62929 +62930 POINT(40.98963705001074 74.39005835124692) bank62930 +62931 POINT(40.44623143781413 74.70246667258435) bank62931 +62932 POINT(41.0008588037121 74.64054997160983) bank62932 +62933 POINT(39.8801032734657 73.76957535614632) bank62933 +62934 POINT(39.75449429958817 74.40841263860754) bank62934 +62935 POINT(40.40682713744496 74.30362147556262) bank62935 +62936 POINT(41.111704589745436 73.34769372150845) bank62936 +62937 POINT(41.342218609508464 74.79405419374379) bank62937 +62938 POINT(40.99500697895233 74.60537735394374) bank62938 +62939 POINT(41.42768003516629 73.19766020910286) bank62939 +62940 POINT(40.206657851410796 74.00071123132057) bank62940 +62941 POINT(39.76368595665283 74.73758885525936) bank62941 +62942 POINT(41.036618008408496 73.5135295002258) bank62942 +62943 POINT(40.834770277414755 74.07926571625963) bank62943 +62944 POINT(41.38976575669584 73.14028686039971) bank62944 +62945 POINT(40.74452816491641 73.69736557964227) bank62945 +62946 POINT(39.99363851597912 73.82642965486721) bank62946 +62947 POINT(40.884306365940716 74.73018123995261) bank62947 +62948 POINT(40.20932377666757 74.4052851854768) bank62948 +62949 POINT(40.777613455808776 74.65747515633912) bank62949 +62950 POINT(41.398963015481414 74.57935224988475) bank62950 +62951 POINT(40.93704002436646 74.70215642910924) bank62951 +62952 POINT(40.218588772414954 74.70144941067645) bank62952 +62953 POINT(39.82177466134636 74.62332410354345) bank62953 +62954 POINT(41.07287498991885 73.1787896979131) bank62954 +62955 POINT(40.78376895590561 74.4606983891285) bank62955 +62956 POINT(40.89426515099954 74.74434113427729) bank62956 +62957 POINT(41.68547443099261 73.60721186427597) bank62957 +62958 POINT(40.304718126648154 73.53476525100446) bank62958 +62959 POINT(40.71813742969881 73.88987913865188) bank62959 +62960 POINT(40.4230543290508 73.34276908388017) bank62960 +62961 POINT(39.92009469734267 73.45539892881233) bank62961 +62962 POINT(40.821120345213295 73.70235507507425) bank62962 +62963 POINT(39.87324094836942 74.64512205853686) bank62963 +62964 POINT(40.4468441452108 73.23032565176658) bank62964 +62965 POINT(41.08951616195002 74.39101891604814) bank62965 +62966 POINT(40.306100105254274 73.19080677892451) bank62966 +62967 POINT(39.94747336561589 73.63256600221028) bank62967 +62968 POINT(41.19734225066739 73.76217178929035) bank62968 +62969 POINT(41.54425871456513 73.69780162868435) bank62969 +62970 POINT(40.72243343120889 73.23351624819118) bank62970 +62971 POINT(41.22050461495163 74.18860395581163) bank62971 +62972 POINT(40.62646485212238 73.4803462961691) bank62972 +62973 POINT(39.93473524633003 74.33220748325485) bank62973 +62974 POINT(39.78621575978658 74.599106923865) bank62974 +62975 POINT(40.636324420986156 73.9853986825348) bank62975 +62976 POINT(40.43100648408026 74.53709019384927) bank62976 +62977 POINT(41.07550529681152 74.82271529330075) bank62977 +62978 POINT(39.80260711202328 74.4059377609894) bank62978 +62979 POINT(41.633119430069776 73.95192659279222) bank62979 +62980 POINT(40.73058877415999 74.25048171830369) bank62980 +62981 POINT(40.63244657519313 73.49483136316458) bank62981 +62982 POINT(40.80279189045342 73.97808473420586) bank62982 +62983 POINT(40.110612289956904 73.67394150647065) bank62983 +62984 POINT(41.10745433102765 73.17379920606908) bank62984 +62985 POINT(40.05365397944085 73.45639765243637) bank62985 +62986 POINT(39.79085679982263 74.26690772431662) bank62986 +62987 POINT(39.89563230016631 74.0632163017258) bank62987 +62988 POINT(40.61957400588114 73.53732915627869) bank62988 +62989 POINT(40.47055403227042 73.83222643630312) bank62989 +62990 POINT(41.001690201459304 74.71081502866267) bank62990 +62991 POINT(39.74873011051473 74.37861688308331) bank62991 +62992 POINT(40.94977658769085 73.5450451050744) bank62992 +62993 POINT(39.876313731099046 74.71359324241564) bank62993 +62994 POINT(39.754610135897344 74.2250511891762) bank62994 +62995 POINT(41.186859284254865 74.88450187801983) bank62995 +62996 POINT(40.81209859720351 73.29202264047338) bank62996 +62997 POINT(40.664366617606035 74.56642897566796) bank62997 +62998 POINT(40.77317995700434 73.80274572991287) bank62998 +62999 POINT(40.25979937471531 73.28221279514233) bank62999 +63000 POINT(41.494195981784664 74.05902129638915) bank63000 +63001 POINT(40.68129591376639 73.37616999317643) bank63001 +63002 POINT(41.27582294298086 74.96158437659514) bank63002 +63003 POINT(40.47659041823336 73.44227076701569) bank63003 +63004 POINT(41.61608933178773 74.60890010956105) bank63004 +63005 POINT(41.42296399529536 73.7439101399544) bank63005 +63006 POINT(40.46410025072987 73.98321612418589) bank63006 +63007 POINT(40.123432693080446 73.1153473580624) bank63007 +63008 POINT(40.54533467901926 74.01280399884855) bank63008 +63009 POINT(40.25003964815919 74.08033815554136) bank63009 +63010 POINT(40.14428768797696 74.15493017136079) bank63010 +63011 POINT(41.132479241922226 73.09227988304093) bank63011 +63012 POINT(39.81190890456968 73.08688475510411) bank63012 +63013 POINT(40.604206138524766 74.24634755762335) bank63013 +63014 POINT(39.86708125046878 73.95382310833739) bank63014 +63015 POINT(39.78461008415414 73.63373627127051) bank63015 +63016 POINT(40.29218077946712 73.32979604660584) bank63016 +63017 POINT(40.75209566377345 73.21096214082144) bank63017 +63018 POINT(40.321903846891686 73.96470987316138) bank63018 +63019 POINT(40.79462021217026 73.4285424573698) bank63019 +63020 POINT(41.514102157236806 73.86203126486942) bank63020 +63021 POINT(40.66296053005754 73.24617882743374) bank63021 +63022 POINT(41.62977115283658 74.6746436110707) bank63022 +63023 POINT(41.276599145228275 73.48900612315737) bank63023 +63024 POINT(40.27558154082251 73.53270658863978) bank63024 +63025 POINT(41.14920001946317 74.36995153949223) bank63025 +63026 POINT(40.054406599365656 74.47800027690187) bank63026 +63027 POINT(40.989281584847454 73.08374667344847) bank63027 +63028 POINT(40.05058489044791 73.58154416300985) bank63028 +63029 POINT(40.04848036172081 74.181770136327) bank63029 +63030 POINT(40.38364994909935 74.13604996987807) bank63030 +63031 POINT(39.8703429815324 74.4358500865358) bank63031 +63032 POINT(40.08189637464126 73.89218721470063) bank63032 +63033 POINT(40.11640694989045 73.89052161751074) bank63033 +63034 POINT(41.52735137727929 74.3256022537678) bank63034 +63035 POINT(40.21883885998976 73.67043545216966) bank63035 +63036 POINT(41.13051231833718 74.69090251927513) bank63036 +63037 POINT(41.17525021528108 73.9849871531727) bank63037 +63038 POINT(40.12227906614118 73.21472805725404) bank63038 +63039 POINT(41.09469566078347 73.37661828108901) bank63039 +63040 POINT(40.82861286521152 73.42339855066079) bank63040 +63041 POINT(41.12906664608858 73.88400249824366) bank63041 +63042 POINT(40.61639872122543 73.79016658542398) bank63042 +63043 POINT(39.91251835612454 73.44753644906744) bank63043 +63044 POINT(39.95946289334216 74.62558960340728) bank63044 +63045 POINT(40.81504354712601 74.1369247247265) bank63045 +63046 POINT(41.251315633794924 74.88393930205683) bank63046 +63047 POINT(40.230884464000226 73.4365936110176) bank63047 +63048 POINT(41.35267821167821 73.7738954459633) bank63048 +63049 POINT(41.02324100923275 74.17951465504102) bank63049 +63050 POINT(40.53051611284299 73.4406687051953) bank63050 +63051 POINT(40.812445446893705 73.88074060050255) bank63051 +63052 POINT(41.57679684228847 73.69748872579135) bank63052 +63053 POINT(40.26182215587119 74.79806920681365) bank63053 +63054 POINT(40.006201916473785 73.8083895863211) bank63054 +63055 POINT(40.85312631660062 74.34723768062172) bank63055 +63056 POINT(39.94367150815625 74.77024941382736) bank63056 +63057 POINT(40.98606709139218 74.89010445651255) bank63057 +63058 POINT(41.14076975710465 73.03895548985699) bank63058 +63059 POINT(40.45081450505124 73.58116875229203) bank63059 +63060 POINT(41.04218569839613 73.89642331755707) bank63060 +63061 POINT(40.59986486229211 74.37515160222372) bank63061 +63062 POINT(40.632880015134845 74.30796450376893) bank63062 +63063 POINT(40.299339441216276 74.37218336914401) bank63063 +63064 POINT(41.48346668899659 74.44074206371783) bank63064 +63065 POINT(41.68263384586234 74.11154813526221) bank63065 +63066 POINT(40.131330616550294 73.57522912432161) bank63066 +63067 POINT(40.47739252676216 73.49614161340105) bank63067 +63068 POINT(39.717446038135215 73.26463262808143) bank63068 +63069 POINT(41.699413621390406 74.12747343206242) bank63069 +63070 POINT(39.7942671483965 73.49948704100984) bank63070 +63071 POINT(41.09224867468135 74.45648271085312) bank63071 +63072 POINT(39.863294356372066 73.70259971757335) bank63072 +63073 POINT(41.37936562654925 73.96230399583477) bank63073 +63074 POINT(41.2533041644531 74.4319965750861) bank63074 +63075 POINT(40.836977219732496 74.57589273370658) bank63075 +63076 POINT(39.82862527256762 73.27802771526254) bank63076 +63077 POINT(39.90171599316045 73.92773923556646) bank63077 +63078 POINT(40.288001421045095 73.61520431005498) bank63078 +63079 POINT(41.02460070399803 73.61616723462045) bank63079 +63080 POINT(41.24563981894251 74.38050908618182) bank63080 +63081 POINT(40.023092990972 74.68240191945397) bank63081 +63082 POINT(39.794105960177944 74.64732027965793) bank63082 +63083 POINT(41.117945329871525 73.18408321027279) bank63083 +63084 POINT(39.92305351633768 73.47139779353756) bank63084 +63085 POINT(41.225472524992 73.45492285851243) bank63085 +63086 POINT(41.30070551513943 73.59080897313521) bank63086 +63087 POINT(40.062172850230766 73.87705541374042) bank63087 +63088 POINT(39.76611502643072 73.87842590052242) bank63088 +63089 POINT(41.67055205757648 73.61009961859304) bank63089 +63090 POINT(41.49973785417742 73.94412782306794) bank63090 +63091 POINT(39.805648038340806 74.07351792692937) bank63091 +63092 POINT(40.768817488805645 74.78005561468743) bank63092 +63093 POINT(41.083102553470454 73.0117403831747) bank63093 +63094 POINT(40.67525967792802 74.85834543588071) bank63094 +63095 POINT(40.31772442841777 74.71162254789397) bank63095 +63096 POINT(41.42101573401611 74.2421394698798) bank63096 +63097 POINT(41.286769311349936 74.86136449128055) bank63097 +63098 POINT(40.96710464459056 73.36443268502872) bank63098 +63099 POINT(39.75731451388115 74.06071066268288) bank63099 +63100 POINT(39.78703105346648 73.15231614321131) bank63100 +63101 POINT(39.806409707892655 74.05270705037486) bank63101 +63102 POINT(40.13283350936418 74.3977894376511) bank63102 +63103 POINT(40.706408921996406 73.02981325573617) bank63103 +63104 POINT(41.08875808213465 73.78494742739557) bank63104 +63105 POINT(40.13122914022393 73.12717085163752) bank63105 +63106 POINT(40.980519884137045 73.13913183710237) bank63106 +63107 POINT(39.924112867339 73.54189702877991) bank63107 +63108 POINT(40.13928140967882 73.41197752073654) bank63108 +63109 POINT(40.092052468498025 73.23294263304314) bank63109 +63110 POINT(40.21061933967677 73.87425396881687) bank63110 +63111 POINT(40.58805286089145 73.32573512884487) bank63111 +63112 POINT(41.31676442685918 73.37922820566257) bank63112 +63113 POINT(40.232717124922544 74.92353000676113) bank63113 +63114 POINT(40.802043583393925 73.90867439434213) bank63114 +63115 POINT(39.79892343436967 74.97527948419962) bank63115 +63116 POINT(40.37920965706691 74.38144186032295) bank63116 +63117 POINT(39.80851899236946 74.26721623782235) bank63117 +63118 POINT(40.832576058617846 74.07614004942127) bank63118 +63119 POINT(40.983640003747134 74.64986745507647) bank63119 +63120 POINT(41.58136473890845 73.04131686571503) bank63120 +63121 POINT(41.29699463111987 74.84916670927983) bank63121 +63122 POINT(41.29472717255604 74.28797080632555) bank63122 +63123 POINT(41.62656646051999 73.65273159297111) bank63123 +63124 POINT(40.49620079665234 74.02399994438622) bank63124 +63125 POINT(41.628515858197346 74.97132231469817) bank63125 +63126 POINT(40.69794257614231 73.9420232121668) bank63126 +63127 POINT(40.226136774652026 74.75011136008054) bank63127 +63128 POINT(40.816976159902865 73.53804039836749) bank63128 +63129 POINT(40.656966030907526 73.97816723875906) bank63129 +63130 POINT(41.31947776447504 73.32402086893633) bank63130 +63131 POINT(40.337217511405534 74.87681016924776) bank63131 +63132 POINT(40.03405585759373 73.24824221091693) bank63132 +63133 POINT(41.65819179070943 73.93403123499603) bank63133 +63134 POINT(40.74100421145734 74.6865174259148) bank63134 +63135 POINT(41.685666306997256 73.6813105757123) bank63135 +63136 POINT(40.92963035816077 74.45879018390174) bank63136 +63137 POINT(41.04020390022549 74.22450571103644) bank63137 +63138 POINT(41.59298245682863 73.91525275943786) bank63138 +63139 POINT(40.41376592472039 74.7523375147461) bank63139 +63140 POINT(41.61269037055434 73.2172130692619) bank63140 +63141 POINT(41.6270428445186 73.82601372106016) bank63141 +63142 POINT(40.24498586791971 74.51577787220239) bank63142 +63143 POINT(40.786006099063904 74.82271984045954) bank63143 +63144 POINT(41.08238107831251 74.15636876381907) bank63144 +63145 POINT(40.79950732930791 74.72513659063492) bank63145 +63146 POINT(40.890671633444605 74.48488817820468) bank63146 +63147 POINT(40.29416937966605 74.85942691940876) bank63147 +63148 POINT(40.4958201166802 73.6314383375877) bank63148 +63149 POINT(40.328543886649555 73.9103564021368) bank63149 +63150 POINT(40.32814993542558 73.68581789038299) bank63150 +63151 POINT(40.202954646409665 74.69839386299476) bank63151 +63152 POINT(40.44487397705009 73.44963933517269) bank63152 +63153 POINT(40.41795801569211 74.21416671493493) bank63153 +63154 POINT(41.355150118249064 74.98047216323917) bank63154 +63155 POINT(41.25678761017313 74.3169974846828) bank63155 +63156 POINT(40.995546775914356 73.75921851449831) bank63156 +63157 POINT(40.99950196975868 74.87205529196737) bank63157 +63158 POINT(41.44785018821537 73.08299707962831) bank63158 +63159 POINT(40.262062873470825 73.64330139586419) bank63159 +63160 POINT(41.33558249138236 74.58471711188369) bank63160 +63161 POINT(41.54088615083381 74.60240525036703) bank63161 +63162 POINT(41.348081296545956 73.29227020560249) bank63162 +63163 POINT(41.42817755901142 73.91398084022934) bank63163 +63164 POINT(41.65078118450395 73.6095380599402) bank63164 +63165 POINT(39.93104654360186 73.64763317022262) bank63165 +63166 POINT(41.49543034545026 73.402146944579) bank63166 +63167 POINT(41.46289828845935 74.23322261221512) bank63167 +63168 POINT(40.88633124771627 74.72823059742156) bank63168 +63169 POINT(40.4211113167318 73.91034403070596) bank63169 +63170 POINT(40.86812480440297 74.19128339772409) bank63170 +63171 POINT(40.869894285776034 73.07517990451467) bank63171 +63172 POINT(40.375951148920755 74.19370207067472) bank63172 +63173 POINT(41.00738971731881 73.13372066480743) bank63173 +63174 POINT(40.48538764805681 73.73231684243903) bank63174 +63175 POINT(40.365562365857066 73.33644642800691) bank63175 +63176 POINT(40.353146191886985 73.5165189526561) bank63176 +63177 POINT(41.15933725543526 73.41872564385501) bank63177 +63178 POINT(39.93192770757282 73.7765988644452) bank63178 +63179 POINT(40.944154864851754 73.60488635456552) bank63179 +63180 POINT(41.44364824314601 74.72030856833945) bank63180 +63181 POINT(41.66433167969842 73.94223729518744) bank63181 +63182 POINT(41.251446920605275 74.39679172046772) bank63182 +63183 POINT(39.88040714889725 74.95586659391292) bank63183 +63184 POINT(39.9860571378518 74.65160149240616) bank63184 +63185 POINT(40.01037255616171 74.67796314890646) bank63185 +63186 POINT(41.26752294041477 74.1220381623172) bank63186 +63187 POINT(39.886059859264044 74.20623200625084) bank63187 +63188 POINT(39.72743579147127 74.02917856989463) bank63188 +63189 POINT(41.04034702572391 73.93121236266434) bank63189 +63190 POINT(41.53612114808673 74.88499525632373) bank63190 +63191 POINT(40.176494033089796 74.66305755432474) bank63191 +63192 POINT(40.61425623021034 73.6527181798313) bank63192 +63193 POINT(41.40403702049624 74.13549873531746) bank63193 +63194 POINT(41.0126411318011 73.5547285360312) bank63194 +63195 POINT(39.75356160154288 73.13166943511767) bank63195 +63196 POINT(41.67079959737861 73.50048327123451) bank63196 +63197 POINT(39.83774171351368 74.37727087620053) bank63197 +63198 POINT(40.77309755726593 74.06082577440725) bank63198 +63199 POINT(41.44767952657426 74.96809723529256) bank63199 +63200 POINT(40.15308263054437 74.51259238525655) bank63200 +63201 POINT(40.759148069781375 74.69385248675628) bank63201 +63202 POINT(41.42359282435122 74.48128376614724) bank63202 +63203 POINT(41.282977075067066 74.71908703952812) bank63203 +63204 POINT(40.31129857168153 73.47901915991339) bank63204 +63205 POINT(41.249194307696314 74.3814545389733) bank63205 +63206 POINT(41.42867414620088 74.26472957829026) bank63206 +63207 POINT(40.06496274215716 74.35723702828186) bank63207 +63208 POINT(40.79608576044854 74.99339586785344) bank63208 +63209 POINT(39.797863010021956 73.17716278268655) bank63209 +63210 POINT(41.277563357631706 74.35515777744634) bank63210 +63211 POINT(40.651631629734254 74.54592945306882) bank63211 +63212 POINT(39.94802384389832 74.57424396473714) bank63212 +63213 POINT(40.964671645792066 73.751989297428) bank63213 +63214 POINT(41.549630295817046 74.22285695721271) bank63214 +63215 POINT(39.93315567032119 73.99481411082765) bank63215 +63216 POINT(40.84333727012916 73.75555575917409) bank63216 +63217 POINT(40.28550864504103 74.52229672485622) bank63217 +63218 POINT(40.91697457521565 74.99517521235475) bank63218 +63219 POINT(41.00167069922749 74.68504079150162) bank63219 +63220 POINT(41.34044125987265 73.19325960942055) bank63220 +63221 POINT(41.678112874560426 73.02983937524799) bank63221 +63222 POINT(39.94736562298784 73.68866879993959) bank63222 +63223 POINT(40.74251471101374 74.39146675183086) bank63223 +63224 POINT(41.628898093179096 73.75526934845976) bank63224 +63225 POINT(41.68411647804392 74.132858351587) bank63225 +63226 POINT(40.66701449485031 74.37360816465592) bank63226 +63227 POINT(41.11765580955857 74.38278491762613) bank63227 +63228 POINT(40.565463734955806 73.29627094313508) bank63228 +63229 POINT(41.34855974876834 73.82229826661755) bank63229 +63230 POINT(40.487233713228925 74.23246184272752) bank63230 +63231 POINT(41.553265288705475 73.94141459779627) bank63231 +63232 POINT(40.52156004050098 74.0047055287042) bank63232 +63233 POINT(40.39014569499232 73.96980124273006) bank63233 +63234 POINT(40.06719496043236 73.11564828093617) bank63234 +63235 POINT(40.664802701849595 74.93215011934473) bank63235 +63236 POINT(41.123718773906724 73.70062278775605) bank63236 +63237 POINT(41.171242147122676 73.05274910662091) bank63237 +63238 POINT(39.88178393778383 74.03489241046002) bank63238 +63239 POINT(40.87346430848544 73.44447836813784) bank63239 +63240 POINT(40.43131132482751 73.41260488886677) bank63240 +63241 POINT(40.073403146398654 74.88932100923482) bank63241 +63242 POINT(41.051375887104925 73.1773372841596) bank63242 +63243 POINT(41.68689692640934 73.67323540313811) bank63243 +63244 POINT(40.02080225581431 73.12523771036952) bank63244 +63245 POINT(41.326297936162156 73.8031942009719) bank63245 +63246 POINT(40.14171695161148 74.22989968498462) bank63246 +63247 POINT(40.771928289101545 73.70189443610374) bank63247 +63248 POINT(41.25509035611007 74.90718760024043) bank63248 +63249 POINT(41.06491545106153 73.31687934591159) bank63249 +63250 POINT(39.80612106284329 74.03202529844238) bank63250 +63251 POINT(41.027207096692685 73.58222648606613) bank63251 +63252 POINT(40.927823046146855 73.13992094990826) bank63252 +63253 POINT(41.01258456617794 73.80542610586706) bank63253 +63254 POINT(40.93057633693332 74.7175043364426) bank63254 +63255 POINT(40.01683972304261 73.60092625774925) bank63255 +63256 POINT(41.465163623904516 74.81307262767854) bank63256 +63257 POINT(40.88522090456104 73.21273272177939) bank63257 +63258 POINT(41.33818640440886 73.18015538948829) bank63258 +63259 POINT(41.541385983250144 74.01928248739128) bank63259 +63260 POINT(39.75998727478691 73.70234768770212) bank63260 +63261 POINT(40.85185314873474 73.21055518046366) bank63261 +63262 POINT(40.07277495359243 73.4784522494027) bank63262 +63263 POINT(40.99062858230184 73.7494985233333) bank63263 +63264 POINT(40.6988328105513 74.04907187798185) bank63264 +63265 POINT(39.79894005694601 74.74175165527066) bank63265 +63266 POINT(40.76428005531911 73.78616311344219) bank63266 +63267 POINT(41.48854927396978 73.67957463400533) bank63267 +63268 POINT(40.03264080737635 74.44745511081766) bank63268 +63269 POINT(41.18430280606566 74.56735320445972) bank63269 +63270 POINT(41.19012967719583 73.69751901043752) bank63270 +63271 POINT(41.49981918678296 73.02867816992614) bank63271 +63272 POINT(39.87080043381332 74.39583739909213) bank63272 +63273 POINT(40.37010235289572 74.88600495660089) bank63273 +63274 POINT(40.223978230924374 74.9378617263819) bank63274 +63275 POINT(40.590082405266884 74.54757886708526) bank63275 +63276 POINT(41.10964656703169 74.62332978553336) bank63276 +63277 POINT(40.482755435555866 74.23108351797363) bank63277 +63278 POINT(40.85523815561382 73.28022158257734) bank63278 +63279 POINT(40.05941194350674 73.81809839277561) bank63279 +63280 POINT(39.94777611122665 73.91884275733493) bank63280 +63281 POINT(39.90676091713408 73.17297268367265) bank63281 +63282 POINT(40.66768397015633 73.56711732198409) bank63282 +63283 POINT(40.70757887297757 74.0900258960339) bank63283 +63284 POINT(41.57998275569937 73.11051790007328) bank63284 +63285 POINT(39.82054853396732 74.66451531411442) bank63285 +63286 POINT(39.94821469377668 74.73829330402002) bank63286 +63287 POINT(40.26006712551366 73.61018399091107) bank63287 +63288 POINT(39.91871065583626 74.75770141415705) bank63288 +63289 POINT(40.97788222023405 74.93063789454533) bank63289 +63290 POINT(40.5316765284177 73.46667849383456) bank63290 +63291 POINT(40.65676152431803 73.78116472614336) bank63291 +63292 POINT(40.9518309619054 74.00223990792516) bank63292 +63293 POINT(39.9778402684759 74.4595164188541) bank63293 +63294 POINT(41.45886115810105 74.71757606066164) bank63294 +63295 POINT(40.64238929968014 74.09147876482275) bank63295 +63296 POINT(40.07404457636632 73.80525210782302) bank63296 +63297 POINT(40.93873781215886 73.8334300329203) bank63297 +63298 POINT(39.718177995917685 73.89745086662542) bank63298 +63299 POINT(41.67752497387902 74.0882366670919) bank63299 +63300 POINT(41.19987225866216 73.43271668168403) bank63300 +63301 POINT(40.641797894464844 74.16948670260615) bank63301 +63302 POINT(40.36479969155295 73.93829535609369) bank63302 +63303 POINT(41.053895649551784 74.96317632464674) bank63303 +63304 POINT(40.60276903974188 74.22269410625279) bank63304 +63305 POINT(40.40061598959611 74.25259024053449) bank63305 +63306 POINT(41.01560261699729 74.76051675251034) bank63306 +63307 POINT(40.34982262749672 73.48827673914293) bank63307 +63308 POINT(40.292402566066635 74.95099126764539) bank63308 +63309 POINT(40.73121889938467 74.4590796151097) bank63309 +63310 POINT(41.671416404420604 73.07036946382951) bank63310 +63311 POINT(40.911558087791306 73.66146489671797) bank63311 +63312 POINT(41.59663063800025 74.67361502449951) bank63312 +63313 POINT(41.23768585059408 73.25295361044793) bank63313 +63314 POINT(40.67738620134632 74.42360355219188) bank63314 +63315 POINT(39.98275906636386 74.60510381334478) bank63315 +63316 POINT(40.98658536010808 74.63606505987423) bank63316 +63317 POINT(39.91036222932405 74.37934613785326) bank63317 +63318 POINT(39.913671661258086 73.6177096175639) bank63318 +63319 POINT(40.99255968302737 73.64093989841075) bank63319 +63320 POINT(41.5801460973473 73.05410250784466) bank63320 +63321 POINT(40.567078618900815 73.30401499064256) bank63321 +63322 POINT(40.69172081178451 74.31941958258129) bank63322 +63323 POINT(41.47752738993191 74.94577625595583) bank63323 +63324 POINT(40.2007255678976 74.06935959522954) bank63324 +63325 POINT(40.01124429762301 73.57902385378151) bank63325 +63326 POINT(40.553109870867715 73.35392931939077) bank63326 +63327 POINT(41.14647126526085 74.12588375584016) bank63327 +63328 POINT(40.748939885318805 74.5383887008856) bank63328 +63329 POINT(40.41584563347336 73.7277410012425) bank63329 +63330 POINT(40.092201447221186 73.63195522329642) bank63330 +63331 POINT(41.12994222108895 73.8930636813244) bank63331 +63332 POINT(41.62390260555375 74.5658774868644) bank63332 +63333 POINT(40.46242307654108 74.61810019291768) bank63333 +63334 POINT(40.083910564484626 74.53206728539928) bank63334 +63335 POINT(41.27924511722881 74.62730246798027) bank63335 +63336 POINT(40.85833073766256 73.84317098351174) bank63336 +63337 POINT(41.607600602052024 73.85899380644875) bank63337 +63338 POINT(41.36239921168797 74.0273297544967) bank63338 +63339 POINT(40.73908442078797 74.21887358850655) bank63339 +63340 POINT(40.567742169469156 74.17808073035259) bank63340 +63341 POINT(40.46835413708012 73.2744432378283) bank63341 +63342 POINT(41.297424759616256 74.30025076613632) bank63342 +63343 POINT(39.956390453277855 73.64132624955369) bank63343 +63344 POINT(41.607316807054424 74.47129613924629) bank63344 +63345 POINT(41.40290995811652 74.13676943594032) bank63345 +63346 POINT(40.488088044825425 74.60191818224281) bank63346 +63347 POINT(40.864446335646804 74.70522153960239) bank63347 +63348 POINT(41.298835006743225 73.67683318197875) bank63348 +63349 POINT(40.25459304681257 74.76974062341766) bank63349 +63350 POINT(40.6491056687812 73.82398334932459) bank63350 +63351 POINT(41.11552198289188 74.19004768776652) bank63351 +63352 POINT(40.49508055711384 73.2266144541386) bank63352 +63353 POINT(40.00097254806879 74.83634556235546) bank63353 +63354 POINT(40.98970121675586 74.93914430707892) bank63354 +63355 POINT(40.301554220913175 73.43109885812014) bank63355 +63356 POINT(40.4379595468603 74.02324638089581) bank63356 +63357 POINT(40.86552866862021 74.65054494855862) bank63357 +63358 POINT(41.111263540948364 74.03495311895121) bank63358 +63359 POINT(40.37565109204997 73.3001026215094) bank63359 +63360 POINT(41.02464788688042 73.14958885146638) bank63360 +63361 POINT(40.617157782969606 73.19864522962654) bank63361 +63362 POINT(41.68808162491451 73.87962566914176) bank63362 +63363 POINT(40.184531989214676 74.85185739305699) bank63363 +63364 POINT(40.60435761758663 74.88343802908238) bank63364 +63365 POINT(39.733556160121424 73.92045482504528) bank63365 +63366 POINT(41.120625142061535 73.55836040269388) bank63366 +63367 POINT(41.34778102133351 73.9169934285697) bank63367 +63368 POINT(39.91618726124216 74.92449154107712) bank63368 +63369 POINT(40.10675944507601 73.14235730927027) bank63369 +63370 POINT(39.8092782135231 73.10999694719554) bank63370 +63371 POINT(40.58734183918878 74.42286990489796) bank63371 +63372 POINT(40.6215905732158 74.8573065678178) bank63372 +63373 POINT(40.950752991689555 74.24757933296684) bank63373 +63374 POINT(40.795607579759775 74.32499571201322) bank63374 +63375 POINT(40.46277251483278 74.72690937722676) bank63375 +63376 POINT(41.20423563355726 74.04778993377418) bank63376 +63377 POINT(40.08332257298477 74.49554642210055) bank63377 +63378 POINT(39.83346894462036 73.71566212567758) bank63378 +63379 POINT(40.776855630012534 74.18040701449546) bank63379 +63380 POINT(40.7479626111064 74.51651550945172) bank63380 +63381 POINT(41.04033029034045 73.73916867407857) bank63381 +63382 POINT(40.341097181751245 73.15110776766052) bank63382 +63383 POINT(41.13206570242009 73.45679351867874) bank63383 +63384 POINT(41.116062856412164 73.94342032071921) bank63384 +63385 POINT(40.63753916664503 74.10187055628572) bank63385 +63386 POINT(40.62189680085596 73.85063987892595) bank63386 +63387 POINT(39.87015273708041 73.18893822949268) bank63387 +63388 POINT(41.67490784463083 73.32327805815868) bank63388 +63389 POINT(40.53368252148125 74.71465650349303) bank63389 +63390 POINT(41.06272921220177 73.83033305178446) bank63390 +63391 POINT(40.48787344344813 73.14659013428422) bank63391 +63392 POINT(39.94772001224063 73.51764632205374) bank63392 +63393 POINT(41.0725393188687 73.01036047427138) bank63393 +63394 POINT(40.17982288203978 73.0556540144035) bank63394 +63395 POINT(40.078111365757465 73.23501205828731) bank63395 +63396 POINT(40.29364801285025 74.84768620726908) bank63396 +63397 POINT(41.61526691067488 74.92433016080864) bank63397 +63398 POINT(40.90161771880311 74.77689198683703) bank63398 +63399 POINT(39.78527878878356 73.33999332156291) bank63399 +63400 POINT(41.64039561916011 74.50149364833598) bank63400 +63401 POINT(40.892519223453995 73.64343922919076) bank63401 +63402 POINT(40.751076134974525 73.13792953789232) bank63402 +63403 POINT(40.85225792469262 73.9318136367848) bank63403 +63404 POINT(39.91041179260957 74.88877437829095) bank63404 +63405 POINT(41.642111436776574 74.65821402159978) bank63405 +63406 POINT(40.955477659549814 73.14892291486285) bank63406 +63407 POINT(41.61407990436586 73.96376083990371) bank63407 +63408 POINT(41.18716863466605 73.9515715627822) bank63408 +63409 POINT(41.54491893278363 74.38654291699952) bank63409 +63410 POINT(40.345435946437064 73.70468592182306) bank63410 +63411 POINT(40.45116069289884 73.48105596207508) bank63411 +63412 POINT(40.46332043664673 73.2610882976229) bank63412 +63413 POINT(39.79798540156928 74.4168209002996) bank63413 +63414 POINT(40.064017018058166 73.44937621723588) bank63414 +63415 POINT(40.408672947593814 74.66538782153643) bank63415 +63416 POINT(41.290581545069855 74.22819111158333) bank63416 +63417 POINT(41.111989901040666 74.95069770008494) bank63417 +63418 POINT(39.81617990817139 74.28783392475319) bank63418 +63419 POINT(40.317833804718816 74.28969428672683) bank63419 +63420 POINT(40.970119732689604 73.82845435579188) bank63420 +63421 POINT(40.51390794522243 73.88964603966765) bank63421 +63422 POINT(41.07451967160661 74.5915878106455) bank63422 +63423 POINT(40.40454040685454 73.91171583360797) bank63423 +63424 POINT(40.77017396875788 73.08110931358682) bank63424 +63425 POINT(41.412394284255214 73.20417906321924) bank63425 +63426 POINT(41.434373151034876 74.38239510263584) bank63426 +63427 POINT(41.62152990318843 74.68121236617282) bank63427 +63428 POINT(40.05165859460306 74.7089650791901) bank63428 +63429 POINT(40.876253209511646 74.77865287245912) bank63429 +63430 POINT(40.774472872170705 73.1675691007798) bank63430 +63431 POINT(41.486856233601955 74.42479504410305) bank63431 +63432 POINT(40.95533522919017 74.90063390857478) bank63432 +63433 POINT(41.00272985767294 74.20434259296376) bank63433 +63434 POINT(39.96756166334985 73.83128920524237) bank63434 +63435 POINT(41.46608032756595 74.48599334390572) bank63435 +63436 POINT(41.524357179091936 73.04383112875632) bank63436 +63437 POINT(40.09308111922461 74.1804507419139) bank63437 +63438 POINT(40.983226704890306 73.14852285844351) bank63438 +63439 POINT(40.77888577067062 73.64228861685496) bank63439 +63440 POINT(40.21638900606811 74.79502679709475) bank63440 +63441 POINT(41.667990737598245 74.76101629672566) bank63441 +63442 POINT(41.39923894392578 74.98820052733568) bank63442 +63443 POINT(40.836528837642284 73.27278695246189) bank63443 +63444 POINT(40.11857682534547 74.84620494903855) bank63444 +63445 POINT(39.903527108889264 73.34801562912604) bank63445 +63446 POINT(40.61914122249575 74.61650039054494) bank63446 +63447 POINT(40.72560678237377 74.89173326629994) bank63447 +63448 POINT(39.91553656385917 73.6111231715659) bank63448 +63449 POINT(40.74097378214841 73.9071917513587) bank63449 +63450 POINT(41.6842056173206 74.57788493819346) bank63450 +63451 POINT(41.577185294338506 74.16999374947356) bank63451 +63452 POINT(40.62390371449142 74.16047948059482) bank63452 +63453 POINT(40.944510222089754 74.85893577210734) bank63453 +63454 POINT(40.77043911988491 73.07971558381428) bank63454 +63455 POINT(39.92844564955535 73.72796354921329) bank63455 +63456 POINT(41.441141435460025 73.59104978664787) bank63456 +63457 POINT(41.29216470610187 73.30859688587653) bank63457 +63458 POINT(40.98652802307983 73.17988080874711) bank63458 +63459 POINT(40.502988197454364 74.71770393949436) bank63459 +63460 POINT(41.36494728383352 74.1929219474719) bank63460 +63461 POINT(40.34088380662584 73.28313632127534) bank63461 +63462 POINT(40.67884317830838 73.87705254471969) bank63462 +63463 POINT(41.55542863219133 74.58908623309422) bank63463 +63464 POINT(40.2475513420372 73.11598534521427) bank63464 +63465 POINT(40.95064143997825 74.46860895155716) bank63465 +63466 POINT(40.49074616499119 73.23341093291572) bank63466 +63467 POINT(41.37702971973878 73.21092799333579) bank63467 +63468 POINT(40.0889634876576 73.6203229417109) bank63468 +63469 POINT(40.45242555051992 74.61110834237495) bank63469 +63470 POINT(40.70331723145872 73.72145409864561) bank63470 +63471 POINT(40.391778852220995 74.66626584090878) bank63471 +63472 POINT(40.14136682550109 74.4646107792549) bank63472 +63473 POINT(40.55261968397445 73.6381472829579) bank63473 +63474 POINT(40.24206367556829 73.44412387601906) bank63474 +63475 POINT(39.89966488079663 74.11270323370174) bank63475 +63476 POINT(41.630233879873465 73.90425495530502) bank63476 +63477 POINT(40.96221564320924 74.63620136079793) bank63477 +63478 POINT(40.23973686816491 74.35991616430331) bank63478 +63479 POINT(39.92919899953632 73.60292149989576) bank63479 +63480 POINT(39.95334689966269 73.65379923135895) bank63480 +63481 POINT(40.301416761045914 74.07731396002073) bank63481 +63482 POINT(40.939521183896424 73.16845887749133) bank63482 +63483 POINT(40.30772554919152 74.46751829526062) bank63483 +63484 POINT(41.093554929686114 73.3036130657853) bank63484 +63485 POINT(40.277658242321245 73.33972350759937) bank63485 +63486 POINT(40.62632847180864 73.51871617212412) bank63486 +63487 POINT(41.569109326116084 74.54527074958587) bank63487 +63488 POINT(39.77891768557937 73.36190434318262) bank63488 +63489 POINT(41.516625449283836 73.65693745156358) bank63489 +63490 POINT(40.57202294179493 73.74274853718224) bank63490 +63491 POINT(39.91797037742322 74.57496813860708) bank63491 +63492 POINT(40.67755633742191 73.00985129048006) bank63492 +63493 POINT(41.24837107853737 73.52559362531998) bank63493 +63494 POINT(39.81531415961241 74.28983994078703) bank63494 +63495 POINT(40.2339088500423 73.21695336070667) bank63495 +63496 POINT(41.26903938005267 73.17073156598346) bank63496 +63497 POINT(40.80129485630087 74.33746265820406) bank63497 +63498 POINT(41.16732279802174 73.7566037147212) bank63498 +63499 POINT(41.6524793058496 74.45254951834117) bank63499 +63500 POINT(39.8071551824186 74.71839508662346) bank63500 +63501 POINT(39.768290939362664 73.02879230327478) bank63501 +63502 POINT(41.52404407746731 74.92645641352502) bank63502 +63503 POINT(40.73342330487493 73.5743334543941) bank63503 +63504 POINT(41.30878197945931 74.75236793817669) bank63504 +63505 POINT(40.76335794766998 73.24170620792282) bank63505 +63506 POINT(41.32166093643876 74.28313156928733) bank63506 +63507 POINT(39.85050526489501 73.50573377135477) bank63507 +63508 POINT(40.2375956070515 74.49946597237695) bank63508 +63509 POINT(40.78048287082351 74.53246447455174) bank63509 +63510 POINT(40.700083060518544 74.36620379935201) bank63510 +63511 POINT(40.13255838607862 74.78160967052143) bank63511 +63512 POINT(40.174515060234064 74.35171578525518) bank63512 +63513 POINT(40.71525187824317 73.68725192248885) bank63513 +63514 POINT(41.326890590923796 73.4009993747134) bank63514 +63515 POINT(41.699181257777745 74.48217600366323) bank63515 +63516 POINT(40.54983397918027 73.08068652956445) bank63516 +63517 POINT(40.68466894797913 73.2475134278869) bank63517 +63518 POINT(41.12501764610976 73.87600682260916) bank63518 +63519 POINT(41.59276648808067 74.19435766864264) bank63519 +63520 POINT(40.009230748151396 74.99822982759424) bank63520 +63521 POINT(41.13686211538294 74.02463338711213) bank63521 +63522 POINT(40.28375970177498 74.8310021597098) bank63522 +63523 POINT(40.246624596212484 74.59218644948501) bank63523 +63524 POINT(39.71818235755301 73.98601274332206) bank63524 +63525 POINT(41.08950849878256 73.25819753442106) bank63525 +63526 POINT(41.40775861698369 74.26176788806148) bank63526 +63527 POINT(40.76385663219472 73.98150117475961) bank63527 +63528 POINT(40.52235009549683 73.4095332975328) bank63528 +63529 POINT(41.444910173615305 73.44735336390295) bank63529 +63530 POINT(41.48978775061044 74.92047681655077) bank63530 +63531 POINT(41.13810728845266 73.81783531266724) bank63531 +63532 POINT(40.53025167779992 73.76636756932845) bank63532 +63533 POINT(40.693700603087684 74.63492996165544) bank63533 +63534 POINT(40.32943844596229 74.10521254804992) bank63534 +63535 POINT(41.284825184751064 74.30880187255909) bank63535 +63536 POINT(40.29869881959587 74.57423032761577) bank63536 +63537 POINT(41.39877168179724 74.62890805309317) bank63537 +63538 POINT(40.909409255564384 73.37682439944216) bank63538 +63539 POINT(40.20403874641017 73.84143283127199) bank63539 +63540 POINT(39.89022798854209 74.5714873891685) bank63540 +63541 POINT(40.51155716802945 74.48787839954457) bank63541 +63542 POINT(41.26581533520755 73.6529779294271) bank63542 +63543 POINT(40.47251590016293 74.96206920704195) bank63543 +63544 POINT(41.61566589009062 74.75849696204074) bank63544 +63545 POINT(39.798389640207084 73.78832487230805) bank63545 +63546 POINT(40.514827065765004 74.13057251044764) bank63546 +63547 POINT(41.04046897507413 73.60628823896866) bank63547 +63548 POINT(41.43879656190594 73.48775348290384) bank63548 +63549 POINT(41.71177980579671 73.986289868355) bank63549 +63550 POINT(40.67387822860995 73.21262657906786) bank63550 +63551 POINT(41.39871216659063 73.79771739666863) bank63551 +63552 POINT(40.322878612578315 73.4935494978247) bank63552 +63553 POINT(40.15192783511567 74.86437531064888) bank63553 +63554 POINT(40.519322582847735 73.14572666515889) bank63554 +63555 POINT(41.020833030123 74.75372260541447) bank63555 +63556 POINT(40.59366267306209 74.82694690120086) bank63556 +63557 POINT(40.221249122177944 74.16822103901363) bank63557 +63558 POINT(40.37353182643182 74.53110758212667) bank63558 +63559 POINT(41.69665844837049 73.30605678554262) bank63559 +63560 POINT(41.27710641900792 73.03394988412026) bank63560 +63561 POINT(41.4314445371834 74.74776099483915) bank63561 +63562 POINT(41.50781172844628 74.11398372319847) bank63562 +63563 POINT(40.55856267864941 73.57722223510552) bank63563 +63564 POINT(40.57100424099212 74.29931602620931) bank63564 +63565 POINT(41.37013639101555 73.4751195287246) bank63565 +63566 POINT(40.39395165430899 74.49618102376711) bank63566 +63567 POINT(41.34904581241238 74.38157690518393) bank63567 +63568 POINT(39.94199028574594 73.01694674643488) bank63568 +63569 POINT(40.22636862658201 73.08393430877743) bank63569 +63570 POINT(40.17881427437707 74.40868025822982) bank63570 +63571 POINT(40.61877188543691 74.47630651369754) bank63571 +63572 POINT(40.03961399012706 74.02698880893408) bank63572 +63573 POINT(40.61918287983771 73.96356013169851) bank63573 +63574 POINT(41.13229558113769 73.5479531322035) bank63574 +63575 POINT(40.200168755399346 74.93701191683665) bank63575 +63576 POINT(39.925283560994934 73.74592384923861) bank63576 +63577 POINT(40.20240702696176 73.29697075848193) bank63577 +63578 POINT(40.060858711780405 73.71995911127543) bank63578 +63579 POINT(40.19180565149238 74.35068003850664) bank63579 +63580 POINT(41.0851999687785 74.798745884566) bank63580 +63581 POINT(41.07798486975596 74.63875168697436) bank63581 +63582 POINT(41.21658237807537 73.38708826567795) bank63582 +63583 POINT(40.273555740171616 73.37907103474593) bank63583 +63584 POINT(40.92183144986608 74.38702469885821) bank63584 +63585 POINT(40.62380808024793 73.65626989082469) bank63585 +63586 POINT(41.42226504955658 73.63564675426056) bank63586 +63587 POINT(39.86453084082697 73.37763816451556) bank63587 +63588 POINT(40.47101170892409 74.31620766505992) bank63588 +63589 POINT(39.943473266166336 74.06399767914671) bank63589 +63590 POINT(40.30250486558386 74.80703960769061) bank63590 +63591 POINT(40.55570190794953 74.92466751719331) bank63591 +63592 POINT(41.457459911372645 74.18108627930842) bank63592 +63593 POINT(40.28842804701395 74.33995088450926) bank63593 +63594 POINT(39.924705940351146 73.9061459097132) bank63594 +63595 POINT(40.254752222775146 74.0847890653073) bank63595 +63596 POINT(41.52456423838896 74.36062246832003) bank63596 +63597 POINT(39.97651430345528 74.15042025848253) bank63597 +63598 POINT(40.56870640010872 74.75887978938773) bank63598 +63599 POINT(41.41962358307851 74.92833245460504) bank63599 +63600 POINT(40.87652419209984 73.89053422835549) bank63600 +63601 POINT(41.0666793351504 73.7819854581971) bank63601 +63602 POINT(41.57662615669001 73.23289059658016) bank63602 +63603 POINT(39.94797878268911 74.5444554170689) bank63603 +63604 POINT(41.18312375967125 74.07013299774633) bank63604 +63605 POINT(40.62352634482108 74.31284291036582) bank63605 +63606 POINT(40.50557397367304 73.66962860699822) bank63606 +63607 POINT(41.00612933124819 73.16198831233667) bank63607 +63608 POINT(40.38304598704381 74.75682153702924) bank63608 +63609 POINT(40.414577995500245 74.18252303998786) bank63609 +63610 POINT(41.21612112106012 74.63465499758578) bank63610 +63611 POINT(40.07351925939294 74.9851214860204) bank63611 +63612 POINT(40.72683219187743 74.87396276073663) bank63612 +63613 POINT(41.135507524592676 74.68321669227542) bank63613 +63614 POINT(40.597959615983044 74.86419187337977) bank63614 +63615 POINT(40.513526019866795 74.98256137766548) bank63615 +63616 POINT(41.20634492362622 73.52166020496223) bank63616 +63617 POINT(41.31200076791145 73.00893154414872) bank63617 +63618 POINT(41.44991840047457 73.30757467445406) bank63618 +63619 POINT(41.00900942768287 73.73682034072071) bank63619 +63620 POINT(40.562390370750045 73.16365268517796) bank63620 +63621 POINT(40.59747157102917 73.95741335064108) bank63621 +63622 POINT(40.246876216464834 74.57175554734495) bank63622 +63623 POINT(41.158903983377996 74.42672856031777) bank63623 +63624 POINT(41.057332543451665 74.33520890460771) bank63624 +63625 POINT(39.966103810205354 74.62428555044592) bank63625 +63626 POINT(40.67444122758416 74.47565064774074) bank63626 +63627 POINT(39.835970474596294 74.894452298978) bank63627 +63628 POINT(40.62635821997652 74.53788389327458) bank63628 +63629 POINT(41.15860001454749 74.31040788133903) bank63629 +63630 POINT(40.97124064835476 73.78842350219561) bank63630 +63631 POINT(40.342564452006684 74.11177645968375) bank63631 +63632 POINT(40.23838566084703 73.16598852434052) bank63632 +63633 POINT(40.75947863437338 73.13237319799639) bank63633 +63634 POINT(39.89940017884026 73.05544473754138) bank63634 +63635 POINT(41.53709480736907 74.16892742139082) bank63635 +63636 POINT(40.22105532154347 74.89983546643366) bank63636 +63637 POINT(41.012216010513384 73.2042041248243) bank63637 +63638 POINT(40.55482380911998 73.55226266060366) bank63638 +63639 POINT(40.637605622906385 73.58404444911314) bank63639 +63640 POINT(40.99961513916963 74.6239060125389) bank63640 +63641 POINT(40.638883183242434 74.98336806087872) bank63641 +63642 POINT(40.288870285212944 73.0853382951774) bank63642 +63643 POINT(41.22505058920695 73.59483357912332) bank63643 +63644 POINT(41.70647044831599 74.65212085372242) bank63644 +63645 POINT(41.29371227664007 73.37587897960069) bank63645 +63646 POINT(40.52671748105761 74.46235420814837) bank63646 +63647 POINT(41.36558606974622 74.30350837014981) bank63647 +63648 POINT(40.29227672451356 74.56973306598807) bank63648 +63649 POINT(40.584228314425786 73.72731003503264) bank63649 +63650 POINT(39.87698742489608 74.08388832325014) bank63650 +63651 POINT(41.038885381428095 73.95883915884204) bank63651 +63652 POINT(40.44986223596549 73.88911773342046) bank63652 +63653 POINT(39.83681352944859 73.76143395811927) bank63653 +63654 POINT(40.455667316615205 74.74257842319099) bank63654 +63655 POINT(41.33607279374637 73.49957443321388) bank63655 +63656 POINT(40.23269311318935 73.67747043279947) bank63656 +63657 POINT(40.29405522492319 73.0129672128304) bank63657 +63658 POINT(41.350022879687714 74.0761880867657) bank63658 +63659 POINT(41.39157547262115 74.53965743399695) bank63659 +63660 POINT(40.55828754964862 74.65490398871866) bank63660 +63661 POINT(39.995773716944576 74.0249753488278) bank63661 +63662 POINT(41.264581360716136 73.44554440061907) bank63662 +63663 POINT(40.02439268861638 74.14094196458268) bank63663 +63664 POINT(41.55132758699578 73.39534711834759) bank63664 +63665 POINT(40.44906868722672 73.72793240001616) bank63665 +63666 POINT(39.86047827800749 73.86044060983238) bank63666 +63667 POINT(39.73875929777541 73.62516135443259) bank63667 +63668 POINT(40.59670685565478 73.20813955729771) bank63668 +63669 POINT(41.50501504585934 73.76437330239733) bank63669 +63670 POINT(40.522493953772994 73.0602024018551) bank63670 +63671 POINT(39.928835524314394 74.87270775080468) bank63671 +63672 POINT(41.378516903380024 74.6867237373131) bank63672 +63673 POINT(39.85148547444146 73.61054653083482) bank63673 +63674 POINT(40.244507275626916 73.3393679531959) bank63674 +63675 POINT(40.69369532230711 74.36440059862207) bank63675 +63676 POINT(40.11990303073143 73.77149958012201) bank63676 +63677 POINT(40.71204442355621 74.65130098384286) bank63677 +63678 POINT(39.72946679680072 73.04620599041164) bank63678 +63679 POINT(40.541536468952586 74.56941178162414) bank63679 +63680 POINT(41.58718289249903 73.17799470667988) bank63680 +63681 POINT(41.47621898144516 73.94911078476723) bank63681 +63682 POINT(40.03000322040548 74.22035606121463) bank63682 +63683 POINT(40.32849937144289 74.25472361713781) bank63683 +63684 POINT(40.02304381175402 73.05754809369348) bank63684 +63685 POINT(40.46258512045303 73.87277570895124) bank63685 +63686 POINT(40.673977199591896 74.12314650642767) bank63686 +63687 POINT(39.96056899728072 73.42925237363856) bank63687 +63688 POINT(41.127688903607684 74.69261972088488) bank63688 +63689 POINT(41.704084203010126 73.89003233115723) bank63689 +63690 POINT(39.837404776783536 74.90139963859005) bank63690 +63691 POINT(40.243048076310394 74.05731722683146) bank63691 +63692 POINT(41.374552134590246 74.3723819821126) bank63692 +63693 POINT(41.271399204758524 74.20930398847999) bank63693 +63694 POINT(40.11971870052002 73.24564737880803) bank63694 +63695 POINT(40.01191302062412 74.61418210363529) bank63695 +63696 POINT(41.10439360256856 74.98400785587195) bank63696 +63697 POINT(40.74169179582235 73.63810187379086) bank63697 +63698 POINT(41.37352810953971 73.4670365741574) bank63698 +63699 POINT(39.93681197444229 73.92324206261235) bank63699 +63700 POINT(41.55922924051236 74.56746853042772) bank63700 +63701 POINT(39.71907307800801 74.3374795078285) bank63701 +63702 POINT(41.587960782517 73.42593094243664) bank63702 +63703 POINT(40.792834856517906 73.76427313637107) bank63703 +63704 POINT(39.817184816254624 73.64899781719069) bank63704 +63705 POINT(39.805014488531135 73.56654451842842) bank63705 +63706 POINT(39.967921461211475 74.92715545586881) bank63706 +63707 POINT(41.56883934188003 74.62991247294775) bank63707 +63708 POINT(39.8346381553512 74.13981389146089) bank63708 +63709 POINT(40.50049931900554 74.69727440390632) bank63709 +63710 POINT(40.87298331866979 73.99387895839882) bank63710 +63711 POINT(41.16095404474943 74.7441489027537) bank63711 +63712 POINT(41.48737761528526 73.73244209025897) bank63712 +63713 POINT(41.51012788338634 74.31952155475416) bank63713 +63714 POINT(40.01746329219316 73.49564567141988) bank63714 +63715 POINT(40.55348982971658 74.98040555144297) bank63715 +63716 POINT(40.59291019891721 73.50849174113694) bank63716 +63717 POINT(40.50489227726296 73.20613942250945) bank63717 +63718 POINT(41.28487799406632 73.66133382351512) bank63718 +63719 POINT(39.8088732674677 73.79706247831218) bank63719 +63720 POINT(40.43055259603019 73.91302195816198) bank63720 +63721 POINT(40.36055281844753 73.1577799807731) bank63721 +63722 POINT(41.59224387052696 73.57306797698446) bank63722 +63723 POINT(41.25514813434475 73.98774081963802) bank63723 +63724 POINT(40.461496079356806 73.84690772430592) bank63724 +63725 POINT(41.1441175364952 73.85771590237766) bank63725 +63726 POINT(39.982071303581975 74.69225163528512) bank63726 +63727 POINT(40.064128929047 74.46281554226432) bank63727 +63728 POINT(40.214231491174054 74.61414604113776) bank63728 +63729 POINT(41.2302561886921 73.06716348372179) bank63729 +63730 POINT(40.90245354283159 74.04322849962715) bank63730 +63731 POINT(40.38918651645784 73.21410509415088) bank63731 +63732 POINT(40.49606352641517 73.9061303078188) bank63732 +63733 POINT(40.414463372235375 74.66890736932962) bank63733 +63734 POINT(40.82215654611099 73.17857121656365) bank63734 +63735 POINT(41.19907653075221 73.92298606987858) bank63735 +63736 POINT(40.82906843785414 73.51391964740861) bank63736 +63737 POINT(39.78130057689082 73.3880625750115) bank63737 +63738 POINT(40.42931510952165 73.42587010162143) bank63738 +63739 POINT(40.10896468368173 73.30078385787044) bank63739 +63740 POINT(41.698018415013536 73.16583748340936) bank63740 +63741 POINT(41.656955203933336 74.19539686321048) bank63741 +63742 POINT(41.018289006308535 73.86726818215683) bank63742 +63743 POINT(41.65105778118179 73.1136728033678) bank63743 +63744 POINT(41.598345024669214 74.18099219116085) bank63744 +63745 POINT(41.41402036599034 73.99050457407213) bank63745 +63746 POINT(40.23652987891682 73.26504578420327) bank63746 +63747 POINT(40.997676942725775 73.99149093246427) bank63747 +63748 POINT(40.83014612907604 74.26001208259854) bank63748 +63749 POINT(41.57812150392139 73.42746542454393) bank63749 +63750 POINT(40.48855166124769 73.35569748932808) bank63750 +63751 POINT(40.83903043337067 74.33058167627955) bank63751 +63752 POINT(41.34848874658756 73.92093696575493) bank63752 +63753 POINT(40.26351462771177 74.44965903889238) bank63753 +63754 POINT(41.31466437136169 73.49687275171512) bank63754 +63755 POINT(41.10500142066676 74.7966667992139) bank63755 +63756 POINT(40.962679937781814 73.85702062012193) bank63756 +63757 POINT(40.70112471506833 74.10984330494537) bank63757 +63758 POINT(41.02859293439288 74.2154946498589) bank63758 +63759 POINT(41.26140381058492 74.64717724884851) bank63759 +63760 POINT(40.036211889641194 74.47155205705218) bank63760 +63761 POINT(40.270543165003744 74.56758157835607) bank63761 +63762 POINT(40.977244810335904 73.18406646396123) bank63762 +63763 POINT(40.14933959845034 73.14727584393864) bank63763 +63764 POINT(40.37146043565902 74.04435571904703) bank63764 +63765 POINT(40.45011563773741 74.95773146217343) bank63765 +63766 POINT(40.227711740357414 73.74202038634759) bank63766 +63767 POINT(40.7543713761869 74.82036541927901) bank63767 +63768 POINT(40.020928078663445 73.91682672261992) bank63768 +63769 POINT(40.0878054181363 74.69394274337553) bank63769 +63770 POINT(41.14346404754499 73.77855698927988) bank63770 +63771 POINT(39.89164338584555 73.13947033764015) bank63771 +63772 POINT(39.74744870398526 73.78283031259392) bank63772 +63773 POINT(41.36290676348241 74.61151517699818) bank63773 +63774 POINT(41.12373278412399 74.23810346347547) bank63774 +63775 POINT(40.74620832887899 73.85329083394561) bank63775 +63776 POINT(39.77567626158195 74.13084760471844) bank63776 +63777 POINT(40.04099718852961 74.43460968042116) bank63777 +63778 POINT(40.42479405461459 74.39805640775229) bank63778 +63779 POINT(40.73799156929751 74.49440031757237) bank63779 +63780 POINT(41.63988541840733 74.12917107573895) bank63780 +63781 POINT(41.0823467557088 74.5622664937096) bank63781 +63782 POINT(41.23549535515898 73.34753055644785) bank63782 +63783 POINT(40.835045545976385 73.30735386801794) bank63783 +63784 POINT(40.97853388336843 73.2022607681174) bank63784 +63785 POINT(41.42918503063729 74.61177439540427) bank63785 +63786 POINT(41.133535192035026 74.11824616219015) bank63786 +63787 POINT(40.61469451430701 74.67489377152842) bank63787 +63788 POINT(41.11031834025916 74.33753261937781) bank63788 +63789 POINT(39.81847226963957 73.74274853552448) bank63789 +63790 POINT(41.13372375363708 73.44241886273271) bank63790 +63791 POINT(41.2816328406272 73.52388842647306) bank63791 +63792 POINT(40.98730146741743 73.21470148885712) bank63792 +63793 POINT(41.53869279612259 73.07080344199565) bank63793 +63794 POINT(40.228902621517186 74.7387522742855) bank63794 +63795 POINT(40.15912251289178 73.9738647015154) bank63795 +63796 POINT(41.65757338116362 73.88161022789764) bank63796 +63797 POINT(40.391124951270655 73.69997409816838) bank63797 +63798 POINT(39.99567726847718 73.23678231975484) bank63798 +63799 POINT(40.53501079800036 74.39946521427791) bank63799 +63800 POINT(40.66639144115941 74.21832989967815) bank63800 +63801 POINT(39.99731829230811 73.00633274115356) bank63801 +63802 POINT(40.0267853636831 73.01924017615856) bank63802 +63803 POINT(39.9566058105272 74.00055638029065) bank63803 +63804 POINT(40.57255769489821 74.50676954493899) bank63804 +63805 POINT(39.94836245861988 73.6194777821651) bank63805 +63806 POINT(40.34564312161811 74.5180655105321) bank63806 +63807 POINT(41.54798860521838 74.39620235988623) bank63807 +63808 POINT(40.57623122136055 73.01404629817836) bank63808 +63809 POINT(39.986880994352944 74.98740192653963) bank63809 +63810 POINT(39.861277919077374 74.5519362906842) bank63810 +63811 POINT(40.65155123626644 74.30664713759174) bank63811 +63812 POINT(40.673768863255496 74.79948885223368) bank63812 +63813 POINT(41.56015531037025 74.67261554520007) bank63813 +63814 POINT(40.680550277213726 74.46061553320503) bank63814 +63815 POINT(39.94653747288524 73.37111853326779) bank63815 +63816 POINT(41.20100565477767 74.7449285148842) bank63816 +63817 POINT(40.414464701022034 73.13305772776451) bank63817 +63818 POINT(40.48876418386429 73.09518405967275) bank63818 +63819 POINT(40.16005587968665 73.75638099610687) bank63819 +63820 POINT(40.201392741324625 73.24485829916462) bank63820 +63821 POINT(41.44772463619868 73.91419961530852) bank63821 +63822 POINT(40.91724573033996 73.93615234104543) bank63822 +63823 POINT(40.60039913081442 73.77517551675886) bank63823 +63824 POINT(41.3924080577636 73.42207473960974) bank63824 +63825 POINT(40.213008957124465 73.22167657768861) bank63825 +63826 POINT(40.21126805146031 74.25338282266338) bank63826 +63827 POINT(40.0494254213934 73.52385630470798) bank63827 +63828 POINT(41.14596903371148 74.09906405912646) bank63828 +63829 POINT(40.996436101789826 73.04598653893171) bank63829 +63830 POINT(40.059120360494504 73.71906297607248) bank63830 +63831 POINT(41.62787815614959 74.51051027337212) bank63831 +63832 POINT(41.46961309929229 73.69300724577688) bank63832 +63833 POINT(40.34414518542952 73.0063539723959) bank63833 +63834 POINT(41.07676285968689 73.8619901030265) bank63834 +63835 POINT(40.29080565383424 73.73973309999293) bank63835 +63836 POINT(40.24779453715689 74.78601563096079) bank63836 +63837 POINT(40.44033102785125 74.77386779117805) bank63837 +63838 POINT(41.69506009561524 74.04764072711262) bank63838 +63839 POINT(40.80029018856018 74.13610786732124) bank63839 +63840 POINT(39.920419178696854 73.20890116318093) bank63840 +63841 POINT(41.42607966859343 74.77251321363528) bank63841 +63842 POINT(40.06256775097088 74.98012873398284) bank63842 +63843 POINT(40.71675304806992 73.36699858574298) bank63843 +63844 POINT(41.06634066821698 73.74815968222771) bank63844 +63845 POINT(41.48592985156765 74.93903713499255) bank63845 +63846 POINT(41.1100775967006 73.2730545800302) bank63846 +63847 POINT(39.91544557673816 73.03766852194963) bank63847 +63848 POINT(39.8553376669096 73.48726480768882) bank63848 +63849 POINT(39.7406653347018 74.34314142605072) bank63849 +63850 POINT(41.291229786381145 73.73537789417911) bank63850 +63851 POINT(39.8125607372157 74.20015478883514) bank63851 +63852 POINT(40.41736614865083 74.934887980134) bank63852 +63853 POINT(41.4010341189176 74.77455927671753) bank63853 +63854 POINT(41.31871552869078 73.8391145271823) bank63854 +63855 POINT(41.67899353705293 73.01096598591944) bank63855 +63856 POINT(40.56734280916335 73.71874044175797) bank63856 +63857 POINT(40.885814973580835 73.53283579448885) bank63857 +63858 POINT(40.70879935192244 74.21102490826252) bank63858 +63859 POINT(41.35146649198293 73.3823609541814) bank63859 +63860 POINT(39.95695225268224 73.63632775664392) bank63860 +63861 POINT(41.303327969966034 73.2236097944722) bank63861 +63862 POINT(40.67729204376879 74.11966789747214) bank63862 +63863 POINT(40.543503155112305 73.38980550770648) bank63863 +63864 POINT(40.11934585361026 74.35854033096749) bank63864 +63865 POINT(41.21417799267852 74.66642765929372) bank63865 +63866 POINT(40.18782256337361 74.87866838613868) bank63866 +63867 POINT(40.02108494133286 74.8892295765399) bank63867 +63868 POINT(40.121578417775545 74.3781471986088) bank63868 +63869 POINT(39.80098427795611 73.98512606883737) bank63869 +63870 POINT(41.431059226512644 73.2926119547297) bank63870 +63871 POINT(40.02078575670015 73.60200153093857) bank63871 +63872 POINT(40.97986086981833 73.30444475631812) bank63872 +63873 POINT(40.684882365845276 73.62774074473478) bank63873 +63874 POINT(41.49121033401218 73.69009081589235) bank63874 +63875 POINT(39.78372856028986 74.42811019736412) bank63875 +63876 POINT(40.046883425346586 73.43119999868296) bank63876 +63877 POINT(41.1240288690876 73.68221817694827) bank63877 +63878 POINT(39.99663625334554 73.27341762792896) bank63878 +63879 POINT(41.38730593707866 73.43393940601351) bank63879 +63880 POINT(41.42760523881921 74.92367203915309) bank63880 +63881 POINT(40.24834189638792 74.79913678874136) bank63881 +63882 POINT(40.285009163259176 74.04970612433323) bank63882 +63883 POINT(39.78269754359199 73.22726047439333) bank63883 +63884 POINT(39.940837425223364 73.81991429423941) bank63884 +63885 POINT(40.984365234548406 74.15031507242922) bank63885 +63886 POINT(39.76966305749018 73.73831778615674) bank63886 +63887 POINT(39.8170139226481 74.95357286798581) bank63887 +63888 POINT(39.746197476287385 73.43452119049925) bank63888 +63889 POINT(41.256364311683726 73.70612692562698) bank63889 +63890 POINT(40.788484128434796 73.17989025516977) bank63890 +63891 POINT(41.39280453576648 74.2777376566028) bank63891 +63892 POINT(41.25066037985375 74.88418542630266) bank63892 +63893 POINT(41.538205797565176 73.12709338176973) bank63893 +63894 POINT(40.339243348018805 74.41373663588207) bank63894 +63895 POINT(41.54240747356248 73.85546250366284) bank63895 +63896 POINT(41.10732185603284 74.23313834394085) bank63896 +63897 POINT(41.1099916498675 74.38996297174307) bank63897 +63898 POINT(40.523609344149484 74.53665925631434) bank63898 +63899 POINT(41.09778502875084 73.71777131201631) bank63899 +63900 POINT(40.12448434358392 74.78413267645043) bank63900 +63901 POINT(41.65162855051138 74.6259689836137) bank63901 +63902 POINT(41.583640430466296 74.25801772027042) bank63902 +63903 POINT(41.38807828283627 73.84643522243006) bank63903 +63904 POINT(40.2290134766368 74.66711065113788) bank63904 +63905 POINT(40.85790127328711 74.98504698140492) bank63905 +63906 POINT(41.117719621430034 73.99787642806903) bank63906 +63907 POINT(40.660104276920556 73.18839645229538) bank63907 +63908 POINT(39.98921164418566 73.87496829048581) bank63908 +63909 POINT(40.34068995543646 73.8200769074322) bank63909 +63910 POINT(40.89444129376384 73.71251257817462) bank63910 +63911 POINT(40.914228160313996 74.21049982288261) bank63911 +63912 POINT(40.08258416124614 74.22229742649283) bank63912 +63913 POINT(41.55452271535864 74.11814623748037) bank63913 +63914 POINT(41.402482162863386 73.42112498055344) bank63914 +63915 POINT(40.64192761476338 74.05562137833893) bank63915 +63916 POINT(40.99389259935114 74.38373436352543) bank63916 +63917 POINT(40.005005340954355 73.55514759616152) bank63917 +63918 POINT(41.62858974469213 74.87992926992673) bank63918 +63919 POINT(41.29614435598627 73.89308223638248) bank63919 +63920 POINT(41.60908300805494 73.7712649644227) bank63920 +63921 POINT(40.05658322473296 73.24439845300304) bank63921 +63922 POINT(41.04665431060479 73.12215195101075) bank63922 +63923 POINT(40.11099565301737 74.65294547104332) bank63923 +63924 POINT(40.13036597923652 74.40293191185262) bank63924 +63925 POINT(41.6198071884937 73.1221954175906) bank63925 +63926 POINT(40.708859761289006 73.64130204449239) bank63926 +63927 POINT(41.01301127793967 74.76657691419605) bank63927 +63928 POINT(41.0367680809871 74.27441981649788) bank63928 +63929 POINT(40.25025951937789 73.99793890115654) bank63929 +63930 POINT(41.08397781867198 73.89458623395998) bank63930 +63931 POINT(40.51949643017088 74.32998813104601) bank63931 +63932 POINT(39.91054887561816 74.44674968170519) bank63932 +63933 POINT(41.101048784567645 74.2930544478601) bank63933 +63934 POINT(41.559431415593174 74.1481804713705) bank63934 +63935 POINT(40.174636870429346 74.40179555920729) bank63935 +63936 POINT(41.00663058234506 74.62666371189495) bank63936 +63937 POINT(40.81939885198965 73.82404217153415) bank63937 +63938 POINT(39.81655416630234 74.09743640540852) bank63938 +63939 POINT(41.71194116583193 74.35351847500938) bank63939 +63940 POINT(40.99184818882191 73.32265761005722) bank63940 +63941 POINT(41.688548806248164 74.06381505525427) bank63941 +63942 POINT(41.18835625999965 73.35945830523434) bank63942 +63943 POINT(40.309801667298544 73.44799772235032) bank63943 +63944 POINT(40.214489423554255 73.70871470044058) bank63944 +63945 POINT(40.1680848521749 74.15613616985362) bank63945 +63946 POINT(40.28994313548647 74.51061051984172) bank63946 +63947 POINT(41.67649520011194 73.95475456566058) bank63947 +63948 POINT(41.24652178122961 74.88183808979645) bank63948 +63949 POINT(40.567476696047116 74.65586622356832) bank63949 +63950 POINT(40.174946547425044 73.4400885813335) bank63950 +63951 POINT(40.07307114361982 73.62636419399173) bank63951 +63952 POINT(40.532339901869335 73.52407949567875) bank63952 +63953 POINT(40.25743912519009 74.0702706935851) bank63953 +63954 POINT(41.49701637326828 73.81891579328611) bank63954 +63955 POINT(41.228964630950614 73.26209906155037) bank63955 +63956 POINT(40.469891880297936 74.27102274122842) bank63956 +63957 POINT(40.57447047403751 74.80138908768816) bank63957 +63958 POINT(41.650164336365584 73.64536644360459) bank63958 +63959 POINT(40.21660947829989 74.94741796023919) bank63959 +63960 POINT(41.059363217716125 73.07540080974941) bank63960 +63961 POINT(41.5670833879944 74.01858855175959) bank63961 +63962 POINT(39.81790634530355 74.76385764788326) bank63962 +63963 POINT(39.92258392933826 73.38292490046068) bank63963 +63964 POINT(40.21842616522978 74.66410135249053) bank63964 +63965 POINT(40.229153848814654 74.71027986914805) bank63965 +63966 POINT(40.56526124096256 74.26479786278558) bank63966 +63967 POINT(40.831260546866936 73.11386365909675) bank63967 +63968 POINT(39.99652533514976 74.53119232240046) bank63968 +63969 POINT(41.53306112046942 73.18532136148751) bank63969 +63970 POINT(40.11340096301277 74.41134978246262) bank63970 +63971 POINT(40.095680842633385 73.26960580672745) bank63971 +63972 POINT(41.32097772991698 74.39792481569106) bank63972 +63973 POINT(40.57083890073603 73.16319829326707) bank63973 +63974 POINT(40.46733004707184 73.58752023887956) bank63974 +63975 POINT(40.78464452896792 73.80730835194389) bank63975 +63976 POINT(40.012760974728025 73.43702861488231) bank63976 +63977 POINT(41.51698124140906 73.17718042650307) bank63977 +63978 POINT(40.43597446769278 74.85620559035348) bank63978 +63979 POINT(40.75125426314948 73.88800105143523) bank63979 +63980 POINT(39.965890930470486 74.4337679754426) bank63980 +63981 POINT(40.54203240208245 73.58583638784135) bank63981 +63982 POINT(40.17321705000214 74.95919932181532) bank63982 +63983 POINT(41.27313792333293 74.44392560358924) bank63983 +63984 POINT(41.04480728677142 73.26608173364185) bank63984 +63985 POINT(41.046068240720686 74.33760284367378) bank63985 +63986 POINT(41.119359501928 73.67928189527223) bank63986 +63987 POINT(40.29020491492955 74.37713306160327) bank63987 +63988 POINT(40.40699664076594 74.14637501336718) bank63988 +63989 POINT(41.55160517901575 73.44247523878987) bank63989 +63990 POINT(39.76679766658246 74.91581781401744) bank63990 +63991 POINT(39.925338459025454 74.46086417871804) bank63991 +63992 POINT(40.35207730272488 73.71679181836323) bank63992 +63993 POINT(40.65929834445315 73.19227390269353) bank63993 +63994 POINT(41.260332790144574 73.7600110736235) bank63994 +63995 POINT(41.64364054265859 74.58341735421355) bank63995 +63996 POINT(40.94562352841826 74.23134104736378) bank63996 +63997 POINT(40.31081984390917 73.90205399609117) bank63997 +63998 POINT(41.23799354199189 73.83366351311986) bank63998 +63999 POINT(41.52373284945958 73.80660033134838) bank63999 +64000 POINT(40.377115650660144 73.01632682182377) bank64000 +64001 POINT(39.94510176123302 73.57784026849536) bank64001 +64002 POINT(41.191047955119096 74.57182928451503) bank64002 +64003 POINT(40.66473202721398 74.64564870579113) bank64003 +64004 POINT(40.70436679527436 73.33398930713007) bank64004 +64005 POINT(41.570935525000195 74.46945970581861) bank64005 +64006 POINT(40.807476024426215 73.93479795805744) bank64006 +64007 POINT(40.53385322292407 74.57965022941522) bank64007 +64008 POINT(40.385080943783294 74.06307935331012) bank64008 +64009 POINT(39.77478335421651 73.79985851614481) bank64009 +64010 POINT(39.88462891340716 73.48651692732884) bank64010 +64011 POINT(41.3253540572276 74.44746342840276) bank64011 +64012 POINT(40.22201246276664 73.52676358193571) bank64012 +64013 POINT(41.655314642741814 74.47444875649278) bank64013 +64014 POINT(40.985367759006934 73.64650541556128) bank64014 +64015 POINT(40.14149573601653 73.78433715011981) bank64015 +64016 POINT(41.36812331009689 74.55212549853292) bank64016 +64017 POINT(40.67456403655389 73.57698237721299) bank64017 +64018 POINT(41.20467558726479 73.01318365257454) bank64018 +64019 POINT(41.51951629998589 74.9092951843842) bank64019 +64020 POINT(41.62396396572319 74.03437119201409) bank64020 +64021 POINT(40.014448845861935 73.23548797184077) bank64021 +64022 POINT(40.2300848179057 74.47632225524127) bank64022 +64023 POINT(40.4264786061973 73.7413656025387) bank64023 +64024 POINT(39.995975413286395 73.7344227238103) bank64024 +64025 POINT(41.705523328685494 73.44036631536198) bank64025 +64026 POINT(40.5986468050798 74.92653746344482) bank64026 +64027 POINT(41.14498257754856 73.04718993680902) bank64027 +64028 POINT(40.928249298080296 74.11736029967757) bank64028 +64029 POINT(40.40542862658257 73.47990815697368) bank64029 +64030 POINT(40.64707343862209 73.35828355426351) bank64030 +64031 POINT(40.689440768355816 74.75336212502586) bank64031 +64032 POINT(41.44328123763702 73.25025609971597) bank64032 +64033 POINT(41.357382724625985 73.36728868785208) bank64033 +64034 POINT(40.98983011683961 74.68346273669286) bank64034 +64035 POINT(40.76661144732224 73.20225201083265) bank64035 +64036 POINT(40.65022181676581 73.56692193055024) bank64036 +64037 POINT(41.043978213272105 74.66955891167643) bank64037 +64038 POINT(40.43609259802134 73.5531308095338) bank64038 +64039 POINT(39.75735585797421 73.720495403552) bank64039 +64040 POINT(40.80583479429022 73.90085162929364) bank64040 +64041 POINT(41.48887291448137 73.52950214767468) bank64041 +64042 POINT(39.83488063606092 74.9667615935319) bank64042 +64043 POINT(40.74838936962865 74.08107778541553) bank64043 +64044 POINT(40.02375076290833 73.77268792359497) bank64044 +64045 POINT(40.140871343735355 73.75322616227811) bank64045 +64046 POINT(40.03736126482422 73.08951151034698) bank64046 +64047 POINT(40.67953399696627 74.25358237628447) bank64047 +64048 POINT(40.25528511167532 73.8397960177314) bank64048 +64049 POINT(40.15014158636412 74.30346323029829) bank64049 +64050 POINT(40.51237658720877 73.6713985057616) bank64050 +64051 POINT(41.65560008227994 73.14983371604171) bank64051 +64052 POINT(40.404889901838935 74.77196229729121) bank64052 +64053 POINT(40.53771544876051 74.39768779052119) bank64053 +64054 POINT(40.4361932065036 74.30819057870904) bank64054 +64055 POINT(40.44783289922826 73.24165710414184) bank64055 +64056 POINT(41.04567560843986 74.0145554204621) bank64056 +64057 POINT(40.39826905504984 74.99923602974003) bank64057 +64058 POINT(40.81272613608107 74.1877404048905) bank64058 +64059 POINT(40.338002518444625 73.99397654981642) bank64059 +64060 POINT(40.42555967750894 73.03611480848765) bank64060 +64061 POINT(41.07944873835118 73.5131769927995) bank64061 +64062 POINT(40.30724200024949 74.38849149167173) bank64062 +64063 POINT(40.0203888371407 73.19875079614995) bank64063 +64064 POINT(40.38084537367806 73.82333407392817) bank64064 +64065 POINT(41.596697960212474 74.72093318133115) bank64065 +64066 POINT(41.594992383423666 73.09091038670626) bank64066 +64067 POINT(41.55210102315303 73.60537554189787) bank64067 +64068 POINT(41.17565194390526 73.2372579318251) bank64068 +64069 POINT(40.4813907874087 74.81163108809024) bank64069 +64070 POINT(41.21245469596921 74.12749523749383) bank64070 +64071 POINT(40.03213217619495 74.98328621174595) bank64071 +64072 POINT(41.23543534190415 73.65702502881051) bank64072 +64073 POINT(39.75843639845618 73.22710958430454) bank64073 +64074 POINT(40.72421770902634 73.35784099736333) bank64074 +64075 POINT(40.388557828579714 74.37219854716133) bank64075 +64076 POINT(40.29046511416539 74.76720842730482) bank64076 +64077 POINT(40.04436611710718 73.58907558338181) bank64077 +64078 POINT(41.495571377257455 73.79989760293336) bank64078 +64079 POINT(41.41361703571996 74.79187057772037) bank64079 +64080 POINT(41.5557175506367 74.4035881206096) bank64080 +64081 POINT(40.08237135981636 74.50092908170683) bank64081 +64082 POINT(41.03473301578395 74.77947027829123) bank64082 +64083 POINT(40.96195503141367 74.33793352469318) bank64083 +64084 POINT(40.91054098998678 74.18512088782273) bank64084 +64085 POINT(40.809602522697254 74.2319849802302) bank64085 +64086 POINT(40.79846075860493 74.62198747212997) bank64086 +64087 POINT(40.99307882234752 73.55655858220567) bank64087 +64088 POINT(40.63614055244771 73.4020670730381) bank64088 +64089 POINT(40.59672560996062 73.23899235285239) bank64089 +64090 POINT(40.964521816423236 74.68213682600512) bank64090 +64091 POINT(41.04811095015926 73.06143063131486) bank64091 +64092 POINT(41.057110091158485 73.39849890922014) bank64092 +64093 POINT(39.88868242786431 73.8465592984861) bank64093 +64094 POINT(40.24659000066262 74.43564355483156) bank64094 +64095 POINT(39.855871145238396 74.54905041698271) bank64095 +64096 POINT(40.7774967168194 74.87124515046125) bank64096 +64097 POINT(40.78735429499572 73.8815027594862) bank64097 +64098 POINT(40.963866042013045 73.0704367639523) bank64098 +64099 POINT(40.47659716484208 74.79585901111082) bank64099 +64100 POINT(40.51939890186636 73.91673110833229) bank64100 +64101 POINT(40.55112106220612 73.89857300372344) bank64101 +64102 POINT(41.187362547606455 73.60263887930276) bank64102 +64103 POINT(41.29238240067229 74.70852625057971) bank64103 +64104 POINT(39.87776236255278 73.0968376269197) bank64104 +64105 POINT(40.51967741387891 74.84305161067343) bank64105 +64106 POINT(40.61223769321056 74.00832577333392) bank64106 +64107 POINT(39.8811630623724 74.52891423916333) bank64107 +64108 POINT(41.165229744407185 73.74298863242875) bank64108 +64109 POINT(40.2853901444206 74.6246735423847) bank64109 +64110 POINT(40.92929001865454 74.62466948434275) bank64110 +64111 POINT(41.39068117571147 73.48140698542166) bank64111 +64112 POINT(40.780127297678824 74.32449362263051) bank64112 +64113 POINT(40.432783175061275 74.42941867758398) bank64113 +64114 POINT(41.58015326919939 73.63116821271126) bank64114 +64115 POINT(40.834126010281395 74.62131856387366) bank64115 +64116 POINT(41.08216519521055 73.72544594943709) bank64116 +64117 POINT(40.6548924588174 74.05732509906082) bank64117 +64118 POINT(40.67074842047812 73.2151777650382) bank64118 +64119 POINT(40.226184933800454 73.09237291726512) bank64119 +64120 POINT(40.64866445442812 74.12634541402673) bank64120 +64121 POINT(39.96074279447614 73.62369460016102) bank64121 +64122 POINT(40.98177284978798 73.30450236086092) bank64122 +64123 POINT(40.76402218549098 73.5748594298251) bank64123 +64124 POINT(40.375063766967834 74.41063762203247) bank64124 +64125 POINT(40.686815055382226 74.09309446179826) bank64125 +64126 POINT(41.43845329017487 74.3615019042965) bank64126 +64127 POINT(40.22732244034263 74.36191841072775) bank64127 +64128 POINT(41.3423372070564 74.21114674885986) bank64128 +64129 POINT(40.74805175375104 73.35791835161473) bank64129 +64130 POINT(40.161744565134235 73.31338684207714) bank64130 +64131 POINT(41.55574300592194 74.37261086945931) bank64131 +64132 POINT(41.58367065966714 73.2087242680564) bank64132 +64133 POINT(40.21396430671688 73.33484067083766) bank64133 +64134 POINT(40.036355754015666 73.93513116984533) bank64134 +64135 POINT(41.244934012095264 73.60878957522924) bank64135 +64136 POINT(39.75047592703932 73.40181686880659) bank64136 +64137 POINT(40.90158963585911 73.09860428070516) bank64137 +64138 POINT(40.58564533804074 74.80925063804267) bank64138 +64139 POINT(41.214711346840424 73.78997873889864) bank64139 +64140 POINT(41.13253201570829 73.06675889983737) bank64140 +64141 POINT(39.768677325751575 74.35488021283781) bank64141 +64142 POINT(40.38032779504963 74.94079811026201) bank64142 +64143 POINT(39.77011002944411 73.58595155928462) bank64143 +64144 POINT(40.63803395536945 73.92069437826471) bank64144 +64145 POINT(40.24469109382645 73.47454706693487) bank64145 +64146 POINT(40.615565652143374 74.60847511515425) bank64146 +64147 POINT(40.34492686634837 74.11472186135467) bank64147 +64148 POINT(40.661538584552595 73.80235778432483) bank64148 +64149 POINT(41.151579210220675 74.35139615653617) bank64149 +64150 POINT(39.96244204178861 73.4720676467387) bank64150 +64151 POINT(39.93993661469347 73.23349339710464) bank64151 +64152 POINT(39.86938807579569 74.494580492712) bank64152 +64153 POINT(40.46360305989624 74.76199723206963) bank64153 +64154 POINT(40.62254226794718 74.89327196640056) bank64154 +64155 POINT(40.300806100336956 74.35154588008494) bank64155 +64156 POINT(40.15279802576234 74.38844985105983) bank64156 +64157 POINT(41.269955152679856 73.6856448707444) bank64157 +64158 POINT(39.86804766303299 73.92121812721457) bank64158 +64159 POINT(40.39227255074857 73.4680683716507) bank64159 +64160 POINT(41.21531412921398 74.37295272688173) bank64160 +64161 POINT(41.60617980899504 74.27449932011272) bank64161 +64162 POINT(40.86815641647711 73.09487328462413) bank64162 +64163 POINT(39.96915230499065 73.32087655350854) bank64163 +64164 POINT(41.071390036143505 74.32190167537225) bank64164 +64165 POINT(39.85320249645199 74.49000687143445) bank64165 +64166 POINT(39.832379549793814 74.23273487065188) bank64166 +64167 POINT(41.578772857523994 73.7337610430095) bank64167 +64168 POINT(40.966122949842024 74.67510532730451) bank64168 +64169 POINT(40.269131282391434 73.57774630856473) bank64169 +64170 POINT(40.802943082924074 73.1876119909859) bank64170 +64171 POINT(40.55054896122248 73.55655451200442) bank64171 +64172 POINT(39.9068165138558 74.7420720313587) bank64172 +64173 POINT(39.82970410272634 74.25044156189209) bank64173 +64174 POINT(41.13548557235465 74.5549915397767) bank64174 +64175 POINT(41.24946561319058 74.19379529619339) bank64175 +64176 POINT(39.73925198205647 73.20236033912654) bank64176 +64177 POINT(41.33688769854641 73.4575294097328) bank64177 +64178 POINT(41.695379699147345 74.05859510628855) bank64178 +64179 POINT(41.46773850040824 74.91488459578244) bank64179 +64180 POINT(41.58954253432959 74.41180205972367) bank64180 +64181 POINT(39.72145981076024 73.7816175516678) bank64181 +64182 POINT(40.883616720184364 74.98593026357163) bank64182 +64183 POINT(41.536114639076985 73.64842635661034) bank64183 +64184 POINT(41.683456957735366 73.60668187074432) bank64184 +64185 POINT(41.078530090395674 73.9888458708221) bank64185 +64186 POINT(40.04584080444829 73.23109755928425) bank64186 +64187 POINT(41.052364159060566 73.68197980982123) bank64187 +64188 POINT(41.4586560123035 74.52855625753742) bank64188 +64189 POINT(41.52339307344248 74.54486277631916) bank64189 +64190 POINT(39.85207485306184 73.4230690621051) bank64190 +64191 POINT(40.877920660924694 73.93274464050404) bank64191 +64192 POINT(40.3956354847063 73.57533112636723) bank64192 +64193 POINT(40.130232151247434 74.74943797960557) bank64193 +64194 POINT(40.53188753843422 74.77231801144853) bank64194 +64195 POINT(40.314124949315996 74.8458382732621) bank64195 +64196 POINT(40.46363094988326 74.682785104397) bank64196 +64197 POINT(39.804215595740516 73.33923586318477) bank64197 +64198 POINT(40.72484763579312 73.53216126274977) bank64198 +64199 POINT(39.931203179412165 73.86409573589404) bank64199 +64200 POINT(40.377362609928525 73.44642963373522) bank64200 +64201 POINT(40.30891680670056 74.27631615698566) bank64201 +64202 POINT(41.398978445708615 74.56935000876811) bank64202 +64203 POINT(40.53717675308014 73.65200840418508) bank64203 +64204 POINT(41.11759124724996 73.60147321163946) bank64204 +64205 POINT(41.579324418061496 73.96556962571225) bank64205 +64206 POINT(40.127045637023706 74.3684445100596) bank64206 +64207 POINT(39.8465497854569 74.61589377583175) bank64207 +64208 POINT(41.46223384343295 73.19923408902945) bank64208 +64209 POINT(40.041108075672554 73.0347225153063) bank64209 +64210 POINT(40.706310381113056 73.8328613884156) bank64210 +64211 POINT(40.41694634324767 73.20454205582743) bank64211 +64212 POINT(40.77429857473744 74.52821745866646) bank64212 +64213 POINT(40.80702388484749 73.91848521193896) bank64213 +64214 POINT(40.395463512224794 74.65798546412732) bank64214 +64215 POINT(41.3032054108176 74.06206701402581) bank64215 +64216 POINT(40.67148633225404 73.27349571786235) bank64216 +64217 POINT(40.35566081067558 74.83092127822789) bank64217 +64218 POINT(41.07793004976632 73.8685107714999) bank64218 +64219 POINT(40.14443396955245 73.35382702477764) bank64219 +64220 POINT(40.252688358007504 73.32121155093884) bank64220 +64221 POINT(40.329035471615235 74.39364657481504) bank64221 +64222 POINT(40.99028971940711 73.13031285366365) bank64222 +64223 POINT(41.14849005627209 73.81439676998605) bank64223 +64224 POINT(41.24510083981382 73.02137406804064) bank64224 +64225 POINT(39.877039192117 73.92897801598586) bank64225 +64226 POINT(40.616485074933806 75.00394705797623) bank64226 +64227 POINT(40.04690003194479 74.70125235420514) bank64227 +64228 POINT(39.97347786011223 73.91854644512108) bank64228 +64229 POINT(41.540234398435814 74.21585161096688) bank64229 +64230 POINT(41.295977609986515 74.56597822430206) bank64230 +64231 POINT(40.803604092583264 73.14462174543941) bank64231 +64232 POINT(40.20262526284333 73.93123424705945) bank64232 +64233 POINT(39.75754364310855 74.02118970742553) bank64233 +64234 POINT(41.2040771619389 73.9209649464717) bank64234 +64235 POINT(40.81820626607726 74.3478747189652) bank64235 +64236 POINT(41.36909902173228 73.24437256576576) bank64236 +64237 POINT(41.324945738575195 74.97027343547904) bank64237 +64238 POINT(39.94787591529757 73.3544447498616) bank64238 +64239 POINT(40.08377627376076 73.05417770813607) bank64239 +64240 POINT(41.45585682679214 74.82751674153926) bank64240 +64241 POINT(41.1493421100239 73.10163339740642) bank64241 +64242 POINT(41.540853853250205 73.23846636875561) bank64242 +64243 POINT(41.66179878897561 73.30036176044439) bank64243 +64244 POINT(40.254197040837376 73.4557901627163) bank64244 +64245 POINT(41.37319287615163 73.7685896434272) bank64245 +64246 POINT(41.40793872656393 73.21447706371877) bank64246 +64247 POINT(40.267364489337446 73.76290049963276) bank64247 +64248 POINT(40.348692669928035 73.24655612926533) bank64248 +64249 POINT(39.950930386336594 73.28908812454955) bank64249 +64250 POINT(41.29533801969419 74.12119583809049) bank64250 +64251 POINT(40.472426965149246 74.17385410593057) bank64251 +64252 POINT(40.84576890759314 74.08011623961917) bank64252 +64253 POINT(40.06750465355062 73.24868835186189) bank64253 +64254 POINT(40.338729736635734 74.91835814675399) bank64254 +64255 POINT(41.27973573508196 73.53991506305834) bank64255 +64256 POINT(41.050495964769354 74.22658503272602) bank64256 +64257 POINT(40.25795470331847 73.94095108605418) bank64257 +64258 POINT(40.87363654299726 73.90559759585187) bank64258 +64259 POINT(41.66635298330179 73.24895678735643) bank64259 +64260 POINT(41.27542921420727 74.38877840685038) bank64260 +64261 POINT(40.63333595673163 73.92075562164693) bank64261 +64262 POINT(39.833362915772774 73.98013893782125) bank64262 +64263 POINT(39.825031463029966 73.29957606306678) bank64263 +64264 POINT(39.82725693145144 73.0208070774731) bank64264 +64265 POINT(40.92065979215304 74.86457940597145) bank64265 +64266 POINT(41.533836523864515 74.06529816105818) bank64266 +64267 POINT(40.35623371366634 74.54529209842781) bank64267 +64268 POINT(40.26167400147508 73.31497797305997) bank64268 +64269 POINT(40.05775250815907 74.84044218677019) bank64269 +64270 POINT(40.8443064546152 73.35894512899056) bank64270 +64271 POINT(41.26393948789123 74.62566839938961) bank64271 +64272 POINT(40.0515757817275 74.06933267955937) bank64272 +64273 POINT(39.909416080539984 74.50783854266723) bank64273 +64274 POINT(41.4247476496646 74.91115257230184) bank64274 +64275 POINT(40.51489611041827 74.9300871815843) bank64275 +64276 POINT(41.27289405110065 73.13939537495241) bank64276 +64277 POINT(39.999448883507384 73.61689056578578) bank64277 +64278 POINT(40.91096986227675 73.16968452995621) bank64278 +64279 POINT(41.13393458400915 74.42638073323329) bank64279 +64280 POINT(41.47511709128109 73.0223435539033) bank64280 +64281 POINT(40.39613517582104 74.52896687563795) bank64281 +64282 POINT(40.695511594330796 73.26301978083714) bank64282 +64283 POINT(40.2078124357184 73.91275660042993) bank64283 +64284 POINT(40.254277313517235 73.72773510704701) bank64284 +64285 POINT(41.44425148733751 74.79600516740638) bank64285 +64286 POINT(41.69124749979699 74.82418472674185) bank64286 +64287 POINT(41.63102289857563 73.55140959413892) bank64287 +64288 POINT(41.15956856847768 74.89894856737487) bank64288 +64289 POINT(41.421105019539056 74.40656829405421) bank64289 +64290 POINT(40.057173448494545 74.47223443289595) bank64290 +64291 POINT(40.53370457366177 73.87283766443771) bank64291 +64292 POINT(41.281125798785894 73.1272691293196) bank64292 +64293 POINT(40.028457946208995 74.71859193212144) bank64293 +64294 POINT(40.872081254799575 74.91863382972326) bank64294 +64295 POINT(41.274162877179855 73.42697431515867) bank64295 +64296 POINT(40.24448898597742 73.70712120545731) bank64296 +64297 POINT(40.3459069838039 73.76955760147125) bank64297 +64298 POINT(40.518376439868604 73.0858195046815) bank64298 +64299 POINT(39.98957731152372 73.19333574051396) bank64299 +64300 POINT(40.50573314252427 74.9737870160774) bank64300 +64301 POINT(41.699410140048855 73.13082771233901) bank64301 +64302 POINT(39.93984649053126 73.57618706830621) bank64302 +64303 POINT(40.712855489013016 73.48675649211367) bank64303 +64304 POINT(40.11265681280252 74.27836411385118) bank64304 +64305 POINT(41.15020861655374 74.08499213123412) bank64305 +64306 POINT(41.05112503109629 73.72193785621477) bank64306 +64307 POINT(41.01718419010639 74.18094138655361) bank64307 +64308 POINT(40.66112344904132 74.11794546368807) bank64308 +64309 POINT(40.99697166076228 74.731919640292) bank64309 +64310 POINT(41.30593283889097 73.21611281780426) bank64310 +64311 POINT(40.46618587123019 74.73439696987359) bank64311 +64312 POINT(40.10007140788004 74.00585460977778) bank64312 +64313 POINT(40.42660835949209 74.74211560181406) bank64313 +64314 POINT(40.2395307504419 73.59971628800312) bank64314 +64315 POINT(40.333368942300076 74.8652386340117) bank64315 +64316 POINT(40.074221538583345 73.39283210973082) bank64316 +64317 POINT(40.135827687721644 73.704417670305) bank64317 +64318 POINT(41.01409749580194 74.59446571515511) bank64318 +64319 POINT(39.8877410641394 73.40092345442484) bank64319 +64320 POINT(41.70700378888024 73.11643940700135) bank64320 +64321 POINT(41.38628408625509 73.2093320131877) bank64321 +64322 POINT(39.77422475152614 74.84885023214535) bank64322 +64323 POINT(41.10035765287033 73.69857980030665) bank64323 +64324 POINT(41.6587343411614 73.57721126258346) bank64324 +64325 POINT(40.106494481335965 73.42013086722953) bank64325 +64326 POINT(41.45537043216848 74.82884375068366) bank64326 +64327 POINT(41.040942891558835 74.06083412639309) bank64327 +64328 POINT(41.01819859193671 74.11640303959864) bank64328 +64329 POINT(40.35472904544579 73.17848548848264) bank64329 +64330 POINT(39.77845229245686 73.57652158342492) bank64330 +64331 POINT(40.92994081460265 73.45039425954369) bank64331 +64332 POINT(41.13325915290808 73.32037852020689) bank64332 +64333 POINT(40.88918666593538 74.52233931499566) bank64333 +64334 POINT(40.793277667543634 74.36581807241707) bank64334 +64335 POINT(40.0211650777953 74.50443684708698) bank64335 +64336 POINT(39.811754950328904 74.33749003569312) bank64336 +64337 POINT(41.10937566832838 74.10408272632351) bank64337 +64338 POINT(41.07769686014279 73.96640962345977) bank64338 +64339 POINT(41.42247016411093 74.06270007837425) bank64339 +64340 POINT(39.82791253529747 73.82948250631743) bank64340 +64341 POINT(40.63362632944645 73.61923267993488) bank64341 +64342 POINT(40.298978088271696 73.96549100090328) bank64342 +64343 POINT(40.86644195040275 73.13767320945136) bank64343 +64344 POINT(40.61732737909709 73.09065333065679) bank64344 +64345 POINT(40.95683395318059 73.42298137453234) bank64345 +64346 POINT(41.4380933002301 73.62040156797434) bank64346 +64347 POINT(40.678841140001545 74.793586841737) bank64347 +64348 POINT(39.97804478158403 74.27812266432875) bank64348 +64349 POINT(40.83708442462719 73.88904150598577) bank64349 +64350 POINT(41.34444601727678 74.38239894509825) bank64350 +64351 POINT(40.908231108437825 74.03148823616071) bank64351 +64352 POINT(41.33684548908047 73.50543444639145) bank64352 +64353 POINT(39.92816838068717 73.73122853117651) bank64353 +64354 POINT(39.97552949291819 73.45594895911454) bank64354 +64355 POINT(41.403866122132754 74.2144186093235) bank64355 +64356 POINT(40.90178734376791 74.4582934997509) bank64356 +64357 POINT(39.872976110114514 73.1639877440723) bank64357 +64358 POINT(40.499042015432565 74.60606670258096) bank64358 +64359 POINT(41.469263177843146 74.51678144169975) bank64359 +64360 POINT(40.35933792289332 74.57407541038378) bank64360 +64361 POINT(41.31365227345184 74.70067620613462) bank64361 +64362 POINT(40.672826326474976 74.99646687127215) bank64362 +64363 POINT(40.11771876237811 74.53449666195246) bank64363 +64364 POINT(41.55493858610159 74.73709929115613) bank64364 +64365 POINT(40.6310121211386 74.54632546376179) bank64365 +64366 POINT(41.288101156041705 73.70251201199594) bank64366 +64367 POINT(39.83676932445579 73.06098000792561) bank64367 +64368 POINT(40.63665402364047 74.95869023277596) bank64368 +64369 POINT(39.92041400904317 74.0936169473528) bank64369 +64370 POINT(40.242550199239254 74.68120606354634) bank64370 +64371 POINT(40.22904793497018 74.30175123387542) bank64371 +64372 POINT(41.44772683222682 73.78650056065305) bank64372 +64373 POINT(40.42717676035966 73.59015152473096) bank64373 +64374 POINT(41.56918320090445 74.6713745462592) bank64374 +64375 POINT(41.589333915730656 73.56514189115444) bank64375 +64376 POINT(40.700344911187024 74.50318528891721) bank64376 +64377 POINT(40.17202252402842 73.54535224985703) bank64377 +64378 POINT(40.77890963367793 74.14147470555606) bank64378 +64379 POINT(40.408243238869666 73.01237023156955) bank64379 +64380 POINT(40.608883672382504 73.52973051959017) bank64380 +64381 POINT(40.76905067383909 73.62142428192254) bank64381 +64382 POINT(41.04971065177819 74.65292485133885) bank64382 +64383 POINT(40.211635484886635 74.42155387591315) bank64383 +64384 POINT(40.391065321038084 74.63339965026022) bank64384 +64385 POINT(40.858715607956206 73.32784896407114) bank64385 +64386 POINT(41.06251970791287 73.056701577114) bank64386 +64387 POINT(39.99056319465638 73.77310739433875) bank64387 +64388 POINT(40.28263047790663 74.49332972060431) bank64388 +64389 POINT(40.04571782268758 74.55998626815898) bank64389 +64390 POINT(41.64256836338679 74.62046734334096) bank64390 +64391 POINT(41.04874705340227 74.62458623121302) bank64391 +64392 POINT(41.458844350991995 74.01634497073744) bank64392 +64393 POINT(41.19367959767859 73.84376780073748) bank64393 +64394 POINT(41.00021898924012 73.33038676009822) bank64394 +64395 POINT(40.46327404272684 73.26859046178691) bank64395 +64396 POINT(41.52879390608451 73.26291591015469) bank64396 +64397 POINT(40.08221556470018 73.33392005660829) bank64397 +64398 POINT(40.388374712240655 73.83892602262887) bank64398 +64399 POINT(41.55854938393721 74.94493702856359) bank64399 +64400 POINT(40.05006167136014 74.30871245950254) bank64400 +64401 POINT(41.211568795213665 73.34738807979606) bank64401 +64402 POINT(41.11027114454718 74.75999015803103) bank64402 +64403 POINT(40.053504465236045 73.66808417910948) bank64403 +64404 POINT(40.0259089446631 74.11008975911075) bank64404 +64405 POINT(41.54204489604351 74.07802723354285) bank64405 +64406 POINT(40.718695207296 73.85579145110592) bank64406 +64407 POINT(41.577874741892 73.49290998704144) bank64407 +64408 POINT(40.55219471696856 73.60499650816277) bank64408 +64409 POINT(39.7467888299233 74.345958113345) bank64409 +64410 POINT(40.79641634525358 73.51658610284534) bank64410 +64411 POINT(41.11420518592372 74.09365058199796) bank64411 +64412 POINT(40.71948478199572 74.67162044358051) bank64412 +64413 POINT(41.17267053365422 73.77383262272852) bank64413 +64414 POINT(39.72717168923242 74.14842826761989) bank64414 +64415 POINT(41.61936798108427 74.78592998045731) bank64415 +64416 POINT(40.86439463720349 74.19645966130129) bank64416 +64417 POINT(40.94474104203197 73.33137535604614) bank64417 +64418 POINT(41.550022914554646 73.8016342707114) bank64418 +64419 POINT(40.039466823822 73.7640891991251) bank64419 +64420 POINT(40.5124330627722 73.9081989891147) bank64420 +64421 POINT(41.13514145825316 73.3560760796777) bank64421 +64422 POINT(41.053471152941206 74.86551592072843) bank64422 +64423 POINT(40.840751168938574 73.4713160336487) bank64423 +64424 POINT(41.11844880439139 73.62802163467455) bank64424 +64425 POINT(39.83208902254665 73.2540115892617) bank64425 +64426 POINT(40.83259642754836 74.50903967748613) bank64426 +64427 POINT(40.43413816276494 73.25666902085966) bank64427 +64428 POINT(41.50773505354054 73.73620506971308) bank64428 +64429 POINT(41.13680004912759 73.69185481734443) bank64429 +64430 POINT(40.9815993243948 73.82486887277489) bank64430 +64431 POINT(40.72347102734785 73.26776493259044) bank64431 +64432 POINT(41.11910539044698 74.53178565942653) bank64432 +64433 POINT(41.52694038614637 73.74697031025242) bank64433 +64434 POINT(40.277000999817616 73.82556730462072) bank64434 +64435 POINT(40.16831365472385 74.37808407371831) bank64435 +64436 POINT(40.033798275645886 74.21594776760868) bank64436 +64437 POINT(41.30790707592885 74.37312428805755) bank64437 +64438 POINT(40.98087776045391 74.20676267138876) bank64438 +64439 POINT(39.82914235119274 73.75694203962615) bank64439 +64440 POINT(40.568154750893925 73.25674420246158) bank64440 +64441 POINT(40.91106137880113 74.2659977957414) bank64441 +64442 POINT(40.386728474632086 74.14926222099099) bank64442 +64443 POINT(41.086737445424774 74.8111689896914) bank64443 +64444 POINT(39.9703290828374 73.70829979875066) bank64444 +64445 POINT(40.107414383336426 74.93514618538251) bank64445 +64446 POINT(41.09940042824798 73.5233068937092) bank64446 +64447 POINT(40.33931428628634 73.28210944681103) bank64447 +64448 POINT(41.366542140520735 74.55059016950045) bank64448 +64449 POINT(41.70113583629604 74.35051699447311) bank64449 +64450 POINT(41.39079849878415 73.82434209044821) bank64450 +64451 POINT(40.55536323821796 74.04788826360455) bank64451 +64452 POINT(39.736841241978084 73.79126766918944) bank64452 +64453 POINT(41.65161624005558 74.24977620309635) bank64453 +64454 POINT(40.09279756749319 73.20344255982991) bank64454 +64455 POINT(40.692063442765075 73.917177976162) bank64455 +64456 POINT(40.14018232218841 73.67812416990421) bank64456 +64457 POINT(41.578943706378325 73.6786051280125) bank64457 +64458 POINT(41.66201007711837 73.98012222963501) bank64458 +64459 POINT(40.75936036881787 74.57772775506302) bank64459 +64460 POINT(40.127011595701816 73.11038749578533) bank64460 +64461 POINT(40.50536951898946 73.9159834493064) bank64461 +64462 POINT(41.01471743613608 73.34222964536406) bank64462 +64463 POINT(40.69292182461013 73.02535966733896) bank64463 +64464 POINT(41.68689675957115 73.38137536575448) bank64464 +64465 POINT(40.808795129125976 74.45768709891813) bank64465 +64466 POINT(40.77457752502326 73.38301693742262) bank64466 +64467 POINT(40.94141072271129 74.23366452455285) bank64467 +64468 POINT(40.56857778336193 74.64569974676343) bank64468 +64469 POINT(39.732068712049916 73.33392182940909) bank64469 +64470 POINT(40.2054871240146 74.46260955340158) bank64470 +64471 POINT(41.029086415864526 73.2177545050835) bank64471 +64472 POINT(40.787727023117974 74.58677289551139) bank64472 +64473 POINT(40.058614302575705 74.08196680148097) bank64473 +64474 POINT(40.25879192948456 74.91175094668915) bank64474 +64475 POINT(39.862691253548064 74.5799683702555) bank64475 +64476 POINT(40.039635493522596 74.75235742969645) bank64476 +64477 POINT(40.342671153583005 74.0031772799278) bank64477 +64478 POINT(40.27167466858165 73.77142330210279) bank64478 +64479 POINT(41.24311783038784 73.6220711307852) bank64479 +64480 POINT(40.358135612939726 74.22449005632618) bank64480 +64481 POINT(40.52451067103935 74.5976445909004) bank64481 +64482 POINT(39.971081962046775 74.86387609926884) bank64482 +64483 POINT(40.713240841846314 73.1250017157882) bank64483 +64484 POINT(40.86448051181464 74.24653786644124) bank64484 +64485 POINT(39.77332799909079 73.76764372071374) bank64485 +64486 POINT(39.878186213878664 73.2311485230749) bank64486 +64487 POINT(40.811522519762256 73.57321359942674) bank64487 +64488 POINT(40.79154288114468 74.33362367617273) bank64488 +64489 POINT(39.82695167344337 74.83596861127546) bank64489 +64490 POINT(41.31760611289134 73.94094106393591) bank64490 +64491 POINT(40.37838725994396 74.69363244627402) bank64491 +64492 POINT(41.54469865805798 73.43561041462347) bank64492 +64493 POINT(39.92642115614976 74.3943488574696) bank64493 +64494 POINT(41.63629176294607 73.15354609022394) bank64494 +64495 POINT(39.9424270586514 74.33651803691991) bank64495 +64496 POINT(41.42929803066258 74.4061287032857) bank64496 +64497 POINT(41.17552183678112 74.60650271041003) bank64497 +64498 POINT(39.96255450120684 73.8074686211552) bank64498 +64499 POINT(41.482571026314645 74.16872165879823) bank64499 +64500 POINT(40.748570003530915 73.68255329401391) bank64500 +64501 POINT(40.302738817976866 73.02688448263756) bank64501 +64502 POINT(39.89556979659819 74.37682851576459) bank64502 +64503 POINT(41.68166300804752 73.53053671514773) bank64503 +64504 POINT(40.36996924026147 74.35057463991234) bank64504 +64505 POINT(40.64309097208949 74.88286320000803) bank64505 +64506 POINT(41.39086514407169 73.17689254926061) bank64506 +64507 POINT(40.65535171361263 74.32844495279386) bank64507 +64508 POINT(41.14792297283153 74.13494602497437) bank64508 +64509 POINT(40.73085110161269 74.26019135450316) bank64509 +64510 POINT(40.252046298649304 74.19936808924064) bank64510 +64511 POINT(40.29300394424496 73.96634176448876) bank64511 +64512 POINT(39.73128490626501 73.07054611153154) bank64512 +64513 POINT(41.45763891625379 74.38947332169141) bank64513 +64514 POINT(41.52189351803801 74.46340187004847) bank64514 +64515 POINT(41.335638300583945 73.20761847204628) bank64515 +64516 POINT(39.79969230192945 73.57278529648774) bank64516 +64517 POINT(40.360571511165986 74.63870693665363) bank64517 +64518 POINT(41.60106639832698 74.12129677310868) bank64518 +64519 POINT(40.86223918781118 73.75645136074077) bank64519 +64520 POINT(41.44165050362991 74.00872319512277) bank64520 +64521 POINT(40.20357649010942 73.23484443336321) bank64521 +64522 POINT(41.010887642267406 74.28521540449822) bank64522 +64523 POINT(41.005541939173526 73.05293216112642) bank64523 +64524 POINT(40.423772826565 73.46460209916046) bank64524 +64525 POINT(40.78932272298611 73.22884846193566) bank64525 +64526 POINT(40.93951696878902 74.83784254353546) bank64526 +64527 POINT(40.21312954008481 74.98892841601204) bank64527 +64528 POINT(40.901859940752956 74.87197971681124) bank64528 +64529 POINT(40.97453164030495 73.37152174390928) bank64529 +64530 POINT(40.32867453777687 73.27030305388922) bank64530 +64531 POINT(40.75496470118131 74.33816432868169) bank64531 +64532 POINT(40.090194955834164 74.32893508702855) bank64532 +64533 POINT(40.34650081640769 74.56757520240275) bank64533 +64534 POINT(41.5276691233671 74.76147553052482) bank64534 +64535 POINT(41.62921476865208 74.0103590055457) bank64535 +64536 POINT(39.881510221406955 74.37022812146196) bank64536 +64537 POINT(41.112316054428334 73.79179208134954) bank64537 +64538 POINT(40.76337302734692 73.46367114583524) bank64538 +64539 POINT(40.65183671103186 73.95331545568634) bank64539 +64540 POINT(40.582281947493435 74.33222481227457) bank64540 +64541 POINT(41.22892224889272 74.46101067678681) bank64541 +64542 POINT(41.65110760453741 74.35943634292333) bank64542 +64543 POINT(40.80545560068037 74.65716566211081) bank64543 +64544 POINT(40.805066080311036 74.58554912895889) bank64544 +64545 POINT(40.15665390602313 74.77837319185252) bank64545 +64546 POINT(39.76323589438905 74.98403212901783) bank64546 +64547 POINT(40.50827038857135 73.68771753676322) bank64547 +64548 POINT(41.02562294817957 74.1938773196051) bank64548 +64549 POINT(41.63871760241266 73.56470929504202) bank64549 +64550 POINT(41.650534693581875 74.48644493661763) bank64550 +64551 POINT(40.70867851009626 73.6185476131046) bank64551 +64552 POINT(40.93084096483769 73.04509081124266) bank64552 +64553 POINT(40.620688942349275 73.32271782092124) bank64553 +64554 POINT(40.923158862102504 74.83768308299801) bank64554 +64555 POINT(40.308779199074955 73.48813245032031) bank64555 +64556 POINT(40.79923377062863 74.2717328067939) bank64556 +64557 POINT(40.966299191409256 74.19050392037093) bank64557 +64558 POINT(40.73023612526671 74.0870511945656) bank64558 +64559 POINT(41.5831316561204 74.56120462923604) bank64559 +64560 POINT(40.698517551717906 73.71149971581826) bank64560 +64561 POINT(40.067193118820086 73.25203412018557) bank64561 +64562 POINT(41.07435151178063 74.14778045720905) bank64562 +64563 POINT(41.27606894040527 73.82408822905929) bank64563 +64564 POINT(41.04951389383914 74.323850497819) bank64564 +64565 POINT(40.55979643326279 74.50126728552257) bank64565 +64566 POINT(40.275014518573435 73.80083869355558) bank64566 +64567 POINT(40.37886798485677 73.58746346830077) bank64567 +64568 POINT(39.77502588681186 73.75728025040233) bank64568 +64569 POINT(40.47512129758678 73.1996947616933) bank64569 +64570 POINT(40.17778649235853 73.70620164586013) bank64570 +64571 POINT(39.71628311827643 74.43006136739372) bank64571 +64572 POINT(40.49781709948212 74.31402565259184) bank64572 +64573 POINT(40.539573356585926 74.56103787349019) bank64573 +64574 POINT(41.442651424060294 73.70275734929983) bank64574 +64575 POINT(40.01087696378685 74.81670024032405) bank64575 +64576 POINT(39.71822765798334 74.51582165997658) bank64576 +64577 POINT(41.17937456698823 74.93000708207119) bank64577 +64578 POINT(40.63099059582802 74.88654247949393) bank64578 +64579 POINT(41.13492979535874 73.79131686659093) bank64579 +64580 POINT(41.33410005209903 73.82303084333645) bank64580 +64581 POINT(40.810006348750825 74.44030554695561) bank64581 +64582 POINT(40.8311447322672 74.5514400635105) bank64582 +64583 POINT(41.16451912541581 74.98691043220546) bank64583 +64584 POINT(41.38113889520686 73.13913371911309) bank64584 +64585 POINT(40.24535584017064 74.8476060024812) bank64585 +64586 POINT(40.41623411690089 73.49292183140457) bank64586 +64587 POINT(40.840478898230266 73.62508993588757) bank64587 +64588 POINT(40.29056788897521 73.22578007344994) bank64588 +64589 POINT(40.72359342810318 74.24133315028142) bank64589 +64590 POINT(41.339413478647515 73.93810592406518) bank64590 +64591 POINT(41.303566743555756 73.57857165231522) bank64591 +64592 POINT(39.941607962679946 74.23630914731487) bank64592 +64593 POINT(41.26752373242247 74.71567657316349) bank64593 +64594 POINT(39.80590974580258 74.04957757385796) bank64594 +64595 POINT(41.38918127256548 73.05126954614249) bank64595 +64596 POINT(40.725399812348826 73.37193657542235) bank64596 +64597 POINT(40.14207923238019 73.81548225635323) bank64597 +64598 POINT(40.749016284192486 74.81214005281751) bank64598 +64599 POINT(39.73798448838837 74.1454865872523) bank64599 +64600 POINT(40.5831355520485 74.02737866428842) bank64600 +64601 POINT(40.87254380577456 74.59234841295077) bank64601 +64602 POINT(40.315981097793056 73.72118183442782) bank64602 +64603 POINT(40.14433028398985 74.7345528645432) bank64603 +64604 POINT(39.905054320620785 74.29161419257206) bank64604 +64605 POINT(40.4570285444894 73.08103696249209) bank64605 +64606 POINT(40.32985656426879 73.57976916840309) bank64606 +64607 POINT(39.93697800202358 74.69345304471531) bank64607 +64608 POINT(40.47577105432502 74.56060351771991) bank64608 +64609 POINT(41.58559935349529 73.41276418970617) bank64609 +64610 POINT(40.686889580303095 73.10702896436764) bank64610 +64611 POINT(40.508729647069075 73.80552495732083) bank64611 +64612 POINT(40.37689626099854 73.66377449683534) bank64612 +64613 POINT(39.76119865533744 73.69390555573173) bank64613 +64614 POINT(40.30949158614409 74.84519737731547) bank64614 +64615 POINT(39.8706227981267 74.45231545125708) bank64615 +64616 POINT(40.736052121719744 74.97818208321767) bank64616 +64617 POINT(40.83191449890804 73.4627367758944) bank64617 +64618 POINT(41.28388220117716 73.24611252665899) bank64618 +64619 POINT(41.39822075690356 74.940524127411) bank64619 +64620 POINT(41.364089816640224 74.94296002849462) bank64620 +64621 POINT(40.28156023440806 74.93064432913054) bank64621 +64622 POINT(40.85758051473331 73.38952961404415) bank64622 +64623 POINT(41.0691699539782 74.49651355488274) bank64623 +64624 POINT(40.21895467426444 73.10686089125392) bank64624 +64625 POINT(40.439405485216575 74.24042637598176) bank64625 +64626 POINT(41.35808883857567 73.38111979888029) bank64626 +64627 POINT(41.05068625995564 73.7662189833831) bank64627 +64628 POINT(41.42460929909456 74.04178566943675) bank64628 +64629 POINT(41.275305371476456 73.3794482307474) bank64629 +64630 POINT(41.53414659587339 74.9752401751718) bank64630 +64631 POINT(41.11951238728464 73.95507490909519) bank64631 +64632 POINT(41.55857761537543 74.58502279385598) bank64632 +64633 POINT(40.28701078800943 73.88880280634292) bank64633 +64634 POINT(41.42451096678465 74.17391512084362) bank64634 +64635 POINT(39.967396632580204 73.49956265040953) bank64635 +64636 POINT(40.43274452766634 73.87283129232522) bank64636 +64637 POINT(41.663986320747874 73.26832192352313) bank64637 +64638 POINT(41.51640375862909 73.7962267782156) bank64638 +64639 POINT(40.538054779895646 73.76422154531534) bank64639 +64640 POINT(40.272456717306156 73.70686335747361) bank64640 +64641 POINT(40.28602351396663 73.02306426600252) bank64641 +64642 POINT(41.55551434011114 74.9569648758862) bank64642 +64643 POINT(41.517859394707465 73.3972696557046) bank64643 +64644 POINT(41.193923147244874 73.40191787326702) bank64644 +64645 POINT(39.79622638219196 74.88824918887443) bank64645 +64646 POINT(40.24398855255871 74.22833658080071) bank64646 +64647 POINT(41.11188639118123 74.79969603534124) bank64647 +64648 POINT(40.35658394955514 73.71220029010244) bank64648 +64649 POINT(40.12517926965923 74.85239780501513) bank64649 +64650 POINT(39.85699735145636 73.49868794365452) bank64650 +64651 POINT(39.782612845633565 73.65431580605137) bank64651 +64652 POINT(41.7072496894782 73.84008107871813) bank64652 +64653 POINT(40.38357399950917 74.8024765451279) bank64653 +64654 POINT(40.064077159586326 74.38549200458847) bank64654 +64655 POINT(40.732490843814524 73.94861776621086) bank64655 +64656 POINT(40.803307668905376 73.90262806326535) bank64656 +64657 POINT(40.17639956175706 73.77813048245049) bank64657 +64658 POINT(41.4937300466367 74.21825163812869) bank64658 +64659 POINT(39.89954808389068 74.2458449349376) bank64659 +64660 POINT(40.27732591586195 74.69368885007498) bank64660 +64661 POINT(40.5488802250959 74.24737076919212) bank64661 +64662 POINT(40.09968024514015 74.14788512911817) bank64662 +64663 POINT(39.80547498739866 74.93639140424129) bank64663 +64664 POINT(41.29129540254926 74.34751587622888) bank64664 +64665 POINT(41.079278044544516 74.62415027705345) bank64665 +64666 POINT(41.4810232793992 73.66800332259986) bank64666 +64667 POINT(40.27553433566513 73.44371463962636) bank64667 +64668 POINT(41.24754409061649 74.91994326142056) bank64668 +64669 POINT(40.82786914819131 74.4780909532406) bank64669 +64670 POINT(40.569200667784244 73.48937997317421) bank64670 +64671 POINT(40.21030500962327 73.52605357605945) bank64671 +64672 POINT(41.51698414157012 74.46238708692428) bank64672 +64673 POINT(41.35391791186888 74.56102435445334) bank64673 +64674 POINT(40.122036967446405 73.99757871100698) bank64674 +64675 POINT(39.77624276953461 74.25824511386008) bank64675 +64676 POINT(40.97974260021629 73.97489928047465) bank64676 +64677 POINT(41.50698694095 74.37389477782814) bank64677 +64678 POINT(40.72129440267534 74.86356194399855) bank64678 +64679 POINT(40.37965912242831 73.87785726455282) bank64679 +64680 POINT(40.65218419679146 73.2599729984106) bank64680 +64681 POINT(40.51416543919427 73.4742199738438) bank64681 +64682 POINT(41.540744411874556 74.94938323664677) bank64682 +64683 POINT(40.585502630299615 73.9876049239944) bank64683 +64684 POINT(40.18044787892236 74.82856069132056) bank64684 +64685 POINT(40.05640771268042 73.05215825298241) bank64685 +64686 POINT(40.787798434545564 73.94954649557118) bank64686 +64687 POINT(41.39390104174028 74.82238962870848) bank64687 +64688 POINT(40.042501268804905 73.6709491751126) bank64688 +64689 POINT(41.048450436141586 73.88953305524582) bank64689 +64690 POINT(39.79287441183126 74.29829428819949) bank64690 +64691 POINT(40.38469957887608 73.38464534448833) bank64691 +64692 POINT(40.734221320167755 74.75298843491072) bank64692 +64693 POINT(40.738432129965986 74.67718775779612) bank64693 +64694 POINT(39.92140770047784 73.00762029030554) bank64694 +64695 POINT(41.35157635144624 74.51088252393247) bank64695 +64696 POINT(40.341362199684106 73.57314818923736) bank64696 +64697 POINT(40.9804292872188 74.28907679590704) bank64697 +64698 POINT(39.97755576732472 73.11365625062858) bank64698 +64699 POINT(40.16013150135947 73.50416645232072) bank64699 +64700 POINT(40.85813123070335 74.39179592025495) bank64700 +64701 POINT(39.730608970782484 73.70454609283915) bank64701 +64702 POINT(40.83841214176593 73.4751565784014) bank64702 +64703 POINT(40.844769172291485 74.41209833689344) bank64703 +64704 POINT(39.83890079598932 73.24052962996919) bank64704 +64705 POINT(40.547345288446394 74.40980617733628) bank64705 +64706 POINT(40.73093150860739 74.80737173597859) bank64706 +64707 POINT(40.295201411783474 74.41330163320158) bank64707 +64708 POINT(39.88571702791939 74.35805720550795) bank64708 +64709 POINT(41.19040036587522 74.19234917860817) bank64709 +64710 POINT(41.36899431375193 73.21587563980113) bank64710 +64711 POINT(41.09314446906769 74.79895068056521) bank64711 +64712 POINT(41.19739668819178 74.80424502565931) bank64712 +64713 POINT(40.1637274172668 73.59752039411346) bank64713 +64714 POINT(41.26851681421499 73.49672425538182) bank64714 +64715 POINT(40.132668963586696 74.3952069196809) bank64715 +64716 POINT(39.78539211743054 74.13828365500655) bank64716 +64717 POINT(40.73412863888821 74.36507741272057) bank64717 +64718 POINT(40.39014120104363 74.40973212650815) bank64718 +64719 POINT(40.45699618834385 73.97601572029497) bank64719 +64720 POINT(40.164878061193804 74.04041665635228) bank64720 +64721 POINT(40.468712485084474 74.79749692189993) bank64721 +64722 POINT(41.213604226394565 73.88359670020064) bank64722 +64723 POINT(41.15439632102961 74.06229745944465) bank64723 +64724 POINT(39.981113811660705 74.99672238258388) bank64724 +64725 POINT(40.314271168829045 73.2547920287669) bank64725 +64726 POINT(39.776646990960636 73.5461988341974) bank64726 +64727 POINT(40.32775757960615 74.89464460661222) bank64727 +64728 POINT(40.25938769959297 74.12067719166916) bank64728 +64729 POINT(41.5359277416079 73.7806367259699) bank64729 +64730 POINT(41.335059404659525 73.27450450521675) bank64730 +64731 POINT(39.81260327284079 74.72499213140813) bank64731 +64732 POINT(40.484043676094565 73.89613595123465) bank64732 +64733 POINT(40.00518161963786 74.8375639492819) bank64733 +64734 POINT(41.50633722659365 73.34882133186517) bank64734 +64735 POINT(40.41171498606169 74.47885339995362) bank64735 +64736 POINT(40.33518066742965 73.4452625304411) bank64736 +64737 POINT(40.66133000674812 74.49120023381053) bank64737 +64738 POINT(39.88617451772476 73.6302090026392) bank64738 +64739 POINT(40.70948515274854 74.17248128678018) bank64739 +64740 POINT(40.741390955968555 73.95659517513052) bank64740 +64741 POINT(39.920633392849254 73.39854697005504) bank64741 +64742 POINT(40.634739468779664 73.98656672961985) bank64742 +64743 POINT(41.395843851330106 74.90099181929583) bank64743 +64744 POINT(39.75298435331634 73.6679912752655) bank64744 +64745 POINT(40.76850923617315 74.27715680249248) bank64745 +64746 POINT(40.37585327840479 73.65609137442007) bank64746 +64747 POINT(41.24117521586353 73.76462098247002) bank64747 +64748 POINT(41.647107001975165 74.29789234405243) bank64748 +64749 POINT(39.95551928697248 74.70323013276466) bank64749 +64750 POINT(41.49526406166378 73.47119779528502) bank64750 +64751 POINT(40.088261743553886 73.25473599255916) bank64751 +64752 POINT(39.94335823540698 73.21110355321294) bank64752 +64753 POINT(40.39495399373662 73.70978931250337) bank64753 +64754 POINT(41.55841291724448 73.40704511241317) bank64754 +64755 POINT(41.69034353777211 73.46518321705238) bank64755 +64756 POINT(41.455898985289416 74.40981027749125) bank64756 +64757 POINT(40.004190541604586 73.38958523784812) bank64757 +64758 POINT(39.95765609055013 73.49471765822302) bank64758 +64759 POINT(39.860109748743206 74.81972757934248) bank64759 +64760 POINT(41.20985723712606 74.35657749927488) bank64760 +64761 POINT(41.40690249882231 73.1330180131168) bank64761 +64762 POINT(41.48239884473438 74.33635019685558) bank64762 +64763 POINT(40.74323267218432 74.78813521721027) bank64763 +64764 POINT(41.50633925977916 74.9587603836507) bank64764 +64765 POINT(40.72052604857924 73.29331376357597) bank64765 +64766 POINT(40.04592582705357 74.88228102447671) bank64766 +64767 POINT(40.851684463343545 74.31049975005232) bank64767 +64768 POINT(39.77503377967128 73.13460748204511) bank64768 +64769 POINT(40.33621612754252 73.46324645951874) bank64769 +64770 POINT(40.818345344541314 73.690984563217) bank64770 +64771 POINT(40.588569948994014 73.20123771486783) bank64771 +64772 POINT(40.59235057785261 74.92061213058903) bank64772 +64773 POINT(40.942527724762826 73.19819019974213) bank64773 +64774 POINT(40.93516851125058 74.03079207699992) bank64774 +64775 POINT(40.57933300299058 74.68168691123614) bank64775 +64776 POINT(41.0286035836657 74.2768383287268) bank64776 +64777 POINT(40.278988141440784 74.79011800590862) bank64777 +64778 POINT(40.46722987889959 74.15818517475395) bank64778 +64779 POINT(40.406102907804836 74.98422415991925) bank64779 +64780 POINT(40.28618388627093 74.90007383148496) bank64780 +64781 POINT(39.84005974453468 73.49485676172424) bank64781 +64782 POINT(41.589754924447476 74.31340491600108) bank64782 +64783 POINT(40.665250217282434 74.35067770023713) bank64783 +64784 POINT(40.768837375743104 74.34342227732232) bank64784 +64785 POINT(40.57455814115301 73.71297641618906) bank64785 +64786 POINT(40.17369124625737 74.44466736196584) bank64786 +64787 POINT(41.28973113017929 73.31708941913978) bank64787 +64788 POINT(40.06165862652552 74.7550063676163) bank64788 +64789 POINT(40.068696227484715 73.40857285573225) bank64789 +64790 POINT(41.20858288733251 74.008979365729) bank64790 +64791 POINT(40.078853033895 74.96815982631453) bank64791 +64792 POINT(41.604097556177166 74.51007510798013) bank64792 +64793 POINT(39.84312073203052 74.42101119545157) bank64793 +64794 POINT(39.76162178100471 73.69659600178387) bank64794 +64795 POINT(40.37969797088683 73.3210299204539) bank64795 +64796 POINT(41.638112755150324 73.19820415965857) bank64796 +64797 POINT(40.88988579768423 73.33966852412757) bank64797 +64798 POINT(41.02508691417141 74.12767408598448) bank64798 +64799 POINT(40.81586169040342 74.59849390103892) bank64799 +64800 POINT(40.14867856168 74.1109928959836) bank64800 +64801 POINT(40.61113205039755 74.64472886597059) bank64801 +64802 POINT(39.7228199076696 73.88809100639192) bank64802 +64803 POINT(40.27954251990545 73.29853709635137) bank64803 +64804 POINT(40.249683260132606 73.02397464576416) bank64804 +64805 POINT(40.8740432848929 74.92711536911725) bank64805 +64806 POINT(40.246187632770955 74.66129350303765) bank64806 +64807 POINT(40.19848180136699 73.94723507737247) bank64807 +64808 POINT(41.631751578557044 74.67186698333354) bank64808 +64809 POINT(41.207060135364735 73.86456100118278) bank64809 +64810 POINT(39.87779841675451 73.48976897139757) bank64810 +64811 POINT(41.294940007921625 73.67541838615283) bank64811 +64812 POINT(41.06929957728644 73.94970133625705) bank64812 +64813 POINT(41.19939094351906 74.48985436565097) bank64813 +64814 POINT(41.69565039389617 73.0145499676555) bank64814 +64815 POINT(41.39891508451632 74.47514111702769) bank64815 +64816 POINT(41.023757357616205 73.5272436646754) bank64816 +64817 POINT(40.549567332813396 74.70484795234978) bank64817 +64818 POINT(41.2794137341961 74.81825300603059) bank64818 +64819 POINT(39.978414088340074 74.51612415548176) bank64819 +64820 POINT(40.82823285613619 74.64845713341332) bank64820 +64821 POINT(41.37872906238945 73.62858043959943) bank64821 +64822 POINT(41.232589552430824 74.74632158021838) bank64822 +64823 POINT(40.08285433025534 74.39948785220079) bank64823 +64824 POINT(40.82398406653488 74.22756594699551) bank64824 +64825 POINT(40.051541404436534 73.18405978241891) bank64825 +64826 POINT(41.4916744925628 73.79033103563091) bank64826 +64827 POINT(40.67740261293176 74.26692967161407) bank64827 +64828 POINT(40.21285504346728 74.46112080696518) bank64828 +64829 POINT(39.882595336712555 73.49582603653963) bank64829 +64830 POINT(41.2421870509525 74.6145620659385) bank64830 +64831 POINT(40.47040584497222 74.52829366170306) bank64831 +64832 POINT(40.97488938487209 73.74508632237814) bank64832 +64833 POINT(40.11699042015954 73.78117102096998) bank64833 +64834 POINT(40.70616639033412 73.84978908585514) bank64834 +64835 POINT(40.85253784402578 73.52118676528339) bank64835 +64836 POINT(40.09685134318285 73.42966976939901) bank64836 +64837 POINT(40.97129937524772 74.66186763010296) bank64837 +64838 POINT(40.268904998532726 73.86028288992735) bank64838 +64839 POINT(40.62946690057198 74.3801554185094) bank64839 +64840 POINT(40.48966970540069 73.36180272960317) bank64840 +64841 POINT(40.0387577345723 73.26842594969497) bank64841 +64842 POINT(39.969055653394484 73.63994764398677) bank64842 +64843 POINT(40.875547590055234 74.17239586668727) bank64843 +64844 POINT(41.627222271665076 74.95858089828637) bank64844 +64845 POINT(41.270059270701104 73.20804913992508) bank64845 +64846 POINT(41.00540408805748 73.9633082918784) bank64846 +64847 POINT(41.64165833990559 73.50984702108674) bank64847 +64848 POINT(40.485590935987055 73.7426974898917) bank64848 +64849 POINT(41.37632209217494 74.07266095573034) bank64849 +64850 POINT(40.7184707182464 74.36295783536599) bank64850 +64851 POINT(40.51120943445154 74.35159961239009) bank64851 +64852 POINT(41.537014181120085 74.42431466218264) bank64852 +64853 POINT(39.758401681281704 74.18359838467222) bank64853 +64854 POINT(41.27774874323535 74.17897200101463) bank64854 +64855 POINT(41.046253740762076 74.52787004641718) bank64855 +64856 POINT(40.77947254479358 73.9767728696215) bank64856 +64857 POINT(41.66819305290258 73.89099872145754) bank64857 +64858 POINT(41.02342460927297 74.6173773701254) bank64858 +64859 POINT(40.76718735208588 74.24388289307318) bank64859 +64860 POINT(41.509609461424475 74.16984212692535) bank64860 +64861 POINT(41.6944270784151 73.63591610809364) bank64861 +64862 POINT(41.34483360504518 73.0310115000775) bank64862 +64863 POINT(41.03858233705561 74.20510063350497) bank64863 +64864 POINT(41.6197676917573 74.17369978862031) bank64864 +64865 POINT(39.99383946011954 73.91969454564114) bank64865 +64866 POINT(39.76303098530001 73.96014870087959) bank64866 +64867 POINT(41.413316964243386 74.17601526127712) bank64867 +64868 POINT(41.260854271585615 73.99621146158482) bank64868 +64869 POINT(40.07805485217469 73.48433177151227) bank64869 +64870 POINT(41.13426149177275 73.0485826631321) bank64870 +64871 POINT(40.11787132593779 74.0735036556295) bank64871 +64872 POINT(41.57653209111891 74.71195760370753) bank64872 +64873 POINT(41.504521487169086 74.6957979895208) bank64873 +64874 POINT(40.90609966003545 73.18498535876417) bank64874 +64875 POINT(40.28050976908335 74.3430954819891) bank64875 +64876 POINT(40.25193357246925 73.88832678444021) bank64876 +64877 POINT(40.9749523989005 73.32734932833674) bank64877 +64878 POINT(41.26701597463742 74.40125897923394) bank64878 +64879 POINT(40.73959030318033 74.49949074681467) bank64879 +64880 POINT(39.72136488651291 74.98499063161205) bank64880 +64881 POINT(40.66115891233515 73.37111075556464) bank64881 +64882 POINT(41.23525539223132 73.36477423623859) bank64882 +64883 POINT(40.00228935030784 73.19890734618569) bank64883 +64884 POINT(41.62582924467374 74.81285369572018) bank64884 +64885 POINT(41.419484247111605 73.29492933266084) bank64885 +64886 POINT(40.68573829709654 74.49604954529511) bank64886 +64887 POINT(39.88715309971931 73.04704770949465) bank64887 +64888 POINT(41.377900424573205 73.28740889871703) bank64888 +64889 POINT(39.75582027586242 73.77207355500897) bank64889 +64890 POINT(40.25753481716795 74.31878676626792) bank64890 +64891 POINT(40.864639778201 73.73483795674944) bank64891 +64892 POINT(41.54744265497108 73.44820411054747) bank64892 +64893 POINT(40.31382999556442 73.49527530191047) bank64893 +64894 POINT(41.57917500362762 74.81023128942515) bank64894 +64895 POINT(41.325291761186676 74.59151263467527) bank64895 +64896 POINT(41.68089818358154 74.90033229768147) bank64896 +64897 POINT(39.825398785471016 73.9047996579934) bank64897 +64898 POINT(39.809503227938784 74.09919329734173) bank64898 +64899 POINT(40.50401396333831 74.69094569058382) bank64899 +64900 POINT(41.53441536851269 74.61516454383188) bank64900 +64901 POINT(40.157540908181645 74.3076175814276) bank64901 +64902 POINT(39.94641583317457 74.32361558113858) bank64902 +64903 POINT(39.75981321764696 74.87759515629617) bank64903 +64904 POINT(41.24218132898542 74.28747816155406) bank64904 +64905 POINT(40.196262293976254 73.13663627746651) bank64905 +64906 POINT(40.51779507061378 73.38717124558903) bank64906 +64907 POINT(40.2330176812254 74.42963988171442) bank64907 +64908 POINT(41.13003858011081 74.99121412447002) bank64908 +64909 POINT(40.08728879038528 74.27509894567669) bank64909 +64910 POINT(40.67671587564196 73.41647208477944) bank64910 +64911 POINT(40.9283677981099 74.86991598147219) bank64911 +64912 POINT(41.57393663449591 74.68969720989257) bank64912 +64913 POINT(40.734615642397316 73.66467766721495) bank64913 +64914 POINT(41.1957837209313 74.71478458214784) bank64914 +64915 POINT(40.96670884581204 74.04045776405798) bank64915 +64916 POINT(40.25082576829303 74.28409938050095) bank64916 +64917 POINT(41.26341422966769 73.19655696571712) bank64917 +64918 POINT(40.78779021520531 74.42592873475995) bank64918 +64919 POINT(40.662618455819 73.53618199372592) bank64919 +64920 POINT(41.15202247509623 73.20203362717608) bank64920 +64921 POINT(40.68365184502567 73.94961802486638) bank64921 +64922 POINT(40.59031438318012 73.35121542392912) bank64922 +64923 POINT(40.01366263218717 74.01370721921262) bank64923 +64924 POINT(41.3224182091861 74.12909916340939) bank64924 +64925 POINT(41.49050934858273 74.26843108699016) bank64925 +64926 POINT(41.5436319046786 74.72129894910624) bank64926 +64927 POINT(39.81229297404397 73.85310148071856) bank64927 +64928 POINT(40.96898083780085 73.97595458178114) bank64928 +64929 POINT(40.55847070977662 74.0500269723088) bank64929 +64930 POINT(41.53129295597319 73.796330335352) bank64930 +64931 POINT(41.69456081671639 73.4362206136202) bank64931 +64932 POINT(39.92141181067168 73.03112000769008) bank64932 +64933 POINT(39.89340877599539 73.50684655811558) bank64933 +64934 POINT(40.170323751125395 73.41823553321774) bank64934 +64935 POINT(40.4791940364894 73.38132540636646) bank64935 +64936 POINT(40.59724340872865 74.75465010158) bank64936 +64937 POINT(40.58114226969135 74.01197050075075) bank64937 +64938 POINT(39.863016157556174 74.12336924279657) bank64938 +64939 POINT(39.94428510681682 73.00780703122511) bank64939 +64940 POINT(41.09587895575073 73.95119101424383) bank64940 +64941 POINT(40.18184348203817 73.55736524720268) bank64941 +64942 POINT(39.82412392355928 74.72334290648848) bank64942 +64943 POINT(41.50071623625284 74.15623491585798) bank64943 +64944 POINT(40.97449035512002 73.70456605542691) bank64944 +64945 POINT(40.5215109795741 73.2825438911231) bank64945 +64946 POINT(40.967762226238754 73.58011105255859) bank64946 +64947 POINT(41.53888660765135 73.30536863946932) bank64947 +64948 POINT(40.11562625323436 74.70620989719635) bank64948 +64949 POINT(40.603604149490565 73.5322469172505) bank64949 +64950 POINT(40.029031779759755 74.58309367684248) bank64950 +64951 POINT(40.75406724490527 74.02386542781146) bank64951 +64952 POINT(41.00893234659827 73.95538005032185) bank64952 +64953 POINT(39.77740560920694 74.67539899958912) bank64953 +64954 POINT(41.152765769353145 73.17446227433139) bank64954 +64955 POINT(40.47167168421678 73.80640606774233) bank64955 +64956 POINT(41.014336701387506 74.59713504349392) bank64956 +64957 POINT(40.77559341298184 74.63436327268519) bank64957 +64958 POINT(39.82912278797493 74.3813434105328) bank64958 +64959 POINT(40.99475661770422 74.59200593892197) bank64959 +64960 POINT(40.85496726677182 73.06458952483807) bank64960 +64961 POINT(39.91869926366169 74.12400075080238) bank64961 +64962 POINT(40.849877149093146 73.58301239555055) bank64962 +64963 POINT(41.71093121955506 73.4452307997605) bank64963 +64964 POINT(41.56054783119216 73.58066739074985) bank64964 +64965 POINT(41.527350274087155 74.06263289648435) bank64965 +64966 POINT(41.606350146403116 73.6387228052768) bank64966 +64967 POINT(41.46422630525465 73.38131704236638) bank64967 +64968 POINT(41.42110669456898 73.14196164518926) bank64968 +64969 POINT(40.58689445967374 74.15093003655402) bank64969 +64970 POINT(41.29794297547612 74.78803756728578) bank64970 +64971 POINT(40.1173452005363 73.62185189552588) bank64971 +64972 POINT(40.24305539482361 73.16849495003305) bank64972 +64973 POINT(41.33756890126488 74.12414470340258) bank64973 +64974 POINT(41.30084672081017 74.19668996279486) bank64974 +64975 POINT(40.20044104768081 74.01422405083619) bank64975 +64976 POINT(41.628071124537755 73.29176390032576) bank64976 +64977 POINT(41.679490083792196 73.85443263059724) bank64977 +64978 POINT(41.03224480311795 73.73005004491158) bank64978 +64979 POINT(41.3109610636703 73.47157450278742) bank64979 +64980 POINT(40.845074812348734 74.72876781737213) bank64980 +64981 POINT(40.46188952183704 73.76835154889973) bank64981 +64982 POINT(39.80580137012613 73.02084543822106) bank64982 +64983 POINT(40.582777865154455 74.01324237735095) bank64983 +64984 POINT(41.71145873705051 74.21733737605551) bank64984 +64985 POINT(40.942434192123585 73.83797084293722) bank64985 +64986 POINT(40.792941023930624 73.9877397972158) bank64986 +64987 POINT(40.43534496550813 74.92577829883197) bank64987 +64988 POINT(40.06765348638788 74.40083387781995) bank64988 +64989 POINT(41.632857493732644 73.80720110104951) bank64989 +64990 POINT(41.419704351374996 74.78082744633694) bank64990 +64991 POINT(40.634085938756996 73.4370411443111) bank64991 +64992 POINT(41.615563753875435 74.19463822750784) bank64992 +64993 POINT(41.111292830200746 74.37209840850733) bank64993 +64994 POINT(41.62504311364235 74.20591589614432) bank64994 +64995 POINT(41.02118259744923 74.95672677111195) bank64995 +64996 POINT(40.17727571131134 73.52908629076342) bank64996 +64997 POINT(41.12792716379647 73.92762184238622) bank64997 +64998 POINT(41.68371487600171 73.6388838454077) bank64998 +64999 POINT(41.26097262983765 74.24199174383243) bank64999 +65000 POINT(39.95967334130847 73.69890193570964) bank65000 +65001 POINT(40.09648769875555 73.99404074373209) bank65001 +65002 POINT(39.95417962530931 74.48629076632147) bank65002 +65003 POINT(41.59321894689405 73.47208477032567) bank65003 +65004 POINT(40.17961364031044 73.04744754854849) bank65004 +65005 POINT(40.7821483674144 73.78565842541916) bank65005 +65006 POINT(40.98341061859208 74.22285425203066) bank65006 +65007 POINT(41.38436992084613 73.77333835415314) bank65007 +65008 POINT(40.67114319886585 73.99015310488346) bank65008 +65009 POINT(40.752706206310584 73.21655572906484) bank65009 +65010 POINT(41.28228139226863 73.76490200022725) bank65010 +65011 POINT(40.41542793210784 73.29269814178778) bank65011 +65012 POINT(40.817954590875914 74.17505686965937) bank65012 +65013 POINT(40.88513787757323 74.00032733277848) bank65013 +65014 POINT(41.44835088760936 74.06124373541904) bank65014 +65015 POINT(40.25593844398387 73.81354948691393) bank65015 +65016 POINT(41.69256659592353 74.71908683305305) bank65016 +65017 POINT(40.808855112847844 73.77076151567388) bank65017 +65018 POINT(40.9056063980697 73.30863121454371) bank65018 +65019 POINT(41.55420160665217 73.12272588233745) bank65019 +65020 POINT(41.30475960600598 73.95128888117476) bank65020 +65021 POINT(40.59927460850382 73.26422886570737) bank65021 +65022 POINT(40.079830895936496 73.76818826154658) bank65022 +65023 POINT(40.954860851021124 74.34668693245067) bank65023 +65024 POINT(41.571430332227024 74.94471434219035) bank65024 +65025 POINT(40.70569843015983 74.29542168523159) bank65025 +65026 POINT(40.84549516935694 74.6817148602532) bank65026 +65027 POINT(40.86184288914535 74.13620348082767) bank65027 +65028 POINT(40.36932380968782 74.60843722672631) bank65028 +65029 POINT(41.49080240608467 73.9837610908338) bank65029 +65030 POINT(41.494532741698144 73.13869044546713) bank65030 +65031 POINT(40.546653289226924 74.33707662017558) bank65031 +65032 POINT(41.704973162819 73.91281724475078) bank65032 +65033 POINT(41.40790048362156 74.38710795031582) bank65033 +65034 POINT(40.011252151177196 74.70108063319014) bank65034 +65035 POINT(41.66404681366006 74.62015370752765) bank65035 +65036 POINT(40.668874116604606 73.73520676316018) bank65036 +65037 POINT(41.60979621133979 73.22593313795468) bank65037 +65038 POINT(40.04131667975642 74.63312443274864) bank65038 +65039 POINT(40.956568800017536 73.31575964388675) bank65039 +65040 POINT(40.16821298168359 74.50507084849616) bank65040 +65041 POINT(40.92432374386991 74.76618676240618) bank65041 +65042 POINT(41.27140147436084 74.37999463864757) bank65042 +65043 POINT(40.19191480613601 73.53563466910632) bank65043 +65044 POINT(40.35897744321309 74.34455897857706) bank65044 +65045 POINT(41.16197054848104 74.9771856461424) bank65045 +65046 POINT(41.09790640074267 73.14913291788821) bank65046 +65047 POINT(41.11522956998165 73.88704689443016) bank65047 +65048 POINT(41.189574305299054 73.87088263661869) bank65048 +65049 POINT(40.116675054910324 73.95146370174889) bank65049 +65050 POINT(40.008125311794664 73.09583078801444) bank65050 +65051 POINT(39.773431792116796 74.26153378218463) bank65051 +65052 POINT(41.50777315680158 73.4312961953666) bank65052 +65053 POINT(41.222854246429584 73.0282961978846) bank65053 +65054 POINT(40.169178696853194 73.56423969573382) bank65054 +65055 POINT(41.6775681114786 73.98343660448928) bank65055 +65056 POINT(41.410971978956 73.84405289707114) bank65056 +65057 POINT(40.726720256604864 74.12815896633555) bank65057 +65058 POINT(39.74958290937342 74.18390496683509) bank65058 +65059 POINT(40.49076356582519 73.94883722644424) bank65059 +65060 POINT(41.044212965860915 74.95413367256295) bank65060 +65061 POINT(40.18265687953511 73.08746351164689) bank65061 +65062 POINT(41.152888467494215 74.37452183872672) bank65062 +65063 POINT(40.20733638784946 73.61222118948379) bank65063 +65064 POINT(40.04343167172311 73.32001650606163) bank65064 +65065 POINT(39.76677030372096 73.70269387476425) bank65065 +65066 POINT(39.865169125028046 74.27091920791696) bank65066 +65067 POINT(41.41638260244247 74.79640452720513) bank65067 +65068 POINT(40.01679854642341 73.47220616774541) bank65068 +65069 POINT(41.13486613553794 74.84097769716651) bank65069 +65070 POINT(40.171682561704124 73.27816887096209) bank65070 +65071 POINT(41.6364723980171 74.5473708093084) bank65071 +65072 POINT(41.55081729074789 74.13475087085577) bank65072 +65073 POINT(40.24170178922509 73.77850560845926) bank65073 +65074 POINT(40.440106723082536 74.184202531341) bank65074 +65075 POINT(39.76243778597035 73.31224996515175) bank65075 +65076 POINT(40.15634539842437 74.10100970894693) bank65076 +65077 POINT(40.97810440544243 73.36015770107878) bank65077 +65078 POINT(40.222843158408075 74.12163383172864) bank65078 +65079 POINT(40.35680900693407 74.78137071870744) bank65079 +65080 POINT(41.431594536966706 74.3807162825219) bank65080 +65081 POINT(40.14723220782523 74.74645347681327) bank65081 +65082 POINT(40.07387091901075 73.2264547478668) bank65082 +65083 POINT(40.058398333294555 74.3414533756595) bank65083 +65084 POINT(41.51686985349116 74.47351149560215) bank65084 +65085 POINT(39.8743689549478 74.1225836565637) bank65085 +65086 POINT(41.531000663234074 73.45358503901528) bank65086 +65087 POINT(40.05355312927893 73.64788173119662) bank65087 +65088 POINT(40.59414030223334 74.18129649283689) bank65088 +65089 POINT(39.997716929371606 74.90601955315839) bank65089 +65090 POINT(40.38131065101953 74.00204675419785) bank65090 +65091 POINT(40.87290368264385 73.34114738374436) bank65091 +65092 POINT(41.34487387059601 74.2657676294017) bank65092 +65093 POINT(40.308864164538946 73.12844477831037) bank65093 +65094 POINT(40.6104602314252 73.84469330794514) bank65094 +65095 POINT(40.93737318490189 74.62722600661318) bank65095 +65096 POINT(40.00921034190586 74.83535351837892) bank65096 +65097 POINT(40.1016301031481 73.04577908347781) bank65097 +65098 POINT(41.235515708615544 73.38369738421729) bank65098 +65099 POINT(39.74455253157657 74.7158863687284) bank65099 +65100 POINT(41.22041415308564 73.17754004248533) bank65100 +65101 POINT(40.286991818941054 73.75725066361234) bank65101 +65102 POINT(39.982452289020976 73.56441033017634) bank65102 +65103 POINT(41.61565956620926 74.39364224701696) bank65103 +65104 POINT(41.034978278880786 74.44512000734261) bank65104 +65105 POINT(41.30785975847245 73.88221414562614) bank65105 +65106 POINT(40.56075056774596 73.86784837584746) bank65106 +65107 POINT(40.283235246186244 74.41007740952224) bank65107 +65108 POINT(41.66550741158435 73.36585771286452) bank65108 +65109 POINT(40.6601181864233 74.22897884315985) bank65109 +65110 POINT(40.8727214311092 73.00607144872562) bank65110 +65111 POINT(41.54523994500274 74.71093361182044) bank65111 +65112 POINT(40.315941980026885 74.68144387315363) bank65112 +65113 POINT(40.1921519892217 73.98707287460527) bank65113 +65114 POINT(39.91716366867478 74.78816685763914) bank65114 +65115 POINT(39.80962244164116 74.43923522201521) bank65115 +65116 POINT(41.63710986560936 74.07460775860372) bank65116 +65117 POINT(40.06505402973183 74.9266066063392) bank65117 +65118 POINT(41.08256315574459 73.37307836041812) bank65118 +65119 POINT(40.07034414793554 73.82184858717252) bank65119 +65120 POINT(40.39113263399286 73.8282285333765) bank65120 +65121 POINT(40.28654530732126 73.84985033153468) bank65121 +65122 POINT(41.46645477545452 74.99930274697118) bank65122 +65123 POINT(40.67044949680952 74.7262278837852) bank65123 +65124 POINT(41.131417291456096 74.46388996924365) bank65124 +65125 POINT(39.97364586627607 73.4333021461991) bank65125 +65126 POINT(40.31399789378703 73.62758609578725) bank65126 +65127 POINT(40.66287261378441 74.20003083859847) bank65127 +65128 POINT(40.50573068826442 74.25414299661604) bank65128 +65129 POINT(41.108171157745865 73.0235292907213) bank65129 +65130 POINT(40.332740044179495 74.83673356277059) bank65130 +65131 POINT(41.05651776541475 73.30774981517692) bank65131 +65132 POINT(40.359009748666445 74.64375232712499) bank65132 +65133 POINT(40.64366828108276 74.5950203313572) bank65133 +65134 POINT(41.521565889571285 74.14307385052518) bank65134 +65135 POINT(40.1267049847832 74.88520851881438) bank65135 +65136 POINT(39.727714741215465 74.33943396145801) bank65136 +65137 POINT(39.99148403748158 73.56601842546179) bank65137 +65138 POINT(39.84282879595119 73.08134834711915) bank65138 +65139 POINT(39.90024352120912 74.41096348015154) bank65139 +65140 POINT(39.72952274647318 74.10985107672101) bank65140 +65141 POINT(41.061491189574724 75.00066485637998) bank65141 +65142 POINT(40.75892625927406 73.79157566435997) bank65142 +65143 POINT(40.44150527939204 74.10408467238182) bank65143 +65144 POINT(40.730894933045 73.46892374405762) bank65144 +65145 POINT(40.804231297760936 74.22772388031333) bank65145 +65146 POINT(39.991573030150555 74.11412206874337) bank65146 +65147 POINT(39.85579699053562 74.25804933886991) bank65147 +65148 POINT(40.982218101566694 74.017918704311) bank65148 +65149 POINT(40.39653736288677 74.72386637357246) bank65149 +65150 POINT(40.66215992606406 74.76314320445339) bank65150 +65151 POINT(40.77140091664821 73.7499733076017) bank65151 +65152 POINT(39.78040440319586 73.09036811194487) bank65152 +65153 POINT(41.241885212618655 74.71266477276794) bank65153 +65154 POINT(41.38051912612731 73.94833404415563) bank65154 +65155 POINT(40.66960852403448 74.95626827587127) bank65155 +65156 POINT(40.32727106136804 74.97921644547363) bank65156 +65157 POINT(41.49735646755001 74.6073341467383) bank65157 +65158 POINT(40.54795212493822 73.70639889547577) bank65158 +65159 POINT(41.05788611667427 73.08217751823307) bank65159 +65160 POINT(40.55461944878253 73.56074639870843) bank65160 +65161 POINT(40.59186529431119 73.08845798776642) bank65161 +65162 POINT(40.19646834531809 73.43114599002762) bank65162 +65163 POINT(40.28269442455638 73.53222370323002) bank65163 +65164 POINT(40.67212124445015 74.46159353206741) bank65164 +65165 POINT(40.334962562841596 73.22056993866737) bank65165 +65166 POINT(39.95910707474234 73.36937192135132) bank65166 +65167 POINT(40.02188831599997 74.4917118575965) bank65167 +65168 POINT(39.83789112052099 74.98266528330001) bank65168 +65169 POINT(41.649854335817594 74.79987942434498) bank65169 +65170 POINT(40.526695642722714 74.48403902267032) bank65170 +65171 POINT(40.792291899375705 73.81274509912772) bank65171 +65172 POINT(41.686354491533194 74.51883539646184) bank65172 +65173 POINT(40.356110166834185 74.31804119867131) bank65173 +65174 POINT(40.417493719912336 73.71299470786579) bank65174 +65175 POINT(40.139105136467386 74.31240814814613) bank65175 +65176 POINT(41.61209139169894 73.73522371081407) bank65176 +65177 POINT(40.17405820515845 74.08466552329534) bank65177 +65178 POINT(41.69950879209503 73.41729372807059) bank65178 +65179 POINT(40.24861511093601 74.84859287128596) bank65179 +65180 POINT(41.53321533192378 74.45670820197093) bank65180 +65181 POINT(39.853383499915 73.67586825721236) bank65181 +65182 POINT(41.11985300895276 73.51748978529888) bank65182 +65183 POINT(40.839737764712524 73.00960144727577) bank65183 +65184 POINT(40.17676070534193 74.70994885364823) bank65184 +65185 POINT(40.60458691484083 74.37230706044471) bank65185 +65186 POINT(39.92698401806289 73.19454248263939) bank65186 +65187 POINT(41.11076079653243 73.20485643661453) bank65187 +65188 POINT(41.56665996414186 73.48803289440745) bank65188 +65189 POINT(40.210667260433915 74.82190420537263) bank65189 +65190 POINT(41.68802492672967 73.75659250023939) bank65190 +65191 POINT(40.34416268624824 73.77479061578877) bank65191 +65192 POINT(40.59671423778073 73.01065728973754) bank65192 +65193 POINT(41.40770021779532 73.01757926056655) bank65193 +65194 POINT(41.060783809836664 74.1131897985684) bank65194 +65195 POINT(41.60145329418504 74.5361708981255) bank65195 +65196 POINT(41.37745071945918 73.44401701561479) bank65196 +65197 POINT(41.373737017057245 73.2959287236922) bank65197 +65198 POINT(40.888461053205944 74.55114766674423) bank65198 +65199 POINT(40.993680809031865 73.25552520667522) bank65199 +65200 POINT(40.28666776927884 74.71940260442331) bank65200 +65201 POINT(41.167092250659586 73.29607548305853) bank65201 +65202 POINT(40.45691517204275 74.49236028147669) bank65202 +65203 POINT(41.12852221694341 73.61132237691456) bank65203 +65204 POINT(41.04394837371609 74.63850722223229) bank65204 +65205 POINT(39.838897651265796 74.33439984808822) bank65205 +65206 POINT(40.10123420916288 73.46253049716101) bank65206 +65207 POINT(40.53538146671517 74.82442885862612) bank65207 +65208 POINT(40.42395944931616 74.38235003395835) bank65208 +65209 POINT(41.09205584703949 74.01466128697751) bank65209 +65210 POINT(39.868739285295575 73.9345339602246) bank65210 +65211 POINT(40.087063822276455 74.80630040548489) bank65211 +65212 POINT(40.96571715105222 74.30994136378885) bank65212 +65213 POINT(41.16613333495043 73.69116968957354) bank65213 +65214 POINT(41.528194089356745 74.42610825868444) bank65214 +65215 POINT(40.049833458182896 74.15490263462657) bank65215 +65216 POINT(40.65887360200247 73.8200343447923) bank65216 +65217 POINT(41.667997331122905 74.07564702298711) bank65217 +65218 POINT(41.286160353555445 73.31607594735966) bank65218 +65219 POINT(41.21430463296007 73.13724656643417) bank65219 +65220 POINT(39.953319973870954 74.71751530819222) bank65220 +65221 POINT(41.57994765352351 73.82070520664101) bank65221 +65222 POINT(40.31137398102284 74.83258039600567) bank65222 +65223 POINT(41.04938485376621 73.06699296199) bank65223 +65224 POINT(41.668424942917326 73.97586903192946) bank65224 +65225 POINT(41.326881633585366 73.5623236544405) bank65225 +65226 POINT(39.92408189320635 74.0653666362592) bank65226 +65227 POINT(40.13998838037547 74.63482164711799) bank65227 +65228 POINT(41.04277549729221 73.40335577999717) bank65228 +65229 POINT(41.199330210462264 74.46190528714395) bank65229 +65230 POINT(41.550566059555536 74.80377936248247) bank65230 +65231 POINT(40.20027288614391 73.15044310882553) bank65231 +65232 POINT(40.588246144054196 73.56688746339323) bank65232 +65233 POINT(41.10377015483569 73.04995432731867) bank65233 +65234 POINT(41.65139752223361 74.12832819685272) bank65234 +65235 POINT(39.87383936535579 74.96213730444978) bank65235 +65236 POINT(41.17191157773795 74.95759210280603) bank65236 +65237 POINT(40.72627105487787 74.42911724243133) bank65237 +65238 POINT(39.924598152030185 73.20283782580717) bank65238 +65239 POINT(41.217853996270534 74.50523206142486) bank65239 +65240 POINT(40.39791799835362 74.21313676018147) bank65240 +65241 POINT(39.979419970556194 74.3833943454085) bank65241 +65242 POINT(40.53884707175537 73.21295441681566) bank65242 +65243 POINT(40.80227135399118 74.9134212837235) bank65243 +65244 POINT(40.74157515867426 73.88420672526908) bank65244 +65245 POINT(40.30467634082059 74.19000199984286) bank65245 +65246 POINT(41.54227708817835 74.09525076540234) bank65246 +65247 POINT(41.07605638066492 74.07066383558038) bank65247 +65248 POINT(39.844045452889226 74.14097176594973) bank65248 +65249 POINT(41.40608562988055 74.37095233902805) bank65249 +65250 POINT(41.03255561752684 73.70121800186446) bank65250 +65251 POINT(40.75616000810197 74.81651242435875) bank65251 +65252 POINT(41.20961818174122 74.00492603666453) bank65252 +65253 POINT(41.69747324571203 74.30495742310778) bank65253 +65254 POINT(41.690386457048845 73.54731473030729) bank65254 +65255 POINT(40.18696487093968 74.49344829918611) bank65255 +65256 POINT(39.761979726663476 73.60042902665178) bank65256 +65257 POINT(39.83913876312995 74.59845437097994) bank65257 +65258 POINT(41.60039335303778 74.64321822899615) bank65258 +65259 POINT(41.13503515612661 73.01084406572478) bank65259 +65260 POINT(40.07971383918944 73.46801329900116) bank65260 +65261 POINT(40.265026235723965 73.69934239780153) bank65261 +65262 POINT(40.41777671103312 74.22549923300858) bank65262 +65263 POINT(40.16601314371788 73.15949726430756) bank65263 +65264 POINT(40.91709256490106 74.92902804932056) bank65264 +65265 POINT(41.164553692209424 74.97664283766096) bank65265 +65266 POINT(41.32734008933138 73.22075481892786) bank65266 +65267 POINT(41.649995980736996 73.9282719631727) bank65267 +65268 POINT(41.612660210842805 74.02791721074972) bank65268 +65269 POINT(41.37766147587405 74.48758867582777) bank65269 +65270 POINT(39.77036187715399 73.23668812277904) bank65270 +65271 POINT(40.070731324412314 73.01078972551382) bank65271 +65272 POINT(40.89986274468072 74.06624730679084) bank65272 +65273 POINT(40.92897862937152 74.41605355382285) bank65273 +65274 POINT(40.04060065270738 74.30661322927553) bank65274 +65275 POINT(40.737456851330485 73.2373090023363) bank65275 +65276 POINT(40.36905072800435 73.65316521203228) bank65276 +65277 POINT(39.981186750302456 73.57617195004991) bank65277 +65278 POINT(41.393674862698866 74.38620483334323) bank65278 +65279 POINT(41.59641194568677 73.67394784795498) bank65279 +65280 POINT(41.374600322659745 74.05116850607473) bank65280 +65281 POINT(41.11777618559722 74.85067299721494) bank65281 +65282 POINT(39.79347681163679 74.2438834209591) bank65282 +65283 POINT(40.04797120368301 73.73452023359172) bank65283 +65284 POINT(41.70486706096765 73.92663788163269) bank65284 +65285 POINT(39.99325283561794 74.53753983127297) bank65285 +65286 POINT(41.345778206214305 73.62172919893385) bank65286 +65287 POINT(39.992837095054924 73.92577120866632) bank65287 +65288 POINT(40.80603361696652 73.06697644791683) bank65288 +65289 POINT(40.78851387360154 74.30205872176008) bank65289 +65290 POINT(40.93595905423071 74.90214182837269) bank65290 +65291 POINT(40.35398660499936 73.75880332519738) bank65291 +65292 POINT(41.083848714227635 73.19048024522593) bank65292 +65293 POINT(39.79130774056182 73.5195926782632) bank65293 +65294 POINT(40.38875658163631 74.32973178676683) bank65294 +65295 POINT(39.91125315346388 73.8002411283189) bank65295 +65296 POINT(41.59724720337302 74.41902021192678) bank65296 +65297 POINT(40.31019102807295 74.22201087852427) bank65297 +65298 POINT(41.294259375123104 73.01408061025927) bank65298 +65299 POINT(40.03557714535495 73.66646496566811) bank65299 +65300 POINT(39.83088993443276 74.75992733469849) bank65300 +65301 POINT(40.475093265095836 73.19570664053418) bank65301 +65302 POINT(39.715675396032154 73.15910362188919) bank65302 +65303 POINT(40.816844986192486 74.55872862299358) bank65303 +65304 POINT(40.51326670691539 73.70996591147676) bank65304 +65305 POINT(41.34224112800714 73.5586595613781) bank65305 +65306 POINT(40.99856889719518 74.62456737235877) bank65306 +65307 POINT(40.60735158639629 73.05996217564325) bank65307 +65308 POINT(39.78017728833396 74.42913614102952) bank65308 +65309 POINT(39.748288318615145 74.28254005282164) bank65309 +65310 POINT(41.06016983278257 74.98092045575753) bank65310 +65311 POINT(40.03806638164317 73.60795431972142) bank65311 +65312 POINT(41.55724933251925 74.7207710280019) bank65312 +65313 POINT(39.889121277299026 74.37025016378267) bank65313 +65314 POINT(41.23590952688858 73.17606637008737) bank65314 +65315 POINT(40.94540152095417 73.98669792501063) bank65315 +65316 POINT(39.82044285549999 74.88579007706791) bank65316 +65317 POINT(41.51522832374738 74.49318473028511) bank65317 +65318 POINT(40.690809481577546 74.05779930693491) bank65318 +65319 POINT(41.12800162024379 74.40451283306548) bank65319 +65320 POINT(41.61438990275629 73.17947427076219) bank65320 +65321 POINT(41.61095661590137 74.26146331344881) bank65321 +65322 POINT(40.606047018410386 74.33763487584935) bank65322 +65323 POINT(41.252974165841934 73.66122761901171) bank65323 +65324 POINT(41.67474937857831 74.96407932129111) bank65324 +65325 POINT(40.572167995197304 74.74278546044755) bank65325 +65326 POINT(40.18716407802765 73.6736714863144) bank65326 +65327 POINT(41.03987820202967 73.45132081882986) bank65327 +65328 POINT(40.200987469609984 74.34984099016492) bank65328 +65329 POINT(40.12963710019185 73.13063735303334) bank65329 +65330 POINT(40.86498699947981 75.00010843683252) bank65330 +65331 POINT(39.922179834611256 74.7703690904126) bank65331 +65332 POINT(39.99405188146143 73.54252969943362) bank65332 +65333 POINT(41.70422207189864 74.55692951262935) bank65333 +65334 POINT(40.180995112845274 74.73547112422176) bank65334 +65335 POINT(41.65504138302848 74.7160530374813) bank65335 +65336 POINT(40.373945089852874 73.3187230638247) bank65336 +65337 POINT(40.7556641767375 74.69793677101245) bank65337 +65338 POINT(39.96360334253844 73.88890246292348) bank65338 +65339 POINT(40.21225585553981 73.4888859261291) bank65339 +65340 POINT(40.29687671105959 74.36574917992739) bank65340 +65341 POINT(41.389685428984706 74.89445864705687) bank65341 +65342 POINT(40.64724277544804 73.58590470556268) bank65342 +65343 POINT(40.69160064683565 74.02161855293912) bank65343 +65344 POINT(39.76502891207245 73.28096536794067) bank65344 +65345 POINT(40.31885713024466 74.93229491110777) bank65345 +65346 POINT(41.239231687262155 74.98392197422783) bank65346 +65347 POINT(41.166710908447676 74.99679154233974) bank65347 +65348 POINT(41.37464585385343 73.68120990168529) bank65348 +65349 POINT(40.073212510686496 73.66955590911026) bank65349 +65350 POINT(40.755308187223285 73.24823310954834) bank65350 +65351 POINT(40.03934782794401 74.1855161857267) bank65351 +65352 POINT(39.948275916447884 74.37002991054484) bank65352 +65353 POINT(40.030040795447626 73.31748252806801) bank65353 +65354 POINT(40.36355967937037 74.77318941137575) bank65354 +65355 POINT(40.28597565176935 74.94927480647839) bank65355 +65356 POINT(41.07821429591437 74.66004744155447) bank65356 +65357 POINT(39.99690327589651 73.41576015929255) bank65357 +65358 POINT(41.63108710005074 74.43317835764995) bank65358 +65359 POINT(41.480085507142064 73.57011106773396) bank65359 +65360 POINT(40.88410291543126 73.50239283028928) bank65360 +65361 POINT(40.230574538807005 73.87626602004092) bank65361 +65362 POINT(40.595041123270406 74.85134168940205) bank65362 +65363 POINT(41.406528617042156 74.44294105465343) bank65363 +65364 POINT(41.53041476370272 73.32018453022765) bank65364 +65365 POINT(40.75082898670765 73.28112877957535) bank65365 +65366 POINT(40.72752360809195 73.03166042951868) bank65366 +65367 POINT(40.64986216082535 73.21338548452242) bank65367 +65368 POINT(40.3445077847929 73.81597268184703) bank65368 +65369 POINT(40.524895440823315 73.14025568449075) bank65369 +65370 POINT(41.63512006316147 74.06225693336081) bank65370 +65371 POINT(40.80871375620797 73.96061135196447) bank65371 +65372 POINT(41.26979242618674 73.56128159237787) bank65372 +65373 POINT(41.0361574567313 73.31326515287678) bank65373 +65374 POINT(40.78696757005209 73.14516521920044) bank65374 +65375 POINT(41.00723288355892 73.19387036401497) bank65375 +65376 POINT(40.04920580326088 73.23256937826697) bank65376 +65377 POINT(41.52257445044353 73.29884184088266) bank65377 +65378 POINT(40.01701456676716 74.74479150012719) bank65378 +65379 POINT(40.44236179080164 73.8394986376317) bank65379 +65380 POINT(40.57330182801438 73.5730606208363) bank65380 +65381 POINT(40.67116582053351 73.86776428878237) bank65381 +65382 POINT(39.9570482286989 73.2874542624797) bank65382 +65383 POINT(40.48582317859838 74.87572335652247) bank65383 +65384 POINT(39.84719327624462 74.24619160764935) bank65384 +65385 POINT(41.201998972716964 74.72212472225765) bank65385 +65386 POINT(40.11022352853785 74.04378117590278) bank65386 +65387 POINT(40.622996764586425 73.29391994178181) bank65387 +65388 POINT(41.50867440960069 73.9556239963766) bank65388 +65389 POINT(40.85421825280128 74.10048001498639) bank65389 +65390 POINT(41.4916156459803 73.81874757831697) bank65390 +65391 POINT(40.08837781493078 74.36335281199052) bank65391 +65392 POINT(40.279238470399264 74.0230797386521) bank65392 +65393 POINT(39.94926118751906 73.59959116593082) bank65393 +65394 POINT(41.25816247757851 74.60254903325003) bank65394 +65395 POINT(40.68549455942478 73.13664385912267) bank65395 +65396 POINT(40.833165231294515 74.70760692211888) bank65396 +65397 POINT(39.80754122789015 74.37947870593985) bank65397 +65398 POINT(39.73749499874605 73.52504428904231) bank65398 +65399 POINT(39.991322291420026 74.71904504555445) bank65399 +65400 POINT(41.007068959531125 74.17599045429903) bank65400 +65401 POINT(40.73080690494727 73.39533803081281) bank65401 +65402 POINT(41.28326281191717 73.70875174609048) bank65402 +65403 POINT(40.4744874657996 73.53089933039064) bank65403 +65404 POINT(39.93148714930737 73.49040119083061) bank65404 +65405 POINT(40.003550312368866 73.11952769880891) bank65405 +65406 POINT(41.426406285350474 73.54577786081967) bank65406 +65407 POINT(39.79288346070935 74.55180424095433) bank65407 +65408 POINT(40.912485764410214 74.20670191174665) bank65408 +65409 POINT(40.900872919331725 74.99157641021206) bank65409 +65410 POINT(41.661996730465084 73.03058971644307) bank65410 +65411 POINT(41.07845483675091 73.34891860830211) bank65411 +65412 POINT(40.627277764188484 74.08959507094113) bank65412 +65413 POINT(39.894124346332234 74.00322122433447) bank65413 +65414 POINT(40.19488353246931 74.49206302367085) bank65414 +65415 POINT(40.54866223641681 73.80406314968715) bank65415 +65416 POINT(41.66085231808513 74.17694875504567) bank65416 +65417 POINT(41.01100946277433 73.51307411606268) bank65417 +65418 POINT(41.39893916909693 73.26182641292854) bank65418 +65419 POINT(40.13885563772101 73.62866514998512) bank65419 +65420 POINT(40.77338572706679 73.30547216754319) bank65420 +65421 POINT(39.82957582474623 73.18810080519879) bank65421 +65422 POINT(40.1831953136974 74.19584285505022) bank65422 +65423 POINT(40.240604225107816 73.46298592513848) bank65423 +65424 POINT(40.363714586710636 74.25365541931367) bank65424 +65425 POINT(41.40313128408048 73.47359719367472) bank65425 +65426 POINT(39.91911282771515 73.5215436419104) bank65426 +65427 POINT(40.5915709663617 74.16834578830796) bank65427 +65428 POINT(40.39749872853059 73.80788377476068) bank65428 +65429 POINT(40.059206724400745 73.14512048323802) bank65429 +65430 POINT(40.48061046522506 74.47924928375399) bank65430 +65431 POINT(41.509627969665246 74.8530269690087) bank65431 +65432 POINT(40.693271826471275 73.82036126333098) bank65432 +65433 POINT(39.91729417349951 74.62919668637035) bank65433 +65434 POINT(41.5777714905068 74.92985457359626) bank65434 +65435 POINT(40.07629477326582 74.69513566098955) bank65435 +65436 POINT(39.83848325935905 74.23861176152714) bank65436 +65437 POINT(40.97788984059596 74.35502842351939) bank65437 +65438 POINT(41.014125603511154 74.95830403423365) bank65438 +65439 POINT(40.881460369952954 74.78694268189027) bank65439 +65440 POINT(40.80678284521828 73.0782728974502) bank65440 +65441 POINT(40.814382055749256 74.26397554159874) bank65441 +65442 POINT(40.34691226569443 73.04124252676434) bank65442 +65443 POINT(40.88158762198712 74.97476060980436) bank65443 +65444 POINT(40.08991989899327 74.8001643151534) bank65444 +65445 POINT(40.252394853065084 73.74551342979608) bank65445 +65446 POINT(40.81797330950313 74.21067611961095) bank65446 +65447 POINT(39.93561912121308 74.87627110556187) bank65447 +65448 POINT(41.11011235174994 74.43271136495295) bank65448 +65449 POINT(40.020350943996725 74.30415363789345) bank65449 +65450 POINT(40.9328992341162 73.56042557491955) bank65450 +65451 POINT(40.54795752154736 74.63042796853645) bank65451 +65452 POINT(41.48528299800419 74.06699634138825) bank65452 +65453 POINT(40.39290392201799 74.35023038610021) bank65453 +65454 POINT(39.733621000326295 74.11727451449813) bank65454 +65455 POINT(40.59055382411492 73.67100669865786) bank65455 +65456 POINT(40.16111184349223 73.59611159654087) bank65456 +65457 POINT(39.859268202704214 74.29096983939881) bank65457 +65458 POINT(40.90273742080226 74.51315845781771) bank65458 +65459 POINT(41.415614513890674 73.76529786749549) bank65459 +65460 POINT(41.36381258475327 73.99293405696784) bank65460 +65461 POINT(40.477590808753455 73.34966878952405) bank65461 +65462 POINT(41.623548633861326 74.2387377025908) bank65462 +65463 POINT(41.28764113994809 74.46813424447764) bank65463 +65464 POINT(41.13551900073058 74.03318100951864) bank65464 +65465 POINT(40.3765256034488 74.26060429669069) bank65465 +65466 POINT(41.081447266580746 74.31725244200726) bank65466 +65467 POINT(41.0259341113176 74.12887385204398) bank65467 +65468 POINT(40.34294889153322 74.76080593374417) bank65468 +65469 POINT(41.331896740126915 73.22846627583827) bank65469 +65470 POINT(40.01734539543321 73.67697749573219) bank65470 +65471 POINT(40.957268582417555 74.723073188365) bank65471 +65472 POINT(40.831519856635246 74.67613938342153) bank65472 +65473 POINT(39.74734344857931 73.20551158623196) bank65473 +65474 POINT(41.39946045633132 73.30972238833779) bank65474 +65475 POINT(40.24379653417198 74.05469550350682) bank65475 +65476 POINT(40.52542914642716 73.00870040141807) bank65476 +65477 POINT(40.91296959970144 74.80855870069044) bank65477 +65478 POINT(41.312881524664 74.62469502733362) bank65478 +65479 POINT(40.722030693303964 74.12471699490862) bank65479 +65480 POINT(41.1167132289256 74.27863845767375) bank65480 +65481 POINT(40.916858560761455 73.84283506854474) bank65481 +65482 POINT(40.94240151679041 73.23066283726715) bank65482 +65483 POINT(40.432969722587856 74.47521875344086) bank65483 +65484 POINT(40.98227209781769 73.70034326966561) bank65484 +65485 POINT(41.584258393089584 73.868943145388) bank65485 +65486 POINT(41.645926002693834 73.0582304794082) bank65486 +65487 POINT(40.39954367114451 74.02897140042846) bank65487 +65488 POINT(40.469320126332555 74.35407848676961) bank65488 +65489 POINT(39.96109217294573 73.92334368292973) bank65489 +65490 POINT(41.03482497977385 74.08210534830354) bank65490 +65491 POINT(40.52164570667182 73.37278578545491) bank65491 +65492 POINT(41.28728697860686 73.59013517476554) bank65492 +65493 POINT(41.521651416098955 74.12314998672201) bank65493 +65494 POINT(40.43502839399969 73.08069687556086) bank65494 +65495 POINT(40.072513520746305 73.8415343360258) bank65495 +65496 POINT(41.495474264511266 74.00918946659105) bank65496 +65497 POINT(41.50864532454011 74.16706823226598) bank65497 +65498 POINT(40.59213168466613 73.89234217324955) bank65498 +65499 POINT(39.923832006811004 73.31181893729904) bank65499 +65500 POINT(40.24799384382785 74.97282644403809) bank65500 +65501 POINT(40.37101678676158 74.3571391955567) bank65501 +65502 POINT(40.43322153366002 73.81148358271228) bank65502 +65503 POINT(40.38140094823986 74.26622035404894) bank65503 +65504 POINT(39.79415645369141 73.2231898393915) bank65504 +65505 POINT(41.21806939745384 74.89005054603729) bank65505 +65506 POINT(40.80806713788407 73.63825572125188) bank65506 +65507 POINT(40.50397549438024 73.73310773288357) bank65507 +65508 POINT(39.753639230419694 74.73695194909912) bank65508 +65509 POINT(40.577978902945055 73.50153125294648) bank65509 +65510 POINT(40.247680039136284 74.83399508150784) bank65510 +65511 POINT(41.450246753325835 73.9498751042399) bank65511 +65512 POINT(41.3832823083265 74.05826109033123) bank65512 +65513 POINT(41.426327050639514 74.68214052067846) bank65513 +65514 POINT(39.85921846960991 74.66870640875918) bank65514 +65515 POINT(40.19066761661492 74.30103408947728) bank65515 +65516 POINT(40.135046798233304 73.61174611866535) bank65516 +65517 POINT(41.656231071061 74.2221889006341) bank65517 +65518 POINT(40.10075921993296 74.27579784939411) bank65518 +65519 POINT(39.92915810368245 74.28440281637421) bank65519 +65520 POINT(40.16934733870477 74.0109539536217) bank65520 +65521 POINT(40.86414724623197 74.45538617510483) bank65521 +65522 POINT(39.86577551582626 74.23953315637178) bank65522 +65523 POINT(40.55247471790094 74.69870112936758) bank65523 +65524 POINT(40.0403352922069 73.7128883247008) bank65524 +65525 POINT(41.01741434622905 73.7936660842094) bank65525 +65526 POINT(41.20541468825063 74.53050493563599) bank65526 +65527 POINT(40.38247491072916 74.95062980680498) bank65527 +65528 POINT(41.52968782336263 74.56042054599392) bank65528 +65529 POINT(40.713923854074125 73.97665196292279) bank65529 +65530 POINT(40.67480183471561 74.86243542894259) bank65530 +65531 POINT(40.09973626938299 73.90333165766006) bank65531 +65532 POINT(41.18490358672857 74.20426895853085) bank65532 +65533 POINT(41.3312576442375 74.15570711604316) bank65533 +65534 POINT(41.136136636764135 73.41802637246568) bank65534 +65535 POINT(40.937031355263116 73.45949330130863) bank65535 +65536 POINT(40.52626917684591 73.24596296611588) bank65536 +65537 POINT(39.84173394414469 74.6716904729348) bank65537 +65538 POINT(41.250229660146736 73.49463853540304) bank65538 +65539 POINT(39.96703134089825 73.46663314574502) bank65539 +65540 POINT(41.61223233748228 74.63414357949794) bank65540 +65541 POINT(41.33416581859166 74.19977853894468) bank65541 +65542 POINT(40.37057694445593 74.63017749645051) bank65542 +65543 POINT(40.17957383674657 73.81321268325581) bank65543 +65544 POINT(40.93248586659628 74.63075423679838) bank65544 +65545 POINT(41.3697289625068 74.00083341989946) bank65545 +65546 POINT(39.813085304739666 73.94046252403463) bank65546 +65547 POINT(41.58181355391534 74.54093226381097) bank65547 +65548 POINT(40.829374341894344 73.31851390361886) bank65548 +65549 POINT(39.847316242440705 74.2584445158138) bank65549 +65550 POINT(41.01915902112752 73.40847880310577) bank65550 +65551 POINT(41.535617335031006 73.60500165967194) bank65551 +65552 POINT(40.857302262444954 74.49104847682385) bank65552 +65553 POINT(40.28472593720648 73.26624406920175) bank65553 +65554 POINT(41.18947101524794 74.70562542810356) bank65554 +65555 POINT(41.19068575754033 73.74870367558853) bank65555 +65556 POINT(40.60982911678204 74.66679449519268) bank65556 +65557 POINT(39.74104332679186 73.03397376671516) bank65557 +65558 POINT(41.12138507034137 74.57983924898768) bank65558 +65559 POINT(40.76612033574771 74.10327127738319) bank65559 +65560 POINT(41.56006653374503 74.22508645962823) bank65560 +65561 POINT(40.96379084109549 73.72926285577911) bank65561 +65562 POINT(41.468289126303844 74.43147857901685) bank65562 +65563 POINT(39.79426594643023 74.4616441574916) bank65563 +65564 POINT(39.73149363857922 74.99980961332147) bank65564 +65565 POINT(41.17721684735444 74.29178257958638) bank65565 +65566 POINT(40.10293560662855 74.64240060731522) bank65566 +65567 POINT(41.13289173854597 74.15572302104016) bank65567 +65568 POINT(40.12697885733009 73.50781655908135) bank65568 +65569 POINT(40.10025477228414 74.8428942519782) bank65569 +65570 POINT(40.34450914570087 73.19907855944271) bank65570 +65571 POINT(41.302951080017145 73.87185001517899) bank65571 +65572 POINT(40.518095134577564 73.08466574974145) bank65572 +65573 POINT(41.16132759950468 73.37130819522083) bank65573 +65574 POINT(40.3791340108218 73.73758487358025) bank65574 +65575 POINT(41.16065474060809 74.20925230321667) bank65575 +65576 POINT(39.78111665010061 73.62712791917502) bank65576 +65577 POINT(40.07114641494133 74.87162784310614) bank65577 +65578 POINT(41.45325211689662 73.43508412604373) bank65578 +65579 POINT(40.743302923167334 73.03069157671877) bank65579 +65580 POINT(41.004778845986145 73.39003174129898) bank65580 +65581 POINT(41.24204999532254 73.05420181273186) bank65581 +65582 POINT(41.511199504304656 74.81508685237786) bank65582 +65583 POINT(40.55359713554323 73.34226968924528) bank65583 +65584 POINT(41.07584327631254 73.85511811777897) bank65584 +65585 POINT(41.09793735317605 74.87360654402825) bank65585 +65586 POINT(41.69640922570364 73.63911102158359) bank65586 +65587 POINT(41.4771621284215 74.11412300268616) bank65587 +65588 POINT(40.822726402042626 73.03549255490624) bank65588 +65589 POINT(40.68643486386693 73.3923329253069) bank65589 +65590 POINT(41.69617277258562 73.78307833672791) bank65590 +65591 POINT(40.20794152046764 74.544934655649) bank65591 +65592 POINT(41.13371781922648 73.0134096743968) bank65592 +65593 POINT(39.84416073314985 73.72975487706762) bank65593 +65594 POINT(40.833892790742375 74.90895805540931) bank65594 +65595 POINT(41.39288490391037 73.56189696576854) bank65595 +65596 POINT(39.83165443388411 73.45281229596917) bank65596 +65597 POINT(40.88530949687266 73.65105438202399) bank65597 +65598 POINT(40.712308904735664 74.99774241277926) bank65598 +65599 POINT(40.32830805837006 73.9670689214612) bank65599 +65600 POINT(40.24896805888605 73.61307227495142) bank65600 +65601 POINT(40.80127466095663 74.80947817720094) bank65601 +65602 POINT(40.1427266891606 74.92186111467085) bank65602 +65603 POINT(40.62171977801743 73.76659806692476) bank65603 +65604 POINT(40.510023142672004 73.39385350225666) bank65604 +65605 POINT(40.09697324611389 74.5727639226111) bank65605 +65606 POINT(40.476738450848174 74.24520004922394) bank65606 +65607 POINT(41.64718445462924 73.45923280975373) bank65607 +65608 POINT(40.89545378941212 73.65624183809854) bank65608 +65609 POINT(39.94611676844692 73.7548130691094) bank65609 +65610 POINT(40.245266126238185 74.27189092969346) bank65610 +65611 POINT(41.44335791596527 74.725375340274) bank65611 +65612 POINT(39.759119818744566 74.62971185750392) bank65612 +65613 POINT(40.44780263591253 73.02470682564474) bank65613 +65614 POINT(41.07217814532747 74.25259870984507) bank65614 +65615 POINT(41.21396965893903 74.60587231881436) bank65615 +65616 POINT(41.332139337702564 73.91320426336095) bank65616 +65617 POINT(41.101778283236015 74.13057278361833) bank65617 +65618 POINT(40.16737425936962 73.48755629303278) bank65618 +65619 POINT(39.777214202051255 74.37798231174774) bank65619 +65620 POINT(40.59135687478513 74.42994975972026) bank65620 +65621 POINT(40.70813471821221 73.67525082510811) bank65621 +65622 POINT(40.89351835845617 73.6406030485586) bank65622 +65623 POINT(39.93105881158247 74.98899143714488) bank65623 +65624 POINT(40.5826111562528 73.49466702061252) bank65624 +65625 POINT(40.30594763689554 73.46051752349425) bank65625 +65626 POINT(40.96818018919565 73.9330732316769) bank65626 +65627 POINT(41.26042745707289 74.01830307710483) bank65627 +65628 POINT(41.041486418696586 74.92989768880639) bank65628 +65629 POINT(41.22741706338965 74.27232699391696) bank65629 +65630 POINT(40.00757892066915 74.77561705897904) bank65630 +65631 POINT(39.983390018349944 74.86472914373493) bank65631 +65632 POINT(40.17462522964115 74.98355755115259) bank65632 +65633 POINT(40.18105658177687 73.25231878694609) bank65633 +65634 POINT(40.97014878176655 73.48552790048882) bank65634 +65635 POINT(40.16435572230747 74.78847949652214) bank65635 +65636 POINT(41.18896379207572 74.01360438144047) bank65636 +65637 POINT(41.32351807795545 73.87288412581779) bank65637 +65638 POINT(41.0958716887408 74.34942159313594) bank65638 +65639 POINT(41.30997792614836 73.23824258706335) bank65639 +65640 POINT(40.68879419848792 73.53478074971217) bank65640 +65641 POINT(39.96625747018571 74.67076358218095) bank65641 +65642 POINT(39.834446111437934 73.11021022212114) bank65642 +65643 POINT(41.6532762285691 73.7931678745549) bank65643 +65644 POINT(40.28201390243725 73.67367442026868) bank65644 +65645 POINT(40.914339921253706 73.72524513236363) bank65645 +65646 POINT(40.72227991243335 74.4931549855213) bank65646 +65647 POINT(41.49355772875025 74.36695402709938) bank65647 +65648 POINT(39.805108396159014 73.00903192465948) bank65648 +65649 POINT(39.91801695301014 74.16415426502371) bank65649 +65650 POINT(41.43100303062628 74.3742649788922) bank65650 +65651 POINT(41.700753865154276 73.4251600414266) bank65651 +65652 POINT(41.669389728369794 74.59747163356921) bank65652 +65653 POINT(39.71464242875788 74.36970582034948) bank65653 +65654 POINT(41.18399242152949 73.52002587863927) bank65654 +65655 POINT(40.3392764077421 73.9965422264923) bank65655 +65656 POINT(39.75911732943656 73.65193338343121) bank65656 +65657 POINT(41.60848857783282 73.52299103149343) bank65657 +65658 POINT(40.82903423304581 74.10632794143706) bank65658 +65659 POINT(40.002685052000835 74.4152899075852) bank65659 +65660 POINT(40.014380420982896 74.03607966064757) bank65660 +65661 POINT(41.40162479741642 74.40693232276439) bank65661 +65662 POINT(40.92686160001519 74.76520611651941) bank65662 +65663 POINT(41.052850816862254 74.43549874132218) bank65663 +65664 POINT(40.19066045358032 74.52451226303495) bank65664 +65665 POINT(40.00430122494498 74.22385993802054) bank65665 +65666 POINT(39.76346466259127 73.23646822122149) bank65666 +65667 POINT(39.96747480127438 73.9018082161109) bank65667 +65668 POINT(41.67673228681894 74.75796111104665) bank65668 +65669 POINT(40.43839094502866 74.73440619528729) bank65669 +65670 POINT(40.162687408086995 74.54241088738895) bank65670 +65671 POINT(40.32609693519233 74.3291541660746) bank65671 +65672 POINT(40.78415758685067 73.09405322767215) bank65672 +65673 POINT(41.5390044651076 74.47188694191085) bank65673 +65674 POINT(39.847839359008425 73.01130259712303) bank65674 +65675 POINT(41.04107518200385 74.44536685052526) bank65675 +65676 POINT(41.066081316389905 73.70925119064452) bank65676 +65677 POINT(40.03623803371317 74.60558350767765) bank65677 +65678 POINT(40.19774362525814 74.97393103291661) bank65678 +65679 POINT(41.539429242619825 73.30911951013164) bank65679 +65680 POINT(41.30250825524049 73.01936986058976) bank65680 +65681 POINT(41.033794229055744 74.59309850479964) bank65681 +65682 POINT(41.086294943819425 74.66108414004601) bank65682 +65683 POINT(41.3475562524159 74.42394737612443) bank65683 +65684 POINT(40.59472765326905 73.09949981341015) bank65684 +65685 POINT(40.18198450851735 73.20151835645832) bank65685 +65686 POINT(41.600588565667046 73.9987608062589) bank65686 +65687 POINT(39.92768871075319 73.83252198749075) bank65687 +65688 POINT(39.85612845768411 73.55460268870556) bank65688 +65689 POINT(40.855616771951794 73.54269732982165) bank65689 +65690 POINT(41.49164061631268 73.22237844458438) bank65690 +65691 POINT(39.7226661108622 73.74997033289452) bank65691 +65692 POINT(40.662603330778666 73.49165062699292) bank65692 +65693 POINT(41.20044422790173 73.33438663247534) bank65693 +65694 POINT(40.38850686834621 74.10718514363295) bank65694 +65695 POINT(41.131541653207734 74.0566081424789) bank65695 +65696 POINT(41.08324809947054 73.66497862202185) bank65696 +65697 POINT(39.891368074489726 73.71026758692318) bank65697 +65698 POINT(40.6421349966767 74.87426777349106) bank65698 +65699 POINT(41.7016327645203 74.16554555538936) bank65699 +65700 POINT(41.04299963344499 73.2521566544453) bank65700 +65701 POINT(40.6202464109703 73.53334141642357) bank65701 +65702 POINT(41.095196690626345 74.10351430644998) bank65702 +65703 POINT(40.447928547445954 74.75497468625869) bank65703 +65704 POINT(41.35212971364336 73.68526182021307) bank65704 +65705 POINT(41.5315611219524 74.18322111055586) bank65705 +65706 POINT(40.41628883185224 74.75373598076902) bank65706 +65707 POINT(40.49558354789564 74.84358932577756) bank65707 +65708 POINT(41.60626074885953 73.39582307464315) bank65708 +65709 POINT(39.77850246663952 74.38797162424764) bank65709 +65710 POINT(40.7167858523864 73.68303619439864) bank65710 +65711 POINT(41.32718006626378 74.87316090426172) bank65711 +65712 POINT(40.4139347469683 73.03127480595951) bank65712 +65713 POINT(39.915847761912886 74.62286546614708) bank65713 +65714 POINT(39.7697269451971 73.17572326846542) bank65714 +65715 POINT(41.229140505275204 73.46469296018329) bank65715 +65716 POINT(40.55351810736544 73.84071474628813) bank65716 +65717 POINT(40.506655398457475 73.23142427534094) bank65717 +65718 POINT(41.45952210074113 74.00578816090417) bank65718 +65719 POINT(41.668331124905755 74.39585521891182) bank65719 +65720 POINT(40.02441511170368 74.46812029478066) bank65720 +65721 POINT(40.74033951807275 74.43258995194552) bank65721 +65722 POINT(39.928642902742304 73.10009258827887) bank65722 +65723 POINT(41.56058670154549 74.0735451753908) bank65723 +65724 POINT(40.962773925114064 73.04179343985938) bank65724 +65725 POINT(39.9026719679931 74.22299602887269) bank65725 +65726 POINT(40.9474574968426 73.39064749927498) bank65726 +65727 POINT(40.21942315000643 73.46377893852724) bank65727 +65728 POINT(40.99237151372151 74.2200260388011) bank65728 +65729 POINT(40.78936484064016 74.42190979665476) bank65729 +65730 POINT(40.346985186365366 73.73452595827321) bank65730 +65731 POINT(41.049562593722435 74.7075858944658) bank65731 +65732 POINT(40.09487054972853 73.63789854301152) bank65732 +65733 POINT(39.99940486957935 74.50188143776731) bank65733 +65734 POINT(40.52169872946856 73.21334071156429) bank65734 +65735 POINT(41.29610640796299 73.9250120800205) bank65735 +65736 POINT(40.47906483189921 74.29209417684413) bank65736 +65737 POINT(41.02431293350813 74.31477251096581) bank65737 +65738 POINT(41.062456124223296 74.7020711662835) bank65738 +65739 POINT(39.898188290540375 73.33340006465143) bank65739 +65740 POINT(40.328394787334545 73.93603907410277) bank65740 +65741 POINT(39.9860752654959 73.85382803178675) bank65741 +65742 POINT(41.10896465291481 74.03160826174008) bank65742 +65743 POINT(40.825968127777195 73.20105543587175) bank65743 +65744 POINT(40.91698277130264 73.60243898634214) bank65744 +65745 POINT(41.02417570015804 74.06345571492758) bank65745 +65746 POINT(40.40799928271182 73.27601129460766) bank65746 +65747 POINT(39.88953272855259 74.296667553271) bank65747 +65748 POINT(39.71745961507051 74.78767459790743) bank65748 +65749 POINT(41.10434635468435 74.16782185677415) bank65749 +65750 POINT(41.13790065312441 73.72302785760573) bank65750 +65751 POINT(40.09335946812042 74.78568034666877) bank65751 +65752 POINT(40.32883901949783 73.16444942402393) bank65752 +65753 POINT(41.57041572554431 73.82244504270977) bank65753 +65754 POINT(41.380045133182975 73.48959640041238) bank65754 +65755 POINT(41.344816832888135 73.7464423990462) bank65755 +65756 POINT(40.17998946157496 74.81129327702952) bank65756 +65757 POINT(39.91952453574703 74.14774327573808) bank65757 +65758 POINT(41.12880124565262 74.80059601139824) bank65758 +65759 POINT(40.243850054910865 73.2826743122421) bank65759 +65760 POINT(41.40853464402786 74.57758297140703) bank65760 +65761 POINT(40.80676567935115 73.6960189524216) bank65761 +65762 POINT(40.721846041368266 74.26494574694226) bank65762 +65763 POINT(40.81153711860406 74.93312348852093) bank65763 +65764 POINT(40.50278044326476 73.60556486241371) bank65764 +65765 POINT(41.65145432042498 74.32010517052979) bank65765 +65766 POINT(41.317079422627984 73.72357833931883) bank65766 +65767 POINT(39.79613390987809 73.96528688341829) bank65767 +65768 POINT(40.88576163817932 74.11645178230305) bank65768 +65769 POINT(40.93224028670904 74.40075729317108) bank65769 +65770 POINT(40.698840555715385 74.75914677501818) bank65770 +65771 POINT(41.11859285857391 74.15907690019989) bank65771 +65772 POINT(41.182989730176466 73.62277039508648) bank65772 +65773 POINT(40.00279961074907 73.11121421349834) bank65773 +65774 POINT(41.414344370813005 73.86961745750453) bank65774 +65775 POINT(40.708549702864396 73.0181413215991) bank65775 +65776 POINT(40.70194196081805 74.48083601641375) bank65776 +65777 POINT(40.699901064621216 73.33771590036461) bank65777 +65778 POINT(40.25606364626372 73.15316548153633) bank65778 +65779 POINT(40.4287418268419 74.08133021728251) bank65779 +65780 POINT(40.49282809248329 74.79283233242978) bank65780 +65781 POINT(40.966740167753244 73.07434524271873) bank65781 +65782 POINT(41.484564326567956 74.38372282142069) bank65782 +65783 POINT(40.98451244146801 74.21547361164727) bank65783 +65784 POINT(40.32294320046063 74.65924492127925) bank65784 +65785 POINT(40.29553337105961 74.43437422714746) bank65785 +65786 POINT(41.68305287985214 73.73749888111377) bank65786 +65787 POINT(40.00530944688209 74.21709905399558) bank65787 +65788 POINT(40.90420398671041 74.29394843443863) bank65788 +65789 POINT(39.84347077457742 74.54326381416759) bank65789 +65790 POINT(40.63921435796339 73.74851328217494) bank65790 +65791 POINT(40.58460590139977 73.67656089792382) bank65791 +65792 POINT(40.7599660138761 74.71504763524898) bank65792 +65793 POINT(41.239151099329455 73.6359016385025) bank65793 +65794 POINT(39.948379045258875 73.17109470441558) bank65794 +65795 POINT(40.574714058190054 74.20381340755962) bank65795 +65796 POINT(41.44728407427976 73.76974738965981) bank65796 +65797 POINT(40.79373116420244 74.84891400109473) bank65797 +65798 POINT(40.56889501355398 74.56155341577004) bank65798 +65799 POINT(41.47418151519438 73.63841956291569) bank65799 +65800 POINT(40.65814171046574 74.05254290632062) bank65800 +65801 POINT(41.585692455984095 74.83992009177624) bank65801 +65802 POINT(40.22818649626525 73.24280091405832) bank65802 +65803 POINT(41.51372415626803 73.1677608954306) bank65803 +65804 POINT(41.27463092170809 74.99158644432846) bank65804 +65805 POINT(41.6999439360432 73.08298089361375) bank65805 +65806 POINT(41.54201693185874 73.87475530741511) bank65806 +65807 POINT(40.88788036248068 73.64804306889249) bank65807 +65808 POINT(39.75198970817461 74.03661236393344) bank65808 +65809 POINT(40.59245735304082 73.29800292168161) bank65809 +65810 POINT(40.49346481435235 73.12158112092823) bank65810 +65811 POINT(40.001188037568205 73.15107992033757) bank65811 +65812 POINT(41.551629051247495 73.9958807436143) bank65812 +65813 POINT(41.456288402722336 74.53693116061352) bank65813 +65814 POINT(41.685965071179126 73.98249575160733) bank65814 +65815 POINT(40.930342772594415 74.65701298583492) bank65815 +65816 POINT(40.08093488058848 74.94700693212064) bank65816 +65817 POINT(40.15135700301774 73.23950570428788) bank65817 +65818 POINT(41.170304664085414 73.47864350131138) bank65818 +65819 POINT(39.94875097423267 74.60269565402766) bank65819 +65820 POINT(40.78191904957209 74.4832457607885) bank65820 +65821 POINT(41.62188835619766 73.75689791531038) bank65821 +65822 POINT(40.69858994404459 74.7033396348223) bank65822 +65823 POINT(41.45326372834295 73.38430289828604) bank65823 +65824 POINT(41.49510278745918 73.52735760344773) bank65824 +65825 POINT(40.70396161242955 74.53271089113204) bank65825 +65826 POINT(40.63652295767822 73.32436926406596) bank65826 +65827 POINT(40.591291797658386 73.21809490445152) bank65827 +65828 POINT(40.631354655442976 73.89284893334285) bank65828 +65829 POINT(41.51289809629322 74.99941406772466) bank65829 +65830 POINT(40.828481893257035 73.44456032505533) bank65830 +65831 POINT(39.98891025049408 74.30252577013799) bank65831 +65832 POINT(41.68302218081381 73.33631457985686) bank65832 +65833 POINT(41.05500281818834 74.4173553778247) bank65833 +65834 POINT(39.83207649596736 74.26950274588363) bank65834 +65835 POINT(39.847199787191464 74.57194509006334) bank65835 +65836 POINT(40.821559352400826 74.16063038159646) bank65836 +65837 POINT(41.18044529562492 73.75355666312306) bank65837 +65838 POINT(41.64024870185735 74.8006296644062) bank65838 +65839 POINT(39.728754220612906 74.48209603972379) bank65839 +65840 POINT(40.79811312379415 73.36258676379497) bank65840 +65841 POINT(40.95909640167565 73.74920825934625) bank65841 +65842 POINT(39.7372280211929 74.7253600115916) bank65842 +65843 POINT(41.06029498645454 73.09238351672577) bank65843 +65844 POINT(40.33196775008519 74.52375133935377) bank65844 +65845 POINT(41.168272209151695 74.25768583727793) bank65845 +65846 POINT(40.01826887604956 73.53343011854477) bank65846 +65847 POINT(40.926874198444516 73.11776405344499) bank65847 +65848 POINT(41.234201412708956 73.92686153395084) bank65848 +65849 POINT(41.1172801286217 74.33256135114598) bank65849 +65850 POINT(40.74314262065464 73.00818042329907) bank65850 +65851 POINT(40.813645826010735 73.48741097740668) bank65851 +65852 POINT(41.49726541445117 73.52550776319042) bank65852 +65853 POINT(40.73290074525836 73.71252321482854) bank65853 +65854 POINT(39.750391915064924 74.54527868081878) bank65854 +65855 POINT(41.50307543302042 74.46231389601174) bank65855 +65856 POINT(40.18427972258361 73.8809891047491) bank65856 +65857 POINT(41.02385372346479 74.66008171755178) bank65857 +65858 POINT(40.68560190386468 73.18403937889141) bank65858 +65859 POINT(41.6187449160869 74.12209054083267) bank65859 +65860 POINT(41.25556353082392 74.74342663664285) bank65860 +65861 POINT(40.23148239874656 73.72461583211708) bank65861 +65862 POINT(40.911162056288774 73.06691559099112) bank65862 +65863 POINT(40.99378589312651 73.48502219723001) bank65863 +65864 POINT(41.392960403589434 73.89352991628157) bank65864 +65865 POINT(40.40644943949332 73.29109401150504) bank65865 +65866 POINT(40.23639071978051 74.6869485621764) bank65866 +65867 POINT(40.29011369884087 74.44302623649385) bank65867 +65868 POINT(41.241698215948404 73.33601106408427) bank65868 +65869 POINT(40.90235427017013 74.47558951778697) bank65869 +65870 POINT(40.44353806983438 73.65089485104582) bank65870 +65871 POINT(41.5225430204233 74.05813530618703) bank65871 +65872 POINT(40.61807014625414 74.07430883364964) bank65872 +65873 POINT(40.34284278444019 74.28071684655794) bank65873 +65874 POINT(40.98154407623875 74.18439924776804) bank65874 +65875 POINT(41.220272728251985 73.15079759347893) bank65875 +65876 POINT(40.59521693490953 74.35650788128753) bank65876 +65877 POINT(41.387508217458034 74.68900769298865) bank65877 +65878 POINT(41.63774353965772 74.76188145898153) bank65878 +65879 POINT(39.77187182400899 74.96024461994934) bank65879 +65880 POINT(39.9087980230939 73.38608073385636) bank65880 +65881 POINT(40.385983874919376 73.85947232062847) bank65881 +65882 POINT(39.82817494863438 73.61010836567769) bank65882 +65883 POINT(40.101992612496296 74.06038590394364) bank65883 +65884 POINT(41.18603281375053 74.66238005447333) bank65884 +65885 POINT(39.863440669392865 73.48482316246684) bank65885 +65886 POINT(41.425340206741005 73.31264744796546) bank65886 +65887 POINT(40.97384882184794 74.47219502596369) bank65887 +65888 POINT(39.7428847252517 73.52434566795462) bank65888 +65889 POINT(39.73806718310412 73.01716737282835) bank65889 +65890 POINT(41.48567981811492 74.33777358646928) bank65890 +65891 POINT(40.676549162791865 73.9581797968607) bank65891 +65892 POINT(39.89559144185352 74.73650049635114) bank65892 +65893 POINT(40.6217836987966 74.39306792271192) bank65893 +65894 POINT(40.110622523417376 74.8087135864563) bank65894 +65895 POINT(40.31930507388111 74.2355141331099) bank65895 +65896 POINT(40.803461095262115 73.45341919574419) bank65896 +65897 POINT(41.519063352474966 73.16021321018238) bank65897 +65898 POINT(41.10591815196408 74.80665418255495) bank65898 +65899 POINT(40.753604897055645 73.41662480107632) bank65899 +65900 POINT(41.19258630286755 73.3417214426042) bank65900 +65901 POINT(40.36851980565095 73.54400545418329) bank65901 +65902 POINT(40.94011635635722 74.24403251060876) bank65902 +65903 POINT(39.88666718895866 73.62878112198398) bank65903 +65904 POINT(40.49266484324211 74.68598371905821) bank65904 +65905 POINT(41.37561874583655 73.72470558914507) bank65905 +65906 POINT(40.289946775604726 73.18336820995866) bank65906 +65907 POINT(40.346624379591915 74.81550030906196) bank65907 +65908 POINT(40.281207717651384 74.86117609914267) bank65908 +65909 POINT(40.05804118841892 74.09963344230805) bank65909 +65910 POINT(39.935812395899525 73.13054983958868) bank65910 +65911 POINT(40.238309901426575 74.04523628048767) bank65911 +65912 POINT(40.60824061713004 73.83931391882554) bank65912 +65913 POINT(40.667751180654875 74.05421594859791) bank65913 +65914 POINT(40.900990622219766 74.63454745080396) bank65914 +65915 POINT(40.556581754818104 74.79442488808965) bank65915 +65916 POINT(41.119310383613225 74.35302567079415) bank65916 +65917 POINT(39.911252255895434 73.10392696206351) bank65917 +65918 POINT(40.733120936396475 74.33630954837791) bank65918 +65919 POINT(41.422938888384344 74.87256482458925) bank65919 +65920 POINT(39.813997207846235 73.57489308133333) bank65920 +65921 POINT(39.85539317940208 73.64793256688004) bank65921 +65922 POINT(41.69913480567278 74.08708021816174) bank65922 +65923 POINT(41.2616507635114 74.93809921697752) bank65923 +65924 POINT(41.583962700636086 74.835680890771) bank65924 +65925 POINT(40.931647558382906 73.036226943732) bank65925 +65926 POINT(39.930340130745805 74.58156739288572) bank65926 +65927 POINT(41.055648238267786 73.28347907553426) bank65927 +65928 POINT(40.24596868784629 74.86934970051752) bank65928 +65929 POINT(40.809577801986954 73.20361984881025) bank65929 +65930 POINT(41.07719684277007 74.77733822474056) bank65930 +65931 POINT(39.921764676672666 73.6139103047967) bank65931 +65932 POINT(41.62266828944873 73.97603751616613) bank65932 +65933 POINT(40.34262151070112 73.92444887539679) bank65933 +65934 POINT(40.24519398429552 73.89529263192263) bank65934 +65935 POINT(39.73564312681826 74.12581214365146) bank65935 +65936 POINT(40.55133645633916 74.57155317590963) bank65936 +65937 POINT(40.56341099965356 73.51444114039244) bank65937 +65938 POINT(40.58049126286847 74.0079874844876) bank65938 +65939 POINT(41.3947252028842 74.38228593165753) bank65939 +65940 POINT(40.944081298379665 73.06535761257322) bank65940 +65941 POINT(40.37894409780591 73.6063007057772) bank65941 +65942 POINT(40.733850870572255 74.24748334495477) bank65942 +65943 POINT(40.761806662630306 74.24037495587393) bank65943 +65944 POINT(40.56335685946499 73.79055844295142) bank65944 +65945 POINT(41.35986770457894 74.0203828487251) bank65945 +65946 POINT(41.49520289931179 73.8927603055796) bank65946 +65947 POINT(41.21647846657639 74.84385926760241) bank65947 +65948 POINT(41.697487280745705 74.93943116390685) bank65948 +65949 POINT(40.927877175831945 74.76699743845117) bank65949 +65950 POINT(40.46669897248597 73.09661374574445) bank65950 +65951 POINT(41.50080264703411 73.36034862226255) bank65951 +65952 POINT(41.52651834525112 73.11747476148236) bank65952 +65953 POINT(40.207718352919336 74.97423220189368) bank65953 +65954 POINT(40.38543333224186 74.23829377100378) bank65954 +65955 POINT(39.80720788089094 74.37998857692259) bank65955 +65956 POINT(40.92160846304891 73.78358267967636) bank65956 +65957 POINT(40.21632592743183 74.82939987854881) bank65957 +65958 POINT(40.49571415744971 73.64385232578455) bank65958 +65959 POINT(39.98730419056752 74.42256525394075) bank65959 +65960 POINT(40.5308932433436 73.71517010006356) bank65960 +65961 POINT(40.48700219998613 73.55743511053159) bank65961 +65962 POINT(40.09504970243682 73.06611791875909) bank65962 +65963 POINT(40.05617391346148 73.16795711023346) bank65963 +65964 POINT(40.33802592790049 74.05171628007717) bank65964 +65965 POINT(40.35500205963848 74.81724121870795) bank65965 +65966 POINT(40.16123410063483 74.0476010782362) bank65966 +65967 POINT(39.922532977752574 73.74596648775632) bank65967 +65968 POINT(41.68051789167877 74.14159902434048) bank65968 +65969 POINT(41.43468182753567 73.29611079553194) bank65969 +65970 POINT(40.09563820998264 73.18495086955559) bank65970 +65971 POINT(41.46268154984548 73.55520497348333) bank65971 +65972 POINT(40.58642172927312 73.14531923774476) bank65972 +65973 POINT(41.09127073891531 74.59126152696646) bank65973 +65974 POINT(40.71796525647777 73.91783784299997) bank65974 +65975 POINT(40.89886586087276 74.90467649892457) bank65975 +65976 POINT(40.97778595238348 74.47072124301286) bank65976 +65977 POINT(40.02276644253924 73.19700560046456) bank65977 +65978 POINT(40.476989285283274 74.1091697801864) bank65978 +65979 POINT(41.03282996483974 74.06606609519059) bank65979 +65980 POINT(40.08490693201339 74.40225880965396) bank65980 +65981 POINT(40.137286141368826 73.74990911064704) bank65981 +65982 POINT(41.366482797233964 73.34788858384096) bank65982 +65983 POINT(40.75641760437263 73.6108618119863) bank65983 +65984 POINT(41.334796692949915 73.6709570049681) bank65984 +65985 POINT(41.48709409265073 73.63262852277617) bank65985 +65986 POINT(40.69015859992732 74.95925173240751) bank65986 +65987 POINT(40.34240825193662 74.99269431434531) bank65987 +65988 POINT(41.47344630764028 73.3901074533812) bank65988 +65989 POINT(40.55820348135748 74.4416574063825) bank65989 +65990 POINT(41.44441778743676 74.3980033187001) bank65990 +65991 POINT(41.44537864095748 73.69891262596806) bank65991 +65992 POINT(39.7571211784016 73.30749067552506) bank65992 +65993 POINT(40.553882278179074 73.83524127455121) bank65993 +65994 POINT(40.27378237758759 74.04743308332765) bank65994 +65995 POINT(41.563766677542205 74.26675373102331) bank65995 +65996 POINT(40.728688554271 74.69912126091863) bank65996 +65997 POINT(41.483029893364346 74.38021074574712) bank65997 +65998 POINT(39.83604783492397 74.48914238692738) bank65998 +65999 POINT(40.360620749407 73.15169150064364) bank65999 +66000 POINT(40.323825484284406 73.65172476044668) bank66000 +66001 POINT(40.37932349291354 74.32613487683128) bank66001 +66002 POINT(41.49000736669518 73.2083673979203) bank66002 +66003 POINT(40.69415250227394 74.86368534506681) bank66003 +66004 POINT(40.92510022546523 74.76468317103792) bank66004 +66005 POINT(41.26781070639814 74.7690666495294) bank66005 +66006 POINT(40.606241837654835 73.98414983709942) bank66006 +66007 POINT(40.99822300955647 73.79763099269694) bank66007 +66008 POINT(40.770946761377154 73.42970588946372) bank66008 +66009 POINT(40.59939930971616 73.48883353265599) bank66009 +66010 POINT(40.26758224770269 73.8258509117536) bank66010 +66011 POINT(40.12764580381988 73.56774332018611) bank66011 +66012 POINT(40.12346391642433 73.93011463452538) bank66012 +66013 POINT(40.27625269978923 73.39788743320115) bank66013 +66014 POINT(39.76183377883451 73.04436171707522) bank66014 +66015 POINT(40.72436539071994 74.02575556214752) bank66015 +66016 POINT(40.44197731533072 73.84506784782764) bank66016 +66017 POINT(41.15283699838463 73.10211928225662) bank66017 +66018 POINT(40.50901791010451 74.96361019955516) bank66018 +66019 POINT(40.263528875672215 73.51462983539193) bank66019 +66020 POINT(39.92153049221187 73.92411577525118) bank66020 +66021 POINT(40.55906951452809 74.09489831239145) bank66021 +66022 POINT(41.178290523453605 74.83797726543678) bank66022 +66023 POINT(40.965690391472144 73.05077983053303) bank66023 +66024 POINT(40.92818106955829 73.99401846739563) bank66024 +66025 POINT(40.76856210350725 73.25988604640166) bank66025 +66026 POINT(40.07433486697788 73.46960785294058) bank66026 +66027 POINT(40.22930310436988 73.26956149844993) bank66027 +66028 POINT(41.00292647427325 74.12098798128513) bank66028 +66029 POINT(41.1988284743307 73.62387552674872) bank66029 +66030 POINT(41.26085065795245 74.39976668013857) bank66030 +66031 POINT(40.476735671605304 73.76801817747624) bank66031 +66032 POINT(40.41473678290359 74.73166358869877) bank66032 +66033 POINT(40.82980756626478 74.51023702710513) bank66033 +66034 POINT(40.98568952374719 74.93352343627572) bank66034 +66035 POINT(40.23921483190906 73.37428854131649) bank66035 +66036 POINT(40.38933825950202 74.27002387919653) bank66036 +66037 POINT(39.93544374451863 74.65798546706102) bank66037 +66038 POINT(40.09259209923732 74.81822276903583) bank66038 +66039 POINT(40.40321480510334 74.9217897497402) bank66039 +66040 POINT(41.520027868825856 73.29874662136794) bank66040 +66041 POINT(40.537379148717385 74.8596631105097) bank66041 +66042 POINT(40.35628030086868 73.31380819104662) bank66042 +66043 POINT(40.03816077070008 74.42009792649851) bank66043 +66044 POINT(39.928934548374265 73.14629167844663) bank66044 +66045 POINT(40.48608671349339 73.64291721310639) bank66045 +66046 POINT(40.96849466099849 74.23484826154176) bank66046 +66047 POINT(41.395205360644454 74.18365556972904) bank66047 +66048 POINT(39.78114337657499 73.51275770922851) bank66048 +66049 POINT(39.73783256167207 74.72771552098732) bank66049 +66050 POINT(40.85620714031755 74.34823375137238) bank66050 +66051 POINT(40.467189126558964 73.48120091948881) bank66051 +66052 POINT(40.42489402212566 74.35607931429129) bank66052 +66053 POINT(40.58620196058298 74.22912217982832) bank66053 +66054 POINT(41.00935883615542 74.64590275339249) bank66054 +66055 POINT(41.2343201580589 74.82349465962544) bank66055 +66056 POINT(41.68069340402685 74.05067310624486) bank66056 +66057 POINT(39.79826428302298 73.75358930448256) bank66057 +66058 POINT(41.610991965250584 73.79991427596403) bank66058 +66059 POINT(40.79736365040634 73.99324599090366) bank66059 +66060 POINT(40.84127351443902 73.66424598709325) bank66060 +66061 POINT(41.183365962149004 73.71853750245059) bank66061 +66062 POINT(40.683691484327326 73.51308327716352) bank66062 +66063 POINT(40.079012283555414 74.61165877552929) bank66063 +66064 POINT(41.398707607688536 73.5700173588884) bank66064 +66065 POINT(41.5353976046015 73.02329963895218) bank66065 +66066 POINT(41.70857750434062 73.93810619861613) bank66066 +66067 POINT(40.107286211630885 73.57618897267896) bank66067 +66068 POINT(41.06478772100042 74.35156616218674) bank66068 +66069 POINT(41.17230291669192 73.01732689309931) bank66069 +66070 POINT(40.44649636692826 73.89152331410976) bank66070 +66071 POINT(39.97395796498545 74.07048480464312) bank66071 +66072 POINT(40.15970116940964 73.49115076426183) bank66072 +66073 POINT(40.371960591526296 73.89491805800108) bank66073 +66074 POINT(40.56714963566854 74.56863538398964) bank66074 +66075 POINT(40.42936070705389 73.7585271030139) bank66075 +66076 POINT(40.32554011511094 73.3955732206221) bank66076 +66077 POINT(40.2597264456152 74.65053938551283) bank66077 +66078 POINT(40.919859616911396 74.26435038759902) bank66078 +66079 POINT(40.748062992015356 74.71306307097791) bank66079 +66080 POINT(40.559960705328706 73.43591571545089) bank66080 +66081 POINT(40.55268932041689 73.07939454156049) bank66081 +66082 POINT(41.30624056367445 73.14683569956905) bank66082 +66083 POINT(39.96139995831068 73.2447318754865) bank66083 +66084 POINT(40.28589135277251 74.50333724373942) bank66084 +66085 POINT(41.27473677644124 74.4067240699814) bank66085 +66086 POINT(40.018730873960315 74.41521224173167) bank66086 +66087 POINT(39.98228029119128 74.37942226022716) bank66087 +66088 POINT(41.65224536932518 73.5216552990442) bank66088 +66089 POINT(39.908571564638244 73.30408305059152) bank66089 +66090 POINT(41.62043783280006 73.85525193136034) bank66090 +66091 POINT(40.290552862657215 73.49024688350144) bank66091 +66092 POINT(39.925735668748196 73.81152835434169) bank66092 +66093 POINT(39.856900630785866 74.78371352124131) bank66093 +66094 POINT(41.19027397440477 73.50468036741789) bank66094 +66095 POINT(41.556293295877 74.2107420962605) bank66095 +66096 POINT(41.3108013013595 73.86867596561906) bank66096 +66097 POINT(41.43728776559761 74.89811560143791) bank66097 +66098 POINT(41.68164968070085 74.87696828765164) bank66098 +66099 POINT(40.54688559578253 74.34087776564076) bank66099 +66100 POINT(40.05646844511302 74.34206167777258) bank66100 +66101 POINT(39.75943875422774 73.12669155957038) bank66101 +66102 POINT(41.265640927439776 73.46507306670702) bank66102 +66103 POINT(40.04735149802684 74.76057032646867) bank66103 +66104 POINT(41.664817303719985 74.71937710506955) bank66104 +66105 POINT(41.57290350937041 73.96480353214112) bank66105 +66106 POINT(40.87778416531768 73.67835004373163) bank66106 +66107 POINT(41.52406775011969 73.4125152293003) bank66107 +66108 POINT(40.381280000483585 73.24450221275278) bank66108 +66109 POINT(40.841472080347785 74.79406523514554) bank66109 +66110 POINT(40.33810572390677 73.08866854051364) bank66110 +66111 POINT(41.5123801755274 74.88224595219171) bank66111 +66112 POINT(41.033725762811905 74.51855237327668) bank66112 +66113 POINT(40.54326587470387 73.87782956377731) bank66113 +66114 POINT(39.75329877728724 74.19718237635995) bank66114 +66115 POINT(41.10485312956423 74.97512739627481) bank66115 +66116 POINT(40.5900459379069 73.45351827638275) bank66116 +66117 POINT(40.27748030801954 73.3001808155644) bank66117 +66118 POINT(40.9382276381769 73.5453118641304) bank66118 +66119 POINT(41.496428549024095 73.36068646869226) bank66119 +66120 POINT(41.43611705305939 74.570507733248) bank66120 +66121 POINT(41.334621733229454 74.63126558720049) bank66121 +66122 POINT(41.43863105977414 73.29608902236419) bank66122 +66123 POINT(39.964510232028616 74.33808812810648) bank66123 +66124 POINT(40.72526561279094 74.81849161495283) bank66124 +66125 POINT(41.67033906375407 73.0627588739411) bank66125 +66126 POINT(40.13843660714917 74.16169631590898) bank66126 +66127 POINT(40.30842929364755 73.78419795419477) bank66127 +66128 POINT(39.753716348121145 73.43215194096852) bank66128 +66129 POINT(41.68154752932464 74.13848836806915) bank66129 +66130 POINT(40.92335810250192 73.89006034270452) bank66130 +66131 POINT(40.30264752864839 74.76502553965454) bank66131 +66132 POINT(41.25858973599634 73.27402932677953) bank66132 +66133 POINT(41.447965758091264 73.8264035717399) bank66133 +66134 POINT(40.69084373341397 74.94270238782556) bank66134 +66135 POINT(40.15688926785921 74.73695510563252) bank66135 +66136 POINT(40.93836464954508 73.45508422880421) bank66136 +66137 POINT(40.67105338739757 74.89036867482542) bank66137 +66138 POINT(40.468595829384824 74.00121923213909) bank66138 +66139 POINT(41.34540699212392 73.95615723232754) bank66139 +66140 POINT(40.35808918994945 73.66757493149842) bank66140 +66141 POINT(41.3557830246024 73.10838975385317) bank66141 +66142 POINT(41.015210043890974 74.18158477685382) bank66142 +66143 POINT(40.681647118355556 74.8637118904276) bank66143 +66144 POINT(41.38135967783461 74.39062417809409) bank66144 +66145 POINT(40.98602137870876 73.94097824313016) bank66145 +66146 POINT(40.027137299545494 74.7778300855088) bank66146 +66147 POINT(40.54672673180177 73.04522344871245) bank66147 +66148 POINT(40.330980363749255 73.9642408596576) bank66148 +66149 POINT(40.76702451735591 74.1606502391557) bank66149 +66150 POINT(39.8952096608266 74.44506649478896) bank66150 +66151 POINT(39.814889554221516 74.3333093457149) bank66151 +66152 POINT(40.08560705792383 73.05094390372584) bank66152 +66153 POINT(40.64104474000368 73.25225451181201) bank66153 +66154 POINT(39.71572245846833 74.56379422124799) bank66154 +66155 POINT(40.51808361616164 73.7379309609123) bank66155 +66156 POINT(41.384904305416846 73.83584366602075) bank66156 +66157 POINT(39.90748039411283 73.17124237112932) bank66157 +66158 POINT(40.99690500980059 73.5681998925567) bank66158 +66159 POINT(41.54253444013848 74.12437167758783) bank66159 +66160 POINT(40.203590648735336 73.75384084424482) bank66160 +66161 POINT(41.22578611576897 74.27184220063963) bank66161 +66162 POINT(41.17997861135335 74.19547061636287) bank66162 +66163 POINT(40.731720960732225 74.74872542655548) bank66163 +66164 POINT(40.04984163610698 74.47078627713218) bank66164 +66165 POINT(40.852465268674564 74.49552275255115) bank66165 +66166 POINT(41.51046691004158 74.25066661806497) bank66166 +66167 POINT(40.82181193476231 74.72528336429217) bank66167 +66168 POINT(40.795013526691164 74.0195595017603) bank66168 +66169 POINT(41.6546510661771 74.94201390555926) bank66169 +66170 POINT(40.40559579907843 74.31886930326564) bank66170 +66171 POINT(41.67142229017017 74.30927452742803) bank66171 +66172 POINT(39.745739440523586 73.41820373619869) bank66172 +66173 POINT(41.48734532141501 74.07187808000282) bank66173 +66174 POINT(40.16665643029687 73.91958890976457) bank66174 +66175 POINT(40.302997391002776 74.80167731874707) bank66175 +66176 POINT(41.64341329418579 73.75516809762472) bank66176 +66177 POINT(40.391530466537624 73.50482271919363) bank66177 +66178 POINT(40.39778837536326 74.73772753434511) bank66178 +66179 POINT(40.302562223154666 74.7087699229903) bank66179 +66180 POINT(40.143208640106074 74.3155676556624) bank66180 +66181 POINT(41.68057727287445 74.74354153123667) bank66181 +66182 POINT(40.83052094684671 74.43828187175673) bank66182 +66183 POINT(40.25656195098797 74.5236774298202) bank66183 +66184 POINT(40.247491104651075 73.77676861425329) bank66184 +66185 POINT(40.24515848466003 73.81250465418167) bank66185 +66186 POINT(39.730806422750916 73.6077289852818) bank66186 +66187 POINT(40.24224928976314 73.64909756330398) bank66187 +66188 POINT(40.16549365409263 73.76120087421633) bank66188 +66189 POINT(40.36564464114605 73.96867667944296) bank66189 +66190 POINT(41.13196103665515 73.21761781806697) bank66190 +66191 POINT(40.29429101008998 73.07857870235891) bank66191 +66192 POINT(40.65414621326205 74.79730870849386) bank66192 +66193 POINT(41.334434198935305 73.11666905432332) bank66193 +66194 POINT(40.416443671469956 74.5551263195532) bank66194 +66195 POINT(41.34930709791503 74.0124833045568) bank66195 +66196 POINT(40.41938619985443 74.03663833564526) bank66196 +66197 POINT(40.80621953515792 73.7430434585471) bank66197 +66198 POINT(41.10970883217957 74.73065648665539) bank66198 +66199 POINT(41.30208367902744 73.60458074417387) bank66199 +66200 POINT(41.29549537576519 73.72687753930039) bank66200 +66201 POINT(41.24568970443803 73.70524794820273) bank66201 +66202 POINT(40.52868087809417 73.45116937024046) bank66202 +66203 POINT(40.797805310514946 74.85332926346051) bank66203 +66204 POINT(39.99441663799972 73.48304787837853) bank66204 +66205 POINT(40.088650000140106 73.73717262535276) bank66205 +66206 POINT(39.86745125864821 73.01647601391427) bank66206 +66207 POINT(40.02817058152284 73.16725641631996) bank66207 +66208 POINT(41.49004859266017 74.66003565628881) bank66208 +66209 POINT(41.24842579478275 74.60643783286157) bank66209 +66210 POINT(40.83245627016367 74.38912080713905) bank66210 +66211 POINT(40.91676683786384 73.36794710866636) bank66211 +66212 POINT(41.46961585142302 73.25310494243558) bank66212 +66213 POINT(40.73433212525586 74.43205689574255) bank66213 +66214 POINT(39.75568708940727 74.9423929425818) bank66214 +66215 POINT(41.22488898316486 74.77410745777216) bank66215 +66216 POINT(40.84530394186924 74.45227137682842) bank66216 +66217 POINT(39.74494607813636 73.24779047698821) bank66217 +66218 POINT(40.80466408356441 73.26613711614415) bank66218 +66219 POINT(40.8294918302104 74.6520876963269) bank66219 +66220 POINT(40.69556852215999 73.03632897471583) bank66220 +66221 POINT(41.45583527202645 74.51355349726266) bank66221 +66222 POINT(40.15725676419493 73.02934649529945) bank66222 +66223 POINT(41.2311545805004 73.31849888260363) bank66223 +66224 POINT(40.8387613262927 73.83397940805945) bank66224 +66225 POINT(41.599724520640585 74.65953229839015) bank66225 +66226 POINT(40.427053913215644 74.5384173743316) bank66226 +66227 POINT(39.823550734546885 74.07450761163565) bank66227 +66228 POINT(41.3817122560947 74.68767542990312) bank66228 +66229 POINT(40.23971229134022 74.5018039860122) bank66229 +66230 POINT(40.397695851317366 73.85506168700303) bank66230 +66231 POINT(41.15144317048156 73.92918687287884) bank66231 +66232 POINT(40.73157111611483 74.51397706683315) bank66232 +66233 POINT(41.11381689431086 73.15788851431834) bank66233 +66234 POINT(40.497538620985885 74.69615221380283) bank66234 +66235 POINT(41.341634712710764 73.42533362143404) bank66235 +66236 POINT(40.93188546114162 73.43273686967059) bank66236 +66237 POINT(40.174772297743466 73.20621166649691) bank66237 +66238 POINT(40.84252253135762 73.12547998190374) bank66238 +66239 POINT(41.41423577683802 73.13047097259194) bank66239 +66240 POINT(40.19850695119655 73.5530448854764) bank66240 +66241 POINT(40.06476022733306 73.90749795783225) bank66241 +66242 POINT(40.59657259080968 73.87955062251434) bank66242 +66243 POINT(40.41124233706975 74.40694436188954) bank66243 +66244 POINT(40.995738074049164 73.96827225140531) bank66244 +66245 POINT(41.59044116416233 73.01156991787653) bank66245 +66246 POINT(39.87877893172727 74.0474372261819) bank66246 +66247 POINT(40.856850728590565 73.2014350906075) bank66247 +66248 POINT(41.62009633037254 73.75672788667633) bank66248 +66249 POINT(41.10339483279043 73.1748547862657) bank66249 +66250 POINT(41.11914810078024 73.38641039215813) bank66250 +66251 POINT(39.81412890819495 73.16789816443153) bank66251 +66252 POINT(41.60593391664178 74.00983317630907) bank66252 +66253 POINT(40.451605048757386 74.54658207488312) bank66253 +66254 POINT(39.811152321980956 74.84749038328133) bank66254 +66255 POINT(41.070118817062124 73.93376258636275) bank66255 +66256 POINT(41.44574229050297 73.58637397099034) bank66256 +66257 POINT(40.13518404830223 74.20033309415996) bank66257 +66258 POINT(40.79569492280737 73.69984715585561) bank66258 +66259 POINT(39.74983008950047 73.94345834277098) bank66259 +66260 POINT(40.41521923985129 74.7610315730805) bank66260 +66261 POINT(39.99465424259706 74.04449292574864) bank66261 +66262 POINT(40.05682244318006 73.71482158122267) bank66262 +66263 POINT(40.42212574317058 73.81493207234095) bank66263 +66264 POINT(41.4529473010797 73.3763879872411) bank66264 +66265 POINT(40.660750582881604 74.25485438801101) bank66265 +66266 POINT(40.994383193829606 73.50672489021633) bank66266 +66267 POINT(40.25106847432901 74.70761918161429) bank66267 +66268 POINT(40.364963773522085 74.08547088082871) bank66268 +66269 POINT(40.186208201753665 74.13941401725903) bank66269 +66270 POINT(39.968118089191485 74.68026401853986) bank66270 +66271 POINT(41.323244745989825 73.61634940403373) bank66271 +66272 POINT(41.68983898151962 74.91702586226447) bank66272 +66273 POINT(40.19650809440406 74.17948194291543) bank66273 +66274 POINT(39.761806497604034 74.22484869120362) bank66274 +66275 POINT(39.81999170673795 74.03812813067121) bank66275 +66276 POINT(40.62576180319679 73.48737222011727) bank66276 +66277 POINT(41.694712295051886 74.88686058879873) bank66277 +66278 POINT(41.3780562888979 73.06935650972324) bank66278 +66279 POINT(40.85214268049355 74.8786948907486) bank66279 +66280 POINT(39.949091081279896 73.33957544613737) bank66280 +66281 POINT(39.90345152959458 73.64190792643215) bank66281 +66282 POINT(41.52430552808083 74.36082282137443) bank66282 +66283 POINT(40.232535809332816 74.33573692634762) bank66283 +66284 POINT(40.04003184458537 74.47111091813893) bank66284 +66285 POINT(41.52665353966167 74.25534151513176) bank66285 +66286 POINT(40.34237773143132 73.48529113944679) bank66286 +66287 POINT(41.41952074118726 74.71616019527615) bank66287 +66288 POINT(40.39480888980447 73.86043755449745) bank66288 +66289 POINT(40.775428625052555 73.50207984399651) bank66289 +66290 POINT(40.38933241760857 73.63043572005355) bank66290 +66291 POINT(40.99231383516987 74.47218086681767) bank66291 +66292 POINT(41.18931263303505 74.2250399660604) bank66292 +66293 POINT(39.81986196096408 74.81320916468522) bank66293 +66294 POINT(40.5355769268241 73.36861761502351) bank66294 +66295 POINT(40.69359406882097 73.73111801442691) bank66295 +66296 POINT(41.03799735415346 73.62475022050182) bank66296 +66297 POINT(39.74550664579623 74.86041329680238) bank66297 +66298 POINT(41.204192878273545 73.89311026985939) bank66298 +66299 POINT(40.45399966568603 73.73094498075204) bank66299 +66300 POINT(40.27971532706256 73.17560716605753) bank66300 +66301 POINT(41.168505760520645 73.93411323112102) bank66301 +66302 POINT(40.17871064981024 74.98556437237936) bank66302 +66303 POINT(40.961344466004384 73.31275313443334) bank66303 +66304 POINT(39.789391713698194 74.46278788867815) bank66304 +66305 POINT(40.56292708357907 73.2745553166308) bank66305 +66306 POINT(40.05350642678934 74.21741035677977) bank66306 +66307 POINT(41.2345932142108 73.84345681678327) bank66307 +66308 POINT(40.1259524037846 74.62880200333876) bank66308 +66309 POINT(41.211446377683714 74.12391027416359) bank66309 +66310 POINT(41.03721005430067 73.96255467091802) bank66310 +66311 POINT(40.378179182006534 73.67601409172788) bank66311 +66312 POINT(40.315558245773595 74.39166079519298) bank66312 +66313 POINT(40.17390539452232 73.69369174208421) bank66313 +66314 POINT(40.58125904674181 74.47927009666444) bank66314 +66315 POINT(39.72088189374048 73.55732712781257) bank66315 +66316 POINT(40.84055038012178 73.69846654341107) bank66316 +66317 POINT(41.30112688490162 74.38550888619515) bank66317 +66318 POINT(40.04064264916614 74.42320939867902) bank66318 +66319 POINT(40.46572582808242 73.80192959362799) bank66319 +66320 POINT(40.127293754731774 73.03904531941144) bank66320 +66321 POINT(40.273034635587685 73.20044388825451) bank66321 +66322 POINT(39.745351222296954 73.28298256270045) bank66322 +66323 POINT(40.12820748478177 73.53695099878068) bank66323 +66324 POINT(40.356063047532636 74.60337450926353) bank66324 +66325 POINT(39.852952606299745 74.41304151833748) bank66325 +66326 POINT(40.014503444812306 74.51211681636094) bank66326 +66327 POINT(40.89547396106528 74.91885992413667) bank66327 +66328 POINT(41.108372318484186 74.4779514635606) bank66328 +66329 POINT(41.20628198934346 74.20870964270516) bank66329 +66330 POINT(40.67025531312526 74.14520241347995) bank66330 +66331 POINT(40.76890592401984 74.47511648397898) bank66331 +66332 POINT(41.48717215156821 73.09740854770217) bank66332 +66333 POINT(40.7232149154992 73.34053455572136) bank66333 +66334 POINT(41.55008628030976 74.43431194535991) bank66334 +66335 POINT(40.44548583113925 74.22216323549888) bank66335 +66336 POINT(40.85639717241412 74.2484358101796) bank66336 +66337 POINT(41.070939471945586 73.72066072406801) bank66337 +66338 POINT(40.73702234857883 73.04521923709808) bank66338 +66339 POINT(40.31279241011538 74.2393886920454) bank66339 +66340 POINT(41.6135399501168 73.2480286971496) bank66340 +66341 POINT(40.60729458718288 74.61088930397116) bank66341 +66342 POINT(40.5982037167679 74.51978707299722) bank66342 +66343 POINT(40.71919870422659 73.58377317593) bank66343 +66344 POINT(39.820692826934504 73.08654659549167) bank66344 +66345 POINT(41.33747788902427 73.89167127653602) bank66345 +66346 POINT(40.71093400799566 74.91802499896906) bank66346 +66347 POINT(40.99614784570634 73.39355782339582) bank66347 +66348 POINT(41.29290298484277 73.85572189501143) bank66348 +66349 POINT(41.09965720264821 73.1034365132184) bank66349 +66350 POINT(40.195875075928015 73.28328413762078) bank66350 +66351 POINT(40.812365650025356 74.71878258579903) bank66351 +66352 POINT(39.962964130558305 73.43586916947648) bank66352 +66353 POINT(40.85849505975355 73.79169942894079) bank66353 +66354 POINT(40.25842552432334 74.13419400302472) bank66354 +66355 POINT(39.97118604170783 73.6768035309584) bank66355 +66356 POINT(41.63696152262584 74.70870999020862) bank66356 +66357 POINT(41.37570562922244 74.8407376163222) bank66357 +66358 POINT(39.82326171903996 74.47493605496537) bank66358 +66359 POINT(40.02217492137966 74.27630663398357) bank66359 +66360 POINT(40.313216608302284 73.56538470051957) bank66360 +66361 POINT(39.716501371537596 73.94203232042051) bank66361 +66362 POINT(41.02878284389749 74.4743731225951) bank66362 +66363 POINT(40.337841293468294 73.78247080892662) bank66363 +66364 POINT(40.56486899229966 73.09622413950129) bank66364 +66365 POINT(40.716497308370506 74.56011809396836) bank66365 +66366 POINT(41.26911314793425 73.79932630679076) bank66366 +66367 POINT(40.70595170809607 74.35395456907796) bank66367 +66368 POINT(40.04049683170709 74.44508295675558) bank66368 +66369 POINT(41.526894671600644 73.27738072344572) bank66369 +66370 POINT(40.023526018765025 74.2831730355044) bank66370 +66371 POINT(41.31089841771872 73.65002137692342) bank66371 +66372 POINT(41.314373321440364 74.02749134907451) bank66372 +66373 POINT(41.39706006433697 74.26105454835164) bank66373 +66374 POINT(39.71513128677868 73.24512353713487) bank66374 +66375 POINT(41.035808008802306 73.8589441141362) bank66375 +66376 POINT(41.14554324050811 74.83467055090807) bank66376 +66377 POINT(41.216028430923714 74.36060900701999) bank66377 +66378 POINT(41.45571893399072 73.53113945646557) bank66378 +66379 POINT(40.96068064988155 74.9660389890727) bank66379 +66380 POINT(40.98358307853419 74.38015722540727) bank66380 +66381 POINT(41.43104297874662 73.15907446585442) bank66381 +66382 POINT(40.79951566885829 74.3460180027318) bank66382 +66383 POINT(40.190647294670825 73.25543861478138) bank66383 +66384 POINT(41.5082884759326 74.44955792704413) bank66384 +66385 POINT(40.50765067845222 73.55581775213916) bank66385 +66386 POINT(41.005445752869875 73.20150248564313) bank66386 +66387 POINT(39.944847600335216 73.10358903109817) bank66387 +66388 POINT(41.33771767177438 73.32805300716133) bank66388 +66389 POINT(41.107140826969605 73.77451200285293) bank66389 +66390 POINT(40.77514245423171 73.65038299737378) bank66390 +66391 POINT(39.77619829221144 73.6530576355299) bank66391 +66392 POINT(41.6583776797431 73.4068572869503) bank66392 +66393 POINT(40.836605739167865 74.4202330355786) bank66393 +66394 POINT(40.488417724711276 74.26262026853466) bank66394 +66395 POINT(40.80164272153837 74.40508257067434) bank66395 +66396 POINT(41.083692985701276 73.60722876580918) bank66396 +66397 POINT(40.73307648843119 74.32599002979549) bank66397 +66398 POINT(40.44811908011851 73.09836698793742) bank66398 +66399 POINT(40.01972534461598 74.12285217888713) bank66399 +66400 POINT(41.54538646429985 74.39317002568535) bank66400 +66401 POINT(41.43722428334499 73.94581247730186) bank66401 +66402 POINT(41.148293562282724 73.62266069849557) bank66402 +66403 POINT(39.75140095489518 73.44028201525498) bank66403 +66404 POINT(40.49932828060372 73.42913490965545) bank66404 +66405 POINT(40.18406671153764 73.41408115036401) bank66405 +66406 POINT(41.50389722290561 74.98852809294762) bank66406 +66407 POINT(41.3786266069508 73.57703275901758) bank66407 +66408 POINT(40.615549819879526 73.88369298194964) bank66408 +66409 POINT(41.45443574759998 74.44431720226522) bank66409 +66410 POINT(40.62263596846513 74.46498512958493) bank66410 +66411 POINT(41.2938764324516 73.25296483916283) bank66411 +66412 POINT(40.54603281939613 74.38908089360451) bank66412 +66413 POINT(40.909416449753394 74.79336369320485) bank66413 +66414 POINT(39.910752854761796 74.28688122702378) bank66414 +66415 POINT(40.22830324931103 74.90381941406332) bank66415 +66416 POINT(41.114736467132666 74.55162176316063) bank66416 +66417 POINT(41.095127954222775 74.58908950418788) bank66417 +66418 POINT(39.833773058528024 73.43757390875193) bank66418 +66419 POINT(40.69234374119703 73.74478394095578) bank66419 +66420 POINT(40.111933380746414 74.6339686605153) bank66420 +66421 POINT(39.90945019948652 74.65477687428188) bank66421 +66422 POINT(41.20186797427994 74.35281788333923) bank66422 +66423 POINT(40.25450750235947 73.21472247382377) bank66423 +66424 POINT(41.34783398060383 73.18722473211761) bank66424 +66425 POINT(40.208315998537316 73.70243950111187) bank66425 +66426 POINT(41.568501448762575 73.590843397958) bank66426 +66427 POINT(40.329943583519174 73.41646262232054) bank66427 +66428 POINT(41.54499949114295 74.48018723527989) bank66428 +66429 POINT(40.51380328066537 74.82755294833767) bank66429 +66430 POINT(39.71648952297642 74.92796542518226) bank66430 +66431 POINT(40.17665300407672 73.02170475751714) bank66431 +66432 POINT(41.4281848040407 74.46343904168951) bank66432 +66433 POINT(40.94314697339841 73.12081384615243) bank66433 +66434 POINT(40.26560226418423 74.93674002886931) bank66434 +66435 POINT(41.37715449286956 73.23620613418453) bank66435 +66436 POINT(40.3022173710604 73.69983875603481) bank66436 +66437 POINT(40.91897435932262 74.2620154175758) bank66437 +66438 POINT(40.66163117556687 74.10762656084853) bank66438 +66439 POINT(40.254071702209444 74.34824501930403) bank66439 +66440 POINT(40.77122972362814 73.75855628696247) bank66440 +66441 POINT(40.51434963146428 74.05989085300463) bank66441 +66442 POINT(41.36618293578152 74.07246725489009) bank66442 +66443 POINT(41.6178089693001 74.49654175610507) bank66443 +66444 POINT(40.49742087768799 73.86987531320193) bank66444 +66445 POINT(40.62926668394175 73.8810292620917) bank66445 +66446 POINT(39.8069298289081 74.32192345479476) bank66446 +66447 POINT(40.33826310364584 74.68434464408261) bank66447 +66448 POINT(40.71772231505596 74.71958047042872) bank66448 +66449 POINT(41.62011296290663 74.30285473695582) bank66449 +66450 POINT(40.54672381614582 73.44831946194523) bank66450 +66451 POINT(40.84346237525734 73.30446874233509) bank66451 +66452 POINT(40.65882652597823 73.37710987904377) bank66452 +66453 POINT(40.57963662134331 73.76459169407138) bank66453 +66454 POINT(40.47383680184109 74.6421972049711) bank66454 +66455 POINT(40.43927921913045 74.44144315553915) bank66455 +66456 POINT(41.22116366252374 74.00915301644734) bank66456 +66457 POINT(40.178093251693404 74.764926737737) bank66457 +66458 POINT(40.66478160171707 74.9225242433176) bank66458 +66459 POINT(41.70049601089369 74.85010872401207) bank66459 +66460 POINT(39.8373401868904 74.83072280604064) bank66460 +66461 POINT(40.853368048532964 74.08321112241217) bank66461 +66462 POINT(41.1146270947832 74.17138462627146) bank66462 +66463 POINT(40.74069206588354 74.47781954976783) bank66463 +66464 POINT(41.05758214015815 74.64818001613489) bank66464 +66465 POINT(40.23273838594372 74.84168505897848) bank66465 +66466 POINT(41.616410209270946 74.35892923681479) bank66466 +66467 POINT(39.97743201120311 74.01754518668783) bank66467 +66468 POINT(40.11990685969164 73.44581375012655) bank66468 +66469 POINT(41.478188279688354 74.83256416233519) bank66469 +66470 POINT(40.26502496377572 73.02866883086365) bank66470 +66471 POINT(40.18897272733943 74.77693222846194) bank66471 +66472 POINT(41.196474431658814 73.47980514757582) bank66472 +66473 POINT(40.41384365858074 73.44479913017575) bank66473 +66474 POINT(40.55495839047199 74.18061877763407) bank66474 +66475 POINT(40.58744347222963 74.09889214070724) bank66475 +66476 POINT(41.543198092945964 73.1111708022393) bank66476 +66477 POINT(41.04333537103686 73.17689793656974) bank66477 +66478 POINT(40.97641324249683 73.32874181238415) bank66478 +66479 POINT(39.90089475078891 74.30654149395302) bank66479 +66480 POINT(41.691064044666426 73.08550996329764) bank66480 +66481 POINT(40.6095660115674 73.51620160963462) bank66481 +66482 POINT(40.6258769304017 73.9653037195955) bank66482 +66483 POINT(41.397624473241834 74.84087495509509) bank66483 +66484 POINT(40.44813725678256 74.1877008405358) bank66484 +66485 POINT(41.66565392467502 73.01797332868713) bank66485 +66486 POINT(41.25532479438064 74.7101540818551) bank66486 +66487 POINT(41.063693262285426 74.06021793996423) bank66487 +66488 POINT(40.28689369230643 73.41740285555481) bank66488 +66489 POINT(41.007232256372205 74.3222922857794) bank66489 +66490 POINT(41.53367356572848 74.62271704629065) bank66490 +66491 POINT(40.84525455113995 74.44043407887119) bank66491 +66492 POINT(40.28578818681689 73.12427513011484) bank66492 +66493 POINT(41.203405873080385 74.43964894060085) bank66493 +66494 POINT(40.6743033438686 74.01446357187805) bank66494 +66495 POINT(41.60536458333867 73.63706216231905) bank66495 +66496 POINT(40.185325955341334 73.67648840015356) bank66496 +66497 POINT(41.16470677991166 74.15005066195431) bank66497 +66498 POINT(40.114120890707184 73.45263615546074) bank66498 +66499 POINT(41.4764828235713 74.94629369227125) bank66499 +66500 POINT(39.88105095843458 74.91760751925007) bank66500 +66501 POINT(39.90257176254198 73.54994332433118) bank66501 +66502 POINT(39.899491367101504 74.29974065863303) bank66502 +66503 POINT(40.068508270743116 74.60722959311487) bank66503 +66504 POINT(40.87504522327888 73.22931355425064) bank66504 +66505 POINT(40.18295591098135 74.9517563587404) bank66505 +66506 POINT(39.8919151691496 74.9852195058129) bank66506 +66507 POINT(41.38760491949239 74.4687639200398) bank66507 +66508 POINT(40.80874927562872 74.77760468383471) bank66508 +66509 POINT(40.95046818648889 73.13847338085687) bank66509 +66510 POINT(40.31432162758609 74.27841744413175) bank66510 +66511 POINT(40.284595509910424 74.82293124430271) bank66511 +66512 POINT(40.94879128757981 74.48207525028594) bank66512 +66513 POINT(40.58268314425748 74.54074696654642) bank66513 +66514 POINT(41.55265172945814 74.3805161513193) bank66514 +66515 POINT(41.560181326160794 73.6061721879225) bank66515 +66516 POINT(41.481717481686076 73.55093186419609) bank66516 +66517 POINT(40.338611720164366 74.83003398191761) bank66517 +66518 POINT(40.06705616751201 74.06732609433018) bank66518 +66519 POINT(41.0131597805908 73.19234788327151) bank66519 +66520 POINT(41.225180631683195 74.82542904480209) bank66520 +66521 POINT(41.06121586822124 74.17340534283085) bank66521 +66522 POINT(41.483417948249645 73.75450438120028) bank66522 +66523 POINT(40.701360910599 73.37986505688603) bank66523 +66524 POINT(40.5724484310229 74.30728011074282) bank66524 +66525 POINT(40.42873671537942 73.23565520141324) bank66525 +66526 POINT(41.07455304806592 73.04564242767796) bank66526 +66527 POINT(41.426972451250144 74.88427599482705) bank66527 +66528 POINT(40.572079160114335 73.25411095975385) bank66528 +66529 POINT(40.95393768743055 74.5371559979516) bank66529 +66530 POINT(40.21397844787533 73.70859307371855) bank66530 +66531 POINT(40.00352877167496 74.4719386582022) bank66531 +66532 POINT(40.627614225856625 74.69589231809987) bank66532 +66533 POINT(40.58902436545598 74.18515506515234) bank66533 +66534 POINT(41.57283309710208 74.76725125319676) bank66534 +66535 POINT(40.206400535396604 74.82802057975414) bank66535 +66536 POINT(40.9616952879049 74.70192162830115) bank66536 +66537 POINT(40.99025721201636 74.89784470492545) bank66537 +66538 POINT(40.95409411982358 74.5418490490861) bank66538 +66539 POINT(40.18382420440001 74.31744802551934) bank66539 +66540 POINT(41.307055162242065 73.47146061198906) bank66540 +66541 POINT(40.307546164870914 73.42003456877238) bank66541 +66542 POINT(40.22234015924008 74.1348133881369) bank66542 +66543 POINT(40.85159416160557 74.55375489773154) bank66543 +66544 POINT(41.44700693253316 73.55367448615078) bank66544 +66545 POINT(39.836519729756944 73.6256140445575) bank66545 +66546 POINT(40.11781534948834 73.34157229720857) bank66546 +66547 POINT(41.49360651291683 74.46398487082404) bank66547 +66548 POINT(41.59632687803554 74.42494498834317) bank66548 +66549 POINT(40.301500613355984 73.7717285217931) bank66549 +66550 POINT(41.41211877000948 73.06157726065649) bank66550 +66551 POINT(40.79080870223686 74.38015410396672) bank66551 +66552 POINT(40.700480553600414 73.76763130716164) bank66552 +66553 POINT(39.75311709024365 74.29642660510214) bank66553 +66554 POINT(39.8355544079925 73.53898518933609) bank66554 +66555 POINT(39.98316609195922 73.47221697494851) bank66555 +66556 POINT(40.688329506863866 74.82983718878468) bank66556 +66557 POINT(40.16610891765238 74.70601181243106) bank66557 +66558 POINT(39.82605953044005 74.95185242583487) bank66558 +66559 POINT(40.069144920247695 74.34167142807907) bank66559 +66560 POINT(40.25140285761445 73.64154969542733) bank66560 +66561 POINT(40.77203914573015 73.67655144555093) bank66561 +66562 POINT(41.2709084260868 73.27984117592403) bank66562 +66563 POINT(40.525100806195795 74.59823647686909) bank66563 +66564 POINT(40.525849695149 74.89168179847952) bank66564 +66565 POINT(41.46758998163685 73.56764722156015) bank66565 +66566 POINT(40.8947032250639 74.48396892961833) bank66566 +66567 POINT(39.75582013240607 75.0019898195135) bank66567 +66568 POINT(41.05516233367873 74.04945877733891) bank66568 +66569 POINT(40.75958185943634 73.19326118504998) bank66569 +66570 POINT(41.27360541174253 74.39562104550102) bank66570 +66571 POINT(40.9093222849172 74.50469677156994) bank66571 +66572 POINT(41.326959932450784 74.99031039648304) bank66572 +66573 POINT(40.47222987429438 74.37724453433542) bank66573 +66574 POINT(41.044523901162776 74.59611787311763) bank66574 +66575 POINT(39.901014575862504 74.1470377009106) bank66575 +66576 POINT(39.98717789861967 73.37120445694264) bank66576 +66577 POINT(41.46643305640112 73.67708509963128) bank66577 +66578 POINT(40.15628857286741 73.04339668140946) bank66578 +66579 POINT(40.121138009988044 74.46256495249317) bank66579 +66580 POINT(40.51451752857268 74.0132771650945) bank66580 +66581 POINT(39.83533844570668 73.95616685534833) bank66581 +66582 POINT(40.57947795423691 73.07843703460529) bank66582 +66583 POINT(41.11903150380893 74.3627053282892) bank66583 +66584 POINT(41.30069465828815 74.4874640385569) bank66584 +66585 POINT(40.61730980023976 73.31324577120523) bank66585 +66586 POINT(40.61372282067573 73.65463635508414) bank66586 +66587 POINT(41.25441872481889 73.74210642901954) bank66587 +66588 POINT(40.441206396664164 73.24246132316183) bank66588 +66589 POINT(40.87792743284061 74.84600295497549) bank66589 +66590 POINT(41.2278234820497 74.12719431118397) bank66590 +66591 POINT(40.545395308576396 73.80575461548906) bank66591 +66592 POINT(40.380066073978774 73.24996304430401) bank66592 +66593 POINT(41.254737447198394 74.95940879921264) bank66593 +66594 POINT(40.38191417664826 73.41038854885) bank66594 +66595 POINT(40.864772587273244 73.17117054035823) bank66595 +66596 POINT(40.88503767352286 74.66010152529415) bank66596 +66597 POINT(40.39778607849442 73.88538136796586) bank66597 +66598 POINT(40.53424278191831 74.11511504932517) bank66598 +66599 POINT(40.6899099747594 73.86852754282155) bank66599 +66600 POINT(39.906771297861816 74.03006384707211) bank66600 +66601 POINT(39.72174359823541 73.43706860594716) bank66601 +66602 POINT(40.60585670483316 73.22289640441838) bank66602 +66603 POINT(41.43267772903478 73.63582280431181) bank66603 +66604 POINT(40.864395882145715 73.45328782151913) bank66604 +66605 POINT(40.72984977113378 74.72714241964908) bank66605 +66606 POINT(40.18132616529775 73.78464781683498) bank66606 +66607 POINT(41.560448117670546 74.4209541581326) bank66607 +66608 POINT(40.900437637048775 74.3170414299735) bank66608 +66609 POINT(39.794241084526476 73.69473915156168) bank66609 +66610 POINT(41.56326672284151 73.18209499672949) bank66610 +66611 POINT(40.86901071069075 73.78689162090417) bank66611 +66612 POINT(41.07644245635032 74.09728980130201) bank66612 +66613 POINT(41.20826188815859 74.63740578661645) bank66613 +66614 POINT(40.132651607740186 74.22690251890664) bank66614 +66615 POINT(41.07600213350507 73.00939310236825) bank66615 +66616 POINT(39.91844181089981 73.99483473667024) bank66616 +66617 POINT(41.050960958245966 74.40005034256266) bank66617 +66618 POINT(40.50784575137645 73.84487499172322) bank66618 +66619 POINT(41.32774879946247 73.16993969865754) bank66619 +66620 POINT(39.85879088451591 73.81744897483452) bank66620 +66621 POINT(40.26183464479067 73.49683698408873) bank66621 +66622 POINT(40.747726656606446 74.11646535522864) bank66622 +66623 POINT(40.82633485614092 74.31201856073085) bank66623 +66624 POINT(41.59390661472769 74.21924328686688) bank66624 +66625 POINT(40.74742074920922 74.49067596117914) bank66625 +66626 POINT(40.75061988115111 74.8159674386425) bank66626 +66627 POINT(41.17100516652549 73.2251823733472) bank66627 +66628 POINT(40.19303074516807 74.94439881875786) bank66628 +66629 POINT(41.285423500335526 73.85399588998159) bank66629 +66630 POINT(39.846101855748735 74.16646293847849) bank66630 +66631 POINT(41.288231605166764 73.40810809223062) bank66631 +66632 POINT(39.747677319504405 73.761468292939) bank66632 +66633 POINT(39.723054133786505 73.38655388172192) bank66633 +66634 POINT(40.90124080293412 73.45344473269412) bank66634 +66635 POINT(41.09213442233753 73.35941390305815) bank66635 +66636 POINT(40.828451802070376 73.46835161229126) bank66636 +66637 POINT(41.16534045363118 74.83574390343199) bank66637 +66638 POINT(39.810881786179785 73.01542025892505) bank66638 +66639 POINT(41.244508450461744 74.42730013088612) bank66639 +66640 POINT(40.4187599199478 73.22239803224419) bank66640 +66641 POINT(40.32695616304252 73.17057850093789) bank66641 +66642 POINT(41.47317485070988 73.12761110569599) bank66642 +66643 POINT(40.18895124104574 74.72584219140224) bank66643 +66644 POINT(39.71622805119476 73.03204789016985) bank66644 +66645 POINT(40.3641778860856 73.3108541965268) bank66645 +66646 POINT(40.81894931318679 74.50809251886392) bank66646 +66647 POINT(40.72164399224077 74.67332325695075) bank66647 +66648 POINT(41.5657399089533 74.36327936179168) bank66648 +66649 POINT(41.54180089811581 74.78070332309768) bank66649 +66650 POINT(41.19902622492138 74.03381198541821) bank66650 +66651 POINT(40.34573188666831 74.77847399945786) bank66651 +66652 POINT(40.308548122080005 73.18304418272731) bank66652 +66653 POINT(39.762350977987005 74.7842736171669) bank66653 +66654 POINT(40.49331816290315 74.27739256675913) bank66654 +66655 POINT(39.84623110745607 73.69782072214713) bank66655 +66656 POINT(39.873774435528745 73.48124966714653) bank66656 +66657 POINT(40.84773904460877 74.54388060611345) bank66657 +66658 POINT(40.91061075116547 73.41611644573668) bank66658 +66659 POINT(41.68621730344692 74.30965011066466) bank66659 +66660 POINT(41.58039480793539 74.01708782882656) bank66660 +66661 POINT(41.16656857474468 73.71683317291583) bank66661 +66662 POINT(40.73358859905362 74.58980977031412) bank66662 +66663 POINT(40.15303689443788 73.45605762269372) bank66663 +66664 POINT(40.17934764610422 74.02664487015977) bank66664 +66665 POINT(40.7770600646847 73.18542090030992) bank66665 +66666 POINT(40.36178990985382 73.37259201186505) bank66666 +66667 POINT(39.81001095675183 74.01075793622663) bank66667 +66668 POINT(39.71826336300367 73.83343851560991) bank66668 +66669 POINT(39.838787902650985 74.16164373747738) bank66669 +66670 POINT(40.67285425299855 74.50270581682834) bank66670 +66671 POINT(40.54133726617363 73.63070546927031) bank66671 +66672 POINT(40.42942586868799 73.8277919143427) bank66672 +66673 POINT(39.91341745487922 74.97180731226905) bank66673 +66674 POINT(40.12988595295324 73.18546816009392) bank66674 +66675 POINT(40.6521290071122 73.1468783597327) bank66675 +66676 POINT(40.85051962545142 73.45456051805597) bank66676 +66677 POINT(40.88260445608358 74.33767126642198) bank66677 +66678 POINT(40.78382808926905 73.86791958149378) bank66678 +66679 POINT(41.151362651108165 74.32772065051763) bank66679 +66680 POINT(40.604541477665826 74.18949458092645) bank66680 +66681 POINT(40.05081614258578 74.54576902314137) bank66681 +66682 POINT(41.55804743171005 74.73377482097133) bank66682 +66683 POINT(40.13136085802209 74.64683615119914) bank66683 +66684 POINT(41.316379803481794 73.30371178864617) bank66684 +66685 POINT(39.99766656511056 73.70285276808933) bank66685 +66686 POINT(39.93813859508442 74.27760721480539) bank66686 +66687 POINT(41.686201358115085 74.48257484683309) bank66687 +66688 POINT(40.831631166914654 74.89373889954885) bank66688 +66689 POINT(40.56546926585648 73.55374107198539) bank66689 +66690 POINT(40.34013342374632 73.27158104873088) bank66690 +66691 POINT(40.76922278301231 73.3127008081241) bank66691 +66692 POINT(41.596148631406095 74.26499762306399) bank66692 +66693 POINT(40.86657816990649 73.85173931120389) bank66693 +66694 POINT(40.25765862163632 73.9349428586927) bank66694 +66695 POINT(39.985638810710626 73.01624304681533) bank66695 +66696 POINT(39.81502285864805 74.09538702036701) bank66696 +66697 POINT(40.59532024365156 73.6431484516194) bank66697 +66698 POINT(41.150346349890675 74.74036463595394) bank66698 +66699 POINT(40.33137298191455 74.92958666616877) bank66699 +66700 POINT(40.927984976386746 73.90932282497361) bank66700 +66701 POINT(41.390273487705 74.88731531866775) bank66701 +66702 POINT(40.06376333588025 74.5120178731188) bank66702 +66703 POINT(40.07831658083083 74.62835000661767) bank66703 +66704 POINT(41.545368364615314 74.27792460473074) bank66704 +66705 POINT(40.43611774044819 74.53499387198129) bank66705 +66706 POINT(40.29912262890046 74.85787937555557) bank66706 +66707 POINT(39.807486967953515 73.12330483962643) bank66707 +66708 POINT(40.07677164261761 73.13459050680919) bank66708 +66709 POINT(41.11402936000502 73.15658853052481) bank66709 +66710 POINT(41.03549962945434 74.64467165631879) bank66710 +66711 POINT(40.001109865949445 73.98917235669488) bank66711 +66712 POINT(41.01937552162019 73.68107195446619) bank66712 +66713 POINT(41.53574758977285 74.82537084935701) bank66713 +66714 POINT(39.81570011295912 74.74704393742272) bank66714 +66715 POINT(41.413254946412735 74.41304103636519) bank66715 +66716 POINT(41.23059626021866 73.64214766031995) bank66716 +66717 POINT(40.509899660790126 73.02331569883994) bank66717 +66718 POINT(39.95660413559953 74.62572235329164) bank66718 +66719 POINT(41.00560118688557 74.04281258601137) bank66719 +66720 POINT(40.84458502024535 73.94000429861933) bank66720 +66721 POINT(40.98062320770051 73.24112647673482) bank66721 +66722 POINT(40.41186628405841 74.19126378762225) bank66722 +66723 POINT(40.29903223687988 73.7513356116947) bank66723 +66724 POINT(40.68070021348771 73.75687635990595) bank66724 +66725 POINT(40.15779992070363 73.01640769614814) bank66725 +66726 POINT(40.86840170815015 73.3036988770883) bank66726 +66727 POINT(40.688170949116326 73.7680105853079) bank66727 +66728 POINT(41.43480696378264 74.82970655305415) bank66728 +66729 POINT(40.055217913684494 74.85551122649309) bank66729 +66730 POINT(40.159420667540765 73.53659444458052) bank66730 +66731 POINT(41.651532609521524 73.02412712406253) bank66731 +66732 POINT(40.46636489890804 74.92933361963406) bank66732 +66733 POINT(40.74892022362008 74.97018237793127) bank66733 +66734 POINT(40.87126558261844 73.50602782668064) bank66734 +66735 POINT(40.64381359864299 73.80645900572037) bank66735 +66736 POINT(40.17196921251561 73.7895742945543) bank66736 +66737 POINT(41.66491671588981 74.97207945665792) bank66737 +66738 POINT(39.924878379262815 73.2835317872476) bank66738 +66739 POINT(40.77981750398607 74.56777396550883) bank66739 +66740 POINT(40.16049993127345 73.78553615233861) bank66740 +66741 POINT(41.3305944988844 74.29131589770432) bank66741 +66742 POINT(40.72211911446816 74.29341484892767) bank66742 +66743 POINT(41.16638256610214 74.50144682856717) bank66743 +66744 POINT(40.58715009575469 73.36773581574059) bank66744 +66745 POINT(39.74791253121323 74.76043804709458) bank66745 +66746 POINT(40.55350408581852 73.52125066586967) bank66746 +66747 POINT(41.40161285359876 73.30796125057921) bank66747 +66748 POINT(41.28453524307595 74.05033337162658) bank66748 +66749 POINT(39.79102821264213 73.82526119345816) bank66749 +66750 POINT(39.91216498423676 73.42182452389747) bank66750 +66751 POINT(40.82521547960372 73.6719417151666) bank66751 +66752 POINT(40.97496085393687 74.33732285129895) bank66752 +66753 POINT(39.84213076150772 74.05411914811724) bank66753 +66754 POINT(40.929572631781774 73.41803100830946) bank66754 +66755 POINT(40.25420144660118 74.82794302693982) bank66755 +66756 POINT(41.12554491515548 73.84128748716763) bank66756 +66757 POINT(40.67729539494517 73.44222639460557) bank66757 +66758 POINT(41.16550283141381 74.42222093397211) bank66758 +66759 POINT(39.803463150227515 73.85436454008547) bank66759 +66760 POINT(41.395468812398235 74.87834920281857) bank66760 +66761 POINT(41.562641813610185 74.61710632193999) bank66761 +66762 POINT(41.63831966671745 73.78076267828898) bank66762 +66763 POINT(41.21572664163502 74.79950849438157) bank66763 +66764 POINT(41.7004327230703 73.62250917959203) bank66764 +66765 POINT(41.04002479299965 74.28146634134016) bank66765 +66766 POINT(40.88500089557524 74.87140248443755) bank66766 +66767 POINT(41.06043415223817 73.8002298971846) bank66767 +66768 POINT(41.002730202441825 74.29919107125079) bank66768 +66769 POINT(41.406075895613036 74.25013674706646) bank66769 +66770 POINT(40.162265037208265 73.44662553507568) bank66770 +66771 POINT(39.880092845880576 74.51871562497037) bank66771 +66772 POINT(40.38507380084031 74.7747062290499) bank66772 +66773 POINT(40.9250969134955 74.47460063759836) bank66773 +66774 POINT(40.40383920427729 73.32204086262153) bank66774 +66775 POINT(40.14903120440702 74.15709323868627) bank66775 +66776 POINT(40.50302582179494 73.757385740179) bank66776 +66777 POINT(40.17405891238337 74.48842774641659) bank66777 +66778 POINT(39.945572249379026 74.47201057649877) bank66778 +66779 POINT(40.93457077502475 74.67510233247158) bank66779 +66780 POINT(41.14686328400559 74.61877776001607) bank66780 +66781 POINT(40.24526929882975 73.88471815014115) bank66781 +66782 POINT(41.55115216746762 74.88388872417596) bank66782 +66783 POINT(40.68749748323965 73.83208833186718) bank66783 +66784 POINT(41.65519578714165 73.79775203138612) bank66784 +66785 POINT(41.115733823481875 73.10280202284882) bank66785 +66786 POINT(41.3064245677152 73.63903655882446) bank66786 +66787 POINT(41.68051788740514 74.129213322595) bank66787 +66788 POINT(40.98000144707567 73.51143511290032) bank66788 +66789 POINT(40.5463993864051 73.14365532016309) bank66789 +66790 POINT(41.5106866603766 74.79396367706624) bank66790 +66791 POINT(41.69543962402957 74.943401478974) bank66791 +66792 POINT(41.40708491973726 74.20584533759056) bank66792 +66793 POINT(41.508435223647595 73.87134685574706) bank66793 +66794 POINT(41.327599970829404 74.07249685378945) bank66794 +66795 POINT(39.718050114310365 73.90970550207214) bank66795 +66796 POINT(41.64012355033385 74.87474639290055) bank66796 +66797 POINT(41.42342903525418 73.20932037831552) bank66797 +66798 POINT(39.927972797222374 74.73923027510962) bank66798 +66799 POINT(41.20498648743884 74.49010634570334) bank66799 +66800 POINT(40.301881775552424 74.82667116849318) bank66800 +66801 POINT(39.99705902599193 74.24641230737112) bank66801 +66802 POINT(40.63860992431808 73.97498931458507) bank66802 +66803 POINT(39.951639191588114 73.678021405119) bank66803 +66804 POINT(39.786069239401165 73.36019227389876) bank66804 +66805 POINT(41.42799910251911 74.27752142649442) bank66805 +66806 POINT(39.72557800745772 74.38943488888732) bank66806 +66807 POINT(41.66612171589187 73.64107156170792) bank66807 +66808 POINT(41.40086569743164 74.55090960422945) bank66808 +66809 POINT(40.6328883708024 74.2979034962945) bank66809 +66810 POINT(40.01802851766844 74.82523046880604) bank66810 +66811 POINT(41.0063796897203 74.35896904404453) bank66811 +66812 POINT(40.07356994622771 73.11472391291099) bank66812 +66813 POINT(40.9764639116495 73.23987834936342) bank66813 +66814 POINT(41.14332508580461 74.95209311164032) bank66814 +66815 POINT(40.66291470831552 73.48761577734821) bank66815 +66816 POINT(40.040785957420425 73.56913733376622) bank66816 +66817 POINT(41.33352131217597 74.72845938078855) bank66817 +66818 POINT(40.37002057881772 74.21086830397627) bank66818 +66819 POINT(40.51376011286359 74.02352332929902) bank66819 +66820 POINT(41.47658757976952 73.47236362660693) bank66820 +66821 POINT(40.63052925077241 73.11430553456296) bank66821 +66822 POINT(41.34387921769367 73.11056794011414) bank66822 +66823 POINT(40.229087061451835 73.99160099416797) bank66823 +66824 POINT(40.15784708959632 74.75772225560576) bank66824 +66825 POINT(39.8758993509462 73.41128622680611) bank66825 +66826 POINT(40.12175243320066 74.54572245241272) bank66826 +66827 POINT(39.97735857671191 74.58568961949554) bank66827 +66828 POINT(41.22777699887243 73.61582565722345) bank66828 +66829 POINT(40.59796082886029 73.68567417705115) bank66829 +66830 POINT(40.050428282883786 74.95048846054506) bank66830 +66831 POINT(41.33303446045972 74.65365899722995) bank66831 +66832 POINT(41.489826809935494 74.0628073522875) bank66832 +66833 POINT(40.84615909621343 74.78159463328906) bank66833 +66834 POINT(41.365393487776274 73.8557996128708) bank66834 +66835 POINT(40.11667410021067 74.10464372722498) bank66835 +66836 POINT(41.193220795478 74.34506334210707) bank66836 +66837 POINT(40.582257983777524 73.95863922278166) bank66837 +66838 POINT(41.22421583168621 73.23246523058322) bank66838 +66839 POINT(40.467096307107646 73.92869348556363) bank66839 +66840 POINT(41.18561715656893 73.16655321848398) bank66840 +66841 POINT(39.73338086489748 73.75778298676215) bank66841 +66842 POINT(40.34516042682179 73.48175779695302) bank66842 +66843 POINT(40.98399178351869 74.49477855597226) bank66843 +66844 POINT(40.760795364242696 74.07711798876936) bank66844 +66845 POINT(40.57827371477422 73.06602709958365) bank66845 +66846 POINT(41.342568255259835 74.73598393442576) bank66846 +66847 POINT(40.648736769083875 74.67324917017436) bank66847 +66848 POINT(40.515817292123074 74.17169232424402) bank66848 +66849 POINT(41.70549851467574 74.09331283846497) bank66849 +66850 POINT(40.75190789815664 73.16767088562213) bank66850 +66851 POINT(41.59182216244881 74.18334332665081) bank66851 +66852 POINT(40.74557884201561 73.26997866294502) bank66852 +66853 POINT(41.24383313736598 74.9335037301354) bank66853 +66854 POINT(40.31814405061217 74.62662683813024) bank66854 +66855 POINT(40.850907770340186 74.5543409964526) bank66855 +66856 POINT(40.9116164712581 73.29300515325964) bank66856 +66857 POINT(41.27407564613316 73.14214888752399) bank66857 +66858 POINT(41.140546379870266 74.90968327321299) bank66858 +66859 POINT(41.27819853875736 73.31221748012916) bank66859 +66860 POINT(41.35162116675904 73.89657296237877) bank66860 +66861 POINT(41.31846236261392 73.05266866810113) bank66861 +66862 POINT(41.69711948317754 74.35884745209609) bank66862 +66863 POINT(40.77098078399781 73.5402591174133) bank66863 +66864 POINT(39.93325798850303 73.75680323368945) bank66864 +66865 POINT(40.87788088453982 74.74523257982061) bank66865 +66866 POINT(40.926953667049 73.1031582700419) bank66866 +66867 POINT(40.335171286232914 73.3132756591991) bank66867 +66868 POINT(40.34793901918291 74.41995060411799) bank66868 +66869 POINT(41.342127274319886 73.45150269182719) bank66869 +66870 POINT(39.88699711544542 74.75858742840455) bank66870 +66871 POINT(40.57470902286066 74.82910584393774) bank66871 +66872 POINT(39.84087345079084 74.2863416464825) bank66872 +66873 POINT(40.058293295672755 73.20010046614384) bank66873 +66874 POINT(40.074680952423684 73.69506871251285) bank66874 +66875 POINT(41.12027661262606 73.76092130569359) bank66875 +66876 POINT(39.74769306355011 73.08495368676196) bank66876 +66877 POINT(41.30394616722572 73.33062140749362) bank66877 +66878 POINT(40.78798427957933 73.85281264563389) bank66878 +66879 POINT(41.249722388664495 73.87517064824998) bank66879 +66880 POINT(41.17714214103619 73.01208347543222) bank66880 +66881 POINT(40.8732358967793 74.47792788534234) bank66881 +66882 POINT(41.314760440382514 73.61803092777131) bank66882 +66883 POINT(41.10033538537331 73.30357687579021) bank66883 +66884 POINT(41.27049156987763 73.78583086550391) bank66884 +66885 POINT(40.24867928543969 74.8549536937284) bank66885 +66886 POINT(41.501161182827424 74.51239547068658) bank66886 +66887 POINT(40.002231740507895 74.53098376471623) bank66887 +66888 POINT(40.911451328394726 74.7971393420559) bank66888 +66889 POINT(41.2795378333327 74.66309495806127) bank66889 +66890 POINT(40.064361938406904 74.0036380460687) bank66890 +66891 POINT(40.420364644757456 73.26307496717585) bank66891 +66892 POINT(41.029263737979235 73.26664530093699) bank66892 +66893 POINT(41.14599545901207 73.61763677552504) bank66893 +66894 POINT(41.10806988067349 74.69108920119133) bank66894 +66895 POINT(40.95356140449826 74.49960979028089) bank66895 +66896 POINT(41.4758946598609 74.00733961514656) bank66896 +66897 POINT(40.6611230224151 73.80323554654282) bank66897 +66898 POINT(40.320452584175406 74.94577015852111) bank66898 +66899 POINT(40.57618498868989 74.61100837069625) bank66899 +66900 POINT(40.53555680965932 73.38590967768567) bank66900 +66901 POINT(39.84391998199352 73.59800475239658) bank66901 +66902 POINT(40.27546195046567 74.6833095343695) bank66902 +66903 POINT(40.01960650679602 73.56798741178392) bank66903 +66904 POINT(39.91244069741149 73.0114484558394) bank66904 +66905 POINT(41.39948718369073 73.33570721254897) bank66905 +66906 POINT(40.32350738231992 74.82867213147938) bank66906 +66907 POINT(39.806929008292315 73.98905730799478) bank66907 +66908 POINT(40.74006978904066 73.3397161881837) bank66908 +66909 POINT(41.64049704602914 74.19614911743179) bank66909 +66910 POINT(40.310211919098094 73.146471580849) bank66910 +66911 POINT(39.75008285269463 73.15807100207027) bank66911 +66912 POINT(40.696020293144315 73.46281887395885) bank66912 +66913 POINT(40.803769376174834 73.1324415879632) bank66913 +66914 POINT(40.10589362305866 74.15372393841885) bank66914 +66915 POINT(39.84808186236258 73.43379336012316) bank66915 +66916 POINT(40.58345574317163 74.80205892144696) bank66916 +66917 POINT(40.28474597336275 74.79524392085123) bank66917 +66918 POINT(40.56156434100912 74.65564365480208) bank66918 +66919 POINT(39.947436530071194 73.7490062129122) bank66919 +66920 POINT(40.31798862800308 74.1975389901273) bank66920 +66921 POINT(41.66689554560817 74.3380637097111) bank66921 +66922 POINT(40.04797441939299 73.1538131778953) bank66922 +66923 POINT(41.53768360516165 74.87377419997667) bank66923 +66924 POINT(40.14395132454184 74.05611239536546) bank66924 +66925 POINT(40.34024272074921 74.98327758050978) bank66925 +66926 POINT(40.34833642901092 74.95787772378506) bank66926 +66927 POINT(41.47523076495092 74.99378977105107) bank66927 +66928 POINT(40.50544528426332 73.29958628377979) bank66928 +66929 POINT(40.631145308346284 74.66176995674105) bank66929 +66930 POINT(40.187749382087226 74.81398041932243) bank66930 +66931 POINT(40.74535317836176 74.19659206151438) bank66931 +66932 POINT(41.20624328758459 74.66057493794243) bank66932 +66933 POINT(40.62328172485872 73.01889964470858) bank66933 +66934 POINT(40.23937743809482 73.8490597353919) bank66934 +66935 POINT(40.0422682621481 73.33130507611018) bank66935 +66936 POINT(39.75365921625042 73.48784847990522) bank66936 +66937 POINT(41.37030804524681 74.146891845) bank66937 +66938 POINT(40.25003371570013 74.50764885426474) bank66938 +66939 POINT(40.80552789554964 73.83568457381983) bank66939 +66940 POINT(41.4246264095887 74.37223548877272) bank66940 +66941 POINT(40.75444127523582 74.07323061242022) bank66941 +66942 POINT(40.19253243265429 73.45802152695718) bank66942 +66943 POINT(40.56815714493392 74.26021761147008) bank66943 +66944 POINT(40.984426021681685 74.80751493752032) bank66944 +66945 POINT(40.8085669726948 74.86171085243643) bank66945 +66946 POINT(40.12280133447827 73.59473761439423) bank66946 +66947 POINT(40.18852776305307 74.06604607266786) bank66947 +66948 POINT(41.04854732265576 74.66912339607794) bank66948 +66949 POINT(39.99022512208377 74.43671195451785) bank66949 +66950 POINT(40.37286410693322 74.32253098279608) bank66950 +66951 POINT(40.22417079020147 74.6641098568943) bank66951 +66952 POINT(40.07694956836565 73.56572989785887) bank66952 +66953 POINT(40.94067669868535 74.93979966888047) bank66953 +66954 POINT(40.550361338423755 73.36157565357897) bank66954 +66955 POINT(40.54483349883691 74.69123709069206) bank66955 +66956 POINT(41.63825143926438 73.38292497685171) bank66956 +66957 POINT(41.04682470613745 73.8799565673842) bank66957 +66958 POINT(41.63317337335974 73.08335097880638) bank66958 +66959 POINT(41.07876672049179 73.31927247644181) bank66959 +66960 POINT(41.442348601818686 73.62546716442644) bank66960 +66961 POINT(40.706326124096485 74.57562607655268) bank66961 +66962 POINT(41.342222343501525 74.13311481304873) bank66962 +66963 POINT(40.70670320008105 73.77013177305714) bank66963 +66964 POINT(40.76523139942194 74.14290794340378) bank66964 +66965 POINT(40.34794092883433 73.50912686175688) bank66965 +66966 POINT(40.00355485326803 73.88854589060729) bank66966 +66967 POINT(40.88549194126799 74.9243924361499) bank66967 +66968 POINT(40.32522579039571 73.25710147158115) bank66968 +66969 POINT(39.9599227200472 74.6827714728199) bank66969 +66970 POINT(40.51657782974827 74.13535102832144) bank66970 +66971 POINT(40.1584182624802 73.73006342304515) bank66971 +66972 POINT(41.63023238287553 74.53795754781919) bank66972 +66973 POINT(40.17405087739233 74.53243400164074) bank66973 +66974 POINT(40.068317625037665 73.37700575935439) bank66974 +66975 POINT(40.34180342469874 73.85996535331883) bank66975 +66976 POINT(39.88644360055597 73.35569064792277) bank66976 +66977 POINT(41.4765766721686 73.03748360173135) bank66977 +66978 POINT(40.24342806052206 73.74216440102441) bank66978 +66979 POINT(40.76172313691748 73.60998279504675) bank66979 +66980 POINT(39.84487289764693 74.50690009006058) bank66980 +66981 POINT(41.61885110162855 74.60309378616655) bank66981 +66982 POINT(40.600630413002314 74.65141503240918) bank66982 +66983 POINT(40.27120696713501 73.37764401266136) bank66983 +66984 POINT(39.94205349019216 73.53345744923058) bank66984 +66985 POINT(41.1246633111647 73.93141479754293) bank66985 +66986 POINT(40.306485634035305 73.93077811596991) bank66986 +66987 POINT(41.495504106570166 74.3621574390778) bank66987 +66988 POINT(40.22808720485012 74.77173640779259) bank66988 +66989 POINT(40.148248642775975 73.86058250558244) bank66989 +66990 POINT(40.64638317819168 74.89294823516687) bank66990 +66991 POINT(40.43498726738756 73.41685679567136) bank66991 +66992 POINT(40.57352450442362 73.19723764348677) bank66992 +66993 POINT(40.42574084603813 73.77488907372832) bank66993 +66994 POINT(40.04344701031949 73.40994913570837) bank66994 +66995 POINT(40.749619702178364 73.72616721831298) bank66995 +66996 POINT(39.854341305739844 74.12694749430706) bank66996 +66997 POINT(40.96461680324338 74.64000322935503) bank66997 +66998 POINT(41.470714765223235 73.6941294534213) bank66998 +66999 POINT(40.577590331560195 73.8269001669269) bank66999 +67000 POINT(41.55260857372554 73.64224642297934) bank67000 +67001 POINT(39.99066263078437 74.22303298004934) bank67001 +67002 POINT(40.89910808842094 73.13450982855436) bank67002 +67003 POINT(41.35177097853555 73.68858906803153) bank67003 +67004 POINT(41.20021336230981 74.59578423820054) bank67004 +67005 POINT(40.50759275488733 74.5054431809974) bank67005 +67006 POINT(39.91297221721073 74.07213733984912) bank67006 +67007 POINT(41.24774296434012 73.78588261670521) bank67007 +67008 POINT(40.22417190426549 73.02252367359382) bank67008 +67009 POINT(40.63717497873152 74.11277072593644) bank67009 +67010 POINT(40.93810745206407 73.4098672428377) bank67010 +67011 POINT(40.20821014799667 73.35478870037754) bank67011 +67012 POINT(40.141850351113426 73.40399286839111) bank67012 +67013 POINT(39.875692870950125 73.07339054133885) bank67013 +67014 POINT(39.97555619175594 73.33424845632413) bank67014 +67015 POINT(41.128379320633506 74.49324643537426) bank67015 +67016 POINT(40.656845522243366 73.62082549771232) bank67016 +67017 POINT(41.02788701877986 74.03912973237202) bank67017 +67018 POINT(40.54652114079835 74.92410495515095) bank67018 +67019 POINT(40.71595950873994 74.42613257395253) bank67019 +67020 POINT(41.01649305618047 73.29245242209421) bank67020 +67021 POINT(40.02480676063303 73.57623320872924) bank67021 +67022 POINT(41.26847655973365 74.18102069029635) bank67022 +67023 POINT(41.31745159299812 74.68048126361613) bank67023 +67024 POINT(41.16333543765445 73.23880188443786) bank67024 +67025 POINT(41.233527717559205 73.36428757808454) bank67025 +67026 POINT(39.7985297931975 74.01364648868604) bank67026 +67027 POINT(41.47907206443585 74.00881059488069) bank67027 +67028 POINT(40.75226710424113 74.8126426594635) bank67028 +67029 POINT(40.1319728360047 73.85921210104252) bank67029 +67030 POINT(39.76074220433697 73.58916665338761) bank67030 +67031 POINT(40.27055014960097 74.54976022844961) bank67031 +67032 POINT(40.46949088575743 73.84863190247202) bank67032 +67033 POINT(40.58780517531279 74.9647227667748) bank67033 +67034 POINT(39.86666550522802 74.16363016856172) bank67034 +67035 POINT(40.13563426780819 73.3902454462838) bank67035 +67036 POINT(40.13810495229688 73.59827370120617) bank67036 +67037 POINT(41.17235287425884 74.37971658213175) bank67037 +67038 POINT(41.633345291147194 73.24025989171919) bank67038 +67039 POINT(41.22142927988904 73.51062190640222) bank67039 +67040 POINT(40.97416695771698 73.50477583590292) bank67040 +67041 POINT(41.216164580583786 74.9785107681525) bank67041 +67042 POINT(41.01317495260435 73.8267028041067) bank67042 +67043 POINT(39.83124171834493 73.9368049767435) bank67043 +67044 POINT(40.582115574733116 74.64885393011146) bank67044 +67045 POINT(41.26862765998698 73.83825690760608) bank67045 +67046 POINT(41.02177976657318 74.12052486224148) bank67046 +67047 POINT(40.943490214421416 73.16784016514164) bank67047 +67048 POINT(40.39176686469911 73.45224299792801) bank67048 +67049 POINT(39.88790294132635 73.84033902873287) bank67049 +67050 POINT(40.742153772826846 73.52558003645217) bank67050 +67051 POINT(41.69998244912105 74.65824368575568) bank67051 +67052 POINT(39.779893662854825 73.6713542552977) bank67052 +67053 POINT(41.31639125463297 74.81407164950183) bank67053 +67054 POINT(41.00999159412282 74.60231935855704) bank67054 +67055 POINT(40.30970920125931 74.43083797953348) bank67055 +67056 POINT(41.221129741461866 74.07492466979986) bank67056 +67057 POINT(41.257559002639134 74.93438225180198) bank67057 +67058 POINT(40.21205236104149 73.66777772401517) bank67058 +67059 POINT(41.084656919198544 74.79272482756231) bank67059 +67060 POINT(39.873177494122174 74.91881937562033) bank67060 +67061 POINT(39.910538875219956 74.29551924817277) bank67061 +67062 POINT(41.39033087675888 74.3853246975761) bank67062 +67063 POINT(39.74310628952022 74.00271788563077) bank67063 +67064 POINT(40.84393930672869 73.13025199573264) bank67064 +67065 POINT(40.3766931324721 74.00831272295521) bank67065 +67066 POINT(39.961838210249795 74.56980292054998) bank67066 +67067 POINT(40.61270466774597 74.43345802178759) bank67067 +67068 POINT(40.020613867936454 74.96751157767812) bank67068 +67069 POINT(40.26366943763986 74.58896183685367) bank67069 +67070 POINT(40.79965452813206 73.30254048734542) bank67070 +67071 POINT(39.82420797908917 73.87680701917499) bank67071 +67072 POINT(41.43722688670704 74.6393398967705) bank67072 +67073 POINT(39.720657486880484 74.91660867885632) bank67073 +67074 POINT(40.50502702191641 73.88448479177751) bank67074 +67075 POINT(40.25420611085795 73.10620937345558) bank67075 +67076 POINT(41.58753295159754 74.12520101742322) bank67076 +67077 POINT(40.15266615357265 74.1560689057253) bank67077 +67078 POINT(39.96629109844785 73.60958644627927) bank67078 +67079 POINT(40.31352762285774 74.82974873567936) bank67079 +67080 POINT(40.12504774906519 73.88152408303658) bank67080 +67081 POINT(40.79070375059732 74.12466176337509) bank67081 +67082 POINT(41.58389849478148 74.40421375994288) bank67082 +67083 POINT(39.73437988657368 73.96270089356234) bank67083 +67084 POINT(41.64298992694046 73.49571324642024) bank67084 +67085 POINT(40.30427191823826 74.8799858653454) bank67085 +67086 POINT(41.28676564986975 74.05712165400087) bank67086 +67087 POINT(41.65492117739186 74.93290214378094) bank67087 +67088 POINT(40.114842284651004 73.80639535435097) bank67088 +67089 POINT(40.18034317103113 73.16029678242639) bank67089 +67090 POINT(41.698749748361784 73.88232476771753) bank67090 +67091 POINT(40.420499230579736 73.88864626196477) bank67091 +67092 POINT(40.36155050325055 74.08387075497255) bank67092 +67093 POINT(40.57802573416664 74.1078793888883) bank67093 +67094 POINT(40.68224819694353 73.01805373629983) bank67094 +67095 POINT(41.548325856639664 74.57171497943303) bank67095 +67096 POINT(41.475600062615726 73.35345780594693) bank67096 +67097 POINT(41.10170788500958 74.63088474522236) bank67097 +67098 POINT(40.38007831195164 73.56767908741743) bank67098 +67099 POINT(41.30171437186509 73.2773185304486) bank67099 +67100 POINT(41.50534488886497 73.01345426258491) bank67100 +67101 POINT(41.16360345147987 73.79904437371015) bank67101 +67102 POINT(41.70028089827859 74.42836204024651) bank67102 +67103 POINT(39.720888106499224 74.4906286364783) bank67103 +67104 POINT(41.0428455854926 74.91789801695901) bank67104 +67105 POINT(39.765933750129584 73.02052852332423) bank67105 +67106 POINT(39.91980339197782 73.25123734482177) bank67106 +67107 POINT(39.93874049465348 74.09102856252557) bank67107 +67108 POINT(40.95384721713221 73.59116125990883) bank67108 +67109 POINT(40.97148531349507 74.97993210027225) bank67109 +67110 POINT(41.49336793300252 73.0200704869888) bank67110 +67111 POINT(41.07331387834734 74.65295077006742) bank67111 +67112 POINT(39.92210754885307 73.40118308632562) bank67112 +67113 POINT(40.531057508986855 73.05533166949061) bank67113 +67114 POINT(41.28866280284865 74.16791698187514) bank67114 +67115 POINT(39.91307890705307 74.99920591282296) bank67115 +67116 POINT(40.67824703873478 73.45602065427525) bank67116 +67117 POINT(41.65112958419506 74.78719547363295) bank67117 +67118 POINT(41.55783426270371 74.3259677328394) bank67118 +67119 POINT(41.68961732294163 74.18793037498568) bank67119 +67120 POINT(40.72297258539398 73.81479654732058) bank67120 +67121 POINT(40.10074550156248 74.98720665964446) bank67121 +67122 POINT(41.01489911913081 74.38692262152347) bank67122 +67123 POINT(40.99513722521803 73.73592187263552) bank67123 +67124 POINT(40.14496109848079 74.24911071763015) bank67124 +67125 POINT(41.1004074498027 73.74948531666944) bank67125 +67126 POINT(40.452790985788674 74.47134269302627) bank67126 +67127 POINT(41.38187566174421 73.6912703685779) bank67127 +67128 POINT(39.87771486509839 74.45078805844759) bank67128 +67129 POINT(40.849645494988046 73.9087966150783) bank67129 +67130 POINT(40.21898307354483 73.60203404841447) bank67130 +67131 POINT(40.10308626102819 74.10799976524825) bank67131 +67132 POINT(40.28925244670197 74.38247375037318) bank67132 +67133 POINT(41.212428962164715 73.13834832451066) bank67133 +67134 POINT(41.599804836422315 73.92713252611237) bank67134 +67135 POINT(40.32356932191456 73.6690262879976) bank67135 +67136 POINT(41.02003907492125 73.35102745605995) bank67136 +67137 POINT(39.96163454506746 74.81290102350208) bank67137 +67138 POINT(39.84048574673388 73.16509867147022) bank67138 +67139 POINT(39.759805521403585 73.07494973232006) bank67139 +67140 POINT(40.675114321734014 73.14777562461492) bank67140 +67141 POINT(39.984095923753145 74.83437937794409) bank67141 +67142 POINT(41.70243660520246 74.76403933110625) bank67142 +67143 POINT(40.936932775183 74.52607836605917) bank67143 +67144 POINT(41.059565335887164 74.75604127255046) bank67144 +67145 POINT(40.68819497869 73.49233238381548) bank67145 +67146 POINT(41.46733752702999 73.53799688239538) bank67146 +67147 POINT(40.721809052017434 74.2641081369798) bank67147 +67148 POINT(40.58101131061534 73.81010621921695) bank67148 +67149 POINT(40.00823821183724 74.04380654738192) bank67149 +67150 POINT(41.641046175386705 73.7015836428626) bank67150 +67151 POINT(40.463301780116424 73.74033907863782) bank67151 +67152 POINT(40.76096649134636 73.61965399198633) bank67152 +67153 POINT(39.89090008395826 74.66014265672591) bank67153 +67154 POINT(41.042164846042944 74.41870455832185) bank67154 +67155 POINT(41.22119004080805 74.92172788076613) bank67155 +67156 POINT(41.22835365067121 74.48336697296988) bank67156 +67157 POINT(41.56135857393685 73.78708106170743) bank67157 +67158 POINT(41.05623787907142 74.23072214002882) bank67158 +67159 POINT(40.22543975816867 74.87640624433142) bank67159 +67160 POINT(40.20907408379426 73.99777695778269) bank67160 +67161 POINT(40.597082868199514 74.9211849748936) bank67161 +67162 POINT(40.10853081110946 73.87462632070344) bank67162 +67163 POINT(40.48305710600997 73.9471521983222) bank67163 +67164 POINT(40.84746290655003 73.77002181701252) bank67164 +67165 POINT(41.34217680035996 74.68675112990684) bank67165 +67166 POINT(41.32891549365931 74.08551596228881) bank67166 +67167 POINT(40.57673804092619 73.39281141347615) bank67167 +67168 POINT(41.596538120425734 73.99921707178025) bank67168 +67169 POINT(41.268589574995204 74.2901912731231) bank67169 +67170 POINT(40.87776912712111 74.24342223941953) bank67170 +67171 POINT(41.25401840078406 74.97872716934955) bank67171 +67172 POINT(39.76954673870892 74.28632202572221) bank67172 +67173 POINT(39.75847299972043 74.29184547251144) bank67173 +67174 POINT(39.93340809228408 73.08062384093708) bank67174 +67175 POINT(39.981411930961514 73.59516751866067) bank67175 +67176 POINT(41.330040002787754 73.46480164422233) bank67176 +67177 POINT(40.18710238975178 74.20488885892566) bank67177 +67178 POINT(39.79023328994213 73.36379712116859) bank67178 +67179 POINT(41.204138412373304 73.45053220180833) bank67179 +67180 POINT(40.7078030379478 74.80156612774093) bank67180 +67181 POINT(40.609447674759146 74.9453338904915) bank67181 +67182 POINT(40.59150151312097 73.94947380814338) bank67182 +67183 POINT(41.383289631948536 73.5955940012112) bank67183 +67184 POINT(40.91078707405898 73.62804938615605) bank67184 +67185 POINT(40.17038429347574 73.83639506593602) bank67185 +67186 POINT(41.12345385592481 74.49820365708) bank67186 +67187 POINT(41.083302221974904 73.96686685685556) bank67187 +67188 POINT(41.201517064781 73.177255814788) bank67188 +67189 POINT(41.63245672527122 73.2561825382148) bank67189 +67190 POINT(40.76267780823044 74.17778594884723) bank67190 +67191 POINT(41.62694812277732 74.96295933331741) bank67191 +67192 POINT(40.81939680181255 73.34324725795628) bank67192 +67193 POINT(40.25925536797421 74.6534729631348) bank67193 +67194 POINT(40.890252863352515 74.95329457184735) bank67194 +67195 POINT(40.34438758012371 73.18665790837005) bank67195 +67196 POINT(40.564274467290225 73.8741343930268) bank67196 +67197 POINT(41.55882038673868 73.14189742625221) bank67197 +67198 POINT(40.551846478107976 73.81244491119918) bank67198 +67199 POINT(40.29439983310172 73.31001687679993) bank67199 +67200 POINT(40.67888116214295 74.40969031363521) bank67200 +67201 POINT(40.45761165417931 74.40565069943071) bank67201 +67202 POINT(41.02258535358824 74.411527772537) bank67202 +67203 POINT(40.72948439701896 74.5860985516033) bank67203 +67204 POINT(41.39412545980821 73.23675457404654) bank67204 +67205 POINT(39.76713851395116 74.30171199259445) bank67205 +67206 POINT(40.49543289659239 73.49327808415909) bank67206 +67207 POINT(40.96686080212892 74.86254603045118) bank67207 +67208 POINT(41.28946699148499 74.43752389710855) bank67208 +67209 POINT(41.403083491531724 73.04710694949306) bank67209 +67210 POINT(41.62975499910817 73.70818777398333) bank67210 +67211 POINT(41.38965901129329 74.0104650004723) bank67211 +67212 POINT(40.74863878895251 74.81037397920284) bank67212 +67213 POINT(41.53445806406725 74.03811275746617) bank67213 +67214 POINT(40.22619908494309 74.73269029143762) bank67214 +67215 POINT(39.82504037227534 74.24826349432988) bank67215 +67216 POINT(41.40887062988305 73.68772063806365) bank67216 +67217 POINT(40.93343910241195 74.47538690037973) bank67217 +67218 POINT(40.44113798929513 73.7563918065198) bank67218 +67219 POINT(40.509023111051825 74.37003204459576) bank67219 +67220 POINT(40.83545072912532 74.40569768296864) bank67220 +67221 POINT(40.135467830786084 74.77184696663114) bank67221 +67222 POINT(39.87841840434956 73.16119831365842) bank67222 +67223 POINT(41.44748053769396 74.3885057230714) bank67223 +67224 POINT(40.28546295677761 74.75860771421968) bank67224 +67225 POINT(39.74706059828975 73.99419640658283) bank67225 +67226 POINT(41.037190071789794 74.53868201587275) bank67226 +67227 POINT(39.89843524887807 74.94207045528704) bank67227 +67228 POINT(40.47462002903243 73.88048980701686) bank67228 +67229 POINT(41.49733521839537 73.3955369244211) bank67229 +67230 POINT(40.04062629771424 74.61623761160634) bank67230 +67231 POINT(41.19230058181857 74.12063014789098) bank67231 +67232 POINT(41.48634099772985 74.7129208635638) bank67232 +67233 POINT(40.70729533010756 74.19354198721338) bank67233 +67234 POINT(39.84083897385681 74.95431521103589) bank67234 +67235 POINT(40.17595964820853 74.15146581071731) bank67235 +67236 POINT(40.57758122358217 73.63425287095713) bank67236 +67237 POINT(40.65494840065413 73.33946015257743) bank67237 +67238 POINT(41.296107407439834 73.07028208002232) bank67238 +67239 POINT(40.621284899897574 73.64244736909902) bank67239 +67240 POINT(40.872529850272734 74.24643378546118) bank67240 +67241 POINT(40.21330189953179 74.94299256176276) bank67241 +67242 POINT(40.82970952534512 74.07037266957315) bank67242 +67243 POINT(41.65100046844178 74.15135314419031) bank67243 +67244 POINT(39.968411302581295 74.45478895536327) bank67244 +67245 POINT(41.56266045224512 74.67056456392312) bank67245 +67246 POINT(40.2610465310578 74.56141319455024) bank67246 +67247 POINT(41.363648933797805 74.42600379233369) bank67247 +67248 POINT(40.455442931707715 74.03884624299337) bank67248 +67249 POINT(40.397262866160276 73.34334864533272) bank67249 +67250 POINT(41.4356155915095 73.94603012719112) bank67250 +67251 POINT(41.66632434044749 74.49051653016566) bank67251 +67252 POINT(40.135001574089344 74.93376061840422) bank67252 +67253 POINT(41.14574246871804 73.82435049905851) bank67253 +67254 POINT(39.73013775916327 74.747139049799) bank67254 +67255 POINT(39.874386779727935 74.86200579543679) bank67255 +67256 POINT(41.47711900222023 73.50842106861033) bank67256 +67257 POINT(40.452757049313604 74.09752688654983) bank67257 +67258 POINT(40.57383079441983 73.15430936213704) bank67258 +67259 POINT(41.573227078469806 73.5159332810309) bank67259 +67260 POINT(40.27500807894759 74.60770935896932) bank67260 +67261 POINT(39.943709850214795 73.46425862132045) bank67261 +67262 POINT(40.90457286929861 74.97185066093942) bank67262 +67263 POINT(39.7972660760544 74.5955028135635) bank67263 +67264 POINT(39.81046826040892 73.07256474753183) bank67264 +67265 POINT(40.061454596379804 74.34896647707603) bank67265 +67266 POINT(40.76847778083807 73.0594498094956) bank67266 +67267 POINT(40.606176895791826 74.84639780456642) bank67267 +67268 POINT(41.013734278873294 74.96672864430033) bank67268 +67269 POINT(40.118104066374656 74.84731072414074) bank67269 +67270 POINT(40.523591452644034 74.75863657321129) bank67270 +67271 POINT(39.84278046543559 74.7155020018134) bank67271 +67272 POINT(41.49862960708801 74.72843289639911) bank67272 +67273 POINT(40.393019464058895 73.20536636660711) bank67273 +67274 POINT(41.138646576890295 73.14298549715944) bank67274 +67275 POINT(41.558649763441245 73.31483165998246) bank67275 +67276 POINT(41.12968864219922 74.51844740519086) bank67276 +67277 POINT(40.25124395823463 73.91663794577612) bank67277 +67278 POINT(41.42012371424681 74.67346136292194) bank67278 +67279 POINT(40.22040708687998 74.21769263248707) bank67279 +67280 POINT(40.4607472714143 73.90583026998685) bank67280 +67281 POINT(41.420768605020726 73.30914336089187) bank67281 +67282 POINT(40.851151383855395 74.00229174704265) bank67282 +67283 POINT(40.35369269565758 74.479493995648) bank67283 +67284 POINT(39.85408601236432 73.59601398444613) bank67284 +67285 POINT(40.2754525733119 74.51864715236323) bank67285 +67286 POINT(40.19274924017919 74.3865309947178) bank67286 +67287 POINT(39.93026188361502 73.61048531064283) bank67287 +67288 POINT(41.26292851375911 74.90298039701071) bank67288 +67289 POINT(41.03937828656079 74.44131371523255) bank67289 +67290 POINT(41.5740061160778 74.97944361239807) bank67290 +67291 POINT(40.95483193423015 73.23061356919452) bank67291 +67292 POINT(39.90012725510039 74.18887578286349) bank67292 +67293 POINT(39.920985865663944 74.53229675824028) bank67293 +67294 POINT(40.24366441687558 74.14375970721292) bank67294 +67295 POINT(40.92393127125361 74.37351488438016) bank67295 +67296 POINT(40.43660418348666 73.86334697275181) bank67296 +67297 POINT(39.86581235180275 73.10169952883327) bank67297 +67298 POINT(40.87669950008713 74.26939089820095) bank67298 +67299 POINT(40.68612086644591 74.92378451867872) bank67299 +67300 POINT(41.284704294192935 73.26883540417704) bank67300 +67301 POINT(41.32251729057927 74.37700140254351) bank67301 +67302 POINT(40.400438746157455 73.66711230260616) bank67302 +67303 POINT(40.64437307622238 74.27062225380955) bank67303 +67304 POINT(40.203369084592865 73.99020107396728) bank67304 +67305 POINT(40.90228581637971 73.66365109305376) bank67305 +67306 POINT(41.17430967817485 74.58332799924376) bank67306 +67307 POINT(40.59103669367347 74.67102537300673) bank67307 +67308 POINT(40.02311305337453 73.1278710727763) bank67308 +67309 POINT(41.298247797918805 74.90506844322329) bank67309 +67310 POINT(41.29350395264736 74.68280340460868) bank67310 +67311 POINT(41.6325226649547 74.89511585858763) bank67311 +67312 POINT(41.620791084564445 74.3655937096354) bank67312 +67313 POINT(40.318913541983086 73.9227700694787) bank67313 +67314 POINT(40.29146564675374 74.93577721691133) bank67314 +67315 POINT(41.041521486032735 73.34816589508668) bank67315 +67316 POINT(41.13125215670161 73.2995916564824) bank67316 +67317 POINT(40.74803270806973 73.27933407955324) bank67317 +67318 POINT(41.07646967805391 73.50780986242529) bank67318 +67319 POINT(39.887605566642335 74.08561365226714) bank67319 +67320 POINT(40.18227145917013 73.16963806845277) bank67320 +67321 POINT(39.7349659583538 73.9117235248533) bank67321 +67322 POINT(40.81308386056015 73.3392880206339) bank67322 +67323 POINT(40.817325518098286 74.93677882307917) bank67323 +67324 POINT(40.48182361530724 73.85045465116339) bank67324 +67325 POINT(41.16620846355681 74.0374218027215) bank67325 +67326 POINT(40.695759320543054 74.20399453199707) bank67326 +67327 POINT(41.061250117189665 74.97852591552224) bank67327 +67328 POINT(40.8422974085182 74.62919815494968) bank67328 +67329 POINT(41.44047165573398 74.76142576270928) bank67329 +67330 POINT(40.26642042041777 73.9872600200487) bank67330 +67331 POINT(41.69157952438738 74.57167630568242) bank67331 +67332 POINT(40.94544086295459 73.31345919877087) bank67332 +67333 POINT(39.79150281822915 74.92018772223953) bank67333 +67334 POINT(40.75107749649719 73.40220760369994) bank67334 +67335 POINT(40.52803710550765 73.04941605762082) bank67335 +67336 POINT(40.920819609222384 74.00225115177115) bank67336 +67337 POINT(39.82872197067833 73.54478886919192) bank67337 +67338 POINT(39.95623834779233 73.62728226981879) bank67338 +67339 POINT(41.242175994874145 73.94968573985099) bank67339 +67340 POINT(40.243594527188066 73.86461062171009) bank67340 +67341 POINT(41.51788931658976 74.83276972958654) bank67341 +67342 POINT(39.968218950539274 74.88726875165727) bank67342 +67343 POINT(40.19157446815951 73.74060785927341) bank67343 +67344 POINT(41.081378817155354 74.33559770325601) bank67344 +67345 POINT(40.452852631308836 73.94797647726617) bank67345 +67346 POINT(40.81915824236368 73.05031948192631) bank67346 +67347 POINT(41.2548325449534 74.98977137243936) bank67347 +67348 POINT(40.36189128606422 74.54513880902923) bank67348 +67349 POINT(40.03574696642628 73.71634813555956) bank67349 +67350 POINT(39.76084558815616 73.52283461253162) bank67350 +67351 POINT(41.658318218122155 73.50563928146804) bank67351 +67352 POINT(40.156535167192445 74.33173821582527) bank67352 +67353 POINT(41.60875009791759 74.5580916099227) bank67353 +67354 POINT(41.346862517344235 73.393364078143) bank67354 +67355 POINT(41.01145862983153 73.0725491317512) bank67355 +67356 POINT(41.0009118508466 74.59770269267706) bank67356 +67357 POINT(40.523517964687635 74.33696525919079) bank67357 +67358 POINT(39.98624299582626 73.32287986718204) bank67358 +67359 POINT(41.59153046701104 73.7458882974495) bank67359 +67360 POINT(40.86068212851129 73.37775945331566) bank67360 +67361 POINT(41.698162850409595 73.62730913228451) bank67361 +67362 POINT(41.38736603014035 74.27796389557726) bank67362 +67363 POINT(39.71763890008972 73.5281619256066) bank67363 +67364 POINT(40.41068097850706 73.64648532506682) bank67364 +67365 POINT(41.52164883051075 74.71620685990082) bank67365 +67366 POINT(40.55045067792192 74.0256177040826) bank67366 +67367 POINT(39.948156475075194 74.35771117232525) bank67367 +67368 POINT(40.149505300397394 73.83682899323118) bank67368 +67369 POINT(40.977441460314786 73.74132504095014) bank67369 +67370 POINT(41.0492528053593 73.72590844640592) bank67370 +67371 POINT(40.54406881361547 74.18766076996197) bank67371 +67372 POINT(39.73038101117789 73.65821805239193) bank67372 +67373 POINT(40.76919550954298 73.08641194508574) bank67373 +67374 POINT(39.92747797217905 74.59022225912936) bank67374 +67375 POINT(40.79054502645733 73.74627689153283) bank67375 +67376 POINT(40.32070518600449 74.25963072928647) bank67376 +67377 POINT(39.994531075489064 73.18846900651293) bank67377 +67378 POINT(40.568681632139736 73.1447383077661) bank67378 +67379 POINT(41.24438564346129 74.55923466401455) bank67379 +67380 POINT(41.1813327353986 73.22484979003865) bank67380 +67381 POINT(40.512866708708266 73.05723532856764) bank67381 +67382 POINT(40.72032965698901 73.39896467489882) bank67382 +67383 POINT(40.167263493574396 74.20878632204028) bank67383 +67384 POINT(40.11391030333506 73.26024105514298) bank67384 +67385 POINT(41.02126642417754 74.64961196271048) bank67385 +67386 POINT(39.735155335619645 74.43635541768755) bank67386 +67387 POINT(40.83417783923568 73.8208733188802) bank67387 +67388 POINT(39.72364140242782 74.90348871524351) bank67388 +67389 POINT(39.9148131869384 73.56299603563356) bank67389 +67390 POINT(41.09740485884886 74.23601736504308) bank67390 +67391 POINT(39.713414854131784 74.81104541018091) bank67391 +67392 POINT(39.86969726722727 73.33055876136538) bank67392 +67393 POINT(41.52793740485854 73.3953627826963) bank67393 +67394 POINT(40.89137985921421 74.71274125604057) bank67394 +67395 POINT(39.80974535916702 73.22272311148055) bank67395 +67396 POINT(41.03479600374194 74.25104565832626) bank67396 +67397 POINT(40.011418967950156 73.6592978913151) bank67397 +67398 POINT(40.475374237803685 73.86725075742751) bank67398 +67399 POINT(40.85348001033734 73.45315731795715) bank67399 +67400 POINT(39.81812647534237 73.7752347293985) bank67400 +67401 POINT(39.853820503623034 73.09528280271209) bank67401 +67402 POINT(40.41455084024584 73.2715963256799) bank67402 +67403 POINT(40.58157001459312 73.67440904933059) bank67403 +67404 POINT(40.973877284316245 73.47889016034297) bank67404 +67405 POINT(40.12377914737949 74.27960620417413) bank67405 +67406 POINT(41.5328100162887 73.14515038679201) bank67406 +67407 POINT(39.81927589035031 73.05403783318445) bank67407 +67408 POINT(39.79704229523009 73.02203711619185) bank67408 +67409 POINT(41.37863074408086 74.14133536205883) bank67409 +67410 POINT(39.94813926658213 74.64938311778806) bank67410 +67411 POINT(41.51615966852505 74.158370463946) bank67411 +67412 POINT(41.35624092084083 73.04365680030156) bank67412 +67413 POINT(40.81194984465537 73.07054974128194) bank67413 +67414 POINT(39.76823504777406 74.10765978763966) bank67414 +67415 POINT(40.971206650361175 74.80619384208957) bank67415 +67416 POINT(40.21594568047068 74.92975805340755) bank67416 +67417 POINT(41.67986958750209 74.28778699416051) bank67417 +67418 POINT(40.517592837392726 74.7366949624785) bank67418 +67419 POINT(40.03766684739544 74.20496228792665) bank67419 +67420 POINT(41.22705429788329 74.68350005877085) bank67420 +67421 POINT(41.31610596176098 73.4938985235349) bank67421 +67422 POINT(41.34395729468125 74.04134629890392) bank67422 +67423 POINT(41.567413384370035 73.49909040618883) bank67423 +67424 POINT(41.23815644787581 73.47946468309787) bank67424 +67425 POINT(41.28180581776177 73.33840757571019) bank67425 +67426 POINT(39.73296196704825 73.79719317323877) bank67426 +67427 POINT(40.99157319074575 73.36282959292096) bank67427 +67428 POINT(39.99069489136683 74.06220949688571) bank67428 +67429 POINT(40.53175804856447 73.09587250656685) bank67429 +67430 POINT(40.52347798586945 73.93719773649906) bank67430 +67431 POINT(39.7484461700564 73.47315550941849) bank67431 +67432 POINT(41.54657887431958 73.23359315344233) bank67432 +67433 POINT(41.55699511311775 73.17953457599634) bank67433 +67434 POINT(40.568933019060836 74.16759413864445) bank67434 +67435 POINT(41.231503406064846 73.32755584657718) bank67435 +67436 POINT(41.169229556077156 73.75614302279855) bank67436 +67437 POINT(39.917475820871765 75.0017427572718) bank67437 +67438 POINT(41.03253673448956 74.49567265163364) bank67438 +67439 POINT(41.07885272943437 73.53759304427888) bank67439 +67440 POINT(41.60819684505005 73.32616699840398) bank67440 +67441 POINT(40.92201396991226 73.77925596429861) bank67441 +67442 POINT(40.53733724575754 74.50397471421437) bank67442 +67443 POINT(41.3660015234887 74.5842070115078) bank67443 +67444 POINT(41.21094662225077 74.27169666051759) bank67444 +67445 POINT(40.768211163167784 73.39961684729192) bank67445 +67446 POINT(41.206689188284585 73.3787191846147) bank67446 +67447 POINT(41.0609911564911 74.91676577688642) bank67447 +67448 POINT(39.736278914286515 73.19266279046495) bank67448 +67449 POINT(41.524115803045014 74.3616647633863) bank67449 +67450 POINT(41.210106909285194 74.74193521436551) bank67450 +67451 POINT(40.48392624805558 74.83592054223078) bank67451 +67452 POINT(39.780626262988854 74.37640498041435) bank67452 +67453 POINT(40.27706837094988 73.65868968486959) bank67453 +67454 POINT(41.26005215674717 74.80105027178357) bank67454 +67455 POINT(41.07500293622138 73.48022416318547) bank67455 +67456 POINT(40.77744247437936 74.70479629753244) bank67456 +67457 POINT(41.50995956577369 73.93082983334705) bank67457 +67458 POINT(40.164792986203985 74.09082013831055) bank67458 +67459 POINT(39.74095319778812 74.50849570582889) bank67459 +67460 POINT(41.298227166994664 73.9556710681464) bank67460 +67461 POINT(41.41508880221786 73.10851224116654) bank67461 +67462 POINT(41.08164289229522 73.2841013969717) bank67462 +67463 POINT(40.1741802212663 73.73334425005585) bank67463 +67464 POINT(40.872143388582344 73.88151516260999) bank67464 +67465 POINT(39.72196408304667 74.48314302285046) bank67465 +67466 POINT(41.10803025838924 74.7050357489982) bank67466 +67467 POINT(41.26337780014158 73.2442502129938) bank67467 +67468 POINT(41.52598939681004 74.49793163652484) bank67468 +67469 POINT(39.82491937103826 73.4225655173398) bank67469 +67470 POINT(40.787652835919054 73.22653723044407) bank67470 +67471 POINT(40.3068012576415 73.98186062435578) bank67471 +67472 POINT(41.42438703884794 74.09840600877259) bank67472 +67473 POINT(39.98100486887755 73.42385837103268) bank67473 +67474 POINT(40.8224794617195 74.51234727471385) bank67474 +67475 POINT(40.26389357636994 73.97116994211639) bank67475 +67476 POINT(39.745547470990154 73.42490129596744) bank67476 +67477 POINT(41.66718056290474 73.1131402832294) bank67477 +67478 POINT(41.102072281835525 73.25452007010588) bank67478 +67479 POINT(41.03493185405832 73.78590623922167) bank67479 +67480 POINT(39.99267682382988 74.25314994320722) bank67480 +67481 POINT(40.88470833396723 74.45996133908287) bank67481 +67482 POINT(41.557413012358495 74.89917726591283) bank67482 +67483 POINT(41.53624996186651 74.00620264196871) bank67483 +67484 POINT(40.53191744277903 74.77255756383882) bank67484 +67485 POINT(41.04319153412116 74.61785570452396) bank67485 +67486 POINT(41.67084759413868 74.2100874445918) bank67486 +67487 POINT(41.02483912092304 74.25626770560146) bank67487 +67488 POINT(41.04732057438789 74.77607268710999) bank67488 +67489 POINT(39.715339899704894 73.38149809570719) bank67489 +67490 POINT(41.17543788963278 73.53777138557525) bank67490 +67491 POINT(39.92140015601598 74.25207779358828) bank67491 +67492 POINT(41.520074253225076 73.58529517493128) bank67492 +67493 POINT(41.412938880045715 73.74854902192006) bank67493 +67494 POINT(40.31812944301699 73.54245963637699) bank67494 +67495 POINT(41.078549771373545 73.71017712083761) bank67495 +67496 POINT(40.59857268529809 74.22036918287152) bank67496 +67497 POINT(40.628742144942045 74.72864668499041) bank67497 +67498 POINT(39.74877413183015 73.84080080238331) bank67498 +67499 POINT(41.70520926305227 73.2854595781636) bank67499 +67500 POINT(41.03050392758108 74.85429084491676) bank67500 +67501 POINT(41.41771326240281 73.76019816592783) bank67501 +67502 POINT(40.541423299591635 73.52049971882698) bank67502 +67503 POINT(41.41148293226116 73.66958590451905) bank67503 +67504 POINT(39.976042657701065 74.20547107765466) bank67504 +67505 POINT(39.88086403600213 74.55949681228287) bank67505 +67506 POINT(41.15986720553053 74.7679413139405) bank67506 +67507 POINT(41.17322390732268 74.7925875778986) bank67507 +67508 POINT(40.2998062102254 74.50583392960114) bank67508 +67509 POINT(39.81824118172501 73.96833160803736) bank67509 +67510 POINT(41.057703958879614 74.60205034725769) bank67510 +67511 POINT(41.5078891253325 73.61045909980857) bank67511 +67512 POINT(41.43477769822706 74.10431998823161) bank67512 +67513 POINT(40.7640743480986 73.84132949330005) bank67513 +67514 POINT(40.98567756628019 73.3139003913132) bank67514 +67515 POINT(39.76925652877577 73.52364966624381) bank67515 +67516 POINT(40.464565832834175 73.97664145224566) bank67516 +67517 POINT(39.74951338395047 73.49918233747968) bank67517 +67518 POINT(40.785952129528305 74.89966983808192) bank67518 +67519 POINT(41.08313268776818 74.13394631348518) bank67519 +67520 POINT(39.798048172631006 73.64949664382246) bank67520 +67521 POINT(39.869087138142916 73.0601109670162) bank67521 +67522 POINT(41.59117724147793 74.7408098473874) bank67522 +67523 POINT(40.85420158295174 73.01514766045423) bank67523 +67524 POINT(40.24432161459064 73.4036544602215) bank67524 +67525 POINT(40.3838609589927 74.93044691475035) bank67525 +67526 POINT(40.47732578016911 73.4086617441708) bank67526 +67527 POINT(39.90333534649486 73.04868102671217) bank67527 +67528 POINT(40.11495153945444 73.82262692822204) bank67528 +67529 POINT(40.18560476061711 74.20795626526801) bank67529 +67530 POINT(40.104461465191456 73.47745799459986) bank67530 +67531 POINT(41.0863850301093 74.23237376960768) bank67531 +67532 POINT(39.89652625914234 73.98267718547547) bank67532 +67533 POINT(41.34850559441801 73.75769550381224) bank67533 +67534 POINT(40.363824587079634 74.6496971375729) bank67534 +67535 POINT(40.57939283707081 74.4697183172805) bank67535 +67536 POINT(40.23423105074613 73.82475433463057) bank67536 +67537 POINT(39.785173033529574 74.69506611695246) bank67537 +67538 POINT(39.93021659606141 74.35937252838455) bank67538 +67539 POINT(40.87229202978889 74.08398149228677) bank67539 +67540 POINT(40.421786720265125 74.15119945219806) bank67540 +67541 POINT(40.31336455706345 73.32719922615905) bank67541 +67542 POINT(40.08228916103283 74.24933866674053) bank67542 +67543 POINT(40.37069446916808 73.60005059892801) bank67543 +67544 POINT(41.4387623197429 74.13432246846143) bank67544 +67545 POINT(41.44464685725722 74.74190731617492) bank67545 +67546 POINT(40.240612518841345 74.4678593150267) bank67546 +67547 POINT(41.46128015592563 74.34900779482034) bank67547 +67548 POINT(41.557408563268375 74.56970863222155) bank67548 +67549 POINT(41.184112739292566 74.09204185665452) bank67549 +67550 POINT(41.54872247527972 73.03491614487986) bank67550 +67551 POINT(41.12705981824955 74.28298248616126) bank67551 +67552 POINT(40.19837404467661 74.06196311476546) bank67552 +67553 POINT(40.39245486610057 73.02817338274811) bank67553 +67554 POINT(41.26483381346253 73.11894015005508) bank67554 +67555 POINT(40.95907966739634 73.10031618804805) bank67555 +67556 POINT(41.41250301049413 73.71064195138432) bank67556 +67557 POINT(40.59640794027807 74.65531173434958) bank67557 +67558 POINT(40.49818690140053 74.21051341846439) bank67558 +67559 POINT(41.10625691286763 74.53824113002173) bank67559 +67560 POINT(41.653983795894405 73.71283987275609) bank67560 +67561 POINT(41.12628856690984 73.0533580707687) bank67561 +67562 POINT(41.13820051595212 74.57395237191781) bank67562 +67563 POINT(40.44792120129016 73.56361947235698) bank67563 +67564 POINT(40.03042547526919 73.23803873602685) bank67564 +67565 POINT(41.079108697030385 73.71609209858524) bank67565 +67566 POINT(40.4792099357224 74.22648144756265) bank67566 +67567 POINT(41.120576839104544 73.24142443094672) bank67567 +67568 POINT(40.358346699371 73.79651807834239) bank67568 +67569 POINT(40.63106446122721 74.14116683463659) bank67569 +67570 POINT(41.57879989195636 74.13568018557719) bank67570 +67571 POINT(41.5533128849566 74.24100160439126) bank67571 +67572 POINT(40.24725299797994 74.01762344535824) bank67572 +67573 POINT(41.632006130200914 74.35659640739914) bank67573 +67574 POINT(40.48268199523499 73.06986682472558) bank67574 +67575 POINT(40.81132657489522 73.25456627985695) bank67575 +67576 POINT(41.64205981985377 74.4035448998804) bank67576 +67577 POINT(40.53537411850753 73.44694487160561) bank67577 +67578 POINT(39.97930747880865 74.60792040693667) bank67578 +67579 POINT(40.71474825603528 73.34087034333562) bank67579 +67580 POINT(40.39519407516853 73.3503489435598) bank67580 +67581 POINT(40.0203606596918 73.41011224823671) bank67581 +67582 POINT(40.24366784447404 74.12665158909624) bank67582 +67583 POINT(40.47607155495851 74.00947179592106) bank67583 +67584 POINT(39.759008451860275 74.30871962152156) bank67584 +67585 POINT(41.22515356121126 73.5094599942382) bank67585 +67586 POINT(40.28585888623986 73.38626277148153) bank67586 +67587 POINT(40.52350825029189 73.20199752763338) bank67587 +67588 POINT(41.53074158187288 74.02008253895742) bank67588 +67589 POINT(41.49805344381484 74.673854754733) bank67589 +67590 POINT(40.21116270137578 73.556454383256) bank67590 +67591 POINT(41.49151098743056 74.78593079995844) bank67591 +67592 POINT(40.582595169774756 74.49096899645998) bank67592 +67593 POINT(40.65083870837745 73.90617216955059) bank67593 +67594 POINT(40.50676041494844 74.81506347051665) bank67594 +67595 POINT(41.54972134032281 74.7990490595096) bank67595 +67596 POINT(41.3999574621254 73.94318087095941) bank67596 +67597 POINT(40.73138381173821 73.84699322750463) bank67597 +67598 POINT(40.85843033946488 74.41279606936874) bank67598 +67599 POINT(39.787738823694255 73.08529243038306) bank67599 +67600 POINT(39.77410963109572 74.99200434062517) bank67600 +67601 POINT(40.692456474440036 74.02221424772512) bank67601 +67602 POINT(40.237417232106836 73.2979344396063) bank67602 +67603 POINT(40.554774893113695 74.89176347725346) bank67603 +67604 POINT(40.97636913043932 74.5701538208775) bank67604 +67605 POINT(40.530675587482854 73.84519982116676) bank67605 +67606 POINT(40.971138273718104 74.28480561121133) bank67606 +67607 POINT(40.76375756795385 74.60045571949352) bank67607 +67608 POINT(41.630493722261704 73.32132216035804) bank67608 +67609 POINT(41.18353919496237 73.05419503989708) bank67609 +67610 POINT(41.44560658847731 74.86101167159742) bank67610 +67611 POINT(40.445342265573544 74.70066836721217) bank67611 +67612 POINT(40.117781974406284 73.75980368383327) bank67612 +67613 POINT(40.97506326540625 73.4327126668356) bank67613 +67614 POINT(40.241239624218025 74.02806951355106) bank67614 +67615 POINT(41.319549428227184 73.8983639421444) bank67615 +67616 POINT(39.96808484846383 74.39847858637246) bank67616 +67617 POINT(41.376630636981375 74.67835189946146) bank67617 +67618 POINT(41.19041406614901 74.33420829838114) bank67618 +67619 POINT(40.67998491665878 73.40704105927256) bank67619 +67620 POINT(40.930322372276805 73.81955841524048) bank67620 +67621 POINT(40.620940497343824 74.32740684890379) bank67621 +67622 POINT(41.413812979881 74.51266616585114) bank67622 +67623 POINT(40.08651333857227 74.94276974873398) bank67623 +67624 POINT(40.36528192642609 74.97852479906304) bank67624 +67625 POINT(39.8590352255696 73.50051028548002) bank67625 +67626 POINT(41.593386712895864 74.41912410572868) bank67626 +67627 POINT(40.89674912922752 74.09336647919402) bank67627 +67628 POINT(41.69034513439331 73.19759854037568) bank67628 +67629 POINT(40.11179252331034 73.61140469265396) bank67629 +67630 POINT(40.58443293242959 74.50152871678777) bank67630 +67631 POINT(40.7261059299527 74.21992601211278) bank67631 +67632 POINT(41.46133890851064 74.00234211265024) bank67632 +67633 POINT(40.838562387291105 74.9661136293631) bank67633 +67634 POINT(40.53100004347857 73.64801845311834) bank67634 +67635 POINT(39.74971716310912 74.11831826631806) bank67635 +67636 POINT(40.90898615408906 73.6820729391724) bank67636 +67637 POINT(40.37399164262555 74.75987353096886) bank67637 +67638 POINT(40.47983389803532 74.39837737909029) bank67638 +67639 POINT(41.34469322213029 73.74157097547156) bank67639 +67640 POINT(40.71601087329128 74.05276130649393) bank67640 +67641 POINT(40.31717600997283 74.10073453111904) bank67641 +67642 POINT(41.68624597648792 74.81931172006745) bank67642 +67643 POINT(39.88398320564203 74.10236691233092) bank67643 +67644 POINT(41.67382565557315 73.06488127981464) bank67644 +67645 POINT(39.88158453279864 74.18083411574369) bank67645 +67646 POINT(40.99604918288076 73.53089749875237) bank67646 +67647 POINT(40.04310623479627 74.4984202705659) bank67647 +67648 POINT(40.94834165667996 74.7849412909018) bank67648 +67649 POINT(40.07013030503503 74.25751410570783) bank67649 +67650 POINT(40.657499980823474 74.28886330680274) bank67650 +67651 POINT(41.118794858306536 73.43893203159529) bank67651 +67652 POINT(41.36178396519587 73.06531301471718) bank67652 +67653 POINT(40.60525118093334 73.21761404133818) bank67653 +67654 POINT(40.11346857423242 73.34297708361727) bank67654 +67655 POINT(40.947002454152475 73.97646105034622) bank67655 +67656 POINT(41.01166425531392 74.78487029680187) bank67656 +67657 POINT(41.427639881849075 73.74840024841934) bank67657 +67658 POINT(40.90974667330919 73.5293081653612) bank67658 +67659 POINT(40.81345885482785 73.31311637072848) bank67659 +67660 POINT(40.0759131239443 73.03045446801198) bank67660 +67661 POINT(39.76146416940887 74.32025467292519) bank67661 +67662 POINT(41.3930719395316 74.12557717372454) bank67662 +67663 POINT(41.69222087416501 74.81805304553816) bank67663 +67664 POINT(40.12610405575492 73.8371852425627) bank67664 +67665 POINT(41.58936697516752 74.02840993734414) bank67665 +67666 POINT(40.56561819575984 73.50646532399108) bank67666 +67667 POINT(40.39049548124698 74.05853561440492) bank67667 +67668 POINT(40.09121017533763 73.55686522624428) bank67668 +67669 POINT(40.734167723196094 74.69511891915558) bank67669 +67670 POINT(41.16495384525206 74.94067527124466) bank67670 +67671 POINT(41.60402810129112 73.16449130316289) bank67671 +67672 POINT(41.62630891592186 74.20785860621537) bank67672 +67673 POINT(41.31328402887495 74.81800266760196) bank67673 +67674 POINT(40.220278703674936 74.09736578395574) bank67674 +67675 POINT(40.15127109631278 74.11288132682975) bank67675 +67676 POINT(39.77358718254022 74.60390804322788) bank67676 +67677 POINT(40.06917754900816 73.82502849032205) bank67677 +67678 POINT(40.71044268404334 74.78807288502028) bank67678 +67679 POINT(40.20478890672335 73.5353520615749) bank67679 +67680 POINT(40.936695606827094 74.72728261274813) bank67680 +67681 POINT(39.83509770243058 73.39465608005821) bank67681 +67682 POINT(39.87138492871453 73.88835105223734) bank67682 +67683 POINT(40.16171165151183 73.64398366091785) bank67683 +67684 POINT(41.29885071638433 74.80720364083164) bank67684 +67685 POINT(40.034692253487016 73.74692908632227) bank67685 +67686 POINT(40.799874829405745 74.84463584516756) bank67686 +67687 POINT(40.31177585254517 74.3612090836097) bank67687 +67688 POINT(40.60559875018306 73.48691172542175) bank67688 +67689 POINT(40.499934484801734 74.78824526165099) bank67689 +67690 POINT(41.175280000656464 73.5388897155781) bank67690 +67691 POINT(40.933252607884604 73.74977782203185) bank67691 +67692 POINT(41.29626298232179 73.74580973083414) bank67692 +67693 POINT(40.77473092968181 74.89824057763155) bank67693 +67694 POINT(41.19057455763337 74.63215008105595) bank67694 +67695 POINT(39.81665441706305 73.85131660302844) bank67695 +67696 POINT(40.08130919613682 73.35908121643583) bank67696 +67697 POINT(40.71820724504366 73.30514104030178) bank67697 +67698 POINT(40.11340796345897 74.04984044984742) bank67698 +67699 POINT(41.12899063245077 74.9717686168404) bank67699 +67700 POINT(41.18561622237589 73.04231244285542) bank67700 +67701 POINT(40.15496334229433 73.98081105186871) bank67701 +67702 POINT(40.176363963898396 74.00479777729777) bank67702 +67703 POINT(40.244397131740165 74.36747280753973) bank67703 +67704 POINT(39.93628973547978 74.85996095564764) bank67704 +67705 POINT(41.413688775234554 73.31266474379056) bank67705 +67706 POINT(40.72069329809289 73.47245677442938) bank67706 +67707 POINT(40.09910481557324 74.69226480213798) bank67707 +67708 POINT(40.477690991593796 73.0739043639962) bank67708 +67709 POINT(40.52820973157448 73.31303496540711) bank67709 +67710 POINT(41.1794177738709 74.36715314778137) bank67710 +67711 POINT(40.54059609793096 74.98547714984738) bank67711 +67712 POINT(39.98919960629638 73.90672525695406) bank67712 +67713 POINT(41.40168978448192 74.40701433430199) bank67713 +67714 POINT(40.1640044146442 74.31755909041797) bank67714 +67715 POINT(39.985756362470596 74.04104889271274) bank67715 +67716 POINT(39.71308873370784 74.98037568100145) bank67716 +67717 POINT(40.828485590686604 74.07620479763769) bank67717 +67718 POINT(40.921703569203906 73.56577528704123) bank67718 +67719 POINT(41.249498201992175 73.59630489284605) bank67719 +67720 POINT(40.60268201867761 73.01706535279405) bank67720 +67721 POINT(41.43313924029743 74.12123537409714) bank67721 +67722 POINT(40.149642081616875 74.92804951694195) bank67722 +67723 POINT(39.87821219801839 74.40699396954355) bank67723 +67724 POINT(41.6427847278066 74.52244682885465) bank67724 +67725 POINT(40.039247074975194 73.9680943786759) bank67725 +67726 POINT(40.38725195917289 73.13830057245465) bank67726 +67727 POINT(39.80428279865479 73.28566807904274) bank67727 +67728 POINT(40.125165026276754 73.62081102823709) bank67728 +67729 POINT(40.045596061650556 74.06031570829505) bank67729 +67730 POINT(39.99209253590803 74.72254497371995) bank67730 +67731 POINT(41.6547107728569 73.64349148368711) bank67731 +67732 POINT(41.62365704756889 74.3362954458689) bank67732 +67733 POINT(39.926525855013914 73.56943108322007) bank67733 +67734 POINT(41.534105288696395 73.0878366785007) bank67734 +67735 POINT(39.81392227034126 73.05511931269719) bank67735 +67736 POINT(40.55511044037386 74.0967431406553) bank67736 +67737 POINT(39.86012879953491 74.38532011155841) bank67737 +67738 POINT(39.73817358995869 74.31797409042662) bank67738 +67739 POINT(39.937018632342784 73.82142143457803) bank67739 +67740 POINT(40.55254668045586 73.4875410505359) bank67740 +67741 POINT(40.57610712748562 73.7794478149884) bank67741 +67742 POINT(40.06140408359995 74.67841194519926) bank67742 +67743 POINT(40.99388007444973 73.38456831325476) bank67743 +67744 POINT(41.200257446520645 73.72862712635246) bank67744 +67745 POINT(40.77227837605333 73.82830516245322) bank67745 +67746 POINT(41.23116433336657 73.83164508814282) bank67746 +67747 POINT(39.76666728932459 74.66571823358495) bank67747 +67748 POINT(40.013527863168775 73.07654223714376) bank67748 +67749 POINT(40.46985864610128 74.64249716327478) bank67749 +67750 POINT(40.4889223039172 73.54103674179045) bank67750 +67751 POINT(41.15715624044791 74.16287080433939) bank67751 +67752 POINT(41.05751746387089 74.84937954651573) bank67752 +67753 POINT(40.94973959935966 74.39912729889443) bank67753 +67754 POINT(40.6539167590456 74.54215849043034) bank67754 +67755 POINT(41.703094581161885 74.9975688031027) bank67755 +67756 POINT(41.15526879803656 74.68888465744584) bank67756 +67757 POINT(40.35040919001232 74.90293766841577) bank67757 +67758 POINT(41.3688771483533 74.55318610991466) bank67758 +67759 POINT(40.815644365664845 74.20022478401567) bank67759 +67760 POINT(41.25797698439786 73.08891589337543) bank67760 +67761 POINT(40.323967971227994 74.05232931302199) bank67761 +67762 POINT(39.82949619561757 74.39949616246899) bank67762 +67763 POINT(41.42566578059954 74.95465366105017) bank67763 +67764 POINT(41.4941056140942 73.06992619530268) bank67764 +67765 POINT(41.41465646067639 73.48889863639134) bank67765 +67766 POINT(40.231614424871545 74.79743619108731) bank67766 +67767 POINT(41.46920901428618 73.95709800637339) bank67767 +67768 POINT(40.85882593476081 73.11154947949579) bank67768 +67769 POINT(40.39888048097638 74.713498011867) bank67769 +67770 POINT(41.70449373476501 74.62596249819167) bank67770 +67771 POINT(40.49673668705756 74.87197621323166) bank67771 +67772 POINT(40.43248729147228 73.06546175859326) bank67772 +67773 POINT(41.19382495164973 75.00131798488668) bank67773 +67774 POINT(41.663082419777034 74.185983183734) bank67774 +67775 POINT(40.72179677682559 73.05159356170289) bank67775 +67776 POINT(39.79660976505994 73.21063039568637) bank67776 +67777 POINT(41.441119183414784 74.11683912830826) bank67777 +67778 POINT(40.47621619992162 73.94582563228653) bank67778 +67779 POINT(39.723507800471246 74.91431503226329) bank67779 +67780 POINT(40.998762219307956 73.30839163348853) bank67780 +67781 POINT(41.40127398182777 73.46189552004238) bank67781 +67782 POINT(40.739566516810115 73.56725969029911) bank67782 +67783 POINT(41.57641754383847 74.44324035061862) bank67783 +67784 POINT(40.08459405795209 73.88071990703789) bank67784 +67785 POINT(40.43881888125101 73.79707709292894) bank67785 +67786 POINT(41.35476584950971 74.68708191157303) bank67786 +67787 POINT(39.82913056207641 73.89231570552306) bank67787 +67788 POINT(39.98689693648987 73.58540019310396) bank67788 +67789 POINT(41.173749297993794 73.10976695422171) bank67789 +67790 POINT(41.086036695091465 74.90524724485799) bank67790 +67791 POINT(41.37378650179825 74.68551184316098) bank67791 +67792 POINT(40.190666515545026 73.30211120085202) bank67792 +67793 POINT(40.00280299376679 73.52096745371139) bank67793 +67794 POINT(41.624592502807644 74.25381012544014) bank67794 +67795 POINT(39.76736841582717 73.42002833771744) bank67795 +67796 POINT(39.94776424788887 74.69954560500078) bank67796 +67797 POINT(41.30826124411477 73.34949119739221) bank67797 +67798 POINT(39.94578897258945 74.36540745222682) bank67798 +67799 POINT(41.45523956518101 74.539552030559) bank67799 +67800 POINT(40.962273976768856 74.94465424770723) bank67800 +67801 POINT(39.90362086867437 74.51373596588223) bank67801 +67802 POINT(40.157588926543 74.41750353496526) bank67802 +67803 POINT(40.298324367146286 74.61585233530695) bank67803 +67804 POINT(41.698472935630704 74.14182874426812) bank67804 +67805 POINT(41.071168883927875 73.93924310590629) bank67805 +67806 POINT(41.66252244039558 74.9167477474319) bank67806 +67807 POINT(40.07260341168687 73.4978940048161) bank67807 +67808 POINT(40.46240596312245 73.84849021911974) bank67808 +67809 POINT(39.86420173846187 73.66606119519646) bank67809 +67810 POINT(40.662289631913865 74.81796325274888) bank67810 +67811 POINT(41.27960202922876 74.24247425459778) bank67811 +67812 POINT(40.6618452324901 73.68284093424084) bank67812 +67813 POINT(40.32524533873246 74.43595391880544) bank67813 +67814 POINT(40.57881801366136 74.15094089001745) bank67814 +67815 POINT(40.23694045661871 74.49537601097573) bank67815 +67816 POINT(40.41871788425026 74.63430824154538) bank67816 +67817 POINT(40.12649385443223 73.16126748898132) bank67817 +67818 POINT(40.080764559202976 74.96118334904683) bank67818 +67819 POINT(40.93350398373394 74.35545647872124) bank67819 +67820 POINT(41.6010931027144 74.04770605264616) bank67820 +67821 POINT(40.72868255400619 73.42069095687762) bank67821 +67822 POINT(40.04633536988346 73.61862031454271) bank67822 +67823 POINT(40.97166182764061 73.70386537520078) bank67823 +67824 POINT(40.273698027696724 74.69724634698423) bank67824 +67825 POINT(40.62187174495828 73.28625693475384) bank67825 +67826 POINT(40.86096629883613 73.63267409427459) bank67826 +67827 POINT(40.44527694944384 74.4618748984585) bank67827 +67828 POINT(40.355903594289465 74.01278451634914) bank67828 +67829 POINT(39.89479574003688 73.2572118229061) bank67829 +67830 POINT(41.307691294143716 74.61320275656927) bank67830 +67831 POINT(40.46242342429955 73.14613588057293) bank67831 +67832 POINT(39.92760351830871 73.85761223517031) bank67832 +67833 POINT(40.56640779231054 73.34527168158127) bank67833 +67834 POINT(41.28196560879726 74.10913758623049) bank67834 +67835 POINT(41.41996722838525 74.97852503532174) bank67835 +67836 POINT(40.496752830772984 74.97753570622031) bank67836 +67837 POINT(40.59284816896066 73.12246038670075) bank67837 +67838 POINT(40.34864489271168 73.98453101252066) bank67838 +67839 POINT(41.08723444318888 74.920503695276) bank67839 +67840 POINT(40.590546176321055 74.85186252500652) bank67840 +67841 POINT(40.64183469123107 73.07273549950432) bank67841 +67842 POINT(39.83205091369033 73.39931627753808) bank67842 +67843 POINT(41.14925865944251 74.8326423741062) bank67843 +67844 POINT(39.94766189920489 73.31010026580823) bank67844 +67845 POINT(40.39584336508773 74.15502806084783) bank67845 +67846 POINT(39.90546052720299 74.19023710082871) bank67846 +67847 POINT(41.62442787810485 73.59626965423492) bank67847 +67848 POINT(40.27748728236471 74.72751306678627) bank67848 +67849 POINT(40.57281699692803 74.68281626557602) bank67849 +67850 POINT(41.333952339496804 73.13159937774448) bank67850 +67851 POINT(40.861475394393054 73.77638282727115) bank67851 +67852 POINT(40.98139223668933 73.65064190137224) bank67852 +67853 POINT(40.428837796882206 74.35696379402444) bank67853 +67854 POINT(40.9988238846384 73.89230628564012) bank67854 +67855 POINT(40.05836482928518 73.03574768891882) bank67855 +67856 POINT(40.98678609904394 73.6265783828608) bank67856 +67857 POINT(41.00477886511074 74.43624572738177) bank67857 +67858 POINT(40.553798500020086 74.77222588796057) bank67858 +67859 POINT(41.25016289903333 74.84959147448365) bank67859 +67860 POINT(41.652425501234994 73.05046881273483) bank67860 +67861 POINT(39.85006916104836 73.72822101097987) bank67861 +67862 POINT(41.19072139605586 74.9297871681278) bank67862 +67863 POINT(41.07698541609196 73.37767142757492) bank67863 +67864 POINT(39.91923411354891 73.49450056875234) bank67864 +67865 POINT(40.34328743174821 74.7458736717097) bank67865 +67866 POINT(41.410743082925734 73.8533733626314) bank67866 +67867 POINT(40.333305799992004 74.63924313850173) bank67867 +67868 POINT(41.440175278988136 74.61435134101701) bank67868 +67869 POINT(40.63848804057183 74.5657275208936) bank67869 +67870 POINT(40.122405921320436 74.43292359255962) bank67870 +67871 POINT(40.89089422459243 74.36982788620548) bank67871 +67872 POINT(40.556540903691875 73.88310196147238) bank67872 +67873 POINT(41.33913981163698 73.83836266211459) bank67873 +67874 POINT(41.01975181961361 74.00296495352245) bank67874 +67875 POINT(40.38393230672777 73.55483189658398) bank67875 +67876 POINT(39.851347534855115 73.12717137396437) bank67876 +67877 POINT(41.416379773752176 73.42553571817048) bank67877 +67878 POINT(41.14042710124468 73.05941457166442) bank67878 +67879 POINT(39.859111437468975 73.13298534578854) bank67879 +67880 POINT(40.65040804049768 73.54341014172739) bank67880 +67881 POINT(41.55940858221948 73.35851852461661) bank67881 +67882 POINT(39.95913783476173 74.82106843257546) bank67882 +67883 POINT(40.06865233921432 74.12644289321501) bank67883 +67884 POINT(40.7374354298177 74.63328343014032) bank67884 +67885 POINT(40.8383746501926 73.98854840865496) bank67885 +67886 POINT(40.359002155289645 73.0474043854407) bank67886 +67887 POINT(40.33240244908557 74.11101008108014) bank67887 +67888 POINT(40.678091230501 74.14659528293247) bank67888 +67889 POINT(41.539616030235656 74.37046422483989) bank67889 +67890 POINT(41.13965821524085 74.03841513896893) bank67890 +67891 POINT(41.31015066820462 74.1894049442239) bank67891 +67892 POINT(40.87826591598177 73.97730186361433) bank67892 +67893 POINT(41.492708591611674 73.56063626745397) bank67893 +67894 POINT(40.33312210306126 74.51437757011837) bank67894 +67895 POINT(40.90601444716445 74.69813174948015) bank67895 +67896 POINT(40.04538523304652 74.40886963842067) bank67896 +67897 POINT(41.246559847619444 74.95607151904791) bank67897 +67898 POINT(40.24063575816022 73.77785761804601) bank67898 +67899 POINT(40.24796647951637 74.89137475708479) bank67899 +67900 POINT(40.43509201750336 74.232542965823) bank67900 +67901 POINT(40.98914357759502 74.05519955074996) bank67901 +67902 POINT(41.039078567272334 74.25088350432254) bank67902 +67903 POINT(41.129219066427076 74.34752911118774) bank67903 +67904 POINT(40.883752906739474 74.6576444309364) bank67904 +67905 POINT(41.206569817972984 74.82002856036578) bank67905 +67906 POINT(41.58233107023351 74.7047756462487) bank67906 +67907 POINT(40.54591718234707 73.71325444825602) bank67907 +67908 POINT(40.69171633137826 74.89950232974698) bank67908 +67909 POINT(41.544449158094615 74.9931213588263) bank67909 +67910 POINT(41.21299755364543 73.41162784008141) bank67910 +67911 POINT(41.05216219908021 74.62506499747502) bank67911 +67912 POINT(40.36346154066 73.9491662560325) bank67912 +67913 POINT(39.93834192733572 73.82574886165847) bank67913 +67914 POINT(39.898765182556296 73.61238827180065) bank67914 +67915 POINT(41.31992041054102 74.35623546262484) bank67915 +67916 POINT(41.04441713567764 73.96714149412789) bank67916 +67917 POINT(41.25601619869974 74.56658817017913) bank67917 +67918 POINT(40.982824160677595 74.05017816572447) bank67918 +67919 POINT(40.538081947087534 74.22476965161161) bank67919 +67920 POINT(40.1145672035461 73.91619774864013) bank67920 +67921 POINT(39.81700719996046 74.47235568409663) bank67921 +67922 POINT(39.86187686326793 74.76588759691813) bank67922 +67923 POINT(40.66541619473458 74.14693105934238) bank67923 +67924 POINT(40.19663819442263 74.46939438106284) bank67924 +67925 POINT(40.52063855172619 73.35615499162269) bank67925 +67926 POINT(39.72200649743492 73.97913650295546) bank67926 +67927 POINT(40.20915082691475 73.28978717569449) bank67927 +67928 POINT(40.32456141892344 74.91308755102887) bank67928 +67929 POINT(40.60225074217664 73.64550553397684) bank67929 +67930 POINT(40.09220861532131 73.54742977649333) bank67930 +67931 POINT(39.93621156238874 74.83409203522629) bank67931 +67932 POINT(40.41253615633638 73.08875919937644) bank67932 +67933 POINT(41.056659375585305 74.6157918304277) bank67933 +67934 POINT(40.557831463235146 73.44126967664667) bank67934 +67935 POINT(40.85478988495176 74.77383064632178) bank67935 +67936 POINT(40.85093201701491 74.90838219804557) bank67936 +67937 POINT(41.60209823656769 74.71412634883427) bank67937 +67938 POINT(41.1956645624309 74.19685590084168) bank67938 +67939 POINT(40.10587430323417 73.08783183210475) bank67939 +67940 POINT(41.07149997815704 73.05589836110246) bank67940 +67941 POINT(41.09367312251585 73.50465258825132) bank67941 +67942 POINT(41.711475645161464 73.46496032482351) bank67942 +67943 POINT(40.44002894581041 73.68558914830014) bank67943 +67944 POINT(40.58339565177621 74.72552022302219) bank67944 +67945 POINT(41.316355421073986 73.43652043242045) bank67945 +67946 POINT(41.590246425484466 74.45632717183122) bank67946 +67947 POINT(40.74972608684367 73.8778248804733) bank67947 +67948 POINT(40.613482803695675 74.47501755529883) bank67948 +67949 POINT(40.513298185643784 73.30986368632361) bank67949 +67950 POINT(39.75838298823279 73.97369183825937) bank67950 +67951 POINT(41.64829516976664 73.62505123540548) bank67951 +67952 POINT(41.537145889297534 73.35992204646689) bank67952 +67953 POINT(40.32494760758905 74.37375794347574) bank67953 +67954 POINT(41.121289005154765 73.0831107968557) bank67954 +67955 POINT(40.948345419379386 74.38596765323605) bank67955 +67956 POINT(40.44537888505991 74.17484188699876) bank67956 +67957 POINT(40.96649409145506 73.75282043125884) bank67957 +67958 POINT(40.72655859621749 74.20936415827241) bank67958 +67959 POINT(41.23446678675647 74.58847105484368) bank67959 +67960 POINT(41.47578639702889 73.41814656172765) bank67960 +67961 POINT(39.72493210551693 73.45683183305037) bank67961 +67962 POINT(40.94135989468268 74.0338661591592) bank67962 +67963 POINT(40.465367220383435 74.30273860607765) bank67963 +67964 POINT(40.0654540049918 74.7434845417146) bank67964 +67965 POINT(40.199307656179215 73.8984572588185) bank67965 +67966 POINT(41.26546751122122 73.0450315234779) bank67966 +67967 POINT(40.37137779608273 73.17802146445067) bank67967 +67968 POINT(39.88385792758021 73.84943076008956) bank67968 +67969 POINT(40.05317719329911 74.60301904706992) bank67969 +67970 POINT(41.400170723842855 73.06855040034625) bank67970 +67971 POINT(41.36845298960242 74.5982996787415) bank67971 +67972 POINT(41.415176925110956 73.22721040267243) bank67972 +67973 POINT(41.141292540986186 74.01448892296989) bank67973 +67974 POINT(41.64630544899176 74.03060091243297) bank67974 +67975 POINT(41.63368629749377 73.2165619419708) bank67975 +67976 POINT(40.76302998344389 74.56455423745808) bank67976 +67977 POINT(40.44421343100662 73.58205520580256) bank67977 +67978 POINT(40.50341845904095 74.23399995043923) bank67978 +67979 POINT(40.947081692253356 74.30147425356404) bank67979 +67980 POINT(39.960809452435704 74.75676407941964) bank67980 +67981 POINT(41.47461866650162 74.01966344486091) bank67981 +67982 POINT(40.771973062293156 73.47356185588913) bank67982 +67983 POINT(40.544672022402686 73.38677021542776) bank67983 +67984 POINT(41.315256893746444 73.12692725219948) bank67984 +67985 POINT(40.94250801380024 73.31762011033825) bank67985 +67986 POINT(40.96694873650017 73.60405181473381) bank67986 +67987 POINT(40.12357553736423 74.59634309881244) bank67987 +67988 POINT(40.2839629815196 73.67450294256835) bank67988 +67989 POINT(41.45916165959801 73.01424326035105) bank67989 +67990 POINT(41.65105887939623 73.85236992723763) bank67990 +67991 POINT(40.150291764555746 73.24457776880675) bank67991 +67992 POINT(39.801732140999896 74.93651882352628) bank67992 +67993 POINT(40.78726234983172 74.20169518247302) bank67993 +67994 POINT(40.76031849147777 74.79169993846257) bank67994 +67995 POINT(40.72595234596541 74.93751593096066) bank67995 +67996 POINT(40.74279456104838 74.57612190966076) bank67996 +67997 POINT(40.80784953516623 74.78766242657689) bank67997 +67998 POINT(40.26253408071076 73.79472448258542) bank67998 +67999 POINT(39.88005658043236 74.80911265155872) bank67999 +68000 POINT(41.23130771979608 73.02770595285526) bank68000 +68001 POINT(40.23675497383808 74.96275308720827) bank68001 +68002 POINT(41.08310422548273 74.26820523504261) bank68002 +68003 POINT(41.622898112354136 74.69173607448893) bank68003 +68004 POINT(40.96812526082519 74.86007007627583) bank68004 +68005 POINT(39.83061424694928 73.59603236189808) bank68005 +68006 POINT(40.49421765436988 74.50221075049593) bank68006 +68007 POINT(41.53077475665893 73.40587831909819) bank68007 +68008 POINT(40.381930351641365 74.4596526256542) bank68008 +68009 POINT(40.60520781113162 74.3985738544953) bank68009 +68010 POINT(40.924815574029495 73.54731246535329) bank68010 +68011 POINT(40.49267760537164 73.34251241636105) bank68011 +68012 POINT(41.50038767039765 74.5527690895752) bank68012 +68013 POINT(41.08447663559695 74.16799173474271) bank68013 +68014 POINT(40.17390543127931 74.27040101650172) bank68014 +68015 POINT(40.12180820538634 73.86965620299682) bank68015 +68016 POINT(41.54294556422676 73.4156565474608) bank68016 +68017 POINT(40.44154866051794 73.19637745661008) bank68017 +68018 POINT(40.48509586973655 73.01532213790634) bank68018 +68019 POINT(41.37003360042565 74.65076749187364) bank68019 +68020 POINT(40.761933088176875 74.6546788587935) bank68020 +68021 POINT(40.58897275132237 73.13803835599842) bank68021 +68022 POINT(40.9116712124044 73.30673564111846) bank68022 +68023 POINT(40.31144548746124 73.17660276772156) bank68023 +68024 POINT(41.59923969224694 74.90941353159153) bank68024 +68025 POINT(40.92091673392939 74.82048505318181) bank68025 +68026 POINT(41.67221193570053 73.29951367888366) bank68026 +68027 POINT(39.90172982532078 73.43975316737748) bank68027 +68028 POINT(40.776522106489935 74.39206132225138) bank68028 +68029 POINT(40.44026135771787 73.82399578626338) bank68029 +68030 POINT(41.48771422324213 73.9259233414727) bank68030 +68031 POINT(41.397814884047655 73.05396221775207) bank68031 +68032 POINT(40.09290926577425 74.42167749206146) bank68032 +68033 POINT(39.96857719578463 74.18558614005926) bank68033 +68034 POINT(40.760401416562146 74.8087821594657) bank68034 +68035 POINT(41.29531614446714 74.57208399982179) bank68035 +68036 POINT(41.08678853876268 74.55585769198149) bank68036 +68037 POINT(41.66323588385777 74.14351462482591) bank68037 +68038 POINT(40.952767929907395 73.29646453932908) bank68038 +68039 POINT(40.899073704133066 74.15603253973454) bank68039 +68040 POINT(41.6410385997275 74.8977270457905) bank68040 +68041 POINT(39.73523290375094 73.3463874094535) bank68041 +68042 POINT(40.67995240603237 74.10614492449858) bank68042 +68043 POINT(40.30656723955302 74.64794186280616) bank68043 +68044 POINT(41.697341248799134 74.97742695424763) bank68044 +68045 POINT(40.78889074728178 74.32467823107403) bank68045 +68046 POINT(40.66174198479006 73.2673309834961) bank68046 +68047 POINT(41.403527407891836 73.88122222964026) bank68047 +68048 POINT(41.15136164110098 74.58410588084762) bank68048 +68049 POINT(40.28227534707765 74.84170275931875) bank68049 +68050 POINT(40.87738775508501 74.87839394493207) bank68050 +68051 POINT(40.025884074401006 74.19097235213674) bank68051 +68052 POINT(40.73742705824969 73.99791186843096) bank68052 +68053 POINT(41.607143802530395 73.08496494341958) bank68053 +68054 POINT(40.01689679192225 74.69552051050577) bank68054 +68055 POINT(40.024031006929285 74.4257159424311) bank68055 +68056 POINT(41.055385209621434 74.92724226873746) bank68056 +68057 POINT(41.66064971189088 74.2384815018718) bank68057 +68058 POINT(41.628389403963425 74.4248106539846) bank68058 +68059 POINT(41.3020036306807 73.91264957454268) bank68059 +68060 POINT(40.66595092719947 74.00196440502054) bank68060 +68061 POINT(40.344906153283766 74.47115809213176) bank68061 +68062 POINT(41.01124952057634 73.47150278997525) bank68062 +68063 POINT(40.39670036922698 74.13098417443295) bank68063 +68064 POINT(40.192642052722505 74.36358565250369) bank68064 +68065 POINT(40.15349309463994 73.84576651874964) bank68065 +68066 POINT(40.194701160994086 73.89555355883226) bank68066 +68067 POINT(40.66139311194146 73.37224523553941) bank68067 +68068 POINT(39.9972289741711 74.2102952480348) bank68068 +68069 POINT(39.950275422836974 74.7161706806809) bank68069 +68070 POINT(39.96009734071953 73.47898210288899) bank68070 +68071 POINT(40.72080161727812 73.47938527368524) bank68071 +68072 POINT(40.50069850555915 74.60452647170011) bank68072 +68073 POINT(41.015073553880605 73.00709925907363) bank68073 +68074 POINT(40.548018327070956 74.53844491150033) bank68074 +68075 POINT(41.03835325995633 73.07192431597683) bank68075 +68076 POINT(41.11509527118794 74.04468690737326) bank68076 +68077 POINT(40.76166481924621 73.02136317307826) bank68077 +68078 POINT(40.02810582829986 73.1752244274446) bank68078 +68079 POINT(41.11936526919789 74.81864342124472) bank68079 +68080 POINT(41.1956166545113 74.45618509569329) bank68080 +68081 POINT(40.81195035706164 74.0222351478316) bank68081 +68082 POINT(41.666561237368484 73.38259083057119) bank68082 +68083 POINT(41.26108148479022 74.9157652103866) bank68083 +68084 POINT(40.568602320360355 74.52510290710141) bank68084 +68085 POINT(40.54192244567856 74.62977169130613) bank68085 +68086 POINT(40.32368181348928 74.19765653131778) bank68086 +68087 POINT(40.44246221116552 74.97114331370958) bank68087 +68088 POINT(40.11917033382804 73.79175415954093) bank68088 +68089 POINT(41.06488640166376 73.18643475797353) bank68089 +68090 POINT(39.945009406669364 74.4961191635791) bank68090 +68091 POINT(40.61993871627335 74.27692324950112) bank68091 +68092 POINT(40.90766193956865 73.06589615673653) bank68092 +68093 POINT(39.79472592669632 73.45126170383679) bank68093 +68094 POINT(41.377724982327116 73.4887015199934) bank68094 +68095 POINT(40.85871701859564 73.04132986946786) bank68095 +68096 POINT(41.38932428975862 73.93293519247041) bank68096 +68097 POINT(41.313094868753936 73.44702856978252) bank68097 +68098 POINT(39.990201385580505 74.96046595136858) bank68098 +68099 POINT(41.30567757506776 74.67250746888003) bank68099 +68100 POINT(41.64006786431502 73.7690007376038) bank68100 +68101 POINT(41.33904609225485 74.69647384495565) bank68101 +68102 POINT(39.97659829043288 73.96013912479316) bank68102 +68103 POINT(41.310284638070364 74.71910063155508) bank68103 +68104 POINT(40.58972179300513 74.07585891673268) bank68104 +68105 POINT(40.32123429583044 73.1973847999698) bank68105 +68106 POINT(39.95051206060507 73.74054491751833) bank68106 +68107 POINT(39.97201743448173 73.06886469111757) bank68107 +68108 POINT(41.370827965771156 74.82710317578577) bank68108 +68109 POINT(40.57469835074074 73.41417266745312) bank68109 +68110 POINT(40.97827251233365 73.81255409393403) bank68110 +68111 POINT(40.69877687365604 74.77144945839166) bank68111 +68112 POINT(39.759958962671064 74.04911911874986) bank68112 +68113 POINT(41.04840003762606 74.20793739512479) bank68113 +68114 POINT(41.138350306440756 74.19389929871951) bank68114 +68115 POINT(40.924678247222545 73.55036378209871) bank68115 +68116 POINT(40.65589638341718 74.42125568248298) bank68116 +68117 POINT(40.42485540053709 73.60286784250071) bank68117 +68118 POINT(41.57738255140907 73.51554440741388) bank68118 +68119 POINT(40.496383143560834 73.70800069957119) bank68119 +68120 POINT(41.68897272644994 74.34349893465297) bank68120 +68121 POINT(39.92454905092412 73.58445061768542) bank68121 +68122 POINT(40.613127384310815 73.55158838401354) bank68122 +68123 POINT(40.28537100131917 74.58755325452029) bank68123 +68124 POINT(40.23345330992812 74.68263926885615) bank68124 +68125 POINT(39.91301553126243 73.36876967887436) bank68125 +68126 POINT(41.16327136305419 74.19434037955135) bank68126 +68127 POINT(40.505103219275014 74.55182479280461) bank68127 +68128 POINT(40.203098362950335 73.53860179949746) bank68128 +68129 POINT(40.816376801543434 74.01007864998724) bank68129 +68130 POINT(39.76033097157291 74.83134138711138) bank68130 +68131 POINT(40.6267829136794 73.6306253465466) bank68131 +68132 POINT(40.06673868127613 73.8366670301178) bank68132 +68133 POINT(41.50057557700048 73.20243765635168) bank68133 +68134 POINT(39.88995519401584 73.73655824813544) bank68134 +68135 POINT(39.895587996250285 73.51198420709237) bank68135 +68136 POINT(39.90444269935086 74.06760301515828) bank68136 +68137 POINT(40.821790161110776 74.33846900983059) bank68137 +68138 POINT(40.70943654024129 74.15213955365401) bank68138 +68139 POINT(40.01947065702086 73.84488645480485) bank68139 +68140 POINT(41.14231606140024 73.17076921532835) bank68140 +68141 POINT(41.61413358267916 73.14727287747455) bank68141 +68142 POINT(40.57083146912877 73.10126338026764) bank68142 +68143 POINT(40.06690706175074 73.71936358516544) bank68143 +68144 POINT(39.956276303579834 74.50753492528409) bank68144 +68145 POINT(40.43186241380229 73.33071852426689) bank68145 +68146 POINT(41.65838867756848 73.28590316929632) bank68146 +68147 POINT(41.13898617220231 73.55625595662033) bank68147 +68148 POINT(41.23309493323946 74.56939865686353) bank68148 +68149 POINT(40.695789773743556 73.63328522400522) bank68149 +68150 POINT(41.25571104266172 74.33877047367334) bank68150 +68151 POINT(40.617361126133765 74.1348041671342) bank68151 +68152 POINT(40.85593082327564 74.02793226056446) bank68152 +68153 POINT(40.86590602425443 74.25535689515375) bank68153 +68154 POINT(39.911131170353265 73.34564463129458) bank68154 +68155 POINT(41.699224272501375 74.63371419019107) bank68155 +68156 POINT(40.780197662656626 74.16204538913084) bank68156 +68157 POINT(41.45182790457227 74.92271520041325) bank68157 +68158 POINT(41.459585473610936 73.30072868793148) bank68158 +68159 POINT(40.919033760733534 73.90481173184376) bank68159 +68160 POINT(40.48309822445005 73.11006323180574) bank68160 +68161 POINT(39.942169677225856 73.0329947378802) bank68161 +68162 POINT(41.44125913891718 73.4033303541473) bank68162 +68163 POINT(40.00264841862534 73.92338267063748) bank68163 +68164 POINT(41.14911662270867 74.47570728350195) bank68164 +68165 POINT(40.147355094329434 74.44303282524727) bank68165 +68166 POINT(40.85118365132612 73.57410433918855) bank68166 +68167 POINT(40.77783311293489 73.85304616630437) bank68167 +68168 POINT(40.81570327703669 74.99497250244367) bank68168 +68169 POINT(40.59121483510057 73.88366490518945) bank68169 +68170 POINT(40.25050797979315 73.42813009610225) bank68170 +68171 POINT(40.22684270685224 73.84249636283971) bank68171 +68172 POINT(41.47910534033574 74.65048037407487) bank68172 +68173 POINT(41.422486513886625 73.20976552612501) bank68173 +68174 POINT(41.5311749114721 73.19329842837674) bank68174 +68175 POINT(40.643723972951264 73.82749348520032) bank68175 +68176 POINT(39.96107745422654 73.47567254361418) bank68176 +68177 POINT(41.641766768289386 74.18547512625776) bank68177 +68178 POINT(40.63382070404169 73.19107390649037) bank68178 +68179 POINT(40.714914216509236 73.93169217931855) bank68179 +68180 POINT(40.22137482679736 74.45598578925521) bank68180 +68181 POINT(40.922873198949915 73.98270792386651) bank68181 +68182 POINT(40.35440546806335 73.31821632125751) bank68182 +68183 POINT(41.528120594422695 75.00454697789405) bank68183 +68184 POINT(39.87321856685322 74.83893284265238) bank68184 +68185 POINT(41.13169932060532 74.5279643600777) bank68185 +68186 POINT(39.89237677037086 74.3634912616321) bank68186 +68187 POINT(40.72972785045938 74.04309204475246) bank68187 +68188 POINT(40.59953361917379 74.17062087982802) bank68188 +68189 POINT(40.597342897896816 73.12220711124674) bank68189 +68190 POINT(39.75168866486781 73.85620031070567) bank68190 +68191 POINT(41.35708178038189 73.06834324983986) bank68191 +68192 POINT(39.81531748681698 73.94741690434404) bank68192 +68193 POINT(40.86641578423076 74.24948484273325) bank68193 +68194 POINT(40.828299133839565 73.13547495057433) bank68194 +68195 POINT(40.42766359657626 74.96362166761045) bank68195 +68196 POINT(39.842416928836776 73.05599711844954) bank68196 +68197 POINT(41.28965792664188 73.79380650347356) bank68197 +68198 POINT(40.418286781631124 74.60954070034012) bank68198 +68199 POINT(40.24320599602896 74.0215787633033) bank68199 +68200 POINT(41.32850532742946 73.96779294048783) bank68200 +68201 POINT(40.62013446254519 74.92686804044686) bank68201 +68202 POINT(41.31857378159375 73.30687073612033) bank68202 +68203 POINT(41.37074861628423 73.8898090892256) bank68203 +68204 POINT(40.40714239207062 73.17722698388131) bank68204 +68205 POINT(41.30538589848022 74.48712595227538) bank68205 +68206 POINT(40.711711376270415 73.49220902639362) bank68206 +68207 POINT(41.379861868275164 73.39116049567147) bank68207 +68208 POINT(40.49840504440932 74.95109628077745) bank68208 +68209 POINT(41.21430151796041 73.15045952278079) bank68209 +68210 POINT(41.06837255433133 74.51895900890713) bank68210 +68211 POINT(40.18179807956895 73.20890307473567) bank68211 +68212 POINT(41.11475154765227 73.77582989685615) bank68212 +68213 POINT(41.18658121255265 73.65055499739775) bank68213 +68214 POINT(41.63905325553953 75.00035709340335) bank68214 +68215 POINT(40.59195378829372 74.16829005603164) bank68215 +68216 POINT(41.03311359426284 74.56244460227282) bank68216 +68217 POINT(40.65907646650932 73.29999921059952) bank68217 +68218 POINT(40.157830904808094 74.92603494038448) bank68218 +68219 POINT(39.8218334231617 73.40770479522699) bank68219 +68220 POINT(40.026087843084625 73.67488480222724) bank68220 +68221 POINT(39.8477568866285 73.87037069827316) bank68221 +68222 POINT(41.562525718823295 74.95362721893073) bank68222 +68223 POINT(40.80099098117652 74.3596270890456) bank68223 +68224 POINT(40.72176970271237 74.38811772770265) bank68224 +68225 POINT(41.630960612125484 74.19832413283521) bank68225 +68226 POINT(40.6485703824903 73.58484230137952) bank68226 +68227 POINT(40.289981064024964 73.74595276308644) bank68227 +68228 POINT(40.79475451871777 73.83007998135102) bank68228 +68229 POINT(41.34887243792947 74.3036100159318) bank68229 +68230 POINT(39.88009964801234 73.68714043216981) bank68230 +68231 POINT(39.77418150505169 73.24414746878608) bank68231 +68232 POINT(41.16354331448577 74.55999315831598) bank68232 +68233 POINT(40.52927007864745 73.40094192679315) bank68233 +68234 POINT(40.45734491851324 74.71224953921796) bank68234 +68235 POINT(41.02839925514792 74.9537708153961) bank68235 +68236 POINT(39.92540061354404 73.19077055632489) bank68236 +68237 POINT(41.64121729200816 74.8721152341052) bank68237 +68238 POINT(40.079601862470106 73.3552404522591) bank68238 +68239 POINT(41.135960513976876 74.42873714868674) bank68239 +68240 POINT(40.33958515441129 73.3414147058024) bank68240 +68241 POINT(40.47615074001646 73.54624070911517) bank68241 +68242 POINT(41.24057173384041 73.80545736462534) bank68242 +68243 POINT(40.70432759552194 74.1485746856991) bank68243 +68244 POINT(41.16832493445176 74.02301887561246) bank68244 +68245 POINT(39.99320546847846 73.090494168033) bank68245 +68246 POINT(39.80473348644497 73.1589084073525) bank68246 +68247 POINT(40.21718837304697 74.44177338029021) bank68247 +68248 POINT(41.54552695320047 74.0087791970156) bank68248 +68249 POINT(39.71605769782713 73.60717621830098) bank68249 +68250 POINT(41.06589270459266 73.73459305376913) bank68250 +68251 POINT(39.994872387035485 73.37433589548348) bank68251 +68252 POINT(39.95307529926376 74.77609181110434) bank68252 +68253 POINT(40.73370245200297 74.19082231067043) bank68253 +68254 POINT(41.667651523531035 73.8401809785433) bank68254 +68255 POINT(41.43386154912396 73.41698199262449) bank68255 +68256 POINT(41.27143788387493 73.75838418720545) bank68256 +68257 POINT(40.440231023052455 74.40464640690423) bank68257 +68258 POINT(41.70278304396422 73.87825421871682) bank68258 +68259 POINT(41.41443402137632 73.52990446622042) bank68259 +68260 POINT(41.11851085020299 73.94560156937652) bank68260 +68261 POINT(40.929868331464505 73.49450315648261) bank68261 +68262 POINT(39.822308060710895 74.36858435912917) bank68262 +68263 POINT(41.51268892059317 73.02953458575229) bank68263 +68264 POINT(41.28051719754646 73.22856724431304) bank68264 +68265 POINT(40.67342254656068 74.23323664874015) bank68265 +68266 POINT(40.322692780661924 74.65715293254445) bank68266 +68267 POINT(40.122289918399694 74.36448578578768) bank68267 +68268 POINT(40.23834348411355 73.5314458660406) bank68268 +68269 POINT(39.916203911995304 74.8839000753333) bank68269 +68270 POINT(40.60501197191437 73.87496352418891) bank68270 +68271 POINT(40.098149135364004 73.26927289022312) bank68271 +68272 POINT(40.73952136139011 74.62349023412703) bank68272 +68273 POINT(40.40434115520655 74.68149306556505) bank68273 +68274 POINT(40.8623820704878 73.24266080316916) bank68274 +68275 POINT(40.80745049002306 73.44126225472121) bank68275 +68276 POINT(41.03833702156831 74.36624764511744) bank68276 +68277 POINT(40.261574939003786 74.4322718741236) bank68277 +68278 POINT(39.74131369257527 74.5755494653249) bank68278 +68279 POINT(40.354955141122865 74.24084972868329) bank68279 +68280 POINT(40.79094759435579 73.8421419287937) bank68280 +68281 POINT(41.101232895608035 73.3521368250543) bank68281 +68282 POINT(40.959575681735394 73.85044959931474) bank68282 +68283 POINT(40.5333267620533 73.3044179989481) bank68283 +68284 POINT(40.37715831053446 74.05633608322455) bank68284 +68285 POINT(39.77685087054569 74.2282402346911) bank68285 +68286 POINT(39.876622996368944 74.23382814414158) bank68286 +68287 POINT(40.188228205833745 73.87842189272021) bank68287 +68288 POINT(41.395238138696655 73.78942316622827) bank68288 +68289 POINT(40.91500185595497 73.7660233593538) bank68289 +68290 POINT(41.07566293791017 74.54037299457947) bank68290 +68291 POINT(41.531940887971764 74.82424775740708) bank68291 +68292 POINT(41.37727881861038 73.18663109739914) bank68292 +68293 POINT(39.96461567590755 74.81615488594397) bank68293 +68294 POINT(41.333938176185264 73.8564686362405) bank68294 +68295 POINT(39.76756705249244 73.4667361089008) bank68295 +68296 POINT(41.118628508360715 74.64685163621483) bank68296 +68297 POINT(40.49692636168574 73.69464883883036) bank68297 +68298 POINT(40.32018400370302 73.36031273082138) bank68298 +68299 POINT(41.437018732927825 73.57762349392061) bank68299 +68300 POINT(39.97236901637294 74.06110247988836) bank68300 +68301 POINT(41.086996614265495 74.46924089853674) bank68301 +68302 POINT(39.7432658751181 74.56112055990712) bank68302 +68303 POINT(40.17175199936446 73.41567906639719) bank68303 +68304 POINT(41.38741876273542 73.49545076512965) bank68304 +68305 POINT(41.33684827917436 74.11903422857499) bank68305 +68306 POINT(40.49831239999746 74.42512140680633) bank68306 +68307 POINT(39.877330570687384 73.2986706197391) bank68307 +68308 POINT(39.735164542457966 73.32642173533608) bank68308 +68309 POINT(40.507707319749684 73.96445026891799) bank68309 +68310 POINT(40.78140473174844 74.73600525593454) bank68310 +68311 POINT(39.81601361627077 74.06885097260789) bank68311 +68312 POINT(41.54094365462085 74.46841814095845) bank68312 +68313 POINT(40.211812962379945 74.70127184400918) bank68313 +68314 POINT(39.96100446359466 74.82072599777749) bank68314 +68315 POINT(41.123605720081585 73.40545509211137) bank68315 +68316 POINT(41.05975514784259 73.50686836774136) bank68316 +68317 POINT(39.95157539443031 73.59243812141177) bank68317 +68318 POINT(41.03257852027005 74.77058861697962) bank68318 +68319 POINT(41.15142973074114 73.64301813974748) bank68319 +68320 POINT(40.17278681070777 73.65614410912062) bank68320 +68321 POINT(41.651609569171235 73.86528789071156) bank68321 +68322 POINT(39.997230122338784 73.5800309508062) bank68322 +68323 POINT(40.05345304603024 73.42914721487239) bank68323 +68324 POINT(41.56711168650782 73.96062638156587) bank68324 +68325 POINT(39.82568158753146 73.1792370273121) bank68325 +68326 POINT(41.49945952470442 73.82003800249292) bank68326 +68327 POINT(41.29138968408888 73.38936545731785) bank68327 +68328 POINT(40.462687280904305 74.05975190306738) bank68328 +68329 POINT(40.998305024121905 74.95910822152823) bank68329 +68330 POINT(41.024819710234866 73.18415724208263) bank68330 +68331 POINT(41.418787925643 74.23042914040406) bank68331 +68332 POINT(40.09864737650534 73.3647574047636) bank68332 +68333 POINT(41.46808762142116 74.77210097000334) bank68333 +68334 POINT(41.08748746221815 74.4531206195454) bank68334 +68335 POINT(41.15927374834088 73.79225491622289) bank68335 +68336 POINT(40.50025765994633 74.4464513286047) bank68336 +68337 POINT(41.16575366890993 73.72869503972619) bank68337 +68338 POINT(40.49486503273165 74.19985376650429) bank68338 +68339 POINT(40.40008274799881 73.65747458838987) bank68339 +68340 POINT(40.69957106300417 73.69204296128981) bank68340 +68341 POINT(39.792010574438706 74.46336537774705) bank68341 +68342 POINT(39.93883368252777 73.10068233801752) bank68342 +68343 POINT(39.87933490831192 73.8489214735024) bank68343 +68344 POINT(40.47417949015038 73.13535056753392) bank68344 +68345 POINT(39.87033645325362 74.62584449289933) bank68345 +68346 POINT(41.51018224532917 74.14638207687663) bank68346 +68347 POINT(40.73331659091401 73.43190343267236) bank68347 +68348 POINT(39.94274677100537 73.07130342372623) bank68348 +68349 POINT(40.88189976618786 73.79145908105815) bank68349 +68350 POINT(40.79705410373895 73.89659120760363) bank68350 +68351 POINT(40.99708057200072 74.55755543374269) bank68351 +68352 POINT(41.29614512960273 73.82699813991424) bank68352 +68353 POINT(40.167414325993605 74.36788408080872) bank68353 +68354 POINT(41.50255905368189 74.01367315599788) bank68354 +68355 POINT(40.300303446148945 73.4024201940428) bank68355 +68356 POINT(40.87106599541322 73.95919332734114) bank68356 +68357 POINT(41.070938294420266 74.46075792379378) bank68357 +68358 POINT(39.79425261079803 74.81681616235325) bank68358 +68359 POINT(41.39697524884328 74.43984201916861) bank68359 +68360 POINT(40.51056338226756 74.62215934930208) bank68360 +68361 POINT(41.35912171235295 73.98124745813716) bank68361 +68362 POINT(40.30456861534872 74.6680561779278) bank68362 +68363 POINT(41.60839040019219 73.28360277047295) bank68363 +68364 POINT(41.39060648331597 73.43592551144192) bank68364 +68365 POINT(40.63454557380681 73.88116616766885) bank68365 +68366 POINT(41.289362841441765 73.54208295898106) bank68366 +68367 POINT(41.0487378567769 74.44583188127571) bank68367 +68368 POINT(40.670661026521444 74.91427797861301) bank68368 +68369 POINT(40.048104220011126 73.24837916903756) bank68369 +68370 POINT(40.67106093822671 73.38076780320479) bank68370 +68371 POINT(39.76904268492034 74.07449467015515) bank68371 +68372 POINT(40.419624841521305 73.65114851676832) bank68372 +68373 POINT(40.41043325266218 74.09194426137213) bank68373 +68374 POINT(41.690965049110176 74.8066331894466) bank68374 +68375 POINT(41.04497495626118 74.17733287569682) bank68375 +68376 POINT(41.59008070645579 73.60298034401116) bank68376 +68377 POINT(40.426762766944556 74.92014155253753) bank68377 +68378 POINT(40.83524670813031 73.62559724545355) bank68378 +68379 POINT(40.80479783165192 73.38850529167617) bank68379 +68380 POINT(41.58489030637439 73.33304250196028) bank68380 +68381 POINT(40.62846522379381 74.19089801764504) bank68381 +68382 POINT(41.14115497900686 74.75020410269288) bank68382 +68383 POINT(40.935135385979 73.57699626472642) bank68383 +68384 POINT(41.64946490550804 74.78089511998488) bank68384 +68385 POINT(41.13330282009575 73.80466509950821) bank68385 +68386 POINT(40.36804529170943 73.71572623019047) bank68386 +68387 POINT(41.62089708825459 74.91103267153801) bank68387 +68388 POINT(41.37840096748491 74.76105402809245) bank68388 +68389 POINT(40.575575599206104 74.39179775038689) bank68389 +68390 POINT(41.669728075040354 74.40059628719324) bank68390 +68391 POINT(39.81095294160358 73.82071227961157) bank68391 +68392 POINT(40.25680627798432 73.93149950975767) bank68392 +68393 POINT(40.95503221944971 73.96176102644165) bank68393 +68394 POINT(41.645808598128646 73.70564927510712) bank68394 +68395 POINT(41.5669506926841 73.64420045423992) bank68395 +68396 POINT(40.90107588640056 73.50501252784298) bank68396 +68397 POINT(40.1895724191525 74.23429501731292) bank68397 +68398 POINT(40.81854208046929 73.35327281426076) bank68398 +68399 POINT(40.484081500936036 73.36123408223335) bank68399 +68400 POINT(40.79731919752743 74.82353472816905) bank68400 +68401 POINT(40.8521937679377 73.58803218522809) bank68401 +68402 POINT(40.91337858586152 73.98826047868951) bank68402 +68403 POINT(40.35403589108817 74.61526863915591) bank68403 +68404 POINT(41.56314066980482 73.2719872466358) bank68404 +68405 POINT(40.45916497328317 74.20510875898003) bank68405 +68406 POINT(41.613619491692255 73.13764427217366) bank68406 +68407 POINT(39.889327961649805 73.03971486017632) bank68407 +68408 POINT(40.54820577758082 74.51384848986872) bank68408 +68409 POINT(41.39706161283277 74.47656660455931) bank68409 +68410 POINT(39.730194991985016 74.46053273010058) bank68410 +68411 POINT(41.04597001020434 74.47884131513324) bank68411 +68412 POINT(39.7562659554499 73.1724013450493) bank68412 +68413 POINT(40.116051166302086 74.64192152970713) bank68413 +68414 POINT(40.5044309557888 74.5290706687283) bank68414 +68415 POINT(40.42803976817476 73.94377214296807) bank68415 +68416 POINT(40.323710533784194 74.58411291649433) bank68416 +68417 POINT(40.684548860217056 73.42288916531558) bank68417 +68418 POINT(40.369825776226875 74.93693190596844) bank68418 +68419 POINT(40.538487586635924 74.87479225235596) bank68419 +68420 POINT(40.0872309813576 73.70462968285743) bank68420 +68421 POINT(40.608732744798516 74.81291377212828) bank68421 +68422 POINT(40.82595650390088 74.89205432061779) bank68422 +68423 POINT(40.1368686686327 73.12734328789045) bank68423 +68424 POINT(40.65803017244889 74.123424046872) bank68424 +68425 POINT(40.3912734702679 73.78673056323183) bank68425 +68426 POINT(41.423631576284386 73.2990106505359) bank68426 +68427 POINT(41.660806912505336 74.84831791121725) bank68427 +68428 POINT(39.74828025639151 73.19490696944901) bank68428 +68429 POINT(40.273533329202536 73.37585317861209) bank68429 +68430 POINT(40.26841872309838 73.9467726015035) bank68430 +68431 POINT(40.849825348922224 73.81215219086897) bank68431 +68432 POINT(41.19884721121503 74.44868719942482) bank68432 +68433 POINT(39.80337418848903 73.57140300797501) bank68433 +68434 POINT(40.290065743692 73.76736067547333) bank68434 +68435 POINT(40.69113160991251 74.4271271379896) bank68435 +68436 POINT(40.08657276338559 74.37782169538345) bank68436 +68437 POINT(40.96184678433016 73.39048331629431) bank68437 +68438 POINT(41.02897662250651 74.54299595255931) bank68438 +68439 POINT(41.692900083408986 73.02479074032094) bank68439 +68440 POINT(39.85176819294155 73.51048072939852) bank68440 +68441 POINT(41.551090903937514 73.89975480102096) bank68441 +68442 POINT(40.809537961477325 73.07064652283015) bank68442 +68443 POINT(40.86525220853045 73.68887247861079) bank68443 +68444 POINT(41.591504439839454 73.88250635666282) bank68444 +68445 POINT(41.091155061014774 73.6085304439161) bank68445 +68446 POINT(39.98096253794822 73.6545984225619) bank68446 +68447 POINT(41.46784557923538 74.276740576073) bank68447 +68448 POINT(40.09216252223177 74.36163690492008) bank68448 +68449 POINT(41.355054914737266 74.66474315440796) bank68449 +68450 POINT(40.51048360464001 74.86383596859082) bank68450 +68451 POINT(41.07953238817786 73.10043515246953) bank68451 +68452 POINT(40.850822409306666 74.98513794791447) bank68452 +68453 POINT(41.348571162933204 74.69404988734568) bank68453 +68454 POINT(40.52136839380579 73.12965062360854) bank68454 +68455 POINT(41.6623004793056 74.107846364554) bank68455 +68456 POINT(40.913800422540206 74.49234313189925) bank68456 +68457 POINT(41.03464592825988 73.90955527395818) bank68457 +68458 POINT(40.26608262510233 74.9810525763198) bank68458 +68459 POINT(40.9448324609785 74.3254609809524) bank68459 +68460 POINT(41.449513239988796 74.79589078918768) bank68460 +68461 POINT(41.18348911673619 73.84487294621951) bank68461 +68462 POINT(41.03243848929679 74.54670084222388) bank68462 +68463 POINT(41.68908007933862 73.61055162700949) bank68463 +68464 POINT(40.75923962421091 73.3770176839903) bank68464 +68465 POINT(41.4741035700071 73.87721776428653) bank68465 +68466 POINT(40.77292312029561 73.69880454139896) bank68466 +68467 POINT(40.699704952124826 74.74164248215828) bank68467 +68468 POINT(41.08318861377611 73.20575316494761) bank68468 +68469 POINT(40.00896713446484 73.42236673896275) bank68469 +68470 POINT(41.38398532577142 74.25594019381147) bank68470 +68471 POINT(41.19915363684173 73.16551861984327) bank68471 +68472 POINT(41.106636279127706 74.49159397618064) bank68472 +68473 POINT(41.24422839094923 73.2839485858086) bank68473 +68474 POINT(41.570570175856496 74.6279353468307) bank68474 +68475 POINT(39.845585535054475 73.0952812828558) bank68475 +68476 POINT(40.30740848191059 73.7946324995926) bank68476 +68477 POINT(41.30673237614854 74.92132870004909) bank68477 +68478 POINT(41.627237031122164 73.91081647278773) bank68478 +68479 POINT(40.14695052911599 73.75783148736073) bank68479 +68480 POINT(40.040855408261514 74.0690762863448) bank68480 +68481 POINT(40.982503485842365 74.74814043701367) bank68481 +68482 POINT(39.9802977148323 73.83816501662018) bank68482 +68483 POINT(41.0506563694808 74.05983730470817) bank68483 +68484 POINT(40.61827654918359 74.46248356038004) bank68484 +68485 POINT(40.74498208284214 74.68543539073528) bank68485 +68486 POINT(39.75837691882193 74.40076334591949) bank68486 +68487 POINT(41.6386225765895 73.13910398556781) bank68487 +68488 POINT(41.28385913430378 74.10033714053233) bank68488 +68489 POINT(41.111725330608614 73.029988443149) bank68489 +68490 POINT(41.28736489760903 73.44959377803319) bank68490 +68491 POINT(41.362088169949665 75.00151089224349) bank68491 +68492 POINT(40.069198671072286 73.12917978042594) bank68492 +68493 POINT(40.10975006830259 73.29765143330349) bank68493 +68494 POINT(40.55980642923595 73.93215881895107) bank68494 +68495 POINT(41.45815776326942 74.55563473493929) bank68495 +68496 POINT(40.72485850050094 73.57475837654599) bank68496 +68497 POINT(41.684355117821596 73.71667284243996) bank68497 +68498 POINT(40.42128501522326 73.0351850308474) bank68498 +68499 POINT(41.290414860528536 73.60753513593922) bank68499 +68500 POINT(40.566819216819425 74.06176832432722) bank68500 +68501 POINT(40.42218110362665 74.69973091450909) bank68501 +68502 POINT(41.19730717604336 74.98952638827228) bank68502 +68503 POINT(40.52778790730621 73.50716511947863) bank68503 +68504 POINT(41.31359837238927 74.82315729723041) bank68504 +68505 POINT(40.82736859350278 74.76191962986707) bank68505 +68506 POINT(39.9064635595896 73.71607490805216) bank68506 +68507 POINT(40.52858649102614 73.70124207543599) bank68507 +68508 POINT(40.31282396463663 74.14024251772298) bank68508 +68509 POINT(41.57502268249423 73.62087645728774) bank68509 +68510 POINT(41.226875316633134 73.26437601048161) bank68510 +68511 POINT(41.24337444592576 74.09982419449071) bank68511 +68512 POINT(41.42691450270744 74.4262688612106) bank68512 +68513 POINT(39.82654095441238 73.72471435756574) bank68513 +68514 POINT(40.54350906383985 73.58245334082227) bank68514 +68515 POINT(40.94973904340821 74.3306573558192) bank68515 +68516 POINT(41.42350050560051 74.84238237335262) bank68516 +68517 POINT(41.17515282892211 73.68036273909088) bank68517 +68518 POINT(39.955164522291206 74.49056154209552) bank68518 +68519 POINT(41.38549206724774 73.33252366813295) bank68519 +68520 POINT(41.456716710056675 73.80657235623595) bank68520 +68521 POINT(40.10804751851275 74.68009427174464) bank68521 +68522 POINT(41.335728392326715 73.20307497357038) bank68522 +68523 POINT(41.05416613603945 73.36907189686646) bank68523 +68524 POINT(39.916763965000804 74.95115928659739) bank68524 +68525 POINT(39.792230166181824 74.46885156231974) bank68525 +68526 POINT(40.73200478319769 74.68025826512199) bank68526 +68527 POINT(40.6009293352395 73.71453971366671) bank68527 +68528 POINT(41.68083481336111 73.19087682536409) bank68528 +68529 POINT(39.974643580053645 73.61087230501245) bank68529 +68530 POINT(40.85420264631045 74.8332575701038) bank68530 +68531 POINT(40.51472118598942 73.74579317366612) bank68531 +68532 POINT(41.24506641240488 73.29230448279408) bank68532 +68533 POINT(41.17681647297932 73.62214826705383) bank68533 +68534 POINT(41.583471143043354 74.81753535505982) bank68534 +68535 POINT(40.049147199856336 74.39585488635022) bank68535 +68536 POINT(40.742828141989534 74.5488033198981) bank68536 +68537 POINT(41.01925977918033 73.18133236786085) bank68537 +68538 POINT(39.788989218068764 73.26949363499969) bank68538 +68539 POINT(40.616557204909626 73.72056662842843) bank68539 +68540 POINT(39.81927092945614 73.09475599191123) bank68540 +68541 POINT(39.96561322374808 73.82393506285942) bank68541 +68542 POINT(40.20040679082644 74.6250080328494) bank68542 +68543 POINT(41.03966065466748 74.03044812931606) bank68543 +68544 POINT(39.76604354229274 74.59925502123345) bank68544 +68545 POINT(41.48105745140759 73.40448989514653) bank68545 +68546 POINT(41.161125454827086 74.44567250261608) bank68546 +68547 POINT(40.801832441626864 74.66485579685978) bank68547 +68548 POINT(39.75570451312704 74.09905771716366) bank68548 +68549 POINT(40.06469484772502 73.9108014310351) bank68549 +68550 POINT(39.73704750907139 74.85037920269146) bank68550 +68551 POINT(41.432077709614134 73.94283909368707) bank68551 +68552 POINT(40.58996731894674 74.408103475574) bank68552 +68553 POINT(41.62425186351321 74.05287717750791) bank68553 +68554 POINT(40.42137899139086 74.8869209961912) bank68554 +68555 POINT(40.992562816673704 74.85626167792115) bank68555 +68556 POINT(39.98879158723577 73.40045980176443) bank68556 +68557 POINT(40.89939518594673 74.54181970843699) bank68557 +68558 POINT(41.52548483060694 73.29135396333754) bank68558 +68559 POINT(40.466513318207326 73.42716741214247) bank68559 +68560 POINT(41.248152635855845 73.53753449855593) bank68560 +68561 POINT(41.185414445150336 73.89854658301626) bank68561 +68562 POINT(39.84052741097502 74.14022003765771) bank68562 +68563 POINT(40.94120496791824 74.39561396638196) bank68563 +68564 POINT(41.29863104178379 74.44536702573157) bank68564 +68565 POINT(40.52158413817219 74.74144322516956) bank68565 +68566 POINT(40.41429275801587 74.69907785417719) bank68566 +68567 POINT(41.17896720220823 74.47099808678624) bank68567 +68568 POINT(40.19851970360104 73.20934797073186) bank68568 +68569 POINT(40.26370577337718 74.25407953031932) bank68569 +68570 POINT(40.461459930220386 74.11803502830587) bank68570 +68571 POINT(41.178133861246 74.91616585341156) bank68571 +68572 POINT(40.07561592692659 74.83969677223422) bank68572 +68573 POINT(41.465746632954676 73.96741585157943) bank68573 +68574 POINT(40.47366566735005 74.41012078435622) bank68574 +68575 POINT(40.43298346103882 73.38388230653105) bank68575 +68576 POINT(40.04040742263853 74.28228560385179) bank68576 +68577 POINT(40.60704575508915 74.43726203210394) bank68577 +68578 POINT(40.17045597327515 73.86624037842729) bank68578 +68579 POINT(40.5483905373492 73.58335144304799) bank68579 +68580 POINT(40.314560010871034 73.3339077612471) bank68580 +68581 POINT(40.84172315227614 74.0239870243912) bank68581 +68582 POINT(41.19662217839616 74.42679842551074) bank68582 +68583 POINT(40.3744343225122 73.92732203036422) bank68583 +68584 POINT(39.73078431202031 74.74851645119014) bank68584 +68585 POINT(41.08747226513707 74.04028026971972) bank68585 +68586 POINT(40.82358016789654 74.78375449117367) bank68586 +68587 POINT(40.54143614949774 73.85071425581563) bank68587 +68588 POINT(41.080991383975494 74.03698822116726) bank68588 +68589 POINT(41.07405895917008 74.8355908224329) bank68589 +68590 POINT(39.79302604539271 74.64256284174672) bank68590 +68591 POINT(40.972034143179734 74.07718817579644) bank68591 +68592 POINT(40.51397017472918 73.7833587818528) bank68592 +68593 POINT(40.311175392709956 74.71271265181898) bank68593 +68594 POINT(41.44141712567106 73.93142583073768) bank68594 +68595 POINT(40.75649672856437 74.27440478041076) bank68595 +68596 POINT(39.74567239457377 74.5542355323236) bank68596 +68597 POINT(41.447989372882375 74.33484476486463) bank68597 +68598 POINT(41.42396823175296 74.30524428267245) bank68598 +68599 POINT(40.211569326883264 73.7780015169033) bank68599 +68600 POINT(41.09438786175465 74.05397979712716) bank68600 +68601 POINT(40.17031309785734 74.20656836377603) bank68601 +68602 POINT(39.905099167268965 73.21498878207335) bank68602 +68603 POINT(40.35459531763667 74.02921394709237) bank68603 +68604 POINT(41.18748993006937 73.09252423679756) bank68604 +68605 POINT(40.82507247217581 73.52100382520157) bank68605 +68606 POINT(40.6753418112522 73.66349955522196) bank68606 +68607 POINT(41.29471619442374 73.35528682625707) bank68607 +68608 POINT(41.44975791650198 73.59345941833286) bank68608 +68609 POINT(40.542058453672134 74.54045814860653) bank68609 +68610 POINT(40.34585768627436 74.27192776895211) bank68610 +68611 POINT(41.308444866743514 74.64980409889438) bank68611 +68612 POINT(40.72626849652751 74.25918129980863) bank68612 +68613 POINT(41.23313017203585 74.81084747235015) bank68613 +68614 POINT(39.74156474747407 73.2469809104287) bank68614 +68615 POINT(41.67168279089186 73.35609064456716) bank68615 +68616 POINT(40.14954752005565 73.12686486449495) bank68616 +68617 POINT(39.73437908329209 73.70478374690823) bank68617 +68618 POINT(40.200268390672974 74.96042651997708) bank68618 +68619 POINT(41.43952080263842 73.6356240113009) bank68619 +68620 POINT(40.727523616841864 73.54028785765396) bank68620 +68621 POINT(40.443319138292665 73.60017193725704) bank68621 +68622 POINT(40.34662242003916 73.11138421919756) bank68622 +68623 POINT(40.73083724473424 74.70547082467267) bank68623 +68624 POINT(39.90802953921346 74.7540908592007) bank68624 +68625 POINT(41.444401482943555 74.27060508385797) bank68625 +68626 POINT(41.02033898644584 74.74291411336172) bank68626 +68627 POINT(40.47307237888191 74.64043957459039) bank68627 +68628 POINT(41.66991693908327 74.06771598323898) bank68628 +68629 POINT(40.60710651059138 73.4215788757719) bank68629 +68630 POINT(41.56413713992547 73.30502208635619) bank68630 +68631 POINT(40.036308710206775 74.26517986808855) bank68631 +68632 POINT(41.584030247513944 74.15394823505055) bank68632 +68633 POINT(40.645686326087535 73.98554582407517) bank68633 +68634 POINT(41.48519859224055 73.9942289129507) bank68634 +68635 POINT(40.856317243235736 74.94123712325246) bank68635 +68636 POINT(40.68102722825532 73.99801772842017) bank68636 +68637 POINT(41.67362280836278 73.78977998720147) bank68637 +68638 POINT(41.68850013277766 74.83631987451665) bank68638 +68639 POINT(40.240093038590594 73.6473028118658) bank68639 +68640 POINT(41.125903441506196 73.27060720274795) bank68640 +68641 POINT(41.592537899184144 73.51315590370113) bank68641 +68642 POINT(40.77697262781232 73.73433818646718) bank68642 +68643 POINT(41.060665577799845 74.61100780431943) bank68643 +68644 POINT(39.839991970389754 73.68543779797906) bank68644 +68645 POINT(40.32027354727235 74.02605623406228) bank68645 +68646 POINT(41.34640643204462 73.29815464583415) bank68646 +68647 POINT(40.57068449312982 73.30659916057944) bank68647 +68648 POINT(40.36861199392313 73.07009088111741) bank68648 +68649 POINT(41.6387146837282 74.64215543199754) bank68649 +68650 POINT(40.31947253034112 73.19489151194567) bank68650 +68651 POINT(41.46628277627977 74.58786714299283) bank68651 +68652 POINT(40.66130590888522 73.10049494648725) bank68652 +68653 POINT(40.621692239929786 73.88977256393238) bank68653 +68654 POINT(40.72375074997698 74.00966316856636) bank68654 +68655 POINT(40.843362211966536 74.27688676389226) bank68655 +68656 POINT(40.95662225274299 73.21166949396842) bank68656 +68657 POINT(39.73266462370587 74.7429431010317) bank68657 +68658 POINT(40.802149862876554 73.20457510588857) bank68658 +68659 POINT(41.254688981920836 74.72896418376055) bank68659 +68660 POINT(40.3127491399045 73.44321161672889) bank68660 +68661 POINT(40.415627000252584 73.34906460258007) bank68661 +68662 POINT(39.87926671444291 74.23622626028566) bank68662 +68663 POINT(41.56740552908664 73.88291818377402) bank68663 +68664 POINT(39.816640685880316 73.02425731353921) bank68664 +68665 POINT(40.78652964529776 73.21051345811591) bank68665 +68666 POINT(41.18289034654989 74.07900490918887) bank68666 +68667 POINT(41.51996692084562 73.68407450823604) bank68667 +68668 POINT(40.813501146156554 73.2173415111295) bank68668 +68669 POINT(40.46837897182309 74.75057558000258) bank68669 +68670 POINT(40.615223358633045 74.58485135373049) bank68670 +68671 POINT(39.85781528359284 73.66827607184828) bank68671 +68672 POINT(39.721655457647735 74.86750946409711) bank68672 +68673 POINT(39.762243935183676 74.54736431221113) bank68673 +68674 POINT(40.585450680786394 74.76534779253686) bank68674 +68675 POINT(39.83192345010999 74.85567712315341) bank68675 +68676 POINT(40.510174516909245 73.99752473924146) bank68676 +68677 POINT(41.37498936268123 74.18769835517872) bank68677 +68678 POINT(40.56120737375004 74.60786357603376) bank68678 +68679 POINT(40.604797289670046 73.34641615503108) bank68679 +68680 POINT(39.99765605166926 73.9811170011486) bank68680 +68681 POINT(41.690547536212726 73.11041821535164) bank68681 +68682 POINT(40.282885360221755 74.08971210842635) bank68682 +68683 POINT(39.78066789703146 74.03258595075195) bank68683 +68684 POINT(40.71797677123223 73.80419204254373) bank68684 +68685 POINT(41.02798045918824 74.22869601097342) bank68685 +68686 POINT(40.43845116371738 73.1816540918326) bank68686 +68687 POINT(39.8610134293581 73.05472089224007) bank68687 +68688 POINT(40.788217050932744 74.66012387065041) bank68688 +68689 POINT(40.26894571636025 74.41108106069258) bank68689 +68690 POINT(40.09041197674622 74.8508694261784) bank68690 +68691 POINT(41.40584566260585 73.96032589205848) bank68691 +68692 POINT(40.00726845402386 73.45154363602438) bank68692 +68693 POINT(40.025019186297484 74.73353397424185) bank68693 +68694 POINT(41.59383608157899 73.93775407740304) bank68694 +68695 POINT(40.80132038901473 74.19894894255144) bank68695 +68696 POINT(40.451788208557055 73.81673603590423) bank68696 +68697 POINT(41.656540259849955 73.06467008626159) bank68697 +68698 POINT(40.85213835295516 74.33547944052803) bank68698 +68699 POINT(41.304018429718724 74.34425212857558) bank68699 +68700 POINT(40.70508605602833 73.79199966448138) bank68700 +68701 POINT(40.40559235496864 73.3741247046359) bank68701 +68702 POINT(41.38208669052035 73.05801764421294) bank68702 +68703 POINT(41.419039734561395 73.30050655098891) bank68703 +68704 POINT(40.26555626284227 73.63723055976732) bank68704 +68705 POINT(39.72901576133282 74.38849183139052) bank68705 +68706 POINT(41.27464697090348 73.39740764538078) bank68706 +68707 POINT(41.66547903247704 74.34728680568814) bank68707 +68708 POINT(40.581679175697495 73.23253428111154) bank68708 +68709 POINT(40.315395415359 73.71922851740305) bank68709 +68710 POINT(41.047976020341956 74.77516461183903) bank68710 +68711 POINT(41.63346268107635 74.4963943300344) bank68711 +68712 POINT(39.9132849962644 74.93972955026835) bank68712 +68713 POINT(40.08305209827056 73.0554413604352) bank68713 +68714 POINT(41.345386853251625 74.10407999337693) bank68714 +68715 POINT(39.89132434675552 74.14366605008658) bank68715 +68716 POINT(39.98449891631522 73.85939204298344) bank68716 +68717 POINT(41.46723084297155 74.06600403226172) bank68717 +68718 POINT(41.170798019624215 73.45293774153593) bank68718 +68719 POINT(40.76952155900301 74.52699747412518) bank68719 +68720 POINT(41.6881655887043 74.50173883530283) bank68720 +68721 POINT(40.807814461186936 73.20321125395631) bank68721 +68722 POINT(41.45221591572818 74.66656212958466) bank68722 +68723 POINT(41.124485394607134 74.80025734949258) bank68723 +68724 POINT(41.14261805931173 73.191351329938) bank68724 +68725 POINT(41.68040755726389 73.0442286116966) bank68725 +68726 POINT(39.99897779121123 73.22690059791779) bank68726 +68727 POINT(40.97641960766083 73.7340303035836) bank68727 +68728 POINT(40.04635816552771 74.78583653786289) bank68728 +68729 POINT(41.53572243744559 74.15557232893136) bank68729 +68730 POINT(41.53707775664096 74.54379427462126) bank68730 +68731 POINT(39.846241628663776 74.53089406603017) bank68731 +68732 POINT(39.94567772224195 74.74595548806245) bank68732 +68733 POINT(40.66058418053308 73.05819762141442) bank68733 +68734 POINT(40.545878884749776 73.60126369379806) bank68734 +68735 POINT(40.39300711304751 73.24638786071614) bank68735 +68736 POINT(41.26770618872572 73.18611976588724) bank68736 +68737 POINT(41.31800923605165 74.978545305195) bank68737 +68738 POINT(40.48943619128083 73.15659614586124) bank68738 +68739 POINT(41.259076047387204 74.426459394139) bank68739 +68740 POINT(41.54827845294942 74.58926706334253) bank68740 +68741 POINT(40.162659198193346 74.23978506662081) bank68741 +68742 POINT(39.76617352205296 73.72123975127202) bank68742 +68743 POINT(41.64569736874854 73.39542087903791) bank68743 +68744 POINT(40.68356815580436 74.38457772915451) bank68744 +68745 POINT(40.40471737102067 74.62625633160182) bank68745 +68746 POINT(40.24492139284237 74.34724330324138) bank68746 +68747 POINT(40.65517010285168 74.63209940782262) bank68747 +68748 POINT(41.07618073472768 74.02736787089613) bank68748 +68749 POINT(41.62627293263012 73.20326163125974) bank68749 +68750 POINT(40.69868262084456 74.04796196162988) bank68750 +68751 POINT(41.576467389335164 73.10287679905595) bank68751 +68752 POINT(40.613958718190474 73.30231123961221) bank68752 +68753 POINT(40.02605848272475 74.93704384922438) bank68753 +68754 POINT(39.826301901317755 73.76766683270331) bank68754 +68755 POINT(39.76301404853847 73.34780963032333) bank68755 +68756 POINT(39.93550877763254 73.55715078592593) bank68756 +68757 POINT(40.42948713588587 74.26048533221694) bank68757 +68758 POINT(41.36284424841878 74.67400949462558) bank68758 +68759 POINT(40.12515840429607 74.16131541746228) bank68759 +68760 POINT(41.306606114727316 73.97384913845772) bank68760 +68761 POINT(40.98968174326648 73.249160316849) bank68761 +68762 POINT(41.63629128199991 73.05082955071369) bank68762 +68763 POINT(41.62913657867066 74.66276622200549) bank68763 +68764 POINT(39.72981439019832 74.30532923243155) bank68764 +68765 POINT(40.81422031304205 74.45278765103704) bank68765 +68766 POINT(40.344668236332005 74.09069993705148) bank68766 +68767 POINT(40.85592925644503 73.86438027523698) bank68767 +68768 POINT(41.4320694889554 73.36301595738354) bank68768 +68769 POINT(39.75345690531845 74.68969165735092) bank68769 +68770 POINT(40.9181797016891 74.68194494312827) bank68770 +68771 POINT(40.451448278526875 74.88173688517875) bank68771 +68772 POINT(41.562227217673055 74.57811871914079) bank68772 +68773 POINT(40.93502138516438 74.90082553173882) bank68773 +68774 POINT(41.08117813501435 73.83941274769509) bank68774 +68775 POINT(40.76550541529629 74.15777402132312) bank68775 +68776 POINT(41.35803040480798 74.66329908427572) bank68776 +68777 POINT(40.99051629746928 74.1145166362078) bank68777 +68778 POINT(39.86311082982753 74.06583870449786) bank68778 +68779 POINT(40.00963293780357 74.74974680567838) bank68779 +68780 POINT(39.923529955478884 73.66716218648935) bank68780 +68781 POINT(40.98909513838973 73.5320911272957) bank68781 +68782 POINT(41.41705019818317 73.14209700779858) bank68782 +68783 POINT(40.783520333964155 74.9698820061666) bank68783 +68784 POINT(40.825249284059026 73.28249529215333) bank68784 +68785 POINT(39.8359017399525 74.83917609256432) bank68785 +68786 POINT(41.16806303591987 74.76185402809658) bank68786 +68787 POINT(40.377220216446155 74.21550084078771) bank68787 +68788 POINT(40.57684615375495 73.21687893350563) bank68788 +68789 POINT(40.66392002578846 73.9387428232006) bank68789 +68790 POINT(41.011007846606844 74.800710509792) bank68790 +68791 POINT(40.880635501625775 73.68619603737153) bank68791 +68792 POINT(41.686828965891216 73.21627547864276) bank68792 +68793 POINT(40.79340845110288 73.91051654048808) bank68793 +68794 POINT(41.49462503723566 74.49872853059507) bank68794 +68795 POINT(41.35942000393088 73.61376469830817) bank68795 +68796 POINT(40.15159432674072 73.9895699797373) bank68796 +68797 POINT(41.54768832936577 74.02179280934926) bank68797 +68798 POINT(41.121712380881085 74.84330423917827) bank68798 +68799 POINT(40.91403824150469 74.48469217687125) bank68799 +68800 POINT(39.81650641604173 74.2864006155111) bank68800 +68801 POINT(40.12955386052412 73.44304736897577) bank68801 +68802 POINT(40.8358392256349 73.36985814969582) bank68802 +68803 POINT(41.18126213871467 74.34950209098666) bank68803 +68804 POINT(40.94075690671717 73.85865404094963) bank68804 +68805 POINT(41.19979382221293 74.29354373870522) bank68805 +68806 POINT(41.40126546404367 73.33907447432794) bank68806 +68807 POINT(40.171509165623114 73.51488019144506) bank68807 +68808 POINT(40.61531856798871 73.83647371079553) bank68808 +68809 POINT(40.67566524074397 73.38210203540858) bank68809 +68810 POINT(41.47746737035493 74.04876901443005) bank68810 +68811 POINT(40.756759232890616 73.72452359427187) bank68811 +68812 POINT(41.41612978864665 74.90047285942994) bank68812 +68813 POINT(41.3107038674685 73.96261738985739) bank68813 +68814 POINT(41.6861134543041 73.05414425451113) bank68814 +68815 POINT(39.90516681684916 73.50559258265106) bank68815 +68816 POINT(40.24494070952816 74.03969997521487) bank68816 +68817 POINT(40.257807274823875 74.09454556475963) bank68817 +68818 POINT(41.691045714303925 74.6266147238583) bank68818 +68819 POINT(40.40939320570524 74.34961137303887) bank68819 +68820 POINT(41.20532727514789 73.83526280670654) bank68820 +68821 POINT(41.292962625565046 74.05402948332227) bank68821 +68822 POINT(39.97918818274369 73.61148064578731) bank68822 +68823 POINT(41.36873581742729 73.08906818509516) bank68823 +68824 POINT(40.30044112996444 74.72196647380304) bank68824 +68825 POINT(40.97518604488592 74.28170921750768) bank68825 +68826 POINT(41.38972279233864 73.44311837947532) bank68826 +68827 POINT(41.638723829529596 74.0824413996589) bank68827 +68828 POINT(39.782761150244326 74.56072031637825) bank68828 +68829 POINT(41.63328271931903 74.25940439544063) bank68829 +68830 POINT(40.112937283425595 74.50331762261936) bank68830 +68831 POINT(40.82912070517891 74.50174504597773) bank68831 +68832 POINT(40.878569041674176 74.6388112795886) bank68832 +68833 POINT(41.20802953220007 73.51099643852685) bank68833 +68834 POINT(40.23475968965747 74.97428729185076) bank68834 +68835 POINT(40.76060762021348 74.87811608818178) bank68835 +68836 POINT(41.146932894003875 74.22876836330549) bank68836 +68837 POINT(40.049023867052284 74.74606766867727) bank68837 +68838 POINT(40.52945305965918 73.5024506306018) bank68838 +68839 POINT(40.82574901710027 74.60485838702715) bank68839 +68840 POINT(41.6757533856831 74.1526232721396) bank68840 +68841 POINT(40.974531306397346 73.17119387765878) bank68841 +68842 POINT(40.4556940463348 74.19765239426034) bank68842 +68843 POINT(41.39183904656456 74.94014697458503) bank68843 +68844 POINT(41.020778460179415 74.09744025412647) bank68844 +68845 POINT(41.694924668392765 74.81312431797758) bank68845 +68846 POINT(40.71564718986 74.90028779469853) bank68846 +68847 POINT(40.97224280842369 74.82939518779108) bank68847 +68848 POINT(41.148432621710576 74.32045204309901) bank68848 +68849 POINT(41.22234370840024 73.2040429919412) bank68849 +68850 POINT(40.46343754417575 74.49555528125391) bank68850 +68851 POINT(41.03787961777437 73.73526157997865) bank68851 +68852 POINT(41.654130880036384 74.19224913505003) bank68852 +68853 POINT(41.377171178306405 75.00339449059109) bank68853 +68854 POINT(41.049801134967964 74.51673998503033) bank68854 +68855 POINT(40.557949315626296 74.9594276317881) bank68855 +68856 POINT(39.800732258695646 74.77050880355688) bank68856 +68857 POINT(40.51389876009017 74.37084251147986) bank68857 +68858 POINT(41.35669740597391 73.52068617655142) bank68858 +68859 POINT(40.795216380831654 73.21710473613584) bank68859 +68860 POINT(41.129353319678636 73.3736523854325) bank68860 +68861 POINT(40.99400095718409 74.06831367257956) bank68861 +68862 POINT(40.71266665041632 74.3030174618861) bank68862 +68863 POINT(41.01245859127744 73.94391830842292) bank68863 +68864 POINT(39.87995739929706 73.35140694172554) bank68864 +68865 POINT(41.57515233054784 74.49373705770105) bank68865 +68866 POINT(41.038069950790934 73.80360066871587) bank68866 +68867 POINT(40.919280455019766 74.4737713714454) bank68867 +68868 POINT(41.10835483425449 74.4668919665881) bank68868 +68869 POINT(41.15228891256644 73.7340706559163) bank68869 +68870 POINT(39.736101837523364 73.97286739364459) bank68870 +68871 POINT(41.37439917567626 73.43643647701836) bank68871 +68872 POINT(41.521870559744784 75.00419709167885) bank68872 +68873 POINT(41.50931254840014 74.58055709891867) bank68873 +68874 POINT(39.84822103769839 74.8704962539636) bank68874 +68875 POINT(39.84623124818968 74.67386530269663) bank68875 +68876 POINT(40.30150391722508 74.17075100705081) bank68876 +68877 POINT(40.81642680831848 73.72360494443495) bank68877 +68878 POINT(40.36093753901762 73.06508706368363) bank68878 +68879 POINT(40.910389265589245 74.48357720260886) bank68879 +68880 POINT(39.75790803940416 74.23372993548175) bank68880 +68881 POINT(40.356537342186314 73.71187007908497) bank68881 +68882 POINT(40.475853147885765 73.00629528309933) bank68882 +68883 POINT(40.26917559562913 73.55209497078323) bank68883 +68884 POINT(40.35431269069102 74.789426350077) bank68884 +68885 POINT(39.970679631681556 74.9967097664858) bank68885 +68886 POINT(39.86798528900027 73.06790559612185) bank68886 +68887 POINT(39.9364892421504 74.95388548134238) bank68887 +68888 POINT(40.906838196321715 74.03143502553205) bank68888 +68889 POINT(41.29330957965433 73.67309933148208) bank68889 +68890 POINT(40.42981408941128 73.34138477441577) bank68890 +68891 POINT(39.74690429311058 74.70033814050797) bank68891 +68892 POINT(40.6698642983074 74.45623564838516) bank68892 +68893 POINT(41.249314358273615 74.00439171656954) bank68893 +68894 POINT(39.89525728956302 73.80937826753633) bank68894 +68895 POINT(40.571124985315095 73.42065667315653) bank68895 +68896 POINT(40.67255867604746 73.91053222766274) bank68896 +68897 POINT(41.42488824156637 74.79466367909671) bank68897 +68898 POINT(41.20434249961708 74.6061085193636) bank68898 +68899 POINT(40.50010958331103 74.91040305112362) bank68899 +68900 POINT(40.88752891151249 74.85096926453559) bank68900 +68901 POINT(40.335207847523144 74.66659225018601) bank68901 +68902 POINT(39.84793096068504 74.70651809657502) bank68902 +68903 POINT(39.94966616891291 74.13020834526354) bank68903 +68904 POINT(40.20009464898565 73.56229657527922) bank68904 +68905 POINT(41.52324606771482 73.72790149951607) bank68905 +68906 POINT(41.48534294057811 73.11732539766717) bank68906 +68907 POINT(40.73101313416229 74.89342304129254) bank68907 +68908 POINT(41.335436439900334 73.80119451644819) bank68908 +68909 POINT(40.92917501227105 73.34023168898207) bank68909 +68910 POINT(41.51694751481075 74.6070714521264) bank68910 +68911 POINT(39.735755103031565 74.51676743905728) bank68911 +68912 POINT(41.592653123279824 74.25459967546378) bank68912 +68913 POINT(41.256570493317085 74.56331102297952) bank68913 +68914 POINT(39.772584875193765 73.47620374858809) bank68914 +68915 POINT(41.18375214677042 74.46801699584069) bank68915 +68916 POINT(40.97691020890037 74.69441373218474) bank68916 +68917 POINT(41.523686145597935 73.96103682955665) bank68917 +68918 POINT(40.4032651898889 74.05909635052944) bank68918 +68919 POINT(41.31767288930075 74.6961632384723) bank68919 +68920 POINT(41.08489792313787 74.68446233197562) bank68920 +68921 POINT(40.37341253733038 74.62579906586858) bank68921 +68922 POINT(40.40387897953708 73.84186590425149) bank68922 +68923 POINT(40.708592150864526 74.62119093295225) bank68923 +68924 POINT(40.14485461919369 74.18276406647888) bank68924 +68925 POINT(39.93680766950815 74.60111580152468) bank68925 +68926 POINT(39.78799923476977 74.78956769709737) bank68926 +68927 POINT(41.15148843951257 73.65691326143158) bank68927 +68928 POINT(39.805171336079724 74.43124258836492) bank68928 +68929 POINT(41.18110612438275 73.91921194569046) bank68929 +68930 POINT(40.50020939266969 74.75628034385124) bank68930 +68931 POINT(40.78780579312151 74.85739237130167) bank68931 +68932 POINT(41.01397611560743 74.66203296604974) bank68932 +68933 POINT(39.808127876374236 74.10082975784346) bank68933 +68934 POINT(41.2543535580999 74.25860184562615) bank68934 +68935 POINT(40.82182615882175 74.76436074706739) bank68935 +68936 POINT(41.614849968194 74.01761007846883) bank68936 +68937 POINT(40.281999318898436 74.61540272860613) bank68937 +68938 POINT(40.934863936995534 73.37792899829171) bank68938 +68939 POINT(39.773416994848155 73.0816367412197) bank68939 +68940 POINT(40.48995824157729 74.8412213030522) bank68940 +68941 POINT(40.48917052158547 73.83637888120991) bank68941 +68942 POINT(40.632558415982444 74.47361400568663) bank68942 +68943 POINT(40.16431651546373 74.84659931607608) bank68943 +68944 POINT(39.77132813798938 74.17719506199823) bank68944 +68945 POINT(41.045028356164856 73.76885272488539) bank68945 +68946 POINT(39.87686456164474 73.12154482105194) bank68946 +68947 POINT(40.713433194392394 73.83615879755774) bank68947 +68948 POINT(40.91636211308102 73.93698939570375) bank68948 +68949 POINT(41.069124574869655 74.93691829206332) bank68949 +68950 POINT(40.82778312160349 74.83556090783193) bank68950 +68951 POINT(41.518624297544264 73.63882666892287) bank68951 +68952 POINT(39.767757458259354 74.0486639265653) bank68952 +68953 POINT(40.960195049987604 73.70927074127158) bank68953 +68954 POINT(40.556348361777296 73.23780915074366) bank68954 +68955 POINT(39.76168705033688 74.49620977717846) bank68955 +68956 POINT(40.582058411965974 73.9351637559554) bank68956 +68957 POINT(40.8426645841842 74.6152635548448) bank68957 +68958 POINT(40.86493818066782 73.95360040819229) bank68958 +68959 POINT(41.55282176893719 73.41875873593423) bank68959 +68960 POINT(41.186740177614055 74.99092312703046) bank68960 +68961 POINT(40.61184381828878 74.75002171125699) bank68961 +68962 POINT(40.310262389026775 73.8747364845629) bank68962 +68963 POINT(41.64774388037871 73.77399130652653) bank68963 +68964 POINT(41.00170861839607 73.65896011774687) bank68964 +68965 POINT(41.49867564943611 73.23894076516015) bank68965 +68966 POINT(39.80183924304269 74.07297900261568) bank68966 +68967 POINT(40.18435491320163 74.32150547101254) bank68967 +68968 POINT(40.14628275410175 73.82365669727089) bank68968 +68969 POINT(41.364873574053895 73.6301499045396) bank68969 +68970 POINT(41.139321407152025 73.48122758203623) bank68970 +68971 POINT(41.06989307797368 73.90483548684753) bank68971 +68972 POINT(40.36763888868182 74.94385905978227) bank68972 +68973 POINT(40.08182943981675 74.55837250489395) bank68973 +68974 POINT(41.05311904304504 74.444052409024) bank68974 +68975 POINT(40.25909206165763 73.99011026695186) bank68975 +68976 POINT(41.63373439044177 74.68447559283327) bank68976 +68977 POINT(41.15361400452402 73.73868480093762) bank68977 +68978 POINT(41.44390804931861 74.40936006440876) bank68978 +68979 POINT(40.721387541795 73.4041614702026) bank68979 +68980 POINT(39.73972775577477 73.22549745364395) bank68980 +68981 POINT(40.85098296869116 74.3018413337556) bank68981 +68982 POINT(40.76287777821902 74.3390482467522) bank68982 +68983 POINT(41.22282773262251 73.39142231394418) bank68983 +68984 POINT(40.54860542474998 74.00816979463931) bank68984 +68985 POINT(40.10136639727132 74.6484585928367) bank68985 +68986 POINT(39.85543506690541 73.51867036660772) bank68986 +68987 POINT(39.99438480670183 74.33405868169994) bank68987 +68988 POINT(40.848819967774276 74.16498297905652) bank68988 +68989 POINT(40.076397774825914 74.89617958640787) bank68989 +68990 POINT(41.205449725793216 73.66269093921478) bank68990 +68991 POINT(40.18288872744258 74.52291639011916) bank68991 +68992 POINT(39.76767135809003 74.8921899691033) bank68992 +68993 POINT(40.69755638947842 74.08989416030317) bank68993 +68994 POINT(40.91698861000256 73.7647913138671) bank68994 +68995 POINT(41.05518621667154 73.274937065373) bank68995 +68996 POINT(41.24759129547794 74.48014477576457) bank68996 +68997 POINT(40.475661887145336 74.98185153436715) bank68997 +68998 POINT(40.861569573287206 73.7867551077304) bank68998 +68999 POINT(41.49973499630041 74.21893479123008) bank68999 +69000 POINT(41.177300362123525 74.0401056337949) bank69000 +69001 POINT(41.531204445522555 74.52822724109572) bank69001 +69002 POINT(40.731919883281584 73.96818006857355) bank69002 +69003 POINT(41.46005933710306 73.59569952184651) bank69003 +69004 POINT(40.2695865900295 73.79828618031875) bank69004 +69005 POINT(40.654244513902846 73.11614421668015) bank69005 +69006 POINT(41.69205328556253 74.4873885797727) bank69006 +69007 POINT(39.79322815225076 73.67072099016099) bank69007 +69008 POINT(40.117303801414934 73.7124153682334) bank69008 +69009 POINT(39.77950658890799 73.28155499931015) bank69009 +69010 POINT(41.07023856064718 73.0822271636261) bank69010 +69011 POINT(40.1389368818884 73.87275921101822) bank69011 +69012 POINT(39.81730856859088 73.1092673366358) bank69012 +69013 POINT(41.369951300939185 73.46805717139716) bank69013 +69014 POINT(40.14513310327537 73.73669317455607) bank69014 +69015 POINT(40.20054558169027 74.39804937869879) bank69015 +69016 POINT(39.810607215852926 74.07726061081412) bank69016 +69017 POINT(40.93205420469903 74.0638973960164) bank69017 +69018 POINT(40.253047211449626 73.63829737360678) bank69018 +69019 POINT(41.42567244523668 73.39062281749034) bank69019 +69020 POINT(40.2898169855977 74.98268122197393) bank69020 +69021 POINT(40.688717951201305 74.92648107180888) bank69021 +69022 POINT(40.82231222306651 73.14952000354556) bank69022 +69023 POINT(40.912658175465175 73.13091564905434) bank69023 +69024 POINT(41.02899948900263 73.51101191246786) bank69024 +69025 POINT(41.56212177307096 73.8023876638461) bank69025 +69026 POINT(40.49522714150172 73.29756634416921) bank69026 +69027 POINT(40.1913540809943 73.98104587587055) bank69027 +69028 POINT(39.84081457118852 74.7922606554024) bank69028 +69029 POINT(41.13521683483205 73.02342594836428) bank69029 +69030 POINT(40.56740157515792 73.62111477528968) bank69030 +69031 POINT(40.318810110935544 74.26893008130561) bank69031 +69032 POINT(40.06577377548019 73.59611174539424) bank69032 +69033 POINT(40.42818245341175 74.4225770177392) bank69033 +69034 POINT(40.04726885910232 74.38249731483329) bank69034 +69035 POINT(40.79595313941098 74.98976338490075) bank69035 +69036 POINT(41.145894124249004 74.81609798811978) bank69036 +69037 POINT(40.46268896814325 74.58416166564487) bank69037 +69038 POINT(41.24702997204634 74.5181278900574) bank69038 +69039 POINT(39.71351571024862 74.38536559256355) bank69039 +69040 POINT(41.57643373341891 74.46356754128755) bank69040 +69041 POINT(40.736911728097496 74.06809676525667) bank69041 +69042 POINT(40.441122308741214 74.58973275039577) bank69042 +69043 POINT(40.302583162697466 74.81604506055095) bank69043 +69044 POINT(40.4536195363668 74.92271624214959) bank69044 +69045 POINT(39.87241684097333 73.74766210660796) bank69045 +69046 POINT(40.00147930276147 74.63914189394966) bank69046 +69047 POINT(39.86576551315544 74.58251225787802) bank69047 +69048 POINT(39.73427540826738 74.09190113175798) bank69048 +69049 POINT(40.62991633422816 73.83082226382365) bank69049 +69050 POINT(40.32253013986254 73.32331005892149) bank69050 +69051 POINT(40.533857960768934 73.92110365773013) bank69051 +69052 POINT(41.11036747464718 73.13848100259433) bank69052 +69053 POINT(41.68292759517639 73.91019344352902) bank69053 +69054 POINT(39.790346135789754 73.42351454362328) bank69054 +69055 POINT(40.48184160282068 74.33463670156873) bank69055 +69056 POINT(41.19152493969483 74.12740950275854) bank69056 +69057 POINT(40.39972931491308 73.57050518511396) bank69057 +69058 POINT(41.05870962385384 74.23721903775711) bank69058 +69059 POINT(41.660043187966316 73.52050557745939) bank69059 +69060 POINT(39.900573222430914 75.00456016780885) bank69060 +69061 POINT(40.706287104714015 73.36007116176918) bank69061 +69062 POINT(41.36038656139158 73.80145028107631) bank69062 +69063 POINT(40.98957277106243 73.54674779515328) bank69063 +69064 POINT(41.56501041135516 73.33127384551342) bank69064 +69065 POINT(41.13623337974282 74.96660153918178) bank69065 +69066 POINT(40.0688468438604 73.07366787491648) bank69066 +69067 POINT(40.06882863014857 73.59836397368365) bank69067 +69068 POINT(41.15241426846975 73.50646430017858) bank69068 +69069 POINT(40.32167851357981 73.71032226223282) bank69069 +69070 POINT(40.0088984730971 73.46363529856407) bank69070 +69071 POINT(39.9156193596874 74.857859871764) bank69071 +69072 POINT(40.22865923236882 73.62283078282388) bank69072 +69073 POINT(40.59259661469469 73.37988676167834) bank69073 +69074 POINT(41.1337856734579 73.27563004844767) bank69074 +69075 POINT(41.146779141199765 73.46559713796339) bank69075 +69076 POINT(40.78063015719979 73.63084128301865) bank69076 +69077 POINT(41.57832310603743 74.51449723831846) bank69077 +69078 POINT(41.58955489649089 73.85104490487213) bank69078 +69079 POINT(40.60219747250321 73.0707424897953) bank69079 +69080 POINT(40.66118884870171 74.23511875322673) bank69080 +69081 POINT(40.75638399278279 74.78383461607842) bank69081 +69082 POINT(39.966536927766015 74.24944319462067) bank69082 +69083 POINT(40.96785096408694 74.64877793505129) bank69083 +69084 POINT(40.051494124603856 74.24980062503285) bank69084 +69085 POINT(39.79096910354354 74.98740643085534) bank69085 +69086 POINT(40.36229461458763 73.41934803030135) bank69086 +69087 POINT(41.132360682342394 74.73492383575241) bank69087 +69088 POINT(41.26175360796044 74.25656459737637) bank69088 +69089 POINT(39.75235041161101 74.65272837683857) bank69089 +69090 POINT(40.17278519414721 73.99607716637442) bank69090 +69091 POINT(40.8292758432634 73.77067992840178) bank69091 +69092 POINT(41.55847154617049 74.47949148616853) bank69092 +69093 POINT(40.874562927489144 74.9966869719694) bank69093 +69094 POINT(40.223855277955394 73.05625528333496) bank69094 +69095 POINT(40.50593911198604 74.0741778729649) bank69095 +69096 POINT(40.35191985905192 74.68783066821986) bank69096 +69097 POINT(41.25286225949587 73.14071619411459) bank69097 +69098 POINT(41.3784914222923 74.41093957051505) bank69098 +69099 POINT(41.610809084132725 73.47873581064498) bank69099 +69100 POINT(39.766494406916046 74.7773186129391) bank69100 +69101 POINT(41.51643616657352 73.44939856704246) bank69101 +69102 POINT(39.906020541805766 74.46543612733014) bank69102 +69103 POINT(40.30869128176062 73.99869627851375) bank69103 +69104 POINT(41.58806003062175 74.41706945922078) bank69104 +69105 POINT(39.741019319904 73.74539979179309) bank69105 +69106 POINT(40.832050311188844 73.23393489551754) bank69106 +69107 POINT(40.61742337758149 74.61207397796083) bank69107 +69108 POINT(41.065441055209774 73.0192828835189) bank69108 +69109 POINT(40.1358142100161 74.41425711477652) bank69109 +69110 POINT(40.32516179930025 73.32585212146913) bank69110 +69111 POINT(39.79247915119465 73.93114375872504) bank69111 +69112 POINT(41.04031347924001 74.00500444246966) bank69112 +69113 POINT(41.25581243165611 74.53863315909716) bank69113 +69114 POINT(39.74802706437845 73.38730603406368) bank69114 +69115 POINT(40.62992278081988 73.17096341708813) bank69115 +69116 POINT(39.86171303926449 73.92801206226946) bank69116 +69117 POINT(40.3561797874626 73.39009098774862) bank69117 +69118 POINT(41.4114714779422 74.18208111524667) bank69118 +69119 POINT(41.5828091602014 74.18044037220753) bank69119 +69120 POINT(41.14216790279719 73.09343857324824) bank69120 +69121 POINT(40.29257671574605 73.89074793531822) bank69121 +69122 POINT(41.71105680319425 73.28804465589786) bank69122 +69123 POINT(41.45781486381294 74.53979044036707) bank69123 +69124 POINT(40.3211375302741 74.39008805196657) bank69124 +69125 POINT(40.39525618974481 74.3721499640825) bank69125 +69126 POINT(40.62906425651997 74.0564712553156) bank69126 +69127 POINT(39.99337456679283 73.32841475615497) bank69127 +69128 POINT(40.2583650604228 74.19034042734937) bank69128 +69129 POINT(40.89759669580258 74.3158386288799) bank69129 +69130 POINT(41.1221963277974 74.38533011795957) bank69130 +69131 POINT(40.822341895240996 73.0767565400665) bank69131 +69132 POINT(40.795979959482345 74.17325258245795) bank69132 +69133 POINT(40.09841124462906 73.88738427440288) bank69133 +69134 POINT(40.06750934667509 73.41775154811994) bank69134 +69135 POINT(40.174160866364474 74.4385640291268) bank69135 +69136 POINT(41.617908195019446 73.36832379378478) bank69136 +69137 POINT(39.87053711255196 73.36410347160535) bank69137 +69138 POINT(40.51116898067945 73.33235863885166) bank69138 +69139 POINT(39.73485295408545 74.09059138316164) bank69139 +69140 POINT(41.26409125354884 74.53644708240061) bank69140 +69141 POINT(41.311297610278494 74.41614887577879) bank69141 +69142 POINT(41.37255657759754 74.8498686904816) bank69142 +69143 POINT(40.479605524676764 73.1873244084925) bank69143 +69144 POINT(40.60472669573335 74.30942565981779) bank69144 +69145 POINT(40.00234257301605 73.10412404516394) bank69145 +69146 POINT(40.227819158015656 73.46068433674535) bank69146 +69147 POINT(41.24714426273112 74.83593025747511) bank69147 +69148 POINT(41.23329030651268 73.3650977387505) bank69148 +69149 POINT(41.388409309283475 73.06416679175217) bank69149 +69150 POINT(40.7819345687596 73.75287385562095) bank69150 +69151 POINT(41.2703609823697 73.21836768115297) bank69151 +69152 POINT(39.848967481552506 74.07097973741067) bank69152 +69153 POINT(40.10462743629142 73.91882478061186) bank69153 +69154 POINT(40.22651696493347 74.19453389886401) bank69154 +69155 POINT(40.26081681540939 74.1974211765898) bank69155 +69156 POINT(40.28017236885028 74.00137892729519) bank69156 +69157 POINT(41.43581266808613 73.10348770008122) bank69157 +69158 POINT(41.029091261060195 73.21531252813179) bank69158 +69159 POINT(41.41265924683521 74.18204672387569) bank69159 +69160 POINT(40.22505291588545 73.39642429245501) bank69160 +69161 POINT(40.291163323341635 73.36328480639263) bank69161 +69162 POINT(40.78348488624391 73.958440814149) bank69162 +69163 POINT(41.68338519513871 74.92896536961418) bank69163 +69164 POINT(40.00974511754662 73.81594920939607) bank69164 +69165 POINT(39.91436010793466 73.47323717320766) bank69165 +69166 POINT(39.987766668380885 73.81087576863216) bank69166 +69167 POINT(41.15190222070303 73.46719052395133) bank69167 +69168 POINT(40.19873237485966 74.77712779401013) bank69168 +69169 POINT(41.18870915524706 74.00137629816123) bank69169 +69170 POINT(40.336247850597076 74.11921573566379) bank69170 +69171 POINT(39.84167297331921 74.73630770940242) bank69171 +69172 POINT(40.52645023970233 73.59014437603032) bank69172 +69173 POINT(41.20230554712526 73.54826158855968) bank69173 +69174 POINT(41.59726015023208 73.75371267058055) bank69174 +69175 POINT(39.77530021330822 74.17888365903569) bank69175 +69176 POINT(40.78946702459934 73.79170063022018) bank69176 +69177 POINT(41.36248961163855 73.36168446979457) bank69177 +69178 POINT(40.35189676618768 73.27713588788009) bank69178 +69179 POINT(41.50378848942091 73.18728535622228) bank69179 +69180 POINT(40.91786541506276 73.89159002068008) bank69180 +69181 POINT(41.70059754356472 74.30457278717547) bank69181 +69182 POINT(41.56749906378671 73.03588834192276) bank69182 +69183 POINT(39.88192278049764 73.22814245533574) bank69183 +69184 POINT(39.79100578951253 73.32874423441051) bank69184 +69185 POINT(41.04236666638515 73.17992492050257) bank69185 +69186 POINT(41.56941379797703 74.93162926782422) bank69186 +69187 POINT(40.57894834538642 74.61980465580147) bank69187 +69188 POINT(40.43915866854471 74.13496833676554) bank69188 +69189 POINT(40.55715547568836 74.6579302953051) bank69189 +69190 POINT(39.93126058498151 74.03028689556379) bank69190 +69191 POINT(41.158051363517934 73.26562576010178) bank69191 +69192 POINT(41.38063308332226 73.88489847108362) bank69192 +69193 POINT(39.94350588833971 73.73744319297501) bank69193 +69194 POINT(39.75981525587647 74.19482604330749) bank69194 +69195 POINT(40.55968981768293 74.46314064551812) bank69195 +69196 POINT(40.809463301558736 73.80482636790562) bank69196 +69197 POINT(40.00465818313544 73.92728327548497) bank69197 +69198 POINT(40.36339792615386 74.89737816441342) bank69198 +69199 POINT(41.57517979458261 74.84579254872436) bank69199 +69200 POINT(40.12828443356515 74.40342049215839) bank69200 +69201 POINT(40.3206151004082 73.79435104288288) bank69201 +69202 POINT(40.42531769622329 73.65845273286843) bank69202 +69203 POINT(40.011880654006454 74.57524295775126) bank69203 +69204 POINT(41.50530001428575 73.19668724901501) bank69204 +69205 POINT(40.64792004871349 74.48638843156235) bank69205 +69206 POINT(40.27666338703838 74.32333155621983) bank69206 +69207 POINT(41.203669489217916 73.7764482936772) bank69207 +69208 POINT(40.47624000805694 74.26401285670033) bank69208 +69209 POINT(40.3008604587323 74.24666094952804) bank69209 +69210 POINT(39.972048255714505 74.89540918856304) bank69210 +69211 POINT(40.548706065513066 74.65717112504977) bank69211 +69212 POINT(40.328936511034016 73.9448067282394) bank69212 +69213 POINT(41.51357811447343 74.94333835420419) bank69213 +69214 POINT(40.56858157938904 74.0709167599509) bank69214 +69215 POINT(39.83420928187224 74.85757480172603) bank69215 +69216 POINT(41.40457201246812 74.41527793444803) bank69216 +69217 POINT(40.9843717716242 73.89322262282647) bank69217 +69218 POINT(41.36797467031411 73.69752432714293) bank69218 +69219 POINT(40.30431436658698 73.16007020565121) bank69219 +69220 POINT(39.77904905335089 73.51593834911802) bank69220 +69221 POINT(40.47415043745724 73.62119323043099) bank69221 +69222 POINT(41.16207387695013 74.62406973968086) bank69222 +69223 POINT(41.20785440019086 74.15741113574768) bank69223 +69224 POINT(41.19613749628724 74.30109291124936) bank69224 +69225 POINT(41.694988086567626 74.56136311980825) bank69225 +69226 POINT(41.65161070238129 73.75538333741994) bank69226 +69227 POINT(39.7850946305578 74.28800988422104) bank69227 +69228 POINT(40.94710562063489 74.8302850001427) bank69228 +69229 POINT(40.36228836196733 73.12964300375877) bank69229 +69230 POINT(40.68399314831333 74.29690717892107) bank69230 +69231 POINT(40.70573515424937 73.81944163968072) bank69231 +69232 POINT(40.43438834908099 73.15359661921133) bank69232 +69233 POINT(40.56844669395054 73.98879472036154) bank69233 +69234 POINT(40.27117385836926 73.0654344809776) bank69234 +69235 POINT(40.430764965460064 74.76412812734023) bank69235 +69236 POINT(39.89200845246098 73.3796912259704) bank69236 +69237 POINT(40.766246044915206 74.14972986061534) bank69237 +69238 POINT(41.22440821892768 73.72554626017622) bank69238 +69239 POINT(41.01769352903915 74.53554693323315) bank69239 +69240 POINT(41.08912947855179 73.90518582174089) bank69240 +69241 POINT(40.896685242415096 74.35132491664) bank69241 +69242 POINT(40.35733900359235 74.42822638196809) bank69242 +69243 POINT(40.20602438613105 74.49271378887904) bank69243 +69244 POINT(40.63956533510789 74.12241110786806) bank69244 +69245 POINT(40.99086616281219 74.31525068187482) bank69245 +69246 POINT(41.57912727933306 74.3448950750976) bank69246 +69247 POINT(40.84883093754304 74.88523865177744) bank69247 +69248 POINT(40.74652034882605 73.96154174441604) bank69248 +69249 POINT(41.67989159248862 74.3559658329383) bank69249 +69250 POINT(41.1528491442362 74.59099245738483) bank69250 +69251 POINT(40.286653108954894 73.84938732928181) bank69251 +69252 POINT(40.646698601989314 74.22751319252444) bank69252 +69253 POINT(40.8932155382237 73.7517412111743) bank69253 +69254 POINT(40.412483360740346 74.59046799264853) bank69254 +69255 POINT(39.96328351885762 74.59883630936606) bank69255 +69256 POINT(40.631416575552635 73.01333586308893) bank69256 +69257 POINT(40.41679187563838 73.81938255782634) bank69257 +69258 POINT(41.25558327276702 74.12108049524117) bank69258 +69259 POINT(41.326669933215044 73.0268333842) bank69259 +69260 POINT(40.74878279954455 73.9523532034194) bank69260 +69261 POINT(39.740838932737695 74.55079222842853) bank69261 +69262 POINT(41.120868635066294 74.35434515069817) bank69262 +69263 POINT(41.217313455737056 73.89858213222986) bank69263 +69264 POINT(41.10869595618316 74.22619270312997) bank69264 +69265 POINT(40.73912185729427 74.38759985496455) bank69265 +69266 POINT(39.95744504783521 74.07503793346852) bank69266 +69267 POINT(40.145712302833566 73.27052691408853) bank69267 +69268 POINT(40.16068774410872 73.55140933526496) bank69268 +69269 POINT(40.09331791844467 73.82516579724356) bank69269 +69270 POINT(40.48476931597905 73.3012927215598) bank69270 +69271 POINT(41.500759591012816 74.95269962716719) bank69271 +69272 POINT(40.83376577875893 73.51445048920814) bank69272 +69273 POINT(40.22767268264352 73.78910074627655) bank69273 +69274 POINT(40.48074475680901 74.52572852459086) bank69274 +69275 POINT(41.45806496981978 74.14395729888405) bank69275 +69276 POINT(39.81860699509373 74.79491002329972) bank69276 +69277 POINT(40.81690902134861 74.72154725946325) bank69277 +69278 POINT(39.82379473781023 73.43315055807555) bank69278 +69279 POINT(39.72763817048466 73.3350435004255) bank69279 +69280 POINT(39.71550179887853 73.3193826986438) bank69280 +69281 POINT(40.25249907339574 74.64055695245592) bank69281 +69282 POINT(40.677452775874585 73.15654186343764) bank69282 +69283 POINT(40.40179528556784 73.12070137201685) bank69283 +69284 POINT(39.921167989051575 73.38495764982453) bank69284 +69285 POINT(40.15784754630321 74.04649436573453) bank69285 +69286 POINT(40.916432238545205 74.6216994231544) bank69286 +69287 POINT(41.16826564139203 73.78070535178244) bank69287 +69288 POINT(41.297429462668504 74.72955916947677) bank69288 +69289 POINT(40.417636270366586 74.4912543368635) bank69289 +69290 POINT(41.50817824013609 73.60041879060631) bank69290 +69291 POINT(41.49113141142132 73.149751206001) bank69291 +69292 POINT(40.8213181968355 74.32746369547888) bank69292 +69293 POINT(41.69085417854578 73.40070787828132) bank69293 +69294 POINT(40.64545884791585 74.71859362264749) bank69294 +69295 POINT(40.274459687086896 73.88731218080822) bank69295 +69296 POINT(40.282317437195594 73.12826771976003) bank69296 +69297 POINT(41.64418143375811 74.19657416226659) bank69297 +69298 POINT(40.41325301914675 73.63135584912786) bank69298 +69299 POINT(41.55111610334768 73.01335215755047) bank69299 +69300 POINT(41.20893669021398 73.94845546139769) bank69300 +69301 POINT(40.049870267251286 74.00128523029994) bank69301 +69302 POINT(41.402820788302755 73.78428617810061) bank69302 +69303 POINT(40.18177494390714 73.81196174024815) bank69303 +69304 POINT(40.2396202723141 73.34519571193105) bank69304 +69305 POINT(40.806612053618046 73.37437926702015) bank69305 +69306 POINT(39.93136519185604 73.20004693132606) bank69306 +69307 POINT(40.15163020669183 74.50101675455174) bank69307 +69308 POINT(40.527769780825366 74.1451379532491) bank69308 +69309 POINT(40.675670074388265 73.82901763120148) bank69309 +69310 POINT(40.11052991323713 73.09067350408071) bank69310 +69311 POINT(40.896575052911714 73.71121427329778) bank69311 +69312 POINT(39.85101062938758 73.32906817155225) bank69312 +69313 POINT(40.04384578876052 73.1468686199404) bank69313 +69314 POINT(41.200640855542716 73.3849034049547) bank69314 +69315 POINT(41.59782092416822 73.37317859659106) bank69315 +69316 POINT(41.521265371423596 74.50795141142892) bank69316 +69317 POINT(41.68438705840718 73.02887416228194) bank69317 +69318 POINT(41.54658905343311 74.12330091649707) bank69318 +69319 POINT(40.116578887536456 74.06080617233286) bank69319 +69320 POINT(40.35379471154133 74.11496923774432) bank69320 +69321 POINT(39.993226680714315 73.36627862853709) bank69321 +69322 POINT(40.277064295315135 73.87512272168884) bank69322 +69323 POINT(41.154146104536245 74.26234069530324) bank69323 +69324 POINT(40.74979540696172 74.37996763578047) bank69324 +69325 POINT(40.723677687205466 74.94027302272484) bank69325 +69326 POINT(39.91430946273979 74.45401857343674) bank69326 +69327 POINT(40.87903701778681 74.47425625420436) bank69327 +69328 POINT(41.241971227064376 74.18368118763493) bank69328 +69329 POINT(41.33706421329581 74.35599205828605) bank69329 +69330 POINT(40.29944071216994 73.34158507461082) bank69330 +69331 POINT(39.97003546262524 73.21796278137715) bank69331 +69332 POINT(41.679536014532644 74.45067600625492) bank69332 +69333 POINT(40.39579284305552 74.47940926675231) bank69333 +69334 POINT(40.43436804348596 73.56149228457227) bank69334 +69335 POINT(40.05039041393587 73.71268756188789) bank69335 +69336 POINT(39.91088511848241 74.45366490663729) bank69336 +69337 POINT(40.27224314892253 74.32371558500452) bank69337 +69338 POINT(40.633506768495444 74.06319780748446) bank69338 +69339 POINT(41.60482669045251 74.74065036918466) bank69339 +69340 POINT(41.40979091703538 73.04908377102922) bank69340 +69341 POINT(40.923010909364336 73.33317073671307) bank69341 +69342 POINT(40.76538145845197 74.48732918337336) bank69342 +69343 POINT(41.14421197480325 73.89420146339647) bank69343 +69344 POINT(40.17885617198392 74.12101175044099) bank69344 +69345 POINT(41.37159635346555 73.82539649062967) bank69345 +69346 POINT(40.16431397775127 73.17948215873434) bank69346 +69347 POINT(39.88198381085026 73.80614437098419) bank69347 +69348 POINT(40.88107942255206 73.48567857633653) bank69348 +69349 POINT(41.15781662913809 73.57578420002409) bank69349 +69350 POINT(41.292760899814716 73.73485808683384) bank69350 +69351 POINT(39.95917858083904 73.14222183262025) bank69351 +69352 POINT(39.87975141382316 74.89275712992674) bank69352 +69353 POINT(40.70674832220728 74.02076889706943) bank69353 +69354 POINT(41.00555052678036 74.76286690375323) bank69354 +69355 POINT(41.28056290785939 74.41722067205951) bank69355 +69356 POINT(40.62270005379269 74.08025839694848) bank69356 +69357 POINT(40.58204623611002 74.94334992706217) bank69357 +69358 POINT(40.724060703812995 73.7705506218107) bank69358 +69359 POINT(41.18461775460753 73.08068437340776) bank69359 +69360 POINT(41.49266154780662 73.47008962395182) bank69360 +69361 POINT(39.92867062495085 73.89058142023734) bank69361 +69362 POINT(41.0878128499455 73.69006764107719) bank69362 +69363 POINT(40.235737079183615 73.03425770542259) bank69363 +69364 POINT(40.00839997440217 73.95168654400264) bank69364 +69365 POINT(40.909252243209274 73.10417922683202) bank69365 +69366 POINT(41.27849816679602 73.93570286409299) bank69366 +69367 POINT(41.58425197302241 73.45281012471872) bank69367 +69368 POINT(41.39802045755924 74.44522177480974) bank69368 +69369 POINT(41.31635072459181 73.82822961034019) bank69369 +69370 POINT(40.808928112228394 73.5214400944389) bank69370 +69371 POINT(40.50225062850511 73.12451430024754) bank69371 +69372 POINT(39.9612335751711 74.01456725933384) bank69372 +69373 POINT(41.64722840033861 74.44575268944537) bank69373 +69374 POINT(39.90210406009335 74.63808731450001) bank69374 +69375 POINT(40.528608978672054 73.40973929381283) bank69375 +69376 POINT(40.54393913362761 74.72181807249936) bank69376 +69377 POINT(40.86369155797831 73.46713325798684) bank69377 +69378 POINT(40.998895671179405 74.56476222280521) bank69378 +69379 POINT(39.7949411172076 74.46714735996358) bank69379 +69380 POINT(41.57789614653085 73.40975803900297) bank69380 +69381 POINT(41.04306684030914 74.32659924244713) bank69381 +69382 POINT(40.62745349512726 73.73462821896294) bank69382 +69383 POINT(41.457120093520714 73.95701514677572) bank69383 +69384 POINT(40.39686343033975 74.47626950496402) bank69384 +69385 POINT(41.34409475233969 73.20447415279797) bank69385 +69386 POINT(40.493277511730874 73.06839279464398) bank69386 +69387 POINT(40.91193556300305 73.44258101948556) bank69387 +69388 POINT(40.62418101831682 73.04229062163206) bank69388 +69389 POINT(41.40966160957685 73.42907069725985) bank69389 +69390 POINT(40.79641410996553 73.1894631079038) bank69390 +69391 POINT(40.245062573830594 73.18649414874893) bank69391 +69392 POINT(40.59507768604087 73.8590769441105) bank69392 +69393 POINT(40.82512639300544 74.9657455234357) bank69393 +69394 POINT(41.08364351200951 73.74412009308847) bank69394 +69395 POINT(40.80075518995017 74.91013002573906) bank69395 +69396 POINT(40.77347426770867 74.5947257567483) bank69396 +69397 POINT(40.96591427061251 73.14257925765807) bank69397 +69398 POINT(39.89011666536463 74.41191268964623) bank69398 +69399 POINT(41.57473057664754 74.18689556941648) bank69399 +69400 POINT(40.59403997255861 73.68844821878767) bank69400 +69401 POINT(40.916432063830165 73.42576059397197) bank69401 +69402 POINT(39.88925593169276 74.32095930293508) bank69402 +69403 POINT(40.67595468759708 73.41079716166803) bank69403 +69404 POINT(41.245754036484364 74.85164664852105) bank69404 +69405 POINT(40.073313934675184 74.41404926721121) bank69405 +69406 POINT(40.82454929649855 74.71320759880848) bank69406 +69407 POINT(41.279943504559064 73.66056042174499) bank69407 +69408 POINT(40.956690347900114 74.9688143110089) bank69408 +69409 POINT(39.7648499646225 74.38176572931091) bank69409 +69410 POINT(40.725762860753065 74.94194343374043) bank69410 +69411 POINT(40.2894982856716 74.88383945175316) bank69411 +69412 POINT(40.06770640332563 73.40384221958985) bank69412 +69413 POINT(40.75815927160594 74.02148560116729) bank69413 +69414 POINT(40.546368108264794 73.97104443059277) bank69414 +69415 POINT(41.06338091361836 73.74527348632787) bank69415 +69416 POINT(41.095642972750824 73.02140134659005) bank69416 +69417 POINT(40.44396703932826 73.75372350987391) bank69417 +69418 POINT(40.5313250509791 73.70025391460281) bank69418 +69419 POINT(40.67398093154726 73.98079906759878) bank69419 +69420 POINT(41.238476999347974 73.33312904116694) bank69420 +69421 POINT(40.36095287489042 73.3888973596583) bank69421 +69422 POINT(41.088744034906796 74.15084597738984) bank69422 +69423 POINT(41.4724775266589 73.38655132167086) bank69423 +69424 POINT(39.7214751489396 73.77392939657713) bank69424 +69425 POINT(40.57024362865477 74.16982114054213) bank69425 +69426 POINT(40.67780392000387 74.0497046823082) bank69426 +69427 POINT(41.460655855394 74.14680502668571) bank69427 +69428 POINT(39.81895236707724 73.88128212838066) bank69428 +69429 POINT(41.003674843002266 74.61158430033943) bank69429 +69430 POINT(41.6127049293852 73.61289744418383) bank69430 +69431 POINT(39.79088723025662 74.30075806539557) bank69431 +69432 POINT(40.50076586232471 73.60520750969484) bank69432 +69433 POINT(39.714265397955565 74.56873157598841) bank69433 +69434 POINT(41.640174354559036 74.66444937031636) bank69434 +69435 POINT(40.31335593175723 74.9617554213536) bank69435 +69436 POINT(40.77840224884394 73.01789608216991) bank69436 +69437 POINT(41.253794630881714 73.7333830486957) bank69437 +69438 POINT(40.851662980561095 74.10284816608964) bank69438 +69439 POINT(40.585489504824956 73.10683861617534) bank69439 +69440 POINT(40.12696420358761 74.58017670843361) bank69440 +69441 POINT(39.76075904188527 74.77265454156016) bank69441 +69442 POINT(40.66071777400728 74.39659181766947) bank69442 +69443 POINT(40.77871560170708 73.96451618667835) bank69443 +69444 POINT(40.49244910805401 74.37635677304597) bank69444 +69445 POINT(39.887631360396654 73.14042667271802) bank69445 +69446 POINT(40.289697573702 73.83814515661903) bank69446 +69447 POINT(40.99085043618774 73.96601212254026) bank69447 +69448 POINT(39.78569845633422 74.96566809006158) bank69448 +69449 POINT(41.01940413563826 73.08940906939637) bank69449 +69450 POINT(41.66166255598069 74.03297010155079) bank69450 +69451 POINT(41.628467975955324 73.71413881681401) bank69451 +69452 POINT(41.21362102543081 73.62506932653389) bank69452 +69453 POINT(41.44098459815868 73.6899507054132) bank69453 +69454 POINT(41.51609369259065 73.32781260502236) bank69454 +69455 POINT(41.528288748979165 74.43180781677208) bank69455 +69456 POINT(40.08521891548863 74.34080585080999) bank69456 +69457 POINT(41.0277074384443 74.37640946495662) bank69457 +69458 POINT(41.41046072255616 73.87619425069849) bank69458 +69459 POINT(41.21993628775261 74.81432160421619) bank69459 +69460 POINT(40.479739013793605 74.86923408451047) bank69460 +69461 POINT(41.54336901255819 74.75991774248855) bank69461 +69462 POINT(40.377751720945476 73.15942644597898) bank69462 +69463 POINT(40.183947513455266 74.59838539335445) bank69463 +69464 POINT(41.29038687329245 74.35951456148098) bank69464 +69465 POINT(39.81866227751308 73.52528439334739) bank69465 +69466 POINT(41.57291282613876 74.5694171330495) bank69466 +69467 POINT(41.44033896164345 74.4255998801934) bank69467 +69468 POINT(41.6643202055181 73.90708167944774) bank69468 +69469 POINT(41.138060722508754 73.54844342082906) bank69469 +69470 POINT(40.153329716444105 73.32479712798057) bank69470 +69471 POINT(40.48907732347007 74.47108215045606) bank69471 +69472 POINT(40.033015055626045 74.19458874136592) bank69472 +69473 POINT(39.87419660706056 74.53029201041413) bank69473 +69474 POINT(39.71986941337322 73.96067779345945) bank69474 +69475 POINT(40.071847191640074 74.01149943871383) bank69475 +69476 POINT(41.02413705409754 74.2709334289364) bank69476 +69477 POINT(40.28391269322145 73.30006473191402) bank69477 +69478 POINT(41.06905387179668 73.214702655258) bank69478 +69479 POINT(40.69627863588248 73.70324759281127) bank69479 +69480 POINT(40.23300552063393 73.07038948029555) bank69480 +69481 POINT(40.60108446354573 73.455329454848) bank69481 +69482 POINT(40.27372541169431 74.74370170730893) bank69482 +69483 POINT(40.00209074803892 74.4132835413299) bank69483 +69484 POINT(39.74273627704923 73.47718289651112) bank69484 +69485 POINT(41.59927120999352 73.99584675110385) bank69485 +69486 POINT(40.23477276890373 74.85393905345164) bank69486 +69487 POINT(40.2912884510224 73.77273334888318) bank69487 +69488 POINT(40.56571823338324 73.51433725432105) bank69488 +69489 POINT(40.92582326663744 73.15319350112664) bank69489 +69490 POINT(40.82221516041527 74.74028083251278) bank69490 +69491 POINT(40.02010983889052 74.85174777665087) bank69491 +69492 POINT(41.31950015382945 73.98993528433925) bank69492 +69493 POINT(40.01290716354835 73.10293108425837) bank69493 +69494 POINT(40.49343476583065 74.90883452250891) bank69494 +69495 POINT(39.803681997699165 73.93949336967721) bank69495 +69496 POINT(40.40903055198144 73.04934474194536) bank69496 +69497 POINT(41.55423791465348 73.33124968267491) bank69497 +69498 POINT(41.68355698624086 74.58844209241711) bank69498 +69499 POINT(41.38730643730766 73.46550040332174) bank69499 +69500 POINT(40.92097697465298 74.84588940143207) bank69500 +69501 POINT(40.69493871694171 73.16933799219342) bank69501 +69502 POINT(40.82314178164064 73.32654265104588) bank69502 +69503 POINT(40.098342561944975 74.1524886291093) bank69503 +69504 POINT(40.514065193669616 74.97628814072324) bank69504 +69505 POINT(40.78834247175628 73.59150642056733) bank69505 +69506 POINT(40.64066139188257 74.14115657239128) bank69506 +69507 POINT(40.96860711558483 74.99190253374051) bank69507 +69508 POINT(39.94271302283815 73.1762809729756) bank69508 +69509 POINT(40.612467155879855 74.92517553533303) bank69509 +69510 POINT(41.11897879495045 73.66895485011356) bank69510 +69511 POINT(41.537974355969304 73.7477415941774) bank69511 +69512 POINT(40.60278341858098 74.35461974900419) bank69512 +69513 POINT(40.126698443017474 74.26128337187393) bank69513 +69514 POINT(40.08407795784535 74.97984664015065) bank69514 +69515 POINT(41.06875329182172 74.3098290356598) bank69515 +69516 POINT(40.38398769702374 73.61728147656262) bank69516 +69517 POINT(41.27241204537795 73.6365629892694) bank69517 +69518 POINT(41.120521498894654 73.13699524674931) bank69518 +69519 POINT(40.679164911806154 74.68667392233915) bank69519 +69520 POINT(41.33788375746123 73.48148599979902) bank69520 +69521 POINT(40.46143721230702 74.8878084203525) bank69521 +69522 POINT(40.612125041071494 73.50451695824744) bank69522 +69523 POINT(41.65128801372097 74.44522962738947) bank69523 +69524 POINT(39.72111318745533 74.5864274055955) bank69524 +69525 POINT(41.4265950118359 73.57323232991973) bank69525 +69526 POINT(41.11825041500304 73.45129387216379) bank69526 +69527 POINT(41.70649492703278 73.05623584087424) bank69527 +69528 POINT(39.879047723477214 74.84586450777785) bank69528 +69529 POINT(40.07107566056476 74.90444750739704) bank69529 +69530 POINT(41.009351312046654 73.50579488100216) bank69530 +69531 POINT(40.94190145078066 73.52145181884177) bank69531 +69532 POINT(40.19225050878918 74.28283710627386) bank69532 +69533 POINT(41.15077793647209 74.10095000422821) bank69533 +69534 POINT(40.48139057176166 74.61982292501497) bank69534 +69535 POINT(41.4028008211202 73.73999440042142) bank69535 +69536 POINT(41.07843589481418 74.9253039691169) bank69536 +69537 POINT(41.26237641773054 74.8166937332343) bank69537 +69538 POINT(40.55528775930566 73.30366108885119) bank69538 +69539 POINT(40.252842434937286 74.81067218628243) bank69539 +69540 POINT(41.1369131392579 74.16077578558577) bank69540 +69541 POINT(40.35081959807388 73.47123677548726) bank69541 +69542 POINT(41.57653390428144 74.77796973130805) bank69542 +69543 POINT(41.57792465588142 74.9456675285976) bank69543 +69544 POINT(40.18372664556406 73.66689484154806) bank69544 +69545 POINT(41.59882261448408 73.98285862054392) bank69545 +69546 POINT(40.50907815292846 73.8489281299917) bank69546 +69547 POINT(41.03594478409364 74.26365022114857) bank69547 +69548 POINT(40.648032329802355 74.42989354386371) bank69548 +69549 POINT(41.46417684941792 74.96538776244071) bank69549 +69550 POINT(40.405170624140986 74.30194307278136) bank69550 +69551 POINT(40.75699861488084 73.10486281875022) bank69551 +69552 POINT(41.17538404552863 74.705898125253) bank69552 +69553 POINT(41.0987591702664 74.4048252575762) bank69553 +69554 POINT(40.58037781561949 74.007529013437) bank69554 +69555 POINT(40.98285014119453 74.22677840846397) bank69555 +69556 POINT(40.53907534560707 73.42341815196808) bank69556 +69557 POINT(41.21766551880982 73.0871501139787) bank69557 +69558 POINT(41.11351814662365 74.99192100568621) bank69558 +69559 POINT(41.03196809011855 73.62902981842281) bank69559 +69560 POINT(41.71135832023906 73.52375783575361) bank69560 +69561 POINT(40.817252234067766 74.99624920275285) bank69561 +69562 POINT(40.76462582095714 74.32654976340373) bank69562 +69563 POINT(41.247178515650496 74.32837659237607) bank69563 +69564 POINT(40.4198622697779 74.12026963324325) bank69564 +69565 POINT(40.88728075353449 74.14847038682582) bank69565 +69566 POINT(41.582817045741066 73.78891019020145) bank69566 +69567 POINT(40.958474448564075 73.81495899874487) bank69567 +69568 POINT(40.34707613791429 73.93641273152947) bank69568 +69569 POINT(41.133444496222836 74.6274118747515) bank69569 +69570 POINT(40.512016079350154 73.58633125348355) bank69570 +69571 POINT(40.638567469638375 74.23361938139477) bank69571 +69572 POINT(40.83576034112611 73.36339258676543) bank69572 +69573 POINT(40.00238461179609 73.1749296850181) bank69573 +69574 POINT(39.950170029246145 73.74499093223285) bank69574 +69575 POINT(41.427977468730965 74.33381640487916) bank69575 +69576 POINT(40.1742837933929 73.1029465339133) bank69576 +69577 POINT(40.9761839025112 73.23187700854217) bank69577 +69578 POINT(40.578548315143536 73.06307569613433) bank69578 +69579 POINT(40.933366331893716 74.94374885934258) bank69579 +69580 POINT(41.05241894157854 73.92264715203558) bank69580 +69581 POINT(40.7617030975965 74.95578475579941) bank69581 +69582 POINT(40.506172839691125 73.12745842865127) bank69582 +69583 POINT(39.96551087955535 74.70540542966788) bank69583 +69584 POINT(40.28842017228554 73.51569704844982) bank69584 +69585 POINT(40.08539797873487 73.07464944167738) bank69585 +69586 POINT(39.98700866579671 74.99102320373714) bank69586 +69587 POINT(40.17878941461657 73.85128153317496) bank69587 +69588 POINT(41.22326768145114 73.18305033213919) bank69588 +69589 POINT(39.73604943083893 73.87429574688643) bank69589 +69590 POINT(39.979044782903976 73.40366372733538) bank69590 +69591 POINT(41.36670080617837 73.85630341152414) bank69591 +69592 POINT(40.59327641601247 73.92951740927357) bank69592 +69593 POINT(40.611687220718615 73.10355502886922) bank69593 +69594 POINT(39.87005900694291 74.24860267174235) bank69594 +69595 POINT(39.825320984667954 73.449017169768) bank69595 +69596 POINT(40.165821742703756 74.99564844213187) bank69596 +69597 POINT(39.75416977008981 73.91879012281397) bank69597 +69598 POINT(40.17774233085562 73.02488986784839) bank69598 +69599 POINT(41.014520427884094 73.10963303837455) bank69599 +69600 POINT(40.36105699278095 73.55738247782021) bank69600 +69601 POINT(40.27029203802188 74.61522870882685) bank69601 +69602 POINT(40.54394728562219 73.34935185331297) bank69602 +69603 POINT(40.19818056334761 74.31425772008045) bank69603 +69604 POINT(41.30093597430254 74.17639890585123) bank69604 +69605 POINT(40.87140761608065 74.87386163790745) bank69605 +69606 POINT(40.31518243049314 74.46845085912301) bank69606 +69607 POINT(40.40899286294591 74.39106356413387) bank69607 +69608 POINT(40.67419959017469 73.49123561860945) bank69608 +69609 POINT(39.88350138540929 74.19921275889982) bank69609 +69610 POINT(39.84487673449763 74.66810207622892) bank69610 +69611 POINT(40.23782262487795 73.72249122044298) bank69611 +69612 POINT(41.11939990533719 73.13100265405546) bank69612 +69613 POINT(40.01901677227988 73.33921631091984) bank69613 +69614 POINT(40.72782261157016 74.34401312252743) bank69614 +69615 POINT(41.5498216048868 74.48918545356145) bank69615 +69616 POINT(41.3830359136821 73.37163500011007) bank69616 +69617 POINT(41.3706199353408 74.74692300856502) bank69617 +69618 POINT(41.147454604466915 74.00172412907482) bank69618 +69619 POINT(41.701166912931846 74.33163544602726) bank69619 +69620 POINT(40.35288031562675 74.89866250337535) bank69620 +69621 POINT(41.66809873763848 73.11695326366109) bank69621 +69622 POINT(40.94323105986378 73.02475243900273) bank69622 +69623 POINT(40.158606221775386 74.53214907301111) bank69623 +69624 POINT(39.79657836977985 73.76733776552221) bank69624 +69625 POINT(40.42857537975794 73.25423299736967) bank69625 +69626 POINT(40.454082435040675 74.78469390117715) bank69626 +69627 POINT(40.89715083961139 73.29999729444611) bank69627 +69628 POINT(40.188510592393904 73.63436224160402) bank69628 +69629 POINT(40.023668109029764 74.03511897127254) bank69629 +69630 POINT(39.80159396136825 74.22221068736044) bank69630 +69631 POINT(40.78041430669037 74.48191189817982) bank69631 +69632 POINT(40.6171076922222 73.69009170895328) bank69632 +69633 POINT(40.319597172378614 74.40269085841663) bank69633 +69634 POINT(39.91621146682224 73.5852489271052) bank69634 +69635 POINT(41.644644362157806 74.92960599443906) bank69635 +69636 POINT(40.82291728800059 74.447802605156) bank69636 +69637 POINT(40.37570359932539 73.70816861036955) bank69637 +69638 POINT(40.431082687759115 73.82698683227017) bank69638 +69639 POINT(40.444318445059494 73.5904976638512) bank69639 +69640 POINT(40.62109465677673 73.21302327174547) bank69640 +69641 POINT(40.29593140891978 74.76905031150909) bank69641 +69642 POINT(41.0837173919368 74.40986367858386) bank69642 +69643 POINT(41.475546929834174 73.4213231840901) bank69643 +69644 POINT(40.97800184486282 73.08202240940814) bank69644 +69645 POINT(41.33751846472521 73.95978778922412) bank69645 +69646 POINT(40.77371891495031 73.21857382624655) bank69646 +69647 POINT(40.74970277105209 73.55041889419377) bank69647 +69648 POINT(41.18568294538168 73.45814644765717) bank69648 +69649 POINT(41.695414721167325 74.40282124171877) bank69649 +69650 POINT(40.171552904170404 73.6971701469274) bank69650 +69651 POINT(40.13081228348731 73.18798657726398) bank69651 +69652 POINT(40.456305146335666 73.72170948464169) bank69652 +69653 POINT(40.072691577974894 73.03341254019121) bank69653 +69654 POINT(40.91045412251201 74.16583370239326) bank69654 +69655 POINT(40.024789785892985 73.7175405933047) bank69655 +69656 POINT(39.94991389787004 73.25303302078068) bank69656 +69657 POINT(40.786039939195454 73.11225066700645) bank69657 +69658 POINT(40.02007434577257 73.10240436768603) bank69658 +69659 POINT(41.497448581364615 73.74312751836092) bank69659 +69660 POINT(41.179557633330504 73.22902644232774) bank69660 +69661 POINT(41.49590724515202 74.23080816663742) bank69661 +69662 POINT(41.074064055110746 75.0050033477951) bank69662 +69663 POINT(41.463640283532705 73.41955589814216) bank69663 +69664 POINT(41.127569666476475 73.94967220720699) bank69664 +69665 POINT(40.45599440083497 73.1922313422085) bank69665 +69666 POINT(39.95792973057507 74.89694917772569) bank69666 +69667 POINT(40.99579213294796 73.14183941748574) bank69667 +69668 POINT(40.75945658489315 73.17712631926354) bank69668 +69669 POINT(40.35737085349009 73.37813959412864) bank69669 +69670 POINT(40.288340021542886 74.2262306799832) bank69670 +69671 POINT(41.49885009362593 74.04304387931984) bank69671 +69672 POINT(39.88743107578886 73.05510234635246) bank69672 +69673 POINT(41.65026683970341 74.34596954809811) bank69673 +69674 POINT(40.953536266268415 73.2796793300657) bank69674 +69675 POINT(40.90346626427037 73.4981899959362) bank69675 +69676 POINT(40.57774103464676 73.13525674771894) bank69676 +69677 POINT(40.40739706114074 73.58745749571783) bank69677 +69678 POINT(41.60753251302278 74.69817814223866) bank69678 +69679 POINT(41.36451676908279 73.32812860008127) bank69679 +69680 POINT(40.445823953504245 73.91428380760833) bank69680 +69681 POINT(40.11400040921248 74.42144925012741) bank69681 +69682 POINT(40.71020471301173 74.9136097929105) bank69682 +69683 POINT(40.59131008901143 73.40582192548459) bank69683 +69684 POINT(40.280493254950215 73.81179429815028) bank69684 +69685 POINT(40.15279114883778 73.90644535376599) bank69685 +69686 POINT(41.638608996732636 73.0928554682783) bank69686 +69687 POINT(41.48919404863669 74.17324234571493) bank69687 +69688 POINT(41.02607150842286 73.97717888982116) bank69688 +69689 POINT(40.38128872409023 74.21909288103238) bank69689 +69690 POINT(40.381306240790515 74.40207723805499) bank69690 +69691 POINT(40.85056565412353 74.25342062768664) bank69691 +69692 POINT(41.58518177026344 74.9527567035851) bank69692 +69693 POINT(41.51463924727322 74.3043471068651) bank69693 +69694 POINT(41.01999551912121 73.75798007590006) bank69694 +69695 POINT(40.54902898384839 74.01989627010863) bank69695 +69696 POINT(40.79871305979136 73.08346909091576) bank69696 +69697 POINT(41.42876950098112 74.83214696702065) bank69697 +69698 POINT(40.84128650401083 73.27283186433706) bank69698 +69699 POINT(41.45275993651435 73.51718062825657) bank69699 +69700 POINT(40.94403103841598 74.83125149287596) bank69700 +69701 POINT(40.767038711501435 73.1748483141825) bank69701 +69702 POINT(40.08679963716683 73.30923927181196) bank69702 +69703 POINT(41.15741517842817 73.2177035560934) bank69703 +69704 POINT(40.62642144636716 73.14561260764191) bank69704 +69705 POINT(40.69467384762831 73.58589723480387) bank69705 +69706 POINT(40.98684189446843 73.68770818963193) bank69706 +69707 POINT(41.21732096123298 74.48610923123326) bank69707 +69708 POINT(40.14805535696814 74.84766398177382) bank69708 +69709 POINT(41.37616778239319 74.91272782519259) bank69709 +69710 POINT(40.091806605727534 73.75313081782667) bank69710 +69711 POINT(39.99108632232367 73.86314364668584) bank69711 +69712 POINT(41.03499138785858 73.977787746352) bank69712 +69713 POINT(40.520727946012734 74.288193872448) bank69713 +69714 POINT(40.06894977991725 73.95785447042591) bank69714 +69715 POINT(39.71767646578974 73.47914434328753) bank69715 +69716 POINT(40.40053588637708 74.16068642313665) bank69716 +69717 POINT(40.51972155125386 74.7813654590534) bank69717 +69718 POINT(40.95381327103091 73.4853842108692) bank69718 +69719 POINT(40.28080139188832 74.57817779585174) bank69719 +69720 POINT(40.78051422110667 74.1529272023967) bank69720 +69721 POINT(41.62573473481787 74.54392526157592) bank69721 +69722 POINT(40.46820491115904 74.79277344379476) bank69722 +69723 POINT(40.20311254497157 73.50014331688506) bank69723 +69724 POINT(39.88689451255179 74.76056433960079) bank69724 +69725 POINT(40.363347835477455 73.62932436775243) bank69725 +69726 POINT(40.385213553573294 73.12299396561154) bank69726 +69727 POINT(41.62684310210285 74.00862325066106) bank69727 +69728 POINT(41.65828008158159 73.32657540359989) bank69728 +69729 POINT(40.58383682223393 74.83743177751906) bank69729 +69730 POINT(41.46204572142331 73.69265166709214) bank69730 +69731 POINT(39.91522064212001 73.7000683324222) bank69731 +69732 POINT(39.97117579819814 73.65333207537802) bank69732 +69733 POINT(40.81415035813126 74.75451102380919) bank69733 +69734 POINT(41.302386682373196 73.3602744757274) bank69734 +69735 POINT(40.736203434963684 74.74794350556995) bank69735 +69736 POINT(40.99268316359375 73.74442979544015) bank69736 +69737 POINT(40.436781867932964 74.09015257037176) bank69737 +69738 POINT(41.285270595010445 74.14195873892207) bank69738 +69739 POINT(40.18159371049525 73.32167436444647) bank69739 +69740 POINT(40.56776072309079 73.81114323371156) bank69740 +69741 POINT(40.27712396369703 73.68515696628769) bank69741 +69742 POINT(40.96909984142431 73.47467084698056) bank69742 +69743 POINT(40.7045596955367 74.37465203116436) bank69743 +69744 POINT(40.89539841309027 73.56629190426504) bank69744 +69745 POINT(40.06973127378549 74.5342434875484) bank69745 +69746 POINT(41.66398401685462 74.86934978968385) bank69746 +69747 POINT(40.39408165264522 73.55847804924616) bank69747 +69748 POINT(40.63645850070649 73.18593988347068) bank69748 +69749 POINT(41.11763236838814 74.70422797106409) bank69749 +69750 POINT(40.18193085039963 74.20967334725557) bank69750 +69751 POINT(41.62376580369805 73.65041339793477) bank69751 +69752 POINT(41.08447534689918 74.3185754694842) bank69752 +69753 POINT(40.33392942339008 73.9101282301877) bank69753 +69754 POINT(41.47536279306565 73.13472138478136) bank69754 +69755 POINT(39.90714844460039 73.51472553955361) bank69755 +69756 POINT(41.42365028658207 74.19892750674877) bank69756 +69757 POINT(41.07953958363434 74.51743167420128) bank69757 +69758 POINT(39.99261494602779 73.55752638178066) bank69758 +69759 POINT(41.04406000826748 73.85071736274513) bank69759 +69760 POINT(40.93186751252417 73.76129262339643) bank69760 +69761 POINT(40.22948825800335 74.18082580025143) bank69761 +69762 POINT(41.39853542888457 73.26898531572597) bank69762 +69763 POINT(41.049266679618015 74.4046104300649) bank69763 +69764 POINT(41.665890238577084 74.96746632173453) bank69764 +69765 POINT(40.497108226918044 74.27760984889532) bank69765 +69766 POINT(40.07631899815958 73.03226100091707) bank69766 +69767 POINT(40.7466144867165 74.49977175653837) bank69767 +69768 POINT(41.319971340726354 73.71528804503801) bank69768 +69769 POINT(41.321581184490974 73.61708674150975) bank69769 +69770 POINT(40.671912852706605 74.60911959384798) bank69770 +69771 POINT(41.14030500183006 74.77004543255211) bank69771 +69772 POINT(39.972234213260236 74.21315683639791) bank69772 +69773 POINT(39.897330175907925 74.8562100225061) bank69773 +69774 POINT(40.00440912683121 73.46184390911571) bank69774 +69775 POINT(40.126692716742724 73.69511752950173) bank69775 +69776 POINT(41.66419101759763 74.20073936570705) bank69776 +69777 POINT(39.74308379001461 73.55097046200656) bank69777 +69778 POINT(40.483247350906645 73.49356572687701) bank69778 +69779 POINT(40.599019137817436 73.23927307612693) bank69779 +69780 POINT(39.851237547857146 73.24023951822141) bank69780 +69781 POINT(39.77469864224041 74.43987804506278) bank69781 +69782 POINT(40.74095326094408 74.43524950846637) bank69782 +69783 POINT(40.09603149826422 74.26987030881125) bank69783 +69784 POINT(41.42749759721527 73.77813543753949) bank69784 +69785 POINT(40.50080607502007 73.43750968603428) bank69785 +69786 POINT(39.971177662952904 73.82209546646197) bank69786 +69787 POINT(41.345422554324635 74.58047199654736) bank69787 +69788 POINT(41.498561561209755 74.71325513424499) bank69788 +69789 POINT(40.47166404243183 73.32450442401223) bank69789 +69790 POINT(40.04342929262823 73.40750383832355) bank69790 +69791 POINT(40.890351667213714 74.31481520047394) bank69791 +69792 POINT(41.140943403922535 73.03015771489487) bank69792 +69793 POINT(41.59340856584734 73.23331552930593) bank69793 +69794 POINT(40.142860444858904 74.28628616713831) bank69794 +69795 POINT(41.461213218018436 74.75242722206112) bank69795 +69796 POINT(41.207349769172275 74.98425694108529) bank69796 +69797 POINT(41.270926261823476 73.94752656718877) bank69797 +69798 POINT(40.65815423382621 73.49673882027244) bank69798 +69799 POINT(39.97216420573129 74.09196503698455) bank69799 +69800 POINT(41.7065187117833 74.65588416889724) bank69800 +69801 POINT(40.577015047885894 74.54805477091165) bank69801 +69802 POINT(40.328056747324396 74.98797337969951) bank69802 +69803 POINT(39.83156219451861 74.25412879474568) bank69803 +69804 POINT(41.40414608223728 74.15724903423896) bank69804 +69805 POINT(40.784254293790504 73.31479836062297) bank69805 +69806 POINT(39.78461268027603 74.93081930844062) bank69806 +69807 POINT(41.20452288431533 73.03015070253366) bank69807 +69808 POINT(39.96964311215108 74.31299627143729) bank69808 +69809 POINT(41.36992463075505 73.95916774396146) bank69809 +69810 POINT(40.181842078991735 74.20164653859163) bank69810 +69811 POINT(40.14333309354047 73.66615339717764) bank69811 +69812 POINT(40.32051498442238 74.51858496130633) bank69812 +69813 POINT(40.3536163633173 74.93875712044932) bank69813 +69814 POINT(41.25579640830992 73.13816978412935) bank69814 +69815 POINT(40.66141046221292 74.07366290533594) bank69815 +69816 POINT(41.152938367563905 73.17650388225539) bank69816 +69817 POINT(39.85444450203359 73.5590808959928) bank69817 +69818 POINT(41.075518705610584 75.00489896734634) bank69818 +69819 POINT(40.28352667630898 74.10316597761788) bank69819 +69820 POINT(40.32899855153426 73.0493305348501) bank69820 +69821 POINT(39.75262930692623 74.37100193552203) bank69821 +69822 POINT(41.424419163514365 74.69885272571237) bank69822 +69823 POINT(40.850791639452204 74.25973331998192) bank69823 +69824 POINT(40.01400225807665 74.90544900057625) bank69824 +69825 POINT(40.879701454122184 73.03809494025684) bank69825 +69826 POINT(41.41748691307998 74.66396389383166) bank69826 +69827 POINT(40.04267654009171 73.67529903219064) bank69827 +69828 POINT(40.16067777468285 74.30951139806898) bank69828 +69829 POINT(40.27223939744144 74.51325040701609) bank69829 +69830 POINT(40.649384352159416 74.42965783512068) bank69830 +69831 POINT(40.23875116958022 73.89799883533234) bank69831 +69832 POINT(40.59040365096428 74.70616331688873) bank69832 +69833 POINT(39.727118610507006 74.17613831913414) bank69833 +69834 POINT(40.03737630960669 74.9134421566128) bank69834 +69835 POINT(40.36366264599306 73.98203831821651) bank69835 +69836 POINT(39.83704745403383 73.19229384262994) bank69836 +69837 POINT(41.44071987807539 73.85700001818206) bank69837 +69838 POINT(41.56397436976646 74.77897545297255) bank69838 +69839 POINT(41.059623108344866 73.82860440824362) bank69839 +69840 POINT(40.1946377016309 74.54432900071227) bank69840 +69841 POINT(40.31139519985778 74.64995574260008) bank69841 +69842 POINT(40.29342778980039 74.57300163139323) bank69842 +69843 POINT(40.82322868467156 74.04062143077059) bank69843 +69844 POINT(41.23980111497182 73.1477936478771) bank69844 +69845 POINT(40.30025519003946 74.90188966968454) bank69845 +69846 POINT(40.16317679822381 74.00936155007422) bank69846 +69847 POINT(39.99361386549447 73.7703656498521) bank69847 +69848 POINT(41.6452221374966 73.71539535397818) bank69848 +69849 POINT(40.58938223476386 74.80985827164184) bank69849 +69850 POINT(40.48528160708681 73.45458393480678) bank69850 +69851 POINT(39.86830154942846 74.47751827025945) bank69851 +69852 POINT(40.17344288542589 74.82055796637003) bank69852 +69853 POINT(40.66453619893498 73.96969640551974) bank69853 +69854 POINT(39.82440583802767 73.01637609331955) bank69854 +69855 POINT(40.0401572449148 74.94575464666094) bank69855 +69856 POINT(40.873132955693656 74.17180344841312) bank69856 +69857 POINT(41.12237084421086 74.60029596230956) bank69857 +69858 POINT(40.79468189400824 73.91523318726577) bank69858 +69859 POINT(41.2767757825857 73.91857392987266) bank69859 +69860 POINT(41.173269573083566 74.58182176961597) bank69860 +69861 POINT(40.077555273346185 74.01632179446779) bank69861 +69862 POINT(39.81667753750459 73.8024150268424) bank69862 +69863 POINT(40.498103132073346 74.18959864240614) bank69863 +69864 POINT(41.375013861466215 73.63524909308576) bank69864 +69865 POINT(41.348883931496594 73.0223074080868) bank69865 +69866 POINT(40.582292113486666 74.58591975609234) bank69866 +69867 POINT(40.79140178331276 74.47052733440512) bank69867 +69868 POINT(40.97072353106879 73.86055196389485) bank69868 +69869 POINT(39.72918527000775 74.91080909335975) bank69869 +69870 POINT(39.757676504660644 74.81531263092795) bank69870 +69871 POINT(40.824335258248354 73.41831801712961) bank69871 +69872 POINT(40.24689606824745 74.41590718448063) bank69872 +69873 POINT(40.0578466000521 73.42165168961628) bank69873 +69874 POINT(39.95530918623831 73.8042770058391) bank69874 +69875 POINT(41.035412624446735 73.73441100291069) bank69875 +69876 POINT(40.753867095123425 74.83576843805247) bank69876 +69877 POINT(39.98070028940033 73.30864638762354) bank69877 +69878 POINT(40.242956533152714 74.2730841464437) bank69878 +69879 POINT(41.198189217678 73.18843400228212) bank69879 +69880 POINT(41.31036473316553 73.87511909177381) bank69880 +69881 POINT(40.53590688535488 74.83538074625801) bank69881 +69882 POINT(40.94243988769631 73.3968396831122) bank69882 +69883 POINT(41.082505610151266 73.43728641249575) bank69883 +69884 POINT(40.75262038235821 74.63796530341759) bank69884 +69885 POINT(41.04864417260366 73.33034075748972) bank69885 +69886 POINT(40.63815115749518 73.28203153748467) bank69886 +69887 POINT(40.51895728929744 74.78440965001631) bank69887 +69888 POINT(41.09298335875006 73.46745642529345) bank69888 +69889 POINT(40.668387399435844 74.76293059957514) bank69889 +69890 POINT(41.340573517474965 74.80971181310312) bank69890 +69891 POINT(40.65640556588326 73.7412268690452) bank69891 +69892 POINT(41.582455500262284 73.06942063241108) bank69892 +69893 POINT(41.24674750165524 74.67051080818582) bank69893 +69894 POINT(40.03757389812878 73.48107207269031) bank69894 +69895 POINT(40.78472388412727 74.32775295777307) bank69895 +69896 POINT(41.26701451624509 73.92295038054145) bank69896 +69897 POINT(41.25009899629012 73.65104378702826) bank69897 +69898 POINT(40.95168978935075 74.34071507773537) bank69898 +69899 POINT(40.33477071109345 74.39439342925682) bank69899 +69900 POINT(40.856793045345434 74.69130868658814) bank69900 +69901 POINT(40.15904104401162 73.85060970776213) bank69901 +69902 POINT(40.446940755078245 73.9027435800236) bank69902 +69903 POINT(39.77701605241761 73.20858660105365) bank69903 +69904 POINT(40.76235881661832 74.29707382398351) bank69904 +69905 POINT(40.90584482964178 74.47622800069303) bank69905 +69906 POINT(41.22620836554907 73.22900636507133) bank69906 +69907 POINT(40.47425504324095 73.87548520803263) bank69907 +69908 POINT(41.040317458750394 73.75255983565096) bank69908 +69909 POINT(40.75594961697171 74.79832543041769) bank69909 +69910 POINT(39.84409444622816 73.89579269823284) bank69910 +69911 POINT(39.94942152238777 74.27230159749283) bank69911 +69912 POINT(41.46298628769485 74.658209623348) bank69912 +69913 POINT(41.67011015723547 74.43048199625818) bank69913 +69914 POINT(41.46188482756807 74.48843185769083) bank69914 +69915 POINT(40.47349239238578 73.9888026793478) bank69915 +69916 POINT(41.19602001206684 73.47960538037611) bank69916 +69917 POINT(40.38196942940478 74.64766178026781) bank69917 +69918 POINT(39.84594472441996 74.56255764776135) bank69918 +69919 POINT(41.023886618100036 74.46508453950445) bank69919 +69920 POINT(40.88619116829053 74.87483976206853) bank69920 +69921 POINT(41.056600345740144 74.3871456961395) bank69921 +69922 POINT(41.61725722160377 74.3597292644141) bank69922 +69923 POINT(41.452117513352476 73.16518117998554) bank69923 +69924 POINT(39.902324268723035 74.76221368590478) bank69924 +69925 POINT(41.662620885659905 73.78039181483042) bank69925 +69926 POINT(39.88500379836646 74.41415960626051) bank69926 +69927 POINT(40.98308881041297 74.4792410439412) bank69927 +69928 POINT(39.787504322116604 74.98478922031758) bank69928 +69929 POINT(40.20344343209051 74.77800587413846) bank69929 +69930 POINT(40.16451319581849 74.48108670136988) bank69930 +69931 POINT(41.18762061865284 74.14231796074202) bank69931 +69932 POINT(40.11968055393099 74.01409263417304) bank69932 +69933 POINT(41.32529879007144 73.04047688215883) bank69933 +69934 POINT(40.10480665961555 73.84964776030593) bank69934 +69935 POINT(39.816008817958895 73.20974916312996) bank69935 +69936 POINT(41.222857906940554 73.23926615437573) bank69936 +69937 POINT(41.358798683756426 73.15569667151675) bank69937 +69938 POINT(40.08301091689194 73.6939151146675) bank69938 +69939 POINT(40.0146759842311 74.99442509371441) bank69939 +69940 POINT(40.725347562263 73.50314688420565) bank69940 +69941 POINT(40.56428010109309 74.67052262810518) bank69941 +69942 POINT(40.70439029508499 74.72814483108864) bank69942 +69943 POINT(40.061855519753394 74.65208344988481) bank69943 +69944 POINT(40.193347964390014 74.48656883663598) bank69944 +69945 POINT(41.46342743151954 74.3866598092726) bank69945 +69946 POINT(40.312918650137355 73.65057947456135) bank69946 +69947 POINT(41.351050431434395 74.70136743637957) bank69947 +69948 POINT(41.3172966957792 73.32589161348409) bank69948 +69949 POINT(41.2890290656737 73.05760927236697) bank69949 +69950 POINT(40.93211638603284 74.03896635298715) bank69950 +69951 POINT(41.67519695960042 74.29564163841597) bank69951 +69952 POINT(40.158258845105415 74.16491074388068) bank69952 +69953 POINT(41.61654141583629 74.40555750252494) bank69953 +69954 POINT(40.68177671167051 73.81835158861516) bank69954 +69955 POINT(40.72521939469189 73.0291331713032) bank69955 +69956 POINT(40.01191549543585 74.16798485387757) bank69956 +69957 POINT(40.432828509888445 74.36957152481587) bank69957 +69958 POINT(40.599022270418715 73.2538377960034) bank69958 +69959 POINT(40.25194245645341 73.88070882626748) bank69959 +69960 POINT(39.84638218965452 74.15481391295094) bank69960 +69961 POINT(39.95841993487391 73.53067140798235) bank69961 +69962 POINT(40.86798324288337 74.18103159944103) bank69962 +69963 POINT(40.81196053630883 73.77270604806606) bank69963 +69964 POINT(40.85207588195237 74.56134921935669) bank69964 +69965 POINT(41.59682321197262 73.94179522173822) bank69965 +69966 POINT(40.109781239891 74.91880627873932) bank69966 +69967 POINT(40.23823988129374 73.35285235649026) bank69967 +69968 POINT(40.85980622605053 74.7928830099454) bank69968 +69969 POINT(41.23394943126188 73.72318049968374) bank69969 +69970 POINT(39.76730750300378 74.76039813982497) bank69970 +69971 POINT(40.58072277865818 73.03232479728673) bank69971 +69972 POINT(41.1616591581729 74.08422896170144) bank69972 +69973 POINT(40.105594922805054 73.42934662842461) bank69973 +69974 POINT(41.21106217948789 73.6040369517884) bank69974 +69975 POINT(41.21325515913996 74.57987223667688) bank69975 +69976 POINT(41.439934885618875 74.25398841445308) bank69976 +69977 POINT(40.86124484845859 74.6092328721018) bank69977 +69978 POINT(39.91888780998042 73.37418703209384) bank69978 +69979 POINT(41.0313092600107 73.60296019651005) bank69979 +69980 POINT(40.49816868967213 74.97681774070502) bank69980 +69981 POINT(40.14245279942101 74.211959328011) bank69981 +69982 POINT(40.97328811944868 73.06557430323576) bank69982 +69983 POINT(41.30898009514831 73.1777074696147) bank69983 +69984 POINT(40.650547398569444 74.0216308613618) bank69984 +69985 POINT(41.613836331222494 73.21422332150946) bank69985 +69986 POINT(41.307524742222704 73.22403500145407) bank69986 +69987 POINT(41.33668340404277 73.9961976750245) bank69987 +69988 POINT(41.116359764272545 73.90423307807924) bank69988 +69989 POINT(40.1803095495958 73.17261763282039) bank69989 +69990 POINT(40.4509040569029 74.0018280215943) bank69990 +69991 POINT(40.034682573297054 73.30749085583489) bank69991 +69992 POINT(40.17128877501108 73.61043841277244) bank69992 +69993 POINT(40.016823327998814 74.25413629608302) bank69993 +69994 POINT(40.66930422611468 73.69221068195846) bank69994 +69995 POINT(40.75718734487427 73.24447134720579) bank69995 +69996 POINT(39.86005642093937 74.62341455463286) bank69996 +69997 POINT(40.072962472770726 74.92932273963407) bank69997 +69998 POINT(40.23369804104735 73.43176523289983) bank69998 +69999 POINT(40.82517409265095 74.45697634767986) bank69999 +70000 POINT(41.25301762001696 74.15520386837999) bank70000 +70001 POINT(41.48634448985885 73.72047294002486) bank70001 +70002 POINT(40.8300716124689 73.14476328536801) bank70002 +70003 POINT(40.40305290458823 74.09167612336653) bank70003 +70004 POINT(41.10099342260331 73.4427660514652) bank70004 +70005 POINT(40.75152966972668 73.3172987671282) bank70005 +70006 POINT(41.36300526682327 74.15544632277148) bank70006 +70007 POINT(41.311965270690855 74.66326667410381) bank70007 +70008 POINT(40.87877526112902 73.7814700553652) bank70008 +70009 POINT(40.101504429018306 73.00772310174172) bank70009 +70010 POINT(40.232387778323606 73.24484141174118) bank70010 +70011 POINT(39.895167746333144 73.72722577613881) bank70011 +70012 POINT(40.2516318869489 73.7807538499843) bank70012 +70013 POINT(40.02395479791351 73.71411368839125) bank70013 +70014 POINT(40.40386810056453 73.83834320134598) bank70014 +70015 POINT(40.49414038636224 74.46021102707381) bank70015 +70016 POINT(41.0711906242439 73.79293991525297) bank70016 +70017 POINT(40.864127690408644 74.0966682021047) bank70017 +70018 POINT(40.88499169424519 73.3233308546654) bank70018 +70019 POINT(41.38754072013887 74.39001127539885) bank70019 +70020 POINT(40.90505010640016 74.0543529378449) bank70020 +70021 POINT(39.88779259952221 74.6005392490575) bank70021 +70022 POINT(41.5305962178293 74.16666328619078) bank70022 +70023 POINT(40.110197569121866 73.20466846553892) bank70023 +70024 POINT(40.650782754076616 73.76706575764406) bank70024 +70025 POINT(40.29377731458585 74.24641040953321) bank70025 +70026 POINT(39.813926143157495 74.66470996257024) bank70026 +70027 POINT(41.3642468511923 74.87653411121272) bank70027 +70028 POINT(39.72347417324226 74.25063626780663) bank70028 +70029 POINT(39.90878148601943 74.63805866942398) bank70029 +70030 POINT(41.64807617982284 74.7914149758524) bank70030 +70031 POINT(41.069755882159285 74.82702761807191) bank70031 +70032 POINT(40.07454286225236 74.40168076798119) bank70032 +70033 POINT(41.12389979147913 73.51004678807817) bank70033 +70034 POINT(41.001634342188126 74.42221550594499) bank70034 +70035 POINT(40.26729212355981 74.30195709994759) bank70035 +70036 POINT(39.81598178728752 73.45049775033048) bank70036 +70037 POINT(40.923574907641246 74.49552715445245) bank70037 +70038 POINT(40.16197304153807 74.75405936190771) bank70038 +70039 POINT(39.86097461622838 73.6214810791395) bank70039 +70040 POINT(39.74705105940758 74.89482504947314) bank70040 +70041 POINT(40.149928083298 74.90600273976615) bank70041 +70042 POINT(41.04591451297133 74.71403554882268) bank70042 +70043 POINT(41.44842095743036 74.1995582598505) bank70043 +70044 POINT(39.99489629844306 73.1236309252918) bank70044 +70045 POINT(41.71100294420765 73.93935333978499) bank70045 +70046 POINT(40.93953383554722 73.68137765682788) bank70046 +70047 POINT(40.67624687804792 74.58048250732655) bank70047 +70048 POINT(40.17353403881163 73.71354234564092) bank70048 +70049 POINT(40.58267518716135 74.4988804589338) bank70049 +70050 POINT(39.984464263007645 73.01685568547397) bank70050 +70051 POINT(41.3748850394463 73.07600098392368) bank70051 +70052 POINT(41.224704750006445 73.13826066291261) bank70052 +70053 POINT(41.22923374879556 74.73793910531069) bank70053 +70054 POINT(39.99020528089362 74.12068818235134) bank70054 +70055 POINT(41.15334130514314 73.5185246906759) bank70055 +70056 POINT(41.40169698267855 74.53924285265727) bank70056 +70057 POINT(41.00863178889807 73.28553448473738) bank70057 +70058 POINT(41.165506496667795 74.9544050511106) bank70058 +70059 POINT(40.912287818245034 73.71026813857607) bank70059 +70060 POINT(40.50816009649497 74.79217127004232) bank70060 +70061 POINT(39.905192372754215 73.79440999711471) bank70061 +70062 POINT(40.81699317259074 73.60139094827161) bank70062 +70063 POINT(40.45829629970187 73.84242472665203) bank70063 +70064 POINT(40.42422922205013 74.88933185711706) bank70064 +70065 POINT(40.060183460894926 73.9482579415428) bank70065 +70066 POINT(40.45632973973125 74.26289994240523) bank70066 +70067 POINT(40.499100102792724 74.69869103198603) bank70067 +70068 POINT(41.068702931204584 73.47070878790927) bank70068 +70069 POINT(40.542289742690016 74.37164263263215) bank70069 +70070 POINT(39.77497387786917 73.77791432679324) bank70070 +70071 POINT(40.91766433758385 74.43694165762297) bank70071 +70072 POINT(40.591498129268686 73.53569018332426) bank70072 +70073 POINT(40.705561575698574 74.81517509550818) bank70073 +70074 POINT(41.608939749302 74.33916151270093) bank70074 +70075 POINT(41.6748048492127 74.23117389809687) bank70075 +70076 POINT(40.77298572289169 74.16003882963504) bank70076 +70077 POINT(40.27826489315983 74.30416335291184) bank70077 +70078 POINT(40.995738514891336 74.92434831639788) bank70078 +70079 POINT(41.10242337760339 74.40693407223092) bank70079 +70080 POINT(39.81075736834284 73.49292993945602) bank70080 +70081 POINT(41.63165457243498 73.46290785860886) bank70081 +70082 POINT(40.76608634805099 73.77386586590451) bank70082 +70083 POINT(40.41363981596416 74.87384792965184) bank70083 +70084 POINT(40.84945810438308 74.53948723497776) bank70084 +70085 POINT(41.32482473504606 74.229111714479) bank70085 +70086 POINT(40.92369788517086 74.65593835186738) bank70086 +70087 POINT(41.554987047989556 73.31765715613223) bank70087 +70088 POINT(41.287279235696 74.55693237421585) bank70088 +70089 POINT(40.94121113923498 74.33028241391115) bank70089 +70090 POINT(40.560732953546264 74.00852079387842) bank70090 +70091 POINT(41.043428563261116 73.68435309816293) bank70091 +70092 POINT(40.0181430852384 74.70763718291867) bank70092 +70093 POINT(40.47667466221837 73.73712860806091) bank70093 +70094 POINT(40.42745521402233 73.03139813727854) bank70094 +70095 POINT(41.064546600293454 73.17854911287684) bank70095 +70096 POINT(41.32838046356503 73.03947806555264) bank70096 +70097 POINT(40.08452329783274 73.8698530746058) bank70097 +70098 POINT(40.1012267929761 74.49668643105456) bank70098 +70099 POINT(40.155633090280084 74.26246304934385) bank70099 +70100 POINT(41.08428805356474 73.27157212788327) bank70100 +70101 POINT(40.3817060327029 74.98272631810228) bank70101 +70102 POINT(40.4253273847673 74.50745496544323) bank70102 +70103 POINT(39.86549372304986 74.0603850830504) bank70103 +70104 POINT(40.26400628211871 73.57543518134607) bank70104 +70105 POINT(41.12965510263045 74.91445229960762) bank70105 +70106 POINT(40.6537067648199 73.29244316293985) bank70106 +70107 POINT(40.71333483373392 74.96964666745221) bank70107 +70108 POINT(40.294417559638866 73.15583053554946) bank70108 +70109 POINT(39.87692656781406 73.40283078266806) bank70109 +70110 POINT(41.05613915128958 73.80372180452463) bank70110 +70111 POINT(40.99528834730213 74.09416525749337) bank70111 +70112 POINT(41.64713460704584 74.99580347162465) bank70112 +70113 POINT(41.48059498592613 74.53932219178976) bank70113 +70114 POINT(41.52618629664836 73.49680007316178) bank70114 +70115 POINT(39.72955816892646 74.56734116997069) bank70115 +70116 POINT(40.93143624262774 73.29921178844988) bank70116 +70117 POINT(40.999813633088316 74.9139841935116) bank70117 +70118 POINT(39.80690767556601 73.92432051565372) bank70118 +70119 POINT(40.5047461131877 73.9755775091426) bank70119 +70120 POINT(40.37547498711945 74.8193100890661) bank70120 +70121 POINT(39.84331448789297 74.53759568620069) bank70121 +70122 POINT(41.10096872004919 73.4807143895519) bank70122 +70123 POINT(40.44760289091129 74.74576244301623) bank70123 +70124 POINT(40.016993997713705 73.76175404053633) bank70124 +70125 POINT(40.51509323580312 74.22133171368088) bank70125 +70126 POINT(40.98159751135189 73.26039291906622) bank70126 +70127 POINT(40.384213874365116 73.56066259881558) bank70127 +70128 POINT(40.42008285699653 74.61244109901014) bank70128 +70129 POINT(39.76737884047463 73.43368754808726) bank70129 +70130 POINT(40.34781214144271 73.09688608647694) bank70130 +70131 POINT(40.373241520864674 74.15360606636816) bank70131 +70132 POINT(41.08079140898864 73.28883029265988) bank70132 +70133 POINT(39.8464413228927 73.2451293159603) bank70133 +70134 POINT(40.26924307748871 73.17717339231474) bank70134 +70135 POINT(41.450357344640864 73.99326861140997) bank70135 +70136 POINT(40.851741931963474 74.24656551600162) bank70136 +70137 POINT(39.76342131398809 73.04869550639924) bank70137 +70138 POINT(40.11464163088544 73.31626318348043) bank70138 +70139 POINT(40.05004421665291 74.53703061318384) bank70139 +70140 POINT(40.72262925064112 74.46752135783892) bank70140 +70141 POINT(40.72646599418165 73.74105864612675) bank70141 +70142 POINT(41.04778152520919 73.87599188092521) bank70142 +70143 POINT(40.533170455036746 74.11607340069912) bank70143 +70144 POINT(40.89470773317367 73.45510802687942) bank70144 +70145 POINT(40.40274309772225 74.91621228681036) bank70145 +70146 POINT(41.31184199046693 74.55458054221984) bank70146 +70147 POINT(39.85142424552448 73.15045454040404) bank70147 +70148 POINT(41.399424536544245 74.19109527972465) bank70148 +70149 POINT(39.997658194265085 73.27314577841122) bank70149 +70150 POINT(40.02469458047283 74.14946393370289) bank70150 +70151 POINT(40.30244463524567 74.7502414830631) bank70151 +70152 POINT(41.251490064687665 74.39910144624062) bank70152 +70153 POINT(39.9199796027933 74.97535132677102) bank70153 +70154 POINT(40.71272452913472 74.39548392573516) bank70154 +70155 POINT(41.135264934819 74.63357453600099) bank70155 +70156 POINT(41.68653941744476 74.80754970463366) bank70156 +70157 POINT(39.90694647661984 73.84570790775105) bank70157 +70158 POINT(40.38347026610113 74.7224819469202) bank70158 +70159 POINT(41.1949171875332 73.26389897244805) bank70159 +70160 POINT(41.61586486853597 73.30743199480881) bank70160 +70161 POINT(41.012071640791 74.35999844797396) bank70161 +70162 POINT(41.341601807616485 73.1951408259995) bank70162 +70163 POINT(40.444249637133616 73.11358852904632) bank70163 +70164 POINT(39.98976680585706 74.3336737269085) bank70164 +70165 POINT(40.755529869844274 73.37111660453427) bank70165 +70166 POINT(39.85765784631874 74.9499688613457) bank70166 +70167 POINT(40.62150631102786 73.57904142427569) bank70167 +70168 POINT(39.99079112510859 74.40642400951475) bank70168 +70169 POINT(41.558676579815554 74.18565114231946) bank70169 +70170 POINT(41.01257282928603 74.1833929651376) bank70170 +70171 POINT(40.5044678706582 73.69952187744435) bank70171 +70172 POINT(40.51115639899863 74.57875574704799) bank70172 +70173 POINT(40.454991803535364 74.67922634357957) bank70173 +70174 POINT(40.27732987392029 73.12172745854996) bank70174 +70175 POINT(40.98540998677592 73.1199614103198) bank70175 +70176 POINT(41.082753171962125 73.87953143066954) bank70176 +70177 POINT(40.95929923100487 74.76723229649463) bank70177 +70178 POINT(40.96824520419383 74.83961261329297) bank70178 +70179 POINT(41.186444562777005 74.61007071521637) bank70179 +70180 POINT(39.90250036916908 74.05319465998944) bank70180 +70181 POINT(40.35171049832293 73.08844832492017) bank70181 +70182 POINT(40.73510122671807 74.11847784681216) bank70182 +70183 POINT(40.671600399057795 74.29772406770266) bank70183 +70184 POINT(41.104257917116946 73.5728283662118) bank70184 +70185 POINT(40.01807359843617 73.46710136664855) bank70185 +70186 POINT(39.715435665100145 73.13162020046543) bank70186 +70187 POINT(40.07048329344082 73.20557638170526) bank70187 +70188 POINT(41.371983908636615 73.37120186967697) bank70188 +70189 POINT(40.48576063852001 73.67785415359626) bank70189 +70190 POINT(40.98823560983869 73.47924861785508) bank70190 +70191 POINT(41.07579198001427 73.32268487968497) bank70191 +70192 POINT(40.965520918981404 74.00389250452764) bank70192 +70193 POINT(40.2669529394148 73.88748320578391) bank70193 +70194 POINT(40.40025816855668 73.94006236207102) bank70194 +70195 POINT(39.75467397394336 74.02247916268765) bank70195 +70196 POINT(40.46787018252061 73.00971513354948) bank70196 +70197 POINT(41.547092248136686 74.37101143580922) bank70197 +70198 POINT(40.79977862767009 73.70664207665764) bank70198 +70199 POINT(40.2600024707582 73.69392475387002) bank70199 +70200 POINT(39.73072185752116 73.77792733690559) bank70200 +70201 POINT(40.943521719400124 74.49273954800555) bank70201 +70202 POINT(41.55400827715338 74.5913279522674) bank70202 +70203 POINT(41.35731254501152 73.04428153728625) bank70203 +70204 POINT(40.861870895373535 74.51795122978548) bank70204 +70205 POINT(41.11955560874001 73.8243423758329) bank70205 +70206 POINT(39.83652606338904 73.25617027308662) bank70206 +70207 POINT(40.233949299439956 73.05418679188124) bank70207 +70208 POINT(41.22707122598003 74.70158986570215) bank70208 +70209 POINT(39.77091071099508 73.74140554819854) bank70209 +70210 POINT(41.086992259883125 73.53448186361183) bank70210 +70211 POINT(40.223469568026196 74.51362949239781) bank70211 +70212 POINT(40.51411262982471 74.29998791958259) bank70212 +70213 POINT(41.100810793129675 73.30752142172628) bank70213 +70214 POINT(39.863214017355176 74.49818919242557) bank70214 +70215 POINT(40.20402277154618 73.57403726222016) bank70215 +70216 POINT(40.67979514964648 74.8137905413439) bank70216 +70217 POINT(40.744509736685224 74.70487768581026) bank70217 +70218 POINT(41.15562304595348 74.99786262568236) bank70218 +70219 POINT(40.359981258432576 74.8555594454548) bank70219 +70220 POINT(40.66368171760747 73.80277058703629) bank70220 +70221 POINT(40.21857747640867 73.42131864959781) bank70221 +70222 POINT(41.65057091413839 73.48216980537832) bank70222 +70223 POINT(39.85704197112077 73.24254064104309) bank70223 +70224 POINT(41.571410740108796 74.71526056406498) bank70224 +70225 POINT(41.498358443340514 73.65905208208594) bank70225 +70226 POINT(40.2135651962278 73.63653881043071) bank70226 +70227 POINT(40.02542432141552 73.03233662108524) bank70227 +70228 POINT(40.80187237984674 73.36406041234372) bank70228 +70229 POINT(39.91701896319617 73.27971344225136) bank70229 +70230 POINT(41.559306678255865 74.06696427437396) bank70230 +70231 POINT(39.83277018654205 73.84508409726008) bank70231 +70232 POINT(40.82732230095645 73.63626035178119) bank70232 +70233 POINT(41.054694807593634 74.10226675375915) bank70233 +70234 POINT(40.9601531237184 73.57765835637133) bank70234 +70235 POINT(41.05548271589485 74.5782156716883) bank70235 +70236 POINT(40.644581350804756 73.6122448380676) bank70236 +70237 POINT(40.181557538533156 74.12475430432423) bank70237 +70238 POINT(39.76107509940549 74.19060930780986) bank70238 +70239 POINT(40.37218611747427 74.37014628406703) bank70239 +70240 POINT(41.511610363189135 74.58602831588081) bank70240 +70241 POINT(40.473914059148214 74.69071272102491) bank70241 +70242 POINT(41.70423822560539 74.59636997317153) bank70242 +70243 POINT(41.369628030914484 73.66701631654517) bank70243 +70244 POINT(39.81067499809724 73.68031184370058) bank70244 +70245 POINT(40.30611535540102 74.1405710557901) bank70245 +70246 POINT(40.7315457062347 73.371587039754) bank70246 +70247 POINT(39.802218427749224 73.68338071893378) bank70247 +70248 POINT(40.644475768843435 74.43355906098142) bank70248 +70249 POINT(41.4499654594497 74.08520838108811) bank70249 +70250 POINT(41.23793823552563 73.53272284959182) bank70250 +70251 POINT(41.34689566268211 73.90641593420408) bank70251 +70252 POINT(40.549272984850376 73.08205771526518) bank70252 +70253 POINT(41.21426606799247 74.2881681336304) bank70253 +70254 POINT(40.07010814627019 74.5659643754742) bank70254 +70255 POINT(40.060357246283985 73.34540843573896) bank70255 +70256 POINT(40.41013255817344 74.40882887922598) bank70256 +70257 POINT(40.80970719744624 73.269773284228) bank70257 +70258 POINT(40.643501486529395 74.408562981673) bank70258 +70259 POINT(40.50815716406331 74.49433209569933) bank70259 +70260 POINT(40.65570424087079 73.93644513880534) bank70260 +70261 POINT(40.72118571856725 73.94409096406396) bank70261 +70262 POINT(41.01039840700026 73.3780093319617) bank70262 +70263 POINT(41.387645156128826 74.0025165100183) bank70263 +70264 POINT(41.334019611718844 74.9708571320444) bank70264 +70265 POINT(40.922627823006714 74.35950820266109) bank70265 +70266 POINT(40.77141501248824 74.90524670262832) bank70266 +70267 POINT(41.470093584013384 73.37234961897379) bank70267 +70268 POINT(40.11215445099632 74.82635823504305) bank70268 +70269 POINT(41.035767287240795 74.61138016006757) bank70269 +70270 POINT(40.081012268319164 73.22146602989088) bank70270 +70271 POINT(41.690651497451405 73.21326046463079) bank70271 +70272 POINT(41.602912449325466 74.47746632820719) bank70272 +70273 POINT(39.875322037066944 73.13545662581168) bank70273 +70274 POINT(40.407320064829335 73.66812084287903) bank70274 +70275 POINT(40.692128618578835 74.1611631036102) bank70275 +70276 POINT(41.072978542829254 74.16820665894139) bank70276 +70277 POINT(40.970118515202046 74.28443778911252) bank70277 +70278 POINT(39.834191171589474 73.64330281902376) bank70278 +70279 POINT(41.290069225815735 73.02954323953225) bank70279 +70280 POINT(39.94341077111665 74.47119486639546) bank70280 +70281 POINT(41.68023770754311 74.55059608311504) bank70281 +70282 POINT(40.260625253158636 73.41914718446856) bank70282 +70283 POINT(40.995778612836354 73.79222715235251) bank70283 +70284 POINT(40.40557690682009 74.05214673949213) bank70284 +70285 POINT(40.5147584308893 73.35921146869418) bank70285 +70286 POINT(41.22869379902553 74.7701954745391) bank70286 +70287 POINT(40.37923768563256 74.97215960135142) bank70287 +70288 POINT(40.21703356906861 73.8641772643734) bank70288 +70289 POINT(41.20874963718414 74.39081548940591) bank70289 +70290 POINT(40.750929032251236 74.90958531645224) bank70290 +70291 POINT(40.17200372629834 73.86502846686308) bank70291 +70292 POINT(41.35535312864592 73.13066373387558) bank70292 +70293 POINT(40.2675626100317 74.95082687743049) bank70293 +70294 POINT(40.46713441900245 73.49705727786447) bank70294 +70295 POINT(41.15067824634219 74.60052224200989) bank70295 +70296 POINT(39.783253441252874 74.58122721048161) bank70296 +70297 POINT(40.683886074941945 73.62042163402658) bank70297 +70298 POINT(40.63068208037837 74.73381587940989) bank70298 +70299 POINT(41.22090283774152 74.31924143372241) bank70299 +70300 POINT(41.62123051131634 73.41914998930584) bank70300 +70301 POINT(40.88618334497646 73.93768814366408) bank70301 +70302 POINT(40.29980982764968 74.72630291205343) bank70302 +70303 POINT(40.9558329258891 73.16027157419137) bank70303 +70304 POINT(40.52352922690556 73.11560326242254) bank70304 +70305 POINT(40.25345012538335 73.2423185029593) bank70305 +70306 POINT(40.987274715412234 74.4479132056718) bank70306 +70307 POINT(39.78729978066132 74.24358210473393) bank70307 +70308 POINT(41.58794998521852 74.80303970012612) bank70308 +70309 POINT(40.055482382531196 74.85239541115793) bank70309 +70310 POINT(40.68421762184902 74.78097409339313) bank70310 +70311 POINT(39.9609062266868 73.95063631759214) bank70311 +70312 POINT(40.95318307152628 73.90430798290173) bank70312 +70313 POINT(40.55861241745872 74.3777504499696) bank70313 +70314 POINT(40.76072850718002 74.14245944697319) bank70314 +70315 POINT(40.94925697259968 74.3890030462741) bank70315 +70316 POINT(41.6958694781273 73.7760832157441) bank70316 +70317 POINT(40.84652652941382 73.67838058612494) bank70317 +70318 POINT(41.175343303391735 73.41218772979366) bank70318 +70319 POINT(39.790617962132806 74.72631587359373) bank70319 +70320 POINT(40.36691086340998 74.80584134883684) bank70320 +70321 POINT(39.759167929719325 74.47807137559592) bank70321 +70322 POINT(40.01228179060514 74.97900347609706) bank70322 +70323 POINT(40.61511351665074 74.04949809947452) bank70323 +70324 POINT(40.28440313738772 73.02177660306296) bank70324 +70325 POINT(40.04601009607876 74.2130674946026) bank70325 +70326 POINT(40.101434263433944 74.19111718679066) bank70326 +70327 POINT(39.85271187777569 74.7074347891422) bank70327 +70328 POINT(40.476864836952046 73.71233054421418) bank70328 +70329 POINT(40.880969729839556 73.16305776447413) bank70329 +70330 POINT(41.66727196860063 74.77405585488926) bank70330 +70331 POINT(41.545816578357595 73.91517670735362) bank70331 +70332 POINT(39.988646842328535 73.56234639311755) bank70332 +70333 POINT(41.08087187126615 74.25538034531175) bank70333 +70334 POINT(41.65566547923945 74.37083982512414) bank70334 +70335 POINT(40.85860534047455 73.1270390863573) bank70335 +70336 POINT(39.910266227052816 73.42293149937399) bank70336 +70337 POINT(41.667953678455135 73.82566322810226) bank70337 +70338 POINT(41.26438877804525 74.28739018834804) bank70338 +70339 POINT(41.23144259706938 74.884502142703) bank70339 +70340 POINT(41.663478252204364 73.02038533614913) bank70340 +70341 POINT(41.30267649889545 73.85732689645893) bank70341 +70342 POINT(40.59065850321508 74.37080935586677) bank70342 +70343 POINT(41.06177368651193 74.4039537618829) bank70343 +70344 POINT(41.07242731127135 74.47477906243) bank70344 +70345 POINT(41.137522242586016 74.30496614544516) bank70345 +70346 POINT(41.154627711438 74.48651656275217) bank70346 +70347 POINT(40.903437402073116 74.85411589960651) bank70347 +70348 POINT(39.82873196678674 73.41950804576797) bank70348 +70349 POINT(41.380468623778974 74.36932828289207) bank70349 +70350 POINT(40.55839589189491 74.23374509165595) bank70350 +70351 POINT(41.457858736215144 73.43283839010287) bank70351 +70352 POINT(41.50750846725811 74.87526593399686) bank70352 +70353 POINT(40.909418331058404 73.74223237948257) bank70353 +70354 POINT(40.015992960648795 73.06776923146485) bank70354 +70355 POINT(40.26786745050973 73.26783818294058) bank70355 +70356 POINT(41.159664155824125 73.75608418489873) bank70356 +70357 POINT(40.05029107698788 74.72308397514728) bank70357 +70358 POINT(40.92067262461828 73.5738214256588) bank70358 +70359 POINT(40.07220833260122 73.34208220859621) bank70359 +70360 POINT(41.444484501620266 74.01178738418638) bank70360 +70361 POINT(40.92916631017184 73.84001503656587) bank70361 +70362 POINT(41.08142038076068 74.1256576998102) bank70362 +70363 POINT(40.594346036691284 74.0621971889512) bank70363 +70364 POINT(40.6948717513811 74.51315515864184) bank70364 +70365 POINT(40.741141862395324 74.26465826909912) bank70365 +70366 POINT(40.47880063734113 73.60523604315433) bank70366 +70367 POINT(40.63463408853854 74.70194783052676) bank70367 +70368 POINT(39.956600434836496 73.1803912338622) bank70368 +70369 POINT(40.363841412776516 73.02850492214286) bank70369 +70370 POINT(40.75752184556333 73.2880947351655) bank70370 +70371 POINT(41.380066221023114 73.81430176045164) bank70371 +70372 POINT(40.27520260489736 73.85827493528441) bank70372 +70373 POINT(39.71507970162261 73.31675388172512) bank70373 +70374 POINT(40.61118104647984 73.02933720376161) bank70374 +70375 POINT(40.332075210160596 74.43368611207919) bank70375 +70376 POINT(41.2681046342342 74.80145735946806) bank70376 +70377 POINT(40.341061883712065 74.39948484173684) bank70377 +70378 POINT(41.48941076787499 73.68233948141844) bank70378 +70379 POINT(40.855633510297444 74.1599836630907) bank70379 +70380 POINT(40.02793247664774 73.44942577984462) bank70380 +70381 POINT(40.030946091145104 73.92767046551322) bank70381 +70382 POINT(40.387673913928815 74.31768436251821) bank70382 +70383 POINT(40.12933266844081 73.27891359674238) bank70383 +70384 POINT(40.71775727527693 74.88427429114148) bank70384 +70385 POINT(40.75775055526133 74.42847345066355) bank70385 +70386 POINT(41.42766006425393 74.14345447283394) bank70386 +70387 POINT(40.44943375664335 74.33331287993725) bank70387 +70388 POINT(40.34000577665089 74.89878594092599) bank70388 +70389 POINT(40.32078705545675 73.61551681528061) bank70389 +70390 POINT(39.862117712421856 73.37058325842992) bank70390 +70391 POINT(39.835008401388194 75.00418839077864) bank70391 +70392 POINT(40.48271557593061 73.72253801262767) bank70392 +70393 POINT(40.52741690328423 74.79568082044727) bank70393 +70394 POINT(39.752080878572535 73.22134795731344) bank70394 +70395 POINT(40.58925263480796 73.94388873830812) bank70395 +70396 POINT(40.64446683808092 74.16002283059974) bank70396 +70397 POINT(40.92348859114879 73.71604024768767) bank70397 +70398 POINT(40.418543989005926 74.15909499149174) bank70398 +70399 POINT(40.90357766360936 73.74382475482497) bank70399 +70400 POINT(41.122843442833336 74.65272245925503) bank70400 +70401 POINT(39.75221913372897 74.14231481127399) bank70401 +70402 POINT(41.08257464538176 73.22991231286768) bank70402 +70403 POINT(40.52250296750836 74.52513445642366) bank70403 +70404 POINT(40.031726990539426 74.21583020453917) bank70404 +70405 POINT(40.161563017990794 73.32847053149878) bank70405 +70406 POINT(40.24912077189312 73.05537398850278) bank70406 +70407 POINT(40.33265009608162 73.81093538174565) bank70407 +70408 POINT(40.493095537909426 73.23135666789778) bank70408 +70409 POINT(40.674859808150984 73.95591043118833) bank70409 +70410 POINT(40.60461837218674 74.70909739913554) bank70410 +70411 POINT(40.07665842581554 74.79917982251455) bank70411 +70412 POINT(40.58640914758132 74.24022848187606) bank70412 +70413 POINT(41.66202101381902 73.54722537934303) bank70413 +70414 POINT(40.431334912981406 74.36094046013001) bank70414 +70415 POINT(40.47383278917708 73.83324792232412) bank70415 +70416 POINT(41.252912315920206 74.47794018761492) bank70416 +70417 POINT(40.36537218137533 73.49838729930279) bank70417 +70418 POINT(41.319208863532054 74.51953476688254) bank70418 +70419 POINT(40.14638136467721 74.84348423077093) bank70419 +70420 POINT(39.76003537670231 73.57849086117376) bank70420 +70421 POINT(40.177474214729784 73.99636278270943) bank70421 +70422 POINT(41.649845597254114 73.4514767076733) bank70422 +70423 POINT(41.26466684568359 73.03420309703971) bank70423 +70424 POINT(41.63510841557742 73.6466374503613) bank70424 +70425 POINT(40.397305986024 74.66525081901155) bank70425 +70426 POINT(40.7239486124564 74.56159934584285) bank70426 +70427 POINT(40.60772638963284 74.66725075925531) bank70427 +70428 POINT(40.91951658919748 73.33007969901597) bank70428 +70429 POINT(41.49882670147177 73.25725663988425) bank70429 +70430 POINT(40.68929014521028 74.6202421516464) bank70430 +70431 POINT(41.68499626809153 74.5226381761248) bank70431 +70432 POINT(40.66589531604412 74.44701703328896) bank70432 +70433 POINT(41.425678664816594 73.3721366998325) bank70433 +70434 POINT(41.38756536064228 74.99281662105066) bank70434 +70435 POINT(41.249581296957146 73.29316506677615) bank70435 +70436 POINT(41.268503203289114 74.29042598904614) bank70436 +70437 POINT(40.419846922822366 73.72761054379005) bank70437 +70438 POINT(40.6010173877022 73.3115967546469) bank70438 +70439 POINT(40.22926804579262 73.36375328069646) bank70439 +70440 POINT(40.21014898664965 74.91075079935423) bank70440 +70441 POINT(40.09982289229695 74.04729033861197) bank70441 +70442 POINT(41.657614192735274 73.6207915749993) bank70442 +70443 POINT(41.50161508508729 74.08393291200386) bank70443 +70444 POINT(40.538993427352466 74.71533854404126) bank70444 +70445 POINT(40.858021989893786 73.3768376084314) bank70445 +70446 POINT(39.88054764564489 73.06212970855863) bank70446 +70447 POINT(39.86452234634518 73.46015620172639) bank70447 +70448 POINT(40.67824811599098 74.62371536820419) bank70448 +70449 POINT(40.78975261070683 74.48053327204765) bank70449 +70450 POINT(41.43602125908545 73.86806515619033) bank70450 +70451 POINT(41.15422136359717 74.8539152240677) bank70451 +70452 POINT(41.241898800143936 73.5838650311152) bank70452 +70453 POINT(40.54719429717056 74.46948216822486) bank70453 +70454 POINT(40.83190093675816 73.76507971800562) bank70454 +70455 POINT(41.03958313044855 74.48665103479047) bank70455 +70456 POINT(39.74485028013779 74.66781033807094) bank70456 +70457 POINT(40.92230567591912 73.56737520289953) bank70457 +70458 POINT(40.38853532732516 74.04955707965104) bank70458 +70459 POINT(40.886730562953 73.08336150423634) bank70459 +70460 POINT(41.63073932311476 73.35714200615031) bank70460 +70461 POINT(41.37817395456398 73.14839010595317) bank70461 +70462 POINT(40.75382020571258 73.772191934272) bank70462 +70463 POINT(41.35702659966391 74.54821307696784) bank70463 +70464 POINT(41.54453714246589 73.41429504260459) bank70464 +70465 POINT(41.33424930368248 74.79541794263105) bank70465 +70466 POINT(39.79549797589122 73.06768195895675) bank70466 +70467 POINT(40.609757574826816 74.80632179220265) bank70467 +70468 POINT(41.62477729877664 74.72753292286342) bank70468 +70469 POINT(40.885588409927216 74.86986596147338) bank70469 +70470 POINT(40.73169229150578 73.27826925753972) bank70470 +70471 POINT(41.11058989260557 73.95738031378714) bank70471 +70472 POINT(41.524444871607635 73.45700955639596) bank70472 +70473 POINT(41.145727515775384 73.6234805363617) bank70473 +70474 POINT(41.3362914102568 74.57723262781693) bank70474 +70475 POINT(41.657399602335126 74.60361851042931) bank70475 +70476 POINT(40.00238992583281 73.37339495653843) bank70476 +70477 POINT(39.94314693000277 73.1554871156006) bank70477 +70478 POINT(40.93688526230265 73.62807908984706) bank70478 +70479 POINT(40.495827889272796 74.70628562211232) bank70479 +70480 POINT(40.70334917650463 74.33392537512033) bank70480 +70481 POINT(40.22065107840311 73.0358076220457) bank70481 +70482 POINT(40.81663333993998 74.54825601101928) bank70482 +70483 POINT(40.268753788110914 74.21670978702448) bank70483 +70484 POINT(41.60799791268443 74.47147504833076) bank70484 +70485 POINT(40.612792474979855 73.09514819116447) bank70485 +70486 POINT(41.27692924465586 73.16103816975816) bank70486 +70487 POINT(41.55835011647204 74.46187022065845) bank70487 +70488 POINT(41.21779369526848 73.41492559384524) bank70488 +70489 POINT(40.84077447810254 73.50572856248118) bank70489 +70490 POINT(41.473814954026565 73.4295585115957) bank70490 +70491 POINT(41.41926463797992 74.18693742108665) bank70491 +70492 POINT(40.36888280690541 74.93509897257256) bank70492 +70493 POINT(40.592582274497296 74.61978806970356) bank70493 +70494 POINT(41.617086009842964 73.12233771796525) bank70494 +70495 POINT(41.58563890443329 73.75141573658146) bank70495 +70496 POINT(41.235344312495556 74.41519720918046) bank70496 +70497 POINT(40.64596218378022 74.93313590430185) bank70497 +70498 POINT(40.17340711921213 73.80170675090919) bank70498 +70499 POINT(40.08853457642594 74.27229090613152) bank70499 +70500 POINT(40.84793104402022 73.86022873139358) bank70500 +70501 POINT(40.274226410030394 74.3548299810667) bank70501 +70502 POINT(40.843651860188466 74.16581307408802) bank70502 +70503 POINT(41.18035559955018 73.05509863552645) bank70503 +70504 POINT(41.705701543099 74.807442092397) bank70504 +70505 POINT(39.77815419368829 74.6409964061731) bank70505 +70506 POINT(40.17607483225201 74.07646132020436) bank70506 +70507 POINT(40.37478232835193 73.8479237086137) bank70507 +70508 POINT(41.603802463458926 73.60870158795362) bank70508 +70509 POINT(39.7987318540621 73.78921196132548) bank70509 +70510 POINT(41.18173231303773 74.18778272467155) bank70510 +70511 POINT(40.52834931956572 74.84544604747771) bank70511 +70512 POINT(41.35660462627072 74.29504390504324) bank70512 +70513 POINT(39.84337994591836 74.95591059694401) bank70513 +70514 POINT(41.41267011145181 73.77361251712031) bank70514 +70515 POINT(41.12666624069095 74.01699267988613) bank70515 +70516 POINT(40.64628867376787 74.55375465588294) bank70516 +70517 POINT(40.2501327101966 73.8030911290514) bank70517 +70518 POINT(40.40136686522225 73.7066838260576) bank70518 +70519 POINT(40.96838995405475 73.77191551143447) bank70519 +70520 POINT(40.47913448671473 74.21275804933146) bank70520 +70521 POINT(39.83459276667869 73.28033299088308) bank70521 +70522 POINT(41.4617953118013 73.28031956901118) bank70522 +70523 POINT(41.672344784621586 73.055746180338) bank70523 +70524 POINT(40.25957370760004 74.43937469392198) bank70524 +70525 POINT(41.56180470220227 74.49990237873739) bank70525 +70526 POINT(39.88128182584733 74.55506079356557) bank70526 +70527 POINT(40.91004086016709 74.78978675928552) bank70527 +70528 POINT(40.12786010624656 73.03163846052026) bank70528 +70529 POINT(41.02773580486077 73.43343120423874) bank70529 +70530 POINT(40.54079493425525 73.03352079050632) bank70530 +70531 POINT(39.85857102049864 73.20862840064224) bank70531 +70532 POINT(41.49215622206363 74.8154387860699) bank70532 +70533 POINT(40.407358084877366 74.75256219188242) bank70533 +70534 POINT(39.83282339602009 74.73719831418128) bank70534 +70535 POINT(41.553695808448126 74.80800194697812) bank70535 +70536 POINT(41.61077292286414 73.69938222526879) bank70536 +70537 POINT(39.85330739263151 74.43783666749768) bank70537 +70538 POINT(39.77168950666025 74.61980222649741) bank70538 +70539 POINT(40.172473018837586 73.95413118528883) bank70539 +70540 POINT(40.36843666825774 73.50583227553868) bank70540 +70541 POINT(40.91539338390978 73.4305150922074) bank70541 +70542 POINT(40.52898545527987 73.06729112070094) bank70542 +70543 POINT(40.18165377314137 73.63992976436386) bank70543 +70544 POINT(39.95965955858059 74.26049870507666) bank70544 +70545 POINT(39.931291954449925 74.18215185812464) bank70545 +70546 POINT(40.81709083811637 73.17075483294747) bank70546 +70547 POINT(41.14978775806558 73.39479038001225) bank70547 +70548 POINT(40.34216348765543 74.6247801376048) bank70548 +70549 POINT(40.14584044862771 74.80986262630012) bank70549 +70550 POINT(39.981453440010156 73.04463320465757) bank70550 +70551 POINT(40.44426039428698 73.02482300992962) bank70551 +70552 POINT(39.95828417553026 73.9862259890881) bank70552 +70553 POINT(41.63923266590776 74.83371654777645) bank70553 +70554 POINT(40.213814663537185 74.12104099653502) bank70554 +70555 POINT(41.71062693045123 73.89821500145551) bank70555 +70556 POINT(39.933070837870076 73.82159497870877) bank70556 +70557 POINT(41.41576025767455 74.33225167731317) bank70557 +70558 POINT(41.498083906080076 73.36182332586351) bank70558 +70559 POINT(40.96287327589929 73.87976771424398) bank70559 +70560 POINT(40.09944418300058 73.40844690895777) bank70560 +70561 POINT(41.505709603711004 74.77367046493131) bank70561 +70562 POINT(40.52776803907328 74.76586008216655) bank70562 +70563 POINT(41.119543099164304 74.74270042217785) bank70563 +70564 POINT(41.245386155318194 74.11666878473294) bank70564 +70565 POINT(41.376265842040574 73.40173529793046) bank70565 +70566 POINT(40.40997632477005 74.54512765237038) bank70566 +70567 POINT(41.521289978958215 74.04358944536423) bank70567 +70568 POINT(40.94464427253525 73.15683310811721) bank70568 +70569 POINT(40.17052075836777 74.47728354592866) bank70569 +70570 POINT(40.09270789307728 74.50446111995677) bank70570 +70571 POINT(39.73725252906395 73.49794810374905) bank70571 +70572 POINT(40.350779443865264 74.6780243975794) bank70572 +70573 POINT(40.43119658349043 73.81936277987536) bank70573 +70574 POINT(40.53742928099665 74.54917937161422) bank70574 +70575 POINT(40.13310888179204 73.44926286925913) bank70575 +70576 POINT(41.20303092669827 74.80928848681447) bank70576 +70577 POINT(41.29207800084711 73.9005395846438) bank70577 +70578 POINT(41.10634106749563 73.0977663214574) bank70578 +70579 POINT(41.55213318185317 73.08924580344453) bank70579 +70580 POINT(40.960162466061284 74.49436879430037) bank70580 +70581 POINT(40.773099215729715 74.35450542588292) bank70581 +70582 POINT(40.455529904792236 73.383063351899) bank70582 +70583 POINT(41.05826667363397 74.45330607640469) bank70583 +70584 POINT(39.893628928922666 73.11412438781178) bank70584 +70585 POINT(41.22916166789761 73.77003387329611) bank70585 +70586 POINT(40.469032825964874 74.79864150766018) bank70586 +70587 POINT(39.766460200020205 73.57109810923129) bank70587 +70588 POINT(40.46495344560313 73.23013311655137) bank70588 +70589 POINT(39.772007384095694 73.81981621076858) bank70589 +70590 POINT(40.79507234386953 74.18963383499575) bank70590 +70591 POINT(40.339925662317896 74.21017405106802) bank70591 +70592 POINT(39.992247085344715 73.00633422370302) bank70592 +70593 POINT(40.670839018434975 74.96617198962433) bank70593 +70594 POINT(40.08327116275643 74.98475982453041) bank70594 +70595 POINT(40.82672552855863 73.27335387203031) bank70595 +70596 POINT(41.67736863620989 74.14042210428191) bank70596 +70597 POINT(41.045925926805076 73.33037973041827) bank70597 +70598 POINT(39.940624564297366 73.23673784870854) bank70598 +70599 POINT(41.63881178126512 74.99905857739095) bank70599 +70600 POINT(40.20073590690965 74.00155374742141) bank70600 +70601 POINT(40.05470102361011 74.67593483308214) bank70601 +70602 POINT(40.843101554185296 74.11576116498887) bank70602 +70603 POINT(41.12973895267161 73.27083287997566) bank70603 +70604 POINT(41.31380238306977 73.82476538189646) bank70604 +70605 POINT(41.06962645134461 74.53575455710241) bank70605 +70606 POINT(40.251448021538465 74.40855223151286) bank70606 +70607 POINT(41.33279578689195 73.9916176168947) bank70607 +70608 POINT(39.850477688123135 73.03683026936122) bank70608 +70609 POINT(41.667115036980995 73.08342367187485) bank70609 +70610 POINT(40.35836515329522 73.39271717131423) bank70610 +70611 POINT(40.40434407407166 73.13943437440224) bank70611 +70612 POINT(40.33989528706341 74.28331280402905) bank70612 +70613 POINT(40.10465431181085 74.74399086496454) bank70613 +70614 POINT(41.086597240716614 73.15035225369209) bank70614 +70615 POINT(40.836250015193094 73.7159309869884) bank70615 +70616 POINT(40.26739331478776 74.51287774724973) bank70616 +70617 POINT(41.21885121047769 73.02807809176014) bank70617 +70618 POINT(41.51848796049893 73.38331277529753) bank70618 +70619 POINT(40.026975708196716 73.57509666921565) bank70619 +70620 POINT(40.29851099599877 73.8535909127161) bank70620 +70621 POINT(40.5444071576086 74.39244495657368) bank70621 +70622 POINT(39.986567655235625 73.53847031140887) bank70622 +70623 POINT(40.551159873789075 74.5440123706445) bank70623 +70624 POINT(40.586248967806476 74.57125228233168) bank70624 +70625 POINT(39.78702481029164 73.11592294256651) bank70625 +70626 POINT(40.85732652439016 73.67774696397163) bank70626 +70627 POINT(39.85329683662342 73.17870356631757) bank70627 +70628 POINT(39.96197340395098 74.57547888466888) bank70628 +70629 POINT(41.08768149185502 74.95874608240514) bank70629 +70630 POINT(41.346937575542654 73.10512562774767) bank70630 +70631 POINT(41.06778592089349 73.54681732903828) bank70631 +70632 POINT(41.317528411250365 73.65335720762668) bank70632 +70633 POINT(40.69565634491865 73.0731508733964) bank70633 +70634 POINT(41.037372346456465 73.37449166873664) bank70634 +70635 POINT(41.066634371387416 74.0073140545846) bank70635 +70636 POINT(41.68856084381123 74.3884277083814) bank70636 +70637 POINT(40.93276104408172 74.33771050176479) bank70637 +70638 POINT(39.82426581639049 73.95658841768383) bank70638 +70639 POINT(41.381099886723476 74.46584908437194) bank70639 +70640 POINT(41.07974606331764 74.0916881301117) bank70640 +70641 POINT(41.571613247298714 73.41014233404503) bank70641 +70642 POINT(39.96008105945379 74.61243040503301) bank70642 +70643 POINT(40.131812938141685 74.71241724604857) bank70643 +70644 POINT(40.805090278168244 74.76446899483155) bank70644 +70645 POINT(39.81858395121863 74.84020925988085) bank70645 +70646 POINT(40.98548773673109 73.35493473487463) bank70646 +70647 POINT(40.69854513830315 74.23123306487088) bank70647 +70648 POINT(40.41147297067524 73.0811744728392) bank70648 +70649 POINT(41.64918954197777 74.11481923110246) bank70649 +70650 POINT(40.86341088081327 74.71428786945619) bank70650 +70651 POINT(40.09627465347794 74.3982941382991) bank70651 +70652 POINT(40.24237146009696 73.233198027475) bank70652 +70653 POINT(39.88056500242866 74.55760447085048) bank70653 +70654 POINT(40.27577672815363 73.30414083537786) bank70654 +70655 POINT(41.571807458801636 73.36249929401251) bank70655 +70656 POINT(41.549301175746415 73.2373502863741) bank70656 +70657 POINT(40.9645559618183 73.76052092476293) bank70657 +70658 POINT(41.628862147462534 74.29604286686126) bank70658 +70659 POINT(40.344570423030746 73.71976560818229) bank70659 +70660 POINT(41.54071588965479 74.7817628775038) bank70660 +70661 POINT(40.04230743765878 73.28399071191957) bank70661 +70662 POINT(39.88975464323007 73.88631239528844) bank70662 +70663 POINT(40.325459711298805 73.76442991918876) bank70663 +70664 POINT(40.720865462748634 74.41687156067431) bank70664 +70665 POINT(40.1037138614173 74.46470852425855) bank70665 +70666 POINT(40.706712531661445 74.41883502943779) bank70666 +70667 POINT(39.97913457267555 73.5624295783928) bank70667 +70668 POINT(40.91520937711592 74.62453770107032) bank70668 +70669 POINT(39.848429224092726 73.15358184614352) bank70669 +70670 POINT(41.455096214628824 74.43836596130976) bank70670 +70671 POINT(39.730660604891774 74.94926876020952) bank70671 +70672 POINT(41.38444765982408 74.60446652407435) bank70672 +70673 POINT(40.328312708441636 73.33481621229481) bank70673 +70674 POINT(41.553341946054324 74.60933815904274) bank70674 +70675 POINT(41.51294413164165 74.67953356913125) bank70675 +70676 POINT(40.630972490435745 73.12519179048098) bank70676 +70677 POINT(41.53664054649447 74.50135535848598) bank70677 +70678 POINT(40.32524905419657 73.47131649847937) bank70678 +70679 POINT(41.296377424751135 74.63635944049518) bank70679 +70680 POINT(40.54772527686515 73.79068080151305) bank70680 +70681 POINT(40.26947132888359 74.05286164080354) bank70681 +70682 POINT(41.159109004406396 73.5993217420346) bank70682 +70683 POINT(40.72133286496879 73.498683354983) bank70683 +70684 POINT(40.064887233547815 73.9451478319586) bank70684 +70685 POINT(41.51732378879414 74.53985767958166) bank70685 +70686 POINT(41.6510586207932 73.36923231371634) bank70686 +70687 POINT(40.96828368074779 74.38951535822684) bank70687 +70688 POINT(40.004338836652956 74.08970823316969) bank70688 +70689 POINT(40.90355266384603 73.23593246569186) bank70689 +70690 POINT(39.791262926916076 74.3695509609528) bank70690 +70691 POINT(41.073711533514874 74.25356789319787) bank70691 +70692 POINT(41.107951913240676 73.96660565946478) bank70692 +70693 POINT(40.4816613829866 73.57953931966716) bank70693 +70694 POINT(40.55457121888878 74.84066477996312) bank70694 +70695 POINT(39.84930710768279 74.48615580257572) bank70695 +70696 POINT(40.8560919551375 73.97569165239427) bank70696 +70697 POINT(39.998178953453554 73.75301108997664) bank70697 +70698 POINT(40.41264575955979 73.2388054112152) bank70698 +70699 POINT(41.48287936114117 74.82298078307474) bank70699 +70700 POINT(41.40606575277157 74.99747221621739) bank70700 +70701 POINT(40.34971296043205 74.56464221197034) bank70701 +70702 POINT(40.373858434248355 74.05646358273626) bank70702 +70703 POINT(41.636998473918105 73.98018839508605) bank70703 +70704 POINT(40.48662778017955 73.47094857495071) bank70704 +70705 POINT(40.02086305989507 73.88211057037906) bank70705 +70706 POINT(40.88317057889722 74.92127750933791) bank70706 +70707 POINT(40.01569368428553 74.38979194957021) bank70707 +70708 POINT(40.54016889600198 73.83477662211631) bank70708 +70709 POINT(40.221490715314054 74.20872841972961) bank70709 +70710 POINT(40.40457028859731 73.09919886119384) bank70710 +70711 POINT(39.72804567174738 74.83185628918564) bank70711 +70712 POINT(39.90335705136126 73.38633315404294) bank70712 +70713 POINT(41.01879027955195 73.54952937902877) bank70713 +70714 POINT(41.54109151069419 73.40966044482984) bank70714 +70715 POINT(40.659004395599446 73.59977905150586) bank70715 +70716 POINT(40.917375429349306 73.29841888495696) bank70716 +70717 POINT(40.707182295140306 74.91396354091063) bank70717 +70718 POINT(40.918955273550274 73.23368423548477) bank70718 +70719 POINT(40.63815524966321 74.1525218648716) bank70719 +70720 POINT(39.8129493425119 73.96832822401225) bank70720 +70721 POINT(40.262111256758246 73.82705759813412) bank70721 +70722 POINT(40.297241782735995 73.32394708371737) bank70722 +70723 POINT(41.06802010383079 73.69459577395108) bank70723 +70724 POINT(39.79681250989176 73.21355080370331) bank70724 +70725 POINT(40.043199710215674 73.94544587682977) bank70725 +70726 POINT(41.00129072302598 73.63674142085998) bank70726 +70727 POINT(41.620002876823705 74.11700890287062) bank70727 +70728 POINT(41.205481159356964 73.23364669358503) bank70728 +70729 POINT(40.93782318254193 74.19551221668543) bank70729 +70730 POINT(40.37398936496913 74.84511125821805) bank70730 +70731 POINT(39.88026080748508 75.00373862573596) bank70731 +70732 POINT(41.70090557693269 74.7801339897231) bank70732 +70733 POINT(41.26430926161399 73.34828021198302) bank70733 +70734 POINT(40.78746029670313 74.83865883191059) bank70734 +70735 POINT(41.43879334927932 74.75447972524321) bank70735 +70736 POINT(40.97846584156178 74.28784402417298) bank70736 +70737 POINT(41.2584417817232 73.07183117714617) bank70737 +70738 POINT(41.65481187980178 74.92529524012656) bank70738 +70739 POINT(40.258929249488745 74.77552281274828) bank70739 +70740 POINT(40.8073434102842 73.70024889133926) bank70740 +70741 POINT(40.36910090887563 73.59541122746441) bank70741 +70742 POINT(39.90915979886918 73.13779910944533) bank70742 +70743 POINT(40.0968760701664 74.02915308213173) bank70743 +70744 POINT(41.52220522480175 74.70196136994485) bank70744 +70745 POINT(41.38485980680498 74.46025706621305) bank70745 +70746 POINT(41.23552516879668 74.57739165570571) bank70746 +70747 POINT(40.86174525857544 73.2686697224223) bank70747 +70748 POINT(40.66951062605919 73.94838835035313) bank70748 +70749 POINT(41.66847093191818 73.56367998409645) bank70749 +70750 POINT(40.24220863088857 73.61743127412043) bank70750 +70751 POINT(40.485084034638945 74.14096189253253) bank70751 +70752 POINT(40.522699766715284 74.81954276326272) bank70752 +70753 POINT(40.73824626191102 74.01742537506242) bank70753 +70754 POINT(40.099997233759794 74.03565467206326) bank70754 +70755 POINT(40.39128309203715 73.96786830186282) bank70755 +70756 POINT(40.31492297251873 74.463587017306) bank70756 +70757 POINT(41.65204753860965 74.17052401073305) bank70757 +70758 POINT(41.01349905679807 74.32755001942866) bank70758 +70759 POINT(40.98079593626424 73.62907381918075) bank70759 +70760 POINT(41.25942323076515 73.78608413948099) bank70760 +70761 POINT(40.904405479892525 74.76263373171173) bank70761 +70762 POINT(40.81073566470333 74.74622159875095) bank70762 +70763 POINT(40.89089379396373 74.57403394522613) bank70763 +70764 POINT(41.434807057204246 73.35937498736308) bank70764 +70765 POINT(39.75134401154039 74.55094257684938) bank70765 +70766 POINT(40.35314732946833 74.2391203736616) bank70766 +70767 POINT(40.89197915029651 74.59118735138178) bank70767 +70768 POINT(39.75408120560496 74.16719827708646) bank70768 +70769 POINT(41.35788718018907 74.26722131243318) bank70769 +70770 POINT(40.09367133400296 74.74557129605361) bank70770 +70771 POINT(41.71128935788617 74.566471148775) bank70771 +70772 POINT(40.561650305368275 74.86584250019014) bank70772 +70773 POINT(40.64090387246724 73.33766793952294) bank70773 +70774 POINT(40.43163073772569 74.97891337635735) bank70774 +70775 POINT(40.01909488065616 74.73304601036023) bank70775 +70776 POINT(40.121751073766724 73.43439230174141) bank70776 +70777 POINT(40.975645916956076 74.99509629246442) bank70777 +70778 POINT(41.5265417678683 73.21897465477787) bank70778 +70779 POINT(39.93354765071992 73.86833271431588) bank70779 +70780 POINT(41.13251198951832 74.32957129330266) bank70780 +70781 POINT(39.9215135279613 73.7008216376117) bank70781 +70782 POINT(39.72680662544595 73.93101473412442) bank70782 +70783 POINT(41.38757044010521 73.04132462482434) bank70783 +70784 POINT(41.463142229666445 74.16694062417668) bank70784 +70785 POINT(40.77436867150008 73.27919351752392) bank70785 +70786 POINT(41.33786272340385 73.21874977478224) bank70786 +70787 POINT(39.764033570192254 74.87342100514412) bank70787 +70788 POINT(41.55378970412513 74.47333956969791) bank70788 +70789 POINT(40.064589628126456 73.57624514808313) bank70789 +70790 POINT(40.648916486108426 74.19856179113961) bank70790 +70791 POINT(40.55092018849802 74.0558749734498) bank70791 +70792 POINT(40.25787355410482 73.98794945724705) bank70792 +70793 POINT(40.317783548957394 73.96078211687788) bank70793 +70794 POINT(40.109087496155304 74.62974810662472) bank70794 +70795 POINT(41.6603902131155 73.1314418517096) bank70795 +70796 POINT(40.64692780762052 73.52300722414103) bank70796 +70797 POINT(41.03989024018187 74.65199096359805) bank70797 +70798 POINT(40.34572521849273 73.22471827844976) bank70798 +70799 POINT(41.30547523849339 74.68390623911554) bank70799 +70800 POINT(41.70213201040454 74.880134188578) bank70800 +70801 POINT(40.30740299470019 73.21279751394876) bank70801 +70802 POINT(41.68173265073443 74.73673630971292) bank70802 +70803 POINT(40.92022669544179 74.32657536287248) bank70803 +70804 POINT(41.63500986789038 74.77317923224513) bank70804 +70805 POINT(41.29347909472466 74.43870854575586) bank70805 +70806 POINT(40.95646349993538 74.48758764893431) bank70806 +70807 POINT(41.47329611652873 73.33670393984353) bank70807 +70808 POINT(40.444645609799814 74.18968561046208) bank70808 +70809 POINT(39.840630879517285 73.45392380883669) bank70809 +70810 POINT(39.97615733511892 74.13978735306809) bank70810 +70811 POINT(40.36180960325601 74.89015694749786) bank70811 +70812 POINT(41.2503804244851 74.7113334018798) bank70812 +70813 POINT(40.42803108791804 73.85319656043879) bank70813 +70814 POINT(40.18061395972124 74.63847714538882) bank70814 +70815 POINT(41.364540953744424 74.43792908623809) bank70815 +70816 POINT(41.272727540200506 74.27760816968251) bank70816 +70817 POINT(41.61533329619982 74.88164183918167) bank70817 +70818 POINT(39.810786222361216 73.97169592097679) bank70818 +70819 POINT(41.190806288845096 73.29108320064049) bank70819 +70820 POINT(41.16086736580916 74.64201668310155) bank70820 +70821 POINT(41.05004282383975 73.90479720006235) bank70821 +70822 POINT(41.34665807100472 73.30122598954607) bank70822 +70823 POINT(41.61478198997098 74.30153422054256) bank70823 +70824 POINT(40.53858666716308 73.71604318751312) bank70824 +70825 POINT(40.30953781622156 74.78722310801292) bank70825 +70826 POINT(40.50852605160907 74.90999526310996) bank70826 +70827 POINT(39.74382472538094 74.78378242512258) bank70827 +70828 POINT(41.270431357238955 73.14218947855456) bank70828 +70829 POINT(40.51480724771224 73.76338832102354) bank70829 +70830 POINT(41.238489438594605 73.23742003918636) bank70830 +70831 POINT(41.48959566085802 74.08574473928876) bank70831 +70832 POINT(41.40822637660353 73.15013268873585) bank70832 +70833 POINT(41.644066239015324 73.44735168269544) bank70833 +70834 POINT(41.47875602075453 73.2598558405156) bank70834 +70835 POINT(41.34963594426629 73.26459178798169) bank70835 +70836 POINT(41.26707217117945 73.6677943947207) bank70836 +70837 POINT(40.989633085099946 73.04557433774917) bank70837 +70838 POINT(41.60763798497274 73.84709107951171) bank70838 +70839 POINT(41.51538082356274 74.26371978274324) bank70839 +70840 POINT(39.93858323294491 73.90472411106468) bank70840 +70841 POINT(41.256303572164434 73.5564189931638) bank70841 +70842 POINT(41.167041143784246 74.56862928689326) bank70842 +70843 POINT(41.639095533425035 73.34887183018326) bank70843 +70844 POINT(41.1814439840669 74.05424644210278) bank70844 +70845 POINT(40.193820905018526 73.38305144414396) bank70845 +70846 POINT(40.27401332935866 74.83885016367005) bank70846 +70847 POINT(40.309004121225435 74.99211719021814) bank70847 +70848 POINT(40.7818119671778 74.93344010110053) bank70848 +70849 POINT(40.05547719237218 74.11143493475022) bank70849 +70850 POINT(41.69452487905437 74.49690967389927) bank70850 +70851 POINT(40.05148981469739 74.48899968445009) bank70851 +70852 POINT(41.15530099373946 74.16831138754013) bank70852 +70853 POINT(40.42073564324314 74.41373866341756) bank70853 +70854 POINT(39.80030647991824 73.48102852817229) bank70854 +70855 POINT(41.59791878587953 74.3754053014355) bank70855 +70856 POINT(41.28764848087237 74.78825460784823) bank70856 +70857 POINT(39.941651807348705 73.82267922022363) bank70857 +70858 POINT(40.01306432386182 73.3383581175304) bank70858 +70859 POINT(40.708781292356086 73.6448006485185) bank70859 +70860 POINT(40.667936036176634 73.18021625271635) bank70860 +70861 POINT(40.64354072138669 74.82753619473853) bank70861 +70862 POINT(40.0936450959085 73.49616678590456) bank70862 +70863 POINT(41.46259715583013 74.81068936905986) bank70863 +70864 POINT(40.118527838830715 74.91727154968055) bank70864 +70865 POINT(40.738933456638506 74.36624963670151) bank70865 +70866 POINT(41.35399076001532 74.11831857156209) bank70866 +70867 POINT(41.57692742751577 73.33010770341438) bank70867 +70868 POINT(41.02415199678489 73.89533253256991) bank70868 +70869 POINT(40.81558434087519 73.45171010071647) bank70869 +70870 POINT(39.95999790761222 74.81847716034805) bank70870 +70871 POINT(40.57615151995567 73.63309280628528) bank70871 +70872 POINT(40.80717134590401 73.70661819041703) bank70872 +70873 POINT(39.92831898162618 73.6309807384836) bank70873 +70874 POINT(40.56885742775608 73.4470926639847) bank70874 +70875 POINT(40.28336818547941 74.78713744115772) bank70875 +70876 POINT(40.07879634232433 74.2302083504445) bank70876 +70877 POINT(40.01161760813942 74.97214030293583) bank70877 +70878 POINT(41.71048386157202 74.80464692881301) bank70878 +70879 POINT(41.31698057703393 73.01338360102731) bank70879 +70880 POINT(40.246012673398205 73.15034943660434) bank70880 +70881 POINT(41.56992385805205 74.68138916672733) bank70881 +70882 POINT(40.51412173483552 74.83234445031493) bank70882 +70883 POINT(39.85210024143732 74.71866545019643) bank70883 +70884 POINT(39.78897294209894 73.8664773557974) bank70884 +70885 POINT(40.26491361617599 73.45969513408907) bank70885 +70886 POINT(41.189496597807064 74.23846358672071) bank70886 +70887 POINT(41.66033420281847 74.77960307507696) bank70887 +70888 POINT(41.488402276106925 73.15981117964) bank70888 +70889 POINT(40.972844039062906 74.44300617415246) bank70889 +70890 POINT(39.73405998572085 74.64648693937674) bank70890 +70891 POINT(41.60186937091035 75.00338333975522) bank70891 +70892 POINT(40.81997807135776 74.81453207337384) bank70892 +70893 POINT(40.023610030117105 74.93158856595711) bank70893 +70894 POINT(40.3355294998545 73.91969832425205) bank70894 +70895 POINT(39.87714203132249 74.11902359604089) bank70895 +70896 POINT(41.619448552911784 73.59338951737848) bank70896 +70897 POINT(40.905641958333504 74.90725903113699) bank70897 +70898 POINT(40.45006244377847 73.71655752762767) bank70898 +70899 POINT(41.111746053549396 73.34045069301877) bank70899 +70900 POINT(39.89491775721172 73.63394533545143) bank70900 +70901 POINT(39.72391726510029 74.97130789068551) bank70901 +70902 POINT(41.03000322128911 74.67756925830187) bank70902 +70903 POINT(41.021469512545096 74.97245176637776) bank70903 +70904 POINT(40.384225957489235 73.12869761081353) bank70904 +70905 POINT(41.23291845847303 73.76367526624243) bank70905 +70906 POINT(40.92608549873979 73.43490993168888) bank70906 +70907 POINT(40.22411946901616 74.54145544315293) bank70907 +70908 POINT(41.1580546737624 73.85350987251049) bank70908 +70909 POINT(39.85208205547941 74.4315804533061) bank70909 +70910 POINT(40.69973540219977 73.1205132245564) bank70910 +70911 POINT(41.31862574312029 73.65987232039225) bank70911 +70912 POINT(39.79686646348826 73.66211691768609) bank70912 +70913 POINT(40.98526679593768 73.69205718172219) bank70913 +70914 POINT(40.30385662432303 74.62149517610824) bank70914 +70915 POINT(41.16608217368992 74.20272552450693) bank70915 +70916 POINT(41.40712441173721 74.05035436702194) bank70916 +70917 POINT(40.040218236558225 73.9440378522915) bank70917 +70918 POINT(41.323009690673096 74.07394402597421) bank70918 +70919 POINT(41.41408107799714 74.31937948524184) bank70919 +70920 POINT(41.20502525953364 73.38826749102614) bank70920 +70921 POINT(40.91384864087196 73.86567183524505) bank70921 +70922 POINT(41.31566458042238 74.23171776166592) bank70922 +70923 POINT(40.060174313854176 73.71643668836593) bank70923 +70924 POINT(40.372017269691604 73.93991883832942) bank70924 +70925 POINT(41.57786923443995 73.84608009969648) bank70925 +70926 POINT(40.5181413398454 73.04466524798215) bank70926 +70927 POINT(41.47368085293344 74.79978049804805) bank70927 +70928 POINT(40.8531211466823 74.1700697926294) bank70928 +70929 POINT(40.42914201683111 74.90317157260398) bank70929 +70930 POINT(40.75945766457083 74.77506015318765) bank70930 +70931 POINT(40.33141410199358 73.4956069150156) bank70931 +70932 POINT(41.05757187690382 74.96848764627465) bank70932 +70933 POINT(40.57097110991831 74.62750993190589) bank70933 +70934 POINT(40.31017808317843 74.38116385814419) bank70934 +70935 POINT(41.59775967093526 73.03310357513901) bank70935 +70936 POINT(40.67362899006563 73.8334411240742) bank70936 +70937 POINT(40.61295688558768 74.74397340429942) bank70937 +70938 POINT(40.35631789523683 73.89427678320295) bank70938 +70939 POINT(41.55171329545511 73.1852572437794) bank70939 +70940 POINT(40.441484637112225 74.60643209648676) bank70940 +70941 POINT(40.15158419970716 73.8805358123844) bank70941 +70942 POINT(40.8366108015673 73.34023292877535) bank70942 +70943 POINT(40.00362414583373 74.46193201025423) bank70943 +70944 POINT(41.30752400476224 74.31165216156667) bank70944 +70945 POINT(40.42155248205255 73.69166021344826) bank70945 +70946 POINT(40.840833540622846 73.78061608058339) bank70946 +70947 POINT(39.86527271061948 74.37654146835612) bank70947 +70948 POINT(40.55829860951846 74.01418515316627) bank70948 +70949 POINT(41.51074735457043 73.39559203490788) bank70949 +70950 POINT(40.77799813233318 74.04351036440917) bank70950 +70951 POINT(40.977697792721095 73.32649190860478) bank70951 +70952 POINT(39.879729344727146 73.78671331664985) bank70952 +70953 POINT(40.46999973391401 73.72195073860236) bank70953 +70954 POINT(39.965205325448665 73.19987510520143) bank70954 +70955 POINT(40.04489258120831 74.20854797256851) bank70955 +70956 POINT(41.61150819175796 73.38483432753408) bank70956 +70957 POINT(40.476942280311526 74.37353224149928) bank70957 +70958 POINT(40.382558754402716 73.32083368530822) bank70958 +70959 POINT(39.99604748375462 73.53976178975284) bank70959 +70960 POINT(40.33540175789393 74.33846239798925) bank70960 +70961 POINT(39.93930428678244 74.38813163448445) bank70961 +70962 POINT(41.06155725109653 74.78291083560865) bank70962 +70963 POINT(39.90659997884367 74.73767752482313) bank70963 +70964 POINT(41.33582593615003 73.06604718942674) bank70964 +70965 POINT(40.880074626986975 73.41489211989689) bank70965 +70966 POINT(41.609035150352085 73.86160811855318) bank70966 +70967 POINT(41.19183849862321 73.51473268936653) bank70967 +70968 POINT(40.225025390102104 74.39633501323243) bank70968 +70969 POINT(40.96071647187004 73.47813481959105) bank70969 +70970 POINT(40.47332455230686 73.05457379985054) bank70970 +70971 POINT(39.83872618261785 74.96835334570414) bank70971 +70972 POINT(40.42841147567893 74.39641653995254) bank70972 +70973 POINT(40.148887560570756 74.98771400419184) bank70973 +70974 POINT(39.907600744455216 73.49520257093184) bank70974 +70975 POINT(40.236290340273754 73.6336114532934) bank70975 +70976 POINT(41.5944280234604 73.3916490412868) bank70976 +70977 POINT(39.99502261131698 73.92686069621199) bank70977 +70978 POINT(40.27828419215674 74.2204649204693) bank70978 +70979 POINT(39.76294284437581 74.63871746253366) bank70979 +70980 POINT(40.724150158173025 73.01121970655771) bank70980 +70981 POINT(41.0573924937726 74.50230062591889) bank70981 +70982 POINT(41.147704694242 73.62905126548338) bank70982 +70983 POINT(40.35669868764157 74.77931293166641) bank70983 +70984 POINT(40.02540031125593 74.33534299248231) bank70984 +70985 POINT(41.226297255396844 73.63488677090113) bank70985 +70986 POINT(41.647877870434534 74.80661609714022) bank70986 +70987 POINT(41.350888437482254 74.16395437575862) bank70987 +70988 POINT(41.33362575590447 74.44728980464085) bank70988 +70989 POINT(39.80163989311587 74.68377115155558) bank70989 +70990 POINT(39.89608833613126 74.49836735434157) bank70990 +70991 POINT(41.578477881078705 73.87318910727161) bank70991 +70992 POINT(39.739422969627356 73.33572367456951) bank70992 +70993 POINT(41.68752873185911 74.62913202444602) bank70993 +70994 POINT(40.923028954264694 73.06939985144825) bank70994 +70995 POINT(41.680381780844385 73.52848195067628) bank70995 +70996 POINT(40.351744862052485 74.9378707462769) bank70996 +70997 POINT(40.68900856913682 74.55991933936373) bank70997 +70998 POINT(41.01862331342227 74.10672280260036) bank70998 +70999 POINT(41.62972539441396 74.29220109951086) bank70999 +71000 POINT(40.52842221327275 73.44244850359873) bank71000 +71001 POINT(41.03001945092571 73.75265949611331) bank71001 +71002 POINT(40.45480974257439 73.01166705634718) bank71002 +71003 POINT(40.466811426754816 73.31697090761948) bank71003 +71004 POINT(40.23182830096087 73.46056674619054) bank71004 +71005 POINT(40.51411080952517 73.8027058727935) bank71005 +71006 POINT(39.784624089766425 74.26386255128914) bank71006 +71007 POINT(40.4252901502251 73.57300269440226) bank71007 +71008 POINT(40.694061671384794 73.05455074648896) bank71008 +71009 POINT(39.98741946720636 73.80813165918823) bank71009 +71010 POINT(40.92704225718993 74.89988985356624) bank71010 +71011 POINT(40.39588997291983 73.02673641627801) bank71011 +71012 POINT(39.784491221763176 74.98780094703274) bank71012 +71013 POINT(40.119718307836294 74.63000764798116) bank71013 +71014 POINT(41.237216325261315 74.51253812703033) bank71014 +71015 POINT(39.84009074932129 73.58919150661305) bank71015 +71016 POINT(41.03751196200708 74.59096149524379) bank71016 +71017 POINT(41.15243448302347 74.18228670140533) bank71017 +71018 POINT(40.557701197672166 74.62097327328381) bank71018 +71019 POINT(40.61548022807031 73.69290859644485) bank71019 +71020 POINT(39.90816696825413 74.85292329983089) bank71020 +71021 POINT(40.3536363617962 74.57790595722165) bank71021 +71022 POINT(41.66283154949581 74.06719998902122) bank71022 +71023 POINT(40.98083062444165 73.12384823880026) bank71023 +71024 POINT(40.95107466879237 74.96455096111943) bank71024 +71025 POINT(41.14993119555479 73.89275394534104) bank71025 +71026 POINT(41.28403888589153 74.528086286664) bank71026 +71027 POINT(40.656359061025405 74.46516424422849) bank71027 +71028 POINT(41.46311116807688 73.34855059878296) bank71028 +71029 POINT(41.02894046745968 73.03435561327822) bank71029 +71030 POINT(41.250972119451575 74.02974964493224) bank71030 +71031 POINT(40.06793589351659 74.34172276409993) bank71031 +71032 POINT(40.627538869310015 74.77027178652305) bank71032 +71033 POINT(41.088289681493066 73.3093818336429) bank71033 +71034 POINT(41.017328258653066 74.03489567923457) bank71034 +71035 POINT(39.8203829961723 74.86849580935888) bank71035 +71036 POINT(40.205062690825564 74.88458123868514) bank71036 +71037 POINT(39.7714906733088 73.6787732255039) bank71037 +71038 POINT(39.90603579011884 74.39162790670817) bank71038 +71039 POINT(40.19633764509536 74.78539026675716) bank71039 +71040 POINT(41.10110981068913 74.16536758576886) bank71040 +71041 POINT(40.73228261608858 74.91404211011599) bank71041 +71042 POINT(40.836183872264826 74.39968217923425) bank71042 +71043 POINT(39.8536254127691 74.64799742949886) bank71043 +71044 POINT(40.61046251957249 73.33630894406991) bank71044 +71045 POINT(41.01671552260469 73.73281744223114) bank71045 +71046 POINT(41.65716434196566 74.83599225393621) bank71046 +71047 POINT(41.568282993480096 74.24743267162253) bank71047 +71048 POINT(39.72542400972126 73.11787144828662) bank71048 +71049 POINT(41.36965345597455 73.70718892825435) bank71049 +71050 POINT(40.17045265963292 74.04478096606462) bank71050 +71051 POINT(40.420278205835466 74.63464049128733) bank71051 +71052 POINT(40.71103266334638 73.86974820209376) bank71052 +71053 POINT(40.822660565367016 73.61840706256493) bank71053 +71054 POINT(40.562184683953795 73.90084600937618) bank71054 +71055 POINT(39.81226960498057 74.22789034427754) bank71055 +71056 POINT(40.09323061324533 73.65621182838483) bank71056 +71057 POINT(40.19461396758778 73.92405185567415) bank71057 +71058 POINT(40.43869044593485 74.13400360585565) bank71058 +71059 POINT(39.91126870748447 73.09992846740572) bank71059 +71060 POINT(41.316452147681275 74.78035405122985) bank71060 +71061 POINT(39.939701434267135 74.07321819922456) bank71061 +71062 POINT(39.78601240814165 74.1612242900116) bank71062 +71063 POINT(41.23708790524041 74.13256689107594) bank71063 +71064 POINT(40.846289135908435 73.20880859827066) bank71064 +71065 POINT(40.37192548232939 75.00071087191041) bank71065 +71066 POINT(41.110531916089045 74.39114579473868) bank71066 +71067 POINT(41.7114286697358 73.95796790677328) bank71067 +71068 POINT(39.86178456043352 73.94697962240305) bank71068 +71069 POINT(40.94133410508118 74.99960022272411) bank71069 +71070 POINT(40.21682880790993 74.66291312743712) bank71070 +71071 POINT(40.5703827138726 73.49670751946542) bank71071 +71072 POINT(41.2906344332883 74.44256569613766) bank71072 +71073 POINT(41.43012075948278 74.92103986269721) bank71073 +71074 POINT(40.32743766589943 74.28066829250564) bank71074 +71075 POINT(40.61914925008578 74.15530705761226) bank71075 +71076 POINT(40.42624737613238 73.28706432101802) bank71076 +71077 POINT(41.35900812783908 74.7765126027436) bank71077 +71078 POINT(40.8477834900852 73.60139540383898) bank71078 +71079 POINT(39.90956518354812 73.06517709864964) bank71079 +71080 POINT(40.408464280998494 73.03237739880193) bank71080 +71081 POINT(40.174539905985974 74.09083855405325) bank71081 +71082 POINT(40.13622707611664 73.28698436371229) bank71082 +71083 POINT(39.923876315477834 74.54843278154148) bank71083 +71084 POINT(41.26761209962996 73.92737928852337) bank71084 +71085 POINT(41.35755845835528 74.78061673173858) bank71085 +71086 POINT(40.51497103262098 73.11406496004321) bank71086 +71087 POINT(40.811543937660815 73.04816995534028) bank71087 +71088 POINT(41.59847779794284 73.04012021497444) bank71088 +71089 POINT(40.45002050508159 74.43495568435189) bank71089 +71090 POINT(41.593491166073726 73.19586546830128) bank71090 +71091 POINT(39.997052962575594 73.60266576719158) bank71091 +71092 POINT(39.75767708040456 74.81405782377114) bank71092 +71093 POINT(40.627586597390184 74.19139753049681) bank71093 +71094 POINT(41.460010455382765 74.90667686433943) bank71094 +71095 POINT(41.554038168316026 73.06549728065991) bank71095 +71096 POINT(40.75568973356857 73.94828626393988) bank71096 +71097 POINT(40.56129000127409 74.31714046541566) bank71097 +71098 POINT(39.73131979205075 73.77235248921895) bank71098 +71099 POINT(41.33736084269184 74.67920467415307) bank71099 +71100 POINT(40.07495281176023 73.51164083079561) bank71100 +71101 POINT(40.982207156466096 74.77051235582148) bank71101 +71102 POINT(40.304161036334314 74.1986579454131) bank71102 +71103 POINT(39.925592067009525 73.29755055231385) bank71103 +71104 POINT(41.24060246114079 73.76355774905502) bank71104 +71105 POINT(40.61258863511787 74.29457926055262) bank71105 +71106 POINT(41.29546294890726 74.36214922612874) bank71106 +71107 POINT(39.735044064585935 74.27873365237738) bank71107 +71108 POINT(40.40773842947123 74.91088303533977) bank71108 +71109 POINT(40.2867551750233 74.93110221471363) bank71109 +71110 POINT(40.0798686634248 73.08374711187936) bank71110 +71111 POINT(40.25198033646406 74.62745877468313) bank71111 +71112 POINT(40.61224903979337 74.88773462411775) bank71112 +71113 POINT(39.87599614095399 73.86436626965926) bank71113 +71114 POINT(40.0432106682425 74.68502637791526) bank71114 +71115 POINT(40.846323875765854 73.01534963851559) bank71115 +71116 POINT(40.505830975581866 73.88896756919732) bank71116 +71117 POINT(40.55169780037974 73.56692106740945) bank71117 +71118 POINT(40.422441667503165 74.70222078594422) bank71118 +71119 POINT(41.67945913024123 74.19441210452706) bank71119 +71120 POINT(39.96495369939793 73.63808118195658) bank71120 +71121 POINT(41.061525825976744 73.9876133536232) bank71121 +71122 POINT(40.05792602534437 74.45869568151113) bank71122 +71123 POINT(40.70797940283433 73.11187965230995) bank71123 +71124 POINT(40.36911984170695 73.81565611150091) bank71124 +71125 POINT(40.83036076241849 73.9214407221817) bank71125 +71126 POINT(41.70607603630086 73.91568966168623) bank71126 +71127 POINT(40.044177670257895 74.95084529367489) bank71127 +71128 POINT(41.49021077276808 73.16789809546282) bank71128 +71129 POINT(41.38412448747815 73.7118067856656) bank71129 +71130 POINT(40.81112444080903 73.33177805475493) bank71130 +71131 POINT(40.18147509676362 74.73114522279826) bank71131 +71132 POINT(41.33538164790882 74.31246086301148) bank71132 +71133 POINT(41.44158319999366 73.54767966784635) bank71133 +71134 POINT(40.893648908686856 73.02829152796919) bank71134 +71135 POINT(40.189676516055265 73.98169351028697) bank71135 +71136 POINT(40.361144583988676 74.50840869911417) bank71136 +71137 POINT(40.42767217652222 73.09123408830672) bank71137 +71138 POINT(40.200543550105635 74.34039990175265) bank71138 +71139 POINT(40.118794763295014 74.0139806066825) bank71139 +71140 POINT(40.97422934745695 74.43570680546146) bank71140 +71141 POINT(41.22583501883454 73.41059343585371) bank71141 +71142 POINT(41.359720182756355 74.39143252525768) bank71142 +71143 POINT(41.21719001142127 73.65457658706048) bank71143 +71144 POINT(40.73265376213534 73.64635498931999) bank71144 +71145 POINT(40.07644750716437 74.73154704005363) bank71145 +71146 POINT(41.39640618376991 74.83941796101098) bank71146 +71147 POINT(40.77911141149537 74.07655943413202) bank71147 +71148 POINT(40.91338253369855 74.00400363962926) bank71148 +71149 POINT(41.33516584384658 74.67733495854746) bank71149 +71150 POINT(40.39549372173049 74.47894338235707) bank71150 +71151 POINT(40.30618490760483 73.70029507812325) bank71151 +71152 POINT(40.71705061823212 73.29472416231745) bank71152 +71153 POINT(39.844739803754045 73.59476544624606) bank71153 +71154 POINT(40.041866969249156 73.99409737618831) bank71154 +71155 POINT(40.753853614645635 74.88215696482015) bank71155 +71156 POINT(41.25505259731663 73.44846804644625) bank71156 +71157 POINT(40.71932933740701 74.19616500538001) bank71157 +71158 POINT(40.26474584305303 74.20510158103313) bank71158 +71159 POINT(41.29447953997831 74.51343679935073) bank71159 +71160 POINT(40.897939274496125 73.60338655923863) bank71160 +71161 POINT(40.8856349752879 73.41452218162068) bank71161 +71162 POINT(40.796843175398536 74.81577008065062) bank71162 +71163 POINT(40.27280315241858 73.998553644043) bank71163 +71164 POINT(40.54394676670595 73.84241889349005) bank71164 +71165 POINT(41.493538583280106 74.39626491922995) bank71165 +71166 POINT(40.40286235035718 74.48992785292207) bank71166 +71167 POINT(40.452841803052564 74.1370339153238) bank71167 +71168 POINT(40.45336791816226 73.65863812498674) bank71168 +71169 POINT(39.969892429524144 74.39301101389293) bank71169 +71170 POINT(41.385742856199464 74.73150940597688) bank71170 +71171 POINT(41.274698585464115 74.14120721204037) bank71171 +71172 POINT(41.03168835126923 73.2383695061274) bank71172 +71173 POINT(40.91163380082683 73.19809219269312) bank71173 +71174 POINT(40.74140962548855 74.82872811728963) bank71174 +71175 POINT(39.88060569002891 73.53323359994359) bank71175 +71176 POINT(40.99226018812822 74.67677209975385) bank71176 +71177 POINT(39.77057949996099 73.52506451111941) bank71177 +71178 POINT(41.05154066050624 73.59380045249749) bank71178 +71179 POINT(40.701534399863085 74.26859504505568) bank71179 +71180 POINT(41.175022315840174 74.89315687711255) bank71180 +71181 POINT(40.73773876553131 74.30757068890033) bank71181 +71182 POINT(41.260444604652385 73.21881195883022) bank71182 +71183 POINT(40.29624609817927 74.86616831410012) bank71183 +71184 POINT(40.27845029134774 73.41568617859458) bank71184 +71185 POINT(40.823387645691945 73.38361135436193) bank71185 +71186 POINT(41.02668463399666 73.04703094514024) bank71186 +71187 POINT(41.41499471305459 74.41660312321414) bank71187 +71188 POINT(41.07594976995425 74.09907346250495) bank71188 +71189 POINT(41.12497766622106 73.95134025769079) bank71189 +71190 POINT(39.92443086717901 73.34806908885596) bank71190 +71191 POINT(41.10371637253987 73.77190760697125) bank71191 +71192 POINT(40.23665523183139 73.49597551246573) bank71192 +71193 POINT(41.61487423428594 74.20699859106442) bank71193 +71194 POINT(41.03034336173512 74.50177372674317) bank71194 +71195 POINT(40.52917523978621 73.040783854855) bank71195 +71196 POINT(41.339164499577706 73.49161600295862) bank71196 +71197 POINT(40.15720395041438 74.87042100610405) bank71197 +71198 POINT(40.18896114097828 74.53227163134538) bank71198 +71199 POINT(39.92097048861514 74.51554745497515) bank71199 +71200 POINT(41.2937904517045 74.8796341715404) bank71200 +71201 POINT(41.39605232063043 73.30742943705513) bank71201 +71202 POINT(40.86564364584833 74.74163511016381) bank71202 +71203 POINT(41.40057502096498 73.30868376231138) bank71203 +71204 POINT(40.37690685761888 73.55155838626948) bank71204 +71205 POINT(41.627558363139144 73.8915262319251) bank71205 +71206 POINT(41.68100008450316 74.90598236951567) bank71206 +71207 POINT(40.474308413416054 74.70106351220994) bank71207 +71208 POINT(39.944012950086055 73.24528934347705) bank71208 +71209 POINT(40.87024667372725 73.3508678243731) bank71209 +71210 POINT(40.17427038823926 74.54976394843975) bank71210 +71211 POINT(39.97194020410859 73.98772388714656) bank71211 +71212 POINT(40.25626324212768 74.03044037245115) bank71212 +71213 POINT(40.928789589782 73.72809467292795) bank71213 +71214 POINT(40.22213648033093 74.78756248043625) bank71214 +71215 POINT(41.45692294933267 73.26981322588902) bank71215 +71216 POINT(40.75264810868575 73.8221697734977) bank71216 +71217 POINT(41.200985437732 74.84286591992934) bank71217 +71218 POINT(41.387647459995996 73.885299471177) bank71218 +71219 POINT(39.93182102548851 73.32545263562122) bank71219 +71220 POINT(39.80865363262947 74.04770142016797) bank71220 +71221 POINT(40.95731126951942 74.71138895893333) bank71221 +71222 POINT(40.08626675960331 73.47547356773671) bank71222 +71223 POINT(40.79375419525964 73.75102536373299) bank71223 +71224 POINT(40.934411121579366 74.93326055557718) bank71224 +71225 POINT(40.67031635110513 74.61044557688048) bank71225 +71226 POINT(40.54581991502711 74.65988360719146) bank71226 +71227 POINT(40.971804436083715 73.02832709508928) bank71227 +71228 POINT(41.230709105795434 74.06175233552035) bank71228 +71229 POINT(41.33771141011403 74.13307423560369) bank71229 +71230 POINT(40.01501175939671 73.8472090665694) bank71230 +71231 POINT(41.65236344055984 73.24649511631786) bank71231 +71232 POINT(40.312782134140036 74.99773876806937) bank71232 +71233 POINT(39.83129857032393 73.79553268975411) bank71233 +71234 POINT(41.433143752803275 74.46291551380772) bank71234 +71235 POINT(40.27110183043386 74.67265306293041) bank71235 +71236 POINT(41.386272203976624 74.85928117812568) bank71236 +71237 POINT(41.02788264761873 73.49263485766576) bank71237 +71238 POINT(41.67921083261827 74.76015070893575) bank71238 +71239 POINT(39.871904975840884 73.39615414576069) bank71239 +71240 POINT(41.59724740372963 74.73969568030421) bank71240 +71241 POINT(41.294054450834224 73.16928124671298) bank71241 +71242 POINT(40.50698786453823 74.29607966907328) bank71242 +71243 POINT(41.43656587334189 74.36162669349564) bank71243 +71244 POINT(41.09070695411572 74.38149134455145) bank71244 +71245 POINT(40.76169674646163 73.04943721657078) bank71245 +71246 POINT(40.52361982291448 74.38072016293529) bank71246 +71247 POINT(40.28409243570948 73.81058583519396) bank71247 +71248 POINT(40.00512351293103 74.27002883824647) bank71248 +71249 POINT(41.32258830386552 73.76928532224728) bank71249 +71250 POINT(41.53237895111863 74.95288138893326) bank71250 +71251 POINT(40.295130344879766 73.02717244380212) bank71251 +71252 POINT(41.31385559741018 73.58783067761422) bank71252 +71253 POINT(41.645309965801516 73.4945708320267) bank71253 +71254 POINT(41.02492276284456 73.60700978222941) bank71254 +71255 POINT(39.767382222206294 74.14370066851001) bank71255 +71256 POINT(41.354759073033044 74.24102685976098) bank71256 +71257 POINT(40.71060839671739 74.71887694049686) bank71257 +71258 POINT(40.40110475770713 73.64558066788395) bank71258 +71259 POINT(41.68295629136209 74.01164262736748) bank71259 +71260 POINT(41.378607454543115 74.09820772016819) bank71260 +71261 POINT(40.559255951598345 74.57180880856674) bank71261 +71262 POINT(40.12205816570568 73.69800202397172) bank71262 +71263 POINT(40.659327585365496 74.9162343630893) bank71263 +71264 POINT(41.27975280918005 73.47963755584665) bank71264 +71265 POINT(40.03855871087785 73.26170948050428) bank71265 +71266 POINT(40.285486382493126 74.48754457786521) bank71266 +71267 POINT(40.860711996511434 74.05346428351801) bank71267 +71268 POINT(41.416037394770555 73.04487470747637) bank71268 +71269 POINT(40.25965422299236 73.54531649241589) bank71269 +71270 POINT(39.964901539879534 73.64210171442204) bank71270 +71271 POINT(41.61635295251904 73.25472975805772) bank71271 +71272 POINT(39.99000236099803 74.21178522454304) bank71272 +71273 POINT(39.72372195758954 73.72059476374426) bank71273 +71274 POINT(41.185905877952116 74.15811468281046) bank71274 +71275 POINT(40.82770382610594 74.82773076210333) bank71275 +71276 POINT(41.20858648046385 73.62961047813258) bank71276 +71277 POINT(39.7150896773079 73.40850375562282) bank71277 +71278 POINT(40.092077554964334 74.91099513952231) bank71278 +71279 POINT(41.13535357519905 74.87593816021814) bank71279 +71280 POINT(40.00458325396645 74.81590998487273) bank71280 +71281 POINT(41.30289829504979 73.48547020932479) bank71281 +71282 POINT(41.054123741356705 74.36121281748463) bank71282 +71283 POINT(41.591312738724824 74.01299467868955) bank71283 +71284 POINT(40.95246782351864 74.7532139861345) bank71284 +71285 POINT(40.518353745160105 74.06464836086653) bank71285 +71286 POINT(39.83504060693514 74.80112963036055) bank71286 +71287 POINT(41.52684719701861 73.22506602130979) bank71287 +71288 POINT(39.88962612313202 74.60515146846805) bank71288 +71289 POINT(41.382086135131864 73.69544681970704) bank71289 +71290 POINT(39.7504089275047 74.74907157499028) bank71290 +71291 POINT(39.87098182377179 73.59193066242638) bank71291 +71292 POINT(39.723978970228515 73.234308782752) bank71292 +71293 POINT(40.69612086435734 73.78668107543261) bank71293 +71294 POINT(40.80523908417272 73.16675319398868) bank71294 +71295 POINT(41.57071429281474 73.56079635449612) bank71295 +71296 POINT(40.48115012537777 74.31262315830789) bank71296 +71297 POINT(40.405877146086176 74.14770669735496) bank71297 +71298 POINT(40.266692664810655 73.45831479991568) bank71298 +71299 POINT(40.273924287276444 73.6291903681258) bank71299 +71300 POINT(41.63033350514324 74.40324233606212) bank71300 +71301 POINT(41.137999768008186 74.35916634465826) bank71301 +71302 POINT(39.74452636635254 74.4917646624802) bank71302 +71303 POINT(40.54341701796857 73.56490452798408) bank71303 +71304 POINT(40.403790711983895 74.24933183269052) bank71304 +71305 POINT(40.47176131109136 74.75721693401445) bank71305 +71306 POINT(40.866178931821274 74.59174759833363) bank71306 +71307 POINT(40.89675002087038 73.10498598929777) bank71307 +71308 POINT(40.01021096878106 74.19317464654029) bank71308 +71309 POINT(40.96553975693222 73.06419805933439) bank71309 +71310 POINT(40.12737064167948 74.93999691492836) bank71310 +71311 POINT(40.08663130140496 73.74456602378685) bank71311 +71312 POINT(40.02980959059011 73.34639697696691) bank71312 +71313 POINT(40.3756984338232 74.58303391025005) bank71313 +71314 POINT(41.477792749874176 73.46862912293354) bank71314 +71315 POINT(40.127932416882466 73.54294615625568) bank71315 +71316 POINT(40.77378254162763 74.64374942019633) bank71316 +71317 POINT(40.69970188024028 74.75602673001364) bank71317 +71318 POINT(40.21912415260023 73.58829940338856) bank71318 +71319 POINT(40.04995162544325 74.84287937621735) bank71319 +71320 POINT(40.86213632369761 73.26126391026193) bank71320 +71321 POINT(41.29021599028659 74.77070306716972) bank71321 +71322 POINT(39.883032404183595 73.14374273983846) bank71322 +71323 POINT(40.35979416382911 73.01404750593618) bank71323 +71324 POINT(41.00015025490847 73.3384243210232) bank71324 +71325 POINT(41.59233296813989 73.09588885146601) bank71325 +71326 POINT(41.572847717125946 73.32545543114475) bank71326 +71327 POINT(41.450223533607655 74.32799989220557) bank71327 +71328 POINT(40.625954672520734 74.69482366986779) bank71328 +71329 POINT(41.16949409682594 73.18886630409185) bank71329 +71330 POINT(39.780786841894205 73.18444116725988) bank71330 +71331 POINT(40.52439854902986 73.60251803914197) bank71331 +71332 POINT(41.221419585697994 73.29209561997511) bank71332 +71333 POINT(39.842576493318084 74.36799721072988) bank71333 +71334 POINT(40.09036453859178 74.26599747969752) bank71334 +71335 POINT(40.42189548233711 73.52928424509635) bank71335 +71336 POINT(40.60919557662478 74.89057813801034) bank71336 +71337 POINT(41.69037143281039 73.05997525972856) bank71337 +71338 POINT(41.30170103476979 73.53272586772962) bank71338 +71339 POINT(40.34927080783613 73.35026808537289) bank71339 +71340 POINT(40.39854468104703 73.54505358328011) bank71340 +71341 POINT(41.449254038211656 73.95182678318517) bank71341 +71342 POINT(40.091874177427535 74.9828834406496) bank71342 +71343 POINT(41.634334960061715 74.92486138566849) bank71343 +71344 POINT(40.773596706782925 73.59461204057446) bank71344 +71345 POINT(40.24620344831956 73.68711630544975) bank71345 +71346 POINT(41.15300518731036 74.80360181196994) bank71346 +71347 POINT(40.006499305276144 74.70247728098438) bank71347 +71348 POINT(41.05486852265181 74.24172769855235) bank71348 +71349 POINT(40.86016701968561 73.77513069089025) bank71349 +71350 POINT(40.670867561563604 73.03197664965396) bank71350 +71351 POINT(41.15575371628619 73.32827130533413) bank71351 +71352 POINT(41.517908613065295 73.83337156677862) bank71352 +71353 POINT(39.75876428760617 73.70947440646937) bank71353 +71354 POINT(41.17461308241576 73.06621956199507) bank71354 +71355 POINT(40.120167388964816 73.5892317223959) bank71355 +71356 POINT(40.25553303566122 73.99094294396049) bank71356 +71357 POINT(41.01538146964442 73.240208258298) bank71357 +71358 POINT(41.41573775940886 73.60174958699665) bank71358 +71359 POINT(41.23764733421405 73.56409100391427) bank71359 +71360 POINT(40.81708546573493 73.25354216565813) bank71360 +71361 POINT(40.03943028948653 73.73569925631912) bank71361 +71362 POINT(40.46042532644517 73.24336182282578) bank71362 +71363 POINT(39.86400977139434 73.54552243489013) bank71363 +71364 POINT(40.74575795020919 73.8430173537711) bank71364 +71365 POINT(41.06742175628261 74.66266303748627) bank71365 +71366 POINT(41.08513316786379 74.08903464022559) bank71366 +71367 POINT(40.74697596141907 74.19887210067412) bank71367 +71368 POINT(39.89207467138774 74.01890653944902) bank71368 +71369 POINT(40.81483573989811 73.69910620204372) bank71369 +71370 POINT(40.783749279461226 73.14608232457701) bank71370 +71371 POINT(40.14277251015021 74.22758563707907) bank71371 +71372 POINT(40.04653446037044 74.96141577685536) bank71372 +71373 POINT(41.55002759161016 74.5602496329524) bank71373 +71374 POINT(41.46426945672297 73.76872924943275) bank71374 +71375 POINT(40.30085295043039 74.67832337085379) bank71375 +71376 POINT(39.779942678077006 73.94771418896019) bank71376 +71377 POINT(41.59033013359344 73.92179236666811) bank71377 +71378 POINT(41.322124226341685 73.63548259421404) bank71378 +71379 POINT(40.51358987755266 73.06608833908417) bank71379 +71380 POINT(40.73241134459305 74.15154578098017) bank71380 +71381 POINT(41.03869267613021 74.02161479622795) bank71381 +71382 POINT(39.71640646821443 74.56493676986433) bank71382 +71383 POINT(41.052142168577674 73.46060082845914) bank71383 +71384 POINT(41.26431815387932 74.75470435652616) bank71384 +71385 POINT(40.17669445389133 74.27719273752679) bank71385 +71386 POINT(41.03534590729844 73.79665423495973) bank71386 +71387 POINT(39.85090959596354 73.85507115832816) bank71387 +71388 POINT(41.519718063064126 73.06289031773744) bank71388 +71389 POINT(40.06125904425401 74.92504932184038) bank71389 +71390 POINT(40.31229552366758 74.89273543550976) bank71390 +71391 POINT(40.6115502453388 73.20984559902094) bank71391 +71392 POINT(40.07723332449582 74.02471884352835) bank71392 +71393 POINT(39.969931608999275 73.96460634801878) bank71393 +71394 POINT(40.45818637792657 74.62516739689403) bank71394 +71395 POINT(40.10947588878197 74.61238821485875) bank71395 +71396 POINT(40.547900014912805 74.03894544594918) bank71396 +71397 POINT(40.46960404655717 74.47474548931771) bank71397 +71398 POINT(41.18105260211653 74.61668693730842) bank71398 +71399 POINT(39.91742332087603 74.92634198260984) bank71399 +71400 POINT(39.85148410610221 74.62586218156522) bank71400 +71401 POINT(41.21809167741431 74.88760950239897) bank71401 +71402 POINT(40.918497452658336 74.9342889724173) bank71402 +71403 POINT(40.87691951770612 73.6106564661111) bank71403 +71404 POINT(41.637359710776096 74.86630600695017) bank71404 +71405 POINT(40.822730953342955 74.90351467414048) bank71405 +71406 POINT(41.55362868913938 73.46739904882948) bank71406 +71407 POINT(40.7791919235727 73.49410971919016) bank71407 +71408 POINT(41.5691795239043 74.14254100214636) bank71408 +71409 POINT(40.49077995115538 74.76360965360566) bank71409 +71410 POINT(40.039703991493624 73.31092783294254) bank71410 +71411 POINT(40.795785611535926 74.80649944161081) bank71411 +71412 POINT(39.98111149937145 73.86518522022342) bank71412 +71413 POINT(39.94499517052936 74.84704533463552) bank71413 +71414 POINT(41.650394067906774 73.74616965067821) bank71414 +71415 POINT(40.447215212721964 74.88619333827998) bank71415 +71416 POINT(41.1745738859467 73.04474683907146) bank71416 +71417 POINT(41.45638446955756 73.37280741560755) bank71417 +71418 POINT(41.3178000601052 74.61982153098721) bank71418 +71419 POINT(40.35049222970937 73.62077529963787) bank71419 +71420 POINT(39.87290795084798 74.56463795070896) bank71420 +71421 POINT(39.818182398936436 74.00922472387563) bank71421 +71422 POINT(40.213709180770934 74.6637361930234) bank71422 +71423 POINT(39.85587140233071 73.1427532931411) bank71423 +71424 POINT(41.25841662945151 74.68116101046407) bank71424 +71425 POINT(40.20072796499523 73.09136427132682) bank71425 +71426 POINT(41.70646269604093 74.98276048665964) bank71426 +71427 POINT(41.55316641733194 73.29660073934384) bank71427 +71428 POINT(41.322352757870235 73.19377722751113) bank71428 +71429 POINT(40.77300637768927 74.38697788635972) bank71429 +71430 POINT(40.00036310608636 73.60685638698807) bank71430 +71431 POINT(40.47769780600986 74.81253323436826) bank71431 +71432 POINT(40.89899482943439 74.51090989576113) bank71432 +71433 POINT(41.6544884510514 73.49395537352157) bank71433 +71434 POINT(41.07132593918401 73.96182449160247) bank71434 +71435 POINT(40.88408820058508 74.56937138500633) bank71435 +71436 POINT(41.28727210123738 73.22907238055231) bank71436 +71437 POINT(40.160330218862434 74.27344867611494) bank71437 +71438 POINT(40.25721944182178 73.0080102522664) bank71438 +71439 POINT(39.85653358196485 74.36938811253499) bank71439 +71440 POINT(39.85489394066002 73.12321721644213) bank71440 +71441 POINT(41.03382089743924 73.12121837211751) bank71441 +71442 POINT(40.25332715246968 73.63271336186833) bank71442 +71443 POINT(39.80321908776986 73.34290390693671) bank71443 +71444 POINT(40.4528704711596 74.67496029896458) bank71444 +71445 POINT(41.61717053661701 73.12525574590236) bank71445 +71446 POINT(41.00954954791716 74.6371162655409) bank71446 +71447 POINT(40.61666686735034 73.63265528444799) bank71447 +71448 POINT(40.30357757697347 74.75663863565843) bank71448 +71449 POINT(41.09062710263043 73.20392362694837) bank71449 +71450 POINT(39.76694783402472 74.49991813194262) bank71450 +71451 POINT(41.39349260120684 74.30972581385173) bank71451 +71452 POINT(40.49851791802516 74.19131339810856) bank71452 +71453 POINT(39.73403394525356 74.91935319425765) bank71453 +71454 POINT(41.30726715458444 73.58718132689246) bank71454 +71455 POINT(41.15795503919447 73.07134228879093) bank71455 +71456 POINT(41.47218440055952 74.18962750121166) bank71456 +71457 POINT(39.75642436725222 73.96283652007318) bank71457 +71458 POINT(41.451554540667814 73.73143036135916) bank71458 +71459 POINT(39.95490357685197 73.24103697226667) bank71459 +71460 POINT(41.53187635569628 73.99062695807895) bank71460 +71461 POINT(40.73389319004453 74.46236071661633) bank71461 +71462 POINT(39.74117865939115 74.9457168242476) bank71462 +71463 POINT(41.0975296876237 74.97606558002067) bank71463 +71464 POINT(40.36849408719641 74.08471104552052) bank71464 +71465 POINT(41.46762771269212 73.09586230264756) bank71465 +71466 POINT(39.77608373250635 74.09063309875276) bank71466 +71467 POINT(41.26472489990916 74.20288420135914) bank71467 +71468 POINT(40.68030593919588 74.95990346557637) bank71468 +71469 POINT(41.25040209656812 74.3195274780073) bank71469 +71470 POINT(41.65784301657781 73.42583101024077) bank71470 +71471 POINT(40.16350506915501 73.64584778527309) bank71471 +71472 POINT(40.74071454195623 74.86994121359396) bank71472 +71473 POINT(39.88639416401448 74.79619230194656) bank71473 +71474 POINT(39.75999887148387 73.31153811103694) bank71474 +71475 POINT(40.5031882913394 74.21433868280532) bank71475 +71476 POINT(40.695746699912 73.23225013086822) bank71476 +71477 POINT(40.261500205652624 73.72189541682611) bank71477 +71478 POINT(39.920654684130106 74.85004831495418) bank71478 +71479 POINT(40.768813655068655 74.39523346185256) bank71479 +71480 POINT(40.27517469436316 74.82778448706648) bank71480 +71481 POINT(40.06566142137283 74.68133027986536) bank71481 +71482 POINT(39.93990007619458 73.17178337586135) bank71482 +71483 POINT(40.57499431255591 73.402991970082) bank71483 +71484 POINT(40.40204607940623 74.08408649950545) bank71484 +71485 POINT(41.36848358835166 73.46976115366772) bank71485 +71486 POINT(40.59293997959442 74.04390946891527) bank71486 +71487 POINT(41.31730400652568 73.2489907717878) bank71487 +71488 POINT(41.552319326040205 73.0612153907012) bank71488 +71489 POINT(41.65981493584592 74.52943683956634) bank71489 +71490 POINT(39.742066478496454 73.05397831230574) bank71490 +71491 POINT(41.02909756172858 74.90031190297483) bank71491 +71492 POINT(41.03077369218566 73.66943216041989) bank71492 +71493 POINT(40.551630133606054 74.84858374424695) bank71493 +71494 POINT(40.744453139940134 74.87782173617445) bank71494 +71495 POINT(41.16189180208851 74.9364457768874) bank71495 +71496 POINT(40.0351487722877 74.2232684913925) bank71496 +71497 POINT(41.05166830753125 73.30387593496626) bank71497 +71498 POINT(41.005177802487786 73.04460532965868) bank71498 +71499 POINT(39.782629844133304 74.17042447044804) bank71499 +71500 POINT(39.791217390849376 74.77746144602207) bank71500 +71501 POINT(41.33761812550706 73.21361174111611) bank71501 +71502 POINT(41.40740234097785 74.48784216468287) bank71502 +71503 POINT(41.28926760835874 74.25642246042909) bank71503 +71504 POINT(41.390083001967504 73.98471304187743) bank71504 +71505 POINT(40.33883081391854 73.77851019100481) bank71505 +71506 POINT(40.94915268727381 74.20257857353825) bank71506 +71507 POINT(40.339193082866814 74.54960491506472) bank71507 +71508 POINT(40.31045340277129 74.60289963758252) bank71508 +71509 POINT(40.80770941713999 74.59252716710739) bank71509 +71510 POINT(40.404497987524536 73.89850154629642) bank71510 +71511 POINT(40.249973932463384 73.091438875068) bank71511 +71512 POINT(40.72788740788655 73.67706504774353) bank71512 +71513 POINT(40.41014054121781 75.00556757916246) bank71513 +71514 POINT(40.279880778070996 73.81370140057251) bank71514 +71515 POINT(41.432184148271176 73.26511525051828) bank71515 +71516 POINT(41.56144357797466 73.54698055076554) bank71516 +71517 POINT(41.62813332152262 74.49617601973787) bank71517 +71518 POINT(40.27038831020151 73.25663667047672) bank71518 +71519 POINT(41.477645132189856 73.55863379867213) bank71519 +71520 POINT(41.26258751371422 74.80430346816368) bank71520 +71521 POINT(40.76816484581636 73.61469396308453) bank71521 +71522 POINT(40.084331302794354 74.70092673544957) bank71522 +71523 POINT(41.11729538865307 74.03854172830526) bank71523 +71524 POINT(41.28366014413776 74.50617521705425) bank71524 +71525 POINT(40.14135894000497 73.00871400665706) bank71525 +71526 POINT(40.55300316987403 74.61137464433355) bank71526 +71527 POINT(41.35369601083206 73.21423867043309) bank71527 +71528 POINT(40.35199017229249 73.32414268652931) bank71528 +71529 POINT(41.20735703139209 74.69165819853954) bank71529 +71530 POINT(41.32305814424297 73.51240870568954) bank71530 +71531 POINT(39.74679466879472 74.58768876353265) bank71531 +71532 POINT(41.16198806136337 74.69331281486238) bank71532 +71533 POINT(40.398549498996815 73.11620243898531) bank71533 +71534 POINT(40.30785985259289 74.32396227405074) bank71534 +71535 POINT(40.636024776619124 74.74265654489679) bank71535 +71536 POINT(40.354824192876514 73.34861594749097) bank71536 +71537 POINT(40.859582853039 73.5493557791884) bank71537 +71538 POINT(40.13868150271363 74.96531504337511) bank71538 +71539 POINT(40.51341418112558 74.05018638030377) bank71539 +71540 POINT(41.626742324736696 74.14862164411763) bank71540 +71541 POINT(39.971292888296176 73.02173364849068) bank71541 +71542 POINT(40.42761970843458 74.6270241562549) bank71542 +71543 POINT(41.610682776353116 74.61437474761588) bank71543 +71544 POINT(39.81103743893894 74.67131230855068) bank71544 +71545 POINT(40.84171269902265 73.97702635445899) bank71545 +71546 POINT(40.933944054023435 73.42587217951682) bank71546 +71547 POINT(41.343766965434206 73.69980537682062) bank71547 +71548 POINT(41.68578725658876 74.18834767144219) bank71548 +71549 POINT(40.51845355500525 73.34667924283862) bank71549 +71550 POINT(40.69383903394693 73.06732851708054) bank71550 +71551 POINT(40.96699285531593 74.78417672339538) bank71551 +71552 POINT(40.99817032483936 73.39793611442653) bank71552 +71553 POINT(40.69640845952824 74.40064421150898) bank71553 +71554 POINT(40.00713846921768 73.0443314399758) bank71554 +71555 POINT(40.85437968223356 74.18465884789721) bank71555 +71556 POINT(39.90280812227448 74.27949733921568) bank71556 +71557 POINT(40.39953622706145 74.91703062855395) bank71557 +71558 POINT(40.20448943558575 73.2883569039924) bank71558 +71559 POINT(41.62124143197164 74.141139571054) bank71559 +71560 POINT(40.081230742277434 74.48062322095612) bank71560 +71561 POINT(41.590837866062955 74.12328846747792) bank71561 +71562 POINT(40.826628653667406 73.73477416428689) bank71562 +71563 POINT(40.194031065346586 74.0441896856893) bank71563 +71564 POINT(41.49423907428756 74.66415437834084) bank71564 +71565 POINT(40.5916557051493 74.06583412707738) bank71565 +71566 POINT(40.27739169478808 73.78193202377429) bank71566 +71567 POINT(40.35385978217059 73.09844448184754) bank71567 +71568 POINT(40.610816521196526 74.95049827173064) bank71568 +71569 POINT(41.55246760440024 73.44382778914412) bank71569 +71570 POINT(39.831608349592216 73.7271966206022) bank71570 +71571 POINT(41.219651267350606 73.006714472826) bank71571 +71572 POINT(41.348904484580345 73.71333343757703) bank71572 +71573 POINT(40.704703565793515 73.39936795139737) bank71573 +71574 POINT(40.39999245606764 74.76589613667069) bank71574 +71575 POINT(41.516553644968766 74.96263865474688) bank71575 +71576 POINT(40.83828776817322 74.36404886043815) bank71576 +71577 POINT(39.87131352621849 73.70962694055595) bank71577 +71578 POINT(40.56650062511832 73.81433140114102) bank71578 +71579 POINT(41.24750467305209 74.83970029606327) bank71579 +71580 POINT(41.66000199822457 73.01284667148958) bank71580 +71581 POINT(41.60062696276379 73.47681942408418) bank71581 +71582 POINT(40.07386420821483 74.36030291620982) bank71582 +71583 POINT(40.8108873203837 74.37729033712677) bank71583 +71584 POINT(41.36487426414729 74.2139452984298) bank71584 +71585 POINT(41.418727531434584 74.5152395014975) bank71585 +71586 POINT(40.81398757191997 73.09805746146345) bank71586 +71587 POINT(40.82741134216735 73.59194500701845) bank71587 +71588 POINT(41.081159488905506 74.44790572075563) bank71588 +71589 POINT(40.878138364397905 73.35380747158348) bank71589 +71590 POINT(41.31156509458328 74.1460192393841) bank71590 +71591 POINT(39.894902367987704 73.31023471076408) bank71591 +71592 POINT(41.06598861006826 74.05810827941959) bank71592 +71593 POINT(40.409983351107954 73.73430622553741) bank71593 +71594 POINT(41.44547656000662 73.11229331666941) bank71594 +71595 POINT(40.05673582321869 74.28364536434816) bank71595 +71596 POINT(41.325361866170454 74.24866198102434) bank71596 +71597 POINT(40.3331393456301 74.84453902684679) bank71597 +71598 POINT(41.21828012871595 74.66176394422907) bank71598 +71599 POINT(40.45657044539609 74.60352172978133) bank71599 +71600 POINT(40.53629866513167 74.07856105668586) bank71600 +71601 POINT(40.078939689730234 73.57095575379422) bank71601 +71602 POINT(40.75236052233535 73.71258614504656) bank71602 +71603 POINT(41.46943129073342 74.11150369137223) bank71603 +71604 POINT(41.45195151763923 74.93759977129902) bank71604 +71605 POINT(41.442920933670734 74.4418757056839) bank71605 +71606 POINT(41.49530141419748 73.07166169206162) bank71606 +71607 POINT(41.401801094137184 73.51553757213858) bank71607 +71608 POINT(39.78139336209249 74.43893530151271) bank71608 +71609 POINT(40.40845432326852 73.66113052305451) bank71609 +71610 POINT(41.08793676553642 73.34493887965772) bank71610 +71611 POINT(40.48263299452456 73.50542482802709) bank71611 +71612 POINT(40.8753837305135 74.76742064650674) bank71612 +71613 POINT(41.40909337155035 73.41498637839895) bank71613 +71614 POINT(41.34180085885381 73.41508220779956) bank71614 +71615 POINT(40.624279882046146 74.71627222212767) bank71615 +71616 POINT(40.1138456205464 73.58404107198517) bank71616 +71617 POINT(41.29013829012868 73.28528566020087) bank71617 +71618 POINT(41.20724210372374 73.7564628290222) bank71618 +71619 POINT(40.66094284074456 73.70518077640519) bank71619 +71620 POINT(40.60802105708607 74.41969345168565) bank71620 +71621 POINT(40.919086061731264 73.92462033214616) bank71621 +71622 POINT(40.20949722325106 74.61784953510225) bank71622 +71623 POINT(40.41488714351642 74.50629019380288) bank71623 +71624 POINT(41.05509318685788 74.72436281211542) bank71624 +71625 POINT(41.03022366483873 74.74359446776768) bank71625 +71626 POINT(39.80410748308371 74.83791065592642) bank71626 +71627 POINT(40.62687027679276 73.7941568020171) bank71627 +71628 POINT(40.55525253690758 73.88251097151843) bank71628 +71629 POINT(41.5693278485197 74.4373298997153) bank71629 +71630 POINT(40.37464962022931 74.3654666962468) bank71630 +71631 POINT(39.82622652748286 73.41769347996136) bank71631 +71632 POINT(41.57473081909804 73.29028503779003) bank71632 +71633 POINT(40.030992385412134 73.72871207015974) bank71633 +71634 POINT(40.215098531949884 73.88177805922801) bank71634 +71635 POINT(41.143264020662215 73.1535422897289) bank71635 +71636 POINT(41.32907645668591 74.5641128228859) bank71636 +71637 POINT(40.246390483322884 73.72590209251459) bank71637 +71638 POINT(39.991697259392595 73.0753403837145) bank71638 +71639 POINT(39.99797620233368 74.07303147614215) bank71639 +71640 POINT(41.68524774671604 73.57837987674543) bank71640 +71641 POINT(39.89008752116715 73.80865947544004) bank71641 +71642 POINT(41.208467532435364 74.0256134650324) bank71642 +71643 POINT(40.539924146961496 73.23440307648973) bank71643 +71644 POINT(40.96311652408625 73.77242795479006) bank71644 +71645 POINT(41.12943809903313 74.92980313673442) bank71645 +71646 POINT(40.341829091764374 74.00099719822836) bank71646 +71647 POINT(41.48556717551015 75.00490229403545) bank71647 +71648 POINT(39.8649578824222 74.55163466253366) bank71648 +71649 POINT(39.928714909533575 74.56008888323925) bank71649 +71650 POINT(40.518798082186606 74.69992224168521) bank71650 +71651 POINT(41.03632958313597 74.57112108480688) bank71651 +71652 POINT(40.678851246303886 74.21030600187161) bank71652 +71653 POINT(39.886350201261 74.13917674033637) bank71653 +71654 POINT(40.31873185572978 73.79343550561263) bank71654 +71655 POINT(39.96851030947769 73.53475234459269) bank71655 +71656 POINT(39.859596332855645 73.59860900114826) bank71656 +71657 POINT(40.81607192585201 73.58739668702825) bank71657 +71658 POINT(39.80594678746654 73.33546465328457) bank71658 +71659 POINT(41.651535579195524 73.47340578916999) bank71659 +71660 POINT(40.450214473195345 74.56911025854397) bank71660 +71661 POINT(41.124943127634815 73.46396335986447) bank71661 +71662 POINT(41.61896457842076 74.4666071154319) bank71662 +71663 POINT(40.12967662085621 74.01046092824804) bank71663 +71664 POINT(41.69300818367289 74.39678493370327) bank71664 +71665 POINT(40.38849551842943 73.23777539015693) bank71665 +71666 POINT(39.94108539695716 74.72332756964373) bank71666 +71667 POINT(39.98243279795722 74.02306013314833) bank71667 +71668 POINT(39.79684534267658 74.12188928536465) bank71668 +71669 POINT(40.204300917412084 73.84447882906866) bank71669 +71670 POINT(40.72176773046191 74.56584074934202) bank71670 +71671 POINT(41.28843265516033 73.72086499597926) bank71671 +71672 POINT(40.630870927144635 73.86183927731089) bank71672 +71673 POINT(40.16967156313933 74.91809511456026) bank71673 +71674 POINT(41.58204160296053 73.36420372434827) bank71674 +71675 POINT(40.13357539183695 74.47650736586542) bank71675 +71676 POINT(40.349882131455146 73.31038154261135) bank71676 +71677 POINT(41.521801494192665 74.0031581636667) bank71677 +71678 POINT(40.38262674104374 74.69016865842056) bank71678 +71679 POINT(40.81922152944542 73.45539768855905) bank71679 +71680 POINT(41.41726717190927 73.58085312272792) bank71680 +71681 POINT(40.726552802858755 74.05172688848867) bank71681 +71682 POINT(39.74369782227591 73.83263771057472) bank71682 +71683 POINT(39.71936933135385 73.68449654140845) bank71683 +71684 POINT(40.16415387930076 73.58243741030175) bank71684 +71685 POINT(41.33011428335325 73.06752223971627) bank71685 +71686 POINT(41.70579020682861 73.91243236006402) bank71686 +71687 POINT(41.697646182866 74.29079948951198) bank71687 +71688 POINT(40.97698325542075 74.53805908717317) bank71688 +71689 POINT(40.586156792586 74.93693759619885) bank71689 +71690 POINT(40.07193876100783 73.21733912465945) bank71690 +71691 POINT(40.65196406496162 74.30927661775168) bank71691 +71692 POINT(40.53234054135693 73.42401184985764) bank71692 +71693 POINT(41.68021791946465 74.70380698938727) bank71693 +71694 POINT(41.02969952652869 74.92253200033352) bank71694 +71695 POINT(39.772854120375825 73.60632721385018) bank71695 +71696 POINT(41.65585228530753 74.76981189376372) bank71696 +71697 POINT(39.75315425234722 73.9717226433913) bank71697 +71698 POINT(39.79896914496908 74.62375756705877) bank71698 +71699 POINT(41.12744523273426 73.4189429332128) bank71699 +71700 POINT(40.64630589342855 73.94824438557593) bank71700 +71701 POINT(41.57180369674167 74.68079575014225) bank71701 +71702 POINT(39.7438842126019 74.13012420965798) bank71702 +71703 POINT(40.767050476131494 74.30205693427989) bank71703 +71704 POINT(40.24634171704779 73.43360823029738) bank71704 +71705 POINT(40.17651717745892 73.76013114591369) bank71705 +71706 POINT(40.943064683015145 73.40137285466972) bank71706 +71707 POINT(40.988520379964775 73.34084200280894) bank71707 +71708 POINT(41.33485094951848 74.29320133168349) bank71708 +71709 POINT(40.127562349848674 74.48830137002432) bank71709 +71710 POINT(39.830393389484215 74.6669020938423) bank71710 +71711 POINT(39.80074816154626 73.84032636789723) bank71711 +71712 POINT(41.375404583442204 73.98421223268045) bank71712 +71713 POINT(40.3656732174214 73.7752242150352) bank71713 +71714 POINT(40.64783740771684 73.92692323258927) bank71714 +71715 POINT(41.15224069362757 74.89878365275734) bank71715 +71716 POINT(41.28416405671932 73.46014631933978) bank71716 +71717 POINT(40.16548813203869 74.55360179171966) bank71717 +71718 POINT(40.89469162262069 74.30516791982663) bank71718 +71719 POINT(39.967405942539685 74.91181323419072) bank71719 +71720 POINT(40.597506519201524 73.22688292949643) bank71720 +71721 POINT(39.91364729139208 74.47528820495658) bank71721 +71722 POINT(41.03722842205439 74.23984422900749) bank71722 +71723 POINT(41.36224744181363 74.27715585631479) bank71723 +71724 POINT(41.54768809744608 73.4117478795899) bank71724 +71725 POINT(40.92080021404983 74.22084375479791) bank71725 +71726 POINT(40.68649023702677 74.95342333614325) bank71726 +71727 POINT(39.8316343622443 73.59197198145749) bank71727 +71728 POINT(41.630288099569654 74.19706463150494) bank71728 +71729 POINT(39.96699272321353 74.92478275371633) bank71729 +71730 POINT(41.17524586111729 73.76172872353963) bank71730 +71731 POINT(41.23702283235823 73.35193274286284) bank71731 +71732 POINT(41.29448880883087 73.78251648015622) bank71732 +71733 POINT(41.281793157383184 73.78125317788951) bank71733 +71734 POINT(39.890210503203264 73.99643117050073) bank71734 +71735 POINT(40.99362099671528 74.83588772379619) bank71735 +71736 POINT(39.875170059213566 73.24627162449603) bank71736 +71737 POINT(41.69110475869526 74.28512427794408) bank71737 +71738 POINT(39.941489488071966 73.43587733892869) bank71738 +71739 POINT(39.91469813097528 75.00028246243374) bank71739 +71740 POINT(39.851576362881964 74.07402648906168) bank71740 +71741 POINT(40.95812417100609 73.25545248854979) bank71741 +71742 POINT(40.13616687134758 74.59448543417994) bank71742 +71743 POINT(41.68143610864059 74.81256701086117) bank71743 +71744 POINT(41.59264360596211 73.62868888877875) bank71744 +71745 POINT(39.865252140934565 74.89629604219219) bank71745 +71746 POINT(40.052838929842444 73.61438774119816) bank71746 +71747 POINT(41.47124741884322 74.39773648856278) bank71747 +71748 POINT(41.54614949212675 74.19901038086235) bank71748 +71749 POINT(40.74679532910117 74.509817479789) bank71749 +71750 POINT(40.79334386313353 74.23234713316158) bank71750 +71751 POINT(41.12405860164854 74.19342633203009) bank71751 +71752 POINT(40.308839448532076 73.31988501708295) bank71752 +71753 POINT(41.5233827460435 74.82683291897736) bank71753 +71754 POINT(40.531172398817915 73.21727091196735) bank71754 +71755 POINT(41.50722763137687 74.48893921474844) bank71755 +71756 POINT(41.591169388742465 74.72521708787009) bank71756 +71757 POINT(41.01307108067072 74.39161984260949) bank71757 +71758 POINT(41.44335187440063 73.83581750248874) bank71758 +71759 POINT(40.68940122805433 74.38199246392743) bank71759 +71760 POINT(40.24677955671079 74.46936865929017) bank71760 +71761 POINT(40.17653244105984 74.28785629359871) bank71761 +71762 POINT(40.0100661698065 73.66192935253082) bank71762 +71763 POINT(41.41322105017844 74.46997671673257) bank71763 +71764 POINT(41.48270447031487 74.85960293165097) bank71764 +71765 POINT(41.04586356589337 73.97788752777818) bank71765 +71766 POINT(40.56674583454247 73.70595209330315) bank71766 +71767 POINT(41.33252013919465 74.78542469127017) bank71767 +71768 POINT(41.48565188192821 73.01558609867872) bank71768 +71769 POINT(40.62084414699443 74.80905936546515) bank71769 +71770 POINT(40.37044374410875 73.68668435905658) bank71770 +71771 POINT(40.36395717009065 73.59268926148876) bank71771 +71772 POINT(40.089789141124946 74.92354668789471) bank71772 +71773 POINT(41.21857598795803 74.9525110041863) bank71773 +71774 POINT(41.09617156127576 74.27518720720057) bank71774 +71775 POINT(41.31957336046233 74.5904365303045) bank71775 +71776 POINT(40.05804549986989 74.6971850260196) bank71776 +71777 POINT(41.56346569455065 73.36425063454016) bank71777 +71778 POINT(40.74551456719876 73.18335872072483) bank71778 +71779 POINT(39.895277775717304 73.41888539053869) bank71779 +71780 POINT(40.92873832398815 74.43869048176764) bank71780 +71781 POINT(40.855186301859305 73.7529621380872) bank71781 +71782 POINT(41.69342648753713 74.35451442448756) bank71782 +71783 POINT(41.39380145233112 73.92693079071206) bank71783 +71784 POINT(41.5155242396795 74.47799363782819) bank71784 +71785 POINT(40.27465819151482 74.63426050166423) bank71785 +71786 POINT(40.719933439630445 73.91511431177136) bank71786 +71787 POINT(40.68976994728117 73.1917796596326) bank71787 +71788 POINT(40.64170797407108 74.06025883853243) bank71788 +71789 POINT(40.115423682359605 73.35124006409539) bank71789 +71790 POINT(39.74345049878415 74.82393289106216) bank71790 +71791 POINT(41.00414438479128 74.79279613513528) bank71791 +71792 POINT(41.38796584053005 74.70817488108601) bank71792 +71793 POINT(41.280585087048934 73.02906183555699) bank71793 +71794 POINT(40.9798653152231 73.4021640627231) bank71794 +71795 POINT(40.598904826098234 73.77646948619635) bank71795 +71796 POINT(40.84819449833059 73.68523876195577) bank71796 +71797 POINT(40.65189808984671 74.46568962884223) bank71797 +71798 POINT(41.68089676294241 73.63248173320201) bank71798 +71799 POINT(41.224138159216736 73.89643848105479) bank71799 +71800 POINT(39.870331304746486 73.45771879540607) bank71800 +71801 POINT(41.56143992354291 73.09235555663373) bank71801 +71802 POINT(41.03217532801815 73.34569792050775) bank71802 +71803 POINT(40.916147024608186 74.30734422501324) bank71803 +71804 POINT(41.48800827747157 73.05275191318843) bank71804 +71805 POINT(41.11232548265387 73.05189582372236) bank71805 +71806 POINT(40.27407212713098 73.47013601103087) bank71806 +71807 POINT(40.157491686728335 73.173907372709) bank71807 +71808 POINT(40.68757855626579 73.53427724828485) bank71808 +71809 POINT(39.864020156680255 74.38283175119874) bank71809 +71810 POINT(41.488543196043224 73.93586468686934) bank71810 +71811 POINT(40.25625454155714 73.81511684742225) bank71811 +71812 POINT(40.082020178187875 74.01369074469902) bank71812 +71813 POINT(40.08523666228465 73.4683530635504) bank71813 +71814 POINT(41.26842061932447 74.75269273830196) bank71814 +71815 POINT(40.56619113544206 74.99562345814316) bank71815 +71816 POINT(40.55910362541137 73.60687585076303) bank71816 +71817 POINT(39.85904640105688 74.69820861217868) bank71817 +71818 POINT(41.48708928641559 73.50404606448872) bank71818 +71819 POINT(40.33353419183633 73.01295903529717) bank71819 +71820 POINT(39.78033143704133 73.45770664048878) bank71820 +71821 POINT(40.64876075431766 73.18991423263765) bank71821 +71822 POINT(40.82547189242409 73.58520530819996) bank71822 +71823 POINT(40.79929948763344 73.11487260391124) bank71823 +71824 POINT(40.630413039645106 73.64621163378438) bank71824 +71825 POINT(41.08576275524828 73.1426360598126) bank71825 +71826 POINT(41.22518501147404 73.50690656864845) bank71826 +71827 POINT(39.74621035297109 74.40712801691052) bank71827 +71828 POINT(39.92525258511363 74.54665496425807) bank71828 +71829 POINT(40.138263874957694 73.43066125038983) bank71829 +71830 POINT(41.02991554561474 74.24652755254759) bank71830 +71831 POINT(40.11482430910502 73.69063998294) bank71831 +71832 POINT(41.45425550727215 73.20464399061866) bank71832 +71833 POINT(40.86594587363145 74.1876754461265) bank71833 +71834 POINT(40.61490563618942 73.00848477831954) bank71834 +71835 POINT(39.79242374332424 74.33648173260809) bank71835 +71836 POINT(41.17415684858462 74.62742314939926) bank71836 +71837 POINT(41.32846367343331 73.89711330034217) bank71837 +71838 POINT(40.51393708088321 74.81686547240761) bank71838 +71839 POINT(40.73599876769628 73.75654381525024) bank71839 +71840 POINT(39.826733878688195 74.39618632011432) bank71840 +71841 POINT(40.064730099854536 73.11261569137005) bank71841 +71842 POINT(41.106502646961175 74.18984320367056) bank71842 +71843 POINT(41.46352664972139 73.47672499149307) bank71843 +71844 POINT(39.919870149215704 73.24644713960178) bank71844 +71845 POINT(40.00551731905714 74.9647595726731) bank71845 +71846 POINT(41.15038980064168 74.74964052931253) bank71846 +71847 POINT(41.52116271597324 73.59430606574328) bank71847 +71848 POINT(40.66872083802398 74.74268918437322) bank71848 +71849 POINT(41.18306621519343 73.98072707706119) bank71849 +71850 POINT(40.518840211064074 74.38257728003649) bank71850 +71851 POINT(40.567493030125064 74.83403591157003) bank71851 +71852 POINT(41.18502977001843 73.34315262904889) bank71852 +71853 POINT(41.02842392853193 73.26418459276276) bank71853 +71854 POINT(41.01093106522556 73.37690668759915) bank71854 +71855 POINT(41.35400937686235 74.78526122321358) bank71855 +71856 POINT(40.44911007163877 73.4133941061004) bank71856 +71857 POINT(40.20682096755098 73.44393855284663) bank71857 +71858 POINT(40.36973946057872 74.4851526130542) bank71858 +71859 POINT(40.95500575847113 73.83372348493234) bank71859 +71860 POINT(39.87327657848585 74.42436489021483) bank71860 +71861 POINT(41.100052581293895 73.5464731428754) bank71861 +71862 POINT(41.40605753754915 73.20193084577024) bank71862 +71863 POINT(40.393775502295284 73.21760650947007) bank71863 +71864 POINT(41.10882103690065 73.66277068289763) bank71864 +71865 POINT(41.07285372412966 74.89375730359336) bank71865 +71866 POINT(40.096288480105066 73.10034783281228) bank71866 +71867 POINT(41.468163491694526 74.20637380432277) bank71867 +71868 POINT(41.13468881828806 73.41241051722295) bank71868 +71869 POINT(40.226316148643434 74.89926030842682) bank71869 +71870 POINT(40.55237640558108 74.19780824832398) bank71870 +71871 POINT(39.798863182180156 73.11790416956372) bank71871 +71872 POINT(39.817444142487844 73.06920839065918) bank71872 +71873 POINT(39.90863044018421 73.24827923862526) bank71873 +71874 POINT(40.657532690541416 74.91792907845398) bank71874 +71875 POINT(40.78608119957063 74.83924855983574) bank71875 +71876 POINT(40.671248971442445 74.11001812173797) bank71876 +71877 POINT(40.64028611257521 74.35125580965122) bank71877 +71878 POINT(41.17240651903833 73.20159144173572) bank71878 +71879 POINT(39.95605733719635 73.88112530252856) bank71879 +71880 POINT(40.4463216051215 74.5757921660575) bank71880 +71881 POINT(41.69727669587533 74.57033348556611) bank71881 +71882 POINT(40.32646444330901 73.03903951804662) bank71882 +71883 POINT(40.370180124203074 74.44099742609347) bank71883 +71884 POINT(40.67229013931921 73.04432781401924) bank71884 +71885 POINT(39.85691945871686 74.32313793453338) bank71885 +71886 POINT(40.2875379094922 73.98527055753294) bank71886 +71887 POINT(40.136853951997864 74.44687980127468) bank71887 +71888 POINT(41.10148718253815 73.92953247280711) bank71888 +71889 POINT(39.75178512500817 73.43923550247969) bank71889 +71890 POINT(39.73308406491383 73.24847364579429) bank71890 +71891 POINT(40.086229121749426 74.68426190864845) bank71891 +71892 POINT(40.076414115485726 74.55122867160337) bank71892 +71893 POINT(41.37618203696912 74.66747032028387) bank71893 +71894 POINT(40.88750726123326 73.37153474242504) bank71894 +71895 POINT(41.60914276980678 74.22123147649087) bank71895 +71896 POINT(40.97848287776402 74.06388085223807) bank71896 +71897 POINT(40.65985794055478 74.60406290695225) bank71897 +71898 POINT(41.40837911546769 73.49398747782729) bank71898 +71899 POINT(41.410937400223574 73.1977805750502) bank71899 +71900 POINT(41.62628263484813 74.91579296825233) bank71900 +71901 POINT(41.19720175382924 74.72526175228772) bank71901 +71902 POINT(40.89612411520007 73.85350065035666) bank71902 +71903 POINT(41.24881622931656 73.9421502010624) bank71903 +71904 POINT(39.98146008678831 73.60063255531243) bank71904 +71905 POINT(40.84517662170533 74.18881678102245) bank71905 +71906 POINT(40.58840169416801 74.00575180545867) bank71906 +71907 POINT(39.93413733542429 73.14388715858074) bank71907 +71908 POINT(41.021958583545086 74.41720288167556) bank71908 +71909 POINT(40.620506162458916 73.1436884841461) bank71909 +71910 POINT(39.97939529345524 74.32106360176196) bank71910 +71911 POINT(41.25587263497444 73.44935699631026) bank71911 +71912 POINT(40.880381611194 74.98215737120994) bank71912 +71913 POINT(40.57146532712095 73.55136351076983) bank71913 +71914 POINT(39.81584186196854 74.75202335251792) bank71914 +71915 POINT(40.651740819102756 74.01251436208256) bank71915 +71916 POINT(40.641267675799085 74.78125943091422) bank71916 +71917 POINT(40.170645246439356 73.82195315157483) bank71917 +71918 POINT(41.45111626772397 73.27749204724006) bank71918 +71919 POINT(40.827178751736604 74.21726695826453) bank71919 +71920 POINT(41.63027694120952 73.01941169915433) bank71920 +71921 POINT(39.90134922641352 74.29731153588457) bank71921 +71922 POINT(40.51085338554561 74.18908072202044) bank71922 +71923 POINT(41.350930654309046 74.37187386791368) bank71923 +71924 POINT(40.08412501710943 74.93612263914926) bank71924 +71925 POINT(40.20103059185346 73.89948943579398) bank71925 +71926 POINT(40.90110527883704 74.03058114849183) bank71926 +71927 POINT(40.38837334795139 74.2915851804608) bank71927 +71928 POINT(40.55144189486304 73.66460571366868) bank71928 +71929 POINT(40.46965673804495 73.63183241823872) bank71929 +71930 POINT(40.3518445409516 73.79438566771613) bank71930 +71931 POINT(40.748983397151875 74.30887435418995) bank71931 +71932 POINT(40.30700272633639 74.87964662557461) bank71932 +71933 POINT(40.73604617173457 73.73425241909) bank71933 +71934 POINT(41.17034624858437 73.36021552275756) bank71934 +71935 POINT(41.52894736998939 74.52390163407496) bank71935 +71936 POINT(41.298039394664464 74.28621741880141) bank71936 +71937 POINT(40.513482212095006 74.82343100211548) bank71937 +71938 POINT(40.0279698647311 74.57732429135743) bank71938 +71939 POINT(40.18050819393393 74.85221499048475) bank71939 +71940 POINT(40.44022029945607 73.34491244361833) bank71940 +71941 POINT(41.24820261843383 73.77860452199307) bank71941 +71942 POINT(40.48721685656781 74.54558296082634) bank71942 +71943 POINT(40.38680549014637 74.08138109136888) bank71943 +71944 POINT(40.5388169445462 73.14471562491906) bank71944 +71945 POINT(39.851675385755456 73.3296686361179) bank71945 +71946 POINT(41.622331455150665 73.31669135959413) bank71946 +71947 POINT(40.03504146614252 73.77455160580263) bank71947 +71948 POINT(41.69733763916027 74.27795793574612) bank71948 +71949 POINT(40.27333108911444 74.51624171235963) bank71949 +71950 POINT(40.4540387277963 73.56083248319779) bank71950 +71951 POINT(41.621681615006125 73.88678016055795) bank71951 +71952 POINT(41.39487111865241 74.14347106819453) bank71952 +71953 POINT(41.50429642708903 73.03993249656739) bank71953 +71954 POINT(39.955159454281 73.17915691706881) bank71954 +71955 POINT(40.72738738477917 73.08542047510845) bank71955 +71956 POINT(40.936836276453064 73.63893141773447) bank71956 +71957 POINT(39.753226429045476 73.29989983536379) bank71957 +71958 POINT(40.239234647370175 73.01227202018299) bank71958 +71959 POINT(40.6771198950358 73.45732895454191) bank71959 +71960 POINT(41.13427442698591 73.70939418489532) bank71960 +71961 POINT(41.41793693489409 73.57941471638833) bank71961 +71962 POINT(41.04126358352151 73.92968573642267) bank71962 +71963 POINT(40.72451237924319 74.01144959964628) bank71963 +71964 POINT(40.663428042689226 74.58529582492056) bank71964 +71965 POINT(40.06722539876494 74.70025295971158) bank71965 +71966 POINT(40.10664115043562 74.6218829468588) bank71966 +71967 POINT(40.536818052542635 74.9969879665058) bank71967 +71968 POINT(41.14375440867139 73.29558220220247) bank71968 +71969 POINT(41.489680375280514 74.87866122256186) bank71969 +71970 POINT(40.24581427857722 73.3568762496029) bank71970 +71971 POINT(40.60361568768318 73.84711433308685) bank71971 +71972 POINT(40.78781603715923 74.12467658661767) bank71972 +71973 POINT(40.69640282516556 73.11576834688563) bank71973 +71974 POINT(40.9241385603559 73.37510491625237) bank71974 +71975 POINT(41.47006670246884 74.03191343190223) bank71975 +71976 POINT(40.20156487605747 74.98717215043744) bank71976 +71977 POINT(40.97169354958726 73.63270848541733) bank71977 +71978 POINT(41.307793319211406 74.56031191093537) bank71978 +71979 POINT(40.6311465853027 73.23301625159631) bank71979 +71980 POINT(40.545389707594516 73.57162881084359) bank71980 +71981 POINT(41.4256471729047 73.17430958698165) bank71981 +71982 POINT(41.30417101117067 73.63007379662923) bank71982 +71983 POINT(41.32049363350669 74.73685206605484) bank71983 +71984 POINT(40.661861289701754 74.14866319372692) bank71984 +71985 POINT(40.469225394132245 74.89965392980875) bank71985 +71986 POINT(39.7982040326805 73.93940374223133) bank71986 +71987 POINT(41.55020427984475 73.4071043839863) bank71987 +71988 POINT(39.91957400254412 73.98215750234507) bank71988 +71989 POINT(41.669344638265635 74.00213531437547) bank71989 +71990 POINT(40.62522668306987 74.60977122694615) bank71990 +71991 POINT(41.357994749559516 73.03469901301615) bank71991 +71992 POINT(41.6303176419671 74.00652241081426) bank71992 +71993 POINT(39.89572373102487 73.32842643164375) bank71993 +71994 POINT(40.81629973734085 74.73263389865551) bank71994 +71995 POINT(39.820086871537896 74.99690674119589) bank71995 +71996 POINT(40.376718799345596 74.20005193501595) bank71996 +71997 POINT(40.76873415574265 74.3038128249792) bank71997 +71998 POINT(40.2234664154127 73.78258407053706) bank71998 +71999 POINT(39.877514128614486 74.69320329274866) bank71999 +72000 POINT(40.47651003866323 74.13159057095937) bank72000 +72001 POINT(41.17245145135737 73.12159029611219) bank72001 +72002 POINT(40.39135092400291 74.0365708457075) bank72002 +72003 POINT(41.71139213578868 73.14602302366143) bank72003 +72004 POINT(40.33187084007311 73.01212687483086) bank72004 +72005 POINT(41.10318823279003 74.5935791526065) bank72005 +72006 POINT(40.13170173380631 73.69088805449168) bank72006 +72007 POINT(40.681499104302986 74.15754191629983) bank72007 +72008 POINT(40.609626133636965 73.45549911742796) bank72008 +72009 POINT(41.16282804827597 73.90195716700227) bank72009 +72010 POINT(41.50892965832401 74.9573724016376) bank72010 +72011 POINT(41.34585090766987 74.28275266799974) bank72011 +72012 POINT(41.00079244010184 73.40102010838174) bank72012 +72013 POINT(41.42383060262836 73.23630757178756) bank72013 +72014 POINT(41.5239532002147 74.59334353458355) bank72014 +72015 POINT(40.96630262624369 73.76356082922042) bank72015 +72016 POINT(41.68270608452905 74.13917105081899) bank72016 +72017 POINT(39.868464951454584 73.49305924325823) bank72017 +72018 POINT(40.03127670339289 74.56940810884274) bank72018 +72019 POINT(40.08929264187834 73.88031033390159) bank72019 +72020 POINT(40.048770942587645 74.76040231436525) bank72020 +72021 POINT(41.032424698967 74.37170058206526) bank72021 +72022 POINT(40.326821277132 73.08498158020491) bank72022 +72023 POINT(40.78200577602775 73.02543387727067) bank72023 +72024 POINT(41.61221000979077 74.29039262422779) bank72024 +72025 POINT(39.89320847641188 73.01126333994296) bank72025 +72026 POINT(40.71945475941336 73.68502472356417) bank72026 +72027 POINT(40.36773109957892 73.63333948630331) bank72027 +72028 POINT(41.28836661837839 74.35260089813895) bank72028 +72029 POINT(41.55777386897865 74.71832449166332) bank72029 +72030 POINT(40.25902820263713 74.55168968270837) bank72030 +72031 POINT(39.78050423917636 73.99620421853875) bank72031 +72032 POINT(39.88862367411366 74.8463085663604) bank72032 +72033 POINT(39.97402401501211 73.68045250158801) bank72033 +72034 POINT(41.41303917576188 74.49062182219632) bank72034 +72035 POINT(41.3109134753058 73.43353989630644) bank72035 +72036 POINT(41.02409809641288 73.55614155202318) bank72036 +72037 POINT(39.84781093641259 74.22663822652132) bank72037 +72038 POINT(40.528508496570176 73.26038337385825) bank72038 +72039 POINT(39.77001597967249 73.04263015969859) bank72039 +72040 POINT(41.00027618063416 74.96685439973015) bank72040 +72041 POINT(40.29238353547211 74.20948186148148) bank72041 +72042 POINT(40.34944531567143 73.07608407416213) bank72042 +72043 POINT(41.64460993905504 73.76798190550817) bank72043 +72044 POINT(40.05690556898306 73.13590297616496) bank72044 +72045 POINT(40.24559668104796 73.90423440448728) bank72045 +72046 POINT(39.9888344912502 73.19892295142313) bank72046 +72047 POINT(41.629546754948024 73.34094255908603) bank72047 +72048 POINT(40.584486412234874 73.84194452312215) bank72048 +72049 POINT(39.871717488068356 74.21511134847523) bank72049 +72050 POINT(40.14170966716288 73.83458007048185) bank72050 +72051 POINT(39.84094650350089 74.9844556601705) bank72051 +72052 POINT(40.758506867128375 74.55782044824011) bank72052 +72053 POINT(40.258114812103 73.68334941029438) bank72053 +72054 POINT(41.26159953749589 74.99356680123344) bank72054 +72055 POINT(41.3350955700184 74.0689373193883) bank72055 +72056 POINT(41.29120903833834 73.37003398089453) bank72056 +72057 POINT(40.68302979481115 74.34107590183827) bank72057 +72058 POINT(41.490233562842455 74.54892858636755) bank72058 +72059 POINT(41.04533839435072 74.65636943462461) bank72059 +72060 POINT(39.74036516030145 73.36125690595867) bank72060 +72061 POINT(41.28846451445542 74.59320665557495) bank72061 +72062 POINT(40.795788850099164 73.28954581661107) bank72062 +72063 POINT(41.13510037836864 73.26654981523463) bank72063 +72064 POINT(40.867642816963674 74.24697201259606) bank72064 +72065 POINT(39.87035301677994 74.79661359941379) bank72065 +72066 POINT(41.19451565879901 73.59716666507795) bank72066 +72067 POINT(41.634747773760296 73.86698299485138) bank72067 +72068 POINT(41.54180042670444 73.7302822800604) bank72068 +72069 POINT(41.70548216814738 74.38840429507029) bank72069 +72070 POINT(40.26508282091172 73.77257575967434) bank72070 +72071 POINT(41.53780484254196 73.69837258923913) bank72071 +72072 POINT(40.45042358535708 75.0039727798596) bank72072 +72073 POINT(41.35263022625111 74.25106912733533) bank72073 +72074 POINT(41.242958122056066 73.00651389104299) bank72074 +72075 POINT(40.44977847207588 73.73741226796574) bank72075 +72076 POINT(41.45182332910658 74.26490927812117) bank72076 +72077 POINT(40.011042099645636 73.67790218274138) bank72077 +72078 POINT(41.5253034276028 74.9009622595661) bank72078 +72079 POINT(40.85241189036009 74.4972183044094) bank72079 +72080 POINT(40.64266274707603 73.35803158108011) bank72080 +72081 POINT(41.399220088481115 74.75170249127464) bank72081 +72082 POINT(40.162047907511685 73.14054320636393) bank72082 +72083 POINT(41.52263590574218 73.21376434134531) bank72083 +72084 POINT(40.50204846811832 74.1189474724984) bank72084 +72085 POINT(40.39656899195776 75.00524884101304) bank72085 +72086 POINT(41.02731868658219 73.20162146303552) bank72086 +72087 POINT(41.082390495203484 74.67928521374978) bank72087 +72088 POINT(39.90670615115502 74.68429080753863) bank72088 +72089 POINT(39.90763504755971 73.30344567023033) bank72089 +72090 POINT(40.84984542462758 74.26316810255781) bank72090 +72091 POINT(41.625904719352796 73.76941720486833) bank72091 +72092 POINT(40.354794176949916 73.69425553759069) bank72092 +72093 POINT(40.56952014924099 74.50341952539799) bank72093 +72094 POINT(40.283938683651925 74.15057204524292) bank72094 +72095 POINT(40.870899073884445 74.71028165136404) bank72095 +72096 POINT(40.20259852054696 74.3333351456989) bank72096 +72097 POINT(39.95433340534004 74.19253200107256) bank72097 +72098 POINT(39.90948899709295 73.07310912229784) bank72098 +72099 POINT(40.051411446765464 74.03825460789184) bank72099 +72100 POINT(41.7072541670667 74.03992951798747) bank72100 +72101 POINT(39.91069238000475 74.04063626458236) bank72101 +72102 POINT(41.08710749797219 74.31325044163975) bank72102 +72103 POINT(41.09965103827548 73.52162878982666) bank72103 +72104 POINT(40.322212221433226 73.45351013238053) bank72104 +72105 POINT(40.855444771603835 74.4254327094147) bank72105 +72106 POINT(40.8976596063557 73.6021747760708) bank72106 +72107 POINT(39.928588845377746 74.05528847398948) bank72107 +72108 POINT(41.53706763870097 74.10979439550374) bank72108 +72109 POINT(40.992838928375306 74.91641927859209) bank72109 +72110 POINT(41.01589316136134 73.52982765620195) bank72110 +72111 POINT(41.43008768073131 74.50124808627113) bank72111 +72112 POINT(41.50918851776599 74.93025480890648) bank72112 +72113 POINT(40.180024753945276 73.82366769476873) bank72113 +72114 POINT(40.29775111471358 74.14078314587184) bank72114 +72115 POINT(40.950656499675965 74.2619264364495) bank72115 +72116 POINT(41.10557690660149 73.7024190889983) bank72116 +72117 POINT(41.621218736769514 73.34175618603504) bank72117 +72118 POINT(40.79428398843926 74.35906621159187) bank72118 +72119 POINT(41.547285068559475 74.77788140965154) bank72119 +72120 POINT(41.25513426270708 74.86821846741368) bank72120 +72121 POINT(40.92670489099423 73.44380057809644) bank72121 +72122 POINT(40.570148070821624 74.67908582739871) bank72122 +72123 POINT(40.95504391009985 73.30179272015909) bank72123 +72124 POINT(41.12349105480325 74.03121834277263) bank72124 +72125 POINT(40.53613482190655 74.4886913092266) bank72125 +72126 POINT(40.635191399823846 73.72153181738832) bank72126 +72127 POINT(41.04687434884265 74.56493329378544) bank72127 +72128 POINT(39.77945439269405 74.2369202064536) bank72128 +72129 POINT(40.896002335203534 73.63883353251101) bank72129 +72130 POINT(40.54922338353314 74.16744298199343) bank72130 +72131 POINT(39.87219829463087 74.22894802928147) bank72131 +72132 POINT(41.33207573130949 74.4970699944881) bank72132 +72133 POINT(40.06471640519664 74.78419632856212) bank72133 +72134 POINT(40.455358531764844 73.269519900526) bank72134 +72135 POINT(40.78945796114655 73.1522764658099) bank72135 +72136 POINT(39.95559826103441 74.90338958875307) bank72136 +72137 POINT(40.78937591031286 73.63659386352506) bank72137 +72138 POINT(39.85172853493334 74.29201232953326) bank72138 +72139 POINT(40.85799082063685 74.69014239880434) bank72139 +72140 POINT(41.27281736208033 74.29140139612268) bank72140 +72141 POINT(40.98091300583962 73.09850864072395) bank72141 +72142 POINT(40.58084077783249 73.15857339087474) bank72142 +72143 POINT(39.761378716469586 74.92718477596858) bank72143 +72144 POINT(40.98420713845663 74.7977859879419) bank72144 +72145 POINT(41.2946305326427 73.26923692556902) bank72145 +72146 POINT(41.264160714001676 73.81908355314538) bank72146 +72147 POINT(40.45492633419549 74.86439207535608) bank72147 +72148 POINT(41.214713642971894 74.58934175706776) bank72148 +72149 POINT(41.179149092867924 74.34077525829628) bank72149 +72150 POINT(41.53159421720029 73.9350552895554) bank72150 +72151 POINT(40.103350829722814 74.85301553267783) bank72151 +72152 POINT(41.02404529187737 73.21259074072404) bank72152 +72153 POINT(41.493662848223785 73.93645841804847) bank72153 +72154 POINT(41.21260412089728 73.2796858257074) bank72154 +72155 POINT(41.61726671008432 73.84309817872571) bank72155 +72156 POINT(41.14642559250855 74.47698305828831) bank72156 +72157 POINT(41.353313146710626 73.23254822179788) bank72157 +72158 POINT(40.15372174481011 73.40269774263905) bank72158 +72159 POINT(39.91441844677429 73.22273371941115) bank72159 +72160 POINT(41.15501276000579 74.64124226902604) bank72160 +72161 POINT(41.51817256964215 73.50254938407112) bank72161 +72162 POINT(40.68036879515921 73.14215015282383) bank72162 +72163 POINT(39.863366552073785 73.06745537346006) bank72163 +72164 POINT(40.85771915172671 73.77806737052595) bank72164 +72165 POINT(41.17103216019604 73.19524785137224) bank72165 +72166 POINT(41.19441167078271 73.01777897517103) bank72166 +72167 POINT(41.14183674631564 73.7744626599195) bank72167 +72168 POINT(41.38783679269902 74.79934194999142) bank72168 +72169 POINT(40.29926229195546 74.29887483164137) bank72169 +72170 POINT(40.71027037424768 74.55896035584361) bank72170 +72171 POINT(40.6654545557391 74.41149395360931) bank72171 +72172 POINT(39.885999148373365 74.59017594197071) bank72172 +72173 POINT(40.68398023576826 73.13338670177558) bank72173 +72174 POINT(40.687022311191505 73.18431292479555) bank72174 +72175 POINT(41.52823475769659 73.22795099982964) bank72175 +72176 POINT(39.99442998137226 74.04669394763805) bank72176 +72177 POINT(40.30837796468451 74.42238585220468) bank72177 +72178 POINT(39.81304979886717 74.67947438246257) bank72178 +72179 POINT(41.65016840560575 74.87881223162927) bank72179 +72180 POINT(41.50304391745125 74.20219070329219) bank72180 +72181 POINT(41.65655861960961 73.20043789557268) bank72181 +72182 POINT(41.62113045730042 73.37407488569377) bank72182 +72183 POINT(41.369736214006124 74.50047267756574) bank72183 +72184 POINT(40.68661501913498 73.08859809022921) bank72184 +72185 POINT(39.95982103964584 73.73726014074506) bank72185 +72186 POINT(40.68547819824757 74.8494261136798) bank72186 +72187 POINT(39.828368313358304 74.6567773069431) bank72187 +72188 POINT(41.114113233141424 74.1551500820821) bank72188 +72189 POINT(40.16105551137812 73.86844752410552) bank72189 +72190 POINT(40.507844488341625 74.95284513810438) bank72190 +72191 POINT(41.49770031396072 73.56442406253767) bank72191 +72192 POINT(41.152824888741996 74.20083194199566) bank72192 +72193 POINT(41.57413487960949 73.80821755417455) bank72193 +72194 POINT(41.07022963485576 74.57565067470136) bank72194 +72195 POINT(41.16415769350019 73.60326058494451) bank72195 +72196 POINT(40.51632357504205 74.82404031481099) bank72196 +72197 POINT(40.33103739544565 74.12798382380069) bank72197 +72198 POINT(40.814657712839654 74.6203982709124) bank72198 +72199 POINT(41.07606936802828 73.93979873877979) bank72199 +72200 POINT(40.4048718540961 73.65225669599496) bank72200 +72201 POINT(40.223275471746895 73.31370354020328) bank72201 +72202 POINT(40.64045867326125 73.13272971988033) bank72202 +72203 POINT(40.168935475814976 74.4496516241075) bank72203 +72204 POINT(40.4707531733411 73.49795396073138) bank72204 +72205 POINT(40.72733529085308 73.7322649254123) bank72205 +72206 POINT(41.251675382103194 74.34277495770675) bank72206 +72207 POINT(40.57446284557047 73.55259975438658) bank72207 +72208 POINT(41.12726437892041 74.67397142276917) bank72208 +72209 POINT(40.51446996575797 74.1675676766391) bank72209 +72210 POINT(40.56902809126051 74.00136559692794) bank72210 +72211 POINT(40.67945595337849 74.75740545051562) bank72211 +72212 POINT(40.271952058496225 74.1549732056828) bank72212 +72213 POINT(41.68812017883598 74.93926681520269) bank72213 +72214 POINT(40.93222710736128 74.38430849071878) bank72214 +72215 POINT(41.22417591375952 74.44632316275053) bank72215 +72216 POINT(40.229932418838644 73.95313094167975) bank72216 +72217 POINT(40.90082787238913 74.867235532838) bank72217 +72218 POINT(40.3574453442672 73.04734995415077) bank72218 +72219 POINT(40.09855885978542 73.50785082791732) bank72219 +72220 POINT(40.659871158615246 73.07363484308352) bank72220 +72221 POINT(41.14482200019974 73.09438110147323) bank72221 +72222 POINT(41.66277946374073 73.73292653427912) bank72222 +72223 POINT(41.26933054525806 74.80515948730455) bank72223 +72224 POINT(39.957074817626996 73.13650356679001) bank72224 +72225 POINT(41.54689475034109 73.90610492225265) bank72225 +72226 POINT(39.78689481705231 74.29929505036277) bank72226 +72227 POINT(39.790140895248314 73.76431510569982) bank72227 +72228 POINT(40.6827790248066 73.74921257452958) bank72228 +72229 POINT(41.17908434041896 74.63730542596022) bank72229 +72230 POINT(40.71969662844991 73.30707242475789) bank72230 +72231 POINT(41.61038478954054 73.79043279419274) bank72231 +72232 POINT(41.05152531903539 74.13090067121885) bank72232 +72233 POINT(40.01425541164883 73.4906542911891) bank72233 +72234 POINT(39.7189650484891 74.20631600573819) bank72234 +72235 POINT(40.96798265466153 74.49452620279351) bank72235 +72236 POINT(41.136328680869205 74.0807486195353) bank72236 +72237 POINT(41.6932415012851 74.76238912433517) bank72237 +72238 POINT(40.39618689083358 73.93241453834548) bank72238 +72239 POINT(40.382779799545276 73.71007113893204) bank72239 +72240 POINT(41.262070329256844 73.21352461443686) bank72240 +72241 POINT(40.99324051551765 73.9976268569832) bank72241 +72242 POINT(39.77270708077561 73.69971513276538) bank72242 +72243 POINT(41.29020429430471 74.51172685002315) bank72243 +72244 POINT(40.8592262243481 73.78258939896327) bank72244 +72245 POINT(40.90057661175706 74.84457144735616) bank72245 +72246 POINT(41.096338987001864 73.39788084110637) bank72246 +72247 POINT(41.710282735633335 74.79789555509522) bank72247 +72248 POINT(40.91861549047584 73.81300328480881) bank72248 +72249 POINT(40.06204457584844 73.24765698408213) bank72249 +72250 POINT(41.23014692433699 74.43971047707913) bank72250 +72251 POINT(40.48441981466919 73.90959464080464) bank72251 +72252 POINT(40.13986077622031 73.11649387353415) bank72252 +72253 POINT(40.37929583858145 74.70639926727193) bank72253 +72254 POINT(41.247240066129606 73.1223818053823) bank72254 +72255 POINT(40.19497990517699 74.36071112101682) bank72255 +72256 POINT(40.25390784890457 73.74435646034836) bank72256 +72257 POINT(40.399229533245155 74.99992708781677) bank72257 +72258 POINT(41.43889002488802 73.10301614748775) bank72258 +72259 POINT(41.63299279892889 74.74215833912449) bank72259 +72260 POINT(40.13736709398175 73.95295063766514) bank72260 +72261 POINT(41.4429774092614 73.8634815351318) bank72261 +72262 POINT(41.528451194735865 73.04564703221317) bank72262 +72263 POINT(40.99305738587329 73.78388587487267) bank72263 +72264 POINT(39.940900781009034 73.12762662799368) bank72264 +72265 POINT(41.49286405445195 74.15415643085176) bank72265 +72266 POINT(40.307054812592746 73.09432656771445) bank72266 +72267 POINT(41.27924947700208 73.66858739850935) bank72267 +72268 POINT(39.93140619286644 73.53542880278641) bank72268 +72269 POINT(41.406750503361685 74.46670124375457) bank72269 +72270 POINT(40.03429248618289 74.22860231316444) bank72270 +72271 POINT(41.03145640154268 73.57451678499929) bank72271 +72272 POINT(41.15214532416364 73.41969832937002) bank72272 +72273 POINT(41.245193418875196 73.71712857468617) bank72273 +72274 POINT(40.12862131679046 73.39350138187731) bank72274 +72275 POINT(41.13062009716135 74.83546640070782) bank72275 +72276 POINT(41.1417311877077 73.1264366604748) bank72276 +72277 POINT(40.104128667294844 73.62348066971224) bank72277 +72278 POINT(39.71544183353964 73.17597454992774) bank72278 +72279 POINT(41.563706783007454 74.74886923918562) bank72279 +72280 POINT(40.480046663475306 74.38651123055139) bank72280 +72281 POINT(40.677441882711314 73.03261376414203) bank72281 +72282 POINT(41.24010701511822 74.28592551527144) bank72282 +72283 POINT(41.6182364851289 73.41292506074578) bank72283 +72284 POINT(41.67847771947321 74.10403178486443) bank72284 +72285 POINT(40.49819047770098 73.56004602806505) bank72285 +72286 POINT(41.3831865938711 73.29359718597387) bank72286 +72287 POINT(40.29759736053157 74.03590271151039) bank72287 +72288 POINT(40.90379582016251 74.1091028064208) bank72288 +72289 POINT(40.23616295755415 74.42054699578283) bank72289 +72290 POINT(39.89334722633924 73.3254192384098) bank72290 +72291 POINT(40.107733578292454 74.99161872924803) bank72291 +72292 POINT(39.9910819618718 74.84124380217318) bank72292 +72293 POINT(41.670551339108215 74.39415846762748) bank72293 +72294 POINT(39.92462464009834 74.84328477932486) bank72294 +72295 POINT(40.62454100360347 74.54071594384581) bank72295 +72296 POINT(40.73158368715334 74.0626585397936) bank72296 +72297 POINT(41.37029978786301 74.1924357112526) bank72297 +72298 POINT(40.63266498413656 74.16375364454186) bank72298 +72299 POINT(40.52028698200088 74.35374612479853) bank72299 +72300 POINT(40.162062790078494 73.93181587786714) bank72300 +72301 POINT(41.31569764180743 73.74684553907846) bank72301 +72302 POINT(40.73220505926055 73.2211724248157) bank72302 +72303 POINT(40.221265830980016 73.25947926482182) bank72303 +72304 POINT(41.21148484863645 74.21131649803229) bank72304 +72305 POINT(40.38469611385017 74.99517923490315) bank72305 +72306 POINT(40.027122728667834 73.19789338378826) bank72306 +72307 POINT(39.718470992029374 74.465451051608) bank72307 +72308 POINT(39.76508639127265 73.90922269677708) bank72308 +72309 POINT(40.67895237398657 74.65820681134547) bank72309 +72310 POINT(41.49957481000292 74.31327450438286) bank72310 +72311 POINT(40.785033842051085 73.60847926723073) bank72311 +72312 POINT(39.81361252160939 74.01466671446812) bank72312 +72313 POINT(40.44312527243049 73.3165541867457) bank72313 +72314 POINT(40.177649118257726 73.6179514183747) bank72314 +72315 POINT(41.697714843383835 73.23180342094118) bank72315 +72316 POINT(40.74753568468812 73.64295930259324) bank72316 +72317 POINT(41.71124115192002 73.9933320958567) bank72317 +72318 POINT(41.53481382001959 74.31494527611551) bank72318 +72319 POINT(40.54549804909355 74.74675695911263) bank72319 +72320 POINT(41.40026795890304 73.63816490967756) bank72320 +72321 POINT(40.922643999245835 74.19027638406281) bank72321 +72322 POINT(41.27095038257414 75.00439087691835) bank72322 +72323 POINT(41.30471479377478 73.80369257554821) bank72323 +72324 POINT(40.75892047372957 73.17608738362988) bank72324 +72325 POINT(41.04264605427882 73.44264477586886) bank72325 +72326 POINT(40.21189632812222 74.91308616785248) bank72326 +72327 POINT(41.53753373464542 73.77020351782896) bank72327 +72328 POINT(41.5166348765338 73.49338115973218) bank72328 +72329 POINT(41.20626642226974 73.51784984533549) bank72329 +72330 POINT(40.430388278763736 74.35585982211754) bank72330 +72331 POINT(41.682960281802465 74.53326122628228) bank72331 +72332 POINT(39.9560696426421 74.69471888649421) bank72332 +72333 POINT(39.733520230220705 74.98838580809516) bank72333 +72334 POINT(40.02408244355642 73.72341134946292) bank72334 +72335 POINT(40.85930825875432 74.76641012659182) bank72335 +72336 POINT(40.2393741963103 74.64558805944073) bank72336 +72337 POINT(40.368315189195904 73.79000952661609) bank72337 +72338 POINT(39.720208986478916 74.02602673357303) bank72338 +72339 POINT(40.84843928142803 73.98806806164284) bank72339 +72340 POINT(40.20118165770983 73.59060508991513) bank72340 +72341 POINT(40.27555724660435 73.86878162199176) bank72341 +72342 POINT(41.530652678861955 73.03119852666174) bank72342 +72343 POINT(41.5528727614232 74.58988916486251) bank72343 +72344 POINT(40.3403270843248 73.49831995644574) bank72344 +72345 POINT(40.70785771590854 73.87594779665676) bank72345 +72346 POINT(40.414975135430495 73.34410931450653) bank72346 +72347 POINT(40.74802405271905 73.98858910752125) bank72347 +72348 POINT(40.00430763328335 73.62076588804663) bank72348 +72349 POINT(40.735105980769355 74.67829297861252) bank72349 +72350 POINT(40.92188033293877 74.5373099180969) bank72350 +72351 POINT(41.49289896923115 73.96591138411699) bank72351 +72352 POINT(39.78888212591327 73.39596196376834) bank72352 +72353 POINT(40.49244749248156 73.36702033702429) bank72353 +72354 POINT(40.58456770749285 73.04770081050428) bank72354 +72355 POINT(40.97195928058662 74.26070953153624) bank72355 +72356 POINT(40.5694920448826 74.58044332585509) bank72356 +72357 POINT(40.13415088255761 73.68800334692983) bank72357 +72358 POINT(40.13764805897901 74.76218586481937) bank72358 +72359 POINT(41.55019732455464 73.06012128983535) bank72359 +72360 POINT(40.3640060287067 73.07749199625039) bank72360 +72361 POINT(40.196508980951386 73.40928136198583) bank72361 +72362 POINT(41.37061225287073 73.58402757804934) bank72362 +72363 POINT(41.209434513113045 73.38185310521479) bank72363 +72364 POINT(41.70568439372575 74.17330465152459) bank72364 +72365 POINT(41.18154575460654 74.56566331445433) bank72365 +72366 POINT(41.19116203068544 73.07400677112982) bank72366 +72367 POINT(40.109209468689016 73.68298412124315) bank72367 +72368 POINT(40.67050916969234 74.49964439688712) bank72368 +72369 POINT(40.77257221345863 73.49655474714403) bank72369 +72370 POINT(40.33586230955217 74.35263202453919) bank72370 +72371 POINT(41.50091682767201 74.3681096788821) bank72371 +72372 POINT(40.794570045160306 73.87597290198529) bank72372 +72373 POINT(40.882968681921184 73.95764916904997) bank72373 +72374 POINT(40.223464525763575 74.70718254675467) bank72374 +72375 POINT(41.45093498327802 73.18826802713363) bank72375 +72376 POINT(41.23611564654677 74.03698661316977) bank72376 +72377 POINT(41.4678125257531 74.52143958994017) bank72377 +72378 POINT(40.351576261496646 73.03242993445579) bank72378 +72379 POINT(40.872168395501795 74.6454981478741) bank72379 +72380 POINT(40.469385293804514 74.88086809456938) bank72380 +72381 POINT(40.300144189395766 73.7951804293548) bank72381 +72382 POINT(40.538409101773055 74.0524072230547) bank72382 +72383 POINT(41.2917158296706 74.7891751422454) bank72383 +72384 POINT(39.984785181891795 73.20071721302529) bank72384 +72385 POINT(41.581262450674664 73.93865113059276) bank72385 +72386 POINT(41.344294530077825 74.03262621527) bank72386 +72387 POINT(40.81634213976643 73.47598989967751) bank72387 +72388 POINT(41.56032361903852 74.88421505831757) bank72388 +72389 POINT(40.14409440247077 74.55267291902818) bank72389 +72390 POINT(41.2829780929271 73.17955919235807) bank72390 +72391 POINT(41.40573141938744 73.89574716145626) bank72391 +72392 POINT(40.99380906188469 73.76380545696296) bank72392 +72393 POINT(41.40276013460101 74.51202584993317) bank72393 +72394 POINT(41.22926417080193 73.06545170761648) bank72394 +72395 POINT(40.51500219710362 73.38687870669719) bank72395 +72396 POINT(40.385633558240556 73.25823898140814) bank72396 +72397 POINT(41.434349641108845 74.96027026350349) bank72397 +72398 POINT(40.56756391115855 74.41114346633414) bank72398 +72399 POINT(40.279188967147604 74.64390435010023) bank72399 +72400 POINT(41.0898243168914 74.00795443472161) bank72400 +72401 POINT(41.07892447738431 74.05712408915811) bank72401 +72402 POINT(41.110148777252604 74.04702351727694) bank72402 +72403 POINT(40.04039066437355 74.13566060046405) bank72403 +72404 POINT(41.086097762065684 73.2753370518177) bank72404 +72405 POINT(41.5357002480759 73.01647450215287) bank72405 +72406 POINT(41.11686954341104 74.60228995573823) bank72406 +72407 POINT(41.35761389355528 73.8296443848792) bank72407 +72408 POINT(41.684157248605054 74.5735174945476) bank72408 +72409 POINT(41.188792106578596 74.60357394728301) bank72409 +72410 POINT(40.06422332875971 74.26743089902205) bank72410 +72411 POINT(41.66121856147728 75.00485141903013) bank72411 +72412 POINT(40.083276897358274 73.19955886089758) bank72412 +72413 POINT(40.97802913709336 74.74127624740467) bank72413 +72414 POINT(39.9268381326124 74.39885872792956) bank72414 +72415 POINT(40.44977243068311 74.90334699575712) bank72415 +72416 POINT(40.89954290986174 74.38281067635596) bank72416 +72417 POINT(40.579511890936125 73.26088315515699) bank72417 +72418 POINT(41.36954685314884 74.96093753577233) bank72418 +72419 POINT(40.283623859182526 73.44235738262792) bank72419 +72420 POINT(40.74392139277415 73.8551699951676) bank72420 +72421 POINT(40.7817078307636 74.27556508545254) bank72421 +72422 POINT(40.36025906707458 73.63362977007391) bank72422 +72423 POINT(39.86465221096075 74.70395595580112) bank72423 +72424 POINT(40.87017693722177 73.23843321405774) bank72424 +72425 POINT(40.6761652942665 74.50308189964184) bank72425 +72426 POINT(39.82739319733549 74.27092743920065) bank72426 +72427 POINT(40.923998347020664 73.79132786516546) bank72427 +72428 POINT(39.80522244594118 73.85106425529752) bank72428 +72429 POINT(40.628257894444765 73.1793561394297) bank72429 +72430 POINT(40.39612549521955 73.93409383216985) bank72430 +72431 POINT(40.702010428790956 74.94721969606536) bank72431 +72432 POINT(41.157617835678636 74.35164765069544) bank72432 +72433 POINT(41.09709775405546 74.68689415281321) bank72433 +72434 POINT(41.23256995934687 73.7422656301629) bank72434 +72435 POINT(40.58564352147152 74.19625492549034) bank72435 +72436 POINT(39.992996345459005 73.65154405377965) bank72436 +72437 POINT(41.080645847166686 74.39340451609942) bank72437 +72438 POINT(41.548691044087036 73.43074764534775) bank72438 +72439 POINT(39.79376921293302 73.11947680628185) bank72439 +72440 POINT(40.11875026095759 74.60726966755183) bank72440 +72441 POINT(40.95992952402081 74.44568935018687) bank72441 +72442 POINT(40.01800678030155 73.97214177499102) bank72442 +72443 POINT(40.57598048555939 73.35533427695613) bank72443 +72444 POINT(41.19107086798307 74.29137108950731) bank72444 +72445 POINT(40.14281821008712 74.08824623703683) bank72445 +72446 POINT(39.7836257790168 73.12390681499309) bank72446 +72447 POINT(40.47377693474065 73.1698292236436) bank72447 +72448 POINT(40.07632780078922 73.23541478063058) bank72448 +72449 POINT(41.29195035029169 74.14592144822794) bank72449 +72450 POINT(40.87408653398759 73.61591825193112) bank72450 +72451 POINT(41.545232636443075 73.44346880866428) bank72451 +72452 POINT(39.83984561448155 73.81493507514594) bank72452 +72453 POINT(40.9698886053499 73.86107026490541) bank72453 +72454 POINT(41.24333462321468 74.49469311840032) bank72454 +72455 POINT(40.44576256845655 74.04520429335254) bank72455 +72456 POINT(39.926399799001764 73.50233733555712) bank72456 +72457 POINT(41.32440575721976 74.44238660563526) bank72457 +72458 POINT(40.06129617422555 74.44076956804733) bank72458 +72459 POINT(40.010377489064126 74.0696784847431) bank72459 +72460 POINT(39.95153918819415 73.65039467538764) bank72460 +72461 POINT(41.33273242255436 74.7696940805617) bank72461 +72462 POINT(41.00266713452693 74.98995809417406) bank72462 +72463 POINT(40.42502064399549 74.43102815187108) bank72463 +72464 POINT(41.23243414272108 74.65261520669584) bank72464 +72465 POINT(40.619888048116536 74.06517611277009) bank72465 +72466 POINT(40.703035373831646 73.90926155908829) bank72466 +72467 POINT(40.91221242555097 73.79014280741187) bank72467 +72468 POINT(41.03212223684767 73.07615285460722) bank72468 +72469 POINT(41.521247483785196 73.09871574470573) bank72469 +72470 POINT(40.650911529998595 73.74307242345591) bank72470 +72471 POINT(40.476578857021195 74.96808314486523) bank72471 +72472 POINT(40.10942567572965 73.54506816660464) bank72472 +72473 POINT(41.58207771135897 74.6461016169914) bank72473 +72474 POINT(39.92018642805818 74.92174655356295) bank72474 +72475 POINT(41.63420617184288 73.96427991932521) bank72475 +72476 POINT(40.12684038397861 74.09514143491683) bank72476 +72477 POINT(40.545021792056055 74.93748223600039) bank72477 +72478 POINT(41.47783833210005 73.89941905674577) bank72478 +72479 POINT(40.36696780188549 74.70695430477315) bank72479 +72480 POINT(40.238074902721706 74.62850974575369) bank72480 +72481 POINT(40.84040658636866 73.92666825225139) bank72481 +72482 POINT(40.28048814479716 74.91894625214037) bank72482 +72483 POINT(40.20385486423064 74.88044924843447) bank72483 +72484 POINT(40.69304917315246 73.82735324122599) bank72484 +72485 POINT(40.356505291557255 74.98372728898998) bank72485 +72486 POINT(40.287619083060676 73.12234628832455) bank72486 +72487 POINT(40.58036471968829 74.47367470398146) bank72487 +72488 POINT(40.89310619411505 74.84048405852191) bank72488 +72489 POINT(41.20393171853459 73.99377225258115) bank72489 +72490 POINT(41.423213360055854 73.17174331906708) bank72490 +72491 POINT(40.27900145157065 74.85579233161452) bank72491 +72492 POINT(41.46509657191291 74.06805678951594) bank72492 +72493 POINT(40.699264267503935 74.24220294723045) bank72493 +72494 POINT(39.9016231288686 73.10602454958365) bank72494 +72495 POINT(41.507310063112065 74.19641824315649) bank72495 +72496 POINT(41.51042640075326 74.61687554267313) bank72496 +72497 POINT(40.035137203565434 73.69266268447824) bank72497 +72498 POINT(40.67359970859578 74.8316113326064) bank72498 +72499 POINT(39.8517606664838 73.12954956664849) bank72499 +72500 POINT(40.36920408167927 73.41046298493112) bank72500 +72501 POINT(40.15594609085788 74.83741230152297) bank72501 +72502 POINT(41.67190911325826 74.5915404654082) bank72502 +72503 POINT(40.62551992461869 73.30738519235237) bank72503 +72504 POINT(40.186522410209456 74.47008794443126) bank72504 +72505 POINT(41.07120839437722 74.34959906396098) bank72505 +72506 POINT(41.5869345727018 73.31248599157033) bank72506 +72507 POINT(41.47296947784297 74.61008271104298) bank72507 +72508 POINT(41.18891353772342 73.7382727578876) bank72508 +72509 POINT(39.94605965758871 74.59219595170416) bank72509 +72510 POINT(41.09192821471291 73.58641789267868) bank72510 +72511 POINT(40.97856144665924 74.3567685242927) bank72511 +72512 POINT(39.8953712552619 74.29539327572098) bank72512 +72513 POINT(40.41035732936262 74.26955402800033) bank72513 +72514 POINT(40.13102093649335 73.43058102700199) bank72514 +72515 POINT(40.69652369265356 73.25574855319) bank72515 +72516 POINT(40.955384955125915 73.76460218223009) bank72516 +72517 POINT(40.55596532623724 74.49916598643958) bank72517 +72518 POINT(40.56223246232564 73.62515898729694) bank72518 +72519 POINT(40.97443466326201 74.51317517742434) bank72519 +72520 POINT(41.68595947134296 74.88289400101809) bank72520 +72521 POINT(40.37251348463125 74.89854049031324) bank72521 +72522 POINT(41.2571860126033 73.74318978803491) bank72522 +72523 POINT(40.8921092379767 73.05416333416136) bank72523 +72524 POINT(40.65368294236485 73.96292345399492) bank72524 +72525 POINT(40.429670274101 74.31225668207435) bank72525 +72526 POINT(40.95466489318673 73.34393641153382) bank72526 +72527 POINT(40.83628798445194 73.13646315920386) bank72527 +72528 POINT(41.50455682383948 74.24482086203729) bank72528 +72529 POINT(40.38493775324619 73.46782063387886) bank72529 +72530 POINT(39.847187251481195 73.2377076912903) bank72530 +72531 POINT(40.221005496436085 74.596641165435) bank72531 +72532 POINT(40.13478418099966 74.80244463354782) bank72532 +72533 POINT(40.14440471397297 73.6691353377947) bank72533 +72534 POINT(41.487561158400446 73.39281976105511) bank72534 +72535 POINT(41.26720147737148 73.70649893573662) bank72535 +72536 POINT(40.80671350020175 73.42787085420912) bank72536 +72537 POINT(40.550007381966516 74.36308683449296) bank72537 +72538 POINT(40.62200667024381 74.74244161332174) bank72538 +72539 POINT(40.71135860688335 73.95492778745691) bank72539 +72540 POINT(41.525845008502245 73.20973506757683) bank72540 +72541 POINT(40.45197186390032 74.35049572060197) bank72541 +72542 POINT(40.323722586145564 73.3726587751972) bank72542 +72543 POINT(41.52369696064331 73.06204509218153) bank72543 +72544 POINT(40.82581492201412 73.35985036526351) bank72544 +72545 POINT(40.40333151076097 73.47284239308009) bank72545 +72546 POINT(40.08080780226456 74.581769802155) bank72546 +72547 POINT(39.7751362358507 73.88365913067109) bank72547 +72548 POINT(41.16365940851615 74.5076851903666) bank72548 +72549 POINT(40.403556949224985 73.10687979495772) bank72549 +72550 POINT(41.59841132572226 73.26889410290312) bank72550 +72551 POINT(40.72039724980344 74.25607491970263) bank72551 +72552 POINT(40.68562283328864 74.12150503023942) bank72552 +72553 POINT(40.56725482951636 74.17394017221034) bank72553 +72554 POINT(41.354670349048085 74.36739675137076) bank72554 +72555 POINT(41.6336182782873 73.45329094791981) bank72555 +72556 POINT(41.15594228662293 74.40767331250264) bank72556 +72557 POINT(41.21435685474442 74.80452477293107) bank72557 +72558 POINT(41.312821640844064 74.68652938791985) bank72558 +72559 POINT(40.10360918587206 74.47532673222372) bank72559 +72560 POINT(39.78934513931151 74.86680544464022) bank72560 +72561 POINT(40.45732009985056 74.78385957568236) bank72561 +72562 POINT(40.52389642855559 73.23256196121511) bank72562 +72563 POINT(40.30095589920959 74.12324793006589) bank72563 +72564 POINT(39.91487052274169 73.90325224195169) bank72564 +72565 POINT(40.538980921764775 73.15487676691357) bank72565 +72566 POINT(40.66852215492476 73.11162735457665) bank72566 +72567 POINT(40.61383685639087 73.908668690003) bank72567 +72568 POINT(41.562865325976595 74.49401144986267) bank72568 +72569 POINT(40.39305318086876 73.7217138792602) bank72569 +72570 POINT(41.20910885232632 74.70582898840169) bank72570 +72571 POINT(39.77234707140371 73.76473826463861) bank72571 +72572 POINT(40.44429173301706 74.87140186235531) bank72572 +72573 POINT(40.37826211644594 73.65682580567696) bank72573 +72574 POINT(39.872934027491354 73.54700869819607) bank72574 +72575 POINT(40.34650287994329 73.42254132813429) bank72575 +72576 POINT(40.08056460226189 74.32338137624788) bank72576 +72577 POINT(40.585574235595 73.25387307045328) bank72577 +72578 POINT(40.47985865144782 73.21029639835275) bank72578 +72579 POINT(41.11122780702648 74.17944302345349) bank72579 +72580 POINT(41.175070941618316 74.31804231543042) bank72580 +72581 POINT(40.69188177349147 74.82259332893193) bank72581 +72582 POINT(40.853666747172795 74.02393270601274) bank72582 +72583 POINT(41.16779998252525 73.14749577429409) bank72583 +72584 POINT(40.356138219865194 73.22833755269089) bank72584 +72585 POINT(39.778531118476735 74.53300737985582) bank72585 +72586 POINT(41.4163515178245 74.43714455921724) bank72586 +72587 POINT(40.38932236332268 73.08696740594577) bank72587 +72588 POINT(41.70126158860732 74.2558972302111) bank72588 +72589 POINT(40.8503259679463 74.79570351322798) bank72589 +72590 POINT(41.29813512326799 73.45186519133388) bank72590 +72591 POINT(41.14050751483065 73.47501519837782) bank72591 +72592 POINT(41.447842514780945 74.23907759303084) bank72592 +72593 POINT(41.485718982142785 74.73941249275754) bank72593 +72594 POINT(40.62304759865622 74.26523111109915) bank72594 +72595 POINT(40.19310717712692 73.3682388588673) bank72595 +72596 POINT(40.38930197012215 73.25219651070648) bank72596 +72597 POINT(41.48980139440377 74.01323772210249) bank72597 +72598 POINT(40.188048737588765 74.29142936575407) bank72598 +72599 POINT(40.718517911408426 74.5402218041194) bank72599 +72600 POINT(41.04387231916225 73.61677845858986) bank72600 +72601 POINT(40.035485634862404 74.66775930663248) bank72601 +72602 POINT(39.974244385993224 73.48951308440343) bank72602 +72603 POINT(41.18989722025733 74.22747167644633) bank72603 +72604 POINT(40.28565591128199 74.66395201329439) bank72604 +72605 POINT(39.94903937359039 74.58697399446936) bank72605 +72606 POINT(41.3905418715741 73.1016055341898) bank72606 +72607 POINT(39.75894769646161 73.71409724743052) bank72607 +72608 POINT(41.368587207123745 74.10096827147339) bank72608 +72609 POINT(39.745245766849465 73.66422580926955) bank72609 +72610 POINT(40.77583035047482 73.13802415442173) bank72610 +72611 POINT(41.33969585741546 73.41375019522651) bank72611 +72612 POINT(39.936175609631654 73.50208195838555) bank72612 +72613 POINT(41.615342188096825 73.07885163644086) bank72613 +72614 POINT(39.80823103190624 73.5275838410565) bank72614 +72615 POINT(40.103876913006474 74.21005286727564) bank72615 +72616 POINT(41.25714299324612 73.98169089626687) bank72616 +72617 POINT(39.97595432499967 74.50480851476256) bank72617 +72618 POINT(40.44660272183961 74.43563341717305) bank72618 +72619 POINT(40.237831396514544 73.46136251743906) bank72619 +72620 POINT(40.952049632946824 73.17448532341021) bank72620 +72621 POINT(41.22783638783118 74.07831190404059) bank72621 +72622 POINT(40.26125821361182 74.55523229732864) bank72622 +72623 POINT(41.373733147514 74.82435537225115) bank72623 +72624 POINT(40.96551413614706 73.59047544415444) bank72624 +72625 POINT(41.624260195376316 73.16259209820683) bank72625 +72626 POINT(41.14571029322194 73.79749259261781) bank72626 +72627 POINT(41.09365595911521 74.87675621636106) bank72627 +72628 POINT(39.917970190553774 73.20496840068702) bank72628 +72629 POINT(40.522001803662896 73.39385558330042) bank72629 +72630 POINT(41.19516824432191 73.81189262696803) bank72630 +72631 POINT(41.615908930526224 74.25726634719874) bank72631 +72632 POINT(41.628718111778056 74.21834586382053) bank72632 +72633 POINT(40.10257553838743 74.35525717932475) bank72633 +72634 POINT(40.88662976929669 73.411960584883) bank72634 +72635 POINT(40.96515450988024 74.21090864169844) bank72635 +72636 POINT(39.89483519798866 74.05624968864434) bank72636 +72637 POINT(41.01712513919662 73.55617332786524) bank72637 +72638 POINT(40.63346888827773 74.81308776685718) bank72638 +72639 POINT(41.18737641427483 73.54014160522121) bank72639 +72640 POINT(40.13298583232168 74.19164854473382) bank72640 +72641 POINT(40.60815614350577 73.20641863119616) bank72641 +72642 POINT(41.291082731177546 73.2289488151173) bank72642 +72643 POINT(40.352896421891366 74.83143049393514) bank72643 +72644 POINT(41.60998827116273 73.16924082365314) bank72644 +72645 POINT(39.77964586501466 73.81144544376212) bank72645 +72646 POINT(39.77672005389435 74.1038109117586) bank72646 +72647 POINT(39.98484255331813 73.3822219200166) bank72647 +72648 POINT(40.23254080506064 73.84014333526486) bank72648 +72649 POINT(40.53581962305665 73.90605664697355) bank72649 +72650 POINT(40.54702265920784 74.31622974900603) bank72650 +72651 POINT(41.56463053034625 74.32661207905282) bank72651 +72652 POINT(40.60910914982703 73.36886454986573) bank72652 +72653 POINT(41.073637595503065 73.79399099086031) bank72653 +72654 POINT(40.53273248660807 74.77527882619849) bank72654 +72655 POINT(40.97689461059454 74.36320007648672) bank72655 +72656 POINT(40.28684749271208 73.29304074512754) bank72656 +72657 POINT(40.95512799550774 73.07323198766414) bank72657 +72658 POINT(40.09671523519103 74.36839570905902) bank72658 +72659 POINT(41.00133648812166 74.11389670287724) bank72659 +72660 POINT(40.677484069836964 73.54207212819594) bank72660 +72661 POINT(39.78644716749524 73.03653068291263) bank72661 +72662 POINT(41.65815424721552 73.18416815554916) bank72662 +72663 POINT(40.427495561787104 73.09922724572678) bank72663 +72664 POINT(41.58000942670498 73.55594476982148) bank72664 +72665 POINT(41.671161270301305 74.88142130400922) bank72665 +72666 POINT(40.36861418171228 73.31455585918779) bank72666 +72667 POINT(41.380175979049625 74.62335893044025) bank72667 +72668 POINT(40.581048440628805 74.3777633566672) bank72668 +72669 POINT(40.469223941839125 73.52530360847034) bank72669 +72670 POINT(40.33689241461424 74.54032231840068) bank72670 +72671 POINT(41.51232438315996 73.8846787869276) bank72671 +72672 POINT(39.96328834507276 73.50249981042876) bank72672 +72673 POINT(40.5432667647344 73.77024213088737) bank72673 +72674 POINT(41.66617455044233 73.66934107481391) bank72674 +72675 POINT(40.442061201380525 74.37668399216054) bank72675 +72676 POINT(41.32281674158735 74.47917961823798) bank72676 +72677 POINT(41.399838620383484 73.18659738367538) bank72677 +72678 POINT(41.27008947678338 73.07130734921024) bank72678 +72679 POINT(41.32694594329497 73.94147271065522) bank72679 +72680 POINT(39.74533946123675 73.40603404863623) bank72680 +72681 POINT(40.80675216218868 73.01965594990804) bank72681 +72682 POINT(40.07434461331375 73.40782556175051) bank72682 +72683 POINT(40.780582235920484 74.3216828257761) bank72683 +72684 POINT(40.77629145602157 73.87744970174019) bank72684 +72685 POINT(41.33540629150487 74.64165566535486) bank72685 +72686 POINT(40.66436109274414 74.24875132781719) bank72686 +72687 POINT(40.46971856794725 73.42544406760192) bank72687 +72688 POINT(40.67577612031262 73.06783478827349) bank72688 +72689 POINT(40.48007306020337 73.46493677269778) bank72689 +72690 POINT(40.5299083298322 74.71490506312138) bank72690 +72691 POINT(40.96395046034613 73.21894476201783) bank72691 +72692 POINT(40.00972293638942 74.15139057500744) bank72692 +72693 POINT(40.75121718496866 73.59682002896756) bank72693 +72694 POINT(40.23031134097878 73.90464123045246) bank72694 +72695 POINT(40.53202221155464 74.053672723544) bank72695 +72696 POINT(39.91551748618354 73.80850452877796) bank72696 +72697 POINT(40.85481190741514 73.15601952767534) bank72697 +72698 POINT(41.12438148561234 74.94387259703934) bank72698 +72699 POINT(40.02476589469483 74.25827545851405) bank72699 +72700 POINT(40.756574599566264 73.32525186666982) bank72700 +72701 POINT(41.49332814099806 73.1641788530611) bank72701 +72702 POINT(40.45680459172871 73.97193372426436) bank72702 +72703 POINT(40.67844435731453 74.49005567127692) bank72703 +72704 POINT(41.65330080931162 73.09406133812183) bank72704 +72705 POINT(40.8300212441602 73.34246850991329) bank72705 +72706 POINT(40.90763752691239 74.80065991429275) bank72706 +72707 POINT(41.44586637899046 73.15198493668049) bank72707 +72708 POINT(41.694283386798304 73.63914999982218) bank72708 +72709 POINT(41.43658883328003 74.0935197852796) bank72709 +72710 POINT(40.38771893248623 74.63679225686401) bank72710 +72711 POINT(41.359559994309144 74.65011209611565) bank72711 +72712 POINT(39.92920322673218 74.79137072424723) bank72712 +72713 POINT(40.63285375406184 73.22354263269231) bank72713 +72714 POINT(41.53219730587113 73.94890976003853) bank72714 +72715 POINT(40.834341824607066 74.02273689981082) bank72715 +72716 POINT(40.827587716799165 73.04991659342951) bank72716 +72717 POINT(41.61840072096404 73.50316896051683) bank72717 +72718 POINT(40.053717284755585 74.960595920584) bank72718 +72719 POINT(40.00161710940351 73.80369470731013) bank72719 +72720 POINT(40.32113040361244 74.18530102838798) bank72720 +72721 POINT(40.08714257840261 74.49523687155907) bank72721 +72722 POINT(40.54666667996889 73.96910669991404) bank72722 +72723 POINT(40.90453599356437 73.4033204178679) bank72723 +72724 POINT(41.55643435841221 73.5701860799368) bank72724 +72725 POINT(40.02849713327099 73.0828556149964) bank72725 +72726 POINT(41.347259079540194 73.24695059365932) bank72726 +72727 POINT(40.76546716460891 73.28531485032131) bank72727 +72728 POINT(40.18760053551353 74.67002109577133) bank72728 +72729 POINT(41.07940493743888 74.99898275952495) bank72729 +72730 POINT(40.702894516198874 73.03147113549844) bank72730 +72731 POINT(40.88343211234722 74.59200241005686) bank72731 +72732 POINT(39.95462651840896 74.06533832924731) bank72732 +72733 POINT(39.89868420892123 73.10068543319814) bank72733 +72734 POINT(40.13248235546482 74.30776081421666) bank72734 +72735 POINT(41.059891282489716 74.3216126775123) bank72735 +72736 POINT(41.44686785981404 73.20279034313431) bank72736 +72737 POINT(39.95603010190799 73.66706067579906) bank72737 +72738 POINT(39.74479989720324 74.16340544383523) bank72738 +72739 POINT(41.20604648804655 74.4727824319257) bank72739 +72740 POINT(41.52394963465842 74.0569817029159) bank72740 +72741 POINT(41.539881719363365 74.46305172936691) bank72741 +72742 POINT(40.50632493433878 73.96191553417155) bank72742 +72743 POINT(39.93088361760656 73.49820267487249) bank72743 +72744 POINT(41.56160285849656 74.48314143961376) bank72744 +72745 POINT(40.94503476185931 74.3575924961114) bank72745 +72746 POINT(40.72148402642448 74.5204203021784) bank72746 +72747 POINT(39.93317048606634 74.55829399824941) bank72747 +72748 POINT(40.50177281499663 74.43631620743959) bank72748 +72749 POINT(39.81769956765714 74.54993826742789) bank72749 +72750 POINT(40.38539663297479 74.42065745088038) bank72750 +72751 POINT(40.35942220437404 73.11028529184249) bank72751 +72752 POINT(40.44602533257082 73.58960208566792) bank72752 +72753 POINT(39.7996975744841 74.23369938871741) bank72753 +72754 POINT(39.85409252973803 73.53174828654004) bank72754 +72755 POINT(41.22175234291891 74.88085692300774) bank72755 +72756 POINT(40.585666444892304 74.41994741392351) bank72756 +72757 POINT(40.76712887379345 73.29563840161103) bank72757 +72758 POINT(40.240195048184624 73.98801883835432) bank72758 +72759 POINT(40.412105174540976 74.89472484710618) bank72759 +72760 POINT(40.025083790014754 74.30924041586283) bank72760 +72761 POINT(39.73734338844646 73.76403512152211) bank72761 +72762 POINT(39.774117599517716 73.05178843409419) bank72762 +72763 POINT(40.13692355572038 74.2584540144398) bank72763 +72764 POINT(41.4051376459719 73.04923298023135) bank72764 +72765 POINT(40.53685094741417 73.38362182817718) bank72765 +72766 POINT(40.73518950160426 73.00697287401267) bank72766 +72767 POINT(41.134535656040924 74.24089896259808) bank72767 +72768 POINT(39.99544487203537 74.87791777162921) bank72768 +72769 POINT(40.5694569974703 74.60096706879536) bank72769 +72770 POINT(41.22674112587855 73.7479705400586) bank72770 +72771 POINT(39.771621816312546 73.83492393985757) bank72771 +72772 POINT(40.06537401844219 74.4203351002961) bank72772 +72773 POINT(41.229650855125 73.28527696690878) bank72773 +72774 POINT(40.19103692544909 73.34128345386137) bank72774 +72775 POINT(40.139425210473746 74.74552799211364) bank72775 +72776 POINT(40.905156032289206 74.62749306426669) bank72776 +72777 POINT(41.048247177167305 73.6724161193355) bank72777 +72778 POINT(41.629704800519995 73.96987151354622) bank72778 +72779 POINT(39.80112905248823 73.69513621762017) bank72779 +72780 POINT(40.192550696483075 73.84414965581745) bank72780 +72781 POINT(40.46337945917491 74.1241562094961) bank72781 +72782 POINT(41.56716758660004 74.75994062986045) bank72782 +72783 POINT(40.745470765976386 73.5768663348868) bank72783 +72784 POINT(39.906779053069364 74.66343533937919) bank72784 +72785 POINT(40.79319900491209 73.28973388736387) bank72785 +72786 POINT(40.01081183585294 73.19538508563815) bank72786 +72787 POINT(41.215864697442605 74.19829863018512) bank72787 +72788 POINT(40.08364857155881 73.26449780545629) bank72788 +72789 POINT(40.11900796129985 74.662081757809) bank72789 +72790 POINT(41.14147573198898 73.30379813318578) bank72790 +72791 POINT(41.54412454946668 73.32184780195979) bank72791 +72792 POINT(41.28596751611807 73.55510870800866) bank72792 +72793 POINT(40.120423761361536 73.90880188675649) bank72793 +72794 POINT(40.70386628212522 73.20024482224711) bank72794 +72795 POINT(39.859416503626996 74.4929866011233) bank72795 +72796 POINT(41.61118816742837 74.97618965365777) bank72796 +72797 POINT(40.13207024735347 73.46813362366845) bank72797 +72798 POINT(39.73116824922819 73.93154881233829) bank72798 +72799 POINT(41.667529692796386 74.49049432870264) bank72799 +72800 POINT(40.55178223300186 73.81835357806666) bank72800 +72801 POINT(40.669605163550166 73.7084504146149) bank72801 +72802 POINT(40.037674849146384 73.5479704252612) bank72802 +72803 POINT(40.322443424279825 74.06896230090808) bank72803 +72804 POINT(40.98262035380173 74.84880820387919) bank72804 +72805 POINT(40.794863091623924 74.38132436535737) bank72805 +72806 POINT(40.17482682527014 73.2170770255316) bank72806 +72807 POINT(41.09874459496125 73.00621124384885) bank72807 +72808 POINT(40.835536705803015 73.13850766635323) bank72808 +72809 POINT(41.16051864777451 74.62157846032305) bank72809 +72810 POINT(41.16952353392736 74.74021218609128) bank72810 +72811 POINT(40.431019033061595 73.48225992764822) bank72811 +72812 POINT(41.09216033676671 74.15694868090837) bank72812 +72813 POINT(39.85692986253144 73.92312255676883) bank72813 +72814 POINT(40.2838095719284 73.65914935028574) bank72814 +72815 POINT(40.93716250492975 74.69972776037483) bank72815 +72816 POINT(40.06522442524148 74.24565560927374) bank72816 +72817 POINT(39.982947339222555 74.64797957639371) bank72817 +72818 POINT(39.9517270311396 74.0627047587078) bank72818 +72819 POINT(41.518705226564165 73.7514093362901) bank72819 +72820 POINT(40.47055258255888 74.57520696631272) bank72820 +72821 POINT(40.39197496558091 74.7253048562934) bank72821 +72822 POINT(40.26883117300616 74.87411968597495) bank72822 +72823 POINT(41.22053608546309 74.11504117569223) bank72823 +72824 POINT(40.051578585098696 73.88706910346728) bank72824 +72825 POINT(41.40116526172043 74.48481181355356) bank72825 +72826 POINT(40.07162762336069 74.15902327078051) bank72826 +72827 POINT(40.65090412361148 74.16080953788111) bank72827 +72828 POINT(40.81562249509349 74.20963030338814) bank72828 +72829 POINT(40.38434843154777 73.6719447478181) bank72829 +72830 POINT(39.89912353713384 74.23012071035211) bank72830 +72831 POINT(39.829152952449114 73.99028820419664) bank72831 +72832 POINT(40.20235126349736 73.1095595720201) bank72832 +72833 POINT(39.75494860082533 74.83667819302963) bank72833 +72834 POINT(41.0263792002412 73.3764096483857) bank72834 +72835 POINT(40.39283556087549 73.8296873693329) bank72835 +72836 POINT(40.6686205953157 73.06234096750873) bank72836 +72837 POINT(41.62030194620928 74.15295381492497) bank72837 +72838 POINT(40.77558134843216 74.91699335919293) bank72838 +72839 POINT(40.503058421124145 73.14571135303251) bank72839 +72840 POINT(41.09407891775937 73.74821327067407) bank72840 +72841 POINT(40.54042806629867 73.40924082421736) bank72841 +72842 POINT(39.826376658629584 74.03523158835607) bank72842 +72843 POINT(39.93225623313033 73.69764510443906) bank72843 +72844 POINT(41.151753798537136 73.05122997783991) bank72844 +72845 POINT(41.024217064078 74.08124115866619) bank72845 +72846 POINT(41.05249868098102 74.78239329463045) bank72846 +72847 POINT(41.31524678405913 73.25129749216032) bank72847 +72848 POINT(40.45726486527121 73.78460129672565) bank72848 +72849 POINT(40.10160277669033 73.82648016539791) bank72849 +72850 POINT(41.08461004171811 73.97157208954182) bank72850 +72851 POINT(41.33371174477909 74.17712213336196) bank72851 +72852 POINT(40.8488283143325 74.93932437051093) bank72852 +72853 POINT(41.673166194843056 73.96156160078105) bank72853 +72854 POINT(39.879467531426 73.2085221870645) bank72854 +72855 POINT(39.904628101641165 74.18647559638363) bank72855 +72856 POINT(39.99102384139662 74.79185100768129) bank72856 +72857 POINT(40.19583923082902 74.28370895074859) bank72857 +72858 POINT(41.23625418408404 74.3455377660706) bank72858 +72859 POINT(41.180364739671056 73.24488461404025) bank72859 +72860 POINT(39.78341706138005 73.20024894143236) bank72860 +72861 POINT(41.56644285180104 74.23826510820714) bank72861 +72862 POINT(41.660812932167396 73.48978530239316) bank72862 +72863 POINT(41.071146201646656 73.23510845585146) bank72863 +72864 POINT(41.65253631425461 74.13471796149258) bank72864 +72865 POINT(40.637625907576506 74.72740683546434) bank72865 +72866 POINT(41.10621135119718 73.50681632440366) bank72866 +72867 POINT(40.131837951958694 74.64682399071381) bank72867 +72868 POINT(40.64127110735232 74.14408680795714) bank72868 +72869 POINT(40.48216977278912 74.51115843697374) bank72869 +72870 POINT(40.849406150319474 73.42828922848311) bank72870 +72871 POINT(40.57374509078995 73.10806444806197) bank72871 +72872 POINT(40.969171718925416 73.17355179408659) bank72872 +72873 POINT(41.027521528001955 74.79354777782324) bank72873 +72874 POINT(40.12262077072239 74.14415037067666) bank72874 +72875 POINT(40.15205188240658 73.49169594320061) bank72875 +72876 POINT(41.200959741097094 74.60301889160804) bank72876 +72877 POINT(41.21205156750803 73.85798050873667) bank72877 +72878 POINT(41.44702295566493 73.29678148774315) bank72878 +72879 POINT(40.517709411295456 73.61100922090459) bank72879 +72880 POINT(39.8051068878399 74.96933316228215) bank72880 +72881 POINT(41.30910772871732 74.81157394792237) bank72881 +72882 POINT(40.88015767522607 74.14821566198005) bank72882 +72883 POINT(40.54951107792883 73.65678944049402) bank72883 +72884 POINT(39.88631250120532 74.46587368688316) bank72884 +72885 POINT(40.812082786887295 74.16505941808809) bank72885 +72886 POINT(41.45176331299616 73.49411507948653) bank72886 +72887 POINT(40.06596618099109 74.35506040761285) bank72887 +72888 POINT(40.71963594226666 73.91371147093692) bank72888 +72889 POINT(40.60707581386082 74.03299073957061) bank72889 +72890 POINT(39.927694881772034 73.82514975063212) bank72890 +72891 POINT(41.396751130177094 73.80202583722742) bank72891 +72892 POINT(41.24347203491677 73.16731424908883) bank72892 +72893 POINT(40.16436258565642 73.81065076154475) bank72893 +72894 POINT(40.42393677754243 73.71579789998401) bank72894 +72895 POINT(39.73631133691021 73.66811484617556) bank72895 +72896 POINT(40.96692943782035 74.18499070993612) bank72896 +72897 POINT(39.83265110731121 73.92425697253982) bank72897 +72898 POINT(41.68260221528159 73.73003654071941) bank72898 +72899 POINT(41.65138384217474 74.07732037675125) bank72899 +72900 POINT(39.83905840969736 73.44801038827761) bank72900 +72901 POINT(41.242622936736424 73.28740767273837) bank72901 +72902 POINT(40.07075431646466 74.95583316521906) bank72902 +72903 POINT(41.577028278635936 74.92959741817056) bank72903 +72904 POINT(41.51618333992761 74.56766269481237) bank72904 +72905 POINT(41.55012734365852 73.26955238576377) bank72905 +72906 POINT(39.83929497277705 74.95751130826774) bank72906 +72907 POINT(40.99285540034614 74.67912681767835) bank72907 +72908 POINT(39.810993927180185 74.05740973071583) bank72908 +72909 POINT(41.1599396678189 74.82492561885029) bank72909 +72910 POINT(40.57772731327029 73.93607410455282) bank72910 +72911 POINT(41.68577684329295 74.46071012753471) bank72911 +72912 POINT(40.402293307137185 73.48969855120431) bank72912 +72913 POINT(41.28804412238339 73.85326304517339) bank72913 +72914 POINT(40.53461555414803 73.38835115740511) bank72914 +72915 POINT(41.12097314749683 73.40677940653345) bank72915 +72916 POINT(41.33847492586507 73.19578365500588) bank72916 +72917 POINT(40.06576819202021 73.22037219932326) bank72917 +72918 POINT(39.96833782126171 73.11858628600031) bank72918 +72919 POINT(40.84695549087991 74.88049370812) bank72919 +72920 POINT(41.00043096541289 73.91076467685083) bank72920 +72921 POINT(41.4027770621514 73.77692433954576) bank72921 +72922 POINT(39.72345233130869 73.94245402577414) bank72922 +72923 POINT(40.65608993430278 74.91760808036871) bank72923 +72924 POINT(40.22515706584693 74.70180103844493) bank72924 +72925 POINT(41.64653548697147 73.05878858438058) bank72925 +72926 POINT(41.60500040762214 74.63690006521617) bank72926 +72927 POINT(40.962827133356726 74.72166498693986) bank72927 +72928 POINT(41.18152165805206 73.28999270318816) bank72928 +72929 POINT(39.795773317548296 73.31768129619164) bank72929 +72930 POINT(40.083411814329395 73.17784991974585) bank72930 +72931 POINT(41.47320301310312 73.85828808872839) bank72931 +72932 POINT(40.91847048908061 74.99799987366686) bank72932 +72933 POINT(41.505495557182094 73.86482224708247) bank72933 +72934 POINT(41.35645541023207 74.64599239355452) bank72934 +72935 POINT(40.72341275246303 73.90332126231483) bank72935 +72936 POINT(40.174340080074174 74.07124942397778) bank72936 +72937 POINT(40.698034499332145 73.13031808477791) bank72937 +72938 POINT(40.577366875957566 73.80434058507159) bank72938 +72939 POINT(41.3574323646732 73.87618127155231) bank72939 +72940 POINT(40.9001836157112 73.82245931416514) bank72940 +72941 POINT(39.845030866070374 74.46674932116935) bank72941 +72942 POINT(41.07936485756692 74.2468427199754) bank72942 +72943 POINT(40.30522750627203 74.84740009625601) bank72943 +72944 POINT(41.64235485789887 74.79322384569465) bank72944 +72945 POINT(40.806688895077 73.94231330117624) bank72945 +72946 POINT(40.488452165203746 73.09198111259988) bank72946 +72947 POINT(41.021795936561006 74.92785801876664) bank72947 +72948 POINT(40.99357707233516 73.78286004716124) bank72948 +72949 POINT(40.115985843146 74.15306812272085) bank72949 +72950 POINT(40.535607944389724 74.14139675706322) bank72950 +72951 POINT(39.74520813475677 74.02365514258331) bank72951 +72952 POINT(40.161263579753495 74.46223348812181) bank72952 +72953 POINT(40.065853288995626 73.03858057789905) bank72953 +72954 POINT(40.118979436090676 73.10013463633375) bank72954 +72955 POINT(41.42447491406636 74.82014096510318) bank72955 +72956 POINT(40.75651165977964 74.36019548655545) bank72956 +72957 POINT(41.35193137184901 73.63877227075949) bank72957 +72958 POINT(40.88960848805792 73.81168592478647) bank72958 +72959 POINT(40.449760704925815 74.34480861999364) bank72959 +72960 POINT(39.7653100457765 73.2694336403952) bank72960 +72961 POINT(41.367182157951696 73.64793428498626) bank72961 +72962 POINT(40.11381649574314 73.41703291991819) bank72962 +72963 POINT(40.77156853144751 73.32655944166873) bank72963 +72964 POINT(41.65918822716075 73.48356564377777) bank72964 +72965 POINT(40.13239541166389 74.30088488827519) bank72965 +72966 POINT(41.54272616265383 74.03942152086476) bank72966 +72967 POINT(40.146720872868705 74.20166276545389) bank72967 +72968 POINT(40.1947350064021 73.80178948236345) bank72968 +72969 POINT(39.82753614912626 74.11724190471917) bank72969 +72970 POINT(40.03797507651467 74.30752643418218) bank72970 +72971 POINT(40.28279963565974 74.98852565092056) bank72971 +72972 POINT(40.79368591485407 74.53869929298517) bank72972 +72973 POINT(41.60379562434018 73.69163624799472) bank72973 +72974 POINT(41.32591542578011 74.87421581958104) bank72974 +72975 POINT(41.14544682367697 74.6137657361333) bank72975 +72976 POINT(40.733886499513225 74.61542459529568) bank72976 +72977 POINT(40.876601305570574 74.61781260219364) bank72977 +72978 POINT(39.95767358938588 73.2520024285591) bank72978 +72979 POINT(40.68515141010696 73.02301654779166) bank72979 +72980 POINT(39.96805027725251 73.13673157873187) bank72980 +72981 POINT(40.33772503859988 73.93268617248951) bank72981 +72982 POINT(41.397505409597905 73.11591710253578) bank72982 +72983 POINT(40.20326545995913 73.32499806303917) bank72983 +72984 POINT(40.79389221337615 74.72227431844196) bank72984 +72985 POINT(40.438501063490975 73.5618003123341) bank72985 +72986 POINT(40.81932556290565 73.53785141711272) bank72986 +72987 POINT(40.699352583483844 73.6892838724613) bank72987 +72988 POINT(41.44557212906894 73.98736422522437) bank72988 +72989 POINT(40.43457578115326 73.36904224943676) bank72989 +72990 POINT(41.28642007316878 73.99767737233503) bank72990 +72991 POINT(41.38603013714897 74.04919772713684) bank72991 +72992 POINT(41.05225922262188 73.20884078528229) bank72992 +72993 POINT(39.92359938747834 74.91505157092551) bank72993 +72994 POINT(40.238478288509086 74.64896022064086) bank72994 +72995 POINT(40.13102720779951 73.05119735709579) bank72995 +72996 POINT(40.31103221986181 74.4756055221531) bank72996 +72997 POINT(39.853899187118614 73.11226533372614) bank72997 +72998 POINT(39.919541374722684 74.32363245873358) bank72998 +72999 POINT(40.65412864361897 73.51342481982002) bank72999 +73000 POINT(41.488235082960195 74.33249686113103) bank73000 +73001 POINT(41.55324979425904 73.5302730781348) bank73001 +73002 POINT(41.336421792454956 74.73057512795862) bank73002 +73003 POINT(41.051708200909644 73.74118698830104) bank73003 +73004 POINT(40.501075166711544 74.78047090721795) bank73004 +73005 POINT(40.919713855735075 73.09277004455248) bank73005 +73006 POINT(41.680792192070086 73.51518921017824) bank73006 +73007 POINT(40.7001079314226 74.92099525696156) bank73007 +73008 POINT(40.62172452160376 73.13812350147849) bank73008 +73009 POINT(40.428539475357546 74.7684280507244) bank73009 +73010 POINT(40.54880618360315 73.25037768377072) bank73010 +73011 POINT(41.18161682769792 73.8102935718819) bank73011 +73012 POINT(40.25871987796133 74.30781144619039) bank73012 +73013 POINT(40.976222913485344 74.50584413556855) bank73013 +73014 POINT(40.25086902451017 74.46660004207259) bank73014 +73015 POINT(41.46524712970463 74.6578878391258) bank73015 +73016 POINT(41.620089702059985 74.74910393436674) bank73016 +73017 POINT(41.643485162653874 73.775444506516) bank73017 +73018 POINT(41.09237301495265 73.10690517982364) bank73018 +73019 POINT(40.60731582382977 73.54121431959909) bank73019 +73020 POINT(41.41011338313113 74.06128541184688) bank73020 +73021 POINT(41.2557745459868 74.69407263696102) bank73021 +73022 POINT(41.48811264027836 73.44931956620675) bank73022 +73023 POINT(41.48082962451169 73.11882231878721) bank73023 +73024 POINT(40.715387933420764 73.9408981182591) bank73024 +73025 POINT(41.5638793083469 74.11199957664269) bank73025 +73026 POINT(41.08157283822394 73.51768994703184) bank73026 +73027 POINT(41.32955779497794 74.89094538752106) bank73027 +73028 POINT(41.232067784968386 74.52205855158124) bank73028 +73029 POINT(41.532261752685976 73.89686582322908) bank73029 +73030 POINT(40.44908360168118 73.06290402859189) bank73030 +73031 POINT(40.64496530697402 74.95530800547212) bank73031 +73032 POINT(39.89537782603676 73.38659946772466) bank73032 +73033 POINT(40.62984156702713 74.85609980381953) bank73033 +73034 POINT(39.96829333359191 73.52094217725742) bank73034 +73035 POINT(40.069305073920425 73.87775012297267) bank73035 +73036 POINT(40.6766949850491 74.49952647235267) bank73036 +73037 POINT(40.32781901352792 73.86777965373449) bank73037 +73038 POINT(40.906672961637256 73.0723490773637) bank73038 +73039 POINT(40.1819813049362 74.10726604100627) bank73039 +73040 POINT(40.86722298676257 74.63752320332077) bank73040 +73041 POINT(41.35315523253762 73.31553729386505) bank73041 +73042 POINT(40.67300369256796 73.28643380541168) bank73042 +73043 POINT(40.29650645508631 73.36121510043617) bank73043 +73044 POINT(41.65562666422987 73.28386328566167) bank73044 +73045 POINT(40.96045826681421 73.48569293382583) bank73045 +73046 POINT(41.074425698811375 73.2584000946176) bank73046 +73047 POINT(41.08057474113724 73.05997950090226) bank73047 +73048 POINT(41.46769141366532 73.81263047655972) bank73048 +73049 POINT(39.728946084229854 74.38221984047847) bank73049 +73050 POINT(41.22555289275197 73.10298622794029) bank73050 +73051 POINT(40.19418589042568 74.46746391720733) bank73051 +73052 POINT(41.011445132173996 73.52747954622332) bank73052 +73053 POINT(40.628144155696475 74.38534665455735) bank73053 +73054 POINT(40.74251112134191 73.9666362195454) bank73054 +73055 POINT(40.98676423770353 73.01694075379389) bank73055 +73056 POINT(41.60684331080274 74.00166964529731) bank73056 +73057 POINT(40.01615871486262 73.11250446010642) bank73057 +73058 POINT(41.24609196269749 73.5096768348061) bank73058 +73059 POINT(40.22322110577071 73.3985721175085) bank73059 +73060 POINT(39.79771765161837 74.11413641762685) bank73060 +73061 POINT(40.99969064034878 74.35100776785336) bank73061 +73062 POINT(41.54730654972207 74.27806579441175) bank73062 +73063 POINT(40.62976199115979 73.14665638061504) bank73063 +73064 POINT(40.783308296645416 73.3497027175525) bank73064 +73065 POINT(40.90599960466682 73.46739330082424) bank73065 +73066 POINT(40.180826443668245 74.9723585672067) bank73066 +73067 POINT(39.74976521217838 73.98409488027275) bank73067 +73068 POINT(40.66581993023094 74.03758968013376) bank73068 +73069 POINT(40.31065738835665 74.73565141567994) bank73069 +73070 POINT(40.08969462442925 73.92008105412748) bank73070 +73071 POINT(41.4766694316856 74.43265379428341) bank73071 +73072 POINT(40.04720480319191 73.04250455475511) bank73072 +73073 POINT(40.690643337919084 73.61945417155094) bank73073 +73074 POINT(40.69859877675432 74.25682149747563) bank73074 +73075 POINT(40.56644468556457 74.11259192428962) bank73075 +73076 POINT(40.790372690474946 73.69670886038676) bank73076 +73077 POINT(41.55651261436006 74.17959221586467) bank73077 +73078 POINT(40.45534321222657 73.98526348097373) bank73078 +73079 POINT(39.81732978990885 74.40539836779477) bank73079 +73080 POINT(40.547213857970284 73.75758904655126) bank73080 +73081 POINT(41.64734931543463 73.82014756719607) bank73081 +73082 POINT(40.9985632259962 74.35804598454513) bank73082 +73083 POINT(41.19162781937029 74.2724155530302) bank73083 +73084 POINT(40.68828327468509 74.46751206373683) bank73084 +73085 POINT(40.488391519271566 74.77633383799082) bank73085 +73086 POINT(40.21626172268418 73.77994851322177) bank73086 +73087 POINT(40.40180453439166 73.388177076712) bank73087 +73088 POINT(40.31851393389494 73.37139844944139) bank73088 +73089 POINT(39.91445739601058 73.10716007407805) bank73089 +73090 POINT(40.5466787789185 74.83931684950058) bank73090 +73091 POINT(41.319323110365985 73.51952855598152) bank73091 +73092 POINT(40.854910381742194 74.72393570720354) bank73092 +73093 POINT(40.870409532510614 74.4788548798154) bank73093 +73094 POINT(39.9785177362227 73.87005665778885) bank73094 +73095 POINT(40.506558580088424 73.92708467844062) bank73095 +73096 POINT(39.802289387992225 74.88558733682024) bank73096 +73097 POINT(40.759524341893545 73.97169217454463) bank73097 +73098 POINT(41.42709652743491 73.61389646793279) bank73098 +73099 POINT(41.57991892354931 73.36630990591217) bank73099 +73100 POINT(40.73331584001497 74.2583661374055) bank73100 +73101 POINT(40.742727724391564 73.88629105374615) bank73101 +73102 POINT(40.613007772890164 73.09278574505066) bank73102 +73103 POINT(40.036050591946605 74.55744539993755) bank73103 +73104 POINT(41.2602220212918 73.4404899006594) bank73104 +73105 POINT(41.07615795324955 73.00658919293286) bank73105 +73106 POINT(40.73729090542571 74.38000092033248) bank73106 +73107 POINT(40.700664438655025 73.34984102656443) bank73107 +73108 POINT(40.853258300713044 74.08825827292739) bank73108 +73109 POINT(40.69917651178884 73.55195038040745) bank73109 +73110 POINT(41.57049826951543 74.78194824452994) bank73110 +73111 POINT(41.175088394251695 74.77050570538266) bank73111 +73112 POINT(41.58059552810904 73.21604528371834) bank73112 +73113 POINT(39.834908342374455 74.28411828263484) bank73113 +73114 POINT(40.5399663725537 73.61523370280399) bank73114 +73115 POINT(41.454295177345415 73.88648481986506) bank73115 +73116 POINT(40.29230146387576 74.25043938023812) bank73116 +73117 POINT(39.831138565449606 74.39366187112887) bank73117 +73118 POINT(40.98926595769582 74.07155336613435) bank73118 +73119 POINT(40.6704458621183 74.97575256921206) bank73119 +73120 POINT(40.60576958470487 73.92927700187363) bank73120 +73121 POINT(41.691674373772685 74.50091678657728) bank73121 +73122 POINT(40.44283690747624 74.34928587965199) bank73122 +73123 POINT(40.011636464642166 74.44257263308901) bank73123 +73124 POINT(40.140481441395245 74.80846078860907) bank73124 +73125 POINT(39.9727597029921 74.37602341794275) bank73125 +73126 POINT(40.55238346455439 74.54880067622409) bank73126 +73127 POINT(40.049557402232104 74.94537152560588) bank73127 +73128 POINT(40.401032478694106 74.6064858703353) bank73128 +73129 POINT(40.180580319069 73.18126887097826) bank73129 +73130 POINT(41.6067850405466 73.69230749309986) bank73130 +73131 POINT(39.77121889157139 73.30983694758535) bank73131 +73132 POINT(40.05171564029483 74.3685825794138) bank73132 +73133 POINT(41.50809010452196 73.89720990252114) bank73133 +73134 POINT(40.83179989792983 73.80425797613492) bank73134 +73135 POINT(41.560360689382875 73.02203867202975) bank73135 +73136 POINT(40.2748588562335 74.82875448331713) bank73136 +73137 POINT(40.20415728168271 73.3049540599045) bank73137 +73138 POINT(41.36250415240619 73.45622304594583) bank73138 +73139 POINT(39.90196104008731 74.99324846758444) bank73139 +73140 POINT(39.87638031759289 73.05421797068475) bank73140 +73141 POINT(41.31316637052496 74.88012636481082) bank73141 +73142 POINT(41.24317537925055 74.92655944096312) bank73142 +73143 POINT(41.2270921937434 73.58557414763438) bank73143 +73144 POINT(40.374114160522964 73.6738540875387) bank73144 +73145 POINT(40.855821221433054 74.12873731406332) bank73145 +73146 POINT(41.09758566055738 73.05107786879334) bank73146 +73147 POINT(40.133341888870376 74.95306434458955) bank73147 +73148 POINT(40.750231711768116 73.2130277938732) bank73148 +73149 POINT(41.68910702175426 73.06322484037489) bank73149 +73150 POINT(40.465218878718474 73.06742241564959) bank73150 +73151 POINT(41.33240258641995 73.07281220990858) bank73151 +73152 POINT(41.113213067401446 74.22447958979245) bank73152 +73153 POINT(41.33299637454157 74.08115988150233) bank73153 +73154 POINT(41.16714531698996 74.46340396089411) bank73154 +73155 POINT(40.60594290757617 73.63591685628403) bank73155 +73156 POINT(41.518745067103666 73.69250391232836) bank73156 +73157 POINT(39.73363018686638 74.0482538897739) bank73157 +73158 POINT(40.50208029120641 74.93201647276757) bank73158 +73159 POINT(40.13952574892989 74.76666409721503) bank73159 +73160 POINT(40.823519775123216 73.25251663115156) bank73160 +73161 POINT(39.9828586915484 74.61192158528829) bank73161 +73162 POINT(40.29760303776586 74.0775775487585) bank73162 +73163 POINT(41.51593566874285 73.4278931839331) bank73163 +73164 POINT(39.987336819949995 74.395258891896) bank73164 +73165 POINT(41.13639930198312 73.67361522332044) bank73165 +73166 POINT(41.07051860769778 74.96866672351013) bank73166 +73167 POINT(41.08462009019327 73.15514858238237) bank73167 +73168 POINT(40.29501772845247 73.9192068767811) bank73168 +73169 POINT(41.01704691974939 73.19369883729698) bank73169 +73170 POINT(40.39438407579094 73.6239872796944) bank73170 +73171 POINT(41.1993686644945 74.75738215332446) bank73171 +73172 POINT(41.68664233600847 74.6812551396161) bank73172 +73173 POINT(40.84164684417932 73.18653438765381) bank73173 +73174 POINT(41.63437112239698 74.06507645694113) bank73174 +73175 POINT(39.96435123379114 74.5050492787845) bank73175 +73176 POINT(40.59745960312353 74.10532220822776) bank73176 +73177 POINT(40.082722600102834 74.28924676663443) bank73177 +73178 POINT(40.370480227303915 73.25158139779062) bank73178 +73179 POINT(40.94274220744428 73.21665057760796) bank73179 +73180 POINT(40.16670085125723 74.71559036194672) bank73180 +73181 POINT(39.74314810713327 73.475271000675) bank73181 +73182 POINT(40.13755095955403 73.54814747006009) bank73182 +73183 POINT(41.42900616980454 73.14592906456268) bank73183 +73184 POINT(41.0457811629945 73.41799901854257) bank73184 +73185 POINT(40.49443008796918 74.24318720013945) bank73185 +73186 POINT(40.587856374494464 73.40161161053382) bank73186 +73187 POINT(40.08087912055557 74.12572237875581) bank73187 +73188 POINT(41.678947345840655 74.71495709600421) bank73188 +73189 POINT(41.45882318355859 73.53232331662029) bank73189 +73190 POINT(40.96899552293733 73.07151029937705) bank73190 +73191 POINT(39.95137179918307 73.724125817833) bank73191 +73192 POINT(39.861899878275366 73.93143306807005) bank73192 +73193 POINT(41.34372540800075 74.00748870866799) bank73193 +73194 POINT(39.91915524829433 73.07627987172276) bank73194 +73195 POINT(40.18015081890417 73.88033275775493) bank73195 +73196 POINT(40.59393787716407 73.78430031467651) bank73196 +73197 POINT(39.81738746315052 73.20886955113691) bank73197 +73198 POINT(41.03924564948904 73.95523363297491) bank73198 +73199 POINT(40.97199518350777 73.05837509097317) bank73199 +73200 POINT(41.1538671706339 74.45732688433111) bank73200 +73201 POINT(40.75107744273046 73.66475081690072) bank73201 +73202 POINT(39.92227484431831 73.32946715230408) bank73202 +73203 POINT(41.078453316304376 73.33367053171634) bank73203 +73204 POINT(41.42785950967909 73.66040394290872) bank73204 +73205 POINT(40.04071685405783 73.46315218241526) bank73205 +73206 POINT(39.79204855351572 73.32618667609002) bank73206 +73207 POINT(40.51238964129524 74.51145167189708) bank73207 +73208 POINT(40.43405725484784 73.83026183568522) bank73208 +73209 POINT(40.377475182118765 73.40136011352818) bank73209 +73210 POINT(40.55387171877355 74.6044614604637) bank73210 +73211 POINT(41.29549975236222 74.74164973270769) bank73211 +73212 POINT(41.2392981494053 74.21877719910606) bank73212 +73213 POINT(41.45601934742263 74.8073236539523) bank73213 +73214 POINT(40.896883556488945 73.43781059022695) bank73214 +73215 POINT(41.49479775375263 74.1772311516636) bank73215 +73216 POINT(41.3316135300992 73.8140026915772) bank73216 +73217 POINT(41.4637867318227 73.30934471435505) bank73217 +73218 POINT(41.44877623775072 74.93868585006129) bank73218 +73219 POINT(40.74068352890654 73.15546319616932) bank73219 +73220 POINT(40.83179313499587 73.11358133271276) bank73220 +73221 POINT(39.71785131699261 73.87805834271312) bank73221 +73222 POINT(39.84511538374174 73.00834961574675) bank73222 +73223 POINT(40.43072872164652 73.31411942215644) bank73223 +73224 POINT(40.35972044481426 73.81539633183509) bank73224 +73225 POINT(39.77358016857858 74.22395344454239) bank73225 +73226 POINT(40.173406161245744 74.30793465499173) bank73226 +73227 POINT(40.88221332079441 74.7248689954535) bank73227 +73228 POINT(41.2561051025402 74.80433317178708) bank73228 +73229 POINT(41.20211864440018 74.53143365419012) bank73229 +73230 POINT(40.13533821792114 74.86163890821588) bank73230 +73231 POINT(41.183064994486244 73.37269220101884) bank73231 +73232 POINT(39.96609451949097 74.19257737049809) bank73232 +73233 POINT(40.39491576647274 74.31094435054233) bank73233 +73234 POINT(40.222301121414795 73.12018471794502) bank73234 +73235 POINT(40.25760516896713 74.53047051122967) bank73235 +73236 POINT(40.95383750425886 74.33803919427073) bank73236 +73237 POINT(40.573670705006165 73.72150738195346) bank73237 +73238 POINT(41.22415539450341 73.71056390395832) bank73238 +73239 POINT(40.162542724990175 74.32943510176281) bank73239 +73240 POINT(40.648816560257906 73.63416278956464) bank73240 +73241 POINT(40.33327521593367 73.91585466659944) bank73241 +73242 POINT(40.717688304756294 73.37965814554245) bank73242 +73243 POINT(41.49167711649997 74.73127503965338) bank73243 +73244 POINT(40.466945222382215 73.55038351189867) bank73244 +73245 POINT(41.59944769364601 73.07545979994752) bank73245 +73246 POINT(40.819565889866084 74.96395301636193) bank73246 +73247 POINT(41.021449719538474 74.15592721665568) bank73247 +73248 POINT(41.11755203445831 73.35793281794598) bank73248 +73249 POINT(40.28994723382543 74.77850595268833) bank73249 +73250 POINT(39.829178166371136 74.18538155792508) bank73250 +73251 POINT(40.12098769146372 73.6585866496692) bank73251 +73252 POINT(41.05842427755335 73.23215922592605) bank73252 +73253 POINT(40.18180685942391 73.31562512150461) bank73253 +73254 POINT(40.50987163048246 73.45418955718407) bank73254 +73255 POINT(40.19299586119302 74.53115123758502) bank73255 +73256 POINT(40.80459160215722 73.66006887525437) bank73256 +73257 POINT(39.727606579185114 73.28774009766472) bank73257 +73258 POINT(41.48794759698519 73.66403467760838) bank73258 +73259 POINT(41.20254039121564 74.18650614751286) bank73259 +73260 POINT(41.227187999224796 74.63723323921545) bank73260 +73261 POINT(40.72953834509894 73.66888139937576) bank73261 +73262 POINT(40.75216738901339 74.44289003498157) bank73262 +73263 POINT(41.30871399190717 73.40624946198632) bank73263 +73264 POINT(41.447966288199886 73.50896710820635) bank73264 +73265 POINT(41.04903685362853 73.69684015526775) bank73265 +73266 POINT(39.84769203375805 74.91580910188767) bank73266 +73267 POINT(40.13134169480413 74.77060917299129) bank73267 +73268 POINT(41.1781527541636 73.85287866916856) bank73268 +73269 POINT(40.933275762141086 74.84633381608253) bank73269 +73270 POINT(40.56016502133323 74.40568284288187) bank73270 +73271 POINT(40.97642871719076 73.97276914058234) bank73271 +73272 POINT(40.32738883548556 73.91099480029426) bank73272 +73273 POINT(39.98715877848063 73.14104199430601) bank73273 +73274 POINT(40.674697949231884 73.51858017957481) bank73274 +73275 POINT(40.885113437363486 74.32758482680423) bank73275 +73276 POINT(40.88298411895579 73.30353317687623) bank73276 +73277 POINT(40.841516833273964 74.81836971303082) bank73277 +73278 POINT(41.0608216638866 73.22741374728325) bank73278 +73279 POINT(41.31647764265052 73.52384697317568) bank73279 +73280 POINT(40.39612022109199 74.23516226369301) bank73280 +73281 POINT(41.40323557195644 73.07812318174422) bank73281 +73282 POINT(40.76807817957194 74.98199416380402) bank73282 +73283 POINT(41.43349394335243 73.5718001482178) bank73283 +73284 POINT(41.14562629757145 74.9931424946719) bank73284 +73285 POINT(41.24321663026728 73.29660683690051) bank73285 +73286 POINT(40.49571502687159 73.89007425323531) bank73286 +73287 POINT(39.92682842469198 74.01277363852589) bank73287 +73288 POINT(40.902777898707015 73.05856876002487) bank73288 +73289 POINT(41.306707481185946 74.86091128396238) bank73289 +73290 POINT(40.17343872292647 74.45361502263454) bank73290 +73291 POINT(40.56524669203563 74.7485357826151) bank73291 +73292 POINT(39.74891767292774 73.34986223904033) bank73292 +73293 POINT(40.039151603172996 74.43874976683495) bank73293 +73294 POINT(41.48062209829844 74.19487069354248) bank73294 +73295 POINT(40.4770745831966 74.9584930292355) bank73295 +73296 POINT(39.918731638643635 74.68252496340986) bank73296 +73297 POINT(40.587654427176815 74.86031788343537) bank73297 +73298 POINT(41.11685885780072 73.52237072210323) bank73298 +73299 POINT(41.468736689190635 74.13696583823044) bank73299 +73300 POINT(40.29000923826415 74.29232368192991) bank73300 +73301 POINT(39.731117505861775 73.03122480937091) bank73301 +73302 POINT(41.52763464086413 74.89208459938305) bank73302 +73303 POINT(41.47347360735965 74.65041206822278) bank73303 +73304 POINT(40.18448009514315 73.01639349876814) bank73304 +73305 POINT(41.68616937386514 74.39744902486603) bank73305 +73306 POINT(40.9140127501192 73.69371619925882) bank73306 +73307 POINT(41.70017643033084 74.98095323885437) bank73307 +73308 POINT(40.233052621087154 73.76694420092942) bank73308 +73309 POINT(41.125554052778064 74.24129605921966) bank73309 +73310 POINT(41.68826067822028 74.08507702133676) bank73310 +73311 POINT(40.88219084804943 73.72529325776712) bank73311 +73312 POINT(40.75100690791465 74.68715706274068) bank73312 +73313 POINT(40.028235164519344 74.64401581717098) bank73313 +73314 POINT(41.08587575425209 73.91417016067436) bank73314 +73315 POINT(41.4150986257607 74.04380184463187) bank73315 +73316 POINT(41.70066513663691 74.7466614941458) bank73316 +73317 POINT(40.98358240112002 73.76389307257553) bank73317 +73318 POINT(41.29561666328223 74.36485173986235) bank73318 +73319 POINT(40.39450296120234 73.76202425675818) bank73319 +73320 POINT(41.32477087162937 73.61485563951707) bank73320 +73321 POINT(41.10079767577718 74.42379447143722) bank73321 +73322 POINT(40.189609858828945 73.30322040564168) bank73322 +73323 POINT(40.271565719481245 74.00947545690084) bank73323 +73324 POINT(40.67513282382102 74.56423659931369) bank73324 +73325 POINT(40.58560602559001 73.48801645239476) bank73325 +73326 POINT(41.68869695249554 73.93316037507519) bank73326 +73327 POINT(41.366114926669376 73.20407279364505) bank73327 +73328 POINT(40.655740198038465 74.29826436736457) bank73328 +73329 POINT(41.05578663860083 73.6666770299469) bank73329 +73330 POINT(41.59087469553604 74.4121740461784) bank73330 +73331 POINT(41.60883956732302 74.48423555074974) bank73331 +73332 POINT(40.99439423392749 74.76954893521662) bank73332 +73333 POINT(41.2974581546042 73.00791026522441) bank73333 +73334 POINT(40.50709776141376 74.06651900202023) bank73334 +73335 POINT(40.84901701229623 73.52104772629163) bank73335 +73336 POINT(40.51019392338944 74.06277784673372) bank73336 +73337 POINT(40.05606137716132 73.70935344936709) bank73337 +73338 POINT(40.988792229984846 74.36019272910077) bank73338 +73339 POINT(40.93229371550199 73.08502507605085) bank73339 +73340 POINT(40.135672336578516 74.05670823338872) bank73340 +73341 POINT(41.151111794440276 73.14469633843963) bank73341 +73342 POINT(39.97081437513356 74.72339117180049) bank73342 +73343 POINT(41.35338229824712 74.18846964856719) bank73343 +73344 POINT(41.092989861110006 73.69240735553244) bank73344 +73345 POINT(39.74821689125871 73.66861001673566) bank73345 +73346 POINT(41.302597784424066 73.4495868872945) bank73346 +73347 POINT(39.72994762468124 74.51675796079223) bank73347 +73348 POINT(41.39111521642638 74.47900472830081) bank73348 +73349 POINT(40.95508527472364 73.68597804139591) bank73349 +73350 POINT(40.18655130712383 73.87833799522093) bank73350 +73351 POINT(41.41482010615951 73.941422826398) bank73351 +73352 POINT(41.02758858362771 74.23901800382006) bank73352 +73353 POINT(41.59963064387084 74.93952356006344) bank73353 +73354 POINT(41.513415330847764 73.2945765681466) bank73354 +73355 POINT(41.04835031918335 74.88848575416958) bank73355 +73356 POINT(41.266271698363184 73.41993653193306) bank73356 +73357 POINT(40.659273724130315 74.88352867826562) bank73357 +73358 POINT(40.45820645997341 73.80101537352625) bank73358 +73359 POINT(40.12463284751056 73.55021889191553) bank73359 +73360 POINT(40.97352102102671 73.23710947400845) bank73360 +73361 POINT(39.93600759439467 73.3976623090105) bank73361 +73362 POINT(40.02304837837251 73.0342717322551) bank73362 +73363 POINT(39.751803499077404 74.41342561378438) bank73363 +73364 POINT(40.933946929141136 74.28450907755678) bank73364 +73365 POINT(40.82435042030659 73.74814731084156) bank73365 +73366 POINT(40.01316231353931 74.1780202356309) bank73366 +73367 POINT(40.15612550591686 73.41496361992411) bank73367 +73368 POINT(40.05079147200216 73.77710670800107) bank73368 +73369 POINT(40.37647795997582 73.80698811960362) bank73369 +73370 POINT(40.83615192254467 74.19060944509387) bank73370 +73371 POINT(40.08736206904143 74.6326077404652) bank73371 +73372 POINT(40.66206867897809 74.8418370809249) bank73372 +73373 POINT(41.70009254624209 74.15361027065613) bank73373 +73374 POINT(41.67103703394459 73.90012961997297) bank73374 +73375 POINT(40.02952893870175 74.44545115369588) bank73375 +73376 POINT(39.75935131107077 73.37321029170448) bank73376 +73377 POINT(39.98540357053002 74.97417443659695) bank73377 +73378 POINT(40.44734430726563 74.796819096563) bank73378 +73379 POINT(40.19403566630336 74.94217123037036) bank73379 +73380 POINT(41.57928844094805 73.19566303354554) bank73380 +73381 POINT(40.39022482319743 74.79101120457203) bank73381 +73382 POINT(39.856627139090804 73.14759235754308) bank73382 +73383 POINT(40.061839140119545 74.66997066238241) bank73383 +73384 POINT(41.60781355945395 73.2699774950064) bank73384 +73385 POINT(41.686382495270074 74.9744313478171) bank73385 +73386 POINT(40.26366286267778 73.68972869345527) bank73386 +73387 POINT(39.96977097834662 73.19504244711676) bank73387 +73388 POINT(40.516480069163535 74.1337370401411) bank73388 +73389 POINT(41.51455437351347 73.18214028494204) bank73389 +73390 POINT(40.98530829435294 73.79094162475175) bank73390 +73391 POINT(40.41757484419991 73.41326580144352) bank73391 +73392 POINT(40.474320664331735 74.23644694610078) bank73392 +73393 POINT(40.695602358935126 73.59741297860526) bank73393 +73394 POINT(40.38238358675656 74.52482665737195) bank73394 +73395 POINT(40.44366527828057 73.59289173587555) bank73395 +73396 POINT(40.57366156334931 74.97126504911088) bank73396 +73397 POINT(40.94479514380303 74.30716451869655) bank73397 +73398 POINT(41.073498518875766 73.49214528266472) bank73398 +73399 POINT(39.889712402465015 73.29700962540373) bank73399 +73400 POINT(40.92005922981685 73.55741197379466) bank73400 +73401 POINT(40.650307236428304 74.938419221758) bank73401 +73402 POINT(40.4252954415568 73.80357481738176) bank73402 +73403 POINT(40.81276460607433 73.54535475989995) bank73403 +73404 POINT(41.048957716504624 74.50107621743773) bank73404 +73405 POINT(40.42362866305039 73.80881217392958) bank73405 +73406 POINT(40.298546610686024 74.21288362466377) bank73406 +73407 POINT(40.01320711223353 74.93216964036642) bank73407 +73408 POINT(40.69520274158744 73.1253908395061) bank73408 +73409 POINT(40.98357176952809 73.3389619837417) bank73409 +73410 POINT(40.10628880126741 74.12252810682347) bank73410 +73411 POINT(40.949485263074 74.27031697876687) bank73411 +73412 POINT(40.92039311421069 74.30017294655389) bank73412 +73413 POINT(40.83515766125285 73.46467615641704) bank73413 +73414 POINT(40.95494656644598 73.39795739608525) bank73414 +73415 POINT(40.73210529447314 74.51742714226873) bank73415 +73416 POINT(39.941532322983235 73.95694619974721) bank73416 +73417 POINT(41.26063186550342 74.82172323776277) bank73417 +73418 POINT(40.48337237049892 74.6588876424138) bank73418 +73419 POINT(40.75324061036702 74.62439112963065) bank73419 +73420 POINT(41.65163962098187 73.44517275498602) bank73420 +73421 POINT(40.80692993389514 73.30805935022818) bank73421 +73422 POINT(40.72918723361839 73.40355201376747) bank73422 +73423 POINT(39.80073734485024 74.24249955161571) bank73423 +73424 POINT(40.076927902797564 74.37795611947632) bank73424 +73425 POINT(41.464498137740186 73.60285270997353) bank73425 +73426 POINT(40.66232490318437 74.75035529344508) bank73426 +73427 POINT(40.294909937418105 74.81217404292724) bank73427 +73428 POINT(40.64538002202151 73.92162299823056) bank73428 +73429 POINT(40.20874760021727 73.22188837546092) bank73429 +73430 POINT(39.99694384362763 73.8007719788141) bank73430 +73431 POINT(40.424067464495295 73.49667433059004) bank73431 +73432 POINT(40.33086760711673 73.59405059079717) bank73432 +73433 POINT(40.12560761461105 73.22949619986713) bank73433 +73434 POINT(41.07008879956006 74.0851055933341) bank73434 +73435 POINT(41.225812003098 74.56594295592392) bank73435 +73436 POINT(40.68188777519399 74.23395266694688) bank73436 +73437 POINT(41.01664975087073 74.39584290043165) bank73437 +73438 POINT(41.614022773746704 73.48275117829675) bank73438 +73439 POINT(40.09028596229853 74.57184087353932) bank73439 +73440 POINT(41.53586590887326 74.86978564720009) bank73440 +73441 POINT(40.59194998418134 73.28789145781951) bank73441 +73442 POINT(40.58902707099877 74.58245213628304) bank73442 +73443 POINT(40.28879220065664 74.92682789542366) bank73443 +73444 POINT(40.484290082379715 74.84288263548572) bank73444 +73445 POINT(40.51609656490069 74.92928887468526) bank73445 +73446 POINT(41.07379873639259 73.00888596780565) bank73446 +73447 POINT(39.728956923985194 73.32100875788768) bank73447 +73448 POINT(41.457584269243746 73.08122101655262) bank73448 +73449 POINT(40.687716633884264 73.79758732957427) bank73449 +73450 POINT(39.95223992427716 73.85176815275216) bank73450 +73451 POINT(40.64350839816635 73.47963638194081) bank73451 +73452 POINT(40.14549442559269 73.66344116311964) bank73452 +73453 POINT(41.17083891986948 74.77288700473834) bank73453 +73454 POINT(40.00870663325713 73.598376569773) bank73454 +73455 POINT(40.543513712830816 74.12689543095046) bank73455 +73456 POINT(41.58777435030741 74.6827195111039) bank73456 +73457 POINT(40.42970813804447 73.29003704472625) bank73457 +73458 POINT(40.2893454079649 74.35889410501814) bank73458 +73459 POINT(41.58804723602319 74.42119235499108) bank73459 +73460 POINT(40.01919714770741 73.3871827386442) bank73460 +73461 POINT(41.66876638037635 73.16070670832703) bank73461 +73462 POINT(40.7352394575306 73.10658843411537) bank73462 +73463 POINT(41.351538593399944 74.76391675911685) bank73463 +73464 POINT(39.76647628043315 73.12660809740295) bank73464 +73465 POINT(39.96098520950733 74.69079436073955) bank73465 +73466 POINT(40.91138884082308 73.81531672145366) bank73466 +73467 POINT(39.83239259162648 74.14786344065499) bank73467 +73468 POINT(40.31929385284705 74.238314029206) bank73468 +73469 POINT(41.27844075301664 74.80992314751848) bank73469 +73470 POINT(40.95299631263885 73.51508495516948) bank73470 +73471 POINT(41.63380812655499 74.31867674150367) bank73471 +73472 POINT(40.33272451897187 73.16649632970174) bank73472 +73473 POINT(40.54478611012804 74.99369084893212) bank73473 +73474 POINT(41.406962081205265 73.7618301427795) bank73474 +73475 POINT(41.707006738230824 74.00741489527924) bank73475 +73476 POINT(40.631217187343225 73.80777977620087) bank73476 +73477 POINT(40.701971468074056 75.00585294235378) bank73477 +73478 POINT(40.25739377721999 74.3640460663478) bank73478 +73479 POINT(40.691879018505006 73.15217999253082) bank73479 +73480 POINT(40.67785298900909 74.22903627910395) bank73480 +73481 POINT(40.49775149621426 74.5290243325591) bank73481 +73482 POINT(41.25175290805206 74.98031768775142) bank73482 +73483 POINT(40.47226205167125 73.45608660970639) bank73483 +73484 POINT(40.32592413935271 73.27989460721534) bank73484 +73485 POINT(40.670141975479474 74.94998059365501) bank73485 +73486 POINT(41.239205797819025 74.07187711774644) bank73486 +73487 POINT(40.50334698187469 73.73920275205434) bank73487 +73488 POINT(40.558729399527124 74.45032175673413) bank73488 +73489 POINT(41.31945546713509 73.15793992521489) bank73489 +73490 POINT(39.946561331861695 74.72863226382577) bank73490 +73491 POINT(41.34329621362035 74.30225658915687) bank73491 +73492 POINT(41.1779632025164 74.33363980921105) bank73492 +73493 POINT(41.34105672789854 74.2487380109392) bank73493 +73494 POINT(39.941411050070776 75.00256710163713) bank73494 +73495 POINT(39.7634322641403 73.20834625271644) bank73495 +73496 POINT(41.24333495742588 73.25488079945909) bank73496 +73497 POINT(41.244636957266835 74.43942382067524) bank73497 +73498 POINT(40.1611345636129 74.45563447434542) bank73498 +73499 POINT(41.59078555730939 74.52043245513825) bank73499 +73500 POINT(40.049291402284354 73.12094659208734) bank73500 +73501 POINT(39.91471897280621 73.09759643833517) bank73501 +73502 POINT(40.83697166230439 74.32558900708399) bank73502 +73503 POINT(40.73456822713928 73.81167547237035) bank73503 +73504 POINT(41.334813828006745 74.91261414958433) bank73504 +73505 POINT(40.196053471840465 74.52257195192561) bank73505 +73506 POINT(40.80645023911461 74.82197457639771) bank73506 +73507 POINT(40.651426417427004 74.37608179505571) bank73507 +73508 POINT(41.20136876165126 73.61074136223692) bank73508 +73509 POINT(41.61326245819188 74.9671494035091) bank73509 +73510 POINT(41.15989100140572 73.30361177066672) bank73510 +73511 POINT(40.8616677062541 73.80418808197823) bank73511 +73512 POINT(41.36412813383537 74.9630294693348) bank73512 +73513 POINT(39.74942603920904 74.14147326749024) bank73513 +73514 POINT(40.08989465411679 73.20510691126643) bank73514 +73515 POINT(39.91781689770783 73.1809220963208) bank73515 +73516 POINT(41.3686529598779 73.29436823493296) bank73516 +73517 POINT(39.762406128255826 73.4927296646358) bank73517 +73518 POINT(39.89719410849135 73.26635436124644) bank73518 +73519 POINT(40.88865755096993 74.23231971931808) bank73519 +73520 POINT(40.45697334036548 73.1936421296136) bank73520 +73521 POINT(41.29321214317932 73.44338898570369) bank73521 +73522 POINT(39.80950780958466 74.98055697708718) bank73522 +73523 POINT(40.16668133824419 74.08651762353945) bank73523 +73524 POINT(41.04326529527019 73.30462076855272) bank73524 +73525 POINT(40.2306372055495 73.03574130830204) bank73525 +73526 POINT(40.542250246127665 73.48111777858139) bank73526 +73527 POINT(40.392733153319796 73.91051311054977) bank73527 +73528 POINT(41.02635944165667 73.67423212280744) bank73528 +73529 POINT(41.23259319565627 73.6254095822931) bank73529 +73530 POINT(40.56731095472128 73.4209741167212) bank73530 +73531 POINT(40.3706845382359 74.57597324960611) bank73531 +73532 POINT(39.96022513906022 74.72819141346571) bank73532 +73533 POINT(41.32212996009535 73.56871051141187) bank73533 +73534 POINT(40.20184832483161 73.30466045577487) bank73534 +73535 POINT(41.06183335184291 73.20361727225779) bank73535 +73536 POINT(40.564634225503745 74.87917918775581) bank73536 +73537 POINT(40.431668217513085 74.27029750573026) bank73537 +73538 POINT(40.57645650523618 73.55763171814743) bank73538 +73539 POINT(40.422840274605 73.42490957845328) bank73539 +73540 POINT(41.632186110738914 73.28472268558944) bank73540 +73541 POINT(41.508674226737384 74.00815156818533) bank73541 +73542 POINT(40.796378902681326 74.27057951943682) bank73542 +73543 POINT(40.55610013249729 74.59284063650891) bank73543 +73544 POINT(40.37644222411195 73.07852369231802) bank73544 +73545 POINT(41.01404743173278 73.68902298007202) bank73545 +73546 POINT(41.16995310842717 73.39321409208202) bank73546 +73547 POINT(40.78285330437015 73.55499795446114) bank73547 +73548 POINT(41.60096819095742 74.90927893326128) bank73548 +73549 POINT(41.357445731889754 74.01947168071162) bank73549 +73550 POINT(41.412998792063306 74.918926425164) bank73550 +73551 POINT(40.36176788046985 74.34470959709502) bank73551 +73552 POINT(41.47120619779626 74.10762307168974) bank73552 +73553 POINT(40.32462715255381 73.03088185887252) bank73553 +73554 POINT(40.74589736222395 73.22128590697426) bank73554 +73555 POINT(40.38440013481355 73.51855236884765) bank73555 +73556 POINT(39.72024056352 74.4004707839948) bank73556 +73557 POINT(39.926134281220946 74.50812730147726) bank73557 +73558 POINT(40.79983279337363 73.83950528000207) bank73558 +73559 POINT(41.508711743182495 73.81439321207544) bank73559 +73560 POINT(40.25481103959908 73.56217578884231) bank73560 +73561 POINT(40.15253448944508 73.14023842890258) bank73561 +73562 POINT(40.16033684446209 74.33694728746624) bank73562 +73563 POINT(41.19940228991954 73.86908202397338) bank73563 +73564 POINT(40.00543313379043 74.51672754307874) bank73564 +73565 POINT(41.37435485984672 73.45367198469863) bank73565 +73566 POINT(41.52299228846424 73.947341812542) bank73566 +73567 POINT(40.75018475367229 74.66725846732182) bank73567 +73568 POINT(41.257107858194885 73.26412108891267) bank73568 +73569 POINT(41.41912945974242 73.19107647913856) bank73569 +73570 POINT(41.66136067065276 73.9360420370337) bank73570 +73571 POINT(40.666061672862455 74.17836348893306) bank73571 +73572 POINT(39.71634193482748 73.43187127933695) bank73572 +73573 POINT(40.37798833471817 73.76267453803338) bank73573 +73574 POINT(39.85907228301699 74.82904202317638) bank73574 +73575 POINT(40.11389697447639 74.56653699854961) bank73575 +73576 POINT(40.81929835222342 74.14736553266764) bank73576 +73577 POINT(40.375185202470774 74.3736447987113) bank73577 +73578 POINT(40.459946341428804 74.47173707499071) bank73578 +73579 POINT(40.56729248696908 73.55121438730322) bank73579 +73580 POINT(40.17956823117903 74.45142786471199) bank73580 +73581 POINT(40.21940624094187 73.79714601336494) bank73581 +73582 POINT(39.75837622104145 73.99228717443673) bank73582 +73583 POINT(41.64124239672497 74.56791841630316) bank73583 +73584 POINT(41.276640670889876 74.40704275315785) bank73584 +73585 POINT(40.165944206607534 73.08047558049557) bank73585 +73586 POINT(39.789640247605305 74.09755691719268) bank73586 +73587 POINT(39.875900495553466 74.02846498709759) bank73587 +73588 POINT(40.10579707007621 73.89045032748793) bank73588 +73589 POINT(40.226230778815655 73.01956027320352) bank73589 +73590 POINT(39.86914540995607 74.0432640999385) bank73590 +73591 POINT(41.04337580542885 73.6873859699017) bank73591 +73592 POINT(40.075779598794995 74.15869715920319) bank73592 +73593 POINT(39.74283083685944 74.18653158573305) bank73593 +73594 POINT(40.08285043860399 74.53626751468761) bank73594 +73595 POINT(40.1760744014939 74.93714711528209) bank73595 +73596 POINT(40.132634883885046 73.20920097730666) bank73596 +73597 POINT(40.46744048670936 74.04655945239625) bank73597 +73598 POINT(39.75178498981621 73.39653878946821) bank73598 +73599 POINT(40.72844504480145 74.37088430451146) bank73599 +73600 POINT(41.21377096371091 74.11082306605948) bank73600 +73601 POINT(40.31961582506979 74.35837403375746) bank73601 +73602 POINT(39.790212134950345 73.99185936510851) bank73602 +73603 POINT(39.92588977707416 74.78779059417204) bank73603 +73604 POINT(39.8462677064753 73.83976557888104) bank73604 +73605 POINT(40.405664242252904 73.54548072016127) bank73605 +73606 POINT(39.96793062048882 73.7927888401782) bank73606 +73607 POINT(40.59529088381026 73.8410694722935) bank73607 +73608 POINT(40.070563255063036 73.32028758070211) bank73608 +73609 POINT(40.952807675974405 74.26176185196809) bank73609 +73610 POINT(40.01786315317962 73.95583318932204) bank73610 +73611 POINT(41.67421575191973 74.56339923427515) bank73611 +73612 POINT(41.5087919490187 74.57691844048539) bank73612 +73613 POINT(40.296041662581146 73.10854200925804) bank73613 +73614 POINT(41.711050783182095 74.328552409141) bank73614 +73615 POINT(41.410608372327225 74.68498874707669) bank73615 +73616 POINT(40.47809053617645 74.54017329963085) bank73616 +73617 POINT(39.928735485321965 74.72850797772676) bank73617 +73618 POINT(40.708530340258655 73.87821241770325) bank73618 +73619 POINT(40.43813847503153 73.9618484349087) bank73619 +73620 POINT(41.67768773845729 73.52665754251633) bank73620 +73621 POINT(40.05843076292097 73.33895373025629) bank73621 +73622 POINT(40.45412220327687 73.4396355133428) bank73622 +73623 POINT(40.72604639171124 74.07397486051639) bank73623 +73624 POINT(41.630739768125295 73.6019164118573) bank73624 +73625 POINT(40.25798750890069 74.56930990180273) bank73625 +73626 POINT(40.29313550840661 74.9896719329132) bank73626 +73627 POINT(39.9144488487319 74.65509625079662) bank73627 +73628 POINT(41.14970013389113 73.91686167654703) bank73628 +73629 POINT(39.77017880579484 74.23467595450671) bank73629 +73630 POINT(41.248522243826685 74.91691283678604) bank73630 +73631 POINT(41.546624599630775 73.83653401755859) bank73631 +73632 POINT(40.97727655210547 74.96740019961923) bank73632 +73633 POINT(40.58841022938956 73.5700586558519) bank73633 +73634 POINT(41.69114851358191 74.17756583860132) bank73634 +73635 POINT(41.5604744675292 74.77183777695171) bank73635 +73636 POINT(40.9616192209709 73.89830653039795) bank73636 +73637 POINT(40.2392886201667 74.34358185305484) bank73637 +73638 POINT(39.75311321097055 73.42123025317362) bank73638 +73639 POINT(39.75035225153839 73.24787943189101) bank73639 +73640 POINT(39.85616450224744 74.9447457143102) bank73640 +73641 POINT(40.288837430885884 74.18500309429106) bank73641 +73642 POINT(40.31281704922335 73.07658544882484) bank73642 +73643 POINT(41.041743224108494 74.18004702393348) bank73643 +73644 POINT(41.23067769615192 74.36651037242585) bank73644 +73645 POINT(40.418731718288825 74.76674850081332) bank73645 +73646 POINT(40.60374947992733 74.4979708330375) bank73646 +73647 POINT(41.63144217970657 73.8985363977819) bank73647 +73648 POINT(40.73398656519296 74.07630958328524) bank73648 +73649 POINT(40.98518969251336 74.2421601785749) bank73649 +73650 POINT(40.19703574084173 74.52141610136681) bank73650 +73651 POINT(40.11666757803057 74.64095652548211) bank73651 +73652 POINT(41.29217690878553 74.21068481949052) bank73652 +73653 POINT(40.88561648752286 74.3121614968475) bank73653 +73654 POINT(40.063994604523586 73.56615574009952) bank73654 +73655 POINT(40.31823021041065 73.59133649201574) bank73655 +73656 POINT(40.38634011764269 73.04325320958672) bank73656 +73657 POINT(41.52614181298678 74.51640170589353) bank73657 +73658 POINT(41.3616300364605 74.64585840900463) bank73658 +73659 POINT(41.20692097282641 74.64945226797082) bank73659 +73660 POINT(41.33260706833465 73.68096309086114) bank73660 +73661 POINT(40.83092188656717 74.8934688004617) bank73661 +73662 POINT(39.957654921239204 73.75920516667986) bank73662 +73663 POINT(41.11825697722996 73.17017976794241) bank73663 +73664 POINT(39.921151601065986 73.20780550722016) bank73664 +73665 POINT(40.25543461886633 73.5199915538696) bank73665 +73666 POINT(41.64567349873725 74.43917796507209) bank73666 +73667 POINT(40.584927344955254 73.2751723716068) bank73667 +73668 POINT(41.15413682234417 73.51892253481668) bank73668 +73669 POINT(41.40862696418487 74.66790992928358) bank73669 +73670 POINT(40.23570731231722 73.0159165767675) bank73670 +73671 POINT(40.56218778298966 73.65640111883471) bank73671 +73672 POINT(40.038803065711775 73.12179950049786) bank73672 +73673 POINT(41.33643844753779 74.9857643661561) bank73673 +73674 POINT(41.43280534341926 73.0654457730731) bank73674 +73675 POINT(40.72505821614902 74.21541070562145) bank73675 +73676 POINT(41.29400656093131 73.49460448140488) bank73676 +73677 POINT(41.2666709955211 73.56498485659394) bank73677 +73678 POINT(39.85506883120692 73.50973210655134) bank73678 +73679 POINT(41.2928033063819 74.38811521393535) bank73679 +73680 POINT(39.812509962346056 73.25704958365743) bank73680 +73681 POINT(40.7147044022896 74.5029877622798) bank73681 +73682 POINT(39.86073771065246 73.06275101228297) bank73682 +73683 POINT(40.18360618191353 74.83598732209089) bank73683 +73684 POINT(41.11611422627472 74.44716748272396) bank73684 +73685 POINT(39.717426473891685 73.46185519588343) bank73685 +73686 POINT(40.101429282139236 74.15909746600306) bank73686 +73687 POINT(41.03200484567004 73.16879103645009) bank73687 +73688 POINT(41.004436664754174 74.05106448950757) bank73688 +73689 POINT(40.35149654389302 74.90414053457278) bank73689 +73690 POINT(41.15202818101156 73.85119463821351) bank73690 +73691 POINT(41.224898125750585 73.7390528047126) bank73691 +73692 POINT(41.138007329562626 73.65050835895363) bank73692 +73693 POINT(39.78229084943301 74.7623195739778) bank73693 +73694 POINT(40.05285252006697 73.77459718021142) bank73694 +73695 POINT(40.642576261315405 73.19287778782291) bank73695 +73696 POINT(40.39917014633282 74.39904747785079) bank73696 +73697 POINT(39.81286347002407 74.14757404077008) bank73697 +73698 POINT(41.066507404666666 73.9772491317941) bank73698 +73699 POINT(41.337234154179704 74.90327707152761) bank73699 +73700 POINT(40.781879727432624 73.23594548708269) bank73700 +73701 POINT(40.012802499987586 74.61950892126374) bank73701 +73702 POINT(41.24800028822102 74.5456879629043) bank73702 +73703 POINT(41.310080202939005 73.60763487070312) bank73703 +73704 POINT(40.52985220717623 74.05286334121747) bank73704 +73705 POINT(40.74100737273251 74.65488961887276) bank73705 +73706 POINT(41.48548952797776 74.78777051562115) bank73706 +73707 POINT(41.68559838103132 74.22004521408581) bank73707 +73708 POINT(39.85187716169662 74.54956234302107) bank73708 +73709 POINT(40.05780217199496 74.23110883750994) bank73709 +73710 POINT(41.38288213414217 73.51655477092426) bank73710 +73711 POINT(40.61829792903801 73.7575009746758) bank73711 +73712 POINT(40.1167461310937 73.61636279276316) bank73712 +73713 POINT(40.95426013981062 73.38380611804872) bank73713 +73714 POINT(40.463917410775686 73.84006306811186) bank73714 +73715 POINT(40.004084436946265 73.29825874116938) bank73715 +73716 POINT(40.89242310749647 73.38747787763803) bank73716 +73717 POINT(41.669361409922836 74.35488934283039) bank73717 +73718 POINT(41.28282765570998 73.1310362105846) bank73718 +73719 POINT(41.47146693539611 74.35577604010423) bank73719 +73720 POINT(40.66889083032863 73.09654901461076) bank73720 +73721 POINT(40.90687310974444 74.56273645193379) bank73721 +73722 POINT(40.020882630830506 73.75298858040861) bank73722 +73723 POINT(40.043852374717034 73.87483807370302) bank73723 +73724 POINT(41.11286367757908 74.37918075857752) bank73724 +73725 POINT(41.65883673803742 73.31686894293745) bank73725 +73726 POINT(40.57543435089445 73.13932493199088) bank73726 +73727 POINT(41.049896281583806 74.30359072095817) bank73727 +73728 POINT(39.91200834887532 73.9717431534209) bank73728 +73729 POINT(40.0422068557045 73.02559577495815) bank73729 +73730 POINT(40.89622565567345 73.09918569806945) bank73730 +73731 POINT(41.47647832348225 74.61149424662027) bank73731 +73732 POINT(41.53375940225967 74.78966200506102) bank73732 +73733 POINT(40.202204335215896 74.25852763637145) bank73733 +73734 POINT(40.09044319949382 73.6961950902769) bank73734 +73735 POINT(41.642936578202054 74.16539965827472) bank73735 +73736 POINT(41.36571573535946 73.50908400776548) bank73736 +73737 POINT(40.47855300447843 73.47374241715816) bank73737 +73738 POINT(40.440803763521366 74.50730116026756) bank73738 +73739 POINT(40.49855310045515 74.12279805416394) bank73739 +73740 POINT(40.64760221052316 73.50464789603586) bank73740 +73741 POINT(41.25245049439245 74.05270523355586) bank73741 +73742 POINT(40.16097370215154 74.33912166367865) bank73742 +73743 POINT(40.17568047636014 73.11664509355847) bank73743 +73744 POINT(40.43086912567304 73.43349026087331) bank73744 +73745 POINT(41.67381160159223 74.75275631489848) bank73745 +73746 POINT(40.81085864552236 74.06352858250942) bank73746 +73747 POINT(40.78002662285404 73.05457284244721) bank73747 +73748 POINT(40.336121608857056 74.3454753690402) bank73748 +73749 POINT(40.26813018798801 74.56966813772836) bank73749 +73750 POINT(40.2036930713279 74.27347261975221) bank73750 +73751 POINT(40.47775369918012 73.76252708446727) bank73751 +73752 POINT(40.49969972721783 74.12515000144595) bank73752 +73753 POINT(40.06737338739296 73.26614109557572) bank73753 +73754 POINT(40.192754756945895 74.21870414059745) bank73754 +73755 POINT(40.0489220057897 74.23503828228783) bank73755 +73756 POINT(40.64973006551789 74.65224345366929) bank73756 +73757 POINT(40.277705071805336 74.21324525758871) bank73757 +73758 POINT(40.19762133159448 74.16259434468263) bank73758 +73759 POINT(41.105916022505895 73.44899092360527) bank73759 +73760 POINT(40.50488837346079 74.33391375776898) bank73760 +73761 POINT(39.77145009796066 73.46815008645423) bank73761 +73762 POINT(41.16609277966983 73.97338045914415) bank73762 +73763 POINT(41.63531491392176 74.61180963030924) bank73763 +73764 POINT(41.202520740997905 74.90253332792325) bank73764 +73765 POINT(40.72800702687311 73.28357885253759) bank73765 +73766 POINT(41.25984876496095 74.77653722280314) bank73766 +73767 POINT(39.90721599611572 73.81825340426809) bank73767 +73768 POINT(41.5046624141739 73.33951355423632) bank73768 +73769 POINT(39.77999332240798 73.90034893229806) bank73769 +73770 POINT(41.04398123043346 73.32577626807189) bank73770 +73771 POINT(41.429331290007305 74.55744676916112) bank73771 +73772 POINT(39.98575579987941 74.10859385268432) bank73772 +73773 POINT(39.92609761565483 74.50179680366986) bank73773 +73774 POINT(40.85263986050568 73.50703434856256) bank73774 +73775 POINT(40.714054091329224 74.05909236357881) bank73775 +73776 POINT(39.88123019775842 74.49113586245389) bank73776 +73777 POINT(40.737576951248016 74.82243605164996) bank73777 +73778 POINT(40.52634153522988 73.3273713543818) bank73778 +73779 POINT(40.628289357218065 74.0441425510186) bank73779 +73780 POINT(40.489030245661446 74.82143857097161) bank73780 +73781 POINT(40.384019758780525 74.40678902387405) bank73781 +73782 POINT(40.21675499084187 74.80873155963856) bank73782 +73783 POINT(41.40090564533283 73.14296467156348) bank73783 +73784 POINT(41.56823097120422 74.67075564325268) bank73784 +73785 POINT(40.48410889212086 73.48612316788449) bank73785 +73786 POINT(40.361217772236635 73.93429201587566) bank73786 +73787 POINT(40.315583388070976 73.65175631082154) bank73787 +73788 POINT(39.94688640916316 73.35649557948017) bank73788 +73789 POINT(40.52044809439651 74.54581539452977) bank73789 +73790 POINT(40.74622358184329 73.23098371755312) bank73790 +73791 POINT(41.09760999001646 73.63434902957488) bank73791 +73792 POINT(40.20533867652188 73.40330551401438) bank73792 +73793 POINT(41.069809751829 73.80058740549089) bank73793 +73794 POINT(41.53724052240471 74.67613171254042) bank73794 +73795 POINT(39.767463709888396 73.52956639615678) bank73795 +73796 POINT(40.610937067845036 74.83381406707605) bank73796 +73797 POINT(41.50074289339975 74.79694031102547) bank73797 +73798 POINT(41.06589660950624 73.67925978141933) bank73798 +73799 POINT(41.57169319276014 73.6517402764928) bank73799 +73800 POINT(41.65374358995821 73.70108976699271) bank73800 +73801 POINT(41.15813862919573 74.99363937678874) bank73801 +73802 POINT(41.25481969889649 73.1227156144556) bank73802 +73803 POINT(40.66350826819328 73.9403680575454) bank73803 +73804 POINT(41.22266992272529 74.98708199750982) bank73804 +73805 POINT(40.17051630683472 73.69154836542205) bank73805 +73806 POINT(39.73136351315805 73.79111355045714) bank73806 +73807 POINT(40.835817685783226 74.97986221784083) bank73807 +73808 POINT(41.07215484400231 73.10616394060666) bank73808 +73809 POINT(41.47715696869404 73.47556541293193) bank73809 +73810 POINT(40.31146950397562 74.77112556512039) bank73810 +73811 POINT(39.78925631738687 74.36027128589522) bank73811 +73812 POINT(40.693139285503975 73.48842510488426) bank73812 +73813 POINT(41.18382800684316 74.37022628549404) bank73813 +73814 POINT(41.262084751417646 74.77946256603232) bank73814 +73815 POINT(41.39582868493564 74.78061056128182) bank73815 +73816 POINT(39.79853787811175 74.33920422966587) bank73816 +73817 POINT(39.8017467736211 74.97626635672263) bank73817 +73818 POINT(40.27677433916274 74.23139298135959) bank73818 +73819 POINT(41.6334066544894 74.53481449829687) bank73819 +73820 POINT(40.35200995821561 73.4104596480714) bank73820 +73821 POINT(41.16948415156821 73.18435472556) bank73821 +73822 POINT(41.616164956774966 73.57943512904666) bank73822 +73823 POINT(41.074364726191256 74.60610364832209) bank73823 +73824 POINT(39.99213942027167 74.47373401157805) bank73824 +73825 POINT(40.84807682187116 73.91048695323622) bank73825 +73826 POINT(40.49887431052603 74.70346636837033) bank73826 +73827 POINT(41.16331569267441 74.73501321800603) bank73827 +73828 POINT(41.092923942217766 74.43280120769046) bank73828 +73829 POINT(40.4257193174329 75.00415796651691) bank73829 +73830 POINT(39.96321538041696 74.52861350583692) bank73830 +73831 POINT(41.29494226236187 73.59965965500267) bank73831 +73832 POINT(41.489285845511596 74.54493473773402) bank73832 +73833 POINT(41.29467485965781 73.43715450981661) bank73833 +73834 POINT(39.858247696526064 74.75976383367438) bank73834 +73835 POINT(40.5679331379695 73.96524761643792) bank73835 +73836 POINT(40.7242347240933 73.09265147566829) bank73836 +73837 POINT(41.076737409825625 74.90470661570376) bank73837 +73838 POINT(40.55731857424914 73.55516744835502) bank73838 +73839 POINT(40.115992780948154 73.03832304225884) bank73839 +73840 POINT(41.217417113481886 73.22512986941935) bank73840 +73841 POINT(41.08965459889604 74.051551380604) bank73841 +73842 POINT(39.715456221816794 74.46218603209697) bank73842 +73843 POINT(40.713863889181525 74.76469878449412) bank73843 +73844 POINT(40.24766273626278 73.64889508602857) bank73844 +73845 POINT(40.031452981263996 74.99636234768806) bank73845 +73846 POINT(40.28715087608733 73.61201104500729) bank73846 +73847 POINT(40.91414480167588 74.75563388571469) bank73847 +73848 POINT(40.722562835329036 73.94829104346042) bank73848 +73849 POINT(40.13748163111751 73.5560367896719) bank73849 +73850 POINT(41.09686534820144 74.43487624562616) bank73850 +73851 POINT(41.497807783756464 73.73039405178349) bank73851 +73852 POINT(41.42492428695207 74.26189602414048) bank73852 +73853 POINT(40.176396644145356 74.39674125088946) bank73853 +73854 POINT(40.42783451559078 73.40288036046007) bank73854 +73855 POINT(40.39723733518552 74.233202064319) bank73855 +73856 POINT(40.821379324662686 74.85972400252888) bank73856 +73857 POINT(41.29984284757137 73.45096531792294) bank73857 +73858 POINT(40.403819937325046 74.6284986757714) bank73858 +73859 POINT(41.196193940256876 73.33369937206909) bank73859 +73860 POINT(40.49218963841057 74.08559960647001) bank73860 +73861 POINT(41.01615453142028 73.23856287619749) bank73861 +73862 POINT(40.54906431595056 74.93058069454355) bank73862 +73863 POINT(39.870112160883515 73.80159465790085) bank73863 +73864 POINT(41.17584975147158 74.24717952354527) bank73864 +73865 POINT(41.06929924220501 73.33719229084616) bank73865 +73866 POINT(41.70347239742092 74.98563544675355) bank73866 +73867 POINT(39.796694438864115 73.59420416028456) bank73867 +73868 POINT(40.68981370229574 74.75343665199783) bank73868 +73869 POINT(40.200864582974496 74.13127482917734) bank73869 +73870 POINT(40.523834357070996 73.04932938252989) bank73870 +73871 POINT(40.06825205712196 74.48098869021194) bank73871 +73872 POINT(39.94613194054962 74.61889678013239) bank73872 +73873 POINT(41.337653138365944 74.5573065430753) bank73873 +73874 POINT(40.09928897011028 73.13554441197098) bank73874 +73875 POINT(40.50111870977038 73.79021991898074) bank73875 +73876 POINT(40.17034862798057 73.65008548938755) bank73876 +73877 POINT(40.19600168662597 73.42709005374246) bank73877 +73878 POINT(41.26416596181152 73.10580480494697) bank73878 +73879 POINT(41.10859812434496 74.68122745392581) bank73879 +73880 POINT(41.411390524859605 73.55939105768573) bank73880 +73881 POINT(39.93566339801303 74.63625489102584) bank73881 +73882 POINT(40.67699988284804 73.71913543108246) bank73882 +73883 POINT(39.85137548194956 73.23750080360202) bank73883 +73884 POINT(41.616308205610075 74.39291670930307) bank73884 +73885 POINT(40.300601147723015 73.16509212195558) bank73885 +73886 POINT(40.84684176356139 74.26122344106422) bank73886 +73887 POINT(40.246362210490226 74.66762097309896) bank73887 +73888 POINT(41.328988856425596 74.26682126066703) bank73888 +73889 POINT(40.71090744513049 73.06918077226857) bank73889 +73890 POINT(41.033532025828606 74.85294295286394) bank73890 +73891 POINT(40.72932780137874 73.7138621066265) bank73891 +73892 POINT(40.593065364779676 73.88600645736638) bank73892 +73893 POINT(41.16038157421082 74.94945771308136) bank73893 +73894 POINT(41.239316086790105 74.49345704175016) bank73894 +73895 POINT(40.261048578201674 74.77992171099294) bank73895 +73896 POINT(40.7896408820836 73.71033269979631) bank73896 +73897 POINT(41.1968673874018 74.40257250515582) bank73897 +73898 POINT(40.77202581454981 74.91869042580961) bank73898 +73899 POINT(40.33556143922009 73.16938851870437) bank73899 +73900 POINT(40.61750874622057 73.39941067168587) bank73900 +73901 POINT(40.72879238850378 74.18509107755895) bank73901 +73902 POINT(40.94482368546732 74.53189174949782) bank73902 +73903 POINT(40.02301852587163 74.74053488199348) bank73903 +73904 POINT(40.731121489416694 74.16442257499948) bank73904 +73905 POINT(41.38891369290361 73.15851136544205) bank73905 +73906 POINT(40.70695809407147 74.94884085467868) bank73906 +73907 POINT(40.85166546835403 73.3050157017768) bank73907 +73908 POINT(40.04372056546833 73.97963220730888) bank73908 +73909 POINT(40.7123319960468 74.71343172091673) bank73909 +73910 POINT(39.819563217636926 73.55551653431725) bank73910 +73911 POINT(40.72669982102863 74.20983206662497) bank73911 +73912 POINT(41.42665808006712 73.25359734740523) bank73912 +73913 POINT(40.437790514972896 73.9331779233798) bank73913 +73914 POINT(40.5510909093381 73.56524280336227) bank73914 +73915 POINT(40.13478681732559 73.48243388253618) bank73915 +73916 POINT(40.033536599569935 74.3139238252932) bank73916 +73917 POINT(41.19652019433163 74.64650797625653) bank73917 +73918 POINT(40.388114422805124 73.45650659079762) bank73918 +73919 POINT(41.388292078891496 74.77614036207979) bank73919 +73920 POINT(39.779076241417414 73.82395631727788) bank73920 +73921 POINT(41.34964772128345 74.35115465100458) bank73921 +73922 POINT(39.75336195452565 74.47261685661812) bank73922 +73923 POINT(40.6367753733167 73.2350397832207) bank73923 +73924 POINT(41.39258676778488 74.37923924328116) bank73924 +73925 POINT(40.8596528774908 74.74352741300487) bank73925 +73926 POINT(40.98180362517751 74.1965312245186) bank73926 +73927 POINT(41.700811653092266 74.79282358658287) bank73927 +73928 POINT(40.01988453771322 74.44996350834485) bank73928 +73929 POINT(41.302246633876095 74.35107853519796) bank73929 +73930 POINT(40.14935222388456 73.45065339962088) bank73930 +73931 POINT(41.51828675855681 74.04992927674652) bank73931 +73932 POINT(39.71481195767081 73.13147716551192) bank73932 +73933 POINT(41.36354844972326 73.03426249437175) bank73933 +73934 POINT(40.288011570788896 73.29646152631909) bank73934 +73935 POINT(40.41423216146315 73.95741645278133) bank73935 +73936 POINT(39.7429248322917 73.20171116400381) bank73936 +73937 POINT(41.54362885465662 74.75291870278028) bank73937 +73938 POINT(41.27412678014901 74.89252486926978) bank73938 +73939 POINT(41.68539367236912 73.12404543289907) bank73939 +73940 POINT(41.71251778239162 73.91383846602412) bank73940 +73941 POINT(41.32664034190537 73.8821432938376) bank73941 +73942 POINT(41.091670418404874 73.82429234822732) bank73942 +73943 POINT(40.63102754603213 74.57959097035541) bank73943 +73944 POINT(40.24865092415321 74.6103167824833) bank73944 +73945 POINT(40.636036797065806 73.80170721246354) bank73945 +73946 POINT(40.64306011997272 73.91016574437202) bank73946 +73947 POINT(41.24631026125138 73.24158388454863) bank73947 +73948 POINT(40.33888114075318 73.79725132445388) bank73948 +73949 POINT(40.53016028380423 74.70240681613303) bank73949 +73950 POINT(41.5468548951689 73.97761421723717) bank73950 +73951 POINT(40.20678046556271 74.60637932477161) bank73951 +73952 POINT(41.051805543148305 73.23366014461602) bank73952 +73953 POINT(41.01067095722353 74.95661050727595) bank73953 +73954 POINT(40.28712052451625 74.6940998350759) bank73954 +73955 POINT(41.38073597920762 73.94054365218784) bank73955 +73956 POINT(39.864280338203834 73.51577566992248) bank73956 +73957 POINT(40.2385860949389 74.20898410711736) bank73957 +73958 POINT(39.805851810105416 73.72942768512219) bank73958 +73959 POINT(40.509109851178906 73.11949509794299) bank73959 +73960 POINT(40.18446285360055 74.36218810377977) bank73960 +73961 POINT(39.89170616511441 73.19703515103399) bank73961 +73962 POINT(41.302394389276074 73.81290910557523) bank73962 +73963 POINT(39.84961088225159 74.58192130416346) bank73963 +73964 POINT(40.397844111706604 74.15312856419158) bank73964 +73965 POINT(41.41135547000474 73.60138035145766) bank73965 +73966 POINT(40.53005312054168 73.66533309492657) bank73966 +73967 POINT(40.32697896573529 73.9692385535768) bank73967 +73968 POINT(41.26096291710469 73.48043702294983) bank73968 +73969 POINT(41.055111093736244 74.85973958919848) bank73969 +73970 POINT(40.50549759308528 74.36788472083553) bank73970 +73971 POINT(39.814191296116505 73.26028531721136) bank73971 +73972 POINT(40.016232782834926 74.1361497681486) bank73972 +73973 POINT(40.301812130805196 74.83474795423848) bank73973 +73974 POINT(39.8883343269751 73.05410047817662) bank73974 +73975 POINT(41.47715262760531 74.90311918928103) bank73975 +73976 POINT(41.11994606593358 73.13519992006275) bank73976 +73977 POINT(41.59326829519459 73.659391864825) bank73977 +73978 POINT(40.92506478842346 73.1391038803702) bank73978 +73979 POINT(41.15098411021597 74.40556548188313) bank73979 +73980 POINT(41.47913852503435 73.93004660782216) bank73980 +73981 POINT(40.82642209343326 74.65234244310375) bank73981 +73982 POINT(40.734212967155486 73.69492979492378) bank73982 +73983 POINT(40.46478929108413 74.20355706331492) bank73983 +73984 POINT(40.40884180976252 74.55489775389371) bank73984 +73985 POINT(40.00482869424532 73.89772605290308) bank73985 +73986 POINT(40.50903301178155 74.52386128972223) bank73986 +73987 POINT(40.25945326066513 74.94240543342862) bank73987 +73988 POINT(40.61650448297658 73.19611668559492) bank73988 +73989 POINT(41.465114920751134 73.64393211801587) bank73989 +73990 POINT(41.652568909461735 74.6438889245935) bank73990 +73991 POINT(40.02854267987222 73.84509189950253) bank73991 +73992 POINT(40.64083842098239 74.29354700145814) bank73992 +73993 POINT(41.13438697269909 74.57773347703493) bank73993 +73994 POINT(40.478173139269096 73.70290127183861) bank73994 +73995 POINT(39.92375594084044 74.86820420745512) bank73995 +73996 POINT(39.844742847199 74.09242086319199) bank73996 +73997 POINT(41.65517389600654 73.17494389847309) bank73997 +73998 POINT(40.88675915890462 73.66906298983871) bank73998 +73999 POINT(39.9702374831964 73.2141518285166) bank73999 +74000 POINT(40.98150320516391 73.92700498626934) bank74000 +74001 POINT(40.779218008572684 73.28489590000261) bank74001 +74002 POINT(41.48420305033726 74.25420071141238) bank74002 +74003 POINT(41.358776101605635 73.15751237234046) bank74003 +74004 POINT(40.51702306394437 74.32632779633066) bank74004 +74005 POINT(41.45541674653021 74.80434315900496) bank74005 +74006 POINT(41.640259614211075 74.22233839273147) bank74006 +74007 POINT(39.75850689211632 74.46976161220401) bank74007 +74008 POINT(40.49118422896189 74.5772310201786) bank74008 +74009 POINT(41.51680422718792 74.60605666982426) bank74009 +74010 POINT(40.382779490495174 73.01233548906721) bank74010 +74011 POINT(40.872547541467576 73.05954243420832) bank74011 +74012 POINT(41.039994025805896 74.28319879713881) bank74012 +74013 POINT(41.026119567745695 73.85830254777639) bank74013 +74014 POINT(40.76730992438125 73.36914481490437) bank74014 +74015 POINT(40.25912034672793 73.23574760663034) bank74015 +74016 POINT(40.41420744358725 73.41794829082887) bank74016 +74017 POINT(40.71201315946057 74.16143518591436) bank74017 +74018 POINT(41.202322619464546 74.27820274825434) bank74018 +74019 POINT(40.085694453268616 74.71808708542022) bank74019 +74020 POINT(40.227987356391566 74.1861929068649) bank74020 +74021 POINT(41.05947468144504 73.07790263529988) bank74021 +74022 POINT(41.32879406574012 74.55651593431949) bank74022 +74023 POINT(41.095857994121104 73.65155709297125) bank74023 +74024 POINT(40.7860631009595 73.44933191702272) bank74024 +74025 POINT(40.913210688949036 74.6465416877647) bank74025 +74026 POINT(39.72343933895686 74.12282677538005) bank74026 +74027 POINT(40.38547255753474 73.24569991379744) bank74027 +74028 POINT(41.6341173602158 73.32077571746491) bank74028 +74029 POINT(39.82908883009084 73.31724266819698) bank74029 +74030 POINT(40.20390341934772 73.92698576884034) bank74030 +74031 POINT(40.06807952398216 74.95299374531598) bank74031 +74032 POINT(40.71948335572904 74.25529423111499) bank74032 +74033 POINT(40.702495214601086 73.62725245114363) bank74033 +74034 POINT(40.57877603305028 74.0476198564276) bank74034 +74035 POINT(40.614228128878935 74.33211899608968) bank74035 +74036 POINT(40.423741979470734 74.86772728861015) bank74036 +74037 POINT(41.15367666299914 74.5583677950572) bank74037 +74038 POINT(40.37864306239615 74.07744585969644) bank74038 +74039 POINT(40.621992409805216 73.4301403862074) bank74039 +74040 POINT(39.871428649252124 74.12716947718822) bank74040 +74041 POINT(40.15026260786135 73.32295153523734) bank74041 +74042 POINT(40.69774271108898 73.62651486785238) bank74042 +74043 POINT(40.613171436485544 73.7803424657739) bank74043 +74044 POINT(40.531815166142195 74.46199697917712) bank74044 +74045 POINT(41.158715704344914 74.97321510390495) bank74045 +74046 POINT(40.81618019791791 74.35577921060889) bank74046 +74047 POINT(40.87866067765963 73.85998765703454) bank74047 +74048 POINT(39.823985689265754 74.58500696784127) bank74048 +74049 POINT(40.63709869142421 73.01922761795305) bank74049 +74050 POINT(41.12288046120114 74.39719407575674) bank74050 +74051 POINT(41.326670022237195 74.08313837409818) bank74051 +74052 POINT(40.08179314728648 74.76101228997481) bank74052 +74053 POINT(40.912894248166786 73.56975240900236) bank74053 +74054 POINT(41.68990756348787 74.35772038072668) bank74054 +74055 POINT(40.666879813706934 73.28501794646378) bank74055 +74056 POINT(41.085950805597655 73.80084551779517) bank74056 +74057 POINT(40.77402033435354 74.46040353024057) bank74057 +74058 POINT(41.10664576792039 74.88831225404164) bank74058 +74059 POINT(40.946240844629685 74.85581449982209) bank74059 +74060 POINT(39.86145921900351 73.29324294132687) bank74060 +74061 POINT(40.269207006728514 73.88873643484197) bank74061 +74062 POINT(39.713406497994896 74.04003937807578) bank74062 +74063 POINT(41.25318036708356 73.29545771900352) bank74063 +74064 POINT(40.76858592972617 74.90137324766796) bank74064 +74065 POINT(40.345148616690906 73.23517069799496) bank74065 +74066 POINT(41.401534607031515 73.03097925088294) bank74066 +74067 POINT(41.47116275952714 74.74674443790114) bank74067 +74068 POINT(40.65881669494025 73.89454980026912) bank74068 +74069 POINT(41.51122701210422 73.63937103893723) bank74069 +74070 POINT(40.766661896671145 74.9889063546995) bank74070 +74071 POINT(40.17596232450371 73.27878455285258) bank74071 +74072 POINT(40.05398683069664 74.59874882351934) bank74072 +74073 POINT(41.004060286613694 74.12033649042046) bank74073 +74074 POINT(40.80207345037972 74.19649723503403) bank74074 +74075 POINT(39.817864614165046 73.95244864433954) bank74075 +74076 POINT(40.57028903668285 74.36411290358882) bank74076 +74077 POINT(40.148310624832334 74.10239789514786) bank74077 +74078 POINT(39.8344990802747 74.63934439126866) bank74078 +74079 POINT(40.858208233539976 74.64617816736806) bank74079 +74080 POINT(41.33690086719535 74.13077722157021) bank74080 +74081 POINT(39.812070595120574 73.88767163661069) bank74081 +74082 POINT(39.89616608891708 74.23151317752202) bank74082 +74083 POINT(41.381353468331106 73.58788943086192) bank74083 +74084 POINT(41.13389690302646 73.62869567115777) bank74084 +74085 POINT(41.14795301512416 73.27801793779426) bank74085 +74086 POINT(40.68513218199069 74.64462941232289) bank74086 +74087 POINT(41.382876912730644 74.5043602921153) bank74087 +74088 POINT(40.60306739595061 73.69768222212971) bank74088 +74089 POINT(41.13403316762371 74.52099467488907) bank74089 +74090 POINT(41.548084000313104 74.45775407137224) bank74090 +74091 POINT(40.19684537380686 73.21549448816738) bank74091 +74092 POINT(41.052129623788936 74.25555971673285) bank74092 +74093 POINT(40.70727018213387 74.75452934173006) bank74093 +74094 POINT(41.073250481122884 74.94779541383012) bank74094 +74095 POINT(40.376579239154175 73.28078784822931) bank74095 +74096 POINT(39.94051143840963 74.97101949634568) bank74096 +74097 POINT(40.818222343998485 73.07705913680883) bank74097 +74098 POINT(40.96428827974629 73.9725033956805) bank74098 +74099 POINT(40.56726271825161 74.33370506739416) bank74099 +74100 POINT(41.66026452260692 74.24001090257791) bank74100 +74101 POINT(39.98815998861362 74.9649513767918) bank74101 +74102 POINT(41.315241749354094 73.99542543416457) bank74102 +74103 POINT(40.47094780681162 73.55803520792722) bank74103 +74104 POINT(41.47646061800778 73.32402654185967) bank74104 +74105 POINT(39.82403049914037 74.77624369935272) bank74105 +74106 POINT(40.141794349849754 73.85686766335887) bank74106 +74107 POINT(41.197283830838714 73.85460184132782) bank74107 +74108 POINT(39.82240214246609 74.17833827527338) bank74108 +74109 POINT(41.6335189906292 73.7025189105634) bank74109 +74110 POINT(41.43952381818991 74.57444239456781) bank74110 +74111 POINT(41.538984337142445 74.01961778486147) bank74111 +74112 POINT(40.55384426223163 73.16827939284761) bank74112 +74113 POINT(41.18719200223246 74.6567096873258) bank74113 +74114 POINT(41.273526705181546 74.97668912898749) bank74114 +74115 POINT(40.70198427945497 73.89946771743897) bank74115 +74116 POINT(40.932330827029716 73.92678327893645) bank74116 +74117 POINT(40.38018145712331 74.78389700702274) bank74117 +74118 POINT(40.402917884258486 73.45559128134217) bank74118 +74119 POINT(40.03379234629633 74.9657436782538) bank74119 +74120 POINT(40.31712542298102 73.39219086387072) bank74120 +74121 POINT(41.13633496163909 73.40958191039236) bank74121 +74122 POINT(40.18484888020395 73.52790171919449) bank74122 +74123 POINT(41.24698961902235 74.31163819328606) bank74123 +74124 POINT(41.109808373813564 74.2040962463199) bank74124 +74125 POINT(41.09605276177377 74.22599683628961) bank74125 +74126 POINT(40.659012853962984 73.66326335074417) bank74126 +74127 POINT(40.69148071406162 74.31157179428298) bank74127 +74128 POINT(40.00533672424564 74.12582056798207) bank74128 +74129 POINT(40.47569765112023 74.13975920089347) bank74129 +74130 POINT(41.66541877450935 74.15291377689684) bank74130 +74131 POINT(40.69802858925637 73.44452154900584) bank74131 +74132 POINT(41.62910626988324 74.18958001868845) bank74132 +74133 POINT(41.24562567943862 73.11437384738412) bank74133 +74134 POINT(41.211030099279405 73.88156607541845) bank74134 +74135 POINT(40.80348934580199 74.75121255552743) bank74135 +74136 POINT(40.74807709681869 74.91956438953427) bank74136 +74137 POINT(40.00825191321887 74.28644846308242) bank74137 +74138 POINT(40.08544810192581 73.72344390321271) bank74138 +74139 POINT(41.26639779446518 73.01134929461735) bank74139 +74140 POINT(41.57511204018324 74.70640559300803) bank74140 +74141 POINT(41.11731005202059 74.6931445396439) bank74141 +74142 POINT(39.92221843174644 74.49373294790844) bank74142 +74143 POINT(40.9810010831486 73.03223768678397) bank74143 +74144 POINT(40.24532078434221 74.9065817202029) bank74144 +74145 POINT(39.87358180595016 73.98662620537009) bank74145 +74146 POINT(40.43819327636839 74.93872215801224) bank74146 +74147 POINT(40.08384232897602 73.21842943152475) bank74147 +74148 POINT(41.2612327336122 73.98580794660009) bank74148 +74149 POINT(39.82257943278158 73.20576695971863) bank74149 +74150 POINT(40.384276940909395 74.72150191661473) bank74150 +74151 POINT(39.77293861976211 74.52635908500406) bank74151 +74152 POINT(41.56548123167522 73.9317213621374) bank74152 +74153 POINT(39.972805802205876 73.74612383623138) bank74153 +74154 POINT(41.413551216563306 73.39156531933997) bank74154 +74155 POINT(39.759493762034225 74.32915679089604) bank74155 +74156 POINT(40.562458627580455 74.73997260933358) bank74156 +74157 POINT(41.68055976619039 73.6230317103611) bank74157 +74158 POINT(41.344864146888796 74.04833235268737) bank74158 +74159 POINT(39.76546168607082 73.40193612607742) bank74159 +74160 POINT(39.825877802122974 73.69430736147226) bank74160 +74161 POINT(40.52955882458053 74.77036329939827) bank74161 +74162 POINT(40.07432128723362 74.44668215980394) bank74162 +74163 POINT(41.43936976765449 73.41880424872669) bank74163 +74164 POINT(40.820737155919836 74.09804226129327) bank74164 +74165 POINT(40.81937627112435 74.25381405806999) bank74165 +74166 POINT(41.54232376027584 73.7600872858215) bank74166 +74167 POINT(40.194600198996575 74.57044347282883) bank74167 +74168 POINT(41.27469228578284 74.54927092331967) bank74168 +74169 POINT(41.20573608986102 73.70144593500183) bank74169 +74170 POINT(40.08707443316611 74.82005289739315) bank74170 +74171 POINT(40.113239907562225 74.53811551560285) bank74171 +74172 POINT(40.71176859508704 74.86497710827169) bank74172 +74173 POINT(39.92579386110136 74.61246435363668) bank74173 +74174 POINT(41.54112967444232 74.03211085602204) bank74174 +74175 POINT(41.475500630470506 73.44910043737262) bank74175 +74176 POINT(40.87361513001145 74.80053495457783) bank74176 +74177 POINT(41.21720196842345 73.44960554270354) bank74177 +74178 POINT(39.98443969876156 73.14452760624589) bank74178 +74179 POINT(39.80648115404851 73.58577272853859) bank74179 +74180 POINT(40.38840784800645 74.18601349750426) bank74180 +74181 POINT(40.630077765730874 74.54753527292479) bank74181 +74182 POINT(40.543857589165675 73.16445298547313) bank74182 +74183 POINT(41.12232121619095 73.998941889536) bank74183 +74184 POINT(39.796971054423764 73.38195100083863) bank74184 +74185 POINT(41.01000068035374 74.86435169649477) bank74185 +74186 POINT(40.912808205058305 73.82632809684826) bank74186 +74187 POINT(40.74909457249528 74.13688630660099) bank74187 +74188 POINT(40.271058959043486 74.78904614787142) bank74188 +74189 POINT(41.1387660863666 74.21523805862516) bank74189 +74190 POINT(40.560092997259346 73.02496509100051) bank74190 +74191 POINT(39.90180301301792 74.33764177071585) bank74191 +74192 POINT(41.002460840407366 74.69846287282864) bank74192 +74193 POINT(40.20502348000872 73.1519378634204) bank74193 +74194 POINT(41.46506380646671 73.50717738783132) bank74194 +74195 POINT(40.560836933879735 73.08208854869491) bank74195 +74196 POINT(39.89922102834172 73.02548765672866) bank74196 +74197 POINT(40.718101846486405 73.18289505896485) bank74197 +74198 POINT(39.97992751566657 74.41798657887568) bank74198 +74199 POINT(41.1012651295503 74.72465374239597) bank74199 +74200 POINT(41.372191359608735 74.29010310976523) bank74200 +74201 POINT(40.393278717384504 74.25923657227736) bank74201 +74202 POINT(40.38272605468369 73.75993312674136) bank74202 +74203 POINT(40.829912965700444 73.73524997556103) bank74203 +74204 POINT(40.47407546974105 73.29206869021121) bank74204 +74205 POINT(39.876394820050905 74.26865784987865) bank74205 +74206 POINT(39.83043247004429 74.83136725724485) bank74206 +74207 POINT(41.16064492549484 74.05548015168351) bank74207 +74208 POINT(40.963455432075286 73.97823089506132) bank74208 +74209 POINT(39.74572308643674 73.06317806699921) bank74209 +74210 POINT(40.36253293118022 73.53170962038112) bank74210 +74211 POINT(41.352592657126316 74.77207819375947) bank74211 +74212 POINT(40.5654369821538 73.02278636517761) bank74212 +74213 POINT(40.50320050648713 74.59415603089853) bank74213 +74214 POINT(39.86846971715238 73.45610941811415) bank74214 +74215 POINT(41.672699755795705 73.9310026639598) bank74215 +74216 POINT(40.328489835983305 74.7158601093483) bank74216 +74217 POINT(41.16938008803184 73.82712773086168) bank74217 +74218 POINT(40.68586826856727 73.59455001659) bank74218 +74219 POINT(41.267582156717886 74.47785545425243) bank74219 +74220 POINT(39.86445918269517 74.60917776293952) bank74220 +74221 POINT(41.116349496080204 73.72198660749771) bank74221 +74222 POINT(39.78286601787786 74.8628219407956) bank74222 +74223 POINT(40.16611157773447 74.76333147834816) bank74223 +74224 POINT(41.08935414099253 73.5280414507297) bank74224 +74225 POINT(41.16934478884817 74.82592966705812) bank74225 +74226 POINT(41.15380881679745 73.29303944360787) bank74226 +74227 POINT(40.525307553721746 73.99241661052548) bank74227 +74228 POINT(41.452104157127756 74.78394132103678) bank74228 +74229 POINT(39.728972456267336 73.68093724401872) bank74229 +74230 POINT(40.431485727313266 74.56769353831255) bank74230 +74231 POINT(40.35482598967744 74.8427496981111) bank74231 +74232 POINT(40.2866406473168 74.39024614593981) bank74232 +74233 POINT(40.33610094429157 73.22441605141114) bank74233 +74234 POINT(41.15035418875139 73.47134513788075) bank74234 +74235 POINT(40.93243499036599 73.77240669958755) bank74235 +74236 POINT(39.79916983896523 74.26620398471763) bank74236 +74237 POINT(41.21463787276695 73.08371936693864) bank74237 +74238 POINT(40.82702494420324 74.31654335757536) bank74238 +74239 POINT(40.66765762761535 74.58341832718985) bank74239 +74240 POINT(41.31550504698672 73.59429541284311) bank74240 +74241 POINT(40.680441987049264 74.32103423294835) bank74241 +74242 POINT(40.36107398997216 74.08434170492768) bank74242 +74243 POINT(39.90630869760467 73.7025326150295) bank74243 +74244 POINT(40.1271100849071 73.27682321348041) bank74244 +74245 POINT(40.324384116259395 74.4903110821349) bank74245 +74246 POINT(40.27063327177293 74.62358764604151) bank74246 +74247 POINT(40.39793413373702 73.31429560172658) bank74247 +74248 POINT(41.608069564072885 73.54085013469403) bank74248 +74249 POINT(40.235216184051374 74.02377649812685) bank74249 +74250 POINT(40.80569813968021 74.97143374721057) bank74250 +74251 POINT(40.388849184706935 74.379342014737) bank74251 +74252 POINT(41.202187922205944 73.74530912718652) bank74252 +74253 POINT(41.62766905952595 73.41175355458527) bank74253 +74254 POINT(40.933033623888505 74.44784721697278) bank74254 +74255 POINT(40.30197993914324 74.18740358340922) bank74255 +74256 POINT(39.950246746610695 74.59444208933488) bank74256 +74257 POINT(40.434395619011305 74.06803119409417) bank74257 +74258 POINT(39.99377636309646 74.73684597757307) bank74258 +74259 POINT(40.22520806523586 73.5639597690785) bank74259 +74260 POINT(41.570261128892994 74.15223873146347) bank74260 +74261 POINT(39.96522773544765 73.38413315442573) bank74261 +74262 POINT(40.971086174778385 73.11731862834671) bank74262 +74263 POINT(41.115390026953776 74.73639747376819) bank74263 +74264 POINT(40.50684456186928 74.89407090783847) bank74264 +74265 POINT(40.36946744666577 73.57246053507521) bank74265 +74266 POINT(41.58006574207543 73.34454392512387) bank74266 +74267 POINT(40.366887081304526 73.98817621066947) bank74267 +74268 POINT(39.81640461242669 73.66677060352495) bank74268 +74269 POINT(39.80625324721059 74.39458729104152) bank74269 +74270 POINT(40.68244123564867 74.78917625057866) bank74270 +74271 POINT(40.99234321339128 73.60002000339412) bank74271 +74272 POINT(40.946399551354126 73.28798539633705) bank74272 +74273 POINT(40.40484545137085 73.09633770266451) bank74273 +74274 POINT(40.00932007932625 74.1383826711527) bank74274 +74275 POINT(39.84221461742696 74.12146366657431) bank74275 +74276 POINT(41.14308389716729 74.30382062841382) bank74276 +74277 POINT(41.62154980014551 74.66383983076123) bank74277 +74278 POINT(40.08600393832745 73.20230171121666) bank74278 +74279 POINT(39.85426302040641 73.21109193962766) bank74279 +74280 POINT(40.891070366795205 73.43708499551491) bank74280 +74281 POINT(39.76148385672514 74.37169514298161) bank74281 +74282 POINT(40.61888220143135 74.33955283366014) bank74282 +74283 POINT(40.73745016916019 73.27377812210732) bank74283 +74284 POINT(41.43239883291422 73.35929255067686) bank74284 +74285 POINT(39.8906394637646 74.11617233720686) bank74285 +74286 POINT(40.80025318406438 74.91531777658474) bank74286 +74287 POINT(39.91263223921159 74.02775591731083) bank74287 +74288 POINT(39.8670771398115 73.51363124913722) bank74288 +74289 POINT(39.799555074006165 73.7541662309237) bank74289 +74290 POINT(40.87281782499129 74.79235402857738) bank74290 +74291 POINT(41.56011427567488 74.46156484707325) bank74291 +74292 POINT(41.088173014886976 73.38425616864761) bank74292 +74293 POINT(40.33291567461959 74.39268936995141) bank74293 +74294 POINT(40.10385934766154 74.13681885432655) bank74294 +74295 POINT(40.35723329070537 73.93558482212865) bank74295 +74296 POINT(41.34606551389041 74.47126161579867) bank74296 +74297 POINT(41.04782445506749 73.7795058427127) bank74297 +74298 POINT(40.343276665388736 74.92341398969698) bank74298 +74299 POINT(41.39895695245 73.30149540110625) bank74299 +74300 POINT(40.54996488163861 73.1562875284593) bank74300 +74301 POINT(40.93142329759677 74.52566688619966) bank74301 +74302 POINT(40.75642052035002 73.41107347174608) bank74302 +74303 POINT(40.589935669465795 73.1500643119775) bank74303 +74304 POINT(40.89636533623125 74.42850073305607) bank74304 +74305 POINT(41.113445912371375 73.45473066113571) bank74305 +74306 POINT(40.32520206501836 74.6031504315183) bank74306 +74307 POINT(41.426083508839646 74.51819993597624) bank74307 +74308 POINT(41.61264508978878 74.21173171251759) bank74308 +74309 POINT(41.21302487398559 73.36364010656885) bank74309 +74310 POINT(40.7274371081811 73.81717136647967) bank74310 +74311 POINT(41.18327085315141 74.0620744875956) bank74311 +74312 POINT(41.52273286609219 73.17379470882443) bank74312 +74313 POINT(40.65043820899347 74.41498054874523) bank74313 +74314 POINT(41.483892102802 73.04880640046281) bank74314 +74315 POINT(40.10642612064686 74.53978001042003) bank74315 +74316 POINT(41.67417788745504 74.41157155890886) bank74316 +74317 POINT(41.19622693188913 73.9341096577076) bank74317 +74318 POINT(41.372924314624676 74.22199973359524) bank74318 +74319 POINT(40.186128689646225 73.17629948035062) bank74319 +74320 POINT(40.81849591568463 73.33939336520727) bank74320 +74321 POINT(41.11280099353944 74.95621929067931) bank74321 +74322 POINT(40.03986451102102 73.10088404084648) bank74322 +74323 POINT(41.50164548054935 74.04000540104268) bank74323 +74324 POINT(40.8914216538541 73.48260061601944) bank74324 +74325 POINT(39.894271374530085 74.670599205824) bank74325 +74326 POINT(40.20715084344792 73.30824383607455) bank74326 +74327 POINT(40.7674457810391 73.0821281477544) bank74327 +74328 POINT(40.37748982138094 73.81427426560654) bank74328 +74329 POINT(39.874984187259834 74.82193793793182) bank74329 +74330 POINT(41.383663434315054 73.33452305382006) bank74330 +74331 POINT(39.807400655718446 73.3391415701965) bank74331 +74332 POINT(41.26955572508029 74.28206181609931) bank74332 +74333 POINT(39.89694994035095 74.71380363345592) bank74333 +74334 POINT(40.58633532377661 74.1158031011329) bank74334 +74335 POINT(41.39728478779451 74.20571431236223) bank74335 +74336 POINT(41.058452595130206 74.49173251000201) bank74336 +74337 POINT(41.463371012095784 73.25129140804665) bank74337 +74338 POINT(39.76624584792424 73.16364472253242) bank74338 +74339 POINT(40.29370508838159 74.73878851103011) bank74339 +74340 POINT(41.14284387277515 74.09809417906091) bank74340 +74341 POINT(41.0679706253746 74.32397258954106) bank74341 +74342 POINT(40.12798015140981 74.17151706452475) bank74342 +74343 POINT(40.70246488216033 74.74282439855116) bank74343 +74344 POINT(40.64746358020017 73.34304855444098) bank74344 +74345 POINT(40.972772180135685 74.54210740695711) bank74345 +74346 POINT(40.06309350462505 74.6489613674391) bank74346 +74347 POINT(40.98889183368631 74.46116419433255) bank74347 +74348 POINT(41.54604801270265 73.61169476126929) bank74348 +74349 POINT(41.47515473334918 73.93350406130926) bank74349 +74350 POINT(41.61827383209229 74.61432675784145) bank74350 +74351 POINT(40.72029536328535 74.04998440740903) bank74351 +74352 POINT(39.90533579121773 73.76943375382895) bank74352 +74353 POINT(41.49408663002134 74.55501202237572) bank74353 +74354 POINT(41.37584689704001 73.94588459245892) bank74354 +74355 POINT(40.4643444591498 74.02857732915759) bank74355 +74356 POINT(41.255792686639005 74.24290642066659) bank74356 +74357 POINT(40.4086185457832 73.94156713908194) bank74357 +74358 POINT(40.93686187657528 73.333172514634) bank74358 +74359 POINT(40.91205350882959 74.13327756894033) bank74359 +74360 POINT(40.76629657356324 74.61700336541222) bank74360 +74361 POINT(39.99663308262607 73.92231267756073) bank74361 +74362 POINT(41.37691321156605 73.10877000321689) bank74362 +74363 POINT(41.61048974892535 74.65456954088175) bank74363 +74364 POINT(41.35419609501766 73.1479813910998) bank74364 +74365 POINT(41.494705748383254 73.48936301824308) bank74365 +74366 POINT(39.97389901211032 73.73224320075043) bank74366 +74367 POINT(39.79916724066657 74.13013148917769) bank74367 +74368 POINT(40.60963080704882 74.02649511787496) bank74368 +74369 POINT(39.841872577576574 74.75393203400031) bank74369 +74370 POINT(40.19555123135439 74.08777968744577) bank74370 +74371 POINT(40.02616028243933 74.41205882969649) bank74371 +74372 POINT(40.18317203335696 74.13022807065727) bank74372 +74373 POINT(40.64689060079819 73.30457142014218) bank74373 +74374 POINT(41.44834693976667 73.38056849259709) bank74374 +74375 POINT(40.645311357918054 73.60358198733987) bank74375 +74376 POINT(41.64111240164202 73.21654410327079) bank74376 +74377 POINT(40.30647717248865 74.50620502957003) bank74377 +74378 POINT(40.208841001006434 74.58232586720787) bank74378 +74379 POINT(40.22044462656937 74.73325056784635) bank74379 +74380 POINT(41.111900663960064 73.64216492343765) bank74380 +74381 POINT(40.4950854086277 73.2764316572006) bank74381 +74382 POINT(40.61328306037983 73.21821795953343) bank74382 +74383 POINT(40.871486360828335 74.48784396611873) bank74383 +74384 POINT(40.52490149504742 74.16181788400675) bank74384 +74385 POINT(40.82356443996021 73.23588799134978) bank74385 +74386 POINT(39.90935943728549 73.68701666459053) bank74386 +74387 POINT(39.93563847123894 74.28442508939014) bank74387 +74388 POINT(40.03762124340054 73.97433838834026) bank74388 +74389 POINT(41.27120039171027 73.93636036921649) bank74389 +74390 POINT(41.28055397100227 74.57088948930144) bank74390 +74391 POINT(40.52513090521921 74.33519290173207) bank74391 +74392 POINT(40.44154023635502 74.06228013846287) bank74392 +74393 POINT(39.828820668319935 74.38574937316075) bank74393 +74394 POINT(40.38703677547749 74.56739083141288) bank74394 +74395 POINT(40.95404219337353 74.62197796729308) bank74395 +74396 POINT(40.871044336643564 74.71812498656857) bank74396 +74397 POINT(40.75545541458671 74.01846651278765) bank74397 +74398 POINT(39.76577224914136 73.69502553444967) bank74398 +74399 POINT(40.87574567708044 74.31960391402356) bank74399 +74400 POINT(40.02071258889208 73.12516064481993) bank74400 +74401 POINT(40.633204946810906 74.3620523877786) bank74401 +74402 POINT(40.5301535842952 73.22986428438979) bank74402 +74403 POINT(40.12383468105783 73.92637993073664) bank74403 +74404 POINT(40.195129875903305 74.4690483889586) bank74404 +74405 POINT(40.606077366019015 74.76277957926293) bank74405 +74406 POINT(41.16511270204303 74.31193773429246) bank74406 +74407 POINT(41.38794265027726 74.93086432727416) bank74407 +74408 POINT(41.41273781482638 74.03195237110499) bank74408 +74409 POINT(39.91116170667252 73.03416280424342) bank74409 +74410 POINT(40.699967925764085 74.3091550571843) bank74410 +74411 POINT(40.19267778825338 74.68226610485583) bank74411 +74412 POINT(40.72579572139149 73.34550555387156) bank74412 +74413 POINT(41.667627098043 73.73164309937098) bank74413 +74414 POINT(39.95073133576823 74.78972016946709) bank74414 +74415 POINT(40.07002715533267 74.2766821989221) bank74415 +74416 POINT(39.8888758654718 73.1880149991858) bank74416 +74417 POINT(41.29834952859886 73.87593840528247) bank74417 +74418 POINT(40.580908168510426 73.30760561779189) bank74418 +74419 POINT(39.82830059748374 74.21734332460237) bank74419 +74420 POINT(40.088671160368804 74.2499663197609) bank74420 +74421 POINT(40.48746708097677 73.79236899458304) bank74421 +74422 POINT(39.96052620660126 73.12252284308204) bank74422 +74423 POINT(40.50044317870295 73.61627552336483) bank74423 +74424 POINT(41.257338226284766 74.32356858521486) bank74424 +74425 POINT(41.27986221564974 74.16226866737564) bank74425 +74426 POINT(40.72575559716805 74.82509221192521) bank74426 +74427 POINT(41.33468556077169 73.05853436732879) bank74427 +74428 POINT(39.88090467982413 73.29118404126551) bank74428 +74429 POINT(41.0297381965852 74.58343075692497) bank74429 +74430 POINT(41.016503418203555 73.05431761382837) bank74430 +74431 POINT(40.25361639509492 74.48195354205741) bank74431 +74432 POINT(41.14261015416527 74.31715397336221) bank74432 +74433 POINT(40.5723477726038 73.41447958540572) bank74433 +74434 POINT(39.81067849479964 74.91860508830023) bank74434 +74435 POINT(40.19685225392645 73.42741489085066) bank74435 +74436 POINT(40.15326406534228 74.73520210931993) bank74436 +74437 POINT(41.10427106095074 74.8259843712392) bank74437 +74438 POINT(41.621904075135475 74.97565918651495) bank74438 +74439 POINT(40.84076782678737 73.7217672992815) bank74439 +74440 POINT(40.044608688388465 73.92404494528932) bank74440 +74441 POINT(40.35766938399079 74.55297855789904) bank74441 +74442 POINT(41.36207366489178 74.29769155920386) bank74442 +74443 POINT(41.56088253139758 74.04049919345228) bank74443 +74444 POINT(40.666234234193055 74.70687002483183) bank74444 +74445 POINT(40.710052937132474 73.71994823178059) bank74445 +74446 POINT(40.42622278764602 74.12625383490202) bank74446 +74447 POINT(41.26211207187962 73.19999999294271) bank74447 +74448 POINT(40.28589037472224 73.34869999912924) bank74448 +74449 POINT(41.54114846500166 73.58453202494849) bank74449 +74450 POINT(40.729065500174755 73.04877407211545) bank74450 +74451 POINT(40.50252409297597 74.42715473155144) bank74451 +74452 POINT(41.50007605037007 73.8115797418334) bank74452 +74453 POINT(40.08595843525274 73.12871130824742) bank74453 +74454 POINT(41.274550444284124 73.72313530827128) bank74454 +74455 POINT(41.01004042837901 73.7452392983112) bank74455 +74456 POINT(41.1965005009419 73.80071862496736) bank74456 +74457 POINT(40.81746314237397 73.2067042942158) bank74457 +74458 POINT(39.83286100342189 73.33822927086847) bank74458 +74459 POINT(40.361892897528726 73.18979810926501) bank74459 +74460 POINT(41.192361433857634 74.09137066341415) bank74460 +74461 POINT(40.85786974528885 74.85375505062113) bank74461 +74462 POINT(39.72329861316644 73.38270332869709) bank74462 +74463 POINT(39.769933199797336 74.2666852620057) bank74463 +74464 POINT(41.46555380953612 74.48239379754239) bank74464 +74465 POINT(39.87421503958072 73.0763389290184) bank74465 +74466 POINT(39.91371130678511 73.44007021618224) bank74466 +74467 POINT(40.50655449296255 73.78341872465356) bank74467 +74468 POINT(41.52405357445476 73.6897745373555) bank74468 +74469 POINT(40.86631693934405 74.41129766738904) bank74469 +74470 POINT(40.679163227692186 74.8048593535253) bank74470 +74471 POINT(41.29802852797508 74.22967098764549) bank74471 +74472 POINT(40.0849547878762 73.25490113622425) bank74472 +74473 POINT(41.196529565839185 73.72658422899238) bank74473 +74474 POINT(40.133698023140234 74.38667877243125) bank74474 +74475 POINT(41.11213438168511 73.08021753894546) bank74475 +74476 POINT(41.217461367915334 73.20672552591054) bank74476 +74477 POINT(40.06325319164588 74.07033699796312) bank74477 +74478 POINT(40.010624310657946 74.62981321582188) bank74478 +74479 POINT(41.569805632015665 74.08771929639136) bank74479 +74480 POINT(39.87592283623572 73.61130493131179) bank74480 +74481 POINT(39.891256315758106 75.00254309530983) bank74481 +74482 POINT(40.90506664006475 73.55480702649928) bank74482 +74483 POINT(40.48726060638931 74.77291847691242) bank74483 +74484 POINT(40.050037683918475 74.8767002960892) bank74484 +74485 POINT(39.89713859855551 73.35394766728108) bank74485 +74486 POINT(40.71834204115299 73.69179330551987) bank74486 +74487 POINT(40.82207791678737 74.73376158321251) bank74487 +74488 POINT(41.08328054831729 73.84339767190835) bank74488 +74489 POINT(40.8251931728055 73.98242055315708) bank74489 +74490 POINT(40.101647754652014 74.13200931446704) bank74490 +74491 POINT(40.53578235487559 74.18288495914462) bank74491 +74492 POINT(41.569341063343664 73.95017219320552) bank74492 +74493 POINT(39.77383255401664 73.0770702197463) bank74493 +74494 POINT(40.585649502317 73.81351510253127) bank74494 +74495 POINT(39.863670954139415 74.52481818669135) bank74495 +74496 POINT(40.31603828968518 73.64236835103273) bank74496 +74497 POINT(41.62047783199097 73.44848375177689) bank74497 +74498 POINT(41.34779371939117 73.03398443817451) bank74498 +74499 POINT(40.88936387879811 74.5683080561897) bank74499 +74500 POINT(40.163978093588256 73.1673055020258) bank74500 +74501 POINT(40.34859061217116 74.99175190681015) bank74501 +74502 POINT(40.06594270265466 74.8164906764636) bank74502 +74503 POINT(39.88711012449874 74.82811326604805) bank74503 +74504 POINT(40.0133757660538 73.69081439661738) bank74504 +74505 POINT(41.32941193540447 73.61392121235758) bank74505 +74506 POINT(40.59349863995655 74.04177068190009) bank74506 +74507 POINT(41.32416273344824 74.85549409992315) bank74507 +74508 POINT(41.55281510169162 74.84795704573749) bank74508 +74509 POINT(40.31417805756715 73.10491209912199) bank74509 +74510 POINT(40.192185148796035 74.69696828685528) bank74510 +74511 POINT(39.73124267493131 73.03566269584772) bank74511 +74512 POINT(40.86084080527283 73.769670120172) bank74512 +74513 POINT(40.30943452593239 73.62984730604676) bank74513 +74514 POINT(40.240335877730935 73.42998481401142) bank74514 +74515 POINT(40.70467452354783 74.39631736692553) bank74515 +74516 POINT(40.016914709353145 73.47998174755966) bank74516 +74517 POINT(39.99386562403914 73.72343047248346) bank74517 +74518 POINT(39.92913647855064 73.25561145261655) bank74518 +74519 POINT(40.12052573413561 73.35415292921356) bank74519 +74520 POINT(40.21112337176329 74.73119378742028) bank74520 +74521 POINT(40.6231670593471 74.07417924223736) bank74521 +74522 POINT(40.58217944764211 73.69372915228045) bank74522 +74523 POINT(39.923262054541254 74.29975091248157) bank74523 +74524 POINT(40.34465051515232 73.01601050500891) bank74524 +74525 POINT(41.30055466797046 73.4910425380581) bank74525 +74526 POINT(40.44049777246578 74.59429123241937) bank74526 +74527 POINT(41.6842562671975 74.162709569422) bank74527 +74528 POINT(40.836068736855474 74.59815329579503) bank74528 +74529 POINT(40.23226697998174 73.53202689981943) bank74529 +74530 POINT(39.937002547340256 73.73430035743503) bank74530 +74531 POINT(41.53356223155084 73.4143728514239) bank74531 +74532 POINT(39.8242041548753 73.94399155135184) bank74532 +74533 POINT(41.18287826325642 74.69873599982162) bank74533 +74534 POINT(40.09812965225642 73.17387098425961) bank74534 +74535 POINT(41.12163223748999 74.83719174010041) bank74535 +74536 POINT(41.24306793402209 74.28568728431142) bank74536 +74537 POINT(41.019933899297214 73.39595046471693) bank74537 +74538 POINT(40.65496190443291 73.69649913892898) bank74538 +74539 POINT(39.84130434375045 73.7264000168312) bank74539 +74540 POINT(40.9001985921459 74.13341113800108) bank74540 +74541 POINT(40.916058331412344 73.57897895452346) bank74541 +74542 POINT(39.81489112197676 73.99232629068472) bank74542 +74543 POINT(41.18447098395845 73.34920857182868) bank74543 +74544 POINT(41.45816947615464 73.80569421247097) bank74544 +74545 POINT(41.009661277866456 73.96317658639653) bank74545 +74546 POINT(41.081693576990936 73.15691707219288) bank74546 +74547 POINT(41.15557964945201 73.37833166493694) bank74547 +74548 POINT(41.42946702116351 73.26413251952962) bank74548 +74549 POINT(40.31152030757935 73.76148645739309) bank74549 +74550 POINT(40.879466121195435 73.35852379323093) bank74550 +74551 POINT(41.5739458121387 74.36469783894279) bank74551 +74552 POINT(40.147682118640034 73.44451171809506) bank74552 +74553 POINT(40.27752790580169 74.06074533750797) bank74553 +74554 POINT(41.663960230677944 73.9192277082351) bank74554 +74555 POINT(40.84385147933224 73.55898229848174) bank74555 +74556 POINT(39.78853743312537 74.28820172750808) bank74556 +74557 POINT(41.3862248270538 74.69135064158054) bank74557 +74558 POINT(40.5011885033928 74.70138252659143) bank74558 +74559 POINT(40.618437574618845 74.41190724811116) bank74559 +74560 POINT(40.30281156074418 73.44705567469927) bank74560 +74561 POINT(41.30708035367462 74.75265734705356) bank74561 +74562 POINT(41.44719545930298 74.0449761840745) bank74562 +74563 POINT(40.60552871264246 73.37696712907659) bank74563 +74564 POINT(40.58389854792178 73.18031319066493) bank74564 +74565 POINT(40.711096721896006 73.28266162380466) bank74565 +74566 POINT(40.527045678131266 73.7455371870326) bank74566 +74567 POINT(40.77897442867248 73.31069320227924) bank74567 +74568 POINT(40.37075358465171 74.27755888308607) bank74568 +74569 POINT(40.18313294078417 74.087914211757) bank74569 +74570 POINT(40.52664609331317 73.96830696043197) bank74570 +74571 POINT(40.953654649478224 73.60486124549425) bank74571 +74572 POINT(40.797224526167724 73.60323102008417) bank74572 +74573 POINT(40.09930116929174 73.59462680546554) bank74573 +74574 POINT(41.151401880269894 74.51419885604699) bank74574 +74575 POINT(40.34934091733676 74.1656059661959) bank74575 +74576 POINT(41.09881583613725 73.17289314742257) bank74576 +74577 POINT(41.1651778355899 74.1811948350691) bank74577 +74578 POINT(41.08725571785422 74.46784567646051) bank74578 +74579 POINT(41.10195391330649 74.11139720117593) bank74579 +74580 POINT(41.26180753797066 74.95265637912246) bank74580 +74581 POINT(40.985248469743006 73.63210669608725) bank74581 +74582 POINT(39.878508437677816 73.88852373314718) bank74582 +74583 POINT(40.313188335073576 73.08860136411997) bank74583 +74584 POINT(40.34356579802069 73.45943024661064) bank74584 +74585 POINT(40.40038817556702 73.53761852379716) bank74585 +74586 POINT(41.1232658688358 74.2476269199476) bank74586 +74587 POINT(41.42544867053561 74.01218520589251) bank74587 +74588 POINT(40.25602930364694 73.9815475026906) bank74588 +74589 POINT(40.09396902547569 74.35523940503397) bank74589 +74590 POINT(41.42549861385253 73.51611550978289) bank74590 +74591 POINT(41.38871035947607 74.92277404178566) bank74591 +74592 POINT(40.25546667022489 73.16833519670352) bank74592 +74593 POINT(40.842502446468814 73.17385018648028) bank74593 +74594 POINT(40.349198584726686 73.69482093744723) bank74594 +74595 POINT(39.71793273584425 74.21372152887709) bank74595 +74596 POINT(40.74128914947623 74.29236156965995) bank74596 +74597 POINT(39.778751190922506 74.18371294645853) bank74597 +74598 POINT(40.50053522557114 74.23689600663609) bank74598 +74599 POINT(40.29236431233212 74.01846684604381) bank74599 +74600 POINT(41.454979058363755 73.13396864994805) bank74600 +74601 POINT(39.82222377200256 73.98493759004342) bank74601 +74602 POINT(40.07408330914911 74.64291723752052) bank74602 +74603 POINT(39.800077467209846 74.68265677049068) bank74603 +74604 POINT(39.77573522512146 73.85924936069037) bank74604 +74605 POINT(40.286044930083264 74.89503533799925) bank74605 +74606 POINT(41.2178346997913 73.22353882614296) bank74606 +74607 POINT(40.04497466035413 73.37405672604558) bank74607 +74608 POINT(40.63376768713346 74.01574551595009) bank74608 +74609 POINT(39.7300581617725 73.44281236576617) bank74609 +74610 POINT(39.95972180757111 74.16452130794757) bank74610 +74611 POINT(40.70484114646804 73.52917986777977) bank74611 +74612 POINT(41.36786364169785 73.48192605666063) bank74612 +74613 POINT(39.86079735532842 73.17103798762948) bank74613 +74614 POINT(41.01995181053078 74.25130888055682) bank74614 +74615 POINT(41.01249607787088 74.30997625365858) bank74615 +74616 POINT(40.576620727867926 74.1957192267359) bank74616 +74617 POINT(41.34654474332399 74.58137737563176) bank74617 +74618 POINT(40.61944582325654 74.08848493448335) bank74618 +74619 POINT(40.65970564283884 73.19796802020109) bank74619 +74620 POINT(41.614110360997834 74.79577163366166) bank74620 +74621 POINT(41.693578009122795 74.690101825129) bank74621 +74622 POINT(40.6609662384089 73.61178900449747) bank74622 +74623 POINT(41.481930702300126 73.61445373062543) bank74623 +74624 POINT(40.97942266296247 73.73732312337825) bank74624 +74625 POINT(40.64415499562428 74.3000540540135) bank74625 +74626 POINT(41.59628125185247 73.3801261102256) bank74626 +74627 POINT(39.74931477468638 74.87247730232824) bank74627 +74628 POINT(40.80764796308203 74.27414828041573) bank74628 +74629 POINT(40.36338234096031 73.23352434168486) bank74629 +74630 POINT(39.90356089584226 74.30917832306464) bank74630 +74631 POINT(40.82969385799058 73.76793907468596) bank74631 +74632 POINT(41.520341489475435 74.34951294992199) bank74632 +74633 POINT(41.5222724999036 74.93295454570979) bank74633 +74634 POINT(39.99730843297492 74.13973316617111) bank74634 +74635 POINT(41.291536368808316 73.97688182662408) bank74635 +74636 POINT(41.668541526127456 73.75003894845082) bank74636 +74637 POINT(40.9001129334348 73.14641189503514) bank74637 +74638 POINT(40.227423296279554 73.30303101231662) bank74638 +74639 POINT(40.82500829621233 73.84780772402777) bank74639 +74640 POINT(41.39063521440904 74.40387156835416) bank74640 +74641 POINT(41.53833721009926 73.7427057750757) bank74641 +74642 POINT(40.24317298257523 73.75224891012778) bank74642 +74643 POINT(41.325813190483906 74.67720324142485) bank74643 +74644 POINT(40.44537683797521 73.74135530602305) bank74644 +74645 POINT(40.07177324920664 74.81376215658086) bank74645 +74646 POINT(40.389771688505185 74.42570162616734) bank74646 +74647 POINT(40.735026257730425 74.0731025615035) bank74647 +74648 POINT(39.74904901758218 74.97952989073474) bank74648 +74649 POINT(41.40467542471426 73.01849503983252) bank74649 +74650 POINT(40.998068210483886 74.05544997361505) bank74650 +74651 POINT(39.78382034325499 74.9303258074269) bank74651 +74652 POINT(39.936493605672005 74.38517736483448) bank74652 +74653 POINT(39.874551490176636 74.05514349304799) bank74653 +74654 POINT(40.28388482182819 73.75563869905936) bank74654 +74655 POINT(40.97886528736279 73.48663676047855) bank74655 +74656 POINT(40.188652093693904 73.54966881303996) bank74656 +74657 POINT(40.05565107767723 74.07696173720933) bank74657 +74658 POINT(40.57551155058522 74.7146683026026) bank74658 +74659 POINT(40.71594657184491 73.88760546712811) bank74659 +74660 POINT(39.9066360681758 73.68607972756023) bank74660 +74661 POINT(41.453915902224644 74.1028349285961) bank74661 +74662 POINT(40.667545708907326 74.29681056726523) bank74662 +74663 POINT(41.51721783623299 74.35115494690508) bank74663 +74664 POINT(41.01823685875577 74.8548323933459) bank74664 +74665 POINT(40.50929862366216 74.67022160005396) bank74665 +74666 POINT(40.71696789142116 73.29042174234561) bank74666 +74667 POINT(39.987600402465326 74.87584765822115) bank74667 +74668 POINT(40.751311204592845 74.46645559049722) bank74668 +74669 POINT(41.56167114426429 74.71969912419826) bank74669 +74670 POINT(41.19971866413518 73.0225606721474) bank74670 +74671 POINT(40.81085872522652 74.8386793420328) bank74671 +74672 POINT(40.10925385564202 74.17578163248722) bank74672 +74673 POINT(41.226804670879034 73.08636233642892) bank74673 +74674 POINT(39.83738641177245 73.6302683721228) bank74674 +74675 POINT(41.205649344158346 74.9986587811018) bank74675 +74676 POINT(41.38652019127135 73.88465892044944) bank74676 +74677 POINT(40.14508749648368 74.3680058356428) bank74677 +74678 POINT(40.18312256374938 74.03507106828843) bank74678 +74679 POINT(40.37213355169349 73.99537077519668) bank74679 +74680 POINT(40.36193558468725 73.35040845808713) bank74680 +74681 POINT(40.06532295571992 73.09738360384534) bank74681 +74682 POINT(40.04546961043929 73.72910839974051) bank74682 +74683 POINT(41.16791038476135 73.44711725584156) bank74683 +74684 POINT(40.21198333037273 74.13952581210887) bank74684 +74685 POINT(40.23547019161933 73.08152644820767) bank74685 +74686 POINT(41.6325486231539 73.92222900658292) bank74686 +74687 POINT(41.65303494481999 73.00900040013178) bank74687 +74688 POINT(40.115177906301454 73.13458645817161) bank74688 +74689 POINT(40.570143973194654 73.17545817815976) bank74689 +74690 POINT(40.933896360564376 73.61565641710197) bank74690 +74691 POINT(39.997987460463854 74.18023415958717) bank74691 +74692 POINT(40.78857230994132 73.64642814808856) bank74692 +74693 POINT(40.928722572379385 74.89455429879742) bank74693 +74694 POINT(39.80953685460463 73.07534637734373) bank74694 +74695 POINT(41.470305286273266 73.34866955744155) bank74695 +74696 POINT(40.73996283878296 73.39766866037301) bank74696 +74697 POINT(40.41658244115129 74.9437135058585) bank74697 +74698 POINT(40.71741667927377 74.20845283153783) bank74698 +74699 POINT(39.76535505677151 73.45366714957551) bank74699 +74700 POINT(41.108545851695716 74.84974903362976) bank74700 +74701 POINT(40.516397722327156 74.04731819825965) bank74701 +74702 POINT(40.485657229438225 73.57346576570947) bank74702 +74703 POINT(40.97450352712146 73.38306048249576) bank74703 +74704 POINT(41.18113990702729 74.00280715660853) bank74704 +74705 POINT(39.817080880306314 74.74260085584848) bank74705 +74706 POINT(41.12431875095536 74.90411766926485) bank74706 +74707 POINT(40.98986598499973 74.66397112018458) bank74707 +74708 POINT(40.26715578175732 74.08305307080256) bank74708 +74709 POINT(40.12493275818546 74.9841860389788) bank74709 +74710 POINT(41.625360685486385 73.85155927095944) bank74710 +74711 POINT(40.40501209605852 74.18309660099631) bank74711 +74712 POINT(41.189735311928004 73.62694915472669) bank74712 +74713 POINT(39.84674743433541 74.76181823987842) bank74713 +74714 POINT(40.47676784028406 74.78675785428412) bank74714 +74715 POINT(41.64099142401138 73.91117370873724) bank74715 +74716 POINT(40.58599743074812 74.28444843004786) bank74716 +74717 POINT(40.0012544010926 73.23152020428216) bank74717 +74718 POINT(39.79821827996657 73.78684892360906) bank74718 +74719 POINT(41.52967520737743 74.43385997906033) bank74719 +74720 POINT(41.40698123357304 73.45850115576384) bank74720 +74721 POINT(41.35677120807276 74.99925833279538) bank74721 +74722 POINT(40.83993366057739 73.0712971441143) bank74722 +74723 POINT(39.80633807549151 73.51512563013438) bank74723 +74724 POINT(41.519032328646425 73.79398982455403) bank74724 +74725 POINT(40.048132944865785 74.50321096247035) bank74725 +74726 POINT(40.92046132221309 74.98329780262183) bank74726 +74727 POINT(40.1608623324065 73.12519423063836) bank74727 +74728 POINT(41.32654807586719 73.43274491905554) bank74728 +74729 POINT(40.09620185138072 74.4602331179908) bank74729 +74730 POINT(40.32973059262262 74.32675335132288) bank74730 +74731 POINT(40.65658935333926 73.61096541335155) bank74731 +74732 POINT(41.0906417851136 74.58346278731207) bank74732 +74733 POINT(41.44920150375736 74.2715791982148) bank74733 +74734 POINT(39.98619486075383 74.65963064170316) bank74734 +74735 POINT(40.12978988923314 74.16034031392462) bank74735 +74736 POINT(39.80813577887924 73.6717323069661) bank74736 +74737 POINT(39.970570923778766 73.90653454474432) bank74737 +74738 POINT(41.276690335562165 73.97213446161821) bank74738 +74739 POINT(41.373330795057804 73.37231188788877) bank74739 +74740 POINT(40.65213822817797 74.27934319449412) bank74740 +74741 POINT(39.96921018564659 74.5009967836996) bank74741 +74742 POINT(40.98907268703845 73.73732550349926) bank74742 +74743 POINT(41.47918551733119 74.2223845752236) bank74743 +74744 POINT(41.29514152250836 74.33230211445432) bank74744 +74745 POINT(40.17781559787447 74.79404185917711) bank74745 +74746 POINT(40.15150906789834 73.34968545729336) bank74746 +74747 POINT(40.720195471041485 74.78224487536745) bank74747 +74748 POINT(40.300957687535444 73.59978209827258) bank74748 +74749 POINT(40.357693322681484 73.09321864397634) bank74749 +74750 POINT(39.87159930301428 74.23753100812442) bank74750 +74751 POINT(41.1480318955037 73.18605691857248) bank74751 +74752 POINT(40.54806849877477 74.95204086516202) bank74752 +74753 POINT(40.30469669168581 73.08366505543765) bank74753 +74754 POINT(40.9505380122087 73.9649047324479) bank74754 +74755 POINT(41.40525139057165 73.84427347039374) bank74755 +74756 POINT(40.40055276575308 74.925746482539) bank74756 +74757 POINT(40.083503693947236 74.69139400067183) bank74757 +74758 POINT(40.89012102720312 73.46182567635435) bank74758 +74759 POINT(41.612656070065526 74.73630607290313) bank74759 +74760 POINT(40.677462685254426 73.95382070232418) bank74760 +74761 POINT(40.86244353191144 74.32984548358881) bank74761 +74762 POINT(41.50122340841774 74.99425740998672) bank74762 +74763 POINT(41.019924474081385 73.09391727675872) bank74763 +74764 POINT(39.81816423217845 74.09339096795341) bank74764 +74765 POINT(40.63143190632869 73.91217308799874) bank74765 +74766 POINT(40.465984768520016 74.00101031012004) bank74766 +74767 POINT(40.10442918809605 73.19547555517755) bank74767 +74768 POINT(40.80293367923527 74.65363088210034) bank74768 +74769 POINT(40.25761273613831 74.52141398765995) bank74769 +74770 POINT(41.262889644586714 74.15518370400734) bank74770 +74771 POINT(39.83160552254764 73.1313760172856) bank74771 +74772 POINT(40.65206313039692 74.25566897454226) bank74772 +74773 POINT(40.188345876356 73.01443224929534) bank74773 +74774 POINT(40.6662427871693 73.32912960567221) bank74774 +74775 POINT(41.5796457790886 73.0389978249506) bank74775 +74776 POINT(40.36752805524226 73.60679522090508) bank74776 +74777 POINT(39.98400268459121 73.66221852745254) bank74777 +74778 POINT(41.096806595000665 74.29119427413129) bank74778 +74779 POINT(39.73804725719954 74.72115187142064) bank74779 +74780 POINT(40.251040801939105 74.84470212626428) bank74780 +74781 POINT(39.86664531886759 74.11841299201983) bank74781 +74782 POINT(40.8029097281433 74.57738682559281) bank74782 +74783 POINT(41.68739972800965 74.75030551401238) bank74783 +74784 POINT(41.13535037690313 73.77783315522194) bank74784 +74785 POINT(41.444852620215386 73.39858062780415) bank74785 +74786 POINT(41.194152798791364 73.79167627282082) bank74786 +74787 POINT(40.87083620788556 73.69624751018344) bank74787 +74788 POINT(41.40855883779248 74.59500018850055) bank74788 +74789 POINT(39.77099078321456 74.58786287983905) bank74789 +74790 POINT(41.64811066907708 74.64572051510875) bank74790 +74791 POINT(40.07280603189384 74.68879709806019) bank74791 +74792 POINT(40.90817320848095 74.03674047827876) bank74792 +74793 POINT(41.03470683161644 74.10218326703564) bank74793 +74794 POINT(40.85545268626462 73.40120540650146) bank74794 +74795 POINT(41.20796471400763 73.17461750591403) bank74795 +74796 POINT(40.253208938410985 74.64524587926674) bank74796 +74797 POINT(40.83042377256169 73.22482650598488) bank74797 +74798 POINT(40.12950010796942 73.19521659085608) bank74798 +74799 POINT(40.48386051851122 74.742829924531) bank74799 +74800 POINT(39.97645042382361 73.5132354567602) bank74800 +74801 POINT(41.21537626313639 74.06707634480523) bank74801 +74802 POINT(40.22902106850765 74.9507734029065) bank74802 +74803 POINT(40.87254208440705 74.8192076066326) bank74803 +74804 POINT(40.24425893164308 73.59658097439448) bank74804 +74805 POINT(41.19396757206452 74.17978313102613) bank74805 +74806 POINT(41.53602722984251 74.80143739804316) bank74806 +74807 POINT(40.83394930886589 74.04069862788332) bank74807 +74808 POINT(39.869750960847234 73.04816523345009) bank74808 +74809 POINT(41.23987650021124 73.88581263374442) bank74809 +74810 POINT(40.49322107963895 74.39011057960644) bank74810 +74811 POINT(40.73706933657427 73.6585466491732) bank74811 +74812 POINT(40.96544328609672 74.89357573935041) bank74812 +74813 POINT(41.20201221700604 73.56318273356423) bank74813 +74814 POINT(41.05489980462104 74.14847977532804) bank74814 +74815 POINT(41.17030384184323 74.31026899391756) bank74815 +74816 POINT(40.0243036657822 73.63035887714335) bank74816 +74817 POINT(40.958548643601894 73.94882018292019) bank74817 +74818 POINT(41.51106477837955 73.67419279687677) bank74818 +74819 POINT(40.77888349889843 74.08230404073282) bank74819 +74820 POINT(40.25135718781114 74.31733630668707) bank74820 +74821 POINT(41.46211986823746 74.85162982466488) bank74821 +74822 POINT(41.472925024334515 74.57103992914004) bank74822 +74823 POINT(41.12619449569934 74.28565810194446) bank74823 +74824 POINT(41.48980938915222 73.33754493300583) bank74824 +74825 POINT(41.12427761195075 73.67944163882646) bank74825 +74826 POINT(39.85685423404362 73.37560603881168) bank74826 +74827 POINT(41.134744098918134 74.95156738991214) bank74827 +74828 POINT(39.767156882866885 74.19008436923755) bank74828 +74829 POINT(40.64714311127764 74.716148157634) bank74829 +74830 POINT(40.37796309833878 73.64357607177115) bank74830 +74831 POINT(41.101120143203396 74.66943031752051) bank74831 +74832 POINT(39.7497429203254 74.75097128794637) bank74832 +74833 POINT(39.84758714052441 74.90912364080273) bank74833 +74834 POINT(41.697643399410715 73.72058770008668) bank74834 +74835 POINT(41.59207053258677 73.30411193974987) bank74835 +74836 POINT(40.17542870541681 73.71780506441057) bank74836 +74837 POINT(39.93773412789366 73.39518701312065) bank74837 +74838 POINT(39.75047509336245 74.80820474619884) bank74838 +74839 POINT(41.25848024669836 74.0741269851027) bank74839 +74840 POINT(40.6083571717453 73.43770895174008) bank74840 +74841 POINT(41.51243769396213 74.16897890981048) bank74841 +74842 POINT(39.90911054736316 74.6927999000563) bank74842 +74843 POINT(40.98184141512263 73.53767940259708) bank74843 +74844 POINT(39.91959592609338 73.86303901522913) bank74844 +74845 POINT(39.79002244764371 73.5367307946707) bank74845 +74846 POINT(41.67986106488509 73.18306954875305) bank74846 +74847 POINT(41.19105296081729 73.61368067006823) bank74847 +74848 POINT(40.562429760564534 74.57704548937627) bank74848 +74849 POINT(40.83598526946206 73.06430527141215) bank74849 +74850 POINT(40.194708328429144 74.1522473306002) bank74850 +74851 POINT(39.76093222179934 73.16410650525525) bank74851 +74852 POINT(40.57166513014466 73.51602227734432) bank74852 +74853 POINT(40.128771299481 74.8580761675149) bank74853 +74854 POINT(40.96997982949598 73.60144888131983) bank74854 +74855 POINT(40.254425230283985 74.7202852444447) bank74855 +74856 POINT(40.68479356862237 73.78059607917744) bank74856 +74857 POINT(39.93653686237246 74.1800824038793) bank74857 +74858 POINT(41.2962062647488 73.08204973715243) bank74858 +74859 POINT(40.10469227111948 74.69107599751848) bank74859 +74860 POINT(40.60034707986486 74.46133509799783) bank74860 +74861 POINT(41.4774956010289 74.80573247146576) bank74861 +74862 POINT(41.613455992141965 74.54992188200283) bank74862 +74863 POINT(40.49906547355048 73.93388130684023) bank74863 +74864 POINT(41.417475652666845 74.70649127318158) bank74864 +74865 POINT(40.373946085108436 74.33300386737264) bank74865 +74866 POINT(41.61292063683298 73.97579116549863) bank74866 +74867 POINT(40.504234179928766 73.05846690834234) bank74867 +74868 POINT(41.47909102702141 73.1155418405412) bank74868 +74869 POINT(40.400481110669176 74.81598397199427) bank74869 +74870 POINT(41.06940604256594 73.29563533227949) bank74870 +74871 POINT(40.793489266586846 73.49049915876068) bank74871 +74872 POINT(40.96100494685098 73.74399184124961) bank74872 +74873 POINT(41.695777080326074 73.2048579846991) bank74873 +74874 POINT(41.302591013215846 74.6339296361655) bank74874 +74875 POINT(40.68568573047047 74.42944147894754) bank74875 +74876 POINT(41.5881992232631 73.54891313259532) bank74876 +74877 POINT(41.333775143719784 74.60999443874363) bank74877 +74878 POINT(40.60619045500621 73.31962151453953) bank74878 +74879 POINT(41.004946769137376 74.54829248049292) bank74879 +74880 POINT(41.06540977238902 74.89829988207927) bank74880 +74881 POINT(40.01827031012391 74.60441279273194) bank74881 +74882 POINT(40.1170691109351 73.16990965467147) bank74882 +74883 POINT(41.140222771612265 73.83983600033598) bank74883 +74884 POINT(41.05949219431623 73.63294234791299) bank74884 +74885 POINT(40.434177707122515 74.2045088264232) bank74885 +74886 POINT(40.897462223888844 73.05399551682112) bank74886 +74887 POINT(41.292316999190426 73.32098790355634) bank74887 +74888 POINT(41.0319618335058 73.60023345815353) bank74888 +74889 POINT(40.611869554946736 73.02828933142854) bank74889 +74890 POINT(40.518712712631626 74.25838908585149) bank74890 +74891 POINT(41.274184420893114 74.8908733757121) bank74891 +74892 POINT(40.60235043981338 73.55588066607126) bank74892 +74893 POINT(40.56857171009152 73.85129650701953) bank74893 +74894 POINT(40.50048382695746 73.03988235817953) bank74894 +74895 POINT(39.9887342173291 73.1475778292742) bank74895 +74896 POINT(40.44518453739805 74.95111781054905) bank74896 +74897 POINT(41.33933851060716 73.63439658197088) bank74897 +74898 POINT(41.649625481214144 74.0129269225256) bank74898 +74899 POINT(39.87012427283899 74.21104158442438) bank74899 +74900 POINT(39.732474890126696 73.95572755617641) bank74900 +74901 POINT(39.920263631591396 74.74261309328314) bank74901 +74902 POINT(41.406259528200486 73.66576146278737) bank74902 +74903 POINT(40.93259667665043 73.49550654922236) bank74903 +74904 POINT(40.514661295366516 73.33774761492614) bank74904 +74905 POINT(39.93313142061454 73.11346590154379) bank74905 +74906 POINT(41.69396373805882 73.92343814378134) bank74906 +74907 POINT(40.70599434549662 73.28846240022077) bank74907 +74908 POINT(40.89937441790141 74.01478337399205) bank74908 +74909 POINT(39.757433276041915 74.30952310831023) bank74909 +74910 POINT(40.646163637249664 74.14759646836382) bank74910 +74911 POINT(41.58873916820462 73.01769891196102) bank74911 +74912 POINT(40.692984316377746 73.45835309834676) bank74912 +74913 POINT(40.63461128902674 73.79156782175588) bank74913 +74914 POINT(40.25393406736845 73.49259404352127) bank74914 +74915 POINT(41.28030342327957 73.4324240584725) bank74915 +74916 POINT(40.835375194245714 73.99757127289296) bank74916 +74917 POINT(39.82743735321954 74.63701481206134) bank74917 +74918 POINT(41.685199759664414 74.66673671302023) bank74918 +74919 POINT(41.42059638095764 74.67160364182187) bank74919 +74920 POINT(41.56676363196084 73.44122075724952) bank74920 +74921 POINT(40.38298723744652 74.19862890718319) bank74921 +74922 POINT(40.06607739712745 74.21882865740248) bank74922 +74923 POINT(40.62645352369999 74.21742959676548) bank74923 +74924 POINT(41.151614823558816 73.77328951973365) bank74924 +74925 POINT(41.27242184087832 74.62292792998566) bank74925 +74926 POINT(41.43937150290574 74.18242441835748) bank74926 +74927 POINT(40.7728003815084 73.7911836921564) bank74927 +74928 POINT(40.605173699666594 74.19963722247236) bank74928 +74929 POINT(40.25560403083894 73.96664617771806) bank74929 +74930 POINT(41.18764257492687 73.20358888459249) bank74930 +74931 POINT(41.30870570794433 74.4317227251405) bank74931 +74932 POINT(41.11899521559369 74.15831098629651) bank74932 +74933 POINT(41.15724093072864 74.32084748408985) bank74933 +74934 POINT(40.04205924470342 74.18392633447306) bank74934 +74935 POINT(40.70855171951376 74.25121379266083) bank74935 +74936 POINT(40.52042935600156 73.51825306928) bank74936 +74937 POINT(41.5017096467515 74.37311052277572) bank74937 +74938 POINT(40.59944460695931 74.7893218344911) bank74938 +74939 POINT(41.01950229202802 73.64457186515833) bank74939 +74940 POINT(40.16261814949211 74.05323664080917) bank74940 +74941 POINT(41.34325219230532 74.28684213297538) bank74941 +74942 POINT(41.04945368686698 73.52035902053873) bank74942 +74943 POINT(40.60976041522027 74.94975154844627) bank74943 +74944 POINT(40.44011982781193 74.725575258413) bank74944 +74945 POINT(40.968248002929535 73.72817339635401) bank74945 +74946 POINT(41.35347735926608 74.99122452849632) bank74946 +74947 POINT(41.34379456080779 74.06170188168764) bank74947 +74948 POINT(40.64850372643678 73.41827514199542) bank74948 +74949 POINT(40.47985328785638 73.29027810820322) bank74949 +74950 POINT(40.26705146577298 74.93956876507993) bank74950 +74951 POINT(40.297430604283264 73.36054608841808) bank74951 +74952 POINT(41.36279345885253 74.38506850782476) bank74952 +74953 POINT(41.395803353682126 73.45197796244298) bank74953 +74954 POINT(40.543679176735864 73.35020705044299) bank74954 +74955 POINT(41.05293535135482 73.50025212086604) bank74955 +74956 POINT(39.8916911822405 74.87757714423593) bank74956 +74957 POINT(40.937722384611604 73.49036072594524) bank74957 +74958 POINT(41.64523996953245 74.13643285899212) bank74958 +74959 POINT(41.06216306474861 74.02721037433463) bank74959 +74960 POINT(40.85517819203771 73.5259663713023) bank74960 +74961 POINT(40.4426718065023 74.25303634433422) bank74961 +74962 POINT(40.72372742763247 73.1551358296058) bank74962 +74963 POINT(40.212687616653 74.1504693015824) bank74963 +74964 POINT(41.67145009957186 74.53163564138741) bank74964 +74965 POINT(41.122510326940166 73.36736179769434) bank74965 +74966 POINT(41.42070244044766 73.50614534130658) bank74966 +74967 POINT(40.01341677705673 73.21502591804385) bank74967 +74968 POINT(39.733399813902125 74.7462311477651) bank74968 +74969 POINT(40.17788250647019 73.7354778361608) bank74969 +74970 POINT(39.861408032259746 74.7408206331634) bank74970 +74971 POINT(39.98816649977823 74.60412739259425) bank74971 +74972 POINT(39.8005613669765 74.7479522933976) bank74972 +74973 POINT(39.860809493356776 74.12309839552076) bank74973 +74974 POINT(41.1830591323627 73.02340925124294) bank74974 +74975 POINT(41.212347995994385 74.37124977231882) bank74975 +74976 POINT(41.33815180063288 74.13933464082388) bank74976 +74977 POINT(41.17329411279244 74.83271272418352) bank74977 +74978 POINT(40.57313369788667 73.89924229566216) bank74978 +74979 POINT(40.58026135927589 74.76047426164162) bank74979 +74980 POINT(41.3839334292345 74.79286224039913) bank74980 +74981 POINT(40.39517708754846 73.69528425312636) bank74981 +74982 POINT(39.97924180905253 74.7552994610804) bank74982 +74983 POINT(40.54780477680716 74.41035502297416) bank74983 +74984 POINT(41.4722885056331 73.39313967139474) bank74984 +74985 POINT(39.98187627477569 74.66209690255245) bank74985 +74986 POINT(40.75803254317608 74.2928735964037) bank74986 +74987 POINT(41.380677614935024 74.56937982992753) bank74987 +74988 POINT(40.661878097020185 74.6951456120155) bank74988 +74989 POINT(41.457453884318284 74.30158508519243) bank74989 +74990 POINT(40.53355555338478 74.66232075791599) bank74990 +74991 POINT(41.32096357224268 74.26538742631062) bank74991 +74992 POINT(41.690184153296435 73.62340034555203) bank74992 +74993 POINT(41.61258241999332 73.2934004078523) bank74993 +74994 POINT(41.41434596066069 74.29532462305394) bank74994 +74995 POINT(40.24793468990581 73.10068239847686) bank74995 +74996 POINT(40.3803915836633 74.17871914704045) bank74996 +74997 POINT(41.298582284801945 74.03737094323843) bank74997 +74998 POINT(41.15982624080742 73.53109729877502) bank74998 +74999 POINT(40.95604474452402 74.59204543536309) bank74999 +75000 POINT(40.13790410474595 74.01696651694958) bank75000 +75001 POINT(41.67424868789233 74.3808620910928) bank75001 +75002 POINT(41.54607281565316 74.31583290611736) bank75002 +75003 POINT(40.358175927997635 74.44987720483611) bank75003 +75004 POINT(41.645588975115075 73.30151868830703) bank75004 +75005 POINT(40.94622397587482 73.03386942419765) bank75005 +75006 POINT(40.969737096863696 74.15492273448999) bank75006 +75007 POINT(39.78933269369994 73.27824908849256) bank75007 +75008 POINT(41.28278116419642 74.00755053437948) bank75008 +75009 POINT(41.637616840721336 73.41070177433922) bank75009 +75010 POINT(41.49241514776393 73.02451198268882) bank75010 +75011 POINT(40.36688695987053 74.49038502353257) bank75011 +75012 POINT(39.95436611316393 74.88650939492545) bank75012 +75013 POINT(40.8200062254682 74.99260793645196) bank75013 +75014 POINT(40.61439214627232 73.83309575939114) bank75014 +75015 POINT(40.901961508753914 74.15660864799499) bank75015 +75016 POINT(39.95083946350093 73.41968018095477) bank75016 +75017 POINT(40.8244165083749 73.63938072106197) bank75017 +75018 POINT(40.17689037342634 73.29370361290812) bank75018 +75019 POINT(40.10721519290629 73.75992101120099) bank75019 +75020 POINT(40.52152317600737 74.92206536864965) bank75020 +75021 POINT(41.16048884029869 74.59516037603066) bank75021 +75022 POINT(41.43639602221537 74.06243113746639) bank75022 +75023 POINT(41.36788403695456 73.94522236589195) bank75023 +75024 POINT(40.24073655553769 73.23185122470981) bank75024 +75025 POINT(40.11826158547812 73.26253204428772) bank75025 +75026 POINT(40.655687267152906 74.73718313096069) bank75026 +75027 POINT(41.34259663939485 74.73069887465925) bank75027 +75028 POINT(40.65470742520309 73.71065933709762) bank75028 +75029 POINT(41.66534518024196 73.5799780354944) bank75029 +75030 POINT(40.737369450852 73.70893331895974) bank75030 +75031 POINT(41.708935307326406 73.52081960615263) bank75031 +75032 POINT(40.17546225982694 74.21393129415496) bank75032 +75033 POINT(40.46260673389756 73.14348863639282) bank75033 +75034 POINT(40.79200257821468 73.79227485203099) bank75034 +75035 POINT(41.655321770531145 74.10876780404665) bank75035 +75036 POINT(41.203456396806956 73.95375426203456) bank75036 +75037 POINT(40.805630586457426 73.26243285059667) bank75037 +75038 POINT(40.13275269050399 73.89915359038966) bank75038 +75039 POINT(41.62690074179653 73.94877682928013) bank75039 +75040 POINT(40.19326205025964 73.84209590126127) bank75040 +75041 POINT(41.50335885141123 73.27145592419512) bank75041 +75042 POINT(40.81318348833329 73.86690960505685) bank75042 +75043 POINT(40.764007764290575 73.85538154005671) bank75043 +75044 POINT(41.225013053677 74.60477693396611) bank75044 +75045 POINT(41.22963923172505 74.45477000671778) bank75045 +75046 POINT(40.07329730912624 74.85532083695689) bank75046 +75047 POINT(40.65831198159166 74.17164708210338) bank75047 +75048 POINT(39.8315982832029 73.85860445740438) bank75048 +75049 POINT(41.69217935285073 73.67916567560847) bank75049 +75050 POINT(41.135939731941484 74.43335990983853) bank75050 +75051 POINT(41.277541197017854 73.32299636288711) bank75051 +75052 POINT(40.692326870636094 74.4885775647337) bank75052 +75053 POINT(40.302189960274056 74.86458817767807) bank75053 +75054 POINT(41.4668843616178 73.88229619110103) bank75054 +75055 POINT(41.29601156625842 73.9115809432343) bank75055 +75056 POINT(40.9716866657543 74.65993991808145) bank75056 +75057 POINT(39.92620344243403 73.73850488709242) bank75057 +75058 POINT(41.43285940605968 73.53501871283825) bank75058 +75059 POINT(41.62087768570927 74.37280044688671) bank75059 +75060 POINT(41.38480714263906 73.20809371222187) bank75060 +75061 POINT(40.309088108656326 74.33189594162202) bank75061 +75062 POINT(41.62983510837101 74.3248405189798) bank75062 +75063 POINT(41.54852415030964 73.57784384971788) bank75063 +75064 POINT(39.904996346944046 73.7721131246756) bank75064 +75065 POINT(41.65760247627401 74.8293086324477) bank75065 +75066 POINT(41.28240504332165 73.54637494632443) bank75066 +75067 POINT(40.09150016288509 74.9814119759669) bank75067 +75068 POINT(41.16766773688477 74.08879630460979) bank75068 +75069 POINT(41.35979215340314 73.37004381685594) bank75069 +75070 POINT(40.836703537212955 73.59090483849371) bank75070 +75071 POINT(40.301537062361426 74.96853022529524) bank75071 +75072 POINT(40.49644282037942 73.48139910963107) bank75072 +75073 POINT(40.71793303282319 74.8388287677011) bank75073 +75074 POINT(39.828120678805305 74.52957931428107) bank75074 +75075 POINT(40.841776506496146 73.20214506186416) bank75075 +75076 POINT(40.226268528556474 74.8131396277862) bank75076 +75077 POINT(39.8297074326355 73.57344314387586) bank75077 +75078 POINT(41.508606958484656 73.70102202045136) bank75078 +75079 POINT(41.29799761148452 74.12480474108104) bank75079 +75080 POINT(40.06215586564132 73.66300583999929) bank75080 +75081 POINT(41.571623152698876 74.8061061700506) bank75081 +75082 POINT(41.39382988967179 73.58842968111254) bank75082 +75083 POINT(40.693851818859045 74.24863315733353) bank75083 +75084 POINT(40.57254753023956 74.16464231663346) bank75084 +75085 POINT(40.43855499075137 74.26475735148553) bank75085 +75086 POINT(40.91335458430007 74.8842409539998) bank75086 +75087 POINT(41.00794494827215 73.89961097803948) bank75087 +75088 POINT(41.13066782509331 73.69107152375197) bank75088 +75089 POINT(40.007500839256224 74.64357688529786) bank75089 +75090 POINT(39.88310741377759 73.74900731243869) bank75090 +75091 POINT(41.32892324102299 74.31265883817011) bank75091 +75092 POINT(40.3783109803699 73.16780997963203) bank75092 +75093 POINT(40.55188988772719 74.08503602084548) bank75093 +75094 POINT(40.63842426562016 73.89609128237669) bank75094 +75095 POINT(41.2994325570796 74.46573786241659) bank75095 +75096 POINT(41.133957849940955 74.89248055709706) bank75096 +75097 POINT(41.55791491140332 74.71763245255367) bank75097 +75098 POINT(40.12968555639871 73.56453115554301) bank75098 +75099 POINT(40.88932661553176 73.98447666271852) bank75099 +75100 POINT(40.8775422658165 73.09331144993777) bank75100 +75101 POINT(40.181540040958154 74.70258717367037) bank75101 +75102 POINT(40.737364643983696 74.0920726941526) bank75102 +75103 POINT(41.44761323535658 73.12832053841056) bank75103 +75104 POINT(40.27281191271638 74.75888399423785) bank75104 +75105 POINT(40.81248987540478 74.68406965829953) bank75105 +75106 POINT(40.64153364265833 73.14032334778298) bank75106 +75107 POINT(41.268635894585415 74.3490189098485) bank75107 +75108 POINT(40.327974316441804 74.77249120581223) bank75108 +75109 POINT(41.10380664935453 73.76597146587328) bank75109 +75110 POINT(41.6228546241455 74.14836298759415) bank75110 +75111 POINT(41.55642617340451 74.0664544054084) bank75111 +75112 POINT(41.2252081965862 73.42463556880777) bank75112 +75113 POINT(41.62936064623045 74.56171310078722) bank75113 +75114 POINT(41.41725535491025 74.73444700922) bank75114 +75115 POINT(41.58428948270718 74.40820115508366) bank75115 +75116 POINT(39.802633083768924 73.341620177151) bank75116 +75117 POINT(41.50119484078405 74.96310938602102) bank75117 +75118 POINT(40.582471890476285 74.27619175105043) bank75118 +75119 POINT(40.0609182460175 73.81792873496184) bank75119 +75120 POINT(40.935367971835696 73.46699140941791) bank75120 +75121 POINT(40.0581645386824 74.2036777882633) bank75121 +75122 POINT(40.603625185479395 73.29540053978607) bank75122 +75123 POINT(40.62435034948185 73.29993492811799) bank75123 +75124 POINT(39.72106369496571 73.71788981395014) bank75124 +75125 POINT(40.3485121071433 74.54833546793527) bank75125 +75126 POINT(41.267006386661976 74.2133636686468) bank75126 +75127 POINT(40.42741666967823 73.28707646565807) bank75127 +75128 POINT(39.89062528406511 73.35753903107874) bank75128 +75129 POINT(39.802960005009744 73.37849601464019) bank75129 +75130 POINT(40.298192195132394 73.89143139822171) bank75130 +75131 POINT(41.49185827973978 73.93268958132889) bank75131 +75132 POINT(40.49900939832781 74.52320816857463) bank75132 +75133 POINT(41.61121860660215 73.6376452499486) bank75133 +75134 POINT(41.5309580558927 73.94906123059062) bank75134 +75135 POINT(41.24211202914581 74.3567183796264) bank75135 +75136 POINT(39.878192624689675 73.21852849862766) bank75136 +75137 POINT(39.837399988593994 74.39835040141195) bank75137 +75138 POINT(39.94101706891153 74.21499241058808) bank75138 +75139 POINT(40.710736221964375 74.68601253883905) bank75139 +75140 POINT(40.12732987152148 73.33403287824659) bank75140 +75141 POINT(40.02632264338321 73.98629864774615) bank75141 +75142 POINT(40.959640686146365 74.9396642071675) bank75142 +75143 POINT(40.91253622048321 73.36493304850792) bank75143 +75144 POINT(40.159726351628194 73.48299991798186) bank75144 +75145 POINT(40.28455841081345 74.18852200185891) bank75145 +75146 POINT(40.975617983103334 74.04406288443563) bank75146 +75147 POINT(40.32775501397491 73.69887672806426) bank75147 +75148 POINT(40.70097280367098 73.5644501773714) bank75148 +75149 POINT(41.49823298107315 74.31027246507469) bank75149 +75150 POINT(41.11175937420257 73.45366364499031) bank75150 +75151 POINT(41.22994984116304 74.54070674623243) bank75151 +75152 POINT(39.95595036994847 74.80257422340247) bank75152 +75153 POINT(41.38649355730823 74.88058407128293) bank75153 +75154 POINT(40.83701759696684 74.62763164674128) bank75154 +75155 POINT(41.462121637351245 73.62183548710117) bank75155 +75156 POINT(41.64800122370445 74.88110385233992) bank75156 +75157 POINT(40.57668850571402 73.76104782088238) bank75157 +75158 POINT(40.54725866955185 73.57286298911895) bank75158 +75159 POINT(40.63404814643118 74.97569178042319) bank75159 +75160 POINT(40.13021792987145 73.23789415295074) bank75160 +75161 POINT(40.31456640733235 74.04497033278463) bank75161 +75162 POINT(41.37567262191144 74.29045616150648) bank75162 +75163 POINT(40.181502418949535 74.3595275369757) bank75163 +75164 POINT(40.73092567660584 74.02969259279386) bank75164 +75165 POINT(40.685658794929196 74.04430010226717) bank75165 +75166 POINT(41.4123762571488 73.07815309718377) bank75166 +75167 POINT(40.44172557227635 74.35269853530566) bank75167 +75168 POINT(39.73202066070613 74.21504404126532) bank75168 +75169 POINT(40.531694144776154 74.88908399834524) bank75169 +75170 POINT(40.907561231329865 73.0262468401271) bank75170 +75171 POINT(41.0421052503321 74.49599780872943) bank75171 +75172 POINT(40.62695385691892 74.02586023236304) bank75172 +75173 POINT(40.04206185267028 74.53786490567992) bank75173 +75174 POINT(40.7256440618013 74.2043747361968) bank75174 +75175 POINT(41.110866237098165 74.0090920412691) bank75175 +75176 POINT(41.38713113550317 74.09358118754463) bank75176 +75177 POINT(40.818353581784855 73.54307870233313) bank75177 +75178 POINT(41.63140518917846 74.3138071427865) bank75178 +75179 POINT(41.65543981607921 74.29902194770915) bank75179 +75180 POINT(39.98854229582815 74.81814805202171) bank75180 +75181 POINT(40.68386486941926 73.0920968585087) bank75181 +75182 POINT(40.545810326811754 74.46930867049545) bank75182 +75183 POINT(40.74662346697324 74.83308206054745) bank75183 +75184 POINT(40.50902876892826 74.38972981187348) bank75184 +75185 POINT(41.30149820064292 74.69176322571573) bank75185 +75186 POINT(41.630225027625734 73.75788383839573) bank75186 +75187 POINT(41.120863899121865 74.04348211901767) bank75187 +75188 POINT(40.94502063873044 73.25071872599523) bank75188 +75189 POINT(41.5426509008263 73.0885269881476) bank75189 +75190 POINT(41.21263880315682 74.07745410514966) bank75190 +75191 POINT(39.74884349325182 73.34147070097072) bank75191 +75192 POINT(40.76837177452593 74.79219028881101) bank75192 +75193 POINT(40.60278138309973 74.42581879777971) bank75193 +75194 POINT(39.88715279406085 73.7176771555227) bank75194 +75195 POINT(40.3599789282181 73.40896200349127) bank75195 +75196 POINT(40.62191634882428 74.48428428345991) bank75196 +75197 POINT(40.63727101410442 74.18739195517705) bank75197 +75198 POINT(40.91863694508752 74.04875802173575) bank75198 +75199 POINT(41.59101206426233 74.34858612058765) bank75199 +75200 POINT(41.21145924620791 74.05739710562153) bank75200 +75201 POINT(40.199032702888516 73.43397317247356) bank75201 +75202 POINT(40.241346426303274 73.16966683360246) bank75202 +75203 POINT(40.279024682897116 74.77027436433123) bank75203 +75204 POINT(39.830737220252864 73.92646964608436) bank75204 +75205 POINT(41.37271640187919 73.78794495953802) bank75205 +75206 POINT(39.86116546906915 74.60658020063222) bank75206 +75207 POINT(39.81015746300604 73.46603715796793) bank75207 +75208 POINT(40.310715460373196 74.76585750322144) bank75208 +75209 POINT(41.294639988438746 73.99919683426202) bank75209 +75210 POINT(39.81678506236243 73.75874911341981) bank75210 +75211 POINT(41.28461341122035 74.4215976192429) bank75211 +75212 POINT(40.20486861971536 74.47893620771448) bank75212 +75213 POINT(41.43118509085038 74.00032031719466) bank75213 +75214 POINT(41.49654182156723 73.68380990413293) bank75214 +75215 POINT(41.38691301648883 74.57656302551521) bank75215 +75216 POINT(40.67814700725423 74.5270320381615) bank75216 +75217 POINT(41.342576198291376 73.64715937956245) bank75217 +75218 POINT(40.26498865532702 74.29912488583884) bank75218 +75219 POINT(40.49960267938981 74.76391044407382) bank75219 +75220 POINT(40.69460893770527 73.24989581025984) bank75220 +75221 POINT(40.334606278512815 73.45058159449675) bank75221 +75222 POINT(41.1070268530437 74.45569926431843) bank75222 +75223 POINT(41.11350681708678 73.95114168795844) bank75223 +75224 POINT(40.76466706367742 73.83932733975338) bank75224 +75225 POINT(39.85380711433231 73.01758432739679) bank75225 +75226 POINT(41.117502746136374 74.26278829522845) bank75226 +75227 POINT(40.198347893671674 73.65629995896067) bank75227 +75228 POINT(40.420855036957434 74.05343903241422) bank75228 +75229 POINT(40.27109620201211 74.96810305556698) bank75229 +75230 POINT(39.88482278277413 73.18569192575612) bank75230 +75231 POINT(40.36377865758619 73.69983535245728) bank75231 +75232 POINT(41.50215192884942 73.71250783612216) bank75232 +75233 POINT(40.82291211108759 74.8074457137037) bank75233 +75234 POINT(40.18751594383123 74.99433289575927) bank75234 +75235 POINT(41.0772183348656 74.7466310750974) bank75235 +75236 POINT(40.86409115871682 74.80904682438145) bank75236 +75237 POINT(39.94335833233674 74.96487588192825) bank75237 +75238 POINT(40.46701809728065 73.46304953156489) bank75238 +75239 POINT(40.81748486749359 73.99221583237703) bank75239 +75240 POINT(41.03343728557601 74.99304729255147) bank75240 +75241 POINT(40.57893017361791 73.54386742955707) bank75241 +75242 POINT(40.088862975758346 74.2567957432564) bank75242 +75243 POINT(41.71172515616728 73.8959521418748) bank75243 +75244 POINT(40.058533094783265 73.0267394443667) bank75244 +75245 POINT(39.96154091686075 74.9198988155106) bank75245 +75246 POINT(39.772174318826856 74.81572402970012) bank75246 +75247 POINT(40.660214053705026 74.2887358350169) bank75247 +75248 POINT(41.34398671978832 73.03384558660969) bank75248 +75249 POINT(40.50319487732642 74.35478045539251) bank75249 +75250 POINT(40.3570832155315 73.78591279059899) bank75250 +75251 POINT(40.54997380364344 74.88858043342086) bank75251 +75252 POINT(41.5432818446784 73.86530319632728) bank75252 +75253 POINT(40.09548279134683 73.32362257750842) bank75253 +75254 POINT(41.144516048895454 73.54724399117943) bank75254 +75255 POINT(40.78634708787185 74.02495313306544) bank75255 +75256 POINT(41.669674352080165 73.48275750664982) bank75256 +75257 POINT(41.65383983148559 74.26224043993003) bank75257 +75258 POINT(41.36248422016749 74.50958750695955) bank75258 +75259 POINT(39.724231262191324 73.0203881176213) bank75259 +75260 POINT(40.56901713671941 74.24267453338588) bank75260 +75261 POINT(41.32325936166286 73.02596836148105) bank75261 +75262 POINT(41.52970948306954 74.61018207776897) bank75262 +75263 POINT(40.4168066826791 74.77922197176491) bank75263 +75264 POINT(40.1027028930613 73.69882949759499) bank75264 +75265 POINT(40.735609087237556 74.04789088996579) bank75265 +75266 POINT(41.09998746834816 73.08874895633306) bank75266 +75267 POINT(40.54303855417695 75.00245911939255) bank75267 +75268 POINT(40.98800175475378 73.15731291165402) bank75268 +75269 POINT(40.71103833641376 74.40959225229022) bank75269 +75270 POINT(41.439192968305726 74.87735980570635) bank75270 +75271 POINT(40.982432445248676 73.14909841740771) bank75271 +75272 POINT(39.75258483170336 73.51973949132962) bank75272 +75273 POINT(40.55117943584972 73.49775875303455) bank75273 +75274 POINT(40.234917812616075 74.87119685706942) bank75274 +75275 POINT(39.815487860541495 73.43206304254507) bank75275 +75276 POINT(41.46907373223213 73.95364529328594) bank75276 +75277 POINT(40.61752628792585 73.24167122528736) bank75277 +75278 POINT(41.29062469996135 74.43905069878298) bank75278 +75279 POINT(40.35356977355697 74.56022672578743) bank75279 +75280 POINT(40.06110256453803 74.90100238023021) bank75280 +75281 POINT(41.60726666125079 74.17944806851814) bank75281 +75282 POINT(39.96935885208266 74.43847978469243) bank75282 +75283 POINT(40.74246926784569 73.6994956332231) bank75283 +75284 POINT(40.45996592752397 73.67070135800309) bank75284 +75285 POINT(41.60931137479844 74.1746644087641) bank75285 +75286 POINT(41.437938326268046 73.78252142842044) bank75286 +75287 POINT(41.595942161657106 74.59958856736787) bank75287 +75288 POINT(40.75833361747196 73.44828150358828) bank75288 +75289 POINT(41.66127643759157 74.1476297079013) bank75289 +75290 POINT(40.633249967838104 73.89313351820908) bank75290 +75291 POINT(41.13872372543808 73.10670383128038) bank75291 +75292 POINT(41.430996772518014 74.23587890002774) bank75292 +75293 POINT(39.722698173226675 73.35378365888381) bank75293 +75294 POINT(39.99678341304272 73.62121283086404) bank75294 +75295 POINT(40.71775131644982 73.67414948579476) bank75295 +75296 POINT(39.93031146471941 74.22156305237999) bank75296 +75297 POINT(40.94698252182606 73.4057214071925) bank75297 +75298 POINT(40.420698959643694 74.77992557666289) bank75298 +75299 POINT(39.97107879241313 73.96614750836406) bank75299 +75300 POINT(40.20434178468737 73.78706403712039) bank75300 +75301 POINT(41.01938749877534 74.45933331548774) bank75301 +75302 POINT(41.48943572388534 73.87211751579989) bank75302 +75303 POINT(40.672229985474864 73.84030365013389) bank75303 +75304 POINT(40.934861485640134 74.38096916730628) bank75304 +75305 POINT(41.38425533720651 74.14759181089379) bank75305 +75306 POINT(41.46547219084878 73.1323402202159) bank75306 +75307 POINT(41.31659696220736 74.07438916636377) bank75307 +75308 POINT(40.75807901343956 73.2424472912063) bank75308 +75309 POINT(39.71746717330272 73.46976640096874) bank75309 +75310 POINT(40.068979234261064 73.12363416624913) bank75310 +75311 POINT(41.118940506380966 73.66702813889876) bank75311 +75312 POINT(40.60216281142404 74.13859591760048) bank75312 +75313 POINT(40.606885792318465 74.19843320276358) bank75313 +75314 POINT(40.71593721535879 73.23053696285915) bank75314 +75315 POINT(40.055595599193886 73.92740351132291) bank75315 +75316 POINT(41.5165660462226 74.90160284250986) bank75316 +75317 POINT(41.65633398141706 73.1472213253035) bank75317 +75318 POINT(41.39513950741185 74.83251731396334) bank75318 +75319 POINT(40.891644623939754 74.40408826286742) bank75319 +75320 POINT(40.7889311673533 73.28288563256044) bank75320 +75321 POINT(40.839884060697706 74.48441032575128) bank75321 +75322 POINT(40.824987137409735 74.39304366781559) bank75322 +75323 POINT(40.647698575596976 73.63856289929933) bank75323 +75324 POINT(40.96811354967413 74.35723778495979) bank75324 +75325 POINT(39.878974886879156 73.1933325701684) bank75325 +75326 POINT(40.73371607253298 74.5552586337229) bank75326 +75327 POINT(41.362817902856804 73.07771634060484) bank75327 +75328 POINT(40.19451211903896 73.33701469269647) bank75328 +75329 POINT(40.76161659109712 73.13745187127617) bank75329 +75330 POINT(39.82173582227612 73.24513534780529) bank75330 +75331 POINT(41.15882095896947 73.16614242327132) bank75331 +75332 POINT(41.67020134614533 74.65472379062966) bank75332 +75333 POINT(41.039722624637584 73.93230814911419) bank75333 +75334 POINT(40.31619892973901 74.19097533200492) bank75334 +75335 POINT(40.99831981184965 73.8827748146882) bank75335 +75336 POINT(41.13840024532323 74.67469377171423) bank75336 +75337 POINT(41.48124026103909 73.5583836729602) bank75337 +75338 POINT(40.13649490695948 74.75700703111679) bank75338 +75339 POINT(41.43168983097047 74.29592761629395) bank75339 +75340 POINT(40.624529043040035 74.5278452018403) bank75340 +75341 POINT(41.51455629622817 73.7384541484287) bank75341 +75342 POINT(40.17876971598405 74.78142417575035) bank75342 +75343 POINT(39.96735308388301 73.72708301886405) bank75343 +75344 POINT(40.34506859830712 74.75123300538743) bank75344 +75345 POINT(41.30407526066922 75.00197154110893) bank75345 +75346 POINT(41.13204304903397 74.64814528343051) bank75346 +75347 POINT(41.51316498868045 73.45281812662246) bank75347 +75348 POINT(40.733313645744296 73.97676528521538) bank75348 +75349 POINT(40.30062927388504 74.90858062266206) bank75349 +75350 POINT(40.94241616870315 74.80417991091109) bank75350 +75351 POINT(40.62194050979299 73.81360294610846) bank75351 +75352 POINT(41.67716950796082 73.53675942219414) bank75352 +75353 POINT(41.555970095761 74.03709739578544) bank75353 +75354 POINT(41.32451095011356 74.92219469188184) bank75354 +75355 POINT(40.507927346399384 74.28831774632194) bank75355 +75356 POINT(40.87253512388103 73.61715435947669) bank75356 +75357 POINT(40.02836698440064 74.77467586539697) bank75357 +75358 POINT(40.570373097576905 74.74837531101785) bank75358 +75359 POINT(40.64628519549388 73.08847310084816) bank75359 +75360 POINT(39.95742699653503 73.43178972834008) bank75360 +75361 POINT(40.244679062064336 74.66972451807223) bank75361 +75362 POINT(40.69716843390972 73.11399301538609) bank75362 +75363 POINT(39.8740529311952 73.53435860666812) bank75363 +75364 POINT(40.0062314717728 74.13839747405738) bank75364 +75365 POINT(40.977890889880385 73.16140763177562) bank75365 +75366 POINT(41.24669996316333 73.25877920553273) bank75366 +75367 POINT(39.900980618943 73.60271289570316) bank75367 +75368 POINT(40.89482403400582 73.7433643110847) bank75368 +75369 POINT(40.11519015536036 73.15654347599374) bank75369 +75370 POINT(40.83510538163992 74.74375812363823) bank75370 +75371 POINT(40.40738851920129 74.54746775019444) bank75371 +75372 POINT(40.04418913607182 74.06456948642366) bank75372 +75373 POINT(41.19594294057609 73.44209813118586) bank75373 +75374 POINT(40.15689308394735 74.70184048831307) bank75374 +75375 POINT(40.59383630597395 73.36609896672063) bank75375 +75376 POINT(40.72113568608046 73.47022481355494) bank75376 +75377 POINT(40.96965080146446 75.00265882300145) bank75377 +75378 POINT(41.452264889657556 74.92281333933225) bank75378 +75379 POINT(41.656698776542065 73.50035822293016) bank75379 +75380 POINT(41.09380938324803 73.4728548155251) bank75380 +75381 POINT(40.68932928910346 73.28099780126588) bank75381 +75382 POINT(40.159499197840965 73.76459542822109) bank75382 +75383 POINT(41.36999568200875 73.06061366733314) bank75383 +75384 POINT(41.033969283271986 74.47749739808029) bank75384 +75385 POINT(39.80805590990686 73.0981410340238) bank75385 +75386 POINT(40.58899352772673 74.3356496362945) bank75386 +75387 POINT(41.248935178174364 74.51860220322992) bank75387 +75388 POINT(39.77527549140964 73.59611965819268) bank75388 +75389 POINT(40.44783287067561 74.96108597230194) bank75389 +75390 POINT(40.42693215411453 74.16327486671786) bank75390 +75391 POINT(41.56574326070198 74.49834264856112) bank75391 +75392 POINT(41.17729426980307 74.23462625716458) bank75392 +75393 POINT(40.34608947023761 74.22932582782424) bank75393 +75394 POINT(41.51971960308234 74.79213875218453) bank75394 +75395 POINT(41.65258836415552 74.39693860894741) bank75395 +75396 POINT(41.33195335164463 74.36409384140097) bank75396 +75397 POINT(41.214080197920666 73.50213458197759) bank75397 +75398 POINT(39.835730416695625 74.28037385881746) bank75398 +75399 POINT(40.91280856504529 73.84727882034778) bank75399 +75400 POINT(40.23247123618397 73.57957873013962) bank75400 +75401 POINT(40.2180650503705 73.95550476985946) bank75401 +75402 POINT(40.85068390783968 74.60900424143038) bank75402 +75403 POINT(40.393195154535285 73.31026936760333) bank75403 +75404 POINT(41.36626255967602 74.18538659415556) bank75404 +75405 POINT(41.055940311793414 74.38330952911706) bank75405 +75406 POINT(41.465852825805534 73.50300750316319) bank75406 +75407 POINT(41.35959626656394 74.13435836516138) bank75407 +75408 POINT(39.92020619570566 74.20624254169199) bank75408 +75409 POINT(40.50100414577934 73.21223845377247) bank75409 +75410 POINT(40.24842543488157 73.00912235295993) bank75410 +75411 POINT(40.47157006786107 74.93611686551169) bank75411 +75412 POINT(40.82865076457568 73.44027262303246) bank75412 +75413 POINT(41.6265148639694 73.14932384960969) bank75413 +75414 POINT(41.55957443062425 73.89753202585648) bank75414 +75415 POINT(40.70608559519118 74.29749744779946) bank75415 +75416 POINT(40.46032397437874 74.17365603852407) bank75416 +75417 POINT(40.49181059865636 74.64204645890925) bank75417 +75418 POINT(41.22272196417554 73.98672654188688) bank75418 +75419 POINT(39.912851157428086 73.72573381067419) bank75419 +75420 POINT(40.23892903004488 74.5027425204251) bank75420 +75421 POINT(40.497176578154644 74.38870074759618) bank75421 +75422 POINT(39.84450719612725 74.4318724168408) bank75422 +75423 POINT(41.001320192023805 74.37003047287399) bank75423 +75424 POINT(39.782039498189064 74.73708788708348) bank75424 +75425 POINT(41.15924929722787 74.02255451929113) bank75425 +75426 POINT(40.972164074371534 74.5740276647429) bank75426 +75427 POINT(41.051504862006425 74.99023401541993) bank75427 +75428 POINT(40.09540063424179 73.59559802056371) bank75428 +75429 POINT(41.526972950461804 74.52314630058865) bank75429 +75430 POINT(40.76518989369771 74.66948911514595) bank75430 +75431 POINT(39.790592015004385 74.27044179383846) bank75431 +75432 POINT(41.178168178296495 73.94759487926419) bank75432 +75433 POINT(40.49397436637496 74.73576716207532) bank75433 +75434 POINT(40.43208217083861 73.42163604240758) bank75434 +75435 POINT(40.33198955718303 74.82337079760892) bank75435 +75436 POINT(41.221021734149936 74.11917222771572) bank75436 +75437 POINT(39.95539460258395 74.90360704083115) bank75437 +75438 POINT(40.59202082218244 74.00647089599367) bank75438 +75439 POINT(41.387899676545864 73.25615166727415) bank75439 +75440 POINT(40.78338918443202 74.29398689638954) bank75440 +75441 POINT(41.631136727648624 74.73376397815825) bank75441 +75442 POINT(41.42127059485837 73.6900201139323) bank75442 +75443 POINT(40.88310829524288 73.97310765715031) bank75443 +75444 POINT(40.98048995581248 73.34403152288496) bank75444 +75445 POINT(40.54914915607932 74.20117460418051) bank75445 +75446 POINT(41.109884495094946 73.95520058310704) bank75446 +75447 POINT(40.00592348105596 74.8870776965118) bank75447 +75448 POINT(41.21968339309826 73.86889752344595) bank75448 +75449 POINT(41.708702106508596 74.96432225649308) bank75449 +75450 POINT(40.53997073027372 73.37565888826802) bank75450 +75451 POINT(40.783433109196174 73.52301093195005) bank75451 +75452 POINT(40.80581929053214 73.6201522813158) bank75452 +75453 POINT(40.62706702459035 73.00765854361715) bank75453 +75454 POINT(39.73914906718423 73.63344605470397) bank75454 +75455 POINT(41.70275422944954 74.75623640479651) bank75455 +75456 POINT(39.813528630426845 74.30578351705067) bank75456 +75457 POINT(40.16019722774335 74.02234576108809) bank75457 +75458 POINT(40.16152833019973 73.41390381523046) bank75458 +75459 POINT(41.588986990615666 74.69785644403909) bank75459 +75460 POINT(40.812426807700874 73.1682510185244) bank75460 +75461 POINT(40.8137060111937 73.23380136760404) bank75461 +75462 POINT(40.39576821091521 74.39077836200221) bank75462 +75463 POINT(41.53955211848281 74.7712271115924) bank75463 +75464 POINT(40.263473888175646 73.79100516138695) bank75464 +75465 POINT(40.02801122401458 73.87076458926389) bank75465 +75466 POINT(41.60063006649671 74.95631027037118) bank75466 +75467 POINT(40.43957496540926 74.35948309289785) bank75467 +75468 POINT(39.93887138943973 74.79693633517122) bank75468 +75469 POINT(40.53251201571627 73.20593538241053) bank75469 +75470 POINT(41.399568310148176 74.74903697437678) bank75470 +75471 POINT(40.17884716167948 74.4557267902956) bank75471 +75472 POINT(41.692689081950036 74.79474613943235) bank75472 +75473 POINT(41.27197111774783 74.85390876318273) bank75473 +75474 POINT(40.820532532719184 74.37391887412753) bank75474 +75475 POINT(40.1677797154852 74.50288968480659) bank75475 +75476 POINT(40.99039730814822 74.87521684510185) bank75476 +75477 POINT(40.51789214237245 73.76387519256491) bank75477 +75478 POINT(41.29386064178955 74.5530391409143) bank75478 +75479 POINT(41.05005458242858 74.26281943706685) bank75479 +75480 POINT(40.833293424572325 74.64234048684631) bank75480 +75481 POINT(41.05247097355341 73.24100250023245) bank75481 +75482 POINT(40.31470471108179 74.69133532153658) bank75482 +75483 POINT(41.07456893030048 74.39706050988873) bank75483 +75484 POINT(40.824179471774144 74.3611300255181) bank75484 +75485 POINT(39.91477067712515 73.66022332420799) bank75485 +75486 POINT(41.123479808984115 74.63386984498989) bank75486 +75487 POINT(39.90331472746417 73.95896544275602) bank75487 +75488 POINT(40.749227474699154 74.7129202095533) bank75488 +75489 POINT(39.97477816280441 74.99000267200176) bank75489 +75490 POINT(40.568830691452256 74.44246907106739) bank75490 +75491 POINT(40.15628497721753 73.30291561007246) bank75491 +75492 POINT(41.39738492630691 73.47519964014492) bank75492 +75493 POINT(41.19268300221476 73.97795709791022) bank75493 +75494 POINT(41.61133652385518 74.54254642308842) bank75494 +75495 POINT(41.196035407786304 73.93384170441033) bank75495 +75496 POINT(40.015648551380465 73.29595735119996) bank75496 +75497 POINT(40.45829340612814 74.77053770710855) bank75497 +75498 POINT(39.959339645114966 73.43434037531915) bank75498 +75499 POINT(41.60013412989137 73.17901222743066) bank75499 +75500 POINT(40.955731092639766 73.03291656338538) bank75500 +75501 POINT(41.572342098744635 74.70794872392283) bank75501 +75502 POINT(41.138182727637904 73.62398138499215) bank75502 +75503 POINT(41.645666815457126 74.0718394457187) bank75503 +75504 POINT(40.015137878747055 74.16346380087174) bank75504 +75505 POINT(41.60249151597003 74.73548788879849) bank75505 +75506 POINT(41.65020052988536 73.97643712439447) bank75506 +75507 POINT(41.589682077822694 74.48688239633235) bank75507 +75508 POINT(40.55672072860864 74.50740379694172) bank75508 +75509 POINT(41.22968906667193 73.37831208639449) bank75509 +75510 POINT(40.610745950244095 73.69369956472872) bank75510 +75511 POINT(41.461711163759 74.64170701938168) bank75511 +75512 POINT(40.81346216316426 73.72198571336386) bank75512 +75513 POINT(40.23443353293204 73.20806570623493) bank75513 +75514 POINT(40.79417289452007 73.61881097096159) bank75514 +75515 POINT(40.8418000629374 74.83076214985834) bank75515 +75516 POINT(40.25112846914451 74.530158337434) bank75516 +75517 POINT(39.92039532958148 73.39875152619345) bank75517 +75518 POINT(40.683614821206845 73.66705585035706) bank75518 +75519 POINT(41.66691628849449 73.18979862850651) bank75519 +75520 POINT(40.90339004241349 74.59835755233001) bank75520 +75521 POINT(40.673987052848496 73.97900954426406) bank75521 +75522 POINT(41.000903072028564 73.94610526667532) bank75522 +75523 POINT(41.48930351441894 73.04211107502111) bank75523 +75524 POINT(41.60219592287549 73.18524929088318) bank75524 +75525 POINT(40.94069661384949 74.64149847399342) bank75525 +75526 POINT(39.716446429676516 73.5815448845851) bank75526 +75527 POINT(40.66438939377645 73.334765761231) bank75527 +75528 POINT(41.47219465482186 73.42760100561354) bank75528 +75529 POINT(40.734581736634695 74.05629774925875) bank75529 +75530 POINT(41.22373612295619 73.97650891135815) bank75530 +75531 POINT(40.329431952586155 73.09916486227175) bank75531 +75532 POINT(40.553712697533356 74.63130212334704) bank75532 +75533 POINT(40.15778178561125 74.77077931076721) bank75533 +75534 POINT(39.989093295421966 74.66091651738238) bank75534 +75535 POINT(41.69975659134117 73.85760493828343) bank75535 +75536 POINT(40.323580506361274 73.07915681922428) bank75536 +75537 POINT(41.37699014779809 74.05948895618957) bank75537 +75538 POINT(39.82430188159219 74.73353407524196) bank75538 +75539 POINT(39.85495386830559 74.49943772789125) bank75539 +75540 POINT(41.48340679810714 73.62979582542742) bank75540 +75541 POINT(40.25053066284858 74.51342402137291) bank75541 +75542 POINT(41.27072554414145 73.33365131026895) bank75542 +75543 POINT(41.334144563262626 74.90680960158588) bank75543 +75544 POINT(41.48834442301506 74.0130253466586) bank75544 +75545 POINT(40.757801503053045 74.75331124367803) bank75545 +75546 POINT(41.12134365041213 74.07142647822982) bank75546 +75547 POINT(40.48221772481375 73.39082050339385) bank75547 +75548 POINT(41.68811850009404 74.6580024590288) bank75548 +75549 POINT(41.57361754934826 74.16999916079062) bank75549 +75550 POINT(39.932585352471065 74.75754736568254) bank75550 +75551 POINT(41.57123868583734 73.0894934899408) bank75551 +75552 POINT(40.599354892234274 73.45372582926926) bank75552 +75553 POINT(40.477922980205555 73.45032183291121) bank75553 +75554 POINT(40.75230610020205 73.01638624539505) bank75554 +75555 POINT(40.827706197716104 74.43086885259116) bank75555 +75556 POINT(39.80355386756909 73.76008883319486) bank75556 +75557 POINT(39.82290753420938 74.46089574095708) bank75557 +75558 POINT(40.54246917282429 74.28270044336197) bank75558 +75559 POINT(41.56254933274643 74.31958376264319) bank75559 +75560 POINT(41.47779291675448 73.33616717451557) bank75560 +75561 POINT(40.152588519148225 74.1603792300443) bank75561 +75562 POINT(39.89787972399634 73.11860096479752) bank75562 +75563 POINT(41.308025247542496 74.4597064795509) bank75563 +75564 POINT(41.540121429484216 74.57914110605272) bank75564 +75565 POINT(40.43880569190755 73.74308132896103) bank75565 +75566 POINT(40.975525795464755 73.65169368984928) bank75566 +75567 POINT(41.18100666171485 74.33399916601881) bank75567 +75568 POINT(41.280108032572194 73.21153897408689) bank75568 +75569 POINT(41.48808342188888 75.00406655298862) bank75569 +75570 POINT(40.55994739514708 74.70335796021459) bank75570 +75571 POINT(40.23188084346304 74.85044391558425) bank75571 +75572 POINT(41.50571038424199 74.84155023774429) bank75572 +75573 POINT(40.777660579812036 74.8779727322574) bank75573 +75574 POINT(41.67748092743701 73.21825592840325) bank75574 +75575 POINT(41.478748123311746 73.89465127613637) bank75575 +75576 POINT(39.78173905835098 73.45207226496223) bank75576 +75577 POINT(41.12848161987273 74.72644536404857) bank75577 +75578 POINT(40.716258628153476 74.33644054578296) bank75578 +75579 POINT(40.75166541405346 74.00980249278567) bank75579 +75580 POINT(41.607073241207125 73.03335777543035) bank75580 +75581 POINT(39.90722572453592 74.56042058903023) bank75581 +75582 POINT(41.45002198428098 73.09183083264347) bank75582 +75583 POINT(41.39156545256988 73.46275986491739) bank75583 +75584 POINT(40.522444096416855 73.19638940580136) bank75584 +75585 POINT(40.37365440902992 73.42865666678425) bank75585 +75586 POINT(40.340171156185924 74.21805376368975) bank75586 +75587 POINT(41.58049474861653 73.21260276223705) bank75587 +75588 POINT(40.29309982007701 73.90318149024438) bank75588 +75589 POINT(41.48332505909612 73.48308550741135) bank75589 +75590 POINT(41.420637770632204 73.42607925142865) bank75590 +75591 POINT(41.29278777873493 73.51088443512288) bank75591 +75592 POINT(41.01538776677047 74.97253823473535) bank75592 +75593 POINT(40.790246127470695 74.89408277407858) bank75593 +75594 POINT(40.45665727078129 73.48641183778625) bank75594 +75595 POINT(40.755912071005575 74.10393925572477) bank75595 +75596 POINT(41.13271298459272 73.65094715046374) bank75596 +75597 POINT(39.93212395581949 74.73002718312223) bank75597 +75598 POINT(40.03611132673967 73.03992949665471) bank75598 +75599 POINT(40.69862452625485 74.47869069611394) bank75599 +75600 POINT(41.40411852417822 73.37404147484969) bank75600 +75601 POINT(40.49716121990965 73.37435004243507) bank75601 +75602 POINT(41.53925866755824 74.75504743297982) bank75602 +75603 POINT(40.08191262497366 73.87600404133238) bank75603 +75604 POINT(40.21548269125475 74.1853050188841) bank75604 +75605 POINT(40.709916989004654 74.35269812277807) bank75605 +75606 POINT(40.874630439854904 74.49764673003773) bank75606 +75607 POINT(41.51420979262413 74.28049529796687) bank75607 +75608 POINT(40.63411314712267 74.58784393275218) bank75608 +75609 POINT(39.78901901795078 73.5657067670054) bank75609 +75610 POINT(41.494341058142126 73.3355935788762) bank75610 +75611 POINT(40.22550238408846 74.1306596862193) bank75611 +75612 POINT(41.558069220614456 74.94818421911326) bank75612 +75613 POINT(40.79699989003509 73.74648840751128) bank75613 +75614 POINT(40.092694501104454 73.63588939299551) bank75614 +75615 POINT(41.02751972158476 74.92568747491008) bank75615 +75616 POINT(41.655276218900646 74.63983469559992) bank75616 +75617 POINT(40.86193298979102 74.05972051624035) bank75617 +75618 POINT(41.11837799267174 73.75629795429839) bank75618 +75619 POINT(41.626043509620494 74.57092927709559) bank75619 +75620 POINT(41.69017188157564 73.03406684610309) bank75620 +75621 POINT(41.11645385773817 74.78342740419575) bank75621 +75622 POINT(40.65895706028218 74.03876565622102) bank75622 +75623 POINT(41.30596202475808 74.5221871774141) bank75623 +75624 POINT(39.93861950968306 73.5968858049148) bank75624 +75625 POINT(40.4609568166941 73.03697366269385) bank75625 +75626 POINT(40.156351797928295 74.0845038086544) bank75626 +75627 POINT(41.399926848254616 74.95478398429464) bank75627 +75628 POINT(40.17967319758671 74.3834774225577) bank75628 +75629 POINT(40.28182447130299 73.2884194214796) bank75629 +75630 POINT(40.98057121732202 73.17024233713221) bank75630 +75631 POINT(40.43367162096968 74.92195203565164) bank75631 +75632 POINT(39.71793544918562 73.68168338454093) bank75632 +75633 POINT(41.39420188988637 74.67763323732406) bank75633 +75634 POINT(41.70911312091847 74.2777654595185) bank75634 +75635 POINT(40.513443493295696 73.65513005071412) bank75635 +75636 POINT(41.01521055252811 74.53063257125585) bank75636 +75637 POINT(41.65457550100472 73.80047413923047) bank75637 +75638 POINT(40.98188893974999 73.94665548937427) bank75638 +75639 POINT(40.35881054696345 73.21487250989783) bank75639 +75640 POINT(40.301517436904696 73.05746108392576) bank75640 +75641 POINT(40.850703302343824 73.79954928945675) bank75641 +75642 POINT(40.429832395651054 74.56454537793523) bank75642 +75643 POINT(39.856933688925984 73.92383868085177) bank75643 +75644 POINT(41.26885484505167 73.85646033322217) bank75644 +75645 POINT(41.256025932670404 73.0416787532575) bank75645 +75646 POINT(40.214862965938146 73.23393026459141) bank75646 +75647 POINT(40.17120264752136 74.0197617014781) bank75647 +75648 POINT(39.86889924932671 73.47209391012525) bank75648 +75649 POINT(41.68263909468871 73.35319065973485) bank75649 +75650 POINT(40.73601524619067 74.42733443522684) bank75650 +75651 POINT(40.28580841915078 74.23406281998425) bank75651 +75652 POINT(41.45498392298952 74.21869567830416) bank75652 +75653 POINT(40.400117531972384 73.59104438188086) bank75653 +75654 POINT(40.27392691913063 74.5131708654947) bank75654 +75655 POINT(41.5192514933695 74.24579205069976) bank75655 +75656 POINT(40.23958729227202 74.16687499624057) bank75656 +75657 POINT(41.43670897756017 73.0322006530929) bank75657 +75658 POINT(39.96980962535774 73.88014491931916) bank75658 +75659 POINT(41.61348734518806 74.2184080799971) bank75659 +75660 POINT(40.97528163146258 74.2183185528403) bank75660 +75661 POINT(41.08705236245476 73.06015510978064) bank75661 +75662 POINT(40.77179724783665 74.61331214384117) bank75662 +75663 POINT(40.15706503810412 73.7312406845053) bank75663 +75664 POINT(40.98224910496172 73.05915409110142) bank75664 +75665 POINT(40.094316892737794 74.09993271054714) bank75665 +75666 POINT(39.88798363051047 74.23183042843351) bank75666 +75667 POINT(41.2420308791562 74.37606426884928) bank75667 +75668 POINT(40.60495193019573 73.0892079531601) bank75668 +75669 POINT(40.86578092774304 73.969329576466) bank75669 +75670 POINT(41.20276195595164 74.64153394646486) bank75670 +75671 POINT(40.14266061834883 74.38891414544754) bank75671 +75672 POINT(40.13750138705101 73.19112865493076) bank75672 +75673 POINT(40.792913214629486 74.36357223580264) bank75673 +75674 POINT(40.49208415880238 74.34514739670021) bank75674 +75675 POINT(41.10059392899024 73.70698073274677) bank75675 +75676 POINT(41.01820775253412 74.76544662065008) bank75676 +75677 POINT(40.608635677991614 73.84240771618494) bank75677 +75678 POINT(41.56964060627019 74.68748037620972) bank75678 +75679 POINT(41.01793825656481 75.0007837861762) bank75679 +75680 POINT(40.463119583166204 74.87315314658757) bank75680 +75681 POINT(41.13566416131213 73.89114167979498) bank75681 +75682 POINT(40.296272404700304 73.9611497123071) bank75682 +75683 POINT(40.79682560726006 74.86633787667697) bank75683 +75684 POINT(41.07220923043482 73.28960959207555) bank75684 +75685 POINT(39.937503048547626 74.8033024589062) bank75685 +75686 POINT(40.83093570674963 73.30975801785333) bank75686 +75687 POINT(40.17596707903251 73.6770005152149) bank75687 +75688 POINT(40.652002114927846 74.05047329787301) bank75688 +75689 POINT(40.544424923415185 74.909785570546) bank75689 +75690 POINT(39.98787014472293 74.65782964252072) bank75690 +75691 POINT(41.32148261614207 73.7450449413131) bank75691 +75692 POINT(39.95384729685102 74.38082265042735) bank75692 +75693 POINT(40.072685351290644 74.7032585970862) bank75693 +75694 POINT(39.82445028078636 73.31723501268036) bank75694 +75695 POINT(40.165169988216306 73.0370268589242) bank75695 +75696 POINT(40.39907503172013 73.48027218756867) bank75696 +75697 POINT(41.268669563708336 73.01626607693747) bank75697 +75698 POINT(40.80500639595495 73.81749022242212) bank75698 +75699 POINT(41.28126749218914 73.5828649152554) bank75699 +75700 POINT(40.34611598430397 74.49134919387376) bank75700 +75701 POINT(41.446465514058644 73.50677607099063) bank75701 +75702 POINT(40.74106288553901 73.04075706024139) bank75702 +75703 POINT(41.21642491604116 73.06439271645205) bank75703 +75704 POINT(41.17036269465756 73.51676743874977) bank75704 +75705 POINT(40.978450719013956 73.34747919965204) bank75705 +75706 POINT(40.197349051228265 74.12220086988792) bank75706 +75707 POINT(41.48793028353529 74.0954807382448) bank75707 +75708 POINT(40.84641663128777 74.09565464190197) bank75708 +75709 POINT(40.937196134073275 73.68520041221316) bank75709 +75710 POINT(40.53255830755449 73.43287613025129) bank75710 +75711 POINT(40.50406869909953 73.64326763067842) bank75711 +75712 POINT(41.30376163160192 73.1213496678545) bank75712 +75713 POINT(41.24135460595934 73.66048272878616) bank75713 +75714 POINT(40.78110715377953 73.4155450354176) bank75714 +75715 POINT(40.221629349492865 74.76377747946803) bank75715 +75716 POINT(39.74661323861662 74.61474633715666) bank75716 +75717 POINT(41.01612515030479 74.72751984657015) bank75717 +75718 POINT(40.106831886231106 73.24070675271868) bank75718 +75719 POINT(41.41910186740566 74.30164824240929) bank75719 +75720 POINT(39.98874053209939 73.21754300211823) bank75720 +75721 POINT(40.9401758789745 73.05787391199358) bank75721 +75722 POINT(40.69097076208238 74.29629597362215) bank75722 +75723 POINT(39.97805846782378 73.15097066282843) bank75723 +75724 POINT(41.424054509296234 74.05079747850289) bank75724 +75725 POINT(40.082718767453606 73.98597859630951) bank75725 +75726 POINT(41.060931009615814 74.85809547140632) bank75726 +75727 POINT(40.99257900001002 74.81501007738296) bank75727 +75728 POINT(41.285642198705695 73.6082389011517) bank75728 +75729 POINT(41.51521131175078 74.13872456108453) bank75729 +75730 POINT(41.436660212577216 74.5980959728569) bank75730 +75731 POINT(40.06422278425112 74.5474187995263) bank75731 +75732 POINT(41.49958836310277 73.57573627364016) bank75732 +75733 POINT(40.03875585561438 73.82308583030975) bank75733 +75734 POINT(39.99832545478979 73.01073885199439) bank75734 +75735 POINT(41.66824648905696 74.00193613820767) bank75735 +75736 POINT(41.56156888679024 74.51792123836388) bank75736 +75737 POINT(40.44563417029712 73.29742780440101) bank75737 +75738 POINT(40.20228374119536 73.48246107951142) bank75738 +75739 POINT(40.43898555593622 74.02567240469887) bank75739 +75740 POINT(39.942616809041915 74.30885280700372) bank75740 +75741 POINT(40.85276754825635 74.74746463723247) bank75741 +75742 POINT(40.61892166380562 73.53450381928586) bank75742 +75743 POINT(39.72602309658004 74.9562815306123) bank75743 +75744 POINT(41.450176246292244 74.71163834569876) bank75744 +75745 POINT(41.3166559015949 73.29410376219872) bank75745 +75746 POINT(40.47469129624232 73.49050395667582) bank75746 +75747 POINT(40.18510089498324 74.73890670050787) bank75747 +75748 POINT(40.15801503565603 74.35088680552819) bank75748 +75749 POINT(41.64167094739546 73.70109948173409) bank75749 +75750 POINT(40.94476035461537 73.29077701271989) bank75750 +75751 POINT(41.262706694898334 73.56996274459614) bank75751 +75752 POINT(40.679376109326384 73.53146421081186) bank75752 +75753 POINT(41.592826751739736 74.51517561552568) bank75753 +75754 POINT(41.19661879226069 73.09351244342666) bank75754 +75755 POINT(41.20651039851003 74.98697825470539) bank75755 +75756 POINT(41.44787294344966 74.14507956982325) bank75756 +75757 POINT(41.23511800816193 74.18251990247963) bank75757 +75758 POINT(39.99655806605506 74.37427081385043) bank75758 +75759 POINT(40.62341736138784 74.98243293321885) bank75759 +75760 POINT(41.447706378394514 73.55785103346514) bank75760 +75761 POINT(40.45417882120088 73.66539211215228) bank75761 +75762 POINT(40.84987533463402 73.75332078077331) bank75762 +75763 POINT(41.295397383472675 73.12831650960263) bank75763 +75764 POINT(41.293762831078084 73.3325885726503) bank75764 +75765 POINT(41.38790026525597 73.96859393473085) bank75765 +75766 POINT(41.28692774505963 74.92640266685301) bank75766 +75767 POINT(40.06009276396803 74.91281552244841) bank75767 +75768 POINT(39.74865432848368 73.27661878153647) bank75768 +75769 POINT(41.46866517809495 74.62910151422405) bank75769 +75770 POINT(41.4283706534444 74.26683367719073) bank75770 +75771 POINT(40.01058865432965 73.2546508560304) bank75771 +75772 POINT(41.612135696450785 74.41938824794103) bank75772 +75773 POINT(41.24248103805193 74.20666238971873) bank75773 +75774 POINT(41.06789604097539 73.01028624410047) bank75774 +75775 POINT(41.57850932596851 74.66350687500972) bank75775 +75776 POINT(41.23225877509965 73.73457209609104) bank75776 +75777 POINT(41.064733773198675 73.59661074692673) bank75777 +75778 POINT(41.01062904475329 73.73763567654576) bank75778 +75779 POINT(40.34875237941882 74.57518783519266) bank75779 +75780 POINT(40.176473140342935 73.60742473999574) bank75780 +75781 POINT(39.98827856320821 73.48493850583876) bank75781 +75782 POINT(41.33239160172489 73.89680323340737) bank75782 +75783 POINT(40.29735845634772 74.85892261548771) bank75783 +75784 POINT(40.13806844008286 74.5762101468654) bank75784 +75785 POINT(40.09909583757827 73.4701582939221) bank75785 +75786 POINT(40.904855944464416 74.78743628786788) bank75786 +75787 POINT(40.99526213263047 73.99802172196128) bank75787 +75788 POINT(41.09204582533827 74.16801471012518) bank75788 +75789 POINT(40.02516027084492 74.85214004075438) bank75789 +75790 POINT(40.96927998557416 74.37914210236015) bank75790 +75791 POINT(39.956809789993585 73.2141312013508) bank75791 +75792 POINT(39.822478761640554 74.44679822045792) bank75792 +75793 POINT(41.630803793257314 73.88085249647989) bank75793 +75794 POINT(41.46893588583137 73.12833031192497) bank75794 +75795 POINT(40.430721030224475 74.04186426841667) bank75795 +75796 POINT(41.648585342405575 73.96997925045383) bank75796 +75797 POINT(39.96152185285191 74.81471136951939) bank75797 +75798 POINT(40.68027718345064 73.69216998574589) bank75798 +75799 POINT(40.08973203750123 74.94792446011122) bank75799 +75800 POINT(40.75089137774527 73.82979698499781) bank75800 +75801 POINT(40.737115630741705 73.27422210343546) bank75801 +75802 POINT(41.534977795514145 73.96022313048472) bank75802 +75803 POINT(41.37579974142357 74.84979043554843) bank75803 +75804 POINT(40.70234635642124 73.13886111283651) bank75804 +75805 POINT(41.27533758019796 74.95340553568019) bank75805 +75806 POINT(41.35444746637361 73.29655303453437) bank75806 +75807 POINT(40.34531793101425 73.6179900106543) bank75807 +75808 POINT(39.826821823131255 73.81448766256278) bank75808 +75809 POINT(41.29269656455599 73.17950473277226) bank75809 +75810 POINT(41.61091591534874 73.19308121019402) bank75810 +75811 POINT(41.24577517526116 73.37244034127086) bank75811 +75812 POINT(40.22046687759585 74.82622920185293) bank75812 +75813 POINT(41.03014077253584 74.15941883789401) bank75813 +75814 POINT(41.544370617367285 74.09956303454199) bank75814 +75815 POINT(41.259371973832366 74.81467983689605) bank75815 +75816 POINT(41.00164037490001 73.87877888701031) bank75816 +75817 POINT(41.43748390487175 73.36370311919289) bank75817 +75818 POINT(41.46097456317013 74.6059814049046) bank75818 +75819 POINT(40.53314943926678 73.27001736815363) bank75819 +75820 POINT(40.36074811393228 73.69470823176363) bank75820 +75821 POINT(41.26709815557933 73.29730691333877) bank75821 +75822 POINT(40.77692987298722 73.64890314365643) bank75822 +75823 POINT(40.08332824072329 74.9080993987745) bank75823 +75824 POINT(39.88685362384737 74.71940332752467) bank75824 +75825 POINT(40.10342399721213 74.70665103673802) bank75825 +75826 POINT(40.60319483814444 73.47441799754844) bank75826 +75827 POINT(40.6248765104338 73.93511541926888) bank75827 +75828 POINT(41.037732369596156 74.96790411997739) bank75828 +75829 POINT(40.62689563851784 73.46377503062098) bank75829 +75830 POINT(39.96023113015003 74.3291528635456) bank75830 +75831 POINT(41.13665508507243 74.11241668411853) bank75831 +75832 POINT(40.833997195604184 73.73130222038931) bank75832 +75833 POINT(40.55908405957431 73.33960841336858) bank75833 +75834 POINT(41.01806562279517 74.80482781151517) bank75834 +75835 POINT(40.01101785701676 73.93652112909662) bank75835 +75836 POINT(40.868335362387185 73.07859419274175) bank75836 +75837 POINT(41.34602470084693 73.33530259106269) bank75837 +75838 POINT(41.23685506412511 73.53528210537263) bank75838 +75839 POINT(40.90986497124641 74.73125489341804) bank75839 +75840 POINT(40.564706148133546 74.23441380596049) bank75840 +75841 POINT(40.57407663772859 74.20679646663291) bank75841 +75842 POINT(40.65722186262408 74.30644860122314) bank75842 +75843 POINT(40.79887375743446 73.93794646749852) bank75843 +75844 POINT(41.050895833759306 73.54166647905315) bank75844 +75845 POINT(40.161058953918925 73.9977719921619) bank75845 +75846 POINT(39.77741687607511 74.57961552681326) bank75846 +75847 POINT(40.96925912397249 74.41367432273424) bank75847 +75848 POINT(40.35300678278631 74.82482081894206) bank75848 +75849 POINT(40.37145993648399 74.83146473452729) bank75849 +75850 POINT(40.62033740068676 73.44775936801354) bank75850 +75851 POINT(40.22051752081663 73.49502873769599) bank75851 +75852 POINT(39.8820047300243 73.51789856476445) bank75852 +75853 POINT(39.944146301852996 74.84027435021777) bank75853 +75854 POINT(41.113159536662664 73.44177951012396) bank75854 +75855 POINT(39.99538545966802 74.36806658532889) bank75855 +75856 POINT(41.39556924641652 74.53784767104996) bank75856 +75857 POINT(41.40859696755537 74.47830814988944) bank75857 +75858 POINT(40.115230055319365 73.11691642814046) bank75858 +75859 POINT(41.59925291698329 73.48468089835643) bank75859 +75860 POINT(40.89113412480826 73.64839077396985) bank75860 +75861 POINT(40.10030908516731 74.41245607321297) bank75861 +75862 POINT(40.742148628074006 74.72693829397778) bank75862 +75863 POINT(40.60390329766372 74.88472633379264) bank75863 +75864 POINT(40.4031037341324 73.8004809391924) bank75864 +75865 POINT(40.589760943231084 73.34946678657849) bank75865 +75866 POINT(40.537574663284836 73.43206536386208) bank75866 +75867 POINT(41.24191859191029 74.19557907941635) bank75867 +75868 POINT(39.82225510890582 74.91452758042244) bank75868 +75869 POINT(40.91320057724008 73.91683827301253) bank75869 +75870 POINT(41.07920782325965 74.24983946406788) bank75870 +75871 POINT(41.24249708735359 73.2716478126959) bank75871 +75872 POINT(40.24540506147923 74.18463485143452) bank75872 +75873 POINT(40.3289152604019 73.64277962364179) bank75873 +75874 POINT(40.076902972402884 74.87725167502965) bank75874 +75875 POINT(40.06925691119321 73.10078705131318) bank75875 +75876 POINT(40.280648936528884 74.1683970517113) bank75876 +75877 POINT(40.74562361884166 74.01276147301071) bank75877 +75878 POINT(40.042112351650175 74.6475477068463) bank75878 +75879 POINT(40.70128511163539 74.9272690643526) bank75879 +75880 POINT(41.088954903441 74.3325811468543) bank75880 +75881 POINT(41.0960012084498 74.39862325830107) bank75881 +75882 POINT(39.86942794097782 74.42513225513812) bank75882 +75883 POINT(40.74532936944211 73.22519099522813) bank75883 +75884 POINT(40.72934023016448 73.12562730804821) bank75884 +75885 POINT(40.83612270481494 74.79825190895602) bank75885 +75886 POINT(41.191231523816654 74.87162221334083) bank75886 +75887 POINT(41.69520010132669 73.35164374647779) bank75887 +75888 POINT(41.039383016179876 73.8949311708491) bank75888 +75889 POINT(41.02075486175125 74.8105693782688) bank75889 +75890 POINT(39.7357109232741 73.26065957707547) bank75890 +75891 POINT(40.899786813784296 73.63631895027933) bank75891 +75892 POINT(40.017057415952685 73.59968436878756) bank75892 +75893 POINT(41.61654637895165 74.26976480227916) bank75893 +75894 POINT(41.274747951814945 74.18285438219378) bank75894 +75895 POINT(40.22175236157292 74.07614556243512) bank75895 +75896 POINT(41.303271299882006 74.21091889554835) bank75896 +75897 POINT(40.46152157775849 73.8262362937327) bank75897 +75898 POINT(40.08083290327995 74.17507196061015) bank75898 +75899 POINT(41.32037507634463 74.32324633532708) bank75899 +75900 POINT(40.51338481662573 73.03003029759974) bank75900 +75901 POINT(41.539396722976306 73.21199467126092) bank75901 +75902 POINT(40.083468286919995 74.29668861789474) bank75902 +75903 POINT(40.82258253697527 74.45773957872746) bank75903 +75904 POINT(41.71263932291745 73.4947868915661) bank75904 +75905 POINT(41.49229011278815 73.30942341852769) bank75905 +75906 POINT(40.42643695545404 74.4961769703905) bank75906 +75907 POINT(41.532115217927284 74.91694909888663) bank75907 +75908 POINT(40.247587734929965 74.4004939149811) bank75908 +75909 POINT(41.23604867236911 74.9003352830432) bank75909 +75910 POINT(40.21097961110289 74.99144814253061) bank75910 +75911 POINT(41.19733565135074 73.98514914022995) bank75911 +75912 POINT(40.043574502452394 74.86399722825595) bank75912 +75913 POINT(39.87195816741294 74.71829449401744) bank75913 +75914 POINT(40.951796456220826 73.81312534596452) bank75914 +75915 POINT(41.432091619954086 73.56502238108665) bank75915 +75916 POINT(40.668210229509 73.61924974408397) bank75916 +75917 POINT(41.01475414551478 73.40217339780136) bank75917 +75918 POINT(41.6945849090462 73.55678730511438) bank75918 +75919 POINT(41.15315332227883 73.51163930959984) bank75919 +75920 POINT(41.64426754432191 73.88044099763096) bank75920 +75921 POINT(41.497248404450396 73.12617548583569) bank75921 +75922 POINT(39.885804837235426 74.55228214000292) bank75922 +75923 POINT(40.38822499090589 73.41631509956173) bank75923 +75924 POINT(40.926405524034145 73.66596981060067) bank75924 +75925 POINT(39.84035602464247 73.2798473901757) bank75925 +75926 POINT(39.978743985215154 74.246424083685) bank75926 +75927 POINT(40.87347129786264 73.23962881149379) bank75927 +75928 POINT(40.14363891446456 73.96533875359768) bank75928 +75929 POINT(41.43819780753439 74.29356974384513) bank75929 +75930 POINT(41.568594444548026 73.12686634996818) bank75930 +75931 POINT(41.51322038031577 74.96195669448419) bank75931 +75932 POINT(41.428928632068654 74.67724025942303) bank75932 +75933 POINT(39.88844214177722 74.1542300626574) bank75933 +75934 POINT(41.250698558162426 73.44217027799749) bank75934 +75935 POINT(41.69094156625895 74.73725762326012) bank75935 +75936 POINT(40.89971507793964 73.58255344537919) bank75936 +75937 POINT(41.39793098053615 73.75415932704291) bank75937 +75938 POINT(40.618512163774696 73.089092263557) bank75938 +75939 POINT(39.83879108692682 73.218557510368) bank75939 +75940 POINT(41.492645729036084 73.95033603062775) bank75940 +75941 POINT(41.60909986313243 74.8065820717548) bank75941 +75942 POINT(40.11653185151995 73.04508190812318) bank75942 +75943 POINT(40.86179703298373 73.24542582015528) bank75943 +75944 POINT(41.54852912863483 74.39773645604527) bank75944 +75945 POINT(40.083479593802096 74.30388947143834) bank75945 +75946 POINT(40.9064829755457 74.19860823046739) bank75946 +75947 POINT(41.4865697717071 73.314718376123) bank75947 +75948 POINT(40.06875318681543 73.09041339136482) bank75948 +75949 POINT(39.89274165878104 74.66762653402712) bank75949 +75950 POINT(40.919495198800675 73.3672060131335) bank75950 +75951 POINT(41.542052673087845 74.64155671311542) bank75951 +75952 POINT(40.25042970190237 73.4254224166335) bank75952 +75953 POINT(40.75944563936592 73.19678455316438) bank75953 +75954 POINT(40.96395929645148 73.92821022002187) bank75954 +75955 POINT(39.77302634530951 73.25761395171696) bank75955 +75956 POINT(40.233354088539954 73.36567453807253) bank75956 +75957 POINT(41.25421767429536 74.07558357985926) bank75957 +75958 POINT(40.474161171723985 73.27066282721957) bank75958 +75959 POINT(41.57276758138574 74.89464032072645) bank75959 +75960 POINT(40.71181411684325 74.7049832187616) bank75960 +75961 POINT(40.12683907736563 74.13162009034994) bank75961 +75962 POINT(39.76457809643051 73.20692690457379) bank75962 +75963 POINT(39.92286475760091 74.40774844864899) bank75963 +75964 POINT(39.819594687884035 74.56531183170127) bank75964 +75965 POINT(41.62419202620878 73.01123615769329) bank75965 +75966 POINT(40.42820127884033 74.76081601506569) bank75966 +75967 POINT(40.044842086902804 74.6551192623505) bank75967 +75968 POINT(41.33479747736755 73.76873557005288) bank75968 +75969 POINT(40.64166949007326 74.25022815759942) bank75969 +75970 POINT(40.472758453412766 74.95987637158183) bank75970 +75971 POINT(40.8789761931471 74.51155434354409) bank75971 +75972 POINT(39.841859523767944 74.17214325098153) bank75972 +75973 POINT(40.05002253234679 73.35887807466591) bank75973 +75974 POINT(41.44840925969661 73.99914333429193) bank75974 +75975 POINT(40.173366790541316 73.04777330622349) bank75975 +75976 POINT(39.965657327825554 74.1474861971894) bank75976 +75977 POINT(40.00173461898217 74.22181957894946) bank75977 +75978 POINT(39.99068710789821 73.34637553843426) bank75978 +75979 POINT(41.38724950539049 73.82464610944928) bank75979 +75980 POINT(39.873937332505484 73.78524018417008) bank75980 +75981 POINT(40.653993949091166 73.28536528212521) bank75981 +75982 POINT(40.24648754797708 74.315716397206) bank75982 +75983 POINT(40.554210458629356 73.42048694581443) bank75983 +75984 POINT(40.11830502656003 73.51216908663186) bank75984 +75985 POINT(40.6712598049535 73.53039959873905) bank75985 +75986 POINT(40.62982967583967 74.81445472990418) bank75986 +75987 POINT(40.28294579190942 74.42453320684056) bank75987 +75988 POINT(39.912903464333496 73.77299324741375) bank75988 +75989 POINT(39.970006125577136 74.25591132142075) bank75989 +75990 POINT(39.89482774863925 73.1340762358712) bank75990 +75991 POINT(41.01597727134941 73.4668599008273) bank75991 +75992 POINT(41.66560023784211 74.41167261719309) bank75992 +75993 POINT(40.55855773775598 73.42498299385062) bank75993 +75994 POINT(41.19996728586382 74.9931877950711) bank75994 +75995 POINT(41.28609899132823 74.94118347347153) bank75995 +75996 POINT(41.182562389065275 74.90291721218277) bank75996 +75997 POINT(41.42621251436155 74.29753982432521) bank75997 +75998 POINT(39.721795859846154 74.91713444971285) bank75998 +75999 POINT(41.18349912576454 73.6757514446513) bank75999 +76000 POINT(41.524861502383686 73.7683904044943) bank76000 +76001 POINT(41.501847159262056 73.41163529521175) bank76001 +76002 POINT(40.576963791801795 74.00612584020466) bank76002 +76003 POINT(40.00087098257502 74.47654193077219) bank76003 +76004 POINT(39.785665008120176 74.4573786483194) bank76004 +76005 POINT(39.96300441179789 74.29752436057245) bank76005 +76006 POINT(40.98089119367774 74.07477518664442) bank76006 +76007 POINT(40.69680949781711 73.215563410906) bank76007 +76008 POINT(39.95240649493757 74.2972016754639) bank76008 +76009 POINT(41.23253215853098 73.82212883935885) bank76009 +76010 POINT(39.97601451024376 73.07779935398526) bank76010 +76011 POINT(40.059392037861784 73.04045076769098) bank76011 +76012 POINT(40.086836716425466 73.27618011821876) bank76012 +76013 POINT(39.97783522396942 74.55668911349686) bank76013 +76014 POINT(41.44268825739319 74.47016516771112) bank76014 +76015 POINT(40.84880746405456 73.24945623526816) bank76015 +76016 POINT(41.12281602015604 73.03075035498351) bank76016 +76017 POINT(40.6481975776593 73.20335535272164) bank76017 +76018 POINT(40.82019373193969 74.09103079167727) bank76018 +76019 POINT(41.12686083659609 74.7931893629631) bank76019 +76020 POINT(41.56789956306865 74.97950646560975) bank76020 +76021 POINT(40.742749418178875 73.69827762261153) bank76021 +76022 POINT(40.072971514278784 73.12474207222506) bank76022 +76023 POINT(40.25443648833032 74.13834099093833) bank76023 +76024 POINT(40.44337296263777 73.60873162720074) bank76024 +76025 POINT(41.135727726944744 74.95031051431756) bank76025 +76026 POINT(40.209949946319746 73.03590836845484) bank76026 +76027 POINT(40.39001201308414 74.59071907435174) bank76027 +76028 POINT(41.02077780195352 73.62944445928316) bank76028 +76029 POINT(41.07601497707373 73.40841554687745) bank76029 +76030 POINT(41.60539261273747 74.4628309324933) bank76030 +76031 POINT(40.34410822135426 73.25822341048024) bank76031 +76032 POINT(40.334802578553436 74.93102634632757) bank76032 +76033 POINT(41.70015847125282 73.76006149381631) bank76033 +76034 POINT(40.18431640241851 74.86063697648743) bank76034 +76035 POINT(41.06364044648926 73.81769932435834) bank76035 +76036 POINT(41.107689361043455 74.3330919224702) bank76036 +76037 POINT(41.685560562222975 73.91187344351698) bank76037 +76038 POINT(40.33048295007988 73.03260204154188) bank76038 +76039 POINT(40.52405687097488 74.02880314056382) bank76039 +76040 POINT(39.97533834857528 74.9980717226639) bank76040 +76041 POINT(41.239799076098976 73.55816633731284) bank76041 +76042 POINT(39.90438592085207 73.51743482600502) bank76042 +76043 POINT(41.637766585838456 73.69706643333726) bank76043 +76044 POINT(40.20477629214584 74.32488719262116) bank76044 +76045 POINT(41.21621851748658 73.73156064222236) bank76045 +76046 POINT(40.984296684411376 73.65284757081407) bank76046 +76047 POINT(40.863636091246875 74.8795919637816) bank76047 +76048 POINT(39.841228379834526 74.79902337482763) bank76048 +76049 POINT(39.74988812621808 73.04377163005063) bank76049 +76050 POINT(41.647486866182106 74.72984322097797) bank76050 +76051 POINT(40.03623371712709 73.44059489863803) bank76051 +76052 POINT(41.682260040658676 74.25567963955982) bank76052 +76053 POINT(39.858478818079455 74.90784325315322) bank76053 +76054 POINT(41.46466564705216 73.13009472915705) bank76054 +76055 POINT(40.777134136030924 74.63407545192653) bank76055 +76056 POINT(40.45781461158663 74.9515118567343) bank76056 +76057 POINT(40.38697805602373 73.53202893514774) bank76057 +76058 POINT(40.493837115259105 73.91905191550602) bank76058 +76059 POINT(40.984635877710964 74.48212424626922) bank76059 +76060 POINT(40.16492147545275 73.25524070077519) bank76060 +76061 POINT(41.50714076394954 73.73490295771535) bank76061 +76062 POINT(41.47573313931885 74.883364276044) bank76062 +76063 POINT(40.28335314430026 74.14157813539921) bank76063 +76064 POINT(40.8354218083947 73.194419388603) bank76064 +76065 POINT(40.4456616072189 74.1929967350238) bank76065 +76066 POINT(41.627852940737235 74.85845614676347) bank76066 +76067 POINT(39.794604441778205 73.63302864925362) bank76067 +76068 POINT(41.56796587820208 73.61542500194189) bank76068 +76069 POINT(40.793285208570985 74.46029670434714) bank76069 +76070 POINT(40.814116344301574 73.48852201457713) bank76070 +76071 POINT(40.01847507570819 74.66573404118078) bank76071 +76072 POINT(41.17242022190974 73.57714387489699) bank76072 +76073 POINT(41.044176652257306 74.38688028575979) bank76073 +76074 POINT(40.649649692561425 74.47975386598722) bank76074 +76075 POINT(41.094226158838524 74.40670230027995) bank76075 +76076 POINT(39.908128613614934 73.96902319605795) bank76076 +76077 POINT(41.10732202722895 73.31133988756454) bank76077 +76078 POINT(40.533677591046064 73.68259090667688) bank76078 +76079 POINT(40.18483478338109 74.88502835327297) bank76079 +76080 POINT(40.03798247245051 73.9793811506594) bank76080 +76081 POINT(40.83139619856526 74.41731368553917) bank76081 +76082 POINT(40.21218945462238 74.12126453391535) bank76082 +76083 POINT(41.016434016011914 75.00317362758169) bank76083 +76084 POINT(39.859064618379264 74.57991325208593) bank76084 +76085 POINT(39.89612457365236 73.7929151225943) bank76085 +76086 POINT(41.009779048664406 73.75067411230809) bank76086 +76087 POINT(39.76328682342673 74.44386776290939) bank76087 +76088 POINT(40.18510354527065 74.79700670846229) bank76088 +76089 POINT(40.3647097015279 74.78651130151665) bank76089 +76090 POINT(41.06572491533848 74.7848787958794) bank76090 +76091 POINT(41.63215586167816 74.97234145036457) bank76091 +76092 POINT(40.78589926674373 74.93888964567125) bank76092 +76093 POINT(41.67229353702254 73.93902157314686) bank76093 +76094 POINT(40.31603446978577 74.12211323286694) bank76094 +76095 POINT(40.14007275340862 74.94884494145184) bank76095 +76096 POINT(41.34592503245656 74.85767383201974) bank76096 +76097 POINT(40.28834958201527 74.97829495499639) bank76097 +76098 POINT(40.16924845414277 73.32920257500798) bank76098 +76099 POINT(40.7255644252135 73.79593661328583) bank76099 +76100 POINT(39.81675732981495 74.49234235390914) bank76100 +76101 POINT(39.991306732961945 74.37736343820397) bank76101 +76102 POINT(41.42119612778182 73.40469743277393) bank76102 +76103 POINT(41.08991468959997 74.45503230293755) bank76103 +76104 POINT(40.23795567771883 74.7784330942031) bank76104 +76105 POINT(40.37565582816272 74.79066938446178) bank76105 +76106 POINT(41.27141655564415 74.2493402283989) bank76106 +76107 POINT(40.76081433127276 74.90991055657126) bank76107 +76108 POINT(41.55078361283277 73.90799697863852) bank76108 +76109 POINT(41.1783684578532 74.31073044080502) bank76109 +76110 POINT(41.12016041737463 73.02123296325816) bank76110 +76111 POINT(41.13876622039694 73.33962624467897) bank76111 +76112 POINT(41.5154820825215 74.05777036037489) bank76112 +76113 POINT(41.422472705122324 74.61888853887406) bank76113 +76114 POINT(41.22457288600887 74.69020288241325) bank76114 +76115 POINT(41.56580208969579 73.97480135663665) bank76115 +76116 POINT(41.53986095372024 73.77521196185086) bank76116 +76117 POINT(41.238884993055194 74.9193198281861) bank76117 +76118 POINT(40.3398330009541 74.81301585973125) bank76118 +76119 POINT(39.848257691058166 73.59204114346306) bank76119 +76120 POINT(39.82189888572941 74.67931587095345) bank76120 +76121 POINT(40.54534091229965 74.12190850183494) bank76121 +76122 POINT(41.0888225245187 73.19157284054796) bank76122 +76123 POINT(40.56853172798721 73.66576762050916) bank76123 +76124 POINT(41.27111448801945 74.67754423193463) bank76124 +76125 POINT(41.195987198317795 73.50922870135072) bank76125 +76126 POINT(40.460293751144455 73.96781965952479) bank76126 +76127 POINT(41.478747602111056 73.72860022439978) bank76127 +76128 POINT(41.146502525637395 73.75838907302614) bank76128 +76129 POINT(41.28488757529672 73.65443032242325) bank76129 +76130 POINT(40.828802326680176 74.62505420261988) bank76130 +76131 POINT(40.03383469084871 74.83464346582686) bank76131 +76132 POINT(41.01723465200326 73.29000739993198) bank76132 +76133 POINT(40.26044085605873 73.07996190500879) bank76133 +76134 POINT(40.01186683520268 73.52188739964815) bank76134 +76135 POINT(40.790013070960086 73.23713267694764) bank76135 +76136 POINT(40.90572421123257 73.74112219068573) bank76136 +76137 POINT(40.05466057240833 73.1749510274512) bank76137 +76138 POINT(39.732494215350606 73.44556695606117) bank76138 +76139 POINT(40.54193476663012 73.73825438442312) bank76139 +76140 POINT(40.600575991045055 74.22804924149992) bank76140 +76141 POINT(39.84935831074559 74.89587471644185) bank76141 +76142 POINT(39.93630530974494 74.73867929477068) bank76142 +76143 POINT(40.32775493887953 73.24983143578494) bank76143 +76144 POINT(40.2769610285511 74.71505644641697) bank76144 +76145 POINT(41.224383422233885 74.13444464315835) bank76145 +76146 POINT(41.3385239501811 74.30724156567265) bank76146 +76147 POINT(40.84911305498679 74.82045025263918) bank76147 +76148 POINT(40.81773510154213 73.8385051835018) bank76148 +76149 POINT(40.09126990217712 73.22045021409156) bank76149 +76150 POINT(41.642887003620785 74.64257972998655) bank76150 +76151 POINT(40.17998274005025 73.56898585079) bank76151 +76152 POINT(41.25891102649415 74.02459765918218) bank76152 +76153 POINT(41.33748753271925 74.5100701743514) bank76153 +76154 POINT(40.51801522319822 74.19952591282238) bank76154 +76155 POINT(40.61012208113361 73.58112248789031) bank76155 +76156 POINT(40.23706189037562 74.90214807009568) bank76156 +76157 POINT(39.95337341123757 73.82991613239967) bank76157 +76158 POINT(39.87235967715021 74.68210596589017) bank76158 +76159 POINT(40.09712526167978 74.58488030426557) bank76159 +76160 POINT(40.907212798375724 73.8427393455928) bank76160 +76161 POINT(40.12740529375144 74.83203891763493) bank76161 +76162 POINT(40.59754329279626 73.13572183571297) bank76162 +76163 POINT(41.207774794147035 74.31678328547113) bank76163 +76164 POINT(40.78034458278987 74.35876198006183) bank76164 +76165 POINT(41.10807314018234 73.46729925488451) bank76165 +76166 POINT(40.307649179892444 74.10697804743545) bank76166 +76167 POINT(40.48214472981119 74.57373680781026) bank76167 +76168 POINT(40.7538088837854 74.59145352610666) bank76168 +76169 POINT(39.78742387229379 73.95229114019801) bank76169 +76170 POINT(40.574479232895 74.05262671267691) bank76170 +76171 POINT(39.844704583331435 73.79876719085088) bank76171 +76172 POINT(41.22038086537768 74.34591851502252) bank76172 +76173 POINT(41.15152871244567 73.28452481400632) bank76173 +76174 POINT(40.605360990003845 73.78919202990751) bank76174 +76175 POINT(41.237642125333544 74.70810058126334) bank76175 +76176 POINT(41.58877032339268 73.71662925596647) bank76176 +76177 POINT(40.478547218799775 74.18656581497342) bank76177 +76178 POINT(39.786154293095905 73.85785804182056) bank76178 +76179 POINT(39.79831632752362 73.0413608844472) bank76179 +76180 POINT(40.71666595569839 73.38326232630634) bank76180 +76181 POINT(40.959957390121765 73.25873553058042) bank76181 +76182 POINT(40.378751814738784 73.75138944515372) bank76182 +76183 POINT(41.00312903445765 73.95226131432221) bank76183 +76184 POINT(40.03040894062803 73.56904567519098) bank76184 +76185 POINT(40.54034475646508 73.50126371184479) bank76185 +76186 POINT(40.53905537263949 74.84362499121147) bank76186 +76187 POINT(41.440049939691974 74.13812944374492) bank76187 +76188 POINT(40.71728438496793 74.43073560176397) bank76188 +76189 POINT(40.527754792907885 74.13339690149418) bank76189 +76190 POINT(41.259199625737274 74.87203621767642) bank76190 +76191 POINT(41.01075918765801 73.92351840718894) bank76191 +76192 POINT(40.63320301742093 73.07022565808589) bank76192 +76193 POINT(40.58031527699814 74.56103757686405) bank76193 +76194 POINT(40.62348309643191 74.12721589174586) bank76194 +76195 POINT(40.065311723826404 74.81220327915399) bank76195 +76196 POINT(40.94930586020425 74.3184767398196) bank76196 +76197 POINT(40.19210664148048 73.55699559412362) bank76197 +76198 POINT(40.78441716073011 74.79701604067813) bank76198 +76199 POINT(40.75384653228106 74.70758011100646) bank76199 +76200 POINT(40.82515193084328 74.65387713627139) bank76200 +76201 POINT(40.647682280114196 74.13342480744286) bank76201 +76202 POINT(40.955707956778895 73.44786628274264) bank76202 +76203 POINT(40.58282548636798 74.85045497313821) bank76203 +76204 POINT(41.239059193766096 73.74151915589411) bank76204 +76205 POINT(40.537184168087705 74.2364297409604) bank76205 +76206 POINT(41.70204017142086 73.60739814790608) bank76206 +76207 POINT(41.66195372707952 74.67031510138855) bank76207 +76208 POINT(40.02681299629972 74.00538466539018) bank76208 +76209 POINT(39.75617770817283 73.93439717140112) bank76209 +76210 POINT(39.78840047865814 73.69398429412432) bank76210 +76211 POINT(41.45895835196004 74.26210256621555) bank76211 +76212 POINT(40.08074874797916 74.87655836242955) bank76212 +76213 POINT(41.05672050380956 74.72834504864007) bank76213 +76214 POINT(41.285979579327 74.11728933542418) bank76214 +76215 POINT(40.799322410651826 73.8230160794936) bank76215 +76216 POINT(40.42769360888732 74.42074291704529) bank76216 +76217 POINT(41.24195478014798 73.01827671734357) bank76217 +76218 POINT(40.40700288797028 73.16915155731954) bank76218 +76219 POINT(40.49219161711909 74.36629997426724) bank76219 +76220 POINT(41.31427591181376 74.68210678197457) bank76220 +76221 POINT(40.26286023406309 73.89512640116496) bank76221 +76222 POINT(40.84380522438319 73.84418433655144) bank76222 +76223 POINT(39.8856490461802 73.09018082369076) bank76223 +76224 POINT(41.25112035468973 74.97939122767728) bank76224 +76225 POINT(40.22260513856572 73.35111560655864) bank76225 +76226 POINT(41.21850884796839 73.9678030394722) bank76226 +76227 POINT(41.32285068462857 73.40791713071407) bank76227 +76228 POINT(41.25462791745708 74.21452521399223) bank76228 +76229 POINT(40.59269156760575 73.7624098943545) bank76229 +76230 POINT(40.85711743130982 74.40778887693646) bank76230 +76231 POINT(41.42825557870211 73.44111551771414) bank76231 +76232 POINT(40.70707202711223 74.74586699772902) bank76232 +76233 POINT(41.445470012757866 74.10530555056432) bank76233 +76234 POINT(40.30802804901479 73.28457925693938) bank76234 +76235 POINT(40.598112552437264 74.75004523309471) bank76235 +76236 POINT(41.115528433459055 73.20430307932541) bank76236 +76237 POINT(40.33899063171631 73.2980059600357) bank76237 +76238 POINT(41.162366025870696 73.44705712503958) bank76238 +76239 POINT(40.36753693432931 73.07501032878253) bank76239 +76240 POINT(41.38926665126574 74.69274459339609) bank76240 +76241 POINT(40.000500886006975 73.98263164170469) bank76241 +76242 POINT(40.923707440236434 74.63182349989799) bank76242 +76243 POINT(39.979652831188815 74.51522975309013) bank76243 +76244 POINT(41.15906529625559 74.42725015956282) bank76244 +76245 POINT(40.929480876525766 73.42939027242899) bank76245 +76246 POINT(40.25098665449248 74.49696740759111) bank76246 +76247 POINT(40.33447953774457 74.2863973187145) bank76247 +76248 POINT(41.04644330620234 74.28407773023622) bank76248 +76249 POINT(40.30557272161664 73.3955506618983) bank76249 +76250 POINT(41.294571164346564 73.89854334916501) bank76250 +76251 POINT(39.7481610198759 73.4877108067754) bank76251 +76252 POINT(39.77703629861989 74.49014841985702) bank76252 +76253 POINT(40.195583981168745 74.99848015344779) bank76253 +76254 POINT(41.37367030330845 73.72896610697312) bank76254 +76255 POINT(40.72786053865097 74.37445263412391) bank76255 +76256 POINT(41.00022911120062 73.08940791982363) bank76256 +76257 POINT(41.050825612110245 74.84350744317533) bank76257 +76258 POINT(41.21435113799457 74.67708855930536) bank76258 +76259 POINT(41.175488284499174 74.33216794261419) bank76259 +76260 POINT(41.49156477831877 73.60580295880027) bank76260 +76261 POINT(41.68241051335796 73.64424496260303) bank76261 +76262 POINT(40.31519396703341 74.87288037235444) bank76262 +76263 POINT(40.38277508889502 73.25612923716974) bank76263 +76264 POINT(41.31329013496525 74.1592392242366) bank76264 +76265 POINT(41.22248815394258 74.66606051804195) bank76265 +76266 POINT(40.52671575481292 73.41248583547717) bank76266 +76267 POINT(41.63639742815247 74.521542500706) bank76267 +76268 POINT(41.1350576941815 74.08226712019477) bank76268 +76269 POINT(39.72872293232014 74.67148670263647) bank76269 +76270 POINT(39.79735954701756 73.93217240642973) bank76270 +76271 POINT(40.155545771505125 73.55167214651372) bank76271 +76272 POINT(40.38311990235325 73.62873417851522) bank76272 +76273 POINT(41.382790494414834 73.50018000873654) bank76273 +76274 POINT(40.96420728514536 74.9156889170679) bank76274 +76275 POINT(41.006363338177636 74.79404503356251) bank76275 +76276 POINT(40.737867461090545 73.69870136288334) bank76276 +76277 POINT(41.488329151593014 74.43460451529485) bank76277 +76278 POINT(41.37417323189889 74.83296049342597) bank76278 +76279 POINT(40.84539634230777 74.8734101111516) bank76279 +76280 POINT(41.19398655819987 73.04037799350127) bank76280 +76281 POINT(39.8835465401766 74.86324161272965) bank76281 +76282 POINT(41.488873162775626 73.30364571315182) bank76282 +76283 POINT(40.851371067518606 74.23871028972832) bank76283 +76284 POINT(40.32327763950497 73.64629375650549) bank76284 +76285 POINT(41.26771014410783 74.43348737209429) bank76285 +76286 POINT(40.79659946123397 74.83873115338325) bank76286 +76287 POINT(41.07376023575788 74.08884482284503) bank76287 +76288 POINT(40.54610231776617 73.28816831903804) bank76288 +76289 POINT(39.813255160120946 74.14699546470624) bank76289 +76290 POINT(40.65248903856235 73.0828003857085) bank76290 +76291 POINT(41.67235061170759 73.6216121264134) bank76291 +76292 POINT(40.875071728604766 74.13976235068601) bank76292 +76293 POINT(41.46827316619668 74.75772015364527) bank76293 +76294 POINT(40.5537990633996 74.90366895838078) bank76294 +76295 POINT(40.17914869439993 74.87467403320863) bank76295 +76296 POINT(41.13810916990222 73.59508256571162) bank76296 +76297 POINT(40.709532095407596 73.90942373500879) bank76297 +76298 POINT(39.92009017900672 74.61290918748756) bank76298 +76299 POINT(41.44785279007336 74.92278293687198) bank76299 +76300 POINT(40.61597904393699 73.98556051779323) bank76300 +76301 POINT(41.547595966047595 73.11951584546377) bank76301 +76302 POINT(41.42770663709481 73.02052939665231) bank76302 +76303 POINT(39.73219078986777 73.03370434927508) bank76303 +76304 POINT(40.71349519076023 73.87555420828927) bank76304 +76305 POINT(40.18927007597982 73.23462455862338) bank76305 +76306 POINT(41.303046627675734 74.82451162441285) bank76306 +76307 POINT(40.06148728768142 74.03657566448639) bank76307 +76308 POINT(40.122017419068676 73.50074343619039) bank76308 +76309 POINT(40.42428146866167 74.52220346263258) bank76309 +76310 POINT(39.94304187847788 73.3726190665264) bank76310 +76311 POINT(40.383950791149886 74.75639488215438) bank76311 +76312 POINT(40.60212566763884 73.31750738846995) bank76312 +76313 POINT(41.67724816192285 73.79414761691736) bank76313 +76314 POINT(40.828015589283716 73.43532223783042) bank76314 +76315 POINT(40.717494676223126 73.46179821526553) bank76315 +76316 POINT(41.13470008448098 74.74929825606256) bank76316 +76317 POINT(40.14406941991852 73.79433438018916) bank76317 +76318 POINT(41.08219032197644 74.4949322991163) bank76318 +76319 POINT(40.55827332016811 74.44539653316022) bank76319 +76320 POINT(40.33723881472508 74.62277318374441) bank76320 +76321 POINT(40.4803123349871 73.20036855748315) bank76321 +76322 POINT(41.65937283554223 73.3174455479665) bank76322 +76323 POINT(41.522047435146064 74.3969207666778) bank76323 +76324 POINT(40.22565128075497 73.58283143928303) bank76324 +76325 POINT(41.43912159179344 74.92640049321845) bank76325 +76326 POINT(41.149721832160644 74.71021539413074) bank76326 +76327 POINT(41.67946358070215 74.81715546444973) bank76327 +76328 POINT(40.61052092965543 73.80085456941582) bank76328 +76329 POINT(40.49801172860795 74.27411425112471) bank76329 +76330 POINT(39.91630573949983 73.1298110766278) bank76330 +76331 POINT(40.56783775212678 73.98977581380532) bank76331 +76332 POINT(40.9336098783677 73.6063984991557) bank76332 +76333 POINT(41.69121484413193 74.74342673519028) bank76333 +76334 POINT(41.60933847746316 74.43765277676623) bank76334 +76335 POINT(40.86240378131191 73.73306316800081) bank76335 +76336 POINT(40.40441284841722 74.14397134693792) bank76336 +76337 POINT(41.52156393417912 74.90237806094844) bank76337 +76338 POINT(41.114300586897166 73.1477597403777) bank76338 +76339 POINT(41.5193588930511 74.78448566693704) bank76339 +76340 POINT(40.93411745226074 73.78137253771776) bank76340 +76341 POINT(40.43918758406067 74.47232439521615) bank76341 +76342 POINT(41.05573883216679 73.78468504340428) bank76342 +76343 POINT(40.962917063715146 73.95044747703685) bank76343 +76344 POINT(41.384220198366826 74.16665125373129) bank76344 +76345 POINT(41.4505976493148 73.74304403710072) bank76345 +76346 POINT(40.46442913416431 74.43019877556013) bank76346 +76347 POINT(40.34806384597672 73.35882344475964) bank76347 +76348 POINT(40.337018576530795 74.24876821541667) bank76348 +76349 POINT(41.128432353068284 74.49329081650416) bank76349 +76350 POINT(41.027298570032606 73.43849752033603) bank76350 +76351 POINT(40.40278571230309 73.55993051454917) bank76351 +76352 POINT(41.06759472287132 74.52591955577243) bank76352 +76353 POINT(40.192173569035035 73.04469537834402) bank76353 +76354 POINT(41.27081474144471 74.82973328899595) bank76354 +76355 POINT(41.258378028607616 74.6795398695251) bank76355 +76356 POINT(40.687273758686956 74.17104206350034) bank76356 +76357 POINT(40.69992510141938 73.17741264874483) bank76357 +76358 POINT(41.363600571710705 74.0579897530906) bank76358 +76359 POINT(41.28484981286314 73.71193627211646) bank76359 +76360 POINT(41.3195182162862 74.38881612237485) bank76360 +76361 POINT(41.5359329773254 74.32449382289562) bank76361 +76362 POINT(39.82348054893211 73.27129427824242) bank76362 +76363 POINT(40.74419000650677 74.45553784300941) bank76363 +76364 POINT(40.82851605493481 73.39669509372736) bank76364 +76365 POINT(39.809168544931275 73.32760619107812) bank76365 +76366 POINT(40.97762659069692 74.46544943680033) bank76366 +76367 POINT(41.466298326721166 74.81351233376347) bank76367 +76368 POINT(40.814855775940615 73.09441475481448) bank76368 +76369 POINT(40.176819118887416 74.14147976239971) bank76369 +76370 POINT(39.951026164266764 74.39391042161351) bank76370 +76371 POINT(40.48094761661116 74.89784070332615) bank76371 +76372 POINT(40.50030676038324 74.27520152202237) bank76372 +76373 POINT(40.763768733908684 74.89029696491441) bank76373 +76374 POINT(40.83657090237249 73.1547939999326) bank76374 +76375 POINT(40.54024403038839 73.2571884081121) bank76375 +76376 POINT(40.84082657825642 73.03382322600554) bank76376 +76377 POINT(40.823035332367226 73.50749321628754) bank76377 +76378 POINT(41.25293253979777 73.50215397687052) bank76378 +76379 POINT(40.6266478339967 73.80515264220038) bank76379 +76380 POINT(40.167537514917434 74.94374980304264) bank76380 +76381 POINT(39.83447191837493 73.25815274700419) bank76381 +76382 POINT(39.84092759260836 74.32573773799892) bank76382 +76383 POINT(41.667572813229754 74.71016400310648) bank76383 +76384 POINT(40.15191320153377 74.14081628806969) bank76384 +76385 POINT(41.30139296150466 73.08826433330482) bank76385 +76386 POINT(40.4534233550828 73.24051902853834) bank76386 +76387 POINT(39.75779319137249 73.81210936654408) bank76387 +76388 POINT(41.62994204400451 73.69069640985622) bank76388 +76389 POINT(40.172548051571084 73.13328571179038) bank76389 +76390 POINT(40.10042232195796 73.61147757069624) bank76390 +76391 POINT(40.950039659004474 74.31635016828166) bank76391 +76392 POINT(41.39511603440592 74.47095748654407) bank76392 +76393 POINT(39.9807681950633 74.22680194932657) bank76393 +76394 POINT(40.99737535570277 73.64864977896502) bank76394 +76395 POINT(40.64830837021971 73.7171832146276) bank76395 +76396 POINT(41.66925477590624 73.0096642199276) bank76396 +76397 POINT(39.81876348152988 74.10744319969493) bank76397 +76398 POINT(40.45810630185316 74.3797498354131) bank76398 +76399 POINT(40.86711787839246 73.08808711296273) bank76399 +76400 POINT(40.207229336611846 74.2991220178084) bank76400 +76401 POINT(40.41126236194035 74.93286999977647) bank76401 +76402 POINT(41.08471621882624 74.91125993994062) bank76402 +76403 POINT(40.37545355439452 74.79172847811397) bank76403 +76404 POINT(41.558757401560534 73.32924443779174) bank76404 +76405 POINT(39.74330947881757 73.16365735156009) bank76405 +76406 POINT(39.99885000240214 73.41823322540533) bank76406 +76407 POINT(41.1587994818009 74.03316015831965) bank76407 +76408 POINT(41.33831082328474 74.8436172398148) bank76408 +76409 POINT(39.72021182522251 74.06377734361922) bank76409 +76410 POINT(40.43331748531403 74.33073227454528) bank76410 +76411 POINT(40.467668529527145 74.84522797137058) bank76411 +76412 POINT(40.41255457262397 73.10368886104247) bank76412 +76413 POINT(40.48750071078124 73.02314122281261) bank76413 +76414 POINT(40.35878878657414 73.74547778144812) bank76414 +76415 POINT(40.93343309535157 74.42272833041126) bank76415 +76416 POINT(41.0788118744069 74.51071665333734) bank76416 +76417 POINT(40.40935802544571 73.56319296169345) bank76417 +76418 POINT(40.934640875403694 74.58988789527368) bank76418 +76419 POINT(40.333683785434204 74.87181348640033) bank76419 +76420 POINT(40.070083227908185 73.31477604770889) bank76420 +76421 POINT(39.89994344736836 73.99373031095566) bank76421 +76422 POINT(41.57306603596362 74.94385094310867) bank76422 +76423 POINT(41.625367133492254 74.97186402583961) bank76423 +76424 POINT(39.80882588318263 74.01189330596593) bank76424 +76425 POINT(40.51568244651215 74.30152137403212) bank76425 +76426 POINT(41.0924517383378 74.37061025183458) bank76426 +76427 POINT(39.86106854892195 74.1647121809042) bank76427 +76428 POINT(39.92567521527996 74.92161831126475) bank76428 +76429 POINT(40.71865229006492 74.53734825804258) bank76429 +76430 POINT(41.37705956358265 74.02702275355225) bank76430 +76431 POINT(40.34576451327629 74.17351986332287) bank76431 +76432 POINT(40.2046116863489 73.32962371523513) bank76432 +76433 POINT(39.79754863471267 73.41740741551378) bank76433 +76434 POINT(40.183256158984605 73.57615811132618) bank76434 +76435 POINT(40.29403904062032 74.96043626711612) bank76435 +76436 POINT(40.36884866659911 73.74672393855806) bank76436 +76437 POINT(41.70678318062007 73.91655983493932) bank76437 +76438 POINT(41.61215632141078 74.28106161637214) bank76438 +76439 POINT(39.904171416586536 74.97682950292221) bank76439 +76440 POINT(41.44182305366233 74.16521537283984) bank76440 +76441 POINT(40.03742114606126 73.44266257431228) bank76441 +76442 POINT(39.989688309443686 74.77447319055885) bank76442 +76443 POINT(40.06898745314328 73.07787221045129) bank76443 +76444 POINT(40.052990045169864 73.47560379418272) bank76444 +76445 POINT(40.29385885268871 73.17208167804344) bank76445 +76446 POINT(40.07274492838026 73.70501574413494) bank76446 +76447 POINT(41.221353028683104 74.09836606954727) bank76447 +76448 POINT(41.2606814008099 73.35800869273552) bank76448 +76449 POINT(39.95164406516637 73.92678414606148) bank76449 +76450 POINT(40.33109493722577 74.69649420972388) bank76450 +76451 POINT(40.75282953868892 73.53702372551275) bank76451 +76452 POINT(39.85359934185286 73.45455635276166) bank76452 +76453 POINT(41.48382624576497 73.68095547148995) bank76453 +76454 POINT(41.388119503042056 73.1012675708043) bank76454 +76455 POINT(39.77715517616483 73.72869809977028) bank76455 +76456 POINT(41.38701938650274 74.4900142763112) bank76456 +76457 POINT(40.67714060671752 74.24651418274516) bank76457 +76458 POINT(40.721724007285054 73.65549685005699) bank76458 +76459 POINT(40.33875620148307 73.7975810124562) bank76459 +76460 POINT(40.43682870532161 74.50454351527705) bank76460 +76461 POINT(41.704222401660886 73.72364090959428) bank76461 +76462 POINT(41.26580416077923 73.35658335064221) bank76462 +76463 POINT(41.47586831072416 73.64170917348847) bank76463 +76464 POINT(41.037176593352854 74.08503167730665) bank76464 +76465 POINT(41.27903902127755 74.77662784617671) bank76465 +76466 POINT(40.3965472235685 73.81247477175364) bank76466 +76467 POINT(41.46230062315459 73.94062304156587) bank76467 +76468 POINT(41.068433171693044 74.56155313946148) bank76468 +76469 POINT(41.38701128050638 74.81206464595014) bank76469 +76470 POINT(41.00430243122712 74.56710426867701) bank76470 +76471 POINT(39.9170759200399 73.34368988934692) bank76471 +76472 POINT(41.10435130859953 74.65990024484417) bank76472 +76473 POINT(41.29593059743868 73.71259164664394) bank76473 +76474 POINT(41.62709761606459 73.32465273274337) bank76474 +76475 POINT(40.96244180227524 74.65346071696742) bank76475 +76476 POINT(39.98824254492917 73.20726338153189) bank76476 +76477 POINT(40.48432033453033 74.50989455344477) bank76477 +76478 POINT(41.6509823772458 73.40324690804914) bank76478 +76479 POINT(41.491925782287815 73.6032852757128) bank76479 +76480 POINT(39.965724407165276 73.8558111593477) bank76480 +76481 POINT(41.313149864194074 73.95527952251636) bank76481 +76482 POINT(41.168646265333194 74.01247983595172) bank76482 +76483 POINT(40.66378364007217 74.98794954833343) bank76483 +76484 POINT(40.86773896674561 73.98419418528945) bank76484 +76485 POINT(39.939501460697464 74.16116587678474) bank76485 +76486 POINT(39.8004888000744 74.33371831494048) bank76486 +76487 POINT(41.07359916891436 74.95852991536731) bank76487 +76488 POINT(41.521727489794706 73.18496182858121) bank76488 +76489 POINT(41.11727863637616 73.42020654471163) bank76489 +76490 POINT(40.64803603052564 73.08103731919914) bank76490 +76491 POINT(41.01726406946 73.97645358521424) bank76491 +76492 POINT(39.73963977016086 73.58745970272345) bank76492 +76493 POINT(40.2574198956394 73.24513360124713) bank76493 +76494 POINT(40.56049095077483 74.06609312482732) bank76494 +76495 POINT(41.276609781908924 73.48850213318032) bank76495 +76496 POINT(40.23229997977688 74.04205195906172) bank76496 +76497 POINT(40.251232930509644 73.83634916529599) bank76497 +76498 POINT(41.018202328034214 73.90062582667147) bank76498 +76499 POINT(40.02112244001428 74.29982986341142) bank76499 +76500 POINT(40.083546207885455 74.14230501242263) bank76500 +76501 POINT(40.06313805101949 74.59932882085907) bank76501 +76502 POINT(41.04795939174848 74.31959946576113) bank76502 +76503 POINT(41.64330633001324 74.12187711868245) bank76503 +76504 POINT(41.20312888383884 73.41481712333774) bank76504 +76505 POINT(40.18455373272753 74.37271888196369) bank76505 +76506 POINT(41.162360317887504 73.68229459675698) bank76506 +76507 POINT(40.66745503737853 73.62503210683539) bank76507 +76508 POINT(41.12064919117287 73.63632550975161) bank76508 +76509 POINT(39.74294433912926 74.90407747275837) bank76509 +76510 POINT(40.59964410623559 75.00553705028123) bank76510 +76511 POINT(40.083555372603044 73.22689596638587) bank76511 +76512 POINT(40.6263070618096 74.34221088781955) bank76512 +76513 POINT(41.30395444729705 73.62262372838008) bank76513 +76514 POINT(40.875496969705566 73.65642427257649) bank76514 +76515 POINT(40.932570630109744 74.47865598361125) bank76515 +76516 POINT(41.17028265383703 74.19721347227551) bank76516 +76517 POINT(40.968421543752335 73.53645635909315) bank76517 +76518 POINT(41.57461127378008 73.66701165348276) bank76518 +76519 POINT(40.48851211763713 73.26322407610411) bank76519 +76520 POINT(39.95705310558641 73.42594892888629) bank76520 +76521 POINT(40.63000389208321 74.91806856678687) bank76521 +76522 POINT(41.501900056409816 73.19869904916968) bank76522 +76523 POINT(40.26898229839447 73.46705210434313) bank76523 +76524 POINT(39.87078892681942 73.67013109119297) bank76524 +76525 POINT(41.15157288439371 73.69998944738656) bank76525 +76526 POINT(40.79008858962914 74.81172121188649) bank76526 +76527 POINT(40.52347809890652 74.40546002611575) bank76527 +76528 POINT(41.23627651475632 74.16147744659762) bank76528 +76529 POINT(40.12878474290813 74.64497025843792) bank76529 +76530 POINT(41.6460818636771 74.4820947136463) bank76530 +76531 POINT(40.6648141710074 73.56527143827381) bank76531 +76532 POINT(40.344964072452996 74.33269008067752) bank76532 +76533 POINT(41.56174790276287 73.41350234354962) bank76533 +76534 POINT(41.301825365327126 73.37639024345538) bank76534 +76535 POINT(40.87642232578012 73.53272553647474) bank76535 +76536 POINT(39.80615671574567 74.95880286271125) bank76536 +76537 POINT(39.894477586878544 74.29957907229524) bank76537 +76538 POINT(40.944461045053586 74.63221738592523) bank76538 +76539 POINT(41.676618646316655 73.16477937360978) bank76539 +76540 POINT(39.905437667734866 74.12270743296536) bank76540 +76541 POINT(40.282846495689846 74.18709112943806) bank76541 +76542 POINT(41.48357005979745 74.15800820776685) bank76542 +76543 POINT(41.349176139299246 74.24257743817532) bank76543 +76544 POINT(40.83431881820924 74.15284565768555) bank76544 +76545 POINT(41.46226933496019 74.10950995392984) bank76545 +76546 POINT(41.47955810414206 74.3677566276017) bank76546 +76547 POINT(41.12988061204781 73.09315135489214) bank76547 +76548 POINT(41.712776274090125 73.73862844208759) bank76548 +76549 POINT(40.146394052937545 74.95871393010586) bank76549 +76550 POINT(41.42315905926041 73.3406145377268) bank76550 +76551 POINT(41.507182537629795 74.17283303187357) bank76551 +76552 POINT(40.4887418918306 73.31748038962814) bank76552 +76553 POINT(40.398667422884806 73.64385681472832) bank76553 +76554 POINT(39.89288069720032 74.14603136929587) bank76554 +76555 POINT(40.794471150354184 73.73525916028984) bank76555 +76556 POINT(41.354407346428985 74.20421132275546) bank76556 +76557 POINT(41.00962419051337 73.04464482015105) bank76557 +76558 POINT(40.21347013395296 74.41896700040718) bank76558 +76559 POINT(40.238812806004695 73.56613616623612) bank76559 +76560 POINT(39.967492281002 73.10102585190053) bank76560 +76561 POINT(41.08023297923839 74.57512227062234) bank76561 +76562 POINT(40.87986995851746 73.85923591116008) bank76562 +76563 POINT(41.170716791731415 73.91604769211938) bank76563 +76564 POINT(40.01385146408735 73.12764748198775) bank76564 +76565 POINT(40.240157238096764 73.03633192073407) bank76565 +76566 POINT(40.424069133694765 73.22574923449491) bank76566 +76567 POINT(40.28907786185771 74.16058724006794) bank76567 +76568 POINT(41.56688916035676 74.73205830911182) bank76568 +76569 POINT(40.124285572673394 73.14787045842387) bank76569 +76570 POINT(40.8623343742643 73.15651321424998) bank76570 +76571 POINT(40.74112260712239 73.33544936259662) bank76571 +76572 POINT(40.63008079563821 74.6160461826078) bank76572 +76573 POINT(41.586926538865654 73.92553204202987) bank76573 +76574 POINT(41.10130381554464 74.39750513420462) bank76574 +76575 POINT(40.03238652013319 74.62759522053692) bank76575 +76576 POINT(41.383877682779115 73.15936753857848) bank76576 +76577 POINT(41.6574161348536 73.3299859477158) bank76577 +76578 POINT(39.77110031806581 73.06118776928862) bank76578 +76579 POINT(40.57855405638085 74.85098834759339) bank76579 +76580 POINT(40.79207545462769 73.61342881386618) bank76580 +76581 POINT(39.89155738604456 73.71018529591481) bank76581 +76582 POINT(41.14756683233172 74.83805973662994) bank76582 +76583 POINT(40.304425788329986 73.6382641487409) bank76583 +76584 POINT(41.187596925673624 74.79298029833475) bank76584 +76585 POINT(41.126774118662816 73.09172292663536) bank76585 +76586 POINT(41.509702367056015 73.44907517010726) bank76586 +76587 POINT(40.344197755721645 74.92300490742771) bank76587 +76588 POINT(41.04787224923561 74.43171734697658) bank76588 +76589 POINT(40.44493092989302 73.37704890530507) bank76589 +76590 POINT(41.15711201383768 73.78953968488159) bank76590 +76591 POINT(41.10845355539318 74.61556350632014) bank76591 +76592 POINT(41.36803854921188 74.33638258578262) bank76592 +76593 POINT(41.143356492463404 73.95213862327458) bank76593 +76594 POINT(40.599656713617506 73.83577468622286) bank76594 +76595 POINT(41.68872212833399 74.18018615929923) bank76595 +76596 POINT(39.920240372258746 74.41067638153073) bank76596 +76597 POINT(41.52646424079576 74.61369605999862) bank76597 +76598 POINT(41.40267099776798 74.49583856330757) bank76598 +76599 POINT(40.534255227639214 73.95644153421702) bank76599 +76600 POINT(40.2416795679802 74.54308118214288) bank76600 +76601 POINT(41.28025425878958 73.92849825831179) bank76601 +76602 POINT(41.21066945144867 73.40021510050035) bank76602 +76603 POINT(41.297367469171945 73.30553025683022) bank76603 +76604 POINT(41.05844157299932 73.35937308765477) bank76604 +76605 POINT(40.24679105909763 74.71628016880634) bank76605 +76606 POINT(40.657926311551364 74.670236129087) bank76606 +76607 POINT(40.565792627381484 73.2097456223721) bank76607 +76608 POINT(40.43978403648369 74.21362793258237) bank76608 +76609 POINT(41.201640213557646 74.31828852036578) bank76609 +76610 POINT(41.02384482278326 73.62890307928576) bank76610 +76611 POINT(40.253326145453656 73.41065707166683) bank76611 +76612 POINT(41.41899837193002 73.76898243854178) bank76612 +76613 POINT(40.737033700190366 73.06602956629054) bank76613 +76614 POINT(40.44742283658728 73.07097627658406) bank76614 +76615 POINT(40.047444844551094 74.04475486097574) bank76615 +76616 POINT(40.731368406545656 73.63798235741733) bank76616 +76617 POINT(39.7618120252075 73.3991949337942) bank76617 +76618 POINT(40.10759635267763 73.84935527503974) bank76618 +76619 POINT(41.66753789223821 74.25554929369332) bank76619 +76620 POINT(40.05921759278724 74.31577330361525) bank76620 +76621 POINT(41.21134863236947 73.37460375062106) bank76621 +76622 POINT(40.73490938024024 73.60209506320186) bank76622 +76623 POINT(40.78007062309622 74.936500865742) bank76623 +76624 POINT(40.95205845175236 74.58914894238448) bank76624 +76625 POINT(39.909633601326334 74.56399837045667) bank76625 +76626 POINT(40.27071039347263 73.37236274407287) bank76626 +76627 POINT(40.70646625393629 73.64120316225659) bank76627 +76628 POINT(40.68033915080442 73.49189859233219) bank76628 +76629 POINT(39.81855177766059 73.35534012272089) bank76629 +76630 POINT(40.815536021379316 73.64784165637865) bank76630 +76631 POINT(40.97440535164851 73.33051237306769) bank76631 +76632 POINT(40.466630735174604 74.96932769624587) bank76632 +76633 POINT(39.90182405245783 74.75623926245693) bank76633 +76634 POINT(40.31159197574626 73.30188708864333) bank76634 +76635 POINT(39.79741736109644 73.75467294680244) bank76635 +76636 POINT(40.06048433476521 73.92556929930313) bank76636 +76637 POINT(39.775815574698356 73.21757792553035) bank76637 +76638 POINT(41.04053484343072 74.38921149638509) bank76638 +76639 POINT(41.52021793720666 73.23154301783048) bank76639 +76640 POINT(39.77205337363927 74.25318037008113) bank76640 +76641 POINT(41.236771479808944 73.59779482885443) bank76641 +76642 POINT(40.216022037731015 73.40764445906524) bank76642 +76643 POINT(40.219300056632164 73.14324268563531) bank76643 +76644 POINT(40.138037182707656 74.10564350193047) bank76644 +76645 POINT(41.144316252724344 73.173662199566) bank76645 +76646 POINT(40.53062550892449 74.3471265197543) bank76646 +76647 POINT(40.311389137011844 73.90500132770221) bank76647 +76648 POINT(40.23036976629118 73.17994519581136) bank76648 +76649 POINT(39.83304112108595 73.6001323904578) bank76649 +76650 POINT(40.19590220925817 73.92553845128259) bank76650 +76651 POINT(39.8387770889282 73.8452175429122) bank76651 +76652 POINT(40.90363865099995 73.63888127984589) bank76652 +76653 POINT(41.405845663464206 73.1051897603733) bank76653 +76654 POINT(40.771017014296774 73.09394982513285) bank76654 +76655 POINT(40.40856688556344 74.3860454138192) bank76655 +76656 POINT(41.59830632680531 73.46233486050066) bank76656 +76657 POINT(41.243726310344705 74.66355926883027) bank76657 +76658 POINT(41.17395479928186 73.37752550039694) bank76658 +76659 POINT(39.79898608599223 73.03666456144369) bank76659 +76660 POINT(41.4830115044825 74.9583609656633) bank76660 +76661 POINT(40.24894638030841 73.93826667471987) bank76661 +76662 POINT(40.695878489092046 73.39899316667531) bank76662 +76663 POINT(40.62524365071462 73.22073024975339) bank76663 +76664 POINT(41.51299935960004 74.7327716436275) bank76664 +76665 POINT(41.626503384190755 74.38576039222848) bank76665 +76666 POINT(40.08903012623907 73.31106875234602) bank76666 +76667 POINT(41.36642637254209 74.21412658302573) bank76667 +76668 POINT(41.08245207400613 73.74940149628048) bank76668 +76669 POINT(40.22588373680126 73.66559299688926) bank76669 +76670 POINT(41.159061423606644 73.69774569006447) bank76670 +76671 POINT(39.76410364684035 73.36385127701432) bank76671 +76672 POINT(40.19032281316281 73.67217576475332) bank76672 +76673 POINT(41.38495573885343 74.40497397298016) bank76673 +76674 POINT(40.24133582380292 73.67569478363882) bank76674 +76675 POINT(39.916062975180694 73.88142279988382) bank76675 +76676 POINT(40.120020970505934 73.3718915952631) bank76676 +76677 POINT(41.41761127177163 73.45858873365424) bank76677 +76678 POINT(41.25367297369999 73.51570647759348) bank76678 +76679 POINT(41.516356061526785 74.57540354946214) bank76679 +76680 POINT(41.10213144396433 74.0721082048983) bank76680 +76681 POINT(40.08554648755273 73.05041540317842) bank76681 +76682 POINT(40.74721305856616 74.39327060218143) bank76682 +76683 POINT(40.866816335495336 74.22105533097881) bank76683 +76684 POINT(40.41344740623623 74.44543892332611) bank76684 +76685 POINT(41.277989718542656 73.19793473017494) bank76685 +76686 POINT(40.714785065760324 74.24900443172045) bank76686 +76687 POINT(39.742319850717934 73.59306069586397) bank76687 +76688 POINT(41.1993942647033 73.94877907828752) bank76688 +76689 POINT(40.03304229580472 74.58856120725189) bank76689 +76690 POINT(40.59625942742419 74.33582951471168) bank76690 +76691 POINT(40.79905198566742 74.2669758875856) bank76691 +76692 POINT(41.03378609400137 74.35926460723442) bank76692 +76693 POINT(39.792736193747366 73.228041777139) bank76693 +76694 POINT(41.36617423805503 74.54098227653316) bank76694 +76695 POINT(41.27522680454495 74.45631211495505) bank76695 +76696 POINT(39.92214774360654 74.20405354302568) bank76696 +76697 POINT(40.840173241641395 73.89631711841756) bank76697 +76698 POINT(40.07925537717868 74.81041447227292) bank76698 +76699 POINT(41.37458077150124 74.2387913135662) bank76699 +76700 POINT(39.73060630002709 73.607309969225) bank76700 +76701 POINT(39.82685127056935 74.52275686189677) bank76701 +76702 POINT(41.65819291056672 73.423139174127) bank76702 +76703 POINT(40.507691168204964 74.45498219002056) bank76703 +76704 POINT(41.52092363212894 73.16492250747878) bank76704 +76705 POINT(39.94067554502112 73.06293375545413) bank76705 +76706 POINT(39.8715841855464 73.48230056800841) bank76706 +76707 POINT(41.138916562974735 74.6326312318571) bank76707 +76708 POINT(40.27460751394628 74.55575239919432) bank76708 +76709 POINT(41.346976709193534 74.19558543263767) bank76709 +76710 POINT(40.64632750897651 74.57764215050469) bank76710 +76711 POINT(41.63035035384244 73.24879104099601) bank76711 +76712 POINT(41.05579585362811 74.01146421454425) bank76712 +76713 POINT(41.10460991991423 73.06310485846998) bank76713 +76714 POINT(40.281137612123146 74.59534772273676) bank76714 +76715 POINT(40.19129573715301 73.82956293117017) bank76715 +76716 POINT(39.73624205794871 73.38929659618886) bank76716 +76717 POINT(41.30001605044369 74.90507522819486) bank76717 +76718 POINT(40.96228684823403 74.83691799844283) bank76718 +76719 POINT(40.550132852675354 73.42496350074941) bank76719 +76720 POINT(40.89539055604664 73.09708474696804) bank76720 +76721 POINT(40.508172465190924 74.01172976953593) bank76721 +76722 POINT(40.34040410566665 74.49162530123134) bank76722 +76723 POINT(41.27879949172847 73.99080252316963) bank76723 +76724 POINT(40.84291104211406 74.7744121435386) bank76724 +76725 POINT(39.724807048270385 74.17056955878677) bank76725 +76726 POINT(40.07389017065252 74.41413298089066) bank76726 +76727 POINT(39.94263798446312 73.39782513618755) bank76727 +76728 POINT(41.00813943752415 74.35850747851036) bank76728 +76729 POINT(41.28711681393008 73.48410448444513) bank76729 +76730 POINT(40.666838105650235 74.88637011472991) bank76730 +76731 POINT(41.23253360488772 73.06962649988304) bank76731 +76732 POINT(40.10103474729769 74.61432990956024) bank76732 +76733 POINT(39.97618731971507 73.28987953323819) bank76733 +76734 POINT(41.22714979896154 74.44669159413496) bank76734 +76735 POINT(41.67953931780237 74.19948458376129) bank76735 +76736 POINT(40.484781021595744 74.26962425946684) bank76736 +76737 POINT(41.60759943826148 73.69709793076551) bank76737 +76738 POINT(41.69030804336783 73.24036068722519) bank76738 +76739 POINT(40.81302124571832 73.81568717992772) bank76739 +76740 POINT(39.721886851769774 73.396033608863) bank76740 +76741 POINT(41.086723880677944 74.39017799147841) bank76741 +76742 POINT(40.38495033184115 74.95333716891568) bank76742 +76743 POINT(41.35656951920047 74.50609435552491) bank76743 +76744 POINT(40.21214204124654 74.46612920796086) bank76744 +76745 POINT(41.52781140793206 73.15587085078981) bank76745 +76746 POINT(40.3000963388374 73.38217229378579) bank76746 +76747 POINT(40.921013308274915 73.75540369053803) bank76747 +76748 POINT(40.663397087212914 74.88852925634306) bank76748 +76749 POINT(40.72318101106779 74.70391610642449) bank76749 +76750 POINT(41.061991506422174 74.39903406054893) bank76750 +76751 POINT(41.4375643094932 73.77281287983267) bank76751 +76752 POINT(41.01393529483891 74.22338844543984) bank76752 +76753 POINT(41.602153378282985 74.02582659839672) bank76753 +76754 POINT(41.447816594026875 73.69105772282401) bank76754 +76755 POINT(39.85450047662621 74.21133545785871) bank76755 +76756 POINT(39.995796317311985 74.21613212816729) bank76756 +76757 POINT(39.88635971169326 73.96348388231822) bank76757 +76758 POINT(41.697595144668334 74.2887519699659) bank76758 +76759 POINT(40.93273801783778 74.88251236037794) bank76759 +76760 POINT(40.815918726703536 74.89581273869955) bank76760 +76761 POINT(40.86907464009774 73.61638148885635) bank76761 +76762 POINT(41.44416769331224 74.85562604608725) bank76762 +76763 POINT(40.27391599793173 74.49329812932956) bank76763 +76764 POINT(41.268541609215404 74.12622499844304) bank76764 +76765 POINT(40.0625474425189 73.27816158549517) bank76765 +76766 POINT(40.54383038249096 74.43316282782162) bank76766 +76767 POINT(39.803293463365634 74.94106928235425) bank76767 +76768 POINT(41.59604262467891 73.56384937192446) bank76768 +76769 POINT(40.15772026543435 74.68135496444138) bank76769 +76770 POINT(40.066385907102486 74.9937220110865) bank76770 +76771 POINT(39.853378847736664 74.92549991420046) bank76771 +76772 POINT(41.187983258802916 74.56786540003986) bank76772 +76773 POINT(40.31125378782689 73.36485874256428) bank76773 +76774 POINT(41.03988271487644 73.51825884143966) bank76774 +76775 POINT(41.613886923798155 73.78707020650482) bank76775 +76776 POINT(40.789468374269866 74.65280456483826) bank76776 +76777 POINT(40.44148912358654 74.58598301711632) bank76777 +76778 POINT(40.93218763540597 74.21155965466352) bank76778 +76779 POINT(40.38397077686641 74.62618238262782) bank76779 +76780 POINT(40.84397325279178 74.52691087446935) bank76780 +76781 POINT(41.49646685950949 73.39937281477391) bank76781 +76782 POINT(41.078070989575956 74.61393686082741) bank76782 +76783 POINT(41.35104179187208 75.00307958609167) bank76783 +76784 POINT(39.95721546376397 74.13036666913499) bank76784 +76785 POINT(40.5676318968559 73.99360326122381) bank76785 +76786 POINT(40.50859967731373 73.47661251482526) bank76786 +76787 POINT(41.4543678716938 73.73102617099768) bank76787 +76788 POINT(40.232924358473156 74.34188077934098) bank76788 +76789 POINT(40.243513058699705 73.90566866831972) bank76789 +76790 POINT(41.02563730067652 73.84019587753228) bank76790 +76791 POINT(40.958999809349265 73.1682428208164) bank76791 +76792 POINT(40.6302563274627 73.5572775457489) bank76792 +76793 POINT(39.852828732275384 73.3009516347855) bank76793 +76794 POINT(40.45565330825232 73.04586701193098) bank76794 +76795 POINT(39.879336210498224 74.40288805681409) bank76795 +76796 POINT(40.41645060859334 74.57875015081105) bank76796 +76797 POINT(41.0784963847069 74.05774743630498) bank76797 +76798 POINT(40.233079242137265 73.7781314160782) bank76798 +76799 POINT(39.87373255847828 73.78322406233907) bank76799 +76800 POINT(41.30938172647525 74.53419936933614) bank76800 +76801 POINT(40.909081625173 74.48916416770633) bank76801 +76802 POINT(41.042021744176694 73.8897146798133) bank76802 +76803 POINT(40.70923750878498 74.39519880394202) bank76803 +76804 POINT(41.20101630886373 74.89442113121676) bank76804 +76805 POINT(39.914482821322906 73.06122026358561) bank76805 +76806 POINT(41.4568292551733 73.23249221389644) bank76806 +76807 POINT(40.2548738636596 73.90686393764437) bank76807 +76808 POINT(41.17904140386759 73.6853431936177) bank76808 +76809 POINT(40.12717845933774 73.1851292344923) bank76809 +76810 POINT(41.13730970759268 74.86676403314792) bank76810 +76811 POINT(40.94482829679172 73.88060134968963) bank76811 +76812 POINT(40.769221067929905 73.19075304635733) bank76812 +76813 POINT(39.79694273104712 74.99142723807567) bank76813 +76814 POINT(39.8767233805684 73.37516409845725) bank76814 +76815 POINT(39.712827140275444 74.49995727117962) bank76815 +76816 POINT(40.094399647478184 74.8107977327142) bank76816 +76817 POINT(40.89408210784292 73.39894394350387) bank76817 +76818 POINT(39.92299687666321 73.55100655168053) bank76818 +76819 POINT(40.89576775186837 73.9892399326441) bank76819 +76820 POINT(41.09803748006147 74.50913760325064) bank76820 +76821 POINT(39.76195205156907 73.85604563619694) bank76821 +76822 POINT(40.792770630564874 74.93941323377756) bank76822 +76823 POINT(40.71345042016938 73.45974911423407) bank76823 +76824 POINT(40.377162474872485 73.78593387890058) bank76824 +76825 POINT(40.840829036946445 73.88939192635465) bank76825 +76826 POINT(40.29696478828755 73.97854729497725) bank76826 +76827 POINT(41.17975602550202 73.56321873486594) bank76827 +76828 POINT(40.504613227965 74.871407220624) bank76828 +76829 POINT(40.2683799042893 74.84211115739521) bank76829 +76830 POINT(41.18011949158211 74.38883770380735) bank76830 +76831 POINT(41.35194770235948 73.75119288168491) bank76831 +76832 POINT(40.78140177519 73.94735536297097) bank76832 +76833 POINT(40.235051160477916 74.12892924361832) bank76833 +76834 POINT(40.85327287233449 73.34829778103217) bank76834 +76835 POINT(39.774615488450856 74.06713046736958) bank76835 +76836 POINT(40.07829521035196 73.3362588954809) bank76836 +76837 POINT(39.81635809151508 73.02955693104184) bank76837 +76838 POINT(40.48070893841856 73.38479022667644) bank76838 +76839 POINT(41.1882636910644 73.7730893955434) bank76839 +76840 POINT(41.67122250150971 73.27069149507895) bank76840 +76841 POINT(39.9981545515964 73.48770101726508) bank76841 +76842 POINT(39.916183266435404 73.21662349988) bank76842 +76843 POINT(40.59599633058194 74.33510005062547) bank76843 +76844 POINT(40.751492314114046 73.43045480550218) bank76844 +76845 POINT(40.72257450670033 74.37432671394703) bank76845 +76846 POINT(40.33334798347702 74.95047747880646) bank76846 +76847 POINT(40.95448340985828 74.53391268162909) bank76847 +76848 POINT(39.723853479900015 74.9204399032723) bank76848 +76849 POINT(39.96159623210916 74.45850376636265) bank76849 +76850 POINT(40.70338211970266 74.03599154269104) bank76850 +76851 POINT(40.84313223070964 74.35918633887168) bank76851 +76852 POINT(41.5826555523093 74.65766513024151) bank76852 +76853 POINT(41.37715260866194 74.605609536118) bank76853 +76854 POINT(39.732407591237276 74.94431820578336) bank76854 +76855 POINT(40.117311329121286 74.588856114346) bank76855 +76856 POINT(40.835875186627625 73.12903969471411) bank76856 +76857 POINT(41.474690665643784 73.47267370598959) bank76857 +76858 POINT(40.104482418664475 73.62421334653585) bank76858 +76859 POINT(39.99528044863675 74.30030234978658) bank76859 +76860 POINT(40.39369351266946 73.32389444704438) bank76860 +76861 POINT(40.14464202123385 74.45470288792245) bank76861 +76862 POINT(41.15712257232536 73.96873330605062) bank76862 +76863 POINT(41.507765067883284 74.92859261427601) bank76863 +76864 POINT(40.70526805183784 74.16805573244703) bank76864 +76865 POINT(40.055259169635995 74.09250048059072) bank76865 +76866 POINT(40.809296950046225 73.0514165288918) bank76866 +76867 POINT(40.70962621550684 74.14470271152928) bank76867 +76868 POINT(40.716948237710106 73.90335956968957) bank76868 +76869 POINT(41.339286252743726 73.32652000358044) bank76869 +76870 POINT(40.66258153265379 74.62924627357661) bank76870 +76871 POINT(41.15525437800085 73.57986324824692) bank76871 +76872 POINT(40.59352662649988 74.01714557403051) bank76872 +76873 POINT(40.34624744072759 74.50066872390734) bank76873 +76874 POINT(41.3028212435457 73.7568535406493) bank76874 +76875 POINT(41.433282652807996 74.46336082083712) bank76875 +76876 POINT(40.10316418228954 73.99270793443924) bank76876 +76877 POINT(40.070733145127896 74.6380159799243) bank76877 +76878 POINT(40.60624180158753 74.85382044621699) bank76878 +76879 POINT(40.46561573152472 73.1444537585798) bank76879 +76880 POINT(39.71396368961178 73.73950342447893) bank76880 +76881 POINT(40.32151235668615 74.55906898049516) bank76881 +76882 POINT(40.68423123649997 74.92249649339226) bank76882 +76883 POINT(41.52810817169059 74.55892915121966) bank76883 +76884 POINT(40.513550100021696 73.5315122948137) bank76884 +76885 POINT(40.56020715138651 74.73000174627006) bank76885 +76886 POINT(41.180843551042436 74.99763076132349) bank76886 +76887 POINT(40.5209062780734 74.81087522728575) bank76887 +76888 POINT(39.77843104297255 74.61897846588892) bank76888 +76889 POINT(41.35358162507575 74.07226226604703) bank76889 +76890 POINT(41.00021119901995 74.88244529181186) bank76890 +76891 POINT(41.543970827993355 74.61818411696304) bank76891 +76892 POINT(40.45806045687843 73.5568962804714) bank76892 +76893 POINT(41.42416489516209 73.94780756858735) bank76893 +76894 POINT(40.76530903650545 74.26683347277282) bank76894 +76895 POINT(39.91631774805193 74.28276099739396) bank76895 +76896 POINT(41.35710237905937 73.79350191032115) bank76896 +76897 POINT(41.461903972670335 74.43114850640325) bank76897 +76898 POINT(40.75520424902006 73.81994512548873) bank76898 +76899 POINT(41.53407153488448 73.9337646943022) bank76899 +76900 POINT(41.2940552191657 74.50521938191245) bank76900 +76901 POINT(41.05915036829845 74.05582294711209) bank76901 +76902 POINT(39.93006417036662 74.91778873081276) bank76902 +76903 POINT(40.889946585503246 73.17491664406268) bank76903 +76904 POINT(40.84290853796311 73.09562322193999) bank76904 +76905 POINT(41.540120218948196 73.94875134622711) bank76905 +76906 POINT(40.92912143315032 73.77226266334506) bank76906 +76907 POINT(40.080714565895775 74.81108946212937) bank76907 +76908 POINT(39.75207008360773 74.37307281398759) bank76908 +76909 POINT(41.48138749906192 73.53841809500611) bank76909 +76910 POINT(40.35161855929242 74.3573679739869) bank76910 +76911 POINT(39.9934489249722 74.78231907779858) bank76911 +76912 POINT(41.06563856906671 73.22769206224363) bank76912 +76913 POINT(39.9242876407769 73.11559483683199) bank76913 +76914 POINT(41.16879873167511 73.36332734150764) bank76914 +76915 POINT(41.00844956846036 73.25161126831344) bank76915 +76916 POINT(41.103757286132826 74.95045256229264) bank76916 +76917 POINT(40.07441627948884 74.42287541113028) bank76917 +76918 POINT(41.28050033465624 74.8782439299555) bank76918 +76919 POINT(40.883280947896885 73.85144429402273) bank76919 +76920 POINT(40.46986854660072 73.69231671932297) bank76920 +76921 POINT(40.953305615821755 73.00612930014778) bank76921 +76922 POINT(40.52859505361398 73.70295215261716) bank76922 +76923 POINT(40.41109855967393 73.54382083082953) bank76923 +76924 POINT(40.07595633243447 73.03646905684911) bank76924 +76925 POINT(40.80415620009629 74.98663772343214) bank76925 +76926 POINT(40.107453088297085 73.05885953091986) bank76926 +76927 POINT(41.38251632729751 74.7200379482549) bank76927 +76928 POINT(41.61210298043109 74.49034604731689) bank76928 +76929 POINT(40.82218092700067 74.11316587782235) bank76929 +76930 POINT(40.44639645215204 73.89004993711848) bank76930 +76931 POINT(41.01864028202968 73.76684731882978) bank76931 +76932 POINT(41.65237648466932 74.13942261277657) bank76932 +76933 POINT(40.57840948033201 73.17134768107772) bank76933 +76934 POINT(40.15993726134944 73.51005460491113) bank76934 +76935 POINT(40.29654768103387 73.66351260916423) bank76935 +76936 POINT(40.38268639565582 73.6910699483778) bank76936 +76937 POINT(40.74338145852825 73.56795499188603) bank76937 +76938 POINT(40.31335353581423 73.77520814526538) bank76938 +76939 POINT(41.03149045754477 73.33887748341861) bank76939 +76940 POINT(40.82731676022854 73.06331295149153) bank76940 +76941 POINT(41.25313936369941 73.36083400145594) bank76941 +76942 POINT(40.90539113607621 73.1074230334273) bank76942 +76943 POINT(39.832471034038555 74.83522418929871) bank76943 +76944 POINT(40.13244397902989 74.82503070938054) bank76944 +76945 POINT(40.6569993725278 74.62741151946112) bank76945 +76946 POINT(40.76643509957258 74.7898042309753) bank76946 +76947 POINT(41.56722530277175 73.07704619307813) bank76947 +76948 POINT(40.34538994502752 74.01983262265053) bank76948 +76949 POINT(41.205031257238346 74.1665577877992) bank76949 +76950 POINT(39.78788042358259 73.66851259938743) bank76950 +76951 POINT(40.984749454340836 74.17918769248828) bank76951 +76952 POINT(41.67687869459178 74.68887424243042) bank76952 +76953 POINT(40.52980279137284 74.68560802484618) bank76953 +76954 POINT(40.63356523657337 73.25190409462445) bank76954 +76955 POINT(40.67673246457572 73.05607863200548) bank76955 +76956 POINT(40.09601510565436 74.79488076393679) bank76956 +76957 POINT(41.70077865408206 73.27272565522652) bank76957 +76958 POINT(41.36028156762479 74.73546630027059) bank76958 +76959 POINT(40.00880408770877 74.28505568973037) bank76959 +76960 POINT(40.274245820076395 74.91868828435017) bank76960 +76961 POINT(41.53318976554818 74.11646746215908) bank76961 +76962 POINT(41.28815563082879 73.95965387077054) bank76962 +76963 POINT(41.58887209947301 73.72522747376563) bank76963 +76964 POINT(40.757480868407214 73.26992147443069) bank76964 +76965 POINT(40.893707933518364 73.17344816983488) bank76965 +76966 POINT(39.858651145843425 74.66427787241214) bank76966 +76967 POINT(40.135810504478556 74.55286521775135) bank76967 +76968 POINT(41.11487940064374 74.98699915413981) bank76968 +76969 POINT(41.14163094256899 74.29628079460868) bank76969 +76970 POINT(41.30382534832168 74.49587569044476) bank76970 +76971 POINT(41.029157793713225 74.84291430106718) bank76971 +76972 POINT(40.59017688104188 74.85539970498311) bank76972 +76973 POINT(40.40521316377269 74.43564968023857) bank76973 +76974 POINT(41.55853535912843 73.105982322558) bank76974 +76975 POINT(41.1767059147825 74.36187783135271) bank76975 +76976 POINT(40.39669122149847 74.34334075448913) bank76976 +76977 POINT(41.109739531832204 73.03550743311904) bank76977 +76978 POINT(41.46264712258194 74.08174790282261) bank76978 +76979 POINT(39.72662072830664 74.99857386515644) bank76979 +76980 POINT(40.183846751095246 73.63422607858305) bank76980 +76981 POINT(40.74140163549143 73.61096712291476) bank76981 +76982 POINT(39.99583473170583 73.50789578147683) bank76982 +76983 POINT(40.554200211882794 74.95834069520593) bank76983 +76984 POINT(39.80893017446658 73.02143989980782) bank76984 +76985 POINT(41.67871979972646 74.46839053241325) bank76985 +76986 POINT(40.60877193284737 73.75221449673447) bank76986 +76987 POINT(41.42091914342904 73.83664307347705) bank76987 +76988 POINT(41.637113843005224 74.59549396166112) bank76988 +76989 POINT(39.76848572082169 74.55570746542962) bank76989 +76990 POINT(41.07443477711801 73.86641981455874) bank76990 +76991 POINT(39.8791847948465 73.23025488029032) bank76991 +76992 POINT(40.65233260914794 74.99223344624207) bank76992 +76993 POINT(41.45938201515988 74.17207354183061) bank76993 +76994 POINT(41.06818001498006 73.89617301582263) bank76994 +76995 POINT(41.66659714103062 73.38414104700601) bank76995 +76996 POINT(39.96962924853378 74.2070963144113) bank76996 +76997 POINT(39.91275265030774 74.90034307156373) bank76997 +76998 POINT(40.2362564144713 74.70794092902119) bank76998 +76999 POINT(40.110238374170905 73.95493926310667) bank76999 +77000 POINT(40.46158777983828 73.68675529708082) bank77000 +77001 POINT(39.78193105137483 74.2441037443344) bank77001 +77002 POINT(40.78717702571533 74.45404038976537) bank77002 +77003 POINT(40.43670817566857 73.04751233391842) bank77003 +77004 POINT(40.940575070634885 73.07346534539401) bank77004 +77005 POINT(41.37044600680094 74.02333322296519) bank77005 +77006 POINT(39.744105528009186 73.25343913453791) bank77006 +77007 POINT(41.3489661867369 73.99249854652506) bank77007 +77008 POINT(41.21488326841484 74.53320104914152) bank77008 +77009 POINT(41.68004242518295 74.66366154142784) bank77009 +77010 POINT(41.51056612266928 73.47693964709872) bank77010 +77011 POINT(40.526861093206875 73.54032328372482) bank77011 +77012 POINT(40.492783340442614 74.86701336341974) bank77012 +77013 POINT(41.68632908324937 74.08103499914029) bank77013 +77014 POINT(40.38213255762975 73.5533694980659) bank77014 +77015 POINT(40.178000572971115 73.21559989630997) bank77015 +77016 POINT(41.27565410969052 74.9086157649146) bank77016 +77017 POINT(40.767228225999006 74.86786638210987) bank77017 +77018 POINT(41.19057527267346 74.17022482776476) bank77018 +77019 POINT(40.02550147821816 74.54737850313161) bank77019 +77020 POINT(39.79484548893355 74.32500378517312) bank77020 +77021 POINT(39.83261208273911 74.99624985780646) bank77021 +77022 POINT(41.54757899346963 74.02055201744675) bank77022 +77023 POINT(40.252226622976224 73.93762908177527) bank77023 +77024 POINT(40.58278280098016 74.37055674601474) bank77024 +77025 POINT(40.89926225125415 73.69397992158264) bank77025 +77026 POINT(40.82362089990189 74.8499678137448) bank77026 +77027 POINT(39.72173523303957 74.07237990571328) bank77027 +77028 POINT(40.404604613010626 74.25989522240914) bank77028 +77029 POINT(40.45510237270014 73.96165845961956) bank77029 +77030 POINT(39.72010927140212 73.52882898011826) bank77030 +77031 POINT(41.68146846374171 74.65886170168855) bank77031 +77032 POINT(40.418922477893325 74.57037506984918) bank77032 +77033 POINT(40.65180842309793 74.69169573418229) bank77033 +77034 POINT(40.5946441601474 73.62048491631104) bank77034 +77035 POINT(41.65047681841757 73.66868833337988) bank77035 +77036 POINT(41.01699984603182 73.65009839148755) bank77036 +77037 POINT(40.2281258906152 74.30383236768071) bank77037 +77038 POINT(41.348220908604056 74.72549274690739) bank77038 +77039 POINT(40.27712058178078 74.92266823918969) bank77039 +77040 POINT(40.99483024883823 74.51553135872433) bank77040 +77041 POINT(41.66487934982466 74.67730724257562) bank77041 +77042 POINT(41.65917944975227 73.28226773257005) bank77042 +77043 POINT(39.946010340933405 74.95797011054363) bank77043 +77044 POINT(39.80842489207606 73.24851048695054) bank77044 +77045 POINT(39.886231050124735 73.75541653084949) bank77045 +77046 POINT(40.53188843815999 73.50166054275422) bank77046 +77047 POINT(39.852856413280435 73.57837504630902) bank77047 +77048 POINT(41.60635446784084 74.73870898551844) bank77048 +77049 POINT(41.50192602555729 74.33863649913205) bank77049 +77050 POINT(41.10914779548196 73.80420685528638) bank77050 +77051 POINT(41.40622491337463 74.23208049417471) bank77051 +77052 POINT(41.3671497743876 74.9792259202484) bank77052 +77053 POINT(40.253801710763184 73.7624417047685) bank77053 +77054 POINT(40.13190674061145 73.12989354477315) bank77054 +77055 POINT(39.965642200378895 73.75623626339501) bank77055 +77056 POINT(40.92906976270959 73.13484289507879) bank77056 +77057 POINT(40.12575952507713 74.4059977693269) bank77057 +77058 POINT(41.68552420928999 73.63493656840757) bank77058 +77059 POINT(41.708280952242184 74.44100673862788) bank77059 +77060 POINT(39.84670761013789 74.22729899022985) bank77060 +77061 POINT(39.91199098764786 74.67692288829488) bank77061 +77062 POINT(41.300750421670266 73.47868770092519) bank77062 +77063 POINT(41.27291378902836 73.72211144363294) bank77063 +77064 POINT(41.47482500909462 73.14816973802003) bank77064 +77065 POINT(39.760019019708835 74.60868316169638) bank77065 +77066 POINT(39.77983813092968 74.30416394185325) bank77066 +77067 POINT(41.43170023198239 74.29811590043617) bank77067 +77068 POINT(41.16375993755709 73.24763472151082) bank77068 +77069 POINT(40.0231808523221 73.273405000147) bank77069 +77070 POINT(39.84475017661533 73.21458877790376) bank77070 +77071 POINT(39.75839213682468 74.79456187757823) bank77071 +77072 POINT(40.48714665158939 73.9331952993922) bank77072 +77073 POINT(41.53745871643135 73.29603739721458) bank77073 +77074 POINT(41.3410666201344 73.8353797735486) bank77074 +77075 POINT(41.093749730463294 73.55939344059783) bank77075 +77076 POINT(39.90763922131702 73.98843496355286) bank77076 +77077 POINT(40.32481370419556 74.72634030295227) bank77077 +77078 POINT(41.5240050497103 73.89470568829364) bank77078 +77079 POINT(39.948386412568894 73.33642525625154) bank77079 +77080 POINT(40.415587749502336 74.76948172243644) bank77080 +77081 POINT(41.65135671065123 73.53271594172718) bank77081 +77082 POINT(39.797081135229575 74.29255789987621) bank77082 +77083 POINT(40.9222412713759 74.04994654198136) bank77083 +77084 POINT(41.23536722015353 73.05502671047812) bank77084 +77085 POINT(40.733347236183334 73.19746026757541) bank77085 +77086 POINT(41.52874441113213 74.50593125040463) bank77086 +77087 POINT(41.41739140918112 73.95528460162959) bank77087 +77088 POINT(41.5498090423192 73.89633540634016) bank77088 +77089 POINT(40.47632600461221 74.50707564923336) bank77089 +77090 POINT(41.610833509766195 73.82121481626453) bank77090 +77091 POINT(40.8958617502035 74.88224787388786) bank77091 +77092 POINT(40.0453392759241 74.51930681701468) bank77092 +77093 POINT(40.048457394196745 74.72527401714193) bank77093 +77094 POINT(41.47956087780781 73.84928253046655) bank77094 +77095 POINT(41.01563242088521 73.38847495749984) bank77095 +77096 POINT(41.070395326505015 74.3441854769822) bank77096 +77097 POINT(41.55428392015964 74.39873793421275) bank77097 +77098 POINT(41.4941822070478 74.80445918178933) bank77098 +77099 POINT(40.33027269875754 73.3279041327122) bank77099 +77100 POINT(40.98339307653455 73.81751742383274) bank77100 +77101 POINT(39.73778728943519 74.4385987956051) bank77101 +77102 POINT(40.099353130091785 73.34863309649934) bank77102 +77103 POINT(40.503593703971745 73.99775850665205) bank77103 +77104 POINT(41.2083769514879 73.33389735287331) bank77104 +77105 POINT(40.38461657639395 73.9216695739137) bank77105 +77106 POINT(41.16980599945212 73.57075376398191) bank77106 +77107 POINT(41.648962537063696 74.24772293838629) bank77107 +77108 POINT(41.50434832698771 73.93139713343459) bank77108 +77109 POINT(40.18306097320574 73.3618634124538) bank77109 +77110 POINT(40.55383114994356 74.47275057728635) bank77110 +77111 POINT(40.13124610408536 73.76991301944548) bank77111 +77112 POINT(41.48449620651301 74.71937152689313) bank77112 +77113 POINT(40.83403048359101 73.35859785918949) bank77113 +77114 POINT(40.183946431163825 74.45370506052849) bank77114 +77115 POINT(40.77737241564091 73.92105225078502) bank77115 +77116 POINT(41.34455428927282 73.61847679813998) bank77116 +77117 POINT(40.32245939922814 73.18444570710908) bank77117 +77118 POINT(41.048863934885574 73.44503071973318) bank77118 +77119 POINT(40.12241444846012 73.31361367359048) bank77119 +77120 POINT(40.423912281668876 73.57540646547902) bank77120 +77121 POINT(41.48676470166653 73.23082262104408) bank77121 +77122 POINT(40.59600294552396 74.75549239945187) bank77122 +77123 POINT(41.49974654262653 73.15237420846097) bank77123 +77124 POINT(40.63823740875416 74.25073724857) bank77124 +77125 POINT(40.30208666349178 73.9985282875227) bank77125 +77126 POINT(41.06058009624309 73.92702066726984) bank77126 +77127 POINT(41.04985393937023 74.75747546646923) bank77127 +77128 POINT(41.166470679440565 74.65240896063328) bank77128 +77129 POINT(40.592599063822625 74.01271131430036) bank77129 +77130 POINT(41.19979359145132 74.78798247002898) bank77130 +77131 POINT(40.59250443358349 74.72852374123919) bank77131 +77132 POINT(41.41573074771033 74.06043109677213) bank77132 +77133 POINT(40.090668762467686 74.20133013826644) bank77133 +77134 POINT(41.499841928051275 74.43289199838517) bank77134 +77135 POINT(40.57573729121356 74.51211557121707) bank77135 +77136 POINT(39.99058808132339 73.49002236779903) bank77136 +77137 POINT(40.0899263657835 74.36015056306704) bank77137 +77138 POINT(40.622275315000756 73.27178069395465) bank77138 +77139 POINT(40.27575235489753 73.96840605043242) bank77139 +77140 POINT(41.200700954941624 74.8709361346008) bank77140 +77141 POINT(40.159230615147564 74.29533565016102) bank77141 +77142 POINT(40.89574130202951 75.00022772476292) bank77142 +77143 POINT(40.34213517108366 74.50623877814631) bank77143 +77144 POINT(41.491088817401334 73.28267167918584) bank77144 +77145 POINT(40.37783421792565 73.53000604746946) bank77145 +77146 POINT(40.61210076735381 73.6161163236145) bank77146 +77147 POINT(41.321150121711426 73.59008943344423) bank77147 +77148 POINT(41.341871966882415 73.94749468210168) bank77148 +77149 POINT(41.29520319647807 73.04834527468401) bank77149 +77150 POINT(40.159860551082055 74.70325732585839) bank77150 +77151 POINT(41.65400469506866 74.88697535399386) bank77151 +77152 POINT(41.411104850492194 73.87828975966477) bank77152 +77153 POINT(40.082751775715714 74.86764458826808) bank77153 +77154 POINT(41.26612679034908 73.32737807818) bank77154 +77155 POINT(41.05504702812119 74.9128411518391) bank77155 +77156 POINT(40.72553994969287 73.23129017604214) bank77156 +77157 POINT(41.54321227743653 74.40179791973233) bank77157 +77158 POINT(40.93521320483786 74.74944027413576) bank77158 +77159 POINT(41.14119546394634 73.87865690823428) bank77159 +77160 POINT(40.08105035855887 74.76498005975826) bank77160 +77161 POINT(40.243239031950715 74.56602430457129) bank77161 +77162 POINT(41.34611698228369 73.02377221234875) bank77162 +77163 POINT(40.3001465406346 74.38313683503274) bank77163 +77164 POINT(39.78266504348 74.87615730010339) bank77164 +77165 POINT(40.53878981641763 74.50190313332827) bank77165 +77166 POINT(41.141283239074845 73.58772957272627) bank77166 +77167 POINT(39.91047333483227 73.3240983533992) bank77167 +77168 POINT(40.128268135146534 74.9590229689439) bank77168 +77169 POINT(41.56931343368002 73.41819969407864) bank77169 +77170 POINT(40.99661221690817 73.09357196266923) bank77170 +77171 POINT(40.25814791887615 74.65311792523937) bank77171 +77172 POINT(41.46453101966347 74.45096296939666) bank77172 +77173 POINT(40.7743187767122 74.15917535151432) bank77173 +77174 POINT(41.15942146790802 74.40613587090071) bank77174 +77175 POINT(39.753658396194474 73.73244374269892) bank77175 +77176 POINT(41.67305603838004 73.08108940434151) bank77176 +77177 POINT(41.46547020075356 73.82886453886653) bank77177 +77178 POINT(41.07125615674269 74.8520063873746) bank77178 +77179 POINT(41.40836208308353 74.89572421113633) bank77179 +77180 POINT(41.46335996906721 73.2291198909103) bank77180 +77181 POINT(41.027358092170495 74.48746215160115) bank77181 +77182 POINT(40.12637489978334 73.78432202557971) bank77182 +77183 POINT(40.23402516563754 74.36645053915981) bank77183 +77184 POINT(40.54963898759245 73.06847252149569) bank77184 +77185 POINT(40.012352821786365 73.392283666476) bank77185 +77186 POINT(40.190435119663896 74.12439580645604) bank77186 +77187 POINT(39.79356043234685 74.56065134990666) bank77187 +77188 POINT(40.497120812017464 74.66212249528124) bank77188 +77189 POINT(40.75083357896429 73.86269642409471) bank77189 +77190 POINT(40.066289135344014 73.78406045244635) bank77190 +77191 POINT(39.754167990522696 74.29488319816367) bank77191 +77192 POINT(41.23261249987863 73.8836470467275) bank77192 +77193 POINT(41.27393689996103 74.12455231684515) bank77193 +77194 POINT(39.85460130786115 74.74130337398171) bank77194 +77195 POINT(41.576716738008315 73.27274246137215) bank77195 +77196 POINT(39.8632570641773 74.3455270457623) bank77196 +77197 POINT(41.704800196480505 73.34787158386912) bank77197 +77198 POINT(40.085858378483664 74.24068352062156) bank77198 +77199 POINT(40.09466023922751 74.53166383222901) bank77199 +77200 POINT(40.94792254141781 74.96583428340982) bank77200 +77201 POINT(41.620890890092284 73.48811060201773) bank77201 +77202 POINT(40.3239683674704 73.14377421752116) bank77202 +77203 POINT(40.60641798461522 74.70276594226605) bank77203 +77204 POINT(39.89181720876164 73.28559131618218) bank77204 +77205 POINT(40.11010903134053 73.58170115095588) bank77205 +77206 POINT(41.27899697108728 73.31701724119073) bank77206 +77207 POINT(41.5240746930383 73.211761892674) bank77207 +77208 POINT(39.74072718588963 74.30904310881458) bank77208 +77209 POINT(41.31783641787532 73.93251819358423) bank77209 +77210 POINT(41.191269483476844 74.99551598550435) bank77210 +77211 POINT(39.74461442483864 74.56660693112009) bank77211 +77212 POINT(41.306069204705494 73.42303309430328) bank77212 +77213 POINT(41.41370385387858 74.64876326500233) bank77213 +77214 POINT(40.02353793860717 73.34509588001804) bank77214 +77215 POINT(40.64043089368373 74.95854231396818) bank77215 +77216 POINT(40.95009622116087 74.72744754420216) bank77216 +77217 POINT(40.20332234675855 73.54572455438229) bank77217 +77218 POINT(39.95049797046233 73.75361006666849) bank77218 +77219 POINT(40.889980401486284 73.3928696035432) bank77219 +77220 POINT(41.1519903008073 73.25609225402889) bank77220 +77221 POINT(40.64285851319473 73.0370114428713) bank77221 +77222 POINT(41.26697609117637 73.20152613666397) bank77222 +77223 POINT(41.562005644043026 73.26203071584719) bank77223 +77224 POINT(40.43735909876816 74.62596634981149) bank77224 +77225 POINT(40.952761152441255 73.98964550339446) bank77225 +77226 POINT(40.84177477695543 74.31561169813084) bank77226 +77227 POINT(40.049986790433806 73.0660488175676) bank77227 +77228 POINT(40.880837790109545 73.47759834412952) bank77228 +77229 POINT(41.65746414145968 73.64649262536932) bank77229 +77230 POINT(40.77599281384845 73.2793384316712) bank77230 +77231 POINT(40.90607603804184 74.41564758062717) bank77231 +77232 POINT(40.6409973723246 73.3211815418779) bank77232 +77233 POINT(41.332057882287664 74.9175880031003) bank77233 +77234 POINT(40.14727650595394 74.26238577525167) bank77234 +77235 POINT(40.49005056443226 74.27312177961468) bank77235 +77236 POINT(40.651753585381215 73.2348814906481) bank77236 +77237 POINT(40.142855102312915 74.82783739229387) bank77237 +77238 POINT(40.686432769494694 74.71056925418296) bank77238 +77239 POINT(40.44635216220863 74.52078615018269) bank77239 +77240 POINT(41.46010553394908 74.60889899917626) bank77240 +77241 POINT(40.484081339477456 74.348298878047) bank77241 +77242 POINT(40.9604637599242 73.62303438815667) bank77242 +77243 POINT(39.81096978495713 73.05243908509817) bank77243 +77244 POINT(41.30103393195405 73.5345768345329) bank77244 +77245 POINT(41.42564131203602 73.67154844709563) bank77245 +77246 POINT(40.56862048248715 74.78595880446535) bank77246 +77247 POINT(41.67419793032519 73.44871417753735) bank77247 +77248 POINT(40.48743721420628 74.28652594199139) bank77248 +77249 POINT(40.79256406091899 74.58967397749275) bank77249 +77250 POINT(40.45801508192314 73.05930246035999) bank77250 +77251 POINT(41.67107392157529 74.4959302293957) bank77251 +77252 POINT(40.05746114130166 74.3248335607697) bank77252 +77253 POINT(40.99007229730548 73.99186474239488) bank77253 +77254 POINT(41.19121562188304 74.7326008692192) bank77254 +77255 POINT(41.69765293580858 73.5407919941212) bank77255 +77256 POINT(40.88575689540534 73.54210766709043) bank77256 +77257 POINT(40.18809471039472 74.60107910684943) bank77257 +77258 POINT(40.09057012851789 73.1150057803528) bank77258 +77259 POINT(40.590591632375784 74.2792271040495) bank77259 +77260 POINT(40.85482085672308 73.33640172253978) bank77260 +77261 POINT(40.33595114056312 74.96853159136212) bank77261 +77262 POINT(39.72403646785234 74.78081806581218) bank77262 +77263 POINT(40.668490727255026 73.072480691271) bank77263 +77264 POINT(40.43809760174077 73.25062020243428) bank77264 +77265 POINT(40.277038724654645 73.61010513097702) bank77265 +77266 POINT(40.67710560316899 73.10757912655333) bank77266 +77267 POINT(40.13436252384956 74.9997880449581) bank77267 +77268 POINT(41.09789634881622 73.96060213696784) bank77268 +77269 POINT(41.480752931071585 74.28767755305864) bank77269 +77270 POINT(39.91369937669145 74.0898842361089) bank77270 +77271 POINT(40.64443912660544 73.95092041627164) bank77271 +77272 POINT(40.520699970003015 74.56475576555584) bank77272 +77273 POINT(39.94908371597574 73.72196611112848) bank77273 +77274 POINT(40.30470158922243 74.41387516384741) bank77274 +77275 POINT(40.502126607068035 73.90108448650005) bank77275 +77276 POINT(41.57826889481667 73.22414944176506) bank77276 +77277 POINT(40.03091580956811 74.13493601532888) bank77277 +77278 POINT(39.808745903819094 73.8225673901571) bank77278 +77279 POINT(41.38809034408168 74.27857623983239) bank77279 +77280 POINT(40.90053483994465 74.68224886148579) bank77280 +77281 POINT(40.014377466531855 73.14702137803853) bank77281 +77282 POINT(40.27564337586024 73.37581373623759) bank77282 +77283 POINT(41.26809087669248 73.83397176226008) bank77283 +77284 POINT(40.52199761005412 74.54452007640117) bank77284 +77285 POINT(40.24515711642932 74.70112710094853) bank77285 +77286 POINT(40.170700810705526 73.96374695227149) bank77286 +77287 POINT(41.670361830998765 74.26663752727337) bank77287 +77288 POINT(39.806066576838994 74.40462255957317) bank77288 +77289 POINT(41.01384943941502 74.02637107213471) bank77289 +77290 POINT(40.28588492067208 74.24651145098048) bank77290 +77291 POINT(40.334040398086515 74.11203915157637) bank77291 +77292 POINT(41.42870780284828 74.53610979435771) bank77292 +77293 POINT(41.073641478151515 73.14319712270105) bank77293 +77294 POINT(40.16025407510309 73.29707171142874) bank77294 +77295 POINT(41.03404346662459 73.51851723511884) bank77295 +77296 POINT(40.49147031445841 73.60903871217513) bank77296 +77297 POINT(40.33148716105745 73.21224913138035) bank77297 +77298 POINT(40.00760599535448 74.46617939655158) bank77298 +77299 POINT(41.0279852277955 73.25066206942316) bank77299 +77300 POINT(40.06469188360739 74.74796916745628) bank77300 +77301 POINT(39.88341966485349 73.46709783449631) bank77301 +77302 POINT(40.44283051015192 73.6627727877743) bank77302 +77303 POINT(40.105038809863665 74.20536767923453) bank77303 +77304 POINT(41.04550952656453 73.93976628952764) bank77304 +77305 POINT(41.09741905141648 73.59632486037485) bank77305 +77306 POINT(39.7779240741054 74.156667004523) bank77306 +77307 POINT(40.62227554525235 73.60898781050255) bank77307 +77308 POINT(41.496364695998636 73.50496238914239) bank77308 +77309 POINT(41.51156755268925 73.18888697311417) bank77309 +77310 POINT(39.89739055453166 74.57965425715469) bank77310 +77311 POINT(40.71516184117932 73.39351189131585) bank77311 +77312 POINT(40.14106247264077 73.56670167468434) bank77312 +77313 POINT(40.9736793147796 74.59857964655544) bank77313 +77314 POINT(40.29310414149294 74.72087488235351) bank77314 +77315 POINT(41.083432565020246 74.12679920918974) bank77315 +77316 POINT(40.03972362713778 73.09832981886645) bank77316 +77317 POINT(41.4764504572612 73.8789546948172) bank77317 +77318 POINT(41.048488407454656 73.64757693154931) bank77318 +77319 POINT(40.34906008067755 74.07598258381663) bank77319 +77320 POINT(39.94314663858094 73.79514834601902) bank77320 +77321 POINT(40.5337662239988 73.34671636596596) bank77321 +77322 POINT(39.95624027019307 73.72290214797032) bank77322 +77323 POINT(39.91528449502809 73.6069472172239) bank77323 +77324 POINT(40.81332816750209 74.87313346839497) bank77324 +77325 POINT(41.28265055997465 74.4205058017954) bank77325 +77326 POINT(39.97694644893549 74.91390136754114) bank77326 +77327 POINT(40.55725579227554 74.21189554517329) bank77327 +77328 POINT(40.692111242792414 74.639537755187) bank77328 +77329 POINT(40.6190935586916 74.29878786976938) bank77329 +77330 POINT(40.880094967804034 74.50805099691655) bank77330 +77331 POINT(40.275057160441136 73.62646018316663) bank77331 +77332 POINT(41.35506861013324 73.84168546859382) bank77332 +77333 POINT(41.52324488364717 74.19141551660391) bank77333 +77334 POINT(40.49824574273925 74.49034205204701) bank77334 +77335 POINT(41.467086711515975 73.0689846106452) bank77335 +77336 POINT(41.13589701546847 74.92923100008522) bank77336 +77337 POINT(40.18123476608007 73.35634199056216) bank77337 +77338 POINT(40.2952095463163 74.12311056816834) bank77338 +77339 POINT(40.995665226613205 74.6189129540122) bank77339 +77340 POINT(41.41729660091998 74.65045210265306) bank77340 +77341 POINT(40.42698447451533 73.59760155398516) bank77341 +77342 POINT(39.787930110038936 73.83596122393043) bank77342 +77343 POINT(41.68556003686661 74.06906236577733) bank77343 +77344 POINT(40.98482343451124 74.31417420448409) bank77344 +77345 POINT(40.243143079260264 74.01943040658223) bank77345 +77346 POINT(40.231591995455126 74.39213662053552) bank77346 +77347 POINT(41.36337374115048 73.3269755968073) bank77347 +77348 POINT(41.10031270537689 74.00684688282587) bank77348 +77349 POINT(39.73435131629577 74.63572426081619) bank77349 +77350 POINT(40.34927321834347 73.33207173591553) bank77350 +77351 POINT(40.91501460419995 74.81427217494199) bank77351 +77352 POINT(40.463585421869126 73.15377143680568) bank77352 +77353 POINT(41.52890446964767 74.6624749758212) bank77353 +77354 POINT(39.872265660420155 74.96314671926115) bank77354 +77355 POINT(40.802025717216516 74.61341258926754) bank77355 +77356 POINT(40.844130759137556 73.90455028345217) bank77356 +77357 POINT(40.201440455904375 74.05802698593862) bank77357 +77358 POINT(41.31282541500547 73.9902386433905) bank77358 +77359 POINT(41.27111771688187 73.55889914861469) bank77359 +77360 POINT(40.26536205366516 74.7572912380739) bank77360 +77361 POINT(41.31727253947261 74.51476868810802) bank77361 +77362 POINT(41.113119983942674 73.02853806034855) bank77362 +77363 POINT(39.87121412536482 74.44131669538747) bank77363 +77364 POINT(40.02164366391374 73.06421001579879) bank77364 +77365 POINT(41.29488026697871 74.51062091890597) bank77365 +77366 POINT(41.68781102489905 74.24388867156149) bank77366 +77367 POINT(40.88229703650145 74.36307372871927) bank77367 +77368 POINT(40.35221436861808 74.12511932147028) bank77368 +77369 POINT(41.19521074767961 74.3520587906366) bank77369 +77370 POINT(41.35314091665347 74.18434286737914) bank77370 +77371 POINT(41.16575928555824 74.96915781214673) bank77371 +77372 POINT(41.70672382299923 74.40610867276372) bank77372 +77373 POINT(41.4665073122695 73.17832909750717) bank77373 +77374 POINT(41.188355195869875 73.87248771442118) bank77374 +77375 POINT(39.72367624503683 74.03249818072109) bank77375 +77376 POINT(41.15597809696413 73.30192136809346) bank77376 +77377 POINT(41.05702173021862 73.17326514198996) bank77377 +77378 POINT(40.29757009696486 73.7952552091276) bank77378 +77379 POINT(40.7273769274864 74.94437678005855) bank77379 +77380 POINT(40.89052924756055 73.83107741814943) bank77380 +77381 POINT(41.49479404101163 73.77695826939603) bank77381 +77382 POINT(40.14083857262354 74.92144915783372) bank77382 +77383 POINT(41.50651289880304 74.60190979199903) bank77383 +77384 POINT(40.181980839864224 73.16753399403696) bank77384 +77385 POINT(40.14107800893671 74.26579404490808) bank77385 +77386 POINT(41.10912762185519 73.19760271216572) bank77386 +77387 POINT(41.29614511646537 74.86639770876265) bank77387 +77388 POINT(41.67257935322412 74.67306186969321) bank77388 +77389 POINT(41.53807453728055 73.42442490773318) bank77389 +77390 POINT(41.1822115471171 74.7746110953598) bank77390 +77391 POINT(41.19643221529307 74.52333904153137) bank77391 +77392 POINT(40.25872088975797 73.9759750935347) bank77392 +77393 POINT(40.08578491480668 73.142342306181) bank77393 +77394 POINT(41.03432377291615 73.60388166233055) bank77394 +77395 POINT(40.82108462758182 74.41690254001797) bank77395 +77396 POINT(41.60569379553586 74.90141062632117) bank77396 +77397 POINT(40.67130625648333 73.04924786410336) bank77397 +77398 POINT(40.81863775641615 73.61340094297529) bank77398 +77399 POINT(40.11786286085062 74.29848262292352) bank77399 +77400 POINT(39.749834479134954 73.42197794821327) bank77400 +77401 POINT(40.47218322301972 73.24865667119217) bank77401 +77402 POINT(41.05499232672923 73.90379034472905) bank77402 +77403 POINT(40.139706351619665 74.16396264166767) bank77403 +77404 POINT(41.21106099469589 73.28449224130078) bank77404 +77405 POINT(40.08892458221612 73.09176052769975) bank77405 +77406 POINT(40.85369225085409 74.45042753442456) bank77406 +77407 POINT(40.030466528902274 73.21364970966208) bank77407 +77408 POINT(39.82716935770444 73.62243492537348) bank77408 +77409 POINT(40.75908348848568 74.29756828713978) bank77409 +77410 POINT(41.620569394018055 75.00590231170786) bank77410 +77411 POINT(40.374949740669905 73.36782478931256) bank77411 +77412 POINT(40.779795138384785 73.50067202245172) bank77412 +77413 POINT(39.904369818790315 73.65170351596808) bank77413 +77414 POINT(40.55013892815475 73.9213894032794) bank77414 +77415 POINT(41.51849729200162 74.4158743231276) bank77415 +77416 POINT(40.128200866846555 74.12627018873926) bank77416 +77417 POINT(39.82008917225027 74.78494307920285) bank77417 +77418 POINT(40.67228798651296 74.4889934452906) bank77418 +77419 POINT(41.64532831976595 74.06750220516527) bank77419 +77420 POINT(40.50456188649941 74.63850644074176) bank77420 +77421 POINT(40.452676852536555 74.29184196581296) bank77421 +77422 POINT(41.53550947483633 73.47350259161685) bank77422 +77423 POINT(41.04234477514719 74.48398119861668) bank77423 +77424 POINT(41.09816724484676 73.50795457405682) bank77424 +77425 POINT(41.08257311785794 73.0279019463339) bank77425 +77426 POINT(40.99588519719289 74.43301959462337) bank77426 +77427 POINT(41.58957398225365 74.37619551844796) bank77427 +77428 POINT(40.627597959480056 74.98326872657752) bank77428 +77429 POINT(40.2413700405361 74.27244151482562) bank77429 +77430 POINT(41.54899762842093 74.26981817050685) bank77430 +77431 POINT(41.53642792758659 74.04845536383556) bank77431 +77432 POINT(40.96636333979122 73.95874740139573) bank77432 +77433 POINT(40.74896666099308 73.01179354803794) bank77433 +77434 POINT(40.85282133568284 74.77742968154352) bank77434 +77435 POINT(40.149400959985776 74.9569390485814) bank77435 +77436 POINT(40.139405655797695 74.93442301973863) bank77436 +77437 POINT(40.48835200903435 73.33632900633634) bank77437 +77438 POINT(40.2909583698841 74.86570242477858) bank77438 +77439 POINT(41.49558695157952 74.71242276046587) bank77439 +77440 POINT(41.42584233935916 74.6817690168045) bank77440 +77441 POINT(40.16582697802255 73.67669654268016) bank77441 +77442 POINT(40.25617765801937 74.03514483961317) bank77442 +77443 POINT(40.16501916023325 74.14702179461766) bank77443 +77444 POINT(41.5207922247458 74.69101206553937) bank77444 +77445 POINT(41.45214112998715 74.19407886355218) bank77445 +77446 POINT(40.38851962028776 73.08021734503298) bank77446 +77447 POINT(41.07735287535068 73.14525980309833) bank77447 +77448 POINT(41.423904739095114 73.04555526440348) bank77448 +77449 POINT(41.27880692972694 74.2880727113221) bank77449 +77450 POINT(41.07044822949661 73.88897484010316) bank77450 +77451 POINT(40.78821412965442 73.29120509955541) bank77451 +77452 POINT(40.42988195302979 73.08873843866266) bank77452 +77453 POINT(39.85515462905309 74.65799346375206) bank77453 +77454 POINT(41.50796178447728 74.75530129658729) bank77454 +77455 POINT(41.0087840978659 74.22505109665647) bank77455 +77456 POINT(40.57061100759666 74.97002248796537) bank77456 +77457 POINT(41.144664280063175 73.8280992533082) bank77457 +77458 POINT(40.984095343454825 73.94291462205379) bank77458 +77459 POINT(40.637381695951454 74.94809888134637) bank77459 +77460 POINT(40.85633958967855 73.48925001630619) bank77460 +77461 POINT(40.91485410119297 73.04018372876722) bank77461 +77462 POINT(40.976232557969844 74.27607476350944) bank77462 +77463 POINT(40.640863869963766 73.18626483697201) bank77463 +77464 POINT(40.855203863900364 74.86610044041328) bank77464 +77465 POINT(40.96391843071482 73.88940706353692) bank77465 +77466 POINT(40.00355174007405 73.52533534659918) bank77466 +77467 POINT(41.13832241485201 73.01219013203269) bank77467 +77468 POINT(40.698178394945344 73.8730801132629) bank77468 +77469 POINT(41.377438439623184 73.27781281001874) bank77469 +77470 POINT(40.31587124201802 74.73405085157171) bank77470 +77471 POINT(41.42865354029401 74.15864142835296) bank77471 +77472 POINT(40.70708876181572 73.15322780680343) bank77472 +77473 POINT(41.04929171589382 74.31225155850923) bank77473 +77474 POINT(39.95675563071166 74.92699180275278) bank77474 +77475 POINT(41.6250477753963 74.62065145900428) bank77475 +77476 POINT(40.31659291766668 73.58850311742599) bank77476 +77477 POINT(41.63969890261623 74.76158966852107) bank77477 +77478 POINT(41.44427844149925 74.71030335615872) bank77478 +77479 POINT(40.518611199092796 73.58981794374387) bank77479 +77480 POINT(40.10018200362608 73.2680372172501) bank77480 +77481 POINT(39.71554705618266 73.7828074092044) bank77481 +77482 POINT(40.713971495531744 74.18696237907614) bank77482 +77483 POINT(41.278629083749934 73.10646865740796) bank77483 +77484 POINT(39.79551003264029 74.44632997485336) bank77484 +77485 POINT(39.90148212828245 74.50965760768275) bank77485 +77486 POINT(40.558352134656374 73.8982781350102) bank77486 +77487 POINT(40.78113110427085 74.85170346734022) bank77487 +77488 POINT(41.640414994312145 73.45073031366464) bank77488 +77489 POINT(39.93978493222763 73.92743276727843) bank77489 +77490 POINT(39.803990949362166 73.52249684475102) bank77490 +77491 POINT(40.05882233936624 73.54389485926147) bank77491 +77492 POINT(41.316246070118694 73.09085504087713) bank77492 +77493 POINT(40.86276440915768 73.10634781149854) bank77493 +77494 POINT(40.1372688643043 73.05928038525491) bank77494 +77495 POINT(41.34329035052453 73.82674114593352) bank77495 +77496 POINT(39.98849883912854 74.4697302895979) bank77496 +77497 POINT(39.721834635083724 73.30119252504481) bank77497 +77498 POINT(41.45001727993252 73.84418944013484) bank77498 +77499 POINT(40.32389030642538 74.71563257287778) bank77499 +77500 POINT(40.626850837738985 73.64950121681618) bank77500 +77501 POINT(39.828957280135285 73.04397035991474) bank77501 +77502 POINT(41.64635084409321 73.81217036674177) bank77502 +77503 POINT(40.598530811444775 73.01607379875016) bank77503 +77504 POINT(40.370759594736086 73.03986923853937) bank77504 +77505 POINT(40.43096723500302 73.30096844730717) bank77505 +77506 POINT(40.04033450114136 74.56122580639533) bank77506 +77507 POINT(41.68791935682434 74.98185154904729) bank77507 +77508 POINT(39.971911308244735 73.59989734657556) bank77508 +77509 POINT(41.165650143953265 74.72610273642026) bank77509 +77510 POINT(40.86778478392295 74.23158706770754) bank77510 +77511 POINT(40.986640152883076 73.70472930456943) bank77511 +77512 POINT(41.31070167499752 73.61784138109137) bank77512 +77513 POINT(40.84201698865913 73.96401441975128) bank77513 +77514 POINT(41.283112668961785 73.38884962931533) bank77514 +77515 POINT(39.73611281576557 73.26829409838078) bank77515 +77516 POINT(41.43177506540422 74.11738608736697) bank77516 +77517 POINT(41.07931289936212 74.73275582474255) bank77517 +77518 POINT(40.31185106381581 73.12245270894283) bank77518 +77519 POINT(40.13866196128921 74.18613196604183) bank77519 +77520 POINT(39.82181557990737 73.24271444587717) bank77520 +77521 POINT(40.352105144235324 73.46967634063718) bank77521 +77522 POINT(40.49522744851591 74.25899121350072) bank77522 +77523 POINT(40.773123063064546 74.48382160141551) bank77523 +77524 POINT(40.91787841271918 73.43610768068245) bank77524 +77525 POINT(39.825593706072745 73.15735712173573) bank77525 +77526 POINT(40.82840338480544 73.06591308029064) bank77526 +77527 POINT(41.17705353964334 74.20556123684695) bank77527 +77528 POINT(41.3508116779618 73.61564683571362) bank77528 +77529 POINT(40.42669879031792 73.23430956904186) bank77529 +77530 POINT(39.959323216951105 73.03247656277416) bank77530 +77531 POINT(40.49525186104432 74.03685213232063) bank77531 +77532 POINT(41.393424742464056 74.1148890016132) bank77532 +77533 POINT(41.26036469528047 73.57771331327254) bank77533 +77534 POINT(40.646960571171434 73.56314991308359) bank77534 +77535 POINT(40.09401433517831 74.19687572967818) bank77535 +77536 POINT(40.11461079666468 74.27222939338753) bank77536 +77537 POINT(40.191011793858884 74.57072658946636) bank77537 +77538 POINT(41.04225689344988 73.26175650846596) bank77538 +77539 POINT(41.4788445955863 73.38169763564572) bank77539 +77540 POINT(41.20013873265371 74.57715545252677) bank77540 +77541 POINT(41.119018277906385 73.55742203084722) bank77541 +77542 POINT(39.91708997772383 74.00587713100336) bank77542 +77543 POINT(39.997000815043954 74.09972810813075) bank77543 +77544 POINT(41.33945193079883 73.22786031426253) bank77544 +77545 POINT(41.567536410408984 74.97930403755385) bank77545 +77546 POINT(41.63553777363966 74.88152473252643) bank77546 +77547 POINT(40.454683467606074 73.98948148851653) bank77547 +77548 POINT(41.51919723264659 74.26949557918505) bank77548 +77549 POINT(41.105199811451115 74.3072120214696) bank77549 +77550 POINT(40.90242313685433 74.97366652526) bank77550 +77551 POINT(41.26610447284278 74.88257966391971) bank77551 +77552 POINT(41.38927602114592 74.41908743059682) bank77552 +77553 POINT(40.134689548136706 73.31878583091128) bank77553 +77554 POINT(40.13419697162219 73.66426128057486) bank77554 +77555 POINT(41.45332965956648 74.91538646487345) bank77555 +77556 POINT(41.36893704202597 73.51149615049152) bank77556 +77557 POINT(40.63019768547146 73.79195884018594) bank77557 +77558 POINT(40.977185103940165 73.18213501391119) bank77558 +77559 POINT(41.59641532346423 73.96197041692727) bank77559 +77560 POINT(40.874507285459906 74.23674492675232) bank77560 +77561 POINT(41.385357575300624 74.90302483834525) bank77561 +77562 POINT(41.487096021917075 73.71698089865761) bank77562 +77563 POINT(41.04078457442124 74.13129104660885) bank77563 +77564 POINT(40.32618568019694 73.4746660719145) bank77564 +77565 POINT(40.694428117366755 74.30585301117856) bank77565 +77566 POINT(40.54572845621703 73.16264031076231) bank77566 +77567 POINT(40.02027779163306 74.71264438566952) bank77567 +77568 POINT(40.72989268604661 73.91382125403479) bank77568 +77569 POINT(40.49453875647266 74.69994121100845) bank77569 +77570 POINT(40.5851240261765 74.56243044260889) bank77570 +77571 POINT(39.73812231257769 73.89043882417576) bank77571 +77572 POINT(41.42909465497125 73.82283198561106) bank77572 +77573 POINT(39.79002713497813 73.97784559321097) bank77573 +77574 POINT(40.65837030001203 74.39841586222846) bank77574 +77575 POINT(40.864175230175405 73.5234477790408) bank77575 +77576 POINT(41.287239154234065 73.73804546651226) bank77576 +77577 POINT(40.29315148709864 73.87005774021296) bank77577 +77578 POINT(41.47857593261882 73.00693597677521) bank77578 +77579 POINT(41.595759620678166 74.09734985704935) bank77579 +77580 POINT(41.23535095860473 74.84356409334168) bank77580 +77581 POINT(41.2981339002815 73.66807836694198) bank77581 +77582 POINT(41.45752543646374 74.68515774023294) bank77582 +77583 POINT(41.05483878556409 75.0013571257891) bank77583 +77584 POINT(41.298555724612434 73.77799590617596) bank77584 +77585 POINT(41.443792295272736 74.98187568299683) bank77585 +77586 POINT(40.926665286668616 73.80839641725942) bank77586 +77587 POINT(39.78973157824432 74.76539673669365) bank77587 +77588 POINT(41.49158657551811 74.96350489867105) bank77588 +77589 POINT(41.534274065842354 73.84142570941124) bank77589 +77590 POINT(39.76504304221416 74.74785852282642) bank77590 +77591 POINT(41.15958114409871 73.11167590118903) bank77591 +77592 POINT(41.60028498891923 73.64254359655638) bank77592 +77593 POINT(41.51170043404431 74.52898183068184) bank77593 +77594 POINT(41.512469561495195 73.83525884450493) bank77594 +77595 POINT(41.46924342276842 73.24467551233572) bank77595 +77596 POINT(40.039482632723995 73.81628714993542) bank77596 +77597 POINT(40.18345858468448 74.5004330138617) bank77597 +77598 POINT(41.22156288832081 73.62300185977871) bank77598 +77599 POINT(41.64739252244644 74.17732977928763) bank77599 +77600 POINT(41.638912681751094 74.47021571191539) bank77600 +77601 POINT(39.84256082041629 73.61365139538876) bank77601 +77602 POINT(39.75274698480801 74.36363866509829) bank77602 +77603 POINT(40.61867441185791 74.40422087666556) bank77603 +77604 POINT(41.42276460806083 73.24489621799627) bank77604 +77605 POINT(39.7881745598983 74.98175129101325) bank77605 +77606 POINT(41.07813682424672 73.68298788415409) bank77606 +77607 POINT(40.30544263959436 74.31920431254414) bank77607 +77608 POINT(40.28345413329145 74.80633092060192) bank77608 +77609 POINT(39.928012893898256 74.34112508216704) bank77609 +77610 POINT(41.53161199387013 74.0516161549385) bank77610 +77611 POINT(40.585756042660734 73.01150075945452) bank77611 +77612 POINT(40.584273581970905 74.71952220344927) bank77612 +77613 POINT(39.803827949822015 73.92548443861513) bank77613 +77614 POINT(41.24456262898932 74.28976650254174) bank77614 +77615 POINT(40.894948010917446 74.54701906550343) bank77615 +77616 POINT(40.89258278469493 73.29340654849419) bank77616 +77617 POINT(40.65765918221564 73.60954599688652) bank77617 +77618 POINT(40.427773591474754 73.93321926924148) bank77618 +77619 POINT(41.10778370241247 74.64995675904586) bank77619 +77620 POINT(41.68820345206283 74.73676315740356) bank77620 +77621 POINT(41.05779628700858 74.7649739317007) bank77621 +77622 POINT(41.12565062310096 74.73157690083653) bank77622 +77623 POINT(40.74225651125209 74.78237673056047) bank77623 +77624 POINT(40.99856714595752 74.19676219630949) bank77624 +77625 POINT(40.38750911631253 74.28708485468375) bank77625 +77626 POINT(40.33094350100719 74.86471447642826) bank77626 +77627 POINT(41.17285569865029 74.9340598226599) bank77627 +77628 POINT(40.866152672358616 74.10914463527381) bank77628 +77629 POINT(40.56919025173395 73.89009632266281) bank77629 +77630 POINT(39.91991875205214 73.04959997258563) bank77630 +77631 POINT(41.245006215819885 74.9517825355825) bank77631 +77632 POINT(40.06919224788071 74.60951764813608) bank77632 +77633 POINT(41.14150036330796 74.71198103522174) bank77633 +77634 POINT(40.107138551847555 73.22086020755056) bank77634 +77635 POINT(41.2335716568806 73.48350251865531) bank77635 +77636 POINT(39.84947025649196 74.44329458508926) bank77636 +77637 POINT(40.52249438211482 74.84147952058977) bank77637 +77638 POINT(41.324524753860864 74.38474425323079) bank77638 +77639 POINT(40.6052344916893 74.13998821336004) bank77639 +77640 POINT(40.18439036985841 74.81775911812609) bank77640 +77641 POINT(40.51082808479027 73.26317304085318) bank77641 +77642 POINT(39.90836172325632 74.97099352809832) bank77642 +77643 POINT(39.844045117166964 74.65495035887353) bank77643 +77644 POINT(40.163445357581935 73.28683280696539) bank77644 +77645 POINT(40.696540683124795 74.17128817054163) bank77645 +77646 POINT(40.22349857433417 74.66069504892621) bank77646 +77647 POINT(40.825422959276466 73.65657568404778) bank77647 +77648 POINT(41.54774833442748 73.47130969454602) bank77648 +77649 POINT(40.463078858264666 73.0779636806328) bank77649 +77650 POINT(41.08692586139533 73.86780774225257) bank77650 +77651 POINT(40.803064095125876 74.31544828453501) bank77651 +77652 POINT(41.275363941715014 73.39695636241049) bank77652 +77653 POINT(40.63424556581857 74.67284424376611) bank77653 +77654 POINT(40.14922822270421 73.75906888308316) bank77654 +77655 POINT(41.11592315833372 73.33071491014759) bank77655 +77656 POINT(40.36694888961265 73.46280763772788) bank77656 +77657 POINT(40.936097371693414 73.955508278994) bank77657 +77658 POINT(40.72093765673976 74.78606025787793) bank77658 +77659 POINT(40.24968718102907 74.35545213606319) bank77659 +77660 POINT(39.7660432411917 74.82527166955443) bank77660 +77661 POINT(41.64599384947239 73.64726651530762) bank77661 +77662 POINT(41.07607394954003 74.78581395866965) bank77662 +77663 POINT(41.61879340758979 73.21105090880793) bank77663 +77664 POINT(41.664848439701686 73.40571927824598) bank77664 +77665 POINT(40.16817535142423 74.0370476343181) bank77665 +77666 POINT(40.07431295841374 74.69595378043313) bank77666 +77667 POINT(41.56385322193887 73.3619451658826) bank77667 +77668 POINT(39.8294551801413 74.2032150286878) bank77668 +77669 POINT(40.833692619160935 73.21693994343421) bank77669 +77670 POINT(40.425544537809 73.98341481433131) bank77670 +77671 POINT(39.844478429934476 74.54776371876113) bank77671 +77672 POINT(40.278551151046095 74.8520094978447) bank77672 +77673 POINT(39.94316650997408 73.10053110946475) bank77673 +77674 POINT(40.59347881072665 73.20236829027806) bank77674 +77675 POINT(41.69871505741302 73.55900984367871) bank77675 +77676 POINT(40.40035901003418 73.45598898209327) bank77676 +77677 POINT(41.687350472716425 73.57006244797394) bank77677 +77678 POINT(40.40685112309783 74.75675850482624) bank77678 +77679 POINT(40.49393028475848 74.46686385788567) bank77679 +77680 POINT(40.15388886293766 74.32807394357003) bank77680 +77681 POINT(40.709116620885546 73.59418280068493) bank77681 +77682 POINT(40.68757313162287 73.44708235790165) bank77682 +77683 POINT(40.24078689203863 73.67495284875437) bank77683 +77684 POINT(40.73052221832781 73.24286538537669) bank77684 +77685 POINT(41.10718129670212 73.92494563710167) bank77685 +77686 POINT(41.11905199628482 73.15036682962682) bank77686 +77687 POINT(41.38343958060545 74.42436654529297) bank77687 +77688 POINT(40.76788519207964 73.7812198283086) bank77688 +77689 POINT(40.28552445972555 73.40223190722539) bank77689 +77690 POINT(40.52429682576723 74.71416797418641) bank77690 +77691 POINT(40.80492254377474 74.9183325536669) bank77691 +77692 POINT(41.19606224625745 74.84296285180567) bank77692 +77693 POINT(41.14984167268004 73.94973866822932) bank77693 +77694 POINT(40.52415253367138 73.11631555497323) bank77694 +77695 POINT(40.648937117563285 73.49253701619217) bank77695 +77696 POINT(40.01755719775771 74.14299512169751) bank77696 +77697 POINT(40.3316203099834 74.19758878380718) bank77697 +77698 POINT(40.5913661643108 74.89386964757959) bank77698 +77699 POINT(41.199860791276336 74.75663096240434) bank77699 +77700 POINT(40.90256911779187 73.96728942684685) bank77700 +77701 POINT(41.12615321633832 74.45916894678508) bank77701 +77702 POINT(40.43348322332509 74.59642312964023) bank77702 +77703 POINT(40.60304277850063 74.89027419205343) bank77703 +77704 POINT(40.91108028839116 74.6901181308058) bank77704 +77705 POINT(39.89616934055181 74.6973032798832) bank77705 +77706 POINT(41.324417236657936 75.00208628667203) bank77706 +77707 POINT(41.0688340269175 74.0373347088877) bank77707 +77708 POINT(40.30361297494969 74.12774954311843) bank77708 +77709 POINT(41.05202862224923 73.76886194493028) bank77709 +77710 POINT(40.3174650048204 74.54501631902912) bank77710 +77711 POINT(40.56130196397783 73.34413456116094) bank77711 +77712 POINT(40.99011218058494 74.57960710267002) bank77712 +77713 POINT(40.025996097515886 74.43631856309072) bank77713 +77714 POINT(41.64251295400018 73.75194617620271) bank77714 +77715 POINT(41.27676479082555 73.16829924063782) bank77715 +77716 POINT(39.887950071694824 73.40662203224026) bank77716 +77717 POINT(40.289005605909864 74.46327521695706) bank77717 +77718 POINT(41.312885456183594 73.34373416732576) bank77718 +77719 POINT(39.96193189243726 73.61070811653607) bank77719 +77720 POINT(41.53181407254056 73.07778583150342) bank77720 +77721 POINT(40.00067099382729 73.62611224858726) bank77721 +77722 POINT(41.69002732263297 73.58907232499166) bank77722 +77723 POINT(40.812079958561455 73.75021475754887) bank77723 +77724 POINT(39.97682026037301 73.2745571368389) bank77724 +77725 POINT(41.546817615162325 73.07297998747636) bank77725 +77726 POINT(40.70324843287359 74.70804116488931) bank77726 +77727 POINT(39.991227909563925 73.48611874320778) bank77727 +77728 POINT(40.98698193948684 73.83731292941924) bank77728 +77729 POINT(40.518295324767294 74.20549232646371) bank77729 +77730 POINT(40.48489072386316 73.04185028689902) bank77730 +77731 POINT(40.86869257772142 73.73992796094417) bank77731 +77732 POINT(39.959564198404586 74.73249592173772) bank77732 +77733 POINT(41.18756886459307 73.88544886754678) bank77733 +77734 POINT(40.93414142057211 74.2795593013037) bank77734 +77735 POINT(41.36571853420163 74.5947653485552) bank77735 +77736 POINT(39.72359923560268 74.89908212751634) bank77736 +77737 POINT(41.62918022699813 74.65846610750224) bank77737 +77738 POINT(40.33637846265613 73.30873551283969) bank77738 +77739 POINT(41.685011820999556 74.1328306492883) bank77739 +77740 POINT(40.97390242795383 74.01366269464843) bank77740 +77741 POINT(41.12172328227587 74.63485125540765) bank77741 +77742 POINT(41.4388046557871 73.93932147338597) bank77742 +77743 POINT(39.769649676457405 74.47662832032081) bank77743 +77744 POINT(40.59765296092846 73.03625809742357) bank77744 +77745 POINT(39.93551404798468 73.92715256657675) bank77745 +77746 POINT(41.46274073878989 73.17710859228104) bank77746 +77747 POINT(39.74469708532015 73.40590152748321) bank77747 +77748 POINT(40.23568002889523 73.06815708853895) bank77748 +77749 POINT(39.759576107631844 74.5724159787892) bank77749 +77750 POINT(41.56921301766049 74.04668516293737) bank77750 +77751 POINT(40.21611152628216 74.47398304642607) bank77751 +77752 POINT(40.11391491811025 74.62188442801286) bank77752 +77753 POINT(40.58917121980092 73.10325206965213) bank77753 +77754 POINT(40.07293960710276 73.5176830873844) bank77754 +77755 POINT(39.97287850753492 74.83665105743657) bank77755 +77756 POINT(41.20142300546273 73.93959665937317) bank77756 +77757 POINT(41.178975240062805 74.21701933632613) bank77757 +77758 POINT(40.918934480195844 73.04533803069252) bank77758 +77759 POINT(41.066139651605766 74.82185647085822) bank77759 +77760 POINT(39.76661749310726 73.22816546961667) bank77760 +77761 POINT(41.07150387643324 73.97103257127097) bank77761 +77762 POINT(39.950967333932645 73.1722506566121) bank77762 +77763 POINT(40.53140347295316 74.53609093251458) bank77763 +77764 POINT(40.063855359253004 74.72020049756304) bank77764 +77765 POINT(39.74064081314121 74.2699279021151) bank77765 +77766 POINT(40.604724824619844 73.692883282253) bank77766 +77767 POINT(39.84936605419918 74.88609637687264) bank77767 +77768 POINT(40.53314500373665 73.89384533483133) bank77768 +77769 POINT(40.74563192575094 73.14012662167454) bank77769 +77770 POINT(41.26700634871068 73.17941047358013) bank77770 +77771 POINT(39.95749884742477 74.73827171379045) bank77771 +77772 POINT(40.3714124323493 73.26943644617607) bank77772 +77773 POINT(41.532685714721204 73.27388057322773) bank77773 +77774 POINT(40.72824382673823 74.54136419637483) bank77774 +77775 POINT(40.64307147808228 73.71173149665219) bank77775 +77776 POINT(39.91449808715122 73.33977080519914) bank77776 +77777 POINT(41.26738451289507 73.30763123201002) bank77777 +77778 POINT(40.76185264314211 73.03164382691826) bank77778 +77779 POINT(40.40430678572872 74.18095170467222) bank77779 +77780 POINT(41.33831583357631 74.37550651250497) bank77780 +77781 POINT(41.37519403746766 74.67076967691385) bank77781 +77782 POINT(40.772469014149465 74.94286421909017) bank77782 +77783 POINT(39.9354604594406 73.1581486577852) bank77783 +77784 POINT(39.831632365434075 73.01407306260954) bank77784 +77785 POINT(40.13986253431444 73.75510411511364) bank77785 +77786 POINT(41.31595218397578 74.11847201913069) bank77786 +77787 POINT(40.65567867440905 74.25763166053883) bank77787 +77788 POINT(41.392165211363825 74.7829930243937) bank77788 +77789 POINT(41.328360177629754 73.57547102584093) bank77789 +77790 POINT(41.67733082037864 74.93064351283091) bank77790 +77791 POINT(40.02900951983441 74.66974782864052) bank77791 +77792 POINT(39.815101876933625 74.42352603885159) bank77792 +77793 POINT(40.09032202052121 74.07631222092216) bank77793 +77794 POINT(41.462801229394444 74.74667347237485) bank77794 +77795 POINT(41.51260319012829 74.7664806612502) bank77795 +77796 POINT(40.374475029715065 74.92658755937791) bank77796 +77797 POINT(40.67941661982584 74.75669166957873) bank77797 +77798 POINT(40.04510109980913 73.08832652460639) bank77798 +77799 POINT(40.715612000194966 74.91957492594234) bank77799 +77800 POINT(41.12824605056637 73.17162175177927) bank77800 +77801 POINT(41.06537700191602 73.90472474076844) bank77801 +77802 POINT(41.62169380318779 74.20517833296286) bank77802 +77803 POINT(39.777658115856816 74.97413189894436) bank77803 +77804 POINT(39.77313834724153 74.6620891062096) bank77804 +77805 POINT(40.43722407297682 74.01561066738421) bank77805 +77806 POINT(39.90934923656631 73.05052907546057) bank77806 +77807 POINT(41.70842967839492 74.50199695346667) bank77807 +77808 POINT(41.35321933534187 74.58346009734854) bank77808 +77809 POINT(41.33358955736145 73.59071186901953) bank77809 +77810 POINT(40.558985034248956 73.15295050726597) bank77810 +77811 POINT(40.395341553744345 74.6879937345615) bank77811 +77812 POINT(40.63709724381974 73.02489438771606) bank77812 +77813 POINT(41.12288706313447 74.77366104967835) bank77813 +77814 POINT(40.27142271619471 73.0428133286116) bank77814 +77815 POINT(40.521272126012036 74.86921110041014) bank77815 +77816 POINT(41.68521263793418 74.81955928161157) bank77816 +77817 POINT(40.85799643711082 74.86397464676608) bank77817 +77818 POINT(41.047966973636385 73.40765863658626) bank77818 +77819 POINT(40.720246713484414 74.73347274987866) bank77819 +77820 POINT(39.86053647201586 73.43274547696157) bank77820 +77821 POINT(40.97692245585441 74.76993458208055) bank77821 +77822 POINT(40.09466185896001 74.93977296866582) bank77822 +77823 POINT(40.551584964525674 74.85480386880171) bank77823 +77824 POINT(40.035261912643676 74.91811070790632) bank77824 +77825 POINT(39.742223990333756 74.46251080206916) bank77825 +77826 POINT(40.83182595203536 74.9143432610126) bank77826 +77827 POINT(40.41476464925042 73.40547998482539) bank77827 +77828 POINT(39.936244257863386 73.81783337632737) bank77828 +77829 POINT(40.80538365188843 74.31572580969622) bank77829 +77830 POINT(40.50617698500662 74.18391123357469) bank77830 +77831 POINT(39.886581747432444 73.86480743892595) bank77831 +77832 POINT(40.22006979767898 74.99937915779321) bank77832 +77833 POINT(40.91746967884712 74.50868913683748) bank77833 +77834 POINT(39.8941926648475 74.89845515357153) bank77834 +77835 POINT(41.63247696070641 73.792480924575) bank77835 +77836 POINT(40.7646073208122 73.27877574967759) bank77836 +77837 POINT(41.0420758221276 73.92917803918809) bank77837 +77838 POINT(40.5000014173405 73.9549374586294) bank77838 +77839 POINT(40.030726278624066 73.02621107735608) bank77839 +77840 POINT(41.65610213387657 74.28966034646191) bank77840 +77841 POINT(40.00983084741327 73.79588112009333) bank77841 +77842 POINT(40.98486335174358 73.33016956490613) bank77842 +77843 POINT(39.715559302604014 74.4896130732324) bank77843 +77844 POINT(40.69621899691222 73.13679098690538) bank77844 +77845 POINT(41.64849355250782 74.12733510004186) bank77845 +77846 POINT(41.54171730203925 73.5462812763169) bank77846 +77847 POINT(41.5331814015522 74.371533129169) bank77847 +77848 POINT(39.877562662996496 73.76512363104757) bank77848 +77849 POINT(40.76297439579914 73.04203101554087) bank77849 +77850 POINT(41.39063998804329 74.15322023964501) bank77850 +77851 POINT(40.03064288600642 73.10108408149132) bank77851 +77852 POINT(41.31499755933261 74.36538522700711) bank77852 +77853 POINT(40.375569350467806 73.98992155169195) bank77853 +77854 POINT(40.43243526122142 73.00919693422709) bank77854 +77855 POINT(39.71860931623999 73.80045230834472) bank77855 +77856 POINT(40.14023539937917 73.39819075033466) bank77856 +77857 POINT(41.1786584910916 73.27312972779767) bank77857 +77858 POINT(40.95082444857192 73.37549564538388) bank77858 +77859 POINT(40.20027751548558 73.56432394844643) bank77859 +77860 POINT(41.50938686455484 73.98823553415926) bank77860 +77861 POINT(40.82634391006152 74.70896236600042) bank77861 +77862 POINT(40.7170262433615 74.80461800244458) bank77862 +77863 POINT(40.387744055652725 73.22828964589232) bank77863 +77864 POINT(40.14757922388926 74.1657195910563) bank77864 +77865 POINT(39.819138475953366 74.33683713664561) bank77865 +77866 POINT(41.328558234845424 74.27275410967223) bank77866 +77867 POINT(41.32232298902872 73.57228553596677) bank77867 +77868 POINT(40.60692980300112 73.03198029916234) bank77868 +77869 POINT(40.43677282450704 74.72231546506642) bank77869 +77870 POINT(40.782054695166934 74.83215390762558) bank77870 +77871 POINT(40.7799830345216 74.45851284959758) bank77871 +77872 POINT(40.05022573630848 74.27096751078057) bank77872 +77873 POINT(41.54545291351209 74.90601788781656) bank77873 +77874 POINT(41.157458406252125 74.52563870649223) bank77874 +77875 POINT(40.217670906268694 74.00204979594152) bank77875 +77876 POINT(40.80485887630249 73.90879411229785) bank77876 +77877 POINT(40.7148741672124 73.90545212063746) bank77877 +77878 POINT(40.91807836387192 74.1182712346021) bank77878 +77879 POINT(40.93721753996581 74.46469133541189) bank77879 +77880 POINT(39.882325186291034 74.45397089619695) bank77880 +77881 POINT(41.58345584594113 73.74890619676358) bank77881 +77882 POINT(40.78189415709947 73.07279746631377) bank77882 +77883 POINT(41.665182632603575 74.40622352940764) bank77883 +77884 POINT(40.23554938772579 73.09961260240311) bank77884 +77885 POINT(41.1887556923358 73.52820798435152) bank77885 +77886 POINT(40.25800368995625 73.94424745041917) bank77886 +77887 POINT(40.20788790792919 74.13947938276817) bank77887 +77888 POINT(41.47553231009682 74.75747986160225) bank77888 +77889 POINT(40.359190403296566 74.44652876931809) bank77889 +77890 POINT(40.421593056294405 73.13238036443994) bank77890 +77891 POINT(40.509220367942085 73.23520538364906) bank77891 +77892 POINT(41.16827164107425 73.68934296330266) bank77892 +77893 POINT(41.2817084840879 74.07563211773638) bank77893 +77894 POINT(40.29581467763219 73.08910181991428) bank77894 +77895 POINT(41.4428367794035 73.08172977662683) bank77895 +77896 POINT(41.65129533870374 74.34036924949018) bank77896 +77897 POINT(40.11625926639588 74.24521083944644) bank77897 +77898 POINT(41.39511529068232 74.35929984760091) bank77898 +77899 POINT(40.56527098002876 74.70058275067082) bank77899 +77900 POINT(40.137870740512625 73.95926845984958) bank77900 +77901 POINT(40.241311714537034 73.59530862176854) bank77901 +77902 POINT(41.03638147912368 74.5415279784633) bank77902 +77903 POINT(41.161383542975116 74.91018167356162) bank77903 +77904 POINT(41.45425184863147 74.82558004765552) bank77904 +77905 POINT(40.30111380039131 74.6612626164927) bank77905 +77906 POINT(39.812318903615974 74.03543669248262) bank77906 +77907 POINT(40.24057735281673 74.68852250105218) bank77907 +77908 POINT(40.93723200711722 73.91336781416297) bank77908 +77909 POINT(40.652250518889275 74.7349796217122) bank77909 +77910 POINT(39.72900078544111 74.24530531461022) bank77910 +77911 POINT(40.64218242981115 73.42726139545451) bank77911 +77912 POINT(41.19591082134679 74.71118238625519) bank77912 +77913 POINT(40.050090602585236 74.67341643970349) bank77913 +77914 POINT(39.965020457315646 73.44832632414793) bank77914 +77915 POINT(39.875476958312966 73.99653391076559) bank77915 +77916 POINT(41.18702574457437 74.40896244630234) bank77916 +77917 POINT(40.22957905644163 73.45305905349814) bank77917 +77918 POINT(40.40148015069287 73.50188132337814) bank77918 +77919 POINT(41.53762059094389 74.14724962206641) bank77919 +77920 POINT(40.58770755960439 74.18197204868106) bank77920 +77921 POINT(41.53634187985131 73.83973407157836) bank77921 +77922 POINT(41.528666849275915 73.04367132611189) bank77922 +77923 POINT(40.74539876335113 74.47067565225325) bank77923 +77924 POINT(41.6113707039145 74.74843941225684) bank77924 +77925 POINT(41.38128440791128 73.5500575829768) bank77925 +77926 POINT(41.107042853034386 73.9333664379084) bank77926 +77927 POINT(40.69025019304432 74.1752695094848) bank77927 +77928 POINT(40.18823120134823 73.1094958888416) bank77928 +77929 POINT(40.40886459800165 73.20381952345359) bank77929 +77930 POINT(40.08657364357485 74.53634908968868) bank77930 +77931 POINT(40.58183167397805 74.66525215715585) bank77931 +77932 POINT(40.13963367894455 74.02958973697949) bank77932 +77933 POINT(41.14493092932143 74.12841907664283) bank77933 +77934 POINT(39.91640297616458 74.21846714231106) bank77934 +77935 POINT(41.269288697499306 74.10701834661381) bank77935 +77936 POINT(41.40892848746027 74.70587629471608) bank77936 +77937 POINT(41.29473648244381 74.95038302241655) bank77937 +77938 POINT(41.399041397357266 74.91481641199483) bank77938 +77939 POINT(40.03395931654503 74.77794756446448) bank77939 +77940 POINT(40.85934513114547 73.50508139874832) bank77940 +77941 POINT(40.751459201876344 73.57199868314194) bank77941 +77942 POINT(41.258911945292546 74.19408779482505) bank77942 +77943 POINT(41.60024685550357 73.75410531265416) bank77943 +77944 POINT(41.12616941540206 74.52380366183435) bank77944 +77945 POINT(40.777320562726345 73.45550285938056) bank77945 +77946 POINT(40.799077859105694 73.61318861489114) bank77946 +77947 POINT(39.912510127208094 74.93733367923417) bank77947 +77948 POINT(41.61945825230387 73.8971008241133) bank77948 +77949 POINT(40.68261893644909 74.21259603221297) bank77949 +77950 POINT(41.253345065239806 73.82011189064099) bank77950 +77951 POINT(41.11935222215338 73.95327387953745) bank77951 +77952 POINT(40.921894653642745 74.13187920389436) bank77952 +77953 POINT(40.30546849431904 74.57600120122451) bank77953 +77954 POINT(40.614110203954944 73.69264407925345) bank77954 +77955 POINT(40.60140762770609 74.55724971663668) bank77955 +77956 POINT(41.034473512056444 74.05110653753724) bank77956 +77957 POINT(40.35333180885792 73.14853475093733) bank77957 +77958 POINT(40.332400791912086 74.73627189067356) bank77958 +77959 POINT(39.969554542383044 73.355852642227) bank77959 +77960 POINT(41.224466441307726 73.04718967795151) bank77960 +77961 POINT(41.25857173226168 74.087957142405) bank77961 +77962 POINT(41.09250705820332 73.3320285504076) bank77962 +77963 POINT(40.48661819608364 74.72065905818977) bank77963 +77964 POINT(40.167596207761136 74.40639269775441) bank77964 +77965 POINT(41.33598752218144 74.5672389072688) bank77965 +77966 POINT(41.39080030879167 73.19264596547025) bank77966 +77967 POINT(40.91991821564304 73.01584365358985) bank77967 +77968 POINT(41.43392354588486 73.04086808572495) bank77968 +77969 POINT(39.93252138628542 73.46826650162741) bank77969 +77970 POINT(40.71426353195866 73.05735451705196) bank77970 +77971 POINT(39.79087706474158 74.4997023580665) bank77971 +77972 POINT(41.06781775083226 74.12309281028475) bank77972 +77973 POINT(40.3563517306681 73.9990684011635) bank77973 +77974 POINT(40.17970747567895 73.65349381382129) bank77974 +77975 POINT(39.790851788027524 73.76730665763428) bank77975 +77976 POINT(40.77082216110105 74.2247833407102) bank77976 +77977 POINT(40.845713458844486 73.41092430527026) bank77977 +77978 POINT(40.57525995577518 74.70860911342554) bank77978 +77979 POINT(40.602206792851746 74.13305571197898) bank77979 +77980 POINT(40.71982379422805 73.64810396674838) bank77980 +77981 POINT(40.728582085414715 73.70245168516983) bank77981 +77982 POINT(41.43712859524449 74.00844747577479) bank77982 +77983 POINT(40.80626286789123 73.5887179542765) bank77983 +77984 POINT(41.55071811389268 74.51501974831378) bank77984 +77985 POINT(40.65058975478528 73.6737658380907) bank77985 +77986 POINT(41.45854440908513 73.95077072595632) bank77986 +77987 POINT(40.88227490261652 74.37338398700842) bank77987 +77988 POINT(40.8027963616006 73.24886041171611) bank77988 +77989 POINT(39.90221701194463 73.30088433758746) bank77989 +77990 POINT(39.83315880605608 74.64950512484303) bank77990 +77991 POINT(39.9663499049672 73.85434320937462) bank77991 +77992 POINT(40.11024426032418 74.1795271193313) bank77992 +77993 POINT(39.85150765575586 74.244157634441) bank77993 +77994 POINT(41.347492803139836 74.16812400110797) bank77994 +77995 POINT(41.463063714656464 73.36727686347135) bank77995 +77996 POINT(40.9230924031066 73.90290444769221) bank77996 +77997 POINT(41.66071753824603 74.18538448313758) bank77997 +77998 POINT(40.845998513784906 73.21988255964594) bank77998 +77999 POINT(40.43502706288375 74.77265712066887) bank77999 +78000 POINT(40.704052929839456 74.90867792773412) bank78000 +78001 POINT(41.618507224877675 74.82351904206199) bank78001 +78002 POINT(41.54026725633154 74.14717292180498) bank78002 +78003 POINT(40.06133391135996 73.06851628617625) bank78003 +78004 POINT(41.65245591488183 74.66467135605829) bank78004 +78005 POINT(39.94546835347669 73.47061749622691) bank78005 +78006 POINT(41.023042008393354 74.27250100567905) bank78006 +78007 POINT(40.741337201217 74.40156277985244) bank78007 +78008 POINT(41.66616089260179 74.17328068120482) bank78008 +78009 POINT(40.202595932541264 74.86719654447757) bank78009 +78010 POINT(40.069132033159974 74.5901819925995) bank78010 +78011 POINT(40.023100971089704 74.59700522732896) bank78011 +78012 POINT(40.54521218830057 74.95454759784948) bank78012 +78013 POINT(41.64489936052459 73.05718313870915) bank78013 +78014 POINT(40.025287324284115 73.23171002305065) bank78014 +78015 POINT(40.747296891751475 73.00955200843217) bank78015 +78016 POINT(40.77983056795858 74.45563179629852) bank78016 +78017 POINT(40.223183129554315 73.85269507820871) bank78017 +78018 POINT(39.97297597550805 74.34262672930214) bank78018 +78019 POINT(40.703520561216926 73.62859609427007) bank78019 +78020 POINT(41.452906214084614 74.50782166268381) bank78020 +78021 POINT(41.19096276731581 73.71766781002975) bank78021 +78022 POINT(39.97863074214961 73.12231994760026) bank78022 +78023 POINT(41.19420296413301 73.1526789837256) bank78023 +78024 POINT(40.67703550246298 74.04931531232009) bank78024 +78025 POINT(41.097083520512754 73.59313093619613) bank78025 +78026 POINT(41.66633494177841 74.8404453747244) bank78026 +78027 POINT(41.54656158114607 74.50497740172607) bank78027 +78028 POINT(41.62762366531057 74.89841723055217) bank78028 +78029 POINT(41.29343099345451 73.12983107029223) bank78029 +78030 POINT(41.16965424538359 74.85261140806277) bank78030 +78031 POINT(41.23347914360707 74.77599216092682) bank78031 +78032 POINT(40.376200554192266 73.10698112651869) bank78032 +78033 POINT(40.841547085470935 73.7525689794193) bank78033 +78034 POINT(41.13582785550171 74.22132775669661) bank78034 +78035 POINT(40.88314027771766 73.03080781989308) bank78035 +78036 POINT(40.46879562962886 73.05933713526323) bank78036 +78037 POINT(40.058415865505765 74.37548706415305) bank78037 +78038 POINT(41.31473951520739 74.5337279454513) bank78038 +78039 POINT(40.23104982848995 74.97550777629064) bank78039 +78040 POINT(41.30996847084138 73.33043480560595) bank78040 +78041 POINT(41.22552428680849 74.70861139682486) bank78041 +78042 POINT(40.24544469940958 74.15835292010296) bank78042 +78043 POINT(39.76458052370345 73.40370998626587) bank78043 +78044 POINT(41.2905463830405 73.16781805940659) bank78044 +78045 POINT(40.887466586550055 74.26127634177354) bank78045 +78046 POINT(41.38437941983644 73.63280192341463) bank78046 +78047 POINT(41.53554705732433 73.12054321892357) bank78047 +78048 POINT(40.38524200061985 74.79531251740379) bank78048 +78049 POINT(39.95891155050656 74.97764662336816) bank78049 +78050 POINT(40.88617946766528 73.8235000088806) bank78050 +78051 POINT(40.47907676865286 74.01195008749824) bank78051 +78052 POINT(41.451318597261384 74.21049047053009) bank78052 +78053 POINT(40.79342402342994 73.52892586894882) bank78053 +78054 POINT(40.10734503716493 74.7809160974799) bank78054 +78055 POINT(41.422837017535244 73.10398892824107) bank78055 +78056 POINT(41.00124528153349 74.88586551055705) bank78056 +78057 POINT(41.06912452191855 74.8204248158666) bank78057 +78058 POINT(40.81511370693897 73.24636768434175) bank78058 +78059 POINT(41.662614211739005 74.44661546531377) bank78059 +78060 POINT(39.794629522445135 73.69400118199329) bank78060 +78061 POINT(41.562659935244454 74.4072391998575) bank78061 +78062 POINT(40.69511027111864 74.00764852651241) bank78062 +78063 POINT(40.07836917100552 73.23331063274604) bank78063 +78064 POINT(39.78776830801596 73.67657429067341) bank78064 +78065 POINT(41.70245750681934 73.61102590567647) bank78065 +78066 POINT(40.06059406576012 73.45741094807448) bank78066 +78067 POINT(40.44949761989321 74.06178766556515) bank78067 +78068 POINT(40.66330544513305 73.7798116144502) bank78068 +78069 POINT(40.90511955914113 74.67068767902975) bank78069 +78070 POINT(41.45383830247322 73.33692930183716) bank78070 +78071 POINT(40.58064130728999 73.31988454407283) bank78071 +78072 POINT(39.90374984054815 74.56275978641044) bank78072 +78073 POINT(41.64057293545685 74.38768402905652) bank78073 +78074 POINT(41.379797870249476 74.58784439243982) bank78074 +78075 POINT(40.186300647758095 74.34971855350442) bank78075 +78076 POINT(40.04277820159056 74.78744945445548) bank78076 +78077 POINT(41.398933809632254 74.28298867558951) bank78077 +78078 POINT(41.17348248574871 75.00317680372801) bank78078 +78079 POINT(40.41062817818128 73.25513220596095) bank78079 +78080 POINT(41.37168522923122 74.4750455145582) bank78080 +78081 POINT(40.944212205818175 73.18627284033944) bank78081 +78082 POINT(40.22212385877741 73.05007520965975) bank78082 +78083 POINT(41.07350158278752 73.38528828770674) bank78083 +78084 POINT(41.35578163056233 74.34206810115131) bank78084 +78085 POINT(39.85787047260313 74.90680679282467) bank78085 +78086 POINT(41.33639099261641 74.52178850175825) bank78086 +78087 POINT(41.49307786576881 73.73265753548993) bank78087 +78088 POINT(39.741047818949745 73.87221054346594) bank78088 +78089 POINT(41.60935103637671 73.97849339839784) bank78089 +78090 POINT(41.020290962604086 74.45486110222154) bank78090 +78091 POINT(40.93971531083302 74.73002463344675) bank78091 +78092 POINT(40.00989842309115 74.61676142331797) bank78092 +78093 POINT(40.64154068053626 73.59047616111224) bank78093 +78094 POINT(40.22866777355166 73.6369970383893) bank78094 +78095 POINT(40.19856846706795 74.13632221525248) bank78095 +78096 POINT(41.14987396348798 73.54781086630484) bank78096 +78097 POINT(39.920554106000004 74.77158348436936) bank78097 +78098 POINT(41.58636764327552 74.10155270796358) bank78098 +78099 POINT(40.588999863367484 73.15458921673661) bank78099 +78100 POINT(39.87629107414342 73.2290694433199) bank78100 +78101 POINT(41.08225293499806 73.67412456297308) bank78101 +78102 POINT(39.98706029642448 73.12632741049366) bank78102 +78103 POINT(40.32256404844407 73.54266624736324) bank78103 +78104 POINT(40.25677700964865 73.57315183568) bank78104 +78105 POINT(40.23866937990903 74.62103962109181) bank78105 +78106 POINT(40.591243313051336 74.72190223382134) bank78106 +78107 POINT(39.894136819956536 74.85900945880587) bank78107 +78108 POINT(41.35813372637499 74.27517961631511) bank78108 +78109 POINT(41.457032976679926 73.46657399056565) bank78109 +78110 POINT(40.8025933974105 74.72041802722823) bank78110 +78111 POINT(40.51078683037896 74.85208932696398) bank78111 +78112 POINT(41.34715901888649 74.06734090192684) bank78112 +78113 POINT(41.30483706801415 74.67530932399185) bank78113 +78114 POINT(40.91058802819238 74.94006434372506) bank78114 +78115 POINT(41.42152825079391 74.67183159548188) bank78115 +78116 POINT(39.95032230697992 74.92207803604835) bank78116 +78117 POINT(41.00283222513562 74.64255862994355) bank78117 +78118 POINT(41.5077256231946 74.39509228179134) bank78118 +78119 POINT(40.56150728637276 73.29391477413873) bank78119 +78120 POINT(40.5331235018364 73.4889995029197) bank78120 +78121 POINT(40.59007367902513 73.50262525077133) bank78121 +78122 POINT(40.756955194081335 74.62188065839318) bank78122 +78123 POINT(40.001943503376744 74.84399448527255) bank78123 +78124 POINT(39.8569686397659 73.95126226167949) bank78124 +78125 POINT(41.19892717648627 73.87267925620604) bank78125 +78126 POINT(41.319284522314696 74.50882316463402) bank78126 +78127 POINT(40.517016660325666 74.31385007127739) bank78127 +78128 POINT(41.29736498654939 74.93696459198941) bank78128 +78129 POINT(41.68419851524534 73.28681207939931) bank78129 +78130 POINT(40.857534924626435 73.28295341472996) bank78130 +78131 POINT(40.16608472033959 74.20973760798456) bank78131 +78132 POINT(41.58221331594846 73.69897226025677) bank78132 +78133 POINT(41.2644943872048 73.64349192367047) bank78133 +78134 POINT(40.83670420348969 73.47204263785243) bank78134 +78135 POINT(41.196487956991646 74.36054139786296) bank78135 +78136 POINT(41.049598630119725 73.18219920222643) bank78136 +78137 POINT(41.367888476081994 73.56748862241001) bank78137 +78138 POINT(40.283112687968504 74.0716514862982) bank78138 +78139 POINT(40.73378491058241 73.87753317515828) bank78139 +78140 POINT(41.38216661363521 74.9418461110525) bank78140 +78141 POINT(41.42192743261036 73.54604685695737) bank78141 +78142 POINT(40.95504118515391 74.2272921880959) bank78142 +78143 POINT(41.32224580206279 74.80292698190104) bank78143 +78144 POINT(41.0742917255974 73.55877487658537) bank78144 +78145 POINT(40.73665316574816 74.9890635322172) bank78145 +78146 POINT(39.84018245027085 73.47649004723456) bank78146 +78147 POINT(41.16579161233893 74.25176213010438) bank78147 +78148 POINT(40.544417869218385 74.96247645729554) bank78148 +78149 POINT(40.005419215928065 74.84924731642944) bank78149 +78150 POINT(40.90875400188962 73.68176215107474) bank78150 +78151 POINT(41.41397376153135 74.48905700382777) bank78151 +78152 POINT(39.78219281869405 74.98015921995629) bank78152 +78153 POINT(41.309583353470835 74.63643911938394) bank78153 +78154 POINT(41.687217853780325 74.73040250045953) bank78154 +78155 POINT(41.71159955060212 74.05837351313748) bank78155 +78156 POINT(40.801497201675154 73.42666218716164) bank78156 +78157 POINT(39.75795350024995 74.15919527494285) bank78157 +78158 POINT(41.4452287505676 74.37593869098636) bank78158 +78159 POINT(40.21851201539281 73.0224261165493) bank78159 +78160 POINT(39.834791325227336 73.107568650045) bank78160 +78161 POINT(40.40663984956474 73.78992406615902) bank78161 +78162 POINT(40.004006810733635 74.37075862851326) bank78162 +78163 POINT(41.33569552113587 74.65201534445544) bank78163 +78164 POINT(40.7415384285954 74.90918666818146) bank78164 +78165 POINT(39.79202408877884 73.24067081435892) bank78165 +78166 POINT(40.494550418407385 74.5562151497452) bank78166 +78167 POINT(41.46723818113655 74.88867798222344) bank78167 +78168 POINT(40.45165666546149 73.82824410345285) bank78168 +78169 POINT(40.68425540885169 74.80707654249102) bank78169 +78170 POINT(41.429225731017034 73.49429257663228) bank78170 +78171 POINT(40.442192650941315 73.5233066133661) bank78171 +78172 POINT(40.42835044741962 74.67887144554047) bank78172 +78173 POINT(40.16508794868117 74.71006353448547) bank78173 +78174 POINT(40.34555734150643 73.15237445911274) bank78174 +78175 POINT(40.388545020623745 73.60496657677704) bank78175 +78176 POINT(40.42005492988546 74.94108352283034) bank78176 +78177 POINT(39.83727145725838 74.19920448870205) bank78177 +78178 POINT(40.76768350745734 74.67662191101604) bank78178 +78179 POINT(40.72654164473166 73.8069003008756) bank78179 +78180 POINT(40.3677076586384 73.65904859715057) bank78180 +78181 POINT(41.06232778109283 73.04783808934798) bank78181 +78182 POINT(40.96848213032746 74.69097904326927) bank78182 +78183 POINT(41.472804365597426 74.41890803980182) bank78183 +78184 POINT(40.06349749833043 73.30468754914524) bank78184 +78185 POINT(40.471906418355196 73.37049911783221) bank78185 +78186 POINT(40.34503950757939 74.79721109527706) bank78186 +78187 POINT(41.011106904521796 73.16721345270238) bank78187 +78188 POINT(40.48492954814657 73.66844563015871) bank78188 +78189 POINT(41.543014893379386 73.54221124587575) bank78189 +78190 POINT(40.54454383789368 73.16514579879038) bank78190 +78191 POINT(40.37690047371152 74.53369274437385) bank78191 +78192 POINT(40.46250246953516 73.11168612713976) bank78192 +78193 POINT(40.60321069954775 74.39331574735488) bank78193 +78194 POINT(39.936478418666916 73.55964157295611) bank78194 +78195 POINT(39.957667781448116 73.23495155624764) bank78195 +78196 POINT(39.94806934899996 73.6682859457007) bank78196 +78197 POINT(40.73434551873532 74.33972290645443) bank78197 +78198 POINT(40.605520670725625 74.39295044896488) bank78198 +78199 POINT(39.71521800293543 73.97872269081074) bank78199 +78200 POINT(41.47635757395777 73.20974050187546) bank78200 +78201 POINT(40.88116526085999 73.14058685979822) bank78201 +78202 POINT(40.17742213391807 74.49565198603041) bank78202 +78203 POINT(40.86914044801208 74.57422086636029) bank78203 +78204 POINT(40.37884674116619 73.00960871165981) bank78204 +78205 POINT(41.005645997189006 74.59054032690847) bank78205 +78206 POINT(40.23978131398207 74.17615731731495) bank78206 +78207 POINT(39.92351352870787 73.83415883713853) bank78207 +78208 POINT(41.11796823046112 73.28247557410941) bank78208 +78209 POINT(40.421625762260696 73.33357248826599) bank78209 +78210 POINT(39.796353317864856 74.16768072449968) bank78210 +78211 POINT(41.09657892660605 73.0174987163904) bank78211 +78212 POINT(40.504428402353234 73.21074382597578) bank78212 +78213 POINT(40.83341543243161 73.67867918687617) bank78213 +78214 POINT(41.039067318642125 73.70886927756288) bank78214 +78215 POINT(39.88369450897354 74.16096460166534) bank78215 +78216 POINT(39.9667094568393 73.099080840517) bank78216 +78217 POINT(40.15467414220829 73.12957596091016) bank78217 +78218 POINT(40.965896752357885 73.00646525880602) bank78218 +78219 POINT(40.72584539579568 73.871397508499) bank78219 +78220 POINT(40.621422622064635 74.2621050370419) bank78220 +78221 POINT(40.26128525769107 73.56878547598414) bank78221 +78222 POINT(41.190478203523014 73.85431775199373) bank78222 +78223 POINT(41.40243659076786 74.6165497005271) bank78223 +78224 POINT(40.17245456920849 74.68133883948562) bank78224 +78225 POINT(40.90629067198324 74.1606177330018) bank78225 +78226 POINT(40.455899213865344 74.74666749034131) bank78226 +78227 POINT(40.36599786881784 74.15623848277048) bank78227 +78228 POINT(40.50071225586515 73.2708195320218) bank78228 +78229 POINT(41.537938391878164 74.0518501672668) bank78229 +78230 POINT(40.88951278164399 73.59581516908355) bank78230 +78231 POINT(41.65837324958672 73.29165488188409) bank78231 +78232 POINT(40.37982235427799 73.66255729434589) bank78232 +78233 POINT(41.54451648323689 73.89501671104811) bank78233 +78234 POINT(40.79835920959601 74.78856640293539) bank78234 +78235 POINT(39.873756639810495 73.30240293888714) bank78235 +78236 POINT(41.08808769642829 74.97155316331708) bank78236 +78237 POINT(40.858043582441056 74.04920624451938) bank78237 +78238 POINT(40.26827994839272 74.24442559138299) bank78238 +78239 POINT(40.770098335028194 74.52107288455753) bank78239 +78240 POINT(40.54124266927388 74.90733964770853) bank78240 +78241 POINT(41.349716221022376 74.21901245317162) bank78241 +78242 POINT(40.956674713059776 74.98161910327856) bank78242 +78243 POINT(39.81085234539653 73.57453688840526) bank78243 +78244 POINT(40.34454539125637 74.18978667567018) bank78244 +78245 POINT(41.36518278473626 74.44009811958024) bank78245 +78246 POINT(41.38510716185709 73.12524279803708) bank78246 +78247 POINT(39.8814259821492 74.56462896298143) bank78247 +78248 POINT(41.6385297255523 73.57187027280375) bank78248 +78249 POINT(39.86145966880726 74.16035244838704) bank78249 +78250 POINT(40.19366266482532 74.93226457146011) bank78250 +78251 POINT(40.31690520285925 73.34196465965516) bank78251 +78252 POINT(39.780871538234145 73.63627751018895) bank78252 +78253 POINT(41.03928370863011 74.82705083728291) bank78253 +78254 POINT(39.71513129442124 74.94800427469758) bank78254 +78255 POINT(40.18733090812973 73.5333646524931) bank78255 +78256 POINT(40.046310593902156 74.95450490383749) bank78256 +78257 POINT(40.56071161499822 73.66471079288071) bank78257 +78258 POINT(40.440088673663375 73.09359724619793) bank78258 +78259 POINT(39.71527279952437 74.89126895795975) bank78259 +78260 POINT(41.1994955731531 74.2899713701369) bank78260 +78261 POINT(41.6707627836206 74.12176693215935) bank78261 +78262 POINT(40.30239477195826 73.20085270750398) bank78262 +78263 POINT(39.89450113719795 74.09744391794419) bank78263 +78264 POINT(41.29612235148304 74.62914514698699) bank78264 +78265 POINT(39.893878730693956 74.64512050449682) bank78265 +78266 POINT(40.43925443144238 74.56102643396756) bank78266 +78267 POINT(41.30070349368479 73.64269321229025) bank78267 +78268 POINT(40.72204637194917 74.99111871256395) bank78268 +78269 POINT(41.518460154780556 73.43794862317367) bank78269 +78270 POINT(40.21366791360852 73.47713808673781) bank78270 +78271 POINT(41.30926854484322 73.89849599310178) bank78271 +78272 POINT(40.518559678292085 74.80155381694861) bank78272 +78273 POINT(41.570568069971046 73.07274754342825) bank78273 +78274 POINT(40.39753455303376 74.64488910413355) bank78274 +78275 POINT(40.615857679560044 73.26672424089686) bank78275 +78276 POINT(40.89612527173339 74.12205516790739) bank78276 +78277 POINT(40.67704502967168 73.83564824764143) bank78277 +78278 POINT(41.3501778896257 74.02354134177124) bank78278 +78279 POINT(40.851288711951895 73.65421037780216) bank78279 +78280 POINT(40.00209617804249 74.82469064588408) bank78280 +78281 POINT(41.151929907685 74.55424171705661) bank78281 +78282 POINT(41.46684630764915 74.53478143997104) bank78282 +78283 POINT(40.95437180000175 74.80452628960619) bank78283 +78284 POINT(40.53051616917077 73.83953707414696) bank78284 +78285 POINT(41.123203344816055 74.8227946659047) bank78285 +78286 POINT(40.376276230630644 73.44328572757787) bank78286 +78287 POINT(41.01042405865493 73.08435871926413) bank78287 +78288 POINT(40.68176949475876 74.21705145903132) bank78288 +78289 POINT(41.64744313675578 73.84653969466193) bank78289 +78290 POINT(41.43876958674915 73.7660239489042) bank78290 +78291 POINT(40.02461561385495 74.20410462995142) bank78291 +78292 POINT(40.873496008286146 73.88608672296894) bank78292 +78293 POINT(41.369210494385975 73.75849920130297) bank78293 +78294 POINT(41.57432317893637 74.23829809575564) bank78294 +78295 POINT(40.4140206471804 74.70680747231991) bank78295 +78296 POINT(39.90498338575105 74.6796227832152) bank78296 +78297 POINT(40.68944718314395 74.21936856025) bank78297 +78298 POINT(40.593147260430726 73.13076261828137) bank78298 +78299 POINT(40.19574007804281 73.87115038577134) bank78299 +78300 POINT(40.80551829344806 73.52466134065084) bank78300 +78301 POINT(40.74757453722836 73.65045089778432) bank78301 +78302 POINT(41.51375625861954 73.05206019406455) bank78302 +78303 POINT(41.45920999117456 74.42396110255301) bank78303 +78304 POINT(41.02313498611509 74.1525323210793) bank78304 +78305 POINT(41.2896459603048 73.40583799253368) bank78305 +78306 POINT(40.14121556612359 74.72931022426413) bank78306 +78307 POINT(41.443833488477246 74.57762414984988) bank78307 +78308 POINT(40.68245967047157 74.30848714229015) bank78308 +78309 POINT(40.71330248325105 74.14491154348426) bank78309 +78310 POINT(39.84380204319562 73.44996589832354) bank78310 +78311 POINT(40.123616999869014 74.85485616796404) bank78311 +78312 POINT(39.8764304662795 73.49851465576013) bank78312 +78313 POINT(40.99610287545097 73.70033345957363) bank78313 +78314 POINT(40.457886788852825 74.82243140625778) bank78314 +78315 POINT(39.87281341496541 74.58240546864387) bank78315 +78316 POINT(40.08523076256557 74.28153266858082) bank78316 +78317 POINT(40.08534316261647 74.04291745177036) bank78317 +78318 POINT(40.80811610653782 73.35298631744529) bank78318 +78319 POINT(41.70113777450838 73.8193229518627) bank78319 +78320 POINT(40.887023751011235 73.63294161959759) bank78320 +78321 POINT(40.50513436542568 74.02293160632598) bank78321 +78322 POINT(41.16457847738434 73.31555893921102) bank78322 +78323 POINT(41.11112490798942 74.41386265370284) bank78323 +78324 POINT(40.46890574579287 73.45896840481791) bank78324 +78325 POINT(39.740893298495536 74.81200846378317) bank78325 +78326 POINT(40.51661085610023 73.49405135451065) bank78326 +78327 POINT(40.18970915282445 74.86367798218387) bank78327 +78328 POINT(40.71554081620199 73.58730945394636) bank78328 +78329 POINT(40.88187090540485 73.25154023457426) bank78329 +78330 POINT(39.91474013351317 73.18869770514453) bank78330 +78331 POINT(39.74722508802135 73.4744654388489) bank78331 +78332 POINT(40.28601484219821 73.9584930267542) bank78332 +78333 POINT(41.415020540903164 73.44475142309321) bank78333 +78334 POINT(40.321740915713 74.67999797937287) bank78334 +78335 POINT(40.734848562143654 74.4630188482479) bank78335 +78336 POINT(40.18447642326647 73.36921180713843) bank78336 +78337 POINT(41.22128071808409 73.47090278004438) bank78337 +78338 POINT(40.64358403679421 74.27479646295795) bank78338 +78339 POINT(41.15745740550074 74.62842811995867) bank78339 +78340 POINT(40.48180315317605 74.24748647200235) bank78340 +78341 POINT(40.23971663311205 74.524056449492) bank78341 +78342 POINT(40.00542528563194 73.00723424720653) bank78342 +78343 POINT(40.210793661120576 73.97301649002107) bank78343 +78344 POINT(41.45007000122511 73.91480648333525) bank78344 +78345 POINT(41.69639925898693 74.48303256624585) bank78345 +78346 POINT(40.998086795126156 73.08598485898662) bank78346 +78347 POINT(40.152074438177884 74.97501820529888) bank78347 +78348 POINT(41.568994092398306 73.1887237288103) bank78348 +78349 POINT(40.967109856779174 73.97426992860488) bank78349 +78350 POINT(40.83449869750199 74.44322773158733) bank78350 +78351 POINT(40.600276445942704 73.34109445543594) bank78351 +78352 POINT(40.003871008112796 73.44690625698307) bank78352 +78353 POINT(40.112535885394 74.94659593907828) bank78353 +78354 POINT(40.91507195397202 73.96283098425369) bank78354 +78355 POINT(41.361608426525684 74.70990933346279) bank78355 +78356 POINT(40.197537401889946 73.81621004161245) bank78356 +78357 POINT(40.09217122193232 73.42465874061335) bank78357 +78358 POINT(40.48542583713296 74.03196673682783) bank78358 +78359 POINT(40.94469843886556 74.15654568168102) bank78359 +78360 POINT(39.77781181499661 74.21912209712906) bank78360 +78361 POINT(40.468558237398796 73.18746427621231) bank78361 +78362 POINT(40.485262712409366 74.66320816799909) bank78362 +78363 POINT(41.60456933085089 74.81845328725538) bank78363 +78364 POINT(40.7856509008776 74.93028116481732) bank78364 +78365 POINT(41.073443782969946 73.01828595579359) bank78365 +78366 POINT(40.53644630227027 73.67580928360485) bank78366 +78367 POINT(40.61813311721069 73.09294566339312) bank78367 +78368 POINT(40.9290646130827 73.9847346321984) bank78368 +78369 POINT(39.89914530223224 74.60552471037334) bank78369 +78370 POINT(40.24658847543519 73.71496632075916) bank78370 +78371 POINT(40.28877795263687 73.789275452986) bank78371 +78372 POINT(40.16754840219878 74.63223292564315) bank78372 +78373 POINT(41.54944997024182 74.54526217796547) bank78373 +78374 POINT(41.37661706562601 74.4644698904659) bank78374 +78375 POINT(40.60552098065269 73.86500603810894) bank78375 +78376 POINT(40.55622081108573 73.93739613161573) bank78376 +78377 POINT(41.6466886372777 73.16731889472914) bank78377 +78378 POINT(40.74243448704575 74.43403770537415) bank78378 +78379 POINT(40.67341796874898 73.17296599797663) bank78379 +78380 POINT(40.36685100320335 74.46932130157647) bank78380 +78381 POINT(40.77578371370214 73.5761312691124) bank78381 +78382 POINT(40.91413821345949 73.59971018891501) bank78382 +78383 POINT(39.942331316779565 74.27622912777633) bank78383 +78384 POINT(40.29620442143076 73.49059250346804) bank78384 +78385 POINT(40.4467425679292 74.92808398396522) bank78385 +78386 POINT(40.8639861710711 74.99382269976682) bank78386 +78387 POINT(40.25122261921251 73.72222737775834) bank78387 +78388 POINT(39.738909897143806 74.285839060552) bank78388 +78389 POINT(41.50120102833809 74.15392850282936) bank78389 +78390 POINT(40.39795194667548 74.14936485153133) bank78390 +78391 POINT(41.40976713618836 74.56436072031855) bank78391 +78392 POINT(41.453765190981976 74.77124578658767) bank78392 +78393 POINT(41.1471397566033 74.5650264568264) bank78393 +78394 POINT(41.554020675239606 74.88842246184703) bank78394 +78395 POINT(40.270976210136965 74.0630835487148) bank78395 +78396 POINT(41.02299053500863 74.26830479017372) bank78396 +78397 POINT(40.42234288825841 73.35491871002368) bank78397 +78398 POINT(40.37437816721413 73.31983150153523) bank78398 +78399 POINT(40.45049645014026 74.73415704797353) bank78399 +78400 POINT(40.36539855036522 73.04448007508462) bank78400 +78401 POINT(40.51076671850981 73.31792332531545) bank78401 +78402 POINT(41.41350317573005 74.10431500677515) bank78402 +78403 POINT(40.99413447820811 74.90475827335595) bank78403 +78404 POINT(40.62696022298162 73.82837031094088) bank78404 +78405 POINT(39.77912272829313 74.39344417974932) bank78405 +78406 POINT(40.69889210029576 73.20179554415253) bank78406 +78407 POINT(40.362173771466615 73.96578911386347) bank78407 +78408 POINT(40.96166953491411 74.82547661841998) bank78408 +78409 POINT(40.647881777052376 74.68248325207172) bank78409 +78410 POINT(40.932757107439016 74.63636020082751) bank78410 +78411 POINT(40.35201176068417 73.12990559504303) bank78411 +78412 POINT(39.936085570171926 74.26737823272994) bank78412 +78413 POINT(40.087995251330106 74.97700271770864) bank78413 +78414 POINT(41.09965270882489 74.77995150249244) bank78414 +78415 POINT(40.30948294699598 73.41333338319075) bank78415 +78416 POINT(41.497321272427314 73.54988795973634) bank78416 +78417 POINT(40.16509104174522 74.82603561726357) bank78417 +78418 POINT(41.18631059642703 74.3169070709161) bank78418 +78419 POINT(41.70713397143204 74.96611188717746) bank78419 +78420 POINT(40.368729604585624 73.57700657296759) bank78420 +78421 POINT(41.27182613442573 73.21780812994713) bank78421 +78422 POINT(40.67576742109646 74.58074915043385) bank78422 +78423 POINT(41.67479000144486 74.02456029524751) bank78423 +78424 POINT(40.11050684474055 73.12050333792811) bank78424 +78425 POINT(39.83864715315648 73.28678789656408) bank78425 +78426 POINT(40.14655179728253 73.82493983854391) bank78426 +78427 POINT(40.876340464221485 73.67604587059287) bank78427 +78428 POINT(41.05340218626432 73.069099225378) bank78428 +78429 POINT(40.82710557077141 73.67233032389308) bank78429 +78430 POINT(40.126495144897966 73.78185527614868) bank78430 +78431 POINT(40.19209568932391 74.85790858187936) bank78431 +78432 POINT(39.859747436607776 73.87707396412951) bank78432 +78433 POINT(40.950509637109995 73.37378745515035) bank78433 +78434 POINT(40.88143826948662 73.67645726109443) bank78434 +78435 POINT(40.79024684782489 74.96549433444778) bank78435 +78436 POINT(41.5551570886638 74.78182781686638) bank78436 +78437 POINT(41.02180620807141 73.55024492888634) bank78437 +78438 POINT(41.11123854305934 73.93689292854694) bank78438 +78439 POINT(41.32057949160775 73.6342218618299) bank78439 +78440 POINT(40.50857790087298 73.98237735470548) bank78440 +78441 POINT(41.01886133142405 74.04505256942147) bank78441 +78442 POINT(40.48536107994459 74.38464111688319) bank78442 +78443 POINT(39.94960248939735 74.41006535587506) bank78443 +78444 POINT(41.67169408229328 73.35722302864492) bank78444 +78445 POINT(41.667489306835385 74.37191765292603) bank78445 +78446 POINT(40.705601055542196 73.33214084626624) bank78446 +78447 POINT(41.38958320296763 73.89171345521831) bank78447 +78448 POINT(40.831678641872685 73.6187456616253) bank78448 +78449 POINT(40.00000865381358 74.35906768033479) bank78449 +78450 POINT(40.529730819757305 74.16092309252628) bank78450 +78451 POINT(40.565076791162824 74.35429559488017) bank78451 +78452 POINT(41.30186010293433 74.92157954718009) bank78452 +78453 POINT(39.96930355938533 74.90157566024037) bank78453 +78454 POINT(41.367777628516286 73.8936513177199) bank78454 +78455 POINT(40.8551723407991 74.49898471190527) bank78455 +78456 POINT(39.82092324952709 74.6179480575317) bank78456 +78457 POINT(40.01392296670539 73.28463741521499) bank78457 +78458 POINT(39.74705602745323 73.17928682492303) bank78458 +78459 POINT(39.81169458751165 73.95060907676401) bank78459 +78460 POINT(40.46921823238258 73.28591050990298) bank78460 +78461 POINT(40.14178036609865 74.56291763190475) bank78461 +78462 POINT(41.27999503317308 74.68373943559409) bank78462 +78463 POINT(40.283437591289214 73.34085456225846) bank78463 +78464 POINT(40.13470322214743 73.28677211343256) bank78464 +78465 POINT(41.448336815103545 74.22982445633829) bank78465 +78466 POINT(40.220547028659716 73.05665102073289) bank78466 +78467 POINT(40.709009359385405 74.97988523440956) bank78467 +78468 POINT(39.73034516983605 74.00707138937217) bank78468 +78469 POINT(40.97657946680683 74.55574580299823) bank78469 +78470 POINT(41.2760184177764 74.12764384294695) bank78470 +78471 POINT(40.47522212681555 74.9697377637732) bank78471 +78472 POINT(41.387449926320265 73.67642690874543) bank78472 +78473 POINT(40.270014690467555 73.25111962280215) bank78473 +78474 POINT(40.12325037573266 73.03192244981643) bank78474 +78475 POINT(40.26907020274686 74.79827499515758) bank78475 +78476 POINT(41.629930939694894 74.99328672993036) bank78476 +78477 POINT(39.94460984430559 73.52118418635398) bank78477 +78478 POINT(41.266565152909315 74.04936058296012) bank78478 +78479 POINT(40.60250811188024 74.83139900333533) bank78479 +78480 POINT(40.18600107919141 74.91673411556545) bank78480 +78481 POINT(40.681235544522266 73.14168402952323) bank78481 +78482 POINT(41.0464120608128 73.21124612691784) bank78482 +78483 POINT(40.5327499729327 74.57169983922775) bank78483 +78484 POINT(41.08745402488565 73.42197961939647) bank78484 +78485 POINT(40.75292318920721 73.26129441250143) bank78485 +78486 POINT(41.616286744757666 73.72939986870959) bank78486 +78487 POINT(41.246360989732324 74.45912975222598) bank78487 +78488 POINT(40.53078377923247 73.26011458545048) bank78488 +78489 POINT(40.66852005225783 74.86022730773894) bank78489 +78490 POINT(39.99093241182708 73.05116926277783) bank78490 +78491 POINT(40.151374790681096 74.94890374316157) bank78491 +78492 POINT(41.601903113248746 73.63956396894251) bank78492 +78493 POINT(41.075217215895464 73.45809363185948) bank78493 +78494 POINT(41.554028331938696 73.745822836326) bank78494 +78495 POINT(41.39533052074837 73.57911190981815) bank78495 +78496 POINT(39.831259476372644 73.01904074270989) bank78496 +78497 POINT(40.869353341824045 74.1578889796009) bank78497 +78498 POINT(40.94694844026256 74.47308153758648) bank78498 +78499 POINT(41.19686782744796 74.02209200854702) bank78499 +78500 POINT(41.29836115280557 73.87157241351696) bank78500 +78501 POINT(41.66243937011075 73.46183807361197) bank78501 +78502 POINT(40.57422297583511 74.57993117708088) bank78502 +78503 POINT(41.23567400386591 74.02348461638266) bank78503 +78504 POINT(40.418968786098326 73.14434369795873) bank78504 +78505 POINT(41.07462331977773 73.87277059394269) bank78505 +78506 POINT(40.01904607832109 74.64425345466998) bank78506 +78507 POINT(41.34016168316444 73.43165100651265) bank78507 +78508 POINT(40.35896224706594 74.26458181718412) bank78508 +78509 POINT(41.25359611441987 74.07115914958044) bank78509 +78510 POINT(40.08054323375139 74.09582626231473) bank78510 +78511 POINT(40.65904554903244 73.87444688105245) bank78511 +78512 POINT(39.79590499595356 74.87231197375087) bank78512 +78513 POINT(40.67055547613136 73.95081486643957) bank78513 +78514 POINT(41.251503484863186 73.14591169118165) bank78514 +78515 POINT(39.938188857732264 74.80095018165935) bank78515 +78516 POINT(40.266890190240844 73.48452353265635) bank78516 +78517 POINT(41.07671720496443 73.51806025330048) bank78517 +78518 POINT(39.91704220841718 73.30388856004284) bank78518 +78519 POINT(39.7509744858079 74.47018886452862) bank78519 +78520 POINT(40.595740100926704 74.26431427870726) bank78520 +78521 POINT(41.528519702461786 73.64681537733111) bank78521 +78522 POINT(40.69954951497049 73.00607594350295) bank78522 +78523 POINT(41.392029714346464 74.76580444987455) bank78523 +78524 POINT(40.460727122785535 73.23480529049529) bank78524 +78525 POINT(39.94807448969609 74.44468162184015) bank78525 +78526 POINT(41.29996527816443 74.50174010408419) bank78526 +78527 POINT(40.57520456910256 73.96638862103644) bank78527 +78528 POINT(40.912387416863304 74.72055122580176) bank78528 +78529 POINT(40.37692498718294 74.09803916676259) bank78529 +78530 POINT(41.2874511864576 74.99160677027788) bank78530 +78531 POINT(41.27759309198875 74.74644334740177) bank78531 +78532 POINT(40.929950030550934 74.02549151236056) bank78532 +78533 POINT(39.917354180759475 73.32128264112367) bank78533 +78534 POINT(40.13043580565274 73.14786148036612) bank78534 +78535 POINT(40.55534546144638 73.16823699658725) bank78535 +78536 POINT(39.82018226143155 74.11951635579709) bank78536 +78537 POINT(40.17433097512718 73.59565843814568) bank78537 +78538 POINT(41.104155908734434 74.94355615206398) bank78538 +78539 POINT(39.8323932556063 73.3705604160444) bank78539 +78540 POINT(40.41233802192962 74.09806241140208) bank78540 +78541 POINT(40.87135387271694 74.17211472918073) bank78541 +78542 POINT(39.768829151782 73.62504180152224) bank78542 +78543 POINT(41.14437140286898 74.2083362186062) bank78543 +78544 POINT(40.45790072164294 74.65026798484618) bank78544 +78545 POINT(40.22000819739797 74.74101412275063) bank78545 +78546 POINT(40.86979573523786 73.03023498062602) bank78546 +78547 POINT(40.77481697078409 73.08081469159497) bank78547 +78548 POINT(40.768531062065875 74.06221356165192) bank78548 +78549 POINT(41.54757978492791 74.8301768254154) bank78549 +78550 POINT(40.300540012589785 73.64400316075356) bank78550 +78551 POINT(40.870158189094774 73.17878792312055) bank78551 +78552 POINT(39.94752546284095 74.00058371213194) bank78552 +78553 POINT(40.460691160464556 74.15998379939717) bank78553 +78554 POINT(41.34331608009058 74.77465386143625) bank78554 +78555 POINT(40.79580853747875 73.65547749977513) bank78555 +78556 POINT(41.68299221799783 74.52607978214867) bank78556 +78557 POINT(41.117113743255665 73.06409626463102) bank78557 +78558 POINT(39.87036886569997 73.8062262329271) bank78558 +78559 POINT(41.65093009976639 73.69110484158881) bank78559 +78560 POINT(41.07375807266799 74.94720356864724) bank78560 +78561 POINT(41.05567362976975 73.6167854313546) bank78561 +78562 POINT(40.37893320595971 73.12805781989307) bank78562 +78563 POINT(40.43367134553761 74.77955587893084) bank78563 +78564 POINT(41.07337184073205 74.81423849563137) bank78564 +78565 POINT(41.1039431966883 74.02440244751497) bank78565 +78566 POINT(40.427213129495165 74.6192114423053) bank78566 +78567 POINT(41.70758320798551 73.49119649265721) bank78567 +78568 POINT(39.79163115168089 73.90267203185752) bank78568 +78569 POINT(41.1096884214968 74.2555420546183) bank78569 +78570 POINT(39.823712390723216 74.1960288834911) bank78570 +78571 POINT(41.544337426494366 74.24345600426224) bank78571 +78572 POINT(39.886287189281504 74.20104548199754) bank78572 +78573 POINT(41.36872490343791 74.87155000958072) bank78573 +78574 POINT(39.99299622957895 73.42108183369825) bank78574 +78575 POINT(41.34086076587012 74.84966752814279) bank78575 +78576 POINT(41.20106275492863 75.00445793182342) bank78576 +78577 POINT(40.739291517179126 73.04134582055065) bank78577 +78578 POINT(40.20120663242001 73.09495106503806) bank78578 +78579 POINT(39.71756484178734 74.86735230446853) bank78579 +78580 POINT(41.29732069603901 73.3136915959107) bank78580 +78581 POINT(39.91381114539096 74.8170509397275) bank78581 +78582 POINT(40.0076204660378 74.01903443270949) bank78582 +78583 POINT(41.00418529434767 73.4373878085354) bank78583 +78584 POINT(40.025203896688254 73.85246169535414) bank78584 +78585 POINT(40.44974497202092 73.65331836223054) bank78585 +78586 POINT(40.414399724808156 74.39678470867999) bank78586 +78587 POINT(41.144545005389496 74.25923742673562) bank78587 +78588 POINT(40.77224514548782 73.53946261086489) bank78588 +78589 POINT(40.653423853680536 74.39786603350926) bank78589 +78590 POINT(40.12623319943301 73.05369932203193) bank78590 +78591 POINT(40.74559379478125 73.91978262791005) bank78591 +78592 POINT(39.96737859183946 73.88650825758559) bank78592 +78593 POINT(41.41104279871695 73.4391250503368) bank78593 +78594 POINT(39.73305754279734 74.53154877953165) bank78594 +78595 POINT(41.067137798643415 74.6214170130818) bank78595 +78596 POINT(40.98559615609437 74.31855261105336) bank78596 +78597 POINT(40.2855858831081 74.72856752050265) bank78597 +78598 POINT(40.51370728382445 74.99455376661395) bank78598 +78599 POINT(40.531114658519904 74.85700192979262) bank78599 +78600 POINT(41.7026923385867 74.48771730510856) bank78600 +78601 POINT(41.57584555776874 73.47957209582208) bank78601 +78602 POINT(41.40368539749553 74.6365182977648) bank78602 +78603 POINT(40.994417823944666 73.44026969481833) bank78603 +78604 POINT(40.59055994012475 74.72615586224693) bank78604 +78605 POINT(41.3158250926653 73.71642221639124) bank78605 +78606 POINT(40.910396832877815 74.96146120335794) bank78606 +78607 POINT(40.81032054372837 74.14137705792753) bank78607 +78608 POINT(40.92158223700088 74.15657373606159) bank78608 +78609 POINT(41.298291628005956 73.20887227582661) bank78609 +78610 POINT(40.368313371965755 73.59520838932323) bank78610 +78611 POINT(41.06801039577977 74.7768853505373) bank78611 +78612 POINT(40.690710700947115 73.25583352060025) bank78612 +78613 POINT(40.8564871649585 74.12934774884347) bank78613 +78614 POINT(40.12683964081981 74.62041893419776) bank78614 +78615 POINT(41.45342990786505 74.74803969453276) bank78615 +78616 POINT(40.783759903445606 73.48906007714334) bank78616 +78617 POINT(40.53112534354253 73.11613621005517) bank78617 +78618 POINT(40.27133787881856 74.76881199493228) bank78618 +78619 POINT(40.94974851124823 73.60910874490585) bank78619 +78620 POINT(41.44697525446 73.02579382304704) bank78620 +78621 POINT(40.41609358637412 73.43358500660356) bank78621 +78622 POINT(41.15666640706532 73.90639205085492) bank78622 +78623 POINT(41.580464573635304 74.89956943619352) bank78623 +78624 POINT(41.02948953757981 74.01447502256376) bank78624 +78625 POINT(39.99173647890989 73.69941860032405) bank78625 +78626 POINT(39.967820901822584 73.2720587725388) bank78626 +78627 POINT(41.18094727034533 74.37693636332119) bank78627 +78628 POINT(40.624649981305105 73.1134790174565) bank78628 +78629 POINT(40.56012812942617 74.2389963973913) bank78629 +78630 POINT(41.55124541881188 74.07568368209499) bank78630 +78631 POINT(40.988118888249176 73.68530314096452) bank78631 +78632 POINT(39.849824345847765 73.45931336904013) bank78632 +78633 POINT(39.847131904035535 74.44671059295386) bank78633 +78634 POINT(40.494512354385485 74.13147429916893) bank78634 +78635 POINT(41.678678160362786 74.89681462297362) bank78635 +78636 POINT(40.21640407730948 74.3015353452495) bank78636 +78637 POINT(39.87582455526744 74.0263384065731) bank78637 +78638 POINT(41.369280874757294 73.53094614732557) bank78638 +78639 POINT(41.29630185566392 74.43047857362495) bank78639 +78640 POINT(40.44024014384167 73.0767252490807) bank78640 +78641 POINT(41.04592489848884 74.90892558936434) bank78641 +78642 POINT(39.87050400404256 74.46397652309847) bank78642 +78643 POINT(40.323790013670006 74.45560550011685) bank78643 +78644 POINT(41.13311881723305 74.42549289824053) bank78644 +78645 POINT(41.03015307216662 74.26861592839953) bank78645 +78646 POINT(39.966283298206356 73.01560919457508) bank78646 +78647 POINT(40.64125961885018 73.27553562023822) bank78647 +78648 POINT(40.62181639038522 74.23713433914017) bank78648 +78649 POINT(41.68774892456591 74.3134467588828) bank78649 +78650 POINT(41.11337583623296 73.91682989978764) bank78650 +78651 POINT(41.40472348550373 74.24506933710501) bank78651 +78652 POINT(39.88257426618729 73.60277926909063) bank78652 +78653 POINT(40.85326241682641 74.11236536598479) bank78653 +78654 POINT(39.8167736318217 73.40628589907611) bank78654 +78655 POINT(41.47621687851707 73.45512204413679) bank78655 +78656 POINT(39.81454269889849 74.20706044263451) bank78656 +78657 POINT(39.88587450096965 74.31335155285446) bank78657 +78658 POINT(40.558742676138934 74.23869808758741) bank78658 +78659 POINT(40.66129358301375 73.63526431164495) bank78659 +78660 POINT(39.869552417030725 73.97942691585995) bank78660 +78661 POINT(40.357272264497475 73.32227381696464) bank78661 +78662 POINT(41.28269359673416 74.38659101863017) bank78662 +78663 POINT(41.52935657084197 73.9760469100332) bank78663 +78664 POINT(39.95811938527894 73.23365463077862) bank78664 +78665 POINT(41.52789774063748 73.46304517399565) bank78665 +78666 POINT(39.910616180587176 74.0482699747559) bank78666 +78667 POINT(40.0549757390222 74.35132006571504) bank78667 +78668 POINT(40.967348961621866 74.85044036670529) bank78668 +78669 POINT(41.57133157511447 73.25269495669338) bank78669 +78670 POINT(40.01621780380957 74.09252335669426) bank78670 +78671 POINT(41.05613182137264 74.88887436454465) bank78671 +78672 POINT(40.52777663425661 73.47087374476901) bank78672 +78673 POINT(40.450234801394856 74.92350361679264) bank78673 +78674 POINT(39.91192649552842 73.87509980402581) bank78674 +78675 POINT(40.09006242917697 74.24320516164539) bank78675 +78676 POINT(40.60897729406402 73.37567958373309) bank78676 +78677 POINT(40.00804778479823 73.79799396517971) bank78677 +78678 POINT(39.85698000640277 73.06088380266367) bank78678 +78679 POINT(40.1513822261017 73.54714037885861) bank78679 +78680 POINT(40.154321225671474 74.99612963291486) bank78680 +78681 POINT(41.36550470306158 74.7690937478872) bank78681 +78682 POINT(40.8142340642939 74.10837333759162) bank78682 +78683 POINT(41.39787078900949 73.12051606283804) bank78683 +78684 POINT(40.95762330821042 74.32430074585331) bank78684 +78685 POINT(40.45641940640284 73.57952303649651) bank78685 +78686 POINT(40.327453372545875 73.49712961671352) bank78686 +78687 POINT(39.98147239954101 74.98152726195285) bank78687 +78688 POINT(39.779641837090836 73.35667220785376) bank78688 +78689 POINT(40.65473673922208 74.86586759024938) bank78689 +78690 POINT(41.55252267567846 73.20565438272787) bank78690 +78691 POINT(40.96247208820521 73.34622251251382) bank78691 +78692 POINT(41.28955483868059 73.13869294538297) bank78692 +78693 POINT(41.44954030558294 73.99508699111041) bank78693 +78694 POINT(41.07111248727892 73.53663165098371) bank78694 +78695 POINT(41.24543701431431 73.06990601691264) bank78695 +78696 POINT(40.50768290354006 74.6818953962649) bank78696 +78697 POINT(40.808417973256184 74.4877479796642) bank78697 +78698 POINT(40.4997933904048 73.7716751736498) bank78698 +78699 POINT(41.632338602856315 73.68534326948799) bank78699 +78700 POINT(40.93741831898101 74.90634891491855) bank78700 +78701 POINT(40.12441120076928 73.93044712095285) bank78701 +78702 POINT(41.05608752464084 74.90829558249878) bank78702 +78703 POINT(41.20097132167927 74.22544336278368) bank78703 +78704 POINT(41.24061085115931 73.51912226062004) bank78704 +78705 POINT(40.79190576797637 73.80415022767333) bank78705 +78706 POINT(41.03105613915966 73.6403880110229) bank78706 +78707 POINT(39.82554000892203 73.89904637922652) bank78707 +78708 POINT(40.274911401090264 73.77558507886553) bank78708 +78709 POINT(40.96040930128464 74.29055587280634) bank78709 +78710 POINT(41.65220093766686 74.00103839123625) bank78710 +78711 POINT(40.01969632512858 73.75297865028574) bank78711 +78712 POINT(41.663303411715376 73.26345285473275) bank78712 +78713 POINT(41.148763501716324 74.67423420450962) bank78713 +78714 POINT(40.025563275386084 74.67193919894238) bank78714 +78715 POINT(40.58119180502084 73.60222984752309) bank78715 +78716 POINT(41.44607957315373 74.47630665210676) bank78716 +78717 POINT(41.11146471046355 73.61912844412934) bank78717 +78718 POINT(40.33015576305188 74.56357450012025) bank78718 +78719 POINT(40.454867887197835 73.51144751114093) bank78719 +78720 POINT(41.3179610853953 73.66374860424087) bank78720 +78721 POINT(40.906511727121284 74.29746624179083) bank78721 +78722 POINT(41.07606415380971 74.81755681175144) bank78722 +78723 POINT(40.139050607936774 73.86425006637616) bank78723 +78724 POINT(40.472980877271354 73.06746780911125) bank78724 +78725 POINT(40.955435564409406 73.31800379398864) bank78725 +78726 POINT(40.9543349833708 74.19100535547078) bank78726 +78727 POINT(40.76893332789665 73.26416420137934) bank78727 +78728 POINT(39.80562320276305 74.05903273803855) bank78728 +78729 POINT(41.20614032132171 74.31152752194764) bank78729 +78730 POINT(39.93879030275927 73.47113224448077) bank78730 +78731 POINT(40.59531088100694 74.8944091120935) bank78731 +78732 POINT(41.667470948550864 74.1016880287458) bank78732 +78733 POINT(41.45555914588904 73.41205646306666) bank78733 +78734 POINT(39.790447539770824 73.20874401350993) bank78734 +78735 POINT(40.77289462966255 73.94025377404411) bank78735 +78736 POINT(40.17532590974052 73.81730204765982) bank78736 +78737 POINT(40.221450937921716 74.17505497807343) bank78737 +78738 POINT(41.29363143385163 74.35080929662897) bank78738 +78739 POINT(41.69843653244363 74.71605349954926) bank78739 +78740 POINT(39.96912698745929 74.6085568608627) bank78740 +78741 POINT(40.71387177840714 74.52379370677068) bank78741 +78742 POINT(40.27786399517096 74.59241459647393) bank78742 +78743 POINT(39.767462529440536 74.97328942844327) bank78743 +78744 POINT(41.709500790094296 73.69560866802779) bank78744 +78745 POINT(40.751914764999455 73.44829711098704) bank78745 +78746 POINT(40.71752713294355 73.088167294677) bank78746 +78747 POINT(41.51081422274743 73.2770307608196) bank78747 +78748 POINT(41.68296268045552 74.40292752305943) bank78748 +78749 POINT(41.27248702007021 74.57302945455194) bank78749 +78750 POINT(40.02239193779383 73.56737190838538) bank78750 +78751 POINT(41.24100120122355 74.09907419491937) bank78751 +78752 POINT(40.303439378076874 73.35079771371639) bank78752 +78753 POINT(40.72618957682375 74.66298139225425) bank78753 +78754 POINT(41.311042840204415 73.06151267115024) bank78754 +78755 POINT(40.9925471051871 73.94865595036934) bank78755 +78756 POINT(40.79431503281028 73.21030675801843) bank78756 +78757 POINT(40.52817866380218 74.98747792369377) bank78757 +78758 POINT(41.22199545617083 74.10733899588706) bank78758 +78759 POINT(40.750977220107856 74.2278583197426) bank78759 +78760 POINT(41.63816868929331 73.99692200802653) bank78760 +78761 POINT(40.429822486466044 74.2322819339638) bank78761 +78762 POINT(41.04276952944634 73.57808479395361) bank78762 +78763 POINT(40.323060008402244 74.75160905920252) bank78763 +78764 POINT(41.03477034355257 73.8315934720961) bank78764 +78765 POINT(41.33132598236315 74.29768118866743) bank78765 +78766 POINT(40.467003043702796 73.73567744577072) bank78766 +78767 POINT(39.820589797300215 73.43859395597617) bank78767 +78768 POINT(40.571246323759276 74.8986690442728) bank78768 +78769 POINT(40.85687106587443 73.2672703959411) bank78769 +78770 POINT(40.848048346810394 74.97846688159251) bank78770 +78771 POINT(39.96236731311426 74.39077174894385) bank78771 +78772 POINT(40.42532840780336 74.74818644344718) bank78772 +78773 POINT(40.640723679754196 74.94121050565246) bank78773 +78774 POINT(41.64099819543642 74.49736492978205) bank78774 +78775 POINT(41.237722223722926 73.25415715949916) bank78775 +78776 POINT(40.749283201957496 73.69851619560215) bank78776 +78777 POINT(41.51419194987883 74.60897848732999) bank78777 +78778 POINT(41.51414881760484 74.57846302226935) bank78778 +78779 POINT(40.0414324515276 73.4723381867483) bank78779 +78780 POINT(40.58754766100459 73.44723217809846) bank78780 +78781 POINT(40.879040852410775 74.4415452994448) bank78781 +78782 POINT(41.389575850701306 74.73434786342128) bank78782 +78783 POINT(40.30797084480673 74.82740269315522) bank78783 +78784 POINT(39.784964489006654 73.71577917752428) bank78784 +78785 POINT(41.68501613707475 73.01063354937358) bank78785 +78786 POINT(40.812487348898614 73.44305728521546) bank78786 +78787 POINT(40.08087685612874 73.66316138453458) bank78787 +78788 POINT(41.30986407493525 74.00632489909181) bank78788 +78789 POINT(39.95691645585592 73.49752424072341) bank78789 +78790 POINT(41.141920352921986 73.2044939675267) bank78790 +78791 POINT(40.05970746481273 73.62000949098122) bank78791 +78792 POINT(40.69077474011234 74.04126826558777) bank78792 +78793 POINT(40.54929755614458 74.63891997697232) bank78793 +78794 POINT(41.23239838176085 74.71027937873647) bank78794 +78795 POINT(41.05435735722505 73.3218598851491) bank78795 +78796 POINT(39.71554871826921 74.16897948398993) bank78796 +78797 POINT(40.255882002212736 73.4642670143207) bank78797 +78798 POINT(39.72369336107519 74.21699790490744) bank78798 +78799 POINT(40.14763638281374 74.98213498805556) bank78799 +78800 POINT(39.84260615771928 73.9003960749704) bank78800 +78801 POINT(40.18256893526464 73.90189471433979) bank78801 +78802 POINT(39.79295582138847 74.1163831596145) bank78802 +78803 POINT(40.42278429976213 73.46792873470939) bank78803 +78804 POINT(41.484872035023024 73.33035785562863) bank78804 +78805 POINT(41.462894848294816 74.20175538758444) bank78805 +78806 POINT(41.223306198357726 74.34391528291317) bank78806 +78807 POINT(39.73268438514756 73.27672889088288) bank78807 +78808 POINT(41.05137197465106 74.40358452598736) bank78808 +78809 POINT(40.19744304594338 73.32966535437299) bank78809 +78810 POINT(39.82649613178679 74.6483421215925) bank78810 +78811 POINT(41.36372941289039 73.31734930847112) bank78811 +78812 POINT(39.97812551653273 73.37779209472858) bank78812 +78813 POINT(41.68931644487932 74.34384566384277) bank78813 +78814 POINT(41.153392671048664 74.39280661557999) bank78814 +78815 POINT(40.37571546963708 73.02367245094136) bank78815 +78816 POINT(40.80585473712742 73.97280568127978) bank78816 +78817 POINT(40.279385092781204 73.13642124821074) bank78817 +78818 POINT(40.18790280668587 74.73486430376839) bank78818 +78819 POINT(40.730075245961665 73.25589171842392) bank78819 +78820 POINT(41.40711446121958 73.16995952426059) bank78820 +78821 POINT(41.56516172531788 74.88002288378617) bank78821 +78822 POINT(40.821152286038796 74.99375684999607) bank78822 +78823 POINT(41.022099157197125 74.66058795396329) bank78823 +78824 POINT(40.76711434942523 73.45272721029042) bank78824 +78825 POINT(40.580447694179576 73.06636833145096) bank78825 +78826 POINT(40.39824155124689 73.95585123468062) bank78826 +78827 POINT(40.205230182118996 73.44149872968153) bank78827 +78828 POINT(41.40774274421533 74.82035365480935) bank78828 +78829 POINT(40.45365563424112 74.2431967691157) bank78829 +78830 POINT(41.282466255485254 73.5345090188133) bank78830 +78831 POINT(41.29306058889125 73.69160133811125) bank78831 +78832 POINT(40.73761971250055 73.88738408393253) bank78832 +78833 POINT(40.03235834922704 73.47195392338138) bank78833 +78834 POINT(41.49792400417228 74.44352271578823) bank78834 +78835 POINT(40.39713984378159 73.23897992219142) bank78835 +78836 POINT(39.71314159685092 73.97232154683219) bank78836 +78837 POINT(40.60700651245181 74.6352184491281) bank78837 +78838 POINT(41.59785066280443 73.18594185010554) bank78838 +78839 POINT(40.79431339412716 74.97124402975524) bank78839 +78840 POINT(41.65613207187506 74.27794767020703) bank78840 +78841 POINT(41.511864664901644 74.91410727643559) bank78841 +78842 POINT(41.11287104591508 73.53487446254002) bank78842 +78843 POINT(40.362041332559635 74.55153484515907) bank78843 +78844 POINT(40.45512838928229 74.27513808928887) bank78844 +78845 POINT(40.050349306652535 73.7358849542849) bank78845 +78846 POINT(41.543700700717075 74.50761234111573) bank78846 +78847 POINT(40.2843002173724 73.47467249655789) bank78847 +78848 POINT(41.65124008397459 73.02601418871494) bank78848 +78849 POINT(40.933822109569434 74.77997942358458) bank78849 +78850 POINT(41.52049062659886 74.66707610930625) bank78850 +78851 POINT(41.268219101514674 73.86098392387181) bank78851 +78852 POINT(41.49871320658304 73.47599819103128) bank78852 +78853 POINT(39.81678654002889 74.75725940882197) bank78853 +78854 POINT(41.71100357485836 73.10194147114463) bank78854 +78855 POINT(40.13676465079147 73.13310000621642) bank78855 +78856 POINT(39.75673903689687 73.45079038168387) bank78856 +78857 POINT(40.27989048655901 73.63059815948866) bank78857 +78858 POINT(40.551930320942134 73.72528670454699) bank78858 +78859 POINT(40.56693741048796 74.7281364908627) bank78859 +78860 POINT(40.895014502621514 73.87943580854233) bank78860 +78861 POINT(40.18332271767676 73.41081368409195) bank78861 +78862 POINT(40.529885135579526 74.35612063728311) bank78862 +78863 POINT(41.68509224819045 75.0049789855382) bank78863 +78864 POINT(39.80618884965368 73.85478859727684) bank78864 +78865 POINT(40.404589064755534 74.48330377510312) bank78865 +78866 POINT(40.61263625732643 74.2996833657259) bank78866 +78867 POINT(39.900087357010065 74.69570367284342) bank78867 +78868 POINT(41.56776568363627 73.92953892381371) bank78868 +78869 POINT(40.37160765815932 73.34018108209547) bank78869 +78870 POINT(40.73659483143285 73.5682532051124) bank78870 +78871 POINT(41.129659824189275 74.77898467357997) bank78871 +78872 POINT(41.24476698853891 74.51634497629257) bank78872 +78873 POINT(40.29618603954382 73.08723659668561) bank78873 +78874 POINT(40.890666491255594 74.87561770638102) bank78874 +78875 POINT(41.563512420125626 74.72014648841713) bank78875 +78876 POINT(41.258634434114136 73.88041415667053) bank78876 +78877 POINT(39.82577374065216 74.1750331268471) bank78877 +78878 POINT(41.43879806250552 73.56306509107796) bank78878 +78879 POINT(40.32692341796738 74.39375246404214) bank78879 +78880 POINT(41.58279193763984 73.042210475753) bank78880 +78881 POINT(40.69329351534645 73.78688431761509) bank78881 +78882 POINT(41.062970049865484 73.74201427391183) bank78882 +78883 POINT(41.56155838712544 74.04052171140168) bank78883 +78884 POINT(40.42922070739196 74.39190656512028) bank78884 +78885 POINT(40.30777860659253 73.01275016938102) bank78885 +78886 POINT(40.832165867501054 73.16134788975025) bank78886 +78887 POINT(40.92063692382038 73.8037149663258) bank78887 +78888 POINT(40.447753445745796 74.6656747560666) bank78888 +78889 POINT(40.97733891196846 73.71539765452484) bank78889 +78890 POINT(40.392317147374854 74.11933823072322) bank78890 +78891 POINT(39.97261021513716 73.5524963244918) bank78891 +78892 POINT(40.07856435833089 73.93775827552001) bank78892 +78893 POINT(39.87947259993379 74.74262317864874) bank78893 +78894 POINT(40.177256846944864 74.87437737916349) bank78894 +78895 POINT(39.95390072815253 74.17586601738255) bank78895 +78896 POINT(40.02486053029625 73.8511791238687) bank78896 +78897 POINT(41.55778299254175 74.184004058427) bank78897 +78898 POINT(41.65168612417158 73.52728286579858) bank78898 +78899 POINT(40.73952361276028 74.93426789981953) bank78899 +78900 POINT(40.48620970030415 74.6719035087323) bank78900 +78901 POINT(40.81904799604735 74.81796174789747) bank78901 +78902 POINT(39.89853001720476 73.48536463700422) bank78902 +78903 POINT(40.408653148600315 73.4062463259916) bank78903 +78904 POINT(41.50871884084733 73.97784417177643) bank78904 +78905 POINT(39.998855438855024 74.99944874063075) bank78905 +78906 POINT(40.01190386675013 74.55850628715714) bank78906 +78907 POINT(41.63798598805037 73.39126027646543) bank78907 +78908 POINT(41.57964978145407 74.33387088810117) bank78908 +78909 POINT(41.48422868241191 74.54258830186706) bank78909 +78910 POINT(41.63664885460193 73.67188112169673) bank78910 +78911 POINT(40.70552980590422 74.07969714299594) bank78911 +78912 POINT(39.99156880882878 73.44344498406898) bank78912 +78913 POINT(41.26909408465786 73.8520083620786) bank78913 +78914 POINT(41.5390880355695 73.52893045404412) bank78914 +78915 POINT(40.04204354752237 74.0608674837804) bank78915 +78916 POINT(40.156394929345815 73.31636836266784) bank78916 +78917 POINT(40.81470073729777 74.56722228065318) bank78917 +78918 POINT(41.609293680966935 74.83779913647346) bank78918 +78919 POINT(40.16105603590569 74.06648701559484) bank78919 +78920 POINT(41.550700989109814 73.76710178678545) bank78920 +78921 POINT(41.317108881135944 73.65054245085831) bank78921 +78922 POINT(40.70863427001056 74.35526728576959) bank78922 +78923 POINT(39.72860051728847 73.45693545959566) bank78923 +78924 POINT(41.14700389067945 74.69943449951923) bank78924 +78925 POINT(40.56938229604652 73.77472119004008) bank78925 +78926 POINT(40.56490198669567 74.48233594654165) bank78926 +78927 POINT(41.20961928614827 73.08845608608154) bank78927 +78928 POINT(40.9367785062438 73.26791933699418) bank78928 +78929 POINT(40.204598309860806 73.98862387057444) bank78929 +78930 POINT(41.37307246762079 73.97799439543785) bank78930 +78931 POINT(41.0333306144176 73.74608047880295) bank78931 +78932 POINT(40.7603256591074 73.65106806926822) bank78932 +78933 POINT(41.1101867810509 73.14822253668218) bank78933 +78934 POINT(41.04639608816174 73.2789612803135) bank78934 +78935 POINT(40.573618008269754 73.57253634423684) bank78935 +78936 POINT(41.093374102496774 73.0541926701604) bank78936 +78937 POINT(40.75488343593151 74.5944006172051) bank78937 +78938 POINT(41.5825259275314 73.93584047600855) bank78938 +78939 POINT(40.81199534581581 73.20773579667767) bank78939 +78940 POINT(40.06692751123193 74.70777624959135) bank78940 +78941 POINT(41.173857808634644 73.56724390958195) bank78941 +78942 POINT(39.74329616258141 73.35591412452351) bank78942 +78943 POINT(39.88383018137521 74.5498398910645) bank78943 +78944 POINT(40.44917181613379 74.5042762084786) bank78944 +78945 POINT(40.04213884116886 74.05377118168029) bank78945 +78946 POINT(40.38633138537111 74.12895678279531) bank78946 +78947 POINT(41.65577861953593 74.58547177158799) bank78947 +78948 POINT(40.71691737231107 74.20534811580634) bank78948 +78949 POINT(39.88017506646601 73.20696664903164) bank78949 +78950 POINT(41.29568783876333 74.05286463513313) bank78950 +78951 POINT(41.04812356400289 73.99100968440321) bank78951 +78952 POINT(41.200769480840854 73.65256108062563) bank78952 +78953 POINT(40.04935435299739 74.1093162200879) bank78953 +78954 POINT(41.253336619311256 73.05497368564143) bank78954 +78955 POINT(40.20852210923509 74.41247893841451) bank78955 +78956 POINT(41.5336904086577 74.78245610918549) bank78956 +78957 POINT(39.92075292766836 74.2776368393791) bank78957 +78958 POINT(40.60992661149931 73.0291544277399) bank78958 +78959 POINT(39.71704201441604 74.31444469029022) bank78959 +78960 POINT(39.75560449895099 73.53760042056781) bank78960 +78961 POINT(40.97635283978175 73.82566097571231) bank78961 +78962 POINT(41.58831682596221 74.79086214236524) bank78962 +78963 POINT(40.2882776427307 73.27301429150515) bank78963 +78964 POINT(40.57239674137048 73.92863708674199) bank78964 +78965 POINT(40.682078573088276 73.04188532476813) bank78965 +78966 POINT(41.666632798928745 73.28852562673148) bank78966 +78967 POINT(40.27034332865681 73.96833142898232) bank78967 +78968 POINT(40.031230986714945 73.34582707997909) bank78968 +78969 POINT(41.571492611426564 74.09440972275893) bank78969 +78970 POINT(39.90134353647569 74.3569092268501) bank78970 +78971 POINT(41.15681043401656 74.097055539758) bank78971 +78972 POINT(40.308750855360415 73.42332379875032) bank78972 +78973 POINT(40.740442056836166 74.44951379260347) bank78973 +78974 POINT(40.48175777303545 74.84700485185462) bank78974 +78975 POINT(40.0319739969356 74.16449777847743) bank78975 +78976 POINT(40.465930794051445 73.77933557620703) bank78976 +78977 POINT(41.21332661118662 73.55594881167735) bank78977 +78978 POINT(40.297846265968886 73.95216181713737) bank78978 +78979 POINT(40.52655701799371 74.44358925096067) bank78979 +78980 POINT(40.148352761990765 74.0729612655552) bank78980 +78981 POINT(39.79789488244867 73.08173488636382) bank78981 +78982 POINT(39.778323276624675 74.08397669888265) bank78982 +78983 POINT(41.22300253787852 73.14158317501294) bank78983 +78984 POINT(39.76296550880623 74.31546280358292) bank78984 +78985 POINT(40.30321317459997 74.56711409052518) bank78985 +78986 POINT(40.78756131106009 74.81285030349423) bank78986 +78987 POINT(41.25377627915034 74.96147104187962) bank78987 +78988 POINT(41.52578922066281 73.74304431357632) bank78988 +78989 POINT(40.142956002623855 73.75424608585685) bank78989 +78990 POINT(40.70423349214568 73.54985801117427) bank78990 +78991 POINT(40.626873286607704 74.69063746585046) bank78991 +78992 POINT(40.64107877178268 73.62198891587376) bank78992 +78993 POINT(40.741015878199214 74.75286550205475) bank78993 +78994 POINT(40.91466918373147 73.31025518185027) bank78994 +78995 POINT(41.61610358373652 74.39508749963099) bank78995 +78996 POINT(41.27015358959787 74.62249820378335) bank78996 +78997 POINT(40.492123382455155 74.38987078652195) bank78997 +78998 POINT(40.70147917281176 73.26125446208984) bank78998 +78999 POINT(41.52720766875134 74.04073397370973) bank78999 +79000 POINT(40.02314136371357 73.68629894090346) bank79000 +79001 POINT(40.268823662556166 74.03411593583206) bank79001 +79002 POINT(40.4017579806006 73.37524045230407) bank79002 +79003 POINT(40.65778471548152 74.47424952431273) bank79003 +79004 POINT(40.42581218733414 73.3871896268189) bank79004 +79005 POINT(41.21698580050941 74.22898427496554) bank79005 +79006 POINT(40.58258804841014 73.04492743037929) bank79006 +79007 POINT(41.32948106789516 74.25220867335608) bank79007 +79008 POINT(40.504325281487375 74.43388973213467) bank79008 +79009 POINT(41.413850604518096 74.60221908804408) bank79009 +79010 POINT(40.919367108292434 74.55013444235783) bank79010 +79011 POINT(40.79144147129537 73.02937131918881) bank79011 +79012 POINT(40.704944638280715 73.34010540869217) bank79012 +79013 POINT(41.60169216034284 74.13885837246735) bank79013 +79014 POINT(39.85442986471905 73.15625168190863) bank79014 +79015 POINT(40.243501863887076 73.71262967103732) bank79015 +79016 POINT(41.129275610423484 74.28163683593002) bank79016 +79017 POINT(40.532111516471886 74.84408252245176) bank79017 +79018 POINT(41.45527408721515 73.17185244809008) bank79018 +79019 POINT(41.16458995427162 73.87218826202091) bank79019 +79020 POINT(41.37799871120467 73.76420784884728) bank79020 +79021 POINT(40.514716421480564 73.89717598175177) bank79021 +79022 POINT(40.32623741043953 73.3590665505534) bank79022 +79023 POINT(40.66358675191199 74.16592699952851) bank79023 +79024 POINT(40.88607626114603 74.66779959469162) bank79024 +79025 POINT(41.4201080295451 74.64338222797203) bank79025 +79026 POINT(41.660234649954454 73.49795340720232) bank79026 +79027 POINT(40.20893443707548 73.80934313607986) bank79027 +79028 POINT(40.24844929006476 74.18375804153692) bank79028 +79029 POINT(40.78473878915649 73.1481159877072) bank79029 +79030 POINT(40.05760458346036 74.59023806726111) bank79030 +79031 POINT(40.9511506232881 73.96389359166983) bank79031 +79032 POINT(41.27084081006979 73.58804657340188) bank79032 +79033 POINT(41.00972792174788 73.28968839540569) bank79033 +79034 POINT(40.215587057031115 73.5235335676941) bank79034 +79035 POINT(41.11496676331928 74.58731992738801) bank79035 +79036 POINT(41.24613440350992 73.22254320868211) bank79036 +79037 POINT(40.51530352288039 73.22419948213157) bank79037 +79038 POINT(40.198571283430795 74.67671837953603) bank79038 +79039 POINT(40.17202091407343 74.49387531181134) bank79039 +79040 POINT(40.7558113285587 73.16904297207545) bank79040 +79041 POINT(40.97571491699794 73.8991191841175) bank79041 +79042 POINT(40.068348180009764 73.24459343244912) bank79042 +79043 POINT(40.550562502551074 73.6727186104986) bank79043 +79044 POINT(40.11594124519313 73.01072032748675) bank79044 +79045 POINT(40.13135195025586 74.06979173664845) bank79045 +79046 POINT(40.95240462636939 73.81523983718752) bank79046 +79047 POINT(40.40871971781438 73.23570556948964) bank79047 +79048 POINT(40.966063280429466 74.82869772714199) bank79048 +79049 POINT(39.92875652212128 74.77023156193434) bank79049 +79050 POINT(41.06843301016304 74.88879511039445) bank79050 +79051 POINT(39.78796588760748 73.11287440034063) bank79051 +79052 POINT(39.85095041642976 74.29335106995887) bank79052 +79053 POINT(40.63507635963686 74.42696610353795) bank79053 +79054 POINT(41.36125880022936 73.6394489278416) bank79054 +79055 POINT(40.361476557336694 74.4504380926875) bank79055 +79056 POINT(39.91924990081719 73.38337718443647) bank79056 +79057 POINT(39.78469064792638 73.80580046914638) bank79057 +79058 POINT(40.58594597977276 73.14218546929723) bank79058 +79059 POINT(40.73658857566886 73.70780815364868) bank79059 +79060 POINT(40.39129094913303 73.26304293210178) bank79060 +79061 POINT(40.10723325492816 74.00217665517835) bank79061 +79062 POINT(40.019435717423065 74.68186363033716) bank79062 +79063 POINT(40.39263079887156 73.70481095836712) bank79063 +79064 POINT(41.03764861984273 73.837386977807) bank79064 +79065 POINT(40.90781569132694 74.53571126511245) bank79065 +79066 POINT(41.35868727028155 73.39038025749613) bank79066 +79067 POINT(40.651357527515856 74.01341660980945) bank79067 +79068 POINT(41.144540782551424 74.85896550380987) bank79068 +79069 POINT(41.30656568703843 73.40286086468438) bank79069 +79070 POINT(40.581610981276974 74.2572432891176) bank79070 +79071 POINT(40.13497109313139 73.65471298933788) bank79071 +79072 POINT(41.12632537755718 74.09463792540882) bank79072 +79073 POINT(40.49869197412637 74.42710409276806) bank79073 +79074 POINT(39.755973002831226 74.03908620703484) bank79074 +79075 POINT(40.807947871949565 74.67640194963539) bank79075 +79076 POINT(40.60282744893776 74.94316706473599) bank79076 +79077 POINT(41.13060895671574 73.14505300239156) bank79077 +79078 POINT(40.15814918406522 74.07499176154482) bank79078 +79079 POINT(40.04167264785076 73.28969599140777) bank79079 +79080 POINT(40.92774643414091 74.10685574104686) bank79080 +79081 POINT(40.07956493766361 75.00318851574606) bank79081 +79082 POINT(40.072595610909715 73.44659686771769) bank79082 +79083 POINT(40.75752585686545 74.53102492506075) bank79083 +79084 POINT(41.157338338797125 73.2594602632135) bank79084 +79085 POINT(40.890757439873376 73.28340617492121) bank79085 +79086 POINT(39.933810472097505 73.2658737852737) bank79086 +79087 POINT(41.19730975257104 73.9291153695408) bank79087 +79088 POINT(40.971316022968466 74.18839790670822) bank79088 +79089 POINT(40.02892536080936 73.61484021771858) bank79089 +79090 POINT(40.91790328615017 73.97637850629255) bank79090 +79091 POINT(41.485946206115095 73.8301576286226) bank79091 +79092 POINT(40.47720752136427 74.0013205363143) bank79092 +79093 POINT(40.27164521168017 74.8951571682348) bank79093 +79094 POINT(41.04839761134195 74.38858191703599) bank79094 +79095 POINT(40.14316960350832 74.76925544773515) bank79095 +79096 POINT(40.40292582411321 73.33549978426106) bank79096 +79097 POINT(41.480164683543926 73.52262430994779) bank79097 +79098 POINT(40.89489522558663 73.49057314578978) bank79098 +79099 POINT(40.224084665644924 73.39956260997215) bank79099 +79100 POINT(41.19109406234479 74.19056978465954) bank79100 +79101 POINT(40.59690291237647 74.46732068185337) bank79101 +79102 POINT(40.33756879646682 74.44307799279535) bank79102 +79103 POINT(40.727183068588 74.52668281659402) bank79103 +79104 POINT(41.0380603817936 74.46913246060626) bank79104 +79105 POINT(40.54576722262668 73.33637582900626) bank79105 +79106 POINT(39.73939180038618 74.009334819151) bank79106 +79107 POINT(41.1293524122012 74.36730760958113) bank79107 +79108 POINT(41.3934514159612 74.08344821904386) bank79108 +79109 POINT(41.23282007463386 73.96737416189548) bank79109 +79110 POINT(41.52116047067905 74.12439411795629) bank79110 +79111 POINT(41.64507423850649 73.54586666370925) bank79111 +79112 POINT(41.346211909088495 74.453749328286) bank79112 +79113 POINT(41.11554900845717 73.14931786920584) bank79113 +79114 POINT(40.44151730465707 74.18279211865098) bank79114 +79115 POINT(41.07773836070056 74.34642058185841) bank79115 +79116 POINT(40.563078657670424 73.69382358008421) bank79116 +79117 POINT(40.68645127693579 74.78882387100941) bank79117 +79118 POINT(40.76540064051933 73.64765934680166) bank79118 +79119 POINT(41.17844253688955 74.99356598341618) bank79119 +79120 POINT(40.60287155360279 73.30092731908931) bank79120 +79121 POINT(41.62818853471621 73.21919019586433) bank79121 +79122 POINT(40.56218495919235 74.52183092008421) bank79122 +79123 POINT(41.14509657454553 73.36929295449438) bank79123 +79124 POINT(41.0883736683662 74.31639317246047) bank79124 +79125 POINT(39.965901710542866 73.8188465832695) bank79125 +79126 POINT(40.81913799210719 74.1362030062818) bank79126 +79127 POINT(40.47755153333486 73.23703069694052) bank79127 +79128 POINT(39.758072348783 73.67128307303028) bank79128 +79129 POINT(39.878845105342094 74.2615808978809) bank79129 +79130 POINT(40.67845711247164 74.85657287258148) bank79130 +79131 POINT(40.39507782154572 73.9167321768499) bank79131 +79132 POINT(39.8339798348026 74.42283528986307) bank79132 +79133 POINT(41.289377832909935 74.09836786907402) bank79133 +79134 POINT(40.54201099504337 73.87035446995407) bank79134 +79135 POINT(39.9675983629352 73.01661906316785) bank79135 +79136 POINT(39.80685562094853 73.46809527464637) bank79136 +79137 POINT(40.499440084961826 73.77287319056262) bank79137 +79138 POINT(40.056194408199374 74.2146841751231) bank79138 +79139 POINT(40.034656055303685 74.96741529330023) bank79139 +79140 POINT(40.79912993305295 73.19339501537351) bank79140 +79141 POINT(41.70536436755451 73.89745025107405) bank79141 +79142 POINT(41.53563321747946 73.45715970179079) bank79142 +79143 POINT(41.45990162055088 73.61349992384669) bank79143 +79144 POINT(40.05420650778431 73.63513504720946) bank79144 +79145 POINT(40.69718535544204 73.61054405063047) bank79145 +79146 POINT(41.694758701916 74.14404507724775) bank79146 +79147 POINT(41.518710311990475 73.06127785756132) bank79147 +79148 POINT(40.083881258110594 73.66343547606344) bank79148 +79149 POINT(40.26345268340063 73.75990328626051) bank79149 +79150 POINT(39.92567229615816 74.5235661319502) bank79150 +79151 POINT(40.68117350609769 73.71768652954962) bank79151 +79152 POINT(40.786563702564244 74.73044264621132) bank79152 +79153 POINT(41.15918192588056 74.48554707866843) bank79153 +79154 POINT(40.14630449318609 73.1190164600217) bank79154 +79155 POINT(41.602236204225285 74.09882060337478) bank79155 +79156 POINT(40.6144842060849 74.1482813929419) bank79156 +79157 POINT(39.74291181727011 73.71580203540528) bank79157 +79158 POINT(41.279363870298106 73.03704970578575) bank79158 +79159 POINT(41.358615628085836 73.17023275240372) bank79159 +79160 POINT(40.47016349940144 74.50799920212387) bank79160 +79161 POINT(40.90983576922564 73.90833640173248) bank79161 +79162 POINT(41.419850453720215 73.88150715926612) bank79162 +79163 POINT(40.789783314996605 73.74892690490707) bank79163 +79164 POINT(41.29088966163429 74.07323480950348) bank79164 +79165 POINT(40.69022621887214 73.87744307960884) bank79165 +79166 POINT(40.45167600148623 74.29366168396146) bank79166 +79167 POINT(39.78266044726919 74.62370073522322) bank79167 +79168 POINT(40.35381352402325 73.12185608822242) bank79168 +79169 POINT(40.88752703996627 73.21874911823929) bank79169 +79170 POINT(40.44875631967821 73.91889912626776) bank79170 +79171 POINT(41.67988281330096 73.36457653739882) bank79171 +79172 POINT(39.781461225048716 73.9431101016713) bank79172 +79173 POINT(40.437644883814976 73.8998154728893) bank79173 +79174 POINT(40.90159312401988 74.55114050880506) bank79174 +79175 POINT(40.903938257584194 74.65117637144915) bank79175 +79176 POINT(41.22318914654188 74.92639451482626) bank79176 +79177 POINT(40.7070824536857 73.4161158847734) bank79177 +79178 POINT(39.78602447470139 73.75493066180319) bank79178 +79179 POINT(39.94780600022381 74.56440408500306) bank79179 +79180 POINT(40.435156605196035 73.58316576038297) bank79180 +79181 POINT(39.72180923522584 74.9417830627307) bank79181 +79182 POINT(40.06257660278739 73.39767395117626) bank79182 +79183 POINT(41.10723661561884 74.46360639137244) bank79183 +79184 POINT(39.81520777607229 74.6248668326431) bank79184 +79185 POINT(41.4369901375505 74.17346357321833) bank79185 +79186 POINT(41.13328921958391 74.97554209561561) bank79186 +79187 POINT(40.082432686420084 73.28951902954593) bank79187 +79188 POINT(40.54049494269259 73.15233227414126) bank79188 +79189 POINT(39.811310289819744 73.6672353117238) bank79189 +79190 POINT(40.37164543915122 74.42288947881399) bank79190 +79191 POINT(40.36493823788016 73.63117627392234) bank79191 +79192 POINT(40.04085468289216 74.19746701727752) bank79192 +79193 POINT(41.11909696294439 73.67736068224859) bank79193 +79194 POINT(39.94959514908261 74.01766128582591) bank79194 +79195 POINT(40.53146206651646 74.68234115066308) bank79195 +79196 POINT(41.57950122953315 73.81822970878599) bank79196 +79197 POINT(40.88039084820943 74.46324330889911) bank79197 +79198 POINT(40.66530538191238 74.67593730453102) bank79198 +79199 POINT(39.800016612451635 74.0193912028219) bank79199 +79200 POINT(41.156151177459776 74.5316255883696) bank79200 +79201 POINT(40.6419028385437 74.96918909846008) bank79201 +79202 POINT(40.80469585278656 73.93467551081477) bank79202 +79203 POINT(40.10534839746312 73.42514758490624) bank79203 +79204 POINT(41.394076623364185 73.47684431658384) bank79204 +79205 POINT(40.83579277989697 73.85033993434469) bank79205 +79206 POINT(41.41455674357057 73.43516786979995) bank79206 +79207 POINT(41.01963638504557 74.71116721663523) bank79207 +79208 POINT(40.83181348447227 74.07991292100581) bank79208 +79209 POINT(40.328133733068974 74.21511978712192) bank79209 +79210 POINT(40.7349385601109 73.13195590960161) bank79210 +79211 POINT(40.0092618840303 74.4227384400303) bank79211 +79212 POINT(40.47807125672081 73.55871543573085) bank79212 +79213 POINT(39.73488100298273 73.48808978373512) bank79213 +79214 POINT(40.81617889009484 74.96570308415562) bank79214 +79215 POINT(41.24417980863547 73.25503964066196) bank79215 +79216 POINT(41.03383542294473 74.19903132318021) bank79216 +79217 POINT(40.95817346324749 73.71690909048547) bank79217 +79218 POINT(39.946624137101104 73.17784280988627) bank79218 +79219 POINT(41.41493045679413 73.99739972236652) bank79219 +79220 POINT(41.200233912418504 74.74514561509646) bank79220 +79221 POINT(40.445964428524526 73.10455119976689) bank79221 +79222 POINT(41.22917671096283 74.99021963021822) bank79222 +79223 POINT(40.71002883687176 74.94821926994081) bank79223 +79224 POINT(39.95072923472916 73.70831131585277) bank79224 +79225 POINT(41.07594842432501 73.52972713030958) bank79225 +79226 POINT(39.79897364030413 74.66977904285038) bank79226 +79227 POINT(40.98709877770656 74.94795012830645) bank79227 +79228 POINT(41.437634016196824 74.93781638679697) bank79228 +79229 POINT(41.47557011349035 73.71808683901496) bank79229 +79230 POINT(40.38780340504321 73.75587204939384) bank79230 +79231 POINT(41.06274850016964 74.37122226350273) bank79231 +79232 POINT(41.46933569634724 74.98932502989749) bank79232 +79233 POINT(40.32328704606784 73.60982179806415) bank79233 +79234 POINT(40.010407462394774 73.4095873098351) bank79234 +79235 POINT(40.595675960087064 74.72425083324607) bank79235 +79236 POINT(39.813119411411506 74.28444113054289) bank79236 +79237 POINT(41.00019442817522 74.16526871403477) bank79237 +79238 POINT(40.04459880035341 74.87326379350068) bank79238 +79239 POINT(41.203386907184544 74.61126180283541) bank79239 +79240 POINT(40.83414680538195 73.97589960409834) bank79240 +79241 POINT(41.700544533089584 73.94057219192672) bank79241 +79242 POINT(41.62669751147317 73.9514493406891) bank79242 +79243 POINT(40.51157773504721 73.51284120584423) bank79243 +79244 POINT(40.943252609120094 73.28746463200667) bank79244 +79245 POINT(39.79313159268644 74.57209242451309) bank79245 +79246 POINT(41.47481652024104 74.03049236395933) bank79246 +79247 POINT(41.478750755405564 74.42999835746919) bank79247 +79248 POINT(39.994039799648505 74.31527957114189) bank79248 +79249 POINT(40.97240240562561 74.78187714959192) bank79249 +79250 POINT(41.35168876110591 73.16020600648818) bank79250 +79251 POINT(40.606778632597994 74.43858628712883) bank79251 +79252 POINT(41.671413239331685 73.09728320264793) bank79252 +79253 POINT(41.360879827041 73.47010324352453) bank79253 +79254 POINT(40.25491387346693 74.28385512457633) bank79254 +79255 POINT(39.77740968135971 74.0590353721016) bank79255 +79256 POINT(41.25470090424419 73.43490182477032) bank79256 +79257 POINT(40.134342261250076 73.58092756933797) bank79257 +79258 POINT(40.40181689288985 73.55015530646655) bank79258 +79259 POINT(41.12347122335203 73.32727541551463) bank79259 +79260 POINT(41.38992059386457 74.90270366495969) bank79260 +79261 POINT(41.06581009702249 73.79839179694281) bank79261 +79262 POINT(40.96912593523964 73.84358528060244) bank79262 +79263 POINT(40.09761219766059 73.54669889305866) bank79263 +79264 POINT(40.51546091153245 73.43828678605986) bank79264 +79265 POINT(41.21616227739994 74.64669449264696) bank79265 +79266 POINT(41.56630658872212 73.17629730878882) bank79266 +79267 POINT(40.57295681600715 73.22922793717257) bank79267 +79268 POINT(39.92202513103511 73.33910683125131) bank79268 +79269 POINT(41.62882204996075 74.15031195837031) bank79269 +79270 POINT(40.72704309304036 73.75412736261738) bank79270 +79271 POINT(40.18841987471747 73.20147903632586) bank79271 +79272 POINT(41.69862683127252 74.44589087366215) bank79272 +79273 POINT(41.29313265646071 73.49495010876939) bank79273 +79274 POINT(41.33953302261532 74.96467069123162) bank79274 +79275 POINT(39.94291671371742 74.69807637720916) bank79275 +79276 POINT(40.09496035549566 73.30649082684093) bank79276 +79277 POINT(40.85192041621317 73.32544837658239) bank79277 +79278 POINT(39.82070138946475 73.37007386956616) bank79278 +79279 POINT(40.186976489024474 75.00336303158528) bank79279 +79280 POINT(41.605564887562274 73.8995486815985) bank79280 +79281 POINT(40.77361611408209 73.15290903378151) bank79281 +79282 POINT(41.179827729782595 73.86312478093409) bank79282 +79283 POINT(40.71595155571036 74.34753410092506) bank79283 +79284 POINT(40.568025853913824 74.35677622728132) bank79284 +79285 POINT(40.05344865089297 74.4635374807812) bank79285 +79286 POINT(39.88086570589963 74.04349649091468) bank79286 +79287 POINT(39.73451948240823 74.1143290799356) bank79287 +79288 POINT(41.54330722523238 74.93193107606166) bank79288 +79289 POINT(39.85788755826921 73.86643276013672) bank79289 +79290 POINT(40.095459571322884 73.85144108112459) bank79290 +79291 POINT(39.906864418460344 74.92064254114163) bank79291 +79292 POINT(41.398822374528116 73.84052560800056) bank79292 +79293 POINT(40.94330616744797 73.78358784876319) bank79293 +79294 POINT(41.14047732620461 73.32706453908344) bank79294 +79295 POINT(40.648430153327624 73.25425569640382) bank79295 +79296 POINT(40.87865959839352 73.98866377633229) bank79296 +79297 POINT(41.242134351371966 73.95693057397678) bank79297 +79298 POINT(41.675910046096426 73.64676558454747) bank79298 +79299 POINT(40.1810367304289 73.11607597381048) bank79299 +79300 POINT(40.24440386763265 74.49259315150572) bank79300 +79301 POINT(41.5366558535843 73.46522575047591) bank79301 +79302 POINT(40.90933074676053 74.46218713529376) bank79302 +79303 POINT(39.97858731979102 73.6574051067903) bank79303 +79304 POINT(40.031455764407426 73.7882112812843) bank79304 +79305 POINT(39.871485103007856 74.88320123031404) bank79305 +79306 POINT(41.03361066750459 74.41767622214189) bank79306 +79307 POINT(40.51507133318343 73.64649873116444) bank79307 +79308 POINT(40.57535474551507 74.59264678823257) bank79308 +79309 POINT(40.642534486881836 73.7822950243739) bank79309 +79310 POINT(41.444010586647956 73.1673179021208) bank79310 +79311 POINT(41.68667631473534 73.7603089642869) bank79311 +79312 POINT(40.79349820871239 73.71013336560347) bank79312 +79313 POINT(40.93481895976295 73.28772269716362) bank79313 +79314 POINT(41.61698182848753 73.85095775644808) bank79314 +79315 POINT(40.117427511512744 73.42386806946247) bank79315 +79316 POINT(41.53284005810737 74.52433304808049) bank79316 +79317 POINT(40.97498291572176 73.6121459771671) bank79317 +79318 POINT(40.63148906561569 73.2165749857869) bank79318 +79319 POINT(40.81516652771246 73.54203216794573) bank79319 +79320 POINT(41.55181226234984 74.59738441536373) bank79320 +79321 POINT(41.68554335980028 73.22450720602525) bank79321 +79322 POINT(40.33091117270204 74.07938102582011) bank79322 +79323 POINT(41.33876994604156 74.63733401016368) bank79323 +79324 POINT(40.554244190475465 74.30990489762891) bank79324 +79325 POINT(39.898231909352575 74.07765349435145) bank79325 +79326 POINT(39.88938208883966 73.25404865558372) bank79326 +79327 POINT(40.28683310238604 73.17396675018374) bank79327 +79328 POINT(40.942736252189164 73.59620707195039) bank79328 +79329 POINT(40.26849465527007 73.0268463323038) bank79329 +79330 POINT(39.826686686376014 73.4870108774229) bank79330 +79331 POINT(40.70128973905532 74.93242572135185) bank79331 +79332 POINT(41.19687372963676 73.96696861161251) bank79332 +79333 POINT(41.30556032313599 74.2840291251816) bank79333 +79334 POINT(40.09146106231451 74.46800274886111) bank79334 +79335 POINT(40.70292324058719 73.98303274146159) bank79335 +79336 POINT(41.595669503075335 73.07770062621285) bank79336 +79337 POINT(40.640984054203216 74.26661807747777) bank79337 +79338 POINT(39.87724497314806 74.23473394848462) bank79338 +79339 POINT(41.25669919957753 73.25969973915349) bank79339 +79340 POINT(40.570396440956316 74.70038689420234) bank79340 +79341 POINT(41.56208772210655 73.65556280820844) bank79341 +79342 POINT(40.581533593806824 74.18988240948804) bank79342 +79343 POINT(40.16352011210374 73.7912530776914) bank79343 +79344 POINT(40.27595396630973 74.3718054196372) bank79344 +79345 POINT(40.18382493159544 74.47693914505943) bank79345 +79346 POINT(40.81526931758078 73.16827363402994) bank79346 +79347 POINT(40.185895271671676 74.54982168676396) bank79347 +79348 POINT(41.70882902793357 74.86155666527958) bank79348 +79349 POINT(40.4173354542417 73.70638391544824) bank79349 +79350 POINT(40.46774664230055 73.88945426830016) bank79350 +79351 POINT(41.597506023197155 73.0435201856057) bank79351 +79352 POINT(41.366868371292156 73.55483442689763) bank79352 +79353 POINT(40.336310483866384 73.08895006445042) bank79353 +79354 POINT(39.725143168863816 73.42727142801255) bank79354 +79355 POINT(41.359557811808 73.97439097173249) bank79355 +79356 POINT(39.8670372031644 74.44190499463609) bank79356 +79357 POINT(39.96996852891901 73.44505952333816) bank79357 +79358 POINT(40.343486556312925 73.93722749735848) bank79358 +79359 POINT(41.24739509637773 74.58929642080186) bank79359 +79360 POINT(39.73502377762797 74.13980234041887) bank79360 +79361 POINT(41.43291053646438 73.42706619693558) bank79361 +79362 POINT(40.22695492902544 73.22289222343093) bank79362 +79363 POINT(40.587977318524935 73.35043238479135) bank79363 +79364 POINT(41.363675081254335 74.75844911502014) bank79364 +79365 POINT(41.57800160844647 74.29258174487495) bank79365 +79366 POINT(41.253591755332266 74.59825217997007) bank79366 +79367 POINT(40.00674998251384 73.69447103624881) bank79367 +79368 POINT(40.51796193747038 73.56316896184177) bank79368 +79369 POINT(41.053069508756096 73.08702341581156) bank79369 +79370 POINT(40.23458635386246 74.52441007852806) bank79370 +79371 POINT(40.42435837124021 74.17987128042009) bank79371 +79372 POINT(40.22312466980923 74.35740964955171) bank79372 +79373 POINT(39.96528449890765 73.30857411192562) bank79373 +79374 POINT(41.33259206831174 73.84437512597816) bank79374 +79375 POINT(40.774750855109204 74.37546094848639) bank79375 +79376 POINT(39.87286539557201 73.32618612774286) bank79376 +79377 POINT(41.29419070239141 74.26273631869172) bank79377 +79378 POINT(39.86627254864537 74.49279700462522) bank79378 +79379 POINT(40.73280709539421 74.16704638178018) bank79379 +79380 POINT(40.007650297009405 73.74427956422645) bank79380 +79381 POINT(41.22306172621152 73.16043166507201) bank79381 +79382 POINT(39.77061678373187 74.57761977929928) bank79382 +79383 POINT(41.262452212032045 75.00266365624766) bank79383 +79384 POINT(40.55689119061015 74.2974027104982) bank79384 +79385 POINT(39.98273790837915 74.54994214835543) bank79385 +79386 POINT(41.002351936259515 74.68071278808333) bank79386 +79387 POINT(40.98532073017357 73.29750233504774) bank79387 +79388 POINT(40.83326755462112 73.98473246134198) bank79388 +79389 POINT(40.57765423400412 74.2013824737234) bank79389 +79390 POINT(40.435855688324935 73.40634711449195) bank79390 +79391 POINT(41.207604390670895 73.45851283640879) bank79391 +79392 POINT(40.468821216022015 73.71226229540943) bank79392 +79393 POINT(40.500117465674144 73.3463029855868) bank79393 +79394 POINT(39.80349065018043 74.18769944820335) bank79394 +79395 POINT(40.27624437584519 74.29346683127703) bank79395 +79396 POINT(39.743923526037854 74.87195292288904) bank79396 +79397 POINT(41.010590026056526 73.42989986936492) bank79397 +79398 POINT(40.142974443583675 73.19950751219135) bank79398 +79399 POINT(40.90150235707085 74.73081538056898) bank79399 +79400 POINT(39.79453985587493 74.17084632490084) bank79400 +79401 POINT(41.3021690628206 73.93270731092338) bank79401 +79402 POINT(40.81461279944418 73.9511811535957) bank79402 +79403 POINT(40.19978777870759 74.19500331672299) bank79403 +79404 POINT(40.575448962076095 74.39496606872397) bank79404 +79405 POINT(39.89547085124913 74.15905718455393) bank79405 +79406 POINT(40.508535683391784 74.10903202734302) bank79406 +79407 POINT(39.94818006397253 74.96635539466578) bank79407 +79408 POINT(39.913406998544424 74.23160369859393) bank79408 +79409 POINT(40.464506650765564 74.99663163004132) bank79409 +79410 POINT(40.089758793522364 73.76503777093691) bank79410 +79411 POINT(39.82932550198746 74.69628576770435) bank79411 +79412 POINT(40.5960746969983 73.41664347442878) bank79412 +79413 POINT(40.41433139298033 74.31384462898696) bank79413 +79414 POINT(41.56612433342289 73.66264596414216) bank79414 +79415 POINT(40.44159842477821 74.26190246638956) bank79415 +79416 POINT(40.61241569407744 74.58446151313234) bank79416 +79417 POINT(40.77484005559548 74.90826767971345) bank79417 +79418 POINT(40.86410856503955 73.72096477622141) bank79418 +79419 POINT(40.48398081169622 73.14115787601501) bank79419 +79420 POINT(39.77731630035337 74.99635428811385) bank79420 +79421 POINT(40.59002196628227 73.42427847388558) bank79421 +79422 POINT(41.1872733226295 73.98866471604899) bank79422 +79423 POINT(40.985796471850236 73.71300931579238) bank79423 +79424 POINT(40.96821163569154 73.36628297839572) bank79424 +79425 POINT(40.649383837445384 74.64774403590621) bank79425 +79426 POINT(41.1315834280111 74.63221591631861) bank79426 +79427 POINT(40.38635438285643 73.68612782886197) bank79427 +79428 POINT(40.120642878218995 73.62464154981362) bank79428 +79429 POINT(39.94475707380606 73.42924269040694) bank79429 +79430 POINT(40.74167185490059 74.72996581989666) bank79430 +79431 POINT(41.352240133714844 73.87409161201862) bank79431 +79432 POINT(40.51320213852652 74.28315573346372) bank79432 +79433 POINT(39.72621247913493 73.93811487245333) bank79433 +79434 POINT(40.11813366394354 74.8711828887652) bank79434 +79435 POINT(40.90145797684264 73.8741862071943) bank79435 +79436 POINT(41.61350331529559 73.11811412359131) bank79436 +79437 POINT(41.676135539900265 74.40552990336099) bank79437 +79438 POINT(40.62237409303214 74.75100458427512) bank79438 +79439 POINT(41.37551934497903 73.4465181747839) bank79439 +79440 POINT(40.40157905104892 74.0905006103374) bank79440 +79441 POINT(41.49018851390041 74.74696364746647) bank79441 +79442 POINT(39.879816679660344 73.29228223649478) bank79442 +79443 POINT(41.68879834080493 73.1653017550534) bank79443 +79444 POINT(40.23113736667732 73.46040093737574) bank79444 +79445 POINT(41.081077368227476 74.48959947859088) bank79445 +79446 POINT(41.22658605768989 73.57733442239969) bank79446 +79447 POINT(40.88850630650491 73.9720154308111) bank79447 +79448 POINT(40.13594786068572 74.00345845761112) bank79448 +79449 POINT(41.09768448180406 74.2355466754667) bank79449 +79450 POINT(41.6158700538625 74.3712779570712) bank79450 +79451 POINT(41.65899965297503 73.8317061746966) bank79451 +79452 POINT(40.920170491832685 73.60624496336416) bank79452 +79453 POINT(40.96808187448778 73.23737920348077) bank79453 +79454 POINT(41.2857687046331 74.01619175544212) bank79454 +79455 POINT(40.62714226642151 73.3463081010249) bank79455 +79456 POINT(41.27382398766881 73.33319167064262) bank79456 +79457 POINT(40.59759017397109 74.61256535940069) bank79457 +79458 POINT(40.74745711060998 74.75371747226097) bank79458 +79459 POINT(41.234251226066974 73.84888002510299) bank79459 +79460 POINT(41.25795064625686 73.3245462078524) bank79460 +79461 POINT(40.559134469061746 74.85937651823922) bank79461 +79462 POINT(40.810800109223926 73.0394349310249) bank79462 +79463 POINT(41.22546269827126 74.24496617883842) bank79463 +79464 POINT(40.95623179169223 73.14780446646928) bank79464 +79465 POINT(40.404436982978815 74.48421101754963) bank79465 +79466 POINT(40.969141859563855 73.12194809703853) bank79466 +79467 POINT(40.347847545889024 73.6466615614139) bank79467 +79468 POINT(41.27877249627933 73.16937304145944) bank79468 +79469 POINT(39.93149874689226 74.13958178530008) bank79469 +79470 POINT(40.436365088150886 73.18965381889028) bank79470 +79471 POINT(41.26502589383341 73.91823518782357) bank79471 +79472 POINT(40.786474293601685 74.72844539636138) bank79472 +79473 POINT(40.49702367323431 74.44220744864583) bank79473 +79474 POINT(41.25570945993352 73.48551074037599) bank79474 +79475 POINT(41.12305798613775 74.01127272929101) bank79475 +79476 POINT(41.383453473700676 73.93456753846377) bank79476 +79477 POINT(40.17963050420363 74.29057880827533) bank79477 +79478 POINT(39.785637271711416 73.16225924975592) bank79478 +79479 POINT(41.39959868451015 74.20136875971474) bank79479 +79480 POINT(39.7173626227608 73.50589972877606) bank79480 +79481 POINT(40.2620873217271 74.41728860765974) bank79481 +79482 POINT(41.0359905838338 74.60230398104316) bank79482 +79483 POINT(40.23228674204369 73.61523821562811) bank79483 +79484 POINT(40.31719027156006 73.15274273109377) bank79484 +79485 POINT(40.76036393537954 73.24346039910597) bank79485 +79486 POINT(41.303512381637375 74.42378797326474) bank79486 +79487 POINT(40.59840267369278 74.62803529712892) bank79487 +79488 POINT(40.8918483359366 74.04678450814514) bank79488 +79489 POINT(40.97845553347535 74.85875695795693) bank79489 +79490 POINT(41.37011140001766 73.58240904000047) bank79490 +79491 POINT(40.42022792192517 73.79244217497572) bank79491 +79492 POINT(41.328345835825964 73.08895845839827) bank79492 +79493 POINT(39.808158747331134 73.42728567880503) bank79493 +79494 POINT(41.62053046995068 74.78926190925922) bank79494 +79495 POINT(40.50923630434945 73.64103981924652) bank79495 +79496 POINT(41.05167639757893 73.56703960579632) bank79496 +79497 POINT(40.94703556494051 74.34942008206025) bank79497 +79498 POINT(41.583375065485505 74.52259159334047) bank79498 +79499 POINT(40.02678999358565 74.16621662224526) bank79499 +79500 POINT(39.86299259322777 73.19255383407403) bank79500 +79501 POINT(41.10975189413467 74.55257517574012) bank79501 +79502 POINT(40.17636050810818 74.80802312361119) bank79502 +79503 POINT(39.86969169184752 74.98743231069032) bank79503 +79504 POINT(41.529345499460746 74.9209155659361) bank79504 +79505 POINT(40.76622467611913 74.63492121955666) bank79505 +79506 POINT(41.11592003523859 73.27608506001081) bank79506 +79507 POINT(39.978989694630975 74.55269088662439) bank79507 +79508 POINT(40.72136750122561 74.71173814960477) bank79508 +79509 POINT(40.09422111283433 73.59079855651805) bank79509 +79510 POINT(40.137675681395926 74.76925357758333) bank79510 +79511 POINT(40.60643077211938 73.84099216216472) bank79511 +79512 POINT(40.81762847524447 74.32599247305232) bank79512 +79513 POINT(40.83344188255904 74.56089461083542) bank79513 +79514 POINT(40.35672720751207 73.2711467067824) bank79514 +79515 POINT(40.40869776690504 74.37237718374958) bank79515 +79516 POINT(40.16623140608567 74.9302604950841) bank79516 +79517 POINT(40.933885969211474 74.81418565544121) bank79517 +79518 POINT(40.40134196623566 74.13834695128311) bank79518 +79519 POINT(40.99951565006317 74.70545179687403) bank79519 +79520 POINT(40.942241325575935 73.45846723554362) bank79520 +79521 POINT(40.04319177413866 73.40908824074928) bank79521 +79522 POINT(41.61289311110817 73.59752893287158) bank79522 +79523 POINT(40.280062955069496 74.01666021793851) bank79523 +79524 POINT(39.80348240305372 74.55069633670338) bank79524 +79525 POINT(40.09745008124908 74.73504657532402) bank79525 +79526 POINT(40.93087846733864 73.41792636283613) bank79526 +79527 POINT(41.02610312687127 74.92383168345297) bank79527 +79528 POINT(40.175307758084344 73.43272896428967) bank79528 +79529 POINT(41.23864074782556 74.55433594851227) bank79529 +79530 POINT(40.817610004916865 74.49376802864089) bank79530 +79531 POINT(40.548627604803514 73.40850294588914) bank79531 +79532 POINT(41.38332777582003 73.10390151314283) bank79532 +79533 POINT(41.507926002459236 74.75786477724402) bank79533 +79534 POINT(41.11448233784328 73.01655306782388) bank79534 +79535 POINT(40.892403344090916 73.97725976030091) bank79535 +79536 POINT(41.03529513334336 73.83185354652062) bank79536 +79537 POINT(41.170333195092226 73.31859319759238) bank79537 +79538 POINT(40.49535189118674 73.77513232722647) bank79538 +79539 POINT(40.59263562206491 74.91972452080995) bank79539 +79540 POINT(40.86605038837501 73.87697466303143) bank79540 +79541 POINT(40.04068135263977 74.65792826432696) bank79541 +79542 POINT(39.94141682803061 74.32678402124739) bank79542 +79543 POINT(40.693489516176335 74.41846805087262) bank79543 +79544 POINT(40.65491334122793 73.35497974425711) bank79544 +79545 POINT(41.708537631010856 74.64006809051959) bank79545 +79546 POINT(40.96272241651507 74.05626871734765) bank79546 +79547 POINT(41.32113743524173 74.7107436815503) bank79547 +79548 POINT(40.21447862459256 74.35425822240651) bank79548 +79549 POINT(39.87387960368256 74.00956189463821) bank79549 +79550 POINT(41.291244266158955 73.88731314221671) bank79550 +79551 POINT(41.69975421081643 73.89805794581042) bank79551 +79552 POINT(40.93553758773259 73.27291616181472) bank79552 +79553 POINT(41.08535987414389 73.30423395218841) bank79553 +79554 POINT(40.145007449732304 73.39152298469115) bank79554 +79555 POINT(39.85758305359572 74.80017866836809) bank79555 +79556 POINT(41.15753268363728 73.96405178819424) bank79556 +79557 POINT(40.89075404421479 74.95114910992453) bank79557 +79558 POINT(40.87579503892741 73.4510036072389) bank79558 +79559 POINT(40.82887271280287 74.8658285417535) bank79559 +79560 POINT(40.97187470249487 73.28733746389177) bank79560 +79561 POINT(40.76599821990418 74.16810982087986) bank79561 +79562 POINT(41.18877095097205 74.30478348934489) bank79562 +79563 POINT(40.96811763720308 73.62175637518021) bank79563 +79564 POINT(40.92004182302957 73.20107538345042) bank79564 +79565 POINT(41.557824011716235 74.62343007943845) bank79565 +79566 POINT(39.863899708735794 73.55105701555328) bank79566 +79567 POINT(41.17521257074253 74.55404601653844) bank79567 +79568 POINT(40.154544091516236 74.26860614644261) bank79568 +79569 POINT(40.87163705051146 74.62286832083907) bank79569 +79570 POINT(40.32920202824574 74.15111314184368) bank79570 +79571 POINT(41.358448879936574 73.2000209330514) bank79571 +79572 POINT(41.592611838977014 74.93589740976365) bank79572 +79573 POINT(40.076644745388705 73.85392601186663) bank79573 +79574 POINT(40.91876401864278 74.61738814738307) bank79574 +79575 POINT(40.87733110976131 73.93768948514484) bank79575 +79576 POINT(40.425129850927235 73.6434069934749) bank79576 +79577 POINT(40.802550736789904 74.70486419870208) bank79577 +79578 POINT(41.133005195421795 74.58073130373515) bank79578 +79579 POINT(41.119093112173346 74.22081261067936) bank79579 +79580 POINT(40.83136100504318 74.60509835952838) bank79580 +79581 POINT(41.6783747024352 73.24896601092027) bank79581 +79582 POINT(39.93324479217398 74.3351995568847) bank79582 +79583 POINT(41.237061173541676 73.06021075394088) bank79583 +79584 POINT(40.34749352124289 74.00928916942492) bank79584 +79585 POINT(40.86385656054477 73.1077802806515) bank79585 +79586 POINT(40.746780414755946 74.66662208503011) bank79586 +79587 POINT(40.14048761919916 73.41333667455089) bank79587 +79588 POINT(41.45364165073966 74.11589986566047) bank79588 +79589 POINT(39.85724968575542 73.8828663749495) bank79589 +79590 POINT(40.3811413422728 74.64111966038892) bank79590 +79591 POINT(39.92308370416918 73.2793524513568) bank79591 +79592 POINT(40.689506134342736 74.6644841583469) bank79592 +79593 POINT(41.66117721792714 74.45513630956866) bank79593 +79594 POINT(40.476120082367295 74.55011256733032) bank79594 +79595 POINT(40.40228980024666 74.9003569572234) bank79595 +79596 POINT(40.83139854411283 73.39487174231684) bank79596 +79597 POINT(40.70804445091058 74.31662492896685) bank79597 +79598 POINT(40.67076458902214 74.20562879320185) bank79598 +79599 POINT(39.76026918433427 74.6039729834242) bank79599 +79600 POINT(39.767922949518244 74.23875240359897) bank79600 +79601 POINT(40.14316255178971 73.74074270474138) bank79601 +79602 POINT(40.58059368362237 74.21574063986809) bank79602 +79603 POINT(40.49736153131037 73.02376278241464) bank79603 +79604 POINT(41.10277006529002 74.78996088194442) bank79604 +79605 POINT(40.367661489244156 73.03085092021547) bank79605 +79606 POINT(41.24854151050177 74.3266538377407) bank79606 +79607 POINT(39.96133679642482 73.55342501171347) bank79607 +79608 POINT(40.357091873586434 74.01463618692763) bank79608 +79609 POINT(40.49562653583198 73.44721371044929) bank79609 +79610 POINT(40.70253225793618 73.45825073686545) bank79610 +79611 POINT(40.18932147508986 74.51952673649585) bank79611 +79612 POINT(40.51465446292783 74.50862076113877) bank79612 +79613 POINT(40.865024768293914 73.83728282026985) bank79613 +79614 POINT(41.5608564883165 74.1374957656377) bank79614 +79615 POINT(41.34926633339073 73.31744119585954) bank79615 +79616 POINT(40.38428936873021 73.11472484485563) bank79616 +79617 POINT(41.27422696660107 73.10921587949873) bank79617 +79618 POINT(41.34367573568477 74.44803107008416) bank79618 +79619 POINT(39.94083335989845 74.25397451433899) bank79619 +79620 POINT(40.936524992597086 74.94063235350333) bank79620 +79621 POINT(40.490599788430515 74.02804783461137) bank79621 +79622 POINT(40.8898453576929 74.40452539773541) bank79622 +79623 POINT(41.51339296412265 73.09548837994559) bank79623 +79624 POINT(40.36483014496602 73.35622700702095) bank79624 +79625 POINT(41.23666326717484 73.94719915049052) bank79625 +79626 POINT(41.479292950076044 74.74864669180539) bank79626 +79627 POINT(40.54222962947074 74.26350245044637) bank79627 +79628 POINT(41.596376559717115 73.48970967757802) bank79628 +79629 POINT(40.31544872636936 74.38065235112406) bank79629 +79630 POINT(41.06704579720059 74.29759580681124) bank79630 +79631 POINT(40.94567458645163 73.54022646299201) bank79631 +79632 POINT(40.94632677190335 73.53367949845085) bank79632 +79633 POINT(39.90620740552654 73.13304408478743) bank79633 +79634 POINT(40.83015304562045 74.65886360453374) bank79634 +79635 POINT(40.53897383187054 74.70220855438882) bank79635 +79636 POINT(41.61860942444986 73.30194650027366) bank79636 +79637 POINT(40.69444642816586 74.0561711600638) bank79637 +79638 POINT(41.053933926327794 74.63372976117657) bank79638 +79639 POINT(41.654053954895396 74.96244682321932) bank79639 +79640 POINT(40.35824356526702 73.0828505564019) bank79640 +79641 POINT(40.596532790795656 74.64502068030619) bank79641 +79642 POINT(40.56722335051602 73.85993208324163) bank79642 +79643 POINT(41.1011843028924 73.21103524844493) bank79643 +79644 POINT(40.96263018806743 73.70519749096914) bank79644 +79645 POINT(40.59227700293249 73.90069333448409) bank79645 +79646 POINT(40.7610055776341 74.1521994423989) bank79646 +79647 POINT(39.77049336092945 73.46719036832924) bank79647 +79648 POINT(40.94364945132102 73.79605417285613) bank79648 +79649 POINT(40.6602489755587 73.36258312139476) bank79649 +79650 POINT(40.16077742071128 74.81269334643551) bank79650 +79651 POINT(40.14682169003048 74.53217361680686) bank79651 +79652 POINT(41.13638235090289 73.88207544058167) bank79652 +79653 POINT(40.970401251888234 74.2178332193415) bank79653 +79654 POINT(40.113671375841356 73.13561179571136) bank79654 +79655 POINT(39.806397779916985 74.12813601607884) bank79655 +79656 POINT(39.80419097985819 73.0660318352918) bank79656 +79657 POINT(40.805906529099786 74.59959545896074) bank79657 +79658 POINT(41.181961224588036 74.77315431171421) bank79658 +79659 POINT(40.74727875212068 73.9109905146287) bank79659 +79660 POINT(41.269990679269696 74.1664634345946) bank79660 +79661 POINT(39.76210579766854 74.03540331037205) bank79661 +79662 POINT(40.324510417709256 74.88703877970026) bank79662 +79663 POINT(41.527732188671536 73.00757757510483) bank79663 +79664 POINT(40.73856904042554 74.28674857235153) bank79664 +79665 POINT(40.807496166888555 74.84207303309543) bank79665 +79666 POINT(40.35031435624496 74.70516382241257) bank79666 +79667 POINT(41.61993295778992 73.4565857541197) bank79667 +79668 POINT(41.53770834396715 73.3742993256326) bank79668 +79669 POINT(39.81456415839475 74.7929656262524) bank79669 +79670 POINT(40.23919507423371 74.73145208729362) bank79670 +79671 POINT(40.41492694086968 74.16790704457942) bank79671 +79672 POINT(40.78184198396246 74.6574753168069) bank79672 +79673 POINT(40.91878974094549 73.30711722355154) bank79673 +79674 POINT(41.58281657100421 73.46541648325724) bank79674 +79675 POINT(41.28538150358997 74.75520379589138) bank79675 +79676 POINT(40.72944099761305 74.84416679174022) bank79676 +79677 POINT(40.57849764415858 73.90126236527283) bank79677 +79678 POINT(40.16782519950762 73.38106121915081) bank79678 +79679 POINT(41.52967516005688 73.3009860770218) bank79679 +79680 POINT(41.230770581412415 74.79397033645138) bank79680 +79681 POINT(40.346026806871656 73.70134836622209) bank79681 +79682 POINT(40.666946900905344 73.14837158092762) bank79682 +79683 POINT(40.245047261281044 74.92878789339464) bank79683 +79684 POINT(41.43136578876442 74.2899129320707) bank79684 +79685 POINT(40.02565902272699 74.56814036209235) bank79685 +79686 POINT(41.40340023058263 74.7983536858182) bank79686 +79687 POINT(39.743126944247216 74.73804323640485) bank79687 +79688 POINT(40.607467149082964 74.8661706975535) bank79688 +79689 POINT(41.09457010614563 73.27328782584111) bank79689 +79690 POINT(40.74524601309319 73.52002193896264) bank79690 +79691 POINT(40.1347543863373 74.07386749545626) bank79691 +79692 POINT(40.97672208894549 74.85709600053453) bank79692 +79693 POINT(40.414598162469154 74.67485804427783) bank79693 +79694 POINT(40.83238151828459 73.12992682274627) bank79694 +79695 POINT(41.645069966921184 74.32757858444938) bank79695 +79696 POINT(41.20968077762493 74.73247024872433) bank79696 +79697 POINT(41.04300597577776 73.10029763706156) bank79697 +79698 POINT(41.19200112010828 74.69804622087771) bank79698 +79699 POINT(40.66682022101598 74.44548935655418) bank79699 +79700 POINT(41.473286000172195 74.53460827334325) bank79700 +79701 POINT(41.20074199046436 74.4600388416499) bank79701 +79702 POINT(40.529502541103625 73.74498707921447) bank79702 +79703 POINT(41.17437823260477 74.40144184579987) bank79703 +79704 POINT(41.05538752272867 73.03169280694081) bank79704 +79705 POINT(40.32705406272685 73.62246354770275) bank79705 +79706 POINT(41.21707550657337 74.80251016841359) bank79706 +79707 POINT(41.214712610727055 74.67842071169248) bank79707 +79708 POINT(40.57923704719285 73.52852239768467) bank79708 +79709 POINT(40.135154584679704 74.03434366945555) bank79709 +79710 POINT(39.78069058581074 74.05658412204262) bank79710 +79711 POINT(40.52806205284874 73.77386314994418) bank79711 +79712 POINT(39.79498194342004 73.40888248370135) bank79712 +79713 POINT(40.180414821725854 74.15184841830008) bank79713 +79714 POINT(39.78790693728382 74.0256932107555) bank79714 +79715 POINT(40.14210777986499 73.53715361541464) bank79715 +79716 POINT(40.2741183214025 74.75714640619917) bank79716 +79717 POINT(41.63375115533976 74.24086206934786) bank79717 +79718 POINT(40.75049096525326 73.32369519400768) bank79718 +79719 POINT(40.834193618196814 74.69472464701799) bank79719 +79720 POINT(40.20428542517687 74.14641367235552) bank79720 +79721 POINT(40.68607785478253 74.6458240920863) bank79721 +79722 POINT(40.65801601286336 74.62753727624049) bank79722 +79723 POINT(40.39081305200787 74.18599710395873) bank79723 +79724 POINT(40.84091469018914 74.96670050378103) bank79724 +79725 POINT(41.208648753372714 73.5958897055153) bank79725 +79726 POINT(41.568698117186976 74.69841701598497) bank79726 +79727 POINT(41.374063820144535 74.9537656029532) bank79727 +79728 POINT(40.68280026847319 74.75114970831324) bank79728 +79729 POINT(40.55025573915933 73.14742930574641) bank79729 +79730 POINT(41.48385466270885 74.28549138807833) bank79730 +79731 POINT(40.596604919697526 74.23891218987383) bank79731 +79732 POINT(41.120957509915726 74.57157941691322) bank79732 +79733 POINT(40.02572097577976 73.68868805010571) bank79733 +79734 POINT(40.251507524020724 73.3266341322666) bank79734 +79735 POINT(41.15021410269325 73.82948058559566) bank79735 +79736 POINT(40.726420440863784 73.73741763725621) bank79736 +79737 POINT(40.85850625859493 74.15843911963175) bank79737 +79738 POINT(41.08638704394946 74.92417872260472) bank79738 +79739 POINT(41.5309551353367 74.23827333558881) bank79739 +79740 POINT(41.580022425232386 73.08232883961873) bank79740 +79741 POINT(39.837376417772795 74.55629429508278) bank79741 +79742 POINT(40.09147862586032 74.11863695226499) bank79742 +79743 POINT(41.17782430613193 74.6234638218089) bank79743 +79744 POINT(41.2818433466481 74.4740602301793) bank79744 +79745 POINT(41.2301677603924 74.79340176131969) bank79745 +79746 POINT(41.6391371025656 73.40206337280856) bank79746 +79747 POINT(41.20245119113405 74.80519968090496) bank79747 +79748 POINT(40.51351510356339 74.2815365747776) bank79748 +79749 POINT(41.04234302040303 73.63402221854408) bank79749 +79750 POINT(40.593128347298574 74.7578938452894) bank79750 +79751 POINT(41.34162777409271 74.47800772740035) bank79751 +79752 POINT(40.57263705119429 73.7208722620905) bank79752 +79753 POINT(41.165418774517356 74.09534708950096) bank79753 +79754 POINT(41.22406756631105 74.23805534971005) bank79754 +79755 POINT(40.924692998714434 73.9537637924578) bank79755 +79756 POINT(40.08825273813244 74.15281683201724) bank79756 +79757 POINT(41.06461451658357 74.7096856039585) bank79757 +79758 POINT(39.97964796925846 74.204324967791) bank79758 +79759 POINT(41.6526549125655 74.5154426678901) bank79759 +79760 POINT(41.571740709625395 74.45646119008737) bank79760 +79761 POINT(41.46346729482302 74.98418185394468) bank79761 +79762 POINT(40.14886186254547 73.98139658647544) bank79762 +79763 POINT(39.75590585475444 74.50296785046244) bank79763 +79764 POINT(39.811310272850065 73.8546570915377) bank79764 +79765 POINT(41.66143482738242 74.15809099654598) bank79765 +79766 POINT(41.256131942059596 74.72054115116083) bank79766 +79767 POINT(41.26527016067692 73.85070360360996) bank79767 +79768 POINT(40.62396549341712 74.86062216244031) bank79768 +79769 POINT(41.511265629144766 74.89182557501587) bank79769 +79770 POINT(39.81214437436445 73.93038276952797) bank79770 +79771 POINT(39.755281069556816 73.92200055510182) bank79771 +79772 POINT(40.7266201258928 73.14640705896916) bank79772 +79773 POINT(41.37907683941858 74.83331142866659) bank79773 +79774 POINT(39.873385946889506 74.1409237727003) bank79774 +79775 POINT(41.356674790161 74.48067050659692) bank79775 +79776 POINT(40.85744637484765 73.56440266860032) bank79776 +79777 POINT(40.37117624786133 73.89917227058231) bank79777 +79778 POINT(41.30815291313292 73.4491046493681) bank79778 +79779 POINT(39.95132918959923 73.68360116099075) bank79779 +79780 POINT(41.340766398430524 74.7998580839178) bank79780 +79781 POINT(39.825394832646104 74.34741393474417) bank79781 +79782 POINT(40.554966046195425 73.06849506201613) bank79782 +79783 POINT(41.4999332854498 74.97137134778333) bank79783 +79784 POINT(40.17346985366706 74.87544330779055) bank79784 +79785 POINT(40.45849872916037 74.11730754599873) bank79785 +79786 POINT(40.812718494720826 74.3113605891088) bank79786 +79787 POINT(41.34390491668956 74.83952816374828) bank79787 +79788 POINT(39.79277122530612 73.6574781347081) bank79788 +79789 POINT(39.83800895248991 73.91261780049021) bank79789 +79790 POINT(41.69527677039622 74.5706672712084) bank79790 +79791 POINT(40.98815863560501 74.53628269934563) bank79791 +79792 POINT(39.7157030866278 74.4078861829461) bank79792 +79793 POINT(41.17114789111418 73.11447878278365) bank79793 +79794 POINT(40.52391552730734 74.01602749097148) bank79794 +79795 POINT(40.733808804264264 74.3323753252925) bank79795 +79796 POINT(41.61497933536444 73.96460440896406) bank79796 +79797 POINT(40.424174362457684 73.96006506375198) bank79797 +79798 POINT(40.044372246392896 74.77202713623888) bank79798 +79799 POINT(41.347462473598014 74.77409757979413) bank79799 +79800 POINT(41.65293996246372 74.06333249348425) bank79800 +79801 POINT(41.05175384095187 73.24193252611111) bank79801 +79802 POINT(41.17648383982265 74.8313390058511) bank79802 +79803 POINT(40.34609254498565 74.46582807124) bank79803 +79804 POINT(40.749414872538786 73.77495835664297) bank79804 +79805 POINT(39.98941719757522 73.34499369275359) bank79805 +79806 POINT(41.06845422446813 73.72718189297737) bank79806 +79807 POINT(40.73231629955675 74.59951145708432) bank79807 +79808 POINT(40.0629368409312 73.28808020475493) bank79808 +79809 POINT(39.870288637926684 74.20718883460115) bank79809 +79810 POINT(39.78678700586554 74.00903665653114) bank79810 +79811 POINT(41.016632520003846 73.66739935569923) bank79811 +79812 POINT(41.69899849290643 73.96349128718546) bank79812 +79813 POINT(40.04217906886469 73.32598586735976) bank79813 +79814 POINT(40.95233272661661 74.55223380121585) bank79814 +79815 POINT(41.00651856090275 73.68311833447186) bank79815 +79816 POINT(40.70108277011488 73.97152174321754) bank79816 +79817 POINT(39.85326138867507 73.51441090414971) bank79817 +79818 POINT(39.89903316091655 74.35693634961164) bank79818 +79819 POINT(39.90839425444543 73.02600709001969) bank79819 +79820 POINT(40.90001749421276 73.10442150699485) bank79820 +79821 POINT(40.25139414213681 73.84561466510048) bank79821 +79822 POINT(41.44062084337681 73.1709303671215) bank79822 +79823 POINT(40.04987851260692 73.99993350448742) bank79823 +79824 POINT(39.86439971577636 74.856349371627) bank79824 +79825 POINT(41.05420818376441 73.85396773016407) bank79825 +79826 POINT(40.44670814273047 74.83277117218107) bank79826 +79827 POINT(40.100200471962296 73.36965716475206) bank79827 +79828 POINT(41.65444601317632 74.46308574809821) bank79828 +79829 POINT(40.123343188556824 74.72786946567926) bank79829 +79830 POINT(40.946798259506885 74.67591143978333) bank79830 +79831 POINT(39.72732495673145 74.85660355446805) bank79831 +79832 POINT(40.727868852834405 73.13406389404317) bank79832 +79833 POINT(41.426556733847065 74.433164898135) bank79833 +79834 POINT(41.479998951486536 73.89121668572518) bank79834 +79835 POINT(41.68149064533687 73.63298397200406) bank79835 +79836 POINT(41.30686686740836 74.13277078109381) bank79836 +79837 POINT(41.194228825656516 73.54320108351541) bank79837 +79838 POINT(40.696361669627464 73.23652547010903) bank79838 +79839 POINT(40.65710191244531 73.18701185591152) bank79839 +79840 POINT(41.47263477810988 73.17928120101516) bank79840 +79841 POINT(41.34664446094397 74.73189265923688) bank79841 +79842 POINT(41.026263886393714 74.28486332499133) bank79842 +79843 POINT(40.13969857484971 74.26543287247418) bank79843 +79844 POINT(40.622350901728964 74.6718950205894) bank79844 +79845 POINT(41.325167442759806 74.5093220414007) bank79845 +79846 POINT(40.05072785596588 74.16808515426375) bank79846 +79847 POINT(40.90627831009039 73.22645786589955) bank79847 +79848 POINT(40.56417111596591 74.86886806306087) bank79848 +79849 POINT(40.70581365447894 73.85715907427927) bank79849 +79850 POINT(40.30363344789674 74.48885141687161) bank79850 +79851 POINT(40.684837235532264 73.92045069792967) bank79851 +79852 POINT(40.60438873772436 73.2053698466812) bank79852 +79853 POINT(41.65210761547415 74.2842122474298) bank79853 +79854 POINT(40.63984662964059 74.42862903818849) bank79854 +79855 POINT(40.95940957327364 73.79312700793714) bank79855 +79856 POINT(41.47392990819159 73.96770373526942) bank79856 +79857 POINT(41.03049649112466 73.55685872879762) bank79857 +79858 POINT(40.41340150806644 74.71307837787235) bank79858 +79859 POINT(39.90316071871313 73.33445508559733) bank79859 +79860 POINT(40.562060471184225 73.54169202272442) bank79860 +79861 POINT(41.35444167170997 73.28606302914979) bank79861 +79862 POINT(40.53898031452287 74.88802962958455) bank79862 +79863 POINT(41.07833194862441 73.7091216747583) bank79863 +79864 POINT(40.83065363252206 73.4899361409553) bank79864 +79865 POINT(41.21267898579076 74.1546654254257) bank79865 +79866 POINT(40.90706577891841 74.74835410725562) bank79866 +79867 POINT(40.951061673983965 74.57105411017719) bank79867 +79868 POINT(41.525619014795005 74.32128603006699) bank79868 +79869 POINT(41.51714767188144 74.34370405553933) bank79869 +79870 POINT(40.45052295114619 73.8314136970432) bank79870 +79871 POINT(41.4665435168977 73.0853721426201) bank79871 +79872 POINT(41.56435857656584 73.4345444280987) bank79872 +79873 POINT(41.10205127416635 74.70658951151111) bank79873 +79874 POINT(40.74344201267565 73.04236855863724) bank79874 +79875 POINT(41.20486722297595 74.77338585640007) bank79875 +79876 POINT(41.03689512165088 73.42060009045248) bank79876 +79877 POINT(41.289476695988064 74.34246395565737) bank79877 +79878 POINT(40.11202595906105 73.7689390645492) bank79878 +79879 POINT(40.55568218649213 73.72111150167134) bank79879 +79880 POINT(40.21951083843018 74.59878782554492) bank79880 +79881 POINT(40.95550265767558 74.8455578493081) bank79881 +79882 POINT(40.94370450959913 73.37640765113629) bank79882 +79883 POINT(40.22750633009049 74.05256994798042) bank79883 +79884 POINT(40.85276772485061 74.82779670191466) bank79884 +79885 POINT(40.63603902155781 73.67341932058382) bank79885 +79886 POINT(41.255648621526824 73.11037451214771) bank79886 +79887 POINT(40.027050928177566 73.4511276602612) bank79887 +79888 POINT(40.607711191495156 74.76813311160171) bank79888 +79889 POINT(41.15558958368247 73.01767076678848) bank79889 +79890 POINT(40.85316983374055 73.13262295605335) bank79890 +79891 POINT(40.083176895427506 73.87780785125173) bank79891 +79892 POINT(40.960627819455674 74.67983662532482) bank79892 +79893 POINT(41.11600995452015 74.78457323484102) bank79893 +79894 POINT(40.5186476176744 73.64438943072234) bank79894 +79895 POINT(40.42936805298525 74.05441625552425) bank79895 +79896 POINT(41.01551211014723 74.60852199113738) bank79896 +79897 POINT(39.96994666051497 73.21683674296223) bank79897 +79898 POINT(39.99281483205717 73.75760312558853) bank79898 +79899 POINT(40.223452694322894 73.85571519738409) bank79899 +79900 POINT(41.309879793285965 73.3447187513008) bank79900 +79901 POINT(41.622525128823405 74.99033759604573) bank79901 +79902 POINT(40.48353747595027 74.9269226752834) bank79902 +79903 POINT(41.40956674723807 73.64575856828026) bank79903 +79904 POINT(40.091506975685284 74.77853343763165) bank79904 +79905 POINT(39.77204654648746 74.45491367581656) bank79905 +79906 POINT(41.14122230197994 74.48557412422039) bank79906 +79907 POINT(41.32309224235465 74.83640752573238) bank79907 +79908 POINT(41.36310530864908 73.36632141296518) bank79908 +79909 POINT(39.76501987283574 74.19094881547366) bank79909 +79910 POINT(39.87409644966993 73.75950485637578) bank79910 +79911 POINT(39.72779547608606 73.79604616897711) bank79911 +79912 POINT(41.29826698671167 73.37399322412962) bank79912 +79913 POINT(41.202340345288555 73.98086244829155) bank79913 +79914 POINT(40.15813387398407 73.62668312278099) bank79914 +79915 POINT(40.852666620518384 73.30259732541475) bank79915 +79916 POINT(40.86654129709005 74.281474022194) bank79916 +79917 POINT(40.997185117053384 73.348361327674) bank79917 +79918 POINT(40.91146271615278 74.54708779321902) bank79918 +79919 POINT(40.98128779828214 74.81733526462615) bank79919 +79920 POINT(40.14518143155639 73.1182744526031) bank79920 +79921 POINT(40.52764970834349 74.86389593982318) bank79921 +79922 POINT(41.10639631461414 73.9856295070466) bank79922 +79923 POINT(41.42599917327609 73.3906359252753) bank79923 +79924 POINT(40.516925310387236 73.22941480094195) bank79924 +79925 POINT(41.56242482339927 74.63050601585388) bank79925 +79926 POINT(40.08516589808927 74.35271165065278) bank79926 +79927 POINT(39.771296452353255 74.36947432530867) bank79927 +79928 POINT(41.5010858842231 73.14245297921592) bank79928 +79929 POINT(40.713124347111176 74.68248429494618) bank79929 +79930 POINT(40.63517505880887 73.54682885159902) bank79930 +79931 POINT(40.54129077158133 74.99920229952276) bank79931 +79932 POINT(39.94388705833182 74.59864005055263) bank79932 +79933 POINT(41.66200335847314 73.49145454794028) bank79933 +79934 POINT(41.11085922451815 74.32396075860895) bank79934 +79935 POINT(40.73436948880709 74.13358794167296) bank79935 +79936 POINT(41.218920710842646 74.21111315180174) bank79936 +79937 POINT(40.238944446918495 73.15035907118151) bank79937 +79938 POINT(41.00110599021782 74.65011902169519) bank79938 +79939 POINT(41.50779518404017 74.8539725411618) bank79939 +79940 POINT(39.73913925641412 75.00023585281767) bank79940 +79941 POINT(41.542559405056565 74.82978191342846) bank79941 +79942 POINT(41.17039841086702 74.61310850654701) bank79942 +79943 POINT(40.41626828904602 74.25991787715604) bank79943 +79944 POINT(39.74295392979133 73.28102457889472) bank79944 +79945 POINT(40.95642164110234 73.17993224808238) bank79945 +79946 POINT(41.27902430025939 74.7722852070377) bank79946 +79947 POINT(41.12981533766591 74.48991504859185) bank79947 +79948 POINT(41.626628958008176 73.76260835898921) bank79948 +79949 POINT(41.20557466395479 73.36008209790799) bank79949 +79950 POINT(40.165297867743625 73.68557571075935) bank79950 +79951 POINT(41.247606832091606 74.07556753483836) bank79951 +79952 POINT(41.33293008425458 74.5076812136119) bank79952 +79953 POINT(41.265695157970555 74.56535521274746) bank79953 +79954 POINT(40.47996340946351 73.23879583294702) bank79954 +79955 POINT(39.754556498224076 73.84829979934565) bank79955 +79956 POINT(40.94537825109025 74.62250436100497) bank79956 +79957 POINT(40.40106879575395 74.74868248757139) bank79957 +79958 POINT(40.1894189653638 73.16114836717688) bank79958 +79959 POINT(39.85408864153568 73.8868126579796) bank79959 +79960 POINT(41.1281744811301 73.3738756133693) bank79960 +79961 POINT(40.435602619082545 73.74376027130283) bank79961 +79962 POINT(40.03704657306651 74.30550701319383) bank79962 +79963 POINT(41.08349832583727 74.32778157303684) bank79963 +79964 POINT(40.555176088351566 73.90653466758758) bank79964 +79965 POINT(40.667814564351254 73.23023274804788) bank79965 +79966 POINT(41.023003978932984 73.27285346218878) bank79966 +79967 POINT(41.405363502232234 74.84945230844833) bank79967 +79968 POINT(40.52305305873895 73.06280067921313) bank79968 +79969 POINT(39.89940628389189 73.45839560248398) bank79969 +79970 POINT(40.74669530015082 73.58056560367972) bank79970 +79971 POINT(39.79777309977084 74.76459010191485) bank79971 +79972 POINT(41.061656455096106 73.67810505194136) bank79972 +79973 POINT(39.81875230694618 74.11041881415889) bank79973 +79974 POINT(41.15280331375747 74.98087554539545) bank79974 +79975 POINT(40.36879527442118 74.57635056482052) bank79975 +79976 POINT(41.49891595449124 73.58938781499062) bank79976 +79977 POINT(41.43864541721496 73.40371012817268) bank79977 +79978 POINT(40.34882781785125 73.85183401564817) bank79978 +79979 POINT(41.368984436414486 74.32946589580078) bank79979 +79980 POINT(39.71703466755758 73.24484812929134) bank79980 +79981 POINT(39.814410349222875 74.87952803337286) bank79981 +79982 POINT(41.37992652438732 74.0796579297519) bank79982 +79983 POINT(40.20757569293232 73.55456734779929) bank79983 +79984 POINT(40.40748015227607 74.49654891379798) bank79984 +79985 POINT(40.87619005079871 73.57436877349376) bank79985 +79986 POINT(41.1747119708205 74.85197008016846) bank79986 +79987 POINT(41.279383157270836 74.22761397710265) bank79987 +79988 POINT(40.117668476293616 73.04270384403678) bank79988 +79989 POINT(40.41654895069121 74.32372257797888) bank79989 +79990 POINT(40.75122754711912 73.47321531908986) bank79990 +79991 POINT(40.935528301415935 74.58154223659653) bank79991 +79992 POINT(40.14040213265182 73.53918562484495) bank79992 +79993 POINT(40.20828397359909 74.40932053259338) bank79993 +79994 POINT(40.32361701374458 74.43484407968619) bank79994 +79995 POINT(39.749587822318084 73.79011998467314) bank79995 +79996 POINT(40.15401879446633 73.02705773731392) bank79996 +79997 POINT(40.76923031073092 74.65452333030531) bank79997 +79998 POINT(39.940656288168114 74.50992558121092) bank79998 +79999 POINT(41.0670312229842 74.47452401465186) bank79999 +80000 POINT(41.665928805239396 74.07254696913198) bank80000 +80001 POINT(40.06674746581432 73.635323202954) bank80001 +80002 POINT(40.19783605446477 73.71777725043312) bank80002 +80003 POINT(40.72432612662513 73.92796793975282) bank80003 +80004 POINT(41.43223353097963 74.67483328965955) bank80004 +80005 POINT(40.42266114382417 73.60058767022628) bank80005 +80006 POINT(41.42761140817747 73.77571959223631) bank80006 +80007 POINT(40.91723879037988 74.07548243229333) bank80007 +80008 POINT(40.26037744101959 74.3372049114585) bank80008 +80009 POINT(39.92492040958 74.4280834107312) bank80009 +80010 POINT(40.26209902772924 74.57936473978056) bank80010 +80011 POINT(40.44833356735786 73.08291923002244) bank80011 +80012 POINT(39.79893939187889 73.4398817861764) bank80012 +80013 POINT(41.22370061148602 74.4569091748057) bank80013 +80014 POINT(41.45450904996631 73.07976375390017) bank80014 +80015 POINT(41.068318346431525 74.36762911176254) bank80015 +80016 POINT(40.32098837935536 74.7251729083717) bank80016 +80017 POINT(41.56225455064873 74.93731515056854) bank80017 +80018 POINT(40.564791464293656 74.96042805407077) bank80018 +80019 POINT(40.45656948124294 73.75743837477958) bank80019 +80020 POINT(40.418339336563406 74.75350956064928) bank80020 +80021 POINT(40.25269594127491 74.59024385215965) bank80021 +80022 POINT(41.067102419896415 73.22294273231597) bank80022 +80023 POINT(41.43451520795243 73.79127563076297) bank80023 +80024 POINT(40.139066622830626 74.31226132365641) bank80024 +80025 POINT(39.7860082020829 73.94198647754241) bank80025 +80026 POINT(40.96678360252509 73.90771385172106) bank80026 +80027 POINT(41.470133829531584 74.24847126573475) bank80027 +80028 POINT(41.62774317344975 73.64336601238652) bank80028 +80029 POINT(39.77850682293905 73.09092120814559) bank80029 +80030 POINT(41.488928242685354 73.33854232728264) bank80030 +80031 POINT(39.89836901103921 73.37392904607711) bank80031 +80032 POINT(39.87998874803498 73.22149055335231) bank80032 +80033 POINT(40.09625541658259 74.58482127840287) bank80033 +80034 POINT(39.86229505387023 74.7626704504753) bank80034 +80035 POINT(41.50833338059209 73.53373006003531) bank80035 +80036 POINT(41.56206139522751 73.97474550405909) bank80036 +80037 POINT(41.06462683295341 74.70925933254763) bank80037 +80038 POINT(40.529918689294064 73.62240852649613) bank80038 +80039 POINT(41.20663092473966 73.67693886142153) bank80039 +80040 POINT(41.181689150060265 73.76606931656809) bank80040 +80041 POINT(40.83838199960122 74.48264567170693) bank80041 +80042 POINT(41.621123881774075 73.06353051487996) bank80042 +80043 POINT(40.29610168108133 74.6084827009786) bank80043 +80044 POINT(41.59423780959952 73.8020535518946) bank80044 +80045 POINT(40.17511018129521 74.45514136449982) bank80045 +80046 POINT(41.05186813137045 73.06821055716532) bank80046 +80047 POINT(41.43147425953074 73.82740098521124) bank80047 +80048 POINT(41.681098039494465 74.06472290448035) bank80048 +80049 POINT(41.31647111908046 74.65201090225189) bank80049 +80050 POINT(39.91169526745274 73.31167720786327) bank80050 +80051 POINT(40.04430747174598 74.13854265254564) bank80051 +80052 POINT(39.722359942330996 74.17992457864248) bank80052 +80053 POINT(40.50532580355306 73.06687118069942) bank80053 +80054 POINT(40.64906719902663 73.54090708831589) bank80054 +80055 POINT(41.673561679995615 74.11925341846566) bank80055 +80056 POINT(41.62878251029042 73.87446048366692) bank80056 +80057 POINT(39.77764152868632 73.45185743352309) bank80057 +80058 POINT(41.046414675509546 74.59564810109482) bank80058 +80059 POINT(40.610022394859406 74.23573291796004) bank80059 +80060 POINT(39.92364570000012 74.53684769480813) bank80060 +80061 POINT(40.17722503088077 73.30496563707928) bank80061 +80062 POINT(41.637817599038044 74.9474025762838) bank80062 +80063 POINT(40.80790970722004 74.228285496105) bank80063 +80064 POINT(41.68964018385741 73.23825090690133) bank80064 +80065 POINT(40.24302289915018 74.05274512068395) bank80065 +80066 POINT(41.71102650293764 74.9998062974483) bank80066 +80067 POINT(41.222693943677484 74.73338348212786) bank80067 +80068 POINT(41.545963535330245 74.76973806591198) bank80068 +80069 POINT(39.725753635888125 74.42985438745694) bank80069 +80070 POINT(40.58842730728013 73.16469648030433) bank80070 +80071 POINT(41.46282228595801 74.6770776416937) bank80071 +80072 POINT(40.51979364470297 73.24910532582366) bank80072 +80073 POINT(40.44027425399707 73.63388238264686) bank80073 +80074 POINT(40.89228339366116 74.0233927692235) bank80074 +80075 POINT(39.82277790194812 74.55570954411927) bank80075 +80076 POINT(41.08898415131528 74.88640914770035) bank80076 +80077 POINT(41.51485040361468 73.2050820146478) bank80077 +80078 POINT(39.91978283876837 74.95009277733092) bank80078 +80079 POINT(39.96832240379792 73.3023160826392) bank80079 +80080 POINT(39.96760996097992 74.89497382932932) bank80080 +80081 POINT(41.0702250905965 74.35171715587823) bank80081 +80082 POINT(40.21659586820359 74.20864700605392) bank80082 +80083 POINT(41.7007635985375 73.88350788792327) bank80083 +80084 POINT(41.46719718659689 73.06738825546515) bank80084 +80085 POINT(41.31722056793744 73.01919063452995) bank80085 +80086 POINT(41.03790306665751 73.82156146724343) bank80086 +80087 POINT(39.77523655319995 74.07589135662755) bank80087 +80088 POINT(40.201469579989144 73.58609622980848) bank80088 +80089 POINT(40.60446398511509 73.63152358625433) bank80089 +80090 POINT(41.32072331111784 73.92570448108549) bank80090 +80091 POINT(41.55341867366868 74.61818138250291) bank80091 +80092 POINT(40.646616987437525 74.43441228094053) bank80092 +80093 POINT(39.97489650461499 73.98097770260519) bank80093 +80094 POINT(40.96302348935776 73.2139617669681) bank80094 +80095 POINT(40.61814208889532 74.96724801053111) bank80095 +80096 POINT(40.93541459247019 73.51300706040296) bank80096 +80097 POINT(40.82330423277891 74.0672621434927) bank80097 +80098 POINT(40.886964170395295 74.5081959760118) bank80098 +80099 POINT(40.936247662353615 73.06568834593413) bank80099 +80100 POINT(40.46552859509079 73.59495365297393) bank80100 +80101 POINT(40.45343815960664 74.66255221484218) bank80101 +80102 POINT(40.50352350062155 73.3868172134399) bank80102 +80103 POINT(41.053778018738555 74.28560804207713) bank80103 +80104 POINT(41.02581445981785 73.19901454652229) bank80104 +80105 POINT(41.63061510923383 73.20414566976311) bank80105 +80106 POINT(40.62346991823796 74.10577633669344) bank80106 +80107 POINT(40.23407291492333 74.65935385952092) bank80107 +80108 POINT(40.33703768084728 73.19141346400714) bank80108 +80109 POINT(40.91280021066205 74.80851675140512) bank80109 +80110 POINT(40.19577131834493 74.68264093904659) bank80110 +80111 POINT(41.6196158194607 74.25367768344888) bank80111 +80112 POINT(39.92042506028627 73.96925603431478) bank80112 +80113 POINT(41.60682964788706 74.00453616356823) bank80113 +80114 POINT(40.447061192167666 73.04192270818935) bank80114 +80115 POINT(41.02550367748046 74.0807732350327) bank80115 +80116 POINT(40.6897604128771 74.69440541311911) bank80116 +80117 POINT(41.52762686276841 73.91686761531408) bank80117 +80118 POINT(40.096492235336406 73.2670414718815) bank80118 +80119 POINT(40.71619603103601 74.44318358438716) bank80119 +80120 POINT(39.98675268997027 73.29288921903994) bank80120 +80121 POINT(40.934622655480936 73.90038067389142) bank80121 +80122 POINT(40.235932264004944 74.74891581601513) bank80122 +80123 POINT(41.12696009289015 74.55180372249914) bank80123 +80124 POINT(40.71590082864277 73.51580146745339) bank80124 +80125 POINT(40.7311898648633 73.26065487276446) bank80125 +80126 POINT(40.63217327021922 73.4810124190194) bank80126 +80127 POINT(41.13211304905589 74.7523181508129) bank80127 +80128 POINT(41.65736902088523 74.2218543714706) bank80128 +80129 POINT(39.88161033953188 73.786420653039) bank80129 +80130 POINT(40.294814311419536 73.85317836391584) bank80130 +80131 POINT(40.26425621626748 74.25533812185078) bank80131 +80132 POINT(39.83104936961684 73.2952961159241) bank80132 +80133 POINT(40.52117802453358 74.46708747303134) bank80133 +80134 POINT(40.31281721525458 73.78841966071649) bank80134 +80135 POINT(40.15795142615258 74.84180507832231) bank80135 +80136 POINT(41.3907990390594 74.31583388224435) bank80136 +80137 POINT(40.39145515945498 73.44406499666916) bank80137 +80138 POINT(41.159729739256306 74.27161870333214) bank80138 +80139 POINT(41.47616187266034 73.08524200251175) bank80139 +80140 POINT(40.996501311694296 73.46495405000302) bank80140 +80141 POINT(39.81670982360802 74.17774450188885) bank80141 +80142 POINT(41.2757415708344 73.72680819915549) bank80142 +80143 POINT(41.29288266741327 73.48104264149337) bank80143 +80144 POINT(40.44866218270293 74.86305655968889) bank80144 +80145 POINT(41.16010299116396 73.55938539101092) bank80145 +80146 POINT(41.65621090642821 74.02466609654479) bank80146 +80147 POINT(40.92958949193628 73.39386713181842) bank80147 +80148 POINT(41.435471523094144 73.63891754403164) bank80148 +80149 POINT(41.54656683624183 74.41512078562077) bank80149 +80150 POINT(41.58456608672564 73.63606330960576) bank80150 +80151 POINT(39.97125487022285 73.28302655071964) bank80151 +80152 POINT(41.07061296796322 73.65748183960781) bank80152 +80153 POINT(41.18601058533445 73.77875413518669) bank80153 +80154 POINT(40.06669957423968 74.18199124387625) bank80154 +80155 POINT(40.69978790624083 74.90132269887411) bank80155 +80156 POINT(41.179605354099664 74.43857967624776) bank80156 +80157 POINT(41.34573494511106 73.74100186312866) bank80157 +80158 POINT(40.850568263637605 73.10007112003565) bank80158 +80159 POINT(40.5888596247714 74.98804505123864) bank80159 +80160 POINT(40.136926357703146 73.7228346780005) bank80160 +80161 POINT(39.88179395558328 74.64491267888029) bank80161 +80162 POINT(41.59845671495317 74.39109022434833) bank80162 +80163 POINT(40.12200666773402 73.72011277311472) bank80163 +80164 POINT(41.29748328914624 73.52488434829186) bank80164 +80165 POINT(41.3198250560996 74.19887583268313) bank80165 +80166 POINT(40.20225362765994 73.61482158731405) bank80166 +80167 POINT(41.22040357464072 74.66052009976062) bank80167 +80168 POINT(41.11461710385243 73.84200004013462) bank80168 +80169 POINT(40.724836205201555 74.70855230844397) bank80169 +80170 POINT(41.45508294213797 73.50777373219296) bank80170 +80171 POINT(40.94536885326515 73.1882031132942) bank80171 +80172 POINT(40.56104678418685 73.59007989001073) bank80172 +80173 POINT(39.82378481023845 74.67172616227549) bank80173 +80174 POINT(41.32763962120747 74.46791642639957) bank80174 +80175 POINT(41.4758083881534 74.98120672831497) bank80175 +80176 POINT(40.962214464689815 73.82913995937109) bank80176 +80177 POINT(41.06818511146609 73.21907469064347) bank80177 +80178 POINT(40.097799296408525 73.33420948354076) bank80178 +80179 POINT(40.163153801274326 74.28839409163756) bank80179 +80180 POINT(41.52656946169496 73.30052921999085) bank80180 +80181 POINT(41.21028518718021 74.91489535444697) bank80181 +80182 POINT(39.816417184140654 73.83722392559808) bank80182 +80183 POINT(40.331589987429155 74.29342232092024) bank80183 +80184 POINT(39.92325047827046 73.2367662037858) bank80184 +80185 POINT(41.32855849685726 74.17126575972512) bank80185 +80186 POINT(39.777030587896846 74.51672187979021) bank80186 +80187 POINT(40.44482862178158 74.63117513416415) bank80187 +80188 POINT(40.340970761768546 73.768137763011) bank80188 +80189 POINT(40.1646903727271 74.06806296885966) bank80189 +80190 POINT(41.47084430484636 74.30768634324596) bank80190 +80191 POINT(41.52472285582667 74.68582203487934) bank80191 +80192 POINT(40.13883134768648 74.48963144299448) bank80192 +80193 POINT(40.04680311047323 73.35022412921012) bank80193 +80194 POINT(39.89815284784495 74.57087705219284) bank80194 +80195 POINT(39.8882704859224 73.78298349193695) bank80195 +80196 POINT(39.79979054563974 74.91974034654876) bank80196 +80197 POINT(41.51334151637602 73.87249318120688) bank80197 +80198 POINT(41.31089549770565 74.0329599433423) bank80198 +80199 POINT(40.04646411974169 73.02721037619361) bank80199 +80200 POINT(41.359444643373585 74.27557937491989) bank80200 +80201 POINT(41.028997971184175 73.61481884542142) bank80201 +80202 POINT(40.775550366915475 73.92560737853005) bank80202 +80203 POINT(40.50873588545518 73.59560690387531) bank80203 +80204 POINT(40.6981642302238 74.56034933298389) bank80204 +80205 POINT(40.56640222266002 74.0939949602242) bank80205 +80206 POINT(40.864025563836876 74.88417992533094) bank80206 +80207 POINT(39.972336113652155 74.1568946653017) bank80207 +80208 POINT(40.276901142006814 74.39167586317069) bank80208 +80209 POINT(41.538477997919 73.5194926104962) bank80209 +80210 POINT(41.36903517168969 74.50985514328151) bank80210 +80211 POINT(41.60371148497666 74.20864649748941) bank80211 +80212 POINT(41.668973537370725 74.79407162664064) bank80212 +80213 POINT(39.9916519251866 73.56081150111605) bank80213 +80214 POINT(40.67079703748283 73.1862323199723) bank80214 +80215 POINT(40.29373822000013 74.28454457350082) bank80215 +80216 POINT(40.657754222913915 73.75984814671442) bank80216 +80217 POINT(39.80369017490953 73.53884548403909) bank80217 +80218 POINT(39.87249933496688 73.55857715872627) bank80218 +80219 POINT(41.1632573727699 73.8375300085745) bank80219 +80220 POINT(41.540753694592894 74.37232474429074) bank80220 +80221 POINT(41.621081833630925 74.9790047894062) bank80221 +80222 POINT(41.651175689525914 73.91566664693073) bank80222 +80223 POINT(40.11515071975399 74.56407064277187) bank80223 +80224 POINT(39.82569272614232 73.90361956059473) bank80224 +80225 POINT(40.924057483631195 74.8801666417445) bank80225 +80226 POINT(41.50927725063188 74.62473980932054) bank80226 +80227 POINT(41.47242243028155 74.51313386125535) bank80227 +80228 POINT(39.71789564880908 74.27663037122736) bank80228 +80229 POINT(41.196365620359174 74.24918895090927) bank80229 +80230 POINT(40.42940188269376 73.51122529124369) bank80230 +80231 POINT(40.573119242893284 74.62392424385415) bank80231 +80232 POINT(40.54244436862963 74.88389718260332) bank80232 +80233 POINT(40.481833985458096 74.46882233263624) bank80233 +80234 POINT(39.71386232602847 73.92773885808404) bank80234 +80235 POINT(40.83150064378579 74.16933475260394) bank80235 +80236 POINT(41.5079558349804 74.4276093209102) bank80236 +80237 POINT(40.73495820733795 73.34117766645315) bank80237 +80238 POINT(40.89048959767306 74.5253391249922) bank80238 +80239 POINT(41.416435771029214 74.00852017333116) bank80239 +80240 POINT(39.71374279962289 73.61423954097187) bank80240 +80241 POINT(40.770274751113035 74.43125347363863) bank80241 +80242 POINT(41.66413915258499 74.92909006265614) bank80242 +80243 POINT(41.61667706020264 73.02175201668915) bank80243 +80244 POINT(41.539851245148505 74.47273434170802) bank80244 +80245 POINT(41.54572625114409 73.82292171497934) bank80245 +80246 POINT(40.06178572420997 74.96250295625114) bank80246 +80247 POINT(39.83589860051635 73.87349881330746) bank80247 +80248 POINT(41.45882495514659 74.0531496036568) bank80248 +80249 POINT(40.597016765954145 73.34922520746625) bank80249 +80250 POINT(40.39264304776587 74.38139162252887) bank80250 +80251 POINT(41.31271188798523 73.77272540268036) bank80251 +80252 POINT(41.414538816039205 74.54926469630054) bank80252 +80253 POINT(41.30924358536604 74.55422276749634) bank80253 +80254 POINT(41.57996707916127 74.64155943723124) bank80254 +80255 POINT(40.84564907009367 74.36837288205969) bank80255 +80256 POINT(40.54830246918779 73.30283441751767) bank80256 +80257 POINT(41.51407924842154 73.51888566365515) bank80257 +80258 POINT(40.83602910709769 73.95893253852843) bank80258 +80259 POINT(41.3034180386063 74.05878643047403) bank80259 +80260 POINT(40.404395734827816 75.00247099602753) bank80260 +80261 POINT(40.857094484559475 73.75633532831345) bank80261 +80262 POINT(41.02464478465699 73.03334737654853) bank80262 +80263 POINT(41.60803081676542 73.32103963892166) bank80263 +80264 POINT(40.2782270007599 73.02409951408451) bank80264 +80265 POINT(40.551085314331374 73.87596145444773) bank80265 +80266 POINT(40.09987950961045 73.5995752573317) bank80266 +80267 POINT(40.73788164191906 74.96321223107633) bank80267 +80268 POINT(41.70581197566688 74.4657367349102) bank80268 +80269 POINT(39.72336261753733 73.44083065530187) bank80269 +80270 POINT(41.16404796127485 73.9606315443051) bank80270 +80271 POINT(40.13352531511316 75.0022239212888) bank80271 +80272 POINT(40.3202052390959 73.58681857877863) bank80272 +80273 POINT(41.56888135503715 73.4316573436943) bank80273 +80274 POINT(40.13781733520673 73.28795259189823) bank80274 +80275 POINT(41.27676139804182 74.24398462595857) bank80275 +80276 POINT(39.991643269831016 74.69953156140346) bank80276 +80277 POINT(41.272141880599044 73.7673666862833) bank80277 +80278 POINT(40.86572851648236 74.5337266221334) bank80278 +80279 POINT(40.01432466640847 73.06761425814061) bank80279 +80280 POINT(40.32448278584349 73.23000660545635) bank80280 +80281 POINT(41.07511286631463 73.69424472335942) bank80281 +80282 POINT(41.092345609316354 74.41072155406458) bank80282 +80283 POINT(41.59963952835417 73.84750757392607) bank80283 +80284 POINT(40.96725582839457 74.92299060581733) bank80284 +80285 POINT(41.166907540536535 73.81638749256491) bank80285 +80286 POINT(39.744067217070025 73.15882414775042) bank80286 +80287 POINT(40.856279660240716 74.27625022272883) bank80287 +80288 POINT(40.317626819983325 74.8061689109374) bank80288 +80289 POINT(41.70013707321009 73.28406629259841) bank80289 +80290 POINT(40.779416047566855 74.01595617153507) bank80290 +80291 POINT(40.84390156544058 73.73155077088231) bank80291 +80292 POINT(40.40039770890805 73.42165999243517) bank80292 +80293 POINT(40.84541415212339 73.57373364971588) bank80293 +80294 POINT(41.38072727807312 73.06468874153454) bank80294 +80295 POINT(41.705123438403724 73.93034722954535) bank80295 +80296 POINT(41.07112928857855 73.30393314304995) bank80296 +80297 POINT(40.69102463615855 74.17194206091177) bank80297 +80298 POINT(39.927217157378 74.30588165348742) bank80298 +80299 POINT(39.82780953122696 73.6174359045429) bank80299 +80300 POINT(41.38716190513515 74.54610416165052) bank80300 +80301 POINT(40.57419875954375 73.01912912913738) bank80301 +80302 POINT(40.57725021210629 73.79414876623245) bank80302 +80303 POINT(40.71024755837562 73.71836763230479) bank80303 +80304 POINT(41.5913517210706 74.91990327377249) bank80304 +80305 POINT(41.206296077063556 74.20422869018864) bank80305 +80306 POINT(40.18535308025933 73.01048819076256) bank80306 +80307 POINT(41.49602335784639 73.24815904868967) bank80307 +80308 POINT(39.93744801120608 74.62063619235013) bank80308 +80309 POINT(40.61566388207637 73.3576248798661) bank80309 +80310 POINT(39.748685932103506 74.25599783328578) bank80310 +80311 POINT(41.653877486556105 74.98713801661074) bank80311 +80312 POINT(41.25240636520479 73.59500048435203) bank80312 +80313 POINT(40.12259525920111 73.48163232912128) bank80313 +80314 POINT(39.850519711195915 74.33201238643761) bank80314 +80315 POINT(41.34115664367341 73.11561430357052) bank80315 +80316 POINT(40.19354198195611 74.72827150230432) bank80316 +80317 POINT(41.26975696392134 74.41028668495547) bank80317 +80318 POINT(40.17911189441143 73.89072157866103) bank80318 +80319 POINT(40.3157016056911 73.37844392704923) bank80319 +80320 POINT(39.83964438089489 74.90731854180997) bank80320 +80321 POINT(40.85659823511291 74.82621363378145) bank80321 +80322 POINT(40.37685986873494 74.74276270135528) bank80322 +80323 POINT(40.56668709427796 73.38766424142138) bank80323 +80324 POINT(40.72301927560282 74.81205748309227) bank80324 +80325 POINT(41.467202783339076 74.62326094452479) bank80325 +80326 POINT(40.23101881110272 73.56662397427111) bank80326 +80327 POINT(40.247886377749836 74.74193401460067) bank80327 +80328 POINT(40.014439746195556 74.82871554445914) bank80328 +80329 POINT(39.761610001551105 74.36152677697206) bank80329 +80330 POINT(41.13215259516995 73.49756070714416) bank80330 +80331 POINT(40.41740824326149 73.17056062293236) bank80331 +80332 POINT(40.122767814132146 74.34034356895073) bank80332 +80333 POINT(41.58847430139771 73.57947693386316) bank80333 +80334 POINT(39.875801605853866 73.78175489917788) bank80334 +80335 POINT(40.70084401903314 73.86763254845702) bank80335 +80336 POINT(40.52674319170093 74.44806128050334) bank80336 +80337 POINT(41.163432736905854 74.66971427877563) bank80337 +80338 POINT(41.09658192698864 73.83479137293938) bank80338 +80339 POINT(40.130505417957224 73.27583343698996) bank80339 +80340 POINT(40.62892952400587 73.2085215469159) bank80340 +80341 POINT(40.868476520743855 73.22575603575169) bank80341 +80342 POINT(40.02835553586595 74.84312188719417) bank80342 +80343 POINT(41.1892409526813 73.13239227532885) bank80343 +80344 POINT(41.17441241149032 74.09213878994758) bank80344 +80345 POINT(40.599759937019996 73.98338494582157) bank80345 +80346 POINT(40.47485333022975 74.43143816994092) bank80346 +80347 POINT(39.83287917583453 73.0470050146209) bank80347 +80348 POINT(39.812337556489325 73.64173001435407) bank80348 +80349 POINT(39.98556989546073 73.86053806966865) bank80349 +80350 POINT(41.5254624797104 73.062809504112) bank80350 +80351 POINT(41.270193074630065 73.85342329164571) bank80351 +80352 POINT(41.00796147595783 73.45085576259581) bank80352 +80353 POINT(40.888633650808096 74.48735291381938) bank80353 +80354 POINT(41.2797390299122 73.07057530095153) bank80354 +80355 POINT(40.26984739351867 73.37778485386406) bank80355 +80356 POINT(41.603944924780734 74.14763367738888) bank80356 +80357 POINT(39.87648670532068 74.13018077988099) bank80357 +80358 POINT(41.097651863427046 73.65742951659924) bank80358 +80359 POINT(39.87364605989152 74.93389196014992) bank80359 +80360 POINT(40.91115313511623 73.79692351559288) bank80360 +80361 POINT(40.18085193188968 73.15979490178742) bank80361 +80362 POINT(41.56751842716793 74.09491431130648) bank80362 +80363 POINT(40.07938021760598 74.70203563385405) bank80363 +80364 POINT(39.84376306783055 74.8822265913072) bank80364 +80365 POINT(40.808481474804154 73.9984164203322) bank80365 +80366 POINT(39.90482344385492 73.33556582469005) bank80366 +80367 POINT(40.21416784970869 74.61749573606896) bank80367 +80368 POINT(41.099691353235094 74.88995801572004) bank80368 +80369 POINT(41.488335805983255 73.98392425171389) bank80369 +80370 POINT(39.81274648514362 73.01915783133083) bank80370 +80371 POINT(39.92734638809494 74.1067089841674) bank80371 +80372 POINT(41.4993059297478 73.74742970599883) bank80372 +80373 POINT(41.10534764405393 74.47258061586909) bank80373 +80374 POINT(41.633832467226924 73.6425508687066) bank80374 +80375 POINT(41.371284182001105 73.2861124426583) bank80375 +80376 POINT(40.75614746757503 74.84924448779573) bank80376 +80377 POINT(41.277929324644006 74.11856178191707) bank80377 +80378 POINT(41.565215072096244 73.39934816615673) bank80378 +80379 POINT(41.39844286764178 74.69304928454626) bank80379 +80380 POINT(40.19825416937394 73.5136908990845) bank80380 +80381 POINT(39.88884680125752 73.3371188766433) bank80381 +80382 POINT(40.64178676181902 73.46564486876538) bank80382 +80383 POINT(40.589787465577075 73.39278414421213) bank80383 +80384 POINT(40.855728128312194 73.04296393069755) bank80384 +80385 POINT(41.348322054939715 74.65678194049184) bank80385 +80386 POINT(39.97173211506682 74.44928191595622) bank80386 +80387 POINT(40.232119402814114 74.74453930490169) bank80387 +80388 POINT(40.96135627841785 73.6871337038464) bank80388 +80389 POINT(40.576664201037495 73.21444813888131) bank80389 +80390 POINT(40.68338128989137 73.77675510134772) bank80390 +80391 POINT(40.75412053961344 73.13504151740162) bank80391 +80392 POINT(41.71115048264907 73.40268310150307) bank80392 +80393 POINT(41.310548438731935 74.38266010087689) bank80393 +80394 POINT(41.70637102079978 73.3052877421601) bank80394 +80395 POINT(40.60429380623063 74.28860524500571) bank80395 +80396 POINT(39.92819808724454 74.47845817809038) bank80396 +80397 POINT(40.039689346481964 73.60741148416851) bank80397 +80398 POINT(41.1144630520979 74.86929103963244) bank80398 +80399 POINT(41.374679765194585 73.91513956813223) bank80399 +80400 POINT(39.86237533149453 73.29726502940811) bank80400 +80401 POINT(41.382626491061444 73.1766672503736) bank80401 +80402 POINT(39.81360451835106 74.67906746519583) bank80402 +80403 POINT(39.82834897406592 74.595478521567) bank80403 +80404 POINT(39.774880724739226 73.71307065585114) bank80404 +80405 POINT(40.854194137055764 74.7002393657198) bank80405 +80406 POINT(41.5925050647701 73.51297300034255) bank80406 +80407 POINT(40.55905234432123 74.3724044290143) bank80407 +80408 POINT(41.328815375430416 74.42961096309828) bank80408 +80409 POINT(40.044970284192864 73.61694754142108) bank80409 +80410 POINT(40.197839113002544 74.28857501014514) bank80410 +80411 POINT(41.56827604741016 74.95074446504374) bank80411 +80412 POINT(40.5746644549708 73.1897577952285) bank80412 +80413 POINT(40.803629486917806 74.6556177282474) bank80413 +80414 POINT(41.300121860036974 73.08597192003678) bank80414 +80415 POINT(41.26253867014533 74.37099314724601) bank80415 +80416 POINT(40.83448396999875 74.98041277010311) bank80416 +80417 POINT(40.15647964054489 73.569146844374) bank80417 +80418 POINT(40.13544331877906 73.41684581780733) bank80418 +80419 POINT(40.22363977902272 73.24973853075339) bank80419 +80420 POINT(39.78234738401222 74.09640304761014) bank80420 +80421 POINT(41.405262216884616 74.64990223497061) bank80421 +80422 POINT(41.13955506358925 73.10836782686) bank80422 +80423 POINT(41.43856993460687 74.43657521479795) bank80423 +80424 POINT(40.19733376475039 73.13194344633158) bank80424 +80425 POINT(40.507602558610195 73.43155028155783) bank80425 +80426 POINT(40.37875036208391 73.55778355132014) bank80426 +80427 POINT(41.438454518315226 73.48494541919106) bank80427 +80428 POINT(41.523560435935046 74.69128459495164) bank80428 +80429 POINT(40.929388265051905 73.9929320144766) bank80429 +80430 POINT(40.16628273658933 74.40855265496761) bank80430 +80431 POINT(40.84698617712221 73.91764122453495) bank80431 +80432 POINT(41.42147308842105 73.15025908285922) bank80432 +80433 POINT(40.18532299203809 73.32830532243172) bank80433 +80434 POINT(40.89022969141375 74.69861086178454) bank80434 +80435 POINT(40.60375522824716 74.08930976168072) bank80435 +80436 POINT(41.044333361390976 74.37526771329492) bank80436 +80437 POINT(41.16463662227158 74.24074324578329) bank80437 +80438 POINT(41.38633574375874 74.97437690033469) bank80438 +80439 POINT(41.34349779336127 74.26972468962335) bank80439 +80440 POINT(40.94979781753873 74.16805444017454) bank80440 +80441 POINT(41.05167112110832 74.34312790200349) bank80441 +80442 POINT(41.48085064030973 74.03612190821177) bank80442 +80443 POINT(40.56547957795986 73.72652056893509) bank80443 +80444 POINT(40.301568214511086 74.71439023933245) bank80444 +80445 POINT(40.936953336831984 73.22200463188153) bank80445 +80446 POINT(41.21706809060552 73.45518126568116) bank80446 +80447 POINT(41.70587025175666 73.66869038371051) bank80447 +80448 POINT(41.0023941640395 73.82001647738217) bank80448 +80449 POINT(40.59079943489346 74.42117771250888) bank80449 +80450 POINT(41.57738917967194 74.21396636706018) bank80450 +80451 POINT(40.30279798543712 74.57717687764698) bank80451 +80452 POINT(40.946373630567045 74.90809378225315) bank80452 +80453 POINT(41.31494937564958 73.53366102162997) bank80453 +80454 POINT(39.764150556673336 74.0421622242249) bank80454 +80455 POINT(40.56326133870038 74.21903043507555) bank80455 +80456 POINT(40.34213299344241 74.12924793254673) bank80456 +80457 POINT(41.6095415169505 74.33353400922074) bank80457 +80458 POINT(40.36511492675693 73.06780753842081) bank80458 +80459 POINT(40.46470319022622 73.47564566088128) bank80459 +80460 POINT(41.06970438276922 74.34019264749223) bank80460 +80461 POINT(39.87400062618081 73.4739083739434) bank80461 +80462 POINT(40.91738543536032 74.96599588850425) bank80462 +80463 POINT(41.66588177875272 73.08286627260223) bank80463 +80464 POINT(41.43152176726538 73.33897957562638) bank80464 +80465 POINT(41.63857638705299 73.84317520231758) bank80465 +80466 POINT(40.90990638350638 74.06048260437373) bank80466 +80467 POINT(41.245759470355125 73.27606776207068) bank80467 +80468 POINT(40.30715232731114 74.48604761058051) bank80468 +80469 POINT(39.96704352872262 74.00448088930361) bank80469 +80470 POINT(41.356817545002926 73.91759422209854) bank80470 +80471 POINT(41.08732915314827 73.71901525215522) bank80471 +80472 POINT(39.75223115920427 73.95920615799034) bank80472 +80473 POINT(40.3376369577521 73.50260683121107) bank80473 +80474 POINT(41.492294046176816 73.21160676208498) bank80474 +80475 POINT(39.74602095186304 74.34788379095954) bank80475 +80476 POINT(41.45802994444408 74.99646666627726) bank80476 +80477 POINT(41.053574016000525 74.26731027718964) bank80477 +80478 POINT(41.52187055788833 74.81903438978995) bank80478 +80479 POINT(40.14689101138764 74.26920377665846) bank80479 +80480 POINT(41.10193896086522 73.71272814082702) bank80480 +80481 POINT(41.43373583474509 73.06910559314619) bank80481 +80482 POINT(40.81528588998489 74.174354335538) bank80482 +80483 POINT(40.57937248335188 73.85548262177612) bank80483 +80484 POINT(40.57078326756766 73.33387161400546) bank80484 +80485 POINT(40.65468203110976 74.63210337353996) bank80485 +80486 POINT(41.50229077620084 73.21097670332) bank80486 +80487 POINT(41.64402998873636 73.23226384329605) bank80487 +80488 POINT(39.799555240614794 74.04596225641433) bank80488 +80489 POINT(41.22129942143864 73.91398171023096) bank80489 +80490 POINT(41.6231929779235 73.8345626063648) bank80490 +80491 POINT(39.73768746724765 73.9316432298743) bank80491 +80492 POINT(41.161792220437825 74.74838650218193) bank80492 +80493 POINT(40.81720186567597 74.28105902254507) bank80493 +80494 POINT(41.0262479772376 74.2059197747523) bank80494 +80495 POINT(41.249183312259206 73.62800255390275) bank80495 +80496 POINT(39.76168377296884 74.18104227445608) bank80496 +80497 POINT(39.729864943857955 74.40866507290274) bank80497 +80498 POINT(39.73121687089085 73.17273193072732) bank80498 +80499 POINT(41.691817158632915 74.44290938603753) bank80499 +80500 POINT(41.289956184273166 73.66150818703046) bank80500 +80501 POINT(41.587146666576714 74.71889263045418) bank80501 +80502 POINT(41.679848469611244 74.26143846784589) bank80502 +80503 POINT(39.72547734579807 73.46823682321147) bank80503 +80504 POINT(40.230118671945874 73.25538670177568) bank80504 +80505 POINT(41.30470764669823 74.20915724597027) bank80505 +80506 POINT(40.98856021414459 73.47012820925144) bank80506 +80507 POINT(41.39672042011113 73.0250664347408) bank80507 +80508 POINT(40.279405114674965 73.36988873777479) bank80508 +80509 POINT(41.24377442469953 74.96193487606921) bank80509 +80510 POINT(41.50429060398422 73.11567707841212) bank80510 +80511 POINT(41.3286807197042 73.40772740987185) bank80511 +80512 POINT(40.14276422761085 73.73018255626438) bank80512 +80513 POINT(40.35724179314223 74.42237384918212) bank80513 +80514 POINT(40.31378480926961 74.46094082693331) bank80514 +80515 POINT(40.62000806840336 74.12301375779134) bank80515 +80516 POINT(39.71789785959412 74.98624770164592) bank80516 +80517 POINT(40.18623400641747 74.03198400475068) bank80517 +80518 POINT(41.00398253799241 73.64105145692145) bank80518 +80519 POINT(39.870278737121254 73.88868538200542) bank80519 +80520 POINT(39.87443674971139 73.31528968877828) bank80520 +80521 POINT(41.15024883098708 74.96171074816021) bank80521 +80522 POINT(41.26819521492962 74.41265940040014) bank80522 +80523 POINT(41.52995853788374 74.19392023999465) bank80523 +80524 POINT(39.96727593171956 73.12182277840684) bank80524 +80525 POINT(40.83517405685359 73.28608143110509) bank80525 +80526 POINT(40.73470111922188 73.82252708414703) bank80526 +80527 POINT(40.38854449944835 73.53279253391729) bank80527 +80528 POINT(39.839501794556654 73.94979262929722) bank80528 +80529 POINT(40.874154230231746 74.75925171017175) bank80529 +80530 POINT(41.01163293214457 74.82617337828171) bank80530 +80531 POINT(41.027382480198014 73.45988436165511) bank80531 +80532 POINT(41.16591104995812 73.33928420098687) bank80532 +80533 POINT(40.019944938162084 74.7736813246543) bank80533 +80534 POINT(41.289168227732766 73.14695895409243) bank80534 +80535 POINT(40.74723470274285 73.62621239291502) bank80535 +80536 POINT(40.86771819132791 73.61064905510213) bank80536 +80537 POINT(41.02709135784515 74.677889040234) bank80537 +80538 POINT(41.62058949922652 74.5713599079926) bank80538 +80539 POINT(40.4636033907062 74.26234828881935) bank80539 +80540 POINT(40.78921510648508 74.74388265408356) bank80540 +80541 POINT(40.973084554646384 74.24103896359128) bank80541 +80542 POINT(40.96626080734395 74.02509056909501) bank80542 +80543 POINT(40.566204704344756 73.62799632148534) bank80543 +80544 POINT(40.20917628579279 73.12745564028448) bank80544 +80545 POINT(41.08449408478412 73.47352990180933) bank80545 +80546 POINT(41.057443308287795 74.22162028837496) bank80546 +80547 POINT(41.08703650669407 74.7992558491145) bank80547 +80548 POINT(40.60154905258569 74.6122597311659) bank80548 +80549 POINT(41.08749468402384 74.83450963492857) bank80549 +80550 POINT(41.14510458506366 73.95505889547502) bank80550 +80551 POINT(41.315406194599895 73.5733891251265) bank80551 +80552 POINT(39.73938328939577 73.43370515931923) bank80552 +80553 POINT(39.75558450650424 73.31852539045649) bank80553 +80554 POINT(40.35356882099604 74.89808133707152) bank80554 +80555 POINT(40.53429405785002 73.21304361971346) bank80555 +80556 POINT(41.32108025171014 73.88898878550718) bank80556 +80557 POINT(41.46128324199847 74.83752332714074) bank80557 +80558 POINT(40.34411925700776 73.08390599635929) bank80558 +80559 POINT(39.8208359634032 73.56001874559972) bank80559 +80560 POINT(41.146755922718235 73.45931579248534) bank80560 +80561 POINT(41.48446230981263 73.21140833583175) bank80561 +80562 POINT(40.929921708355934 73.55705315712035) bank80562 +80563 POINT(40.06579415431691 74.17073039334885) bank80563 +80564 POINT(39.85955871599502 74.68630426244108) bank80564 +80565 POINT(41.43544972576835 73.42914161184787) bank80565 +80566 POINT(40.978061545456754 73.99102705059975) bank80566 +80567 POINT(40.263168028419216 74.58374970306214) bank80567 +80568 POINT(39.84036877193652 73.56130974541648) bank80568 +80569 POINT(39.860909270110746 73.41842534532866) bank80569 +80570 POINT(41.51679764372071 74.8788087144881) bank80570 +80571 POINT(41.58859553433194 74.66089700859536) bank80571 +80572 POINT(40.44262063922516 74.46053514156743) bank80572 +80573 POINT(40.15156558644011 73.65861956635953) bank80573 +80574 POINT(40.02662373482295 74.56552394739371) bank80574 +80575 POINT(41.178460967976584 74.44678744473799) bank80575 +80576 POINT(40.478006050415054 73.05161945851971) bank80576 +80577 POINT(41.36292450945709 74.43177325045004) bank80577 +80578 POINT(41.12394673754094 73.33048039936959) bank80578 +80579 POINT(40.97983256808781 73.02511798573924) bank80579 +80580 POINT(40.95568831186513 74.36166121107392) bank80580 +80581 POINT(41.38081541083463 74.97391915672299) bank80581 +80582 POINT(41.12358880629884 74.46416019612278) bank80582 +80583 POINT(40.374334803144585 73.81166501158084) bank80583 +80584 POINT(41.30145967737308 74.2616857694045) bank80584 +80585 POINT(41.18805261053028 74.88668565423954) bank80585 +80586 POINT(41.18624214527359 73.7805516814812) bank80586 +80587 POINT(39.889889055544835 74.53325922547667) bank80587 +80588 POINT(40.49511626105584 73.94090289305402) bank80588 +80589 POINT(41.304538582589835 74.63005336916981) bank80589 +80590 POINT(40.475373584919986 74.36445824635186) bank80590 +80591 POINT(40.86413579886677 74.75894461491467) bank80591 +80592 POINT(40.86851058489068 74.04036237242467) bank80592 +80593 POINT(40.92499530113969 73.42743503111072) bank80593 +80594 POINT(41.05978584659317 73.39581366580607) bank80594 +80595 POINT(39.85999165262429 73.86174339863307) bank80595 +80596 POINT(40.5894981052502 73.67801570728186) bank80596 +80597 POINT(41.21737231116568 73.93024910083172) bank80597 +80598 POINT(41.65684355022961 73.06511179136601) bank80598 +80599 POINT(40.43593612526595 73.59160213198884) bank80599 +80600 POINT(40.067402079696826 73.26786953233324) bank80600 +80601 POINT(41.12251282653754 74.66954678927382) bank80601 +80602 POINT(40.82156928084871 73.34194603650565) bank80602 +80603 POINT(39.778536044145476 74.76579169821001) bank80603 +80604 POINT(40.65896302512468 73.32309983817994) bank80604 +80605 POINT(41.20320295313981 74.23443758418925) bank80605 +80606 POINT(41.707087599227414 74.37225866515105) bank80606 +80607 POINT(39.972859339445854 73.26263247418537) bank80607 +80608 POINT(40.44267019287088 74.52701048208498) bank80608 +80609 POINT(41.4616433237906 74.68964686221268) bank80609 +80610 POINT(41.48411618979954 73.73622803915475) bank80610 +80611 POINT(39.97594162273664 73.23201846049001) bank80611 +80612 POINT(40.87368922971321 74.54777825924613) bank80612 +80613 POINT(40.636036276405235 74.28458986374214) bank80613 +80614 POINT(40.36406701126569 73.1743545062869) bank80614 +80615 POINT(41.65630916583414 74.86921972896322) bank80615 +80616 POINT(41.19613771044193 73.86327112303071) bank80616 +80617 POINT(39.87078924209895 73.88338814541801) bank80617 +80618 POINT(41.09240943939559 74.31302557191205) bank80618 +80619 POINT(41.29992855103843 73.6607772979914) bank80619 +80620 POINT(41.475043951940044 73.21011929190749) bank80620 +80621 POINT(40.030901572726236 73.64247328702919) bank80621 +80622 POINT(40.65200317803824 74.85723410555653) bank80622 +80623 POINT(40.31453880938296 74.57290866678423) bank80623 +80624 POINT(39.77417481256209 74.21635751836439) bank80624 +80625 POINT(40.59450331014753 74.24512283881585) bank80625 +80626 POINT(41.36961786472563 73.27452870133446) bank80626 +80627 POINT(40.829942545125355 74.8582276760474) bank80627 +80628 POINT(40.70372093740065 73.92870375314232) bank80628 +80629 POINT(39.95813428681869 74.18928822909798) bank80629 +80630 POINT(41.50152537774023 73.47338729374758) bank80630 +80631 POINT(40.42851690394119 74.45978861993191) bank80631 +80632 POINT(40.647220957808486 73.37973230291387) bank80632 +80633 POINT(41.28876947878937 74.56352759696713) bank80633 +80634 POINT(40.042153512404965 73.09348610203118) bank80634 +80635 POINT(40.86955230507272 73.84890764831727) bank80635 +80636 POINT(41.42183885783448 74.39731718084049) bank80636 +80637 POINT(40.63729027327061 73.02529476695506) bank80637 +80638 POINT(41.51167702973092 74.5705535165695) bank80638 +80639 POINT(41.64245048654503 73.27640997248537) bank80639 +80640 POINT(40.33184999488388 74.89231858165499) bank80640 +80641 POINT(41.13143224582838 74.86807475121465) bank80641 +80642 POINT(41.560953073629506 73.48908786955347) bank80642 +80643 POINT(39.75893273262815 73.08613096952689) bank80643 +80644 POINT(40.284346181345036 74.17835512948707) bank80644 +80645 POINT(40.717145737933635 74.46970781128621) bank80645 +80646 POINT(40.50202069625314 74.15152253295885) bank80646 +80647 POINT(41.6747228853678 74.06905382662002) bank80647 +80648 POINT(41.40067402151268 74.83941591300443) bank80648 +80649 POINT(41.57049702850838 73.15879534787769) bank80649 +80650 POINT(40.632822000395464 74.37241659179821) bank80650 +80651 POINT(41.19845824400148 73.17512022619279) bank80651 +80652 POINT(40.831748388620106 74.21214587601868) bank80652 +80653 POINT(41.56764775219002 73.57395826411013) bank80653 +80654 POINT(40.38842526830235 73.7106271069377) bank80654 +80655 POINT(40.20063380934324 73.8109179332923) bank80655 +80656 POINT(40.88131804071294 73.4904400854847) bank80656 +80657 POINT(41.40550326808285 73.58491822509089) bank80657 +80658 POINT(41.429299599069104 73.49781840250895) bank80658 +80659 POINT(40.625084741143446 74.74199231496067) bank80659 +80660 POINT(40.10871080386682 74.87496306556592) bank80660 +80661 POINT(39.749855403892724 74.94367376417699) bank80661 +80662 POINT(40.95257102676814 74.81781384412392) bank80662 +80663 POINT(41.27621814408262 74.51216325783815) bank80663 +80664 POINT(40.202355845165044 74.15819394982067) bank80664 +80665 POINT(39.74233701703891 73.58101256195758) bank80665 +80666 POINT(40.3433031482113 74.96733685702016) bank80666 +80667 POINT(41.55713991244792 73.58944848196884) bank80667 +80668 POINT(39.845089672682874 73.66504616283474) bank80668 +80669 POINT(41.17289832229241 73.3196641069973) bank80669 +80670 POINT(41.64072402453121 74.99562958456617) bank80670 +80671 POINT(41.33917422245031 74.52789799404823) bank80671 +80672 POINT(41.15718389697973 73.4403904619466) bank80672 +80673 POINT(39.94791151612 74.65691289451063) bank80673 +80674 POINT(40.72928157279522 74.86673367884615) bank80674 +80675 POINT(40.41991469521715 73.71444444934427) bank80675 +80676 POINT(41.127483577819454 74.02739440436406) bank80676 +80677 POINT(40.68891077157219 73.58078930661279) bank80677 +80678 POINT(41.435211540155876 74.5305561989972) bank80678 +80679 POINT(41.37132165427629 74.6400771958079) bank80679 +80680 POINT(40.90658238875609 73.16049864685003) bank80680 +80681 POINT(39.98690741371915 74.94783386439154) bank80681 +80682 POINT(41.62388041620896 73.62250474784828) bank80682 +80683 POINT(40.00675598791189 74.28235066374091) bank80683 +80684 POINT(40.10442803030719 74.83096877608943) bank80684 +80685 POINT(41.62295685300727 74.94264275260849) bank80685 +80686 POINT(41.011182446153335 74.42938808636202) bank80686 +80687 POINT(39.777358173440334 74.71813850300335) bank80687 +80688 POINT(39.90301634494399 73.22846249044771) bank80688 +80689 POINT(40.73201827707991 74.26599494597272) bank80689 +80690 POINT(40.38479716456864 74.07544289252245) bank80690 +80691 POINT(40.581188698671966 74.12922115945605) bank80691 +80692 POINT(39.72239211990423 74.31669033325426) bank80692 +80693 POINT(39.86718015019926 74.24624815023742) bank80693 +80694 POINT(41.03581781481067 73.46407336904157) bank80694 +80695 POINT(39.83572201198191 74.8665785041802) bank80695 +80696 POINT(41.70012644765873 73.66425725843389) bank80696 +80697 POINT(39.835469747406975 74.90832976079133) bank80697 +80698 POINT(41.65210228493351 73.19424588773659) bank80698 +80699 POINT(40.01650153775515 73.98124451532868) bank80699 +80700 POINT(39.86148175768954 74.44590791819178) bank80700 +80701 POINT(40.717367524261604 73.87941233888125) bank80701 +80702 POINT(41.034153830099555 74.74205318579632) bank80702 +80703 POINT(40.886358837484806 74.69444822846056) bank80703 +80704 POINT(40.63403048084322 74.14337375899355) bank80704 +80705 POINT(40.1186000359241 74.60394228058874) bank80705 +80706 POINT(40.6806973779548 74.10710208964527) bank80706 +80707 POINT(40.69783144279321 74.88258843891482) bank80707 +80708 POINT(39.87745981503811 74.26408580811307) bank80708 +80709 POINT(41.2033882998506 73.58607338028459) bank80709 +80710 POINT(41.449210546408494 73.83709019070685) bank80710 +80711 POINT(39.80586194092921 74.95771009288677) bank80711 +80712 POINT(40.61134325582784 74.0533765933961) bank80712 +80713 POINT(40.53769895375803 74.00854697938897) bank80713 +80714 POINT(40.23159000441945 74.0555344107254) bank80714 +80715 POINT(40.45850892033542 73.95631313690288) bank80715 +80716 POINT(41.036428536619084 74.7917201723662) bank80716 +80717 POINT(40.68046110059237 73.71898360679988) bank80717 +80718 POINT(40.24137066965306 74.74714189618474) bank80718 +80719 POINT(39.76250664506116 73.48176960498581) bank80719 +80720 POINT(39.89681630733595 74.03223921880942) bank80720 +80721 POINT(40.641172761699906 73.17314483721957) bank80721 +80722 POINT(40.529711389309846 73.39333587408211) bank80722 +80723 POINT(41.69131958131271 73.41273789816913) bank80723 +80724 POINT(41.63105647737998 73.36028085583374) bank80724 +80725 POINT(40.092289142379215 73.27904691264861) bank80725 +80726 POINT(40.396284727780625 73.49615506252738) bank80726 +80727 POINT(41.022118800676864 73.14068034917547) bank80727 +80728 POINT(41.630317254953276 74.44429060072673) bank80728 +80729 POINT(41.17072863580919 74.97381445505746) bank80729 +80730 POINT(41.221217076923516 74.96836433889635) bank80730 +80731 POINT(40.874927323356566 74.23265180890435) bank80731 +80732 POINT(41.259853598600344 74.73182857047958) bank80732 +80733 POINT(39.790511341828115 73.50077107370527) bank80733 +80734 POINT(40.39401719053442 73.02210671911641) bank80734 +80735 POINT(40.596959714868284 74.68566271053238) bank80735 +80736 POINT(41.41667505736725 74.45901618026137) bank80736 +80737 POINT(41.641136530001354 73.06313703366746) bank80737 +80738 POINT(39.841588283997055 74.76453128269196) bank80738 +80739 POINT(39.73869123662246 74.44086482839049) bank80739 +80740 POINT(40.91810078393989 73.58409106519028) bank80740 +80741 POINT(41.650046164421475 73.02159616261774) bank80741 +80742 POINT(40.64865380777506 74.58426166450302) bank80742 +80743 POINT(40.08259506581066 74.58450796779903) bank80743 +80744 POINT(40.871252614799 74.96308078961762) bank80744 +80745 POINT(39.89702162734142 74.28492789825471) bank80745 +80746 POINT(40.91096056373468 74.15622842721837) bank80746 +80747 POINT(41.19760712455444 74.26793545256385) bank80747 +80748 POINT(39.83400551434173 73.39298272851102) bank80748 +80749 POINT(39.71432429295589 73.7653814807408) bank80749 +80750 POINT(39.873434700076146 74.68357987902488) bank80750 +80751 POINT(41.030785885032266 73.24711300236523) bank80751 +80752 POINT(39.960350409081286 74.129927568135) bank80752 +80753 POINT(41.26098594878768 73.92175771771615) bank80753 +80754 POINT(41.33001019642436 74.03045449060356) bank80754 +80755 POINT(40.65301152968138 74.13723948435225) bank80755 +80756 POINT(41.272805536613205 74.00996803929335) bank80756 +80757 POINT(40.07333898677971 74.35606478950459) bank80757 +80758 POINT(40.46765138947609 73.79300852044503) bank80758 +80759 POINT(41.414486337710606 73.4466123812014) bank80759 +80760 POINT(41.540988825511555 73.7952127820395) bank80760 +80761 POINT(40.054618079881706 74.23238690315314) bank80761 +80762 POINT(40.98395502912757 74.0196353440965) bank80762 +80763 POINT(41.373076863639 73.65487759561229) bank80763 +80764 POINT(40.546605750642094 73.87451919431263) bank80764 +80765 POINT(40.896471243325266 73.3310638606357) bank80765 +80766 POINT(39.94472567032802 74.2902351530627) bank80766 +80767 POINT(40.82083321102082 74.50676027220435) bank80767 +80768 POINT(40.70475988323487 74.7848609952022) bank80768 +80769 POINT(41.47611516668379 74.55537630823412) bank80769 +80770 POINT(41.370523235730374 73.8204056634778) bank80770 +80771 POINT(41.41543221103546 73.19952660528854) bank80771 +80772 POINT(41.50557579951994 74.81759510985619) bank80772 +80773 POINT(40.23427925269268 74.56086215199629) bank80773 +80774 POINT(41.332991198362656 73.01115181744865) bank80774 +80775 POINT(41.66922236544185 74.38917179628505) bank80775 +80776 POINT(40.0182640735396 74.78402232763443) bank80776 +80777 POINT(40.62548639719848 73.64626477280859) bank80777 +80778 POINT(41.24858963925254 73.99100070079484) bank80778 +80779 POINT(39.98128568130961 73.15999130821322) bank80779 +80780 POINT(40.66766170884427 73.25447884502474) bank80780 +80781 POINT(41.51487123761512 73.8997392865652) bank80781 +80782 POINT(41.248242572245395 74.06595565476755) bank80782 +80783 POINT(40.078198932445616 73.08498978812368) bank80783 +80784 POINT(40.340818452675435 73.0866752486759) bank80784 +80785 POINT(41.61774430321024 74.44168566695409) bank80785 +80786 POINT(40.96440953599264 73.39211617463909) bank80786 +80787 POINT(41.116233365020626 74.8219000123642) bank80787 +80788 POINT(41.53574241369319 74.67821219004283) bank80788 +80789 POINT(40.193786652341025 73.6104327567752) bank80789 +80790 POINT(39.914775341447886 74.84603542942918) bank80790 +80791 POINT(40.06678460781391 74.39816378903497) bank80791 +80792 POINT(40.44398141536535 74.37289204941534) bank80792 +80793 POINT(41.501055692599415 73.16419514798433) bank80793 +80794 POINT(40.6263028149401 73.25735834325322) bank80794 +80795 POINT(41.43822966706095 74.38127037292942) bank80795 +80796 POINT(40.802170352880104 74.4236353563409) bank80796 +80797 POINT(41.331114112401536 73.40926918966171) bank80797 +80798 POINT(40.67563074564632 74.81991527409684) bank80798 +80799 POINT(41.387900083240964 74.44845494147812) bank80799 +80800 POINT(39.93466128077384 73.6738473268077) bank80800 +80801 POINT(39.85639445974685 74.69236653354724) bank80801 +80802 POINT(39.728926014235505 73.62815986249511) bank80802 +80803 POINT(40.29615797755509 73.04262663305234) bank80803 +80804 POINT(39.90841818555959 73.3315635269385) bank80804 +80805 POINT(40.36157486947853 73.32974739527003) bank80805 +80806 POINT(41.551211233653696 74.69157182138284) bank80806 +80807 POINT(40.596159103164794 73.49214411810672) bank80807 +80808 POINT(40.57229823815456 74.6612836191168) bank80808 +80809 POINT(40.50811576636665 74.00765447802) bank80809 +80810 POINT(41.41158287508412 74.0005362462566) bank80810 +80811 POINT(40.557293168007526 74.55216784387913) bank80811 +80812 POINT(41.68916040725554 73.31382486343357) bank80812 +80813 POINT(40.98361025291543 74.41504090488287) bank80813 +80814 POINT(41.64079982887 73.19752388717588) bank80814 +80815 POINT(40.931480214991055 74.36577549902245) bank80815 +80816 POINT(41.16066126355496 73.67457088467916) bank80816 +80817 POINT(41.309329233230585 74.72674483594152) bank80817 +80818 POINT(41.61038464273525 73.55178997269473) bank80818 +80819 POINT(40.39017007902072 73.61292577323454) bank80819 +80820 POINT(41.67895152151332 73.43970596709543) bank80820 +80821 POINT(40.75317780905503 74.74685608370073) bank80821 +80822 POINT(41.37591229747449 73.60090731389097) bank80822 +80823 POINT(41.59230270816057 73.80097041962632) bank80823 +80824 POINT(41.16054010947594 73.99941537978103) bank80824 +80825 POINT(40.03750429102203 73.39601440057899) bank80825 +80826 POINT(41.114313380465696 73.17564042475166) bank80826 +80827 POINT(41.30794320953073 73.37193403789358) bank80827 +80828 POINT(40.37495427850775 73.57744798197876) bank80828 +80829 POINT(41.188574527929916 74.95507123140378) bank80829 +80830 POINT(40.92023215228461 75.00460039348157) bank80830 +80831 POINT(41.58361156721688 73.46464182238329) bank80831 +80832 POINT(40.548187177788826 73.01720940165279) bank80832 +80833 POINT(40.738542288399806 74.09275880180653) bank80833 +80834 POINT(41.204790199065116 74.03270125745969) bank80834 +80835 POINT(41.30729024237044 74.67587191753589) bank80835 +80836 POINT(41.31565342117839 73.99550720794316) bank80836 +80837 POINT(40.83494653884388 74.11033336817364) bank80837 +80838 POINT(39.83013932671782 73.21604467604081) bank80838 +80839 POINT(41.52517414247758 73.30554712152606) bank80839 +80840 POINT(41.65921953797292 74.08262283033966) bank80840 +80841 POINT(40.72673853997029 73.10516917878664) bank80841 +80842 POINT(40.04917334360852 74.58354480184312) bank80842 +80843 POINT(40.319833132980875 73.82617780625596) bank80843 +80844 POINT(40.67450576235638 73.71439760472974) bank80844 +80845 POINT(40.18993690774468 74.98615886136898) bank80845 +80846 POINT(39.878423626879716 74.99965299998152) bank80846 +80847 POINT(41.34575213175136 74.77227015471621) bank80847 +80848 POINT(40.08378262979221 74.74341137252914) bank80848 +80849 POINT(40.519190335949666 73.12659415267198) bank80849 +80850 POINT(40.739366999655374 73.77658702741448) bank80850 +80851 POINT(39.749202381747565 74.26433025310064) bank80851 +80852 POINT(40.37392622995652 74.0914570688057) bank80852 +80853 POINT(40.9070598357539 74.86098405359036) bank80853 +80854 POINT(40.596766792440704 73.48129607407708) bank80854 +80855 POINT(39.770237914628275 74.33611331631481) bank80855 +80856 POINT(41.14900389664662 73.69456054895818) bank80856 +80857 POINT(40.74318120070098 74.8945477866715) bank80857 +80858 POINT(40.01801392696655 73.03083992122734) bank80858 +80859 POINT(41.20294687700664 74.95629322817092) bank80859 +80860 POINT(41.02189141867718 74.37286274102593) bank80860 +80861 POINT(40.878653669572145 74.72637201851374) bank80861 +80862 POINT(41.668983026796646 73.93049819240677) bank80862 +80863 POINT(41.09361681387671 73.55333104611472) bank80863 +80864 POINT(40.00983573725228 73.69728661203064) bank80864 +80865 POINT(40.282195254742064 73.61070047251096) bank80865 +80866 POINT(40.69250797474045 74.63345998873562) bank80866 +80867 POINT(41.14222502305545 73.54784130263398) bank80867 +80868 POINT(41.65335042891042 74.06468444685208) bank80868 +80869 POINT(41.20304659263814 74.32973293420152) bank80869 +80870 POINT(39.95297353786183 73.35978929726451) bank80870 +80871 POINT(39.90940485245864 74.62116480602289) bank80871 +80872 POINT(40.69225924506046 73.60276719722755) bank80872 +80873 POINT(41.089348643470125 73.46518446436778) bank80873 +80874 POINT(40.21294811806658 74.05779401054656) bank80874 +80875 POINT(40.11555540615037 74.90418936425255) bank80875 +80876 POINT(40.531540286555206 73.94607083587495) bank80876 +80877 POINT(40.1877563472441 73.81995481723618) bank80877 +80878 POINT(41.6378198697404 73.68987465235811) bank80878 +80879 POINT(41.57113668030512 74.93849772899028) bank80879 +80880 POINT(40.89477939784436 74.83314207356199) bank80880 +80881 POINT(40.60976594441803 73.81860962985384) bank80881 +80882 POINT(39.81415977584214 74.42408386121822) bank80882 +80883 POINT(41.328233636327624 73.71236211143379) bank80883 +80884 POINT(40.750652326578596 73.76712044282816) bank80884 +80885 POINT(40.55864729178423 73.62508776319692) bank80885 +80886 POINT(40.388877196354024 74.3308247060971) bank80886 +80887 POINT(40.22636953625448 74.37579461983393) bank80887 +80888 POINT(40.2403944194659 74.0321230817836) bank80888 +80889 POINT(40.85725323620447 74.30607524891869) bank80889 +80890 POINT(40.17957305808084 73.44198889314904) bank80890 +80891 POINT(40.405522751840614 74.2551596412573) bank80891 +80892 POINT(40.88348128870168 74.43423691003112) bank80892 +80893 POINT(40.567043885415536 74.96537775555748) bank80893 +80894 POINT(40.439240475225986 74.68178187673696) bank80894 +80895 POINT(40.12095901136712 73.54741281111649) bank80895 +80896 POINT(40.963346759751516 74.57883876108001) bank80896 +80897 POINT(39.88729955001834 74.0065232614035) bank80897 +80898 POINT(41.134662581519095 74.78491900835317) bank80898 +80899 POINT(41.64955370354698 74.56281732184627) bank80899 +80900 POINT(41.367984015652794 74.67512672941324) bank80900 +80901 POINT(40.92198344476677 73.39260546604453) bank80901 +80902 POINT(40.43848552561215 74.73120071109108) bank80902 +80903 POINT(39.86781489613025 74.70878629795628) bank80903 +80904 POINT(41.65435504860902 74.5757456244378) bank80904 +80905 POINT(40.12928702274265 74.67687544963165) bank80905 +80906 POINT(40.99855982183112 73.48051410340716) bank80906 +80907 POINT(40.742482268714774 74.43243638447866) bank80907 +80908 POINT(41.160715041079676 73.787699586143) bank80908 +80909 POINT(40.93968987788119 74.75211334069353) bank80909 +80910 POINT(40.86575873117136 74.44873733640466) bank80910 +80911 POINT(39.71370953567126 74.47124935075188) bank80911 +80912 POINT(41.03174864644162 73.05173305160697) bank80912 +80913 POINT(41.489879312284735 73.03775302390221) bank80913 +80914 POINT(40.8390409400503 73.7508895426007) bank80914 +80915 POINT(41.169804879038146 74.73723593985075) bank80915 +80916 POINT(40.2660001713727 73.6181526178966) bank80916 +80917 POINT(40.45346105424797 74.64370627311094) bank80917 +80918 POINT(41.11355239128059 73.26532252262993) bank80918 +80919 POINT(39.840516379418816 74.61395606805077) bank80919 +80920 POINT(39.88102567420791 74.84126802965895) bank80920 +80921 POINT(41.332550833200365 73.23541676624308) bank80921 +80922 POINT(40.787868408967114 73.34212192784689) bank80922 +80923 POINT(41.58115191269922 74.8290967894919) bank80923 +80924 POINT(41.349136074732215 73.93438041247214) bank80924 +80925 POINT(41.54218208641568 73.4443217379682) bank80925 +80926 POINT(40.296567440287994 73.23691715526218) bank80926 +80927 POINT(41.34488303909425 73.99073652466065) bank80927 +80928 POINT(40.88098959367228 74.21977635021751) bank80928 +80929 POINT(40.04673583008528 73.918755158312) bank80929 +80930 POINT(40.56922566911727 73.19628436420199) bank80930 +80931 POINT(40.6395513260092 74.49537419128986) bank80931 +80932 POINT(39.88742162546034 74.38624209359908) bank80932 +80933 POINT(40.084993228175286 74.94989741404844) bank80933 +80934 POINT(41.114688742815275 74.7952526718548) bank80934 +80935 POINT(39.81217841742049 73.55902183422936) bank80935 +80936 POINT(39.76527702546645 74.60611758531442) bank80936 +80937 POINT(40.4253237025452 74.18334393662145) bank80937 +80938 POINT(40.09503261200583 74.39169311289133) bank80938 +80939 POINT(41.54920671546141 74.28886304731752) bank80939 +80940 POINT(40.77776055802957 73.12637773176361) bank80940 +80941 POINT(41.152092498900245 73.68856497786835) bank80941 +80942 POINT(40.933215942668525 74.83320221796188) bank80942 +80943 POINT(40.214508281207344 73.95379520153129) bank80943 +80944 POINT(40.085268428178146 73.63989206882498) bank80944 +80945 POINT(40.44099041770551 73.84088275352217) bank80945 +80946 POINT(41.05612639969319 73.43871487557358) bank80946 +80947 POINT(40.27324875308892 73.23926455434356) bank80947 +80948 POINT(40.301008727338164 73.74944138571188) bank80948 +80949 POINT(40.38922864659123 74.38810231881045) bank80949 +80950 POINT(40.74466198702371 74.56765680949773) bank80950 +80951 POINT(39.96766830744021 73.50967665351186) bank80951 +80952 POINT(41.106627326217065 74.21658835832305) bank80952 +80953 POINT(41.17057812940551 73.7380607039607) bank80953 +80954 POINT(40.54968444813102 74.08309787831969) bank80954 +80955 POINT(40.82602375050504 74.67551571398846) bank80955 +80956 POINT(40.73130485783686 73.62101647563756) bank80956 +80957 POINT(39.998627434736036 73.5262550432467) bank80957 +80958 POINT(41.28518690284049 73.63470343999369) bank80958 +80959 POINT(40.96511324155624 74.79012823907992) bank80959 +80960 POINT(40.78157174045783 73.31139641598972) bank80960 +80961 POINT(40.13252188673743 73.31428253350508) bank80961 +80962 POINT(40.58653775051346 74.30988561875643) bank80962 +80963 POINT(41.33585120334955 74.0445327782432) bank80963 +80964 POINT(41.18931202760217 74.0032892825821) bank80964 +80965 POINT(41.252521995721104 74.71135526581217) bank80965 +80966 POINT(40.421567730385775 73.93275721884088) bank80966 +80967 POINT(40.895362202278605 73.29163663057027) bank80967 +80968 POINT(40.89897631357978 73.38273216914664) bank80968 +80969 POINT(40.90143747993848 74.96326251440313) bank80969 +80970 POINT(41.40110876047205 73.61342416994765) bank80970 +80971 POINT(40.68652429407937 73.24518124115626) bank80971 +80972 POINT(39.8304282198221 74.40579974037121) bank80972 +80973 POINT(40.13010319254787 74.1244152498275) bank80973 +80974 POINT(40.9802689431735 73.72590409247508) bank80974 +80975 POINT(40.293409184899694 73.32494502968467) bank80975 +80976 POINT(41.47809004859103 73.64355225802923) bank80976 +80977 POINT(41.30316400799182 73.28061568015094) bank80977 +80978 POINT(41.5501155937534 73.8204342127767) bank80978 +80979 POINT(39.84709453212376 74.83619532379493) bank80979 +80980 POINT(41.3582585805281 73.72657137941141) bank80980 +80981 POINT(40.20093169351417 73.148769107017) bank80981 +80982 POINT(39.766412900080084 73.85472671115116) bank80982 +80983 POINT(41.45594727734095 73.7317001664684) bank80983 +80984 POINT(40.96284171951722 73.55049805606448) bank80984 +80985 POINT(41.02776266034708 74.81178618003763) bank80985 +80986 POINT(40.57275912517339 74.85369620434157) bank80986 +80987 POINT(41.27193275940258 74.55757913397707) bank80987 +80988 POINT(41.107536832316526 73.892633099377) bank80988 +80989 POINT(41.43671324416255 73.21894690679603) bank80989 +80990 POINT(41.04450741891881 74.08720596970706) bank80990 +80991 POINT(41.56654453440831 73.03248454930058) bank80991 +80992 POINT(40.249306891104155 74.30844624981592) bank80992 +80993 POINT(40.13977578468569 73.93340444929828) bank80993 +80994 POINT(41.69699811739597 73.05460848969852) bank80994 +80995 POINT(41.13799946905198 74.23285435885757) bank80995 +80996 POINT(41.32428528714684 74.80497609669938) bank80996 +80997 POINT(40.42272727185411 73.47859367430603) bank80997 +80998 POINT(39.83860438189416 73.10964290528025) bank80998 +80999 POINT(41.660002074512875 74.26977891389542) bank80999 +81000 POINT(39.891810522608615 74.30047073838989) bank81000 +81001 POINT(40.987805102608 73.70107218169768) bank81001 +81002 POINT(40.934675837763194 73.67525259049886) bank81002 +81003 POINT(40.54179052528838 73.12773994357084) bank81003 +81004 POINT(40.97343433875665 73.34212139729806) bank81004 +81005 POINT(40.756734603995454 74.03275071845658) bank81005 +81006 POINT(39.990397364393374 73.98602253699602) bank81006 +81007 POINT(40.46873694024442 74.3421167661244) bank81007 +81008 POINT(40.05238697502654 73.60297353211672) bank81008 +81009 POINT(39.90441874252818 73.62402134649825) bank81009 +81010 POINT(39.92565668038578 73.81766835448052) bank81010 +81011 POINT(41.438068929941316 73.34597319061741) bank81011 +81012 POINT(40.449015300901046 74.46951420917682) bank81012 +81013 POINT(41.34329109330052 74.31923535352392) bank81013 +81014 POINT(41.329890343954844 74.84425948288337) bank81014 +81015 POINT(39.745570897607514 73.18102979924605) bank81015 +81016 POINT(39.99322040792525 73.2739013719449) bank81016 +81017 POINT(40.003162073182416 73.66823455780793) bank81017 +81018 POINT(40.1121288911398 74.43660719027086) bank81018 +81019 POINT(40.74333276833568 73.0911386441884) bank81019 +81020 POINT(41.61102961442481 74.39563643668438) bank81020 +81021 POINT(40.72055502152071 73.45994221686993) bank81021 +81022 POINT(40.65937114770547 74.84604142800136) bank81022 +81023 POINT(41.518779451206484 74.57430268069439) bank81023 +81024 POINT(41.45598976832108 73.92313332667311) bank81024 +81025 POINT(40.39828315971197 74.1310200855133) bank81025 +81026 POINT(40.25258200297461 74.72118763268506) bank81026 +81027 POINT(39.72355815879733 73.83513874193346) bank81027 +81028 POINT(41.14810059453695 73.1244806046556) bank81028 +81029 POINT(41.64348594282161 73.59651751480351) bank81029 +81030 POINT(41.18529761288725 73.74629903646505) bank81030 +81031 POINT(41.6718084805739 74.98224713007484) bank81031 +81032 POINT(40.1742189791188 74.34084011565129) bank81032 +81033 POINT(39.76112436278545 73.9445165347102) bank81033 +81034 POINT(40.68898500195781 73.31221491320406) bank81034 +81035 POINT(41.155214608587414 73.85412857575092) bank81035 +81036 POINT(40.339242854084475 74.40387237807535) bank81036 +81037 POINT(40.09907837069539 73.73869740430207) bank81037 +81038 POINT(41.674463675439405 73.52285134337491) bank81038 +81039 POINT(40.78207508284826 73.01761832690856) bank81039 +81040 POINT(41.4990163380657 74.94843913547669) bank81040 +81041 POINT(40.828024922439596 74.36732327593505) bank81041 +81042 POINT(40.392991811251605 74.51205597370485) bank81042 +81043 POINT(41.51254319743731 74.39210234717493) bank81043 +81044 POINT(40.729849859483906 74.58245188813088) bank81044 +81045 POINT(40.713867345317006 74.00754086447249) bank81045 +81046 POINT(39.95888081885827 73.19524832374874) bank81046 +81047 POINT(40.84799954546604 73.94472857828804) bank81047 +81048 POINT(40.1698326262676 74.60801105854716) bank81048 +81049 POINT(41.55533284405837 73.89848858561282) bank81049 +81050 POINT(41.20143493956013 74.32630544451447) bank81050 +81051 POINT(40.583480035683245 73.67633232554942) bank81051 +81052 POINT(41.65838836532308 73.36216095264308) bank81052 +81053 POINT(41.49965353241051 73.06636498341582) bank81053 +81054 POINT(40.344173348000076 73.5291956430849) bank81054 +81055 POINT(41.60226677186739 73.66062327120186) bank81055 +81056 POINT(41.43700986680929 73.63547249297572) bank81056 +81057 POINT(40.78901027499421 73.4385266036776) bank81057 +81058 POINT(40.173249671655405 74.56348602736861) bank81058 +81059 POINT(41.38237162801342 73.90933813054595) bank81059 +81060 POINT(40.6779274604004 73.81821603731171) bank81060 +81061 POINT(41.70254272433528 73.70444977269449) bank81061 +81062 POINT(40.861187677529315 73.0760176887833) bank81062 +81063 POINT(41.42359303261509 74.12560235528306) bank81063 +81064 POINT(40.583796614664976 73.8502352308777) bank81064 +81065 POINT(39.85769939694784 74.8631627538627) bank81065 +81066 POINT(41.43419928297956 74.2375930353181) bank81066 +81067 POINT(39.789701838682014 73.51876760332885) bank81067 +81068 POINT(41.65283592032615 74.53124680585302) bank81068 +81069 POINT(40.26085430833622 74.55298102691215) bank81069 +81070 POINT(40.90615956341777 73.46420797678795) bank81070 +81071 POINT(40.41380592637809 73.8938946750609) bank81071 +81072 POINT(40.967774497493174 74.68072823439044) bank81072 +81073 POINT(41.47800724064833 74.09455050445291) bank81073 +81074 POINT(41.018386565875424 74.89920357128001) bank81074 +81075 POINT(40.189667843754904 73.16005790288956) bank81075 +81076 POINT(40.25249377348773 73.43431240653422) bank81076 +81077 POINT(39.91238304362324 74.61132367944111) bank81077 +81078 POINT(40.579646001530456 74.16013065698002) bank81078 +81079 POINT(41.50630943778836 73.78817363018129) bank81079 +81080 POINT(40.16591835118047 74.5017018629388) bank81080 +81081 POINT(40.05804559739419 74.84514812672863) bank81081 +81082 POINT(41.07795661062157 73.03713880133864) bank81082 +81083 POINT(40.88516358561699 74.71116182935502) bank81083 +81084 POINT(40.500087948272316 74.60864370534831) bank81084 +81085 POINT(40.55804791579269 74.75880830434059) bank81085 +81086 POINT(41.22786893348881 74.64533670387242) bank81086 +81087 POINT(40.88003104009373 73.6287833981442) bank81087 +81088 POINT(40.259006690176626 73.3519778098259) bank81088 +81089 POINT(39.986638198386935 73.0792896995434) bank81089 +81090 POINT(40.28277482419788 73.96605291601199) bank81090 +81091 POINT(40.70378176774861 74.45123137567653) bank81091 +81092 POINT(41.389759639667275 73.3908271482271) bank81092 +81093 POINT(39.92629100273491 74.51841774749366) bank81093 +81094 POINT(40.002016049361146 74.41288743077033) bank81094 +81095 POINT(40.92334612774166 73.55453039906732) bank81095 +81096 POINT(39.81256095507863 73.2577868477701) bank81096 +81097 POINT(40.925277206019146 74.06424892284963) bank81097 +81098 POINT(41.29832976361232 74.08336298714742) bank81098 +81099 POINT(41.70122960214277 74.58942912451475) bank81099 +81100 POINT(40.10504164620904 74.74696131956529) bank81100 +81101 POINT(40.74457324485098 73.65681245823333) bank81101 +81102 POINT(40.0931606953532 73.87603531771363) bank81102 +81103 POINT(40.444164432086545 74.55118107805663) bank81103 +81104 POINT(40.208821042045116 73.89964933138491) bank81104 +81105 POINT(40.0739355299154 73.62456024411986) bank81105 +81106 POINT(41.35735144120973 74.12590340999692) bank81106 +81107 POINT(39.80444632861868 73.34266557404048) bank81107 +81108 POINT(40.25684570606097 73.96752493671679) bank81108 +81109 POINT(40.42079116290413 73.05309735519297) bank81109 +81110 POINT(40.275037865974866 73.15003005738266) bank81110 +81111 POINT(40.55801288270114 73.64656821794242) bank81111 +81112 POINT(41.393009523785906 74.00282861902794) bank81112 +81113 POINT(39.9587821462396 74.12788379551452) bank81113 +81114 POINT(40.467360082544545 73.78513586023121) bank81114 +81115 POINT(39.926427310943616 75.00342007534293) bank81115 +81116 POINT(39.765132892338336 74.43711972742746) bank81116 +81117 POINT(40.19146218721986 74.08377902494388) bank81117 +81118 POINT(41.175912037820815 74.01814005160253) bank81118 +81119 POINT(40.1526516164937 74.70369183880455) bank81119 +81120 POINT(41.36349956545586 74.46016437658545) bank81120 +81121 POINT(41.00534914549334 74.49601504305586) bank81121 +81122 POINT(40.17916564516748 74.12967421588752) bank81122 +81123 POINT(40.921444638762495 73.4789051836996) bank81123 +81124 POINT(41.09360490408815 74.43622842054458) bank81124 +81125 POINT(41.678187558477866 74.51942367267714) bank81125 +81126 POINT(41.29439779102256 73.27941882454685) bank81126 +81127 POINT(40.76040798905816 73.64907740517036) bank81127 +81128 POINT(41.40077127360663 74.33441150354732) bank81128 +81129 POINT(40.757763742910264 74.61367884497646) bank81129 +81130 POINT(40.40574870725946 73.74912789707327) bank81130 +81131 POINT(39.88472018246619 73.7176730272222) bank81131 +81132 POINT(41.64684684043254 73.34088787634433) bank81132 +81133 POINT(39.87461965434477 73.61664908615401) bank81133 +81134 POINT(41.26817343712971 73.4164701604814) bank81134 +81135 POINT(41.12098892192331 73.33585995950978) bank81135 +81136 POINT(40.066814463962615 74.78581484814987) bank81136 +81137 POINT(41.59531684381789 73.45799896888028) bank81137 +81138 POINT(41.310267032177435 73.12930780268707) bank81138 +81139 POINT(41.688486190134185 73.4197904181348) bank81139 +81140 POINT(39.914292059854446 74.5330375099072) bank81140 +81141 POINT(41.31407544373993 73.98398969037164) bank81141 +81142 POINT(40.195141219222826 74.9799616166958) bank81142 +81143 POINT(40.595588693604405 73.74766386258858) bank81143 +81144 POINT(41.43264499794293 73.16016000774532) bank81144 +81145 POINT(40.98346082117288 74.62510451310858) bank81145 +81146 POINT(41.50697842993724 73.05117922630143) bank81146 +81147 POINT(40.24289833089508 73.91456769791262) bank81147 +81148 POINT(40.87732674281798 74.03316191450651) bank81148 +81149 POINT(41.40640914482358 74.23815080639834) bank81149 +81150 POINT(41.29559778321853 74.79244173930374) bank81150 +81151 POINT(40.24032634642488 73.0805897395198) bank81151 +81152 POINT(40.00227611687808 74.61212249444418) bank81152 +81153 POINT(40.99479399320708 74.53039094383584) bank81153 +81154 POINT(40.096178833880124 74.10735411191608) bank81154 +81155 POINT(40.67793699816191 73.88898987745632) bank81155 +81156 POINT(41.52088541942916 74.12763662363706) bank81156 +81157 POINT(40.75952943937386 73.90095318759715) bank81157 +81158 POINT(40.840819081134455 73.3840486125175) bank81158 +81159 POINT(41.075949862114015 74.0095921608605) bank81159 +81160 POINT(40.11742630776535 74.45654710395063) bank81160 +81161 POINT(40.11997112078077 74.4266414276127) bank81161 +81162 POINT(41.129824347083016 73.84454128971667) bank81162 +81163 POINT(40.946779007062204 74.64150218433376) bank81163 +81164 POINT(41.24796005960373 74.0377216998637) bank81164 +81165 POINT(41.34234747436857 73.56693152237676) bank81165 +81166 POINT(40.236019475740925 74.1521280549772) bank81166 +81167 POINT(39.77244996553163 74.75481360058475) bank81167 +81168 POINT(41.287403994375936 73.44231279065625) bank81168 +81169 POINT(39.7379168762595 74.32289255103431) bank81169 +81170 POINT(39.88909027038471 73.65391523413277) bank81170 +81171 POINT(41.53951805313351 74.42055461908855) bank81171 +81172 POINT(40.283059320005506 73.55346590464275) bank81172 +81173 POINT(41.13797486724237 73.3320580489071) bank81173 +81174 POINT(41.54248098374272 73.7786026470345) bank81174 +81175 POINT(40.52017928304416 73.28661296424666) bank81175 +81176 POINT(41.04560793231588 73.0275948418519) bank81176 +81177 POINT(40.49761893328176 74.10794982702888) bank81177 +81178 POINT(39.97540570407615 73.77322819930912) bank81178 +81179 POINT(41.229861160806465 74.14757760727188) bank81179 +81180 POINT(41.10887999804902 73.80748743182141) bank81180 +81181 POINT(39.81185251196329 74.07116642119479) bank81181 +81182 POINT(41.68423942482565 73.87501748074546) bank81182 +81183 POINT(40.846912720042 74.1257638578202) bank81183 +81184 POINT(41.21211533001186 73.88852919776461) bank81184 +81185 POINT(40.71448937924944 74.18881281964025) bank81185 +81186 POINT(40.23314309084634 74.51790505562977) bank81186 +81187 POINT(40.62548470706443 73.3691592113026) bank81187 +81188 POINT(39.724954112674794 73.32004924986188) bank81188 +81189 POINT(40.21065903716508 74.98834743959084) bank81189 +81190 POINT(40.54125929672364 73.74425861533598) bank81190 +81191 POINT(40.9763489033772 74.56171602686966) bank81191 +81192 POINT(40.33170503307991 74.25362089423349) bank81192 +81193 POINT(40.784877317377685 73.46282663134704) bank81193 +81194 POINT(40.91912504428513 73.10720146195881) bank81194 +81195 POINT(40.62653640202406 74.01113892283142) bank81195 +81196 POINT(41.51561819920468 73.86978518861349) bank81196 +81197 POINT(39.973079861434634 73.54318982875225) bank81197 +81198 POINT(40.83653596417039 73.52401806323695) bank81198 +81199 POINT(40.985292829489595 73.74478460290335) bank81199 +81200 POINT(40.52834045489914 74.41358059584076) bank81200 +81201 POINT(39.995025559598815 73.38576309969483) bank81201 +81202 POINT(41.23659355480083 74.73351741021038) bank81202 +81203 POINT(41.15358492129336 73.93192751265916) bank81203 +81204 POINT(41.61137139630767 73.76275251107589) bank81204 +81205 POINT(40.975256051318524 74.926337403324) bank81205 +81206 POINT(40.90672343056488 74.90423850236986) bank81206 +81207 POINT(41.60213430912182 74.44021231236667) bank81207 +81208 POINT(41.25184764991767 73.65259223659433) bank81208 +81209 POINT(40.07849709272481 74.95693271181115) bank81209 +81210 POINT(40.809274066439315 74.34808031180029) bank81210 +81211 POINT(41.318392800742984 74.14184613353933) bank81211 +81212 POINT(41.29128866838982 73.54579021096919) bank81212 +81213 POINT(40.87794044955311 73.69462782254507) bank81213 +81214 POINT(41.4438727393397 73.66001295371417) bank81214 +81215 POINT(41.150798283342674 73.37842167301301) bank81215 +81216 POINT(40.61521490190442 73.18236118597676) bank81216 +81217 POINT(41.333455615488525 73.1206236120028) bank81217 +81218 POINT(41.02500512733983 74.774676228936) bank81218 +81219 POINT(40.120146977563934 73.64269935172113) bank81219 +81220 POINT(41.658169477351805 74.31915989998299) bank81220 +81221 POINT(41.2278716549063 73.10318600544971) bank81221 +81222 POINT(39.89327398537876 73.36710193531961) bank81222 +81223 POINT(40.595500463432366 74.7745551453301) bank81223 +81224 POINT(39.96709161195669 74.11807825607501) bank81224 +81225 POINT(41.00647250772803 74.29200049962809) bank81225 +81226 POINT(40.54480725560096 75.0006114823891) bank81226 +81227 POINT(40.26668807604923 74.97120370736066) bank81227 +81228 POINT(41.51218555915831 74.16050056569034) bank81228 +81229 POINT(41.6888431582372 74.8515451176495) bank81229 +81230 POINT(40.047242472803326 73.46226247954482) bank81230 +81231 POINT(41.102701333781354 74.11857453513366) bank81231 +81232 POINT(41.05094523178104 74.4975871313035) bank81232 +81233 POINT(40.19915520793229 74.02272989442892) bank81233 +81234 POINT(40.61646211682983 74.75745760094283) bank81234 +81235 POINT(40.79211569734242 74.31671143845979) bank81235 +81236 POINT(41.41894955648425 74.30567097557164) bank81236 +81237 POINT(41.583442110000156 74.07176125640702) bank81237 +81238 POINT(41.07554027894841 74.25260040235096) bank81238 +81239 POINT(41.20358884015706 73.02683270658103) bank81239 +81240 POINT(39.98026970909156 74.99220644563516) bank81240 +81241 POINT(39.841729475290585 74.38538600344438) bank81241 +81242 POINT(39.80008116903882 73.74373254292121) bank81242 +81243 POINT(41.25121440664102 74.59856550553323) bank81243 +81244 POINT(40.30116248062132 73.99056343167247) bank81244 +81245 POINT(40.924807260171285 73.56884284217551) bank81245 +81246 POINT(41.357308299099856 73.9837376225468) bank81246 +81247 POINT(40.5490408667791 74.36082386022188) bank81247 +81248 POINT(40.117405884397485 73.4449353050984) bank81248 +81249 POINT(41.563949535515775 74.87032958447902) bank81249 +81250 POINT(41.2190372394957 73.23326301155474) bank81250 +81251 POINT(40.771060423836474 73.19617801568175) bank81251 +81252 POINT(40.621588769758176 74.94005547563638) bank81252 +81253 POINT(40.78741456845039 73.64337779406456) bank81253 +81254 POINT(41.67373103177838 73.63686295403137) bank81254 +81255 POINT(39.86537596693605 73.98681277867506) bank81255 +81256 POINT(39.85984000524789 74.65310504034295) bank81256 +81257 POINT(41.6095817101305 73.83012280402974) bank81257 +81258 POINT(40.679270306289396 73.5441488146606) bank81258 +81259 POINT(39.957474900555496 73.65641451223378) bank81259 +81260 POINT(41.05316327445234 73.39450645613393) bank81260 +81261 POINT(41.23908203924115 73.2188683485384) bank81261 +81262 POINT(40.42008287283205 73.93180116015928) bank81262 +81263 POINT(40.7628337464628 74.3093119524656) bank81263 +81264 POINT(40.074991448079466 73.70042441061346) bank81264 +81265 POINT(40.51611145808598 74.41017133842139) bank81265 +81266 POINT(41.03378118252147 74.35999411317857) bank81266 +81267 POINT(40.616738053017734 74.14136703672376) bank81267 +81268 POINT(41.46500023452034 73.80039206748593) bank81268 +81269 POINT(40.10772572058243 74.69706779895468) bank81269 +81270 POINT(40.510084915111555 74.53337499665356) bank81270 +81271 POINT(40.35211563490921 73.61978326411236) bank81271 +81272 POINT(41.54515981803809 73.0096657391765) bank81272 +81273 POINT(41.4696535498072 74.37448453554187) bank81273 +81274 POINT(41.6203603802641 74.72884786722906) bank81274 +81275 POINT(39.77649815158077 74.5661534667984) bank81275 +81276 POINT(41.04310345962779 73.16519046558942) bank81276 +81277 POINT(40.87283180836824 74.92147173974958) bank81277 +81278 POINT(40.81938294732168 74.1243110410506) bank81278 +81279 POINT(41.15136105350683 73.42781858356341) bank81279 +81280 POINT(40.812322455757524 73.23434817526362) bank81280 +81281 POINT(41.55393084165442 73.68072870482578) bank81281 +81282 POINT(40.73124705672362 73.55716982302313) bank81282 +81283 POINT(41.661514789591656 73.69469698676207) bank81283 +81284 POINT(39.80993305916493 74.34001390684209) bank81284 +81285 POINT(41.2457121102603 73.8847701769329) bank81285 +81286 POINT(40.71569238955904 74.62664277509631) bank81286 +81287 POINT(41.622925709217505 74.43619625846485) bank81287 +81288 POINT(41.43315279848738 73.34422368522598) bank81288 +81289 POINT(40.23979339557503 73.2261111769344) bank81289 +81290 POINT(41.053664393405995 73.07939029546637) bank81290 +81291 POINT(39.865622240576826 73.99271918503025) bank81291 +81292 POINT(39.96917731976905 74.63835497004546) bank81292 +81293 POINT(39.81932336634984 74.94643024577505) bank81293 +81294 POINT(40.69778352287767 73.61757504317697) bank81294 +81295 POINT(41.48360302698872 74.15493916932023) bank81295 +81296 POINT(40.094147025080666 74.05141408249268) bank81296 +81297 POINT(41.54705554643688 73.47556372441842) bank81297 +81298 POINT(41.25513006676023 74.81981809545623) bank81298 +81299 POINT(40.90890364083714 74.08582779173062) bank81299 +81300 POINT(41.333911160038355 73.52829471819068) bank81300 +81301 POINT(39.904976161118455 74.67098224758261) bank81301 +81302 POINT(41.08799105950641 74.35873185066727) bank81302 +81303 POINT(39.71600019421477 73.94481828007652) bank81303 +81304 POINT(40.18494244326871 74.21788865845771) bank81304 +81305 POINT(40.349100071006234 74.22087171696408) bank81305 +81306 POINT(41.309409665412346 74.52706884588392) bank81306 +81307 POINT(41.41182402762186 74.81337462719752) bank81307 +81308 POINT(39.99488954648973 73.70255298870178) bank81308 +81309 POINT(40.573324929197774 73.2760143304124) bank81309 +81310 POINT(40.10389911545599 73.71007843068082) bank81310 +81311 POINT(41.229161900494205 73.88074102392643) bank81311 +81312 POINT(40.328231988737706 74.00036845926034) bank81312 +81313 POINT(40.28602606050415 73.98959341205915) bank81313 +81314 POINT(40.491444762312206 73.81755316582056) bank81314 +81315 POINT(39.84623925715174 73.59405305939819) bank81315 +81316 POINT(40.78933141888514 74.84807437207698) bank81316 +81317 POINT(40.03075934798487 73.4344669818795) bank81317 +81318 POINT(40.11462536916929 74.22724027246495) bank81318 +81319 POINT(40.71192580609469 74.50883280324165) bank81319 +81320 POINT(39.99553402455279 73.97281828897086) bank81320 +81321 POINT(41.05975050474809 74.40493169241098) bank81321 +81322 POINT(40.18469098702576 74.34300217934349) bank81322 +81323 POINT(40.107066714845125 73.05300092755564) bank81323 +81324 POINT(40.85940039891498 73.90142531282379) bank81324 +81325 POINT(40.54686388848767 74.5245340980673) bank81325 +81326 POINT(41.68117295527136 74.86508303646826) bank81326 +81327 POINT(41.361916980848534 73.73979107516797) bank81327 +81328 POINT(41.571838186454094 74.693458477616) bank81328 +81329 POINT(40.779309830455574 73.15135780625053) bank81329 +81330 POINT(40.275957949478645 74.92416523788462) bank81330 +81331 POINT(40.45817784657646 73.93123089857497) bank81331 +81332 POINT(40.83108645557108 74.3308315681589) bank81332 +81333 POINT(41.093722949072756 74.988193179104) bank81333 +81334 POINT(40.8407083718158 74.06733580726176) bank81334 +81335 POINT(40.28277241541122 74.42265149169215) bank81335 +81336 POINT(39.94341254656466 73.98180710693427) bank81336 +81337 POINT(41.52970774272299 73.42044513892208) bank81337 +81338 POINT(40.36151693946276 74.0153048600651) bank81338 +81339 POINT(40.804343245614234 73.81152868796764) bank81339 +81340 POINT(40.87936153787199 74.08749318617372) bank81340 +81341 POINT(41.55605684598589 74.93111408354953) bank81341 +81342 POINT(40.44893168226448 74.94934551865788) bank81342 +81343 POINT(41.35916884207094 73.09182087703034) bank81343 +81344 POINT(41.23174020987038 74.40856793577255) bank81344 +81345 POINT(41.46241099391632 74.55107311149523) bank81345 +81346 POINT(40.20678029575075 74.43383705400089) bank81346 +81347 POINT(41.438527283823916 74.51967265248327) bank81347 +81348 POINT(41.3648867992828 74.76189387124455) bank81348 +81349 POINT(40.02695884461807 74.98117045049713) bank81349 +81350 POINT(41.58623121198345 74.67504071585714) bank81350 +81351 POINT(41.076171928458294 73.11782648937918) bank81351 +81352 POINT(41.508591268729965 74.8455798859931) bank81352 +81353 POINT(40.1353031659662 74.95041166119765) bank81353 +81354 POINT(41.01689289225887 74.59114784532959) bank81354 +81355 POINT(41.54610857932827 73.25150337986274) bank81355 +81356 POINT(41.024877954798754 73.3262466731624) bank81356 +81357 POINT(41.13407696655434 73.35696343614828) bank81357 +81358 POINT(40.208961132677345 74.88455868425342) bank81358 +81359 POINT(40.068222270706755 74.77206735483966) bank81359 +81360 POINT(40.760319584134216 74.29526466814522) bank81360 +81361 POINT(40.60734372081169 73.90211950766293) bank81361 +81362 POINT(39.82696466388265 73.94476008996183) bank81362 +81363 POINT(39.98911092624254 74.78997862850268) bank81363 +81364 POINT(40.529336125893174 74.88538688962464) bank81364 +81365 POINT(39.865410881915786 73.07381243014964) bank81365 +81366 POINT(39.92445641126073 73.8805896682629) bank81366 +81367 POINT(39.997824484532416 74.86519759872128) bank81367 +81368 POINT(41.7047140633197 73.57236236609607) bank81368 +81369 POINT(40.184380589861064 74.43022849286825) bank81369 +81370 POINT(39.91106564869136 74.32884880639375) bank81370 +81371 POINT(39.81360363804586 74.25252738852154) bank81371 +81372 POINT(41.1101879766292 73.7608037866676) bank81372 +81373 POINT(39.817122539147405 74.4862204634606) bank81373 +81374 POINT(41.45517628081899 74.27225661454986) bank81374 +81375 POINT(39.980618274601866 73.8986473137244) bank81375 +81376 POINT(39.778263672294614 74.7651499315432) bank81376 +81377 POINT(40.80357667619613 73.14299897398051) bank81377 +81378 POINT(41.67891537399936 73.2597181790858) bank81378 +81379 POINT(40.86876396359993 73.6202671481723) bank81379 +81380 POINT(41.045835651687845 74.69671099420754) bank81380 +81381 POINT(40.897031892154935 74.06658644145152) bank81381 +81382 POINT(40.036523382793554 74.00365061105168) bank81382 +81383 POINT(40.72606768647438 74.87649921379392) bank81383 +81384 POINT(41.53688209951989 73.8415580405126) bank81384 +81385 POINT(40.631786813071 74.75306492614746) bank81385 +81386 POINT(41.63900595894851 73.6462483025527) bank81386 +81387 POINT(41.70241754111728 73.26402316699019) bank81387 +81388 POINT(40.099727383138266 73.32582374250023) bank81388 +81389 POINT(40.70838784309266 73.32118471672715) bank81389 +81390 POINT(40.486251693853305 74.52945319102027) bank81390 +81391 POINT(39.98776489687653 73.42831980221025) bank81391 +81392 POINT(39.92266720910392 74.27953155502762) bank81392 +81393 POINT(40.91556674438564 73.09798352297858) bank81393 +81394 POINT(41.21153792435836 73.2325732990997) bank81394 +81395 POINT(40.34835554378628 74.71106343981137) bank81395 +81396 POINT(41.15312398327265 73.52074132647999) bank81396 +81397 POINT(39.788565213422146 73.51339827337279) bank81397 +81398 POINT(41.69451313684062 73.88016613126317) bank81398 +81399 POINT(41.10297216716214 74.96543703299905) bank81399 +81400 POINT(40.010477753054424 73.79034189242728) bank81400 +81401 POINT(40.45208521730396 74.83972146788646) bank81401 +81402 POINT(39.938072248005604 73.28063906881773) bank81402 +81403 POINT(40.91185074244107 74.85489301258416) bank81403 +81404 POINT(41.31406997315537 74.80878406312353) bank81404 +81405 POINT(39.85118631309944 73.44911205229045) bank81405 +81406 POINT(40.131044189709996 73.88469468262468) bank81406 +81407 POINT(39.8183954061469 74.70594314392144) bank81407 +81408 POINT(41.42151812008451 73.32601580991917) bank81408 +81409 POINT(40.49661826382328 74.19611556514529) bank81409 +81410 POINT(40.5111420490806 73.57627915785783) bank81410 +81411 POINT(40.661882728941535 74.0614197460919) bank81411 +81412 POINT(40.78633318808932 74.25558600269903) bank81412 +81413 POINT(40.76556471685915 74.5945156554948) bank81413 +81414 POINT(41.17353558448292 73.09777787001342) bank81414 +81415 POINT(39.89404576223209 74.0670961231734) bank81415 +81416 POINT(40.81007801338343 74.00686320409471) bank81416 +81417 POINT(41.10721600111183 73.63977559402434) bank81417 +81418 POINT(40.06610670447326 73.70564614054896) bank81418 +81419 POINT(40.88605669524238 73.03350742849126) bank81419 +81420 POINT(41.62803572030085 73.37997649495972) bank81420 +81421 POINT(40.88126612449361 73.52703027775514) bank81421 +81422 POINT(39.80365093965902 73.60322900705086) bank81422 +81423 POINT(40.300554881033285 73.90461659457864) bank81423 +81424 POINT(40.51349480556774 73.8762471412628) bank81424 +81425 POINT(41.188219491679774 74.6006279772208) bank81425 +81426 POINT(40.93315186342743 73.16329969981689) bank81426 +81427 POINT(41.12832118194112 74.98638320542706) bank81427 +81428 POINT(41.19351771822192 74.80395422871607) bank81428 +81429 POINT(41.71093418553602 74.40189784231238) bank81429 +81430 POINT(39.88029183817207 73.03625587522626) bank81430 +81431 POINT(40.78445585399206 73.74969175127451) bank81431 +81432 POINT(40.610984248544575 73.14861041019391) bank81432 +81433 POINT(41.08637748256593 74.86024000001706) bank81433 +81434 POINT(40.763161437120466 74.71904839283243) bank81434 +81435 POINT(41.66069609892984 74.65868842007487) bank81435 +81436 POINT(40.828838683472625 74.04330284989842) bank81436 +81437 POINT(40.641993768358425 73.39319326416467) bank81437 +81438 POINT(39.73339464256851 73.0536311321599) bank81438 +81439 POINT(41.29508578763277 73.69445450135782) bank81439 +81440 POINT(40.70795057209499 74.42192531448059) bank81440 +81441 POINT(40.47841107220879 73.43007722562425) bank81441 +81442 POINT(39.768086914830775 74.96920083614803) bank81442 +81443 POINT(39.87648431521211 73.54970753653947) bank81443 +81444 POINT(40.36862763172549 73.17383557048825) bank81444 +81445 POINT(39.84183688405848 74.7456938486753) bank81445 +81446 POINT(41.338563305874054 74.90519828233249) bank81446 +81447 POINT(41.59541185551488 74.14679556430042) bank81447 +81448 POINT(41.031773861243515 74.1059597656967) bank81448 +81449 POINT(39.72566778101776 74.31882495883963) bank81449 +81450 POINT(40.664126528453586 73.64836814796654) bank81450 +81451 POINT(40.09635548441024 73.33560213588328) bank81451 +81452 POINT(41.35435115918739 74.46651941419434) bank81452 +81453 POINT(39.74041664166111 73.18408578278971) bank81453 +81454 POINT(41.45458725366755 73.19252980446628) bank81454 +81455 POINT(40.9974103997844 74.16642567974364) bank81455 +81456 POINT(39.82761658270577 73.05126122527248) bank81456 +81457 POINT(40.187373127279805 74.4001070183796) bank81457 +81458 POINT(41.36457174775034 74.65346859272866) bank81458 +81459 POINT(40.58860526664119 73.67333444850051) bank81459 +81460 POINT(40.737422587048464 73.53293213053244) bank81460 +81461 POINT(40.97691571267337 73.21629470744605) bank81461 +81462 POINT(40.135356380369544 73.10229554132451) bank81462 +81463 POINT(41.08088468474946 73.46073640652918) bank81463 +81464 POINT(40.66653643106824 73.01711908451503) bank81464 +81465 POINT(41.32490389881956 74.65193644158585) bank81465 +81466 POINT(40.204657168601265 74.28657945315655) bank81466 +81467 POINT(41.28691200748948 74.3228653685892) bank81467 +81468 POINT(41.33505783719819 74.60021818736878) bank81468 +81469 POINT(39.931513357344166 73.81720037054254) bank81469 +81470 POINT(40.25066270143908 73.25776207809488) bank81470 +81471 POINT(40.40344183695401 73.23815494049735) bank81471 +81472 POINT(41.630096509476246 73.51661243275805) bank81472 +81473 POINT(41.619684997500286 74.66615782095455) bank81473 +81474 POINT(40.91239582158332 74.24678371031263) bank81474 +81475 POINT(40.25531862887847 73.7344689612239) bank81475 +81476 POINT(41.67388846672033 73.35762043999773) bank81476 +81477 POINT(40.290062056875314 73.84240113753414) bank81477 +81478 POINT(40.98826469121763 74.37892700057438) bank81478 +81479 POINT(39.97050007425248 74.84157404879235) bank81479 +81480 POINT(41.10613864760396 73.65239124334165) bank81480 +81481 POINT(39.89336884047937 73.06191820664422) bank81481 +81482 POINT(40.543623841958656 73.88645946803526) bank81482 +81483 POINT(40.77652266481395 73.93459713378132) bank81483 +81484 POINT(41.549282635870334 74.71849396416337) bank81484 +81485 POINT(40.86552967820814 73.71365131665198) bank81485 +81486 POINT(41.54727743248859 74.52445749446527) bank81486 +81487 POINT(41.20040231686114 74.21767566366616) bank81487 +81488 POINT(41.62052400622149 73.84810905242728) bank81488 +81489 POINT(40.42155981824259 73.82121781622304) bank81489 +81490 POINT(41.58719591259585 74.44596484621022) bank81490 +81491 POINT(40.55414438413457 74.46062915896438) bank81491 +81492 POINT(40.79634000017676 73.82269408373597) bank81492 +81493 POINT(41.19763918653988 73.52600692575925) bank81493 +81494 POINT(41.223963230348716 74.94036518178962) bank81494 +81495 POINT(39.88968721063368 73.30204052042139) bank81495 +81496 POINT(41.23608684472574 73.71021342877708) bank81496 +81497 POINT(40.88235940085677 74.18781532641162) bank81497 +81498 POINT(41.653929967446295 74.11261381692641) bank81498 +81499 POINT(39.93483691658564 74.94435312191838) bank81499 +81500 POINT(40.11626042898057 74.11915103870884) bank81500 +81501 POINT(40.79407361466711 73.43345095663601) bank81501 +81502 POINT(40.89536509405341 73.67114047642355) bank81502 +81503 POINT(40.31117667878448 74.43780322732762) bank81503 +81504 POINT(41.49251684700773 74.40991704782485) bank81504 +81505 POINT(41.49265255089061 73.87282446807814) bank81505 +81506 POINT(41.497013481725475 74.66670830626656) bank81506 +81507 POINT(40.22217023814725 73.06315759161087) bank81507 +81508 POINT(40.61138742278308 74.4948523842994) bank81508 +81509 POINT(40.5068161955228 73.58192023567103) bank81509 +81510 POINT(40.57589147272273 73.90130727662861) bank81510 +81511 POINT(41.238979571425766 74.41631523696577) bank81511 +81512 POINT(39.97284850505877 73.3138052536943) bank81512 +81513 POINT(40.16078674253077 74.22671949475554) bank81513 +81514 POINT(40.418520877682255 74.83565837218626) bank81514 +81515 POINT(41.51429251539645 74.48498445181882) bank81515 +81516 POINT(41.20020308574057 73.14048538780905) bank81516 +81517 POINT(41.583534591226396 73.85554325308884) bank81517 +81518 POINT(40.96128847505422 73.38996787420851) bank81518 +81519 POINT(40.781475718357534 73.37986115101074) bank81519 +81520 POINT(40.77838337081459 74.97348547042893) bank81520 +81521 POINT(39.86902788653747 73.07313826665613) bank81521 +81522 POINT(41.32427002936324 74.24950070233635) bank81522 +81523 POINT(40.61255944264053 73.24356298650953) bank81523 +81524 POINT(40.49727798851951 73.32504168207939) bank81524 +81525 POINT(40.036231535186644 74.76748125155426) bank81525 +81526 POINT(40.20257285061188 73.43833005341553) bank81526 +81527 POINT(39.80674571385807 74.58058443882875) bank81527 +81528 POINT(41.57115591451488 74.16864429599309) bank81528 +81529 POINT(40.545859137776105 73.53717264941302) bank81529 +81530 POINT(40.096690083036094 73.27006653667883) bank81530 +81531 POINT(40.472317293817184 73.05253177500616) bank81531 +81532 POINT(41.48463019968088 73.13809641820502) bank81532 +81533 POINT(40.713801869282314 74.45706090286527) bank81533 +81534 POINT(39.733269996556324 74.19272547041324) bank81534 +81535 POINT(39.732430552983836 73.88062354095564) bank81535 +81536 POINT(41.21233800918394 74.73484303625905) bank81536 +81537 POINT(41.11409865496747 74.35633502972149) bank81537 +81538 POINT(40.18621948046987 74.7461631819949) bank81538 +81539 POINT(40.65459069318515 73.35176719474566) bank81539 +81540 POINT(40.51955650584553 73.31453296929101) bank81540 +81541 POINT(40.1082294503652 74.66826666260485) bank81541 +81542 POINT(41.1434619516577 73.72427576731285) bank81542 +81543 POINT(41.05804527775397 73.48997918767493) bank81543 +81544 POINT(41.63395177259501 74.10747607298212) bank81544 +81545 POINT(40.7472064372947 74.28668053025886) bank81545 +81546 POINT(40.551826174432556 73.96858687306334) bank81546 +81547 POINT(41.1778713242762 74.5823393892066) bank81547 +81548 POINT(40.32998541179404 73.05067296126872) bank81548 +81549 POINT(41.51894244663686 73.33634215991555) bank81549 +81550 POINT(41.26585461385503 73.9179526729128) bank81550 +81551 POINT(40.608444614017 74.42848993018848) bank81551 +81552 POINT(40.529695030809435 74.48462122082636) bank81552 +81553 POINT(39.84338961651449 74.40631895131033) bank81553 +81554 POINT(40.43953729225881 73.19688569173299) bank81554 +81555 POINT(41.55039624581324 74.16688894627987) bank81555 +81556 POINT(40.54449077346084 73.4758504277774) bank81556 +81557 POINT(41.33305515852609 73.84723057016737) bank81557 +81558 POINT(41.532889265665865 74.29786625376099) bank81558 +81559 POINT(41.193130107843594 74.89863297547498) bank81559 +81560 POINT(41.40806357163157 74.20688914795483) bank81560 +81561 POINT(41.421275494526974 74.28578162442065) bank81561 +81562 POINT(40.03137171817984 73.62799341793273) bank81562 +81563 POINT(39.903450025546114 73.91140353319815) bank81563 +81564 POINT(40.80814375832197 74.11272239570515) bank81564 +81565 POINT(40.770813107433966 74.07132272807266) bank81565 +81566 POINT(40.88761585088347 74.85438528057091) bank81566 +81567 POINT(40.04987696852776 73.46163544765186) bank81567 +81568 POINT(41.66489781622019 74.27516234543604) bank81568 +81569 POINT(41.66892081141657 74.20447589296754) bank81569 +81570 POINT(40.80915585965909 73.76916474214342) bank81570 +81571 POINT(39.7773842958358 74.73359584676658) bank81571 +81572 POINT(40.26412571973126 74.50506281417528) bank81572 +81573 POINT(40.78264170644212 73.96286910017602) bank81573 +81574 POINT(40.89089590935818 73.1721326855036) bank81574 +81575 POINT(41.07698589376357 74.34955602794531) bank81575 +81576 POINT(41.384971939677094 73.14726209585304) bank81576 +81577 POINT(41.10114740719836 73.66195739813033) bank81577 +81578 POINT(40.42986934816018 73.41959764910926) bank81578 +81579 POINT(41.261861579236424 73.2298531229394) bank81579 +81580 POINT(39.80685030183161 74.52517806310445) bank81580 +81581 POINT(41.64246161056394 73.70602983171297) bank81581 +81582 POINT(41.25740946257219 73.62677521322277) bank81582 +81583 POINT(41.12948066424009 73.82977026892256) bank81583 +81584 POINT(40.82456403673873 74.34859546287565) bank81584 +81585 POINT(39.75273157305524 73.40230018024823) bank81585 +81586 POINT(40.65626254362719 74.49648861960787) bank81586 +81587 POINT(40.03086064701756 73.08076443596028) bank81587 +81588 POINT(40.27454870070156 74.62727043565076) bank81588 +81589 POINT(41.31852843368372 73.54780809359207) bank81589 +81590 POINT(41.5820608521513 73.05129157271476) bank81590 +81591 POINT(40.681449047196686 74.09563442398007) bank81591 +81592 POINT(39.9481448586939 73.92106381304144) bank81592 +81593 POINT(41.54933250127199 73.75048492017139) bank81593 +81594 POINT(40.91345763323144 73.46572304359962) bank81594 +81595 POINT(39.96071854212868 74.84465137150944) bank81595 +81596 POINT(40.16484338971822 73.44864638562336) bank81596 +81597 POINT(40.97985939917952 73.21471141255083) bank81597 +81598 POINT(41.07207697068081 73.69874888926344) bank81598 +81599 POINT(41.458240971731726 73.3898576018495) bank81599 +81600 POINT(40.20070765752015 73.43651347984105) bank81600 +81601 POINT(40.47844462460674 74.2243563781822) bank81601 +81602 POINT(40.845266516129456 73.87877527840054) bank81602 +81603 POINT(39.867622631875705 74.96549884753087) bank81603 +81604 POINT(40.338796533430234 74.86626414230132) bank81604 +81605 POINT(41.41794903321674 73.15203218187509) bank81605 +81606 POINT(41.46203261603017 74.16755527296161) bank81606 +81607 POINT(41.275830478341184 74.44237555532132) bank81607 +81608 POINT(40.076306952820254 73.27139950762796) bank81608 +81609 POINT(40.037264755807335 74.3724400393177) bank81609 +81610 POINT(40.38702479834644 74.43728702537399) bank81610 +81611 POINT(40.822857675619176 74.40097607774962) bank81611 +81612 POINT(41.16113322277284 73.60878866038122) bank81612 +81613 POINT(41.13197978938148 73.66183286829457) bank81613 +81614 POINT(40.924139691740855 74.4495962165723) bank81614 +81615 POINT(39.75218893404563 73.02554403702905) bank81615 +81616 POINT(39.833551809423845 73.63301465977503) bank81616 +81617 POINT(41.3487786583167 74.43103913350187) bank81617 +81618 POINT(41.26935026034614 74.53029386358631) bank81618 +81619 POINT(39.874757990617695 73.94460731511329) bank81619 +81620 POINT(40.13125869212802 74.66050590728312) bank81620 +81621 POINT(40.235521905866165 74.57441853753141) bank81621 +81622 POINT(40.69967908826304 73.8038531002016) bank81622 +81623 POINT(41.49362595550703 74.24417063399851) bank81623 +81624 POINT(39.85751832239613 73.32665482797637) bank81624 +81625 POINT(41.45241474207218 74.35771610194818) bank81625 +81626 POINT(41.000768280685854 73.44076879079492) bank81626 +81627 POINT(41.51212774658768 74.09638047888086) bank81627 +81628 POINT(40.68454177513823 74.11688348685342) bank81628 +81629 POINT(41.6292966696231 74.8407766741622) bank81629 +81630 POINT(40.45908380151267 73.53482563098642) bank81630 +81631 POINT(41.106690710517235 74.30798003085796) bank81631 +81632 POINT(40.64147214532926 74.18084259213094) bank81632 +81633 POINT(41.29390129673943 73.22954169671237) bank81633 +81634 POINT(40.26339373728433 73.3852910214847) bank81634 +81635 POINT(41.048487055131 74.5154767478025) bank81635 +81636 POINT(40.22789813017121 73.80557520195451) bank81636 +81637 POINT(41.520375100390154 74.90630048663073) bank81637 +81638 POINT(40.335285479917125 73.86672952279349) bank81638 +81639 POINT(41.61024577661065 74.60385844901673) bank81639 +81640 POINT(39.81250542819925 74.61486462338576) bank81640 +81641 POINT(40.623638845063496 74.69808376539058) bank81641 +81642 POINT(41.614548804258106 73.87236427732064) bank81642 +81643 POINT(39.883785070225166 74.48991479567326) bank81643 +81644 POINT(39.87371817894921 74.75556231206052) bank81644 +81645 POINT(41.33349170146887 74.30521089093526) bank81645 +81646 POINT(41.49120660158169 73.64976148651132) bank81646 +81647 POINT(40.45408005752736 73.12836407836463) bank81647 +81648 POINT(41.340268091291684 74.0310386491276) bank81648 +81649 POINT(41.21849016056419 74.48339497879226) bank81649 +81650 POINT(40.98705630183441 74.75340307049142) bank81650 +81651 POINT(41.670257037215954 74.73637326033085) bank81651 +81652 POINT(39.73466755733077 73.08380179336659) bank81652 +81653 POINT(39.8517486242348 73.5057444329305) bank81653 +81654 POINT(40.94616273491003 74.62648423986556) bank81654 +81655 POINT(40.68700863975546 73.39247448578097) bank81655 +81656 POINT(41.18257322695378 74.61702159994682) bank81656 +81657 POINT(40.02172179551282 74.72393341302495) bank81657 +81658 POINT(41.555908458260525 73.82797539847526) bank81658 +81659 POINT(41.483531065053064 73.70930125563535) bank81659 +81660 POINT(39.83426967953106 73.77774529793501) bank81660 +81661 POINT(40.29932085174834 73.26754677988261) bank81661 +81662 POINT(41.60028714003676 74.4564258133463) bank81662 +81663 POINT(40.31272661757183 73.25112628072274) bank81663 +81664 POINT(41.50325044987294 75.00258041773276) bank81664 +81665 POINT(39.7578316630229 74.13012573491463) bank81665 +81666 POINT(40.52404975759267 73.64672441834668) bank81666 +81667 POINT(41.0657446572509 74.87299177042689) bank81667 +81668 POINT(40.19872406203792 73.71424393518832) bank81668 +81669 POINT(41.4536380182547 73.52746493856036) bank81669 +81670 POINT(40.70784615877975 74.49188902016218) bank81670 +81671 POINT(39.790663363726466 73.45800371093395) bank81671 +81672 POINT(41.552359319817306 73.58750799863233) bank81672 +81673 POINT(41.16822317985105 74.18178182644225) bank81673 +81674 POINT(39.735820307543214 74.66457714408287) bank81674 +81675 POINT(39.84160272262851 74.5770921944446) bank81675 +81676 POINT(41.04317489712388 73.02910973276389) bank81676 +81677 POINT(41.52271248643976 74.31465573211203) bank81677 +81678 POINT(40.51481049228012 73.07157310546827) bank81678 +81679 POINT(39.921167359438044 74.86674305328641) bank81679 +81680 POINT(41.665536329153184 73.28284742374987) bank81680 +81681 POINT(41.33872722151558 74.17727816934917) bank81681 +81682 POINT(41.24145295577147 73.41788188426894) bank81682 +81683 POINT(40.8482286264493 74.79693211451544) bank81683 +81684 POINT(41.49459307481667 74.31550829794865) bank81684 +81685 POINT(41.31811606325244 73.37130639768216) bank81685 +81686 POINT(40.14959376812096 74.31129721726221) bank81686 +81687 POINT(40.18550654197482 73.10709333332133) bank81687 +81688 POINT(41.5324118756438 73.51395433524486) bank81688 +81689 POINT(41.315254587680606 74.62037390257562) bank81689 +81690 POINT(39.95095161461492 74.99064763000382) bank81690 +81691 POINT(40.389319846988684 73.26794035757699) bank81691 +81692 POINT(41.53723191885649 74.32973996176943) bank81692 +81693 POINT(40.97421523917903 73.4310385846975) bank81693 +81694 POINT(40.30126484571914 74.50318028408101) bank81694 +81695 POINT(40.08412207128623 73.55723183847132) bank81695 +81696 POINT(41.179796386041495 73.65657141798516) bank81696 +81697 POINT(39.78277551743208 74.78383200465072) bank81697 +81698 POINT(39.825665911573495 74.24883246672758) bank81698 +81699 POINT(40.74613161014407 73.41913228516543) bank81699 +81700 POINT(40.74246122599445 74.47613397216297) bank81700 +81701 POINT(40.85708580215376 73.2740932537189) bank81701 +81702 POINT(40.452492885390456 73.03269251960694) bank81702 +81703 POINT(39.85096439175975 73.93913462741476) bank81703 +81704 POINT(40.77952424399056 73.3921987814256) bank81704 +81705 POINT(40.95484074231853 73.62928927414904) bank81705 +81706 POINT(40.34888914457969 73.79251838948205) bank81706 +81707 POINT(40.99984978153778 74.95902749476804) bank81707 +81708 POINT(41.65692417957977 74.33013506327528) bank81708 +81709 POINT(40.85179773086399 74.76270084108187) bank81709 +81710 POINT(39.98762051633146 73.01564199036346) bank81710 +81711 POINT(40.83682426656202 74.69981826146767) bank81711 +81712 POINT(40.58251830714496 73.18756767022694) bank81712 +81713 POINT(40.904583121674236 74.62764504919325) bank81713 +81714 POINT(39.909608113091274 74.98620491759962) bank81714 +81715 POINT(39.94547545243282 73.43652990351612) bank81715 +81716 POINT(40.79123859388469 74.19390178083478) bank81716 +81717 POINT(40.01320257205829 74.91590730333174) bank81717 +81718 POINT(41.03625341164761 73.23161465698152) bank81718 +81719 POINT(40.349047120281696 73.69242382328432) bank81719 +81720 POINT(41.621491935951155 74.42379786707804) bank81720 +81721 POINT(40.732119264267055 73.39895757607796) bank81721 +81722 POINT(41.48096622896411 73.21759108836045) bank81722 +81723 POINT(40.88932781723152 73.74014008708619) bank81723 +81724 POINT(41.19071956005767 74.90573246179943) bank81724 +81725 POINT(40.10971090544458 73.54260081027584) bank81725 +81726 POINT(41.66498950341322 74.93094648230279) bank81726 +81727 POINT(41.351174419361946 73.05846420413542) bank81727 +81728 POINT(41.547076905111325 73.21424274606119) bank81728 +81729 POINT(41.32700189177013 73.9876916610194) bank81729 +81730 POINT(41.652901587077324 74.85433356803847) bank81730 +81731 POINT(39.96883582943231 73.9282392167305) bank81731 +81732 POINT(41.32889883196093 73.10574980182938) bank81732 +81733 POINT(41.70424492028908 74.23854229393051) bank81733 +81734 POINT(40.66964707091341 74.46548172973837) bank81734 +81735 POINT(40.363332917435095 74.78287584689534) bank81735 +81736 POINT(41.63814449273824 73.08958197681275) bank81736 +81737 POINT(40.72365830879435 74.93122669317242) bank81737 +81738 POINT(40.95025916994486 73.69254348551453) bank81738 +81739 POINT(40.28881644757354 73.34289191900632) bank81739 +81740 POINT(40.9920937961576 74.68714937307006) bank81740 +81741 POINT(40.53615205487605 74.41202644779119) bank81741 +81742 POINT(40.835461964266905 73.5442662032739) bank81742 +81743 POINT(40.30555155181386 73.19417468505272) bank81743 +81744 POINT(41.1346441775438 73.38251527839171) bank81744 +81745 POINT(41.55780646916455 73.79078193177044) bank81745 +81746 POINT(40.87926965314066 73.03474209355133) bank81746 +81747 POINT(40.451600081689406 74.60798051090292) bank81747 +81748 POINT(40.089761405085454 74.47692952261707) bank81748 +81749 POINT(40.505021274338844 73.90383575175609) bank81749 +81750 POINT(40.450282364182485 73.48081384902676) bank81750 +81751 POINT(40.576363025786684 73.54927428775923) bank81751 +81752 POINT(40.87413391550879 74.36590900360277) bank81752 +81753 POINT(40.816818379577676 73.78599381655752) bank81753 +81754 POINT(40.85972861881391 73.07064218377374) bank81754 +81755 POINT(41.46134689318086 73.7759961750287) bank81755 +81756 POINT(41.38524899931974 73.27874679418485) bank81756 +81757 POINT(40.430201341473314 74.17975209916075) bank81757 +81758 POINT(41.420145267979144 73.5004523553363) bank81758 +81759 POINT(39.8956776736878 73.04958626166922) bank81759 +81760 POINT(41.396477336516 73.85693529294694) bank81760 +81761 POINT(39.8201259035696 73.52783155841935) bank81761 +81762 POINT(41.292234514424294 74.23013286730347) bank81762 +81763 POINT(40.61494478236997 73.66081546578425) bank81763 +81764 POINT(41.55536598392459 73.48758845949901) bank81764 +81765 POINT(41.6200709169219 74.72950644236369) bank81765 +81766 POINT(41.69473398364322 74.8734760222302) bank81766 +81767 POINT(40.62731267040606 74.24413856275778) bank81767 +81768 POINT(41.37462835988846 73.4825852804415) bank81768 +81769 POINT(40.38010189368591 73.12633481206875) bank81769 +81770 POINT(41.59099416434674 74.42953285763787) bank81770 +81771 POINT(41.66225283015025 74.69943907508576) bank81771 +81772 POINT(39.92562551345334 73.68034856379987) bank81772 +81773 POINT(40.20248987226756 73.29420278428265) bank81773 +81774 POINT(41.438910413026555 74.99082163600575) bank81774 +81775 POINT(41.208726699400984 74.58687647177271) bank81775 +81776 POINT(40.86865670960155 73.36638944949834) bank81776 +81777 POINT(39.80369739479176 73.74171208294285) bank81777 +81778 POINT(40.81791499757439 74.04311201174623) bank81778 +81779 POINT(40.500288306072655 73.90731462504833) bank81779 +81780 POINT(40.749969400602886 74.00427863107132) bank81780 +81781 POINT(40.06456505033686 73.78772986517187) bank81781 +81782 POINT(40.0428200139955 73.10063723847361) bank81782 +81783 POINT(39.830166706205674 73.73463049078914) bank81783 +81784 POINT(40.53361143003888 74.71711754912059) bank81784 +81785 POINT(41.652062229400194 74.57682750126884) bank81785 +81786 POINT(40.34821710529771 73.8874367396636) bank81786 +81787 POINT(40.714113896106 74.30004612078359) bank81787 +81788 POINT(41.251753857857544 73.04300172153677) bank81788 +81789 POINT(41.05568108830129 73.36137320300017) bank81789 +81790 POINT(41.69675998096431 74.53188608738864) bank81790 +81791 POINT(41.22699072889686 74.51165422600467) bank81791 +81792 POINT(39.96641013733 73.74264660763012) bank81792 +81793 POINT(39.82319681141237 74.03204914541064) bank81793 +81794 POINT(41.11799455480975 73.65817847069609) bank81794 +81795 POINT(40.72817858421367 73.83419266529455) bank81795 +81796 POINT(41.207373898818844 74.58066314624654) bank81796 +81797 POINT(41.26349127569057 74.57512588381063) bank81797 +81798 POINT(40.779565693156094 74.33639926724592) bank81798 +81799 POINT(39.91341050734255 73.58998591658755) bank81799 +81800 POINT(41.471722825112515 74.2192471882368) bank81800 +81801 POINT(40.80272993365458 74.75642018478098) bank81801 +81802 POINT(39.91415598252319 73.15135801397594) bank81802 +81803 POINT(40.06291673757791 73.02599795069742) bank81803 +81804 POINT(41.68897097086546 73.20124072585897) bank81804 +81805 POINT(41.43326754162925 74.56709967293764) bank81805 +81806 POINT(39.79498767350805 73.98848945694952) bank81806 +81807 POINT(41.68101333966645 74.07590670966815) bank81807 +81808 POINT(40.08502637881049 74.73392939773342) bank81808 +81809 POINT(40.67263709139771 74.2980362585411) bank81809 +81810 POINT(41.01165626970413 73.8851132332784) bank81810 +81811 POINT(39.733031113678365 74.50963585771895) bank81811 +81812 POINT(41.40507888002423 74.78640858653286) bank81812 +81813 POINT(39.82558230358367 74.71665344649345) bank81813 +81814 POINT(40.12469306766294 74.5311163799621) bank81814 +81815 POINT(39.74710085225936 73.34675840987192) bank81815 +81816 POINT(40.913025618917004 74.08639480659488) bank81816 +81817 POINT(41.26976132245475 74.73906080615016) bank81817 +81818 POINT(41.07382594563796 73.06498663554403) bank81818 +81819 POINT(41.70705300001003 74.41176434684441) bank81819 +81820 POINT(40.97345697961107 73.34346818517002) bank81820 +81821 POINT(40.51570356570223 73.74139321990037) bank81821 +81822 POINT(40.06079701298764 74.5771989714327) bank81822 +81823 POINT(41.07377832464844 74.17479968696554) bank81823 +81824 POINT(40.924464538413666 73.38304615221881) bank81824 +81825 POINT(40.01603402927726 73.5570983275786) bank81825 +81826 POINT(40.877181153420125 73.20797777946495) bank81826 +81827 POINT(40.622658042266416 74.41089758523088) bank81827 +81828 POINT(39.72256619127601 73.75484901263373) bank81828 +81829 POINT(41.33646724425253 74.66116894876488) bank81829 +81830 POINT(41.52265436290321 74.96442490055746) bank81830 +81831 POINT(39.965921477295566 74.90013332365146) bank81831 +81832 POINT(41.46719135842329 74.19491121200201) bank81832 +81833 POINT(40.174996373074414 74.78511054981762) bank81833 +81834 POINT(40.61600935285995 73.27793251207798) bank81834 +81835 POINT(40.395653633966035 73.85927492375207) bank81835 +81836 POINT(41.104099076894805 73.42035465921492) bank81836 +81837 POINT(41.39396627323418 74.68789528140763) bank81837 +81838 POINT(41.645105327563655 73.03467539163533) bank81838 +81839 POINT(40.62591337195008 74.85513406979898) bank81839 +81840 POINT(40.651099836338744 74.913316561867) bank81840 +81841 POINT(40.290993087131355 74.6566357627172) bank81841 +81842 POINT(40.10856252381823 74.89821210569909) bank81842 +81843 POINT(40.098259817544005 74.97679904778649) bank81843 +81844 POINT(41.33777971190474 73.8738876816507) bank81844 +81845 POINT(40.69927136663237 74.11228998615557) bank81845 +81846 POINT(41.376713241136926 73.6882379725607) bank81846 +81847 POINT(41.298397852883966 74.09881807149401) bank81847 +81848 POINT(41.64972497680036 74.47602267268275) bank81848 +81849 POINT(39.99342853316857 74.0140865408481) bank81849 +81850 POINT(40.84112238563985 74.40627556691442) bank81850 +81851 POINT(40.333013826204024 73.56382482494442) bank81851 +81852 POINT(40.53526105339892 74.19108222456539) bank81852 +81853 POINT(40.523700048059254 73.79549696236239) bank81853 +81854 POINT(40.794790688527144 74.67513611464338) bank81854 +81855 POINT(40.296588027956226 73.84105624962169) bank81855 +81856 POINT(40.99920863115571 74.43887638058887) bank81856 +81857 POINT(40.77929399257443 73.93988735220907) bank81857 +81858 POINT(40.73278455831017 73.87079992286885) bank81858 +81859 POINT(40.211827729520145 74.3787654263182) bank81859 +81860 POINT(41.06137436485094 74.05460030169105) bank81860 +81861 POINT(41.50804325804035 73.24184219739743) bank81861 +81862 POINT(41.442156643785246 74.72804460265233) bank81862 +81863 POINT(41.41734209339125 73.43200428046565) bank81863 +81864 POINT(39.81307006857842 73.15217867521204) bank81864 +81865 POINT(40.46649245767115 74.77873202946752) bank81865 +81866 POINT(39.959500352662786 74.92153970216356) bank81866 +81867 POINT(40.17421420918207 73.03602205520632) bank81867 +81868 POINT(40.51882120165614 74.72724662019314) bank81868 +81869 POINT(39.84840125372325 73.43105785519984) bank81869 +81870 POINT(39.911063930530325 73.20559624947137) bank81870 +81871 POINT(39.71908088501171 74.67474829908103) bank81871 +81872 POINT(40.9493917996825 73.18420053188623) bank81872 +81873 POINT(41.03485158222598 74.78997525618368) bank81873 +81874 POINT(39.985885350571365 74.4765571557161) bank81874 +81875 POINT(40.754641510387536 74.1167402735165) bank81875 +81876 POINT(39.7158449193435 74.47980581675287) bank81876 +81877 POINT(40.10413288788169 73.3051979921857) bank81877 +81878 POINT(40.88600171332613 74.51953095201819) bank81878 +81879 POINT(40.00244007469723 73.5728620611424) bank81879 +81880 POINT(40.99602099551448 73.78489550291647) bank81880 +81881 POINT(41.24531645525704 74.87737604640904) bank81881 +81882 POINT(40.52603813949668 74.25937583266958) bank81882 +81883 POINT(39.914018134361186 74.88790619709765) bank81883 +81884 POINT(41.0235560125876 74.4770097392418) bank81884 +81885 POINT(40.5687960629339 73.05990884238419) bank81885 +81886 POINT(40.02310709695969 73.06585606172915) bank81886 +81887 POINT(40.416303322057324 74.19165585030765) bank81887 +81888 POINT(40.17120919766221 73.76169710674121) bank81888 +81889 POINT(40.00990674992535 73.27747210760029) bank81889 +81890 POINT(40.377054249623626 74.04853656554637) bank81890 +81891 POINT(40.653506003093796 74.24687815436751) bank81891 +81892 POINT(39.94680692088497 74.92874702768626) bank81892 +81893 POINT(40.16703696613032 73.36133689252365) bank81893 +81894 POINT(41.54027815405143 73.53168095921777) bank81894 +81895 POINT(40.287335956287755 73.62178526111924) bank81895 +81896 POINT(40.47027873703939 73.81328389874103) bank81896 +81897 POINT(40.7829989858432 74.54766794209768) bank81897 +81898 POINT(40.98467189228532 74.51272134358095) bank81898 +81899 POINT(39.82397546553002 74.4810593019876) bank81899 +81900 POINT(40.7142800197028 74.77749628630644) bank81900 +81901 POINT(40.2325617836946 73.5712292674876) bank81901 +81902 POINT(41.48915422380176 73.55685912747539) bank81902 +81903 POINT(40.31701833475108 73.40569380565245) bank81903 +81904 POINT(40.665777241311574 73.54059687100874) bank81904 +81905 POINT(40.5241825049778 74.32466868902947) bank81905 +81906 POINT(39.9854819632093 74.97803486882147) bank81906 +81907 POINT(40.643856736384286 74.71757939299795) bank81907 +81908 POINT(41.289544146947776 73.44173356160567) bank81908 +81909 POINT(40.14523049757748 74.35628104043431) bank81909 +81910 POINT(40.36278764131968 73.66289091493545) bank81910 +81911 POINT(41.28022199159795 74.25384818045671) bank81911 +81912 POINT(40.59286400828429 74.1703923742934) bank81912 +81913 POINT(41.461906928951684 73.9075090177234) bank81913 +81914 POINT(40.536884942616254 73.31523118068206) bank81914 +81915 POINT(41.45449154176651 74.51169354816673) bank81915 +81916 POINT(40.74641164629975 73.89994274407987) bank81916 +81917 POINT(40.129076895397446 74.90747628819145) bank81917 +81918 POINT(40.0525599307419 73.66754291574775) bank81918 +81919 POINT(40.76851747452488 74.56389249540378) bank81919 +81920 POINT(41.49416386076368 73.53355080881015) bank81920 +81921 POINT(41.37349041196643 74.2261422540645) bank81921 +81922 POINT(40.85833935916182 74.34061770294187) bank81922 +81923 POINT(39.88423088360491 73.68371720892426) bank81923 +81924 POINT(40.79732551987982 74.40560900282672) bank81924 +81925 POINT(41.59928046855489 73.59746556757263) bank81925 +81926 POINT(39.78323323846746 73.4190425136059) bank81926 +81927 POINT(40.88968677059187 73.18165984642475) bank81927 +81928 POINT(39.96728432369669 74.13077672911975) bank81928 +81929 POINT(40.0849241483443 74.39454844458832) bank81929 +81930 POINT(41.3670033401714 74.5198123789037) bank81930 +81931 POINT(40.04117733975279 74.20037579194448) bank81931 +81932 POINT(39.991157183530945 74.22398050481439) bank81932 +81933 POINT(39.86796700228428 73.11881094022418) bank81933 +81934 POINT(40.913073296153584 74.0165325936854) bank81934 +81935 POINT(39.80105216447578 73.0468385349706) bank81935 +81936 POINT(39.99559017096435 74.26029264004784) bank81936 +81937 POINT(41.439261107543366 73.65422280138979) bank81937 +81938 POINT(40.358296583029926 74.56672756925208) bank81938 +81939 POINT(40.84660296804839 74.25156594573875) bank81939 +81940 POINT(40.99453308228557 74.17701732585111) bank81940 +81941 POINT(40.38275821553846 74.49815621775818) bank81941 +81942 POINT(40.31973871822893 74.16671412806171) bank81942 +81943 POINT(40.8684577892193 74.74930893211804) bank81943 +81944 POINT(40.137303081097365 74.38127713121754) bank81944 +81945 POINT(39.87080799672054 74.81289427416907) bank81945 +81946 POINT(41.64840966044592 73.52675933037669) bank81946 +81947 POINT(41.31646365094983 74.88014756168842) bank81947 +81948 POINT(41.41928571136664 74.46235620494033) bank81948 +81949 POINT(40.40927327114529 73.63681529824561) bank81949 +81950 POINT(41.166076600919105 73.75825601003352) bank81950 +81951 POINT(40.58265835222613 73.24733975921015) bank81951 +81952 POINT(40.833743346149504 74.41978983425562) bank81952 +81953 POINT(40.164689400988 74.21734717476049) bank81953 +81954 POINT(39.791956985335595 74.29180131396566) bank81954 +81955 POINT(40.673243746298155 74.2728321745805) bank81955 +81956 POINT(41.28840410878277 73.39128495817742) bank81956 +81957 POINT(41.480676299813545 74.23658976749111) bank81957 +81958 POINT(41.443227402986764 74.38680085741774) bank81958 +81959 POINT(39.78599443496951 73.5300945659224) bank81959 +81960 POINT(40.86685681876462 73.7872926110251) bank81960 +81961 POINT(41.0168464870231 74.93941863175604) bank81961 +81962 POINT(41.01157802021169 73.18440715494725) bank81962 +81963 POINT(41.09764470293397 74.63225759485437) bank81963 +81964 POINT(40.23130221020441 73.38772620097761) bank81964 +81965 POINT(40.248580276988946 74.53120735829309) bank81965 +81966 POINT(41.42489125534521 73.91373518378076) bank81966 +81967 POINT(40.17301519446006 74.59229674959839) bank81967 +81968 POINT(41.25706459595634 74.73940084333488) bank81968 +81969 POINT(40.3530751558721 74.01446560414696) bank81969 +81970 POINT(40.21128993373584 74.21965076613435) bank81970 +81971 POINT(41.57887386716921 74.13827959041815) bank81971 +81972 POINT(40.680954091641446 74.01234844948937) bank81972 +81973 POINT(41.184961806775654 74.5105174953381) bank81973 +81974 POINT(41.12984454455186 73.89473872430871) bank81974 +81975 POINT(40.11425190480654 74.5625823590757) bank81975 +81976 POINT(41.45755188131941 74.59429657880175) bank81976 +81977 POINT(40.826849928596 74.07829027509865) bank81977 +81978 POINT(41.64890541204971 74.97451362351013) bank81978 +81979 POINT(40.12081852908176 73.20951545506861) bank81979 +81980 POINT(41.23207993924048 73.31726116219741) bank81980 +81981 POINT(41.46226355403466 74.00049965908907) bank81981 +81982 POINT(40.29137718775795 74.78108005538508) bank81982 +81983 POINT(41.473274678224016 74.38925899941088) bank81983 +81984 POINT(40.038351534466685 73.81815823368) bank81984 +81985 POINT(41.591311874968454 74.33150729654758) bank81985 +81986 POINT(40.33076715503743 73.88210274327267) bank81986 +81987 POINT(40.392467873922286 74.30030244332639) bank81987 +81988 POINT(41.07385760281075 74.58563817628398) bank81988 +81989 POINT(40.28084767228189 73.65497353549445) bank81989 +81990 POINT(40.14594350450543 73.69844420526213) bank81990 +81991 POINT(39.84027788813006 73.63838085990244) bank81991 +81992 POINT(40.120054493601046 74.6865348471673) bank81992 +81993 POINT(40.57838492819947 73.81281970594317) bank81993 +81994 POINT(41.004282227462404 73.31286897049999) bank81994 +81995 POINT(40.871311226861664 74.11777648848886) bank81995 +81996 POINT(39.84318102825233 73.92067084405289) bank81996 +81997 POINT(40.96365487531987 73.56229227831656) bank81997 +81998 POINT(40.28776857017134 74.37924896591075) bank81998 +81999 POINT(40.71146716503276 73.66073081556178) bank81999 +82000 POINT(41.69977819953919 74.14027748802351) bank82000 +82001 POINT(40.20689582730261 74.32454974460612) bank82001 +82002 POINT(41.226949613626104 73.7442460545958) bank82002 +82003 POINT(40.55936769901483 73.09023473440305) bank82003 +82004 POINT(39.76308161312746 73.44400363928112) bank82004 +82005 POINT(40.290430132678786 74.84265285068366) bank82005 +82006 POINT(40.640780649569336 74.858199411514) bank82006 +82007 POINT(39.98751405249033 74.9930445758804) bank82007 +82008 POINT(41.1194865637833 73.88783084875148) bank82008 +82009 POINT(41.192760244924486 74.61471533227429) bank82009 +82010 POINT(41.197461365781216 73.35529394565489) bank82010 +82011 POINT(41.43309925281531 73.52789630695597) bank82011 +82012 POINT(41.49214100547072 74.60062395456472) bank82012 +82013 POINT(40.027838283789 73.5567432965625) bank82013 +82014 POINT(41.522737942771535 74.54425275763988) bank82014 +82015 POINT(41.71208967812628 73.36354245542269) bank82015 +82016 POINT(40.80764266266745 74.01899119984274) bank82016 +82017 POINT(40.95070503190535 73.26028560545053) bank82017 +82018 POINT(39.89599772390016 74.68676445748972) bank82018 +82019 POINT(40.766602639916066 73.71425071985449) bank82019 +82020 POINT(40.70076242301379 74.92133033812517) bank82020 +82021 POINT(39.92387557519078 73.88273607843458) bank82021 +82022 POINT(40.06783067885408 73.53215009241093) bank82022 +82023 POINT(40.562177851559646 74.85635184442033) bank82023 +82024 POINT(40.6798846400913 74.83425554962082) bank82024 +82025 POINT(40.37148503603635 74.13297121860357) bank82025 +82026 POINT(41.41145782526712 74.22419888088686) bank82026 +82027 POINT(40.66041184583561 73.84255891192183) bank82027 +82028 POINT(41.17118368162468 73.49907110498289) bank82028 +82029 POINT(41.339184835821115 73.83838052275797) bank82029 +82030 POINT(40.717841728843084 74.03434377096232) bank82030 +82031 POINT(40.57876149692971 73.15308204714815) bank82031 +82032 POINT(40.38139327889735 73.94738999083422) bank82032 +82033 POINT(39.723472726899836 74.4138136613479) bank82033 +82034 POINT(39.94198819346094 73.64723433894676) bank82034 +82035 POINT(40.54119611833071 73.41610143066949) bank82035 +82036 POINT(41.069469839386585 73.01038801331681) bank82036 +82037 POINT(40.21190680124247 73.84976840839333) bank82037 +82038 POINT(40.00356284390627 74.04277365773838) bank82038 +82039 POINT(39.986625694299896 74.42441097253042) bank82039 +82040 POINT(40.61784984666768 73.23041225438769) bank82040 +82041 POINT(41.419743946247145 74.85329321651565) bank82041 +82042 POINT(41.130499521209074 73.62578760552486) bank82042 +82043 POINT(39.878980031044996 73.5215216984052) bank82043 +82044 POINT(40.66477650754928 74.85239869705677) bank82044 +82045 POINT(41.19356321818718 74.81621577417289) bank82045 +82046 POINT(41.16577499537422 74.1251428432935) bank82046 +82047 POINT(40.648976720231474 74.53065747246097) bank82047 +82048 POINT(40.0322958263149 73.85215959062019) bank82048 +82049 POINT(40.6511868754935 73.41388929769461) bank82049 +82050 POINT(39.97766189468225 74.10554429346492) bank82050 +82051 POINT(40.06140790163235 73.36889225428357) bank82051 +82052 POINT(40.32954452819275 73.92276722258237) bank82052 +82053 POINT(40.686392597741374 73.55714490687713) bank82053 +82054 POINT(40.91099964618261 74.34351921786707) bank82054 +82055 POINT(39.90774950332555 74.83742660946828) bank82055 +82056 POINT(41.4548623816551 73.0370191768035) bank82056 +82057 POINT(41.325006445766896 74.18545953271327) bank82057 +82058 POINT(39.90664885828702 74.90314635152531) bank82058 +82059 POINT(41.444127317693145 74.80479326983749) bank82059 +82060 POINT(40.49478700229063 74.01389540612215) bank82060 +82061 POINT(40.268516504346586 73.19140310376163) bank82061 +82062 POINT(41.47859452019577 73.18375862517955) bank82062 +82063 POINT(41.12270939353801 73.69569603533593) bank82063 +82064 POINT(41.55753726963917 73.44903266496681) bank82064 +82065 POINT(40.707734686893126 74.22728049027162) bank82065 +82066 POINT(39.73594472893396 73.39868503432412) bank82066 +82067 POINT(39.97000273198708 73.54190848677878) bank82067 +82068 POINT(41.24727513475154 74.81149915763534) bank82068 +82069 POINT(41.368094346755555 73.56805764505748) bank82069 +82070 POINT(39.93900643040562 73.82989927081954) bank82070 +82071 POINT(41.46114952808787 73.71355609451692) bank82071 +82072 POINT(41.621309391810854 73.12582140143583) bank82072 +82073 POINT(41.2400258688967 73.57541899589512) bank82073 +82074 POINT(39.807081696481696 74.58523719972194) bank82074 +82075 POINT(40.721227276912686 74.08104324150972) bank82075 +82076 POINT(40.71146989869195 74.2319090559662) bank82076 +82077 POINT(40.10951236206992 74.64008570961063) bank82077 +82078 POINT(41.07205666121457 73.77774423584432) bank82078 +82079 POINT(39.93653508317062 73.09412070268868) bank82079 +82080 POINT(40.83558669809987 73.32678665500396) bank82080 +82081 POINT(40.44952924956735 74.1611604256142) bank82081 +82082 POINT(39.74406339701089 73.14707513213223) bank82082 +82083 POINT(40.03435447481246 73.21308371560035) bank82083 +82084 POINT(39.92467592089177 73.86052113598139) bank82084 +82085 POINT(40.7002281686532 73.54789277224788) bank82085 +82086 POINT(41.2198694207175 74.55518478143445) bank82086 +82087 POINT(41.20614050050002 74.13822194439545) bank82087 +82088 POINT(40.41693108667557 73.86627236373789) bank82088 +82089 POINT(39.981326460550804 74.67105450708637) bank82089 +82090 POINT(41.48393559738182 74.58333733052807) bank82090 +82091 POINT(41.693140994938254 73.33225117856941) bank82091 +82092 POINT(40.850013862959194 73.8438084319243) bank82092 +82093 POINT(39.77803612779497 74.86693298804279) bank82093 +82094 POINT(40.92106974431717 74.75214853959436) bank82094 +82095 POINT(41.49064339847068 74.56475305097563) bank82095 +82096 POINT(40.64533595762243 74.7639272062027) bank82096 +82097 POINT(40.80118079534393 73.94674205653253) bank82097 +82098 POINT(40.24998994734026 74.40443334282358) bank82098 +82099 POINT(39.90513960983398 73.03344582043383) bank82099 +82100 POINT(39.72093840002298 74.68862100143889) bank82100 +82101 POINT(41.49566910556198 74.68438097057916) bank82101 +82102 POINT(40.188353756795856 74.28559872114558) bank82102 +82103 POINT(40.95796798146058 74.37571350220163) bank82103 +82104 POINT(39.73110501309026 74.84984508640058) bank82104 +82105 POINT(40.18487268615176 74.62164976461341) bank82105 +82106 POINT(39.82064281181198 74.93354795129451) bank82106 +82107 POINT(41.39406441196532 74.61194472665225) bank82107 +82108 POINT(41.56900585680056 73.52562208276333) bank82108 +82109 POINT(40.002774173304424 73.80641307046179) bank82109 +82110 POINT(40.86341636888042 73.9140234287659) bank82110 +82111 POINT(41.11566960375856 74.1261623078487) bank82111 +82112 POINT(40.285226395653474 74.27498365671687) bank82112 +82113 POINT(40.63739353666694 73.54834724140073) bank82113 +82114 POINT(40.49690148099305 73.0845434772512) bank82114 +82115 POINT(39.936855710436475 73.1531331343) bank82115 +82116 POINT(41.04754112290208 73.59655568793252) bank82116 +82117 POINT(41.3543336863139 74.05704894866994) bank82117 +82118 POINT(41.271200702320456 74.03074863088345) bank82118 +82119 POINT(40.99822519091935 73.77158035253387) bank82119 +82120 POINT(41.254921635777116 74.39036980749864) bank82120 +82121 POINT(41.0634832052405 73.48677450870274) bank82121 +82122 POINT(41.64065791105479 74.76528382839348) bank82122 +82123 POINT(40.541248010591474 73.67648620011552) bank82123 +82124 POINT(41.40724281016713 74.10049672128375) bank82124 +82125 POINT(40.66266818974577 74.57580155698886) bank82125 +82126 POINT(39.732487194371984 73.71680026383179) bank82126 +82127 POINT(40.58256403810187 73.79157957200184) bank82127 +82128 POINT(40.098650542605384 74.6403404844296) bank82128 +82129 POINT(41.26417476638628 73.8591484753857) bank82129 +82130 POINT(41.17337986102444 74.43149385654684) bank82130 +82131 POINT(40.19845381036784 74.54319860334988) bank82131 +82132 POINT(40.86599183977017 73.24736364151413) bank82132 +82133 POINT(40.953087936925186 74.80341542913126) bank82133 +82134 POINT(41.179195841556485 74.3071400344133) bank82134 +82135 POINT(40.28057621404293 73.8179133726852) bank82135 +82136 POINT(40.237340296521495 74.90315110340839) bank82136 +82137 POINT(40.31100485623492 73.25791146297269) bank82137 +82138 POINT(40.61482247313042 74.42940778280744) bank82138 +82139 POINT(41.347296717965406 74.30474685022976) bank82139 +82140 POINT(39.9848739422484 74.15910445880756) bank82140 +82141 POINT(41.36442940573424 74.9756744846561) bank82141 +82142 POINT(40.25660983891823 74.95641579805138) bank82142 +82143 POINT(41.6530082971391 74.68632083734038) bank82143 +82144 POINT(40.007131964451034 73.85189687793535) bank82144 +82145 POINT(41.568814049975614 73.94728063328759) bank82145 +82146 POINT(40.21721856469011 74.50514290700873) bank82146 +82147 POINT(39.83653430242003 74.9284851811375) bank82147 +82148 POINT(41.13468017980173 74.1529807284707) bank82148 +82149 POINT(40.8520753252476 73.66653186697523) bank82149 +82150 POINT(40.72586477939179 74.5169917442985) bank82150 +82151 POINT(40.90862974753401 73.5520583631691) bank82151 +82152 POINT(39.82591723055841 73.18582335300053) bank82152 +82153 POINT(40.46866759716351 74.5977156315891) bank82153 +82154 POINT(41.06168143266096 74.94479294663897) bank82154 +82155 POINT(40.766453245601454 73.53197131970623) bank82155 +82156 POINT(40.911422093979084 73.16016178273568) bank82156 +82157 POINT(41.22535962942127 73.54137144007159) bank82157 +82158 POINT(41.57091605570051 73.7633367920132) bank82158 +82159 POINT(41.132680644597926 74.99670219488497) bank82159 +82160 POINT(40.187122361086566 74.40389992994197) bank82160 +82161 POINT(40.49807547365511 73.1387912456766) bank82161 +82162 POINT(40.93707381830161 74.41066916752256) bank82162 +82163 POINT(41.693739362588516 74.03228427053587) bank82163 +82164 POINT(41.31921408198465 74.58220767311845) bank82164 +82165 POINT(39.83711635718976 74.23406992547663) bank82165 +82166 POINT(41.54038336671007 74.319254714241) bank82166 +82167 POINT(40.33415019163843 74.57968813625718) bank82167 +82168 POINT(40.64717071349992 73.764274470679) bank82168 +82169 POINT(41.23540193929999 73.54554796555855) bank82169 +82170 POINT(39.97008655732199 74.01954301188165) bank82170 +82171 POINT(41.36030567506218 74.68749221896279) bank82171 +82172 POINT(40.54919342156755 74.25773424141433) bank82172 +82173 POINT(40.41324388339932 73.5855684038284) bank82173 +82174 POINT(41.33674126217969 74.57725128443022) bank82174 +82175 POINT(40.75916993901197 74.47168797264882) bank82175 +82176 POINT(40.95852762218359 73.28870349118354) bank82176 +82177 POINT(41.179129276724495 74.45858863546536) bank82177 +82178 POINT(40.3405324346189 73.26282606586638) bank82178 +82179 POINT(40.34561707528508 73.62211398279986) bank82179 +82180 POINT(40.96506853701905 74.080790034814) bank82180 +82181 POINT(40.744500806513884 73.46379098847638) bank82181 +82182 POINT(41.016482375697066 74.56490961756252) bank82182 +82183 POINT(41.04981355717052 74.95709082989231) bank82183 +82184 POINT(39.86340630259064 73.97961395373055) bank82184 +82185 POINT(41.14917386434078 73.93654075729869) bank82185 +82186 POINT(41.09829530186455 73.48594457136868) bank82186 +82187 POINT(40.54726750732423 73.4500470230378) bank82187 +82188 POINT(40.697776093796136 73.44913270323995) bank82188 +82189 POINT(40.90460645753608 73.18472312412075) bank82189 +82190 POINT(40.9811501758095 74.6521094498805) bank82190 +82191 POINT(39.725866105616326 73.63679250565457) bank82191 +82192 POINT(40.20997454408652 74.75133727653387) bank82192 +82193 POINT(41.68501090752263 74.92075675575927) bank82193 +82194 POINT(40.79356794691078 73.1206427354386) bank82194 +82195 POINT(40.76094245706143 73.98095468565089) bank82195 +82196 POINT(39.95036818108609 73.28735678880736) bank82196 +82197 POINT(41.464732904799945 73.67165730771545) bank82197 +82198 POINT(41.29326730126312 74.23768804793382) bank82198 +82199 POINT(39.80405018240546 73.31415301130451) bank82199 +82200 POINT(40.85006967316934 74.76488593381976) bank82200 +82201 POINT(41.51259322928017 73.10499854522106) bank82201 +82202 POINT(40.85861822282583 73.0742532353445) bank82202 +82203 POINT(41.377031934824004 73.30171655568624) bank82203 +82204 POINT(40.79634317311054 73.02921388990822) bank82204 +82205 POINT(40.54182299551539 73.01034001272645) bank82205 +82206 POINT(40.41861781622379 73.70220300670373) bank82206 +82207 POINT(40.33063260814603 73.98039368069608) bank82207 +82208 POINT(41.04825220767134 74.45262047613964) bank82208 +82209 POINT(41.035964532915195 73.43595367365268) bank82209 +82210 POINT(41.69191968190956 73.9065219912187) bank82210 +82211 POINT(41.09017722997579 74.6594142208499) bank82211 +82212 POINT(41.021487736991396 74.19540959475782) bank82212 +82213 POINT(41.5260232841435 74.78876644449136) bank82213 +82214 POINT(39.84475093905386 74.01275303958067) bank82214 +82215 POINT(40.21276194549522 74.59694404598983) bank82215 +82216 POINT(40.411534699234736 74.45670644344392) bank82216 +82217 POINT(41.29690839104775 73.25739905821011) bank82217 +82218 POINT(40.13222900464466 74.6330276828393) bank82218 +82219 POINT(41.49457118890372 74.98258005803041) bank82219 +82220 POINT(40.71380460064101 73.7558315127655) bank82220 +82221 POINT(41.03598218130227 73.54510720602434) bank82221 +82222 POINT(39.72586029937397 74.92067681978169) bank82222 +82223 POINT(40.16925687916696 73.60688274872857) bank82223 +82224 POINT(41.707109590665276 74.12910047487803) bank82224 +82225 POINT(41.67932421756217 74.3922155123415) bank82225 +82226 POINT(39.994823235139656 74.3598820082876) bank82226 +82227 POINT(41.4184865304463 74.92778388397396) bank82227 +82228 POINT(41.356420923730965 73.04518905360078) bank82228 +82229 POINT(40.34468051483543 73.4186498158687) bank82229 +82230 POINT(41.420577141239576 74.24820043948067) bank82230 +82231 POINT(40.040847539723615 73.92598958245158) bank82231 +82232 POINT(40.065331245092985 74.66819888359561) bank82232 +82233 POINT(40.289284096105895 73.12385993663095) bank82233 +82234 POINT(40.09407509055638 74.69867661262327) bank82234 +82235 POINT(40.65133289763175 73.06171227283455) bank82235 +82236 POINT(40.328548433720776 73.18616887129909) bank82236 +82237 POINT(40.10240747542322 73.44848300846662) bank82237 +82238 POINT(40.16322899385631 73.6789621249527) bank82238 +82239 POINT(41.01527799428361 73.73555124924188) bank82239 +82240 POINT(40.956781771801715 73.96538284023828) bank82240 +82241 POINT(41.5221275924804 74.71854748609175) bank82241 +82242 POINT(40.75649151655219 74.9322685967734) bank82242 +82243 POINT(40.647786772083165 74.96013988091903) bank82243 +82244 POINT(41.670142183761506 74.32081884327278) bank82244 +82245 POINT(40.81522490139646 73.3365359491523) bank82245 +82246 POINT(40.52572796105467 74.99278236523756) bank82246 +82247 POINT(40.79104482378117 73.52739833696076) bank82247 +82248 POINT(40.47630017597622 74.83613121879397) bank82248 +82249 POINT(40.93204630155775 73.7943793754257) bank82249 +82250 POINT(41.299888794271325 74.46842111680543) bank82250 +82251 POINT(40.43279644702616 74.22064532739296) bank82251 +82252 POINT(41.25474934509775 73.52119866813034) bank82252 +82253 POINT(41.40308989277523 74.24944354144776) bank82253 +82254 POINT(39.81647687843976 74.40740443726254) bank82254 +82255 POINT(39.88691238838495 73.48335641465808) bank82255 +82256 POINT(41.44222346430832 74.6189635565456) bank82256 +82257 POINT(40.27171362688253 73.17277671413405) bank82257 +82258 POINT(40.51822847495935 74.88321143961734) bank82258 +82259 POINT(40.28902349062328 73.34136820024939) bank82259 +82260 POINT(41.09511959104848 74.3610234785133) bank82260 +82261 POINT(39.92961759327449 74.78313382445234) bank82261 +82262 POINT(40.42761512697012 73.22261541320543) bank82262 +82263 POINT(40.46046101718993 74.92518291522828) bank82263 +82264 POINT(40.37052377216945 74.10270531445158) bank82264 +82265 POINT(40.73539877125425 73.95792091857949) bank82265 +82266 POINT(41.069337502011145 74.01761958055835) bank82266 +82267 POINT(40.53524909600071 74.9623137507079) bank82267 +82268 POINT(39.751899347161306 74.99388889768932) bank82268 +82269 POINT(40.47438528444445 74.91880626564462) bank82269 +82270 POINT(41.26078109378121 73.56720236025781) bank82270 +82271 POINT(39.81522232836456 73.60645152239428) bank82271 +82272 POINT(40.69279309161184 74.71078396753965) bank82272 +82273 POINT(40.697991071305566 73.07252918719038) bank82273 +82274 POINT(41.02593011034231 74.0137209305041) bank82274 +82275 POINT(41.599281107446664 73.10941634239039) bank82275 +82276 POINT(40.32447931320471 73.97710058939929) bank82276 +82277 POINT(41.67301890183536 74.48822511919664) bank82277 +82278 POINT(39.95066800063878 73.31561718646587) bank82278 +82279 POINT(39.830739550677706 73.27932963221325) bank82279 +82280 POINT(40.11333810584244 74.47116145990618) bank82280 +82281 POINT(40.241019083591915 74.50098594452112) bank82281 +82282 POINT(41.26666241903202 74.12044774539216) bank82282 +82283 POINT(40.3769091362683 74.54171037949455) bank82283 +82284 POINT(41.365864028360754 74.39063859202993) bank82284 +82285 POINT(40.179691464101936 74.28853001818753) bank82285 +82286 POINT(39.89395169014366 73.9093213181111) bank82286 +82287 POINT(40.32848583456247 73.32811168235183) bank82287 +82288 POINT(40.08075476785805 73.3602899906135) bank82288 +82289 POINT(40.137825839715035 73.41665864731648) bank82289 +82290 POINT(41.51156265671628 73.71208220683876) bank82290 +82291 POINT(41.30752812763444 73.63542144516096) bank82291 +82292 POINT(41.53110717667837 73.50388389588993) bank82292 +82293 POINT(41.28450918745572 74.96091630553052) bank82293 +82294 POINT(40.767289390382885 73.84800258776917) bank82294 +82295 POINT(40.67296196643827 74.8504950942948) bank82295 +82296 POINT(41.43470599935974 73.25191636998144) bank82296 +82297 POINT(40.3676823993994 73.62887833751205) bank82297 +82298 POINT(40.495711365622164 74.14226392486434) bank82298 +82299 POINT(40.3877499497357 73.36217251945129) bank82299 +82300 POINT(40.97033384460663 73.62853302284523) bank82300 +82301 POINT(40.748239842540684 74.41316823135553) bank82301 +82302 POINT(41.31413826803931 74.60899143093742) bank82302 +82303 POINT(41.263060223998906 74.12108149743257) bank82303 +82304 POINT(40.35632858628841 73.6560302466999) bank82304 +82305 POINT(41.307168224302856 74.86925166428713) bank82305 +82306 POINT(40.50882190411695 73.86012636065777) bank82306 +82307 POINT(40.244024860134886 74.82199595325177) bank82307 +82308 POINT(39.74733732700547 74.78690156739981) bank82308 +82309 POINT(40.174054152023835 74.35085558274892) bank82309 +82310 POINT(41.58875861417774 74.0711355251939) bank82310 +82311 POINT(39.83413929516085 73.36032163191324) bank82311 +82312 POINT(40.37446695873288 74.89572566278565) bank82312 +82313 POINT(41.188934212057326 74.92004579113355) bank82313 +82314 POINT(40.5224219824005 74.66824074849663) bank82314 +82315 POINT(41.001643248452005 73.55130272961702) bank82315 +82316 POINT(41.683877441546244 73.81796781652977) bank82316 +82317 POINT(40.90289164954957 74.87630140137854) bank82317 +82318 POINT(41.13610581065845 73.04761245220271) bank82318 +82319 POINT(41.31364702066626 73.31329325130672) bank82319 +82320 POINT(40.20348062732468 73.77245312263635) bank82320 +82321 POINT(41.22980859372241 73.80462774900008) bank82321 +82322 POINT(40.52366894789481 74.69507404010633) bank82322 +82323 POINT(40.301580025676344 73.86571572253055) bank82323 +82324 POINT(40.56501770222063 74.27738495157732) bank82324 +82325 POINT(40.405033791756416 73.42590414207883) bank82325 +82326 POINT(40.83785659104306 73.60959086948709) bank82326 +82327 POINT(40.88392063540251 74.11915183847627) bank82327 +82328 POINT(40.17122922496611 74.50642157920895) bank82328 +82329 POINT(40.53956419607352 74.85925632219117) bank82329 +82330 POINT(40.835632598147896 73.71321428619753) bank82330 +82331 POINT(40.404082559250725 73.69551873401709) bank82331 +82332 POINT(41.48790124344307 74.2909016818642) bank82332 +82333 POINT(39.73253075826318 73.59029399238506) bank82333 +82334 POINT(41.13184419785239 73.65418568197225) bank82334 +82335 POINT(39.8747072461709 73.82649264892979) bank82335 +82336 POINT(41.56690937520263 73.84490704444465) bank82336 +82337 POINT(39.81559459941408 74.28021497995026) bank82337 +82338 POINT(39.99765932643905 73.85680712689965) bank82338 +82339 POINT(41.52862716268492 74.70286036234486) bank82339 +82340 POINT(41.13485689070273 74.05724536880179) bank82340 +82341 POINT(40.339833995178225 74.91067518862825) bank82341 +82342 POINT(41.46697306723847 74.74834503974682) bank82342 +82343 POINT(41.58224541627313 74.497996604576) bank82343 +82344 POINT(39.88995626017696 74.80909109701265) bank82344 +82345 POINT(40.48120868783051 74.82726795201597) bank82345 +82346 POINT(40.73082107183413 73.9230388665016) bank82346 +82347 POINT(40.76909573731034 74.4258054782057) bank82347 +82348 POINT(41.645312080958504 73.9366696271821) bank82348 +82349 POINT(39.76154348023117 74.47880321023702) bank82349 +82350 POINT(39.904067310442635 74.50265674165276) bank82350 +82351 POINT(39.813883071705874 74.2775036508488) bank82351 +82352 POINT(40.26181815938542 73.08943388506907) bank82352 +82353 POINT(40.623175167190375 73.88022107496886) bank82353 +82354 POINT(40.32260419068456 73.67652284594874) bank82354 +82355 POINT(41.59001870397074 74.98637637347979) bank82355 +82356 POINT(39.823326248851465 74.85095471400051) bank82356 +82357 POINT(40.46604187379536 73.43025792519089) bank82357 +82358 POINT(40.39615074150308 73.28979953518957) bank82358 +82359 POINT(41.22840301958368 74.84861156624603) bank82359 +82360 POINT(40.18517237508202 74.6397948061995) bank82360 +82361 POINT(41.27049122449376 74.6062592152628) bank82361 +82362 POINT(40.24671719604599 73.29503182992842) bank82362 +82363 POINT(41.09155276030178 73.24267411165515) bank82363 +82364 POINT(40.72704402904196 74.48238709538732) bank82364 +82365 POINT(40.00490286694356 74.06968715329485) bank82365 +82366 POINT(41.099710089248326 74.89672172197602) bank82366 +82367 POINT(39.98059426194855 73.68162095246038) bank82367 +82368 POINT(41.48666416790458 74.26981160259437) bank82368 +82369 POINT(41.276872068484195 74.26116902095454) bank82369 +82370 POINT(40.7172906166529 74.40568418508411) bank82370 +82371 POINT(39.94895490414002 73.32141470760513) bank82371 +82372 POINT(40.57886503812794 74.94014978897468) bank82372 +82373 POINT(41.490691842256325 74.42392847162968) bank82373 +82374 POINT(40.77789626041238 74.43389390544681) bank82374 +82375 POINT(41.33122258723803 73.58157878662564) bank82375 +82376 POINT(40.64223907409497 74.22685199403787) bank82376 +82377 POINT(41.06612287335979 74.45446786738418) bank82377 +82378 POINT(39.910762252286155 74.27119780657789) bank82378 +82379 POINT(41.67819356530694 73.88152800029059) bank82379 +82380 POINT(41.470899439528225 74.57545304773924) bank82380 +82381 POINT(41.63824714751775 74.70899023838429) bank82381 +82382 POINT(39.75061651006137 74.1185931906034) bank82382 +82383 POINT(40.8224370289797 73.40917035159755) bank82383 +82384 POINT(41.219216866677286 74.16465337330528) bank82384 +82385 POINT(40.00102287071054 73.32698350150176) bank82385 +82386 POINT(40.768787054364054 74.6798619188148) bank82386 +82387 POINT(39.9134475059179 74.66895313617313) bank82387 +82388 POINT(40.810665954679955 73.65751482762685) bank82388 +82389 POINT(40.56722400688068 74.03836103643215) bank82389 +82390 POINT(41.515551786516625 74.01360565257956) bank82390 +82391 POINT(40.29458058484353 74.20107077330526) bank82391 +82392 POINT(40.257422585764566 74.51874873827421) bank82392 +82393 POINT(40.363755087710125 74.02975777627945) bank82393 +82394 POINT(40.00178517277331 73.73508095040008) bank82394 +82395 POINT(40.82871417667798 74.24121528669228) bank82395 +82396 POINT(40.337731191066666 73.73585250409765) bank82396 +82397 POINT(40.97143913965737 73.3152883284697) bank82397 +82398 POINT(40.90853125741524 74.7855107347137) bank82398 +82399 POINT(41.311357710908354 74.25402940024232) bank82399 +82400 POINT(41.089414832673384 73.13395302150931) bank82400 +82401 POINT(40.17752303758106 74.80745109201311) bank82401 +82402 POINT(39.84092118492054 74.03596650489601) bank82402 +82403 POINT(40.43244621536766 74.00674361473483) bank82403 +82404 POINT(41.65491759304964 74.88656758527874) bank82404 +82405 POINT(40.677543007283184 73.82429304996623) bank82405 +82406 POINT(41.39370767589861 73.26330725395007) bank82406 +82407 POINT(40.016712653004646 74.7460610028614) bank82407 +82408 POINT(40.76963529259427 74.55833512078051) bank82408 +82409 POINT(41.00855822061522 73.82443292262421) bank82409 +82410 POINT(41.680748059676915 74.91596314050696) bank82410 +82411 POINT(41.36226117464191 73.57448357148795) bank82411 +82412 POINT(39.817344417405494 74.0299781864669) bank82412 +82413 POINT(40.92441564353173 73.59763089554207) bank82413 +82414 POINT(40.65897013354516 74.77577386300881) bank82414 +82415 POINT(41.42822770168411 74.4842592399231) bank82415 +82416 POINT(39.96975571753098 74.93160065097936) bank82416 +82417 POINT(39.72597374014667 74.05771841219119) bank82417 +82418 POINT(40.83747171416617 73.32680941857585) bank82418 +82419 POINT(39.92246425985286 73.18449779415987) bank82419 +82420 POINT(40.1156161416353 73.38434508779586) bank82420 +82421 POINT(40.15646668628223 73.19816806768264) bank82421 +82422 POINT(39.94772423903206 73.69702457873923) bank82422 +82423 POINT(40.99749006311717 73.30550148757808) bank82423 +82424 POINT(40.22712486833317 73.83310493071454) bank82424 +82425 POINT(40.30154636822704 73.3366047572835) bank82425 +82426 POINT(40.63303069583648 73.68476350858712) bank82426 +82427 POINT(40.51896814163737 73.67371053119176) bank82427 +82428 POINT(40.01307564214208 74.19774076385191) bank82428 +82429 POINT(41.20007235037122 73.7083697313718) bank82429 +82430 POINT(40.61448751314137 73.01950171136481) bank82430 +82431 POINT(40.61445310048259 74.31758123959774) bank82431 +82432 POINT(41.06504083026777 74.15668817725698) bank82432 +82433 POINT(41.47664198604202 73.80315814500612) bank82433 +82434 POINT(40.155730106900684 73.45853952183494) bank82434 +82435 POINT(40.42023338815903 73.83860977788771) bank82435 +82436 POINT(41.333636317911456 73.75976680212437) bank82436 +82437 POINT(40.66217633865654 73.16816700810381) bank82437 +82438 POINT(40.425107471803685 73.85692443432767) bank82438 +82439 POINT(41.567107923085075 73.85106291810733) bank82439 +82440 POINT(41.26047697382569 74.27289184799277) bank82440 +82441 POINT(41.06740354430533 74.78735483150822) bank82441 +82442 POINT(39.97349873239176 74.04007722268275) bank82442 +82443 POINT(40.813125480110635 73.05119598237557) bank82443 +82444 POINT(41.092305013486566 74.53763555472717) bank82444 +82445 POINT(40.24811048808864 74.71204487749809) bank82445 +82446 POINT(39.81525134940151 73.3708428125328) bank82446 +82447 POINT(41.64483406328024 73.04514701391405) bank82447 +82448 POINT(41.19975182906701 73.6387076853975) bank82448 +82449 POINT(40.7171373929687 73.79565795795195) bank82449 +82450 POINT(40.699693005979334 73.76555296436155) bank82450 +82451 POINT(40.794468957052246 74.70869436313185) bank82451 +82452 POINT(40.35398521586963 73.63246775181125) bank82452 +82453 POINT(40.002701936040616 74.67918032682775) bank82453 +82454 POINT(41.37675188363103 74.47488700618513) bank82454 +82455 POINT(39.716084958274536 74.86057858996594) bank82455 +82456 POINT(41.103637451711364 73.55978519632221) bank82456 +82457 POINT(40.72502517778375 74.20398293047467) bank82457 +82458 POINT(40.94456941839187 74.54494616167305) bank82458 +82459 POINT(41.60692310246037 74.7858111865294) bank82459 +82460 POINT(40.501284138905156 73.57969922303951) bank82460 +82461 POINT(40.52767546600942 73.82385319713285) bank82461 +82462 POINT(41.496242903548755 74.96428089368227) bank82462 +82463 POINT(40.7616238303322 74.02175986838571) bank82463 +82464 POINT(40.750365790642086 73.25144801664352) bank82464 +82465 POINT(40.30327722462361 74.92284111730376) bank82465 +82466 POINT(39.79953487357386 74.35127142312221) bank82466 +82467 POINT(41.359875394871914 73.67944080333649) bank82467 +82468 POINT(40.03918150648494 74.30197836963552) bank82468 +82469 POINT(40.742567883482906 73.28944781241998) bank82469 +82470 POINT(40.49744746152437 74.46816380077885) bank82470 +82471 POINT(41.06677931918022 74.64604067791687) bank82471 +82472 POINT(39.831773383167466 74.40782231580448) bank82472 +82473 POINT(40.63690391881797 73.92799399207759) bank82473 +82474 POINT(40.765487210217245 74.55271426220172) bank82474 +82475 POINT(41.076945407651095 74.54165112076097) bank82475 +82476 POINT(40.592480829489794 74.5795118325631) bank82476 +82477 POINT(40.750403248509464 73.3160138443455) bank82477 +82478 POINT(40.85113478499936 74.88715266868694) bank82478 +82479 POINT(40.85134554197166 74.25880178198031) bank82479 +82480 POINT(40.176097592761366 74.7412553967879) bank82480 +82481 POINT(39.74384655254695 74.05856870820446) bank82481 +82482 POINT(40.02914055747115 73.06659911687537) bank82482 +82483 POINT(40.75030553131066 73.62089008550215) bank82483 +82484 POINT(39.845431914867824 74.15941688690572) bank82484 +82485 POINT(41.114378922976 73.42889584542641) bank82485 +82486 POINT(40.23579254277846 74.0192129207331) bank82486 +82487 POINT(40.06788549919504 74.08473021634802) bank82487 +82488 POINT(40.52942577638339 73.081935070267) bank82488 +82489 POINT(40.28078741117093 73.57457073849707) bank82489 +82490 POINT(40.05621135668051 74.2018455183749) bank82490 +82491 POINT(41.127710853172765 74.72280269660035) bank82491 +82492 POINT(40.00475486079025 73.19511091564459) bank82492 +82493 POINT(39.722745940628236 74.09606216852501) bank82493 +82494 POINT(40.88737743452739 74.45811106916204) bank82494 +82495 POINT(40.826083909451306 74.3826587325627) bank82495 +82496 POINT(41.595090296796414 73.66350782900592) bank82496 +82497 POINT(40.51693582470305 74.72474064015151) bank82497 +82498 POINT(40.747670859182136 73.19213733428741) bank82498 +82499 POINT(41.56741955378291 74.81874906554255) bank82499 +82500 POINT(41.3891807219642 74.01156831584858) bank82500 +82501 POINT(41.34354956086217 73.20617024920453) bank82501 +82502 POINT(41.707644609065596 73.80503421073098) bank82502 +82503 POINT(41.193089367231984 74.74154458288413) bank82503 +82504 POINT(40.26213390617076 73.66963672151712) bank82504 +82505 POINT(40.728277030288076 73.48791075333955) bank82505 +82506 POINT(39.81125257562776 74.50239657407309) bank82506 +82507 POINT(41.51997419832877 74.27628616495926) bank82507 +82508 POINT(41.6868355861254 74.09498295162855) bank82508 +82509 POINT(41.19113511489786 73.26521938566552) bank82509 +82510 POINT(41.58860172550115 73.79871591842267) bank82510 +82511 POINT(41.668732651215805 73.04981570491404) bank82511 +82512 POINT(40.313789519401666 73.83850629495981) bank82512 +82513 POINT(40.09952293220059 73.09126076999343) bank82513 +82514 POINT(39.892973417728996 73.71595285425974) bank82514 +82515 POINT(41.45773895068451 73.13063353949775) bank82515 +82516 POINT(40.69725638615684 74.71015361798445) bank82516 +82517 POINT(41.663187601890876 73.76874027846688) bank82517 +82518 POINT(41.698195580955904 73.33470241911553) bank82518 +82519 POINT(39.89401782407434 74.96333354871318) bank82519 +82520 POINT(40.97155519012378 73.3732345863789) bank82520 +82521 POINT(40.105996800658325 73.23398694506164) bank82521 +82522 POINT(41.44481644366687 74.32217119561115) bank82522 +82523 POINT(41.658551018415956 74.51683261076) bank82523 +82524 POINT(41.36678784480705 74.92649938338356) bank82524 +82525 POINT(40.222768340415946 74.43254113817872) bank82525 +82526 POINT(39.84057844440433 74.62748623054439) bank82526 +82527 POINT(40.9371412257035 73.86696247745812) bank82527 +82528 POINT(40.53054049066263 73.19835232083152) bank82528 +82529 POINT(40.71193233247316 73.9924960791149) bank82529 +82530 POINT(41.22799096352081 74.38871691479505) bank82530 +82531 POINT(39.78070614185938 73.89592295271213) bank82531 +82532 POINT(40.793516931040514 73.89742857222201) bank82532 +82533 POINT(41.16225476536218 74.84468897771704) bank82533 +82534 POINT(40.8707404870767 73.1333125859607) bank82534 +82535 POINT(40.62926792366107 73.41978335068112) bank82535 +82536 POINT(41.61055637864681 73.1007113264137) bank82536 +82537 POINT(41.257753992035816 74.26906222032528) bank82537 +82538 POINT(40.624553098810175 73.08314758707107) bank82538 +82539 POINT(40.00435134414012 74.5578229902678) bank82539 +82540 POINT(40.70395373864327 73.18837316210534) bank82540 +82541 POINT(40.21675752407763 73.27493316462235) bank82541 +82542 POINT(40.04206300743591 73.37539358332062) bank82542 +82543 POINT(39.83058356321364 73.79960501550327) bank82543 +82544 POINT(41.421226575287506 73.60939685438306) bank82544 +82545 POINT(39.89744861808054 74.03138311728091) bank82545 +82546 POINT(40.10366479704322 74.78399155477744) bank82546 +82547 POINT(40.09765371232458 74.45195008892287) bank82547 +82548 POINT(41.33302538248452 74.52375920056444) bank82548 +82549 POINT(41.414163831888295 73.51256893605753) bank82549 +82550 POINT(41.136428748837965 73.29800331507131) bank82550 +82551 POINT(41.50781112307057 74.48169115937807) bank82551 +82552 POINT(39.87133120229917 73.77631718444697) bank82552 +82553 POINT(41.15253688026293 74.76051450468859) bank82553 +82554 POINT(39.907588390003504 74.13296452018575) bank82554 +82555 POINT(39.77022016995651 74.44528930703456) bank82555 +82556 POINT(39.8751346161195 74.78271453488212) bank82556 +82557 POINT(41.02516463466424 74.36396570156764) bank82557 +82558 POINT(41.255578025276115 73.8496869586873) bank82558 +82559 POINT(41.54653685926944 74.29114340139392) bank82559 +82560 POINT(41.380523458127364 74.08404635318603) bank82560 +82561 POINT(41.683819223064404 73.69156198779217) bank82561 +82562 POINT(41.383061184680486 74.34582879691483) bank82562 +82563 POINT(41.68269175748626 74.1083118307905) bank82563 +82564 POINT(40.64043651208527 74.9549132280697) bank82564 +82565 POINT(41.437140970389635 74.74135599656792) bank82565 +82566 POINT(41.0097805721026 73.47676648723255) bank82566 +82567 POINT(41.06665389529368 74.11670563553186) bank82567 +82568 POINT(40.95041966699257 74.18282131449516) bank82568 +82569 POINT(41.5111611501055 73.93032497364936) bank82569 +82570 POINT(41.17485902998405 74.44217976371283) bank82570 +82571 POINT(41.48422533355192 74.1870357034056) bank82571 +82572 POINT(41.02088036020013 73.16108234968982) bank82572 +82573 POINT(40.38584816880707 73.8604182395529) bank82573 +82574 POINT(40.39729224682802 73.96113417491338) bank82574 +82575 POINT(41.58555329591784 74.88806087242962) bank82575 +82576 POINT(41.260440776414974 73.70762432762156) bank82576 +82577 POINT(41.02096992493741 74.22089630599416) bank82577 +82578 POINT(41.50762468536325 73.32339394979219) bank82578 +82579 POINT(41.506784909519226 74.42609889605231) bank82579 +82580 POINT(40.11569951950487 74.64197681831702) bank82580 +82581 POINT(39.77705388923552 74.86348179523857) bank82581 +82582 POINT(40.21618562300059 73.46662596233905) bank82582 +82583 POINT(40.10475126887266 74.94685446179321) bank82583 +82584 POINT(41.66435065796492 74.75813758896132) bank82584 +82585 POINT(40.11167945661861 74.38764881315338) bank82585 +82586 POINT(40.42074425127373 73.10497010263681) bank82586 +82587 POINT(41.23958651721921 73.35498003132354) bank82587 +82588 POINT(40.436389679882076 74.02640434952964) bank82588 +82589 POINT(40.11458473674697 74.87621153259428) bank82589 +82590 POINT(41.05734208357458 73.21036749661407) bank82590 +82591 POINT(41.06772585891987 74.70033060511196) bank82591 +82592 POINT(40.85789821255381 73.78592415508419) bank82592 +82593 POINT(40.1871896986082 74.93059372539008) bank82593 +82594 POINT(40.7180121123284 74.22876333881842) bank82594 +82595 POINT(41.578080359006975 73.11136733637204) bank82595 +82596 POINT(40.731283099957714 74.33936499471935) bank82596 +82597 POINT(40.223384756747286 74.34711228767308) bank82597 +82598 POINT(40.3933989943967 73.24485262277571) bank82598 +82599 POINT(41.68486616956666 74.08088618769742) bank82599 +82600 POINT(41.63685457321455 73.29501362608785) bank82600 +82601 POINT(40.42995297046272 73.08118820397067) bank82601 +82602 POINT(40.01680961237427 74.97335252399226) bank82602 +82603 POINT(40.196389481346586 73.97489424115118) bank82603 +82604 POINT(41.41708500092809 73.81894366928665) bank82604 +82605 POINT(40.86825954572507 74.60967901137366) bank82605 +82606 POINT(41.43093420220954 74.50144191612662) bank82606 +82607 POINT(40.48439498926399 73.20057668617075) bank82607 +82608 POINT(41.36843282226018 73.66285939593028) bank82608 +82609 POINT(39.895996613781406 74.43325221726826) bank82609 +82610 POINT(40.03557506035684 74.79520524958406) bank82610 +82611 POINT(40.090417606990854 73.44992040171579) bank82611 +82612 POINT(40.73296920224446 73.99539119919575) bank82612 +82613 POINT(40.718764638388365 74.39452835680594) bank82613 +82614 POINT(40.97540592086973 74.87617946358635) bank82614 +82615 POINT(39.90091843208658 74.81293987604654) bank82615 +82616 POINT(41.378167107785565 73.52947311025572) bank82616 +82617 POINT(41.553446894987985 73.12997346210567) bank82617 +82618 POINT(39.71805252209399 74.07159419673884) bank82618 +82619 POINT(40.72985430808057 73.25063367179906) bank82619 +82620 POINT(41.578523135278594 73.79915959816636) bank82620 +82621 POINT(40.27142572218302 74.16056103425595) bank82621 +82622 POINT(40.23915947087712 74.62406588750696) bank82622 +82623 POINT(41.204253626328196 74.53898894882762) bank82623 +82624 POINT(41.31924283873504 74.0011494651599) bank82624 +82625 POINT(40.29834379104621 73.56774527024245) bank82625 +82626 POINT(40.32064384104966 73.14660531363856) bank82626 +82627 POINT(40.226450829765355 74.68140156060917) bank82627 +82628 POINT(41.13227830721487 74.23920385370549) bank82628 +82629 POINT(39.802689381086836 74.65970776089824) bank82629 +82630 POINT(40.90443097417279 73.4059733783413) bank82630 +82631 POINT(40.864076181322616 73.38416574021976) bank82631 +82632 POINT(41.57548631880301 73.125353747771) bank82632 +82633 POINT(40.39392247278572 73.3701634730494) bank82633 +82634 POINT(40.2989094860665 74.34028854881205) bank82634 +82635 POINT(39.7245237555046 73.92589972246427) bank82635 +82636 POINT(40.32040111137658 74.7442997315169) bank82636 +82637 POINT(41.475134073367734 73.92274387412976) bank82637 +82638 POINT(39.7940094666454 74.62998634105747) bank82638 +82639 POINT(41.1167773362755 73.4450073238624) bank82639 +82640 POINT(41.5423495782808 74.84218846943949) bank82640 +82641 POINT(40.714456476523594 74.56629326697187) bank82641 +82642 POINT(40.28165181247998 73.65118368300635) bank82642 +82643 POINT(40.97348033074872 73.80335875805423) bank82643 +82644 POINT(40.74813134405919 74.20669541962862) bank82644 +82645 POINT(41.15317805425234 74.74768077316503) bank82645 +82646 POINT(40.73133353769937 73.49342705471697) bank82646 +82647 POINT(41.08697742751551 74.95413413083395) bank82647 +82648 POINT(41.374648926401974 74.04059797435389) bank82648 +82649 POINT(41.33884945557062 74.64133561055225) bank82649 +82650 POINT(41.08716594487523 74.59999587720884) bank82650 +82651 POINT(40.701916451327264 73.69170625025441) bank82651 +82652 POINT(41.58915355237007 73.39728717868473) bank82652 +82653 POINT(41.31546496778396 73.94482242154979) bank82653 +82654 POINT(40.25239000136568 73.90809326424163) bank82654 +82655 POINT(41.53130890820053 73.2830636262576) bank82655 +82656 POINT(41.01193535085937 73.49335060780831) bank82656 +82657 POINT(41.08353177959421 73.18953945254403) bank82657 +82658 POINT(41.673145788496214 73.86986096967976) bank82658 +82659 POINT(40.10077663348483 74.07935193999134) bank82659 +82660 POINT(40.087893406706335 74.66135240136717) bank82660 +82661 POINT(39.795005586197796 73.72771759171498) bank82661 +82662 POINT(40.07416646119688 74.5989511813018) bank82662 +82663 POINT(39.800515038434924 73.85886796223512) bank82663 +82664 POINT(40.089895644495655 74.48360650611708) bank82664 +82665 POINT(41.668754752129786 73.78825082716122) bank82665 +82666 POINT(39.740689947359435 74.7926203473294) bank82666 +82667 POINT(40.97273522977363 73.03134138036746) bank82667 +82668 POINT(41.68391840173154 73.21647572018931) bank82668 +82669 POINT(41.119896624874116 73.52390449270807) bank82669 +82670 POINT(41.093493114049195 74.64528414179682) bank82670 +82671 POINT(39.84946877745205 74.3213596028841) bank82671 +82672 POINT(41.21888961609937 73.24696147948491) bank82672 +82673 POINT(41.56654471139267 73.03975464093662) bank82673 +82674 POINT(40.55799929228771 73.87400485356589) bank82674 +82675 POINT(40.4305046456357 73.52375734929291) bank82675 +82676 POINT(40.40069386849479 74.95516568474707) bank82676 +82677 POINT(40.14266515259708 74.04627495676388) bank82677 +82678 POINT(40.73966917844326 73.76272970746643) bank82678 +82679 POINT(40.268117574175086 74.10297763105724) bank82679 +82680 POINT(39.947113002321686 74.77257823603782) bank82680 +82681 POINT(39.72270728337709 73.90726507767742) bank82681 +82682 POINT(41.42280315538043 74.41220176042971) bank82682 +82683 POINT(40.914698653115494 73.80983757941317) bank82683 +82684 POINT(40.427415931920855 73.61308079951557) bank82684 +82685 POINT(39.801178454039224 73.75468867491615) bank82685 +82686 POINT(41.20091573568468 74.87292834170142) bank82686 +82687 POINT(41.36344303709687 74.77298976252) bank82687 +82688 POINT(39.93161551920602 73.60291288358206) bank82688 +82689 POINT(40.69049070670264 73.98292918723644) bank82689 +82690 POINT(41.13018021675395 74.69498049160283) bank82690 +82691 POINT(41.04745463864925 73.82015262120619) bank82691 +82692 POINT(40.312476234012095 74.99473995181553) bank82692 +82693 POINT(40.05260071239945 74.7899212825656) bank82693 +82694 POINT(39.94315448917239 73.02464024822844) bank82694 +82695 POINT(40.22098817691805 73.27476602509444) bank82695 +82696 POINT(40.20201023287466 73.5114210457935) bank82696 +82697 POINT(40.141561713258376 73.76271043800364) bank82697 +82698 POINT(40.83656672643981 73.96161345353904) bank82698 +82699 POINT(39.904504533492855 73.30503462865032) bank82699 +82700 POINT(40.81172616637266 73.33316843232342) bank82700 +82701 POINT(41.27018418151825 74.91976641134707) bank82701 +82702 POINT(40.35199601515542 74.31209851343074) bank82702 +82703 POINT(40.89431427124658 73.6771224710571) bank82703 +82704 POINT(41.598130150806654 73.88291008352263) bank82704 +82705 POINT(40.401181766493195 73.49478370043236) bank82705 +82706 POINT(40.6730713882426 73.39317077122558) bank82706 +82707 POINT(40.13545494681492 73.60418470731915) bank82707 +82708 POINT(39.79704610674527 74.50368210688538) bank82708 +82709 POINT(41.321403695111336 73.72710539935102) bank82709 +82710 POINT(40.92445328791242 74.54783242466762) bank82710 +82711 POINT(40.31484193844078 74.8240497311492) bank82711 +82712 POINT(40.20753122941594 73.7479374419686) bank82712 +82713 POINT(41.20445266089054 74.71969771915069) bank82713 +82714 POINT(40.8144824566919 73.48768067266487) bank82714 +82715 POINT(40.14376418304883 74.28972583879958) bank82715 +82716 POINT(41.084943677845544 73.31477346442162) bank82716 +82717 POINT(40.40293540333778 73.96716927117069) bank82717 +82718 POINT(40.00787806328133 73.88539549452248) bank82718 +82719 POINT(40.36314340193696 74.20820521007863) bank82719 +82720 POINT(41.5183050966044 73.19646879991167) bank82720 +82721 POINT(40.535560116184726 74.74172376010016) bank82721 +82722 POINT(41.270514290737694 73.87260946772409) bank82722 +82723 POINT(39.96666321741907 73.77667829932504) bank82723 +82724 POINT(41.19014211977039 74.92650776871625) bank82724 +82725 POINT(41.19683153295044 73.4938713474909) bank82725 +82726 POINT(41.24438079559075 74.9675400663751) bank82726 +82727 POINT(40.56326659291399 74.42917586434135) bank82727 +82728 POINT(40.09074383283359 73.94061275948432) bank82728 +82729 POINT(40.61262903176542 73.89400936916864) bank82729 +82730 POINT(41.122583410558235 74.38239144295022) bank82730 +82731 POINT(40.06105285130106 74.73461246771024) bank82731 +82732 POINT(40.09969962093822 73.68522809748471) bank82732 +82733 POINT(41.11156285129973 73.68484340300908) bank82733 +82734 POINT(41.231793893151725 74.41611192995651) bank82734 +82735 POINT(41.18079432478767 73.17373124843411) bank82735 +82736 POINT(40.072489768295135 74.92279867837281) bank82736 +82737 POINT(41.29461754759712 73.13036923787394) bank82737 +82738 POINT(41.28048870233265 73.83427380321157) bank82738 +82739 POINT(40.4670545926203 74.98052531749114) bank82739 +82740 POINT(40.78735785021635 74.26488387383105) bank82740 +82741 POINT(39.73199485382614 73.28081172284045) bank82741 +82742 POINT(41.665765957410834 74.36508730499202) bank82742 +82743 POINT(41.196208505828196 74.54433154038183) bank82743 +82744 POINT(39.98883719400898 73.17018025366463) bank82744 +82745 POINT(40.70304753348018 73.43667413810276) bank82745 +82746 POINT(40.2040050544784 73.9067596803398) bank82746 +82747 POINT(41.37080098971202 73.35386318976457) bank82747 +82748 POINT(41.432227024337244 73.86359576371599) bank82748 +82749 POINT(40.26915505309651 73.31453658500155) bank82749 +82750 POINT(41.30248566650441 74.84962315473489) bank82750 +82751 POINT(39.73549072974641 73.16863813020372) bank82751 +82752 POINT(41.09343662008156 74.16720684933063) bank82752 +82753 POINT(40.68101375703368 73.90367865774888) bank82753 +82754 POINT(40.50939945567399 74.77932654639704) bank82754 +82755 POINT(40.30571729769635 73.62852286672613) bank82755 +82756 POINT(40.525201843949944 74.60187544186934) bank82756 +82757 POINT(40.72356646129297 73.59290103822379) bank82757 +82758 POINT(40.1101709883403 74.5076905731064) bank82758 +82759 POINT(41.61097867139307 74.54622571474388) bank82759 +82760 POINT(40.64147709854303 74.8628142377503) bank82760 +82761 POINT(40.88538657678954 74.5149202907775) bank82761 +82762 POINT(40.97408269662892 73.0580556261908) bank82762 +82763 POINT(40.126775052475445 73.04654566257682) bank82763 +82764 POINT(39.96662937684518 74.910061687953) bank82764 +82765 POINT(40.56002226876008 73.37671905886856) bank82765 +82766 POINT(40.6628720078164 73.63573564448508) bank82766 +82767 POINT(41.1833219405239 74.11994060749186) bank82767 +82768 POINT(40.34251693035714 74.62320603164402) bank82768 +82769 POINT(40.62068178024939 74.77278107575452) bank82769 +82770 POINT(41.17812973617241 73.28795149548684) bank82770 +82771 POINT(40.13703619769619 73.37202570165354) bank82771 +82772 POINT(40.77475566774233 74.2513298093205) bank82772 +82773 POINT(41.69895293961226 73.39546336569319) bank82773 +82774 POINT(41.53430866802751 74.0544387490828) bank82774 +82775 POINT(41.50928145462192 73.99821428790769) bank82775 +82776 POINT(40.264070589727375 73.38703612530787) bank82776 +82777 POINT(41.51583017586714 73.89021363917576) bank82777 +82778 POINT(40.770671016218756 74.26406235390337) bank82778 +82779 POINT(40.48662713051807 73.35931276234189) bank82779 +82780 POINT(40.2730389412264 73.9658889492068) bank82780 +82781 POINT(40.75034676433934 73.75485338184468) bank82781 +82782 POINT(40.12865822954809 73.25104226558797) bank82782 +82783 POINT(40.97301541447216 73.90240940474087) bank82783 +82784 POINT(41.33556311230465 74.05227693682681) bank82784 +82785 POINT(41.18939476437316 73.58702399450938) bank82785 +82786 POINT(40.85579349265483 74.28080647457097) bank82786 +82787 POINT(40.23129671765545 74.79133429880487) bank82787 +82788 POINT(41.263820315605805 74.48112113749401) bank82788 +82789 POINT(39.9692909163796 73.55275438861513) bank82789 +82790 POINT(40.81651187183588 73.83780102347225) bank82790 +82791 POINT(40.90458137269991 74.89413703514889) bank82791 +82792 POINT(41.111752474578594 73.74340018685379) bank82792 +82793 POINT(40.526604839196786 73.79262953519792) bank82793 +82794 POINT(39.89422484163273 73.32107889609506) bank82794 +82795 POINT(41.44254999472137 73.11641333912635) bank82795 +82796 POINT(40.33833782715178 74.54494546429605) bank82796 +82797 POINT(39.9364454426071 73.14864907533124) bank82797 +82798 POINT(41.541514093212484 74.1753518496876) bank82798 +82799 POINT(39.80025377521786 74.56973180795212) bank82799 +82800 POINT(41.69320369475627 73.72045657064598) bank82800 +82801 POINT(40.52774866507189 75.00202229813036) bank82801 +82802 POINT(40.18584398938919 73.53957738336366) bank82802 +82803 POINT(40.06661636680367 74.24949469828147) bank82803 +82804 POINT(41.243492197334156 73.30035647931624) bank82804 +82805 POINT(39.81704641956352 74.89850739896416) bank82805 +82806 POINT(40.139064535898235 73.71196449180675) bank82806 +82807 POINT(40.090979233132664 73.81812588134311) bank82807 +82808 POINT(40.564341529491074 73.6617961698379) bank82808 +82809 POINT(39.90798448147316 73.93905385887875) bank82809 +82810 POINT(41.06081501460559 74.8684750058331) bank82810 +82811 POINT(41.598735398450806 73.08371495595384) bank82811 +82812 POINT(41.11635611317866 74.81853458432934) bank82812 +82813 POINT(39.80646688572896 74.59077992586298) bank82813 +82814 POINT(41.15392162238316 74.98447179572094) bank82814 +82815 POINT(40.11970357229561 74.34205686805586) bank82815 +82816 POINT(40.90586812962387 74.30155628961678) bank82816 +82817 POINT(40.748929552782165 74.1251922089579) bank82817 +82818 POINT(41.24033577987124 73.42763793270292) bank82818 +82819 POINT(40.12741311914248 73.82820308510435) bank82819 +82820 POINT(40.85856047730832 74.40481530531459) bank82820 +82821 POINT(40.27095899044323 73.46666542582766) bank82821 +82822 POINT(40.01935087693534 73.59244419060877) bank82822 +82823 POINT(40.5221348621112 73.42149857260775) bank82823 +82824 POINT(40.977760022773545 73.46587565701834) bank82824 +82825 POINT(40.71152686196306 73.35444023824833) bank82825 +82826 POINT(39.839851698356135 74.94124792185355) bank82826 +82827 POINT(40.60577183434955 73.85640175378002) bank82827 +82828 POINT(41.698395464441305 73.15106705939314) bank82828 +82829 POINT(39.765965042441934 73.4522074595565) bank82829 +82830 POINT(40.12560573836612 73.436765183781) bank82830 +82831 POINT(40.3691643607841 74.70133150515453) bank82831 +82832 POINT(39.91753971398608 73.71191599087325) bank82832 +82833 POINT(40.576190274390505 73.91275671997434) bank82833 +82834 POINT(41.18924729451246 74.51957397063977) bank82834 +82835 POINT(40.42012813139947 74.96974681143588) bank82835 +82836 POINT(40.56307696208994 73.99243877548814) bank82836 +82837 POINT(40.323463608704806 74.98046712894386) bank82837 +82838 POINT(41.66699538310676 73.83481477337564) bank82838 +82839 POINT(40.80872214647406 73.00974638443849) bank82839 +82840 POINT(40.57765297893217 73.36167287192715) bank82840 +82841 POINT(40.17389490826052 73.21256872944204) bank82841 +82842 POINT(40.79510420088093 74.73346158808988) bank82842 +82843 POINT(40.81665285636229 73.1806329602309) bank82843 +82844 POINT(40.86957832429704 73.24182128375328) bank82844 +82845 POINT(41.380874066381246 73.34835304811158) bank82845 +82846 POINT(39.9533025320656 73.27722492573481) bank82846 +82847 POINT(40.97149082004554 73.55180480812612) bank82847 +82848 POINT(40.37479014757599 73.42938009091613) bank82848 +82849 POINT(40.30543324204909 73.51983982953615) bank82849 +82850 POINT(41.54212644351679 74.59303847617875) bank82850 +82851 POINT(40.001635477012044 74.17122566182019) bank82851 +82852 POINT(41.12351478269088 73.62876481294693) bank82852 +82853 POINT(39.76803159146371 74.12299084838345) bank82853 +82854 POINT(39.734993297285314 73.63782876942476) bank82854 +82855 POINT(40.74263959064811 74.1743881203776) bank82855 +82856 POINT(41.625218257029296 74.6903763852912) bank82856 +82857 POINT(40.34307243006222 73.89469034808009) bank82857 +82858 POINT(40.614037947462485 73.53177513950362) bank82858 +82859 POINT(41.34266198170715 73.12570917391606) bank82859 +82860 POINT(41.67271062745057 74.51181251321795) bank82860 +82861 POINT(40.69661138163058 74.68045775108781) bank82861 +82862 POINT(40.149445336982026 74.42537194833879) bank82862 +82863 POINT(40.31025648369438 74.77260765682546) bank82863 +82864 POINT(41.54306673732134 73.21248451944138) bank82864 +82865 POINT(40.83648808588921 73.37043729589601) bank82865 +82866 POINT(41.450911256711315 74.86461020299699) bank82866 +82867 POINT(40.38751427215202 74.55654045677582) bank82867 +82868 POINT(41.47845778091411 74.8919612912482) bank82868 +82869 POINT(41.68235998336764 74.73971415584435) bank82869 +82870 POINT(41.674389251147744 73.90594977827945) bank82870 +82871 POINT(40.64048073868964 74.40710119593298) bank82871 +82872 POINT(40.02028752038449 73.0691858766139) bank82872 +82873 POINT(40.94387277995542 74.84001706189919) bank82873 +82874 POINT(40.76959156935474 73.78743739595697) bank82874 +82875 POINT(40.021708500225024 74.85816615074337) bank82875 +82876 POINT(40.06539811778502 74.71837442715339) bank82876 +82877 POINT(41.429915563790864 73.41142286452654) bank82877 +82878 POINT(40.18542560650339 74.53105421665502) bank82878 +82879 POINT(40.91858663894406 73.86809738038322) bank82879 +82880 POINT(41.094227070950126 73.03214167124831) bank82880 +82881 POINT(40.236105895530805 73.52772640699595) bank82881 +82882 POINT(41.4993435646364 74.00795625351755) bank82882 +82883 POINT(40.23237131686414 74.1131156566905) bank82883 +82884 POINT(40.5284228779503 73.37289326203455) bank82884 +82885 POINT(41.408424984499945 74.43199297323983) bank82885 +82886 POINT(41.59574590825619 73.96840012246484) bank82886 +82887 POINT(40.43533305350057 74.4721699264964) bank82887 +82888 POINT(41.065782055526036 74.11338412329991) bank82888 +82889 POINT(40.94159011003726 74.26126628914629) bank82889 +82890 POINT(40.74972625974478 74.80158197106789) bank82890 +82891 POINT(41.360969230143596 73.1193932896567) bank82891 +82892 POINT(40.038817128808546 73.42716859255385) bank82892 +82893 POINT(40.90072915924623 73.48302060490444) bank82893 +82894 POINT(41.4162984944453 74.22789576655903) bank82894 +82895 POINT(40.10422086254385 74.4756025386198) bank82895 +82896 POINT(40.468218751774636 74.59101201169956) bank82896 +82897 POINT(41.102853445868384 73.97369635035615) bank82897 +82898 POINT(41.540678234021925 73.54697881892234) bank82898 +82899 POINT(41.39139051300501 73.39761108270989) bank82899 +82900 POINT(40.8121848797442 73.60317270878016) bank82900 +82901 POINT(40.679951820863096 74.38015708330946) bank82901 +82902 POINT(41.58091042745572 73.36798317309301) bank82902 +82903 POINT(40.60527014363041 74.48979067743392) bank82903 +82904 POINT(39.86954013769452 73.59247181931319) bank82904 +82905 POINT(40.461053219538385 73.32560546241665) bank82905 +82906 POINT(41.313461059813136 73.7452493693847) bank82906 +82907 POINT(39.729985653480625 74.64436642119433) bank82907 +82908 POINT(40.85217745529959 73.45668251969703) bank82908 +82909 POINT(40.011091955659964 73.62921979564945) bank82909 +82910 POINT(41.28038455107866 73.38495415365767) bank82910 +82911 POINT(40.25129662971742 73.21443961279209) bank82911 +82912 POINT(40.1127752042775 73.61527680842813) bank82912 +82913 POINT(40.52862697861826 73.17055253795405) bank82913 +82914 POINT(41.23522844459749 73.31751548592901) bank82914 +82915 POINT(40.68493385249271 74.34127431742789) bank82915 +82916 POINT(40.78633238049814 73.61461449683803) bank82916 +82917 POINT(40.695280728461626 74.92241492346896) bank82917 +82918 POINT(40.80573109229992 74.85086114778406) bank82918 +82919 POINT(40.25183612357036 74.13492417688482) bank82919 +82920 POINT(41.60276411524083 74.53843643092742) bank82920 +82921 POINT(41.33177655645873 73.90904366236312) bank82921 +82922 POINT(41.50483021737136 74.38031462002613) bank82922 +82923 POINT(40.401743904545796 73.26282139846201) bank82923 +82924 POINT(41.05334088635654 74.81167267262269) bank82924 +82925 POINT(41.06674071783948 73.3646296137313) bank82925 +82926 POINT(41.010129804364446 73.93708563962419) bank82926 +82927 POINT(41.038291922420036 73.1208819976383) bank82927 +82928 POINT(41.53292180927359 73.24581371563593) bank82928 +82929 POINT(40.768409536951395 74.77767138274659) bank82929 +82930 POINT(40.053657969045034 74.0021292320822) bank82930 +82931 POINT(41.40521725546638 73.96241167941079) bank82931 +82932 POINT(40.92362528488475 73.4347034245797) bank82932 +82933 POINT(39.7936722822199 73.74481003291862) bank82933 +82934 POINT(40.08659114092632 74.45969007755504) bank82934 +82935 POINT(40.01395313223973 73.65007781116768) bank82935 +82936 POINT(41.3089912629423 73.4432214262368) bank82936 +82937 POINT(40.531954337850095 74.3279166519504) bank82937 +82938 POINT(41.03692304113932 74.40184288051707) bank82938 +82939 POINT(40.572160040070834 73.91980262275906) bank82939 +82940 POINT(39.896942343875224 73.04351804656632) bank82940 +82941 POINT(39.822785634664086 74.80861726937266) bank82941 +82942 POINT(40.55838951634286 74.36226656704244) bank82942 +82943 POINT(40.78422107176558 74.22462174216284) bank82943 +82944 POINT(40.789562457660246 73.11115357389531) bank82944 +82945 POINT(40.962730574358005 73.28055972799834) bank82945 +82946 POINT(40.39422984644699 73.77903340210784) bank82946 +82947 POINT(40.627305192083234 74.91540284281126) bank82947 +82948 POINT(40.67900438114384 73.613059501617) bank82948 +82949 POINT(41.47807812610463 74.13338307491111) bank82949 +82950 POINT(41.56246040306653 73.37881091802804) bank82950 +82951 POINT(41.32646519076495 73.55614947928599) bank82951 +82952 POINT(40.86688167719589 74.82244834821205) bank82952 +82953 POINT(41.6574281203805 74.12394183363884) bank82953 +82954 POINT(39.89019496817277 74.9450553259715) bank82954 +82955 POINT(40.548803816202586 73.68623603365992) bank82955 +82956 POINT(40.761678651094826 73.75365549986712) bank82956 +82957 POINT(41.13155877660544 74.52147519646346) bank82957 +82958 POINT(41.009119978188714 74.73773425278068) bank82958 +82959 POINT(40.19469480528463 74.45426994502559) bank82959 +82960 POINT(39.945598665435995 73.02056867143035) bank82960 +82961 POINT(40.965393901687214 73.6684104884266) bank82961 +82962 POINT(40.44464546957214 73.79865335422205) bank82962 +82963 POINT(39.76086686222324 73.14902446080997) bank82963 +82964 POINT(40.37239180729108 74.22500012912082) bank82964 +82965 POINT(41.71000240944389 73.80442910580518) bank82965 +82966 POINT(41.14927373811813 74.64414826185504) bank82966 +82967 POINT(41.1245650181302 74.38591604921658) bank82967 +82968 POINT(41.57973725830977 74.56092706289968) bank82968 +82969 POINT(40.32670265337888 74.37724582278999) bank82969 +82970 POINT(40.731691490632045 73.13653172393961) bank82970 +82971 POINT(41.65312579916732 74.45901022788296) bank82971 +82972 POINT(40.31304992605216 74.0407410301615) bank82972 +82973 POINT(40.26754738089501 74.5653272115661) bank82973 +82974 POINT(39.94462574363727 74.49542484175707) bank82974 +82975 POINT(41.66446354063969 74.01822786225559) bank82975 +82976 POINT(41.642332593662296 73.05676740373286) bank82976 +82977 POINT(40.663047305132075 73.03385903231442) bank82977 +82978 POINT(40.05606658805382 74.19346726796722) bank82978 +82979 POINT(41.52555385324791 73.86847721411904) bank82979 +82980 POINT(40.89619029930648 73.02554449961104) bank82980 +82981 POINT(39.96182651387428 73.20749318329696) bank82981 +82982 POINT(41.002922547320885 73.72037628684242) bank82982 +82983 POINT(39.88865318846051 73.1848575532586) bank82983 +82984 POINT(41.06364860360289 73.52866907839491) bank82984 +82985 POINT(40.03510199812302 73.59675312762761) bank82985 +82986 POINT(41.18288172634788 73.77091631904018) bank82986 +82987 POINT(40.00841679800078 74.716482176027) bank82987 +82988 POINT(40.26619444613745 73.27838177946491) bank82988 +82989 POINT(39.74433555827768 73.75984977927415) bank82989 +82990 POINT(41.42913129171119 74.67083663284554) bank82990 +82991 POINT(40.39745564353719 73.97820812377586) bank82991 +82992 POINT(41.303412102204746 74.78460581310463) bank82992 +82993 POINT(40.34438563248066 74.15203797599507) bank82993 +82994 POINT(41.06105565747175 73.31614525447002) bank82994 +82995 POINT(40.977213368935566 73.89982598187214) bank82995 +82996 POINT(41.359472441974404 73.58465526432342) bank82996 +82997 POINT(40.97374750171748 74.05908720447194) bank82997 +82998 POINT(40.48803794695364 74.80474345246671) bank82998 +82999 POINT(40.13661785824792 74.47351477182556) bank82999 +83000 POINT(40.71865166054041 74.28529406179824) bank83000 +83001 POINT(39.77987083164796 74.81380485575362) bank83001 +83002 POINT(39.772683241521264 73.70103441192316) bank83002 +83003 POINT(40.31340089372901 73.85006323750966) bank83003 +83004 POINT(40.87322022818558 73.52688828066543) bank83004 +83005 POINT(39.762659413282556 74.9331125618875) bank83005 +83006 POINT(39.990165883280355 74.06252885876177) bank83006 +83007 POINT(41.06328265253076 73.07503076201378) bank83007 +83008 POINT(39.9230738837148 74.69087902341472) bank83008 +83009 POINT(41.466357450513705 73.70169168484395) bank83009 +83010 POINT(41.12728222996931 73.11494181833393) bank83010 +83011 POINT(40.77940394205531 73.82710422386855) bank83011 +83012 POINT(41.460786766020455 74.50254696210317) bank83012 +83013 POINT(41.0054013971757 73.6161153688661) bank83013 +83014 POINT(41.68586212220931 73.87927738884613) bank83014 +83015 POINT(40.68574722636098 73.01954459443384) bank83015 +83016 POINT(41.027541822659934 74.22023184718881) bank83016 +83017 POINT(40.36945907392465 73.79427039005563) bank83017 +83018 POINT(41.699207963080106 73.8777033168561) bank83018 +83019 POINT(41.0575616454866 73.0706647171634) bank83019 +83020 POINT(40.50867297613833 74.63960476222388) bank83020 +83021 POINT(40.645356218232585 74.53388761837489) bank83021 +83022 POINT(41.4984960972988 74.07140468639128) bank83022 +83023 POINT(41.17456703859132 73.8721248833328) bank83023 +83024 POINT(40.127289646639646 74.50529887927924) bank83024 +83025 POINT(41.04603980471374 74.37438925476339) bank83025 +83026 POINT(39.90000763090648 73.34352236853495) bank83026 +83027 POINT(40.86940774024558 74.87968720922086) bank83027 +83028 POINT(39.79570262650491 73.4275691374016) bank83028 +83029 POINT(41.27582617506075 74.90351695090304) bank83029 +83030 POINT(40.97845284468966 73.45914768974052) bank83030 +83031 POINT(41.22761374741132 74.27459494247509) bank83031 +83032 POINT(41.270041883290645 74.34459590701347) bank83032 +83033 POINT(41.21307615057996 73.81668286424117) bank83033 +83034 POINT(41.34148403372871 74.82476976343484) bank83034 +83035 POINT(40.266552350176184 74.09684358705631) bank83035 +83036 POINT(41.13284559743786 74.74817257641921) bank83036 +83037 POINT(41.19188102664055 73.33384545698577) bank83037 +83038 POINT(40.58433744736237 74.63398025837925) bank83038 +83039 POINT(40.117192312284594 73.37821539448674) bank83039 +83040 POINT(41.43881812403422 73.97931130335782) bank83040 +83041 POINT(39.76421110000411 74.5900352049898) bank83041 +83042 POINT(41.48304523741095 73.1003899203769) bank83042 +83043 POINT(40.31885784624762 74.68799128909059) bank83043 +83044 POINT(40.439647995414184 73.77669517150257) bank83044 +83045 POINT(40.2251587496237 74.83853517795549) bank83045 +83046 POINT(41.08369104973688 73.53498747471512) bank83046 +83047 POINT(40.43915701189036 74.94942334816699) bank83047 +83048 POINT(40.21963175828668 73.16055762977054) bank83048 +83049 POINT(40.49166776546763 73.6540269908862) bank83049 +83050 POINT(41.29670921796499 74.74631567407214) bank83050 +83051 POINT(40.76812992243439 73.60243224897579) bank83051 +83052 POINT(41.29330357759157 73.5010467927517) bank83052 +83053 POINT(40.35584306823001 73.04800127503256) bank83053 +83054 POINT(41.236098785103195 73.43054586709728) bank83054 +83055 POINT(40.74867936982301 73.04886218860479) bank83055 +83056 POINT(41.130884966080984 74.68307357494304) bank83056 +83057 POINT(41.51310403034314 74.19771032636801) bank83057 +83058 POINT(41.565783248716286 74.80587048422221) bank83058 +83059 POINT(40.223249924543126 73.53518526763749) bank83059 +83060 POINT(40.88710138690078 74.08577510050287) bank83060 +83061 POINT(40.29916992503631 73.03214440018607) bank83061 +83062 POINT(41.06653345047681 73.80463520760235) bank83062 +83063 POINT(41.43714599720461 73.85835557209516) bank83063 +83064 POINT(40.45289168688162 73.31110383339448) bank83064 +83065 POINT(40.33207358769057 73.1993413143313) bank83065 +83066 POINT(40.27914877631524 74.23582558799811) bank83066 +83067 POINT(39.95005785118766 73.08884343851753) bank83067 +83068 POINT(40.64964119849561 73.63957800860165) bank83068 +83069 POINT(41.52935037830606 73.07166681731992) bank83069 +83070 POINT(41.54461805803393 73.27502616526992) bank83070 +83071 POINT(40.489850443163355 73.62063424346917) bank83071 +83072 POINT(40.12218038618331 73.07874591145108) bank83072 +83073 POINT(40.925956102687074 74.39580870040471) bank83073 +83074 POINT(40.44171604864193 73.27494646443644) bank83074 +83075 POINT(39.83795994715532 73.88534933698057) bank83075 +83076 POINT(40.026106129945475 74.36974857670681) bank83076 +83077 POINT(40.06955987246368 73.37916730779723) bank83077 +83078 POINT(40.085177829253254 74.47738575706614) bank83078 +83079 POINT(39.80478001347914 73.29959475276203) bank83079 +83080 POINT(41.168419640617515 73.266209075213) bank83080 +83081 POINT(40.030618599868816 74.47013414214659) bank83081 +83082 POINT(41.476218014436846 73.285992521511) bank83082 +83083 POINT(40.85351135613392 73.71257056736003) bank83083 +83084 POINT(40.84732189348138 74.31707683326171) bank83084 +83085 POINT(41.18876109167125 73.33150626323928) bank83085 +83086 POINT(41.51153295039207 74.55384624534446) bank83086 +83087 POINT(40.27058518156731 73.08619793445926) bank83087 +83088 POINT(40.86133205422563 74.50406888415333) bank83088 +83089 POINT(41.292406963265115 73.35866327915338) bank83089 +83090 POINT(40.353543142059145 74.81462141324666) bank83090 +83091 POINT(39.715828236477975 74.65199281384271) bank83091 +83092 POINT(41.62706358801972 74.87034491359071) bank83092 +83093 POINT(39.832954531713554 74.7206728902475) bank83093 +83094 POINT(40.21252562760073 73.71780494076981) bank83094 +83095 POINT(41.507188863939064 73.84469083081136) bank83095 +83096 POINT(40.68435411184989 74.5175201530726) bank83096 +83097 POINT(41.67828741395492 74.23200194245224) bank83097 +83098 POINT(41.074596138777245 73.06144663764091) bank83098 +83099 POINT(40.976441632029704 74.30159397605368) bank83099 +83100 POINT(40.905465541389965 74.79415710258293) bank83100 +83101 POINT(40.16850058779209 74.2770911082403) bank83101 +83102 POINT(41.61054521370151 74.96752166579368) bank83102 +83103 POINT(40.01972021068135 74.38930364264232) bank83103 +83104 POINT(40.44067154139213 73.2551954181486) bank83104 +83105 POINT(41.09936236875075 73.9470517721936) bank83105 +83106 POINT(41.48032883541593 74.07275394645471) bank83106 +83107 POINT(41.31544016730103 74.58396670342111) bank83107 +83108 POINT(39.91409447737847 74.29664962592784) bank83108 +83109 POINT(40.40611944337819 73.713384976173) bank83109 +83110 POINT(40.08753744829746 73.32507375057003) bank83110 +83111 POINT(40.67458123215818 73.86198116162929) bank83111 +83112 POINT(41.561529040341966 74.13122177100776) bank83112 +83113 POINT(41.37950777939846 73.83375875674201) bank83113 +83114 POINT(41.28215720076609 73.23363006764413) bank83114 +83115 POINT(40.3819956992623 74.76676685548813) bank83115 +83116 POINT(40.9684003608888 73.7511268084725) bank83116 +83117 POINT(40.66293210966183 73.71214498732292) bank83117 +83118 POINT(41.15498478631724 74.85768265334119) bank83118 +83119 POINT(40.17403600914146 74.07732226759572) bank83119 +83120 POINT(41.13857366121247 73.21297196963266) bank83120 +83121 POINT(40.589305056229314 73.52390274720662) bank83121 +83122 POINT(39.96525884565569 73.70310973460097) bank83122 +83123 POINT(39.7912934956622 73.62221333578916) bank83123 +83124 POINT(39.78426521267608 73.83526538795579) bank83124 +83125 POINT(40.58558624774703 73.88685494984222) bank83125 +83126 POINT(40.978712719123635 74.6790407980962) bank83126 +83127 POINT(39.89799348273094 73.02025131824684) bank83127 +83128 POINT(40.417860690073 74.03391973160906) bank83128 +83129 POINT(41.33211143419176 74.81961323142767) bank83129 +83130 POINT(40.03081742184689 74.06033105658202) bank83130 +83131 POINT(41.69536834955703 74.21507167412705) bank83131 +83132 POINT(40.167000436744416 74.68644521756966) bank83132 +83133 POINT(41.060257911728776 74.53669322400901) bank83133 +83134 POINT(40.484839114666684 73.85069832386218) bank83134 +83135 POINT(40.78208492025592 73.58181417098761) bank83135 +83136 POINT(40.262445197269436 74.55366524057291) bank83136 +83137 POINT(40.708973532465635 74.87274702355725) bank83137 +83138 POINT(40.31131466159735 73.08952060043202) bank83138 +83139 POINT(40.27173366992373 73.73995773512645) bank83139 +83140 POINT(40.563028109397095 74.9813770352891) bank83140 +83141 POINT(40.36195499437991 74.04682778978346) bank83141 +83142 POINT(39.92429938983735 73.57680635876227) bank83142 +83143 POINT(40.474544672976975 74.34769221163539) bank83143 +83144 POINT(41.27312891166663 73.86963625814609) bank83144 +83145 POINT(39.84258742076406 75.0036571975837) bank83145 +83146 POINT(41.296621982641554 74.7110545078663) bank83146 +83147 POINT(40.50363129960286 73.12468136611112) bank83147 +83148 POINT(41.360727418068855 73.34642431304182) bank83148 +83149 POINT(41.4487796759765 74.31877747036704) bank83149 +83150 POINT(41.424779109402216 74.67903845023565) bank83150 +83151 POINT(39.96873309813775 73.53567801678524) bank83151 +83152 POINT(40.8796655911443 74.91711017742439) bank83152 +83153 POINT(41.17409733179654 73.63031268494923) bank83153 +83154 POINT(40.23861740645037 74.38855668715138) bank83154 +83155 POINT(40.11084224861833 74.86381549716893) bank83155 +83156 POINT(39.81595467099259 73.90671926891423) bank83156 +83157 POINT(40.86904923928365 73.22306045843588) bank83157 +83158 POINT(40.108208538687414 74.20772795903765) bank83158 +83159 POINT(40.03978969085954 73.26189678802656) bank83159 +83160 POINT(40.364495252562094 73.01710338329818) bank83160 +83161 POINT(39.864900680081206 74.94264302313637) bank83161 +83162 POINT(40.59706537932947 74.57331357452009) bank83162 +83163 POINT(40.02840655508622 74.6215937755091) bank83163 +83164 POINT(41.55558312178366 73.68261577037359) bank83164 +83165 POINT(40.15893774092227 73.84613370303617) bank83165 +83166 POINT(41.413948529173865 74.80390970007632) bank83166 +83167 POINT(40.104857849178124 74.98440114518618) bank83167 +83168 POINT(41.360255171822075 74.13566455748648) bank83168 +83169 POINT(41.49330501247818 74.31258391478565) bank83169 +83170 POINT(41.58811750609286 73.88289135287229) bank83170 +83171 POINT(39.89502543661594 74.1980110396429) bank83171 +83172 POINT(40.572123305424135 74.55327765506136) bank83172 +83173 POINT(39.86299805042849 73.78418612216181) bank83173 +83174 POINT(40.755453276073496 74.4770725616955) bank83174 +83175 POINT(39.77277285620774 74.39691905510351) bank83175 +83176 POINT(41.26327009876292 74.3807413820444) bank83176 +83177 POINT(40.94447297693114 73.48325879681157) bank83177 +83178 POINT(40.92102016684772 73.08834413290144) bank83178 +83179 POINT(41.25903711089602 73.62516373281501) bank83179 +83180 POINT(40.29371961406713 74.66668726417123) bank83180 +83181 POINT(41.550658445248644 74.92864693231586) bank83181 +83182 POINT(40.37532600054346 73.74776104256554) bank83182 +83183 POINT(41.12112213803597 74.29274703781267) bank83183 +83184 POINT(40.92045720297921 73.2524681606099) bank83184 +83185 POINT(41.22495823939846 74.16239643865767) bank83185 +83186 POINT(41.65684783676991 73.46769466803067) bank83186 +83187 POINT(41.046797469082456 73.06195247798895) bank83187 +83188 POINT(40.99742198330512 73.2393366273586) bank83188 +83189 POINT(41.10429788319887 73.26918612951505) bank83189 +83190 POINT(41.710189825508515 74.81147975183379) bank83190 +83191 POINT(39.873985719995744 73.07118423343562) bank83191 +83192 POINT(41.09854844877895 74.7860835529311) bank83192 +83193 POINT(39.85018832662652 74.76835748401194) bank83193 +83194 POINT(41.14060253548098 74.25148864380421) bank83194 +83195 POINT(39.91997559103757 74.5396601764125) bank83195 +83196 POINT(41.03263663901784 73.11925523795446) bank83196 +83197 POINT(40.8778371225985 73.45404212370867) bank83197 +83198 POINT(40.12821626659859 74.94033144249067) bank83198 +83199 POINT(40.06316958513235 73.64987236644279) bank83199 +83200 POINT(40.694293097365964 73.97161993057514) bank83200 +83201 POINT(41.04630110307597 74.70796523151601) bank83201 +83202 POINT(40.27621199234974 74.62865026979804) bank83202 +83203 POINT(41.54295646159261 74.4827393191739) bank83203 +83204 POINT(40.28898334374998 74.70223743113955) bank83204 +83205 POINT(41.67406198690936 74.26999380241467) bank83205 +83206 POINT(41.19168498920913 74.6596974951513) bank83206 +83207 POINT(40.383382097991785 74.44364559844804) bank83207 +83208 POINT(41.564301132226845 74.74660178093156) bank83208 +83209 POINT(41.32159857264539 74.66682436552748) bank83209 +83210 POINT(40.992731410048854 73.58589136724905) bank83210 +83211 POINT(41.417868264649655 74.44210230822682) bank83211 +83212 POINT(40.03452884884136 74.6989089150823) bank83212 +83213 POINT(41.67822670274012 74.4591043099007) bank83213 +83214 POINT(40.134651986027244 74.07571100906374) bank83214 +83215 POINT(41.58278743269155 73.4500454549259) bank83215 +83216 POINT(41.64784375214686 73.43798579123794) bank83216 +83217 POINT(39.717822606032726 73.98693257198745) bank83217 +83218 POINT(40.73798226257644 74.23596409911669) bank83218 +83219 POINT(41.355530062321144 73.67155069034993) bank83219 +83220 POINT(41.621020404633 73.03530975847845) bank83220 +83221 POINT(41.01291706847652 73.13583346280869) bank83221 +83222 POINT(41.59860064722493 73.13907602029731) bank83222 +83223 POINT(40.74972361953655 73.73207331633209) bank83223 +83224 POINT(40.20872103908572 73.51532845062725) bank83224 +83225 POINT(40.03895850039529 74.67477939309678) bank83225 +83226 POINT(40.452141855746476 74.21176505915433) bank83226 +83227 POINT(40.67478955202903 74.9001435955425) bank83227 +83228 POINT(40.23971540380917 74.37360481391188) bank83228 +83229 POINT(40.400381381317466 73.04307854618312) bank83229 +83230 POINT(41.49051711622972 73.9174520745085) bank83230 +83231 POINT(39.9287141657393 74.73188383075822) bank83231 +83232 POINT(41.100124128877766 74.99429106250886) bank83232 +83233 POINT(40.87018610553766 74.45005616058236) bank83233 +83234 POINT(39.894376859554676 73.70455560212163) bank83234 +83235 POINT(40.68067265456766 74.50285748238363) bank83235 +83236 POINT(39.7683411395558 74.58949680491118) bank83236 +83237 POINT(40.13782539713829 73.05884791237774) bank83237 +83238 POINT(40.18122113941018 74.00943058031757) bank83238 +83239 POINT(40.73402270130573 74.61485697352174) bank83239 +83240 POINT(40.05320070411501 74.01077725348236) bank83240 +83241 POINT(41.67861372607072 74.37835486715191) bank83241 +83242 POINT(39.86836248596731 74.71602468211769) bank83242 +83243 POINT(40.333615564391344 74.37593050313016) bank83243 +83244 POINT(39.823824742915946 74.76080088842173) bank83244 +83245 POINT(40.68001506263372 73.52398182353785) bank83245 +83246 POINT(40.023682888845066 73.42157206673023) bank83246 +83247 POINT(40.84971243905194 73.37622420874236) bank83247 +83248 POINT(40.98388265415724 73.21222637870666) bank83248 +83249 POINT(39.772018395104624 73.19739769424577) bank83249 +83250 POINT(41.12002958180913 73.42913867800573) bank83250 +83251 POINT(40.70813767958044 73.59253619332128) bank83251 +83252 POINT(41.021508148928525 74.79218987398166) bank83252 +83253 POINT(40.781425416917244 73.15814866415373) bank83253 +83254 POINT(40.017132433504806 74.4742447823245) bank83254 +83255 POINT(41.25626217866171 73.02696317249989) bank83255 +83256 POINT(41.46773522836047 73.65882883049031) bank83256 +83257 POINT(40.662430814294176 74.33548469953328) bank83257 +83258 POINT(39.80328501490377 74.66368660357786) bank83258 +83259 POINT(40.93754421052559 74.7695811504085) bank83259 +83260 POINT(40.587923996056745 74.1527492151885) bank83260 +83261 POINT(41.54570438925775 74.34194987697681) bank83261 +83262 POINT(39.90245307107966 73.31560358059703) bank83262 +83263 POINT(40.808862996220775 73.20815574710953) bank83263 +83264 POINT(41.368570148319094 73.44952850060662) bank83264 +83265 POINT(41.06523866886855 74.31324868376512) bank83265 +83266 POINT(40.888723036854486 73.18377190375945) bank83266 +83267 POINT(40.677102535419394 74.01875579130382) bank83267 +83268 POINT(39.97640088470676 74.20404548823672) bank83268 +83269 POINT(41.32031074738879 74.80970306586462) bank83269 +83270 POINT(39.99036783270435 73.26090475376944) bank83270 +83271 POINT(40.7519221175596 74.91307711742971) bank83271 +83272 POINT(40.801511470765895 74.84728679900824) bank83272 +83273 POINT(39.83770243720809 73.43906212780443) bank83273 +83274 POINT(39.86805525012802 73.58914865569305) bank83274 +83275 POINT(41.23260847785972 74.44934651532327) bank83275 +83276 POINT(40.17431283811327 74.7536707369537) bank83276 +83277 POINT(40.57659589058892 74.09084656117336) bank83277 +83278 POINT(40.175979683149286 74.65231173406258) bank83278 +83279 POINT(40.91847150284547 73.23945685883774) bank83279 +83280 POINT(40.91299033402795 73.73056140272811) bank83280 +83281 POINT(40.39136232005541 73.03221311040352) bank83281 +83282 POINT(40.139122070760415 73.8837262385838) bank83282 +83283 POINT(40.74228585589931 73.60927764476544) bank83283 +83284 POINT(40.23683693646588 73.62343225002732) bank83284 +83285 POINT(40.882997628613914 74.54261985415404) bank83285 +83286 POINT(40.35006828257605 73.61034675095489) bank83286 +83287 POINT(41.13855405275211 73.73641495393251) bank83287 +83288 POINT(41.68201241935325 74.73371999433704) bank83288 +83289 POINT(41.09659216016342 74.40664563930802) bank83289 +83290 POINT(40.557214293214095 74.1139177245313) bank83290 +83291 POINT(40.821580565798904 73.97770389943052) bank83291 +83292 POINT(39.910524567857735 73.2583329620223) bank83292 +83293 POINT(41.600054596355 74.56741042858043) bank83293 +83294 POINT(40.4376968438927 73.14773927142707) bank83294 +83295 POINT(40.666973266153114 73.11068410521703) bank83295 +83296 POINT(39.841085422028414 73.80459993635583) bank83296 +83297 POINT(40.73292095988197 73.54005912004965) bank83297 +83298 POINT(39.962354974381334 73.30554392356005) bank83298 +83299 POINT(40.33248676225697 73.17435940297628) bank83299 +83300 POINT(40.267208727534396 73.16280245798782) bank83300 +83301 POINT(40.800188478258136 73.27976300928252) bank83301 +83302 POINT(40.19099474310611 73.95726563901844) bank83302 +83303 POINT(40.52688952898166 74.70257081408049) bank83303 +83304 POINT(40.04547546594172 74.66637563729496) bank83304 +83305 POINT(41.18232958095644 73.85211131271507) bank83305 +83306 POINT(41.692176487634306 73.31672207716508) bank83306 +83307 POINT(39.87316872376452 74.92615779645371) bank83307 +83308 POINT(41.66220917628225 73.17432112460416) bank83308 +83309 POINT(40.279050831161115 75.00157111001593) bank83309 +83310 POINT(39.84344499675045 74.37083100752184) bank83310 +83311 POINT(40.59167039548196 73.55485543317225) bank83311 +83312 POINT(41.54249150678892 73.79646829077748) bank83312 +83313 POINT(39.94764792192113 74.7910875204916) bank83313 +83314 POINT(40.39964394879862 74.2886586872749) bank83314 +83315 POINT(40.83685101359697 73.70482573082218) bank83315 +83316 POINT(40.44964400072956 74.53819302987854) bank83316 +83317 POINT(40.679508895604506 74.08469467772574) bank83317 +83318 POINT(41.68541991219192 74.83292031595501) bank83318 +83319 POINT(41.23238793222016 73.01280382842607) bank83319 +83320 POINT(41.56494421707565 73.1469333025726) bank83320 +83321 POINT(41.5252950886014 73.11075149703117) bank83321 +83322 POINT(40.41418800535008 73.30065451383673) bank83322 +83323 POINT(41.02969470553512 74.30940949031232) bank83323 +83324 POINT(41.47038906743588 73.06314038425629) bank83324 +83325 POINT(41.03312575497285 74.77083265995972) bank83325 +83326 POINT(40.12945435952656 73.23182318544433) bank83326 +83327 POINT(40.21272453842973 74.63441118612837) bank83327 +83328 POINT(41.28823583325552 74.32918843111759) bank83328 +83329 POINT(41.181695449604234 74.67091612053079) bank83329 +83330 POINT(40.43981957729092 74.47125965280385) bank83330 +83331 POINT(39.82727320000038 74.91263874718594) bank83331 +83332 POINT(40.979197827868575 74.42752073159842) bank83332 +83333 POINT(41.381397100518726 73.9102380382959) bank83333 +83334 POINT(41.04438187707892 73.64134819787682) bank83334 +83335 POINT(41.39072706266957 73.15817995451812) bank83335 +83336 POINT(40.27660984995944 74.98033645442935) bank83336 +83337 POINT(41.680520279501906 73.15358424294321) bank83337 +83338 POINT(40.16561438351784 74.54747449170864) bank83338 +83339 POINT(40.42452478350374 74.17779959559383) bank83339 +83340 POINT(40.30763433077804 73.6144711948728) bank83340 +83341 POINT(39.86644335550997 74.58774872498013) bank83341 +83342 POINT(40.9884527610097 73.72573096330218) bank83342 +83343 POINT(40.12245068740446 73.57531098584126) bank83343 +83344 POINT(40.01357013217217 73.22996304371075) bank83344 +83345 POINT(41.64861008781224 73.20450275555137) bank83345 +83346 POINT(40.59393067059832 73.38792033698283) bank83346 +83347 POINT(41.23638837963829 73.64513438678637) bank83347 +83348 POINT(41.55587233575221 73.10532640399211) bank83348 +83349 POINT(40.101078950630395 73.50812244226297) bank83349 +83350 POINT(40.99133816547211 73.69459847874836) bank83350 +83351 POINT(41.41718142999627 73.92445503694329) bank83351 +83352 POINT(39.88106914045893 74.02143003049534) bank83352 +83353 POINT(40.19859771730436 73.488781669145) bank83353 +83354 POINT(40.47606897406772 74.91062100527502) bank83354 +83355 POINT(40.57168660413669 74.27792045415437) bank83355 +83356 POINT(40.186171186013546 73.58984224355089) bank83356 +83357 POINT(40.88170972358111 73.12711130684669) bank83357 +83358 POINT(40.60983446532006 74.78515250740608) bank83358 +83359 POINT(41.53940722332044 73.67536360349428) bank83359 +83360 POINT(40.0075026265465 73.30663876827973) bank83360 +83361 POINT(41.688463304049854 73.70660070827641) bank83361 +83362 POINT(40.22740471728654 73.35100004457466) bank83362 +83363 POINT(39.96789749804807 74.96871907729354) bank83363 +83364 POINT(40.79598936906384 74.92670834678333) bank83364 +83365 POINT(41.030903889415164 73.54826873913777) bank83365 +83366 POINT(39.77324830416357 74.53133264056243) bank83366 +83367 POINT(40.76214053933316 74.82401148598277) bank83367 +83368 POINT(40.37151011118227 73.06088124099001) bank83368 +83369 POINT(41.14640103587078 74.96309591478278) bank83369 +83370 POINT(40.44631093591589 73.98956093480756) bank83370 +83371 POINT(40.40533927663129 73.39319791068706) bank83371 +83372 POINT(39.93342445327002 73.15899564178267) bank83372 +83373 POINT(40.65446124208297 74.6126637135997) bank83373 +83374 POINT(40.90164277693208 74.19083272840274) bank83374 +83375 POINT(39.96137904762438 74.39340423129157) bank83375 +83376 POINT(40.68374429153417 74.22301397483828) bank83376 +83377 POINT(41.39631746707481 73.07254047424057) bank83377 +83378 POINT(40.00626308002526 74.41060109963058) bank83378 +83379 POINT(41.17106238641135 73.35571146006635) bank83379 +83380 POINT(40.69120055579534 73.59448984958105) bank83380 +83381 POINT(40.050679308409414 74.72155521000418) bank83381 +83382 POINT(40.21131022384851 74.1236984512682) bank83382 +83383 POINT(39.97442309935703 74.7617312782966) bank83383 +83384 POINT(41.14546780940858 73.22352816460214) bank83384 +83385 POINT(40.97781642209238 74.89700427072142) bank83385 +83386 POINT(39.812067983155586 74.32316604693631) bank83386 +83387 POINT(40.23420687997988 74.71556601895323) bank83387 +83388 POINT(41.39673534555281 73.741418010799) bank83388 +83389 POINT(41.27223163153311 73.7975288486575) bank83389 +83390 POINT(40.344232455747374 74.95608500665065) bank83390 +83391 POINT(41.16714490726876 73.59457251970788) bank83391 +83392 POINT(40.001153449585885 73.19972597817899) bank83392 +83393 POINT(40.895644760221835 73.56354150397075) bank83393 +83394 POINT(39.83143846815817 74.41678002191594) bank83394 +83395 POINT(41.67777821979159 73.5429012355052) bank83395 +83396 POINT(40.991284984309004 74.13866524472208) bank83396 +83397 POINT(41.39824046756005 73.65056969328238) bank83397 +83398 POINT(39.97041872529245 73.099303423488) bank83398 +83399 POINT(39.93733083036455 74.44206215129252) bank83399 +83400 POINT(39.952090610041566 73.94771372628274) bank83400 +83401 POINT(40.686913252269626 73.79853880822999) bank83401 +83402 POINT(41.51981750649179 73.67717196783853) bank83402 +83403 POINT(40.5937062906487 73.4546689398515) bank83403 +83404 POINT(39.7400902599931 74.8667113650214) bank83404 +83405 POINT(41.46764869081392 74.09445147373702) bank83405 +83406 POINT(40.3480524343032 73.09071796230931) bank83406 +83407 POINT(40.56706262921527 74.8021468282461) bank83407 +83408 POINT(41.637039865245534 74.12314075594365) bank83408 +83409 POINT(41.333019561529255 74.93272874097264) bank83409 +83410 POINT(40.61869015921654 74.48537257262899) bank83410 +83411 POINT(40.74922666010811 73.83798933563388) bank83411 +83412 POINT(41.70749299501606 73.50651722138878) bank83412 +83413 POINT(40.865684613783166 74.19640636183878) bank83413 +83414 POINT(41.67078240468554 74.059154103204) bank83414 +83415 POINT(41.06534262328636 74.76703614774871) bank83415 +83416 POINT(39.739075464473984 73.29656976757389) bank83416 +83417 POINT(41.28085848003956 74.88374611317639) bank83417 +83418 POINT(40.32673061391026 73.47446514876987) bank83418 +83419 POINT(41.084043348759046 74.9823301467806) bank83419 +83420 POINT(40.91599529077113 74.09974139388076) bank83420 +83421 POINT(40.924320329398924 73.65578072816713) bank83421 +83422 POINT(40.817335773847866 74.62947135159197) bank83422 +83423 POINT(40.61732032859605 73.96660633869928) bank83423 +83424 POINT(41.34567650734533 74.5901398564014) bank83424 +83425 POINT(41.295125889471024 74.37883075785784) bank83425 +83426 POINT(41.52692798708824 74.37905761314151) bank83426 +83427 POINT(40.10973403028089 74.90438559559915) bank83427 +83428 POINT(40.622155594232105 74.39484738723573) bank83428 +83429 POINT(39.824805374083375 73.24138274392256) bank83429 +83430 POINT(40.771862426074485 73.85585860158712) bank83430 +83431 POINT(39.996265094780625 73.18234071031688) bank83431 +83432 POINT(40.42431691970248 74.65439953982089) bank83432 +83433 POINT(41.12187796948979 73.29891043825313) bank83433 +83434 POINT(40.10655203338633 73.0362504106492) bank83434 +83435 POINT(40.17015679762164 73.16214886412156) bank83435 +83436 POINT(40.03211975160807 73.54310879264129) bank83436 +83437 POINT(39.86906567127028 73.52455470013697) bank83437 +83438 POINT(41.501674235566625 73.04298869223086) bank83438 +83439 POINT(40.513805069468255 74.8656059665842) bank83439 +83440 POINT(40.374520098976745 73.53743060715362) bank83440 +83441 POINT(41.130283333444595 73.09055451217071) bank83441 +83442 POINT(41.606605379443636 74.66074344442721) bank83442 +83443 POINT(39.853964439193184 74.87488917158234) bank83443 +83444 POINT(40.44959118546782 74.59085589091818) bank83444 +83445 POINT(40.58541794205863 73.70672873739876) bank83445 +83446 POINT(41.06794008319025 74.18236346947751) bank83446 +83447 POINT(40.90467766290448 74.780724635115) bank83447 +83448 POINT(40.725345624654025 74.01725013842983) bank83448 +83449 POINT(40.125368934921475 74.92490915832865) bank83449 +83450 POINT(41.532798919675216 73.43112944129766) bank83450 +83451 POINT(40.64485297885803 74.39950645347814) bank83451 +83452 POINT(40.14150032413849 74.48437271254663) bank83452 +83453 POINT(41.306775543012805 74.34781169866146) bank83453 +83454 POINT(40.04411073147488 73.5440325186628) bank83454 +83455 POINT(41.30563584903848 73.62982804415392) bank83455 +83456 POINT(40.70382022971954 74.89285299063435) bank83456 +83457 POINT(40.310583769286794 74.28426689295154) bank83457 +83458 POINT(40.656374911548355 74.39532591775095) bank83458 +83459 POINT(41.70445635928742 74.26097550966448) bank83459 +83460 POINT(40.949361652471644 74.07896054095785) bank83460 +83461 POINT(40.911464434928526 74.68663615547662) bank83461 +83462 POINT(40.404938575818846 74.7572333254097) bank83462 +83463 POINT(40.55617835628138 73.11235149392218) bank83463 +83464 POINT(40.91102756605328 74.68999138512976) bank83464 +83465 POINT(41.234337788411445 73.41339138895596) bank83465 +83466 POINT(40.438998416397475 73.51290473371441) bank83466 +83467 POINT(40.73154772833169 74.83552615061258) bank83467 +83468 POINT(41.05862241804062 74.32463192109365) bank83468 +83469 POINT(41.33190909918288 73.16791186006914) bank83469 +83470 POINT(40.918592072616605 73.13903723296248) bank83470 +83471 POINT(40.52177867423278 74.07282116643272) bank83471 +83472 POINT(40.90153502495837 73.78610981717691) bank83472 +83473 POINT(41.539115463872776 74.09173300358816) bank83473 +83474 POINT(41.43564284792012 73.40659109713553) bank83474 +83475 POINT(39.816543394957044 74.9958909216942) bank83475 +83476 POINT(41.30697845092065 74.79170472862404) bank83476 +83477 POINT(40.647222598109835 73.83616410559793) bank83477 +83478 POINT(41.49117754762466 74.26084040370772) bank83478 +83479 POINT(41.63467128392558 73.93427427246888) bank83479 +83480 POINT(40.80170160570385 73.06915125922735) bank83480 +83481 POINT(40.46641531249097 74.23542775568126) bank83481 +83482 POINT(40.37197857034385 74.95107275195356) bank83482 +83483 POINT(40.12068332469497 73.82995040231042) bank83483 +83484 POINT(40.10607485337249 74.09715262092496) bank83484 +83485 POINT(39.76117868658123 74.14278648046043) bank83485 +83486 POINT(40.623209044567325 73.20748752509152) bank83486 +83487 POINT(41.12028000986953 73.93436077720928) bank83487 +83488 POINT(40.512560633959964 74.79527910997983) bank83488 +83489 POINT(40.58192170015124 74.9572373202656) bank83489 +83490 POINT(40.1151423039838 74.28285696153735) bank83490 +83491 POINT(41.688618964875324 73.93856517111527) bank83491 +83492 POINT(40.5030863486239 73.97137961765847) bank83492 +83493 POINT(41.460502602205445 73.53277420210999) bank83493 +83494 POINT(41.3919868804152 74.64058956776525) bank83494 +83495 POINT(40.575354994613626 74.04364814326904) bank83495 +83496 POINT(40.896454619261974 73.4088961583686) bank83496 +83497 POINT(41.198530906610884 74.60117816058347) bank83497 +83498 POINT(39.86892862001194 74.44959872308712) bank83498 +83499 POINT(40.2217502933854 73.02439652637251) bank83499 +83500 POINT(41.70487307874617 74.86922487865151) bank83500 +83501 POINT(41.32840357918767 74.01823644772841) bank83501 +83502 POINT(41.5455452135174 74.34234224462868) bank83502 +83503 POINT(40.88069442929761 74.73041811758837) bank83503 +83504 POINT(40.894243076729474 73.63930522686739) bank83504 +83505 POINT(41.325321499000054 73.87163162627473) bank83505 +83506 POINT(40.625735742178904 74.67124725244074) bank83506 +83507 POINT(41.667003667865536 73.71790812063684) bank83507 +83508 POINT(41.66528168239816 74.25343913190483) bank83508 +83509 POINT(40.68446118315114 74.99860392225665) bank83509 +83510 POINT(40.190883591441946 73.90879588251534) bank83510 +83511 POINT(40.83060262059374 73.06479358561431) bank83511 +83512 POINT(41.50253178774222 74.35094315926996) bank83512 +83513 POINT(40.95724613383343 74.50449003170995) bank83513 +83514 POINT(40.047876071819616 74.30301641628925) bank83514 +83515 POINT(40.49448540292308 74.65162596092414) bank83515 +83516 POINT(40.61071001554223 73.14440882495781) bank83516 +83517 POINT(40.09082260250792 73.13089312895713) bank83517 +83518 POINT(39.98610719674527 73.84433957570468) bank83518 +83519 POINT(40.247717307056824 74.711511630381) bank83519 +83520 POINT(41.47947633647121 73.03843249893981) bank83520 +83521 POINT(41.32760008529028 73.87645912366132) bank83521 +83522 POINT(40.03982403584787 73.65337708214966) bank83522 +83523 POINT(41.35951471250486 74.19163853923467) bank83523 +83524 POINT(41.32211531939627 74.51649462967399) bank83524 +83525 POINT(39.960286237300615 73.84023574117755) bank83525 +83526 POINT(41.69841946525686 74.8523452282381) bank83526 +83527 POINT(40.26348530370566 74.42267209291276) bank83527 +83528 POINT(40.074345756891525 74.96045663353686) bank83528 +83529 POINT(41.27649558461772 74.11086974455819) bank83529 +83530 POINT(41.28428299892997 74.9735009489602) bank83530 +83531 POINT(39.961043746100636 73.11465504653125) bank83531 +83532 POINT(40.12418412676506 74.40737943598765) bank83532 +83533 POINT(40.38387757905079 73.5413092242673) bank83533 +83534 POINT(40.85386785003561 73.38228460635946) bank83534 +83535 POINT(40.16450995013593 74.71548455858486) bank83535 +83536 POINT(41.7119035015292 74.05825520816306) bank83536 +83537 POINT(40.022417551170044 73.42708179253644) bank83537 +83538 POINT(39.75930261404575 74.36036968896241) bank83538 +83539 POINT(40.29886270293802 73.7716405804437) bank83539 +83540 POINT(40.25779844649167 74.56405673337134) bank83540 +83541 POINT(41.055593516667884 73.58728375129749) bank83541 +83542 POINT(40.7814899297025 74.46765749248839) bank83542 +83543 POINT(41.189768781966556 74.19315438008913) bank83543 +83544 POINT(41.20293131917757 73.91852199147458) bank83544 +83545 POINT(41.273487733803535 74.50576835501565) bank83545 +83546 POINT(41.66520072953448 73.04038159062164) bank83546 +83547 POINT(41.28464570119147 73.89022688917073) bank83547 +83548 POINT(39.89206556264166 73.93715093279391) bank83548 +83549 POINT(41.65738669697124 74.53804622767673) bank83549 +83550 POINT(40.53215689192506 73.45503058409466) bank83550 +83551 POINT(40.90476233704094 73.92130534446501) bank83551 +83552 POINT(40.08938318450983 74.71054776143464) bank83552 +83553 POINT(41.26385057107985 73.56859148167523) bank83553 +83554 POINT(40.298291245091356 73.04069286903524) bank83554 +83555 POINT(39.7175016355507 74.54915824965893) bank83555 +83556 POINT(40.63450815638605 74.35540169114458) bank83556 +83557 POINT(40.75285028548391 74.99218384861992) bank83557 +83558 POINT(41.42514909608961 73.5699926833176) bank83558 +83559 POINT(40.10016217555989 74.34029034782806) bank83559 +83560 POINT(40.41729385420879 74.84118033182114) bank83560 +83561 POINT(40.919549832592274 74.33092414364319) bank83561 +83562 POINT(40.33755892391504 74.98095199879486) bank83562 +83563 POINT(40.94352639947126 74.78997417764315) bank83563 +83564 POINT(40.28743351127154 73.96089247425851) bank83564 +83565 POINT(40.68370890055074 74.65657458634196) bank83565 +83566 POINT(40.50161062171673 73.89440096408413) bank83566 +83567 POINT(40.92769091591402 73.11615428770949) bank83567 +83568 POINT(40.36886894907883 74.30517833821777) bank83568 +83569 POINT(40.701133214046784 74.19281254360453) bank83569 +83570 POINT(41.22195917524551 73.40797376334004) bank83570 +83571 POINT(40.452858966705165 74.2582015591872) bank83571 +83572 POINT(41.61861712932029 74.11799734840629) bank83572 +83573 POINT(41.4965180582702 74.3885243209627) bank83573 +83574 POINT(40.43569940181368 73.214101004063) bank83574 +83575 POINT(40.438420629806565 74.85156900798468) bank83575 +83576 POINT(41.62450796618112 74.51460808972213) bank83576 +83577 POINT(40.38143594272532 74.62767351599638) bank83577 +83578 POINT(40.920480209129586 74.74053115631166) bank83578 +83579 POINT(39.94981651230248 73.18236723213907) bank83579 +83580 POINT(40.71495950164568 73.62384396395493) bank83580 +83581 POINT(39.915870213603434 74.72247511936361) bank83581 +83582 POINT(40.705772514682046 74.67213909705389) bank83582 +83583 POINT(40.42036608402807 74.90647698359405) bank83583 +83584 POINT(40.01336943175797 74.34863691079647) bank83584 +83585 POINT(41.460633053724706 75.00420652401347) bank83585 +83586 POINT(40.86211975542276 73.48747700525011) bank83586 +83587 POINT(40.23286649689447 74.40972431847963) bank83587 +83588 POINT(41.61242056581602 73.44916986289758) bank83588 +83589 POINT(41.16485317079517 74.6692315885601) bank83589 +83590 POINT(40.311502280035086 73.9199465436392) bank83590 +83591 POINT(40.11977935894285 74.31403825544517) bank83591 +83592 POINT(40.07157630643595 73.11148752515837) bank83592 +83593 POINT(40.40144532736179 73.17032684552343) bank83593 +83594 POINT(41.119934791372486 73.22017934218917) bank83594 +83595 POINT(40.751266121304944 73.47320451701006) bank83595 +83596 POINT(41.15680785633058 74.91082940055698) bank83596 +83597 POINT(41.63180274698152 74.6599048225731) bank83597 +83598 POINT(41.005354465228514 73.01194470986593) bank83598 +83599 POINT(39.82403661668581 73.63651056051373) bank83599 +83600 POINT(40.559949999490115 74.7665691091616) bank83600 +83601 POINT(41.07481341657963 73.24899333324703) bank83601 +83602 POINT(39.99214661212604 73.39699258293395) bank83602 +83603 POINT(40.95292969104476 73.29191718883635) bank83603 +83604 POINT(40.159771997116806 74.33177762170979) bank83604 +83605 POINT(40.70722311129771 74.11909047864971) bank83605 +83606 POINT(40.81398919166532 73.27414281035388) bank83606 +83607 POINT(40.67070936580933 73.62284568683643) bank83607 +83608 POINT(41.451046787060044 73.77818264858068) bank83608 +83609 POINT(40.15819679787097 73.77013236504135) bank83609 +83610 POINT(39.97792442628453 73.79004836128516) bank83610 +83611 POINT(40.5314593416753 74.01347809632631) bank83611 +83612 POINT(40.042365955007824 73.80624436030553) bank83612 +83613 POINT(40.67434462757057 73.73386346929277) bank83613 +83614 POINT(40.929733243396676 73.38301150820097) bank83614 +83615 POINT(40.29683079747376 73.9937581964566) bank83615 +83616 POINT(40.7564944144051 73.75266934936231) bank83616 +83617 POINT(39.95900965445852 74.70184944895801) bank83617 +83618 POINT(40.58287339353765 73.01245772244478) bank83618 +83619 POINT(41.70366043480336 73.61793412407621) bank83619 +83620 POINT(40.2637903029233 74.92977914404948) bank83620 +83621 POINT(40.511260704042584 74.699827135082) bank83621 +83622 POINT(40.344911991667594 74.64174074086117) bank83622 +83623 POINT(40.95131017955546 73.79947542333716) bank83623 +83624 POINT(40.03734820993531 74.75647173249651) bank83624 +83625 POINT(39.719245636214374 73.83363527345615) bank83625 +83626 POINT(41.26957296404122 73.52234314173334) bank83626 +83627 POINT(40.314497777435676 74.12450172609768) bank83627 +83628 POINT(41.48251533860596 73.69939206874922) bank83628 +83629 POINT(40.16481985940876 74.07380670507897) bank83629 +83630 POINT(41.19997342191605 73.98795566710155) bank83630 +83631 POINT(41.570877485670955 74.17917962742375) bank83631 +83632 POINT(40.915284204632854 73.77572532102036) bank83632 +83633 POINT(40.58434622455457 74.31109053031145) bank83633 +83634 POINT(39.72715082139426 73.95257763202505) bank83634 +83635 POINT(40.87884667050985 73.28590097136122) bank83635 +83636 POINT(39.72599278984758 73.58033706023706) bank83636 +83637 POINT(39.74107185553776 74.63860573083775) bank83637 +83638 POINT(39.96014968793234 74.84246350864696) bank83638 +83639 POINT(40.6148420151277 74.13379713770013) bank83639 +83640 POINT(41.23487285585673 73.43576878178811) bank83640 +83641 POINT(40.5428865167279 73.73559972529758) bank83641 +83642 POINT(41.55064701718955 73.4248162170227) bank83642 +83643 POINT(40.19904512241436 74.9459527818835) bank83643 +83644 POINT(40.01310769136978 73.89446921157268) bank83644 +83645 POINT(41.09981075312202 73.73685002675758) bank83645 +83646 POINT(40.45103008178605 73.21261168374163) bank83646 +83647 POINT(41.5798732753724 73.65950671288645) bank83647 +83648 POINT(40.16640571710021 74.61947365341076) bank83648 +83649 POINT(40.09395194852367 73.55216766555868) bank83649 +83650 POINT(40.42378948175795 74.37507972204153) bank83650 +83651 POINT(40.5049192402909 73.11453559436757) bank83651 +83652 POINT(41.56192835790782 73.24472869812217) bank83652 +83653 POINT(40.1912384834865 74.14128739570879) bank83653 +83654 POINT(41.32086778856418 74.59279723652405) bank83654 +83655 POINT(41.098275133118506 73.8487591241615) bank83655 +83656 POINT(41.13858754742302 73.43082937542886) bank83656 +83657 POINT(40.54799741322695 74.32871328568156) bank83657 +83658 POINT(41.64400872508519 74.18872327181535) bank83658 +83659 POINT(41.05617532063551 73.43869567951482) bank83659 +83660 POINT(39.79453061727337 73.42158971487453) bank83660 +83661 POINT(40.68420486219386 74.07131877365404) bank83661 +83662 POINT(40.26398051000146 73.75750328983469) bank83662 +83663 POINT(40.9856572600814 74.21808034603094) bank83663 +83664 POINT(41.57204088664773 74.32952502895219) bank83664 +83665 POINT(41.236237247046574 73.88608989196345) bank83665 +83666 POINT(40.10486850620014 73.34238813889861) bank83666 +83667 POINT(41.087793377730485 73.10181461412002) bank83667 +83668 POINT(41.3529575896046 74.70143728818044) bank83668 +83669 POINT(40.332563468189775 74.71979533413375) bank83669 +83670 POINT(41.161438662011655 74.55152040743992) bank83670 +83671 POINT(40.283786319878246 74.4310402689344) bank83671 +83672 POINT(41.46185719898627 74.5376410942601) bank83672 +83673 POINT(40.47537727850858 73.34314874289691) bank83673 +83674 POINT(41.46142170868367 73.64039605744571) bank83674 +83675 POINT(40.30114529684996 74.16235980154363) bank83675 +83676 POINT(39.88285660263212 74.56002502178534) bank83676 +83677 POINT(40.58419677359264 73.05970870271527) bank83677 +83678 POINT(39.98542060793464 74.50052745597569) bank83678 +83679 POINT(40.81925160595069 74.40606070809332) bank83679 +83680 POINT(40.15913786917732 74.8786293636489) bank83680 +83681 POINT(41.52531913221643 74.24156054756313) bank83681 +83682 POINT(39.92791280276781 73.17394565607597) bank83682 +83683 POINT(40.04690591934516 74.1458293228635) bank83683 +83684 POINT(40.069297027519035 73.2917082550915) bank83684 +83685 POINT(40.107426202464424 74.16971185591017) bank83685 +83686 POINT(41.02269939659329 73.74072243313591) bank83686 +83687 POINT(40.58362542432223 74.25567114825515) bank83687 +83688 POINT(40.99395507627216 73.09193777105953) bank83688 +83689 POINT(41.43818800131931 74.93224057271443) bank83689 +83690 POINT(40.86366805838562 73.75391251169411) bank83690 +83691 POINT(39.78825757681143 73.49049628081646) bank83691 +83692 POINT(39.784335501231546 74.04017488044326) bank83692 +83693 POINT(41.4260677348997 74.03406310995169) bank83693 +83694 POINT(40.68433162353751 73.43155429178786) bank83694 +83695 POINT(40.57219640709927 74.78788846812779) bank83695 +83696 POINT(40.52187619996326 73.22977907579318) bank83696 +83697 POINT(41.44914549010966 73.66210976505018) bank83697 +83698 POINT(41.50464575925156 74.85260076306238) bank83698 +83699 POINT(41.351019323128384 74.35180081088684) bank83699 +83700 POINT(41.52136177813879 73.63446732254371) bank83700 +83701 POINT(40.63031833571561 74.52378262474758) bank83701 +83702 POINT(39.933772692634804 73.72410785511929) bank83702 +83703 POINT(40.97087403923652 73.97647363025355) bank83703 +83704 POINT(39.91729549985469 74.11966502724246) bank83704 +83705 POINT(40.20660511262517 74.46251243053077) bank83705 +83706 POINT(40.132739321963676 74.71376970030867) bank83706 +83707 POINT(39.92131867655512 74.87004304644317) bank83707 +83708 POINT(39.89928423813307 74.24108024622706) bank83708 +83709 POINT(41.393333067056254 73.25429858669747) bank83709 +83710 POINT(41.69934130336311 74.5503102688362) bank83710 +83711 POINT(40.888979074608145 73.75133321741113) bank83711 +83712 POINT(40.2566098555278 73.97187298835638) bank83712 +83713 POINT(40.06594157107236 73.204179831147) bank83713 +83714 POINT(41.22504229310663 73.2858281306404) bank83714 +83715 POINT(39.802645037058774 74.09583046027703) bank83715 +83716 POINT(40.67779466214713 73.55646902140204) bank83716 +83717 POINT(41.3155944318041 74.6103546120716) bank83717 +83718 POINT(40.91686597497041 73.24870939972942) bank83718 +83719 POINT(40.765156932647 74.19645385145309) bank83719 +83720 POINT(41.254559554691056 73.38731117712365) bank83720 +83721 POINT(40.441491579641685 74.2514781208505) bank83721 +83722 POINT(40.75189614421442 73.99080977296046) bank83722 +83723 POINT(39.81442013429064 73.63777750739608) bank83723 +83724 POINT(40.09621638928572 73.820788648234) bank83724 +83725 POINT(39.86567766525795 74.52166509075256) bank83725 +83726 POINT(40.69787181013443 74.2871300589175) bank83726 +83727 POINT(40.711648341303366 74.51073375643789) bank83727 +83728 POINT(41.61672334360406 73.16018394714803) bank83728 +83729 POINT(39.983550972218914 73.16200506257213) bank83729 +83730 POINT(40.529662829243634 73.5460832010902) bank83730 +83731 POINT(40.75068297755417 74.20771906570972) bank83731 +83732 POINT(41.68647861565359 73.77785560943346) bank83732 +83733 POINT(40.333569249384745 73.2236305007344) bank83733 +83734 POINT(41.51216225587505 74.59071163185797) bank83734 +83735 POINT(40.339852303769874 74.85618805943419) bank83735 +83736 POINT(41.060637504774434 74.53493510184468) bank83736 +83737 POINT(39.931470199316664 73.94437327891387) bank83737 +83738 POINT(40.601380404363255 73.17426991075567) bank83738 +83739 POINT(40.88294379352745 73.39034725668063) bank83739 +83740 POINT(40.23152764637893 73.8234193818383) bank83740 +83741 POINT(40.36792674182771 73.65482599436697) bank83741 +83742 POINT(41.014671595935134 73.5147036331291) bank83742 +83743 POINT(40.38843196284469 75.0047900387012) bank83743 +83744 POINT(40.86522470211905 74.729816198344) bank83744 +83745 POINT(40.310614853725006 73.75605747971481) bank83745 +83746 POINT(40.47947329814409 74.97079938576107) bank83746 +83747 POINT(39.98346623795299 74.89514105649073) bank83747 +83748 POINT(40.83605017864089 74.49090852022891) bank83748 +83749 POINT(41.35536001663597 73.64320562401991) bank83749 +83750 POINT(40.88924652896996 74.9135559156299) bank83750 +83751 POINT(39.81586277867165 74.51555201508567) bank83751 +83752 POINT(40.55257899473706 73.79722426958108) bank83752 +83753 POINT(40.44074190732111 73.90077241100805) bank83753 +83754 POINT(41.18993880750409 73.01433008261888) bank83754 +83755 POINT(39.93117610394906 73.10527046482434) bank83755 +83756 POINT(40.605918428180004 73.47884533718204) bank83756 +83757 POINT(40.102896295924346 73.2490849630831) bank83757 +83758 POINT(40.96546927485318 73.69924323293304) bank83758 +83759 POINT(40.183224805920524 73.9083696287444) bank83759 +83760 POINT(40.5817865836122 73.5012454614151) bank83760 +83761 POINT(40.37771647593543 74.00625218525428) bank83761 +83762 POINT(41.02478083314995 74.77638714379928) bank83762 +83763 POINT(41.21527820292583 74.83075777242222) bank83763 +83764 POINT(39.99108199502509 73.01594713540344) bank83764 +83765 POINT(40.645559794004534 74.27034739709505) bank83765 +83766 POINT(40.020413382205405 74.44721560112876) bank83766 +83767 POINT(41.2350286430793 73.01536658776803) bank83767 +83768 POINT(41.43746923395321 74.24636066002635) bank83768 +83769 POINT(40.956214229034465 73.36833877380847) bank83769 +83770 POINT(41.037038126047605 74.26504519913745) bank83770 +83771 POINT(40.60550332294912 74.2744823597922) bank83771 +83772 POINT(40.21894580482636 74.83680338807719) bank83772 +83773 POINT(40.00679932701515 73.89317911349582) bank83773 +83774 POINT(41.04460131851056 74.29000368564022) bank83774 +83775 POINT(41.085530621800935 73.34841921031749) bank83775 +83776 POINT(40.22425258758719 73.90351311201722) bank83776 +83777 POINT(40.30257103472012 73.10649093631534) bank83777 +83778 POINT(41.629589777315815 74.85311923102722) bank83778 +83779 POINT(40.325141212860466 73.52423216224625) bank83779 +83780 POINT(41.56898930943384 73.41366284357284) bank83780 +83781 POINT(40.546174602363976 74.5034042473234) bank83781 +83782 POINT(41.288288607797675 73.80452617093721) bank83782 +83783 POINT(40.84865671760194 73.21727643922146) bank83783 +83784 POINT(40.6251393414994 73.35282380501053) bank83784 +83785 POINT(39.83149227532252 74.28305384798932) bank83785 +83786 POINT(40.22728980828023 73.1124882462839) bank83786 +83787 POINT(40.438892488996 74.24026949662755) bank83787 +83788 POINT(39.88890209700078 73.41334739332513) bank83788 +83789 POINT(39.87092284612139 74.51821181388728) bank83789 +83790 POINT(41.35589595332591 73.7418631501366) bank83790 +83791 POINT(39.92855329914429 74.35327497901591) bank83791 +83792 POINT(40.3302628315152 74.64566244626717) bank83792 +83793 POINT(41.51417886718152 73.64164157579843) bank83793 +83794 POINT(40.49201842597337 74.35233217216967) bank83794 +83795 POINT(41.40086735474464 73.89803489917804) bank83795 +83796 POINT(40.932976106136245 73.10464426277444) bank83796 +83797 POINT(40.955765300068926 73.53566125231602) bank83797 +83798 POINT(41.52427576993009 73.61949493527065) bank83798 +83799 POINT(40.859337667036115 74.05831972706883) bank83799 +83800 POINT(39.73754178890188 73.9682254948481) bank83800 +83801 POINT(40.08695675889574 73.51675420464477) bank83801 +83802 POINT(41.64369210553917 73.68654568494024) bank83802 +83803 POINT(41.49868475663518 74.99460626570374) bank83803 +83804 POINT(41.676112812326714 73.26183465458598) bank83804 +83805 POINT(40.07842218144597 73.35981914547057) bank83805 +83806 POINT(41.65651127979799 73.90063607089847) bank83806 +83807 POINT(40.95314509389558 73.82036407595277) bank83807 +83808 POINT(40.25995799362174 74.72737707687874) bank83808 +83809 POINT(40.52925125735969 73.19268745921012) bank83809 +83810 POINT(40.764696561450236 74.20175045611578) bank83810 +83811 POINT(40.327434781892094 73.03245438636752) bank83811 +83812 POINT(40.07340535868511 73.60410103824272) bank83812 +83813 POINT(41.270105327580005 74.684181916973) bank83813 +83814 POINT(41.15588972619119 74.661025920314) bank83814 +83815 POINT(41.46263405181006 73.22409073863413) bank83815 +83816 POINT(40.433723899183825 73.89279431111615) bank83816 +83817 POINT(41.43270532442224 73.68376283106474) bank83817 +83818 POINT(40.458986151196264 73.5450583530936) bank83818 +83819 POINT(40.818270932864486 73.65426645140072) bank83819 +83820 POINT(41.060315313427274 73.65316845126937) bank83820 +83821 POINT(40.78833196617669 74.06308311584188) bank83821 +83822 POINT(41.2711642047621 74.92748318567645) bank83822 +83823 POINT(40.52412529852946 74.99132483844755) bank83823 +83824 POINT(40.525700335082774 74.57109544973741) bank83824 +83825 POINT(41.60280115048483 73.40974385889076) bank83825 +83826 POINT(41.005897952236474 74.54771959390979) bank83826 +83827 POINT(40.072779311411786 73.35427295157186) bank83827 +83828 POINT(40.49463676033256 73.74310982211837) bank83828 +83829 POINT(39.78188358995658 74.70879468204579) bank83829 +83830 POINT(40.641685422153756 73.99274460648049) bank83830 +83831 POINT(41.07214173457718 74.29087746026217) bank83831 +83832 POINT(39.939132878721914 74.95337526518394) bank83832 +83833 POINT(41.60738466097668 73.99423457551482) bank83833 +83834 POINT(39.92529508641087 73.1540548974457) bank83834 +83835 POINT(40.04042824189739 73.85339475016666) bank83835 +83836 POINT(41.50962063087344 74.5330885986185) bank83836 +83837 POINT(41.31307979736264 74.33175586725318) bank83837 +83838 POINT(40.6087039156814 73.86585653928286) bank83838 +83839 POINT(39.812723514744555 74.98190606436552) bank83839 +83840 POINT(41.1380238273881 73.55912690993236) bank83840 +83841 POINT(40.593335266740795 74.65531822447954) bank83841 +83842 POINT(41.675591158615454 74.39176790777749) bank83842 +83843 POINT(40.8442118542491 74.80221134164167) bank83843 +83844 POINT(41.49944525657076 74.51258816033554) bank83844 +83845 POINT(41.22969730327265 73.65799519600897) bank83845 +83846 POINT(40.794253059506126 73.18138400157075) bank83846 +83847 POINT(40.76688382444335 74.66270001461803) bank83847 +83848 POINT(40.177536721747025 73.24555098366957) bank83848 +83849 POINT(40.77759449885237 74.15605657618536) bank83849 +83850 POINT(40.260038522421475 73.72335336409212) bank83850 +83851 POINT(39.844548327363064 74.20569243735488) bank83851 +83852 POINT(40.2267729928753 74.00988217110437) bank83852 +83853 POINT(40.547261644749724 73.4695428546678) bank83853 +83854 POINT(41.4943444889299 73.6554331652324) bank83854 +83855 POINT(41.38697677474064 74.37836939596322) bank83855 +83856 POINT(41.59872179113236 74.12712507553823) bank83856 +83857 POINT(40.7574047664648 74.62170373668722) bank83857 +83858 POINT(40.137210031372064 73.65527589664723) bank83858 +83859 POINT(40.57511198858756 73.86494209081916) bank83859 +83860 POINT(40.87344927471103 73.98099810767755) bank83860 +83861 POINT(40.73057025423621 74.021102185181) bank83861 +83862 POINT(40.609098210632766 73.86962476319678) bank83862 +83863 POINT(39.791195678532695 74.99286614873965) bank83863 +83864 POINT(40.360696917509784 74.61588476070828) bank83864 +83865 POINT(41.062515150686 73.77685066820641) bank83865 +83866 POINT(41.191033584684725 74.33810107545138) bank83866 +83867 POINT(40.498045431910725 74.50322640660032) bank83867 +83868 POINT(40.59009443860569 73.36634918038766) bank83868 +83869 POINT(40.55011649595607 73.94380373903793) bank83869 +83870 POINT(40.66904639613428 73.49570634523062) bank83870 +83871 POINT(40.356939215805184 74.44884237341311) bank83871 +83872 POINT(41.355021641941846 73.23331237914314) bank83872 +83873 POINT(40.720291871428564 74.30385488257586) bank83873 +83874 POINT(41.14660737509735 74.2851233795399) bank83874 +83875 POINT(40.985827217617334 74.0089314248414) bank83875 +83876 POINT(41.586638339750024 74.44286677609779) bank83876 +83877 POINT(39.894504455741874 74.34168488331747) bank83877 +83878 POINT(40.17200257846145 74.21761383233596) bank83878 +83879 POINT(41.15635374667307 73.3825562082272) bank83879 +83880 POINT(40.49075293452543 73.52472728362008) bank83880 +83881 POINT(39.79119845597652 73.00672528035804) bank83881 +83882 POINT(40.453717773572784 74.144639774224) bank83882 +83883 POINT(40.54774079648011 74.79452133110472) bank83883 +83884 POINT(41.443948972992764 73.77703151798103) bank83884 +83885 POINT(41.53360088678045 74.99884985872212) bank83885 +83886 POINT(40.387771979124594 73.59263287818989) bank83886 +83887 POINT(39.74670144931303 73.65633393762582) bank83887 +83888 POINT(41.573224192222305 74.75081511126626) bank83888 +83889 POINT(39.89178106506235 73.47213469389573) bank83889 +83890 POINT(40.47667725645134 73.23035428826411) bank83890 +83891 POINT(40.74710213490364 73.23863598644056) bank83891 +83892 POINT(41.183947257108244 73.65399894631177) bank83892 +83893 POINT(40.40357169452778 74.94345090944655) bank83893 +83894 POINT(40.924664719667156 74.46318527484279) bank83894 +83895 POINT(41.54087301163849 73.20503257531894) bank83895 +83896 POINT(39.97942826705568 73.03660152455593) bank83896 +83897 POINT(41.294644671060375 73.52561406143847) bank83897 +83898 POINT(40.472715312684635 73.07136606309463) bank83898 +83899 POINT(39.76110073902135 74.29163036233258) bank83899 +83900 POINT(40.43115624554658 74.39632867748739) bank83900 +83901 POINT(40.565042705715925 74.18713647326442) bank83901 +83902 POINT(40.143456286813304 74.04904654358802) bank83902 +83903 POINT(40.35861981171521 74.31907698739865) bank83903 +83904 POINT(40.65107624637907 74.54414763684974) bank83904 +83905 POINT(41.15803390937519 74.03743625034896) bank83905 +83906 POINT(40.50113014810308 74.79958068893099) bank83906 +83907 POINT(39.89094389582935 73.58777866676687) bank83907 +83908 POINT(41.60980334880406 73.93139268559526) bank83908 +83909 POINT(40.06607086121686 73.0675716279646) bank83909 +83910 POINT(40.943304671711 73.69901606539212) bank83910 +83911 POINT(41.31401653552052 74.91663518553763) bank83911 +83912 POINT(41.24334706593465 73.23655356901031) bank83912 +83913 POINT(40.69470619173285 73.49762337395896) bank83913 +83914 POINT(40.77080848970653 74.62348621197383) bank83914 +83915 POINT(40.11990691378773 74.83157507196033) bank83915 +83916 POINT(39.98628474549346 73.10614297109325) bank83916 +83917 POINT(39.94653185395717 74.45053165973125) bank83917 +83918 POINT(41.10367483420206 73.72277747477504) bank83918 +83919 POINT(39.916924989003455 73.74436072754904) bank83919 +83920 POINT(40.54777185900238 73.1360335083682) bank83920 +83921 POINT(40.95833645164122 74.92288862895352) bank83921 +83922 POINT(39.843415586081846 74.01605258639351) bank83922 +83923 POINT(40.49994209162536 74.31733297558985) bank83923 +83924 POINT(39.80609883752599 74.98227313160464) bank83924 +83925 POINT(40.42244575337659 73.29176277427493) bank83925 +83926 POINT(41.21583098329396 73.2457387250518) bank83926 +83927 POINT(40.35713071398781 74.264029158064) bank83927 +83928 POINT(40.42692380532942 74.24907958966682) bank83928 +83929 POINT(41.253600719152615 73.93213555310504) bank83929 +83930 POINT(41.58824845478169 74.07337943764892) bank83930 +83931 POINT(40.317211109715885 74.1443840941868) bank83931 +83932 POINT(41.141898680556636 73.94604363975027) bank83932 +83933 POINT(40.16606079465366 73.55890336247161) bank83933 +83934 POINT(41.456352019547595 73.36473488306581) bank83934 +83935 POINT(40.147061834775435 74.64576461208267) bank83935 +83936 POINT(40.52376115884866 73.9995493378055) bank83936 +83937 POINT(41.09621302313916 73.89757291354401) bank83937 +83938 POINT(39.90600803281951 73.50488231999161) bank83938 +83939 POINT(39.96707820392388 73.9953367702638) bank83939 +83940 POINT(40.37842580947597 74.96813673898055) bank83940 +83941 POINT(39.77804666314553 74.50273461815453) bank83941 +83942 POINT(40.43583711531382 73.79300778794834) bank83942 +83943 POINT(41.5445209770837 73.24794149416132) bank83943 +83944 POINT(41.656002182384796 74.07471113577643) bank83944 +83945 POINT(41.15800176520842 74.85163057736685) bank83945 +83946 POINT(40.336433429786915 73.66245487032499) bank83946 +83947 POINT(41.14994778471702 73.4753766320348) bank83947 +83948 POINT(40.71956627911873 74.89055210878594) bank83948 +83949 POINT(41.137734680467 73.18149169333759) bank83949 +83950 POINT(40.14668736630923 74.76159047334248) bank83950 +83951 POINT(41.52593257295613 73.48632250782714) bank83951 +83952 POINT(40.106381831767344 74.35611690356734) bank83952 +83953 POINT(40.764500147196934 73.68354913113625) bank83953 +83954 POINT(39.814514741746116 74.98732530917988) bank83954 +83955 POINT(40.89760923933542 74.94980093452656) bank83955 +83956 POINT(41.194284610640466 73.57115062634902) bank83956 +83957 POINT(40.81019178840628 73.90593902110032) bank83957 +83958 POINT(41.20904541948905 73.27644158627805) bank83958 +83959 POINT(41.40655546246724 73.12570620175849) bank83959 +83960 POINT(40.62185757097023 74.95247481088664) bank83960 +83961 POINT(40.18714794451618 74.58661190494304) bank83961 +83962 POINT(40.453915002046415 73.6642897331651) bank83962 +83963 POINT(40.60215673160312 73.35605632883032) bank83963 +83964 POINT(41.087943603780865 73.71301228951909) bank83964 +83965 POINT(39.75363168109988 74.30429299696362) bank83965 +83966 POINT(40.41904666874981 74.92151523482386) bank83966 +83967 POINT(40.42584235849649 73.44136678891812) bank83967 +83968 POINT(40.3303400378202 73.37235395043028) bank83968 +83969 POINT(41.52883714599075 73.36546522466823) bank83969 +83970 POINT(40.90626911478368 73.76685738930769) bank83970 +83971 POINT(40.77271623723604 74.11178548696336) bank83971 +83972 POINT(40.98448468415673 74.90196470843239) bank83972 +83973 POINT(39.94206551195594 74.27817055761372) bank83973 +83974 POINT(40.583426175551374 74.98194325844142) bank83974 +83975 POINT(39.90285530749181 73.76014620500906) bank83975 +83976 POINT(40.917800220235804 73.96525097818848) bank83976 +83977 POINT(40.034109848476426 73.70232884176981) bank83977 +83978 POINT(41.486774849826986 73.66329373786354) bank83978 +83979 POINT(40.90183938314478 74.07202476308846) bank83979 +83980 POINT(41.09140486868672 73.4009558583447) bank83980 +83981 POINT(40.62938100145393 73.21447326352315) bank83981 +83982 POINT(40.78200483508392 74.74531124644636) bank83982 +83983 POINT(41.37788925644759 74.31537883716018) bank83983 +83984 POINT(40.69452277570918 73.75935378347772) bank83984 +83985 POINT(40.65290994651351 73.11066902897616) bank83985 +83986 POINT(41.517672653157135 73.23381209542751) bank83986 +83987 POINT(41.64565291428256 73.5384759063416) bank83987 +83988 POINT(41.70421216718207 73.55527890013717) bank83988 +83989 POINT(41.4145419854849 73.77912516176664) bank83989 +83990 POINT(41.32341705869336 73.8097417303889) bank83990 +83991 POINT(41.56284225755214 74.86892958636591) bank83991 +83992 POINT(40.63954463173254 74.65064982826792) bank83992 +83993 POINT(40.627819512695666 73.25983202140479) bank83993 +83994 POINT(40.27090499993797 74.02405580564543) bank83994 +83995 POINT(40.9801810207867 74.55957291382859) bank83995 +83996 POINT(39.81319133666973 74.614488597918) bank83996 +83997 POINT(39.888689371437955 73.21700111510349) bank83997 +83998 POINT(40.05792779641682 74.66896742703996) bank83998 +83999 POINT(40.343821077378124 74.01528489063445) bank83999 +84000 POINT(40.91677691162006 73.41476802915913) bank84000 +84001 POINT(41.07003131993544 73.64249674590864) bank84001 +84002 POINT(40.17759844421106 74.92608274904507) bank84002 +84003 POINT(41.103570744377116 73.60752097000795) bank84003 +84004 POINT(41.38083735608585 74.05401180736482) bank84004 +84005 POINT(40.78835402570529 74.5385659785257) bank84005 +84006 POINT(40.01086002986049 73.10918359773156) bank84006 +84007 POINT(41.04479531435058 74.87630986690166) bank84007 +84008 POINT(40.75752082214926 73.81470378760606) bank84008 +84009 POINT(41.02369270857608 74.48296749270145) bank84009 +84010 POINT(40.47209292910598 73.64113206737076) bank84010 +84011 POINT(41.00661760475787 73.77259642966135) bank84011 +84012 POINT(40.653901147705994 73.30556454262096) bank84012 +84013 POINT(40.79479193790429 74.54320310865832) bank84013 +84014 POINT(40.142169020971835 73.19309331374116) bank84014 +84015 POINT(40.359163516155725 73.96306180936506) bank84015 +84016 POINT(40.62689529881746 74.76835998949497) bank84016 +84017 POINT(40.83134824107595 73.11285895373977) bank84017 +84018 POINT(40.91776734847688 74.76112186885855) bank84018 +84019 POINT(40.49657748080924 73.28045651396417) bank84019 +84020 POINT(40.660451684458906 73.23215180644742) bank84020 +84021 POINT(40.08554597158893 73.93500941382979) bank84021 +84022 POINT(41.6856042329202 73.84255344183298) bank84022 +84023 POINT(41.19988239624827 73.27727038128586) bank84023 +84024 POINT(41.28235733984902 73.54366600512982) bank84024 +84025 POINT(39.99543761015252 74.55900588164437) bank84025 +84026 POINT(39.73051858533452 74.58518335998131) bank84026 +84027 POINT(40.68555288501919 74.4293840154857) bank84027 +84028 POINT(41.542927800299 73.85673260697705) bank84028 +84029 POINT(40.188425841222596 73.89506593912644) bank84029 +84030 POINT(39.799498410640474 74.49640948269754) bank84030 +84031 POINT(40.29525748576828 74.63087621570432) bank84031 +84032 POINT(40.52226301363593 74.54577566323745) bank84032 +84033 POINT(40.47208979664205 74.50507305748641) bank84033 +84034 POINT(40.15168525470423 74.4061237772506) bank84034 +84035 POINT(41.68669504524489 74.32805325475647) bank84035 +84036 POINT(41.5774429948699 74.61238258258564) bank84036 +84037 POINT(41.254763516122274 73.09229203589) bank84037 +84038 POINT(40.509483767936864 73.15235969018931) bank84038 +84039 POINT(40.968480297341664 73.75117188303456) bank84039 +84040 POINT(41.56268184013822 73.39079204949253) bank84040 +84041 POINT(40.640573478017195 74.0454558946903) bank84041 +84042 POINT(41.711647075471966 73.96729982249921) bank84042 +84043 POINT(41.503312615048145 74.25886167945106) bank84043 +84044 POINT(41.46840618854973 73.91243535547972) bank84044 +84045 POINT(40.81737652010695 74.51476986416601) bank84045 +84046 POINT(41.20280120889415 73.26950993426142) bank84046 +84047 POINT(41.018067767274 73.25070982318994) bank84047 +84048 POINT(40.534295824877766 74.3420703306005) bank84048 +84049 POINT(41.25158603388431 74.21625522724845) bank84049 +84050 POINT(41.29809241604948 73.99252102745731) bank84050 +84051 POINT(40.76875170701162 74.34557761860256) bank84051 +84052 POINT(40.76419375633058 73.32502215398773) bank84052 +84053 POINT(40.20227678685486 73.2477910533089) bank84053 +84054 POINT(41.18260487779937 73.68589494183698) bank84054 +84055 POINT(41.02395383577459 74.03988739988233) bank84055 +84056 POINT(40.36012179913134 74.47530824318636) bank84056 +84057 POINT(41.10695705522987 73.92003804541022) bank84057 +84058 POINT(39.90098930338089 74.99850213333967) bank84058 +84059 POINT(41.601085894584216 73.10242264806848) bank84059 +84060 POINT(40.85300000542074 74.19055494783717) bank84060 +84061 POINT(41.31985830299206 73.68836183182869) bank84061 +84062 POINT(41.71244714464565 73.19162718442864) bank84062 +84063 POINT(40.165926940980064 74.65594342718177) bank84063 +84064 POINT(41.57364922522043 73.96296520078971) bank84064 +84065 POINT(39.76317827418029 73.2113633690859) bank84065 +84066 POINT(41.43994410007169 73.79201909362217) bank84066 +84067 POINT(40.898316126475294 74.81676674238557) bank84067 +84068 POINT(41.53650252310657 74.4719479473125) bank84068 +84069 POINT(40.5094266148162 74.69898762004695) bank84069 +84070 POINT(40.84208460932179 73.62807932724645) bank84070 +84071 POINT(39.80844332963965 73.83189643212596) bank84071 +84072 POINT(40.68925738624696 74.18619986195651) bank84072 +84073 POINT(39.91850542260558 73.01016825306588) bank84073 +84074 POINT(41.16371612631939 73.67238600417986) bank84074 +84075 POINT(41.124438689260835 73.9149171672186) bank84075 +84076 POINT(41.42160470137057 73.85721474787418) bank84076 +84077 POINT(39.85645880896682 73.97324653615438) bank84077 +84078 POINT(41.188444858642 74.86444450504484) bank84078 +84079 POINT(40.23664888757177 73.34119466863515) bank84079 +84080 POINT(41.18250705005679 73.3608320633014) bank84080 +84081 POINT(40.1241546586602 74.6809381386022) bank84081 +84082 POINT(41.338229111259906 74.8794179096429) bank84082 +84083 POINT(41.450097036620164 73.98257756006329) bank84083 +84084 POINT(41.63343961672903 74.8413345069847) bank84084 +84085 POINT(40.24026922666451 74.27114632260653) bank84085 +84086 POINT(40.2664688106844 73.45465077020057) bank84086 +84087 POINT(41.61264726216759 73.46008279594976) bank84087 +84088 POINT(40.66389232140794 74.2805401736017) bank84088 +84089 POINT(40.80915224480428 73.70664109234335) bank84089 +84090 POINT(40.067292133546424 74.33714718707698) bank84090 +84091 POINT(40.54320212905818 73.92632926991062) bank84091 +84092 POINT(41.09059245266927 74.94113379188877) bank84092 +84093 POINT(41.29992774007947 74.97783837726138) bank84093 +84094 POINT(41.45209939238983 73.01148195813659) bank84094 +84095 POINT(41.49590307228684 74.06704356921277) bank84095 +84096 POINT(41.12414423169641 74.79343407989218) bank84096 +84097 POINT(40.2405404261714 74.11257603665578) bank84097 +84098 POINT(41.41161018034844 73.21963677024658) bank84098 +84099 POINT(41.401515636628 74.34934963689739) bank84099 +84100 POINT(41.11040340200887 74.32031487453168) bank84100 +84101 POINT(40.512029272070286 74.99936708367197) bank84101 +84102 POINT(41.19202086595639 74.61316540452167) bank84102 +84103 POINT(41.70089296701634 73.56413890348706) bank84103 +84104 POINT(41.06234702488877 74.0187052927354) bank84104 +84105 POINT(41.42737781093684 74.43771546968775) bank84105 +84106 POINT(41.16725518352217 73.9625164751437) bank84106 +84107 POINT(40.46789873716256 74.86979978359912) bank84107 +84108 POINT(40.70280720859903 74.83241637417774) bank84108 +84109 POINT(39.92986275529938 73.11245306223992) bank84109 +84110 POINT(40.453113216004105 74.45616592526582) bank84110 +84111 POINT(40.27867437062396 74.72213226822008) bank84111 +84112 POINT(40.97870094009667 74.79110233320574) bank84112 +84113 POINT(40.29418358030246 74.90730238183508) bank84113 +84114 POINT(40.319246518256875 74.08034828052165) bank84114 +84115 POINT(41.13899323577303 73.57479402063068) bank84115 +84116 POINT(41.01994012367218 73.77214612593264) bank84116 +84117 POINT(40.55004465360173 73.96921519787408) bank84117 +84118 POINT(41.563852738114356 73.10481748465125) bank84118 +84119 POINT(40.28253254436408 74.8534728772186) bank84119 +84120 POINT(40.189919650883006 74.06030005880224) bank84120 +84121 POINT(40.80800093589002 73.9654165528951) bank84121 +84122 POINT(40.684983354751886 74.24007211153388) bank84122 +84123 POINT(41.02513613844512 73.21580961835477) bank84123 +84124 POINT(41.1509435150298 74.84328751403503) bank84124 +84125 POINT(40.48868491671647 74.99643806455005) bank84125 +84126 POINT(40.28939391804072 74.39774042699095) bank84126 +84127 POINT(39.82600898781467 74.24670444557958) bank84127 +84128 POINT(41.43815806173202 73.5342784797897) bank84128 +84129 POINT(41.10994235329963 74.0432313299654) bank84129 +84130 POINT(40.71524870565655 73.10774961753087) bank84130 +84131 POINT(40.74966961260155 74.32267612049708) bank84131 +84132 POINT(40.27377794979856 74.10391466624449) bank84132 +84133 POINT(41.076929208989995 74.02143762156508) bank84133 +84134 POINT(40.469296368964805 73.27614280393777) bank84134 +84135 POINT(40.05568028871483 73.28080085559434) bank84135 +84136 POINT(39.813186381490596 73.81325665895227) bank84136 +84137 POINT(41.586638218729725 73.1969688916773) bank84137 +84138 POINT(40.704315389390594 74.41809298295027) bank84138 +84139 POINT(41.708901111024 74.10179313913062) bank84139 +84140 POINT(41.382692168800695 73.9964568139168) bank84140 +84141 POINT(41.62572400934023 74.26007586436997) bank84141 +84142 POINT(40.87810074440078 74.25273021626943) bank84142 +84143 POINT(40.99300120386047 74.97328797075049) bank84143 +84144 POINT(41.24622545422156 74.47375722152916) bank84144 +84145 POINT(41.1523792222569 74.64909972478351) bank84145 +84146 POINT(40.89616660797296 73.41628966733624) bank84146 +84147 POINT(40.0021626045098 74.569333214253) bank84147 +84148 POINT(41.39534672316637 74.07263896435937) bank84148 +84149 POINT(41.3707536258435 74.15867883759555) bank84149 +84150 POINT(41.353389559079574 74.7179933296091) bank84150 +84151 POINT(40.560130979052154 73.85061261125978) bank84151 +84152 POINT(40.4653494582393 73.74914145619218) bank84152 +84153 POINT(39.83614171349696 73.62175047270948) bank84153 +84154 POINT(41.403515210418504 74.78936285820313) bank84154 +84155 POINT(39.764086689739536 74.97903826167811) bank84155 +84156 POINT(40.04400454244499 73.54914568837964) bank84156 +84157 POINT(40.4052541125351 74.78894118559826) bank84157 +84158 POINT(40.323898413630126 74.92859495660062) bank84158 +84159 POINT(41.41012305946542 74.6011319741164) bank84159 +84160 POINT(41.35001897922123 74.87649477697867) bank84160 +84161 POINT(40.03577136766399 73.51275037207876) bank84161 +84162 POINT(40.588659327654234 74.63150852594178) bank84162 +84163 POINT(41.426837242471535 74.58451204332553) bank84163 +84164 POINT(40.10197143669735 74.71895939794182) bank84164 +84165 POINT(41.36588618682579 73.22358492346366) bank84165 +84166 POINT(40.66847634688289 73.67977121490786) bank84166 +84167 POINT(39.72248619910323 74.4211076048275) bank84167 +84168 POINT(39.88317368823635 74.82857433072016) bank84168 +84169 POINT(41.18486785078831 73.95679756860454) bank84169 +84170 POINT(41.343713661002475 73.43771232631993) bank84170 +84171 POINT(39.80025043986676 73.70284061421866) bank84171 +84172 POINT(40.04189473611494 74.15515610131955) bank84172 +84173 POINT(39.95773110991061 74.56668051994592) bank84173 +84174 POINT(40.16457473152438 73.04445263283881) bank84174 +84175 POINT(41.21521651216366 74.13882601849748) bank84175 +84176 POINT(41.08644824574793 74.11127650087066) bank84176 +84177 POINT(41.29090250954587 73.05762965678711) bank84177 +84178 POINT(41.588302523154205 74.63920333642243) bank84178 +84179 POINT(40.86651652487813 74.55765333071083) bank84179 +84180 POINT(41.00195592397876 73.05408062076789) bank84180 +84181 POINT(39.840529639885105 74.8614025194443) bank84181 +84182 POINT(39.713233796042864 74.02896952476439) bank84182 +84183 POINT(40.358154712119635 73.3611413481033) bank84183 +84184 POINT(41.37757575559756 74.6024470436608) bank84184 +84185 POINT(39.95793984021798 73.80737116960879) bank84185 +84186 POINT(41.36126154864435 74.17668095702538) bank84186 +84187 POINT(40.67828734064409 74.75762480503376) bank84187 +84188 POINT(39.7418520630381 73.63253875979643) bank84188 +84189 POINT(41.37927841776737 74.4956805901386) bank84189 +84190 POINT(40.57561825496576 74.35020333354898) bank84190 +84191 POINT(41.22366318269592 73.59850485541746) bank84191 +84192 POINT(40.631472725178455 74.24791737967867) bank84192 +84193 POINT(40.270030658729816 73.96107809247582) bank84193 +84194 POINT(40.987776352150206 74.5767219639317) bank84194 +84195 POINT(41.017033435641004 74.21755814677685) bank84195 +84196 POINT(40.68243060031094 74.63149289912505) bank84196 +84197 POINT(41.31520789093737 74.01739836315633) bank84197 +84198 POINT(40.60816051043618 74.42695218664326) bank84198 +84199 POINT(40.92408224256305 74.92594166267948) bank84199 +84200 POINT(39.77522084835692 74.54539392736766) bank84200 +84201 POINT(41.44337671877816 74.4702925417512) bank84201 +84202 POINT(41.63568275361945 73.0451521266242) bank84202 +84203 POINT(41.10995211070567 73.06876824697653) bank84203 +84204 POINT(39.790409257010694 73.70426835486643) bank84204 +84205 POINT(40.32539747915043 73.77245470815113) bank84205 +84206 POINT(41.47396131437205 73.61003252591641) bank84206 +84207 POINT(40.474090495737215 74.88981139666835) bank84207 +84208 POINT(41.42379680136604 74.35671652198138) bank84208 +84209 POINT(41.52567330353248 73.86467307891324) bank84209 +84210 POINT(39.9862878816177 73.15861138640713) bank84210 +84211 POINT(40.22652626762886 74.89574249992864) bank84211 +84212 POINT(39.88819974662738 74.32686997953877) bank84212 +84213 POINT(40.21698093281491 73.94955578500863) bank84213 +84214 POINT(40.33448577792443 74.97171481745904) bank84214 +84215 POINT(40.18826123700892 74.56551836925102) bank84215 +84216 POINT(41.56778896552669 73.95752036217205) bank84216 +84217 POINT(39.77662093719701 74.38857499298729) bank84217 +84218 POINT(41.12673643066354 74.59167319507297) bank84218 +84219 POINT(40.936311426521826 74.5020201954477) bank84219 +84220 POINT(40.88096969434305 73.46149669184058) bank84220 +84221 POINT(41.47584796106546 73.07415568341095) bank84221 +84222 POINT(41.19035845076349 74.1416655303701) bank84222 +84223 POINT(39.91269739729136 74.24767781814448) bank84223 +84224 POINT(41.39688121360336 73.36423430496961) bank84224 +84225 POINT(40.90995230592747 73.03476643634927) bank84225 +84226 POINT(41.65910608705537 74.64392404192175) bank84226 +84227 POINT(41.19885900621107 74.28540438313462) bank84227 +84228 POINT(40.46360156042253 73.03936842209586) bank84228 +84229 POINT(41.512737790921754 73.96464625748236) bank84229 +84230 POINT(39.97332465275934 74.47673490948931) bank84230 +84231 POINT(39.911057868582525 73.25807198172934) bank84231 +84232 POINT(41.31761578792167 73.79585682773751) bank84232 +84233 POINT(41.13423526976178 73.70085640940003) bank84233 +84234 POINT(41.17989544632277 74.31560272555744) bank84234 +84235 POINT(40.25717604714032 73.45107652897887) bank84235 +84236 POINT(40.802963230440554 73.6874713267931) bank84236 +84237 POINT(39.91426178408566 74.20865542015227) bank84237 +84238 POINT(40.0783138912976 74.36049307481944) bank84238 +84239 POINT(40.933143209797436 74.40017063099846) bank84239 +84240 POINT(40.4698325112067 73.15359834879112) bank84240 +84241 POINT(40.44977430348407 73.18465992028433) bank84241 +84242 POINT(41.4569461892432 73.27616083973321) bank84242 +84243 POINT(40.49611746640412 74.41558556514995) bank84243 +84244 POINT(40.45660222581192 74.90954882101376) bank84244 +84245 POINT(40.82439303643867 74.69797073684559) bank84245 +84246 POINT(39.8364893631635 74.48191267301395) bank84246 +84247 POINT(41.05394032101447 73.98537709649274) bank84247 +84248 POINT(41.14855598288035 74.3290619907932) bank84248 +84249 POINT(41.387318074186076 73.03273247625638) bank84249 +84250 POINT(41.628282484425924 73.68333909756619) bank84250 +84251 POINT(39.759189873018876 74.25044198576097) bank84251 +84252 POINT(41.20460324572061 73.25763322178966) bank84252 +84253 POINT(41.223666795516394 73.66774049849502) bank84253 +84254 POINT(40.307273249801234 74.89889468784263) bank84254 +84255 POINT(41.34007222682837 74.94383312847688) bank84255 +84256 POINT(40.410706806717606 73.78063037862546) bank84256 +84257 POINT(41.5634983573635 73.91935844535725) bank84257 +84258 POINT(39.90280187767517 73.0354953569052) bank84258 +84259 POINT(41.11442454944611 74.74430202643904) bank84259 +84260 POINT(41.23500013850691 73.45461729684021) bank84260 +84261 POINT(41.211539462350345 74.07600136744513) bank84261 +84262 POINT(39.770603538707995 73.7080047570922) bank84262 +84263 POINT(40.86879709149213 74.86979122181867) bank84263 +84264 POINT(40.50502422045311 74.20611626341004) bank84264 +84265 POINT(40.90971692727149 74.43914669189353) bank84265 +84266 POINT(39.864688960545436 73.07162813588516) bank84266 +84267 POINT(40.315773352873904 74.87302108652722) bank84267 +84268 POINT(40.77262280158062 73.19357193200368) bank84268 +84269 POINT(39.85671864498036 74.05324804996219) bank84269 +84270 POINT(40.016217285481744 73.61547819352491) bank84270 +84271 POINT(41.20242608010258 73.78798351876075) bank84271 +84272 POINT(41.28479971136417 74.27269156620518) bank84272 +84273 POINT(41.456071896079955 74.26946685264292) bank84273 +84274 POINT(41.673041577907284 74.20161322806719) bank84274 +84275 POINT(40.98154184788932 73.7458540458369) bank84275 +84276 POINT(40.28337295759135 73.41991254856185) bank84276 +84277 POINT(41.632802317162415 73.45555812835495) bank84277 +84278 POINT(41.228888420037556 73.80373195203421) bank84278 +84279 POINT(40.89817568731697 73.67345654945038) bank84279 +84280 POINT(40.60071985649671 73.23636456009162) bank84280 +84281 POINT(41.20333238547679 74.60142386265646) bank84281 +84282 POINT(40.64846909239255 73.18927649821947) bank84282 +84283 POINT(41.64291733096117 73.2820346064285) bank84283 +84284 POINT(41.29235409675454 74.83796539076788) bank84284 +84285 POINT(41.157599434282076 73.16597816516254) bank84285 +84286 POINT(41.25523097937916 74.43519074867238) bank84286 +84287 POINT(40.786127914522254 73.59567170285175) bank84287 +84288 POINT(41.67679028472385 74.01453731758104) bank84288 +84289 POINT(40.256809947936084 74.5952878496642) bank84289 +84290 POINT(41.07895737342004 74.94541500547156) bank84290 +84291 POINT(40.317705106684265 73.89377312134401) bank84291 +84292 POINT(40.807729171932024 73.07354713043692) bank84292 +84293 POINT(40.93966278663804 74.15643495074873) bank84293 +84294 POINT(41.31814937702496 74.37071332461588) bank84294 +84295 POINT(39.771158426011255 74.22717680973524) bank84295 +84296 POINT(40.722009006562814 73.38619662564578) bank84296 +84297 POINT(40.77801731883588 74.969769172473) bank84297 +84298 POINT(41.091883225115936 73.10343779108298) bank84298 +84299 POINT(40.20212933703937 73.78404877918624) bank84299 +84300 POINT(41.54716203463443 74.23818882317852) bank84300 +84301 POINT(40.19018398580863 73.11452369962015) bank84301 +84302 POINT(39.88512275496768 73.56880586947103) bank84302 +84303 POINT(41.39444260243189 74.48972696043185) bank84303 +84304 POINT(40.436456970830896 73.89225681008486) bank84304 +84305 POINT(41.39038899130818 74.21709102798417) bank84305 +84306 POINT(40.212416226357284 73.97629695741438) bank84306 +84307 POINT(40.03293345407739 73.48068193139522) bank84307 +84308 POINT(39.76743219180775 74.16179717691124) bank84308 +84309 POINT(41.141894272461975 74.04568470566997) bank84309 +84310 POINT(40.3203219262938 74.25000290454697) bank84310 +84311 POINT(40.82052156635867 74.36869859570837) bank84311 +84312 POINT(40.99465845414634 74.22175002602849) bank84312 +84313 POINT(40.4881967969779 74.78145003998183) bank84313 +84314 POINT(40.060928477327636 75.00295382277362) bank84314 +84315 POINT(40.37078692023727 73.30195177257976) bank84315 +84316 POINT(41.444973255664664 74.24547988050192) bank84316 +84317 POINT(40.16434061889144 73.75554102339635) bank84317 +84318 POINT(40.92675041226966 73.21749939671935) bank84318 +84319 POINT(41.05804086779294 74.81924458969168) bank84319 +84320 POINT(39.740825324543316 73.80715950043941) bank84320 +84321 POINT(41.46015075426416 74.47825449836796) bank84321 +84322 POINT(41.319520156075924 73.9674782436756) bank84322 +84323 POINT(40.52376832648355 73.50792910159008) bank84323 +84324 POINT(40.38056340852928 73.36825518678309) bank84324 +84325 POINT(39.89286743286727 73.89662770639026) bank84325 +84326 POINT(40.258202925988165 73.89467250338377) bank84326 +84327 POINT(40.49630011173425 74.43170048963604) bank84327 +84328 POINT(41.41634497454878 73.99610744055673) bank84328 +84329 POINT(40.114773279172304 74.15293559100735) bank84329 +84330 POINT(39.88094648634063 73.99590600653954) bank84330 +84331 POINT(39.88193212609378 73.25398116005931) bank84331 +84332 POINT(40.70105403970494 73.71817085764253) bank84332 +84333 POINT(41.55363453885122 74.61733981544255) bank84333 +84334 POINT(40.15322014597029 73.80336378854089) bank84334 +84335 POINT(41.68784496900312 73.69659855754792) bank84335 +84336 POINT(41.21562838275367 73.76062328320613) bank84336 +84337 POINT(40.7291493749121 73.48210234234186) bank84337 +84338 POINT(41.02355615421523 74.6242873750024) bank84338 +84339 POINT(39.761608636081995 73.26608737784589) bank84339 +84340 POINT(40.55988999727204 73.13274120622206) bank84340 +84341 POINT(40.35332696639302 74.34519580094815) bank84341 +84342 POINT(39.81538024276422 74.46103214417752) bank84342 +84343 POINT(40.94322185646039 74.0905704707719) bank84343 +84344 POINT(40.667928047418926 73.94534078255623) bank84344 +84345 POINT(41.509213893314254 74.52981177255579) bank84345 +84346 POINT(40.72650063460403 73.73020628898063) bank84346 +84347 POINT(40.013217557724424 74.32421110907067) bank84347 +84348 POINT(40.49439368551347 73.24394900457303) bank84348 +84349 POINT(40.15090174289747 73.41830590092961) bank84349 +84350 POINT(40.188563384799 74.03826714711961) bank84350 +84351 POINT(40.687916546211625 73.39283359422654) bank84351 +84352 POINT(41.200607309117736 73.47227113624137) bank84352 +84353 POINT(39.80958646045464 74.89255346474674) bank84353 +84354 POINT(41.17299168755318 74.77251560919778) bank84354 +84355 POINT(40.689009174002116 74.24966188504328) bank84355 +84356 POINT(39.77251285761893 74.49594963890378) bank84356 +84357 POINT(41.306384315959434 73.55062879842745) bank84357 +84358 POINT(41.667177121942714 74.24707923274052) bank84358 +84359 POINT(40.33976633634494 73.28914715657758) bank84359 +84360 POINT(40.91768374210364 74.57157725543811) bank84360 +84361 POINT(41.397817740531664 73.85229973021909) bank84361 +84362 POINT(40.98684253740683 73.73650670713249) bank84362 +84363 POINT(40.03508530779288 74.24586262239497) bank84363 +84364 POINT(41.54639023690913 74.34751886621625) bank84364 +84365 POINT(41.00734880178885 74.30640931135969) bank84365 +84366 POINT(39.91242268487227 73.20455578123419) bank84366 +84367 POINT(40.632515785480024 73.9873060828996) bank84367 +84368 POINT(41.26799322149821 74.2585513060156) bank84368 +84369 POINT(40.767942359698665 74.11218814591524) bank84369 +84370 POINT(40.341287507800835 73.06289509397331) bank84370 +84371 POINT(41.46674234138359 73.87760323316402) bank84371 +84372 POINT(40.760971364660385 73.2087695502353) bank84372 +84373 POINT(41.051767597517745 74.01817294825331) bank84373 +84374 POINT(41.08707635511372 74.53513354401552) bank84374 +84375 POINT(40.45549519331638 73.92286411626719) bank84375 +84376 POINT(41.10708103867748 74.07484324266329) bank84376 +84377 POINT(39.7571911197812 74.19197103683847) bank84377 +84378 POINT(41.65225910945526 74.70409657545532) bank84378 +84379 POINT(40.55063617991511 74.94900589671546) bank84379 +84380 POINT(41.529982161736925 73.37385327497093) bank84380 +84381 POINT(41.376266315605356 73.66367549777956) bank84381 +84382 POINT(41.2250440693371 73.27461485671388) bank84382 +84383 POINT(41.34262194336757 74.47652538010277) bank84383 +84384 POINT(39.752459544047774 73.09709844243936) bank84384 +84385 POINT(40.90811455271185 74.5757448434185) bank84385 +84386 POINT(41.689001040225044 74.31775893825387) bank84386 +84387 POINT(41.706577673948814 73.0524336662456) bank84387 +84388 POINT(40.87623324021273 73.18982452809037) bank84388 +84389 POINT(39.719505252823794 74.97369130168121) bank84389 +84390 POINT(39.812909112481954 73.76401903594865) bank84390 +84391 POINT(41.47063017019375 74.26723511333226) bank84391 +84392 POINT(41.67259965238969 73.92821938363478) bank84392 +84393 POINT(40.960641981756886 74.95193695377613) bank84393 +84394 POINT(40.19330959695046 74.3323064199244) bank84394 +84395 POINT(40.23027053441292 74.80482411444595) bank84395 +84396 POINT(41.45552605810485 73.14868363306732) bank84396 +84397 POINT(40.44317284855633 74.76176773644421) bank84397 +84398 POINT(41.44565244470635 73.87609630498947) bank84398 +84399 POINT(40.15472433870413 74.4966976335562) bank84399 +84400 POINT(39.87485198140816 73.49377661301787) bank84400 +84401 POINT(41.39674709656405 74.80303365672768) bank84401 +84402 POINT(40.28105144287646 73.57150445506147) bank84402 +84403 POINT(40.04162600675943 73.37245149384843) bank84403 +84404 POINT(41.523882480521465 73.86072295336102) bank84404 +84405 POINT(40.66036636766034 73.0087023751972) bank84405 +84406 POINT(41.30857731650749 74.38401857725287) bank84406 +84407 POINT(41.013884883668105 73.43048570207769) bank84407 +84408 POINT(39.855565958818204 73.78315802730125) bank84408 +84409 POINT(39.89631416881978 74.49488126602417) bank84409 +84410 POINT(39.99425029592721 73.99718982911213) bank84410 +84411 POINT(41.629332259648265 74.65621443532255) bank84411 +84412 POINT(41.22743932322602 73.05509762957806) bank84412 +84413 POINT(39.72763896636665 74.55885619368874) bank84413 +84414 POINT(40.822698370845295 73.1608400129595) bank84414 +84415 POINT(41.32933629769185 74.95623981966577) bank84415 +84416 POINT(40.96646911512747 73.1593250371105) bank84416 +84417 POINT(40.407873608185994 73.17698145000358) bank84417 +84418 POINT(40.384888639593356 74.92537181612255) bank84418 +84419 POINT(40.01207848855625 74.26746615061154) bank84419 +84420 POINT(39.795941588685594 73.39289794015441) bank84420 +84421 POINT(40.0944066568423 73.67361369674707) bank84421 +84422 POINT(41.170332042586395 74.97924810188884) bank84422 +84423 POINT(41.49190535387049 74.74877197819141) bank84423 +84424 POINT(40.92674101583089 74.48201772418312) bank84424 +84425 POINT(41.24421841991683 74.15216025582373) bank84425 +84426 POINT(41.00198864527421 73.76934863731285) bank84426 +84427 POINT(41.391766913869475 74.4407156772315) bank84427 +84428 POINT(41.30115474631916 73.95673286010462) bank84428 +84429 POINT(40.275026520585904 74.90626270877462) bank84429 +84430 POINT(40.0569289371258 73.0894008377534) bank84430 +84431 POINT(39.86082729583343 74.01679479877171) bank84431 +84432 POINT(41.1815220141138 74.47278701180532) bank84432 +84433 POINT(41.06087378443075 73.70601055869682) bank84433 +84434 POINT(39.965819235799515 73.92705518135942) bank84434 +84435 POINT(40.50906115153823 73.51972297465375) bank84435 +84436 POINT(40.41791323294199 73.55957682421378) bank84436 +84437 POINT(40.411988424209895 73.67457819465106) bank84437 +84438 POINT(40.89512411292974 73.3758977716294) bank84438 +84439 POINT(41.702902082357106 73.64790363551246) bank84439 +84440 POINT(41.452548959993756 74.89819560045898) bank84440 +84441 POINT(40.20719508062603 73.80163729884545) bank84441 +84442 POINT(39.846020264101796 73.446731020961) bank84442 +84443 POINT(39.750697579403216 74.49088237433028) bank84443 +84444 POINT(41.69811711972061 73.64146531492645) bank84444 +84445 POINT(40.336525678555134 73.27924393225283) bank84445 +84446 POINT(40.11048973163908 74.7912951090596) bank84446 +84447 POINT(40.89364806918987 73.17341869922106) bank84447 +84448 POINT(39.8506724286163 73.63696651293857) bank84448 +84449 POINT(39.876494618075775 73.3194683092926) bank84449 +84450 POINT(40.601758192422075 73.57526222105797) bank84450 +84451 POINT(41.23964416348196 73.02784910685114) bank84451 +84452 POINT(39.896512775177584 74.28455234333084) bank84452 +84453 POINT(40.92107519870605 74.5367658729223) bank84453 +84454 POINT(40.19332907487055 73.49488137349265) bank84454 +84455 POINT(40.45266645806826 73.59133024961294) bank84455 +84456 POINT(40.754416505074694 73.22505558761605) bank84456 +84457 POINT(40.283716952883125 74.84295138806677) bank84457 +84458 POINT(40.99691675238233 73.45206410089364) bank84458 +84459 POINT(40.879032779013365 74.96101220431787) bank84459 +84460 POINT(41.17177146684558 74.01921001294093) bank84460 +84461 POINT(41.565239942954946 73.43589977467758) bank84461 +84462 POINT(41.19510334438689 74.1817155730874) bank84462 +84463 POINT(41.681597630294455 73.96576706511311) bank84463 +84464 POINT(40.53463020784683 74.54023563503212) bank84464 +84465 POINT(41.45775814085277 73.62080836873601) bank84465 +84466 POINT(40.73360380464362 74.13656810986963) bank84466 +84467 POINT(41.532842690794865 73.34275449588839) bank84467 +84468 POINT(39.91306995935749 74.49690725864318) bank84468 +84469 POINT(40.18680440477923 73.53862971304942) bank84469 +84470 POINT(40.011396435985176 73.02676004346071) bank84470 +84471 POINT(41.09814643280403 73.94374259673066) bank84471 +84472 POINT(41.47324964008209 74.62756834113497) bank84472 +84473 POINT(41.59434168365787 73.18936789837088) bank84473 +84474 POINT(40.55877966203865 73.44733380862625) bank84474 +84475 POINT(40.04381594562499 74.7863424226853) bank84475 +84476 POINT(41.517070727935526 74.20165171807827) bank84476 +84477 POINT(40.80709712176974 73.19361783447717) bank84477 +84478 POINT(41.355557896579136 73.01351878924743) bank84478 +84479 POINT(40.30750080412831 73.80090786743186) bank84479 +84480 POINT(40.64974449830998 73.65556780631297) bank84480 +84481 POINT(40.9523458038484 74.12225668594263) bank84481 +84482 POINT(40.53799515390552 74.14371961298264) bank84482 +84483 POINT(40.33804888567015 73.85157768216114) bank84483 +84484 POINT(40.15286079761149 73.62081978720829) bank84484 +84485 POINT(41.056507077744655 73.35489022907863) bank84485 +84486 POINT(40.82297444906189 73.78397405909375) bank84486 +84487 POINT(40.37720900455192 73.76901806019005) bank84487 +84488 POINT(41.70624875562168 73.95391349505333) bank84488 +84489 POINT(41.48682515585503 73.06399069131825) bank84489 +84490 POINT(40.67293895692256 74.59280944806078) bank84490 +84491 POINT(40.428313614798 74.77376727540286) bank84491 +84492 POINT(39.75113556114003 74.80606016817737) bank84492 +84493 POINT(41.570467625572235 73.01481980666479) bank84493 +84494 POINT(40.81133378240778 74.85624447940111) bank84494 +84495 POINT(41.404611360274664 73.60368710117733) bank84495 +84496 POINT(41.04741158965216 73.83249397422752) bank84496 +84497 POINT(40.1614792482478 73.89658518904864) bank84497 +84498 POINT(41.27946795842135 74.17708272550784) bank84498 +84499 POINT(40.124988277934435 74.90080616276477) bank84499 +84500 POINT(40.13396704372962 73.31935888648624) bank84500 +84501 POINT(41.426324010178085 74.43458283732967) bank84501 +84502 POINT(41.59821613399402 73.21816102891466) bank84502 +84503 POINT(40.779986031521695 73.69677211932054) bank84503 +84504 POINT(39.99734836291262 73.21451087169974) bank84504 +84505 POINT(41.018770478184216 74.16416168851288) bank84505 +84506 POINT(40.069018682752265 73.5390031395336) bank84506 +84507 POINT(41.57679768896493 74.84969249798789) bank84507 +84508 POINT(40.86699715371928 73.66789793586396) bank84508 +84509 POINT(41.410271110323386 74.94583114092734) bank84509 +84510 POINT(40.046090576818116 74.62492383452226) bank84510 +84511 POINT(40.94409344472269 73.2239473873893) bank84511 +84512 POINT(39.93050242914443 73.70710326031706) bank84512 +84513 POINT(41.47267502012984 74.82824155741666) bank84513 +84514 POINT(41.020018895937056 73.65999396951008) bank84514 +84515 POINT(40.29749374960508 74.92495491412203) bank84515 +84516 POINT(39.97145321353309 74.23501039238668) bank84516 +84517 POINT(40.65571384039741 73.05198534982217) bank84517 +84518 POINT(40.585934714221075 73.95493414691593) bank84518 +84519 POINT(41.66396633950174 74.10096344461684) bank84519 +84520 POINT(41.23986047304164 73.65920605074753) bank84520 +84521 POINT(41.30824486798403 73.37338127698612) bank84521 +84522 POINT(41.4258256316825 74.34108277301465) bank84522 +84523 POINT(40.82540010512851 73.11612637322364) bank84523 +84524 POINT(41.358078731501486 74.63289916090066) bank84524 +84525 POINT(41.649582128429344 74.78624015174661) bank84525 +84526 POINT(41.40854273127627 74.18621163432103) bank84526 +84527 POINT(40.685428589318924 74.212311180839) bank84527 +84528 POINT(39.97081148556273 74.4916771592201) bank84528 +84529 POINT(41.071753812376485 73.6555632173541) bank84529 +84530 POINT(40.463156423349886 74.47137114034501) bank84530 +84531 POINT(40.87878369338988 73.51793681680522) bank84531 +84532 POINT(40.75533004564484 74.2141255357184) bank84532 +84533 POINT(39.994753587074115 74.35354016472718) bank84533 +84534 POINT(41.136779044398665 73.36345031978409) bank84534 +84535 POINT(40.28345093944213 74.95106142008513) bank84535 +84536 POINT(41.27117563015324 74.9287949253808) bank84536 +84537 POINT(41.28107007872819 73.06307333026004) bank84537 +84538 POINT(39.837015868518314 74.11584055119377) bank84538 +84539 POINT(41.1971691729658 74.2005243954771) bank84539 +84540 POINT(39.80282395232254 73.82572827623038) bank84540 +84541 POINT(40.925795155688256 74.93410073645698) bank84541 +84542 POINT(39.91432326465139 73.7543285453879) bank84542 +84543 POINT(39.85848504903093 74.96415923299165) bank84543 +84544 POINT(41.12317977993816 74.09648600633783) bank84544 +84545 POINT(40.39748853030853 73.72963547295095) bank84545 +84546 POINT(40.65326789050723 74.93256573364265) bank84546 +84547 POINT(40.75433624356029 73.72914940058001) bank84547 +84548 POINT(40.91282922561731 73.78537285689829) bank84548 +84549 POINT(40.16859747018554 73.7282343665708) bank84549 +84550 POINT(41.4575398720938 73.05018285855496) bank84550 +84551 POINT(39.77886677163437 74.04937975315241) bank84551 +84552 POINT(40.914927267054075 74.60427124119255) bank84552 +84553 POINT(41.55632251325129 73.33587748995275) bank84553 +84554 POINT(40.96409041655222 74.9969467108484) bank84554 +84555 POINT(41.18240049617315 74.20092632949184) bank84555 +84556 POINT(41.12014646397396 73.0595787294294) bank84556 +84557 POINT(39.82507778790773 73.78433202035706) bank84557 +84558 POINT(40.80614275502632 74.81623656808478) bank84558 +84559 POINT(40.58071076816238 73.38181795826736) bank84559 +84560 POINT(41.20716208850347 73.78348473441302) bank84560 +84561 POINT(41.427513190812654 73.08706050876755) bank84561 +84562 POINT(39.80217289529662 74.24170520049982) bank84562 +84563 POINT(40.358615905563056 73.39656815762187) bank84563 +84564 POINT(41.564440860873255 73.33598011045416) bank84564 +84565 POINT(40.18723687810894 74.15559481626002) bank84565 +84566 POINT(39.93367803227925 74.86778865625945) bank84566 +84567 POINT(41.27717948006263 73.65244464753731) bank84567 +84568 POINT(41.51825856480133 73.90812377901104) bank84568 +84569 POINT(41.64775377456612 73.82962903496816) bank84569 +84570 POINT(40.13065932692002 74.86082596289708) bank84570 +84571 POINT(41.21664004956433 74.27719366177753) bank84571 +84572 POINT(41.393137233918154 73.41612040258954) bank84572 +84573 POINT(41.39254812772062 74.75733011652007) bank84573 +84574 POINT(41.460352074092064 73.84457783393256) bank84574 +84575 POINT(41.593337352451684 73.11357572872373) bank84575 +84576 POINT(41.122890262991945 74.46545435743222) bank84576 +84577 POINT(39.903386349423144 73.98221042570366) bank84577 +84578 POINT(41.03599274148215 74.41157363807349) bank84578 +84579 POINT(40.65819355718876 74.34671274756182) bank84579 +84580 POINT(41.185761010521624 73.9370344796747) bank84580 +84581 POINT(40.736540094613034 73.22074966690397) bank84581 +84582 POINT(40.64543891037085 73.13805421067546) bank84582 +84583 POINT(40.165969524991695 74.9478122393631) bank84583 +84584 POINT(41.440455083398945 73.48576807369331) bank84584 +84585 POINT(40.98054950155271 73.66925475090859) bank84585 +84586 POINT(41.05201032356954 73.22726414193757) bank84586 +84587 POINT(40.52123756394547 73.48852610586539) bank84587 +84588 POINT(40.49584267545733 74.2258255976835) bank84588 +84589 POINT(40.07545213609253 73.39533318630988) bank84589 +84590 POINT(39.886708941647626 73.25911282369637) bank84590 +84591 POINT(40.99460314306113 74.47782394159508) bank84591 +84592 POINT(40.098193170639576 73.62068596025752) bank84592 +84593 POINT(40.49772566372537 73.02036636901438) bank84593 +84594 POINT(41.44211947181783 73.7775530577662) bank84594 +84595 POINT(40.21705075503513 74.72653216273017) bank84595 +84596 POINT(39.71930192058353 73.30638119073981) bank84596 +84597 POINT(40.363439498116406 73.22818214900566) bank84597 +84598 POINT(41.46838212664313 73.24615217991395) bank84598 +84599 POINT(40.82539462222348 74.80761875635991) bank84599 +84600 POINT(41.58964541984324 73.80205290010068) bank84600 +84601 POINT(41.32463215402365 73.24409529472597) bank84601 +84602 POINT(39.87952111146498 74.2514507320621) bank84602 +84603 POINT(40.31942163632249 74.29864874890266) bank84603 +84604 POINT(40.654190681221 73.16956723365075) bank84604 +84605 POINT(41.66878988066504 74.53866315533537) bank84605 +84606 POINT(40.75630928045722 74.45536726508399) bank84606 +84607 POINT(41.059969821260815 73.12775256117999) bank84607 +84608 POINT(41.57344282725178 73.53707414745269) bank84608 +84609 POINT(40.19400041942273 74.59600196648772) bank84609 +84610 POINT(40.59515179012961 73.03382544973968) bank84610 +84611 POINT(40.069983548100154 73.99037498013062) bank84611 +84612 POINT(40.457442584058214 73.10483799983864) bank84612 +84613 POINT(41.60249010067597 74.98330534878852) bank84613 +84614 POINT(41.437402876432586 74.57805528067131) bank84614 +84615 POINT(40.48251589105313 73.76080157292328) bank84615 +84616 POINT(39.79554619389108 73.1617437057214) bank84616 +84617 POINT(40.79317907194546 73.51458546168038) bank84617 +84618 POINT(41.334058409726175 74.5718898602845) bank84618 +84619 POINT(41.224889533578576 73.67513772658548) bank84619 +84620 POINT(40.979937585971825 74.09685191105736) bank84620 +84621 POINT(39.78195047475843 73.30869137445735) bank84621 +84622 POINT(39.92410691317585 73.71236726271204) bank84622 +84623 POINT(40.77622333041811 73.7451022529038) bank84623 +84624 POINT(41.51748973219333 74.45369623825034) bank84624 +84625 POINT(40.88688380276303 74.0114416396617) bank84625 +84626 POINT(39.757632245527184 73.15481290491142) bank84626 +84627 POINT(40.280455892245214 74.47488004200933) bank84627 +84628 POINT(40.82363050210067 74.65301201160531) bank84628 +84629 POINT(40.03949403073817 74.35204257236704) bank84629 +84630 POINT(40.60356528069509 73.59905138324864) bank84630 +84631 POINT(40.23584192865141 73.7666647238934) bank84631 +84632 POINT(40.25157980184916 74.37758284032145) bank84632 +84633 POINT(41.09854824739082 74.43040556585255) bank84633 +84634 POINT(41.14850063318604 74.41677349541389) bank84634 +84635 POINT(41.15070984707097 73.01049640658644) bank84635 +84636 POINT(40.49446976474236 73.77332774562645) bank84636 +84637 POINT(40.14493093291262 73.92473829861947) bank84637 +84638 POINT(40.74551198286553 74.44216494261622) bank84638 +84639 POINT(40.3448344382807 73.69862072258587) bank84639 +84640 POINT(40.510879778017795 74.53503269667695) bank84640 +84641 POINT(40.343239061273174 74.77465630763696) bank84641 +84642 POINT(40.74940525602649 74.73019570991902) bank84642 +84643 POINT(39.77280830321597 74.13275712661051) bank84643 +84644 POINT(39.9557325953194 74.22744836766198) bank84644 +84645 POINT(40.262908057970456 74.15059191536636) bank84645 +84646 POINT(40.935403633278874 73.43972602940326) bank84646 +84647 POINT(40.801182531234645 73.95119030490599) bank84647 +84648 POINT(40.58281499405375 74.82362099615554) bank84648 +84649 POINT(40.79101548372527 74.56775196612023) bank84649 +84650 POINT(40.268015151191854 73.26308494552258) bank84650 +84651 POINT(40.93956540929672 73.64408738495452) bank84651 +84652 POINT(39.88718225749368 73.05097096137195) bank84652 +84653 POINT(41.350886610147754 74.6057496930161) bank84653 +84654 POINT(41.10645933298058 74.89350225596993) bank84654 +84655 POINT(41.58383572493766 74.7430155473168) bank84655 +84656 POINT(41.24423326407878 73.15401690850358) bank84656 +84657 POINT(40.45311101288213 73.35323205745314) bank84657 +84658 POINT(40.76402370609878 73.06422311056168) bank84658 +84659 POINT(39.71674000822877 73.54926924644958) bank84659 +84660 POINT(41.0984115172493 74.54921099325941) bank84660 +84661 POINT(40.29691840422697 74.02939169611788) bank84661 +84662 POINT(39.88600699734561 73.30166580303326) bank84662 +84663 POINT(40.504718199691354 74.19751821864786) bank84663 +84664 POINT(40.71534305359036 74.63323153359832) bank84664 +84665 POINT(40.536162776881 74.66968509576138) bank84665 +84666 POINT(41.105354854481796 73.96220562322351) bank84666 +84667 POINT(40.78594451290073 74.28942060978092) bank84667 +84668 POINT(40.92418646496483 74.86046081022201) bank84668 +84669 POINT(39.718909929813435 73.96796216017289) bank84669 +84670 POINT(40.161104475743905 73.54125629467079) bank84670 +84671 POINT(40.85004761689099 73.27556827246887) bank84671 +84672 POINT(40.08124106990042 74.85560564213768) bank84672 +84673 POINT(41.60656346917678 73.47761446606783) bank84673 +84674 POINT(41.33443356695896 73.31721748209125) bank84674 +84675 POINT(39.802072980898565 74.18065181081344) bank84675 +84676 POINT(40.18927380841776 73.90494088551037) bank84676 +84677 POINT(41.196990829329735 74.25328721890328) bank84677 +84678 POINT(39.89081217567635 73.9411962465143) bank84678 +84679 POINT(41.10333935844668 74.94965373766019) bank84679 +84680 POINT(39.92142710278441 74.9879463327786) bank84680 +84681 POINT(40.339533870143754 74.83126618034832) bank84681 +84682 POINT(40.82339485256048 73.4395141247824) bank84682 +84683 POINT(40.0535899546692 74.7495923807536) bank84683 +84684 POINT(40.08011293406594 74.135121415169) bank84684 +84685 POINT(40.68758223022207 73.74112222353712) bank84685 +84686 POINT(41.354274638844664 73.52206515052269) bank84686 +84687 POINT(41.621535599524314 74.8483367337102) bank84687 +84688 POINT(40.1399206084456 74.20649566104436) bank84688 +84689 POINT(41.65331445343309 73.47660813180569) bank84689 +84690 POINT(39.844453120274416 73.0579375126173) bank84690 +84691 POINT(41.265503928666796 73.82755960577909) bank84691 +84692 POINT(41.426037917793956 73.67852070580219) bank84692 +84693 POINT(41.15505326134597 74.90412365979499) bank84693 +84694 POINT(40.91041487760927 73.3435855102322) bank84694 +84695 POINT(40.87967391630154 73.624966369256) bank84695 +84696 POINT(41.50310061938031 73.42846900004601) bank84696 +84697 POINT(41.275780886489486 74.38329073775981) bank84697 +84698 POINT(41.017755226078954 73.93733462827616) bank84698 +84699 POINT(40.6399507878807 73.10605309017588) bank84699 +84700 POINT(40.987630323305325 74.84796404879287) bank84700 +84701 POINT(40.90410001735178 74.36262336217035) bank84701 +84702 POINT(40.02374093339705 74.73441300861133) bank84702 +84703 POINT(41.701717491092055 73.0126249728274) bank84703 +84704 POINT(40.63944518274169 74.3092814639427) bank84704 +84705 POINT(40.75818686411833 74.57303051658822) bank84705 +84706 POINT(41.59957155853564 73.89265390433995) bank84706 +84707 POINT(40.894917429946716 73.72833806746904) bank84707 +84708 POINT(40.31213797952825 73.21158870686048) bank84708 +84709 POINT(41.03709598640278 74.73761770360049) bank84709 +84710 POINT(40.26270932713193 73.50111739053406) bank84710 +84711 POINT(40.80765198004997 74.59502446199318) bank84711 +84712 POINT(39.88558642077057 74.23276014255654) bank84712 +84713 POINT(41.38195740644916 73.05578279639988) bank84713 +84714 POINT(39.8006549494085 73.22094483049052) bank84714 +84715 POINT(40.910093709973616 74.18118838262043) bank84715 +84716 POINT(41.63290706630541 73.55313341876109) bank84716 +84717 POINT(40.38259411353167 73.92808619752728) bank84717 +84718 POINT(41.523999022255325 73.45952624918577) bank84718 +84719 POINT(40.592241817154935 74.77316436303927) bank84719 +84720 POINT(40.56099348835519 74.71373525465988) bank84720 +84721 POINT(40.3416353749183 73.95011847761556) bank84721 +84722 POINT(41.366138243485665 73.354740143658) bank84722 +84723 POINT(40.05192959138102 74.19142167328772) bank84723 +84724 POINT(41.369455055853535 73.67340342579679) bank84724 +84725 POINT(40.22552593159847 73.0327179162305) bank84725 +84726 POINT(41.124449761259925 73.4893869598574) bank84726 +84727 POINT(40.77512765307621 73.57944817236168) bank84727 +84728 POINT(40.779811945634385 74.3707901304813) bank84728 +84729 POINT(39.72919440150673 74.74028552491482) bank84729 +84730 POINT(41.270964576817576 73.54042615347193) bank84730 +84731 POINT(41.49328706883148 73.61855782781082) bank84731 +84732 POINT(40.32668636987378 73.75653746863604) bank84732 +84733 POINT(41.19313731716469 74.0425989021401) bank84733 +84734 POINT(41.70765800977027 74.34296444064074) bank84734 +84735 POINT(39.93690712239367 74.14810469212647) bank84735 +84736 POINT(39.739311946601546 74.17434001103149) bank84736 +84737 POINT(39.86947690698983 73.2482167688067) bank84737 +84738 POINT(41.14252608824782 74.58096441406879) bank84738 +84739 POINT(39.75600766627178 73.52505086071314) bank84739 +84740 POINT(39.92110305600402 73.28393775083352) bank84740 +84741 POINT(40.646778009158936 74.98811452836584) bank84741 +84742 POINT(41.33849187568352 74.2843771950534) bank84742 +84743 POINT(40.463541269943406 74.54209349329412) bank84743 +84744 POINT(39.758194600863455 74.64086636495628) bank84744 +84745 POINT(41.332343183318294 74.25479145018195) bank84745 +84746 POINT(41.63945178033487 73.814264499458) bank84746 +84747 POINT(40.15840067995436 73.1112068882387) bank84747 +84748 POINT(40.918177034158155 73.44376844756403) bank84748 +84749 POINT(41.412265534152105 74.6169353666243) bank84749 +84750 POINT(39.74619929034002 74.32504961811469) bank84750 +84751 POINT(41.244702951180884 74.77652069047802) bank84751 +84752 POINT(40.07312515588553 73.0592305131242) bank84752 +84753 POINT(39.75716715241303 74.63337531802253) bank84753 +84754 POINT(40.13669874644777 74.08255207529783) bank84754 +84755 POINT(40.906189822961416 74.46526692421358) bank84755 +84756 POINT(41.40192531361276 73.32095941154442) bank84756 +84757 POINT(39.9235879275952 74.18249258403225) bank84757 +84758 POINT(40.186522955656635 74.98351928272969) bank84758 +84759 POINT(39.941154136768425 73.76288399186241) bank84759 +84760 POINT(41.492181398026375 73.68512586504207) bank84760 +84761 POINT(40.525516455129576 73.8256696394956) bank84761 +84762 POINT(40.57157483366054 74.04297554665281) bank84762 +84763 POINT(40.96805935552275 73.49827026928943) bank84763 +84764 POINT(41.20135391738824 74.7093639241934) bank84764 +84765 POINT(40.70669817587085 73.40257814745549) bank84765 +84766 POINT(41.07544946741004 74.01288156350411) bank84766 +84767 POINT(40.37155824715931 74.33491085709042) bank84767 +84768 POINT(40.88847213662853 74.0679102153223) bank84768 +84769 POINT(39.90151868409192 74.70339962850004) bank84769 +84770 POINT(40.97959816946709 73.29895436966503) bank84770 +84771 POINT(39.9376599616071 73.7105411766976) bank84771 +84772 POINT(39.76354926972252 73.33917483631937) bank84772 +84773 POINT(40.0569511950698 74.2570680078291) bank84773 +84774 POINT(40.561071585499704 74.85151655525007) bank84774 +84775 POINT(41.40882414702848 73.11953531401967) bank84775 +84776 POINT(40.31678403680699 73.39746189233254) bank84776 +84777 POINT(40.25727968575202 73.24172100865785) bank84777 +84778 POINT(40.3364942164423 74.01024475001482) bank84778 +84779 POINT(39.79633172555417 73.50665961494353) bank84779 +84780 POINT(40.19988165585726 73.07171771497788) bank84780 +84781 POINT(41.19381709349863 74.57032290509048) bank84781 +84782 POINT(39.81507407317368 73.44465675353922) bank84782 +84783 POINT(41.46291694781647 73.33207535632346) bank84783 +84784 POINT(40.71701318423353 74.1353520446303) bank84784 +84785 POINT(41.079735255530274 74.13312779721132) bank84785 +84786 POINT(41.155595753161634 73.7614266629172) bank84786 +84787 POINT(40.43965252808644 73.35474177613966) bank84787 +84788 POINT(41.010567725552576 73.15434988744545) bank84788 +84789 POINT(41.70838343785476 74.05718846490338) bank84789 +84790 POINT(39.820485320225835 74.71863661161447) bank84790 +84791 POINT(41.382018783389576 73.47204591583638) bank84791 +84792 POINT(39.899868558012486 73.05531922135562) bank84792 +84793 POINT(41.2437338997805 73.91692371342408) bank84793 +84794 POINT(39.951795371674194 74.28682066163474) bank84794 +84795 POINT(41.31135331461747 73.31474462148792) bank84795 +84796 POINT(40.62541756996216 73.99528470571568) bank84796 +84797 POINT(40.50408998121406 74.07553117774421) bank84797 +84798 POINT(41.53622373720002 73.84227811096999) bank84798 +84799 POINT(40.85621518828438 73.13052593492208) bank84799 +84800 POINT(39.95813656076606 73.54311946709221) bank84800 +84801 POINT(40.08820781906358 74.53605928204873) bank84801 +84802 POINT(40.293252552101336 74.11538913401708) bank84802 +84803 POINT(40.40002308109909 73.75352986064567) bank84803 +84804 POINT(39.92016396543193 74.61311170740846) bank84804 +84805 POINT(40.8243164050144 73.99670458699862) bank84805 +84806 POINT(40.04744245272513 73.2177832656901) bank84806 +84807 POINT(41.13810592516507 74.74981318816835) bank84807 +84808 POINT(40.24674763727862 73.18901219349057) bank84808 +84809 POINT(40.52413255364669 73.73236454833746) bank84809 +84810 POINT(40.23890699377262 74.24381254058383) bank84810 +84811 POINT(41.384319888961855 73.9556932996562) bank84811 +84812 POINT(41.04082013643569 74.80610611869137) bank84812 +84813 POINT(40.44103857912305 73.78016569800477) bank84813 +84814 POINT(40.741873867415215 74.90764881800499) bank84814 +84815 POINT(41.423844638523576 73.04107082764193) bank84815 +84816 POINT(40.27277642128731 74.12921727228186) bank84816 +84817 POINT(39.73315299304724 74.23015498367269) bank84817 +84818 POINT(39.71950982211653 74.43046001903666) bank84818 +84819 POINT(40.15732209581568 73.63130173388362) bank84819 +84820 POINT(40.65538679966938 74.2442174584858) bank84820 +84821 POINT(41.67592504643139 74.20169898674095) bank84821 +84822 POINT(41.53670580085654 74.85303442491218) bank84822 +84823 POINT(40.79302649074193 73.9470511927045) bank84823 +84824 POINT(40.839441468375085 73.17133888434992) bank84824 +84825 POINT(40.321078334922746 74.39381770252812) bank84825 +84826 POINT(40.2380942395659 73.52290811362391) bank84826 +84827 POINT(40.03659406681321 73.13480428481947) bank84827 +84828 POINT(41.154364425140585 73.02761456552041) bank84828 +84829 POINT(41.6785286150932 74.1036835191108) bank84829 +84830 POINT(40.834899374654434 73.37504830560327) bank84830 +84831 POINT(40.92346281425914 73.7379127278435) bank84831 +84832 POINT(40.141050295740946 74.9789493580706) bank84832 +84833 POINT(39.7886551920626 73.8512608047954) bank84833 +84834 POINT(40.32828101471635 74.08165069953924) bank84834 +84835 POINT(40.2611361855462 73.27652188112788) bank84835 +84836 POINT(41.3776081040019 73.82793173243218) bank84836 +84837 POINT(40.008018957846915 73.82412909599915) bank84837 +84838 POINT(40.26266549394067 74.28019819408723) bank84838 +84839 POINT(41.414867490762234 73.69629726541628) bank84839 +84840 POINT(40.62171177648422 73.87281821241513) bank84840 +84841 POINT(41.63342914509027 74.55674054935042) bank84841 +84842 POINT(41.506690557465724 73.04473453006818) bank84842 +84843 POINT(40.53775704244999 73.87541481341493) bank84843 +84844 POINT(40.53766555323289 73.56127192277519) bank84844 +84845 POINT(39.97651808210472 73.03689733161791) bank84845 +84846 POINT(40.93040870098758 74.97232773460222) bank84846 +84847 POINT(41.36278251025656 73.61010467597289) bank84847 +84848 POINT(41.700859185386996 73.17769219920886) bank84848 +84849 POINT(40.917327684569415 74.97040939517572) bank84849 +84850 POINT(40.92936635527756 73.33415132877128) bank84850 +84851 POINT(40.0306703698103 74.51496912583158) bank84851 +84852 POINT(40.57451597920699 74.34949586491794) bank84852 +84853 POINT(39.87898510314117 74.82557728150162) bank84853 +84854 POINT(41.691691606757445 74.85719027896643) bank84854 +84855 POINT(39.830167187328016 73.16168637994627) bank84855 +84856 POINT(39.97974260671435 73.66586434488342) bank84856 +84857 POINT(40.81834318401125 73.86454009023798) bank84857 +84858 POINT(40.999852908906234 73.5408159930643) bank84858 +84859 POINT(41.439978322731136 74.2918798120164) bank84859 +84860 POINT(41.45006986954198 74.01597551463915) bank84860 +84861 POINT(41.144520410824974 74.95561961133235) bank84861 +84862 POINT(40.247922457703524 73.21707619333442) bank84862 +84863 POINT(40.57112413860675 73.28249519786021) bank84863 +84864 POINT(40.10262831319221 73.50089219623631) bank84864 +84865 POINT(41.33127366643465 73.92546054983345) bank84865 +84866 POINT(40.76176471819909 73.65918166715389) bank84866 +84867 POINT(39.71675192373149 73.27825489261616) bank84867 +84868 POINT(39.900062903138554 73.07544357046818) bank84868 +84869 POINT(41.509037481334204 73.01124393439136) bank84869 +84870 POINT(41.44154804750998 73.76066856303947) bank84870 +84871 POINT(40.82481356694182 74.30120758969798) bank84871 +84872 POINT(40.421235554618505 74.88611054888422) bank84872 +84873 POINT(40.375886889864645 74.07200049404162) bank84873 +84874 POINT(41.127017685883885 74.26138334728343) bank84874 +84875 POINT(40.5876215786855 74.55878930783392) bank84875 +84876 POINT(40.61517510833271 74.79865058628292) bank84876 +84877 POINT(40.93590293674379 73.5144809518084) bank84877 +84878 POINT(39.75401210617793 75.00091967501656) bank84878 +84879 POINT(39.94798343991766 74.93314920474064) bank84879 +84880 POINT(40.182585749807245 73.72668060713438) bank84880 +84881 POINT(41.685490667964864 73.47051801705621) bank84881 +84882 POINT(40.53439054493965 73.30052797840108) bank84882 +84883 POINT(40.20361560892883 74.10835568002027) bank84883 +84884 POINT(40.7897678919628 74.90807505769978) bank84884 +84885 POINT(40.35245905033637 73.44591390469222) bank84885 +84886 POINT(40.12314193236575 74.14008160583428) bank84886 +84887 POINT(40.77180451051824 73.88390005831046) bank84887 +84888 POINT(40.02726998231158 74.57011106525341) bank84888 +84889 POINT(40.859671134923 73.72939120033949) bank84889 +84890 POINT(39.995121427778145 74.10053700851229) bank84890 +84891 POINT(39.86132409306201 73.02642725745461) bank84891 +84892 POINT(41.051103391558165 74.6888752608384) bank84892 +84893 POINT(39.90455534257657 74.55533155562368) bank84893 +84894 POINT(40.97699278680646 74.27837485738094) bank84894 +84895 POINT(40.416245901396856 73.86711348333276) bank84895 +84896 POINT(41.079758660077246 73.9621713241006) bank84896 +84897 POINT(40.99508123728966 74.34802145233647) bank84897 +84898 POINT(40.02712912411383 73.39730911196318) bank84898 +84899 POINT(40.7757627328364 73.23439496297874) bank84899 +84900 POINT(40.42288596008982 74.8348486529275) bank84900 +84901 POINT(41.00815865485082 73.26748292861453) bank84901 +84902 POINT(40.85931808408378 73.25900351507677) bank84902 +84903 POINT(41.66932711090836 73.96191259506543) bank84903 +84904 POINT(41.22962698370169 73.63048222672917) bank84904 +84905 POINT(41.638274194085014 74.13746694423493) bank84905 +84906 POINT(40.55142306130475 74.01164353732055) bank84906 +84907 POINT(41.03139640917832 74.76509433827422) bank84907 +84908 POINT(40.025924627646575 74.80130256214207) bank84908 +84909 POINT(40.61134836806538 74.66931289407759) bank84909 +84910 POINT(40.88517994121515 74.98710651065366) bank84910 +84911 POINT(40.229570285752395 73.74799831517869) bank84911 +84912 POINT(40.414764273230205 74.19781216091944) bank84912 +84913 POINT(39.80024999496203 74.78813714546102) bank84913 +84914 POINT(39.75609338156059 74.20014776330918) bank84914 +84915 POINT(40.513839431106064 74.415291984974) bank84915 +84916 POINT(39.98017099140923 73.96552829838691) bank84916 +84917 POINT(39.948528458537496 73.93078459711938) bank84917 +84918 POINT(40.973152771899365 74.31483516962263) bank84918 +84919 POINT(41.382415108293266 73.40928104750502) bank84919 +84920 POINT(40.12591504281231 74.1045890694466) bank84920 +84921 POINT(39.94809495365457 74.98918129997455) bank84921 +84922 POINT(40.395694075462735 73.20811720048155) bank84922 +84923 POINT(40.22056652670755 74.17069898726363) bank84923 +84924 POINT(41.16493347081154 74.99588343963097) bank84924 +84925 POINT(39.95886345342508 74.69790721394887) bank84925 +84926 POINT(40.87409721318107 74.1175891310458) bank84926 +84927 POINT(39.74919364599611 74.8725594085039) bank84927 +84928 POINT(39.870112380509546 73.25691090117135) bank84928 +84929 POINT(40.28746582473734 73.56901129318511) bank84929 +84930 POINT(40.302845963828986 74.69467625685861) bank84930 +84931 POINT(41.138865409645454 73.18863910016952) bank84931 +84932 POINT(41.40354688289777 74.84894897420283) bank84932 +84933 POINT(40.56633175883066 73.21643986725717) bank84933 +84934 POINT(40.7849860151362 73.58449134845564) bank84934 +84935 POINT(41.20329596930216 73.71210290081663) bank84935 +84936 POINT(40.046468209602686 73.14836938800195) bank84936 +84937 POINT(41.03688576762695 73.94307046870703) bank84937 +84938 POINT(39.86048398607724 73.85415780249613) bank84938 +84939 POINT(40.19245866115138 74.63014160870931) bank84939 +84940 POINT(39.75205693799425 73.70057432271027) bank84940 +84941 POINT(41.14082610578572 74.12885001574078) bank84941 +84942 POINT(40.660905270389996 74.76004794266987) bank84942 +84943 POINT(39.952793540164016 74.15600259001899) bank84943 +84944 POINT(40.768105748237105 73.33247265457648) bank84944 +84945 POINT(39.79379144556959 74.22570582563925) bank84945 +84946 POINT(39.802441953880894 74.80547561859558) bank84946 +84947 POINT(41.561386162664014 74.9491868939301) bank84947 +84948 POINT(40.64297562983283 73.40891484360219) bank84948 +84949 POINT(41.38720212490782 73.16575825774981) bank84949 +84950 POINT(40.57748001117046 74.17896541230634) bank84950 +84951 POINT(40.024469727436234 74.23977281423336) bank84951 +84952 POINT(41.59844976612029 73.96178401264831) bank84952 +84953 POINT(41.677791249477345 73.22017146569212) bank84953 +84954 POINT(39.97879592183985 73.72994674952463) bank84954 +84955 POINT(41.31730835246904 73.73982602488296) bank84955 +84956 POINT(41.42413908181186 74.59051710577353) bank84956 +84957 POINT(40.08214222432576 74.9890579082126) bank84957 +84958 POINT(41.22688748949661 73.9607145354837) bank84958 +84959 POINT(40.25508368510064 73.61757535532935) bank84959 +84960 POINT(40.42317716294563 73.8182164612076) bank84960 +84961 POINT(41.115562379187615 73.34267255841195) bank84961 +84962 POINT(40.64074087031161 73.30636047196639) bank84962 +84963 POINT(41.26094575812417 74.18896691335313) bank84963 +84964 POINT(40.87068583351066 73.78507910001753) bank84964 +84965 POINT(41.17949871853681 73.71549277781362) bank84965 +84966 POINT(41.401607461486606 74.11440522541237) bank84966 +84967 POINT(41.57723470538304 73.59999244133739) bank84967 +84968 POINT(41.18996076002674 73.01148887623899) bank84968 +84969 POINT(40.46749564970892 73.65666218406139) bank84969 +84970 POINT(41.5035372247186 73.16088012035516) bank84970 +84971 POINT(41.49462856658386 73.62661994543805) bank84971 +84972 POINT(41.49459216043548 75.00089975302863) bank84972 +84973 POINT(40.30272323159257 74.596711513547) bank84973 +84974 POINT(40.62179132394968 74.34010503149877) bank84974 +84975 POINT(40.133225328757916 73.69644341147193) bank84975 +84976 POINT(41.390378934868316 73.41176609505314) bank84976 +84977 POINT(41.10386419359085 73.52062288307066) bank84977 +84978 POINT(41.34065761513627 74.9731339226176) bank84978 +84979 POINT(40.16754706859619 73.2718960274514) bank84979 +84980 POINT(40.84177411329026 74.26233553507282) bank84980 +84981 POINT(40.84020138620268 73.53560618758426) bank84981 +84982 POINT(40.60909693475359 74.4379091393168) bank84982 +84983 POINT(41.165107664582315 74.44186288754908) bank84983 +84984 POINT(40.71154187964665 74.43446974765855) bank84984 +84985 POINT(40.391404218905556 74.35590861791786) bank84985 +84986 POINT(41.46101011631512 73.43887937073916) bank84986 +84987 POINT(39.89956443905959 73.64250263939553) bank84987 +84988 POINT(41.25795570599682 73.42451142757852) bank84988 +84989 POINT(40.306509495479546 73.85160626064086) bank84989 +84990 POINT(41.37126874902882 73.93011878231962) bank84990 +84991 POINT(41.43344788888133 74.87217546906294) bank84991 +84992 POINT(40.2920816841839 73.38494511654307) bank84992 +84993 POINT(39.90031378432554 73.96702327395515) bank84993 +84994 POINT(41.65533995587817 74.47917249928705) bank84994 +84995 POINT(40.185052547413264 74.99877523414605) bank84995 +84996 POINT(40.4335213973467 74.8124353448121) bank84996 +84997 POINT(41.46382962247818 74.43976579515144) bank84997 +84998 POINT(40.2960978868497 74.21926096478221) bank84998 +84999 POINT(41.11232368805333 73.41787154389674) bank84999 +85000 POINT(40.09894842968408 73.37071461473863) bank85000 +85001 POINT(40.14546679269376 74.85789652796392) bank85001 +85002 POINT(40.22257426879102 74.72427401879574) bank85002 +85003 POINT(41.18200970113653 73.03631227126193) bank85003 +85004 POINT(41.34586481584408 73.41971765238935) bank85004 +85005 POINT(40.52723455349622 73.28230295455701) bank85005 +85006 POINT(41.42611567096118 73.0507221077862) bank85006 +85007 POINT(41.101407123911315 73.69506452817812) bank85007 +85008 POINT(40.23352209902934 74.47219532920616) bank85008 +85009 POINT(41.08627544525456 73.07247066998225) bank85009 +85010 POINT(41.182209220698255 73.90233443672668) bank85010 +85011 POINT(41.34631106862603 73.9963360738824) bank85011 +85012 POINT(40.413719584067564 74.69547089737716) bank85012 +85013 POINT(41.6698795706119 73.04735552173919) bank85013 +85014 POINT(40.88185975405678 74.14045965583271) bank85014 +85015 POINT(40.926124449704936 74.34298515706033) bank85015 +85016 POINT(41.043103454025726 74.26248454136221) bank85016 +85017 POINT(41.535634353968156 73.33226235960592) bank85017 +85018 POINT(39.80999821323504 73.13658955703345) bank85018 +85019 POINT(39.96188918427755 73.4271305415553) bank85019 +85020 POINT(39.997695278639746 74.27085857481165) bank85020 +85021 POINT(40.44701540237907 74.91459283439974) bank85021 +85022 POINT(41.15265675003541 73.74593537357703) bank85022 +85023 POINT(40.283084953056914 74.97410756424138) bank85023 +85024 POINT(41.53207911148091 73.32228844175286) bank85024 +85025 POINT(39.81216628507188 73.10934950365001) bank85025 +85026 POINT(39.96229617821277 73.07705974948246) bank85026 +85027 POINT(40.289423755745204 73.54293299218918) bank85027 +85028 POINT(40.29064658422662 73.0139886507785) bank85028 +85029 POINT(41.41526993958916 73.83802207881529) bank85029 +85030 POINT(40.95033124744161 74.14664255060269) bank85030 +85031 POINT(40.80141408499517 74.4668324363729) bank85031 +85032 POINT(41.27591661757342 74.08905468979947) bank85032 +85033 POINT(41.221870257497734 74.1818844192464) bank85033 +85034 POINT(39.78624899342369 74.17660005313367) bank85034 +85035 POINT(41.463044344780585 74.14496659283189) bank85035 +85036 POINT(40.10770194780484 73.19028421187512) bank85036 +85037 POINT(40.87064548471236 73.98297680582957) bank85037 +85038 POINT(41.10890160848732 73.70380210005716) bank85038 +85039 POINT(40.58283971610444 74.03089651322384) bank85039 +85040 POINT(41.00037562510754 73.84953644429721) bank85040 +85041 POINT(41.411270373662276 74.79611638645186) bank85041 +85042 POINT(40.79514840059594 74.99422142575926) bank85042 +85043 POINT(40.95665763639599 74.52705556070117) bank85043 +85044 POINT(41.70878599993131 73.45367849120399) bank85044 +85045 POINT(41.101595135416545 74.0237319717779) bank85045 +85046 POINT(39.835696933112736 74.69128099175164) bank85046 +85047 POINT(41.02017391820846 73.22829551698521) bank85047 +85048 POINT(39.86954628155185 74.33513713776968) bank85048 +85049 POINT(40.57166061214011 73.23258179463213) bank85049 +85050 POINT(40.73893067740279 74.52869831337685) bank85050 +85051 POINT(40.497826372151955 74.42027129826099) bank85051 +85052 POINT(41.0296359564553 73.6489472362536) bank85052 +85053 POINT(39.94777098079372 73.46078691785861) bank85053 +85054 POINT(40.06327709517576 74.43130334702164) bank85054 +85055 POINT(40.64056090257004 74.85939005291885) bank85055 +85056 POINT(40.07210852050004 74.37961854353472) bank85056 +85057 POINT(41.08594098783206 73.33389954301755) bank85057 +85058 POINT(41.18736277752468 74.66282281483623) bank85058 +85059 POINT(41.09766034871086 74.18615908489194) bank85059 +85060 POINT(39.96147895039498 73.8808267198766) bank85060 +85061 POINT(40.671714232530476 73.57690717307831) bank85061 +85062 POINT(39.782800058007034 73.25989546987302) bank85062 +85063 POINT(41.34762119764998 74.24485262768104) bank85063 +85064 POINT(40.35755397268336 73.7689816434044) bank85064 +85065 POINT(40.386326007195315 73.11229404496162) bank85065 +85066 POINT(41.59962048462504 74.04080221875562) bank85066 +85067 POINT(40.707832180774844 73.64882218216015) bank85067 +85068 POINT(40.150802350973294 73.44330013159072) bank85068 +85069 POINT(40.731856263931135 73.02082321365214) bank85069 +85070 POINT(41.22907882453788 73.72187947334885) bank85070 +85071 POINT(41.68376813445123 74.18131529662014) bank85071 +85072 POINT(41.116306884802114 74.55677135270894) bank85072 +85073 POINT(40.071000006422935 74.98149767714412) bank85073 +85074 POINT(40.9714118882964 74.73484359526155) bank85074 +85075 POINT(41.534471950996625 74.88456573487079) bank85075 +85076 POINT(40.7766730037774 74.03405982085404) bank85076 +85077 POINT(41.695607765020384 74.59572423204119) bank85077 +85078 POINT(41.148346837629354 73.95513238498788) bank85078 +85079 POINT(41.58702897652631 73.73296514130801) bank85079 +85080 POINT(40.63454990487752 73.94121137482924) bank85080 +85081 POINT(40.64426324763026 74.19127436491817) bank85081 +85082 POINT(41.032657847648096 73.17583811153057) bank85082 +85083 POINT(41.335466494050245 73.29211394852317) bank85083 +85084 POINT(40.29762653061549 74.83562029603969) bank85084 +85085 POINT(40.65564245477818 74.04533130431253) bank85085 +85086 POINT(41.549143931885176 74.7829987076442) bank85086 +85087 POINT(41.33772145247461 73.84096925569818) bank85087 +85088 POINT(39.816163975111174 73.04966589220315) bank85088 +85089 POINT(40.79033231381634 73.97614668738933) bank85089 +85090 POINT(39.91634080569359 74.56757406729787) bank85090 +85091 POINT(39.89596741588908 73.07332855105149) bank85091 +85092 POINT(40.29596588250499 74.99155419660083) bank85092 +85093 POINT(41.00096781149677 73.09973847649604) bank85093 +85094 POINT(41.65844451080023 74.43932069782437) bank85094 +85095 POINT(40.35045294707042 74.37646786472357) bank85095 +85096 POINT(41.62786178460337 73.99818548001852) bank85096 +85097 POINT(41.67949667515754 73.11270290988806) bank85097 +85098 POINT(40.8584125897667 73.112656324324) bank85098 +85099 POINT(40.14061317715055 74.24305375560098) bank85099 +85100 POINT(40.57402214438992 74.4295647126075) bank85100 +85101 POINT(41.449199874955724 73.27456158649733) bank85101 +85102 POINT(40.32122614099252 74.12179757354237) bank85102 +85103 POINT(40.44930712283849 73.37886336592746) bank85103 +85104 POINT(40.02156966906467 73.42430830483723) bank85104 +85105 POINT(40.13930509052208 73.98075877460147) bank85105 +85106 POINT(40.674215204802664 74.7330306827515) bank85106 +85107 POINT(40.56327838297412 74.66436582726156) bank85107 +85108 POINT(41.274203661006354 74.28127043572859) bank85108 +85109 POINT(40.521626536211876 73.60134396026088) bank85109 +85110 POINT(40.419371713825136 74.0027293114442) bank85110 +85111 POINT(40.93627930451747 74.48982382336935) bank85111 +85112 POINT(41.62672954804761 74.62236729980826) bank85112 +85113 POINT(39.74672882454838 73.89272730833332) bank85113 +85114 POINT(41.40541240567647 74.46977305018126) bank85114 +85115 POINT(41.16990662283566 73.04326099652808) bank85115 +85116 POINT(40.207883362714675 73.510813567788) bank85116 +85117 POINT(40.409167902871616 73.73843727993741) bank85117 +85118 POINT(40.50546554859651 73.96534139374342) bank85118 +85119 POINT(40.92864576534242 74.24140264345694) bank85119 +85120 POINT(41.0252302472489 73.43122252552838) bank85120 +85121 POINT(41.4198035652103 73.50289651108673) bank85121 +85122 POINT(41.464286097844386 74.2408501619355) bank85122 +85123 POINT(39.92861246052967 73.4864780308108) bank85123 +85124 POINT(41.237284160908025 74.97494269620273) bank85124 +85125 POINT(39.85078484873197 74.8107229778546) bank85125 +85126 POINT(40.45661663546973 73.81619691987562) bank85126 +85127 POINT(40.801372466392415 74.7498825154246) bank85127 +85128 POINT(40.36010045708795 74.24744776711553) bank85128 +85129 POINT(41.086846497199744 74.74744030271609) bank85129 +85130 POINT(40.97875323165388 74.73080738239892) bank85130 +85131 POINT(40.52002191784141 73.8125577990316) bank85131 +85132 POINT(40.986180346729306 74.62470504656824) bank85132 +85133 POINT(40.93498383844696 73.93475048790896) bank85133 +85134 POINT(41.133213093969545 73.02968793003406) bank85134 +85135 POINT(40.94567455236232 73.65983560288906) bank85135 +85136 POINT(40.3485890534336 74.06098974621347) bank85136 +85137 POINT(41.29244894909011 74.0840751180779) bank85137 +85138 POINT(40.77786492874614 74.97490094736116) bank85138 +85139 POINT(40.25136281038793 74.40650055012676) bank85139 +85140 POINT(40.9872315334953 73.88687203876252) bank85140 +85141 POINT(39.94954843963758 74.11595168315971) bank85141 +85142 POINT(39.774818288327054 74.74658364505376) bank85142 +85143 POINT(41.6192568663274 73.94738971573933) bank85143 +85144 POINT(40.30113333847411 73.74595659384111) bank85144 +85145 POINT(40.44951176752698 73.3996101412081) bank85145 +85146 POINT(40.110965834806976 74.43416203093112) bank85146 +85147 POINT(41.15044410948569 73.94850497768144) bank85147 +85148 POINT(40.40098065936572 73.59710491615509) bank85148 +85149 POINT(39.978961154802725 73.19997017203411) bank85149 +85150 POINT(41.38098918217349 74.84494609187284) bank85150 +85151 POINT(40.50674478945273 73.48645712918015) bank85151 +85152 POINT(40.60001405004271 74.08264592407338) bank85152 +85153 POINT(41.43631053334892 74.85281235462905) bank85153 +85154 POINT(41.06418618415421 74.46369199221304) bank85154 +85155 POINT(41.422902966202756 73.71159721292254) bank85155 +85156 POINT(40.09393207929596 73.27246299188945) bank85156 +85157 POINT(40.867873698820084 74.7499660936255) bank85157 +85158 POINT(41.449909150212804 74.03739398568334) bank85158 +85159 POINT(40.24904676816605 73.31052690722953) bank85159 +85160 POINT(41.580293719978116 74.34531411365381) bank85160 +85161 POINT(41.30409822721288 73.18651437220448) bank85161 +85162 POINT(41.08884813330406 73.25646895993559) bank85162 +85163 POINT(41.5111928640494 74.46875080482914) bank85163 +85164 POINT(40.12864174252129 73.5960271209419) bank85164 +85165 POINT(40.796919779372004 73.072037359633) bank85165 +85166 POINT(41.099238446226636 73.22354307432578) bank85166 +85167 POINT(39.94996067776075 73.00856980231629) bank85167 +85168 POINT(40.27725858491974 73.67405297173681) bank85168 +85169 POINT(41.322514078388245 74.97618248921069) bank85169 +85170 POINT(39.86553103779856 73.687525136981) bank85170 +85171 POINT(41.343188447762934 73.12775396785162) bank85171 +85172 POINT(41.47157314265386 74.11488317689948) bank85172 +85173 POINT(41.04687365826216 74.3987741868263) bank85173 +85174 POINT(41.25887430561219 74.08125254056075) bank85174 +85175 POINT(41.18975444921618 74.05162397255452) bank85175 +85176 POINT(40.71794956392318 74.67725238100724) bank85176 +85177 POINT(41.0830476133308 73.26223530271132) bank85177 +85178 POINT(41.66060949613811 73.10471249210207) bank85178 +85179 POINT(39.883646520166806 73.48920478078588) bank85179 +85180 POINT(39.73288038442391 74.42059685574408) bank85180 +85181 POINT(40.93446300027555 74.7184905977405) bank85181 +85182 POINT(39.965983942158594 74.11964397566899) bank85182 +85183 POINT(40.85988993429461 73.15910142337853) bank85183 +85184 POINT(40.81295400193058 73.97990849586604) bank85184 +85185 POINT(41.63234246565624 73.11160357550361) bank85185 +85186 POINT(41.38380923999746 73.24330954577785) bank85186 +85187 POINT(39.832881631045886 73.87368438790504) bank85187 +85188 POINT(41.70779594952609 74.63489589622601) bank85188 +85189 POINT(40.690739520831606 74.58648715375482) bank85189 +85190 POINT(41.58286666733299 74.31141173386986) bank85190 +85191 POINT(41.466520348617195 73.02750351678513) bank85191 +85192 POINT(40.0650617247488 73.6960814943104) bank85192 +85193 POINT(40.57009827746286 73.58964362690483) bank85193 +85194 POINT(40.44973094099178 73.23451526295064) bank85194 +85195 POINT(40.842383379348256 73.99325685991532) bank85195 +85196 POINT(40.33670390598552 73.58272768434347) bank85196 +85197 POINT(40.392647217433336 74.74357845422257) bank85197 +85198 POINT(40.001356674196245 74.40222485721952) bank85198 +85199 POINT(41.66008129564614 74.65073538893036) bank85199 +85200 POINT(40.33633471168742 73.6751938748465) bank85200 +85201 POINT(41.59625992182911 73.42969843841547) bank85201 +85202 POINT(41.243061605204616 74.39138256799527) bank85202 +85203 POINT(40.515521716817574 74.65767784918339) bank85203 +85204 POINT(40.37147715038426 73.74843231872244) bank85204 +85205 POINT(41.2522759678411 73.50746410317905) bank85205 +85206 POINT(40.9128724405317 73.17923289895461) bank85206 +85207 POINT(41.37077517157194 73.71663026545133) bank85207 +85208 POINT(40.4903305606593 73.12979903892594) bank85208 +85209 POINT(39.81697274769594 74.37749405459529) bank85209 +85210 POINT(40.805851505735646 73.24611547551842) bank85210 +85211 POINT(40.72696146986642 73.80879602218883) bank85211 +85212 POINT(41.649681438948235 73.58626233794331) bank85212 +85213 POINT(39.95609022320713 73.18413283748441) bank85213 +85214 POINT(41.4333145049234 74.40548330074735) bank85214 +85215 POINT(40.75176477977633 73.05741370982567) bank85215 +85216 POINT(40.58883939904987 74.26031292836643) bank85216 +85217 POINT(41.03401931999604 73.29794090420607) bank85217 +85218 POINT(40.906775223732126 73.41450856749636) bank85218 +85219 POINT(40.53798532140941 74.88821902036935) bank85219 +85220 POINT(41.41371312157786 73.15927539469475) bank85220 +85221 POINT(40.792431120073985 73.09516522992048) bank85221 +85222 POINT(40.145662249583054 74.29426848844571) bank85222 +85223 POINT(40.911987502258185 74.08394036508524) bank85223 +85224 POINT(40.15113955043251 73.47822647677205) bank85224 +85225 POINT(41.34957109379791 74.39653401346872) bank85225 +85226 POINT(41.5628976096052 73.18684111779669) bank85226 +85227 POINT(40.29334310697493 73.50926178992692) bank85227 +85228 POINT(41.07549459760293 73.05435693526522) bank85228 +85229 POINT(41.671032156916304 74.32937539333042) bank85229 +85230 POINT(41.71084407327948 74.07570553383783) bank85230 +85231 POINT(40.61350318750968 74.87278617622734) bank85231 +85232 POINT(40.42475495236509 73.683518768101) bank85232 +85233 POINT(41.61240485556437 73.39068227035051) bank85233 +85234 POINT(39.93600663149177 74.66927943870964) bank85234 +85235 POINT(41.385690588016374 74.51675972417262) bank85235 +85236 POINT(41.1775349857175 74.18870477371668) bank85236 +85237 POINT(40.191643939048454 74.82527823772999) bank85237 +85238 POINT(40.6254910643939 73.46869253368601) bank85238 +85239 POINT(40.07155186838856 73.23447250648826) bank85239 +85240 POINT(40.646804497633816 74.31104562988834) bank85240 +85241 POINT(41.584533090775906 73.86283030148346) bank85241 +85242 POINT(40.56080164003415 74.93666571004918) bank85242 +85243 POINT(41.02118666497116 74.67213889693157) bank85243 +85244 POINT(40.575683447479264 73.79525057395061) bank85244 +85245 POINT(39.74175169839284 74.73265330640369) bank85245 +85246 POINT(39.917611700786864 73.80847008791761) bank85246 +85247 POINT(39.874984118636 73.19085439780967) bank85247 +85248 POINT(41.44780898407153 74.43999642161943) bank85248 +85249 POINT(40.373964279976484 73.89293473009812) bank85249 +85250 POINT(40.79620359628946 73.231792846425) bank85250 +85251 POINT(39.977581445280144 73.31997727741795) bank85251 +85252 POINT(41.22226395699504 74.61478511866736) bank85252 +85253 POINT(40.16491372735399 73.10314607549985) bank85253 +85254 POINT(41.42999077710149 74.66957875587988) bank85254 +85255 POINT(40.64656342044703 74.0225284742501) bank85255 +85256 POINT(40.64656545504301 73.89371130660896) bank85256 +85257 POINT(40.03502703554992 74.29102423702456) bank85257 +85258 POINT(40.812564093200145 73.83865075701804) bank85258 +85259 POINT(40.845191580596186 73.6342481227303) bank85259 +85260 POINT(39.76027156973147 74.76282140058986) bank85260 +85261 POINT(41.52398045968935 74.48679147941121) bank85261 +85262 POINT(40.39396476827585 74.383114682959) bank85262 +85263 POINT(40.285564684222116 73.71511144553276) bank85263 +85264 POINT(40.18894212129712 74.55905989116943) bank85264 +85265 POINT(39.87733532343842 74.44194504356732) bank85265 +85266 POINT(40.37468767284649 73.5083235603765) bank85266 +85267 POINT(40.19721920640121 73.46490069954578) bank85267 +85268 POINT(40.308168293680566 74.25817512098709) bank85268 +85269 POINT(40.542121676218 74.65477839409625) bank85269 +85270 POINT(41.59661444138251 74.04863323985032) bank85270 +85271 POINT(40.71396201656418 73.88243626918742) bank85271 +85272 POINT(40.893075057377935 74.28138818075008) bank85272 +85273 POINT(41.15788471072574 73.52675340293554) bank85273 +85274 POINT(41.60183027722411 74.92353550035057) bank85274 +85275 POINT(40.419877841947105 73.57980266410587) bank85275 +85276 POINT(40.1536843684404 74.11712304000258) bank85276 +85277 POINT(40.90478157084701 74.88260357803473) bank85277 +85278 POINT(40.150111701755364 73.6626741976535) bank85278 +85279 POINT(40.44521531466597 73.33983348935101) bank85279 +85280 POINT(39.76471187568963 74.6956170266782) bank85280 +85281 POINT(40.42319040037369 74.7562134014459) bank85281 +85282 POINT(40.843010322206254 74.0514021288107) bank85282 +85283 POINT(40.89106071102179 74.63654641593953) bank85283 +85284 POINT(41.38105748054464 73.48854772868305) bank85284 +85285 POINT(41.60954489979237 74.1298337279884) bank85285 +85286 POINT(41.51391954710958 73.05894284766677) bank85286 +85287 POINT(41.01404621179413 74.8839426953247) bank85287 +85288 POINT(40.99529156166375 74.77362454164759) bank85288 +85289 POINT(41.39341109405302 73.28823850257609) bank85289 +85290 POINT(40.88439377742458 74.30996528001336) bank85290 +85291 POINT(40.289099117383756 74.30325893241992) bank85291 +85292 POINT(40.03085346507249 73.89717908326509) bank85292 +85293 POINT(40.88272486828159 74.18688489963881) bank85293 +85294 POINT(41.01292266935641 73.07285208701637) bank85294 +85295 POINT(40.228243002602184 74.68047872090433) bank85295 +85296 POINT(41.36256373423031 73.02213627307404) bank85296 +85297 POINT(39.96983125020641 74.67618893233417) bank85297 +85298 POINT(40.17072210189485 73.44959623611079) bank85298 +85299 POINT(40.643298513207085 73.97056093308443) bank85299 +85300 POINT(40.25326900069339 73.54619903931817) bank85300 +85301 POINT(40.946352001905474 73.72555153369268) bank85301 +85302 POINT(41.56335101678948 73.58445391136587) bank85302 +85303 POINT(40.021756984217845 73.90295903787701) bank85303 +85304 POINT(41.17818350085318 74.26436035854498) bank85304 +85305 POINT(39.73580281255847 74.97991001796895) bank85305 +85306 POINT(40.0355150626526 73.98982659989765) bank85306 +85307 POINT(40.130433217819395 74.62369333861714) bank85307 +85308 POINT(40.49083020288829 73.6486225326921) bank85308 +85309 POINT(40.25846475841872 73.01311456333255) bank85309 +85310 POINT(39.94706614434439 73.04571349597299) bank85310 +85311 POINT(40.16195188902344 74.56135248902159) bank85311 +85312 POINT(39.772389614457886 73.39599197876572) bank85312 +85313 POINT(40.28508805830147 73.04762379173195) bank85313 +85314 POINT(39.751287214926876 73.41623602731299) bank85314 +85315 POINT(41.17799279787721 73.54100173664077) bank85315 +85316 POINT(41.13881799087844 73.49615875335732) bank85316 +85317 POINT(40.69468768520383 74.14490380680594) bank85317 +85318 POINT(40.98796004425605 74.20664222373257) bank85318 +85319 POINT(41.54741221276468 73.57095138592975) bank85319 +85320 POINT(40.65348118754834 74.39283352853822) bank85320 +85321 POINT(40.635824978163086 73.33361267010258) bank85321 +85322 POINT(41.60439560395151 73.83755802530288) bank85322 +85323 POINT(39.71802571080105 73.4807634495602) bank85323 +85324 POINT(40.96480233198949 73.45404410074968) bank85324 +85325 POINT(39.85178060801746 74.44811648899446) bank85325 +85326 POINT(39.89591011657083 73.83033265987449) bank85326 +85327 POINT(41.37471023545947 73.14732513329913) bank85327 +85328 POINT(40.968295042973395 73.90966385193626) bank85328 +85329 POINT(40.8477755337935 74.7119046674502) bank85329 +85330 POINT(41.67250329782196 74.61401415746637) bank85330 +85331 POINT(41.20451043639253 74.7742320464836) bank85331 +85332 POINT(41.164323914361304 74.63000038352092) bank85332 +85333 POINT(41.57647744717674 74.44371629304774) bank85333 +85334 POINT(40.53246468493005 73.42636362544006) bank85334 +85335 POINT(40.490341315839196 74.9787551400556) bank85335 +85336 POINT(40.23681347866651 73.41258685425547) bank85336 +85337 POINT(40.63974498140194 74.80619205304572) bank85337 +85338 POINT(41.48161679348154 74.81106389916336) bank85338 +85339 POINT(39.77106955262151 73.58447285887118) bank85339 +85340 POINT(39.949765889303656 74.31519348657666) bank85340 +85341 POINT(39.90752856170101 74.59617192892759) bank85341 +85342 POINT(40.084952704026726 73.02704504144694) bank85342 +85343 POINT(41.280112149007785 73.19918466440782) bank85343 +85344 POINT(41.60614586162248 74.1163011243672) bank85344 +85345 POINT(40.31597860480046 74.10952450492063) bank85345 +85346 POINT(39.87942218594148 74.61289702316014) bank85346 +85347 POINT(40.74206996307752 73.92355713750095) bank85347 +85348 POINT(39.792726949320986 73.2720729074719) bank85348 +85349 POINT(40.35298565602575 74.02703328817194) bank85349 +85350 POINT(40.74706234000372 73.08568426998913) bank85350 +85351 POINT(40.22780898705168 74.0177694491425) bank85351 +85352 POINT(41.17981015669706 73.10676142050069) bank85352 +85353 POINT(41.566705212969005 73.1586166867114) bank85353 +85354 POINT(40.29607605701237 73.94607065881284) bank85354 +85355 POINT(40.950236766999566 73.2430171076568) bank85355 +85356 POINT(39.88354774440979 73.58565170864907) bank85356 +85357 POINT(40.414596815175244 74.78548198568794) bank85357 +85358 POINT(39.77263613738273 74.43398926433876) bank85358 +85359 POINT(40.26132586720291 73.95711244864694) bank85359 +85360 POINT(41.21578005614797 73.60889473775903) bank85360 +85361 POINT(40.64359113728973 73.80778043220701) bank85361 +85362 POINT(41.31100133090769 74.3663615363297) bank85362 +85363 POINT(40.25365476474821 73.7067628305799) bank85363 +85364 POINT(41.0607443101858 74.31001117235853) bank85364 +85365 POINT(41.65740622879256 73.18126693517836) bank85365 +85366 POINT(41.236525184183904 74.81611452394029) bank85366 +85367 POINT(39.91179694929139 73.38725339222124) bank85367 +85368 POINT(39.82499440350088 74.11469270867693) bank85368 +85369 POINT(41.37898026363844 73.12421471287286) bank85369 +85370 POINT(40.35851987794127 74.29567510695723) bank85370 +85371 POINT(40.67181024177846 74.95568055776964) bank85371 +85372 POINT(41.61789722943512 74.98894904007868) bank85372 +85373 POINT(40.46965505916356 75.0054813745023) bank85373 +85374 POINT(41.37538097847054 74.1252466902788) bank85374 +85375 POINT(41.34458151152092 74.96420133643684) bank85375 +85376 POINT(40.91819312036236 73.24134554972859) bank85376 +85377 POINT(39.83036130779348 75.00044631037991) bank85377 +85378 POINT(39.907123423710985 74.87423321442262) bank85378 +85379 POINT(39.770692225299406 74.02679055394057) bank85379 +85380 POINT(40.95916972072597 73.042992987657) bank85380 +85381 POINT(40.85375669098721 73.78769432375196) bank85381 +85382 POINT(40.4983103943329 73.82609338704651) bank85382 +85383 POINT(40.503059770947104 73.59789553009975) bank85383 +85384 POINT(41.42582551711379 73.38285019255085) bank85384 +85385 POINT(40.94434868103297 74.21840447893831) bank85385 +85386 POINT(40.535214413642755 74.14637711920838) bank85386 +85387 POINT(40.6591847700774 74.46071483051863) bank85387 +85388 POINT(40.18877457184796 73.4064047745092) bank85388 +85389 POINT(41.38979384800984 74.42899487261252) bank85389 +85390 POINT(40.159094954365564 73.07454349729873) bank85390 +85391 POINT(41.352215676008754 74.56692845536202) bank85391 +85392 POINT(41.503533526112264 73.03818099176826) bank85392 +85393 POINT(40.718824600594075 74.39834691024204) bank85393 +85394 POINT(41.590609966500146 74.06802821907078) bank85394 +85395 POINT(40.89499683780588 73.6900332198458) bank85395 +85396 POINT(39.83961323457243 73.75098075394314) bank85396 +85397 POINT(40.41337663622792 74.83237675760961) bank85397 +85398 POINT(41.07297641793908 74.44093168373665) bank85398 +85399 POINT(40.272361730907456 73.0946372869612) bank85399 +85400 POINT(40.38388723478247 73.94260591056741) bank85400 +85401 POINT(41.38047965382743 73.99325711751426) bank85401 +85402 POINT(41.587376624765845 73.02258286041392) bank85402 +85403 POINT(40.982624792301834 73.02833526443341) bank85403 +85404 POINT(41.61688930071476 73.88728719102033) bank85404 +85405 POINT(41.38963217176529 73.60410678770846) bank85405 +85406 POINT(41.23543899716506 74.13227184953858) bank85406 +85407 POINT(40.54319893665117 73.82174913562133) bank85407 +85408 POINT(41.348695786326516 73.78732552936785) bank85408 +85409 POINT(40.26252065829681 74.44908225318542) bank85409 +85410 POINT(39.85107974286198 73.61155579681405) bank85410 +85411 POINT(40.715788517248235 74.23552792272446) bank85411 +85412 POINT(40.83048959361913 74.6872799282862) bank85412 +85413 POINT(40.29987084482732 74.37984130654667) bank85413 +85414 POINT(40.21901613786435 74.6019466135848) bank85414 +85415 POINT(41.46644138099536 74.460420514153) bank85415 +85416 POINT(40.23085373788987 73.37422869040536) bank85416 +85417 POINT(40.99761364048411 74.3323097034127) bank85417 +85418 POINT(40.60798069224219 74.88299732827124) bank85418 +85419 POINT(40.99193077603531 73.39164229629843) bank85419 +85420 POINT(40.993222347871416 73.23237135350725) bank85420 +85421 POINT(40.17268143754724 73.61388733877484) bank85421 +85422 POINT(40.515268359840825 73.59003678749356) bank85422 +85423 POINT(40.61602999301489 73.93166044639848) bank85423 +85424 POINT(41.34043213481016 74.89288992832593) bank85424 +85425 POINT(41.30717658553011 73.90086767883248) bank85425 +85426 POINT(41.36599120592226 73.9648135804126) bank85426 +85427 POINT(40.22522398578908 74.65535271011744) bank85427 +85428 POINT(40.116202819430654 73.93741063542087) bank85428 +85429 POINT(41.46723888913728 73.73299374613755) bank85429 +85430 POINT(41.63726340519227 73.91513580286058) bank85430 +85431 POINT(41.20873866492882 73.12874776046031) bank85431 +85432 POINT(41.20073215072629 74.27306137581847) bank85432 +85433 POINT(40.61389544123929 73.68616995336136) bank85433 +85434 POINT(40.77479285604671 74.25366456203473) bank85434 +85435 POINT(41.47099440765736 73.6432279633226) bank85435 +85436 POINT(41.47312345736095 73.45385201028189) bank85436 +85437 POINT(41.2029961959987 73.64972867665423) bank85437 +85438 POINT(41.67688228504351 73.61520986267043) bank85438 +85439 POINT(39.807267840068036 73.77815174652339) bank85439 +85440 POINT(41.06971507675006 74.47138843942099) bank85440 +85441 POINT(41.378531077468786 74.21167123347318) bank85441 +85442 POINT(39.72973006670142 73.24764957514877) bank85442 +85443 POINT(40.13909875705794 73.57954380658812) bank85443 +85444 POINT(41.13399012296157 74.27185904505839) bank85444 +85445 POINT(40.16579574512002 74.61236320116474) bank85445 +85446 POINT(41.54956847683423 73.96338764270358) bank85446 +85447 POINT(40.29900619686258 73.62854302349567) bank85447 +85448 POINT(41.057350082289965 74.81917074578368) bank85448 +85449 POINT(40.681573303412655 73.5863933731992) bank85449 +85450 POINT(39.843320432614206 74.27075407363105) bank85450 +85451 POINT(40.91701159809345 74.54587113162269) bank85451 +85452 POINT(40.48876093292161 73.45330714680114) bank85452 +85453 POINT(41.17207925895979 74.32419247082387) bank85453 +85454 POINT(39.78722854206585 74.87215958450699) bank85454 +85455 POINT(40.33284018171339 73.8362403628854) bank85455 +85456 POINT(40.356184962804925 74.51609388537749) bank85456 +85457 POINT(41.37658058752831 73.80917759046392) bank85457 +85458 POINT(41.27656907228047 73.25345236680471) bank85458 +85459 POINT(41.54567225678324 74.28719065025504) bank85459 +85460 POINT(40.28319357412547 73.85140577840458) bank85460 +85461 POINT(40.054001863659614 74.17311370904335) bank85461 +85462 POINT(40.300814874366026 73.05983965690899) bank85462 +85463 POINT(40.04152540386637 74.27437793849553) bank85463 +85464 POINT(41.61330545387294 73.72269530672874) bank85464 +85465 POINT(41.58319673806671 73.68394067330063) bank85465 +85466 POINT(40.92756191845677 73.13969108881473) bank85466 +85467 POINT(40.74212839808767 73.94173454303791) bank85467 +85468 POINT(41.19059423824345 73.67945816971356) bank85468 +85469 POINT(41.098898012242024 74.28038425559738) bank85469 +85470 POINT(41.53396010351416 74.21692767029157) bank85470 +85471 POINT(40.3368797305587 74.90506529471716) bank85471 +85472 POINT(40.2417177764377 73.19021896434646) bank85472 +85473 POINT(41.236412912658864 74.14821135961533) bank85473 +85474 POINT(39.761656676153514 73.82063143011871) bank85474 +85475 POINT(40.56075796341613 73.28121316028925) bank85475 +85476 POINT(41.67184049499026 74.63373919791613) bank85476 +85477 POINT(39.74152094040867 74.07692491590076) bank85477 +85478 POINT(40.797466269748455 74.88659469134643) bank85478 +85479 POINT(40.388053560271295 73.01400464493791) bank85479 +85480 POINT(40.624659567783496 73.67909979716455) bank85480 +85481 POINT(40.690762084778775 74.4124004194199) bank85481 +85482 POINT(39.861025169079234 74.26316778458454) bank85482 +85483 POINT(41.52960072613505 73.32644978292231) bank85483 +85484 POINT(39.75146192135665 73.47844177859444) bank85484 +85485 POINT(40.24256346973553 73.41493957071323) bank85485 +85486 POINT(40.72087915487965 74.15408282901011) bank85486 +85487 POINT(41.13563912763278 74.0829126590841) bank85487 +85488 POINT(40.603987422942446 74.12669652052017) bank85488 +85489 POINT(41.117181830316305 74.7965236193462) bank85489 +85490 POINT(41.39870704142134 74.99271029692042) bank85490 +85491 POINT(40.179292377793885 74.89957899106895) bank85491 +85492 POINT(40.14380925387986 74.7149120911819) bank85492 +85493 POINT(41.63218756075418 73.89807430654776) bank85493 +85494 POINT(41.114798014612965 73.35326858895574) bank85494 +85495 POINT(41.02978202858706 73.66915960925222) bank85495 +85496 POINT(41.46450046354051 74.25343436409572) bank85496 +85497 POINT(40.429694593316455 73.23162263641015) bank85497 +85498 POINT(41.54687973333562 74.96522135975488) bank85498 +85499 POINT(39.97722692127113 73.42791948431908) bank85499 +85500 POINT(40.803773078836564 73.06593362147397) bank85500 +85501 POINT(40.15196089754002 74.16914937815801) bank85501 +85502 POINT(41.49879525658568 73.56234723581615) bank85502 +85503 POINT(41.20008478538989 73.57134724511111) bank85503 +85504 POINT(39.82851180587241 73.43807689656609) bank85504 +85505 POINT(41.26277761544527 73.73583854805588) bank85505 +85506 POINT(40.14929756753669 74.40873189786426) bank85506 +85507 POINT(41.6175203588029 73.83415480613573) bank85507 +85508 POINT(41.33513073823177 73.454457558365) bank85508 +85509 POINT(40.56039088950493 74.56016975731168) bank85509 +85510 POINT(41.522025193241106 73.64490168453345) bank85510 +85511 POINT(41.13145067480946 74.57328309039929) bank85511 +85512 POINT(40.5366642586138 74.39004155587534) bank85512 +85513 POINT(41.614867725714014 74.63520134324207) bank85513 +85514 POINT(40.4674065784 73.89026536932893) bank85514 +85515 POINT(41.07394820817352 74.10340573861875) bank85515 +85516 POINT(41.33115768927607 74.4045047896607) bank85516 +85517 POINT(41.00566155714565 73.02798731924136) bank85517 +85518 POINT(40.917648146109535 73.73049157950494) bank85518 +85519 POINT(40.58024353848539 73.90223945214325) bank85519 +85520 POINT(40.522605496122495 74.05842157624706) bank85520 +85521 POINT(40.81680581053233 73.1299826221599) bank85521 +85522 POINT(40.47090226209154 73.11151120963271) bank85522 +85523 POINT(39.85014520128306 73.5720802482124) bank85523 +85524 POINT(40.07420058720382 73.74873144264953) bank85524 +85525 POINT(41.329245593774104 73.80522220940665) bank85525 +85526 POINT(41.67819003861368 74.37130846504745) bank85526 +85527 POINT(39.72397130603523 74.5737898487844) bank85527 +85528 POINT(40.24548508708463 73.64278291994056) bank85528 +85529 POINT(41.57604773568819 73.24999601594739) bank85529 +85530 POINT(39.758892250864676 73.946094797594) bank85530 +85531 POINT(41.4999619569909 73.94108629787675) bank85531 +85532 POINT(40.572890792182996 73.91511711757195) bank85532 +85533 POINT(40.44333324563565 73.56445171253655) bank85533 +85534 POINT(41.239759727030915 73.4698300504357) bank85534 +85535 POINT(41.57261971444552 74.85319915228648) bank85535 +85536 POINT(39.90081274681171 73.35079261969648) bank85536 +85537 POINT(40.308085198451046 74.62500529698409) bank85537 +85538 POINT(40.02895825043469 73.93017787524242) bank85538 +85539 POINT(41.155903383499215 74.0895643658321) bank85539 +85540 POINT(40.47471269858104 73.80131952636823) bank85540 +85541 POINT(40.92243243918267 74.40108444730453) bank85541 +85542 POINT(40.885311001678026 73.72366999175676) bank85542 +85543 POINT(40.968743244004656 74.81729687168104) bank85543 +85544 POINT(41.080964860711525 74.41418425884183) bank85544 +85545 POINT(41.60395247687485 74.61397454044533) bank85545 +85546 POINT(40.548298983385145 73.38561882614546) bank85546 +85547 POINT(41.335671490026336 74.43949890341814) bank85547 +85548 POINT(41.390039974809106 74.45373596102249) bank85548 +85549 POINT(40.56996833465706 73.13395091255566) bank85549 +85550 POINT(41.28813000593332 73.2426989938665) bank85550 +85551 POINT(41.10757311746492 73.93791333987572) bank85551 +85552 POINT(41.11602910468684 73.56538446494231) bank85552 +85553 POINT(41.541568184132394 73.48814010866954) bank85553 +85554 POINT(39.735107683361534 74.73506950837104) bank85554 +85555 POINT(40.528993856372495 74.64842055700869) bank85555 +85556 POINT(41.54461545044339 73.94493558599193) bank85556 +85557 POINT(40.740684712390824 73.61137340625228) bank85557 +85558 POINT(40.511570774379166 73.81005018685897) bank85558 +85559 POINT(41.593699885347434 73.42797912236449) bank85559 +85560 POINT(41.23626860631101 74.27650615428689) bank85560 +85561 POINT(40.712803121005386 74.0288436633611) bank85561 +85562 POINT(39.836737082057226 73.63503949431892) bank85562 +85563 POINT(39.73068698163094 74.43841943421738) bank85563 +85564 POINT(39.854833102526094 73.35656583392155) bank85564 +85565 POINT(40.46530442176547 74.62353464913694) bank85565 +85566 POINT(40.01737396604586 73.84736601973) bank85566 +85567 POINT(41.41655703789433 73.35842108800176) bank85567 +85568 POINT(41.396362635311135 73.41083209087928) bank85568 +85569 POINT(41.50584736500154 74.80269305130685) bank85569 +85570 POINT(40.3676229534076 74.04494890240724) bank85570 +85571 POINT(41.32967293064936 73.31226059666535) bank85571 +85572 POINT(40.319825137101205 74.32444755693037) bank85572 +85573 POINT(41.107443914842534 74.03411027858009) bank85573 +85574 POINT(39.76356105355759 73.18734647925307) bank85574 +85575 POINT(41.38125429789884 73.07311620138442) bank85575 +85576 POINT(40.091716031233446 74.9646291944426) bank85576 +85577 POINT(41.35261503799418 74.37587786654666) bank85577 +85578 POINT(40.31116441392102 73.16261053415803) bank85578 +85579 POINT(41.23229849753306 73.17511241283266) bank85579 +85580 POINT(41.0844185769872 74.8732282679589) bank85580 +85581 POINT(40.11047196095766 73.2022841077237) bank85581 +85582 POINT(40.580375011898155 73.57919951708635) bank85582 +85583 POINT(40.18133279843641 74.79648382184772) bank85583 +85584 POINT(40.66020276827225 74.04914599782393) bank85584 +85585 POINT(40.19835435511959 73.68003185049048) bank85585 +85586 POINT(41.414533681113454 73.11809100475338) bank85586 +85587 POINT(40.64210715264956 74.5942702601818) bank85587 +85588 POINT(41.135253848737584 73.35302454785548) bank85588 +85589 POINT(39.87234072295243 73.5464223672939) bank85589 +85590 POINT(41.498602957114876 74.38991749577625) bank85590 +85591 POINT(41.49277191474717 73.23058380536276) bank85591 +85592 POINT(41.524195451715805 73.47752517358435) bank85592 +85593 POINT(39.85519324601059 73.29664571452746) bank85593 +85594 POINT(40.9159167148717 73.89340047075949) bank85594 +85595 POINT(40.62821870860523 73.81436865288636) bank85595 +85596 POINT(40.74453899329715 75.0019881377974) bank85596 +85597 POINT(40.843588235567964 74.93005410746623) bank85597 +85598 POINT(40.74646009131291 73.23496855690429) bank85598 +85599 POINT(40.29657829959855 73.97026269666286) bank85599 +85600 POINT(40.30255934668411 74.96245424432671) bank85600 +85601 POINT(39.82536908658627 74.542670905499) bank85601 +85602 POINT(41.25861072005121 73.64776750956277) bank85602 +85603 POINT(39.86001370337911 74.73273793460186) bank85603 +85604 POINT(41.13221403560708 73.48323726231527) bank85604 +85605 POINT(40.44376988017057 74.46546951466472) bank85605 +85606 POINT(40.20204104047408 73.12763124250496) bank85606 +85607 POINT(41.03688268125732 73.050563580473) bank85607 +85608 POINT(39.95361915355958 74.56336005866729) bank85608 +85609 POINT(41.65789106658032 73.04649169956554) bank85609 +85610 POINT(40.55612176559113 74.55726596140461) bank85610 +85611 POINT(41.10581745448838 73.47990723006998) bank85611 +85612 POINT(41.15940346241105 73.41977723303653) bank85612 +85613 POINT(40.13377238347415 73.70989701576119) bank85613 +85614 POINT(40.44226980796769 74.02709334484433) bank85614 +85615 POINT(40.08401698579931 73.48607704958967) bank85615 +85616 POINT(39.95181520314174 74.75833151941947) bank85616 +85617 POINT(41.41137585909537 74.52479017951929) bank85617 +85618 POINT(41.686897999279694 73.76169473451174) bank85618 +85619 POINT(40.291920873185994 74.12685022008965) bank85619 +85620 POINT(40.35268660348568 73.91814106095947) bank85620 +85621 POINT(39.98778270721564 73.61835662103769) bank85621 +85622 POINT(41.16238368110635 73.82246564476225) bank85622 +85623 POINT(39.980416346360585 74.70720548609663) bank85623 +85624 POINT(41.02010887849898 74.04573895979942) bank85624 +85625 POINT(40.42429400930395 73.26968648585806) bank85625 +85626 POINT(40.02212055554484 73.80011394822576) bank85626 +85627 POINT(41.145450299447205 74.36328864693556) bank85627 +85628 POINT(41.523142175898094 74.30961345563921) bank85628 +85629 POINT(40.183245644871775 73.64289154778469) bank85629 +85630 POINT(40.90089579668856 74.92205422168212) bank85630 +85631 POINT(40.91527220981564 73.38653278123519) bank85631 +85632 POINT(41.18690969531312 74.06919088809481) bank85632 +85633 POINT(40.250203372304206 73.79182138279795) bank85633 +85634 POINT(40.25711899601441 74.69434966595722) bank85634 +85635 POINT(41.56633982907815 73.20637992624854) bank85635 +85636 POINT(41.27435037422262 74.64552461669885) bank85636 +85637 POINT(41.53516123040528 74.84327875150174) bank85637 +85638 POINT(41.68292525122434 73.31981534297441) bank85638 +85639 POINT(41.53871121283781 74.73691588680478) bank85639 +85640 POINT(40.64368705151412 74.13300665327957) bank85640 +85641 POINT(40.95245051956499 73.66405617985937) bank85641 +85642 POINT(40.56924058388141 73.27835630811725) bank85642 +85643 POINT(41.04710354991176 74.17937967121448) bank85643 +85644 POINT(40.50899629237412 73.27025800580735) bank85644 +85645 POINT(40.01290183167025 73.75950488577377) bank85645 +85646 POINT(41.36854869399299 74.65274970644808) bank85646 +85647 POINT(40.207719969492956 74.16308347951872) bank85647 +85648 POINT(41.16612157005317 74.49770143341735) bank85648 +85649 POINT(39.72415610031546 73.98877256263734) bank85649 +85650 POINT(41.43439860757872 74.21485871844833) bank85650 +85651 POINT(39.8488514769312 74.58987874549834) bank85651 +85652 POINT(40.11201710967847 73.33754486350742) bank85652 +85653 POINT(39.90505760492192 74.31638741133155) bank85653 +85654 POINT(40.52055361039991 74.46255275173914) bank85654 +85655 POINT(41.40175715179143 73.69064519370512) bank85655 +85656 POINT(40.32985095085266 74.52587663096932) bank85656 +85657 POINT(41.40469062019029 74.01456585502252) bank85657 +85658 POINT(41.190493775666646 74.15714693363161) bank85658 +85659 POINT(40.04823724954304 74.01498929522677) bank85659 +85660 POINT(40.33268457204563 74.8028797362692) bank85660 +85661 POINT(41.108848228020484 74.87764284282711) bank85661 +85662 POINT(40.10209226961173 73.81756441807558) bank85662 +85663 POINT(41.25228678383424 73.082687047873) bank85663 +85664 POINT(41.47449302561655 74.7447628185184) bank85664 +85665 POINT(40.192949856426935 73.82459121339329) bank85665 +85666 POINT(40.81020783327208 74.51033432986878) bank85666 +85667 POINT(40.219651098919485 74.92275615391783) bank85667 +85668 POINT(40.38784299735899 74.5072629772477) bank85668 +85669 POINT(40.10641535823417 74.42237446723342) bank85669 +85670 POINT(40.29149975408876 73.14727570214697) bank85670 +85671 POINT(39.84422124590227 74.03134030188566) bank85671 +85672 POINT(39.80728736081051 74.32864813447739) bank85672 +85673 POINT(41.04523519670595 73.47750579516504) bank85673 +85674 POINT(39.80815554756204 73.46365741435204) bank85674 +85675 POINT(40.49644632323791 73.99157142664805) bank85675 +85676 POINT(39.72119803572938 74.01256202209346) bank85676 +85677 POINT(40.997753656127145 73.75567599539694) bank85677 +85678 POINT(40.747537810083585 73.44133295162659) bank85678 +85679 POINT(41.42232213764897 73.27029884153995) bank85679 +85680 POINT(40.48831521398448 73.47657510267693) bank85680 +85681 POINT(40.38393030616482 73.4494565046802) bank85681 +85682 POINT(41.47204502405691 74.32951370831121) bank85682 +85683 POINT(40.56430570318566 74.30192432436672) bank85683 +85684 POINT(39.84087163579747 73.25505212630772) bank85684 +85685 POINT(41.385801592547985 73.67730878682595) bank85685 +85686 POINT(41.19621812288159 74.16130278506374) bank85686 +85687 POINT(40.369521541310526 73.04491123357623) bank85687 +85688 POINT(40.94632849821873 74.86331966954715) bank85688 +85689 POINT(40.52417255987898 73.90858516507515) bank85689 +85690 POINT(40.732203793979174 74.39499780999688) bank85690 +85691 POINT(40.091149324981494 73.36321342361103) bank85691 +85692 POINT(40.79250408512881 73.75232064388418) bank85692 +85693 POINT(39.73581951726998 73.36931698590752) bank85693 +85694 POINT(40.411783089175586 73.12443741648565) bank85694 +85695 POINT(41.69482339442859 74.96227878756305) bank85695 +85696 POINT(41.00831307266453 74.82709434066726) bank85696 +85697 POINT(40.08078012175701 74.21004746570905) bank85697 +85698 POINT(41.018507879836 74.07012103930299) bank85698 +85699 POINT(39.806699091558315 74.67178025149673) bank85699 +85700 POINT(41.621568910100464 73.29397636645146) bank85700 +85701 POINT(39.87727240044119 74.60931399445431) bank85701 +85702 POINT(40.42591399161889 73.87212036503695) bank85702 +85703 POINT(40.532073192719785 73.02693274388908) bank85703 +85704 POINT(40.07879473173477 74.99473251610631) bank85704 +85705 POINT(40.69764718166574 73.60749162288904) bank85705 +85706 POINT(41.12502208389617 74.0631632177968) bank85706 +85707 POINT(41.3352614211215 74.89782686417037) bank85707 +85708 POINT(40.94047778237589 73.54451037775922) bank85708 +85709 POINT(40.69514973493009 73.03067859452881) bank85709 +85710 POINT(40.227991704226135 74.96131009816072) bank85710 +85711 POINT(41.18322048164871 74.54418189265357) bank85711 +85712 POINT(41.105457723661196 73.30888302280222) bank85712 +85713 POINT(41.42653808748436 74.18658215447543) bank85713 +85714 POINT(41.25082788909461 74.92340202475717) bank85714 +85715 POINT(39.95233150676274 74.17402630209237) bank85715 +85716 POINT(39.74787782278288 73.83355221761471) bank85716 +85717 POINT(40.334006972709005 73.22745974418409) bank85717 +85718 POINT(40.40808168995621 73.02627137108675) bank85718 +85719 POINT(41.08664655335639 73.06362305346164) bank85719 +85720 POINT(40.49570780895226 74.75923864926396) bank85720 +85721 POINT(39.990196102767655 73.79832476382968) bank85721 +85722 POINT(39.96361929455 74.34373215044452) bank85722 +85723 POINT(41.017777115016855 73.33679267602525) bank85723 +85724 POINT(40.64272461338943 73.78345952588577) bank85724 +85725 POINT(41.124087792719955 73.18638216637271) bank85725 +85726 POINT(41.4110874236539 74.31516814006652) bank85726 +85727 POINT(40.478389669493055 73.7508025794592) bank85727 +85728 POINT(40.12795716367677 74.84059756353444) bank85728 +85729 POINT(39.96215363231503 74.7704576086152) bank85729 +85730 POINT(41.43489104085276 74.10358345064316) bank85730 +85731 POINT(40.33935620050995 74.2923420334648) bank85731 +85732 POINT(41.542270639665496 73.28246585816142) bank85732 +85733 POINT(39.98066600877235 74.1300755205271) bank85733 +85734 POINT(41.20487535247978 73.29431914590211) bank85734 +85735 POINT(40.067046083912494 73.46114842296156) bank85735 +85736 POINT(41.3683217960454 73.15583775686129) bank85736 +85737 POINT(40.73937370864591 74.16688382111691) bank85737 +85738 POINT(41.3714498845605 74.26486537256474) bank85738 +85739 POINT(39.72355140192895 73.02261558973778) bank85739 +85740 POINT(41.1197514602103 74.48962032196819) bank85740 +85741 POINT(40.308060955199394 74.32012913885738) bank85741 +85742 POINT(41.39081292945522 73.49469872915573) bank85742 +85743 POINT(41.545803800145876 74.83249059744293) bank85743 +85744 POINT(40.759401330331265 73.94924255162044) bank85744 +85745 POINT(41.407451863863805 74.01461059630847) bank85745 +85746 POINT(40.03360899490607 73.63038491186387) bank85746 +85747 POINT(40.32805534249922 74.69656802992591) bank85747 +85748 POINT(40.32649903578393 74.04316811269848) bank85748 +85749 POINT(41.41894671505295 74.45154746000456) bank85749 +85750 POINT(41.37677198834876 74.88750392449707) bank85750 +85751 POINT(40.57589677961448 73.96876802309788) bank85751 +85752 POINT(41.68891341421968 73.09051679785316) bank85752 +85753 POINT(40.53844560960755 74.3194344614547) bank85753 +85754 POINT(41.59116745476611 74.56379212994335) bank85754 +85755 POINT(39.928829444472996 74.41686949896935) bank85755 +85756 POINT(40.23258058267794 74.32986226729687) bank85756 +85757 POINT(41.27655323837127 73.48491482630448) bank85757 +85758 POINT(40.05127087351413 73.13415453056358) bank85758 +85759 POINT(40.61969196967397 73.66105006923755) bank85759 +85760 POINT(40.39698828056957 74.88283352528137) bank85760 +85761 POINT(40.160504557408146 73.87464232815518) bank85761 +85762 POINT(40.57851988945207 74.09814136958492) bank85762 +85763 POINT(40.847500272835454 73.3501110288044) bank85763 +85764 POINT(39.991900959544324 74.2066355228912) bank85764 +85765 POINT(39.72523208116674 74.35094790303827) bank85765 +85766 POINT(40.021006427620904 73.81565956126708) bank85766 +85767 POINT(40.10163022783647 74.12498971911981) bank85767 +85768 POINT(40.79763545457364 73.70175391673337) bank85768 +85769 POINT(40.51120928401593 74.12679720335538) bank85769 +85770 POINT(41.12307630144284 73.80020562578075) bank85770 +85771 POINT(41.204550216505204 74.2995015928805) bank85771 +85772 POINT(40.19381451784581 73.51598328218351) bank85772 +85773 POINT(40.19307192949855 74.05986704030904) bank85773 +85774 POINT(40.7074542462729 73.64370801734127) bank85774 +85775 POINT(40.12659696845947 73.18478492496669) bank85775 +85776 POINT(41.65314013463387 73.51161031105347) bank85776 +85777 POINT(40.22389666931369 74.52247038472085) bank85777 +85778 POINT(40.149303424472876 74.40575549601638) bank85778 +85779 POINT(41.13121510950806 73.28893292594586) bank85779 +85780 POINT(40.39089643089412 73.03138305125181) bank85780 +85781 POINT(40.57958913433658 74.4689527865783) bank85781 +85782 POINT(41.26000550959506 74.33963050414873) bank85782 +85783 POINT(40.778684624840075 74.82130069050204) bank85783 +85784 POINT(40.79812401220611 74.4013770989496) bank85784 +85785 POINT(39.890347103740055 74.88820095856703) bank85785 +85786 POINT(40.78666761538556 73.18980358793966) bank85786 +85787 POINT(40.34943584673144 74.22404829173637) bank85787 +85788 POINT(39.82873551655387 74.41264587606406) bank85788 +85789 POINT(39.83225805564647 74.37461605882069) bank85789 +85790 POINT(40.78672989926871 73.99078086712956) bank85790 +85791 POINT(40.90584832376991 74.92338646905642) bank85791 +85792 POINT(41.17709541992342 73.6911807977155) bank85792 +85793 POINT(41.29355962541481 73.69897833594777) bank85793 +85794 POINT(40.920085743259015 74.51551727243097) bank85794 +85795 POINT(41.32478698773717 74.5226500096487) bank85795 +85796 POINT(41.67467145492168 73.65676321203404) bank85796 +85797 POINT(40.87580439596715 74.94058356270315) bank85797 +85798 POINT(41.21854643233877 73.40993648653554) bank85798 +85799 POINT(41.31405242714966 73.36523218504458) bank85799 +85800 POINT(40.547704513541184 74.61746934424697) bank85800 +85801 POINT(41.29161168521297 74.43762941357204) bank85801 +85802 POINT(41.093446963720936 73.41636405661761) bank85802 +85803 POINT(39.93668748522017 74.14188835232426) bank85803 +85804 POINT(39.79262483924321 74.885871736241) bank85804 +85805 POINT(40.4763842769833 73.18435007084923) bank85805 +85806 POINT(39.846069589897986 73.57881255884871) bank85806 +85807 POINT(40.317395243216914 73.04373964070285) bank85807 +85808 POINT(41.56755876459265 73.77691279436307) bank85808 +85809 POINT(40.356962011175625 73.326270796872) bank85809 +85810 POINT(40.09141642848693 73.64462072765461) bank85810 +85811 POINT(40.7144696198295 74.5693468076354) bank85811 +85812 POINT(41.289557245502515 73.35453972011102) bank85812 +85813 POINT(39.9917286422736 74.39568275051099) bank85813 +85814 POINT(41.40357652047797 74.80638765873668) bank85814 +85815 POINT(41.32376147614482 74.80245037339459) bank85815 +85816 POINT(41.626662416858075 73.14388819258158) bank85816 +85817 POINT(40.48938604333593 74.81949410986547) bank85817 +85818 POINT(41.42764077511128 73.70480779222513) bank85818 +85819 POINT(40.851005226737605 73.90282544374031) bank85819 +85820 POINT(41.61320009283866 73.22862103701837) bank85820 +85821 POINT(40.10097820926956 73.36138704194727) bank85821 +85822 POINT(41.2020089795491 73.15705425321026) bank85822 +85823 POINT(40.0117535328874 73.13068964306316) bank85823 +85824 POINT(41.64843880271326 73.7211009129022) bank85824 +85825 POINT(40.962513218207455 73.49657182392644) bank85825 +85826 POINT(39.8127033307921 74.73205478617751) bank85826 +85827 POINT(40.10879932545344 73.63587237447976) bank85827 +85828 POINT(39.84157864304168 73.76319151895424) bank85828 +85829 POINT(40.31207425631805 74.46237521242696) bank85829 +85830 POINT(40.426176735852145 74.42338301555996) bank85830 +85831 POINT(41.70899491251413 73.8400393459968) bank85831 +85832 POINT(39.74356718298372 73.86598706246318) bank85832 +85833 POINT(41.1210987473314 73.86684034527595) bank85833 +85834 POINT(41.16280519121027 74.58666496698079) bank85834 +85835 POINT(40.80804695821046 73.45686746764719) bank85835 +85836 POINT(41.17930624219098 74.46554410871966) bank85836 +85837 POINT(40.297159647962076 73.92558594838131) bank85837 +85838 POINT(40.12782629466024 74.74978249030879) bank85838 +85839 POINT(40.3019038028131 74.78700291586074) bank85839 +85840 POINT(40.01453997613087 73.91353892924748) bank85840 +85841 POINT(41.37538514746336 74.96682208418709) bank85841 +85842 POINT(39.86002323616197 74.62499647388938) bank85842 +85843 POINT(39.77264129354842 74.8485179889902) bank85843 +85844 POINT(39.80362411879626 74.10404874875701) bank85844 +85845 POINT(40.62396071009752 73.91810757643324) bank85845 +85846 POINT(40.16817072920375 73.73288336037376) bank85846 +85847 POINT(39.77790351077689 73.72163273698713) bank85847 +85848 POINT(41.19822218033987 73.08184986262583) bank85848 +85849 POINT(40.57945863956948 73.7626439241286) bank85849 +85850 POINT(41.264094008429105 73.31237382628404) bank85850 +85851 POINT(40.999844281333885 73.36835355709415) bank85851 +85852 POINT(41.27536153982637 73.86712299813998) bank85852 +85853 POINT(41.015460255937505 74.67768678857361) bank85853 +85854 POINT(39.85905579201084 74.84879174252501) bank85854 +85855 POINT(40.38522290610901 73.94316702916619) bank85855 +85856 POINT(40.90087390797633 74.13870149983082) bank85856 +85857 POINT(40.377788316137746 73.61111177718368) bank85857 +85858 POINT(39.91703807364923 73.65399346382111) bank85858 +85859 POINT(39.836531238082166 73.93320713863731) bank85859 +85860 POINT(41.16587781270763 73.58175333867328) bank85860 +85861 POINT(41.709370862598156 73.84596502131026) bank85861 +85862 POINT(40.239611204032215 74.90584067604675) bank85862 +85863 POINT(40.937329146232024 74.59455046452166) bank85863 +85864 POINT(40.3157848601392 73.37232333436408) bank85864 +85865 POINT(41.401519501220115 74.46074221311366) bank85865 +85866 POINT(40.738512558690395 74.05407091018593) bank85866 +85867 POINT(39.910035656855484 74.21832532166407) bank85867 +85868 POINT(41.1343692138283 73.1620729358747) bank85868 +85869 POINT(39.81828642430832 74.08230924343906) bank85869 +85870 POINT(41.27656765822522 74.3405039898591) bank85870 +85871 POINT(41.43347128191055 73.71490237240155) bank85871 +85872 POINT(40.358978254936076 73.42183395183729) bank85872 +85873 POINT(40.927062841608695 73.62445930221966) bank85873 +85874 POINT(40.13731415567432 74.64350072016967) bank85874 +85875 POINT(41.579512095519064 73.53121329352821) bank85875 +85876 POINT(40.715937676046984 73.63801129179627) bank85876 +85877 POINT(41.02431230936431 73.17345965410966) bank85877 +85878 POINT(39.8249103191611 74.62939020383295) bank85878 +85879 POINT(41.5864132958218 74.48563731232521) bank85879 +85880 POINT(41.58492792296091 74.33261904434072) bank85880 +85881 POINT(41.70403606714805 74.30922058026549) bank85881 +85882 POINT(40.76720825628467 73.40204009282796) bank85882 +85883 POINT(40.06863809829813 74.35814359317467) bank85883 +85884 POINT(39.92880084110013 74.34409039559253) bank85884 +85885 POINT(41.25826251393712 73.22640393679295) bank85885 +85886 POINT(39.80905726680534 74.39665808248272) bank85886 +85887 POINT(41.576778296507115 74.70748159081842) bank85887 +85888 POINT(40.914300061094245 74.50140526288052) bank85888 +85889 POINT(39.79805676801618 74.21680290581673) bank85889 +85890 POINT(39.920964594145154 74.93993033412234) bank85890 +85891 POINT(39.917000920558955 74.99326532971969) bank85891 +85892 POINT(40.14498587287944 74.56052185354264) bank85892 +85893 POINT(39.79320794536245 74.85561736741886) bank85893 +85894 POINT(40.30000213016652 73.72384303235155) bank85894 +85895 POINT(41.575064526875785 73.1374992969178) bank85895 +85896 POINT(39.89151578096414 74.28741873341797) bank85896 +85897 POINT(41.584806390999624 74.92650118795085) bank85897 +85898 POINT(40.49178847151311 73.68670174592228) bank85898 +85899 POINT(40.048533555122475 73.33123997879433) bank85899 +85900 POINT(41.64202258898048 73.17454751213711) bank85900 +85901 POINT(39.717493663002884 74.77415965993136) bank85901 +85902 POINT(40.78920589553092 74.83399039101768) bank85902 +85903 POINT(41.60436468937286 74.95251503824413) bank85903 +85904 POINT(39.73645634467753 73.40867227731543) bank85904 +85905 POINT(41.11372030194787 73.57633136972986) bank85905 +85906 POINT(40.36941052421162 74.11537565380857) bank85906 +85907 POINT(40.950282250775686 74.83945941584568) bank85907 +85908 POINT(41.155990392379586 74.30609405266651) bank85908 +85909 POINT(41.611095289603554 73.19990119753638) bank85909 +85910 POINT(41.613663534313694 74.18715402962627) bank85910 +85911 POINT(41.47118665018018 74.90984260416806) bank85911 +85912 POINT(40.09204582765725 73.67805822966514) bank85912 +85913 POINT(40.52788351639477 73.62993642426846) bank85913 +85914 POINT(40.03162638487738 74.11341266808417) bank85914 +85915 POINT(41.4045754679348 74.76774475387734) bank85915 +85916 POINT(40.308947420658875 73.42287918611608) bank85916 +85917 POINT(41.24303296286172 73.71126536302714) bank85917 +85918 POINT(40.7584468459724 73.73518787035287) bank85918 +85919 POINT(41.62279966131641 74.97449882236523) bank85919 +85920 POINT(40.33555779583032 74.44545011730995) bank85920 +85921 POINT(39.892986537449644 74.08269429216772) bank85921 +85922 POINT(40.79472635684482 73.11981196275104) bank85922 +85923 POINT(40.332422024943284 73.91347124488473) bank85923 +85924 POINT(40.53488166311414 74.00840212221426) bank85924 +85925 POINT(41.083578047621714 73.04718474532028) bank85925 +85926 POINT(39.87320639777267 74.72757483224153) bank85926 +85927 POINT(40.7054352018948 74.05443513583762) bank85927 +85928 POINT(40.069007099954774 74.86710872540294) bank85928 +85929 POINT(40.12389782737715 74.34746115465867) bank85929 +85930 POINT(40.946629496214975 73.91851382790558) bank85930 +85931 POINT(41.039050457027535 73.71590588620542) bank85931 +85932 POINT(41.12058240721448 74.7002752719827) bank85932 +85933 POINT(40.40223948932063 73.82173072085038) bank85933 +85934 POINT(40.40823414698291 74.63763704706986) bank85934 +85935 POINT(41.70387917255815 73.40633369388556) bank85935 +85936 POINT(40.77935146972794 74.24185219198296) bank85936 +85937 POINT(41.393242693138745 74.19326277868592) bank85937 +85938 POINT(41.50300526611506 74.50780985731762) bank85938 +85939 POINT(40.739587117635686 74.11502502458104) bank85939 +85940 POINT(41.43247824238562 74.24513718868839) bank85940 +85941 POINT(40.489841987606326 74.81438999923265) bank85941 +85942 POINT(39.74266786973811 74.24225543005261) bank85942 +85943 POINT(39.92827581543158 73.51913156838562) bank85943 +85944 POINT(39.76966450654052 74.18642714839307) bank85944 +85945 POINT(39.774519823456366 73.2494504141625) bank85945 +85946 POINT(40.39643134702061 74.43080045767913) bank85946 +85947 POINT(40.04541143493664 73.54453789787816) bank85947 +85948 POINT(41.208741243506296 74.9534136206961) bank85948 +85949 POINT(41.33591759817586 73.37547937799414) bank85949 +85950 POINT(40.11858442371576 74.36296225020997) bank85950 +85951 POINT(41.42965529223589 73.70287675431163) bank85951 +85952 POINT(40.39752598742243 74.66668274099447) bank85952 +85953 POINT(41.3407187263807 73.1219757930077) bank85953 +85954 POINT(41.66891208832202 73.4728493339428) bank85954 +85955 POINT(40.2532590081979 74.99124107764108) bank85955 +85956 POINT(40.59541093970267 73.69572280247304) bank85956 +85957 POINT(40.51159444386228 73.39946741610105) bank85957 +85958 POINT(41.2234864581127 73.25649167885281) bank85958 +85959 POINT(40.961283726506515 74.14379259434126) bank85959 +85960 POINT(41.18909326051511 73.01502135034332) bank85960 +85961 POINT(39.84641591415776 73.88954299006274) bank85961 +85962 POINT(40.23266010100987 73.61652454082301) bank85962 +85963 POINT(39.959163061122936 73.82987121904434) bank85963 +85964 POINT(39.77896280593564 73.0574328389355) bank85964 +85965 POINT(39.769925658740675 73.2528075957893) bank85965 +85966 POINT(41.02495384148432 74.6233139393849) bank85966 +85967 POINT(41.39823561137004 73.20926706867226) bank85967 +85968 POINT(40.28823633536709 73.58624654136023) bank85968 +85969 POINT(40.733021499875306 73.96719320179915) bank85969 +85970 POINT(40.933843760339634 73.81561592344578) bank85970 +85971 POINT(41.21796118434769 74.1031544749224) bank85971 +85972 POINT(41.581844766333575 73.28303353782304) bank85972 +85973 POINT(41.13953438542846 73.6186638737999) bank85973 +85974 POINT(40.52863629925161 74.75781643786348) bank85974 +85975 POINT(40.720283116467726 74.11929420332739) bank85975 +85976 POINT(39.72303494860394 74.11547276927747) bank85976 +85977 POINT(39.88369190444564 74.5114862775857) bank85977 +85978 POINT(39.93258419126367 74.29787520941409) bank85978 +85979 POINT(40.935766527421045 74.9765792213218) bank85979 +85980 POINT(40.96515667855432 74.10306964283012) bank85980 +85981 POINT(40.74927047388871 73.6087656508119) bank85981 +85982 POINT(41.51813008060874 74.28723442113875) bank85982 +85983 POINT(39.7524252283478 74.69030425255593) bank85983 +85984 POINT(40.57834851672846 74.95069976750494) bank85984 +85985 POINT(40.95836550902647 73.15719345420915) bank85985 +85986 POINT(40.1511982419695 74.57779488882937) bank85986 +85987 POINT(40.6390511313386 74.11369233244034) bank85987 +85988 POINT(41.09754305088959 73.87062488554159) bank85988 +85989 POINT(40.281456405198746 74.80702656319382) bank85989 +85990 POINT(40.14800401686253 74.73657877876443) bank85990 +85991 POINT(39.74281939418449 74.48274567858375) bank85991 +85992 POINT(41.44477118738164 73.3337838681286) bank85992 +85993 POINT(41.195993148951494 73.62317929645992) bank85993 +85994 POINT(39.736440458646214 74.8210422838077) bank85994 +85995 POINT(40.15152479392121 74.93255049589372) bank85995 +85996 POINT(41.14810952441101 74.19466739922723) bank85996 +85997 POINT(39.7956082577592 73.27972950488137) bank85997 +85998 POINT(41.08139123785472 73.73942260302526) bank85998 +85999 POINT(41.6512696666311 74.86022569845805) bank85999 +86000 POINT(40.392069919967376 73.38783588149983) bank86000 +86001 POINT(40.43908701414271 73.98010818137799) bank86001 +86002 POINT(41.21251876550378 75.00268095367521) bank86002 +86003 POINT(40.74396139441798 74.4849499911368) bank86003 +86004 POINT(40.69596106289176 73.64640408393987) bank86004 +86005 POINT(41.47735619168757 74.96148643780086) bank86005 +86006 POINT(40.691753455798654 74.90183589319629) bank86006 +86007 POINT(41.292596926831216 74.3040655139416) bank86007 +86008 POINT(40.75114039157829 73.23387899139092) bank86008 +86009 POINT(40.90564128003454 73.99671886226834) bank86009 +86010 POINT(41.252031751761116 74.99421433630326) bank86010 +86011 POINT(41.55069446657278 74.099680920388) bank86011 +86012 POINT(41.655215064135184 73.49696167200558) bank86012 +86013 POINT(39.71549148480902 73.52626992309028) bank86013 +86014 POINT(40.361464211856585 74.4078187683936) bank86014 +86015 POINT(40.01540820675393 73.89216830558371) bank86015 +86016 POINT(41.17508458794881 74.20812668876928) bank86016 +86017 POINT(40.25650854959101 73.97700917135188) bank86017 +86018 POINT(40.19215724740025 74.84611506165234) bank86018 +86019 POINT(39.75518926188823 73.12758734147903) bank86019 +86020 POINT(40.76232656174833 74.46645458982924) bank86020 +86021 POINT(41.02207930960588 74.65840024497545) bank86021 +86022 POINT(39.83852192154273 74.8767721090336) bank86022 +86023 POINT(40.564052947648214 74.85509859648973) bank86023 +86024 POINT(40.902747672446054 73.59514311362696) bank86024 +86025 POINT(40.00566094119651 73.54320761198844) bank86025 +86026 POINT(41.04237937547393 73.92139073207487) bank86026 +86027 POINT(40.88571648766236 73.07638252996924) bank86027 +86028 POINT(40.71951955576064 74.41281430880166) bank86028 +86029 POINT(40.17914026841328 74.85178297222895) bank86029 +86030 POINT(41.440444356577814 74.386878710637) bank86030 +86031 POINT(41.42277874603555 74.44443274330864) bank86031 +86032 POINT(41.34928006874142 73.0492071622229) bank86032 +86033 POINT(41.61470062306317 74.45658614673344) bank86033 +86034 POINT(41.224473277721884 73.8632396986782) bank86034 +86035 POINT(41.680689745447474 73.91966076571973) bank86035 +86036 POINT(40.04527458077522 73.95300859532186) bank86036 +86037 POINT(41.34764391985766 74.9441943115108) bank86037 +86038 POINT(41.39536716080087 73.76829846585794) bank86038 +86039 POINT(39.940023198416434 73.59240338447127) bank86039 +86040 POINT(41.07023775969689 73.16899021505105) bank86040 +86041 POINT(40.21515165819811 74.87402116662278) bank86041 +86042 POINT(40.03303992929372 74.35786970583614) bank86042 +86043 POINT(39.76043522697628 74.63927143160964) bank86043 +86044 POINT(40.92308079419444 74.82120857001212) bank86044 +86045 POINT(40.24743201541353 74.1828397020325) bank86045 +86046 POINT(40.139446742588774 74.25324695594708) bank86046 +86047 POINT(40.002917319283775 73.32551882098619) bank86047 +86048 POINT(41.399974394589236 73.42412730366165) bank86048 +86049 POINT(39.84649914253776 73.32574271178278) bank86049 +86050 POINT(39.7691328188755 74.88154669682612) bank86050 +86051 POINT(41.26341065142351 73.58431355875935) bank86051 +86052 POINT(41.56882799639355 74.03730850088795) bank86052 +86053 POINT(41.20420468590537 73.46663235247142) bank86053 +86054 POINT(41.14309446152978 73.35152396916371) bank86054 +86055 POINT(39.94213665604017 74.94218166848027) bank86055 +86056 POINT(41.02739945147113 74.17602384848257) bank86056 +86057 POINT(39.88299110931491 74.20333314815458) bank86057 +86058 POINT(40.54690605448293 73.05297660137151) bank86058 +86059 POINT(40.90740862223992 73.83881141041718) bank86059 +86060 POINT(40.26151581389463 74.2559518709003) bank86060 +86061 POINT(39.8563314259294 73.94100367129411) bank86061 +86062 POINT(41.0669584736612 74.51680019787206) bank86062 +86063 POINT(40.06615314261609 74.63609152641024) bank86063 +86064 POINT(41.432371581662345 74.17517690282111) bank86064 +86065 POINT(41.286371323557105 73.20645555167096) bank86065 +86066 POINT(39.73600871874311 73.54193392566195) bank86066 +86067 POINT(40.59134988864275 74.8422181161759) bank86067 +86068 POINT(41.26200237602284 74.00201328476236) bank86068 +86069 POINT(41.43063090359648 73.83196995901777) bank86069 +86070 POINT(40.882399623028334 73.27381167557468) bank86070 +86071 POINT(40.17397421492007 74.41810649430224) bank86071 +86072 POINT(40.03853250653816 73.20478231946646) bank86072 +86073 POINT(39.755302253152045 74.54474981719491) bank86073 +86074 POINT(40.30638373206939 73.22387142948861) bank86074 +86075 POINT(39.78993523679016 73.36374915310087) bank86075 +86076 POINT(41.049769302909205 73.67181243056551) bank86076 +86077 POINT(40.218471674755776 74.25697473090655) bank86077 +86078 POINT(41.37523832231461 73.08243857184824) bank86078 +86079 POINT(41.66266743313308 74.1483968064471) bank86079 +86080 POINT(40.09549348132298 74.72546140635056) bank86080 +86081 POINT(41.602205546608545 74.74117563519883) bank86081 +86082 POINT(41.20660089937015 73.28545212534065) bank86082 +86083 POINT(40.862374549694934 74.74792308016727) bank86083 +86084 POINT(40.51494508072328 74.68070418298053) bank86084 +86085 POINT(41.520968211353484 73.71626200409257) bank86085 +86086 POINT(41.32729955288104 73.44796549824201) bank86086 +86087 POINT(40.40041686352505 74.13535331507678) bank86087 +86088 POINT(40.69944220902061 73.1225810084771) bank86088 +86089 POINT(39.94047488085799 74.63046643764824) bank86089 +86090 POINT(39.82757127102214 73.32324990812059) bank86090 +86091 POINT(39.8508449245382 74.15533755402201) bank86091 +86092 POINT(41.699329196320306 74.41467908450718) bank86092 +86093 POINT(41.18272944381399 74.38496345700472) bank86093 +86094 POINT(40.66853107114035 73.26589603485857) bank86094 +86095 POINT(40.33643805888464 74.77548281476183) bank86095 +86096 POINT(41.07470206513634 74.83647442910326) bank86096 +86097 POINT(40.51180619466098 73.30542502060204) bank86097 +86098 POINT(41.175836554041794 74.07355743978833) bank86098 +86099 POINT(41.02390093209722 73.9195591168201) bank86099 +86100 POINT(41.635513986747426 73.3532643457) bank86100 +86101 POINT(41.070915497831 74.9986517300549) bank86101 +86102 POINT(41.206756061162885 74.9740598813745) bank86102 +86103 POINT(40.68567901577563 73.20286913182463) bank86103 +86104 POINT(41.31481505933395 74.2643171901491) bank86104 +86105 POINT(40.10035394556176 73.4222913824369) bank86105 +86106 POINT(41.26288797653811 74.01232128947764) bank86106 +86107 POINT(40.48540606348528 73.12736992677978) bank86107 +86108 POINT(40.117531654515425 73.83655694673465) bank86108 +86109 POINT(41.28597757754208 74.48972608966912) bank86109 +86110 POINT(40.768641790310326 74.87666544154852) bank86110 +86111 POINT(39.96960765259838 74.6635966039218) bank86111 +86112 POINT(41.4464426693841 73.59564644584088) bank86112 +86113 POINT(40.78131807984218 73.18682806436199) bank86113 +86114 POINT(41.26766278804278 74.96202006860567) bank86114 +86115 POINT(40.221481964557576 74.15680910707418) bank86115 +86116 POINT(40.09129937053846 73.03540018325171) bank86116 +86117 POINT(40.0304986025014 74.40976966600446) bank86117 +86118 POINT(40.261337039062234 73.67567825347172) bank86118 +86119 POINT(40.57420847980561 74.1734219887339) bank86119 +86120 POINT(40.18156343662226 73.7691460024758) bank86120 +86121 POINT(40.486016381979105 74.47008765680208) bank86121 +86122 POINT(39.85605569447126 73.67961590779383) bank86122 +86123 POINT(40.68458070491336 73.930147408308) bank86123 +86124 POINT(41.61101356256011 73.03361629758268) bank86124 +86125 POINT(40.19139294552305 74.33217372570023) bank86125 +86126 POINT(41.57556701729765 74.70539283499582) bank86126 +86127 POINT(40.868306070598216 74.20131420112851) bank86127 +86128 POINT(41.667863957924375 74.23423069502604) bank86128 +86129 POINT(39.87489543004822 74.91408353818855) bank86129 +86130 POINT(41.15789858159783 73.60095035922532) bank86130 +86131 POINT(41.420637951567564 73.1749332861776) bank86131 +86132 POINT(41.60296019322443 74.04386658982945) bank86132 +86133 POINT(41.05873288871672 73.97825959448222) bank86133 +86134 POINT(41.50709036895742 73.7403218717559) bank86134 +86135 POINT(41.29375403288321 73.48720535790379) bank86135 +86136 POINT(41.35173860396268 73.1941087987599) bank86136 +86137 POINT(41.292849666321736 73.74592054142282) bank86137 +86138 POINT(41.518080349918996 74.31110070406632) bank86138 +86139 POINT(40.868484615481044 74.9395818693317) bank86139 +86140 POINT(40.650055432766564 73.09836944634986) bank86140 +86141 POINT(39.75522508386512 73.00656898943184) bank86141 +86142 POINT(40.192215742786395 73.76542602111367) bank86142 +86143 POINT(40.516527455197874 73.94119815431489) bank86143 +86144 POINT(40.58307798302744 74.41749466185883) bank86144 +86145 POINT(39.77487039221293 73.68016771231004) bank86145 +86146 POINT(41.157058747432565 73.8645157792231) bank86146 +86147 POINT(40.50674941798362 73.07719721536147) bank86147 +86148 POINT(41.50361675015184 74.562269951591) bank86148 +86149 POINT(40.95496727001239 73.85070965313125) bank86149 +86150 POINT(40.86148103045805 73.54832265992765) bank86150 +86151 POINT(40.368946942716654 74.83775260629214) bank86151 +86152 POINT(41.284082350520876 73.0383659760421) bank86152 +86153 POINT(40.928481371395314 73.2172576485399) bank86153 +86154 POINT(40.35297603092104 74.60036975574617) bank86154 +86155 POINT(40.52018226766522 74.33567292735995) bank86155 +86156 POINT(40.77931912127718 73.63054584481696) bank86156 +86157 POINT(41.36570556804326 74.92901315958609) bank86157 +86158 POINT(40.15891407858781 73.62175496474993) bank86158 +86159 POINT(39.98535080026151 73.80203843647558) bank86159 +86160 POINT(39.98607639039247 73.89943744947185) bank86160 +86161 POINT(39.87534594417124 73.67855777863285) bank86161 +86162 POINT(40.916033971980035 73.9388292602701) bank86162 +86163 POINT(40.96603072361151 73.08453420272282) bank86163 +86164 POINT(40.20980966752209 74.0967060164025) bank86164 +86165 POINT(41.05274969466331 73.87176948237784) bank86165 +86166 POINT(41.408189063458366 74.29848522966554) bank86166 +86167 POINT(41.52635065189631 74.78123295927163) bank86167 +86168 POINT(41.46254358267104 74.00266476242246) bank86168 +86169 POINT(41.554669305490194 74.3511996796534) bank86169 +86170 POINT(39.76676763771367 73.23229783690567) bank86170 +86171 POINT(40.36517835934344 74.05213586926743) bank86171 +86172 POINT(41.070946553255126 73.4616926086401) bank86172 +86173 POINT(40.815738099074046 74.92396271067393) bank86173 +86174 POINT(39.77565649078329 73.21209696789792) bank86174 +86175 POINT(41.65670742388424 73.23414384079867) bank86175 +86176 POINT(40.11370754898263 74.86215164181222) bank86176 +86177 POINT(41.008588448802236 74.03922123888744) bank86177 +86178 POINT(40.39459104557285 74.71223054078943) bank86178 +86179 POINT(40.32230074641294 74.22931437015835) bank86179 +86180 POINT(41.210070096893304 73.45764223693261) bank86180 +86181 POINT(41.450303321951594 73.06274881852255) bank86181 +86182 POINT(40.552127111836306 74.7064582586638) bank86182 +86183 POINT(40.964089819672864 73.16771228809588) bank86183 +86184 POINT(41.66203512582292 74.50964894885009) bank86184 +86185 POINT(40.02313873428801 73.9739272125793) bank86185 +86186 POINT(40.85442839604016 74.5569832135929) bank86186 +86187 POINT(39.95208053550888 74.51874964319715) bank86187 +86188 POINT(40.346230807680634 73.7522993592361) bank86188 +86189 POINT(40.97621406787713 74.15442038925761) bank86189 +86190 POINT(40.23621744161549 73.72859243602188) bank86190 +86191 POINT(41.11046947145076 74.44844823329106) bank86191 +86192 POINT(41.698692453063195 73.2712285479242) bank86192 +86193 POINT(40.19302751071939 73.49337334692318) bank86193 +86194 POINT(40.758279914350474 74.00807603018424) bank86194 +86195 POINT(40.55198010385172 74.91271314103706) bank86195 +86196 POINT(40.467658512600586 73.17796836132388) bank86196 +86197 POINT(40.32502070773996 73.5512164318489) bank86197 +86198 POINT(40.84110825036583 73.83009195659167) bank86198 +86199 POINT(40.096238150704046 73.59202116049836) bank86199 +86200 POINT(41.369764535629315 74.80866846547386) bank86200 +86201 POINT(40.65675273673875 74.8936261417755) bank86201 +86202 POINT(39.8809791058209 74.19859139206845) bank86202 +86203 POINT(40.6204724157615 74.5341220798825) bank86203 +86204 POINT(39.847797342321186 74.4195505992043) bank86204 +86205 POINT(40.62236424353212 73.03797960903465) bank86205 +86206 POINT(40.8760549051211 74.63630551591676) bank86206 +86207 POINT(39.72480142418054 74.61336255744824) bank86207 +86208 POINT(40.40760719370757 73.61270924892705) bank86208 +86209 POINT(40.73268499468882 74.88772603219446) bank86209 +86210 POINT(40.90251988903712 74.61872907389163) bank86210 +86211 POINT(40.87998759185116 73.63079458500897) bank86211 +86212 POINT(40.79394219359847 73.67076259764119) bank86212 +86213 POINT(41.34033886483671 74.60308609931575) bank86213 +86214 POINT(41.57735996160459 73.8073447247555) bank86214 +86215 POINT(39.77452405746814 73.41754642214792) bank86215 +86216 POINT(41.263978531423355 74.28186038827528) bank86216 +86217 POINT(40.931275754079884 73.7839877163569) bank86217 +86218 POINT(40.292256592656855 73.14363106166599) bank86218 +86219 POINT(39.99265869143724 73.30146951497498) bank86219 +86220 POINT(40.31762848153811 74.98654496967154) bank86220 +86221 POINT(41.38632392267722 73.71710414015354) bank86221 +86222 POINT(41.299206532642735 74.8038079274771) bank86222 +86223 POINT(40.08989831771731 73.32453445815422) bank86223 +86224 POINT(40.532017499005306 74.99883965539016) bank86224 +86225 POINT(40.46894195480217 73.16033914961625) bank86225 +86226 POINT(41.065891557788994 73.24082085622113) bank86226 +86227 POINT(39.849386562348286 73.15158035155504) bank86227 +86228 POINT(40.55702918071573 73.89941123337738) bank86228 +86229 POINT(41.113408791827204 74.97130065325129) bank86229 +86230 POINT(40.45458029948766 73.53736224712007) bank86230 +86231 POINT(41.16134940230839 74.26749445765526) bank86231 +86232 POINT(40.46671306943182 74.58837687614059) bank86232 +86233 POINT(40.071891441735175 74.94147456633404) bank86233 +86234 POINT(40.80052091484924 74.91866764643272) bank86234 +86235 POINT(41.45968400839746 74.85238897775055) bank86235 +86236 POINT(40.8755600561606 73.15586362483336) bank86236 +86237 POINT(39.82190117087431 73.36386043316526) bank86237 +86238 POINT(40.42254828416434 74.17893066792094) bank86238 +86239 POINT(40.23495983254931 73.46970287628287) bank86239 +86240 POINT(41.38109029804696 73.24654008159285) bank86240 +86241 POINT(40.71181853550934 74.90678939678209) bank86241 +86242 POINT(40.46536369428175 73.44604475154206) bank86242 +86243 POINT(40.04906526235475 73.34334981385483) bank86243 +86244 POINT(41.29688988866237 74.53168966153727) bank86244 +86245 POINT(39.932419723469465 73.08356970671447) bank86245 +86246 POINT(41.06856176380032 74.35048931784362) bank86246 +86247 POINT(41.45143152196657 73.48820217691687) bank86247 +86248 POINT(40.18921885598256 74.5197658010058) bank86248 +86249 POINT(41.2589068506492 74.97705096371776) bank86249 +86250 POINT(39.83486869367112 73.44058715411857) bank86250 +86251 POINT(40.436358733719466 74.12062798583159) bank86251 +86252 POINT(40.586615646723104 73.79289656567053) bank86252 +86253 POINT(40.798571281108714 73.15827085024215) bank86253 +86254 POINT(39.91547193914615 73.8575475738703) bank86254 +86255 POINT(41.66236381178913 73.07203226198267) bank86255 +86256 POINT(40.10613493878973 73.56996736314824) bank86256 +86257 POINT(41.5294338075828 74.23190348566501) bank86257 +86258 POINT(41.065345407222146 74.76298859707182) bank86258 +86259 POINT(41.080649434133655 73.82841151234923) bank86259 +86260 POINT(40.50608822873492 73.05513737375028) bank86260 +86261 POINT(41.549137506207785 74.66453262102918) bank86261 +86262 POINT(41.69989057322442 74.35239598596586) bank86262 +86263 POINT(40.75021797826591 73.18015160293561) bank86263 +86264 POINT(40.72288393782677 74.05829968124642) bank86264 +86265 POINT(41.226931803358035 74.0488371313849) bank86265 +86266 POINT(40.086058339354224 73.72736331783261) bank86266 +86267 POINT(41.04729629315724 73.94999399549955) bank86267 +86268 POINT(40.68811982557332 73.74200586808519) bank86268 +86269 POINT(40.51419321103512 74.21891950335157) bank86269 +86270 POINT(41.391501711956984 73.55358501557888) bank86270 +86271 POINT(40.93531896290579 73.09984271683119) bank86271 +86272 POINT(41.10361762073507 74.05315257698295) bank86272 +86273 POINT(40.142159011838956 73.83125078319975) bank86273 +86274 POINT(41.51732606195734 74.01383775105391) bank86274 +86275 POINT(41.2257496268628 74.17679526692149) bank86275 +86276 POINT(41.42827704750434 73.15280565810885) bank86276 +86277 POINT(40.687672865176744 74.72300446555255) bank86277 +86278 POINT(39.88889967668278 74.45748144620174) bank86278 +86279 POINT(40.062087157313066 74.61440415744478) bank86279 +86280 POINT(41.602013940261465 73.95330963769756) bank86280 +86281 POINT(41.26781738647789 74.42340737872156) bank86281 +86282 POINT(40.65342728047257 74.50685640403603) bank86282 +86283 POINT(40.62744427274074 74.84782582172073) bank86283 +86284 POINT(41.50143714205473 74.30151998334514) bank86284 +86285 POINT(40.76586097899887 74.67971189413501) bank86285 +86286 POINT(40.009440503848396 73.49431561698852) bank86286 +86287 POINT(40.89337595564294 74.85017993430081) bank86287 +86288 POINT(39.9192495377153 73.36530617003869) bank86288 +86289 POINT(41.59877153249923 74.27046428853396) bank86289 +86290 POINT(41.53821550714841 74.58548744827546) bank86290 +86291 POINT(40.59342333189288 74.02540241113331) bank86291 +86292 POINT(40.75260875190071 74.81025973249014) bank86292 +86293 POINT(40.77878721559374 73.27502641267465) bank86293 +86294 POINT(40.82190851929041 73.30073021511896) bank86294 +86295 POINT(39.90409397720307 74.75305473750704) bank86295 +86296 POINT(40.34385830542651 73.1388238405828) bank86296 +86297 POINT(40.723458801468226 73.45104202785055) bank86297 +86298 POINT(41.29809566607603 74.62210618419213) bank86298 +86299 POINT(40.61777538329382 74.34020965440253) bank86299 +86300 POINT(40.971187709183475 73.80434823709871) bank86300 +86301 POINT(40.10881082081029 74.22056916197718) bank86301 +86302 POINT(40.8746468109788 74.87728222412143) bank86302 +86303 POINT(41.178182292305515 74.22808266103826) bank86303 +86304 POINT(39.90175226622647 74.67455949910806) bank86304 +86305 POINT(40.35417982664758 73.92508698366156) bank86305 +86306 POINT(40.555263699146835 73.36478778213376) bank86306 +86307 POINT(41.17650048733595 73.58799766445935) bank86307 +86308 POINT(40.835435797048895 73.68163077245138) bank86308 +86309 POINT(39.9397804649515 73.15851985093244) bank86309 +86310 POINT(40.58813234568055 73.9527499440764) bank86310 +86311 POINT(41.11770184792731 73.1035226445743) bank86311 +86312 POINT(41.64014070201583 73.40905208361501) bank86312 +86313 POINT(40.9797116507458 74.54742251728868) bank86313 +86314 POINT(40.306511595975245 73.27559716362144) bank86314 +86315 POINT(40.46806704455504 74.861478459685) bank86315 +86316 POINT(41.62075799405709 73.69495461859921) bank86316 +86317 POINT(40.02983746337867 74.7987452846063) bank86317 +86318 POINT(40.71471713460847 74.07399339047636) bank86318 +86319 POINT(39.90498917489844 73.11619095354932) bank86319 +86320 POINT(40.97590744449184 73.37198538316932) bank86320 +86321 POINT(40.94248030641999 74.37140083385825) bank86321 +86322 POINT(40.5218322589216 73.27800850426166) bank86322 +86323 POINT(40.446501709658754 73.80193660655787) bank86323 +86324 POINT(39.94310078841378 74.97727238390247) bank86324 +86325 POINT(40.71309709569709 73.085281024028) bank86325 +86326 POINT(41.640762106339565 74.9735615411368) bank86326 +86327 POINT(40.25589973735838 74.91165350166766) bank86327 +86328 POINT(39.73908883643066 73.43396347993516) bank86328 +86329 POINT(40.774938468773826 73.19838595756183) bank86329 +86330 POINT(40.812427870066195 74.87856199168854) bank86330 +86331 POINT(41.48329867306229 73.50782467730136) bank86331 +86332 POINT(41.65907759669602 73.32293993826914) bank86332 +86333 POINT(39.72330120330156 73.47586100229645) bank86333 +86334 POINT(40.114427666621495 74.72929571627033) bank86334 +86335 POINT(39.828657600368906 73.37544235897997) bank86335 +86336 POINT(40.484036720023546 74.70914489649692) bank86336 +86337 POINT(40.34454286310175 73.5053410404057) bank86337 +86338 POINT(40.76605784725838 73.8358063048921) bank86338 +86339 POINT(41.21880049909691 74.3839383170306) bank86339 +86340 POINT(40.983473005211906 73.95618587678541) bank86340 +86341 POINT(39.981944332572134 73.72397731791133) bank86341 +86342 POINT(41.279978384220186 73.593409548743) bank86342 +86343 POINT(40.01851657511119 73.80903051635067) bank86343 +86344 POINT(39.85645436270438 73.915652406705) bank86344 +86345 POINT(41.68595577012686 74.73993884354086) bank86345 +86346 POINT(40.54359225281735 74.91581874616547) bank86346 +86347 POINT(39.82281983346334 74.50435325285024) bank86347 +86348 POINT(41.67913579234196 73.50002391788067) bank86348 +86349 POINT(39.92964794007635 73.07830793629508) bank86349 +86350 POINT(39.73311803306364 73.0839015606725) bank86350 +86351 POINT(40.77760188526369 74.06414380801914) bank86351 +86352 POINT(40.7744428148437 74.62451838880969) bank86352 +86353 POINT(40.626610814162895 74.50138848166462) bank86353 +86354 POINT(41.34492998155636 73.55011227380467) bank86354 +86355 POINT(41.22460359022299 73.56738199072858) bank86355 +86356 POINT(40.10888905584184 73.65801777567111) bank86356 +86357 POINT(41.25953103126693 74.18303924128325) bank86357 +86358 POINT(40.973678257482504 74.88487962222436) bank86358 +86359 POINT(40.54863572485342 73.57046712360157) bank86359 +86360 POINT(41.6973100798938 74.04861637582732) bank86360 +86361 POINT(39.87472705725215 73.87881765345314) bank86361 +86362 POINT(40.076310902954674 74.1421418540218) bank86362 +86363 POINT(40.45119895647819 73.20729803170323) bank86363 +86364 POINT(39.71884247621963 74.30931939131177) bank86364 +86365 POINT(41.116226067062776 74.61775998650344) bank86365 +86366 POINT(40.35383882005649 74.32399832646743) bank86366 +86367 POINT(41.64386362998153 74.46730260212324) bank86367 +86368 POINT(41.130163300100854 73.42940898517732) bank86368 +86369 POINT(40.0186456387311 73.77073928802278) bank86369 +86370 POINT(40.83065091504912 74.9296175825901) bank86370 +86371 POINT(39.776936537697985 73.29279884835752) bank86371 +86372 POINT(41.47359775819363 73.47340222966761) bank86372 +86373 POINT(40.57090358027143 73.8046818934615) bank86373 +86374 POINT(40.76477796026324 74.3495708934037) bank86374 +86375 POINT(39.88988796242461 73.90373430111903) bank86375 +86376 POINT(40.54969554924815 73.45600783651038) bank86376 +86377 POINT(40.751222301733954 73.61809572352041) bank86377 +86378 POINT(40.77552916313787 74.49838956391933) bank86378 +86379 POINT(41.32988044217875 73.76170465197234) bank86379 +86380 POINT(40.45736773796926 74.73532202523477) bank86380 +86381 POINT(41.30107757883271 74.35945313865379) bank86381 +86382 POINT(40.04040820460609 74.6898156538038) bank86382 +86383 POINT(41.01957340810171 73.53749883576187) bank86383 +86384 POINT(39.7865281671031 73.96270509066845) bank86384 +86385 POINT(41.36892350188731 73.90470015194516) bank86385 +86386 POINT(40.874941906517705 74.1344558452583) bank86386 +86387 POINT(41.55359354514763 74.7139867270422) bank86387 +86388 POINT(41.39783068763087 73.27850721734895) bank86388 +86389 POINT(39.77089175970208 74.44641927897976) bank86389 +86390 POINT(40.56336200696488 73.39708688499081) bank86390 +86391 POINT(41.02032714878782 73.19841906133186) bank86391 +86392 POINT(39.75338347139246 73.66698677713211) bank86392 +86393 POINT(39.74676980863459 74.84353008785918) bank86393 +86394 POINT(41.67848325923384 74.3668417050732) bank86394 +86395 POINT(41.493195702002865 73.33686513880632) bank86395 +86396 POINT(40.919937708941454 73.25463778029632) bank86396 +86397 POINT(41.505700519583655 73.90217531757922) bank86397 +86398 POINT(40.386724156725705 74.04231637103454) bank86398 +86399 POINT(41.35806561259218 73.05308258868222) bank86399 +86400 POINT(40.65657829916411 74.33806508521393) bank86400 +86401 POINT(40.293863292745996 74.65502393588802) bank86401 +86402 POINT(41.2998247272291 74.85726824429298) bank86402 +86403 POINT(41.380460829166935 74.96461834177853) bank86403 +86404 POINT(40.00973325525291 74.9500322885018) bank86404 +86405 POINT(40.559451581854454 74.08372637963737) bank86405 +86406 POINT(40.21490047929912 74.10355205133622) bank86406 +86407 POINT(39.95988194101124 73.5697079851951) bank86407 +86408 POINT(41.05119272044669 73.70561372730558) bank86408 +86409 POINT(40.1662196967621 74.09798103474549) bank86409 +86410 POINT(41.21888381031328 73.65766289265385) bank86410 +86411 POINT(39.89500561728627 73.63887366737666) bank86411 +86412 POINT(40.24478007820216 74.38417807814837) bank86412 +86413 POINT(41.06507612434585 73.95008436718814) bank86413 +86414 POINT(40.42724052210229 74.21556648706195) bank86414 +86415 POINT(40.43513280987754 74.7809587920301) bank86415 +86416 POINT(40.06321598779727 74.52046336597303) bank86416 +86417 POINT(41.4921731614587 73.27811106342673) bank86417 +86418 POINT(41.25256799257824 74.24897308088129) bank86418 +86419 POINT(40.36261722540126 73.5674356201966) bank86419 +86420 POINT(40.55438727914696 74.15178113621957) bank86420 +86421 POINT(41.38143221473958 73.34189400593337) bank86421 +86422 POINT(39.997321186322665 74.30323649591115) bank86422 +86423 POINT(41.60741158386916 73.20323436815512) bank86423 +86424 POINT(40.829150038561245 73.02938941370108) bank86424 +86425 POINT(40.07117577497798 74.91898536199322) bank86425 +86426 POINT(40.77364356651107 74.65763059230486) bank86426 +86427 POINT(39.830775221147476 74.0932098334014) bank86427 +86428 POINT(40.20453540139072 74.37070609713756) bank86428 +86429 POINT(41.406959802494725 73.97123446969812) bank86429 +86430 POINT(41.5859950959993 73.56956992774106) bank86430 +86431 POINT(40.53547303389854 74.57504703448308) bank86431 +86432 POINT(40.43851111279054 73.85078775268335) bank86432 +86433 POINT(40.098205247415756 74.49282513479616) bank86433 +86434 POINT(40.523530281045325 74.87181847077882) bank86434 +86435 POINT(40.94207665145053 73.15035314499946) bank86435 +86436 POINT(40.882528949389574 73.90585663884828) bank86436 +86437 POINT(40.54191034567154 74.82062974170425) bank86437 +86438 POINT(41.5882673710621 74.09959816150348) bank86438 +86439 POINT(40.30975372169425 74.95155833670924) bank86439 +86440 POINT(41.44515080083233 73.27884734208305) bank86440 +86441 POINT(40.35306034999751 74.72070517225602) bank86441 +86442 POINT(41.21492754534381 74.30044658061358) bank86442 +86443 POINT(39.9635092301418 74.85234227348752) bank86443 +86444 POINT(41.11756684027774 73.66749042525296) bank86444 +86445 POINT(40.0272098910541 74.93163123481105) bank86445 +86446 POINT(39.83357867731643 74.98942957750258) bank86446 +86447 POINT(41.42628588845044 74.04620149832799) bank86447 +86448 POINT(40.25209777338403 73.55504759536434) bank86448 +86449 POINT(39.85522894975841 74.66264960902454) bank86449 +86450 POINT(39.91869413642318 73.42749330729065) bank86450 +86451 POINT(39.93107292444247 73.8240894670273) bank86451 +86452 POINT(40.207860948835176 73.14703599921828) bank86452 +86453 POINT(40.63972665729186 73.25241458594893) bank86453 +86454 POINT(39.77421749517946 73.52976439251833) bank86454 +86455 POINT(40.590791598264346 73.37548813180396) bank86455 +86456 POINT(41.19060349319493 74.83970174635225) bank86456 +86457 POINT(40.87303475594887 73.30943681344945) bank86457 +86458 POINT(41.071517858703395 73.01572640557299) bank86458 +86459 POINT(41.09079966541012 73.73850998795378) bank86459 +86460 POINT(40.242025321970516 73.64009420338277) bank86460 +86461 POINT(40.30941790991755 74.12871383492461) bank86461 +86462 POINT(40.97967277047221 74.0370254146384) bank86462 +86463 POINT(41.03087427510128 74.99858532559111) bank86463 +86464 POINT(41.19114486200232 74.3885663779696) bank86464 +86465 POINT(40.575384553273175 74.646081701394) bank86465 +86466 POINT(40.16417522755873 73.33988531688718) bank86466 +86467 POINT(41.43632793394969 74.51305990376011) bank86467 +86468 POINT(40.57476583297537 74.98528072670317) bank86468 +86469 POINT(40.679583160520494 74.53765916810703) bank86469 +86470 POINT(40.356998401372685 74.83689284686776) bank86470 +86471 POINT(39.80986626884027 74.19482292651126) bank86471 +86472 POINT(39.91354483018021 73.95307218420798) bank86472 +86473 POINT(39.7496723253838 74.75976971112014) bank86473 +86474 POINT(40.296691114927654 74.09278657503938) bank86474 +86475 POINT(40.13887576099551 73.26617132168869) bank86475 +86476 POINT(40.191175721097686 74.78925286695566) bank86476 +86477 POINT(40.523395482570564 73.06398456111528) bank86477 +86478 POINT(41.11741658143934 73.09607557753458) bank86478 +86479 POINT(40.22978821789955 73.56210577377334) bank86479 +86480 POINT(41.3422819114104 73.49659907792878) bank86480 +86481 POINT(39.91187177460776 73.49498382978503) bank86481 +86482 POINT(41.65733308009547 73.41936585201111) bank86482 +86483 POINT(40.20357915111645 74.6242274551492) bank86483 +86484 POINT(39.80900294640868 74.84846321796601) bank86484 +86485 POINT(41.44105978058323 73.90972531899367) bank86485 +86486 POINT(40.84895864997485 74.08039477580678) bank86486 +86487 POINT(41.026619502622935 73.73363029209754) bank86487 +86488 POINT(40.71819897982821 73.29495821470705) bank86488 +86489 POINT(40.985464853247485 73.27988139445722) bank86489 +86490 POINT(41.591122104133625 73.025558472673) bank86490 +86491 POINT(41.2762275387663 73.28174459551757) bank86491 +86492 POINT(41.55224793120407 74.05708902837497) bank86492 +86493 POINT(40.77033721455286 74.50187460468887) bank86493 +86494 POINT(41.64270266246554 74.26979809832049) bank86494 +86495 POINT(41.1357292186223 73.05879315373193) bank86495 +86496 POINT(40.994094221457544 73.45257678252997) bank86496 +86497 POINT(41.31552498016143 73.75239464118745) bank86497 +86498 POINT(41.14348197565128 73.5013033655604) bank86498 +86499 POINT(40.75625223270296 74.12610199215828) bank86499 +86500 POINT(40.10798406883338 74.34111163674501) bank86500 +86501 POINT(40.392813547370984 74.43105121098883) bank86501 +86502 POINT(41.109704486658764 74.47215940111658) bank86502 +86503 POINT(39.84721730287531 73.61178993962662) bank86503 +86504 POINT(41.416239506976595 73.35339683530181) bank86504 +86505 POINT(41.41981773259829 74.49160283204694) bank86505 +86506 POINT(39.77858554268114 73.82691593611503) bank86506 +86507 POINT(41.35714004259331 74.62719486842686) bank86507 +86508 POINT(39.90898864665825 74.01977933527337) bank86508 +86509 POINT(40.59983900287741 73.91575988138383) bank86509 +86510 POINT(40.91789622794532 74.99788678355353) bank86510 +86511 POINT(40.167945231700976 74.90292939778548) bank86511 +86512 POINT(40.647950827103955 74.48822423768418) bank86512 +86513 POINT(40.671079471041054 74.76078821091629) bank86513 +86514 POINT(41.32732335337718 73.65068161497122) bank86514 +86515 POINT(40.66329113387746 74.88162631726148) bank86515 +86516 POINT(41.34165505419201 74.78152172551874) bank86516 +86517 POINT(39.912822941835834 73.86912990459766) bank86517 +86518 POINT(40.07606877369138 73.15872778689884) bank86518 +86519 POINT(40.91557324442821 74.72464796658655) bank86519 +86520 POINT(40.38079954898868 74.73767995061078) bank86520 +86521 POINT(40.32368890016397 74.85920338267485) bank86521 +86522 POINT(41.30373401445381 74.43820853487085) bank86522 +86523 POINT(41.13725511121195 74.08393348490607) bank86523 +86524 POINT(40.8198447821021 74.66999863032234) bank86524 +86525 POINT(41.335732077050885 74.66339160590746) bank86525 +86526 POINT(41.045868482408814 74.64086799123623) bank86526 +86527 POINT(40.73133278520395 74.63889475440604) bank86527 +86528 POINT(40.417989930310426 73.27308944641801) bank86528 +86529 POINT(41.510707596097284 73.45905064475815) bank86529 +86530 POINT(40.89399852550606 73.08512587694574) bank86530 +86531 POINT(39.81996690058826 73.95924488109625) bank86531 +86532 POINT(40.873580888687854 73.05689517968008) bank86532 +86533 POINT(39.75560268117786 74.28556744073201) bank86533 +86534 POINT(40.27407633850831 74.98247936101674) bank86534 +86535 POINT(41.24588918160814 74.49647117311808) bank86535 +86536 POINT(41.37844374287441 74.08363364889037) bank86536 +86537 POINT(40.00584391616412 74.46094334856079) bank86537 +86538 POINT(41.310753194776765 74.73638798732219) bank86538 +86539 POINT(41.70429257438972 74.47282958422453) bank86539 +86540 POINT(41.006635649491294 73.806371670561) bank86540 +86541 POINT(41.58070618494363 73.01503653268492) bank86541 +86542 POINT(40.402274782338374 74.91876397023573) bank86542 +86543 POINT(39.7797995367307 73.11022503488334) bank86543 +86544 POINT(41.35620030828888 74.11411876591657) bank86544 +86545 POINT(40.4513798952906 74.17688771218572) bank86545 +86546 POINT(40.824643877315985 73.57679649990574) bank86546 +86547 POINT(40.60314735378382 74.40659337119511) bank86547 +86548 POINT(40.18797254166707 74.77948311195159) bank86548 +86549 POINT(41.271661425530155 73.84004103826919) bank86549 +86550 POINT(41.00997273824846 73.65301455406482) bank86550 +86551 POINT(39.89180926468758 74.00644923814251) bank86551 +86552 POINT(40.07199716110177 73.15656729327733) bank86552 +86553 POINT(41.07490727456752 73.11051760514745) bank86553 +86554 POINT(39.84025423178151 73.58281888654845) bank86554 +86555 POINT(40.28311291031527 74.65641266179986) bank86555 +86556 POINT(40.36594425856091 73.90826674870779) bank86556 +86557 POINT(40.6685035089974 74.33729884514062) bank86557 +86558 POINT(40.84322453465168 73.61112780525792) bank86558 +86559 POINT(41.421302407582154 74.21137583151781) bank86559 +86560 POINT(41.031269346433234 73.07717019028637) bank86560 +86561 POINT(40.957207722808775 74.81080763673812) bank86561 +86562 POINT(41.01120589354118 74.22290733468283) bank86562 +86563 POINT(40.87909081438473 74.61873978870665) bank86563 +86564 POINT(40.23441837194483 74.4454316055171) bank86564 +86565 POINT(40.00190123412273 74.33367686169908) bank86565 +86566 POINT(40.16995344755146 73.41378421133089) bank86566 +86567 POINT(41.04494888747843 73.89318937948026) bank86567 +86568 POINT(40.76775062406935 73.15125849647089) bank86568 +86569 POINT(41.011293047057464 73.82116520458015) bank86569 +86570 POINT(39.74077158806329 73.55860639385764) bank86570 +86571 POINT(40.193861714505736 74.82624816711856) bank86571 +86572 POINT(40.54230326299195 73.65148900337036) bank86572 +86573 POINT(39.94756421940412 73.8150445086874) bank86573 +86574 POINT(40.548890172103896 74.15174618772228) bank86574 +86575 POINT(41.56990890274001 74.6179902509089) bank86575 +86576 POINT(41.253329214534624 73.97047794695973) bank86576 +86577 POINT(40.78156177089576 74.24439077678879) bank86577 +86578 POINT(40.88618906774996 74.275863551866) bank86578 +86579 POINT(40.52388559396025 74.62107600892567) bank86579 +86580 POINT(40.054195560467925 73.94035373810051) bank86580 +86581 POINT(41.566307860185134 74.50180068529424) bank86581 +86582 POINT(39.88903872287607 73.94201962962042) bank86582 +86583 POINT(40.29344493263832 73.63428100447706) bank86583 +86584 POINT(40.89765959606247 74.80681536342512) bank86584 +86585 POINT(40.14391746966995 73.89535660288817) bank86585 +86586 POINT(41.5968158430665 73.58835123580238) bank86586 +86587 POINT(40.06483060331824 74.58382392707372) bank86587 +86588 POINT(40.25384097582101 74.67297505105645) bank86588 +86589 POINT(40.01669211738715 74.79736726342128) bank86589 +86590 POINT(40.42000918553215 74.46737967563989) bank86590 +86591 POINT(40.49612019970072 73.16980857436062) bank86591 +86592 POINT(39.863535863131496 73.36617106716133) bank86592 +86593 POINT(40.04801447675764 73.79451274673893) bank86593 +86594 POINT(41.39618963074752 73.93457925364834) bank86594 +86595 POINT(39.82044714141136 73.57332966358939) bank86595 +86596 POINT(40.39407749196134 74.15865831781652) bank86596 +86597 POINT(40.028556266924326 74.5634193333771) bank86597 +86598 POINT(39.891066231729056 74.16257204287854) bank86598 +86599 POINT(39.8252379607302 73.02375052029161) bank86599 +86600 POINT(40.33459087770617 74.37776852796829) bank86600 +86601 POINT(41.58642375735697 74.54065165377285) bank86601 +86602 POINT(40.67302792894839 73.63364322829877) bank86602 +86603 POINT(41.1989864512589 74.20099518304377) bank86603 +86604 POINT(40.044984987382506 73.58005935691332) bank86604 +86605 POINT(41.68093702010478 74.17705286401852) bank86605 +86606 POINT(40.696512108150266 73.14655981431824) bank86606 +86607 POINT(40.104817516174364 73.48804222226799) bank86607 +86608 POINT(41.05530583562692 74.6321389357318) bank86608 +86609 POINT(39.77511355754733 74.11629148723966) bank86609 +86610 POINT(40.56267835552094 73.80121867885542) bank86610 +86611 POINT(39.942262589271266 73.81023939185498) bank86611 +86612 POINT(40.15197359550586 74.88946136172865) bank86612 +86613 POINT(40.68932859911488 73.75424466080264) bank86613 +86614 POINT(41.38471276615285 73.11741222187158) bank86614 +86615 POINT(41.279834060279335 74.01797709988962) bank86615 +86616 POINT(40.05074944594059 74.4768483088585) bank86616 +86617 POINT(41.20201212115326 74.54537434891694) bank86617 +86618 POINT(40.700519901903235 73.72218232905496) bank86618 +86619 POINT(40.99667604939967 73.60182736702349) bank86619 +86620 POINT(40.85744233822653 74.6446662399841) bank86620 +86621 POINT(40.644282109871405 73.33067253600554) bank86621 +86622 POINT(41.17578433634462 74.1552301669164) bank86622 +86623 POINT(41.56385430698107 73.91929940110029) bank86623 +86624 POINT(40.046038910732136 73.89290755866887) bank86624 +86625 POINT(41.29356009962455 74.2901669460947) bank86625 +86626 POINT(41.70532449619212 73.16140664101276) bank86626 +86627 POINT(41.34606981490655 74.82800814349424) bank86627 +86628 POINT(41.32589623745995 74.7119324546676) bank86628 +86629 POINT(40.85464994015665 73.52259623153252) bank86629 +86630 POINT(40.84501949268928 74.25581779801453) bank86630 +86631 POINT(40.838939225979445 73.65021033562128) bank86631 +86632 POINT(41.35611341305314 73.72826809841055) bank86632 +86633 POINT(40.55463556721901 74.96712614550736) bank86633 +86634 POINT(40.30431816260067 73.75473200277659) bank86634 +86635 POINT(41.01903816378347 73.52927794630375) bank86635 +86636 POINT(41.53402756888746 73.19453228418516) bank86636 +86637 POINT(40.02455747237526 73.36841377395159) bank86637 +86638 POINT(40.15568993090352 73.61018007159073) bank86638 +86639 POINT(41.49368284872268 74.50528053134411) bank86639 +86640 POINT(40.29590161136272 74.16516713112672) bank86640 +86641 POINT(41.40692161427115 74.91980873461011) bank86641 +86642 POINT(41.426082107681296 73.05163308142342) bank86642 +86643 POINT(41.31000862677021 74.22762323786019) bank86643 +86644 POINT(40.21674539100979 73.27306451137156) bank86644 +86645 POINT(40.15412958243072 73.4551119171792) bank86645 +86646 POINT(40.43600696184033 74.19581011527534) bank86646 +86647 POINT(40.487125377757145 74.44524685413027) bank86647 +86648 POINT(39.974644081175384 73.59793479342638) bank86648 +86649 POINT(41.0805687206124 74.80454918264927) bank86649 +86650 POINT(40.197770954184556 73.04344018639165) bank86650 +86651 POINT(40.97537374013252 74.26807484724985) bank86651 +86652 POINT(41.141156803441305 74.42402189659231) bank86652 +86653 POINT(41.31064644564519 74.88771308574525) bank86653 +86654 POINT(41.45416945151015 74.95733059368537) bank86654 +86655 POINT(41.354296798218236 74.94759179694748) bank86655 +86656 POINT(40.707543991580025 73.37236908287086) bank86656 +86657 POINT(39.83282665352608 74.70609262600296) bank86657 +86658 POINT(41.32162960663635 73.81960176996286) bank86658 +86659 POINT(40.97002297552883 74.51743125718741) bank86659 +86660 POINT(41.59859485352492 74.34979441131709) bank86660 +86661 POINT(41.05618182406832 74.61337476499013) bank86661 +86662 POINT(40.310050111511714 73.92444870412646) bank86662 +86663 POINT(40.422173702516176 73.70870562122013) bank86663 +86664 POINT(40.03330065713116 74.0292957979049) bank86664 +86665 POINT(41.397045895525864 73.3029138547589) bank86665 +86666 POINT(40.32981273942907 74.78555368355714) bank86666 +86667 POINT(39.88269401446389 73.99897085909947) bank86667 +86668 POINT(40.93412131677725 73.32086617512053) bank86668 +86669 POINT(40.63707232672073 74.24003306528083) bank86669 +86670 POINT(40.535062088360895 73.3125956108197) bank86670 +86671 POINT(40.92908131703348 73.05690088436586) bank86671 +86672 POINT(40.61820551234992 74.92034944838902) bank86672 +86673 POINT(41.671586355595316 73.7942802165497) bank86673 +86674 POINT(41.35779446231031 74.76461158473873) bank86674 +86675 POINT(40.73923342692261 74.43167540059672) bank86675 +86676 POINT(40.47064713400786 74.91399830393378) bank86676 +86677 POINT(41.21434469887836 74.00560606608481) bank86677 +86678 POINT(40.086868451526314 73.94662329823588) bank86678 +86679 POINT(41.57148013442284 74.5422377437238) bank86679 +86680 POINT(41.656745277492654 74.28942918565807) bank86680 +86681 POINT(40.9602208564427 74.7460034520805) bank86681 +86682 POINT(40.067942885529 73.18131113592113) bank86682 +86683 POINT(41.27195940834679 73.7341697771251) bank86683 +86684 POINT(40.3397665117913 73.45919101681031) bank86684 +86685 POINT(40.50848784781581 74.16770838515833) bank86685 +86686 POINT(41.141968842785445 73.10176392430529) bank86686 +86687 POINT(39.884634873314866 73.94262549821983) bank86687 +86688 POINT(39.87574278376212 74.30804596055518) bank86688 +86689 POINT(40.72204610963338 74.11265047696318) bank86689 +86690 POINT(41.11017135186649 73.2663473243351) bank86690 +86691 POINT(41.20879319008964 73.32753420826177) bank86691 +86692 POINT(41.41093241157158 74.74865986461221) bank86692 +86693 POINT(40.70803317867182 74.14252619359955) bank86693 +86694 POINT(40.974373911260834 73.58262392702089) bank86694 +86695 POINT(40.823911195893885 74.25279324840727) bank86695 +86696 POINT(40.78434412061718 74.92312524248655) bank86696 +86697 POINT(41.4222674386433 73.81938088414738) bank86697 +86698 POINT(40.58917770828084 74.69461859441503) bank86698 +86699 POINT(40.307976330512574 74.91243308560381) bank86699 +86700 POINT(40.90689929962364 74.31842385617347) bank86700 +86701 POINT(39.84455864251313 73.24337394428888) bank86701 +86702 POINT(41.6625577511403 73.13945176708901) bank86702 +86703 POINT(40.366554850327404 74.84487500040224) bank86703 +86704 POINT(40.72251144341371 74.37859322098393) bank86704 +86705 POINT(39.98956138643528 73.27355258261638) bank86705 +86706 POINT(40.689541482425255 74.5522470153893) bank86706 +86707 POINT(41.50865912747767 73.80364203687853) bank86707 +86708 POINT(39.72036684403963 74.48254428244906) bank86708 +86709 POINT(40.33389874601336 74.8441950746622) bank86709 +86710 POINT(40.10162844086154 74.48389970975364) bank86710 +86711 POINT(41.47252562625465 73.5500840280607) bank86711 +86712 POINT(39.97217595851938 73.52930923655839) bank86712 +86713 POINT(41.64979884487178 74.0462910437547) bank86713 +86714 POINT(40.23616872247628 74.2012538988082) bank86714 +86715 POINT(39.801693449605715 74.93501324469709) bank86715 +86716 POINT(40.19610116379828 73.54464117111618) bank86716 +86717 POINT(39.836811636489905 74.033895063762) bank86717 +86718 POINT(40.93206973256435 74.40729709601804) bank86718 +86719 POINT(40.415758340624606 73.39697841252992) bank86719 +86720 POINT(41.63927635479547 74.51281929496274) bank86720 +86721 POINT(41.165868755447526 74.38204258623736) bank86721 +86722 POINT(39.91292799715562 74.42654605022814) bank86722 +86723 POINT(39.86948180891831 74.21327737283902) bank86723 +86724 POINT(41.15040200053711 74.27115581323456) bank86724 +86725 POINT(39.83772068425956 74.60152826695604) bank86725 +86726 POINT(40.79718480922647 73.69528405972335) bank86726 +86727 POINT(40.24712062482023 73.35090449759913) bank86727 +86728 POINT(41.56750728744886 74.52534966037493) bank86728 +86729 POINT(40.170212067263925 73.42980289396539) bank86729 +86730 POINT(41.56973206359242 73.2110619177211) bank86730 +86731 POINT(41.01488460373511 74.738991608822) bank86731 +86732 POINT(39.95310238649765 74.46422759007773) bank86732 +86733 POINT(40.62225847530817 74.12748034302477) bank86733 +86734 POINT(40.6242004053602 73.00810505168262) bank86734 +86735 POINT(40.99978731175132 74.00748573528195) bank86735 +86736 POINT(40.78531692982 74.87375280197669) bank86736 +86737 POINT(41.27772582783736 73.65603817456042) bank86737 +86738 POINT(41.615552714804316 74.46170912400852) bank86738 +86739 POINT(41.579231552069984 74.38612274404325) bank86739 +86740 POINT(40.7081831311765 73.23528117883588) bank86740 +86741 POINT(39.732205015034786 73.84597215249664) bank86741 +86742 POINT(40.01568930614621 73.2075470609076) bank86742 +86743 POINT(41.53652299570938 74.6168787799368) bank86743 +86744 POINT(40.75152775948858 74.46130323288457) bank86744 +86745 POINT(41.17528926036158 74.28556422834995) bank86745 +86746 POINT(39.73970164787648 73.2987603331799) bank86746 +86747 POINT(40.6748533501538 74.24591761106115) bank86747 +86748 POINT(40.27944359113872 74.36338376857684) bank86748 +86749 POINT(41.26169926752863 74.23279565203755) bank86749 +86750 POINT(40.61799133080768 73.62216127285973) bank86750 +86751 POINT(41.57497911514491 73.03962045627564) bank86751 +86752 POINT(41.33028795887211 73.06708222091814) bank86752 +86753 POINT(40.8914399853067 73.18349455020987) bank86753 +86754 POINT(40.86050848475325 74.28208997024252) bank86754 +86755 POINT(39.98154533195692 73.48899176240762) bank86755 +86756 POINT(40.76607821365164 73.04688919284584) bank86756 +86757 POINT(41.57717830688598 74.07982299619137) bank86757 +86758 POINT(40.214478403372716 73.9114285886584) bank86758 +86759 POINT(39.756880637469976 74.38101565213687) bank86759 +86760 POINT(41.07067750938294 74.52060829907762) bank86760 +86761 POINT(40.76978259163493 74.78862412338698) bank86761 +86762 POINT(41.3624326182959 74.22456887366546) bank86762 +86763 POINT(41.57216343789912 73.72995326390928) bank86763 +86764 POINT(40.04286452148187 74.46264833404823) bank86764 +86765 POINT(40.8375790646319 73.34096721221718) bank86765 +86766 POINT(41.18238400572447 73.90076391359138) bank86766 +86767 POINT(40.24799450672682 74.63130712317955) bank86767 +86768 POINT(40.74036040253127 74.34130332121008) bank86768 +86769 POINT(39.78199547506069 73.58319001619074) bank86769 +86770 POINT(39.78443433941668 75.00322054746806) bank86770 +86771 POINT(41.19711598189397 73.1274671900262) bank86771 +86772 POINT(40.37758588786154 73.74485321299392) bank86772 +86773 POINT(41.36794516155622 73.3392121789278) bank86773 +86774 POINT(40.83630542800454 73.73190605291785) bank86774 +86775 POINT(41.47526321346116 73.82673065221519) bank86775 +86776 POINT(41.43761335168769 74.6141524801157) bank86776 +86777 POINT(39.85064927713365 73.64674317514005) bank86777 +86778 POINT(41.280920767925764 73.15907668375661) bank86778 +86779 POINT(41.43733861774662 74.8161752699385) bank86779 +86780 POINT(40.193688465442314 74.6479169286206) bank86780 +86781 POINT(41.00787026173693 73.90882354294178) bank86781 +86782 POINT(41.47205474684978 73.58786306100892) bank86782 +86783 POINT(41.665905105923834 73.21954514455585) bank86783 +86784 POINT(40.22269528395943 73.9098963390623) bank86784 +86785 POINT(40.899229961899024 74.90067068383274) bank86785 +86786 POINT(39.7548043554545 73.23396969238209) bank86786 +86787 POINT(39.833230101895076 73.38585542028908) bank86787 +86788 POINT(40.18021306885406 73.68273928459394) bank86788 +86789 POINT(41.00262571746991 74.7986042285583) bank86789 +86790 POINT(40.81062372004764 73.62826477611982) bank86790 +86791 POINT(41.40962916257213 73.13584639358481) bank86791 +86792 POINT(40.39681971160074 74.08435843244722) bank86792 +86793 POINT(41.487144325045904 74.44984436931998) bank86793 +86794 POINT(40.697399260328744 73.80020864360797) bank86794 +86795 POINT(40.535685851384635 74.81379165739419) bank86795 +86796 POINT(39.81008738570256 74.42453913192973) bank86796 +86797 POINT(41.07133878054172 74.14272295568978) bank86797 +86798 POINT(40.55074620660634 73.20789838914432) bank86798 +86799 POINT(40.8540503172589 73.60809935435789) bank86799 +86800 POINT(40.79551437244648 73.08564405354173) bank86800 +86801 POINT(40.47712516203841 74.65643755251209) bank86801 +86802 POINT(40.09572641432716 73.31403891897703) bank86802 +86803 POINT(40.21161713179772 74.35068986404868) bank86803 +86804 POINT(41.573468680289295 73.47562804519342) bank86804 +86805 POINT(39.98471072766417 73.09389929288453) bank86805 +86806 POINT(39.771655773797356 74.59261882057035) bank86806 +86807 POINT(40.87670357089194 73.62608331836957) bank86807 +86808 POINT(40.75595751846512 73.34084017273328) bank86808 +86809 POINT(39.823820209784074 74.86419034007498) bank86809 +86810 POINT(40.89297909107263 74.65407707037402) bank86810 +86811 POINT(40.24639193554485 73.53122097810815) bank86811 +86812 POINT(41.39933223938407 73.01730676653423) bank86812 +86813 POINT(40.08297179587238 73.12841168815814) bank86813 +86814 POINT(39.83566980009983 74.81415746500579) bank86814 +86815 POINT(40.01042651845053 74.81399593593272) bank86815 +86816 POINT(40.49155189097034 74.46256425030703) bank86816 +86817 POINT(41.1766021104451 73.6364531767106) bank86817 +86818 POINT(40.97605252979852 73.78834909373982) bank86818 +86819 POINT(41.271923546272184 73.36583024969536) bank86819 +86820 POINT(40.43401048614768 73.6907637213037) bank86820 +86821 POINT(41.62617615161273 74.88960465526932) bank86821 +86822 POINT(41.32916206224649 74.61208761148086) bank86822 +86823 POINT(40.514654495907564 73.22001095546078) bank86823 +86824 POINT(40.66946592250878 75.00100379508869) bank86824 +86825 POINT(40.95321853024561 74.90430661724601) bank86825 +86826 POINT(40.35203551384538 74.92552212972966) bank86826 +86827 POINT(41.28048635820614 73.13558288012814) bank86827 +86828 POINT(40.49200446801262 73.76155789711899) bank86828 +86829 POINT(39.93090458183075 73.7588297546683) bank86829 +86830 POINT(41.37664708557784 74.56604727342125) bank86830 +86831 POINT(41.28539850704639 74.90090785407114) bank86831 +86832 POINT(41.63337403437136 73.02592515781953) bank86832 +86833 POINT(39.98029283157181 73.14136908080788) bank86833 +86834 POINT(41.57193443814042 73.67783468262812) bank86834 +86835 POINT(39.81975584881564 74.30028953052911) bank86835 +86836 POINT(40.42409296506775 73.20616471244638) bank86836 +86837 POINT(40.67511548863493 73.36431654723188) bank86837 +86838 POINT(40.35205261892967 74.08309839065409) bank86838 +86839 POINT(41.29836095312798 73.92663284940758) bank86839 +86840 POINT(41.21661351481158 74.14458995039311) bank86840 +86841 POINT(40.27722762988467 73.43058560687612) bank86841 +86842 POINT(40.997727317893066 73.51736796307041) bank86842 +86843 POINT(41.533608118004345 73.93913930229255) bank86843 +86844 POINT(40.91958587440726 73.25591479709632) bank86844 +86845 POINT(41.035135609930215 73.08022685417885) bank86845 +86846 POINT(40.49349334258472 74.07815274945786) bank86846 +86847 POINT(41.62137202757379 74.27485358838561) bank86847 +86848 POINT(40.082581797836575 74.96858894816856) bank86848 +86849 POINT(40.00776610456717 74.9435323924613) bank86849 +86850 POINT(39.831894674506536 74.49974018862763) bank86850 +86851 POINT(41.071178660344714 74.78592686326051) bank86851 +86852 POINT(40.7004146500389 73.18605953326725) bank86852 +86853 POINT(41.3879422140738 74.9061675281691) bank86853 +86854 POINT(41.71201846890243 74.05246196602042) bank86854 +86855 POINT(39.95123085894073 74.86434733130908) bank86855 +86856 POINT(40.684927741736296 73.9346662150318) bank86856 +86857 POINT(40.373437353680266 74.78888905662815) bank86857 +86858 POINT(40.523009726540906 74.7897813298911) bank86858 +86859 POINT(41.279493917746365 73.56400040215156) bank86859 +86860 POINT(41.651836327658636 73.30340848841898) bank86860 +86861 POINT(40.85319513090683 73.16537541486326) bank86861 +86862 POINT(39.92955584134384 74.50819478526847) bank86862 +86863 POINT(41.39625963127885 74.4310163391917) bank86863 +86864 POINT(40.72262863690241 74.54694196367558) bank86864 +86865 POINT(40.95662477242434 74.56123826435643) bank86865 +86866 POINT(41.02674172962702 74.12725031089784) bank86866 +86867 POINT(41.25847688457308 73.90758763973854) bank86867 +86868 POINT(40.43223929496758 73.7568691630123) bank86868 +86869 POINT(41.609171728860765 73.79308008488132) bank86869 +86870 POINT(41.45635454647203 74.12434549328012) bank86870 +86871 POINT(41.58280854035393 74.17738498144128) bank86871 +86872 POINT(40.70104546169759 74.13494139319609) bank86872 +86873 POINT(41.49200418916386 74.05236236809795) bank86873 +86874 POINT(40.58701250739667 73.04517052353847) bank86874 +86875 POINT(40.47490674252771 73.93276626205352) bank86875 +86876 POINT(40.739840359019404 74.70724371864848) bank86876 +86877 POINT(40.494417205012994 74.43618814992801) bank86877 +86878 POINT(40.69368178770785 73.9966402213025) bank86878 +86879 POINT(40.50200528344166 73.00841370374219) bank86879 +86880 POINT(40.07881756611956 73.45730443750597) bank86880 +86881 POINT(39.792889725283565 73.51886464466894) bank86881 +86882 POINT(41.50952300432043 74.5416775812708) bank86882 +86883 POINT(40.986241012726275 74.70289154207593) bank86883 +86884 POINT(40.322573895652354 73.53076845100341) bank86884 +86885 POINT(40.244235594486945 73.12484749590946) bank86885 +86886 POINT(40.079838017104024 73.19089380350904) bank86886 +86887 POINT(40.67591564412692 74.61542222152532) bank86887 +86888 POINT(40.633017915831545 74.80641178468773) bank86888 +86889 POINT(41.201262904830365 73.49163791483615) bank86889 +86890 POINT(39.852324963757724 74.61551354625252) bank86890 +86891 POINT(41.44006621137541 74.11839234606012) bank86891 +86892 POINT(40.670385831884595 74.42872508787225) bank86892 +86893 POINT(40.69408226924486 73.05739884146901) bank86893 +86894 POINT(40.36497352425098 74.10725498963791) bank86894 +86895 POINT(41.59081209660288 74.7440932389298) bank86895 +86896 POINT(41.45425627061194 73.15507380856) bank86896 +86897 POINT(41.00167502368975 73.06316600723542) bank86897 +86898 POINT(40.65745299117512 74.96788984472023) bank86898 +86899 POINT(40.6697996168117 73.89015738328) bank86899 +86900 POINT(41.6421888704495 74.88536534205595) bank86900 +86901 POINT(41.444080286413104 74.92926323981438) bank86901 +86902 POINT(40.66845688022494 73.37381374424221) bank86902 +86903 POINT(39.90761826761919 73.8303039120171) bank86903 +86904 POINT(40.01277284054522 74.17589865425975) bank86904 +86905 POINT(40.07789499909648 74.41148776689928) bank86905 +86906 POINT(40.34769500801822 73.83935884472348) bank86906 +86907 POINT(39.77318021138025 74.82870798610058) bank86907 +86908 POINT(40.372343747700306 74.48763187065175) bank86908 +86909 POINT(41.57580062953986 73.15096419197981) bank86909 +86910 POINT(40.71217910323029 73.52814596420157) bank86910 +86911 POINT(41.42427415077156 74.24164518830524) bank86911 +86912 POINT(40.1575425755869 74.174129787683) bank86912 +86913 POINT(40.36943449251857 74.11451490535217) bank86913 +86914 POINT(40.37589112506892 73.08104151829451) bank86914 +86915 POINT(41.41879066263298 73.04090081867714) bank86915 +86916 POINT(40.40574501970162 74.95054126046921) bank86916 +86917 POINT(40.12496702609147 73.93190371816878) bank86917 +86918 POINT(41.43504040274375 74.53396658521187) bank86918 +86919 POINT(40.9109309027012 73.88389917996295) bank86919 +86920 POINT(41.25132444883045 73.48465456574488) bank86920 +86921 POINT(40.286690668227145 74.4830055077214) bank86921 +86922 POINT(41.3213842310127 73.17124153786845) bank86922 +86923 POINT(40.929939644606556 73.74879862299947) bank86923 +86924 POINT(39.903957841671456 73.73078085609929) bank86924 +86925 POINT(41.44757481581295 73.18023654922389) bank86925 +86926 POINT(40.03159321475485 73.74756626200073) bank86926 +86927 POINT(41.30075399102306 74.79073515652581) bank86927 +86928 POINT(41.02282149342788 74.55106390207884) bank86928 +86929 POINT(40.502008368590474 73.19325859246065) bank86929 +86930 POINT(40.86837433046705 73.74052421115539) bank86930 +86931 POINT(41.25155285375783 73.09795283970662) bank86931 +86932 POINT(40.45008723608863 73.41423527721568) bank86932 +86933 POINT(40.80935554805113 73.40532037197333) bank86933 +86934 POINT(40.91356984778106 74.88390071621724) bank86934 +86935 POINT(39.764786288756106 73.5071050946375) bank86935 +86936 POINT(41.51336188317743 73.99803379417156) bank86936 +86937 POINT(40.89972644594748 73.5972115706055) bank86937 +86938 POINT(40.24720080358333 74.75969761877137) bank86938 +86939 POINT(41.47277341099508 73.7406915507957) bank86939 +86940 POINT(39.90628680383135 74.70852002051468) bank86940 +86941 POINT(40.54391146359199 74.80164592023883) bank86941 +86942 POINT(41.482055651580175 73.18358528819958) bank86942 +86943 POINT(40.34452841882174 73.02429809678064) bank86943 +86944 POINT(40.321804048505655 74.91956648210022) bank86944 +86945 POINT(41.4761835285582 74.09629097615718) bank86945 +86946 POINT(40.43919294549925 74.7747657270294) bank86946 +86947 POINT(39.94737187643523 73.49725156627316) bank86947 +86948 POINT(41.39332303207062 74.2151821772271) bank86948 +86949 POINT(39.71925036074595 73.57105710613486) bank86949 +86950 POINT(41.15694466157329 73.2648730819794) bank86950 +86951 POINT(40.88425937104483 73.91213841737165) bank86951 +86952 POINT(41.052594483322316 73.08364113056676) bank86952 +86953 POINT(39.76658504816908 73.25330340794534) bank86953 +86954 POINT(41.424128827327856 73.92415923482814) bank86954 +86955 POINT(41.578125776552476 74.0478541722349) bank86955 +86956 POINT(39.877114793730435 73.24866256398494) bank86956 +86957 POINT(41.54357656000404 73.8248888400541) bank86957 +86958 POINT(40.142855501574715 73.78825506351976) bank86958 +86959 POINT(41.194074865109606 73.63151937938275) bank86959 +86960 POINT(40.275851174483854 73.53096029427297) bank86960 +86961 POINT(40.331445413054524 74.29565147341464) bank86961 +86962 POINT(40.61535854657576 74.21023374974831) bank86962 +86963 POINT(40.47370374543196 74.39461380666269) bank86963 +86964 POINT(39.72435056655635 74.16815847598282) bank86964 +86965 POINT(40.00583712425849 74.1251589075563) bank86965 +86966 POINT(41.47233842387022 74.32214320627978) bank86966 +86967 POINT(41.24121885046638 74.68503138679587) bank86967 +86968 POINT(41.02257393313767 73.06037387047711) bank86968 +86969 POINT(41.59315429915477 73.89131168090816) bank86969 +86970 POINT(41.563132061635464 73.70748407314215) bank86970 +86971 POINT(40.88458624661895 74.00270570029093) bank86971 +86972 POINT(40.8161843116998 73.80291248997335) bank86972 +86973 POINT(40.75896458402596 73.11886411813403) bank86973 +86974 POINT(40.306398434332536 74.52048379194073) bank86974 +86975 POINT(41.175339864275124 73.87720810468394) bank86975 +86976 POINT(41.40212040070525 74.71927452310543) bank86976 +86977 POINT(40.46525267507447 74.91908609139482) bank86977 +86978 POINT(39.78170724243423 73.95204791867283) bank86978 +86979 POINT(40.7318259686231 74.90786752683569) bank86979 +86980 POINT(40.38372793137871 74.53634393318255) bank86980 +86981 POINT(40.06533185238941 74.67067494815655) bank86981 +86982 POINT(40.6580067130547 73.02340095506439) bank86982 +86983 POINT(40.977260732918495 74.12689696745541) bank86983 +86984 POINT(41.06106600120264 74.16118195095756) bank86984 +86985 POINT(41.48490733860404 74.91654955548206) bank86985 +86986 POINT(39.71956090020923 73.24359651467812) bank86986 +86987 POINT(40.13066878501609 73.4627474210178) bank86987 +86988 POINT(41.61177745542143 73.72356869964061) bank86988 +86989 POINT(41.207098856196325 73.1369413284604) bank86989 +86990 POINT(40.830922066119086 74.9444804866275) bank86990 +86991 POINT(40.91535797542141 74.12511725235679) bank86991 +86992 POINT(40.79346966841669 74.2798132938009) bank86992 +86993 POINT(40.082730201719755 74.47024127119576) bank86993 +86994 POINT(40.99769832635885 74.58808285181159) bank86994 +86995 POINT(40.15898203187204 74.05736996854397) bank86995 +86996 POINT(40.212521670590164 74.00863470835091) bank86996 +86997 POINT(41.29677751242299 74.40979328379439) bank86997 +86998 POINT(40.28575164930708 73.62316459195844) bank86998 +86999 POINT(41.398964889314506 74.17121161748251) bank86999 +87000 POINT(39.906296800789676 74.6590870276093) bank87000 +87001 POINT(41.13631185264737 73.97778479997764) bank87001 +87002 POINT(41.155936684311605 74.33278560851446) bank87002 +87003 POINT(40.04926873782422 74.23285572083778) bank87003 +87004 POINT(40.21645830583068 73.3520325409067) bank87004 +87005 POINT(39.987828078317584 73.4072741254715) bank87005 +87006 POINT(41.56189773774553 74.3852021380781) bank87006 +87007 POINT(40.76043656196239 74.82462253325906) bank87007 +87008 POINT(41.31677785789652 74.3101341747246) bank87008 +87009 POINT(41.525298539299534 74.57018855454213) bank87009 +87010 POINT(39.72806549579893 73.54067632638778) bank87010 +87011 POINT(40.98432896960383 74.350072531858) bank87011 +87012 POINT(41.35140508753343 73.22737305423524) bank87012 +87013 POINT(40.548695350577276 74.74323889102267) bank87013 +87014 POINT(39.8247775839205 74.78928915484899) bank87014 +87015 POINT(40.800185464443494 73.5702314044896) bank87015 +87016 POINT(40.08226462543818 74.9340931618581) bank87016 +87017 POINT(40.03098735011302 74.93630763685826) bank87017 +87018 POINT(40.02559802468094 74.3646861843422) bank87018 +87019 POINT(41.344492299858054 74.39209861762932) bank87019 +87020 POINT(41.422612808083535 73.49085916084017) bank87020 +87021 POINT(41.50797666235647 74.42672651844423) bank87021 +87022 POINT(41.26098408730492 74.54213253173445) bank87022 +87023 POINT(39.8400515430575 73.25262766521372) bank87023 +87024 POINT(40.37451354210787 74.41015664306806) bank87024 +87025 POINT(41.63741476623027 74.96233143480904) bank87025 +87026 POINT(39.78961137878922 73.47563138803741) bank87026 +87027 POINT(41.40613503257509 73.74254370153842) bank87027 +87028 POINT(40.090780106755645 74.71786831592966) bank87028 +87029 POINT(40.61762171487008 73.11707624792064) bank87029 +87030 POINT(40.65276907975836 74.67115050954435) bank87030 +87031 POINT(40.35131879904134 73.412138719686) bank87031 +87032 POINT(40.27531053768904 73.32560162160931) bank87032 +87033 POINT(39.919390602657685 73.41634138325385) bank87033 +87034 POINT(40.72961397783428 74.30485394286737) bank87034 +87035 POINT(40.805634959115004 74.2528115944058) bank87035 +87036 POINT(40.35591453469362 73.89834451001795) bank87036 +87037 POINT(40.58891742834462 74.24283785198958) bank87037 +87038 POINT(41.63460429754461 74.73304975406595) bank87038 +87039 POINT(40.089746417715624 74.1341159042976) bank87039 +87040 POINT(40.22030508641648 73.87387164497616) bank87040 +87041 POINT(40.06995078690535 73.53688638067572) bank87041 +87042 POINT(41.31325246360336 73.96625695548103) bank87042 +87043 POINT(40.59903509181749 73.44718424241321) bank87043 +87044 POINT(40.13078853395385 73.81903076389244) bank87044 +87045 POINT(40.97921083780274 74.25777525045052) bank87045 +87046 POINT(40.80745374121547 73.20840178102104) bank87046 +87047 POINT(40.12970321910197 73.39872739376149) bank87047 +87048 POINT(40.20752389202403 74.1565208775314) bank87048 +87049 POINT(40.82481065339876 74.42131692700573) bank87049 +87050 POINT(39.85419716192746 73.01868987132936) bank87050 +87051 POINT(40.84007623081379 74.4346167812399) bank87051 +87052 POINT(39.92113174208785 73.5317208174521) bank87052 +87053 POINT(40.89583397301038 74.14903190544668) bank87053 +87054 POINT(41.27863183552452 74.72566061037298) bank87054 +87055 POINT(40.67283146759646 73.17489435934195) bank87055 +87056 POINT(40.51495052116128 74.91562778599553) bank87056 +87057 POINT(40.98562776507282 74.13927817916459) bank87057 +87058 POINT(40.72332088320941 74.05416311283291) bank87058 +87059 POINT(41.20816214333296 73.03551153295318) bank87059 +87060 POINT(40.17098776110787 74.65397542048882) bank87060 +87061 POINT(40.22698042857904 74.67919906206708) bank87061 +87062 POINT(41.4535040039368 74.41250998664421) bank87062 +87063 POINT(40.391882029528844 73.22634158893487) bank87063 +87064 POINT(39.95283460551925 74.86861798044632) bank87064 +87065 POINT(40.490099623394876 74.14974449048316) bank87065 +87066 POINT(40.50645565052806 73.96434058558505) bank87066 +87067 POINT(39.99060525870907 74.06965914476339) bank87067 +87068 POINT(40.48250950464503 74.4910339190633) bank87068 +87069 POINT(40.356119485433496 74.49004704865986) bank87069 +87070 POINT(40.84110065636426 74.75840948894326) bank87070 +87071 POINT(40.22025303608865 74.50123400184687) bank87071 +87072 POINT(41.08551799260296 74.05218019869582) bank87072 +87073 POINT(40.584407014138115 73.2207086425949) bank87073 +87074 POINT(40.92896414854966 74.30102401035028) bank87074 +87075 POINT(40.66028784642475 74.75656775760964) bank87075 +87076 POINT(41.596065474526334 73.56568245012745) bank87076 +87077 POINT(40.65871343405682 74.27854849496494) bank87077 +87078 POINT(40.441760340255186 74.30420590707192) bank87078 +87079 POINT(40.17404998261748 74.82338589497583) bank87079 +87080 POINT(41.02733434813485 73.03375042912356) bank87080 +87081 POINT(40.297393899227565 73.29344057097248) bank87081 +87082 POINT(41.212798898312556 74.53717802894157) bank87082 +87083 POINT(40.07051682766816 73.25991809565735) bank87083 +87084 POINT(40.81744947409276 73.25334723253228) bank87084 +87085 POINT(40.483847992251874 73.95289281015532) bank87085 +87086 POINT(41.02598004445603 74.00549091267297) bank87086 +87087 POINT(40.492933636196774 74.5928637519485) bank87087 +87088 POINT(40.45350744483673 73.93219158321551) bank87088 +87089 POINT(41.06941144622763 73.05681980973444) bank87089 +87090 POINT(41.499866667376956 73.49038655586433) bank87090 +87091 POINT(40.202060182604754 74.24045884875999) bank87091 +87092 POINT(40.546557859601215 73.97094039302199) bank87092 +87093 POINT(40.08466084502558 73.40305351042805) bank87093 +87094 POINT(40.41276348786397 74.19473467462079) bank87094 +87095 POINT(39.76145828125205 74.9765727849344) bank87095 +87096 POINT(41.05248855845876 73.17556540830235) bank87096 +87097 POINT(41.40250458837667 73.87981342354881) bank87097 +87098 POINT(41.70047189562761 74.31544501501828) bank87098 +87099 POINT(39.87378910884147 73.69094774355888) bank87099 +87100 POINT(40.30683560042536 73.63398292415086) bank87100 +87101 POINT(41.021883558704126 74.20447083206832) bank87101 +87102 POINT(41.54170991406919 73.60786758836983) bank87102 +87103 POINT(40.396457626534705 73.32422249532023) bank87103 +87104 POINT(39.76822721595485 74.40084417736868) bank87104 +87105 POINT(41.14625108432921 74.78211225331565) bank87105 +87106 POINT(41.15537216708242 73.5806461768819) bank87106 +87107 POINT(39.972009820547676 74.62643400563803) bank87107 +87108 POINT(40.011128641452444 74.22215518729975) bank87108 +87109 POINT(40.02224455537911 74.92648272367444) bank87109 +87110 POINT(40.789266438353735 73.78653912663748) bank87110 +87111 POINT(41.06128177738106 73.42342573288454) bank87111 +87112 POINT(40.1803999287639 74.14478797921383) bank87112 +87113 POINT(40.63595384240792 74.49026432312543) bank87113 +87114 POINT(40.403988302118655 74.63406327063646) bank87114 +87115 POINT(40.33145344355744 74.50158212720821) bank87115 +87116 POINT(39.904649076874904 73.1194070200376) bank87116 +87117 POINT(41.11292675326119 73.78515495038073) bank87117 +87118 POINT(41.211334257793524 74.61683345897842) bank87118 +87119 POINT(41.00732959878778 74.8344961401379) bank87119 +87120 POINT(39.96864688172121 73.96227437314153) bank87120 +87121 POINT(39.98824076537194 74.13584204039252) bank87121 +87122 POINT(41.46784566833867 73.447517225704) bank87122 +87123 POINT(40.70780984359661 74.39723152626078) bank87123 +87124 POINT(40.165936499949055 74.32382128180659) bank87124 +87125 POINT(40.720402467108336 73.58902884770187) bank87125 +87126 POINT(40.19424786238561 74.10899923665538) bank87126 +87127 POINT(41.44062139796851 73.88738831589083) bank87127 +87128 POINT(40.04591564868222 73.8651530772569) bank87128 +87129 POINT(40.032070179931374 74.11799576906402) bank87129 +87130 POINT(40.654258090226875 73.43028615538199) bank87130 +87131 POINT(40.261820114546765 73.1319150801553) bank87131 +87132 POINT(41.12963589248945 74.67042840307968) bank87132 +87133 POINT(41.34344774591919 74.74880854363725) bank87133 +87134 POINT(41.66733494111405 74.52123662149695) bank87134 +87135 POINT(40.73232836094352 74.1244591934629) bank87135 +87136 POINT(40.92753358867829 74.29164356725876) bank87136 +87137 POINT(41.219511069954855 73.49448033991672) bank87137 +87138 POINT(40.7964881844344 73.02060363742066) bank87138 +87139 POINT(39.83858501473961 74.4297024122886) bank87139 +87140 POINT(39.98895803495797 74.05682837390586) bank87140 +87141 POINT(41.49449433413302 73.68614807088673) bank87141 +87142 POINT(41.51877519620331 73.28215420282378) bank87142 +87143 POINT(40.46107316836201 73.78096218931942) bank87143 +87144 POINT(41.1902942198785 74.26394561207935) bank87144 +87145 POINT(41.6489590735363 74.77188222862276) bank87145 +87146 POINT(41.2282015362572 73.02370736572888) bank87146 +87147 POINT(40.003413893086126 74.37779159844034) bank87147 +87148 POINT(40.110721934515865 74.85805197982494) bank87148 +87149 POINT(40.38253664674351 73.513753719348) bank87149 +87150 POINT(40.35029222975171 74.41336612581841) bank87150 +87151 POINT(40.288973154180944 74.4732418977056) bank87151 +87152 POINT(40.911109767612764 73.2494479960038) bank87152 +87153 POINT(40.502664278804936 73.5947721348317) bank87153 +87154 POINT(41.33798256672155 74.93643516102678) bank87154 +87155 POINT(41.61930104841275 73.03953578339014) bank87155 +87156 POINT(41.324495466216405 74.49022253915626) bank87156 +87157 POINT(41.5406058552782 73.32359821986583) bank87157 +87158 POINT(41.685367876019065 73.3757262137942) bank87158 +87159 POINT(39.860449286900966 73.30532053780573) bank87159 +87160 POINT(40.44413348955684 74.75004122500272) bank87160 +87161 POINT(40.3728618613876 73.63799758756167) bank87161 +87162 POINT(40.977424178025835 73.86571484430458) bank87162 +87163 POINT(40.74075223048947 74.3136315770499) bank87163 +87164 POINT(40.11960833795928 74.04150029688256) bank87164 +87165 POINT(40.19208356807453 74.24706264119588) bank87165 +87166 POINT(41.165657236473216 74.55304490751332) bank87166 +87167 POINT(40.777704881058526 73.82514502226927) bank87167 +87168 POINT(41.00372805963822 74.0662347221832) bank87168 +87169 POINT(40.45065070376186 74.30647933026555) bank87169 +87170 POINT(40.9143478749392 73.84341547397894) bank87170 +87171 POINT(41.11444118642995 73.75079293758752) bank87171 +87172 POINT(39.750543960273696 74.94566199799007) bank87172 +87173 POINT(40.26163620634162 73.87225978778905) bank87173 +87174 POINT(41.57245668597416 74.28918657717074) bank87174 +87175 POINT(41.574487586572005 73.9972603406792) bank87175 +87176 POINT(41.1647189630871 74.74420979727687) bank87176 +87177 POINT(41.139919403695096 74.58810950784563) bank87177 +87178 POINT(41.08945749782845 74.69877488419313) bank87178 +87179 POINT(41.67706106160256 73.62528251786888) bank87179 +87180 POINT(41.347759059506366 73.13931020039031) bank87180 +87181 POINT(41.28176116844031 73.79608297132273) bank87181 +87182 POINT(41.3966059481174 74.6848629415596) bank87182 +87183 POINT(40.12893164245746 74.67079080584294) bank87183 +87184 POINT(39.933268771387056 73.81912815140885) bank87184 +87185 POINT(40.546040891951726 74.37063623698077) bank87185 +87186 POINT(41.21787452467204 74.74864703649249) bank87186 +87187 POINT(41.239034688206374 74.08415601647789) bank87187 +87188 POINT(41.68821356371229 74.41955873319878) bank87188 +87189 POINT(41.59672648782543 73.34997992607067) bank87189 +87190 POINT(40.51225807315346 74.0329829142576) bank87190 +87191 POINT(39.85940433220509 74.2804688003396) bank87191 +87192 POINT(40.299919701960704 74.68816158721081) bank87192 +87193 POINT(41.18759845630377 74.94296414520369) bank87193 +87194 POINT(41.00623247074242 73.46058703200734) bank87194 +87195 POINT(40.766628916330866 74.38881809000614) bank87195 +87196 POINT(41.655519358631416 74.51353214420719) bank87196 +87197 POINT(40.535596909351625 74.4649046687074) bank87197 +87198 POINT(40.45332101747328 74.96744996452378) bank87198 +87199 POINT(40.26372446395428 73.70154513521751) bank87199 +87200 POINT(40.531148280850054 73.86028212055497) bank87200 +87201 POINT(40.43377471414263 74.36832911788849) bank87201 +87202 POINT(41.649641189535906 73.30674523662823) bank87202 +87203 POINT(41.54736613478296 73.01500168031832) bank87203 +87204 POINT(41.562979912506535 73.57591456623604) bank87204 +87205 POINT(39.984164274808116 74.4479061547571) bank87205 +87206 POINT(40.26625344284657 74.96894175394264) bank87206 +87207 POINT(40.85205596162456 73.64027421439293) bank87207 +87208 POINT(40.47045823180034 74.30134575023634) bank87208 +87209 POINT(40.248431118646955 73.36936834706405) bank87209 +87210 POINT(41.06221572681526 74.03069244295132) bank87210 +87211 POINT(40.3716246012184 74.68288261074404) bank87211 +87212 POINT(40.67046516908088 73.91212591710494) bank87212 +87213 POINT(41.65454010952171 74.86330963573465) bank87213 +87214 POINT(41.496931403332646 74.15613878235403) bank87214 +87215 POINT(40.43318898532599 74.62266970044308) bank87215 +87216 POINT(40.92869160808658 73.04193716289244) bank87216 +87217 POINT(40.28811870577688 73.37170451794927) bank87217 +87218 POINT(41.05898759508874 74.16665143758313) bank87218 +87219 POINT(40.156739411291234 74.19786406803556) bank87219 +87220 POINT(39.87993201497215 73.04277336074735) bank87220 +87221 POINT(40.08422308518161 74.37560253685015) bank87221 +87222 POINT(41.58589624767954 73.39714554426445) bank87222 +87223 POINT(40.82751001247015 74.38901695024748) bank87223 +87224 POINT(40.094878805723624 74.85128789321807) bank87224 +87225 POINT(40.673359198629825 74.00185117763479) bank87225 +87226 POINT(40.27702764629694 73.79447822458563) bank87226 +87227 POINT(39.73935062857805 74.20294311885438) bank87227 +87228 POINT(41.40156053507935 74.73777352793292) bank87228 +87229 POINT(40.72670486330201 74.68249047480745) bank87229 +87230 POINT(40.61388034737472 74.7656213496378) bank87230 +87231 POINT(41.373366530568056 73.22889648262262) bank87231 +87232 POINT(41.598023012550556 73.26274683435658) bank87232 +87233 POINT(40.43130521346165 73.4061803069309) bank87233 +87234 POINT(39.95892467112134 73.76514601124914) bank87234 +87235 POINT(40.94264871814554 74.05501650789432) bank87235 +87236 POINT(41.10709104601444 73.82292266773874) bank87236 +87237 POINT(41.66151810546255 73.48416338691928) bank87237 +87238 POINT(41.27320786177715 74.62549247575046) bank87238 +87239 POINT(40.844120920329985 74.71456495716056) bank87239 +87240 POINT(40.5732131639033 73.20869846684796) bank87240 +87241 POINT(41.20821423244623 73.87128770320598) bank87241 +87242 POINT(41.58667051412566 74.45286359821304) bank87242 +87243 POINT(40.27376687577027 74.51920685431163) bank87243 +87244 POINT(41.25176068212111 74.73541724377783) bank87244 +87245 POINT(40.23361416504071 73.58069600962668) bank87245 +87246 POINT(39.8755673733194 74.92156923751037) bank87246 +87247 POINT(40.54557109878705 73.4491300654498) bank87247 +87248 POINT(39.971978930435455 74.16805329520437) bank87248 +87249 POINT(41.34157925194231 73.75281137386285) bank87249 +87250 POINT(41.34223968854273 74.84866856181722) bank87250 +87251 POINT(41.425994973110704 73.56565388255383) bank87251 +87252 POINT(41.69934771300263 74.71255763402657) bank87252 +87253 POINT(40.809375262702105 73.35169700141263) bank87253 +87254 POINT(40.69485321442673 74.62123050019866) bank87254 +87255 POINT(41.268703777796254 73.12372861195782) bank87255 +87256 POINT(41.071124181752914 73.4240943456883) bank87256 +87257 POINT(41.053946615375345 74.52279491921189) bank87257 +87258 POINT(40.81344609599176 73.59301167813784) bank87258 +87259 POINT(41.253341156323266 73.80859907200006) bank87259 +87260 POINT(39.74701618114279 73.28332385464309) bank87260 +87261 POINT(41.07286752217339 74.98275642423538) bank87261 +87262 POINT(39.93688658491602 74.08904036064415) bank87262 +87263 POINT(40.788655868808384 73.63704357910075) bank87263 +87264 POINT(40.98444678597135 74.61841579867759) bank87264 +87265 POINT(39.87964981254359 73.01948848623404) bank87265 +87266 POINT(40.473804194199644 73.06306586986804) bank87266 +87267 POINT(40.84397261150577 73.49903023931371) bank87267 +87268 POINT(41.06741614117038 73.91690200625382) bank87268 +87269 POINT(40.544105708890385 74.91560812421187) bank87269 +87270 POINT(41.55304507041082 74.29382093682011) bank87270 +87271 POINT(39.89413924345336 74.12865202004193) bank87271 +87272 POINT(41.00014456976639 73.92016258810317) bank87272 +87273 POINT(41.13405031472471 74.26569500607) bank87273 +87274 POINT(40.19187291862613 74.3356841529228) bank87274 +87275 POINT(40.907212054084354 73.29191227224287) bank87275 +87276 POINT(41.09249753055941 73.19257234415294) bank87276 +87277 POINT(39.83250361401612 74.75469154378317) bank87277 +87278 POINT(40.09947720358857 74.73222024162418) bank87278 +87279 POINT(41.215785158870055 74.15985500598089) bank87279 +87280 POINT(41.46266895892781 73.87510080790979) bank87280 +87281 POINT(40.23320328634895 73.72260485336645) bank87281 +87282 POINT(40.95722637114676 74.47276692968767) bank87282 +87283 POINT(40.97958515262529 74.95829978743464) bank87283 +87284 POINT(39.82824494209914 74.53239796965731) bank87284 +87285 POINT(40.01394146353472 74.56873638734007) bank87285 +87286 POINT(39.88451842737778 73.3829494177495) bank87286 +87287 POINT(40.15277940353337 73.73474422736535) bank87287 +87288 POINT(41.41435495203809 74.67230865219976) bank87288 +87289 POINT(40.727757813317815 73.85945307920095) bank87289 +87290 POINT(40.56382956250431 73.20771070928706) bank87290 +87291 POINT(41.168765944756714 74.57367801655838) bank87291 +87292 POINT(41.28442491790975 73.39182315290319) bank87292 +87293 POINT(39.880498151929295 74.93135168503423) bank87293 +87294 POINT(41.65846986871268 73.21818724762866) bank87294 +87295 POINT(40.62401750174046 74.95054573014303) bank87295 +87296 POINT(40.787842630686605 73.94665148929357) bank87296 +87297 POINT(39.80745853498803 74.50729936575387) bank87297 +87298 POINT(39.87105569030273 74.557053345453) bank87298 +87299 POINT(41.23660564895731 74.08329892039026) bank87299 +87300 POINT(40.61796507359459 73.43098024259936) bank87300 +87301 POINT(40.28248150537048 73.39956340191083) bank87301 +87302 POINT(40.6836718169287 74.26711722729787) bank87302 +87303 POINT(41.39809180469381 73.19548833164657) bank87303 +87304 POINT(41.27355236316219 74.92550892297987) bank87304 +87305 POINT(40.05668460930339 74.82995922492894) bank87305 +87306 POINT(40.427674964139094 73.86923048135806) bank87306 +87307 POINT(41.32411023496233 73.33060637898498) bank87307 +87308 POINT(41.53823506308861 73.96780011255974) bank87308 +87309 POINT(40.15916835910157 73.1118954768666) bank87309 +87310 POINT(40.147918168251664 73.72140778702845) bank87310 +87311 POINT(40.13397091359073 74.55474305253243) bank87311 +87312 POINT(40.502242978824604 73.15406475005496) bank87312 +87313 POINT(41.021626636084335 74.22679854428932) bank87313 +87314 POINT(39.76533956482361 73.30806921024804) bank87314 +87315 POINT(41.29212153932151 74.70091640588983) bank87315 +87316 POINT(40.804585419161796 73.71800446019951) bank87316 +87317 POINT(40.58339915869551 74.93562543168397) bank87317 +87318 POINT(39.745514900580524 73.68657023889244) bank87318 +87319 POINT(41.5715668207953 74.2760344088833) bank87319 +87320 POINT(40.022748464321 73.98346996158752) bank87320 +87321 POINT(40.141874679995574 74.47208197063946) bank87321 +87322 POINT(40.08879258930867 74.9494651117091) bank87322 +87323 POINT(41.41553783230274 73.23156700215588) bank87323 +87324 POINT(40.03478417853703 73.83174285504906) bank87324 +87325 POINT(39.90184982352123 74.37013209320786) bank87325 +87326 POINT(40.17229919516008 74.14727826101311) bank87326 +87327 POINT(40.39063648597446 73.18129023992559) bank87327 +87328 POINT(41.41188210614637 74.83321220014707) bank87328 +87329 POINT(39.78770198088899 74.99986406187193) bank87329 +87330 POINT(41.00807228538795 73.5688209019007) bank87330 +87331 POINT(40.76293910342528 74.20081393941123) bank87331 +87332 POINT(40.99131124872167 74.45752664700319) bank87332 +87333 POINT(40.22984907268184 73.5577846651354) bank87333 +87334 POINT(41.17216515476485 74.23197039009317) bank87334 +87335 POINT(40.05474258102826 74.75883339963855) bank87335 +87336 POINT(41.66446653167444 74.27793188800872) bank87336 +87337 POINT(41.556930607780345 74.09935060751101) bank87337 +87338 POINT(40.23984249529456 74.27855107155578) bank87338 +87339 POINT(40.40415332157451 74.11541551456507) bank87339 +87340 POINT(40.137786822144825 74.51732116474366) bank87340 +87341 POINT(41.457498457249 74.48315580676439) bank87341 +87342 POINT(39.75944820893126 74.08508921061606) bank87342 +87343 POINT(39.961635100403605 74.0401783417237) bank87343 +87344 POINT(40.352487845596315 74.27400847069858) bank87344 +87345 POINT(40.451783999678355 74.67424644299342) bank87345 +87346 POINT(40.75803770528106 73.68965924375335) bank87346 +87347 POINT(41.3950224756231 73.25503656301464) bank87347 +87348 POINT(39.888168971874975 73.82396755470154) bank87348 +87349 POINT(40.74512662614182 74.44739440990998) bank87349 +87350 POINT(41.128174544766104 73.5139814411448) bank87350 +87351 POINT(40.621276254439415 73.68917869101517) bank87351 +87352 POINT(41.66780350127843 73.01047037204061) bank87352 +87353 POINT(39.996221353269554 73.70442915923687) bank87353 +87354 POINT(41.03517515428727 74.50299328988208) bank87354 +87355 POINT(41.51601233799449 74.59402091932942) bank87355 +87356 POINT(40.92289949256826 73.62061401750232) bank87356 +87357 POINT(40.31716888425962 74.4226139569705) bank87357 +87358 POINT(40.81202069192636 73.89836610720623) bank87358 +87359 POINT(40.09882047082858 74.46395339457723) bank87359 +87360 POINT(40.68354124503408 73.14011554051244) bank87360 +87361 POINT(41.387353465655934 73.01615669222414) bank87361 +87362 POINT(40.30772474083857 74.19229139641914) bank87362 +87363 POINT(39.93099087269142 74.61962774744896) bank87363 +87364 POINT(40.362543841653334 74.53896101099639) bank87364 +87365 POINT(41.697692627043665 74.34431677234585) bank87365 +87366 POINT(41.18797638615114 73.71309870113954) bank87366 +87367 POINT(39.9076626760159 74.31727580974153) bank87367 +87368 POINT(39.78142977567871 73.12431315756763) bank87368 +87369 POINT(40.62423549076002 73.67323224423228) bank87369 +87370 POINT(40.2617597597344 73.20084895301248) bank87370 +87371 POINT(41.70674556329727 74.32825092518229) bank87371 +87372 POINT(41.1797867364982 74.29774244770461) bank87372 +87373 POINT(40.75349329705578 74.885221635067) bank87373 +87374 POINT(41.16063064158676 73.93011172414568) bank87374 +87375 POINT(41.188656116237446 74.34392065120521) bank87375 +87376 POINT(41.67849154696255 74.74635151239481) bank87376 +87377 POINT(40.303479133483656 74.40518967902906) bank87377 +87378 POINT(40.25128181501531 73.61791656151847) bank87378 +87379 POINT(41.378462243334 73.5426273578372) bank87379 +87380 POINT(41.285438991794244 73.13909123033609) bank87380 +87381 POINT(40.314845742357846 74.45754703534774) bank87381 +87382 POINT(40.79280755203329 73.93858317809952) bank87382 +87383 POINT(40.1964818067074 74.74525570607047) bank87383 +87384 POINT(40.12763913504736 73.62769567548914) bank87384 +87385 POINT(40.387704720471795 73.27000309467896) bank87385 +87386 POINT(41.48129218584627 74.52434255768132) bank87386 +87387 POINT(39.91097788784749 74.58160017223585) bank87387 +87388 POINT(41.381818990718976 73.26491781878735) bank87388 +87389 POINT(40.31905372364701 74.03014238253893) bank87389 +87390 POINT(40.81860052393362 73.48653491230253) bank87390 +87391 POINT(40.10804771223224 74.90190951329531) bank87391 +87392 POINT(41.5940282241071 74.798174344582) bank87392 +87393 POINT(40.432040049431095 74.1455423689423) bank87393 +87394 POINT(40.490375779731195 73.11997264330145) bank87394 +87395 POINT(39.783401462586326 74.66882916831212) bank87395 +87396 POINT(40.26892768388594 73.2997942817182) bank87396 +87397 POINT(40.93439892233552 74.84359658865004) bank87397 +87398 POINT(39.82052630942022 73.56728019438421) bank87398 +87399 POINT(40.886525942452 73.85295918420871) bank87399 +87400 POINT(41.46512077329362 74.1404470732252) bank87400 +87401 POINT(40.959360544576654 73.88593949165413) bank87401 +87402 POINT(39.82349371836714 74.29793061407712) bank87402 +87403 POINT(41.14266979503149 74.29232642201697) bank87403 +87404 POINT(39.78739881919414 73.19033180620202) bank87404 +87405 POINT(41.18635062480054 73.4778974031277) bank87405 +87406 POINT(41.51871961287977 74.47401949432545) bank87406 +87407 POINT(40.26862065570529 73.7659757772584) bank87407 +87408 POINT(41.46669119936059 73.75494099346048) bank87408 +87409 POINT(41.50903427564115 73.03385618601072) bank87409 +87410 POINT(40.66295380145628 74.55393240205295) bank87410 +87411 POINT(40.91478232571776 74.59175039134591) bank87411 +87412 POINT(40.77356949886833 73.42852171837873) bank87412 +87413 POINT(40.15370589734835 73.83380477047858) bank87413 +87414 POINT(41.10073214252151 73.18835229219825) bank87414 +87415 POINT(39.761200840524374 74.56884933825515) bank87415 +87416 POINT(40.175896075721724 74.27487263782982) bank87416 +87417 POINT(41.51484467489097 74.3173452119021) bank87417 +87418 POINT(41.13270559166721 73.82201926120361) bank87418 +87419 POINT(41.52993578017565 73.0509962487462) bank87419 +87420 POINT(40.350660789365975 73.11348992903389) bank87420 +87421 POINT(40.141019223112465 74.83628795818515) bank87421 +87422 POINT(40.34235218517006 74.46269224183615) bank87422 +87423 POINT(41.348862839345585 74.20412343649193) bank87423 +87424 POINT(39.80421792519289 74.30947092048999) bank87424 +87425 POINT(41.679588980291264 74.66941155828314) bank87425 +87426 POINT(41.4887927147923 73.49314462992363) bank87426 +87427 POINT(41.48085783291496 74.06995049725683) bank87427 +87428 POINT(40.47984951160903 74.36004802356445) bank87428 +87429 POINT(40.09809123668731 73.0971718136415) bank87429 +87430 POINT(40.95311695875637 74.84932255053583) bank87430 +87431 POINT(41.57862575479537 74.04507985641499) bank87431 +87432 POINT(41.17917064937153 74.95221229723082) bank87432 +87433 POINT(41.413250266545504 73.33434317062552) bank87433 +87434 POINT(40.962104658453356 74.44583820413781) bank87434 +87435 POINT(40.63192420824767 73.14731325897802) bank87435 +87436 POINT(40.19491824523301 74.45520573908968) bank87436 +87437 POINT(39.911892689265486 74.86148875980689) bank87437 +87438 POINT(41.22581581275965 74.7600785926602) bank87438 +87439 POINT(41.231107282880416 74.8685496221974) bank87439 +87440 POINT(40.61038842019806 74.93429182420573) bank87440 +87441 POINT(39.71715111350105 74.34722314477524) bank87441 +87442 POINT(41.378937907323696 73.05225577025864) bank87442 +87443 POINT(41.3850510014375 74.86656849681313) bank87443 +87444 POINT(41.354274827186416 74.40840475068413) bank87444 +87445 POINT(41.54032027642419 74.55189394251916) bank87445 +87446 POINT(41.040000429270755 73.33288440477905) bank87446 +87447 POINT(40.80262056495321 74.53414291025027) bank87447 +87448 POINT(40.990469543762785 73.69449067470792) bank87448 +87449 POINT(41.2848821577389 73.6772375680539) bank87449 +87450 POINT(40.8909778944977 73.61584030188061) bank87450 +87451 POINT(41.52305876091175 74.92510624577864) bank87451 +87452 POINT(41.62733633411268 73.67382889668364) bank87452 +87453 POINT(40.71128829265146 74.92661716405193) bank87453 +87454 POINT(41.458664513976004 73.35243509893336) bank87454 +87455 POINT(41.218240190076735 73.0948584344817) bank87455 +87456 POINT(40.85048253368732 73.58830607439312) bank87456 +87457 POINT(40.89845112757779 73.29239683581702) bank87457 +87458 POINT(39.983511535883245 73.37038687401785) bank87458 +87459 POINT(41.18751313342526 74.1078532996421) bank87459 +87460 POINT(41.473467090993 74.1356204593881) bank87460 +87461 POINT(41.43968663089078 73.17673686104376) bank87461 +87462 POINT(41.05668601016766 73.53183154875794) bank87462 +87463 POINT(40.99099624708742 74.90548791657287) bank87463 +87464 POINT(40.655086546292104 74.30675207818658) bank87464 +87465 POINT(40.46114465822332 74.61622676635866) bank87465 +87466 POINT(41.46968483177428 73.09903674797022) bank87466 +87467 POINT(40.401658034124615 73.4634352067403) bank87467 +87468 POINT(40.547151445976475 74.74377242928888) bank87468 +87469 POINT(41.242175377951106 73.60353821467226) bank87469 +87470 POINT(40.565298013109 74.00198675838774) bank87470 +87471 POINT(41.49130416990926 74.99396102637819) bank87471 +87472 POINT(41.07226215069473 74.63226134698628) bank87472 +87473 POINT(40.56314946359875 74.83686744398953) bank87473 +87474 POINT(40.92962806777175 73.98271046902745) bank87474 +87475 POINT(40.54335338040339 73.1622793831926) bank87475 +87476 POINT(40.27927573506893 74.83270770382458) bank87476 +87477 POINT(41.60014734290727 74.79876031413687) bank87477 +87478 POINT(40.8192818681988 74.52284840353407) bank87478 +87479 POINT(41.68126997767641 74.6067952803387) bank87479 +87480 POINT(41.094283709813745 74.83212647808178) bank87480 +87481 POINT(41.563080501055985 74.65713607585616) bank87481 +87482 POINT(41.44984733095624 74.18822544810298) bank87482 +87483 POINT(41.031813694984976 73.65947892658116) bank87483 +87484 POINT(39.952400098879615 73.97622183086642) bank87484 +87485 POINT(40.44853599417058 73.09469964041499) bank87485 +87486 POINT(41.046418740122874 74.91617236969196) bank87486 +87487 POINT(40.502613659431205 74.53405676219894) bank87487 +87488 POINT(41.050225623453436 73.72530749693257) bank87488 +87489 POINT(40.8842130432847 74.09827188802794) bank87489 +87490 POINT(40.95992123326536 73.80261479761683) bank87490 +87491 POINT(40.97110346801465 73.6379319490175) bank87491 +87492 POINT(40.01768766328978 74.90710171427851) bank87492 +87493 POINT(40.060165359351515 73.77119748425149) bank87493 +87494 POINT(41.202963574600695 74.31013939705286) bank87494 +87495 POINT(39.759500532770076 74.74776661121984) bank87495 +87496 POINT(40.08471959328122 73.75110190246835) bank87496 +87497 POINT(41.18759708923903 74.14327916831951) bank87497 +87498 POINT(40.02822001626639 74.77470735581377) bank87498 +87499 POINT(41.32395002716605 73.36034881407404) bank87499 +87500 POINT(40.191085300586565 73.9840134548774) bank87500 +87501 POINT(40.542788134099574 74.12453124376327) bank87501 +87502 POINT(40.859714213253426 74.5789412361153) bank87502 +87503 POINT(39.983416962566224 74.61358690587525) bank87503 +87504 POINT(40.253585225487804 73.6313098203631) bank87504 +87505 POINT(41.164971017155366 74.29269343241909) bank87505 +87506 POINT(40.642334757353986 74.13334244209832) bank87506 +87507 POINT(40.35057726385654 73.9884841356109) bank87507 +87508 POINT(41.30630502008653 73.41466849277869) bank87508 +87509 POINT(41.129910326019676 73.05993792858541) bank87509 +87510 POINT(40.31915125658966 74.24907991723077) bank87510 +87511 POINT(40.696728166663824 73.03181800084155) bank87511 +87512 POINT(40.498093993923966 73.40043924120664) bank87512 +87513 POINT(41.08286767865974 73.9335105971055) bank87513 +87514 POINT(41.68338770982191 73.93279998380503) bank87514 +87515 POINT(40.51566305286876 74.34943673261563) bank87515 +87516 POINT(40.678947619275746 74.16206336808328) bank87516 +87517 POINT(41.474117438664024 74.1552986052544) bank87517 +87518 POINT(41.13180282068392 74.55219687171862) bank87518 +87519 POINT(40.719198437169204 74.0832399036249) bank87519 +87520 POINT(41.0337124374842 74.61319916779298) bank87520 +87521 POINT(41.70152200831126 73.2638824790698) bank87521 +87522 POINT(41.22855933724631 73.25710519715122) bank87522 +87523 POINT(41.11607994455315 73.09236829901504) bank87523 +87524 POINT(41.308925763397866 73.37643971850066) bank87524 +87525 POINT(41.10429485784642 74.3527202904753) bank87525 +87526 POINT(41.24176560613573 73.4905882047743) bank87526 +87527 POINT(41.25009836677473 74.59289118731735) bank87527 +87528 POINT(41.540262265933144 73.08835817872725) bank87528 +87529 POINT(40.427364022116755 73.80993437410804) bank87529 +87530 POINT(41.26868446216038 74.09134710300344) bank87530 +87531 POINT(40.84329214276798 73.58441416724231) bank87531 +87532 POINT(41.59989613560392 74.43177690532345) bank87532 +87533 POINT(41.534114940341 73.29207337730942) bank87533 +87534 POINT(40.01504461902367 73.94811851702791) bank87534 +87535 POINT(40.769632225647776 73.00919684278476) bank87535 +87536 POINT(41.42518545341095 75.00146792189547) bank87536 +87537 POINT(39.94747016102933 74.72951990619511) bank87537 +87538 POINT(40.871446776386996 73.9075692873729) bank87538 +87539 POINT(40.02145707724721 73.91389834191665) bank87539 +87540 POINT(41.3130000135159 73.41250098220675) bank87540 +87541 POINT(39.761403660206135 73.63942321237448) bank87541 +87542 POINT(41.60297377489469 73.52900933686364) bank87542 +87543 POINT(41.64113714551146 74.40058656006902) bank87543 +87544 POINT(40.348261709276244 73.68300731814912) bank87544 +87545 POINT(39.720314322863295 74.99615207193347) bank87545 +87546 POINT(40.0812148145413 74.24549161111294) bank87546 +87547 POINT(40.72233994693631 73.71090392713421) bank87547 +87548 POINT(40.477565589715 73.94248405282733) bank87548 +87549 POINT(40.2541840611734 74.01522078345086) bank87549 +87550 POINT(40.85642120484988 73.31583966085539) bank87550 +87551 POINT(39.900437972904115 74.09411471659512) bank87551 +87552 POINT(41.202668089352365 73.62097363786751) bank87552 +87553 POINT(40.129894272064824 74.41785423932392) bank87553 +87554 POINT(41.02623465022967 73.63916190680158) bank87554 +87555 POINT(41.28161868103916 74.69539791068466) bank87555 +87556 POINT(41.065246047274215 74.52403019741466) bank87556 +87557 POINT(40.499834742304564 73.42770278978557) bank87557 +87558 POINT(40.388338422969355 74.61037573095854) bank87558 +87559 POINT(40.183029590042146 74.12607173881047) bank87559 +87560 POINT(40.64779178976307 73.4039155862061) bank87560 +87561 POINT(40.77852219716118 73.16064866747136) bank87561 +87562 POINT(39.74901125639792 73.46561476361269) bank87562 +87563 POINT(41.11847027420627 74.19682112900306) bank87563 +87564 POINT(41.57448250235465 73.12357990841822) bank87564 +87565 POINT(39.79238573003574 73.69325942161045) bank87565 +87566 POINT(41.61122752112586 74.63291241151867) bank87566 +87567 POINT(41.14017998200666 74.2702520259865) bank87567 +87568 POINT(40.66123071303089 74.93574702296048) bank87568 +87569 POINT(41.32288504137922 73.53197194102476) bank87569 +87570 POINT(39.98097689110564 73.94164084712335) bank87570 +87571 POINT(40.84435351667724 74.01218827463634) bank87571 +87572 POINT(41.096704840326524 73.9371294124533) bank87572 +87573 POINT(40.79239364424535 74.96346559518271) bank87573 +87574 POINT(40.50937768583976 74.6057581985533) bank87574 +87575 POINT(41.36293300833746 73.59569463812603) bank87575 +87576 POINT(39.87626304265141 73.71248388085269) bank87576 +87577 POINT(41.53850011430876 74.7533073852637) bank87577 +87578 POINT(40.64303747474289 74.51463396010675) bank87578 +87579 POINT(40.75165761609935 73.9138790401434) bank87579 +87580 POINT(40.280127086923734 74.46875841556741) bank87580 +87581 POINT(40.937968030672195 73.49112285032547) bank87581 +87582 POINT(41.56605639600753 74.79243098031418) bank87582 +87583 POINT(41.44600285598009 74.62319225566775) bank87583 +87584 POINT(39.848149880025765 74.83737317764879) bank87584 +87585 POINT(41.518041797089495 73.35417199537815) bank87585 +87586 POINT(40.00157719834244 74.2175079666605) bank87586 +87587 POINT(40.539447551959285 74.64963680476616) bank87587 +87588 POINT(39.783432138658775 73.31978281679385) bank87588 +87589 POINT(41.43607569506994 74.38589132970131) bank87589 +87590 POINT(40.063134207163834 73.93737506712273) bank87590 +87591 POINT(41.47318641612907 74.46912066042293) bank87591 +87592 POINT(40.09028886543405 74.09388248228794) bank87592 +87593 POINT(39.75559677715047 73.67980719121636) bank87593 +87594 POINT(39.88061504619777 73.57972334653418) bank87594 +87595 POINT(40.41494797938165 73.90285311875662) bank87595 +87596 POINT(39.73013435991114 73.51939379004516) bank87596 +87597 POINT(41.520552611197246 74.56861240470519) bank87597 +87598 POINT(41.34842735810098 74.82533356495794) bank87598 +87599 POINT(40.01601120522643 74.5474785399316) bank87599 +87600 POINT(40.35448689667406 74.07859069189344) bank87600 +87601 POINT(40.869126109481975 74.030944592918) bank87601 +87602 POINT(40.63291719158388 74.05109906336061) bank87602 +87603 POINT(40.4909554885282 74.94884775614979) bank87603 +87604 POINT(40.69881864838257 73.27018019806165) bank87604 +87605 POINT(40.23775072805564 74.89031568515237) bank87605 +87606 POINT(41.37945280641347 73.49163622249304) bank87606 +87607 POINT(40.631037372154445 74.66623425938698) bank87607 +87608 POINT(40.94371320925038 73.83640515215902) bank87608 +87609 POINT(40.77064049626247 74.83955813146783) bank87609 +87610 POINT(40.96511596379098 73.54988665189461) bank87610 +87611 POINT(39.92608977866923 73.61945543562538) bank87611 +87612 POINT(41.52023914817073 73.19189053982144) bank87612 +87613 POINT(40.13161606895438 74.49338633393347) bank87613 +87614 POINT(40.433445133922774 74.71075147066492) bank87614 +87615 POINT(40.634606328272824 74.92432380920181) bank87615 +87616 POINT(41.126381363319595 73.07188777376247) bank87616 +87617 POINT(41.547707421703954 74.93732827878029) bank87617 +87618 POINT(41.49205264911994 74.58160503530586) bank87618 +87619 POINT(40.893883589857325 73.22569191459958) bank87619 +87620 POINT(41.67498909993439 73.22299812640495) bank87620 +87621 POINT(40.39245405168115 73.75434213966871) bank87621 +87622 POINT(41.28460799616287 73.53419988664164) bank87622 +87623 POINT(40.56130903197162 74.47785755312782) bank87623 +87624 POINT(41.64745459755058 74.88477213631565) bank87624 +87625 POINT(41.15585309902304 74.31795978319971) bank87625 +87626 POINT(40.27774562300476 73.96570944398016) bank87626 +87627 POINT(41.62098143423981 74.11046261692044) bank87627 +87628 POINT(40.478892467115244 74.61269428756604) bank87628 +87629 POINT(40.3272602939915 74.40717223488093) bank87629 +87630 POINT(41.45231003221524 73.1800865412993) bank87630 +87631 POINT(40.066974274307746 73.39253113694552) bank87631 +87632 POINT(40.73029170878789 73.17829886430216) bank87632 +87633 POINT(39.73369409739135 73.36648558839264) bank87633 +87634 POINT(41.52399376492191 74.90135283318898) bank87634 +87635 POINT(40.16367603927001 74.3933681953) bank87635 +87636 POINT(41.10529860302327 73.8763061432157) bank87636 +87637 POINT(40.88003710134939 74.91034038683802) bank87637 +87638 POINT(41.15271068162472 73.41292861917454) bank87638 +87639 POINT(40.737105463445324 74.40622526277676) bank87639 +87640 POINT(40.6299647719776 74.99791787578323) bank87640 +87641 POINT(41.710881810028965 73.47934107096616) bank87641 +87642 POINT(40.875410585242264 73.59577213648176) bank87642 +87643 POINT(40.30236008941422 73.84026119138987) bank87643 +87644 POINT(41.53860162802084 74.78997419260826) bank87644 +87645 POINT(41.36110416446582 74.92551331945066) bank87645 +87646 POINT(40.15064588648576 73.02965321534512) bank87646 +87647 POINT(41.48064585894196 74.9314002917946) bank87647 +87648 POINT(41.11984688005105 73.50969720072776) bank87648 +87649 POINT(41.230700438020456 73.54457750312677) bank87649 +87650 POINT(40.78173188382496 74.70352798694283) bank87650 +87651 POINT(41.45074012421671 74.25694995228972) bank87651 +87652 POINT(41.101907371344474 73.68299570196973) bank87652 +87653 POINT(40.77441171613857 74.92505648178788) bank87653 +87654 POINT(40.3602716869541 74.4072372162368) bank87654 +87655 POINT(41.567394107700565 73.13946120257977) bank87655 +87656 POINT(39.85260763446081 73.01075867346516) bank87656 +87657 POINT(41.13895601816944 74.64924908544864) bank87657 +87658 POINT(41.005807914969665 74.63872291751835) bank87658 +87659 POINT(40.98046389206656 73.97205420644451) bank87659 +87660 POINT(41.661673829215516 74.79762889455276) bank87660 +87661 POINT(40.2290696900059 74.39840575077802) bank87661 +87662 POINT(40.41036472157939 74.49047461893102) bank87662 +87663 POINT(41.51307680936289 74.40810192145874) bank87663 +87664 POINT(41.079002387366664 74.51414735545062) bank87664 +87665 POINT(40.348501774450064 74.96045418049393) bank87665 +87666 POINT(41.64736551608841 74.35954621719293) bank87666 +87667 POINT(39.87053797159079 73.62416125543527) bank87667 +87668 POINT(40.950773946522304 74.85110797092699) bank87668 +87669 POINT(40.08201319291026 74.80700013065358) bank87669 +87670 POINT(41.168947025266284 73.96290946793114) bank87670 +87671 POINT(41.49024723120751 73.69276183620131) bank87671 +87672 POINT(40.581716458404394 74.06481321003932) bank87672 +87673 POINT(41.4547500565621 74.39471472371866) bank87673 +87674 POINT(40.44976810179743 73.65688780310842) bank87674 +87675 POINT(40.00511485819891 73.4025498974946) bank87675 +87676 POINT(41.3926570331172 74.03104413653628) bank87676 +87677 POINT(40.83899522137352 73.75495084171312) bank87677 +87678 POINT(41.6632002838805 73.74635664109105) bank87678 +87679 POINT(40.45549810844512 74.02332008965385) bank87679 +87680 POINT(41.308755192138406 74.1662432308645) bank87680 +87681 POINT(40.2595466882391 74.20545611264893) bank87681 +87682 POINT(39.805367348967266 74.31162399037484) bank87682 +87683 POINT(39.890806219047924 73.59633376074113) bank87683 +87684 POINT(41.42116893251328 73.19744044843931) bank87684 +87685 POINT(40.79307011571652 74.03625741078676) bank87685 +87686 POINT(40.588440692043484 74.20647466553007) bank87686 +87687 POINT(40.986308799030105 74.14667318579482) bank87687 +87688 POINT(40.26619435263203 73.78691149371694) bank87688 +87689 POINT(40.211958845038204 73.5967663600004) bank87689 +87690 POINT(41.20319130823845 74.40754369143855) bank87690 +87691 POINT(40.99959885226602 74.41188328121957) bank87691 +87692 POINT(41.495519629905324 74.64462992411386) bank87692 +87693 POINT(40.892574643667004 73.43293674053764) bank87693 +87694 POINT(41.03649367885475 73.38023455428397) bank87694 +87695 POINT(41.06419536117417 74.5487325724316) bank87695 +87696 POINT(40.26394097028823 74.15183652723758) bank87696 +87697 POINT(40.102691825789755 73.70315229017775) bank87697 +87698 POINT(41.29609451514545 73.83138312854919) bank87698 +87699 POINT(41.056534973607704 73.62343631302299) bank87699 +87700 POINT(41.31545796415801 73.76641082088435) bank87700 +87701 POINT(40.84689291982656 74.08817600886326) bank87701 +87702 POINT(41.147431244553594 73.14045057137957) bank87702 +87703 POINT(41.14024627246312 74.18161163812461) bank87703 +87704 POINT(40.65116812827521 73.66000433904262) bank87704 +87705 POINT(39.83595376497384 73.2894828500768) bank87705 +87706 POINT(41.628872185074115 74.50016249741404) bank87706 +87707 POINT(40.18789398226599 73.33136820253677) bank87707 +87708 POINT(40.39072826204532 74.26115805621278) bank87708 +87709 POINT(40.364953792786714 74.54301199043107) bank87709 +87710 POINT(40.37060426320015 74.01408075128116) bank87710 +87711 POINT(40.26894758682457 73.506239350667) bank87711 +87712 POINT(41.269686444200964 73.36693251414152) bank87712 +87713 POINT(41.64398312788583 74.60846962891542) bank87713 +87714 POINT(39.94965246470043 73.46140409837315) bank87714 +87715 POINT(40.822919423053015 74.70322813750013) bank87715 +87716 POINT(40.931527156893374 73.45875137678826) bank87716 +87717 POINT(41.124175303004336 74.44758891790178) bank87717 +87718 POINT(41.426644416855 74.41190644784707) bank87718 +87719 POINT(41.65502731071832 74.4156174862177) bank87719 +87720 POINT(41.32529208274723 74.66803150896875) bank87720 +87721 POINT(41.56418411093683 73.0637188938599) bank87721 +87722 POINT(39.896257946202866 74.44600658293312) bank87722 +87723 POINT(41.47684561887045 73.2788831815372) bank87723 +87724 POINT(41.01190254160473 74.49707629581754) bank87724 +87725 POINT(41.51971186652351 73.55301180229944) bank87725 +87726 POINT(40.191949642910174 73.31744358230915) bank87726 +87727 POINT(41.35588019459647 74.69244190643109) bank87727 +87728 POINT(39.90329538801451 73.80201940809337) bank87728 +87729 POINT(41.56733554100484 73.8839691609379) bank87729 +87730 POINT(39.71967639022442 74.84907982817491) bank87730 +87731 POINT(40.90565764769605 73.81540006900703) bank87731 +87732 POINT(40.82640684904622 74.24180260520015) bank87732 +87733 POINT(40.38745881788464 74.19586697344737) bank87733 +87734 POINT(41.69399431347615 74.28022194574405) bank87734 +87735 POINT(40.83612323271207 73.3930729040747) bank87735 +87736 POINT(39.97002682666779 74.21824150665489) bank87736 +87737 POINT(39.86872443099545 74.34112049911089) bank87737 +87738 POINT(40.05836168252039 73.6374446852498) bank87738 +87739 POINT(40.64852576327012 74.59236859194371) bank87739 +87740 POINT(41.119582267172596 74.78058961924457) bank87740 +87741 POINT(41.13216943282282 74.05992493981034) bank87741 +87742 POINT(39.869079094319474 74.25609461989069) bank87742 +87743 POINT(41.325446826770204 73.27401936869951) bank87743 +87744 POINT(40.08200009395312 74.22542684610664) bank87744 +87745 POINT(40.86228669878767 73.67063496922869) bank87745 +87746 POINT(40.965245461707624 74.80355044750863) bank87746 +87747 POINT(40.033326709559624 73.2039096803361) bank87747 +87748 POINT(40.35178894041465 74.35244676944896) bank87748 +87749 POINT(39.73539207251188 74.4188680315735) bank87749 +87750 POINT(41.62520403189889 74.21466912232081) bank87750 +87751 POINT(41.33202871645369 74.01204201264008) bank87751 +87752 POINT(41.32469645376556 74.71247720909832) bank87752 +87753 POINT(41.34681257006511 74.28926500312842) bank87753 +87754 POINT(39.95548814791654 74.68819533216134) bank87754 +87755 POINT(40.43978751343804 74.82450491085825) bank87755 +87756 POINT(39.89633608919728 73.77707002945142) bank87756 +87757 POINT(41.41923087324982 73.09718666145258) bank87757 +87758 POINT(40.21877013719748 74.62930577942387) bank87758 +87759 POINT(40.16127063217777 73.91530635235266) bank87759 +87760 POINT(40.58471104022482 73.89297388048858) bank87760 +87761 POINT(41.023096385710545 74.65245025608803) bank87761 +87762 POINT(41.247605653244904 73.89266720501935) bank87762 +87763 POINT(40.66460633592521 73.20142039371522) bank87763 +87764 POINT(41.007365998469005 73.61252695234535) bank87764 +87765 POINT(41.090938325910145 74.01624932106103) bank87765 +87766 POINT(40.785486797974684 74.30710925995899) bank87766 +87767 POINT(40.10473627948661 74.80495300689928) bank87767 +87768 POINT(41.3580615982218 73.96045781046968) bank87768 +87769 POINT(39.92371893265702 73.69958243872199) bank87769 +87770 POINT(40.336493483635806 74.64152340758608) bank87770 +87771 POINT(41.220561331259645 73.22976587011159) bank87771 +87772 POINT(41.5257686399176 74.50172558636459) bank87772 +87773 POINT(41.565648155630655 73.09691986667633) bank87773 +87774 POINT(39.98616105131482 74.83702313756905) bank87774 +87775 POINT(41.66974030274646 73.24979650057863) bank87775 +87776 POINT(39.84214821172007 73.78437288332134) bank87776 +87777 POINT(41.20217981227658 73.29575359086627) bank87777 +87778 POINT(40.33900965743484 74.9382472784676) bank87778 +87779 POINT(41.4710356328977 74.65378414529411) bank87779 +87780 POINT(40.38656402171799 73.40722081999223) bank87780 +87781 POINT(39.723999091605926 73.50158750603828) bank87781 +87782 POINT(40.923308967093504 74.89577967549619) bank87782 +87783 POINT(41.2732260947614 73.68697931210573) bank87783 +87784 POINT(40.15885600769634 74.8634265290429) bank87784 +87785 POINT(41.55085857324973 74.96790251905782) bank87785 +87786 POINT(41.64214616687388 73.48301655246222) bank87786 +87787 POINT(41.081453688186656 73.58857724989637) bank87787 +87788 POINT(41.57666250657428 73.61865222144989) bank87788 +87789 POINT(40.85888375258707 73.06578120281084) bank87789 +87790 POINT(40.63758316434192 73.53005300316559) bank87790 +87791 POINT(41.32673985662662 73.47166110046646) bank87791 +87792 POINT(40.612679268840886 73.45625990028951) bank87792 +87793 POINT(40.166097423020176 74.17856018935294) bank87793 +87794 POINT(41.70390734409113 73.77969976059894) bank87794 +87795 POINT(41.18948547210865 74.89371335870501) bank87795 +87796 POINT(39.738437188271696 73.7610455480432) bank87796 +87797 POINT(39.81313133378615 73.56565846541757) bank87797 +87798 POINT(41.66882494163942 74.2935255643966) bank87798 +87799 POINT(41.48424407717024 74.08785376737879) bank87799 +87800 POINT(40.58310799318707 74.87998293307402) bank87800 +87801 POINT(40.264878267837254 73.32469839230865) bank87801 +87802 POINT(40.95467795049991 73.77498579360883) bank87802 +87803 POINT(39.80721095570985 74.39193429979703) bank87803 +87804 POINT(39.76333508041407 74.68703856593493) bank87804 +87805 POINT(40.236494482298724 73.61807458186462) bank87805 +87806 POINT(40.76683619154465 74.45287189043718) bank87806 +87807 POINT(41.35953308032802 73.91230029806646) bank87807 +87808 POINT(41.60527492355821 73.28321849261462) bank87808 +87809 POINT(39.98683222629089 74.96040084739708) bank87809 +87810 POINT(40.77205144257221 74.52176512441415) bank87810 +87811 POINT(40.518403168432975 74.12295589363332) bank87811 +87812 POINT(41.663770770792574 74.67656550683401) bank87812 +87813 POINT(40.87963978111666 73.0196292157598) bank87813 +87814 POINT(41.10709505666461 73.9083406636203) bank87814 +87815 POINT(41.32434832111003 73.99416688037711) bank87815 +87816 POINT(41.315692718965984 73.76912589783566) bank87816 +87817 POINT(41.34188643634877 74.54088672007559) bank87817 +87818 POINT(40.806693504582334 73.13386088953054) bank87818 +87819 POINT(40.030331857358284 73.36516221155989) bank87819 +87820 POINT(41.31526763339407 74.10904843288696) bank87820 +87821 POINT(41.28411454816117 73.3109660895074) bank87821 +87822 POINT(41.415631010671866 73.75363769369183) bank87822 +87823 POINT(41.30019508071582 74.57998106606325) bank87823 +87824 POINT(39.95047989441917 73.11255497956955) bank87824 +87825 POINT(40.84911472349732 73.84987811820619) bank87825 +87826 POINT(40.01043967660907 73.24208913953525) bank87826 +87827 POINT(40.71571591506383 74.66908462711913) bank87827 +87828 POINT(40.15266108936438 73.630206064623) bank87828 +87829 POINT(40.4011762219532 74.73498738711847) bank87829 +87830 POINT(39.948643760113015 74.27792186696774) bank87830 +87831 POINT(40.90719870807184 73.66497514507594) bank87831 +87832 POINT(40.05309317752752 74.26565000475273) bank87832 +87833 POINT(40.85612074263223 74.7512527527075) bank87833 +87834 POINT(40.105746464733016 74.05075531140433) bank87834 +87835 POINT(40.3468980007275 74.48278741962098) bank87835 +87836 POINT(40.87192156302805 74.4712637963065) bank87836 +87837 POINT(40.195423078593116 74.73617567204987) bank87837 +87838 POINT(39.91211355593321 74.36614281438997) bank87838 +87839 POINT(39.970992105332584 73.57557435673225) bank87839 +87840 POINT(39.80764194183169 74.48076452265315) bank87840 +87841 POINT(40.63015746734725 74.22687461714206) bank87841 +87842 POINT(41.53896828503033 73.5468164133887) bank87842 +87843 POINT(40.786705081866955 74.42377484251462) bank87843 +87844 POINT(40.41555901097457 74.98053731258614) bank87844 +87845 POINT(39.762792533958404 73.95033583874998) bank87845 +87846 POINT(40.427526322807346 73.35411128337613) bank87846 +87847 POINT(40.87387608290932 73.03375385442129) bank87847 +87848 POINT(40.817007580786694 74.23187211775803) bank87848 +87849 POINT(41.56984527974614 73.1240996262334) bank87849 +87850 POINT(40.02456528844177 73.95503653236842) bank87850 +87851 POINT(40.13415931164875 74.31386615306894) bank87851 +87852 POINT(41.23852075792198 74.68204929678635) bank87852 +87853 POINT(40.35818734750867 73.7648511398327) bank87853 +87854 POINT(41.5907751622648 74.01658220625023) bank87854 +87855 POINT(41.00563864322846 74.11708831792791) bank87855 +87856 POINT(40.82183512155769 73.56619072392799) bank87856 +87857 POINT(41.6004735376286 73.07322879503819) bank87857 +87858 POINT(41.53268868838592 74.1838717093502) bank87858 +87859 POINT(39.92687437331546 74.52380287025503) bank87859 +87860 POINT(40.622023509493665 74.82361806491103) bank87860 +87861 POINT(40.757840609968405 73.61404727053727) bank87861 +87862 POINT(39.897234409046156 74.04800267656272) bank87862 +87863 POINT(41.550967735958274 73.17390610166817) bank87863 +87864 POINT(40.5890902647227 73.25269655911141) bank87864 +87865 POINT(41.69036341331211 74.69306910367101) bank87865 +87866 POINT(41.25066596501951 73.17988051432773) bank87866 +87867 POINT(39.76389350699534 73.37242518058113) bank87867 +87868 POINT(41.60987546295862 74.1772022914199) bank87868 +87869 POINT(40.970957679144234 73.28819030386043) bank87869 +87870 POINT(40.45135590306725 74.01904019950813) bank87870 +87871 POINT(41.40660293764628 74.2724941866884) bank87871 +87872 POINT(40.756227272441535 73.22196051562413) bank87872 +87873 POINT(40.42851682180075 73.04750186167169) bank87873 +87874 POINT(41.18655557708808 74.40545274711126) bank87874 +87875 POINT(40.5972812519392 73.33201215814316) bank87875 +87876 POINT(39.90727226721236 74.93291097788347) bank87876 +87877 POINT(40.03740888339482 74.49396730935229) bank87877 +87878 POINT(39.834888897960155 73.94592243078839) bank87878 +87879 POINT(40.99570054429467 73.8749261052722) bank87879 +87880 POINT(41.02194552765776 74.01324993652553) bank87880 +87881 POINT(41.09211406173022 74.79433863078374) bank87881 +87882 POINT(41.43403156580114 74.98448148013841) bank87882 +87883 POINT(41.00236731766726 74.48083485631462) bank87883 +87884 POINT(40.11358475871912 73.0911101526828) bank87884 +87885 POINT(41.40308875353073 74.55722713080519) bank87885 +87886 POINT(41.54847372228103 74.62633440582182) bank87886 +87887 POINT(39.83006668017618 74.00227666511394) bank87887 +87888 POINT(41.01394573279438 73.33827874926625) bank87888 +87889 POINT(41.43640866112728 74.13563987780452) bank87889 +87890 POINT(40.86638425516579 74.50044313196115) bank87890 +87891 POINT(40.71907512564536 74.72928362041843) bank87891 +87892 POINT(39.83983593895363 73.16053109712016) bank87892 +87893 POINT(40.48694718439232 74.0181660634918) bank87893 +87894 POINT(40.381234849896465 74.05510238884841) bank87894 +87895 POINT(40.60418645131288 73.74093205490735) bank87895 +87896 POINT(40.11205875043856 74.98623252293639) bank87896 +87897 POINT(39.84513701424966 74.0651845025092) bank87897 +87898 POINT(40.9664263026665 74.70584989872206) bank87898 +87899 POINT(41.389609149134216 73.79781405109813) bank87899 +87900 POINT(40.76861076229543 73.72277917306681) bank87900 +87901 POINT(39.913994325971416 74.97025597332107) bank87901 +87902 POINT(40.38678829034422 73.73018770414137) bank87902 +87903 POINT(40.17494622872127 74.30092258211226) bank87903 +87904 POINT(40.98030753429338 74.99678674438907) bank87904 +87905 POINT(39.79341993974196 73.42333656252413) bank87905 +87906 POINT(40.36217446298848 73.45372462712264) bank87906 +87907 POINT(39.799449637938594 73.76916058533907) bank87907 +87908 POINT(41.504971067078706 74.12180141460132) bank87908 +87909 POINT(39.752900326870744 73.91859105940415) bank87909 +87910 POINT(41.12872679576847 73.57859714274862) bank87910 +87911 POINT(40.177639937348765 74.67341717179983) bank87911 +87912 POINT(40.11811875120815 73.85375719486235) bank87912 +87913 POINT(40.68045393056691 74.97068273285704) bank87913 +87914 POINT(40.366447766661196 73.71726276668005) bank87914 +87915 POINT(41.26772062477109 73.81857649860565) bank87915 +87916 POINT(40.430856765856454 74.22235331117639) bank87916 +87917 POINT(41.31477525120984 73.90338564527536) bank87917 +87918 POINT(40.90724568145423 73.21987620797718) bank87918 +87919 POINT(40.378984643712755 74.64380408541578) bank87919 +87920 POINT(40.44662651759245 73.40956483456753) bank87920 +87921 POINT(41.35042996221509 74.95990979730232) bank87921 +87922 POINT(40.28144144310214 73.79579314369914) bank87922 +87923 POINT(39.75555820437089 73.3563107804509) bank87923 +87924 POINT(40.55351220787705 73.24797011634826) bank87924 +87925 POINT(40.57153912084595 73.77094009944769) bank87925 +87926 POINT(41.34719377980678 74.67632412566759) bank87926 +87927 POINT(40.034712433717225 74.82118787618946) bank87927 +87928 POINT(41.55204265668848 74.86134477633436) bank87928 +87929 POINT(40.54407459457508 74.42708745523254) bank87929 +87930 POINT(40.63988086887253 73.83072130018849) bank87930 +87931 POINT(40.345253383931556 74.08480356468198) bank87931 +87932 POINT(41.06443029530958 73.7584979790998) bank87932 +87933 POINT(41.65511532236666 74.92903148514459) bank87933 +87934 POINT(40.705569701051196 74.22740932053047) bank87934 +87935 POINT(40.42882053478534 74.40310783111163) bank87935 +87936 POINT(40.602027590552645 74.59170746362615) bank87936 +87937 POINT(41.001426212596876 73.614882594124) bank87937 +87938 POINT(40.594650797374136 74.50702604312092) bank87938 +87939 POINT(40.27735723685154 73.30312271353634) bank87939 +87940 POINT(41.46890455626985 74.73673995703085) bank87940 +87941 POINT(39.732543501699084 73.30047370677865) bank87941 +87942 POINT(41.16022076932776 74.90313167945153) bank87942 +87943 POINT(41.60094148078315 74.0385629302556) bank87943 +87944 POINT(41.38764895210175 73.09624323587661) bank87944 +87945 POINT(40.01684775839098 74.97830687714766) bank87945 +87946 POINT(40.07397972390901 73.5898073069467) bank87946 +87947 POINT(40.773817116506784 74.30955588951855) bank87947 +87948 POINT(41.47094329400009 73.91175614762693) bank87948 +87949 POINT(40.96966833664475 73.44598945991979) bank87949 +87950 POINT(41.568963213672504 73.98190691828843) bank87950 +87951 POINT(41.27319594583489 73.67705540709103) bank87951 +87952 POINT(39.78051307256928 74.08918400060381) bank87952 +87953 POINT(41.56238325588264 74.73354544507167) bank87953 +87954 POINT(41.107931919931076 74.39189878000603) bank87954 +87955 POINT(40.10481162338099 73.87539881065354) bank87955 +87956 POINT(40.94411169052244 73.27654363730397) bank87956 +87957 POINT(39.85972641260232 74.63149540789776) bank87957 +87958 POINT(40.538643983254715 74.57373074796236) bank87958 +87959 POINT(39.97909708280514 73.57174150702683) bank87959 +87960 POINT(40.7834562118744 74.43169246215682) bank87960 +87961 POINT(40.710449663575915 73.64787493027903) bank87961 +87962 POINT(40.00633845061207 73.56082261907372) bank87962 +87963 POINT(41.57162865897162 74.80648440947006) bank87963 +87964 POINT(40.06878732592406 73.60225015653398) bank87964 +87965 POINT(41.02513464808573 73.97860979069327) bank87965 +87966 POINT(40.300877282476485 74.4541866079325) bank87966 +87967 POINT(41.17368934921368 73.06507001701307) bank87967 +87968 POINT(40.340648659837726 74.81580600063027) bank87968 +87969 POINT(40.61426345864514 73.57068066696579) bank87969 +87970 POINT(40.199137388682324 74.70090741882694) bank87970 +87971 POINT(40.226304721365274 74.53905696446365) bank87971 +87972 POINT(39.84322210550738 74.45914705997431) bank87972 +87973 POINT(40.41900469485655 74.34162278113364) bank87973 +87974 POINT(40.43892927457987 73.10609279258425) bank87974 +87975 POINT(41.033525125426834 74.39213330239994) bank87975 +87976 POINT(40.28873058980852 73.07461894690564) bank87976 +87977 POINT(41.04658449876882 74.67192420977177) bank87977 +87978 POINT(41.4969174063322 74.51147008909244) bank87978 +87979 POINT(40.57446860350753 73.53811531901343) bank87979 +87980 POINT(41.319520958096476 73.36675565988243) bank87980 +87981 POINT(40.22271246235222 73.12299654356313) bank87981 +87982 POINT(40.806837807710195 73.75678343595527) bank87982 +87983 POINT(41.189709409744616 74.15558125001186) bank87983 +87984 POINT(40.75698404355763 73.68459648124636) bank87984 +87985 POINT(41.260947261160986 73.73251801624868) bank87985 +87986 POINT(40.92772421956499 73.40597531933442) bank87986 +87987 POINT(40.4309471235198 73.96613808123476) bank87987 +87988 POINT(40.0109889383568 74.1433848980667) bank87988 +87989 POINT(40.732628851002794 73.57807760437412) bank87989 +87990 POINT(41.44606237583726 73.99753075476049) bank87990 +87991 POINT(40.80531004194886 74.04113308843024) bank87991 +87992 POINT(40.34164807056936 73.68665718064881) bank87992 +87993 POINT(41.23672571215788 73.77617025701468) bank87993 +87994 POINT(41.170133226538354 73.97332332567073) bank87994 +87995 POINT(41.448754718270614 73.399678978479) bank87995 +87996 POINT(41.16467307046774 74.874579264475) bank87996 +87997 POINT(41.028333198104576 73.38877537255466) bank87997 +87998 POINT(40.30897463630373 73.70725150695048) bank87998 +87999 POINT(41.22289347327751 73.68518666063197) bank87999 +88000 POINT(41.349767378976765 74.44516091691168) bank88000 +88001 POINT(40.15951347217908 74.14721841910035) bank88001 +88002 POINT(40.42919184364177 73.68555620254914) bank88002 +88003 POINT(41.51479091947103 73.55843236333666) bank88003 +88004 POINT(41.1460459921575 74.98313462047011) bank88004 +88005 POINT(39.91909470061676 73.08360238468323) bank88005 +88006 POINT(39.75358803893273 74.17595549121924) bank88006 +88007 POINT(39.98361637529134 74.25469800563123) bank88007 +88008 POINT(39.92850986909574 74.62008914097875) bank88008 +88009 POINT(39.962342763623354 74.57455318352066) bank88009 +88010 POINT(40.12841642823489 73.2657904100965) bank88010 +88011 POINT(41.44275328590819 73.48489961725748) bank88011 +88012 POINT(40.63068090498581 74.84561666573454) bank88012 +88013 POINT(40.099329780886194 74.70640844529413) bank88013 +88014 POINT(40.769645936073665 74.55215646601266) bank88014 +88015 POINT(40.160477878705116 73.89333602716903) bank88015 +88016 POINT(39.782073131626106 74.21624687009053) bank88016 +88017 POINT(40.4793149836836 74.39996860762126) bank88017 +88018 POINT(40.67512836253945 73.57338863928788) bank88018 +88019 POINT(41.70929938410556 74.80482306926362) bank88019 +88020 POINT(41.033986701404274 74.46090579813445) bank88020 +88021 POINT(41.282720368022666 74.40738770664804) bank88021 +88022 POINT(40.170373305074826 73.98451354401637) bank88022 +88023 POINT(40.63764156679955 74.3666895629491) bank88023 +88024 POINT(41.28579503681092 74.4809250206886) bank88024 +88025 POINT(39.901876085048116 73.0541866867661) bank88025 +88026 POINT(41.570177078112565 73.37175434993729) bank88026 +88027 POINT(41.68809830258395 73.56109944910348) bank88027 +88028 POINT(40.26561150204103 74.47311177942194) bank88028 +88029 POINT(41.489443305885025 74.06616479314874) bank88029 +88030 POINT(40.322180790724765 73.93015720663153) bank88030 +88031 POINT(40.576883513180796 74.19661408349532) bank88031 +88032 POINT(41.672111409469856 73.22518550756087) bank88032 +88033 POINT(39.80044605560035 73.14911773822304) bank88033 +88034 POINT(40.7003660278584 74.79116643697749) bank88034 +88035 POINT(40.28439836005077 74.23897873719177) bank88035 +88036 POINT(41.331853192522985 73.9300492670696) bank88036 +88037 POINT(39.731786377728874 73.56832624675799) bank88037 +88038 POINT(41.12149355581117 74.72600453359365) bank88038 +88039 POINT(40.135792102330704 74.89424802331055) bank88039 +88040 POINT(39.73225367043354 73.36273411456305) bank88040 +88041 POINT(41.611850589392546 73.98754263928555) bank88041 +88042 POINT(41.412427550576375 73.06451865005769) bank88042 +88043 POINT(40.46659442594614 73.27694797568927) bank88043 +88044 POINT(40.50815245191301 74.35679485595213) bank88044 +88045 POINT(41.06023573614353 73.76620598678298) bank88045 +88046 POINT(39.807195896853706 74.9637361191473) bank88046 +88047 POINT(40.39436177726395 73.4275464621477) bank88047 +88048 POINT(40.3703310614114 73.79004421365552) bank88048 +88049 POINT(40.03293983172813 73.95907543758494) bank88049 +88050 POINT(40.43715812256597 73.26390085235685) bank88050 +88051 POINT(40.029673333242535 74.78523000075198) bank88051 +88052 POINT(41.14312594960966 74.60451757311624) bank88052 +88053 POINT(40.06305808181175 73.20971304442544) bank88053 +88054 POINT(40.79069273389912 73.54521186533422) bank88054 +88055 POINT(41.26366906359437 74.25259354328114) bank88055 +88056 POINT(41.50954519170399 73.68518118118777) bank88056 +88057 POINT(39.884731137859816 73.42468829420247) bank88057 +88058 POINT(40.03788129690293 74.25956909005765) bank88058 +88059 POINT(39.730832309435854 73.28937692239805) bank88059 +88060 POINT(41.447511876386926 74.43004700068411) bank88060 +88061 POINT(40.59084321826879 74.02257830759703) bank88061 +88062 POINT(40.82367305137069 74.50578312763582) bank88062 +88063 POINT(41.6222713664371 73.41154280019347) bank88063 +88064 POINT(41.47603971381049 74.21541939882405) bank88064 +88065 POINT(39.87177845513107 74.65992803031662) bank88065 +88066 POINT(41.23578817031198 73.09444407658621) bank88066 +88067 POINT(40.93611394665802 73.52023795424861) bank88067 +88068 POINT(40.808705849316006 73.53180232982805) bank88068 +88069 POINT(40.22041857536342 74.84458899145584) bank88069 +88070 POINT(41.284596138542426 73.59033474835518) bank88070 +88071 POINT(40.18204088109371 73.91694781258516) bank88071 +88072 POINT(41.24735130656814 74.53178089016298) bank88072 +88073 POINT(41.119971422111185 74.75015993483152) bank88073 +88074 POINT(40.41076681913495 74.3351742181476) bank88074 +88075 POINT(40.30049307173905 73.95282845154645) bank88075 +88076 POINT(41.52899841194206 74.06097554068708) bank88076 +88077 POINT(41.48402444638482 74.77965771357185) bank88077 +88078 POINT(39.95253985630703 74.66272691871063) bank88078 +88079 POINT(41.03358890165359 73.7278385243922) bank88079 +88080 POINT(39.959288715279584 73.83951918626234) bank88080 +88081 POINT(40.44478270456844 73.26833161297333) bank88081 +88082 POINT(40.4709243873021 74.1169104091006) bank88082 +88083 POINT(40.37554861174027 74.41254652067357) bank88083 +88084 POINT(40.63632842012887 74.03350988403409) bank88084 +88085 POINT(40.84820359469118 74.55605693297439) bank88085 +88086 POINT(41.65703192093842 74.68966578150714) bank88086 +88087 POINT(40.60663559905124 74.98126978296742) bank88087 +88088 POINT(40.58154870206611 73.59268646448135) bank88088 +88089 POINT(40.04693767189365 73.8176881817638) bank88089 +88090 POINT(41.42538104345316 74.649228172965) bank88090 +88091 POINT(41.33535802109646 73.54784584058686) bank88091 +88092 POINT(40.08810803820436 73.72182049389771) bank88092 +88093 POINT(40.33786973643055 74.95639098548163) bank88093 +88094 POINT(41.43732757042426 73.90655259636642) bank88094 +88095 POINT(41.12608601943046 74.40348310221927) bank88095 +88096 POINT(40.90965445755347 74.72163541878295) bank88096 +88097 POINT(40.687436833281794 73.95565652652857) bank88097 +88098 POINT(40.08259183940551 73.73658556837117) bank88098 +88099 POINT(41.41855277649916 73.40386119287214) bank88099 +88100 POINT(39.94734738238305 73.5013552669794) bank88100 +88101 POINT(40.397209800242855 74.45559978556236) bank88101 +88102 POINT(40.174017108377605 74.98908503435308) bank88102 +88103 POINT(40.81183882519247 73.9002565460591) bank88103 +88104 POINT(39.73551906617186 74.94361293778609) bank88104 +88105 POINT(40.51718453403532 74.62792785546067) bank88105 +88106 POINT(40.61136509340583 73.54269124030908) bank88106 +88107 POINT(41.33067259380886 73.22409608574516) bank88107 +88108 POINT(40.737052617025796 74.98505611955343) bank88108 +88109 POINT(40.83192079574176 73.04986184741163) bank88109 +88110 POINT(40.122028677542104 73.5675302230845) bank88110 +88111 POINT(39.91816293799407 73.42331054392405) bank88111 +88112 POINT(39.7946444038746 73.82590550137753) bank88112 +88113 POINT(41.658249132696945 74.88113194708122) bank88113 +88114 POINT(40.1769443911512 74.7349238001992) bank88114 +88115 POINT(39.8486221197717 74.63393391237886) bank88115 +88116 POINT(39.73027819717845 73.72641686951238) bank88116 +88117 POINT(40.94840989969749 74.55858619615972) bank88117 +88118 POINT(40.1692300487084 74.7154013592701) bank88118 +88119 POINT(40.0658314234724 74.72763707629521) bank88119 +88120 POINT(39.97124905000838 74.19215668554621) bank88120 +88121 POINT(41.59906690546491 74.37281121570176) bank88121 +88122 POINT(41.603708027994 73.65988561713296) bank88122 +88123 POINT(41.01241006590955 74.4064530125055) bank88123 +88124 POINT(41.34508507616561 73.74551469398078) bank88124 +88125 POINT(41.25572605395489 73.0748729749856) bank88125 +88126 POINT(40.966324019129594 73.55543314012311) bank88126 +88127 POINT(40.242451399074255 73.97544211081849) bank88127 +88128 POINT(40.28164250639156 74.64127932373299) bank88128 +88129 POINT(40.119839812665816 74.07491291174976) bank88129 +88130 POINT(40.533373793523715 74.6830300813174) bank88130 +88131 POINT(41.15545828037189 74.52933389433947) bank88131 +88132 POINT(41.21227791528653 73.71244537230173) bank88132 +88133 POINT(40.22758222148516 74.95296643969192) bank88133 +88134 POINT(39.825815849772106 73.65641065185186) bank88134 +88135 POINT(41.55142720278035 73.2818715318784) bank88135 +88136 POINT(40.36158413406981 74.47171934283972) bank88136 +88137 POINT(41.396846044889685 74.18549405868075) bank88137 +88138 POINT(40.3007427509568 73.78458854504939) bank88138 +88139 POINT(40.43307201178049 73.87237580833227) bank88139 +88140 POINT(41.07974685321333 73.44730931713393) bank88140 +88141 POINT(40.99115100531058 74.42825370357193) bank88141 +88142 POINT(41.17934009976885 74.37281697504133) bank88142 +88143 POINT(41.282166372779336 74.82112997046639) bank88143 +88144 POINT(41.411293858387474 73.86601111447057) bank88144 +88145 POINT(41.57458162445839 74.17420970341551) bank88145 +88146 POINT(41.650643438576914 73.02673362089403) bank88146 +88147 POINT(40.7027644171244 74.26646476458632) bank88147 +88148 POINT(40.727134891634606 73.44620230265107) bank88148 +88149 POINT(39.997059474831126 74.65648107006008) bank88149 +88150 POINT(39.87095470923482 73.38763701348016) bank88150 +88151 POINT(40.93162855176102 73.62443465054226) bank88151 +88152 POINT(40.3317939403771 74.98373802614095) bank88152 +88153 POINT(41.39638523877003 74.80413803208108) bank88153 +88154 POINT(41.65452145334683 74.81125739597624) bank88154 +88155 POINT(40.130152538558896 73.41099286674857) bank88155 +88156 POINT(41.46276471977266 74.17861305944302) bank88156 +88157 POINT(40.41510166907773 74.44931292593394) bank88157 +88158 POINT(40.9931871753163 74.41351912340288) bank88158 +88159 POINT(39.90866718842453 73.64874810878176) bank88159 +88160 POINT(40.733727593712864 73.43671146216539) bank88160 +88161 POINT(41.063812530832436 74.52122059130208) bank88161 +88162 POINT(40.72373132734927 73.60275807847475) bank88162 +88163 POINT(39.72713531533819 74.3278720638713) bank88163 +88164 POINT(40.45166322569176 73.9020578468292) bank88164 +88165 POINT(40.03490313766104 74.35756103668083) bank88165 +88166 POINT(39.79971144548327 74.78399499296519) bank88166 +88167 POINT(41.683409173043096 74.56395581968503) bank88167 +88168 POINT(40.75297772671196 73.22475995813961) bank88168 +88169 POINT(40.00108114833861 74.93687752010058) bank88169 +88170 POINT(41.191023736314506 74.46666323303288) bank88170 +88171 POINT(40.84732056992047 73.38009316397017) bank88171 +88172 POINT(40.08164025963436 74.76978308467737) bank88172 +88173 POINT(40.18531027755865 74.72885950056862) bank88173 +88174 POINT(40.89941129538871 73.44372752524536) bank88174 +88175 POINT(40.30893615929182 74.67093999802577) bank88175 +88176 POINT(41.32760163389694 74.86652504965126) bank88176 +88177 POINT(40.9607051311376 73.69944470562383) bank88177 +88178 POINT(39.80210325208164 74.65987879382524) bank88178 +88179 POINT(40.89393483215439 74.6015159313591) bank88179 +88180 POINT(40.508922804640605 74.68142396182145) bank88180 +88181 POINT(39.88787745444781 73.29891935909971) bank88181 +88182 POINT(39.72972062984139 74.57143398272925) bank88182 +88183 POINT(41.661246639601714 73.35414978219615) bank88183 +88184 POINT(41.494765129070664 74.94901203641332) bank88184 +88185 POINT(41.0935717687773 73.65623374518694) bank88185 +88186 POINT(40.3793948493244 73.90384103677667) bank88186 +88187 POINT(40.78012287467344 74.75804830133754) bank88187 +88188 POINT(41.28171273073498 74.83350013383023) bank88188 +88189 POINT(39.88452477447688 73.33771808484424) bank88189 +88190 POINT(40.94352644910912 74.49392133857307) bank88190 +88191 POINT(41.526653277861534 73.92216152433808) bank88191 +88192 POINT(39.83994356676507 74.72920152867539) bank88192 +88193 POINT(39.85091877636817 73.29952379139998) bank88193 +88194 POINT(41.34657624133614 74.54672899033629) bank88194 +88195 POINT(39.78171155872977 74.06056442976222) bank88195 +88196 POINT(41.30235211032852 73.37240032540463) bank88196 +88197 POINT(40.40416021029904 73.58990467245685) bank88197 +88198 POINT(41.20465098912543 74.21012475900751) bank88198 +88199 POINT(41.17909825589862 74.37542683914009) bank88199 +88200 POINT(40.95719311432822 74.6889335302114) bank88200 +88201 POINT(41.54666776443266 73.95106105405634) bank88201 +88202 POINT(40.69561633985215 73.46437190570354) bank88202 +88203 POINT(40.34955041020921 74.34454514177199) bank88203 +88204 POINT(40.4965026354198 73.40382778334124) bank88204 +88205 POINT(39.96731920556694 74.7855693569208) bank88205 +88206 POINT(41.400072906997856 73.08708193525844) bank88206 +88207 POINT(41.685562691037106 74.96717823378651) bank88207 +88208 POINT(39.815784456329794 73.51232919815952) bank88208 +88209 POINT(40.72006805510822 74.90205618013142) bank88209 +88210 POINT(39.830917718590456 73.53191738734608) bank88210 +88211 POINT(40.524637217423326 74.13120009131684) bank88211 +88212 POINT(40.056894661063986 73.16744648660082) bank88212 +88213 POINT(40.0654651097789 73.48791644851006) bank88213 +88214 POINT(40.24361124810575 73.98442546544585) bank88214 +88215 POINT(41.41979250469983 74.59477957935366) bank88215 +88216 POINT(40.696215647213265 73.43247923288467) bank88216 +88217 POINT(40.34001751996679 73.3901871915792) bank88217 +88218 POINT(39.92741588195195 73.90820644904194) bank88218 +88219 POINT(40.29309926899363 73.14265010752777) bank88219 +88220 POINT(41.69661218853875 73.23797279849143) bank88220 +88221 POINT(41.64274305762218 74.13401817424969) bank88221 +88222 POINT(40.42262128761741 74.40235355468958) bank88222 +88223 POINT(41.02823346960053 74.40726533271022) bank88223 +88224 POINT(41.33708285521224 74.34937237674548) bank88224 +88225 POINT(41.568527355475226 73.96941186103713) bank88225 +88226 POINT(39.79345880568574 73.06825481668925) bank88226 +88227 POINT(39.73479749628575 73.14640475583236) bank88227 +88228 POINT(41.545127624417674 74.97865204824586) bank88228 +88229 POINT(40.32819366638313 73.54602716256234) bank88229 +88230 POINT(41.162556963264464 73.79777791934174) bank88230 +88231 POINT(40.5264047135103 73.57659641888662) bank88231 +88232 POINT(41.21968554044788 73.53655890963715) bank88232 +88233 POINT(40.06246336064765 73.16805063436188) bank88233 +88234 POINT(40.18443240964065 74.3540135811409) bank88234 +88235 POINT(40.94438527215781 74.45074826927583) bank88235 +88236 POINT(39.931866806897624 73.07304657304651) bank88236 +88237 POINT(41.129677177976816 73.50389222073059) bank88237 +88238 POINT(40.7749952571179 74.03187197436704) bank88238 +88239 POINT(41.478238841486665 73.97728369321231) bank88239 +88240 POINT(40.104612232500095 73.9225499274674) bank88240 +88241 POINT(40.46661276555907 74.3701893751319) bank88241 +88242 POINT(41.418545073374474 73.5312052146542) bank88242 +88243 POINT(41.670721627868765 74.76577023694755) bank88243 +88244 POINT(40.19165545328419 73.5611705003718) bank88244 +88245 POINT(40.27049591282592 73.52502031666637) bank88245 +88246 POINT(41.19745626332018 74.28054335308433) bank88246 +88247 POINT(41.20812922140544 73.87642354366812) bank88247 +88248 POINT(39.78771849788276 73.78708798767221) bank88248 +88249 POINT(40.72973968744473 74.04282343300933) bank88249 +88250 POINT(41.09481423232279 74.702183964979) bank88250 +88251 POINT(39.72588021123836 73.20384932781695) bank88251 +88252 POINT(39.843747571148484 73.09959656369938) bank88252 +88253 POINT(41.208566829742054 73.52646644782341) bank88253 +88254 POINT(41.10138001125586 73.15045105855646) bank88254 +88255 POINT(41.68123573398729 74.38810414649164) bank88255 +88256 POINT(40.74904878216585 74.91323255947388) bank88256 +88257 POINT(39.85886300537648 74.48023430437613) bank88257 +88258 POINT(40.62130378423358 74.19390785998579) bank88258 +88259 POINT(40.74882184596753 73.55878035040095) bank88259 +88260 POINT(41.322530523162435 73.32827088664695) bank88260 +88261 POINT(41.248149595051444 73.8691183876855) bank88261 +88262 POINT(40.10420067282037 73.9766520061091) bank88262 +88263 POINT(40.26571836482604 74.69306416572276) bank88263 +88264 POINT(41.01310526987257 74.07991551650358) bank88264 +88265 POINT(40.83829440191581 73.7091510799731) bank88265 +88266 POINT(41.431095420346004 73.09467831062948) bank88266 +88267 POINT(40.083218497640175 74.94970939466072) bank88267 +88268 POINT(41.42562033626074 73.74580017670569) bank88268 +88269 POINT(41.269216271751574 73.31316536078786) bank88269 +88270 POINT(39.94336359966137 73.78627537846441) bank88270 +88271 POINT(40.416872303040556 74.38326549884617) bank88271 +88272 POINT(41.221232475127174 73.5407674196408) bank88272 +88273 POINT(40.046585005077034 74.3893157533933) bank88273 +88274 POINT(40.295352438523516 74.58003543685815) bank88274 +88275 POINT(41.02896767429536 74.69375181117661) bank88275 +88276 POINT(41.43674673603757 73.19092285632071) bank88276 +88277 POINT(40.78833303600395 74.16236363972773) bank88277 +88278 POINT(41.33261708723426 73.34489583796105) bank88278 +88279 POINT(41.69985461214768 74.34086556090226) bank88279 +88280 POINT(40.39250564714234 74.55667429868136) bank88280 +88281 POINT(39.83269299434913 73.77526080220817) bank88281 +88282 POINT(41.65641978713711 74.87998263981538) bank88282 +88283 POINT(41.561431909725925 74.31115313477076) bank88283 +88284 POINT(41.01733265672056 74.40247837959147) bank88284 +88285 POINT(40.97089078307123 74.50080069866993) bank88285 +88286 POINT(41.07896541159357 74.4023697225258) bank88286 +88287 POINT(41.46243673482273 74.39124311760801) bank88287 +88288 POINT(40.163379953868635 74.61400519683663) bank88288 +88289 POINT(40.26223668835797 74.264749972064) bank88289 +88290 POINT(39.859254639741486 74.9127789484533) bank88290 +88291 POINT(40.28552551433978 74.81051023211532) bank88291 +88292 POINT(40.90197969419103 74.44438587882519) bank88292 +88293 POINT(41.2626655950898 74.82684333175736) bank88293 +88294 POINT(41.00812243337407 73.07543235669637) bank88294 +88295 POINT(39.817777230077446 74.25570912254939) bank88295 +88296 POINT(39.771302856856764 74.18562991675057) bank88296 +88297 POINT(40.28590714671957 74.26565313847635) bank88297 +88298 POINT(41.68084138289258 73.4739054084466) bank88298 +88299 POINT(40.79240030610468 74.01599608395286) bank88299 +88300 POINT(40.574929595820535 73.96768846937783) bank88300 +88301 POINT(41.529945062403165 73.6143896350037) bank88301 +88302 POINT(41.2479744412048 74.5418393629673) bank88302 +88303 POINT(41.317898242132614 73.05162697788344) bank88303 +88304 POINT(40.67747257600293 74.17357865975285) bank88304 +88305 POINT(41.16344863541853 74.52923101013211) bank88305 +88306 POINT(40.590451520441206 73.84092551661816) bank88306 +88307 POINT(41.66952796766508 74.41656477834147) bank88307 +88308 POINT(40.419795615487494 73.09032230358413) bank88308 +88309 POINT(41.13472456840927 73.1173001689777) bank88309 +88310 POINT(41.086868497068366 74.09158650498283) bank88310 +88311 POINT(41.15875088577308 74.77762024159756) bank88311 +88312 POINT(41.20281624984182 73.49485056896053) bank88312 +88313 POINT(39.91140850930176 74.55545774799121) bank88313 +88314 POINT(41.16194114555745 74.47303433464256) bank88314 +88315 POINT(40.5766419030667 73.2555226244412) bank88315 +88316 POINT(41.141806473349604 73.93222251179995) bank88316 +88317 POINT(41.2726929580572 74.89638240250477) bank88317 +88318 POINT(41.18430128255185 74.37304707433047) bank88318 +88319 POINT(40.18735826762752 74.69034310487655) bank88319 +88320 POINT(41.68756552081125 74.23959361406776) bank88320 +88321 POINT(39.77953506342209 74.85147588117746) bank88321 +88322 POINT(40.5585192375163 73.18109802667848) bank88322 +88323 POINT(41.46304425074683 74.56427711243512) bank88323 +88324 POINT(40.680267348366485 74.05740955650096) bank88324 +88325 POINT(40.44797019102723 73.4053576386559) bank88325 +88326 POINT(40.725943951263865 73.4962301395168) bank88326 +88327 POINT(40.03006582588901 74.74729856096322) bank88327 +88328 POINT(40.77452693477998 74.42492892184293) bank88328 +88329 POINT(40.30661500492794 73.42431751531781) bank88329 +88330 POINT(41.012421483090776 74.09523380814795) bank88330 +88331 POINT(40.176085570013775 73.31955001617841) bank88331 +88332 POINT(40.557178039972385 74.37904690723106) bank88332 +88333 POINT(40.028121516366014 74.93323679058109) bank88333 +88334 POINT(41.13735516905999 73.62029544467691) bank88334 +88335 POINT(40.81644465790718 73.75080462625314) bank88335 +88336 POINT(40.53681355181208 74.68464660358448) bank88336 +88337 POINT(39.84630402546247 74.20033458288145) bank88337 +88338 POINT(40.75253063943478 73.80298899057061) bank88338 +88339 POINT(39.82856979114675 74.56399215200413) bank88339 +88340 POINT(40.67836032309009 74.75087898716733) bank88340 +88341 POINT(40.29033825317425 73.85121070569143) bank88341 +88342 POINT(39.865800601534715 73.47667717058351) bank88342 +88343 POINT(41.49012556200097 73.65913661960782) bank88343 +88344 POINT(40.43572914147685 74.14535671676428) bank88344 +88345 POINT(40.53032694117787 74.70912588243817) bank88345 +88346 POINT(40.549952648727825 74.18917377153838) bank88346 +88347 POINT(39.815424362403476 74.30320388972947) bank88347 +88348 POINT(41.3335563398244 73.40982295389146) bank88348 +88349 POINT(41.60328984574345 73.59356202632523) bank88349 +88350 POINT(40.61743916444148 73.19606889398813) bank88350 +88351 POINT(41.544071801737786 73.41782530628393) bank88351 +88352 POINT(40.49655943292009 73.54440860911888) bank88352 +88353 POINT(40.492823923687226 74.54226520912303) bank88353 +88354 POINT(39.855099874923134 74.3892052363267) bank88354 +88355 POINT(41.21217053473596 73.10769628196879) bank88355 +88356 POINT(40.19296521443321 74.00319895124504) bank88356 +88357 POINT(41.34545010079983 74.60271179671352) bank88357 +88358 POINT(40.55569655308127 73.26871250583328) bank88358 +88359 POINT(40.96920752539216 74.41319131549542) bank88359 +88360 POINT(40.27727267069675 74.0811615476345) bank88360 +88361 POINT(40.09957461384163 74.93094815700518) bank88361 +88362 POINT(39.839153494074175 74.56254173113747) bank88362 +88363 POINT(39.98467205400683 73.61831560377158) bank88363 +88364 POINT(39.91480803526048 73.55856030978512) bank88364 +88365 POINT(39.73341085101928 73.4270320965141) bank88365 +88366 POINT(40.7781362359338 73.24385646079254) bank88366 +88367 POINT(40.1904199720809 74.44627927103258) bank88367 +88368 POINT(40.870642525035144 73.78954418427516) bank88368 +88369 POINT(40.78536849958827 74.50142320718169) bank88369 +88370 POINT(41.17786743279609 74.71056347539924) bank88370 +88371 POINT(39.71440769831992 73.0581329733198) bank88371 +88372 POINT(41.474526785823876 74.74715551982021) bank88372 +88373 POINT(41.462631152176854 74.05066109243892) bank88373 +88374 POINT(40.76068132422436 74.73400155639283) bank88374 +88375 POINT(39.974300657589616 74.99124358752589) bank88375 +88376 POINT(40.2278551650396 74.97146901247866) bank88376 +88377 POINT(40.338551907162525 73.93461212910691) bank88377 +88378 POINT(40.744141788016464 73.89521744629947) bank88378 +88379 POINT(40.10536901590727 74.20676562243953) bank88379 +88380 POINT(41.165801736244156 74.27116075199493) bank88380 +88381 POINT(41.54928258816406 74.20867553645137) bank88381 +88382 POINT(40.786544041127435 74.98535895105391) bank88382 +88383 POINT(40.85447638582838 73.74638984699652) bank88383 +88384 POINT(41.66978781572437 74.17390481605328) bank88384 +88385 POINT(40.46715586287701 74.86274194653187) bank88385 +88386 POINT(41.287885353799446 74.34835213329507) bank88386 +88387 POINT(39.852859496917546 74.91018181140714) bank88387 +88388 POINT(39.80514109480397 74.43360850117656) bank88388 +88389 POINT(40.934146335548824 74.76857248168358) bank88389 +88390 POINT(40.33999599976654 74.07708807007515) bank88390 +88391 POINT(39.94798576215997 74.51331146262658) bank88391 +88392 POINT(39.76066096610751 74.45783297505884) bank88392 +88393 POINT(40.220505189168954 73.88867032430684) bank88393 +88394 POINT(40.94527511016043 74.03969098209646) bank88394 +88395 POINT(41.0062304838376 74.52381568324223) bank88395 +88396 POINT(39.80413214542266 74.86989373229333) bank88396 +88397 POINT(41.45196635107577 74.55800568789977) bank88397 +88398 POINT(40.12774699773167 73.30535397400931) bank88398 +88399 POINT(40.01161210614894 73.12932624355385) bank88399 +88400 POINT(41.637278641283274 73.80655934334287) bank88400 +88401 POINT(41.630477128246774 74.76540225397443) bank88401 +88402 POINT(41.53861699606766 74.23205701322844) bank88402 +88403 POINT(40.5479396687281 73.25384121490114) bank88403 +88404 POINT(40.18480565494693 73.62626345196014) bank88404 +88405 POINT(40.936230927613444 73.14318908768058) bank88405 +88406 POINT(40.38914270091451 74.826634877719) bank88406 +88407 POINT(40.18881117991888 73.03630922875334) bank88407 +88408 POINT(40.663301211349115 73.91718832684502) bank88408 +88409 POINT(41.08842352951057 73.10038664413644) bank88409 +88410 POINT(40.89630896545794 73.86296618019921) bank88410 +88411 POINT(41.453983961258494 74.59392642735037) bank88411 +88412 POINT(41.16962696416093 73.73155664586578) bank88412 +88413 POINT(40.92691504510135 73.42383988017181) bank88413 +88414 POINT(39.77857139262683 74.18885096727053) bank88414 +88415 POINT(40.20222005756395 74.96464950832683) bank88415 +88416 POINT(41.24704233865707 74.29556766698403) bank88416 +88417 POINT(40.84318340457266 74.09427356224842) bank88417 +88418 POINT(41.518430626415785 73.73573347804403) bank88418 +88419 POINT(41.639329728311786 74.9089232205524) bank88419 +88420 POINT(39.913338257648455 73.50492046649326) bank88420 +88421 POINT(40.041522809389306 74.40830687379403) bank88421 +88422 POINT(41.56668817096467 74.45999062811455) bank88422 +88423 POINT(39.85524286160618 74.90128143734756) bank88423 +88424 POINT(40.43275398048577 74.812257779741) bank88424 +88425 POINT(40.60989616187208 73.96347773491608) bank88425 +88426 POINT(40.870393049674924 73.11263250950891) bank88426 +88427 POINT(39.80408048456456 73.72078976852276) bank88427 +88428 POINT(41.65354632670175 74.25932301017471) bank88428 +88429 POINT(41.31628335900433 73.67005559430498) bank88429 +88430 POINT(40.7679416558002 73.71995553140137) bank88430 +88431 POINT(40.97139648528828 73.47288562855277) bank88431 +88432 POINT(40.39785794716963 73.65500411847053) bank88432 +88433 POINT(41.47750722290839 73.6880876640545) bank88433 +88434 POINT(41.48998444201794 74.94195278258238) bank88434 +88435 POINT(41.62664455172287 74.85372574381638) bank88435 +88436 POINT(40.00387187556387 74.20869681196383) bank88436 +88437 POINT(40.588526937689494 74.72295142619154) bank88437 +88438 POINT(39.75636744483315 73.39968341444339) bank88438 +88439 POINT(40.029781505019734 73.57606535782926) bank88439 +88440 POINT(40.72571848381004 73.1030554336061) bank88440 +88441 POINT(40.534200381282496 73.36953901205524) bank88441 +88442 POINT(40.95319548036407 74.5422784591798) bank88442 +88443 POINT(40.84984331925362 73.98841076737556) bank88443 +88444 POINT(40.77040620993318 73.33226027898175) bank88444 +88445 POINT(40.34485577900646 73.4939079138342) bank88445 +88446 POINT(41.06578828423161 74.77510550396562) bank88446 +88447 POINT(40.59646565849764 74.68652986368257) bank88447 +88448 POINT(40.24506598333239 73.24070905364349) bank88448 +88449 POINT(41.24307459540974 73.79908724144892) bank88449 +88450 POINT(41.18913732054461 74.6936126886316) bank88450 +88451 POINT(40.07047599874591 73.81099642720031) bank88451 +88452 POINT(40.50963748619474 74.38549777130699) bank88452 +88453 POINT(39.920187847840445 74.79595976597577) bank88453 +88454 POINT(39.92882332779124 73.52420914488522) bank88454 +88455 POINT(40.418816706801685 74.31334705212795) bank88455 +88456 POINT(41.567657692823985 73.30304219915784) bank88456 +88457 POINT(41.36563692909956 74.17603861172441) bank88457 +88458 POINT(40.86600546138611 74.93791114951941) bank88458 +88459 POINT(39.949725371231644 73.95835958741317) bank88459 +88460 POINT(40.88196771438242 73.9774407004608) bank88460 +88461 POINT(41.04463832355207 74.49495196272656) bank88461 +88462 POINT(41.30630849561956 73.22222827419108) bank88462 +88463 POINT(40.388825720219174 73.02990196336482) bank88463 +88464 POINT(41.27842713314283 73.8783081559619) bank88464 +88465 POINT(41.6997236139822 74.91836980784065) bank88465 +88466 POINT(41.005296019318465 74.89114923984033) bank88466 +88467 POINT(41.100386131868056 73.85592339717411) bank88467 +88468 POINT(40.977931895486634 73.49249364271532) bank88468 +88469 POINT(41.623934048177986 74.91608284077135) bank88469 +88470 POINT(40.14187090510801 74.46614216571756) bank88470 +88471 POINT(39.82527372558396 73.10857394776598) bank88471 +88472 POINT(39.74093529737979 73.26967754159358) bank88472 +88473 POINT(40.64306845052971 74.82272840695467) bank88473 +88474 POINT(40.100934716632175 73.38102922318771) bank88474 +88475 POINT(40.810866858987744 73.36591208684914) bank88475 +88476 POINT(41.50135418788909 73.90226506307576) bank88476 +88477 POINT(40.01873011774412 73.25819078781466) bank88477 +88478 POINT(39.832812774287646 73.56833263447649) bank88478 +88479 POINT(41.01971532389085 74.56019249761972) bank88479 +88480 POINT(39.878106491980226 74.71262044867304) bank88480 +88481 POINT(40.25539833336542 73.69428866758469) bank88481 +88482 POINT(41.36173400988149 73.72215388384409) bank88482 +88483 POINT(40.68798144524938 74.62394594696208) bank88483 +88484 POINT(41.19612436881085 73.28924643706729) bank88484 +88485 POINT(40.066925838514 74.46281583555829) bank88485 +88486 POINT(40.94372587644002 74.30626024846299) bank88486 +88487 POINT(41.426037474228906 73.15098136530996) bank88487 +88488 POINT(41.41964382305227 73.04781097652695) bank88488 +88489 POINT(39.763914330533744 73.55099310046714) bank88489 +88490 POINT(41.01586183915131 73.56353232280371) bank88490 +88491 POINT(40.50353824920574 73.61937819498826) bank88491 +88492 POINT(41.456004950932936 73.4434111065037) bank88492 +88493 POINT(40.38562881930301 74.78241035003883) bank88493 +88494 POINT(39.95994323634124 74.05729745322881) bank88494 +88495 POINT(41.07832574927197 73.92753316074926) bank88495 +88496 POINT(39.95150804457852 74.0372340508208) bank88496 +88497 POINT(41.031451026553114 74.43700627891172) bank88497 +88498 POINT(40.334866420577924 73.50566156626954) bank88498 +88499 POINT(40.94103202933921 73.5333618310864) bank88499 +88500 POINT(41.18671519855488 74.23447866035148) bank88500 +88501 POINT(40.480742558385245 74.62208844589442) bank88501 +88502 POINT(39.98628371612917 74.47500455820905) bank88502 +88503 POINT(40.36266957642332 73.24402148370449) bank88503 +88504 POINT(41.31110787768928 74.53864083977858) bank88504 +88505 POINT(40.798846302750405 73.80015570085284) bank88505 +88506 POINT(40.35455047148631 74.23555524304064) bank88506 +88507 POINT(40.466521299954366 74.87929083229203) bank88507 +88508 POINT(41.478979227278806 73.71762518005754) bank88508 +88509 POINT(41.25403256685491 73.98173488362158) bank88509 +88510 POINT(41.432286765754 73.3984043772897) bank88510 +88511 POINT(40.835054670498515 73.2066529124394) bank88511 +88512 POINT(40.47256519344548 74.4618567425467) bank88512 +88513 POINT(41.3868767819108 74.55016295054901) bank88513 +88514 POINT(40.755894831730394 73.40340186211198) bank88514 +88515 POINT(40.78364975536962 73.91485124568841) bank88515 +88516 POINT(41.57964085435705 73.32796234147146) bank88516 +88517 POINT(40.75829762099116 74.88161472824065) bank88517 +88518 POINT(40.84637877800288 74.08444326150816) bank88518 +88519 POINT(39.99071065715665 74.38839963580655) bank88519 +88520 POINT(41.3442727771453 73.60085361288537) bank88520 +88521 POINT(41.37783730855949 74.59316998965924) bank88521 +88522 POINT(40.63401265155811 73.11198249449656) bank88522 +88523 POINT(40.182129140549726 74.95772696536372) bank88523 +88524 POINT(40.94495937963439 73.87612385549977) bank88524 +88525 POINT(41.19899779722268 73.53276741581205) bank88525 +88526 POINT(40.525516564871346 74.06853700997412) bank88526 +88527 POINT(40.05110369973133 74.58886235435963) bank88527 +88528 POINT(40.34614453336761 74.87918516353137) bank88528 +88529 POINT(40.63433772583788 73.55907045430105) bank88529 +88530 POINT(40.350663615540256 74.93537333147572) bank88530 +88531 POINT(39.86964527031694 73.64771444826054) bank88531 +88532 POINT(39.87840962388116 73.12103540452522) bank88532 +88533 POINT(41.045823293308416 74.11757323868402) bank88533 +88534 POINT(41.081137538891205 74.37728309158838) bank88534 +88535 POINT(40.57464069517543 74.66588481706346) bank88535 +88536 POINT(40.35902831938752 73.23714051759329) bank88536 +88537 POINT(40.80607826507147 74.59993478645694) bank88537 +88538 POINT(41.45920755609033 73.13938412067033) bank88538 +88539 POINT(41.13147761554949 73.28693129416818) bank88539 +88540 POINT(40.85171296262937 74.3162640479086) bank88540 +88541 POINT(40.526102666792674 73.98436688303015) bank88541 +88542 POINT(40.966548422201754 74.77716895715282) bank88542 +88543 POINT(40.00435004942662 74.71885382449607) bank88543 +88544 POINT(40.52531405046193 73.74586026075147) bank88544 +88545 POINT(41.528316581644866 73.85855388963569) bank88545 +88546 POINT(41.30448602755574 74.43179810739271) bank88546 +88547 POINT(39.76055417744193 74.1550647437844) bank88547 +88548 POINT(41.24341369852989 74.1094037091674) bank88548 +88549 POINT(40.0475731138249 74.21876711359567) bank88549 +88550 POINT(40.29542577217698 73.27291952044803) bank88550 +88551 POINT(41.17663822919076 74.0120462119669) bank88551 +88552 POINT(41.71219611215923 73.18753864379877) bank88552 +88553 POINT(39.71947055749083 73.10249019252534) bank88553 +88554 POINT(41.29799276353593 74.48613458034629) bank88554 +88555 POINT(41.28679018348958 73.49503494284913) bank88555 +88556 POINT(40.95583339512283 73.54451218292411) bank88556 +88557 POINT(39.82585952571969 74.26212519888377) bank88557 +88558 POINT(40.66939589117606 74.40103394944535) bank88558 +88559 POINT(40.61706727735264 74.88677829370104) bank88559 +88560 POINT(40.63111224184677 74.56555052038516) bank88560 +88561 POINT(41.08724915323785 73.4437264262047) bank88561 +88562 POINT(40.95442733327051 73.6645421824973) bank88562 +88563 POINT(40.02916717314191 73.60445008830393) bank88563 +88564 POINT(40.37960792844731 73.61024160563801) bank88564 +88565 POINT(40.14675069573753 74.04912951498949) bank88565 +88566 POINT(41.01451017041071 73.21492490187254) bank88566 +88567 POINT(41.415765780012855 73.02910227599543) bank88567 +88568 POINT(40.37589313765892 74.63318769456376) bank88568 +88569 POINT(40.910694395357716 73.27561381340116) bank88569 +88570 POINT(40.332899112775 73.21034010235283) bank88570 +88571 POINT(41.0261993228454 74.35566552806446) bank88571 +88572 POINT(41.30268363969941 73.57816769680852) bank88572 +88573 POINT(41.68530062830296 73.76441441856694) bank88573 +88574 POINT(41.00958615177379 74.58490167111376) bank88574 +88575 POINT(40.21424941921259 73.54985337653885) bank88575 +88576 POINT(39.865961198388106 73.24212981732978) bank88576 +88577 POINT(41.20650296021227 73.70926558885975) bank88577 +88578 POINT(41.52007429630202 74.18138394816528) bank88578 +88579 POINT(41.27740612187425 74.68759335295663) bank88579 +88580 POINT(41.22489775606661 73.40319879022748) bank88580 +88581 POINT(41.32860544533928 73.75744071742446) bank88581 +88582 POINT(41.39183379849681 74.97163267426936) bank88582 +88583 POINT(40.302373569180794 73.40847695268104) bank88583 +88584 POINT(40.84541940241914 74.31570468173973) bank88584 +88585 POINT(41.168400636390786 73.9258149903244) bank88585 +88586 POINT(40.060108749346156 74.71647137512974) bank88586 +88587 POINT(41.42125111939593 73.64976764060194) bank88587 +88588 POINT(39.97978650090442 73.6686438195328) bank88588 +88589 POINT(40.16644004388337 73.65673109798784) bank88589 +88590 POINT(40.13885440118673 73.21501835757034) bank88590 +88591 POINT(40.08961294204511 73.43481691945829) bank88591 +88592 POINT(40.11097118890532 74.15185850748468) bank88592 +88593 POINT(40.776317924109335 73.02413997924957) bank88593 +88594 POINT(41.042952789318015 74.93111142911773) bank88594 +88595 POINT(40.81958500090934 73.94924097267617) bank88595 +88596 POINT(40.00325296068778 74.91982805796646) bank88596 +88597 POINT(40.10419917625853 74.4907388670944) bank88597 +88598 POINT(40.354330344610034 73.8727574406833) bank88598 +88599 POINT(39.9733204530246 74.7481814790167) bank88599 +88600 POINT(41.05061878286733 74.22407794087387) bank88600 +88601 POINT(40.36290133103801 74.57037285812422) bank88601 +88602 POINT(41.00884085545304 73.01091937599112) bank88602 +88603 POINT(40.036332534418705 73.73837912331427) bank88603 +88604 POINT(40.66658866118297 73.23876890164209) bank88604 +88605 POINT(40.9477727607037 74.05025961286307) bank88605 +88606 POINT(40.930959145170306 74.11735394496036) bank88606 +88607 POINT(40.46989000614063 74.41607443937234) bank88607 +88608 POINT(39.896983769525946 73.42862404442397) bank88608 +88609 POINT(40.75324285040407 73.20210614731354) bank88609 +88610 POINT(40.63794876565521 74.48178714274627) bank88610 +88611 POINT(40.67851994912518 74.9405593763376) bank88611 +88612 POINT(40.66536296659008 73.33260711866066) bank88612 +88613 POINT(41.10267208142564 74.42662201299345) bank88613 +88614 POINT(41.052188217915706 74.80787715852115) bank88614 +88615 POINT(41.10779403693496 73.0498996366892) bank88615 +88616 POINT(39.715506917483836 73.64634369708952) bank88616 +88617 POINT(41.4275142745121 74.95836676481254) bank88617 +88618 POINT(40.60217560132675 73.3397696476481) bank88618 +88619 POINT(40.471221540521036 73.55823240334239) bank88619 +88620 POINT(39.984253040917984 73.55726237614444) bank88620 +88621 POINT(41.02222382494141 74.88815290206958) bank88621 +88622 POINT(40.77889793996236 74.07939401013456) bank88622 +88623 POINT(41.284028626336216 73.32271597050885) bank88623 +88624 POINT(40.4982585263819 74.5698611101859) bank88624 +88625 POINT(39.863003014538194 73.67612778336662) bank88625 +88626 POINT(40.88854018844842 74.08256652717776) bank88626 +88627 POINT(40.3062328635542 74.11051042515093) bank88627 +88628 POINT(41.21047018286018 73.57501579159104) bank88628 +88629 POINT(40.45582314570323 74.59489559950428) bank88629 +88630 POINT(39.91594849893895 74.72833939020408) bank88630 +88631 POINT(40.675764240276834 73.25078020373108) bank88631 +88632 POINT(40.07455107126535 73.72318418906455) bank88632 +88633 POINT(40.08254314600613 74.15446604809453) bank88633 +88634 POINT(41.70731253469359 74.22340646436754) bank88634 +88635 POINT(40.930598575884446 74.5755378977037) bank88635 +88636 POINT(41.25674589067135 73.56180042118645) bank88636 +88637 POINT(40.868675273937455 73.60538676251595) bank88637 +88638 POINT(41.17553575788573 73.34619718632506) bank88638 +88639 POINT(40.07854273091264 74.1933723869249) bank88639 +88640 POINT(40.121361719982104 74.707194074242) bank88640 +88641 POINT(41.67987720244994 73.83959769167537) bank88641 +88642 POINT(40.161452958315685 73.49694732277752) bank88642 +88643 POINT(40.78035661303839 73.60988614048867) bank88643 +88644 POINT(40.18635915369845 73.48251052567552) bank88644 +88645 POINT(41.35329212724658 73.19071709539955) bank88645 +88646 POINT(41.02682915239167 74.91904231532098) bank88646 +88647 POINT(40.2487985440445 73.32706946640091) bank88647 +88648 POINT(40.81376858530385 74.84851675907599) bank88648 +88649 POINT(40.883489239949526 74.9544794950046) bank88649 +88650 POINT(41.38918563531244 74.13827024780461) bank88650 +88651 POINT(40.1952271714281 73.12797851265398) bank88651 +88652 POINT(40.86190285334902 73.09297151426797) bank88652 +88653 POINT(40.361480258438704 73.96497375165751) bank88653 +88654 POINT(41.45887612477587 73.44783670990793) bank88654 +88655 POINT(41.0581180063584 73.40344689463313) bank88655 +88656 POINT(40.10325908529908 74.89277354748032) bank88656 +88657 POINT(41.084749966564374 73.7167642044029) bank88657 +88658 POINT(39.99996736995413 73.07069313298241) bank88658 +88659 POINT(40.92274354909775 74.98321632105215) bank88659 +88660 POINT(39.95700804239406 73.12985388847625) bank88660 +88661 POINT(40.888447288653246 73.01341800840854) bank88661 +88662 POINT(41.07286944090561 74.75981408944747) bank88662 +88663 POINT(40.37475582657528 74.75986361971528) bank88663 +88664 POINT(40.41485994942895 74.439355342906) bank88664 +88665 POINT(40.908616271827604 73.06261735304936) bank88665 +88666 POINT(40.657021667391014 74.68699650095589) bank88666 +88667 POINT(39.72468251453119 73.7263191380657) bank88667 +88668 POINT(40.2254933580371 74.63215835220626) bank88668 +88669 POINT(41.41978781043709 73.95072606583682) bank88669 +88670 POINT(40.356891678904944 73.40237969524033) bank88670 +88671 POINT(41.304019238688646 73.43696295890834) bank88671 +88672 POINT(40.74460030874093 73.33305433034766) bank88672 +88673 POINT(40.54728366818169 74.07103149758909) bank88673 +88674 POINT(41.23512848859726 74.30401244603823) bank88674 +88675 POINT(40.93772290833229 74.9516228893017) bank88675 +88676 POINT(41.38782320869485 74.12990212536022) bank88676 +88677 POINT(40.23289060165594 74.32841033853057) bank88677 +88678 POINT(40.83914984028276 73.13038970555426) bank88678 +88679 POINT(40.476424498871644 73.50029343312342) bank88679 +88680 POINT(41.05565613157796 74.88500255417942) bank88680 +88681 POINT(41.107450868445646 73.77452305693994) bank88681 +88682 POINT(40.65496312239907 74.64513610825244) bank88682 +88683 POINT(41.47762434606401 74.18716299163157) bank88683 +88684 POINT(39.91831539745513 73.01136346882865) bank88684 +88685 POINT(41.54097187642195 73.76443862403556) bank88685 +88686 POINT(40.032635003274315 74.69560078974038) bank88686 +88687 POINT(41.24510326912519 74.64461877737342) bank88687 +88688 POINT(40.71941910571354 74.90588378644219) bank88688 +88689 POINT(40.97665459603806 74.2485452157914) bank88689 +88690 POINT(41.42309985748339 73.37829050354142) bank88690 +88691 POINT(40.472463386802104 74.99303488931189) bank88691 +88692 POINT(41.36824956229169 74.05115675278992) bank88692 +88693 POINT(40.36610645014155 74.13892542582003) bank88693 +88694 POINT(41.1655627215454 74.29732732779455) bank88694 +88695 POINT(41.315844694205786 74.56818356070725) bank88695 +88696 POINT(39.808170015662256 74.67688776436131) bank88696 +88697 POINT(40.98485902274221 73.578550451005) bank88697 +88698 POINT(40.24393865428243 73.53285233531122) bank88698 +88699 POINT(39.95383249938509 73.97838609421792) bank88699 +88700 POINT(41.06864526380532 74.79395841928041) bank88700 +88701 POINT(40.54916691106718 73.78851274582057) bank88701 +88702 POINT(40.78346794263065 73.20865202855613) bank88702 +88703 POINT(41.69704221372345 74.02871445285331) bank88703 +88704 POINT(41.654772010030484 73.73873934469317) bank88704 +88705 POINT(41.17689680213969 74.29263230858795) bank88705 +88706 POINT(39.750309989447864 74.88220601180423) bank88706 +88707 POINT(40.093241649178324 73.72409818721309) bank88707 +88708 POINT(40.4770429154337 73.06216777867105) bank88708 +88709 POINT(41.050322035228575 73.82573786758971) bank88709 +88710 POINT(41.17953093820421 74.56396555398186) bank88710 +88711 POINT(39.92963792536193 74.34930775756561) bank88711 +88712 POINT(40.518268399771664 73.26204222492295) bank88712 +88713 POINT(40.40443978764323 73.07361213606583) bank88713 +88714 POINT(40.2902496590706 74.77273407155971) bank88714 +88715 POINT(39.78297184670005 74.97486467260494) bank88715 +88716 POINT(40.00232333337515 74.61282502261807) bank88716 +88717 POINT(41.68545441522503 74.70673840228656) bank88717 +88718 POINT(40.27778618226508 74.03627301457003) bank88718 +88719 POINT(41.583582069657965 73.62983329254119) bank88719 +88720 POINT(40.470841876296774 74.20857328103082) bank88720 +88721 POINT(40.43197323663394 74.9433365210754) bank88721 +88722 POINT(40.69782657586449 74.32878065985014) bank88722 +88723 POINT(40.34568408779867 74.22412424280367) bank88723 +88724 POINT(39.96008271995652 73.44167368684951) bank88724 +88725 POINT(41.432377486879055 74.61480207635127) bank88725 +88726 POINT(39.903677495896005 74.27223136064924) bank88726 +88727 POINT(40.76963612704942 74.51609472054436) bank88727 +88728 POINT(41.29803909914802 73.3492945169036) bank88728 +88729 POINT(41.39889223599098 73.17722794162567) bank88729 +88730 POINT(39.83107507320573 74.67169322600185) bank88730 +88731 POINT(41.18445874865703 74.89439488019707) bank88731 +88732 POINT(40.81560238014008 74.69908451758835) bank88732 +88733 POINT(39.82747597061734 73.8930305671021) bank88733 +88734 POINT(40.06730977885379 74.10212638546125) bank88734 +88735 POINT(41.64303141291683 74.08368062788614) bank88735 +88736 POINT(41.632207309506256 74.70364062601338) bank88736 +88737 POINT(41.597631641213404 73.3681809895058) bank88737 +88738 POINT(40.29133446743216 74.82259634175276) bank88738 +88739 POINT(41.129253118404264 74.71482301735358) bank88739 +88740 POINT(41.13569792694271 74.11997595937062) bank88740 +88741 POINT(40.189723743561395 73.08899035832997) bank88741 +88742 POINT(41.19790300282455 73.35000950295293) bank88742 +88743 POINT(39.81218786722394 73.27133178757893) bank88743 +88744 POINT(41.160888017721916 74.29538757625053) bank88744 +88745 POINT(40.58752629016862 73.8030422100053) bank88745 +88746 POINT(40.62246639977988 74.39437091461035) bank88746 +88747 POINT(39.74864538522349 73.70810258917294) bank88747 +88748 POINT(41.358323975605586 74.58310799707785) bank88748 +88749 POINT(39.80170979319811 73.6025752735227) bank88749 +88750 POINT(40.149289729167165 73.47532710897946) bank88750 +88751 POINT(40.442111835049026 74.3398689251159) bank88751 +88752 POINT(40.80882155518124 73.53196422136554) bank88752 +88753 POINT(39.844939111115536 74.05524674335848) bank88753 +88754 POINT(40.9770749404442 73.07374575240169) bank88754 +88755 POINT(40.09629826273358 73.45524200358703) bank88755 +88756 POINT(40.20032353499413 74.49269348665058) bank88756 +88757 POINT(41.669738684725054 74.67857912962673) bank88757 +88758 POINT(39.8352306708368 73.35452484910272) bank88758 +88759 POINT(41.53209822921528 74.51212037627278) bank88759 +88760 POINT(41.496503442770944 73.19756670569328) bank88760 +88761 POINT(41.16312658832201 74.36497687507634) bank88761 +88762 POINT(40.993238489478145 74.62610949645683) bank88762 +88763 POINT(40.14746601235148 73.71478788437466) bank88763 +88764 POINT(40.374643979421016 74.98000295848296) bank88764 +88765 POINT(41.53627219736243 74.60673578459556) bank88765 +88766 POINT(40.74379453742231 73.80698238468835) bank88766 +88767 POINT(40.4847865658403 73.36460108449452) bank88767 +88768 POINT(40.22464384788261 73.93791647897609) bank88768 +88769 POINT(40.46629970479522 74.81054170919047) bank88769 +88770 POINT(40.16214112594132 73.34771494876667) bank88770 +88771 POINT(40.181990393890004 73.47950994106984) bank88771 +88772 POINT(41.165578440960786 73.21911732942348) bank88772 +88773 POINT(41.562669288232954 73.79972209986018) bank88773 +88774 POINT(40.872364344679305 73.97696032600986) bank88774 +88775 POINT(41.57043223532303 74.83891906388904) bank88775 +88776 POINT(41.567928404562025 74.31772665330813) bank88776 +88777 POINT(41.64682081395748 74.00221978132598) bank88777 +88778 POINT(40.806231074644224 74.53332938522858) bank88778 +88779 POINT(41.299586768735445 74.50145575563404) bank88779 +88780 POINT(41.364927512651384 73.2723209125012) bank88780 +88781 POINT(41.12562548377388 73.51611229694097) bank88781 +88782 POINT(41.49557912107849 74.50904953376892) bank88782 +88783 POINT(40.235148906038866 75.00337639174121) bank88783 +88784 POINT(41.70285639964068 74.69507288480644) bank88784 +88785 POINT(40.277603553648355 74.49387894508907) bank88785 +88786 POINT(40.75236306390066 74.6083245742281) bank88786 +88787 POINT(39.914053245764016 74.63114884191171) bank88787 +88788 POINT(39.791032880580985 74.75358028690735) bank88788 +88789 POINT(39.80625768460258 73.68591419365502) bank88789 +88790 POINT(40.88604552551747 73.49097811812264) bank88790 +88791 POINT(40.839336232134286 73.30212430818827) bank88791 +88792 POINT(40.891084996072244 74.39726371733417) bank88792 +88793 POINT(41.23006783103761 73.29097526082641) bank88793 +88794 POINT(41.628275704176694 73.00783501800106) bank88794 +88795 POINT(39.96856903834556 73.26845663625446) bank88795 +88796 POINT(40.303959598317554 73.10177529192477) bank88796 +88797 POINT(39.94708632776713 73.24046514623674) bank88797 +88798 POINT(41.096124212583085 74.46826610579801) bank88798 +88799 POINT(40.79433902702633 74.91908869487033) bank88799 +88800 POINT(41.46744014538828 73.55994166151163) bank88800 +88801 POINT(40.73005147534216 73.83573347084328) bank88801 +88802 POINT(40.04762622195343 73.33612047041672) bank88802 +88803 POINT(40.802648097540896 74.97815377883664) bank88803 +88804 POINT(41.4630978100446 74.49657826018971) bank88804 +88805 POINT(40.23580492449959 74.79420913344896) bank88805 +88806 POINT(39.747358933105694 74.57595458795548) bank88806 +88807 POINT(40.635544622904845 74.80507344312902) bank88807 +88808 POINT(39.782807881825015 74.48263607681662) bank88808 +88809 POINT(41.220808163130876 74.42299335335008) bank88809 +88810 POINT(39.94296928159167 74.60282152299784) bank88810 +88811 POINT(41.27573009092651 73.76595751018061) bank88811 +88812 POINT(41.50780499163229 73.44956557118779) bank88812 +88813 POINT(40.731534819273996 73.26502729913) bank88813 +88814 POINT(41.383768733730726 74.3983217197702) bank88814 +88815 POINT(41.40318530032742 73.70634134306114) bank88815 +88816 POINT(41.185238011283495 73.8398839479348) bank88816 +88817 POINT(40.736083464092125 74.4664703736934) bank88817 +88818 POINT(41.32725022774677 74.14275589540186) bank88818 +88819 POINT(41.29371626044689 74.72982445564466) bank88819 +88820 POINT(40.10189254097322 74.2656352048257) bank88820 +88821 POINT(39.88684527084342 74.24847787875969) bank88821 +88822 POINT(40.203791601342516 74.18863222804436) bank88822 +88823 POINT(40.43466520420441 74.34205939307452) bank88823 +88824 POINT(40.68841569936085 74.23973263233425) bank88824 +88825 POINT(40.657766158305016 74.29951527103759) bank88825 +88826 POINT(41.20814978092248 74.29953114565134) bank88826 +88827 POINT(40.93521135377848 74.2507208408187) bank88827 +88828 POINT(41.361123319633535 74.16243065189704) bank88828 +88829 POINT(41.632930128720815 73.30519588446539) bank88829 +88830 POINT(40.69745874170141 73.98279051708857) bank88830 +88831 POINT(40.273343134979484 73.29036021147954) bank88831 +88832 POINT(39.74067835883137 74.3633515520707) bank88832 +88833 POINT(41.65708231093442 73.56964009408829) bank88833 +88834 POINT(41.36955498759211 74.4145391537527) bank88834 +88835 POINT(41.398573751950565 73.76197389631932) bank88835 +88836 POINT(39.91793995705389 74.30554019707829) bank88836 +88837 POINT(40.2033564840016 73.35523962852265) bank88837 +88838 POINT(41.12995282181323 74.41085052940892) bank88838 +88839 POINT(41.26226699722929 74.50952677856138) bank88839 +88840 POINT(40.78794865455189 73.43342118329453) bank88840 +88841 POINT(40.22009247283107 73.06808993607912) bank88841 +88842 POINT(40.038145419934025 74.77119587783876) bank88842 +88843 POINT(39.82893321894833 74.18144999004268) bank88843 +88844 POINT(41.346564734023936 74.98554230462446) bank88844 +88845 POINT(41.19554647089553 74.21004112427075) bank88845 +88846 POINT(41.004316703869755 73.6652873397605) bank88846 +88847 POINT(41.37213990809873 74.85314778888471) bank88847 +88848 POINT(40.65760534030249 74.96149406381302) bank88848 +88849 POINT(39.82972133015994 74.84749085942096) bank88849 +88850 POINT(40.42518026158183 75.00314430839003) bank88850 +88851 POINT(40.53844090150002 73.9818180746309) bank88851 +88852 POINT(40.12141972030201 74.48592731683766) bank88852 +88853 POINT(40.63947499472223 73.47088453250306) bank88853 +88854 POINT(41.484175736664284 73.87108574626667) bank88854 +88855 POINT(40.786893371964 73.73689256657849) bank88855 +88856 POINT(40.90649569593652 74.81547546730205) bank88856 +88857 POINT(41.62207940024095 73.86909183773078) bank88857 +88858 POINT(40.45415259244648 74.83568919462654) bank88858 +88859 POINT(41.675274925584134 73.67112252415124) bank88859 +88860 POINT(40.35217724659659 74.18993943842942) bank88860 +88861 POINT(41.66192670288208 73.88693950096827) bank88861 +88862 POINT(40.65559042348667 73.93991173753749) bank88862 +88863 POINT(39.71711179097178 73.68286315720903) bank88863 +88864 POINT(40.25197128910538 73.96073408092126) bank88864 +88865 POINT(39.7326731478184 74.64832003684386) bank88865 +88866 POINT(40.9594115303923 74.63099649439526) bank88866 +88867 POINT(40.56835153196418 74.91485667015351) bank88867 +88868 POINT(41.26345844195798 74.14619744308669) bank88868 +88869 POINT(39.921582557942955 73.16538156382026) bank88869 +88870 POINT(40.428303097903225 74.21031970223373) bank88870 +88871 POINT(41.476490880686136 74.05685399912541) bank88871 +88872 POINT(40.05543380015184 74.47591527778125) bank88872 +88873 POINT(41.478030650788476 73.77131043481232) bank88873 +88874 POINT(41.336285823529245 73.1779595067008) bank88874 +88875 POINT(41.03485721623981 74.24030227409055) bank88875 +88876 POINT(41.121052081150474 73.0234819883908) bank88876 +88877 POINT(40.3623147066679 73.08090540220753) bank88877 +88878 POINT(39.83012679898437 73.34507460962632) bank88878 +88879 POINT(41.16434237581328 73.22306658897519) bank88879 +88880 POINT(40.8120516217166 74.0718318487966) bank88880 +88881 POINT(40.97753129942237 73.96464939483981) bank88881 +88882 POINT(39.90570161011509 73.66170775408119) bank88882 +88883 POINT(39.75090078158309 73.0144008924414) bank88883 +88884 POINT(40.90791752738849 73.25563447317191) bank88884 +88885 POINT(40.022898183268836 73.10081941744576) bank88885 +88886 POINT(40.36575217564419 73.17390419840089) bank88886 +88887 POINT(40.57705971376142 74.41945091592781) bank88887 +88888 POINT(41.08691370598797 73.28734048367299) bank88888 +88889 POINT(40.08356041700559 74.64062807432491) bank88889 +88890 POINT(41.63286891157438 73.9482173222551) bank88890 +88891 POINT(40.19949297332469 73.04026919948222) bank88891 +88892 POINT(40.12506106102809 74.41250936998269) bank88892 +88893 POINT(39.96892568864786 74.0688614617606) bank88893 +88894 POINT(41.34301428426651 74.2276998605708) bank88894 +88895 POINT(39.75859465443491 74.3961513848383) bank88895 +88896 POINT(39.79393896313036 74.47169502935422) bank88896 +88897 POINT(40.59212548961333 73.65043959168364) bank88897 +88898 POINT(41.331611965944205 73.25054790389551) bank88898 +88899 POINT(40.676407937086815 74.51381387549934) bank88899 +88900 POINT(41.03660813682884 74.24582982424252) bank88900 +88901 POINT(40.207561802952036 73.84066966878878) bank88901 +88902 POINT(41.434462742878274 73.49175414201093) bank88902 +88903 POINT(40.042091236591155 74.90433727223329) bank88903 +88904 POINT(41.00035927374137 74.74281204058803) bank88904 +88905 POINT(39.8593819169149 74.48219803989274) bank88905 +88906 POINT(40.82950556423053 73.02326386462394) bank88906 +88907 POINT(40.70354770958013 73.77605788797837) bank88907 +88908 POINT(40.16283903349842 73.56409361249045) bank88908 +88909 POINT(40.99215011569587 73.07463048685348) bank88909 +88910 POINT(41.00300820276114 73.40524603370528) bank88910 +88911 POINT(40.5936019873725 74.59733008523087) bank88911 +88912 POINT(41.268332177083366 73.58913072854303) bank88912 +88913 POINT(39.81303332530201 74.2315689760548) bank88913 +88914 POINT(41.47761586405081 73.60323211323565) bank88914 +88915 POINT(41.20203889569201 74.54393567351592) bank88915 +88916 POINT(40.58866614437814 73.47775315389754) bank88916 +88917 POINT(41.545177510047 73.50276783985666) bank88917 +88918 POINT(41.09055827101282 74.39789108019792) bank88918 +88919 POINT(41.044010974189575 74.46887055068888) bank88919 +88920 POINT(40.6672308992732 73.9342573120888) bank88920 +88921 POINT(40.206581613229254 73.05171652289762) bank88921 +88922 POINT(41.13595744319791 73.67491474050613) bank88922 +88923 POINT(41.39424011026046 73.7929756061891) bank88923 +88924 POINT(40.59770573152574 73.22750382592122) bank88924 +88925 POINT(41.49010895783804 73.27059871462762) bank88925 +88926 POINT(40.31005038443552 74.57546950852675) bank88926 +88927 POINT(41.217973093484034 74.50496953294098) bank88927 +88928 POINT(41.029394323664214 73.55135140204065) bank88928 +88929 POINT(40.831799933761715 74.59035834171765) bank88929 +88930 POINT(39.98030962098516 74.81677938861237) bank88930 +88931 POINT(40.4827959977658 74.30716710235801) bank88931 +88932 POINT(39.71784297687248 74.52320392219937) bank88932 +88933 POINT(39.848938166056726 73.34176780529627) bank88933 +88934 POINT(41.40859379979106 74.67916059646538) bank88934 +88935 POINT(41.706758076980734 74.09034304138989) bank88935 +88936 POINT(40.563055153635524 74.73193095287651) bank88936 +88937 POINT(40.30872822734643 74.55437675540955) bank88937 +88938 POINT(41.45740050526075 74.59341468665178) bank88938 +88939 POINT(40.27404556714555 73.34275682309416) bank88939 +88940 POINT(40.99378363666448 73.35917429886561) bank88940 +88941 POINT(39.98465214163014 73.6985517021103) bank88941 +88942 POINT(40.33797299880713 74.1060251490597) bank88942 +88943 POINT(40.04090189242547 73.57886608820685) bank88943 +88944 POINT(40.71936037228269 74.88693334813465) bank88944 +88945 POINT(40.38580518784086 74.92419741401011) bank88945 +88946 POINT(40.633749347228644 74.15265237480565) bank88946 +88947 POINT(41.614885041764055 74.33062680380144) bank88947 +88948 POINT(41.702630786729955 73.31599669389337) bank88948 +88949 POINT(40.28367350376734 74.69537804111437) bank88949 +88950 POINT(40.1180839330903 73.98918903876941) bank88950 +88951 POINT(40.9459084875075 74.41307131511454) bank88951 +88952 POINT(40.27567804262151 74.37389698568848) bank88952 +88953 POINT(40.72107908340038 74.18665273491177) bank88953 +88954 POINT(40.6173172222501 74.21060499169661) bank88954 +88955 POINT(40.07062114804546 73.12498927846563) bank88955 +88956 POINT(41.420644251211606 74.25699120464785) bank88956 +88957 POINT(39.758619179407106 73.66029920600403) bank88957 +88958 POINT(40.94868079209966 74.72888665853482) bank88958 +88959 POINT(40.210041811974754 74.7054419547866) bank88959 +88960 POINT(41.50571573869809 74.83859074389916) bank88960 +88961 POINT(39.93536769585801 73.59413604580834) bank88961 +88962 POINT(41.56666531591498 73.32858974776127) bank88962 +88963 POINT(40.375351613149476 74.86575258464738) bank88963 +88964 POINT(40.230895565497704 73.25695959853694) bank88964 +88965 POINT(40.45463376614626 74.84955283670433) bank88965 +88966 POINT(41.63298893526289 73.04439316308044) bank88966 +88967 POINT(40.62200251941592 74.29908372556301) bank88967 +88968 POINT(41.587293525932466 74.85308276707717) bank88968 +88969 POINT(40.15247254888047 74.07014719249604) bank88969 +88970 POINT(40.89995694184608 73.25065587550468) bank88970 +88971 POINT(40.12863601965306 74.59882218504885) bank88971 +88972 POINT(41.39560152026228 74.7021704326101) bank88972 +88973 POINT(40.65405051139084 74.71745860628029) bank88973 +88974 POINT(41.0182802538536 73.48900613637272) bank88974 +88975 POINT(41.36664620927454 73.68426931002827) bank88975 +88976 POINT(40.36888817114385 73.93119864622084) bank88976 +88977 POINT(40.16663172905398 73.87907269964505) bank88977 +88978 POINT(40.30319328195317 73.66447710041574) bank88978 +88979 POINT(40.20507798575362 73.36598278414084) bank88979 +88980 POINT(39.93415748104034 74.67484348571737) bank88980 +88981 POINT(41.04573348759038 73.90819796053772) bank88981 +88982 POINT(40.0890784193608 73.36808100742789) bank88982 +88983 POINT(40.72237697613699 74.57619682909383) bank88983 +88984 POINT(40.27789363895629 73.40617487382062) bank88984 +88985 POINT(40.73559176308334 74.12108721001165) bank88985 +88986 POINT(41.126036352162224 74.36862052060157) bank88986 +88987 POINT(40.23948429652501 74.05255849057232) bank88987 +88988 POINT(39.86830485107336 73.20298646193733) bank88988 +88989 POINT(39.81931607378111 74.54312768952092) bank88989 +88990 POINT(40.4162532898677 73.8800488723921) bank88990 +88991 POINT(40.08843329474161 73.5607084861348) bank88991 +88992 POINT(40.7055778325708 73.93193949903126) bank88992 +88993 POINT(41.23849763723955 73.21439737542796) bank88993 +88994 POINT(41.454237645248654 73.61077186878795) bank88994 +88995 POINT(40.12900837245848 73.26292559692679) bank88995 +88996 POINT(40.14475469944559 73.32078391931465) bank88996 +88997 POINT(41.59305118018192 73.15338447249033) bank88997 +88998 POINT(40.93497160324415 73.06334090994828) bank88998 +88999 POINT(40.66045811819847 73.44109795947995) bank88999 +89000 POINT(40.921581339681104 74.31580045384764) bank89000 +89001 POINT(39.8985035458761 73.31703625848819) bank89001 +89002 POINT(39.81866239342442 73.80044692597355) bank89002 +89003 POINT(41.05050585946525 74.41546954040247) bank89003 +89004 POINT(40.00881674099854 73.30570860343934) bank89004 +89005 POINT(40.00018735200245 73.10973489135621) bank89005 +89006 POINT(41.1170487985985 74.58110098897653) bank89006 +89007 POINT(41.641408140315235 74.98149630577737) bank89007 +89008 POINT(40.56058292779546 73.39565585444929) bank89008 +89009 POINT(40.20342573523785 73.38676377859812) bank89009 +89010 POINT(41.012681562529664 73.75203025326113) bank89010 +89011 POINT(39.84206368933685 73.46346869694104) bank89011 +89012 POINT(40.412862887828155 74.35396011446423) bank89012 +89013 POINT(40.67216373607093 74.30508588965152) bank89013 +89014 POINT(40.89737869398052 73.53816360079553) bank89014 +89015 POINT(41.36232890334136 73.2744240093976) bank89015 +89016 POINT(41.1305577747089 73.12893576583117) bank89016 +89017 POINT(39.93698444928915 74.00197427397086) bank89017 +89018 POINT(40.9955147842615 74.87811063462958) bank89018 +89019 POINT(41.338255545249254 74.56513862493098) bank89019 +89020 POINT(40.94041404807975 74.42743322739652) bank89020 +89021 POINT(41.66466433104307 74.63259437338247) bank89021 +89022 POINT(41.016700027612366 74.91524539677505) bank89022 +89023 POINT(41.21072993952108 73.52028599089068) bank89023 +89024 POINT(41.33290643191905 74.15345453074616) bank89024 +89025 POINT(41.08079149960013 74.11570623892023) bank89025 +89026 POINT(40.5227076126893 74.9142360955167) bank89026 +89027 POINT(39.75060073235557 74.47317746722251) bank89027 +89028 POINT(41.09707642093999 73.44770824350726) bank89028 +89029 POINT(40.86681262223934 74.59130746966102) bank89029 +89030 POINT(40.9222401355941 74.27493475192715) bank89030 +89031 POINT(40.14851418533452 73.1952832727057) bank89031 +89032 POINT(41.42235665251896 73.23559144432568) bank89032 +89033 POINT(40.492784526119436 73.72923666250965) bank89033 +89034 POINT(41.415626070082034 74.81014893190611) bank89034 +89035 POINT(40.204454866829 74.62896802759698) bank89035 +89036 POINT(40.678557376835606 73.9457612540066) bank89036 +89037 POINT(39.91025024681227 74.92136918998654) bank89037 +89038 POINT(40.90721844815071 74.3374245672619) bank89038 +89039 POINT(41.50947760486638 73.46786522197472) bank89039 +89040 POINT(39.96608133079548 74.38206554189982) bank89040 +89041 POINT(39.98340592633326 74.35282735933356) bank89041 +89042 POINT(40.25482496545637 74.79869991002893) bank89042 +89043 POINT(40.59106093878827 74.60685308118319) bank89043 +89044 POINT(40.5584070416478 74.28940591962161) bank89044 +89045 POINT(40.639415976884834 74.29692856134179) bank89045 +89046 POINT(40.47049520159984 74.64133745221963) bank89046 +89047 POINT(40.568847941198726 73.98610063249052) bank89047 +89048 POINT(41.33782265088383 73.92856564324039) bank89048 +89049 POINT(39.94192780705614 73.55008002870794) bank89049 +89050 POINT(40.42543755463413 74.53832664691568) bank89050 +89051 POINT(41.667207032702805 73.93786410924608) bank89051 +89052 POINT(41.35248928256813 74.68391408140403) bank89052 +89053 POINT(41.625000380285194 74.36716738681736) bank89053 +89054 POINT(40.83489322589494 73.135123888072) bank89054 +89055 POINT(41.588194447598255 73.83061754149291) bank89055 +89056 POINT(40.28087127229679 74.43816908206938) bank89056 +89057 POINT(41.428646886907565 73.5679583003724) bank89057 +89058 POINT(39.747563189494855 73.43407714442537) bank89058 +89059 POINT(41.061186783126786 74.89382212229489) bank89059 +89060 POINT(41.5218172232818 74.79678968404649) bank89060 +89061 POINT(40.833378774330974 74.64458597506257) bank89061 +89062 POINT(39.92188638726942 74.9191266547787) bank89062 +89063 POINT(41.62684037546294 74.22944124529364) bank89063 +89064 POINT(41.187940979122985 73.05218494740002) bank89064 +89065 POINT(40.36707037377405 74.20114864587688) bank89065 +89066 POINT(40.25693276395844 74.47598073625348) bank89066 +89067 POINT(40.78987467402507 74.67549031839532) bank89067 +89068 POINT(40.01827061096631 73.94514609332431) bank89068 +89069 POINT(39.89266431636528 74.43785500861726) bank89069 +89070 POINT(40.53452678872075 73.6477315083265) bank89070 +89071 POINT(40.00721604444122 73.92252619824446) bank89071 +89072 POINT(39.819443567279464 73.8840468240962) bank89072 +89073 POINT(40.149775915184975 74.39397072362196) bank89073 +89074 POINT(41.12896585187023 73.59696654687461) bank89074 +89075 POINT(40.59321549533648 74.44869498039914) bank89075 +89076 POINT(41.51065663589175 74.98962600789307) bank89076 +89077 POINT(40.79882109493212 74.37447601987263) bank89077 +89078 POINT(39.875085446972534 73.28118121751093) bank89078 +89079 POINT(40.50352874606597 73.05117343341712) bank89079 +89080 POINT(41.28649704812715 74.6463032565815) bank89080 +89081 POINT(41.2950408845338 74.20118688526794) bank89081 +89082 POINT(40.79647706276435 74.69423617436834) bank89082 +89083 POINT(40.963344309397634 73.79631568951588) bank89083 +89084 POINT(39.95963911753307 74.90287677284691) bank89084 +89085 POINT(40.51791582439075 74.94298781250787) bank89085 +89086 POINT(41.69562982157842 74.60196834908221) bank89086 +89087 POINT(41.48131804744265 74.51020661219307) bank89087 +89088 POINT(39.88518187495225 74.35454147379251) bank89088 +89089 POINT(40.87989042930771 73.30183083892884) bank89089 +89090 POINT(40.20172529226589 73.073910324837) bank89090 +89091 POINT(41.04269998432066 73.15779227418054) bank89091 +89092 POINT(40.91905991048717 74.68569677259538) bank89092 +89093 POINT(40.289625858796505 74.23672201650771) bank89093 +89094 POINT(40.88854320856634 74.07719209951019) bank89094 +89095 POINT(41.26172911493396 74.66579038454557) bank89095 +89096 POINT(40.602517608531485 73.94394189636517) bank89096 +89097 POINT(40.590470270065914 73.89319078673542) bank89097 +89098 POINT(41.59615347427442 74.2972128981724) bank89098 +89099 POINT(39.76180009867396 74.56460336888723) bank89099 +89100 POINT(41.448441387808344 73.38221948683051) bank89100 +89101 POINT(39.82651723363746 74.43910031154157) bank89101 +89102 POINT(41.31360786710212 73.2036018756772) bank89102 +89103 POINT(41.39927899039215 73.83431018332261) bank89103 +89104 POINT(40.47280675335342 74.87151840081314) bank89104 +89105 POINT(41.35719759877735 73.54868218396824) bank89105 +89106 POINT(41.4999277873216 74.27337501033526) bank89106 +89107 POINT(40.4363187165347 73.63358882240188) bank89107 +89108 POINT(41.683704877877226 74.03346439351374) bank89108 +89109 POINT(40.35078857844983 73.47014925136622) bank89109 +89110 POINT(40.846928563164724 73.87773578952508) bank89110 +89111 POINT(40.13503070753534 74.90900888886233) bank89111 +89112 POINT(40.82120520311553 74.36934818151198) bank89112 +89113 POINT(41.199726099060584 73.66797204857751) bank89113 +89114 POINT(40.73310457416397 74.41035866584032) bank89114 +89115 POINT(41.5136194898772 74.2691821270906) bank89115 +89116 POINT(41.52250184326052 73.5828807346) bank89116 +89117 POINT(40.834058849632214 73.93801842930725) bank89117 +89118 POINT(39.78244148601616 74.66766404814082) bank89118 +89119 POINT(39.7724139806356 74.48561833885236) bank89119 +89120 POINT(39.73982536627326 73.90691853181006) bank89120 +89121 POINT(40.54682000459465 74.07545972474364) bank89121 +89122 POINT(41.138822003672296 74.1157711442884) bank89122 +89123 POINT(40.4420044286896 74.60694271133703) bank89123 +89124 POINT(40.949415102871555 73.88696220116856) bank89124 +89125 POINT(41.652070255992676 73.95885000392714) bank89125 +89126 POINT(40.35072138242165 73.67586424414753) bank89126 +89127 POINT(40.43726374351882 74.9448414979883) bank89127 +89128 POINT(41.109762862078746 73.58099799433907) bank89128 +89129 POINT(41.24841376553307 73.61769997152811) bank89129 +89130 POINT(40.490525715192966 74.0162204241734) bank89130 +89131 POINT(41.077009287879704 74.1914120593677) bank89131 +89132 POINT(40.04065891912927 73.5356175604627) bank89132 +89133 POINT(41.596379945185774 74.75361249584908) bank89133 +89134 POINT(39.74649862430194 74.35917065639713) bank89134 +89135 POINT(41.34889377399135 74.7330379645261) bank89135 +89136 POINT(39.92748925881157 74.80936967576982) bank89136 +89137 POINT(41.36237528246776 74.33913808268329) bank89137 +89138 POINT(40.422513798986415 74.04818138845285) bank89138 +89139 POINT(41.35227414092324 74.76621402953778) bank89139 +89140 POINT(41.20458729188236 73.11215897339596) bank89140 +89141 POINT(41.051519158267496 74.3324662272089) bank89141 +89142 POINT(40.92364020740746 74.87329748863549) bank89142 +89143 POINT(40.575632086955494 73.19376494673156) bank89143 +89144 POINT(39.97246632866622 73.90989760810636) bank89144 +89145 POINT(41.64286483882876 74.57194578180263) bank89145 +89146 POINT(39.97093670793953 73.94947541389021) bank89146 +89147 POINT(40.244697304142484 73.398885852649) bank89147 +89148 POINT(40.60601021406923 73.01215677705397) bank89148 +89149 POINT(40.789409520161705 74.26080852786656) bank89149 +89150 POINT(39.91395911676535 74.04662112461027) bank89150 +89151 POINT(40.032975235485196 73.75944902186205) bank89151 +89152 POINT(41.08545758433495 73.50959669289462) bank89152 +89153 POINT(40.11058301870593 74.75124691767263) bank89153 +89154 POINT(40.65914205029791 73.94904500823368) bank89154 +89155 POINT(40.35018224416368 73.37010749139557) bank89155 +89156 POINT(40.56021592696784 74.63761841370828) bank89156 +89157 POINT(40.10211136089349 73.84587623103681) bank89157 +89158 POINT(41.55906868670176 73.21733770185172) bank89158 +89159 POINT(41.54452382715416 73.99128115700742) bank89159 +89160 POINT(40.127342120621854 74.80678461360935) bank89160 +89161 POINT(40.1663941787997 73.56787605224494) bank89161 +89162 POINT(40.39782625281414 73.25332875500942) bank89162 +89163 POINT(41.39126563269869 73.81975972055952) bank89163 +89164 POINT(41.646249966961946 73.48242956361658) bank89164 +89165 POINT(40.25066593388429 73.01821916756175) bank89165 +89166 POINT(41.38917428696749 74.1548377623755) bank89166 +89167 POINT(39.94615868126568 73.7504965462328) bank89167 +89168 POINT(40.39625248647452 74.6364512597512) bank89168 +89169 POINT(40.224708255465316 73.6734176437773) bank89169 +89170 POINT(40.526931579709796 74.45125228435464) bank89170 +89171 POINT(41.111555506670946 74.22189299748293) bank89171 +89172 POINT(40.66006475265448 73.58436013888057) bank89172 +89173 POINT(40.173225081032136 73.73977604255995) bank89173 +89174 POINT(39.71966330897963 74.05675597622042) bank89174 +89175 POINT(41.34330742822084 74.95079380794921) bank89175 +89176 POINT(40.147098970851765 74.18467653897407) bank89176 +89177 POINT(41.50023031332494 73.67526243674727) bank89177 +89178 POINT(40.56399122472659 74.95682501781668) bank89178 +89179 POINT(40.204487658942845 74.39311502286706) bank89179 +89180 POINT(40.19056194407708 74.03856380744091) bank89180 +89181 POINT(39.778482851326864 73.13744782003667) bank89181 +89182 POINT(41.34804871413845 73.64721836642858) bank89182 +89183 POINT(41.459620281949064 74.55875466178641) bank89183 +89184 POINT(40.76712515093951 74.24176571155985) bank89184 +89185 POINT(40.641726861623525 73.26549741867858) bank89185 +89186 POINT(39.881902475690154 73.22896413024787) bank89186 +89187 POINT(41.586386754539404 73.23472660232633) bank89187 +89188 POINT(39.810576183910875 73.1128696772727) bank89188 +89189 POINT(40.73500927681954 74.21939467841354) bank89189 +89190 POINT(39.95497684305918 74.78841710727541) bank89190 +89191 POINT(39.727331072777474 74.27472236010345) bank89191 +89192 POINT(40.34816521915316 74.64376619592616) bank89192 +89193 POINT(40.40930076730947 73.33442047215061) bank89193 +89194 POINT(41.42882356527313 73.63880211185939) bank89194 +89195 POINT(41.415671980773354 73.23628399927335) bank89195 +89196 POINT(40.44988925256117 73.19600675411822) bank89196 +89197 POINT(40.717865777409166 74.31761226799941) bank89197 +89198 POINT(41.324829333079286 73.35574755426954) bank89198 +89199 POINT(41.216518585167094 74.57894215746225) bank89199 +89200 POINT(41.050381055241374 74.27477899487195) bank89200 +89201 POINT(40.61157928436012 73.97551174162005) bank89201 +89202 POINT(39.85512653967782 74.1029722055364) bank89202 +89203 POINT(40.462013689767126 73.39129030503615) bank89203 +89204 POINT(40.60491516495254 74.41773721957475) bank89204 +89205 POINT(41.44885150840578 73.9808591529544) bank89205 +89206 POINT(41.5349886262024 74.30410940755593) bank89206 +89207 POINT(40.24702947098648 73.43453038261408) bank89207 +89208 POINT(41.694656974415054 73.82748020502048) bank89208 +89209 POINT(41.01538146178823 73.56929446564916) bank89209 +89210 POINT(40.93952285261805 74.18499849686158) bank89210 +89211 POINT(40.40102216713545 74.81286502808895) bank89211 +89212 POINT(41.164798418212875 74.11453591723013) bank89212 +89213 POINT(41.266505419278424 74.34816152680237) bank89213 +89214 POINT(40.029302821201185 73.95383020467439) bank89214 +89215 POINT(41.49208817534928 73.28647367955142) bank89215 +89216 POINT(39.92310417865005 73.81437100523365) bank89216 +89217 POINT(41.16446691999497 73.89515978876561) bank89217 +89218 POINT(41.06657319334017 74.76508519007572) bank89218 +89219 POINT(40.2513519923924 73.81252045432038) bank89219 +89220 POINT(41.67382785849372 74.03706284909505) bank89220 +89221 POINT(40.1971882216643 74.27467445650137) bank89221 +89222 POINT(39.723230352706125 73.35986930573509) bank89222 +89223 POINT(39.94241742106986 74.8229616365975) bank89223 +89224 POINT(39.88314376722481 74.47180814468177) bank89224 +89225 POINT(41.40635781958219 74.61124664586373) bank89225 +89226 POINT(40.68750190409515 74.87940926136551) bank89226 +89227 POINT(41.00279236725326 73.75828596273897) bank89227 +89228 POINT(40.925922726230574 74.15420216104151) bank89228 +89229 POINT(40.306949520970846 73.3007822034712) bank89229 +89230 POINT(39.930053477929235 74.34245545970633) bank89230 +89231 POINT(40.82922010216161 74.80399050826907) bank89231 +89232 POINT(40.657838088155586 74.08049606730548) bank89232 +89233 POINT(40.54514654163368 74.09085801764887) bank89233 +89234 POINT(41.0696804711566 74.95728641859863) bank89234 +89235 POINT(39.95164787096785 73.28584617448675) bank89235 +89236 POINT(41.226294905942204 73.93819383777203) bank89236 +89237 POINT(41.31507803530751 74.65219523369595) bank89237 +89238 POINT(40.47660943351428 73.53237629428078) bank89238 +89239 POINT(40.63938788533197 73.17876424337507) bank89239 +89240 POINT(40.27007575598764 74.81009363389958) bank89240 +89241 POINT(41.304665366076456 73.16701732710399) bank89241 +89242 POINT(41.70422762921251 74.60568452661305) bank89242 +89243 POINT(40.61613874272588 74.14278679492634) bank89243 +89244 POINT(40.932177251194204 73.83684849621471) bank89244 +89245 POINT(40.054230227229986 73.97330690841375) bank89245 +89246 POINT(40.30341808406983 74.2767571399906) bank89246 +89247 POINT(41.59270431137941 74.2581089465915) bank89247 +89248 POINT(40.63691359640405 73.75095338937487) bank89248 +89249 POINT(40.573303286007786 74.79132484843002) bank89249 +89250 POINT(40.94545317195119 74.0010495979422) bank89250 +89251 POINT(40.0281340200647 74.05942174789155) bank89251 +89252 POINT(41.450108325145976 73.23619819603742) bank89252 +89253 POINT(41.42448045285987 74.64098977993886) bank89253 +89254 POINT(40.330838490646244 73.75657405290467) bank89254 +89255 POINT(40.0471196507969 73.5856434199018) bank89255 +89256 POINT(41.19001380788431 73.16938439849322) bank89256 +89257 POINT(41.248393466930814 74.14097608212437) bank89257 +89258 POINT(39.903529938405704 74.72097875045102) bank89258 +89259 POINT(41.51815100042857 74.71152948141113) bank89259 +89260 POINT(40.92176516850624 74.6336642038958) bank89260 +89261 POINT(39.7791643579862 73.37209430434605) bank89261 +89262 POINT(40.49465796158012 73.70541508643615) bank89262 +89263 POINT(41.612356869539596 73.24902769876947) bank89263 +89264 POINT(41.06702118044701 74.67718545084765) bank89264 +89265 POINT(41.432827613608175 73.95340479307798) bank89265 +89266 POINT(40.18188798844912 74.77139595091589) bank89266 +89267 POINT(41.66129740922318 74.44493174850452) bank89267 +89268 POINT(39.768417221933646 73.92394250829145) bank89268 +89269 POINT(40.358699441171666 74.99796900899572) bank89269 +89270 POINT(40.16638488705958 73.17513776359) bank89270 +89271 POINT(41.311890095291524 74.4983861048224) bank89271 +89272 POINT(40.58828199499927 73.70306868851273) bank89272 +89273 POINT(39.94869753356157 74.69570704398807) bank89273 +89274 POINT(40.1565959842223 74.96524939260283) bank89274 +89275 POINT(41.305798579600555 74.7766055144085) bank89275 +89276 POINT(39.858391329271676 73.59209945841893) bank89276 +89277 POINT(40.54154768089257 73.91041291110764) bank89277 +89278 POINT(41.2049665063256 74.90940454243594) bank89278 +89279 POINT(41.69496673195704 74.56281346404198) bank89279 +89280 POINT(40.22310557637992 73.82092410758135) bank89280 +89281 POINT(41.354325035318936 73.47104827682338) bank89281 +89282 POINT(41.04918235735197 74.34792766300158) bank89282 +89283 POINT(40.349834269601935 74.90271440377495) bank89283 +89284 POINT(41.21170162172053 74.86577482740664) bank89284 +89285 POINT(41.56645972011541 73.99819789178855) bank89285 +89286 POINT(40.96409841922684 73.15405423147286) bank89286 +89287 POINT(41.29324517108803 74.82993572947373) bank89287 +89288 POINT(41.36051561414017 74.84376084039961) bank89288 +89289 POINT(39.95569133014176 73.48493643028695) bank89289 +89290 POINT(41.59553375389455 74.68647987273528) bank89290 +89291 POINT(40.299730945319496 75.0010324928723) bank89291 +89292 POINT(41.47986258618689 74.33376680431623) bank89292 +89293 POINT(39.744623932252125 73.99300535504612) bank89293 +89294 POINT(40.87261700363424 74.87515849574712) bank89294 +89295 POINT(40.716506596377634 74.82388287676217) bank89295 +89296 POINT(40.2480721960314 73.89382744249835) bank89296 +89297 POINT(40.965093397579295 73.58894312472643) bank89297 +89298 POINT(40.293822862691876 74.56669862825072) bank89298 +89299 POINT(40.611985290754724 74.74720802041249) bank89299 +89300 POINT(41.391038787186424 74.64083252512499) bank89300 +89301 POINT(41.559845840877195 74.74952803458164) bank89301 +89302 POINT(41.61062563357558 74.97918033520264) bank89302 +89303 POINT(41.48769293265551 73.30179795059047) bank89303 +89304 POINT(40.459368860305396 73.47414578157118) bank89304 +89305 POINT(41.60159405510721 73.7476888988942) bank89305 +89306 POINT(41.25887576614681 74.51897048155912) bank89306 +89307 POINT(41.32953458411913 74.62183670689315) bank89307 +89308 POINT(41.132978540376165 74.42368619814751) bank89308 +89309 POINT(39.76595227382622 73.7401970490453) bank89309 +89310 POINT(40.006061792775476 74.80339559974735) bank89310 +89311 POINT(41.671844800979315 73.02006863168485) bank89311 +89312 POINT(41.37020439654679 74.2587173933786) bank89312 +89313 POINT(40.778631900156036 73.23914154557451) bank89313 +89314 POINT(40.22218095360158 73.95078053430015) bank89314 +89315 POINT(40.93586867405662 73.1982586144302) bank89315 +89316 POINT(41.00397289051201 74.78089703530915) bank89316 +89317 POINT(39.7526474655405 73.90869689125746) bank89317 +89318 POINT(39.77567654900162 73.10017714159584) bank89318 +89319 POINT(41.603708448670965 74.25444194167869) bank89319 +89320 POINT(40.30790150741813 74.65787669372246) bank89320 +89321 POINT(40.16553974503473 73.77970929030579) bank89321 +89322 POINT(41.52759229283913 74.45750299132935) bank89322 +89323 POINT(40.65658744282346 73.19886497010228) bank89323 +89324 POINT(41.16717984769989 73.86345134696117) bank89324 +89325 POINT(41.25477820105782 73.63798659674221) bank89325 +89326 POINT(40.236948738268644 73.01543208316166) bank89326 +89327 POINT(40.63881742778104 74.21965466035421) bank89327 +89328 POINT(40.14768548265367 73.10203461135472) bank89328 +89329 POINT(41.55969079710899 73.37901234342381) bank89329 +89330 POINT(41.13221909842581 74.08103832186556) bank89330 +89331 POINT(41.39373653036979 73.66767721628989) bank89331 +89332 POINT(40.53532247616233 74.78492306990672) bank89332 +89333 POINT(40.696611722153676 74.10845440980079) bank89333 +89334 POINT(40.10855824592035 73.38691705523271) bank89334 +89335 POINT(41.04187497807644 74.07751944054118) bank89335 +89336 POINT(41.631434303540345 74.07019323332987) bank89336 +89337 POINT(41.37831491422867 73.59695181065635) bank89337 +89338 POINT(40.29002863066633 73.72124239640873) bank89338 +89339 POINT(40.251664197843944 73.21160932929376) bank89339 +89340 POINT(40.54176554082254 74.60574939665754) bank89340 +89341 POINT(40.54965920365987 73.60768813093983) bank89341 +89342 POINT(39.72780479483545 73.02775320910013) bank89342 +89343 POINT(40.86969388335479 73.06124695754264) bank89343 +89344 POINT(41.517691152219115 73.88627500846775) bank89344 +89345 POINT(39.74417679284891 73.84582191494249) bank89345 +89346 POINT(41.32892059316131 74.39679431492746) bank89346 +89347 POINT(41.007773552527674 73.45200738902747) bank89347 +89348 POINT(40.15779190251853 74.70299771137445) bank89348 +89349 POINT(41.330650872910304 74.09221268481305) bank89349 +89350 POINT(40.3701714702351 73.13262472670908) bank89350 +89351 POINT(41.278179090570504 74.94135740968825) bank89351 +89352 POINT(39.97823236196528 73.9303078969925) bank89352 +89353 POINT(40.98653458409059 74.70862599440724) bank89353 +89354 POINT(41.14928741457254 74.55996553319306) bank89354 +89355 POINT(39.80963269096021 74.74795518684084) bank89355 +89356 POINT(41.40999038003885 74.22204917756721) bank89356 +89357 POINT(40.86783343825014 74.53899757711822) bank89357 +89358 POINT(40.89076208855136 73.66700576524943) bank89358 +89359 POINT(41.13252180316069 73.16823868449671) bank89359 +89360 POINT(39.862724829237486 74.27748978196716) bank89360 +89361 POINT(40.632987195172745 73.08501531274825) bank89361 +89362 POINT(41.52173809023611 74.06763354845197) bank89362 +89363 POINT(39.75575971516079 73.47612308429083) bank89363 +89364 POINT(41.54856966002878 73.31035850964726) bank89364 +89365 POINT(40.17139811225949 74.28783389224296) bank89365 +89366 POINT(40.54836274538499 74.0516064605305) bank89366 +89367 POINT(41.33629770492137 73.89197963709931) bank89367 +89368 POINT(40.29848097841942 74.4897685263272) bank89368 +89369 POINT(39.910921483627554 74.87601882254427) bank89369 +89370 POINT(40.38540232195691 73.95829459111424) bank89370 +89371 POINT(40.99984693374664 73.02278952889493) bank89371 +89372 POINT(40.146521629968554 73.34810494485643) bank89372 +89373 POINT(40.86670129363003 74.92636656385156) bank89373 +89374 POINT(40.19685329216015 73.4984988261107) bank89374 +89375 POINT(40.60053541915638 73.58993096173845) bank89375 +89376 POINT(41.20599475233781 73.25585004817064) bank89376 +89377 POINT(40.66478500868893 73.98312396898174) bank89377 +89378 POINT(41.586836316945295 73.04856022783156) bank89378 +89379 POINT(40.40930880341781 73.8358546972108) bank89379 +89380 POINT(41.539651745256066 74.53917937789674) bank89380 +89381 POINT(41.45100286858241 73.24806415986882) bank89381 +89382 POINT(40.775416800085104 73.36182827180127) bank89382 +89383 POINT(40.456119764979725 73.82872385260917) bank89383 +89384 POINT(41.184618488716396 73.612420175864) bank89384 +89385 POINT(40.770779518286844 74.41402099210995) bank89385 +89386 POINT(40.10079791914343 73.92391928394501) bank89386 +89387 POINT(40.55459602865462 74.34076642527047) bank89387 +89388 POINT(39.71315400085868 73.76928147246461) bank89388 +89389 POINT(41.261550155280304 73.03661883800346) bank89389 +89390 POINT(40.229738290407774 73.75332286226266) bank89390 +89391 POINT(41.45177289537317 73.82037205018352) bank89391 +89392 POINT(41.1184109671815 74.84357078483522) bank89392 +89393 POINT(41.57276443126533 74.10972737760468) bank89393 +89394 POINT(40.60601703803929 74.08506379629068) bank89394 +89395 POINT(40.25910682428331 73.87244385955468) bank89395 +89396 POINT(40.794385335818255 73.50159104318683) bank89396 +89397 POINT(40.50343075316762 74.40441619443868) bank89397 +89398 POINT(41.40340404558936 74.84937205353039) bank89398 +89399 POINT(40.81935296035019 74.39955991094033) bank89399 +89400 POINT(40.93529015736722 73.4092668102174) bank89400 +89401 POINT(40.86764857962645 74.46227937260973) bank89401 +89402 POINT(41.12299976663712 74.80558206409071) bank89402 +89403 POINT(40.04289733815522 73.46639282522823) bank89403 +89404 POINT(41.49162363344785 73.73973442095597) bank89404 +89405 POINT(39.837810575257066 74.90860911369407) bank89405 +89406 POINT(39.83474384882854 73.1737957449948) bank89406 +89407 POINT(40.985706847663955 74.9249747065641) bank89407 +89408 POINT(41.18711215672237 73.61794700016306) bank89408 +89409 POINT(39.90913848842701 73.66681675434455) bank89409 +89410 POINT(41.062728495210074 74.36532399406494) bank89410 +89411 POINT(40.72290509511576 74.34930118692249) bank89411 +89412 POINT(40.153579678133255 73.10477176665862) bank89412 +89413 POINT(40.245802958468985 74.41471444419594) bank89413 +89414 POINT(40.38972035188492 74.29383660225885) bank89414 +89415 POINT(41.49258380872366 73.50530241972513) bank89415 +89416 POINT(40.54495504007706 74.57007303190284) bank89416 +89417 POINT(41.2557069038057 74.49585587414761) bank89417 +89418 POINT(40.48550408102772 73.89668985692516) bank89418 +89419 POINT(41.48588297218201 74.42846182171981) bank89419 +89420 POINT(40.36717618557155 73.2659258335139) bank89420 +89421 POINT(40.68102483202522 73.69280797751152) bank89421 +89422 POINT(40.91486185746265 74.79673996179218) bank89422 +89423 POINT(41.64430014761793 74.31772653740053) bank89423 +89424 POINT(41.29746495555821 74.66129698551035) bank89424 +89425 POINT(40.90176750937113 73.26313332469105) bank89425 +89426 POINT(39.91373319303161 74.61590794333512) bank89426 +89427 POINT(40.78120103589315 73.29258199905829) bank89427 +89428 POINT(40.05288798152571 74.2505612462272) bank89428 +89429 POINT(39.71651322333421 73.34352024771611) bank89429 +89430 POINT(39.87991834717503 74.4372621293829) bank89430 +89431 POINT(40.17679932821853 74.87950572948576) bank89431 +89432 POINT(41.3092994127561 73.6374674850882) bank89432 +89433 POINT(41.44929309708676 73.03729413135568) bank89433 +89434 POINT(40.61494483929598 73.49516111072646) bank89434 +89435 POINT(41.24723703510532 73.89071961744504) bank89435 +89436 POINT(41.08253116833472 73.72660408080846) bank89436 +89437 POINT(41.412640530581044 74.2957731165868) bank89437 +89438 POINT(41.525399356998726 74.5960818014016) bank89438 +89439 POINT(41.59346500309476 74.91075031891668) bank89439 +89440 POINT(39.90717239521614 74.42738416816675) bank89440 +89441 POINT(40.86652605246217 73.70746872658826) bank89441 +89442 POINT(40.25071519267736 73.76024990095631) bank89442 +89443 POINT(40.215018293088924 74.19484331596091) bank89443 +89444 POINT(41.26691472785936 73.64555148371556) bank89444 +89445 POINT(40.16127592234824 74.01370699107703) bank89445 +89446 POINT(41.045661470106765 74.24356518897493) bank89446 +89447 POINT(40.47689117550546 73.73514469797531) bank89447 +89448 POINT(40.833160156439625 74.39975218797369) bank89448 +89449 POINT(40.39468878317359 73.84995877253327) bank89449 +89450 POINT(40.45497575935824 74.06100098199624) bank89450 +89451 POINT(40.21614563764791 73.52197936426178) bank89451 +89452 POINT(40.32723853335742 74.61243420737233) bank89452 +89453 POINT(41.39403421083029 73.32908092671526) bank89453 +89454 POINT(40.32075733756951 74.73061301251771) bank89454 +89455 POINT(41.286083609191174 73.86834304143431) bank89455 +89456 POINT(39.98323034701465 73.79227475009411) bank89456 +89457 POINT(40.46856561178746 73.45431589355854) bank89457 +89458 POINT(39.78455421216643 74.74901166435473) bank89458 +89459 POINT(40.63856062854602 74.21641789208032) bank89459 +89460 POINT(41.07019644191757 74.34248471404457) bank89460 +89461 POINT(41.67030724829007 74.5899522943232) bank89461 +89462 POINT(40.47951154576461 73.95765713664755) bank89462 +89463 POINT(41.18541198937464 74.5845992264167) bank89463 +89464 POINT(39.83944361392884 74.45840596144426) bank89464 +89465 POINT(41.60138176447152 74.09644480848021) bank89465 +89466 POINT(40.761245522492345 74.10137121508316) bank89466 +89467 POINT(40.568973503564095 73.65936276121) bank89467 +89468 POINT(41.11751645105585 74.87192922114144) bank89468 +89469 POINT(41.48120629589955 73.66122978562458) bank89469 +89470 POINT(40.47818928859504 74.6231372869818) bank89470 +89471 POINT(40.66958818276767 74.09913581378285) bank89471 +89472 POINT(40.75295191076684 74.88701889695798) bank89472 +89473 POINT(40.85578014113764 74.10407392172229) bank89473 +89474 POINT(41.57686478798507 74.58364352718326) bank89474 +89475 POINT(41.70992479068965 74.58803344976309) bank89475 +89476 POINT(40.4218690333831 74.86678550346002) bank89476 +89477 POINT(41.52642227486556 74.47938871742656) bank89477 +89478 POINT(40.781161445842464 73.66523957279752) bank89478 +89479 POINT(40.58004561724509 74.53763467352105) bank89479 +89480 POINT(41.566792112940846 73.65131443187686) bank89480 +89481 POINT(39.71616872355619 74.47007004478498) bank89481 +89482 POINT(40.91657971483394 73.29971027643964) bank89482 +89483 POINT(40.10665019051345 74.82725245751026) bank89483 +89484 POINT(40.638153495963095 74.0618445212805) bank89484 +89485 POINT(40.67741976072446 74.8147842285083) bank89485 +89486 POINT(41.17996121766643 74.07838092103749) bank89486 +89487 POINT(41.50458008009038 73.0565674126283) bank89487 +89488 POINT(39.90734251870754 73.36876978760466) bank89488 +89489 POINT(41.47783338995929 73.739989358372) bank89489 +89490 POINT(41.01644578717789 74.81959586599461) bank89490 +89491 POINT(41.302083968123725 74.9302000231094) bank89491 +89492 POINT(39.9945800588849 73.66500278711395) bank89492 +89493 POINT(41.584384079127034 74.71103672554064) bank89493 +89494 POINT(40.66147247452134 74.30102424495419) bank89494 +89495 POINT(41.68111272332145 73.66406319883781) bank89495 +89496 POINT(41.2380316895317 74.6587444494556) bank89496 +89497 POINT(40.51862615746968 73.78524346446318) bank89497 +89498 POINT(40.98574295180572 74.78404262866268) bank89498 +89499 POINT(39.98397407451205 73.98221343276393) bank89499 +89500 POINT(40.9219897879832 73.77086245771429) bank89500 +89501 POINT(41.20752231235526 74.45300988668394) bank89501 +89502 POINT(39.77901437074331 74.15597042635285) bank89502 +89503 POINT(41.41248880374164 73.37992248798895) bank89503 +89504 POINT(41.117902258023335 73.99673624925781) bank89504 +89505 POINT(39.887570413580285 74.77659693422223) bank89505 +89506 POINT(40.656053996814016 73.38072569978601) bank89506 +89507 POINT(40.21835117129602 74.66285861153032) bank89507 +89508 POINT(40.939347554520324 74.69203509260788) bank89508 +89509 POINT(41.15112996473733 74.7519216971129) bank89509 +89510 POINT(41.02354005372072 73.61376896265111) bank89510 +89511 POINT(41.44834336110038 74.0219703308096) bank89511 +89512 POINT(40.49754287441115 73.89955914238978) bank89512 +89513 POINT(41.03251746757519 74.46952921995045) bank89513 +89514 POINT(40.7005038847544 73.48169916937881) bank89514 +89515 POINT(41.02973457777748 73.35196029687144) bank89515 +89516 POINT(40.228567359957964 74.89651585515419) bank89516 +89517 POINT(40.437964569690706 73.96291902911369) bank89517 +89518 POINT(40.67532663999141 74.62291528457958) bank89518 +89519 POINT(39.808909657384234 73.24442950210586) bank89519 +89520 POINT(41.45860878323497 74.22018505600782) bank89520 +89521 POINT(41.419432201106844 73.79156765251254) bank89521 +89522 POINT(41.552590050713654 74.2172842114548) bank89522 +89523 POINT(40.44410024559652 73.24500165469512) bank89523 +89524 POINT(41.26091079961539 74.24309503626037) bank89524 +89525 POINT(40.888104912585504 73.70115067394558) bank89525 +89526 POINT(41.398440750016796 74.81762646072536) bank89526 +89527 POINT(39.91960328369432 73.08881319560287) bank89527 +89528 POINT(40.36936526169799 74.79660052593842) bank89528 +89529 POINT(39.76383420175237 73.09673035446016) bank89529 +89530 POINT(39.949434419348464 74.7476152622923) bank89530 +89531 POINT(40.920124377668444 73.45570137610251) bank89531 +89532 POINT(41.38306381660303 74.94607860249549) bank89532 +89533 POINT(40.988391401473976 73.69174998737051) bank89533 +89534 POINT(39.87161368327557 73.65705636480959) bank89534 +89535 POINT(40.785552260084785 73.09970315825692) bank89535 +89536 POINT(40.5168529695217 74.00655979904194) bank89536 +89537 POINT(40.7821315259204 74.06636567091022) bank89537 +89538 POINT(41.502493212916136 73.98961110035275) bank89538 +89539 POINT(40.21931589112078 73.7428250697567) bank89539 +89540 POINT(40.967810559545 73.49171620400304) bank89540 +89541 POINT(41.3372738406675 73.39918717667413) bank89541 +89542 POINT(41.41394527119903 73.25409415764621) bank89542 +89543 POINT(40.74374770119997 73.8858721965149) bank89543 +89544 POINT(41.704012946566095 73.88513789424906) bank89544 +89545 POINT(41.229004027617556 74.77721313282282) bank89545 +89546 POINT(40.342080920240015 74.58259269484505) bank89546 +89547 POINT(40.35884502091858 73.5528357877059) bank89547 +89548 POINT(41.26821711740917 73.80032201291542) bank89548 +89549 POINT(40.94442510709516 73.45833424366285) bank89549 +89550 POINT(40.608423429891566 74.69388424107865) bank89550 +89551 POINT(40.23405471493276 73.23078837125065) bank89551 +89552 POINT(40.66191994806469 74.47210766386918) bank89552 +89553 POINT(41.476892452493914 73.54746987263987) bank89553 +89554 POINT(40.83270160342033 74.59035813118508) bank89554 +89555 POINT(40.00956467971844 73.9887274419376) bank89555 +89556 POINT(40.02481331556466 73.26950573334464) bank89556 +89557 POINT(41.42041613903428 73.07324722260233) bank89557 +89558 POINT(40.99050181932044 74.71182247972685) bank89558 +89559 POINT(40.624344209084704 74.39553280671068) bank89559 +89560 POINT(41.14818461910643 73.25310381751) bank89560 +89561 POINT(41.10022292495067 74.91585462216878) bank89561 +89562 POINT(41.25074175662676 73.93071405925973) bank89562 +89563 POINT(40.45054444273319 74.17025767525232) bank89563 +89564 POINT(41.007983555090775 73.47633365011056) bank89564 +89565 POINT(41.21478871630863 74.80058334807994) bank89565 +89566 POINT(40.21211091322515 74.02879014544946) bank89566 +89567 POINT(41.04468410217382 74.66089935567493) bank89567 +89568 POINT(40.511606915614664 74.51960643066356) bank89568 +89569 POINT(40.907540400823706 73.6658776163332) bank89569 +89570 POINT(40.37792154483746 73.67053806696264) bank89570 +89571 POINT(39.955338939272835 73.55455052042477) bank89571 +89572 POINT(40.78866802853776 74.536000216184) bank89572 +89573 POINT(40.07988919418946 74.39048710402727) bank89573 +89574 POINT(40.18045433051389 73.8136355180237) bank89574 +89575 POINT(41.017886571169996 74.01355215082229) bank89575 +89576 POINT(41.694414207826576 74.47261938978235) bank89576 +89577 POINT(40.18234466643034 74.26597597350079) bank89577 +89578 POINT(40.81400810729045 74.62885174731696) bank89578 +89579 POINT(41.380747342056836 74.61471804878171) bank89579 +89580 POINT(41.357059984417944 74.54322559336013) bank89580 +89581 POINT(40.805999115434986 73.57941375764236) bank89581 +89582 POINT(40.79751836454571 74.52933241647033) bank89582 +89583 POINT(40.37992106537886 74.80988700871126) bank89583 +89584 POINT(41.310033806206484 73.41831417018312) bank89584 +89585 POINT(41.58872349365668 74.97399921898719) bank89585 +89586 POINT(40.258603583918664 74.80692449375772) bank89586 +89587 POINT(40.588243904367474 73.12273688064039) bank89587 +89588 POINT(41.16838748399042 74.05001464299843) bank89588 +89589 POINT(40.23748186212683 73.41946109431959) bank89589 +89590 POINT(40.63669303871761 74.27795707620321) bank89590 +89591 POINT(40.63118990850828 74.35902720868651) bank89591 +89592 POINT(40.59147546009151 73.83934756418698) bank89592 +89593 POINT(41.17974665031231 73.56374222603185) bank89593 +89594 POINT(41.160639003424684 74.94273718628683) bank89594 +89595 POINT(40.005943646117196 73.94335096473311) bank89595 +89596 POINT(40.69781729558551 74.32373976780664) bank89596 +89597 POINT(41.46120754255455 73.07019389011992) bank89597 +89598 POINT(40.76750185653838 74.45094048973905) bank89598 +89599 POINT(40.92672858480784 74.00040046979805) bank89599 +89600 POINT(40.44381273181713 73.72786278776279) bank89600 +89601 POINT(41.52182513803425 73.16531303886042) bank89601 +89602 POINT(41.242630416941054 73.40746919149811) bank89602 +89603 POINT(40.313805572768516 73.28534931844516) bank89603 +89604 POINT(40.86850774549621 74.57852804166157) bank89604 +89605 POINT(40.62942046006517 74.08438055664433) bank89605 +89606 POINT(40.864440725350704 74.36900977822307) bank89606 +89607 POINT(41.20540446821828 73.173707227945) bank89607 +89608 POINT(41.277521245828716 73.21795323816869) bank89608 +89609 POINT(40.5173072317713 73.43062319111974) bank89609 +89610 POINT(41.66962533827728 73.31855745110454) bank89610 +89611 POINT(40.220432057991836 74.64303847669898) bank89611 +89612 POINT(40.3845031292824 74.68691843825026) bank89612 +89613 POINT(40.13582513943475 73.6908212089826) bank89613 +89614 POINT(40.52025904324202 74.87947338769246) bank89614 +89615 POINT(40.95403770580386 73.46554138227383) bank89615 +89616 POINT(40.98836476531097 73.3892029035012) bank89616 +89617 POINT(40.74020131912343 74.73195380203988) bank89617 +89618 POINT(40.45761062755599 73.87810760551929) bank89618 +89619 POINT(41.6260923266447 74.3549441805696) bank89619 +89620 POINT(41.078462925811486 73.60480192754446) bank89620 +89621 POINT(40.69176742304126 74.96828636298605) bank89621 +89622 POINT(40.257297177184576 73.12441440758799) bank89622 +89623 POINT(40.41586878260559 73.33579004886343) bank89623 +89624 POINT(40.17073256144711 73.38019046373945) bank89624 +89625 POINT(41.58093153891857 73.78128129076619) bank89625 +89626 POINT(41.40142676723067 73.48786786159476) bank89626 +89627 POINT(40.6674352998884 74.24324141567577) bank89627 +89628 POINT(41.6055518215811 73.2313678878909) bank89628 +89629 POINT(41.683148623917496 73.57783530263764) bank89629 +89630 POINT(40.26350191137213 73.59601142132834) bank89630 +89631 POINT(39.80938439700604 74.36316089039495) bank89631 +89632 POINT(39.860271376187285 73.98509067726096) bank89632 +89633 POINT(40.23239747104601 74.73300418951955) bank89633 +89634 POINT(40.70677979437266 73.8685500250822) bank89634 +89635 POINT(41.57494948258762 74.57939881273961) bank89635 +89636 POINT(40.558460537254426 74.66747590762921) bank89636 +89637 POINT(40.247124629513976 73.73638733783194) bank89637 +89638 POINT(41.18320334580854 74.53655873890591) bank89638 +89639 POINT(41.35745610640308 74.62534199334044) bank89639 +89640 POINT(40.2471370280516 74.99623791534138) bank89640 +89641 POINT(41.214588020337864 74.0998286442635) bank89641 +89642 POINT(41.093256684092424 74.05716615320912) bank89642 +89643 POINT(39.952971043270594 74.25487495296983) bank89643 +89644 POINT(41.05729064356712 73.64216327976895) bank89644 +89645 POINT(41.545065788225884 74.85680612829533) bank89645 +89646 POINT(41.47708527669564 74.59203910183251) bank89646 +89647 POINT(40.81207280671886 74.11983350015147) bank89647 +89648 POINT(40.581187229760964 74.50034956041154) bank89648 +89649 POINT(40.576690606769596 74.839005770024) bank89649 +89650 POINT(40.63172973539641 74.62317761590863) bank89650 +89651 POINT(40.412292292402995 73.11409864637827) bank89651 +89652 POINT(40.00431226676714 74.191995398445) bank89652 +89653 POINT(40.97479984094862 74.03362154907273) bank89653 +89654 POINT(39.82356498633242 73.9353878940421) bank89654 +89655 POINT(40.686918291521195 74.83360850622662) bank89655 +89656 POINT(40.500350616861226 74.674794113161) bank89656 +89657 POINT(39.71790207320331 74.63370121232009) bank89657 +89658 POINT(39.95899799317112 74.84049055138168) bank89658 +89659 POINT(40.09093072109068 73.58037498042327) bank89659 +89660 POINT(40.824519938217854 73.44380287854379) bank89660 +89661 POINT(41.303120117040024 73.59878049373113) bank89661 +89662 POINT(40.392649377822856 73.47698806112969) bank89662 +89663 POINT(41.634861031308624 73.33642113675897) bank89663 +89664 POINT(41.611249155459845 73.14786305786131) bank89664 +89665 POINT(40.98702586344002 74.5032179282064) bank89665 +89666 POINT(41.525359534446714 74.86620124129637) bank89666 +89667 POINT(40.92168758443842 74.4500159001876) bank89667 +89668 POINT(41.39828502834554 74.7585029212877) bank89668 +89669 POINT(41.403714616724095 73.94862728799923) bank89669 +89670 POINT(41.16442377559513 73.85586338274798) bank89670 +89671 POINT(40.535808742613185 74.34184895889777) bank89671 +89672 POINT(40.06377757762683 73.61470450929981) bank89672 +89673 POINT(41.515084140745756 74.36599927726589) bank89673 +89674 POINT(40.39493635464393 73.24758625349669) bank89674 +89675 POINT(40.94905574338228 74.23248609848388) bank89675 +89676 POINT(40.48183079734739 74.36118799719364) bank89676 +89677 POINT(40.33199632744666 74.22905950607604) bank89677 +89678 POINT(40.00410512820997 73.76290189115412) bank89678 +89679 POINT(39.76196865849871 73.88834872444748) bank89679 +89680 POINT(40.85997886469639 74.32522090436476) bank89680 +89681 POINT(39.98278592977362 73.81885078497929) bank89681 +89682 POINT(40.85683153704014 73.97721738109149) bank89682 +89683 POINT(41.13323961595923 73.19575723828866) bank89683 +89684 POINT(40.58964820731403 74.80863225459484) bank89684 +89685 POINT(40.742452262946244 73.77312729011945) bank89685 +89686 POINT(39.7855753183631 73.52911127337806) bank89686 +89687 POINT(40.50458537993228 73.954417882254) bank89687 +89688 POINT(39.772518877934814 73.06355870438588) bank89688 +89689 POINT(40.58452900457774 73.27461319748154) bank89689 +89690 POINT(41.209535416033866 74.76600623829863) bank89690 +89691 POINT(40.51137441953171 74.1974788162404) bank89691 +89692 POINT(41.199884588841165 74.8452566296408) bank89692 +89693 POINT(41.64531092588706 74.78396588804118) bank89693 +89694 POINT(39.729273702991144 74.96402220112145) bank89694 +89695 POINT(41.48248112661281 73.07826919075414) bank89695 +89696 POINT(40.480971688814904 74.54703489024398) bank89696 +89697 POINT(40.90343673682005 73.2566240681132) bank89697 +89698 POINT(40.26725643911841 73.68015696821446) bank89698 +89699 POINT(40.36042666717261 73.60596824985421) bank89699 +89700 POINT(40.19624590880913 73.73231153724034) bank89700 +89701 POINT(40.292432698324184 74.66457337855795) bank89701 +89702 POINT(40.313261844833036 73.15558751334503) bank89702 +89703 POINT(39.833214953519715 74.96500947674488) bank89703 +89704 POINT(41.12468075775471 73.69769953686719) bank89704 +89705 POINT(41.27332521878854 73.77782289856009) bank89705 +89706 POINT(40.016496670944896 74.79856981545339) bank89706 +89707 POINT(40.85136528440449 74.6110372677578) bank89707 +89708 POINT(41.291096857882486 74.00025243711896) bank89708 +89709 POINT(41.51983613874527 73.69927968220007) bank89709 +89710 POINT(41.062923838145096 73.69461405085154) bank89710 +89711 POINT(40.931882428497815 74.52663015211868) bank89711 +89712 POINT(40.6679519152952 74.60487707843023) bank89712 +89713 POINT(40.65607126858156 74.49051029874914) bank89713 +89714 POINT(39.73073767993931 73.64717175682824) bank89714 +89715 POINT(41.23883560547607 74.79029815989938) bank89715 +89716 POINT(40.03172647362194 73.24751894519102) bank89716 +89717 POINT(41.060867413973725 73.75908677293704) bank89717 +89718 POINT(39.750693633054546 73.59847573627366) bank89718 +89719 POINT(40.297642456976355 73.60281110522423) bank89719 +89720 POINT(40.11540901470004 73.46139488004756) bank89720 +89721 POINT(40.17109979163143 74.98740797750521) bank89721 +89722 POINT(40.20875218660487 73.4004258921913) bank89722 +89723 POINT(41.20391734753189 74.52019195581255) bank89723 +89724 POINT(41.41418391239378 74.67680388274245) bank89724 +89725 POINT(41.1611058080075 74.19749448626276) bank89725 +89726 POINT(39.744610561676375 73.20642785308108) bank89726 +89727 POINT(40.69627117296564 73.24592494072238) bank89727 +89728 POINT(41.058718655974104 74.37728574193372) bank89728 +89729 POINT(39.90338080342988 74.5546475664691) bank89729 +89730 POINT(40.004494864069535 73.82854303664485) bank89730 +89731 POINT(40.27248282211734 74.29910581305934) bank89731 +89732 POINT(40.392781686192805 73.98868168587946) bank89732 +89733 POINT(41.589468808676486 73.34325491621404) bank89733 +89734 POINT(41.30033512936824 73.77957560583495) bank89734 +89735 POINT(39.72101605789227 73.63017952970252) bank89735 +89736 POINT(41.18392014721239 73.5617621260832) bank89736 +89737 POINT(40.7359716528966 73.61391132660361) bank89737 +89738 POINT(41.12565729840348 74.21647911307677) bank89738 +89739 POINT(39.92947811005948 73.06406776010641) bank89739 +89740 POINT(40.20742034581862 74.3968977651359) bank89740 +89741 POINT(40.209067631393786 73.84773196613973) bank89741 +89742 POINT(39.8891904121787 74.43835739316819) bank89742 +89743 POINT(41.40913791492657 74.65037964709158) bank89743 +89744 POINT(40.24679204489123 74.17904432577801) bank89744 +89745 POINT(39.73262977038302 73.48472523483142) bank89745 +89746 POINT(39.925140117136415 73.19612885746263) bank89746 +89747 POINT(40.69603402310811 74.01532591150492) bank89747 +89748 POINT(41.34944294481969 74.92923871378659) bank89748 +89749 POINT(40.94205502843813 73.97711676356727) bank89749 +89750 POINT(41.41060890437253 73.58650098911617) bank89750 +89751 POINT(40.26809487012498 73.99720534457339) bank89751 +89752 POINT(41.2565316250248 74.82594145642702) bank89752 +89753 POINT(41.101249347562316 73.70108980207446) bank89753 +89754 POINT(39.97756898962777 73.93242570765653) bank89754 +89755 POINT(41.181764035666816 73.37657457754092) bank89755 +89756 POINT(40.835846097136695 74.54571683442019) bank89756 +89757 POINT(41.483590325591344 74.44953562203773) bank89757 +89758 POINT(40.73331771264337 74.42139843449151) bank89758 +89759 POINT(40.17704736835727 74.95149520180968) bank89759 +89760 POINT(40.130019454047016 74.62492524120398) bank89760 +89761 POINT(39.95540265514206 73.90911985828907) bank89761 +89762 POINT(40.958316232544576 74.32978825178387) bank89762 +89763 POINT(40.04362247963968 74.95258467702618) bank89763 +89764 POINT(40.16294581680348 74.41703072267772) bank89764 +89765 POINT(40.63531039017456 73.12058281687388) bank89765 +89766 POINT(40.75906171420138 73.13606739804243) bank89766 +89767 POINT(41.12667176633941 73.28042355684873) bank89767 +89768 POINT(40.528765860522114 73.17822873340076) bank89768 +89769 POINT(41.59981653126427 74.50635940266626) bank89769 +89770 POINT(41.70305731110588 73.51078309869257) bank89770 +89771 POINT(40.01420031038201 74.71547297708199) bank89771 +89772 POINT(39.7272523752355 73.34102117896076) bank89772 +89773 POINT(41.68855388783007 74.97959937413033) bank89773 +89774 POINT(40.65490337077536 73.23412915247555) bank89774 +89775 POINT(40.258365556993354 74.78053167501714) bank89775 +89776 POINT(40.3449124828335 74.22516366180243) bank89776 +89777 POINT(40.27191255659289 73.58113722218522) bank89777 +89778 POINT(39.753154606311334 74.52977467147032) bank89778 +89779 POINT(41.35607370175637 73.53447041391739) bank89779 +89780 POINT(40.081155315107786 73.13294364594755) bank89780 +89781 POINT(40.31029062332226 73.70220656986704) bank89781 +89782 POINT(40.01039127028377 74.17401350767409) bank89782 +89783 POINT(40.7612499857774 74.89142117396786) bank89783 +89784 POINT(39.89568458026994 73.18185132828862) bank89784 +89785 POINT(40.21809514790891 74.22272270874728) bank89785 +89786 POINT(39.79191729132786 73.08272789117838) bank89786 +89787 POINT(39.95746300873268 73.46044563910523) bank89787 +89788 POINT(41.16289284789214 74.61651556289682) bank89788 +89789 POINT(41.59525176945946 74.1538343546333) bank89789 +89790 POINT(40.12132495273664 74.46986336998633) bank89790 +89791 POINT(40.20048280226835 74.94643614382716) bank89791 +89792 POINT(39.94671246049983 73.81781174669422) bank89792 +89793 POINT(40.898746172210984 74.26939135076897) bank89793 +89794 POINT(39.93777761757858 73.40205087894442) bank89794 +89795 POINT(40.60373993201681 73.12425400561577) bank89795 +89796 POINT(40.610527745082194 74.64785589470154) bank89796 +89797 POINT(40.089196061286756 74.74806015863491) bank89797 +89798 POINT(41.03437311645569 74.86474653449957) bank89798 +89799 POINT(39.71817189320526 74.88323385416211) bank89799 +89800 POINT(41.42170549206438 74.2458567492111) bank89800 +89801 POINT(41.623930069980524 74.18527771322513) bank89801 +89802 POINT(39.967070895032656 73.81900119688655) bank89802 +89803 POINT(41.611011513472164 73.50348956980658) bank89803 +89804 POINT(40.12497278327993 74.28451269431032) bank89804 +89805 POINT(40.51718890128097 74.8922640377366) bank89805 +89806 POINT(41.480572236923194 74.4409653564581) bank89806 +89807 POINT(41.20247456285916 74.46705298779149) bank89807 +89808 POINT(40.157273498537435 75.00216662979841) bank89808 +89809 POINT(40.53017829422005 73.29910103501207) bank89809 +89810 POINT(41.24591934181629 73.67732607210411) bank89810 +89811 POINT(40.40863329532927 74.59102157124146) bank89811 +89812 POINT(40.444713315691516 73.08346080797936) bank89812 +89813 POINT(40.74118276929186 74.79474380169226) bank89813 +89814 POINT(39.914636485262974 73.02458290987524) bank89814 +89815 POINT(39.98200768555024 73.85985071687908) bank89815 +89816 POINT(40.327382448227574 73.43538939452813) bank89816 +89817 POINT(41.145621146352674 74.44166705811895) bank89817 +89818 POINT(41.20856309230915 74.72797558273288) bank89818 +89819 POINT(40.96579323150108 74.18430931969345) bank89819 +89820 POINT(40.84912988307421 73.45086134879702) bank89820 +89821 POINT(40.00126420315843 73.33701725178028) bank89821 +89822 POINT(39.94526398734679 73.75640664936671) bank89822 +89823 POINT(41.19615328349646 73.34047201811549) bank89823 +89824 POINT(40.371073349649905 73.67832958185241) bank89824 +89825 POINT(41.55977497693912 73.23728340079899) bank89825 +89826 POINT(40.770561714406426 74.47795200316818) bank89826 +89827 POINT(41.239545508807474 74.16911820988334) bank89827 +89828 POINT(40.59271989452617 73.12942097040616) bank89828 +89829 POINT(40.07553444438922 74.58934756315912) bank89829 +89830 POINT(41.63335341551109 74.35858884181883) bank89830 +89831 POINT(40.40183566399789 73.93246391028384) bank89831 +89832 POINT(40.28578816307461 73.95687943074958) bank89832 +89833 POINT(40.200642100566576 74.86342327743132) bank89833 +89834 POINT(40.04986432895192 75.00419802485618) bank89834 +89835 POINT(39.84787375410445 74.799279587908) bank89835 +89836 POINT(41.14540739579993 73.65199071353658) bank89836 +89837 POINT(41.382284009094256 73.69370163106788) bank89837 +89838 POINT(40.3967101912908 74.37907407031858) bank89838 +89839 POINT(40.702071112603804 74.19323293003457) bank89839 +89840 POINT(41.383041003525946 73.43743239210289) bank89840 +89841 POINT(40.55733860328295 73.67643991927207) bank89841 +89842 POINT(40.44762902753957 73.27141956241063) bank89842 +89843 POINT(41.448095470379315 74.949418278971) bank89843 +89844 POINT(41.08453889182674 74.72010846343208) bank89844 +89845 POINT(41.151209777400766 74.88919510174534) bank89845 +89846 POINT(40.03622376046244 73.1224853204385) bank89846 +89847 POINT(41.08308005672231 73.7258679993974) bank89847 +89848 POINT(41.272840380700146 74.51495482573813) bank89848 +89849 POINT(40.4093418611163 73.86807219196119) bank89849 +89850 POINT(41.68772640031788 73.86464413557783) bank89850 +89851 POINT(39.967820661861026 74.84936417956166) bank89851 +89852 POINT(40.081451935982514 73.06785050198224) bank89852 +89853 POINT(40.11315697412807 74.10675758166539) bank89853 +89854 POINT(40.10775056630287 73.15704605455034) bank89854 +89855 POINT(40.01927174047048 75.00562748413817) bank89855 +89856 POINT(40.707778612695044 73.77616536318516) bank89856 +89857 POINT(40.413754641907765 73.93372156908359) bank89857 +89858 POINT(39.89752577889 74.98900448226401) bank89858 +89859 POINT(41.48605452321425 73.9719974690193) bank89859 +89860 POINT(40.23480889442259 74.37394310050023) bank89860 +89861 POINT(41.676417315213754 73.70217680295949) bank89861 +89862 POINT(39.71395630018415 73.45033210182282) bank89862 +89863 POINT(39.9967807386412 73.86179109184941) bank89863 +89864 POINT(39.97940498780783 73.60025524710919) bank89864 +89865 POINT(40.37535675397639 73.44944927511644) bank89865 +89866 POINT(40.929232092562295 74.55033880101067) bank89866 +89867 POINT(40.71022565146086 73.26138620140077) bank89867 +89868 POINT(40.41360450341497 73.87809310384742) bank89868 +89869 POINT(40.76739481946028 74.82092067389516) bank89869 +89870 POINT(41.59327634948983 74.34103823680951) bank89870 +89871 POINT(40.58891097598004 74.31424839339476) bank89871 +89872 POINT(40.80233256195674 74.39016308823685) bank89872 +89873 POINT(40.51666490119622 73.5218064801701) bank89873 +89874 POINT(40.018253643298934 73.13473168143864) bank89874 +89875 POINT(40.35415635005413 73.76686591252376) bank89875 +89876 POINT(40.13658781673522 74.71869569087255) bank89876 +89877 POINT(41.35939997177636 74.16286115939737) bank89877 +89878 POINT(40.61022958987774 74.56104568552861) bank89878 +89879 POINT(40.97833214024348 73.7299248351509) bank89879 +89880 POINT(40.61433368241752 73.55455776818685) bank89880 +89881 POINT(39.85152371184637 73.7108152549341) bank89881 +89882 POINT(41.61160654639964 73.29117164454455) bank89882 +89883 POINT(40.58537495747933 73.05068480030607) bank89883 +89884 POINT(40.76841154406691 73.11617370214493) bank89884 +89885 POINT(40.182588409986806 73.88987427747945) bank89885 +89886 POINT(41.5438378196797 74.17429171555453) bank89886 +89887 POINT(41.518637307190204 73.30181842179017) bank89887 +89888 POINT(40.93556853929093 74.23236039312106) bank89888 +89889 POINT(41.49865516529285 74.89900156976286) bank89889 +89890 POINT(40.70020603285562 73.32063678508382) bank89890 +89891 POINT(40.65398726807059 73.68393393876256) bank89891 +89892 POINT(40.6947807167717 74.7394946296031) bank89892 +89893 POINT(40.6569181900424 73.63478502250268) bank89893 +89894 POINT(40.94360954881194 74.87896726094843) bank89894 +89895 POINT(40.76642810720373 73.86863064268047) bank89895 +89896 POINT(40.9819652618495 74.37013609735521) bank89896 +89897 POINT(41.23544059732815 73.27663445671313) bank89897 +89898 POINT(41.33196427139613 73.17000670570206) bank89898 +89899 POINT(39.98463262709308 73.24869818732337) bank89899 +89900 POINT(41.0727782323224 74.90166173269755) bank89900 +89901 POINT(40.805450720552926 73.31846519016672) bank89901 +89902 POINT(39.90871240084144 74.62018157128128) bank89902 +89903 POINT(41.63263234693786 74.25022148364249) bank89903 +89904 POINT(40.90174240427858 74.3809229326886) bank89904 +89905 POINT(39.805703741333765 73.09013667071456) bank89905 +89906 POINT(40.32319638106763 73.49728848506368) bank89906 +89907 POINT(40.43993450751821 74.79317623029728) bank89907 +89908 POINT(41.19584243787405 73.75399902232418) bank89908 +89909 POINT(39.84950366537014 73.14475268115713) bank89909 +89910 POINT(40.40855805195509 74.22629328224497) bank89910 +89911 POINT(40.90353955990084 73.48976774593645) bank89911 +89912 POINT(41.29228341581532 73.28269870098288) bank89912 +89913 POINT(40.919393621320324 74.5170518490154) bank89913 +89914 POINT(39.80599504443104 74.47227232808774) bank89914 +89915 POINT(40.76675905749375 74.74903778008172) bank89915 +89916 POINT(41.30928538889578 74.30884595640283) bank89916 +89917 POINT(40.892297642584374 73.89956176939751) bank89917 +89918 POINT(39.78153034416359 74.27496382051592) bank89918 +89919 POINT(40.081130820757004 73.69538590505816) bank89919 +89920 POINT(40.128307686795814 74.48024295529372) bank89920 +89921 POINT(39.7983075266749 74.85341050877219) bank89921 +89922 POINT(41.23766731071348 73.95702044088402) bank89922 +89923 POINT(40.474695986866344 73.99470557807487) bank89923 +89924 POINT(40.4194055430176 73.73564144071614) bank89924 +89925 POINT(41.46445309669493 74.4291830975205) bank89925 +89926 POINT(40.72472098769102 74.31083968720445) bank89926 +89927 POINT(40.294716237561076 73.67956557788706) bank89927 +89928 POINT(39.81949821500778 74.02310464939248) bank89928 +89929 POINT(40.601155742037434 73.50344494174324) bank89929 +89930 POINT(41.638881278172846 73.28496673552492) bank89930 +89931 POINT(40.91731833989404 74.27436890105942) bank89931 +89932 POINT(41.36692323519521 74.597071403447) bank89932 +89933 POINT(40.594070026790966 73.60804716453916) bank89933 +89934 POINT(41.65108163170373 73.61109993690688) bank89934 +89935 POINT(40.3766011147198 74.83886789684267) bank89935 +89936 POINT(41.00366903967969 74.23789198674085) bank89936 +89937 POINT(39.80532085962827 74.85448032886644) bank89937 +89938 POINT(41.42616344253476 73.40639212701346) bank89938 +89939 POINT(39.86240750845385 74.4700447166637) bank89939 +89940 POINT(40.53043233498824 74.11638117471837) bank89940 +89941 POINT(41.33467805641713 74.83009964230422) bank89941 +89942 POINT(41.561600068955094 73.16321000823159) bank89942 +89943 POINT(40.38086124510069 74.4855130045436) bank89943 +89944 POINT(40.18043523860277 73.40845933749503) bank89944 +89945 POINT(40.996902740123595 74.16288082921469) bank89945 +89946 POINT(39.78907557976557 73.27254581495691) bank89946 +89947 POINT(40.07536850151132 74.01010311859962) bank89947 +89948 POINT(41.34818219432698 74.59064252053868) bank89948 +89949 POINT(40.02437983121655 73.97137369343488) bank89949 +89950 POINT(40.01749797907078 74.90689918453428) bank89950 +89951 POINT(40.756579704339515 74.05816686510538) bank89951 +89952 POINT(41.56984029163872 73.89716098251846) bank89952 +89953 POINT(41.283261124592364 74.98106157517245) bank89953 +89954 POINT(40.49004698960609 73.22829608302264) bank89954 +89955 POINT(40.29118280639846 73.17532642432045) bank89955 +89956 POINT(41.114666258940304 74.77831771505069) bank89956 +89957 POINT(41.45554134416309 74.64438742874643) bank89957 +89958 POINT(40.67669328354506 73.50896364913807) bank89958 +89959 POINT(41.302545472658714 74.87485301608665) bank89959 +89960 POINT(40.637652597833245 74.0714268623308) bank89960 +89961 POINT(41.668395311358246 73.07054706684795) bank89961 +89962 POINT(40.04731823916147 73.4888411138595) bank89962 +89963 POINT(40.896547835812406 73.63320740831868) bank89963 +89964 POINT(41.26730719411027 73.96351212683616) bank89964 +89965 POINT(41.14065176268497 74.55151673188445) bank89965 +89966 POINT(40.33428760514755 74.23238455150488) bank89966 +89967 POINT(41.418618314075566 73.17961216172881) bank89967 +89968 POINT(40.23609796371447 74.31067295826479) bank89968 +89969 POINT(41.254942942257756 74.91756912210819) bank89969 +89970 POINT(41.4264642865026 74.15103412271941) bank89970 +89971 POINT(41.658308788736065 73.48630711745164) bank89971 +89972 POINT(40.79816340263322 74.20525483171201) bank89972 +89973 POINT(41.39893860411009 74.65330155641085) bank89973 +89974 POINT(40.41519869023079 73.92870626320975) bank89974 +89975 POINT(40.230922967953894 73.93654178877826) bank89975 +89976 POINT(40.9389603503737 74.22765670478881) bank89976 +89977 POINT(40.66770046943569 74.48578342784728) bank89977 +89978 POINT(39.98525524788897 74.9491226805119) bank89978 +89979 POINT(40.11734341689219 74.21380038922294) bank89979 +89980 POINT(40.27942071230642 73.15421204193508) bank89980 +89981 POINT(40.47959062153028 73.67973935533445) bank89981 +89982 POINT(40.30217681063006 74.37114159243761) bank89982 +89983 POINT(39.9194563665614 73.90973188088077) bank89983 +89984 POINT(41.61088903126438 73.65520688958624) bank89984 +89985 POINT(41.66982249987769 73.15587752556544) bank89985 +89986 POINT(40.32102724263752 73.55683377225812) bank89986 +89987 POINT(39.750976947674765 73.42797392590792) bank89987 +89988 POINT(40.44302328961658 74.62812229254425) bank89988 +89989 POINT(41.595840902188215 74.63882834796084) bank89989 +89990 POINT(41.09620088676129 73.3699907794028) bank89990 +89991 POINT(40.43153803093177 74.44229150265238) bank89991 +89992 POINT(40.01636642452248 74.263997348709) bank89992 +89993 POINT(39.8486286457459 74.73603990395665) bank89993 +89994 POINT(39.84015171945837 73.90909227987342) bank89994 +89995 POINT(41.08458581769618 73.7227994632799) bank89995 +89996 POINT(41.10696518001631 74.95998893997569) bank89996 +89997 POINT(40.27897536448572 74.02831415875838) bank89997 +89998 POINT(41.154017505968156 74.62703795401738) bank89998 +89999 POINT(40.32445345952497 74.1525407980614) bank89999 +90000 POINT(39.8613352152226 74.42486956621713) bank90000 +90001 POINT(40.79766904798537 73.39315601809186) bank90001 +90002 POINT(40.266120498751405 74.38785229410162) bank90002 +90003 POINT(41.05841610727628 73.16365837204548) bank90003 +90004 POINT(40.579886752171404 74.09527456727038) bank90004 +90005 POINT(40.763489317819946 74.57587794623178) bank90005 +90006 POINT(40.862701994808106 73.05965417903063) bank90006 +90007 POINT(40.19063899884331 73.64988563281409) bank90007 +90008 POINT(41.1240933362278 73.379460616947) bank90008 +90009 POINT(41.30523947939383 73.32265020964411) bank90009 +90010 POINT(41.06415290628099 73.70773141370472) bank90010 +90011 POINT(41.360638790757775 74.27857252199095) bank90011 +90012 POINT(40.55304286698798 74.89488068099855) bank90012 +90013 POINT(40.34392550592335 73.660450086322) bank90013 +90014 POINT(40.962379612696814 73.84080059905385) bank90014 +90015 POINT(41.33168399774945 74.48872679554476) bank90015 +90016 POINT(41.15799502040855 73.33436078652622) bank90016 +90017 POINT(41.61931533426878 73.9471341127019) bank90017 +90018 POINT(40.39924680207478 73.3765578081113) bank90018 +90019 POINT(39.782487994563084 74.48172893248407) bank90019 +90020 POINT(40.48242750630551 73.52997507495202) bank90020 +90021 POINT(40.847692725713024 73.13365815379711) bank90021 +90022 POINT(39.81238699694007 73.4936064937349) bank90022 +90023 POINT(39.91367174652825 73.09170236243884) bank90023 +90024 POINT(40.67359994070976 73.45603000806325) bank90024 +90025 POINT(40.405682111399955 73.79874993323595) bank90025 +90026 POINT(40.93030178380335 73.18381390946195) bank90026 +90027 POINT(40.14220172622037 74.79439432247511) bank90027 +90028 POINT(40.47646897710406 73.05933261315656) bank90028 +90029 POINT(40.55972199937787 74.75804592396739) bank90029 +90030 POINT(39.880031767267035 74.9040872166986) bank90030 +90031 POINT(41.492336845617324 73.06304083809627) bank90031 +90032 POINT(40.44153867807747 74.9501116879126) bank90032 +90033 POINT(40.90857359953555 73.11182281274333) bank90033 +90034 POINT(40.481184981337954 73.75316343939262) bank90034 +90035 POINT(40.82409433255206 73.133002467077) bank90035 +90036 POINT(41.180148814310485 74.01551927208911) bank90036 +90037 POINT(39.73488013186235 73.64887749596298) bank90037 +90038 POINT(40.89022774964716 74.65002710757854) bank90038 +90039 POINT(40.62788003907594 73.6842259333882) bank90039 +90040 POINT(41.52634330762275 73.91153363318604) bank90040 +90041 POINT(40.32039588936922 74.984063688586) bank90041 +90042 POINT(40.92364357649777 74.26744379857905) bank90042 +90043 POINT(41.12982267292262 73.07231669129757) bank90043 +90044 POINT(41.351905682151 73.03154004655119) bank90044 +90045 POINT(40.80223618234928 73.86378609670876) bank90045 +90046 POINT(41.5175732221929 73.74967520826156) bank90046 +90047 POINT(40.600250614825164 73.50432311495499) bank90047 +90048 POINT(40.387022189864325 73.71542587103258) bank90048 +90049 POINT(40.54861720258655 73.14878955499299) bank90049 +90050 POINT(40.344159771228775 73.37003513787573) bank90050 +90051 POINT(41.37199180030995 74.83569897325708) bank90051 +90052 POINT(40.77091481425859 74.93933605092002) bank90052 +90053 POINT(40.6105672866423 73.44109279289546) bank90053 +90054 POINT(39.73172379982947 74.87294706015831) bank90054 +90055 POINT(40.502049603494115 73.09609371338652) bank90055 +90056 POINT(39.73877721936717 74.0295161946668) bank90056 +90057 POINT(40.24519166966612 74.24398318115084) bank90057 +90058 POINT(40.916654595968375 73.43120731301762) bank90058 +90059 POINT(40.563849976616766 73.47813983151964) bank90059 +90060 POINT(40.333429712129465 73.26937208854447) bank90060 +90061 POINT(41.60006115749039 74.82339406000968) bank90061 +90062 POINT(40.77335148903259 74.12665419310827) bank90062 +90063 POINT(41.65634575696791 74.84159447065474) bank90063 +90064 POINT(39.85949656471022 74.04612967347147) bank90064 +90065 POINT(41.24047999040643 74.03395577593523) bank90065 +90066 POINT(41.459754682580225 73.94521445517357) bank90066 +90067 POINT(41.66744885621129 74.12354645710988) bank90067 +90068 POINT(39.820121243033746 74.89994423366255) bank90068 +90069 POINT(41.60886441892433 74.26864374139178) bank90069 +90070 POINT(41.434088277880164 73.29585183824058) bank90070 +90071 POINT(40.28768050388906 73.83584942022489) bank90071 +90072 POINT(40.52720418521985 74.68184730463818) bank90072 +90073 POINT(41.6156555670547 74.64584452673117) bank90073 +90074 POINT(40.08294440071119 74.07155865750289) bank90074 +90075 POINT(40.496170862574296 74.22192087609585) bank90075 +90076 POINT(40.20948182933265 74.75958992987651) bank90076 +90077 POINT(40.29965249423339 73.87420383828915) bank90077 +90078 POINT(41.59517631937413 74.04895898557338) bank90078 +90079 POINT(40.66170605064853 73.85829788077571) bank90079 +90080 POINT(40.74911056082228 74.2662976573216) bank90080 +90081 POINT(41.66928802335809 73.8987174897306) bank90081 +90082 POINT(40.6422753939484 73.07549977680219) bank90082 +90083 POINT(41.15570291362386 73.54822422858726) bank90083 +90084 POINT(41.06626358478001 74.40762871257434) bank90084 +90085 POINT(40.72823330994991 73.08567034899404) bank90085 +90086 POINT(40.5656019293939 73.38661551827632) bank90086 +90087 POINT(40.00130070721544 74.61363747783278) bank90087 +90088 POINT(41.370976899778654 73.81418201916372) bank90088 +90089 POINT(40.96011538271808 74.5110825505184) bank90089 +90090 POINT(40.95093317665257 74.41731991447678) bank90090 +90091 POINT(40.07420820199752 73.42248477319993) bank90091 +90092 POINT(40.58807133386894 73.23714589378989) bank90092 +90093 POINT(40.33473443642288 73.85456474734583) bank90093 +90094 POINT(41.65851665180821 74.64589001155797) bank90094 +90095 POINT(40.88385738404001 73.24715920944217) bank90095 +90096 POINT(40.70268447673269 73.4805349040556) bank90096 +90097 POINT(40.689859156599965 74.37939646132249) bank90097 +90098 POINT(40.62661527950133 73.09345737214133) bank90098 +90099 POINT(40.827319717702274 74.74273763608045) bank90099 +90100 POINT(40.33672204522905 74.00851229797364) bank90100 +90101 POINT(40.44017689079455 73.29490349629872) bank90101 +90102 POINT(40.34279641426288 73.16224279875114) bank90102 +90103 POINT(40.990833649163314 73.08944366070692) bank90103 +90104 POINT(41.20604814168268 74.20488130364262) bank90104 +90105 POINT(41.59468315952437 73.19676664879341) bank90105 +90106 POINT(41.21456814773171 73.23465345457733) bank90106 +90107 POINT(41.52694974646438 74.33232669077049) bank90107 +90108 POINT(39.97350480074311 74.80653819076873) bank90108 +90109 POINT(40.16205348014932 74.09972243901012) bank90109 +90110 POINT(41.30681106221577 74.93207880050213) bank90110 +90111 POINT(40.49421094300432 73.92019006098162) bank90111 +90112 POINT(40.8956926769484 73.40449263682076) bank90112 +90113 POINT(41.70783272934899 73.93531540713376) bank90113 +90114 POINT(41.277986194445376 74.13456514974789) bank90114 +90115 POINT(39.82350908803383 74.00383289513788) bank90115 +90116 POINT(41.68493784149605 73.70369353293094) bank90116 +90117 POINT(41.45907006877753 74.43823646944453) bank90117 +90118 POINT(40.76750067036342 73.32499816038569) bank90118 +90119 POINT(41.107002750182964 73.42968719134697) bank90119 +90120 POINT(40.91665318760842 73.27080510673794) bank90120 +90121 POINT(40.65423902455808 74.80302867951498) bank90121 +90122 POINT(40.23981797839268 73.43390742127423) bank90122 +90123 POINT(40.285272079536284 73.02664047635736) bank90123 +90124 POINT(40.98353365912488 74.60072291643094) bank90124 +90125 POINT(40.05879749312626 73.81988709263727) bank90125 +90126 POINT(41.11752904790302 73.45333227667919) bank90126 +90127 POINT(40.866779665992944 74.79430188833328) bank90127 +90128 POINT(40.59030032157397 73.42666565492496) bank90128 +90129 POINT(40.68130350885554 73.00874110019346) bank90129 +90130 POINT(41.33200041538608 74.2241074891337) bank90130 +90131 POINT(40.44483146306045 73.71902631382326) bank90131 +90132 POINT(39.83634174652205 73.30920162570824) bank90132 +90133 POINT(40.766972312054925 74.95387898276621) bank90133 +90134 POINT(40.84804433408393 73.61539728681086) bank90134 +90135 POINT(41.107661066385184 73.3396426691662) bank90135 +90136 POINT(40.10886840857652 75.00320546324731) bank90136 +90137 POINT(40.521469724686064 74.49514853831388) bank90137 +90138 POINT(41.699289839224505 74.34356189899975) bank90138 +90139 POINT(40.97438767177707 74.9435388297295) bank90139 +90140 POINT(40.29545054622028 73.97647304346877) bank90140 +90141 POINT(41.59968148835416 73.7147196858771) bank90141 +90142 POINT(41.54574824123912 74.81553380578379) bank90142 +90143 POINT(40.71195749700138 73.10363934896017) bank90143 +90144 POINT(40.608601695913244 73.92370408229947) bank90144 +90145 POINT(41.64326981950373 73.20489581794358) bank90145 +90146 POINT(40.30574033536051 73.44745864681526) bank90146 +90147 POINT(40.75814605364883 74.38061751297545) bank90147 +90148 POINT(40.59243795598749 73.55122557775591) bank90148 +90149 POINT(40.946993171777514 74.3709398586753) bank90149 +90150 POINT(41.67425951118007 73.8321187447948) bank90150 +90151 POINT(39.853495400104336 74.70932190992401) bank90151 +90152 POINT(41.59616887183773 74.06737268919701) bank90152 +90153 POINT(41.42981987351119 74.9593106835855) bank90153 +90154 POINT(40.03513759933737 74.11749033727075) bank90154 +90155 POINT(41.379517923482666 73.23490500150393) bank90155 +90156 POINT(41.24657129221145 73.09684284318178) bank90156 +90157 POINT(41.08618296676734 73.15915172162953) bank90157 +90158 POINT(40.31614139321522 73.62623497530973) bank90158 +90159 POINT(41.22133439025686 73.14205232004456) bank90159 +90160 POINT(41.01398811578298 74.3784368411576) bank90160 +90161 POINT(39.831475725113464 73.41965493583638) bank90161 +90162 POINT(40.40586000395342 74.44476144508484) bank90162 +90163 POINT(40.94436742152997 73.75921529225398) bank90163 +90164 POINT(40.985313583554095 73.24662887459527) bank90164 +90165 POINT(41.60684060499606 73.40186122132287) bank90165 +90166 POINT(40.59455800311677 73.67464866010661) bank90166 +90167 POINT(40.086655484848755 74.43919598144333) bank90167 +90168 POINT(39.85728590543114 74.6587524749339) bank90168 +90169 POINT(41.50881791210507 73.58425342071968) bank90169 +90170 POINT(41.2299231900564 74.97121990050975) bank90170 +90171 POINT(41.6058692798931 74.25005498696261) bank90171 +90172 POINT(40.55684883449805 73.53564180898464) bank90172 +90173 POINT(40.31443181407551 74.20572356924485) bank90173 +90174 POINT(40.933210936691886 74.42983808991121) bank90174 +90175 POINT(40.710618867905985 73.67565814745372) bank90175 +90176 POINT(41.20843312865526 73.47710820843687) bank90176 +90177 POINT(41.059936035331816 73.85540856725801) bank90177 +90178 POINT(40.06716207520346 73.43542839104002) bank90178 +90179 POINT(40.04758430187829 73.16646649334425) bank90179 +90180 POINT(39.83113213642618 73.01998441323371) bank90180 +90181 POINT(40.242299834264685 74.92171177162649) bank90181 +90182 POINT(41.65445536968175 73.93885661036904) bank90182 +90183 POINT(40.01008968565467 73.34063272095105) bank90183 +90184 POINT(41.13390862137891 74.18358378734226) bank90184 +90185 POINT(41.52285111963489 73.69120144479812) bank90185 +90186 POINT(40.7450627203436 74.65224732282518) bank90186 +90187 POINT(40.44545202225151 73.37679942618163) bank90187 +90188 POINT(41.03335950398586 73.86101874252306) bank90188 +90189 POINT(41.709949209520595 73.67665140789688) bank90189 +90190 POINT(41.63068549780782 73.69659989455234) bank90190 +90191 POINT(39.83047526073818 73.22336938766976) bank90191 +90192 POINT(41.29981942308439 74.48820412813747) bank90192 +90193 POINT(40.76594454336345 74.59985059099697) bank90193 +90194 POINT(40.88879634625112 73.83950091709717) bank90194 +90195 POINT(41.49932871318375 74.9884732717106) bank90195 +90196 POINT(40.03829632184005 74.86878628689648) bank90196 +90197 POINT(41.20630808471234 73.05353552638455) bank90197 +90198 POINT(40.364605643647714 74.47277624538567) bank90198 +90199 POINT(40.92091696876438 73.73979192596572) bank90199 +90200 POINT(40.8444825850991 74.4816871329604) bank90200 +90201 POINT(40.46490847280423 73.71758746299645) bank90201 +90202 POINT(40.526096515353174 74.33462030393852) bank90202 +90203 POINT(40.95782445967485 74.52908904556216) bank90203 +90204 POINT(40.486537077985034 74.66525172157263) bank90204 +90205 POINT(41.70084231931671 75.00092233675625) bank90205 +90206 POINT(41.569667872130275 73.78334499437817) bank90206 +90207 POINT(39.79027321443319 74.4233033559228) bank90207 +90208 POINT(39.93255939420575 74.13074199679818) bank90208 +90209 POINT(40.630304433821934 74.5828920694867) bank90209 +90210 POINT(41.329656049615494 74.41418588738068) bank90210 +90211 POINT(40.9694315157609 73.134940347959) bank90211 +90212 POINT(40.2497372620224 74.55868669533002) bank90212 +90213 POINT(39.98744932698709 73.71569680059535) bank90213 +90214 POINT(40.56444999870203 73.67669725559401) bank90214 +90215 POINT(40.2729780843248 73.62185234761648) bank90215 +90216 POINT(41.158858524640344 73.4956435110967) bank90216 +90217 POINT(40.33607292240961 73.39818802660106) bank90217 +90218 POINT(39.825584016565415 74.71001712022607) bank90218 +90219 POINT(40.41653820689751 73.81110178654409) bank90219 +90220 POINT(40.354191371906886 74.05809216827073) bank90220 +90221 POINT(40.6573139547747 73.96806510220134) bank90221 +90222 POINT(41.42117124050555 73.80970893056218) bank90222 +90223 POINT(40.043042196912964 74.57280834968762) bank90223 +90224 POINT(40.269977944776144 74.47920680860408) bank90224 +90225 POINT(41.62235463020466 74.44728353033011) bank90225 +90226 POINT(41.312487006946846 74.41741475692035) bank90226 +90227 POINT(41.268181999235736 74.17693452018355) bank90227 +90228 POINT(41.616506509652325 73.68839847534433) bank90228 +90229 POINT(40.108113814472844 74.49330787053857) bank90229 +90230 POINT(40.84660894017172 74.24806599388415) bank90230 +90231 POINT(39.806910687168646 73.62911781807925) bank90231 +90232 POINT(41.57327318381413 74.02235116611403) bank90232 +90233 POINT(41.553566458440656 74.57629722798043) bank90233 +90234 POINT(40.80209465207058 74.90281454702458) bank90234 +90235 POINT(40.0242418021178 73.80161823671753) bank90235 +90236 POINT(39.88477630309445 74.69616800694182) bank90236 +90237 POINT(40.64026672585194 74.4831449720673) bank90237 +90238 POINT(40.41726478530779 73.8609418851448) bank90238 +90239 POINT(41.50046950198572 73.40882683330854) bank90239 +90240 POINT(41.38711651504177 74.42814045544291) bank90240 +90241 POINT(41.12099592600248 74.23143200674872) bank90241 +90242 POINT(40.84990529751341 74.46978082729352) bank90242 +90243 POINT(39.76071965449903 73.11847512461532) bank90243 +90244 POINT(39.836963809539206 73.3828451946428) bank90244 +90245 POINT(40.30478809499474 74.6204082048584) bank90245 +90246 POINT(40.00483005732784 73.64879120331776) bank90246 +90247 POINT(41.489102205510974 74.63874260358213) bank90247 +90248 POINT(41.082221925591796 74.47824325954247) bank90248 +90249 POINT(40.20112756051638 74.74462872610698) bank90249 +90250 POINT(40.178185953083286 74.27991349573138) bank90250 +90251 POINT(39.83220558869713 73.72440366438029) bank90251 +90252 POINT(40.96320618194597 74.56633654343233) bank90252 +90253 POINT(39.97650912283161 74.51177954439541) bank90253 +90254 POINT(40.75494789309008 74.73894362126016) bank90254 +90255 POINT(41.128718461694724 73.72405280355689) bank90255 +90256 POINT(40.25328924483644 74.24215613678959) bank90256 +90257 POINT(40.38775787836028 73.67628658319215) bank90257 +90258 POINT(39.86626568482438 73.64690598748956) bank90258 +90259 POINT(40.44223772247099 73.87833418178228) bank90259 +90260 POINT(41.70831025527132 74.6924187378403) bank90260 +90261 POINT(40.42192479088158 74.52486146894613) bank90261 +90262 POINT(39.8262195149388 74.9577956851768) bank90262 +90263 POINT(41.10937081943018 74.85900468822037) bank90263 +90264 POINT(41.55086631281206 74.97630162718052) bank90264 +90265 POINT(40.31332385306383 74.05234387760063) bank90265 +90266 POINT(40.53616222968751 73.9098143447956) bank90266 +90267 POINT(40.06386479270054 74.08895209209388) bank90267 +90268 POINT(40.139723600461544 74.26640193648372) bank90268 +90269 POINT(41.36780910409287 74.53077885466058) bank90269 +90270 POINT(41.17800447817696 74.19251306190424) bank90270 +90271 POINT(40.55826679875804 73.65150719541367) bank90271 +90272 POINT(40.55879114519496 73.40066835447944) bank90272 +90273 POINT(40.45471726651338 73.76647320543968) bank90273 +90274 POINT(41.074646376666685 73.59467220379153) bank90274 +90275 POINT(39.90242852032738 74.71571233162659) bank90275 +90276 POINT(40.28102437224036 73.16038960543095) bank90276 +90277 POINT(40.46397583312738 74.34544073860424) bank90277 +90278 POINT(41.015389688303244 73.35753340119136) bank90278 +90279 POINT(40.74157976736795 74.02935559695307) bank90279 +90280 POINT(39.801442210638704 73.76880339441163) bank90280 +90281 POINT(41.35282679383727 73.96642661819662) bank90281 +90282 POINT(40.02676674974565 74.19132212535638) bank90282 +90283 POINT(40.33986215592226 74.62148968313292) bank90283 +90284 POINT(41.467433612438484 74.72582157102809) bank90284 +90285 POINT(40.93326073036266 74.85702642847421) bank90285 +90286 POINT(40.126567969778165 73.91410190544772) bank90286 +90287 POINT(39.82740296965057 73.64695086943968) bank90287 +90288 POINT(40.952961502525994 73.07262763352487) bank90288 +90289 POINT(40.69686112273676 74.59005457885227) bank90289 +90290 POINT(39.83543080402287 73.11820738348909) bank90290 +90291 POINT(41.62220965851355 73.96553260850334) bank90291 +90292 POINT(40.160329498116425 74.03516209911781) bank90292 +90293 POINT(40.59488441141736 74.76450001148366) bank90293 +90294 POINT(40.02842043017185 73.95376344893009) bank90294 +90295 POINT(40.12047291943658 73.14611177618208) bank90295 +90296 POINT(40.52903244211144 73.81538709297831) bank90296 +90297 POINT(40.72570321616749 74.41564218103045) bank90297 +90298 POINT(40.28216689886206 73.42119998476292) bank90298 +90299 POINT(40.69872971517796 74.33885743458704) bank90299 +90300 POINT(40.68672796942884 73.31563996987566) bank90300 +90301 POINT(39.91154368023392 74.50265907064833) bank90301 +90302 POINT(40.62874041605837 74.15930273268779) bank90302 +90303 POINT(40.19903540369076 74.8101405528653) bank90303 +90304 POINT(41.261091394915724 73.44960502777765) bank90304 +90305 POINT(40.12006334867062 74.01060594194196) bank90305 +90306 POINT(41.25845918486893 73.81830645471778) bank90306 +90307 POINT(40.81355647087482 73.40426147026923) bank90307 +90308 POINT(40.3764494669199 73.08281644247786) bank90308 +90309 POINT(41.48713546160675 73.37665693317132) bank90309 +90310 POINT(40.03022549557103 74.56166356641582) bank90310 +90311 POINT(40.287998900816966 73.94797352708943) bank90311 +90312 POINT(40.89833114437372 74.08811430715903) bank90312 +90313 POINT(40.99520469573752 73.88872472472237) bank90313 +90314 POINT(41.0642248756782 74.22771599101495) bank90314 +90315 POINT(40.831588186785545 73.58250703588902) bank90315 +90316 POINT(39.91356215848876 73.34346280022913) bank90316 +90317 POINT(40.61576545719593 73.37380421562285) bank90317 +90318 POINT(41.12889501420154 73.13453680058264) bank90318 +90319 POINT(40.72594500642218 74.89860315051178) bank90319 +90320 POINT(41.042307496822964 73.24450633586589) bank90320 +90321 POINT(40.96528784402086 73.74688475139584) bank90321 +90322 POINT(39.724982865434505 74.65301771863588) bank90322 +90323 POINT(39.71363270266497 73.06890535120348) bank90323 +90324 POINT(40.437308118338116 73.0279959270873) bank90324 +90325 POINT(40.9618223825734 73.80733872690507) bank90325 +90326 POINT(41.56335624430481 74.09356022093371) bank90326 +90327 POINT(39.850296733341594 74.38333310405153) bank90327 +90328 POINT(40.17164770334967 73.30704170935344) bank90328 +90329 POINT(40.623823135185646 73.76888815592324) bank90329 +90330 POINT(39.756118089789595 73.34904138234265) bank90330 +90331 POINT(39.98455960952654 73.26678776991666) bank90331 +90332 POINT(41.14860888358087 73.82008946621748) bank90332 +90333 POINT(41.61231008681267 74.32562262577761) bank90333 +90334 POINT(41.329341305543785 74.32174088621446) bank90334 +90335 POINT(40.39887946680632 74.602243943704) bank90335 +90336 POINT(40.95256684095164 74.14458166353025) bank90336 +90337 POINT(40.42331703226097 74.23490264808936) bank90337 +90338 POINT(41.510832411320955 73.30741610841554) bank90338 +90339 POINT(40.00834913349014 74.61889088802161) bank90339 +90340 POINT(41.22952717005424 73.06094278451216) bank90340 +90341 POINT(39.96640366014643 73.73638357974714) bank90341 +90342 POINT(41.37499638697197 73.50101865286581) bank90342 +90343 POINT(39.960058677971915 73.17330095647966) bank90343 +90344 POINT(40.902282481462386 73.9079654716612) bank90344 +90345 POINT(40.41445767495509 73.36589698651218) bank90345 +90346 POINT(40.5273662217089 73.14214333564027) bank90346 +90347 POINT(40.03572116142719 73.82101392792701) bank90347 +90348 POINT(40.07386412474728 73.06815466405608) bank90348 +90349 POINT(39.79903989869945 74.12700393613629) bank90349 +90350 POINT(39.97760139833591 73.77890233146114) bank90350 +90351 POINT(41.41848331163812 73.0530391455188) bank90351 +90352 POINT(40.41371417269827 73.60287307338322) bank90352 +90353 POINT(40.887177430447636 73.65432535686647) bank90353 +90354 POINT(40.162277973755764 73.32886950074594) bank90354 +90355 POINT(40.43992262927017 73.40180693078804) bank90355 +90356 POINT(39.91952532171669 74.31304325437695) bank90356 +90357 POINT(41.23473703944863 74.37304413418646) bank90357 +90358 POINT(40.073160669121904 73.27082860711015) bank90358 +90359 POINT(40.04265715376707 74.66053155126231) bank90359 +90360 POINT(40.81423440726337 74.95900402393409) bank90360 +90361 POINT(40.41055193989289 74.30931408025393) bank90361 +90362 POINT(40.818560311415936 74.09226511176946) bank90362 +90363 POINT(39.80138487059638 74.44274991595594) bank90363 +90364 POINT(40.73908080351228 73.26616868117814) bank90364 +90365 POINT(40.05002517874192 74.74192690129587) bank90365 +90366 POINT(39.87013860297727 73.64052729318068) bank90366 +90367 POINT(40.55964854065138 73.86632652287085) bank90367 +90368 POINT(41.2552168554284 74.96626059403746) bank90368 +90369 POINT(41.036150254596734 73.98619851095084) bank90369 +90370 POINT(40.85719529614448 73.1145816493091) bank90370 +90371 POINT(39.7543171148061 73.32195418895849) bank90371 +90372 POINT(40.51748725079904 74.98662000846456) bank90372 +90373 POINT(39.9727924560632 74.35291251822343) bank90373 +90374 POINT(39.9667004105934 73.14792369116194) bank90374 +90375 POINT(40.66622593942114 74.86472798267341) bank90375 +90376 POINT(40.01092290600261 74.07593415185679) bank90376 +90377 POINT(40.31479630601474 74.86282101702932) bank90377 +90378 POINT(40.510359014725175 73.98769206816795) bank90378 +90379 POINT(40.92166225746941 73.47111852899732) bank90379 +90380 POINT(40.490427342221395 73.76515403166584) bank90380 +90381 POINT(39.92398240744633 74.8300064103062) bank90381 +90382 POINT(41.69896732325289 73.90726145019288) bank90382 +90383 POINT(40.35888541776642 74.10552050447363) bank90383 +90384 POINT(40.58842592418819 74.89452868878195) bank90384 +90385 POINT(40.8640077342861 74.55918387369397) bank90385 +90386 POINT(40.97592613733348 73.3391521005647) bank90386 +90387 POINT(41.26720890886117 73.54955710050878) bank90387 +90388 POINT(41.17803409106907 74.43660990608741) bank90388 +90389 POINT(41.461228675460205 74.380094457194) bank90389 +90390 POINT(39.72948963576922 74.56966012587057) bank90390 +90391 POINT(40.547795653282186 73.02214537820889) bank90391 +90392 POINT(40.965499576378264 73.85667515489466) bank90392 +90393 POINT(40.67627997203221 73.19487727850691) bank90393 +90394 POINT(41.19872869057218 74.54295437206756) bank90394 +90395 POINT(40.35128856071586 73.97022093030697) bank90395 +90396 POINT(40.80939926229687 73.94402463522096) bank90396 +90397 POINT(41.43271409325833 73.33589840256813) bank90397 +90398 POINT(41.49048941236926 74.2288226408126) bank90398 +90399 POINT(40.428041691817526 74.90622825059657) bank90399 +90400 POINT(40.59581422066093 73.8154568753443) bank90400 +90401 POINT(41.42740663524457 73.1839857905814) bank90401 +90402 POINT(39.98075268509429 74.64033638696947) bank90402 +90403 POINT(39.97880560374397 74.77235767717106) bank90403 +90404 POINT(40.07411499133342 74.20325576361137) bank90404 +90405 POINT(40.26840483134022 73.22643544701701) bank90405 +90406 POINT(39.97216497470384 73.690928653202) bank90406 +90407 POINT(40.95439329586356 74.96976389981222) bank90407 +90408 POINT(40.34634397002064 74.47884973878642) bank90408 +90409 POINT(41.64795328019159 73.37169613401274) bank90409 +90410 POINT(40.683750635482376 74.62594697974939) bank90410 +90411 POINT(41.40964308258526 74.20559562693597) bank90411 +90412 POINT(40.865270606850046 74.56923216806037) bank90412 +90413 POINT(40.65330741220549 74.04772186792353) bank90413 +90414 POINT(41.4795016160912 74.59710911428266) bank90414 +90415 POINT(39.93141752446428 74.9892800774219) bank90415 +90416 POINT(39.75466062273655 73.87145072658896) bank90416 +90417 POINT(41.009106114535214 73.99119561929528) bank90417 +90418 POINT(41.530535436517994 74.98650793146639) bank90418 +90419 POINT(40.75022863518414 73.86216741597207) bank90419 +90420 POINT(39.77335293552341 74.69402232781857) bank90420 +90421 POINT(40.09129059676022 74.4926477214434) bank90421 +90422 POINT(40.63726733589552 74.34190531124011) bank90422 +90423 POINT(41.262302519405274 74.42060612740153) bank90423 +90424 POINT(39.997411318228565 74.97729658340457) bank90424 +90425 POINT(41.00832001380283 74.74930079618962) bank90425 +90426 POINT(41.652683321345506 74.25475517765958) bank90426 +90427 POINT(40.97273576713068 74.49736899216039) bank90427 +90428 POINT(40.00541275775521 74.8778671988591) bank90428 +90429 POINT(40.01464391780182 73.0376476239779) bank90429 +90430 POINT(40.4348020697604 73.28083690300036) bank90430 +90431 POINT(40.95617257001011 73.08198978179804) bank90431 +90432 POINT(40.20170416941501 73.44577732730355) bank90432 +90433 POINT(41.48327754604528 74.19026861978895) bank90433 +90434 POINT(40.947859227105674 74.65701515575611) bank90434 +90435 POINT(40.87013325856067 73.48591753734105) bank90435 +90436 POINT(40.38933517328247 73.85318131655629) bank90436 +90437 POINT(40.38013681071111 73.13674468729275) bank90437 +90438 POINT(40.837982055491 74.46567472446665) bank90438 +90439 POINT(41.60057622005171 74.64464557392587) bank90439 +90440 POINT(40.16338615015683 73.19079341977127) bank90440 +90441 POINT(40.04755418786233 73.84635362698634) bank90441 +90442 POINT(41.67953176380032 74.35484204823867) bank90442 +90443 POINT(40.55706932901906 74.88148398813566) bank90443 +90444 POINT(39.74550073686493 74.78271759693841) bank90444 +90445 POINT(39.72116941468491 73.82487324933508) bank90445 +90446 POINT(39.954551847466 74.96706315594085) bank90446 +90447 POINT(39.88217177933641 73.33979200601739) bank90447 +90448 POINT(40.69702609602895 73.81940514069156) bank90448 +90449 POINT(41.01701129560724 73.97685781296171) bank90449 +90450 POINT(40.82769941055224 73.60281734054567) bank90450 +90451 POINT(41.70685493560555 74.78865937632463) bank90451 +90452 POINT(41.391974530637654 74.73226256357472) bank90452 +90453 POINT(41.45553341712447 73.52536828943967) bank90453 +90454 POINT(40.43357731858109 74.40576295025527) bank90454 +90455 POINT(40.712999243607285 73.09515223836648) bank90455 +90456 POINT(40.968222595516394 74.03760724581137) bank90456 +90457 POINT(41.135979137988336 74.00140136108703) bank90457 +90458 POINT(39.997246087056624 73.02041467820371) bank90458 +90459 POINT(39.857964550064494 74.66324833921276) bank90459 +90460 POINT(40.75711863100038 73.6792370882427) bank90460 +90461 POINT(39.940724419951394 74.3225022802241) bank90461 +90462 POINT(41.3722089572069 74.38373937041433) bank90462 +90463 POINT(41.058130928504426 73.5923677673402) bank90463 +90464 POINT(41.548494357851126 73.59365798433352) bank90464 +90465 POINT(41.0643656102133 74.55871605043876) bank90465 +90466 POINT(39.8134942636116 73.68948959587802) bank90466 +90467 POINT(40.65338063821404 73.48670857658215) bank90467 +90468 POINT(40.88595140891652 73.22684304000312) bank90468 +90469 POINT(40.869115111735475 74.96575743987691) bank90469 +90470 POINT(41.08187373235382 73.28552944127438) bank90470 +90471 POINT(41.16975829185464 73.54609590669274) bank90471 +90472 POINT(40.094411575027046 73.48808569590547) bank90472 +90473 POINT(39.79989668140861 74.01519690810223) bank90473 +90474 POINT(40.8764308264367 73.04187013232756) bank90474 +90475 POINT(40.373163053938974 74.94815943676738) bank90475 +90476 POINT(39.80860989844901 74.23556790388882) bank90476 +90477 POINT(41.50966119753221 73.71468780245421) bank90477 +90478 POINT(39.992149789320344 74.57227949732298) bank90478 +90479 POINT(40.005427696730436 73.52067333088618) bank90479 +90480 POINT(40.549169365319614 73.898942101377) bank90480 +90481 POINT(40.92338050011745 73.95794258631561) bank90481 +90482 POINT(40.786807536637106 74.68403282845453) bank90482 +90483 POINT(41.40124708930683 74.25383062406114) bank90483 +90484 POINT(40.86963600650026 74.15830620486972) bank90484 +90485 POINT(40.96674150685469 74.54266783671339) bank90485 +90486 POINT(40.6658049007321 74.37324542714704) bank90486 +90487 POINT(40.694491792729806 73.9379074698377) bank90487 +90488 POINT(40.7032033850588 74.27774605175159) bank90488 +90489 POINT(40.53579561512057 73.5972270125631) bank90489 +90490 POINT(41.527717512647314 73.67313702979808) bank90490 +90491 POINT(41.50265937803803 74.42488098292937) bank90491 +90492 POINT(39.76256783530851 74.95122931191848) bank90492 +90493 POINT(41.50907619253454 73.37612679718659) bank90493 +90494 POINT(40.935467970727224 74.65390465639028) bank90494 +90495 POINT(40.153273637360336 74.74062707151428) bank90495 +90496 POINT(40.800192458072196 73.52672386887211) bank90496 +90497 POINT(40.97975837848325 73.89444304397406) bank90497 +90498 POINT(41.554889107501545 74.08744989275678) bank90498 +90499 POINT(40.84219981582588 73.59411007679226) bank90499 +90500 POINT(40.17367493161077 74.81090607347494) bank90500 +90501 POINT(40.641090250342224 73.97166532937572) bank90501 +90502 POINT(41.227036981687185 73.35855459066627) bank90502 +90503 POINT(41.51441137941355 73.04135209679247) bank90503 +90504 POINT(39.74613453559349 74.52904166663504) bank90504 +90505 POINT(41.24179053475364 74.32860576931054) bank90505 +90506 POINT(40.88706420370678 74.9494151301095) bank90506 +90507 POINT(41.29739986792731 74.96532303538505) bank90507 +90508 POINT(40.914821444673265 74.76226725813727) bank90508 +90509 POINT(41.15728265869444 73.19418424273326) bank90509 +90510 POINT(39.97155789202796 74.33428007407808) bank90510 +90511 POINT(41.354684901351376 74.89179470092506) bank90511 +90512 POINT(40.106530718293726 73.54442945429471) bank90512 +90513 POINT(40.37952606353536 73.91493631890023) bank90513 +90514 POINT(40.04923715061115 74.79623928935905) bank90514 +90515 POINT(41.38686196057134 73.85886440446161) bank90515 +90516 POINT(39.76932035231679 73.1612336123485) bank90516 +90517 POINT(41.05062884728422 73.69323247978582) bank90517 +90518 POINT(41.52134735570658 74.93636735326646) bank90518 +90519 POINT(40.61579080816579 73.02904198157478) bank90519 +90520 POINT(41.00778743129985 73.03081846776043) bank90520 +90521 POINT(40.363320868636556 73.69773411952305) bank90521 +90522 POINT(41.61855651075125 74.84287035383811) bank90522 +90523 POINT(41.69922182918914 73.86736743083796) bank90523 +90524 POINT(41.328825013153974 74.09824143294185) bank90524 +90525 POINT(41.434739120950766 74.8856006984523) bank90525 +90526 POINT(39.78011750237533 73.07656379196919) bank90526 +90527 POINT(41.28817821432979 74.41310550111127) bank90527 +90528 POINT(40.283768536667814 74.819042942237) bank90528 +90529 POINT(40.9358948207664 74.34788701175918) bank90529 +90530 POINT(41.0754475795543 73.5102001954516) bank90530 +90531 POINT(41.45434417065659 73.78792846381397) bank90531 +90532 POINT(40.3969659746455 73.22595443340266) bank90532 +90533 POINT(41.707387274779514 73.41248026037813) bank90533 +90534 POINT(41.68572320339267 74.80037270974377) bank90534 +90535 POINT(40.390269302901714 73.59396922161953) bank90535 +90536 POINT(41.68396513062624 74.93416295558643) bank90536 +90537 POINT(41.27808876772866 73.499708352682) bank90537 +90538 POINT(40.710186238489406 74.80369771530889) bank90538 +90539 POINT(40.17408878801049 73.9250682127397) bank90539 +90540 POINT(39.990418152343786 74.6176296800463) bank90540 +90541 POINT(40.60567636807417 74.2721928526109) bank90541 +90542 POINT(41.665661282632755 74.94434142752588) bank90542 +90543 POINT(39.73335597501012 74.6311692871718) bank90543 +90544 POINT(40.977608816415426 74.71595425328778) bank90544 +90545 POINT(41.65286018009652 74.46658915547921) bank90545 +90546 POINT(40.401039795989426 73.8081997354874) bank90546 +90547 POINT(40.35949275060113 73.28935937141887) bank90547 +90548 POINT(39.90626952976938 74.09394975495903) bank90548 +90549 POINT(40.72360554224164 74.1319422571545) bank90549 +90550 POINT(39.95993160539659 73.1222803043543) bank90550 +90551 POINT(41.55242671921839 74.54525747823702) bank90551 +90552 POINT(40.36415518037978 74.81420493571501) bank90552 +90553 POINT(41.20022973869209 74.09727189791644) bank90553 +90554 POINT(39.98881171001375 74.30501387340033) bank90554 +90555 POINT(41.12407471141841 73.3860069517721) bank90555 +90556 POINT(41.12707773033172 73.36603417484179) bank90556 +90557 POINT(40.763883820189214 73.44119651428849) bank90557 +90558 POINT(39.895421656183366 73.6971247999675) bank90558 +90559 POINT(40.34213222608138 73.23277595881096) bank90559 +90560 POINT(39.89196627493986 73.06823366821354) bank90560 +90561 POINT(41.48276639652492 74.80569086373464) bank90561 +90562 POINT(41.339491322127536 74.83231264444711) bank90562 +90563 POINT(40.70897610217595 74.4098108020879) bank90563 +90564 POINT(40.0903894116266 73.9141813823527) bank90564 +90565 POINT(41.26771365737588 74.09064658241807) bank90565 +90566 POINT(39.89973154720067 74.27919601056014) bank90566 +90567 POINT(41.54132504917635 74.11091734722007) bank90567 +90568 POINT(40.99702171987318 73.5977708782443) bank90568 +90569 POINT(40.866976198417966 74.20628835934717) bank90569 +90570 POINT(40.53762217941249 73.58679803648037) bank90570 +90571 POINT(40.860698509287715 73.42648703012532) bank90571 +90572 POINT(40.91626082515736 73.81732999348765) bank90572 +90573 POINT(40.869138803015595 74.27769255436613) bank90573 +90574 POINT(40.204184911438276 74.49596358336373) bank90574 +90575 POINT(41.448964762252736 73.78013303357814) bank90575 +90576 POINT(40.384059886485836 73.89076877777005) bank90576 +90577 POINT(41.2220804461913 74.79952433868728) bank90577 +90578 POINT(41.535073595551154 74.90672542632267) bank90578 +90579 POINT(40.285532958095644 74.22273550159073) bank90579 +90580 POINT(41.18694517421301 74.90507299534484) bank90580 +90581 POINT(40.32622532682201 74.86542827805197) bank90581 +90582 POINT(39.77039154418291 73.7899825877378) bank90582 +90583 POINT(41.36821595642107 73.44337476685195) bank90583 +90584 POINT(39.78150344068171 73.28840674821713) bank90584 +90585 POINT(40.12266195162766 73.44105437234343) bank90585 +90586 POINT(41.40401489033668 73.78912831572119) bank90586 +90587 POINT(40.013042521306865 74.72905781390584) bank90587 +90588 POINT(41.61597015932689 74.74047597365235) bank90588 +90589 POINT(40.87742218351569 73.8767042353511) bank90589 +90590 POINT(40.215769300557746 73.56711568758614) bank90590 +90591 POINT(39.95353143126892 73.54440685549781) bank90591 +90592 POINT(40.520107296599065 73.37161715819742) bank90592 +90593 POINT(40.8494624464805 73.47444324927362) bank90593 +90594 POINT(40.97100079885556 73.74879386401186) bank90594 +90595 POINT(41.53882429267745 73.84097016398223) bank90595 +90596 POINT(40.43502769437655 73.49094586264013) bank90596 +90597 POINT(41.22905047691663 73.55792010056075) bank90597 +90598 POINT(41.67175376542332 73.26111484751183) bank90598 +90599 POINT(40.799728611345664 74.3229492645178) bank90599 +90600 POINT(39.889529892400866 73.33001843129315) bank90600 +90601 POINT(41.02374649826434 74.91772548700098) bank90601 +90602 POINT(40.5051617856765 74.57570022321383) bank90602 +90603 POINT(41.50903692273967 74.53466609467631) bank90603 +90604 POINT(40.86933864279766 73.54522693700756) bank90604 +90605 POINT(41.30456876451296 73.79137011113549) bank90605 +90606 POINT(40.03430683665458 74.18749871362007) bank90606 +90607 POINT(40.500436566069524 74.12391966146052) bank90607 +90608 POINT(40.25280276852837 74.70138008034928) bank90608 +90609 POINT(41.6379337100216 74.7378310805493) bank90609 +90610 POINT(40.263523749584145 74.887616790254) bank90610 +90611 POINT(39.78856194605396 73.27601482056419) bank90611 +90612 POINT(40.619265827873555 73.66580145176475) bank90612 +90613 POINT(41.25495139649206 74.62648051096666) bank90613 +90614 POINT(40.12518562165222 73.40270595065799) bank90614 +90615 POINT(40.50266851505395 73.91784709670253) bank90615 +90616 POINT(40.38900457034304 73.39806149182952) bank90616 +90617 POINT(41.32102267245464 73.1248748609737) bank90617 +90618 POINT(40.535660848224985 73.4229738222678) bank90618 +90619 POINT(40.446918465099714 74.48912385064553) bank90619 +90620 POINT(40.35109763452463 73.34121502455243) bank90620 +90621 POINT(40.382189646638224 73.95046496458616) bank90621 +90622 POINT(40.93627243501129 74.29795760083735) bank90622 +90623 POINT(40.39636471546145 73.58748088870847) bank90623 +90624 POINT(40.640193079233 74.71525709661265) bank90624 +90625 POINT(39.852244958901395 74.92678166036471) bank90625 +90626 POINT(40.78501632674625 73.13249847563232) bank90626 +90627 POINT(40.36219784314018 74.77003402672773) bank90627 +90628 POINT(41.47051912765522 73.25958153659737) bank90628 +90629 POINT(41.1369138415092 74.4931852086156) bank90629 +90630 POINT(39.749674542573004 73.63621642688263) bank90630 +90631 POINT(40.013821568189904 74.6330814020329) bank90631 +90632 POINT(40.74074032389602 74.22742413887981) bank90632 +90633 POINT(41.101117780656274 74.50561023649225) bank90633 +90634 POINT(39.72365119132448 74.46390049864624) bank90634 +90635 POINT(41.246850852159355 74.720567542473) bank90635 +90636 POINT(41.44688386974835 74.55252086228963) bank90636 +90637 POINT(41.486272888229166 73.15517858898998) bank90637 +90638 POINT(40.15848486594775 73.34593456893342) bank90638 +90639 POINT(40.72484885578071 74.38774534223982) bank90639 +90640 POINT(40.04643709173911 74.41723034663738) bank90640 +90641 POINT(39.72192243632905 74.42931032873685) bank90641 +90642 POINT(40.00174893897903 73.10543785494116) bank90642 +90643 POINT(40.57883584204829 73.91928182819157) bank90643 +90644 POINT(41.3097848759798 74.18216747747394) bank90644 +90645 POINT(40.24913647455915 74.95925075844166) bank90645 +90646 POINT(40.27715975546796 73.42666333448956) bank90646 +90647 POINT(39.99413193535767 73.21531100184832) bank90647 +90648 POINT(40.52439829041299 74.74959540104719) bank90648 +90649 POINT(41.145318190568105 73.23807651638549) bank90649 +90650 POINT(41.49204087501251 74.23695239843242) bank90650 +90651 POINT(40.587589669515964 74.47296869146952) bank90651 +90652 POINT(40.629367300509735 74.97077912360558) bank90652 +90653 POINT(41.46779568840181 73.82115034890369) bank90653 +90654 POINT(40.56712991090636 74.73937230843804) bank90654 +90655 POINT(40.227270310813566 73.98236711471134) bank90655 +90656 POINT(39.75365872791309 73.90568797364323) bank90656 +90657 POINT(41.36313770969612 74.34327082426014) bank90657 +90658 POINT(41.26687921589078 74.15590776296884) bank90658 +90659 POINT(41.37877078848473 73.56823999261275) bank90659 +90660 POINT(40.70270918520152 74.72336211676388) bank90660 +90661 POINT(41.48451158934204 73.97532591836284) bank90661 +90662 POINT(39.79929674967443 73.36221654729349) bank90662 +90663 POINT(41.16912891709438 74.17198748808481) bank90663 +90664 POINT(39.916169963671315 73.17430924844146) bank90664 +90665 POINT(40.3907123018292 73.51895506021222) bank90665 +90666 POINT(41.32773635114743 73.69040068467604) bank90666 +90667 POINT(40.70743669626611 74.64436919283209) bank90667 +90668 POINT(41.15634534765118 74.42229844921444) bank90668 +90669 POINT(40.060794473841746 73.22592918782446) bank90669 +90670 POINT(40.44936935881914 73.56862968805258) bank90670 +90671 POINT(41.579394381292516 73.2965049545785) bank90671 +90672 POINT(41.39541417370709 74.19898416443975) bank90672 +90673 POINT(40.34176874918657 74.31007646831927) bank90673 +90674 POINT(41.0104309662228 74.41341503236093) bank90674 +90675 POINT(39.782971435307665 73.78206570806269) bank90675 +90676 POINT(40.93408462075178 74.49715025531992) bank90676 +90677 POINT(40.12183810074773 73.83157360029375) bank90677 +90678 POINT(41.57920634844544 74.58633308694387) bank90678 +90679 POINT(41.06372564538186 73.62977601559402) bank90679 +90680 POINT(39.753254847133576 74.61915711326984) bank90680 +90681 POINT(40.22955780211572 73.17711032099469) bank90681 +90682 POINT(41.30849620778583 73.80092070889856) bank90682 +90683 POINT(41.25117784856696 74.74948577512674) bank90683 +90684 POINT(40.61876473761936 73.01262994337341) bank90684 +90685 POINT(39.9795582791059 74.02194957398031) bank90685 +90686 POINT(40.22349140530929 74.68271407672401) bank90686 +90687 POINT(40.678996587623224 73.72161626504177) bank90687 +90688 POINT(40.14855191366391 74.08205653933511) bank90688 +90689 POINT(41.46250054512108 74.66791685566899) bank90689 +90690 POINT(39.81739346341645 74.37625396936257) bank90690 +90691 POINT(40.52171598798338 73.0375988174303) bank90691 +90692 POINT(39.90132123146026 73.20661611738954) bank90692 +90693 POINT(39.9338778811451 74.2810387979255) bank90693 +90694 POINT(41.387853776495795 74.01151386350439) bank90694 +90695 POINT(41.57432368595534 74.28730025886276) bank90695 +90696 POINT(41.260885132080084 74.96871673938509) bank90696 +90697 POINT(39.726376208549425 73.67710048570426) bank90697 +90698 POINT(40.730369872904284 73.0255074002187) bank90698 +90699 POINT(41.59765365465778 73.7899881880185) bank90699 +90700 POINT(39.83803208958081 74.83184456002114) bank90700 +90701 POINT(40.041621293752435 73.19761241707366) bank90701 +90702 POINT(40.05560888770675 73.1773615424342) bank90702 +90703 POINT(40.33828577066775 74.26461991214508) bank90703 +90704 POINT(41.41944090292928 73.29460853756808) bank90704 +90705 POINT(41.61554761265421 73.4308244094862) bank90705 +90706 POINT(41.07929465424386 74.48170814731144) bank90706 +90707 POINT(40.20602580125427 74.65481012814674) bank90707 +90708 POINT(40.624696777297615 74.76580780789267) bank90708 +90709 POINT(40.83341924075001 74.29984082665715) bank90709 +90710 POINT(39.75526410121407 74.82679887995414) bank90710 +90711 POINT(40.90163079017283 73.07863723999938) bank90711 +90712 POINT(41.11879583590408 73.5707210270969) bank90712 +90713 POINT(41.51071269129824 74.99698936962844) bank90713 +90714 POINT(41.24729874596612 74.91611419347167) bank90714 +90715 POINT(40.9449138494296 74.70166616922198) bank90715 +90716 POINT(41.11676948575061 73.50098721139905) bank90716 +90717 POINT(40.429337459799555 74.2551533034162) bank90717 +90718 POINT(41.54279207070008 74.61536443325218) bank90718 +90719 POINT(41.545045598553415 74.34411109383718) bank90719 +90720 POINT(41.10577803609914 74.1585395741142) bank90720 +90721 POINT(40.15050165677941 74.48046874593454) bank90721 +90722 POINT(40.44078957898188 73.33313853388535) bank90722 +90723 POINT(40.67099285054106 74.7221982306055) bank90723 +90724 POINT(41.56458831034186 73.63189038400905) bank90724 +90725 POINT(41.69670854619471 74.59598730369916) bank90725 +90726 POINT(40.55659723102419 73.86325285690343) bank90726 +90727 POINT(40.32802189105248 74.25996185563537) bank90727 +90728 POINT(39.94738925833015 73.78654747269783) bank90728 +90729 POINT(40.163820559021325 74.6148045866018) bank90729 +90730 POINT(41.2186185532696 74.01678434238376) bank90730 +90731 POINT(40.073631873599304 73.72043366203816) bank90731 +90732 POINT(40.66110773076275 73.41279340066413) bank90732 +90733 POINT(40.84169985455867 74.62301874994094) bank90733 +90734 POINT(41.1694728492725 73.52582406233078) bank90734 +90735 POINT(39.88885655002035 74.21038039371383) bank90735 +90736 POINT(40.206808006044476 74.7361560370739) bank90736 +90737 POINT(39.80287190444771 73.67915742760688) bank90737 +90738 POINT(39.98006705907584 73.9272640652349) bank90738 +90739 POINT(41.26391325679805 74.52220465143401) bank90739 +90740 POINT(40.452893100463264 73.02479281281785) bank90740 +90741 POINT(40.39001246719621 74.3006946169598) bank90741 +90742 POINT(41.70286786745785 74.40643501220745) bank90742 +90743 POINT(40.04470751392266 74.8728331597037) bank90743 +90744 POINT(39.718539625323245 73.29784957852455) bank90744 +90745 POINT(41.08094084581106 73.81433407940358) bank90745 +90746 POINT(41.48527507649784 74.87237607234786) bank90746 +90747 POINT(41.274058815428035 74.16633397393377) bank90747 +90748 POINT(41.65133673842297 73.72131321844192) bank90748 +90749 POINT(40.590999304900336 74.27193213399663) bank90749 +90750 POINT(41.24118490989602 73.07078061633719) bank90750 +90751 POINT(41.45248998410713 73.96687840632107) bank90751 +90752 POINT(41.25335396227061 74.50650621896874) bank90752 +90753 POINT(40.375324699034984 74.97704738861913) bank90753 +90754 POINT(39.7260731080427 73.65299400868592) bank90754 +90755 POINT(41.02228437616524 73.6996788865284) bank90755 +90756 POINT(40.669006337069575 74.62692446034157) bank90756 +90757 POINT(39.86092615991691 73.93168472135264) bank90757 +90758 POINT(40.00477705163905 73.54447710083018) bank90758 +90759 POINT(40.00091776058418 73.664720694793) bank90759 +90760 POINT(41.33681822363834 74.49082514286373) bank90760 +90761 POINT(40.72070575434879 73.12972251612564) bank90761 +90762 POINT(40.77563807658181 74.73018362321821) bank90762 +90763 POINT(41.27492833398806 74.542947018229) bank90763 +90764 POINT(41.26797649885883 74.94409881153331) bank90764 +90765 POINT(39.871763045161345 73.71432912989165) bank90765 +90766 POINT(40.78809073362306 74.76998769944186) bank90766 +90767 POINT(41.46237989605078 73.97814547109292) bank90767 +90768 POINT(40.048545080377195 73.446187713536) bank90768 +90769 POINT(39.852288552849565 73.43778342488399) bank90769 +90770 POINT(41.41185171321802 74.42769063264592) bank90770 +90771 POINT(41.30898734517739 74.71973195117965) bank90771 +90772 POINT(40.30113952867575 73.5154457438221) bank90772 +90773 POINT(40.71995101539647 73.35327281606753) bank90773 +90774 POINT(39.85016877412775 74.43760628483055) bank90774 +90775 POINT(40.03268867661641 74.77312497064837) bank90775 +90776 POINT(40.056346080914075 74.49203805753456) bank90776 +90777 POINT(40.38455978757869 74.18834395617527) bank90777 +90778 POINT(39.90700712146256 73.62514809534005) bank90778 +90779 POINT(41.18938613757445 74.08112699857747) bank90779 +90780 POINT(41.246703157525076 73.63419278321116) bank90780 +90781 POINT(40.127272410157595 73.33697311228367) bank90781 +90782 POINT(40.300128218187176 73.39962173932516) bank90782 +90783 POINT(41.47427265505648 73.0211215740977) bank90783 +90784 POINT(41.356475106736625 73.32046153875183) bank90784 +90785 POINT(40.592707275070964 74.76946108616569) bank90785 +90786 POINT(40.977476814429735 74.02171219478689) bank90786 +90787 POINT(41.202157177261014 74.04463730975638) bank90787 +90788 POINT(40.2216294682289 74.08748094334304) bank90788 +90789 POINT(40.32835004940966 73.16010859895611) bank90789 +90790 POINT(40.67521880112965 73.20095957532388) bank90790 +90791 POINT(41.69371703557118 74.58493903682134) bank90791 +90792 POINT(40.03376208632401 74.57873028332429) bank90792 +90793 POINT(41.496601055890764 74.07197649094806) bank90793 +90794 POINT(41.04077618226728 73.97237031162683) bank90794 +90795 POINT(41.381834012367676 73.94803115398628) bank90795 +90796 POINT(39.93291943520105 74.7094151123171) bank90796 +90797 POINT(41.23405348280787 74.76219007583283) bank90797 +90798 POINT(40.69767188650814 73.7584474991322) bank90798 +90799 POINT(40.508469343843295 73.40271127437549) bank90799 +90800 POINT(40.7985067233505 73.78358691063258) bank90800 +90801 POINT(40.63571608277256 74.55365695646951) bank90801 +90802 POINT(40.70845792723717 73.7841562761086) bank90802 +90803 POINT(41.32764887575838 73.94664170854047) bank90803 +90804 POINT(40.54343696749529 73.03809170896768) bank90804 +90805 POINT(40.63915341607169 73.51532764109649) bank90805 +90806 POINT(40.6190100013925 74.43373941795316) bank90806 +90807 POINT(40.220147687548426 74.06697621240201) bank90807 +90808 POINT(39.86274407618302 74.91428297383194) bank90808 +90809 POINT(40.46108797731268 74.68719904756584) bank90809 +90810 POINT(40.2376674929227 74.43366299901831) bank90810 +90811 POINT(40.05121061223578 74.53737222444654) bank90811 +90812 POINT(39.754156018680895 74.25452473931969) bank90812 +90813 POINT(41.14446819468622 74.86602559127223) bank90813 +90814 POINT(39.767122022439324 73.47215930035611) bank90814 +90815 POINT(40.58708424201907 73.07582421533034) bank90815 +90816 POINT(41.62609478909324 73.37604944074047) bank90816 +90817 POINT(41.60875710957536 74.38912170510366) bank90817 +90818 POINT(40.29862006526079 73.4883153681913) bank90818 +90819 POINT(40.843476924543744 74.34803879117197) bank90819 +90820 POINT(39.96967105790106 74.89382771221946) bank90820 +90821 POINT(41.342735392036666 74.9794606179895) bank90821 +90822 POINT(41.32819438828558 74.24706345885618) bank90822 +90823 POINT(39.80835385124704 74.17082182799061) bank90823 +90824 POINT(40.890182384772885 74.16875354140838) bank90824 +90825 POINT(41.62394064061677 74.19815050053728) bank90825 +90826 POINT(41.35549566740808 74.11730988722532) bank90826 +90827 POINT(40.34731802142211 74.13991015258264) bank90827 +90828 POINT(40.30463579332679 74.62720113781424) bank90828 +90829 POINT(39.896171080930834 73.39114613268667) bank90829 +90830 POINT(41.118983208704286 73.01041383871703) bank90830 +90831 POINT(39.779411879659456 74.86181894747122) bank90831 +90832 POINT(40.00486332106412 74.85978332283797) bank90832 +90833 POINT(40.16505417548027 73.37940615397255) bank90833 +90834 POINT(41.23892467940682 74.10731674200413) bank90834 +90835 POINT(40.33962394566434 74.98301632479156) bank90835 +90836 POINT(41.667540426727925 73.21215142025468) bank90836 +90837 POINT(40.55026005849325 74.57474416882204) bank90837 +90838 POINT(39.7978587907278 74.89603985755062) bank90838 +90839 POINT(40.12719342874287 73.64946463943615) bank90839 +90840 POINT(41.31249716361237 74.2176833747237) bank90840 +90841 POINT(41.379440164965665 74.2954420006521) bank90841 +90842 POINT(41.131773752661104 74.62176860939793) bank90842 +90843 POINT(40.06819733139453 73.12738268352595) bank90843 +90844 POINT(41.579720307408486 74.45505560801757) bank90844 +90845 POINT(41.13255728639596 73.05798705058618) bank90845 +90846 POINT(39.91575836883814 73.59437661621257) bank90846 +90847 POINT(39.73360245123864 74.99074905598758) bank90847 +90848 POINT(40.33939539084458 74.44097110090523) bank90848 +90849 POINT(40.94694919197994 73.1902560080491) bank90849 +90850 POINT(40.71545981613163 74.1182117320771) bank90850 +90851 POINT(41.439550910087675 73.71976696492031) bank90851 +90852 POINT(41.67819869661476 74.81866124824967) bank90852 +90853 POINT(40.67629173287841 74.9524455799797) bank90853 +90854 POINT(40.57555154252662 73.9997906972775) bank90854 +90855 POINT(39.72425390970712 73.0156931523907) bank90855 +90856 POINT(41.67427953570929 73.62053311882063) bank90856 +90857 POINT(41.251490867643284 74.18560185123526) bank90857 +90858 POINT(41.24732902080758 74.80389022453707) bank90858 +90859 POINT(40.45854140538448 74.43987732393504) bank90859 +90860 POINT(40.48103169944202 74.04297565521465) bank90860 +90861 POINT(40.65696958619804 74.68949304935732) bank90861 +90862 POINT(41.70715677792862 74.3549853208279) bank90862 +90863 POINT(39.994452837948856 73.83953461417052) bank90863 +90864 POINT(40.26261764786247 74.84835061855094) bank90864 +90865 POINT(40.29343359492348 74.39482771268862) bank90865 +90866 POINT(40.02089655013229 74.65441735622244) bank90866 +90867 POINT(40.04279332195825 73.1790556006129) bank90867 +90868 POINT(41.619170364727324 73.29957003156028) bank90868 +90869 POINT(40.488635606135844 73.74734005624964) bank90869 +90870 POINT(41.13708921105371 73.60289893292841) bank90870 +90871 POINT(41.68279567654803 73.93754064169987) bank90871 +90872 POINT(41.7065723317994 73.96825883214737) bank90872 +90873 POINT(40.828793269137115 73.75815665491642) bank90873 +90874 POINT(40.703952420510646 73.17155961450233) bank90874 +90875 POINT(41.38451043231416 74.22544213188895) bank90875 +90876 POINT(41.095347035494996 73.5428776651978) bank90876 +90877 POINT(40.03722232103162 74.66121522757176) bank90877 +90878 POINT(40.77408938342628 73.21681407146538) bank90878 +90879 POINT(41.302064515697374 74.37597092474397) bank90879 +90880 POINT(40.803269664649356 74.44417737607655) bank90880 +90881 POINT(41.689626456772345 73.13939609455082) bank90881 +90882 POINT(41.06887388460034 73.81872456683386) bank90882 +90883 POINT(41.3969469788382 73.23935118992337) bank90883 +90884 POINT(41.37732203951474 74.04075171643055) bank90884 +90885 POINT(41.30955568481314 73.71288848939301) bank90885 +90886 POINT(40.7128293336683 73.45967702813452) bank90886 +90887 POINT(40.93966693990602 73.5534594734735) bank90887 +90888 POINT(41.52449264305027 74.53932921624963) bank90888 +90889 POINT(39.849633419048445 73.83460774483058) bank90889 +90890 POINT(41.167918936766895 74.64163123588081) bank90890 +90891 POINT(41.096244793991154 74.00852855658461) bank90891 +90892 POINT(41.16751081617566 74.96215072964395) bank90892 +90893 POINT(40.361178103840295 73.79090025664013) bank90893 +90894 POINT(40.79135494604801 74.81666628312617) bank90894 +90895 POINT(40.97229331935419 73.62931565634459) bank90895 +90896 POINT(41.02528336587227 73.94818644579139) bank90896 +90897 POINT(41.26342208480405 74.07616123169224) bank90897 +90898 POINT(40.81525118756419 74.9720099783525) bank90898 +90899 POINT(40.24959524382446 74.20547513230156) bank90899 +90900 POINT(40.983585707883364 73.01966941427644) bank90900 +90901 POINT(39.9165030389329 73.03631466562713) bank90901 +90902 POINT(41.33299429262365 73.90050153653058) bank90902 +90903 POINT(40.949434539138934 73.29789195166298) bank90903 +90904 POINT(40.36177509167922 73.88866820252883) bank90904 +90905 POINT(41.378437362173386 74.01723898610034) bank90905 +90906 POINT(40.73805612596096 74.66371091043935) bank90906 +90907 POINT(40.111224356987776 73.29926701744783) bank90907 +90908 POINT(40.06048436132941 73.27589325669818) bank90908 +90909 POINT(40.84114309435101 73.77791136342613) bank90909 +90910 POINT(41.60448439832311 74.07249628799886) bank90910 +90911 POINT(40.6592506070662 73.17893842714868) bank90911 +90912 POINT(41.23063259422806 73.07627912775655) bank90912 +90913 POINT(40.29721002616381 74.84120348691327) bank90913 +90914 POINT(41.46178171833335 73.43914444251489) bank90914 +90915 POINT(40.163482519921416 74.00301489545055) bank90915 +90916 POINT(40.60901820298058 74.31271192199496) bank90916 +90917 POINT(40.023451751760675 73.15131987586528) bank90917 +90918 POINT(40.905388157723806 73.26039145295142) bank90918 +90919 POINT(40.59820235402014 73.56748592063609) bank90919 +90920 POINT(40.809468958274906 73.98796961276709) bank90920 +90921 POINT(41.6688088403816 74.90786147090743) bank90921 +90922 POINT(39.99599920126097 74.16587374469735) bank90922 +90923 POINT(41.642545083316726 73.38573119642545) bank90923 +90924 POINT(41.22409616942527 73.01395142255873) bank90924 +90925 POINT(40.847922025808856 73.64359439787309) bank90925 +90926 POINT(39.87038018604148 74.8332464497998) bank90926 +90927 POINT(40.15000886207112 73.25981223973481) bank90927 +90928 POINT(40.93145615926109 74.34706093913044) bank90928 +90929 POINT(41.16320140590298 74.46694277909955) bank90929 +90930 POINT(41.25077907959529 74.00798564929134) bank90930 +90931 POINT(40.76551752379477 73.66950690970864) bank90931 +90932 POINT(40.81070683213807 73.72397521253316) bank90932 +90933 POINT(41.39963423371797 74.73020477576338) bank90933 +90934 POINT(40.852135158921214 74.65152268953128) bank90934 +90935 POINT(41.13182902605462 74.81581664741698) bank90935 +90936 POINT(40.85279242053319 74.52991796129162) bank90936 +90937 POINT(41.650907818914206 74.1805757720386) bank90937 +90938 POINT(40.80780305415829 74.60642141982774) bank90938 +90939 POINT(41.12970206288385 73.40220274058692) bank90939 +90940 POINT(41.13561823048135 74.0503584489508) bank90940 +90941 POINT(41.370779133618626 74.07364943280704) bank90941 +90942 POINT(40.81655669997271 74.5211103022694) bank90942 +90943 POINT(41.56612787963534 73.20830282056865) bank90943 +90944 POINT(41.362899811755916 74.09634112762305) bank90944 +90945 POINT(40.9638280226793 74.27794293910604) bank90945 +90946 POINT(40.24405686929716 74.32342508511402) bank90946 +90947 POINT(40.22738403391784 73.02258493447836) bank90947 +90948 POINT(40.65986234006073 74.5761293564707) bank90948 +90949 POINT(40.92121407234861 73.86273134975943) bank90949 +90950 POINT(41.16850221121075 74.71493753468158) bank90950 +90951 POINT(40.21986449512633 74.81276754442683) bank90951 +90952 POINT(41.445271179736764 74.56909109411114) bank90952 +90953 POINT(39.93901463981525 74.00835536920385) bank90953 +90954 POINT(40.79479072245737 73.31487820783815) bank90954 +90955 POINT(40.96651403508486 73.51194466442442) bank90955 +90956 POINT(41.09981471078843 74.35846233351609) bank90956 +90957 POINT(41.59271652377423 74.27592188770834) bank90957 +90958 POINT(40.90646216033916 73.82613767990279) bank90958 +90959 POINT(40.41000392179911 73.30752251949365) bank90959 +90960 POINT(40.958269549601695 73.378452089779) bank90960 +90961 POINT(40.51612357936301 74.93281740429883) bank90961 +90962 POINT(41.04676323106261 74.22981122765545) bank90962 +90963 POINT(41.070175994621586 73.71871570341379) bank90963 +90964 POINT(41.20838514203781 73.67890921362527) bank90964 +90965 POINT(39.8630394866225 74.05729299258064) bank90965 +90966 POINT(41.23401182707855 73.07325738816515) bank90966 +90967 POINT(40.84576807663711 73.42647463579203) bank90967 +90968 POINT(41.43463508965407 74.01822657132668) bank90968 +90969 POINT(40.70088333096946 73.50929415245602) bank90969 +90970 POINT(40.585091783071384 73.38673985263323) bank90970 +90971 POINT(41.535117678516116 74.09325702141481) bank90971 +90972 POINT(41.58741576442338 74.81872191596786) bank90972 +90973 POINT(41.17804760205377 73.96016948296072) bank90973 +90974 POINT(40.836410018757476 74.8063089768311) bank90974 +90975 POINT(40.23201345266752 74.02746218595007) bank90975 +90976 POINT(40.86214120287462 73.60039130090752) bank90976 +90977 POINT(40.281769183544405 74.05335917684509) bank90977 +90978 POINT(40.205682564332854 73.33045248631545) bank90978 +90979 POINT(40.226659125692784 74.39046102366476) bank90979 +90980 POINT(40.128323759314625 73.12163785076119) bank90980 +90981 POINT(40.39055251328519 73.99485213761052) bank90981 +90982 POINT(41.122484618148754 74.79756071772279) bank90982 +90983 POINT(41.239222280856126 73.07430159574386) bank90983 +90984 POINT(39.747692277071444 73.28580707907885) bank90984 +90985 POINT(41.37732626944891 74.21530496623289) bank90985 +90986 POINT(39.88398373602193 74.49217771774575) bank90986 +90987 POINT(41.07643993564504 74.78923265639276) bank90987 +90988 POINT(41.553374956518745 73.19792327614466) bank90988 +90989 POINT(40.404323254848194 74.51876526946515) bank90989 +90990 POINT(41.17611292070018 73.32409674425777) bank90990 +90991 POINT(40.277711026796695 74.26339844278138) bank90991 +90992 POINT(40.40105112962646 73.24257810110204) bank90992 +90993 POINT(40.16068005583808 74.81398807475132) bank90993 +90994 POINT(41.53158130160303 74.20436679589206) bank90994 +90995 POINT(40.695493235722545 73.55363818157157) bank90995 +90996 POINT(40.294178521872226 73.74781499957606) bank90996 +90997 POINT(40.38474646633004 74.62587612188278) bank90997 +90998 POINT(41.008486210248584 74.14479942972166) bank90998 +90999 POINT(41.53685000334063 73.01248920727437) bank90999 +91000 POINT(41.08159626720948 74.87631413284494) bank91000 +91001 POINT(41.332350937707844 73.0482513142828) bank91001 +91002 POINT(41.09262164392334 73.01341210341117) bank91002 +91003 POINT(39.9169871137423 74.36669048364033) bank91003 +91004 POINT(39.82808820007323 74.65540484648328) bank91004 +91005 POINT(39.80343529388804 73.45804504637735) bank91005 +91006 POINT(41.47042956394541 74.19913634495758) bank91006 +91007 POINT(41.62890288428211 74.48556606711114) bank91007 +91008 POINT(40.43552743468049 73.81969028851243) bank91008 +91009 POINT(41.08143882676493 73.6168216226863) bank91009 +91010 POINT(40.98366489894686 74.69403278812452) bank91010 +91011 POINT(41.5145495475209 73.11919923073995) bank91011 +91012 POINT(41.14285035030236 74.97542016361328) bank91012 +91013 POINT(41.3255577193681 74.40594337020129) bank91013 +91014 POINT(40.671103722666444 73.07040348491475) bank91014 +91015 POINT(40.900396720476806 73.95428020955029) bank91015 +91016 POINT(40.35677779482441 73.44893097936703) bank91016 +91017 POINT(40.16291426103911 74.9708019208155) bank91017 +91018 POINT(39.773807006578664 74.01108421606445) bank91018 +91019 POINT(41.70350150105835 74.87605230552197) bank91019 +91020 POINT(41.688676148775535 74.71615889735118) bank91020 +91021 POINT(40.68485872774595 73.0625472095941) bank91021 +91022 POINT(41.14433717243987 74.10529548731672) bank91022 +91023 POINT(40.02358824719234 74.76430278043941) bank91023 +91024 POINT(40.53652774176607 73.84859315589763) bank91024 +91025 POINT(40.58034313319435 74.37206672105266) bank91025 +91026 POINT(40.32832559999512 73.04788895563833) bank91026 +91027 POINT(41.169297444474175 73.64168902149012) bank91027 +91028 POINT(41.51958907074829 73.68145997827467) bank91028 +91029 POINT(40.06491108896064 73.95691419312054) bank91029 +91030 POINT(39.91939705548565 74.04602479665387) bank91030 +91031 POINT(41.35350795560966 74.69695597710349) bank91031 +91032 POINT(40.548499350725955 73.57698335859278) bank91032 +91033 POINT(40.36919875333765 73.64206764857498) bank91033 +91034 POINT(39.84331209632842 73.03599272227315) bank91034 +91035 POINT(41.488990522157145 73.91248125516087) bank91035 +91036 POINT(41.39704921955143 73.06819460623635) bank91036 +91037 POINT(40.84826160999613 74.96557432368158) bank91037 +91038 POINT(40.26130893289501 73.89691886998972) bank91038 +91039 POINT(40.25986924391848 73.51206794028708) bank91039 +91040 POINT(40.52153779847938 74.05894075928315) bank91040 +91041 POINT(40.30300880355097 73.66462698455673) bank91041 +91042 POINT(40.425028748748765 74.88522409756823) bank91042 +91043 POINT(39.77128361167314 74.13987148946008) bank91043 +91044 POINT(41.378616411004046 73.47574173892738) bank91044 +91045 POINT(40.7189557607372 74.77565270312878) bank91045 +91046 POINT(39.934578042183794 74.84481328746165) bank91046 +91047 POINT(41.19404438510481 74.4054837783798) bank91047 +91048 POINT(41.480747175703584 73.60612934023185) bank91048 +91049 POINT(40.13602946260663 73.64064471024734) bank91049 +91050 POINT(41.289401291246136 74.47376624085761) bank91050 +91051 POINT(41.02111322933822 73.03119826049861) bank91051 +91052 POINT(40.951986452836046 73.25044054366253) bank91052 +91053 POINT(39.83795732795963 74.77991922339814) bank91053 +91054 POINT(39.807660802447224 74.11341743380245) bank91054 +91055 POINT(40.85477203026262 73.08413278220569) bank91055 +91056 POINT(39.82330793900454 73.33329988616715) bank91056 +91057 POINT(39.87300967901876 73.18199780360243) bank91057 +91058 POINT(41.42573193952401 74.15956873194568) bank91058 +91059 POINT(40.41682903483741 73.28220053887031) bank91059 +91060 POINT(40.764372424850535 73.81795213450938) bank91060 +91061 POINT(40.01782901979644 73.40348686429091) bank91061 +91062 POINT(40.932208222265295 73.3848444057519) bank91062 +91063 POINT(41.19783855542141 74.70570910462769) bank91063 +91064 POINT(39.9048308880259 74.1751183589853) bank91064 +91065 POINT(41.65951477761118 73.42957252382818) bank91065 +91066 POINT(39.852854523296045 74.17489079142888) bank91066 +91067 POINT(40.59289985733866 73.99337310698192) bank91067 +91068 POINT(40.25601453338794 74.7004899426892) bank91068 +91069 POINT(40.72348166214551 73.06511718610585) bank91069 +91070 POINT(39.80524290428331 73.98073087404623) bank91070 +91071 POINT(40.45685756040243 73.48011251396744) bank91071 +91072 POINT(41.05010594264866 74.02789068956187) bank91072 +91073 POINT(40.969168292797455 74.04269004346563) bank91073 +91074 POINT(41.562773823212034 73.29274768452788) bank91074 +91075 POINT(41.63207585449664 74.25171269831984) bank91075 +91076 POINT(40.173426260102545 74.63929502577201) bank91076 +91077 POINT(41.088082405188146 74.07185242964563) bank91077 +91078 POINT(40.41145207766499 74.77134986524972) bank91078 +91079 POINT(41.09977907980259 73.65744357823887) bank91079 +91080 POINT(41.03128297229837 73.40183627825928) bank91080 +91081 POINT(41.4710707318054 73.48100172580426) bank91081 +91082 POINT(41.23839736374997 74.23227572193947) bank91082 +91083 POINT(40.77278218506476 74.382610285655) bank91083 +91084 POINT(39.9564181769116 73.75749748755702) bank91084 +91085 POINT(39.75988940745779 74.38807807934712) bank91085 +91086 POINT(40.281663805304795 74.02260764713525) bank91086 +91087 POINT(40.20986952057491 74.55658924768036) bank91087 +91088 POINT(41.21979855663183 73.99360575355159) bank91088 +91089 POINT(40.86088712162455 73.67364944031068) bank91089 +91090 POINT(41.690015779768444 74.45842035541979) bank91090 +91091 POINT(41.59661859709574 74.79415473018287) bank91091 +91092 POINT(40.043290600336434 73.33638970227207) bank91092 +91093 POINT(40.96787604783523 73.00709126909716) bank91093 +91094 POINT(40.77344109363067 73.1512990299838) bank91094 +91095 POINT(40.67132804602162 73.9197749641731) bank91095 +91096 POINT(41.128162128059095 74.12838029825107) bank91096 +91097 POINT(40.220171800662094 74.75066044332178) bank91097 +91098 POINT(41.32626919624626 74.63851174998284) bank91098 +91099 POINT(41.25626507745237 73.85233119654617) bank91099 +91100 POINT(39.848048916588226 74.47895637378579) bank91100 +91101 POINT(41.41360605609247 74.1273389485919) bank91101 +91102 POINT(41.133892499874854 74.18298266406946) bank91102 +91103 POINT(40.75700440319686 73.70008906713305) bank91103 +91104 POINT(40.626004839877126 73.20033404313665) bank91104 +91105 POINT(40.383305999535075 74.34106684232246) bank91105 +91106 POINT(40.50998506504183 74.44611557414528) bank91106 +91107 POINT(41.622581147603896 73.90129439887546) bank91107 +91108 POINT(41.59748072877437 74.9887355048987) bank91108 +91109 POINT(41.648577570742084 74.37112514100185) bank91109 +91110 POINT(41.51919279055348 75.00317433032157) bank91110 +91111 POINT(41.24087186170846 73.58619774293554) bank91111 +91112 POINT(39.94447322848796 74.77706270409891) bank91112 +91113 POINT(41.23504380759428 73.10678073151782) bank91113 +91114 POINT(40.227837353020405 73.40201396875477) bank91114 +91115 POINT(41.580129356849866 74.62506078001005) bank91115 +91116 POINT(40.05190555571188 73.25605825129914) bank91116 +91117 POINT(41.6218339058216 73.08784026683999) bank91117 +91118 POINT(39.99705622089464 73.19898888477232) bank91118 +91119 POINT(39.99926359599722 73.84425578555728) bank91119 +91120 POINT(39.828421949312514 74.46486092565732) bank91120 +91121 POINT(39.91805168057483 74.85976172821047) bank91121 +91122 POINT(41.54342309667953 73.99510925192155) bank91122 +91123 POINT(40.46725342925666 73.67414653455697) bank91123 +91124 POINT(40.19028986562619 74.24677859277011) bank91124 +91125 POINT(40.85618508408235 73.38439069379093) bank91125 +91126 POINT(40.81275667341686 73.47728342774498) bank91126 +91127 POINT(40.69916655527109 74.44632849813914) bank91127 +91128 POINT(39.984984422108816 74.75552100848391) bank91128 +91129 POINT(41.64287019359057 73.75086735321936) bank91129 +91130 POINT(39.821498892249686 73.71256042111905) bank91130 +91131 POINT(40.97834484186298 74.00041616913056) bank91131 +91132 POINT(40.92578438040502 74.99866711288817) bank91132 +91133 POINT(41.37656841149352 74.39639217805211) bank91133 +91134 POINT(40.7823464429698 74.37462431895626) bank91134 +91135 POINT(40.43757905621532 74.10699406370803) bank91135 +91136 POINT(41.70072729056897 74.55364834206021) bank91136 +91137 POINT(40.21563573310984 74.18175500820533) bank91137 +91138 POINT(41.66195632169671 74.75039946105875) bank91138 +91139 POINT(40.195862796875886 74.3273422285442) bank91139 +91140 POINT(41.685891150078334 74.44355125384725) bank91140 +91141 POINT(40.78298777429859 74.42785032819832) bank91141 +91142 POINT(40.557820075797096 74.28441054144552) bank91142 +91143 POINT(40.422939060196 74.70445386614558) bank91143 +91144 POINT(40.59582189923852 74.408851910125) bank91144 +91145 POINT(40.74568424293378 74.46887619079618) bank91145 +91146 POINT(41.11925330748698 74.69273804164642) bank91146 +91147 POINT(40.68380226915445 73.51878354196631) bank91147 +91148 POINT(40.08531304233158 74.56481801762152) bank91148 +91149 POINT(40.60359954710041 74.56982512005558) bank91149 +91150 POINT(39.93702282691735 73.69444039111562) bank91150 +91151 POINT(40.464263468666594 74.65414912204366) bank91151 +91152 POINT(40.57086715777076 74.23233000390316) bank91152 +91153 POINT(40.89298778114014 73.6050161659392) bank91153 +91154 POINT(40.06112218216614 74.48450580176983) bank91154 +91155 POINT(40.4827490992748 73.63093328347563) bank91155 +91156 POINT(40.596488291831484 74.68414358378554) bank91156 +91157 POINT(41.47773314170872 74.19758311854088) bank91157 +91158 POINT(40.05676571114967 74.67015609862098) bank91158 +91159 POINT(40.69650676645544 73.96726438339198) bank91159 +91160 POINT(41.33499847025403 74.53537578113267) bank91160 +91161 POINT(41.26268638378124 73.44301076384666) bank91161 +91162 POINT(40.85655660535409 74.39507671310461) bank91162 +91163 POINT(40.45142319033686 73.45278349718228) bank91163 +91164 POINT(41.315797248713494 73.41643370618803) bank91164 +91165 POINT(41.04146024853912 74.47121036472701) bank91165 +91166 POINT(40.91281507176079 73.54831839578125) bank91166 +91167 POINT(41.58273962200658 74.2972045091355) bank91167 +91168 POINT(41.491388254808214 74.88137405314157) bank91168 +91169 POINT(40.37975626859209 73.02902731336863) bank91169 +91170 POINT(40.112693996353265 73.13118336077922) bank91170 +91171 POINT(39.96823541108987 74.9341848375501) bank91171 +91172 POINT(40.25395994664756 73.28398456823703) bank91172 +91173 POINT(39.918935916651414 74.30879931575372) bank91173 +91174 POINT(41.31175508978314 73.13658173336654) bank91174 +91175 POINT(40.73897280317461 73.9162457922535) bank91175 +91176 POINT(40.89867023803591 73.65658419014498) bank91176 +91177 POINT(40.65735609582491 73.7522470076779) bank91177 +91178 POINT(39.94089944952803 74.34264685232323) bank91178 +91179 POINT(41.160457839461735 74.88366315846436) bank91179 +91180 POINT(40.06510653931655 74.42683981502569) bank91180 +91181 POINT(40.02201040399863 74.29386860695213) bank91181 +91182 POINT(39.91136930983479 74.84746112361988) bank91182 +91183 POINT(40.25715798295643 73.82764099548532) bank91183 +91184 POINT(40.87714029540659 74.88989642881566) bank91184 +91185 POINT(41.528149669793535 74.48695777439109) bank91185 +91186 POINT(40.743077165725126 73.65442129049117) bank91186 +91187 POINT(41.000669703194205 74.42533933015014) bank91187 +91188 POINT(40.071478531311485 73.95732662027154) bank91188 +91189 POINT(40.95981008937499 74.58871442309815) bank91189 +91190 POINT(41.269643261251424 73.08168895023037) bank91190 +91191 POINT(40.70182777633223 74.52441639226305) bank91191 +91192 POINT(40.146783225581466 74.59785467689073) bank91192 +91193 POINT(40.28208564217626 73.88124793099632) bank91193 +91194 POINT(41.38229186735369 74.1793021391017) bank91194 +91195 POINT(41.27635956625574 73.17007530697008) bank91195 +91196 POINT(40.66856562702076 74.38422347255556) bank91196 +91197 POINT(39.95514505797597 73.87944551343438) bank91197 +91198 POINT(41.3149469173943 73.53375344451794) bank91198 +91199 POINT(40.14564643732864 74.63360352988558) bank91199 +91200 POINT(40.3888970192791 74.55484870957885) bank91200 +91201 POINT(41.24160293306326 73.37539094105352) bank91201 +91202 POINT(40.34919353204165 73.07125036145962) bank91202 +91203 POINT(41.495601205998526 74.28875435817575) bank91203 +91204 POINT(40.45163210005655 74.36914073771523) bank91204 +91205 POINT(41.420012945963414 74.15625828363841) bank91205 +91206 POINT(40.95845745425658 73.9414531487282) bank91206 +91207 POINT(40.098822146457614 74.85217640875166) bank91207 +91208 POINT(40.98055813627761 73.49773519875032) bank91208 +91209 POINT(39.77527078166115 73.15361802789013) bank91209 +91210 POINT(40.66268434891179 74.41291519866512) bank91210 +91211 POINT(40.7560842845769 73.3708411381473) bank91211 +91212 POINT(41.26826106654089 73.1367088727878) bank91212 +91213 POINT(40.00128467032899 74.11600876050571) bank91213 +91214 POINT(40.04553104120341 74.92690005255764) bank91214 +91215 POINT(41.51204492585708 74.30905953313821) bank91215 +91216 POINT(41.10512800310197 74.79253270445285) bank91216 +91217 POINT(40.259947693270675 74.40771758707847) bank91217 +91218 POINT(40.19120151931416 74.47767030056217) bank91218 +91219 POINT(40.784015420146474 73.7515457569564) bank91219 +91220 POINT(39.848257201858345 73.8763439527936) bank91220 +91221 POINT(41.634919172709886 73.535432816765) bank91221 +91222 POINT(41.54498257180931 74.06145270895058) bank91222 +91223 POINT(41.04550630891084 73.5079004550638) bank91223 +91224 POINT(41.61300869942029 74.80762024478157) bank91224 +91225 POINT(40.946739094034356 74.94364331562083) bank91225 +91226 POINT(40.10257167813404 73.75468556358248) bank91226 +91227 POINT(39.885163149058954 73.15042917529544) bank91227 +91228 POINT(41.585424127471974 74.9656854654441) bank91228 +91229 POINT(41.554347283203555 74.25652742771885) bank91229 +91230 POINT(39.85603566284611 74.06572559532997) bank91230 +91231 POINT(40.66199738975185 74.64609165897231) bank91231 +91232 POINT(40.678735356973576 74.8217706567245) bank91232 +91233 POINT(41.57312898877209 73.13097171184958) bank91233 +91234 POINT(41.07789230529863 74.74395823382707) bank91234 +91235 POINT(40.14549326053736 74.74415240826595) bank91235 +91236 POINT(41.24995916749236 73.72195920062994) bank91236 +91237 POINT(41.275903938261465 73.27304571919784) bank91237 +91238 POINT(41.24100101231988 73.45996564018228) bank91238 +91239 POINT(41.27837659313074 73.46839953814897) bank91239 +91240 POINT(39.85995509326412 73.05584898552567) bank91240 +91241 POINT(41.67882303111075 73.26334345912075) bank91241 +91242 POINT(39.974889974678604 74.41815034838753) bank91242 +91243 POINT(40.98767986894965 74.87453487480752) bank91243 +91244 POINT(41.44782405285922 73.26756269096299) bank91244 +91245 POINT(41.51842115833573 73.17162058634149) bank91245 +91246 POINT(40.67004103576829 73.5693244677796) bank91246 +91247 POINT(41.458744250890945 74.6114972386472) bank91247 +91248 POINT(40.73680938901606 73.02189600513171) bank91248 +91249 POINT(41.263096025636905 74.55715372099779) bank91249 +91250 POINT(40.147106376313864 74.16737696019752) bank91250 +91251 POINT(40.21791599887101 74.04270786376536) bank91251 +91252 POINT(41.03143711223791 73.54731867971142) bank91252 +91253 POINT(41.26521720637657 74.90107113253683) bank91253 +91254 POINT(41.3513319165958 73.96810876538062) bank91254 +91255 POINT(40.71847455337506 74.18937831702469) bank91255 +91256 POINT(40.72558439182321 73.78849896046079) bank91256 +91257 POINT(40.86716495746596 73.93202563994254) bank91257 +91258 POINT(40.22004267637925 73.47494975723458) bank91258 +91259 POINT(40.15762812605868 73.02152834578317) bank91259 +91260 POINT(41.636977171880666 73.72222811615991) bank91260 +91261 POINT(40.60294999359826 73.08183853280133) bank91261 +91262 POINT(39.722693045221575 74.34635680814927) bank91262 +91263 POINT(40.53686986313547 73.38601665102138) bank91263 +91264 POINT(40.03081874076265 73.2136523363789) bank91264 +91265 POINT(41.680783122898475 74.81754352462457) bank91265 +91266 POINT(40.94364903611129 73.81771259460069) bank91266 +91267 POINT(40.71731134005072 74.15814025202639) bank91267 +91268 POINT(41.456168929929134 74.19977356189011) bank91268 +91269 POINT(40.43200727943021 73.70854118594937) bank91269 +91270 POINT(40.09527053342461 74.22079765557919) bank91270 +91271 POINT(40.21274401665849 74.31890687088628) bank91271 +91272 POINT(40.09850749767531 73.58961369972629) bank91272 +91273 POINT(41.48475403486137 74.17803994658914) bank91273 +91274 POINT(40.52275432558705 73.50447497415195) bank91274 +91275 POINT(40.436926692898595 74.5329655731172) bank91275 +91276 POINT(40.633951586676595 73.61539025614536) bank91276 +91277 POINT(40.10154142419258 74.46900784600487) bank91277 +91278 POINT(39.8575652765119 74.50005957023399) bank91278 +91279 POINT(40.07565196629643 73.36827096380614) bank91279 +91280 POINT(40.24424448747451 73.92055477444029) bank91280 +91281 POINT(40.48642605988893 73.93251281037955) bank91281 +91282 POINT(41.21621871114547 73.97439088882437) bank91282 +91283 POINT(41.49568755985275 73.94377209795016) bank91283 +91284 POINT(39.86498434061525 74.13595127319297) bank91284 +91285 POINT(39.94331202540076 73.48989276773095) bank91285 +91286 POINT(40.15283479942176 73.05472933354778) bank91286 +91287 POINT(40.95182834625379 74.26274999123565) bank91287 +91288 POINT(40.38614721231562 73.42650862832183) bank91288 +91289 POINT(40.71967193970702 74.72024052393641) bank91289 +91290 POINT(40.100438867686535 74.44882003276629) bank91290 +91291 POINT(40.65972596825142 74.85413244045765) bank91291 +91292 POINT(40.70495908332813 73.72991817084116) bank91292 +91293 POINT(41.481717658092286 73.93040391018249) bank91293 +91294 POINT(40.58548066593933 74.48000602938363) bank91294 +91295 POINT(41.228208041379716 73.25966757499468) bank91295 +91296 POINT(41.44542847745589 74.50188555260507) bank91296 +91297 POINT(40.50948315481696 74.57681894465118) bank91297 +91298 POINT(41.176775669657 74.42982224461379) bank91298 +91299 POINT(41.63424733142809 74.83708294501655) bank91299 +91300 POINT(40.990987074254136 73.37808485846156) bank91300 +91301 POINT(40.99342438896954 74.18216251531265) bank91301 +91302 POINT(40.79059460554023 74.64384891298698) bank91302 +91303 POINT(41.34349032477472 73.3729626482646) bank91303 +91304 POINT(41.14100928449088 74.49171272721884) bank91304 +91305 POINT(41.496993924874864 73.4187238267955) bank91305 +91306 POINT(40.128436858866145 74.16300586274821) bank91306 +91307 POINT(40.47182030359306 74.80046626670213) bank91307 +91308 POINT(40.158915965002684 73.45554841437712) bank91308 +91309 POINT(41.11461051893264 75.00353217564167) bank91309 +91310 POINT(39.81999577666932 74.59927183836557) bank91310 +91311 POINT(39.790969682140044 73.56818486039947) bank91311 +91312 POINT(39.81764107001986 74.57820350910312) bank91312 +91313 POINT(40.846169527737054 74.88048282303797) bank91313 +91314 POINT(40.909028159955895 73.209735626437) bank91314 +91315 POINT(40.36249534893244 74.21018788516217) bank91315 +91316 POINT(40.7975100371568 74.42295312604129) bank91316 +91317 POINT(41.36554494066725 73.4119779950843) bank91317 +91318 POINT(41.71177172798422 74.68304263508084) bank91318 +91319 POINT(39.7745688410702 73.72858920014465) bank91319 +91320 POINT(40.45201101377045 73.28901492139062) bank91320 +91321 POINT(40.531409419986424 73.32940877823252) bank91321 +91322 POINT(39.74915117783452 73.90296147721224) bank91322 +91323 POINT(41.462437279704645 74.27142780573934) bank91323 +91324 POINT(40.36946550108928 74.82617162020549) bank91324 +91325 POINT(40.82144857960803 74.67495859803383) bank91325 +91326 POINT(41.109219896679626 73.65964986150584) bank91326 +91327 POINT(41.13459889560172 73.24998168575543) bank91327 +91328 POINT(40.40910736777452 73.76140360975194) bank91328 +91329 POINT(41.710162738116516 73.09794931768641) bank91329 +91330 POINT(41.32966446293838 73.3423156544574) bank91330 +91331 POINT(40.136612384016324 73.27127073063929) bank91331 +91332 POINT(41.39980351589742 73.08050824159291) bank91332 +91333 POINT(41.0982007560737 73.60152848923771) bank91333 +91334 POINT(40.538400180738655 74.19466597801616) bank91334 +91335 POINT(40.62354964340329 74.06181240905875) bank91335 +91336 POINT(40.64770478067533 74.0727919738685) bank91336 +91337 POINT(41.3152321525849 73.47972502455833) bank91337 +91338 POINT(41.04004116777335 73.9874740423108) bank91338 +91339 POINT(40.683848686872764 74.81825803162178) bank91339 +91340 POINT(41.37230939669005 73.81022560164463) bank91340 +91341 POINT(41.05545793006975 74.27882012935439) bank91341 +91342 POINT(39.74033369196983 73.76685254249352) bank91342 +91343 POINT(40.578139909425474 73.25583224183035) bank91343 +91344 POINT(39.75586053364405 74.39480426352249) bank91344 +91345 POINT(40.013615089754865 74.3470980968903) bank91345 +91346 POINT(40.947239168082824 73.11935531504153) bank91346 +91347 POINT(41.35016925731345 73.38820864812243) bank91347 +91348 POINT(41.02910931355396 74.14864326360849) bank91348 +91349 POINT(39.95243708592151 73.7213436665673) bank91349 +91350 POINT(39.83483021578551 74.36696358334872) bank91350 +91351 POINT(40.151983996503056 73.48326709235991) bank91351 +91352 POINT(40.41762354785149 73.14851313092684) bank91352 +91353 POINT(40.67593569108206 74.41694557795107) bank91353 +91354 POINT(40.54491471223872 74.08336154890685) bank91354 +91355 POINT(40.2517409599397 73.3727949904599) bank91355 +91356 POINT(41.67291399059623 73.63396929724382) bank91356 +91357 POINT(40.77724078863061 73.92431888253948) bank91357 +91358 POINT(40.67232250124649 73.09680227746856) bank91358 +91359 POINT(40.93424123660965 74.05122387588167) bank91359 +91360 POINT(41.64481364757008 74.92374821493067) bank91360 +91361 POINT(40.19160091394534 73.1589118718929) bank91361 +91362 POINT(40.04001027237051 74.04912713508666) bank91362 +91363 POINT(40.49251646718502 74.71211106437433) bank91363 +91364 POINT(41.34562586138591 73.98638873006051) bank91364 +91365 POINT(41.03567246005789 74.50514614501603) bank91365 +91366 POINT(39.73085257953326 73.8607591959088) bank91366 +91367 POINT(41.612013790647595 74.82383056382393) bank91367 +91368 POINT(41.18740811002624 74.64030171516403) bank91368 +91369 POINT(39.857134578356664 74.74421979005967) bank91369 +91370 POINT(40.975506724077164 73.56476701699927) bank91370 +91371 POINT(40.785069356234 73.61328255329204) bank91371 +91372 POINT(41.69884403168795 73.49848157153477) bank91372 +91373 POINT(40.51492196914673 73.37407084582681) bank91373 +91374 POINT(41.51184942294166 74.9220597320555) bank91374 +91375 POINT(41.651845549047145 73.83182692351575) bank91375 +91376 POINT(40.02373757458028 73.8245465955837) bank91376 +91377 POINT(41.090704935010535 74.8840502273764) bank91377 +91378 POINT(41.33091815227111 74.29507845417484) bank91378 +91379 POINT(41.55016291941126 73.37522681389149) bank91379 +91380 POINT(41.097726940369 74.08840800780376) bank91380 +91381 POINT(39.83406688383024 74.0516006794488) bank91381 +91382 POINT(40.848729782289816 74.52021562499846) bank91382 +91383 POINT(39.97862106771113 73.93500053503301) bank91383 +91384 POINT(40.83921011435112 74.94112503286013) bank91384 +91385 POINT(41.553987237289114 74.80985621557862) bank91385 +91386 POINT(41.653364139401205 74.06163375517798) bank91386 +91387 POINT(40.32790404831137 74.2495839605391) bank91387 +91388 POINT(40.51733351452051 73.19781499055034) bank91388 +91389 POINT(40.1562462072028 74.83699192082345) bank91389 +91390 POINT(40.152841853563665 74.5174299538661) bank91390 +91391 POINT(40.32143522191725 73.07461851088878) bank91391 +91392 POINT(40.09755384305204 73.41487201631384) bank91392 +91393 POINT(41.14333387559427 73.01008029406043) bank91393 +91394 POINT(40.13827838442525 74.04453011189139) bank91394 +91395 POINT(40.779938372681066 74.74634545384012) bank91395 +91396 POINT(41.02122919036773 73.87353169266798) bank91396 +91397 POINT(41.53589411483101 74.54890800520336) bank91397 +91398 POINT(39.77713170882461 74.7824480967714) bank91398 +91399 POINT(40.70573532242249 73.39276729795898) bank91399 +91400 POINT(41.448862460976855 73.09047965266649) bank91400 +91401 POINT(40.87483538453888 73.76993621972191) bank91401 +91402 POINT(40.26362192721159 74.39536074572831) bank91402 +91403 POINT(41.462502230797405 73.16445536299601) bank91403 +91404 POINT(41.085664182002276 73.85887060102658) bank91404 +91405 POINT(40.656566451298794 74.29634126063749) bank91405 +91406 POINT(40.59830831003705 73.02549612579091) bank91406 +91407 POINT(40.557049828658 74.70647914314793) bank91407 +91408 POINT(40.81365719470396 74.12559283902196) bank91408 +91409 POINT(40.89627484829566 73.14624334332403) bank91409 +91410 POINT(41.20560219342579 73.28704786659469) bank91410 +91411 POINT(41.13265794357492 74.03311266200984) bank91411 +91412 POINT(40.048546809989894 73.0448812329694) bank91412 +91413 POINT(41.05955472016424 73.49501383678741) bank91413 +91414 POINT(40.66250395082937 74.35320321158517) bank91414 +91415 POINT(40.72297128123306 73.72693198952163) bank91415 +91416 POINT(40.945173099559575 74.3235106068214) bank91416 +91417 POINT(40.32649727833104 73.78137339841489) bank91417 +91418 POINT(39.75810562078788 74.9533121704796) bank91418 +91419 POINT(40.05521391209045 73.17796742390593) bank91419 +91420 POINT(40.79539180281605 74.30941375781376) bank91420 +91421 POINT(39.72881978698123 74.51082442005993) bank91421 +91422 POINT(41.26084697928841 74.63059358010278) bank91422 +91423 POINT(40.31356760611784 73.07700736286779) bank91423 +91424 POINT(40.95824642554059 74.12140017172172) bank91424 +91425 POINT(40.779522637191924 73.87439486863086) bank91425 +91426 POINT(41.38521068012314 73.71857201850878) bank91426 +91427 POINT(40.15051000324672 73.26367726093221) bank91427 +91428 POINT(41.04320804753765 73.00769321331616) bank91428 +91429 POINT(39.86090488174337 74.62623806169606) bank91429 +91430 POINT(39.80496929645349 73.20471314259557) bank91430 +91431 POINT(41.06898046859483 74.44435188592097) bank91431 +91432 POINT(41.612768208612025 74.36437013130102) bank91432 +91433 POINT(40.20217287232198 74.63007511493136) bank91433 +91434 POINT(40.884895372752645 74.18605192268515) bank91434 +91435 POINT(40.65466500360815 74.33051801471382) bank91435 +91436 POINT(39.894609419626185 73.83379662797222) bank91436 +91437 POINT(41.079807463977666 74.17310472762509) bank91437 +91438 POINT(39.94818423655524 73.73639334102656) bank91438 +91439 POINT(41.622799565004776 73.99013706325522) bank91439 +91440 POINT(40.92033117215085 74.21299049818685) bank91440 +91441 POINT(41.132158215392714 74.22813977639734) bank91441 +91442 POINT(39.96823605293943 73.61597627010705) bank91442 +91443 POINT(41.37509531341542 74.07715040950805) bank91443 +91444 POINT(41.290573656002174 74.2249647413933) bank91444 +91445 POINT(41.47376393211473 74.21547529327253) bank91445 +91446 POINT(40.89557931014851 73.86556931124773) bank91446 +91447 POINT(40.20288269689265 73.69090064288874) bank91447 +91448 POINT(41.01425655093357 74.56843534822659) bank91448 +91449 POINT(41.60551805094586 73.08151826952772) bank91449 +91450 POINT(40.99756572628892 73.90405678127905) bank91450 +91451 POINT(41.244059891194404 73.35134098384721) bank91451 +91452 POINT(40.923376309279746 73.86365060728887) bank91452 +91453 POINT(40.05552500852214 73.17190983351941) bank91453 +91454 POINT(41.331753204006304 73.97547297556483) bank91454 +91455 POINT(40.07685230507688 73.72518605684313) bank91455 +91456 POINT(40.599143907633504 74.31351426035013) bank91456 +91457 POINT(40.66218353555124 73.9838790547725) bank91457 +91458 POINT(40.96265564114649 74.29007538285951) bank91458 +91459 POINT(40.44737638261678 74.16455034718317) bank91459 +91460 POINT(41.010487447234574 74.66458881800227) bank91460 +91461 POINT(41.17237513562725 74.44435753754136) bank91461 +91462 POINT(40.008115902328 73.35241448500753) bank91462 +91463 POINT(41.22486800834738 74.62531073943327) bank91463 +91464 POINT(40.67268959522646 74.41556350862636) bank91464 +91465 POINT(41.68506210317855 73.56051249089684) bank91465 +91466 POINT(40.762749716719824 73.79429146816875) bank91466 +91467 POINT(40.13338052987299 74.19271885148686) bank91467 +91468 POINT(39.79488006033907 73.71497062636732) bank91468 +91469 POINT(41.381928069382056 74.78165270331552) bank91469 +91470 POINT(39.956631048588434 73.74447848469863) bank91470 +91471 POINT(41.10605342571473 74.67306202641515) bank91471 +91472 POINT(41.1957398477349 73.30625967196397) bank91472 +91473 POINT(40.974367254504024 74.30019989099553) bank91473 +91474 POINT(40.47355662005042 73.03498745224023) bank91474 +91475 POINT(40.23758880060668 74.82664602175683) bank91475 +91476 POINT(39.90433039033496 73.87949188825856) bank91476 +91477 POINT(41.483583653755744 73.00702352855681) bank91477 +91478 POINT(41.254178590597554 74.74536251361663) bank91478 +91479 POINT(41.59190516131479 74.16301331814239) bank91479 +91480 POINT(40.06904522702539 73.72124747355114) bank91480 +91481 POINT(40.959344927929955 74.13174133967154) bank91481 +91482 POINT(40.381339531962304 74.41640749407846) bank91482 +91483 POINT(40.918016462506245 73.14395502600605) bank91483 +91484 POINT(40.6867766985049 73.91993633946454) bank91484 +91485 POINT(40.28527065685394 74.90834491957793) bank91485 +91486 POINT(41.09165995023158 73.01691256743679) bank91486 +91487 POINT(41.39137077408086 73.25605459349846) bank91487 +91488 POINT(40.74333008240143 74.3501838685594) bank91488 +91489 POINT(41.094829278586026 73.79119521111993) bank91489 +91490 POINT(41.028835239248174 73.77887686781673) bank91490 +91491 POINT(40.41138657225635 73.06426730017458) bank91491 +91492 POINT(39.740971679237866 73.75170470930613) bank91492 +91493 POINT(40.122032889876294 74.64096148472497) bank91493 +91494 POINT(40.98145324378631 73.43558436712793) bank91494 +91495 POINT(39.8278658624062 73.962940266601) bank91495 +91496 POINT(41.00579160590956 73.5126278072697) bank91496 +91497 POINT(41.20433222978086 73.6172167715535) bank91497 +91498 POINT(40.34136646260743 73.20908059840603) bank91498 +91499 POINT(39.84038938188917 74.06697142041439) bank91499 +91500 POINT(39.97769873556433 73.89018308904397) bank91500 +91501 POINT(40.421021068748864 74.83630415847796) bank91501 +91502 POINT(41.19960238205862 74.82702089062964) bank91502 +91503 POINT(40.20877493191338 73.47591671633931) bank91503 +91504 POINT(40.42003746971362 73.79081303437778) bank91504 +91505 POINT(40.55849992332724 74.25760259308035) bank91505 +91506 POINT(40.746857320799144 74.17075126522128) bank91506 +91507 POINT(40.96590072057217 74.38810595412382) bank91507 +91508 POINT(40.37291070854761 74.4172833263424) bank91508 +91509 POINT(41.49423173293 73.47586154423595) bank91509 +91510 POINT(41.58110528568461 73.09311738979754) bank91510 +91511 POINT(39.98088151118372 74.98243008280299) bank91511 +91512 POINT(40.17042533283435 74.13081269874645) bank91512 +91513 POINT(40.038050831333756 74.2322966847123) bank91513 +91514 POINT(41.711613080277324 73.12695142259143) bank91514 +91515 POINT(41.32533679322967 74.40739667962575) bank91515 +91516 POINT(40.73059941412827 73.24292908936773) bank91516 +91517 POINT(41.29077531816783 74.34554294195054) bank91517 +91518 POINT(40.79059975195902 74.48336994801912) bank91518 +91519 POINT(39.896062806303554 74.43509266376762) bank91519 +91520 POINT(40.60849280257449 73.44642129514321) bank91520 +91521 POINT(41.33194317823318 74.80012350044454) bank91521 +91522 POINT(40.72065550452431 73.21834588234525) bank91522 +91523 POINT(40.711661180054946 73.04157699235593) bank91523 +91524 POINT(40.710213592100175 73.81001142708239) bank91524 +91525 POINT(40.59154281338416 74.38667095197259) bank91525 +91526 POINT(41.28781794213718 74.9210627085076) bank91526 +91527 POINT(40.41990387312667 74.61100321679795) bank91527 +91528 POINT(40.68762590223557 74.51684894704096) bank91528 +91529 POINT(40.50887386778847 74.61095701958305) bank91529 +91530 POINT(40.9931374374628 74.4605651450049) bank91530 +91531 POINT(41.548928822998434 73.37495607823726) bank91531 +91532 POINT(40.02710480766332 73.38779746039731) bank91532 +91533 POINT(41.526137027156224 74.3312410547387) bank91533 +91534 POINT(41.06283182728427 74.80158811348757) bank91534 +91535 POINT(39.78531436131623 73.18245388024872) bank91535 +91536 POINT(40.918065918177255 74.7476273490967) bank91536 +91537 POINT(39.981177186332715 74.65642906640208) bank91537 +91538 POINT(41.287035560740605 73.14822834838513) bank91538 +91539 POINT(40.71733962952325 74.21646770829491) bank91539 +91540 POINT(41.38915417728561 73.6527921077667) bank91540 +91541 POINT(39.791535611051096 73.21426887013637) bank91541 +91542 POINT(40.96333756304593 73.39214023097692) bank91542 +91543 POINT(41.53636611814725 74.3553166240217) bank91543 +91544 POINT(40.34699968502409 74.26827715122145) bank91544 +91545 POINT(39.82375673778517 74.11023237490707) bank91545 +91546 POINT(40.143622003870085 73.27651592876826) bank91546 +91547 POINT(39.797971240060875 73.48740371469295) bank91547 +91548 POINT(40.523514540331824 73.78707154153959) bank91548 +91549 POINT(40.67275586616958 73.331762791833) bank91549 +91550 POINT(40.1729707135638 74.2885511335838) bank91550 +91551 POINT(39.98489930831636 74.94393640337894) bank91551 +91552 POINT(40.79266098823982 74.6314014180621) bank91552 +91553 POINT(40.603296374864435 74.20241859560073) bank91553 +91554 POINT(40.688640842054134 73.79878295563778) bank91554 +91555 POINT(40.03654443595627 74.01149593548416) bank91555 +91556 POINT(40.79500448511723 74.23752834311281) bank91556 +91557 POINT(41.46496580541607 73.98093544218638) bank91557 +91558 POINT(39.778524604336724 73.02847558201275) bank91558 +91559 POINT(40.12139339057942 74.81291931684079) bank91559 +91560 POINT(40.40078534075917 73.7488770441515) bank91560 +91561 POINT(40.230479085199676 73.80585170293621) bank91561 +91562 POINT(41.442543136390285 73.72838526977836) bank91562 +91563 POINT(40.26603736441911 74.37803897421975) bank91563 +91564 POINT(40.838320671282986 74.76878647170717) bank91564 +91565 POINT(41.360985578061374 74.00063120544935) bank91565 +91566 POINT(41.07982186910656 73.8688496860004) bank91566 +91567 POINT(41.4532955772613 73.51481563236031) bank91567 +91568 POINT(40.84065205473519 73.42128447530952) bank91568 +91569 POINT(39.88888009730131 74.21320037838352) bank91569 +91570 POINT(39.83180759048311 73.53937770537692) bank91570 +91571 POINT(39.7652230669216 74.68917489537203) bank91571 +91572 POINT(41.49259520192684 73.35829819430572) bank91572 +91573 POINT(40.98832493313791 74.56320688159123) bank91573 +91574 POINT(41.395281493188335 74.74922813200264) bank91574 +91575 POINT(40.058777653835726 75.00352768266399) bank91575 +91576 POINT(40.83546240284754 74.93268521330273) bank91576 +91577 POINT(40.05336237969305 73.48535261767593) bank91577 +91578 POINT(39.97618911782453 74.02272162792205) bank91578 +91579 POINT(41.55753951760166 73.0924971651965) bank91579 +91580 POINT(40.59581406157527 74.51736370591944) bank91580 +91581 POINT(40.42246914787428 74.73146732547954) bank91581 +91582 POINT(40.545635129241475 73.45329096421315) bank91582 +91583 POINT(40.40146688790566 73.48009788351854) bank91583 +91584 POINT(41.619774116442244 74.70607746007191) bank91584 +91585 POINT(40.33539488721277 73.69220850735161) bank91585 +91586 POINT(41.163386865417216 73.30737374556949) bank91586 +91587 POINT(41.22193050464265 73.85006089567945) bank91587 +91588 POINT(40.82746043595012 74.2076276574673) bank91588 +91589 POINT(41.402864494386066 74.29880373481805) bank91589 +91590 POINT(40.448757791537865 74.28137166131049) bank91590 +91591 POINT(40.51342064424423 73.87578288867134) bank91591 +91592 POINT(40.18120053272584 73.22879096663576) bank91592 +91593 POINT(40.4671498082041 74.18028038845017) bank91593 +91594 POINT(40.59880789501117 73.6151008024807) bank91594 +91595 POINT(41.635658192004485 73.62822611176783) bank91595 +91596 POINT(40.70660022382182 73.61277318422383) bank91596 +91597 POINT(41.519997435631325 74.61499860130989) bank91597 +91598 POINT(41.098239134961716 74.08614256421494) bank91598 +91599 POINT(41.07231445917126 74.3047278985905) bank91599 +91600 POINT(41.4026187046981 73.63643304704821) bank91600 +91601 POINT(40.07985887870738 74.16492466835012) bank91601 +91602 POINT(41.58638797914566 73.4180870592266) bank91602 +91603 POINT(41.263566605426995 73.10197490778684) bank91603 +91604 POINT(40.78306793672585 74.69598297925404) bank91604 +91605 POINT(39.94128142757974 74.3339652387221) bank91605 +91606 POINT(40.51551507617967 73.88075673151981) bank91606 +91607 POINT(40.29113463185785 74.23630286145792) bank91607 +91608 POINT(40.61423093867171 74.71737829103822) bank91608 +91609 POINT(41.21181650662653 73.3024900463437) bank91609 +91610 POINT(40.30131101128456 73.55684598154794) bank91610 +91611 POINT(39.83863095969281 74.9301960376546) bank91611 +91612 POINT(41.22215264402933 73.92514651048411) bank91612 +91613 POINT(40.84476455324428 73.44421485002026) bank91613 +91614 POINT(41.62323446672943 73.99834525822723) bank91614 +91615 POINT(40.349896635542024 74.54221527600956) bank91615 +91616 POINT(41.11016270775407 74.63737813774688) bank91616 +91617 POINT(41.18153665964653 74.81658836876858) bank91617 +91618 POINT(40.579243730891164 73.48659193930337) bank91618 +91619 POINT(41.21521937903392 74.41826935420283) bank91619 +91620 POINT(41.54639196019197 74.41568182463563) bank91620 +91621 POINT(41.160272606732576 73.6235067986147) bank91621 +91622 POINT(40.76775233028141 74.9530970008953) bank91622 +91623 POINT(40.417313589860626 73.2277263192629) bank91623 +91624 POINT(40.4131327136633 73.10706525796655) bank91624 +91625 POINT(41.50057075500709 73.83963846956281) bank91625 +91626 POINT(41.014651815649856 74.57775080416266) bank91626 +91627 POINT(40.97951517937297 74.64078520265146) bank91627 +91628 POINT(40.10503309731091 73.57376277754068) bank91628 +91629 POINT(40.608065344729056 74.25534417087843) bank91629 +91630 POINT(40.61328063951106 73.5148036684379) bank91630 +91631 POINT(41.08903137181567 73.248744014441) bank91631 +91632 POINT(40.18578223931527 73.82004695461507) bank91632 +91633 POINT(39.92801677524062 74.92505829663892) bank91633 +91634 POINT(41.46765613935815 73.66679575231807) bank91634 +91635 POINT(39.95707812554824 74.968456021882) bank91635 +91636 POINT(39.97201375872162 74.12591906146734) bank91636 +91637 POINT(40.74445116106187 73.60636511474468) bank91637 +91638 POINT(40.666433515759046 73.54703269611137) bank91638 +91639 POINT(40.048816016931696 74.65882591421284) bank91639 +91640 POINT(41.61075762939891 74.06928549564218) bank91640 +91641 POINT(40.49114936280042 73.35410647525151) bank91641 +91642 POINT(40.435930794307076 73.67650853557564) bank91642 +91643 POINT(41.201943332585905 74.57325849183725) bank91643 +91644 POINT(41.27829692041566 73.78766247320397) bank91644 +91645 POINT(40.500244188966185 74.03597640466369) bank91645 +91646 POINT(40.216344927095584 73.34532104918958) bank91646 +91647 POINT(41.49581363705392 74.91519421500182) bank91647 +91648 POINT(40.19819264584614 74.0338854680328) bank91648 +91649 POINT(39.72589452775341 73.42424705777573) bank91649 +91650 POINT(41.46061268164385 73.6567554364425) bank91650 +91651 POINT(41.288996198776516 73.69342693245792) bank91651 +91652 POINT(41.06690572900095 74.93081497810701) bank91652 +91653 POINT(40.40744448378033 73.04854759088217) bank91653 +91654 POINT(41.60027531453817 73.29180665699292) bank91654 +91655 POINT(40.66105371133602 74.2860990059968) bank91655 +91656 POINT(40.36862296075436 73.3465426683149) bank91656 +91657 POINT(40.50856408037827 74.63105518143271) bank91657 +91658 POINT(41.417184973331615 74.30104367651397) bank91658 +91659 POINT(40.601970398507014 73.34609707828447) bank91659 +91660 POINT(40.767494056519304 73.0320134583403) bank91660 +91661 POINT(41.11653442389672 74.55617056253925) bank91661 +91662 POINT(40.11457540101049 73.69039649753093) bank91662 +91663 POINT(40.690561274460116 74.15812270110455) bank91663 +91664 POINT(39.788794601618164 74.22438679154543) bank91664 +91665 POINT(41.386718885152234 73.95252331070206) bank91665 +91666 POINT(39.766823321867925 74.8848215356598) bank91666 +91667 POINT(41.67559851460718 74.88494097791906) bank91667 +91668 POINT(41.402392616866656 74.12583945985484) bank91668 +91669 POINT(40.358679377626316 73.1555589745156) bank91669 +91670 POINT(40.617090791660644 74.27000511107703) bank91670 +91671 POINT(41.58500661203398 73.88596541700002) bank91671 +91672 POINT(39.83323140906358 74.8541260344926) bank91672 +91673 POINT(39.83193885360305 74.31684459198782) bank91673 +91674 POINT(41.30946315721081 74.31803037148312) bank91674 +91675 POINT(40.889580741033285 73.18354526687256) bank91675 +91676 POINT(40.623759481783814 74.76589318626962) bank91676 +91677 POINT(41.29688049719082 73.74937103265195) bank91677 +91678 POINT(39.84943469847043 73.47722289216065) bank91678 +91679 POINT(41.47619154125012 73.48273181997284) bank91679 +91680 POINT(40.81318060404988 74.65039331590684) bank91680 +91681 POINT(40.623222386445775 73.17009424773012) bank91681 +91682 POINT(40.566839346014426 74.33292548032027) bank91682 +91683 POINT(40.47346569331038 73.68372773066) bank91683 +91684 POINT(40.33410188880875 74.62077203303784) bank91684 +91685 POINT(41.28106620536435 73.341727762008) bank91685 +91686 POINT(39.830325036849835 73.96997284588781) bank91686 +91687 POINT(41.22166367944379 73.26119814812347) bank91687 +91688 POINT(40.337936226387725 74.86893905284049) bank91688 +91689 POINT(39.88197057456688 74.11803643891875) bank91689 +91690 POINT(39.91270852891848 73.73566600040925) bank91690 +91691 POINT(41.1903912335846 73.62964047434407) bank91691 +91692 POINT(40.52382354948851 74.68751298756118) bank91692 +91693 POINT(41.20658056070968 74.79355629066998) bank91693 +91694 POINT(41.0870569249721 74.65225783252389) bank91694 +91695 POINT(39.79400716880906 73.28380189091966) bank91695 +91696 POINT(40.97008257136314 73.69427055547911) bank91696 +91697 POINT(41.66170438495924 74.51495584248144) bank91697 +91698 POINT(40.49715711119776 74.17445653462347) bank91698 +91699 POINT(40.24391612132457 73.17889783297413) bank91699 +91700 POINT(39.76058097129204 74.7790283363225) bank91700 +91701 POINT(40.589992875024876 74.02691914893143) bank91701 +91702 POINT(41.01809288679381 74.5700238171922) bank91702 +91703 POINT(39.76039157517866 74.88874616499183) bank91703 +91704 POINT(41.088238401917685 74.03151141038546) bank91704 +91705 POINT(40.55808182572936 73.09329762997015) bank91705 +91706 POINT(39.95437892073276 73.6692139838151) bank91706 +91707 POINT(40.11820505829198 73.80451110100101) bank91707 +91708 POINT(39.92902797577366 74.25309047793591) bank91708 +91709 POINT(41.42193398778332 73.8872425091211) bank91709 +91710 POINT(41.320798039334186 74.56000953077935) bank91710 +91711 POINT(40.55393030966708 74.44136676412602) bank91711 +91712 POINT(41.10528246973865 73.32158299653905) bank91712 +91713 POINT(41.47977662561823 73.38580121762146) bank91713 +91714 POINT(39.96800250066804 73.2432165523737) bank91714 +91715 POINT(41.103139460639646 74.71818849579779) bank91715 +91716 POINT(41.226648007195195 73.92829350550036) bank91716 +91717 POINT(40.84391611310051 73.2917511263349) bank91717 +91718 POINT(40.248299602611276 74.28384051716677) bank91718 +91719 POINT(41.41475161258772 73.08062954238068) bank91719 +91720 POINT(39.98539038515295 73.62748639237607) bank91720 +91721 POINT(40.486654342986164 73.9828525325533) bank91721 +91722 POINT(41.17638953374003 73.30933774700074) bank91722 +91723 POINT(41.10982121403584 73.55437025959553) bank91723 +91724 POINT(40.68738076754987 73.18512942074646) bank91724 +91725 POINT(40.151350140952 74.51331262383977) bank91725 +91726 POINT(40.15523220611859 74.88915475342316) bank91726 +91727 POINT(40.04255579720089 73.20185322196998) bank91727 +91728 POINT(41.473711272717836 73.06397612906112) bank91728 +91729 POINT(41.48214524358755 74.25766138553894) bank91729 +91730 POINT(40.06497670075844 74.28698326682732) bank91730 +91731 POINT(39.83277955812542 74.07282815966728) bank91731 +91732 POINT(40.1879395143008 74.03769883711092) bank91732 +91733 POINT(41.14678053221843 74.18297688343388) bank91733 +91734 POINT(40.16113196706864 74.87347455560989) bank91734 +91735 POINT(40.5266734946424 73.22727364497177) bank91735 +91736 POINT(41.38777934890027 73.10606854603544) bank91736 +91737 POINT(39.840200433386514 74.21504916422656) bank91737 +91738 POINT(41.632567377008215 74.20886166089866) bank91738 +91739 POINT(41.302870977007174 73.54589685776132) bank91739 +91740 POINT(41.31963200337974 74.10844637987988) bank91740 +91741 POINT(40.197898663520746 74.9689546416785) bank91741 +91742 POINT(39.83337904401604 73.75490934893182) bank91742 +91743 POINT(39.941128673535914 74.40127322505384) bank91743 +91744 POINT(41.13554061654265 74.85995622314708) bank91744 +91745 POINT(40.97073269739649 74.43845308045732) bank91745 +91746 POINT(40.60832562251933 74.42037793629927) bank91746 +91747 POINT(39.97646944320531 74.81825061593) bank91747 +91748 POINT(40.96161746578243 73.63898998093818) bank91748 +91749 POINT(40.35837031370038 74.53947325777743) bank91749 +91750 POINT(39.94200402893424 74.84621853151519) bank91750 +91751 POINT(39.76336254695143 74.2188785780378) bank91751 +91752 POINT(41.576960603475314 74.42044513397161) bank91752 +91753 POINT(40.119526916503354 74.26201484693219) bank91753 +91754 POINT(40.04048217244354 74.20123720161564) bank91754 +91755 POINT(40.21638765668435 74.78406842349095) bank91755 +91756 POINT(40.1441556260217 73.13779095845618) bank91756 +91757 POINT(39.771709940797095 74.9602745882616) bank91757 +91758 POINT(40.52214291281042 74.27628994916938) bank91758 +91759 POINT(40.41598919250494 74.0886647370418) bank91759 +91760 POINT(41.30114433150459 74.08504612371381) bank91760 +91761 POINT(40.024679620998775 73.186679559277) bank91761 +91762 POINT(41.659604251286204 74.3999261983763) bank91762 +91763 POINT(40.82051574598856 73.84643841237767) bank91763 +91764 POINT(41.39710359570693 74.57909207992292) bank91764 +91765 POINT(39.73125458977201 74.36399312000748) bank91765 +91766 POINT(40.273874390657596 74.78157242875257) bank91766 +91767 POINT(40.462164981309265 74.63727116271167) bank91767 +91768 POINT(40.664487993552164 73.16133126948623) bank91768 +91769 POINT(41.067193784019835 73.81721239722485) bank91769 +91770 POINT(41.23005757223898 74.14328536149395) bank91770 +91771 POINT(41.43029212438621 73.6044833953352) bank91771 +91772 POINT(41.36263501249595 73.33885722343017) bank91772 +91773 POINT(39.721676710652744 74.52326894199165) bank91773 +91774 POINT(40.26203070177493 74.89294827204183) bank91774 +91775 POINT(40.206731741027056 73.61380410453826) bank91775 +91776 POINT(41.63103919408014 74.47260586364385) bank91776 +91777 POINT(39.98520528211148 74.7942894303602) bank91777 +91778 POINT(39.976450007022294 73.42192387565473) bank91778 +91779 POINT(41.32054452276222 74.53169552473437) bank91779 +91780 POINT(40.97348323245832 74.67736640138949) bank91780 +91781 POINT(41.16134753147048 73.24643598760258) bank91781 +91782 POINT(39.94680132759536 74.42295569314193) bank91782 +91783 POINT(40.270599287363964 74.60613613268195) bank91783 +91784 POINT(41.697595737246985 73.27276500280192) bank91784 +91785 POINT(40.218399566148754 74.37312279681552) bank91785 +91786 POINT(39.91030891247771 74.8708690353638) bank91786 +91787 POINT(40.051136360685426 74.91655821154882) bank91787 +91788 POINT(40.186115262687096 74.67080382918418) bank91788 +91789 POINT(40.72026512509217 73.29602976070208) bank91789 +91790 POINT(40.25112308317694 74.25186082556056) bank91790 +91791 POINT(39.937624944487325 74.0198433765608) bank91791 +91792 POINT(41.69969873844017 74.32663939890263) bank91792 +91793 POINT(40.367084074838516 73.61820913394473) bank91793 +91794 POINT(40.233384298706426 73.4861806200725) bank91794 +91795 POINT(41.487252460760686 73.04396116665278) bank91795 +91796 POINT(41.14208690289704 73.8218573093734) bank91796 +91797 POINT(40.57595288683292 73.14107979989167) bank91797 +91798 POINT(41.65596543722396 74.72427617020725) bank91798 +91799 POINT(41.10220352892615 73.93634693329277) bank91799 +91800 POINT(41.49340151123691 73.82675076177657) bank91800 +91801 POINT(41.547982168275084 74.4554152772367) bank91801 +91802 POINT(41.01721055203248 74.48738528010323) bank91802 +91803 POINT(41.67940422650877 73.58287503535978) bank91803 +91804 POINT(40.7722900648071 73.70670307637367) bank91804 +91805 POINT(41.224810961053464 73.73970383387599) bank91805 +91806 POINT(41.25415655331967 74.65781611875757) bank91806 +91807 POINT(40.62759827546337 74.09960717328804) bank91807 +91808 POINT(41.65330116948753 74.01707586378389) bank91808 +91809 POINT(41.53990126242489 74.15078640916516) bank91809 +91810 POINT(41.31691023380807 73.08279520157252) bank91810 +91811 POINT(40.419677541254586 73.32967894164733) bank91811 +91812 POINT(39.780553298731064 73.14424964103898) bank91812 +91813 POINT(41.38742036373497 74.35551684674837) bank91813 +91814 POINT(40.20595273444757 74.44368109123657) bank91814 +91815 POINT(40.35615026377685 74.01048327128127) bank91815 +91816 POINT(40.885279324938175 73.84178069104588) bank91816 +91817 POINT(40.78665748997283 74.48140750142842) bank91817 +91818 POINT(40.41074997063005 74.6395811687282) bank91818 +91819 POINT(41.207859037653584 74.2794244662562) bank91819 +91820 POINT(40.96846838712329 73.11671551367942) bank91820 +91821 POINT(39.90196321043214 73.74293758058771) bank91821 +91822 POINT(40.27412132466802 73.23015399342351) bank91822 +91823 POINT(41.25157622317347 74.26673428505002) bank91823 +91824 POINT(41.24718474813773 74.7382286080948) bank91824 +91825 POINT(40.70700016646196 74.224585823151) bank91825 +91826 POINT(40.93862963246301 73.25634568524633) bank91826 +91827 POINT(39.94200106372275 73.22457093045807) bank91827 +91828 POINT(39.80422165742833 73.48676115683287) bank91828 +91829 POINT(40.92929714737742 74.26399083951398) bank91829 +91830 POINT(41.465373467929105 74.45971061327906) bank91830 +91831 POINT(41.69307734187479 73.53516858775167) bank91831 +91832 POINT(41.34886202410038 74.42322321350667) bank91832 +91833 POINT(39.78096271651237 74.67518534979132) bank91833 +91834 POINT(40.20638415660992 74.56616423844285) bank91834 +91835 POINT(41.19257746197026 74.13675995180567) bank91835 +91836 POINT(41.5148713343941 74.27361343515544) bank91836 +91837 POINT(40.012714043684625 73.27778422494733) bank91837 +91838 POINT(40.87861402874055 74.80751684060309) bank91838 +91839 POINT(40.847669646924174 74.17532600048979) bank91839 +91840 POINT(40.58551308598632 73.57581162390359) bank91840 +91841 POINT(41.4353743484514 73.91307017212229) bank91841 +91842 POINT(39.78612992479421 74.95492094070008) bank91842 +91843 POINT(39.9095992828655 73.44765567740711) bank91843 +91844 POINT(40.87832384294922 74.05663834343835) bank91844 +91845 POINT(41.59691132337544 74.65204192892217) bank91845 +91846 POINT(40.09037644080988 74.39124306546147) bank91846 +91847 POINT(40.68716807889285 74.17778629815173) bank91847 +91848 POINT(41.237202504744715 73.85910454429974) bank91848 +91849 POINT(41.3878959985975 74.58601775998268) bank91849 +91850 POINT(39.844411253422095 74.20314329755868) bank91850 +91851 POINT(41.54238099522729 73.96127375170033) bank91851 +91852 POINT(41.28293057005924 73.77708795254188) bank91852 +91853 POINT(40.89036209452135 73.20399032510142) bank91853 +91854 POINT(40.739944608919735 73.08250093643532) bank91854 +91855 POINT(41.64363394887389 74.6148569753281) bank91855 +91856 POINT(40.97347311322556 74.5560652748236) bank91856 +91857 POINT(40.233215540497866 74.91255741583863) bank91857 +91858 POINT(41.52074393427498 74.28149191054148) bank91858 +91859 POINT(40.42818038167358 73.36644317393888) bank91859 +91860 POINT(40.558591026061926 74.1132056946701) bank91860 +91861 POINT(40.8288021865659 74.61093388115678) bank91861 +91862 POINT(41.37150031061065 73.65301301254132) bank91862 +91863 POINT(40.275786586496906 74.85542930146785) bank91863 +91864 POINT(39.87556296382174 74.4609639910428) bank91864 +91865 POINT(41.33095695208969 74.70245938288137) bank91865 +91866 POINT(41.36270788589263 74.01344095673949) bank91866 +91867 POINT(41.18791371576813 74.57024441954532) bank91867 +91868 POINT(40.90292082987024 74.2517312194479) bank91868 +91869 POINT(40.35055606955939 74.33920471153947) bank91869 +91870 POINT(40.77964576960555 73.74038957075365) bank91870 +91871 POINT(40.05325708379232 73.59433864659849) bank91871 +91872 POINT(40.21125886500511 73.56830265909213) bank91872 +91873 POINT(41.611218683642186 73.87832134966526) bank91873 +91874 POINT(39.85162817039768 73.50391645590302) bank91874 +91875 POINT(40.3148112855276 73.73615857695702) bank91875 +91876 POINT(39.74182743788491 73.07971319557441) bank91876 +91877 POINT(41.03612615154146 74.56369959085595) bank91877 +91878 POINT(40.148952071860556 74.66472109662244) bank91878 +91879 POINT(40.251355838888124 73.46050079638903) bank91879 +91880 POINT(41.3105234401783 73.64127954561229) bank91880 +91881 POINT(41.60583944322605 74.17633295206602) bank91881 +91882 POINT(40.572847441654496 74.76598406840192) bank91882 +91883 POINT(40.997868853365375 73.10520673899508) bank91883 +91884 POINT(41.689144056196724 73.51952521193262) bank91884 +91885 POINT(40.244430625466165 73.94630177004622) bank91885 +91886 POINT(41.12344452510968 74.3492409987163) bank91886 +91887 POINT(40.807325333298294 73.4491125759647) bank91887 +91888 POINT(41.17280650560395 73.22202051562793) bank91888 +91889 POINT(40.02595986431724 73.74560416674952) bank91889 +91890 POINT(40.56498697481643 73.78960600423251) bank91890 +91891 POINT(40.41954967441439 73.00606051989256) bank91891 +91892 POINT(41.28065832521223 73.44126088425418) bank91892 +91893 POINT(40.09966430505261 73.89313719097176) bank91893 +91894 POINT(40.841252916470346 73.70265999342675) bank91894 +91895 POINT(41.00039708082509 74.45360216216832) bank91895 +91896 POINT(41.680591512955175 73.99392764691477) bank91896 +91897 POINT(41.623289619998744 74.69531558821109) bank91897 +91898 POINT(40.41704118979752 73.14362412289934) bank91898 +91899 POINT(40.52017528468878 74.55543211375311) bank91899 +91900 POINT(41.092763224417844 74.25323451702657) bank91900 +91901 POINT(39.8765177369146 74.13630274150555) bank91901 +91902 POINT(41.168705154214585 74.86207831943342) bank91902 +91903 POINT(40.125234626933825 74.35856246446491) bank91903 +91904 POINT(41.16365332955206 74.85913549217169) bank91904 +91905 POINT(40.022538697289136 73.4424139843532) bank91905 +91906 POINT(40.61595483904748 74.13101279587268) bank91906 +91907 POINT(39.733318472016705 74.29185416307206) bank91907 +91908 POINT(40.005096144677765 73.01844274022851) bank91908 +91909 POINT(40.42486828305787 74.07510316531493) bank91909 +91910 POINT(41.194281492930116 74.37151214981118) bank91910 +91911 POINT(41.51626884239504 74.2479460346403) bank91911 +91912 POINT(41.70082612312092 73.44629751440277) bank91912 +91913 POINT(40.84903351945557 73.76922459466559) bank91913 +91914 POINT(40.837485840874756 73.5696789304168) bank91914 +91915 POINT(40.09525087357431 74.36941152584791) bank91915 +91916 POINT(40.06916701583119 73.92024504846529) bank91916 +91917 POINT(41.07519716044126 74.92320028789267) bank91917 +91918 POINT(41.326465210544825 74.0279177982946) bank91918 +91919 POINT(40.825450428922075 74.3874990533931) bank91919 +91920 POINT(41.6016374041054 74.12480762972399) bank91920 +91921 POINT(41.20237967264119 74.65912235347058) bank91921 +91922 POINT(41.06208870465022 74.73736848167555) bank91922 +91923 POINT(40.680967757994324 73.72541398916522) bank91923 +91924 POINT(40.16024402240108 73.4135894612417) bank91924 +91925 POINT(40.32188015352954 74.19185223200076) bank91925 +91926 POINT(40.26579231455282 74.92461619381214) bank91926 +91927 POINT(39.869232443224654 73.60606771091425) bank91927 +91928 POINT(40.25948953161033 74.25930575539384) bank91928 +91929 POINT(39.74122766188488 74.87109464092542) bank91929 +91930 POINT(40.07826638626042 74.04723481766229) bank91930 +91931 POINT(40.237825537900676 73.30570975116096) bank91931 +91932 POINT(40.741584022798996 73.32062935349128) bank91932 +91933 POINT(41.06084671507394 73.66974597417682) bank91933 +91934 POINT(40.48627249712566 74.35418613470607) bank91934 +91935 POINT(41.50577428393638 73.69360207023408) bank91935 +91936 POINT(41.57754911532211 73.7362342664613) bank91936 +91937 POINT(41.32314452759378 74.85860460547947) bank91937 +91938 POINT(41.206489497345736 74.0210572106242) bank91938 +91939 POINT(39.80823709141351 73.82921834718071) bank91939 +91940 POINT(39.80834446140095 74.55259498381321) bank91940 +91941 POINT(40.67255071151521 73.68955575699968) bank91941 +91942 POINT(40.70402060695487 74.47809066244021) bank91942 +91943 POINT(40.23376489534732 74.65270422610537) bank91943 +91944 POINT(40.678786308341465 74.6497451117056) bank91944 +91945 POINT(39.859555919849626 74.71637258563064) bank91945 +91946 POINT(40.20060959644862 73.753901340941) bank91946 +91947 POINT(41.32820069644293 73.62693273688288) bank91947 +91948 POINT(39.924895553343106 74.27241983054684) bank91948 +91949 POINT(40.75052912641792 74.51656533770908) bank91949 +91950 POINT(41.59394313725243 73.45076947406827) bank91950 +91951 POINT(40.53270664968724 74.85745325672698) bank91951 +91952 POINT(41.40250198532026 74.77743871975262) bank91952 +91953 POINT(41.24180266746453 74.21575494852965) bank91953 +91954 POINT(41.19081082672711 73.7587915237611) bank91954 +91955 POINT(41.55099576181018 74.83444226050398) bank91955 +91956 POINT(41.70615640079654 74.34489707192671) bank91956 +91957 POINT(41.54684841697675 73.74522313653101) bank91957 +91958 POINT(40.39077901328133 73.91698385903172) bank91958 +91959 POINT(41.07590737969129 74.84829469002622) bank91959 +91960 POINT(41.69992322990903 73.99702400216205) bank91960 +91961 POINT(40.55124707497072 73.87349932508472) bank91961 +91962 POINT(41.649753139554264 74.68113200478241) bank91962 +91963 POINT(40.229495421523616 73.00899483884848) bank91963 +91964 POINT(41.330091196604826 73.46416230300149) bank91964 +91965 POINT(41.34425590854115 74.54888407902378) bank91965 +91966 POINT(41.52580226469618 74.1841474107541) bank91966 +91967 POINT(41.3729184614609 74.31242322549335) bank91967 +91968 POINT(40.42164524898087 74.55669549589682) bank91968 +91969 POINT(40.97714753647509 73.17459746897588) bank91969 +91970 POINT(41.29300449371839 74.52304491051403) bank91970 +91971 POINT(41.35801916715625 74.33854191985179) bank91971 +91972 POINT(40.7821210793895 73.7754690860965) bank91972 +91973 POINT(40.5745516044569 73.54942411718902) bank91973 +91974 POINT(41.57854238276155 73.01827936827488) bank91974 +91975 POINT(40.26117708108904 73.404175569075) bank91975 +91976 POINT(40.52547639973025 74.91203276002865) bank91976 +91977 POINT(41.14712530526654 73.12066764685103) bank91977 +91978 POINT(41.48006551760776 73.99956366499036) bank91978 +91979 POINT(40.452950010047076 73.53520414777985) bank91979 +91980 POINT(40.680010024267325 73.56168316052016) bank91980 +91981 POINT(39.81802988250693 73.53391999921986) bank91981 +91982 POINT(40.54226985460258 74.5715905466716) bank91982 +91983 POINT(39.755112249931564 74.06541655923836) bank91983 +91984 POINT(40.85460857880924 74.18127139035376) bank91984 +91985 POINT(40.68714475468316 73.02330726654573) bank91985 +91986 POINT(40.18252308180034 73.09599609014627) bank91986 +91987 POINT(40.7812484181376 73.83322526065137) bank91987 +91988 POINT(41.300861538214555 73.67544053710952) bank91988 +91989 POINT(39.734565482035386 74.0456964454595) bank91989 +91990 POINT(41.020648362068506 74.07619002700453) bank91990 +91991 POINT(40.279417107329564 73.77217730010078) bank91991 +91992 POINT(41.100189718823486 74.7212451061293) bank91992 +91993 POINT(40.576435182992995 74.0598630240037) bank91993 +91994 POINT(40.53463528250114 74.00758988203648) bank91994 +91995 POINT(41.50793140931986 73.28546932627746) bank91995 +91996 POINT(40.587981003261575 73.89556126297857) bank91996 +91997 POINT(41.265416496064944 74.43416224401555) bank91997 +91998 POINT(40.70678014771431 73.9350143689291) bank91998 +91999 POINT(39.87181214033571 74.08273384430555) bank91999 +92000 POINT(41.02054493948309 73.59851717218002) bank92000 +92001 POINT(40.576613256361284 73.27118061534803) bank92001 +92002 POINT(41.47135284176322 74.2427801309637) bank92002 +92003 POINT(41.175193785184376 74.93562327060178) bank92003 +92004 POINT(41.12132166154695 73.6864687108668) bank92004 +92005 POINT(40.34525529793828 74.74902509715479) bank92005 +92006 POINT(41.57826921098542 74.40349450879344) bank92006 +92007 POINT(41.584279008007655 74.05413571543056) bank92007 +92008 POINT(40.043038190957915 74.11474371382525) bank92008 +92009 POINT(40.76623800617015 73.0850177889836) bank92009 +92010 POINT(39.88983333696521 74.63059129359382) bank92010 +92011 POINT(41.29840334937631 73.06891764140477) bank92011 +92012 POINT(40.860725736533325 74.0808383824561) bank92012 +92013 POINT(40.387390254468876 73.16005901037346) bank92013 +92014 POINT(39.77753218911015 74.78987536947609) bank92014 +92015 POINT(40.99657928378016 73.39146473338855) bank92015 +92016 POINT(41.533938042871455 74.24838937011637) bank92016 +92017 POINT(40.594552221581836 73.60631084303239) bank92017 +92018 POINT(40.792635817079905 73.71334593340592) bank92018 +92019 POINT(40.10352009720083 73.39563471303573) bank92019 +92020 POINT(41.66952816038086 73.46566143531113) bank92020 +92021 POINT(41.10379623197037 74.55888467429398) bank92021 +92022 POINT(41.11800155612705 73.55878969721769) bank92022 +92023 POINT(41.47342841912987 73.93900541535146) bank92023 +92024 POINT(41.286965801183726 74.57529247181094) bank92024 +92025 POINT(39.76142727810663 74.0409489883622) bank92025 +92026 POINT(40.92959030845981 73.01704951841522) bank92026 +92027 POINT(40.05469409143007 73.59164416176506) bank92027 +92028 POINT(40.59960610146661 73.86660235127616) bank92028 +92029 POINT(40.21590917422167 73.94909784678076) bank92029 +92030 POINT(41.27753835828228 73.26335852860693) bank92030 +92031 POINT(39.955840915695475 73.12883170872179) bank92031 +92032 POINT(41.125148647767105 73.06288011956582) bank92032 +92033 POINT(40.23754091041001 73.25793610694413) bank92033 +92034 POINT(39.75741730469464 73.26967974167185) bank92034 +92035 POINT(41.481865051078344 73.33511906134264) bank92035 +92036 POINT(40.562088318331874 74.79140322248512) bank92036 +92037 POINT(39.7790582751462 73.32715707425638) bank92037 +92038 POINT(40.44631284692249 73.90006686897452) bank92038 +92039 POINT(39.97394045372092 73.56345766883894) bank92039 +92040 POINT(40.47515870013694 73.98475221168297) bank92040 +92041 POINT(41.63406312561174 74.06203324359329) bank92041 +92042 POINT(40.50846846390513 74.30976178472504) bank92042 +92043 POINT(41.23744514601173 73.22443277983623) bank92043 +92044 POINT(40.33485294770784 74.25913623674033) bank92044 +92045 POINT(41.283472887552676 74.97108841522486) bank92045 +92046 POINT(39.78012450024455 74.62289147868202) bank92046 +92047 POINT(41.464642891695256 74.78535760513786) bank92047 +92048 POINT(39.72276027277122 73.81098320887251) bank92048 +92049 POINT(41.6698387363485 74.48455283939441) bank92049 +92050 POINT(40.315036124038755 73.24982569984076) bank92050 +92051 POINT(40.17316070349879 73.24287524836421) bank92051 +92052 POINT(40.78878990363468 74.74749380134554) bank92052 +92053 POINT(40.19467072538813 74.21570857655063) bank92053 +92054 POINT(41.18288800678945 74.05585314235503) bank92054 +92055 POINT(41.67461872124671 73.28162999209344) bank92055 +92056 POINT(40.952918887201385 74.97328136646635) bank92056 +92057 POINT(41.35814948076956 73.25186620248117) bank92057 +92058 POINT(40.29895181206719 73.22000547107905) bank92058 +92059 POINT(40.59890712357078 73.71488715788561) bank92059 +92060 POINT(41.562231205078625 74.27336237160637) bank92060 +92061 POINT(41.36052751566458 73.29015818604174) bank92061 +92062 POINT(39.90478540328215 74.60091423128911) bank92062 +92063 POINT(40.09145877590936 73.55936909925553) bank92063 +92064 POINT(40.89191148557738 74.35309669748786) bank92064 +92065 POINT(40.15074373689712 74.48619042257135) bank92065 +92066 POINT(40.58719869022716 73.95552541701352) bank92066 +92067 POINT(41.15933835290162 73.16450003550759) bank92067 +92068 POINT(39.738865269826434 73.27822761485437) bank92068 +92069 POINT(40.51485329336611 73.06588314997066) bank92069 +92070 POINT(41.56113005531397 73.92357658099024) bank92070 +92071 POINT(40.78283090544148 74.93014406093545) bank92071 +92072 POINT(41.3021322502889 73.91290968350907) bank92072 +92073 POINT(41.49407209412377 74.17079294800605) bank92073 +92074 POINT(39.900288482153506 73.71781501478328) bank92074 +92075 POINT(41.34096799460213 74.46002698693523) bank92075 +92076 POINT(41.24159901731043 73.41538970628523) bank92076 +92077 POINT(41.61802529694508 74.14924782154375) bank92077 +92078 POINT(39.93601244294478 73.18629645984946) bank92078 +92079 POINT(40.55764141883582 73.80079039997054) bank92079 +92080 POINT(41.478721159377734 74.56703252003325) bank92080 +92081 POINT(39.91771677385241 73.83891286402313) bank92081 +92082 POINT(40.093182697210175 74.17387164804299) bank92082 +92083 POINT(40.036479480075755 74.83030330927619) bank92083 +92084 POINT(40.250754043813316 73.85801599480884) bank92084 +92085 POINT(39.99832188752672 73.2027829576232) bank92085 +92086 POINT(40.077563622446554 73.56577251919836) bank92086 +92087 POINT(40.29153684125056 74.59120697400768) bank92087 +92088 POINT(39.72885861364376 74.10943181655882) bank92088 +92089 POINT(39.77159249482709 74.21872095934353) bank92089 +92090 POINT(40.191283683126 73.635620684933) bank92090 +92091 POINT(39.77561502312208 74.42119188544993) bank92091 +92092 POINT(40.00357159893366 74.86071364106522) bank92092 +92093 POINT(40.20418677115429 73.57938644748792) bank92093 +92094 POINT(40.65549072659547 74.2099457648472) bank92094 +92095 POINT(39.86202946412463 74.32514880924539) bank92095 +92096 POINT(41.27182454443562 73.67210697169556) bank92096 +92097 POINT(41.34278555949785 74.24869509886973) bank92097 +92098 POINT(40.81939965038291 73.69207253933601) bank92098 +92099 POINT(40.76459506097579 74.04860745952212) bank92099 +92100 POINT(41.17611795451577 73.7791552109961) bank92100 +92101 POINT(40.97465013332888 74.85881116686647) bank92101 +92102 POINT(40.17320954780898 74.85944041531023) bank92102 +92103 POINT(40.71526921027726 74.82286303244979) bank92103 +92104 POINT(40.116087729642096 74.74281419794521) bank92104 +92105 POINT(41.11378652872221 74.91395357175391) bank92105 +92106 POINT(41.3616607320552 73.72436971541168) bank92106 +92107 POINT(41.135677624915914 73.66599143505118) bank92107 +92108 POINT(40.97874045661183 74.68826369370205) bank92108 +92109 POINT(41.24739498716279 73.94558513866428) bank92109 +92110 POINT(40.44756158036628 74.53141356747298) bank92110 +92111 POINT(41.686500193748564 74.48962719557126) bank92111 +92112 POINT(41.531292718335926 73.70006477338238) bank92112 +92113 POINT(40.50181106982328 74.67178689421304) bank92113 +92114 POINT(40.40064358266264 74.80839600992735) bank92114 +92115 POINT(41.70210640850589 73.05857087747212) bank92115 +92116 POINT(40.1692307009102 73.50043648950783) bank92116 +92117 POINT(41.336995495281776 74.7179027171482) bank92117 +92118 POINT(40.292989304486326 73.96979091965403) bank92118 +92119 POINT(41.03779678064189 74.00466490480129) bank92119 +92120 POINT(40.204699556817566 74.66435716023386) bank92120 +92121 POINT(40.95279195541249 73.50046679932905) bank92121 +92122 POINT(41.43653251618394 73.59399066334583) bank92122 +92123 POINT(40.798184995230265 73.4529919201325) bank92123 +92124 POINT(40.27259152376692 73.8019613510649) bank92124 +92125 POINT(39.8576383849728 73.26060659345744) bank92125 +92126 POINT(41.230209054340854 74.08807846119677) bank92126 +92127 POINT(40.32649413794291 74.43727250999561) bank92127 +92128 POINT(40.371468790787375 74.88660662463734) bank92128 +92129 POINT(41.469899150088395 74.73013683162748) bank92129 +92130 POINT(41.53814799253226 73.40368357224632) bank92130 +92131 POINT(40.101098395739335 74.28129714971949) bank92131 +92132 POINT(40.50348703068117 74.77267236751224) bank92132 +92133 POINT(41.17199806705098 74.51316691089811) bank92133 +92134 POINT(40.48436049889962 73.08773755188085) bank92134 +92135 POINT(41.56941609794297 74.86863480280202) bank92135 +92136 POINT(41.588016040202454 73.08701232595465) bank92136 +92137 POINT(40.2530935393177 73.68382830198573) bank92137 +92138 POINT(40.480995436766705 74.70075071574267) bank92138 +92139 POINT(40.79043645410291 74.33176712605665) bank92139 +92140 POINT(40.32041076976773 74.63972130630994) bank92140 +92141 POINT(40.396976687314485 73.71922875622975) bank92141 +92142 POINT(40.328697210698735 73.17284107028499) bank92142 +92143 POINT(39.97467442047791 74.44484455189198) bank92143 +92144 POINT(41.152201168064394 74.75561056471885) bank92144 +92145 POINT(41.11636877431187 73.60802216496381) bank92145 +92146 POINT(41.27455788418349 74.75851778389602) bank92146 +92147 POINT(41.16836354788932 73.64222448975856) bank92147 +92148 POINT(41.02726748383554 73.04048976960176) bank92148 +92149 POINT(40.060117816277156 74.85454948746602) bank92149 +92150 POINT(40.19724887392126 73.18589780564889) bank92150 +92151 POINT(40.913859909456264 74.87380853219827) bank92151 +92152 POINT(39.812552216533426 73.26692803905766) bank92152 +92153 POINT(40.68920100130411 73.06043058742685) bank92153 +92154 POINT(40.62603866786678 74.00399851894572) bank92154 +92155 POINT(40.52490766558816 73.69317242322191) bank92155 +92156 POINT(39.72679593239107 73.25523149119871) bank92156 +92157 POINT(40.94801721803148 74.51248780242942) bank92157 +92158 POINT(41.505347910471876 74.19288368076886) bank92158 +92159 POINT(41.10914217297557 74.91149006698288) bank92159 +92160 POINT(40.79590614768466 74.43237815209301) bank92160 +92161 POINT(39.98820890767477 73.46556477700918) bank92161 +92162 POINT(40.294544072453874 74.70975076793002) bank92162 +92163 POINT(40.28670756126095 73.48707923899609) bank92163 +92164 POINT(41.33303280220491 73.52255092490604) bank92164 +92165 POINT(39.81235739224977 73.01612997938048) bank92165 +92166 POINT(41.12664757226432 73.20323621268082) bank92166 +92167 POINT(41.2093880683413 73.31302518910222) bank92167 +92168 POINT(41.383092028576606 73.23633083782916) bank92168 +92169 POINT(39.769382338031356 73.34223422518915) bank92169 +92170 POINT(41.04144095252207 73.85274321790858) bank92170 +92171 POINT(41.38641244831923 74.57527045186957) bank92171 +92172 POINT(40.243018907846334 73.78160857776243) bank92172 +92173 POINT(39.96677583071403 74.34249900921205) bank92173 +92174 POINT(40.562724790750096 73.43589194439826) bank92174 +92175 POINT(41.39056311011026 74.46076653867897) bank92175 +92176 POINT(41.20371866803926 73.9010813866155) bank92176 +92177 POINT(40.363943914530374 73.69073835535877) bank92177 +92178 POINT(40.616684402195034 73.01029310497975) bank92178 +92179 POINT(39.74040373431498 74.97905505968535) bank92179 +92180 POINT(39.71736289473368 73.34024794392283) bank92180 +92181 POINT(40.21918361605608 74.058383001442) bank92181 +92182 POINT(39.77863516635574 73.95662195308884) bank92182 +92183 POINT(39.99800934142947 73.6047254201969) bank92183 +92184 POINT(39.71850113230739 73.73362124525599) bank92184 +92185 POINT(40.43767302971264 73.56261787243824) bank92185 +92186 POINT(40.772477266993185 73.96653173224922) bank92186 +92187 POINT(40.13936358381963 74.12126162405201) bank92187 +92188 POINT(40.95193529921512 73.72978587464003) bank92188 +92189 POINT(40.169490318197916 73.8825312255325) bank92189 +92190 POINT(40.258701765973996 73.91016846874736) bank92190 +92191 POINT(40.9507904625887 74.43190438915326) bank92191 +92192 POINT(41.56377423621229 73.11397986375353) bank92192 +92193 POINT(41.61097214025772 74.33635949880279) bank92193 +92194 POINT(41.06734966447727 73.05169000077697) bank92194 +92195 POINT(41.202047311263634 73.18807644155261) bank92195 +92196 POINT(40.87563584496932 74.99444701793992) bank92196 +92197 POINT(40.963279907153805 74.97052265407636) bank92197 +92198 POINT(39.89576866829103 73.32106965728606) bank92198 +92199 POINT(40.561495925859184 74.17663750593884) bank92199 +92200 POINT(40.48438421567773 73.4759088621023) bank92200 +92201 POINT(41.4975087350417 73.8696379139964) bank92201 +92202 POINT(40.62751749240879 74.4600333143225) bank92202 +92203 POINT(39.78195714440291 74.03349382866364) bank92203 +92204 POINT(41.373389710321 73.36900337124192) bank92204 +92205 POINT(41.386480046417766 73.99467949480004) bank92205 +92206 POINT(41.188602303531475 74.00369282636734) bank92206 +92207 POINT(40.4283432710472 74.75284330341238) bank92207 +92208 POINT(40.052814072273485 74.94180591751831) bank92208 +92209 POINT(40.73634526805093 74.45872469746622) bank92209 +92210 POINT(39.71778367325465 74.90023517503805) bank92210 +92211 POINT(41.25703734246755 74.60913105222393) bank92211 +92212 POINT(40.05766188797985 73.01188110237041) bank92212 +92213 POINT(41.28219705981171 73.63885477269284) bank92213 +92214 POINT(41.217023879363545 74.9170777633315) bank92214 +92215 POINT(40.937969214572945 73.09657106906477) bank92215 +92216 POINT(40.71817570047397 74.4862022071723) bank92216 +92217 POINT(40.925543998323015 74.3127393563788) bank92217 +92218 POINT(41.08209238668007 73.75517439522399) bank92218 +92219 POINT(40.656010031540355 73.59236815288585) bank92219 +92220 POINT(41.54077465657211 73.12797086042087) bank92220 +92221 POINT(41.2692485366844 73.64544003763086) bank92221 +92222 POINT(40.188150139686925 73.14033553253238) bank92222 +92223 POINT(40.74179700002256 74.80107239745425) bank92223 +92224 POINT(41.01632634139869 73.37745384328036) bank92224 +92225 POINT(40.0740388598171 73.8906241245418) bank92225 +92226 POINT(41.52075639543719 74.19059285572408) bank92226 +92227 POINT(40.956629159015506 74.00912697661143) bank92227 +92228 POINT(40.043820371425625 73.53448045552594) bank92228 +92229 POINT(41.10058388491092 73.81130931327475) bank92229 +92230 POINT(41.10504392270433 73.02222013625725) bank92230 +92231 POINT(41.236078165492565 74.8107134705658) bank92231 +92232 POINT(40.79183864683573 74.47686709558052) bank92232 +92233 POINT(41.63092419616911 74.31883464237218) bank92233 +92234 POINT(41.08452182601084 73.7858617796672) bank92234 +92235 POINT(41.39220507745686 74.98794003217589) bank92235 +92236 POINT(40.16358079562922 74.2218413509628) bank92236 +92237 POINT(40.72760808706413 73.62315669421014) bank92237 +92238 POINT(41.299749091108595 74.65771900164599) bank92238 +92239 POINT(40.06652840908618 74.08260210554023) bank92239 +92240 POINT(41.16481969484853 73.17576915015019) bank92240 +92241 POINT(40.75690540379083 74.6187094828725) bank92241 +92242 POINT(40.257511268092834 74.62200721248274) bank92242 +92243 POINT(40.59744376217095 73.32658581300866) bank92243 +92244 POINT(41.00668037901188 74.4947953772612) bank92244 +92245 POINT(40.97120760842701 73.92809609703012) bank92245 +92246 POINT(40.23611528617584 73.19523610062302) bank92246 +92247 POINT(40.117332081628184 73.23372951772578) bank92247 +92248 POINT(40.27617535243399 74.48774886627744) bank92248 +92249 POINT(40.40696450719605 73.78518657914205) bank92249 +92250 POINT(40.90992059161043 74.848449355128) bank92250 +92251 POINT(40.43845622388308 73.68726265463422) bank92251 +92252 POINT(39.98308123790326 74.26010246672615) bank92252 +92253 POINT(40.5474750256636 74.45583107102725) bank92253 +92254 POINT(40.94965020920406 74.63947012313957) bank92254 +92255 POINT(40.0034884256624 73.19217665540972) bank92255 +92256 POINT(40.337053858891366 73.97939510334851) bank92256 +92257 POINT(41.10202215405412 73.04656588079006) bank92257 +92258 POINT(40.85343986227565 74.662286743662) bank92258 +92259 POINT(40.3970104849561 73.22744023729935) bank92259 +92260 POINT(41.17653801988309 74.95830053839731) bank92260 +92261 POINT(40.62923624752109 73.15672212805899) bank92261 +92262 POINT(40.46630037002326 74.90207640900141) bank92262 +92263 POINT(41.49456996166693 74.28528543089163) bank92263 +92264 POINT(41.37063313888234 74.93577383796674) bank92264 +92265 POINT(39.74629086447088 73.27979366099095) bank92265 +92266 POINT(40.19925149545759 74.33925012733563) bank92266 +92267 POINT(40.473809345160895 73.70370374995399) bank92267 +92268 POINT(40.02749921226956 73.31281437371436) bank92268 +92269 POINT(41.62363660165549 74.59566531886557) bank92269 +92270 POINT(39.93143163721761 73.36445586963947) bank92270 +92271 POINT(40.370239011718894 73.09947020696843) bank92271 +92272 POINT(40.73510457305635 73.57524303466053) bank92272 +92273 POINT(41.307659832320596 73.51784236820879) bank92273 +92274 POINT(41.49989295282284 74.09468248447826) bank92274 +92275 POINT(40.203859948703105 73.94810217631967) bank92275 +92276 POINT(40.305999153965274 74.85530010769122) bank92276 +92277 POINT(41.369147972075645 73.53369376181632) bank92277 +92278 POINT(41.24780494045713 74.2038797610748) bank92278 +92279 POINT(41.23200173651073 73.36769525046563) bank92279 +92280 POINT(41.61337103977736 73.11865785564272) bank92280 +92281 POINT(40.552174851814755 74.28161867634167) bank92281 +92282 POINT(39.74446432409091 73.6292374718391) bank92282 +92283 POINT(40.895422259947296 74.4038280183114) bank92283 +92284 POINT(40.04559405123362 73.1041671144082) bank92284 +92285 POINT(40.293627024160514 75.0026703490092) bank92285 +92286 POINT(41.65485663520179 73.88523605265686) bank92286 +92287 POINT(39.90171262081109 73.24994664001291) bank92287 +92288 POINT(40.59247963706493 73.42955850040565) bank92288 +92289 POINT(40.04053464333222 74.54369084378563) bank92289 +92290 POINT(40.84760375671176 74.87450856483561) bank92290 +92291 POINT(40.528136927290035 73.64304199446715) bank92291 +92292 POINT(40.909833773088245 73.29739042323021) bank92292 +92293 POINT(40.50378568221502 73.2401564566734) bank92293 +92294 POINT(41.17104700857036 73.4199611683369) bank92294 +92295 POINT(41.686542028436385 73.95151715363069) bank92295 +92296 POINT(40.198377180973985 74.77408364445967) bank92296 +92297 POINT(40.96861759173075 74.79221874771706) bank92297 +92298 POINT(40.55571876937102 74.24159952377941) bank92298 +92299 POINT(41.575860404002334 74.70282268131736) bank92299 +92300 POINT(39.88499285744601 74.273450800251) bank92300 +92301 POINT(41.08293547431949 74.44677457864678) bank92301 +92302 POINT(41.162021481492125 73.99513369025769) bank92302 +92303 POINT(41.38808889737548 74.38473146960521) bank92303 +92304 POINT(40.55015998765429 74.06989984009756) bank92304 +92305 POINT(41.213072306217015 73.55230946136658) bank92305 +92306 POINT(39.76759492917351 74.84874118988103) bank92306 +92307 POINT(41.57488388497597 73.46473139616343) bank92307 +92308 POINT(40.073951416176705 74.37405469850084) bank92308 +92309 POINT(40.84348297656854 74.44603864755405) bank92309 +92310 POINT(40.98353196215391 74.81402272521397) bank92310 +92311 POINT(39.99324206236008 73.57647615859375) bank92311 +92312 POINT(40.979821157541366 74.86801415753574) bank92312 +92313 POINT(40.155809604489356 73.48475402178059) bank92313 +92314 POINT(41.521041475239876 73.68103407061932) bank92314 +92315 POINT(41.58733331846422 74.53946499581373) bank92315 +92316 POINT(39.79952339280609 74.46396578429392) bank92316 +92317 POINT(40.55635490676661 73.97876830357683) bank92317 +92318 POINT(40.55461915903224 73.35333302171406) bank92318 +92319 POINT(39.998492108993226 74.7213779186531) bank92319 +92320 POINT(40.85081892328794 74.4492256243336) bank92320 +92321 POINT(41.50996718608913 73.36244856260093) bank92321 +92322 POINT(40.37504519240664 74.42407155361187) bank92322 +92323 POINT(41.419757163884384 74.72269597803523) bank92323 +92324 POINT(41.21640292090913 74.98022620356923) bank92324 +92325 POINT(40.75690437845579 73.3896888692181) bank92325 +92326 POINT(39.77934900571055 74.97344247835906) bank92326 +92327 POINT(41.129516443846754 74.9315262071279) bank92327 +92328 POINT(41.042879227159204 74.44574432299405) bank92328 +92329 POINT(40.684324035540584 73.28221691119255) bank92329 +92330 POINT(40.38280266982824 73.19576880015524) bank92330 +92331 POINT(40.635254605087745 74.06699333026768) bank92331 +92332 POINT(40.75615787195505 73.20137652313188) bank92332 +92333 POINT(39.90727167530888 73.60773200555764) bank92333 +92334 POINT(40.62212553876869 73.43501909372233) bank92334 +92335 POINT(40.66772124786246 74.19793760985674) bank92335 +92336 POINT(40.99373642615007 74.65916648002842) bank92336 +92337 POINT(40.35075671033523 73.45048516132678) bank92337 +92338 POINT(39.94562060898976 74.6520391271211) bank92338 +92339 POINT(40.67573061854592 74.94316259061338) bank92339 +92340 POINT(40.77675552943735 73.87370088152193) bank92340 +92341 POINT(40.78639059338205 74.8077841373536) bank92341 +92342 POINT(41.03630455538255 74.67233895848096) bank92342 +92343 POINT(40.554963654627294 74.82614321199114) bank92343 +92344 POINT(40.55862939840147 73.18141538142476) bank92344 +92345 POINT(40.16529663118759 73.95186422365231) bank92345 +92346 POINT(40.587592965709916 73.13971314116525) bank92346 +92347 POINT(40.78396866389306 73.22224421330482) bank92347 +92348 POINT(41.266629212808084 73.14757960221205) bank92348 +92349 POINT(40.310843661221895 73.12470715595326) bank92349 +92350 POINT(40.13216300119858 73.57578840479061) bank92350 +92351 POINT(40.21151354165035 74.12392284475563) bank92351 +92352 POINT(40.53122884724603 73.31293432075368) bank92352 +92353 POINT(41.19336801493068 73.31534790801939) bank92353 +92354 POINT(41.17251307055889 74.93900716461843) bank92354 +92355 POINT(41.48292518628919 73.94966556058523) bank92355 +92356 POINT(40.613801507261435 74.53114104469651) bank92356 +92357 POINT(40.34963815811649 73.95143334339056) bank92357 +92358 POINT(39.77338219095096 74.0889018336065) bank92358 +92359 POINT(41.46440206384217 74.19777603163904) bank92359 +92360 POINT(40.2497067453854 73.56638633431103) bank92360 +92361 POINT(41.58280645207648 73.74115757158557) bank92361 +92362 POINT(40.876574880091034 73.40340447064129) bank92362 +92363 POINT(40.17216763304886 73.17699280289742) bank92363 +92364 POINT(41.59108008990587 74.42696014708733) bank92364 +92365 POINT(39.940343361098954 74.66159760829895) bank92365 +92366 POINT(40.96870551019128 74.88856027288904) bank92366 +92367 POINT(41.4831064386155 73.97760835458716) bank92367 +92368 POINT(40.132220207143945 73.97802977476613) bank92368 +92369 POINT(40.65635957567407 73.6797090495147) bank92369 +92370 POINT(39.88375215789239 74.74901107885657) bank92370 +92371 POINT(40.48484100799096 73.74154338551241) bank92371 +92372 POINT(39.93369801745544 73.20718052280516) bank92372 +92373 POINT(41.31845365143939 73.99677807565182) bank92373 +92374 POINT(40.47815310350011 73.05843463756605) bank92374 +92375 POINT(41.62590258227926 73.24452487882158) bank92375 +92376 POINT(40.37998019708898 74.01705475967488) bank92376 +92377 POINT(40.7759992924137 74.33020059816499) bank92377 +92378 POINT(41.0933849707326 73.71913106662306) bank92378 +92379 POINT(40.69010322944125 73.43252936037085) bank92379 +92380 POINT(40.15477101067429 73.1796981347994) bank92380 +92381 POINT(39.823199518350364 73.66645470516237) bank92381 +92382 POINT(40.46199699725338 74.26264232300672) bank92382 +92383 POINT(41.44462546886 74.7266553516261) bank92383 +92384 POINT(40.87014520051484 74.98276535380685) bank92384 +92385 POINT(41.26585496714805 74.21903044734204) bank92385 +92386 POINT(41.54766622201532 74.11830944885475) bank92386 +92387 POINT(41.33314070936847 74.26265304894274) bank92387 +92388 POINT(39.81152113356868 73.37938490629564) bank92388 +92389 POINT(41.063343126719445 73.61277177343047) bank92389 +92390 POINT(40.02255558704928 74.80061226825991) bank92390 +92391 POINT(40.23694766506503 74.54845603521895) bank92391 +92392 POINT(40.16689633548275 74.24349658605182) bank92392 +92393 POINT(40.73858945052542 73.30562361101904) bank92393 +92394 POINT(41.62930493902937 73.11261860645196) bank92394 +92395 POINT(41.27567744414837 73.57889833779343) bank92395 +92396 POINT(41.54350123097424 73.28891113667416) bank92396 +92397 POINT(40.411911137899146 74.89259600802245) bank92397 +92398 POINT(41.2141878176481 73.85072099413736) bank92398 +92399 POINT(40.703854121174345 74.75827706725126) bank92399 +92400 POINT(41.135130930876215 74.61875702359787) bank92400 +92401 POINT(40.720101398070696 74.80386249067897) bank92401 +92402 POINT(41.45826057761427 73.48711461240312) bank92402 +92403 POINT(40.95847475241032 73.7831839162609) bank92403 +92404 POINT(39.876204614479214 74.51441907603359) bank92404 +92405 POINT(41.198734812592114 74.46622690690357) bank92405 +92406 POINT(40.06496928553654 73.17062440469392) bank92406 +92407 POINT(40.69252710744425 74.04015731212682) bank92407 +92408 POINT(41.031264539416334 74.21425508386992) bank92408 +92409 POINT(41.32939004422487 73.92358405306085) bank92409 +92410 POINT(41.321664983195475 73.36465360892824) bank92410 +92411 POINT(40.05141467636322 73.08282143595869) bank92411 +92412 POINT(41.01266794941644 73.91008030508235) bank92412 +92413 POINT(39.83936305632699 74.15396180358141) bank92413 +92414 POINT(41.12229886372458 73.99860084978829) bank92414 +92415 POINT(39.72365737621986 74.68120804173809) bank92415 +92416 POINT(39.90793735412255 73.00659554885864) bank92416 +92417 POINT(41.402198095572366 73.19853414293341) bank92417 +92418 POINT(40.92483460566147 73.5735871069293) bank92418 +92419 POINT(40.0425669955169 74.23161501603403) bank92419 +92420 POINT(40.92453728645254 73.42299359060884) bank92420 +92421 POINT(41.52117624914684 74.94414615198734) bank92421 +92422 POINT(41.336344515129284 73.96391655439946) bank92422 +92423 POINT(41.139539066637056 73.9059628448333) bank92423 +92424 POINT(41.00508474936471 73.81731867786225) bank92424 +92425 POINT(40.13347643925433 73.14080425057509) bank92425 +92426 POINT(41.102051822785164 74.55445791723501) bank92426 +92427 POINT(40.65530422153269 73.14684055987547) bank92427 +92428 POINT(41.67592209820012 74.81181729876411) bank92428 +92429 POINT(41.221923263238196 73.29248815604473) bank92429 +92430 POINT(41.1373874910321 74.99627836243647) bank92430 +92431 POINT(40.67252192193764 74.76038435971368) bank92431 +92432 POINT(41.4312694394121 74.19109354954618) bank92432 +92433 POINT(41.0413215926757 74.45693583359363) bank92433 +92434 POINT(40.745318387303314 74.83668626639327) bank92434 +92435 POINT(40.76111368689341 73.13425609431937) bank92435 +92436 POINT(41.23113470502421 74.20018566092048) bank92436 +92437 POINT(41.407138347615785 73.95436413280575) bank92437 +92438 POINT(41.648827779369 74.11231941156016) bank92438 +92439 POINT(41.55742391374332 74.29666607940145) bank92439 +92440 POINT(40.26220469702983 74.81010831495881) bank92440 +92441 POINT(41.288264603137186 73.18680607200439) bank92441 +92442 POINT(40.68311156572233 73.45744013397695) bank92442 +92443 POINT(40.86256834686863 73.78510265585578) bank92443 +92444 POINT(39.87197973556022 73.99508661463697) bank92444 +92445 POINT(40.723849251815935 73.78788253982023) bank92445 +92446 POINT(40.992866890739855 73.80114520455885) bank92446 +92447 POINT(41.194978307471466 74.86057429585749) bank92447 +92448 POINT(41.59511972232704 73.8552363580719) bank92448 +92449 POINT(41.34312153012244 73.8148851759071) bank92449 +92450 POINT(41.240970157629256 73.02553718659743) bank92450 +92451 POINT(39.73633286264155 73.0933192907658) bank92451 +92452 POINT(41.614100763585164 73.01858205732944) bank92452 +92453 POINT(41.22124533663744 73.65130895764376) bank92453 +92454 POINT(41.0466829548474 73.06923393272832) bank92454 +92455 POINT(40.74771370872083 74.25266610471844) bank92455 +92456 POINT(40.80151007333593 73.79258513853779) bank92456 +92457 POINT(41.526947282207885 73.29786415684589) bank92457 +92458 POINT(41.15775468865123 74.23915110129812) bank92458 +92459 POINT(40.138677430772375 74.65634026496895) bank92459 +92460 POINT(39.90643525624294 73.98948441996703) bank92460 +92461 POINT(41.28009949657223 74.68957575140246) bank92461 +92462 POINT(41.475077483583036 73.87164353326742) bank92462 +92463 POINT(41.387421322510605 73.34765966664465) bank92463 +92464 POINT(40.266712441058914 73.91297247277775) bank92464 +92465 POINT(40.57590016916607 73.68000660891322) bank92465 +92466 POINT(40.02743103815754 74.37000841880466) bank92466 +92467 POINT(41.38353111518475 74.85524398641392) bank92467 +92468 POINT(40.85169042561408 73.22007789322048) bank92468 +92469 POINT(39.745520826192 74.85617795532586) bank92469 +92470 POINT(41.09227764406995 73.93530574399821) bank92470 +92471 POINT(41.4777797196058 73.16839657457434) bank92471 +92472 POINT(41.617704052973956 73.07309687329207) bank92472 +92473 POINT(41.31420451665745 74.44335010259334) bank92473 +92474 POINT(40.97630087366626 74.53856005715456) bank92474 +92475 POINT(40.682953066272916 74.21058099154338) bank92475 +92476 POINT(41.40175877981164 73.95172463710257) bank92476 +92477 POINT(40.814227814338444 73.41119673289145) bank92477 +92478 POINT(40.975869549171136 74.34568659147409) bank92478 +92479 POINT(40.23018346779184 74.17821129523456) bank92479 +92480 POINT(40.40545210334818 73.32362064271479) bank92480 +92481 POINT(40.007633737688245 74.30088684919838) bank92481 +92482 POINT(40.56024082661255 74.39165902605045) bank92482 +92483 POINT(40.236977267341814 74.26510707530794) bank92483 +92484 POINT(39.90145456053574 74.72524072289204) bank92484 +92485 POINT(40.259180358494625 74.91164865847833) bank92485 +92486 POINT(39.85774904630539 74.47597478767345) bank92486 +92487 POINT(39.848488486551595 74.42928338619994) bank92487 +92488 POINT(41.56518883672555 73.20570816064638) bank92488 +92489 POINT(40.601221283679024 73.37248322470374) bank92489 +92490 POINT(40.55642727992933 74.4290901973865) bank92490 +92491 POINT(40.27100721596501 74.18449581187369) bank92491 +92492 POINT(41.405981044979924 74.83586989208476) bank92492 +92493 POINT(41.1825548852851 74.20675089404119) bank92493 +92494 POINT(41.499092213465346 73.7370303639101) bank92494 +92495 POINT(41.07871007943781 73.13346561196535) bank92495 +92496 POINT(41.48578034632067 74.42363465325373) bank92496 +92497 POINT(40.23378933015415 74.20407633117414) bank92497 +92498 POINT(40.244446221738926 74.19654401845416) bank92498 +92499 POINT(40.708018587038254 73.4321548897924) bank92499 +92500 POINT(40.99859551485742 73.10343524088061) bank92500 +92501 POINT(41.36232991965681 73.39744116971733) bank92501 +92502 POINT(41.121354431268855 74.63100140292315) bank92502 +92503 POINT(41.22647211619322 73.34826700794677) bank92503 +92504 POINT(40.45347440813735 74.65934133049379) bank92504 +92505 POINT(41.57843864370758 74.98194607132201) bank92505 +92506 POINT(40.813768002470525 73.0968673168298) bank92506 +92507 POINT(40.12585772036851 73.17085665838238) bank92507 +92508 POINT(40.066831517664795 74.07523428261943) bank92508 +92509 POINT(40.09323103499642 74.16533556676069) bank92509 +92510 POINT(41.68731950931102 73.08144342551188) bank92510 +92511 POINT(40.619599697631614 73.8906159016812) bank92511 +92512 POINT(40.528442780949035 74.9701861471665) bank92512 +92513 POINT(40.41996438046208 74.77837417539844) bank92513 +92514 POINT(40.55411651887999 73.81475858628114) bank92514 +92515 POINT(39.984531401076865 74.89532870525261) bank92515 +92516 POINT(40.51543730237375 74.10821941783902) bank92516 +92517 POINT(40.187265593436074 73.5572981475681) bank92517 +92518 POINT(40.04252818161592 74.30600267388266) bank92518 +92519 POINT(40.32240842167263 73.52542500432116) bank92519 +92520 POINT(41.19499508020872 73.47416310017063) bank92520 +92521 POINT(40.717719934341304 73.30856075053796) bank92521 +92522 POINT(40.97175053922454 73.68946825278678) bank92522 +92523 POINT(40.64709261618645 73.24003012275293) bank92523 +92524 POINT(40.96656230988377 73.44161540276937) bank92524 +92525 POINT(41.38123363040359 74.03083630618445) bank92525 +92526 POINT(39.809338971874375 73.28412677392107) bank92526 +92527 POINT(40.37456238836656 73.63800393509267) bank92527 +92528 POINT(41.54655310909885 74.42806499214831) bank92528 +92529 POINT(40.3745970488894 73.7871557911798) bank92529 +92530 POINT(40.69263282386831 74.79955613393226) bank92530 +92531 POINT(41.09540002171005 74.31943774870423) bank92531 +92532 POINT(41.365690875576824 73.94540648499364) bank92532 +92533 POINT(40.63369274018047 73.31538940336317) bank92533 +92534 POINT(40.685125941016544 74.5242345347586) bank92534 +92535 POINT(41.62484075451347 74.5024737009387) bank92535 +92536 POINT(41.71024507539028 73.93879516159701) bank92536 +92537 POINT(41.19031636396901 74.03017776858373) bank92537 +92538 POINT(40.38423150239156 73.8568021584904) bank92538 +92539 POINT(40.59304367328117 74.32316909470336) bank92539 +92540 POINT(40.42990026245205 73.06564954195169) bank92540 +92541 POINT(40.33430057454788 73.674165181346) bank92541 +92542 POINT(41.26485185437866 73.52483474510869) bank92542 +92543 POINT(40.35790534400562 74.64550359766525) bank92543 +92544 POINT(41.4903390259507 74.50654475105688) bank92544 +92545 POINT(39.85210298294959 73.97604273382876) bank92545 +92546 POINT(39.96266805481075 73.50246534261433) bank92546 +92547 POINT(40.1994005138937 74.64916863039596) bank92547 +92548 POINT(40.47656937139065 74.91259077366803) bank92548 +92549 POINT(40.06303554444815 74.96838133134958) bank92549 +92550 POINT(40.60669141204993 74.91169255273319) bank92550 +92551 POINT(40.64314868167627 74.69043914979825) bank92551 +92552 POINT(40.52881963458492 74.49622376009552) bank92552 +92553 POINT(41.418306470702134 73.21347076810724) bank92553 +92554 POINT(41.16367812521993 74.87271538105344) bank92554 +92555 POINT(39.71916385230985 73.40223337364411) bank92555 +92556 POINT(41.20573522686317 73.60878695855452) bank92556 +92557 POINT(40.870271140872504 74.1330397250696) bank92557 +92558 POINT(40.45261698548533 74.48475781223834) bank92558 +92559 POINT(41.294933790574724 73.91292792377375) bank92559 +92560 POINT(40.44924067904288 74.68374348453288) bank92560 +92561 POINT(41.274460419021715 74.77101779698687) bank92561 +92562 POINT(40.04407737352652 74.17996974673288) bank92562 +92563 POINT(39.788589215626686 74.62298142246662) bank92563 +92564 POINT(40.732857738481265 73.56418855001976) bank92564 +92565 POINT(41.20565577913215 73.46045043987738) bank92565 +92566 POINT(40.37682873807227 73.89470401447065) bank92566 +92567 POINT(40.3067588119478 73.03848531178573) bank92567 +92568 POINT(40.38552544120536 73.39878146894249) bank92568 +92569 POINT(39.7734222461968 73.17202312467386) bank92569 +92570 POINT(41.503973280162825 74.13911689974958) bank92570 +92571 POINT(41.64012981736117 74.24242362592186) bank92571 +92572 POINT(39.867056585262105 74.5596629819214) bank92572 +92573 POINT(39.87069903420894 74.84090822235487) bank92573 +92574 POINT(39.858626218187666 73.46023014814024) bank92574 +92575 POINT(41.058419295596615 73.18848046919503) bank92575 +92576 POINT(40.17926195438928 73.9004871226418) bank92576 +92577 POINT(40.98379711515881 73.08927563498878) bank92577 +92578 POINT(40.96323420860403 73.78913180505246) bank92578 +92579 POINT(39.911693984801026 73.88919623220112) bank92579 +92580 POINT(40.95237668759826 74.54024940588823) bank92580 +92581 POINT(40.057364872819015 74.87584693297035) bank92581 +92582 POINT(41.09379469260271 74.86742423205995) bank92582 +92583 POINT(41.25319339706608 74.85072780469359) bank92583 +92584 POINT(41.19137303872299 74.75074574662408) bank92584 +92585 POINT(40.66254310437409 74.327235053113) bank92585 +92586 POINT(40.34658918962868 74.6347411701407) bank92586 +92587 POINT(39.81900895407121 73.23887674757616) bank92587 +92588 POINT(40.55386954559302 74.25988587440182) bank92588 +92589 POINT(41.319448943679795 73.793032557848) bank92589 +92590 POINT(40.108812625220146 73.44510786323983) bank92590 +92591 POINT(39.82410493604946 74.2762780236226) bank92591 +92592 POINT(41.463027901674934 73.23450628134677) bank92592 +92593 POINT(41.34460023674065 74.54750582562694) bank92593 +92594 POINT(40.310726118372834 74.53018334593867) bank92594 +92595 POINT(40.527028242334424 73.01556111806684) bank92595 +92596 POINT(41.29412592066423 73.80643126211984) bank92596 +92597 POINT(40.647100353197125 73.58288499303711) bank92597 +92598 POINT(41.06964961536189 73.79508588210793) bank92598 +92599 POINT(41.117338333418765 74.21525304440364) bank92599 +92600 POINT(39.80556958593356 74.09459554975602) bank92600 +92601 POINT(39.84202215682739 73.45207163814554) bank92601 +92602 POINT(40.767470710775775 73.44202656563458) bank92602 +92603 POINT(40.235996204504325 73.74095158159152) bank92603 +92604 POINT(40.159171070242806 73.75806791040728) bank92604 +92605 POINT(40.90156007075044 73.17964949469447) bank92605 +92606 POINT(41.67891189694817 73.85236369082381) bank92606 +92607 POINT(40.67870885425523 74.53876797021893) bank92607 +92608 POINT(40.255433835805675 73.59430743043794) bank92608 +92609 POINT(41.33070692263216 74.1495922568965) bank92609 +92610 POINT(40.33207618546028 74.394277149672) bank92610 +92611 POINT(40.06981539858884 74.02678338575052) bank92611 +92612 POINT(41.278209834978476 73.96750799728497) bank92612 +92613 POINT(39.977406640376635 74.14803453890607) bank92613 +92614 POINT(40.506688597316014 73.55231374558868) bank92614 +92615 POINT(41.36458809542424 74.54785967314196) bank92615 +92616 POINT(39.83675784541689 74.26002635298624) bank92616 +92617 POINT(41.58703189204442 74.06830891876287) bank92617 +92618 POINT(40.507408386402375 74.62097730330991) bank92618 +92619 POINT(41.45487019014424 73.74691164380882) bank92619 +92620 POINT(40.280110276498675 74.73069757586879) bank92620 +92621 POINT(40.31202136727489 74.95549516058908) bank92621 +92622 POINT(41.66332841285715 74.8929781304477) bank92622 +92623 POINT(40.325643439763894 73.3342036564735) bank92623 +92624 POINT(40.02704337764964 73.9725202573379) bank92624 +92625 POINT(41.32898123598824 74.3409030465273) bank92625 +92626 POINT(41.35120618160394 73.52877306397325) bank92626 +92627 POINT(40.1108473715757 74.7205394125434) bank92627 +92628 POINT(40.64802145269865 73.0960011987406) bank92628 +92629 POINT(41.02376642735831 74.0973558427434) bank92629 +92630 POINT(41.041965703646795 73.44036280761179) bank92630 +92631 POINT(40.5628063841699 73.98541258941185) bank92631 +92632 POINT(40.041923002471336 73.63527300887202) bank92632 +92633 POINT(41.65086984776684 74.17142349471554) bank92633 +92634 POINT(40.853782035748296 73.82619001324244) bank92634 +92635 POINT(40.931711812531326 74.88516640983778) bank92635 +92636 POINT(40.360495602187505 73.57380919507548) bank92636 +92637 POINT(41.421134686426804 74.26458838976691) bank92637 +92638 POINT(40.66049748337454 74.01830906746717) bank92638 +92639 POINT(41.57451431710901 74.18243976537511) bank92639 +92640 POINT(40.49232919355729 74.2613627282995) bank92640 +92641 POINT(40.0841839724174 74.51600589140793) bank92641 +92642 POINT(41.51070399414827 73.98837539450416) bank92642 +92643 POINT(40.41221142640079 73.90125202875117) bank92643 +92644 POINT(39.78055161308867 73.95177616908043) bank92644 +92645 POINT(40.27346881568732 74.86296734134162) bank92645 +92646 POINT(41.223353682821525 73.33988447588547) bank92646 +92647 POINT(41.435751980895446 73.87365463443496) bank92647 +92648 POINT(40.46659208678368 74.1591351620083) bank92648 +92649 POINT(41.703959212501445 74.23452634813505) bank92649 +92650 POINT(41.08670809551445 73.07151244064863) bank92650 +92651 POINT(40.8249212291065 74.5106531773638) bank92651 +92652 POINT(40.30512272246298 73.11507958085006) bank92652 +92653 POINT(39.930623771165564 73.01413435588988) bank92653 +92654 POINT(40.431420191330254 73.65581082834156) bank92654 +92655 POINT(40.53618458210241 74.37343860731346) bank92655 +92656 POINT(40.26592296785248 73.07320744291208) bank92656 +92657 POINT(41.026013245971754 73.8897623494699) bank92657 +92658 POINT(39.9890072845902 73.8381053578867) bank92658 +92659 POINT(40.09523816736043 73.04826880493758) bank92659 +92660 POINT(40.31446564735398 73.29558456971697) bank92660 +92661 POINT(41.16023322080875 73.46609088347105) bank92661 +92662 POINT(40.605705649130854 73.88845835915036) bank92662 +92663 POINT(40.592567403936805 74.03502762208139) bank92663 +92664 POINT(40.27481668112553 73.34942025360998) bank92664 +92665 POINT(40.179281425572285 74.32482651313401) bank92665 +92666 POINT(40.03027182790274 73.77759511723323) bank92666 +92667 POINT(41.00985058792991 74.31811463210745) bank92667 +92668 POINT(39.75197789325387 73.85105111746435) bank92668 +92669 POINT(40.169813592822706 73.46976633406284) bank92669 +92670 POINT(39.782555094718205 73.46149701467351) bank92670 +92671 POINT(41.25782424324168 73.35645997352076) bank92671 +92672 POINT(40.90045275252096 74.68480545038491) bank92672 +92673 POINT(40.7137503391684 74.34032557631514) bank92673 +92674 POINT(40.81051069878752 74.8152682321135) bank92674 +92675 POINT(41.12202673800084 73.09259535516875) bank92675 +92676 POINT(40.873578070952014 73.28860722579067) bank92676 +92677 POINT(40.594733136072854 74.43670652890965) bank92677 +92678 POINT(40.62295802392142 73.17466599099106) bank92678 +92679 POINT(41.03755884010894 73.88108675127903) bank92679 +92680 POINT(40.21630435031308 74.41634992263539) bank92680 +92681 POINT(40.19991943237622 73.20560600763814) bank92681 +92682 POINT(40.520487687066144 73.73593448407482) bank92682 +92683 POINT(40.84762273976441 74.40282755574675) bank92683 +92684 POINT(41.27272107013274 73.12615575281501) bank92684 +92685 POINT(40.25786416772032 73.71772841995917) bank92685 +92686 POINT(40.12277602438342 74.0760637421633) bank92686 +92687 POINT(41.28199686983419 73.58952541879314) bank92687 +92688 POINT(40.28327681844884 74.28652214940426) bank92688 +92689 POINT(41.30068973784144 73.54555205317133) bank92689 +92690 POINT(39.93163903766101 74.44176082038472) bank92690 +92691 POINT(40.752153239762 74.92770979153151) bank92691 +92692 POINT(40.53615762564826 73.41155469019614) bank92692 +92693 POINT(40.36462130715141 74.52141259711627) bank92693 +92694 POINT(39.970150759391984 74.6890296971728) bank92694 +92695 POINT(41.20462872988916 74.51199211000703) bank92695 +92696 POINT(40.608633149639076 74.6430839252671) bank92696 +92697 POINT(39.89021684624582 73.64508927238367) bank92697 +92698 POINT(40.05253390639972 74.88525248137971) bank92698 +92699 POINT(40.40465203606254 73.87114928785059) bank92699 +92700 POINT(41.165547225825065 73.6196371110121) bank92700 +92701 POINT(40.28652526126086 74.64120972741817) bank92701 +92702 POINT(41.41372764286184 74.4544691692255) bank92702 +92703 POINT(41.26744507754475 74.20087310569127) bank92703 +92704 POINT(41.02680387924162 73.43683910548707) bank92704 +92705 POINT(39.817949778032194 73.13293225830664) bank92705 +92706 POINT(39.77814038581772 73.55955575170658) bank92706 +92707 POINT(40.11985463783809 73.70784394569385) bank92707 +92708 POINT(41.378065892442706 73.99101342825136) bank92708 +92709 POINT(40.162633716130955 74.9836204425746) bank92709 +92710 POINT(40.337423304878875 74.64782748958496) bank92710 +92711 POINT(41.65055961552135 74.5200553705981) bank92711 +92712 POINT(41.309444704283386 73.73794976757699) bank92712 +92713 POINT(41.440234270280165 74.788589006461) bank92713 +92714 POINT(40.67977806103881 73.63883000794571) bank92714 +92715 POINT(40.984823664916135 73.46829859350093) bank92715 +92716 POINT(41.59291744449379 73.54918502870211) bank92716 +92717 POINT(41.703522480532406 74.39007773903352) bank92717 +92718 POINT(40.54536821449764 73.80621249706856) bank92718 +92719 POINT(40.15024300162716 73.36204759582492) bank92719 +92720 POINT(41.641832886321744 74.36261410245618) bank92720 +92721 POINT(40.87313855942063 73.97802325094342) bank92721 +92722 POINT(41.22563335681994 74.67466837732856) bank92722 +92723 POINT(40.98246825179941 74.46478632231468) bank92723 +92724 POINT(40.058270319672026 74.83114552804747) bank92724 +92725 POINT(39.786925774766296 73.69240111167915) bank92725 +92726 POINT(41.23403815595271 74.5621900406159) bank92726 +92727 POINT(40.822730594297106 73.58945515136942) bank92727 +92728 POINT(40.592003055775166 73.8058853351628) bank92728 +92729 POINT(41.396043975454916 73.46776790992017) bank92729 +92730 POINT(40.10180633502709 74.54330680829631) bank92730 +92731 POINT(41.50948563060931 74.5282244339554) bank92731 +92732 POINT(39.938727469295934 74.58833139003559) bank92732 +92733 POINT(41.26940446062732 73.17425372233443) bank92733 +92734 POINT(40.93936012890942 73.70567016975214) bank92734 +92735 POINT(39.76682594085908 74.8178005001081) bank92735 +92736 POINT(40.62482917089764 73.715124586902) bank92736 +92737 POINT(41.49398736718363 73.1872408341769) bank92737 +92738 POINT(41.219772759235795 73.09409970427298) bank92738 +92739 POINT(41.163912377497525 73.27671740617662) bank92739 +92740 POINT(41.6101878015901 74.30637623409613) bank92740 +92741 POINT(40.08040942709572 74.22948382702894) bank92741 +92742 POINT(40.26588209301939 73.20554887245744) bank92742 +92743 POINT(40.22826645646628 73.71039920547035) bank92743 +92744 POINT(40.60826100486015 74.68807116437596) bank92744 +92745 POINT(39.93844667657146 74.92258471051899) bank92745 +92746 POINT(40.533432586562895 73.69300541108446) bank92746 +92747 POINT(40.391925161213486 74.78082215893878) bank92747 +92748 POINT(40.2278642903119 73.44040126411079) bank92748 +92749 POINT(39.750521486958014 73.96420993567428) bank92749 +92750 POINT(40.67889108511917 73.8717988893528) bank92750 +92751 POINT(40.071013469295366 73.78010216487105) bank92751 +92752 POINT(39.86051850534596 73.0953154828123) bank92752 +92753 POINT(40.83868255492884 73.96528362549014) bank92753 +92754 POINT(40.26823352101439 74.67081871720981) bank92754 +92755 POINT(41.67671004136796 73.5444159728621) bank92755 +92756 POINT(39.79619552620259 74.57264705859696) bank92756 +92757 POINT(39.81697005054065 74.02401155735727) bank92757 +92758 POINT(40.4805594470457 74.56546613844333) bank92758 +92759 POINT(40.615841402449675 73.63401606360162) bank92759 +92760 POINT(41.628455850680986 74.0331574746396) bank92760 +92761 POINT(39.98142258523475 74.13675675291815) bank92761 +92762 POINT(40.65018595045315 73.41811487563176) bank92762 +92763 POINT(40.2300240333095 74.43729939939911) bank92763 +92764 POINT(39.93359239357547 73.26586384402536) bank92764 +92765 POINT(41.28807819760575 73.01431175909048) bank92765 +92766 POINT(41.50292791039809 73.11841115258815) bank92766 +92767 POINT(40.32080105709868 74.37988287148373) bank92767 +92768 POINT(41.176843942978316 74.80001797640455) bank92768 +92769 POINT(40.397468925009576 74.16208483837018) bank92769 +92770 POINT(40.00129418457636 74.75229279394962) bank92770 +92771 POINT(40.896286465162106 73.30554036096831) bank92771 +92772 POINT(40.66268288218607 73.49752089464165) bank92772 +92773 POINT(40.809058216216705 73.87785438873429) bank92773 +92774 POINT(40.72595223971749 73.3265382206187) bank92774 +92775 POINT(41.09966626101651 73.58019156503754) bank92775 +92776 POINT(40.151096040300814 74.11701875452455) bank92776 +92777 POINT(40.010330081370746 73.16345735594356) bank92777 +92778 POINT(40.942656984521975 73.63113059558287) bank92778 +92779 POINT(40.272834704565426 73.763587696057) bank92779 +92780 POINT(39.86720491160996 73.8444994265224) bank92780 +92781 POINT(40.70939893316918 74.25151748000789) bank92781 +92782 POINT(40.62320647085403 74.44665073147975) bank92782 +92783 POINT(39.93343239209674 73.31587112390966) bank92783 +92784 POINT(39.84740784780687 73.25186807392171) bank92784 +92785 POINT(40.05192091659485 74.26688950976725) bank92785 +92786 POINT(40.72286967605698 73.72525795427026) bank92786 +92787 POINT(41.13756195724007 73.79081304008596) bank92787 +92788 POINT(40.19601634903447 74.16049469734776) bank92788 +92789 POINT(40.9034028200758 74.7273489861164) bank92789 +92790 POINT(40.95993572080886 74.34287228681463) bank92790 +92791 POINT(39.973726144472415 74.38686742077611) bank92791 +92792 POINT(40.29653993278606 74.87125723397155) bank92792 +92793 POINT(40.08045249453063 73.55989802687316) bank92793 +92794 POINT(40.61008289057053 73.98322394171497) bank92794 +92795 POINT(41.10565327860881 74.00393730496685) bank92795 +92796 POINT(41.523890406626464 74.13130331259332) bank92796 +92797 POINT(40.94241894489934 73.2021871933829) bank92797 +92798 POINT(40.29251527193815 74.90376709309709) bank92798 +92799 POINT(40.092301242571665 73.76618611407432) bank92799 +92800 POINT(41.28014698542913 74.89516086735979) bank92800 +92801 POINT(40.298054112219546 73.69898624480237) bank92801 +92802 POINT(40.959193848203846 73.39385705735886) bank92802 +92803 POINT(41.40550469460659 74.97491934907819) bank92803 +92804 POINT(40.89145135173209 74.40201556615612) bank92804 +92805 POINT(40.387228583735734 73.03928833265174) bank92805 +92806 POINT(41.11246031714999 74.7201651708118) bank92806 +92807 POINT(41.376525281655624 73.96948632028597) bank92807 +92808 POINT(41.584354032478615 73.43417409862107) bank92808 +92809 POINT(41.073697533257345 73.74515139804959) bank92809 +92810 POINT(41.21480679484635 74.61818864004016) bank92810 +92811 POINT(41.591540317652196 73.28287885678465) bank92811 +92812 POINT(39.9105780654622 74.11465925841503) bank92812 +92813 POINT(41.10008233417509 73.1272948399347) bank92813 +92814 POINT(40.238785647668564 74.25517242490542) bank92814 +92815 POINT(41.61377122933194 73.49127834489063) bank92815 +92816 POINT(39.74988335345018 74.81999782925722) bank92816 +92817 POINT(40.04326160348519 74.54333768717618) bank92817 +92818 POINT(41.49344716171301 73.54360781511937) bank92818 +92819 POINT(40.95741112933405 74.88167399500712) bank92819 +92820 POINT(40.41712562196556 74.10440680156717) bank92820 +92821 POINT(40.411989317473484 74.10203748363614) bank92821 +92822 POINT(39.94200912728741 74.77728006505008) bank92822 +92823 POINT(40.50986757535832 73.36778962220086) bank92823 +92824 POINT(40.58302658954805 73.71118150163588) bank92824 +92825 POINT(40.08834918616283 73.32368673751297) bank92825 +92826 POINT(40.171228990075896 73.04888909639226) bank92826 +92827 POINT(40.60027358341115 73.79219707032259) bank92827 +92828 POINT(41.263011410600406 73.43518451836846) bank92828 +92829 POINT(41.59712507711613 73.99769617331661) bank92829 +92830 POINT(40.04616039368259 73.99076859877778) bank92830 +92831 POINT(40.92831559863082 74.86553667667157) bank92831 +92832 POINT(40.00752032542885 73.18053568753076) bank92832 +92833 POINT(41.13560877751502 74.60959353801779) bank92833 +92834 POINT(39.83864702687501 74.9625286459822) bank92834 +92835 POINT(41.14634153334881 73.41868048869588) bank92835 +92836 POINT(39.80234908598507 73.38595409914619) bank92836 +92837 POINT(41.694175883658424 73.83018247852402) bank92837 +92838 POINT(40.58218760600184 74.00208222625679) bank92838 +92839 POINT(39.99658425536053 74.94906100701358) bank92839 +92840 POINT(41.338103321683434 74.31938723311514) bank92840 +92841 POINT(41.35445194129365 73.97876984559748) bank92841 +92842 POINT(40.111923884847656 73.20105807442538) bank92842 +92843 POINT(40.90536490794724 74.11307016207897) bank92843 +92844 POINT(40.081088733715646 74.23434498779686) bank92844 +92845 POINT(39.9728629361095 74.40446836709503) bank92845 +92846 POINT(41.02636451728767 73.36104324577951) bank92846 +92847 POINT(39.9151494184119 74.64838671475167) bank92847 +92848 POINT(41.35362505183312 73.22492836501065) bank92848 +92849 POINT(40.043462885591254 74.1544217669473) bank92849 +92850 POINT(39.87905324678913 73.83605449876319) bank92850 +92851 POINT(40.54967088594292 73.24904749745811) bank92851 +92852 POINT(40.24576289360293 74.91844067733723) bank92852 +92853 POINT(41.29145068300624 74.06886424851287) bank92853 +92854 POINT(40.736114966965744 74.06898248367305) bank92854 +92855 POINT(41.71064851475878 74.6967120361911) bank92855 +92856 POINT(41.41333370848887 73.12034704903876) bank92856 +92857 POINT(40.54838877348419 73.60076828345855) bank92857 +92858 POINT(41.22326138548229 74.49305099280001) bank92858 +92859 POINT(40.50552724479718 73.92617086594943) bank92859 +92860 POINT(40.13817208858015 74.67257607440729) bank92860 +92861 POINT(41.141109678513644 73.3487795527268) bank92861 +92862 POINT(40.147815322842234 73.10853777164778) bank92862 +92863 POINT(41.649302667374144 73.73914108525025) bank92863 +92864 POINT(40.89493590405611 73.59461540414841) bank92864 +92865 POINT(40.86984257491341 73.31894807895068) bank92865 +92866 POINT(41.67262789114023 73.55356916739801) bank92866 +92867 POINT(41.442035531376966 74.53357681065384) bank92867 +92868 POINT(39.85214741042124 73.06860296254824) bank92868 +92869 POINT(40.24145548438405 74.22023355272285) bank92869 +92870 POINT(40.56043077588463 73.05808434404967) bank92870 +92871 POINT(39.92287637914497 74.1724624445109) bank92871 +92872 POINT(40.31943275013927 73.19325077293935) bank92872 +92873 POINT(40.80399084639018 74.75002499937051) bank92873 +92874 POINT(41.44341429895318 73.10124479703784) bank92874 +92875 POINT(40.12971024481333 73.97395047589927) bank92875 +92876 POINT(41.08751433019259 74.7599799109951) bank92876 +92877 POINT(41.47986913740526 74.04760722689628) bank92877 +92878 POINT(40.48000297690398 73.44983902282496) bank92878 +92879 POINT(39.93679956299762 75.00585875022281) bank92879 +92880 POINT(41.54869035303525 73.61513825886361) bank92880 +92881 POINT(41.505011525176606 74.35626635611601) bank92881 +92882 POINT(41.42748496644389 74.59684841055628) bank92882 +92883 POINT(39.82511967919295 74.84374209041891) bank92883 +92884 POINT(41.58139235650604 73.39526080926947) bank92884 +92885 POINT(40.14674649820342 74.68113707109931) bank92885 +92886 POINT(39.829926394025854 74.98169042326008) bank92886 +92887 POINT(39.90207209804202 73.38960777209643) bank92887 +92888 POINT(41.21079021379728 73.6405273339024) bank92888 +92889 POINT(40.829964708189145 73.1317661863403) bank92889 +92890 POINT(41.561614325640654 74.17854024259539) bank92890 +92891 POINT(41.0903486325157 73.04266916659626) bank92891 +92892 POINT(40.83634692617277 73.44014670180815) bank92892 +92893 POINT(41.67048140891722 73.65045687202333) bank92893 +92894 POINT(39.71437066362615 73.91743535925717) bank92894 +92895 POINT(40.806324218609475 73.87418783950963) bank92895 +92896 POINT(41.36890599883548 74.69949805232285) bank92896 +92897 POINT(40.243638786641895 73.37151017587944) bank92897 +92898 POINT(41.46084100366357 74.21724827324067) bank92898 +92899 POINT(41.24223233123044 73.41408111783085) bank92899 +92900 POINT(40.54623306107245 74.91870230332424) bank92900 +92901 POINT(40.517525816173205 74.41100095685204) bank92901 +92902 POINT(41.06183844793167 73.62048977470909) bank92902 +92903 POINT(40.629566308734425 73.2620452238919) bank92903 +92904 POINT(40.3217519162905 73.32517312371868) bank92904 +92905 POINT(40.74544062468865 74.63056139857187) bank92905 +92906 POINT(41.51448063504959 73.29037123324633) bank92906 +92907 POINT(41.46593605909954 74.2142883540955) bank92907 +92908 POINT(40.261613719831594 74.13973605484048) bank92908 +92909 POINT(40.39865682017925 73.5986121652118) bank92909 +92910 POINT(40.39621478535868 74.83374577212417) bank92910 +92911 POINT(40.18831876868148 74.43508639324808) bank92911 +92912 POINT(40.02480226279169 73.96796522263305) bank92912 +92913 POINT(40.939561326825775 74.26716373619972) bank92913 +92914 POINT(40.59974100299511 73.42298057537764) bank92914 +92915 POINT(40.85574025708724 74.22721895563693) bank92915 +92916 POINT(40.908145099882475 73.96018255570165) bank92916 +92917 POINT(40.976269513893065 73.7389570003558) bank92917 +92918 POINT(40.5682677636024 74.11531870813558) bank92918 +92919 POINT(39.84609710832537 73.33784507927953) bank92919 +92920 POINT(41.44375470874077 74.76139653643023) bank92920 +92921 POINT(41.3329075031169 73.30662704087794) bank92921 +92922 POINT(40.20937291475343 73.15730570252391) bank92922 +92923 POINT(40.918553621394146 73.40740292021161) bank92923 +92924 POINT(40.390389334794754 73.24006487058385) bank92924 +92925 POINT(41.37459402035822 73.5312314273762) bank92925 +92926 POINT(40.22900746086713 74.22967863525649) bank92926 +92927 POINT(40.87249260702475 73.42155344472168) bank92927 +92928 POINT(41.51462141601406 73.38204780587314) bank92928 +92929 POINT(39.73241990201742 74.91910869954678) bank92929 +92930 POINT(39.94701499415345 74.69644484916365) bank92930 +92931 POINT(40.708726886151204 73.3719187838441) bank92931 +92932 POINT(40.3192517581591 74.53915437122889) bank92932 +92933 POINT(40.35951139343388 73.54053025265185) bank92933 +92934 POINT(40.90751979568613 73.77272821529246) bank92934 +92935 POINT(39.73248870815945 74.98553930394147) bank92935 +92936 POINT(39.82583823023158 73.25918203624084) bank92936 +92937 POINT(40.02792050472105 74.31034306237926) bank92937 +92938 POINT(40.0063249519471 74.21003319356967) bank92938 +92939 POINT(41.563868524813856 73.90660733133664) bank92939 +92940 POINT(41.11313792588723 73.9462346414675) bank92940 +92941 POINT(41.27899591487029 73.26063100289306) bank92941 +92942 POINT(40.969367514620565 74.7060135564846) bank92942 +92943 POINT(40.68018817477456 74.21164599598114) bank92943 +92944 POINT(40.114935964005014 73.0706692376552) bank92944 +92945 POINT(40.01536872939768 73.82080505081204) bank92945 +92946 POINT(40.12720681997508 74.34028312109497) bank92946 +92947 POINT(40.056185680428285 73.65342811329828) bank92947 +92948 POINT(40.7204487983856 74.43233816940896) bank92948 +92949 POINT(40.86456548672749 74.63856499507739) bank92949 +92950 POINT(41.17071371311931 73.79268836771776) bank92950 +92951 POINT(39.86543358434813 74.99221468412593) bank92951 +92952 POINT(40.58830908710267 73.74299724236656) bank92952 +92953 POINT(41.135075297617334 74.20342885308195) bank92953 +92954 POINT(39.805076374025205 74.33754732755288) bank92954 +92955 POINT(40.04447506308143 73.55857176134856) bank92955 +92956 POINT(40.1778785050313 73.13955805950852) bank92956 +92957 POINT(39.9657070643116 73.82634265419485) bank92957 +92958 POINT(40.170308343109255 74.16442427240882) bank92958 +92959 POINT(39.73232259528569 74.51191682781509) bank92959 +92960 POINT(41.088123053830714 73.21675773337068) bank92960 +92961 POINT(40.04106474437765 74.21306497848069) bank92961 +92962 POINT(40.4328670877428 73.25830631552631) bank92962 +92963 POINT(40.60886501980098 73.5128459994267) bank92963 +92964 POINT(41.43116397935239 74.32701588773399) bank92964 +92965 POINT(40.82177793800334 74.96017517429726) bank92965 +92966 POINT(41.03958457865608 73.68052702977887) bank92966 +92967 POINT(39.91979457289401 74.00683929024584) bank92967 +92968 POINT(41.58511652788597 74.44660087099437) bank92968 +92969 POINT(40.20787549617585 74.36795016645608) bank92969 +92970 POINT(40.0874927107556 73.87845769026099) bank92970 +92971 POINT(41.63462051218363 73.15132433394893) bank92971 +92972 POINT(40.37164519276697 73.51090913783294) bank92972 +92973 POINT(40.29386115994953 73.14378593053324) bank92973 +92974 POINT(40.01879671242712 73.21083263556024) bank92974 +92975 POINT(41.25634986541436 74.94887682754047) bank92975 +92976 POINT(41.31657731204858 74.16543797620774) bank92976 +92977 POINT(40.23583552781017 74.374103164886) bank92977 +92978 POINT(41.47508893358037 73.48998415713433) bank92978 +92979 POINT(41.38858301274792 74.53462121995851) bank92979 +92980 POINT(41.080762675266875 74.81951017635802) bank92980 +92981 POINT(41.26935073184622 75.00255042476344) bank92981 +92982 POINT(40.58482187483095 73.90554274893938) bank92982 +92983 POINT(40.5013092246566 73.00840196236958) bank92983 +92984 POINT(39.96492568989868 73.78409923769726) bank92984 +92985 POINT(41.130376022377014 73.89815964772723) bank92985 +92986 POINT(40.94599969837255 74.78642098520277) bank92986 +92987 POINT(41.66288192540726 73.22006541446932) bank92987 +92988 POINT(39.9754558445984 73.46062154656329) bank92988 +92989 POINT(40.82996880498535 74.0427119296861) bank92989 +92990 POINT(41.667863223036896 73.9505123121622) bank92990 +92991 POINT(39.823651867096025 74.49454059954823) bank92991 +92992 POINT(39.7610687880794 74.57003575732632) bank92992 +92993 POINT(41.334471059335435 74.86444298153546) bank92993 +92994 POINT(40.440330381023045 73.43795822682289) bank92994 +92995 POINT(40.76421787226612 74.03139956420443) bank92995 +92996 POINT(41.228421635799045 73.22094575753047) bank92996 +92997 POINT(41.58150633485316 74.73544679379782) bank92997 +92998 POINT(39.89593389278387 73.41334961945105) bank92998 +92999 POINT(41.07089475834725 74.98026679479317) bank92999 +93000 POINT(40.6815271979581 73.73232351451692) bank93000 +93001 POINT(41.66640980715955 74.11999934636741) bank93001 +93002 POINT(39.829831174510915 73.31763063804921) bank93002 +93003 POINT(41.032977232484185 73.59734531935895) bank93003 +93004 POINT(41.005008806753835 74.72392444151181) bank93004 +93005 POINT(40.095315495110135 73.47206997180649) bank93005 +93006 POINT(41.65260666953732 73.82186107133644) bank93006 +93007 POINT(40.32939388817626 73.2943248066209) bank93007 +93008 POINT(41.30228424522354 73.91231134527408) bank93008 +93009 POINT(41.27105364569938 73.18491326742458) bank93009 +93010 POINT(40.87135412321322 73.32874334859176) bank93010 +93011 POINT(40.99414725457569 74.14531716345678) bank93011 +93012 POINT(39.922072917520914 73.39865287499349) bank93012 +93013 POINT(40.867067516375556 74.84630902839963) bank93013 +93014 POINT(39.884685746288156 73.24125633736953) bank93014 +93015 POINT(41.50958025904728 74.08588896531951) bank93015 +93016 POINT(40.30329750632793 74.28483933011773) bank93016 +93017 POINT(40.3358425596826 74.93119458635269) bank93017 +93018 POINT(40.63189207713278 73.69562835045194) bank93018 +93019 POINT(40.49611759476615 74.08524456557458) bank93019 +93020 POINT(39.85490141618559 73.3844910210502) bank93020 +93021 POINT(40.29071818176799 73.95465164095212) bank93021 +93022 POINT(39.73585566888867 74.49929860245392) bank93022 +93023 POINT(40.00783740299667 73.03418475444673) bank93023 +93024 POINT(40.573704180994724 73.51786797401643) bank93024 +93025 POINT(40.05003802347257 74.97229229230136) bank93025 +93026 POINT(40.89786510134039 74.66953311056945) bank93026 +93027 POINT(41.38347676987527 74.43444544309841) bank93027 +93028 POINT(40.468225828706565 73.12530160552627) bank93028 +93029 POINT(40.14921815949294 73.18881161492614) bank93029 +93030 POINT(40.956537969757754 73.65055898856936) bank93030 +93031 POINT(41.18977701527971 73.89810621577801) bank93031 +93032 POINT(41.37347017836247 74.09873242908083) bank93032 +93033 POINT(41.363747907222496 74.42776843128627) bank93033 +93034 POINT(40.35015231918685 73.37050483428094) bank93034 +93035 POINT(39.85072454578734 73.90470341662844) bank93035 +93036 POINT(40.458724732346184 73.94538956269199) bank93036 +93037 POINT(41.388661989944474 74.78894042147574) bank93037 +93038 POINT(40.06589784693363 74.25547942150074) bank93038 +93039 POINT(40.833393266709294 74.56681448898215) bank93039 +93040 POINT(40.56209013026228 73.04674295760248) bank93040 +93041 POINT(41.360367242043104 73.21913648837194) bank93041 +93042 POINT(40.38585852673838 74.38650608309554) bank93042 +93043 POINT(41.56197930610495 73.26502719708843) bank93043 +93044 POINT(40.45026486507189 74.42886093913876) bank93044 +93045 POINT(41.518869682272495 74.11671690340211) bank93045 +93046 POINT(41.00469660146601 74.35266601322468) bank93046 +93047 POINT(41.48018173075705 73.85559981413549) bank93047 +93048 POINT(40.07508224507882 73.66872149175272) bank93048 +93049 POINT(41.14621775416762 74.71193922026112) bank93049 +93050 POINT(40.13019580311136 73.47677299859168) bank93050 +93051 POINT(41.184999447168984 74.54434232291193) bank93051 +93052 POINT(40.79917790179759 73.99232353340906) bank93052 +93053 POINT(41.14482178746198 73.83513853164344) bank93053 +93054 POINT(41.57412267998203 74.3631696190611) bank93054 +93055 POINT(40.01064611410502 73.93127042022775) bank93055 +93056 POINT(40.99606793211172 73.98309990971431) bank93056 +93057 POINT(41.325399276876965 73.20774819595805) bank93057 +93058 POINT(40.488158817043356 74.3858295582027) bank93058 +93059 POINT(40.07187307981873 73.15393814149495) bank93059 +93060 POINT(40.581771001102766 74.1896600953152) bank93060 +93061 POINT(40.011013515496266 73.2266537679232) bank93061 +93062 POINT(40.456768549166235 73.48658498057172) bank93062 +93063 POINT(40.869272278531476 73.48120764132591) bank93063 +93064 POINT(40.25753971077512 74.83979802100183) bank93064 +93065 POINT(41.674833548144505 74.46936702855893) bank93065 +93066 POINT(41.37703638775368 74.81028529502665) bank93066 +93067 POINT(41.19432861361103 74.99180753026849) bank93067 +93068 POINT(40.69325049025783 74.66836499042365) bank93068 +93069 POINT(40.22445406564733 74.06092832487872) bank93069 +93070 POINT(40.43079642778879 74.33012805060837) bank93070 +93071 POINT(40.812759219622066 74.40070152226232) bank93071 +93072 POINT(40.86050007840642 74.07313271099558) bank93072 +93073 POINT(41.531061107959125 74.7191805968154) bank93073 +93074 POINT(40.22224034663097 73.57651981297863) bank93074 +93075 POINT(39.94800120069626 74.5087507634833) bank93075 +93076 POINT(41.164009113838226 74.59018285102876) bank93076 +93077 POINT(39.73918444070556 73.15830905900671) bank93077 +93078 POINT(40.08192363603589 74.86231616443453) bank93078 +93079 POINT(40.96421552945037 74.00227778489514) bank93079 +93080 POINT(41.56488364477174 73.43019698033876) bank93080 +93081 POINT(41.379310470457185 74.30703604669013) bank93081 +93082 POINT(40.09360153367443 74.36126509029069) bank93082 +93083 POINT(39.933719886132245 74.80374956140965) bank93083 +93084 POINT(40.00781917266339 73.40433239268181) bank93084 +93085 POINT(39.99992031422071 73.98788653843994) bank93085 +93086 POINT(39.720132263051696 74.60723405695026) bank93086 +93087 POINT(40.127088951186145 73.91214096861714) bank93087 +93088 POINT(40.72781392907548 73.4510758819941) bank93088 +93089 POINT(40.65682105959754 73.02806673808334) bank93089 +93090 POINT(40.193903313363 73.05657700448411) bank93090 +93091 POINT(39.93864537030595 73.2495224822544) bank93091 +93092 POINT(40.272968429920546 74.62073646801862) bank93092 +93093 POINT(40.55750287167165 73.69394600148348) bank93093 +93094 POINT(40.3529211195718 74.62328647783094) bank93094 +93095 POINT(40.221791530999965 73.19507481666245) bank93095 +93096 POINT(40.42927658101271 73.59905579783688) bank93096 +93097 POINT(40.645313416931785 73.36805627987601) bank93097 +93098 POINT(39.98509282063907 74.20687645535872) bank93098 +93099 POINT(41.64087885638977 74.8917008317144) bank93099 +93100 POINT(41.23862782761248 73.70784514730812) bank93100 +93101 POINT(41.29712914803997 73.53718782040734) bank93101 +93102 POINT(40.03933733366218 73.25863573189666) bank93102 +93103 POINT(40.97785773818645 74.64901134380163) bank93103 +93104 POINT(41.34396788820199 73.42198211184039) bank93104 +93105 POINT(40.41804803922643 73.4121857049386) bank93105 +93106 POINT(40.93882560456274 73.45522087071039) bank93106 +93107 POINT(41.57760688115507 73.45625208078633) bank93107 +93108 POINT(40.18247511939583 74.16198543260653) bank93108 +93109 POINT(41.49226876175992 73.7860908606286) bank93109 +93110 POINT(41.07096902391475 73.46031934567633) bank93110 +93111 POINT(41.290044892068444 73.0998681845304) bank93111 +93112 POINT(40.5559419106288 73.42691721982968) bank93112 +93113 POINT(41.59701384248201 74.88133654215524) bank93113 +93114 POINT(41.028428893367035 73.11188589704011) bank93114 +93115 POINT(40.483799851271876 74.43093659985095) bank93115 +93116 POINT(40.08535762748609 74.59365393307426) bank93116 +93117 POINT(39.82306271960068 73.45732272754836) bank93117 +93118 POINT(40.97040866294863 73.1696871938557) bank93118 +93119 POINT(40.484406930805314 73.4980487914884) bank93119 +93120 POINT(40.61340252893389 74.40287375639792) bank93120 +93121 POINT(41.68541372048378 73.02880954329302) bank93121 +93122 POINT(40.68307634740735 74.47452058871608) bank93122 +93123 POINT(40.06969157409748 74.75396093350176) bank93123 +93124 POINT(40.09904919987443 74.31641372867655) bank93124 +93125 POINT(40.72989335622425 74.72812312859772) bank93125 +93126 POINT(40.73554237117002 73.35518047505087) bank93126 +93127 POINT(41.30816066347759 73.19289343958373) bank93127 +93128 POINT(39.803770625686155 74.1120740083137) bank93128 +93129 POINT(41.15192570661216 73.94123337949142) bank93129 +93130 POINT(40.56184418953631 73.8833077328405) bank93130 +93131 POINT(39.798165228982064 74.43672317272907) bank93131 +93132 POINT(41.60793957483324 73.65938189576629) bank93132 +93133 POINT(41.397106175154825 74.2820690547162) bank93133 +93134 POINT(41.6181434892124 74.81548752741824) bank93134 +93135 POINT(40.81721658699918 74.55011201466779) bank93135 +93136 POINT(39.932751413585706 74.39845837570745) bank93136 +93137 POINT(41.10183862205284 74.98084776428811) bank93137 +93138 POINT(40.434004832152844 74.13825575909344) bank93138 +93139 POINT(40.310031707371444 74.37919602995589) bank93139 +93140 POINT(40.419103750068516 73.04358769031643) bank93140 +93141 POINT(40.987622671067804 73.76791686235477) bank93141 +93142 POINT(41.25594523292978 73.68944932508005) bank93142 +93143 POINT(40.055077950801845 73.00880516567682) bank93143 +93144 POINT(40.05551060721439 74.38731511703007) bank93144 +93145 POINT(39.79408149143198 73.43653789423507) bank93145 +93146 POINT(40.68487445365976 73.92216648153958) bank93146 +93147 POINT(41.627981602694184 73.1775580835447) bank93147 +93148 POINT(41.58577010429153 74.4935905300036) bank93148 +93149 POINT(40.300921892821584 74.67953846858691) bank93149 +93150 POINT(40.32557127203538 73.90414796589104) bank93150 +93151 POINT(40.16494178801599 74.10748773076438) bank93151 +93152 POINT(41.703603414485 74.88974995562802) bank93152 +93153 POINT(39.95891909982883 74.8225857721448) bank93153 +93154 POINT(41.549151947511625 73.36332029870384) bank93154 +93155 POINT(40.7663249016574 73.22508662140388) bank93155 +93156 POINT(40.03429517934444 73.95335378541677) bank93156 +93157 POINT(39.81463860898469 74.48829019555478) bank93157 +93158 POINT(40.620650368533646 74.59562073100511) bank93158 +93159 POINT(40.99689011054827 73.65322003945035) bank93159 +93160 POINT(39.82766921755306 74.71648239055449) bank93160 +93161 POINT(40.20596298133907 73.07007730356457) bank93161 +93162 POINT(41.16387741605312 74.37795187200688) bank93162 +93163 POINT(40.44227123737915 73.01316261619112) bank93163 +93164 POINT(41.14979695978226 73.70016547506069) bank93164 +93165 POINT(41.04453553798864 74.1099001695647) bank93165 +93166 POINT(40.93608215402748 74.90910960083772) bank93166 +93167 POINT(41.165269532076465 74.30012980516908) bank93167 +93168 POINT(40.42267380034678 73.07108532510772) bank93168 +93169 POINT(41.03296706870051 74.42602720983996) bank93169 +93170 POINT(41.10731917169762 74.85135881349463) bank93170 +93171 POINT(41.248775539959716 73.08474362507539) bank93171 +93172 POINT(40.09923546866173 74.59814364507946) bank93172 +93173 POINT(39.98198663660554 73.41493039008331) bank93173 +93174 POINT(40.16476403108488 74.76732101249374) bank93174 +93175 POINT(41.507917258362255 73.46761140107954) bank93175 +93176 POINT(41.21725843054271 73.74910459734778) bank93176 +93177 POINT(40.35040358609272 73.04626524018784) bank93177 +93178 POINT(40.248284111193236 74.89927242770764) bank93178 +93179 POINT(40.647082586403535 74.19699402990592) bank93179 +93180 POINT(40.76448353132608 74.9885679019885) bank93180 +93181 POINT(40.10690478285701 73.7033944253935) bank93181 +93182 POINT(40.68346175778603 74.203361994899) bank93182 +93183 POINT(40.10181226723786 74.65832732443076) bank93183 +93184 POINT(41.58654037337757 74.18249572870778) bank93184 +93185 POINT(40.676270423200364 74.30597833274808) bank93185 +93186 POINT(41.23270402522616 74.61446877035891) bank93186 +93187 POINT(40.334690564250295 73.26905278370758) bank93187 +93188 POINT(41.16056574875806 74.64260962645372) bank93188 +93189 POINT(40.88633029895345 74.42222214731184) bank93189 +93190 POINT(40.05061451341398 73.93672071170664) bank93190 +93191 POINT(40.72592902049558 73.91610220964722) bank93191 +93192 POINT(41.59859011361465 74.19647382236154) bank93192 +93193 POINT(41.2220713832814 74.16160138544426) bank93193 +93194 POINT(40.29931451644201 74.01940319274496) bank93194 +93195 POINT(41.38534194031599 73.21088064525995) bank93195 +93196 POINT(40.954149740666196 73.72646470768848) bank93196 +93197 POINT(40.06537670200632 73.22059377462277) bank93197 +93198 POINT(40.226800701123295 73.11712236175691) bank93198 +93199 POINT(40.65685013800088 74.86045824098079) bank93199 +93200 POINT(40.297389494208865 74.06407598794439) bank93200 +93201 POINT(40.719179794036464 73.44571069517383) bank93201 +93202 POINT(41.394628423665374 74.49656873433027) bank93202 +93203 POINT(40.699585153645764 73.62025244277713) bank93203 +93204 POINT(41.67618578819745 74.74589494037899) bank93204 +93205 POINT(40.02489137302014 73.06536730868686) bank93205 +93206 POINT(41.7056749735157 74.68197824093751) bank93206 +93207 POINT(40.66937256036795 74.1013525340155) bank93207 +93208 POINT(41.67184673641522 73.82207091735908) bank93208 +93209 POINT(41.55469595112979 74.21377297681246) bank93209 +93210 POINT(40.89203387274317 74.10288189526946) bank93210 +93211 POINT(40.66324085050623 74.50110670249808) bank93211 +93212 POINT(41.31677700305718 73.28000054107652) bank93212 +93213 POINT(41.04377068048021 73.03791039174726) bank93213 +93214 POINT(41.07462601621568 73.35639025848397) bank93214 +93215 POINT(41.271555086232354 74.70274148168433) bank93215 +93216 POINT(41.70572926796249 74.62511967713712) bank93216 +93217 POINT(40.748958206673905 73.13022150268903) bank93217 +93218 POINT(40.79084118793996 74.42011921900364) bank93218 +93219 POINT(40.685584246139626 73.60003035554912) bank93219 +93220 POINT(40.30275214668457 73.26970405101132) bank93220 +93221 POINT(39.91245046965342 74.90667178352841) bank93221 +93222 POINT(40.60081794166706 73.28919427064147) bank93222 +93223 POINT(41.38989160281239 73.5573882118031) bank93223 +93224 POINT(40.062036911192386 73.68558544059557) bank93224 +93225 POINT(41.52371559883751 74.27273014025994) bank93225 +93226 POINT(39.97638534645657 73.72741702458264) bank93226 +93227 POINT(40.822135007068376 74.97000023152246) bank93227 +93228 POINT(41.46687229498506 74.28168780758348) bank93228 +93229 POINT(40.10746375004842 73.27760068182077) bank93229 +93230 POINT(40.45195424319787 74.88328169460455) bank93230 +93231 POINT(40.4303329185587 74.3449509742813) bank93231 +93232 POINT(41.450828929191715 75.0049553708912) bank93232 +93233 POINT(40.57414443710503 73.55640353073187) bank93233 +93234 POINT(39.96815890719905 73.22819503237653) bank93234 +93235 POINT(40.31938659696042 73.58010637424005) bank93235 +93236 POINT(40.75985829752382 73.78579778218523) bank93236 +93237 POINT(41.441803936997786 73.1377141756736) bank93237 +93238 POINT(41.54093416571628 73.21311025653131) bank93238 +93239 POINT(40.17734836183787 73.169935778017) bank93239 +93240 POINT(40.443474035868135 74.7845686679618) bank93240 +93241 POINT(41.25920554701053 73.67072979873048) bank93241 +93242 POINT(41.1932647425762 73.76644748206584) bank93242 +93243 POINT(39.79446946891202 73.0438387239451) bank93243 +93244 POINT(40.19290515382365 73.34427089453582) bank93244 +93245 POINT(40.35975228926615 74.94779833542388) bank93245 +93246 POINT(41.68273143589684 74.27418240609772) bank93246 +93247 POINT(41.20392293678956 74.08090076546085) bank93247 +93248 POINT(41.04181894363567 73.81223987302121) bank93248 +93249 POINT(41.56834757185213 73.91928508866972) bank93249 +93250 POINT(40.87181073108315 73.66115494162663) bank93250 +93251 POINT(40.435288319178845 73.73359985561771) bank93251 +93252 POINT(40.14904955877271 73.15945338686706) bank93252 +93253 POINT(40.48651041123511 74.32724440233898) bank93253 +93254 POINT(40.54950612061414 73.175678426722) bank93254 +93255 POINT(40.36263491173155 74.81049065213594) bank93255 +93256 POINT(40.123867275410156 73.2086886038521) bank93256 +93257 POINT(40.24143647378785 73.43372664717558) bank93257 +93258 POINT(40.97142182540875 74.87389088105232) bank93258 +93259 POINT(41.03918906817575 73.00949016937508) bank93259 +93260 POINT(41.35318211833754 74.46197800574218) bank93260 +93261 POINT(40.395111281327196 74.02432824649199) bank93261 +93262 POINT(41.67544883233697 74.27914598369061) bank93262 +93263 POINT(41.02701839192255 73.1529343693308) bank93263 +93264 POINT(41.631312429957894 74.91471985251204) bank93264 +93265 POINT(40.78194897117459 73.80881770408621) bank93265 +93266 POINT(41.077772605988 74.4025277032287) bank93266 +93267 POINT(41.219561045529936 74.53146665988307) bank93267 +93268 POINT(40.83806138256181 73.46484880383763) bank93268 +93269 POINT(40.97261227976757 73.62009473787717) bank93269 +93270 POINT(40.054634931890526 74.38196438435142) bank93270 +93271 POINT(40.90280452229971 73.39407167039873) bank93271 +93272 POINT(40.20257864810443 74.4095006813564) bank93272 +93273 POINT(41.42580127748159 74.49482730821863) bank93273 +93274 POINT(40.99169887184061 73.53321918745222) bank93274 +93275 POINT(41.248034024981486 73.28862340801935) bank93275 +93276 POINT(39.903311383258234 74.33863667911656) bank93276 +93277 POINT(41.121929243841656 73.26547505635212) bank93277 +93278 POINT(40.01658084974552 74.51994373577188) bank93278 +93279 POINT(41.674110094750134 74.7654532884768) bank93279 +93280 POINT(40.94013874692223 74.84931379646629) bank93280 +93281 POINT(39.734807633107984 74.31321984774561) bank93281 +93282 POINT(40.04039878501192 73.83131294791642) bank93282 +93283 POINT(40.31614755463594 74.52834075725094) bank93283 +93284 POINT(40.841907801306796 74.07084495164943) bank93284 +93285 POINT(39.7985614752945 74.37877656573349) bank93285 +93286 POINT(40.75924473118101 74.53395683250739) bank93286 +93287 POINT(40.73056779072793 73.93893241810947) bank93287 +93288 POINT(40.37581143991396 74.07624714931976) bank93288 +93289 POINT(40.008661996858244 73.59551639676296) bank93289 +93290 POINT(40.08403460102493 73.75431064668717) bank93290 +93291 POINT(39.73140867727146 74.47613196643013) bank93291 +93292 POINT(41.54664732697412 73.25542207359145) bank93292 +93293 POINT(39.88617571149954 74.3368978708134) bank93293 +93294 POINT(39.88005066134738 74.54440687977275) bank93294 +93295 POINT(40.84845812395036 74.37349892751226) bank93295 +93296 POINT(39.92530608769932 73.0095778588857) bank93296 +93297 POINT(40.911909222616075 74.88635538187616) bank93297 +93298 POINT(40.35476715448972 74.04952009666893) bank93298 +93299 POINT(40.27598831115046 73.30075831887477) bank93299 +93300 POINT(41.093157794674234 73.74489271488986) bank93300 +93301 POINT(41.544508268594285 73.41292019375095) bank93301 +93302 POINT(41.25808173372803 73.70730202731453) bank93302 +93303 POINT(40.85221464780672 73.01972430662647) bank93303 +93304 POINT(40.826780243762215 74.0131488620248) bank93304 +93305 POINT(41.697825451238046 74.1713738989073) bank93305 +93306 POINT(41.19974486292181 74.77101237937813) bank93306 +93307 POINT(39.96615301547226 73.1111498924687) bank93307 +93308 POINT(41.49381804593302 74.13893097540144) bank93308 +93309 POINT(39.82003816546174 73.70542550263472) bank93309 +93310 POINT(40.96935524969695 74.71883264006816) bank93310 +93311 POINT(40.359044302195414 74.297577443136) bank93311 +93312 POINT(40.65486286108499 73.4402380330932) bank93312 +93313 POINT(41.27813079598838 73.60235102976561) bank93313 +93314 POINT(40.12002017762104 74.4145453316428) bank93314 +93315 POINT(41.62547515201557 74.89758581240993) bank93315 +93316 POINT(40.24600257228706 73.46091583350655) bank93316 +93317 POINT(40.93649950453598 73.51254584756147) bank93317 +93318 POINT(39.97345202902742 73.0411241871149) bank93318 +93319 POINT(40.12441141091885 73.34050163410708) bank93319 +93320 POINT(40.353975315407844 74.51974579344872) bank93320 +93321 POINT(40.64852242205626 73.19246539222364) bank93321 +93322 POINT(41.37764171927165 74.42307591843631) bank93322 +93323 POINT(40.83742966440906 74.96992669862065) bank93323 +93324 POINT(40.82602639967661 74.04468687415591) bank93324 +93325 POINT(39.89688381150952 74.5861321214975) bank93325 +93326 POINT(40.21126834795972 74.78616170703862) bank93326 +93327 POINT(39.821381723973246 74.81200067450135) bank93327 +93328 POINT(41.159522013576584 73.1043387046445) bank93328 +93329 POINT(39.8896211780351 74.02708437012527) bank93329 +93330 POINT(41.40395416421488 73.62152789948024) bank93330 +93331 POINT(39.787129206591146 74.17933287996387) bank93331 +93332 POINT(40.947496519742344 73.9729710577525) bank93332 +93333 POINT(41.11060572489066 73.87816469970925) bank93333 +93334 POINT(41.21864997605097 73.69913385460957) bank93334 +93335 POINT(39.84896817013396 74.07440383078826) bank93335 +93336 POINT(41.136413751414175 73.4798606861591) bank93336 +93337 POINT(39.87499302814457 74.3747572137903) bank93337 +93338 POINT(41.293081168615025 74.26622027501315) bank93338 +93339 POINT(39.75081790040969 73.87496771908529) bank93339 +93340 POINT(41.30547617433048 73.01827539347143) bank93340 +93341 POINT(40.775263538070575 74.1140714559693) bank93341 +93342 POINT(41.2846245642172 74.19106679465887) bank93342 +93343 POINT(41.59828348087901 73.40587726101785) bank93343 +93344 POINT(40.8121008275049 73.49512976157303) bank93344 +93345 POINT(40.02480807133718 74.71379736158073) bank93345 +93346 POINT(40.513555761750794 73.28609714741198) bank93346 +93347 POINT(40.547590128328046 73.21688430150104) bank93347 +93348 POINT(41.48230598198778 74.83554040372968) bank93348 +93349 POINT(41.43254077445865 74.16482901062706) bank93349 +93350 POINT(40.9721210803906 73.99020151707565) bank93350 +93351 POINT(40.93961005005096 74.24575176727441) bank93351 +93352 POINT(40.180728810820796 73.48133684368199) bank93352 +93353 POINT(40.104357042288335 73.22007443613681) bank93353 +93354 POINT(39.900545894126886 74.42897282997068) bank93354 +93355 POINT(40.87046232461024 74.47859816196983) bank93355 +93356 POINT(41.230612172766925 74.77230702054086) bank93356 +93357 POINT(41.51550987909334 73.52059890206311) bank93357 +93358 POINT(41.42719006933606 74.95175670338543) bank93358 +93359 POINT(40.3976547829834 74.70022717653832) bank93359 +93360 POINT(41.04878682445417 73.97890562411943) bank93360 +93361 POINT(40.85934744131278 73.45765058482336) bank93361 +93362 POINT(41.05636533598435 73.28475147594808) bank93362 +93363 POINT(40.254289584885704 73.04572895366236) bank93363 +93364 POINT(41.61559753825229 74.58917859433933) bank93364 +93365 POINT(39.82775899668944 74.53076812845637) bank93365 +93366 POINT(40.39505714065235 74.46992812007441) bank93366 +93367 POINT(40.55958542847122 74.97486676630353) bank93367 +93368 POINT(40.6046201525503 73.63514080529316) bank93368 +93369 POINT(40.40645863300393 74.56312204747661) bank93369 +93370 POINT(40.611738440132385 74.6568864407773) bank93370 +93371 POINT(41.43486520156332 74.64514507832187) bank93371 +93372 POINT(40.09602243794157 73.93800031514071) bank93372 +93373 POINT(40.86761332869815 74.08430366851417) bank93373 +93374 POINT(40.296265351302125 74.92677985431392) bank93374 +93375 POINT(41.50588752782792 73.33603042859424) bank93375 +93376 POINT(39.719657981748234 74.09223315658414) bank93376 +93377 POINT(40.61766200338052 73.43434734102696) bank93377 +93378 POINT(39.94964476114223 74.54881296743231) bank93378 +93379 POINT(40.43942732811733 73.47691202584626) bank93379 +93380 POINT(41.60152462897774 74.49728556239751) bank93380 +93381 POINT(40.80125872180034 74.21099002168353) bank93381 +93382 POINT(40.34812466618995 74.94822364437272) bank93382 +93383 POINT(39.851452734999434 73.87230344817341) bank93383 +93384 POINT(40.30884704190704 73.775305828174) bank93384 +93385 POINT(39.86744606504701 73.02306250644621) bank93385 +93386 POINT(41.421655952202755 73.84322158089165) bank93386 +93387 POINT(41.09407065134403 74.38134889847832) bank93387 +93388 POINT(41.23721250117205 73.55974196815512) bank93388 +93389 POINT(40.71173901218898 73.48432136772873) bank93389 +93390 POINT(40.09240056712873 73.31794687045405) bank93390 +93391 POINT(40.29869166982077 73.1135171026568) bank93391 +93392 POINT(40.77309230822722 73.51548338434696) bank93392 +93393 POINT(40.05427295445988 74.0852367390862) bank93393 +93394 POINT(40.162263850389074 74.56547960971473) bank93394 +93395 POINT(40.796471772251664 73.07500117867444) bank93395 +93396 POINT(41.65306283658307 73.09729972545666) bank93396 +93397 POINT(40.811382888378546 74.97463690812793) bank93397 +93398 POINT(40.26218998725601 73.88842195759058) bank93398 +93399 POINT(40.30205884634601 74.14626786510847) bank93399 +93400 POINT(39.93724567549229 73.28887496147074) bank93400 +93401 POINT(40.33803221577472 73.05321500856434) bank93401 +93402 POINT(40.03684186497857 73.21906265395873) bank93402 +93403 POINT(39.78307381160097 74.12221421296196) bank93403 +93404 POINT(40.95543453094901 73.77117009853102) bank93404 +93405 POINT(40.38268740182399 73.3523894107951) bank93405 +93406 POINT(40.83099088907145 73.11272208977408) bank93406 +93407 POINT(40.13766151906982 73.03363510484806) bank93407 +93408 POINT(41.5555431498719 74.32909039105371) bank93408 +93409 POINT(40.15408904842138 74.98459507652709) bank93409 +93410 POINT(40.149121724664326 74.02515449697641) bank93410 +93411 POINT(39.93647631604321 73.21646946608728) bank93411 +93412 POINT(40.800980743258755 73.16840896181209) bank93412 +93413 POINT(41.28832194796261 74.33812781577089) bank93413 +93414 POINT(41.689369057033126 73.34918737913779) bank93414 +93415 POINT(39.975158084992415 74.42132201282061) bank93415 +93416 POINT(41.429485738691 74.23062882921386) bank93416 +93417 POINT(41.486006692186585 73.26023786843032) bank93417 +93418 POINT(41.02228887904571 73.27721127065745) bank93418 +93419 POINT(40.42484374361712 74.03698060022236) bank93419 +93420 POINT(40.18024901621223 74.50058430765488) bank93420 +93421 POINT(41.26319622347049 73.00908304324238) bank93421 +93422 POINT(40.920310451169435 74.6499153004441) bank93422 +93423 POINT(41.55367752589071 74.9842731902192) bank93423 +93424 POINT(40.143971116954134 74.58881421861611) bank93424 +93425 POINT(40.396639771175934 74.74469693902661) bank93425 +93426 POINT(40.30402158473753 74.10186428924654) bank93426 +93427 POINT(41.021914891252045 74.79025024183844) bank93427 +93428 POINT(40.14284632945057 73.03795779498435) bank93428 +93429 POINT(40.92524263099914 74.08011965890907) bank93429 +93430 POINT(41.188369756244875 73.37400608321829) bank93430 +93431 POINT(40.35936571901153 74.88722168364245) bank93431 +93432 POINT(40.56525726472148 73.59694385922822) bank93432 +93433 POINT(41.49913392760448 74.22838770309555) bank93433 +93434 POINT(41.45211695569682 73.88354019056173) bank93434 +93435 POINT(40.997536426468784 73.40933035817604) bank93435 +93436 POINT(41.30950541249829 74.37503663237595) bank93436 +93437 POINT(39.93747019501607 74.40574137161094) bank93437 +93438 POINT(40.486478285476096 74.36086839697451) bank93438 +93439 POINT(41.64342852830675 74.55438133585163) bank93439 +93440 POINT(41.18372170977099 73.92521298461058) bank93440 +93441 POINT(40.471521940415236 73.8688496627925) bank93441 +93442 POINT(40.25663729895469 73.67510062335268) bank93442 +93443 POINT(41.21998046158334 73.98108018159999) bank93443 +93444 POINT(40.09933039522643 74.34999044799505) bank93444 +93445 POINT(41.64690673743888 73.92435693121435) bank93445 +93446 POINT(40.3451083880214 73.55918008716907) bank93446 +93447 POINT(40.22641872056787 73.15634708111178) bank93447 +93448 POINT(40.80561895187677 73.71119971599848) bank93448 +93449 POINT(41.563035888600695 74.90211574981831) bank93449 +93450 POINT(40.1392784029307 74.6409360242326) bank93450 +93451 POINT(41.008251769222724 74.40119443277544) bank93451 +93452 POINT(39.73748686586669 73.5677840156744) bank93452 +93453 POINT(39.92107706314774 73.13613611784284) bank93453 +93454 POINT(40.74171771489347 73.41360937899486) bank93454 +93455 POINT(41.40506656072471 73.94237283622348) bank93455 +93456 POINT(41.480468189440806 73.96417543672634) bank93456 +93457 POINT(40.38627101527826 74.73948612713654) bank93457 +93458 POINT(40.25340801657337 73.02719609682792) bank93458 +93459 POINT(41.51582880272058 73.15865352053501) bank93459 +93460 POINT(39.95960758051353 73.40346163077835) bank93460 +93461 POINT(40.969760892056485 74.53319095560798) bank93461 +93462 POINT(40.42621374564048 74.0299065658794) bank93462 +93463 POINT(39.99853982695484 74.48030692397185) bank93463 +93464 POINT(40.62196150306953 73.13459434602758) bank93464 +93465 POINT(40.48781398590783 74.92316271695603) bank93465 +93466 POINT(40.1913583997009 74.37561474686548) bank93466 +93467 POINT(41.32741489448087 74.10127743064007) bank93467 +93468 POINT(40.60606134799685 74.98794261375465) bank93468 +93469 POINT(40.60311348886295 73.54176795246615) bank93469 +93470 POINT(40.53529707846538 73.80611888968147) bank93470 +93471 POINT(40.03155404098462 73.52075718855608) bank93471 +93472 POINT(40.01419632730361 74.24085655161441) bank93472 +93473 POINT(41.47940238784476 74.49704808089768) bank93473 +93474 POINT(40.19282421965577 74.76487495333221) bank93474 +93475 POINT(39.88430229307576 73.57972087488103) bank93475 +93476 POINT(41.54873252928702 74.30645142133945) bank93476 +93477 POINT(40.107023693294344 74.03569859697123) bank93477 +93478 POINT(41.63958210621847 74.91414740195549) bank93478 +93479 POINT(40.147047570871464 73.30744415880503) bank93479 +93480 POINT(40.36116727360531 73.56669074343458) bank93480 +93481 POINT(40.97462012388442 74.76464837349556) bank93481 +93482 POINT(40.83521256111649 73.57273126300555) bank93482 +93483 POINT(39.99410745119481 74.94735715152754) bank93483 +93484 POINT(40.49274880521196 74.85822120935663) bank93484 +93485 POINT(40.67483705039773 74.51560302810766) bank93485 +93486 POINT(41.539315898476715 74.39013974434806) bank93486 +93487 POINT(39.746260989664464 73.39755570376036) bank93487 +93488 POINT(39.81436725107951 74.2667530036073) bank93488 +93489 POINT(41.327724210684586 74.5210646286547) bank93489 +93490 POINT(39.96804116358967 73.36918942188461) bank93490 +93491 POINT(40.960051802566426 74.40670036700205) bank93491 +93492 POINT(40.34120768894893 73.96391896747059) bank93492 +93493 POINT(41.52393887038306 74.4670224081834) bank93493 +93494 POINT(39.804034291821225 73.92162611842552) bank93494 +93495 POINT(41.11139977700543 73.85755466532123) bank93495 +93496 POINT(40.728448758587085 73.97250517420089) bank93496 +93497 POINT(40.5748974615666 73.17181842282584) bank93497 +93498 POINT(40.954467175738124 74.59875911640434) bank93498 +93499 POINT(41.23511191957096 73.63978672286838) bank93499 +93500 POINT(41.658561785360554 74.57196917647953) bank93500 +93501 POINT(41.13238105475799 74.41912436898191) bank93501 +93502 POINT(40.914957799597865 73.64099371848253) bank93502 +93503 POINT(40.309936766923144 73.09948145805254) bank93503 +93504 POINT(40.77880236948175 73.15963558429445) bank93504 +93505 POINT(40.90060892126192 73.8906053782024) bank93505 +93506 POINT(41.11588005428197 73.31771578097006) bank93506 +93507 POINT(40.906594792678405 74.37552274457157) bank93507 +93508 POINT(41.49385233975085 73.15919058111606) bank93508 +93509 POINT(39.86058132720293 74.24221943186052) bank93509 +93510 POINT(40.55461190294785 73.38742454088558) bank93510 +93511 POINT(41.46334063309924 73.42938900144189) bank93511 +93512 POINT(40.64315003799753 74.82641820800967) bank93512 +93513 POINT(40.20285061290534 73.90843352963853) bank93513 +93514 POINT(40.540682619897005 74.36153853653762) bank93514 +93515 POINT(41.16490147059036 74.15431790132602) bank93515 +93516 POINT(40.67193108444915 74.66179721353558) bank93516 +93517 POINT(41.29213166565303 74.73026785832313) bank93517 +93518 POINT(40.74276806209554 74.27174958138578) bank93518 +93519 POINT(40.68402990954471 73.09940203484376) bank93519 +93520 POINT(41.142760931554236 74.90105680064775) bank93520 +93521 POINT(40.67654724205588 73.50734026054128) bank93521 +93522 POINT(41.41376363395618 73.36538856760549) bank93522 +93523 POINT(39.80550673034221 73.23465690139398) bank93523 +93524 POINT(41.29545675046283 74.53921915626606) bank93524 +93525 POINT(39.97589469499781 74.01914682436421) bank93525 +93526 POINT(41.06679639264925 73.0164527278261) bank93526 +93527 POINT(41.263386358984 74.73989070486826) bank93527 +93528 POINT(39.94470528029373 74.00018915456783) bank93528 +93529 POINT(41.65213174825407 74.88499855244665) bank93529 +93530 POINT(40.06497024216455 73.78748731627792) bank93530 +93531 POINT(40.97689903152087 73.36467506500279) bank93531 +93532 POINT(40.34998496509696 73.59135435337363) bank93532 +93533 POINT(40.897536652669984 73.03199857736726) bank93533 +93534 POINT(40.56822492071373 73.89619522150097) bank93534 +93535 POINT(39.784773558848464 74.24706607205667) bank93535 +93536 POINT(41.06709804118519 73.98140717749155) bank93536 +93537 POINT(40.34151190469261 73.15642321436383) bank93537 +93538 POINT(40.349641589223445 73.28262429498623) bank93538 +93539 POINT(40.00205268155635 74.1743290188967) bank93539 +93540 POINT(40.41279080266841 74.45307673638783) bank93540 +93541 POINT(40.881283765175546 73.62474343878999) bank93541 +93542 POINT(40.99406233590543 73.51194178895172) bank93542 +93543 POINT(40.72236567205844 74.27884379218841) bank93543 +93544 POINT(39.89529146630745 74.97326708746849) bank93544 +93545 POINT(40.610493558264345 74.28497875100138) bank93545 +93546 POINT(40.35432287869613 74.32010672617588) bank93546 +93547 POINT(40.59271337400876 74.04672505535746) bank93547 +93548 POINT(41.16268310307296 74.57669121891819) bank93548 +93549 POINT(41.45490753660249 74.15065381882187) bank93549 +93550 POINT(41.56115449177708 74.8949257205912) bank93550 +93551 POINT(39.9264138248263 74.51686377930616) bank93551 +93552 POINT(41.40116434648283 74.47129046235285) bank93552 +93553 POINT(40.38538029807545 74.69897290455164) bank93553 +93554 POINT(40.22243104962545 73.84103933575847) bank93554 +93555 POINT(40.08896517004617 73.69058467704616) bank93555 +93556 POINT(41.23478598816442 73.14489851286197) bank93556 +93557 POINT(39.86334468486676 74.20683108224327) bank93557 +93558 POINT(40.01206251438523 73.72564231994632) bank93558 +93559 POINT(41.042508078299804 74.28317785518472) bank93559 +93560 POINT(41.12549081642174 73.46164154572976) bank93560 +93561 POINT(41.224734083809786 73.69673546201996) bank93561 +93562 POINT(40.459173960839045 74.89379486291044) bank93562 +93563 POINT(40.682752638927134 74.6655039676701) bank93563 +93564 POINT(41.39267230820399 73.17246797247456) bank93564 +93565 POINT(40.50222591090402 73.61663927008014) bank93565 +93566 POINT(39.767649458290435 73.854989974544) bank93566 +93567 POINT(40.864068181205 73.55622757362345) bank93567 +93568 POINT(41.1113235356221 74.1307140708369) bank93568 +93569 POINT(41.05531743081653 73.4050825594322) bank93569 +93570 POINT(39.96588073479231 73.53187349995676) bank93570 +93571 POINT(41.09449167500673 74.95167303949202) bank93571 +93572 POINT(41.69609753141486 73.15963201680667) bank93572 +93573 POINT(40.18054541845025 74.18548319989576) bank93573 +93574 POINT(40.21015625043332 74.96543000192905) bank93574 +93575 POINT(41.35313055065792 73.77650374324425) bank93575 +93576 POINT(40.48314363585312 73.66412647532222) bank93576 +93577 POINT(41.50358504939238 73.37414652954955) bank93577 +93578 POINT(41.410886644180124 73.53796878081361) bank93578 +93579 POINT(39.987320501135414 73.4709771711873) bank93579 +93580 POINT(39.93697573391079 73.66267564203626) bank93580 +93581 POINT(40.75026725313966 73.55421662842673) bank93581 +93582 POINT(41.216355158990766 73.90324942698888) bank93582 +93583 POINT(40.78402579358236 74.73142563739674) bank93583 +93584 POINT(41.52575158906057 74.12055196381756) bank93584 +93585 POINT(41.513459997618575 74.09768113074425) bank93585 +93586 POINT(41.32060903054549 73.95003096273783) bank93586 +93587 POINT(41.35295172823786 73.0693658012338) bank93587 +93588 POINT(40.540224714445195 73.09755377641652) bank93588 +93589 POINT(40.62028335274396 73.19496249526873) bank93589 +93590 POINT(41.13916133279631 73.20019531496902) bank93590 +93591 POINT(41.09069451073424 74.29805340155974) bank93591 +93592 POINT(40.46163012918738 74.4267951506403) bank93592 +93593 POINT(40.252940747158576 73.07094310649946) bank93593 +93594 POINT(41.57724509227564 73.96192348584006) bank93594 +93595 POINT(40.208176988085704 73.44668839791687) bank93595 +93596 POINT(39.920000977921426 74.04711539680285) bank93596 +93597 POINT(39.73120376937642 73.16385048569732) bank93597 +93598 POINT(40.54747555165684 73.59593702839662) bank93598 +93599 POINT(40.78292057287854 74.65512972733785) bank93599 +93600 POINT(41.46412137019143 74.90896914202928) bank93600 +93601 POINT(40.70719942740092 74.31381314092894) bank93601 +93602 POINT(40.623034294566565 74.68985724595974) bank93602 +93603 POINT(40.071269943460244 74.63176172256739) bank93603 +93604 POINT(41.56136544491093 73.08220325723786) bank93604 +93605 POINT(41.443393219291735 73.82792624549027) bank93605 +93606 POINT(41.38565277668849 73.29991595744453) bank93606 +93607 POINT(41.648387292395206 74.56065331025059) bank93607 +93608 POINT(41.083897292152784 73.36925548260271) bank93608 +93609 POINT(40.75466426463675 73.23927119024209) bank93609 +93610 POINT(40.31684058120998 74.65454893524046) bank93610 +93611 POINT(41.56305884466003 74.8595108141216) bank93611 +93612 POINT(40.185965647886434 74.9398949134362) bank93612 +93613 POINT(41.16467078178669 73.31239361576297) bank93613 +93614 POINT(40.25309249011861 73.98673364356588) bank93614 +93615 POINT(41.256118013347745 73.8054942815424) bank93615 +93616 POINT(40.23762319635004 73.15782914970407) bank93616 +93617 POINT(41.232406397454504 73.69041688830234) bank93617 +93618 POINT(40.303825300741345 74.20201926983285) bank93618 +93619 POINT(40.03821092630873 73.7847578439059) bank93619 +93620 POINT(39.79643215745246 74.25882837300342) bank93620 +93621 POINT(40.66035139277265 73.89369818473251) bank93621 +93622 POINT(40.645232727502275 74.52980564418074) bank93622 +93623 POINT(41.4600988049229 74.68393153369293) bank93623 +93624 POINT(41.16871662438569 73.36443308799308) bank93624 +93625 POINT(40.88439106012907 74.56672990904504) bank93625 +93626 POINT(40.09997715115516 73.27445147196066) bank93626 +93627 POINT(41.01345465023241 74.3418799690928) bank93627 +93628 POINT(41.34955683630303 74.13101369113002) bank93628 +93629 POINT(40.82818272599719 73.49966208744382) bank93629 +93630 POINT(40.10622135387008 74.01579285426918) bank93630 +93631 POINT(40.50174405855544 73.23847145579566) bank93631 +93632 POINT(40.57987879897092 74.19565416249678) bank93632 +93633 POINT(40.523707645477884 74.11435545581996) bank93633 +93634 POINT(41.394816132234574 74.04842316326481) bank93634 +93635 POINT(40.951209974437624 74.36995172248884) bank93635 +93636 POINT(40.723198646835364 73.57306277521074) bank93636 +93637 POINT(41.28347551098117 74.32854312719068) bank93637 +93638 POINT(40.944309903157865 74.94179500676303) bank93638 +93639 POINT(40.51451688192055 73.04426111575215) bank93639 +93640 POINT(40.106241762806484 73.19233041532424) bank93640 +93641 POINT(39.95580012932754 74.50518514009549) bank93641 +93642 POINT(40.40155844017611 73.85445869210727) bank93642 +93643 POINT(39.9183754062564 73.58169396572134) bank93643 +93644 POINT(41.553050510027234 73.27932416334896) bank93644 +93645 POINT(41.41360494666481 74.93031135368511) bank93645 +93646 POINT(39.72409471523669 73.04706606353018) bank93646 +93647 POINT(40.58689708982608 74.07770237210342) bank93647 +93648 POINT(40.66559320505316 74.40958398739784) bank93648 +93649 POINT(41.56763854844665 73.25209131078445) bank93649 +93650 POINT(39.82428154777239 74.05272821464219) bank93650 +93651 POINT(41.54420996445401 73.01217030850805) bank93651 +93652 POINT(40.21038741427622 74.1123305073414) bank93652 +93653 POINT(41.02897773439501 74.02757146305755) bank93653 +93654 POINT(40.45769456126411 73.61700414019832) bank93654 +93655 POINT(40.570326939226526 74.17902184759932) bank93655 +93656 POINT(41.14421791113337 74.68710018096661) bank93656 +93657 POINT(41.15296403046831 74.028766752273) bank93657 +93658 POINT(40.59380502056848 74.66997819959357) bank93658 +93659 POINT(41.59647029625881 73.35162496692483) bank93659 +93660 POINT(41.301039085051514 74.72544307565165) bank93660 +93661 POINT(41.33296608853328 74.4614614159201) bank93661 +93662 POINT(39.874019182142064 73.71670115240148) bank93662 +93663 POINT(40.75996026401421 74.44855534935319) bank93663 +93664 POINT(41.463409828828894 74.45917604079408) bank93664 +93665 POINT(39.9538016988769 74.89973061214066) bank93665 +93666 POINT(41.38005489060499 74.8547440106629) bank93666 +93667 POINT(40.25835472272435 74.09396879856698) bank93667 +93668 POINT(40.13515691217327 73.59812329198522) bank93668 +93669 POINT(40.99158818760046 74.08069829622093) bank93669 +93670 POINT(39.75039981956133 73.83559018026698) bank93670 +93671 POINT(40.288879298767 73.60777802252589) bank93671 +93672 POINT(40.45915034439525 73.91225497302372) bank93672 +93673 POINT(40.01690288075307 74.88119577239974) bank93673 +93674 POINT(40.684516958848135 74.76967857008637) bank93674 +93675 POINT(40.11120429945119 73.47542918631255) bank93675 +93676 POINT(40.41476713656397 73.97776504880578) bank93676 +93677 POINT(40.022537391655675 74.329069989479) bank93677 +93678 POINT(40.24948940145189 74.35860293473999) bank93678 +93679 POINT(40.31141254126449 73.63276531993546) bank93679 +93680 POINT(40.495957352452336 74.15134381137332) bank93680 +93681 POINT(40.38666536018968 74.99727398422799) bank93681 +93682 POINT(39.85382394653155 74.35358388053109) bank93682 +93683 POINT(40.89736264435357 74.72409720297009) bank93683 +93684 POINT(40.82947119112574 73.18660373665804) bank93684 +93685 POINT(41.2979860915464 73.28042134072618) bank93685 +93686 POINT(40.103224457929386 74.43823886514143) bank93686 +93687 POINT(40.37206101805914 73.46025930309457) bank93687 +93688 POINT(40.16796241523548 74.2402033893262) bank93688 +93689 POINT(41.302640646271385 73.51058474303626) bank93689 +93690 POINT(40.472231983193126 73.74153051811176) bank93690 +93691 POINT(41.15028193505118 73.5096157464956) bank93691 +93692 POINT(39.89654147417337 74.13588859374124) bank93692 +93693 POINT(40.174165757873936 74.99406707364818) bank93693 +93694 POINT(40.68102591552218 73.24012667987031) bank93694 +93695 POINT(40.43944514499352 73.20659299660323) bank93695 +93696 POINT(40.2817209160212 74.90857680737331) bank93696 +93697 POINT(41.69200731180232 73.70133875826693) bank93697 +93698 POINT(41.2441781933795 74.68672094267868) bank93698 +93699 POINT(40.800923877058366 73.14062023004541) bank93699 +93700 POINT(40.561967207299595 74.12471209041419) bank93700 +93701 POINT(41.675057980703315 74.8052055749912) bank93701 +93702 POINT(41.39022508084674 74.6839275494165) bank93702 +93703 POINT(39.970350685387466 74.05263629874274) bank93703 +93704 POINT(41.49907429783446 73.61107671185098) bank93704 +93705 POINT(39.854642534796184 73.71404559794804) bank93705 +93706 POINT(39.759459768299976 74.39597755082279) bank93706 +93707 POINT(41.13969393186392 74.63686851401515) bank93707 +93708 POINT(41.027067533531635 74.09827594995288) bank93708 +93709 POINT(39.75027387252937 74.40793822773381) bank93709 +93710 POINT(40.22836150244819 73.36748743382427) bank93710 +93711 POINT(40.63227140487942 74.61223011529879) bank93711 +93712 POINT(40.709458496897795 74.38072648883457) bank93712 +93713 POINT(41.37866787128683 74.25237062102133) bank93713 +93714 POINT(41.632616746376165 74.47782225549052) bank93714 +93715 POINT(40.37888743311477 74.30997801461403) bank93715 +93716 POINT(40.07738058886055 73.27731354263719) bank93716 +93717 POINT(40.029894429947284 73.15766810905485) bank93717 +93718 POINT(40.61025676602538 73.98618879297248) bank93718 +93719 POINT(41.17564913945116 74.06359665110786) bank93719 +93720 POINT(41.02815284668269 74.76168591717693) bank93720 +93721 POINT(40.55165841614076 73.98032448548123) bank93721 +93722 POINT(39.75012554141753 74.13556594828583) bank93722 +93723 POINT(41.4941190382311 73.78448163146417) bank93723 +93724 POINT(41.25324650950092 73.74209210039923) bank93724 +93725 POINT(41.00536640892542 73.20467598065646) bank93725 +93726 POINT(41.688588208061496 74.3083305138777) bank93726 +93727 POINT(41.48433177847896 73.37118408721024) bank93727 +93728 POINT(40.53425276730309 73.85101669807246) bank93728 +93729 POINT(41.525958836623865 73.69221493529193) bank93729 +93730 POINT(41.32418182179821 74.22241505274935) bank93730 +93731 POINT(41.70429694527809 73.26302454816572) bank93731 +93732 POINT(40.58234497486482 73.13639008780497) bank93732 +93733 POINT(40.83182475630605 74.53440958749071) bank93733 +93734 POINT(39.87485257321476 73.69060652662691) bank93734 +93735 POINT(41.264502244083 73.29648525583745) bank93735 +93736 POINT(41.471732425228915 74.04087032526569) bank93736 +93737 POINT(40.89371609939186 74.89413757620298) bank93737 +93738 POINT(41.13241885166573 74.16444552445301) bank93738 +93739 POINT(41.17127753291133 74.14331737803889) bank93739 +93740 POINT(40.41815321678435 74.35503159433982) bank93740 +93741 POINT(40.17759828892378 74.16578181187852) bank93741 +93742 POINT(41.651364028161716 73.96536848694552) bank93742 +93743 POINT(41.22621562312147 74.72563587105768) bank93743 +93744 POINT(40.82865187479324 74.52781364200229) bank93744 +93745 POINT(41.094480522407714 74.0036512352757) bank93745 +93746 POINT(41.24134295128478 73.09783539584609) bank93746 +93747 POINT(40.76235436964364 73.0937700192255) bank93747 +93748 POINT(40.94265713761748 73.5155977160163) bank93748 +93749 POINT(40.40823259895042 74.81693755154372) bank93749 +93750 POINT(41.15360161500151 73.73337661556003) bank93750 +93751 POINT(41.51033475338268 73.07023826505173) bank93751 +93752 POINT(40.793878676604635 73.46581683877999) bank93752 +93753 POINT(40.68002498166001 73.44938390954299) bank93753 +93754 POINT(41.33513257798482 73.408494964227) bank93754 +93755 POINT(41.44481875025422 74.17807730459978) bank93755 +93756 POINT(39.7806394644947 73.18835685331891) bank93756 +93757 POINT(40.97215452221978 73.60274660808453) bank93757 +93758 POINT(39.80604273864684 73.46007372965956) bank93758 +93759 POINT(40.46297793373732 74.13170168372844) bank93759 +93760 POINT(40.920700255046725 74.89868120120168) bank93760 +93761 POINT(40.663645654471985 74.34524524448557) bank93761 +93762 POINT(41.3419385757967 73.7099623645036) bank93762 +93763 POINT(41.13546694854188 73.3186497028937) bank93763 +93764 POINT(40.44146470260467 74.33246222511619) bank93764 +93765 POINT(40.73340206936724 74.46648579588351) bank93765 +93766 POINT(40.05398597392042 73.74970393728027) bank93766 +93767 POINT(41.08130058800196 73.64453660761396) bank93767 +93768 POINT(41.617962187673 73.73671738397564) bank93768 +93769 POINT(39.92164749440485 73.89853641735193) bank93769 +93770 POINT(40.75503591822185 74.43337968884556) bank93770 +93771 POINT(41.64359512172452 73.0657025760474) bank93771 +93772 POINT(39.80051124041979 73.47147433006789) bank93772 +93773 POINT(40.55256186150532 73.74743992257488) bank93773 +93774 POINT(40.44510078390671 74.39216207216961) bank93774 +93775 POINT(40.25006694797113 73.31233142387451) bank93775 +93776 POINT(40.2286151503429 74.43734879364501) bank93776 +93777 POINT(41.42146433041568 74.2602618073064) bank93777 +93778 POINT(40.50299764726839 74.71556069855951) bank93778 +93779 POINT(41.64100972514065 73.04572306353546) bank93779 +93780 POINT(41.10957334466012 73.7640057946408) bank93780 +93781 POINT(41.28312038536471 74.18823086921584) bank93781 +93782 POINT(39.74607573901886 74.86467019996337) bank93782 +93783 POINT(41.58310073362726 73.76522640292681) bank93783 +93784 POINT(41.16469835615964 73.38294224165868) bank93784 +93785 POINT(40.56707053156189 74.25376485619583) bank93785 +93786 POINT(40.80278049902237 74.96976408672575) bank93786 +93787 POINT(41.6559471478236 73.74892825445653) bank93787 +93788 POINT(40.585102859930984 74.43592096905012) bank93788 +93789 POINT(40.83370976586749 73.50513745261716) bank93789 +93790 POINT(40.97564643436336 74.3566274025683) bank93790 +93791 POINT(41.356001974291814 73.2899127793746) bank93791 +93792 POINT(40.461871701371855 73.9216379624674) bank93792 +93793 POINT(40.35661587163271 74.73976926048114) bank93793 +93794 POINT(40.74876700404443 73.05654716869718) bank93794 +93795 POINT(40.12136081090006 73.1205030281926) bank93795 +93796 POINT(39.89632985964548 74.31462347388215) bank93796 +93797 POINT(39.98229572137988 74.17606110110837) bank93797 +93798 POINT(39.94397140672792 73.72708736191564) bank93798 +93799 POINT(40.796098230399615 73.76554148981555) bank93799 +93800 POINT(40.9716289723948 73.06792903603609) bank93800 +93801 POINT(40.06467104624175 73.44202549939737) bank93801 +93802 POINT(40.2843306987076 73.08871652873954) bank93802 +93803 POINT(40.6497118561533 74.39265133377086) bank93803 +93804 POINT(41.309988808611905 73.60361339982576) bank93804 +93805 POINT(41.14375353677623 74.33503416314242) bank93805 +93806 POINT(41.11505812877294 73.12095229238493) bank93806 +93807 POINT(39.967467616704056 74.93562121021448) bank93807 +93808 POINT(41.13920343722075 74.94166465162081) bank93808 +93809 POINT(40.904278631692954 73.62597361149335) bank93809 +93810 POINT(40.95646251468766 73.44928381565673) bank93810 +93811 POINT(39.71847285641425 73.81154103940484) bank93811 +93812 POINT(41.24220072018648 73.06404327623076) bank93812 +93813 POINT(41.37710908760665 74.76017132470656) bank93813 +93814 POINT(41.402846810499526 74.5042977919144) bank93814 +93815 POINT(41.1540327870386 73.84324315865238) bank93815 +93816 POINT(41.693089081277236 73.07856995354696) bank93816 +93817 POINT(41.63777292600124 74.54658529165746) bank93817 +93818 POINT(39.84226713354813 73.54261163291227) bank93818 +93819 POINT(40.221881078168515 73.0227169519412) bank93819 +93820 POINT(41.19209969994612 73.29614893394015) bank93820 +93821 POINT(40.5671529217893 74.5546585263346) bank93821 +93822 POINT(41.70939733188774 73.26556984110576) bank93822 +93823 POINT(40.09680864034893 74.83989133186557) bank93823 +93824 POINT(40.524786740200106 74.46870708329595) bank93824 +93825 POINT(40.31384216650051 74.43368717318565) bank93825 +93826 POINT(41.32220124177103 73.21365235419204) bank93826 +93827 POINT(40.929820410887494 73.30692024376927) bank93827 +93828 POINT(41.65737055475607 74.77078731733788) bank93828 +93829 POINT(41.10480993839865 73.0813611325339) bank93829 +93830 POINT(41.06105507872086 73.46903679845319) bank93830 +93831 POINT(41.485360715074364 74.84831715162824) bank93831 +93832 POINT(40.88838804156833 74.54954618070973) bank93832 +93833 POINT(40.29875368921351 73.02868247380414) bank93833 +93834 POINT(41.12903958323417 73.86797668404716) bank93834 +93835 POINT(41.65197727025319 74.95022056352383) bank93835 +93836 POINT(41.53448382172422 73.20836602070811) bank93836 +93837 POINT(40.21415570497963 74.63453838845535) bank93837 +93838 POINT(39.99334679453902 73.03619908400742) bank93838 +93839 POINT(39.79392264427294 73.46932806126358) bank93839 +93840 POINT(39.86347660934795 74.2279860195856) bank93840 +93841 POINT(40.654619739967934 73.97244484234334) bank93841 +93842 POINT(39.90471573442422 73.55344784297313) bank93842 +93843 POINT(41.40296574463774 73.20353139576805) bank93843 +93844 POINT(40.10667193390209 74.4758653606144) bank93844 +93845 POINT(41.63413986472615 74.72078106576073) bank93845 +93846 POINT(39.82755817987014 74.35916216258667) bank93846 +93847 POINT(41.123001740946776 74.49781801728373) bank93847 +93848 POINT(39.72932297233691 73.17747632921122) bank93848 +93849 POINT(41.33905573309444 75.00189491629064) bank93849 +93850 POINT(39.75790545540018 74.79483446183686) bank93850 +93851 POINT(40.92139721308487 73.0864030662567) bank93851 +93852 POINT(40.65441070791403 74.61176832813304) bank93852 +93853 POINT(40.540058506391155 73.67887549435987) bank93853 +93854 POINT(39.71863109287904 74.62471799107777) bank93854 +93855 POINT(40.210692221146275 73.33520203776646) bank93855 +93856 POINT(40.82485803094805 73.5760457211621) bank93856 +93857 POINT(40.29307131089645 73.69091706988817) bank93857 +93858 POINT(39.924935976721805 74.06689650297898) bank93858 +93859 POINT(39.955599844522496 74.5320670183283) bank93859 +93860 POINT(39.850222931107815 73.48286410877074) bank93860 +93861 POINT(40.10373125433765 73.30228649009496) bank93861 +93862 POINT(40.624014155298354 73.50018154646378) bank93862 +93863 POINT(39.85762124542203 73.25981671642877) bank93863 +93864 POINT(40.543297859272826 74.85195841289851) bank93864 +93865 POINT(41.60564117355249 73.31345841663425) bank93865 +93866 POINT(41.24097689636218 74.36009347482396) bank93866 +93867 POINT(40.917067695204324 73.62728869916349) bank93867 +93868 POINT(39.96503974368268 73.27194947914037) bank93868 +93869 POINT(41.267946640845395 73.35557696056449) bank93869 +93870 POINT(39.767135591232574 73.08376833224575) bank93870 +93871 POINT(40.03955285798501 73.06210615281879) bank93871 +93872 POINT(41.28135398282595 73.29116519530845) bank93872 +93873 POINT(40.947499314934234 74.42931441104926) bank93873 +93874 POINT(40.3422435769127 74.07067846515822) bank93874 +93875 POINT(40.36497255785201 74.28464403266342) bank93875 +93876 POINT(39.71605077465687 73.2255419971377) bank93876 +93877 POINT(40.8568455869064 74.40508191123094) bank93877 +93878 POINT(40.987676825346824 74.75574705000082) bank93878 +93879 POINT(41.70758353901086 74.30694579377867) bank93879 +93880 POINT(40.25772957743551 73.95299996782697) bank93880 +93881 POINT(40.410647965709785 74.19078357956973) bank93881 +93882 POINT(40.84919524491339 74.87450481809837) bank93882 +93883 POINT(41.163741281928694 73.48092795992108) bank93883 +93884 POINT(41.51454936427158 73.06741377715954) bank93884 +93885 POINT(40.42457842807426 73.25430914099657) bank93885 +93886 POINT(40.45611830919626 73.0278618802612) bank93886 +93887 POINT(41.57499143797252 74.7375527454846) bank93887 +93888 POINT(39.73147449954283 74.64912330894245) bank93888 +93889 POINT(40.499973573825265 74.54050711632954) bank93889 +93890 POINT(40.0106125603211 73.93854747770686) bank93890 +93891 POINT(39.91231008583318 73.37510584336023) bank93891 +93892 POINT(41.4119681061676 73.41229714327278) bank93892 +93893 POINT(41.686607726393106 74.54843238610185) bank93893 +93894 POINT(40.0597259901142 74.60013749632074) bank93894 +93895 POINT(40.338541124562816 73.92267446234077) bank93895 +93896 POINT(41.0952240768754 74.36392145369396) bank93896 +93897 POINT(40.149561918991736 74.59929887604747) bank93897 +93898 POINT(41.53081084022114 74.72233478332377) bank93898 +93899 POINT(40.893581165216425 74.61927278223796) bank93899 +93900 POINT(40.8317805962474 74.8423809342513) bank93900 +93901 POINT(41.141970904773615 73.18899924401677) bank93901 +93902 POINT(41.59577212282484 73.62727604421472) bank93902 +93903 POINT(40.392661854839744 73.75701344284887) bank93903 +93904 POINT(40.40575566387463 74.24038852175265) bank93904 +93905 POINT(40.307732776620995 73.28166857491259) bank93905 +93906 POINT(41.027054870421374 74.4049287766018) bank93906 +93907 POINT(41.417816311940584 74.67948954973991) bank93907 +93908 POINT(40.39414609450257 74.07334004852751) bank93908 +93909 POINT(41.45338337267417 73.12477749076864) bank93909 +93910 POINT(39.81801090735971 73.56974062363584) bank93910 +93911 POINT(41.614233138903174 74.46406559988692) bank93911 +93912 POINT(41.70465741449576 73.07240316998325) bank93912 +93913 POINT(41.2157485376939 74.71015121683507) bank93913 +93914 POINT(39.83248116402231 73.91066382829828) bank93914 +93915 POINT(40.55919961485796 73.9282504146204) bank93915 +93916 POINT(40.206488561016286 74.2546428045429) bank93916 +93917 POINT(40.786992436325704 73.3002279087768) bank93917 +93918 POINT(40.92095064102138 73.5378936764762) bank93918 +93919 POINT(41.48552734843734 74.41064523007752) bank93919 +93920 POINT(40.30585609895537 74.30957868828844) bank93920 +93921 POINT(40.51224613335742 73.94493617829873) bank93921 +93922 POINT(39.80805582799103 74.35030172308096) bank93922 +93923 POINT(39.82349880637803 73.15426981215796) bank93923 +93924 POINT(41.271242660413144 74.85927731864363) bank93924 +93925 POINT(40.912603333455074 74.51097406676794) bank93925 +93926 POINT(40.114154864030844 74.30798156169298) bank93926 +93927 POINT(40.93005500853776 74.51578738711265) bank93927 +93928 POINT(40.33944095670364 74.16635584316985) bank93928 +93929 POINT(40.38955604974098 73.21274315255087) bank93929 +93930 POINT(40.209083826543775 73.0526377742905) bank93930 +93931 POINT(40.744481891053695 73.69353610625913) bank93931 +93932 POINT(41.286466234509454 74.73061420891226) bank93932 +93933 POINT(39.87644477023531 73.34097514259803) bank93933 +93934 POINT(41.41269672530904 74.99325904893493) bank93934 +93935 POINT(40.54129233395822 74.41384560086412) bank93935 +93936 POINT(40.87242210070911 74.91244560076899) bank93936 +93937 POINT(41.67754510943158 73.11550344301745) bank93937 +93938 POINT(40.72642888125221 73.03355794978303) bank93938 +93939 POINT(40.71794616567378 73.79283104593044) bank93939 +93940 POINT(40.318126279593955 74.15219090701548) bank93940 +93941 POINT(40.87783559277734 74.75402607622378) bank93941 +93942 POINT(39.73570567396052 74.80077557342624) bank93942 +93943 POINT(40.1336536665004 74.38572277707921) bank93943 +93944 POINT(40.21727033429871 74.839173624219) bank93944 +93945 POINT(41.41545144706202 74.16378762277162) bank93945 +93946 POINT(40.779517982436566 74.73265363966783) bank93946 +93947 POINT(41.118775596815546 73.16261039183279) bank93947 +93948 POINT(40.87808467431948 74.63818299064529) bank93948 +93949 POINT(40.481866932641175 74.1135496310093) bank93949 +93950 POINT(40.349922064491956 74.33431960168262) bank93950 +93951 POINT(41.07241307766653 74.2434606257352) bank93951 +93952 POINT(40.16336051608321 73.88771710896029) bank93952 +93953 POINT(40.297672781268155 74.07205355980923) bank93953 +93954 POINT(40.326737172260735 74.06859085298633) bank93954 +93955 POINT(40.83651866540317 74.24744973514504) bank93955 +93956 POINT(39.784595966793766 73.62879804090039) bank93956 +93957 POINT(40.99314872512646 73.86688950764432) bank93957 +93958 POINT(40.69925821156731 74.11256676232722) bank93958 +93959 POINT(41.26776439507219 73.73665778442593) bank93959 +93960 POINT(39.83042774571644 73.70532031047141) bank93960 +93961 POINT(40.22866564352117 74.29755309289169) bank93961 +93962 POINT(41.32030717231042 73.95929019437307) bank93962 +93963 POINT(39.76264541662675 74.93822230252992) bank93963 +93964 POINT(40.1451645251552 73.9718843894131) bank93964 +93965 POINT(41.69054336639209 74.89950781770969) bank93965 +93966 POINT(40.74590476871185 73.89468812815555) bank93966 +93967 POINT(41.69799376514014 74.54253945515569) bank93967 +93968 POINT(39.985271890206896 74.03600653733432) bank93968 +93969 POINT(39.82751683444599 74.73159410463414) bank93969 +93970 POINT(40.00592316416125 73.6507935027137) bank93970 +93971 POINT(41.542097585117354 74.23292615945232) bank93971 +93972 POINT(40.801524651560705 73.4113953029811) bank93972 +93973 POINT(41.14156562468538 74.61383605263947) bank93973 +93974 POINT(40.82165644731662 74.53042026132276) bank93974 +93975 POINT(41.416945483817784 73.72987753312965) bank93975 +93976 POINT(40.60472055147479 74.7580136105673) bank93976 +93977 POINT(41.29954186414203 73.17456024590601) bank93977 +93978 POINT(40.54188632732268 73.6369621917384) bank93978 +93979 POINT(40.28943405400235 74.33954183380578) bank93979 +93980 POINT(39.7258148054719 74.95124029135218) bank93980 +93981 POINT(39.94748845817877 74.14566373622674) bank93981 +93982 POINT(39.75432599061757 73.72917545417619) bank93982 +93983 POINT(39.89976977572263 74.30464006811539) bank93983 +93984 POINT(39.73231726197483 74.93952125172719) bank93984 +93985 POINT(41.45901676435212 73.78167319574658) bank93985 +93986 POINT(39.92100056424891 73.75199161695444) bank93986 +93987 POINT(40.20797384176726 74.40014704622708) bank93987 +93988 POINT(40.94995317245166 73.66657439080102) bank93988 +93989 POINT(40.398236624659006 73.05591389109125) bank93989 +93990 POINT(41.42871446157927 74.2455177231681) bank93990 +93991 POINT(40.736221012936014 73.53610188297395) bank93991 +93992 POINT(40.09171194543602 74.7982919334218) bank93992 +93993 POINT(40.04821894560687 73.74477111642314) bank93993 +93994 POINT(40.5498279701915 74.51680646826577) bank93994 +93995 POINT(40.40298535587348 74.71499119693644) bank93995 +93996 POINT(40.314540865468146 74.18122966302711) bank93996 +93997 POINT(41.528734184089735 73.1604594215732) bank93997 +93998 POINT(40.06909421416687 73.4473658886182) bank93998 +93999 POINT(40.111414662284844 73.83890548469044) bank93999 +94000 POINT(41.333646376501946 74.10748348228925) bank94000 +94001 POINT(40.65953010476248 74.10671728544888) bank94001 +94002 POINT(40.48415725142218 74.1243225051783) bank94002 +94003 POINT(40.3410969661873 74.09821637013103) bank94003 +94004 POINT(41.661325183391334 73.26118163701616) bank94004 +94005 POINT(41.192471334104674 74.46334874363731) bank94005 +94006 POINT(40.91846865632086 73.55187364710041) bank94006 +94007 POINT(41.03297173762559 73.42964549991903) bank94007 +94008 POINT(40.31895335310966 74.71092848050537) bank94008 +94009 POINT(41.358057817910094 74.99741703663273) bank94009 +94010 POINT(41.4629047102066 73.93021616044604) bank94010 +94011 POINT(41.061131726973926 73.1037717839288) bank94011 +94012 POINT(40.23851988845657 73.9401626828442) bank94012 +94013 POINT(39.7671986474165 74.87199520088346) bank94013 +94014 POINT(40.49676334576708 73.73880171136341) bank94014 +94015 POINT(40.21413656121389 74.08618010638084) bank94015 +94016 POINT(40.78429731443006 73.50594054283413) bank94016 +94017 POINT(41.51278390945932 74.08765146951771) bank94017 +94018 POINT(40.638876251071004 74.63929907143888) bank94018 +94019 POINT(41.684319210934774 73.4974169644591) bank94019 +94020 POINT(40.18182057718077 74.72341841844204) bank94020 +94021 POINT(40.38598583004954 74.19788206240686) bank94021 +94022 POINT(41.06537164604789 73.92504694253707) bank94022 +94023 POINT(41.26079983098686 73.70034047747322) bank94023 +94024 POINT(41.69692357208746 73.19421112093903) bank94024 +94025 POINT(40.44566643875383 74.38939340381667) bank94025 +94026 POINT(40.78331596696455 74.94099905428732) bank94026 +94027 POINT(40.74119812454976 74.72414573204242) bank94027 +94028 POINT(41.144233795404965 74.68465957529068) bank94028 +94029 POINT(41.432232490992355 74.61931051878173) bank94029 +94030 POINT(40.63524904496671 73.51165069987134) bank94030 +94031 POINT(40.823843579065176 73.91335110110116) bank94031 +94032 POINT(39.9923961549832 74.84802009497746) bank94032 +94033 POINT(40.9257838779099 74.36090905898118) bank94033 +94034 POINT(39.901281607939595 74.50103813408094) bank94034 +94035 POINT(41.30745692658712 73.6181152912552) bank94035 +94036 POINT(39.95481895260616 74.01583398665842) bank94036 +94037 POINT(40.8727175777303 73.94063327165065) bank94037 +94038 POINT(39.74868824072094 74.53391794703235) bank94038 +94039 POINT(39.728252431945606 74.31266256777083) bank94039 +94040 POINT(40.20903112241678 73.99216246196245) bank94040 +94041 POINT(41.15810497002902 73.2725709746958) bank94041 +94042 POINT(41.40888545502649 73.30610923245584) bank94042 +94043 POINT(40.546863991090554 73.929886857869) bank94043 +94044 POINT(41.064552555638535 74.95910929880414) bank94044 +94045 POINT(41.639321292770596 74.60092879895623) bank94045 +94046 POINT(40.68287249287239 74.64937865065303) bank94046 +94047 POINT(40.88397889422926 73.5281754789388) bank94047 +94048 POINT(39.95680577555873 74.44240876943819) bank94048 +94049 POINT(41.458888511859286 74.22031577263077) bank94049 +94050 POINT(40.89363398354449 73.82370143200467) bank94050 +94051 POINT(41.59846577680769 74.38874493082163) bank94051 +94052 POINT(40.48408979494077 73.59390580582053) bank94052 +94053 POINT(40.93678169624165 73.7614981577616) bank94053 +94054 POINT(41.11416811949161 73.31261738733025) bank94054 +94055 POINT(41.59784045620469 74.24977942301469) bank94055 +94056 POINT(41.60028337406616 74.18578777299369) bank94056 +94057 POINT(39.88375822542559 73.6737897472508) bank94057 +94058 POINT(40.61107205101974 73.71007032497255) bank94058 +94059 POINT(40.94554142859196 73.4315198315243) bank94059 +94060 POINT(39.75955341342097 74.21335854008717) bank94060 +94061 POINT(40.14714804245042 73.0842215450265) bank94061 +94062 POINT(41.33369195941531 74.669263259675) bank94062 +94063 POINT(41.67081853093115 74.65672496223763) bank94063 +94064 POINT(40.26655779689392 74.63107127747159) bank94064 +94065 POINT(39.964969923622895 74.93716532202222) bank94065 +94066 POINT(40.84401101709021 74.12795805373136) bank94066 +94067 POINT(41.60175699841341 74.54325338858875) bank94067 +94068 POINT(41.5004634711486 74.2645596891348) bank94068 +94069 POINT(41.45731535734517 73.15444833211443) bank94069 +94070 POINT(41.01143729509042 73.45066184304676) bank94070 +94071 POINT(41.65872001076436 74.9946766550009) bank94071 +94072 POINT(41.41074379573421 73.77058403440488) bank94072 +94073 POINT(41.413944181615285 73.86455329765255) bank94073 +94074 POINT(40.22610727321694 74.62068892349623) bank94074 +94075 POINT(39.98158406739268 74.78817888763301) bank94075 +94076 POINT(41.45098955424884 73.10945621416474) bank94076 +94077 POINT(40.48175807813791 73.01916945486215) bank94077 +94078 POINT(41.34911589883044 73.78559523161866) bank94078 +94079 POINT(39.843519928682255 73.7855301352203) bank94079 +94080 POINT(41.45390303276519 74.88088405433264) bank94080 +94081 POINT(40.02386168816984 74.23238958097954) bank94081 +94082 POINT(40.6474353868484 74.26775503216442) bank94082 +94083 POINT(41.14515387438252 73.06660172863543) bank94083 +94084 POINT(40.1000358432618 73.48542421110295) bank94084 +94085 POINT(40.28639853335568 73.97170355056366) bank94085 +94086 POINT(40.24009048979752 74.15272356190401) bank94086 +94087 POINT(40.83787303417341 74.96309258297116) bank94087 +94088 POINT(40.57050642780102 73.43338339602055) bank94088 +94089 POINT(40.44010229756049 74.25329958809861) bank94089 +94090 POINT(40.2286360358933 73.54882736167406) bank94090 +94091 POINT(41.69634775072718 73.60756387100702) bank94091 +94092 POINT(40.73130762034435 74.71219287498738) bank94092 +94093 POINT(40.139749800132456 74.15679617899232) bank94093 +94094 POINT(40.69542409793955 73.57304291908953) bank94094 +94095 POINT(41.44855113244683 73.71364945339313) bank94095 +94096 POINT(40.54400417351176 73.37135734036332) bank94096 +94097 POINT(40.00944177089221 74.5796824336631) bank94097 +94098 POINT(40.630837807085946 73.09344794932422) bank94098 +94099 POINT(40.539918170676806 74.59646371918086) bank94099 +94100 POINT(40.674030053087215 74.15025752692016) bank94100 +94101 POINT(40.033971070249194 73.4170872720115) bank94101 +94102 POINT(41.336122651498144 74.58909501968078) bank94102 +94103 POINT(41.2282704009596 74.11645620285597) bank94103 +94104 POINT(39.76851153093422 74.84039087319158) bank94104 +94105 POINT(40.846354842650435 74.56910885131647) bank94105 +94106 POINT(41.648400474682944 73.58899788163173) bank94106 +94107 POINT(41.706565325283606 73.40506600795898) bank94107 +94108 POINT(41.5046058782992 73.50525456850698) bank94108 +94109 POINT(41.42875238694792 73.91209748449235) bank94109 +94110 POINT(40.93307734976276 73.5257166202386) bank94110 +94111 POINT(41.22210903527709 73.85601736513563) bank94111 +94112 POINT(40.65381589135391 73.6920461615714) bank94112 +94113 POINT(40.21092328382678 73.51526596634913) bank94113 +94114 POINT(40.40215684165249 74.09286546971111) bank94114 +94115 POINT(39.72675432807911 73.11264040017204) bank94115 +94116 POINT(40.95926245318772 74.75254993809442) bank94116 +94117 POINT(41.145868497644635 74.42245398352853) bank94117 +94118 POINT(40.20751778081211 73.03843276429183) bank94118 +94119 POINT(41.28175486425461 73.44239147035697) bank94119 +94120 POINT(40.34484433024557 74.13404867942052) bank94120 +94121 POINT(40.04991404150943 73.7560297057716) bank94121 +94122 POINT(40.11594033482809 73.50132628490162) bank94122 +94123 POINT(40.86351657389335 73.33486572634496) bank94123 +94124 POINT(40.64100082544028 74.52102052319756) bank94124 +94125 POINT(40.567969092218576 73.25285923326186) bank94125 +94126 POINT(41.59480669985476 73.38920398920592) bank94126 +94127 POINT(39.97149502287997 73.5790728341358) bank94127 +94128 POINT(40.84424493116099 74.90141990583682) bank94128 +94129 POINT(41.192352108623304 74.6387374589942) bank94129 +94130 POINT(40.60454447444053 73.79515601419102) bank94130 +94131 POINT(40.29560444361066 74.20627574029469) bank94131 +94132 POINT(40.751702502453995 74.99659854334823) bank94132 +94133 POINT(39.72122362328206 73.19502992107653) bank94133 +94134 POINT(41.22900241844376 73.45836871030922) bank94134 +94135 POINT(40.26474450178717 74.16617375175586) bank94135 +94136 POINT(41.429615806770805 73.97723139505251) bank94136 +94137 POINT(40.468153112236344 73.95891407444722) bank94137 +94138 POINT(40.17347125760355 73.53789155645916) bank94138 +94139 POINT(40.18434456107084 73.01216412004577) bank94139 +94140 POINT(41.138465209247904 73.7514272385551) bank94140 +94141 POINT(40.917075024867316 74.76302158938891) bank94141 +94142 POINT(41.35366024588126 74.9857001735572) bank94142 +94143 POINT(41.02474218238447 73.50341111075181) bank94143 +94144 POINT(40.252809860503454 73.31789509505435) bank94144 +94145 POINT(40.71686277055096 73.24724226893782) bank94145 +94146 POINT(41.58470297826412 73.14400184227874) bank94146 +94147 POINT(40.279971477922224 74.48726616214256) bank94147 +94148 POINT(41.29549343759491 74.97851095382735) bank94148 +94149 POINT(40.203649756727884 74.22692157075961) bank94149 +94150 POINT(39.93509582446703 74.10073997454059) bank94150 +94151 POINT(41.63922807937893 73.89041229594226) bank94151 +94152 POINT(40.21756025375836 74.07723020305346) bank94152 +94153 POINT(41.33955198097263 74.23400560535676) bank94153 +94154 POINT(40.38213985970778 73.4752424515284) bank94154 +94155 POINT(40.61568656813539 73.96347866759773) bank94155 +94156 POINT(40.86586595255953 73.14771766316323) bank94156 +94157 POINT(41.616448242543356 74.62452026596415) bank94157 +94158 POINT(39.81071008995221 74.52306648132429) bank94158 +94159 POINT(39.98040845779291 74.24536266434333) bank94159 +94160 POINT(40.32520203091701 73.44699003241669) bank94160 +94161 POINT(40.573113025909365 74.9242044967262) bank94161 +94162 POINT(40.75703892650361 74.56763212614179) bank94162 +94163 POINT(41.348222876351365 73.53481174236819) bank94163 +94164 POINT(40.72057101790682 73.61016387961568) bank94164 +94165 POINT(41.67581948061713 73.77736414922987) bank94165 +94166 POINT(41.12376596759524 74.51605081278765) bank94166 +94167 POINT(40.88215851165382 74.0162452783782) bank94167 +94168 POINT(40.102974121316834 74.41462546492623) bank94168 +94169 POINT(39.83546631239518 73.1965797720253) bank94169 +94170 POINT(40.71390779307015 74.2227088193313) bank94170 +94171 POINT(41.16984243729786 74.84125781023627) bank94171 +94172 POINT(40.301882161397195 73.65560076329342) bank94172 +94173 POINT(39.8103114730363 74.3242604924963) bank94173 +94174 POINT(40.548493939770395 74.82054370245858) bank94174 +94175 POINT(41.67245950015302 74.9091160748882) bank94175 +94176 POINT(41.296961109532674 73.01717252812496) bank94176 +94177 POINT(40.76142284261648 74.75471813192632) bank94177 +94178 POINT(40.45498980138406 73.32418399179194) bank94178 +94179 POINT(40.440781633900535 74.66411335539226) bank94179 +94180 POINT(41.66126174693213 73.26088808658609) bank94180 +94181 POINT(40.16233842464034 73.79687085236968) bank94181 +94182 POINT(39.760202505051254 74.92928069033195) bank94182 +94183 POINT(40.686983323882856 73.4111391761616) bank94183 +94184 POINT(40.50109022098773 74.76295368863535) bank94184 +94185 POINT(40.94104482609202 73.54441914678782) bank94185 +94186 POINT(40.083096615240635 73.39686755918903) bank94186 +94187 POINT(40.29650606343229 74.68072442889046) bank94187 +94188 POINT(40.61766743047149 73.05707369241856) bank94188 +94189 POINT(40.96502683994335 73.95722062130285) bank94189 +94190 POINT(39.86346493148993 74.01256349257585) bank94190 +94191 POINT(41.466507368559114 74.22413550298606) bank94191 +94192 POINT(41.66024496950742 74.02248794299939) bank94192 +94193 POINT(41.35278141900489 74.41748340590307) bank94193 +94194 POINT(40.64046621640741 74.02970087443401) bank94194 +94195 POINT(40.972453155134126 74.46711265306709) bank94195 +94196 POINT(39.905862871929166 74.20922138084748) bank94196 +94197 POINT(41.054568083409954 74.94970386554483) bank94197 +94198 POINT(41.00911667374422 74.45097083728886) bank94198 +94199 POINT(40.77590247159274 73.45723021059497) bank94199 +94200 POINT(39.968888450717856 74.43832653972954) bank94200 +94201 POINT(41.07231034626374 74.53564186720428) bank94201 +94202 POINT(41.65711755138378 73.95768627490499) bank94202 +94203 POINT(41.17823843429049 74.38105379173074) bank94203 +94204 POINT(40.991121657970176 74.47854189596565) bank94204 +94205 POINT(39.861042762214105 73.34183236215146) bank94205 +94206 POINT(39.950199337071055 73.59875304227025) bank94206 +94207 POINT(41.39987936620739 73.65056678407024) bank94207 +94208 POINT(41.475378486401986 73.08566796062279) bank94208 +94209 POINT(39.908612196245585 74.93421221123495) bank94209 +94210 POINT(41.625584481746536 73.1876098364084) bank94210 +94211 POINT(40.8835843721521 74.54907804328539) bank94211 +94212 POINT(40.341534347187675 73.78906754021703) bank94212 +94213 POINT(40.523652170124855 73.90935240292596) bank94213 +94214 POINT(41.39246390161349 74.45418662810377) bank94214 +94215 POINT(39.77455349253733 73.61958260201209) bank94215 +94216 POINT(39.94946247285047 74.91379919636803) bank94216 +94217 POINT(40.75033352736461 74.65428624042316) bank94217 +94218 POINT(40.98920826109576 73.90989512570548) bank94218 +94219 POINT(40.133910758230556 74.82038596982608) bank94219 +94220 POINT(40.227747776217115 74.62181089443007) bank94220 +94221 POINT(41.110230252740706 73.30967980286077) bank94221 +94222 POINT(40.03243748905726 73.83104836928334) bank94222 +94223 POINT(40.65505050773881 74.42749378086151) bank94223 +94224 POINT(41.46075705529597 74.18444308954639) bank94224 +94225 POINT(39.84355086337717 74.48612259923729) bank94225 +94226 POINT(41.708678337155305 73.3098011696982) bank94226 +94227 POINT(41.30486696827163 73.34339471910131) bank94227 +94228 POINT(41.63298240810168 74.20190451816248) bank94228 +94229 POINT(40.315236698790045 74.33943276806369) bank94229 +94230 POINT(39.95013024761903 74.73673286206959) bank94230 +94231 POINT(40.142255270368054 73.33284726310657) bank94231 +94232 POINT(40.02834851443518 73.80764007184636) bank94232 +94233 POINT(40.881081579305 74.81214856970242) bank94233 +94234 POINT(41.28255235679652 73.69471408998176) bank94234 +94235 POINT(40.29070729324763 74.69579448300142) bank94235 +94236 POINT(40.50103120322498 73.00931456925365) bank94236 +94237 POINT(40.80331867694416 73.46600428937359) bank94237 +94238 POINT(41.13947087089965 74.33977994547959) bank94238 +94239 POINT(40.21174677418786 73.52960082388348) bank94239 +94240 POINT(40.99717083597981 74.49731536537604) bank94240 +94241 POINT(40.81982134850315 74.91034700587677) bank94241 +94242 POINT(40.90738211354778 74.04759490916099) bank94242 +94243 POINT(41.078060498973656 74.40055886103802) bank94243 +94244 POINT(41.55997469827375 74.73425246345877) bank94244 +94245 POINT(40.5339379450569 74.12171675865306) bank94245 +94246 POINT(40.6123814180685 73.73797958887228) bank94246 +94247 POINT(41.10100246641623 74.59795688877436) bank94247 +94248 POINT(39.73724871737727 74.80562365432918) bank94248 +94249 POINT(39.96158651461354 73.14839691827336) bank94249 +94250 POINT(40.89478822999827 74.12097958894495) bank94250 +94251 POINT(40.98524165799245 74.7488120120739) bank94251 +94252 POINT(41.67020917308505 74.83283605073052) bank94252 +94253 POINT(39.71288103345577 73.30171206102078) bank94253 +94254 POINT(41.064013420587976 74.68988576004028) bank94254 +94255 POINT(41.142349657574144 73.94006124656244) bank94255 +94256 POINT(40.889039561151726 73.43410302140873) bank94256 +94257 POINT(39.957993212631216 74.53193038586406) bank94257 +94258 POINT(41.24225519416159 74.22408522637426) bank94258 +94259 POINT(40.29878709706036 73.51384052131758) bank94259 +94260 POINT(39.81319005242105 74.17882977132376) bank94260 +94261 POINT(40.041523789731876 74.26114850508843) bank94261 +94262 POINT(40.708345901098596 73.32546815722291) bank94262 +94263 POINT(40.83866056092485 74.92358670015452) bank94263 +94264 POINT(41.70576455470162 74.9003989977184) bank94264 +94265 POINT(40.14031020433299 74.97607915324207) bank94265 +94266 POINT(41.20766278595747 73.29585175749081) bank94266 +94267 POINT(40.72309169853879 73.04900020433826) bank94267 +94268 POINT(40.41446214353637 73.18778648853731) bank94268 +94269 POINT(39.93526765440934 74.65454373679228) bank94269 +94270 POINT(41.35447833417079 73.6850044823053) bank94270 +94271 POINT(41.22220678072049 73.70538026557267) bank94271 +94272 POINT(40.7686914604028 74.7372606046249) bank94272 +94273 POINT(39.87035883489826 74.27535506840877) bank94273 +94274 POINT(40.313493727826085 73.19421652150426) bank94274 +94275 POINT(40.33833617999346 74.31843060738467) bank94275 +94276 POINT(40.99363826820698 74.57129912395246) bank94276 +94277 POINT(40.57298515773659 74.84028255757983) bank94277 +94278 POINT(40.27015300141575 74.59929368382501) bank94278 +94279 POINT(41.024071601159285 73.37392518841537) bank94279 +94280 POINT(41.18906603220507 73.18290853504483) bank94280 +94281 POINT(40.60462453928157 73.10267014330525) bank94281 +94282 POINT(40.58777131538639 74.81727602012108) bank94282 +94283 POINT(40.38812466848304 73.04711328089344) bank94283 +94284 POINT(40.32043084318966 74.34759487232277) bank94284 +94285 POINT(41.170077346863856 73.76737000247735) bank94285 +94286 POINT(41.35960440716599 74.58099590500657) bank94286 +94287 POINT(41.39065669569683 73.95750145329832) bank94287 +94288 POINT(41.18819747138793 73.44169716332009) bank94288 +94289 POINT(41.36769315424391 74.87506138837577) bank94289 +94290 POINT(40.643624598861564 74.19576016111397) bank94290 +94291 POINT(40.53438116431061 73.91805797744358) bank94291 +94292 POINT(39.98496825737136 74.86472392477923) bank94292 +94293 POINT(41.44633570415788 74.72519573584356) bank94293 +94294 POINT(40.96517509653448 74.86096499650532) bank94294 +94295 POINT(40.933246286090075 74.78412545861073) bank94295 +94296 POINT(41.05090108426969 74.71869871496229) bank94296 +94297 POINT(40.046648290603066 73.8515786162195) bank94297 +94298 POINT(40.92343210827615 74.53517016762815) bank94298 +94299 POINT(39.71466147821709 74.91111395027409) bank94299 +94300 POINT(41.09992103242296 74.26028073034679) bank94300 +94301 POINT(39.815583388303516 74.51554702688912) bank94301 +94302 POINT(40.40514961094043 73.84655379709247) bank94302 +94303 POINT(41.31083474391327 73.8236238218783) bank94303 +94304 POINT(41.62227529085121 73.04431730330387) bank94304 +94305 POINT(40.636573913597246 73.44423805070552) bank94305 +94306 POINT(39.792258013678186 74.70017799779721) bank94306 +94307 POINT(40.01576366484179 73.70945156094777) bank94307 +94308 POINT(41.093866763140205 74.71473899513116) bank94308 +94309 POINT(40.14898230352338 73.59713505587905) bank94309 +94310 POINT(41.24824942214931 73.76373653319088) bank94310 +94311 POINT(39.874192918701574 73.02870472900578) bank94311 +94312 POINT(40.74408940237961 74.58931957210262) bank94312 +94313 POINT(40.29393660024636 73.66604892047037) bank94313 +94314 POINT(39.76515354663337 74.78800323479169) bank94314 +94315 POINT(40.05719289195763 74.93959019734848) bank94315 +94316 POINT(39.933684554761555 74.92370285469546) bank94316 +94317 POINT(41.64115247441611 74.46856478740189) bank94317 +94318 POINT(39.77348887814685 74.60720274249708) bank94318 +94319 POINT(40.73290931606024 74.2056130902011) bank94319 +94320 POINT(40.96168450063005 74.03782303797114) bank94320 +94321 POINT(40.60129708446883 74.81304347504721) bank94321 +94322 POINT(40.94542186587521 73.61933812658016) bank94322 +94323 POINT(41.32483046564481 73.64542765509051) bank94323 +94324 POINT(39.72869512931227 73.3791399712201) bank94324 +94325 POINT(40.89633206372082 73.85318996806095) bank94325 +94326 POINT(39.965669756945005 73.96867415572673) bank94326 +94327 POINT(40.44807773196351 73.86478501491287) bank94327 +94328 POINT(41.00242523018081 73.37041455886207) bank94328 +94329 POINT(41.17570044124309 74.15066494667363) bank94329 +94330 POINT(40.11306346700596 73.8188736676131) bank94330 +94331 POINT(41.147803866109626 73.17369451259472) bank94331 +94332 POINT(41.32352950050109 73.18861843098915) bank94332 +94333 POINT(41.4745556751682 74.08557977332909) bank94333 +94334 POINT(39.75375672127553 74.39845984833121) bank94334 +94335 POINT(41.1375243803909 74.76675606449778) bank94335 +94336 POINT(40.99141817987353 74.20403612723062) bank94336 +94337 POINT(41.13475236438912 74.2206807730203) bank94337 +94338 POINT(40.58281169519668 74.13675665292452) bank94338 +94339 POINT(40.05440000496433 73.26726875220535) bank94339 +94340 POINT(40.42240683708572 74.99046839483924) bank94340 +94341 POINT(39.90548472506933 74.18427281689935) bank94341 +94342 POINT(40.15299502560575 73.13078182706575) bank94342 +94343 POINT(40.62623959135285 74.21185227291474) bank94343 +94344 POINT(41.21592685706146 73.12434431310596) bank94344 +94345 POINT(40.90929170267776 74.84722039246192) bank94345 +94346 POINT(41.285730587552905 74.19596364922913) bank94346 +94347 POINT(40.01991047184962 74.60012772814986) bank94347 +94348 POINT(41.25763633317715 74.585902446629) bank94348 +94349 POINT(40.11626855871319 73.6576009313038) bank94349 +94350 POINT(40.2809909519072 74.42420883082872) bank94350 +94351 POINT(40.356531858240714 74.2622336751013) bank94351 +94352 POINT(41.42417072035691 73.9081087306309) bank94352 +94353 POINT(40.95734072352085 74.67701637718753) bank94353 +94354 POINT(40.501151200319796 74.8385122988395) bank94354 +94355 POINT(41.23492020095354 74.02200086400043) bank94355 +94356 POINT(39.721086810420466 75.0016163791208) bank94356 +94357 POINT(40.466163250351876 74.21827822397444) bank94357 +94358 POINT(40.95673223490688 74.57819352344859) bank94358 +94359 POINT(41.135612652772785 74.07631125364277) bank94359 +94360 POINT(40.07141378591829 74.50352646615308) bank94360 +94361 POINT(39.80980498632607 73.99782237452247) bank94361 +94362 POINT(41.207659943514145 74.72487583130832) bank94362 +94363 POINT(41.00017189515261 73.77899988436589) bank94363 +94364 POINT(40.06904196527516 73.57854576852795) bank94364 +94365 POINT(41.558555144054175 73.84048269118708) bank94365 +94366 POINT(40.3342520659455 74.0041727855412) bank94366 +94367 POINT(40.68984052939365 73.0332040070347) bank94367 +94368 POINT(40.42829075511469 73.68525997056915) bank94368 +94369 POINT(40.406085892867836 74.82938660549287) bank94369 +94370 POINT(40.687139527129304 74.70035435790471) bank94370 +94371 POINT(41.3189533784779 74.07609057993831) bank94371 +94372 POINT(40.28619908595476 73.18877155562973) bank94372 +94373 POINT(40.36460179416384 73.81687000860975) bank94373 +94374 POINT(40.79821611654998 73.59803444954092) bank94374 +94375 POINT(40.64269332537215 73.6650857625318) bank94375 +94376 POINT(40.24682987583145 74.19925130807286) bank94376 +94377 POINT(40.20567020338329 73.87881741780181) bank94377 +94378 POINT(40.17100699654341 73.38087421808494) bank94378 +94379 POINT(41.60196638718235 74.66085023674074) bank94379 +94380 POINT(41.02615125301063 74.5786056601082) bank94380 +94381 POINT(40.71488890455818 74.16901094810474) bank94381 +94382 POINT(39.78723735215576 73.61584476142066) bank94382 +94383 POINT(39.93855807004681 74.19710885794589) bank94383 +94384 POINT(40.16135136418212 74.55469028995472) bank94384 +94385 POINT(39.95755411838013 74.6637036634407) bank94385 +94386 POINT(39.90577165217176 73.36264628956127) bank94386 +94387 POINT(40.06597449460913 73.1444341650717) bank94387 +94388 POINT(41.37601001443628 73.66251250166326) bank94388 +94389 POINT(39.78607231747293 75.00335253972418) bank94389 +94390 POINT(41.51259152465112 73.81594729169993) bank94390 +94391 POINT(40.00102579987105 74.74946870109581) bank94391 +94392 POINT(41.696512812802624 73.83563424881639) bank94392 +94393 POINT(39.867574361502655 73.5884257054375) bank94393 +94394 POINT(40.305928582600785 74.30652677026781) bank94394 +94395 POINT(40.98473362286648 74.7073097149091) bank94395 +94396 POINT(40.507896519485584 74.93855638527386) bank94396 +94397 POINT(40.616813434673794 73.13188744126165) bank94397 +94398 POINT(40.092924607400775 74.82777281683495) bank94398 +94399 POINT(41.657973824431316 73.62157385716993) bank94399 +94400 POINT(41.10687220508192 74.17620560892075) bank94400 +94401 POINT(40.11088460889263 73.63095040743094) bank94401 +94402 POINT(40.700805698978186 73.31369498501726) bank94402 +94403 POINT(40.093752641300654 74.76625239982752) bank94403 +94404 POINT(40.07643501087323 74.76315278476801) bank94404 +94405 POINT(41.08393699418535 74.21095316350056) bank94405 +94406 POINT(40.798136636027955 74.14561638830143) bank94406 +94407 POINT(40.32136454307796 73.15917095919616) bank94407 +94408 POINT(40.50776876336501 74.78205796711892) bank94408 +94409 POINT(40.532369873391865 74.0569836003872) bank94409 +94410 POINT(40.250339568062564 74.41307103569636) bank94410 +94411 POINT(41.38896390114164 73.78012273927327) bank94411 +94412 POINT(40.01810229651931 73.35737929442152) bank94412 +94413 POINT(40.35068792356782 73.9291119347354) bank94413 +94414 POINT(40.74237021518534 73.2803420624724) bank94414 +94415 POINT(41.65073856885706 74.0977571507118) bank94415 +94416 POINT(39.736841320478675 74.24946700004725) bank94416 +94417 POINT(40.26994614630451 74.01828571582378) bank94417 +94418 POINT(41.32465506266327 74.25586191735427) bank94418 +94419 POINT(41.074438867352534 73.85336960757476) bank94419 +94420 POINT(41.57219434087472 73.84667894911021) bank94420 +94421 POINT(41.265878544506705 73.3001632322528) bank94421 +94422 POINT(39.9951571247723 73.4768363777264) bank94422 +94423 POINT(40.15410003697632 74.79572946668283) bank94423 +94424 POINT(41.31941112118186 73.48002136722897) bank94424 +94425 POINT(40.57507549714822 74.04816417400565) bank94425 +94426 POINT(41.2680593755472 74.72691084988514) bank94426 +94427 POINT(39.7496234061182 74.019646296115) bank94427 +94428 POINT(41.188668985602604 74.53703252774226) bank94428 +94429 POINT(40.27916114413503 74.44091925645466) bank94429 +94430 POINT(39.85618848193782 74.973847557427) bank94430 +94431 POINT(40.526485274830016 73.53945602162658) bank94431 +94432 POINT(40.58482358644317 74.3330417381787) bank94432 +94433 POINT(41.21395284911706 74.06988675264094) bank94433 +94434 POINT(40.3427306050139 73.05070006630375) bank94434 +94435 POINT(39.95625775381237 73.32984395022422) bank94435 +94436 POINT(40.533410497559714 74.09143740441189) bank94436 +94437 POINT(40.13811050060256 73.36356487317937) bank94437 +94438 POINT(40.529157330217245 74.95019415709658) bank94438 +94439 POINT(41.46385597972082 73.74070544156513) bank94439 +94440 POINT(41.08520624308506 73.37139192040009) bank94440 +94441 POINT(41.707183951920584 73.43589895812437) bank94441 +94442 POINT(41.02050921421807 73.56890159523726) bank94442 +94443 POINT(40.90569422103765 74.91491607805206) bank94443 +94444 POINT(41.301414811247184 74.63806275537443) bank94444 +94445 POINT(41.35629147902886 73.52450544356972) bank94445 +94446 POINT(40.43873832907465 74.94820559928968) bank94446 +94447 POINT(40.03327689761994 73.7596826328445) bank94447 +94448 POINT(39.76096403843396 74.66822646559217) bank94448 +94449 POINT(41.53905348407868 73.68520481359648) bank94449 +94450 POINT(41.40583885969248 73.10661067734873) bank94450 +94451 POINT(39.85746660198763 73.56608347462566) bank94451 +94452 POINT(41.424932302321984 73.24502669079357) bank94452 +94453 POINT(41.32676778459832 74.44998976390094) bank94453 +94454 POINT(40.47078161594148 74.90980746928709) bank94454 +94455 POINT(40.651996125042494 74.61987424124904) bank94455 +94456 POINT(41.49469830376039 74.86528188388317) bank94456 +94457 POINT(41.087610131915426 73.21811928621769) bank94457 +94458 POINT(41.59334697191546 73.40535652588177) bank94458 +94459 POINT(41.06894304582011 73.80310656839141) bank94459 +94460 POINT(40.52745505125504 73.9246533508029) bank94460 +94461 POINT(40.64610624969758 74.61579177348933) bank94461 +94462 POINT(40.823791006308674 74.69771078739015) bank94462 +94463 POINT(40.31730593844119 74.23906690051943) bank94463 +94464 POINT(40.32872023512381 73.65082264652762) bank94464 +94465 POINT(41.133760593604656 73.05969228784541) bank94465 +94466 POINT(40.518642538158154 73.55823192780211) bank94466 +94467 POINT(41.28707156052916 74.17431592123653) bank94467 +94468 POINT(40.59917068928184 73.75833650714765) bank94468 +94469 POINT(39.9856749199439 73.93591924977022) bank94469 +94470 POINT(41.18698306453037 73.98103943978181) bank94470 +94471 POINT(41.10985599650799 74.91360717397933) bank94471 +94472 POINT(41.680594233533775 73.86698162298106) bank94472 +94473 POINT(41.271144141693924 74.57960410291159) bank94473 +94474 POINT(39.914223036718134 74.32859002835957) bank94474 +94475 POINT(40.88304832891056 74.92444097441236) bank94475 +94476 POINT(41.282290697853924 74.31963931241205) bank94476 +94477 POINT(41.37693438150381 73.10383594999428) bank94477 +94478 POINT(41.074189439650524 74.3094290760904) bank94478 +94479 POINT(40.371069307033444 73.71363330902079) bank94479 +94480 POINT(41.02895157060906 74.7723185048682) bank94480 +94481 POINT(39.81182306470226 73.65654076541317) bank94481 +94482 POINT(39.91616535976868 73.32166657806773) bank94482 +94483 POINT(41.56132859014813 73.51507029223755) bank94483 +94484 POINT(40.74790464266464 73.6577657487701) bank94484 +94485 POINT(40.28142453724413 74.66043627214219) bank94485 +94486 POINT(39.78448243973987 74.61545991179983) bank94486 +94487 POINT(41.611111846200885 74.33942088242367) bank94487 +94488 POINT(40.955014596589756 73.62697153550063) bank94488 +94489 POINT(40.186631941284546 73.49136135156365) bank94489 +94490 POINT(41.61896447162437 73.01703099845331) bank94490 +94491 POINT(40.967015218860794 74.8486054585772) bank94491 +94492 POINT(40.78629550619954 74.32521882451162) bank94492 +94493 POINT(39.97676377144324 74.81093139394117) bank94493 +94494 POINT(41.55638784669645 74.73677012664278) bank94494 +94495 POINT(40.657822806451414 73.16759875522267) bank94495 +94496 POINT(39.99217673120634 74.68664454326672) bank94496 +94497 POINT(40.49434025048263 74.06466130317837) bank94497 +94498 POINT(40.722034803932836 73.81997888066257) bank94498 +94499 POINT(40.25296994861733 74.52612915512782) bank94499 +94500 POINT(40.71555448433512 73.56456044292393) bank94500 +94501 POINT(40.83283841532578 74.4221363834594) bank94501 +94502 POINT(39.77159990708376 74.23841760839231) bank94502 +94503 POINT(40.52956190011229 74.49502090383338) bank94503 +94504 POINT(40.67144341984754 73.1032339117004) bank94504 +94505 POINT(40.417438449261894 73.4971816858405) bank94505 +94506 POINT(40.94692780094986 73.84291906698151) bank94506 +94507 POINT(40.05835916246504 73.65924289517086) bank94507 +94508 POINT(41.508605935504 74.07167271769195) bank94508 +94509 POINT(40.49217876441075 74.86653966920103) bank94509 +94510 POINT(40.3336208562521 74.21205955156853) bank94510 +94511 POINT(40.41258158757976 74.9121185541024) bank94511 +94512 POINT(40.34300956815452 73.22217465195803) bank94512 +94513 POINT(39.88265607452944 73.88787023793523) bank94513 +94514 POINT(41.089514584642544 73.38399642722926) bank94514 +94515 POINT(41.43843439934571 74.76058079148463) bank94515 +94516 POINT(41.42977753076083 73.39683580758518) bank94516 +94517 POINT(39.81267483494287 74.5001627898845) bank94517 +94518 POINT(40.78591080801954 74.18368686482503) bank94518 +94519 POINT(39.88769744361235 73.45617960882335) bank94519 +94520 POINT(39.76428257041226 73.61755518879654) bank94520 +94521 POINT(40.21306406790797 73.45717491334305) bank94521 +94522 POINT(41.58325069180458 73.8528547316017) bank94522 +94523 POINT(41.34197544816531 74.78250070754332) bank94523 +94524 POINT(40.240441110588414 73.05046201645943) bank94524 +94525 POINT(39.80720292960977 73.42662304947392) bank94525 +94526 POINT(40.07558390015519 74.73313724172768) bank94526 +94527 POINT(41.69773476029067 73.6585786554409) bank94527 +94528 POINT(40.92060570111621 73.54458986338898) bank94528 +94529 POINT(40.521396046771216 74.78632459490888) bank94529 +94530 POINT(41.34189215054079 74.9880889685032) bank94530 +94531 POINT(40.49741694460303 73.30624832619365) bank94531 +94532 POINT(40.460808646525045 73.3512448344875) bank94532 +94533 POINT(41.422868730887224 74.17030257605245) bank94533 +94534 POINT(40.106053297488785 74.50239013787446) bank94534 +94535 POINT(41.104086276349165 74.42721755618477) bank94535 +94536 POINT(41.592514929870234 73.10423155114235) bank94536 +94537 POINT(40.50684737481447 73.48506912796704) bank94537 +94538 POINT(39.86152006241533 74.21566631281044) bank94538 +94539 POINT(41.280649751275234 74.95998027959433) bank94539 +94540 POINT(40.230200421346204 74.79893702671971) bank94540 +94541 POINT(41.547321410978526 74.29275014381484) bank94541 +94542 POINT(41.024893401794095 73.30248697109204) bank94542 +94543 POINT(40.96932882082581 73.67817002413068) bank94543 +94544 POINT(41.13904633227195 74.18524207008889) bank94544 +94545 POINT(40.053709169828736 73.24423055742182) bank94545 +94546 POINT(39.72980195409853 74.25499821330503) bank94546 +94547 POINT(41.098304661811525 74.69492753714904) bank94547 +94548 POINT(40.54107868467869 74.54787515652275) bank94548 +94549 POINT(40.84128833788204 73.73324184240475) bank94549 +94550 POINT(41.04175816143148 73.32600062740715) bank94550 +94551 POINT(40.79302633663865 73.61345794879978) bank94551 +94552 POINT(40.06870976913356 74.92505003806822) bank94552 +94553 POINT(40.8005717038595 73.79481035243975) bank94553 +94554 POINT(40.596376625851256 73.66347628255599) bank94554 +94555 POINT(40.794944064238436 73.95854446752448) bank94555 +94556 POINT(40.113883889033616 73.22026891833406) bank94556 +94557 POINT(40.09745165134954 73.9525030634761) bank94557 +94558 POINT(41.112596320344 73.48741103648125) bank94558 +94559 POINT(39.951413437229455 74.31121919183714) bank94559 +94560 POINT(40.15077649138212 73.55264388139712) bank94560 +94561 POINT(39.86892530039387 73.12378829555642) bank94561 +94562 POINT(40.9952756334633 73.16153563521998) bank94562 +94563 POINT(40.525602238612905 74.47225262413828) bank94563 +94564 POINT(40.725702422996974 74.25804927860389) bank94564 +94565 POINT(39.90176923100042 74.10219308082686) bank94565 +94566 POINT(41.59111689304141 74.98176105035895) bank94566 +94567 POINT(41.62556302421651 73.46689176217616) bank94567 +94568 POINT(40.459153271047924 73.92901299326029) bank94568 +94569 POINT(41.62494824211592 73.96703846389194) bank94569 +94570 POINT(40.84014681080978 73.5453010967102) bank94570 +94571 POINT(40.08587110887912 73.96053770369522) bank94571 +94572 POINT(40.44339917011736 74.0824757579234) bank94572 +94573 POINT(40.93455925276689 74.31307383378797) bank94573 +94574 POINT(40.895710137104274 74.52932066247958) bank94574 +94575 POINT(40.98665513097034 74.58431354681976) bank94575 +94576 POINT(41.22274382804124 74.69934233555622) bank94576 +94577 POINT(40.70525978894771 73.47293769873505) bank94577 +94578 POINT(41.07941645726672 74.92342841564584) bank94578 +94579 POINT(41.17570736196968 73.81206094998073) bank94579 +94580 POINT(40.494499981503886 73.69552012876204) bank94580 +94581 POINT(40.26977098884155 73.69586950528253) bank94581 +94582 POINT(40.468092998198514 74.83276352035578) bank94582 +94583 POINT(39.9010046382714 73.94713846464994) bank94583 +94584 POINT(40.5896738237867 73.02552987361817) bank94584 +94585 POINT(40.49489935634031 73.83718824008994) bank94585 +94586 POINT(41.242503386323044 74.70435530473813) bank94586 +94587 POINT(39.848047146101855 73.90472205928651) bank94587 +94588 POINT(40.85978072175275 74.46442088766032) bank94588 +94589 POINT(39.918332865894 74.90602688143903) bank94589 +94590 POINT(41.13826182343434 73.04198396770893) bank94590 +94591 POINT(41.4354443964751 75.00296628608653) bank94591 +94592 POINT(41.49574777298021 73.96910363981615) bank94592 +94593 POINT(39.883991094277796 74.52157836785693) bank94593 +94594 POINT(40.49531413434159 74.60875880891734) bank94594 +94595 POINT(40.99730182962953 74.06523088754501) bank94595 +94596 POINT(40.278139185859864 73.51365660577555) bank94596 +94597 POINT(41.694258647482556 74.15302760253002) bank94597 +94598 POINT(41.51967560036641 73.80366467508283) bank94598 +94599 POINT(40.78063117515429 73.10339276148689) bank94599 +94600 POINT(39.99546956385323 74.03681088961982) bank94600 +94601 POINT(41.503910218246496 73.67988247607971) bank94601 +94602 POINT(41.51951058166786 74.6831630179521) bank94602 +94603 POINT(39.86131470724298 74.9687729718175) bank94603 +94604 POINT(40.90853155274829 73.1656669768311) bank94604 +94605 POINT(40.07188503243342 73.50180525999284) bank94605 +94606 POINT(41.381377664195284 74.39198193270335) bank94606 +94607 POINT(40.97122642088973 74.7577935323029) bank94607 +94608 POINT(39.795105359779825 73.95075183216322) bank94608 +94609 POINT(39.838302910448654 73.52379002148848) bank94609 +94610 POINT(41.44372618977681 74.0658381466084) bank94610 +94611 POINT(40.239775317116276 74.42783202761493) bank94611 +94612 POINT(39.97645659396176 74.4241971367924) bank94612 +94613 POINT(41.449196360560556 74.51960977033659) bank94613 +94614 POINT(40.48927295339588 74.11529841602838) bank94614 +94615 POINT(40.89411298670269 74.9218036746198) bank94615 +94616 POINT(40.811157804869445 74.26997054498342) bank94616 +94617 POINT(40.91422968485784 73.71205141237088) bank94617 +94618 POINT(41.6811755506604 73.6590054243676) bank94618 +94619 POINT(41.00895093367155 73.04460425725496) bank94619 +94620 POINT(40.546833400328964 73.16210318664494) bank94620 +94621 POINT(40.766049097365 74.1736812562064) bank94621 +94622 POINT(40.283713303443555 73.34934169670893) bank94622 +94623 POINT(40.172116685518084 73.85418462863609) bank94623 +94624 POINT(40.17980459568752 74.09652878088971) bank94624 +94625 POINT(40.960953482443095 74.04506785701872) bank94625 +94626 POINT(41.56215352200696 74.54055108523464) bank94626 +94627 POINT(40.08630170427378 73.98680319031814) bank94627 +94628 POINT(41.07533532168555 73.16273690347262) bank94628 +94629 POINT(40.64378341054757 74.5164002932725) bank94629 +94630 POINT(40.84275328026889 74.57968379189535) bank94630 +94631 POINT(40.42226176484819 74.22371045192922) bank94631 +94632 POINT(39.979087180076405 74.74943018991263) bank94632 +94633 POINT(40.250404491476374 73.21323259728614) bank94633 +94634 POINT(41.29884324052291 73.02691764871254) bank94634 +94635 POINT(40.96528309134429 73.64822613815059) bank94635 +94636 POINT(40.041869032438186 73.25372058599594) bank94636 +94637 POINT(41.17981404502939 74.67357520367469) bank94637 +94638 POINT(40.50210879129894 73.45668565688847) bank94638 +94639 POINT(41.70162761842188 74.30791911896648) bank94639 +94640 POINT(40.317770504248756 74.3362775360007) bank94640 +94641 POINT(39.783795472200396 73.48162119076991) bank94641 +94642 POINT(40.88805476714254 74.62031595208728) bank94642 +94643 POINT(40.465892373850494 74.5148447421773) bank94643 +94644 POINT(39.820753510889254 73.17783186313136) bank94644 +94645 POINT(40.95093828810333 74.92602989321082) bank94645 +94646 POINT(40.276585886111484 73.3012866342859) bank94646 +94647 POINT(40.75172711105961 73.54037667758877) bank94647 +94648 POINT(40.41313312136394 74.29333924750493) bank94648 +94649 POINT(41.10856744471314 74.02281621849494) bank94649 +94650 POINT(40.83648816008103 74.85597573204592) bank94650 +94651 POINT(41.54987011331284 74.8868651004641) bank94651 +94652 POINT(39.88535308842773 74.45363643335375) bank94652 +94653 POINT(41.422703878405706 74.50192897101053) bank94653 +94654 POINT(41.531441670333486 73.522152724302) bank94654 +94655 POINT(41.22157380276242 73.81252865088148) bank94655 +94656 POINT(40.803997316141555 74.34398489349779) bank94656 +94657 POINT(40.54335031942923 75.00573526002455) bank94657 +94658 POINT(40.69543420298183 74.02285354947382) bank94658 +94659 POINT(40.37819109362228 73.14574788898454) bank94659 +94660 POINT(40.26848703088172 74.18711511786631) bank94660 +94661 POINT(41.13463407427338 73.26856072477172) bank94661 +94662 POINT(40.75811161682913 73.13657722862008) bank94662 +94663 POINT(41.32517945475943 73.91364275583793) bank94663 +94664 POINT(41.50060131731859 73.55339973475043) bank94664 +94665 POINT(40.08689253054548 74.73446683507066) bank94665 +94666 POINT(41.604014737098794 73.97279060448247) bank94666 +94667 POINT(40.96198476342771 73.39721104071427) bank94667 +94668 POINT(40.198555233616474 73.57329478109877) bank94668 +94669 POINT(41.370002384001424 74.18262097301175) bank94669 +94670 POINT(41.29793580242234 74.88132356196947) bank94670 +94671 POINT(41.321252227414455 74.5854625536089) bank94671 +94672 POINT(40.12742748591892 74.4446867160001) bank94672 +94673 POINT(41.12522610151598 73.33754967820134) bank94673 +94674 POINT(40.74463317035792 74.23211839934952) bank94674 +94675 POINT(40.47827527547744 73.31861052301188) bank94675 +94676 POINT(40.78654263868064 74.24192028615089) bank94676 +94677 POINT(41.026885308055455 73.89893764176692) bank94677 +94678 POINT(40.91612012569992 74.77610130132267) bank94678 +94679 POINT(41.30828286346882 74.93113135857581) bank94679 +94680 POINT(40.76841087559398 73.9585807105455) bank94680 +94681 POINT(40.39801748769739 73.23145070041484) bank94681 +94682 POINT(39.75912271392537 73.39015924158326) bank94682 +94683 POINT(40.86903710809543 73.84290057382444) bank94683 +94684 POINT(41.26406656311216 74.80661166310576) bank94684 +94685 POINT(41.00523609749559 73.2141039991847) bank94685 +94686 POINT(40.36887236378576 73.2117654153272) bank94686 +94687 POINT(41.01176564696626 74.57977869001454) bank94687 +94688 POINT(40.16499703803686 73.79670637370059) bank94688 +94689 POINT(40.834648711695856 74.38381214908486) bank94689 +94690 POINT(40.898143329654616 74.03059018844891) bank94690 +94691 POINT(39.75734224285335 73.77096962083277) bank94691 +94692 POINT(40.79639667863915 73.81846741343469) bank94692 +94693 POINT(41.049586243078835 73.8748417142372) bank94693 +94694 POINT(39.91928641762224 73.24915573694038) bank94694 +94695 POINT(41.20694198540972 73.2567779958684) bank94695 +94696 POINT(40.957504223704476 74.32325004555445) bank94696 +94697 POINT(40.864615767558725 73.05963432435551) bank94697 +94698 POINT(40.67350903001105 73.35827693165079) bank94698 +94699 POINT(40.7495840228186 73.15061317516087) bank94699 +94700 POINT(40.86368126192529 74.02873642554643) bank94700 +94701 POINT(40.831403065088104 74.6750589693581) bank94701 +94702 POINT(41.043565499167855 74.54418145941808) bank94702 +94703 POINT(40.18574859891868 74.62272263919051) bank94703 +94704 POINT(39.85070924605817 74.8545327761458) bank94704 +94705 POINT(39.785929245573456 74.23101468597842) bank94705 +94706 POINT(39.71717521688862 74.9257790771505) bank94706 +94707 POINT(40.50947484147948 74.66374279821224) bank94707 +94708 POINT(39.97652634767317 73.68823133220108) bank94708 +94709 POINT(40.41929773155553 73.79594673958977) bank94709 +94710 POINT(40.382030291684686 73.09154349144163) bank94710 +94711 POINT(41.404301424066006 74.3199445272301) bank94711 +94712 POINT(40.54203830174024 74.59939070636275) bank94712 +94713 POINT(41.47566077915601 73.09796882991255) bank94713 +94714 POINT(40.26069992981312 74.38547264975725) bank94714 +94715 POINT(41.27300493781231 74.85565313129966) bank94715 +94716 POINT(41.09227146520431 74.2821675700975) bank94716 +94717 POINT(39.92451445996762 73.1533160932598) bank94717 +94718 POINT(40.838905200032144 74.27205558812628) bank94718 +94719 POINT(40.88023449686488 73.51897485919297) bank94719 +94720 POINT(40.18002414857823 73.83109883463435) bank94720 +94721 POINT(41.359014257486315 74.99429724991738) bank94721 +94722 POINT(41.51290113186242 73.52115220527607) bank94722 +94723 POINT(41.61970687064496 74.74093082173262) bank94723 +94724 POINT(39.983935645583365 73.2169895211518) bank94724 +94725 POINT(39.71883087025307 73.09979620738663) bank94725 +94726 POINT(40.76838843012212 74.93510418575302) bank94726 +94727 POINT(41.4559899555382 73.68096863570442) bank94727 +94728 POINT(41.64782943771582 74.8216782377366) bank94728 +94729 POINT(39.89218246590431 73.68279892532748) bank94729 +94730 POINT(41.66384434254096 74.41747396884377) bank94730 +94731 POINT(41.66808206009044 74.933301803506) bank94731 +94732 POINT(40.28196595322029 73.97143535316171) bank94732 +94733 POINT(40.06013205483952 73.31284977819686) bank94733 +94734 POINT(41.687306259225736 74.74032917198787) bank94734 +94735 POINT(40.741201636022666 73.68966694099491) bank94735 +94736 POINT(39.95924954878538 73.8166889956375) bank94736 +94737 POINT(39.856717270985634 73.37617535997543) bank94737 +94738 POINT(40.686323162358164 74.7810585276543) bank94738 +94739 POINT(40.72942985009198 74.38039906473696) bank94739 +94740 POINT(41.29934895462124 73.20937929012324) bank94740 +94741 POINT(41.319101653952096 73.21529211517345) bank94741 +94742 POINT(40.23402270238938 74.08519082828344) bank94742 +94743 POINT(40.1543934747364 73.80051509234194) bank94743 +94744 POINT(41.175523246317674 73.43227286504556) bank94744 +94745 POINT(41.09466501506265 74.0015863164337) bank94745 +94746 POINT(41.653459388699616 74.4043347314081) bank94746 +94747 POINT(40.861247887765415 73.06672700896691) bank94747 +94748 POINT(40.60753186691753 73.25645936027136) bank94748 +94749 POINT(39.74211587163674 73.95144738764466) bank94749 +94750 POINT(41.28816960718505 73.60417872588809) bank94750 +94751 POINT(39.8203384713844 74.9984541887329) bank94751 +94752 POINT(40.80110815971582 73.16508712663476) bank94752 +94753 POINT(41.69823989889854 73.43972131779404) bank94753 +94754 POINT(41.133266822605655 73.605343577287) bank94754 +94755 POINT(41.35953990424948 73.79718991333866) bank94755 +94756 POINT(40.5884125100311 74.0492534312742) bank94756 +94757 POINT(40.92057924183513 73.81764812935681) bank94757 +94758 POINT(40.78201116190123 74.50545329212339) bank94758 +94759 POINT(41.05985350584461 74.25155365995612) bank94759 +94760 POINT(40.3847011456039 73.99422254923195) bank94760 +94761 POINT(41.2141443100818 74.3265541730453) bank94761 +94762 POINT(40.654003495274075 74.79259388909976) bank94762 +94763 POINT(41.16844943505639 73.48600857651515) bank94763 +94764 POINT(40.869839454687046 74.48374757671297) bank94764 +94765 POINT(41.07746437405661 74.32787333514165) bank94765 +94766 POINT(40.24999058795488 73.93145816388429) bank94766 +94767 POINT(40.902155314755476 74.53664188530952) bank94767 +94768 POINT(39.804564704300475 73.44101171713505) bank94768 +94769 POINT(41.44960141692573 73.49673391932632) bank94769 +94770 POINT(40.02275631110633 74.19437345603447) bank94770 +94771 POINT(41.18971153471841 74.78078628734244) bank94771 +94772 POINT(41.05080825599082 74.13187920387712) bank94772 +94773 POINT(40.304190750959684 73.13913194804194) bank94773 +94774 POINT(39.895768240574476 74.41487409258033) bank94774 +94775 POINT(41.42319546939764 74.54807775068673) bank94775 +94776 POINT(40.144302424580836 74.29332967035276) bank94776 +94777 POINT(41.5968045831673 74.79159954785473) bank94777 +94778 POINT(41.69748719820339 73.85966634772515) bank94778 +94779 POINT(39.90221872285867 73.37471467575499) bank94779 +94780 POINT(41.699321108040756 73.49316318079929) bank94780 +94781 POINT(41.19307730118887 73.37471880538052) bank94781 +94782 POINT(41.664376029336005 74.34180287616813) bank94782 +94783 POINT(40.88807276172774 74.9174620306462) bank94783 +94784 POINT(41.074432020275715 74.51141469774848) bank94784 +94785 POINT(41.47991847945081 74.20633861125637) bank94785 +94786 POINT(41.42831844525757 74.58008937125915) bank94786 +94787 POINT(40.40047480051438 73.37528499593289) bank94787 +94788 POINT(40.1572446777604 74.68841036681785) bank94788 +94789 POINT(39.82317319884157 73.93329310858873) bank94789 +94790 POINT(40.76228533954254 73.17232981104787) bank94790 +94791 POINT(40.83882531280232 74.35650576543667) bank94791 +94792 POINT(40.40963472186998 74.98833724039136) bank94792 +94793 POINT(41.51015738529817 74.80636059680867) bank94793 +94794 POINT(40.469042952042244 73.55613321643341) bank94794 +94795 POINT(39.99048728514399 74.64417975853215) bank94795 +94796 POINT(40.08815823247113 74.90429786459526) bank94796 +94797 POINT(40.30633521554084 74.85942678327402) bank94797 +94798 POINT(40.51150999866031 74.28688056895375) bank94798 +94799 POINT(41.39462887364164 74.18917360154887) bank94799 +94800 POINT(41.40903895246752 74.94381166378484) bank94800 +94801 POINT(41.71221510296997 73.2243949501527) bank94801 +94802 POINT(41.66895866322365 74.6413056586627) bank94802 +94803 POINT(40.37400314653385 73.83550296553874) bank94803 +94804 POINT(41.08624627232636 74.33205009091874) bank94804 +94805 POINT(40.6413478304965 73.39228650751092) bank94805 +94806 POINT(41.462150900394285 74.77738146698907) bank94806 +94807 POINT(39.98861665463842 74.12187477894145) bank94807 +94808 POINT(41.046776822601835 74.91646644820631) bank94808 +94809 POINT(40.226349117755106 74.26486570643877) bank94809 +94810 POINT(40.86877214095971 73.92406268627715) bank94810 +94811 POINT(41.00351025564441 74.09182919859974) bank94811 +94812 POINT(41.534766295401994 73.46401993087599) bank94812 +94813 POINT(41.70124533284428 74.49266195044665) bank94813 +94814 POINT(41.31210086628662 74.7671026379739) bank94814 +94815 POINT(41.18688759172404 74.70743124922805) bank94815 +94816 POINT(40.558251712291785 74.35979246106821) bank94816 +94817 POINT(40.09343670673838 74.15182440183895) bank94817 +94818 POINT(39.991700722574286 73.12337199036804) bank94818 +94819 POINT(39.92589620669032 74.48381544875797) bank94819 +94820 POINT(41.26020629858778 73.83970399540394) bank94820 +94821 POINT(40.32409134564688 73.9105043442612) bank94821 +94822 POINT(41.268377506192614 73.63161252837122) bank94822 +94823 POINT(40.33478754116439 74.89201195094837) bank94823 +94824 POINT(40.169721589724965 73.89452966414933) bank94824 +94825 POINT(41.04352682446187 73.92894034921166) bank94825 +94826 POINT(40.36407824159271 73.56279116050133) bank94826 +94827 POINT(40.99999650881734 73.96685743422374) bank94827 +94828 POINT(40.971822939191796 74.62236184437418) bank94828 +94829 POINT(41.24991223589631 74.41412591291773) bank94829 +94830 POINT(40.15858887655036 74.58987327188711) bank94830 +94831 POINT(41.04267721237864 74.30026686194238) bank94831 +94832 POINT(41.36266660935821 73.49898636061589) bank94832 +94833 POINT(40.19035128425199 74.19890990225025) bank94833 +94834 POINT(40.216240100434014 73.5596416273799) bank94834 +94835 POINT(39.94497467994097 74.00552615054845) bank94835 +94836 POINT(41.56055887059189 74.67686971525546) bank94836 +94837 POINT(40.62171206198365 73.41743464655998) bank94837 +94838 POINT(39.97721444289063 74.97703805783297) bank94838 +94839 POINT(41.40358157308733 73.43923145146024) bank94839 +94840 POINT(41.52994677235366 74.46693311602577) bank94840 +94841 POINT(41.169922922650755 74.95500766827351) bank94841 +94842 POINT(39.968774550067614 74.91235366633653) bank94842 +94843 POINT(41.19132513086995 74.46888766238814) bank94843 +94844 POINT(41.46011055795711 73.11463197011057) bank94844 +94845 POINT(41.578275828104495 74.72189817166014) bank94845 +94846 POINT(40.06456213973183 73.34004849428837) bank94846 +94847 POINT(40.04613867472966 74.0875223960061) bank94847 +94848 POINT(40.09073475420143 73.55172649728007) bank94848 +94849 POINT(40.065781432093054 73.58140058832926) bank94849 +94850 POINT(41.69970153122253 74.45306236995249) bank94850 +94851 POINT(40.780727181586286 74.4034864233243) bank94851 +94852 POINT(40.53189454703332 74.07554419864766) bank94852 +94853 POINT(40.61321203649521 73.01962905834864) bank94853 +94854 POINT(41.345762982018314 73.83352303701183) bank94854 +94855 POINT(40.84920082516556 74.40966622293057) bank94855 +94856 POINT(40.41308025270341 73.36812539151101) bank94856 +94857 POINT(41.23294199383324 74.30210615401508) bank94857 +94858 POINT(39.88949056535535 73.52747871189644) bank94858 +94859 POINT(40.64913895003055 73.01030127277488) bank94859 +94860 POINT(39.77041326627746 74.31407652982553) bank94860 +94861 POINT(39.83676759618252 73.02728791296599) bank94861 +94862 POINT(40.6524345574749 73.46079502393108) bank94862 +94863 POINT(41.07222183013041 74.46190087997816) bank94863 +94864 POINT(41.18762845198955 73.54669803154525) bank94864 +94865 POINT(40.34676428507047 74.9087138538563) bank94865 +94866 POINT(41.23975762708349 74.63766277115933) bank94866 +94867 POINT(40.91624386676149 73.40273788469067) bank94867 +94868 POINT(40.64096446188531 73.21421883568388) bank94868 +94869 POINT(40.87878521637604 74.82903976856642) bank94869 +94870 POINT(40.73801142745804 73.8512946727206) bank94870 +94871 POINT(41.5774759627887 74.53205015019667) bank94871 +94872 POINT(40.04513593816779 74.60982217283181) bank94872 +94873 POINT(41.6286614116417 74.74470783821252) bank94873 +94874 POINT(40.303258138177085 74.54657748524717) bank94874 +94875 POINT(39.96309832926744 73.11005443311984) bank94875 +94876 POINT(39.85519232619761 74.90468554108492) bank94876 +94877 POINT(40.250184023100246 73.33975147767434) bank94877 +94878 POINT(40.35971342853624 74.07024873796223) bank94878 +94879 POINT(41.177042776862116 74.49524738747584) bank94879 +94880 POINT(41.528765278535495 73.32770495746543) bank94880 +94881 POINT(39.73409269624007 73.57717065292266) bank94881 +94882 POINT(40.0338551945028 74.86980598277975) bank94882 +94883 POINT(41.50550628381648 73.84651289046727) bank94883 +94884 POINT(39.90686203152278 73.78017628560995) bank94884 +94885 POINT(40.44735368498958 74.76243667478286) bank94885 +94886 POINT(40.87947551227287 74.1686026305525) bank94886 +94887 POINT(40.63569478770827 74.86242464101198) bank94887 +94888 POINT(39.96563258837137 73.69189684939413) bank94888 +94889 POINT(39.97059796732852 74.99485726863819) bank94889 +94890 POINT(40.804847428217194 73.6755358999832) bank94890 +94891 POINT(41.58950989422222 74.09571576480319) bank94891 +94892 POINT(41.03132331990541 73.94821260854823) bank94892 +94893 POINT(40.11722516937562 74.21787825563781) bank94893 +94894 POINT(40.37247613208849 73.61807062687222) bank94894 +94895 POINT(40.90652642831711 73.17701475557774) bank94895 +94896 POINT(41.66186290460759 73.50942781298754) bank94896 +94897 POINT(41.54107107266409 74.92083074000769) bank94897 +94898 POINT(40.524236491424155 73.84609461315668) bank94898 +94899 POINT(41.414793129090846 74.92245926680378) bank94899 +94900 POINT(39.91601996637725 73.31166009854577) bank94900 +94901 POINT(41.043718220916176 74.93088967734863) bank94901 +94902 POINT(40.75661288234081 73.26380526925492) bank94902 +94903 POINT(40.44312083750402 73.54158413087002) bank94903 +94904 POINT(39.89039437192001 73.99878492376034) bank94904 +94905 POINT(40.57956000266286 73.1365140744575) bank94905 +94906 POINT(40.83556426231885 74.061592407827) bank94906 +94907 POINT(40.79592349977382 74.62797725575064) bank94907 +94908 POINT(41.259752554507806 73.20912775807162) bank94908 +94909 POINT(41.4639749134597 74.52318000884472) bank94909 +94910 POINT(41.712365704181245 74.81581228366552) bank94910 +94911 POINT(40.88806465172181 74.34771031318708) bank94911 +94912 POINT(40.92317518641757 74.64467067371221) bank94912 +94913 POINT(40.66992141246936 74.2369205185899) bank94913 +94914 POINT(40.34228642523128 73.03816910304701) bank94914 +94915 POINT(40.70351515965471 74.78585473985981) bank94915 +94916 POINT(40.41898344733926 74.3526939657353) bank94916 +94917 POINT(39.99451098257383 73.41876563812696) bank94917 +94918 POINT(39.78912616728714 73.26613633100881) bank94918 +94919 POINT(39.84206368042921 73.74570646150114) bank94919 +94920 POINT(40.487686138180486 73.35031215686983) bank94920 +94921 POINT(40.10543755429203 73.27304771980465) bank94921 +94922 POINT(39.95757354530034 74.3294304323783) bank94922 +94923 POINT(40.58202049266005 73.8821857471988) bank94923 +94924 POINT(40.4231500602871 74.23863225708693) bank94924 +94925 POINT(40.78379171740547 74.75022977566869) bank94925 +94926 POINT(40.043158125978245 73.70527322276875) bank94926 +94927 POINT(41.383927159822335 73.57137455983319) bank94927 +94928 POINT(41.594344199246365 74.63004498493243) bank94928 +94929 POINT(40.83629253304923 74.27225404927253) bank94929 +94930 POINT(40.218584044553246 73.13919390647037) bank94930 +94931 POINT(40.54155923056416 74.31921004854091) bank94931 +94932 POINT(40.68572617574995 74.33808876586515) bank94932 +94933 POINT(41.531331293340166 73.71612024114485) bank94933 +94934 POINT(40.37490820238815 74.5622216325332) bank94934 +94935 POINT(40.134342327524564 74.89863463246267) bank94935 +94936 POINT(41.315724599646174 73.65272329196762) bank94936 +94937 POINT(40.313946958489545 73.92198582742198) bank94937 +94938 POINT(41.46348440649307 73.51176807443856) bank94938 +94939 POINT(40.567461637528886 73.6392396480895) bank94939 +94940 POINT(41.161755839101644 73.8984298275109) bank94940 +94941 POINT(40.55728553791607 73.04326204303334) bank94941 +94942 POINT(39.8030211316954 73.29546940645521) bank94942 +94943 POINT(41.61671021619947 73.98915216966118) bank94943 +94944 POINT(40.073062810461614 74.75613543004918) bank94944 +94945 POINT(39.94974175722381 74.95810529740857) bank94945 +94946 POINT(40.203978228202885 73.30515320604408) bank94946 +94947 POINT(41.28949428029403 74.26603976679166) bank94947 +94948 POINT(41.49065032448694 73.7931086989471) bank94948 +94949 POINT(40.49244714921956 74.21548937332378) bank94949 +94950 POINT(41.69714009094678 74.74593419416355) bank94950 +94951 POINT(40.50262428532845 73.21156511592477) bank94951 +94952 POINT(41.088915640148926 74.57845937745361) bank94952 +94953 POINT(41.53663338707551 74.84542127576624) bank94953 +94954 POINT(41.135704414799 74.28478836154555) bank94954 +94955 POINT(41.53597164582146 73.83580346932537) bank94955 +94956 POINT(41.122433510214165 74.79306716532192) bank94956 +94957 POINT(39.994835034556345 74.0137293588567) bank94957 +94958 POINT(40.75525670846669 73.66390279788828) bank94958 +94959 POINT(41.64427345365147 73.75511351362216) bank94959 +94960 POINT(39.996269749498886 73.32321306587592) bank94960 +94961 POINT(40.32330231512279 73.7602473381866) bank94961 +94962 POINT(41.51765931247626 74.92597679024047) bank94962 +94963 POINT(40.89193087142512 74.51814830241634) bank94963 +94964 POINT(41.3077516210163 73.98229350872003) bank94964 +94965 POINT(41.48112479729148 74.37073486843795) bank94965 +94966 POINT(39.95631448325763 73.98931103304137) bank94966 +94967 POINT(40.37101201369288 74.74001344277552) bank94967 +94968 POINT(40.09313503357641 73.99090679201578) bank94968 +94969 POINT(41.00355209141665 74.58680577614204) bank94969 +94970 POINT(40.9529403843079 74.98558360395586) bank94970 +94971 POINT(40.408063215908506 73.78605149054464) bank94971 +94972 POINT(41.560686788716865 74.6065396688446) bank94972 +94973 POINT(40.16413376639344 73.84826700972287) bank94973 +94974 POINT(40.12731277597285 73.9472917545297) bank94974 +94975 POINT(40.84604950744958 74.21762179783607) bank94975 +94976 POINT(40.60364486795607 74.30989579105989) bank94976 +94977 POINT(40.134914524566454 74.78859821120163) bank94977 +94978 POINT(39.97199000792108 73.47842453918925) bank94978 +94979 POINT(40.09053200782492 73.37216937339426) bank94979 +94980 POINT(41.30763607916531 73.91019362675246) bank94980 +94981 POINT(41.276116789479104 74.32525240404291) bank94981 +94982 POINT(41.11128935856414 74.14252558167426) bank94982 +94983 POINT(39.93338278251013 74.72651484965554) bank94983 +94984 POINT(41.21010986195398 73.05292078639566) bank94984 +94985 POINT(39.87416678786501 74.24428408700854) bank94985 +94986 POINT(40.85821691969545 74.65584846093283) bank94986 +94987 POINT(41.60254943337871 74.27210678820445) bank94987 +94988 POINT(40.42005135774215 74.41609779983061) bank94988 +94989 POINT(40.9890202999377 74.3704412254652) bank94989 +94990 POINT(40.98883968967366 74.17731677867049) bank94990 +94991 POINT(40.03608622137119 73.09665487445646) bank94991 +94992 POINT(40.33237402468662 73.85787344878445) bank94992 +94993 POINT(40.49429947603106 73.76305960460071) bank94993 +94994 POINT(40.844662522720974 73.19650723439091) bank94994 +94995 POINT(39.996689072135325 74.62103884589484) bank94995 +94996 POINT(40.0741364294546 73.30333129439737) bank94996 +94997 POINT(41.22864160639639 74.70098307616114) bank94997 +94998 POINT(41.15074796248527 73.66926115236978) bank94998 +94999 POINT(40.419281086999305 74.87879969316398) bank94999 +95000 POINT(41.53895895805522 73.21682470199208) bank95000 +95001 POINT(41.55402565757171 73.53596970272652) bank95001 +95002 POINT(40.97693419936918 73.70368420985515) bank95002 +95003 POINT(41.62943981497429 74.42766011948217) bank95003 +95004 POINT(41.31909454431794 73.20767151989699) bank95004 +95005 POINT(41.65436041332257 73.66010357155777) bank95005 +95006 POINT(40.133852071305384 73.22800574836958) bank95006 +95007 POINT(40.29530294424851 73.9888648509566) bank95007 +95008 POINT(41.49083418707254 74.49068009761663) bank95008 +95009 POINT(41.685074603759155 73.59294388821255) bank95009 +95010 POINT(39.8534794615084 74.47109700704678) bank95010 +95011 POINT(40.78037380887498 73.98001219888823) bank95011 +95012 POINT(41.614977596555256 74.8808311865225) bank95012 +95013 POINT(40.21302982643949 73.05391741160342) bank95013 +95014 POINT(40.60105246452696 73.17295687536215) bank95014 +95015 POINT(40.64362669761018 73.0323400662385) bank95015 +95016 POINT(41.47094952238004 74.06759488405059) bank95016 +95017 POINT(40.86627879012935 73.28874521646958) bank95017 +95018 POINT(40.45416606567931 73.87266585622268) bank95018 +95019 POINT(40.433529466504076 73.53322159398869) bank95019 +95020 POINT(41.29497377307828 74.75428208430995) bank95020 +95021 POINT(40.282493025852354 74.25465208613913) bank95021 +95022 POINT(41.439788388460755 74.20421596439346) bank95022 +95023 POINT(40.370470760366246 73.193809874269) bank95023 +95024 POINT(41.659088613335236 73.22013113595033) bank95024 +95025 POINT(40.56524483176628 73.74747322623556) bank95025 +95026 POINT(40.10269084492062 74.16204853019522) bank95026 +95027 POINT(39.922470227982814 73.42110001222406) bank95027 +95028 POINT(41.17135674510502 73.12553778790434) bank95028 +95029 POINT(40.74281735681665 73.37360373109651) bank95029 +95030 POINT(41.035748618986595 73.95190798237482) bank95030 +95031 POINT(40.21667805274227 73.86125520473719) bank95031 +95032 POINT(40.042107801018645 75.0017374188353) bank95032 +95033 POINT(41.19596167625378 73.68558339333138) bank95033 +95034 POINT(40.55615386131625 73.60993568469387) bank95034 +95035 POINT(41.51676890674158 74.33460471320973) bank95035 +95036 POINT(40.605925803941226 74.16306097229207) bank95036 +95037 POINT(39.85176846074308 74.07654493902871) bank95037 +95038 POINT(41.15694082674752 73.56282498767682) bank95038 +95039 POINT(40.57336758062704 74.36816775780565) bank95039 +95040 POINT(40.26263739307674 73.98828014355158) bank95040 +95041 POINT(41.19484416250013 73.68420624451377) bank95041 +95042 POINT(40.17843936265237 74.94191941796127) bank95042 +95043 POINT(40.05032557270303 73.82829977039621) bank95043 +95044 POINT(39.980949826700076 74.04765614597723) bank95044 +95045 POINT(40.86404356551847 73.24713932763486) bank95045 +95046 POINT(40.62755810097421 74.04575679153358) bank95046 +95047 POINT(40.25531649665988 74.56056204136218) bank95047 +95048 POINT(40.63190787178645 73.23970876036682) bank95048 +95049 POINT(40.70514185610745 73.59091739470568) bank95049 +95050 POINT(39.95794300907613 73.74397135580784) bank95050 +95051 POINT(41.07171711758552 74.65884356110833) bank95051 +95052 POINT(40.74750354520703 73.41693497383406) bank95052 +95053 POINT(40.472836769992945 73.75822079465377) bank95053 +95054 POINT(39.97020873728231 73.97000723967726) bank95054 +95055 POINT(41.28519768784009 73.7032415619786) bank95055 +95056 POINT(40.754745278415285 73.63808356020094) bank95056 +95057 POINT(41.01128587115056 74.25352072757504) bank95057 +95058 POINT(41.11673405802601 74.9450768430995) bank95058 +95059 POINT(40.01272173557174 73.95631079485555) bank95059 +95060 POINT(41.253565870304726 74.77299792567771) bank95060 +95061 POINT(41.02003556433562 73.91286400134415) bank95061 +95062 POINT(41.23214410168135 74.45905570874189) bank95062 +95063 POINT(41.43861194700565 74.05514531527818) bank95063 +95064 POINT(41.56884861718665 73.73914959143973) bank95064 +95065 POINT(41.7027557913044 73.32955511295896) bank95065 +95066 POINT(41.681759291684976 74.1762350903663) bank95066 +95067 POINT(41.17496079875565 73.11083028120724) bank95067 +95068 POINT(40.90503232272932 73.35240346145575) bank95068 +95069 POINT(40.09327340372536 73.91073300916577) bank95069 +95070 POINT(41.1426791348915 73.11692914780755) bank95070 +95071 POINT(41.15859695940265 73.18215156155071) bank95071 +95072 POINT(41.50871462299023 74.91580864403888) bank95072 +95073 POINT(40.544429979565216 73.49046645568934) bank95073 +95074 POINT(40.28748043165738 73.32615523804917) bank95074 +95075 POINT(41.493980530313436 73.96564593538719) bank95075 +95076 POINT(41.55481712372579 74.1165295957446) bank95076 +95077 POINT(39.99110462836987 73.3665507493221) bank95077 +95078 POINT(41.332196001695905 74.92172182243377) bank95078 +95079 POINT(40.48783226194593 73.0717821619736) bank95079 +95080 POINT(40.09262731517264 74.7806833466328) bank95080 +95081 POINT(40.07820260849869 73.31749794642504) bank95081 +95082 POINT(41.495505348794595 74.4581061385244) bank95082 +95083 POINT(41.50474758667518 73.32078810739685) bank95083 +95084 POINT(41.45019138036993 73.8832842573294) bank95084 +95085 POINT(41.42809593482738 73.97299094968479) bank95085 +95086 POINT(40.08213020694645 74.63385732248811) bank95086 +95087 POINT(41.09628307424759 74.6632248193566) bank95087 +95088 POINT(41.455235008851375 74.84621267739583) bank95088 +95089 POINT(40.67805484596421 74.31731145862702) bank95089 +95090 POINT(40.089418751214545 73.4308484573678) bank95090 +95091 POINT(41.23289418367624 74.70339738245553) bank95091 +95092 POINT(41.05671813472786 73.42508949278387) bank95092 +95093 POINT(39.740742728257246 74.37982753430943) bank95093 +95094 POINT(39.88672543060824 74.06172509984579) bank95094 +95095 POINT(41.40923821795246 73.1930341164559) bank95095 +95096 POINT(41.07346924325775 74.97221795270123) bank95096 +95097 POINT(40.97578908738737 74.59419090928345) bank95097 +95098 POINT(40.27337627020646 73.65567473646827) bank95098 +95099 POINT(41.209508736872614 73.87997134490601) bank95099 +95100 POINT(41.18148673428551 73.3662157305119) bank95100 +95101 POINT(40.416058139407426 74.86264185537206) bank95101 +95102 POINT(39.88421524373921 73.49567416499784) bank95102 +95103 POINT(39.75492570438994 73.31959341666209) bank95103 +95104 POINT(40.36506878249552 74.42321065901999) bank95104 +95105 POINT(41.551048474495296 73.4840961018028) bank95105 +95106 POINT(41.69850783196862 73.10493471494522) bank95106 +95107 POINT(41.03976139166945 74.48000572989989) bank95107 +95108 POINT(40.17813364997208 73.33914763963017) bank95108 +95109 POINT(40.34849543074625 73.27848468850055) bank95109 +95110 POINT(40.21228993742843 73.6113678691501) bank95110 +95111 POINT(40.64673707436943 73.09396190662231) bank95111 +95112 POINT(41.13043505748134 73.68488817285521) bank95112 +95113 POINT(39.93912922914397 74.43920566132934) bank95113 +95114 POINT(41.12424157994462 73.49638531183412) bank95114 +95115 POINT(41.462553103890464 73.6226558059694) bank95115 +95116 POINT(40.00489546818384 73.77305148969582) bank95116 +95117 POINT(40.40810381350681 73.06010839233272) bank95117 +95118 POINT(41.01134066321952 73.06193896118226) bank95118 +95119 POINT(41.437857956366315 73.05468803562388) bank95119 +95120 POINT(39.718963485948066 74.4796531000745) bank95120 +95121 POINT(39.81523850210079 74.16787771246346) bank95121 +95122 POINT(40.700679656248774 73.68267330963434) bank95122 +95123 POINT(40.688750946618406 74.7103829147139) bank95123 +95124 POINT(40.38101813613909 73.7571253853776) bank95124 +95125 POINT(41.53481977191164 74.82970602463396) bank95125 +95126 POINT(39.889388006258116 74.35272558334216) bank95126 +95127 POINT(40.82766975506985 74.96075773712764) bank95127 +95128 POINT(41.185395014715766 74.69657588371147) bank95128 +95129 POINT(40.524265703987446 73.29362617460515) bank95129 +95130 POINT(40.420370879314 73.44069664182146) bank95130 +95131 POINT(40.6811113599961 73.22489595236347) bank95131 +95132 POINT(40.20652354015669 74.3771092991491) bank95132 +95133 POINT(40.95093330999809 73.94547157417085) bank95133 +95134 POINT(41.278280678316825 73.07171864026341) bank95134 +95135 POINT(40.53448952911842 74.12217083126704) bank95135 +95136 POINT(39.743066824081644 73.74765369548591) bank95136 +95137 POINT(39.725506323109066 74.41142496799644) bank95137 +95138 POINT(40.434526026264926 74.34888529661289) bank95138 +95139 POINT(40.058276061125284 74.64154212904084) bank95139 +95140 POINT(41.03512975433287 74.01010007207628) bank95140 +95141 POINT(41.305056478631464 73.21803597664315) bank95141 +95142 POINT(41.176863170759376 74.91836406999634) bank95142 +95143 POINT(40.93227227377089 73.77003563531913) bank95143 +95144 POINT(41.19164294723922 73.6516708238525) bank95144 +95145 POINT(40.17235977838083 74.30345853951246) bank95145 +95146 POINT(40.884132511954185 74.4331186770268) bank95146 +95147 POINT(41.204543059184 73.55969593262334) bank95147 +95148 POINT(40.163304135405525 73.8558922227814) bank95148 +95149 POINT(40.12496128987771 74.08780505599502) bank95149 +95150 POINT(40.422618354233876 73.31584686036632) bank95150 +95151 POINT(40.71967416059761 74.36485186569794) bank95151 +95152 POINT(40.17576503993611 74.82621804083242) bank95152 +95153 POINT(41.34241618865038 73.27530151515346) bank95153 +95154 POINT(40.53042906364588 74.19227440425942) bank95154 +95155 POINT(40.571382072320404 74.71865787654977) bank95155 +95156 POINT(40.43826279126121 74.8963057203466) bank95156 +95157 POINT(41.662041753907396 73.29451618442586) bank95157 +95158 POINT(41.02337564897964 74.71396223709623) bank95158 +95159 POINT(39.86460108106261 74.70788625036074) bank95159 +95160 POINT(41.49012952662958 73.07113838836548) bank95160 +95161 POINT(40.642058495478096 74.91097113517185) bank95161 +95162 POINT(41.69674261205548 74.01738327964871) bank95162 +95163 POINT(41.25731771163766 74.37967811339021) bank95163 +95164 POINT(40.22005206454608 74.95258948761953) bank95164 +95165 POINT(40.15858992154701 73.47226716462184) bank95165 +95166 POINT(41.026784037005655 73.4123508384528) bank95166 +95167 POINT(41.567604439855145 74.71018038161934) bank95167 +95168 POINT(40.46072776222704 73.60700387534233) bank95168 +95169 POINT(40.31644736443798 73.81819944059698) bank95169 +95170 POINT(39.83808808173828 73.18123385721506) bank95170 +95171 POINT(41.26805790568541 74.25845464140355) bank95171 +95172 POINT(41.103147601144315 73.26812435773427) bank95172 +95173 POINT(41.54137676872944 74.29048166657381) bank95173 +95174 POINT(40.93259548481478 74.67996920868595) bank95174 +95175 POINT(41.62913830445071 73.06083656216263) bank95175 +95176 POINT(41.360263815696165 73.26049888290464) bank95176 +95177 POINT(40.086962139439 73.43862254377099) bank95177 +95178 POINT(40.190180275814534 74.3029991794883) bank95178 +95179 POINT(40.770469490131134 73.73535219301688) bank95179 +95180 POINT(41.67364982666376 73.49532370002865) bank95180 +95181 POINT(41.21904518973136 73.491281968705) bank95181 +95182 POINT(39.86217696242666 74.4224835090055) bank95182 +95183 POINT(40.088843728409124 73.43816983172674) bank95183 +95184 POINT(39.9066777396137 74.95485758707561) bank95184 +95185 POINT(40.155091290418355 74.41647734136518) bank95185 +95186 POINT(41.581299707789064 74.54572910319784) bank95186 +95187 POINT(41.33324266360774 73.2035207154108) bank95187 +95188 POINT(41.332976355713605 74.65895560810057) bank95188 +95189 POINT(40.327132738434884 73.00964735370734) bank95189 +95190 POINT(41.200104985734754 74.2118902301151) bank95190 +95191 POINT(41.6350662718911 73.56210870232793) bank95191 +95192 POINT(39.77435606291363 74.32244521059494) bank95192 +95193 POINT(40.10124329700515 73.0620448365566) bank95193 +95194 POINT(39.8786395018347 73.43679734064112) bank95194 +95195 POINT(41.271560539137376 74.73617837648321) bank95195 +95196 POINT(40.210319350981266 74.9082109486887) bank95196 +95197 POINT(40.31421024941081 74.11444296905063) bank95197 +95198 POINT(40.24033622196205 73.65156008222125) bank95198 +95199 POINT(41.08894843733555 74.07325158938397) bank95199 +95200 POINT(41.65838561851492 74.69071859114756) bank95200 +95201 POINT(39.89949674017663 73.58847120149791) bank95201 +95202 POINT(40.761576042963526 73.56268517965316) bank95202 +95203 POINT(41.30767849202971 73.43182211329449) bank95203 +95204 POINT(40.199294271283705 73.47157148359153) bank95204 +95205 POINT(40.15227012570105 73.52539195489183) bank95205 +95206 POINT(41.16572107601353 73.43141254275625) bank95206 +95207 POINT(39.84988876209904 74.71934738637133) bank95207 +95208 POINT(40.58735747070301 74.84959866920204) bank95208 +95209 POINT(40.51468652192947 74.70603447835737) bank95209 +95210 POINT(40.18245751332387 73.60168595203145) bank95210 +95211 POINT(41.229672435939726 73.99104469232587) bank95211 +95212 POINT(41.41426251494727 74.60877135300272) bank95212 +95213 POINT(41.221171623945196 74.72540782676303) bank95213 +95214 POINT(40.98103625220085 73.55338761829758) bank95214 +95215 POINT(41.64336921405276 73.23165119138211) bank95215 +95216 POINT(40.6163964169595 73.16388613478722) bank95216 +95217 POINT(39.75963338135878 73.29187021769323) bank95217 +95218 POINT(41.56424178371209 73.21893385627645) bank95218 +95219 POINT(41.7020373543886 74.39535921864599) bank95219 +95220 POINT(40.90000588100741 73.07936116294941) bank95220 +95221 POINT(39.89808373146575 73.46536344104827) bank95221 +95222 POINT(41.39061880122734 74.2614206267639) bank95222 +95223 POINT(40.63248878649037 74.3430386490005) bank95223 +95224 POINT(40.15407885589084 73.5211889499974) bank95224 +95225 POINT(41.245770011350984 74.28875386200669) bank95225 +95226 POINT(40.65903122113501 74.81429075512091) bank95226 +95227 POINT(41.20075350028618 73.30306296366524) bank95227 +95228 POINT(40.5776603836529 74.40440494967417) bank95228 +95229 POINT(41.137093574564204 74.29929082871423) bank95229 +95230 POINT(40.46224695219876 74.23213272024572) bank95230 +95231 POINT(40.37011435003501 74.0837903686916) bank95231 +95232 POINT(40.02010555566103 74.61520837436251) bank95232 +95233 POINT(40.70366888703317 74.11928001601191) bank95233 +95234 POINT(39.84303529303046 73.09585406140134) bank95234 +95235 POINT(39.829725713794716 73.82126638379083) bank95235 +95236 POINT(41.49847186321929 73.54372392217753) bank95236 +95237 POINT(39.82337822468815 74.57645162622944) bank95237 +95238 POINT(40.453721101404966 74.81371439408288) bank95238 +95239 POINT(40.642047307462484 73.531616586113) bank95239 +95240 POINT(41.164388630879685 73.52593161951005) bank95240 +95241 POINT(40.21668371747959 73.91752180331541) bank95241 +95242 POINT(40.31372657156599 74.70711229208638) bank95242 +95243 POINT(40.60475875901667 74.12166117177833) bank95243 +95244 POINT(41.102232048751695 73.11476035543318) bank95244 +95245 POINT(41.70820350366023 74.56365871793558) bank95245 +95246 POINT(40.347519256073575 73.3637833046645) bank95246 +95247 POINT(39.803526854584796 74.25956607920284) bank95247 +95248 POINT(40.94663720824932 73.58623147983708) bank95248 +95249 POINT(41.49883958645754 73.10826519954004) bank95249 +95250 POINT(41.48675337532539 74.28328409665697) bank95250 +95251 POINT(41.2383519435139 73.6307823075555) bank95251 +95252 POINT(40.67528696368946 73.04310555547275) bank95252 +95253 POINT(40.623612387446094 73.39457719214703) bank95253 +95254 POINT(41.387286874372435 73.35454693341845) bank95254 +95255 POINT(40.671456609052825 73.47346934771745) bank95255 +95256 POINT(40.03917023154394 74.91269880014461) bank95256 +95257 POINT(41.30280092653843 74.5579270721795) bank95257 +95258 POINT(41.63632465154085 73.57777778857361) bank95258 +95259 POINT(40.37037027252344 73.85401123216032) bank95259 +95260 POINT(41.24297078721467 73.65311245867281) bank95260 +95261 POINT(41.69641411674145 73.17795047269256) bank95261 +95262 POINT(40.78323119269386 73.59096344145517) bank95262 +95263 POINT(39.85550793677896 73.39444087570317) bank95263 +95264 POINT(41.64180642770922 74.02039693600645) bank95264 +95265 POINT(40.844003956210585 74.87512962546771) bank95265 +95266 POINT(40.15456732450231 73.59561681042217) bank95266 +95267 POINT(39.81894440327342 73.9677000772844) bank95267 +95268 POINT(40.209652204355194 73.15448956741784) bank95268 +95269 POINT(40.61340826287536 73.07707219909197) bank95269 +95270 POINT(39.94915818826081 74.82195315915723) bank95270 +95271 POINT(40.976695515765684 74.14250593215132) bank95271 +95272 POINT(39.91241932630041 74.0392770066246) bank95272 +95273 POINT(40.25049221971595 73.98254782269557) bank95273 +95274 POINT(39.95700932161871 74.42057896563273) bank95274 +95275 POINT(39.74208031105392 74.81167401923076) bank95275 +95276 POINT(40.39876538963508 73.0549311855987) bank95276 +95277 POINT(41.20694443014081 73.02758904103703) bank95277 +95278 POINT(41.45982568267619 74.95545071628267) bank95278 +95279 POINT(39.788113968445515 74.70145196745774) bank95279 +95280 POINT(40.15263664293037 74.69823003251902) bank95280 +95281 POINT(40.51806278854361 73.01700013278224) bank95281 +95282 POINT(40.81097979014103 73.32933626833838) bank95282 +95283 POINT(40.66351617206847 74.47580971765245) bank95283 +95284 POINT(41.56976986619171 74.67132868185017) bank95284 +95285 POINT(40.61151577043926 73.89666008951264) bank95285 +95286 POINT(40.89595064182396 74.65736019989356) bank95286 +95287 POINT(40.07081103095993 74.16139603270716) bank95287 +95288 POINT(40.644078949669606 73.75336346430842) bank95288 +95289 POINT(40.56649631282834 73.87456827120288) bank95289 +95290 POINT(40.630814310747084 74.04531757069618) bank95290 +95291 POINT(41.67483413413631 74.20708829849835) bank95291 +95292 POINT(40.27659921966813 73.49114254676475) bank95292 +95293 POINT(41.40253653552346 74.26966508005776) bank95293 +95294 POINT(41.43745254568276 73.91431970451993) bank95294 +95295 POINT(39.90764804539455 73.41347503851485) bank95295 +95296 POINT(39.774433063708095 73.93595036447327) bank95296 +95297 POINT(39.71452704644474 74.29524047543379) bank95297 +95298 POINT(40.444001234661506 74.61370347363571) bank95298 +95299 POINT(39.77409530422581 74.7594245563193) bank95299 +95300 POINT(40.07605774054768 73.9873421722687) bank95300 +95301 POINT(40.6358826719144 74.61438078191348) bank95301 +95302 POINT(40.670060466133535 73.05916040299837) bank95302 +95303 POINT(40.31954775874752 73.75215469655711) bank95303 +95304 POINT(40.54237419927264 73.68826420764023) bank95304 +95305 POINT(41.69360835752232 74.75921981504136) bank95305 +95306 POINT(40.21364813083403 74.72837891375237) bank95306 +95307 POINT(39.81696783914924 74.60335990385968) bank95307 +95308 POINT(41.092463083126404 73.87604368898735) bank95308 +95309 POINT(41.690031657731915 74.72532005806292) bank95309 +95310 POINT(40.65357223271558 73.78373875236268) bank95310 +95311 POINT(40.55947141287612 73.2859116540768) bank95311 +95312 POINT(41.24512065781516 74.9336863247232) bank95312 +95313 POINT(41.08548817117237 74.93306273562986) bank95313 +95314 POINT(41.641859702700295 74.09316085143523) bank95314 +95315 POINT(41.31293280459097 74.44647902817982) bank95315 +95316 POINT(40.881985123240966 74.32788039042865) bank95316 +95317 POINT(40.53671993451471 74.35359546438039) bank95317 +95318 POINT(41.20378296817204 74.96638037039568) bank95318 +95319 POINT(40.48957313887892 73.4335312722685) bank95319 +95320 POINT(40.99933340615606 73.9438992287947) bank95320 +95321 POINT(40.595596834940636 74.55017167801336) bank95321 +95322 POINT(41.16534114642757 73.16238566533197) bank95322 +95323 POINT(39.85323748523582 74.33143880492982) bank95323 +95324 POINT(40.11767566799606 73.49104296448455) bank95324 +95325 POINT(39.772573177982146 74.61604155961392) bank95325 +95326 POINT(40.754371335422405 73.00915578869393) bank95326 +95327 POINT(40.45757998714893 74.67734278364034) bank95327 +95328 POINT(41.6110847605884 75.00353990160984) bank95328 +95329 POINT(40.97378601035562 73.12297465862981) bank95329 +95330 POINT(40.43144174408775 74.85213788806831) bank95330 +95331 POINT(40.53815591143912 73.33669673407168) bank95331 +95332 POINT(41.39127218368493 73.66840875207767) bank95332 +95333 POINT(40.85435535210236 74.06862817803936) bank95333 +95334 POINT(39.96568187496114 74.06038184577137) bank95334 +95335 POINT(41.36285668255984 73.5107946741614) bank95335 +95336 POINT(39.782205662409275 73.87120990320793) bank95336 +95337 POINT(40.31658523152495 73.43152591148463) bank95337 +95338 POINT(40.32647494781719 73.46445517193462) bank95338 +95339 POINT(40.44420009323758 73.3945008849542) bank95339 +95340 POINT(41.1423973208669 74.75810510553319) bank95340 +95341 POINT(40.98613707969091 74.21545947014148) bank95341 +95342 POINT(40.545369873372486 73.27261396293041) bank95342 +95343 POINT(41.36859522283259 73.16035947138144) bank95343 +95344 POINT(40.73619850744441 74.35294684540987) bank95344 +95345 POINT(40.48929888343278 73.47630712202479) bank95345 +95346 POINT(40.8458640897248 74.53094612494947) bank95346 +95347 POINT(40.65242642401835 75.00226344638986) bank95347 +95348 POINT(40.92518814001001 73.80223631845625) bank95348 +95349 POINT(41.242314318587304 74.12611293922255) bank95349 +95350 POINT(41.06639266512969 73.18189273958579) bank95350 +95351 POINT(40.08812848584791 73.04237222307273) bank95351 +95352 POINT(40.58932014987282 73.60824116131795) bank95352 +95353 POINT(41.103657101655436 73.85256543258203) bank95353 +95354 POINT(40.32447197159824 74.98549009753133) bank95354 +95355 POINT(39.9811079865766 73.39370535624361) bank95355 +95356 POINT(39.939205874928184 73.39999306439096) bank95356 +95357 POINT(40.101470846562115 73.51904421536736) bank95357 +95358 POINT(40.62776024509067 74.16803187992406) bank95358 +95359 POINT(40.01864807167339 74.88098668363546) bank95359 +95360 POINT(40.06088553922337 73.01918884709981) bank95360 +95361 POINT(40.52108917195107 74.92876162588018) bank95361 +95362 POINT(40.922924286216436 74.4991524657378) bank95362 +95363 POINT(40.32589819516967 74.77227179300023) bank95363 +95364 POINT(41.68233560645462 74.61556464769374) bank95364 +95365 POINT(41.47627501103219 73.86768357865844) bank95365 +95366 POINT(41.01789361666393 74.40889908523154) bank95366 +95367 POINT(40.47208893973078 73.82180896410979) bank95367 +95368 POINT(40.84176761351282 74.09318249942241) bank95368 +95369 POINT(39.7549719808689 74.2138254137279) bank95369 +95370 POINT(39.91832805486117 73.26299602200841) bank95370 +95371 POINT(41.61961197763838 73.43171452760225) bank95371 +95372 POINT(41.25050567032458 73.95156631116164) bank95372 +95373 POINT(40.296568190769015 73.71021330565745) bank95373 +95374 POINT(41.21277915102046 74.26446441625403) bank95374 +95375 POINT(40.80147168128511 74.17694915775148) bank95375 +95376 POINT(40.80392731049719 74.47912507256905) bank95376 +95377 POINT(40.262978021507976 73.67685729315971) bank95377 +95378 POINT(41.43685394652561 74.87441125150711) bank95378 +95379 POINT(39.79746573510191 73.83725600393318) bank95379 +95380 POINT(41.160371368619536 74.83320980439247) bank95380 +95381 POINT(41.45726398057822 74.69152780991068) bank95381 +95382 POINT(39.83562796074725 74.89422621945207) bank95382 +95383 POINT(39.96411219933249 73.44801904862315) bank95383 +95384 POINT(40.997851804064524 73.16297698154892) bank95384 +95385 POINT(40.15676556961373 74.39715967323568) bank95385 +95386 POINT(40.277381598437955 74.13001784997593) bank95386 +95387 POINT(39.985060045668995 74.93023150310937) bank95387 +95388 POINT(40.07426354257913 74.882954931246) bank95388 +95389 POINT(39.85917757560517 74.64398701773831) bank95389 +95390 POINT(40.73112000222751 74.63600683682635) bank95390 +95391 POINT(41.55314637259802 74.74529871601975) bank95391 +95392 POINT(40.869301069412586 73.05516709241596) bank95392 +95393 POINT(40.808406924485986 73.21582131104935) bank95393 +95394 POINT(41.067309964905235 73.82117674047446) bank95394 +95395 POINT(41.569462931248246 73.41734636245293) bank95395 +95396 POINT(40.33185483803299 73.3610083519004) bank95396 +95397 POINT(39.88492709170298 74.94145190860479) bank95397 +95398 POINT(40.10958601612317 73.56037957123337) bank95398 +95399 POINT(40.7954921326845 74.28585137132697) bank95399 +95400 POINT(41.1596638745855 73.53417692219679) bank95400 +95401 POINT(41.46210680827725 74.93798518060493) bank95401 +95402 POINT(40.89932859673714 74.73495379983414) bank95402 +95403 POINT(41.700129072140015 74.71313038069238) bank95403 +95404 POINT(41.568420187973544 73.31556239396919) bank95404 +95405 POINT(41.07154115074895 73.22971016715054) bank95405 +95406 POINT(40.51193737733985 74.92476146961786) bank95406 +95407 POINT(39.826486044512905 74.70712406862563) bank95407 +95408 POINT(40.348384868021604 73.59894599035904) bank95408 +95409 POINT(39.9236172224418 74.41818735944669) bank95409 +95410 POINT(40.81311342717407 74.57893694434152) bank95410 +95411 POINT(40.14237883274682 74.26447131440712) bank95411 +95412 POINT(40.752153825892194 74.91587251578768) bank95412 +95413 POINT(40.287614243987136 74.55097477006444) bank95413 +95414 POINT(40.232101870669105 73.5878135745687) bank95414 +95415 POINT(40.86014843678223 74.8233858362267) bank95415 +95416 POINT(41.20328145378293 73.01327247823367) bank95416 +95417 POINT(41.54753493171146 73.67110281033212) bank95417 +95418 POINT(41.103978596026366 74.13093287943818) bank95418 +95419 POINT(41.095216657615644 73.39889136278867) bank95419 +95420 POINT(40.61182959891246 74.64290142438504) bank95420 +95421 POINT(40.549053945289195 73.39390722869418) bank95421 +95422 POINT(40.84111277118921 74.36368769276349) bank95422 +95423 POINT(40.160647348704124 74.72832930885372) bank95423 +95424 POINT(39.832620552749304 74.64097483897206) bank95424 +95425 POINT(40.19195577147563 73.68634945376711) bank95425 +95426 POINT(39.77597803447948 73.2749880871314) bank95426 +95427 POINT(41.52902334764067 73.80665349428988) bank95427 +95428 POINT(40.827583865525206 73.53967475783786) bank95428 +95429 POINT(41.44622473899223 74.54305359568939) bank95429 +95430 POINT(40.29922848118862 73.09170263720088) bank95430 +95431 POINT(40.95302946093499 74.2039748249601) bank95431 +95432 POINT(41.504810516149206 73.19916708735077) bank95432 +95433 POINT(40.6561826673814 73.65343440121498) bank95433 +95434 POINT(41.56385775565167 74.37700802551329) bank95434 +95435 POINT(41.470633976175385 73.82077314397162) bank95435 +95436 POINT(39.93856412724648 74.46767181269118) bank95436 +95437 POINT(40.81512221470193 73.2624298960729) bank95437 +95438 POINT(41.5364155655324 74.49281641128296) bank95438 +95439 POINT(40.12649089816934 74.5150879267591) bank95439 +95440 POINT(41.54190023850138 74.70067378085123) bank95440 +95441 POINT(40.53116102497739 74.27724852042225) bank95441 +95442 POINT(40.15776414631411 74.45160385180627) bank95442 +95443 POINT(40.81585182334945 73.77016459954271) bank95443 +95444 POINT(40.42072326674564 74.58740563914672) bank95444 +95445 POINT(40.16709014932966 73.76474775868729) bank95445 +95446 POINT(39.90990729151855 73.81669460245497) bank95446 +95447 POINT(40.95414108689314 73.1174376164042) bank95447 +95448 POINT(40.168928115647226 74.78212562270674) bank95448 +95449 POINT(40.93712776386726 73.0069499487646) bank95449 +95450 POINT(40.50052840242177 73.92204525008407) bank95450 +95451 POINT(40.35312478388003 73.39315916583287) bank95451 +95452 POINT(40.118629260935315 73.17632115917573) bank95452 +95453 POINT(40.65839836137075 74.73742566186311) bank95453 +95454 POINT(40.15811510134786 73.42260303718456) bank95454 +95455 POINT(40.4961441130223 74.89458758899602) bank95455 +95456 POINT(41.670359452675925 74.43573960715348) bank95456 +95457 POINT(41.296905767834474 73.25224165784137) bank95457 +95458 POINT(40.10750941262297 74.17982774656929) bank95458 +95459 POINT(41.0031375811731 74.70832356556075) bank95459 +95460 POINT(39.768487671684966 73.81168795450218) bank95460 +95461 POINT(40.982184146619765 74.19583629020701) bank95461 +95462 POINT(41.44049927405357 74.11309072905581) bank95462 +95463 POINT(41.46530654629614 74.94799692726282) bank95463 +95464 POINT(41.505307419848045 74.97945375654729) bank95464 +95465 POINT(41.664054572833024 73.27880720567029) bank95465 +95466 POINT(39.77879496758561 73.462213156497) bank95466 +95467 POINT(40.85835931794634 74.91959018429263) bank95467 +95468 POINT(41.315655121678915 74.83689955813625) bank95468 +95469 POINT(40.71452100830644 73.58846361894496) bank95469 +95470 POINT(40.819094749871105 73.65247459550957) bank95470 +95471 POINT(40.24638113696535 74.85919395951281) bank95471 +95472 POINT(40.657397645471406 73.27443849521185) bank95472 +95473 POINT(40.430931172009196 73.24812469260056) bank95473 +95474 POINT(39.88840989210394 74.70568672379078) bank95474 +95475 POINT(40.95201743437472 73.17643623271117) bank95475 +95476 POINT(41.36957334842505 73.7299750225858) bank95476 +95477 POINT(41.70972773337117 73.59120177005839) bank95477 +95478 POINT(41.325066201858256 74.43519462464833) bank95478 +95479 POINT(41.12446635159048 74.90846129579467) bank95479 +95480 POINT(39.99983007110397 74.05241337308107) bank95480 +95481 POINT(40.25313617542703 74.231548389601) bank95481 +95482 POINT(40.745759215959694 74.61131531653817) bank95482 +95483 POINT(40.82121605196729 74.41044780409726) bank95483 +95484 POINT(40.694496506034326 74.0916180601623) bank95484 +95485 POINT(40.72970930209167 74.83167742631994) bank95485 +95486 POINT(41.43749555035246 73.8551067064629) bank95486 +95487 POINT(41.53710757773823 74.5157830189159) bank95487 +95488 POINT(41.30441756909494 73.60495398290351) bank95488 +95489 POINT(40.94830967886445 73.7824701171554) bank95489 +95490 POINT(40.124930823293155 74.3476792487799) bank95490 +95491 POINT(39.751654246553834 73.5267588557859) bank95491 +95492 POINT(40.47778244755507 73.14864397873339) bank95492 +95493 POINT(40.83772518957255 73.39269127119198) bank95493 +95494 POINT(40.093802784568894 74.06090406226524) bank95494 +95495 POINT(41.26786575076302 73.06599646655171) bank95495 +95496 POINT(40.45748321823528 73.05840259613608) bank95496 +95497 POINT(39.90036631680858 73.63219120936533) bank95497 +95498 POINT(39.99461938100069 74.38893701863783) bank95498 +95499 POINT(40.0015486048864 73.28900676249924) bank95499 +95500 POINT(40.06902206355509 73.61572576377274) bank95500 +95501 POINT(41.11326789579708 73.5528807510909) bank95501 +95502 POINT(41.47269233550546 73.03067802244257) bank95502 +95503 POINT(40.6242083401744 73.45916010489674) bank95503 +95504 POINT(40.1966684006096 74.93537599631519) bank95504 +95505 POINT(41.43686863618705 74.67525630274568) bank95505 +95506 POINT(39.776691822958156 74.29444459890917) bank95506 +95507 POINT(40.057312622117614 73.65848479677857) bank95507 +95508 POINT(40.49561020493032 73.28116469461094) bank95508 +95509 POINT(40.15889513215686 73.66423250799765) bank95509 +95510 POINT(41.543778184609 74.07670354925327) bank95510 +95511 POINT(39.76258090114312 74.614770004782) bank95511 +95512 POINT(40.289354032617794 73.27400225508559) bank95512 +95513 POINT(40.211331194679794 73.16146129129937) bank95513 +95514 POINT(40.71652131728558 73.61346359190904) bank95514 +95515 POINT(41.41920074377517 74.88381300851181) bank95515 +95516 POINT(40.83474619558716 74.25601179026386) bank95516 +95517 POINT(40.631540337155066 74.07108376446317) bank95517 +95518 POINT(40.27477492984533 74.71889601784763) bank95518 +95519 POINT(39.9897676096164 74.97771248144612) bank95519 +95520 POINT(40.29382564461957 74.5363032729523) bank95520 +95521 POINT(40.25511097012943 73.61616548312172) bank95521 +95522 POINT(40.567360791492135 74.86407899364731) bank95522 +95523 POINT(41.19140506109996 74.60263334577945) bank95523 +95524 POINT(41.2792016446975 74.04871019740261) bank95524 +95525 POINT(39.8385272620772 74.94117988485833) bank95525 +95526 POINT(40.83102541590698 74.55184035258554) bank95526 +95527 POINT(40.42823297489531 74.55303593849537) bank95527 +95528 POINT(41.71179597206806 74.24082285423053) bank95528 +95529 POINT(41.037709176667406 73.29901449926741) bank95529 +95530 POINT(41.58168012573779 74.25427663093492) bank95530 +95531 POINT(40.583052847966556 74.80955058445839) bank95531 +95532 POINT(41.12635524080567 74.90442467576933) bank95532 +95533 POINT(41.04683500385584 74.17959865860026) bank95533 +95534 POINT(41.18745579946468 73.986043484828) bank95534 +95535 POINT(41.01735796224958 73.86961050001541) bank95535 +95536 POINT(40.02570940636831 74.60424815746) bank95536 +95537 POINT(41.030335571643455 74.47195613446196) bank95537 +95538 POINT(41.21940993957083 73.18920163742462) bank95538 +95539 POINT(41.0988423972083 74.9712309362032) bank95539 +95540 POINT(40.7215557885241 74.37155537979729) bank95540 +95541 POINT(41.01870258410261 74.78866910528815) bank95541 +95542 POINT(41.5524292437392 73.36178535548994) bank95542 +95543 POINT(41.042181038013275 73.19843459788578) bank95543 +95544 POINT(40.27056809921121 73.71914370759532) bank95544 +95545 POINT(39.998952824759535 74.32031325428785) bank95545 +95546 POINT(40.69602795633856 74.24434128715534) bank95546 +95547 POINT(40.563037567476286 74.65466784080448) bank95547 +95548 POINT(40.95973002846752 74.12591677847014) bank95548 +95549 POINT(40.67589815600112 74.88246605015746) bank95549 +95550 POINT(39.98576210621443 74.32754093889015) bank95550 +95551 POINT(41.42395168235735 73.12702940157487) bank95551 +95552 POINT(39.92943487241934 74.44496219655639) bank95552 +95553 POINT(41.479355681730176 74.55006468476806) bank95553 +95554 POINT(41.23027400563396 74.15932203294267) bank95554 +95555 POINT(39.84254140084779 73.0744428973872) bank95555 +95556 POINT(41.42096733189606 73.76063747865383) bank95556 +95557 POINT(40.494889169532 74.47847619144711) bank95557 +95558 POINT(41.18122215255739 73.43535318034598) bank95558 +95559 POINT(40.61063562055946 74.0406428370831) bank95559 +95560 POINT(40.625964825640054 74.43116962365367) bank95560 +95561 POINT(41.645063545939095 73.57605565656732) bank95561 +95562 POINT(40.075055183105206 73.48329972163285) bank95562 +95563 POINT(41.523984794575384 74.77955388953643) bank95563 +95564 POINT(41.21816903908829 73.7478848232859) bank95564 +95565 POINT(39.915400907629305 73.20045952440638) bank95565 +95566 POINT(41.22145678000777 73.02564597769957) bank95566 +95567 POINT(40.33533413291886 73.6857966048639) bank95567 +95568 POINT(41.462859302681935 73.63691311163747) bank95568 +95569 POINT(40.451530083636456 73.05478750840051) bank95569 +95570 POINT(40.25682455149225 73.24941913855366) bank95570 +95571 POINT(39.890583971732404 73.3249063633076) bank95571 +95572 POINT(41.03450222996924 74.53333137009902) bank95572 +95573 POINT(40.83310432418097 74.5721859060958) bank95573 +95574 POINT(40.0758233461217 73.36870488924195) bank95574 +95575 POINT(40.02079048168083 73.45402487677613) bank95575 +95576 POINT(41.66275284469337 74.64238859421303) bank95576 +95577 POINT(41.54693587485802 74.56194971209158) bank95577 +95578 POINT(41.240307042336454 73.54267675833091) bank95578 +95579 POINT(40.1306223092868 74.7771902080878) bank95579 +95580 POINT(40.3483687158237 73.35095600616228) bank95580 +95581 POINT(41.55994451815492 73.08812523694749) bank95581 +95582 POINT(39.87540612536452 73.84464766635715) bank95582 +95583 POINT(40.315823614019386 74.04719277318151) bank95583 +95584 POINT(41.39114539715288 73.91443900125404) bank95584 +95585 POINT(40.67876434898949 74.16483443890894) bank95585 +95586 POINT(39.80826000070394 74.6196884366617) bank95586 +95587 POINT(40.90054481979964 73.26269307512266) bank95587 +95588 POINT(40.49585761925292 73.80344065779695) bank95588 +95589 POINT(40.420274432791146 73.2238323420233) bank95589 +95590 POINT(41.41756985402695 73.60559477185842) bank95590 +95591 POINT(40.076149991468995 74.39731264898161) bank95591 +95592 POINT(39.906840844420756 74.26790977120557) bank95592 +95593 POINT(40.43516565466853 74.30442570057694) bank95593 +95594 POINT(40.1064298741257 73.02649935013677) bank95594 +95595 POINT(40.37951844532795 74.31306399845352) bank95595 +95596 POINT(40.342708253230555 73.27194339500394) bank95596 +95597 POINT(40.27138629952717 73.06844486451695) bank95597 +95598 POINT(41.510666262859466 73.05448458315256) bank95598 +95599 POINT(41.15423904931478 74.30200800243261) bank95599 +95600 POINT(40.11173846096141 74.23731651023678) bank95600 +95601 POINT(40.80385132356279 74.571476834044) bank95601 +95602 POINT(41.16606754121994 74.65172493418558) bank95602 +95603 POINT(40.462816353395155 73.76991803214027) bank95603 +95604 POINT(40.27845136227731 73.87446676287405) bank95604 +95605 POINT(39.74865171945183 73.51899681834864) bank95605 +95606 POINT(40.9288940173987 74.0461816306646) bank95606 +95607 POINT(40.14248807238214 73.9879828520392) bank95607 +95608 POINT(39.73185074215314 73.07324263324702) bank95608 +95609 POINT(40.446603920572976 74.20810703141612) bank95609 +95610 POINT(40.243714111046344 73.59216929223413) bank95610 +95611 POINT(41.06075011589207 73.129832057915) bank95611 +95612 POINT(41.042008318731014 74.80529853020282) bank95612 +95613 POINT(41.25662782554395 73.15149311497821) bank95613 +95614 POINT(41.52495488084224 73.04240027925535) bank95614 +95615 POINT(41.26207793680042 74.75438746180069) bank95615 +95616 POINT(40.04088422027557 74.70442235228475) bank95616 +95617 POINT(41.4061589075442 73.8207529135165) bank95617 +95618 POINT(40.61793681026156 73.16303985794848) bank95618 +95619 POINT(40.980848077656724 74.84169372850255) bank95619 +95620 POINT(40.80684510972564 74.24073105344075) bank95620 +95621 POINT(40.21594163864977 74.73704750061181) bank95621 +95622 POINT(41.60190240822477 73.96440659157227) bank95622 +95623 POINT(39.71572937824122 74.29927873429887) bank95623 +95624 POINT(39.88568148994314 73.92247967574971) bank95624 +95625 POINT(40.62344195403898 73.69292313342363) bank95625 +95626 POINT(41.06990345867072 73.41469400181195) bank95626 +95627 POINT(40.405453169817996 74.15132531907724) bank95627 +95628 POINT(41.17162854381233 74.99496277110781) bank95628 +95629 POINT(40.25486206885548 74.84201452521216) bank95629 +95630 POINT(40.458181340314646 74.22532667823488) bank95630 +95631 POINT(40.088643045934546 74.67763491488577) bank95631 +95632 POINT(40.106377995644635 73.93506077410439) bank95632 +95633 POINT(40.17514054998881 73.68923151820584) bank95633 +95634 POINT(41.634993042195504 73.79312020614351) bank95634 +95635 POINT(39.818360418635585 74.5797378518534) bank95635 +95636 POINT(41.19055713280682 74.48458022354082) bank95636 +95637 POINT(40.60656765749463 73.91565715046768) bank95637 +95638 POINT(41.415974183461415 74.08739867116918) bank95638 +95639 POINT(41.5861467934018 73.70902181991971) bank95639 +95640 POINT(40.48254955960668 74.46821485777724) bank95640 +95641 POINT(40.373129373096 74.22755932497354) bank95641 +95642 POINT(41.14646083608784 74.94205131662049) bank95642 +95643 POINT(41.011562637988746 73.10542840167864) bank95643 +95644 POINT(40.152720114274324 73.51738862352673) bank95644 +95645 POINT(40.80112353203966 74.7182079803019) bank95645 +95646 POINT(40.2690624668968 74.5575806150228) bank95646 +95647 POINT(39.76522268662261 73.67158012891133) bank95647 +95648 POINT(41.70249931570592 73.00850286144247) bank95648 +95649 POINT(40.92041971830106 73.124500478129) bank95649 +95650 POINT(41.268283568832196 73.79658358298518) bank95650 +95651 POINT(40.730459403674516 73.86216105859305) bank95651 +95652 POINT(39.74627364420774 74.5595191535136) bank95652 +95653 POINT(41.110772946677294 74.14574310884741) bank95653 +95654 POINT(41.589166493249216 74.36882626071403) bank95654 +95655 POINT(39.71808659197109 74.76593694153627) bank95655 +95656 POINT(40.24916774247691 73.24905891012203) bank95656 +95657 POINT(40.66681456585081 74.72827570792195) bank95657 +95658 POINT(40.291369407238136 73.71048561811965) bank95658 +95659 POINT(40.89806570330557 74.39381405484126) bank95659 +95660 POINT(40.58462819122093 73.57510247874083) bank95660 +95661 POINT(40.31022268243723 74.02509999058883) bank95661 +95662 POINT(39.77467435743062 73.71492958893651) bank95662 +95663 POINT(40.17279911661563 73.84080159361822) bank95663 +95664 POINT(40.66505481927736 74.72134459266873) bank95664 +95665 POINT(41.029659219186335 74.95025260656136) bank95665 +95666 POINT(40.136641681586234 73.87769401124787) bank95666 +95667 POINT(40.74066190871092 73.00665228837441) bank95667 +95668 POINT(41.3037448641648 74.92203335855548) bank95668 +95669 POINT(41.671275357788254 73.9757137611461) bank95669 +95670 POINT(41.45632536510832 73.90160591105057) bank95670 +95671 POINT(41.05389439750763 74.54497112285694) bank95671 +95672 POINT(40.418895579920424 74.90617559052232) bank95672 +95673 POINT(40.16609447902208 74.77279032431831) bank95673 +95674 POINT(40.46842225778605 74.35366280331512) bank95674 +95675 POINT(40.518442030445236 74.9391797837513) bank95675 +95676 POINT(40.42521612145047 74.16340352449737) bank95676 +95677 POINT(41.41806088115585 74.78431117309937) bank95677 +95678 POINT(39.78917852696998 73.56997522574227) bank95678 +95679 POINT(40.363259366534905 74.69870786265054) bank95679 +95680 POINT(40.459553909221114 74.19771761527514) bank95680 +95681 POINT(40.356659919746086 73.6073176278842) bank95681 +95682 POINT(41.14309425046856 73.868179086759) bank95682 +95683 POINT(41.55843894218744 74.1212226249011) bank95683 +95684 POINT(40.55411931604629 74.03178361159227) bank95684 +95685 POINT(39.8965557986373 73.34088405136326) bank95685 +95686 POINT(41.42360182000729 73.70846664591217) bank95686 +95687 POINT(40.968132673668855 74.78834725329469) bank95687 +95688 POINT(41.144584150339604 73.25922407203088) bank95688 +95689 POINT(40.20469242761976 73.6467855517562) bank95689 +95690 POINT(41.595047309761796 73.28396410704573) bank95690 +95691 POINT(39.75932549764186 74.90040365197461) bank95691 +95692 POINT(40.34385173869789 74.16297924503999) bank95692 +95693 POINT(41.015048783741186 74.1719954534363) bank95693 +95694 POINT(40.69982315312572 73.15176598187176) bank95694 +95695 POINT(40.42447331091591 73.34176938752974) bank95695 +95696 POINT(40.570976459030895 74.30316405576446) bank95696 +95697 POINT(41.26820346613587 74.67584732191922) bank95697 +95698 POINT(40.33648353911033 73.15561034836445) bank95698 +95699 POINT(40.67949041757735 74.37140637630574) bank95699 +95700 POINT(40.09698435867963 73.16212080726778) bank95700 +95701 POINT(41.46001036959763 73.15723729901119) bank95701 +95702 POINT(40.40654885774292 74.25706809002232) bank95702 +95703 POINT(40.80576482025888 74.08887701399432) bank95703 +95704 POINT(40.393655815184644 73.05264532123779) bank95704 +95705 POINT(41.59929138344185 73.67361456923322) bank95705 +95706 POINT(41.42082712176287 74.26849874503252) bank95706 +95707 POINT(40.073023380066246 73.2661993310719) bank95707 +95708 POINT(40.74735908449969 74.2940615938772) bank95708 +95709 POINT(40.079182363481145 74.03604701950879) bank95709 +95710 POINT(41.28013534114235 74.08351834265966) bank95710 +95711 POINT(40.276890782607474 74.19962060350447) bank95711 +95712 POINT(41.23868836025125 73.76922823534379) bank95712 +95713 POINT(40.03535387396929 73.03486676472484) bank95713 +95714 POINT(40.598852274341006 73.80708680228567) bank95714 +95715 POINT(40.715403325474405 73.84269195517737) bank95715 +95716 POINT(40.69144561619802 74.64132024657572) bank95716 +95717 POINT(40.66680869356403 74.0373641160331) bank95717 +95718 POINT(40.52166347147964 73.93429198185473) bank95718 +95719 POINT(40.60356766175053 73.63849919445353) bank95719 +95720 POINT(40.992457263031746 74.33176520826034) bank95720 +95721 POINT(39.88842037201572 73.87323591066769) bank95721 +95722 POINT(41.62999042445893 74.83071343517508) bank95722 +95723 POINT(40.83428563385487 73.04486516467611) bank95723 +95724 POINT(40.75797500046898 74.35591033415567) bank95724 +95725 POINT(40.45874389481554 74.46240347348977) bank95725 +95726 POINT(41.65864007295234 74.60096872849519) bank95726 +95727 POINT(41.36616461548034 74.02003024801114) bank95727 +95728 POINT(41.197943998480035 74.51726139309837) bank95728 +95729 POINT(40.357921020369304 73.32377629412025) bank95729 +95730 POINT(39.95418271351072 73.04674278124263) bank95730 +95731 POINT(40.94032077262683 73.65541492136052) bank95731 +95732 POINT(41.471571504583835 74.78299016563906) bank95732 +95733 POINT(39.7243759176756 73.93461794706828) bank95733 +95734 POINT(41.042659500057155 74.95854860938378) bank95734 +95735 POINT(39.950647894608565 73.9532412307092) bank95735 +95736 POINT(40.912049800201665 73.47817035724267) bank95736 +95737 POINT(40.3188309746189 74.37300447867355) bank95737 +95738 POINT(39.957946476252644 73.81318026252644) bank95738 +95739 POINT(40.361920812276644 73.65481884087025) bank95739 +95740 POINT(41.26235749810505 73.21029832039483) bank95740 +95741 POINT(40.964480877375436 73.08781118734613) bank95741 +95742 POINT(41.38599741219555 73.84268264274452) bank95742 +95743 POINT(40.78223780228868 73.69200738521288) bank95743 +95744 POINT(39.8742974100622 73.15264852985393) bank95744 +95745 POINT(40.236902538824246 73.02917183698834) bank95745 +95746 POINT(39.983132295022514 73.4396931062073) bank95746 +95747 POINT(41.10373947898458 74.79060229781678) bank95747 +95748 POINT(39.912418231110394 74.54176215587529) bank95748 +95749 POINT(41.367053485483844 74.08567128685365) bank95749 +95750 POINT(41.200008652014446 74.93138879634404) bank95750 +95751 POINT(41.42859727157254 73.40361603167943) bank95751 +95752 POINT(40.5095667962006 74.67719613298826) bank95752 +95753 POINT(41.14012212595262 73.99178244402559) bank95753 +95754 POINT(41.508904042715734 73.76703237991883) bank95754 +95755 POINT(39.88456604141387 74.36237405251497) bank95755 +95756 POINT(40.93286487670873 74.98115927028653) bank95756 +95757 POINT(41.25164137779247 74.8676417777845) bank95757 +95758 POINT(39.936228051408875 74.59219413959283) bank95758 +95759 POINT(41.26341614403579 73.64091881053635) bank95759 +95760 POINT(40.92911560868189 74.37445438377586) bank95760 +95761 POINT(40.97448087752427 73.29056381997485) bank95761 +95762 POINT(40.7364997304291 74.32223307043265) bank95762 +95763 POINT(40.557040193496356 74.64319261206339) bank95763 +95764 POINT(40.5181241854506 74.00972700269733) bank95764 +95765 POINT(41.068826495162334 74.35297412833539) bank95765 +95766 POINT(40.63583478627796 73.38732670317476) bank95766 +95767 POINT(39.78552917733159 73.94871002449185) bank95767 +95768 POINT(41.20330036640269 73.08081455391597) bank95768 +95769 POINT(41.52777168420892 74.90637532760934) bank95769 +95770 POINT(40.43719417215316 74.56951439876212) bank95770 +95771 POINT(40.06998059163067 73.23455847350401) bank95771 +95772 POINT(41.5473508405623 73.20376669936202) bank95772 +95773 POINT(41.04253622876597 73.13111705520399) bank95773 +95774 POINT(40.77794575025981 74.00059948990747) bank95774 +95775 POINT(41.684215198321 73.42550585520551) bank95775 +95776 POINT(40.96864445069012 74.87700990203865) bank95776 +95777 POINT(39.957209097148585 74.56640186854827) bank95777 +95778 POINT(40.172854440504864 73.4946858475945) bank95778 +95779 POINT(40.489756210151775 74.51225100438324) bank95779 +95780 POINT(40.55484476164897 74.82872912502073) bank95780 +95781 POINT(40.52000746060004 74.81335209164835) bank95781 +95782 POINT(40.06108987993577 74.00775842686748) bank95782 +95783 POINT(41.61196722483883 73.99022576730488) bank95783 +95784 POINT(41.356019563320494 74.84720440426143) bank95784 +95785 POINT(40.71120146475813 74.7495344627052) bank95785 +95786 POINT(41.06741075827801 74.15810775252062) bank95786 +95787 POINT(40.63383532575597 74.56630558974173) bank95787 +95788 POINT(41.530517779459586 73.42494617229882) bank95788 +95789 POINT(40.502371594620236 73.21646452455163) bank95789 +95790 POINT(40.98516026152004 73.90086193103105) bank95790 +95791 POINT(40.866495825661445 74.67017405052867) bank95791 +95792 POINT(39.8440534745477 74.53635225443104) bank95792 +95793 POINT(41.54493100756391 74.37011307617725) bank95793 +95794 POINT(41.064685382146045 74.92024626343157) bank95794 +95795 POINT(41.213400503512474 74.70497285235824) bank95795 +95796 POINT(40.83825420405744 73.1970416032137) bank95796 +95797 POINT(41.554015127716596 73.35852141067365) bank95797 +95798 POINT(41.16499657740025 73.97489143409813) bank95798 +95799 POINT(39.7149998071706 74.12660759212379) bank95799 +95800 POINT(39.91615389752846 73.28859380989049) bank95800 +95801 POINT(40.61702322347648 74.01453328936157) bank95801 +95802 POINT(40.25378472938936 74.79315354853283) bank95802 +95803 POINT(41.15606777172845 73.9864830610002) bank95803 +95804 POINT(40.436632338844504 73.38575068509321) bank95804 +95805 POINT(40.924355603698395 73.4594050862097) bank95805 +95806 POINT(39.715265870606 73.31334678696109) bank95806 +95807 POINT(40.20999406907108 74.60224677413619) bank95807 +95808 POINT(40.87018768269168 74.20793188598608) bank95808 +95809 POINT(41.565258518919386 73.05536451528424) bank95809 +95810 POINT(40.44334114845385 74.70207620009072) bank95810 +95811 POINT(40.57010179241958 74.55959819224027) bank95811 +95812 POINT(39.9330900812779 73.23108199106021) bank95812 +95813 POINT(39.9442628465074 74.85179002160191) bank95813 +95814 POINT(40.83004319832768 73.57785014136508) bank95814 +95815 POINT(41.59214391283477 73.46554444533871) bank95815 +95816 POINT(40.24384819339039 74.50353903299083) bank95816 +95817 POINT(41.35662117596947 73.59566800342716) bank95817 +95818 POINT(40.94122406647914 73.15773721310856) bank95818 +95819 POINT(40.658275315618674 73.5694088307956) bank95819 +95820 POINT(40.75209786898627 74.97871057096786) bank95820 +95821 POINT(39.910862310762575 74.44461871255889) bank95821 +95822 POINT(40.775469895236405 73.27840346248448) bank95822 +95823 POINT(41.30486666268449 73.89176373672393) bank95823 +95824 POINT(41.03381300863777 73.5400195030276) bank95824 +95825 POINT(41.05017240029087 73.17269740940863) bank95825 +95826 POINT(41.19537033578963 74.32169256863098) bank95826 +95827 POINT(39.92248479696636 73.03829902228689) bank95827 +95828 POINT(39.77135789158475 73.68375587840521) bank95828 +95829 POINT(41.069168805248594 73.65328385528886) bank95829 +95830 POINT(41.368071913418234 73.64728439037864) bank95830 +95831 POINT(40.86078557635228 73.36402639073994) bank95831 +95832 POINT(40.24365247105874 74.46952765318056) bank95832 +95833 POINT(39.98575932816419 73.12356830440781) bank95833 +95834 POINT(40.11291316299077 73.87509029452497) bank95834 +95835 POINT(41.3078128211907 73.94006391172027) bank95835 +95836 POINT(40.06497709889783 74.67155654173473) bank95836 +95837 POINT(40.07188970044053 74.0693714135619) bank95837 +95838 POINT(39.920128019441336 73.49587134706297) bank95838 +95839 POINT(39.858025760699164 74.48085558405343) bank95839 +95840 POINT(41.051676317281 73.69372728176296) bank95840 +95841 POINT(41.051018472936974 74.66118890110883) bank95841 +95842 POINT(39.98583549126663 74.0287864091716) bank95842 +95843 POINT(39.84174508098571 73.10542539381977) bank95843 +95844 POINT(40.926319388861536 73.71798075855625) bank95844 +95845 POINT(39.986626841727585 73.31082360706193) bank95845 +95846 POINT(41.09301687752755 74.37276790424153) bank95846 +95847 POINT(41.28470425148414 74.75672057457665) bank95847 +95848 POINT(41.65262146269985 73.14548902982531) bank95848 +95849 POINT(40.17697876501949 74.47609090060547) bank95849 +95850 POINT(40.76881348168218 73.69758319492855) bank95850 +95851 POINT(41.16985192569366 73.21613089311083) bank95851 +95852 POINT(39.82580895046903 73.04619181946013) bank95852 +95853 POINT(41.57721790706831 74.2727492721368) bank95853 +95854 POINT(41.06518008127166 73.23629964436181) bank95854 +95855 POINT(40.537542178012224 74.21857773125397) bank95855 +95856 POINT(40.36548686897183 73.4018239838947) bank95856 +95857 POINT(41.38821938480531 74.48987483117371) bank95857 +95858 POINT(40.85082111894445 73.21947481768667) bank95858 +95859 POINT(40.838580782496926 73.79561682569422) bank95859 +95860 POINT(40.08359808546257 74.09225456545536) bank95860 +95861 POINT(40.97800761323829 73.03660265104784) bank95861 +95862 POINT(40.432334330762494 74.46420764243638) bank95862 +95863 POINT(40.331769275001996 73.2304971460549) bank95863 +95864 POINT(40.82506672196757 73.48760075974374) bank95864 +95865 POINT(41.23559248022007 74.456447032063) bank95865 +95866 POINT(40.31893023954051 74.80455192831464) bank95866 +95867 POINT(41.513561518758465 73.08595681922434) bank95867 +95868 POINT(39.73485309740567 73.0958257614176) bank95868 +95869 POINT(41.17583432230461 74.19831594994523) bank95869 +95870 POINT(40.340423572098636 73.19909638002564) bank95870 +95871 POINT(40.76018686924023 74.78911412891841) bank95871 +95872 POINT(40.06336050672309 73.94535195486846) bank95872 +95873 POINT(40.187993657613625 74.9151243349693) bank95873 +95874 POINT(40.63901586049904 74.54238090824064) bank95874 +95875 POINT(40.70252216778601 74.7126780982939) bank95875 +95876 POINT(40.08904247731912 73.74097509490647) bank95876 +95877 POINT(40.728807862188944 74.5524043589894) bank95877 +95878 POINT(40.10689364063548 73.47353521297495) bank95878 +95879 POINT(40.20055272433032 73.8885195283998) bank95879 +95880 POINT(41.4266223856318 74.07837894181023) bank95880 +95881 POINT(41.4327836976456 74.1079912502868) bank95881 +95882 POINT(41.58582225923519 74.1990020699529) bank95882 +95883 POINT(40.894800015159376 74.80061317403367) bank95883 +95884 POINT(41.54203975387794 73.5539066069997) bank95884 +95885 POINT(41.06054550378511 74.83638092873625) bank95885 +95886 POINT(41.040057698714094 73.05964890841493) bank95886 +95887 POINT(39.874830942632144 73.50028897467543) bank95887 +95888 POINT(40.947288539116826 74.14521960762833) bank95888 +95889 POINT(41.68360789858761 74.08448433417662) bank95889 +95890 POINT(39.99508324291128 74.92322048575875) bank95890 +95891 POINT(41.221718639502654 74.57367248293231) bank95891 +95892 POINT(39.73044602014581 74.17318047427055) bank95892 +95893 POINT(39.92448187279736 74.84227717718834) bank95893 +95894 POINT(41.57191915455932 73.55967887098518) bank95894 +95895 POINT(40.58610756785838 74.77290659613604) bank95895 +95896 POINT(41.27180314725547 73.24321947965875) bank95896 +95897 POINT(40.07960346675107 74.13155231211768) bank95897 +95898 POINT(41.61901539273865 73.27137344593892) bank95898 +95899 POINT(41.585502331762136 73.1085262426196) bank95899 +95900 POINT(40.29932792492514 73.4427762928304) bank95900 +95901 POINT(40.47539954836492 74.21031862596965) bank95901 +95902 POINT(40.45472573891951 74.63795378494066) bank95902 +95903 POINT(40.64503542834417 74.06895417349548) bank95903 +95904 POINT(41.10324687784012 73.78845976164857) bank95904 +95905 POINT(41.1440060087891 73.89050489097495) bank95905 +95906 POINT(41.16093990448155 73.70179176003667) bank95906 +95907 POINT(40.21728721505043 73.60505319878585) bank95907 +95908 POINT(41.43876903875737 74.33925980907759) bank95908 +95909 POINT(39.79939493555739 73.05172580469743) bank95909 +95910 POINT(41.28590533479835 74.5392155787819) bank95910 +95911 POINT(40.11104092918434 73.9734189137248) bank95911 +95912 POINT(41.33225759855485 73.57017070310869) bank95912 +95913 POINT(41.58816568725622 74.8356900034168) bank95913 +95914 POINT(39.94474401048516 73.09094355198847) bank95914 +95915 POINT(40.386632676628814 74.52925561835288) bank95915 +95916 POINT(40.910107157322436 73.08988948153832) bank95916 +95917 POINT(40.01628551427976 73.36379440739714) bank95917 +95918 POINT(41.06965629329436 73.30938165826406) bank95918 +95919 POINT(39.73278141342668 74.82353387394053) bank95919 +95920 POINT(40.78926273083 74.83519997310016) bank95920 +95921 POINT(39.82003158186554 74.06477625806359) bank95921 +95922 POINT(40.895608691392766 74.97477084963624) bank95922 +95923 POINT(39.726232550254146 74.48617117330711) bank95923 +95924 POINT(41.1324189011405 74.2873162313233) bank95924 +95925 POINT(40.42192509866722 74.25781354591186) bank95925 +95926 POINT(39.802865510047475 73.70551580362637) bank95926 +95927 POINT(41.5870988535866 74.32986777572012) bank95927 +95928 POINT(41.61528497609138 74.71947719008732) bank95928 +95929 POINT(40.372840205355224 74.04358902612175) bank95929 +95930 POINT(41.20907551398827 73.54209765782032) bank95930 +95931 POINT(41.609280510239664 74.47539871806056) bank95931 +95932 POINT(41.3026091598709 73.6495061623112) bank95932 +95933 POINT(40.17349785510649 73.05563028527601) bank95933 +95934 POINT(40.0485602131358 73.64864541868114) bank95934 +95935 POINT(41.32856784719955 73.33434068550321) bank95935 +95936 POINT(41.57380750967031 74.03368363198827) bank95936 +95937 POINT(39.85370657323493 73.19448310658541) bank95937 +95938 POINT(40.849729331510844 73.51455504278402) bank95938 +95939 POINT(41.47014490715592 74.86822704763176) bank95939 +95940 POINT(41.3869732698779 74.0739654604009) bank95940 +95941 POINT(41.478991684193495 73.9394702902507) bank95941 +95942 POINT(40.59791184578826 73.08843384589767) bank95942 +95943 POINT(41.356876558412495 73.60656039625455) bank95943 +95944 POINT(39.90783086537615 73.19836653857033) bank95944 +95945 POINT(41.67819171280037 73.79824243058215) bank95945 +95946 POINT(40.233046540001126 74.16025751847923) bank95946 +95947 POINT(40.87971812423991 74.95770920296282) bank95947 +95948 POINT(40.42023043403522 74.89879508394509) bank95948 +95949 POINT(40.607103136544666 73.89393307179107) bank95949 +95950 POINT(39.80601150183359 74.76365927322507) bank95950 +95951 POINT(41.59880775170352 73.6363159401984) bank95951 +95952 POINT(41.40631275833252 73.15117072545382) bank95952 +95953 POINT(41.166392796135945 74.72762176825826) bank95953 +95954 POINT(41.555280808011226 74.50833760189524) bank95954 +95955 POINT(40.10029042508948 74.92498972610592) bank95955 +95956 POINT(41.098945108547234 73.9560830334792) bank95956 +95957 POINT(40.931152764487585 73.48913268149236) bank95957 +95958 POINT(40.229210401146055 73.64887486442359) bank95958 +95959 POINT(40.02434835047716 73.97230220916067) bank95959 +95960 POINT(40.185066751327504 74.84766523983217) bank95960 +95961 POINT(40.97523900601897 73.64644943185193) bank95961 +95962 POINT(41.61725974399878 73.58134871767801) bank95962 +95963 POINT(40.57192918852457 73.74314967822811) bank95963 +95964 POINT(40.33240316249479 73.72058820016196) bank95964 +95965 POINT(40.938780281654694 74.64639777150055) bank95965 +95966 POINT(41.607003083945926 74.92275205874714) bank95966 +95967 POINT(41.561759984291434 73.76290661559787) bank95967 +95968 POINT(39.85747164411492 74.20822734195545) bank95968 +95969 POINT(41.45663087897756 73.4922542256492) bank95969 +95970 POINT(40.46262944060176 74.81122231510102) bank95970 +95971 POINT(41.54476030221423 73.18259027890885) bank95971 +95972 POINT(40.596959251767935 73.42385219154501) bank95972 +95973 POINT(40.53429550883429 74.6897603625174) bank95973 +95974 POINT(40.15153795021399 74.96565733452233) bank95974 +95975 POINT(40.350217667854004 73.71740003774175) bank95975 +95976 POINT(39.984230572505325 74.73072044855682) bank95976 +95977 POINT(40.12092005453912 74.1265604129995) bank95977 +95978 POINT(40.86360640710938 73.29517126752263) bank95978 +95979 POINT(41.01616623919541 73.63411902192092) bank95979 +95980 POINT(40.727579596645484 73.71355433787332) bank95980 +95981 POINT(41.70036737456557 73.16572959561647) bank95981 +95982 POINT(39.9075919273336 73.3967914445882) bank95982 +95983 POINT(41.280674034066024 73.8557146374497) bank95983 +95984 POINT(39.996516356063125 74.40081311494404) bank95984 +95985 POINT(40.75502323244408 73.3736443209764) bank95985 +95986 POINT(39.75373589363725 73.59719966130704) bank95986 +95987 POINT(39.87413269445667 74.65100680842907) bank95987 +95988 POINT(39.92925270230254 73.57381113397099) bank95988 +95989 POINT(39.93570787883343 74.2294154927845) bank95989 +95990 POINT(41.20481221613392 73.72046824313819) bank95990 +95991 POINT(41.478133580944174 74.81244808643767) bank95991 +95992 POINT(41.13006869881718 74.09000340927103) bank95992 +95993 POINT(40.92303370401823 74.71934714699032) bank95993 +95994 POINT(40.050383405473596 74.02951679120102) bank95994 +95995 POINT(39.96338858161794 73.11171542674862) bank95995 +95996 POINT(40.50895102266693 73.16847186749862) bank95996 +95997 POINT(40.86357876581008 74.78116764298328) bank95997 +95998 POINT(40.16463065881718 74.30288621531943) bank95998 +95999 POINT(40.03889975862518 73.94932197802497) bank95999 +96000 POINT(41.65889693265798 74.41856070516636) bank96000 +96001 POINT(40.58730764915246 73.9290587598814) bank96001 +96002 POINT(41.70595641876805 74.55473954532117) bank96002 +96003 POINT(39.76180495699019 73.91273076059417) bank96003 +96004 POINT(41.020661848093546 74.38957204488109) bank96004 +96005 POINT(41.14909160012333 74.49642453215705) bank96005 +96006 POINT(40.5525786469302 73.4481759052162) bank96006 +96007 POINT(40.6232553665951 73.59329007958226) bank96007 +96008 POINT(40.91936724409325 73.69942217983645) bank96008 +96009 POINT(41.31119159883381 73.9181971141545) bank96009 +96010 POINT(40.756070012081665 74.60803980696264) bank96010 +96011 POINT(40.52176370087003 73.40739897586117) bank96011 +96012 POINT(41.023310805764694 74.07899422776174) bank96012 +96013 POINT(41.00888209270037 73.49719275832233) bank96013 +96014 POINT(40.02637451325403 74.23389668199155) bank96014 +96015 POINT(41.47428036164566 74.25798586343781) bank96015 +96016 POINT(40.804125476662264 73.23917075526366) bank96016 +96017 POINT(40.87850837434769 74.08031203179101) bank96017 +96018 POINT(40.46552663890308 73.25269671198443) bank96018 +96019 POINT(39.848828690150334 73.43576692120726) bank96019 +96020 POINT(40.731762594576956 74.50366001588003) bank96020 +96021 POINT(40.68288262592432 74.21150540798072) bank96021 +96022 POINT(40.484975865664616 73.80212938361973) bank96022 +96023 POINT(39.95441154146031 74.13915661641174) bank96023 +96024 POINT(41.55005402780248 73.09867533300852) bank96024 +96025 POINT(40.5650958409044 73.62368606349868) bank96025 +96026 POINT(41.03229180999541 74.5199964750577) bank96026 +96027 POINT(39.73602704462045 73.29280467474207) bank96027 +96028 POINT(40.4088220069983 73.2124542021143) bank96028 +96029 POINT(41.038223497088424 73.17625168876882) bank96029 +96030 POINT(39.86003643370225 74.8832984719717) bank96030 +96031 POINT(41.620858165788 74.09927833487453) bank96031 +96032 POINT(39.98445371973889 73.0072184797402) bank96032 +96033 POINT(39.925367888703505 73.99808814252665) bank96033 +96034 POINT(39.716982381933974 74.69401493706367) bank96034 +96035 POINT(40.541815655132595 73.37593228955407) bank96035 +96036 POINT(40.43414263616111 74.0687617465472) bank96036 +96037 POINT(40.869818129855425 74.34361946466615) bank96037 +96038 POINT(39.90699826299558 73.34299271457806) bank96038 +96039 POINT(40.63744288744354 73.90087033347365) bank96039 +96040 POINT(41.61198095768111 74.628316719428) bank96040 +96041 POINT(41.54798390099333 74.18706915918823) bank96041 +96042 POINT(40.161037491626956 73.60575411318776) bank96042 +96043 POINT(41.27796952986147 74.33102722530444) bank96043 +96044 POINT(40.76862422794722 73.00765442959306) bank96044 +96045 POINT(41.57427165583407 74.04975420377778) bank96045 +96046 POINT(40.270452773954574 74.1034708340258) bank96046 +96047 POINT(40.34661483907898 73.52749770255232) bank96047 +96048 POINT(41.51731541793042 74.29517447796727) bank96048 +96049 POINT(40.12407249174713 73.1666340094893) bank96049 +96050 POINT(41.2933598475525 74.42924871297345) bank96050 +96051 POINT(40.11495132248143 73.73024845245945) bank96051 +96052 POINT(40.16436903489816 74.89021348869285) bank96052 +96053 POINT(40.270514366039194 73.6771926454212) bank96053 +96054 POINT(41.58427102130254 73.3712852666705) bank96054 +96055 POINT(39.81070511088335 74.29826905668946) bank96055 +96056 POINT(40.876781739890035 74.3885585951154) bank96056 +96057 POINT(40.35069122950438 73.31863840555816) bank96057 +96058 POINT(40.77642482126923 73.7986490418433) bank96058 +96059 POINT(40.422167955665415 73.04966279774862) bank96059 +96060 POINT(40.27407695856426 74.86743478595396) bank96060 +96061 POINT(41.38196502658124 74.57693817078322) bank96061 +96062 POINT(40.71933398235755 73.10575701351709) bank96062 +96063 POINT(40.26426296921853 73.0285934857434) bank96063 +96064 POINT(40.153334914566834 73.8392625121432) bank96064 +96065 POINT(40.55690576820889 73.66529772353537) bank96065 +96066 POINT(41.03568965204354 73.07067360256111) bank96066 +96067 POINT(41.451110372740374 73.7662345737047) bank96067 +96068 POINT(41.41521657251213 73.39624010295893) bank96068 +96069 POINT(40.63040611445135 73.08589547379273) bank96069 +96070 POINT(41.29758299678246 74.54161455515971) bank96070 +96071 POINT(40.58543773045523 74.10010908965431) bank96071 +96072 POINT(41.228187526651 73.88772584215) bank96072 +96073 POINT(41.00302461756254 74.49025039946042) bank96073 +96074 POINT(41.701110669483946 73.40866229048191) bank96074 +96075 POINT(41.596359556861934 74.68536652151293) bank96075 +96076 POINT(40.01127312522581 74.57358163056983) bank96076 +96077 POINT(40.966643992401075 74.22733913336299) bank96077 +96078 POINT(41.120019300003385 74.46837044517292) bank96078 +96079 POINT(40.01296367575066 73.23316949980047) bank96079 +96080 POINT(39.77930231290718 74.33605766482086) bank96080 +96081 POINT(41.089688311139106 73.2609481268149) bank96081 +96082 POINT(41.1693007594231 73.08719798010442) bank96082 +96083 POINT(40.09570595834266 73.97017396962593) bank96083 +96084 POINT(41.20609508784087 73.19646456512694) bank96084 +96085 POINT(41.16738281579315 73.60618357940434) bank96085 +96086 POINT(41.04663436689531 74.739199057298) bank96086 +96087 POINT(40.774190308117596 73.46685305913421) bank96087 +96088 POINT(41.00928694208534 73.07928275192144) bank96088 +96089 POINT(40.510329420595795 73.2050270332186) bank96089 +96090 POINT(41.175298816749624 74.24694023100467) bank96090 +96091 POINT(40.73232340737199 74.85776088209658) bank96091 +96092 POINT(41.56970182920104 74.83581345737805) bank96092 +96093 POINT(40.69824770722112 74.1089634396483) bank96093 +96094 POINT(40.430459414325476 73.0761088241766) bank96094 +96095 POINT(41.22787301655432 73.30352952554826) bank96095 +96096 POINT(40.73027123613038 74.41790831250682) bank96096 +96097 POINT(40.11123693799105 73.37403839121782) bank96097 +96098 POINT(40.0975969373895 73.58443638795843) bank96098 +96099 POINT(40.642196117530155 73.55888240549369) bank96099 +96100 POINT(40.14366736004755 73.69864959378411) bank96100 +96101 POINT(40.886734472262766 74.29097933654376) bank96101 +96102 POINT(40.702748718438116 74.03969469331422) bank96102 +96103 POINT(41.093928869085744 74.64591443029394) bank96103 +96104 POINT(40.72183995791998 74.63885606593489) bank96104 +96105 POINT(40.92749764500976 74.32349482065983) bank96105 +96106 POINT(39.72575457128258 74.07563584013144) bank96106 +96107 POINT(40.5190728231133 73.24598464751253) bank96107 +96108 POINT(40.44999684888132 73.88988526165713) bank96108 +96109 POINT(40.110378103368305 73.25422891443422) bank96109 +96110 POINT(40.334237153804274 73.84759251333715) bank96110 +96111 POINT(40.073244970601905 73.94613693121497) bank96111 +96112 POINT(39.86761327820085 74.79985354807835) bank96112 +96113 POINT(41.33659529798281 73.23451185682484) bank96113 +96114 POINT(40.77930686374199 73.22146713594034) bank96114 +96115 POINT(40.064365940754136 73.04651647060409) bank96115 +96116 POINT(41.37327531606017 74.7478332857268) bank96116 +96117 POINT(41.381416068140126 73.85053675942142) bank96117 +96118 POINT(41.27900432089538 74.12567456459962) bank96118 +96119 POINT(41.014520590378034 73.45219080023138) bank96119 +96120 POINT(41.00910907098332 74.23075969165996) bank96120 +96121 POINT(41.62947491059876 73.0276876127394) bank96121 +96122 POINT(41.60912368984964 74.47590544484345) bank96122 +96123 POINT(39.729408433697 73.76032520655085) bank96123 +96124 POINT(41.602348852683384 74.04336439887776) bank96124 +96125 POINT(39.89541492780376 73.52212857280834) bank96125 +96126 POINT(41.5968260730316 74.97971795231375) bank96126 +96127 POINT(41.659244158596216 74.79948902732909) bank96127 +96128 POINT(40.41064800848777 73.6265735865178) bank96128 +96129 POINT(40.14266132210622 74.65741850836123) bank96129 +96130 POINT(39.957444389948755 73.93805609737572) bank96130 +96131 POINT(40.04613927474151 73.79753397027507) bank96131 +96132 POINT(40.56351482423828 73.35928516856913) bank96132 +96133 POINT(41.037289248045745 74.04338884767455) bank96133 +96134 POINT(41.01152984335437 74.0192363616549) bank96134 +96135 POINT(39.781052861226534 73.80106021939116) bank96135 +96136 POINT(40.707577745252976 74.09775883718346) bank96136 +96137 POINT(40.00795557314084 73.40554078058783) bank96137 +96138 POINT(41.02957531497745 73.1646503495815) bank96138 +96139 POINT(40.94590495102898 74.2211941345274) bank96139 +96140 POINT(40.41946908077044 74.11903594642506) bank96140 +96141 POINT(39.848117022692826 73.29140406275269) bank96141 +96142 POINT(41.4998420301854 73.89830489494894) bank96142 +96143 POINT(41.11996062504085 73.1746053534343) bank96143 +96144 POINT(40.796975076508154 73.69708003222387) bank96144 +96145 POINT(40.58911060658862 74.17656600455375) bank96145 +96146 POINT(40.271247994807126 73.68309062867547) bank96146 +96147 POINT(41.37835842077298 74.49744440667966) bank96147 +96148 POINT(40.51594954641256 73.68041666823825) bank96148 +96149 POINT(41.15277780361558 73.53465964852704) bank96149 +96150 POINT(40.7126138622605 74.52435802880048) bank96150 +96151 POINT(39.94570210524273 74.47306574380671) bank96151 +96152 POINT(40.87287218671389 73.82195717033764) bank96152 +96153 POINT(40.43832746110756 73.46678773977864) bank96153 +96154 POINT(40.13351846263652 74.22991678354653) bank96154 +96155 POINT(41.1008004872188 73.10232036413278) bank96155 +96156 POINT(41.24277699454343 73.0321286015851) bank96156 +96157 POINT(40.972229365319485 73.36844888868114) bank96157 +96158 POINT(40.78280120915407 73.18566703888997) bank96158 +96159 POINT(41.41164010282709 74.0746132581383) bank96159 +96160 POINT(41.0880145040572 74.45540097242714) bank96160 +96161 POINT(40.81954120796297 73.17687191258436) bank96161 +96162 POINT(40.78775923015285 73.97579392822671) bank96162 +96163 POINT(41.29841572918853 73.0674212044888) bank96163 +96164 POINT(41.06337288008032 74.67992308594793) bank96164 +96165 POINT(40.478929536291325 73.09185779195919) bank96165 +96166 POINT(40.98757910624523 73.45462124599634) bank96166 +96167 POINT(40.7434669819172 74.30384464923266) bank96167 +96168 POINT(40.931391768214816 74.3725400088672) bank96168 +96169 POINT(40.02835375364525 73.51648556515812) bank96169 +96170 POINT(41.18495226628256 73.76515125518772) bank96170 +96171 POINT(40.7744358700263 74.32533133767647) bank96171 +96172 POINT(40.56971379838213 74.30107961583114) bank96172 +96173 POINT(40.908270266963605 73.03311765173338) bank96173 +96174 POINT(41.640621822946564 74.63826972426054) bank96174 +96175 POINT(41.257135865799356 73.31445734859737) bank96175 +96176 POINT(41.256995124923534 74.23286163216662) bank96176 +96177 POINT(41.42797687195433 74.89068060467976) bank96177 +96178 POINT(40.61266324517632 74.8140838395016) bank96178 +96179 POINT(41.49059356453347 74.70380856879692) bank96179 +96180 POINT(41.65127744399463 74.9243047504729) bank96180 +96181 POINT(40.133887685969704 73.58321278323812) bank96181 +96182 POINT(39.7733004936908 73.20269064306588) bank96182 +96183 POINT(40.582315169982415 74.2741393450828) bank96183 +96184 POINT(41.43952896119907 73.24736364566806) bank96184 +96185 POINT(41.570201382445255 74.40544991984933) bank96185 +96186 POINT(41.5942119922574 74.77705895429122) bank96186 +96187 POINT(39.82670776782295 74.13137653247752) bank96187 +96188 POINT(41.136739599257815 74.92192600122333) bank96188 +96189 POINT(40.74570911666394 74.35851800875102) bank96189 +96190 POINT(41.6067541542084 74.41715849276542) bank96190 +96191 POINT(41.49566027575835 74.59131173490967) bank96191 +96192 POINT(39.78546209434639 74.14085358428002) bank96192 +96193 POINT(41.4011417124063 73.51527504905788) bank96193 +96194 POINT(41.037830423558454 74.48668262366289) bank96194 +96195 POINT(41.12480251446164 74.89862268611003) bank96195 +96196 POINT(41.123871403545614 73.48061414380365) bank96196 +96197 POINT(41.301232500188014 73.37886639118656) bank96197 +96198 POINT(41.64315056484287 74.73304389370571) bank96198 +96199 POINT(40.441374495952516 74.22356583855827) bank96199 +96200 POINT(41.1765024934816 73.19105439311755) bank96200 +96201 POINT(41.47358599958668 74.28084739016317) bank96201 +96202 POINT(40.93634525493425 73.7103348357432) bank96202 +96203 POINT(39.90711913890624 74.05407121055737) bank96203 +96204 POINT(40.14162006378401 73.59123804799535) bank96204 +96205 POINT(41.5889156834993 73.39392147670867) bank96205 +96206 POINT(40.93890247996059 73.94824840605912) bank96206 +96207 POINT(40.92620259472232 74.0539242851435) bank96207 +96208 POINT(41.106446118313706 74.68453179895214) bank96208 +96209 POINT(41.08418569424057 74.24480929226571) bank96209 +96210 POINT(41.63208157432546 74.34260109087778) bank96210 +96211 POINT(41.049299095934494 73.3013120393449) bank96211 +96212 POINT(41.03959943323115 73.29426331610935) bank96212 +96213 POINT(41.6786192028869 73.28148810786767) bank96213 +96214 POINT(40.3620323296379 74.21114412853939) bank96214 +96215 POINT(40.3007281714177 73.42643064203226) bank96215 +96216 POINT(40.18702791877363 73.40213592735843) bank96216 +96217 POINT(40.23289722511987 73.93637267864908) bank96217 +96218 POINT(41.634937249631044 73.89773195702026) bank96218 +96219 POINT(41.352107590715946 74.32819564121559) bank96219 +96220 POINT(41.66565823393093 74.64743357044445) bank96220 +96221 POINT(41.635916704741554 74.91445940131402) bank96221 +96222 POINT(40.73074721547723 74.61183625934576) bank96222 +96223 POINT(40.32214183757591 74.47318302710634) bank96223 +96224 POINT(40.40275344040381 73.5210283447269) bank96224 +96225 POINT(40.886295579758 74.02513768950195) bank96225 +96226 POINT(40.80181060168688 73.83474995393011) bank96226 +96227 POINT(40.00502637169535 73.6937796835545) bank96227 +96228 POINT(41.08566767610297 74.77972177380545) bank96228 +96229 POINT(40.61704874334673 73.98184607292947) bank96229 +96230 POINT(40.64807467706612 73.4566678687724) bank96230 +96231 POINT(39.98418504899116 73.24330456024522) bank96231 +96232 POINT(41.71009709612019 74.66789251319464) bank96232 +96233 POINT(41.60747027179662 74.12018768960742) bank96233 +96234 POINT(40.62589034697235 73.30339371288572) bank96234 +96235 POINT(40.08535075736846 73.72945014187901) bank96235 +96236 POINT(40.763836373531234 73.67310903516452) bank96236 +96237 POINT(40.9178882007849 74.59242240010639) bank96237 +96238 POINT(41.58669196799167 73.3573313148193) bank96238 +96239 POINT(40.71782596405195 74.36261100890941) bank96239 +96240 POINT(41.030305405901345 73.03800435023473) bank96240 +96241 POINT(39.96115981057553 74.19751669634343) bank96241 +96242 POINT(40.269999833654005 73.312247200695) bank96242 +96243 POINT(39.85892168449681 74.22767450077666) bank96243 +96244 POINT(41.38767776045114 74.37596995832537) bank96244 +96245 POINT(40.655240609411116 74.81114563487998) bank96245 +96246 POINT(40.83998385315702 73.10568272892678) bank96246 +96247 POINT(40.39726652022659 74.48427974847621) bank96247 +96248 POINT(40.8187027272241 74.26127943098999) bank96248 +96249 POINT(41.59707510760264 73.99703673391485) bank96249 +96250 POINT(39.84103343757686 73.16882165881259) bank96250 +96251 POINT(40.56053818891216 73.2269472115949) bank96251 +96252 POINT(40.262617331979776 74.06632752048537) bank96252 +96253 POINT(39.976698487414474 73.87713174603095) bank96253 +96254 POINT(39.71971229737024 73.76824736519941) bank96254 +96255 POINT(39.96164925478916 74.4465877345752) bank96255 +96256 POINT(40.77762665578751 74.32511890601812) bank96256 +96257 POINT(41.4759588512579 74.74083738939348) bank96257 +96258 POINT(41.34888148418343 74.43781512721131) bank96258 +96259 POINT(40.29943286222971 74.01963472972292) bank96259 +96260 POINT(39.86281647000807 73.47373533181977) bank96260 +96261 POINT(40.510831429552944 73.8081881577778) bank96261 +96262 POINT(41.05733951552065 74.28909125646993) bank96262 +96263 POINT(41.1476882170792 74.55602827347863) bank96263 +96264 POINT(39.78843051003994 73.0393684961324) bank96264 +96265 POINT(41.60818760785722 73.85254389606233) bank96265 +96266 POINT(40.329832012880026 74.32946892542962) bank96266 +96267 POINT(41.63074523424967 74.1025558170027) bank96267 +96268 POINT(41.579920282008956 73.36009430311883) bank96268 +96269 POINT(40.41913453016073 73.85574138583485) bank96269 +96270 POINT(41.38815904789942 73.36711401850587) bank96270 +96271 POINT(41.24215699969149 74.67265076883156) bank96271 +96272 POINT(41.206666041324766 74.09330968422435) bank96272 +96273 POINT(39.85889195757642 73.2378627075254) bank96273 +96274 POINT(40.89398845020993 74.59662465340735) bank96274 +96275 POINT(39.76330170444344 74.64212614581483) bank96275 +96276 POINT(41.68376176586653 74.07979340353734) bank96276 +96277 POINT(40.50449674184817 74.06258002415709) bank96277 +96278 POINT(41.52646917888498 74.2499606625711) bank96278 +96279 POINT(41.44233918533648 73.17930920718415) bank96279 +96280 POINT(41.590675681963475 74.06005515051878) bank96280 +96281 POINT(40.19182911142799 73.99231853281725) bank96281 +96282 POINT(40.76303144200527 73.68016379326491) bank96282 +96283 POINT(40.14795598490705 73.09447912354834) bank96283 +96284 POINT(41.31948791604213 74.34925025212485) bank96284 +96285 POINT(40.855700651907746 73.75629607227337) bank96285 +96286 POINT(41.239676912238416 73.56750728773068) bank96286 +96287 POINT(40.43178796283549 73.82093936529262) bank96287 +96288 POINT(40.98938248088194 73.58527389619714) bank96288 +96289 POINT(41.249804423579846 73.3219856983973) bank96289 +96290 POINT(41.623236666488864 73.2126250782874) bank96290 +96291 POINT(40.06473063325465 73.89277107612337) bank96291 +96292 POINT(40.57192047671254 73.51930258051559) bank96292 +96293 POINT(40.823121879164354 74.28229752430701) bank96293 +96294 POINT(40.80303471617199 73.81782481812397) bank96294 +96295 POINT(40.10250298755784 73.66937873974685) bank96295 +96296 POINT(39.7827361325142 74.05862132613885) bank96296 +96297 POINT(40.41747260732992 74.01541249438243) bank96297 +96298 POINT(41.34163618557599 73.42128359466341) bank96298 +96299 POINT(41.24422715716018 73.6319162406558) bank96299 +96300 POINT(40.09840956857646 73.30006296005436) bank96300 +96301 POINT(41.39555393174197 74.98274212869372) bank96301 +96302 POINT(41.30372655238976 74.90356248438555) bank96302 +96303 POINT(40.38534100406358 73.49453092400788) bank96303 +96304 POINT(41.62296825315521 73.28222276680772) bank96304 +96305 POINT(40.079109613506795 74.63623957474259) bank96305 +96306 POINT(40.99215201841904 73.30341609643214) bank96306 +96307 POINT(41.61053146858823 73.14243597513749) bank96307 +96308 POINT(41.2844597170293 73.2375878786945) bank96308 +96309 POINT(39.72450831069208 73.5897374694813) bank96309 +96310 POINT(40.114430694284025 73.35975594309294) bank96310 +96311 POINT(41.6138493622099 74.9655262106743) bank96311 +96312 POINT(40.61055379616324 74.41431606236449) bank96312 +96313 POINT(39.84629657714288 74.8237941855393) bank96313 +96314 POINT(40.293787495492545 74.37183264733882) bank96314 +96315 POINT(40.818314683370964 73.79497379043472) bank96315 +96316 POINT(39.9854828558179 74.0358346947855) bank96316 +96317 POINT(39.774997955830344 74.70162040987228) bank96317 +96318 POINT(40.787732853395674 74.19239164765084) bank96318 +96319 POINT(40.332558524862485 74.17619164656445) bank96319 +96320 POINT(40.67331383139586 73.27300541945388) bank96320 +96321 POINT(40.943652980737895 73.5700613560827) bank96321 +96322 POINT(40.94256192018591 74.37693576383218) bank96322 +96323 POINT(40.1683431431472 73.61250950655041) bank96323 +96324 POINT(40.62045303845847 74.06048912464864) bank96324 +96325 POINT(40.80294679818968 73.04080461489106) bank96325 +96326 POINT(40.507156125531374 73.33120603579225) bank96326 +96327 POINT(40.1219402831817 74.56719600773258) bank96327 +96328 POINT(40.65914242500872 73.74158054222278) bank96328 +96329 POINT(40.5906885353984 73.23003975049348) bank96329 +96330 POINT(40.09624393997514 74.26060034997401) bank96330 +96331 POINT(41.28991676221844 73.34835356717929) bank96331 +96332 POINT(40.24576203914476 73.48546428263165) bank96332 +96333 POINT(41.08640061887682 73.9277378566834) bank96333 +96334 POINT(41.0236292492129 73.95497494934475) bank96334 +96335 POINT(40.73650251524896 74.18431602672983) bank96335 +96336 POINT(41.65542500671174 74.77549505032262) bank96336 +96337 POINT(40.89105765562727 74.32178836920434) bank96337 +96338 POINT(40.553331422266204 73.16122599504514) bank96338 +96339 POINT(41.120196136605855 74.27613896268475) bank96339 +96340 POINT(40.90472536549735 74.51965137542487) bank96340 +96341 POINT(40.35520526476099 73.22079514256379) bank96341 +96342 POINT(41.496736260366504 73.99916320532424) bank96342 +96343 POINT(40.88925579213704 74.82586297340454) bank96343 +96344 POINT(41.49081470511189 73.30380260580102) bank96344 +96345 POINT(40.45895560480426 73.66321240130395) bank96345 +96346 POINT(39.854347477223044 74.10690833201353) bank96346 +96347 POINT(40.343413807608584 74.70974691251983) bank96347 +96348 POINT(41.05991283910582 74.00344110047995) bank96348 +96349 POINT(40.63623251134433 73.4251885516819) bank96349 +96350 POINT(41.228267999051965 74.14885132563445) bank96350 +96351 POINT(40.57705123671487 74.34616197563011) bank96351 +96352 POINT(40.40705761014807 74.4167108631135) bank96352 +96353 POINT(41.089408169591614 74.8453840394538) bank96353 +96354 POINT(41.63257456804512 74.24365295205779) bank96354 +96355 POINT(41.51569464958048 74.23067590991904) bank96355 +96356 POINT(39.97244637576031 73.57877401171554) bank96356 +96357 POINT(40.69679712822271 74.47467958024092) bank96357 +96358 POINT(40.29866103708894 74.81807732958488) bank96358 +96359 POINT(41.50906678563165 74.62749070971242) bank96359 +96360 POINT(40.04346569585355 74.35415379055894) bank96360 +96361 POINT(40.300683719089754 73.23987047429665) bank96361 +96362 POINT(40.61407040056868 73.35955305431732) bank96362 +96363 POINT(41.58363351213165 73.74746225074908) bank96363 +96364 POINT(41.024631536871645 74.81245256204272) bank96364 +96365 POINT(41.39511066608153 73.05830590669602) bank96365 +96366 POINT(41.30624410997191 74.72355989195663) bank96366 +96367 POINT(40.31611610304405 73.45494763429097) bank96367 +96368 POINT(40.23540231883749 73.92720387520332) bank96368 +96369 POINT(41.61635743723368 74.70722074986645) bank96369 +96370 POINT(40.67277381114598 74.4143265545183) bank96370 +96371 POINT(40.80853741980053 73.08100470740573) bank96371 +96372 POINT(40.83728585699775 73.39492382199205) bank96372 +96373 POINT(40.28521226613986 73.2288063006506) bank96373 +96374 POINT(41.429154696026266 74.08465149178338) bank96374 +96375 POINT(41.584571400115784 73.0573214537772) bank96375 +96376 POINT(41.599542377714606 73.43950809495131) bank96376 +96377 POINT(40.70167893173215 73.0093366429572) bank96377 +96378 POINT(40.91839720798412 73.28424765270651) bank96378 +96379 POINT(39.722286795803 74.75286285095456) bank96379 +96380 POINT(40.12536784183135 74.92693442681835) bank96380 +96381 POINT(40.93498962899999 73.78932964864516) bank96381 +96382 POINT(40.29675287102225 74.05413069197799) bank96382 +96383 POINT(40.55466308883244 73.79585938404051) bank96383 +96384 POINT(39.7901762223034 73.8336228871718) bank96384 +96385 POINT(40.867146662326 73.86560508305666) bank96385 +96386 POINT(40.71495102250063 74.75297987919537) bank96386 +96387 POINT(40.42591052191645 74.46185313343051) bank96387 +96388 POINT(40.9164506932698 73.92905470457234) bank96388 +96389 POINT(40.14957594077253 74.02764547572093) bank96389 +96390 POINT(41.71221693392816 73.06173044437001) bank96390 +96391 POINT(39.95429371356638 74.98773689360169) bank96391 +96392 POINT(41.00446771585067 73.9883186065778) bank96392 +96393 POINT(40.89145908844585 74.86308704312493) bank96393 +96394 POINT(41.54172584794546 74.93631185402877) bank96394 +96395 POINT(41.30597130967658 74.99760511613735) bank96395 +96396 POINT(41.342173312832614 73.09471562852261) bank96396 +96397 POINT(39.83095587512812 74.96840959423317) bank96397 +96398 POINT(40.765466276499666 74.644920050936) bank96398 +96399 POINT(41.25961654548486 73.82094919311974) bank96399 +96400 POINT(41.368664293937 73.28249793995394) bank96400 +96401 POINT(41.28814601947713 74.65991015856758) bank96401 +96402 POINT(40.71002020809589 73.40913230210758) bank96402 +96403 POINT(40.28410469057224 73.69460621691572) bank96403 +96404 POINT(40.104133639359645 73.05687527535228) bank96404 +96405 POINT(40.57670232466594 73.66463516112854) bank96405 +96406 POINT(41.604952815839 74.42625575713039) bank96406 +96407 POINT(39.99062047202101 74.59137665087223) bank96407 +96408 POINT(40.74093049354633 74.17486998952306) bank96408 +96409 POINT(40.1355958191098 73.24847681656945) bank96409 +96410 POINT(40.26055995296806 73.3075867151283) bank96410 +96411 POINT(40.77099900436616 74.29639777672344) bank96411 +96412 POINT(41.246341975330765 73.84270506761374) bank96412 +96413 POINT(41.18011478795534 74.35455533007338) bank96413 +96414 POINT(41.51797555723519 73.75515313681225) bank96414 +96415 POINT(41.32357134682347 74.47700621228665) bank96415 +96416 POINT(40.857644584021195 74.35601823935633) bank96416 +96417 POINT(40.78295653960427 73.32435318385438) bank96417 +96418 POINT(41.543046593633484 74.37999839767511) bank96418 +96419 POINT(39.7294063012845 73.65155194763663) bank96419 +96420 POINT(40.61327397601886 74.56874137226418) bank96420 +96421 POINT(41.31443859626037 74.29004993565603) bank96421 +96422 POINT(40.45666251138967 74.37120711899925) bank96422 +96423 POINT(39.918106244836586 74.35238650750458) bank96423 +96424 POINT(39.915617633393325 73.9033799839001) bank96424 +96425 POINT(39.99542350243058 74.99673571650314) bank96425 +96426 POINT(40.91590893126 73.02359180124343) bank96426 +96427 POINT(40.67863211778208 74.646730861008) bank96427 +96428 POINT(40.82020781090711 74.28888634706803) bank96428 +96429 POINT(41.466783013894734 74.70994689025164) bank96429 +96430 POINT(39.912488028437096 74.69263351313603) bank96430 +96431 POINT(39.739557032912636 73.26650590689304) bank96431 +96432 POINT(39.75247989496682 73.45558591978454) bank96432 +96433 POINT(40.40884777855491 73.15831083953336) bank96433 +96434 POINT(40.00966831597228 73.10702821175008) bank96434 +96435 POINT(40.09738601161487 73.27307488380795) bank96435 +96436 POINT(39.85954786530887 74.8538823513925) bank96436 +96437 POINT(40.486114326702356 73.77707412232694) bank96437 +96438 POINT(39.895812436164185 73.24494784996716) bank96438 +96439 POINT(39.92539796903571 74.43519401583868) bank96439 +96440 POINT(41.55671083141197 73.34953076633109) bank96440 +96441 POINT(40.54117145985013 73.74651981381746) bank96441 +96442 POINT(41.55715595688332 73.24408405376194) bank96442 +96443 POINT(40.12330535469391 74.65165297907718) bank96443 +96444 POINT(40.601587924335675 74.67883814222024) bank96444 +96445 POINT(41.43407711407694 74.22349259727943) bank96445 +96446 POINT(41.38780522897263 73.3760101153919) bank96446 +96447 POINT(41.68240059040994 73.39929574228056) bank96447 +96448 POINT(41.23917771814137 73.15114873545554) bank96448 +96449 POINT(41.2220124925986 74.96297419557732) bank96449 +96450 POINT(40.86685721089781 73.01858915588126) bank96450 +96451 POINT(41.27408100413505 74.4833403822391) bank96451 +96452 POINT(40.27667663057608 73.67691344428304) bank96452 +96453 POINT(40.48057257990841 74.8145852937999) bank96453 +96454 POINT(40.07574271962515 73.59645051945495) bank96454 +96455 POINT(41.645430952863855 73.89061797341535) bank96455 +96456 POINT(40.863292376040825 74.50298371618923) bank96456 +96457 POINT(41.39864782632436 74.38364684154972) bank96457 +96458 POINT(40.64886048520308 74.34123493394422) bank96458 +96459 POINT(41.12686627576872 73.65775771599776) bank96459 +96460 POINT(41.25464738883243 74.79117234633696) bank96460 +96461 POINT(41.17074623684833 74.41052151766371) bank96461 +96462 POINT(39.9027359318947 74.93565044816133) bank96462 +96463 POINT(40.62370760260435 73.61083268125081) bank96463 +96464 POINT(41.38440437805453 74.23036936800814) bank96464 +96465 POINT(40.55442452217032 74.7122017305073) bank96465 +96466 POINT(40.32025309118135 73.76203337606877) bank96466 +96467 POINT(41.05538791968254 73.23493591872408) bank96467 +96468 POINT(39.83719165479101 73.18429194535557) bank96468 +96469 POINT(40.30345894682051 74.74270534828425) bank96469 +96470 POINT(40.291600774910414 74.35643642948749) bank96470 +96471 POINT(40.93923737635809 74.92843999615471) bank96471 +96472 POINT(41.44968734752328 73.15078403263345) bank96472 +96473 POINT(41.549075899659414 73.30406547077318) bank96473 +96474 POINT(40.1108707307122 73.31497775704233) bank96474 +96475 POINT(41.581151088900064 74.77666412202176) bank96475 +96476 POINT(40.417999623713165 73.18365674752701) bank96476 +96477 POINT(40.49948669489862 73.4371850996881) bank96477 +96478 POINT(41.40214190403381 73.06728525486666) bank96478 +96479 POINT(39.92161247757561 74.48351025230158) bank96479 +96480 POINT(40.919027024667685 73.48086521559443) bank96480 +96481 POINT(40.196729651570145 74.3845031606514) bank96481 +96482 POINT(40.19235242254766 73.82094305160525) bank96482 +96483 POINT(40.36620660511171 73.77945866245582) bank96483 +96484 POINT(40.715600566361175 74.49268102856661) bank96484 +96485 POINT(41.54550346992211 74.21765752987845) bank96485 +96486 POINT(39.83372913843495 73.68617968168986) bank96486 +96487 POINT(39.84688255584822 73.18007124929132) bank96487 +96488 POINT(40.24152813471841 74.38202500408391) bank96488 +96489 POINT(40.07687359030087 74.84502408059723) bank96489 +96490 POINT(41.13801820851218 74.79092352585613) bank96490 +96491 POINT(40.68116790389492 74.33615604063792) bank96491 +96492 POINT(40.815330084827735 73.69441610569149) bank96492 +96493 POINT(40.11711578579189 73.60132292336115) bank96493 +96494 POINT(40.50600789493471 74.61834159899063) bank96494 +96495 POINT(40.78619042517802 73.64787503925407) bank96495 +96496 POINT(40.04662717132546 74.60138414366007) bank96496 +96497 POINT(40.319208998310565 73.16150156096448) bank96497 +96498 POINT(41.475516459136074 74.90849662744017) bank96498 +96499 POINT(40.49298461577322 73.31831345571436) bank96499 +96500 POINT(41.3513453123532 74.8472486502358) bank96500 +96501 POINT(40.86479928393889 73.63854043104345) bank96501 +96502 POINT(41.7015380088094 74.3844514010937) bank96502 +96503 POINT(40.637876110231986 73.94785554846167) bank96503 +96504 POINT(39.760060270132946 73.57956929613725) bank96504 +96505 POINT(41.34905731994369 73.74970102945308) bank96505 +96506 POINT(39.893381977370616 74.96786237244099) bank96506 +96507 POINT(41.13615769824534 73.94666419043739) bank96507 +96508 POINT(41.33178005822721 73.43060844457864) bank96508 +96509 POINT(40.481700136506014 74.19977643598241) bank96509 +96510 POINT(40.23548719726881 73.57993149159294) bank96510 +96511 POINT(41.47872700498935 73.99993724426322) bank96511 +96512 POINT(40.75014137752954 73.18411525872119) bank96512 +96513 POINT(40.683681752475735 74.621417882868) bank96513 +96514 POINT(40.99516762588541 74.51941898076089) bank96514 +96515 POINT(41.24097653523524 74.93478504948598) bank96515 +96516 POINT(41.523054924542464 73.66423872278945) bank96516 +96517 POINT(41.301234729657644 73.75053072441506) bank96517 +96518 POINT(40.53144868710519 73.90962504940941) bank96518 +96519 POINT(41.10936243028746 74.28347860287145) bank96519 +96520 POINT(40.86461711592581 73.50027423016131) bank96520 +96521 POINT(41.19915138561581 74.5811520123925) bank96521 +96522 POINT(40.54067551511584 73.48127887043836) bank96522 +96523 POINT(40.4149346587888 73.95104689791182) bank96523 +96524 POINT(41.05150595120237 74.2145924026604) bank96524 +96525 POINT(40.844532211592366 73.03180911037124) bank96525 +96526 POINT(39.79952160542862 74.21653094818957) bank96526 +96527 POINT(41.48590350442196 74.79074539013305) bank96527 +96528 POINT(41.55712014026474 73.85496268232889) bank96528 +96529 POINT(41.23632655223725 74.4632801324812) bank96529 +96530 POINT(39.74925618965371 73.2905072905269) bank96530 +96531 POINT(41.68389598553774 73.87954949479133) bank96531 +96532 POINT(41.48236883365182 73.61056418033549) bank96532 +96533 POINT(41.35214438173025 74.67993868180271) bank96533 +96534 POINT(41.30884477054104 74.95849354632772) bank96534 +96535 POINT(40.70180359684595 74.91661733598895) bank96535 +96536 POINT(41.58677213631368 73.47383299505519) bank96536 +96537 POINT(41.65195300287599 73.29050120799663) bank96537 +96538 POINT(39.73929013404986 74.49153758331164) bank96538 +96539 POINT(41.39587489686405 73.91316855514431) bank96539 +96540 POINT(39.95996733705328 74.69640178589364) bank96540 +96541 POINT(40.32324202369524 74.68940361244302) bank96541 +96542 POINT(40.97105655200737 74.91296415383985) bank96542 +96543 POINT(41.611660398956744 74.92042876396302) bank96543 +96544 POINT(41.69483750595508 74.32827576674767) bank96544 +96545 POINT(40.66605165241515 74.28671051158327) bank96545 +96546 POINT(40.87398839724381 73.38456932800612) bank96546 +96547 POINT(40.21955765780816 74.32188695991937) bank96547 +96548 POINT(40.08701642295202 74.97306897982504) bank96548 +96549 POINT(40.37590636792998 74.12222584719024) bank96549 +96550 POINT(40.2470561820777 74.34584792147912) bank96550 +96551 POINT(41.5434336078238 74.65537179522109) bank96551 +96552 POINT(40.971796662408735 74.68991010621458) bank96552 +96553 POINT(41.294806462508255 74.98981112834531) bank96553 +96554 POINT(40.24984351047004 73.10975401711924) bank96554 +96555 POINT(41.650127029763375 73.80212507804077) bank96555 +96556 POINT(39.86966847921619 73.69853070897425) bank96556 +96557 POINT(41.43388017990352 73.27582560577444) bank96557 +96558 POINT(40.64239322200404 74.02761809138397) bank96558 +96559 POINT(41.02930711429154 74.11911589596349) bank96559 +96560 POINT(40.10433449033787 73.2464488276294) bank96560 +96561 POINT(40.159717640161944 73.76191610071483) bank96561 +96562 POINT(41.105998027451854 73.4592942994901) bank96562 +96563 POINT(40.10355413324071 74.9391657478199) bank96563 +96564 POINT(39.98023199194992 75.00162090859781) bank96564 +96565 POINT(40.18580266808101 74.8263533234933) bank96565 +96566 POINT(41.227113668639305 74.08880043254572) bank96566 +96567 POINT(40.259884592750566 73.92116529468503) bank96567 +96568 POINT(41.671010491103914 74.97007154336588) bank96568 +96569 POINT(41.112736515343535 74.13120030369876) bank96569 +96570 POINT(40.58942227498204 73.09653107721537) bank96570 +96571 POINT(41.39633590050489 74.48485234702866) bank96571 +96572 POINT(41.696734722968394 73.40903315705734) bank96572 +96573 POINT(40.44845869539261 73.25577162208171) bank96573 +96574 POINT(41.40518575713811 73.06855535316336) bank96574 +96575 POINT(40.623967188489935 74.2056208292079) bank96575 +96576 POINT(41.4550209684768 74.6161680691752) bank96576 +96577 POINT(40.33894571576629 74.10664917191872) bank96577 +96578 POINT(41.45519736736761 74.49075292254011) bank96578 +96579 POINT(41.41803050970508 74.0442398828919) bank96579 +96580 POINT(41.45527545810633 74.32489811716924) bank96580 +96581 POINT(40.67628832712692 74.6591437963559) bank96581 +96582 POINT(39.83160189268519 74.25081028192719) bank96582 +96583 POINT(40.93480509216756 73.39747020891429) bank96583 +96584 POINT(39.75703619387035 73.53567875196359) bank96584 +96585 POINT(40.823242242441296 73.32237462897737) bank96585 +96586 POINT(39.9825730568164 74.03228116061395) bank96586 +96587 POINT(39.740196428893135 74.57526913974603) bank96587 +96588 POINT(41.57184607589273 74.59359852876473) bank96588 +96589 POINT(39.93142753004336 73.78068090601815) bank96589 +96590 POINT(40.29712575314868 73.76553259778034) bank96590 +96591 POINT(40.57832856385396 74.31736698943257) bank96591 +96592 POINT(41.11762405021539 73.08524440235836) bank96592 +96593 POINT(41.62547894261312 73.03724768606368) bank96593 +96594 POINT(40.08297271351024 73.07967395134231) bank96594 +96595 POINT(39.73426066485231 74.33712102785722) bank96595 +96596 POINT(40.17005796231215 74.29125783727372) bank96596 +96597 POINT(40.89121780983189 73.74458003164585) bank96597 +96598 POINT(40.21549118336433 73.95521257958862) bank96598 +96599 POINT(39.88131405620068 73.70450296285826) bank96599 +96600 POINT(40.19756414805027 74.42052204217845) bank96600 +96601 POINT(40.14399529944334 74.51278001794881) bank96601 +96602 POINT(40.14192459411482 74.89624443164) bank96602 +96603 POINT(41.47204211092872 74.22193327791639) bank96603 +96604 POINT(41.64138102894218 73.38310339420116) bank96604 +96605 POINT(41.3566590126383 74.51782660173666) bank96605 +96606 POINT(40.380477977212756 73.38379638109511) bank96606 +96607 POINT(41.25873889252933 73.03736914878755) bank96607 +96608 POINT(41.4036016517044 73.46123283449658) bank96608 +96609 POINT(39.80932554136859 74.47038068684256) bank96609 +96610 POINT(39.96760443024362 74.2775720223202) bank96610 +96611 POINT(41.19195927907657 74.38180887049633) bank96611 +96612 POINT(41.01624321503843 74.94124869314842) bank96612 +96613 POINT(39.799411021467684 73.13588081322408) bank96613 +96614 POINT(41.678264589375004 73.26473397622968) bank96614 +96615 POINT(40.392778112719874 74.39576284829123) bank96615 +96616 POINT(39.89958966965196 74.75315695154953) bank96616 +96617 POINT(40.173901325578605 73.31218915040252) bank96617 +96618 POINT(41.503980517653325 74.22822245771634) bank96618 +96619 POINT(40.157714402818385 74.14067158522862) bank96619 +96620 POINT(41.37810380829061 73.42979215945331) bank96620 +96621 POINT(39.80686523255496 74.11286712903343) bank96621 +96622 POINT(40.06524702474594 73.39662278999167) bank96622 +96623 POINT(39.793373616590536 74.35857417414306) bank96623 +96624 POINT(40.2407202748734 74.01507872273432) bank96624 +96625 POINT(40.41438631943647 73.0073213818449) bank96625 +96626 POINT(41.69552337357729 73.4986722333863) bank96626 +96627 POINT(40.425611589686234 73.74336548564689) bank96627 +96628 POINT(41.50467824592734 74.99053761344335) bank96628 +96629 POINT(41.108456698994175 73.32047631818142) bank96629 +96630 POINT(40.656116129815196 73.96968984809112) bank96630 +96631 POINT(41.07130896662803 73.79330405662378) bank96631 +96632 POINT(41.3786341230505 74.12184118666579) bank96632 +96633 POINT(40.63374013761977 74.37564743959788) bank96633 +96634 POINT(41.50329674670883 74.06399373119913) bank96634 +96635 POINT(39.9670105509955 74.83347863792031) bank96635 +96636 POINT(41.26096470749861 74.82784970231643) bank96636 +96637 POINT(40.51304056039859 73.07162996519058) bank96637 +96638 POINT(41.487716895363135 74.69519888641378) bank96638 +96639 POINT(41.25415432787831 73.98875058641153) bank96639 +96640 POINT(41.290585200860455 74.43089252539066) bank96640 +96641 POINT(40.21190474791696 74.47308957852765) bank96641 +96642 POINT(41.09198208716753 73.7196199457673) bank96642 +96643 POINT(41.17540661458317 73.25816260235) bank96643 +96644 POINT(39.787444191973194 74.759115654156) bank96644 +96645 POINT(40.63325925136402 73.27935572922135) bank96645 +96646 POINT(40.82787360223607 73.52253313584268) bank96646 +96647 POINT(40.462570403426845 73.87806168215542) bank96647 +96648 POINT(41.036651170947614 74.97164093654432) bank96648 +96649 POINT(40.58617831233911 73.1027821520437) bank96649 +96650 POINT(40.52738197096491 74.17458686282848) bank96650 +96651 POINT(40.70630611631975 73.21938099997817) bank96651 +96652 POINT(40.91422582243813 74.91750628345551) bank96652 +96653 POINT(40.600457352411155 74.49401037683674) bank96653 +96654 POINT(40.995611391949836 74.76278364389513) bank96654 +96655 POINT(40.90893231694463 74.61087108530154) bank96655 +96656 POINT(40.28798850113323 74.72267033357666) bank96656 +96657 POINT(40.05730108544954 73.6015679585647) bank96657 +96658 POINT(40.60696776884088 74.83157300739506) bank96658 +96659 POINT(41.56973374081093 74.24561881291267) bank96659 +96660 POINT(41.18440605244 74.94259287748173) bank96660 +96661 POINT(40.80356249727713 74.95746536287271) bank96661 +96662 POINT(40.80754430157334 73.20859334591289) bank96662 +96663 POINT(41.288895346950746 73.50404187185558) bank96663 +96664 POINT(39.747965008582234 74.42087142481559) bank96664 +96665 POINT(41.16278159310104 73.25453199790634) bank96665 +96666 POINT(40.04365592924812 73.36156839441801) bank96666 +96667 POINT(41.28607625810799 73.93396660592047) bank96667 +96668 POINT(41.39801445778614 74.35139593294008) bank96668 +96669 POINT(41.44622849427267 73.78301717425326) bank96669 +96670 POINT(40.76567768449099 73.87673998702188) bank96670 +96671 POINT(40.426495944792386 74.64781653688323) bank96671 +96672 POINT(39.89981523223048 74.46613389423054) bank96672 +96673 POINT(40.696761719718815 73.48255167662153) bank96673 +96674 POINT(41.30634154668087 73.02516736855397) bank96674 +96675 POINT(41.14858026669427 74.82322600065686) bank96675 +96676 POINT(40.89950859412045 73.43027409711313) bank96676 +96677 POINT(40.08225297887413 74.09993730511052) bank96677 +96678 POINT(41.61734373846214 74.48295563243903) bank96678 +96679 POINT(40.62131495735958 73.82258207220475) bank96679 +96680 POINT(41.4182181963142 73.95548198163091) bank96680 +96681 POINT(40.28381659723964 73.90542329332554) bank96681 +96682 POINT(41.00773365823771 73.88186610351278) bank96682 +96683 POINT(41.254088150489956 74.94562156939881) bank96683 +96684 POINT(40.29418612675506 74.45241759727475) bank96684 +96685 POINT(40.264399041389424 73.65680566667352) bank96685 +96686 POINT(40.38822514221032 73.55529533784636) bank96686 +96687 POINT(41.65201867810892 73.75377320191916) bank96687 +96688 POINT(39.7685362460692 73.93425471223219) bank96688 +96689 POINT(40.06273253115411 73.7998827072326) bank96689 +96690 POINT(41.11407768685101 73.859507674381) bank96690 +96691 POINT(41.278428091779176 74.81799532886804) bank96691 +96692 POINT(40.0698955348347 73.38382170924613) bank96692 +96693 POINT(40.82999632397286 73.46113683962966) bank96693 +96694 POINT(40.32078738699648 74.9946835403574) bank96694 +96695 POINT(40.04544228945826 73.52567315034123) bank96695 +96696 POINT(40.87490424093065 73.99963690804493) bank96696 +96697 POINT(41.43016815869031 73.11230855013075) bank96697 +96698 POINT(41.60053140816977 74.62062184047407) bank96698 +96699 POINT(40.631544183509355 74.9309352806225) bank96699 +96700 POINT(41.38708763089957 73.89254995184262) bank96700 +96701 POINT(40.89656338734699 74.88947705012686) bank96701 +96702 POINT(40.752914853071736 74.80317343367778) bank96702 +96703 POINT(40.30487418690066 74.0627956468503) bank96703 +96704 POINT(40.13079682914914 74.05728637441828) bank96704 +96705 POINT(41.54437671863506 73.72148144098358) bank96705 +96706 POINT(40.15205327951396 73.59729475666383) bank96706 +96707 POINT(41.21612368124753 74.13657610497437) bank96707 +96708 POINT(41.252544066733655 74.01181278324812) bank96708 +96709 POINT(40.03601194919269 74.89836270315843) bank96709 +96710 POINT(40.84860405447467 73.14349172609643) bank96710 +96711 POINT(40.75103069361763 74.05373018296861) bank96711 +96712 POINT(40.73620563171015 74.80868289015235) bank96712 +96713 POINT(41.07253889057248 74.3572453245287) bank96713 +96714 POINT(40.42028157924805 74.0488229903859) bank96714 +96715 POINT(39.884271064947484 73.84142163459964) bank96715 +96716 POINT(41.088450720066 74.1038874647572) bank96716 +96717 POINT(40.81743618462563 73.0170649101421) bank96717 +96718 POINT(40.14338184851595 73.23303474885408) bank96718 +96719 POINT(39.729579071258115 74.19973752183105) bank96719 +96720 POINT(41.488965942599236 73.69213392606355) bank96720 +96721 POINT(40.10861852556809 74.49951730644455) bank96721 +96722 POINT(41.62008425647246 74.40067849305211) bank96722 +96723 POINT(40.15642974665566 74.61465858033696) bank96723 +96724 POINT(41.700326793656764 73.05630387322054) bank96724 +96725 POINT(40.94672061679995 73.81900903502361) bank96725 +96726 POINT(40.990102984911175 74.00076610677034) bank96726 +96727 POINT(41.06202024874763 74.23810944995772) bank96727 +96728 POINT(40.47374030166565 73.94993647808548) bank96728 +96729 POINT(40.01550358332213 73.85669409080401) bank96729 +96730 POINT(41.30442347945961 74.39157357220022) bank96730 +96731 POINT(41.37937606710383 73.70753716114083) bank96731 +96732 POINT(40.77583965672766 74.30207548772218) bank96732 +96733 POINT(40.09574801839949 74.97837713111764) bank96733 +96734 POINT(40.0196927294906 74.07177754349533) bank96734 +96735 POINT(40.83993716678139 74.48127308756582) bank96735 +96736 POINT(41.31996510158198 74.42092034244781) bank96736 +96737 POINT(39.78070192255921 74.38876213407246) bank96737 +96738 POINT(41.1185380569303 74.92714025252323) bank96738 +96739 POINT(40.66360876811708 73.93913583618928) bank96739 +96740 POINT(41.40772898107081 74.96122933819049) bank96740 +96741 POINT(39.91995342024352 73.7677017318007) bank96741 +96742 POINT(40.0852539568081 74.48437887746597) bank96742 +96743 POINT(41.31101766029831 74.96359210264066) bank96743 +96744 POINT(40.47704976839428 73.25761442954426) bank96744 +96745 POINT(40.63185329145296 73.75704616881443) bank96745 +96746 POINT(40.4618040201432 73.4180338384134) bank96746 +96747 POINT(39.78438721259934 74.25455376498937) bank96747 +96748 POINT(40.370025451995645 74.10788270941124) bank96748 +96749 POINT(41.278051629224294 74.7556100718596) bank96749 +96750 POINT(41.07075976102357 73.53754164271578) bank96750 +96751 POINT(40.59590064348777 74.98421735627139) bank96751 +96752 POINT(40.43952266994884 73.9438503070889) bank96752 +96753 POINT(41.29814943297444 74.50254940087709) bank96753 +96754 POINT(40.8811311313399 73.21477527127016) bank96754 +96755 POINT(40.703327205988856 74.09237975537965) bank96755 +96756 POINT(41.34468668967733 74.84151829467822) bank96756 +96757 POINT(41.67351688309568 74.80749631296307) bank96757 +96758 POINT(40.41806572585443 73.30805193780631) bank96758 +96759 POINT(40.24920476211025 74.79809685054629) bank96759 +96760 POINT(39.7534118743784 73.19720112925957) bank96760 +96761 POINT(40.01161669650824 74.37006504384615) bank96761 +96762 POINT(41.11622349433819 73.44683018704484) bank96762 +96763 POINT(41.70552627218047 74.46350702636751) bank96763 +96764 POINT(41.369929734089375 73.10622522053494) bank96764 +96765 POINT(41.10138331006364 73.41430517833854) bank96765 +96766 POINT(40.37011642923967 74.82315266297589) bank96766 +96767 POINT(40.5962069434166 74.77414754240316) bank96767 +96768 POINT(41.440195023091206 74.36722892607621) bank96768 +96769 POINT(39.85050152905386 74.77992913648774) bank96769 +96770 POINT(40.60106323327109 74.6573132556825) bank96770 +96771 POINT(40.82566471834236 74.15610821935825) bank96771 +96772 POINT(41.12223912831562 74.06626869546054) bank96772 +96773 POINT(41.68999181752645 74.22962993195226) bank96773 +96774 POINT(41.056913983515 74.81860326433177) bank96774 +96775 POINT(39.78702559098372 74.59331844072007) bank96775 +96776 POINT(41.62055859477246 74.16006656457705) bank96776 +96777 POINT(40.13493389901589 73.77501431880712) bank96777 +96778 POINT(40.18509901382258 74.82754410400223) bank96778 +96779 POINT(39.77964984449431 73.71301346769678) bank96779 +96780 POINT(40.97244053225891 73.38180222420196) bank96780 +96781 POINT(40.30038138469719 73.02802527329273) bank96781 +96782 POINT(41.54324406677799 73.00793452194995) bank96782 +96783 POINT(41.5279581002733 74.94061412236536) bank96783 +96784 POINT(41.61828774549802 73.26928921504825) bank96784 +96785 POINT(41.190660353953305 73.57876811478755) bank96785 +96786 POINT(41.20389839804547 73.18950061711618) bank96786 +96787 POINT(39.86890238444895 74.72931417656743) bank96787 +96788 POINT(39.956208681470216 73.32578848349621) bank96788 +96789 POINT(41.61200397093116 74.18404051668786) bank96789 +96790 POINT(40.82089779762696 73.68062712881769) bank96790 +96791 POINT(40.62704050195504 73.81681467154023) bank96791 +96792 POINT(39.72789417977508 74.5871185724258) bank96792 +96793 POINT(41.313317997974394 73.07591371226466) bank96793 +96794 POINT(41.579952016091475 73.67985711138522) bank96794 +96795 POINT(40.35120129656945 73.94185856361366) bank96795 +96796 POINT(40.463792316312514 73.60863851137081) bank96796 +96797 POINT(40.50455844479401 74.50771847557999) bank96797 +96798 POINT(40.1728909151745 73.93358638621478) bank96798 +96799 POINT(40.02354951124594 74.4264150511958) bank96799 +96800 POINT(40.17993689609885 74.76628172867743) bank96800 +96801 POINT(40.69918366325662 73.03294605767783) bank96801 +96802 POINT(40.064029675570325 74.32519670061937) bank96802 +96803 POINT(39.7686399628534 74.01056777600861) bank96803 +96804 POINT(41.378878330756336 73.93319349767049) bank96804 +96805 POINT(40.71134054841871 73.2515672809019) bank96805 +96806 POINT(40.87061520281846 73.44134268497007) bank96806 +96807 POINT(39.77823207824643 74.2831612596795) bank96807 +96808 POINT(41.295909807604744 74.51796356193492) bank96808 +96809 POINT(41.548724393783175 74.02665746077706) bank96809 +96810 POINT(40.71259985603621 74.56321005312213) bank96810 +96811 POINT(40.52025398435565 73.7266241282309) bank96811 +96812 POINT(40.93825400020824 73.23865876189763) bank96812 +96813 POINT(41.24797277664219 74.93273433032317) bank96813 +96814 POINT(40.36436226574725 73.52310560427755) bank96814 +96815 POINT(40.73923437991604 73.67612639094439) bank96815 +96816 POINT(40.12517669441191 73.91687236702342) bank96816 +96817 POINT(40.17560697657656 74.72143412033739) bank96817 +96818 POINT(40.8092669811713 74.34694823075912) bank96818 +96819 POINT(40.60502119684275 73.12093823076682) bank96819 +96820 POINT(39.93824005171904 73.04493405116853) bank96820 +96821 POINT(40.53709183308676 73.27158435980805) bank96821 +96822 POINT(40.60919863912616 73.45464390995458) bank96822 +96823 POINT(39.85255038329568 74.35614037071936) bank96823 +96824 POINT(39.72526135498159 73.10475434449747) bank96824 +96825 POINT(39.726331689528244 74.72073864175414) bank96825 +96826 POINT(39.7605987775092 74.73069228903996) bank96826 +96827 POINT(39.89715536777541 73.01882816726136) bank96827 +96828 POINT(40.96330807608921 74.12910147001762) bank96828 +96829 POINT(41.021121204117904 74.22464637396394) bank96829 +96830 POINT(41.18544843924368 74.10334413628918) bank96830 +96831 POINT(40.20055808646204 74.2086308734216) bank96831 +96832 POINT(40.87212719281882 73.03039580110578) bank96832 +96833 POINT(40.217351230754396 74.03629020744214) bank96833 +96834 POINT(40.651951521579214 74.92415837095527) bank96834 +96835 POINT(41.689919748662895 74.28210089709856) bank96835 +96836 POINT(40.56399457761269 74.1747257735738) bank96836 +96837 POINT(41.04791314073966 74.99756711122026) bank96837 +96838 POINT(40.69931879956724 74.56869205219402) bank96838 +96839 POINT(41.30168091890482 73.35063415044326) bank96839 +96840 POINT(40.46101630263764 73.13224357567553) bank96840 +96841 POINT(40.64041141383378 74.52375345561803) bank96841 +96842 POINT(41.230187096424096 74.10548505372702) bank96842 +96843 POINT(40.58790863240889 73.03898733332028) bank96843 +96844 POINT(41.51154892388092 74.3824806705453) bank96844 +96845 POINT(40.74166050286942 73.28925019697404) bank96845 +96846 POINT(40.127548399095005 74.71636275802732) bank96846 +96847 POINT(40.405441396801 73.95446637058298) bank96847 +96848 POINT(39.72206847076045 73.31861215197486) bank96848 +96849 POINT(40.30692009844738 74.98152021835344) bank96849 +96850 POINT(40.90674959990977 73.25467931111284) bank96850 +96851 POINT(41.54828128398557 73.04837705508136) bank96851 +96852 POINT(41.098899417113465 74.89211017486441) bank96852 +96853 POINT(40.80399250585417 74.1096844765845) bank96853 +96854 POINT(41.24129966234102 73.20508790546963) bank96854 +96855 POINT(41.310022903756526 73.67601593794963) bank96855 +96856 POINT(41.5869940662239 74.96546213813086) bank96856 +96857 POINT(41.165306286782446 73.34216808842454) bank96857 +96858 POINT(40.84672583561315 73.13715227885358) bank96858 +96859 POINT(40.55018452213782 74.69684780086648) bank96859 +96860 POINT(40.98073180154524 73.34225654567913) bank96860 +96861 POINT(41.54064522411901 73.88856852633664) bank96861 +96862 POINT(40.067917970918906 74.40603604943637) bank96862 +96863 POINT(41.179254375793384 74.0445762520862) bank96863 +96864 POINT(40.198696774911966 73.5027250273073) bank96864 +96865 POINT(39.91444881128273 73.88774630820785) bank96865 +96866 POINT(40.79522606614445 74.99675254081285) bank96866 +96867 POINT(39.97773398851599 74.48541721395435) bank96867 +96868 POINT(39.95672243118386 73.02824965743656) bank96868 +96869 POINT(41.25388542780337 74.67185831553526) bank96869 +96870 POINT(39.97866829669633 74.49140785522003) bank96870 +96871 POINT(39.80498979425029 73.94831000795855) bank96871 +96872 POINT(39.784850220610124 74.34614803257773) bank96872 +96873 POINT(40.056784282928824 73.59769884759473) bank96873 +96874 POINT(41.57252921275035 74.93514581916274) bank96874 +96875 POINT(39.790160989309896 73.38352686243255) bank96875 +96876 POINT(41.23846557846529 73.32847990488446) bank96876 +96877 POINT(41.253176755421634 73.72510689253728) bank96877 +96878 POINT(41.09600242865243 74.4100113585187) bank96878 +96879 POINT(40.84416721669551 73.19944521974196) bank96879 +96880 POINT(40.68395793002636 73.01098916518121) bank96880 +96881 POINT(41.394297938088506 73.83634475724683) bank96881 +96882 POINT(40.68117347676618 74.25270022290503) bank96882 +96883 POINT(40.26071845357515 74.70808351164283) bank96883 +96884 POINT(40.44772162809462 73.76127861992305) bank96884 +96885 POINT(41.26541356949839 74.26386887377745) bank96885 +96886 POINT(39.85331564771148 74.22109639003814) bank96886 +96887 POINT(40.23764993040817 74.84413362382257) bank96887 +96888 POINT(40.240897082772825 73.47020373811584) bank96888 +96889 POINT(41.705462291808225 73.61585461565322) bank96889 +96890 POINT(39.71870547560412 74.34745751361969) bank96890 +96891 POINT(39.71486210205194 73.14011267128913) bank96891 +96892 POINT(40.69300994129638 74.55311294509058) bank96892 +96893 POINT(41.17564544601083 74.01070252271246) bank96893 +96894 POINT(40.43002166081534 73.30995320710518) bank96894 +96895 POINT(41.37133055801328 74.04284464084809) bank96895 +96896 POINT(41.5093159929398 74.41554392626199) bank96896 +96897 POINT(40.84534154810046 74.57251269798658) bank96897 +96898 POINT(40.48013382200763 74.17972668888962) bank96898 +96899 POINT(40.040713441853306 73.45056396716882) bank96899 +96900 POINT(41.110201592688604 74.74554479451223) bank96900 +96901 POINT(40.770208115638034 74.2569202430886) bank96901 +96902 POINT(39.90506422783117 74.29343439471603) bank96902 +96903 POINT(40.07844125555788 74.6920631763521) bank96903 +96904 POINT(41.02548253579883 73.63976504572973) bank96904 +96905 POINT(40.03186672512555 74.86159868248163) bank96905 +96906 POINT(41.68686905085593 73.70022554325696) bank96906 +96907 POINT(41.095279227317086 73.75418295340823) bank96907 +96908 POINT(40.675804925234374 73.48192613934806) bank96908 +96909 POINT(40.66199591089866 73.13220097341369) bank96909 +96910 POINT(40.13799090131545 73.94321466024695) bank96910 +96911 POINT(40.35820260159991 74.23131451613885) bank96911 +96912 POINT(40.43798674785018 73.64435881119363) bank96912 +96913 POINT(40.63926843394059 74.48151464524095) bank96913 +96914 POINT(40.59269671412136 74.15670541381054) bank96914 +96915 POINT(40.103128380371125 73.19593038249347) bank96915 +96916 POINT(40.054799214365296 73.83697563591146) bank96916 +96917 POINT(40.436621369082516 73.58300311335427) bank96917 +96918 POINT(40.76310119139515 74.17809040945471) bank96918 +96919 POINT(41.07722405616482 74.41500586729698) bank96919 +96920 POINT(41.49894797829315 74.44244937901583) bank96920 +96921 POINT(39.983407207390066 74.62106979116894) bank96921 +96922 POINT(40.510248154458715 74.59317497389908) bank96922 +96923 POINT(41.36412320829561 74.88169690067063) bank96923 +96924 POINT(40.47952514690498 74.44237273599732) bank96924 +96925 POINT(39.83369880740468 73.49053660964134) bank96925 +96926 POINT(40.57802144098847 74.76295474592388) bank96926 +96927 POINT(39.80337562100538 73.82312554231191) bank96927 +96928 POINT(40.429733199776464 73.26651499267368) bank96928 +96929 POINT(41.54619070413017 73.21622893646946) bank96929 +96930 POINT(41.109121712613685 74.91095490327112) bank96930 +96931 POINT(40.01380576336689 73.09373283470728) bank96931 +96932 POINT(40.11848333174273 73.05083379501383) bank96932 +96933 POINT(40.51602215594923 74.68351231654961) bank96933 +96934 POINT(39.78173603003118 74.6835679314449) bank96934 +96935 POINT(41.1162975745076 74.35121147557861) bank96935 +96936 POINT(40.698295696033156 74.45876758541526) bank96936 +96937 POINT(39.966914119731136 74.44298366331138) bank96937 +96938 POINT(41.33445547214031 73.8637332670778) bank96938 +96939 POINT(39.787609483780734 73.99714356124869) bank96939 +96940 POINT(39.75704568439169 73.55015965435088) bank96940 +96941 POINT(40.957332479338106 74.9308310304754) bank96941 +96942 POINT(40.394507900519486 73.77011723978698) bank96942 +96943 POINT(41.21226861318259 74.84379592747264) bank96943 +96944 POINT(39.73735460249888 73.15446346329828) bank96944 +96945 POINT(40.24731797138686 73.47614770326265) bank96945 +96946 POINT(39.92421528938625 73.5490714059715) bank96946 +96947 POINT(40.617016543934376 73.77391583756048) bank96947 +96948 POINT(39.95329416155531 73.08438912984518) bank96948 +96949 POINT(40.16425477538917 73.99861451102838) bank96949 +96950 POINT(41.6798685498542 74.40036080968137) bank96950 +96951 POINT(39.88902197520164 73.79872626901569) bank96951 +96952 POINT(40.31922253041227 74.12889013499917) bank96952 +96953 POINT(40.39045142441646 74.55230170517957) bank96953 +96954 POINT(40.39578407062088 73.25732839272453) bank96954 +96955 POINT(41.26447607698366 73.90132110083377) bank96955 +96956 POINT(41.41022328438624 73.89051804625261) bank96956 +96957 POINT(40.918710220027705 73.37635121246093) bank96957 +96958 POINT(40.681559422107455 73.89748111300594) bank96958 +96959 POINT(41.15148591580778 74.32795772651947) bank96959 +96960 POINT(40.350635433200196 73.19268675899197) bank96960 +96961 POINT(40.66813159806074 74.26663831672451) bank96961 +96962 POINT(40.475492827901945 73.22461264961025) bank96962 +96963 POINT(41.22575675207871 74.23500461504175) bank96963 +96964 POINT(39.8019388514522 73.43942157086282) bank96964 +96965 POINT(40.717665868899935 74.40797562920838) bank96965 +96966 POINT(41.160364725613206 74.09807306060976) bank96966 +96967 POINT(40.985942439250934 74.25596215057617) bank96967 +96968 POINT(40.697696907995386 74.0748089359787) bank96968 +96969 POINT(41.467789844082255 73.95023422549231) bank96969 +96970 POINT(40.629129424319686 73.8681778294343) bank96970 +96971 POINT(41.0823255518521 73.1290921935597) bank96971 +96972 POINT(40.673060842288756 73.1996559201399) bank96972 +96973 POINT(41.062288514634936 73.85518352699485) bank96973 +96974 POINT(40.000398824141755 74.43608770006277) bank96974 +96975 POINT(40.971905052221665 73.44923907957266) bank96975 +96976 POINT(39.75615759062679 74.23983134439956) bank96976 +96977 POINT(41.12093300146387 73.78786081550055) bank96977 +96978 POINT(40.36466024478851 74.99345825532733) bank96978 +96979 POINT(41.51281112377054 73.71345170324582) bank96979 +96980 POINT(40.521170208158466 73.58599256574534) bank96980 +96981 POINT(40.44665456832457 73.67222558538923) bank96981 +96982 POINT(41.58748589236688 73.85827659191912) bank96982 +96983 POINT(40.77525701816739 73.6742275287259) bank96983 +96984 POINT(41.187962556033554 74.871044089235) bank96984 +96985 POINT(40.01023089563343 73.06169366052147) bank96985 +96986 POINT(41.543332501647406 74.32508184237658) bank96986 +96987 POINT(40.17004277778671 73.96334888964876) bank96987 +96988 POINT(41.04121977748708 74.41161850857466) bank96988 +96989 POINT(40.778509753638794 73.19548945174418) bank96989 +96990 POINT(39.75603318856837 73.73681668713495) bank96990 +96991 POINT(40.87499717396031 73.63533989840026) bank96991 +96992 POINT(40.811219966426755 73.99837268196522) bank96992 +96993 POINT(41.27444226167116 73.33490203429682) bank96993 +96994 POINT(39.84365685199878 74.14552281202477) bank96994 +96995 POINT(41.23541981589959 74.1952814658628) bank96995 +96996 POINT(41.07347530468158 74.32428306602498) bank96996 +96997 POINT(41.705640040122205 74.83562075467589) bank96997 +96998 POINT(39.83188647221979 73.1683975858628) bank96998 +96999 POINT(41.24249457734961 73.86542649289449) bank96999 +97000 POINT(41.39463512823874 73.27417259614289) bank97000 +97001 POINT(40.359305163151824 73.306124032472) bank97001 +97002 POINT(41.31717839891994 74.41306475149294) bank97002 +97003 POINT(40.92760761276851 74.8499848539374) bank97003 +97004 POINT(39.808901352538676 74.2629186912524) bank97004 +97005 POINT(41.19259040001085 74.43812187448115) bank97005 +97006 POINT(39.990071784968066 73.36097929340214) bank97006 +97007 POINT(41.44549552186062 73.96529942626196) bank97007 +97008 POINT(39.86806315828557 73.24523183882641) bank97008 +97009 POINT(40.68551143933137 74.51809737417145) bank97009 +97010 POINT(40.4933992635926 74.30908696629339) bank97010 +97011 POINT(40.08562075082912 73.6775595537479) bank97011 +97012 POINT(40.71717657224113 74.29804878807141) bank97012 +97013 POINT(40.92829757657224 74.9738214882465) bank97013 +97014 POINT(41.49631594751686 73.29689632805062) bank97014 +97015 POINT(39.84617799690281 74.22577311732944) bank97015 +97016 POINT(41.33357613869742 74.06505008705435) bank97016 +97017 POINT(39.97229573406747 74.06388804174999) bank97017 +97018 POINT(40.13833833099605 74.14605181032908) bank97018 +97019 POINT(40.35816391971942 73.92589570556359) bank97019 +97020 POINT(41.21176387399335 73.88304174906365) bank97020 +97021 POINT(41.594099780397016 74.0051511738477) bank97021 +97022 POINT(40.052924114372885 73.24385139900947) bank97022 +97023 POINT(40.02887940029452 74.20715314361557) bank97023 +97024 POINT(39.91754091035776 74.98900439053966) bank97024 +97025 POINT(41.16525832739978 74.75006119993878) bank97025 +97026 POINT(40.968256215939135 73.37160591907622) bank97026 +97027 POINT(41.50765148541821 73.64824950764871) bank97027 +97028 POINT(40.77053809173737 74.4769488167166) bank97028 +97029 POINT(40.373713057389345 74.1971931791976) bank97029 +97030 POINT(40.17725362522955 74.08945713462175) bank97030 +97031 POINT(41.361415143589376 74.99705841593432) bank97031 +97032 POINT(40.81393526468877 74.22330665329862) bank97032 +97033 POINT(40.23648256594679 74.71812147385559) bank97033 +97034 POINT(41.518933366667696 74.49573464268089) bank97034 +97035 POINT(39.751268208710286 74.9765195975367) bank97035 +97036 POINT(39.811735578373664 73.51562675670321) bank97036 +97037 POINT(40.23821369723252 73.55755667374662) bank97037 +97038 POINT(40.39488935951525 73.0439056658438) bank97038 +97039 POINT(40.50352308403806 74.31458257690704) bank97039 +97040 POINT(40.427098355665514 73.99344384866296) bank97040 +97041 POINT(41.20912265866417 74.0221241515068) bank97041 +97042 POINT(40.19897155652534 74.87351128277434) bank97042 +97043 POINT(40.00430850387834 74.84530265003363) bank97043 +97044 POINT(40.857481752591546 74.55338585238697) bank97044 +97045 POINT(40.10513335143342 74.70738757976183) bank97045 +97046 POINT(40.04314574447562 74.19093812630764) bank97046 +97047 POINT(41.711375036268464 74.90881668757946) bank97047 +97048 POINT(41.1184724742844 74.08584967387625) bank97048 +97049 POINT(41.303436957284376 74.50893342870063) bank97049 +97050 POINT(40.0219903296722 74.92844619284267) bank97050 +97051 POINT(40.71652224624872 73.36599362927629) bank97051 +97052 POINT(41.264800849820695 73.46188848584829) bank97052 +97053 POINT(41.15718085306893 73.11089340372268) bank97053 +97054 POINT(40.75664698065398 74.21791425006309) bank97054 +97055 POINT(41.22834217409857 74.23379500004818) bank97055 +97056 POINT(41.35905111643639 73.13625819851435) bank97056 +97057 POINT(40.302669898213495 74.78273190818176) bank97057 +97058 POINT(39.85619338895923 74.27978158696543) bank97058 +97059 POINT(40.2774941510727 73.99947802612567) bank97059 +97060 POINT(40.7906483216952 73.8383447364282) bank97060 +97061 POINT(40.72974318058586 73.66823107747733) bank97061 +97062 POINT(40.72514340168788 73.83666152157059) bank97062 +97063 POINT(39.84213172301767 73.7907073260819) bank97063 +97064 POINT(41.25104500150837 73.14962411696878) bank97064 +97065 POINT(41.526842527927684 73.79828004121234) bank97065 +97066 POINT(41.304401864708595 73.02477444173341) bank97066 +97067 POINT(39.78789701778621 74.89970923141085) bank97067 +97068 POINT(41.483864519880115 73.63307870313841) bank97068 +97069 POINT(39.9311145775093 73.08150121376232) bank97069 +97070 POINT(41.32755207726642 74.66249869221897) bank97070 +97071 POINT(40.785905081641275 73.89366958445092) bank97071 +97072 POINT(41.0422127274453 74.75485444745325) bank97072 +97073 POINT(40.59819092478647 74.70251030375317) bank97073 +97074 POINT(40.89423213268168 73.1505743565075) bank97074 +97075 POINT(41.32095816513178 73.89339788454387) bank97075 +97076 POINT(41.037784495772634 73.87349923117513) bank97076 +97077 POINT(41.52538556622231 74.55655768946045) bank97077 +97078 POINT(39.87209685006303 73.93086822085016) bank97078 +97079 POINT(40.342787857864224 74.00241206691025) bank97079 +97080 POINT(40.391793983500776 73.19211807946203) bank97080 +97081 POINT(39.88738039839506 74.2181378300371) bank97081 +97082 POINT(39.7359127034252 74.44172306950057) bank97082 +97083 POINT(41.10878842555017 73.33716538725676) bank97083 +97084 POINT(41.369226740746925 74.77179740779331) bank97084 +97085 POINT(41.4006486292357 74.99216956471523) bank97085 +97086 POINT(40.19500046398173 73.88067837490988) bank97086 +97087 POINT(40.6426042998789 74.24103603172672) bank97087 +97088 POINT(40.39064868485423 73.90407600855278) bank97088 +97089 POINT(39.98817866906434 74.29891130032262) bank97089 +97090 POINT(41.06394920089851 74.45362776949378) bank97090 +97091 POINT(40.26137649421061 74.54429628387669) bank97091 +97092 POINT(40.81321104073306 74.15854866677967) bank97092 +97093 POINT(39.85920594010695 74.59278372900918) bank97093 +97094 POINT(39.88393476629016 73.42307248018396) bank97094 +97095 POINT(39.93166903298275 74.50158147297526) bank97095 +97096 POINT(41.11599242218658 74.77175954635551) bank97096 +97097 POINT(40.02083421701274 74.93361963647017) bank97097 +97098 POINT(39.926311353940385 74.41008508178712) bank97098 +97099 POINT(40.790486613204 74.13682121887176) bank97099 +97100 POINT(40.37678437446456 73.84674845857808) bank97100 +97101 POINT(39.93005399126101 74.03614788302848) bank97101 +97102 POINT(40.74318823414101 73.04541299915604) bank97102 +97103 POINT(40.87634364190471 74.45705674945236) bank97103 +97104 POINT(40.32029523114087 73.95684842686447) bank97104 +97105 POINT(39.84302979819907 73.52029934121835) bank97105 +97106 POINT(40.72213383320783 73.01483792513451) bank97106 +97107 POINT(40.70413897954644 74.1950848655619) bank97107 +97108 POINT(41.527527460310885 73.5971428139702) bank97108 +97109 POINT(41.06618558622456 73.87384168759512) bank97109 +97110 POINT(40.56844675862035 74.51755324697223) bank97110 +97111 POINT(39.772253530079404 74.12625528251206) bank97111 +97112 POINT(40.939390302778726 73.30085335170331) bank97112 +97113 POINT(41.41892839576126 74.18756545468469) bank97113 +97114 POINT(39.844962369447 73.47649940240645) bank97114 +97115 POINT(40.07579445571375 73.75279202236845) bank97115 +97116 POINT(40.39658554979706 74.68228658085297) bank97116 +97117 POINT(40.13953214894446 73.80737918859496) bank97117 +97118 POINT(39.889552324558316 73.85786692302152) bank97118 +97119 POINT(40.53627021150307 73.36873775408803) bank97119 +97120 POINT(40.83721331606798 74.46381934753815) bank97120 +97121 POINT(40.26155826561876 74.29709307624388) bank97121 +97122 POINT(39.96785217917205 73.38144984974812) bank97122 +97123 POINT(41.30449427019418 74.58462819354338) bank97123 +97124 POINT(40.06787284095637 74.06374128263525) bank97124 +97125 POINT(40.11131119101457 74.12878654459028) bank97125 +97126 POINT(41.344069115456456 73.8087970583085) bank97126 +97127 POINT(41.08126796192392 73.86952611557862) bank97127 +97128 POINT(40.299410513356726 74.13110466700617) bank97128 +97129 POINT(40.121542904206166 73.11824160607104) bank97129 +97130 POINT(40.25241972416672 74.8768983624422) bank97130 +97131 POINT(40.389744675577376 74.04761804417066) bank97131 +97132 POINT(41.10277685785203 73.8308476769145) bank97132 +97133 POINT(40.74133215111574 74.13044400443128) bank97133 +97134 POINT(41.0913880829504 74.23609906974919) bank97134 +97135 POINT(40.678429562702945 73.61357863663534) bank97135 +97136 POINT(41.02181228208564 74.04236722542994) bank97136 +97137 POINT(40.16065274624044 74.09273225822773) bank97137 +97138 POINT(40.57402366255687 74.3384975713529) bank97138 +97139 POINT(40.62633123330312 74.25592041774608) bank97139 +97140 POINT(41.466576350007074 73.98808130294823) bank97140 +97141 POINT(40.70604105629701 73.94468944845379) bank97141 +97142 POINT(41.10628514821334 73.1279080034723) bank97142 +97143 POINT(40.00888232971269 73.98243470289295) bank97143 +97144 POINT(40.61881545212217 74.86616937447123) bank97144 +97145 POINT(41.31687745779246 74.35468062350428) bank97145 +97146 POINT(39.79425261653682 74.70738366264914) bank97146 +97147 POINT(41.545336014606534 74.97088281744537) bank97147 +97148 POINT(40.560709244510434 74.31421230915994) bank97148 +97149 POINT(41.57843946747016 74.17826774464677) bank97149 +97150 POINT(41.435604707144506 73.50641340384904) bank97150 +97151 POINT(40.767532902783756 74.99311178034053) bank97151 +97152 POINT(41.64328230595123 74.59928851651627) bank97152 +97153 POINT(39.88337719267392 74.94017801155842) bank97153 +97154 POINT(41.41602586772054 74.31288904218934) bank97154 +97155 POINT(41.2631919759551 74.49945086660635) bank97155 +97156 POINT(41.15120430707908 73.2189999474314) bank97156 +97157 POINT(40.76563065281125 73.08730206970378) bank97157 +97158 POINT(41.11373807269618 73.24769149246843) bank97158 +97159 POINT(41.682153577834505 74.94371474513983) bank97159 +97160 POINT(41.499486413202355 74.62856145489859) bank97160 +97161 POINT(39.998780091258 74.26591425723593) bank97161 +97162 POINT(41.342704407532466 74.8402355593826) bank97162 +97163 POINT(40.67921853751946 73.49096137507834) bank97163 +97164 POINT(40.848951959252844 74.74952090410535) bank97164 +97165 POINT(40.70375687337932 74.24123107960351) bank97165 +97166 POINT(40.163894420062746 74.60489508200294) bank97166 +97167 POINT(40.495736071817994 73.74424346844756) bank97167 +97168 POINT(41.18693515118049 73.26777222337208) bank97168 +97169 POINT(40.046539016644736 74.94753052161686) bank97169 +97170 POINT(41.032856487048896 73.58988713221362) bank97170 +97171 POINT(40.69189311862487 74.80416120263271) bank97171 +97172 POINT(40.96733509448794 73.18006819374777) bank97172 +97173 POINT(39.807314121352924 73.48959329905918) bank97173 +97174 POINT(41.37881978271898 74.3838326867235) bank97174 +97175 POINT(40.58164297785789 73.27887280035483) bank97175 +97176 POINT(39.98196181368592 74.1907575380261) bank97176 +97177 POINT(41.69492337744124 73.58175049638098) bank97177 +97178 POINT(40.60725258583483 74.85091057616064) bank97178 +97179 POINT(41.58104543061875 73.2246847413651) bank97179 +97180 POINT(41.36154513164766 74.81175327394186) bank97180 +97181 POINT(40.00199586617223 73.50177870965753) bank97181 +97182 POINT(39.80977642036194 73.47926004509667) bank97182 +97183 POINT(41.67080165128665 73.1258574961642) bank97183 +97184 POINT(39.72838403564401 73.15887259201246) bank97184 +97185 POINT(41.04236405634471 73.42805553085672) bank97185 +97186 POINT(40.26999150198864 73.59103964076262) bank97186 +97187 POINT(41.01265261064406 73.03710829624403) bank97187 +97188 POINT(41.31366870853617 73.99395417254762) bank97188 +97189 POINT(41.21364036601092 74.47483145686228) bank97189 +97190 POINT(39.971696623874365 74.31035935903249) bank97190 +97191 POINT(39.86666616248689 73.19218594468339) bank97191 +97192 POINT(41.44123067135748 73.38202222210298) bank97192 +97193 POINT(40.01097562328691 73.04384685565282) bank97193 +97194 POINT(40.246965512319704 73.09491881008974) bank97194 +97195 POINT(41.147336626196854 73.88580450685687) bank97195 +97196 POINT(41.630745376456176 74.01314860040917) bank97196 +97197 POINT(41.67937397723873 73.77670590241114) bank97197 +97198 POINT(41.610725332344145 73.22216433210758) bank97198 +97199 POINT(39.8404174081405 73.6034459011055) bank97199 +97200 POINT(41.58716688273178 74.3498425367122) bank97200 +97201 POINT(39.804488283001334 73.41169649542407) bank97201 +97202 POINT(40.76865980833062 73.25023790625818) bank97202 +97203 POINT(40.938842371648214 73.88472004660343) bank97203 +97204 POINT(40.972275509475736 74.47544428218804) bank97204 +97205 POINT(40.09215182893284 74.38423879053892) bank97205 +97206 POINT(41.58963917316858 74.20749175335536) bank97206 +97207 POINT(40.99128655534981 74.0328966714309) bank97207 +97208 POINT(40.18079804126259 73.33250514386665) bank97208 +97209 POINT(40.82421667581873 74.97725989937489) bank97209 +97210 POINT(39.99151602174479 74.72813337756322) bank97210 +97211 POINT(39.85757022426592 74.51451272508142) bank97211 +97212 POINT(40.90739731542602 74.8000806412999) bank97212 +97213 POINT(40.440135954371456 74.34697669841792) bank97213 +97214 POINT(41.255643321891576 74.04912247209273) bank97214 +97215 POINT(40.881555784510425 73.61471949176973) bank97215 +97216 POINT(40.7058634496007 74.81357115379242) bank97216 +97217 POINT(40.42973020213692 73.03643124843151) bank97217 +97218 POINT(41.04058349985618 73.45175742839403) bank97218 +97219 POINT(41.630522255369705 73.03153754491176) bank97219 +97220 POINT(40.95713698403624 74.33802462110754) bank97220 +97221 POINT(40.44208741395129 73.53652337910623) bank97221 +97222 POINT(41.46586957563405 73.52874735347746) bank97222 +97223 POINT(39.82199839588854 74.16958428833534) bank97223 +97224 POINT(40.8493547583476 73.92930501052793) bank97224 +97225 POINT(40.74224924174991 73.3040665564565) bank97225 +97226 POINT(40.070550746859304 73.12181434348818) bank97226 +97227 POINT(40.55292075596379 73.91586894830485) bank97227 +97228 POINT(41.60984613382841 74.46098634051181) bank97228 +97229 POINT(41.43025432121176 73.57722969404703) bank97229 +97230 POINT(40.51128314501067 74.23800476204181) bank97230 +97231 POINT(40.30706191527586 74.96491492354059) bank97231 +97232 POINT(40.4971019257675 74.00011436229572) bank97232 +97233 POINT(41.60787100404832 73.9920793382862) bank97233 +97234 POINT(40.90613421373712 74.66889515267856) bank97234 +97235 POINT(40.59031199230011 74.06352369727804) bank97235 +97236 POINT(40.40138738611067 73.28445734874812) bank97236 +97237 POINT(41.21661249633215 74.67179747003848) bank97237 +97238 POINT(41.22734863947762 73.12965754639045) bank97238 +97239 POINT(40.54910788250893 73.13689943712583) bank97239 +97240 POINT(40.37351712047042 73.2687251904266) bank97240 +97241 POINT(40.03900222511467 73.03028951103978) bank97241 +97242 POINT(41.56358544631296 74.53845134791318) bank97242 +97243 POINT(40.03188495862136 74.1960156323034) bank97243 +97244 POINT(40.51486787196354 73.27521750155971) bank97244 +97245 POINT(41.50181129741986 74.42987252199612) bank97245 +97246 POINT(40.66774658352391 73.39141897413536) bank97246 +97247 POINT(40.26872458603656 74.16068355084629) bank97247 +97248 POINT(40.84042874552328 74.3694864443451) bank97248 +97249 POINT(41.28178746075298 73.2650577798251) bank97249 +97250 POINT(40.02863712796564 73.8148284077387) bank97250 +97251 POINT(41.59649554385158 73.74035969594121) bank97251 +97252 POINT(39.92688646804265 73.48603166255252) bank97252 +97253 POINT(40.05303118366806 73.04266934455187) bank97253 +97254 POINT(41.050852507612376 73.74782793159098) bank97254 +97255 POINT(41.46081353249754 74.95365800988607) bank97255 +97256 POINT(40.60057397488011 74.87776935968228) bank97256 +97257 POINT(40.66798926765698 74.62560155135043) bank97257 +97258 POINT(40.881875839449584 74.64331897471047) bank97258 +97259 POINT(41.48721589539108 74.60898182506907) bank97259 +97260 POINT(40.501713945960006 73.1618666850628) bank97260 +97261 POINT(40.56512070172607 74.46378859581814) bank97261 +97262 POINT(40.03588706969601 73.32077797564997) bank97262 +97263 POINT(40.560613377556656 73.47006235581858) bank97263 +97264 POINT(40.86715947578986 74.0257648127138) bank97264 +97265 POINT(40.07972369067538 74.33415089652449) bank97265 +97266 POINT(40.93491474954133 73.0871889265668) bank97266 +97267 POINT(39.98342133403142 73.60659137288175) bank97267 +97268 POINT(39.73875156340268 74.98588586632015) bank97268 +97269 POINT(40.67941989684109 74.31165478667747) bank97269 +97270 POINT(41.08756827431231 74.06421487987622) bank97270 +97271 POINT(40.99124269243304 74.3538164941538) bank97271 +97272 POINT(41.24960621830836 74.22881258261465) bank97272 +97273 POINT(39.94415512104766 73.59166826281901) bank97273 +97274 POINT(40.15310546843578 73.72226364816035) bank97274 +97275 POINT(40.713001957282984 74.37496728213063) bank97275 +97276 POINT(40.390032642571796 74.08796984932548) bank97276 +97277 POINT(39.98205059410891 74.15913887494479) bank97277 +97278 POINT(40.223413171758224 74.5709270596738) bank97278 +97279 POINT(41.30246751812913 74.72711793193608) bank97279 +97280 POINT(40.3105272702496 73.49125469466489) bank97280 +97281 POINT(39.940390730128 74.81324754607672) bank97281 +97282 POINT(41.70996416837193 73.58724804858109) bank97282 +97283 POINT(40.67809265425751 74.37638291068652) bank97283 +97284 POINT(41.24130555900321 74.13963994184387) bank97284 +97285 POINT(40.26429201901139 73.51703856699844) bank97285 +97286 POINT(41.03524713220836 73.16123598293463) bank97286 +97287 POINT(39.86044103959586 73.73539158313932) bank97287 +97288 POINT(41.048045843711996 73.44687464789621) bank97288 +97289 POINT(40.32832963323765 74.39463529017732) bank97289 +97290 POINT(40.27390643192658 73.81847607926169) bank97290 +97291 POINT(40.68233181387813 74.46356972199254) bank97291 +97292 POINT(41.63442539336834 74.96729148745024) bank97292 +97293 POINT(41.53179464037143 73.12246576683104) bank97293 +97294 POINT(40.373492616524125 73.99018272465962) bank97294 +97295 POINT(41.01756515671941 74.85324436710572) bank97295 +97296 POINT(40.455429478815645 73.19352972298357) bank97296 +97297 POINT(40.48455155693231 74.57695246621567) bank97297 +97298 POINT(40.55806502691937 74.06933425515757) bank97298 +97299 POINT(40.513366514551144 73.27461232555937) bank97299 +97300 POINT(40.462404561907896 74.95008782271664) bank97300 +97301 POINT(41.49662635154996 74.80657371199202) bank97301 +97302 POINT(40.171320374763326 73.59824883241382) bank97302 +97303 POINT(41.45736567323648 73.18313337097507) bank97303 +97304 POINT(41.13403332064948 74.75123156673577) bank97304 +97305 POINT(40.88593547625428 73.67809505752412) bank97305 +97306 POINT(40.97168785686159 74.17044362160843) bank97306 +97307 POINT(40.97696214173438 74.79611557524656) bank97307 +97308 POINT(40.87343939428587 74.5936521965762) bank97308 +97309 POINT(41.598804781439846 73.34905055757564) bank97309 +97310 POINT(41.090034115880435 74.8021176524687) bank97310 +97311 POINT(40.35986563844275 74.36602753632405) bank97311 +97312 POINT(41.40100710334503 74.00036183818507) bank97312 +97313 POINT(40.66021927752482 74.82648267788042) bank97313 +97314 POINT(41.30132192066544 73.65836692450884) bank97314 +97315 POINT(40.03944918467947 73.63136691071199) bank97315 +97316 POINT(41.17436780216868 73.94411418607447) bank97316 +97317 POINT(40.114182485438256 74.79903171685886) bank97317 +97318 POINT(41.467377546308995 73.78248115532207) bank97318 +97319 POINT(40.535388319488796 74.86641069405513) bank97319 +97320 POINT(40.054804898570026 73.21108550633272) bank97320 +97321 POINT(40.991734409535326 74.35422394807362) bank97321 +97322 POINT(40.895612051593204 73.39508809771714) bank97322 +97323 POINT(40.83922730713448 73.77490369804835) bank97323 +97324 POINT(40.09485254184934 73.02583679616595) bank97324 +97325 POINT(39.88318369549528 73.7827482656311) bank97325 +97326 POINT(40.05188369308279 74.82214731960809) bank97326 +97327 POINT(39.96112404683495 74.15729186812598) bank97327 +97328 POINT(39.93515149030637 74.81690133831054) bank97328 +97329 POINT(41.648257858360076 73.99256449595121) bank97329 +97330 POINT(40.48005320614931 74.73855655339061) bank97330 +97331 POINT(39.91248046705436 74.9983512439142) bank97331 +97332 POINT(40.278143509596234 73.38899679401051) bank97332 +97333 POINT(40.72142575238954 74.89487453109359) bank97333 +97334 POINT(41.0939104330521 73.31252055341675) bank97334 +97335 POINT(40.705489885709845 74.1111914265986) bank97335 +97336 POINT(41.60813316668606 73.23845970889913) bank97336 +97337 POINT(39.91702135192759 73.25175454351489) bank97337 +97338 POINT(40.853346298230946 73.64269424167571) bank97338 +97339 POINT(41.640701477225285 74.15018999308415) bank97339 +97340 POINT(41.30663323314729 74.51740089385018) bank97340 +97341 POINT(41.55410718075917 74.13396630243913) bank97341 +97342 POINT(40.45261352003761 74.40744070368532) bank97342 +97343 POINT(41.4414069375605 73.8353681014075) bank97343 +97344 POINT(39.99348764721786 74.89771520305902) bank97344 +97345 POINT(41.07200823602712 73.6824092691086) bank97345 +97346 POINT(40.33532304426231 73.0362184085728) bank97346 +97347 POINT(40.74011592216516 73.58182745093055) bank97347 +97348 POINT(40.64148231007589 74.36288222340193) bank97348 +97349 POINT(40.33723038737619 74.3302251482206) bank97349 +97350 POINT(40.35260095350643 73.6132473159197) bank97350 +97351 POINT(41.25514032717926 73.0215272467245) bank97351 +97352 POINT(40.973330369642 74.19429120330608) bank97352 +97353 POINT(41.59487670420608 73.30824220688912) bank97353 +97354 POINT(39.89604340576508 73.86136804531526) bank97354 +97355 POINT(40.6027442968348 73.11378050137726) bank97355 +97356 POINT(41.04169555142359 74.15714760402288) bank97356 +97357 POINT(40.765861505956345 74.14321423543369) bank97357 +97358 POINT(39.917295300446284 74.41937594701331) bank97358 +97359 POINT(40.77139136245542 73.86151988882133) bank97359 +97360 POINT(41.631620381776564 74.20031629671705) bank97360 +97361 POINT(40.99644217940441 74.98420434297239) bank97361 +97362 POINT(41.00113051817482 73.91071169470253) bank97362 +97363 POINT(40.25349609645683 73.90926436829025) bank97363 +97364 POINT(40.266328074295096 74.38998452868687) bank97364 +97365 POINT(39.72679129526878 74.03298510432546) bank97365 +97366 POINT(40.34198586774328 74.38045366517773) bank97366 +97367 POINT(41.25080628181128 73.88972440029055) bank97367 +97368 POINT(40.94325681432255 74.84665576964943) bank97368 +97369 POINT(40.41763578036438 73.34832154028496) bank97369 +97370 POINT(40.09249989716561 74.68567200722909) bank97370 +97371 POINT(41.383081843298505 73.68669651930664) bank97371 +97372 POINT(39.82698249253642 74.86040877056939) bank97372 +97373 POINT(40.14602103822043 73.62051703992168) bank97373 +97374 POINT(39.88902886983591 74.94713950089685) bank97374 +97375 POINT(40.934947774653224 73.63063828959818) bank97375 +97376 POINT(40.865770914805765 73.9811538720418) bank97376 +97377 POINT(41.11721133554305 74.17405867332211) bank97377 +97378 POINT(41.1095026137341 73.80227785385905) bank97378 +97379 POINT(40.067291887436966 74.033606594762) bank97379 +97380 POINT(40.15140308462358 74.9058556931514) bank97380 +97381 POINT(40.949543119565455 74.49546660805063) bank97381 +97382 POINT(40.98692837223913 73.76525545701124) bank97382 +97383 POINT(40.53673901651086 74.19229986452467) bank97383 +97384 POINT(40.05969262065879 74.72112040014872) bank97384 +97385 POINT(39.91113012995944 73.34623780003032) bank97385 +97386 POINT(40.91536117214821 74.26795174806536) bank97386 +97387 POINT(39.865684760890645 74.97707056961907) bank97387 +97388 POINT(40.57950831209863 74.19020371944211) bank97388 +97389 POINT(41.0940772450478 73.69183324828019) bank97389 +97390 POINT(40.804052212974106 74.94221475941973) bank97390 +97391 POINT(40.2132469540517 74.36668723628905) bank97391 +97392 POINT(39.985170628078016 74.72428782038926) bank97392 +97393 POINT(41.154206973147815 74.91773533709902) bank97393 +97394 POINT(40.77415917669025 73.96174512201105) bank97394 +97395 POINT(40.34114637464045 73.94945867607537) bank97395 +97396 POINT(40.594107039029566 74.25662742748855) bank97396 +97397 POINT(39.761749612303866 73.86190668128879) bank97397 +97398 POINT(41.307062339238946 74.63339799) bank97398 +97399 POINT(40.906880824785546 73.74728555792262) bank97399 +97400 POINT(40.45459349403186 74.31049609016178) bank97400 +97401 POINT(41.30370034184798 73.80621501523572) bank97401 +97402 POINT(40.140578614908854 74.79468690924229) bank97402 +97403 POINT(40.7962163285228 74.97867037766315) bank97403 +97404 POINT(41.69654865174926 73.44417909903859) bank97404 +97405 POINT(40.75285795905187 74.936188545518) bank97405 +97406 POINT(40.019830616653174 73.5580735871084) bank97406 +97407 POINT(41.4027702020289 74.48340536357658) bank97407 +97408 POINT(40.17206574288391 73.42474903207884) bank97408 +97409 POINT(40.48912549255222 73.63849902354613) bank97409 +97410 POINT(41.700647452156744 74.3080870261613) bank97410 +97411 POINT(41.63906410515104 74.46751069159863) bank97411 +97412 POINT(40.10809740194604 74.82055745416706) bank97412 +97413 POINT(40.810226611505136 74.41074911196523) bank97413 +97414 POINT(41.11532773189504 74.75249344876903) bank97414 +97415 POINT(40.91857729799504 74.89592279832809) bank97415 +97416 POINT(41.09115323993745 74.19376662727123) bank97416 +97417 POINT(40.83635047351351 74.7204887423294) bank97417 +97418 POINT(41.61387104287015 73.68456517794753) bank97418 +97419 POINT(40.34568992864593 73.90028357683845) bank97419 +97420 POINT(40.418715096671185 73.23602923277443) bank97420 +97421 POINT(40.18383637510701 74.33818894283715) bank97421 +97422 POINT(41.27073875998168 73.28932239254523) bank97422 +97423 POINT(40.77867373499975 73.70600190273665) bank97423 +97424 POINT(40.044990370877876 74.24141668604977) bank97424 +97425 POINT(41.6320462005617 74.41725822350506) bank97425 +97426 POINT(41.614420096984034 73.3807767712834) bank97426 +97427 POINT(41.48508181823586 74.74110701233991) bank97427 +97428 POINT(40.11070747918468 74.56381782808506) bank97428 +97429 POINT(41.58214002363993 74.55908836131518) bank97429 +97430 POINT(40.47510839841402 73.44026663576679) bank97430 +97431 POINT(40.39224183866446 74.49178765024502) bank97431 +97432 POINT(40.05952143650803 73.16792445356948) bank97432 +97433 POINT(41.5282018091015 73.14586814145154) bank97433 +97434 POINT(40.94893557060609 73.09908054554106) bank97434 +97435 POINT(39.74914295255202 74.39657911588657) bank97435 +97436 POINT(40.35349301904378 73.37059058168617) bank97436 +97437 POINT(40.80526101805627 74.91040867797116) bank97437 +97438 POINT(40.7631126523095 73.30834055994549) bank97438 +97439 POINT(41.348421737825845 74.66676545611885) bank97439 +97440 POINT(41.361322585980936 73.51305462096106) bank97440 +97441 POINT(40.20436210060977 74.96496777987174) bank97441 +97442 POINT(39.80755742312678 73.3218394744213) bank97442 +97443 POINT(39.97536835128139 73.59082107741187) bank97443 +97444 POINT(39.72850620992886 74.21927609239692) bank97444 +97445 POINT(40.83242698804248 74.6196134179793) bank97445 +97446 POINT(40.8428516376279 74.8311435437002) bank97446 +97447 POINT(40.97461536755865 73.15906778783611) bank97447 +97448 POINT(41.14710567044635 74.1669219386245) bank97448 +97449 POINT(40.78871660507483 73.51911546719192) bank97449 +97450 POINT(41.638066118771945 73.16726291882186) bank97450 +97451 POINT(41.226588645570715 74.3581747615527) bank97451 +97452 POINT(41.536039840285795 74.23263183146466) bank97452 +97453 POINT(41.2259348047282 74.96455172723748) bank97453 +97454 POINT(39.94244605779657 73.45599848784023) bank97454 +97455 POINT(40.30690617395953 74.63371668323816) bank97455 +97456 POINT(40.59044308194181 74.27603387448198) bank97456 +97457 POINT(40.713164785311605 73.79059484727016) bank97457 +97458 POINT(40.70483960292619 74.22488487649957) bank97458 +97459 POINT(40.320728071319245 73.1437816291461) bank97459 +97460 POINT(40.113221424739145 73.485260125328) bank97460 +97461 POINT(40.354836834141636 74.54405567384283) bank97461 +97462 POINT(41.58689821010453 74.05163889741281) bank97462 +97463 POINT(40.11320495346509 74.22482010689403) bank97463 +97464 POINT(39.87004930135864 73.29027451026316) bank97464 +97465 POINT(40.434162016986654 73.51452650457738) bank97465 +97466 POINT(41.51743405330155 73.99873995582894) bank97466 +97467 POINT(41.317740281504356 73.18310561580184) bank97467 +97468 POINT(41.633110552111816 73.66043306092098) bank97468 +97469 POINT(40.00068029880462 74.88464294918829) bank97469 +97470 POINT(39.822291595713445 74.09310532685714) bank97470 +97471 POINT(41.00498101800701 74.73825515755102) bank97471 +97472 POINT(40.895442761398265 73.890190766701) bank97472 +97473 POINT(40.08744289289169 73.91506276868961) bank97473 +97474 POINT(41.1818330275536 74.62323512758812) bank97474 +97475 POINT(41.24864539893397 74.02295406825395) bank97475 +97476 POINT(39.732404504986256 73.22145917601337) bank97476 +97477 POINT(40.67011775230394 74.97084106960742) bank97477 +97478 POINT(40.31703252122781 73.65441200498893) bank97478 +97479 POINT(40.90565117500063 74.5015236406694) bank97479 +97480 POINT(40.13197728747539 74.8068527954068) bank97480 +97481 POINT(40.42064695317529 74.45427261803661) bank97481 +97482 POINT(41.607353145312636 74.38561174224324) bank97482 +97483 POINT(40.44920196219467 74.92311648467486) bank97483 +97484 POINT(41.584031948891564 73.2857387960168) bank97484 +97485 POINT(41.26314577933476 74.1457474172779) bank97485 +97486 POINT(39.947656296221446 73.02289292931874) bank97486 +97487 POINT(41.21425207185742 74.78804208904043) bank97487 +97488 POINT(40.69413029298397 73.4688499504179) bank97488 +97489 POINT(40.96479418060804 74.82613093909566) bank97489 +97490 POINT(40.29314015290611 73.43467393882437) bank97490 +97491 POINT(40.65503744524227 74.72922469716487) bank97491 +97492 POINT(39.79991429151885 73.30835279060389) bank97492 +97493 POINT(40.92690184714368 74.06637865860458) bank97493 +97494 POINT(39.726407360082405 73.1259267982354) bank97494 +97495 POINT(40.03550738264473 74.08140926818433) bank97495 +97496 POINT(41.421416326429984 73.14751979747679) bank97496 +97497 POINT(39.85353041068512 74.96817464425501) bank97497 +97498 POINT(41.02834983817076 74.13985563286089) bank97498 +97499 POINT(40.11894081556025 73.02243327503055) bank97499 +97500 POINT(41.35925853583537 73.68546755132293) bank97500 +97501 POINT(40.230691738433826 73.1521612167988) bank97501 +97502 POINT(41.036543477367125 73.60433939363439) bank97502 +97503 POINT(41.2856982116877 73.50426908114248) bank97503 +97504 POINT(40.58669664100341 73.30069609089827) bank97504 +97505 POINT(41.24478102522166 73.18255741370808) bank97505 +97506 POINT(41.49349846214427 74.52082382134668) bank97506 +97507 POINT(41.624365670803 74.70782275226264) bank97507 +97508 POINT(40.30648369622037 73.29088178008372) bank97508 +97509 POINT(39.95172467489639 74.76176932018453) bank97509 +97510 POINT(40.74082429568862 73.47560787532466) bank97510 +97511 POINT(41.381300719598975 73.45133530158684) bank97511 +97512 POINT(40.29880590677191 73.09192500422478) bank97512 +97513 POINT(40.464016669903806 73.03063657244046) bank97513 +97514 POINT(39.85300102689888 74.20152300868871) bank97514 +97515 POINT(41.468597135587764 74.8132287656332) bank97515 +97516 POINT(40.27713812449432 74.47330234255661) bank97516 +97517 POINT(40.55034335965905 74.5307394186709) bank97517 +97518 POINT(40.818076725595404 73.06722267868943) bank97518 +97519 POINT(41.39820775012304 74.70733738669846) bank97519 +97520 POINT(40.983858780415154 74.9065683545406) bank97520 +97521 POINT(41.58432912118455 73.8476148579223) bank97521 +97522 POINT(40.645065312764764 74.45455263619334) bank97522 +97523 POINT(40.633418662206914 73.56274322782566) bank97523 +97524 POINT(40.60206668503027 74.68688736254934) bank97524 +97525 POINT(39.804039625591365 74.4170055000003) bank97525 +97526 POINT(40.79595172673308 74.97230948329063) bank97526 +97527 POINT(40.88667985571134 73.01361558956484) bank97527 +97528 POINT(41.03271730192511 73.2854984444111) bank97528 +97529 POINT(41.15215381382143 74.78976252949337) bank97529 +97530 POINT(39.98740277877816 74.44493065726013) bank97530 +97531 POINT(40.29553583342936 73.38040126416945) bank97531 +97532 POINT(40.1328391373494 73.72833632877452) bank97532 +97533 POINT(40.668990197081975 73.17193215545547) bank97533 +97534 POINT(41.59684842911001 74.81530391195994) bank97534 +97535 POINT(40.47338259782651 73.06075868499481) bank97535 +97536 POINT(41.60797141669955 74.08164173583853) bank97536 +97537 POINT(40.827272577228726 73.41358812052233) bank97537 +97538 POINT(41.33937621321722 74.59594494598316) bank97538 +97539 POINT(40.203058130935176 74.45040717409447) bank97539 +97540 POINT(39.959261832484 74.02833301727264) bank97540 +97541 POINT(40.070002623239866 74.76277450470957) bank97541 +97542 POINT(41.646683750519216 73.51935156622096) bank97542 +97543 POINT(40.770263056734386 73.3883196730794) bank97543 +97544 POINT(41.61912615396736 73.7333281901645) bank97544 +97545 POINT(41.21900106491134 74.63572205353206) bank97545 +97546 POINT(39.763309202785074 73.76964634472164) bank97546 +97547 POINT(41.29748419334992 74.28828235090721) bank97547 +97548 POINT(40.15768798805696 74.15267321217699) bank97548 +97549 POINT(41.244685843027064 73.55023891010956) bank97549 +97550 POINT(40.41144448250631 73.41679351863012) bank97550 +97551 POINT(40.827590971378015 73.20665673621741) bank97551 +97552 POINT(41.014614696782715 74.5082119766108) bank97552 +97553 POINT(41.25353421914277 73.91221496680251) bank97553 +97554 POINT(40.487533207295535 74.60432046095735) bank97554 +97555 POINT(40.96426164719333 74.41835296265782) bank97555 +97556 POINT(40.063289240108546 73.1568182985999) bank97556 +97557 POINT(41.36277901579528 73.38930850334435) bank97557 +97558 POINT(41.57301167001366 73.77338479808955) bank97558 +97559 POINT(41.61510811285319 73.52677919767518) bank97559 +97560 POINT(41.32437144039353 73.8900778971828) bank97560 +97561 POINT(41.582519273092025 73.93490158273198) bank97561 +97562 POINT(41.29254964428336 73.96537832442128) bank97562 +97563 POINT(40.950661185363984 73.50100754593402) bank97563 +97564 POINT(41.66822993048176 73.05941609610134) bank97564 +97565 POINT(40.637986583622215 74.75333373852399) bank97565 +97566 POINT(41.048167688085755 74.40272101430888) bank97566 +97567 POINT(40.00569327671053 73.77867855812461) bank97567 +97568 POINT(41.52027830241391 73.19433504451499) bank97568 +97569 POINT(39.71852834612933 74.98160059652177) bank97569 +97570 POINT(40.96859442501378 74.97400057699438) bank97570 +97571 POINT(40.36538720290034 73.92472243044479) bank97571 +97572 POINT(40.77512206885763 74.88390884675928) bank97572 +97573 POINT(40.71437327160943 73.84713175442896) bank97573 +97574 POINT(40.45414319523733 73.92844952001266) bank97574 +97575 POINT(39.96153983379618 74.08550235897043) bank97575 +97576 POINT(40.65663175009594 73.84730501536804) bank97576 +97577 POINT(41.24908381084345 74.64777186220083) bank97577 +97578 POINT(39.94116628775904 73.9507607814461) bank97578 +97579 POINT(39.76748689798442 74.37738781555642) bank97579 +97580 POINT(41.53647081656158 74.954979560518) bank97580 +97581 POINT(41.544157246190885 74.31346251233137) bank97581 +97582 POINT(40.10478583239243 73.30486692288916) bank97582 +97583 POINT(40.70462853288001 73.66275530967368) bank97583 +97584 POINT(41.656206871031344 74.66742061448225) bank97584 +97585 POINT(39.86758139717801 74.01536123161006) bank97585 +97586 POINT(39.91066106671901 74.54410303756241) bank97586 +97587 POINT(40.438202557278494 74.55072054566672) bank97587 +97588 POINT(41.2816465198141 73.06783143924446) bank97588 +97589 POINT(40.95687474870453 74.62278856367186) bank97589 +97590 POINT(41.028983300539814 74.15204504148483) bank97590 +97591 POINT(41.6530620147548 73.22667806897617) bank97591 +97592 POINT(39.74222866580634 74.92362954412623) bank97592 +97593 POINT(40.38414170251374 74.58355977007271) bank97593 +97594 POINT(41.63212190004857 73.08530872778456) bank97594 +97595 POINT(40.93444499912442 74.4792352742949) bank97595 +97596 POINT(40.48000381829731 74.0088477700025) bank97596 +97597 POINT(40.18603733389256 73.86925150090968) bank97597 +97598 POINT(41.15292254916266 74.09879094440278) bank97598 +97599 POINT(40.64739709161943 74.56188479687978) bank97599 +97600 POINT(41.487741052992796 74.87883053162439) bank97600 +97601 POINT(39.95664043984354 73.82323449041144) bank97601 +97602 POINT(41.62060872830647 73.70747792133264) bank97602 +97603 POINT(41.655326192102294 74.33366964942837) bank97603 +97604 POINT(40.72663384502781 74.744338714255) bank97604 +97605 POINT(41.162885066420614 74.06646609265904) bank97605 +97606 POINT(40.05910109479684 73.84384960635458) bank97606 +97607 POINT(39.975860122425324 74.68140121794436) bank97607 +97608 POINT(39.787725000449385 74.26317810982655) bank97608 +97609 POINT(41.32160678376968 74.34302875556789) bank97609 +97610 POINT(41.30853997525132 74.02134784352056) bank97610 +97611 POINT(40.92165196929951 73.64341575115199) bank97611 +97612 POINT(40.11312703176189 73.13327171551825) bank97612 +97613 POINT(40.90958796802405 73.38603488507188) bank97613 +97614 POINT(40.5510259925323 74.1876722821994) bank97614 +97615 POINT(40.67081195805638 74.68431641559215) bank97615 +97616 POINT(41.62320421411888 74.39682943212595) bank97616 +97617 POINT(39.8291727393208 73.38646608518057) bank97617 +97618 POINT(40.92683397658736 73.46749806311252) bank97618 +97619 POINT(41.1360695284173 74.76290797103741) bank97619 +97620 POINT(39.95585292504095 74.23279973062446) bank97620 +97621 POINT(41.58842660540437 73.01357802906752) bank97621 +97622 POINT(41.51951321875844 74.796296354409) bank97622 +97623 POINT(41.2848612421741 73.33655530377409) bank97623 +97624 POINT(40.84254368909751 73.13613610567174) bank97624 +97625 POINT(41.34074124301023 74.93265650155097) bank97625 +97626 POINT(40.14684947297075 73.7900310526546) bank97626 +97627 POINT(40.80261192295441 73.34155523346085) bank97627 +97628 POINT(40.25316767786092 74.80611022297032) bank97628 +97629 POINT(40.66746438958249 74.44653590827174) bank97629 +97630 POINT(41.605093151940515 75.00217258539384) bank97630 +97631 POINT(40.09522809192737 74.39913017216287) bank97631 +97632 POINT(41.69432402411389 74.08486955710404) bank97632 +97633 POINT(41.46803659586551 73.03987115524576) bank97633 +97634 POINT(40.59741469937281 73.88345154384231) bank97634 +97635 POINT(40.798987986949065 74.40485972874711) bank97635 +97636 POINT(40.04298573053121 74.94687202783578) bank97636 +97637 POINT(40.604196976841855 73.72265032151641) bank97637 +97638 POINT(41.04718047821065 73.81129563432816) bank97638 +97639 POINT(40.41138767123491 73.30866941978508) bank97639 +97640 POINT(41.47923967335079 73.12337165384588) bank97640 +97641 POINT(40.560346718511745 74.26724783480712) bank97641 +97642 POINT(41.425420264542154 73.71394851603692) bank97642 +97643 POINT(40.2521344039423 73.62964616060925) bank97643 +97644 POINT(39.8221396967497 73.81069271758483) bank97644 +97645 POINT(41.506222903075695 73.51540873501787) bank97645 +97646 POINT(40.65645279241045 74.04132447379645) bank97646 +97647 POINT(40.97249294959772 73.75748913501232) bank97647 +97648 POINT(40.662283666662574 73.32374089794668) bank97648 +97649 POINT(41.36917446176822 74.14235089469146) bank97649 +97650 POINT(40.57136222464237 73.165314197947) bank97650 +97651 POINT(40.6485365914965 73.44966183351355) bank97651 +97652 POINT(41.015059127661345 74.65965614538426) bank97652 +97653 POINT(41.20402687680477 73.79878753048487) bank97653 +97654 POINT(41.515665007641886 73.92723819449691) bank97654 +97655 POINT(39.92253810256931 74.1823455134273) bank97655 +97656 POINT(41.555413968977916 74.81500875540584) bank97656 +97657 POINT(41.21168642039487 74.4880949305748) bank97657 +97658 POINT(40.84959183469058 73.13901897197982) bank97658 +97659 POINT(40.56652636436951 75.00571716807616) bank97659 +97660 POINT(40.75616835926519 74.50399905882989) bank97660 +97661 POINT(40.410253112924885 73.37596928185721) bank97661 +97662 POINT(40.17954529261736 74.68926697986063) bank97662 +97663 POINT(40.53091346510666 73.63421046101372) bank97663 +97664 POINT(40.88447289601235 74.48100889964962) bank97664 +97665 POINT(41.51472586723348 73.42497879272653) bank97665 +97666 POINT(39.928739892057926 73.36239465128865) bank97666 +97667 POINT(40.563514319302826 73.76921083583252) bank97667 +97668 POINT(41.217864829336634 74.41435912651482) bank97668 +97669 POINT(40.40709158472366 74.25171561278006) bank97669 +97670 POINT(40.05472611816769 74.58256730058457) bank97670 +97671 POINT(41.071069301765476 74.74065308515947) bank97671 +97672 POINT(41.629594604895246 74.35656650274746) bank97672 +97673 POINT(41.360810058445274 74.57205200339605) bank97673 +97674 POINT(41.58694527629997 74.01004556436895) bank97674 +97675 POINT(40.7623942621437 74.73139210323812) bank97675 +97676 POINT(40.09971414271278 73.58409488772158) bank97676 +97677 POINT(39.88595513895037 73.19468691545872) bank97677 +97678 POINT(40.407946389341355 74.14197665974928) bank97678 +97679 POINT(40.39067020006321 74.75888378448604) bank97679 +97680 POINT(39.723224133268964 74.68835508610627) bank97680 +97681 POINT(40.83849519784481 74.74215840599581) bank97681 +97682 POINT(40.84854899075166 74.34115067257831) bank97682 +97683 POINT(40.120699326646864 74.53593587232426) bank97683 +97684 POINT(40.99631471815431 73.10900048850824) bank97684 +97685 POINT(41.0114378770996 73.18941525785515) bank97685 +97686 POINT(40.43132726741611 73.68341862751885) bank97686 +97687 POINT(40.16116069333664 73.04753277411011) bank97687 +97688 POINT(39.849029173796765 74.36269516347149) bank97688 +97689 POINT(41.21207088596114 73.11222862273951) bank97689 +97690 POINT(40.61157711987234 73.67577532057558) bank97690 +97691 POINT(39.77784350489861 74.56336245506692) bank97691 +97692 POINT(40.50698165807773 74.3817089101569) bank97692 +97693 POINT(40.25004658024503 74.58895102201569) bank97693 +97694 POINT(41.41725953140272 74.28728409520932) bank97694 +97695 POINT(40.686543807253955 74.13876212133964) bank97695 +97696 POINT(41.00593445307599 74.0346471832083) bank97696 +97697 POINT(41.04535120842161 74.15734426861547) bank97697 +97698 POINT(41.04818140982437 74.80810830264616) bank97698 +97699 POINT(41.1851491709236 74.75537167146656) bank97699 +97700 POINT(40.31703459251566 74.36063398785501) bank97700 +97701 POINT(40.37583628896982 74.229790965941) bank97701 +97702 POINT(39.957251523599886 74.62098886884372) bank97702 +97703 POINT(39.79318793023155 73.15811828142773) bank97703 +97704 POINT(40.34230185218665 74.4476150917571) bank97704 +97705 POINT(40.13746802110267 74.0994470667766) bank97705 +97706 POINT(41.532794755641056 73.41137730932311) bank97706 +97707 POINT(40.689082838732176 73.09232776756777) bank97707 +97708 POINT(39.75209541348414 74.15264264191246) bank97708 +97709 POINT(41.20019363198304 74.13006072878098) bank97709 +97710 POINT(40.37647496508957 73.85070843897492) bank97710 +97711 POINT(40.471066100305094 73.71297024043618) bank97711 +97712 POINT(41.09008032661406 73.9581122715393) bank97712 +97713 POINT(40.329064712292016 74.95964447832266) bank97713 +97714 POINT(40.94635360885893 74.8970155351583) bank97714 +97715 POINT(40.17088398455425 74.49672387970026) bank97715 +97716 POINT(41.5126568481289 73.59233096219165) bank97716 +97717 POINT(40.20902332991561 74.45239311000566) bank97717 +97718 POINT(40.82450652699992 74.5816853656455) bank97718 +97719 POINT(40.544413116460625 73.32731508354006) bank97719 +97720 POINT(40.97421170104109 74.88772536012438) bank97720 +97721 POINT(41.3960715881302 73.74946473192506) bank97721 +97722 POINT(39.78826958959558 73.07878939446262) bank97722 +97723 POINT(41.70394098214513 73.67826684446173) bank97723 +97724 POINT(40.1900220644267 73.76611820639202) bank97724 +97725 POINT(39.96771894729848 74.93082974321712) bank97725 +97726 POINT(41.16356052224601 74.2264343491789) bank97726 +97727 POINT(40.01706335910853 74.64381502362798) bank97727 +97728 POINT(40.057420219458024 73.84711207441998) bank97728 +97729 POINT(41.44855716665608 74.10766693655903) bank97729 +97730 POINT(40.32667069194116 73.6798096230515) bank97730 +97731 POINT(40.46497857092158 74.50248202374202) bank97731 +97732 POINT(41.16373705246862 74.07836827109871) bank97732 +97733 POINT(40.67966939013417 74.112110496048) bank97733 +97734 POINT(40.987625869758084 74.80867319068345) bank97734 +97735 POINT(41.29024878327296 74.47698469777895) bank97735 +97736 POINT(40.72395329107303 73.49353423772897) bank97736 +97737 POINT(40.7649582894711 73.17436047465651) bank97737 +97738 POINT(39.762565360725546 74.91674777534307) bank97738 +97739 POINT(40.18671901165003 74.31224297468695) bank97739 +97740 POINT(40.46097123147977 73.23380307674404) bank97740 +97741 POINT(40.96137029404004 73.09878926121084) bank97741 +97742 POINT(40.13937387082917 73.41142778473369) bank97742 +97743 POINT(41.63299336947059 73.75101357490496) bank97743 +97744 POINT(39.92402356290218 74.49811910506074) bank97744 +97745 POINT(41.672114582246024 73.45074050796072) bank97745 +97746 POINT(40.65193614698928 74.97644888902836) bank97746 +97747 POINT(41.51478948026289 74.26759517325623) bank97747 +97748 POINT(39.78310944144673 74.71987771888854) bank97748 +97749 POINT(41.640508134861555 73.77219371903384) bank97749 +97750 POINT(40.97496040997699 73.16872032081844) bank97750 +97751 POINT(41.49244711432136 73.50946891075257) bank97751 +97752 POINT(40.08102225159129 74.04237915948856) bank97752 +97753 POINT(39.88371633972852 73.19945237276808) bank97753 +97754 POINT(40.09601289103314 73.88378712629094) bank97754 +97755 POINT(40.775669929586414 73.93197663569504) bank97755 +97756 POINT(41.54562660120445 74.29622081888007) bank97756 +97757 POINT(40.501084956163425 73.58020883627238) bank97757 +97758 POINT(41.6148169820341 73.81413754310732) bank97758 +97759 POINT(41.06636319624671 75.00552363812652) bank97759 +97760 POINT(41.60860653778836 74.16112828981812) bank97760 +97761 POINT(40.990716038915004 73.19991631455848) bank97761 +97762 POINT(41.52022800138723 74.84974098356383) bank97762 +97763 POINT(41.04086875499066 73.60808528040342) bank97763 +97764 POINT(41.70745416351099 74.9743763847124) bank97764 +97765 POINT(40.72658623044545 74.17546062492625) bank97765 +97766 POINT(41.35497791486016 74.13364396291436) bank97766 +97767 POINT(40.60034391693716 73.02734078639963) bank97767 +97768 POINT(41.69633869555153 73.42532989978935) bank97768 +97769 POINT(40.84884937922034 74.4767958695367) bank97769 +97770 POINT(41.253823469062255 74.43219725547667) bank97770 +97771 POINT(41.15699076086795 73.97867997833684) bank97771 +97772 POINT(40.58844930812543 74.26068000019468) bank97772 +97773 POINT(41.423537622369054 74.08929879782474) bank97773 +97774 POINT(41.69402926224224 73.83897387321613) bank97774 +97775 POINT(39.94599766293758 73.58004942052071) bank97775 +97776 POINT(41.60735485462496 74.51993072105444) bank97776 +97777 POINT(40.929142913461604 74.94062793197479) bank97777 +97778 POINT(40.73247897099764 73.41753832475945) bank97778 +97779 POINT(40.21671451568412 73.2572150817581) bank97779 +97780 POINT(40.368341119584976 74.4842827861951) bank97780 +97781 POINT(41.23281373944461 74.61906801092522) bank97781 +97782 POINT(40.22376998711351 73.1582152496635) bank97782 +97783 POINT(41.08467903376708 74.27636216972597) bank97783 +97784 POINT(41.64175413310773 73.78982577464258) bank97784 +97785 POINT(41.06417455774264 73.83572371384662) bank97785 +97786 POINT(41.12222808859565 74.54084441782332) bank97786 +97787 POINT(40.67958037396298 73.24268979784392) bank97787 +97788 POINT(40.599556547003814 74.17327477025167) bank97788 +97789 POINT(41.39113759038439 73.47152868129209) bank97789 +97790 POINT(40.87112377873484 74.93890749041257) bank97790 +97791 POINT(39.951576288844684 73.95462019360222) bank97791 +97792 POINT(40.67194480701708 73.19617105520432) bank97792 +97793 POINT(41.24245840602537 74.93617122702065) bank97793 +97794 POINT(40.88990818615528 73.73748039145823) bank97794 +97795 POINT(41.061842134756176 74.7318997947705) bank97795 +97796 POINT(41.48670704641071 74.19041796071188) bank97796 +97797 POINT(40.66891601601366 73.03768151285661) bank97797 +97798 POINT(40.37640285332316 73.91679866169098) bank97798 +97799 POINT(41.43035701177406 73.39938325899728) bank97799 +97800 POINT(39.84109729754973 74.03434654814892) bank97800 +97801 POINT(40.64585102003205 73.68778144867682) bank97801 +97802 POINT(40.80447642459459 73.53839375932168) bank97802 +97803 POINT(40.28207714274032 73.66651694471476) bank97803 +97804 POINT(41.01887052536795 73.46955196919414) bank97804 +97805 POINT(41.20802791206019 74.09664153681828) bank97805 +97806 POINT(40.18673636339804 74.24178902465292) bank97806 +97807 POINT(40.873983091046405 73.62819002450895) bank97807 +97808 POINT(40.32779609945448 73.12648107110273) bank97808 +97809 POINT(40.3702258922719 74.41281604858345) bank97809 +97810 POINT(40.15487263983732 74.83910184037683) bank97810 +97811 POINT(40.93766512671284 74.38664928892656) bank97811 +97812 POINT(40.63682272852007 73.96260417565173) bank97812 +97813 POINT(40.84538305406893 74.31041135047231) bank97813 +97814 POINT(41.34717329080217 73.63355557711597) bank97814 +97815 POINT(41.390822237585894 73.18694657666067) bank97815 +97816 POINT(40.28822139767479 74.02818282085619) bank97816 +97817 POINT(40.8415534542845 74.45702028727445) bank97817 +97818 POINT(40.634332327655116 74.49227866177041) bank97818 +97819 POINT(41.200863342396225 74.79314428988378) bank97819 +97820 POINT(40.93950149439194 73.05130573238911) bank97820 +97821 POINT(41.118100910500246 73.60723429954452) bank97821 +97822 POINT(40.1266971174827 73.57540789179762) bank97822 +97823 POINT(39.7568501888399 74.07573999540011) bank97823 +97824 POINT(40.23139621633332 73.93934166689635) bank97824 +97825 POINT(39.99409867289371 73.75072889309916) bank97825 +97826 POINT(41.56976614983658 74.86136984204255) bank97826 +97827 POINT(41.395405442201906 73.52230395067134) bank97827 +97828 POINT(39.71725016649933 74.41171577890628) bank97828 +97829 POINT(41.2629005717191 74.99213905796425) bank97829 +97830 POINT(39.99009427710557 73.17861986125759) bank97830 +97831 POINT(41.1052482571773 74.50612178630735) bank97831 +97832 POINT(41.27725970167092 74.9747858596836) bank97832 +97833 POINT(41.47859699316134 74.39395119573582) bank97833 +97834 POINT(40.8905564621207 74.2168264528076) bank97834 +97835 POINT(40.13938363106093 73.86434239939688) bank97835 +97836 POINT(41.35185683308245 73.96106092833053) bank97836 +97837 POINT(39.89596428365505 73.56008438977352) bank97837 +97838 POINT(40.09131111346474 74.55523699361729) bank97838 +97839 POINT(40.01869522612549 73.51047576956147) bank97839 +97840 POINT(40.40265437390801 74.82015092499527) bank97840 +97841 POINT(41.67991729699472 74.95630179936859) bank97841 +97842 POINT(40.66949441433794 74.14825308859997) bank97842 +97843 POINT(40.88469508874093 74.69672117558585) bank97843 +97844 POINT(40.09454713974629 73.68329721919027) bank97844 +97845 POINT(41.54919373725477 73.27227150758767) bank97845 +97846 POINT(40.09872942957721 73.1053968010422) bank97846 +97847 POINT(40.502096446331606 74.09041398508431) bank97847 +97848 POINT(41.10608363872572 74.51138764391173) bank97848 +97849 POINT(41.48749886684391 73.73739171489403) bank97849 +97850 POINT(41.2611463174755 73.40997794704936) bank97850 +97851 POINT(41.50560762813931 74.17641123640895) bank97851 +97852 POINT(41.20135710394025 73.10610551147694) bank97852 +97853 POINT(40.02170343129911 73.58364990149131) bank97853 +97854 POINT(40.45126638838135 73.15741281741609) bank97854 +97855 POINT(41.39377284035149 74.4348036990381) bank97855 +97856 POINT(40.655848660898485 74.02871169554906) bank97856 +97857 POINT(40.52177930264562 74.36840798826854) bank97857 +97858 POINT(40.56466695267675 74.06043489001546) bank97858 +97859 POINT(39.81018642624708 73.71625453027505) bank97859 +97860 POINT(41.06515514767075 74.6916974968917) bank97860 +97861 POINT(40.141262991634605 74.03928301720143) bank97861 +97862 POINT(40.73273945730513 73.01929091334699) bank97862 +97863 POINT(41.68382397551028 74.24845787348735) bank97863 +97864 POINT(40.90856829979046 74.74515440359332) bank97864 +97865 POINT(40.96714763019985 74.39183664235375) bank97865 +97866 POINT(41.23432067451747 73.27338738924355) bank97866 +97867 POINT(40.76308732079694 73.60373148147994) bank97867 +97868 POINT(39.81376869750315 73.68956397023186) bank97868 +97869 POINT(40.37478414317618 73.66998557818019) bank97869 +97870 POINT(40.90423689290975 73.45915271832573) bank97870 +97871 POINT(40.93714227797472 73.61543631148923) bank97871 +97872 POINT(41.610494597303145 74.89015245335843) bank97872 +97873 POINT(40.60882098794223 74.0364657044904) bank97873 +97874 POINT(40.4960076401303 74.82266090888984) bank97874 +97875 POINT(40.71350523463802 74.31074566167528) bank97875 +97876 POINT(40.00402278942396 73.93267257053766) bank97876 +97877 POINT(40.96378931020168 73.76770957427406) bank97877 +97878 POINT(41.403641413057564 74.34711950093066) bank97878 +97879 POINT(40.96182287663822 74.33591711663837) bank97879 +97880 POINT(40.44340181462384 74.7522478425263) bank97880 +97881 POINT(40.44298037103374 73.28362827778582) bank97881 +97882 POINT(40.840614149295966 74.43890609279786) bank97882 +97883 POINT(40.06609577917623 73.5721760886142) bank97883 +97884 POINT(41.34729023634926 73.93574539902484) bank97884 +97885 POINT(41.14842754683846 73.35574084542509) bank97885 +97886 POINT(41.47952774344954 74.06567067223092) bank97886 +97887 POINT(41.49583653259735 74.10269881312463) bank97887 +97888 POINT(40.462289752880466 74.83668701378055) bank97888 +97889 POINT(41.39642909117483 74.0800369067578) bank97889 +97890 POINT(39.87932673796122 74.51404013234277) bank97890 +97891 POINT(41.06946053086025 74.43136201567141) bank97891 +97892 POINT(40.01416449075056 74.3771948950174) bank97892 +97893 POINT(40.099287235129005 74.94214333904614) bank97893 +97894 POINT(40.826137204465084 73.85972069412801) bank97894 +97895 POINT(41.09593985678694 74.65704384327206) bank97895 +97896 POINT(40.6907215976029 73.42608799900867) bank97896 +97897 POINT(40.28316561082154 74.02357872367968) bank97897 +97898 POINT(40.19250950311314 73.05505054735164) bank97898 +97899 POINT(40.532973834199616 74.59454181105579) bank97899 +97900 POINT(41.68962985718777 74.02964990017713) bank97900 +97901 POINT(40.55584655391033 73.2868336049702) bank97901 +97902 POINT(41.3111948904702 74.20392062513598) bank97902 +97903 POINT(39.88680080358158 74.48323116206645) bank97903 +97904 POINT(41.11113558388974 73.36241537229033) bank97904 +97905 POINT(39.930753831744155 73.58081695803942) bank97905 +97906 POINT(40.930384568587044 74.71680705047591) bank97906 +97907 POINT(41.214484880110376 74.88453459744545) bank97907 +97908 POINT(40.72739803297926 74.99124988004803) bank97908 +97909 POINT(40.82696883270092 73.91391999326902) bank97909 +97910 POINT(41.4803522560621 73.8239302074908) bank97910 +97911 POINT(41.37413394989052 74.13705909198346) bank97911 +97912 POINT(39.93054091648571 74.75741533712561) bank97912 +97913 POINT(41.48139611762169 74.81341856874236) bank97913 +97914 POINT(39.938046957293196 74.22118637105083) bank97914 +97915 POINT(41.401497643176214 74.7583654061716) bank97915 +97916 POINT(41.599883885721766 74.72015469668672) bank97916 +97917 POINT(41.45588639418549 74.98165352061778) bank97917 +97918 POINT(40.48286638498041 74.57199670356883) bank97918 +97919 POINT(40.761723150968656 73.35864547230914) bank97919 +97920 POINT(41.07670004595173 73.50104891519852) bank97920 +97921 POINT(40.488313820983024 73.0929127150595) bank97921 +97922 POINT(40.37171500501129 73.9015516547143) bank97922 +97923 POINT(41.63032562596424 74.30833872881354) bank97923 +97924 POINT(39.87065634523207 73.18256154845791) bank97924 +97925 POINT(41.43137988542696 74.41103400425833) bank97925 +97926 POINT(41.03211788406005 73.27005518999621) bank97926 +97927 POINT(41.43271728749315 73.91142939570754) bank97927 +97928 POINT(41.288885046951144 73.0452925425384) bank97928 +97929 POINT(39.96338221530684 74.25888498738867) bank97929 +97930 POINT(39.94107512125569 73.09641438192517) bank97930 +97931 POINT(40.66230500873466 73.12828722565366) bank97931 +97932 POINT(41.40375644869782 74.09327766184958) bank97932 +97933 POINT(39.881966955560536 73.20714810073537) bank97933 +97934 POINT(39.78553390506933 73.78921872652788) bank97934 +97935 POINT(41.22051275981877 74.17091404962538) bank97935 +97936 POINT(40.82231010326157 74.16422652531206) bank97936 +97937 POINT(39.79556032371473 73.04669900701349) bank97937 +97938 POINT(40.92067398589286 74.74497916553615) bank97938 +97939 POINT(41.31462024874798 74.59764837928566) bank97939 +97940 POINT(40.1001100498052 73.07104056999735) bank97940 +97941 POINT(40.07960576086365 73.57422022144463) bank97941 +97942 POINT(40.47728483534004 74.02143092234033) bank97942 +97943 POINT(40.43184602310444 73.6790808166754) bank97943 +97944 POINT(40.94433080921465 74.40386691288735) bank97944 +97945 POINT(40.360539716186985 74.41826876543152) bank97945 +97946 POINT(40.47493352705279 74.48526831302361) bank97946 +97947 POINT(41.683513173679906 74.96609303615391) bank97947 +97948 POINT(40.50237097831461 73.78255100032601) bank97948 +97949 POINT(40.61417991630547 73.10702193086826) bank97949 +97950 POINT(40.43387985173609 73.1106565861851) bank97950 +97951 POINT(39.952486071205364 74.62700340893404) bank97951 +97952 POINT(40.30160721539591 73.44213642139637) bank97952 +97953 POINT(41.55021554790357 73.26845555942985) bank97953 +97954 POINT(40.64278293083747 73.48464517524856) bank97954 +97955 POINT(40.03141089518888 73.26980930856297) bank97955 +97956 POINT(41.548525049897066 73.45631356860203) bank97956 +97957 POINT(41.34359999653169 73.58307669880787) bank97957 +97958 POINT(39.860820657697666 74.12107613378056) bank97958 +97959 POINT(40.30527010267027 74.87343845184178) bank97959 +97960 POINT(41.21954483752132 74.7923620871001) bank97960 +97961 POINT(41.38420813564535 74.70420656537887) bank97961 +97962 POINT(40.40951429852877 74.61932951771664) bank97962 +97963 POINT(40.130495158796315 74.38491304079909) bank97963 +97964 POINT(40.29902357904522 74.19704780706181) bank97964 +97965 POINT(41.62208177135227 73.76297199601363) bank97965 +97966 POINT(41.47936282367096 74.37005759076978) bank97966 +97967 POINT(41.279818226400515 74.7332035456542) bank97967 +97968 POINT(39.944512717496295 73.03581594236813) bank97968 +97969 POINT(40.12063361220434 74.47936304757164) bank97969 +97970 POINT(41.02816885193481 73.68762916767575) bank97970 +97971 POINT(40.643568624299775 74.94047491192485) bank97971 +97972 POINT(40.573725746268586 74.76013745927034) bank97972 +97973 POINT(41.53212617652502 74.60993338492217) bank97973 +97974 POINT(40.01262014184361 73.79732086378432) bank97974 +97975 POINT(40.89850762258545 74.05199582135195) bank97975 +97976 POINT(40.618228505166734 74.50171049731115) bank97976 +97977 POINT(41.00594725134531 74.03201751481124) bank97977 +97978 POINT(41.692325333259866 74.00025646082955) bank97978 +97979 POINT(41.48679301100364 73.96978092888219) bank97979 +97980 POINT(40.20078258098078 73.05407565849205) bank97980 +97981 POINT(40.579210231201436 73.82886934446638) bank97981 +97982 POINT(40.76993061753044 73.10629728786313) bank97982 +97983 POINT(41.61187078732742 74.8140264234586) bank97983 +97984 POINT(40.53468296176062 74.84187091099471) bank97984 +97985 POINT(39.86279717348459 73.6987157859605) bank97985 +97986 POINT(41.4334598303448 74.65797398889288) bank97986 +97987 POINT(39.79171223140719 74.83247317375627) bank97987 +97988 POINT(40.35304162601009 74.01461066351679) bank97988 +97989 POINT(41.52346442088144 74.59266536464078) bank97989 +97990 POINT(40.515521317580514 74.46394339733293) bank97990 +97991 POINT(39.96145411026249 73.95949602536352) bank97991 +97992 POINT(40.402494183492024 73.91369129018796) bank97992 +97993 POINT(41.69714268605965 74.80054678056864) bank97993 +97994 POINT(41.272454386167 74.52034765628721) bank97994 +97995 POINT(40.550214329604565 73.4763294110271) bank97995 +97996 POINT(39.88594301907095 73.0881495736277) bank97996 +97997 POINT(41.04074289249344 73.97576387089347) bank97997 +97998 POINT(40.36736655768715 73.9682947916297) bank97998 +97999 POINT(41.188508062535384 74.5946630485287) bank97999 +98000 POINT(40.1228414344141 73.18179568174008) bank98000 +98001 POINT(41.235070685585754 73.05921094674281) bank98001 +98002 POINT(40.11922812767536 73.24305558086708) bank98002 +98003 POINT(41.334549465333545 74.87414585926615) bank98003 +98004 POINT(39.76711401301321 73.97604363992423) bank98004 +98005 POINT(41.023937301538496 74.49361150510568) bank98005 +98006 POINT(39.86240197585476 74.39119016510749) bank98006 +98007 POINT(39.76527962715216 73.53574488599398) bank98007 +98008 POINT(40.48510177570826 73.06120866681853) bank98008 +98009 POINT(40.56553242381491 74.2825918846184) bank98009 +98010 POINT(41.42611273576322 73.36428596255575) bank98010 +98011 POINT(40.259125056874375 74.89455935549248) bank98011 +98012 POINT(39.74006442110202 74.34554029312915) bank98012 +98013 POINT(41.46296878840756 73.34012588291228) bank98013 +98014 POINT(40.7366912968516 74.00408356137868) bank98014 +98015 POINT(39.99301830674882 73.99287714697282) bank98015 +98016 POINT(41.34637718948523 73.79386634469829) bank98016 +98017 POINT(41.25484487996397 74.4184848900412) bank98017 +98018 POINT(40.21133016228806 74.04314577534481) bank98018 +98019 POINT(40.581730993415334 74.7390543598207) bank98019 +98020 POINT(40.14277003396067 73.34740485693486) bank98020 +98021 POINT(41.464489682006835 74.38258880248456) bank98021 +98022 POINT(41.13264409730458 74.36448168209826) bank98022 +98023 POINT(41.46355537095868 73.9734376714474) bank98023 +98024 POINT(40.881474616175865 73.27274995743178) bank98024 +98025 POINT(40.2750023221854 74.60444473277572) bank98025 +98026 POINT(41.67394599680989 73.48944178608781) bank98026 +98027 POINT(41.37508247152111 73.52822042817773) bank98027 +98028 POINT(40.904252538520055 74.57234159102524) bank98028 +98029 POINT(40.763246794509385 74.94812071040654) bank98029 +98030 POINT(41.033194059882504 74.77845100832708) bank98030 +98031 POINT(40.49328410808948 73.87117051001331) bank98031 +98032 POINT(40.8261827925353 73.27949589286408) bank98032 +98033 POINT(41.03117882140285 74.47044455623073) bank98033 +98034 POINT(40.63116719470873 74.97220440704845) bank98034 +98035 POINT(41.68896437187362 74.09481627342437) bank98035 +98036 POINT(40.62852141953221 74.70810185331553) bank98036 +98037 POINT(40.22487331155688 73.35739546271199) bank98037 +98038 POINT(40.27871284263262 73.58743904708379) bank98038 +98039 POINT(40.9033008710391 74.53484268744714) bank98039 +98040 POINT(40.66039196092664 74.11059536977545) bank98040 +98041 POINT(39.973651874551116 74.03149595390656) bank98041 +98042 POINT(40.689019892601294 74.24214300915136) bank98042 +98043 POINT(41.22862894978882 73.01069243264014) bank98043 +98044 POINT(41.09216422555305 73.43764690265294) bank98044 +98045 POINT(39.94844432574608 73.7721643722198) bank98045 +98046 POINT(39.75016021767877 74.75555194480658) bank98046 +98047 POINT(40.596555836306365 74.65050629542029) bank98047 +98048 POINT(41.06678467588698 74.8134398783282) bank98048 +98049 POINT(41.50844055659195 73.92851398395075) bank98049 +98050 POINT(40.120688160694336 73.39721921841041) bank98050 +98051 POINT(41.43315771627198 74.45186341688031) bank98051 +98052 POINT(40.84866094544172 74.74550612207389) bank98052 +98053 POINT(41.45563820926881 74.36817157286431) bank98053 +98054 POINT(40.52494655101133 74.5127871146477) bank98054 +98055 POINT(40.553568622525525 73.3236394873986) bank98055 +98056 POINT(40.41355851225633 74.29312874974532) bank98056 +98057 POINT(41.48792193091778 73.52457242349782) bank98057 +98058 POINT(39.97501753165256 74.2833272213118) bank98058 +98059 POINT(40.36279925135004 74.57516118337415) bank98059 +98060 POINT(40.300464231012356 74.10733366218957) bank98060 +98061 POINT(40.378969470676864 73.37802874494216) bank98061 +98062 POINT(39.72394261688582 73.55257406789757) bank98062 +98063 POINT(41.57124536943614 73.27005094404048) bank98063 +98064 POINT(39.948433774589084 74.52222789483362) bank98064 +98065 POINT(40.80328712345477 73.62359433018715) bank98065 +98066 POINT(40.96055661578433 74.52462007326953) bank98066 +98067 POINT(40.757829835155526 74.12333580717507) bank98067 +98068 POINT(41.43831545499361 73.44609447262413) bank98068 +98069 POINT(41.20170009348189 73.98969353683928) bank98069 +98070 POINT(40.0185624514008 73.45029117459679) bank98070 +98071 POINT(40.821236969837585 73.9610373424031) bank98071 +98072 POINT(41.51301196551347 74.01162340231436) bank98072 +98073 POINT(41.33518020901288 74.9282852065395) bank98073 +98074 POINT(40.181895126160946 73.21768016927649) bank98074 +98075 POINT(39.86167675577047 73.27271978398888) bank98075 +98076 POINT(40.09000788365739 73.42979384328301) bank98076 +98077 POINT(39.800446368470304 73.4049449758226) bank98077 +98078 POINT(39.822138193852176 73.29658834939029) bank98078 +98079 POINT(41.58162395118711 74.69064887314903) bank98079 +98080 POINT(41.35887447302594 74.4390999779329) bank98080 +98081 POINT(40.56957191397886 74.64827220052034) bank98081 +98082 POINT(40.59466468652774 74.79258548077051) bank98082 +98083 POINT(40.155515697234215 74.01564487279855) bank98083 +98084 POINT(41.21794669199837 73.6980130169279) bank98084 +98085 POINT(40.09925523633201 73.01947957984142) bank98085 +98086 POINT(39.71746404939333 74.8053128164268) bank98086 +98087 POINT(40.64596112322602 74.85892092895232) bank98087 +98088 POINT(40.93914779080745 73.35031975454504) bank98088 +98089 POINT(41.38141838471421 74.09173392980635) bank98089 +98090 POINT(40.33310388269817 74.64314041679188) bank98090 +98091 POINT(41.64465332965088 74.8184752720096) bank98091 +98092 POINT(40.57689131214113 74.13257256012932) bank98092 +98093 POINT(40.83977864700068 74.64467993245755) bank98093 +98094 POINT(41.37589465836812 74.08598192183378) bank98094 +98095 POINT(40.20720292620241 73.8928194883485) bank98095 +98096 POINT(40.947455683566645 74.83292510800231) bank98096 +98097 POINT(40.754514052193095 74.24245856925684) bank98097 +98098 POINT(40.1192333757658 74.59681169774258) bank98098 +98099 POINT(40.9907733481088 73.5478387444945) bank98099 +98100 POINT(40.62190715522069 74.92928261579142) bank98100 +98101 POINT(41.201935987837366 73.73064357637064) bank98101 +98102 POINT(40.612584460927664 74.44698085375768) bank98102 +98103 POINT(41.08338475987481 74.11627286856402) bank98103 +98104 POINT(41.69677113286901 73.31792887057763) bank98104 +98105 POINT(41.31904517222118 74.4662669169478) bank98105 +98106 POINT(40.27354474461858 74.91597974541781) bank98106 +98107 POINT(41.10881890789533 73.4692159197546) bank98107 +98108 POINT(41.55869230206656 74.12630217748064) bank98108 +98109 POINT(41.09563384462211 73.98128719746747) bank98109 +98110 POINT(40.27089438106534 74.73634611224348) bank98110 +98111 POINT(40.79182135433574 73.22662888617769) bank98111 +98112 POINT(41.39591181438104 74.97260785790894) bank98112 +98113 POINT(40.32303876308823 73.16511577769552) bank98113 +98114 POINT(40.71296987703823 73.52836913335675) bank98114 +98115 POINT(40.46786529893611 74.70654476831251) bank98115 +98116 POINT(39.97427442700683 74.08057897162271) bank98116 +98117 POINT(40.55907635043365 73.19796987828569) bank98117 +98118 POINT(39.74784825662709 74.89422589409662) bank98118 +98119 POINT(41.38154417677881 73.99698916707828) bank98119 +98120 POINT(41.39551085960209 73.32649927153753) bank98120 +98121 POINT(40.82036471144822 74.87302777511509) bank98121 +98122 POINT(39.783947308546736 74.19152825639449) bank98122 +98123 POINT(41.24664535222097 73.9078138285835) bank98123 +98124 POINT(41.21740698754331 74.09639392752533) bank98124 +98125 POINT(40.221058456326375 74.20472638979476) bank98125 +98126 POINT(40.65975480263916 74.49007820794266) bank98126 +98127 POINT(40.62655575692947 73.43111401638984) bank98127 +98128 POINT(39.87355171454104 73.03398787147903) bank98128 +98129 POINT(40.24356574093757 74.31571980896663) bank98129 +98130 POINT(39.96784559415387 73.83130093250092) bank98130 +98131 POINT(40.5578007958501 74.47321028927826) bank98131 +98132 POINT(40.90400905760344 74.3066357548128) bank98132 +98133 POINT(40.39671799040268 74.3519704557611) bank98133 +98134 POINT(41.26494224030109 74.28330573931407) bank98134 +98135 POINT(41.70209636690589 74.46908033980424) bank98135 +98136 POINT(40.98144722815777 74.85317512464795) bank98136 +98137 POINT(39.88681700070532 74.5668188284473) bank98137 +98138 POINT(41.360856559968404 73.21714606079316) bank98138 +98139 POINT(39.77239849889768 74.6214168007545) bank98139 +98140 POINT(41.56498181857584 74.21966254588392) bank98140 +98141 POINT(41.12612342187002 74.9950144903694) bank98141 +98142 POINT(41.53171305455398 73.55514995693173) bank98142 +98143 POINT(39.97372416049726 73.72487583064479) bank98143 +98144 POINT(41.17352983285812 73.86666736864875) bank98144 +98145 POINT(40.671807140987745 74.09594070208107) bank98145 +98146 POINT(41.38869782716052 74.78571282300925) bank98146 +98147 POINT(41.027313135508365 74.52580968678349) bank98147 +98148 POINT(41.52624618657978 74.61633641022999) bank98148 +98149 POINT(41.03473299527503 74.03566225925941) bank98149 +98150 POINT(41.21510260265031 74.34616572031794) bank98150 +98151 POINT(39.878111995721994 73.91306188090047) bank98151 +98152 POINT(40.748228946297615 73.17435519027542) bank98152 +98153 POINT(40.014778586292856 74.8919235349338) bank98153 +98154 POINT(40.2892620698339 73.29719032101802) bank98154 +98155 POINT(40.26299802545633 73.2937200376167) bank98155 +98156 POINT(40.93226749850437 73.64530173164286) bank98156 +98157 POINT(39.79072758468051 74.05834650395579) bank98157 +98158 POINT(41.199900337726696 74.9490491639737) bank98158 +98159 POINT(40.56054438379795 74.88870911578728) bank98159 +98160 POINT(40.677387498945926 74.5886886335604) bank98160 +98161 POINT(41.69553130637329 74.7043286345313) bank98161 +98162 POINT(40.35879447733089 73.68225734424344) bank98162 +98163 POINT(40.28358327872243 74.68493312175644) bank98163 +98164 POINT(40.16339199995265 74.54690824194178) bank98164 +98165 POINT(39.74149147912784 74.74188355118517) bank98165 +98166 POINT(41.140251930396424 73.16134702212834) bank98166 +98167 POINT(40.90038126773992 73.6999241260197) bank98167 +98168 POINT(40.6061977411631 73.38868218301016) bank98168 +98169 POINT(40.22690457459091 73.24060722551657) bank98169 +98170 POINT(40.953822184432575 73.44327289318736) bank98170 +98171 POINT(39.915677300825365 74.985278730001) bank98171 +98172 POINT(40.952525756913474 73.07096433920519) bank98172 +98173 POINT(40.96488010532592 73.1779524032027) bank98173 +98174 POINT(40.10264969963443 74.60700738826831) bank98174 +98175 POINT(40.49827187283972 73.8892289224637) bank98175 +98176 POINT(39.752496042169085 74.5333897423689) bank98176 +98177 POINT(41.47716796369661 74.1530877548397) bank98177 +98178 POINT(41.56615513656786 74.31210580083767) bank98178 +98179 POINT(40.526734857876704 73.92998566794112) bank98179 +98180 POINT(40.00814156969857 74.68703740538452) bank98180 +98181 POINT(40.985062590131605 74.70967814048291) bank98181 +98182 POINT(40.9104474026125 74.63414571510437) bank98182 +98183 POINT(41.60800429201309 73.24237496671955) bank98183 +98184 POINT(40.5458227453445 74.90094858369598) bank98184 +98185 POINT(41.18336770288353 74.01042508463118) bank98185 +98186 POINT(40.10190366571347 73.62941367684812) bank98186 +98187 POINT(40.8813819440852 74.79145501442305) bank98187 +98188 POINT(40.97538599292278 73.58263628482177) bank98188 +98189 POINT(41.0529138254278 73.82206486895369) bank98189 +98190 POINT(41.52600752242766 73.13981502670597) bank98190 +98191 POINT(40.14945372481835 74.50177217964674) bank98191 +98192 POINT(40.38922758459025 74.46099299075756) bank98192 +98193 POINT(41.600213109203814 73.62744435322222) bank98193 +98194 POINT(41.033573089403184 73.80904109626827) bank98194 +98195 POINT(40.46553424494756 74.84835841925302) bank98195 +98196 POINT(40.64497036598533 74.67799683357364) bank98196 +98197 POINT(41.5554697358656 74.4249344320987) bank98197 +98198 POINT(40.52985341926163 74.73315342025292) bank98198 +98199 POINT(40.04603681644574 73.88458050247155) bank98199 +98200 POINT(40.32022748440938 73.36960213507163) bank98200 +98201 POINT(40.86699813488762 74.12116176930579) bank98201 +98202 POINT(41.50202395814709 74.3805810103687) bank98202 +98203 POINT(41.472156165273006 74.00208157268317) bank98203 +98204 POINT(40.615669886959154 74.67940840420442) bank98204 +98205 POINT(40.07477314593836 73.79702947412386) bank98205 +98206 POINT(41.26465318345788 74.89718186592661) bank98206 +98207 POINT(39.97454142727059 74.56321772940731) bank98207 +98208 POINT(41.559854508503136 73.25684572859143) bank98208 +98209 POINT(40.24420483737677 74.46971221162278) bank98209 +98210 POINT(39.975962320772894 74.3222204991676) bank98210 +98211 POINT(40.44344046298059 73.23203992670655) bank98211 +98212 POINT(40.83755048516219 74.92792364427206) bank98212 +98213 POINT(41.551637471812214 73.22015044008893) bank98213 +98214 POINT(41.56610230597143 74.40641091842133) bank98214 +98215 POINT(39.92743535624621 74.91185973167518) bank98215 +98216 POINT(41.411249838818186 73.20364769456941) bank98216 +98217 POINT(39.75100512067051 73.00746042530655) bank98217 +98218 POINT(40.08571527572554 74.022094291585) bank98218 +98219 POINT(40.00968446200747 74.32499599488722) bank98219 +98220 POINT(41.57788210867645 73.984814672128) bank98220 +98221 POINT(41.35488788984536 74.65843165588552) bank98221 +98222 POINT(40.33353079569194 74.74612246438782) bank98222 +98223 POINT(40.22484913167333 73.32414826783219) bank98223 +98224 POINT(40.03923997193141 73.3912689660996) bank98224 +98225 POINT(41.551869967185965 74.67377361117177) bank98225 +98226 POINT(40.20549747340967 74.33687866911349) bank98226 +98227 POINT(40.54256563903835 73.83718685896258) bank98227 +98228 POINT(40.52753923940092 74.6840177496083) bank98228 +98229 POINT(40.19614594380783 73.73216706179235) bank98229 +98230 POINT(41.20529988928594 73.68681237116678) bank98230 +98231 POINT(40.79485453941023 73.42821634617523) bank98231 +98232 POINT(40.309212501420525 73.30398199006736) bank98232 +98233 POINT(40.33881015838056 73.29042828481307) bank98233 +98234 POINT(41.076057002671654 74.73430646300456) bank98234 +98235 POINT(41.239962915152525 73.79624218709469) bank98235 +98236 POINT(40.83331661466904 73.68078822548668) bank98236 +98237 POINT(39.73026123946903 73.56677779534562) bank98237 +98238 POINT(41.1728583760001 73.0512223620161) bank98238 +98239 POINT(40.33925491140628 73.25596958526187) bank98239 +98240 POINT(40.25103137184916 74.30388989069411) bank98240 +98241 POINT(39.84851946957844 73.89735197463833) bank98241 +98242 POINT(40.23120936023091 73.06180566960418) bank98242 +98243 POINT(41.323851324241176 74.14312012598445) bank98243 +98244 POINT(41.0342790291469 74.27356384542185) bank98244 +98245 POINT(40.65258547869947 73.85439525672501) bank98245 +98246 POINT(41.67243400896549 74.95895136995019) bank98246 +98247 POINT(40.008511146205805 73.93194646871538) bank98247 +98248 POINT(39.87077723774063 73.63279637097507) bank98248 +98249 POINT(40.769192630181045 73.21954054019247) bank98249 +98250 POINT(40.53105701369643 73.39515075265365) bank98250 +98251 POINT(41.3346240656644 73.57685840700158) bank98251 +98252 POINT(40.07725778354553 74.18443786639932) bank98252 +98253 POINT(39.76613147719246 73.73272626155187) bank98253 +98254 POINT(39.81038144219477 74.90683906875343) bank98254 +98255 POINT(40.60358088886924 73.59777290763735) bank98255 +98256 POINT(40.65130052436352 74.44000800501429) bank98256 +98257 POINT(41.11741588549175 74.91217699004893) bank98257 +98258 POINT(40.84174638645028 73.77383520679132) bank98258 +98259 POINT(40.92446973846171 73.44854656147666) bank98259 +98260 POINT(40.65845740890721 74.52470689109259) bank98260 +98261 POINT(40.40486052742383 73.78635558013266) bank98261 +98262 POINT(39.92079677859654 74.73519581318232) bank98262 +98263 POINT(39.989503446481024 73.06094924288406) bank98263 +98264 POINT(40.83784066386737 73.07775094850062) bank98264 +98265 POINT(39.94226220473852 73.67178471435321) bank98265 +98266 POINT(40.35224188701132 73.9323286113725) bank98266 +98267 POINT(41.51604578057043 73.0285309315166) bank98267 +98268 POINT(40.29585405350417 74.45505283975818) bank98268 +98269 POINT(41.155215415160065 73.46781451521677) bank98269 +98270 POINT(41.47614113831097 74.02669725682401) bank98270 +98271 POINT(41.555103877030305 74.42910613387262) bank98271 +98272 POINT(41.45899283699956 74.22115356954664) bank98272 +98273 POINT(40.52961850488811 74.18215723342703) bank98273 +98274 POINT(40.8140708991078 73.53355919242067) bank98274 +98275 POINT(41.287906276392725 74.14010424335646) bank98275 +98276 POINT(40.593736165409396 73.4107390980093) bank98276 +98277 POINT(41.69452760460353 73.86591770660682) bank98277 +98278 POINT(40.423707903064326 73.79428167557988) bank98278 +98279 POINT(40.62520296579393 74.63583181266321) bank98279 +98280 POINT(40.20355844717267 74.40177528628357) bank98280 +98281 POINT(40.24754772696658 74.3750229556956) bank98281 +98282 POINT(40.106144258704525 74.12437247817283) bank98282 +98283 POINT(40.66937858963744 74.80749540107773) bank98283 +98284 POINT(41.05739945370978 73.68532159106907) bank98284 +98285 POINT(39.91377615702883 73.91949348001866) bank98285 +98286 POINT(41.418182965427036 73.04062545229877) bank98286 +98287 POINT(39.88804900709071 73.63156767239349) bank98287 +98288 POINT(40.96476097679411 74.85389415244693) bank98288 +98289 POINT(41.119138896128014 74.70404115314061) bank98289 +98290 POINT(40.206257503590415 73.91425356802144) bank98290 +98291 POINT(41.54446778684183 74.22673920581774) bank98291 +98292 POINT(41.26965317708681 74.26613999596523) bank98292 +98293 POINT(39.89898387175503 74.15477283701826) bank98293 +98294 POINT(40.26398241112919 74.22134419934078) bank98294 +98295 POINT(41.44836324249996 74.20796822599645) bank98295 +98296 POINT(40.23697342496824 73.6136782763522) bank98296 +98297 POINT(39.736989718166065 73.40719969686873) bank98297 +98298 POINT(40.5998357800519 73.7759670453724) bank98298 +98299 POINT(41.480690137144435 73.9616864227218) bank98299 +98300 POINT(40.02402811699907 73.55879057453768) bank98300 +98301 POINT(39.872035614325256 74.73017474206169) bank98301 +98302 POINT(39.87728334307445 73.73876662475995) bank98302 +98303 POINT(41.45622192426554 73.22636675317241) bank98303 +98304 POINT(40.402653140032285 73.50129147567426) bank98304 +98305 POINT(40.50476072500933 74.38164563506726) bank98305 +98306 POINT(39.73156300219468 73.57793481762009) bank98306 +98307 POINT(41.53008112358576 74.86055298722796) bank98307 +98308 POINT(40.24899053714047 74.92502807760651) bank98308 +98309 POINT(40.02559357575719 74.9516334680173) bank98309 +98310 POINT(41.156308832546394 73.57147950775321) bank98310 +98311 POINT(40.49177679396849 73.82353336728517) bank98311 +98312 POINT(41.21608129554873 74.72674416840306) bank98312 +98313 POINT(40.192731621636014 74.26490256370982) bank98313 +98314 POINT(41.551945553730285 74.98186906775129) bank98314 +98315 POINT(41.207231225477265 73.99614621384865) bank98315 +98316 POINT(41.231249488056456 74.55505719976875) bank98316 +98317 POINT(41.68353095742712 74.46735003057671) bank98317 +98318 POINT(40.11632489828899 74.01308714040813) bank98318 +98319 POINT(41.382994427535955 73.36767839399056) bank98319 +98320 POINT(41.101011626971136 74.59337431887494) bank98320 +98321 POINT(40.41363711166672 73.55852122676389) bank98321 +98322 POINT(41.25889692265262 74.04723706983575) bank98322 +98323 POINT(40.082080854170734 73.37563934979352) bank98323 +98324 POINT(40.970441168515265 74.86431875404483) bank98324 +98325 POINT(41.490492682702076 73.21359806560199) bank98325 +98326 POINT(40.600257243113084 74.61284497045622) bank98326 +98327 POINT(41.21680758708004 74.93413227102964) bank98327 +98328 POINT(40.309743001810375 73.73700544916508) bank98328 +98329 POINT(40.89859627495427 73.52871325372836) bank98329 +98330 POINT(39.930735454583164 73.64248673924641) bank98330 +98331 POINT(39.879173753583366 74.00132196125213) bank98331 +98332 POINT(40.35295309700338 74.97576093788105) bank98332 +98333 POINT(41.19717619084329 73.69958484314283) bank98333 +98334 POINT(40.32671128047393 74.5942139668657) bank98334 +98335 POINT(40.70631261974354 74.02889539044226) bank98335 +98336 POINT(40.51511656495546 74.00077637364) bank98336 +98337 POINT(40.354797861998584 73.44417524673385) bank98337 +98338 POINT(41.13004513694164 74.73580514646514) bank98338 +98339 POINT(41.52565864130214 74.23313350000261) bank98339 +98340 POINT(41.46170351020093 74.44806702854916) bank98340 +98341 POINT(40.572069858567275 74.32982944347538) bank98341 +98342 POINT(40.169185916090264 74.84814314503423) bank98342 +98343 POINT(41.65354307683715 73.33046374356239) bank98343 +98344 POINT(40.177648367600725 73.95479998293989) bank98344 +98345 POINT(41.47141040776575 74.58936789735887) bank98345 +98346 POINT(41.30485639657998 73.11740288372081) bank98346 +98347 POINT(40.89584482691899 74.46043514344116) bank98347 +98348 POINT(41.69813463244035 74.3993705356779) bank98348 +98349 POINT(40.02391462033738 74.5757989753462) bank98349 +98350 POINT(40.12749724681187 73.55357533529141) bank98350 +98351 POINT(40.08169887002451 73.26003547432818) bank98351 +98352 POINT(41.51655490805398 73.22221190448911) bank98352 +98353 POINT(40.516954570015706 73.24661354087486) bank98353 +98354 POINT(40.99739139994801 74.33707836301562) bank98354 +98355 POINT(40.234235070021576 74.27210068094554) bank98355 +98356 POINT(41.6471056214788 73.3111573052345) bank98356 +98357 POINT(39.9733336061787 74.7554579793406) bank98357 +98358 POINT(39.77222168226282 74.23449865293549) bank98358 +98359 POINT(41.649615783923366 74.49267366403409) bank98359 +98360 POINT(40.429062752079275 74.14278969042073) bank98360 +98361 POINT(40.17863801979801 73.44993700306821) bank98361 +98362 POINT(41.4584504185808 74.55081218886295) bank98362 +98363 POINT(39.99167869563855 74.3701568082346) bank98363 +98364 POINT(40.26474680367052 73.3418104915119) bank98364 +98365 POINT(40.87001690374874 73.46276749810391) bank98365 +98366 POINT(41.48103536040303 74.22658762619704) bank98366 +98367 POINT(40.07011766183662 73.39868674753365) bank98367 +98368 POINT(40.35196551060686 73.04855006233889) bank98368 +98369 POINT(40.44196793759238 74.2835622459627) bank98369 +98370 POINT(40.643452633794155 74.04834976147384) bank98370 +98371 POINT(40.226400316652374 73.88923346658838) bank98371 +98372 POINT(41.4716626304294 74.7536332499583) bank98372 +98373 POINT(40.23152714303828 73.51332782336348) bank98373 +98374 POINT(40.964011552637466 74.69381800846648) bank98374 +98375 POINT(40.544225727442 73.67531147819876) bank98375 +98376 POINT(41.43601745630571 74.34159237167528) bank98376 +98377 POINT(40.95841058754425 73.40973218105535) bank98377 +98378 POINT(39.904243448541756 74.20315364748106) bank98378 +98379 POINT(39.972176826697364 73.42194702479283) bank98379 +98380 POINT(40.39476102256379 74.85353599403679) bank98380 +98381 POINT(40.07556942360753 74.59749831466132) bank98381 +98382 POINT(41.29807040282482 73.70063050971785) bank98382 +98383 POINT(41.26927841071305 73.40864272654665) bank98383 +98384 POINT(40.69307146957369 73.35524039688407) bank98384 +98385 POINT(40.90816909663491 73.88513098383012) bank98385 +98386 POINT(41.32656925962041 73.83362154600542) bank98386 +98387 POINT(41.41629776907051 74.95524115577527) bank98387 +98388 POINT(39.947600192691205 74.24231096585302) bank98388 +98389 POINT(39.7784376218001 74.81699301390006) bank98389 +98390 POINT(40.25151512320502 74.13999242316194) bank98390 +98391 POINT(39.81105780020042 73.88403320517068) bank98391 +98392 POINT(41.62857665393054 73.31651256244773) bank98392 +98393 POINT(40.730146637842324 74.46814829238102) bank98393 +98394 POINT(40.37977933156059 73.66821498714837) bank98394 +98395 POINT(40.194231452296954 74.71711390119418) bank98395 +98396 POINT(40.01134870464145 74.15530861130235) bank98396 +98397 POINT(41.39734935472016 74.94536561211424) bank98397 +98398 POINT(40.57210523839519 74.63350554823714) bank98398 +98399 POINT(40.42103635103485 73.14135115223684) bank98399 +98400 POINT(41.01534514658671 74.5665975383201) bank98400 +98401 POINT(40.75949700905733 74.27681300411356) bank98401 +98402 POINT(40.52656108721278 74.08325807175487) bank98402 +98403 POINT(41.33730175526242 74.8823512204763) bank98403 +98404 POINT(39.919512452235935 73.36021251368429) bank98404 +98405 POINT(40.61675428452094 73.34147430780769) bank98405 +98406 POINT(40.82982712022211 74.72550737301458) bank98406 +98407 POINT(40.58520880561276 73.128894132462) bank98407 +98408 POINT(41.487934569691014 74.22490422897991) bank98408 +98409 POINT(40.88114422605724 74.50587443368943) bank98409 +98410 POINT(40.8668724421214 73.09460244202846) bank98410 +98411 POINT(40.47219729474755 73.04073805338565) bank98411 +98412 POINT(41.171928645268174 74.53621936057675) bank98412 +98413 POINT(41.63792057093293 74.55921501023028) bank98413 +98414 POINT(39.719483708708324 74.13937405336554) bank98414 +98415 POINT(40.18415663549543 73.34687249301945) bank98415 +98416 POINT(40.953804748767816 73.78989750251993) bank98416 +98417 POINT(40.796886175745065 74.37909091898663) bank98417 +98418 POINT(40.06584094420554 73.31940322749928) bank98418 +98419 POINT(40.90313253625732 73.94322543601659) bank98419 +98420 POINT(40.91094603782488 74.04580406380177) bank98420 +98421 POINT(41.42525462194279 73.23999729327967) bank98421 +98422 POINT(41.26091921539782 73.92668727409148) bank98422 +98423 POINT(41.60986224837538 73.67646493855221) bank98423 +98424 POINT(39.74312986053514 74.90989851344476) bank98424 +98425 POINT(40.4278223973795 73.29943538123877) bank98425 +98426 POINT(40.64434371998151 74.35459848564643) bank98426 +98427 POINT(40.821753347465894 73.81211565648252) bank98427 +98428 POINT(41.44547017661712 74.95409523710039) bank98428 +98429 POINT(39.796304572301224 74.73468519843642) bank98429 +98430 POINT(41.67380797738545 73.55959011741085) bank98430 +98431 POINT(40.08890938072176 73.34193411398199) bank98431 +98432 POINT(41.16226288583868 73.33479494970386) bank98432 +98433 POINT(40.908079089153695 74.53128919678728) bank98433 +98434 POINT(40.98588789758407 73.53674884720161) bank98434 +98435 POINT(41.52348026896775 73.11224967385353) bank98435 +98436 POINT(40.891610561820684 73.3151421519548) bank98436 +98437 POINT(40.280897399183296 73.03729497818453) bank98437 +98438 POINT(41.356857588867605 74.576374099074) bank98438 +98439 POINT(40.1297883003741 74.9298026574875) bank98439 +98440 POINT(39.98883600279792 74.51906350570025) bank98440 +98441 POINT(40.913145685932975 74.93887169580853) bank98441 +98442 POINT(40.39307671936235 73.35404950647147) bank98442 +98443 POINT(40.79789295394305 74.51821081981917) bank98443 +98444 POINT(39.757809854995784 73.24705285524742) bank98444 +98445 POINT(40.573877114134426 74.67785986970848) bank98445 +98446 POINT(40.86266741876409 74.2788002572505) bank98446 +98447 POINT(41.51849564963719 73.69455247960637) bank98447 +98448 POINT(40.850856811897856 74.91388374782582) bank98448 +98449 POINT(40.88674415262859 74.42283341931545) bank98449 +98450 POINT(40.98180504137088 74.6229398065788) bank98450 +98451 POINT(40.245075494402094 73.94144603013423) bank98451 +98452 POINT(41.63530421658028 74.30307025046454) bank98452 +98453 POINT(41.50940779095688 74.411793839459) bank98453 +98454 POINT(40.58454618252598 73.41801735355823) bank98454 +98455 POINT(40.11540315075831 73.79145745675075) bank98455 +98456 POINT(39.942214886342036 74.8879851157978) bank98456 +98457 POINT(40.18576322631158 73.08806402481723) bank98457 +98458 POINT(41.384631561512435 73.45767368588015) bank98458 +98459 POINT(41.70247202824688 73.2736380280553) bank98459 +98460 POINT(40.247605749228526 73.56152318780873) bank98460 +98461 POINT(41.575122543522674 73.26804702197174) bank98461 +98462 POINT(39.743558443219975 74.50235356846356) bank98462 +98463 POINT(40.708664018522235 73.0653334702438) bank98463 +98464 POINT(40.103030336436156 74.44233716347938) bank98464 +98465 POINT(40.3777037837018 74.946263226552) bank98465 +98466 POINT(41.37786266591462 73.91334567342507) bank98466 +98467 POINT(41.07957717778936 73.184762330548) bank98467 +98468 POINT(40.28626554531472 74.61258064853828) bank98468 +98469 POINT(41.56296690038449 73.18485427770298) bank98469 +98470 POINT(40.65047639905423 73.77009278762526) bank98470 +98471 POINT(41.47094386942548 74.13715445740193) bank98471 +98472 POINT(41.059007751717324 74.22577440397828) bank98472 +98473 POINT(40.999556258161874 73.9447770000474) bank98473 +98474 POINT(40.50013973762201 73.70761558920024) bank98474 +98475 POINT(41.59195812117659 74.94031900448881) bank98475 +98476 POINT(40.39515149791342 74.53194465789619) bank98476 +98477 POINT(41.548323254078205 73.48660078311873) bank98477 +98478 POINT(41.5808085433855 74.2221729331028) bank98478 +98479 POINT(39.89841353420482 74.98115568146517) bank98479 +98480 POINT(40.11395100532684 74.45115964606394) bank98480 +98481 POINT(39.93505100994464 73.31968439661973) bank98481 +98482 POINT(40.60967002145395 74.884016264057) bank98482 +98483 POINT(40.61183373666411 73.68362676202207) bank98483 +98484 POINT(41.665584659181974 73.39595766642637) bank98484 +98485 POINT(39.86261823964244 74.74632076433505) bank98485 +98486 POINT(41.703897510434174 74.06837371680933) bank98486 +98487 POINT(40.52439780715974 74.12398193200667) bank98487 +98488 POINT(41.516160674661 73.68690205034336) bank98488 +98489 POINT(39.79649576022918 74.4760242002713) bank98489 +98490 POINT(40.90930989631737 74.4605339977776) bank98490 +98491 POINT(41.43981642746386 74.50261804227311) bank98491 +98492 POINT(41.03487775729444 74.13686836419502) bank98492 +98493 POINT(40.28550083547627 73.50073104933844) bank98493 +98494 POINT(41.30030167673829 74.13892731444957) bank98494 +98495 POINT(40.67246964315775 73.19717147496871) bank98495 +98496 POINT(40.46396123311789 73.65774030009759) bank98496 +98497 POINT(41.201675265889726 73.86172895042257) bank98497 +98498 POINT(41.53961477055466 74.27506116270862) bank98498 +98499 POINT(39.98687759529152 74.55879955156985) bank98499 +98500 POINT(39.92483013234374 74.26580640322739) bank98500 +98501 POINT(41.229440784575445 73.94188864684165) bank98501 +98502 POINT(40.105513125253175 73.83090583385041) bank98502 +98503 POINT(39.9306170671774 74.20727584001952) bank98503 +98504 POINT(40.39123072438649 74.74838290607077) bank98504 +98505 POINT(40.25228189357772 74.31380609837325) bank98505 +98506 POINT(39.78994465859518 73.5713982763637) bank98506 +98507 POINT(41.2181304034504 74.73152232651209) bank98507 +98508 POINT(41.63486501478152 73.04967700665598) bank98508 +98509 POINT(41.530981071632844 73.47825206493256) bank98509 +98510 POINT(39.757683099413164 74.00388827377432) bank98510 +98511 POINT(40.009616122347964 73.6951851588598) bank98511 +98512 POINT(41.216055304423534 73.6502511928258) bank98512 +98513 POINT(40.219362830339314 73.17152469228496) bank98513 +98514 POINT(40.21003987350777 73.25482972604583) bank98514 +98515 POINT(40.192774269395215 74.94062070443566) bank98515 +98516 POINT(40.36770642379845 74.49391436723336) bank98516 +98517 POINT(41.12081517284544 74.28571357636467) bank98517 +98518 POINT(39.83432913226734 74.96389479279601) bank98518 +98519 POINT(40.53663592375556 73.60609202418028) bank98519 +98520 POINT(39.90327877069195 74.52528684445224) bank98520 +98521 POINT(40.37192811953013 73.78270152126692) bank98521 +98522 POINT(40.57464954585237 74.43013906649094) bank98522 +98523 POINT(41.170459666168696 73.4228518525825) bank98523 +98524 POINT(39.91905006740635 74.58572782761472) bank98524 +98525 POINT(41.52295721175484 74.96079568289169) bank98525 +98526 POINT(40.466103123814136 74.89089887246865) bank98526 +98527 POINT(41.027318796817184 74.68273701912287) bank98527 +98528 POINT(41.41931513867164 73.28144405057125) bank98528 +98529 POINT(40.06277041385635 73.37153649732966) bank98529 +98530 POINT(40.749639603709205 73.133317973161) bank98530 +98531 POINT(41.62990843837434 73.58724270071772) bank98531 +98532 POINT(40.85413461148677 73.93308800266979) bank98532 +98533 POINT(41.674754913916296 74.57595663165755) bank98533 +98534 POINT(40.64130939236215 73.00733214053797) bank98534 +98535 POINT(41.12584591041921 73.40028771666798) bank98535 +98536 POINT(39.81778365520899 73.43225306502141) bank98536 +98537 POINT(39.953352319827815 74.29086756612777) bank98537 +98538 POINT(39.87802241393378 74.07899298340517) bank98538 +98539 POINT(41.36600270696807 73.51423212913369) bank98539 +98540 POINT(40.28088052380509 74.08460283508194) bank98540 +98541 POINT(41.02901422614008 74.80976934785687) bank98541 +98542 POINT(40.448910883049884 73.42400430582788) bank98542 +98543 POINT(40.015043753800256 74.79298147676374) bank98543 +98544 POINT(40.85974861505679 73.80504953154642) bank98544 +98545 POINT(41.595308315689124 73.40841408128352) bank98545 +98546 POINT(41.256771277534 74.54347332922838) bank98546 +98547 POINT(41.68429667891893 73.19549738357854) bank98547 +98548 POINT(41.26399992843743 73.03060401265333) bank98548 +98549 POINT(41.691073514148066 73.52617643845963) bank98549 +98550 POINT(39.89601241831831 74.28738714591898) bank98550 +98551 POINT(41.316844225458084 73.81141650276072) bank98551 +98552 POINT(39.96692717206375 73.62817813122496) bank98552 +98553 POINT(41.11924173820552 74.48564576488536) bank98553 +98554 POINT(40.57600585615736 74.54175281310643) bank98554 +98555 POINT(40.59276301386684 73.50717655188484) bank98555 +98556 POINT(41.66840916046325 73.87189830397733) bank98556 +98557 POINT(41.14881310146243 74.08423414401202) bank98557 +98558 POINT(40.05582079774409 74.08618070178495) bank98558 +98559 POINT(39.73844695344367 74.55190350908666) bank98559 +98560 POINT(40.59488023272165 74.87011684606233) bank98560 +98561 POINT(41.41614668323083 73.94479877108459) bank98561 +98562 POINT(40.508846396868535 73.24198804431062) bank98562 +98563 POINT(40.68062866821127 74.96436566907923) bank98563 +98564 POINT(40.6522235016008 74.80028433278505) bank98564 +98565 POINT(40.315375871211586 74.96224224734952) bank98565 +98566 POINT(40.13644164428453 73.86323456598188) bank98566 +98567 POINT(41.604892479419675 73.09487013017657) bank98567 +98568 POINT(41.12138940030699 74.34301136690038) bank98568 +98569 POINT(39.94356338937861 74.71980035527882) bank98569 +98570 POINT(41.00242528588366 74.29568865492867) bank98570 +98571 POINT(41.567220317872206 73.76385529442689) bank98571 +98572 POINT(40.841041043646854 74.25793140339837) bank98572 +98573 POINT(40.043946962585395 73.3086732961104) bank98573 +98574 POINT(41.56734855356135 74.66625687860467) bank98574 +98575 POINT(41.586617670372256 73.93088559781746) bank98575 +98576 POINT(41.05499396367019 73.72766412309714) bank98576 +98577 POINT(41.145379211805874 74.16490940269269) bank98577 +98578 POINT(39.8877110289766 74.70294083123342) bank98578 +98579 POINT(40.9382486531567 74.82142130128784) bank98579 +98580 POINT(39.99969927919345 74.21732185173488) bank98580 +98581 POINT(40.12088480134986 73.49480136859303) bank98581 +98582 POINT(41.706023538554874 74.96448515598952) bank98582 +98583 POINT(39.89528623477357 74.68443391160308) bank98583 +98584 POINT(41.060080500813505 73.21145211773475) bank98584 +98585 POINT(40.71179304002239 74.27979846136036) bank98585 +98586 POINT(39.71858025903346 74.17099845763546) bank98586 +98587 POINT(40.99794616085328 73.3825027866594) bank98587 +98588 POINT(40.490267430831466 73.62461250666782) bank98588 +98589 POINT(41.29475132511267 74.41989869207598) bank98589 +98590 POINT(41.131880884972894 74.02531917776298) bank98590 +98591 POINT(40.75360020944435 73.28384818117851) bank98591 +98592 POINT(41.657063488849566 73.69601374790408) bank98592 +98593 POINT(40.204026540545314 74.12669958015705) bank98593 +98594 POINT(40.644477327496496 73.90051250126466) bank98594 +98595 POINT(39.85463960569088 73.53405437939197) bank98595 +98596 POINT(41.076847977476696 73.85278337636042) bank98596 +98597 POINT(41.04732101612855 73.41752379634043) bank98597 +98598 POINT(39.81126711641167 73.8166567997844) bank98598 +98599 POINT(40.17891318233553 73.72604499252512) bank98599 +98600 POINT(40.29552152916321 73.8548169058734) bank98600 +98601 POINT(39.88212556969173 73.23416844885115) bank98601 +98602 POINT(40.28230873342041 74.47488641822103) bank98602 +98603 POINT(39.79525729026343 74.53511534062173) bank98603 +98604 POINT(41.20399299147268 74.86289392173497) bank98604 +98605 POINT(40.365204267305934 73.54834088276216) bank98605 +98606 POINT(40.860300837725916 74.3845865475001) bank98606 +98607 POINT(39.94520789663675 74.71591707939919) bank98607 +98608 POINT(41.40295792136294 73.0112904655997) bank98608 +98609 POINT(41.541048176572374 74.9877188603429) bank98609 +98610 POINT(40.33003921991032 74.57260932540761) bank98610 +98611 POINT(40.861482681291164 74.12746338213276) bank98611 +98612 POINT(41.30964442604641 73.22399475841533) bank98612 +98613 POINT(40.41374065886921 74.25135719857617) bank98613 +98614 POINT(41.689213115928894 73.48350842955571) bank98614 +98615 POINT(40.057353946392645 74.1759530806177) bank98615 +98616 POINT(40.49307111222654 74.35433435214784) bank98616 +98617 POINT(41.125205965067934 73.1609279783429) bank98617 +98618 POINT(39.94170717004178 74.59434697753319) bank98618 +98619 POINT(41.578190673849775 73.44194706326643) bank98619 +98620 POINT(41.0082139774705 74.81969889356215) bank98620 +98621 POINT(41.386494451096254 73.65537451374963) bank98621 +98622 POINT(39.866093157133264 73.11457784465732) bank98622 +98623 POINT(40.582182220191726 74.09320030537732) bank98623 +98624 POINT(39.872408568702475 74.61132342341685) bank98624 +98625 POINT(41.41896735931256 74.63466060401187) bank98625 +98626 POINT(40.07599803839388 74.18612376570586) bank98626 +98627 POINT(41.0086382655529 73.50412941451384) bank98627 +98628 POINT(40.03955268374184 74.51392870593585) bank98628 +98629 POINT(40.0394213614467 74.58187870798251) bank98629 +98630 POINT(41.558159363925476 73.73163465210558) bank98630 +98631 POINT(40.773847281408614 74.0956482710548) bank98631 +98632 POINT(39.808197929214394 74.92049259118902) bank98632 +98633 POINT(39.797052215135295 74.49529795212972) bank98633 +98634 POINT(39.733241868152916 74.7711084237092) bank98634 +98635 POINT(40.40540325136115 73.5923115342415) bank98635 +98636 POINT(40.198460505126285 73.33628408113861) bank98636 +98637 POINT(40.84049893096596 73.20675886512225) bank98637 +98638 POINT(41.0524637851907 73.53497009722317) bank98638 +98639 POINT(39.99945381530877 73.44062721319648) bank98639 +98640 POINT(40.573108630370555 73.10467361952385) bank98640 +98641 POINT(39.91719047543925 73.30563476708294) bank98641 +98642 POINT(40.47394043941259 74.26628736343497) bank98642 +98643 POINT(41.06987723926358 74.42180444971324) bank98643 +98644 POINT(40.70597757794921 73.95949856941417) bank98644 +98645 POINT(41.08852608813573 73.61836110294414) bank98645 +98646 POINT(40.581938170754434 74.22252556703344) bank98646 +98647 POINT(40.73254750879635 73.33668567509211) bank98647 +98648 POINT(39.85915547922084 73.78304434164265) bank98648 +98649 POINT(41.22571335935542 74.2482075936805) bank98649 +98650 POINT(40.78884579943463 73.31493950352741) bank98650 +98651 POINT(40.984146117323554 73.8109393295949) bank98651 +98652 POINT(39.85123736910397 74.93607781308938) bank98652 +98653 POINT(41.45516097378558 73.78885105397478) bank98653 +98654 POINT(40.0307094273597 73.35074550795258) bank98654 +98655 POINT(39.821200617709785 73.9722377132104) bank98655 +98656 POINT(40.83143040265256 74.89947210908755) bank98656 +98657 POINT(40.81941288611526 74.59671680320776) bank98657 +98658 POINT(40.17170114723704 74.62935999583826) bank98658 +98659 POINT(41.70700482253365 73.77745948238825) bank98659 +98660 POINT(41.045106062586875 74.38960240678202) bank98660 +98661 POINT(41.11414001500524 73.51262066234732) bank98661 +98662 POINT(40.29537383853581 73.3537061446699) bank98662 +98663 POINT(40.92953934441904 74.45319581800733) bank98663 +98664 POINT(40.91167767863661 73.08753755787112) bank98664 +98665 POINT(41.56050529598809 73.18846953409195) bank98665 +98666 POINT(40.76391880749172 74.2430231990628) bank98666 +98667 POINT(41.534605267319876 74.79304793292056) bank98667 +98668 POINT(41.70946847476863 73.10109227642009) bank98668 +98669 POINT(40.90690506443513 73.25197494746038) bank98669 +98670 POINT(40.693071020414465 73.76825562453672) bank98670 +98671 POINT(39.77447247792301 74.16986958589445) bank98671 +98672 POINT(41.34949121186774 74.34994664559842) bank98672 +98673 POINT(41.553682979747826 74.28944415748055) bank98673 +98674 POINT(41.00695585316298 74.10232489875563) bank98674 +98675 POINT(41.16847584288975 73.40842241675) bank98675 +98676 POINT(41.7112530879305 73.39552314445643) bank98676 +98677 POINT(40.23941907081556 73.7298376770602) bank98677 +98678 POINT(39.96038805517151 73.90962767685609) bank98678 +98679 POINT(40.35857288654637 74.61539882836784) bank98679 +98680 POINT(41.578696707749174 74.53418410189171) bank98680 +98681 POINT(40.865308658008296 73.78391438444834) bank98681 +98682 POINT(41.50963788992172 73.56892172947668) bank98682 +98683 POINT(40.410395396607086 74.82882810156138) bank98683 +98684 POINT(40.40851421600822 73.55020120949531) bank98684 +98685 POINT(40.63022565689162 74.5352766815072) bank98685 +98686 POINT(40.59637503944222 73.95489419793131) bank98686 +98687 POINT(39.75562673571879 74.04779642815207) bank98687 +98688 POINT(40.37872512770265 74.39963358733756) bank98688 +98689 POINT(41.025737814818775 73.55658971254572) bank98689 +98690 POINT(41.35280229404804 74.66534098397537) bank98690 +98691 POINT(40.06812236273808 74.17423720646761) bank98691 +98692 POINT(41.53999725329801 73.02525473916185) bank98692 +98693 POINT(40.131159472766356 74.49527288440021) bank98693 +98694 POINT(40.59025157161088 73.77020977409447) bank98694 +98695 POINT(41.52033605401143 74.29341704727267) bank98695 +98696 POINT(40.07578978935253 74.85299137075468) bank98696 +98697 POINT(41.30218503573529 74.46767771241807) bank98697 +98698 POINT(39.75033298281539 74.64508220335006) bank98698 +98699 POINT(39.97605354404931 73.44760683574702) bank98699 +98700 POINT(40.315600661444385 74.79581782617105) bank98700 +98701 POINT(41.686948573867696 74.63928086751008) bank98701 +98702 POINT(39.771289966205835 73.2923228179909) bank98702 +98703 POINT(40.20386706727548 73.14628306064452) bank98703 +98704 POINT(40.59308834115351 73.88149037969418) bank98704 +98705 POINT(41.184467738578775 73.30529979123816) bank98705 +98706 POINT(40.0157649301234 73.24306639871533) bank98706 +98707 POINT(39.911944082566016 73.62634975140377) bank98707 +98708 POINT(40.90761399374677 74.82406766587717) bank98708 +98709 POINT(41.6609951209111 74.98163819558796) bank98709 +98710 POINT(41.537110509038705 73.12889253286104) bank98710 +98711 POINT(40.27172968954194 74.15998568731698) bank98711 +98712 POINT(41.0918060309181 73.31516816385103) bank98712 +98713 POINT(39.86356084255774 73.48264015419463) bank98713 +98714 POINT(41.498874096164656 74.15967619585334) bank98714 +98715 POINT(40.32062789034963 73.75658278030824) bank98715 +98716 POINT(40.75382343912251 73.48602776482188) bank98716 +98717 POINT(40.85288750153985 74.30975133776639) bank98717 +98718 POINT(41.56071793984357 73.6715391343748) bank98718 +98719 POINT(41.13211680171938 74.87230871023618) bank98719 +98720 POINT(40.419906259841845 74.2226135979357) bank98720 +98721 POINT(40.58488736516013 74.7012349702607) bank98721 +98722 POINT(39.82702738720112 74.01693818032925) bank98722 +98723 POINT(41.43085378027287 73.2536907455229) bank98723 +98724 POINT(40.85133885127862 73.27221970719539) bank98724 +98725 POINT(39.781504085443814 74.70134945624628) bank98725 +98726 POINT(41.054816402333955 74.10860048389308) bank98726 +98727 POINT(40.95791431353878 73.31013492230072) bank98727 +98728 POINT(40.240747927030085 74.8069149317755) bank98728 +98729 POINT(41.1278024936172 74.24394703714458) bank98729 +98730 POINT(41.40945559296938 74.44415064247605) bank98730 +98731 POINT(41.46248111599592 74.88654123057024) bank98731 +98732 POINT(39.872806588045265 73.13851756173827) bank98732 +98733 POINT(41.61827392335518 73.33257587817258) bank98733 +98734 POINT(41.105809158881236 74.92706255665807) bank98734 +98735 POINT(40.50084198452105 74.98072752875723) bank98735 +98736 POINT(40.747307599808565 74.23282619813952) bank98736 +98737 POINT(40.93622316743515 74.85465075147445) bank98737 +98738 POINT(40.303932272579516 74.51825186682353) bank98738 +98739 POINT(41.42432705189845 73.03342855634351) bank98739 +98740 POINT(40.78618783363866 74.32288955309996) bank98740 +98741 POINT(41.472373860838566 73.3906900597836) bank98741 +98742 POINT(41.61617747053269 74.0511588709297) bank98742 +98743 POINT(40.43980450004214 73.54178847448449) bank98743 +98744 POINT(40.05339781069844 74.41210516560508) bank98744 +98745 POINT(40.58157570832966 74.31951988171002) bank98745 +98746 POINT(40.225968576125155 73.28826030909983) bank98746 +98747 POINT(41.02405493654653 73.4642171058828) bank98747 +98748 POINT(39.88746963269247 74.0631401609192) bank98748 +98749 POINT(41.623589118059776 73.75955846565333) bank98749 +98750 POINT(41.29499520434814 74.02848422610282) bank98750 +98751 POINT(41.67087020977292 74.31135489998077) bank98751 +98752 POINT(41.04304546156449 74.32544558044435) bank98752 +98753 POINT(40.61080429700006 73.11009080349703) bank98753 +98754 POINT(40.67348425692939 73.30863334690088) bank98754 +98755 POINT(41.2387106735215 73.2720419328563) bank98755 +98756 POINT(41.06639058336102 73.28330446026786) bank98756 +98757 POINT(41.35219452465854 73.64833005850443) bank98757 +98758 POINT(40.978708134086844 73.28376219006336) bank98758 +98759 POINT(40.080127254732034 73.6188194336541) bank98759 +98760 POINT(40.77558006827468 73.43021280351145) bank98760 +98761 POINT(40.640116715213175 73.39886670110322) bank98761 +98762 POINT(40.94661664748478 74.59634859627853) bank98762 +98763 POINT(41.41454608344004 74.04106377852294) bank98763 +98764 POINT(39.88224948350139 73.639049515862) bank98764 +98765 POINT(41.52664000961647 74.37971628598713) bank98765 +98766 POINT(40.10394611836945 73.50934126696299) bank98766 +98767 POINT(41.42237942932057 74.82795151261621) bank98767 +98768 POINT(39.88689255092261 73.14958681420703) bank98768 +98769 POINT(41.64534182806096 73.81542572743737) bank98769 +98770 POINT(40.52678767891755 73.2263902192966) bank98770 +98771 POINT(39.92385310829856 73.27296586756803) bank98771 +98772 POINT(41.50935625882846 74.7264247835431) bank98772 +98773 POINT(41.54057902550021 73.70438084100147) bank98773 +98774 POINT(41.50506819976045 73.1467534415826) bank98774 +98775 POINT(40.46026653470715 73.65134130857268) bank98775 +98776 POINT(39.9805251266382 74.72669422724009) bank98776 +98777 POINT(39.71849721469862 73.89205010130323) bank98777 +98778 POINT(40.43897232961151 74.32515962873984) bank98778 +98779 POINT(40.10820989200932 74.26804164084473) bank98779 +98780 POINT(39.937224621462605 73.13965365729763) bank98780 +98781 POINT(40.34947249909507 73.85695552465073) bank98781 +98782 POINT(40.836377240651444 73.61000297621545) bank98782 +98783 POINT(40.01851549017688 74.39832759302341) bank98783 +98784 POINT(39.79018380931166 74.42779923034092) bank98784 +98785 POINT(41.46318389398236 74.64501504742576) bank98785 +98786 POINT(40.03649632000512 74.59341027006248) bank98786 +98787 POINT(40.00660588635954 73.85722864137055) bank98787 +98788 POINT(41.26283471791875 73.9769595643479) bank98788 +98789 POINT(40.16985545274007 73.22247207567541) bank98789 +98790 POINT(41.65252635064631 74.08379589583761) bank98790 +98791 POINT(40.80635837742423 74.07772907928394) bank98791 +98792 POINT(40.84331702330226 73.32618487148373) bank98792 +98793 POINT(40.452554655544446 73.81718709639489) bank98793 +98794 POINT(40.262034577947816 73.40605966726473) bank98794 +98795 POINT(40.056760257397 73.99209172735581) bank98795 +98796 POINT(40.6832617523996 73.34811943331407) bank98796 +98797 POINT(41.01984570805425 73.6111611801211) bank98797 +98798 POINT(40.46080890897612 73.36652559199243) bank98798 +98799 POINT(40.41613028480674 73.24003534765897) bank98799 +98800 POINT(41.0041445059 73.712135253498) bank98800 +98801 POINT(40.93425034767745 73.82560482617406) bank98801 +98802 POINT(40.11199705371953 74.81719352353839) bank98802 +98803 POINT(39.7345640606881 73.60075248708264) bank98803 +98804 POINT(40.51589139921091 74.7441781280877) bank98804 +98805 POINT(39.998549835668086 74.29494305118124) bank98805 +98806 POINT(40.27321286784274 74.5554580737248) bank98806 +98807 POINT(41.22410351524868 74.52381285112114) bank98807 +98808 POINT(41.39625863359146 74.37178496356962) bank98808 +98809 POINT(41.22891819797081 73.20558436473299) bank98809 +98810 POINT(41.215107558054804 73.49266171668901) bank98810 +98811 POINT(41.23571131944389 74.05514703415611) bank98811 +98812 POINT(40.84718376302799 73.61478928570011) bank98812 +98813 POINT(40.53460942391287 73.01880516307781) bank98813 +98814 POINT(39.99933107712273 73.19002165302427) bank98814 +98815 POINT(41.62496532772304 73.66878036596432) bank98815 +98816 POINT(40.45776970628952 73.6369641626352) bank98816 +98817 POINT(41.50686476172262 74.2078529159695) bank98817 +98818 POINT(40.59941772227106 73.78746585199517) bank98818 +98819 POINT(40.84472668066461 73.55261122390112) bank98819 +98820 POINT(40.10348238036402 73.13748597430578) bank98820 +98821 POINT(41.45947590991914 74.03624623774247) bank98821 +98822 POINT(39.852005356017514 74.00058508765099) bank98822 +98823 POINT(39.806977336616384 73.89358381659048) bank98823 +98824 POINT(41.00894775756672 74.4892651658035) bank98824 +98825 POINT(41.54135723214656 74.45169142739343) bank98825 +98826 POINT(40.27204626925916 73.5687860670794) bank98826 +98827 POINT(40.33772077151625 74.5340374511545) bank98827 +98828 POINT(40.95619640590303 73.42902474073387) bank98828 +98829 POINT(41.35248146730873 74.91942183541914) bank98829 +98830 POINT(39.9967295261507 73.42966407120522) bank98830 +98831 POINT(40.96800995510716 74.59693170779232) bank98831 +98832 POINT(40.32361938714292 73.13504716278699) bank98832 +98833 POINT(40.87428435274464 73.3455284184774) bank98833 +98834 POINT(40.773221469069036 73.86229814677392) bank98834 +98835 POINT(41.274039415760235 74.21654666637693) bank98835 +98836 POINT(39.733778778344494 74.56285895974104) bank98836 +98837 POINT(41.07047609002644 73.9389683584246) bank98837 +98838 POINT(41.32590955385659 74.98240134584847) bank98838 +98839 POINT(40.666943419638024 73.6688223353658) bank98839 +98840 POINT(40.08680982594985 73.29389973626067) bank98840 +98841 POINT(40.4336408452127 73.68562257341809) bank98841 +98842 POINT(41.071076373749996 73.17216085438974) bank98842 +98843 POINT(40.68805716577494 73.24825368739037) bank98843 +98844 POINT(40.006634258230626 73.2565162786469) bank98844 +98845 POINT(40.096275480832304 73.55013684171573) bank98845 +98846 POINT(39.775482550215365 73.49165109996687) bank98846 +98847 POINT(40.5327453694095 74.48892051298999) bank98847 +98848 POINT(41.64916900713684 73.43349069589726) bank98848 +98849 POINT(40.24452582187735 73.08883371558696) bank98849 +98850 POINT(40.66232795636272 73.06186530505978) bank98850 +98851 POINT(41.31640445079225 74.64733241222748) bank98851 +98852 POINT(41.45827923834436 74.72222261078262) bank98852 +98853 POINT(41.610056626275295 73.65504026501235) bank98853 +98854 POINT(40.977749520486995 74.37315130592961) bank98854 +98855 POINT(41.29674675884916 74.8030206013525) bank98855 +98856 POINT(39.98788330186927 73.04439775940722) bank98856 +98857 POINT(40.866535693692605 73.1074807393901) bank98857 +98858 POINT(41.29607202390408 74.30646636214601) bank98858 +98859 POINT(40.63563156644799 74.39881568923093) bank98859 +98860 POINT(40.91493223950677 74.49611702051129) bank98860 +98861 POINT(40.34742999610138 73.75219075899936) bank98861 +98862 POINT(41.00422911903545 73.95435272246796) bank98862 +98863 POINT(40.66170390831302 73.19513887963083) bank98863 +98864 POINT(39.78881542783878 73.94818884426275) bank98864 +98865 POINT(40.35268304644848 74.66950809110124) bank98865 +98866 POINT(41.12293158622167 74.02978443453507) bank98866 +98867 POINT(39.836371839632484 74.13848678073609) bank98867 +98868 POINT(41.45786016930891 73.69819920678009) bank98868 +98869 POINT(39.96579422552089 73.94913935045736) bank98869 +98870 POINT(40.219207145331026 74.41484934249047) bank98870 +98871 POINT(40.900388312352405 73.07046821843788) bank98871 +98872 POINT(41.23221695605424 74.65505968702318) bank98872 +98873 POINT(41.63652135219267 73.05694724299067) bank98873 +98874 POINT(40.342956980796075 73.73320109242803) bank98874 +98875 POINT(40.775297872195715 73.95573976816013) bank98875 +98876 POINT(39.84839384524964 74.32824413547932) bank98876 +98877 POINT(40.127949253607405 74.3454482850477) bank98877 +98878 POINT(40.85044561408567 73.25837618751461) bank98878 +98879 POINT(39.99941559950295 73.60741693103353) bank98879 +98880 POINT(39.90659334081938 73.34179686154194) bank98880 +98881 POINT(40.44404690524539 73.55178976687668) bank98881 +98882 POINT(40.324407465782365 73.15067644008064) bank98882 +98883 POINT(41.03375763035085 73.22756995376066) bank98883 +98884 POINT(40.370892640426106 73.59067307220214) bank98884 +98885 POINT(41.534455070507974 74.54741049349418) bank98885 +98886 POINT(40.36718197100869 73.57860187394651) bank98886 +98887 POINT(41.28193777597059 74.38442115808971) bank98887 +98888 POINT(40.84309766805649 74.7217572873961) bank98888 +98889 POINT(41.17908533354432 74.81919937092961) bank98889 +98890 POINT(40.490282909339506 74.44495563213538) bank98890 +98891 POINT(40.359984422333575 73.89367108531751) bank98891 +98892 POINT(39.86561622120914 73.56290075503274) bank98892 +98893 POINT(41.12864914823284 73.27505977375452) bank98893 +98894 POINT(40.51175499439886 74.44664795504717) bank98894 +98895 POINT(41.330196729266454 73.31985762571098) bank98895 +98896 POINT(40.248837946409424 74.12071587783039) bank98896 +98897 POINT(41.2085185717333 73.0077997556588) bank98897 +98898 POINT(40.19255381021168 74.60581362437155) bank98898 +98899 POINT(39.96113208013394 73.12687917996585) bank98899 +98900 POINT(40.6415588377945 73.86353364059451) bank98900 +98901 POINT(40.6944939463538 73.25315997882085) bank98901 +98902 POINT(40.30069475899453 74.66584439440393) bank98902 +98903 POINT(40.53411627104814 73.90980002378019) bank98903 +98904 POINT(41.19669915766944 73.98373413797205) bank98904 +98905 POINT(40.81647257182026 73.5099125309073) bank98905 +98906 POINT(40.50434347608787 73.84764838550817) bank98906 +98907 POINT(40.18448636511748 73.46332899371889) bank98907 +98908 POINT(39.85563421121325 73.73447977722127) bank98908 +98909 POINT(39.91799774294329 74.33092432307048) bank98909 +98910 POINT(40.991602891551096 74.07638605876222) bank98910 +98911 POINT(40.168578331143955 74.30604967482441) bank98911 +98912 POINT(41.55359109172839 73.52927384333871) bank98912 +98913 POINT(40.747013964448016 74.27753624705093) bank98913 +98914 POINT(41.198929827137874 74.8756908924454) bank98914 +98915 POINT(40.05898714904997 74.83973613130411) bank98915 +98916 POINT(41.50970154913148 74.28381335713581) bank98916 +98917 POINT(41.3775072041115 73.22074990469714) bank98917 +98918 POINT(40.28064164986736 73.80570718311526) bank98918 +98919 POINT(40.081995089758195 74.25537433317562) bank98919 +98920 POINT(39.840669917548766 74.69793174611115) bank98920 +98921 POINT(41.456148315764146 73.42561210918645) bank98921 +98922 POINT(41.166550560911546 74.30962644790347) bank98922 +98923 POINT(40.473404551055594 73.15056641573317) bank98923 +98924 POINT(41.473245979097406 74.05811882312784) bank98924 +98925 POINT(41.55043929682012 74.571235567554) bank98925 +98926 POINT(40.05439820901366 73.56856242444202) bank98926 +98927 POINT(40.342713936006 73.99640216789638) bank98927 +98928 POINT(41.50535526825232 73.95731881053761) bank98928 +98929 POINT(39.88733376614174 73.78049124841137) bank98929 +98930 POINT(41.13511452880806 73.12848547579401) bank98930 +98931 POINT(39.71566706250401 74.32596368474391) bank98931 +98932 POINT(41.376837556862206 74.4576626621253) bank98932 +98933 POINT(39.83596049358227 74.82526940036695) bank98933 +98934 POINT(41.520175620485254 73.40929022872199) bank98934 +98935 POINT(40.93380245357919 73.12910515974527) bank98935 +98936 POINT(40.680826205029945 73.04754827358025) bank98936 +98937 POINT(40.598122121677704 74.74080825106418) bank98937 +98938 POINT(40.69705525059019 73.70194088946651) bank98938 +98939 POINT(40.73572100782668 73.4320253118334) bank98939 +98940 POINT(41.16924483671661 74.4917093971978) bank98940 +98941 POINT(40.096188352890465 73.28532941221201) bank98941 +98942 POINT(41.62700742625705 74.89395503247025) bank98942 +98943 POINT(41.42940356439694 74.19581248456144) bank98943 +98944 POINT(41.51871978163443 74.9204986060117) bank98944 +98945 POINT(40.005997765025334 73.83226197140048) bank98945 +98946 POINT(41.43976362623328 74.52970762113982) bank98946 +98947 POINT(39.7797237228837 73.35252938332229) bank98947 +98948 POINT(41.621923652576626 73.06963075823434) bank98948 +98949 POINT(40.71288858229654 73.62422365478176) bank98949 +98950 POINT(39.96136797017797 74.2837645818459) bank98950 +98951 POINT(41.06049614430632 73.56728778216996) bank98951 +98952 POINT(41.49744959136379 73.6056579173115) bank98952 +98953 POINT(40.661034998491225 74.58132181873721) bank98953 +98954 POINT(41.042772332900526 73.58374270613918) bank98954 +98955 POINT(40.677264075233026 73.24544660577382) bank98955 +98956 POINT(39.74784484989045 74.67765369373765) bank98956 +98957 POINT(41.011227904547596 73.55227936812335) bank98957 +98958 POINT(40.76549626812718 73.93861530268012) bank98958 +98959 POINT(40.52393097317817 74.38862658088375) bank98959 +98960 POINT(40.74436052795808 74.22935053635197) bank98960 +98961 POINT(41.248159366533685 73.40706959171743) bank98961 +98962 POINT(40.64763306541704 73.1601077220351) bank98962 +98963 POINT(40.09138641262629 74.56801497322047) bank98963 +98964 POINT(41.11596900740362 73.23376560515717) bank98964 +98965 POINT(41.12227512768407 73.296488828061) bank98965 +98966 POINT(40.491537365907774 74.81416693400644) bank98966 +98967 POINT(40.59804589146271 74.69503629466725) bank98967 +98968 POINT(40.45620529745834 73.84957259483664) bank98968 +98969 POINT(40.65412280935083 73.13322451479175) bank98969 +98970 POINT(40.673667783506644 73.83521301909246) bank98970 +98971 POINT(41.00247711697067 74.82982792257067) bank98971 +98972 POINT(39.74140198053715 73.27387732650328) bank98972 +98973 POINT(41.149701848111235 73.48802196639124) bank98973 +98974 POINT(40.38647473810985 74.98681043586086) bank98974 +98975 POINT(39.93819767294013 73.59262012461303) bank98975 +98976 POINT(39.90139836515253 73.98104556940505) bank98976 +98977 POINT(41.46159563769947 74.48826095652306) bank98977 +98978 POINT(41.300206662486076 73.53398616915038) bank98978 +98979 POINT(40.533344981755576 73.79338374494115) bank98979 +98980 POINT(40.936043635629275 74.8061481630533) bank98980 +98981 POINT(40.80582606923143 74.8014796686769) bank98981 +98982 POINT(40.672166969123424 73.42431125596242) bank98982 +98983 POINT(40.17946628766201 73.0715211761148) bank98983 +98984 POINT(41.53282685505384 74.47839452990863) bank98984 +98985 POINT(40.12422293726912 74.75090121690046) bank98985 +98986 POINT(41.56185996594626 74.59536259783218) bank98986 +98987 POINT(39.92975945424711 73.10592198216624) bank98987 +98988 POINT(40.032205210801536 74.21542397284207) bank98988 +98989 POINT(40.11459285326302 74.2270888915001) bank98989 +98990 POINT(40.13845552386047 74.36978048340919) bank98990 +98991 POINT(40.887877153869155 74.2390290154699) bank98991 +98992 POINT(39.80625178602857 74.6507727662146) bank98992 +98993 POINT(40.321606437698065 74.40782377176488) bank98993 +98994 POINT(41.085797758316865 74.3191451101113) bank98994 +98995 POINT(40.33447209426216 74.00224485161276) bank98995 +98996 POINT(41.34882082369187 73.07460653972974) bank98996 +98997 POINT(40.75886140096519 74.6851220331489) bank98997 +98998 POINT(40.83322741909819 74.71122273270544) bank98998 +98999 POINT(40.86066644906225 74.55388677054698) bank98999 +99000 POINT(40.11256134221516 73.92272957058718) bank99000 +99001 POINT(40.278679858880196 73.35162268755253) bank99001 +99002 POINT(41.23735776915113 74.8670734147728) bank99002 +99003 POINT(41.00508237429047 74.24939458532818) bank99003 +99004 POINT(41.5501366969506 74.60067807953226) bank99004 +99005 POINT(41.029778878904224 73.66984064376834) bank99005 +99006 POINT(39.88243628606136 74.66836457536056) bank99006 +99007 POINT(41.07485236783673 73.89130431441329) bank99007 +99008 POINT(41.395421916499345 74.66870363677931) bank99008 +99009 POINT(39.94084843425355 73.17867989547314) bank99009 +99010 POINT(40.76802293494144 73.42840337569002) bank99010 +99011 POINT(41.41388492675305 74.41402350400648) bank99011 +99012 POINT(40.39469061237425 73.03844452435942) bank99012 +99013 POINT(40.81601976922827 74.7409570333312) bank99013 +99014 POINT(40.26113915311062 73.24222807250653) bank99014 +99015 POINT(41.3097147781052 73.21172572055484) bank99015 +99016 POINT(41.3563733822405 73.91447948427425) bank99016 +99017 POINT(40.183685421761865 73.75976594532165) bank99017 +99018 POINT(40.19148765840296 73.430720622659) bank99018 +99019 POINT(40.49829310305279 74.4189195003997) bank99019 +99020 POINT(40.0518722485428 73.52782390260484) bank99020 +99021 POINT(41.691896745081564 75.00036545702906) bank99021 +99022 POINT(41.12929899372869 74.56156489949845) bank99022 +99023 POINT(40.50114541249236 74.84527162350743) bank99023 +99024 POINT(41.143854434562165 73.78649240319827) bank99024 +99025 POINT(41.01353360776225 73.7521436719445) bank99025 +99026 POINT(41.178032504901715 73.9557904661334) bank99026 +99027 POINT(40.13165533879343 74.47464615370836) bank99027 +99028 POINT(41.434797054629456 74.50507396660339) bank99028 +99029 POINT(41.581348866666765 74.31218865441501) bank99029 +99030 POINT(41.1067494752961 74.05613194170066) bank99030 +99031 POINT(40.45707524728472 74.69217783054859) bank99031 +99032 POINT(40.48009988340036 74.55994790143042) bank99032 +99033 POINT(40.87345930216452 74.41289009568247) bank99033 +99034 POINT(41.323105293498074 74.79267769330374) bank99034 +99035 POINT(40.58952004812457 73.77784259832751) bank99035 +99036 POINT(41.366308680599396 73.69941847742783) bank99036 +99037 POINT(39.96809985987305 74.94446192104452) bank99037 +99038 POINT(41.67158853035659 74.61434269871395) bank99038 +99039 POINT(41.23009739110393 73.42517572599452) bank99039 +99040 POINT(40.10566203591338 74.63374130047623) bank99040 +99041 POINT(41.12072947995884 74.25776160824697) bank99041 +99042 POINT(40.308361121345456 74.27421515850763) bank99042 +99043 POINT(41.61282501851655 73.9910926496091) bank99043 +99044 POINT(39.82162739708369 73.26112028201663) bank99044 +99045 POINT(41.586856578938466 74.04152140621103) bank99045 +99046 POINT(40.74499865756102 73.81098601208652) bank99046 +99047 POINT(40.001659059098714 74.56414270588057) bank99047 +99048 POINT(40.798982089008376 74.57283651502186) bank99048 +99049 POINT(40.47205479995536 74.48475062600224) bank99049 +99050 POINT(40.483223518333176 74.64037142546124) bank99050 +99051 POINT(39.76936993831459 74.4960819795662) bank99051 +99052 POINT(39.866910199531226 74.31426360337677) bank99052 +99053 POINT(40.31343975266213 74.5490977002439) bank99053 +99054 POINT(40.23698513498539 74.88292170157742) bank99054 +99055 POINT(39.828980388776145 74.57931938970118) bank99055 +99056 POINT(41.69848747111607 73.3267555730059) bank99056 +99057 POINT(41.021831741466094 73.33280446323162) bank99057 +99058 POINT(41.5106143107065 73.43533983186205) bank99058 +99059 POINT(39.885987572533665 73.88013596529603) bank99059 +99060 POINT(40.37630220002305 74.13444038764665) bank99060 +99061 POINT(39.912122906147594 74.87815309306964) bank99061 +99062 POINT(40.153434801799705 73.16026128033624) bank99062 +99063 POINT(40.005214631325934 73.35583170004004) bank99063 +99064 POINT(39.9898939015875 73.85544994251931) bank99064 +99065 POINT(39.93455434772766 73.04963119282885) bank99065 +99066 POINT(40.77700229875197 73.14309576046988) bank99066 +99067 POINT(40.781109565527146 73.37834338158974) bank99067 +99068 POINT(39.88760023234861 73.2681660214255) bank99068 +99069 POINT(40.672801111760535 73.17021090071417) bank99069 +99070 POINT(41.084840051279386 74.50206509153993) bank99070 +99071 POINT(41.33908038095814 73.17093959320755) bank99071 +99072 POINT(41.53171629002025 74.42202397235067) bank99072 +99073 POINT(41.05228505722303 74.79041695124064) bank99073 +99074 POINT(40.603778818504566 74.58581974331874) bank99074 +99075 POINT(39.91128647994299 74.36798033157002) bank99075 +99076 POINT(40.111762071153635 74.69163215813958) bank99076 +99077 POINT(40.57657250457872 73.21593616175959) bank99077 +99078 POINT(41.1492634292035 74.90848819568136) bank99078 +99079 POINT(41.66136740249571 74.45390284258849) bank99079 +99080 POINT(40.31860907835738 73.44026039361414) bank99080 +99081 POINT(40.75542589150212 73.17144555321647) bank99081 +99082 POINT(41.45675633412888 74.21849534887404) bank99082 +99083 POINT(40.17104483490678 73.56243000859666) bank99083 +99084 POINT(41.581351516862206 74.3575052250975) bank99084 +99085 POINT(40.3593753575111 73.43819665113753) bank99085 +99086 POINT(40.21303146397637 73.96363342575854) bank99086 +99087 POINT(40.11626529054601 73.99122235259989) bank99087 +99088 POINT(39.93668964926203 74.77559238050429) bank99088 +99089 POINT(40.11390049759676 74.89650466871196) bank99089 +99090 POINT(41.69712021928852 73.87464481941554) bank99090 +99091 POINT(40.51740887096235 74.22462311493341) bank99091 +99092 POINT(41.62543939030835 73.43055505225048) bank99092 +99093 POINT(40.11898710290611 73.36501968602245) bank99093 +99094 POINT(39.908014181308424 74.54745566966247) bank99094 +99095 POINT(40.28878886394701 73.41870348955861) bank99095 +99096 POINT(41.236159144695506 73.10479400035906) bank99096 +99097 POINT(40.85610565660165 73.98260578886018) bank99097 +99098 POINT(39.753961072251 73.27117685803641) bank99098 +99099 POINT(40.95127866874416 73.76835812455751) bank99099 +99100 POINT(41.68301198654193 73.43516575324854) bank99100 +99101 POINT(39.71427748484486 74.65773558976952) bank99101 +99102 POINT(40.63941288238788 73.4676564401693) bank99102 +99103 POINT(41.62673960036697 73.14345314876216) bank99103 +99104 POINT(41.70056151848004 73.40269259618407) bank99104 +99105 POINT(41.50738778804953 74.86103726180312) bank99105 +99106 POINT(39.78744872062399 74.45987287293377) bank99106 +99107 POINT(39.92502128991009 73.18555203243609) bank99107 +99108 POINT(40.25151773853578 74.93746868323736) bank99108 +99109 POINT(40.67217598360781 73.01895414456793) bank99109 +99110 POINT(40.396636985535885 74.39279477083271) bank99110 +99111 POINT(40.51305123853947 73.55678540557977) bank99111 +99112 POINT(40.95050381747861 73.37707573243611) bank99112 +99113 POINT(39.86318885967434 73.95173323096971) bank99113 +99114 POINT(40.373372374683676 73.28322254834782) bank99114 +99115 POINT(40.09079820846058 74.05943804998114) bank99115 +99116 POINT(40.61338652459213 74.22327104238587) bank99116 +99117 POINT(41.244159210276564 74.27357013566915) bank99117 +99118 POINT(39.71410181015812 73.72274176430476) bank99118 +99119 POINT(40.947293940917895 74.1774242665854) bank99119 +99120 POINT(40.572546498804876 73.01519601323936) bank99120 +99121 POINT(40.667107675021946 73.66713988771086) bank99121 +99122 POINT(40.74433548598757 73.11401034821266) bank99122 +99123 POINT(41.05758512713548 74.8197898938345) bank99123 +99124 POINT(40.59918866918556 74.6155705083393) bank99124 +99125 POINT(40.66799596633015 74.42956969144547) bank99125 +99126 POINT(40.57872585789347 73.2365387059703) bank99126 +99127 POINT(40.93502074678011 73.67218313044715) bank99127 +99128 POINT(39.910857402890024 74.04446074981733) bank99128 +99129 POINT(40.54455584966358 74.19588759487783) bank99129 +99130 POINT(41.0442195462135 74.49076346527829) bank99130 +99131 POINT(41.630372571493 73.06008777012629) bank99131 +99132 POINT(40.684695142077466 73.75708651617354) bank99132 +99133 POINT(40.679945444603405 73.77017402076982) bank99133 +99134 POINT(39.94179029361023 74.53467968444139) bank99134 +99135 POINT(40.98173020222833 73.47391488285753) bank99135 +99136 POINT(40.35567184694659 73.87248156098715) bank99136 +99137 POINT(40.35673349349213 74.36613369752887) bank99137 +99138 POINT(40.894617784012105 73.80361596054868) bank99138 +99139 POINT(41.58972834046633 74.47326575342993) bank99139 +99140 POINT(40.87856234122563 73.60764685721352) bank99140 +99141 POINT(41.56631717013809 74.83958002502857) bank99141 +99142 POINT(39.918584758707844 73.06282443205471) bank99142 +99143 POINT(41.15043834901302 73.23329652759212) bank99143 +99144 POINT(40.1616067415645 73.19668962707534) bank99144 +99145 POINT(40.68539430401456 73.9467211504132) bank99145 +99146 POINT(40.328437722653476 74.03021664146517) bank99146 +99147 POINT(39.82842720056963 74.72151979456484) bank99147 +99148 POINT(40.27018495241477 74.7471025027876) bank99148 +99149 POINT(40.522968712833645 74.89608819263128) bank99149 +99150 POINT(41.18228862872105 73.06128737485487) bank99150 +99151 POINT(40.054311080792374 74.01549075322208) bank99151 +99152 POINT(40.94920007096397 73.89911358296347) bank99152 +99153 POINT(40.91355250064877 74.18356517924445) bank99153 +99154 POINT(39.76716714416523 73.71762969458514) bank99154 +99155 POINT(40.33950434782944 74.705497746405) bank99155 +99156 POINT(40.078990092937985 74.2401445636401) bank99156 +99157 POINT(41.44187796827378 73.89339426567244) bank99157 +99158 POINT(41.58613165150975 74.76676416085128) bank99158 +99159 POINT(40.6441842093976 73.73240598071791) bank99159 +99160 POINT(41.273985368795095 73.24638460438487) bank99160 +99161 POINT(40.48799252522165 74.56778516888775) bank99161 +99162 POINT(40.24971472785035 74.17731401499277) bank99162 +99163 POINT(40.98199378769421 74.26941911521762) bank99163 +99164 POINT(40.81436351671325 73.57701355939342) bank99164 +99165 POINT(41.05620659309204 74.36789879078165) bank99165 +99166 POINT(39.71581894827923 74.64255118432932) bank99166 +99167 POINT(40.95598309288758 74.35039063761793) bank99167 +99168 POINT(40.42983463276636 73.73237290629488) bank99168 +99169 POINT(39.927855473692816 74.7113064389269) bank99169 +99170 POINT(40.59407602074491 74.54165438093439) bank99170 +99171 POINT(40.93299730004687 73.2213144361654) bank99171 +99172 POINT(40.12029628112461 73.11599945756532) bank99172 +99173 POINT(41.27865797277115 74.23273416967206) bank99173 +99174 POINT(40.520673218422786 73.16328067052491) bank99174 +99175 POINT(41.689494464977145 74.01182514627114) bank99175 +99176 POINT(40.65774525016563 74.94929201829397) bank99176 +99177 POINT(40.55583814221734 74.50460089469115) bank99177 +99178 POINT(40.75885026637524 74.75855151364867) bank99178 +99179 POINT(39.76531071978845 74.42218607934223) bank99179 +99180 POINT(40.44380032309637 74.19857432746153) bank99180 +99181 POINT(41.31888427288476 74.20460119937565) bank99181 +99182 POINT(39.719452307345676 73.02160941737779) bank99182 +99183 POINT(40.853482248677864 73.36531394044219) bank99183 +99184 POINT(40.8751099648047 74.02917291723344) bank99184 +99185 POINT(41.27319465980584 74.00724438237297) bank99185 +99186 POINT(40.778128405817206 74.95416915124551) bank99186 +99187 POINT(39.956099243928804 74.25103290339203) bank99187 +99188 POINT(41.22549110887248 73.83574023338467) bank99188 +99189 POINT(40.348180554780775 74.59719752035114) bank99189 +99190 POINT(40.82728407863392 74.37556401437867) bank99190 +99191 POINT(40.871295826628916 73.18147514154437) bank99191 +99192 POINT(40.80285727466549 73.92660229886502) bank99192 +99193 POINT(41.233757835228644 73.21641095387405) bank99193 +99194 POINT(40.01648592629098 74.83467692212079) bank99194 +99195 POINT(40.23650688992965 74.33198753952269) bank99195 +99196 POINT(41.17445419504547 74.37012860352183) bank99196 +99197 POINT(40.70725024219965 74.27636450239943) bank99197 +99198 POINT(39.75689789109084 74.14905867375477) bank99198 +99199 POINT(40.98667293342719 74.97244150413022) bank99199 +99200 POINT(40.56708707873144 73.68241680065806) bank99200 +99201 POINT(40.68060916624237 73.52879062290145) bank99201 +99202 POINT(41.63266467526208 73.49920002531213) bank99202 +99203 POINT(40.86459026898346 74.86988303128804) bank99203 +99204 POINT(41.2054245305256 74.03443199986842) bank99204 +99205 POINT(40.14419001972087 74.77796805283825) bank99205 +99206 POINT(39.76015022375524 73.0283630206794) bank99206 +99207 POINT(41.37581622739869 74.7117666869945) bank99207 +99208 POINT(39.736383455584246 73.68165032007946) bank99208 +99209 POINT(39.78621975414004 73.65655761420531) bank99209 +99210 POINT(40.896952406327465 74.34504577844851) bank99210 +99211 POINT(41.686584161701276 74.38388526229186) bank99211 +99212 POINT(41.46254842145827 73.5357103435257) bank99212 +99213 POINT(41.00067740762574 73.56130741356647) bank99213 +99214 POINT(40.35183072123495 74.16929643873048) bank99214 +99215 POINT(40.26748321502385 74.40446440953654) bank99215 +99216 POINT(41.38323894185319 74.95842118564462) bank99216 +99217 POINT(41.691612506352115 73.96964898181287) bank99217 +99218 POINT(41.52958000583825 74.65369989299305) bank99218 +99219 POINT(41.60592497530085 73.57871943482125) bank99219 +99220 POINT(40.005199741091126 73.7259164278539) bank99220 +99221 POINT(39.97987634269322 74.96066497245049) bank99221 +99222 POINT(41.16933767748257 73.50163523227633) bank99222 +99223 POINT(39.80665420332672 73.96613330000186) bank99223 +99224 POINT(40.81585260858196 73.47832717005045) bank99224 +99225 POINT(41.00465336737939 73.85845699071446) bank99225 +99226 POINT(40.95409625532915 74.67402783880932) bank99226 +99227 POINT(40.13278987058765 73.56879081392809) bank99227 +99228 POINT(40.15883043234085 73.17803257318295) bank99228 +99229 POINT(39.960176540609595 73.45974876084287) bank99229 +99230 POINT(41.49620844383191 73.70139152247515) bank99230 +99231 POINT(39.72993820314838 74.47935914858353) bank99231 +99232 POINT(39.921452533098226 74.66118727722326) bank99232 +99233 POINT(41.253191516209995 73.58180276954648) bank99233 +99234 POINT(40.99781670327139 73.0532577824499) bank99234 +99235 POINT(41.22005838713959 73.23379729732889) bank99235 +99236 POINT(40.296431723775804 74.58579893282322) bank99236 +99237 POINT(41.40170905492589 73.59185974289363) bank99237 +99238 POINT(40.91609939971373 73.36681860457671) bank99238 +99239 POINT(39.85269946608872 73.06727024241458) bank99239 +99240 POINT(41.18316819212195 74.2194700939761) bank99240 +99241 POINT(40.257489048688484 73.7042623946646) bank99241 +99242 POINT(39.81911863463739 74.77658933097692) bank99242 +99243 POINT(40.46063863353221 73.28003311813255) bank99243 +99244 POINT(41.45692440390896 73.16190115457158) bank99244 +99245 POINT(41.095208591536704 73.84743086663273) bank99245 +99246 POINT(40.04817516420552 74.44883285806807) bank99246 +99247 POINT(39.921183716501794 74.36402385970176) bank99247 +99248 POINT(41.261630769687784 74.8954510386249) bank99248 +99249 POINT(41.12172794540096 73.36478256952688) bank99249 +99250 POINT(40.2134752681726 73.90186504823387) bank99250 +99251 POINT(40.88136315906977 73.99048499464661) bank99251 +99252 POINT(40.43914679251771 74.74003461325947) bank99252 +99253 POINT(41.297753836733264 74.16740969714974) bank99253 +99254 POINT(40.23763318512329 74.62152303162458) bank99254 +99255 POINT(40.933312071866624 74.12616440313325) bank99255 +99256 POINT(39.84040358011722 73.86885599602033) bank99256 +99257 POINT(41.388127089490844 73.15275631201223) bank99257 +99258 POINT(41.64605182594893 73.12197587613113) bank99258 +99259 POINT(40.9389325245663 74.88808089954217) bank99259 +99260 POINT(40.1000882954177 73.64650854012515) bank99260 +99261 POINT(40.759736741488474 73.8273391249369) bank99261 +99262 POINT(40.17321621405967 74.30024020195232) bank99262 +99263 POINT(41.27975249839754 73.76946081193722) bank99263 +99264 POINT(40.241679951049846 74.39757598384169) bank99264 +99265 POINT(40.04001764752346 74.28662149798701) bank99265 +99266 POINT(40.825415393052666 73.50862592043191) bank99266 +99267 POINT(41.497612207284774 74.5458023779076) bank99267 +99268 POINT(39.89757731658942 73.46578395333795) bank99268 +99269 POINT(41.081405139766446 73.16264360210111) bank99269 +99270 POINT(41.68454901840053 73.29957286967725) bank99270 +99271 POINT(41.61822350137658 74.0708044199328) bank99271 +99272 POINT(41.35246544420722 74.87812376498671) bank99272 +99273 POINT(40.292911296970935 74.30308339603687) bank99273 +99274 POINT(40.976432125645616 74.97059414523135) bank99274 +99275 POINT(40.752283505852596 74.25290182943425) bank99275 +99276 POINT(39.73572265861864 74.94722518229874) bank99276 +99277 POINT(40.22753155171883 74.27469599160375) bank99277 +99278 POINT(39.92971522165421 74.27705128098762) bank99278 +99279 POINT(40.593943623784476 73.03467244561948) bank99279 +99280 POINT(40.220068101079264 74.9252847044888) bank99280 +99281 POINT(41.18605934314913 74.19169325394027) bank99281 +99282 POINT(39.85404168048097 73.0356138404411) bank99282 +99283 POINT(39.8123331851994 73.79902024121773) bank99283 +99284 POINT(40.917595116801685 74.67627160423497) bank99284 +99285 POINT(40.61632044769772 74.0462932921081) bank99285 +99286 POINT(40.32818451936178 73.02536355650007) bank99286 +99287 POINT(40.000637895667815 74.82715638403545) bank99287 +99288 POINT(40.80957043588612 74.58162108397322) bank99288 +99289 POINT(40.48053687685774 74.49501556382991) bank99289 +99290 POINT(41.1969330463598 74.49130384050635) bank99290 +99291 POINT(40.51730916357785 74.124802360092) bank99291 +99292 POINT(41.68538015518774 73.75525085443006) bank99292 +99293 POINT(39.88736761757333 73.942455404844) bank99293 +99294 POINT(40.25015819968131 73.49832269669302) bank99294 +99295 POINT(40.85429428884577 73.64694880072395) bank99295 +99296 POINT(40.93506092869911 74.42973106048846) bank99296 +99297 POINT(40.1958110132772 74.63467690174667) bank99297 +99298 POINT(39.828147488874556 73.36068163391162) bank99298 +99299 POINT(41.09011295176273 73.8754313058021) bank99299 +99300 POINT(40.7781381621078 74.84557573418796) bank99300 +99301 POINT(40.09793711668667 74.79039266178242) bank99301 +99302 POINT(39.862028908098495 74.40042828208048) bank99302 +99303 POINT(40.90642518433008 74.25952886256597) bank99303 +99304 POINT(41.578206446148165 73.22940114803184) bank99304 +99305 POINT(39.838840747144474 73.10166028487563) bank99305 +99306 POINT(41.67826153892887 74.6801562163512) bank99306 +99307 POINT(40.459725378951305 73.09685796496431) bank99307 +99308 POINT(41.20275336081663 73.28870270257762) bank99308 +99309 POINT(41.33821872814163 73.69285379227475) bank99309 +99310 POINT(41.18808774628073 74.85297122315173) bank99310 +99311 POINT(40.4783770614524 74.13655454002006) bank99311 +99312 POINT(40.72540341271445 73.40571466022797) bank99312 +99313 POINT(40.110252969164804 74.09344942346225) bank99313 +99314 POINT(39.82076142048057 73.24803016851538) bank99314 +99315 POINT(40.33244989982195 73.31777299467028) bank99315 +99316 POINT(40.0904690141879 74.06740072095754) bank99316 +99317 POINT(41.319149357925525 74.69143802388759) bank99317 +99318 POINT(41.13795549540739 73.37095046144502) bank99318 +99319 POINT(41.10696391481884 74.02093772124681) bank99319 +99320 POINT(39.787235628281145 73.7780252230268) bank99320 +99321 POINT(39.98935654950617 73.64219869161839) bank99321 +99322 POINT(40.754970413925776 74.26638053976812) bank99322 +99323 POINT(39.98792828421706 73.48207054005756) bank99323 +99324 POINT(41.22052438531562 73.68357557927467) bank99324 +99325 POINT(40.230071051258825 74.31282356528426) bank99325 +99326 POINT(40.52274148350001 73.99048931605402) bank99326 +99327 POINT(39.763201327677166 74.44903235321242) bank99327 +99328 POINT(41.32532222716372 74.66462345736227) bank99328 +99329 POINT(41.04428094950608 74.52871648839248) bank99329 +99330 POINT(40.791822185442676 73.73535729017999) bank99330 +99331 POINT(39.795929055257936 74.43289856309411) bank99331 +99332 POINT(40.863604890034 74.7574473369318) bank99332 +99333 POINT(41.269567242216695 73.3206284354473) bank99333 +99334 POINT(40.1585408078069 73.89558555977584) bank99334 +99335 POINT(40.56763070744455 74.9410243663809) bank99335 +99336 POINT(40.22424389777948 74.14970260160398) bank99336 +99337 POINT(40.73128993386768 73.01662306057008) bank99337 +99338 POINT(41.52420593786522 73.99972461284271) bank99338 +99339 POINT(39.81059757322342 73.8857729414663) bank99339 +99340 POINT(40.64692472796576 74.71543936212444) bank99340 +99341 POINT(40.06394565657915 74.20821312841322) bank99341 +99342 POINT(41.01795702060007 73.35699346338046) bank99342 +99343 POINT(39.79629865700113 73.54978173118151) bank99343 +99344 POINT(41.455051941642154 73.2742757787466) bank99344 +99345 POINT(40.8793995668204 74.86626246123714) bank99345 +99346 POINT(41.33779292316194 73.82763665585585) bank99346 +99347 POINT(39.73282939664543 74.85504107016199) bank99347 +99348 POINT(40.425605306856994 73.1491054409166) bank99348 +99349 POINT(41.172623567685946 73.67672436348923) bank99349 +99350 POINT(40.77623092589205 73.02252204511328) bank99350 +99351 POINT(40.70617843771015 73.1287293199436) bank99351 +99352 POINT(41.68786240998638 74.37837131717751) bank99352 +99353 POINT(40.36448117243955 73.37001352200554) bank99353 +99354 POINT(41.52224295974657 73.23330480597413) bank99354 +99355 POINT(40.14296368099138 74.05601452521505) bank99355 +99356 POINT(40.5411959806188 74.04491091815555) bank99356 +99357 POINT(40.43656663115488 73.88234384403978) bank99357 +99358 POINT(41.01297192277578 73.06468921300507) bank99358 +99359 POINT(41.63600070865111 73.23783602296508) bank99359 +99360 POINT(41.12067933172276 74.34453632407555) bank99360 +99361 POINT(40.24854097750084 73.18033910692706) bank99361 +99362 POINT(40.13936559421019 74.14046707606005) bank99362 +99363 POINT(40.327998190727975 73.92574972418778) bank99363 +99364 POINT(40.46636148572659 73.62547581930153) bank99364 +99365 POINT(39.77845324544976 74.470632901665) bank99365 +99366 POINT(40.11590811758177 74.28260000334785) bank99366 +99367 POINT(40.91087957476346 73.81330941065994) bank99367 +99368 POINT(41.196123105238065 74.22872425067902) bank99368 +99369 POINT(41.44386025140978 73.04352860287696) bank99369 +99370 POINT(41.03442047321977 73.80751081227655) bank99370 +99371 POINT(40.70817884806839 74.45070604319008) bank99371 +99372 POINT(40.83448457090293 74.41158278195682) bank99372 +99373 POINT(41.28044946019851 73.59094100995209) bank99373 +99374 POINT(40.37371401199295 73.46806250705653) bank99374 +99375 POINT(39.94083896882423 74.71595907972247) bank99375 +99376 POINT(40.464751058453686 73.32987042272585) bank99376 +99377 POINT(40.699601004308356 73.64660963592085) bank99377 +99378 POINT(40.09360730347327 74.73789537929517) bank99378 +99379 POINT(40.985788894112886 73.29173331491204) bank99379 +99380 POINT(41.6240956342489 74.53763808475284) bank99380 +99381 POINT(39.86664413193087 73.11249228423368) bank99381 +99382 POINT(41.466617975798044 74.8163426690983) bank99382 +99383 POINT(41.33466385055566 74.53927407496168) bank99383 +99384 POINT(41.099890641460604 74.62535812379728) bank99384 +99385 POINT(41.49216393845763 74.62170729541992) bank99385 +99386 POINT(40.10197685274947 73.1302162670767) bank99386 +99387 POINT(41.6283970272587 74.56275937096265) bank99387 +99388 POINT(41.382750521559366 73.4869464516018) bank99388 +99389 POINT(40.83579869041633 73.50137509519615) bank99389 +99390 POINT(40.83311487024029 73.56553274179424) bank99390 +99391 POINT(41.01412517163799 74.04285882745107) bank99391 +99392 POINT(39.872126294195354 74.95101558586191) bank99392 +99393 POINT(40.59772096057434 73.05606014570203) bank99393 +99394 POINT(39.76005635503444 74.00315753668248) bank99394 +99395 POINT(40.38771696993286 73.27101794075317) bank99395 +99396 POINT(40.967294972280335 74.10553618107302) bank99396 +99397 POINT(41.467211610599655 73.98019599064428) bank99397 +99398 POINT(40.37675564420013 73.41089732640725) bank99398 +99399 POINT(41.620644436382335 73.49924418148866) bank99399 +99400 POINT(40.2973905923405 73.95011745734008) bank99400 +99401 POINT(40.0242472332721 74.70891708578675) bank99401 +99402 POINT(41.07187877058545 73.3083807641219) bank99402 +99403 POINT(41.40682011122236 74.59792147042057) bank99403 +99404 POINT(41.389521219060185 74.93711100544851) bank99404 +99405 POINT(40.72643223106082 74.50913553593705) bank99405 +99406 POINT(40.90916581610812 75.00028156830793) bank99406 +99407 POINT(40.36251500486676 74.8438449614836) bank99407 +99408 POINT(41.27485242984669 73.08056941228158) bank99408 +99409 POINT(41.066396596067044 74.08773950165109) bank99409 +99410 POINT(40.61341105673264 73.85236233556098) bank99410 +99411 POINT(40.25200510391979 74.88238526851873) bank99411 +99412 POINT(41.41976245534614 74.67806684260276) bank99412 +99413 POINT(39.747883248428494 74.96214728511754) bank99413 +99414 POINT(40.393013175523166 73.06712907322947) bank99414 +99415 POINT(40.425402078993734 73.33394422435913) bank99415 +99416 POINT(41.540743667746526 74.07878180035007) bank99416 +99417 POINT(39.834450302335505 74.14457980294532) bank99417 +99418 POINT(40.102960920171775 74.14308992210184) bank99418 +99419 POINT(41.02478320521756 74.61229278745795) bank99419 +99420 POINT(40.53941340255401 74.91571419335651) bank99420 +99421 POINT(41.04891557345352 73.32398110366438) bank99421 +99422 POINT(40.9655378420192 74.36874809465758) bank99422 +99423 POINT(40.27241142498269 73.89112626364843) bank99423 +99424 POINT(41.68524483089082 73.41279293171749) bank99424 +99425 POINT(40.0535442581123 74.15983936431984) bank99425 +99426 POINT(40.80693034612054 74.25292831073193) bank99426 +99427 POINT(40.2028324141568 73.29661052330206) bank99427 +99428 POINT(39.72663867332873 73.29417930603897) bank99428 +99429 POINT(39.98675976011281 73.0066965953003) bank99429 +99430 POINT(40.52936262493736 74.2487030664012) bank99430 +99431 POINT(40.883499565768076 74.12057665957268) bank99431 +99432 POINT(41.06293699762372 74.55399764547577) bank99432 +99433 POINT(40.726477966796594 73.39155247341093) bank99433 +99434 POINT(41.38427298633782 74.44674195578429) bank99434 +99435 POINT(40.80209018663482 74.47591013425821) bank99435 +99436 POINT(39.88132784334479 73.72151102709958) bank99436 +99437 POINT(39.91666816143995 74.21982656925528) bank99437 +99438 POINT(40.414391800942354 74.4527313051501) bank99438 +99439 POINT(40.74931346592126 74.98228461822593) bank99439 +99440 POINT(39.78917548899517 74.08285109892772) bank99440 +99441 POINT(41.255044686680144 74.81280970418999) bank99441 +99442 POINT(39.95415827502467 73.29339172334318) bank99442 +99443 POINT(39.862424348885014 73.52220097066737) bank99443 +99444 POINT(39.83138990133834 73.25261046037663) bank99444 +99445 POINT(40.123914594113934 73.49877747674232) bank99445 +99446 POINT(39.86913265740726 74.87720696786721) bank99446 +99447 POINT(41.37828128213711 73.05090623880223) bank99447 +99448 POINT(39.9961558147576 74.76554285112245) bank99448 +99449 POINT(40.14810176827616 73.29307479301075) bank99449 +99450 POINT(40.39505701880036 73.44929981079015) bank99450 +99451 POINT(41.373383332192276 73.37733768734661) bank99451 +99452 POINT(41.629915538182175 74.73123020702344) bank99452 +99453 POINT(40.698992593445524 73.34225714614779) bank99453 +99454 POINT(40.03628167115003 74.24742674794452) bank99454 +99455 POINT(40.40467039676485 74.76248898758128) bank99455 +99456 POINT(41.46759945081028 73.25120588588418) bank99456 +99457 POINT(41.12394585464738 73.1345093644097) bank99457 +99458 POINT(41.19843744106277 73.86272724557246) bank99458 +99459 POINT(40.53172838881537 73.16333896738668) bank99459 +99460 POINT(39.77107312438635 74.72836229986194) bank99460 +99461 POINT(40.2113607939831 73.53602065534074) bank99461 +99462 POINT(40.98490515478662 73.18325367275818) bank99462 +99463 POINT(40.98307173980713 73.56071574603966) bank99463 +99464 POINT(41.299611555128415 74.88286765963333) bank99464 +99465 POINT(40.03007951018257 74.69471013561554) bank99465 +99466 POINT(41.04194546596756 73.46571469958444) bank99466 +99467 POINT(39.78176825566501 73.49704147418313) bank99467 +99468 POINT(40.23279725883364 73.16355578779546) bank99468 +99469 POINT(39.969423173199694 73.95593302895138) bank99469 +99470 POINT(41.42824457960495 74.72451283918178) bank99470 +99471 POINT(41.2792004291828 74.22360525382761) bank99471 +99472 POINT(41.62499041481583 74.67357255254666) bank99472 +99473 POINT(40.259672364881546 73.52716380236994) bank99473 +99474 POINT(40.3857499241614 74.86621186911917) bank99474 +99475 POINT(39.973666921877665 73.14911203754706) bank99475 +99476 POINT(40.48494804529009 73.18343325900537) bank99476 +99477 POINT(40.13685326595499 73.36658987782366) bank99477 +99478 POINT(41.4773031051424 73.25660962726516) bank99478 +99479 POINT(40.86897214496486 74.46432254151132) bank99479 +99480 POINT(39.8583531007649 74.07680446810805) bank99480 +99481 POINT(40.506599611993025 74.06030582368) bank99481 +99482 POINT(40.40659495906936 74.23332405298278) bank99482 +99483 POINT(40.2219391719441 73.89763418691045) bank99483 +99484 POINT(40.99466922341052 74.69026451920868) bank99484 +99485 POINT(41.17976763964763 74.74245292066124) bank99485 +99486 POINT(40.33157335043468 74.35634769392799) bank99486 +99487 POINT(41.249384294251676 73.62496508915557) bank99487 +99488 POINT(41.5514552325627 73.66517315562803) bank99488 +99489 POINT(40.69034857139557 73.9953972419754) bank99489 +99490 POINT(40.11597923035283 74.28505932168564) bank99490 +99491 POINT(41.287849043258774 73.58916382954433) bank99491 +99492 POINT(40.93880468676877 74.28108762658488) bank99492 +99493 POINT(39.77137890968624 74.05334691317327) bank99493 +99494 POINT(40.800216886394125 73.09910219818298) bank99494 +99495 POINT(40.999119421090256 74.88860475765976) bank99495 +99496 POINT(41.40429862130245 73.26392040067009) bank99496 +99497 POINT(39.938114799596065 73.43496687371182) bank99497 +99498 POINT(41.47360511534166 74.25287581143972) bank99498 +99499 POINT(40.38832995830491 73.14616565473791) bank99499 +99500 POINT(39.864577587263156 73.87524689714667) bank99500 +99501 POINT(40.81215561521228 73.21275109008292) bank99501 +99502 POINT(39.83448267140129 73.44082049643868) bank99502 +99503 POINT(39.99173415819526 73.92267692520836) bank99503 +99504 POINT(40.36041358067234 74.89456245348369) bank99504 +99505 POINT(41.55874579594767 74.22389527837808) bank99505 +99506 POINT(41.62946601161008 74.24500839087968) bank99506 +99507 POINT(40.30994389567338 73.02844418792817) bank99507 +99508 POINT(40.84004493777367 73.068925123014) bank99508 +99509 POINT(41.37458715918335 74.6583128206673) bank99509 +99510 POINT(40.816846754886946 73.32133507244143) bank99510 +99511 POINT(40.13250513496903 74.97073119039823) bank99511 +99512 POINT(41.539389724856996 74.72724256334655) bank99512 +99513 POINT(40.78256091492794 74.33553411886732) bank99513 +99514 POINT(41.57137961294624 74.72712538707493) bank99514 +99515 POINT(39.91326686119159 74.00844275878798) bank99515 +99516 POINT(41.435646222668794 74.70456304414343) bank99516 +99517 POINT(39.93625995995207 74.76261314029294) bank99517 +99518 POINT(41.48183109329193 74.71714727615021) bank99518 +99519 POINT(40.00124240436409 74.96723957316254) bank99519 +99520 POINT(40.55303628509575 73.1189810306085) bank99520 +99521 POINT(40.39744434510069 73.32385461039212) bank99521 +99522 POINT(40.07210989898749 73.03868189713904) bank99522 +99523 POINT(41.559757337006026 73.83992656443453) bank99523 +99524 POINT(40.68303416102572 74.38006413370445) bank99524 +99525 POINT(40.46873641789003 74.2022075368468) bank99525 +99526 POINT(40.60486152992892 73.77845180952067) bank99526 +99527 POINT(39.89843113745159 74.686340654164) bank99527 +99528 POINT(40.389915975830604 73.69238316248074) bank99528 +99529 POINT(41.5209698717018 73.93232144981535) bank99529 +99530 POINT(40.82790081871219 74.86552465801003) bank99530 +99531 POINT(40.89706715403585 74.43901118321934) bank99531 +99532 POINT(41.24508405301619 74.5134431936929) bank99532 +99533 POINT(39.99008707531721 74.62510925977465) bank99533 +99534 POINT(40.74741091633919 73.73458241111128) bank99534 +99535 POINT(40.08313632269688 73.94398049302455) bank99535 +99536 POINT(41.37047200057078 73.46597962271215) bank99536 +99537 POINT(39.99559317752248 73.08372368955875) bank99537 +99538 POINT(41.35830235790044 74.3451405422284) bank99538 +99539 POINT(40.03323662619184 73.14605861655552) bank99539 +99540 POINT(40.173340252467355 74.52974468443925) bank99540 +99541 POINT(41.006952064600284 73.12820025419737) bank99541 +99542 POINT(40.5523614448516 74.21812324977992) bank99542 +99543 POINT(40.89552914797088 74.6698071047294) bank99543 +99544 POINT(41.56032624694671 73.69010325823218) bank99544 +99545 POINT(41.51252324262007 73.34590135482692) bank99545 +99546 POINT(41.66864845796717 73.77068503287146) bank99546 +99547 POINT(41.6644192015755 74.26863602617618) bank99547 +99548 POINT(41.56944548837038 74.31831743646103) bank99548 +99549 POINT(39.71451426233003 74.78309337608036) bank99549 +99550 POINT(39.82734153950898 74.70984661180317) bank99550 +99551 POINT(40.27587723093103 73.53768661871193) bank99551 +99552 POINT(41.70215853850562 73.00619138746893) bank99552 +99553 POINT(40.839085138400314 74.60801747081487) bank99553 +99554 POINT(41.699576217162665 73.43505808049207) bank99554 +99555 POINT(41.340523506743395 73.13939475597702) bank99555 +99556 POINT(41.22003462941369 74.31897813297027) bank99556 +99557 POINT(39.96789027184201 73.43322441039443) bank99557 +99558 POINT(40.68063050645827 75.00341684438305) bank99558 +99559 POINT(40.149946342560085 73.3387785285922) bank99559 +99560 POINT(40.265813389295495 74.94054226419776) bank99560 +99561 POINT(40.35574703907466 73.24130606399345) bank99561 +99562 POINT(41.00973678321907 73.1641783459132) bank99562 +99563 POINT(39.95744173016539 73.35089055586884) bank99563 +99564 POINT(40.65581290597638 73.2872727896683) bank99564 +99565 POINT(41.26636788934844 73.01710415110485) bank99565 +99566 POINT(41.198785414506055 74.11723066339759) bank99566 +99567 POINT(41.11585743276589 74.06419696928208) bank99567 +99568 POINT(41.200046834764024 74.7554361837729) bank99568 +99569 POINT(40.340476832605226 73.7863117169677) bank99569 +99570 POINT(41.19915236903456 73.52462955352428) bank99570 +99571 POINT(41.27462996032719 73.90379105214575) bank99571 +99572 POINT(40.893184363433896 74.3329612189516) bank99572 +99573 POINT(40.74300473966658 74.78266989232586) bank99573 +99574 POINT(40.51411888496566 73.419457781622) bank99574 +99575 POINT(41.18602284112471 74.47205204158918) bank99575 +99576 POINT(41.4477702606141 74.52974348391518) bank99576 +99577 POINT(40.683571722722895 74.7284163560919) bank99577 +99578 POINT(39.78903636528954 74.0596295026066) bank99578 +99579 POINT(39.821301116102745 73.39719719880188) bank99579 +99580 POINT(40.14832311595854 74.22227063971403) bank99580 +99581 POINT(40.73244276949498 73.22897075023943) bank99581 +99582 POINT(39.780660757048096 73.87218584729003) bank99582 +99583 POINT(40.13410382600863 73.4272094963229) bank99583 +99584 POINT(39.93762164438203 74.9668110672724) bank99584 +99585 POINT(39.71845168904208 74.86159185485565) bank99585 +99586 POINT(40.99063090693575 74.22496031352908) bank99586 +99587 POINT(40.939799742605935 74.93000650787502) bank99587 +99588 POINT(40.09612744795153 74.57406247657113) bank99588 +99589 POINT(40.247425567732776 74.85315229414347) bank99589 +99590 POINT(41.15613471096558 74.68942019460401) bank99590 +99591 POINT(41.70932423955123 73.69480536949823) bank99591 +99592 POINT(40.39860553179395 74.74777104448685) bank99592 +99593 POINT(41.63822687917051 74.19164625933333) bank99593 +99594 POINT(41.294579691499095 74.52423382455291) bank99594 +99595 POINT(41.037990250893465 73.98169013346431) bank99595 +99596 POINT(40.01484845801246 73.30467687793978) bank99596 +99597 POINT(40.12689413727306 74.04360383114825) bank99597 +99598 POINT(41.29820471034675 74.69900613019408) bank99598 +99599 POINT(40.664461954634376 74.08983627673511) bank99599 +99600 POINT(40.23759478802512 74.00532766215036) bank99600 +99601 POINT(40.52765509083491 73.62159806628544) bank99601 +99602 POINT(40.902167410783726 73.45773227940815) bank99602 +99603 POINT(39.7902291373132 74.05107983513903) bank99603 +99604 POINT(41.38723067248678 73.87675149546291) bank99604 +99605 POINT(41.0901517275521 74.12848949501632) bank99605 +99606 POINT(39.82266948462552 74.08265823060438) bank99606 +99607 POINT(41.35165774990283 74.89480861424592) bank99607 +99608 POINT(39.87821282488006 73.96740094574359) bank99608 +99609 POINT(40.05582039093404 74.48965488525228) bank99609 +99610 POINT(40.15846618502839 74.11062228818061) bank99610 +99611 POINT(41.60814690065271 73.68903385852583) bank99611 +99612 POINT(40.459086654376236 73.68588869209538) bank99612 +99613 POINT(40.390871007011974 74.30112473706792) bank99613 +99614 POINT(40.59897028309063 74.25106635535543) bank99614 +99615 POINT(40.148132145958904 74.36670439682055) bank99615 +99616 POINT(40.22597245387874 73.78496248169344) bank99616 +99617 POINT(39.915088350527085 74.85892435724779) bank99617 +99618 POINT(41.22134794125699 74.1242990066308) bank99618 +99619 POINT(40.02289148146339 74.44070498009224) bank99619 +99620 POINT(39.88951482695331 73.00776619360477) bank99620 +99621 POINT(41.140120106405 73.455375378166) bank99621 +99622 POINT(41.2289031268529 73.61597224241541) bank99622 +99623 POINT(39.86693018135123 73.05021980982731) bank99623 +99624 POINT(40.711108268121414 74.1411304971463) bank99624 +99625 POINT(40.08741961139553 73.89239060619296) bank99625 +99626 POINT(40.81008481715738 74.20815136239283) bank99626 +99627 POINT(41.07116602072204 73.21293390764187) bank99627 +99628 POINT(40.23877426464721 73.65438416268978) bank99628 +99629 POINT(41.60886033072262 73.57723661867581) bank99629 +99630 POINT(41.53885407332362 73.59800091503628) bank99630 +99631 POINT(41.12374024465445 73.00842264332401) bank99631 +99632 POINT(39.86139972593373 73.94112378533694) bank99632 +99633 POINT(40.192367569886265 73.68052131158467) bank99633 +99634 POINT(40.28053312856315 73.47100794621274) bank99634 +99635 POINT(40.37005020052253 73.37404372494213) bank99635 +99636 POINT(41.64772807188392 73.78682575486616) bank99636 +99637 POINT(39.835860164895564 74.67552868060756) bank99637 +99638 POINT(41.33180177808876 73.37580330735126) bank99638 +99639 POINT(41.31952001589209 73.45489895515936) bank99639 +99640 POINT(41.19930308216991 73.39040393602625) bank99640 +99641 POINT(41.052025182211146 73.30104095604302) bank99641 +99642 POINT(40.043331330891775 73.69667664539003) bank99642 +99643 POINT(40.09205255875823 74.41912903647203) bank99643 +99644 POINT(41.18501889564364 74.10297052182807) bank99644 +99645 POINT(39.79040468568221 73.4438224954935) bank99645 +99646 POINT(40.25784248929248 73.5657744182717) bank99646 +99647 POINT(39.73676087010495 74.09746428578123) bank99647 +99648 POINT(40.96644430371154 73.87753178377874) bank99648 +99649 POINT(41.000713662841285 73.89451576666042) bank99649 +99650 POINT(40.9234168447559 73.84780205837238) bank99650 +99651 POINT(40.24076432029178 74.54507214206777) bank99651 +99652 POINT(40.59520620124426 73.95108591545605) bank99652 +99653 POINT(39.770894462150956 74.22672784842942) bank99653 +99654 POINT(40.317769639045885 74.70992083270846) bank99654 +99655 POINT(40.578175820288806 73.56907184416194) bank99655 +99656 POINT(39.96288725584183 74.68585407312533) bank99656 +99657 POINT(41.40673774286319 74.26296664340802) bank99657 +99658 POINT(40.327970202452086 73.34526061166912) bank99658 +99659 POINT(41.6540842079506 74.69816009850304) bank99659 +99660 POINT(40.391638351860145 74.03274758350972) bank99660 +99661 POINT(40.02133982248348 74.40714272620244) bank99661 +99662 POINT(39.7524639311408 74.95103049419656) bank99662 +99663 POINT(39.97810675612047 74.67986022216928) bank99663 +99664 POINT(41.26618402713381 73.54747015639441) bank99664 +99665 POINT(41.43849560440375 74.54631285384788) bank99665 +99666 POINT(39.93503915283277 73.59827778862125) bank99666 +99667 POINT(41.05862810246917 73.86315375452816) bank99667 +99668 POINT(41.191126846883094 74.83725190446383) bank99668 +99669 POINT(41.47553985252841 74.9265041295756) bank99669 +99670 POINT(39.776346372245314 73.57636167493293) bank99670 +99671 POINT(40.989550720003564 73.28655816952879) bank99671 +99672 POINT(41.71180379970575 74.39537725838146) bank99672 +99673 POINT(41.565017944132904 74.07550026114957) bank99673 +99674 POINT(40.820476404544834 73.75010223424586) bank99674 +99675 POINT(39.76806257350173 74.97487634307693) bank99675 +99676 POINT(40.50078980746696 73.96290646387283) bank99676 +99677 POINT(40.879163410217885 74.3501320836447) bank99677 +99678 POINT(39.88721614039662 74.20736774469074) bank99678 +99679 POINT(40.678772424304896 74.92091440639206) bank99679 +99680 POINT(40.063885976591614 73.4721119716766) bank99680 +99681 POINT(39.89050915168886 74.46739337800965) bank99681 +99682 POINT(40.195817151947544 73.69642505875134) bank99682 +99683 POINT(40.66225574217772 74.35681557618645) bank99683 +99684 POINT(40.112769225035365 74.45813215951443) bank99684 +99685 POINT(40.43038499198893 73.97604782764019) bank99685 +99686 POINT(40.32529760470357 74.88486694136142) bank99686 +99687 POINT(40.534026715341966 73.76121494325702) bank99687 +99688 POINT(40.72433256596143 73.46239645195928) bank99688 +99689 POINT(40.74564741760718 74.38381658199847) bank99689 +99690 POINT(41.37604998356322 74.37960581412992) bank99690 +99691 POINT(41.36984611162375 73.10139406022711) bank99691 +99692 POINT(40.98291087716429 74.93852391189456) bank99692 +99693 POINT(40.16888637321103 73.7078034544811) bank99693 +99694 POINT(39.75493335742676 73.50008863946901) bank99694 +99695 POINT(41.44747080637378 74.14808537312358) bank99695 +99696 POINT(40.95761980374352 74.38358365404206) bank99696 +99697 POINT(40.88797217422539 73.04029506881601) bank99697 +99698 POINT(41.246247562832984 74.06113148639695) bank99698 +99699 POINT(40.91142308814666 74.67511576883474) bank99699 +99700 POINT(41.64353428549624 73.3189792671249) bank99700 +99701 POINT(41.2642561051052 74.98983863206037) bank99701 +99702 POINT(40.59890196892686 73.25409888613994) bank99702 +99703 POINT(41.67184551044952 73.07219026579796) bank99703 +99704 POINT(41.241594645015134 74.25609457165986) bank99704 +99705 POINT(40.259585279908585 74.21490974958694) bank99705 +99706 POINT(40.00351959288437 73.50337249536656) bank99706 +99707 POINT(41.420878124946654 73.98572756712208) bank99707 +99708 POINT(41.11075079429023 74.19971441200849) bank99708 +99709 POINT(41.505077288361065 74.79413073402816) bank99709 +99710 POINT(40.431547395950936 73.16512889295058) bank99710 +99711 POINT(41.165742150228716 73.91321924020346) bank99711 +99712 POINT(41.41542489117805 74.58904602765622) bank99712 +99713 POINT(41.47753888253611 73.69216197497141) bank99713 +99714 POINT(40.19968176097509 73.80643815010325) bank99714 +99715 POINT(40.931751167680844 74.9461323325442) bank99715 +99716 POINT(41.26230149323933 74.24754492585984) bank99716 +99717 POINT(40.67354662303075 73.52823950242013) bank99717 +99718 POINT(40.335279346859885 73.23412912155847) bank99718 +99719 POINT(40.94002957033392 74.54724056028232) bank99719 +99720 POINT(41.47903344745521 74.95649878379189) bank99720 +99721 POINT(39.902125094413364 73.05533886883033) bank99721 +99722 POINT(40.86751457902297 73.24551488599165) bank99722 +99723 POINT(40.24738663458891 74.38775360164186) bank99723 +99724 POINT(41.3493061375462 74.50320331520207) bank99724 +99725 POINT(40.1976298931125 73.32170493536398) bank99725 +99726 POINT(40.13497613471782 74.20901105311803) bank99726 +99727 POINT(41.03519172611357 74.73354583461727) bank99727 +99728 POINT(39.7530781384866 73.07158643634253) bank99728 +99729 POINT(40.58851221269365 74.82356394111248) bank99729 +99730 POINT(41.32922847576916 73.78381130572073) bank99730 +99731 POINT(41.3773546878033 74.7612801291954) bank99731 +99732 POINT(39.99839679672166 74.68381434893166) bank99732 +99733 POINT(40.64620260407154 74.18962797891616) bank99733 +99734 POINT(39.777954130665336 73.06346683585969) bank99734 +99735 POINT(40.13808150797795 73.55667006109694) bank99735 +99736 POINT(41.20908579569872 74.85347340796346) bank99736 +99737 POINT(40.46973431471428 74.26743471093829) bank99737 +99738 POINT(40.07509805911707 73.90417869493062) bank99738 +99739 POINT(40.99125441347036 73.99597425951121) bank99739 +99740 POINT(41.387109527871225 73.75686407398408) bank99740 +99741 POINT(40.51596755965664 74.17730780702814) bank99741 +99742 POINT(40.02557378688483 74.49081188773383) bank99742 +99743 POINT(41.101783264834665 74.57542488476014) bank99743 +99744 POINT(40.01835063548204 73.98299034442012) bank99744 +99745 POINT(40.50919024995817 73.2922746191496) bank99745 +99746 POINT(40.497223065533454 74.67570639655534) bank99746 +99747 POINT(40.04853679010129 74.88048962700114) bank99747 +99748 POINT(40.96824381965813 73.27783548010999) bank99748 +99749 POINT(40.54668503681513 73.24419363464975) bank99749 +99750 POINT(40.09579850210115 74.0079321324874) bank99750 +99751 POINT(41.511946060948596 73.99514765898712) bank99751 +99752 POINT(41.34013549757612 74.04794029837524) bank99752 +99753 POINT(39.955450825182346 73.4177562981529) bank99753 +99754 POINT(41.691166786691134 74.65007709262679) bank99754 +99755 POINT(40.60564075950073 75.00537784283061) bank99755 +99756 POINT(41.427872719590326 74.16887958477749) bank99756 +99757 POINT(40.07923496048626 74.44295266446763) bank99757 +99758 POINT(40.040767782245446 74.38869986530291) bank99758 +99759 POINT(40.31075732059252 74.44589753505036) bank99759 +99760 POINT(41.65128895052543 73.57706129884475) bank99760 +99761 POINT(39.89630299579009 74.89171576048402) bank99761 +99762 POINT(40.559054895253006 73.90212721660886) bank99762 +99763 POINT(41.27470972658866 73.36083774000764) bank99763 +99764 POINT(40.33424870461992 73.73035417208415) bank99764 +99765 POINT(41.61389556256575 74.12861188501553) bank99765 +99766 POINT(39.721087911409924 74.3050137902508) bank99766 +99767 POINT(41.60958807479336 73.89480845501272) bank99767 +99768 POINT(40.27155430381039 73.89991215763112) bank99768 +99769 POINT(41.216720209989724 73.85784499388933) bank99769 +99770 POINT(40.9907682479857 73.98854289231527) bank99770 +99771 POINT(40.56784352956494 73.30584164730548) bank99771 +99772 POINT(39.906568468599446 74.41599602312422) bank99772 +99773 POINT(41.14373915649139 73.9761759433628) bank99773 +99774 POINT(39.79860299228236 73.56944228007465) bank99774 +99775 POINT(41.4015919886182 74.58262624178002) bank99775 +99776 POINT(40.06453729702771 74.30679959716196) bank99776 +99777 POINT(41.494260465593506 73.01577195846025) bank99777 +99778 POINT(40.0352724368963 73.86181275904504) bank99778 +99779 POINT(41.357292082150344 73.54520235840828) bank99779 +99780 POINT(40.14301931365199 73.89087666045494) bank99780 +99781 POINT(41.687441673049435 73.24609886493593) bank99781 +99782 POINT(41.41432089605065 74.98741720447671) bank99782 +99783 POINT(40.55391518313063 74.08679072861464) bank99783 +99784 POINT(39.9377241797606 74.84255390169251) bank99784 +99785 POINT(41.43108132810607 73.44609161313878) bank99785 +99786 POINT(40.86934010350022 74.40718384892124) bank99786 +99787 POINT(40.742752671758595 74.0318527592968) bank99787 +99788 POINT(41.33576371109175 74.43586826359858) bank99788 +99789 POINT(41.62798471323671 74.09363808979731) bank99789 +99790 POINT(40.2293280011381 74.67023673329516) bank99790 +99791 POINT(41.70453905668917 73.8285643722959) bank99791 +99792 POINT(40.597505407952184 74.45011051807849) bank99792 +99793 POINT(40.39702876764579 73.70964227135734) bank99793 +99794 POINT(41.42971135759244 73.83908989345142) bank99794 +99795 POINT(40.07007255994716 74.48649053686222) bank99795 +99796 POINT(40.70672343826689 73.58017011529543) bank99796 +99797 POINT(39.98419942408576 73.96892413171888) bank99797 +99798 POINT(41.284392183486275 73.37478057241051) bank99798 +99799 POINT(41.17904475252517 73.66528035608003) bank99799 +99800 POINT(40.999477714044794 74.768293520662) bank99800 +99801 POINT(41.20804493424972 73.00713034769875) bank99801 +99802 POINT(41.24345075028129 73.89660955903749) bank99802 +99803 POINT(41.39242753752811 73.31277726590515) bank99803 +99804 POINT(40.35981724796419 74.09426374599921) bank99804 +99805 POINT(41.41283963625619 73.35392419074643) bank99805 +99806 POINT(40.81259588583007 74.80326241250128) bank99806 +99807 POINT(41.07358068561844 74.3938045410182) bank99807 +99808 POINT(40.31159685535505 73.58848350036101) bank99808 +99809 POINT(40.5576917148944 74.43930462745648) bank99809 +99810 POINT(41.03238921300222 74.21201502856627) bank99810 +99811 POINT(40.761071545859416 74.90252632269114) bank99811 +99812 POINT(40.93656474235235 73.81418769365844) bank99812 +99813 POINT(40.98821088329946 74.50600076134475) bank99813 +99814 POINT(40.70758853525957 74.76786592596217) bank99814 +99815 POINT(41.71075847893885 74.89007685131516) bank99815 +99816 POINT(41.552332107098565 74.16552043596519) bank99816 +99817 POINT(41.550656520112256 73.48161458030265) bank99817 +99818 POINT(40.74648929480153 73.12554668411698) bank99818 +99819 POINT(40.48759183475655 73.01674107486483) bank99819 +99820 POINT(40.025623672461194 74.06332050343899) bank99820 +99821 POINT(41.1363521864861 74.80045698876971) bank99821 +99822 POINT(40.28662106462876 73.06431323742837) bank99822 +99823 POINT(39.91636753588753 74.7017065304588) bank99823 +99824 POINT(40.16440303073051 74.25383231849109) bank99824 +99825 POINT(41.50226496264023 74.27865062338593) bank99825 +99826 POINT(40.83409642688257 73.05038438781231) bank99826 +99827 POINT(40.56966154469445 73.84777108559754) bank99827 +99828 POINT(40.049116303121714 74.55336365208107) bank99828 +99829 POINT(40.680584102963415 73.62774987862996) bank99829 +99830 POINT(40.615919066324274 73.63938573068707) bank99830 +99831 POINT(40.49242011345602 73.63611309148145) bank99831 +99832 POINT(40.555178307672186 73.895731983154) bank99832 +99833 POINT(40.26919285969643 73.41844975174277) bank99833 +99834 POINT(40.459364516983285 74.89438009402444) bank99834 +99835 POINT(39.78399472838169 73.1235915528843) bank99835 +99836 POINT(40.4508282309895 74.66997270612481) bank99836 +99837 POINT(41.08332171117219 73.2165091375054) bank99837 +99838 POINT(41.075668615625986 73.30831618844384) bank99838 +99839 POINT(40.276628347034034 74.25806368805424) bank99839 +99840 POINT(39.95363347319659 74.45501990086514) bank99840 +99841 POINT(40.40384473759168 74.46449622084094) bank99841 +99842 POINT(41.622009919718145 73.1473498757908) bank99842 +99843 POINT(40.08061718364605 73.25981384854006) bank99843 +99844 POINT(41.26169222555888 74.83378661747125) bank99844 +99845 POINT(40.60147417760292 74.86105737318158) bank99845 +99846 POINT(39.978581953642724 73.14900158184008) bank99846 +99847 POINT(40.12351369932352 74.1171036301227) bank99847 +99848 POINT(41.20172492572979 74.32384407161916) bank99848 +99849 POINT(41.39706742038328 74.00546020711468) bank99849 +99850 POINT(40.734260487483866 74.01372217387858) bank99850 +99851 POINT(40.87268579188471 73.24247213763498) bank99851 +99852 POINT(39.9120293165437 73.96603453005767) bank99852 +99853 POINT(39.80641668549748 74.02428419127827) bank99853 +99854 POINT(40.037451170511915 73.64770424900695) bank99854 +99855 POINT(41.43810567111362 74.10262387895534) bank99855 +99856 POINT(41.55601998048596 74.33159680724944) bank99856 +99857 POINT(39.962319260879816 74.74575238625935) bank99857 +99858 POINT(41.33438851471447 74.16035096368326) bank99858 +99859 POINT(41.56735524921171 74.24919310078134) bank99859 +99860 POINT(40.89964872870095 74.08109856148248) bank99860 +99861 POINT(39.7607381586716 73.86650719169415) bank99861 +99862 POINT(40.41936233571856 73.16920172435745) bank99862 +99863 POINT(40.13807474864436 74.8373507161585) bank99863 +99864 POINT(40.59444944725497 73.66837983358744) bank99864 +99865 POINT(41.67456826373328 73.94376027970435) bank99865 +99866 POINT(39.78102831788679 74.47199773492692) bank99866 +99867 POINT(41.530428949552686 74.38400670229538) bank99867 +99868 POINT(40.422571293405355 73.75872789159662) bank99868 +99869 POINT(40.468941619030495 73.90873473686852) bank99869 +99870 POINT(41.300544031044225 73.1379536393923) bank99870 +99871 POINT(40.633627003377015 73.88395590421631) bank99871 +99872 POINT(40.8763440292009 73.64466433172193) bank99872 +99873 POINT(40.84708186549671 73.57618613504539) bank99873 +99874 POINT(39.72679246553872 73.71703911122442) bank99874 +99875 POINT(40.14862489595415 73.08221383900761) bank99875 +99876 POINT(41.04575579890695 74.62012513699786) bank99876 +99877 POINT(40.04295277833155 73.01095200519451) bank99877 +99878 POINT(40.82701151012505 74.82618565770038) bank99878 +99879 POINT(40.33547110457927 74.91777035620485) bank99879 +99880 POINT(40.42849203841777 74.4259468189141) bank99880 +99881 POINT(39.88546006400393 73.9796382942266) bank99881 +99882 POINT(40.697029342231616 74.24206806611109) bank99882 +99883 POINT(41.1727019608313 74.61744323054249) bank99883 +99884 POINT(39.86849365516227 73.64984867565468) bank99884 +99885 POINT(39.96949273794996 73.24891591139637) bank99885 +99886 POINT(41.20869443267699 73.70667605135863) bank99886 +99887 POINT(39.77146375255226 73.20778744163253) bank99887 +99888 POINT(40.7386493975651 73.92627213628708) bank99888 +99889 POINT(40.655497902536545 73.04492146439091) bank99889 +99890 POINT(40.565207417926814 74.24514906162956) bank99890 +99891 POINT(40.132416769211105 73.19793392001228) bank99891 +99892 POINT(40.17316374936775 74.66800573117258) bank99892 +99893 POINT(40.51307273758215 74.54656493574416) bank99893 +99894 POINT(41.08513245005426 74.16228425911783) bank99894 +99895 POINT(41.62783714327657 73.33940337252804) bank99895 +99896 POINT(40.55188422608518 74.21312469167152) bank99896 +99897 POINT(40.182721505274955 73.17037233215463) bank99897 +99898 POINT(39.895880113434586 73.92342155992209) bank99898 +99899 POINT(40.00363382645798 73.8636743160256) bank99899 +99900 POINT(40.44465979072473 73.55176497938474) bank99900 +99901 POINT(40.56662996328953 74.70564032768621) bank99901 +99902 POINT(41.13302493553405 74.56851797626265) bank99902 +99903 POINT(40.35480371890482 73.14038563472451) bank99903 +99904 POINT(39.80933014634888 73.54677044511416) bank99904 +99905 POINT(41.30378673128223 73.43561091615713) bank99905 +99906 POINT(40.84073164504263 74.73419601962378) bank99906 +99907 POINT(41.02756701311409 74.5919345147355) bank99907 +99908 POINT(40.312935153629205 73.76068554660998) bank99908 +99909 POINT(40.84133769034949 73.24090178622383) bank99909 +99910 POINT(39.75588850433479 74.40550041562884) bank99910 +99911 POINT(41.4989080975654 73.1796619465919) bank99911 +99912 POINT(40.669207970463674 74.58981439126262) bank99912 +99913 POINT(40.00780508707505 74.19333551140184) bank99913 +99914 POINT(40.74808143577606 73.0737277286786) bank99914 +99915 POINT(39.985706182813125 73.52972173936516) bank99915 +99916 POINT(40.25420540418125 73.1662016817936) bank99916 +99917 POINT(41.022451816160036 74.63546579370453) bank99917 +99918 POINT(41.485780631368584 73.26916180669674) bank99918 +99919 POINT(39.93604685415765 74.43333230577223) bank99919 +99920 POINT(40.036189593006526 73.192826797363) bank99920 +99921 POINT(40.41047513197601 74.8001326990238) bank99921 +99922 POINT(40.420914917130325 73.2127506002165) bank99922 +99923 POINT(41.13954761563434 74.35865455812284) bank99923 +99924 POINT(41.205860539039946 73.91354467192335) bank99924 +99925 POINT(39.716464860152364 73.81410213565198) bank99925 +99926 POINT(40.2981087930056 73.3097073894319) bank99926 +99927 POINT(41.470435249753606 74.7324158652302) bank99927 +99928 POINT(41.580203706988776 73.85053591639777) bank99928 +99929 POINT(40.8474854387046 73.68593838187196) bank99929 +99930 POINT(40.11837308528773 73.92195504515404) bank99930 +99931 POINT(41.00870986277039 74.62284733545832) bank99931 +99932 POINT(41.00437486005805 74.93763334426043) bank99932 +99933 POINT(41.26082877478691 74.57479763415462) bank99933 +99934 POINT(40.95032522154473 73.29955347814239) bank99934 +99935 POINT(40.97997364817744 74.16514726020381) bank99935 +99936 POINT(40.82027685578722 73.32608026207086) bank99936 +99937 POINT(40.69812229438064 74.67459867879485) bank99937 +99938 POINT(41.56117072496772 74.37026838017512) bank99938 +99939 POINT(40.23819813860579 74.74564657569661) bank99939 +99940 POINT(41.07897658671823 74.7266737953851) bank99940 +99941 POINT(41.70260216333776 74.80109081230722) bank99941 +99942 POINT(39.813630936394944 73.15351169089338) bank99942 +99943 POINT(39.84410198428222 74.19141845755698) bank99943 +99944 POINT(40.82876496180762 74.37432977002324) bank99944 +99945 POINT(40.59695987958043 74.49575915192852) bank99945 +99946 POINT(39.859940138591206 73.32056313922702) bank99946 +99947 POINT(40.288154615637644 73.39303165774449) bank99947 +99948 POINT(41.30161846448903 73.21762199422568) bank99948 +99949 POINT(41.70535717275845 74.47819319161565) bank99949 +99950 POINT(41.300437215655904 74.4483847869327) bank99950 +99951 POINT(40.69950453578729 73.69462067439495) bank99951 +99952 POINT(41.04176327536015 73.48211760773154) bank99952 +99953 POINT(41.69641812069675 74.53305243416077) bank99953 +99954 POINT(39.92064192937167 74.58424741603126) bank99954 +99955 POINT(40.684585406627946 74.21793825342567) bank99955 +99956 POINT(41.24676699206626 73.87701028279075) bank99956 +99957 POINT(41.548270847994246 74.65444946182146) bank99957 +99958 POINT(40.47653982186344 74.5119042115012) bank99958 +99959 POINT(40.05522904314307 73.82091082712532) bank99959 +99960 POINT(39.923902420179736 74.54615829790312) bank99960 +99961 POINT(40.87252187838306 73.37967711745237) bank99961 +99962 POINT(40.82171142199608 73.04676184028193) bank99962 +99963 POINT(41.06576411498608 73.91167606245457) bank99963 +99964 POINT(41.423460437733674 74.33053214258284) bank99964 +99965 POINT(40.45626539686721 74.33068683816363) bank99965 +99966 POINT(41.430613797058484 74.08022515681715) bank99966 +99967 POINT(40.09708999355779 73.40946686949428) bank99967 +99968 POINT(41.30055949461617 73.31286285887977) bank99968 +99969 POINT(40.91507688805479 74.7989470187864) bank99969 +99970 POINT(41.3635295421891 73.59884660250613) bank99970 +99971 POINT(41.55915421164758 74.18107071701223) bank99971 +99972 POINT(40.97062120163777 73.88200497192419) bank99972 +99973 POINT(40.83159269276857 74.04814809037441) bank99973 +99974 POINT(41.2690569552279 73.50944483985856) bank99974 +99975 POINT(41.16633335833548 73.868167353304) bank99975 +99976 POINT(40.954525353637955 73.21370391487036) bank99976 +99977 POINT(40.52568050279975 74.05853663274829) bank99977 +99978 POINT(41.41182087733066 73.07249972920009) bank99978 +99979 POINT(40.31295438884781 73.97117493056753) bank99979 +99980 POINT(39.72594315263812 74.80688282600819) bank99980 +99981 POINT(40.025061752447925 73.76974993452876) bank99981 +99982 POINT(41.586090266281616 73.66176296568514) bank99982 +99983 POINT(40.73780118976585 73.70435957146398) bank99983 +99984 POINT(39.77963414966775 73.48233207821474) bank99984 +99985 POINT(41.55263409718127 73.62866361227113) bank99985 +99986 POINT(40.836885768318375 73.75149040199263) bank99986 +99987 POINT(41.42102838433318 74.7900581619226) bank99987 +99988 POINT(39.98083454770275 74.96675431742011) bank99988 +99989 POINT(39.78089311318294 73.36982073414312) bank99989 +99990 POINT(40.41842793828515 73.56488727458257) bank99990 +99991 POINT(39.71339498477921 73.19886487504536) bank99991 +99992 POINT(40.94180069284131 74.51598736557185) bank99992 +99993 POINT(40.65705918635882 73.93559534073596) bank99993 +99994 POINT(39.867413399431015 73.9340398277921) bank99994 +99995 POINT(40.85159891905479 74.63357211310098) bank99995 +99996 POINT(40.930702134431236 74.25292389298905) bank99996 +99997 POINT(40.436039611548125 74.12763224192081) bank99997 +99998 POINT(40.08131617310992 73.2198124755565) bank99998 +99999 POINT(40.18881310066807 74.04522356923557) bank99999 +100000 POINT(40.671528507183176 74.92403575337616) bank100000 +100001 POINT(40.12840126929813 73.01892167712984) bank100001 +100002 POINT(40.09062925874433 73.46629099067235) bank100002 +100003 POINT(39.80267430853821 73.59899702348329) bank100003 +100004 POINT(40.44138957572824 74.52876606909973) bank100004 +100005 POINT(40.50907967440535 73.17950131812326) bank100005 +100006 POINT(40.90100994976256 73.68691476233344) bank100006 +100007 POINT(41.196679459878176 73.9168568504242) bank100007 +100008 POINT(40.86670740943031 74.44240077701026) bank100008 +100009 POINT(39.954257143874855 73.19666632126638) bank100009 +100010 POINT(40.062738352054 74.03625352251932) bank100010 +100011 POINT(40.38605047070545 74.20437231552611) bank100011 +100012 POINT(41.08685587377812 74.34628248832553) bank100012 +100013 POINT(41.64089195714045 73.52788691519387) bank100013 +100014 POINT(39.89920169860847 73.8720002346499) bank100014 +100015 POINT(40.91984430086358 74.71585561396893) bank100015 +100016 POINT(40.829919745203 73.03456959490137) bank100016 +100017 POINT(41.24572191109897 73.29993937449352) bank100017 +100018 POINT(41.49707623687314 73.47973439656501) bank100018 +100019 POINT(39.86731276516405 73.75036772746245) bank100019 +100020 POINT(40.78592003682381 74.13428428817478) bank100020 +100021 POINT(40.652538195141325 74.20189839879794) bank100021 +100022 POINT(39.98696383062306 73.50929715835812) bank100022 +100023 POINT(40.13825908009161 73.85031591000092) bank100023 +100024 POINT(41.30338705995792 74.30800290401405) bank100024 +100025 POINT(40.72924743098233 73.45676802095703) bank100025 +100026 POINT(40.89021683129811 73.00705523675717) bank100026 +100027 POINT(41.21171780970609 74.66242665704208) bank100027 +100028 POINT(40.165077212413564 73.819750295272) bank100028 +100029 POINT(41.30630749005134 73.42400913949045) bank100029 +100030 POINT(41.153962308204974 74.96973756537825) bank100030 +100031 POINT(41.380443513870034 74.00372220243861) bank100031 +100032 POINT(41.2651329727502 73.93611427777763) bank100032 +100033 POINT(40.57239197245349 73.59683216305761) bank100033 +100034 POINT(40.267943546550924 73.10983472512203) bank100034 +100035 POINT(41.26017668823345 73.29071494586628) bank100035 +100036 POINT(40.97640030266151 73.82292181122061) bank100036 +100037 POINT(40.730378425887366 73.04392070090135) bank100037 +100038 POINT(40.065940987831326 73.46392703454265) bank100038 +100039 POINT(40.040594861452966 73.17967922574483) bank100039 +100040 POINT(40.453102279375564 75.00594656107599) bank100040 +100041 POINT(41.320606054174206 73.61555685830467) bank100041 +100042 POINT(41.53409401827099 73.76371842924593) bank100042 +100043 POINT(41.35495245778185 73.3174870986127) bank100043 +100044 POINT(40.51430228699353 73.18358494147633) bank100044 +100045 POINT(41.53572374135498 74.98877217629865) bank100045 +100046 POINT(39.889251194324885 73.64746569031905) bank100046 +100047 POINT(41.70253389084907 74.83860942908477) bank100047 +100048 POINT(40.247960876062976 74.68801925330553) bank100048 +100049 POINT(40.85574646831835 73.46362697582215) bank100049 +100050 POINT(40.19919221855644 73.52243545671584) bank100050 +100051 POINT(40.90969050577888 73.72930410759118) bank100051 +100052 POINT(40.58658565929513 73.98869524521065) bank100052 +100053 POINT(41.5403812538839 74.275261946817) bank100053 +100054 POINT(41.66135735499058 73.84412596354252) bank100054 +100055 POINT(41.20691494952897 74.93812623700599) bank100055 +100056 POINT(41.13553054498106 74.51661930115674) bank100056 +100057 POINT(41.14487298213365 73.77337029776203) bank100057 +100058 POINT(40.777712709694484 73.68265784381764) bank100058 +100059 POINT(40.5625091872866 73.68812719928337) bank100059 +100060 POINT(40.09471970891864 74.17640721042807) bank100060 +100061 POINT(40.29868445083765 74.2268597398588) bank100061 +100062 POINT(39.9767152668071 73.0590080627327) bank100062 +100063 POINT(40.59060722223372 74.50391278239724) bank100063 +100064 POINT(40.627699578725114 73.93240823754164) bank100064 +100065 POINT(41.14640937057254 73.1492202121902) bank100065 +100066 POINT(41.37402722571603 74.04057909954724) bank100066 +100067 POINT(40.977772664560476 73.61012335998204) bank100067 +100068 POINT(41.61863414998233 73.72721898756991) bank100068 +100069 POINT(40.7454413177521 74.61732004262613) bank100069 +100070 POINT(40.60398432987465 73.38611006442173) bank100070 +100071 POINT(41.01519177916646 74.80653870909) bank100071 +100072 POINT(40.83070508466111 74.13699196295673) bank100072 +100073 POINT(40.95767673420632 73.83392464732667) bank100073 +100074 POINT(41.006567009664664 74.377318945002) bank100074 +100075 POINT(41.58300280843359 74.74216687017113) bank100075 +100076 POINT(40.342249806967374 74.31797813541239) bank100076 +100077 POINT(41.177358418318065 74.63235051378209) bank100077 +100078 POINT(41.24645523067575 73.47464091266372) bank100078 +100079 POINT(40.084762808358946 74.08707899597094) bank100079 +100080 POINT(40.07073864179137 74.20957872210838) bank100080 +100081 POINT(40.21728692388077 73.41574442653771) bank100081 +100082 POINT(41.29204710714984 74.87891399270183) bank100082 +100083 POINT(40.44130553862505 74.86709476856991) bank100083 +100084 POINT(41.380140013940824 74.25670473989513) bank100084 +100085 POINT(40.54679006126213 73.23614874040207) bank100085 +100086 POINT(40.04932633589474 74.21195008429704) bank100086 +100087 POINT(40.5234631985841 73.07688202268875) bank100087 +100088 POINT(41.64582681295831 74.54245415075987) bank100088 +100089 POINT(40.78164857915953 73.9966880442858) bank100089 +100090 POINT(41.26312022819362 73.21724470838252) bank100090 +100091 POINT(40.0252557010717 73.70946042930184) bank100091 +100092 POINT(40.49058476092412 73.58124139968828) bank100092 +100093 POINT(41.067765472069404 73.92819215393206) bank100093 +100094 POINT(40.21723946315887 74.90676407903666) bank100094 +100095 POINT(40.26199620226856 73.1108143172367) bank100095 +100096 POINT(41.58269577965083 74.43838324625948) bank100096 +100097 POINT(41.10484556282084 73.8964021032757) bank100097 +100098 POINT(39.73703981142588 73.93796988608281) bank100098 +100099 POINT(40.16463480646012 74.33023511465929) bank100099 +100100 POINT(41.20554162723964 73.7458652365361) bank100100 +100101 POINT(41.00835322079841 73.54026525218971) bank100101 +100102 POINT(41.36044577351041 73.66283164474254) bank100102 +100103 POINT(39.89233538450804 74.57938685033866) bank100103 +100104 POINT(40.858845228702975 74.77843653824027) bank100104 +100105 POINT(41.47351345299033 73.08705941958576) bank100105 +100106 POINT(40.21842578810438 73.49925243425803) bank100106 +100107 POINT(40.99806923851044 74.05676661026204) bank100107 +100108 POINT(40.806205756053096 73.85072840196531) bank100108 +100109 POINT(41.17953687827234 74.88770151745254) bank100109 +100110 POINT(40.64920190399061 73.04874629505731) bank100110 +100111 POINT(40.94209770113615 73.97664118257087) bank100111 +100112 POINT(40.98212330830719 73.25734826314097) bank100112 +100113 POINT(41.45462941298293 73.83562832466625) bank100113 +100114 POINT(40.7904357910664 73.17564686258373) bank100114 +100115 POINT(40.93848625970474 74.6931584770537) bank100115 +100116 POINT(40.24553965348188 73.99040407711001) bank100116 +100117 POINT(41.13673619190508 74.55052647706447) bank100117 +100118 POINT(40.34918844910797 74.84108560767407) bank100118 +100119 POINT(40.6773535883658 73.83953805968936) bank100119 +100120 POINT(40.34758452125021 73.44634334209172) bank100120 +100121 POINT(40.81493233745733 74.32114461876786) bank100121 +100122 POINT(39.78605429587206 74.56444956494839) bank100122 +100123 POINT(40.88482559089542 73.70345096772799) bank100123 +100124 POINT(41.21716820660713 73.14913151959205) bank100124 +100125 POINT(39.82123398094598 74.68870454862379) bank100125 +100126 POINT(41.459645010421596 73.48582316644875) bank100126 +100127 POINT(41.47027973228157 74.02171956683237) bank100127 +100128 POINT(40.28590569598386 74.80587391702072) bank100128 +100129 POINT(40.53127729343689 74.70412548364546) bank100129 +100130 POINT(40.59982371474363 74.31789269622584) bank100130 +100131 POINT(41.28432151126657 73.3733016427803) bank100131 +100132 POINT(39.92298303058584 73.52120293539018) bank100132 +100133 POINT(40.32880867696343 73.30315377064365) bank100133 +100134 POINT(40.460657020331595 74.72675015651106) bank100134 +100135 POINT(40.85050513908434 74.45497101908737) bank100135 +100136 POINT(41.493294900889104 74.51484563388806) bank100136 +100137 POINT(41.44837053775552 73.59842413199333) bank100137 +100138 POINT(40.61538529598562 74.34643909541157) bank100138 +100139 POINT(39.95325059306619 73.17796523687693) bank100139 +100140 POINT(39.93045903388439 73.26826789077965) bank100140 +100141 POINT(40.183230873616374 73.70114275221115) bank100141 +100142 POINT(40.275072061291375 74.58459681360397) bank100142 +100143 POINT(40.49226247191378 73.214034686052) bank100143 +100144 POINT(41.26645719724206 73.94205053692134) bank100144 +100145 POINT(40.82916097885081 73.86063483419329) bank100145 +100146 POINT(41.40630791469936 74.60766455818148) bank100146 +100147 POINT(41.41702424740344 73.37969070732974) bank100147 +100148 POINT(41.17515914495727 74.33207090594888) bank100148 +100149 POINT(40.03312580782551 73.12186921087151) bank100149 +100150 POINT(40.313810562679876 74.35536299753213) bank100150 +100151 POINT(41.20595663982931 73.26311293193505) bank100151 +100152 POINT(41.51431272270678 73.46949943668723) bank100152 +100153 POINT(40.35468445464935 73.57893236388718) bank100153 +100154 POINT(40.070717214111724 73.40647608457111) bank100154 +100155 POINT(40.30027188961512 73.58718797286605) bank100155 +100156 POINT(41.45073470823782 74.94906386511578) bank100156 +100157 POINT(40.43616318211763 74.88915596565113) bank100157 +100158 POINT(41.26342278877575 73.20327046456217) bank100158 +100159 POINT(41.23688926419826 73.63116715629559) bank100159 +100160 POINT(40.982558377947996 73.12824970800433) bank100160 +100161 POINT(41.144002532841824 73.69720602453992) bank100161 +100162 POINT(40.28568421755035 73.53157106161531) bank100162 +100163 POINT(41.159754066110594 74.76746647610796) bank100163 +100164 POINT(41.47968289628277 74.25167612937025) bank100164 +100165 POINT(40.566151604483274 74.46223862042056) bank100165 +100166 POINT(40.34429822883971 73.1894411268155) bank100166 +100167 POINT(41.41343464325603 74.30002809329659) bank100167 +100168 POINT(41.03208627461114 73.59176228024167) bank100168 +100169 POINT(40.7659527051099 74.3352107968884) bank100169 +100170 POINT(40.65154120992419 73.31026230536264) bank100170 +100171 POINT(41.630345471318535 74.39554572884374) bank100171 +100172 POINT(40.10416565254568 74.78996718575199) bank100172 +100173 POINT(40.78548100858996 73.61806075498328) bank100173 +100174 POINT(40.487516368476435 74.06048743260942) bank100174 +100175 POINT(41.44092238474846 73.84197707912436) bank100175 +100176 POINT(40.65178256392451 73.15868449089254) bank100176 +100177 POINT(40.459208590985874 73.45719694905014) bank100177 +100178 POINT(40.59187275509594 74.62356456625476) bank100178 +100179 POINT(41.273441147958884 73.14581767954404) bank100179 +100180 POINT(40.772913395843474 73.5651654650705) bank100180 +100181 POINT(41.50471468413985 73.38442189821109) bank100181 +100182 POINT(39.99714927135553 73.61672766971509) bank100182 +100183 POINT(41.0325814087123 74.43325088723049) bank100183 +100184 POINT(39.800371467048166 73.28695729471126) bank100184 +100185 POINT(40.735204556715786 73.40877264081634) bank100185 +100186 POINT(40.072861505885555 74.06970663719177) bank100186 +100187 POINT(40.81397636654562 73.35617601257171) bank100187 +100188 POINT(40.89307594029804 74.80041291245382) bank100188 +100189 POINT(40.62323396744484 74.99257199710921) bank100189 +100190 POINT(40.600494927080376 74.33761307011466) bank100190 +100191 POINT(40.68001705400278 73.74406240406175) bank100191 +100192 POINT(41.17496309498177 73.27792244642977) bank100192 +100193 POINT(41.52700949020466 73.56822287595207) bank100193 +100194 POINT(40.44174072382086 73.92992173258335) bank100194 +100195 POINT(41.42475601582342 73.4955344274655) bank100195 +100196 POINT(40.543714917832574 73.8047659104643) bank100196 +100197 POINT(40.48563635177738 74.60312879413785) bank100197 +100198 POINT(41.4705513381857 73.90981554005333) bank100198 +100199 POINT(41.47956574361774 74.4273164389728) bank100199 +100200 POINT(40.556037648371365 74.23068953126823) bank100200 +100201 POINT(41.33679608731234 73.16215497042586) bank100201 +100202 POINT(40.8051942576302 73.37063052527367) bank100202 +100203 POINT(39.84236252711862 73.43591649299952) bank100203 +100204 POINT(41.29803584140073 73.66761610943459) bank100204 +100205 POINT(40.03709448059176 74.44677147684554) bank100205 +100206 POINT(39.81483909994114 73.17337642297053) bank100206 +100207 POINT(41.08507576950819 74.91155079270217) bank100207 +100208 POINT(39.796391295520074 74.58063722520858) bank100208 +100209 POINT(39.894084722768056 74.34382662575825) bank100209 +100210 POINT(40.08367003553433 75.00420833876066) bank100210 +100211 POINT(39.752112340728715 73.6904540992731) bank100211 +100212 POINT(41.40123322289372 74.43560538044576) bank100212 +100213 POINT(40.67490058410107 73.41287465565583) bank100213 +100214 POINT(40.329556227513905 74.46333439506179) bank100214 +100215 POINT(40.538618303217355 74.43105025732496) bank100215 +100216 POINT(40.89105661397514 73.52231293895399) bank100216 +100217 POINT(40.248247998990465 73.72030592704975) bank100217 +100218 POINT(41.501437286458604 74.33298162550254) bank100218 +100219 POINT(41.31387319117315 74.1175176947363) bank100219 +100220 POINT(40.82691978846282 74.2467863801169) bank100220 +100221 POINT(41.58946406725114 74.90863738345057) bank100221 +100222 POINT(41.16074068615873 73.40982853393152) bank100222 +100223 POINT(40.40038465474192 73.49778254948421) bank100223 +100224 POINT(40.28636752162261 74.86265256365172) bank100224 +100225 POINT(41.600575546228136 74.31045521355794) bank100225 +100226 POINT(39.90114395889685 74.04174757982148) bank100226 +100227 POINT(40.13705594983776 73.32267287326835) bank100227 +100228 POINT(41.4622974131654 73.66461811906284) bank100228 +100229 POINT(40.777553819673344 74.6547723733912) bank100229 +100230 POINT(40.790865051505804 74.6515468927199) bank100230 +100231 POINT(40.76189949670517 73.86896360307838) bank100231 +100232 POINT(40.13911110123571 74.17534491103213) bank100232 +100233 POINT(40.07426408337731 73.14214356860641) bank100233 +100234 POINT(40.87895019872489 73.47193884353591) bank100234 +100235 POINT(41.047922351929536 74.91262181846933) bank100235 +100236 POINT(40.07659638569956 74.602440017642) bank100236 +100237 POINT(40.997453916696294 73.45197889737288) bank100237 +100238 POINT(40.63593655571977 73.01802856344902) bank100238 +100239 POINT(41.103672715435515 74.4965168365942) bank100239 +100240 POINT(41.63865956037805 74.10880732478913) bank100240 +100241 POINT(40.77165441457774 74.55383213781933) bank100241 +100242 POINT(40.05573808213773 73.87466238176877) bank100242 +100243 POINT(40.91575293674155 73.43651743962354) bank100243 +100244 POINT(41.65565400850939 74.20152383538462) bank100244 +100245 POINT(41.498896436519104 73.11744494199041) bank100245 +100246 POINT(41.25326118721693 74.93945673386324) bank100246 +100247 POINT(41.335805636697536 74.89597905475354) bank100247 +100248 POINT(41.324093061050036 73.3735526470435) bank100248 +100249 POINT(40.70750328956769 73.88767524645388) bank100249 +100250 POINT(40.340513980910316 74.52159249215049) bank100250 +100251 POINT(40.341811329909454 74.18831844030078) bank100251 +100252 POINT(39.77979858804183 74.46363736553751) bank100252 +100253 POINT(41.40859307642775 73.50264401037664) bank100253 +100254 POINT(41.51920985373097 74.73052044196741) bank100254 +100255 POINT(39.80517267137274 74.55141073081992) bank100255 +100256 POINT(40.517521178205335 73.86370276542047) bank100256 +100257 POINT(39.76891984215812 74.97696032261724) bank100257 +100258 POINT(41.14775194646964 73.10180522557353) bank100258 +100259 POINT(41.19499528960814 73.98583933272812) bank100259 +100260 POINT(39.988769242039695 73.57488423203425) bank100260 +100261 POINT(41.11275123689271 74.11268821725824) bank100261 +100262 POINT(41.67963242819209 73.75606033880743) bank100262 +100263 POINT(41.67749368213875 73.38697318874951) bank100263 +100264 POINT(39.804896365478896 73.29698968465038) bank100264 +100265 POINT(41.35552783529859 73.34114971768277) bank100265 +100266 POINT(40.36322689961396 73.2783710420123) bank100266 +100267 POINT(41.67846182700368 73.79042212755317) bank100267 +100268 POINT(40.90144123975872 73.27937608202855) bank100268 +100269 POINT(40.78032574227785 74.28304451928088) bank100269 +100270 POINT(40.101705825932896 73.26823292782237) bank100270 +100271 POINT(39.90935872099504 74.73792049190847) bank100271 +100272 POINT(40.817402830822296 74.34984833200731) bank100272 +100273 POINT(39.99455538598249 74.38244501255991) bank100273 +100274 POINT(41.01922508152828 74.54162276359492) bank100274 +100275 POINT(40.50088451181525 73.26678614476448) bank100275 +100276 POINT(40.044935058494936 74.79215983723437) bank100276 +100277 POINT(40.07200745183133 73.85547365111591) bank100277 +100278 POINT(41.130929811928645 74.69751620616789) bank100278 +100279 POINT(39.85290739922591 74.76869149965268) bank100279 +100280 POINT(41.102747917365434 73.80182252317695) bank100280 +100281 POINT(39.97796781279814 74.86312804513985) bank100281 +100282 POINT(40.543491633683615 73.27117715534105) bank100282 +100283 POINT(39.86768521679812 73.52595732015831) bank100283 +100284 POINT(40.619270008885906 74.60188163435684) bank100284 +100285 POINT(40.05565309547724 73.30078737866133) bank100285 +100286 POINT(41.455342276764476 73.10226693789315) bank100286 +100287 POINT(40.6983908878184 74.77186838431058) bank100287 +100288 POINT(41.0618739533599 74.0504522605549) bank100288 +100289 POINT(40.45210810457958 73.58773287319403) bank100289 +100290 POINT(41.20380730663113 73.883692893294) bank100290 +100291 POINT(40.34192650309171 74.52588202995598) bank100291 +100292 POINT(39.92657369251151 73.20463670415789) bank100292 +100293 POINT(40.318565474979046 73.59843853238775) bank100293 +100294 POINT(40.49864079448973 74.21347176782602) bank100294 +100295 POINT(40.36661066113837 73.95480084201715) bank100295 +100296 POINT(40.41964276275968 74.84873955674402) bank100296 +100297 POINT(41.655218803656 73.00997822540567) bank100297 +100298 POINT(40.115805665344254 73.33070106194882) bank100298 +100299 POINT(41.6513740051631 74.75234548272192) bank100299 +100300 POINT(41.201726358314424 74.23626449388415) bank100300 +100301 POINT(41.21729381682465 73.0566880396012) bank100301 +100302 POINT(40.78272184552842 74.06557427882034) bank100302 +100303 POINT(41.49960532179744 74.36952948932115) bank100303 +100304 POINT(40.365583908548146 74.04729683962759) bank100304 +100305 POINT(41.51746719643836 73.5176829881345) bank100305 +100306 POINT(39.950441569806905 74.33373065780812) bank100306 +100307 POINT(41.1943076371985 74.8237580089734) bank100307 +100308 POINT(41.405180163419715 73.08045233311964) bank100308 +100309 POINT(40.359847071907645 73.11545469369116) bank100309 +100310 POINT(40.38838839398372 73.24339546357594) bank100310 +100311 POINT(40.85991547532082 73.85233601619062) bank100311 +100312 POINT(39.87433575493272 74.23572096306768) bank100312 +100313 POINT(40.56129611889781 73.2893262351737) bank100313 +100314 POINT(40.21782654088048 73.63319726558164) bank100314 +100315 POINT(41.368822802274536 74.24692926830629) bank100315 +100316 POINT(40.91028675405915 74.2428522122752) bank100316 +100317 POINT(41.20874110481318 74.72414310893295) bank100317 +100318 POINT(41.2878771915308 74.92347974643027) bank100318 +100319 POINT(41.491388202645666 74.55160832524884) bank100319 +100320 POINT(41.17415185131883 73.4565464453498) bank100320 +100321 POINT(41.69294519238819 74.970206383745) bank100321 +100322 POINT(41.366615051272355 74.50507679790444) bank100322 +100323 POINT(40.9924374719906 73.25749518942415) bank100323 +100324 POINT(40.07176841913833 74.14464751728312) bank100324 +100325 POINT(40.42899067135862 73.68894317243767) bank100325 +100326 POINT(41.08814620168574 74.11874389400101) bank100326 +100327 POINT(41.462196062285436 73.98047929719118) bank100327 +100328 POINT(40.89224238366041 73.87505175957855) bank100328 +100329 POINT(41.53818474148877 74.4142220631497) bank100329 +100330 POINT(41.54056544528834 74.99217367439073) bank100330 +100331 POINT(40.78533435632087 73.90791900301542) bank100331 +100332 POINT(40.44311163273054 73.08888734705833) bank100332 +100333 POINT(41.43944360707502 73.1291047976946) bank100333 +100334 POINT(39.83756152630518 74.76095380419139) bank100334 +100335 POINT(41.19829467348444 74.99804574160694) bank100335 +100336 POINT(41.0948954143974 74.43745110593298) bank100336 +100337 POINT(41.60630838621916 74.39733378594661) bank100337 +100338 POINT(40.71819192837339 73.6719996918741) bank100338 +100339 POINT(40.08869057850955 74.1677389307206) bank100339 +100340 POINT(39.84570152069816 73.1691181524905) bank100340 +100341 POINT(41.29060685734279 74.53475743890422) bank100341 +100342 POINT(41.63250215639557 73.2655287347812) bank100342 +100343 POINT(40.512318149452774 73.37971827790344) bank100343 +100344 POINT(40.19145413411319 74.0440753354857) bank100344 +100345 POINT(39.84879680379878 74.156941586863) bank100345 +100346 POINT(39.9450207056957 74.00691046554861) bank100346 +100347 POINT(40.57134525748621 74.47968123676151) bank100347 +100348 POINT(41.54970367047541 73.06807787847026) bank100348 +100349 POINT(40.613341571711615 73.32540770525938) bank100349 +100350 POINT(40.730078614533 73.56609442065336) bank100350 +100351 POINT(40.30425636864816 74.72649492301106) bank100351 +100352 POINT(41.698618562892065 74.84868089604491) bank100352 +100353 POINT(40.232851169084206 73.14536497900276) bank100353 +100354 POINT(40.82177758748358 73.68412325912648) bank100354 +100355 POINT(40.28850067061767 73.82612978486921) bank100355 +100356 POINT(40.72695758820646 74.91892375232774) bank100356 +100357 POINT(41.39491614113128 74.71337068946795) bank100357 +100358 POINT(40.068739179888645 74.7246336902856) bank100358 +100359 POINT(39.94967709830588 74.05720904835223) bank100359 +100360 POINT(40.89305385306948 74.3193970274862) bank100360 +100361 POINT(39.91912969481717 73.43258780193844) bank100361 +100362 POINT(40.359231008891214 75.00162679433137) bank100362 +100363 POINT(41.038361668797606 74.72950668173256) bank100363 +100364 POINT(40.78286192265605 74.52036228361546) bank100364 +100365 POINT(40.32279441233385 74.30839393490547) bank100365 +100366 POINT(41.32265277287761 73.30001713327565) bank100366 +100367 POINT(40.67489358576906 73.80793573653767) bank100367 +100368 POINT(40.40210290019691 73.75423952313322) bank100368 +100369 POINT(40.4196303824204 73.70189595093204) bank100369 +100370 POINT(40.650059068464095 74.99335271200013) bank100370 +100371 POINT(40.85114569804933 73.51918715017062) bank100371 +100372 POINT(39.996884412451784 73.33662610310887) bank100372 +100373 POINT(40.20029263717057 73.75163869329133) bank100373 +100374 POINT(40.922362954060056 74.72571843585287) bank100374 +100375 POINT(41.61850264780799 73.18675112623272) bank100375 +100376 POINT(41.216389741095796 74.54403696372246) bank100376 +100377 POINT(41.53170912428957 74.56849810113006) bank100377 +100378 POINT(41.60311573914058 74.12932515286207) bank100378 +100379 POINT(40.8862082598573 73.71793387041124) bank100379 +100380 POINT(41.629695079570816 74.67129082116031) bank100380 +100381 POINT(41.269447067771615 74.64697274575838) bank100381 +100382 POINT(41.172329475495566 73.49900017448809) bank100382 +100383 POINT(40.73361689258513 74.22352626692701) bank100383 +100384 POINT(41.240080301015354 73.7466346772716) bank100384 +100385 POINT(41.61448098476642 74.9567496487273) bank100385 +100386 POINT(41.21588226901859 74.76890406555107) bank100386 +100387 POINT(40.190830670172744 73.4385534500015) bank100387 +100388 POINT(39.936580439976495 73.9450193092259) bank100388 +100389 POINT(41.01403935347028 74.51828832250128) bank100389 +100390 POINT(39.827082136861634 74.90312209965467) bank100390 +100391 POINT(41.01891778579722 74.15050014646809) bank100391 +100392 POINT(40.27131273823047 74.45389703397916) bank100392 +100393 POINT(41.566286851516836 74.15087453577826) bank100393 +100394 POINT(41.08669388806922 73.29862352755157) bank100394 +100395 POINT(40.065466439323416 73.26019836236003) bank100395 +100396 POINT(40.36065791817254 73.01035040950815) bank100396 +100397 POINT(41.63771832348075 74.85176450014708) bank100397 +100398 POINT(40.58719957047628 73.72332893865278) bank100398 +100399 POINT(41.27667669875426 73.91038429850492) bank100399 +100400 POINT(41.553349642049525 74.24571269155112) bank100400 +100401 POINT(41.08327756402575 73.10020008803467) bank100401 +100402 POINT(41.205364199317856 75.00161223360813) bank100402 +100403 POINT(41.16359479994298 73.88845275062816) bank100403 +100404 POINT(41.33632625019516 73.60572531538833) bank100404 +100405 POINT(40.193622047593 73.13994723058275) bank100405 +100406 POINT(41.11443643557351 73.39323404165039) bank100406 +100407 POINT(40.4252546412104 74.83480057472045) bank100407 +100408 POINT(40.76402006700022 73.88983932496711) bank100408 +100409 POINT(40.60118713184098 73.08673722417718) bank100409 +100410 POINT(40.27751863666326 74.85841013650703) bank100410 +100411 POINT(41.008600691634946 73.95877268688245) bank100411 +100412 POINT(41.381708832613164 73.42594199598197) bank100412 +100413 POINT(41.2118782014414 74.19087981964724) bank100413 +100414 POINT(39.79472303319673 74.46709385755817) bank100414 +100415 POINT(39.85339625587082 73.45475037484121) bank100415 +100416 POINT(40.985906133294606 74.00010925961352) bank100416 +100417 POINT(41.584990000420525 74.22659041529285) bank100417 +100418 POINT(40.56271866162083 74.44728274637494) bank100418 +100419 POINT(40.78512957812442 74.5517553638273) bank100419 +100420 POINT(41.64506200111186 74.25171305191438) bank100420 +100421 POINT(40.7157732163234 73.34406841341922) bank100421 +100422 POINT(40.22833629276187 74.65828003084185) bank100422 +100423 POINT(41.357470662139015 74.71079125595787) bank100423 +100424 POINT(40.52687847345437 73.19554539697606) bank100424 +100425 POINT(39.83117442222122 74.46491299468651) bank100425 +100426 POINT(40.21746573315084 73.01888700619321) bank100426 +100427 POINT(39.81896896249741 74.40389231258013) bank100427 +100428 POINT(40.17087715609112 73.43870456067566) bank100428 +100429 POINT(39.76189454869378 74.93325200236819) bank100429 +100430 POINT(39.78543962417816 73.76448456118786) bank100430 +100431 POINT(39.92356023385674 74.16064800522764) bank100431 +100432 POINT(40.782309137609374 73.56167445265761) bank100432 +100433 POINT(40.771862272027334 73.40903474863134) bank100433 +100434 POINT(40.419274843159684 73.4968955116283) bank100434 +100435 POINT(41.289209360504145 74.22477173024151) bank100435 +100436 POINT(40.1584498927399 73.65951976978411) bank100436 +100437 POINT(40.4868022308522 73.72237892333708) bank100437 +100438 POINT(40.025978713253714 73.89600325632965) bank100438 +100439 POINT(40.58146568907897 74.82820278140963) bank100439 +100440 POINT(41.59684563075184 73.31091439038103) bank100440 +100441 POINT(40.795375292952905 73.3869713514002) bank100441 +100442 POINT(41.7025049761712 73.10876634132971) bank100442 +100443 POINT(40.52312654070341 73.3873103020838) bank100443 +100444 POINT(41.00903168374109 74.64817461212415) bank100444 +100445 POINT(40.261820233660615 74.01290763408524) bank100445 +100446 POINT(39.91297705988809 74.86470275504135) bank100446 +100447 POINT(40.97967330760579 73.79170512968804) bank100447 +100448 POINT(41.0328420058547 74.98649156184666) bank100448 +100449 POINT(39.85352799275913 74.54560365677659) bank100449 +100450 POINT(41.27602304847128 74.0743941289272) bank100450 +100451 POINT(40.85502132633118 73.58629480225997) bank100451 +100452 POINT(40.77988505294628 73.40649705521213) bank100452 +100453 POINT(40.77295951302357 73.41364086827805) bank100453 +100454 POINT(40.11162824729051 74.09206084764475) bank100454 +100455 POINT(40.23366976147336 73.35971967310185) bank100455 +100456 POINT(40.954786983295875 73.53421960372322) bank100456 +100457 POINT(40.54306698040137 73.69662769215944) bank100457 +100458 POINT(41.4929556826877 73.79806974877785) bank100458 +100459 POINT(40.308165014976616 73.58323606566111) bank100459 +100460 POINT(40.980666092730225 74.21559961487932) bank100460 +100461 POINT(40.76044525415196 74.01016811654459) bank100461 +100462 POINT(40.13283922790724 74.88970223065921) bank100462 +100463 POINT(41.45486313780963 74.71193626251228) bank100463 +100464 POINT(40.13214515493523 74.05919720426779) bank100464 +100465 POINT(41.251133001216736 73.87083761230194) bank100465 +100466 POINT(39.9160692384228 73.17377415181745) bank100466 +100467 POINT(40.939746152384565 73.62997688227085) bank100467 +100468 POINT(41.04335448185021 74.94896141083191) bank100468 +100469 POINT(40.960174494014105 74.79643135962004) bank100469 +100470 POINT(40.39336668027352 74.55902765066315) bank100470 +100471 POINT(39.801016117272965 73.69265980015008) bank100471 +100472 POINT(41.31605133246002 73.85083345301935) bank100472 +100473 POINT(40.70858480711708 73.97022959635022) bank100473 +100474 POINT(40.1291893206715 73.95504096429462) bank100474 +100475 POINT(40.308214553057105 73.49993782777595) bank100475 +100476 POINT(40.770880168387094 73.59155919117266) bank100476 +100477 POINT(39.75072437242976 74.77279795147203) bank100477 +100478 POINT(40.501824925528844 74.44920651853509) bank100478 +100479 POINT(39.971857457715814 73.63210825368097) bank100479 +100480 POINT(41.67843489483736 73.4728597274673) bank100480 +100481 POINT(41.07339624708131 74.6145907772332) bank100481 +100482 POINT(39.965761390079685 73.67868759418067) bank100482 +100483 POINT(40.66725908316377 73.76222729525162) bank100483 +100484 POINT(39.822136235847005 73.42276667141404) bank100484 +100485 POINT(39.932841047131674 74.23156126226071) bank100485 +100486 POINT(40.07292919831479 74.88282700867705) bank100486 +100487 POINT(40.60691246470357 73.46416493617802) bank100487 +100488 POINT(40.24745445441274 73.78268503676672) bank100488 +100489 POINT(40.63075874703878 74.49691839146816) bank100489 +100490 POINT(40.17607220786115 74.8376760936728) bank100490 +100491 POINT(41.05771821931049 73.93760303860726) bank100491 +100492 POINT(40.26048503675771 74.1506982084693) bank100492 +100493 POINT(41.549257809979096 74.03142820972008) bank100493 +100494 POINT(40.83782124600877 73.93749220473185) bank100494 +100495 POINT(41.599486846067954 73.39290410054961) bank100495 +100496 POINT(40.80783445512703 73.75842618673084) bank100496 +100497 POINT(40.559932293671075 74.99329620648963) bank100497 +100498 POINT(41.03823278855055 73.33365754896276) bank100498 +100499 POINT(40.695647162400945 74.72254920690374) bank100499 +100500 POINT(39.82234706556273 74.3181160401325) bank100500 +100501 POINT(41.101743148347474 74.17717625041934) bank100501 +100502 POINT(41.55193836499888 73.38377923861901) bank100502 +100503 POINT(40.331030460010474 73.47936581963485) bank100503 +100504 POINT(39.91381508318601 74.46420878075104) bank100504 +100505 POINT(40.95237171417586 73.21428212100717) bank100505 +100506 POINT(40.42673924611751 74.88115604996511) bank100506 +100507 POINT(39.84837451685607 73.09025801518476) bank100507 +100508 POINT(40.27584504471226 73.61782972532284) bank100508 +100509 POINT(40.05425072743491 74.18835925437973) bank100509 +100510 POINT(40.445385635546295 73.40707784096676) bank100510 +100511 POINT(40.739294331700265 75.00081791273756) bank100511 +100512 POINT(40.74800888003484 73.58449514795204) bank100512 +100513 POINT(41.52070967663833 73.7440106896946) bank100513 +100514 POINT(40.84969275776513 73.31641729125198) bank100514 +100515 POINT(39.7151075313461 73.20540786945519) bank100515 +100516 POINT(41.17174643914963 73.18931363482213) bank100516 +100517 POINT(39.98917179065003 74.57458300932753) bank100517 +100518 POINT(41.63431257307534 74.53757094457158) bank100518 +100519 POINT(41.29992859271898 73.56290579731727) bank100519 +100520 POINT(40.73888277502868 73.76877054184233) bank100520 +100521 POINT(41.08307308385763 73.50349752271148) bank100521 +100522 POINT(40.572640157803875 74.8067221723638) bank100522 +100523 POINT(40.96136484425277 74.39820914548574) bank100523 +100524 POINT(40.10918340312104 74.03944186383954) bank100524 +100525 POINT(40.17871098881062 74.3725987501016) bank100525 +100526 POINT(40.67884389051063 74.82196518260633) bank100526 +100527 POINT(40.78131657598848 74.96355196855892) bank100527 +100528 POINT(41.68585808243975 74.21217282254287) bank100528 +100529 POINT(40.5937784206825 74.04445930410128) bank100529 +100530 POINT(41.28525367827955 73.84129379932314) bank100530 +100531 POINT(40.87301341557829 74.95959399018717) bank100531 +100532 POINT(41.58378682833937 74.1829393029184) bank100532 +100533 POINT(40.668641956051964 73.54258130343864) bank100533 +100534 POINT(41.309822455763346 73.7086662881815) bank100534 +100535 POINT(40.658750064874184 74.75958199963794) bank100535 +100536 POINT(40.09310013938545 73.75681825204089) bank100536 +100537 POINT(40.67380498657116 74.21586572662585) bank100537 +100538 POINT(40.965599011548626 74.29429211505831) bank100538 +100539 POINT(41.40813912203686 74.54954801054474) bank100539 +100540 POINT(41.637025290577306 74.0820338929628) bank100540 +100541 POINT(40.098824311259804 73.11072366970733) bank100541 +100542 POINT(41.433127908239776 73.02947016553578) bank100542 +100543 POINT(41.21267316581947 74.2031176082355) bank100543 +100544 POINT(40.507463547898176 73.4431310365524) bank100544 +100545 POINT(40.55231371090104 73.78496225334062) bank100545 +100546 POINT(40.25461531620555 73.55829911295797) bank100546 +100547 POINT(40.82287100067865 74.70965490788693) bank100547 +100548 POINT(40.30241988777999 73.19059615452949) bank100548 +100549 POINT(41.46674530260029 74.34158532779745) bank100549 +100550 POINT(40.098758256102386 74.80113182402089) bank100550 +100551 POINT(39.74579731640569 73.78896278269832) bank100551 +100552 POINT(40.561380487601355 73.00643706310956) bank100552 +100553 POINT(39.8986241886816 74.22026985045672) bank100553 +100554 POINT(41.653591870030894 73.7188670389233) bank100554 +100555 POINT(40.18849766085428 74.52585050745019) bank100555 +100556 POINT(40.1394610248511 74.2710463477588) bank100556 +100557 POINT(40.578698113589105 73.46398897059537) bank100557 +100558 POINT(41.001121427058386 74.16912670162873) bank100558 +100559 POINT(40.701556326399704 74.05433866769744) bank100559 +100560 POINT(39.91959900561983 73.83042532293045) bank100560 +100561 POINT(40.31238769892135 73.08666320792774) bank100561 +100562 POINT(41.399155832885356 73.05518986511592) bank100562 +100563 POINT(40.83630261301519 74.09249223201229) bank100563 +100564 POINT(40.13483961727552 73.2686298310024) bank100564 +100565 POINT(41.397446227650896 74.17483558047594) bank100565 +100566 POINT(41.50419990907789 73.52548012532482) bank100566 +100567 POINT(40.23276153034941 73.46234741540664) bank100567 +100568 POINT(41.19687170499122 73.24593534235169) bank100568 +100569 POINT(41.36234749776742 73.50770886177273) bank100569 +100570 POINT(41.65450512456843 74.48039551148753) bank100570 +100571 POINT(41.213893313958195 74.97463654041094) bank100571 +100572 POINT(39.92061182698098 74.75697565940615) bank100572 +100573 POINT(40.18961346212932 74.64750665146174) bank100573 +100574 POINT(40.31967852915111 74.41855753976769) bank100574 +100575 POINT(40.540325209422036 73.43187078070972) bank100575 +100576 POINT(39.835565665997805 73.73737470480098) bank100576 +100577 POINT(41.654683005338 74.49686460223319) bank100577 +100578 POINT(40.42214492870849 73.69552985566642) bank100578 +100579 POINT(39.73120561829865 73.68836305584568) bank100579 +100580 POINT(40.357465784130326 74.08325919066336) bank100580 +100581 POINT(40.23887554472717 74.9844059542199) bank100581 +100582 POINT(40.66422915041416 73.39932273662053) bank100582 +100583 POINT(40.164462538734476 74.95647361964646) bank100583 +100584 POINT(40.347150515671885 74.63080024936778) bank100584 +100585 POINT(40.93948701632412 74.17881072842306) bank100585 +100586 POINT(40.11507092118978 74.95738472823035) bank100586 +100587 POINT(39.775863208629126 73.38353152631652) bank100587 +100588 POINT(41.65727268780837 73.63553054608482) bank100588 +100589 POINT(39.73614417838249 73.4699850135864) bank100589 +100590 POINT(40.93544158809115 74.55739397969133) bank100590 +100591 POINT(40.90141481961314 73.99603327459567) bank100591 +100592 POINT(40.574180138300676 74.54754169259375) bank100592 +100593 POINT(40.160220589236005 74.81319513382235) bank100593 +100594 POINT(40.741361085450556 74.60289571026563) bank100594 +100595 POINT(41.37087158246554 73.33637842354814) bank100595 +100596 POINT(40.98089412426735 73.6705005307924) bank100596 +100597 POINT(41.52037249358587 73.08080819307452) bank100597 +100598 POINT(41.375481082790266 73.07334266925427) bank100598 +100599 POINT(41.455975291219886 73.41194228995441) bank100599 +100600 POINT(41.62683663718435 74.97009957819658) bank100600 +100601 POINT(39.719581463822706 73.37028420148303) bank100601 +100602 POINT(41.560624437340564 73.93130578543001) bank100602 +100603 POINT(41.00417931780134 73.35212869001592) bank100603 +100604 POINT(40.726960109725056 73.96802383562792) bank100604 +100605 POINT(41.17184269563879 74.22547124716374) bank100605 +100606 POINT(41.68804838968379 74.11691163285693) bank100606 +100607 POINT(40.55957448440453 74.72765785323425) bank100607 +100608 POINT(39.86036200158888 73.56375509197105) bank100608 +100609 POINT(40.48573008447587 74.30529109319092) bank100609 +100610 POINT(41.313746421006236 74.52472978526467) bank100610 +100611 POINT(41.23331968175542 74.14510293164673) bank100611 +100612 POINT(40.243316645858975 73.10601295894148) bank100612 +100613 POINT(40.24356053245943 73.30062088905512) bank100613 +100614 POINT(39.75627477272416 74.54697436432467) bank100614 +100615 POINT(40.10891635886342 73.27532006651819) bank100615 +100616 POINT(40.21657127667353 73.8602372849215) bank100616 +100617 POINT(41.53585374807162 73.61521533488366) bank100617 +100618 POINT(40.37694442377001 73.9763364161144) bank100618 +100619 POINT(41.203007199625766 73.93149074041564) bank100619 +100620 POINT(40.48655928087536 73.86464485815313) bank100620 +100621 POINT(40.02754351457803 74.24891303002143) bank100621 +100622 POINT(41.19503746439304 74.65173598252362) bank100622 +100623 POINT(41.64568114267542 74.27454355484024) bank100623 +100624 POINT(41.15225811548898 73.02049239487347) bank100624 +100625 POINT(41.69646999633163 74.65754400787384) bank100625 +100626 POINT(41.2090584591457 73.16441731518626) bank100626 +100627 POINT(41.699652863341456 74.11043854421035) bank100627 +100628 POINT(40.568615678802836 74.8469843124373) bank100628 +100629 POINT(40.076133522383046 73.92626439984889) bank100629 +100630 POINT(40.967690733549055 73.23658644854008) bank100630 +100631 POINT(40.8075532112649 74.99486594884104) bank100631 +100632 POINT(41.14522271602954 74.10420814617251) bank100632 +100633 POINT(40.35992282630992 73.10889909020716) bank100633 +100634 POINT(40.940407056139826 73.94801565178379) bank100634 +100635 POINT(40.136835881511224 74.13528468503652) bank100635 +100636 POINT(40.31759410147224 73.02793043887424) bank100636 +100637 POINT(40.375707825871054 73.74971103547121) bank100637 +100638 POINT(41.06787539635394 73.92192984680212) bank100638 +100639 POINT(41.04962603242817 74.39336908774601) bank100639 +100640 POINT(40.46749095441467 73.60947303506113) bank100640 +100641 POINT(39.7776821416723 73.49910471303714) bank100641 +100642 POINT(41.57688068919158 73.36895542448724) bank100642 +100643 POINT(40.63648348351297 73.59804733292432) bank100643 +100644 POINT(39.97928332349029 73.69914160494167) bank100644 +100645 POINT(40.87078649906279 73.90069378875073) bank100645 +100646 POINT(40.94179305812234 73.11568626677881) bank100646 +100647 POINT(39.829376105453576 74.96373419625583) bank100647 +100648 POINT(41.20605495763201 73.79296115147679) bank100648 +100649 POINT(40.46238698219143 73.96264275500974) bank100649 +100650 POINT(40.41443907804044 73.02877274448086) bank100650 +100651 POINT(40.27592385574508 74.19382073076066) bank100651 +100652 POINT(39.896698419363005 74.09121897949545) bank100652 +100653 POINT(41.4657202799822 73.66676901748541) bank100653 +100654 POINT(41.59523860209805 73.09036026415286) bank100654 +100655 POINT(41.67441317984449 74.03387339373835) bank100655 +100656 POINT(41.5328017921726 73.67516608548765) bank100656 +100657 POINT(40.22982117969978 73.27023645100014) bank100657 +100658 POINT(41.44652449566839 73.88283931560669) bank100658 +100659 POINT(41.22847408340324 74.32305288806474) bank100659 +100660 POINT(40.80859079357373 74.48880134811576) bank100660 +100661 POINT(41.604143900482235 73.02302751586096) bank100661 +100662 POINT(40.71171220317628 74.54133159186536) bank100662 +100663 POINT(40.223931890458424 74.05728931088487) bank100663 +100664 POINT(40.923923648341585 74.26259877960906) bank100664 +100665 POINT(40.332344463466136 73.8346792883146) bank100665 +100666 POINT(40.70693058402099 73.85732985070892) bank100666 +100667 POINT(40.85274221555332 74.17164058208763) bank100667 +100668 POINT(41.61512790949441 74.97100349983307) bank100668 +100669 POINT(41.48338804787589 73.45905819487288) bank100669 +100670 POINT(40.21311878813855 74.51002502414237) bank100670 +100671 POINT(40.703888237800285 73.94902798238978) bank100671 +100672 POINT(40.620725451976746 74.1748442340388) bank100672 +100673 POINT(40.27498954946225 74.63436796533404) bank100673 +100674 POINT(41.43815921326902 74.78207558905369) bank100674 +100675 POINT(41.571927498263385 73.79664466738019) bank100675 +100676 POINT(40.70686056187214 74.0339275450382) bank100676 +100677 POINT(40.405350188004135 74.5925462068865) bank100677 +100678 POINT(40.11066614911457 73.07156592388631) bank100678 +100679 POINT(41.23336365949183 73.57194541765185) bank100679 +100680 POINT(39.824131855963486 74.98111254679911) bank100680 +100681 POINT(40.63042661090713 73.92488269403539) bank100681 +100682 POINT(40.110081322302904 73.05909005067711) bank100682 +100683 POINT(40.45209539000948 74.10239392002913) bank100683 +100684 POINT(41.281590242126164 74.65534912607515) bank100684 +100685 POINT(40.14064546129927 73.91911455334699) bank100685 +100686 POINT(40.14602691847844 74.67030103175945) bank100686 +100687 POINT(39.974037820083474 73.61538522756557) bank100687 +100688 POINT(39.9509844423864 74.88069137261309) bank100688 +100689 POINT(40.55255556824222 74.74549560248859) bank100689 +100690 POINT(41.431481631515986 73.04874505735141) bank100690 +100691 POINT(41.33793936352025 74.47623117515191) bank100691 +100692 POINT(41.44837842232194 74.72999110178749) bank100692 +100693 POINT(40.42983283400842 73.15406710280921) bank100693 +100694 POINT(41.24258022653394 74.39752802508423) bank100694 +100695 POINT(40.829974460600084 74.7732774560029) bank100695 +100696 POINT(41.272013396300316 73.00692347294253) bank100696 +100697 POINT(41.130937613036295 73.76467010206123) bank100697 +100698 POINT(41.44137756400361 74.94831533226716) bank100698 +100699 POINT(40.56916799243681 74.53878595875095) bank100699 +100700 POINT(40.74235235516801 73.79374245080965) bank100700 +100701 POINT(40.45933785808928 73.80729117092345) bank100701 +100702 POINT(41.10477203942627 74.47939488572027) bank100702 +100703 POINT(40.76611642471224 73.53952926024387) bank100703 +100704 POINT(40.21261902947915 73.45189937265911) bank100704 +100705 POINT(40.24741990325414 73.05432625680956) bank100705 +100706 POINT(40.906778466116734 73.64005113439602) bank100706 +100707 POINT(40.336522801791254 74.31107551612496) bank100707 +100708 POINT(39.805590834159915 73.22667182839082) bank100708 +100709 POINT(41.00917857084429 74.58007013932462) bank100709 +100710 POINT(39.7493546135871 74.70329100484881) bank100710 +100711 POINT(39.953429485718374 74.51608267346042) bank100711 +100712 POINT(41.38298562923897 73.03930430444612) bank100712 +100713 POINT(41.03452777513466 74.24704993368358) bank100713 +100714 POINT(40.004933998258075 73.24893578130747) bank100714 +100715 POINT(40.78965804637459 73.74732026840792) bank100715 +100716 POINT(41.299942767025705 74.2763796832561) bank100716 +100717 POINT(41.246532487953544 73.12134745657475) bank100717 +100718 POINT(41.683330673998086 73.01550546361219) bank100718 +100719 POINT(40.490733639428086 73.51808781702194) bank100719 +100720 POINT(40.22051555145927 73.45305080248816) bank100720 +100721 POINT(41.25517116728611 74.58409576013125) bank100721 +100722 POINT(40.70045443301989 73.59041585865577) bank100722 +100723 POINT(40.37570295903465 74.4010705901324) bank100723 +100724 POINT(40.75238247704861 74.08161604962464) bank100724 +100725 POINT(40.00993893451068 74.81696890722183) bank100725 +100726 POINT(41.39088560046146 73.62103238717978) bank100726 +100727 POINT(40.7655856921897 73.75105632160968) bank100727 +100728 POINT(41.65613011395243 73.89303404088852) bank100728 +100729 POINT(39.74949084971929 74.1869740402322) bank100729 +100730 POINT(40.70149339249904 73.81598691612524) bank100730 +100731 POINT(40.094866746067915 73.79873748574666) bank100731 +100732 POINT(40.7686767560101 73.45003496524797) bank100732 +100733 POINT(40.91373401254369 73.86997638996353) bank100733 +100734 POINT(40.42392633135089 74.21614542106637) bank100734 +100735 POINT(40.80234355151263 73.79291596976823) bank100735 +100736 POINT(40.230387098434946 74.8757069517317) bank100736 +100737 POINT(40.78075092271164 74.30287081931165) bank100737 +100738 POINT(39.83702644744335 74.378474853479) bank100738 +100739 POINT(40.27038438512522 73.13000221997844) bank100739 +100740 POINT(41.100464800074924 74.60433099247258) bank100740 +100741 POINT(41.0154723599347 74.18647055209986) bank100741 +100742 POINT(40.02993605221284 73.15796117721005) bank100742 +100743 POINT(39.71786764619444 74.48639076804943) bank100743 +100744 POINT(40.928891843466175 74.25217216184737) bank100744 +100745 POINT(40.63438165376506 74.69371613397897) bank100745 +100746 POINT(41.12351034222157 73.85471715104445) bank100746 +100747 POINT(40.80406508919053 74.60615790568541) bank100747 +100748 POINT(41.634031146885306 73.29564122203877) bank100748 +100749 POINT(40.455821701425585 74.38241310545885) bank100749 +100750 POINT(41.277515056902956 73.86439792872555) bank100750 +100751 POINT(40.665958098077645 74.89259266549503) bank100751 +100752 POINT(40.79163493942753 74.55102010262038) bank100752 +100753 POINT(40.20459304096808 74.3980257117527) bank100753 +100754 POINT(41.558250758842824 74.57530622733273) bank100754 +100755 POINT(41.06366238563435 73.91609372437341) bank100755 +100756 POINT(41.55654438343295 73.42268801764851) bank100756 +100757 POINT(40.270124473229124 74.97123443062705) bank100757 +100758 POINT(40.56545114415584 73.23811438313697) bank100758 +100759 POINT(41.239779818933066 73.01512763880652) bank100759 +100760 POINT(41.579803966514994 74.01336515735174) bank100760 +100761 POINT(41.37734233812471 73.6650557036976) bank100761 +100762 POINT(40.43175594795067 73.03270079881207) bank100762 +100763 POINT(40.33151693729889 74.828646732003) bank100763 +100764 POINT(41.18479365189874 74.42103914860265) bank100764 +100765 POINT(41.20184835648467 73.24838019168875) bank100765 +100766 POINT(41.0352160177733 74.06329063106317) bank100766 +100767 POINT(40.828928191556365 74.079385654825) bank100767 +100768 POINT(40.95994877005528 74.04380977599438) bank100768 +100769 POINT(41.087098917755725 73.06263188019587) bank100769 +100770 POINT(40.52647595611714 73.37676252232401) bank100770 +100771 POINT(39.83194752781852 73.36241789443615) bank100771 +100772 POINT(41.12959918866446 74.60350700124664) bank100772 +100773 POINT(40.93012690161569 74.52384397534611) bank100773 +100774 POINT(40.008038003646014 73.50971594161187) bank100774 +100775 POINT(40.43037346959964 74.35480824567988) bank100775 +100776 POINT(41.45201061547008 73.34841812289527) bank100776 +100777 POINT(41.430491837380835 73.70834761047041) bank100777 +100778 POINT(41.027262741750576 73.85012478774077) bank100778 +100779 POINT(41.33743692285016 73.69870472030908) bank100779 +100780 POINT(40.855209379264636 73.65577727222578) bank100780 +100781 POINT(41.461732933539146 74.45635135954394) bank100781 +100782 POINT(41.69736000902338 73.6299756167133) bank100782 +100783 POINT(40.59775719854082 73.93412174838662) bank100783 +100784 POINT(40.06089684163106 73.53233787158612) bank100784 +100785 POINT(41.14203556000341 73.68114578230535) bank100785 +100786 POINT(40.360322323548836 73.77217864435386) bank100786 +100787 POINT(41.286524572292926 73.12700030605075) bank100787 +100788 POINT(40.178925014811355 74.99215399338931) bank100788 +100789 POINT(40.96233446168494 74.61579353346661) bank100789 +100790 POINT(39.808219947779925 73.12317472100308) bank100790 +100791 POINT(41.27910473348584 73.38772145348166) bank100791 +100792 POINT(39.90684590660945 74.70430330573507) bank100792 +100793 POINT(40.736322744581216 74.31924421064944) bank100793 +100794 POINT(39.878151811225955 73.27372076980734) bank100794 +100795 POINT(41.543728615742886 73.78130666320152) bank100795 +100796 POINT(41.448529521742 73.4040176264215) bank100796 +100797 POINT(40.595645896164434 74.70990251123501) bank100797 +100798 POINT(39.91175425685933 74.97205783880109) bank100798 +100799 POINT(39.90262464702353 74.48455187286451) bank100799 +100800 POINT(40.393249669819774 74.9327622381692) bank100800 +100801 POINT(40.47487757679675 74.18564244092632) bank100801 +100802 POINT(39.967431992481806 74.10859210869117) bank100802 +100803 POINT(41.20378057864093 74.87886359384044) bank100803 +100804 POINT(41.131387836960286 74.43272070533443) bank100804 +100805 POINT(40.03778748116365 73.64365429896748) bank100805 +100806 POINT(41.61246506423888 73.77875187356945) bank100806 +100807 POINT(40.650790741613335 73.23797727745902) bank100807 +100808 POINT(40.02686448499482 74.86963780945679) bank100808 +100809 POINT(40.14840512352624 73.60813741866022) bank100809 +100810 POINT(40.03399680921557 74.62334498373849) bank100810 +100811 POINT(41.52128718564552 73.19872740710824) bank100811 +100812 POINT(41.18457218643599 74.08428259892928) bank100812 +100813 POINT(41.648850761817286 74.29890692565219) bank100813 +100814 POINT(40.921513475255914 74.56238729935615) bank100814 +100815 POINT(40.880779139684506 74.78758816333949) bank100815 +100816 POINT(41.52395139865389 73.9799976690087) bank100816 +100817 POINT(41.50037253412555 73.1507914421547) bank100817 +100818 POINT(40.08686091840293 73.39181459326672) bank100818 +100819 POINT(41.37688889078928 73.30681474601579) bank100819 +100820 POINT(40.145433359442556 74.16800817811838) bank100820 +100821 POINT(40.716838485329255 74.32073389571606) bank100821 +100822 POINT(41.05596117251793 74.35428587946164) bank100822 +100823 POINT(40.9377658645036 73.8023385836359) bank100823 +100824 POINT(40.60816011318182 74.7632843712804) bank100824 +100825 POINT(41.70960984950978 73.8272574283498) bank100825 +100826 POINT(39.74956471460819 73.44396937554109) bank100826 +100827 POINT(41.64163232090351 74.81614186726378) bank100827 +100828 POINT(40.6548529426564 73.69314291842674) bank100828 +100829 POINT(40.27846812857404 73.01038759531343) bank100829 +100830 POINT(40.36671876069636 74.0660622888536) bank100830 +100831 POINT(40.20426507970445 74.16720774954527) bank100831 +100832 POINT(40.98654355728165 73.2158248762225) bank100832 +100833 POINT(40.850381910838536 74.0621268469666) bank100833 +100834 POINT(40.14708779716266 74.24131711774) bank100834 +100835 POINT(40.641147513623764 74.12902126781971) bank100835 +100836 POINT(41.2999247963466 74.29726923889298) bank100836 +100837 POINT(39.74021754886832 74.75572679602948) bank100837 +100838 POINT(40.13904251384358 73.39485901419471) bank100838 +100839 POINT(40.18272952593004 74.50977674756729) bank100839 +100840 POINT(41.623122193729344 74.50128554998858) bank100840 +100841 POINT(41.14354511836153 74.24687784827354) bank100841 +100842 POINT(41.59522714735552 74.82268368148837) bank100842 +100843 POINT(40.91917482928899 74.23442086177697) bank100843 +100844 POINT(41.25401941919337 73.69965175501554) bank100844 +100845 POINT(40.70108299844196 73.81754020032011) bank100845 +100846 POINT(40.92044134137242 73.36670069233175) bank100846 +100847 POINT(40.42091058776657 73.2696818831964) bank100847 +100848 POINT(41.38674925718574 73.30648750941843) bank100848 +100849 POINT(41.380347942933376 73.6011450825235) bank100849 +100850 POINT(40.02000870262506 73.53933301943701) bank100850 +100851 POINT(40.86470044038215 74.46883335489377) bank100851 +100852 POINT(41.14502156702858 73.91753667807063) bank100852 +100853 POINT(40.984354637103436 74.02150185827755) bank100853 +100854 POINT(40.97790630911914 73.800340896811) bank100854 +100855 POINT(41.110890702351575 74.39620494482648) bank100855 +100856 POINT(41.21180658657115 74.92938311544353) bank100856 +100857 POINT(41.53716149285683 74.59908519821215) bank100857 +100858 POINT(39.99064289532775 74.97599024392476) bank100858 +100859 POINT(40.593185885935306 73.92752172387651) bank100859 +100860 POINT(40.51537819087702 74.25391910704906) bank100860 +100861 POINT(41.563660416647394 73.52349462277263) bank100861 +100862 POINT(40.05427886886144 74.66725943438954) bank100862 +100863 POINT(39.73401398710473 73.03938354385755) bank100863 +100864 POINT(39.8936391883999 73.74736667416845) bank100864 +100865 POINT(41.34145503139086 74.67878692556357) bank100865 +100866 POINT(39.76184981734337 74.23986603110637) bank100866 +100867 POINT(40.92450029669944 73.8026226023156) bank100867 +100868 POINT(40.005515879284424 74.95716262200116) bank100868 +100869 POINT(40.66351984896784 74.66282840912652) bank100869 +100870 POINT(39.972284597102146 74.62857081912273) bank100870 +100871 POINT(40.53442710012051 74.41991288972912) bank100871 +100872 POINT(41.60218003217475 74.60891403507591) bank100872 +100873 POINT(40.994339194314634 74.32691065791812) bank100873 +100874 POINT(40.95500035302179 74.18224701358785) bank100874 +100875 POINT(40.1042634124258 74.74513801605352) bank100875 +100876 POINT(40.70151754226798 74.79641892144858) bank100876 +100877 POINT(41.63654177951868 74.39409588908926) bank100877 +100878 POINT(41.64580152120159 74.80530766767157) bank100878 +100879 POINT(41.68991368774515 74.33963152698098) bank100879 +100880 POINT(41.39831153885284 74.9568316013262) bank100880 +100881 POINT(39.774320868379576 73.42127141349279) bank100881 +100882 POINT(40.72304988933062 74.71811648092613) bank100882 +100883 POINT(40.54463796596553 73.97874503957952) bank100883 +100884 POINT(40.67196263231318 73.7107401317418) bank100884 +100885 POINT(41.6687557391083 74.5537592434465) bank100885 +100886 POINT(40.026722694516245 74.38195732723293) bank100886 +100887 POINT(40.652350234866084 73.11217751520377) bank100887 +100888 POINT(39.98976624376566 74.8401361329944) bank100888 +100889 POINT(40.43627964514147 73.76123221363217) bank100889 +100890 POINT(39.938941196273056 74.0481173871682) bank100890 +100891 POINT(41.634418078649276 74.32048642660978) bank100891 +100892 POINT(40.85715106973775 74.76772742171407) bank100892 +100893 POINT(41.05118464603287 74.13424214957104) bank100893 +100894 POINT(41.65465798774711 74.32829702559512) bank100894 +100895 POINT(41.00721146128217 73.61838621741066) bank100895 +100896 POINT(40.75523037415415 74.62709068782304) bank100896 +100897 POINT(41.647850779595984 73.81789769849095) bank100897 +100898 POINT(40.14450770746031 73.2160532390745) bank100898 +100899 POINT(41.240971622938844 73.4623251718508) bank100899 +100900 POINT(41.13619670853549 74.49138734758829) bank100900 +100901 POINT(41.5991873238459 73.70472314896212) bank100901 +100902 POINT(41.48256999355883 74.36891748046872) bank100902 +100903 POINT(39.7995494605738 73.18154746063587) bank100903 +100904 POINT(40.032688594518866 73.68760426512402) bank100904 +100905 POINT(39.90184860485515 74.92927501695743) bank100905 +100906 POINT(40.308621564247105 73.80884958660384) bank100906 +100907 POINT(41.66498829495372 73.20064427297072) bank100907 +100908 POINT(41.65571288624965 73.71918188787296) bank100908 +100909 POINT(40.00575908412087 73.581629738602) bank100909 +100910 POINT(40.77396105760965 74.71978923690834) bank100910 +100911 POINT(41.69069961621608 73.99454290987757) bank100911 +100912 POINT(40.74718538541526 74.60476547380875) bank100912 +100913 POINT(40.94986311139758 73.88914123452867) bank100913 +100914 POINT(40.74688813271296 74.36346811857685) bank100914 +100915 POINT(40.640788868708995 73.66986952615218) bank100915 +100916 POINT(40.73561252788101 74.74362679882978) bank100916 +100917 POINT(39.80152940600878 73.61597976509445) bank100917 +100918 POINT(40.04195353029736 73.30151651267441) bank100918 +100919 POINT(40.87752745399665 74.74887048747263) bank100919 +100920 POINT(39.96072224073547 73.27155074975187) bank100920 +100921 POINT(39.94053613117752 74.50009020305528) bank100921 +100922 POINT(41.155306405695036 74.80806523091869) bank100922 +100923 POINT(40.280953826033176 74.48118628307557) bank100923 +100924 POINT(40.692584109788434 74.40701509455496) bank100924 +100925 POINT(40.1971069076903 73.6936092200793) bank100925 +100926 POINT(40.92287605578272 74.94409463103187) bank100926 +100927 POINT(39.88394637106632 73.36499551486271) bank100927 +100928 POINT(40.09306952874749 74.04588364583658) bank100928 +100929 POINT(40.700099150441915 73.19752952916927) bank100929 +100930 POINT(40.77462085017845 73.28879348540902) bank100930 +100931 POINT(40.556901269837155 74.02084604544568) bank100931 +100932 POINT(41.6101594383919 73.5498936176077) bank100932 +100933 POINT(39.884114344407436 73.04108491445288) bank100933 +100934 POINT(41.23113015996451 73.23086285491273) bank100934 +100935 POINT(41.47571918348196 74.84170608477024) bank100935 +100936 POINT(40.637908811080074 74.79116584268573) bank100936 +100937 POINT(40.340203896282205 74.65095908140681) bank100937 +100938 POINT(40.621271852095965 73.58418605490287) bank100938 +100939 POINT(41.62317157541354 73.04451819440216) bank100939 +100940 POINT(40.51077156280607 73.52119397075734) bank100940 +100941 POINT(40.274463831985784 74.20948719955403) bank100941 +100942 POINT(40.575299309378146 74.57380534041341) bank100942 +100943 POINT(41.400480956087264 74.3455413177814) bank100943 +100944 POINT(40.46564574254801 74.61944488163884) bank100944 +100945 POINT(40.56201827764956 74.59443488389198) bank100945 +100946 POINT(41.483217826579235 74.0956426120527) bank100946 +100947 POINT(40.207507647156085 73.01273081233984) bank100947 +100948 POINT(39.93226115442556 73.38374885359707) bank100948 +100949 POINT(40.63973812053348 73.21649872871501) bank100949 +100950 POINT(40.869991759380916 73.79983588424612) bank100950 +100951 POINT(40.13426965135571 73.2466054399692) bank100951 +100952 POINT(40.86135641327536 73.21145837607072) bank100952 +100953 POINT(40.36934468369607 73.51277065695807) bank100953 +100954 POINT(40.872103369931935 74.56536857387451) bank100954 +100955 POINT(41.382120791843775 74.60733541330113) bank100955 +100956 POINT(40.29021811592469 73.33917373863932) bank100956 +100957 POINT(41.357016021714294 74.3042040015624) bank100957 +100958 POINT(39.889375681698624 74.52710506592406) bank100958 +100959 POINT(40.11989238651433 73.28132037273598) bank100959 +100960 POINT(40.15958161245577 74.744491547674) bank100960 +100961 POINT(41.35635273143045 73.86041153931058) bank100961 +100962 POINT(40.342194790909794 73.33556297541315) bank100962 +100963 POINT(41.21959593656064 74.56416650770855) bank100963 +100964 POINT(41.70383968936378 74.29639274521121) bank100964 +100965 POINT(41.06919198698262 74.45114195472696) bank100965 +100966 POINT(41.535921051602074 74.60213849489936) bank100966 +100967 POINT(40.790314519372885 74.41003594669999) bank100967 +100968 POINT(39.81107402724472 73.54778589544024) bank100968 +100969 POINT(40.77715759580775 73.79147617419689) bank100969 +100970 POINT(40.742111028176964 74.03493275556787) bank100970 +100971 POINT(40.81316275349505 73.44572442058399) bank100971 +100972 POINT(40.37420264254611 74.7214609274921) bank100972 +100973 POINT(41.51451150440741 73.93185544950967) bank100973 +100974 POINT(40.03212953665901 73.88774963191275) bank100974 +100975 POINT(41.21765995268477 74.70578024557705) bank100975 +100976 POINT(40.12674317263603 73.81158454736622) bank100976 +100977 POINT(40.111092106333984 74.7417552698782) bank100977 +100978 POINT(39.832582451376325 73.47796561777159) bank100978 +100979 POINT(40.89199954591102 74.3982028250139) bank100979 +100980 POINT(40.71282599741376 73.49361500838918) bank100980 +100981 POINT(40.76500002265536 74.83932955042263) bank100981 +100982 POINT(41.04078235792378 74.78525506719258) bank100982 +100983 POINT(41.235679077554174 74.32403447411751) bank100983 +100984 POINT(40.2434309002544 73.92878243353653) bank100984 +100985 POINT(40.9533936671133 74.80348698745085) bank100985 +100986 POINT(40.33375301883696 74.27324265654107) bank100986 +100987 POINT(39.7823206031717 73.80012902494238) bank100987 +100988 POINT(41.65283309463042 73.6345976023033) bank100988 +100989 POINT(41.117261208875846 73.3996932234582) bank100989 +100990 POINT(40.76795075135665 73.87559625866949) bank100990 +100991 POINT(41.15411042059649 73.33792409124182) bank100991 +100992 POINT(41.68005085308332 74.14747582750653) bank100992 +100993 POINT(41.035166009385755 74.5590349289712) bank100993 +100994 POINT(40.37804041635041 73.36457975733464) bank100994 +100995 POINT(41.37512348851584 73.33870710369112) bank100995 +100996 POINT(41.170100067855 73.62716451912438) bank100996 +100997 POINT(39.98260808955175 73.77960446597493) bank100997 +100998 POINT(41.15901666077726 74.25040316499388) bank100998 +100999 POINT(41.08840159893649 74.37853938671698) bank100999 +101000 POINT(40.896430585976944 73.25395157794625) bank101000 +101001 POINT(40.62510252036199 74.16026447818014) bank101001 +101002 POINT(40.26035973625583 73.39699947279678) bank101002 +101003 POINT(39.92218347900988 74.30308234422313) bank101003 +101004 POINT(40.450632801077866 73.41020040937788) bank101004 +101005 POINT(40.02407937720283 73.92361204392931) bank101005 +101006 POINT(41.185628665077104 73.604187470581) bank101006 +101007 POINT(40.67384474587214 73.26669375640641) bank101007 +101008 POINT(41.51746828938743 73.14329096860692) bank101008 +101009 POINT(41.62309703667006 74.64772390056022) bank101009 +101010 POINT(41.10334211925556 73.26923777573758) bank101010 +101011 POINT(41.3896335207734 73.11116923151424) bank101011 +101012 POINT(40.82988413890947 73.8182244306876) bank101012 +101013 POINT(40.178555925987624 74.17726006158695) bank101013 +101014 POINT(39.809114645891476 74.96726349342211) bank101014 +101015 POINT(40.18677487521131 74.65118021896537) bank101015 +101016 POINT(40.02738915819191 74.38801901545862) bank101016 +101017 POINT(40.758595007685216 74.03864403084701) bank101017 +101018 POINT(40.03435295304598 73.07427425267058) bank101018 +101019 POINT(41.00236679375373 74.16005003177095) bank101019 +101020 POINT(40.64471488483989 73.16228925404818) bank101020 +101021 POINT(41.082161522247965 73.65061023997634) bank101021 +101022 POINT(41.26344715972106 73.6904507350735) bank101022 +101023 POINT(39.86662239741343 74.45567240906148) bank101023 +101024 POINT(41.5515880851848 73.35059614690391) bank101024 +101025 POINT(40.45408704747519 73.2848517687562) bank101025 +101026 POINT(40.05292760194564 74.30706603634532) bank101026 +101027 POINT(41.445038970970465 74.3579716316059) bank101027 +101028 POINT(41.25883457118215 74.59053247776681) bank101028 +101029 POINT(40.32626498533826 74.71712281045419) bank101029 +101030 POINT(40.70327718637736 74.36356000723107) bank101030 +101031 POINT(40.559305668231225 73.03097178472066) bank101031 +101032 POINT(41.210448244564304 74.68732824359974) bank101032 +101033 POINT(41.70174169183564 73.11164321044303) bank101033 +101034 POINT(39.95836958117395 74.19239584048378) bank101034 +101035 POINT(40.48478051925041 74.66125391190553) bank101035 +101036 POINT(41.585492822477924 73.35984954858097) bank101036 +101037 POINT(40.0529247773065 73.45044914105257) bank101037 +101038 POINT(41.10888613551191 73.10749235979686) bank101038 +101039 POINT(40.612941693763865 74.49574629486379) bank101039 +101040 POINT(39.742695593980926 73.81088804183221) bank101040 +101041 POINT(40.33999877774136 74.02419571890375) bank101041 +101042 POINT(41.117532406903685 73.21026824707403) bank101042 +101043 POINT(41.45189724789101 73.34183009946452) bank101043 +101044 POINT(41.18111992817532 74.86942359620691) bank101044 +101045 POINT(39.79805476035937 73.09992102762773) bank101045 +101046 POINT(41.6325890268622 73.85617388823135) bank101046 +101047 POINT(41.3573440011382 74.69546747357013) bank101047 +101048 POINT(39.89179213948264 74.77496449022952) bank101048 +101049 POINT(39.839576174899186 74.93885123821748) bank101049 +101050 POINT(41.4947934701054 74.38133719421447) bank101050 +101051 POINT(41.39268765454726 74.88377522858525) bank101051 +101052 POINT(40.1672768445438 74.55063877948402) bank101052 +101053 POINT(40.73768428440973 73.87642509828063) bank101053 +101054 POINT(40.33210839129742 74.33406922216335) bank101054 +101055 POINT(40.77601749758218 73.20853940837954) bank101055 +101056 POINT(40.84319176504327 73.7129646053335) bank101056 +101057 POINT(41.03263521096255 74.09835446332141) bank101057 +101058 POINT(39.88015427657376 73.68577407918198) bank101058 +101059 POINT(40.30712441624813 74.84369623704598) bank101059 +101060 POINT(40.55975331369307 73.41929415795684) bank101060 +101061 POINT(40.63012120999466 73.2282042291956) bank101061 +101062 POINT(41.27698787663528 73.23011942422146) bank101062 +101063 POINT(40.2398613507781 73.32773750129866) bank101063 +101064 POINT(41.20212849355289 74.57707786654207) bank101064 +101065 POINT(41.303184197324086 74.0272890379235) bank101065 +101066 POINT(39.755400715380205 74.98560007836264) bank101066 +101067 POINT(41.15215567967314 73.25047957686013) bank101067 +101068 POINT(41.30129136614536 74.21400032277579) bank101068 +101069 POINT(40.3341543257149 74.93980161930222) bank101069 +101070 POINT(40.861027791623854 73.88833401425154) bank101070 +101071 POINT(41.665591589581396 73.00893471822867) bank101071 +101072 POINT(41.564683777425046 74.62264779529751) bank101072 +101073 POINT(40.502499512298236 73.82998363119819) bank101073 +101074 POINT(39.77101825862252 74.31263034062493) bank101074 +101075 POINT(41.38658232481201 73.70995209598378) bank101075 +101076 POINT(41.500090703469695 73.08378170841907) bank101076 +101077 POINT(40.77308872432451 73.3864665822506) bank101077 +101078 POINT(39.74929729831047 73.87394243399878) bank101078 +101079 POINT(41.035028405537446 74.83115680724651) bank101079 +101080 POINT(40.18170295949559 73.37080005089493) bank101080 +101081 POINT(40.85023743285629 74.39642592426748) bank101081 +101082 POINT(40.309317133988344 74.56093488334845) bank101082 +101083 POINT(40.56656421907854 74.12852090045047) bank101083 +101084 POINT(40.59643367104818 74.94810597888544) bank101084 +101085 POINT(41.18565119055939 73.17841717890066) bank101085 +101086 POINT(41.31395286128994 73.67749364213915) bank101086 +101087 POINT(40.84229191307761 73.1310027116946) bank101087 +101088 POINT(40.27073691899139 73.3856917887724) bank101088 +101089 POINT(40.87021040410894 74.79728877873994) bank101089 +101090 POINT(41.096623102922145 74.20433477934773) bank101090 +101091 POINT(40.065597882292415 73.7892421023187) bank101091 +101092 POINT(40.015524025158285 73.16319552240319) bank101092 +101093 POINT(40.69280610969352 73.49373613872008) bank101093 +101094 POINT(40.77483007763696 73.94359938226071) bank101094 +101095 POINT(41.69962866669291 74.99951562149748) bank101095 +101096 POINT(40.80282406279088 73.218954398565) bank101096 +101097 POINT(40.682233271677774 74.92236024358347) bank101097 +101098 POINT(41.67918105014739 73.95830116772477) bank101098 +101099 POINT(41.269035327222575 74.36348274332158) bank101099 +101100 POINT(41.60583043430519 73.09042369450577) bank101100 +101101 POINT(41.3200570135302 73.59277084698881) bank101101 +101102 POINT(41.07140845107052 74.01306556910554) bank101102 +101103 POINT(41.19080150170608 73.13439055203264) bank101103 +101104 POINT(41.24276420433264 73.63890019429485) bank101104 +101105 POINT(40.11987840789921 73.11266997498309) bank101105 +101106 POINT(40.165762531764074 73.74892017882888) bank101106 +101107 POINT(39.89396511374723 73.65088382826539) bank101107 +101108 POINT(41.33561095469265 73.57354709779746) bank101108 +101109 POINT(40.81230382890458 73.72501596363739) bank101109 +101110 POINT(41.09076879320823 74.7125480334978) bank101110 +101111 POINT(39.79289926126005 73.71783396678138) bank101111 +101112 POINT(41.712602388508664 74.90101259769504) bank101112 +101113 POINT(40.93948854898888 73.22387854004992) bank101113 +101114 POINT(41.38184353270699 74.30264497257926) bank101114 +101115 POINT(40.458799522794145 73.78353293787777) bank101115 +101116 POINT(41.39718162130126 74.34153035325679) bank101116 +101117 POINT(41.562117300992576 74.85602036235075) bank101117 +101118 POINT(41.51781915193749 73.5691807698476) bank101118 +101119 POINT(41.121793436337676 74.15281004135727) bank101119 +101120 POINT(40.41829137553558 74.13907365826114) bank101120 +101121 POINT(40.242808167169294 74.00552775339486) bank101121 +101122 POINT(41.622656264384865 73.42687733987839) bank101122 +101123 POINT(40.86645102069894 73.95984804340702) bank101123 +101124 POINT(40.12652101540819 73.3479998920729) bank101124 +101125 POINT(41.450530773158576 74.38513611880106) bank101125 +101126 POINT(40.79138829330433 73.42530135885887) bank101126 +101127 POINT(41.57453255156753 74.3226669726255) bank101127 +101128 POINT(40.90290415846885 74.22945775789665) bank101128 +101129 POINT(41.311827839843815 74.80622546989996) bank101129 +101130 POINT(41.46725546515405 74.97216249551309) bank101130 +101131 POINT(39.88633101558607 73.06011058861084) bank101131 +101132 POINT(40.578753062694595 73.46050440059713) bank101132 +101133 POINT(39.74726316301142 73.0968298837434) bank101133 +101134 POINT(39.84933310598789 74.63150656448246) bank101134 +101135 POINT(41.23691443554978 73.13690942736575) bank101135 +101136 POINT(40.701584663472964 73.32391621186831) bank101136 +101137 POINT(41.481697870658145 74.41325078924051) bank101137 +101138 POINT(41.56655261631032 73.85460471827206) bank101138 +101139 POINT(41.14781193969305 74.58781430203192) bank101139 +101140 POINT(41.51967028256408 74.18384521380186) bank101140 +101141 POINT(41.62684869741066 74.49206243310883) bank101141 +101142 POINT(39.82105478497123 73.02864667690933) bank101142 +101143 POINT(40.90011114364262 74.67615402521015) bank101143 +101144 POINT(41.475204403314535 73.31931772590444) bank101144 +101145 POINT(41.22514380800246 74.5795401460613) bank101145 +101146 POINT(41.086926626173174 73.58840571452082) bank101146 +101147 POINT(40.833141657964056 74.57657418727281) bank101147 +101148 POINT(39.890638362918196 73.37131174614305) bank101148 +101149 POINT(41.27514228336753 74.24557785421644) bank101149 +101150 POINT(41.71144525223776 74.14708251313466) bank101150 +101151 POINT(40.5804155597191 73.3878606427839) bank101151 +101152 POINT(40.6917973085111 73.84749758501569) bank101152 +101153 POINT(41.44227220845384 73.58862959159161) bank101153 +101154 POINT(41.19359577364264 73.09172540747369) bank101154 +101155 POINT(40.353958385923555 73.93276663643871) bank101155 +101156 POINT(40.778708720143605 73.25244045046958) bank101156 +101157 POINT(41.37841005337581 74.16554909128715) bank101157 +101158 POINT(40.38388951855198 74.72295043313075) bank101158 +101159 POINT(39.928250144006164 74.21213401561747) bank101159 +101160 POINT(40.85221964661953 73.55972920872462) bank101160 +101161 POINT(40.772333112296565 74.07832450357321) bank101161 +101162 POINT(40.569095452855535 74.05538051004255) bank101162 +101163 POINT(41.12258160844641 74.1162244663083) bank101163 +101164 POINT(39.80403699372751 74.71305688359865) bank101164 +101165 POINT(40.62170231698412 73.80308204400339) bank101165 +101166 POINT(41.363352030817 74.00057184417808) bank101166 +101167 POINT(40.77242805803733 74.58572054160689) bank101167 +101168 POINT(40.946683333517804 73.80952374390985) bank101168 +101169 POINT(41.58044271812924 73.33015790227246) bank101169 +101170 POINT(40.68352286643604 73.73185924650281) bank101170 +101171 POINT(41.595608454156405 73.66867102354377) bank101171 +101172 POINT(40.09101663499639 73.36764014906402) bank101172 +101173 POINT(40.56332677558072 73.92930970911999) bank101173 +101174 POINT(41.71266088066333 74.35317465354832) bank101174 +101175 POINT(40.46782965455571 73.87890247275521) bank101175 +101176 POINT(40.55079544687075 74.01465026543438) bank101176 +101177 POINT(40.63441898350652 73.9381882900041) bank101177 +101178 POINT(41.18517398159479 73.74794290522388) bank101178 +101179 POINT(41.030128578794226 74.17231948112) bank101179 +101180 POINT(39.977703549111645 74.91596815378942) bank101180 +101181 POINT(40.0605477019867 73.20467548204479) bank101181 +101182 POINT(40.54932211117472 74.85325875344773) bank101182 +101183 POINT(41.231156709024965 74.62564359306839) bank101183 +101184 POINT(40.76739441092081 74.02630345621174) bank101184 +101185 POINT(40.97744703933595 73.63751348270151) bank101185 +101186 POINT(41.60396261760097 74.18878202330038) bank101186 +101187 POINT(41.24419220349221 73.1829007206404) bank101187 +101188 POINT(39.849724948371154 73.90980462032064) bank101188 +101189 POINT(39.91515532077272 74.06720446193164) bank101189 +101190 POINT(41.26611763630343 73.58288837184162) bank101190 +101191 POINT(41.244150434847576 74.61226007101706) bank101191 +101192 POINT(41.12389903928529 73.50498084863555) bank101192 +101193 POINT(40.337110402243326 73.7605456165369) bank101193 +101194 POINT(39.91935737062113 73.14486550659649) bank101194 +101195 POINT(40.27979228683189 74.53235659715531) bank101195 +101196 POINT(41.690209154897246 74.44597824250363) bank101196 +101197 POINT(41.57057136592727 74.58872177673761) bank101197 +101198 POINT(40.71963245496911 73.70573298092378) bank101198 +101199 POINT(40.80473955641178 73.28319879735623) bank101199 +101200 POINT(39.95549049115704 73.76631297006274) bank101200 +101201 POINT(39.827081552177226 73.40295499295125) bank101201 +101202 POINT(40.86568043408033 74.77052307248374) bank101202 +101203 POINT(40.059815616511415 74.46481110342513) bank101203 +101204 POINT(39.94099286439077 73.51305395434989) bank101204 +101205 POINT(40.15892267113637 73.42889024801275) bank101205 +101206 POINT(40.164473148359995 73.84207092165289) bank101206 +101207 POINT(39.822359992551455 74.23869051635843) bank101207 +101208 POINT(41.31557572489901 74.46146124323106) bank101208 +101209 POINT(41.41776392887628 73.9684664132434) bank101209 +101210 POINT(41.28221150463224 74.75616282618428) bank101210 +101211 POINT(41.48203768262073 74.0716937279544) bank101211 +101212 POINT(40.62117277345356 73.40942994667694) bank101212 +101213 POINT(39.79626181533417 74.37347703561277) bank101213 +101214 POINT(40.63568109128725 73.24092023993381) bank101214 +101215 POINT(40.12883460730684 74.69999968269099) bank101215 +101216 POINT(39.78602771023668 74.50673403859604) bank101216 +101217 POINT(40.425902717550926 74.75450962160747) bank101217 +101218 POINT(40.41322899607549 74.3573435970913) bank101218 +101219 POINT(41.50659150535238 74.27822780705121) bank101219 +101220 POINT(41.529025348437536 74.4791656126539) bank101220 +101221 POINT(40.09395779936367 73.85634806630158) bank101221 +101222 POINT(40.68467516170453 73.50960358115198) bank101222 +101223 POINT(40.110660053070134 74.04130036317859) bank101223 +101224 POINT(41.50117539757232 74.02818220421274) bank101224 +101225 POINT(40.83129635185435 74.85263939108866) bank101225 +101226 POINT(40.157159614373505 74.82895771747856) bank101226 +101227 POINT(40.21364133077139 73.99715408606502) bank101227 +101228 POINT(39.85421724062965 73.76491746116072) bank101228 +101229 POINT(41.553230659961756 74.48343678041945) bank101229 +101230 POINT(41.13932567579968 74.8807051772758) bank101230 +101231 POINT(40.33854185528677 73.28757343205902) bank101231 +101232 POINT(40.001676364771626 74.64853942425783) bank101232 +101233 POINT(41.527611528201824 73.07680997177143) bank101233 +101234 POINT(40.11874137210863 74.25118354255338) bank101234 +101235 POINT(41.228221464861356 74.16266577527253) bank101235 +101236 POINT(40.957549448509155 74.66703762351703) bank101236 +101237 POINT(40.24211417733031 73.09310142268913) bank101237 +101238 POINT(40.813151550928914 73.38868350441382) bank101238 +101239 POINT(40.782590185361464 74.78534346526355) bank101239 +101240 POINT(40.289828466188105 74.86600559438119) bank101240 +101241 POINT(41.47137792698685 73.46354500532065) bank101241 +101242 POINT(40.55299004332294 73.96157792682511) bank101242 +101243 POINT(40.484594137079604 73.40101086744983) bank101243 +101244 POINT(41.61468467430676 73.27610785863043) bank101244 +101245 POINT(41.13210479160495 74.72141069876206) bank101245 +101246 POINT(41.073176637939696 73.3864940434252) bank101246 +101247 POINT(40.09679592830814 73.27502203933133) bank101247 +101248 POINT(40.89178131226426 74.47225588425977) bank101248 +101249 POINT(40.32875020983198 74.42170826859137) bank101249 +101250 POINT(40.364400126451244 74.53817861145345) bank101250 +101251 POINT(40.76188787805485 73.08868981156083) bank101251 +101252 POINT(40.60951133612494 74.11897763573808) bank101252 +101253 POINT(41.34504855471962 73.77064488245601) bank101253 +101254 POINT(40.47105184631283 74.59416336239805) bank101254 +101255 POINT(40.764246241193526 73.06509747023247) bank101255 +101256 POINT(41.62301579257908 74.01226499923128) bank101256 +101257 POINT(39.78274808685447 73.57116636873482) bank101257 +101258 POINT(39.92364618388289 74.99420568665361) bank101258 +101259 POINT(41.01315030454167 73.69386949150665) bank101259 +101260 POINT(40.493359420160076 74.05916612944783) bank101260 +101261 POINT(41.49380342913692 73.62011472665019) bank101261 +101262 POINT(40.57266551440211 73.57906227387072) bank101262 +101263 POINT(41.07613346171527 74.9597859720027) bank101263 +101264 POINT(41.36328214995826 73.52408843446729) bank101264 +101265 POINT(41.12019539486437 73.02409802325484) bank101265 +101266 POINT(39.866583394412096 74.89215599387244) bank101266 +101267 POINT(41.63515585955155 73.20530683134822) bank101267 +101268 POINT(40.96453650702618 73.15629183072521) bank101268 +101269 POINT(40.14691776608976 74.28481574170154) bank101269 +101270 POINT(40.45123239024747 74.5471562412568) bank101270 +101271 POINT(40.61959468818015 74.50846237801153) bank101271 +101272 POINT(41.62162003827408 74.04627824790424) bank101272 +101273 POINT(40.80043371201606 73.43448865621569) bank101273 +101274 POINT(39.88927135987568 74.77740532337184) bank101274 +101275 POINT(39.82262805648926 73.76085545240299) bank101275 +101276 POINT(40.43632882405156 73.63403180942402) bank101276 +101277 POINT(41.049884617787086 74.77759315257659) bank101277 +101278 POINT(41.62853334516974 74.5757247415317) bank101278 +101279 POINT(39.71744241120186 74.5690898609521) bank101279 +101280 POINT(40.97347424481916 73.53756723891516) bank101280 +101281 POINT(41.083003057463166 74.49526358561256) bank101281 +101282 POINT(39.74874307961087 74.6387155849188) bank101282 +101283 POINT(40.739077582102084 73.90785577916931) bank101283 +101284 POINT(40.00346982131391 73.96589789005277) bank101284 +101285 POINT(39.95579586056193 73.0988209486255) bank101285 +101286 POINT(41.29050043103436 74.87098009299237) bank101286 +101287 POINT(40.037639534095796 73.61291847483577) bank101287 +101288 POINT(40.43663273689267 74.04586991183486) bank101288 +101289 POINT(41.6368546499249 74.96016463822137) bank101289 +101290 POINT(39.72364235663475 73.29264643858176) bank101290 +101291 POINT(40.23920012560613 73.68865749203424) bank101291 +101292 POINT(40.84029831627455 74.70008905428904) bank101292 +101293 POINT(40.84454701218321 73.72193681251848) bank101293 +101294 POINT(40.38573093120493 74.70953507155211) bank101294 +101295 POINT(41.10450291253264 74.350311610498) bank101295 +101296 POINT(41.176927189847284 74.15998053873622) bank101296 +101297 POINT(41.603154431973536 73.83356334034026) bank101297 +101298 POINT(41.010097794420076 74.42566998817331) bank101298 +101299 POINT(41.261880259493005 74.35817518318119) bank101299 +101300 POINT(41.09193727424682 74.62028694263776) bank101300 +101301 POINT(41.08301291886195 73.93395739942913) bank101301 +101302 POINT(39.7900675886205 73.60318867745046) bank101302 +101303 POINT(40.977541546783655 73.02495311046701) bank101303 +101304 POINT(40.940199151179975 74.94587466623163) bank101304 +101305 POINT(40.187163390005885 73.62615414068053) bank101305 +101306 POINT(41.26819862051029 73.65989914475757) bank101306 +101307 POINT(40.99600427282635 74.35792771232278) bank101307 +101308 POINT(40.31899560998457 74.55648710551283) bank101308 +101309 POINT(39.75884659343788 73.92872767119718) bank101309 +101310 POINT(40.226789653433855 74.75910531204268) bank101310 +101311 POINT(40.67016305150162 73.05443246337879) bank101311 +101312 POINT(40.38503260664936 74.71574480824083) bank101312 +101313 POINT(41.031429016190195 73.19252837219582) bank101313 +101314 POINT(41.589023844947164 74.24244091209798) bank101314 +101315 POINT(40.231967510124036 73.4001786200625) bank101315 +101316 POINT(41.348650255374345 73.58968206353663) bank101316 +101317 POINT(41.24849061004835 74.27996054350787) bank101317 +101318 POINT(41.32503779500677 73.97889042428429) bank101318 +101319 POINT(40.56280744689844 74.38583272089807) bank101319 +101320 POINT(40.81542581677139 73.68480093085262) bank101320 +101321 POINT(41.16985015636637 74.398665437531) bank101321 +101322 POINT(41.4363916585262 73.85942228320557) bank101322 +101323 POINT(40.11731366217841 74.59328057669845) bank101323 +101324 POINT(41.33240306880356 74.76895726641612) bank101324 +101325 POINT(40.11686445902211 74.66831998474957) bank101325 +101326 POINT(39.76640226656491 74.59095536709192) bank101326 +101327 POINT(40.434773283836776 73.25323012008737) bank101327 +101328 POINT(41.41008368416223 74.84548090631883) bank101328 +101329 POINT(41.46080275911146 73.86746348470933) bank101329 +101330 POINT(39.817412478461144 73.42690612198045) bank101330 +101331 POINT(39.959961860283 73.66158197330505) bank101331 +101332 POINT(40.31411494023763 73.67204850184335) bank101332 +101333 POINT(41.60986994034764 73.37781198740325) bank101333 +101334 POINT(40.12014448718396 73.39667389104406) bank101334 +101335 POINT(39.97645906446971 73.21363380228168) bank101335 +101336 POINT(40.5758673419152 73.2526747226239) bank101336 +101337 POINT(41.533255227936074 73.97873619933148) bank101337 +101338 POINT(41.306506363469325 73.22707296122168) bank101338 +101339 POINT(40.35693565139173 74.53209205369104) bank101339 +101340 POINT(40.95075590639837 73.6364405299976) bank101340 +101341 POINT(40.418838404251794 73.34242742417395) bank101341 +101342 POINT(40.463359272948146 73.08708041555674) bank101342 +101343 POINT(41.17160678706992 73.97736589230156) bank101343 +101344 POINT(41.539663506285855 73.16450473260691) bank101344 +101345 POINT(39.9146487582218 74.42264551056911) bank101345 +101346 POINT(40.665396257008794 73.54147951262058) bank101346 +101347 POINT(40.09496673577593 74.00616164817133) bank101347 +101348 POINT(40.88769674597124 73.06007528736725) bank101348 +101349 POINT(40.094378998668404 74.10464377790514) bank101349 +101350 POINT(40.578269639644965 74.81124047866693) bank101350 +101351 POINT(40.7885988928362 73.9230922922221) bank101351 +101352 POINT(40.174262221965904 74.28129826134429) bank101352 +101353 POINT(41.286769210749725 73.3548474199899) bank101353 +101354 POINT(40.25224466402238 74.48585487364365) bank101354 +101355 POINT(40.03554954045529 74.48436511959811) bank101355 +101356 POINT(39.9465957365731 74.90056934163178) bank101356 +101357 POINT(41.68303681984975 73.75362519160167) bank101357 +101358 POINT(40.19025402863492 74.92512376625812) bank101358 +101359 POINT(41.49898102394293 73.45762761849862) bank101359 +101360 POINT(41.3268404959248 73.09760196107204) bank101360 +101361 POINT(41.10300880933569 73.3965245367753) bank101361 +101362 POINT(41.1677283283912 74.31269022632664) bank101362 +101363 POINT(41.28309993513737 74.58552160914623) bank101363 +101364 POINT(41.637126971294705 74.48824740198819) bank101364 +101365 POINT(41.56744271974687 73.99232449839614) bank101365 +101366 POINT(39.97450073593513 73.90183686650482) bank101366 +101367 POINT(41.10521771273481 74.9319148974306) bank101367 +101368 POINT(40.76798597445059 74.15470552579283) bank101368 +101369 POINT(40.93661404008776 74.32326953072838) bank101369 +101370 POINT(40.73465682534859 73.62684605243392) bank101370 +101371 POINT(40.91134252994184 74.73267167411665) bank101371 +101372 POINT(41.19621230806563 74.94059619669027) bank101372 +101373 POINT(40.25172654985825 73.46975548676326) bank101373 +101374 POINT(41.28302286453761 74.12907143927858) bank101374 +101375 POINT(40.56560355064872 74.99930587264302) bank101375 +101376 POINT(40.23553642901936 73.34221771245006) bank101376 +101377 POINT(39.74738313334899 73.41549674823986) bank101377 +101378 POINT(40.719915216979544 74.21147370266448) bank101378 +101379 POINT(40.67495118953132 74.69896816884821) bank101379 +101380 POINT(40.30805336083398 74.29158832359083) bank101380 +101381 POINT(39.96978294720883 74.04970512825467) bank101381 +101382 POINT(39.87789348239763 74.95014863126188) bank101382 +101383 POINT(41.04387566877698 73.87375276624127) bank101383 +101384 POINT(41.25218533785556 73.33379542516285) bank101384 +101385 POINT(41.31311514433861 73.58140810427984) bank101385 +101386 POINT(39.99763040717742 74.56363270811725) bank101386 +101387 POINT(40.874742681547794 74.23836850193764) bank101387 +101388 POINT(41.092441607777154 74.33752152612658) bank101388 +101389 POINT(41.130911308043856 73.34190622437995) bank101389 +101390 POINT(40.85502900753631 73.12998599476417) bank101390 +101391 POINT(40.60493809492493 73.51806890943662) bank101391 +101392 POINT(41.62339140979956 73.63170810847937) bank101392 +101393 POINT(41.45328674813829 73.95734033606283) bank101393 +101394 POINT(40.18510283362567 74.52884453978197) bank101394 +101395 POINT(40.165320546170726 73.8749028412137) bank101395 +101396 POINT(40.38417963334864 74.2391578849853) bank101396 +101397 POINT(40.65550899543121 73.3459991952922) bank101397 +101398 POINT(41.19284847930456 73.54913465346453) bank101398 +101399 POINT(40.188662694626515 74.89891645213955) bank101399 +101400 POINT(40.24276774359688 73.69627279607083) bank101400 +101401 POINT(40.113660135874134 73.86047977755602) bank101401 +101402 POINT(41.69455691832532 73.457431607106) bank101402 +101403 POINT(41.14780002904648 74.78584656776562) bank101403 +101404 POINT(40.72414765693038 73.56543270147075) bank101404 +101405 POINT(40.771985349813605 73.06546616289809) bank101405 +101406 POINT(41.67111851137639 73.92531057915312) bank101406 +101407 POINT(40.93406938040725 73.64413502436734) bank101407 +101408 POINT(40.84418502823096 73.31834472461044) bank101408 +101409 POINT(41.672871799586346 73.0266586287458) bank101409 +101410 POINT(41.22658454893558 74.89343870139568) bank101410 +101411 POINT(40.14354422762136 74.3848652709272) bank101411 +101412 POINT(40.81362243594992 73.85333073530396) bank101412 +101413 POINT(39.90898814508299 74.51063687167611) bank101413 +101414 POINT(41.48427185497202 73.72346960288658) bank101414 +101415 POINT(39.71389883882994 74.05341634683678) bank101415 +101416 POINT(40.59178461715414 73.84587327195081) bank101416 +101417 POINT(41.63872473351076 73.85465999813337) bank101417 +101418 POINT(41.29726005593312 74.6575675615235) bank101418 +101419 POINT(40.55558147459153 74.89937502399174) bank101419 +101420 POINT(41.595284592841224 73.69934485074226) bank101420 +101421 POINT(40.10332464641151 73.82537445904906) bank101421 +101422 POINT(39.77530436240009 74.98610319883716) bank101422 +101423 POINT(39.79299005189755 73.1785959281425) bank101423 +101424 POINT(40.812772297976686 73.6945391090234) bank101424 +101425 POINT(41.14888655188981 73.01811452276527) bank101425 +101426 POINT(41.16007764355594 73.52020034575632) bank101426 +101427 POINT(40.18369849778379 73.12584303650706) bank101427 +101428 POINT(41.00492320025179 74.45195858366455) bank101428 +101429 POINT(41.40480165097198 74.25532664479393) bank101429 +101430 POINT(41.616254241652726 74.99710100811843) bank101430 +101431 POINT(40.92617309942501 73.51228133964524) bank101431 +101432 POINT(41.201413371017466 74.52411879636279) bank101432 +101433 POINT(40.620082760565076 74.07308906904323) bank101433 +101434 POINT(40.71524297527241 74.10892741190423) bank101434 +101435 POINT(40.18658082787883 74.47398892826811) bank101435 +101436 POINT(39.859048614104886 73.75371874219606) bank101436 +101437 POINT(40.15671941646851 74.7094434882236) bank101437 +101438 POINT(41.34520541880279 74.09191551147092) bank101438 +101439 POINT(40.34937773079079 73.75337180388615) bank101439 +101440 POINT(40.367462116339794 74.30336552081448) bank101440 +101441 POINT(39.95861471408235 73.91144283451023) bank101441 +101442 POINT(40.887309030829606 73.73633128162656) bank101442 +101443 POINT(40.26349353315763 73.10728179964252) bank101443 +101444 POINT(40.72899638829394 74.50238460908841) bank101444 +101445 POINT(41.313192865689196 74.24327959328383) bank101445 +101446 POINT(41.02144296307167 73.74641866186506) bank101446 +101447 POINT(40.206594182578726 74.08361120978903) bank101447 +101448 POINT(41.09151304091634 73.86989371150065) bank101448 +101449 POINT(40.900427898754145 73.98851705576766) bank101449 +101450 POINT(41.16111712266377 74.96388295804573) bank101450 +101451 POINT(40.68285306537184 74.9738708610338) bank101451 +101452 POINT(41.54889292354505 73.63320197457216) bank101452 +101453 POINT(41.691223804209606 73.25365731514097) bank101453 +101454 POINT(40.00735617455668 74.9263689173708) bank101454 +101455 POINT(39.886046476941424 73.33105098463858) bank101455 +101456 POINT(41.54935903850814 74.61508167658103) bank101456 +101457 POINT(41.22878791120165 74.54780600464862) bank101457 +101458 POINT(40.049454070583366 74.98223509701292) bank101458 +101459 POINT(41.15952737634964 73.98363600125701) bank101459 +101460 POINT(41.62458278153828 73.0158550798101) bank101460 +101461 POINT(40.76886126044313 74.45836810564346) bank101461 +101462 POINT(41.696937339499414 73.96269596868947) bank101462 +101463 POINT(41.19591574113266 73.39178215363025) bank101463 +101464 POINT(40.137832956252666 73.28768289159657) bank101464 +101465 POINT(40.76383486260153 74.38037891699464) bank101465 +101466 POINT(39.99761438612825 73.32032340555065) bank101466 +101467 POINT(39.98338521827631 73.46888280788701) bank101467 +101468 POINT(41.31174245734226 73.52634384478323) bank101468 +101469 POINT(41.362451845343614 74.65335214568174) bank101469 +101470 POINT(39.80442149091809 74.55798914826073) bank101470 +101471 POINT(40.4323133698395 73.76710420638426) bank101471 +101472 POINT(40.21241935794236 73.47738376404817) bank101472 +101473 POINT(40.16722283018675 73.76456078835815) bank101473 +101474 POINT(39.95954527420041 74.44833507889686) bank101474 +101475 POINT(40.204233086506 74.74617840873721) bank101475 +101476 POINT(39.779982749425706 73.8363711205997) bank101476 +101477 POINT(40.138719885445425 73.9652557942068) bank101477 +101478 POINT(40.485976328046185 74.11743708650945) bank101478 +101479 POINT(40.65601952244671 73.94364520687368) bank101479 +101480 POINT(40.4238059922839 74.68019484771308) bank101480 +101481 POINT(40.88144639915094 73.32517786216037) bank101481 +101482 POINT(40.54756586384547 74.1710807443802) bank101482 +101483 POINT(39.9796901371857 73.44050185923447) bank101483 +101484 POINT(39.97523388609784 74.84354746272761) bank101484 +101485 POINT(41.114315094334906 74.92817650842524) bank101485 +101486 POINT(41.4848442466992 73.74322701338241) bank101486 +101487 POINT(40.1593588085446 73.55371316065009) bank101487 +101488 POINT(40.054616847318684 73.58661255513316) bank101488 +101489 POINT(39.74411774157086 73.5241958711785) bank101489 +101490 POINT(41.129433285544174 73.9372198208274) bank101490 +101491 POINT(41.3517682881476 73.3463262492646) bank101491 +101492 POINT(40.03561116743927 73.38141709865657) bank101492 +101493 POINT(39.963998769967496 73.95338790618918) bank101493 +101494 POINT(40.529455369965476 74.68666431716882) bank101494 +101495 POINT(40.33494079173506 73.10827745375813) bank101495 +101496 POINT(41.22584992047862 74.68575136961077) bank101496 +101497 POINT(40.80021645038268 74.29368757048707) bank101497 +101498 POINT(40.802586207126424 73.91518484056039) bank101498 +101499 POINT(40.18648046164743 74.28129387653681) bank101499 +101500 POINT(40.75654499110838 74.37356681132587) bank101500 +101501 POINT(41.052414560530906 74.42170362502172) bank101501 +101502 POINT(40.40847553952658 74.52791122433128) bank101502 +101503 POINT(40.87126898783701 73.5141348553355) bank101503 +101504 POINT(40.95310373596876 74.84619876507713) bank101504 +101505 POINT(40.083359788185824 73.03046550550103) bank101505 +101506 POINT(40.14042564244953 73.7162799755546) bank101506 +101507 POINT(40.58859976978753 74.22211686414478) bank101507 +101508 POINT(41.27569208639278 73.69072992036251) bank101508 +101509 POINT(41.46893838298064 74.83097676406283) bank101509 +101510 POINT(40.843024870487476 73.6902378850698) bank101510 +101511 POINT(40.703877204934024 73.88622122912886) bank101511 +101512 POINT(39.89246047356376 73.40232342710077) bank101512 +101513 POINT(39.81268376515326 73.90637042181717) bank101513 +101514 POINT(41.09132804873988 74.9524360024135) bank101514 +101515 POINT(41.43355632583904 74.09992304872519) bank101515 +101516 POINT(40.26042355974547 74.146164598845) bank101516 +101517 POINT(41.42150302102954 73.98039313189712) bank101517 +101518 POINT(39.72355711865395 73.0428148770112) bank101518 +101519 POINT(40.245888346643184 73.80951681356754) bank101519 +101520 POINT(40.56443146376095 74.96278783641046) bank101520 +101521 POINT(40.350423827873804 74.23672036264446) bank101521 +101522 POINT(40.412594782231906 74.91720683242696) bank101522 +101523 POINT(40.833829392995234 73.50248732630855) bank101523 +101524 POINT(39.81510535232059 73.56602917945587) bank101524 +101525 POINT(40.79268553476318 74.21885497232523) bank101525 +101526 POINT(40.407752633271876 74.40616755708413) bank101526 +101527 POINT(39.896171637725395 73.10217152585922) bank101527 +101528 POINT(40.101938149223386 74.1003697146889) bank101528 +101529 POINT(39.81958881943147 74.98728650007223) bank101529 +101530 POINT(40.99136369444151 73.54481621580686) bank101530 +101531 POINT(40.280095725500544 74.0502657505803) bank101531 +101532 POINT(41.41210208923931 74.21963217017901) bank101532 +101533 POINT(41.328777664594625 74.89188839416002) bank101533 +101534 POINT(40.818944459402765 74.07427133461316) bank101534 +101535 POINT(40.294182122941194 74.72930022524447) bank101535 +101536 POINT(40.19579496574518 74.16961696019905) bank101536 +101537 POINT(40.5011882611377 74.31103158195955) bank101537 +101538 POINT(40.007972352139376 73.43457281882158) bank101538 +101539 POINT(40.53150203270886 74.76851542044369) bank101539 +101540 POINT(41.14167709113256 73.07233503757881) bank101540 +101541 POINT(40.46552863368586 73.08712950861322) bank101541 +101542 POINT(40.888122500008 73.71669966944107) bank101542 +101543 POINT(40.23143826180438 73.65344241581029) bank101543 +101544 POINT(40.68141841959089 74.89297577067364) bank101544 +101545 POINT(40.12395156116178 73.99512492812131) bank101545 +101546 POINT(41.19663836429768 74.61319960201544) bank101546 +101547 POINT(40.803528296460335 73.3876699850317) bank101547 +101548 POINT(40.22033632637526 73.80368787495163) bank101548 +101549 POINT(41.316578158272655 73.80961246728705) bank101549 +101550 POINT(40.4505705375716 73.24967942979613) bank101550 +101551 POINT(41.35083258567395 74.75133050340645) bank101551 +101552 POINT(40.40043257875643 74.94729370462947) bank101552 +101553 POINT(40.932470364521365 74.31261296428048) bank101553 +101554 POINT(41.537675943170065 74.64723971120935) bank101554 +101555 POINT(40.65357855732115 74.59548184162388) bank101555 +101556 POINT(41.5273751923014 74.94705187983355) bank101556 +101557 POINT(41.61634306061013 73.25912576076689) bank101557 +101558 POINT(41.57278609176444 73.51929801387631) bank101558 +101559 POINT(40.607087279472665 74.07120810425533) bank101559 +101560 POINT(41.32366209914534 73.95421107191576) bank101560 +101561 POINT(41.24647187283729 74.1698896980479) bank101561 +101562 POINT(40.83289950660443 73.37755060817544) bank101562 +101563 POINT(40.05186982413153 74.90081125381785) bank101563 +101564 POINT(41.46487373637154 73.83631646094278) bank101564 +101565 POINT(40.10841983539244 73.8688973849887) bank101565 +101566 POINT(40.54269875197683 74.0326497588834) bank101566 +101567 POINT(40.91393079440957 74.03375787276386) bank101567 +101568 POINT(40.97096066861724 73.46963485293817) bank101568 +101569 POINT(40.677157162295444 74.95018612145431) bank101569 +101570 POINT(40.57519452271415 74.27016956962365) bank101570 +101571 POINT(41.37764603757133 73.5933082214363) bank101571 +101572 POINT(40.70382633875367 74.26700571786105) bank101572 +101573 POINT(41.522204758650076 73.2835171656413) bank101573 +101574 POINT(39.84289487037137 74.5724804462226) bank101574 +101575 POINT(40.04147878783181 73.55191969241281) bank101575 +101576 POINT(40.13097108229756 74.42683237889874) bank101576 +101577 POINT(41.27378669664764 73.7805483749717) bank101577 +101578 POINT(40.101246339919655 73.58510935645333) bank101578 +101579 POINT(40.970285017273284 74.07431382735793) bank101579 +101580 POINT(41.50971274099695 73.22517971800632) bank101580 +101581 POINT(39.84044486059222 73.01563241968599) bank101581 +101582 POINT(41.51241778727721 73.72977783123437) bank101582 +101583 POINT(39.96367509050432 73.81722861016782) bank101583 +101584 POINT(39.93433680263195 73.66172306499537) bank101584 +101585 POINT(40.09239890172046 74.67715790158894) bank101585 +101586 POINT(40.35992551173149 74.70457979828174) bank101586 +101587 POINT(40.044626758852075 74.48220461967604) bank101587 +101588 POINT(40.14778337945992 74.03931829570935) bank101588 +101589 POINT(40.51622750227763 73.8265673872713) bank101589 +101590 POINT(40.784313301439525 73.61808419633304) bank101590 +101591 POINT(40.055654862922026 73.59929408099043) bank101591 +101592 POINT(41.24763033414195 73.8263816620993) bank101592 +101593 POINT(40.96408906510892 74.19023286816389) bank101593 +101594 POINT(41.165749113471364 73.64825904371374) bank101594 +101595 POINT(40.2585947829315 73.67079467362123) bank101595 +101596 POINT(40.40069882482442 74.34951449720296) bank101596 +101597 POINT(40.24262837551575 73.6534519397544) bank101597 +101598 POINT(39.7643096649416 73.3971958129745) bank101598 +101599 POINT(41.22098733988794 74.09043971782853) bank101599 +101600 POINT(41.683360338705064 73.84770536538046) bank101600 +101601 POINT(40.99708714286037 73.81610897404026) bank101601 +101602 POINT(40.01739997732789 74.34276484089189) bank101602 +101603 POINT(41.66538480550908 74.25017991009896) bank101603 +101604 POINT(40.9364838524262 74.59172099483645) bank101604 +101605 POINT(39.75836059368212 74.68186241585386) bank101605 +101606 POINT(41.61235408449985 73.79182842018963) bank101606 +101607 POINT(41.30639559844009 74.22725096691589) bank101607 +101608 POINT(40.7620164357692 73.06940069910472) bank101608 +101609 POINT(39.97250593466273 74.18543227178316) bank101609 +101610 POINT(41.23989887387551 73.80226548389174) bank101610 +101611 POINT(40.962921871962614 73.51830118135388) bank101611 +101612 POINT(41.17725944790551 74.39779108312672) bank101612 +101613 POINT(40.52622780914492 74.56880650169772) bank101613 +101614 POINT(40.13032379425087 73.59225175036792) bank101614 +101615 POINT(40.094243764602865 73.60294416464419) bank101615 +101616 POINT(39.91994529237352 73.4976138944786) bank101616 +101617 POINT(41.67456695203773 74.79296530622256) bank101617 +101618 POINT(40.81272982097505 73.55594425168144) bank101618 +101619 POINT(40.586855511198294 74.0339440743362) bank101619 +101620 POINT(41.350367145604594 74.76336198735174) bank101620 +101621 POINT(39.75486579820679 73.45048514982611) bank101621 +101622 POINT(40.1602510669042 74.47581985523102) bank101622 +101623 POINT(41.44995509498827 73.84945050471406) bank101623 +101624 POINT(40.5487521081818 73.29465607688122) bank101624 +101625 POINT(40.00904371120194 74.6199520507095) bank101625 +101626 POINT(40.376657240922505 74.44315010854285) bank101626 +101627 POINT(41.70362910621007 74.09600173934669) bank101627 +101628 POINT(39.84162837626266 74.15206501437052) bank101628 +101629 POINT(40.28656367223941 74.29687214040695) bank101629 +101630 POINT(41.24698312452961 74.48392375370355) bank101630 +101631 POINT(40.972507447491125 74.00587374955532) bank101631 +101632 POINT(39.811841159641006 73.59933909698343) bank101632 +101633 POINT(40.17726455190049 73.75417624379821) bank101633 +101634 POINT(40.38271753050369 73.75463376528721) bank101634 +101635 POINT(41.29601353570565 74.62892523749859) bank101635 +101636 POINT(41.260469373416775 74.45742962298466) bank101636 +101637 POINT(39.72116295936939 74.64174821772929) bank101637 +101638 POINT(40.49061602653134 73.99010592725976) bank101638 +101639 POINT(41.26553737278524 73.06108538675582) bank101639 +101640 POINT(39.80051019455264 74.5633541451477) bank101640 +101641 POINT(41.105665252691914 74.17044736029246) bank101641 +101642 POINT(40.38839490285216 74.37914083377935) bank101642 +101643 POINT(40.9470701882932 74.55100049439942) bank101643 +101644 POINT(40.57530823765527 73.45655254040587) bank101644 +101645 POINT(40.419190767199034 73.5424580464453) bank101645 +101646 POINT(40.5725353120811 73.59985272564882) bank101646 +101647 POINT(41.045738805675555 73.6619187107913) bank101647 +101648 POINT(40.943218637435564 73.33680520844513) bank101648 +101649 POINT(40.9946817055672 74.37998890566249) bank101649 +101650 POINT(39.8264937470254 74.56590654877857) bank101650 +101651 POINT(40.00116994334326 74.30524368179144) bank101651 +101652 POINT(41.63354667347154 73.60486787615963) bank101652 +101653 POINT(40.941990563165334 73.11768197779809) bank101653 +101654 POINT(41.26601269530151 73.37883860094631) bank101654 +101655 POINT(40.23256878439377 74.2458930051074) bank101655 +101656 POINT(40.14938713180572 73.26700543292209) bank101656 +101657 POINT(39.780710479114646 74.79100767137051) bank101657 +101658 POINT(41.358325108916105 73.48174425662701) bank101658 +101659 POINT(41.67940962687694 73.82475797434024) bank101659 +101660 POINT(40.44212209379169 74.7604177476783) bank101660 +101661 POINT(40.0791201765208 74.18569381368859) bank101661 +101662 POINT(40.2898560565485 73.70636401817114) bank101662 +101663 POINT(41.42831202257422 73.98704230211749) bank101663 +101664 POINT(39.94655106436169 73.11614054595654) bank101664 +101665 POINT(40.29188781247446 73.37635912230955) bank101665 +101666 POINT(40.668359725773826 73.50546388829302) bank101666 +101667 POINT(40.07965597188069 73.11614284333889) bank101667 +101668 POINT(40.08635052312384 74.71758455041652) bank101668 +101669 POINT(39.95658542868191 74.45767218883971) bank101669 +101670 POINT(41.236962358829565 73.2130958258528) bank101670 +101671 POINT(41.53553595576922 73.32007409482124) bank101671 +101672 POINT(40.16710192416948 74.93880643633092) bank101672 +101673 POINT(41.199566684803784 74.00336341558682) bank101673 +101674 POINT(40.29093276533865 74.39289177336494) bank101674 +101675 POINT(39.931588812425936 73.57057851028502) bank101675 +101676 POINT(40.154784194494034 73.87117468158445) bank101676 +101677 POINT(41.504520760361224 74.89563711639052) bank101677 +101678 POINT(41.18129205470227 74.26352003818127) bank101678 +101679 POINT(40.55470341000873 73.6075026464336) bank101679 +101680 POINT(40.43150606560485 73.12867838503504) bank101680 +101681 POINT(40.486968028776545 74.37197083561443) bank101681 +101682 POINT(40.04768787244676 73.88617126345235) bank101682 +101683 POINT(40.53755561217895 74.6801875116501) bank101683 +101684 POINT(40.841985126886954 73.28413588295957) bank101684 +101685 POINT(40.29533191221725 74.43400256777726) bank101685 +101686 POINT(40.058028183007714 74.5985364423992) bank101686 +101687 POINT(41.5579960252782 73.98307269551783) bank101687 +101688 POINT(39.731601431304796 74.8243543575107) bank101688 +101689 POINT(40.36211905241269 74.71867983865636) bank101689 +101690 POINT(40.944723043422506 73.88161611947021) bank101690 +101691 POINT(40.51789094346804 73.75314231760005) bank101691 +101692 POINT(41.48646479929384 73.61808410401024) bank101692 +101693 POINT(40.246289894159794 73.67038192108645) bank101693 +101694 POINT(40.99927934844374 73.17129892795184) bank101694 +101695 POINT(40.287935807353264 73.17945326284054) bank101695 +101696 POINT(40.192896395416255 74.44379253439085) bank101696 +101697 POINT(40.753359334179486 73.5622678168562) bank101697 +101698 POINT(40.24347893403489 73.26939832507189) bank101698 +101699 POINT(40.09606962575306 73.22161657845686) bank101699 +101700 POINT(41.603837044964 73.91980169972396) bank101700 +101701 POINT(40.355113828063445 74.7399555829713) bank101701 +101702 POINT(39.85942522973144 74.40941071997126) bank101702 +101703 POINT(41.23417732152113 73.38607850830391) bank101703 +101704 POINT(40.98944607743233 74.822559609476) bank101704 +101705 POINT(40.31947251923736 73.22377156718007) bank101705 +101706 POINT(41.7050672008003 73.73472190996948) bank101706 +101707 POINT(41.40969641190065 74.30429449351506) bank101707 +101708 POINT(40.005411670715176 74.54455226130092) bank101708 +101709 POINT(39.718693752947274 73.96069023587565) bank101709 +101710 POINT(40.40508822549086 74.77190921957431) bank101710 +101711 POINT(40.585578214511976 74.8531639263229) bank101711 +101712 POINT(40.74284285612116 73.97004745969006) bank101712 +101713 POINT(41.26470583906246 74.00674312838936) bank101713 +101714 POINT(41.674197435615795 73.82639267933217) bank101714 +101715 POINT(40.7616929333007 74.19225640142093) bank101715 +101716 POINT(40.78081106116525 73.47025523104915) bank101716 +101717 POINT(39.97044780782138 73.96288728395042) bank101717 +101718 POINT(40.91139396940297 73.35960233289481) bank101718 +101719 POINT(41.46264642218074 73.19526058615891) bank101719 +101720 POINT(41.306346438465695 74.35481830703444) bank101720 +101721 POINT(39.97286742970716 74.61806279282489) bank101721 +101722 POINT(40.49204162232206 73.3004329817533) bank101722 +101723 POINT(40.428337908416765 74.01077110133343) bank101723 +101724 POINT(40.081430496320415 73.97163480995866) bank101724 +101725 POINT(40.930773054735646 74.25427661596328) bank101725 +101726 POINT(40.695171236216176 73.88891106552185) bank101726 +101727 POINT(40.64107895569627 73.30397679951061) bank101727 +101728 POINT(39.93038163407575 73.83019823335363) bank101728 +101729 POINT(41.518868385166535 73.90118294199819) bank101729 +101730 POINT(40.6490742450697 73.99539005706549) bank101730 +101731 POINT(40.81309146935527 74.99108777889789) bank101731 +101732 POINT(41.27893939945219 74.83282825338608) bank101732 +101733 POINT(40.83914210199255 73.20926860698255) bank101733 +101734 POINT(41.486267805613146 73.51424823974068) bank101734 +101735 POINT(39.8369397996588 74.56737169512171) bank101735 +101736 POINT(40.942722085365936 74.37442838876149) bank101736 +101737 POINT(40.64267222785562 74.47433999647885) bank101737 +101738 POINT(39.71929685982059 74.05243793298327) bank101738 +101739 POINT(39.95942204599461 73.1743453444102) bank101739 +101740 POINT(41.18911420987451 73.8059934000685) bank101740 +101741 POINT(41.60612836385846 74.0384098221027) bank101741 +101742 POINT(41.57203098721166 73.09839645305253) bank101742 +101743 POINT(40.91825639431562 74.65700701662468) bank101743 +101744 POINT(41.324532116745736 74.42823987963001) bank101744 +101745 POINT(39.858776394787526 74.40133300842022) bank101745 +101746 POINT(40.05221277861964 73.78606457802314) bank101746 +101747 POINT(39.904892691447216 73.37354104243032) bank101747 +101748 POINT(41.646648385491666 73.9499326951729) bank101748 +101749 POINT(39.77902312989446 74.07450270672304) bank101749 +101750 POINT(40.36965414894165 74.09912231162804) bank101750 +101751 POINT(41.38412859603868 74.61979060259466) bank101751 +101752 POINT(40.26940883931738 73.10068207725234) bank101752 +101753 POINT(41.30278956067757 73.5131675875628) bank101753 +101754 POINT(40.4434378824234 73.1351749809273) bank101754 +101755 POINT(39.91178401575806 73.81038268870539) bank101755 +101756 POINT(39.95192131792122 74.58700791328286) bank101756 +101757 POINT(40.95846864037884 73.0800227168308) bank101757 +101758 POINT(40.807620478907154 73.27851312128456) bank101758 +101759 POINT(41.21672995002526 73.1857837552777) bank101759 +101760 POINT(40.2116191611452 74.7099549131511) bank101760 +101761 POINT(40.0039064858749 74.6903270405536) bank101761 +101762 POINT(40.31185845096514 73.89772169729748) bank101762 +101763 POINT(41.3709269991109 74.50628187278565) bank101763 +101764 POINT(41.40711614031232 74.72402013870438) bank101764 +101765 POINT(40.119522194597224 73.96589683308392) bank101765 +101766 POINT(41.141556447989316 74.8157887607965) bank101766 +101767 POINT(40.69298659720568 74.37714586202) bank101767 +101768 POINT(40.66066763708624 73.59358520658553) bank101768 +101769 POINT(39.747155190188835 73.6958611102703) bank101769 +101770 POINT(40.05778566877881 74.92777580711322) bank101770 +101771 POINT(40.21497657747124 73.15006441492389) bank101771 +101772 POINT(40.725988218905236 73.4957914183229) bank101772 +101773 POINT(40.32020833164692 73.36281648320397) bank101773 +101774 POINT(41.64643464900025 74.32043098767569) bank101774 +101775 POINT(41.68474496086639 73.78610478225502) bank101775 +101776 POINT(41.30529564086897 74.02983275340944) bank101776 +101777 POINT(39.947463034863354 73.79646437268259) bank101777 +101778 POINT(39.838774093804666 74.04185947323826) bank101778 +101779 POINT(40.4748247029421 74.06414091893296) bank101779 +101780 POINT(40.85686278329202 74.34547466514732) bank101780 +101781 POINT(40.9903119375349 74.2921754755057) bank101781 +101782 POINT(41.003916181356445 74.95510996131364) bank101782 +101783 POINT(41.293092661077665 74.93904214795478) bank101783 +101784 POINT(40.110676361169425 73.18580835156415) bank101784 +101785 POINT(39.785505864934215 73.52539706732522) bank101785 +101786 POINT(40.78726873084318 73.83183246225484) bank101786 +101787 POINT(41.128425719250345 73.03023339852815) bank101787 +101788 POINT(41.52330953962709 73.3013159961476) bank101788 +101789 POINT(39.996032067018994 73.27916831110191) bank101789 +101790 POINT(39.8633877794575 74.00087171113628) bank101790 +101791 POINT(40.25485984030606 74.92891297918838) bank101791 +101792 POINT(40.72412863517802 73.51364976986821) bank101792 +101793 POINT(40.83115009529986 73.71244243147922) bank101793 +101794 POINT(40.69106778239564 74.34647755848587) bank101794 +101795 POINT(40.24951868067954 74.06508960154314) bank101795 +101796 POINT(41.501448666350385 74.72494143308275) bank101796 +101797 POINT(41.108252864619786 73.43743662036351) bank101797 +101798 POINT(41.16315601710805 73.27930856327073) bank101798 +101799 POINT(40.36617327252956 74.47393787127695) bank101799 +101800 POINT(40.95465372369907 74.30472718836145) bank101800 +101801 POINT(41.55315801226291 73.7134748624054) bank101801 +101802 POINT(41.05285913375361 74.38473710403763) bank101802 +101803 POINT(40.81339841096211 73.10912284135227) bank101803 +101804 POINT(41.25515728921483 73.66706154194584) bank101804 +101805 POINT(39.804543857890096 73.67156670953028) bank101805 +101806 POINT(41.196167100974044 74.36798650428128) bank101806 +101807 POINT(40.792574777768635 74.80494035349771) bank101807 +101808 POINT(41.506201301012126 73.79939072127164) bank101808 +101809 POINT(41.708792621739114 74.35625475882381) bank101809 +101810 POINT(41.56999560103937 74.95778960904929) bank101810 +101811 POINT(41.331902582251764 73.70324649378054) bank101811 +101812 POINT(40.29937380444615 74.57345376475924) bank101812 +101813 POINT(39.91407613109118 73.99225399233042) bank101813 +101814 POINT(39.893382084841164 73.28997295514185) bank101814 +101815 POINT(40.52183657219931 73.43571060926722) bank101815 +101816 POINT(41.25957733950838 74.51298917384828) bank101816 +101817 POINT(40.34858320608085 74.80635092166936) bank101817 +101818 POINT(41.5421798482003 74.78760466382504) bank101818 +101819 POINT(41.641838053050705 74.8520227718538) bank101819 +101820 POINT(41.058851812639546 73.28705285358552) bank101820 +101821 POINT(40.14416683003472 73.04655321808093) bank101821 +101822 POINT(40.077561111308235 74.91092824343517) bank101822 +101823 POINT(41.22840034271808 73.14374088963373) bank101823 +101824 POINT(40.022965828155336 73.08372249850126) bank101824 +101825 POINT(40.81213770809677 74.24587824022291) bank101825 +101826 POINT(41.47944394565722 73.02778256606847) bank101826 +101827 POINT(40.96841196609382 73.0911142789866) bank101827 +101828 POINT(41.67093536190088 74.19436797784495) bank101828 +101829 POINT(40.975899749849745 73.38501013784081) bank101829 +101830 POINT(41.202166165333495 74.36090125404475) bank101830 +101831 POINT(40.03233244091364 74.46000527154746) bank101831 +101832 POINT(41.129273184620686 74.33085542682943) bank101832 +101833 POINT(40.8879278791499 73.79829913649974) bank101833 +101834 POINT(41.664850780496636 73.55038980649815) bank101834 +101835 POINT(40.167191745198785 73.23204156379856) bank101835 +101836 POINT(41.06789834820036 74.72746485902947) bank101836 +101837 POINT(41.62959251421688 73.83039010339729) bank101837 +101838 POINT(41.18080044611825 73.64250664047465) bank101838 +101839 POINT(39.796048227193936 74.44398726900629) bank101839 +101840 POINT(41.38884122290825 74.15227817476821) bank101840 +101841 POINT(39.863036491756844 74.23085113319345) bank101841 +101842 POINT(40.43429528809364 74.3250067851024) bank101842 +101843 POINT(41.48293641878886 73.08637552545206) bank101843 +101844 POINT(40.64993552264933 74.43590120624033) bank101844 +101845 POINT(39.96563114046645 74.41174014307386) bank101845 +101846 POINT(41.31816262954085 74.01988116757956) bank101846 +101847 POINT(41.5803111870203 74.4868603612838) bank101847 +101848 POINT(41.44406201450383 74.62669017078862) bank101848 +101849 POINT(40.607741407294604 73.68714139884892) bank101849 +101850 POINT(40.29562476692555 74.36552445438329) bank101850 +101851 POINT(40.26727112114299 74.39372583050863) bank101851 +101852 POINT(41.29204210770682 74.95038764518253) bank101852 +101853 POINT(39.83269069688981 74.73928367617772) bank101853 +101854 POINT(41.38274060534679 74.98871016562485) bank101854 +101855 POINT(40.87718575417384 73.92152857639644) bank101855 +101856 POINT(40.77205887559976 74.40973352670518) bank101856 +101857 POINT(40.732561139202666 73.60657236763917) bank101857 +101858 POINT(41.238482602229105 74.21584059494351) bank101858 +101859 POINT(39.78277133061509 73.50160659411993) bank101859 +101860 POINT(40.742577481769935 74.7749945387104) bank101860 +101861 POINT(41.44303572213114 73.08274889507092) bank101861 +101862 POINT(39.94803707907512 73.70006564768582) bank101862 +101863 POINT(41.10323451798919 74.23660576831004) bank101863 +101864 POINT(39.91189224550701 74.57220074856374) bank101864 +101865 POINT(41.478423765397224 74.69069964144967) bank101865 +101866 POINT(41.47357211137122 73.17121998323613) bank101866 +101867 POINT(40.54950411424857 73.97664940099324) bank101867 +101868 POINT(40.05479814191614 74.3393025308897) bank101868 +101869 POINT(41.079725260721744 73.50952591348864) bank101869 +101870 POINT(41.47691585705798 73.09101390070872) bank101870 +101871 POINT(40.21065487357856 74.81808500894579) bank101871 +101872 POINT(40.591700219334776 74.18497123405317) bank101872 +101873 POINT(41.191232988456534 74.36210847807024) bank101873 +101874 POINT(41.52922957329399 73.5534188780339) bank101874 +101875 POINT(40.74929438889132 74.86595917720584) bank101875 +101876 POINT(40.886237940672736 74.33203979410352) bank101876 +101877 POINT(40.8463578146129 74.44742441812184) bank101877 +101878 POINT(41.00571914736641 74.29254928552797) bank101878 +101879 POINT(40.12502632605848 73.35782528463837) bank101879 +101880 POINT(41.67891558888028 74.01672344586308) bank101880 +101881 POINT(41.501274548550725 75.00431047404813) bank101881 +101882 POINT(40.88138899183394 73.01117084654756) bank101882 +101883 POINT(40.14125867656263 73.16753081560476) bank101883 +101884 POINT(40.38505587796884 73.1137315997808) bank101884 +101885 POINT(40.999402452165164 73.16004346701543) bank101885 +101886 POINT(40.40998649284899 73.08714193621317) bank101886 +101887 POINT(40.723212277471674 74.00956919930107) bank101887 +101888 POINT(40.65033860724041 73.98298283336536) bank101888 +101889 POINT(40.21973902639402 74.75866475326137) bank101889 +101890 POINT(39.809330728155516 73.4967901700138) bank101890 +101891 POINT(41.4905531425522 74.78546504234815) bank101891 +101892 POINT(40.21209440930704 74.82001084905929) bank101892 +101893 POINT(40.411406459934184 73.42894099229899) bank101893 +101894 POINT(40.702719913625636 74.17453789745848) bank101894 +101895 POINT(40.82061681186248 73.16056355509926) bank101895 +101896 POINT(39.87831192883657 74.25166400497194) bank101896 +101897 POINT(41.64363997336382 74.63710270719162) bank101897 +101898 POINT(41.05947318946579 73.148881980171) bank101898 +101899 POINT(39.72200459420993 73.15157595886942) bank101899 +101900 POINT(41.68011402457971 74.76601852168322) bank101900 +101901 POINT(40.40105729815753 74.17668069062177) bank101901 +101902 POINT(41.33547905482584 73.50018252460521) bank101902 +101903 POINT(40.10919930613272 74.19895776328208) bank101903 +101904 POINT(41.220531237479506 73.7841808538397) bank101904 +101905 POINT(40.488458388863485 73.44605961614526) bank101905 +101906 POINT(40.19717636535137 74.25447233133198) bank101906 +101907 POINT(41.338553170076864 73.15737417888002) bank101907 +101908 POINT(40.812504175646325 73.16471306458972) bank101908 +101909 POINT(39.76931022317128 73.46914055373838) bank101909 +101910 POINT(40.824646738247196 74.48857359553888) bank101910 +101911 POINT(40.31381294836652 73.38076881902185) bank101911 +101912 POINT(41.48441885479964 74.03096262743034) bank101912 +101913 POINT(41.553545923997284 74.64714459871462) bank101913 +101914 POINT(40.15677153472616 74.74562000900566) bank101914 +101915 POINT(40.327378318647895 74.9436030101685) bank101915 +101916 POINT(41.1533903105928 74.63869176268807) bank101916 +101917 POINT(40.1115969742915 73.10576835678204) bank101917 +101918 POINT(40.49226091059078 74.07519913435584) bank101918 +101919 POINT(40.45322232446172 74.90392638411174) bank101919 +101920 POINT(39.91371459318159 73.83821852595572) bank101920 +101921 POINT(41.51501115592836 74.96394225567182) bank101921 +101922 POINT(40.221345842988484 73.72676852736898) bank101922 +101923 POINT(40.14655700899877 73.5002670614266) bank101923 +101924 POINT(40.25165934276522 74.56810390174876) bank101924 +101925 POINT(40.41935953442381 73.35953116642142) bank101925 +101926 POINT(40.5684560691855 74.64019708303357) bank101926 +101927 POINT(41.0278114944088 73.16412359876924) bank101927 +101928 POINT(39.87618889914102 74.62430014071087) bank101928 +101929 POINT(40.492603766845455 74.2653357864889) bank101929 +101930 POINT(41.41480762759584 74.5449228507899) bank101930 +101931 POINT(41.07934227601632 73.88472395739709) bank101931 +101932 POINT(41.34423235329492 74.26313750787237) bank101932 +101933 POINT(41.53082554162011 74.90773335102442) bank101933 +101934 POINT(40.82668834886478 73.20348883274104) bank101934 +101935 POINT(40.10292448303055 74.12724672118435) bank101935 +101936 POINT(40.607657136315986 73.7571396236702) bank101936 +101937 POINT(39.780855535006815 73.94588818515962) bank101937 +101938 POINT(40.20032343884276 74.48197875379994) bank101938 +101939 POINT(40.60188191556378 73.76134133554027) bank101939 +101940 POINT(40.65882510710613 74.36969925297734) bank101940 +101941 POINT(39.80633770132037 73.79182279464369) bank101941 +101942 POINT(41.15266540793507 73.15623746649494) bank101942 +101943 POINT(40.129710080502264 73.15988243959293) bank101943 +101944 POINT(40.8981822773085 73.47353281201944) bank101944 +101945 POINT(40.51301507107763 73.6750439306258) bank101945 +101946 POINT(40.284682751758524 74.92022162811045) bank101946 +101947 POINT(41.450726556204856 73.58972369643696) bank101947 +101948 POINT(40.78325385626837 74.4957363128795) bank101948 +101949 POINT(40.10806864672256 73.83747380417825) bank101949 +101950 POINT(40.50627410058868 73.13675332944878) bank101950 +101951 POINT(40.38527357414599 73.68718458157126) bank101951 +101952 POINT(41.16108559097126 74.16356555664684) bank101952 +101953 POINT(40.65949075701062 74.40069039397711) bank101953 +101954 POINT(41.410626797430005 74.032119371458) bank101954 +101955 POINT(40.78706740961114 73.01459392662238) bank101955 +101956 POINT(40.323310958789875 74.71543577637946) bank101956 +101957 POINT(40.884598209752625 74.86104525172735) bank101957 +101958 POINT(41.06503559925878 73.63222804559886) bank101958 +101959 POINT(40.77060390282489 74.8542525846353) bank101959 +101960 POINT(39.86392651685456 74.83793030755974) bank101960 +101961 POINT(41.45377895558262 74.69638621020567) bank101961 +101962 POINT(40.45764787656352 74.85963891646095) bank101962 +101963 POINT(41.23406189478167 74.89648392161024) bank101963 +101964 POINT(40.075459709928495 73.20269017818553) bank101964 +101965 POINT(41.618308681656764 73.6407379599285) bank101965 +101966 POINT(41.33554853370479 74.21358883004413) bank101966 +101967 POINT(40.47102993732804 74.45748513559475) bank101967 +101968 POINT(41.38685008243508 73.80098966785222) bank101968 +101969 POINT(40.042489298304496 73.86772799392932) bank101969 +101970 POINT(40.04233446694939 74.03624031435352) bank101970 +101971 POINT(41.06047899532978 73.76879186592876) bank101971 +101972 POINT(40.36406946435624 74.57654351458217) bank101972 +101973 POINT(41.54742432261545 74.63142807926197) bank101973 +101974 POINT(41.22401889709179 74.45801692697749) bank101974 +101975 POINT(40.583392537653175 74.76496861322265) bank101975 +101976 POINT(41.26074512194499 73.14548210144662) bank101976 +101977 POINT(40.261854614526655 74.13273492155776) bank101977 +101978 POINT(40.87444746551331 75.00206988366371) bank101978 +101979 POINT(41.6793040403539 73.59624774755413) bank101979 +101980 POINT(40.07381406437975 74.0797504856356) bank101980 +101981 POINT(40.16693826123661 74.8644085499116) bank101981 +101982 POINT(40.3165612570299 73.09978965989802) bank101982 +101983 POINT(40.91290702093348 73.50213538795049) bank101983 +101984 POINT(39.71614682425497 74.89834446368062) bank101984 +101985 POINT(40.616293414046055 73.77026071531931) bank101985 +101986 POINT(40.087039415750624 73.67791703161862) bank101986 +101987 POINT(40.09380015204028 74.49099740912916) bank101987 +101988 POINT(40.23939108957657 73.60992648604294) bank101988 +101989 POINT(40.29001445048431 74.03938126045503) bank101989 +101990 POINT(40.64831098796752 74.31680266122105) bank101990 +101991 POINT(41.514314502865005 73.22122850319664) bank101991 +101992 POINT(40.583600163417096 73.85367028360447) bank101992 +101993 POINT(41.4796524267362 73.53382912226294) bank101993 +101994 POINT(41.05557333114505 74.04110576304913) bank101994 +101995 POINT(40.23679288905971 73.67476922922587) bank101995 +101996 POINT(41.053739384551214 73.72686814683678) bank101996 +101997 POINT(40.822228808272286 74.93463574524766) bank101997 +101998 POINT(39.845329482613494 74.17100723602998) bank101998 +101999 POINT(41.32077506528342 74.2667664548236) bank101999 +102000 POINT(40.803231620607974 73.45160888670969) bank102000 +102001 POINT(41.24478111940765 74.75142472826165) bank102001 +102002 POINT(41.02464045755635 74.96908277765598) bank102002 +102003 POINT(41.422969554055655 73.58839184732241) bank102003 +102004 POINT(40.39650350157135 74.15463418907379) bank102004 +102005 POINT(40.36525678111048 73.54027691868663) bank102005 +102006 POINT(41.554443073992196 73.89069646289569) bank102006 +102007 POINT(40.35486184755172 73.71527622479117) bank102007 +102008 POINT(40.53858208651315 73.46307968867637) bank102008 +102009 POINT(40.92548162237647 73.35619357085653) bank102009 +102010 POINT(40.10316085727249 73.23675963536868) bank102010 +102011 POINT(41.70582501758547 74.32342566474995) bank102011 +102012 POINT(40.246968359054655 73.44077169613358) bank102012 +102013 POINT(40.14642227904511 73.20382167863902) bank102013 +102014 POINT(41.20054710825746 74.17015880143366) bank102014 +102015 POINT(41.58467373557749 74.65512208533099) bank102015 +102016 POINT(40.371400881369425 73.95840738573173) bank102016 +102017 POINT(40.65568106815045 74.02083563140492) bank102017 +102018 POINT(40.41203568869313 74.83681957665016) bank102018 +102019 POINT(40.9751619221856 73.72083886757653) bank102019 +102020 POINT(40.50925112340399 73.9043004152119) bank102020 +102021 POINT(41.19240004071582 74.67564572049285) bank102021 +102022 POINT(40.17262095077672 74.94189953005663) bank102022 +102023 POINT(41.10456633798351 74.02010436545847) bank102023 +102024 POINT(41.109356387980945 73.27241199448825) bank102024 +102025 POINT(40.342062810223354 74.16704583815653) bank102025 +102026 POINT(41.47682561979099 73.97543495157376) bank102026 +102027 POINT(41.36270388872558 74.73441933774768) bank102027 +102028 POINT(39.87307874926876 74.89973106340354) bank102028 +102029 POINT(41.22664355009272 73.38645573157862) bank102029 +102030 POINT(41.241592498405836 74.64187481342678) bank102030 +102031 POINT(40.05389589388693 74.22826069032256) bank102031 +102032 POINT(40.48223730969532 73.865110024928) bank102032 +102033 POINT(39.95976950553244 73.3238283232018) bank102033 +102034 POINT(40.7278051873907 73.63538899488061) bank102034 +102035 POINT(40.70262082002163 74.04715882129129) bank102035 +102036 POINT(40.825017178381735 73.95385987320117) bank102036 +102037 POINT(41.153617542567865 73.4366818859082) bank102037 +102038 POINT(40.60564165516025 74.62541666866852) bank102038 +102039 POINT(40.087871645460226 74.69480084904868) bank102039 +102040 POINT(40.48213937950367 74.59106066255082) bank102040 +102041 POINT(40.97169771259327 73.27738732681915) bank102041 +102042 POINT(40.0688826227241 73.35507592623367) bank102042 +102043 POINT(40.29754325775547 73.64317225011588) bank102043 +102044 POINT(40.59205605090117 74.01372604779857) bank102044 +102045 POINT(40.767223324812136 73.88200017034926) bank102045 +102046 POINT(40.725759424699454 74.21396644236786) bank102046 +102047 POINT(39.87924495359144 74.33588627523886) bank102047 +102048 POINT(39.841947393253115 74.35112260505969) bank102048 +102049 POINT(41.29731547313865 74.21767665185307) bank102049 +102050 POINT(41.015698372214 73.27559575109913) bank102050 +102051 POINT(41.35712261782924 73.26128803714563) bank102051 +102052 POINT(40.977656919296805 73.83748968976984) bank102052 +102053 POINT(40.430208781279305 74.34421535669375) bank102053 +102054 POINT(39.8610226365694 73.81481758293049) bank102054 +102055 POINT(40.758316225892614 74.65911757377037) bank102055 +102056 POINT(40.70780424000183 74.24142747133047) bank102056 +102057 POINT(41.509843316057136 74.78051004632154) bank102057 +102058 POINT(41.56285923463257 73.85631072162683) bank102058 +102059 POINT(40.54906000498238 73.03023685440583) bank102059 +102060 POINT(41.380694709838814 73.03949742710705) bank102060 +102061 POINT(39.92664648954766 74.48684518435275) bank102061 +102062 POINT(41.25235791034461 73.08721371979708) bank102062 +102063 POINT(40.88065259415669 74.36744313639888) bank102063 +102064 POINT(40.905567830882774 73.4562601978298) bank102064 +102065 POINT(40.34934922005148 74.07940347561738) bank102065 +102066 POINT(40.38093212320017 74.45067268104175) bank102066 +102067 POINT(40.98683944870349 73.17509545811933) bank102067 +102068 POINT(41.29895809934353 74.01260978065301) bank102068 +102069 POINT(41.460911433256314 74.24558011885216) bank102069 +102070 POINT(40.80541839195317 74.35486406336305) bank102070 +102071 POINT(41.476795036235586 74.01333438994132) bank102071 +102072 POINT(40.37825082280554 73.47670601099489) bank102072 +102073 POINT(40.93233073222384 74.27719021277166) bank102073 +102074 POINT(41.56459170114772 74.23312717158133) bank102074 +102075 POINT(40.71696165916867 74.3776072942779) bank102075 +102076 POINT(41.4946479249918 73.1169239255493) bank102076 +102077 POINT(39.89079875839491 74.51813078369824) bank102077 +102078 POINT(39.81515292334468 74.86266883713789) bank102078 +102079 POINT(40.14054778042141 73.16320198592909) bank102079 +102080 POINT(40.55627822611935 73.57313374688839) bank102080 +102081 POINT(41.481030196297496 74.38578396469306) bank102081 +102082 POINT(40.70714714357734 73.60044799983002) bank102082 +102083 POINT(41.47143059877687 74.11641384355917) bank102083 +102084 POINT(40.44587227541112 73.39753868458985) bank102084 +102085 POINT(40.321286879137695 73.11241759273194) bank102085 +102086 POINT(41.69868005640796 74.41107052101614) bank102086 +102087 POINT(41.04291639333278 73.42913403612381) bank102087 +102088 POINT(40.195877748857264 74.41310834179794) bank102088 +102089 POINT(40.67031715867143 74.50091058542483) bank102089 +102090 POINT(41.19522895079758 73.44765032085853) bank102090 +102091 POINT(40.0577394737633 73.73397699529372) bank102091 +102092 POINT(40.00162267162832 74.43632166462821) bank102092 +102093 POINT(40.70889281412587 73.94780097532103) bank102093 +102094 POINT(41.63735221140633 74.13773076510623) bank102094 +102095 POINT(40.157893739052376 74.82882265889457) bank102095 +102096 POINT(41.15941295092236 74.78649064491081) bank102096 +102097 POINT(40.01916378536393 74.76291545272443) bank102097 +102098 POINT(40.22978643249441 74.98975908992595) bank102098 +102099 POINT(40.801840503755514 74.78741106137295) bank102099 +102100 POINT(41.15946841006812 73.56550003591292) bank102100 +102101 POINT(39.90460634082172 74.54711462898099) bank102101 +102102 POINT(41.18523223709508 73.56239511663257) bank102102 +102103 POINT(40.04607458251522 74.39916284100943) bank102103 +102104 POINT(41.35054332001194 74.41985263385047) bank102104 +102105 POINT(41.54317865299423 73.29757302449882) bank102105 +102106 POINT(41.49843557417522 74.02958206954348) bank102106 +102107 POINT(40.83285956093636 74.16787230529924) bank102107 +102108 POINT(39.916046201754114 73.7710342569898) bank102108 +102109 POINT(40.620122939890976 74.34935805740457) bank102109 +102110 POINT(41.321890830063865 74.57873672413653) bank102110 +102111 POINT(41.40200347412929 74.8458627674305) bank102111 +102112 POINT(41.41367618063621 74.96055489111625) bank102112 +102113 POINT(39.82576901050159 74.62362416527561) bank102113 +102114 POINT(39.87815177917345 74.97888498272793) bank102114 +102115 POINT(40.74947464508673 74.07897797425738) bank102115 +102116 POINT(40.98814154697101 74.16115313383786) bank102116 +102117 POINT(40.867711639524586 73.40120807108632) bank102117 +102118 POINT(40.28335645419746 73.37997676707066) bank102118 +102119 POINT(41.39247625214955 74.2503532319897) bank102119 +102120 POINT(40.677319221202055 74.60697515199412) bank102120 +102121 POINT(40.766693969939524 74.23304120925263) bank102121 +102122 POINT(40.509827660639345 73.66271498936831) bank102122 +102123 POINT(40.713047573420276 73.74055517303681) bank102123 +102124 POINT(40.63233370676365 73.57792596907899) bank102124 +102125 POINT(41.16158392882075 74.96039159452151) bank102125 +102126 POINT(39.9802976138144 73.79581915685898) bank102126 +102127 POINT(41.50148640389322 74.20635659782138) bank102127 +102128 POINT(41.0601709766668 73.5791040089902) bank102128 +102129 POINT(41.03692304773771 73.56544427022074) bank102129 +102130 POINT(39.83063210150877 74.66402335386033) bank102130 +102131 POINT(39.819307270809354 74.0161212339545) bank102131 +102132 POINT(41.70704660703273 73.72527297138133) bank102132 +102133 POINT(41.28872088974638 73.06400902492496) bank102133 +102134 POINT(40.565206137649014 73.0654987169026) bank102134 +102135 POINT(40.24675312242815 74.66809152777594) bank102135 +102136 POINT(40.26475743146716 73.7843238826507) bank102136 +102137 POINT(40.22564384690177 74.54490561813682) bank102137 +102138 POINT(41.64015452475472 73.35484819209785) bank102138 +102139 POINT(40.23316522089619 74.00519663912613) bank102139 +102140 POINT(41.13866067311451 74.19736265589862) bank102140 +102141 POINT(40.62377018522027 73.13762670004098) bank102141 +102142 POINT(41.128384148955774 74.07482135453706) bank102142 +102143 POINT(40.43041848885096 74.23750873792841) bank102143 +102144 POINT(40.80370413354864 74.01865133454719) bank102144 +102145 POINT(40.08113732475325 74.46676126709522) bank102145 +102146 POINT(40.929378805711146 74.6049654822255) bank102146 +102147 POINT(41.52760763332732 73.40013319983026) bank102147 +102148 POINT(40.627309924421816 74.40535385147717) bank102148 +102149 POINT(41.04819868681067 74.84414911638034) bank102149 +102150 POINT(39.82986162792272 73.25514252554997) bank102150 +102151 POINT(39.84340486735203 74.68676318122695) bank102151 +102152 POINT(41.59913279645919 74.0081291078139) bank102152 +102153 POINT(41.4926191840161 74.913157541008) bank102153 +102154 POINT(40.86137663498556 73.99642038755326) bank102154 +102155 POINT(40.69263394331001 74.7552140807787) bank102155 +102156 POINT(40.8247433945397 74.5719121344407) bank102156 +102157 POINT(41.690810048111956 74.6192729964334) bank102157 +102158 POINT(41.328195797979454 74.72082047489891) bank102158 +102159 POINT(40.972344547401505 74.74494768846077) bank102159 +102160 POINT(41.48180550277872 73.31034527858787) bank102160 +102161 POINT(40.321823014368285 74.42345104406165) bank102161 +102162 POINT(39.86716550442268 74.74775488933918) bank102162 +102163 POINT(40.93650101781904 73.99698477095224) bank102163 +102164 POINT(40.05680625996273 74.43727604711376) bank102164 +102165 POINT(39.86646418478622 74.52881708119533) bank102165 +102166 POINT(41.01172623868675 73.05807623436222) bank102166 +102167 POINT(41.16943283602987 73.97810962469612) bank102167 +102168 POINT(39.76436414386081 73.07158743184041) bank102168 +102169 POINT(40.70024469589455 73.83587921399372) bank102169 +102170 POINT(40.29230151577395 73.34360149317129) bank102170 +102171 POINT(41.552887592036214 73.09710128457517) bank102171 +102172 POINT(40.526320197431126 74.92882636101386) bank102172 +102173 POINT(40.85026307397125 74.66599192428207) bank102173 +102174 POINT(41.00801979263206 74.98790410488085) bank102174 +102175 POINT(41.37965769689002 74.11872529747478) bank102175 +102176 POINT(40.5314309694395 74.01589002325925) bank102176 +102177 POINT(40.448292755603546 74.94253792061495) bank102177 +102178 POINT(39.956242215560685 73.6810945714128) bank102178 +102179 POINT(40.806171342111426 73.27137210155226) bank102179 +102180 POINT(40.67138309710216 73.8738437129858) bank102180 +102181 POINT(39.93222915640784 73.03866741646517) bank102181 +102182 POINT(41.41011061198712 73.51721081049207) bank102182 +102183 POINT(39.80917153203385 73.97580192492481) bank102183 +102184 POINT(41.20213236754278 74.33640717105511) bank102184 +102185 POINT(40.65764307991178 73.65240075511952) bank102185 +102186 POINT(40.98708605080605 73.7087286096606) bank102186 +102187 POINT(41.264802137885724 74.74337555580436) bank102187 +102188 POINT(40.80010046056169 73.05508514073948) bank102188 +102189 POINT(40.994596248069854 74.59968435681046) bank102189 +102190 POINT(40.13233966703968 73.89852044461455) bank102190 +102191 POINT(41.32727601956117 73.43371425666454) bank102191 +102192 POINT(39.84817333823911 74.1774454103257) bank102192 +102193 POINT(40.33265310967738 73.21288781288764) bank102193 +102194 POINT(40.40674858491988 74.99194591173966) bank102194 +102195 POINT(40.68754407092302 73.02546548456326) bank102195 +102196 POINT(39.76194936914274 74.6320616288332) bank102196 +102197 POINT(40.91728204434331 73.3642713085194) bank102197 +102198 POINT(40.657268073239166 73.12242966819113) bank102198 +102199 POINT(40.947877585945754 73.78123699091937) bank102199 +102200 POINT(40.52591325495759 73.88877900651542) bank102200 +102201 POINT(41.48194913823768 73.26896690215185) bank102201 +102202 POINT(40.76067970294191 74.49399328916694) bank102202 +102203 POINT(40.301023511324956 74.86501683758772) bank102203 +102204 POINT(40.11754761548626 74.01558172808333) bank102204 +102205 POINT(41.23144745563463 73.51214419643878) bank102205 +102206 POINT(40.467288457497496 73.07004316995936) bank102206 +102207 POINT(41.55664588205874 74.09587361750651) bank102207 +102208 POINT(41.50059519469624 73.66319443499667) bank102208 +102209 POINT(40.42533337498153 74.85363823036495) bank102209 +102210 POINT(41.49906029747011 74.91779388057803) bank102210 +102211 POINT(40.71805831452502 74.04556712533629) bank102211 +102212 POINT(39.984343110398285 74.3141616644142) bank102212 +102213 POINT(41.337355937931946 73.80536944629932) bank102213 +102214 POINT(41.63601377596589 74.58771844407401) bank102214 +102215 POINT(40.09464040211867 74.58110024485705) bank102215 +102216 POINT(41.42753256090514 74.4538570483262) bank102216 +102217 POINT(40.75811171103663 74.93720098530304) bank102217 +102218 POINT(40.7913569848773 73.35491019643756) bank102218 +102219 POINT(40.863929313898666 73.24109304751812) bank102219 +102220 POINT(40.79663541431241 74.76677117369852) bank102220 +102221 POINT(41.144903150783925 73.5457134602473) bank102221 +102222 POINT(41.257330721609996 73.9928645352936) bank102222 +102223 POINT(40.13140494702764 74.27048799196298) bank102223 +102224 POINT(40.12062646880543 73.42446536108378) bank102224 +102225 POINT(40.44562317128985 73.48176006803602) bank102225 +102226 POINT(40.09430215387249 73.08190122439693) bank102226 +102227 POINT(41.38384710548302 73.55649739095962) bank102227 +102228 POINT(41.48870818533586 74.2129816796724) bank102228 +102229 POINT(40.16778456575999 74.66344600942999) bank102229 +102230 POINT(41.20038587763322 73.07937471364981) bank102230 +102231 POINT(40.0090486527719 74.6185006964877) bank102231 +102232 POINT(41.20936274153118 73.04019441403828) bank102232 +102233 POINT(40.12622676554092 74.83594696490464) bank102233 +102234 POINT(41.542126565156885 73.55065674180835) bank102234 +102235 POINT(41.04251688037081 74.29841086856861) bank102235 +102236 POINT(40.81410406147317 73.40749993928749) bank102236 +102237 POINT(40.48348678752837 74.24324962504237) bank102237 +102238 POINT(40.04537433843706 73.4937689210164) bank102238 +102239 POINT(39.969375211331695 74.91773606437556) bank102239 +102240 POINT(40.785201836052345 74.75364539316539) bank102240 +102241 POINT(40.48661990602946 73.35243675023453) bank102241 +102242 POINT(40.21605697858842 74.0225227495315) bank102242 +102243 POINT(41.51962054283843 74.5848456495303) bank102243 +102244 POINT(39.86522184085714 73.48619004956389) bank102244 +102245 POINT(39.88240656748926 74.34523707641274) bank102245 +102246 POINT(40.60931939415597 74.73191370203702) bank102246 +102247 POINT(41.48058926445764 74.0400635686783) bank102247 +102248 POINT(40.229859236131716 74.51559533593445) bank102248 +102249 POINT(39.92262528593136 74.68343411035481) bank102249 +102250 POINT(39.720540114502136 74.70057204056016) bank102250 +102251 POINT(39.86421284324174 73.18504333769391) bank102251 +102252 POINT(41.50119353535674 73.3065991801424) bank102252 +102253 POINT(41.205746861307006 74.71013962407015) bank102253 +102254 POINT(40.85443112112783 74.67886119321871) bank102254 +102255 POINT(41.50643503076646 73.54675642468979) bank102255 +102256 POINT(40.20256978273648 74.54110624729294) bank102256 +102257 POINT(39.737959746537584 73.24765632446821) bank102257 +102258 POINT(40.920704752926326 74.7871375253434) bank102258 +102259 POINT(41.01264266714309 73.6430021099342) bank102259 +102260 POINT(41.446765780947544 74.77482348209057) bank102260 +102261 POINT(40.46883698957363 74.91979474539907) bank102261 +102262 POINT(40.82259922945173 74.00770652098237) bank102262 +102263 POINT(40.709307395829605 73.94030241900977) bank102263 +102264 POINT(40.53934983718279 74.10931975695074) bank102264 +102265 POINT(41.544935845634924 73.70060292502312) bank102265 +102266 POINT(39.76294961954442 73.9739060291765) bank102266 +102267 POINT(41.30954363000486 74.83188496506381) bank102267 +102268 POINT(41.47137106696278 73.13778857816601) bank102268 +102269 POINT(41.048513777259586 73.69131316438605) bank102269 +102270 POINT(40.253647536489424 74.65812680879925) bank102270 +102271 POINT(41.159204315746024 74.30345053341689) bank102271 +102272 POINT(39.983978687323095 74.07158314003107) bank102272 +102273 POINT(40.18429028679182 73.36997976825083) bank102273 +102274 POINT(40.422571640356175 73.8862278111575) bank102274 +102275 POINT(40.200440253648864 73.56625104581936) bank102275 +102276 POINT(40.07479797028637 73.83972300226526) bank102276 +102277 POINT(41.12423003542671 73.49378400397532) bank102277 +102278 POINT(41.16915475475212 74.05578447675306) bank102278 +102279 POINT(40.28451073426654 74.30118584602542) bank102279 +102280 POINT(40.49684824075185 74.46534722052705) bank102280 +102281 POINT(40.06503557142278 74.87465259496305) bank102281 +102282 POINT(40.29799181017277 73.9429931285378) bank102282 +102283 POINT(40.966394421433165 74.10782262841359) bank102283 +102284 POINT(40.29353349293709 74.18704779219927) bank102284 +102285 POINT(40.81756834307138 73.41117475189851) bank102285 +102286 POINT(40.41874406404624 73.35922483443294) bank102286 +102287 POINT(40.439989467155556 73.75615953154802) bank102287 +102288 POINT(39.929010721886456 73.65017240995824) bank102288 +102289 POINT(40.21944587116393 73.87853113482312) bank102289 +102290 POINT(41.09034882186517 74.1989573962538) bank102290 +102291 POINT(39.79773078251982 74.09359465169531) bank102291 +102292 POINT(40.14073368117347 73.83513098058846) bank102292 +102293 POINT(39.99815099311693 74.44903003444384) bank102293 +102294 POINT(40.12979576402201 74.70013467067629) bank102294 +102295 POINT(41.321579974042734 74.37728774413321) bank102295 +102296 POINT(40.38236440475639 73.56317641310369) bank102296 +102297 POINT(41.27448663007989 73.86340693085249) bank102297 +102298 POINT(41.3766189939367 73.09015065004785) bank102298 +102299 POINT(40.36717420949634 74.58054784767718) bank102299 +102300 POINT(40.75120549783067 74.38606052916631) bank102300 +102301 POINT(39.88745285852756 74.25194985993741) bank102301 +102302 POINT(40.649322929675705 74.04811203915348) bank102302 +102303 POINT(41.19215418392699 74.82431563026313) bank102303 +102304 POINT(40.34379815423089 73.7244066952002) bank102304 +102305 POINT(40.03618363837005 74.0779599330116) bank102305 +102306 POINT(41.127676629155616 74.98932941359448) bank102306 +102307 POINT(40.92134763461259 73.32458692139781) bank102307 +102308 POINT(40.627018546277235 73.31344539191048) bank102308 +102309 POINT(41.69294971388919 74.87788014360781) bank102309 +102310 POINT(40.93337256568006 74.73588019512268) bank102310 +102311 POINT(41.25365044115831 73.83711632141434) bank102311 +102312 POINT(39.852840508717165 74.70212081911195) bank102312 +102313 POINT(41.48120434402035 74.13673622955578) bank102313 +102314 POINT(41.52679747924731 73.37544277659616) bank102314 +102315 POINT(41.02695974591252 73.23707884081581) bank102315 +102316 POINT(41.0891516124138 74.32832447639686) bank102316 +102317 POINT(40.98722373982973 73.2515868854989) bank102317 +102318 POINT(41.63787353839128 73.41725912733001) bank102318 +102319 POINT(40.53794690945165 74.35832300530416) bank102319 +102320 POINT(40.07717970651699 73.83361468377774) bank102320 +102321 POINT(41.18634711376421 74.07507314017141) bank102321 +102322 POINT(41.240142362959986 73.02843219747824) bank102322 +102323 POINT(40.76037322955723 74.43709381051366) bank102323 +102324 POINT(40.04634823489426 74.992785478158) bank102324 +102325 POINT(40.45626894782383 74.93231850012377) bank102325 +102326 POINT(40.29389156838786 74.73940865347608) bank102326 +102327 POINT(41.38452592351076 74.15685144974859) bank102327 +102328 POINT(40.86089393130966 74.74819554236407) bank102328 +102329 POINT(40.34776742670381 74.57215857885069) bank102329 +102330 POINT(40.415353445932624 74.62323359660418) bank102330 +102331 POINT(41.1631383156561 74.52297483003606) bank102331 +102332 POINT(40.14199049612126 74.31368202329661) bank102332 +102333 POINT(40.78994221224377 74.34615808528194) bank102333 +102334 POINT(41.64116171228642 73.68063205323645) bank102334 +102335 POINT(40.205395272875705 73.28348467859348) bank102335 +102336 POINT(41.054810594626154 73.38467598199449) bank102336 +102337 POINT(41.66130155686605 74.09475204853527) bank102337 +102338 POINT(39.75583935735466 74.33849171834483) bank102338 +102339 POINT(41.649985288388024 73.11071724257853) bank102339 +102340 POINT(40.511439654786535 73.86213404420042) bank102340 +102341 POINT(41.37315423698082 73.77253349253935) bank102341 +102342 POINT(41.54751677652385 74.63561851484022) bank102342 +102343 POINT(41.054410931509864 74.33376118709364) bank102343 +102344 POINT(40.11236258438671 74.03274620515774) bank102344 +102345 POINT(40.09772356532411 74.76246156573121) bank102345 +102346 POINT(41.06631507345871 74.35668767262659) bank102346 +102347 POINT(40.396489146856 73.4758453958507) bank102347 +102348 POINT(40.18224561418445 74.55188598331324) bank102348 +102349 POINT(40.660102325381864 73.43838149735333) bank102349 +102350 POINT(41.146695227976245 75.00214492272164) bank102350 +102351 POINT(40.940242420928875 73.99253327605135) bank102351 +102352 POINT(40.103101955373546 73.60355313154733) bank102352 +102353 POINT(40.829809829126276 73.81473181758388) bank102353 +102354 POINT(41.07915330844963 74.83978764106425) bank102354 +102355 POINT(40.983355349086295 73.14728806565881) bank102355 +102356 POINT(40.3776978684989 74.44445917747882) bank102356 +102357 POINT(40.740875572328 74.59288663032261) bank102357 +102358 POINT(40.52344563797683 74.20042146718976) bank102358 +102359 POINT(41.39488026678921 74.46759890086855) bank102359 +102360 POINT(41.28466135862227 74.67913984821178) bank102360 +102361 POINT(40.56152695677468 73.82859044606161) bank102361 +102362 POINT(41.250914278390354 74.4193217435304) bank102362 +102363 POINT(41.706861759162784 74.16035568370351) bank102363 +102364 POINT(39.86152297324281 73.56376741783451) bank102364 +102365 POINT(40.31375409574788 74.04150680366874) bank102365 +102366 POINT(40.05614860935038 74.58274899729527) bank102366 +102367 POINT(40.968190627315174 74.106865738318) bank102367 +102368 POINT(40.2538962047934 74.90569353638932) bank102368 +102369 POINT(40.21683171262684 73.71804983814519) bank102369 +102370 POINT(41.086668352917414 73.11714507209443) bank102370 +102371 POINT(40.42286894072805 73.8613719147865) bank102371 +102372 POINT(39.739808319849566 73.19822453867418) bank102372 +102373 POINT(40.19274544175625 73.80689218833766) bank102373 +102374 POINT(41.202968727296934 73.02670899126419) bank102374 +102375 POINT(40.52807453859689 74.08710867072426) bank102375 +102376 POINT(40.220758288687136 74.47579229358895) bank102376 +102377 POINT(41.08238605250963 73.06716168467177) bank102377 +102378 POINT(41.143144829429865 73.0150303422992) bank102378 +102379 POINT(41.62663229120532 74.66957066490528) bank102379 +102380 POINT(40.01936222383799 73.69656623384206) bank102380 +102381 POINT(39.82671590246022 73.88862447218851) bank102381 +102382 POINT(41.067098439278716 74.57275100874502) bank102382 +102383 POINT(40.185884811597425 73.99671948520643) bank102383 +102384 POINT(40.72670725289093 73.07636642212087) bank102384 +102385 POINT(41.297511107611925 74.58767180120621) bank102385 +102386 POINT(41.17119460999795 73.30010766542283) bank102386 +102387 POINT(40.01838086763065 74.48007104401567) bank102387 +102388 POINT(39.8191910775936 74.57580607347506) bank102388 +102389 POINT(40.5354515395387 74.7370910130309) bank102389 +102390 POINT(40.4492796367892 74.06216697299588) bank102390 +102391 POINT(41.63160156801765 73.94101721699838) bank102391 +102392 POINT(41.25745401417024 74.83660387201986) bank102392 +102393 POINT(41.23885015600443 73.30035010739347) bank102393 +102394 POINT(40.22610229196492 73.39564483904938) bank102394 +102395 POINT(41.0847346582922 73.84414502635238) bank102395 +102396 POINT(41.61212802649756 73.58058813770835) bank102396 +102397 POINT(40.73037208928177 74.26150255700121) bank102397 +102398 POINT(40.29709511273304 73.90485532904786) bank102398 +102399 POINT(40.954582818880624 74.70640330880899) bank102399 +102400 POINT(40.809544307893006 73.16896241847499) bank102400 +102401 POINT(40.801691155217796 73.39807567602874) bank102401 +102402 POINT(41.562248071269046 74.36989464623498) bank102402 +102403 POINT(39.843410199614866 74.43015020084304) bank102403 +102404 POINT(41.271233373043806 73.318946799266) bank102404 +102405 POINT(40.718184516319575 74.36177414540761) bank102405 +102406 POINT(39.93587132773186 73.114577641682) bank102406 +102407 POINT(40.95336050530348 73.5683759275853) bank102407 +102408 POINT(40.60323028497873 74.8232755618491) bank102408 +102409 POINT(39.84959605202029 73.17295828692878) bank102409 +102410 POINT(41.37828279299081 73.3318745267104) bank102410 +102411 POINT(39.849338622198566 73.80193458850914) bank102411 +102412 POINT(40.20253087501487 73.21941818186379) bank102412 +102413 POINT(41.1049157145043 74.32552781228846) bank102413 +102414 POINT(40.7362526043419 74.93396420277617) bank102414 +102415 POINT(40.91128830344647 73.39324040347753) bank102415 +102416 POINT(41.67592399125759 73.478944810973) bank102416 +102417 POINT(41.02098384316357 73.42002296555098) bank102417 +102418 POINT(39.79834652963011 73.62266789249449) bank102418 +102419 POINT(41.1685629552365 73.70254042460836) bank102419 +102420 POINT(41.63726512246382 73.57453557326924) bank102420 +102421 POINT(41.086762748538376 74.74023160488197) bank102421 +102422 POINT(40.500812792406954 74.54346494995916) bank102422 +102423 POINT(41.50730889161289 73.04029421548356) bank102423 +102424 POINT(41.31324147996318 73.36462344537725) bank102424 +102425 POINT(40.304722743248455 74.87774681681444) bank102425 +102426 POINT(40.29687030503897 73.96031557661712) bank102426 +102427 POINT(39.97120344588704 73.84512290944322) bank102427 +102428 POINT(40.85199380502871 73.42240607047813) bank102428 +102429 POINT(40.55889218793018 73.33214444221444) bank102429 +102430 POINT(39.889811872673754 73.73630192417555) bank102430 +102431 POINT(40.06347330295649 74.35381700913344) bank102431 +102432 POINT(39.81415736521757 74.42795866883954) bank102432 +102433 POINT(41.60367768380672 74.53082648105884) bank102433 +102434 POINT(41.000554996699705 73.42322185656496) bank102434 +102435 POINT(40.16475766018172 74.04431315159904) bank102435 +102436 POINT(41.133302456047964 73.85145761151618) bank102436 +102437 POINT(40.49074959952057 73.50447250452808) bank102437 +102438 POINT(41.30537700704527 73.35623075771566) bank102438 +102439 POINT(40.78514544563979 74.98346083070406) bank102439 +102440 POINT(40.03134063131007 74.09182220003056) bank102440 +102441 POINT(41.55521437258369 74.01959397935885) bank102441 +102442 POINT(41.45128317890029 73.88591343904763) bank102442 +102443 POINT(39.926483219692315 74.05610162306462) bank102443 +102444 POINT(41.66931280722065 74.74827233831728) bank102444 +102445 POINT(40.846957956371206 73.69949827896939) bank102445 +102446 POINT(39.85768489118785 73.55994761630191) bank102446 +102447 POINT(40.438535542859825 74.82774336079862) bank102447 +102448 POINT(40.34378948344625 73.9042589746206) bank102448 +102449 POINT(40.801506979692505 73.5847659755417) bank102449 +102450 POINT(41.0408906569215 74.92489095318625) bank102450 +102451 POINT(41.52831540387642 73.6187360923746) bank102451 +102452 POINT(40.54037625445088 73.21158020118781) bank102452 +102453 POINT(41.211041868263344 73.54035185150462) bank102453 +102454 POINT(40.41176947734132 73.75499078873872) bank102454 +102455 POINT(40.22807692123064 74.62994793594785) bank102455 +102456 POINT(40.69656009958019 74.06737442115775) bank102456 +102457 POINT(40.02931823902054 74.17892228806168) bank102457 +102458 POINT(41.36652054192297 73.13655466350323) bank102458 +102459 POINT(40.38978487933217 74.51736961702683) bank102459 +102460 POINT(39.760879181200366 74.08246415283696) bank102460 +102461 POINT(41.690302196872125 73.82134528943755) bank102461 +102462 POINT(41.248692869707966 73.84520624416318) bank102462 +102463 POINT(40.2970136064863 74.74955199668545) bank102463 +102464 POINT(40.16528242159535 73.02552277584188) bank102464 +102465 POINT(41.620491055193966 73.6964756187162) bank102465 +102466 POINT(41.63633660080568 74.2161199120715) bank102466 +102467 POINT(40.253023903728504 74.84526316637842) bank102467 +102468 POINT(40.824949156413076 73.71640186828944) bank102468 +102469 POINT(40.074260259160056 74.77340039461998) bank102469 +102470 POINT(40.16757597568101 74.85451844043905) bank102470 +102471 POINT(40.735587555514094 73.18828230857717) bank102471 +102472 POINT(41.354812135661504 73.76507957617662) bank102472 +102473 POINT(40.50946970702936 74.51590543976053) bank102473 +102474 POINT(40.22767389290229 73.11947389473114) bank102474 +102475 POINT(41.28864255290379 73.95861539274154) bank102475 +102476 POINT(40.70816906552457 74.66631635660592) bank102476 +102477 POINT(41.42957691689596 73.22206791958816) bank102477 +102478 POINT(40.16776940632414 73.36526497541526) bank102478 +102479 POINT(40.37459999642529 73.51621385521725) bank102479 +102480 POINT(39.88992582481433 73.0373261643549) bank102480 +102481 POINT(40.778884183202244 73.91823856352809) bank102481 +102482 POINT(41.341479851931226 73.97983021234207) bank102482 +102483 POINT(39.78134823591156 74.33648176503769) bank102483 +102484 POINT(41.69848528214433 74.3819377415526) bank102484 +102485 POINT(40.80632148857468 73.22962372427247) bank102485 +102486 POINT(41.62266465794169 73.04568557777915) bank102486 +102487 POINT(40.20182429196827 73.86161568333343) bank102487 +102488 POINT(40.78362924448613 73.35619230787492) bank102488 +102489 POINT(40.0150352527086 74.88637019343699) bank102489 +102490 POINT(40.42903343424419 73.14716674330008) bank102490 +102491 POINT(39.777102891690916 74.04740114866888) bank102491 +102492 POINT(40.90331858210909 74.12638218682885) bank102492 +102493 POINT(41.005211132234464 73.36255953850403) bank102493 +102494 POINT(41.329410920076114 73.40937343709842) bank102494 +102495 POINT(41.70689346262491 73.75775751348434) bank102495 +102496 POINT(41.06799817977936 74.07017930367336) bank102496 +102497 POINT(40.02005680088518 73.90128392575893) bank102497 +102498 POINT(40.89988542086916 73.16617595346928) bank102498 +102499 POINT(40.10449518593488 73.14521827911351) bank102499 +102500 POINT(40.85065522858284 73.60639305924069) bank102500 +102501 POINT(40.401483279787826 73.74824706104994) bank102501 +102502 POINT(40.17655286141959 73.67867502043289) bank102502 +102503 POINT(40.33705364226387 74.73765030144872) bank102503 +102504 POINT(41.160463001066496 73.70524746849928) bank102504 +102505 POINT(40.744083952952536 73.25327795475253) bank102505 +102506 POINT(41.38841051481445 73.92801637865453) bank102506 +102507 POINT(41.604748170858365 73.85197887186705) bank102507 +102508 POINT(40.13521881750326 73.03380888953502) bank102508 +102509 POINT(39.76862824607125 74.39309124900464) bank102509 +102510 POINT(40.66516345918058 73.98787919013505) bank102510 +102511 POINT(40.81796188969127 73.60945855924834) bank102511 +102512 POINT(40.87469347565817 73.36469167861824) bank102512 +102513 POINT(40.8276663939096 74.04104821000945) bank102513 +102514 POINT(41.23447975916464 74.57762166056555) bank102514 +102515 POINT(41.4701906210453 73.66378095990439) bank102515 +102516 POINT(39.854830276045185 73.71834204460343) bank102516 +102517 POINT(40.343946610445535 74.91559684261036) bank102517 +102518 POINT(41.11308194705277 73.64198899374125) bank102518 +102519 POINT(39.87645992375758 74.04135083499536) bank102519 +102520 POINT(40.75568604160059 74.10579064982339) bank102520 +102521 POINT(41.346819041758025 74.48929669388335) bank102521 +102522 POINT(40.390089614508575 73.6989581895808) bank102522 +102523 POINT(39.74199278758501 74.34892514517968) bank102523 +102524 POINT(41.03442685665356 73.36079322992731) bank102524 +102525 POINT(41.539427212133724 74.3379434693051) bank102525 +102526 POINT(40.66352925300444 73.33180982392676) bank102526 +102527 POINT(40.29416102197573 73.95293925900535) bank102527 +102528 POINT(39.943397582161246 74.96330556349658) bank102528 +102529 POINT(39.94413503637342 74.82456489146378) bank102529 +102530 POINT(41.16976465074171 74.90439440112837) bank102530 +102531 POINT(39.88945238610638 73.67820134911858) bank102531 +102532 POINT(41.70174213621033 73.25963202212537) bank102532 +102533 POINT(41.39412165299102 74.1523725709013) bank102533 +102534 POINT(41.32191622716886 74.20660371617406) bank102534 +102535 POINT(40.17090064499594 73.83989800139797) bank102535 +102536 POINT(40.88522367263929 74.750208156381) bank102536 +102537 POINT(39.75840032721943 73.96643294366156) bank102537 +102538 POINT(39.91997773260723 74.5725521762688) bank102538 +102539 POINT(40.60084365830299 74.73153620313538) bank102539 +102540 POINT(39.92377760977519 74.05197619497665) bank102540 +102541 POINT(39.98901956170393 73.06404853380235) bank102541 +102542 POINT(40.980847352229404 73.17039030973386) bank102542 +102543 POINT(40.05024302561465 74.59123451155918) bank102543 +102544 POINT(39.90191829683064 74.55686903600709) bank102544 +102545 POINT(40.831479236919215 73.84305204967752) bank102545 +102546 POINT(39.72869408096295 73.13466719921233) bank102546 +102547 POINT(40.775115683840504 73.27906304294154) bank102547 +102548 POINT(40.97238039644135 74.95544455941204) bank102548 +102549 POINT(41.57579542470721 74.83389231649986) bank102549 +102550 POINT(41.138461045422424 74.19832105220141) bank102550 +102551 POINT(40.577936636711286 74.08394336841268) bank102551 +102552 POINT(40.42169143825065 74.66365168982847) bank102552 +102553 POINT(39.8206120638167 74.45633221486442) bank102553 +102554 POINT(41.604416090797734 73.2045595349709) bank102554 +102555 POINT(40.27481907927218 74.77909903932208) bank102555 +102556 POINT(40.37965449387228 74.2730117300163) bank102556 +102557 POINT(40.21927349866796 74.38503717766564) bank102557 +102558 POINT(40.180719737524186 74.31991798867114) bank102558 +102559 POINT(40.71853545844492 73.8466040618107) bank102559 +102560 POINT(40.03074999537515 73.19450967656908) bank102560 +102561 POINT(40.265193903356185 73.17814917279925) bank102561 +102562 POINT(40.17199364515542 74.73033820334749) bank102562 +102563 POINT(40.95264031386006 73.56544451344924) bank102563 +102564 POINT(40.682308118904004 74.89107486883562) bank102564 +102565 POINT(39.96096840714714 74.57269941930257) bank102565 +102566 POINT(40.70082005799066 73.54534158812602) bank102566 +102567 POINT(40.842114691775016 74.46632836050894) bank102567 +102568 POINT(40.66067289881078 74.49273224591744) bank102568 +102569 POINT(40.27239485846685 75.00127016903316) bank102569 +102570 POINT(40.81032738977715 73.2030819464776) bank102570 +102571 POINT(40.20964796630002 73.13085046837185) bank102571 +102572 POINT(41.08450018094862 74.8007885796226) bank102572 +102573 POINT(39.88079805975261 73.26999470231581) bank102573 +102574 POINT(41.09934750901494 73.02254254090987) bank102574 +102575 POINT(40.31363510098249 73.63312342140492) bank102575 +102576 POINT(39.73804303453046 73.06667787232219) bank102576 +102577 POINT(40.732881231161755 74.52350396028905) bank102577 +102578 POINT(39.86821490484 73.97148656639317) bank102578 +102579 POINT(41.347750642693335 74.65381379250748) bank102579 +102580 POINT(39.89688551232769 74.78228469119684) bank102580 +102581 POINT(41.430662438600336 74.87856342885766) bank102581 +102582 POINT(40.27327331280437 74.55977974126452) bank102582 +102583 POINT(40.27141537782661 73.46544807087689) bank102583 +102584 POINT(40.77903414340467 74.14714448609728) bank102584 +102585 POINT(40.72792326106382 74.06403167557713) bank102585 +102586 POINT(40.89534489546219 73.2255191020466) bank102586 +102587 POINT(40.50127978727556 74.9447715953574) bank102587 +102588 POINT(40.72454546360111 74.72325778295269) bank102588 +102589 POINT(41.42331145184915 74.17408668986506) bank102589 +102590 POINT(40.68418578247064 74.57110047675987) bank102590 +102591 POINT(40.82771722596716 74.20737642485722) bank102591 +102592 POINT(40.41639917602324 73.40643083338328) bank102592 +102593 POINT(40.17866496340743 73.43223653834082) bank102593 +102594 POINT(41.65634300023039 74.21805394373945) bank102594 +102595 POINT(41.31793803745236 73.73972453220972) bank102595 +102596 POINT(40.2419244352655 74.7657214485709) bank102596 +102597 POINT(39.99545874393671 74.66286665159124) bank102597 +102598 POINT(40.50681624023846 74.37101091509659) bank102598 +102599 POINT(39.97534079977365 74.11283669187753) bank102599 +102600 POINT(39.801839090488926 73.97754665332481) bank102600 +102601 POINT(41.685127464872934 74.17358895260023) bank102601 +102602 POINT(40.55144235986204 74.55526649268482) bank102602 +102603 POINT(41.106583410582395 73.79522878990822) bank102603 +102604 POINT(41.01075393495459 74.73493485351705) bank102604 +102605 POINT(40.78903280327447 74.86026140162201) bank102605 +102606 POINT(41.16930244498168 74.20702105046018) bank102606 +102607 POINT(40.42521429760659 73.89750858098078) bank102607 +102608 POINT(40.10562507201133 74.03446275319571) bank102608 +102609 POINT(39.92458387395993 73.30910220360066) bank102609 +102610 POINT(41.63674595795971 74.49762908350833) bank102610 +102611 POINT(40.288219849960356 74.8508552726209) bank102611 +102612 POINT(41.33697265393706 74.54038368213942) bank102612 +102613 POINT(40.3971712951387 73.94375024536949) bank102613 +102614 POINT(41.0426172835468 73.07526316934596) bank102614 +102615 POINT(40.321161319242414 73.77370538085015) bank102615 +102616 POINT(40.27224408091628 73.92208327319648) bank102616 +102617 POINT(41.54902752060628 73.20512754195337) bank102617 +102618 POINT(39.80234094625649 74.43506098477062) bank102618 +102619 POINT(40.66884773899843 74.3668820467846) bank102619 +102620 POINT(41.50048294937936 74.26467345817146) bank102620 +102621 POINT(40.42446128831225 73.51240067713476) bank102621 +102622 POINT(40.34015957948789 74.17398963581772) bank102622 +102623 POINT(40.92093688068917 73.98385631620563) bank102623 +102624 POINT(41.00437398896356 73.7684688418743) bank102624 +102625 POINT(40.01890075600059 74.38594183377971) bank102625 +102626 POINT(40.126372756937656 73.37460757978751) bank102626 +102627 POINT(40.81173920089371 73.62811681279273) bank102627 +102628 POINT(40.260841124846515 74.33491721201933) bank102628 +102629 POINT(41.41346134578113 74.51376124930212) bank102629 +102630 POINT(39.80629863933837 74.99546611157717) bank102630 +102631 POINT(40.47013917450341 74.1783500629745) bank102631 +102632 POINT(41.43449278217892 73.82684071626892) bank102632 +102633 POINT(39.86817305604855 74.23784021671071) bank102633 +102634 POINT(39.85825699110735 73.62306909682746) bank102634 +102635 POINT(41.5014719302495 74.28276061853295) bank102635 +102636 POINT(41.04237614504559 73.2595397243558) bank102636 +102637 POINT(40.461433745054066 74.61752976307552) bank102637 +102638 POINT(41.081208049210936 74.4797388355442) bank102638 +102639 POINT(41.31135708951873 74.03451567106136) bank102639 +102640 POINT(40.61636187626789 73.6196229437451) bank102640 +102641 POINT(41.043499038317755 74.45740180171651) bank102641 +102642 POINT(41.04000534019195 74.3848727926393) bank102642 +102643 POINT(40.96119524535195 74.66488825959165) bank102643 +102644 POINT(40.89954511676023 73.23395538917211) bank102644 +102645 POINT(41.315500930374824 73.83713499566966) bank102645 +102646 POINT(41.419165434923514 74.85931213507075) bank102646 +102647 POINT(40.11069620090068 74.54761076907612) bank102647 +102648 POINT(40.50294844686497 73.37286925546566) bank102648 +102649 POINT(40.544806141236684 73.92846494958766) bank102649 +102650 POINT(41.668729667483234 74.12583888805646) bank102650 +102651 POINT(41.229358770690155 73.5210280195972) bank102651 +102652 POINT(41.35581942741339 73.60146544276851) bank102652 +102653 POINT(40.64130825146036 74.50927307511915) bank102653 +102654 POINT(41.04966156717825 74.02562891234341) bank102654 +102655 POINT(39.77956403320214 73.63542186820673) bank102655 +102656 POINT(41.45201423508074 74.31557022764987) bank102656 +102657 POINT(40.95777464266565 73.96700613620612) bank102657 +102658 POINT(41.132294030959066 74.65390754757621) bank102658 +102659 POINT(41.64485546145902 73.76071303360396) bank102659 +102660 POINT(40.18172637365793 73.8613854476109) bank102660 +102661 POINT(40.123137280117824 73.94605323701867) bank102661 +102662 POINT(41.1310823885146 73.47178066053405) bank102662 +102663 POINT(39.97132745813802 74.70965912691919) bank102663 +102664 POINT(40.78767574048169 74.35261596678153) bank102664 +102665 POINT(40.86581175763927 74.54010058403522) bank102665 +102666 POINT(40.433751198669334 73.54341286848955) bank102666 +102667 POINT(41.4858674547632 73.33309080625256) bank102667 +102668 POINT(39.796608474549686 73.96126264908557) bank102668 +102669 POINT(40.51818469819167 73.0713984581481) bank102669 +102670 POINT(40.48462660252685 73.20663085717675) bank102670 +102671 POINT(39.96764729044368 74.15407505019076) bank102671 +102672 POINT(41.45340691053224 74.53259259112559) bank102672 +102673 POINT(40.501850870694376 74.7737163140601) bank102673 +102674 POINT(40.36266984595449 74.47753325366278) bank102674 +102675 POINT(40.51268344474629 74.75671619967346) bank102675 +102676 POINT(39.89861460936339 73.28851590324923) bank102676 +102677 POINT(39.83954611195212 74.2859791467351) bank102677 +102678 POINT(41.69301104690478 74.3574078382273) bank102678 +102679 POINT(41.147210870737084 73.75583468253114) bank102679 +102680 POINT(40.54321064907987 73.42603869754143) bank102680 +102681 POINT(40.640075345413024 74.39630148723002) bank102681 +102682 POINT(41.19164209084514 74.90447794319901) bank102682 +102683 POINT(40.312128149903195 73.34570506917663) bank102683 +102684 POINT(39.76376794190454 74.0651929202102) bank102684 +102685 POINT(41.40043242710977 73.86757453006669) bank102685 +102686 POINT(41.381495550557894 73.48092238162035) bank102686 +102687 POINT(40.44779030071554 74.48161149562611) bank102687 +102688 POINT(40.35445608438436 74.89389809207923) bank102688 +102689 POINT(40.19616164539509 73.74796661002823) bank102689 +102690 POINT(40.10487442173228 73.56768879136409) bank102690 +102691 POINT(39.85022132163655 74.21137811701718) bank102691 +102692 POINT(40.078020143197264 74.59479591804151) bank102692 +102693 POINT(39.868694688330145 74.22266797324436) bank102693 +102694 POINT(40.19736445989084 73.96073663895324) bank102694 +102695 POINT(41.59147950816733 74.42071903773463) bank102695 +102696 POINT(40.08277574983769 74.10531773045494) bank102696 +102697 POINT(40.69136771348224 73.77529838321009) bank102697 +102698 POINT(39.98066779105097 73.92842345783525) bank102698 +102699 POINT(40.149236953192556 73.02344204044174) bank102699 +102700 POINT(41.37531665185011 73.81969110870602) bank102700 +102701 POINT(39.848130870896185 74.97438449436285) bank102701 +102702 POINT(40.55883795163148 74.75652957435881) bank102702 +102703 POINT(41.48115567764298 73.9743157880082) bank102703 +102704 POINT(41.623745968765675 73.79047197775505) bank102704 +102705 POINT(41.55721342335016 74.6177970856185) bank102705 +102706 POINT(40.895022201305444 74.8747726531569) bank102706 +102707 POINT(41.02631546316864 74.84854823368225) bank102707 +102708 POINT(39.80488372080029 74.95869152688688) bank102708 +102709 POINT(41.34098468318185 74.60390690302083) bank102709 +102710 POINT(39.839115270453775 74.72356413808443) bank102710 +102711 POINT(40.58719355212438 74.79738694124967) bank102711 +102712 POINT(41.070063570055105 74.56613269528007) bank102712 +102713 POINT(40.31624723871386 73.11593503494834) bank102713 +102714 POINT(41.16701770785269 73.72567148706541) bank102714 +102715 POINT(39.9253495579449 74.0908374605145) bank102715 +102716 POINT(40.7300780142851 74.57721926132285) bank102716 +102717 POINT(41.28642425413656 74.67811711133923) bank102717 +102718 POINT(40.109060326718016 74.03429042775473) bank102718 +102719 POINT(39.998075657727185 74.0461765575443) bank102719 +102720 POINT(40.54782722860398 74.29251309219472) bank102720 +102721 POINT(40.228260205359355 73.22829453266415) bank102721 +102722 POINT(40.15628446139288 73.8081751760595) bank102722 +102723 POINT(41.65640756248601 74.79955271366762) bank102723 +102724 POINT(41.335045690554125 74.33802433216677) bank102724 +102725 POINT(41.23749332062841 74.64137575920215) bank102725 +102726 POINT(41.324412048882934 74.88957156645819) bank102726 +102727 POINT(41.17622073369898 74.69976568193775) bank102727 +102728 POINT(41.30018548769546 73.44132978896481) bank102728 +102729 POINT(40.32642759088937 74.10373114301771) bank102729 +102730 POINT(40.19209589578516 74.12988020428479) bank102730 +102731 POINT(39.73248627902666 74.82742840242403) bank102731 +102732 POINT(40.617545526780525 73.74955678960467) bank102732 +102733 POINT(39.9458814578957 74.97756943684642) bank102733 +102734 POINT(41.67673813114994 74.4777595469723) bank102734 +102735 POINT(40.0015514226835 74.60266259774096) bank102735 +102736 POINT(41.62454123429567 73.80169588671542) bank102736 +102737 POINT(40.062249110267715 73.52036805431187) bank102737 +102738 POINT(41.235539563786965 74.55835160915068) bank102738 +102739 POINT(40.96761040181639 73.91911556040142) bank102739 +102740 POINT(40.47453840466043 74.3167814869764) bank102740 +102741 POINT(39.8013566437846 75.00461309389769) bank102741 +102742 POINT(40.16685358426287 73.81006563267651) bank102742 +102743 POINT(40.662948103633795 73.36334720133215) bank102743 +102744 POINT(41.4786361960142 74.65771824690782) bank102744 +102745 POINT(41.40108523383428 73.53085770783717) bank102745 +102746 POINT(41.00755472263601 73.5911328708678) bank102746 +102747 POINT(41.05477229402034 74.34544372229746) bank102747 +102748 POINT(41.11668803609335 74.67796921060778) bank102748 +102749 POINT(40.48110269217616 74.5097358328164) bank102749 +102750 POINT(40.78969976806683 73.73529534117145) bank102750 +102751 POINT(39.72304450800619 73.6528717894913) bank102751 +102752 POINT(40.965423660002536 73.77256285660584) bank102752 +102753 POINT(40.401242049686 73.49089475017236) bank102753 +102754 POINT(40.873035364973 74.08113050965433) bank102754 +102755 POINT(40.103420092046136 74.21198943966527) bank102755 +102756 POINT(41.63618678257899 74.56971279651826) bank102756 +102757 POINT(41.42587615435087 74.15268232739477) bank102757 +102758 POINT(41.39226517178648 74.13666715084996) bank102758 +102759 POINT(41.53786342952289 74.53101666921802) bank102759 +102760 POINT(41.18778628456844 73.27851421868566) bank102760 +102761 POINT(41.07362571719152 73.5280775090966) bank102761 +102762 POINT(39.85113198115317 73.98946126499972) bank102762 +102763 POINT(40.94328348145465 73.38844860630994) bank102763 +102764 POINT(41.396468603360766 73.14186423304359) bank102764 +102765 POINT(40.273761164290065 73.57504405186815) bank102765 +102766 POINT(41.2808135861969 73.90411479069608) bank102766 +102767 POINT(41.562302216847236 73.37644055390159) bank102767 +102768 POINT(41.01589702204645 73.32526972333012) bank102768 +102769 POINT(40.44151341694123 73.28165726052212) bank102769 +102770 POINT(39.86686394612905 73.09678831783361) bank102770 +102771 POINT(41.7030953960721 73.44879819297643) bank102771 +102772 POINT(40.67357461063167 74.28400432995882) bank102772 +102773 POINT(41.62815186782657 74.04687746488287) bank102773 +102774 POINT(40.115908972217994 74.39479354533243) bank102774 +102775 POINT(41.403105330348154 73.84432735072185) bank102775 +102776 POINT(40.75582719043008 73.43577131005448) bank102776 +102777 POINT(40.00518908783184 74.06153587719318) bank102777 +102778 POINT(39.772676138417516 73.4653381592601) bank102778 +102779 POINT(40.86255146560402 73.33224204877627) bank102779 +102780 POINT(40.82278579677349 73.55686402078665) bank102780 +102781 POINT(41.54489083254756 74.54894654743033) bank102781 +102782 POINT(41.629262245807865 74.04374486261268) bank102782 +102783 POINT(40.31314150499574 74.00555398059277) bank102783 +102784 POINT(39.749235086653925 74.91930666254488) bank102784 +102785 POINT(41.11412571537464 73.73956240211815) bank102785 +102786 POINT(40.64258162227224 73.12561580566347) bank102786 +102787 POINT(41.074175490472584 73.6446456309858) bank102787 +102788 POINT(41.66374972999415 74.31002364313942) bank102788 +102789 POINT(40.593608919530595 74.57834117209823) bank102789 +102790 POINT(40.372938068976325 73.29590507448886) bank102790 +102791 POINT(40.588313169987515 73.99997689157543) bank102791 +102792 POINT(41.11917830230202 73.54795815897917) bank102792 +102793 POINT(40.04029350357239 74.10985019700786) bank102793 +102794 POINT(41.61370592586045 74.57246231836483) bank102794 +102795 POINT(40.695889124068216 74.04220976089536) bank102795 +102796 POINT(40.813506575848585 73.44539943977686) bank102796 +102797 POINT(41.23766058769651 74.44869481991168) bank102797 +102798 POINT(41.26715905160846 74.68997677718933) bank102798 +102799 POINT(40.36847445015989 74.32572211280097) bank102799 +102800 POINT(41.676541500529765 74.25958435076159) bank102800 +102801 POINT(41.420493515590664 74.17129151614911) bank102801 +102802 POINT(41.32765303046198 73.72328549897492) bank102802 +102803 POINT(41.61828914574784 73.83862802277767) bank102803 +102804 POINT(40.64656387038904 74.16248664306778) bank102804 +102805 POINT(40.75209613267355 74.40957023992306) bank102805 +102806 POINT(39.790372172004695 73.33300542969087) bank102806 +102807 POINT(39.7183772732984 73.34360589400048) bank102807 +102808 POINT(40.34397981888767 74.48517482037096) bank102808 +102809 POINT(40.220681926010265 74.4048568784308) bank102809 +102810 POINT(41.64866195677788 74.19180214730079) bank102810 +102811 POINT(41.04385777318289 74.45869896341547) bank102811 +102812 POINT(40.505418646389614 74.3844529757962) bank102812 +102813 POINT(39.99952421148436 73.6186519196706) bank102813 +102814 POINT(41.308637675959716 74.06426150305774) bank102814 +102815 POINT(41.22258568548523 74.71950450879814) bank102815 +102816 POINT(40.91609722054087 73.2535455149906) bank102816 +102817 POINT(40.2355267706208 73.33727885170919) bank102817 +102818 POINT(41.027339564497986 73.51354809574737) bank102818 +102819 POINT(40.40478300130943 74.57304612048085) bank102819 +102820 POINT(41.5778939378983 74.49722863114324) bank102820 +102821 POINT(40.47729489029487 74.92225263117615) bank102821 +102822 POINT(40.46186011286355 73.79835400791663) bank102822 +102823 POINT(41.01717269321633 74.88605879536414) bank102823 +102824 POINT(40.831112335271406 74.44941819111902) bank102824 +102825 POINT(41.2667073741103 73.31085282105819) bank102825 +102826 POINT(40.70797487405112 74.04495974395525) bank102826 +102827 POINT(40.968748003636236 74.40312804819943) bank102827 +102828 POINT(39.74124164698564 74.87403253661245) bank102828 +102829 POINT(41.3060237919313 74.29839553269704) bank102829 +102830 POINT(40.0911439713729 73.28906875303748) bank102830 +102831 POINT(41.28957222288525 74.43957162784865) bank102831 +102832 POINT(40.83319635522883 74.84186035036566) bank102832 +102833 POINT(39.76524703742253 74.53876390829126) bank102833 +102834 POINT(40.97966191722844 73.56353298576927) bank102834 +102835 POINT(40.22136222000945 73.83868012713936) bank102835 +102836 POINT(41.5935114268546 74.1092951369881) bank102836 +102837 POINT(40.084733947194934 73.08305720328642) bank102837 +102838 POINT(40.46430881095431 74.32208967124163) bank102838 +102839 POINT(41.0307234590638 74.47220413285169) bank102839 +102840 POINT(40.11226830289885 73.68594673306204) bank102840 +102841 POINT(41.034619633597366 74.6952889582689) bank102841 +102842 POINT(39.78696893300237 74.94337815936717) bank102842 +102843 POINT(40.03261454839372 73.75778442046084) bank102843 +102844 POINT(40.13342784747828 74.80518163834098) bank102844 +102845 POINT(41.06091537544505 73.84966655070203) bank102845 +102846 POINT(41.30595181877532 74.0583111730203) bank102846 +102847 POINT(40.54016302000247 74.31634535546026) bank102847 +102848 POINT(39.957343590691664 73.72295317786666) bank102848 +102849 POINT(39.807963313335925 74.42877691324775) bank102849 +102850 POINT(40.30777826392409 74.50977943600597) bank102850 +102851 POINT(41.13070608085127 74.92811695169401) bank102851 +102852 POINT(41.4299708304555 73.33097000461478) bank102852 +102853 POINT(40.22200537041611 74.68821554605627) bank102853 +102854 POINT(40.72280223998984 74.513529313636) bank102854 +102855 POINT(41.20962240642377 73.17539266986971) bank102855 +102856 POINT(41.07688280317904 73.94067122158681) bank102856 +102857 POINT(39.89516981160868 73.6430998160972) bank102857 +102858 POINT(41.29380160893558 74.69091602669364) bank102858 +102859 POINT(39.75937608713563 73.25184224691549) bank102859 +102860 POINT(40.47786002718392 74.90084683841498) bank102860 +102861 POINT(41.54710119551357 74.4230247873045) bank102861 +102862 POINT(41.55326794401208 74.53653980253675) bank102862 +102863 POINT(40.242656484063424 74.6314328664483) bank102863 +102864 POINT(40.312230912480075 73.17716759459277) bank102864 +102865 POINT(40.861919819118114 73.89889302537748) bank102865 +102866 POINT(41.02530213279322 73.89699945947008) bank102866 +102867 POINT(39.86493583072777 74.14700066650146) bank102867 +102868 POINT(40.42320604652808 73.23392188901002) bank102868 +102869 POINT(40.18685168515751 73.85808186163648) bank102869 +102870 POINT(40.62633808644645 73.64162045384109) bank102870 +102871 POINT(41.20609577874434 73.06185938537283) bank102871 +102872 POINT(41.629757937485955 73.47345139303684) bank102872 +102873 POINT(40.064529120523 73.53373476525132) bank102873 +102874 POINT(41.52044058113033 74.77747838526207) bank102874 +102875 POINT(40.42959986563931 74.63907238526365) bank102875 +102876 POINT(41.39954353610626 74.26536939375393) bank102876 +102877 POINT(41.35721220870717 73.31895055907157) bank102877 +102878 POINT(40.75596961223048 73.10259665163403) bank102878 +102879 POINT(41.569499979579646 74.15235637578441) bank102879 +102880 POINT(40.526115142378146 73.48014837930127) bank102880 +102881 POINT(40.325944310109925 73.03837591405154) bank102881 +102882 POINT(41.639682017583844 74.4931359813875) bank102882 +102883 POINT(40.92134349155384 73.08098682078139) bank102883 +102884 POINT(41.434745835782635 73.15236170629902) bank102884 +102885 POINT(41.021723115636384 73.7495376343592) bank102885 +102886 POINT(40.49971224224012 73.08765569126281) bank102886 +102887 POINT(41.498597388234614 73.04073847364323) bank102887 +102888 POINT(39.93821095651909 73.74713513669329) bank102888 +102889 POINT(41.521226898992445 73.65670291534133) bank102889 +102890 POINT(39.78238733319312 73.23046503258519) bank102890 +102891 POINT(40.806012282944714 73.81306180720028) bank102891 +102892 POINT(40.161715482706626 73.89880967051411) bank102892 +102893 POINT(40.63178414316689 73.60455760029078) bank102893 +102894 POINT(41.23378946893005 73.64855415474364) bank102894 +102895 POINT(40.51981107842484 73.95884353583246) bank102895 +102896 POINT(40.31856624400137 73.83473767324276) bank102896 +102897 POINT(39.88880410488985 74.95987210996088) bank102897 +102898 POINT(39.76005235548321 74.37287826934866) bank102898 +102899 POINT(40.66861632897212 73.90873243067026) bank102899 +102900 POINT(39.789135034083166 74.37260031998656) bank102900 +102901 POINT(41.62463337465524 73.42379339904434) bank102901 +102902 POINT(40.95477885697146 73.89170204171813) bank102902 +102903 POINT(40.20312237336168 73.28517091667445) bank102903 +102904 POINT(41.6638619305989 73.3477668176678) bank102904 +102905 POINT(41.257393117442476 74.35511842734316) bank102905 +102906 POINT(40.580074768051674 73.70334469022471) bank102906 +102907 POINT(40.61299667746178 74.15690149661886) bank102907 +102908 POINT(41.6075196349789 73.59017663104225) bank102908 +102909 POINT(39.84683853793304 73.45785346413341) bank102909 +102910 POINT(41.511319752827795 74.42098139040365) bank102910 +102911 POINT(41.70913270885679 73.66581788593562) bank102911 +102912 POINT(40.69342503708343 74.87141903300136) bank102912 +102913 POINT(40.406578771739404 73.11054544058845) bank102913 +102914 POINT(40.353396961099406 74.25439548430916) bank102914 +102915 POINT(40.88240482608879 74.81950734492028) bank102915 +102916 POINT(41.13696551161954 74.15389414213406) bank102916 +102917 POINT(40.577488846934216 73.68937894085994) bank102917 +102918 POINT(40.56903816507204 73.78674067225352) bank102918 +102919 POINT(40.30573007669156 73.1941830915114) bank102919 +102920 POINT(41.64253432129095 74.54785537759943) bank102920 +102921 POINT(40.99651003145422 74.60472504849011) bank102921 +102922 POINT(39.973055687721136 74.29438279778071) bank102922 +102923 POINT(39.95704749007455 73.1555828994554) bank102923 +102924 POINT(40.17457897489961 73.23109547617449) bank102924 +102925 POINT(40.61181578260359 73.02385351746736) bank102925 +102926 POINT(40.74502077537109 73.51262482509004) bank102926 +102927 POINT(39.90869789964901 73.82440192556652) bank102927 +102928 POINT(40.5386050174011 73.63359163631183) bank102928 +102929 POINT(41.24894952189697 74.9857723599321) bank102929 +102930 POINT(41.1369768496525 74.90209315481013) bank102930 +102931 POINT(40.470775131525734 74.03958235427997) bank102931 +102932 POINT(39.84509583735049 74.93593018274693) bank102932 +102933 POINT(40.203747852527705 73.88886184120894) bank102933 +102934 POINT(40.22053702268007 74.94220462451536) bank102934 +102935 POINT(41.07935672269178 74.38315247047385) bank102935 +102936 POINT(41.46648009166956 74.0649578435429) bank102936 +102937 POINT(39.77932195352324 73.31592286137563) bank102937 +102938 POINT(40.50720876761688 73.71945690955765) bank102938 +102939 POINT(41.11652429942346 73.15520642850997) bank102939 +102940 POINT(41.64254168749756 73.15573691939804) bank102940 +102941 POINT(40.155211143061535 73.83076120569332) bank102941 +102942 POINT(41.40359676735361 74.03542743452158) bank102942 +102943 POINT(40.481710387372054 73.45181723965536) bank102943 +102944 POINT(41.547644012801676 74.99205802844246) bank102944 +102945 POINT(40.31176996662699 73.6144062054172) bank102945 +102946 POINT(40.14194767064999 73.3990230773802) bank102946 +102947 POINT(41.49063000424521 73.76566859213058) bank102947 +102948 POINT(41.21478782303324 73.47193608242335) bank102948 +102949 POINT(41.315929506875705 73.54913497264211) bank102949 +102950 POINT(40.28243250389811 74.94993510721628) bank102950 +102951 POINT(40.231784739771996 74.83673846251936) bank102951 +102952 POINT(41.535627940954626 73.07567198845756) bank102952 +102953 POINT(40.59592726652941 74.01833187393235) bank102953 +102954 POINT(40.57419076952717 74.33490895640145) bank102954 +102955 POINT(40.44487036428466 73.88309059679976) bank102955 +102956 POINT(40.32360385628419 74.68541535442036) bank102956 +102957 POINT(41.06821293185433 74.83444755716381) bank102957 +102958 POINT(41.176677857301314 73.01483197853747) bank102958 +102959 POINT(41.60171063000922 73.92693338219776) bank102959 +102960 POINT(41.16850499764587 74.53957569033471) bank102960 +102961 POINT(40.06785954136559 73.48049558093106) bank102961 +102962 POINT(41.12755706546418 73.58328518838259) bank102962 +102963 POINT(40.724345001090946 75.00433174727387) bank102963 +102964 POINT(40.97825927610291 74.08385732680964) bank102964 +102965 POINT(40.49139020156263 74.97919373533468) bank102965 +102966 POINT(41.71082327045571 74.43118386262731) bank102966 +102967 POINT(40.295070946644216 73.33273664893801) bank102967 +102968 POINT(41.40494695203917 73.84872891728813) bank102968 +102969 POINT(41.477357921558024 74.28624207868224) bank102969 +102970 POINT(41.48752200563904 73.9008839774933) bank102970 +102971 POINT(40.26154401576993 73.70544914208358) bank102971 +102972 POINT(40.01510403432203 74.0887402899044) bank102972 +102973 POINT(41.418240076298716 73.21499070190902) bank102973 +102974 POINT(40.432903982241974 73.0588974139109) bank102974 +102975 POINT(40.29835752197448 73.0424132873204) bank102975 +102976 POINT(41.07050500997903 73.6465832878924) bank102976 +102977 POINT(40.16880914723926 73.17696528989805) bank102977 +102978 POINT(40.96608130322878 74.62670083910542) bank102978 +102979 POINT(40.58937176021192 73.3914564244957) bank102979 +102980 POINT(41.327179310903766 74.1478441856061) bank102980 +102981 POINT(40.09793371628325 74.2991727241564) bank102981 +102982 POINT(40.650193196773195 74.3221897439427) bank102982 +102983 POINT(40.906097447301605 73.43885672917554) bank102983 +102984 POINT(41.144096669775536 73.27397510114173) bank102984 +102985 POINT(39.924815920056794 74.0727299016907) bank102985 +102986 POINT(40.528342788707846 74.0539099072844) bank102986 +102987 POINT(39.92973439915269 73.63226139145374) bank102987 +102988 POINT(40.58990222201713 74.19339835042645) bank102988 +102989 POINT(39.75637070522681 73.94670870723387) bank102989 +102990 POINT(40.982360951472145 74.61014845933353) bank102990 +102991 POINT(41.4897475786466 74.7706177605504) bank102991 +102992 POINT(40.66892751772199 74.74587464677889) bank102992 +102993 POINT(40.50581391097298 73.45754694065906) bank102993 +102994 POINT(39.771184864166884 73.44997432940953) bank102994 +102995 POINT(41.47638341994354 73.52074553700098) bank102995 +102996 POINT(40.919801491470174 74.51204567530954) bank102996 +102997 POINT(40.04512036351683 73.10181416893688) bank102997 +102998 POINT(39.811400025573 74.36296522054883) bank102998 +102999 POINT(41.31889406885518 73.03724853909178) bank102999 +103000 POINT(40.24163226334066 74.26108504009154) bank103000 +103001 POINT(40.87838156257546 73.16485201786229) bank103001 +103002 POINT(40.70008377335639 74.568973236913) bank103002 +103003 POINT(41.027191464013846 73.8811068373583) bank103003 +103004 POINT(39.761748509659604 74.15804298152644) bank103004 +103005 POINT(40.69009956933591 73.51458476626684) bank103005 +103006 POINT(41.156529070452805 74.1648518464362) bank103006 +103007 POINT(39.786298551758364 74.78721344602576) bank103007 +103008 POINT(41.295971673983246 73.02253341148051) bank103008 +103009 POINT(41.62768575472017 73.990083918035) bank103009 +103010 POINT(41.08409391391105 74.63873502023469) bank103010 +103011 POINT(40.95266302955457 74.03745326857326) bank103011 +103012 POINT(41.06675820559086 73.27583001954241) bank103012 +103013 POINT(40.689511909832156 73.82115368815323) bank103013 +103014 POINT(41.33276719770119 74.55683960858633) bank103014 +103015 POINT(41.58989112155886 74.48273312455743) bank103015 +103016 POINT(39.93931955604954 74.36535057811008) bank103016 +103017 POINT(41.68772816277019 74.56578059658567) bank103017 +103018 POINT(39.87070404181694 74.39571538449374) bank103018 +103019 POINT(41.48732432321818 74.48191900945056) bank103019 +103020 POINT(40.07298276399406 74.24863832573821) bank103020 +103021 POINT(41.52537272778194 74.28436282946072) bank103021 +103022 POINT(39.843409801152326 74.2725355323854) bank103022 +103023 POINT(40.80004719467956 73.51293192880357) bank103023 +103024 POINT(41.60715908720612 73.92369563658669) bank103024 +103025 POINT(40.04301332523672 73.87396504047896) bank103025 +103026 POINT(39.80686034712469 74.68643500230112) bank103026 +103027 POINT(40.648217404333295 73.99072670508119) bank103027 +103028 POINT(40.96570825753884 73.37820081761612) bank103028 +103029 POINT(40.73287698045225 74.02703404118269) bank103029 +103030 POINT(40.68532211518356 74.8718633366226) bank103030 +103031 POINT(40.73872287689524 73.27430146844378) bank103031 +103032 POINT(40.61973115953119 73.41084811962602) bank103032 +103033 POINT(39.774640286913744 73.59224916877723) bank103033 +103034 POINT(40.833449965786315 73.5431769195274) bank103034 +103035 POINT(41.50029454757217 73.77017299069779) bank103035 +103036 POINT(41.210017226164666 74.09395835644177) bank103036 +103037 POINT(40.162970681807096 74.04542776405957) bank103037 +103038 POINT(40.2994349065266 73.85029640218555) bank103038 +103039 POINT(39.89002584363563 74.82552095151334) bank103039 +103040 POINT(40.46410955230788 74.0820144763486) bank103040 +103041 POINT(41.082253643084904 73.9487567195883) bank103041 +103042 POINT(40.1004865833886 74.94737644814185) bank103042 +103043 POINT(40.99907082789989 73.18274049650077) bank103043 +103044 POINT(40.86410048069478 74.88923370625879) bank103044 +103045 POINT(40.29099747127751 73.2102867914719) bank103045 +103046 POINT(41.39793033008766 73.66214700083817) bank103046 +103047 POINT(41.30451833646309 74.08288302775524) bank103047 +103048 POINT(41.06312129182897 73.17123310858295) bank103048 +103049 POINT(40.149659700393585 74.4160226612101) bank103049 +103050 POINT(40.276270461424666 73.3891312191826) bank103050 +103051 POINT(41.16517801872817 73.17253796031821) bank103051 +103052 POINT(41.229030414640995 74.39576345379128) bank103052 +103053 POINT(40.98469387897564 74.4267920416903) bank103053 +103054 POINT(41.62060942697108 74.32880832034162) bank103054 +103055 POINT(40.896190472796285 74.29720442381962) bank103055 +103056 POINT(39.84563576301612 74.82706518598064) bank103056 +103057 POINT(40.897822785590606 74.14810623094041) bank103057 +103058 POINT(40.57086147528782 73.80201835615912) bank103058 +103059 POINT(40.875241041163044 73.17565294603428) bank103059 +103060 POINT(41.67942807085014 74.36474435362416) bank103060 +103061 POINT(40.22623083013417 73.68599060223697) bank103061 +103062 POINT(41.09725568586791 74.46884445093181) bank103062 +103063 POINT(40.532984061264784 74.07001817394597) bank103063 +103064 POINT(39.844216781892776 74.70210750089112) bank103064 +103065 POINT(41.65440966374385 74.25614615099894) bank103065 +103066 POINT(40.149092909316096 74.54498680815068) bank103066 +103067 POINT(41.151281057010664 73.46683496774108) bank103067 +103068 POINT(41.54475665442702 73.09090716824653) bank103068 +103069 POINT(39.81604233756945 74.50769592221226) bank103069 +103070 POINT(40.64051403366596 74.07834275967915) bank103070 +103071 POINT(40.097088571908145 74.21980167420163) bank103071 +103072 POINT(40.727203852699674 74.27427900341061) bank103072 +103073 POINT(40.37705782762578 73.86887283331862) bank103073 +103074 POINT(41.491775672874354 74.43902121476631) bank103074 +103075 POINT(41.66897509473146 73.92765778504818) bank103075 +103076 POINT(41.64924427678048 73.08414795870323) bank103076 +103077 POINT(41.49402952068209 74.0051555694047) bank103077 +103078 POINT(40.849235135874125 74.92363083263011) bank103078 +103079 POINT(41.462210860244966 74.30957700332026) bank103079 +103080 POINT(41.2155097862667 74.64533270529836) bank103080 +103081 POINT(41.6924143068182 73.69415593794791) bank103081 +103082 POINT(40.24519282231775 74.36293801318067) bank103082 +103083 POINT(40.64682066534353 73.31533596540076) bank103083 +103084 POINT(41.25645093118963 74.88197692636729) bank103084 +103085 POINT(41.42223874293933 73.00810288892134) bank103085 +103086 POINT(39.99111741525785 74.00434744842906) bank103086 +103087 POINT(40.66467026024525 74.65539443948) bank103087 +103088 POINT(40.88908052542438 74.09518455195176) bank103088 +103089 POINT(39.914567012802614 73.66879989958535) bank103089 +103090 POINT(41.15302851229673 73.89967965306799) bank103090 +103091 POINT(39.80492957346366 74.54546066769998) bank103091 +103092 POINT(40.41192660940458 74.0413426822489) bank103092 +103093 POINT(40.650578494487526 74.68316231424934) bank103093 +103094 POINT(39.97779584431699 73.79262037591867) bank103094 +103095 POINT(41.38151530473541 74.25678593718645) bank103095 +103096 POINT(40.92089144219596 74.80468378213241) bank103096 +103097 POINT(39.73743725081748 74.16233070899182) bank103097 +103098 POINT(41.46620415980783 74.79333828203988) bank103098 +103099 POINT(40.55954778675745 74.70003748522446) bank103099 +103100 POINT(39.748478169266356 74.19509302887074) bank103100 +103101 POINT(41.6135698725895 73.62863308958548) bank103101 +103102 POINT(40.94118016502685 74.22742738175431) bank103102 +103103 POINT(39.71596800577326 73.7550844783757) bank103103 +103104 POINT(41.45789039156445 73.45909093438918) bank103104 +103105 POINT(40.15715617756624 73.9879668054864) bank103105 +103106 POINT(40.778828102509245 74.12912777906284) bank103106 +103107 POINT(41.26547439667088 74.75241170494036) bank103107 +103108 POINT(39.85982383634686 74.37626639024184) bank103108 +103109 POINT(40.043921770223164 73.46589173625995) bank103109 +103110 POINT(39.760798800452534 73.94237766326246) bank103110 +103111 POINT(40.252030926199 73.78495526852664) bank103111 +103112 POINT(40.64150170984029 74.26292041192296) bank103112 +103113 POINT(39.87950614818094 73.22108489079737) bank103113 +103114 POINT(40.0630053433863 74.40488969747244) bank103114 +103115 POINT(40.971055970612554 73.41312629516825) bank103115 +103116 POINT(41.18266863884645 73.92597304552234) bank103116 +103117 POINT(40.78215801257952 74.71269406126223) bank103117 +103118 POINT(41.665756863965655 74.27688112387358) bank103118 +103119 POINT(41.39965430304526 73.26060145861373) bank103119 +103120 POINT(41.283849086876245 74.95068836977656) bank103120 +103121 POINT(40.86620197345933 74.53630771716203) bank103121 +103122 POINT(41.111205610350325 73.70306097988713) bank103122 +103123 POINT(39.821647953515914 74.57100058075322) bank103123 +103124 POINT(40.13217581316807 73.26471967344686) bank103124 +103125 POINT(40.12620904459262 73.62059917251192) bank103125 +103126 POINT(40.99034784428377 74.6647499511576) bank103126 +103127 POINT(40.509018763867374 74.70113543410878) bank103127 +103128 POINT(41.437074794600235 73.20103200453798) bank103128 +103129 POINT(40.81495037313452 73.5477605908076) bank103129 +103130 POINT(41.445173577591675 74.93153256321874) bank103130 +103131 POINT(41.29761718980074 73.77983303150971) bank103131 +103132 POINT(41.57416072817233 74.35473864675016) bank103132 +103133 POINT(41.1397409913851 74.22244041477929) bank103133 +103134 POINT(41.68051127050471 74.95575513267822) bank103134 +103135 POINT(39.95413203027682 73.23159262423407) bank103135 +103136 POINT(41.0897983910791 73.4447623484973) bank103136 +103137 POINT(39.93450743626821 74.42147940180226) bank103137 +103138 POINT(40.548766140844144 74.300073172406) bank103138 +103139 POINT(40.60611786848497 73.48688105427448) bank103139 +103140 POINT(41.04435393288191 73.70328654549931) bank103140 +103141 POINT(41.11840003925579 74.26035385714628) bank103141 +103142 POINT(41.31413675023594 74.01375282923155) bank103142 +103143 POINT(40.73140653708329 73.83375348996287) bank103143 +103144 POINT(41.08421036069916 74.46124871352328) bank103144 +103145 POINT(41.53612368564445 74.25284397226099) bank103145 +103146 POINT(41.607877700128356 73.39093503417156) bank103146 +103147 POINT(40.94886375034599 73.11262297109431) bank103147 +103148 POINT(40.92343781052034 73.8664749702238) bank103148 +103149 POINT(40.8233536130278 74.15057217705291) bank103149 +103150 POINT(41.67171640753902 73.75223894930703) bank103150 +103151 POINT(39.717434894764715 73.87262984469605) bank103151 +103152 POINT(40.855593238052755 74.60002941658405) bank103152 +103153 POINT(41.21549041826044 74.14689385938986) bank103153 +103154 POINT(41.12852857951378 73.46671660975844) bank103154 +103155 POINT(41.12613768892764 73.93396957974821) bank103155 +103156 POINT(40.79421316432664 74.96895200454901) bank103156 +103157 POINT(41.2676581199297 73.87432226329457) bank103157 +103158 POINT(39.99219250868573 73.19296626233549) bank103158 +103159 POINT(39.72299631533597 73.29960812180201) bank103159 +103160 POINT(40.178103778593574 73.47060216318762) bank103160 +103161 POINT(40.30956370011555 74.24091184611888) bank103161 +103162 POINT(39.95065314484585 74.67177252167431) bank103162 +103163 POINT(41.43791902486385 73.02294523190264) bank103163 +103164 POINT(40.22155190743603 74.53525060215283) bank103164 +103165 POINT(40.834117881160765 73.647428408674) bank103165 +103166 POINT(41.58361643095959 73.4327090599914) bank103166 +103167 POINT(40.89442323957351 73.92987253357028) bank103167 +103168 POINT(41.68774697902156 74.98100390282673) bank103168 +103169 POINT(40.604488167167865 73.69294343172704) bank103169 +103170 POINT(40.35155630759235 73.66453772665986) bank103170 +103171 POINT(40.63946947640663 73.76967076667657) bank103171 +103172 POINT(40.1274264787062 74.52515032929382) bank103172 +103173 POINT(40.88406521472047 73.79290494308823) bank103173 +103174 POINT(40.65835692513959 73.52923368219541) bank103174 +103175 POINT(41.52976428163313 73.43719461351846) bank103175 +103176 POINT(40.423528780100746 73.02084700799139) bank103176 +103177 POINT(41.53438453333001 74.3058697196323) bank103177 +103178 POINT(41.43686315586108 73.8112756139372) bank103178 +103179 POINT(39.987684160279606 74.98480178446117) bank103179 +103180 POINT(41.03146297018631 74.24686323788991) bank103180 +103181 POINT(40.46623775401042 73.22522486569329) bank103181 +103182 POINT(40.18235751034976 73.61169226082326) bank103182 +103183 POINT(41.520030846406904 73.82900245750247) bank103183 +103184 POINT(40.293793596429666 74.93293814064896) bank103184 +103185 POINT(40.90095281597736 73.53084583297066) bank103185 +103186 POINT(40.89172686479482 73.0305561657112) bank103186 +103187 POINT(40.082557945503055 73.07859625535413) bank103187 +103188 POINT(41.27194300598756 74.56162988490809) bank103188 +103189 POINT(41.03596419963953 74.8905068944062) bank103189 +103190 POINT(40.01257434779898 74.97543137933125) bank103190 +103191 POINT(40.62959258984025 73.74620031971855) bank103191 +103192 POINT(41.711460514956315 73.99235073042935) bank103192 +103193 POINT(40.84445749980844 74.65273749770337) bank103193 +103194 POINT(41.005958204012934 73.35504541809416) bank103194 +103195 POINT(41.23352238609713 73.10832060705064) bank103195 +103196 POINT(40.32885748334877 74.16983190892807) bank103196 +103197 POINT(41.576546592917914 73.91236809592196) bank103197 +103198 POINT(40.687357492893625 74.5513705185471) bank103198 +103199 POINT(40.09772247518663 74.67253033974889) bank103199 +103200 POINT(40.70518281943153 73.2848576623425) bank103200 +103201 POINT(40.45522919515045 74.79665088037154) bank103201 +103202 POINT(41.66922601332217 74.76590997650271) bank103202 +103203 POINT(41.184096831941716 73.56734883914753) bank103203 +103204 POINT(40.46436244986968 73.83979951420939) bank103204 +103205 POINT(40.1291156426619 73.71064547124824) bank103205 +103206 POINT(40.058065994367325 73.69178160875853) bank103206 +103207 POINT(41.57753476336814 74.97243805109085) bank103207 +103208 POINT(40.17512270212244 73.07477362429854) bank103208 +103209 POINT(41.7047379124921 74.8256298479774) bank103209 +103210 POINT(39.936938305088034 73.82958031466868) bank103210 +103211 POINT(40.00548291727502 73.3259078846137) bank103211 +103212 POINT(41.45890778627052 73.33500279523297) bank103212 +103213 POINT(40.20732442333882 74.55472591066443) bank103213 +103214 POINT(40.60044031223198 74.40504990804486) bank103214 +103215 POINT(41.27401663786955 74.35487959724249) bank103215 +103216 POINT(40.3501845775148 73.07683462070483) bank103216 +103217 POINT(40.27043649709759 73.19225870384886) bank103217 +103218 POINT(41.193832257949985 73.62378277861711) bank103218 +103219 POINT(40.03889286948365 73.57481522496839) bank103219 +103220 POINT(40.21551141843803 74.67387163848964) bank103220 +103221 POINT(40.38958123254402 74.17507437661278) bank103221 +103222 POINT(40.05141071878905 75.00278533926375) bank103222 +103223 POINT(40.2297321022869 73.18344450527805) bank103223 +103224 POINT(40.02075180194644 73.74879502792463) bank103224 +103225 POINT(39.71539030780267 73.8355050974044) bank103225 +103226 POINT(41.491451995639345 73.73182361201123) bank103226 +103227 POINT(40.47473042818272 74.4235930104282) bank103227 +103228 POINT(40.25900028971387 74.3867708245813) bank103228 +103229 POINT(40.76327244713776 74.2536397476785) bank103229 +103230 POINT(40.28138602860007 73.65109610704708) bank103230 +103231 POINT(40.00876743509156 73.29277827989395) bank103231 +103232 POINT(40.68627196347524 73.17740407862993) bank103232 +103233 POINT(39.80472663875818 73.52444067702854) bank103233 +103234 POINT(40.53280247991221 73.62898353955893) bank103234 +103235 POINT(40.967954965992064 73.37785562259069) bank103235 +103236 POINT(41.4497243860346 74.30432953822678) bank103236 +103237 POINT(41.06319343525322 73.23244509633263) bank103237 +103238 POINT(41.451161089957715 73.03165085146051) bank103238 +103239 POINT(40.16846960001635 73.57570946275257) bank103239 +103240 POINT(40.970193435084326 74.97061875331174) bank103240 +103241 POINT(41.455084770295926 73.83933521682333) bank103241 +103242 POINT(41.244628070848826 74.674659055647) bank103242 +103243 POINT(41.366885293284824 73.95690096768324) bank103243 +103244 POINT(40.69986953853619 73.39759937024098) bank103244 +103245 POINT(40.11880125447399 73.23785955032913) bank103245 +103246 POINT(40.924548108134836 73.42359949297206) bank103246 +103247 POINT(40.07537441317224 74.63189513161826) bank103247 +103248 POINT(39.96789920054355 74.91677884216804) bank103248 +103249 POINT(39.73844471764598 74.42190675408783) bank103249 +103250 POINT(40.40235940193096 74.54218179462397) bank103250 +103251 POINT(41.41847232742344 73.15062480566114) bank103251 +103252 POINT(40.9805262975884 73.5079618531511) bank103252 +103253 POINT(39.92452827316943 74.2539272449429) bank103253 +103254 POINT(39.73691524235517 74.03136205354211) bank103254 +103255 POINT(41.01712221952072 74.87240214286192) bank103255 +103256 POINT(41.45185421116828 73.39515549886796) bank103256 +103257 POINT(39.77150421574148 73.29410034673617) bank103257 +103258 POINT(40.62428557614888 74.08688785223151) bank103258 +103259 POINT(40.301708945839444 73.47393491488528) bank103259 +103260 POINT(40.49081337639313 73.66550606953015) bank103260 +103261 POINT(40.52383590365617 74.06773129506153) bank103261 +103262 POINT(41.15178988297749 74.53492586105689) bank103262 +103263 POINT(41.58038952510118 74.26547670599427) bank103263 +103264 POINT(41.6189475564514 74.19530215298732) bank103264 +103265 POINT(40.1069638165693 74.22412513464573) bank103265 +103266 POINT(40.76657289514398 73.4993119335751) bank103266 +103267 POINT(40.73590719490294 74.59311779671667) bank103267 +103268 POINT(39.76365882005025 74.27413373289552) bank103268 +103269 POINT(40.8177388792676 73.03374319341081) bank103269 +103270 POINT(40.9723703606911 73.49538852330222) bank103270 +103271 POINT(40.83565162703663 74.82766911695747) bank103271 +103272 POINT(40.5901831055871 73.63349286164973) bank103272 +103273 POINT(39.9429460445838 74.669118693627) bank103273 +103274 POINT(39.969995813132925 74.8094183186423) bank103274 +103275 POINT(40.774212782121914 74.71897935999307) bank103275 +103276 POINT(39.96652909181371 74.34615014458241) bank103276 +103277 POINT(41.41501141728736 74.6517482251582) bank103277 +103278 POINT(41.07206039813516 73.45472647176969) bank103278 +103279 POINT(40.55243444352061 74.49185166556263) bank103279 +103280 POINT(40.25996573651176 73.05331103795004) bank103280 +103281 POINT(40.349512365456015 74.30399950434594) bank103281 +103282 POINT(39.967962104947205 74.06367697973802) bank103282 +103283 POINT(39.812094825698914 73.56459280468177) bank103283 +103284 POINT(41.35049767618114 74.1394231291827) bank103284 +103285 POINT(39.939642041114155 73.83319146406531) bank103285 +103286 POINT(41.44751223769147 74.89875993493315) bank103286 +103287 POINT(40.85629885817288 74.84254021948335) bank103287 +103288 POINT(41.401490379088145 74.1057289464055) bank103288 +103289 POINT(40.14224890467309 74.93526518635507) bank103289 +103290 POINT(40.215128506744804 73.09761111879325) bank103290 +103291 POINT(39.76940868020247 74.2397702223258) bank103291 +103292 POINT(40.83014147798235 74.30249480656114) bank103292 +103293 POINT(40.572691023765024 74.05680764274071) bank103293 +103294 POINT(40.56838644733544 74.16211195176123) bank103294 +103295 POINT(40.46683578811466 73.16197865034356) bank103295 +103296 POINT(40.71714039753708 74.85578796184447) bank103296 +103297 POINT(40.68416688675917 73.46987840752774) bank103297 +103298 POINT(41.129938093797755 73.45583625216241) bank103298 +103299 POINT(41.66061764075559 73.75226986092545) bank103299 +103300 POINT(40.71037709062272 73.76188306196634) bank103300 +103301 POINT(40.06588150777463 73.8515636941702) bank103301 +103302 POINT(40.03943627357255 73.96056803524493) bank103302 +103303 POINT(40.83806632856255 73.8871735867052) bank103303 +103304 POINT(41.67153613976606 73.21258527662584) bank103304 +103305 POINT(40.059562543961256 73.87600960543668) bank103305 +103306 POINT(40.99348587857148 73.0617338641196) bank103306 +103307 POINT(41.63873548967901 74.04535010260196) bank103307 +103308 POINT(41.30761613084092 74.99558119300373) bank103308 +103309 POINT(40.713838591159714 73.43091241027231) bank103309 +103310 POINT(40.565190237695724 73.5605430434781) bank103310 +103311 POINT(40.08882687556499 74.62969043403635) bank103311 +103312 POINT(39.8677197742455 73.22262276778658) bank103312 +103313 POINT(40.21215790033207 73.7642365732815) bank103313 +103314 POINT(41.4461994829907 73.59849908561043) bank103314 +103315 POINT(40.60772342606471 73.97694346329341) bank103315 +103316 POINT(41.09303009511969 74.28194571251645) bank103316 +103317 POINT(41.3739080027295 74.06105706427145) bank103317 +103318 POINT(39.78293454086259 73.82517734522006) bank103318 +103319 POINT(41.542316245020935 73.89490633524814) bank103319 +103320 POINT(39.86422886770811 73.57625079905549) bank103320 +103321 POINT(40.57331985788944 74.1990765950045) bank103321 +103322 POINT(41.6573252223405 74.84356091652954) bank103322 +103323 POINT(41.48105898256196 74.07085853002681) bank103323 +103324 POINT(39.780976100266315 73.4253438956349) bank103324 +103325 POINT(41.502448043598214 73.23077725354379) bank103325 +103326 POINT(41.008195241900474 74.44934547295615) bank103326 +103327 POINT(39.73167394540097 73.87345828431214) bank103327 +103328 POINT(41.236566669118986 74.91744420314423) bank103328 +103329 POINT(41.16247352301114 74.33238413294133) bank103329 +103330 POINT(40.75946607720484 73.66516769683493) bank103330 +103331 POINT(40.424291379000884 73.18130315335928) bank103331 +103332 POINT(41.19492032672881 74.15914581160352) bank103332 +103333 POINT(40.83457590762256 73.56416538463188) bank103333 +103334 POINT(40.02349402198854 73.49576814878138) bank103334 +103335 POINT(39.78891954387512 74.36260943894008) bank103335 +103336 POINT(41.087287706320645 73.85407200451897) bank103336 +103337 POINT(41.309186078292264 74.24213057980357) bank103337 +103338 POINT(41.26088159705725 74.04119074202458) bank103338 +103339 POINT(39.73769667727307 73.54194741220213) bank103339 +103340 POINT(41.35657534810441 73.39870220038877) bank103340 +103341 POINT(41.05283184539557 73.19750685277683) bank103341 +103342 POINT(40.43426036863883 74.10081620629059) bank103342 +103343 POINT(41.48900180608374 74.75456897383093) bank103343 +103344 POINT(41.14376666716497 73.73311262148827) bank103344 +103345 POINT(41.42869891210936 73.03244022068073) bank103345 +103346 POINT(41.213508329757104 73.01873322005632) bank103346 +103347 POINT(40.761635057691855 74.46922698612019) bank103347 +103348 POINT(40.78187667973238 73.33249294228584) bank103348 +103349 POINT(40.294358324829325 74.327850862631) bank103349 +103350 POINT(40.90030183931273 73.49493928292519) bank103350 +103351 POINT(41.167970274191745 73.80602007747424) bank103351 +103352 POINT(41.006105319307174 73.7065684480693) bank103352 +103353 POINT(41.324716813380114 73.22729174967962) bank103353 +103354 POINT(41.10036291645273 73.4618606590981) bank103354 +103355 POINT(40.894267354043336 73.39933836253884) bank103355 +103356 POINT(40.27997011127228 74.17455846525785) bank103356 +103357 POINT(39.79258409567055 73.48114667646104) bank103357 +103358 POINT(40.234872693407596 73.84908628945729) bank103358 +103359 POINT(41.69237346161494 73.5231201570424) bank103359 +103360 POINT(40.48485380234819 73.08470087596683) bank103360 +103361 POINT(40.90308923887996 74.8633312864242) bank103361 +103362 POINT(41.42926551768237 73.44351296031651) bank103362 +103363 POINT(40.81187273731193 74.4915645325638) bank103363 +103364 POINT(41.035123008539905 73.92866413157647) bank103364 +103365 POINT(41.094362237349465 73.94866203567368) bank103365 +103366 POINT(40.079300785011355 74.0197883721911) bank103366 +103367 POINT(40.11192845996824 73.02650377818354) bank103367 +103368 POINT(40.06519545573679 74.06551536575346) bank103368 +103369 POINT(40.213334439578496 74.34238503510859) bank103369 +103370 POINT(40.280598893693714 73.29541339324564) bank103370 +103371 POINT(40.98156245521057 74.40518938153876) bank103371 +103372 POINT(40.131286728637356 73.67749669376686) bank103372 +103373 POINT(40.37356315983615 74.45324983229183) bank103373 +103374 POINT(41.650325591589514 74.47346891154612) bank103374 +103375 POINT(41.41098734045319 73.00797224966509) bank103375 +103376 POINT(41.418972517202064 74.37891098493054) bank103376 +103377 POINT(40.18220759225219 74.80906085778179) bank103377 +103378 POINT(40.47983980257891 74.9618812487841) bank103378 +103379 POINT(40.62503432681773 73.83841469270881) bank103379 +103380 POINT(41.485824947048485 74.02347011173326) bank103380 +103381 POINT(41.55703235095531 74.79956781935152) bank103381 +103382 POINT(41.14427452687449 73.98808986534554) bank103382 +103383 POINT(40.56723115024101 73.34431289437966) bank103383 +103384 POINT(41.37200601231318 73.70809946875299) bank103384 +103385 POINT(41.138606039740566 73.17451237784297) bank103385 +103386 POINT(40.18524877909878 74.3510085165818) bank103386 +103387 POINT(40.07551782235065 73.55656857890052) bank103387 +103388 POINT(40.376478831381675 73.94075203164168) bank103388 +103389 POINT(39.95766122176832 73.35107781896268) bank103389 +103390 POINT(40.80998122218499 73.4439765690106) bank103390 +103391 POINT(40.28651364787081 74.61438463248804) bank103391 +103392 POINT(41.32046064371809 74.19021978983294) bank103392 +103393 POINT(41.04382370095284 73.5964422264518) bank103393 +103394 POINT(40.42609557615672 74.04227097690212) bank103394 +103395 POINT(41.70640810318951 73.04715985011818) bank103395 +103396 POINT(40.0498982093044 74.3169574503309) bank103396 +103397 POINT(39.759431804208056 74.62521496492832) bank103397 +103398 POINT(40.580653101244856 73.0778864164764) bank103398 +103399 POINT(40.98097634608568 73.14044442272096) bank103399 +103400 POINT(40.300202920945964 74.00544826018198) bank103400 +103401 POINT(41.38774395595204 74.03052200314185) bank103401 +103402 POINT(40.57597050773524 74.94688718604056) bank103402 +103403 POINT(41.04032674946673 74.56664139216547) bank103403 +103404 POINT(41.390994070112505 73.95670610929244) bank103404 +103405 POINT(39.803750140766645 73.54352763053507) bank103405 +103406 POINT(40.545404997324376 74.09287084486851) bank103406 +103407 POINT(40.85109428285342 74.51823801178631) bank103407 +103408 POINT(40.713185550409804 74.18805556275022) bank103408 +103409 POINT(41.23523307358493 73.46987259755257) bank103409 +103410 POINT(41.11150252081396 74.62323004583621) bank103410 +103411 POINT(39.82057347994411 74.07704018975232) bank103411 +103412 POINT(40.86344532677707 73.3803022884258) bank103412 +103413 POINT(41.04907733279763 74.2303581232636) bank103413 +103414 POINT(41.55327114099366 74.5444613191777) bank103414 +103415 POINT(40.455811444361856 73.53295290249159) bank103415 +103416 POINT(39.94989619304954 73.17356404506256) bank103416 +103417 POINT(40.17098489368625 73.8000676674555) bank103417 +103418 POINT(40.96681198833826 73.75117496534469) bank103418 +103419 POINT(40.22202085874409 73.64022547773938) bank103419 +103420 POINT(41.12293602115158 73.14563997891175) bank103420 +103421 POINT(39.99399612128529 74.84531055773903) bank103421 +103422 POINT(41.379205048535624 73.77977608470799) bank103422 +103423 POINT(40.25864274602801 73.6197434790822) bank103423 +103424 POINT(39.8711250603867 73.39866370665241) bank103424 +103425 POINT(40.1074464732491 74.1859517262349) bank103425 +103426 POINT(40.21827556382134 74.31264482729684) bank103426 +103427 POINT(40.52383717083728 74.62089656430153) bank103427 +103428 POINT(41.00236631285518 73.65584821540502) bank103428 +103429 POINT(39.780550493004334 74.91102721645538) bank103429 +103430 POINT(41.58734966593677 73.65497097942453) bank103430 +103431 POINT(41.04429469628908 74.84066651935498) bank103431 +103432 POINT(41.46661656349858 73.3244902506798) bank103432 +103433 POINT(41.38566237594535 74.06717273826867) bank103433 +103434 POINT(41.3150900006993 74.27332741061777) bank103434 +103435 POINT(39.85058531322501 73.04359138279602) bank103435 +103436 POINT(39.89039589421637 73.39862004862947) bank103436 +103437 POINT(41.05407316081068 73.57362891772965) bank103437 +103438 POINT(40.91773449378516 74.79997456662569) bank103438 +103439 POINT(40.27121074417311 74.29333980658087) bank103439 +103440 POINT(40.19894667107383 74.2864525515435) bank103440 +103441 POINT(41.60320338562513 74.21687080256915) bank103441 +103442 POINT(40.56079149793264 73.73184098176473) bank103442 +103443 POINT(40.83575760547854 73.26191492282493) bank103443 +103444 POINT(40.7306446548459 74.90139424614195) bank103444 +103445 POINT(39.815487023559896 74.11183409114122) bank103445 +103446 POINT(39.81922353319303 73.79838212727057) bank103446 +103447 POINT(40.35915197922323 74.25884397905948) bank103447 +103448 POINT(40.50731668988487 74.64500977400495) bank103448 +103449 POINT(40.570721016802466 73.9322682193605) bank103449 +103450 POINT(41.27719710227858 73.53022528628867) bank103450 +103451 POINT(41.36578122175399 74.61389507214851) bank103451 +103452 POINT(40.19966139838769 74.10301576377631) bank103452 +103453 POINT(41.430470963819666 73.25827764272688) bank103453 +103454 POINT(40.26755056294448 74.57149117075971) bank103454 +103455 POINT(40.11923051414702 73.47613120698378) bank103455 +103456 POINT(40.915454191034016 73.73937647189389) bank103456 +103457 POINT(41.026818074039284 73.56545455427894) bank103457 +103458 POINT(40.65714226850022 74.0661381154131) bank103458 +103459 POINT(40.58095997542902 73.12624920915638) bank103459 +103460 POINT(41.64007722656734 74.75486496572424) bank103460 +103461 POINT(39.85612315758559 74.76562539228154) bank103461 +103462 POINT(40.90654192684327 74.8572745489767) bank103462 +103463 POINT(39.73882878812115 74.06710538302485) bank103463 +103464 POINT(40.74407271186779 74.85299432540229) bank103464 +103465 POINT(40.37911122900359 74.33551843243669) bank103465 +103466 POINT(40.71714563924451 73.15909539615589) bank103466 +103467 POINT(39.754438007153475 73.85376093069128) bank103467 +103468 POINT(40.60998403084641 73.46236086598324) bank103468 +103469 POINT(41.20190629550107 73.88150390759598) bank103469 +103470 POINT(39.77847546131794 74.62201108034036) bank103470 +103471 POINT(41.359811489734895 73.31698343140978) bank103471 +103472 POINT(40.397380023209486 74.67760338772682) bank103472 +103473 POINT(40.92960881675429 73.36743568488875) bank103473 +103474 POINT(40.45099195677339 73.52078427822683) bank103474 +103475 POINT(39.820729420821344 74.72653725880014) bank103475 +103476 POINT(41.32901406388474 74.55944071919451) bank103476 +103477 POINT(41.07806432031619 73.07025922680552) bank103477 +103478 POINT(41.22270153814423 73.38766591267748) bank103478 +103479 POINT(40.75994347703695 74.23927928700249) bank103479 +103480 POINT(39.96189974203784 73.07120259454113) bank103480 +103481 POINT(41.39008475151521 73.41340910081526) bank103481 +103482 POINT(41.366795151383045 74.72371977665244) bank103482 +103483 POINT(39.99038489870565 73.71287507204227) bank103483 +103484 POINT(41.601812580051636 74.98442467198595) bank103484 +103485 POINT(41.012627450484935 74.17414452194481) bank103485 +103486 POINT(40.64637879308112 73.82398280998734) bank103486 +103487 POINT(41.22915438167698 73.9306433323712) bank103487 +103488 POINT(40.07869078408512 74.75059250963254) bank103488 +103489 POINT(39.948416145048654 74.08817867553131) bank103489 +103490 POINT(40.48996536059214 73.90507302284307) bank103490 +103491 POINT(40.92282077945784 74.82272100391413) bank103491 +103492 POINT(40.758116269158975 73.17781318189918) bank103492 +103493 POINT(40.90816858974353 73.44423597554021) bank103493 +103494 POINT(40.65551108884289 74.86241004721296) bank103494 +103495 POINT(41.25985636847354 73.84417603512942) bank103495 +103496 POINT(40.60029452448254 73.83006406532058) bank103496 +103497 POINT(41.08119998154373 73.26688489933784) bank103497 +103498 POINT(39.7180799629729 73.89451631743079) bank103498 +103499 POINT(40.0628180746078 74.99007411650861) bank103499 +103500 POINT(41.24270854513554 73.33292910476742) bank103500 +103501 POINT(40.61915643107121 73.29534952513664) bank103501 +103502 POINT(41.51031060605824 74.21220582726129) bank103502 +103503 POINT(40.75639584247913 73.0757589975698) bank103503 +103504 POINT(40.68147326262007 74.06244244556451) bank103504 +103505 POINT(41.697099565906846 73.38133407296138) bank103505 +103506 POINT(41.47968215668647 74.26002819724465) bank103506 +103507 POINT(41.46506864524251 73.4118914007363) bank103507 +103508 POINT(41.64253516328737 74.6753579340966) bank103508 +103509 POINT(40.873907589954015 74.79802958709989) bank103509 +103510 POINT(40.225944887701274 73.45392506681789) bank103510 +103511 POINT(41.28652472980954 73.90997648658174) bank103511 +103512 POINT(41.63811814211194 73.67953241030213) bank103512 +103513 POINT(41.45684485013233 73.94952994518695) bank103513 +103514 POINT(40.561005609136316 74.5241150805601) bank103514 +103515 POINT(39.91965458747491 73.45239917061619) bank103515 +103516 POINT(40.27085664995825 74.47375349987568) bank103516 +103517 POINT(40.51088089083394 74.63487374050685) bank103517 +103518 POINT(40.396601755864936 74.51324958070305) bank103518 +103519 POINT(41.479341675088214 73.39458360856958) bank103519 +103520 POINT(40.03492830235303 74.7302680008492) bank103520 +103521 POINT(41.16453533514916 73.37896683602175) bank103521 +103522 POINT(41.708369835861674 74.30103591296495) bank103522 +103523 POINT(40.230569464310115 73.8708334183401) bank103523 +103524 POINT(41.130230564100266 73.96966855957344) bank103524 +103525 POINT(40.004177072101506 74.67559423371848) bank103525 +103526 POINT(41.15764177753562 73.86064868538762) bank103526 +103527 POINT(40.887278923408196 74.15368119331826) bank103527 +103528 POINT(41.6605728881316 74.11409914953173) bank103528 +103529 POINT(41.12947981660109 73.31709375784722) bank103529 +103530 POINT(41.29889691835777 74.08194953726961) bank103530 +103531 POINT(40.906893455835196 74.7671469654492) bank103531 +103532 POINT(41.301017627584734 73.07423731171872) bank103532 +103533 POINT(41.00999487240459 73.76390460939787) bank103533 +103534 POINT(41.00219561974489 73.27806701460504) bank103534 +103535 POINT(41.13239234000458 73.61746822250555) bank103535 +103536 POINT(40.4595854709304 74.84027104195991) bank103536 +103537 POINT(41.64173400765918 73.99553361943212) bank103537 +103538 POINT(40.92953161506605 73.11367016146951) bank103538 +103539 POINT(40.30296982984914 73.82252369386077) bank103539 +103540 POINT(40.81272196168529 74.78506305386219) bank103540 +103541 POINT(41.67763620289854 74.01164619459308) bank103541 +103542 POINT(40.3964177255797 74.53946150425105) bank103542 +103543 POINT(40.87730190796468 73.91508782845429) bank103543 +103544 POINT(40.33840356926701 73.36775726475267) bank103544 +103545 POINT(41.42265582849419 73.4978451142449) bank103545 +103546 POINT(40.143066895784536 73.60086403842438) bank103546 +103547 POINT(41.15205173408127 73.76610237989266) bank103547 +103548 POINT(41.24432919790138 73.13931777614954) bank103548 +103549 POINT(41.51384689893392 74.29683750089502) bank103549 +103550 POINT(40.576849092745 74.4969079809215) bank103550 +103551 POINT(39.77638704144015 74.04555339805339) bank103551 +103552 POINT(40.4523223865664 74.17449330749308) bank103552 +103553 POINT(39.88520338729111 73.53086309609834) bank103553 +103554 POINT(41.140648414282055 73.34118525421714) bank103554 +103555 POINT(40.1946256557796 73.8769301992843) bank103555 +103556 POINT(41.13988118992942 73.12337277822662) bank103556 +103557 POINT(39.71990045679131 73.53570814840623) bank103557 +103558 POINT(41.60157073894817 73.62606352928087) bank103558 +103559 POINT(40.885468040092064 75.00154911298387) bank103559 +103560 POINT(41.53525605543411 73.8073975074905) bank103560 +103561 POINT(40.640772641424775 74.28025517619231) bank103561 +103562 POINT(40.730427718888 73.88414497254703) bank103562 +103563 POINT(40.65881873676109 73.9868692955069) bank103563 +103564 POINT(40.371874355218935 74.79566540372241) bank103564 +103565 POINT(41.426866455512894 74.96972364783545) bank103565 +103566 POINT(40.77132022684072 74.11260475258157) bank103566 +103567 POINT(40.94650449451551 73.74871644250963) bank103567 +103568 POINT(40.495243752770186 74.75837685191168) bank103568 +103569 POINT(40.38101631309401 74.00665334744151) bank103569 +103570 POINT(40.77998771779765 73.01579811056307) bank103570 +103571 POINT(39.799634270031355 73.51971269514182) bank103571 +103572 POINT(41.434661069829914 74.17483549416657) bank103572 +103573 POINT(39.946568920012886 74.86730490528188) bank103573 +103574 POINT(41.4019084409075 74.38078815307846) bank103574 +103575 POINT(40.16874438793012 73.83500361927594) bank103575 +103576 POINT(41.127300027050815 74.53941362977346) bank103576 +103577 POINT(41.173162065485016 73.95790638820321) bank103577 +103578 POINT(40.57488641602409 74.32597749645242) bank103578 +103579 POINT(40.090322533200144 74.66713387423368) bank103579 +103580 POINT(39.8376986549442 73.76175461585267) bank103580 +103581 POINT(40.512444142535564 74.35266237723201) bank103581 +103582 POINT(40.23549194685283 74.7834630856645) bank103582 +103583 POINT(39.850541847457635 73.42639043704686) bank103583 +103584 POINT(41.23373733986349 74.55533467067114) bank103584 +103585 POINT(41.61451644812491 74.2092251857515) bank103585 +103586 POINT(41.41685749508018 74.80244479128712) bank103586 +103587 POINT(40.05988137289264 73.24640529465677) bank103587 +103588 POINT(39.89217332475552 73.27459674054666) bank103588 +103589 POINT(40.525935179534386 74.72931480114808) bank103589 +103590 POINT(39.87793525487292 74.44252939223408) bank103590 +103591 POINT(40.13938119373778 74.66266455543077) bank103591 +103592 POINT(40.800775169729455 74.40682471342704) bank103592 +103593 POINT(40.782333571692845 74.11586961324565) bank103593 +103594 POINT(40.871286357239036 74.15439885440976) bank103594 +103595 POINT(39.98174472897792 74.66313814048551) bank103595 +103596 POINT(41.479491759454255 74.449932987191) bank103596 +103597 POINT(40.59552749089349 74.10362455192778) bank103597 +103598 POINT(40.25535965509808 74.70299766822954) bank103598 +103599 POINT(41.63952498623075 74.2858908923177) bank103599 +103600 POINT(41.16201041004802 74.03727242202035) bank103600 +103601 POINT(40.41447939707025 74.50808994676791) bank103601 +103602 POINT(40.04288810880568 74.59227203011324) bank103602 +103603 POINT(41.610142519052175 74.71638989030083) bank103603 +103604 POINT(39.82020368736522 73.05506226309711) bank103604 +103605 POINT(40.25760804671517 74.25332985338945) bank103605 +103606 POINT(39.71366176428497 74.20129300883875) bank103606 +103607 POINT(41.56125384165309 74.12752742948928) bank103607 +103608 POINT(40.13251588731199 73.57659301739261) bank103608 +103609 POINT(40.44896955317647 73.10262047131557) bank103609 +103610 POINT(40.2834084856599 73.90000533052408) bank103610 +103611 POINT(41.17661067162716 74.08526365653114) bank103611 +103612 POINT(41.28890789479718 73.76420986991961) bank103612 +103613 POINT(40.37462936706526 73.1839935959273) bank103613 +103614 POINT(40.59427792327847 74.71879135125083) bank103614 +103615 POINT(41.183551537302684 73.84850112679653) bank103615 +103616 POINT(41.135388464452014 74.3239137373482) bank103616 +103617 POINT(41.51280801788472 74.36756105228386) bank103617 +103618 POINT(41.65344631866152 73.62921200966666) bank103618 +103619 POINT(40.316979179599024 74.46187532676473) bank103619 +103620 POINT(40.87276046288546 73.75583949561816) bank103620 +103621 POINT(41.156043395925245 74.05444123573962) bank103621 +103622 POINT(39.93447087266928 74.93411058052638) bank103622 +103623 POINT(40.98078871520421 74.05897219928076) bank103623 +103624 POINT(41.337455783908 73.64302406177828) bank103624 +103625 POINT(40.801624503238386 74.24735036333574) bank103625 +103626 POINT(41.006599404090174 74.81863525367709) bank103626 +103627 POINT(40.121191642269466 73.04574384644093) bank103627 +103628 POINT(41.28868284947793 73.8918159067684) bank103628 +103629 POINT(41.67188933266674 73.17600437069603) bank103629 +103630 POINT(39.85611869738213 74.93852129313127) bank103630 +103631 POINT(40.38705346040886 73.70086284324417) bank103631 +103632 POINT(41.655041684214986 73.8643018450813) bank103632 +103633 POINT(40.611935691809904 74.18333570306206) bank103633 +103634 POINT(40.894160127307096 74.57541608731448) bank103634 +103635 POINT(39.865193495624304 74.91081398007127) bank103635 +103636 POINT(41.07928408148407 73.3875163333244) bank103636 +103637 POINT(41.382998882396635 73.6578632881235) bank103637 +103638 POINT(40.432718620757385 74.96491544650914) bank103638 +103639 POINT(41.66689783463971 73.01321310133366) bank103639 +103640 POINT(40.331772350511265 74.63404995968462) bank103640 +103641 POINT(40.16205414351339 73.31476662125021) bank103641 +103642 POINT(39.739211328068215 74.81986592521639) bank103642 +103643 POINT(40.96650879081924 73.69694422730501) bank103643 +103644 POINT(41.641531056633816 74.42140541036946) bank103644 +103645 POINT(40.47650500006901 74.08371670555968) bank103645 +103646 POINT(39.89990789739688 74.20593413533383) bank103646 +103647 POINT(40.41398987057704 73.5652744747284) bank103647 +103648 POINT(40.82373368488957 74.43751675564039) bank103648 +103649 POINT(40.350279408331396 74.29931072568071) bank103649 +103650 POINT(40.22320209770895 73.87889521094286) bank103650 +103651 POINT(41.60026131246394 73.74401948851025) bank103651 +103652 POINT(41.14002386613778 73.46765942137007) bank103652 +103653 POINT(41.432807201430435 73.88040912031167) bank103653 +103654 POINT(41.663151849447054 73.78229866163113) bank103654 +103655 POINT(41.10994940087303 73.15240632559045) bank103655 +103656 POINT(41.285010356682 74.25066094533557) bank103656 +103657 POINT(41.16410409798776 74.24555311212694) bank103657 +103658 POINT(40.02428816817441 73.88299671477947) bank103658 +103659 POINT(40.77759565916612 74.0843043103908) bank103659 +103660 POINT(39.964972993456726 73.41858916433011) bank103660 +103661 POINT(40.20219527262289 74.6909722289156) bank103661 +103662 POINT(40.16578761104632 74.524429263017) bank103662 +103663 POINT(40.22485620942866 74.85403251747168) bank103663 +103664 POINT(40.157714833700574 73.82285679270034) bank103664 +103665 POINT(40.46007338977652 73.46076907610166) bank103665 +103666 POINT(39.86417280824403 74.90048976267276) bank103666 +103667 POINT(39.80776386920565 73.08492775677861) bank103667 +103668 POINT(40.99682386334044 74.90124912589988) bank103668 +103669 POINT(39.824143460733055 74.05042159398683) bank103669 +103670 POINT(40.22669489395033 74.33636030890607) bank103670 +103671 POINT(39.885041512346795 73.52247643192695) bank103671 +103672 POINT(41.670430839501314 74.48874825185823) bank103672 +103673 POINT(41.2334446806966 73.0527409006707) bank103673 +103674 POINT(40.22384666762647 74.5850601614523) bank103674 +103675 POINT(41.645683762970386 73.04485408981078) bank103675 +103676 POINT(40.46347701413018 73.38369048939693) bank103676 +103677 POINT(40.14565802797634 73.08569237588107) bank103677 +103678 POINT(41.01603452492663 74.68375360947981) bank103678 +103679 POINT(39.842126429955314 73.88528576457432) bank103679 +103680 POINT(40.356172784116964 74.17643068319407) bank103680 +103681 POINT(41.061633349061005 73.39994050624276) bank103681 +103682 POINT(40.21545647800997 74.24533712392238) bank103682 +103683 POINT(40.732714888828596 74.72793575271149) bank103683 +103684 POINT(41.0140393069038 73.30621213720026) bank103684 +103685 POINT(40.8500212343794 73.26692942553744) bank103685 +103686 POINT(41.52259109158602 73.06441548718291) bank103686 +103687 POINT(41.14755346557466 75.0006294857632) bank103687 +103688 POINT(40.25350496714379 73.0767810989778) bank103688 +103689 POINT(40.55312925990493 74.00444764056004) bank103689 +103690 POINT(40.40020996644742 74.37313057852883) bank103690 +103691 POINT(40.155349251971785 74.119674821156) bank103691 +103692 POINT(41.24736190194372 73.1732688803072) bank103692 +103693 POINT(40.343326726061605 74.71804372544543) bank103693 +103694 POINT(40.847837899275376 73.42131431632748) bank103694 +103695 POINT(41.612703908023846 73.92888928925532) bank103695 +103696 POINT(41.20779175480667 73.06028213718946) bank103696 +103697 POINT(41.493373877234504 73.1455717547004) bank103697 +103698 POINT(40.5377263287238 74.24723472043674) bank103698 +103699 POINT(39.713599364229665 73.20397904408395) bank103699 +103700 POINT(40.97564296678415 73.50980222320544) bank103700 +103701 POINT(40.979885315231876 74.49275053060897) bank103701 +103702 POINT(40.00056138542453 73.51121362113236) bank103702 +103703 POINT(41.07965768631586 73.35134803609486) bank103703 +103704 POINT(41.13090178644753 73.84645690677341) bank103704 +103705 POINT(41.56329344523938 74.84606550884541) bank103705 +103706 POINT(39.7760080618481 73.32703455668354) bank103706 +103707 POINT(40.960710466836815 74.21542744704894) bank103707 +103708 POINT(40.451748407640856 73.89095856994054) bank103708 +103709 POINT(40.38033788025671 74.55852305496526) bank103709 +103710 POINT(41.178204583189576 74.88082158112778) bank103710 +103711 POINT(40.10968468780697 73.66407659153644) bank103711 +103712 POINT(40.580046906190084 74.80451579577702) bank103712 +103713 POINT(41.30268279220909 74.0050325987588) bank103713 +103714 POINT(40.75563462627936 73.82640928124901) bank103714 +103715 POINT(41.26491329531823 74.96418096684333) bank103715 +103716 POINT(39.91648017921962 73.0361024203365) bank103716 +103717 POINT(41.4200035259399 74.39130705165584) bank103717 +103718 POINT(40.799469005870094 74.67445865627325) bank103718 +103719 POINT(40.9735273214833 73.12942181525634) bank103719 +103720 POINT(41.331755919272254 74.87291132774234) bank103720 +103721 POINT(41.33570862922466 74.87211191385863) bank103721 +103722 POINT(40.55100145576951 73.13401335775478) bank103722 +103723 POINT(40.98353848752531 73.24758377799026) bank103723 +103724 POINT(40.14241230570143 73.41483240423356) bank103724 +103725 POINT(40.41814531849007 73.58851927180163) bank103725 +103726 POINT(41.0789571174 74.58521132681669) bank103726 +103727 POINT(40.84271519581907 74.63736256245915) bank103727 +103728 POINT(39.750359273587605 74.88208725592564) bank103728 +103729 POINT(40.53312807721157 74.69898176476013) bank103729 +103730 POINT(39.8847948225726 73.40383669873353) bank103730 +103731 POINT(40.38921555940171 74.67033729427337) bank103731 +103732 POINT(40.04506975007402 73.32554911908025) bank103732 +103733 POINT(39.72734985373754 73.96829150791109) bank103733 +103734 POINT(41.30070119003433 73.12996340008968) bank103734 +103735 POINT(41.618930575262944 74.30975395775482) bank103735 +103736 POINT(39.888157295602554 73.08812051736594) bank103736 +103737 POINT(39.83908502022251 73.85128497927685) bank103737 +103738 POINT(41.51057455575574 74.44382310504619) bank103738 +103739 POINT(40.644268557997435 74.6267670405095) bank103739 +103740 POINT(40.03057451851286 74.44521966099443) bank103740 +103741 POINT(41.64093418543668 73.67566197914759) bank103741 +103742 POINT(41.17335620685658 74.25200018580583) bank103742 +103743 POINT(41.48037140191144 74.49001553571641) bank103743 +103744 POINT(40.940318313665614 73.57298562675398) bank103744 +103745 POINT(41.13003678296716 74.24986741004085) bank103745 +103746 POINT(41.11188645956576 73.2135002048233) bank103746 +103747 POINT(40.67422561548688 73.52258484418327) bank103747 +103748 POINT(40.51679447010688 74.69950319355131) bank103748 +103749 POINT(40.05363785102185 74.3521729406228) bank103749 +103750 POINT(41.28970648463787 73.29053063346527) bank103750 +103751 POINT(40.529821701205 73.34145689382561) bank103751 +103752 POINT(40.17614598034508 73.75774404331835) bank103752 +103753 POINT(41.538502557769675 74.75057337269153) bank103753 +103754 POINT(40.966109305152 74.04497325855426) bank103754 +103755 POINT(39.71535145053155 74.83570286119033) bank103755 +103756 POINT(40.2541810160902 74.65704534404165) bank103756 +103757 POINT(40.655920480595306 74.73739920661066) bank103757 +103758 POINT(40.32752666524571 73.0936731985871) bank103758 +103759 POINT(40.25047065818766 73.15076345167905) bank103759 +103760 POINT(41.655845992554475 74.47657326586545) bank103760 +103761 POINT(40.82573104954537 74.74717588992691) bank103761 +103762 POINT(40.77387622608462 74.64721960153838) bank103762 +103763 POINT(41.67162679269526 74.61227725735948) bank103763 +103764 POINT(40.700601482437015 74.49538622578493) bank103764 +103765 POINT(40.334377267965614 73.13909883196534) bank103765 +103766 POINT(41.01068524110821 73.62781515517247) bank103766 +103767 POINT(40.792873838392275 74.30952182140078) bank103767 +103768 POINT(41.68555253408203 73.28571841640112) bank103768 +103769 POINT(40.70130030066694 74.51175823207791) bank103769 +103770 POINT(40.55618487570683 73.09757936865637) bank103770 +103771 POINT(39.94527540591887 73.54625523580964) bank103771 +103772 POINT(41.274398249527934 74.32167376436408) bank103772 +103773 POINT(39.88592208509824 74.04625953104008) bank103773 +103774 POINT(41.372740827799156 73.52649840114015) bank103774 +103775 POINT(40.00001013940329 73.89316990723049) bank103775 +103776 POINT(41.33768377388742 74.21855513711148) bank103776 +103777 POINT(40.613672340989936 74.67622061021525) bank103777 +103778 POINT(40.02642051035903 74.1592473213937) bank103778 +103779 POINT(41.226308438369294 73.56818173693891) bank103779 +103780 POINT(40.52088452746048 74.71859190945095) bank103780 +103781 POINT(40.801095715226865 73.4353979012161) bank103781 +103782 POINT(40.53843776883715 74.61572980578002) bank103782 +103783 POINT(41.13822482504161 74.85943825226893) bank103783 +103784 POINT(41.311813252017274 73.47331076539979) bank103784 +103785 POINT(39.893871859461456 74.60141021327084) bank103785 +103786 POINT(39.78313599000369 73.44702915787467) bank103786 +103787 POINT(40.98009866344773 74.90877946039569) bank103787 +103788 POINT(41.14292757580357 74.06614581111492) bank103788 +103789 POINT(41.204999183067734 73.20524586564738) bank103789 +103790 POINT(41.24626542974122 73.71757922191333) bank103790 +103791 POINT(41.274632626644035 74.6706230928353) bank103791 +103792 POINT(39.93147982208018 74.38128220893786) bank103792 +103793 POINT(39.8979691633902 74.83830252096797) bank103793 +103794 POINT(40.813849713384286 74.2247255689929) bank103794 +103795 POINT(40.379885502168726 73.301983579389) bank103795 +103796 POINT(41.19584121938639 74.48853495757184) bank103796 +103797 POINT(40.108084799550554 73.89328540902105) bank103797 +103798 POINT(40.12443600271123 73.99181639300477) bank103798 +103799 POINT(39.75241445731504 73.720230863244) bank103799 +103800 POINT(39.989932193045206 74.82308960149682) bank103800 +103801 POINT(39.98746906568736 74.27295237351076) bank103801 +103802 POINT(41.49418313712701 73.34688437444827) bank103802 +103803 POINT(40.82490746474768 73.92063396174152) bank103803 +103804 POINT(39.927320179834425 73.30369308090225) bank103804 +103805 POINT(41.451597734721474 74.73764760893123) bank103805 +103806 POINT(40.73692459868677 74.27437961724085) bank103806 +103807 POINT(40.49538830970449 73.78703786461143) bank103807 +103808 POINT(41.57436470161566 74.04025360017698) bank103808 +103809 POINT(40.0667683311695 74.8623760738171) bank103809 +103810 POINT(40.868492395985086 74.71823487997) bank103810 +103811 POINT(41.2548786837276 74.46185152349774) bank103811 +103812 POINT(39.923708146492636 73.05201103032263) bank103812 +103813 POINT(40.023335889606074 74.09530641900753) bank103813 +103814 POINT(40.30390200963085 74.91362792812929) bank103814 +103815 POINT(40.238710603850656 73.11230185890301) bank103815 +103816 POINT(40.644584023588735 74.51069342192648) bank103816 +103817 POINT(39.88236895735305 73.03326671053193) bank103817 +103818 POINT(40.04403500746262 74.36757684100299) bank103818 +103819 POINT(41.038956682311635 73.97131085046476) bank103819 +103820 POINT(39.94483502992493 73.73030190944921) bank103820 +103821 POINT(39.73629022902663 73.51589232751255) bank103821 +103822 POINT(41.49276898333996 74.78132807380601) bank103822 +103823 POINT(40.08654122529347 74.59404529334932) bank103823 +103824 POINT(40.330608140867646 73.70114193453462) bank103824 +103825 POINT(39.87726391770297 73.91174375613407) bank103825 +103826 POINT(41.42899141155123 73.66995222050738) bank103826 +103827 POINT(40.24120266251137 74.00653492224525) bank103827 +103828 POINT(41.441247397996285 74.11736061922717) bank103828 +103829 POINT(41.200189552490656 74.2428281377223) bank103829 +103830 POINT(40.48127178713284 74.4143990003608) bank103830 +103831 POINT(40.28003129084368 74.10492419704012) bank103831 +103832 POINT(40.078198278762166 74.70575516098602) bank103832 +103833 POINT(41.162810394358 74.03877919236146) bank103833 +103834 POINT(40.63922047328976 73.37208391112401) bank103834 +103835 POINT(40.98210910015107 74.64414776517172) bank103835 +103836 POINT(41.55486633699011 73.80964438735722) bank103836 +103837 POINT(40.131424050496115 74.72559880698205) bank103837 +103838 POINT(40.204652806166685 74.97966994535696) bank103838 +103839 POINT(41.19649069404699 73.49550778588242) bank103839 +103840 POINT(40.11283155493647 73.88036892860363) bank103840 +103841 POINT(40.80930478989695 74.37492102432346) bank103841 +103842 POINT(39.91373216022517 73.08223497901191) bank103842 +103843 POINT(40.132516823836326 73.01424204806698) bank103843 +103844 POINT(39.988376588371274 73.07305053744244) bank103844 +103845 POINT(40.42908391537904 73.52163651725648) bank103845 +103846 POINT(40.797473170158405 74.58077270813708) bank103846 +103847 POINT(39.93174050442696 74.36953096187939) bank103847 +103848 POINT(40.89983332939157 73.7423169080551) bank103848 +103849 POINT(39.8648867785758 73.94334732915868) bank103849 +103850 POINT(40.364823110401794 73.74901123238286) bank103850 +103851 POINT(41.4357545091693 74.36384706183422) bank103851 +103852 POINT(40.82105529331793 73.43283895043882) bank103852 +103853 POINT(40.45576424710289 74.09326591923694) bank103853 +103854 POINT(39.97183873192173 74.70197034836596) bank103854 +103855 POINT(40.81714666470443 74.64974481044356) bank103855 +103856 POINT(41.44373096955035 74.32557134244956) bank103856 +103857 POINT(41.15760360843079 74.46308436845027) bank103857 +103858 POINT(41.54167876843158 73.39232693146681) bank103858 +103859 POINT(41.407153973269196 74.43547002693207) bank103859 +103860 POINT(40.72224560143513 73.01981934861874) bank103860 +103861 POINT(40.22272429393188 73.06814520118029) bank103861 +103862 POINT(40.51472854366278 74.42760159053476) bank103862 +103863 POINT(39.909265589298855 73.14777841274301) bank103863 +103864 POINT(40.41458801533685 73.44171965161891) bank103864 +103865 POINT(41.5876964957831 74.08465848127543) bank103865 +103866 POINT(41.402064355097224 74.89245363175368) bank103866 +103867 POINT(40.730578203980066 74.2252240937838) bank103867 +103868 POINT(39.748494168715446 73.3017651770066) bank103868 +103869 POINT(40.633180470815816 73.6357195991517) bank103869 +103870 POINT(41.548185337109146 73.22145725113396) bank103870 +103871 POINT(41.43942893604711 74.40767009901592) bank103871 +103872 POINT(39.994900789852124 74.21099246836927) bank103872 +103873 POINT(40.62738525915541 74.64807032207797) bank103873 +103874 POINT(40.32328341541193 74.18632224819389) bank103874 +103875 POINT(40.459935401476685 73.52728169772114) bank103875 +103876 POINT(41.372171175928536 73.47159493825826) bank103876 +103877 POINT(40.86794015753018 74.08399708651532) bank103877 +103878 POINT(41.536260505647014 73.09893645317952) bank103878 +103879 POINT(40.53672650377524 74.90105168489903) bank103879 +103880 POINT(40.07812862084259 74.07023291389083) bank103880 +103881 POINT(41.02942058373995 74.60594079124374) bank103881 +103882 POINT(40.230747046654486 73.54168167735652) bank103882 +103883 POINT(41.070435335618875 74.43667181852159) bank103883 +103884 POINT(40.68146678316095 73.09799028912673) bank103884 +103885 POINT(40.547037499743 74.8660010570756) bank103885 +103886 POINT(41.537679453887066 73.93059947330455) bank103886 +103887 POINT(41.46128440433806 73.8998760788706) bank103887 +103888 POINT(39.98895705346241 73.569571151465) bank103888 +103889 POINT(40.06074212231561 74.4458443417267) bank103889 +103890 POINT(40.86488645716224 73.92872076318001) bank103890 +103891 POINT(41.138971546083 74.51631199840043) bank103891 +103892 POINT(41.196181547165786 73.17707958600658) bank103892 +103893 POINT(40.72752145577117 74.8428399389837) bank103893 +103894 POINT(39.92398014267322 74.59725307557326) bank103894 +103895 POINT(41.22792611675675 73.42256419218508) bank103895 +103896 POINT(41.03159638433923 73.19269643777132) bank103896 +103897 POINT(40.935074729040686 74.52137623168633) bank103897 +103898 POINT(41.02575069680149 74.94664216942986) bank103898 +103899 POINT(40.43507312957998 73.36469479237773) bank103899 +103900 POINT(41.69074594647134 73.08482198767766) bank103900 +103901 POINT(40.64955384971672 73.08881864619609) bank103901 +103902 POINT(40.81514932310742 73.4591200588054) bank103902 +103903 POINT(41.100189501588154 73.06570994648584) bank103903 +103904 POINT(40.52389865922698 73.29621630019967) bank103904 +103905 POINT(40.23194815293529 74.24480895842889) bank103905 +103906 POINT(40.3389191486075 73.11306785371295) bank103906 +103907 POINT(41.387480712261564 74.79230379450692) bank103907 +103908 POINT(40.240934071590644 73.26771267637291) bank103908 +103909 POINT(40.28951049916187 73.5601334003495) bank103909 +103910 POINT(41.56167330415951 73.81384959448819) bank103910 +103911 POINT(39.76695366992128 73.9606517145068) bank103911 +103912 POINT(39.96355888119539 73.91089806363279) bank103912 +103913 POINT(41.422108037237784 73.88226755812747) bank103913 +103914 POINT(40.98735478188088 74.21696527305576) bank103914 +103915 POINT(41.67359772440377 74.87892619773162) bank103915 +103916 POINT(39.870908896091926 74.17910486583114) bank103916 +103917 POINT(39.92278522530534 73.1516344404262) bank103917 +103918 POINT(40.17213033792164 74.86545580393059) bank103918 +103919 POINT(41.29999575559272 73.43786786054348) bank103919 +103920 POINT(40.18006924338439 73.03865276288234) bank103920 +103921 POINT(41.38172021026859 74.28907271502135) bank103921 +103922 POINT(40.110776389646716 73.70422151470905) bank103922 +103923 POINT(40.50774751745312 73.67830192322947) bank103923 +103924 POINT(40.61849870436995 74.68289421309277) bank103924 +103925 POINT(39.77895203300891 73.09236629323966) bank103925 +103926 POINT(40.901084805978726 74.79058362307904) bank103926 +103927 POINT(41.464979544175016 74.89191824190866) bank103927 +103928 POINT(40.35232248229163 73.77774063999283) bank103928 +103929 POINT(39.92192778597046 74.6300341231839) bank103929 +103930 POINT(41.68759410412103 73.33236452338538) bank103930 +103931 POINT(39.92649013730157 73.45743144112922) bank103931 +103932 POINT(40.85717113835753 75.00440926861599) bank103932 +103933 POINT(41.56944150419543 73.42397438965924) bank103933 +103934 POINT(41.665315465952624 73.42357077950673) bank103934 +103935 POINT(39.91951916399691 74.87140682516322) bank103935 +103936 POINT(40.57040798656906 74.20702199561696) bank103936 +103937 POINT(40.96209043473931 74.7938151202691) bank103937 +103938 POINT(40.54804903524124 74.42189697270146) bank103938 +103939 POINT(41.021661645574795 73.71823895874618) bank103939 +103940 POINT(40.46893159964158 74.66080158838439) bank103940 +103941 POINT(40.69173767846836 74.2196949067325) bank103941 +103942 POINT(41.30904417506837 74.57946823786139) bank103942 +103943 POINT(40.88567982394642 74.83954638306272) bank103943 +103944 POINT(41.57137581148576 74.7862236092819) bank103944 +103945 POINT(40.075411484077655 74.12827636917928) bank103945 +103946 POINT(40.90682799111705 74.71457501156968) bank103946 +103947 POINT(40.90405304715744 74.53654764020766) bank103947 +103948 POINT(40.20544238087196 73.44860408026261) bank103948 +103949 POINT(40.923723918555595 73.81747648036279) bank103949 +103950 POINT(40.04633082882658 73.227485030841) bank103950 +103951 POINT(40.40406108975412 73.82586339290013) bank103951 +103952 POINT(40.41229877627224 74.37012873169147) bank103952 +103953 POINT(41.59958725156172 73.92975717170592) bank103953 +103954 POINT(41.455538333752564 74.28683016326224) bank103954 +103955 POINT(41.02082885337829 73.69768958717835) bank103955 +103956 POINT(40.000117417265116 74.85538540263374) bank103956 +103957 POINT(41.418420415107775 74.11121597958522) bank103957 +103958 POINT(41.031056554566526 74.18618229418414) bank103958 +103959 POINT(41.68251463599433 73.20304231389237) bank103959 +103960 POINT(41.40174487096113 73.9571899722104) bank103960 +103961 POINT(39.804162073752856 73.6923674917889) bank103961 +103962 POINT(40.27541989806149 74.90297179970455) bank103962 +103963 POINT(40.61295825247927 73.64485250767987) bank103963 +103964 POINT(39.81157168796242 73.5436497464102) bank103964 +103965 POINT(40.55804972556821 74.07367651854385) bank103965 +103966 POINT(40.64434439479844 73.55353439693592) bank103966 +103967 POINT(41.70108338712852 74.42146010540775) bank103967 +103968 POINT(41.67646446850779 74.08631158121322) bank103968 +103969 POINT(40.401855666409176 74.944363577351) bank103969 +103970 POINT(41.225531850949324 74.44654420362238) bank103970 +103971 POINT(41.46530273651496 74.38585685099319) bank103971 +103972 POINT(40.61652782598399 73.50645068349796) bank103972 +103973 POINT(39.817319211457814 73.4654848903214) bank103973 +103974 POINT(40.168570148833666 74.43680957591968) bank103974 +103975 POINT(41.26949343229626 74.23176410156262) bank103975 +103976 POINT(41.17553024706896 74.88934770288823) bank103976 +103977 POINT(39.72038923709333 73.65837286828527) bank103977 +103978 POINT(41.70346761557377 74.79038294972466) bank103978 +103979 POINT(40.27164091675597 74.15828750000793) bank103979 +103980 POINT(39.72189602167151 74.35555300695934) bank103980 +103981 POINT(41.24096450361358 74.77052024876589) bank103981 +103982 POINT(40.70286405566462 73.05200049611491) bank103982 +103983 POINT(40.166985713054245 73.94425507250169) bank103983 +103984 POINT(41.62059960214082 74.09334115148089) bank103984 +103985 POINT(40.93540287667606 74.33234398832622) bank103985 +103986 POINT(40.448857868736795 73.60812965267107) bank103986 +103987 POINT(39.954909640683844 74.99292030987768) bank103987 +103988 POINT(41.4577524797876 73.18705416436087) bank103988 +103989 POINT(41.09848578398077 73.4761042618641) bank103989 +103990 POINT(41.12686494160757 73.04759939447771) bank103990 +103991 POINT(39.98200885004057 73.21923451322827) bank103991 +103992 POINT(40.54116554399424 74.57669081638969) bank103992 +103993 POINT(40.346757006096965 73.89259732281744) bank103993 +103994 POINT(39.73341825853056 74.49567668356354) bank103994 +103995 POINT(41.51929603590119 73.73457177501038) bank103995 +103996 POINT(40.4527809835278 73.34684536973491) bank103996 +103997 POINT(39.748907300124515 74.62540521127453) bank103997 +103998 POINT(39.87781620904774 74.63092963185055) bank103998 +103999 POINT(40.07328106969313 73.26398652411376) bank103999 +104000 POINT(39.90814500644686 73.76895061358398) bank104000 +104001 POINT(40.11977632211993 74.02523824626162) bank104001 +104002 POINT(39.867127429890424 74.57688262576342) bank104002 +104003 POINT(40.47227610017852 73.6244788358857) bank104003 +104004 POINT(41.208398035650575 73.43788185538206) bank104004 +104005 POINT(40.23196586439279 74.00731487370304) bank104005 +104006 POINT(41.20336371564505 73.78324572628497) bank104006 +104007 POINT(40.85232094371529 73.92521395245494) bank104007 +104008 POINT(39.755422993106905 73.19877696811137) bank104008 +104009 POINT(40.9406747751194 73.78467796163582) bank104009 +104010 POINT(41.43723306702274 73.02865318766659) bank104010 +104011 POINT(41.69923782604195 74.3640895247824) bank104011 +104012 POINT(41.68965247713266 73.1817069402915) bank104012 +104013 POINT(40.38987228934681 74.675185334468) bank104013 +104014 POINT(41.34480726559952 73.1235861052881) bank104014 +104015 POINT(40.013368736962 73.28025977720947) bank104015 +104016 POINT(40.95342188035107 73.16461155756883) bank104016 +104017 POINT(40.24786064476966 74.25549635999799) bank104017 +104018 POINT(41.308832085926944 74.27536737078954) bank104018 +104019 POINT(40.62114825795755 73.89111176972719) bank104019 +104020 POINT(40.29920014362525 73.89639299770326) bank104020 +104021 POINT(40.33345031606456 73.0200812408692) bank104021 +104022 POINT(41.10377542622772 74.38862252969854) bank104022 +104023 POINT(40.77057754304444 73.78746251148047) bank104023 +104024 POINT(41.60599838862244 74.13539359836052) bank104024 +104025 POINT(41.03214801400595 74.80824899614562) bank104025 +104026 POINT(40.804201690886515 74.35268602894499) bank104026 +104027 POINT(41.63150157565143 74.26125269037864) bank104027 +104028 POINT(40.523248076978426 73.92639292590863) bank104028 +104029 POINT(40.00039846262841 74.82930259592261) bank104029 +104030 POINT(40.76948865158655 74.23322939600767) bank104030 +104031 POINT(41.497348143578634 73.4158332307012) bank104031 +104032 POINT(41.316042189429474 74.85857518248841) bank104032 +104033 POINT(41.587287949010594 74.33794039417148) bank104033 +104034 POINT(41.6715147589978 74.49972987427064) bank104034 +104035 POINT(41.16822748328461 74.02973924328637) bank104035 +104036 POINT(41.02868345234526 73.6120147863961) bank104036 +104037 POINT(40.37100681488092 73.61280988858329) bank104037 +104038 POINT(41.081864746306856 74.22447675856257) bank104038 +104039 POINT(39.8261051532291 73.26587136161731) bank104039 +104040 POINT(41.42995591963411 74.773275348595) bank104040 +104041 POINT(39.956621927337764 73.31972776147592) bank104041 +104042 POINT(40.95991786885643 73.31840935926293) bank104042 +104043 POINT(40.92013666136065 73.44068704699822) bank104043 +104044 POINT(40.3479622483111 74.21760791840538) bank104044 +104045 POINT(40.3841494026258 73.19110214671929) bank104045 +104046 POINT(39.968427451922686 74.06192777579248) bank104046 +104047 POINT(41.630724526692404 73.43345822870404) bank104047 +104048 POINT(39.86370258453885 73.55410513866481) bank104048 +104049 POINT(41.01496637359042 73.92631391772274) bank104049 +104050 POINT(40.762235339874685 74.37571921329194) bank104050 +104051 POINT(41.329266988739306 73.45587586552043) bank104051 +104052 POINT(41.50481658937294 74.73456217641606) bank104052 +104053 POINT(40.298224409489535 74.0274440318576) bank104053 +104054 POINT(40.23578061168695 74.52857914938096) bank104054 +104055 POINT(41.03527995346049 73.71164427962914) bank104055 +104056 POINT(41.4423550407428 74.74077736362845) bank104056 +104057 POINT(40.290499981786255 73.15473343631885) bank104057 +104058 POINT(40.930012014254665 73.29943288005293) bank104058 +104059 POINT(41.306910900507525 73.7427327276772) bank104059 +104060 POINT(40.17823408948835 74.39464142024012) bank104060 +104061 POINT(39.750477585660036 74.57361995112598) bank104061 +104062 POINT(40.60053608915865 74.50153891072935) bank104062 +104063 POINT(40.46290429609627 74.65918144070004) bank104063 +104064 POINT(41.23383878100402 74.61585554400455) bank104064 +104065 POINT(41.12134450386404 73.06465393118343) bank104065 +104066 POINT(40.12140512027023 74.74326088211616) bank104066 +104067 POINT(40.06419707770099 74.15467382503114) bank104067 +104068 POINT(40.09446926507201 74.39419800653043) bank104068 +104069 POINT(41.43867507134062 73.34358838639098) bank104069 +104070 POINT(40.795610000052775 74.507373911843) bank104070 +104071 POINT(41.40878441545489 73.29279596854943) bank104071 +104072 POINT(39.81416324318722 74.3155836931467) bank104072 +104073 POINT(39.74610462010957 73.864540400188) bank104073 +104074 POINT(40.711903246000205 73.25747244809524) bank104074 +104075 POINT(41.43658921058515 74.13590072271609) bank104075 +104076 POINT(41.42882266177812 73.94972781173057) bank104076 +104077 POINT(41.64562593811658 73.61727552699855) bank104077 +104078 POINT(41.55299165557702 74.16729265296762) bank104078 +104079 POINT(40.291521707774045 73.34460855131138) bank104079 +104080 POINT(40.32561486481282 73.62003347914236) bank104080 +104081 POINT(40.67058717729811 74.05468628032592) bank104081 +104082 POINT(40.858264782783216 74.10542367423213) bank104082 +104083 POINT(41.29062748391171 74.32819359987619) bank104083 +104084 POINT(39.723604868816956 73.56721182091371) bank104084 +104085 POINT(40.2989914652715 73.24422747802588) bank104085 +104086 POINT(41.59395089832427 74.19823955161289) bank104086 +104087 POINT(41.07237125267876 73.77082952252115) bank104087 +104088 POINT(40.48039755275032 74.23659331231731) bank104088 +104089 POINT(40.56259417961926 74.31161095672259) bank104089 +104090 POINT(39.94543750435295 73.50546190623584) bank104090 +104091 POINT(41.33563002630215 74.68187769646946) bank104091 +104092 POINT(40.984397525002194 74.0141148375292) bank104092 +104093 POINT(39.95889370832326 73.40140881428852) bank104093 +104094 POINT(40.859340483451476 74.06573784714024) bank104094 +104095 POINT(39.96579732456534 74.81193308057955) bank104095 +104096 POINT(40.473136768348375 74.42266326340471) bank104096 +104097 POINT(40.073675198475414 73.9024557221957) bank104097 +104098 POINT(39.817932129032286 73.69325854429) bank104098 +104099 POINT(39.905817441755666 74.19325670209078) bank104099 +104100 POINT(40.05347264668596 74.60769474149869) bank104100 +104101 POINT(41.027102743511016 73.06760034283475) bank104101 +104102 POINT(40.93038923479748 73.72908353762965) bank104102 +104103 POINT(39.77958399828004 73.78751983315705) bank104103 +104104 POINT(39.989461139265515 73.76446114244877) bank104104 +104105 POINT(39.840102535022204 74.76868381292351) bank104105 +104106 POINT(41.36665271335567 73.17140778402859) bank104106 +104107 POINT(41.24724967477174 73.5814652139409) bank104107 +104108 POINT(40.46262961093322 73.62482247049968) bank104108 +104109 POINT(40.56688546738877 74.05272258364587) bank104109 +104110 POINT(40.98731975103586 73.82853215870394) bank104110 +104111 POINT(41.199579017258124 73.66658351501593) bank104111 +104112 POINT(41.65154805517704 73.29417433543888) bank104112 +104113 POINT(40.77562316025594 73.69424010798333) bank104113 +104114 POINT(41.23099353254042 74.92937054667847) bank104114 +104115 POINT(40.49430717159162 73.63301774833037) bank104115 +104116 POINT(40.43459213932452 74.30241801660007) bank104116 +104117 POINT(40.95808778596087 74.72493151890622) bank104117 +104118 POINT(41.55977825050719 74.33184685229706) bank104118 +104119 POINT(41.085285298369904 73.07126515423464) bank104119 +104120 POINT(40.16996257152682 74.87873040842405) bank104120 +104121 POINT(41.529422142145165 74.74563697256995) bank104121 +104122 POINT(41.26244263695666 74.03611896549245) bank104122 +104123 POINT(41.027271616764324 73.61794732173716) bank104123 +104124 POINT(40.461601180844866 73.84254816415417) bank104124 +104125 POINT(41.65983408054999 73.7248010457237) bank104125 +104126 POINT(40.57071066188922 73.73541033662671) bank104126 +104127 POINT(40.406104551471024 73.75763281154292) bank104127 +104128 POINT(40.861866426159146 74.27416996542047) bank104128 +104129 POINT(40.44437238166333 73.11331289216949) bank104129 +104130 POINT(40.08955466419491 73.51893072614588) bank104130 +104131 POINT(40.016917462570575 73.25329006658062) bank104131 +104132 POINT(40.336202332664264 74.18969588080935) bank104132 +104133 POINT(40.95466509627094 73.51401816391605) bank104133 +104134 POINT(40.93606313581747 73.75230987877015) bank104134 +104135 POINT(39.90318234418013 74.26291892720184) bank104135 +104136 POINT(40.781176362300926 73.52981833632826) bank104136 +104137 POINT(39.993963751088636 74.8549468958846) bank104137 +104138 POINT(40.77378556621111 74.9562262294369) bank104138 +104139 POINT(39.95503344387805 74.812744796502) bank104139 +104140 POINT(40.30971889517249 74.30592536562045) bank104140 +104141 POINT(41.54271249419603 73.41939409377986) bank104141 +104142 POINT(40.29779104714081 74.18174695683727) bank104142 +104143 POINT(40.70049156052142 73.54171875507888) bank104143 +104144 POINT(40.87221468579583 74.76955445430325) bank104144 +104145 POINT(41.18005843538836 74.14441830012545) bank104145 +104146 POINT(40.54733894327336 74.87985751516418) bank104146 +104147 POINT(40.29738291019025 73.51475057587533) bank104147 +104148 POINT(41.38747291105884 73.97331693084195) bank104148 +104149 POINT(39.81316234483215 74.75300428895994) bank104149 +104150 POINT(40.190620025025034 73.74472188309619) bank104150 +104151 POINT(39.73378649217297 73.6205240367883) bank104151 +104152 POINT(40.47042173191321 74.00352201459569) bank104152 +104153 POINT(41.40928500824964 73.24476099526575) bank104153 +104154 POINT(40.39710939258523 74.40301030649475) bank104154 +104155 POINT(40.18645213208273 74.13065929914346) bank104155 +104156 POINT(41.00530979138792 73.20046374700225) bank104156 +104157 POINT(40.40691021276627 73.1407918487257) bank104157 +104158 POINT(41.36533353823071 73.23322028889758) bank104158 +104159 POINT(39.73840886328882 73.23102447177415) bank104159 +104160 POINT(41.26944380313005 73.52231259685716) bank104160 +104161 POINT(40.03858518878457 74.9738633528123) bank104161 +104162 POINT(41.44595035591313 74.56270299317704) bank104162 +104163 POINT(40.29294665763049 74.49263933956908) bank104163 +104164 POINT(41.39435193959578 74.21811126866668) bank104164 +104165 POINT(40.45182843772646 74.10855804319438) bank104165 +104166 POINT(39.72907551383467 73.06237769336133) bank104166 +104167 POINT(40.20501356601448 73.76267769030231) bank104167 +104168 POINT(41.398161352883655 73.36997190084094) bank104168 +104169 POINT(40.723352154264184 73.10426222631678) bank104169 +104170 POINT(40.811034603706084 74.31752358814613) bank104170 +104171 POINT(40.04123924099125 74.56730051057369) bank104171 +104172 POINT(41.437184974057324 73.01553520614068) bank104172 +104173 POINT(41.16994226081107 73.04891171409157) bank104173 +104174 POINT(40.60353854916111 74.71639895183603) bank104174 +104175 POINT(40.11207004147733 73.39377428104275) bank104175 +104176 POINT(40.60741734072347 73.9150305665901) bank104176 +104177 POINT(39.873925655228064 73.72702133931611) bank104177 +104178 POINT(41.48345210384473 74.27099492055429) bank104178 +104179 POINT(40.04089489752162 73.4795117952998) bank104179 +104180 POINT(40.39140686503915 74.37544284822587) bank104180 +104181 POINT(40.28669502347624 74.09159550637254) bank104181 +104182 POINT(41.60594955296628 74.84615414673358) bank104182 +104183 POINT(41.407395633460624 74.1493620096597) bank104183 +104184 POINT(41.43698305750013 74.30952931355333) bank104184 +104185 POINT(41.513333042095276 73.30599467165871) bank104185 +104186 POINT(41.2480653938385 73.74308483392785) bank104186 +104187 POINT(40.205395735929265 73.85189578328708) bank104187 +104188 POINT(40.34402927587146 74.06063991275943) bank104188 +104189 POINT(41.36948163366869 74.31390476489707) bank104189 +104190 POINT(39.74568420067483 73.02447602397349) bank104190 +104191 POINT(40.09730045424351 73.9389319441465) bank104191 +104192 POINT(40.86528968064092 74.31984264993817) bank104192 +104193 POINT(41.699029592725516 74.36224712437564) bank104193 +104194 POINT(41.176225586121156 73.99857331275483) bank104194 +104195 POINT(41.23963465479447 74.53571831533158) bank104195 +104196 POINT(40.470033691325234 74.10566420849416) bank104196 +104197 POINT(40.72705505606942 74.48193766902357) bank104197 +104198 POINT(40.74627586085595 73.45654786507733) bank104198 +104199 POINT(39.897149116120794 74.86736050811743) bank104199 +104200 POINT(40.49816695641452 73.96356539390133) bank104200 +104201 POINT(40.075557919708146 74.65851036392051) bank104201 +104202 POINT(40.02152578071343 74.23814186687973) bank104202 +104203 POINT(39.73025816441875 73.22745453024794) bank104203 +104204 POINT(40.00362362138056 74.49890191690235) bank104204 +104205 POINT(41.14907949080769 73.11135339927348) bank104205 +104206 POINT(39.87010632514526 74.15392297893203) bank104206 +104207 POINT(39.77810485672794 74.97475223523936) bank104207 +104208 POINT(41.518816541934456 74.72361451362151) bank104208 +104209 POINT(40.27879747592035 73.60656501899847) bank104209 +104210 POINT(40.640548826579824 74.81785180966612) bank104210 +104211 POINT(41.50482625557036 73.0796289167611) bank104211 +104212 POINT(41.445246380383324 73.20233370176284) bank104212 +104213 POINT(40.377505287077426 74.09363520193983) bank104213 +104214 POINT(39.953995786095795 73.66674617752588) bank104214 +104215 POINT(40.04358514031162 74.15899616499367) bank104215 +104216 POINT(40.18709493316459 73.70938309214385) bank104216 +104217 POINT(41.00812305800217 73.44639997386473) bank104217 +104218 POINT(40.71531390969356 74.80418405326677) bank104218 +104219 POINT(41.565940768083664 73.17557546518218) bank104219 +104220 POINT(41.06285242095756 73.48719560538234) bank104220 +104221 POINT(41.57429762380297 73.54070181281924) bank104221 +104222 POINT(40.84942852159058 73.54044758523361) bank104222 +104223 POINT(40.96985200587402 74.30166996992455) bank104223 +104224 POINT(40.767630941849994 74.72900597570444) bank104224 +104225 POINT(39.790098398466284 73.08412381625924) bank104225 +104226 POINT(41.067952383216536 73.16249370960254) bank104226 +104227 POINT(41.54203790473219 74.31733620557391) bank104227 +104228 POINT(41.6642174869073 73.7680097648248) bank104228 +104229 POINT(40.307245386963054 74.7736942494227) bank104229 +104230 POINT(41.12126739498663 74.63264100106596) bank104230 +104231 POINT(40.70083846753931 74.51322632820735) bank104231 +104232 POINT(40.73854085008596 73.40700748992055) bank104232 +104233 POINT(39.984832574946836 73.32726434343655) bank104233 +104234 POINT(41.46994828309322 73.56078000052085) bank104234 +104235 POINT(39.784354063922926 73.8777648694727) bank104235 +104236 POINT(41.41833459429905 74.46824905459445) bank104236 +104237 POINT(41.28289719358646 73.04839772267205) bank104237 +104238 POINT(39.878627259539954 73.70662499635348) bank104238 +104239 POINT(40.12811496043929 73.55607161165027) bank104239 +104240 POINT(41.111950963044194 74.86891370874442) bank104240 +104241 POINT(40.88146906536927 74.45191595477738) bank104241 +104242 POINT(40.08801545914494 73.86759988538998) bank104242 +104243 POINT(40.50468831607719 73.6386611543214) bank104243 +104244 POINT(40.067330765060156 74.4907935194383) bank104244 +104245 POINT(40.9876658675375 74.04701581899567) bank104245 +104246 POINT(40.06677470736506 74.34087168602626) bank104246 +104247 POINT(39.772265415974076 73.21728408986667) bank104247 +104248 POINT(40.4122949381073 73.0788793204332) bank104248 +104249 POINT(41.10708329475387 73.61730482714906) bank104249 +104250 POINT(40.55505300086265 73.81800743279896) bank104250 +104251 POINT(39.78520407145368 74.38446750412005) bank104251 +104252 POINT(40.3183695450257 74.43973284669312) bank104252 +104253 POINT(41.57768145310934 73.8056885534269) bank104253 +104254 POINT(40.443178122345685 73.93196010612466) bank104254 +104255 POINT(39.93360097067471 73.88295572885973) bank104255 +104256 POINT(39.74035866840671 73.52671255679046) bank104256 +104257 POINT(41.05978150962886 74.41363197253654) bank104257 +104258 POINT(40.44274923318177 73.37621127168357) bank104258 +104259 POINT(41.67086402973212 74.4131773899717) bank104259 +104260 POINT(41.3412664434989 73.38054098080974) bank104260 +104261 POINT(40.47496799420016 74.90367239372229) bank104261 +104262 POINT(40.65875750342178 73.62509889459221) bank104262 +104263 POINT(40.787232761616366 73.88783571405204) bank104263 +104264 POINT(41.67062875290566 73.03387562238369) bank104264 +104265 POINT(40.84203340093143 73.49456944926588) bank104265 +104266 POINT(40.501117265797916 74.92402879971247) bank104266 +104267 POINT(40.88425047072934 74.10644842024873) bank104267 +104268 POINT(40.13111053435701 74.90610014637679) bank104268 +104269 POINT(41.496999559287936 74.45186298965925) bank104269 +104270 POINT(41.58294207978703 74.31442135471346) bank104270 +104271 POINT(40.026276265448956 73.75490293811109) bank104271 +104272 POINT(40.52956167453759 74.24023720579503) bank104272 +104273 POINT(41.654635424665706 74.78568516846742) bank104273 +104274 POINT(40.60582012612259 74.60307316050792) bank104274 +104275 POINT(40.068396103576525 73.4831292694163) bank104275 +104276 POINT(40.45652316227222 73.33678515832753) bank104276 +104277 POINT(40.92986702363746 74.3200771834798) bank104277 +104278 POINT(40.38169635246941 74.37633370154106) bank104278 +104279 POINT(39.991509842413834 74.64004701063392) bank104279 +104280 POINT(41.60032731881924 73.53502215609429) bank104280 +104281 POINT(39.91297436693636 74.2193370452799) bank104281 +104282 POINT(40.88259870344916 73.0742877977769) bank104282 +104283 POINT(41.55662493042488 73.36153089465785) bank104283 +104284 POINT(41.130239191624774 73.93759382359111) bank104284 +104285 POINT(41.645833925771115 74.84129048138662) bank104285 +104286 POINT(40.02796291341203 73.31646275887648) bank104286 +104287 POINT(41.14218730472919 74.09101379595712) bank104287 +104288 POINT(40.06979568302837 73.33247463183189) bank104288 +104289 POINT(40.5711288670882 73.06974599001981) bank104289 +104290 POINT(39.85188020566885 74.4888399790813) bank104290 +104291 POINT(41.427767433467665 74.92524026815947) bank104291 +104292 POINT(40.73119495658787 73.26398416165377) bank104292 +104293 POINT(40.26614991001301 74.36323500644929) bank104293 +104294 POINT(41.4417622398638 73.28294962110853) bank104294 +104295 POINT(39.94445329804911 73.9589111294311) bank104295 +104296 POINT(40.93479285904976 74.50944173931802) bank104296 +104297 POINT(39.81406248936712 74.78277353079199) bank104297 +104298 POINT(40.54593812367418 73.38760365398737) bank104298 +104299 POINT(41.27665358653882 74.63919068489743) bank104299 +104300 POINT(41.534069222489556 74.87315232652833) bank104300 +104301 POINT(39.88905614106493 74.85572051345932) bank104301 +104302 POINT(39.89145881205442 74.23277519441687) bank104302 +104303 POINT(40.27904196588562 74.45969018472339) bank104303 +104304 POINT(40.539329730037664 74.7782713043202) bank104304 +104305 POINT(40.882514741119 74.80031897878297) bank104305 +104306 POINT(40.65101357522477 74.89932003677725) bank104306 +104307 POINT(41.520627108926526 73.1488790511791) bank104307 +104308 POINT(39.94003603584409 74.20870716096232) bank104308 +104309 POINT(40.55997887333596 73.57141031204965) bank104309 +104310 POINT(40.12077790127594 74.07020987740127) bank104310 +104311 POINT(41.492156025050114 74.62375628254242) bank104311 +104312 POINT(41.36260171432421 74.92123960663807) bank104312 +104313 POINT(41.035280772031534 74.64182802337552) bank104313 +104314 POINT(40.02609040360162 74.395992025512) bank104314 +104315 POINT(40.82504359896106 74.34612670556157) bank104315 +104316 POINT(40.03064392149679 74.06817613236689) bank104316 +104317 POINT(40.92746782307045 74.17820212939559) bank104317 +104318 POINT(40.3931944135559 73.12821903285396) bank104318 +104319 POINT(40.431980120612195 73.4765481256197) bank104319 +104320 POINT(41.600727506421464 74.97044750966296) bank104320 +104321 POINT(40.14880064136213 73.23324625375574) bank104321 +104322 POINT(40.73248637991733 74.10271274218265) bank104322 +104323 POINT(41.09727143409402 74.54950209447372) bank104323 +104324 POINT(40.64254237224048 74.73709804016018) bank104324 +104325 POINT(41.6946010455552 74.84242279199509) bank104325 +104326 POINT(40.956286171690195 73.5500524001065) bank104326 +104327 POINT(40.00442475227512 73.96080194180173) bank104327 +104328 POINT(41.104146418393285 74.66540530568305) bank104328 +104329 POINT(40.452119922590434 73.4606754129499) bank104329 +104330 POINT(41.546599267523874 74.25347697446139) bank104330 +104331 POINT(41.34242142992375 73.56860871999493) bank104331 +104332 POINT(41.43755207711321 73.43114970348076) bank104332 +104333 POINT(40.11512473267774 74.58463958860041) bank104333 +104334 POINT(41.45444686822991 74.11640675349828) bank104334 +104335 POINT(40.845851547499144 73.99282881814301) bank104335 +104336 POINT(41.47399556471904 74.98493890214142) bank104336 +104337 POINT(40.81603506924665 73.64693193094104) bank104337 +104338 POINT(40.24925654088513 74.9502003098083) bank104338 +104339 POINT(41.66146555239104 74.51171469688987) bank104339 +104340 POINT(41.54700659760904 73.24583359354162) bank104340 +104341 POINT(39.79078084640482 73.08554619171082) bank104341 +104342 POINT(41.469988771253256 74.79849308273363) bank104342 +104343 POINT(39.85046980983531 74.1359425709874) bank104343 +104344 POINT(39.89449891667434 73.44334984493284) bank104344 +104345 POINT(40.1246420029324 74.48493216288387) bank104345 +104346 POINT(40.033657039300465 73.76233739828032) bank104346 +104347 POINT(40.96500122099629 74.93425666163333) bank104347 +104348 POINT(41.01675547937413 74.6602458573104) bank104348 +104349 POINT(41.10854891714204 74.34678534423892) bank104349 +104350 POINT(40.64238378695056 74.75903261011693) bank104350 +104351 POINT(40.857999391556085 73.7160723659468) bank104351 +104352 POINT(40.042824919213004 73.39476950693138) bank104352 +104353 POINT(40.02406361045098 73.21611218318257) bank104353 +104354 POINT(40.93491179649844 73.17845024851228) bank104354 +104355 POINT(40.402422016706865 74.3259561320025) bank104355 +104356 POINT(39.79422307161972 74.51041028247855) bank104356 +104357 POINT(39.8414723836338 73.61530019787489) bank104357 +104358 POINT(41.315785147976854 74.70955057397929) bank104358 +104359 POINT(40.87548061869976 73.14852265811199) bank104359 +104360 POINT(41.60442552972441 73.39980145302168) bank104360 +104361 POINT(40.562019733522106 73.7269912673474) bank104361 +104362 POINT(39.904425582610195 74.23275310576858) bank104362 +104363 POINT(40.77394073453548 74.82597629989166) bank104363 +104364 POINT(39.795781902181844 74.6904530619356) bank104364 +104365 POINT(40.760297947668874 73.0397855251252) bank104365 +104366 POINT(41.200690889653615 74.84910765885945) bank104366 +104367 POINT(40.89512555493281 73.7816611722413) bank104367 +104368 POINT(39.77795861469188 73.94359778027871) bank104368 +104369 POINT(40.867411494521555 74.29553701064839) bank104369 +104370 POINT(40.42008009717237 74.84707433893458) bank104370 +104371 POINT(39.81715479742545 74.08389149937152) bank104371 +104372 POINT(40.15666305927496 73.70959312638414) bank104372 +104373 POINT(39.775597168166506 73.94613116286692) bank104373 +104374 POINT(41.65135747751403 73.12143849272087) bank104374 +104375 POINT(41.3850528317953 73.93482336100882) bank104375 +104376 POINT(39.94984574314365 74.21156778557999) bank104376 +104377 POINT(41.41833271979786 73.12338085451228) bank104377 +104378 POINT(39.72633688485734 74.51353978848627) bank104378 +104379 POINT(40.65524622795253 73.05260749288291) bank104379 +104380 POINT(39.98419538044452 74.05083307220852) bank104380 +104381 POINT(40.20543004784044 73.67396917351734) bank104381 +104382 POINT(40.8541409238247 73.37592925197292) bank104382 +104383 POINT(40.34749806377891 74.89147132198686) bank104383 +104384 POINT(39.98949306275868 74.99132332908934) bank104384 +104385 POINT(41.347179752228 74.42791044502079) bank104385 +104386 POINT(40.63441635300373 73.60277811912826) bank104386 +104387 POINT(41.1590146301842 74.77365243621333) bank104387 +104388 POINT(40.90978303592389 74.20998515093855) bank104388 +104389 POINT(40.75980915826104 73.03693032027063) bank104389 +104390 POINT(41.18907634073829 74.87236284030828) bank104390 +104391 POINT(41.33600743691046 73.69350942577955) bank104391 +104392 POINT(40.43844288349983 73.47478779713646) bank104392 +104393 POINT(40.33545116241275 74.66594512635866) bank104393 +104394 POINT(41.30993392353183 73.41521838188352) bank104394 +104395 POINT(40.9509086633648 73.46710671094624) bank104395 +104396 POINT(39.81237215782155 74.9013441672553) bank104396 +104397 POINT(41.07324159393802 73.09614739423345) bank104397 +104398 POINT(41.26148352701054 73.21239816680782) bank104398 +104399 POINT(41.0748035249339 73.08049040366136) bank104399 +104400 POINT(41.389192016967 74.79230759523671) bank104400 +104401 POINT(41.350951110070284 74.39219667688985) bank104401 +104402 POINT(41.41363777144068 73.85513888446158) bank104402 +104403 POINT(40.6991434018006 73.60674833215724) bank104403 +104404 POINT(40.03607714665457 73.52438066308342) bank104404 +104405 POINT(41.63450863407853 73.63644757235124) bank104405 +104406 POINT(39.94083913619315 73.57594479164078) bank104406 +104407 POINT(41.33600206791717 73.1287641216243) bank104407 +104408 POINT(40.41382847978833 74.99209027767552) bank104408 +104409 POINT(40.412221451772254 73.89636923225603) bank104409 +104410 POINT(40.713687990637986 73.63135379148726) bank104410 +104411 POINT(41.57566447563074 73.14091121135822) bank104411 +104412 POINT(39.98598258771344 74.92822777016443) bank104412 +104413 POINT(40.83895443020748 73.21674281054166) bank104413 +104414 POINT(40.51498236652925 73.36101418586499) bank104414 +104415 POINT(41.537753876603304 74.58013776166375) bank104415 +104416 POINT(40.067180370968174 73.17203238607844) bank104416 +104417 POINT(40.488780597403554 74.02258827072481) bank104417 +104418 POINT(40.26900533681687 73.24236573978071) bank104418 +104419 POINT(40.8200725864819 74.96567964571926) bank104419 +104420 POINT(39.72046056263475 73.10086321408654) bank104420 +104421 POINT(40.28505363934875 73.33449204443643) bank104421 +104422 POINT(40.48422375797011 73.03222174264855) bank104422 +104423 POINT(41.689284280490924 73.63822478791451) bank104423 +104424 POINT(40.74470582656712 73.27371474794833) bank104424 +104425 POINT(40.75684937639706 74.96862098173645) bank104425 +104426 POINT(40.875636238537815 74.89491660205933) bank104426 +104427 POINT(40.398728310435814 73.27164230748275) bank104427 +104428 POINT(39.9472173216751 73.95616378487509) bank104428 +104429 POINT(41.052017476224485 73.43685170653077) bank104429 +104430 POINT(41.51862416688621 73.27504817265455) bank104430 +104431 POINT(40.53891736709064 73.05680329515654) bank104431 +104432 POINT(41.68019565150136 74.45867442230649) bank104432 +104433 POINT(40.83276569438954 73.14506346502107) bank104433 +104434 POINT(41.696067021178656 73.84217135742854) bank104434 +104435 POINT(40.41288027871194 73.9684561182759) bank104435 +104436 POINT(41.54548824505781 74.67523619446914) bank104436 +104437 POINT(39.89891555019303 74.20739782247456) bank104437 +104438 POINT(39.720677661821725 73.45554128224074) bank104438 +104439 POINT(40.35064681616332 74.81626295956706) bank104439 +104440 POINT(41.68980252415373 73.8380600086719) bank104440 +104441 POINT(41.15808468492683 73.9369389252451) bank104441 +104442 POINT(40.06195759216319 73.60032378131041) bank104442 +104443 POINT(39.75187296628055 74.38397810327494) bank104443 +104444 POINT(40.9965093474745 74.71252585355305) bank104444 +104445 POINT(40.40155877625769 74.50035522937742) bank104445 +104446 POINT(41.68711159698832 73.86470357912427) bank104446 +104447 POINT(40.542346044878364 73.62459240638478) bank104447 +104448 POINT(40.96554888861112 74.0958545716138) bank104448 +104449 POINT(40.888352589830774 73.4922186747723) bank104449 +104450 POINT(40.69520335972581 74.68509625070065) bank104450 +104451 POINT(41.19265697478572 73.58381192846791) bank104451 +104452 POINT(41.05004599314533 73.34065745080386) bank104452 +104453 POINT(40.293372437698494 73.1241674744045) bank104453 +104454 POINT(40.41484158812488 73.8056340196267) bank104454 +104455 POINT(40.83494590060201 74.80609141942243) bank104455 +104456 POINT(40.17920308048977 73.07194733308383) bank104456 +104457 POINT(40.00095186157178 73.29181693363016) bank104457 +104458 POINT(39.75407472669708 74.41100471982546) bank104458 +104459 POINT(40.08833043603122 74.36330357319359) bank104459 +104460 POINT(39.96340998258645 74.31674884928192) bank104460 +104461 POINT(40.21542536554808 73.33724468596762) bank104461 +104462 POINT(40.35784636460856 74.39118092107489) bank104462 +104463 POINT(41.600881265366326 74.06661864152785) bank104463 +104464 POINT(39.77676096368409 73.73791099449636) bank104464 +104465 POINT(39.94922136989273 74.54709133828575) bank104465 +104466 POINT(40.891030772573686 74.53168738800797) bank104466 +104467 POINT(40.28739360920139 74.98238138615008) bank104467 +104468 POINT(40.354372084287135 74.96714499151823) bank104468 +104469 POINT(41.39961204539395 73.81729556620773) bank104469 +104470 POINT(40.01439956915959 73.4573502060715) bank104470 +104471 POINT(41.38965947287732 73.46018234142188) bank104471 +104472 POINT(41.275590929128875 73.4105982200929) bank104472 +104473 POINT(40.788968126110746 74.18113102650621) bank104473 +104474 POINT(41.655111016475246 74.185995219464) bank104474 +104475 POINT(40.335570629722866 74.56963286767085) bank104475 +104476 POINT(40.52640163542862 73.06423668334622) bank104476 +104477 POINT(40.248000126095505 74.43007417654368) bank104477 +104478 POINT(41.61397452655353 73.08377393869607) bank104478 +104479 POINT(41.17843181053281 73.18502358814281) bank104479 +104480 POINT(41.26692418805318 73.08936270127238) bank104480 +104481 POINT(41.239521056124055 74.03863986446413) bank104481 +104482 POINT(39.72757436699671 73.9157362224802) bank104482 +104483 POINT(41.70283537186153 74.53236686064902) bank104483 +104484 POINT(39.93263827285967 73.76057870779303) bank104484 +104485 POINT(39.805380104360424 73.5923336657719) bank104485 +104486 POINT(41.41218549711073 74.1140962463398) bank104486 +104487 POINT(41.330039382269376 73.6860053056648) bank104487 +104488 POINT(40.4058665868921 73.47625891803011) bank104488 +104489 POINT(41.05823659956296 73.59492692264533) bank104489 +104490 POINT(41.49164144939785 74.21157962638277) bank104490 +104491 POINT(40.370196255935696 74.41377711139845) bank104491 +104492 POINT(40.21410616791268 74.51160684179358) bank104492 +104493 POINT(41.33956892781444 74.81286416248912) bank104493 +104494 POINT(39.7472130353832 73.76572687929551) bank104494 +104495 POINT(40.20673870218973 74.01146007631799) bank104495 +104496 POINT(40.72344187750503 74.83286710459898) bank104496 +104497 POINT(40.54267137505485 73.24418330724852) bank104497 +104498 POINT(41.03177446622158 74.62497330900848) bank104498 +104499 POINT(40.0507308428974 73.13548984126408) bank104499 +104500 POINT(40.944315631979954 73.11987379784972) bank104500 +104501 POINT(40.1692079007041 74.11432473376227) bank104501 +104502 POINT(40.65863397252349 73.4138828038083) bank104502 +104503 POINT(39.84014290706828 73.92202572815997) bank104503 +104504 POINT(41.23672046680412 74.3881009424098) bank104504 +104505 POINT(40.044585998552954 73.48519324121747) bank104505 +104506 POINT(39.79265275366866 73.895703051365) bank104506 +104507 POINT(41.123664243262894 74.4913505328775) bank104507 +104508 POINT(41.31798891543122 74.25096757982226) bank104508 +104509 POINT(40.95769685385299 74.23331888032438) bank104509 +104510 POINT(40.808170926925044 74.63020824909297) bank104510 +104511 POINT(41.48651354893436 73.68485616514116) bank104511 +104512 POINT(40.403229088428624 74.3143071490575) bank104512 +104513 POINT(40.02500725944338 73.42529535415107) bank104513 +104514 POINT(40.72746460833833 74.78046730475367) bank104514 +104515 POINT(41.060388277775516 74.72779979626519) bank104515 +104516 POINT(41.61994943834811 73.07428553721567) bank104516 +104517 POINT(39.87353710053135 74.06715416425358) bank104517 +104518 POINT(41.299064294693004 74.25143390641331) bank104518 +104519 POINT(41.14687287748315 73.21991599585492) bank104519 +104520 POINT(40.29566591811266 74.39167319305182) bank104520 +104521 POINT(40.918672597272 74.59315809051374) bank104521 +104522 POINT(40.50345627823084 73.70990715685713) bank104522 +104523 POINT(40.30561104286749 74.10504336415231) bank104523 +104524 POINT(41.61810015971434 74.93768227275902) bank104524 +104525 POINT(40.76594901061768 74.29006377931246) bank104525 +104526 POINT(41.608377601304795 74.08619446445574) bank104526 +104527 POINT(40.36252065508111 74.01546657695327) bank104527 +104528 POINT(39.986680067353255 73.55371173965777) bank104528 +104529 POINT(41.508585983472216 73.45610641829518) bank104529 +104530 POINT(39.85114777907103 73.08683984028136) bank104530 +104531 POINT(41.59327769506647 74.53359468029758) bank104531 +104532 POINT(40.385987960084904 73.9131917793795) bank104532 +104533 POINT(40.03495108988333 73.21403930261788) bank104533 +104534 POINT(40.67897976910063 74.6417168823861) bank104534 +104535 POINT(41.28952265535123 74.25762529297475) bank104535 +104536 POINT(40.5274192589906 74.38435925344227) bank104536 +104537 POINT(40.48446006302349 74.74975205123116) bank104537 +104538 POINT(41.164817754757955 74.84210154901437) bank104538 +104539 POINT(40.579427456323415 73.69275031356963) bank104539 +104540 POINT(40.40838934652871 74.83102565929858) bank104540 +104541 POINT(41.30290472295479 73.92913700581238) bank104541 +104542 POINT(40.23631432519922 74.18389195930803) bank104542 +104543 POINT(40.98268207211785 73.96488664032843) bank104543 +104544 POINT(39.71394210144854 74.08519927788622) bank104544 +104545 POINT(41.263574514260114 73.72321434947177) bank104545 +104546 POINT(41.359603658753876 74.62382530970842) bank104546 +104547 POINT(40.1179192587126 73.62472647061152) bank104547 +104548 POINT(39.93732862374405 73.22661160254206) bank104548 +104549 POINT(40.705577121004865 73.41249792024784) bank104549 +104550 POINT(41.53305245896042 73.92053317264295) bank104550 +104551 POINT(40.188797615110566 73.59042599994196) bank104551 +104552 POINT(41.286569296980176 74.37762487695213) bank104552 +104553 POINT(41.00329161507426 73.71354496395823) bank104553 +104554 POINT(40.385135538017266 73.55450240540392) bank104554 +104555 POINT(40.65183445041365 73.95374246830951) bank104555 +104556 POINT(40.7310299316306 73.3966014731732) bank104556 +104557 POINT(40.30325940356422 73.3108076219064) bank104557 +104558 POINT(41.28889430328009 74.30923782730329) bank104558 +104559 POINT(39.96824398623117 74.92924770612737) bank104559 +104560 POINT(40.92410478408723 74.16830346616251) bank104560 +104561 POINT(40.84544100758497 74.51430072984436) bank104561 +104562 POINT(40.01457329827045 74.02845944689165) bank104562 +104563 POINT(40.07735014680883 73.82024369088342) bank104563 +104564 POINT(39.839728218791926 74.77367392217118) bank104564 +104565 POINT(40.671127543913386 73.80076469703062) bank104565 +104566 POINT(41.331132750576785 74.77894478205226) bank104566 +104567 POINT(41.415754286353575 74.41143496677688) bank104567 +104568 POINT(40.5223630966661 74.1744280120226) bank104568 +104569 POINT(40.10708842732116 73.56702604174376) bank104569 +104570 POINT(41.16211911691099 74.1328067118423) bank104570 +104571 POINT(40.41094297292966 73.95880829924756) bank104571 +104572 POINT(39.99904710824856 73.50581274103637) bank104572 +104573 POINT(40.2784605475441 73.88595612113228) bank104573 +104574 POINT(41.18920903043553 73.4514840065082) bank104574 +104575 POINT(40.764756065540894 73.1389689332723) bank104575 +104576 POINT(41.35753003425379 73.87761844305109) bank104576 +104577 POINT(40.49371265754717 74.29827623281565) bank104577 +104578 POINT(40.89092533240072 73.81715841480674) bank104578 +104579 POINT(40.61231607337128 73.37386327024362) bank104579 +104580 POINT(41.4258091375688 74.59773126409969) bank104580 +104581 POINT(40.63534430547176 73.04615047935614) bank104581 +104582 POINT(40.41264365019425 73.40909620868938) bank104582 +104583 POINT(40.160083604566594 73.76619910841286) bank104583 +104584 POINT(41.04415039152581 74.95063506190145) bank104584 +104585 POINT(40.537808219601395 73.32238819652586) bank104585 +104586 POINT(40.95268594602007 73.53586908690042) bank104586 +104587 POINT(40.005699100891846 74.59653668276022) bank104587 +104588 POINT(40.079061542682346 73.40753110636639) bank104588 +104589 POINT(39.881816302591105 74.6157925508002) bank104589 +104590 POINT(41.162795539516836 74.19549005661004) bank104590 +104591 POINT(40.2442007776764 73.12174729072376) bank104591 +104592 POINT(40.662868053984994 73.88427245591153) bank104592 +104593 POINT(41.53978751431921 74.77599294822762) bank104593 +104594 POINT(40.362148880929304 74.46319917650172) bank104594 +104595 POINT(40.949419842343026 73.81026977288056) bank104595 +104596 POINT(39.938940297392364 74.04084500859447) bank104596 +104597 POINT(41.233424974851246 73.69539063775282) bank104597 +104598 POINT(40.190858587805934 73.20550739262124) bank104598 +104599 POINT(40.1533597234645 73.02272510957935) bank104599 +104600 POINT(40.53238942404269 73.87392470450207) bank104600 +104601 POINT(41.505988304624445 74.3399023008977) bank104601 +104602 POINT(41.66610011811012 73.99599004228445) bank104602 +104603 POINT(41.02359589253705 74.61973704165167) bank104603 +104604 POINT(39.99011044108594 73.61015111491994) bank104604 +104605 POINT(41.316124794341015 73.020959977629) bank104605 +104606 POINT(41.030905887949515 74.83940629790888) bank104606 +104607 POINT(39.83870793173514 74.84067449739348) bank104607 +104608 POINT(41.46938991364377 74.27946958310584) bank104608 +104609 POINT(40.61948645887175 73.15662772489644) bank104609 +104610 POINT(40.10048595999082 74.67113231802462) bank104610 +104611 POINT(41.344665563639616 74.85670703430026) bank104611 +104612 POINT(39.87353703410012 73.66298926466365) bank104612 +104613 POINT(39.784026443222565 73.40609970371061) bank104613 +104614 POINT(41.637809266018415 74.1676771221355) bank104614 +104615 POINT(40.81670934120405 73.82357899435715) bank104615 +104616 POINT(41.20084555755311 73.36339461041212) bank104616 +104617 POINT(40.418215081750255 73.56432532541066) bank104617 +104618 POINT(41.38714433789308 74.04001900996103) bank104618 +104619 POINT(40.3916195290441 74.29705087505586) bank104619 +104620 POINT(40.918419514727034 74.29161845057791) bank104620 +104621 POINT(41.31402545879608 73.4199102367276) bank104621 +104622 POINT(41.378863153757614 73.31156043822472) bank104622 +104623 POINT(41.5767111897779 74.81687305306117) bank104623 +104624 POINT(40.642763180786716 74.85594830463546) bank104624 +104625 POINT(39.928344220247965 74.96442697230343) bank104625 +104626 POINT(40.113358059275214 74.54290851250063) bank104626 +104627 POINT(40.16813398686234 74.05073393804291) bank104627 +104628 POINT(40.33077471007985 73.61332950400501) bank104628 +104629 POINT(40.3404751626254 74.7425798353396) bank104629 +104630 POINT(40.582725768815294 74.40112552942891) bank104630 +104631 POINT(41.31215761902519 73.84830570627211) bank104631 +104632 POINT(39.742517616717166 74.67281780361054) bank104632 +104633 POINT(39.85805172288255 74.2726681845086) bank104633 +104634 POINT(40.47025603426249 74.42679501457415) bank104634 +104635 POINT(41.68967177857264 74.08599841600966) bank104635 +104636 POINT(41.10371340336139 74.01479559012355) bank104636 +104637 POINT(40.72964172850856 73.30703806311395) bank104637 +104638 POINT(41.63993080987807 73.20548854808848) bank104638 +104639 POINT(40.050430780624765 73.19781165514637) bank104639 +104640 POINT(41.04603019662566 73.77370274204463) bank104640 +104641 POINT(40.362633167912406 74.47791682831533) bank104641 +104642 POINT(40.906464990808594 73.33041625938947) bank104642 +104643 POINT(41.223736462281984 73.8712906442803) bank104643 +104644 POINT(40.55795217157034 74.24776483228678) bank104644 +104645 POINT(41.69389740243623 74.68306063788084) bank104645 +104646 POINT(40.777636385831165 74.57315795932922) bank104646 +104647 POINT(41.36717776032642 74.16745467291668) bank104647 +104648 POINT(41.11931307479851 73.49383487982794) bank104648 +104649 POINT(40.33714136596761 73.36765774250243) bank104649 +104650 POINT(40.700435675023456 74.62155564274983) bank104650 +104651 POINT(41.01384161260529 73.35674623038231) bank104651 +104652 POINT(40.39731635671592 74.4595172953131) bank104652 +104653 POINT(40.35061197661976 73.87119302392355) bank104653 +104654 POINT(41.531607713273395 73.23954086868586) bank104654 +104655 POINT(41.164705561595675 73.12214755896777) bank104655 +104656 POINT(41.69440414740683 74.06717056579173) bank104656 +104657 POINT(40.99000037362629 74.00027933416735) bank104657 +104658 POINT(39.91062034761396 73.61412957205992) bank104658 +104659 POINT(41.078768494917234 73.63171190576243) bank104659 +104660 POINT(40.22467183548061 73.49301494167882) bank104660 +104661 POINT(40.7674103033231 73.16967235705536) bank104661 +104662 POINT(40.987091272362186 74.45489966100934) bank104662 +104663 POINT(41.10252060789343 73.92179128050311) bank104663 +104664 POINT(40.785191135442616 74.60929651224681) bank104664 +104665 POINT(41.42309537571622 74.26322701412606) bank104665 +104666 POINT(40.76962518130004 74.70384457414646) bank104666 +104667 POINT(39.73683354303468 74.34260834751933) bank104667 +104668 POINT(41.62196659078431 73.4340271682225) bank104668 +104669 POINT(40.413195225916716 73.37421934103455) bank104669 +104670 POINT(40.19637739372466 73.40952637229226) bank104670 +104671 POINT(40.75625102842901 73.94451467433417) bank104671 +104672 POINT(40.57950911326366 74.60733581230195) bank104672 +104673 POINT(40.77690682023041 73.09312615762532) bank104673 +104674 POINT(41.54827303325109 74.16494256642994) bank104674 +104675 POINT(40.135228659159104 73.43902328547757) bank104675 +104676 POINT(41.01911839120785 73.58127526760198) bank104676 +104677 POINT(40.80650495638508 73.81389862103278) bank104677 +104678 POINT(40.712196467260405 73.6690399688646) bank104678 +104679 POINT(40.84080773479081 73.03526396518821) bank104679 +104680 POINT(40.90857951014872 73.77277007831867) bank104680 +104681 POINT(40.4968303445714 74.52842865768432) bank104681 +104682 POINT(40.666129863243384 74.21501455695618) bank104682 +104683 POINT(39.86908611093439 73.64674034816989) bank104683 +104684 POINT(40.275072785701305 73.23316672270653) bank104684 +104685 POINT(40.302104589850465 74.95817823394877) bank104685 +104686 POINT(40.789811886112155 73.34496733458737) bank104686 +104687 POINT(41.03649949785342 74.49994473709604) bank104687 +104688 POINT(40.74695304228142 73.61985115558178) bank104688 +104689 POINT(41.604572016182146 74.57923400744738) bank104689 +104690 POINT(41.221933593915104 73.41649949645016) bank104690 +104691 POINT(40.11128309388078 73.64255795081897) bank104691 +104692 POINT(41.33631133465106 73.13382067170227) bank104692 +104693 POINT(41.40201766837817 74.16195744137165) bank104693 +104694 POINT(40.11603871583202 74.0077468531804) bank104694 +104695 POINT(41.5431392323002 73.30337127209839) bank104695 +104696 POINT(41.69415735366289 73.7960102699049) bank104696 +104697 POINT(40.95907750292278 74.148755275411) bank104697 +104698 POINT(41.55307903648023 74.92970220780565) bank104698 +104699 POINT(39.947492300437645 74.75623569839598) bank104699 +104700 POINT(41.46435602087607 74.15225919753463) bank104700 +104701 POINT(41.55726321862018 73.72916913384401) bank104701 +104702 POINT(39.73976849501602 73.88521189782176) bank104702 +104703 POINT(40.55856529589311 74.09583812106962) bank104703 +104704 POINT(40.56297525317116 73.36686224197824) bank104704 +104705 POINT(39.81709468287122 74.2292080423865) bank104705 +104706 POINT(39.78156158461818 73.7077463505236) bank104706 +104707 POINT(40.25165472037348 74.30059462009856) bank104707 +104708 POINT(41.6599107010649 74.43798087409431) bank104708 +104709 POINT(41.24111430014865 74.9776368520473) bank104709 +104710 POINT(40.57166964395375 74.77621868199202) bank104710 +104711 POINT(39.78681478741201 74.58177433238586) bank104711 +104712 POINT(40.3354985802992 73.6727772163121) bank104712 +104713 POINT(41.17879043528442 74.69741919455083) bank104713 +104714 POINT(39.85905046114324 73.498438733508) bank104714 +104715 POINT(41.07470909837509 74.61033113313785) bank104715 +104716 POINT(41.393840450053396 73.01359751866087) bank104716 +104717 POINT(41.21005650903452 73.99864201714637) bank104717 +104718 POINT(39.997191739615666 73.73038129729555) bank104718 +104719 POINT(41.2403769845886 73.2847029953273) bank104719 +104720 POINT(40.024564690492845 74.02832559397204) bank104720 +104721 POINT(40.98014580344636 74.32046757407596) bank104721 +104722 POINT(41.71252177403085 75.00519047352847) bank104722 +104723 POINT(41.66679726132953 73.0367110949287) bank104723 +104724 POINT(39.84363010415355 73.99912499112261) bank104724 +104725 POINT(40.20817509347176 74.50481796667287) bank104725 +104726 POINT(39.88499494059491 74.18147704187948) bank104726 +104727 POINT(41.234874040063595 73.48883443065051) bank104727 +104728 POINT(41.30019270671916 74.56006150274727) bank104728 +104729 POINT(40.17026853265166 74.57102281305767) bank104729 +104730 POINT(40.658862948054534 73.02641834700664) bank104730 +104731 POINT(41.501333995455816 74.03710230780747) bank104731 +104732 POINT(39.86479370166219 74.23999448433094) bank104732 +104733 POINT(41.58110697099398 73.50789461491577) bank104733 +104734 POINT(40.74842816610048 73.0185439200783) bank104734 +104735 POINT(41.37053986184085 73.35447014596888) bank104735 +104736 POINT(40.632031315566195 73.56696717130116) bank104736 +104737 POINT(40.78652222718547 74.28376621131164) bank104737 +104738 POINT(41.47759583151317 73.90844027063659) bank104738 +104739 POINT(41.49730971014827 73.05404200945847) bank104739 +104740 POINT(39.95276588185323 74.39370563873831) bank104740 +104741 POINT(40.62296016269089 74.70941391276185) bank104741 +104742 POINT(40.17982353623172 73.59172199771349) bank104742 +104743 POINT(40.54312978295796 74.0406954021039) bank104743 +104744 POINT(41.15498198366576 74.3317764579319) bank104744 +104745 POINT(40.62729206643615 73.26667221968921) bank104745 +104746 POINT(40.102610713767135 73.31143122484085) bank104746 +104747 POINT(40.896343193100364 73.71460252377437) bank104747 +104748 POINT(40.80470977903802 73.424523787045) bank104748 +104749 POINT(39.997346072385255 74.71258016721357) bank104749 +104750 POINT(40.03412523540591 74.9716213866093) bank104750 +104751 POINT(41.245542955099786 73.44342488446674) bank104751 +104752 POINT(40.45880669241694 74.1097929362039) bank104752 +104753 POINT(41.00782779005894 74.0580734613415) bank104753 +104754 POINT(40.942041548547444 74.79153954014348) bank104754 +104755 POINT(39.821130329039796 73.43886974578001) bank104755 +104756 POINT(41.416086917978355 74.02991416987084) bank104756 +104757 POINT(40.55624055100503 73.34192855790594) bank104757 +104758 POINT(41.71031512347263 74.31978576738798) bank104758 +104759 POINT(40.10548068765783 74.54524144074483) bank104759 +104760 POINT(41.253852235703135 74.57132399008401) bank104760 +104761 POINT(39.9035714988531 73.49652191040055) bank104761 +104762 POINT(41.212831938385136 73.10067554992898) bank104762 +104763 POINT(40.55076456470303 74.82125701222586) bank104763 +104764 POINT(40.13611623293865 74.21885493448724) bank104764 +104765 POINT(40.50430407428441 74.5566400552696) bank104765 +104766 POINT(40.102460402731445 74.20992416554478) bank104766 +104767 POINT(41.56463796232549 74.89113768626237) bank104767 +104768 POINT(40.727415199042284 74.1510202384532) bank104768 +104769 POINT(39.79694569079572 73.23970568652848) bank104769 +104770 POINT(40.92159276358162 74.34439722240174) bank104770 +104771 POINT(40.383172327333476 73.63351932193179) bank104771 +104772 POINT(41.35102564263142 74.51680550383111) bank104772 +104773 POINT(40.1957115726584 74.62980551045158) bank104773 +104774 POINT(41.05683566936321 74.44831749414637) bank104774 +104775 POINT(41.62863461296838 73.54210092370948) bank104775 +104776 POINT(40.102874090041674 74.77385492955653) bank104776 +104777 POINT(41.321582968949855 74.92649573100395) bank104777 +104778 POINT(40.69207181893609 73.88190127875778) bank104778 +104779 POINT(41.662242488900915 74.63597018783616) bank104779 +104780 POINT(39.77752481304622 74.45655280826756) bank104780 +104781 POINT(40.99490805260793 74.5964593154416) bank104781 +104782 POINT(41.3932915176629 73.89815324281292) bank104782 +104783 POINT(41.177507951197185 74.50338440634647) bank104783 +104784 POINT(41.23894692521255 74.89977383288424) bank104784 +104785 POINT(40.71741571492663 73.87452828618129) bank104785 +104786 POINT(39.996445584591406 73.43868253820087) bank104786 +104787 POINT(40.89008524266145 73.2646811716517) bank104787 +104788 POINT(41.54218255634716 74.0684257693153) bank104788 +104789 POINT(40.59071506177117 73.13412743461885) bank104789 +104790 POINT(39.75268768002569 73.95718820851285) bank104790 +104791 POINT(40.827692537399514 73.36450691216841) bank104791 +104792 POINT(40.62754357599137 74.37078121308626) bank104792 +104793 POINT(39.8641374344482 73.96613380051387) bank104793 +104794 POINT(41.534385523603575 74.19516969359347) bank104794 +104795 POINT(41.298862229289995 73.16080182221083) bank104795 +104796 POINT(41.48275595398614 73.83489725529034) bank104796 +104797 POINT(41.28488540800692 74.89815562983384) bank104797 +104798 POINT(40.46302179969475 73.27061571932508) bank104798 +104799 POINT(41.1061708228045 74.39469724320574) bank104799 +104800 POINT(40.840758892201954 74.87013510397459) bank104800 +104801 POINT(41.32210461331138 73.0519018528804) bank104801 +104802 POINT(40.940980631534224 73.83298068361488) bank104802 +104803 POINT(39.887600485132 74.90980537121071) bank104803 +104804 POINT(41.176503639573404 74.28265695527097) bank104804 +104805 POINT(40.10376594166732 74.57080747447885) bank104805 +104806 POINT(40.032808244998705 74.68721400444578) bank104806 +104807 POINT(40.05083177103036 73.5476615748547) bank104807 +104808 POINT(39.82470685993913 74.0656320849265) bank104808 +104809 POINT(40.77107629790481 74.84807304713416) bank104809 +104810 POINT(40.76219555953645 73.74212741376864) bank104810 +104811 POINT(40.908908093965444 74.3433525911347) bank104811 +104812 POINT(40.11014541300699 73.39804751390896) bank104812 +104813 POINT(40.6783045514123 74.73899247604643) bank104813 +104814 POINT(40.04845571654858 73.10066735643036) bank104814 +104815 POINT(40.804108320673734 74.25328152298503) bank104815 +104816 POINT(40.47722399877994 73.53260871640656) bank104816 +104817 POINT(40.9750075022646 73.99776749974605) bank104817 +104818 POINT(39.97816117670249 74.94321448568715) bank104818 +104819 POINT(41.393120451819996 74.81359852519458) bank104819 +104820 POINT(41.67851004728819 74.78222762859954) bank104820 +104821 POINT(41.63550619699328 74.97932425788596) bank104821 +104822 POINT(40.49448829083759 74.88077333715906) bank104822 +104823 POINT(41.40682771335734 74.66008824282454) bank104823 +104824 POINT(40.052493758054496 74.55942523406225) bank104824 +104825 POINT(41.00842347914565 73.97465685447564) bank104825 +104826 POINT(40.48922226044952 73.53351404959247) bank104826 +104827 POINT(39.958242687425646 74.3329016860873) bank104827 +104828 POINT(39.86512994805218 73.55120902707779) bank104828 +104829 POINT(41.42240487954446 74.93104719904241) bank104829 +104830 POINT(40.67407265797174 74.49991895132175) bank104830 +104831 POINT(39.983300446764574 74.89514767388701) bank104831 +104832 POINT(41.455964196673975 74.58973922169481) bank104832 +104833 POINT(41.23765909046283 73.13682528809527) bank104833 +104834 POINT(39.948304571516104 74.840104422475) bank104834 +104835 POINT(40.52932954663982 73.03314394652327) bank104835 +104836 POINT(41.48455608977579 74.168168127731) bank104836 +104837 POINT(40.55866560221231 73.51188558711121) bank104837 +104838 POINT(40.57598321826781 73.20914011957923) bank104838 +104839 POINT(41.31685788706077 74.1254835536257) bank104839 +104840 POINT(40.60829182161426 74.23600420471554) bank104840 +104841 POINT(40.0078140125229 73.83319493990048) bank104841 +104842 POINT(40.57272181469842 73.67607033882628) bank104842 +104843 POINT(40.56427881151366 74.68713516741242) bank104843 +104844 POINT(41.434351499266384 74.53412150928605) bank104844 +104845 POINT(41.23094906191429 73.66445908247982) bank104845 +104846 POINT(39.92436962277905 73.78690985354544) bank104846 +104847 POINT(41.11646944461501 74.86932594247874) bank104847 +104848 POINT(41.6272401795475 74.8671800130847) bank104848 +104849 POINT(39.92831313235854 73.43854811749377) bank104849 +104850 POINT(39.883465652245214 74.8117352021626) bank104850 +104851 POINT(40.907991741809276 73.91061563925878) bank104851 +104852 POINT(40.28246824813947 73.83506399218585) bank104852 +104853 POINT(40.432890568451796 74.91066966615789) bank104853 +104854 POINT(40.36519376333624 73.07675217058119) bank104854 +104855 POINT(40.57658383556196 73.11353518045307) bank104855 +104856 POINT(40.65374762583219 74.83167243237871) bank104856 +104857 POINT(41.61590074042443 73.2303752472031) bank104857 +104858 POINT(41.30341259775479 74.60953259997646) bank104858 +104859 POINT(41.09845989723761 74.23143877046765) bank104859 +104860 POINT(40.10801354296145 74.17991539561672) bank104860 +104861 POINT(40.42324853844978 74.17709446865199) bank104861 +104862 POINT(40.43703365405178 73.35297837640793) bank104862 +104863 POINT(40.101810726054 73.82843291643205) bank104863 +104864 POINT(39.87182040595237 74.89870966303465) bank104864 +104865 POINT(40.22502708181587 73.72649444093106) bank104865 +104866 POINT(40.178031022678994 74.99668347121228) bank104866 +104867 POINT(41.245993610233306 73.0236681773476) bank104867 +104868 POINT(41.1972996067585 74.00005716929196) bank104868 +104869 POINT(41.37092818150227 73.92178317364159) bank104869 +104870 POINT(41.15849772269157 74.7105285681699) bank104870 +104871 POINT(41.12271837231288 74.25852691163084) bank104871 +104872 POINT(41.430679083041156 74.23633108639571) bank104872 +104873 POINT(41.408949103197976 73.60586622625702) bank104873 +104874 POINT(40.43043640872863 74.39209915691283) bank104874 +104875 POINT(40.52717627625815 74.15778848743989) bank104875 +104876 POINT(41.641120903637834 73.15157616733713) bank104876 +104877 POINT(40.39235920980312 73.19616604956366) bank104877 +104878 POINT(40.57905138101551 74.83808104367546) bank104878 +104879 POINT(40.58998184857772 74.80864737247977) bank104879 +104880 POINT(40.313374334950126 73.65844116511649) bank104880 +104881 POINT(40.11162641130236 74.5278577344824) bank104881 +104882 POINT(41.53709086930223 73.24369084599543) bank104882 +104883 POINT(40.024420961548245 73.41692404348495) bank104883 +104884 POINT(40.989753438295985 73.07199070423374) bank104884 +104885 POINT(41.1211847901597 74.93983256395066) bank104885 +104886 POINT(40.14614429438462 73.26401095093857) bank104886 +104887 POINT(41.55130134642295 74.93138971956624) bank104887 +104888 POINT(40.48210040178494 73.12985054769996) bank104888 +104889 POINT(40.90481953661883 74.9821013560829) bank104889 +104890 POINT(40.42101932858541 73.01684046903536) bank104890 +104891 POINT(41.31039620244869 74.24207748824193) bank104891 +104892 POINT(41.0774108457466 74.20794713222378) bank104892 +104893 POINT(39.91061107472399 73.92874262142206) bank104893 +104894 POINT(39.95267352625709 74.00066121960658) bank104894 +104895 POINT(40.457040352331276 73.74906685052683) bank104895 +104896 POINT(40.84511079165231 73.6756800182942) bank104896 +104897 POINT(40.363218325890266 74.88467704739944) bank104897 +104898 POINT(41.46240691715093 74.00637744708321) bank104898 +104899 POINT(39.97431488034932 74.68939285185617) bank104899 +104900 POINT(40.34516230067281 73.09820822305386) bank104900 +104901 POINT(40.60731950994357 74.39871311461638) bank104901 +104902 POINT(40.82347206345979 74.6148737712943) bank104902 +104903 POINT(41.3210505266628 74.45497633580578) bank104903 +104904 POINT(40.18404702955707 73.0400380224515) bank104904 +104905 POINT(40.4040781190132 73.37548259762816) bank104905 +104906 POINT(41.226413453712716 73.49459149632628) bank104906 +104907 POINT(40.18896444771882 73.76924040297699) bank104907 +104908 POINT(41.22845098472625 74.29754599065127) bank104908 +104909 POINT(40.63459567763315 74.79894273043142) bank104909 +104910 POINT(40.07181642545028 74.8162724044379) bank104910 +104911 POINT(40.44917666180118 74.4800204516979) bank104911 +104912 POINT(41.47160664519823 74.32296062661901) bank104912 +104913 POINT(39.909254013493054 73.78910084240053) bank104913 +104914 POINT(41.33104923828518 73.57552878393946) bank104914 +104915 POINT(41.50617368874287 73.09434847137813) bank104915 +104916 POINT(40.93600033696133 73.18342116802941) bank104916 +104917 POINT(39.973550825868635 74.34888880538132) bank104917 +104918 POINT(41.393445255006284 73.71091258753157) bank104918 +104919 POINT(40.46140455448621 73.9602198302566) bank104919 +104920 POINT(40.18901546394152 73.36028283280382) bank104920 +104921 POINT(39.93571077517094 74.98065488787351) bank104921 +104922 POINT(40.185172468818664 74.19030890550138) bank104922 +104923 POINT(41.09288698494493 73.48299662539154) bank104923 +104924 POINT(41.64354339382393 73.50474219982716) bank104924 +104925 POINT(41.37522572064667 74.12680333557229) bank104925 +104926 POINT(41.21476441012151 73.73049869164362) bank104926 +104927 POINT(41.63271910606162 74.4943490104459) bank104927 +104928 POINT(41.12158764625458 73.09906599943137) bank104928 +104929 POINT(40.69622432171463 74.11265878693118) bank104929 +104930 POINT(40.3567824731907 74.35796660032435) bank104930 +104931 POINT(40.817606808790394 74.67959442476882) bank104931 +104932 POINT(40.07669258755104 74.36054146687022) bank104932 +104933 POINT(40.29052922569725 74.3767888363093) bank104933 +104934 POINT(40.88601033134052 73.32041908900482) bank104934 +104935 POINT(41.292108014165926 74.44230901582961) bank104935 +104936 POINT(39.862815207690325 74.35055943423075) bank104936 +104937 POINT(41.4481064587143 74.38941452827511) bank104937 +104938 POINT(41.461922409822286 74.38590351516247) bank104938 +104939 POINT(40.66102183274077 73.77955109008921) bank104939 +104940 POINT(40.8671428819051 74.5544309987314) bank104940 +104941 POINT(40.67657166046125 74.88240498553878) bank104941 +104942 POINT(41.01313200519812 74.76031548283356) bank104942 +104943 POINT(41.11961006045939 74.13251495510066) bank104943 +104944 POINT(41.20850336535289 74.63269675602002) bank104944 +104945 POINT(40.26513640718923 74.79150044731193) bank104945 +104946 POINT(41.26525383844322 73.16115723444419) bank104946 +104947 POINT(40.63624661551915 73.38535093995453) bank104947 +104948 POINT(40.52393574781644 73.30944139631622) bank104948 +104949 POINT(41.162912595164585 74.46634751820095) bank104949 +104950 POINT(41.016794864756726 73.8650332288215) bank104950 +104951 POINT(39.95569282407652 73.65822366555699) bank104951 +104952 POINT(39.7761532530591 73.98047497091488) bank104952 +104953 POINT(40.76204892648577 74.17467201555205) bank104953 +104954 POINT(40.370557551822735 73.5122214340124) bank104954 +104955 POINT(40.690084148231946 73.59585295982166) bank104955 +104956 POINT(41.33074065206937 73.19464245366969) bank104956 +104957 POINT(41.67032178304812 74.31922376644246) bank104957 +104958 POINT(41.043145391929954 73.50029945830819) bank104958 +104959 POINT(39.739336109032216 73.01183003233997) bank104959 +104960 POINT(40.237339640014 74.40069553866007) bank104960 +104961 POINT(40.09995953697505 74.810522180358) bank104961 +104962 POINT(41.21402837841314 74.1640630662573) bank104962 +104963 POINT(41.52178534544422 73.72185899054416) bank104963 +104964 POINT(41.690413246709426 74.21664547082374) bank104964 +104965 POINT(40.66303483054886 74.82891390547016) bank104965 +104966 POINT(40.16254451769781 74.80215975716028) bank104966 +104967 POINT(40.41291626330846 74.68813067320941) bank104967 +104968 POINT(40.031304129570145 73.09691869262647) bank104968 +104969 POINT(41.33092233675528 73.75702780219706) bank104969 +104970 POINT(41.659194483539885 73.7922985327705) bank104970 +104971 POINT(41.14544529756945 73.34277836007232) bank104971 +104972 POINT(40.83899428736163 74.10472445785153) bank104972 +104973 POINT(41.40682009774712 73.7768501420539) bank104973 +104974 POINT(40.30855171938856 74.80194799178561) bank104974 +104975 POINT(41.33317837165756 74.75826927129211) bank104975 +104976 POINT(41.67617237976827 73.61337527758381) bank104976 +104977 POINT(40.44985376584784 73.74963077108755) bank104977 +104978 POINT(40.853711697999046 73.5664039002616) bank104978 +104979 POINT(40.74382833990749 74.10235061644798) bank104979 +104980 POINT(41.436297892640496 74.96998192848594) bank104980 +104981 POINT(40.163378279725116 74.73122668165679) bank104981 +104982 POINT(40.03828247448528 73.1948810866202) bank104982 +104983 POINT(41.02685011536583 74.62767732340599) bank104983 +104984 POINT(39.821471801084506 74.35192560900556) bank104984 +104985 POINT(39.83431351523401 73.50403751014842) bank104985 +104986 POINT(41.62230550327826 73.92290898329574) bank104986 +104987 POINT(40.911769696797215 74.96340246447274) bank104987 +104988 POINT(41.59880311466665 74.39960528804608) bank104988 +104989 POINT(40.52530621780173 73.94699618517285) bank104989 +104990 POINT(41.53172430676512 74.20039775605386) bank104990 +104991 POINT(40.38288634272538 74.36758621334874) bank104991 +104992 POINT(41.39557960226385 74.19396676971616) bank104992 +104993 POINT(41.00554043137738 73.80955824019668) bank104993 +104994 POINT(41.24424455455719 74.67428469938072) bank104994 +104995 POINT(41.11822717339197 73.57159846483509) bank104995 +104996 POINT(40.31167960808554 73.71485303034463) bank104996 +104997 POINT(40.703806358505076 74.2822304767693) bank104997 +104998 POINT(41.239948546794174 73.45765854375179) bank104998 +104999 POINT(41.71127372391781 73.59982106187263) bank104999 +105000 POINT(40.67817586254137 74.0773879361294) bank105000 +105001 POINT(40.37770728693738 74.32655159357462) bank105001 +105002 POINT(41.066373663603784 73.22895352862452) bank105002 +105003 POINT(39.9139564615886 73.83234369604037) bank105003 +105004 POINT(39.80805859329824 73.16624697248002) bank105004 +105005 POINT(40.11917690130244 73.51900520896109) bank105005 +105006 POINT(41.651129584122536 74.85014203721548) bank105006 +105007 POINT(41.38116791526326 74.9504525991528) bank105007 +105008 POINT(40.847448628888394 74.50158678505335) bank105008 +105009 POINT(40.34804307185251 73.73573490450534) bank105009 +105010 POINT(40.97062785536212 73.63410240022114) bank105010 +105011 POINT(40.108444306720735 74.47992908205232) bank105011 +105012 POINT(40.505886953163 73.5415159222838) bank105012 +105013 POINT(41.36404584580363 73.28100408080617) bank105013 +105014 POINT(41.49518013103653 73.07043091154851) bank105014 +105015 POINT(41.58812197584454 74.72819540633365) bank105015 +105016 POINT(40.33156029589891 74.58791903350462) bank105016 +105017 POINT(41.12410837982386 73.70971545542643) bank105017 +105018 POINT(40.28143318947113 73.90077794568533) bank105018 +105019 POINT(40.8954504475222 74.1860375350953) bank105019 +105020 POINT(41.44320355983106 75.00294732763393) bank105020 +105021 POINT(40.446022059203614 73.30745127898504) bank105021 +105022 POINT(40.994172117239614 74.57853331676012) bank105022 +105023 POINT(40.498610229427086 74.26678802209841) bank105023 +105024 POINT(41.080421969796916 73.85849880345539) bank105024 +105025 POINT(39.81755248831576 74.06309915277825) bank105025 +105026 POINT(40.0469985898644 73.4762299353351) bank105026 +105027 POINT(41.043164582254626 74.85274252211366) bank105027 +105028 POINT(40.29097616063126 74.49451863520625) bank105028 +105029 POINT(40.12514307609437 73.38580964158844) bank105029 +105030 POINT(41.249710322870236 74.63806709121131) bank105030 +105031 POINT(41.08353513497547 74.3609968870776) bank105031 +105032 POINT(40.63116349131514 73.40716911559042) bank105032 +105033 POINT(40.057393218907954 73.91281820287783) bank105033 +105034 POINT(41.652591019108996 74.43970362442717) bank105034 +105035 POINT(41.01322113348381 74.14568049030976) bank105035 +105036 POINT(40.979845448458484 73.2241554376768) bank105036 +105037 POINT(40.55648522242868 73.40186541016399) bank105037 +105038 POINT(40.55845250816754 74.11130435089974) bank105038 +105039 POINT(40.83213318480636 73.26695761281678) bank105039 +105040 POINT(40.925322595809334 73.98627251230698) bank105040 +105041 POINT(40.96002456350164 73.08171903588082) bank105041 +105042 POINT(41.320340487597136 74.92633447183437) bank105042 +105043 POINT(40.052829245239266 74.72936671887197) bank105043 +105044 POINT(39.87648024104395 74.5690701049564) bank105044 +105045 POINT(41.34550756048137 73.48053490684445) bank105045 +105046 POINT(40.966242151228194 74.53570222429852) bank105046 +105047 POINT(41.401710774061954 74.79857506408803) bank105047 +105048 POINT(40.554156747297306 74.70747350282511) bank105048 +105049 POINT(41.330674304981976 73.44821740169317) bank105049 +105050 POINT(40.632675995492335 74.33999869160706) bank105050 +105051 POINT(39.797266904340525 73.17068215037928) bank105051 +105052 POINT(40.505360176296875 74.39821637661703) bank105052 +105053 POINT(40.41936303401002 74.30551810267114) bank105053 +105054 POINT(40.8940680482727 73.66510420762962) bank105054 +105055 POINT(41.66745486474156 73.34158438543406) bank105055 +105056 POINT(40.400165905379 74.60341547985297) bank105056 +105057 POINT(41.282808336619766 73.76629225042353) bank105057 +105058 POINT(39.87512821834458 74.6483905292583) bank105058 +105059 POINT(40.501931113860245 73.88573097657978) bank105059 +105060 POINT(40.16652764414716 74.1061603007871) bank105060 +105061 POINT(41.45476613278192 74.98908993228162) bank105061 +105062 POINT(41.397622671982546 74.40945453000708) bank105062 +105063 POINT(40.73534709324942 73.50620391887804) bank105063 +105064 POINT(40.81986880086753 74.7635343157364) bank105064 +105065 POINT(41.409314275010246 74.33770901132301) bank105065 +105066 POINT(39.745627605663586 73.1861306636159) bank105066 +105067 POINT(41.22191123040097 73.8953784708056) bank105067 +105068 POINT(39.87522210382954 73.5144030857761) bank105068 +105069 POINT(41.58072133949591 74.03591800370563) bank105069 +105070 POINT(40.10061471129191 74.65332599259948) bank105070 +105071 POINT(41.11227113878155 74.1524782282105) bank105071 +105072 POINT(41.011498305623775 74.69931188277343) bank105072 +105073 POINT(41.589851743821505 73.3811679389302) bank105073 +105074 POINT(41.01598941490297 74.62357129591153) bank105074 +105075 POINT(40.90474861899175 73.86476462002392) bank105075 +105076 POINT(41.21570207243461 74.34413745169866) bank105076 +105077 POINT(39.72208718818285 73.14887111634792) bank105077 +105078 POINT(40.54856308423481 74.04553424351046) bank105078 +105079 POINT(41.277548637240905 74.80508714075583) bank105079 +105080 POINT(40.26589800600744 73.24495850593338) bank105080 +105081 POINT(40.889423293241464 74.19204677601172) bank105081 +105082 POINT(39.79970137159756 73.88687257660062) bank105082 +105083 POINT(40.07204034743881 74.16176873998698) bank105083 +105084 POINT(40.52903043332724 74.8784257981414) bank105084 +105085 POINT(40.289830243327486 73.12340895792485) bank105085 +105086 POINT(40.74796859268461 73.8894555822437) bank105086 +105087 POINT(41.22746824663782 73.9949662114914) bank105087 +105088 POINT(41.485116424231904 73.56333683648434) bank105088 +105089 POINT(39.95517897611127 74.96255522856178) bank105089 +105090 POINT(40.76133679329431 73.53291142762612) bank105090 +105091 POINT(41.686908621385264 73.97755759639328) bank105091 +105092 POINT(41.45661326775415 73.45994366262322) bank105092 +105093 POINT(39.91179159370116 73.80199622512664) bank105093 +105094 POINT(40.33356961473039 73.15393019047929) bank105094 +105095 POINT(39.77654666015457 73.59401550814317) bank105095 +105096 POINT(41.25496483927145 74.92019382885111) bank105096 +105097 POINT(40.82035635105422 74.49229202466022) bank105097 +105098 POINT(41.578656078005054 73.50249344116105) bank105098 +105099 POINT(41.18174322424365 73.68149220842571) bank105099 +105100 POINT(41.6413520139508 73.64141591642431) bank105100 +105101 POINT(40.95139208342701 73.77696317641882) bank105101 +105102 POINT(41.050736914216024 73.37136505455875) bank105102 +105103 POINT(39.78237946954844 74.83231753245634) bank105103 +105104 POINT(41.37219486053089 74.57035611249843) bank105104 +105105 POINT(41.347993458135356 74.09227153030325) bank105105 +105106 POINT(40.144274186408005 73.47126954054832) bank105106 +105107 POINT(41.019353265422176 73.23945865866953) bank105107 +105108 POINT(40.46544666781208 73.2738143546149) bank105108 +105109 POINT(40.906903404264185 74.31372269541863) bank105109 +105110 POINT(40.17700340392954 73.73746589468026) bank105110 +105111 POINT(40.62329198704061 73.68430861925965) bank105111 +105112 POINT(40.32508398283164 73.17377517912226) bank105112 +105113 POINT(40.05788712282863 74.89983748954782) bank105113 +105114 POINT(41.3295455377914 74.04308281340246) bank105114 +105115 POINT(39.83649137007657 74.1646009370902) bank105115 +105116 POINT(39.74035370475379 73.77798781830971) bank105116 +105117 POINT(40.982330671904585 74.4983571249936) bank105117 +105118 POINT(41.33078906096569 73.36578764470323) bank105118 +105119 POINT(41.186016487140044 73.88316421417561) bank105119 +105120 POINT(40.47807926570395 74.0679027559786) bank105120 +105121 POINT(39.759033071990096 73.59591745407666) bank105121 +105122 POINT(40.862998658181745 74.82031633464538) bank105122 +105123 POINT(40.32104212403851 74.39820542064825) bank105123 +105124 POINT(40.64890509915698 74.69243722754102) bank105124 +105125 POINT(40.486868757705444 74.10210912160235) bank105125 +105126 POINT(40.24437153651894 73.74623784433493) bank105126 +105127 POINT(40.224225503600984 74.95076188660754) bank105127 +105128 POINT(40.10853574622154 73.11168025023015) bank105128 +105129 POINT(41.14800742618386 74.38746114817664) bank105129 +105130 POINT(40.02946083623341 74.10828360043263) bank105130 +105131 POINT(41.672079089694414 73.45291946195702) bank105131 +105132 POINT(41.188429242556595 74.81812963543453) bank105132 +105133 POINT(39.988174606978774 74.061314623891) bank105133 +105134 POINT(40.96761611797354 74.49418901090573) bank105134 +105135 POINT(41.090693391639874 73.39036796454491) bank105135 +105136 POINT(41.42208144613746 74.20866578694975) bank105136 +105137 POINT(41.04602787567012 74.2945366223622) bank105137 +105138 POINT(40.94108430835703 73.86745652191689) bank105138 +105139 POINT(40.4917716380187 74.4544176774531) bank105139 +105140 POINT(40.28468848573788 73.9712953689158) bank105140 +105141 POINT(41.622983099414476 73.09286956618418) bank105141 +105142 POINT(40.92745050651486 73.32648939638288) bank105142 +105143 POINT(40.29936699766939 74.21530419941195) bank105143 +105144 POINT(40.879019859018314 74.57894776884523) bank105144 +105145 POINT(41.13376645479941 73.6476520634773) bank105145 +105146 POINT(39.735304474616726 73.69768721137928) bank105146 +105147 POINT(41.04605731400211 74.86621297676447) bank105147 +105148 POINT(41.41484924252103 74.91035704097179) bank105148 +105149 POINT(41.578673144107924 74.94682628620775) bank105149 +105150 POINT(40.730964125221064 74.54610410076324) bank105150 +105151 POINT(41.60836398058226 74.32282340038138) bank105151 +105152 POINT(41.3042118515409 73.44597720535846) bank105152 +105153 POINT(39.89598624390944 73.00950653024007) bank105153 +105154 POINT(41.50101151457052 74.35535807843185) bank105154 +105155 POINT(40.40672141454584 73.55578699934865) bank105155 +105156 POINT(40.20398517775596 73.77437641622143) bank105156 +105157 POINT(40.4728388542965 73.22387285478773) bank105157 +105158 POINT(40.97130033803429 73.35720851360233) bank105158 +105159 POINT(41.26254752814682 73.10814125602695) bank105159 +105160 POINT(40.447440281600045 74.74853351906471) bank105160 +105161 POINT(40.19226327361604 73.09145883315969) bank105161 +105162 POINT(40.59897913898406 74.02668372396488) bank105162 +105163 POINT(41.60512446070714 74.65218403019931) bank105163 +105164 POINT(40.40219955038753 74.99728274292521) bank105164 +105165 POINT(41.03564960775386 73.05245354559783) bank105165 +105166 POINT(41.10048457297272 74.65233304946894) bank105166 +105167 POINT(41.44072533859489 73.22433241811413) bank105167 +105168 POINT(39.845838511263956 74.65716893658973) bank105168 +105169 POINT(41.36044750220897 74.72873044954297) bank105169 +105170 POINT(40.56824857886624 73.15406062896949) bank105170 +105171 POINT(40.433565639718154 73.40792940679569) bank105171 +105172 POINT(41.140073619976825 73.8591963938796) bank105172 +105173 POINT(39.76441664756939 73.79934425795321) bank105173 +105174 POINT(41.39037912348131 74.12717638429304) bank105174 +105175 POINT(41.449478075346036 74.80204277143326) bank105175 +105176 POINT(39.99750843541969 74.89011649394224) bank105176 +105177 POINT(39.877891092279214 73.02256727801186) bank105177 +105178 POINT(41.25515894424544 74.89035711934336) bank105178 +105179 POINT(40.32280682950146 74.70580689461715) bank105179 +105180 POINT(41.625593143207816 73.73126777209481) bank105180 +105181 POINT(40.92272608733989 74.00452205718096) bank105181 +105182 POINT(40.556754656048504 73.58558802362192) bank105182 +105183 POINT(39.95101487204468 73.68700358009613) bank105183 +105184 POINT(40.1662800559725 74.26789435836366) bank105184 +105185 POINT(40.719339995557206 73.55181220961289) bank105185 +105186 POINT(39.790420698392836 73.56043599614357) bank105186 +105187 POINT(40.42837416194786 73.20469323252144) bank105187 +105188 POINT(41.482096226621394 74.7021810675333) bank105188 +105189 POINT(40.41892075100391 73.3425995391139) bank105189 +105190 POINT(41.45769234926147 74.28458520967148) bank105190 +105191 POINT(39.75653193930417 74.59865947038551) bank105191 +105192 POINT(39.761714227379464 73.9850396849584) bank105192 +105193 POINT(41.20299455635191 73.24456833974129) bank105193 +105194 POINT(39.89744943829645 74.71452772148746) bank105194 +105195 POINT(41.37402353045741 73.73591556981664) bank105195 +105196 POINT(41.44706128595705 73.66981530923401) bank105196 +105197 POINT(41.26502571149416 74.31959722678656) bank105197 +105198 POINT(40.425207121208715 74.25517770282119) bank105198 +105199 POINT(40.52279092796978 74.03121438892036) bank105199 +105200 POINT(40.55681234921758 73.70865463994026) bank105200 +105201 POINT(39.735131477734434 73.5332144964316) bank105201 +105202 POINT(40.62184586118971 74.60408165729453) bank105202 +105203 POINT(40.84281350075033 73.50494636536405) bank105203 +105204 POINT(40.09632367574894 74.06438498313823) bank105204 +105205 POINT(41.6700173900814 74.74517659733529) bank105205 +105206 POINT(40.979995020273485 74.07222070592228) bank105206 +105207 POINT(40.08541504247347 73.90061592707221) bank105207 +105208 POINT(40.63374825265894 73.4001723615144) bank105208 +105209 POINT(40.13965125024709 73.72669889777112) bank105209 +105210 POINT(41.57966259464559 74.10432481089818) bank105210 +105211 POINT(40.630187629163274 74.34967596744492) bank105211 +105212 POINT(40.994771233703815 73.58682677117865) bank105212 +105213 POINT(41.45191054324844 73.38658668853944) bank105213 +105214 POINT(41.54295561032169 73.85482657224021) bank105214 +105215 POINT(40.15545379604899 73.20582254239253) bank105215 +105216 POINT(39.7796197834668 73.64234029455467) bank105216 +105217 POINT(40.45263753563333 74.84285496928607) bank105217 +105218 POINT(41.52008249804177 74.4063120849079) bank105218 +105219 POINT(41.23274397531832 74.53606894226814) bank105219 +105220 POINT(40.914912504322196 73.81114856884496) bank105220 +105221 POINT(41.52362185186433 73.22352556432355) bank105221 +105222 POINT(39.86886527068699 74.29703130146493) bank105222 +105223 POINT(39.75258641819117 74.446918181297) bank105223 +105224 POINT(41.174317377217655 74.63110353226737) bank105224 +105225 POINT(40.84213512484633 73.77579549122547) bank105225 +105226 POINT(40.909289277846625 74.81395868373086) bank105226 +105227 POINT(40.273598617408325 73.24196462038253) bank105227 +105228 POINT(39.78529360150787 74.34791600530382) bank105228 +105229 POINT(40.493708311377354 74.85040029822808) bank105229 +105230 POINT(41.7071965377404 73.85269632152544) bank105230 +105231 POINT(39.91550073250582 73.60116971818447) bank105231 +105232 POINT(41.68921521512076 73.11914775155087) bank105232 +105233 POINT(40.177054498564765 74.62495658204466) bank105233 +105234 POINT(39.77472112619821 74.79792078227244) bank105234 +105235 POINT(40.81415075961104 73.39964503920159) bank105235 +105236 POINT(40.99000485687233 74.07896196292522) bank105236 +105237 POINT(41.17595826461485 74.31067207818482) bank105237 +105238 POINT(40.45543072981746 73.22469890830412) bank105238 +105239 POINT(40.07268281775325 74.27532336521547) bank105239 +105240 POINT(40.93782252319243 73.5751226181851) bank105240 +105241 POINT(41.186593640970514 74.8434033348326) bank105241 +105242 POINT(41.04948051121132 74.70272820058916) bank105242 +105243 POINT(40.157267077510795 73.43502735272493) bank105243 +105244 POINT(41.67887422870918 74.33341427056664) bank105244 +105245 POINT(40.21632917721933 73.15908913948145) bank105245 +105246 POINT(41.21494009367842 74.64068571978963) bank105246 +105247 POINT(40.38776782989686 73.93855080953956) bank105247 +105248 POINT(41.31660098251735 74.83415204002583) bank105248 +105249 POINT(40.12143926772092 74.57580822143767) bank105249 +105250 POINT(40.866077948139505 74.894732626146) bank105250 +105251 POINT(41.28543825847129 74.71683408968009) bank105251 +105252 POINT(41.684857571785614 73.44990866244885) bank105252 +105253 POINT(40.692807835031424 74.31359516619685) bank105253 +105254 POINT(40.72762909780676 74.11743405159866) bank105254 +105255 POINT(41.438058533074724 73.19028960447798) bank105255 +105256 POINT(40.67603252067967 73.80679262323574) bank105256 +105257 POINT(40.86520181046095 73.68486065484822) bank105257 +105258 POINT(39.87405293280962 74.33414478660127) bank105258 +105259 POINT(40.698390312832274 73.7635342745154) bank105259 +105260 POINT(41.17139618502142 74.86039807430056) bank105260 +105261 POINT(40.77749215133445 73.1796632158853) bank105261 +105262 POINT(40.73501782160523 73.74706687550919) bank105262 +105263 POINT(40.492928825513026 73.38637661080398) bank105263 +105264 POINT(40.41606850032657 74.36901062755543) bank105264 +105265 POINT(40.756745301045484 73.15737858057082) bank105265 +105266 POINT(41.37907497589485 73.60645725880724) bank105266 +105267 POINT(41.4256940886752 74.5372854264437) bank105267 +105268 POINT(39.868239886141225 74.34713431588388) bank105268 +105269 POINT(40.83187642500976 74.70564578017779) bank105269 +105270 POINT(41.270699892847546 74.79355058409377) bank105270 +105271 POINT(40.54196467628452 73.35870446388768) bank105271 +105272 POINT(41.237092142574184 73.73712657900447) bank105272 +105273 POINT(39.892040597375576 74.87636452623853) bank105273 +105274 POINT(40.43783633074851 73.5862686815823) bank105274 +105275 POINT(40.91873805341062 73.54690973565629) bank105275 +105276 POINT(41.637942918098645 74.96100694030523) bank105276 +105277 POINT(40.991240263714666 73.24659852573338) bank105277 +105278 POINT(41.16864058073833 73.24290879316432) bank105278 +105279 POINT(40.589506927747095 73.65055936720974) bank105279 +105280 POINT(40.51644181670778 74.19553768093074) bank105280 +105281 POINT(41.2969774748315 73.88388803800969) bank105281 +105282 POINT(41.34188379446948 73.70363228702192) bank105282 +105283 POINT(40.63386617992438 74.33076513561564) bank105283 +105284 POINT(41.51915447927486 74.43036297149703) bank105284 +105285 POINT(41.54834965077703 73.59198112797122) bank105285 +105286 POINT(40.82458862504302 74.48786103640103) bank105286 +105287 POINT(40.3245870106894 73.89705044615314) bank105287 +105288 POINT(41.37577016030073 74.04786558171618) bank105288 +105289 POINT(40.250262341339855 73.80174788912753) bank105289 +105290 POINT(40.99130790121216 74.47152954458687) bank105290 +105291 POINT(40.693580384297285 74.28856009371282) bank105291 +105292 POINT(40.0297405565923 73.85366445088101) bank105292 +105293 POINT(41.47387240616628 74.78087568910577) bank105293 +105294 POINT(41.092895047567794 74.15145348172248) bank105294 +105295 POINT(41.04012798538829 74.23574031416553) bank105295 +105296 POINT(41.69423361674271 73.17140150765945) bank105296 +105297 POINT(41.02740535192522 74.54338196399142) bank105297 +105298 POINT(39.909110927780254 74.5988180025189) bank105298 +105299 POINT(41.495394504072124 73.50469032688498) bank105299 +105300 POINT(39.716909359131826 73.96211220523638) bank105300 +105301 POINT(40.664846017020885 74.21664430480776) bank105301 +105302 POINT(40.12786403070326 73.02144811735815) bank105302 +105303 POINT(41.00851624952067 74.74869808622205) bank105303 +105304 POINT(40.48836727474898 73.5936457618668) bank105304 +105305 POINT(41.579122313098765 73.53653551867116) bank105305 +105306 POINT(41.39784571619941 73.54603337002447) bank105306 +105307 POINT(41.10700826642472 73.54741532051524) bank105307 +105308 POINT(40.91465828706806 73.63013386856065) bank105308 +105309 POINT(41.057321292946284 74.36296363992606) bank105309 +105310 POINT(40.86543980550198 73.53892447993589) bank105310 +105311 POINT(40.09748338745587 73.98260547618442) bank105311 +105312 POINT(41.03899830242041 73.70288154837702) bank105312 +105313 POINT(40.166723181556215 74.69537796002949) bank105313 +105314 POINT(40.88695855466386 73.49374096742649) bank105314 +105315 POINT(40.96382405580998 74.83235121055854) bank105315 +105316 POINT(39.89989340689705 74.72314109643884) bank105316 +105317 POINT(40.019889281655665 74.27852625804402) bank105317 +105318 POINT(41.29160680508087 74.37636958807566) bank105318 +105319 POINT(41.17877235717324 74.11083458136075) bank105319 +105320 POINT(40.267161451688814 74.07291213413755) bank105320 +105321 POINT(41.137065957494876 73.2998468502845) bank105321 +105322 POINT(40.5587646357623 73.62547043798237) bank105322 +105323 POINT(40.790432189163866 73.74934795758362) bank105323 +105324 POINT(41.183244743119985 74.8150139526285) bank105324 +105325 POINT(40.52308411491688 73.6121960487277) bank105325 +105326 POINT(41.05908916986094 73.79286184904724) bank105326 +105327 POINT(40.39505578479619 73.41408290899894) bank105327 +105328 POINT(39.87350647602125 74.40969285426881) bank105328 +105329 POINT(41.37592248155797 73.70923872685682) bank105329 +105330 POINT(40.169505883779145 73.54818969793448) bank105330 +105331 POINT(41.35363199793343 73.63875269784035) bank105331 +105332 POINT(39.78790805691205 74.50559701166368) bank105332 +105333 POINT(40.51659675703741 74.50947322414328) bank105333 +105334 POINT(40.59003449663981 73.01084161246942) bank105334 +105335 POINT(41.04685270332265 74.90260199260953) bank105335 +105336 POINT(41.013214362068474 73.069837923358) bank105336 +105337 POINT(40.56017374561827 74.10983787821735) bank105337 +105338 POINT(40.46213798177212 74.15631020276227) bank105338 +105339 POINT(40.30462688229837 73.27431299034318) bank105339 +105340 POINT(39.7281562123859 73.15301392094236) bank105340 +105341 POINT(40.62382425063371 74.50221176149041) bank105341 +105342 POINT(40.01124460864688 73.20380275199456) bank105342 +105343 POINT(40.33426306725214 74.06003914812561) bank105343 +105344 POINT(40.85635650554987 73.97007752763804) bank105344 +105345 POINT(41.11750449544556 73.92358866251988) bank105345 +105346 POINT(39.98597096165153 73.17132123539486) bank105346 +105347 POINT(40.25030972398511 73.42067954236741) bank105347 +105348 POINT(40.99362562497685 73.72897257202422) bank105348 +105349 POINT(40.87091134126227 73.54411042130573) bank105349 +105350 POINT(41.612495202434886 73.31121945209827) bank105350 +105351 POINT(40.8073350435486 74.3829491966856) bank105351 +105352 POINT(40.32063232888197 74.02979122844742) bank105352 +105353 POINT(40.944725448991846 74.72193238303653) bank105353 +105354 POINT(39.73190981066368 74.16855543088006) bank105354 +105355 POINT(41.22943566578147 74.13207012166156) bank105355 +105356 POINT(39.73593676890331 73.75036592197924) bank105356 +105357 POINT(40.678740118475055 74.2223971608833) bank105357 +105358 POINT(40.570565453136176 73.08261381231377) bank105358 +105359 POINT(39.887113671870154 74.40300545960912) bank105359 +105360 POINT(41.37953565480921 74.00208604367775) bank105360 +105361 POINT(39.958434032874436 73.80306410627664) bank105361 +105362 POINT(40.08050745436978 73.91622161590912) bank105362 +105363 POINT(40.21567226287002 73.1858575338195) bank105363 +105364 POINT(40.438447446163224 74.06863842001778) bank105364 +105365 POINT(41.106267605141966 73.75225911804071) bank105365 +105366 POINT(40.32761827399042 74.20266334881953) bank105366 +105367 POINT(41.157886333209916 74.07908260345849) bank105367 +105368 POINT(40.40436746748973 74.87890401223237) bank105368 +105369 POINT(40.155101136768025 74.09327554205758) bank105369 +105370 POINT(40.14951892122067 73.55840656132699) bank105370 +105371 POINT(40.773929868652516 73.3478124583401) bank105371 +105372 POINT(41.41719725043747 74.92764990013002) bank105372 +105373 POINT(40.916512132422994 73.17034448970989) bank105373 +105374 POINT(39.97806857669837 73.79205023719538) bank105374 +105375 POINT(40.223082195485425 73.47673782021754) bank105375 +105376 POINT(41.02749970134676 73.25676350583335) bank105376 +105377 POINT(41.38092163941212 74.21736881236238) bank105377 +105378 POINT(40.2198929745249 74.93428689283422) bank105378 +105379 POINT(40.47701510364651 73.83170233855938) bank105379 +105380 POINT(41.431137765719015 73.73353617465973) bank105380 +105381 POINT(41.28291804573342 74.48871021770672) bank105381 +105382 POINT(41.69053642053265 73.15130140407237) bank105382 +105383 POINT(40.40025414490136 73.43661313613325) bank105383 +105384 POINT(40.978802665085 73.88393759953725) bank105384 +105385 POINT(40.43625246525643 74.88199669858867) bank105385 +105386 POINT(39.974997257350005 73.38646739909065) bank105386 +105387 POINT(39.84875181461255 74.47792703319502) bank105387 +105388 POINT(39.78003417752648 73.61698498410891) bank105388 +105389 POINT(41.64623653462498 73.18729389788149) bank105389 +105390 POINT(40.963804072425084 73.0188469997793) bank105390 +105391 POINT(41.24809450636996 74.26814006586139) bank105391 +105392 POINT(41.15567735373859 74.89817015040364) bank105392 +105393 POINT(39.99806278937903 74.52517756009593) bank105393 +105394 POINT(40.595680263604926 74.26687894956322) bank105394 +105395 POINT(39.847204420920875 73.58750998452943) bank105395 +105396 POINT(40.49995208580449 74.11694712481429) bank105396 +105397 POINT(40.575070325517494 74.96931921208285) bank105397 +105398 POINT(40.503196202889384 73.93475875148553) bank105398 +105399 POINT(40.27068305413494 73.95307381038229) bank105399 +105400 POINT(40.580356751448804 73.66577819718187) bank105400 +105401 POINT(40.682935922362894 74.39234260683845) bank105401 +105402 POINT(40.616907681799475 73.79406740287646) bank105402 +105403 POINT(40.28860187845928 73.35439053056807) bank105403 +105404 POINT(40.457114105844624 74.46054160561924) bank105404 +105405 POINT(41.542198460922585 73.7657823833074) bank105405 +105406 POINT(39.93496320067224 74.39316572819963) bank105406 +105407 POINT(40.03783743964484 73.11294852963736) bank105407 +105408 POINT(40.16058007443948 73.06743196083622) bank105408 +105409 POINT(41.10443212424997 73.62263022987663) bank105409 +105410 POINT(41.50181151217442 73.97149394045711) bank105410 +105411 POINT(40.020895352900645 73.54915401946397) bank105411 +105412 POINT(41.67200190703166 74.21576460197734) bank105412 +105413 POINT(40.36218191787769 73.72016005166222) bank105413 +105414 POINT(40.37437474186955 74.51484920096361) bank105414 +105415 POINT(41.5973523289645 73.58173254335807) bank105415 +105416 POINT(40.20830900209265 73.84337048498462) bank105416 +105417 POINT(40.46313996879081 73.05940260266263) bank105417 +105418 POINT(40.3529932158272 74.07703694027563) bank105418 +105419 POINT(40.60871812231042 73.83067126911617) bank105419 +105420 POINT(41.671490302458984 74.71245540729377) bank105420 +105421 POINT(41.02674073236968 73.36160954389965) bank105421 +105422 POINT(40.47715481312936 74.63383117966254) bank105422 +105423 POINT(40.94992942020147 73.17989708512427) bank105423 +105424 POINT(40.633216498512155 74.11013650219735) bank105424 +105425 POINT(40.70159169142226 73.72023658498085) bank105425 +105426 POINT(41.072664969176955 74.51888727698892) bank105426 +105427 POINT(40.621038816189866 74.62480195304943) bank105427 +105428 POINT(40.71276424659754 73.42384007059341) bank105428 +105429 POINT(39.94089536739759 73.44394146821233) bank105429 +105430 POINT(40.21503100429165 73.43457958954754) bank105430 +105431 POINT(41.54986082951132 74.21634660928177) bank105431 +105432 POINT(40.86409403091733 73.60077897796162) bank105432 +105433 POINT(40.518200557424116 74.53992193319638) bank105433 +105434 POINT(40.818624305426674 74.90388624959658) bank105434 +105435 POINT(40.54283895090675 73.47468997452307) bank105435 +105436 POINT(40.292004194493664 73.01214963312451) bank105436 +105437 POINT(41.27491730600386 74.94542233432901) bank105437 +105438 POINT(40.74522856935576 73.48080629777917) bank105438 +105439 POINT(40.37193952980016 73.23866689661587) bank105439 +105440 POINT(41.31451485946059 75.0010302135314) bank105440 +105441 POINT(41.07208673174535 73.35831899342966) bank105441 +105442 POINT(41.156876897670884 74.83708814012985) bank105442 +105443 POINT(41.16365596503219 74.61179331480912) bank105443 +105444 POINT(41.20126898021956 73.18717283303117) bank105444 +105445 POINT(40.56851231469309 73.09113655430248) bank105445 +105446 POINT(41.11900896774239 73.18275773444181) bank105446 +105447 POINT(40.671558966597445 73.53355214761221) bank105447 +105448 POINT(39.88433329017395 74.2302335461602) bank105448 +105449 POINT(41.34365638969412 73.66789605570781) bank105449 +105450 POINT(40.531016646326314 74.11448547771795) bank105450 +105451 POINT(40.719868749096406 74.74753430675409) bank105451 +105452 POINT(41.66866070807023 73.93384429991458) bank105452 +105453 POINT(39.862197982182735 73.02450182872501) bank105453 +105454 POINT(40.601622637744896 74.91770180463017) bank105454 +105455 POINT(40.75473467200278 74.72107790992737) bank105455 +105456 POINT(41.2258211550196 73.81201681192125) bank105456 +105457 POINT(40.38965233573242 73.43105333276331) bank105457 +105458 POINT(41.33394110812301 73.86083330275498) bank105458 +105459 POINT(40.70087793215478 73.560374610566) bank105459 +105460 POINT(40.42872066372557 74.8519476270944) bank105460 +105461 POINT(41.00428010001741 73.2860884813249) bank105461 +105462 POINT(40.50919602418824 74.39599442003016) bank105462 +105463 POINT(40.970340903055856 73.04334018145107) bank105463 +105464 POINT(40.43537786158301 74.85850428970038) bank105464 +105465 POINT(40.47932669283792 74.32788417069133) bank105465 +105466 POINT(39.77189911471514 73.23673633917329) bank105466 +105467 POINT(41.244781157453794 73.86110112755827) bank105467 +105468 POINT(41.51329378405001 74.52735898256378) bank105468 +105469 POINT(41.218949447258396 73.94539428631444) bank105469 +105470 POINT(41.06045005042217 73.14126747331191) bank105470 +105471 POINT(40.28275044943873 74.7384467859731) bank105471 +105472 POINT(40.580062929785356 73.35203603228956) bank105472 +105473 POINT(39.86981719033352 73.13421102306762) bank105473 +105474 POINT(39.845903526654226 73.05475236509363) bank105474 +105475 POINT(39.715117934494714 74.69285596429592) bank105475 +105476 POINT(40.85146531614983 74.65165821365096) bank105476 +105477 POINT(40.59306034919676 73.34946480130706) bank105477 +105478 POINT(40.34420546372295 74.51556328787163) bank105478 +105479 POINT(40.66919568092917 74.14980496781405) bank105479 +105480 POINT(39.81174614090055 73.4406895232897) bank105480 +105481 POINT(41.70594268966002 73.58762892096006) bank105481 +105482 POINT(41.681127699635184 74.90945464375213) bank105482 +105483 POINT(39.95272005430715 74.64715545426677) bank105483 +105484 POINT(39.965109969310944 74.83615466788176) bank105484 +105485 POINT(41.12218247022315 73.16000607891652) bank105485 +105486 POINT(40.204470632793814 74.04222922040962) bank105486 +105487 POINT(40.771362095840324 74.0366157829842) bank105487 +105488 POINT(41.24430986356631 73.96409977608023) bank105488 +105489 POINT(41.16522189716719 73.19139091650287) bank105489 +105490 POINT(40.67913070669264 73.40682137993134) bank105490 +105491 POINT(40.51230264506171 74.14258932262095) bank105491 +105492 POINT(40.12642821640278 73.19760257289433) bank105492 +105493 POINT(40.765080126549044 74.85580341825967) bank105493 +105494 POINT(41.67834365236392 74.73999454023131) bank105494 +105495 POINT(41.10241023994574 74.16857674053274) bank105495 +105496 POINT(41.54355333059668 74.7076103620704) bank105496 +105497 POINT(41.66987374083172 74.96269514378568) bank105497 +105498 POINT(41.07197810057874 74.8929505879929) bank105498 +105499 POINT(40.065999046882474 73.74727843356995) bank105499 +105500 POINT(41.20058547814436 74.98636183635375) bank105500 +105501 POINT(39.87984908985969 74.46486690420193) bank105501 +105502 POINT(39.83421256755547 74.37141305353317) bank105502 +105503 POINT(41.30378151187152 73.28960560425713) bank105503 +105504 POINT(40.458076027588056 73.44054172591807) bank105504 +105505 POINT(39.791677281889214 74.23241799231305) bank105505 +105506 POINT(41.29530814554069 74.3639993936006) bank105506 +105507 POINT(40.199317229206265 74.31941007573339) bank105507 +105508 POINT(40.83452882790897 74.73594268795325) bank105508 +105509 POINT(40.05068433265083 73.56568281318008) bank105509 +105510 POINT(39.77979109529793 73.84124807857827) bank105510 +105511 POINT(40.78218405438561 74.39735442826037) bank105511 +105512 POINT(41.3034253422399 73.51785209521688) bank105512 +105513 POINT(41.513870950754125 73.07526637950936) bank105513 +105514 POINT(40.464476426624785 74.40891100209005) bank105514 +105515 POINT(41.38858571472605 74.51773137284019) bank105515 +105516 POINT(39.72506553989233 74.46896078469554) bank105516 +105517 POINT(41.23653906704897 73.23436785804438) bank105517 +105518 POINT(40.21558973583805 73.03337971224262) bank105518 +105519 POINT(39.881417106387076 73.47043684374043) bank105519 +105520 POINT(40.0409808891885 73.88273527075783) bank105520 +105521 POINT(40.80419924128055 74.54684777534597) bank105521 +105522 POINT(41.070453539837196 74.85310639937298) bank105522 +105523 POINT(41.32502585752286 73.9182501768765) bank105523 +105524 POINT(40.4500708089912 73.82310558705505) bank105524 +105525 POINT(41.14707009582962 73.65151394180344) bank105525 +105526 POINT(39.805831119955 73.32600893544043) bank105526 +105527 POINT(40.14518565519683 74.03942058900653) bank105527 +105528 POINT(41.254009185126336 74.66026631506114) bank105528 +105529 POINT(41.65445500577705 73.64513283931927) bank105529 +105530 POINT(41.663949019951005 74.52785663008085) bank105530 +105531 POINT(41.60723335675264 73.63640152566848) bank105531 +105532 POINT(39.80200410853592 74.52231367512796) bank105532 +105533 POINT(41.17489198270021 73.5365053717753) bank105533 +105534 POINT(41.16395736270969 74.80463861508515) bank105534 +105535 POINT(41.62228911587647 73.49872745767881) bank105535 +105536 POINT(41.4579176909954 73.85522738158035) bank105536 +105537 POINT(40.6217348785335 74.21420314152716) bank105537 +105538 POINT(40.11884331656838 74.07715198356021) bank105538 +105539 POINT(40.84142022766058 74.79338723407639) bank105539 +105540 POINT(41.452512240258855 73.43552244659257) bank105540 +105541 POINT(40.897147052599834 73.73067166700577) bank105541 +105542 POINT(40.42665728204551 74.3290304760149) bank105542 +105543 POINT(40.71456850004246 74.63294477563267) bank105543 +105544 POINT(40.89442291550986 74.50109819327503) bank105544 +105545 POINT(41.27324096602786 74.93354380608642) bank105545 +105546 POINT(40.587824174297936 74.35111301213868) bank105546 +105547 POINT(41.22550115017227 74.88690155899161) bank105547 +105548 POINT(41.203827860979345 74.71520440674001) bank105548 +105549 POINT(39.93977713146179 73.16764265267227) bank105549 +105550 POINT(40.94908205546655 74.24318455376361) bank105550 +105551 POINT(40.607868294811034 73.45384066288696) bank105551 +105552 POINT(40.020353100222614 73.42032876751753) bank105552 +105553 POINT(41.61454076734461 74.71585236280956) bank105553 +105554 POINT(41.29574497043445 74.04183803188377) bank105554 +105555 POINT(40.68089472381088 74.51633900043504) bank105555 +105556 POINT(39.873111727541605 73.96855120224818) bank105556 +105557 POINT(41.247572471863855 73.03718401393657) bank105557 +105558 POINT(41.65554977302987 73.75696643734408) bank105558 +105559 POINT(41.50784494864594 74.99634814426173) bank105559 +105560 POINT(40.73071286955526 73.2307775915819) bank105560 +105561 POINT(41.383771329724524 73.85301375449977) bank105561 +105562 POINT(39.80499893409578 74.35211290265494) bank105562 +105563 POINT(41.36372082778806 74.08045346749637) bank105563 +105564 POINT(40.32313652975629 73.06885554547216) bank105564 +105565 POINT(41.13179122263431 73.19739799495233) bank105565 +105566 POINT(41.5717385552389 74.83324255890803) bank105566 +105567 POINT(41.35820255645608 73.12851719567936) bank105567 +105568 POINT(41.35291824373495 73.70841861163507) bank105568 +105569 POINT(39.93186811067957 73.86431918300329) bank105569 +105570 POINT(41.1489205804822 74.0826149266898) bank105570 +105571 POINT(41.42411017852742 74.45321589855931) bank105571 +105572 POINT(39.879083255905236 74.87114603359186) bank105572 +105573 POINT(40.72633277889706 74.80065348473802) bank105573 +105574 POINT(41.499560195174716 73.19592124064252) bank105574 +105575 POINT(40.907672408096964 73.15329231975579) bank105575 +105576 POINT(41.073315445132806 73.33523313683865) bank105576 +105577 POINT(40.19690173996588 74.82414799243443) bank105577 +105578 POINT(41.307464941476034 74.74959049846852) bank105578 +105579 POINT(40.481620913647326 73.33151818877747) bank105579 +105580 POINT(41.29652240717179 74.02937683545044) bank105580 +105581 POINT(40.70770682622881 74.49042425735959) bank105581 +105582 POINT(40.24141960859085 74.94975624698864) bank105582 +105583 POINT(41.028289046157056 73.20577528533246) bank105583 +105584 POINT(40.62249263204615 74.76415804262793) bank105584 +105585 POINT(41.24737006100423 74.7858908043703) bank105585 +105586 POINT(41.031169722423435 73.35585239156221) bank105586 +105587 POINT(40.42341852439791 74.85883653371609) bank105587 +105588 POINT(41.35186130331083 74.99923141004116) bank105588 +105589 POINT(40.35863431789617 73.92241291260936) bank105589 +105590 POINT(40.55651979052861 74.36164452767416) bank105590 +105591 POINT(40.127032522653074 73.51002153374867) bank105591 +105592 POINT(41.098158575232496 73.29191853069483) bank105592 +105593 POINT(41.44544565251824 74.93911468826528) bank105593 +105594 POINT(40.37130717315117 73.49451459805036) bank105594 +105595 POINT(40.30966069425799 74.13552458767208) bank105595 +105596 POINT(41.35019375435628 74.8588830826063) bank105596 +105597 POINT(39.80086219130409 73.75824248624573) bank105597 +105598 POINT(40.50384778242831 73.0597897205263) bank105598 +105599 POINT(41.677235626512285 73.68466585686909) bank105599 +105600 POINT(41.14775222290626 74.23451643649044) bank105600 +105601 POINT(40.96375820136744 74.10470284323769) bank105601 +105602 POINT(41.26169604839078 73.20186481531478) bank105602 +105603 POINT(40.26651938993472 73.32951239297884) bank105603 +105604 POINT(39.727334614234145 74.31090333763485) bank105604 +105605 POINT(41.00910477381556 73.0314030699696) bank105605 +105606 POINT(40.1937472589392 73.30236450752679) bank105606 +105607 POINT(40.437840464896865 73.13587246626687) bank105607 +105608 POINT(40.0235834100014 74.38313969784068) bank105608 +105609 POINT(39.79562739753063 73.16052535661885) bank105609 +105610 POINT(40.77215679905491 73.51720210753972) bank105610 +105611 POINT(41.13070729790509 74.89413621891326) bank105611 +105612 POINT(40.453075247488655 73.87360831890756) bank105612 +105613 POINT(40.91201875297131 73.94638486504869) bank105613 +105614 POINT(41.65118179382702 73.61679714778496) bank105614 +105615 POINT(41.319918278430954 73.00639906801726) bank105615 +105616 POINT(41.22328822070373 74.24512614355298) bank105616 +105617 POINT(40.57486239947897 73.09307570333334) bank105617 +105618 POINT(41.66829849003154 73.57664785130231) bank105618 +105619 POINT(39.90374511309896 74.40025388203891) bank105619 +105620 POINT(40.54546148670812 73.77272052491425) bank105620 +105621 POINT(39.849085783775685 74.82023628491488) bank105621 +105622 POINT(39.94472003383462 74.29068748310259) bank105622 +105623 POINT(40.95622573813425 74.08038301136773) bank105623 +105624 POINT(40.70342186618738 74.13297982003468) bank105624 +105625 POINT(40.988178049737584 73.65589807625386) bank105625 +105626 POINT(40.77119989499351 73.0204998941876) bank105626 +105627 POINT(40.219316499774415 73.9829467027269) bank105627 +105628 POINT(41.28028483816463 73.19879372672703) bank105628 +105629 POINT(39.92179327845937 73.89717824411599) bank105629 +105630 POINT(40.29271501916932 74.42562630434038) bank105630 +105631 POINT(41.493540978571446 73.4640353256414) bank105631 +105632 POINT(40.25418243267868 73.0091937079559) bank105632 +105633 POINT(40.502734781161145 74.99983718926833) bank105633 +105634 POINT(41.082142456964775 74.72468970923704) bank105634 +105635 POINT(41.51939985002457 74.66159178739973) bank105635 +105636 POINT(40.47087651252468 73.00852586619939) bank105636 +105637 POINT(41.680263868526666 73.42176997333931) bank105637 +105638 POINT(40.949415476655176 74.17663767614178) bank105638 +105639 POINT(39.872853410994594 74.41897516684752) bank105639 +105640 POINT(41.3670323139414 73.20469111892568) bank105640 +105641 POINT(41.624177362621914 73.75344263694443) bank105641 +105642 POINT(41.53722440565154 73.6264905412577) bank105642 +105643 POINT(39.93037007545289 74.93878875709487) bank105643 +105644 POINT(39.792966148519874 73.31397040124165) bank105644 +105645 POINT(40.69040351542309 73.80124392385761) bank105645 +105646 POINT(40.95724202994839 73.5918148639633) bank105646 +105647 POINT(41.116795184728964 73.18556834173714) bank105647 +105648 POINT(41.2524628462729 73.53821569061792) bank105648 +105649 POINT(40.9774988738309 73.96799561948552) bank105649 +105650 POINT(40.81507419252074 74.00441317723346) bank105650 +105651 POINT(40.58980688009649 73.70663652074482) bank105651 +105652 POINT(41.00196309105501 74.75193944671359) bank105652 +105653 POINT(40.08715297196291 74.55224868398447) bank105653 +105654 POINT(40.5323099158648 73.35602167334655) bank105654 +105655 POINT(39.978917401402214 74.39008779364924) bank105655 +105656 POINT(40.98771626337917 74.57418835024805) bank105656 +105657 POINT(40.50815345750629 74.69287392322339) bank105657 +105658 POINT(41.67853328930522 74.49661132770959) bank105658 +105659 POINT(41.11740314889604 73.92739646797381) bank105659 +105660 POINT(41.407163686466845 73.47838802234305) bank105660 +105661 POINT(40.39289039716584 74.09603699050967) bank105661 +105662 POINT(41.148717152741305 74.16114728418205) bank105662 +105663 POINT(40.95186556313373 73.11723942850321) bank105663 +105664 POINT(40.81430775661371 74.22603546939382) bank105664 +105665 POINT(40.66376359290965 74.15719386812317) bank105665 +105666 POINT(41.30063701690549 74.58985131769903) bank105666 +105667 POINT(41.536041979219085 74.10335701324166) bank105667 +105668 POINT(40.95716446810601 74.56769419294642) bank105668 +105669 POINT(40.216259280422506 74.950363026534) bank105669 +105670 POINT(40.58766058885748 73.34114065367987) bank105670 +105671 POINT(41.06503985902893 74.44741990318693) bank105671 +105672 POINT(41.57011128450038 73.78166595148122) bank105672 +105673 POINT(41.455937485861874 73.41498460543548) bank105673 +105674 POINT(40.00496335601508 74.37773171427095) bank105674 +105675 POINT(41.71132740019644 74.1549869555849) bank105675 +105676 POINT(41.550014036899334 73.96078739493682) bank105676 +105677 POINT(40.935669846901185 73.23639481557409) bank105677 +105678 POINT(40.36665176560128 73.05843066549076) bank105678 +105679 POINT(40.82373654399997 73.16438879803258) bank105679 +105680 POINT(40.668615116843725 74.48899690815053) bank105680 +105681 POINT(40.58831974067066 74.04202358703301) bank105681 +105682 POINT(39.934581047506896 74.32458837466902) bank105682 +105683 POINT(40.12936858531726 74.05381138469747) bank105683 +105684 POINT(40.38508735354417 74.52919109622357) bank105684 +105685 POINT(40.33816367473587 74.27536296782927) bank105685 +105686 POINT(41.17513538501473 74.24279478409161) bank105686 +105687 POINT(40.024348811675196 74.56398467834957) bank105687 +105688 POINT(39.97664223463148 73.25721844237988) bank105688 +105689 POINT(40.26233694148391 74.19370127680737) bank105689 +105690 POINT(40.26378950216446 73.72793737951474) bank105690 +105691 POINT(40.22323745331892 73.83690461523925) bank105691 +105692 POINT(40.392418789352135 73.85432225703623) bank105692 +105693 POINT(41.690971409781206 74.22993606408595) bank105693 +105694 POINT(40.62164892213809 74.22819753500619) bank105694 +105695 POINT(40.255065478071444 74.30827361470777) bank105695 +105696 POINT(40.56309455841483 73.36448334432549) bank105696 +105697 POINT(41.47939968357904 73.30684982870223) bank105697 +105698 POINT(41.19337601797461 73.06069554157045) bank105698 +105699 POINT(40.238217083561636 73.19334449648169) bank105699 +105700 POINT(40.03027744202556 73.66879367250917) bank105700 +105701 POINT(40.36601001270481 74.29407618769875) bank105701 +105702 POINT(40.949313211207695 73.16931499686703) bank105702 +105703 POINT(40.19441218601667 73.97116420319577) bank105703 +105704 POINT(40.115566544723805 73.7123552158786) bank105704 +105705 POINT(40.261667367661936 74.43472339088096) bank105705 +105706 POINT(39.80022359111762 74.25047049235982) bank105706 +105707 POINT(39.751567355447094 73.30256841794775) bank105707 +105708 POINT(39.799160187701574 74.8561258702984) bank105708 +105709 POINT(40.9740254320962 74.91451275512863) bank105709 +105710 POINT(40.080646579957914 74.04574482791462) bank105710 +105711 POINT(39.918313286694385 74.79990659633212) bank105711 +105712 POINT(40.55325033672572 74.33150663819701) bank105712 +105713 POINT(40.035771438889356 74.40732082091007) bank105713 +105714 POINT(41.695940841782374 74.67551356014272) bank105714 +105715 POINT(40.768762091920685 73.24795762928807) bank105715 +105716 POINT(41.29972907064044 74.93918237063501) bank105716 +105717 POINT(39.92042609195596 73.380218307472) bank105717 +105718 POINT(41.56261409340571 73.4218024567726) bank105718 +105719 POINT(40.357345960711825 73.38437542507498) bank105719 +105720 POINT(40.3507878921123 74.23807381874657) bank105720 +105721 POINT(40.48708034046146 74.89052769711478) bank105721 +105722 POINT(41.07655748358985 74.85408459351959) bank105722 +105723 POINT(40.162831360105294 73.78832243773047) bank105723 +105724 POINT(40.37798046114527 74.47299135757261) bank105724 +105725 POINT(40.71584447413261 74.22387980182523) bank105725 +105726 POINT(41.292277719967466 74.9222778941148) bank105726 +105727 POINT(41.24640047927775 74.76139436658814) bank105727 +105728 POINT(39.97163420833718 74.32538464469533) bank105728 +105729 POINT(41.15302604938848 74.95352852032438) bank105729 +105730 POINT(40.75775526647151 73.58028199351975) bank105730 +105731 POINT(40.801531299555634 74.13136063892468) bank105731 +105732 POINT(41.07431147193442 74.87232072575699) bank105732 +105733 POINT(40.90182177182421 74.51458413706878) bank105733 +105734 POINT(40.07370934319594 74.2832887236994) bank105734 +105735 POINT(39.76283380294301 74.24803459376048) bank105735 +105736 POINT(40.4821638056543 74.45514979070072) bank105736 +105737 POINT(41.31943597657416 74.17718315312383) bank105737 +105738 POINT(40.874352607792 74.68185841651048) bank105738 +105739 POINT(41.67241314493582 73.41538360685726) bank105739 +105740 POINT(40.545595418012454 73.81828192240306) bank105740 +105741 POINT(40.9010474858215 74.76550199241176) bank105741 +105742 POINT(41.267493083456465 74.35063542961016) bank105742 +105743 POINT(40.916258638350946 73.9541667925883) bank105743 +105744 POINT(39.755565998872584 74.02516777402062) bank105744 +105745 POINT(39.9395991572828 74.71268834579654) bank105745 +105746 POINT(40.672913656881946 74.0995918985311) bank105746 +105747 POINT(39.98021441997411 74.70426723902872) bank105747 +105748 POINT(41.68149451848784 74.60957386204649) bank105748 +105749 POINT(40.6956378699465 73.6120133811331) bank105749 +105750 POINT(40.46260324040802 74.87485811336781) bank105750 +105751 POINT(40.29206815492187 73.60594404540117) bank105751 +105752 POINT(40.77445990421247 74.3980615575037) bank105752 +105753 POINT(40.454632170754806 74.9893325048538) bank105753 +105754 POINT(40.84676689477714 74.71241024804831) bank105754 +105755 POINT(40.82614786653535 74.30877032212433) bank105755 +105756 POINT(39.91340061980251 73.7464302459047) bank105756 +105757 POINT(41.220654860453635 73.80488331622092) bank105757 +105758 POINT(41.531694204088055 73.02361402662567) bank105758 +105759 POINT(40.234734016060216 74.31711370500145) bank105759 +105760 POINT(40.296604543425836 73.6602212916272) bank105760 +105761 POINT(39.8108483477597 74.22168260719779) bank105761 +105762 POINT(39.75449064791723 74.96754368102994) bank105762 +105763 POINT(39.86974486697327 74.60402999110548) bank105763 +105764 POINT(40.55740930171343 74.14606548301022) bank105764 +105765 POINT(39.864096539692625 74.77806225756872) bank105765 +105766 POINT(41.7100783396168 74.57854411367082) bank105766 +105767 POINT(40.00227937544776 74.36259495593733) bank105767 +105768 POINT(41.52724790581033 73.60171707093433) bank105768 +105769 POINT(41.59530965278448 74.73277209427413) bank105769 +105770 POINT(41.2250755673503 73.15915210595661) bank105770 +105771 POINT(39.72085591595953 74.35808110800966) bank105771 +105772 POINT(39.78199808637128 74.8084675574337) bank105772 +105773 POINT(39.71600874181545 73.13797630193639) bank105773 +105774 POINT(39.81971500057906 73.47500925601597) bank105774 +105775 POINT(41.306639171184734 74.0071364424406) bank105775 +105776 POINT(41.54146320468633 73.55266847119924) bank105776 +105777 POINT(40.04031396074571 73.0285503598629) bank105777 +105778 POINT(40.51611563798392 74.75468658643848) bank105778 +105779 POINT(40.98165179364746 73.775079659257) bank105779 +105780 POINT(39.82669675905619 73.49092151468408) bank105780 +105781 POINT(41.302740728190116 73.06561712959906) bank105781 +105782 POINT(41.08818480061573 73.50755060050821) bank105782 +105783 POINT(39.969540972098834 73.76180374568493) bank105783 +105784 POINT(40.75790316157267 74.23743687230834) bank105784 +105785 POINT(40.30070015312581 73.60431818906999) bank105785 +105786 POINT(39.858741395631995 73.203856919376) bank105786 +105787 POINT(40.96193991275853 74.8385435921235) bank105787 +105788 POINT(40.824741782186166 73.27651576117646) bank105788 +105789 POINT(40.51778008879455 74.02915203052524) bank105789 +105790 POINT(41.210423144202956 74.81799439040235) bank105790 +105791 POINT(40.1805443314349 74.6736208133863) bank105791 +105792 POINT(39.86138823583137 73.78713705771735) bank105792 +105793 POINT(40.680411443690325 73.30640507399623) bank105793 +105794 POINT(41.55574149494569 74.56347443151965) bank105794 +105795 POINT(40.432908657829046 73.94892118140478) bank105795 +105796 POINT(40.26767335365763 73.54036959299738) bank105796 +105797 POINT(41.577393467474195 74.97714426978659) bank105797 +105798 POINT(40.7992024184433 74.69094399017358) bank105798 +105799 POINT(41.29305774222825 74.93065691199352) bank105799 +105800 POINT(41.53480851138645 74.81148461928451) bank105800 +105801 POINT(40.004840049581716 74.99452876755076) bank105801 +105802 POINT(40.02506051151149 73.58115372959296) bank105802 +105803 POINT(40.772954574845826 73.94389550764265) bank105803 +105804 POINT(39.943304597895036 73.76861364947352) bank105804 +105805 POINT(40.006555298850905 74.35443371522274) bank105805 +105806 POINT(40.21781871127779 74.94677436637613) bank105806 +105807 POINT(41.68488980172378 74.68021938201662) bank105807 +105808 POINT(40.13567650838632 73.44456550950144) bank105808 +105809 POINT(39.76965875537275 73.70275710617662) bank105809 +105810 POINT(39.734334568933605 74.37376847339597) bank105810 +105811 POINT(41.04465142918301 74.9521487499248) bank105811 +105812 POINT(40.781240191783944 73.26513682309447) bank105812 +105813 POINT(40.3289481110468 74.28041391164722) bank105813 +105814 POINT(39.72945587330362 74.77721338350318) bank105814 +105815 POINT(40.34979688118071 73.73872426356127) bank105815 +105816 POINT(40.1129415002151 73.58322506450736) bank105816 +105817 POINT(41.09453670634816 73.68303746610017) bank105817 +105818 POINT(41.413609153817774 74.05761975218311) bank105818 +105819 POINT(40.626577750905795 73.45682976675494) bank105819 +105820 POINT(41.37159450364469 74.26582174729431) bank105820 +105821 POINT(41.59448979144407 74.25974492054003) bank105821 +105822 POINT(41.61999252183898 73.11034800666617) bank105822 +105823 POINT(41.38260860403034 73.3291629929316) bank105823 +105824 POINT(40.784479519073464 74.38313524108072) bank105824 +105825 POINT(41.22539192695137 74.35585902757371) bank105825 +105826 POINT(40.835375337375446 74.788252143808) bank105826 +105827 POINT(40.29323112681215 74.33021639179583) bank105827 +105828 POINT(41.06524247865783 74.0704117441245) bank105828 +105829 POINT(41.35073514660544 74.67070777109524) bank105829 +105830 POINT(41.312648607265345 73.88605585642303) bank105830 +105831 POINT(39.927240185499095 74.93484890783625) bank105831 +105832 POINT(41.03401578132606 74.54638983081816) bank105832 +105833 POINT(39.87511715963482 73.58161219457865) bank105833 +105834 POINT(39.86860179923866 74.67768167158923) bank105834 +105835 POINT(40.00902600843144 73.98169630519962) bank105835 +105836 POINT(40.33915362788646 73.80764894031918) bank105836 +105837 POINT(40.761631165593364 74.5057182362858) bank105837 +105838 POINT(40.60375115164005 74.19781281304054) bank105838 +105839 POINT(41.65064047797183 74.9975853648774) bank105839 +105840 POINT(40.534852177671446 74.45144456742148) bank105840 +105841 POINT(40.17197460198809 73.39917873635622) bank105841 +105842 POINT(39.952110407292245 74.50682841311264) bank105842 +105843 POINT(41.220441714804565 73.98610831749266) bank105843 +105844 POINT(40.311758581945746 74.17004621266373) bank105844 +105845 POINT(40.43535719703936 74.94831901319141) bank105845 +105846 POINT(41.4438238151343 74.76438059499195) bank105846 +105847 POINT(39.882784060757224 73.32001261250744) bank105847 +105848 POINT(39.77441503204022 74.37642307975204) bank105848 +105849 POINT(39.811319424304784 73.30690552470456) bank105849 +105850 POINT(40.39461542619989 74.50212748961083) bank105850 +105851 POINT(41.14419965591484 74.46369178813262) bank105851 +105852 POINT(39.98207312016398 73.91715864287981) bank105852 +105853 POINT(40.515920571961885 73.48549202827087) bank105853 +105854 POINT(41.153738625846806 74.78843542412085) bank105854 +105855 POINT(40.25337339034415 74.65366458941352) bank105855 +105856 POINT(41.01532426531973 73.43548891486651) bank105856 +105857 POINT(40.04981979339967 73.55099510027287) bank105857 +105858 POINT(39.87794821602479 73.88447428824249) bank105858 +105859 POINT(41.44269207749215 73.32717697582252) bank105859 +105860 POINT(40.045584449704904 73.79195895953906) bank105860 +105861 POINT(41.685814306308 74.28847779923834) bank105861 +105862 POINT(41.47909283367514 74.74974871551554) bank105862 +105863 POINT(41.219295285162865 74.62931726847354) bank105863 +105864 POINT(40.667342917499994 74.14124109260504) bank105864 +105865 POINT(40.661528589256875 73.8211240580736) bank105865 +105866 POINT(40.666807552138906 73.33618275999449) bank105866 +105867 POINT(41.29280639319529 73.12414563993339) bank105867 +105868 POINT(40.11047875394017 74.93050535028414) bank105868 +105869 POINT(40.50414879254795 73.68303268867857) bank105869 +105870 POINT(40.24437812300875 73.88956032082208) bank105870 +105871 POINT(41.31564475462824 74.44399261747894) bank105871 +105872 POINT(41.111251330547404 74.84477907000651) bank105872 +105873 POINT(41.58759173806792 73.41361748114001) bank105873 +105874 POINT(41.55023916646936 73.02706769616275) bank105874 +105875 POINT(40.83579547612846 73.01560446300618) bank105875 +105876 POINT(40.24427889124243 73.9460547511374) bank105876 +105877 POINT(40.68656205886029 73.50775512978983) bank105877 +105878 POINT(40.343243381523834 73.70665430506943) bank105878 +105879 POINT(41.087024862278035 74.03509132234421) bank105879 +105880 POINT(40.39317193568805 74.51805495566406) bank105880 +105881 POINT(40.114450516376856 73.76742446290694) bank105881 +105882 POINT(40.62546250031912 73.91807596725256) bank105882 +105883 POINT(41.66269897202113 74.45058881244142) bank105883 +105884 POINT(40.848695055347186 73.44604588177016) bank105884 +105885 POINT(39.96367211734501 74.33455200188442) bank105885 +105886 POINT(40.17301073427798 73.15385309211132) bank105886 +105887 POINT(39.726908286010044 73.18114033756592) bank105887 +105888 POINT(40.157791462801185 74.13238495926653) bank105888 +105889 POINT(40.2089498277849 73.16344700668392) bank105889 +105890 POINT(40.54194281035409 73.62100608969043) bank105890 +105891 POINT(40.648327249987936 74.84605707891612) bank105891 +105892 POINT(41.16386505519831 73.75777805448342) bank105892 +105893 POINT(39.87906019022858 73.7421317888369) bank105893 +105894 POINT(41.63747024909215 73.99426512417008) bank105894 +105895 POINT(41.70648678285971 74.6290898188024) bank105895 +105896 POINT(41.08089448711476 73.89258248972074) bank105896 +105897 POINT(40.88402135577872 73.25884675896538) bank105897 +105898 POINT(40.66027755259849 73.55253636785403) bank105898 +105899 POINT(40.27430944794273 74.7718907949133) bank105899 +105900 POINT(40.03257784370834 74.67454849630028) bank105900 +105901 POINT(40.02292404079509 74.88028641655036) bank105901 +105902 POINT(40.07546200411998 74.89147543779615) bank105902 +105903 POINT(40.153518013036546 74.2870296238838) bank105903 +105904 POINT(39.990858634040876 74.66235413264745) bank105904 +105905 POINT(40.62695833863139 74.03437258365804) bank105905 +105906 POINT(40.71699689962883 74.78932241530772) bank105906 +105907 POINT(40.979058323770374 74.65233305978049) bank105907 +105908 POINT(41.07674062001635 73.24078438840326) bank105908 +105909 POINT(40.26041504101466 74.22412129386171) bank105909 +105910 POINT(40.73952493119443 74.28056775925681) bank105910 +105911 POINT(41.23905365767 73.8101206364204) bank105911 +105912 POINT(40.42837659115834 73.52502370097693) bank105912 +105913 POINT(40.92890978737477 74.16260366344116) bank105913 +105914 POINT(39.788124018736475 74.28074812452596) bank105914 +105915 POINT(41.464134084168755 74.32145526182238) bank105915 +105916 POINT(40.33698779938341 74.835985715536) bank105916 +105917 POINT(41.23420961618606 74.44421347755238) bank105917 +105918 POINT(40.020626857736836 74.7975626282656) bank105918 +105919 POINT(41.18123212814702 74.45625263450921) bank105919 +105920 POINT(41.63510425521055 73.86468203098747) bank105920 +105921 POINT(40.59017856465073 74.08703576699789) bank105921 +105922 POINT(40.21466697868876 74.40906614101836) bank105922 +105923 POINT(39.91586557138927 73.20227121652896) bank105923 +105924 POINT(41.68565952960809 73.72537365246086) bank105924 +105925 POINT(40.439802426006544 73.34752002022641) bank105925 +105926 POINT(41.27173397308722 74.34947606174029) bank105926 +105927 POINT(40.755730412012376 74.38706020836607) bank105927 +105928 POINT(41.57999969142234 74.99266855238977) bank105928 +105929 POINT(41.381510431407776 74.56463205928385) bank105929 +105930 POINT(41.20713201604295 73.62923649619472) bank105930 +105931 POINT(40.76134978769193 73.53878526717952) bank105931 +105932 POINT(40.97191722123484 74.66892979445045) bank105932 +105933 POINT(41.223306025047506 74.48383049318895) bank105933 +105934 POINT(40.50867771942108 74.66964629504585) bank105934 +105935 POINT(40.07519959487787 74.3320598032256) bank105935 +105936 POINT(41.61837320757416 73.66323533024956) bank105936 +105937 POINT(40.74916133022663 73.07861240707356) bank105937 +105938 POINT(41.52982869372686 73.21639863337097) bank105938 +105939 POINT(40.17548184663025 73.37992156902892) bank105939 +105940 POINT(40.72574764834748 74.59230144084546) bank105940 +105941 POINT(40.52554006287801 74.02688608856869) bank105941 +105942 POINT(40.376591159582716 73.81328548534424) bank105942 +105943 POINT(40.71426064810522 74.41872163175346) bank105943 +105944 POINT(41.21985507150197 74.61944485696003) bank105944 +105945 POINT(40.448570319067485 74.07765365664534) bank105945 +105946 POINT(40.75210322002703 74.25885534919233) bank105946 +105947 POINT(40.00733233297117 73.60266144406404) bank105947 +105948 POINT(40.18169303736449 73.86471224660463) bank105948 +105949 POINT(41.40803679333181 73.77365800495157) bank105949 +105950 POINT(40.9170334970728 73.27208466548606) bank105950 +105951 POINT(39.87642599361053 74.18207773449568) bank105951 +105952 POINT(40.72741688865664 74.09191991204393) bank105952 +105953 POINT(40.33532583709389 73.03412609900643) bank105953 +105954 POINT(39.763495275499224 73.58844164080227) bank105954 +105955 POINT(41.47656337830967 74.35205270017211) bank105955 +105956 POINT(40.503142625974064 73.48534885810498) bank105956 +105957 POINT(39.75444234015855 73.61543678709197) bank105957 +105958 POINT(40.84140369402925 73.7230743339416) bank105958 +105959 POINT(40.00017828380382 74.89385789561159) bank105959 +105960 POINT(39.9540748235123 73.21145661773713) bank105960 +105961 POINT(40.586935951901104 74.1929804904136) bank105961 +105962 POINT(39.74126488164878 74.75883393136729) bank105962 +105963 POINT(41.58552541540205 73.07578993941456) bank105963 +105964 POINT(39.86893882373856 74.60127453462528) bank105964 +105965 POINT(41.40182513504262 74.00364169241138) bank105965 +105966 POINT(40.8251998827782 73.82972155139642) bank105966 +105967 POINT(41.65459411028826 73.68335312636967) bank105967 +105968 POINT(39.87125967063531 73.17347607128136) bank105968 +105969 POINT(41.5217201778401 74.49588405106068) bank105969 +105970 POINT(40.72873041277761 74.52674526321638) bank105970 +105971 POINT(41.051021091783134 73.2798926045105) bank105971 +105972 POINT(41.24326304047293 73.14844607535368) bank105972 +105973 POINT(41.51393977554392 73.94582499094224) bank105973 +105974 POINT(40.458207006039785 73.63035135567723) bank105974 +105975 POINT(41.00290463366097 74.316293006621) bank105975 +105976 POINT(39.978694477073255 74.29511597809214) bank105976 +105977 POINT(41.61401017740378 73.23068963708755) bank105977 +105978 POINT(41.12711058502865 73.53048297460887) bank105978 +105979 POINT(40.28207343361929 73.09657310043208) bank105979 +105980 POINT(39.87355227334746 74.3108240906692) bank105980 +105981 POINT(41.636289093746896 74.05769872420206) bank105981 +105982 POINT(41.46061304923135 74.98386052917756) bank105982 +105983 POINT(41.67391807293415 73.30866579386077) bank105983 +105984 POINT(41.32390391334409 74.87756286187089) bank105984 +105985 POINT(40.90717010421232 73.66499335958174) bank105985 +105986 POINT(40.15329244701915 73.82668434317422) bank105986 +105987 POINT(40.59112970622928 74.56952582597712) bank105987 +105988 POINT(40.93918444856771 74.99516421936825) bank105988 +105989 POINT(41.38224728538198 74.91927644247231) bank105989 +105990 POINT(39.85472080381282 74.14326858087215) bank105990 +105991 POINT(41.49480411268761 73.67986201862098) bank105991 +105992 POINT(41.12116503900499 74.36241551151662) bank105992 +105993 POINT(40.73402840335133 74.11156956606354) bank105993 +105994 POINT(41.34940126085463 73.57732065551426) bank105994 +105995 POINT(40.18853498969743 73.47699155826433) bank105995 +105996 POINT(39.82154942141505 73.09184967068438) bank105996 +105997 POINT(40.78581706693803 74.23867903522357) bank105997 +105998 POINT(40.2294916143145 74.59766955732768) bank105998 +105999 POINT(40.494034895980676 73.173390500364) bank105999 +106000 POINT(40.42712322331504 74.99997984407491) bank106000 +106001 POINT(39.75124709723886 73.20387926780778) bank106001 +106002 POINT(41.01586044173906 74.80851915732745) bank106002 +106003 POINT(41.44639805543858 74.37363922519275) bank106003 +106004 POINT(41.623629514106824 73.27562087422314) bank106004 +106005 POINT(41.4654336054205 74.66397867815763) bank106005 +106006 POINT(41.31162417937709 74.79259147205867) bank106006 +106007 POINT(39.975331854776435 74.46886218370058) bank106007 +106008 POINT(41.63934095674828 74.53206303150868) bank106008 +106009 POINT(40.222986375838715 73.4431104041729) bank106009 +106010 POINT(41.01823178248544 74.60738452794111) bank106010 +106011 POINT(41.62349916752164 74.51195354452904) bank106011 +106012 POINT(40.723207284445024 74.71915736455942) bank106012 +106013 POINT(40.12758637685709 74.60999176298446) bank106013 +106014 POINT(40.01828734232891 74.13111611169224) bank106014 +106015 POINT(41.4090189806146 73.27986015071701) bank106015 +106016 POINT(39.73484315722786 74.40311956612311) bank106016 +106017 POINT(41.10067149051414 74.26083601734724) bank106017 +106018 POINT(40.24156850278884 74.51614111794547) bank106018 +106019 POINT(41.294231373236904 73.47559062037088) bank106019 +106020 POINT(40.8855780770959 73.34636619702111) bank106020 +106021 POINT(39.946699492258176 74.48980180779559) bank106021 +106022 POINT(40.62553273918123 73.74361419528223) bank106022 +106023 POINT(41.00503107028909 73.07462392228952) bank106023 +106024 POINT(40.95141870435149 74.34734174297954) bank106024 +106025 POINT(40.940403446019744 74.67491795070762) bank106025 +106026 POINT(40.89760165203974 74.90038240674427) bank106026 +106027 POINT(40.778971468642276 73.59715363614771) bank106027 +106028 POINT(41.148028255152056 73.24507494908896) bank106028 +106029 POINT(40.83465250476412 73.20400852510134) bank106029 +106030 POINT(41.136617151231775 74.46145399021083) bank106030 +106031 POINT(40.90736646137729 73.18081041468349) bank106031 +106032 POINT(39.85107187347223 74.23885632901198) bank106032 +106033 POINT(39.946690501432975 73.68172066221645) bank106033 +106034 POINT(40.26458324932103 74.59297937737689) bank106034 +106035 POINT(39.90533588893763 73.91318264394533) bank106035 +106036 POINT(39.915483208661094 74.40150978330296) bank106036 +106037 POINT(40.322782743056706 74.08238200027168) bank106037 +106038 POINT(40.361726820885366 74.03777749816075) bank106038 +106039 POINT(40.66715527735933 74.48847270420997) bank106039 +106040 POINT(40.473122667868104 74.22748855553753) bank106040 +106041 POINT(40.987823900144775 74.53598279610739) bank106041 +106042 POINT(40.32229017151578 73.00788652932158) bank106042 +106043 POINT(41.49239312375208 74.24490043812492) bank106043 +106044 POINT(40.783874037581285 74.09096451233572) bank106044 +106045 POINT(40.25736688458444 74.96231605244805) bank106045 +106046 POINT(41.42661059306243 74.20231096816879) bank106046 +106047 POINT(40.013148340032274 74.49870196108778) bank106047 +106048 POINT(40.232508679689616 73.40728470633194) bank106048 +106049 POINT(39.9659572719611 74.43116277474424) bank106049 +106050 POINT(40.821710886363796 74.38370378297017) bank106050 +106051 POINT(39.9954765389291 73.61075395472365) bank106051 +106052 POINT(41.414629824970575 74.54424116423522) bank106052 +106053 POINT(40.824214731571935 73.29233722752524) bank106053 +106054 POINT(40.33192606565582 74.46341597044469) bank106054 +106055 POINT(41.49464012497012 73.05245783671839) bank106055 +106056 POINT(39.85492788378257 74.33118606472925) bank106056 +106057 POINT(41.10516920798894 73.2598333286841) bank106057 +106058 POINT(41.27736630053511 75.00068858681365) bank106058 +106059 POINT(40.99272548738919 73.54755918085323) bank106059 +106060 POINT(41.38616665461485 74.47279572622163) bank106060 +106061 POINT(40.45180964185714 74.33357447557825) bank106061 +106062 POINT(40.99022933585891 74.6104339837113) bank106062 +106063 POINT(40.236440036346316 74.08225740877836) bank106063 +106064 POINT(41.3773797621807 74.88976905160929) bank106064 +106065 POINT(41.465679907436794 73.18131227598778) bank106065 +106066 POINT(40.757791558361504 74.83235844729933) bank106066 +106067 POINT(40.89059632851625 74.61175287721275) bank106067 +106068 POINT(41.392246917190334 73.75999125429306) bank106068 +106069 POINT(40.19715165614069 74.16276819144073) bank106069 +106070 POINT(40.38801889709294 74.4112231766526) bank106070 +106071 POINT(41.40292618627664 73.85662893003617) bank106071 +106072 POINT(41.65367526865049 74.73328867287323) bank106072 +106073 POINT(40.46306641259772 73.93157579667185) bank106073 +106074 POINT(41.34760308785443 73.0478632222922) bank106074 +106075 POINT(40.09183033761913 73.64336566425061) bank106075 +106076 POINT(40.062624551786165 73.85610239071028) bank106076 +106077 POINT(40.84854667770418 74.53756532894117) bank106077 +106078 POINT(40.533717013872625 73.7179441657639) bank106078 +106079 POINT(41.05594544783532 74.45774402921306) bank106079 +106080 POINT(40.050934570901624 74.31584333585533) bank106080 +106081 POINT(41.46130627694692 73.04609171163028) bank106081 +106082 POINT(41.47945903971579 74.59583991498704) bank106082 +106083 POINT(41.31570150891993 73.05413369176536) bank106083 +106084 POINT(40.86200809646873 74.39029448954899) bank106084 +106085 POINT(41.67952440395205 74.24785819677449) bank106085 +106086 POINT(40.248098191791094 74.39274217408627) bank106086 +106087 POINT(40.6480156401715 74.86503073880513) bank106087 +106088 POINT(39.81463693679023 74.44565375183261) bank106088 +106089 POINT(40.514241990573105 74.46697983765004) bank106089 +106090 POINT(40.18137748563193 74.39311200716507) bank106090 +106091 POINT(41.217707611034314 74.36116145597742) bank106091 +106092 POINT(39.73589175440392 74.7677512425967) bank106092 +106093 POINT(40.95548278286581 73.34830545870749) bank106093 +106094 POINT(41.58350565877313 74.04254535035079) bank106094 +106095 POINT(40.23178582495912 74.86923432231717) bank106095 +106096 POINT(39.74184604467909 73.1252412415419) bank106096 +106097 POINT(40.04625611399092 73.80695202869472) bank106097 +106098 POINT(40.18646762181995 74.30171926284352) bank106098 +106099 POINT(41.152138193808895 73.1775711419083) bank106099 +106100 POINT(41.5889702536429 73.24952919297226) bank106100 +106101 POINT(39.715287497407665 74.93843888502519) bank106101 +106102 POINT(41.62964585771244 74.84184439593324) bank106102 +106103 POINT(39.949912700263795 73.12479119007706) bank106103 +106104 POINT(41.35488264149069 73.82796517542869) bank106104 +106105 POINT(41.66147780064852 74.85795829640209) bank106105 +106106 POINT(41.38402822542378 74.49054212114727) bank106106 +106107 POINT(41.33253273655742 74.35369836421204) bank106107 +106108 POINT(41.333921903118394 74.95128731715627) bank106108 +106109 POINT(40.81623471600343 73.06553269779141) bank106109 +106110 POINT(40.1804143355719 74.56364688931927) bank106110 +106111 POINT(40.2769499418829 74.51948670460742) bank106111 +106112 POINT(39.71633062530913 73.64771455839585) bank106112 +106113 POINT(40.23874123489439 74.47143365117198) bank106113 +106114 POINT(41.42265468352602 73.4746921129141) bank106114 +106115 POINT(41.08534497652819 74.60354900939527) bank106115 +106116 POINT(41.217870884469306 74.95494395007424) bank106116 +106117 POINT(40.75297220350878 73.14256387330805) bank106117 +106118 POINT(40.20724229189491 74.52674598651922) bank106118 +106119 POINT(40.46930644015885 74.69445136443449) bank106119 +106120 POINT(40.178964980811834 73.82069420604334) bank106120 +106121 POINT(39.81010427536233 74.81029941981295) bank106121 +106122 POINT(40.38110184554669 73.63585542738755) bank106122 +106123 POINT(39.986395071729035 74.14237814960055) bank106123 +106124 POINT(40.58213399413845 73.69363760357272) bank106124 +106125 POINT(41.67380488944186 73.49062103520808) bank106125 +106126 POINT(40.80524399180156 73.21636226930032) bank106126 +106127 POINT(39.80275992209717 73.88105486424021) bank106127 +106128 POINT(40.768629760262286 73.28504397339704) bank106128 +106129 POINT(39.93520698473811 73.1513859262854) bank106129 +106130 POINT(41.50478688537219 73.48590487635379) bank106130 +106131 POINT(41.06791088959296 74.78864072100399) bank106131 +106132 POINT(40.51040218205621 74.39991697052763) bank106132 +106133 POINT(41.43394568512381 73.18806187666463) bank106133 +106134 POINT(40.751929616147116 73.1013975906724) bank106134 +106135 POINT(40.70986482373445 73.60302445922488) bank106135 +106136 POINT(40.07217037675685 74.99231927448142) bank106136 +106137 POINT(41.32478723060162 74.3155601123527) bank106137 +106138 POINT(41.175802544724434 74.8943270690196) bank106138 +106139 POINT(40.25778553779594 74.48756641470095) bank106139 +106140 POINT(40.134365096205705 73.96673214442089) bank106140 +106141 POINT(40.25948161513772 73.84922534473031) bank106141 +106142 POINT(40.727060235705856 74.44006340641634) bank106142 +106143 POINT(41.258456804818614 74.5178365589302) bank106143 +106144 POINT(39.75538362819735 73.91771641631135) bank106144 +106145 POINT(40.8033741260442 74.31449028553848) bank106145 +106146 POINT(39.81342165586229 74.3765846243774) bank106146 +106147 POINT(40.36461618237735 73.78471179898266) bank106147 +106148 POINT(39.84585883622825 73.59207713891895) bank106148 +106149 POINT(40.53092019792719 73.72221575754966) bank106149 +106150 POINT(40.47659043801614 74.41303687940515) bank106150 +106151 POINT(40.715857908481915 74.29731788669713) bank106151 +106152 POINT(40.63867136389498 74.94430554366883) bank106152 +106153 POINT(40.942148116143315 73.92486854428768) bank106153 +106154 POINT(40.42622589686724 74.45646734560206) bank106154 +106155 POINT(40.73177851755628 74.76075328863395) bank106155 +106156 POINT(40.172061126999445 73.87586930437648) bank106156 +106157 POINT(41.62460142368364 73.78175983730891) bank106157 +106158 POINT(41.27469446805457 74.09722860758703) bank106158 +106159 POINT(40.84603144531353 73.71849462665523) bank106159 +106160 POINT(41.15511803820056 74.13651514503856) bank106160 +106161 POINT(40.71541020328964 74.35708360355149) bank106161 +106162 POINT(41.49811514848297 74.51342524304755) bank106162 +106163 POINT(39.97705780147147 74.21716444379697) bank106163 +106164 POINT(40.82893289417719 73.37931480291083) bank106164 +106165 POINT(40.75421311706068 74.53529657114562) bank106165 +106166 POINT(41.686645925993155 73.78527466038987) bank106166 +106167 POINT(40.09303484521205 74.53417970209722) bank106167 +106168 POINT(39.83885846773361 73.38891009968542) bank106168 +106169 POINT(40.991087820644 73.86960248041794) bank106169 +106170 POINT(40.82225325218034 74.38845958812774) bank106170 +106171 POINT(40.55128746801308 73.64677644180834) bank106171 +106172 POINT(40.73090305578876 74.11828633213602) bank106172 +106173 POINT(39.97655020801599 74.34344139960768) bank106173 +106174 POINT(41.61836725062959 73.03858326298793) bank106174 +106175 POINT(40.30379455105866 74.38956250343891) bank106175 +106176 POINT(39.90095106113409 73.30151983324984) bank106176 +106177 POINT(40.14572412913177 74.21374295564245) bank106177 +106178 POINT(40.978488666299135 74.35311187511384) bank106178 +106179 POINT(40.70919535080404 73.14743098959876) bank106179 +106180 POINT(40.37909016165698 74.37648198991461) bank106180 +106181 POINT(41.55886194659713 74.20842212447687) bank106181 +106182 POINT(40.22210611233543 73.78740101106644) bank106182 +106183 POINT(40.76173977549961 73.15051082038153) bank106183 +106184 POINT(40.27458853970283 74.32086668833567) bank106184 +106185 POINT(41.086599020874715 74.54829927005346) bank106185 +106186 POINT(40.09195531453719 74.80102956849333) bank106186 +106187 POINT(39.94738065991345 74.79248515936102) bank106187 +106188 POINT(40.56613897698932 74.16506981163589) bank106188 +106189 POINT(41.237959131766026 74.3743467291121) bank106189 +106190 POINT(41.38013556885976 74.81888934179555) bank106190 +106191 POINT(40.01590589600261 74.43802658636108) bank106191 +106192 POINT(39.92318472420341 74.41263720883919) bank106192 +106193 POINT(41.50370655371379 74.5383187324426) bank106193 +106194 POINT(41.3474907021353 73.18413028479226) bank106194 +106195 POINT(40.99356278608295 74.63794032575579) bank106195 +106196 POINT(40.573249538769986 73.11919512667716) bank106196 +106197 POINT(40.8156188095115 73.03376243502375) bank106197 +106198 POINT(41.27554367391682 74.64569279678604) bank106198 +106199 POINT(41.18517657877287 73.7005616434908) bank106199 +106200 POINT(40.78568671981806 73.2178642867388) bank106200 +106201 POINT(40.06519460038641 74.99401635977281) bank106201 +106202 POINT(40.21460177563146 74.77106212139824) bank106202 +106203 POINT(40.38025588031044 73.03806801031513) bank106203 +106204 POINT(39.96850751571886 74.72148019852413) bank106204 +106205 POINT(41.508744967002784 73.74017902940714) bank106205 +106206 POINT(41.67375580640227 73.94224943050126) bank106206 +106207 POINT(41.23467470094681 73.33458337708865) bank106207 +106208 POINT(40.07642591382249 73.56925553125129) bank106208 +106209 POINT(39.86022423431669 73.27400954353038) bank106209 +106210 POINT(40.13043272675378 73.75772031483778) bank106210 +106211 POINT(41.507639434658046 74.9576715077782) bank106211 +106212 POINT(40.14513254101762 73.41599953227124) bank106212 +106213 POINT(40.49825221525502 73.09206059723668) bank106213 +106214 POINT(40.558356068502384 73.38702306334419) bank106214 +106215 POINT(40.86834894960163 74.13415793975811) bank106215 +106216 POINT(41.39968050614679 73.13764423457505) bank106216 +106217 POINT(40.68991100000566 73.18120464699994) bank106217 +106218 POINT(40.13250919093781 74.45908595207064) bank106218 +106219 POINT(40.3707617603574 74.39351621650097) bank106219 +106220 POINT(41.67885664176381 73.56477986505183) bank106220 +106221 POINT(40.461039360211466 73.85968145697284) bank106221 +106222 POINT(40.52855020025852 73.72755472502796) bank106222 +106223 POINT(41.63497978660188 73.09056461044881) bank106223 +106224 POINT(40.03680867013605 74.35533989533228) bank106224 +106225 POINT(41.032604215281935 74.67646447100338) bank106225 +106226 POINT(40.94178007431779 74.99983161991895) bank106226 +106227 POINT(40.21835592422697 74.7952321282678) bank106227 +106228 POINT(41.36565589572408 73.31416132024401) bank106228 +106229 POINT(40.802687733570885 74.54185413642054) bank106229 +106230 POINT(40.0222452431374 74.80831159264041) bank106230 +106231 POINT(40.279825136191 73.10219637643281) bank106231 +106232 POINT(40.76339888274071 73.93279896235381) bank106232 +106233 POINT(40.637093843683616 74.45851138885791) bank106233 +106234 POINT(40.0709802226788 73.74113563611465) bank106234 +106235 POINT(41.188722672419296 74.88024125575693) bank106235 +106236 POINT(40.781645802758625 74.794338453738) bank106236 +106237 POINT(40.06529625122492 74.17685217843763) bank106237 +106238 POINT(39.999239860715 74.66767741919789) bank106238 +106239 POINT(40.31687098299787 73.58332589257627) bank106239 +106240 POINT(40.899371036417634 73.54779689510062) bank106240 +106241 POINT(41.69674520152537 73.53885297319775) bank106241 +106242 POINT(41.27334135402144 73.73426004891994) bank106242 +106243 POINT(40.901405281365314 74.09076699787764) bank106243 +106244 POINT(39.889195676757225 74.30096241433691) bank106244 +106245 POINT(40.26223200754959 73.38718858776701) bank106245 +106246 POINT(39.952598687871884 73.27872438125648) bank106246 +106247 POINT(41.0626204325368 74.91403695313386) bank106247 +106248 POINT(40.29357118442129 74.1586992695272) bank106248 +106249 POINT(40.528596698443096 74.62327845057051) bank106249 +106250 POINT(40.63394273048211 74.94972002306469) bank106250 +106251 POINT(41.62525587004488 73.53094976411153) bank106251 +106252 POINT(40.39331690485861 73.56966469626899) bank106252 +106253 POINT(39.84424960412308 73.56961603385567) bank106253 +106254 POINT(39.71448779054703 74.48871666600964) bank106254 +106255 POINT(39.882893059311186 74.41199073106338) bank106255 +106256 POINT(40.140903079498244 73.42941944651925) bank106256 +106257 POINT(41.392501466085314 74.36958433368021) bank106257 +106258 POINT(41.009927272566465 73.99543454718575) bank106258 +106259 POINT(40.18197465014889 74.79159758897079) bank106259 +106260 POINT(41.663859751282295 73.95901118270848) bank106260 +106261 POINT(39.79428738985828 74.4006272750551) bank106261 +106262 POINT(40.058023182837104 73.43706978365574) bank106262 +106263 POINT(40.821441073478674 73.21652008204032) bank106263 +106264 POINT(40.99867715537362 74.2710844759667) bank106264 +106265 POINT(40.25988729037092 73.10438553873122) bank106265 +106266 POINT(39.967007497662664 74.09811426560276) bank106266 +106267 POINT(41.23933845090471 74.95352670660068) bank106267 +106268 POINT(40.55534021649535 74.19544618114385) bank106268 +106269 POINT(41.08016464678591 73.07934352151253) bank106269 +106270 POINT(40.69303670420161 73.28344967775851) bank106270 +106271 POINT(40.144382428088456 73.53565093056272) bank106271 +106272 POINT(40.19643640136722 73.55743386873989) bank106272 +106273 POINT(41.586801384230675 74.95416639668953) bank106273 +106274 POINT(39.71987067132096 73.99123664683802) bank106274 +106275 POINT(40.984634079092004 73.27991252798117) bank106275 +106276 POINT(41.49367260945387 73.81606967651007) bank106276 +106277 POINT(40.33786775733979 74.63987873430665) bank106277 +106278 POINT(39.72980326370316 73.94450950266015) bank106278 +106279 POINT(39.87934849747357 74.04381853979052) bank106279 +106280 POINT(40.47522505499766 73.6216991705246) bank106280 +106281 POINT(40.2629079220981 74.00927629470775) bank106281 +106282 POINT(40.329507659921674 74.27677081066952) bank106282 +106283 POINT(41.660655542052794 73.48562693591754) bank106283 +106284 POINT(41.10302200858324 74.93272828388712) bank106284 +106285 POINT(40.84530872432954 73.89639868963195) bank106285 +106286 POINT(41.1667746066807 74.28405304403347) bank106286 +106287 POINT(41.44541189429345 74.6242966995013) bank106287 +106288 POINT(39.71956175101709 73.1799831917612) bank106288 +106289 POINT(39.92116082038239 73.62453195325385) bank106289 +106290 POINT(41.409269599815964 74.82290478681206) bank106290 +106291 POINT(39.77688715795148 74.6022733156328) bank106291 +106292 POINT(41.17758819332156 73.36608710600538) bank106292 +106293 POINT(40.73517213382161 74.47231843049784) bank106293 +106294 POINT(41.01788585404423 74.45768852037008) bank106294 +106295 POINT(40.15396596572874 73.61355308019164) bank106295 +106296 POINT(41.43557304096086 74.88304409320253) bank106296 +106297 POINT(40.00350724602417 73.24340181987522) bank106297 +106298 POINT(40.021698131479454 74.53406085403455) bank106298 +106299 POINT(40.35929524378557 74.81301574409993) bank106299 +106300 POINT(41.314743379408135 74.00143681403108) bank106300 +106301 POINT(41.597247005587256 74.55704337543946) bank106301 +106302 POINT(41.15644666643483 74.26212748077313) bank106302 +106303 POINT(40.60234999885464 74.76929311194785) bank106303 +106304 POINT(40.919587543049886 73.0120132966282) bank106304 +106305 POINT(41.1162559370067 73.97091069785311) bank106305 +106306 POINT(39.775957748929756 74.11523327779905) bank106306 +106307 POINT(41.133494713187005 73.5861331606287) bank106307 +106308 POINT(40.319059133505796 73.56424355521749) bank106308 +106309 POINT(41.297608081065405 73.14933572314224) bank106309 +106310 POINT(41.59406351998029 73.05581863494547) bank106310 +106311 POINT(39.87126101930474 74.11367687478129) bank106311 +106312 POINT(40.87959763256574 73.5530754852898) bank106312 +106313 POINT(41.30668043998678 73.26628578246017) bank106313 +106314 POINT(40.08847607349368 74.84865444693885) bank106314 +106315 POINT(40.359058008306505 73.48053585484168) bank106315 +106316 POINT(39.73132424217985 74.48090983261714) bank106316 +106317 POINT(40.012425812877545 73.98091774750665) bank106317 +106318 POINT(39.97629310490351 74.45859761755489) bank106318 +106319 POINT(39.862736213894614 73.63823609969457) bank106319 +106320 POINT(40.5588559251353 73.60641949318232) bank106320 +106321 POINT(39.759868217722016 74.85838378880837) bank106321 +106322 POINT(39.96490643952734 73.77272300543386) bank106322 +106323 POINT(41.030949274779886 74.33046371237224) bank106323 +106324 POINT(40.002418947176174 73.66813236910752) bank106324 +106325 POINT(41.24242422059545 73.8039883909727) bank106325 +106326 POINT(39.96351370062874 73.74600655231806) bank106326 +106327 POINT(39.835843558855665 74.58778201154354) bank106327 +106328 POINT(40.46214325286025 73.95751753092817) bank106328 +106329 POINT(41.21025627312306 73.56945234508935) bank106329 +106330 POINT(39.84824298569556 73.07505205913785) bank106330 +106331 POINT(41.23024327632559 74.9556401820692) bank106331 +106332 POINT(41.455366260850674 74.90600630221468) bank106332 +106333 POINT(40.37829873867373 74.44707561714036) bank106333 +106334 POINT(41.22270863700597 74.55592190538263) bank106334 +106335 POINT(40.54024704725502 73.66318299929777) bank106335 +106336 POINT(40.53703876848696 73.21569485517851) bank106336 +106337 POINT(40.0204885484311 74.95399123872095) bank106337 +106338 POINT(40.49161852282336 73.66093612330462) bank106338 +106339 POINT(40.58310300376296 73.71106209874357) bank106339 +106340 POINT(40.03976206259402 73.347824507534) bank106340 +106341 POINT(40.84183179784398 73.902966854675) bank106341 +106342 POINT(40.95767800690858 74.69965182027255) bank106342 +106343 POINT(41.59140442001237 74.57472544747961) bank106343 +106344 POINT(40.64410713970713 74.20394240119145) bank106344 +106345 POINT(40.64850553335386 73.96633713366211) bank106345 +106346 POINT(41.094649782600584 73.99868605468905) bank106346 +106347 POINT(41.422471157024845 74.19421885450686) bank106347 +106348 POINT(40.93493057738181 73.72191500822704) bank106348 +106349 POINT(39.89243061097731 74.00871239706906) bank106349 +106350 POINT(40.50308809176906 73.49275688231857) bank106350 +106351 POINT(40.24248464203013 74.25537313710537) bank106351 +106352 POINT(41.04809561248612 73.53598421545935) bank106352 +106353 POINT(41.056586733449215 73.82923818390802) bank106353 +106354 POINT(39.77302332955575 74.13279403514647) bank106354 +106355 POINT(40.31926434939865 73.1914648877624) bank106355 +106356 POINT(41.14739575484495 74.87237813807641) bank106356 +106357 POINT(39.9770792802969 73.4413920951366) bank106357 +106358 POINT(41.13004791904688 74.86533838278292) bank106358 +106359 POINT(40.10391737281415 74.6950312231304) bank106359 +106360 POINT(41.358188664041776 74.23013264707635) bank106360 +106361 POINT(39.96866278234697 73.7709166521593) bank106361 +106362 POINT(40.037435255690134 73.93503546221645) bank106362 +106363 POINT(41.27542530281939 73.22853224085782) bank106363 +106364 POINT(41.67864465342018 73.99595285604491) bank106364 +106365 POINT(41.396847315473245 74.43623429710735) bank106365 +106366 POINT(40.258731131598466 74.00997473972706) bank106366 +106367 POINT(40.093226158927585 73.04015075965823) bank106367 +106368 POINT(40.82279553059018 74.02833122124784) bank106368 +106369 POINT(41.45007894144151 73.92791892283553) bank106369 +106370 POINT(40.186974252054846 73.32651561207184) bank106370 +106371 POINT(40.53911516033818 74.09279087862187) bank106371 +106372 POINT(40.24446956334096 74.8657529331409) bank106372 +106373 POINT(41.45414248051754 73.26256970715394) bank106373 +106374 POINT(40.11254331032169 73.08726071861696) bank106374 +106375 POINT(40.99714028277775 74.64009798187304) bank106375 +106376 POINT(40.640200176827825 74.71062753756084) bank106376 +106377 POINT(41.4627884812551 74.79489642728493) bank106377 +106378 POINT(40.88952994200003 74.966817494706) bank106378 +106379 POINT(40.098945762369084 74.08481599553777) bank106379 +106380 POINT(41.52353108264957 73.2005698524613) bank106380 +106381 POINT(40.0286905368707 73.42903333000345) bank106381 +106382 POINT(41.65047800160491 74.40831691243335) bank106382 +106383 POINT(41.54071046164998 73.55632652777922) bank106383 +106384 POINT(40.58285786149472 73.44257665765178) bank106384 +106385 POINT(39.96034593280628 74.51635137233114) bank106385 +106386 POINT(40.362896243523195 74.46351365368334) bank106386 +106387 POINT(41.061228674090785 74.61458436555708) bank106387 +106388 POINT(39.857145533316796 74.86037495662804) bank106388 +106389 POINT(40.139033022654196 74.2662171042034) bank106389 +106390 POINT(39.74501576190097 73.53314456689196) bank106390 +106391 POINT(41.31652288120251 74.97620575993135) bank106391 +106392 POINT(41.605616386708334 73.7472628726776) bank106392 +106393 POINT(41.15168460449138 74.0131383841761) bank106393 +106394 POINT(39.81538850984601 73.58668370653953) bank106394 +106395 POINT(39.832794544810184 73.74131528439294) bank106395 +106396 POINT(39.919322502127535 73.67850247917409) bank106396 +106397 POINT(39.99947170528751 74.45272866000653) bank106397 +106398 POINT(41.474838249507066 74.15960311515553) bank106398 +106399 POINT(40.05648773959882 74.78279949655982) bank106399 +106400 POINT(40.513816568815635 73.6004366392741) bank106400 +106401 POINT(40.66468900391784 73.8729599013522) bank106401 +106402 POINT(41.531405935331485 73.67289080684901) bank106402 +106403 POINT(41.33893831710232 73.69978462758674) bank106403 +106404 POINT(40.36683717748192 73.44528762405699) bank106404 +106405 POINT(40.501753428013906 73.60585369399058) bank106405 +106406 POINT(41.03078675428021 73.46351048815916) bank106406 +106407 POINT(40.528366148428276 73.9249616266569) bank106407 +106408 POINT(41.593986633555296 73.8578387037786) bank106408 +106409 POINT(41.16951907619013 73.85040565106722) bank106409 +106410 POINT(41.69947822664031 74.3473485730924) bank106410 +106411 POINT(40.43064790701638 73.32850103092657) bank106411 +106412 POINT(41.17474677462452 74.01871216475722) bank106412 +106413 POINT(41.14339030301741 73.62476703254494) bank106413 +106414 POINT(41.081510510058095 74.80266869826153) bank106414 +106415 POINT(41.439336384578745 73.56852765949681) bank106415 +106416 POINT(40.67100291770166 73.4501714453823) bank106416 +106417 POINT(40.921368738535264 74.18504630696056) bank106417 +106418 POINT(41.373853318619595 73.49210016603797) bank106418 +106419 POINT(40.995279765257145 74.45402896867171) bank106419 +106420 POINT(40.70717519232252 73.87321676828799) bank106420 +106421 POINT(41.336970311604524 74.38582696714985) bank106421 +106422 POINT(39.766365727786194 73.60916686276586) bank106422 +106423 POINT(41.28915706411854 73.79438286432904) bank106423 +106424 POINT(41.36723272633239 73.66433860988093) bank106424 +106425 POINT(39.73957673537393 73.61184600365812) bank106425 +106426 POINT(39.916241046971216 74.44805151308947) bank106426 +106427 POINT(41.14823748159724 74.10565694107046) bank106427 +106428 POINT(40.19568823930577 74.58938066402033) bank106428 +106429 POINT(40.1223833487389 73.29319144459322) bank106429 +106430 POINT(40.233081320552394 74.20509720143554) bank106430 +106431 POINT(40.70544741852632 74.44909590187153) bank106431 +106432 POINT(40.72582473367452 74.46503630414476) bank106432 +106433 POINT(40.40420886011007 73.86194762400217) bank106433 +106434 POINT(40.34876362878044 73.46889262926284) bank106434 +106435 POINT(39.931456619500715 74.58713632526276) bank106435 +106436 POINT(41.00782732079489 74.89164674275425) bank106436 +106437 POINT(40.153175180896724 74.87361780977226) bank106437 +106438 POINT(40.633822180437306 74.76797855068983) bank106438 +106439 POINT(40.83534136015135 74.50340630176845) bank106439 +106440 POINT(41.591242392442545 74.43399362626448) bank106440 +106441 POINT(41.284006854238285 73.64025745316818) bank106441 +106442 POINT(39.76577744353408 74.63084659283604) bank106442 +106443 POINT(40.08868526914344 74.55529228573175) bank106443 +106444 POINT(40.21359685106883 74.48664134541626) bank106444 +106445 POINT(39.92307572915672 73.24463561759617) bank106445 +106446 POINT(40.203491703570016 74.57463749430651) bank106446 +106447 POINT(41.3546653981507 74.76575141868588) bank106447 +106448 POINT(39.80402077345867 74.41308309999057) bank106448 +106449 POINT(40.269827609275715 73.74387035888499) bank106449 +106450 POINT(41.65030354537086 73.54824701227636) bank106450 +106451 POINT(39.975283804210534 73.1286027780508) bank106451 +106452 POINT(40.328910946944 74.31547160920832) bank106452 +106453 POINT(40.81763975825938 74.16637341625896) bank106453 +106454 POINT(39.78668154223314 74.50078997693853) bank106454 +106455 POINT(41.492160078566705 74.67528482180764) bank106455 +106456 POINT(40.47064093598465 74.66996786246061) bank106456 +106457 POINT(40.90182799907629 73.99657274876884) bank106457 +106458 POINT(40.27953554571065 73.4429376355383) bank106458 +106459 POINT(41.05236329449266 73.92692167058037) bank106459 +106460 POINT(40.72299492611068 73.84197214062459) bank106460 +106461 POINT(41.36557553573329 74.76702895897523) bank106461 +106462 POINT(40.69607462290069 73.06023687484556) bank106462 +106463 POINT(39.73240713029646 74.78705122603805) bank106463 +106464 POINT(41.60384277697548 74.09970004809588) bank106464 +106465 POINT(41.501333472829884 74.3848408362763) bank106465 +106466 POINT(39.83804079465488 73.81871992308861) bank106466 +106467 POINT(39.981993744872234 74.50076150281006) bank106467 +106468 POINT(40.460436802283986 73.61886768864231) bank106468 +106469 POINT(40.99602349675484 73.9501962717124) bank106469 +106470 POINT(40.465257856901715 74.62269700644462) bank106470 +106471 POINT(41.62549088459942 74.11678763467995) bank106471 +106472 POINT(40.089190696259365 73.63194833731025) bank106472 +106473 POINT(41.373854166806005 74.94900927572351) bank106473 +106474 POINT(40.23464399340149 73.28626691121157) bank106474 +106475 POINT(39.78397607609717 74.45255784345052) bank106475 +106476 POINT(41.09621149040011 74.617748356321) bank106476 +106477 POINT(41.27683795683256 74.08192707785163) bank106477 +106478 POINT(41.221230114187854 73.88958601081812) bank106478 +106479 POINT(40.43173383558392 74.03760999298709) bank106479 +106480 POINT(40.748808111230474 74.98998115846922) bank106480 +106481 POINT(41.30515690234357 74.21003887950194) bank106481 +106482 POINT(39.77618650814345 73.25773662617773) bank106482 +106483 POINT(41.41982304507053 74.63496901675121) bank106483 +106484 POINT(39.90044446997958 74.97750251911948) bank106484 +106485 POINT(41.53135433473659 74.64149790032768) bank106485 +106486 POINT(40.132293251581814 74.52314850056405) bank106486 +106487 POINT(41.344097201954014 73.23805612105923) bank106487 +106488 POINT(41.268596974025705 73.11764832849927) bank106488 +106489 POINT(41.53791395570399 73.07622213068012) bank106489 +106490 POINT(41.27133838292786 73.97907705590274) bank106490 +106491 POINT(41.44961202365291 74.02314072998584) bank106491 +106492 POINT(39.798075386578965 74.98257724987427) bank106492 +106493 POINT(39.73650120319578 73.36720084407766) bank106493 +106494 POINT(40.56109900954543 74.04264619499605) bank106494 +106495 POINT(41.48449055794918 73.28801573796663) bank106495 +106496 POINT(40.876290034784724 74.54754430802319) bank106496 +106497 POINT(40.363131892811 73.48268550349574) bank106497 +106498 POINT(39.983882483651044 73.26739722592347) bank106498 +106499 POINT(40.84385112256313 73.05001527175395) bank106499 +106500 POINT(39.72795013272963 73.83284316491759) bank106500 +106501 POINT(41.33578988656566 73.71221198039217) bank106501 +106502 POINT(41.187831566581934 73.8206902547285) bank106502 +106503 POINT(41.350158597871356 74.66505726683276) bank106503 +106504 POINT(40.295399302351825 74.82233321143805) bank106504 +106505 POINT(40.38821071885044 73.93121832221959) bank106505 +106506 POINT(40.21444086820183 74.12176407007433) bank106506 +106507 POINT(39.79632110395189 73.65389906458233) bank106507 +106508 POINT(40.94386814026908 74.88132121724522) bank106508 +106509 POINT(39.817273361196825 74.78001923892688) bank106509 +106510 POINT(39.960902067939735 73.20867788410918) bank106510 +106511 POINT(41.272313790415545 74.58183498124505) bank106511 +106512 POINT(40.22772163506148 74.91219878301015) bank106512 +106513 POINT(41.60591084002007 74.08895510033285) bank106513 +106514 POINT(41.432646670239116 73.16036214919966) bank106514 +106515 POINT(40.29168010161779 73.6485924629223) bank106515 +106516 POINT(39.755685263901945 74.67681871747315) bank106516 +106517 POINT(40.074512993425444 73.6499088992246) bank106517 +106518 POINT(40.02572332745777 74.67463976123577) bank106518 +106519 POINT(40.257770466381544 74.75984119175166) bank106519 +106520 POINT(41.6896828778427 74.66905373087488) bank106520 +106521 POINT(40.3098487277082 73.22591547511477) bank106521 +106522 POINT(40.415444986684406 74.2891654864589) bank106522 +106523 POINT(39.91041619652187 74.13206908974948) bank106523 +106524 POINT(40.26721527096716 73.75572540270808) bank106524 +106525 POINT(40.599295423914675 73.32607390381926) bank106525 +106526 POINT(39.960341814056044 74.65829864811371) bank106526 +106527 POINT(40.80164866408104 74.99284058015529) bank106527 +106528 POINT(41.11085215850465 74.61945859297867) bank106528 +106529 POINT(41.39404735835652 73.6508101437237) bank106529 +106530 POINT(40.44242617655893 74.60753083659938) bank106530 +106531 POINT(40.585277321048 73.97543609144608) bank106531 +106532 POINT(41.64375195996572 73.99673685818605) bank106532 +106533 POINT(40.874588994587825 74.92338730260514) bank106533 +106534 POINT(40.041245578817374 73.18809239278465) bank106534 +106535 POINT(41.13332273551916 74.55230367199457) bank106535 +106536 POINT(40.04237074160169 73.30010496363444) bank106536 +106537 POINT(40.63888180018091 74.85427369508183) bank106537 +106538 POINT(40.76612800565262 74.80220520371928) bank106538 +106539 POINT(41.488207503344626 73.01656576430757) bank106539 +106540 POINT(41.457010539146765 74.76348920696731) bank106540 +106541 POINT(40.07733338993552 73.03472459367029) bank106541 +106542 POINT(40.365403580291485 74.35353530955811) bank106542 +106543 POINT(39.959500344633334 74.41574072352024) bank106543 +106544 POINT(41.40363455162166 74.50280994508482) bank106544 +106545 POINT(41.61475450105451 74.21899531928949) bank106545 +106546 POINT(40.32931461786488 73.90865071670133) bank106546 +106547 POINT(41.650895831911825 73.72448183982723) bank106547 +106548 POINT(41.37393953987869 74.20939835176151) bank106548 +106549 POINT(41.23968461743153 73.36753738654728) bank106549 +106550 POINT(41.14129505874542 73.45475323435853) bank106550 +106551 POINT(41.0115505048135 73.11882898541471) bank106551 +106552 POINT(39.900927954456016 74.52183199918801) bank106552 +106553 POINT(40.254474862916155 74.05654217560316) bank106553 +106554 POINT(41.44217354433101 73.68969165531348) bank106554 +106555 POINT(41.41747622106515 74.82513832397449) bank106555 +106556 POINT(39.844194134691094 74.55088719679546) bank106556 +106557 POINT(40.502762089027726 73.90241391994425) bank106557 +106558 POINT(40.60644457047492 73.65063773644513) bank106558 +106559 POINT(40.95289014816455 74.30346751218889) bank106559 +106560 POINT(41.22414921939818 73.8234912399794) bank106560 +106561 POINT(41.11606453128586 74.37566543446015) bank106561 +106562 POINT(40.641870350378355 74.29655326942309) bank106562 +106563 POINT(41.06427320480618 73.41864905049364) bank106563 +106564 POINT(40.07180413367351 74.64363294693446) bank106564 +106565 POINT(40.12320404143702 73.43315082510777) bank106565 +106566 POINT(40.90697428052023 73.39723230374794) bank106566 +106567 POINT(39.88335036958976 74.4306623929031) bank106567 +106568 POINT(40.168018164327506 74.96292796639746) bank106568 +106569 POINT(41.238910526361 73.83628359783485) bank106569 +106570 POINT(39.75779654238703 74.07758571692227) bank106570 +106571 POINT(40.914945590032445 74.09272983292544) bank106571 +106572 POINT(40.946319865016484 73.61689174253424) bank106572 +106573 POINT(41.41115501543868 73.76927139032081) bank106573 +106574 POINT(40.10173629658671 73.98744479882046) bank106574 +106575 POINT(41.09000203313669 73.82152376338198) bank106575 +106576 POINT(41.60147400252447 73.40784898953618) bank106576 +106577 POINT(40.3244169351848 73.58461167757355) bank106577 +106578 POINT(39.941831609433784 73.93397672018219) bank106578 +106579 POINT(41.55738707707237 74.5065501047351) bank106579 +106580 POINT(41.46210375416503 75.00193109102601) bank106580 +106581 POINT(41.57972229402941 74.17057208029871) bank106581 +106582 POINT(40.537473729571396 74.3943710708617) bank106582 +106583 POINT(41.53889373025238 74.33199236433308) bank106583 +106584 POINT(41.23967064164268 73.28173212710938) bank106584 +106585 POINT(41.581907129753276 74.35911161497896) bank106585 +106586 POINT(40.80625928555692 74.62867047320086) bank106586 +106587 POINT(40.75856841058493 74.97707715045279) bank106587 +106588 POINT(40.242427711879785 74.55420957941477) bank106588 +106589 POINT(41.19631282707464 74.74253868620076) bank106589 +106590 POINT(40.87138564940845 73.43407350126188) bank106590 +106591 POINT(39.84204375485983 73.2856338107313) bank106591 +106592 POINT(40.049722579746735 73.89446432347128) bank106592 +106593 POINT(40.38217203831508 73.0251520124) bank106593 +106594 POINT(41.410538238590526 73.6551238627893) bank106594 +106595 POINT(39.86805123071213 73.96717921630182) bank106595 +106596 POINT(41.31903234935282 73.2728946550721) bank106596 +106597 POINT(40.97829519279122 73.30884481782358) bank106597 +106598 POINT(40.3638779374088 74.37913567412542) bank106598 +106599 POINT(41.394399462464044 73.81328980840783) bank106599 +106600 POINT(41.20715996264957 74.63198150705647) bank106600 +106601 POINT(41.312027485606386 74.26136143634936) bank106601 +106602 POINT(40.24144710572665 74.43579710281291) bank106602 +106603 POINT(40.70494615894722 73.69190680567729) bank106603 +106604 POINT(39.87613858187477 73.1939119294638) bank106604 +106605 POINT(40.71885405578706 73.9454407194685) bank106605 +106606 POINT(40.0081258020891 73.48082194213939) bank106606 +106607 POINT(40.492527508022306 74.1143711208376) bank106607 +106608 POINT(39.79273463829709 74.70330145108836) bank106608 +106609 POINT(39.92708038649795 73.91677077873648) bank106609 +106610 POINT(41.07050686654139 73.95744745603172) bank106610 +106611 POINT(39.76853307253236 74.91990812900053) bank106611 +106612 POINT(40.00517285102307 73.59033749268492) bank106612 +106613 POINT(40.50702666015852 74.16953747032963) bank106613 +106614 POINT(40.65552909196566 73.73667330055937) bank106614 +106615 POINT(40.643603516919605 73.74559173532968) bank106615 +106616 POINT(40.45208440625837 74.3107153905129) bank106616 +106617 POINT(40.87112372204122 73.77272126752986) bank106617 +106618 POINT(41.12500197041882 73.78732856882459) bank106618 +106619 POINT(40.64933632879464 73.74746091703959) bank106619 +106620 POINT(41.69619015961063 73.56498041151474) bank106620 +106621 POINT(41.11931977670753 73.31422069028211) bank106621 +106622 POINT(40.56097207728893 74.53857992199175) bank106622 +106623 POINT(41.65154635313515 73.35373891017778) bank106623 +106624 POINT(40.77155591134262 73.70914257594725) bank106624 +106625 POINT(41.44843241557599 74.46432398742924) bank106625 +106626 POINT(41.155015790442405 73.23399563049641) bank106626 +106627 POINT(40.98123009712251 74.60396903993741) bank106627 +106628 POINT(40.99889210247596 74.46440542850128) bank106628 +106629 POINT(39.81057612751729 73.46078398384665) bank106629 +106630 POINT(40.07801530508425 74.46440976861984) bank106630 +106631 POINT(41.143498635062876 74.22858773755107) bank106631 +106632 POINT(39.916946913052506 73.66834187541785) bank106632 +106633 POINT(41.02899963980321 74.48140172897821) bank106633 +106634 POINT(41.23462010208724 74.3268276244323) bank106634 +106635 POINT(40.794382047717114 73.99446743835605) bank106635 +106636 POINT(40.68081228017927 73.5872154735535) bank106636 +106637 POINT(40.52533676930706 73.30223577756956) bank106637 +106638 POINT(40.257095148620316 73.85080390843093) bank106638 +106639 POINT(41.272690259307666 73.5428651908751) bank106639 +106640 POINT(41.38574599702263 73.8025087387429) bank106640 +106641 POINT(40.986391978706486 74.3102682958791) bank106641 +106642 POINT(41.633420303128766 73.12745500326928) bank106642 +106643 POINT(40.68257489163817 73.81735954153643) bank106643 +106644 POINT(40.21371791943717 73.37554087267212) bank106644 +106645 POINT(39.76162630213693 73.69525244371796) bank106645 +106646 POINT(40.20374442902904 74.62792461035548) bank106646 +106647 POINT(40.82860805143857 74.43864847997659) bank106647 +106648 POINT(40.726638524914236 73.51105398227756) bank106648 +106649 POINT(41.025372617749454 74.77585009011861) bank106649 +106650 POINT(41.05925733494555 73.64351356337696) bank106650 +106651 POINT(40.64650270072978 73.63324028015784) bank106651 +106652 POINT(40.27633129660491 74.9753167681918) bank106652 +106653 POINT(39.86548013009865 73.82658563604713) bank106653 +106654 POINT(41.08866231391721 73.728337982001) bank106654 +106655 POINT(39.88372250207367 74.47882345981137) bank106655 +106656 POINT(41.597836348823606 74.347187195638) bank106656 +106657 POINT(39.914546700106875 74.3631116469851) bank106657 +106658 POINT(40.92686349088512 74.15897158275428) bank106658 +106659 POINT(41.12744257761625 73.50689403357909) bank106659 +106660 POINT(40.23233401892667 74.18192126039007) bank106660 +106661 POINT(39.795655816583945 73.0727296290793) bank106661 +106662 POINT(41.28755958637249 74.72024143222316) bank106662 +106663 POINT(40.94638856646619 73.10722443317442) bank106663 +106664 POINT(40.31504271072645 73.47566725336513) bank106664 +106665 POINT(40.09436198734985 73.645009058955) bank106665 +106666 POINT(39.931140183066184 73.79186551838282) bank106666 +106667 POINT(41.56867760832477 73.81260574012144) bank106667 +106668 POINT(41.473262197195254 74.60673688411443) bank106668 +106669 POINT(39.98884168248144 74.3946869842527) bank106669 +106670 POINT(40.38881128430883 73.44925746149093) bank106670 +106671 POINT(40.58101582316028 73.86689150106892) bank106671 +106672 POINT(40.47293818332566 73.05380247465285) bank106672 +106673 POINT(41.13003125225637 73.10636992197125) bank106673 +106674 POINT(40.49619322561337 74.14136715732815) bank106674 +106675 POINT(39.96311735847561 74.71992423307869) bank106675 +106676 POINT(41.50289200014222 74.39984382025168) bank106676 +106677 POINT(40.7140584170615 74.59279854489778) bank106677 +106678 POINT(41.31248376124858 73.20549081532911) bank106678 +106679 POINT(40.26949914975629 73.37190822852503) bank106679 +106680 POINT(41.61107415745246 74.0980771583551) bank106680 +106681 POINT(41.03781584643689 74.97276857136562) bank106681 +106682 POINT(41.522904256445514 73.32025823728202) bank106682 +106683 POINT(40.9593486672957 73.15002881747151) bank106683 +106684 POINT(41.632027391970965 74.76121815356937) bank106684 +106685 POINT(40.590709876707045 74.89111097079646) bank106685 +106686 POINT(41.242491583572765 74.80880903438901) bank106686 +106687 POINT(40.45144591974098 73.59535389621351) bank106687 +106688 POINT(39.95173538151814 73.01396898907385) bank106688 +106689 POINT(40.93226075634665 74.01519621387807) bank106689 +106690 POINT(41.62561844314799 74.29436988521053) bank106690 +106691 POINT(41.144065659247566 74.92487732031965) bank106691 +106692 POINT(40.44207308995696 73.19324216848155) bank106692 +106693 POINT(39.78984288019507 73.24978024515369) bank106693 +106694 POINT(40.235856960913175 74.05150783739055) bank106694 +106695 POINT(40.29500273655354 74.79121995119603) bank106695 +106696 POINT(40.53896097785746 73.12381961845594) bank106696 +106697 POINT(39.8487220945376 74.3432692675121) bank106697 +106698 POINT(40.894300574956176 74.71339504596091) bank106698 +106699 POINT(40.8694641177873 74.3080027898942) bank106699 +106700 POINT(41.41319801541662 74.4113067583872) bank106700 +106701 POINT(40.165819514349664 74.83597911330453) bank106701 +106702 POINT(41.20365840417681 73.8886826843395) bank106702 +106703 POINT(41.01678704053201 74.1175262987656) bank106703 +106704 POINT(40.388901057249704 74.0777051056745) bank106704 +106705 POINT(39.74814680290793 73.70916908358248) bank106705 +106706 POINT(41.38746425586485 73.72042526970652) bank106706 +106707 POINT(40.432919994309934 74.00855404840678) bank106707 +106708 POINT(40.82453165643502 73.55006672079686) bank106708 +106709 POINT(40.560326322107834 73.0509113266239) bank106709 +106710 POINT(40.91949337531589 73.58471776485835) bank106710 +106711 POINT(40.74964220058365 73.37759580317449) bank106711 +106712 POINT(41.13641991762299 74.17411313889612) bank106712 +106713 POINT(40.18610897800589 73.53374331732186) bank106713 +106714 POINT(41.31601287983101 73.42718928165625) bank106714 +106715 POINT(41.335467395677945 73.73730532555027) bank106715 +106716 POINT(41.640286833506444 73.80338348695828) bank106716 +106717 POINT(41.439230555675636 74.45446621013754) bank106717 +106718 POINT(40.49534731041191 73.7417357553859) bank106718 +106719 POINT(40.023245787352714 74.76081204317937) bank106719 +106720 POINT(41.19790185772507 74.66292485291169) bank106720 +106721 POINT(40.93257325049487 74.15719831743644) bank106721 +106722 POINT(40.234745561503125 73.92486999957121) bank106722 +106723 POINT(39.83374329977438 73.63606420267632) bank106723 +106724 POINT(41.1413289443471 74.17253577830832) bank106724 +106725 POINT(41.53544279239437 74.98563413897286) bank106725 +106726 POINT(39.79060004419612 74.09563130155901) bank106726 +106727 POINT(40.808339854520824 73.64775666665032) bank106727 +106728 POINT(40.73183437225711 74.46190704899479) bank106728 +106729 POINT(41.50829367966779 73.8172727146621) bank106729 +106730 POINT(40.74694420700897 74.93169401730991) bank106730 +106731 POINT(41.3903907712969 74.07839990025386) bank106731 +106732 POINT(40.129308874707334 73.59751608008149) bank106732 +106733 POINT(40.51108385715305 74.49022755132154) bank106733 +106734 POINT(40.86364507178455 73.37377439291376) bank106734 +106735 POINT(41.12031734231181 73.33601455798888) bank106735 +106736 POINT(40.477547261440264 73.50792677118592) bank106736 +106737 POINT(40.104375123881965 73.03080916137631) bank106737 +106738 POINT(41.053427506743866 74.31772658479115) bank106738 +106739 POINT(41.587043535329535 73.50532751130875) bank106739 +106740 POINT(41.099237760157614 73.17830322287503) bank106740 +106741 POINT(40.684197248662684 73.75432719234725) bank106741 +106742 POINT(40.03149206184224 73.38723210650593) bank106742 +106743 POINT(40.22812938334989 74.54319876999936) bank106743 +106744 POINT(41.283671644254966 74.9067337511686) bank106744 +106745 POINT(41.66115363816905 74.8006550407627) bank106745 +106746 POINT(40.140461126441174 73.2679141397595) bank106746 +106747 POINT(41.38978137510051 73.89433422575023) bank106747 +106748 POINT(40.704275964546355 74.56056875318667) bank106748 +106749 POINT(41.70163509648986 74.37603237383371) bank106749 +106750 POINT(40.263808433245785 73.4053816987683) bank106750 +106751 POINT(40.01902456172713 74.37632141143118) bank106751 +106752 POINT(40.785721470634385 74.91153093675064) bank106752 +106753 POINT(40.71753833154933 74.61502300113138) bank106753 +106754 POINT(41.38271374048838 73.39923135571715) bank106754 +106755 POINT(40.00182964106184 73.73489310636471) bank106755 +106756 POINT(41.64184753055661 73.23675281792141) bank106756 +106757 POINT(40.10135186137996 74.55224224235306) bank106757 +106758 POINT(39.82255093639971 73.50889346367481) bank106758 +106759 POINT(40.20211145598329 73.63988381727259) bank106759 +106760 POINT(40.23892565726384 74.51700590018183) bank106760 +106761 POINT(41.394544075811105 73.05756030645277) bank106761 +106762 POINT(39.872212365964316 73.58871875808744) bank106762 +106763 POINT(41.01086493674924 74.31196302192573) bank106763 +106764 POINT(41.23353813410876 74.13709991246762) bank106764 +106765 POINT(40.63890073764507 74.4662441073154) bank106765 +106766 POINT(40.8405935143581 73.5387774148878) bank106766 +106767 POINT(40.0104274746382 73.84579861330859) bank106767 +106768 POINT(41.61461887959642 74.61325162336419) bank106768 +106769 POINT(41.40317417308619 74.20682565841992) bank106769 +106770 POINT(40.46587048784713 74.6395381725434) bank106770 +106771 POINT(39.92590442218286 74.44561290110882) bank106771 +106772 POINT(40.86528299585842 74.4877128937925) bank106772 +106773 POINT(41.10767327462565 74.91082342133471) bank106773 +106774 POINT(41.52802432807043 74.01324554304813) bank106774 +106775 POINT(40.54935016482364 74.79664378934) bank106775 +106776 POINT(39.90422846042105 73.52495102471316) bank106776 +106777 POINT(41.0794374077067 73.5170370024947) bank106777 +106778 POINT(41.378652756874935 74.98708291258998) bank106778 +106779 POINT(41.60509367444506 73.18602069948257) bank106779 +106780 POINT(40.06144672290243 74.77082441881954) bank106780 +106781 POINT(41.16278894209648 73.47155629971297) bank106781 +106782 POINT(40.9664707820648 74.58605423200613) bank106782 +106783 POINT(40.620148864655604 73.67295791400632) bank106783 +106784 POINT(39.7152006886108 74.061268288513) bank106784 +106785 POINT(39.76343691297619 74.73417098913255) bank106785 +106786 POINT(40.682368504560536 73.59258465891402) bank106786 +106787 POINT(40.21725803581873 74.18761478811953) bank106787 +106788 POINT(41.60616534432714 73.96593540184975) bank106788 +106789 POINT(39.92203638556462 74.00559093378847) bank106789 +106790 POINT(40.70420450843517 73.47630169592126) bank106790 +106791 POINT(40.35937436889405 74.80932437734616) bank106791 +106792 POINT(40.644477691990076 74.04220330863924) bank106792 +106793 POINT(40.03585714748245 74.80082795763401) bank106793 +106794 POINT(40.52808365166724 74.74012228940295) bank106794 +106795 POINT(39.78559084168602 73.40894469909122) bank106795 +106796 POINT(40.08011183001392 73.89690948832539) bank106796 +106797 POINT(41.04463578263919 74.92783872025211) bank106797 +106798 POINT(40.33923951924616 73.88778564814164) bank106798 +106799 POINT(40.035016738300854 73.19411209301896) bank106799 +106800 POINT(41.60251446147215 74.86987218966824) bank106800 +106801 POINT(40.82836272819745 73.96186105258225) bank106801 +106802 POINT(41.03900719829832 73.25801616071074) bank106802 +106803 POINT(40.393863725946474 74.54068876897672) bank106803 +106804 POINT(41.31856762388217 74.33616229602433) bank106804 +106805 POINT(40.79542549130856 74.19937933472501) bank106805 +106806 POINT(41.563502187108114 74.438191751533) bank106806 +106807 POINT(40.54098583704796 74.90124722407916) bank106807 +106808 POINT(41.69332642032252 73.49052185278424) bank106808 +106809 POINT(40.93846293833446 74.22573815609493) bank106809 +106810 POINT(41.71222784516202 74.42902995457936) bank106810 +106811 POINT(40.266322457030014 73.29990248624928) bank106811 +106812 POINT(40.50089846459194 74.84933406457294) bank106812 +106813 POINT(41.17332088642811 73.90855494462777) bank106813 +106814 POINT(40.218403623111946 73.95969553112383) bank106814 +106815 POINT(39.75027699057433 74.92903022262163) bank106815 +106816 POINT(40.67150228346941 74.1819458588411) bank106816 +106817 POINT(40.09856898483641 73.4811449771933) bank106817 +106818 POINT(40.01916605856243 73.14188954091938) bank106818 +106819 POINT(39.847351806701575 74.5555484957149) bank106819 +106820 POINT(39.71473496152832 73.74472971828277) bank106820 +106821 POINT(40.33780338717898 74.12547721533262) bank106821 +106822 POINT(39.81401586748759 74.37313605199456) bank106822 +106823 POINT(41.39714706394226 74.55115983540078) bank106823 +106824 POINT(41.177857858536804 73.92825079337256) bank106824 +106825 POINT(41.55135002284637 74.52547901858698) bank106825 +106826 POINT(39.98723921294102 73.59172689278917) bank106826 +106827 POINT(41.39937506681184 73.37022695418348) bank106827 +106828 POINT(40.49531115708539 73.90411974768355) bank106828 +106829 POINT(39.790605479426716 73.81195076346536) bank106829 +106830 POINT(40.54006597205586 74.15080788424332) bank106830 +106831 POINT(41.66844226306545 74.83336642358358) bank106831 +106832 POINT(41.28837997492544 74.1167881191861) bank106832 +106833 POINT(41.210135950009025 73.45773490172327) bank106833 +106834 POINT(41.006133837894616 74.00218944726053) bank106834 +106835 POINT(41.28831570438672 74.84858211687158) bank106835 +106836 POINT(40.61405321182409 73.79859202014134) bank106836 +106837 POINT(41.28611133466113 73.35378426564102) bank106837 +106838 POINT(40.8263323191659 73.79770818225035) bank106838 +106839 POINT(40.362458045528335 73.14137079936918) bank106839 +106840 POINT(41.50980103169101 74.187844185821) bank106840 +106841 POINT(40.01096783069818 74.62368198541799) bank106841 +106842 POINT(41.5301021125783 73.6781376053194) bank106842 +106843 POINT(40.22471879609569 74.67053722442144) bank106843 +106844 POINT(41.247032639233254 74.22196489547483) bank106844 +106845 POINT(40.739411506741355 74.82096269018379) bank106845 +106846 POINT(41.27975017951532 73.83625649504431) bank106846 +106847 POINT(40.6232114924535 73.11279042610106) bank106847 +106848 POINT(39.88621997655704 73.0068699746794) bank106848 +106849 POINT(40.100044401628786 73.7874679899092) bank106849 +106850 POINT(41.42917030642919 73.3139234389964) bank106850 +106851 POINT(40.535614834152454 73.61008253354194) bank106851 +106852 POINT(41.50363795734879 73.099554873171) bank106852 +106853 POINT(40.56452198051619 73.87259413820829) bank106853 +106854 POINT(41.609794913599956 73.74895844209995) bank106854 +106855 POINT(40.52875424549332 73.60571415286714) bank106855 +106856 POINT(39.78121821141457 73.1840937025424) bank106856 +106857 POINT(40.190982766153496 73.25111298976573) bank106857 +106858 POINT(40.98783851064011 74.84572189317977) bank106858 +106859 POINT(41.534105593217966 73.27222833273099) bank106859 +106860 POINT(41.33656982627249 73.48831835051371) bank106860 +106861 POINT(40.29243815653671 74.37607130916476) bank106861 +106862 POINT(39.91682052781055 74.76633670382748) bank106862 +106863 POINT(41.47580619933345 74.14608987946937) bank106863 +106864 POINT(40.75723914336407 74.80853149624687) bank106864 +106865 POINT(40.43690093139347 74.1479646597443) bank106865 +106866 POINT(41.21391778867997 73.52017730009521) bank106866 +106867 POINT(41.44079266065157 73.94368782530321) bank106867 +106868 POINT(39.94722364688589 73.6160679510796) bank106868 +106869 POINT(40.446103281315246 74.7764382842164) bank106869 +106870 POINT(40.94738813252974 73.73280856991812) bank106870 +106871 POINT(40.283584614739105 75.00249314261275) bank106871 +106872 POINT(40.184983068479745 73.93492830477831) bank106872 +106873 POINT(39.88717003418345 73.20390592110829) bank106873 +106874 POINT(40.921221688725446 74.49340290083285) bank106874 +106875 POINT(40.96888111429351 74.19112812342583) bank106875 +106876 POINT(39.82402638883565 74.48085173233956) bank106876 +106877 POINT(41.22517184388628 74.44268248313188) bank106877 +106878 POINT(41.51632380728674 73.10338483466819) bank106878 +106879 POINT(40.748186680215646 74.81094516656073) bank106879 +106880 POINT(40.37336444395133 74.85953583917086) bank106880 +106881 POINT(39.914153264228496 73.01412946473528) bank106881 +106882 POINT(41.64954018608259 73.3350010697005) bank106882 +106883 POINT(40.77879581349538 74.85827552904277) bank106883 +106884 POINT(41.48210891375606 73.91892206965285) bank106884 +106885 POINT(40.229477485950376 74.85955885219171) bank106885 +106886 POINT(40.06423589606639 74.66872399909727) bank106886 +106887 POINT(40.1008208124567 74.13880594190458) bank106887 +106888 POINT(39.72435841855924 74.77011946579786) bank106888 +106889 POINT(41.69531280117373 73.58936679164675) bank106889 +106890 POINT(40.694566417313965 73.99124043271941) bank106890 +106891 POINT(39.72770424921959 73.33069551957522) bank106891 +106892 POINT(39.909420008189926 74.05110123354866) bank106892 +106893 POINT(39.937021850494794 75.00597414893309) bank106893 +106894 POINT(40.36859835441216 73.66592161854665) bank106894 +106895 POINT(41.65257858739435 73.31358593699227) bank106895 +106896 POINT(40.83047337709411 73.92001540572336) bank106896 +106897 POINT(40.12031675947282 73.99890325261697) bank106897 +106898 POINT(40.66527740134799 74.1622292648679) bank106898 +106899 POINT(41.482091476591435 73.73240062498098) bank106899 +106900 POINT(41.486516772445874 74.37015970341949) bank106900 +106901 POINT(40.957345743865254 74.9032389752207) bank106901 +106902 POINT(40.71941059905951 73.53787129801917) bank106902 +106903 POINT(40.97923998229707 73.69059325238196) bank106903 +106904 POINT(40.92384064235022 74.14573093921979) bank106904 +106905 POINT(40.484162984052915 74.96397080583355) bank106905 +106906 POINT(40.16254293687316 73.02749755718636) bank106906 +106907 POINT(41.42038872805583 74.96583989408884) bank106907 +106908 POINT(41.71200947736917 74.51641217875324) bank106908 +106909 POINT(40.37690100260903 73.91986061065944) bank106909 +106910 POINT(39.94939311040327 74.20410311142913) bank106910 +106911 POINT(41.441013912866474 74.69240260111978) bank106911 +106912 POINT(40.58729798176515 74.45281665953262) bank106912 +106913 POINT(39.90174085763579 74.55308133884611) bank106913 +106914 POINT(40.66729313147964 73.35741219756257) bank106914 +106915 POINT(41.302259018087526 73.8963595174975) bank106915 +106916 POINT(40.467112014138735 74.0189900039151) bank106916 +106917 POINT(41.630863485265415 73.57124657531111) bank106917 +106918 POINT(39.88578952454195 74.73552202588061) bank106918 +106919 POINT(39.76437485486308 73.4904584969626) bank106919 +106920 POINT(40.1740931291647 73.77244370095288) bank106920 +106921 POINT(40.270539437249326 74.0771039532214) bank106921 +106922 POINT(40.75718451373736 74.43721556816406) bank106922 +106923 POINT(40.32275922375371 73.76017972221159) bank106923 +106924 POINT(40.75299064316499 73.28725149513275) bank106924 +106925 POINT(41.66989531795351 73.52932127615767) bank106925 +106926 POINT(39.87710798456036 74.93858484774927) bank106926 +106927 POINT(39.785869492760945 73.78787415501571) bank106927 +106928 POINT(40.24358727382624 73.08662592640897) bank106928 +106929 POINT(41.642044797871556 73.93059212990867) bank106929 +106930 POINT(41.42402217349724 73.82564621149274) bank106930 +106931 POINT(41.016899237922075 73.91520173277506) bank106931 +106932 POINT(39.96522518665454 74.32038239669895) bank106932 +106933 POINT(39.72460563253536 74.45395880423752) bank106933 +106934 POINT(40.54097332580681 74.88010866550094) bank106934 +106935 POINT(39.71894393917122 73.53773546134762) bank106935 +106936 POINT(40.29160738988804 74.24438622666625) bank106936 +106937 POINT(40.81846879938402 73.5058950044739) bank106937 +106938 POINT(41.52718644241113 74.94645782216178) bank106938 +106939 POINT(41.64048208579823 74.72608681118227) bank106939 +106940 POINT(41.02940970117888 73.22865544840162) bank106940 +106941 POINT(41.18761828651586 73.46624531820319) bank106941 +106942 POINT(41.487602707517645 74.73494795788096) bank106942 +106943 POINT(41.00425746363123 73.69563973437582) bank106943 +106944 POINT(40.92626770788816 74.64950649161257) bank106944 +106945 POINT(40.37498538061606 74.65442313463514) bank106945 +106946 POINT(40.98236066925052 73.80774275215059) bank106946 +106947 POINT(41.0588648527853 74.53093624754726) bank106947 +106948 POINT(40.9541664426202 74.93703449496304) bank106948 +106949 POINT(40.864723987195234 74.63751529906966) bank106949 +106950 POINT(40.88980662592968 74.41179314174263) bank106950 +106951 POINT(40.07150453016131 73.20815894596733) bank106951 +106952 POINT(41.67946590018106 73.67279316204068) bank106952 +106953 POINT(39.791514380275416 74.40621684624692) bank106953 +106954 POINT(40.390927661519385 74.07344562078354) bank106954 +106955 POINT(41.33097327456372 73.76502449330724) bank106955 +106956 POINT(41.630546366613046 74.28053054620972) bank106956 +106957 POINT(40.28204977999563 74.9081129213481) bank106957 +106958 POINT(41.47637933397655 74.5841248022437) bank106958 +106959 POINT(41.18273917768198 74.32406560857207) bank106959 +106960 POINT(41.48915230306343 73.17354924129309) bank106960 +106961 POINT(40.5051238630313 73.72263818333109) bank106961 +106962 POINT(40.285348153216034 73.1062526291881) bank106962 +106963 POINT(41.09979580696376 74.90162032575978) bank106963 +106964 POINT(41.47734217329942 73.07325274118291) bank106964 +106965 POINT(40.10688733819802 74.07022832096051) bank106965 +106966 POINT(40.53675049626642 74.85389113830021) bank106966 +106967 POINT(41.39681739298594 74.80191468871534) bank106967 +106968 POINT(40.32629809033892 73.95902627289469) bank106968 +106969 POINT(41.35362737714049 74.0367626155497) bank106969 +106970 POINT(41.600726223119196 74.23587775572577) bank106970 +106971 POINT(40.301638028420854 73.0406995699241) bank106971 +106972 POINT(40.56232839076467 73.02842259453803) bank106972 +106973 POINT(39.81754138285286 74.6781066645114) bank106973 +106974 POINT(41.21252519226332 74.32985000969926) bank106974 +106975 POINT(41.39880772022397 74.04534908159089) bank106975 +106976 POINT(41.229460005861554 73.59800372700177) bank106976 +106977 POINT(39.805423949905055 73.01669385952198) bank106977 +106978 POINT(39.72692810999722 73.22036519900297) bank106978 +106979 POINT(40.87000536853127 74.9432498166441) bank106979 +106980 POINT(40.09329106990654 74.47644436459456) bank106980 +106981 POINT(41.002072265851965 74.66783495861888) bank106981 +106982 POINT(41.13856539080714 73.1416066272642) bank106982 +106983 POINT(40.011817974410704 73.49129833405382) bank106983 +106984 POINT(40.194658138594754 73.27300698320055) bank106984 +106985 POINT(40.707111172494116 74.63153588365223) bank106985 +106986 POINT(40.81219915635042 74.76123893504767) bank106986 +106987 POINT(40.81023347523695 74.45407183199059) bank106987 +106988 POINT(41.65183808840444 73.0171437207676) bank106988 +106989 POINT(41.265063381528314 74.18196590153848) bank106989 +106990 POINT(39.99888342865671 74.27918717317874) bank106990 +106991 POINT(40.31470591513453 74.46381639009874) bank106991 +106992 POINT(39.82282659008687 74.63161941778026) bank106992 +106993 POINT(41.33942423179118 74.85691605447794) bank106993 +106994 POINT(40.450781687365186 74.94845799393802) bank106994 +106995 POINT(40.18015707909378 74.62625618662535) bank106995 +106996 POINT(39.7369332196541 74.3133668927494) bank106996 +106997 POINT(41.63034917771968 74.70625940191867) bank106997 +106998 POINT(41.39841722379345 73.22563605219645) bank106998 +106999 POINT(40.23706551846777 74.61592299680146) bank106999 +107000 POINT(39.94059401116156 73.29416084514973) bank107000 +107001 POINT(40.416151388056086 73.82528828676304) bank107001 +107002 POINT(41.41233150682583 73.93471550773683) bank107002 +107003 POINT(41.411853831328486 73.21421770425296) bank107003 +107004 POINT(41.43349686668619 74.75401562848349) bank107004 +107005 POINT(40.736029263464815 73.85469947081512) bank107005 +107006 POINT(41.71138006003242 74.17435055140304) bank107006 +107007 POINT(40.26119368746677 73.03355942577508) bank107007 +107008 POINT(41.37655273018341 74.63941832793115) bank107008 +107009 POINT(41.361243697810885 73.51918470959315) bank107009 +107010 POINT(41.14897061397827 74.85643650575919) bank107010 +107011 POINT(39.8311205344112 74.89146232929188) bank107011 +107012 POINT(40.61720818782604 74.85557351877985) bank107012 +107013 POINT(40.60629592938251 73.231746577514) bank107013 +107014 POINT(40.143373586680774 74.64762793769391) bank107014 +107015 POINT(41.47442454159648 74.11671139396276) bank107015 +107016 POINT(40.818460344090454 73.8066002600912) bank107016 +107017 POINT(41.08767637393984 73.76571324235694) bank107017 +107018 POINT(40.77159057388407 74.09432815722964) bank107018 +107019 POINT(40.30909354383688 74.00011678632481) bank107019 +107020 POINT(39.911958644813794 73.09306685721666) bank107020 +107021 POINT(41.38093406748154 73.66216149840436) bank107021 +107022 POINT(40.90659398145732 73.71259718390459) bank107022 +107023 POINT(40.146751589016716 73.5549948215149) bank107023 +107024 POINT(40.772393929155776 74.51761232233508) bank107024 +107025 POINT(40.938509676777805 73.84315426589582) bank107025 +107026 POINT(39.82433543411465 73.44486718978389) bank107026 +107027 POINT(40.29969856701414 74.83296893546394) bank107027 +107028 POINT(39.84521284773937 73.14974404139913) bank107028 +107029 POINT(41.01460041962046 73.13737565780234) bank107029 +107030 POINT(40.143616433115945 74.53720083888682) bank107030 +107031 POINT(41.289190893538944 73.47509036513098) bank107031 +107032 POINT(39.80495915388487 74.47582628969016) bank107032 +107033 POINT(40.61311965054236 73.86291023229298) bank107033 +107034 POINT(40.888787855582876 73.62117698640057) bank107034 +107035 POINT(39.75904800147408 74.76178983879238) bank107035 +107036 POINT(41.40757402353377 73.05169871043287) bank107036 +107037 POINT(40.00264232303637 73.80399068329365) bank107037 +107038 POINT(40.1830940624786 73.5450093320408) bank107038 +107039 POINT(41.29231586366172 73.06599464252466) bank107039 +107040 POINT(41.23860992767316 73.7003796657421) bank107040 +107041 POINT(41.65190121656845 74.3287179080562) bank107041 +107042 POINT(41.20410531853105 73.95545084526596) bank107042 +107043 POINT(40.6611797256138 73.09640813137372) bank107043 +107044 POINT(40.91021475583152 73.20748647218973) bank107044 +107045 POINT(40.171964258887016 73.38318996959966) bank107045 +107046 POINT(40.89512633479535 74.70854592810232) bank107046 +107047 POINT(39.91717446504993 73.45416194299979) bank107047 +107048 POINT(41.37463703724479 73.06515876977245) bank107048 +107049 POINT(41.41460340297632 73.55463435518857) bank107049 +107050 POINT(39.93401882662377 73.14675318061389) bank107050 +107051 POINT(41.31357923371254 74.43175056603353) bank107051 +107052 POINT(40.99852015124631 73.34747068165446) bank107052 +107053 POINT(41.46376922761916 73.45383727389958) bank107053 +107054 POINT(41.69832738108138 73.60579173984894) bank107054 +107055 POINT(40.296193249155685 74.10425721396452) bank107055 +107056 POINT(40.250963957247556 73.8877796212176) bank107056 +107057 POINT(40.42335903807408 74.34232167714798) bank107057 +107058 POINT(41.215191889659906 73.56379044420191) bank107058 +107059 POINT(41.09761063006568 73.69577351130174) bank107059 +107060 POINT(40.893204051331914 73.2081085590358) bank107060 +107061 POINT(41.609293578039754 74.46611477371009) bank107061 +107062 POINT(40.14706918802225 73.73326470470197) bank107062 +107063 POINT(40.13109547713802 74.70404924802878) bank107063 +107064 POINT(39.93014613250254 74.22020885439392) bank107064 +107065 POINT(41.32360747387197 73.03732460364073) bank107065 +107066 POINT(40.43813279906608 73.72697765461363) bank107066 +107067 POINT(41.60315296715576 73.68048992594572) bank107067 +107068 POINT(40.94569653326792 74.0073613688907) bank107068 +107069 POINT(40.94419472706002 73.89809018134912) bank107069 +107070 POINT(39.895435349809745 73.30835668474923) bank107070 +107071 POINT(40.62371474522196 74.84513202927288) bank107071 +107072 POINT(41.16973407557242 74.50217654870588) bank107072 +107073 POINT(41.01196075541535 74.11283933310234) bank107073 +107074 POINT(40.16342047510725 73.45759621673027) bank107074 +107075 POINT(40.189636642396486 73.6327988365739) bank107075 +107076 POINT(41.65101723474684 73.7111815965749) bank107076 +107077 POINT(41.49066714883671 74.77931409009861) bank107077 +107078 POINT(40.73345599660806 74.67401900630617) bank107078 +107079 POINT(40.55290830613727 74.17550025070541) bank107079 +107080 POINT(40.24696690087058 74.59367607719493) bank107080 +107081 POINT(41.69193252123913 73.49415616486395) bank107081 +107082 POINT(41.37617610393995 75.00493920971324) bank107082 +107083 POINT(41.46076600923634 73.95395586559525) bank107083 +107084 POINT(41.34571217232231 74.37071443136583) bank107084 +107085 POINT(40.915510053858064 74.57775983055599) bank107085 +107086 POINT(39.74472063571331 74.42895936236255) bank107086 +107087 POINT(39.91600229798758 73.22722126319) bank107087 +107088 POINT(41.41880170987604 74.02409457609748) bank107088 +107089 POINT(41.43528171040108 74.15374489786349) bank107089 +107090 POINT(40.38393863732037 74.62755196509724) bank107090 +107091 POINT(40.492749160677576 73.28527393604584) bank107091 +107092 POINT(41.302865294651426 73.498964521479) bank107092 +107093 POINT(41.10944588362163 74.54420407994583) bank107093 +107094 POINT(41.413685700293414 74.13194101929231) bank107094 +107095 POINT(41.3319855209867 74.91886615356051) bank107095 +107096 POINT(41.08514315944963 73.01502956680756) bank107096 +107097 POINT(41.017903091407625 73.24796448687283) bank107097 +107098 POINT(40.82085473366115 73.16380823592162) bank107098 +107099 POINT(41.458277325191986 74.51951611391692) bank107099 +107100 POINT(40.80181069293756 73.96086537638381) bank107100 +107101 POINT(40.48677665853959 74.96991826412095) bank107101 +107102 POINT(40.96849691091115 73.74432889817005) bank107102 +107103 POINT(40.4367532078239 74.21486354471081) bank107103 +107104 POINT(41.368579139251864 74.99212534557536) bank107104 +107105 POINT(39.79413480918402 74.90303239112791) bank107105 +107106 POINT(39.9396835727022 74.22287800063252) bank107106 +107107 POINT(41.00096614172727 74.21808437543145) bank107107 +107108 POINT(41.39382662697647 74.27619636556427) bank107108 +107109 POINT(40.55645332961073 74.66317702248146) bank107109 +107110 POINT(39.95260390387858 73.28835485668694) bank107110 +107111 POINT(40.13329329514133 73.17085389150883) bank107111 +107112 POINT(41.694281622715415 73.13555635527361) bank107112 +107113 POINT(40.96429699860375 73.14410115314476) bank107113 +107114 POINT(40.35693388890113 73.79081024274292) bank107114 +107115 POINT(40.30819913770482 74.0949911483787) bank107115 +107116 POINT(41.10923764682732 73.57908041303115) bank107116 +107117 POINT(41.70742022039621 73.91873714065896) bank107117 +107118 POINT(41.63061543480948 73.973551848633) bank107118 +107119 POINT(40.59685379601348 73.33238454828337) bank107119 +107120 POINT(41.594292479895486 74.49335362507357) bank107120 +107121 POINT(41.508140912799625 74.1652284574121) bank107121 +107122 POINT(41.32891043123236 74.2787064066233) bank107122 +107123 POINT(40.03101860240941 74.68773772852607) bank107123 +107124 POINT(40.14806110610323 74.45891192322644) bank107124 +107125 POINT(41.6892233931503 73.0809846656247) bank107125 +107126 POINT(40.30196076572335 73.30532013288848) bank107126 +107127 POINT(41.465860280667385 74.21847258617657) bank107127 +107128 POINT(40.07211389667116 74.95197023721245) bank107128 +107129 POINT(41.070983760268795 73.9332426803125) bank107129 +107130 POINT(39.96571365471482 74.48025967539346) bank107130 +107131 POINT(40.46656024871732 74.9540882188862) bank107131 +107132 POINT(41.50160276074734 74.98337749349285) bank107132 +107133 POINT(40.82591364084318 74.72349307192519) bank107133 +107134 POINT(41.018775839904606 74.13234388809485) bank107134 +107135 POINT(41.46942592101383 74.84581134519279) bank107135 +107136 POINT(40.926084120843115 74.17579094098818) bank107136 +107137 POINT(40.10446645034384 73.77157497985257) bank107137 +107138 POINT(40.87685670276892 74.36069475421115) bank107138 +107139 POINT(41.08881770594459 73.92918794070602) bank107139 +107140 POINT(41.57580978373429 74.01419020843728) bank107140 +107141 POINT(41.16324970709329 74.53850672820181) bank107141 +107142 POINT(40.69295474630482 73.7637611295921) bank107142 +107143 POINT(39.97077919561783 74.68893399704855) bank107143 +107144 POINT(40.15262006604521 73.51317609963864) bank107144 +107145 POINT(40.012755778990694 74.00609233525321) bank107145 +107146 POINT(41.01166798730029 74.90713286795511) bank107146 +107147 POINT(40.707598254178286 73.89440964081898) bank107147 +107148 POINT(40.971117464204546 74.50592458928398) bank107148 +107149 POINT(39.894686073367026 73.32241173492733) bank107149 +107150 POINT(40.82561260496598 73.77149984575493) bank107150 +107151 POINT(40.133017136409556 73.90206801461406) bank107151 +107152 POINT(41.01455262131904 74.38333726555932) bank107152 +107153 POINT(41.37014471605139 73.46214163549155) bank107153 +107154 POINT(41.5595554874384 73.60948210479847) bank107154 +107155 POINT(40.02529450074676 73.53301694187074) bank107155 +107156 POINT(41.22249770253129 73.66110836700773) bank107156 +107157 POINT(41.43163379746579 73.29758640739573) bank107157 +107158 POINT(40.061378987409405 73.10644752124048) bank107158 +107159 POINT(40.015567664844376 73.94646541330872) bank107159 +107160 POINT(41.394127501402004 74.86903860392525) bank107160 +107161 POINT(41.492850322132114 73.21241997599387) bank107161 +107162 POINT(41.54842088765551 73.15502296014331) bank107162 +107163 POINT(41.48677038952847 74.58420374611113) bank107163 +107164 POINT(41.16155055713362 74.27092442383469) bank107164 +107165 POINT(40.05324469333215 73.18519808212342) bank107165 +107166 POINT(40.3084209648247 74.41439435163817) bank107166 +107167 POINT(40.58057237549366 73.17934920468356) bank107167 +107168 POINT(40.988057885580524 74.66853933144013) bank107168 +107169 POINT(40.21683494708032 73.72591105558669) bank107169 +107170 POINT(40.05337438935194 73.78724012613581) bank107170 +107171 POINT(39.77709574261023 73.94298809477158) bank107171 +107172 POINT(39.906206435536 73.60226436579629) bank107172 +107173 POINT(40.378965462999865 74.51373416174218) bank107173 +107174 POINT(40.10811256357813 73.03223648846385) bank107174 +107175 POINT(40.889383807373235 73.78718808053411) bank107175 +107176 POINT(41.08115167959183 73.92504864505331) bank107176 +107177 POINT(40.23742670450596 74.83091913491886) bank107177 +107178 POINT(41.652522873893226 73.07963832514722) bank107178 +107179 POINT(40.43479225131047 73.43113490396725) bank107179 +107180 POINT(40.60702750096216 74.9605867293503) bank107180 +107181 POINT(40.27606401784164 74.67284710096506) bank107181 +107182 POINT(39.76208513098766 74.67341304195966) bank107182 +107183 POINT(40.42265661100258 73.67561218135664) bank107183 +107184 POINT(40.09574499643679 74.63936366343864) bank107184 +107185 POINT(41.517092125547386 73.9086067862485) bank107185 +107186 POINT(40.90121178717284 73.83124110670664) bank107186 +107187 POINT(40.88829693349449 73.45852587481978) bank107187 +107188 POINT(41.21519504781504 73.32364671730653) bank107188 +107189 POINT(39.73041492833873 73.84999639139447) bank107189 +107190 POINT(41.61100251081834 73.78917180960224) bank107190 +107191 POINT(41.357626904751676 74.43420087601218) bank107191 +107192 POINT(40.95990947075783 73.7751730867837) bank107192 +107193 POINT(40.237009805539394 74.5247963895567) bank107193 +107194 POINT(40.825443761730305 73.91635908818131) bank107194 +107195 POINT(41.41500306919135 73.85692483009927) bank107195 +107196 POINT(41.02728738229902 73.61906440434767) bank107196 +107197 POINT(40.78249388330373 73.48271624685543) bank107197 +107198 POINT(40.05948083626757 74.42353736314566) bank107198 +107199 POINT(40.30277135496345 73.89030374155071) bank107199 +107200 POINT(40.32224592265037 73.42701027617879) bank107200 +107201 POINT(40.36422427420296 74.18847174944503) bank107201 +107202 POINT(40.701600105414926 74.2372952384108) bank107202 +107203 POINT(40.56155643645251 74.52585446260574) bank107203 +107204 POINT(40.461153526249234 73.56883794162384) bank107204 +107205 POINT(40.348662761017934 74.81637400376587) bank107205 +107206 POINT(40.81022994726695 74.1406661811564) bank107206 +107207 POINT(41.03642890456762 74.42705779979305) bank107207 +107208 POINT(40.04161583071083 74.23630858901417) bank107208 +107209 POINT(40.1302269560803 73.74780371895402) bank107209 +107210 POINT(41.559267141708254 73.14656936307739) bank107210 +107211 POINT(40.86539967733389 73.16279250290373) bank107211 +107212 POINT(40.555466185431854 74.08121089972816) bank107212 +107213 POINT(41.207744986272424 74.94690298457141) bank107213 +107214 POINT(40.93015013214777 73.7314028365196) bank107214 +107215 POINT(40.56553504024376 73.03988504144324) bank107215 +107216 POINT(39.77931920500693 74.55404591963917) bank107216 +107217 POINT(39.788079734182816 74.24619843014014) bank107217 +107218 POINT(40.523374178309304 73.27548208441524) bank107218 +107219 POINT(39.86529945835861 73.84470151323364) bank107219 +107220 POINT(40.48521603218362 73.1839430018495) bank107220 +107221 POINT(41.36825174695399 73.31366984722328) bank107221 +107222 POINT(40.725052808524374 74.47045186235589) bank107222 +107223 POINT(40.9315856363761 73.44841551094207) bank107223 +107224 POINT(39.799963971765315 74.55060491868957) bank107224 +107225 POINT(40.7092684565514 74.90871299217379) bank107225 +107226 POINT(40.003862236696875 73.69475107853245) bank107226 +107227 POINT(40.21103117885981 74.2755335571333) bank107227 +107228 POINT(40.50270373763854 73.95557117449314) bank107228 +107229 POINT(39.72605266592498 73.41710123264794) bank107229 +107230 POINT(39.899333578760455 74.46642466296194) bank107230 +107231 POINT(40.856797837198776 74.33196027173105) bank107231 +107232 POINT(40.00042087322589 73.42995661277769) bank107232 +107233 POINT(41.56481207122997 74.12842100621799) bank107233 +107234 POINT(39.89253071139345 73.24807361129949) bank107234 +107235 POINT(40.53034366965675 74.57211804092805) bank107235 +107236 POINT(40.31280321081059 73.26454196883793) bank107236 +107237 POINT(40.47776345871027 74.4286181268237) bank107237 +107238 POINT(41.62171848622582 74.01873738984713) bank107238 +107239 POINT(41.67490147863539 74.94843147430848) bank107239 +107240 POINT(40.65943172719495 73.0466840210023) bank107240 +107241 POINT(41.232898176611975 74.61115049578589) bank107241 +107242 POINT(41.281602321117155 73.74079108975305) bank107242 +107243 POINT(39.84321260471608 73.55309381140565) bank107243 +107244 POINT(40.068628980148326 73.17960895140713) bank107244 +107245 POINT(40.49978537144418 74.68388821599278) bank107245 +107246 POINT(41.204270560796104 74.20710684080886) bank107246 +107247 POINT(41.63401629896615 74.25830164489699) bank107247 +107248 POINT(39.81096527047328 74.40984580577914) bank107248 +107249 POINT(41.432841438136016 73.2894623007781) bank107249 +107250 POINT(41.02604016055216 73.89262593039544) bank107250 +107251 POINT(41.09646719949488 74.18805794309934) bank107251 +107252 POINT(40.33510431415369 74.48181298728225) bank107252 +107253 POINT(40.79807765855836 73.67563999596271) bank107253 +107254 POINT(39.76303275729249 74.82804580916755) bank107254 +107255 POINT(41.58798187432346 74.5100572625462) bank107255 +107256 POINT(41.329291985135555 73.0149156970738) bank107256 +107257 POINT(41.08084273435142 73.59735377982439) bank107257 +107258 POINT(41.49888378802506 74.54389299536726) bank107258 +107259 POINT(41.705835394654024 74.97061530419955) bank107259 +107260 POINT(40.72876171476067 73.38432677607112) bank107260 +107261 POINT(40.21187411636301 73.27448305909144) bank107261 +107262 POINT(41.48389007609846 73.93859029883495) bank107262 +107263 POINT(41.39718309630634 73.58645081435655) bank107263 +107264 POINT(40.34632957412728 73.95872568761476) bank107264 +107265 POINT(41.29745210400157 74.07090873250084) bank107265 +107266 POINT(40.82860016238097 73.41448422667598) bank107266 +107267 POINT(39.91946923750818 74.65569814475087) bank107267 +107268 POINT(40.31958393928512 74.75028954008468) bank107268 +107269 POINT(39.96280067545471 73.06784451493662) bank107269 +107270 POINT(40.32453632560055 73.3738269481866) bank107270 +107271 POINT(40.720081651800974 74.10497496868541) bank107271 +107272 POINT(41.22384756541508 73.3615195218456) bank107272 +107273 POINT(40.81383534474379 74.32965979457052) bank107273 +107274 POINT(41.16188346956169 74.29531361892698) bank107274 +107275 POINT(40.368495584468945 74.88769211290314) bank107275 +107276 POINT(40.7001275764599 73.60230121350538) bank107276 +107277 POINT(40.25179713193004 74.3442771341267) bank107277 +107278 POINT(40.62279039093235 73.25465079909962) bank107278 +107279 POINT(41.14130993309415 73.49818128519284) bank107279 +107280 POINT(39.945453868893146 74.72140021234257) bank107280 +107281 POINT(40.1535670436938 73.17503457792606) bank107281 +107282 POINT(40.66654378053025 74.54939069190021) bank107282 +107283 POINT(40.89528694315019 73.51741120678749) bank107283 +107284 POINT(39.752636290316005 74.57214328176036) bank107284 +107285 POINT(40.96895523723914 74.85354471234723) bank107285 +107286 POINT(40.08311110101224 74.55321289634895) bank107286 +107287 POINT(41.63430722094476 74.71212605868179) bank107287 +107288 POINT(41.698562017364786 74.7084182202218) bank107288 +107289 POINT(40.75461711683772 73.60193753631073) bank107289 +107290 POINT(40.638829285816904 74.1198022871739) bank107290 +107291 POINT(40.29002209850087 74.5799800389067) bank107291 +107292 POINT(40.119305456028634 74.94180317895402) bank107292 +107293 POINT(41.459647269251214 74.29939434345555) bank107293 +107294 POINT(41.52924474629347 74.20528465683743) bank107294 +107295 POINT(40.377979193172905 74.61293188732168) bank107295 +107296 POINT(40.41227735455927 73.11388372816415) bank107296 +107297 POINT(40.19271891164407 73.03172960097532) bank107297 +107298 POINT(40.764974692992176 73.24168180544244) bank107298 +107299 POINT(40.468553159185724 73.54375795398535) bank107299 +107300 POINT(40.47608491145404 74.3537261501038) bank107300 +107301 POINT(41.23304531431955 73.32182038853642) bank107301 +107302 POINT(40.98570337830054 73.32309024699943) bank107302 +107303 POINT(41.13601180837336 74.18147900460535) bank107303 +107304 POINT(41.36877648601314 73.14307751850534) bank107304 +107305 POINT(39.956186856160464 73.6328884898844) bank107305 +107306 POINT(40.72752324232046 73.1671711441754) bank107306 +107307 POINT(39.85054609003287 73.83929797209119) bank107307 +107308 POINT(40.212215276517085 74.50259902275732) bank107308 +107309 POINT(41.531357461831156 74.95725840765778) bank107309 +107310 POINT(39.894460720712175 73.1157963574227) bank107310 +107311 POINT(40.65015948928739 74.08133969378966) bank107311 +107312 POINT(40.29925463820672 74.7464483650457) bank107312 +107313 POINT(41.29351893266627 74.38657612062184) bank107313 +107314 POINT(40.037063706844776 74.82777921011083) bank107314 +107315 POINT(41.599491437210496 74.39669636939216) bank107315 +107316 POINT(40.31811814075467 73.41969877301084) bank107316 +107317 POINT(41.289223796581354 73.83983390753362) bank107317 +107318 POINT(40.0965477323788 74.11523758692493) bank107318 +107319 POINT(39.88242070247713 73.4095950118088) bank107319 +107320 POINT(40.86494910087028 74.93915777616458) bank107320 +107321 POINT(41.41295433392899 74.79674469576334) bank107321 +107322 POINT(41.28342137541501 74.30527204166272) bank107322 +107323 POINT(39.91333164571203 74.36473284176893) bank107323 +107324 POINT(40.36434377612495 73.23778906097142) bank107324 +107325 POINT(39.73421673682076 73.44223028045128) bank107325 +107326 POINT(40.55578094595123 74.76457940920294) bank107326 +107327 POINT(39.89694617792392 74.57421975822244) bank107327 +107328 POINT(39.852097969627856 74.20867430086506) bank107328 +107329 POINT(40.13900872607563 74.1193966590352) bank107329 +107330 POINT(41.66862590094882 73.60104777703066) bank107330 +107331 POINT(40.92899748227209 73.6916442280491) bank107331 +107332 POINT(40.799977077766805 74.66256860760986) bank107332 +107333 POINT(40.87589125273082 74.29208513527699) bank107333 +107334 POINT(40.21682775126 73.75539283917104) bank107334 +107335 POINT(39.98443612030336 74.78332646035909) bank107335 +107336 POINT(40.99545672516397 73.32952452880066) bank107336 +107337 POINT(41.35769278822184 74.45263786461464) bank107337 +107338 POINT(39.81525740692554 74.78624716976549) bank107338 +107339 POINT(40.650130397528855 74.96567233552501) bank107339 +107340 POINT(40.43387447713814 74.33472727577752) bank107340 +107341 POINT(40.96149251989527 73.47445030310814) bank107341 +107342 POINT(40.44042632632507 73.03783204898791) bank107342 +107343 POINT(40.20580087855681 74.29613338232824) bank107343 +107344 POINT(39.85973610403145 73.73676922589716) bank107344 +107345 POINT(41.03799801566774 74.10666460068069) bank107345 +107346 POINT(40.737389913470174 73.77040045931714) bank107346 +107347 POINT(40.684227429270145 73.88988549295321) bank107347 +107348 POINT(40.18792285253282 74.04328593082818) bank107348 +107349 POINT(40.07026900829099 74.07012318737641) bank107349 +107350 POINT(41.485233660842184 73.55118881740056) bank107350 +107351 POINT(39.88647012405889 74.21314906174302) bank107351 +107352 POINT(40.01961033382758 73.79908374409325) bank107352 +107353 POINT(40.132043235907894 74.92217893536251) bank107353 +107354 POINT(41.5590131630285 73.05310504796056) bank107354 +107355 POINT(39.830572185449185 73.0349766864094) bank107355 +107356 POINT(41.55573602145754 74.1024064738199) bank107356 +107357 POINT(39.978374305415805 73.24145342333946) bank107357 +107358 POINT(41.6525581837921 74.38649835436601) bank107358 +107359 POINT(40.17181447127636 74.27606541920984) bank107359 +107360 POINT(40.960431752300664 73.14601304517599) bank107360 +107361 POINT(41.068048641286865 74.01216995000796) bank107361 +107362 POINT(40.7909394900809 74.89715160253459) bank107362 +107363 POINT(40.48305610701191 74.48876568604237) bank107363 +107364 POINT(40.9846971601411 73.71465713876472) bank107364 +107365 POINT(41.11862393150467 73.14292006857217) bank107365 +107366 POINT(41.05179865224916 73.30160816975355) bank107366 +107367 POINT(39.824555074285506 73.1644980652236) bank107367 +107368 POINT(40.608116945909885 74.13053077300306) bank107368 +107369 POINT(40.67017295540037 73.24551980356755) bank107369 +107370 POINT(40.95846272518471 74.4324400201899) bank107370 +107371 POINT(40.027923885771415 73.32748144309676) bank107371 +107372 POINT(39.766103278501326 73.02626100717752) bank107372 +107373 POINT(41.245942840151905 73.98470310280793) bank107373 +107374 POINT(40.911741712462884 73.37858163997598) bank107374 +107375 POINT(41.28839842933919 74.74857366074448) bank107375 +107376 POINT(40.0941955067037 74.82955129213124) bank107376 +107377 POINT(39.76842508916563 73.13850050462085) bank107377 +107378 POINT(39.879002114477615 74.86443585386327) bank107378 +107379 POINT(40.663084394743024 73.5819440324377) bank107379 +107380 POINT(40.43531163707114 74.70797612307966) bank107380 +107381 POINT(41.05953552423372 73.57058266873943) bank107381 +107382 POINT(40.09761857438552 74.98107716565255) bank107382 +107383 POINT(41.59637129327285 74.04149877427902) bank107383 +107384 POINT(40.62187356060681 74.4296425293843) bank107384 +107385 POINT(39.87969201602126 74.02812376376387) bank107385 +107386 POINT(41.5278153505909 74.00102955125072) bank107386 +107387 POINT(40.88550227188425 74.22595307044436) bank107387 +107388 POINT(41.34294853187925 73.0616118747506) bank107388 +107389 POINT(39.946925051448986 73.2357803615659) bank107389 +107390 POINT(39.88122393486419 74.69402992310228) bank107390 +107391 POINT(40.270229836231614 73.27521537661922) bank107391 +107392 POINT(40.896747998889644 73.723766887853) bank107392 +107393 POINT(40.498922506484284 73.13543046651212) bank107393 +107394 POINT(41.56371328978955 74.3171415069858) bank107394 +107395 POINT(40.54525428902684 73.62885098783674) bank107395 +107396 POINT(41.059844360954976 74.74164559702237) bank107396 +107397 POINT(40.990060224790795 74.5041252264211) bank107397 +107398 POINT(40.03847443135357 73.53739483929868) bank107398 +107399 POINT(40.496347287240326 73.73379367010284) bank107399 +107400 POINT(40.59774296601181 74.79002547308654) bank107400 +107401 POINT(41.583950319224634 74.96904329717403) bank107401 +107402 POINT(40.75939596611316 74.36304195328398) bank107402 +107403 POINT(40.2335786633732 74.45694557982681) bank107403 +107404 POINT(40.45113414455988 73.45282842144285) bank107404 +107405 POINT(40.57891788349895 74.62490479923147) bank107405 +107406 POINT(40.23274235273651 73.42149721803662) bank107406 +107407 POINT(41.156582481381555 74.31917739420446) bank107407 +107408 POINT(41.68593831093397 73.66415675908755) bank107408 +107409 POINT(40.61379023799863 74.10173876166782) bank107409 +107410 POINT(40.05478602422389 73.47032856041635) bank107410 +107411 POINT(41.709519912907574 73.33196546401027) bank107411 +107412 POINT(40.497987605063685 74.63236683557166) bank107412 +107413 POINT(40.30055019182263 74.54436246235595) bank107413 +107414 POINT(40.702370883622784 73.9394416436019) bank107414 +107415 POINT(41.542198431269696 74.13881537873975) bank107415 +107416 POINT(41.24453944261985 73.8555159780044) bank107416 +107417 POINT(39.944109204115485 74.26165217142012) bank107417 +107418 POINT(39.9749958982243 74.69824783825584) bank107418 +107419 POINT(41.49364498526222 74.10836450356408) bank107419 +107420 POINT(40.98679714605912 73.69784334606003) bank107420 +107421 POINT(39.89510789376168 74.92094116762416) bank107421 +107422 POINT(40.18386714480594 74.58956022905699) bank107422 +107423 POINT(40.86479082219025 73.53797938085505) bank107423 +107424 POINT(40.617731674240936 73.97069049389457) bank107424 +107425 POINT(39.90130879445787 74.27111894728193) bank107425 +107426 POINT(40.0130814714638 74.36043739376336) bank107426 +107427 POINT(40.87556179370686 74.22449842981227) bank107427 +107428 POINT(40.89279905495717 73.51607124458006) bank107428 +107429 POINT(41.29598016544679 73.02915369511342) bank107429 +107430 POINT(40.76794136191883 74.56915400761548) bank107430 +107431 POINT(40.06563354928747 74.1279695246693) bank107431 +107432 POINT(40.556005528963006 73.04030226779699) bank107432 +107433 POINT(40.13376345084821 74.51441198655883) bank107433 +107434 POINT(41.44779854843575 73.92160655502774) bank107434 +107435 POINT(41.029231425902246 73.5830495578116) bank107435 +107436 POINT(41.45031909619776 74.31505041377892) bank107436 +107437 POINT(40.12078460677648 74.72741391783386) bank107437 +107438 POINT(41.53875592631696 73.45487819841968) bank107438 +107439 POINT(40.33708678713685 73.59844182942062) bank107439 +107440 POINT(41.15517815811214 73.90664798355026) bank107440 +107441 POINT(39.73127628703756 73.67136782189515) bank107441 +107442 POINT(40.60892538469038 73.13949161887749) bank107442 +107443 POINT(39.8401508610841 73.02628566708947) bank107443 +107444 POINT(39.73253349733439 74.30742651268413) bank107444 +107445 POINT(40.740430419636766 74.42420833380446) bank107445 +107446 POINT(40.50696903152819 73.17963793826408) bank107446 +107447 POINT(40.02709963048454 73.25371196860422) bank107447 +107448 POINT(40.38076867708767 73.55536729927037) bank107448 +107449 POINT(41.00211215828598 73.59679000611104) bank107449 +107450 POINT(40.46619179412752 73.8763670779498) bank107450 +107451 POINT(40.206750464696704 73.86178170806014) bank107451 +107452 POINT(40.768194604134166 73.14676630611639) bank107452 +107453 POINT(40.75903779130961 74.84439058189058) bank107453 +107454 POINT(41.235771240850056 74.33051530188855) bank107454 +107455 POINT(41.44297396660192 73.04513183053207) bank107455 +107456 POINT(41.025242816442905 73.80759950141132) bank107456 +107457 POINT(39.82103698176208 74.87749558196514) bank107457 +107458 POINT(40.7422711001912 73.52374658948784) bank107458 +107459 POINT(40.74317440953062 74.05247666776847) bank107459 +107460 POINT(41.38796585273073 74.80135764609012) bank107460 +107461 POINT(40.799137975368694 73.38998573780509) bank107461 +107462 POINT(41.14103721432594 73.09439848172009) bank107462 +107463 POINT(41.49662754427114 74.84380508463933) bank107463 +107464 POINT(41.04322113586045 74.8313816704126) bank107464 +107465 POINT(40.58247793607807 74.98401052356209) bank107465 +107466 POINT(41.66182393589485 74.43916168654071) bank107466 +107467 POINT(41.09218899282091 74.58594110307709) bank107467 +107468 POINT(40.7522376374569 74.87764263793811) bank107468 +107469 POINT(39.75884232611878 73.73639686201871) bank107469 +107470 POINT(40.89640460585162 74.77264608998615) bank107470 +107471 POINT(39.823694594836034 73.5793228006083) bank107471 +107472 POINT(40.08153770732751 74.0523317252869) bank107472 +107473 POINT(41.33133627438319 74.20080392212574) bank107473 +107474 POINT(40.133087838590335 74.46895531498396) bank107474 +107475 POINT(40.15595795493998 73.15071425026018) bank107475 +107476 POINT(39.85547448679385 73.65481060308451) bank107476 +107477 POINT(40.53925692851611 74.62317427244245) bank107477 +107478 POINT(41.04781380397077 74.92984493468107) bank107478 +107479 POINT(41.19007757937961 73.1440417379012) bank107479 +107480 POINT(40.55169494060364 73.32833951834044) bank107480 +107481 POINT(40.83462747144866 73.27860840871861) bank107481 +107482 POINT(39.74756090407536 74.9316178239371) bank107482 +107483 POINT(40.167858064277944 74.56670255146237) bank107483 +107484 POINT(41.45150207181947 74.81563556464597) bank107484 +107485 POINT(40.59811515639843 74.46672796134926) bank107485 +107486 POINT(40.10494395265601 74.39066121562662) bank107486 +107487 POINT(41.59292509296238 73.85112420437595) bank107487 +107488 POINT(39.71373079338924 74.73070212707107) bank107488 +107489 POINT(41.284840224559126 73.53936752709754) bank107489 +107490 POINT(40.268165221439126 74.89069955391021) bank107490 +107491 POINT(40.66464067245532 74.6931081836617) bank107491 +107492 POINT(41.149334008417895 74.02537151523018) bank107492 +107493 POINT(41.55323802495199 74.71662601996059) bank107493 +107494 POINT(40.18143010999817 74.36647278041744) bank107494 +107495 POINT(41.088738040207346 73.6934664019004) bank107495 +107496 POINT(40.44850100962073 74.29066133203507) bank107496 +107497 POINT(40.73976399633305 73.33151197584887) bank107497 +107498 POINT(40.111440087081284 73.79759639817446) bank107498 +107499 POINT(40.877189684438484 74.49387680428059) bank107499 +107500 POINT(41.063848115706186 73.5458841838136) bank107500 +107501 POINT(41.173676120616676 73.32083499512528) bank107501 +107502 POINT(41.10390699206692 73.54344158325222) bank107502 +107503 POINT(41.244348358998295 73.51187182869964) bank107503 +107504 POINT(40.97706249095877 74.66309564713458) bank107504 +107505 POINT(41.50474447468106 73.76675578137899) bank107505 +107506 POINT(41.36980630006002 73.2713284392595) bank107506 +107507 POINT(40.369315436706266 73.54258404893531) bank107507 +107508 POINT(39.977103072575105 74.35804413865884) bank107508 +107509 POINT(40.55677466905872 73.72941168769438) bank107509 +107510 POINT(40.32080115220752 74.16203802939208) bank107510 +107511 POINT(41.216080390918385 74.87039639307021) bank107511 +107512 POINT(40.586070098057135 74.58089075756146) bank107512 +107513 POINT(41.1371826308646 73.46596271716214) bank107513 +107514 POINT(39.795237783086016 74.74581559334194) bank107514 +107515 POINT(39.88524925965355 74.40208208560877) bank107515 +107516 POINT(40.53082967041173 74.24705729805076) bank107516 +107517 POINT(41.398051208366674 74.87533081552148) bank107517 +107518 POINT(40.148729381392144 74.47787027075998) bank107518 +107519 POINT(40.737932230508086 73.55752262851321) bank107519 +107520 POINT(40.94183664986038 73.70489656011233) bank107520 +107521 POINT(41.32299845769457 73.86498246320299) bank107521 +107522 POINT(40.42793848861073 73.7367570418437) bank107522 +107523 POINT(39.91152128225649 74.23431951162681) bank107523 +107524 POINT(40.579390222992735 73.36292257185046) bank107524 +107525 POINT(40.80268140687313 73.29353947139894) bank107525 +107526 POINT(39.85864872900193 74.73943268968993) bank107526 +107527 POINT(40.17340915787695 74.79090639370602) bank107527 +107528 POINT(41.701386130581824 73.68345690681603) bank107528 +107529 POINT(40.10362633121491 73.55791044425982) bank107529 +107530 POINT(39.9258190960734 73.51518620315247) bank107530 +107531 POINT(40.53883913934337 73.17298029043432) bank107531 +107532 POINT(41.53671959977875 73.34902237982519) bank107532 +107533 POINT(39.943652660109976 73.99074499108188) bank107533 +107534 POINT(40.936180427602736 73.35006481062216) bank107534 +107535 POINT(41.54152107302205 74.32384172964478) bank107535 +107536 POINT(40.511867093914645 73.67606050630283) bank107536 +107537 POINT(40.57993886652693 74.48021372718439) bank107537 +107538 POINT(41.07574977067789 74.15880009861615) bank107538 +107539 POINT(40.460024662710275 73.12337753825561) bank107539 +107540 POINT(40.30693896294745 74.69239027571527) bank107540 +107541 POINT(41.58194700229859 74.98622228399186) bank107541 +107542 POINT(41.197296161608634 74.3952140680478) bank107542 +107543 POINT(41.61222962384735 73.53420729099406) bank107543 +107544 POINT(40.642212856684296 74.48616977994587) bank107544 +107545 POINT(41.51094754200711 73.070678926186) bank107545 +107546 POINT(41.36517960408861 74.17643435475699) bank107546 +107547 POINT(40.89601463625827 73.86527563468572) bank107547 +107548 POINT(41.43013295175535 74.24512068316106) bank107548 +107549 POINT(41.01323764906802 74.39372000864732) bank107549 +107550 POINT(41.45945746169804 73.95209232028775) bank107550 +107551 POINT(40.01537480868647 73.31976975916636) bank107551 +107552 POINT(40.596867016015366 74.3238543044006) bank107552 +107553 POINT(40.39683043510823 73.54325050320537) bank107553 +107554 POINT(40.89192951474543 73.12941073507287) bank107554 +107555 POINT(40.17733457399869 74.39371793379534) bank107555 +107556 POINT(40.38334589996006 74.31019952910242) bank107556 +107557 POINT(41.70991716442526 74.21413684112946) bank107557 +107558 POINT(40.908364545623705 74.85878749984252) bank107558 +107559 POINT(41.36276672915347 74.35555841469967) bank107559 +107560 POINT(41.67946487046985 74.2691557998034) bank107560 +107561 POINT(41.632414604243785 73.12600260355583) bank107561 +107562 POINT(40.36953231733082 74.10664603112204) bank107562 +107563 POINT(39.767783466654286 73.31363281036784) bank107563 +107564 POINT(41.245479197361185 74.79714198116278) bank107564 +107565 POINT(40.92085466504288 73.01769164588754) bank107565 +107566 POINT(39.721491590241286 74.17548423043806) bank107566 +107567 POINT(40.53972686415416 74.15204882574308) bank107567 +107568 POINT(40.03824802679021 74.57247791057488) bank107568 +107569 POINT(41.23237340397384 73.66642545171025) bank107569 +107570 POINT(39.98795281265072 74.39636531598634) bank107570 +107571 POINT(41.180429463667515 73.56173445662823) bank107571 +107572 POINT(41.140355446596 73.57856506072183) bank107572 +107573 POINT(41.077822721975984 73.83745580194275) bank107573 +107574 POINT(40.05596041242223 74.72771500719149) bank107574 +107575 POINT(40.01367742220094 73.63039932276682) bank107575 +107576 POINT(40.69725505390591 75.00040893198354) bank107576 +107577 POINT(39.81501007186452 73.4974600656013) bank107577 +107578 POINT(39.915617392556705 73.76907747724104) bank107578 +107579 POINT(40.33967734965401 74.81798911162336) bank107579 +107580 POINT(40.59138044389742 74.71130757425246) bank107580 +107581 POINT(41.32690910525529 74.95382619124481) bank107581 +107582 POINT(41.07945569242192 74.22335886270065) bank107582 +107583 POINT(41.168659233535266 74.14995392343859) bank107583 +107584 POINT(41.25058247774758 74.41074986585912) bank107584 +107585 POINT(41.32575015455609 73.41958786104163) bank107585 +107586 POINT(41.081084505581714 73.33525053478661) bank107586 +107587 POINT(40.98075285473399 73.12361848537375) bank107587 +107588 POINT(41.60506323119946 73.9202087985695) bank107588 +107589 POINT(41.53705343791894 74.1109324013606) bank107589 +107590 POINT(41.183019137510186 74.50634238469253) bank107590 +107591 POINT(40.60235728972609 74.39556772579391) bank107591 +107592 POINT(41.68898118357837 73.02626397600075) bank107592 +107593 POINT(40.60874041575408 74.32485546736109) bank107593 +107594 POINT(40.80479293383836 73.23032049284352) bank107594 +107595 POINT(40.81120411366133 73.73094843610706) bank107595 +107596 POINT(41.09046896602575 73.55730574283795) bank107596 +107597 POINT(41.01331713802728 73.69287170561176) bank107597 +107598 POINT(40.32603570372905 73.24408786518367) bank107598 +107599 POINT(39.91975567139078 74.99382101647703) bank107599 +107600 POINT(41.700554534671134 74.00899498295739) bank107600 +107601 POINT(41.6420847317642 73.28053746257227) bank107601 +107602 POINT(41.49297486311024 74.29245512596461) bank107602 +107603 POINT(39.712812877397745 74.15253924413102) bank107603 +107604 POINT(41.52975899361287 74.23129457256726) bank107604 +107605 POINT(41.04676449873354 73.66900648855395) bank107605 +107606 POINT(39.98232006884392 73.45424150272467) bank107606 +107607 POINT(39.756823075628205 74.65658124171027) bank107607 +107608 POINT(41.00266197201636 73.83962440121435) bank107608 +107609 POINT(40.30189318551494 73.91125862776055) bank107609 +107610 POINT(39.85284872866321 73.83472072191516) bank107610 +107611 POINT(40.175153981999564 74.97183825301225) bank107611 +107612 POINT(41.12499312516183 74.0059788266507) bank107612 +107613 POINT(41.69461968983775 75.0030525203366) bank107613 +107614 POINT(40.726716512752006 73.8067709918048) bank107614 +107615 POINT(39.80764318909172 74.40682678151789) bank107615 +107616 POINT(41.40901827391546 73.69103297149783) bank107616 +107617 POINT(40.50223002710536 74.24767046509422) bank107617 +107618 POINT(40.85726197008682 74.27332673898813) bank107618 +107619 POINT(40.557953829679796 74.70907742457035) bank107619 +107620 POINT(40.68327322572053 74.61665245727404) bank107620 +107621 POINT(40.10327920322309 73.08030520943464) bank107621 +107622 POINT(40.33619037885114 74.45509612894564) bank107622 +107623 POINT(40.3830331865405 73.99290342152214) bank107623 +107624 POINT(40.801692053807265 74.59070443273241) bank107624 +107625 POINT(40.68506825290411 73.47079180143237) bank107625 +107626 POINT(40.81170307605975 74.16072863489022) bank107626 +107627 POINT(41.06877170478762 74.09494091425863) bank107627 +107628 POINT(40.08138087166456 73.43348373157693) bank107628 +107629 POINT(41.627330284518855 73.97881754129337) bank107629 +107630 POINT(40.22887752734613 74.648403977035) bank107630 +107631 POINT(41.67957073080481 73.63090712598249) bank107631 +107632 POINT(41.082505247806175 73.60797545446688) bank107632 +107633 POINT(41.238638767624494 74.5209340288621) bank107633 +107634 POINT(41.36569936885951 73.90374334708979) bank107634 +107635 POINT(40.803634488070394 73.36639276309792) bank107635 +107636 POINT(40.26153626629087 74.79467077653693) bank107636 +107637 POINT(41.69748884420793 74.31698902880387) bank107637 +107638 POINT(40.76991644899024 74.21411329445662) bank107638 +107639 POINT(40.10458579090827 74.81072204133339) bank107639 +107640 POINT(40.85320598948828 74.78319321667149) bank107640 +107641 POINT(40.50258347580838 73.30207895291781) bank107641 +107642 POINT(40.13833276677059 74.17410374019384) bank107642 +107643 POINT(41.2066257365465 73.71429837183871) bank107643 +107644 POINT(40.63976610983261 73.68996434866727) bank107644 +107645 POINT(41.49416852902534 74.64769421631456) bank107645 +107646 POINT(41.634304920435206 74.4492528135794) bank107646 +107647 POINT(40.320866217906165 73.26071523824064) bank107647 +107648 POINT(41.305455354217635 74.00091372912945) bank107648 +107649 POINT(40.5032812495834 74.23243011166517) bank107649 +107650 POINT(40.73809830895717 74.54576749327671) bank107650 +107651 POINT(41.57843831154635 74.1351724606999) bank107651 +107652 POINT(40.481300094585634 73.50429364529629) bank107652 +107653 POINT(41.62403244109734 74.20934273228389) bank107653 +107654 POINT(40.78909247896948 73.29042776608502) bank107654 +107655 POINT(40.347468708419854 73.36867745289689) bank107655 +107656 POINT(40.45413445000667 74.29989488975698) bank107656 +107657 POINT(41.257417207119225 73.75022185538546) bank107657 +107658 POINT(41.10633992327325 73.58621389489356) bank107658 +107659 POINT(39.736478597156676 73.40927359131466) bank107659 +107660 POINT(41.65815551780633 74.63535577271034) bank107660 +107661 POINT(41.567005580920615 73.79377087769966) bank107661 +107662 POINT(41.19113150988425 73.99758308232808) bank107662 +107663 POINT(41.67719781325998 74.17199625946108) bank107663 +107664 POINT(40.48972916265969 74.38333053728377) bank107664 +107665 POINT(40.59296680743213 73.41082319307895) bank107665 +107666 POINT(40.01633996499133 74.7372068967531) bank107666 +107667 POINT(41.624058674126786 73.771145549367) bank107667 +107668 POINT(41.604351252218 73.49650518905034) bank107668 +107669 POINT(39.84892650681863 74.78647068007345) bank107669 +107670 POINT(39.947911466081784 73.34258084941719) bank107670 +107671 POINT(40.87695504960935 73.07364872658174) bank107671 +107672 POINT(40.897280132021436 74.41243894058448) bank107672 +107673 POINT(39.94780728375708 73.65449904145329) bank107673 +107674 POINT(41.191682199088426 73.59575886618651) bank107674 +107675 POINT(41.52644033371774 73.63044103872716) bank107675 +107676 POINT(40.649174226529034 74.01565148870132) bank107676 +107677 POINT(40.3279013996183 74.06338923300741) bank107677 +107678 POINT(39.85992094334067 74.28785214160035) bank107678 +107679 POINT(41.6048936465833 73.53771166895181) bank107679 +107680 POINT(41.241787504154026 74.38258113744624) bank107680 +107681 POINT(40.88249392927934 74.35182695102849) bank107681 +107682 POINT(40.470942494526966 73.26822740728147) bank107682 +107683 POINT(41.27493058975292 73.89419251194452) bank107683 +107684 POINT(39.737203973976534 73.02276924557455) bank107684 +107685 POINT(40.88935580728143 73.67661314817333) bank107685 +107686 POINT(39.782820318067095 73.1681169840828) bank107686 +107687 POINT(40.17119723343311 73.52400588503303) bank107687 +107688 POINT(39.98636848971817 74.53469381400198) bank107688 +107689 POINT(41.31578159667848 73.56149029315398) bank107689 +107690 POINT(40.33472362158638 73.55482197781053) bank107690 +107691 POINT(41.237166745555804 73.30986798248081) bank107691 +107692 POINT(40.94986703778761 73.35652230282932) bank107692 +107693 POINT(40.331771601880156 73.71009774964084) bank107693 +107694 POINT(39.78770374727427 74.34126105003004) bank107694 +107695 POINT(40.39737761915439 74.99420397071196) bank107695 +107696 POINT(40.36885909727256 74.32416609382737) bank107696 +107697 POINT(41.283964842851304 74.33777339926192) bank107697 +107698 POINT(41.607157860202484 73.1002872434281) bank107698 +107699 POINT(40.01513456168567 74.12536232678279) bank107699 +107700 POINT(41.09919990216184 73.37483734271477) bank107700 +107701 POINT(40.16708799274024 74.7820553439476) bank107701 +107702 POINT(41.33579965518855 73.79665669910031) bank107702 +107703 POINT(41.27722553234058 73.02667402759307) bank107703 +107704 POINT(40.96979899544713 73.94537320945332) bank107704 +107705 POINT(41.13926915580379 74.51456432737301) bank107705 +107706 POINT(41.53011315860525 73.40801130715501) bank107706 +107707 POINT(39.77707888758541 74.77950021026945) bank107707 +107708 POINT(41.04967136236888 73.18358620866174) bank107708 +107709 POINT(41.060032061404556 73.17127242142219) bank107709 +107710 POINT(39.74373873515862 73.32778899609127) bank107710 +107711 POINT(40.11061050446531 74.68926774562536) bank107711 +107712 POINT(40.614901252292476 73.87232310312103) bank107712 +107713 POINT(39.85800402910224 74.16300096904354) bank107713 +107714 POINT(40.85317594939945 73.58647085345349) bank107714 +107715 POINT(41.28708215532577 73.62458494258149) bank107715 +107716 POINT(40.484841625866196 74.27914483286565) bank107716 +107717 POINT(40.684166433901254 74.86296758272132) bank107717 +107718 POINT(39.850382175714266 74.20662860273822) bank107718 +107719 POINT(40.18231205103491 74.58561123461169) bank107719 +107720 POINT(40.383834374765826 73.43037615523501) bank107720 +107721 POINT(41.44885962059034 73.35851080394932) bank107721 +107722 POINT(39.93083696896644 74.28540854910723) bank107722 +107723 POINT(40.05446736229118 74.23654917155643) bank107723 +107724 POINT(40.1386715918102 74.71542165887512) bank107724 +107725 POINT(41.668865243020726 74.27933720074968) bank107725 +107726 POINT(40.87870696912303 74.21392368597738) bank107726 +107727 POINT(41.50011406925375 74.24404726998087) bank107727 +107728 POINT(40.26795320072772 73.5554449038718) bank107728 +107729 POINT(41.63443436394127 74.52641115148681) bank107729 +107730 POINT(40.255824126131785 73.51925115630328) bank107730 +107731 POINT(40.63910650054594 74.5589477436828) bank107731 +107732 POINT(40.86480256426199 73.47116844730738) bank107732 +107733 POINT(40.072241966303466 73.97580288904571) bank107733 +107734 POINT(40.41245810739489 74.70541775583196) bank107734 +107735 POINT(41.42400061862728 73.76806963775331) bank107735 +107736 POINT(40.58011663388995 73.57921106234845) bank107736 +107737 POINT(41.69340594517278 73.54835937388893) bank107737 +107738 POINT(40.22147294581914 74.46454797547395) bank107738 +107739 POINT(40.766923103522345 73.6551362165606) bank107739 +107740 POINT(41.35671927386956 73.17127231539415) bank107740 +107741 POINT(40.19568570688013 74.81871106186824) bank107741 +107742 POINT(40.48797151088443 74.00354417925651) bank107742 +107743 POINT(41.30851116395523 73.46120903731493) bank107743 +107744 POINT(39.87451138129711 73.13958334039823) bank107744 +107745 POINT(40.70550732646468 73.64014749800856) bank107745 +107746 POINT(40.63599196974906 74.52738898880939) bank107746 +107747 POINT(41.546385121013444 73.97772672774633) bank107747 +107748 POINT(41.583026094842346 74.3633401364442) bank107748 +107749 POINT(41.54219026905175 73.499837047182) bank107749 +107750 POINT(39.92230943054775 73.82437014956322) bank107750 +107751 POINT(41.27244492593516 74.02617693713164) bank107751 +107752 POINT(40.88090288816665 73.588606375967) bank107752 +107753 POINT(39.87923512801924 73.05008875900083) bank107753 +107754 POINT(40.64409556201903 73.66017529614062) bank107754 +107755 POINT(41.43516509518666 74.32176830439631) bank107755 +107756 POINT(41.454132632565404 73.38987410193131) bank107756 +107757 POINT(40.46049510705785 73.20741787833217) bank107757 +107758 POINT(40.1204563993591 73.45360651394417) bank107758 +107759 POINT(40.570792500154354 74.29376992691974) bank107759 +107760 POINT(40.19092961905749 74.84395878385891) bank107760 +107761 POINT(40.32459551174947 74.8027817475681) bank107761 +107762 POINT(41.10314821743932 74.82058141716996) bank107762 +107763 POINT(40.56642517021539 73.7180899710387) bank107763 +107764 POINT(41.06655051224653 73.88675284883266) bank107764 +107765 POINT(41.37252885652165 74.97618272433681) bank107765 +107766 POINT(40.39856514194018 74.8200830669342) bank107766 +107767 POINT(40.276735338938984 74.03083851905582) bank107767 +107768 POINT(41.249554986163886 73.83451841920761) bank107768 +107769 POINT(39.823852061181995 73.45634767712859) bank107769 +107770 POINT(40.797215554220735 73.4937299022099) bank107770 +107771 POINT(40.56954428472183 74.49173255666841) bank107771 +107772 POINT(40.986666527468884 73.06701947417969) bank107772 +107773 POINT(40.52457359655095 74.89897472588382) bank107773 +107774 POINT(40.198720586805294 74.39241468427494) bank107774 +107775 POINT(41.050600336841576 73.95561724696648) bank107775 +107776 POINT(39.96974824382044 73.61358124692843) bank107776 +107777 POINT(40.60839055156905 73.01988823698478) bank107777 +107778 POINT(41.40650160489172 74.74367353092643) bank107778 +107779 POINT(40.385638409781166 74.79584109192575) bank107779 +107780 POINT(39.73906728233971 73.80409860596977) bank107780 +107781 POINT(40.873658439476785 74.41611816851804) bank107781 +107782 POINT(41.53184368290544 74.69157765149389) bank107782 +107783 POINT(41.14846937807809 74.77024053755818) bank107783 +107784 POINT(41.08547062900143 73.0102229860143) bank107784 +107785 POINT(40.79500802113982 74.79617323893679) bank107785 +107786 POINT(41.14981647967886 73.53004696925163) bank107786 +107787 POINT(40.54254904836196 74.05926956206295) bank107787 +107788 POINT(40.08546498486548 74.05115870796442) bank107788 +107789 POINT(40.426751175538264 74.33013668555384) bank107789 +107790 POINT(40.55713139916895 73.05264448736355) bank107790 +107791 POINT(41.618632955979756 73.42426410580526) bank107791 +107792 POINT(40.67051172098739 74.60608498783726) bank107792 +107793 POINT(41.2643713017435 74.74378793092482) bank107793 +107794 POINT(41.2141240403297 73.82586784420936) bank107794 +107795 POINT(40.69821244552371 74.91856043065022) bank107795 +107796 POINT(40.173325486736786 74.60363269047293) bank107796 +107797 POINT(41.646596507553554 73.8193469729414) bank107797 +107798 POINT(41.654662939624195 74.13907248916283) bank107798 +107799 POINT(39.9399038729794 73.29900807398069) bank107799 +107800 POINT(41.55646027596258 74.12533984466998) bank107800 +107801 POINT(40.723907977536655 73.30606576566257) bank107801 +107802 POINT(39.83846869704648 74.23836562764458) bank107802 +107803 POINT(40.45990093987313 74.91892853041692) bank107803 +107804 POINT(40.12768496689977 73.57215713276133) bank107804 +107805 POINT(40.73733333881197 74.4806232415092) bank107805 +107806 POINT(41.51420489332676 73.02913495788533) bank107806 +107807 POINT(41.481705210903904 73.17997941305093) bank107807 +107808 POINT(40.30540707788613 74.99133379470997) bank107808 +107809 POINT(40.70422475657361 73.10833576476006) bank107809 +107810 POINT(40.57151337439281 74.04161415357933) bank107810 +107811 POINT(39.77666840786006 73.50569341104611) bank107811 +107812 POINT(40.892742472398844 73.99988496446643) bank107812 +107813 POINT(39.8574543197982 74.63982501462694) bank107813 +107814 POINT(40.377991151457486 73.06518712190741) bank107814 +107815 POINT(41.09521835466758 74.55462147433887) bank107815 +107816 POINT(40.54874089857205 74.60018530773863) bank107816 +107817 POINT(41.38656290620315 74.82714977171786) bank107817 +107818 POINT(40.54405077857475 74.33745612157908) bank107818 +107819 POINT(41.58195790506389 73.05371392715983) bank107819 +107820 POINT(40.01400593006275 73.8491628079572) bank107820 +107821 POINT(41.30846165429231 73.85422448718565) bank107821 +107822 POINT(39.90173304460736 73.32309898636905) bank107822 +107823 POINT(39.809955908835974 73.14371824566207) bank107823 +107824 POINT(40.26099892612782 74.3345861527173) bank107824 +107825 POINT(41.709727154609226 73.68758788192888) bank107825 +107826 POINT(40.26726245756717 74.15785777724078) bank107826 +107827 POINT(41.157735126491495 74.70493071961785) bank107827 +107828 POINT(39.8405220605477 74.38162164177628) bank107828 +107829 POINT(41.63299651139747 74.64991565455477) bank107829 +107830 POINT(40.905845833463864 73.2866782673052) bank107830 +107831 POINT(40.93824130183911 74.21604209972327) bank107831 +107832 POINT(41.12090814122281 73.5559528250552) bank107832 +107833 POINT(40.07743252187907 73.13117607475284) bank107833 +107834 POINT(41.49670541544106 73.8229539122647) bank107834 +107835 POINT(41.42345697513768 74.78628292090568) bank107835 +107836 POINT(41.67011364831383 73.9349891918181) bank107836 +107837 POINT(40.45887884310316 73.15741168568162) bank107837 +107838 POINT(40.332129221472904 73.5006410893396) bank107838 +107839 POINT(41.05431017397536 74.92036274258452) bank107839 +107840 POINT(40.04287461564809 74.70816435809611) bank107840 +107841 POINT(40.77350220799271 74.52681982569833) bank107841 +107842 POINT(40.26936179523172 73.29253002920568) bank107842 +107843 POINT(41.309896691520386 73.69453940927826) bank107843 +107844 POINT(40.45787067181962 73.01047240607343) bank107844 +107845 POINT(40.94167935871531 73.87029800321805) bank107845 +107846 POINT(40.31584779778422 73.34744188663878) bank107846 +107847 POINT(40.69976996019266 74.40265141817707) bank107847 +107848 POINT(40.73970060385998 74.7333660974747) bank107848 +107849 POINT(41.40128422781418 73.45610035374618) bank107849 +107850 POINT(41.25710173194246 74.1727686895852) bank107850 +107851 POINT(40.47551831145964 73.54169310900409) bank107851 +107852 POINT(39.878998398136225 73.80945779326903) bank107852 +107853 POINT(40.16198879573967 74.42085674491155) bank107853 +107854 POINT(41.07051132107938 74.33657962754486) bank107854 +107855 POINT(40.9725618976813 73.53532081030166) bank107855 +107856 POINT(41.344010130600395 73.88987561780877) bank107856 +107857 POINT(40.58130929521561 73.63269019864805) bank107857 +107858 POINT(41.2621178790636 73.2246177136021) bank107858 +107859 POINT(40.62545178687706 73.58875137800543) bank107859 +107860 POINT(40.5170407386458 74.52996488195356) bank107860 +107861 POINT(40.215767517806306 73.49867347527747) bank107861 +107862 POINT(40.98592124496015 73.32984287526823) bank107862 +107863 POINT(41.69667438621239 74.03782308199997) bank107863 +107864 POINT(41.155745033776924 74.63420234775913) bank107864 +107865 POINT(41.36376369578092 73.09266043762791) bank107865 +107866 POINT(41.01015850205952 73.10980052591381) bank107866 +107867 POINT(39.84544659120302 74.49460435209775) bank107867 +107868 POINT(39.8338727353747 73.33803870056971) bank107868 +107869 POINT(40.77542241413926 74.70370800356946) bank107869 +107870 POINT(40.226192927783664 74.05021289182379) bank107870 +107871 POINT(40.510934235396874 74.07829670863372) bank107871 +107872 POINT(40.394789187625 73.19650766653261) bank107872 +107873 POINT(40.686756983220846 74.67149850427346) bank107873 +107874 POINT(40.581306700259724 73.92688018217976) bank107874 +107875 POINT(39.76308528033288 73.06954162368825) bank107875 +107876 POINT(41.33896436568638 73.64720961027726) bank107876 +107877 POINT(41.154457942403496 74.96057647106078) bank107877 +107878 POINT(40.8698681882318 73.91848033487233) bank107878 +107879 POINT(40.13647252111365 74.13009377437464) bank107879 +107880 POINT(40.21969627414366 73.27898104819) bank107880 +107881 POINT(40.51584633356592 74.18083915931123) bank107881 +107882 POINT(41.66652317376651 73.44303101726511) bank107882 +107883 POINT(40.77318008277508 73.94719619014407) bank107883 +107884 POINT(39.885349982835216 74.72277851759794) bank107884 +107885 POINT(40.36844053482725 73.35964872568276) bank107885 +107886 POINT(39.88962485515705 73.09223131558232) bank107886 +107887 POINT(41.550461427108296 73.59418299712031) bank107887 +107888 POINT(39.90986924507721 73.04417687006972) bank107888 +107889 POINT(41.05239029373699 73.8871341956514) bank107889 +107890 POINT(41.277893743418666 73.08584169975744) bank107890 +107891 POINT(40.16418132052789 74.30695327211095) bank107891 +107892 POINT(40.542271016059914 74.1938786590064) bank107892 +107893 POINT(40.591896629584035 73.84447366978242) bank107893 +107894 POINT(41.53280415795029 74.8649766627633) bank107894 +107895 POINT(39.75827063600659 74.86048355385012) bank107895 +107896 POINT(40.15144543302738 74.24703707968494) bank107896 +107897 POINT(41.51361895233885 74.10197165851686) bank107897 +107898 POINT(40.833997117327314 74.68471940675096) bank107898 +107899 POINT(40.37910555800974 74.16721118258181) bank107899 +107900 POINT(40.073824811658454 74.82063730255871) bank107900 +107901 POINT(40.3369985119474 74.82436260737607) bank107901 +107902 POINT(40.12117743512259 74.39779831950372) bank107902 +107903 POINT(39.7762970920848 74.23582111363851) bank107903 +107904 POINT(41.28145964621049 74.47263133355499) bank107904 +107905 POINT(39.93366313261653 73.51982585515835) bank107905 +107906 POINT(41.085892036662266 74.6324754639666) bank107906 +107907 POINT(41.45930601982556 73.67065852500943) bank107907 +107908 POINT(40.10268129117777 73.01992805484988) bank107908 +107909 POINT(40.3480293204334 73.11226533477279) bank107909 +107910 POINT(41.55319455270429 74.9784674606749) bank107910 +107911 POINT(40.99655314103795 73.71289072998327) bank107911 +107912 POINT(39.92459712084585 74.73084547311267) bank107912 +107913 POINT(40.04996189013978 73.79977117910317) bank107913 +107914 POINT(39.869439809571 74.54550304329719) bank107914 +107915 POINT(39.986372867429374 74.39595499028269) bank107915 +107916 POINT(41.38900033467733 74.29151360381638) bank107916 +107917 POINT(40.357922553204865 74.84933592152339) bank107917 +107918 POINT(39.77308629301936 73.4406739978562) bank107918 +107919 POINT(41.24385140869566 73.98506776355394) bank107919 +107920 POINT(40.90504339637541 74.19278023416557) bank107920 +107921 POINT(39.85112296713329 73.92131572510785) bank107921 +107922 POINT(40.83326951565437 74.2403639610355) bank107922 +107923 POINT(40.21929646911083 74.91733348112488) bank107923 +107924 POINT(40.73217135671511 73.7225331936986) bank107924 +107925 POINT(40.6604593196992 74.17535038806444) bank107925 +107926 POINT(41.4452936218572 74.62397691046337) bank107926 +107927 POINT(41.049085371333454 74.1360418707768) bank107927 +107928 POINT(41.04207847963991 73.57984280165412) bank107928 +107929 POINT(41.36079434181938 73.33540626596587) bank107929 +107930 POINT(40.324916913619596 73.97626577967114) bank107930 +107931 POINT(39.88897979828947 74.31908099689392) bank107931 +107932 POINT(40.30074121410238 74.59482147730775) bank107932 +107933 POINT(41.07805628832389 74.40959009581712) bank107933 +107934 POINT(40.61940075126973 74.53952900220425) bank107934 +107935 POINT(40.035202763396946 73.81237614107373) bank107935 +107936 POINT(40.401178058409194 73.55879666667975) bank107936 +107937 POINT(41.19173168429985 74.2659685283731) bank107937 +107938 POINT(40.66540326763539 74.70174310914221) bank107938 +107939 POINT(41.62647308929033 73.28857969874795) bank107939 +107940 POINT(41.60266884472428 73.4276722730628) bank107940 +107941 POINT(40.313642772343286 74.70271390431559) bank107941 +107942 POINT(40.96428201953113 74.47999734271409) bank107942 +107943 POINT(40.91787013354785 73.64563703177083) bank107943 +107944 POINT(40.83915146425761 74.10555984350584) bank107944 +107945 POINT(40.30386034980143 74.15828082757655) bank107945 +107946 POINT(41.68099988447382 74.5536222275446) bank107946 +107947 POINT(39.75087391243585 73.63364767543992) bank107947 +107948 POINT(40.26704195083848 74.62436575681697) bank107948 +107949 POINT(41.421000007304116 73.81828769027356) bank107949 +107950 POINT(40.42744819574728 73.34584739582782) bank107950 +107951 POINT(40.29687271960469 74.18918009342636) bank107951 +107952 POINT(41.59831699446578 73.43250704992944) bank107952 +107953 POINT(40.69731247244394 74.78402860974377) bank107953 +107954 POINT(40.439482198978894 74.34237982525384) bank107954 +107955 POINT(39.86526564895392 74.82318905933975) bank107955 +107956 POINT(39.82973829803649 73.38691195514203) bank107956 +107957 POINT(41.09167517132254 73.04868028125902) bank107957 +107958 POINT(40.8986763911311 74.33650360116374) bank107958 +107959 POINT(40.746049883744 73.48306362167087) bank107959 +107960 POINT(40.52742192836357 74.78024506212948) bank107960 +107961 POINT(41.46899030297257 73.49652307556619) bank107961 +107962 POINT(41.43628258625558 73.98992168806868) bank107962 +107963 POINT(40.03272668659111 73.72123583682874) bank107963 +107964 POINT(41.42358888819019 73.55144228877553) bank107964 +107965 POINT(39.8303803269838 74.81383722238627) bank107965 +107966 POINT(40.015615966661386 73.66446161701396) bank107966 +107967 POINT(41.51783171807897 74.00546076680924) bank107967 +107968 POINT(41.13230084184136 73.40147941921893) bank107968 +107969 POINT(41.21285431551118 73.8913745042133) bank107969 +107970 POINT(40.045670847651046 73.69120336445954) bank107970 +107971 POINT(40.235272632941935 74.12768945565703) bank107971 +107972 POINT(40.23051886097386 73.23722267481654) bank107972 +107973 POINT(40.69444772944946 73.42809592667712) bank107973 +107974 POINT(40.700055138734044 73.8540292479486) bank107974 +107975 POINT(41.13421189872977 74.28169393135042) bank107975 +107976 POINT(41.54622897319797 74.91255763622755) bank107976 +107977 POINT(41.55546800179656 73.58452290731178) bank107977 +107978 POINT(41.681540622519115 73.85835402213999) bank107978 +107979 POINT(40.7892957289714 73.35495849531162) bank107979 +107980 POINT(41.62914575966056 73.13762859880036) bank107980 +107981 POINT(41.5030990887202 73.41434098757068) bank107981 +107982 POINT(40.4872419210148 73.47511317312932) bank107982 +107983 POINT(40.31374092727633 73.39730623334225) bank107983 +107984 POINT(39.83969189596084 73.83407834052333) bank107984 +107985 POINT(39.823546376466005 73.84127029477368) bank107985 +107986 POINT(39.83005760797098 73.2882349070232) bank107986 +107987 POINT(40.6648804498843 73.61623416148608) bank107987 +107988 POINT(41.457240781438934 74.27463497963988) bank107988 +107989 POINT(40.80977701117964 74.9166087795014) bank107989 +107990 POINT(41.636846756875634 73.43015784443483) bank107990 +107991 POINT(40.28249361135176 73.45354890761756) bank107991 +107992 POINT(40.36693821139016 74.06338713201221) bank107992 +107993 POINT(40.801562416508645 74.07159696758677) bank107993 +107994 POINT(41.17631084038069 74.08869900225316) bank107994 +107995 POINT(41.08319821963264 73.41323710129166) bank107995 +107996 POINT(40.146927726592224 74.31449547970517) bank107996 +107997 POINT(40.74824624630394 73.24344561332244) bank107997 +107998 POINT(41.54274033631377 73.44668782485097) bank107998 +107999 POINT(40.990729105383785 74.71153684260912) bank107999 +108000 POINT(41.05773309280662 74.02848471708927) bank108000 +108001 POINT(39.9563685547112 74.66353680050888) bank108001 +108002 POINT(40.681117437163955 74.30129985577254) bank108002 +108003 POINT(40.88932566174336 74.56777309067878) bank108003 +108004 POINT(39.97056636331311 73.14531529956977) bank108004 +108005 POINT(40.7986484012896 74.49571304541477) bank108005 +108006 POINT(39.88210038573278 73.4772981927526) bank108006 +108007 POINT(39.99859980553464 73.82312000444941) bank108007 +108008 POINT(41.11500074364966 74.05867762537163) bank108008 +108009 POINT(40.18484321070751 73.21869247635237) bank108009 +108010 POINT(40.53310034059117 74.67271124491295) bank108010 +108011 POINT(41.486208538799886 74.18866318435565) bank108011 +108012 POINT(39.74659788169062 74.59006749493133) bank108012 +108013 POINT(40.965705397763664 73.83772849497112) bank108013 +108014 POINT(41.026147775853865 74.10213248389007) bank108014 +108015 POINT(39.823378976685255 73.20859493665445) bank108015 +108016 POINT(40.56481516520439 74.86297659884968) bank108016 +108017 POINT(40.64228464749085 73.5790227257874) bank108017 +108018 POINT(40.709226960228385 73.73289036649497) bank108018 +108019 POINT(40.10031545445152 74.48125361308308) bank108019 +108020 POINT(41.62287415845642 74.40755208433087) bank108020 +108021 POINT(40.526349103350896 74.03955988414381) bank108021 +108022 POINT(41.552812681520486 73.94008801315304) bank108022 +108023 POINT(39.720705922456155 73.43334242506502) bank108023 +108024 POINT(39.92392887919624 74.55577795268378) bank108024 +108025 POINT(40.91661034593766 73.52590777623578) bank108025 +108026 POINT(41.427897112049976 73.62337086047107) bank108026 +108027 POINT(40.035278334167245 73.85583835528625) bank108027 +108028 POINT(41.13726061459559 73.61265349603958) bank108028 +108029 POINT(39.98981400724352 73.32027271201989) bank108029 +108030 POINT(41.25391937128463 73.5693113139939) bank108030 +108031 POINT(39.8181154483353 73.78894036450909) bank108031 +108032 POINT(41.56646624462442 74.64750183864228) bank108032 +108033 POINT(40.53145951289128 73.07227549889978) bank108033 +108034 POINT(40.89820323606039 73.07142679458121) bank108034 +108035 POINT(41.29868783667742 73.44913309571871) bank108035 +108036 POINT(40.721923025891435 73.82973763336939) bank108036 +108037 POINT(40.60605206541149 73.23668058870646) bank108037 +108038 POINT(40.73361865001671 74.60817896397835) bank108038 +108039 POINT(40.09497006756392 74.72582820996045) bank108039 +108040 POINT(40.62595842927472 73.33272362218374) bank108040 +108041 POINT(39.97766827402435 73.78402458109258) bank108041 +108042 POINT(41.5177788857916 74.54649447767756) bank108042 +108043 POINT(41.36793437427526 73.26416782449239) bank108043 +108044 POINT(41.40152027135169 73.03007274384255) bank108044 +108045 POINT(41.284421196578315 74.66888190436092) bank108045 +108046 POINT(40.13980549801611 73.34007404692198) bank108046 +108047 POINT(40.30467898354574 74.06335154146392) bank108047 +108048 POINT(41.67035163091333 73.7896734318693) bank108048 +108049 POINT(40.633347598617114 74.15662391521161) bank108049 +108050 POINT(40.78314511488968 74.94371675524738) bank108050 +108051 POINT(39.917480386347975 73.44983305468027) bank108051 +108052 POINT(41.079091645208415 74.79939244546748) bank108052 +108053 POINT(41.275273023356924 74.34518832683163) bank108053 +108054 POINT(41.41089078184958 73.8283862271925) bank108054 +108055 POINT(41.396718070765445 74.69554137247057) bank108055 +108056 POINT(40.92711580310012 74.8615881194881) bank108056 +108057 POINT(40.01032654282892 74.43819313742198) bank108057 +108058 POINT(40.12889126143025 74.93374269657791) bank108058 +108059 POINT(41.279191422041364 74.02563339221803) bank108059 +108060 POINT(40.07584024772221 74.64174817506459) bank108060 +108061 POINT(40.837547360335755 73.60588079930754) bank108061 +108062 POINT(40.532905670584256 73.53804892545978) bank108062 +108063 POINT(41.658172506545135 74.88832063529759) bank108063 +108064 POINT(40.923839759953054 73.93460023218341) bank108064 +108065 POINT(40.569571154889495 73.61509527295799) bank108065 +108066 POINT(39.93168339923913 74.16702556901534) bank108066 +108067 POINT(39.78479169349203 73.1533413683278) bank108067 +108068 POINT(40.72966947270601 73.71007660195103) bank108068 +108069 POINT(39.84488287457167 74.39525765626631) bank108069 +108070 POINT(40.58037333546776 74.55597573109124) bank108070 +108071 POINT(40.26459581510419 74.1105999915578) bank108071 +108072 POINT(41.53291055372741 73.16546796988882) bank108072 +108073 POINT(41.23191620857762 74.633284428809) bank108073 +108074 POINT(39.87415632683619 73.24045986931952) bank108074 +108075 POINT(40.54005710891906 73.75790212805084) bank108075 +108076 POINT(41.16224473422206 73.74128840649139) bank108076 +108077 POINT(40.30728138998755 73.03177540684237) bank108077 +108078 POINT(41.092005168880036 73.56351877723463) bank108078 +108079 POINT(39.777037917829496 73.7962880937974) bank108079 +108080 POINT(40.61893818053713 74.09165153615037) bank108080 +108081 POINT(39.84205544745595 73.16800476775732) bank108081 +108082 POINT(40.87618669533796 74.18442544265632) bank108082 +108083 POINT(41.44189580461293 74.53190023289942) bank108083 +108084 POINT(41.61781537683973 74.21401136801177) bank108084 +108085 POINT(41.133391155555 73.60100065337845) bank108085 +108086 POINT(40.580967961702974 73.14007647877126) bank108086 +108087 POINT(40.844187938830956 74.21714953809958) bank108087 +108088 POINT(40.92403016891301 73.75813038716362) bank108088 +108089 POINT(41.03361260397048 73.68728071929485) bank108089 +108090 POINT(39.81330742589305 74.64808039317516) bank108090 +108091 POINT(41.176502030139645 74.34289491832806) bank108091 +108092 POINT(39.8213574191441 74.8384990771436) bank108092 +108093 POINT(40.6842913644563 73.88021311288814) bank108093 +108094 POINT(39.75453731513336 73.38362386890256) bank108094 +108095 POINT(40.01119019779949 74.13252208345692) bank108095 +108096 POINT(40.70139055355386 74.86929409173227) bank108096 +108097 POINT(40.18635095843076 73.47857152213199) bank108097 +108098 POINT(40.80417379813326 74.98695987928839) bank108098 +108099 POINT(40.56277776226347 74.8689503898363) bank108099 +108100 POINT(40.843844773459075 73.9107302605082) bank108100 +108101 POINT(41.07030031002087 74.80997404703093) bank108101 +108102 POINT(39.81240434702022 74.46300560698897) bank108102 +108103 POINT(40.62566063860918 74.61148399965603) bank108103 +108104 POINT(41.20693436755865 74.3264947843121) bank108104 +108105 POINT(39.81937257380327 74.8352824201674) bank108105 +108106 POINT(41.45380202018045 73.17094916281589) bank108106 +108107 POINT(41.62243762935897 74.99642101446196) bank108107 +108108 POINT(40.695140286579274 74.44801223579601) bank108108 +108109 POINT(41.12849010199653 74.04746493052356) bank108109 +108110 POINT(40.47644270992469 73.36790917306227) bank108110 +108111 POINT(40.07366276118745 73.0715371274833) bank108111 +108112 POINT(41.32931277165931 74.21154517689617) bank108112 +108113 POINT(41.038830892545306 73.34124695186644) bank108113 +108114 POINT(40.6628875310417 74.85214481325981) bank108114 +108115 POINT(41.05369181331972 73.24835480896581) bank108115 +108116 POINT(40.14192686929897 74.20244964750923) bank108116 +108117 POINT(41.173864501948714 74.28277348549312) bank108117 +108118 POINT(40.69569576660148 73.86357198120042) bank108118 +108119 POINT(41.38439574873656 74.71691369948772) bank108119 +108120 POINT(41.30307900767492 74.45283790007707) bank108120 +108121 POINT(41.04577931540663 73.9882744821334) bank108121 +108122 POINT(39.89211531384458 73.76238178034572) bank108122 +108123 POINT(40.15250902219006 74.30417130078744) bank108123 +108124 POINT(40.58922705661822 74.71311531436126) bank108124 +108125 POINT(40.419593661034526 73.40826063952751) bank108125 +108126 POINT(40.01243782235085 74.14594995003198) bank108126 +108127 POINT(39.971917768389034 74.6930653659374) bank108127 +108128 POINT(40.114937235740214 73.48778162505428) bank108128 +108129 POINT(40.66575486785498 74.76800102321656) bank108129 +108130 POINT(41.31945783908263 74.55212786630966) bank108130 +108131 POINT(40.49030412208656 73.62173116536727) bank108131 +108132 POINT(39.90763368634583 74.78914334882394) bank108132 +108133 POINT(40.2426946598659 74.40035527586609) bank108133 +108134 POINT(40.33007234356536 74.87238312739608) bank108134 +108135 POINT(40.45169869494797 74.29071596945869) bank108135 +108136 POINT(40.83549841718002 74.51438859828107) bank108136 +108137 POINT(41.20929272020714 74.31614366914548) bank108137 +108138 POINT(40.79888431942213 74.71413357376237) bank108138 +108139 POINT(41.605600835840434 73.3916776435359) bank108139 +108140 POINT(41.64376890522034 74.36611178955415) bank108140 +108141 POINT(41.39508454484454 73.1233758268825) bank108141 +108142 POINT(39.78557889816602 73.98475705622222) bank108142 +108143 POINT(40.944056266328644 73.46336444312698) bank108143 +108144 POINT(40.83554202350069 74.7584727758917) bank108144 +108145 POINT(40.97319803577514 74.49308878905774) bank108145 +108146 POINT(40.552321985541134 74.8665840583042) bank108146 +108147 POINT(40.731869320714594 74.55587604583117) bank108147 +108148 POINT(41.58031009288561 73.85841299230934) bank108148 +108149 POINT(40.856727496794214 74.48504663675087) bank108149 +108150 POINT(41.63054315943456 73.72352653899851) bank108150 +108151 POINT(41.61430436858506 73.76051571062332) bank108151 +108152 POINT(39.83578486391809 74.99778643479567) bank108152 +108153 POINT(41.00847186940046 74.93088435981754) bank108153 +108154 POINT(40.7122684237223 73.50263711277057) bank108154 +108155 POINT(41.174125166287794 74.29612415308377) bank108155 +108156 POINT(40.14484241459985 73.18521155057427) bank108156 +108157 POINT(41.26549887583664 74.16402042018461) bank108157 +108158 POINT(39.81141871140175 73.79932246176789) bank108158 +108159 POINT(40.19297089880331 73.95901082424892) bank108159 +108160 POINT(41.47450621531137 74.28403839548677) bank108160 +108161 POINT(40.918723755590214 73.69618048865802) bank108161 +108162 POINT(39.71973088929408 74.74498548267131) bank108162 +108163 POINT(40.95946036556892 73.34128353573104) bank108163 +108164 POINT(40.14550155873751 74.90025168731742) bank108164 +108165 POINT(40.824836717190514 73.32237711356322) bank108165 +108166 POINT(40.49483791173772 74.6161983410449) bank108166 +108167 POINT(40.867093551975294 73.84759032057633) bank108167 +108168 POINT(41.20461299671221 73.89363725089949) bank108168 +108169 POINT(41.078247597130826 73.8035642766962) bank108169 +108170 POINT(41.387180650378575 73.80035714899293) bank108170 +108171 POINT(40.15833549590101 74.38280424381848) bank108171 +108172 POINT(41.28860843258635 73.04125962259971) bank108172 +108173 POINT(40.89298164725629 74.59497583145031) bank108173 +108174 POINT(40.3650357554999 73.00952221653272) bank108174 +108175 POINT(39.95699505899871 74.39641970776533) bank108175 +108176 POINT(40.44555085681115 73.88781761957554) bank108176 +108177 POINT(41.368555428599095 73.80890232482923) bank108177 +108178 POINT(39.82809868560378 74.03579134631464) bank108178 +108179 POINT(41.0038031567521 74.63464637619123) bank108179 +108180 POINT(40.983780583519575 74.23731271997784) bank108180 +108181 POINT(41.46098086370081 73.66846043344142) bank108181 +108182 POINT(40.27353208386762 74.49434956798918) bank108182 +108183 POINT(41.60423210590182 74.09976708160251) bank108183 +108184 POINT(41.657004296967635 74.81750470156064) bank108184 +108185 POINT(40.06188865034501 73.14419917255651) bank108185 +108186 POINT(41.54981450709674 74.22384284972406) bank108186 +108187 POINT(40.04873515256192 73.63330479505373) bank108187 +108188 POINT(40.21367065795409 74.03572132934069) bank108188 +108189 POINT(40.16942497555723 74.1178032864561) bank108189 +108190 POINT(41.13739123289532 74.23954882566773) bank108190 +108191 POINT(41.20689116351613 74.4399645626887) bank108191 +108192 POINT(40.31703622324669 74.80555480767191) bank108192 +108193 POINT(41.19869544528856 74.92443619172428) bank108193 +108194 POINT(40.820291839252164 73.73046521003043) bank108194 +108195 POINT(41.690215343645995 73.37758820619086) bank108195 +108196 POINT(40.458970081718725 74.6112201696347) bank108196 +108197 POINT(41.662528422441845 74.01533334752713) bank108197 +108198 POINT(40.29041708844578 73.88892973624384) bank108198 +108199 POINT(40.65059787827838 73.40144120902038) bank108199 +108200 POINT(41.47092364862629 73.34653977122571) bank108200 +108201 POINT(41.304371910914526 73.59132949180756) bank108201 +108202 POINT(40.9631794995813 73.30922821928075) bank108202 +108203 POINT(40.35683796464173 73.13012821211005) bank108203 +108204 POINT(41.3448231815773 73.69743502720696) bank108204 +108205 POINT(40.67806941357659 73.32294902620966) bank108205 +108206 POINT(39.763071004822166 74.39888653828238) bank108206 +108207 POINT(41.69415075562446 74.07888109725644) bank108207 +108208 POINT(40.284249198992164 73.15288587964488) bank108208 +108209 POINT(39.8610278753924 74.21540743972334) bank108209 +108210 POINT(39.82445105489478 74.7893253181176) bank108210 +108211 POINT(41.19675311601432 73.52927164397379) bank108211 +108212 POINT(39.907545730846586 73.77230806227726) bank108212 +108213 POINT(41.265175747854315 73.33643718798224) bank108213 +108214 POINT(40.70706734951042 74.9594354989864) bank108214 +108215 POINT(40.156123458677044 74.97073145522471) bank108215 +108216 POINT(40.04496028584128 74.36571521361697) bank108216 +108217 POINT(39.939601608528164 74.45132363739977) bank108217 +108218 POINT(39.87679697109158 73.98775290905316) bank108218 +108219 POINT(40.956386717344515 73.1403661075877) bank108219 +108220 POINT(41.449606123584374 73.41939030814295) bank108220 +108221 POINT(40.17799214020388 73.45554086366256) bank108221 +108222 POINT(39.808596415870724 74.29333333830573) bank108222 +108223 POINT(40.49802104952705 74.06749065193249) bank108223 +108224 POINT(40.615577086296575 73.26434228177403) bank108224 +108225 POINT(41.443831649620705 74.07254523726255) bank108225 +108226 POINT(40.265052736513034 74.37135431194828) bank108226 +108227 POINT(41.6900011950672 74.83387831945517) bank108227 +108228 POINT(40.10585875662834 73.46048982252222) bank108228 +108229 POINT(39.784951016320164 73.56186018731714) bank108229 +108230 POINT(40.97770271675474 74.2869964894448) bank108230 +108231 POINT(41.24649975694897 73.32659494900179) bank108231 +108232 POINT(40.61195192559126 74.55944810595582) bank108232 +108233 POINT(40.41823691875913 73.3695898730985) bank108233 +108234 POINT(40.34869129161291 73.42519052945754) bank108234 +108235 POINT(40.17473302353027 73.81884942485674) bank108235 +108236 POINT(41.10906943986885 73.8554802921617) bank108236 +108237 POINT(41.15089355831386 73.39696188735196) bank108237 +108238 POINT(40.76026811618428 73.94780839925383) bank108238 +108239 POINT(41.48129910901013 73.8688071536161) bank108239 +108240 POINT(40.66296340754499 73.58591971856) bank108240 +108241 POINT(41.45693111277224 74.14235143611026) bank108241 +108242 POINT(41.30035052205509 73.79738785707795) bank108242 +108243 POINT(40.33117196637496 73.6417168134265) bank108243 +108244 POINT(40.53380997489338 73.25215381396804) bank108244 +108245 POINT(41.697635352825635 74.3845780108317) bank108245 +108246 POINT(40.477953637444294 73.49316778291134) bank108246 +108247 POINT(41.45940967098885 73.68939412662208) bank108247 +108248 POINT(41.068756904597564 73.06022226077671) bank108248 +108249 POINT(40.12710076690627 73.61269257771364) bank108249 +108250 POINT(40.38651606891886 73.02698658443856) bank108250 +108251 POINT(39.84492858100393 73.19429299764292) bank108251 +108252 POINT(41.114526291267794 73.90566580053583) bank108252 +108253 POINT(41.128791954817494 74.83044911443157) bank108253 +108254 POINT(39.85531560168096 73.23587663369186) bank108254 +108255 POINT(41.34544225107102 74.53941624451886) bank108255 +108256 POINT(40.23427613467046 74.26483944678512) bank108256 +108257 POINT(41.31845866999665 74.26370214975529) bank108257 +108258 POINT(41.30170146318488 74.33622657907685) bank108258 +108259 POINT(39.83580821374488 73.73241719725083) bank108259 +108260 POINT(39.81264100803945 74.35470359647942) bank108260 +108261 POINT(41.14988348715298 74.40496064445647) bank108261 +108262 POINT(40.86486598398378 74.42635968129402) bank108262 +108263 POINT(40.396486731977966 74.25734852440705) bank108263 +108264 POINT(41.147238233869125 74.5891913195567) bank108264 +108265 POINT(39.862311073644456 74.97570141266618) bank108265 +108266 POINT(40.736881992572314 73.76919933716839) bank108266 +108267 POINT(41.44178843734658 73.89746374034523) bank108267 +108268 POINT(40.376791919561214 73.77953638765537) bank108268 +108269 POINT(40.849936577849874 74.28156492877676) bank108269 +108270 POINT(40.259920106926046 73.58445372458417) bank108270 +108271 POINT(41.55162237196113 74.83514992900663) bank108271 +108272 POINT(41.31651339304977 73.35111885053745) bank108272 +108273 POINT(41.37369575919895 73.43945595202135) bank108273 +108274 POINT(40.73167859689527 73.93455097648604) bank108274 +108275 POINT(40.067129636100034 74.83802914339843) bank108275 +108276 POINT(40.483497129028244 73.7622132954085) bank108276 +108277 POINT(41.00014278478743 73.42066958829787) bank108277 +108278 POINT(41.256282450241315 73.6905102390719) bank108278 +108279 POINT(41.24693162263631 73.16911713038657) bank108279 +108280 POINT(40.472438440230164 73.12351248251994) bank108280 +108281 POINT(40.550867550136886 74.88454077902267) bank108281 +108282 POINT(39.96451421624483 74.5921857776622) bank108282 +108283 POINT(40.850398775952385 73.99641984368343) bank108283 +108284 POINT(41.599630881425014 73.37076587267491) bank108284 +108285 POINT(40.58568397176698 73.54390029091624) bank108285 +108286 POINT(40.892444182515824 73.12174987975507) bank108286 +108287 POINT(40.56795169233251 74.08404837239128) bank108287 +108288 POINT(40.45461284523773 74.53746426072698) bank108288 +108289 POINT(40.78511911799907 73.3418057607101) bank108289 +108290 POINT(39.84580656989879 74.38493658052435) bank108290 +108291 POINT(40.73397533144075 74.3457466976524) bank108291 +108292 POINT(41.586051031651486 74.10559258163353) bank108292 +108293 POINT(40.88078569540577 74.00665199377411) bank108293 +108294 POINT(40.175918480460744 73.43861363870036) bank108294 +108295 POINT(40.67616730873509 74.66945331565276) bank108295 +108296 POINT(39.86059837983453 74.25539342180382) bank108296 +108297 POINT(40.89555035515494 73.73539594310142) bank108297 +108298 POINT(40.9373524808814 74.22335030718796) bank108298 +108299 POINT(41.361945188704766 73.96527453688492) bank108299 +108300 POINT(40.67955166689831 73.8022828650084) bank108300 +108301 POINT(40.23969994989626 74.44257518556113) bank108301 +108302 POINT(41.28159797529466 73.01003822407621) bank108302 +108303 POINT(40.56509150637408 73.23978827994867) bank108303 +108304 POINT(41.11979462479438 73.74774288717053) bank108304 +108305 POINT(40.06888939463007 73.90800051964447) bank108305 +108306 POINT(40.567568103787345 74.67574911221898) bank108306 +108307 POINT(40.78618241099189 74.26202060483668) bank108307 +108308 POINT(40.87658313195323 74.46943283702498) bank108308 +108309 POINT(41.38992385323895 73.16428093108644) bank108309 +108310 POINT(39.94761593793945 74.14447785342792) bank108310 +108311 POINT(41.56443356034306 74.84280856774555) bank108311 +108312 POINT(39.96774018025825 74.1582799241198) bank108312 +108313 POINT(41.35531488566321 74.60674712500952) bank108313 +108314 POINT(39.98989251792356 74.1926970961569) bank108314 +108315 POINT(39.748590612857654 73.7643481715568) bank108315 +108316 POINT(41.38531139614061 74.39080924747232) bank108316 +108317 POINT(40.292515077446765 73.57424381730394) bank108317 +108318 POINT(40.80159341459042 74.7718195114194) bank108318 +108319 POINT(40.84830996601248 73.8517492933088) bank108319 +108320 POINT(40.711568484212734 74.8733020662167) bank108320 +108321 POINT(41.134791788310615 74.51819135933276) bank108321 +108322 POINT(40.48929777625543 73.352256654964) bank108322 +108323 POINT(40.76731796065731 73.24460946399054) bank108323 +108324 POINT(40.9028913216495 73.21735751754322) bank108324 +108325 POINT(40.255886575554996 73.28469586705558) bank108325 +108326 POINT(39.718481901393986 74.75695083465301) bank108326 +108327 POINT(39.90444662049007 74.41816746811077) bank108327 +108328 POINT(41.32797218571585 73.94762036322281) bank108328 +108329 POINT(40.16562285083175 74.5628525258591) bank108329 +108330 POINT(41.481257778502595 73.85338967037744) bank108330 +108331 POINT(39.741506605305155 74.2808245911323) bank108331 +108332 POINT(39.99895000729081 73.32786630758233) bank108332 +108333 POINT(39.838428159514436 74.56409264359765) bank108333 +108334 POINT(41.441400170909816 74.70528045800474) bank108334 +108335 POINT(40.96803236793933 74.57867398286007) bank108335 +108336 POINT(40.474774449802 73.0617181829404) bank108336 +108337 POINT(39.93359706845351 74.39775630970503) bank108337 +108338 POINT(41.07357818254694 73.33420746049926) bank108338 +108339 POINT(40.510027188243996 73.12390626034136) bank108339 +108340 POINT(39.810782979087634 74.64232631878451) bank108340 +108341 POINT(40.05867933516994 73.25078547332981) bank108341 +108342 POINT(40.414081381276624 73.22146401560805) bank108342 +108343 POINT(39.804634067441874 73.69551640979223) bank108343 +108344 POINT(40.15058115180171 74.1368875596284) bank108344 +108345 POINT(40.226416413959846 74.33060857673217) bank108345 +108346 POINT(39.7742137796417 73.34138429441667) bank108346 +108347 POINT(40.252989549522596 73.34931568346906) bank108347 +108348 POINT(40.421061156875986 74.67211192124154) bank108348 +108349 POINT(41.08693005383653 74.50430419434142) bank108349 +108350 POINT(40.49052658748782 73.34693464210655) bank108350 +108351 POINT(41.02341970679865 74.43993675057806) bank108351 +108352 POINT(40.535842429170394 74.94073785453922) bank108352 +108353 POINT(40.68567730562204 74.89737836273892) bank108353 +108354 POINT(41.212673864214516 73.40845225547959) bank108354 +108355 POINT(41.254862910889315 73.90394269707791) bank108355 +108356 POINT(40.86228699534579 74.57741943098578) bank108356 +108357 POINT(40.85099374432567 73.66309033722781) bank108357 +108358 POINT(40.55550010353685 73.5426768471257) bank108358 +108359 POINT(39.947367238696316 74.42621813948375) bank108359 +108360 POINT(40.579240839621434 74.89143036352729) bank108360 +108361 POINT(40.546174257928186 73.5814249857801) bank108361 +108362 POINT(40.238314423739745 73.63993151464307) bank108362 +108363 POINT(40.525960753792894 73.02564437261118) bank108363 +108364 POINT(40.55395663333842 74.74297539608524) bank108364 +108365 POINT(40.72225745470016 73.14804811409427) bank108365 +108366 POINT(41.06570365298908 74.09881991211397) bank108366 +108367 POINT(41.02006366645791 74.88773464514182) bank108367 +108368 POINT(40.49124057721522 74.29435061922364) bank108368 +108369 POINT(41.5322860530662 74.18488758799792) bank108369 +108370 POINT(40.70224125261309 74.94472867911168) bank108370 +108371 POINT(40.83296699503315 73.40083327012323) bank108371 +108372 POINT(40.106908202047855 73.65290531941756) bank108372 +108373 POINT(40.949682520001254 74.72150705554377) bank108373 +108374 POINT(41.27853684026025 73.9357052390199) bank108374 +108375 POINT(41.71159751090311 73.74443467129808) bank108375 +108376 POINT(40.250663846918194 73.18458072451799) bank108376 +108377 POINT(40.98067845049 73.08922441473295) bank108377 +108378 POINT(41.35996680444628 74.0789024518529) bank108378 +108379 POINT(41.51272469446897 74.41682451949798) bank108379 +108380 POINT(41.29215133835907 74.33446657632835) bank108380 +108381 POINT(41.27055542316386 74.13798085029282) bank108381 +108382 POINT(40.49898183382832 74.25009280751813) bank108382 +108383 POINT(40.61161117712521 73.75866586153008) bank108383 +108384 POINT(41.02855577074466 73.78180009627015) bank108384 +108385 POINT(40.14034382082646 73.78889096513467) bank108385 +108386 POINT(41.44722990991849 73.49879001079854) bank108386 +108387 POINT(41.10461806954383 73.26723765283188) bank108387 +108388 POINT(41.62259235372919 73.86114920399264) bank108388 +108389 POINT(40.15768354124649 74.8276440286008) bank108389 +108390 POINT(40.1398014254113 74.56398525381243) bank108390 +108391 POINT(40.53000719006605 73.60547725904175) bank108391 +108392 POINT(41.481286930811976 73.53993653714768) bank108392 +108393 POINT(40.446167878248765 73.1194381862741) bank108393 +108394 POINT(40.614802131093484 74.12355302768657) bank108394 +108395 POINT(40.99472127914442 73.87969821979696) bank108395 +108396 POINT(40.868056050641464 73.4899815590576) bank108396 +108397 POINT(40.48979685884361 74.99769547364406) bank108397 +108398 POINT(39.713195913364956 74.40089997440602) bank108398 +108399 POINT(41.036192308984575 74.45340018627583) bank108399 +108400 POINT(40.77505415771559 74.84501622461221) bank108400 +108401 POINT(40.301317171186675 73.4532253407635) bank108401 +108402 POINT(40.92725140757756 73.72571428010878) bank108402 +108403 POINT(39.74472777767242 74.98406201588186) bank108403 +108404 POINT(41.13536598967998 73.58186810478642) bank108404 +108405 POINT(40.66996798099177 73.56818408621616) bank108405 +108406 POINT(40.89041443333105 74.85888101202869) bank108406 +108407 POINT(39.89326024449349 73.9600172903879) bank108407 +108408 POINT(40.90189036979731 73.02750231284149) bank108408 +108409 POINT(41.29498642752369 74.20603452203872) bank108409 +108410 POINT(41.70775872260962 74.99080327188423) bank108410 +108411 POINT(41.26741015127118 73.67233701183486) bank108411 +108412 POINT(41.624657135443876 73.23817566321216) bank108412 +108413 POINT(40.44699847947901 73.40743208569619) bank108413 +108414 POINT(41.224081192941945 73.754797943073) bank108414 +108415 POINT(40.68887542625375 74.87184867594449) bank108415 +108416 POINT(41.29112580205043 73.61234419733736) bank108416 +108417 POINT(40.54858267663834 74.87452170554427) bank108417 +108418 POINT(41.131229212988686 74.69845577724755) bank108418 +108419 POINT(41.09666180238341 74.41066697093962) bank108419 +108420 POINT(41.64370626100893 74.31756434044058) bank108420 +108421 POINT(39.72881619126964 74.41722736553652) bank108421 +108422 POINT(41.15655809755049 73.07341969768262) bank108422 +108423 POINT(41.17915013048824 74.54459467508318) bank108423 +108424 POINT(40.07532060751516 73.27083784283661) bank108424 +108425 POINT(40.17210928329775 74.17589374520897) bank108425 +108426 POINT(41.66760230052524 73.56389959345537) bank108426 +108427 POINT(41.40228416934589 74.63102668579552) bank108427 +108428 POINT(41.44966855234001 74.6111459992978) bank108428 +108429 POINT(41.61562376831512 73.54635203951361) bank108429 +108430 POINT(39.73132092632515 74.87805096924734) bank108430 +108431 POINT(40.639914477524066 73.66952525449148) bank108431 +108432 POINT(40.267345766104185 73.83376769074543) bank108432 +108433 POINT(39.861520486821746 74.61711339652977) bank108433 +108434 POINT(41.03777597658183 74.94782896674057) bank108434 +108435 POINT(40.6001937800435 73.30053059423089) bank108435 +108436 POINT(40.77341440328338 74.21733104553333) bank108436 +108437 POINT(40.40176512742277 74.56976775559711) bank108437 +108438 POINT(41.23510435999318 74.93139460977812) bank108438 +108439 POINT(40.71237013466778 74.85337559507218) bank108439 +108440 POINT(41.19885046850273 73.92320253992935) bank108440 +108441 POINT(40.75347730149651 73.29116082119307) bank108441 +108442 POINT(41.401452319328165 73.66106753840857) bank108442 +108443 POINT(41.26653821890668 74.92376408117384) bank108443 +108444 POINT(40.591424871285064 73.94914814571972) bank108444 +108445 POINT(41.1322691577987 74.23628934505889) bank108445 +108446 POINT(39.76313476008002 73.33556786131088) bank108446 +108447 POINT(40.82064755317849 74.86068270454183) bank108447 +108448 POINT(40.788366312413686 73.88152181569576) bank108448 +108449 POINT(41.403780017918066 73.20111635258388) bank108449 +108450 POINT(40.97662905037879 74.75290143290376) bank108450 +108451 POINT(41.03510154220116 74.3443954760453) bank108451 +108452 POINT(41.03112964792072 73.20001612564032) bank108452 +108453 POINT(39.79048596959207 73.33119667396626) bank108453 +108454 POINT(41.57074852051849 74.56055227797448) bank108454 +108455 POINT(41.60699340400011 74.31715603947562) bank108455 +108456 POINT(40.26739644899617 74.60852320157707) bank108456 +108457 POINT(40.40908843435896 74.91145002747258) bank108457 +108458 POINT(40.242735945707814 74.94659567989304) bank108458 +108459 POINT(40.92976828346817 74.37197879391742) bank108459 +108460 POINT(40.86543672368018 73.34458159288293) bank108460 +108461 POINT(41.706977944388505 74.6625434798109) bank108461 +108462 POINT(40.784037370333174 73.29726760812812) bank108462 +108463 POINT(40.11711617165762 74.31182768439862) bank108463 +108464 POINT(41.000365312868254 74.29874938238368) bank108464 +108465 POINT(41.54526167996087 74.91318514564415) bank108465 +108466 POINT(40.72915173596917 74.19295460647685) bank108466 +108467 POINT(40.23661085267072 73.73853274584177) bank108467 +108468 POINT(40.99610739714222 73.61982527910132) bank108468 +108469 POINT(40.13313491038251 73.29036957639956) bank108469 +108470 POINT(41.340947170814495 74.83959755007085) bank108470 +108471 POINT(39.829510389707536 74.28970219415163) bank108471 +108472 POINT(41.063031147945864 74.36342411646302) bank108472 +108473 POINT(39.999008427796674 74.79772182029552) bank108473 +108474 POINT(41.62086111486455 73.28522522949228) bank108474 +108475 POINT(39.98862479964729 74.68266919526558) bank108475 +108476 POINT(39.807932883187284 74.77685271399329) bank108476 +108477 POINT(40.50760394788104 73.04619810232046) bank108477 +108478 POINT(41.54274196139231 74.33531889674998) bank108478 +108479 POINT(40.900479111141394 73.36899840381263) bank108479 +108480 POINT(40.812062094716126 73.14301441177075) bank108480 +108481 POINT(41.65385075218275 74.61424818807534) bank108481 +108482 POINT(39.88367863197673 73.38985817237057) bank108482 +108483 POINT(40.56565864888094 74.67480593943728) bank108483 +108484 POINT(40.112050653753954 73.01832488770435) bank108484 +108485 POINT(39.75886984382513 74.54659663433628) bank108485 +108486 POINT(41.23248291129223 74.63697488724824) bank108486 +108487 POINT(40.838698163212804 74.9946906147307) bank108487 +108488 POINT(39.79984672535787 73.54549787648584) bank108488 +108489 POINT(40.42648313795387 74.33242473992135) bank108489 +108490 POINT(40.40435666875835 74.22880664573357) bank108490 +108491 POINT(39.83104096554552 74.26876426084135) bank108491 +108492 POINT(41.24717669519166 73.07601828035507) bank108492 +108493 POINT(41.55101437366068 74.78716568933848) bank108493 +108494 POINT(39.98571889944659 73.87424553885509) bank108494 +108495 POINT(39.982671229503154 74.15037275452515) bank108495 +108496 POINT(40.38913414471441 73.86670498529519) bank108496 +108497 POINT(41.12552770306083 74.67999382051647) bank108497 +108498 POINT(40.88220492354971 74.9625519184907) bank108498 +108499 POINT(39.86409593071801 73.2577269648062) bank108499 +108500 POINT(40.63480058612144 73.10595409036424) bank108500 +108501 POINT(41.43000651905922 73.26203425532823) bank108501 +108502 POINT(40.10156217659147 73.53517975386815) bank108502 +108503 POINT(40.16970951927974 73.02691494241714) bank108503 +108504 POINT(40.20430869259022 74.40716536711577) bank108504 +108505 POINT(41.46839538618572 74.9653978804478) bank108505 +108506 POINT(41.48223632974505 74.30060633448377) bank108506 +108507 POINT(40.39972745035332 73.4447808208406) bank108507 +108508 POINT(40.995561493114586 74.6800137009754) bank108508 +108509 POINT(40.970048248453686 73.82550386040995) bank108509 +108510 POINT(41.25650292914448 73.51409070134196) bank108510 +108511 POINT(40.78394506885705 74.53100972590988) bank108511 +108512 POINT(40.97795860283561 74.81255108054424) bank108512 +108513 POINT(41.48323871464606 73.47261646687492) bank108513 +108514 POINT(40.10946940511391 73.12793293825035) bank108514 +108515 POINT(41.02974096289895 74.71261674636673) bank108515 +108516 POINT(40.66957011341113 73.05130626999936) bank108516 +108517 POINT(41.21560371432142 73.97089327922) bank108517 +108518 POINT(41.325239194461794 74.30031805116666) bank108518 +108519 POINT(41.48898671378498 73.89426387327079) bank108519 +108520 POINT(41.48892881947816 73.31641756831114) bank108520 +108521 POINT(41.54653001142358 74.60532370793817) bank108521 +108522 POINT(40.6151395484908 74.47622323505773) bank108522 +108523 POINT(40.19200985828238 74.52576270333128) bank108523 +108524 POINT(41.09798904196291 73.47003580118756) bank108524 +108525 POINT(41.085146259266075 74.41655066316179) bank108525 +108526 POINT(40.68655932132764 74.45640521065856) bank108526 +108527 POINT(39.84233902730583 73.43675571170678) bank108527 +108528 POINT(40.54613680396796 74.2080668657774) bank108528 +108529 POINT(41.093549991504254 73.14041722149815) bank108529 +108530 POINT(40.565489206783866 73.14581110142102) bank108530 +108531 POINT(40.586134845935106 74.21487708081224) bank108531 +108532 POINT(40.93220074090437 74.89345577027207) bank108532 +108533 POINT(41.65426841862886 73.0471467142121) bank108533 +108534 POINT(41.62260728732881 74.73687828564944) bank108534 +108535 POINT(41.58538120952987 73.39686695758589) bank108535 +108536 POINT(40.413887441064055 74.93674847339972) bank108536 +108537 POINT(40.421807043635965 73.6549354193492) bank108537 +108538 POINT(39.95543859857598 74.30017362913105) bank108538 +108539 POINT(41.389928633843994 74.28884741160535) bank108539 +108540 POINT(41.01838322628935 73.08558058661544) bank108540 +108541 POINT(40.63776203518706 74.26251940937792) bank108541 +108542 POINT(41.07026082045015 73.48531840296826) bank108542 +108543 POINT(40.21013116654781 73.2602096565393) bank108543 +108544 POINT(41.11544240798495 74.9351056702216) bank108544 +108545 POINT(41.45035647323762 74.30548501095127) bank108545 +108546 POINT(39.82422815086266 74.67469472697225) bank108546 +108547 POINT(40.45347741054862 74.8131723405999) bank108547 +108548 POINT(41.16878344139647 73.66131312635432) bank108548 +108549 POINT(41.45657049469732 74.60126719487486) bank108549 +108550 POINT(41.47685142947474 74.27044806722088) bank108550 +108551 POINT(41.29086495756613 73.96482537522259) bank108551 +108552 POINT(40.329265118028694 73.98539358796468) bank108552 +108553 POINT(40.95561784975889 73.28569286641664) bank108553 +108554 POINT(39.7641734809638 74.05254962386034) bank108554 +108555 POINT(40.38205744055917 73.70383900145596) bank108555 +108556 POINT(40.08416704405453 74.20451307038857) bank108556 +108557 POINT(41.668375761790045 73.24580621949208) bank108557 +108558 POINT(41.14985085343911 74.75419416680161) bank108558 +108559 POINT(39.88207845693502 73.12510678246204) bank108559 +108560 POINT(41.02646132326575 74.60133056508829) bank108560 +108561 POINT(41.698432956718 74.77645103422307) bank108561 +108562 POINT(39.94277151076848 74.30688794678366) bank108562 +108563 POINT(41.03295537295031 73.05963406906324) bank108563 +108564 POINT(40.14010030739438 74.94831232501609) bank108564 +108565 POINT(41.05075013352933 74.88823936811356) bank108565 +108566 POINT(40.16934788940187 73.20964628417894) bank108566 +108567 POINT(40.03709148899824 73.22166294379002) bank108567 +108568 POINT(41.05934974089931 74.34519389022648) bank108568 +108569 POINT(40.351586411982936 73.85996936663277) bank108569 +108570 POINT(40.2371373295868 74.77952769299941) bank108570 +108571 POINT(41.47271939020435 74.92845267861902) bank108571 +108572 POINT(39.84099996639902 74.14316144408917) bank108572 +108573 POINT(40.60687418965766 74.45308674532872) bank108573 +108574 POINT(40.23489881700504 75.00529490740539) bank108574 +108575 POINT(41.51941304133542 74.24266990903143) bank108575 +108576 POINT(39.92728836900524 74.03459108386494) bank108576 +108577 POINT(41.0043056727901 74.88789689436561) bank108577 +108578 POINT(39.95262582520135 73.96832224797626) bank108578 +108579 POINT(40.75496569826694 74.67079242795367) bank108579 +108580 POINT(41.208127422153105 74.92311413016424) bank108580 +108581 POINT(41.669456717825 74.57232862880238) bank108581 +108582 POINT(40.90507734140833 73.55864927809603) bank108582 +108583 POINT(41.209006070350334 74.65847962540465) bank108583 +108584 POINT(41.140381765755606 74.67991961271728) bank108584 +108585 POINT(40.17235319803639 73.93024150134026) bank108585 +108586 POINT(41.1937635090294 73.03803112494516) bank108586 +108587 POINT(41.11348242726932 73.33699953654461) bank108587 +108588 POINT(41.45272103790389 73.84329753106528) bank108588 +108589 POINT(41.29563086697092 74.7173529032307) bank108589 +108590 POINT(40.76468037233722 73.28390523952547) bank108590 +108591 POINT(41.54345800569583 73.87333489392805) bank108591 +108592 POINT(40.680750771045965 74.35908669176538) bank108592 +108593 POINT(41.26903934488817 74.21330061406175) bank108593 +108594 POINT(39.915565785547535 74.50399901137646) bank108594 +108595 POINT(41.33494887443952 73.25094674119974) bank108595 +108596 POINT(41.69648846869936 74.19809778877851) bank108596 +108597 POINT(40.147993367753315 74.14748572525545) bank108597 +108598 POINT(41.58610741111967 73.12045364180446) bank108598 +108599 POINT(40.113945372273555 74.90971027507662) bank108599 +108600 POINT(40.715227460440076 73.06451568346252) bank108600 +108601 POINT(40.503920958884706 74.14743889758331) bank108601 +108602 POINT(40.27955446761131 74.65156739148591) bank108602 +108603 POINT(40.867324562645905 73.01813980330802) bank108603 +108604 POINT(41.35908285244349 73.74447620000258) bank108604 +108605 POINT(40.10585534813581 74.00667034556322) bank108605 +108606 POINT(39.770256256633346 74.68569422801652) bank108606 +108607 POINT(39.77521810963574 73.01110776005768) bank108607 +108608 POINT(40.296078842236405 74.77961459596317) bank108608 +108609 POINT(40.69526991691408 74.26239207613177) bank108609 +108610 POINT(41.28224104870827 73.20258646593021) bank108610 +108611 POINT(40.61016837780285 73.69224258298456) bank108611 +108612 POINT(40.81581625204598 73.63188130275613) bank108612 +108613 POINT(39.941826707142496 74.5731206166658) bank108613 +108614 POINT(41.61791977543644 73.60432694363901) bank108614 +108615 POINT(39.94906304145486 74.59434112401209) bank108615 +108616 POINT(41.1434422385608 74.41008979184312) bank108616 +108617 POINT(41.20427344300128 73.63908031306502) bank108617 +108618 POINT(40.14593852865298 74.29271863676104) bank108618 +108619 POINT(40.58679146356908 73.28129054945987) bank108619 +108620 POINT(40.68992393175884 73.583935099019) bank108620 +108621 POINT(41.695570253817365 74.43332303706381) bank108621 +108622 POINT(39.84887391044281 73.63662033748003) bank108622 +108623 POINT(40.72946591278681 73.45474703444873) bank108623 +108624 POINT(41.41777267587852 73.51777502923385) bank108624 +108625 POINT(40.98325717294962 73.60146475701075) bank108625 +108626 POINT(41.227346665403275 73.51566352254619) bank108626 +108627 POINT(39.777715393620966 74.64785401196502) bank108627 +108628 POINT(39.815778206652425 74.987157430121) bank108628 +108629 POINT(39.818764823015286 74.9697420309941) bank108629 +108630 POINT(39.78554957550479 74.5786246187125) bank108630 +108631 POINT(40.768679712190256 73.36047533275672) bank108631 +108632 POINT(41.238001096672654 74.97061255985564) bank108632 +108633 POINT(41.41514970153204 73.39498070067206) bank108633 +108634 POINT(41.48570250153309 74.37512189440463) bank108634 +108635 POINT(40.68505780673288 73.58048446055844) bank108635 +108636 POINT(40.88992373839417 73.78352947515985) bank108636 +108637 POINT(40.77636083709382 74.35742111622056) bank108637 +108638 POINT(41.46234889324958 74.93539606698604) bank108638 +108639 POINT(41.62362144088184 73.08454074881999) bank108639 +108640 POINT(41.27588609797328 74.42239941987543) bank108640 +108641 POINT(40.24919129290243 73.92149315491925) bank108641 +108642 POINT(40.74499151547784 74.30668307980052) bank108642 +108643 POINT(40.89684956705841 74.67521625475891) bank108643 +108644 POINT(41.10708939968648 74.15902198920146) bank108644 +108645 POINT(40.57774976455306 73.57098875825966) bank108645 +108646 POINT(40.62473443647646 74.00822034169936) bank108646 +108647 POINT(40.94032916917729 73.17429561071327) bank108647 +108648 POINT(41.28451216727982 73.09473163998742) bank108648 +108649 POINT(40.36793249548425 73.85564157831885) bank108649 +108650 POINT(41.484257732253646 73.14320291600751) bank108650 +108651 POINT(40.34152015472812 73.74843588485706) bank108651 +108652 POINT(40.956480131255574 73.37240903251511) bank108652 +108653 POINT(40.67300241039037 73.14172171768332) bank108653 +108654 POINT(41.15841068470841 73.69503276817025) bank108654 +108655 POINT(41.569542523647925 73.95768114577213) bank108655 +108656 POINT(40.333701694344356 73.37046480092509) bank108656 +108657 POINT(40.301875559102655 73.89475967576944) bank108657 +108658 POINT(41.28042856140657 73.62268410077743) bank108658 +108659 POINT(41.15808197900535 73.62937777375122) bank108659 +108660 POINT(40.698355993647546 73.63725709877014) bank108660 +108661 POINT(41.4963919818364 74.04114260552866) bank108661 +108662 POINT(40.49597923689014 73.54397610747533) bank108662 +108663 POINT(40.99202780581363 73.62727142880206) bank108663 +108664 POINT(41.24615265176816 73.19311618590714) bank108664 +108665 POINT(40.71002685788336 73.37281980915996) bank108665 +108666 POINT(41.408403903770555 73.72762192136652) bank108666 +108667 POINT(40.30202846323518 74.57351453049243) bank108667 +108668 POINT(40.83690163962777 73.10840134396818) bank108668 +108669 POINT(39.9526145345851 74.74994056539002) bank108669 +108670 POINT(41.01435276356304 73.15748014242925) bank108670 +108671 POINT(41.454336656887314 73.88787476014562) bank108671 +108672 POINT(41.158995574703766 74.89040717301295) bank108672 +108673 POINT(40.050320156113784 73.8182157842379) bank108673 +108674 POINT(39.947842109191576 73.9725579795981) bank108674 +108675 POINT(40.94624039122849 74.36997907692889) bank108675 +108676 POINT(40.69809271339847 73.93601896320531) bank108676 +108677 POINT(40.24293156771646 74.72564464754743) bank108677 +108678 POINT(40.764627410594855 74.1399624234426) bank108678 +108679 POINT(41.014391044690285 74.69782279859055) bank108679 +108680 POINT(40.825212857095764 74.37964862711851) bank108680 +108681 POINT(41.56583102080528 74.5816128418335) bank108681 +108682 POINT(40.98564224224573 73.12083652924242) bank108682 +108683 POINT(40.34961635644405 73.95288320578669) bank108683 +108684 POINT(41.012637238151356 74.56403161540379) bank108684 +108685 POINT(41.34150148456916 74.95479912540505) bank108685 +108686 POINT(39.88714878522214 74.32623012607043) bank108686 +108687 POINT(39.84412775213042 73.76953753987334) bank108687 +108688 POINT(39.87769167803495 74.34085984418495) bank108688 +108689 POINT(39.888546338102636 73.1170013731283) bank108689 +108690 POINT(40.4272726955158 74.66254449391012) bank108690 +108691 POINT(40.778298839208475 74.34959650471697) bank108691 +108692 POINT(41.52625665338822 74.6791266535952) bank108692 +108693 POINT(40.22256383916642 74.06782267258579) bank108693 +108694 POINT(40.70000110698175 74.27429485853834) bank108694 +108695 POINT(39.817676840556906 74.2543654429103) bank108695 +108696 POINT(40.53576647709535 74.42693721193156) bank108696 +108697 POINT(39.91152919584639 73.75425190206764) bank108697 +108698 POINT(40.61100502362354 73.02133403020586) bank108698 +108699 POINT(39.79716738622949 74.83519895089086) bank108699 +108700 POINT(41.256869170417914 73.63969048186804) bank108700 +108701 POINT(40.514270016854525 74.10270316101638) bank108701 +108702 POINT(40.85726640655578 73.11865690568061) bank108702 +108703 POINT(40.913031160926465 74.13005953983519) bank108703 +108704 POINT(41.31854341905162 75.00577851448007) bank108704 +108705 POINT(40.1808746581238 74.3929573372122) bank108705 +108706 POINT(40.2345482510448 73.53144725124746) bank108706 +108707 POINT(40.46687902384418 74.12878135874034) bank108707 +108708 POINT(41.182694283856144 73.22156140213846) bank108708 +108709 POINT(39.731342072380016 74.94965665550671) bank108709 +108710 POINT(40.769950697176085 74.39289367476067) bank108710 +108711 POINT(41.687192723278216 75.00338206251743) bank108711 +108712 POINT(40.170719745780794 73.38012560571028) bank108712 +108713 POINT(39.7997846491404 74.69202202183361) bank108713 +108714 POINT(40.47126157370178 74.11904826827698) bank108714 +108715 POINT(40.88784010999267 73.3862721776314) bank108715 +108716 POINT(41.08360882116232 74.30748878117092) bank108716 +108717 POINT(39.7167582437354 73.29598820497583) bank108717 +108718 POINT(40.12028987609045 73.86044261515185) bank108718 +108719 POINT(41.25616320792412 73.64618030101084) bank108719 +108720 POINT(40.64756747573464 73.62164991734888) bank108720 +108721 POINT(40.09978364012017 73.5463084267997) bank108721 +108722 POINT(40.6129830900201 73.27147564780236) bank108722 +108723 POINT(41.41994103610339 73.94924331483816) bank108723 +108724 POINT(40.756573710945794 73.24455584812084) bank108724 +108725 POINT(40.42246425695781 74.22112912138759) bank108725 +108726 POINT(40.63454750888674 74.07483600703772) bank108726 +108727 POINT(40.72710184321882 74.06141491916807) bank108727 +108728 POINT(40.121264669375826 73.82582160405681) bank108728 +108729 POINT(40.1576860815317 73.18405370403988) bank108729 +108730 POINT(39.79288139502346 74.85122346122004) bank108730 +108731 POINT(41.50945859095585 74.50914527843427) bank108731 +108732 POINT(40.18205547682943 74.61811254059288) bank108732 +108733 POINT(41.50586157143272 74.53232208380793) bank108733 +108734 POINT(40.88119318503799 74.40940923222962) bank108734 +108735 POINT(40.45315068780598 73.5476596984326) bank108735 +108736 POINT(41.357401943812384 73.38930924818236) bank108736 +108737 POINT(41.6687162248076 73.50861155427393) bank108737 +108738 POINT(41.31101870671737 73.45159188220154) bank108738 +108739 POINT(40.619454895539896 73.7143045775306) bank108739 +108740 POINT(40.95700265555117 74.87676414733882) bank108740 +108741 POINT(41.68986126175687 73.83336486842123) bank108741 +108742 POINT(41.58391988989257 74.0951194277218) bank108742 +108743 POINT(40.30712535809059 73.61866329871036) bank108743 +108744 POINT(39.7265173784654 73.69281209053115) bank108744 +108745 POINT(39.97941172852132 73.83462436265025) bank108745 +108746 POINT(40.22334036063502 73.78847752538107) bank108746 +108747 POINT(41.48411587306499 73.55897257193753) bank108747 +108748 POINT(41.25095687486595 73.18807953030284) bank108748 +108749 POINT(41.52297891095482 74.4300983121638) bank108749 +108750 POINT(41.4674442117319 73.51900290154313) bank108750 +108751 POINT(40.97715027869774 73.6128172100997) bank108751 +108752 POINT(39.79550836909545 73.83818713466215) bank108752 +108753 POINT(40.409952750771126 73.67718759490867) bank108753 +108754 POINT(41.199647812677945 74.14762239824061) bank108754 +108755 POINT(40.01527315228295 73.7244782296981) bank108755 +108756 POINT(39.79827602596525 74.9818592451957) bank108756 +108757 POINT(40.16482040848656 73.15812940770049) bank108757 +108758 POINT(40.89573461000131 74.81787333561364) bank108758 +108759 POINT(41.091428263891544 73.98190753023773) bank108759 +108760 POINT(41.08324726133606 73.60238461883203) bank108760 +108761 POINT(40.370380662956705 73.77362336684718) bank108761 +108762 POINT(40.65891512595804 73.74707232841905) bank108762 +108763 POINT(40.48272616745491 73.90683563726111) bank108763 +108764 POINT(40.896403468614416 74.56858859286928) bank108764 +108765 POINT(41.336818030214786 73.86052620037596) bank108765 +108766 POINT(40.32439540953269 73.8317222052085) bank108766 +108767 POINT(39.83346352892709 73.38970283021762) bank108767 +108768 POINT(40.32285802704007 73.76934343008769) bank108768 +108769 POINT(41.379471705261 73.63147398822674) bank108769 +108770 POINT(41.6822192940158 73.37826600868124) bank108770 +108771 POINT(39.80013779039579 73.48849392154624) bank108771 +108772 POINT(40.2798213313532 73.3125949065259) bank108772 +108773 POINT(40.93660416798218 73.25779791609804) bank108773 +108774 POINT(40.18656567268308 73.39424367806846) bank108774 +108775 POINT(39.93918571231165 74.9588398214413) bank108775 +108776 POINT(41.496098546872545 74.35935337098724) bank108776 +108777 POINT(40.51577706244346 73.7872727485225) bank108777 +108778 POINT(39.96829193852933 74.79565295961604) bank108778 +108779 POINT(40.72791918385873 73.21659047072836) bank108779 +108780 POINT(41.6155200714972 73.18789656340068) bank108780 +108781 POINT(40.32366125889194 73.36176366146282) bank108781 +108782 POINT(41.236123467965754 74.4791624448129) bank108782 +108783 POINT(41.32031087751373 74.51198297716226) bank108783 +108784 POINT(40.69538169736972 74.4045351606865) bank108784 +108785 POINT(40.294924669663274 73.79555361292509) bank108785 +108786 POINT(40.42446526302453 73.94567564924958) bank108786 +108787 POINT(40.82734993416153 74.82010771939701) bank108787 +108788 POINT(40.573348012501775 73.87096517859743) bank108788 +108789 POINT(41.21379244131833 74.72674165952319) bank108789 +108790 POINT(40.673982536180915 73.84489683776317) bank108790 +108791 POINT(40.99493905219457 73.03953960469966) bank108791 +108792 POINT(40.5927505506914 73.09857130460334) bank108792 +108793 POINT(40.52393696027319 73.30646464830164) bank108793 +108794 POINT(41.21626858267508 74.66710576781874) bank108794 +108795 POINT(39.786386242127364 74.2577487478422) bank108795 +108796 POINT(39.95877390732425 73.63817310322587) bank108796 +108797 POINT(41.094105209872076 74.0482560461869) bank108797 +108798 POINT(41.6163462556415 74.85974473451118) bank108798 +108799 POINT(40.7932585781692 73.13757946959966) bank108799 +108800 POINT(41.64278826801334 73.38530897048487) bank108800 +108801 POINT(40.993254803035384 73.29282990769835) bank108801 +108802 POINT(41.706631330223985 73.89818770720905) bank108802 +108803 POINT(40.55980830597387 73.49399975915217) bank108803 +108804 POINT(41.573065130891734 73.25362943454236) bank108804 +108805 POINT(39.85535540718964 74.82342412841156) bank108805 +108806 POINT(40.30495609428982 73.67900995644065) bank108806 +108807 POINT(40.601763935134095 74.99894054346876) bank108807 +108808 POINT(41.236270235192485 73.55312801973928) bank108808 +108809 POINT(39.77291239297449 74.30427925927083) bank108809 +108810 POINT(40.48134652971433 74.5951249815263) bank108810 +108811 POINT(40.39062643810828 74.27471775869041) bank108811 +108812 POINT(41.361811058500024 73.32161411093139) bank108812 +108813 POINT(41.21235983735443 73.2819634968305) bank108813 +108814 POINT(40.23417234867266 73.22664232531253) bank108814 +108815 POINT(40.41923331477936 74.67010381223886) bank108815 +108816 POINT(40.627946413552515 74.16636629042938) bank108816 +108817 POINT(40.01300120762847 73.03947912400272) bank108817 +108818 POINT(40.5763701983028 73.97008926908809) bank108818 +108819 POINT(40.54410182142555 73.80654668597651) bank108819 +108820 POINT(39.87989347776751 73.43537801061954) bank108820 +108821 POINT(40.55003525463054 74.28692097991153) bank108821 +108822 POINT(40.35843929841397 74.73453718518246) bank108822 +108823 POINT(41.008661375100296 73.73239431929998) bank108823 +108824 POINT(40.71746633854222 74.61955063674024) bank108824 +108825 POINT(39.77463144626209 73.47480603692988) bank108825 +108826 POINT(40.223300201403575 74.11234029642912) bank108826 +108827 POINT(40.38095030793908 74.89683630297526) bank108827 +108828 POINT(41.50886491045544 74.35539606715264) bank108828 +108829 POINT(40.53629343545393 74.14808183720896) bank108829 +108830 POINT(40.195820111104496 74.75845140252291) bank108830 +108831 POINT(41.08258442278911 74.91379204122592) bank108831 +108832 POINT(40.84225481743635 74.59394773904043) bank108832 +108833 POINT(39.89638120123222 73.92059721021329) bank108833 +108834 POINT(41.56708283120169 73.6056662836961) bank108834 +108835 POINT(41.47203288993216 74.0101348453007) bank108835 +108836 POINT(40.930469336545826 74.88600789252351) bank108836 +108837 POINT(40.28902046823315 74.41260653410475) bank108837 +108838 POINT(39.9144781391814 74.19895048637146) bank108838 +108839 POINT(40.41155783713991 74.01693441896016) bank108839 +108840 POINT(41.636415672183276 73.24418421112895) bank108840 +108841 POINT(41.475478531086225 74.15515516720104) bank108841 +108842 POINT(40.2614260483013 74.47019468210068) bank108842 +108843 POINT(39.81195683617666 74.17877946557914) bank108843 +108844 POINT(40.82944755244612 74.90835818992662) bank108844 +108845 POINT(41.4615193894639 73.48748898249654) bank108845 +108846 POINT(40.358316710641034 74.78426358438779) bank108846 +108847 POINT(40.331453083924195 73.93607936811217) bank108847 +108848 POINT(40.688695973101524 74.41088629933937) bank108848 +108849 POINT(40.81554379311791 74.69826011049888) bank108849 +108850 POINT(40.31800450659943 73.14178855723866) bank108850 +108851 POINT(40.172449496775855 74.02720622772677) bank108851 +108852 POINT(40.160165657330964 73.75245393325376) bank108852 +108853 POINT(40.0768532477278 73.36883625177316) bank108853 +108854 POINT(39.71558626611425 73.83752130116177) bank108854 +108855 POINT(40.8318525134477 75.00065941363457) bank108855 +108856 POINT(40.0377218051586 73.16825793522386) bank108856 +108857 POINT(39.92830596906968 73.28536539779012) bank108857 +108858 POINT(40.82210618881949 74.24997416518255) bank108858 +108859 POINT(40.42308259013888 74.55592548242035) bank108859 +108860 POINT(40.06041002542446 73.89265507077667) bank108860 +108861 POINT(39.74215501559816 73.31847430756918) bank108861 +108862 POINT(41.37957396446567 73.86543969171143) bank108862 +108863 POINT(39.932757428195345 73.33245709462902) bank108863 +108864 POINT(40.18184714266462 73.62160682553525) bank108864 +108865 POINT(40.439959281697135 74.06095056429227) bank108865 +108866 POINT(40.27940712678002 74.90669244474634) bank108866 +108867 POINT(40.47192804264536 74.5212281152648) bank108867 +108868 POINT(40.536679678950804 73.75165707305709) bank108868 +108869 POINT(41.51085200687903 73.49363981005207) bank108869 +108870 POINT(41.41436307026488 74.05227241623192) bank108870 +108871 POINT(40.168306728397106 74.49841462135483) bank108871 +108872 POINT(40.45965628071343 73.25629173850233) bank108872 +108873 POINT(40.48254678894109 73.85677831976444) bank108873 +108874 POINT(40.43452160324286 74.10495535436237) bank108874 +108875 POINT(40.10176010956599 73.53724078039787) bank108875 +108876 POINT(40.79906181857546 74.15995978551602) bank108876 +108877 POINT(40.28335078618342 73.14550984325642) bank108877 +108878 POINT(41.38563352396727 73.92272503572035) bank108878 +108879 POINT(40.76018966188012 74.57422412783373) bank108879 +108880 POINT(39.714277277443834 73.50188371925861) bank108880 +108881 POINT(40.94114403424488 73.67528081969776) bank108881 +108882 POINT(41.11646913999697 73.0239462959455) bank108882 +108883 POINT(40.75063791984023 74.16227604706276) bank108883 +108884 POINT(40.89118922796817 73.68413071684138) bank108884 +108885 POINT(41.20102505315608 74.61687926085237) bank108885 +108886 POINT(40.655716015667196 73.10929116044967) bank108886 +108887 POINT(41.4133725869888 73.17154330786653) bank108887 +108888 POINT(40.058956522052796 73.23577280005334) bank108888 +108889 POINT(41.16023424458011 73.62783286366528) bank108889 +108890 POINT(40.7145076190101 73.61296587707506) bank108890 +108891 POINT(40.686051559380545 74.40853384582167) bank108891 +108892 POINT(40.163804525837584 73.93652342122408) bank108892 +108893 POINT(40.54663805773072 73.6952046652802) bank108893 +108894 POINT(40.560409153279004 74.48868688424893) bank108894 +108895 POINT(40.992739524869464 73.69195980125072) bank108895 +108896 POINT(40.16270593555734 74.44721639825295) bank108896 +108897 POINT(41.217460088216036 74.34090316394023) bank108897 +108898 POINT(40.76654392531854 73.6460355191923) bank108898 +108899 POINT(41.44597515572531 74.12200669427727) bank108899 +108900 POINT(41.27696423026593 74.27048338802405) bank108900 +108901 POINT(40.29827846371047 74.45037471559588) bank108901 +108902 POINT(40.98020370668874 74.8163425307855) bank108902 +108903 POINT(39.87622411103953 74.56702958535575) bank108903 +108904 POINT(40.46393692929976 74.98560676195866) bank108904 +108905 POINT(41.31214286922783 73.77108816246555) bank108905 +108906 POINT(40.37666168125637 73.7723480097775) bank108906 +108907 POINT(40.81181026832847 74.83985743385318) bank108907 +108908 POINT(41.500457121981256 74.03341700647199) bank108908 +108909 POINT(40.08518793254404 73.44332400912695) bank108909 +108910 POINT(41.564322260437784 74.74156300369302) bank108910 +108911 POINT(41.41608554961903 74.9735777398458) bank108911 +108912 POINT(40.83009806747214 74.41044128212923) bank108912 +108913 POINT(40.20696868332169 73.25930950722356) bank108913 +108914 POINT(40.30419371983642 73.63610684514163) bank108914 +108915 POINT(39.949407393096195 73.60723421043281) bank108915 +108916 POINT(40.193710553098136 73.04534287770927) bank108916 +108917 POINT(40.48932578319889 74.07709130961071) bank108917 +108918 POINT(40.354580318806995 73.96837495864459) bank108918 +108919 POINT(40.58237958679229 73.90214788060074) bank108919 +108920 POINT(40.83856876621427 73.55772720187828) bank108920 +108921 POINT(40.22696269195978 73.09146152793556) bank108921 +108922 POINT(41.483493807699155 73.72151919955473) bank108922 +108923 POINT(40.50403276819299 74.07440162152098) bank108923 +108924 POINT(39.95349075893483 73.3465825026515) bank108924 +108925 POINT(40.881748640060685 73.21551130800407) bank108925 +108926 POINT(40.27751007684762 73.40286946291549) bank108926 +108927 POINT(40.57446050694166 74.77848219561683) bank108927 +108928 POINT(40.01928166843783 73.25456110558606) bank108928 +108929 POINT(41.26081623016302 74.21740218725024) bank108929 +108930 POINT(40.995590710947056 73.9042556641361) bank108930 +108931 POINT(40.455451334400145 73.99328731602924) bank108931 +108932 POINT(41.59337785250756 73.4902988584347) bank108932 +108933 POINT(40.464100950635775 74.7592530384991) bank108933 +108934 POINT(40.02100066918662 74.5123996393653) bank108934 +108935 POINT(40.172323589632164 73.67245695787565) bank108935 +108936 POINT(40.61386120430965 73.28184735951952) bank108936 +108937 POINT(41.19700196471535 74.74729021971775) bank108937 +108938 POINT(40.55487449363427 73.83867342541362) bank108938 +108939 POINT(41.688850669126104 74.08459717105775) bank108939 +108940 POINT(40.12669596034381 73.24604042793247) bank108940 +108941 POINT(40.66733567351681 74.45672953761637) bank108941 +108942 POINT(41.106788662891425 74.50438499786823) bank108942 +108943 POINT(41.082305415462926 73.40477301781308) bank108943 +108944 POINT(40.08738742640664 74.58607766138758) bank108944 +108945 POINT(41.14075644406658 73.81191609837485) bank108945 +108946 POINT(40.889726117595636 73.79158317883419) bank108946 +108947 POINT(41.349429300740645 74.91074749662128) bank108947 +108948 POINT(41.07509549953956 74.71679241547001) bank108948 +108949 POINT(40.19360628649888 73.11340633239088) bank108949 +108950 POINT(41.211330440099935 73.02651463538437) bank108950 +108951 POINT(39.81110329949211 74.24103782850129) bank108951 +108952 POINT(39.744944023525285 74.15764508684677) bank108952 +108953 POINT(40.865806744642484 74.13392739072243) bank108953 +108954 POINT(41.04312463233095 73.25847670716739) bank108954 +108955 POINT(40.91596924323685 74.05897987758271) bank108955 +108956 POINT(40.29230107270291 73.00777087758897) bank108956 +108957 POINT(41.380428721016635 73.80497073701086) bank108957 +108958 POINT(41.286693874585346 73.62308254958319) bank108958 +108959 POINT(41.12555406836633 74.74396166777225) bank108959 +108960 POINT(40.45110255220795 74.1161451296596) bank108960 +108961 POINT(41.422088324857334 73.64701369161499) bank108961 +108962 POINT(40.01508439571163 74.89464451333025) bank108962 +108963 POINT(39.84663583360461 73.6400676521466) bank108963 +108964 POINT(40.518381072378254 73.52723233830748) bank108964 +108965 POINT(41.35587307209094 73.6120702370003) bank108965 +108966 POINT(41.30799066097054 73.12946535811307) bank108966 +108967 POINT(41.66721964513503 74.28691435304997) bank108967 +108968 POINT(41.242428405636026 74.6210573904518) bank108968 +108969 POINT(41.43194764780751 73.49611519645384) bank108969 +108970 POINT(39.97962053594085 73.47588678724784) bank108970 +108971 POINT(40.49255671513027 74.40920124301375) bank108971 +108972 POINT(40.88682138637487 74.39797757263659) bank108972 +108973 POINT(41.623141492849804 73.54378016194258) bank108973 +108974 POINT(40.692425877557454 74.10565162755579) bank108974 +108975 POINT(39.88628787611595 73.93920683012004) bank108975 +108976 POINT(41.05699806443765 74.51226083754908) bank108976 +108977 POINT(39.857559775642535 73.76262345000025) bank108977 +108978 POINT(39.82054848572388 73.35923217972099) bank108978 +108979 POINT(41.61852527469951 73.80254671686323) bank108979 +108980 POINT(41.57409410931361 74.07602824606998) bank108980 +108981 POINT(40.84631358145781 74.61092018500214) bank108981 +108982 POINT(40.06682072795384 74.25403132853293) bank108982 +108983 POINT(40.5158007240998 73.02969259650983) bank108983 +108984 POINT(40.0810608758562 73.08477090260294) bank108984 +108985 POINT(41.068780387247614 73.15294502674642) bank108985 +108986 POINT(40.44925983111684 73.91721868394916) bank108986 +108987 POINT(40.71782986567001 73.804045751554) bank108987 +108988 POINT(40.30324712274794 73.6447089752792) bank108988 +108989 POINT(39.838408802252694 74.54393041902102) bank108989 +108990 POINT(40.40751499242346 74.88703545147533) bank108990 +108991 POINT(41.230962561956275 73.22033694858905) bank108991 +108992 POINT(41.3528350332472 74.77696726725729) bank108992 +108993 POINT(41.17436125133179 73.59051021903713) bank108993 +108994 POINT(41.589151115856616 74.08977787036991) bank108994 +108995 POINT(40.069432258601424 73.50317258280371) bank108995 +108996 POINT(40.60767925106909 73.50969698200409) bank108996 +108997 POINT(40.51236568028058 73.45437524850394) bank108997 +108998 POINT(41.083298868657344 73.86529938583614) bank108998 +108999 POINT(39.958902543812634 73.85925557759904) bank108999 +109000 POINT(40.66225532275564 74.67482024632505) bank109000 +109001 POINT(41.189728575895515 74.77533529893442) bank109001 +109002 POINT(41.06065861959184 74.40593341183418) bank109002 +109003 POINT(39.98227282881936 74.42414070888871) bank109003 +109004 POINT(41.27160625456404 74.62486784199353) bank109004 +109005 POINT(41.2393845363445 73.09128858572856) bank109005 +109006 POINT(40.93925687677784 74.18488287305836) bank109006 +109007 POINT(41.707451700227345 73.05381357974053) bank109007 +109008 POINT(40.645738717864205 73.93253096915679) bank109008 +109009 POINT(41.5623407155828 74.86548707453233) bank109009 +109010 POINT(40.13967895207259 73.83949557266268) bank109010 +109011 POINT(40.54126502771039 73.74591194797699) bank109011 +109012 POINT(39.729610110411166 73.77994016525918) bank109012 +109013 POINT(40.114435818768925 74.7622227092288) bank109013 +109014 POINT(41.33962781104473 73.59083056875521) bank109014 +109015 POINT(40.150047875555906 73.379159670046) bank109015 +109016 POINT(40.30237726345711 73.19237024633334) bank109016 +109017 POINT(40.2456399547482 74.83405017370893) bank109017 +109018 POINT(41.64726313940768 74.96039637612068) bank109018 +109019 POINT(40.72491647657227 73.7379995393579) bank109019 +109020 POINT(40.12389662987365 73.971953455868) bank109020 +109021 POINT(41.17011252576285 74.59521394738577) bank109021 +109022 POINT(40.97620860979875 74.11808900428416) bank109022 +109023 POINT(40.340789972980666 74.83342132649891) bank109023 +109024 POINT(40.964935786605466 73.59803010225913) bank109024 +109025 POINT(41.51963831070513 74.02954504551731) bank109025 +109026 POINT(40.724915363311055 73.82737544947848) bank109026 +109027 POINT(41.38934913412346 73.29894292246895) bank109027 +109028 POINT(41.40554563996883 74.24836034926342) bank109028 +109029 POINT(40.18479657754131 74.4762282034879) bank109029 +109030 POINT(40.22539551740179 74.09935281500434) bank109030 +109031 POINT(41.24601166696052 74.2171953402704) bank109031 +109032 POINT(41.175760165241364 73.7264227811191) bank109032 +109033 POINT(41.41927059533218 74.14978014656897) bank109033 +109034 POINT(41.20595776361915 73.32666160508953) bank109034 +109035 POINT(40.15728630984991 73.91056791736939) bank109035 +109036 POINT(39.80539875378898 73.16461018308085) bank109036 +109037 POINT(40.79591416019675 73.1399880559673) bank109037 +109038 POINT(41.70746639383078 74.90288504994017) bank109038 +109039 POINT(41.300339674936424 74.34953543368482) bank109039 +109040 POINT(40.49951815334026 73.8502968319144) bank109040 +109041 POINT(40.61907343449459 74.62613940019543) bank109041 +109042 POINT(39.789156823148105 74.32552781273584) bank109042 +109043 POINT(41.094375382166064 73.15486117562884) bank109043 +109044 POINT(39.77490983709847 74.4271602031906) bank109044 +109045 POINT(40.210748300091964 74.96028384955997) bank109045 +109046 POINT(40.698188598288155 74.74119544716451) bank109046 +109047 POINT(40.743356114102205 74.21351803991054) bank109047 +109048 POINT(41.21250187271825 74.73786013211075) bank109048 +109049 POINT(40.38372448305598 74.56645838830322) bank109049 +109050 POINT(40.130376000810486 74.20754772684849) bank109050 +109051 POINT(41.335980613833264 74.22488627378084) bank109051 +109052 POINT(41.14207515094246 74.58378705093385) bank109052 +109053 POINT(40.07675623097687 74.95875273315154) bank109053 +109054 POINT(41.21768497542219 73.38348136567883) bank109054 +109055 POINT(40.08634407065359 74.41049957360578) bank109055 +109056 POINT(40.919277761210346 73.51295976870881) bank109056 +109057 POINT(39.84632808622317 74.95090930046842) bank109057 +109058 POINT(40.356271936954435 73.7872104566418) bank109058 +109059 POINT(39.76974376743187 73.45987879147351) bank109059 +109060 POINT(40.39172913594719 74.70603297066495) bank109060 +109061 POINT(40.70997258932131 74.75965888216984) bank109061 +109062 POINT(41.20688678331948 73.78258431877465) bank109062 +109063 POINT(41.160875387026834 73.06124863016967) bank109063 +109064 POINT(40.24381059200977 74.33010612762212) bank109064 +109065 POINT(39.92933301972892 74.07997125474745) bank109065 +109066 POINT(41.48459696650638 73.47572153398303) bank109066 +109067 POINT(40.350300447819784 74.55741944164443) bank109067 +109068 POINT(40.62796138817117 73.01773241787285) bank109068 +109069 POINT(41.64188447263279 73.35885412338214) bank109069 +109070 POINT(41.06512247836001 73.30600584038751) bank109070 +109071 POINT(41.02024030446825 74.0852594422342) bank109071 +109072 POINT(41.54245630615583 74.99372808968539) bank109072 +109073 POINT(40.71852540891571 73.12690785625497) bank109073 +109074 POINT(40.58773196150222 73.7360176773003) bank109074 +109075 POINT(40.35143473553679 73.58335574359907) bank109075 +109076 POINT(41.517058481189345 74.23228147305849) bank109076 +109077 POINT(41.439517484182076 74.59530396567685) bank109077 +109078 POINT(40.27894369601051 73.53593624886724) bank109078 +109079 POINT(39.831060867528016 74.73173656687955) bank109079 +109080 POINT(41.63476921335256 74.81864404289874) bank109080 +109081 POINT(39.91321708137658 74.79364209363571) bank109081 +109082 POINT(41.35196553444827 74.80475757978672) bank109082 +109083 POINT(41.44535701830874 74.50280662540591) bank109083 +109084 POINT(41.69811554846736 74.04437202429743) bank109084 +109085 POINT(41.636968539144604 74.38481686773761) bank109085 +109086 POINT(39.93574953953347 73.34660651413633) bank109086 +109087 POINT(41.56003452833709 73.28629344709353) bank109087 +109088 POINT(41.11350346993782 73.02265178918675) bank109088 +109089 POINT(40.40691912231531 73.53845539273752) bank109089 +109090 POINT(40.75062285944718 74.77988207968868) bank109090 +109091 POINT(40.3602571511144 73.7600865399516) bank109091 +109092 POINT(40.88383889763457 74.15782453003878) bank109092 +109093 POINT(41.16099439291151 74.45474292749341) bank109093 +109094 POINT(40.87876311469947 73.07086479194248) bank109094 +109095 POINT(41.282420483775525 73.43725932677215) bank109095 +109096 POINT(41.325232010845596 74.84290856110084) bank109096 +109097 POINT(40.383272977754245 73.42552761809925) bank109097 +109098 POINT(41.09837554443414 74.90375007376451) bank109098 +109099 POINT(40.39926793971626 74.2564560059346) bank109099 +109100 POINT(39.91150837350508 73.88904945042518) bank109100 +109101 POINT(40.518908006498386 73.49069756974666) bank109101 +109102 POINT(41.63642186171997 73.14680553825987) bank109102 +109103 POINT(41.35014099585661 73.37278155764204) bank109103 +109104 POINT(39.96316354128645 73.47353307975912) bank109104 +109105 POINT(40.12901013562685 73.94092094360461) bank109105 +109106 POINT(41.176373325503086 73.34937045463818) bank109106 +109107 POINT(40.943683301780325 73.86498568046996) bank109107 +109108 POINT(40.16562068760541 73.58579624210682) bank109108 +109109 POINT(40.43942913533077 73.94838734911498) bank109109 +109110 POINT(39.838212975207995 74.36538273635698) bank109110 +109111 POINT(40.50758374434036 74.34829987048788) bank109111 +109112 POINT(40.13171548907003 74.55707344087679) bank109112 +109113 POINT(39.851254033590735 73.5277125772387) bank109113 +109114 POINT(41.55549662253421 73.74031091392978) bank109114 +109115 POINT(41.191216111209876 74.99796622141767) bank109115 +109116 POINT(41.21569173398813 73.81245783805275) bank109116 +109117 POINT(39.86708282724316 74.27009175822343) bank109117 +109118 POINT(40.10076032474297 74.09613166325784) bank109118 +109119 POINT(40.277017497862616 74.18113445542573) bank109119 +109120 POINT(39.716821454438815 73.09374521298083) bank109120 +109121 POINT(40.37695527653461 73.46320639119986) bank109121 +109122 POINT(41.61180851796907 73.97740912085413) bank109122 +109123 POINT(39.9688296189348 74.14671592683659) bank109123 +109124 POINT(40.19113120819586 73.41812590776658) bank109124 +109125 POINT(41.69642517587934 73.07719056384279) bank109125 +109126 POINT(40.59168151054965 73.502608294392) bank109126 +109127 POINT(40.03201531345889 74.46169892546973) bank109127 +109128 POINT(41.68518749485834 73.51401981279962) bank109128 +109129 POINT(39.93378360096342 73.045721357757) bank109129 +109130 POINT(40.43050327726519 74.08871536961584) bank109130 +109131 POINT(40.73782815538624 74.32757626078651) bank109131 +109132 POINT(40.68990867898419 73.73077906177663) bank109132 +109133 POINT(40.97829118080088 73.64230267718449) bank109133 +109134 POINT(40.38845107295002 73.44410914324538) bank109134 +109135 POINT(41.23196860036381 74.7298461254221) bank109135 +109136 POINT(40.524635348816695 73.43987433526505) bank109136 +109137 POINT(40.01097076710944 73.25091324022395) bank109137 +109138 POINT(39.85010123958017 73.3969137919197) bank109138 +109139 POINT(40.84514631248544 74.1679998365624) bank109139 +109140 POINT(39.894489152663894 73.737249822412) bank109140 +109141 POINT(40.33928093634363 74.57784914787135) bank109141 +109142 POINT(39.79560478099479 73.3600327703989) bank109142 +109143 POINT(40.18065716502047 74.79174889653385) bank109143 +109144 POINT(39.98534420142376 74.87583972822482) bank109144 +109145 POINT(40.54470880107104 74.81593907445087) bank109145 +109146 POINT(41.18434478616632 73.72664593147051) bank109146 +109147 POINT(41.47496120750175 73.15938279306954) bank109147 +109148 POINT(40.30549573226618 74.25061066296553) bank109148 +109149 POINT(40.11083421357187 73.16636735312952) bank109149 +109150 POINT(40.56204312851207 73.49434012994163) bank109150 +109151 POINT(39.87630892155836 73.2920670481924) bank109151 +109152 POINT(41.048648865578336 73.5991852228378) bank109152 +109153 POINT(40.872140400388396 73.94749732074779) bank109153 +109154 POINT(41.443754843734304 74.19082146648432) bank109154 +109155 POINT(39.83842335837115 73.23471486294451) bank109155 +109156 POINT(41.190960814487994 73.42280161878955) bank109156 +109157 POINT(40.51325929834614 73.5602958061099) bank109157 +109158 POINT(40.31652907525951 73.65387337333935) bank109158 +109159 POINT(40.75613042455247 73.49304204415287) bank109159 +109160 POINT(40.96526857398927 73.17237727502395) bank109160 +109161 POINT(39.76531295709229 73.28141520491394) bank109161 +109162 POINT(40.863088675958515 74.07744235533772) bank109162 +109163 POINT(41.70247479685553 74.4443735608212) bank109163 +109164 POINT(39.981890804803385 74.9300188142584) bank109164 +109165 POINT(41.33193854573157 73.84507580847344) bank109165 +109166 POINT(40.39805009515078 73.96000321046738) bank109166 +109167 POINT(40.909574392909704 73.47209922185536) bank109167 +109168 POINT(40.3106057204223 73.58091141705623) bank109168 +109169 POINT(40.26704018275314 74.83999636579021) bank109169 +109170 POINT(40.82308597330294 74.19580324594206) bank109170 +109171 POINT(41.23450196791631 73.78467427941308) bank109171 +109172 POINT(41.18623888690615 73.51978001956523) bank109172 +109173 POINT(39.879514691439844 73.72624879372589) bank109173 +109174 POINT(41.11411927903471 73.89443508432548) bank109174 +109175 POINT(39.99524269447203 73.45328599238864) bank109175 +109176 POINT(40.12413134110219 73.82007966866843) bank109176 +109177 POINT(40.16438539513757 73.47100658412042) bank109177 +109178 POINT(41.51716673593529 74.0001515565142) bank109178 +109179 POINT(40.49829914964853 74.7663547575565) bank109179 +109180 POINT(40.50592637781212 74.89214708189478) bank109180 +109181 POINT(40.02450278243585 73.8653340736001) bank109181 +109182 POINT(41.19598843220486 74.6608044829693) bank109182 +109183 POINT(40.554369808161354 74.36032801246174) bank109183 +109184 POINT(40.62393073833693 74.27562155064817) bank109184 +109185 POINT(40.25690778261663 74.73739804404366) bank109185 +109186 POINT(40.041224459061354 74.65055457594083) bank109186 +109187 POINT(40.83882674722765 73.48232433262642) bank109187 +109188 POINT(41.35862682810048 73.25123022315789) bank109188 +109189 POINT(41.513379826249846 73.75947208968864) bank109189 +109190 POINT(39.889191035119886 73.95179570470872) bank109190 +109191 POINT(39.84135892042061 73.63924623270391) bank109191 +109192 POINT(40.46612480515685 74.94320545848772) bank109192 +109193 POINT(41.686856178528856 74.235761429436) bank109193 +109194 POINT(40.55978554140612 74.58038663921576) bank109194 +109195 POINT(41.247559524958675 74.28222694977195) bank109195 +109196 POINT(40.04210689306759 74.36502253857245) bank109196 +109197 POINT(40.69883579965957 74.49724915369998) bank109197 +109198 POINT(41.32680357420731 73.69642390395063) bank109198 +109199 POINT(40.08493383038802 73.14609555088181) bank109199 +109200 POINT(41.03572286628949 73.24640912551415) bank109200 +109201 POINT(41.678513731135425 74.35944538438586) bank109201 +109202 POINT(40.45473379872054 73.65635402023968) bank109202 +109203 POINT(40.98034940288824 74.4046130907743) bank109203 +109204 POINT(41.4593084099063 74.30030685154982) bank109204 +109205 POINT(40.76565535386341 73.31245414152508) bank109205 +109206 POINT(40.82611349225797 74.5920132208522) bank109206 +109207 POINT(40.75586326291929 73.87201407275103) bank109207 +109208 POINT(40.89753607044579 74.20138602540536) bank109208 +109209 POINT(40.99716262419342 74.97897616264889) bank109209 +109210 POINT(39.77372897013466 73.63167625073542) bank109210 +109211 POINT(40.432772170076 73.44872184541241) bank109211 +109212 POINT(40.502813502591756 74.82871343111758) bank109212 +109213 POINT(40.77524294806179 73.85146144889045) bank109213 +109214 POINT(40.974278923395026 73.03146160465106) bank109214 +109215 POINT(40.54524311047011 74.5455479842557) bank109215 +109216 POINT(39.84947549746629 73.47119601411124) bank109216 +109217 POINT(40.983301799905554 73.62806803912976) bank109217 +109218 POINT(41.286051809241584 73.86420027104931) bank109218 +109219 POINT(40.321412552769274 74.33644122663026) bank109219 +109220 POINT(40.996055107854474 73.76029240894576) bank109220 +109221 POINT(40.42199791260931 74.94382937112788) bank109221 +109222 POINT(40.61748481165117 74.1659224790587) bank109222 +109223 POINT(39.99738729069023 73.70028190681712) bank109223 +109224 POINT(39.782009210682595 74.99315965118299) bank109224 +109225 POINT(40.628369160201366 73.59977936328184) bank109225 +109226 POINT(40.316916409098475 74.67178526136107) bank109226 +109227 POINT(39.716003261345364 74.21997261185429) bank109227 +109228 POINT(41.58672700664159 74.96434085633203) bank109228 +109229 POINT(41.007121840544144 73.37051772672851) bank109229 +109230 POINT(41.04953548665071 73.3138411599361) bank109230 +109231 POINT(39.74379194138422 74.9715409529528) bank109231 +109232 POINT(41.123750138547 74.1877677098833) bank109232 +109233 POINT(40.28887919528466 74.54012324750803) bank109233 +109234 POINT(40.183356543902775 73.56882305601722) bank109234 +109235 POINT(40.478039727006944 74.05085471041652) bank109235 +109236 POINT(41.19370741209674 74.52381243109359) bank109236 +109237 POINT(41.19505194205287 73.11433594897295) bank109237 +109238 POINT(41.48320375498402 73.53552543727706) bank109238 +109239 POINT(39.87284603450202 73.9723500192739) bank109239 +109240 POINT(40.39583535924199 73.39572073604084) bank109240 +109241 POINT(40.123946549069046 74.5604491616083) bank109241 +109242 POINT(40.235872758640866 74.28888567219768) bank109242 +109243 POINT(41.10659642125822 73.97307252525074) bank109243 +109244 POINT(39.791961129393094 73.50635376354492) bank109244 +109245 POINT(39.85941805988413 74.62638684862453) bank109245 +109246 POINT(41.1563123216049 74.70694142692327) bank109246 +109247 POINT(41.51615974180866 74.40484821161958) bank109247 +109248 POINT(40.07547409793086 73.81379771239294) bank109248 +109249 POINT(40.345512778616616 74.0184313115236) bank109249 +109250 POINT(39.974404022444745 74.36833821671127) bank109250 +109251 POINT(39.90861972052438 74.91537069778296) bank109251 +109252 POINT(40.66955168455522 73.49592965911276) bank109252 +109253 POINT(40.05997417444819 73.76919909393695) bank109253 +109254 POINT(40.80663191572815 73.9989316295493) bank109254 +109255 POINT(40.61983812536954 73.11386553076389) bank109255 +109256 POINT(40.00126763084021 73.50292232383212) bank109256 +109257 POINT(39.998309978259115 74.8128231681564) bank109257 +109258 POINT(41.034571567454265 74.57004461202376) bank109258 +109259 POINT(41.246696185712835 73.03430620290082) bank109259 +109260 POINT(39.80143752084961 74.02140852939286) bank109260 +109261 POINT(41.60174326845378 73.84293976827766) bank109261 +109262 POINT(41.396184224732224 73.37237901252993) bank109262 +109263 POINT(40.40372629555316 74.95099296157991) bank109263 +109264 POINT(41.08220816973798 74.31371021577635) bank109264 +109265 POINT(41.56964375194313 73.9800518834214) bank109265 +109266 POINT(39.885725890978165 73.8822890259538) bank109266 +109267 POINT(41.54020262084424 73.9401286990142) bank109267 +109268 POINT(40.96899372742558 73.82360480028231) bank109268 +109269 POINT(41.466860341166665 74.39866397973414) bank109269 +109270 POINT(40.55759346617658 74.53863894023333) bank109270 +109271 POINT(40.513461792717116 73.09516050489175) bank109271 +109272 POINT(39.84188484223901 74.42624727159553) bank109272 +109273 POINT(40.21240580725006 73.93771752837523) bank109273 +109274 POINT(40.93097533454826 73.11261517940325) bank109274 +109275 POINT(41.005180207362756 74.6584878861455) bank109275 +109276 POINT(40.34746120838286 74.11965131426206) bank109276 +109277 POINT(40.46593135197253 73.08706485795686) bank109277 +109278 POINT(41.508204405107726 74.27931438197876) bank109278 +109279 POINT(40.378192224378395 73.31897079582666) bank109279 +109280 POINT(41.110458211827726 74.03517376315806) bank109280 +109281 POINT(40.47560685278589 73.28748451223524) bank109281 +109282 POINT(40.38951664726046 74.75259546021721) bank109282 +109283 POINT(40.0573234840049 73.83472771843913) bank109283 +109284 POINT(39.93340006826667 73.72579839699995) bank109284 +109285 POINT(40.949754894406475 74.59809912107868) bank109285 +109286 POINT(40.843109492866986 73.56324437065888) bank109286 +109287 POINT(39.81776688914384 73.39492998135754) bank109287 +109288 POINT(39.9188675876048 73.43698463702074) bank109288 +109289 POINT(39.99319277385741 74.32134161859098) bank109289 +109290 POINT(41.47114060501763 74.79242322248233) bank109290 +109291 POINT(40.1483717416877 74.4961284705315) bank109291 +109292 POINT(40.58323442431074 73.28467422333354) bank109292 +109293 POINT(40.10134599990051 74.37750374001044) bank109293 +109294 POINT(41.252964486495635 73.91878276482088) bank109294 +109295 POINT(40.48564849236332 73.34585723450367) bank109295 +109296 POINT(41.66714955088762 73.66488782030174) bank109296 +109297 POINT(41.270693726230476 73.04678570499429) bank109297 +109298 POINT(41.63430477389372 73.77180545947206) bank109298 +109299 POINT(41.12541680746605 74.73876039003696) bank109299 +109300 POINT(40.62026971005886 73.67933707189937) bank109300 +109301 POINT(40.40400979586563 74.0159455672343) bank109301 +109302 POINT(40.71042851298062 74.27543188019307) bank109302 +109303 POINT(41.10296421201051 73.49432213772768) bank109303 +109304 POINT(40.26014044704826 74.12795988703579) bank109304 +109305 POINT(41.67267239777742 74.05183244632187) bank109305 +109306 POINT(40.06839804680186 73.92819328052089) bank109306 +109307 POINT(40.77359416721251 74.0539790352015) bank109307 +109308 POINT(41.249576591888115 74.96267268512172) bank109308 +109309 POINT(40.977804550902135 74.48142364097673) bank109309 +109310 POINT(39.793219734889576 74.5787563758853) bank109310 +109311 POINT(40.44788383867852 73.95735345209472) bank109311 +109312 POINT(40.82079435046456 73.60351051337938) bank109312 +109313 POINT(40.67701990255524 73.58433992372534) bank109313 +109314 POINT(41.376282696387406 74.61092075835123) bank109314 +109315 POINT(40.579443578396344 74.97061384479503) bank109315 +109316 POINT(40.52482950591424 73.1984283750412) bank109316 +109317 POINT(40.16549940475297 74.46732636804705) bank109317 +109318 POINT(40.58047933621609 73.67452468064668) bank109318 +109319 POINT(40.981354508594265 74.81274567216286) bank109319 +109320 POINT(40.271758617406164 74.3528300546211) bank109320 +109321 POINT(40.26701231633264 73.69877721453014) bank109321 +109322 POINT(41.03302985232694 74.18252864598539) bank109322 +109323 POINT(41.59948048673713 74.16557675214922) bank109323 +109324 POINT(40.59564872133656 73.08883208288516) bank109324 +109325 POINT(41.57701323783512 73.14262967517374) bank109325 +109326 POINT(41.22929808678147 74.57805545689752) bank109326 +109327 POINT(40.647125702136414 74.48102766899171) bank109327 +109328 POINT(40.9906190087663 74.1336399415567) bank109328 +109329 POINT(41.551121088049335 74.92707129799571) bank109329 +109330 POINT(40.52620697390076 74.57857963054246) bank109330 +109331 POINT(40.57854397821804 73.61420108195225) bank109331 +109332 POINT(40.19349115243424 74.77462173842112) bank109332 +109333 POINT(41.62152778886276 73.44356906670656) bank109333 +109334 POINT(41.02808448282923 74.62041717234338) bank109334 +109335 POINT(40.49514323222534 73.55743232552987) bank109335 +109336 POINT(40.938788053181696 73.07554516640847) bank109336 +109337 POINT(40.469807152922826 74.45310846996678) bank109337 +109338 POINT(41.44434817127125 74.79367204503045) bank109338 +109339 POINT(40.54769784018631 73.73837748078921) bank109339 +109340 POINT(41.653285940523496 74.65986981306584) bank109340 +109341 POINT(41.439855130351496 73.87211395997754) bank109341 +109342 POINT(39.949552445081764 73.59163670488361) bank109342 +109343 POINT(40.78630857777886 73.63975400903293) bank109343 +109344 POINT(39.808262842111844 74.32347801340944) bank109344 +109345 POINT(40.87498637630053 73.77071123658965) bank109345 +109346 POINT(40.832039704331066 74.51067343913628) bank109346 +109347 POINT(40.72565784308138 73.74868874571241) bank109347 +109348 POINT(39.741067702679224 73.99937577838847) bank109348 +109349 POINT(39.74888114108294 74.8435325869215) bank109349 +109350 POINT(41.20996850004427 73.09873193920986) bank109350 +109351 POINT(41.24763474677454 73.05629161857873) bank109351 +109352 POINT(39.800943556971184 73.25005220793457) bank109352 +109353 POINT(40.6476512010598 73.73832878342898) bank109353 +109354 POINT(40.45931393688255 74.23177236969586) bank109354 +109355 POINT(40.43059497466411 73.09620604734796) bank109355 +109356 POINT(40.63882032033055 74.1541030603072) bank109356 +109357 POINT(40.353612591368936 74.51708671120336) bank109357 +109358 POINT(40.016691804401766 74.19005175475675) bank109358 +109359 POINT(39.836183419550856 73.71091514871483) bank109359 +109360 POINT(40.94575645009926 74.1424249300098) bank109360 +109361 POINT(40.100907790883 73.61121135050846) bank109361 +109362 POINT(41.373986433655816 74.7095464544867) bank109362 +109363 POINT(41.52385315226068 73.05990550069336) bank109363 +109364 POINT(41.41241523925525 74.31824771814654) bank109364 +109365 POINT(40.829237505643164 74.46570719043227) bank109365 +109366 POINT(40.06679020108952 74.81987596138113) bank109366 +109367 POINT(41.03291515743228 74.6934472237809) bank109367 +109368 POINT(41.5032919963102 74.9631292072428) bank109368 +109369 POINT(39.72199147508618 74.141869688521) bank109369 +109370 POINT(41.22122046387097 73.60713290385753) bank109370 +109371 POINT(41.517207872456645 74.72504363309635) bank109371 +109372 POINT(40.90310560344638 73.8795979429937) bank109372 +109373 POINT(39.86161015986722 73.27130091392527) bank109373 +109374 POINT(40.290622979139584 73.43270185945214) bank109374 +109375 POINT(40.45831647660232 73.40718719678625) bank109375 +109376 POINT(40.000292547821225 74.44266068244758) bank109376 +109377 POINT(41.69300811115238 74.95500574190794) bank109377 +109378 POINT(41.05659343517993 74.08946345654198) bank109378 +109379 POINT(40.850446119697246 74.00032971836694) bank109379 +109380 POINT(41.35482937344013 74.2243463634387) bank109380 +109381 POINT(39.71873818171928 74.81142032135264) bank109381 +109382 POINT(40.30025539621276 74.76652167875676) bank109382 +109383 POINT(41.18595334033451 74.26101609126091) bank109383 +109384 POINT(39.99193228835028 73.04094671259091) bank109384 +109385 POINT(41.69167675119782 74.90607314683035) bank109385 +109386 POINT(41.413968362256895 74.3562660544546) bank109386 +109387 POINT(41.230707174054 74.49486449395657) bank109387 +109388 POINT(41.014231286110935 73.28535854491285) bank109388 +109389 POINT(41.692252634127414 73.63469401586728) bank109389 +109390 POINT(40.136604650463916 74.54913524966473) bank109390 +109391 POINT(41.16218834127958 74.0002156998636) bank109391 +109392 POINT(40.80811749306872 74.7775975240068) bank109392 +109393 POINT(41.53265777362053 74.90168196103039) bank109393 +109394 POINT(40.05344313191194 73.03184582841328) bank109394 +109395 POINT(41.45527761264809 73.54507175519977) bank109395 +109396 POINT(40.34679530157233 73.45256690915754) bank109396 +109397 POINT(41.31253156852772 73.13885620230062) bank109397 +109398 POINT(40.58991878072496 73.75244141160286) bank109398 +109399 POINT(41.00026034274363 74.98169969589043) bank109399 +109400 POINT(41.54412365372678 73.04798183177154) bank109400 +109401 POINT(41.55807121822299 74.54501013648448) bank109401 +109402 POINT(40.765057171977965 73.3709826316941) bank109402 +109403 POINT(39.796205148467614 74.80370520590114) bank109403 +109404 POINT(40.59373757158645 74.78293043437067) bank109404 +109405 POINT(40.135450914838195 73.71603407703205) bank109405 +109406 POINT(39.902743700764596 73.1872954603921) bank109406 +109407 POINT(40.523589176464576 74.06391683544334) bank109407 +109408 POINT(39.78575887040528 73.41839112838119) bank109408 +109409 POINT(39.77981355646379 74.33210680089135) bank109409 +109410 POINT(39.77608404693802 74.16840074077679) bank109410 +109411 POINT(40.138759889938775 74.1862930678885) bank109411 +109412 POINT(41.3543236174135 74.49974619740141) bank109412 +109413 POINT(40.94207742009776 74.63561683241919) bank109413 +109414 POINT(40.8721836627163 73.70795141139985) bank109414 +109415 POINT(41.352960187845746 74.46603741313064) bank109415 +109416 POINT(40.19098881372058 74.95618718737447) bank109416 +109417 POINT(40.45299768262412 73.81340631327478) bank109417 +109418 POINT(40.48127055885644 73.67600937807877) bank109418 +109419 POINT(39.87628973987617 73.64679170011419) bank109419 +109420 POINT(39.83781676416569 73.40393169829338) bank109420 +109421 POINT(40.35924179292492 74.4179642399427) bank109421 +109422 POINT(40.504372102450354 74.58445896337754) bank109422 +109423 POINT(41.60024100896111 74.28566459295018) bank109423 +109424 POINT(41.33822670650265 73.84845490572712) bank109424 +109425 POINT(40.487759862694816 74.98553200352865) bank109425 +109426 POINT(40.465999317353415 74.7048778760797) bank109426 +109427 POINT(39.996201473172704 73.94992436728757) bank109427 +109428 POINT(40.990481871372374 73.21150899206194) bank109428 +109429 POINT(40.46021814480298 73.99521590297343) bank109429 +109430 POINT(41.14433732892101 74.48698677772258) bank109430 +109431 POINT(40.976329328558855 73.67038943519265) bank109431 +109432 POINT(40.49364183798605 73.56053806278335) bank109432 +109433 POINT(41.12365276323155 73.29868162574395) bank109433 +109434 POINT(40.36332227136614 74.74028276186112) bank109434 +109435 POINT(39.81762939607505 73.82159842439076) bank109435 +109436 POINT(41.37819001623791 73.55685844055665) bank109436 +109437 POINT(41.23268840100693 73.95628540082656) bank109437 +109438 POINT(41.502363019703495 73.75876620481841) bank109438 +109439 POINT(40.19791005252355 73.22261174323341) bank109439 +109440 POINT(40.23533799402803 74.97388375380737) bank109440 +109441 POINT(40.64521535017259 74.22660800283943) bank109441 +109442 POINT(40.19728804762462 74.87348797636447) bank109442 +109443 POINT(41.078376719014564 73.58344025688206) bank109443 +109444 POINT(40.10697399079625 74.94392377023378) bank109444 +109445 POINT(41.34299313005361 73.6845774470643) bank109445 +109446 POINT(40.766268401809704 74.96145592558578) bank109446 +109447 POINT(41.33633000284379 73.70665500364821) bank109447 +109448 POINT(41.537604625150976 74.57661743771412) bank109448 +109449 POINT(41.66436519822016 73.34322774055755) bank109449 +109450 POINT(41.20959554465751 74.46949610696124) bank109450 +109451 POINT(40.09384221158769 73.44303632390918) bank109451 +109452 POINT(40.2595447817101 74.25932264122457) bank109452 +109453 POINT(40.531816364891455 74.60796957197952) bank109453 +109454 POINT(41.64555215744441 74.65885708299065) bank109454 +109455 POINT(41.66726554134011 73.6023234629345) bank109455 +109456 POINT(40.724501493195504 74.3967426048045) bank109456 +109457 POINT(41.633911342981904 74.68920990865762) bank109457 +109458 POINT(41.64430426987929 74.24850049557901) bank109458 +109459 POINT(41.686201148509774 73.67175963786846) bank109459 +109460 POINT(40.467592753302625 73.52620472553623) bank109460 +109461 POINT(39.766750491679225 73.32897924653953) bank109461 +109462 POINT(41.54729177902132 74.44353972347453) bank109462 +109463 POINT(41.626822442614106 74.56566615906868) bank109463 +109464 POINT(41.566987162789644 74.45836791430895) bank109464 +109465 POINT(40.50262721712624 74.88641224651698) bank109465 +109466 POINT(41.49715398933087 73.2178797657985) bank109466 +109467 POINT(40.02346399020163 74.21186880506245) bank109467 +109468 POINT(41.341041216991314 73.53946750550993) bank109468 +109469 POINT(40.984893678660065 74.9909475061039) bank109469 +109470 POINT(40.70253235273518 74.2658251159305) bank109470 +109471 POINT(41.5944916320547 74.32369247466612) bank109471 +109472 POINT(39.91060257220283 73.77243085144954) bank109472 +109473 POINT(41.19657118958544 73.64480934705618) bank109473 +109474 POINT(41.59743149184007 74.59632510091375) bank109474 +109475 POINT(40.276708346003396 75.00239792333397) bank109475 +109476 POINT(41.55508558921728 73.94043376130895) bank109476 +109477 POINT(41.60417693924524 74.78518975571085) bank109477 +109478 POINT(40.63389444849391 74.65808017259096) bank109478 +109479 POINT(40.842490920862865 74.35505310411448) bank109479 +109480 POINT(40.59230435633542 73.80531465740174) bank109480 +109481 POINT(39.83549310137028 74.63633263937136) bank109481 +109482 POINT(41.4297948103995 73.6171362648943) bank109482 +109483 POINT(40.62849280738339 74.40456000199478) bank109483 +109484 POINT(40.25190969918146 73.10580267751895) bank109484 +109485 POINT(41.55298772322508 74.32481425704489) bank109485 +109486 POINT(41.49301832777524 73.71607946970109) bank109486 +109487 POINT(40.97106702146861 74.4984111036985) bank109487 +109488 POINT(40.80254187331335 73.38721663102747) bank109488 +109489 POINT(40.738539228655824 73.38263529522507) bank109489 +109490 POINT(41.16728986715617 73.96542788976117) bank109490 +109491 POINT(39.82442272332318 74.11586799611905) bank109491 +109492 POINT(40.14677109874954 73.90352060532311) bank109492 +109493 POINT(40.39449675737103 73.86194015609092) bank109493 +109494 POINT(40.11824953431928 73.21798192019499) bank109494 +109495 POINT(39.791681819926175 74.9314344822251) bank109495 +109496 POINT(41.121023041500735 74.30230653873942) bank109496 +109497 POINT(39.758805192436405 74.6460768447328) bank109497 +109498 POINT(40.67614252726656 73.04792573489807) bank109498 +109499 POINT(41.63767607307299 74.61718232543765) bank109499 +109500 POINT(39.831745297106906 73.68238933644164) bank109500 +109501 POINT(41.508477074131356 73.57280438066122) bank109501 +109502 POINT(40.521307215288566 73.02950392530911) bank109502 +109503 POINT(41.25438756019188 73.85525871852235) bank109503 +109504 POINT(41.34635721539006 74.55299424589892) bank109504 +109505 POINT(40.664149706685095 74.3286964783545) bank109505 +109506 POINT(41.34797264376886 74.4879665008036) bank109506 +109507 POINT(40.47417565189957 73.02804140856276) bank109507 +109508 POINT(41.36871569529143 73.95084625003469) bank109508 +109509 POINT(40.7291039432679 73.8651357478228) bank109509 +109510 POINT(40.95989512276392 74.3091882614971) bank109510 +109511 POINT(40.051202405737165 74.16471387426937) bank109511 +109512 POINT(40.0461390038995 74.18239804628469) bank109512 +109513 POINT(39.978864296179935 74.90305320417603) bank109513 +109514 POINT(40.0173439956257 74.29189041520337) bank109514 +109515 POINT(40.197040424668955 74.94131471442978) bank109515 +109516 POINT(40.226217509009786 73.4024276606268) bank109516 +109517 POINT(39.91271026118658 73.49477880100481) bank109517 +109518 POINT(41.35527565554045 74.46248999739922) bank109518 +109519 POINT(40.7169925536257 73.38804312308305) bank109519 +109520 POINT(40.01034267920705 74.42448465124961) bank109520 +109521 POINT(41.42876825430409 74.58275904895613) bank109521 +109522 POINT(41.54355739042808 74.8465382037605) bank109522 +109523 POINT(40.5304112794913 73.24363210241769) bank109523 +109524 POINT(41.529584247481544 74.46878923950987) bank109524 +109525 POINT(40.85937755030699 73.08898710214702) bank109525 +109526 POINT(40.883988424944704 74.58137744238208) bank109526 +109527 POINT(40.82057019678535 73.55032505509087) bank109527 +109528 POINT(41.37455441999021 73.16267279403361) bank109528 +109529 POINT(39.727136337899886 74.88897803768754) bank109529 +109530 POINT(40.220457387966235 74.51134695574332) bank109530 +109531 POINT(41.19879577922803 74.2393283109147) bank109531 +109532 POINT(40.657345920258614 73.68695636298364) bank109532 +109533 POINT(40.32815942571474 74.8975274345801) bank109533 +109534 POINT(40.94766489952002 73.18693202922019) bank109534 +109535 POINT(41.324265271081664 73.79070052336404) bank109535 +109536 POINT(41.14182140767984 73.86413273518639) bank109536 +109537 POINT(40.12516232685259 73.8359842718552) bank109537 +109538 POINT(41.55248172389524 74.57869193949116) bank109538 +109539 POINT(40.00764952632348 73.19846425013502) bank109539 +109540 POINT(40.11778377753174 74.0155311024969) bank109540 +109541 POINT(41.34131777657982 74.24826394581261) bank109541 +109542 POINT(40.8750462856716 73.90108379001482) bank109542 +109543 POINT(39.931665967205205 73.76491024581955) bank109543 +109544 POINT(41.38780173222171 73.53592271933142) bank109544 +109545 POINT(40.36334812319424 74.49283939517511) bank109545 +109546 POINT(40.445504091791314 73.15678426969777) bank109546 +109547 POINT(39.92510224062842 74.9605271707371) bank109547 +109548 POINT(40.0061192015174 74.71578230665112) bank109548 +109549 POINT(40.54041314104023 74.68917155726677) bank109549 +109550 POINT(40.285800029781036 74.58188783103164) bank109550 +109551 POINT(40.99448768140581 73.8609325466559) bank109551 +109552 POINT(39.80328894881922 73.28333027262698) bank109552 +109553 POINT(39.721647688484836 74.04802023914706) bank109553 +109554 POINT(40.216710028701264 73.57136241865516) bank109554 +109555 POINT(40.57053710568113 74.03011461353952) bank109555 +109556 POINT(40.27760489158846 74.1696253167962) bank109556 +109557 POINT(39.75478311689097 73.12386190896305) bank109557 +109558 POINT(40.55278413710483 73.57346131419209) bank109558 +109559 POINT(39.993453209422974 74.36921514586837) bank109559 +109560 POINT(40.44503801507872 73.46108828359235) bank109560 +109561 POINT(40.9946649296081 73.10372600229084) bank109561 +109562 POINT(40.2316619453954 73.25410511636882) bank109562 +109563 POINT(41.573042102506875 74.55071196003824) bank109563 +109564 POINT(41.26828308142234 74.21820562215856) bank109564 +109565 POINT(41.55942976832407 74.40010071959544) bank109565 +109566 POINT(40.92091439145389 73.94846373006028) bank109566 +109567 POINT(40.29988475247609 74.25336962864229) bank109567 +109568 POINT(40.44904362535866 73.32587489747579) bank109568 +109569 POINT(40.924912899429884 73.99023720138594) bank109569 +109570 POINT(39.90404790863708 74.93482061111119) bank109570 +109571 POINT(40.46556573367814 74.20155189898924) bank109571 +109572 POINT(40.12650706597692 74.29085983676426) bank109572 +109573 POINT(40.79747412818518 73.7310005206449) bank109573 +109574 POINT(39.91953878533138 74.30791763950305) bank109574 +109575 POINT(40.17973470565313 73.52144766282727) bank109575 +109576 POINT(41.227540454904286 73.89416243377786) bank109576 +109577 POINT(39.7419951133192 74.86135444877146) bank109577 +109578 POINT(40.04442918305628 74.31307504168785) bank109578 +109579 POINT(41.40097832832749 74.60584708602342) bank109579 +109580 POINT(41.352978335771716 74.31648997546797) bank109580 +109581 POINT(41.01516831899705 73.62142541349989) bank109581 +109582 POINT(41.42252209740563 73.1751361876222) bank109582 +109583 POINT(40.424546548211474 74.46242986566186) bank109583 +109584 POINT(39.86386024636142 74.66143456906704) bank109584 +109585 POINT(40.000221522667104 74.51915987110378) bank109585 +109586 POINT(41.5103213100347 74.61918986536239) bank109586 +109587 POINT(39.9349730854176 73.68543003086982) bank109587 +109588 POINT(40.94852133971978 73.12385142875553) bank109588 +109589 POINT(41.53141083820562 73.0813108406223) bank109589 +109590 POINT(40.63511734097628 74.14236293956456) bank109590 +109591 POINT(40.48178426758965 73.4718553342144) bank109591 +109592 POINT(40.357876643216855 73.12997598081193) bank109592 +109593 POINT(41.30233777617996 73.42973655970167) bank109593 +109594 POINT(41.110537318159345 74.85985258497556) bank109594 +109595 POINT(41.310877295179 74.83515674983018) bank109595 +109596 POINT(39.951726720721375 74.40217564672781) bank109596 +109597 POINT(40.127283986843494 73.11620294533995) bank109597 +109598 POINT(41.28709596969109 73.99438319416606) bank109598 +109599 POINT(41.532110381203815 74.01145993012761) bank109599 +109600 POINT(41.37797529507947 73.56540729952334) bank109600 +109601 POINT(40.34394773852611 74.09821286078154) bank109601 +109602 POINT(41.406547791594775 74.8518659116462) bank109602 +109603 POINT(39.90603516190877 73.21842092096489) bank109603 +109604 POINT(41.11326264215217 73.71755922476153) bank109604 +109605 POINT(41.68169222313832 73.57713733496078) bank109605 +109606 POINT(40.13658827194794 73.78812446084773) bank109606 +109607 POINT(41.547549018775854 73.53915322115435) bank109607 +109608 POINT(40.386166624362204 73.0701122953295) bank109608 +109609 POINT(41.29499900483066 73.07483982907706) bank109609 +109610 POINT(40.156445325515364 74.768639326451) bank109610 +109611 POINT(41.53850955971189 74.25632987046355) bank109611 +109612 POINT(41.304689392985395 73.76590354426608) bank109612 +109613 POINT(40.9109453713957 74.60571444261032) bank109613 +109614 POINT(40.2857461508486 74.37898190906446) bank109614 +109615 POINT(41.19673126049923 73.86779790251686) bank109615 +109616 POINT(40.56005036981046 73.28643672212453) bank109616 +109617 POINT(41.28683879402819 74.1004569054535) bank109617 +109618 POINT(41.430975287100985 73.4565567396721) bank109618 +109619 POINT(40.9590776340089 74.81383315768268) bank109619 +109620 POINT(41.50944952099918 74.0915051800145) bank109620 +109621 POINT(39.99649118474885 74.41190024985698) bank109621 +109622 POINT(40.78001743644027 73.38663325005942) bank109622 +109623 POINT(40.79599668152935 74.48668040047947) bank109623 +109624 POINT(41.19901568424521 73.58035994588096) bank109624 +109625 POINT(41.68978766841868 74.93373384560559) bank109625 +109626 POINT(41.04019807628561 74.68462249154378) bank109626 +109627 POINT(41.706748043733725 73.99000511657354) bank109627 +109628 POINT(41.361979255884144 74.59925387388098) bank109628 +109629 POINT(40.175306175807606 73.98721624917569) bank109629 +109630 POINT(40.805590540697764 73.23572966652749) bank109630 +109631 POINT(40.94561807855596 73.35096833729914) bank109631 +109632 POINT(41.60474372417241 74.88593502028012) bank109632 +109633 POINT(41.19407774336993 74.99487824749086) bank109633 +109634 POINT(39.912333942247 74.909617671303) bank109634 +109635 POINT(41.231468416441594 73.33720430335013) bank109635 +109636 POINT(41.13414350293437 74.10239575708091) bank109636 +109637 POINT(41.01229247680498 74.96515466348589) bank109637 +109638 POINT(40.02434939866888 73.64237604841465) bank109638 +109639 POINT(41.098260226357986 74.55480292476621) bank109639 +109640 POINT(41.165339353451216 73.21785355546135) bank109640 +109641 POINT(39.96187844880917 74.44706925889591) bank109641 +109642 POINT(40.54263433043928 73.21060949745585) bank109642 +109643 POINT(40.51084944644036 74.18794846822276) bank109643 +109644 POINT(40.43474865037829 74.18825873558464) bank109644 +109645 POINT(41.23410760142524 73.2109457325818) bank109645 +109646 POINT(40.928237467971385 73.51037906081635) bank109646 +109647 POINT(41.62951800129977 75.00385184507138) bank109647 +109648 POINT(41.068872419319376 74.00490711361503) bank109648 +109649 POINT(41.5021388941339 74.65846523277135) bank109649 +109650 POINT(41.46350755107057 73.27047135406542) bank109650 +109651 POINT(40.60773736153182 73.38171424273727) bank109651 +109652 POINT(41.461257048747136 74.79474353990643) bank109652 +109653 POINT(41.33164985534795 74.26353400036966) bank109653 +109654 POINT(39.80668700584427 73.6946774849661) bank109654 +109655 POINT(40.47129395543665 74.80196435233951) bank109655 +109656 POINT(41.244514764966056 74.0330261164253) bank109656 +109657 POINT(41.389398498477554 73.88715218696626) bank109657 +109658 POINT(40.43686779944026 73.07029928119996) bank109658 +109659 POINT(40.17759779202027 74.91773791442367) bank109659 +109660 POINT(39.97619190202977 73.13963491352239) bank109660 +109661 POINT(40.771429189626886 73.99367695748714) bank109661 +109662 POINT(39.905019759778 74.1096307693927) bank109662 +109663 POINT(39.9212317709367 73.18187451580897) bank109663 +109664 POINT(41.043622389970515 73.143684980978) bank109664 +109665 POINT(41.693404192989846 73.86200516740271) bank109665 +109666 POINT(40.59318037267192 74.76401637279635) bank109666 +109667 POINT(39.723624411022165 73.64164800821533) bank109667 +109668 POINT(40.397778010359005 73.85598352545738) bank109668 +109669 POINT(40.00823443885842 74.29515473402597) bank109669 +109670 POINT(40.66328605858169 73.43256685736766) bank109670 +109671 POINT(40.3430393109226 73.1924306425167) bank109671 +109672 POINT(39.788142601857835 74.09129612009784) bank109672 +109673 POINT(39.719804911353144 74.29314408697405) bank109673 +109674 POINT(40.475159582204604 74.52170405836428) bank109674 +109675 POINT(40.141267838244396 74.33190799375035) bank109675 +109676 POINT(40.76852957096603 74.0073774139444) bank109676 +109677 POINT(40.19083790529408 73.39098189019967) bank109677 +109678 POINT(40.24720605564316 73.01148821371918) bank109678 +109679 POINT(40.594400734277464 73.15465101572667) bank109679 +109680 POINT(40.0127089069319 73.3257121129682) bank109680 +109681 POINT(41.54111036684961 73.71090557968938) bank109681 +109682 POINT(40.24439510293187 73.34092586719325) bank109682 +109683 POINT(40.748944967319886 73.92525496415345) bank109683 +109684 POINT(40.23509055293393 73.68955363432596) bank109684 +109685 POINT(40.43120521715758 74.45726261859396) bank109685 +109686 POINT(39.841863009234096 74.69416635285168) bank109686 +109687 POINT(39.82020069076477 73.14253156197982) bank109687 +109688 POINT(41.4711748887917 73.775671579307) bank109688 +109689 POINT(41.67061325813725 74.50980790570047) bank109689 +109690 POINT(41.225819295394125 74.0517081535638) bank109690 +109691 POINT(40.51548684297811 73.50366067729315) bank109691 +109692 POINT(41.478884616307674 74.74294736215846) bank109692 +109693 POINT(41.45392738814235 73.91210637523298) bank109693 +109694 POINT(41.66641384883843 74.17906687384709) bank109694 +109695 POINT(41.61072442978614 74.93536124447382) bank109695 +109696 POINT(40.82479855024935 73.7046077838735) bank109696 +109697 POINT(41.5899039998024 74.89459367705976) bank109697 +109698 POINT(41.07723840027627 73.01604971473441) bank109698 +109699 POINT(41.39127888068024 73.85255150196771) bank109699 +109700 POINT(40.941410098310996 74.07361071692266) bank109700 +109701 POINT(41.59441940969024 73.47543616430889) bank109701 +109702 POINT(40.780904333728586 73.09962679057064) bank109702 +109703 POINT(39.99800445476773 74.32619065448014) bank109703 +109704 POINT(41.66354362536273 73.079442242178) bank109704 +109705 POINT(41.186033304251715 73.7639588115089) bank109705 +109706 POINT(40.536717301863334 73.18686020290156) bank109706 +109707 POINT(41.58039273046815 74.68393979633464) bank109707 +109708 POINT(40.844029206285015 73.30457622918576) bank109708 +109709 POINT(41.69035790664565 73.60586807612741) bank109709 +109710 POINT(39.88533131341097 73.96110049744598) bank109710 +109711 POINT(40.31573780676273 74.7291501646392) bank109711 +109712 POINT(40.949213425773095 74.81061469954436) bank109712 +109713 POINT(41.61681216708523 74.18286383126188) bank109713 +109714 POINT(40.06823329621535 73.7699753723501) bank109714 +109715 POINT(41.028188921100956 74.86797738245448) bank109715 +109716 POINT(40.92819610041797 74.06027012405828) bank109716 +109717 POINT(40.50579178763654 74.63316080554027) bank109717 +109718 POINT(40.488402662278205 73.80796042778049) bank109718 +109719 POINT(40.80120239142858 73.75025561063507) bank109719 +109720 POINT(40.43230223246692 73.14200244122569) bank109720 +109721 POINT(41.61071399493485 73.42874637843065) bank109721 +109722 POINT(41.382197481024775 73.44172776323774) bank109722 +109723 POINT(40.96086210881619 73.56641192932916) bank109723 +109724 POINT(40.25719729112244 73.25012901621564) bank109724 +109725 POINT(40.883081671641364 74.07466241027888) bank109725 +109726 POINT(40.60653602657782 74.09343040282748) bank109726 +109727 POINT(41.67147344837246 73.55353999965533) bank109727 +109728 POINT(39.84427066380888 74.58581240039422) bank109728 +109729 POINT(39.82951176371639 73.57580111828318) bank109729 +109730 POINT(41.541749187932005 74.66743269389991) bank109730 +109731 POINT(41.57883978694653 73.70644660899133) bank109731 +109732 POINT(40.82567472041768 74.91095141421992) bank109732 +109733 POINT(40.228246200139445 73.5338013220708) bank109733 +109734 POINT(40.350687208931404 74.68330711157255) bank109734 +109735 POINT(40.01458794181226 74.69508449284059) bank109735 +109736 POINT(40.59589098912793 74.26792560264084) bank109736 +109737 POINT(40.25203350462041 73.05443381111282) bank109737 +109738 POINT(41.11575243955907 74.37115560885206) bank109738 +109739 POINT(41.52698778425828 74.06560082668753) bank109739 +109740 POINT(41.35227664166899 74.56701771226734) bank109740 +109741 POINT(40.77220995934123 73.74493230539078) bank109741 +109742 POINT(41.62692583922132 74.74535961725434) bank109742 +109743 POINT(40.21055128165362 74.1409229549652) bank109743 +109744 POINT(40.34458418823664 73.84462242337867) bank109744 +109745 POINT(40.45582085650271 74.94536737232612) bank109745 +109746 POINT(40.99243637288788 74.67039211008398) bank109746 +109747 POINT(41.56572703584742 74.32455262207368) bank109747 +109748 POINT(39.989509090194666 73.9503092612865) bank109748 +109749 POINT(39.88807857146142 74.9384971627106) bank109749 +109750 POINT(40.68753767979857 74.68904060245788) bank109750 +109751 POINT(40.266063873358505 74.85819890887176) bank109751 +109752 POINT(40.216843570348736 73.90356635003373) bank109752 +109753 POINT(39.91148178360124 73.01284676067937) bank109753 +109754 POINT(41.47425469475044 74.86297053718972) bank109754 +109755 POINT(40.701588947956374 73.41482939313856) bank109755 +109756 POINT(41.26354041457173 74.1044127799021) bank109756 +109757 POINT(40.63103106334465 73.59040739689596) bank109757 +109758 POINT(41.48361655214834 74.93140911003206) bank109758 +109759 POINT(41.69494936783716 74.72092461387464) bank109759 +109760 POINT(40.50972416439196 74.68400894396339) bank109760 +109761 POINT(40.88955085359419 74.68918117649838) bank109761 +109762 POINT(40.89232525326669 74.81132836700958) bank109762 +109763 POINT(41.398907366211745 74.34987976680037) bank109763 +109764 POINT(39.85147337034047 73.6148619763552) bank109764 +109765 POINT(41.47410760018497 74.17812474427103) bank109765 +109766 POINT(41.245863944662375 74.22975810088725) bank109766 +109767 POINT(41.30305868131374 74.32766389261843) bank109767 +109768 POINT(41.00622145984802 74.55887172751176) bank109768 +109769 POINT(39.74352406421966 74.43335488288906) bank109769 +109770 POINT(39.97209705817106 73.28573735828493) bank109770 +109771 POINT(41.37027422634584 73.04333426303417) bank109771 +109772 POINT(41.222711770825164 74.33021060853177) bank109772 +109773 POINT(41.294718103956015 74.53174456709853) bank109773 +109774 POINT(39.836493062812956 74.48948253131628) bank109774 +109775 POINT(40.1514492404778 73.35981064713827) bank109775 +109776 POINT(41.099641004129055 74.98520234280738) bank109776 +109777 POINT(40.27942288359233 73.70926978817626) bank109777 +109778 POINT(40.396334261377554 73.96365432654827) bank109778 +109779 POINT(40.62918942410362 73.78221818622453) bank109779 +109780 POINT(41.21165568296983 73.78969240586207) bank109780 +109781 POINT(41.10345203366533 73.52341602125577) bank109781 +109782 POINT(41.072110124798876 73.90283865585613) bank109782 +109783 POINT(39.76059583521081 74.79115206125033) bank109783 +109784 POINT(40.195747559266906 74.96435116777101) bank109784 +109785 POINT(41.44817353133136 73.58587024083887) bank109785 +109786 POINT(41.02729303526471 73.18122439725309) bank109786 +109787 POINT(41.679765653266145 74.63127842772832) bank109787 +109788 POINT(41.61685010636579 74.1496026281446) bank109788 +109789 POINT(40.6711660283232 74.40607674051847) bank109789 +109790 POINT(40.50487030431139 74.00402942865462) bank109790 +109791 POINT(39.89914501075434 74.87366474890226) bank109791 +109792 POINT(41.114397621770856 73.95850064158462) bank109792 +109793 POINT(41.068860931104155 73.25070160254394) bank109793 +109794 POINT(40.937449083313176 74.05202212452475) bank109794 +109795 POINT(40.98315299934448 74.10050549746573) bank109795 +109796 POINT(41.19101527905948 73.61441450923037) bank109796 +109797 POINT(40.13044304005649 73.84257703207389) bank109797 +109798 POINT(39.960132124515155 73.6303293259643) bank109798 +109799 POINT(41.02713751448384 73.37782634241874) bank109799 +109800 POINT(40.840368460302635 73.82604151485603) bank109800 +109801 POINT(40.39369034653408 73.23216974359785) bank109801 +109802 POINT(40.485830504378384 73.5827291584217) bank109802 +109803 POINT(40.671796673665256 74.1424066833006) bank109803 +109804 POINT(40.34001427460273 73.04408113329004) bank109804 +109805 POINT(40.92312416800713 73.04204244527412) bank109805 +109806 POINT(41.20361620772612 74.88906202979956) bank109806 +109807 POINT(40.695785252836686 73.8836038317568) bank109807 +109808 POINT(40.9023331875611 74.55291224790416) bank109808 +109809 POINT(40.23482471452468 74.90040349170388) bank109809 +109810 POINT(39.955003283683176 74.91281275340846) bank109810 +109811 POINT(40.5501034071393 74.1885793059942) bank109811 +109812 POINT(39.93729902986388 73.49782613517107) bank109812 +109813 POINT(40.13605928924013 73.18007123688616) bank109813 +109814 POINT(40.33035693732522 74.68426261951838) bank109814 +109815 POINT(40.29622917315377 74.3820589272144) bank109815 +109816 POINT(39.8160972281285 73.66973244980036) bank109816 +109817 POINT(41.38028868487721 74.76200951278027) bank109817 +109818 POINT(41.41841603000452 73.23946340452494) bank109818 +109819 POINT(40.70395078053136 74.17797183884882) bank109819 +109820 POINT(40.87840863048581 73.42005347995499) bank109820 +109821 POINT(40.06567812238425 74.4269945544465) bank109821 +109822 POINT(40.602718708514765 74.59342293877653) bank109822 +109823 POINT(40.183771920254934 74.08787196729493) bank109823 +109824 POINT(41.583354773963755 73.48015306677488) bank109824 +109825 POINT(40.45833533622952 74.89579352156007) bank109825 +109826 POINT(40.71084383792658 73.61000922494354) bank109826 +109827 POINT(40.17668436343205 73.46757507635643) bank109827 +109828 POINT(40.10505545001986 73.55018107911343) bank109828 +109829 POINT(41.36078306800999 73.84666250393005) bank109829 +109830 POINT(41.59485872291325 74.61204510988436) bank109830 +109831 POINT(40.78997751411285 74.85562977892322) bank109831 +109832 POINT(40.717352185635896 73.01416695727886) bank109832 +109833 POINT(41.302045114634154 74.21113498585544) bank109833 +109834 POINT(39.7680731527959 73.42061684092043) bank109834 +109835 POINT(41.525292700999316 73.4059526896701) bank109835 +109836 POINT(39.96433709618152 73.15528620596797) bank109836 +109837 POINT(41.71016510774665 73.36714316447625) bank109837 +109838 POINT(40.34008859131914 73.44629696102757) bank109838 +109839 POINT(39.995632877908584 74.21366823105348) bank109839 +109840 POINT(40.77156224079141 74.07690392836062) bank109840 +109841 POINT(40.03017629988781 73.20175376839522) bank109841 +109842 POINT(41.61554892739406 73.97384897344044) bank109842 +109843 POINT(41.27828215703774 73.3539021429944) bank109843 +109844 POINT(39.87136172617606 74.41175847657586) bank109844 +109845 POINT(41.471793638225236 74.16561345460876) bank109845 +109846 POINT(41.43269492445473 73.23872597886253) bank109846 +109847 POINT(41.05507870754636 74.53716536434891) bank109847 +109848 POINT(41.38778502244145 73.26943939849927) bank109848 +109849 POINT(41.08587027001082 73.139540586787) bank109849 +109850 POINT(40.831917618550584 73.83734870062756) bank109850 +109851 POINT(41.579173101145976 74.98715096145854) bank109851 +109852 POINT(40.20947699274001 73.30684910157943) bank109852 +109853 POINT(40.48154486465814 74.28216922770177) bank109853 +109854 POINT(41.33517965642471 74.18167501708305) bank109854 +109855 POINT(40.70223012130746 73.95617192658723) bank109855 +109856 POINT(40.35036697210124 73.05133734722055) bank109856 +109857 POINT(41.07239235887482 73.49437643818104) bank109857 +109858 POINT(40.96997152621842 73.21940624397547) bank109858 +109859 POINT(41.42225873104348 74.07579420163252) bank109859 +109860 POINT(41.30908454272062 73.91624283951731) bank109860 +109861 POINT(41.55368933846986 73.85944546093295) bank109861 +109862 POINT(41.15376852110971 73.10786240450973) bank109862 +109863 POINT(41.54177264306059 73.01728309401923) bank109863 +109864 POINT(40.67967329551358 74.58537917118937) bank109864 +109865 POINT(39.79468203289877 73.69903558331575) bank109865 +109866 POINT(39.983363847627906 74.41460206578341) bank109866 +109867 POINT(39.825955057073855 73.03403092935606) bank109867 +109868 POINT(39.97745358429075 74.40876152559194) bank109868 +109869 POINT(40.99260644589289 73.11516821095755) bank109869 +109870 POINT(40.32104083704037 74.33949603091924) bank109870 +109871 POINT(39.78184567300883 73.50100393495609) bank109871 +109872 POINT(40.3151394268907 74.48477008213901) bank109872 +109873 POINT(41.14802569473963 73.98699772792988) bank109873 +109874 POINT(39.89172119434979 73.34148412962394) bank109874 +109875 POINT(41.13126309080674 73.75256203358317) bank109875 +109876 POINT(41.213414145800485 74.70026197333978) bank109876 +109877 POINT(40.063183660324256 74.88713279758738) bank109877 +109878 POINT(40.20958072631692 73.19117635801096) bank109878 +109879 POINT(40.21278990312639 73.70371922097371) bank109879 +109880 POINT(40.81783156723969 73.60490975791453) bank109880 +109881 POINT(40.651787393348734 74.26093294587248) bank109881 +109882 POINT(40.08521579125047 74.91210533515279) bank109882 +109883 POINT(41.174916583808525 74.20410689448724) bank109883 +109884 POINT(41.63651545410432 73.89303452165296) bank109884 +109885 POINT(40.19900253071248 74.85415544262102) bank109885 +109886 POINT(40.467196022214495 74.54589131013527) bank109886 +109887 POINT(39.809097411487464 74.22277462864034) bank109887 +109888 POINT(40.59492260049363 73.90018130693593) bank109888 +109889 POINT(41.259708711246304 73.06746512607603) bank109889 +109890 POINT(40.679769945915695 73.02723325071942) bank109890 +109891 POINT(40.59054867216723 73.58341663795116) bank109891 +109892 POINT(41.58133035302573 73.70460575872127) bank109892 +109893 POINT(41.19136127209752 74.01689831324552) bank109893 +109894 POINT(40.621606637544495 74.58363943063046) bank109894 +109895 POINT(41.1940494120231 74.42418715880324) bank109895 +109896 POINT(41.296346652802505 73.0269192761825) bank109896 +109897 POINT(41.13431833223259 74.7367446427007) bank109897 +109898 POINT(41.34430992417256 74.37645939176187) bank109898 +109899 POINT(40.532101123308685 73.17572000679965) bank109899 +109900 POINT(40.7982583592739 74.82096450634016) bank109900 +109901 POINT(39.737354686927695 73.76018098347005) bank109901 +109902 POINT(41.28302865235399 74.26230544986885) bank109902 +109903 POINT(40.69578832190433 74.84156432980306) bank109903 +109904 POINT(41.414568850782054 74.03330287400618) bank109904 +109905 POINT(40.60178400754109 73.63007619303286) bank109905 +109906 POINT(40.38984930813377 74.40160790894376) bank109906 +109907 POINT(40.82264683794875 73.72489289941713) bank109907 +109908 POINT(41.65521978695774 74.40941490894417) bank109908 +109909 POINT(41.702859364386086 73.16051702411507) bank109909 +109910 POINT(40.38188332288968 73.60026248336234) bank109910 +109911 POINT(40.944882818537025 74.56980433493769) bank109911 +109912 POINT(40.604032654917354 74.44189484403218) bank109912 +109913 POINT(40.09966357230898 74.91795758096468) bank109913 +109914 POINT(41.14415576373345 73.36099130152562) bank109914 +109915 POINT(41.691022562239496 73.58967907531056) bank109915 +109916 POINT(39.91655964754686 74.38726208427498) bank109916 +109917 POINT(40.3778290157872 74.84800471438484) bank109917 +109918 POINT(40.98851308307301 74.40164142629581) bank109918 +109919 POINT(41.533317175477634 74.11422982607938) bank109919 +109920 POINT(39.807508198336414 74.20475478742249) bank109920 +109921 POINT(39.75282267760231 74.97660003384473) bank109921 +109922 POINT(41.06438514786323 74.24012181827375) bank109922 +109923 POINT(41.50391409228085 73.06532147209927) bank109923 +109924 POINT(39.94097876470126 74.62420407571815) bank109924 +109925 POINT(40.908183444117824 74.93184261846575) bank109925 +109926 POINT(40.43101760498111 74.676593105692) bank109926 +109927 POINT(39.844320644199364 73.42115733058156) bank109927 +109928 POINT(39.898764796895605 74.07165491591472) bank109928 +109929 POINT(40.43345932681317 74.94828769820514) bank109929 +109930 POINT(40.47888533660217 74.35983179514429) bank109930 +109931 POINT(41.59418570380615 74.12924631362127) bank109931 +109932 POINT(40.400285106080354 74.49730456838049) bank109932 +109933 POINT(41.24394400067125 74.69643123302464) bank109933 +109934 POINT(40.30423249822136 74.8647766963177) bank109934 +109935 POINT(41.5064577884322 73.89950323945003) bank109935 +109936 POINT(41.51208937666381 74.61996851095624) bank109936 +109937 POINT(39.872485470844936 74.19533315285555) bank109937 +109938 POINT(40.62127820210611 74.22625941275771) bank109938 +109939 POINT(41.47640232829989 74.71388192495593) bank109939 +109940 POINT(39.921425992796486 73.37279995254947) bank109940 +109941 POINT(40.658447583210275 73.72415613245198) bank109941 +109942 POINT(41.377308897011055 73.92454919699053) bank109942 +109943 POINT(40.4189439854404 74.97222492704755) bank109943 +109944 POINT(40.781557316276896 73.09759639938041) bank109944 +109945 POINT(40.37606760010758 74.85897229717446) bank109945 +109946 POINT(41.585352378849045 73.61582925497939) bank109946 +109947 POINT(40.344813922448274 73.9763774781636) bank109947 +109948 POINT(39.98188491143149 73.04022563970011) bank109948 +109949 POINT(40.89955824451479 73.46168036651815) bank109949 +109950 POINT(40.11666658167428 73.5405686220126) bank109950 +109951 POINT(41.277094528674596 74.69783934319267) bank109951 +109952 POINT(41.1320159989114 74.48783806187964) bank109952 +109953 POINT(41.670890028435714 74.50343146114098) bank109953 +109954 POINT(40.26097826022258 74.1338760355389) bank109954 +109955 POINT(41.604128706045906 74.89961978271734) bank109955 +109956 POINT(40.16012427128201 74.4422713398745) bank109956 +109957 POINT(39.96652310061053 74.85431745913435) bank109957 +109958 POINT(39.853487683002456 73.79279161221297) bank109958 +109959 POINT(41.5702476329865 74.45586710446955) bank109959 +109960 POINT(40.50022641945819 74.88223195147953) bank109960 +109961 POINT(40.87562027373429 73.73944575064542) bank109961 +109962 POINT(40.24412265201677 73.81391416103523) bank109962 +109963 POINT(40.978598522141944 74.8730877281648) bank109963 +109964 POINT(39.80722385284312 74.09294482549483) bank109964 +109965 POINT(40.19355907859537 74.23139280362035) bank109965 +109966 POINT(41.09592993257691 74.73878395735473) bank109966 +109967 POINT(40.21893148584311 74.6997303975982) bank109967 +109968 POINT(40.82518795030222 74.40884728600723) bank109968 +109969 POINT(41.03415653622014 74.9130170550084) bank109969 +109970 POINT(39.97265540080745 73.81104443515439) bank109970 +109971 POINT(39.815320966332344 74.76279260126935) bank109971 +109972 POINT(41.21586580143148 73.69744425592407) bank109972 +109973 POINT(40.880833289592594 74.65947937793705) bank109973 +109974 POINT(40.18802574010046 74.54853198698524) bank109974 +109975 POINT(40.704887475814594 73.2522928215557) bank109975 +109976 POINT(40.74762206743769 74.13459537382853) bank109976 +109977 POINT(41.29867609885109 74.43672258813737) bank109977 +109978 POINT(40.91254984757388 74.14571237405494) bank109978 +109979 POINT(41.041493535564825 73.9921765509651) bank109979 +109980 POINT(40.12161187083303 74.70947006015021) bank109980 +109981 POINT(40.49257902556643 74.6481645867361) bank109981 +109982 POINT(41.25634417379694 74.96835117766555) bank109982 +109983 POINT(40.2401768788819 73.35361707321728) bank109983 +109984 POINT(40.585097186879906 74.84640546859418) bank109984 +109985 POINT(40.13421048181866 73.87036828178742) bank109985 +109986 POINT(39.929745111432496 74.41456575358933) bank109986 +109987 POINT(40.71456773297841 73.3543623101305) bank109987 +109988 POINT(40.274335450570774 74.99699640015031) bank109988 +109989 POINT(41.701742757787116 73.72314624289353) bank109989 +109990 POINT(39.725972125087885 73.95634903564267) bank109990 +109991 POINT(39.734337227025286 73.56610805993476) bank109991 +109992 POINT(41.26275519435711 74.2524148927969) bank109992 +109993 POINT(40.31646185367553 73.74505982498582) bank109993 +109994 POINT(40.99049691784402 74.7430781814321) bank109994 +109995 POINT(41.46220406605095 73.04856819949603) bank109995 +109996 POINT(40.072414135190215 74.0743909421966) bank109996 +109997 POINT(39.84163867710507 74.67557389047037) bank109997 +109998 POINT(41.040919266609556 73.24396496022926) bank109998 +109999 POINT(41.07856048405539 74.02955908524883) bank109999 +110000 POINT(40.720707008485626 73.04866550698331) bank110000 +110001 POINT(39.92233542906156 74.55323154598412) bank110001 +110002 POINT(40.052740337255045 74.79480433228431) bank110002 +110003 POINT(39.98242501023795 74.40762824712287) bank110003 +110004 POINT(41.130516298078156 74.17257888460229) bank110004 +110005 POINT(40.104732608931734 73.69103953606846) bank110005 +110006 POINT(39.97128864840586 73.38732817583328) bank110006 +110007 POINT(41.59632436310829 74.35088756254405) bank110007 +110008 POINT(41.575615691793416 73.99166439940865) bank110008 +110009 POINT(41.497922152841575 73.0911334108572) bank110009 +110010 POINT(40.449712693547774 73.16192090895592) bank110010 +110011 POINT(41.46975528936783 74.66472294701525) bank110011 +110012 POINT(40.9685901118646 74.28201966636793) bank110012 +110013 POINT(41.55062475559438 74.45002076814296) bank110013 +110014 POINT(40.82212901693729 74.00254781721742) bank110014 +110015 POINT(40.63948481509331 73.90395896923954) bank110015 +110016 POINT(40.72488888779953 74.83231223217713) bank110016 +110017 POINT(39.81440446081189 74.70277157555094) bank110017 +110018 POINT(41.374427123364875 74.77917818761445) bank110018 +110019 POINT(40.725273794005425 74.83761786864515) bank110019 +110020 POINT(40.66211798361012 74.62106189745124) bank110020 +110021 POINT(41.26724438742507 74.84525749267438) bank110021 +110022 POINT(40.20703307287172 73.38320230459802) bank110022 +110023 POINT(40.10488912178618 73.59860223956754) bank110023 +110024 POINT(40.398877691030506 74.60992376046683) bank110024 +110025 POINT(41.34947081790508 73.29064865502919) bank110025 +110026 POINT(41.42708759114214 73.3017491757819) bank110026 +110027 POINT(40.408880598989114 73.26832311069995) bank110027 +110028 POINT(40.18249782236871 74.55238759002525) bank110028 +110029 POINT(41.43911664872543 74.04107546498811) bank110029 +110030 POINT(40.28638205073245 74.11776801219047) bank110030 +110031 POINT(41.331215504568384 74.17886513850904) bank110031 +110032 POINT(41.430149110357405 73.28547080035538) bank110032 +110033 POINT(40.89302503116 73.72623903500053) bank110033 +110034 POINT(40.843222790685985 74.24436895164268) bank110034 +110035 POINT(40.57564925926661 74.55453877469579) bank110035 +110036 POINT(41.15224111302798 74.45210751277014) bank110036 +110037 POINT(41.309632717217305 73.97109090259292) bank110037 +110038 POINT(41.23782131921735 74.91335519148835) bank110038 +110039 POINT(40.10841602432447 73.22912249186544) bank110039 +110040 POINT(41.672890162730326 73.99433139869181) bank110040 +110041 POINT(41.25620692706462 74.02136061001157) bank110041 +110042 POINT(40.292008139875755 74.74837573796938) bank110042 +110043 POINT(40.46604581078414 73.75111626919788) bank110043 +110044 POINT(40.58554632020702 73.04461562569969) bank110044 +110045 POINT(40.115808162416066 73.0989264316539) bank110045 +110046 POINT(39.89722595662914 73.55126002107878) bank110046 +110047 POINT(40.94727889193216 73.15598802046625) bank110047 +110048 POINT(40.10283490409644 73.846968484622) bank110048 +110049 POINT(40.89602384013064 73.99261347818907) bank110049 +110050 POINT(40.204390972950144 74.32123523161468) bank110050 +110051 POINT(41.43927629031272 73.40450844307047) bank110051 +110052 POINT(41.35285871510879 74.18827685754654) bank110052 +110053 POINT(41.564853496338415 73.27511249259541) bank110053 +110054 POINT(40.883102700109085 73.98621588623467) bank110054 +110055 POINT(40.87557968054426 74.22960846631504) bank110055 +110056 POINT(40.584426738400225 73.30188807245706) bank110056 +110057 POINT(41.323825599458914 73.07596617984335) bank110057 +110058 POINT(40.06119029659344 73.05568347985154) bank110058 +110059 POINT(40.337877869305906 74.22737622704929) bank110059 +110060 POINT(40.335662780298776 73.24100636496767) bank110060 +110061 POINT(40.932906128488206 73.6350458994724) bank110061 +110062 POINT(40.142499231855524 74.52377593074937) bank110062 +110063 POINT(40.8443888859013 74.87145003275884) bank110063 +110064 POINT(40.56743497694451 73.30093639629224) bank110064 +110065 POINT(40.53421983297111 74.84692054276915) bank110065 +110066 POINT(39.73256769068303 74.07136268556772) bank110066 +110067 POINT(41.00885710910665 74.97348930211272) bank110067 +110068 POINT(40.00082986558757 73.46578158518687) bank110068 +110069 POINT(41.30739004037473 73.99205321700782) bank110069 +110070 POINT(39.8663842586632 74.312694180022) bank110070 +110071 POINT(40.29068664537615 73.73765737815441) bank110071 +110072 POINT(41.29117213240191 74.09425631799685) bank110072 +110073 POINT(41.09217096323814 73.20972354635111) bank110073 +110074 POINT(41.245948473185074 73.75022387714124) bank110074 +110075 POINT(40.100907609012715 74.88409162184676) bank110075 +110076 POINT(40.9105556139212 73.59530734751617) bank110076 +110077 POINT(40.18821090460712 73.91918753703638) bank110077 +110078 POINT(40.25239336302851 73.58441375047947) bank110078 +110079 POINT(39.88855944878046 73.16591791014989) bank110079 +110080 POINT(41.02522003668995 73.29645115878466) bank110080 +110081 POINT(40.23048425376689 74.75283597364569) bank110081 +110082 POINT(41.45275679706356 73.59792778038604) bank110082 +110083 POINT(40.74187537217161 74.38417872010325) bank110083 +110084 POINT(40.37174470417157 73.13198177269983) bank110084 +110085 POINT(40.349123167744494 74.6013721709898) bank110085 +110086 POINT(41.68143335445864 74.9448114026997) bank110086 +110087 POINT(41.69623941516527 74.53004232569278) bank110087 +110088 POINT(40.19281791616659 74.10156180045789) bank110088 +110089 POINT(41.52129795100851 74.13168959175839) bank110089 +110090 POINT(41.40488134541096 74.33335358151113) bank110090 +110091 POINT(40.64052921569577 74.32549144415155) bank110091 +110092 POINT(40.641454424772945 74.68431583912388) bank110092 +110093 POINT(41.59561555386123 74.97642541658024) bank110093 +110094 POINT(40.211687754839566 74.17052427095855) bank110094 +110095 POINT(41.28173443943493 73.94930980330004) bank110095 +110096 POINT(39.73554004566881 73.95087688775948) bank110096 +110097 POINT(40.5455858662076 74.00797778875065) bank110097 +110098 POINT(40.411941778599164 74.39889223228451) bank110098 +110099 POINT(41.435889857244 73.58508509313826) bank110099 +110100 POINT(40.166554817173235 74.50918921590545) bank110100 +110101 POINT(40.88625938266706 73.2315940377361) bank110101 +110102 POINT(41.03321614980495 74.60225088595224) bank110102 +110103 POINT(39.7343471090532 74.82637595455415) bank110103 +110104 POINT(41.51678672139451 73.82567132770738) bank110104 +110105 POINT(40.13164866704609 73.02852416358557) bank110105 +110106 POINT(39.874504112526125 73.25368716159123) bank110106 +110107 POINT(40.01363938918391 74.36373078501913) bank110107 +110108 POINT(41.263868302740825 73.14185734439002) bank110108 +110109 POINT(40.788682842730196 74.92651746871651) bank110109 +110110 POINT(41.01622437690362 73.80499192324287) bank110110 +110111 POINT(41.47088971342651 73.74603347668268) bank110111 +110112 POINT(41.6109811815549 73.45010138994358) bank110112 +110113 POINT(41.698821621057924 74.48762723840132) bank110113 +110114 POINT(40.45731527875959 74.54796611000798) bank110114 +110115 POINT(41.68401442058333 74.11935467154927) bank110115 +110116 POINT(41.438695843824014 73.33274829685317) bank110116 +110117 POINT(40.02917300448482 73.84933201026254) bank110117 +110118 POINT(40.94710484694372 74.49139678005363) bank110118 +110119 POINT(40.035513497294026 74.4855983972568) bank110119 +110120 POINT(41.483472429190115 73.42385517270056) bank110120 +110121 POINT(41.19008521926374 74.7599329484606) bank110121 +110122 POINT(41.02172755018142 74.28071220902115) bank110122 +110123 POINT(40.386262819217286 74.25306453759197) bank110123 +110124 POINT(41.040483373211856 74.33638715657833) bank110124 +110125 POINT(40.171990369186915 73.76435480545523) bank110125 +110126 POINT(40.41631139819204 74.4844151070859) bank110126 +110127 POINT(40.53616106122313 74.62203046714441) bank110127 +110128 POINT(41.37752719018194 74.17560132174951) bank110128 +110129 POINT(40.64779451066448 74.27076434073484) bank110129 +110130 POINT(40.848224837811834 73.99973843833632) bank110130 +110131 POINT(39.71739699071618 73.83731373891764) bank110131 +110132 POINT(41.57305172787496 73.28730390948039) bank110132 +110133 POINT(41.550750563087995 73.20375875356815) bank110133 +110134 POINT(39.82381707301474 73.58007425118745) bank110134 +110135 POINT(40.44644488529219 74.31307410638591) bank110135 +110136 POINT(40.942292401267025 73.23048738476454) bank110136 +110137 POINT(40.098461047978034 73.50480533242713) bank110137 +110138 POINT(41.61192610866494 74.1398747983511) bank110138 +110139 POINT(40.60023327977705 73.82996546318142) bank110139 +110140 POINT(40.733301650204744 74.9531394761836) bank110140 +110141 POINT(41.63018261664357 73.37757457083454) bank110141 +110142 POINT(40.855124888005264 74.5955389572788) bank110142 +110143 POINT(40.53984356469166 73.43139185601713) bank110143 +110144 POINT(41.35026806980611 73.56050024733662) bank110144 +110145 POINT(41.5388206411108 73.65258012147963) bank110145 +110146 POINT(41.26028711609375 74.83475636218377) bank110146 +110147 POINT(41.12834505238518 73.71581018530163) bank110147 +110148 POINT(39.83397365646134 73.75778965812283) bank110148 +110149 POINT(41.18733151526038 74.3464898436677) bank110149 +110150 POINT(40.717513505501174 73.31110966790915) bank110150 +110151 POINT(41.688852343664074 73.66509269858675) bank110151 +110152 POINT(40.42041119961371 74.92464358893545) bank110152 +110153 POINT(41.4897593284733 74.68015432304051) bank110153 +110154 POINT(41.11843971149351 73.64587069948917) bank110154 +110155 POINT(40.90809788899991 73.95052912728556) bank110155 +110156 POINT(41.22371218659555 73.15422972040375) bank110156 +110157 POINT(40.05336035108167 74.80701463447902) bank110157 +110158 POINT(40.30311455359375 74.22138865590154) bank110158 +110159 POINT(40.88849551735464 74.62463323034851) bank110159 +110160 POINT(39.94426312581992 74.97274451061331) bank110160 +110161 POINT(39.733907969631844 74.48937731659751) bank110161 +110162 POINT(40.96678316559842 74.68801415928287) bank110162 +110163 POINT(39.87512455302157 73.44269673640419) bank110163 +110164 POINT(39.7293568406798 74.37135429088059) bank110164 +110165 POINT(41.5106014649946 73.772757141744) bank110165 +110166 POINT(41.6819910025212 74.75982897199643) bank110166 +110167 POINT(41.137106327842645 73.65764159106457) bank110167 +110168 POINT(40.87510624642826 73.68796329957047) bank110168 +110169 POINT(40.470250088190014 73.96908980328432) bank110169 +110170 POINT(40.95902964840228 74.8223957800822) bank110170 +110171 POINT(40.854463652914156 73.92504692962905) bank110171 +110172 POINT(40.79862450803796 73.69782648722365) bank110172 +110173 POINT(40.04494657167459 73.46618417086783) bank110173 +110174 POINT(39.72480431555759 74.17713763339233) bank110174 +110175 POINT(41.25492460765467 73.4868815753649) bank110175 +110176 POINT(39.79775502076825 74.34395564586013) bank110176 +110177 POINT(40.81457924386652 74.42887396674209) bank110177 +110178 POINT(40.01733435427332 73.80950218280515) bank110178 +110179 POINT(41.47717881725889 73.06067787871426) bank110179 +110180 POINT(41.46194368361546 73.27845227708569) bank110180 +110181 POINT(39.77072558334961 73.92667212133225) bank110181 +110182 POINT(40.32157095826741 73.22047412453254) bank110182 +110183 POINT(40.42984065403263 74.47722968051099) bank110183 +110184 POINT(41.446290311115426 73.9033068694427) bank110184 +110185 POINT(40.54459390141492 73.95649572526054) bank110185 +110186 POINT(40.211361507670595 73.85385043550012) bank110186 +110187 POINT(41.226765622794794 73.66591496442243) bank110187 +110188 POINT(40.25846491976687 73.28970023227956) bank110188 +110189 POINT(39.93272577812189 73.46511616956082) bank110189 +110190 POINT(39.812457516297314 73.84211050164008) bank110190 +110191 POINT(40.545361622506206 73.7901280388581) bank110191 +110192 POINT(40.56776976060806 73.5907234609054) bank110192 +110193 POINT(41.37692259165908 74.47042412042947) bank110193 +110194 POINT(40.10008615674175 73.05642325052982) bank110194 +110195 POINT(41.13988567580494 74.91413949393247) bank110195 +110196 POINT(39.82478624584569 74.81504451543576) bank110196 +110197 POINT(41.07421595627295 73.10903614114527) bank110197 +110198 POINT(40.71171458614728 74.45461493781444) bank110198 +110199 POINT(40.960423966891184 73.3972004522785) bank110199 +110200 POINT(39.788703870433274 74.7793020586377) bank110200 +110201 POINT(41.27827717198532 74.75505277889081) bank110201 +110202 POINT(40.14652390213127 74.10611372529299) bank110202 +110203 POINT(41.57145778065064 73.40688495628665) bank110203 +110204 POINT(39.95023175003405 73.58017467071245) bank110204 +110205 POINT(41.26992917587385 73.32810875477622) bank110205 +110206 POINT(40.811731115830305 73.4604393906116) bank110206 +110207 POINT(41.37879902255592 73.72556322244517) bank110207 +110208 POINT(40.30161431159727 73.36543437480756) bank110208 +110209 POINT(40.55647207932894 73.81391995176101) bank110209 +110210 POINT(41.542346950770735 74.95236155998371) bank110210 +110211 POINT(39.880124803743705 73.50301454758565) bank110211 +110212 POINT(41.052846953596216 74.20080177560031) bank110212 +110213 POINT(41.336992274021135 74.26704244081748) bank110213 +110214 POINT(40.441790668465856 73.63627784581655) bank110214 +110215 POINT(41.26598831509627 73.07451883051372) bank110215 +110216 POINT(39.82590972323612 73.98666993984052) bank110216 +110217 POINT(41.52429750567176 73.49312190537864) bank110217 +110218 POINT(41.24924041554394 73.67255902783711) bank110218 +110219 POINT(40.232643378222335 73.46486061828871) bank110219 +110220 POINT(39.77653425619186 74.64071426491431) bank110220 +110221 POINT(41.589482206829345 73.54232988911771) bank110221 +110222 POINT(40.8054923890846 73.7619066071842) bank110222 +110223 POINT(39.88243333444803 74.17203655646614) bank110223 +110224 POINT(40.232952834911266 74.2465367527845) bank110224 +110225 POINT(40.89126680036061 74.59187557837804) bank110225 +110226 POINT(40.75479072635591 73.62328953552787) bank110226 +110227 POINT(40.034505826833396 74.71529135093171) bank110227 +110228 POINT(41.1338937374148 74.08172634347825) bank110228 +110229 POINT(41.20562844896929 74.58212037507238) bank110229 +110230 POINT(40.041191939154345 73.12617839738462) bank110230 +110231 POINT(40.679322798936816 74.3415229336459) bank110231 +110232 POINT(41.2484841016202 73.64820330838043) bank110232 +110233 POINT(40.87241750577252 74.88351378116778) bank110233 +110234 POINT(40.31852039544762 74.28392341295424) bank110234 +110235 POINT(40.14088613538244 74.95085662367391) bank110235 +110236 POINT(41.57399913843612 73.4955339508638) bank110236 +110237 POINT(39.73204119384215 73.98608739933726) bank110237 +110238 POINT(41.283183489715306 73.24928492816358) bank110238 +110239 POINT(40.56208583816901 74.18428242052163) bank110239 +110240 POINT(40.16176905821088 74.09426187079697) bank110240 +110241 POINT(41.31961815904568 73.05516085396636) bank110241 +110242 POINT(39.73141951447207 73.54467931043062) bank110242 +110243 POINT(40.51276668534203 73.98063457757284) bank110243 +110244 POINT(40.211994601881514 74.05142161820534) bank110244 +110245 POINT(41.41501286212045 74.07582919802158) bank110245 +110246 POINT(40.65757317611152 73.5171442691966) bank110246 +110247 POINT(41.6650340383648 73.65697258266117) bank110247 +110248 POINT(40.291763731821625 74.29072189389896) bank110248 +110249 POINT(40.92858770421455 73.75165807905219) bank110249 +110250 POINT(40.79189856099999 73.19819713069957) bank110250 +110251 POINT(40.179173453362495 74.59993610921106) bank110251 +110252 POINT(40.89147505057537 73.68995536198435) bank110252 +110253 POINT(39.71795766896989 74.71907082767646) bank110253 +110254 POINT(40.85588463152526 73.94910296136939) bank110254 +110255 POINT(41.36335718409353 73.85562068037704) bank110255 +110256 POINT(40.38603259503513 73.05993215618109) bank110256 +110257 POINT(39.94032252137493 73.67248181637916) bank110257 +110258 POINT(40.07450733752363 73.51605744956106) bank110258 +110259 POINT(40.50826836072413 73.41953587632491) bank110259 +110260 POINT(40.04049651925201 74.06247783451707) bank110260 +110261 POINT(39.95020898418813 73.82886802780932) bank110261 +110262 POINT(40.305681325902796 74.04435617381586) bank110262 +110263 POINT(40.980416170697275 73.66946378733034) bank110263 +110264 POINT(39.80276454828906 74.6012286665789) bank110264 +110265 POINT(40.17856727704194 74.6200773882245) bank110265 +110266 POINT(39.760506189052236 73.50914333603943) bank110266 +110267 POINT(40.29781543209137 73.52914233613367) bank110267 +110268 POINT(40.2003096853141 74.11074904751521) bank110268 +110269 POINT(41.69114889990011 74.64394869536675) bank110269 +110270 POINT(40.164493559507605 73.07270516258278) bank110270 +110271 POINT(40.88844835081811 74.27276280521497) bank110271 +110272 POINT(41.410081646578405 73.08133427039256) bank110272 +110273 POINT(41.02642837413142 73.43141013486171) bank110273 +110274 POINT(41.35928277548249 73.62025023356881) bank110274 +110275 POINT(41.20824429793671 74.77796883907014) bank110275 +110276 POINT(40.22969237247569 73.75235277952778) bank110276 +110277 POINT(40.57090371123952 74.56103573198618) bank110277 +110278 POINT(41.2919932055695 74.89746347189906) bank110278 +110279 POINT(40.05922184941358 74.99825774031316) bank110279 +110280 POINT(40.54738234059974 74.05871685064689) bank110280 +110281 POINT(39.873702391180686 73.14331285865714) bank110281 +110282 POINT(40.40817888640345 74.1319503090313) bank110282 +110283 POINT(41.03212114442297 74.75785133598083) bank110283 +110284 POINT(40.43235902211415 73.69815047899074) bank110284 +110285 POINT(40.239245041201045 74.08647790614104) bank110285 +110286 POINT(40.6565097369463 74.87467596332014) bank110286 +110287 POINT(40.76442147439829 73.17892177920628) bank110287 +110288 POINT(40.344095756310644 73.27393886428672) bank110288 +110289 POINT(41.5864063374789 74.16323955046636) bank110289 +110290 POINT(40.31175334950134 74.56945978466014) bank110290 +110291 POINT(40.47802439849841 74.67887254240443) bank110291 +110292 POINT(41.342240747246535 73.87175841273934) bank110292 +110293 POINT(41.08101018276102 73.64745614773909) bank110293 +110294 POINT(41.271394040980255 73.90938501175377) bank110294 +110295 POINT(41.49445054085419 73.92708734187462) bank110295 +110296 POINT(40.17873168355132 74.56170960085166) bank110296 +110297 POINT(41.2784941033965 74.12359859923669) bank110297 +110298 POINT(40.120233319634835 74.38111412618447) bank110298 +110299 POINT(40.48851912066585 73.69996364634994) bank110299 +110300 POINT(40.722774300359745 73.6261830639585) bank110300 +110301 POINT(40.97533464064288 74.72491049813404) bank110301 +110302 POINT(39.78765319797311 73.44214832590805) bank110302 +110303 POINT(39.938469014916116 74.3683994509224) bank110303 +110304 POINT(40.15003330255773 74.81173671643732) bank110304 +110305 POINT(40.231109293794326 73.74322967873157) bank110305 +110306 POINT(41.6617364242462 73.64477447436542) bank110306 +110307 POINT(41.07072816793693 74.57224832981932) bank110307 +110308 POINT(40.54520053352518 74.03087458450673) bank110308 +110309 POINT(40.251625296925745 74.6127903568683) bank110309 +110310 POINT(41.58992831100696 74.32794998009595) bank110310 +110311 POINT(41.059183771493785 73.90487180015256) bank110311 +110312 POINT(41.15852888611908 73.4851062521748) bank110312 +110313 POINT(41.191515225958454 73.5003670865747) bank110313 +110314 POINT(40.496178794527594 73.83703715463322) bank110314 +110315 POINT(40.008932600928354 73.11660771608975) bank110315 +110316 POINT(39.96171468694242 74.57974121589011) bank110316 +110317 POINT(40.08608189088851 73.07680529472236) bank110317 +110318 POINT(40.07222987721443 74.51331689226973) bank110318 +110319 POINT(40.727200935783884 73.81312241052233) bank110319 +110320 POINT(40.01112643352817 74.24784624746657) bank110320 +110321 POINT(41.52622180515698 73.1480782895335) bank110321 +110322 POINT(41.06672819435919 73.1647694425357) bank110322 +110323 POINT(40.71974613542423 74.57395371750376) bank110323 +110324 POINT(40.30058769667862 73.2503183975809) bank110324 +110325 POINT(41.47711838616126 73.48770731069092) bank110325 +110326 POINT(40.31224510534875 73.59567193515085) bank110326 +110327 POINT(41.27844437575387 74.29492297953769) bank110327 +110328 POINT(40.06450850178479 73.57500603915031) bank110328 +110329 POINT(40.78816794729827 74.88062780079974) bank110329 +110330 POINT(40.82766431666809 73.98683967419677) bank110330 +110331 POINT(41.41015713494671 73.87733632797381) bank110331 +110332 POINT(39.990831217694506 74.7292075426983) bank110332 +110333 POINT(40.100414007471684 73.08567876782024) bank110333 +110334 POINT(41.094135523296586 73.22539871191681) bank110334 +110335 POINT(41.234696882966695 74.59938417552094) bank110335 +110336 POINT(40.36750149833226 73.33334291091074) bank110336 +110337 POINT(40.77390197592719 74.85779354415573) bank110337 +110338 POINT(40.20948314160408 74.82275776487738) bank110338 +110339 POINT(41.52522525924957 74.88143280996655) bank110339 +110340 POINT(40.498521837579915 74.0291902672239) bank110340 +110341 POINT(41.259379338404074 74.44416356004731) bank110341 +110342 POINT(39.91749021174323 74.04845272657853) bank110342 +110343 POINT(39.853573954857474 73.58831225729978) bank110343 +110344 POINT(40.8390778587441 74.72080720331579) bank110344 +110345 POINT(41.392930405203856 73.2163225235284) bank110345 +110346 POINT(39.83506331492279 74.43449533529255) bank110346 +110347 POINT(41.65055925474554 74.26412263646424) bank110347 +110348 POINT(39.74797342701392 73.50370700602541) bank110348 +110349 POINT(41.55105124833592 74.25920377903455) bank110349 +110350 POINT(41.16002681893807 74.20973439932037) bank110350 +110351 POINT(41.133804001416635 74.73804831654319) bank110351 +110352 POINT(40.553133997294175 73.7496598797629) bank110352 +110353 POINT(40.64429161739401 73.2766403077336) bank110353 +110354 POINT(41.12000793849748 73.7528518621389) bank110354 +110355 POINT(40.8671113657427 74.26751814097673) bank110355 +110356 POINT(41.42773066310828 74.09807363713726) bank110356 +110357 POINT(40.46019369894523 73.22108935213602) bank110357 +110358 POINT(41.57545360536192 74.54486505037504) bank110358 +110359 POINT(41.20297619434709 73.76105027152065) bank110359 +110360 POINT(40.922843163065934 73.1854745554175) bank110360 +110361 POINT(41.712677584254564 73.70543927296589) bank110361 +110362 POINT(39.762978987052435 73.88069765538668) bank110362 +110363 POINT(40.332289151011075 74.71292868759699) bank110363 +110364 POINT(41.119535432144104 74.72319568581065) bank110364 +110365 POINT(39.866219375853696 73.69108680790079) bank110365 +110366 POINT(40.328155184939625 73.85339314221098) bank110366 +110367 POINT(40.23807992800511 74.18183476437423) bank110367 +110368 POINT(39.75665041822951 74.54023845307775) bank110368 +110369 POINT(41.02575371391587 74.2761725816421) bank110369 +110370 POINT(41.681109885955095 74.07330423871016) bank110370 +110371 POINT(39.889619965160676 74.74077686164429) bank110371 +110372 POINT(39.86134906237265 74.35216129157635) bank110372 +110373 POINT(40.542538317150736 74.48107596646459) bank110373 +110374 POINT(39.7426568681773 73.65615493190666) bank110374 +110375 POINT(41.40541433683973 73.01392465080765) bank110375 +110376 POINT(39.76738566580235 73.92158905721162) bank110376 +110377 POINT(39.97343345155695 74.90244336352954) bank110377 +110378 POINT(41.06134231121282 73.34005325540248) bank110378 +110379 POINT(41.31958511354286 74.45303238314983) bank110379 +110380 POINT(40.85601854887564 74.61376617143124) bank110380 +110381 POINT(40.100568145420766 74.43363195546193) bank110381 +110382 POINT(41.57299482039928 73.27589381533357) bank110382 +110383 POINT(40.251995944130606 74.6789940334209) bank110383 +110384 POINT(40.23316079495616 74.12387380789252) bank110384 +110385 POINT(41.039467141614345 74.06815024935538) bank110385 +110386 POINT(40.75240651805114 73.8990170307702) bank110386 +110387 POINT(40.97959881002197 74.76236215168254) bank110387 +110388 POINT(40.68637396966177 73.05789105620202) bank110388 +110389 POINT(40.057951108643735 74.17034284054613) bank110389 +110390 POINT(40.67393983925981 73.34681458949433) bank110390 +110391 POINT(41.348053191174714 73.14411161147636) bank110391 +110392 POINT(41.10444926210789 74.03174000041466) bank110392 +110393 POINT(41.24107755283893 74.0210455978075) bank110393 +110394 POINT(41.708968609543575 74.30923719148424) bank110394 +110395 POINT(41.42074052654963 74.76635633473569) bank110395 +110396 POINT(40.779365375472956 74.33782105652747) bank110396 +110397 POINT(40.520107569141736 73.15234994817854) bank110397 +110398 POINT(41.673215348296864 73.32564265941058) bank110398 +110399 POINT(39.914148210943345 74.82391039778952) bank110399 +110400 POINT(41.667873842918034 73.56670111920937) bank110400 +110401 POINT(39.868293267458604 74.24415082999069) bank110401 +110402 POINT(40.80893858623031 73.26463063655518) bank110402 +110403 POINT(41.48910308383319 74.09657593811494) bank110403 +110404 POINT(40.68427452960795 73.71481171101384) bank110404 +110405 POINT(41.58304855839285 74.3780371157233) bank110405 +110406 POINT(39.98842348523024 73.16873831135588) bank110406 +110407 POINT(40.421483091653016 73.95691841174241) bank110407 +110408 POINT(40.97714607714988 73.39557515563611) bank110408 +110409 POINT(39.74643199015266 73.20038104978721) bank110409 +110410 POINT(39.73020526808774 74.38372784963379) bank110410 +110411 POINT(40.331064385605636 74.42460383594663) bank110411 +110412 POINT(40.322065964349555 74.50305709099966) bank110412 +110413 POINT(40.88180572329243 74.77146683300279) bank110413 +110414 POINT(41.25960407263116 73.08428078508761) bank110414 +110415 POINT(41.654047633393354 74.74351812958508) bank110415 +110416 POINT(41.553244562121655 74.93210816197983) bank110416 +110417 POINT(40.17097979965921 74.46721980172767) bank110417 +110418 POINT(40.94495935250781 74.83889039062166) bank110418 +110419 POINT(39.778506303031456 74.1942069951441) bank110419 +110420 POINT(41.27775865526597 74.02950243900703) bank110420 +110421 POINT(40.04807749469098 74.42347599886439) bank110421 +110422 POINT(40.803987355682814 74.4075002121702) bank110422 +110423 POINT(40.94624800772883 74.85392689188542) bank110423 +110424 POINT(41.270384819837155 73.5929457787998) bank110424 +110425 POINT(39.89340633591691 74.18461589690243) bank110425 +110426 POINT(40.988645041481874 73.48899681686704) bank110426 +110427 POINT(40.614868972302055 73.06657042033275) bank110427 +110428 POINT(40.9199280017117 73.01377508362884) bank110428 +110429 POINT(41.24930948965685 74.96559327502966) bank110429 +110430 POINT(41.35043801691605 74.32783005337458) bank110430 +110431 POINT(40.3115374004171 73.45124614162106) bank110431 +110432 POINT(40.02110109004619 74.76798882711067) bank110432 +110433 POINT(40.23903669234724 74.35186595415158) bank110433 +110434 POINT(41.24869440899249 73.24695338690977) bank110434 +110435 POINT(41.64452511202246 73.56178630077733) bank110435 +110436 POINT(40.44767467284833 74.66408002284345) bank110436 +110437 POINT(40.635482406830654 73.16333366803568) bank110437 +110438 POINT(40.19977330430748 73.40557390930468) bank110438 +110439 POINT(40.64050710118804 73.28153468370851) bank110439 +110440 POINT(40.99135366116224 73.10134810167135) bank110440 +110441 POINT(41.42642374034409 73.67493541089374) bank110441 +110442 POINT(40.410275234607845 73.59339958248098) bank110442 +110443 POINT(41.50638955486957 73.14937478380145) bank110443 +110444 POINT(40.63105353503679 74.72144907397788) bank110444 +110445 POINT(40.80584352685788 73.28528726079385) bank110445 +110446 POINT(40.27065138074681 73.00655172202514) bank110446 +110447 POINT(40.35795621910402 74.60817591366694) bank110447 +110448 POINT(41.30118291389502 73.86938590034775) bank110448 +110449 POINT(41.39020440829745 74.37736117162183) bank110449 +110450 POINT(41.36780824455949 74.34063777013318) bank110450 +110451 POINT(39.90894475584541 73.92441036218628) bank110451 +110452 POINT(40.48572864645294 73.31764582470063) bank110452 +110453 POINT(40.863003240318854 73.45726124853435) bank110453 +110454 POINT(40.692379215992794 74.11815697382842) bank110454 +110455 POINT(40.502596075074734 73.42530192220146) bank110455 +110456 POINT(39.78363246868863 74.13285692881215) bank110456 +110457 POINT(40.453122766946805 73.90845537525502) bank110457 +110458 POINT(41.21915226496373 74.84199635817366) bank110458 +110459 POINT(41.053785529530835 74.33428718943168) bank110459 +110460 POINT(40.341934390804944 73.80430460753719) bank110460 +110461 POINT(40.05663586502891 73.87173390262546) bank110461 +110462 POINT(41.34909531681607 74.23624216874931) bank110462 +110463 POINT(41.0805193945645 74.52191296834421) bank110463 +110464 POINT(41.065648560536914 74.79860661452587) bank110464 +110465 POINT(39.85960548146333 73.80077091360293) bank110465 +110466 POINT(41.31903931914407 74.64538260514837) bank110466 +110467 POINT(40.06961509577061 74.94849264168288) bank110467 +110468 POINT(40.73636342330248 73.14955068926997) bank110468 +110469 POINT(39.94090817329268 73.29660284703725) bank110469 +110470 POINT(41.60716943945871 73.76446979655017) bank110470 +110471 POINT(41.54028900679995 74.82646648540133) bank110471 +110472 POINT(41.28992870065234 73.1772823627878) bank110472 +110473 POINT(40.281233381087794 74.95873068542765) bank110473 +110474 POINT(39.98701033789658 74.06382725566424) bank110474 +110475 POINT(40.239376158640624 73.88693251584206) bank110475 +110476 POINT(41.63847684867562 73.41436869802492) bank110476 +110477 POINT(40.99753391426212 74.9208406037448) bank110477 +110478 POINT(40.87906168258044 73.61851259021786) bank110478 +110479 POINT(41.33865082733734 73.35283790426672) bank110479 +110480 POINT(39.797556728763965 74.20786435819166) bank110480 +110481 POINT(39.88150941950074 74.1090200263192) bank110481 +110482 POINT(39.75751241296959 74.83221089718681) bank110482 +110483 POINT(40.20396293701957 74.8770808117983) bank110483 +110484 POINT(40.6249754938302 73.7568286847376) bank110484 +110485 POINT(40.365556895618774 73.64675406060687) bank110485 +110486 POINT(40.73203780008682 74.39498935503012) bank110486 +110487 POINT(41.03865536545074 74.47482094660074) bank110487 +110488 POINT(41.54090062067113 74.8472881547626) bank110488 +110489 POINT(40.743917250471426 74.68493027173892) bank110489 +110490 POINT(40.73830403264351 73.0599160130229) bank110490 +110491 POINT(40.63247542638961 74.70712432920938) bank110491 +110492 POINT(41.60278744957428 73.93955895061633) bank110492 +110493 POINT(39.72896188853351 74.02035665039443) bank110493 +110494 POINT(40.10802097992933 73.66821805753479) bank110494 +110495 POINT(40.53881989620911 73.56011304654852) bank110495 +110496 POINT(39.9636194451358 74.3441110485728) bank110496 +110497 POINT(40.77665198054006 74.86908136316352) bank110497 +110498 POINT(41.262403808001615 73.3888575113143) bank110498 +110499 POINT(40.82264854257571 73.33438291858998) bank110499 +110500 POINT(39.921028621935456 74.37778341997807) bank110500 +110501 POINT(41.66266716442296 74.79898452992248) bank110501 +110502 POINT(41.57435775736917 73.31778984727639) bank110502 +110503 POINT(40.472407439103236 73.94171024706426) bank110503 +110504 POINT(40.06205195715663 73.75802622962694) bank110504 +110505 POINT(40.49475943526717 73.05164690141993) bank110505 +110506 POINT(41.190337987451095 73.94080805405805) bank110506 +110507 POINT(40.49881867340309 73.43529048097805) bank110507 +110508 POINT(40.28581147939498 74.22889109412314) bank110508 +110509 POINT(41.391421428457214 74.7018311499669) bank110509 +110510 POINT(39.715658348959025 74.62127062753737) bank110510 +110511 POINT(40.84482164772522 74.89937702601028) bank110511 +110512 POINT(40.04672254655053 74.3566568891457) bank110512 +110513 POINT(40.88194222296333 74.78102646657968) bank110513 +110514 POINT(41.36331301034071 74.23847583391372) bank110514 +110515 POINT(40.91029228894343 74.7340072737873) bank110515 +110516 POINT(40.316703188554094 73.99049724256201) bank110516 +110517 POINT(40.13627696135879 74.33484407194598) bank110517 +110518 POINT(40.99666569422588 74.49170612365631) bank110518 +110519 POINT(41.182071260877905 73.36509971944992) bank110519 +110520 POINT(41.61631158798574 73.41082260783233) bank110520 +110521 POINT(40.5249953569948 73.44444785697856) bank110521 +110522 POINT(41.33784218357181 73.0831387549982) bank110522 +110523 POINT(40.73747346457043 73.33409880594014) bank110523 +110524 POINT(40.85691773324899 73.61151905327596) bank110524 +110525 POINT(40.35842599583729 73.25268167755621) bank110525 +110526 POINT(40.11753310373276 73.678032115595) bank110526 +110527 POINT(39.94438805237591 74.46433577306246) bank110527 +110528 POINT(41.155024333741615 73.31128979856659) bank110528 +110529 POINT(41.34007092762582 74.2432506690263) bank110529 +110530 POINT(40.388939682950415 74.35496863987711) bank110530 +110531 POINT(40.23442483492889 74.22854847474677) bank110531 +110532 POINT(41.64261603975538 73.31507553993141) bank110532 +110533 POINT(41.15276815915642 74.24000884884855) bank110533 +110534 POINT(39.85495779992124 74.97092347255617) bank110534 +110535 POINT(40.42626426511748 74.71502518562221) bank110535 +110536 POINT(40.777039279531024 74.30177901791562) bank110536 +110537 POINT(39.84643094746918 73.61645680074406) bank110537 +110538 POINT(40.196484233953115 73.85350932511517) bank110538 +110539 POINT(41.248886258871636 74.20820005622852) bank110539 +110540 POINT(40.969704056052095 74.64406295274645) bank110540 +110541 POINT(41.24833729122468 73.78059513147582) bank110541 +110542 POINT(41.16346042152565 73.19313222855315) bank110542 +110543 POINT(40.733190182181076 73.53879591691644) bank110543 +110544 POINT(41.21149710334168 74.72899433392276) bank110544 +110545 POINT(40.25848513850199 74.05928458197972) bank110545 +110546 POINT(40.054020796650825 74.65060446891151) bank110546 +110547 POINT(41.3024556519919 73.1529178916438) bank110547 +110548 POINT(41.51627439791394 74.2321536221918) bank110548 +110549 POINT(40.91731569053693 74.8409502543318) bank110549 +110550 POINT(41.22916663669993 74.8531684102522) bank110550 +110551 POINT(40.52806658651495 73.29403254351608) bank110551 +110552 POINT(40.12036180140595 74.52701048801624) bank110552 +110553 POINT(41.359673205536815 74.01103979833948) bank110553 +110554 POINT(40.795723721052866 74.47095938869312) bank110554 +110555 POINT(40.99332907298424 74.50185249294034) bank110555 +110556 POINT(41.25336086168132 73.8828442954692) bank110556 +110557 POINT(40.032467126767926 73.08725336040952) bank110557 +110558 POINT(41.5920075520579 74.5647886378329) bank110558 +110559 POINT(41.037082226219944 73.30531502197277) bank110559 +110560 POINT(41.236462532229766 74.46533275399987) bank110560 +110561 POINT(40.039683406594776 73.00741014017242) bank110561 +110562 POINT(40.08205573269793 74.31501776404261) bank110562 +110563 POINT(40.48480113550644 73.07263576416666) bank110563 +110564 POINT(39.896499188472106 73.91659166509413) bank110564 +110565 POINT(39.83750372531361 73.94735986023038) bank110565 +110566 POINT(39.806037129929386 74.86246147630521) bank110566 +110567 POINT(41.1609520350086 74.42587342787655) bank110567 +110568 POINT(40.039936382360274 73.02016287573541) bank110568 +110569 POINT(40.34692876762808 74.27830006709546) bank110569 +110570 POINT(40.43358063236416 74.94804465207346) bank110570 +110571 POINT(40.16023363871638 73.95869646567883) bank110571 +110572 POINT(41.47451288863282 74.34299474324538) bank110572 +110573 POINT(40.476620270360996 73.6014604630294) bank110573 +110574 POINT(41.058236393038044 74.868119730662) bank110574 +110575 POINT(40.296403905607356 74.1212562936576) bank110575 +110576 POINT(40.99608953128647 73.66974395585218) bank110576 +110577 POINT(40.17441194300902 74.74820054735994) bank110577 +110578 POINT(40.65150861766814 74.59272458203921) bank110578 +110579 POINT(40.963368836352906 74.75952203590037) bank110579 +110580 POINT(41.01813208208326 74.21062141754358) bank110580 +110581 POINT(39.92876261328128 73.93626267168081) bank110581 +110582 POINT(40.210165976838475 74.37966969520103) bank110582 +110583 POINT(40.35655138438605 73.06554520491575) bank110583 +110584 POINT(40.70114786897123 73.53646219235439) bank110584 +110585 POINT(41.09307651170163 73.24888539492794) bank110585 +110586 POINT(40.91269017725313 74.20219569705237) bank110586 +110587 POINT(41.010542421239556 73.79569407807467) bank110587 +110588 POINT(40.07297203772768 74.63664999479313) bank110588 +110589 POINT(40.354818952935744 74.25658129102997) bank110589 +110590 POINT(40.0033949873798 73.13090640900248) bank110590 +110591 POINT(39.74219940705019 73.57546135107575) bank110591 +110592 POINT(40.526499750894935 74.19660427324898) bank110592 +110593 POINT(40.43766812620862 74.59539345776686) bank110593 +110594 POINT(41.41455808500499 73.82524198663178) bank110594 +110595 POINT(40.265627625857725 73.8115582399286) bank110595 +110596 POINT(41.43181183448049 73.34183999882687) bank110596 +110597 POINT(41.3229946089733 74.78033931205428) bank110597 +110598 POINT(40.805106086332664 73.91169263784143) bank110598 +110599 POINT(40.968105726509954 74.06707213166005) bank110599 +110600 POINT(40.36564415967763 74.264681697616) bank110600 +110601 POINT(40.17870303954544 74.27570251764868) bank110601 +110602 POINT(40.09027915460769 74.8298616154779) bank110602 +110603 POINT(41.35968331821306 73.32881230148617) bank110603 +110604 POINT(40.51342655444301 73.45998103477149) bank110604 +110605 POINT(41.65963822641533 73.40970760770531) bank110605 +110606 POINT(41.19808836582377 73.93087349351303) bank110606 +110607 POINT(41.414732788682976 74.91818608469863) bank110607 +110608 POINT(39.94605527038239 73.28550004491116) bank110608 +110609 POINT(41.298617291334125 73.403521376951) bank110609 +110610 POINT(40.48084125642589 74.56019609490467) bank110610 +110611 POINT(41.42273999142793 74.71440557186541) bank110611 +110612 POINT(40.255834308476544 73.71681556758588) bank110612 +110613 POINT(40.03831467141857 74.21520354037575) bank110613 +110614 POINT(40.418041977898824 73.67235668160161) bank110614 +110615 POINT(39.88530690539643 73.40364272227332) bank110615 +110616 POINT(40.80701676794822 73.3305687200357) bank110616 +110617 POINT(40.87511735363865 74.92250734711044) bank110617 +110618 POINT(40.50287502777186 73.26093646655247) bank110618 +110619 POINT(40.659345556668065 74.03307308416402) bank110619 +110620 POINT(40.302537974353505 73.26280721660717) bank110620 +110621 POINT(41.009289713406794 74.66613865031195) bank110621 +110622 POINT(41.009022540521116 74.7487916494819) bank110622 +110623 POINT(41.49008806148388 73.93508758765883) bank110623 +110624 POINT(40.035799222614166 73.32657279357913) bank110624 +110625 POINT(40.84118543138486 74.45538657419456) bank110625 +110626 POINT(40.76764225048373 74.7427741005162) bank110626 +110627 POINT(41.052965682351825 74.86829124529721) bank110627 +110628 POINT(40.46939608310434 74.68082858626428) bank110628 +110629 POINT(40.95820750724942 73.84349598802457) bank110629 +110630 POINT(40.37177787659519 74.79173554666782) bank110630 +110631 POINT(41.08530963691928 74.22760379037209) bank110631 +110632 POINT(40.42518738065691 74.65898028881894) bank110632 +110633 POINT(41.221047234546376 73.726798071323) bank110633 +110634 POINT(40.30085946542851 73.12624501966495) bank110634 +110635 POINT(40.27239314134072 74.84270291538174) bank110635 +110636 POINT(39.95186027291466 73.17059280185819) bank110636 +110637 POINT(41.128752127397675 73.68048360327953) bank110637 +110638 POINT(40.87378066290697 74.55024857784248) bank110638 +110639 POINT(39.99497663892048 73.64557891630668) bank110639 +110640 POINT(40.81099968122903 74.80051435827389) bank110640 +110641 POINT(41.632043344889276 74.20415752381606) bank110641 +110642 POINT(41.426209761673014 73.02954295716935) bank110642 +110643 POINT(40.378425029815574 73.98093924424215) bank110643 +110644 POINT(40.8747573571516 73.79664259053757) bank110644 +110645 POINT(40.52345817331716 74.18442108704305) bank110645 +110646 POINT(41.55243491051308 74.6042456728499) bank110646 +110647 POINT(41.3643210959912 74.69123750963858) bank110647 +110648 POINT(39.77362570271023 73.37377521753079) bank110648 +110649 POINT(40.96402741434017 73.90810289629651) bank110649 +110650 POINT(41.39851150462054 74.13513617103781) bank110650 +110651 POINT(41.45394353988544 74.43789255868418) bank110651 +110652 POINT(41.37149100560836 73.70368679741921) bank110652 +110653 POINT(40.1676248582865 74.59855943449563) bank110653 +110654 POINT(41.49115146164118 73.49401992065889) bank110654 +110655 POINT(41.23924392655288 73.56626943445697) bank110655 +110656 POINT(40.270206428626054 74.36646127680028) bank110656 +110657 POINT(40.111095845902256 73.01203265039099) bank110657 +110658 POINT(40.785652642161146 73.3602744893741) bank110658 +110659 POINT(40.52978432045201 73.51735899939419) bank110659 +110660 POINT(41.10338002540817 74.12925863745072) bank110660 +110661 POINT(41.10977858429122 74.27110675234012) bank110661 +110662 POINT(40.93143972514388 73.06375841600409) bank110662 +110663 POINT(41.345859659534604 74.23145120365109) bank110663 +110664 POINT(40.08244294515992 74.3762790771208) bank110664 +110665 POINT(40.20879057719982 73.94025731569678) bank110665 +110666 POINT(41.391245652722745 73.51218305710728) bank110666 +110667 POINT(40.06460859717995 73.54025395726522) bank110667 +110668 POINT(40.82544392745534 74.04115845971583) bank110668 +110669 POINT(39.97183770497041 73.312905510755) bank110669 +110670 POINT(41.23907463788224 73.39084412357153) bank110670 +110671 POINT(40.08671760203952 73.59957625099163) bank110671 +110672 POINT(40.903014761682215 74.63793158105032) bank110672 +110673 POINT(40.65478429055534 73.6228429411065) bank110673 +110674 POINT(41.50301964033126 74.90979277573565) bank110674 +110675 POINT(39.74311529205939 73.56800267624615) bank110675 +110676 POINT(40.62421067265866 73.1165264668538) bank110676 +110677 POINT(41.085024113125215 73.72457647433596) bank110677 +110678 POINT(40.51620677675111 74.25907195726717) bank110678 +110679 POINT(40.557214426750676 74.71335660007048) bank110679 +110680 POINT(40.73710414326488 73.45162565634266) bank110680 +110681 POINT(40.22269652889892 74.12534061376893) bank110681 +110682 POINT(41.51332781231912 73.33211554534411) bank110682 +110683 POINT(41.40325652016019 74.45161298224134) bank110683 +110684 POINT(39.72217014086844 73.16127895526922) bank110684 +110685 POINT(40.452438848507775 73.12427493964535) bank110685 +110686 POINT(40.184779922658116 73.15579371010782) bank110686 +110687 POINT(39.825101300602896 73.05496568346116) bank110687 +110688 POINT(40.18285760925132 74.62554208044018) bank110688 +110689 POINT(40.675229505107424 73.2532200048612) bank110689 +110690 POINT(40.563176079683764 73.18315499877335) bank110690 +110691 POINT(40.4012191567011 74.423862954336) bank110691 +110692 POINT(41.57873839216047 73.66819686708261) bank110692 +110693 POINT(41.25002526166996 74.55324882802208) bank110693 +110694 POINT(41.12783694517131 73.11584512652047) bank110694 +110695 POINT(40.42027997182057 74.45661862550078) bank110695 +110696 POINT(40.16546964701248 74.34784400461449) bank110696 +110697 POINT(41.128813821310715 73.04500322822261) bank110697 +110698 POINT(40.155522380002516 73.52549618263356) bank110698 +110699 POINT(40.95529964128368 74.54773553857873) bank110699 +110700 POINT(39.89067283702932 73.53277845411495) bank110700 +110701 POINT(41.365061195816835 74.7043355453457) bank110701 +110702 POINT(41.02708779676946 73.49907340977299) bank110702 +110703 POINT(40.421024067080765 73.87291345631517) bank110703 +110704 POINT(40.486378813120616 73.16030154002145) bank110704 +110705 POINT(40.36503271904475 74.99015136152798) bank110705 +110706 POINT(41.58767513500059 74.48012073231692) bank110706 +110707 POINT(40.643395977244595 74.0560866546192) bank110707 +110708 POINT(41.28845463682516 73.66686499310313) bank110708 +110709 POINT(41.06862325192552 74.22439630959654) bank110709 +110710 POINT(40.12762678933517 73.92107814675654) bank110710 +110711 POINT(41.108568079876235 74.83453181802237) bank110711 +110712 POINT(40.47556114917493 74.34190702591904) bank110712 +110713 POINT(40.974230543287604 73.49669721048933) bank110713 +110714 POINT(39.79472090053488 73.39588783859598) bank110714 +110715 POINT(39.83191335537382 74.0480007704877) bank110715 +110716 POINT(41.15768644945938 74.03388057876255) bank110716 +110717 POINT(41.41823397422117 73.59271826377756) bank110717 +110718 POINT(41.00975939252457 74.73110436427388) bank110718 +110719 POINT(40.81454200927169 74.37845665921232) bank110719 +110720 POINT(39.75269345178344 74.03065680228369) bank110720 +110721 POINT(41.621187314090015 73.84854062247771) bank110721 +110722 POINT(41.398211764940235 74.2335177067955) bank110722 +110723 POINT(39.98404119275905 74.89098256334853) bank110723 +110724 POINT(41.27324138989804 74.28088623322888) bank110724 +110725 POINT(39.944748993682346 73.79718699251053) bank110725 +110726 POINT(39.94193610005333 73.78282736404437) bank110726 +110727 POINT(40.066717124609376 74.90936076179813) bank110727 +110728 POINT(41.374626094180506 73.2265054385555) bank110728 +110729 POINT(40.547087176047555 74.87928609598696) bank110729 +110730 POINT(41.70577438521372 73.25486095751886) bank110730 +110731 POINT(41.02093207353507 74.7639114229633) bank110731 +110732 POINT(41.25521101646483 74.64769633756266) bank110732 +110733 POINT(39.884169361081526 73.77869796663013) bank110733 +110734 POINT(40.547670925774696 73.80954606767675) bank110734 +110735 POINT(40.2076489805972 73.68404187292964) bank110735 +110736 POINT(41.002138535456375 74.99430051603645) bank110736 +110737 POINT(39.788818408856855 73.18860097986969) bank110737 +110738 POINT(40.088020251176694 74.40217527745615) bank110738 +110739 POINT(39.84519737882662 73.26791344121928) bank110739 +110740 POINT(41.12055944595876 73.94460490811625) bank110740 +110741 POINT(39.84325019309533 74.91012402382243) bank110741 +110742 POINT(41.62845456207676 74.47894007987205) bank110742 +110743 POINT(41.00925084627952 73.9782640747915) bank110743 +110744 POINT(40.502141015781824 73.22583741424161) bank110744 +110745 POINT(41.31665191544209 73.82667363174878) bank110745 +110746 POINT(39.96439558970537 73.34606377375144) bank110746 +110747 POINT(40.46929516143886 73.50758168406243) bank110747 +110748 POINT(41.31251442485759 74.13583807189778) bank110748 +110749 POINT(41.096708754941396 73.19980508536099) bank110749 +110750 POINT(40.48854890834076 74.39783545767902) bank110750 +110751 POINT(41.12316156249368 73.54523218955639) bank110751 +110752 POINT(41.11130370586471 73.6117301108798) bank110752 +110753 POINT(40.251848156388306 74.97835633122862) bank110753 +110754 POINT(41.187870266529856 74.18439139000914) bank110754 +110755 POINT(40.28603826719291 74.12528920458432) bank110755 +110756 POINT(41.265882330313815 73.33926916183442) bank110756 +110757 POINT(41.128060656198485 74.82347335938556) bank110757 +110758 POINT(41.69363989955765 73.46270309503085) bank110758 +110759 POINT(41.08868748961376 74.72751068426999) bank110759 +110760 POINT(40.51339010548481 74.68873232240094) bank110760 +110761 POINT(40.3772902527997 73.14100598310321) bank110761 +110762 POINT(41.44165547317731 73.8708469993163) bank110762 +110763 POINT(41.02146016921517 73.63033062190499) bank110763 +110764 POINT(41.54988489459683 74.99117432326427) bank110764 +110765 POINT(41.533247705025715 73.95257740425252) bank110765 +110766 POINT(41.21110248286441 73.17854494825133) bank110766 +110767 POINT(41.69628501963098 74.46359703907297) bank110767 +110768 POINT(41.50971921340713 74.63786290114236) bank110768 +110769 POINT(39.889651628883584 73.54384270477668) bank110769 +110770 POINT(39.83810837544292 73.06773809027442) bank110770 +110771 POINT(41.07811835168214 73.87961360753738) bank110771 +110772 POINT(39.982938256355816 73.81744306748202) bank110772 +110773 POINT(40.69500567884092 73.57465364305625) bank110773 +110774 POINT(41.056580632590226 73.51704838776483) bank110774 +110775 POINT(41.42823138125617 74.01922129416062) bank110775 +110776 POINT(41.17989800819137 73.15195870235678) bank110776 +110777 POINT(41.22123274174303 73.75832593660935) bank110777 +110778 POINT(40.49672702534266 74.82267900024733) bank110778 +110779 POINT(40.01957975156832 74.51347771678388) bank110779 +110780 POINT(40.88995754197707 74.11068509170148) bank110780 +110781 POINT(40.29899415953997 73.3999747833718) bank110781 +110782 POINT(40.306803871121645 74.12836163682131) bank110782 +110783 POINT(40.24962332281231 74.8371915550452) bank110783 +110784 POINT(39.969641032461915 73.44092196630439) bank110784 +110785 POINT(40.38434206379165 73.11732900858695) bank110785 +110786 POINT(41.12317090166233 74.87898318612142) bank110786 +110787 POINT(41.56669108256013 73.21012153201455) bank110787 +110788 POINT(40.30295929285015 74.75791959794742) bank110788 +110789 POINT(41.26741995629913 73.8613553825778) bank110789 +110790 POINT(41.52114926594603 74.02741981139573) bank110790 +110791 POINT(41.69544156440751 74.24842113694861) bank110791 +110792 POINT(40.46813725873381 74.71647916374665) bank110792 +110793 POINT(40.63338500610991 73.43561371141918) bank110793 +110794 POINT(41.32793940333848 74.5322752466272) bank110794 +110795 POINT(41.24246872348526 74.7516571650389) bank110795 +110796 POINT(40.62647835836963 74.9757827846549) bank110796 +110797 POINT(40.82141301319668 73.28027036140811) bank110797 +110798 POINT(40.317665840091365 74.29257391195232) bank110798 +110799 POINT(41.70885839332986 73.329030898393) bank110799 +110800 POINT(40.41132358199802 73.8976647026235) bank110800 +110801 POINT(41.57951907585476 74.48984458873231) bank110801 +110802 POINT(40.20738205313523 74.53209727046377) bank110802 +110803 POINT(40.79424570543514 74.86679831275468) bank110803 +110804 POINT(40.56281923923023 74.71782542155036) bank110804 +110805 POINT(40.099387166355406 74.13661376650415) bank110805 +110806 POINT(40.25714668568699 73.70723950909257) bank110806 +110807 POINT(40.40107822757557 73.39603647736945) bank110807 +110808 POINT(39.84173354156768 73.85871850033554) bank110808 +110809 POINT(41.45954270163063 73.655440088022) bank110809 +110810 POINT(40.714213630775596 74.97021165498232) bank110810 +110811 POINT(41.45358836263817 73.56533139109766) bank110811 +110812 POINT(40.264975485650794 74.70288414765972) bank110812 +110813 POINT(40.31814687744828 74.93147815886903) bank110813 +110814 POINT(40.31173068550622 74.41168324087852) bank110814 +110815 POINT(40.244270370122216 73.98286588512073) bank110815 +110816 POINT(41.70117881261413 74.50582205116793) bank110816 +110817 POINT(40.2820338164932 73.45036805545115) bank110817 +110818 POINT(41.312597455319974 74.71269513779892) bank110818 +110819 POINT(40.720345961245016 73.74392640508214) bank110819 +110820 POINT(40.37873733544828 74.09628298724394) bank110820 +110821 POINT(41.627125255599715 74.41961788182908) bank110821 +110822 POINT(39.77010301619896 73.32559284900749) bank110822 +110823 POINT(40.77426230584752 74.21870099528861) bank110823 +110824 POINT(40.6838712136566 74.09673979865435) bank110824 +110825 POINT(40.354535745411596 73.77781765245865) bank110825 +110826 POINT(40.77079857102776 74.79106034730523) bank110826 +110827 POINT(40.70335255493569 75.00164783314597) bank110827 +110828 POINT(40.44081434072559 73.20066731850697) bank110828 +110829 POINT(41.04724081798806 74.2587357847544) bank110829 +110830 POINT(39.91663563888973 74.84187468144897) bank110830 +110831 POINT(40.60416214567424 74.62346565809213) bank110831 +110832 POINT(41.68672862476063 74.60911794472769) bank110832 +110833 POINT(40.10769223713345 74.57868104675607) bank110833 +110834 POINT(40.23165765098341 74.3996610305753) bank110834 +110835 POINT(41.423085547559566 73.04978354451026) bank110835 +110836 POINT(41.23770008800626 73.63258450045704) bank110836 +110837 POINT(40.220116527665816 73.13893111175658) bank110837 +110838 POINT(40.315212235714284 74.7661628302142) bank110838 +110839 POINT(41.55694063281986 74.72876381759633) bank110839 +110840 POINT(40.92910300464754 73.19048607752353) bank110840 +110841 POINT(39.79556883540274 74.16859935858389) bank110841 +110842 POINT(40.694459202444904 74.97415988810319) bank110842 +110843 POINT(39.82818634165056 74.49725985288292) bank110843 +110844 POINT(40.60500805997979 74.35435442973244) bank110844 +110845 POINT(40.45669889065711 74.34486594264943) bank110845 +110846 POINT(41.527260280070024 73.3941016824696) bank110846 +110847 POINT(41.47268474871452 73.99219534194123) bank110847 +110848 POINT(41.49603991347843 73.37283087562861) bank110848 +110849 POINT(39.84176272485541 73.10759723559244) bank110849 +110850 POINT(39.772387822464395 74.33877920985613) bank110850 +110851 POINT(40.63637110133149 74.09199232167569) bank110851 +110852 POINT(41.630534904158125 73.49926806573023) bank110852 +110853 POINT(40.81677891190642 73.8229270171278) bank110853 +110854 POINT(41.239026832523045 73.46814653837211) bank110854 +110855 POINT(39.96464111029047 73.34150371414397) bank110855 +110856 POINT(40.30280780616662 73.35489827582312) bank110856 +110857 POINT(41.0920432408996 73.9189313092623) bank110857 +110858 POINT(40.050086734471 74.32066671469042) bank110858 +110859 POINT(40.04790207059055 74.95797050258754) bank110859 +110860 POINT(41.56170445276325 73.05652864485687) bank110860 +110861 POINT(39.731750341225386 73.70075150349311) bank110861 +110862 POINT(40.2810958832168 73.08349053521765) bank110862 +110863 POINT(39.92254089480803 73.86674858738715) bank110863 +110864 POINT(41.07989434720802 73.24034931598783) bank110864 +110865 POINT(39.94696907185388 74.2834782663089) bank110865 +110866 POINT(40.09400669015177 74.53153201233576) bank110866 +110867 POINT(39.7222306927292 73.83620013428228) bank110867 +110868 POINT(40.41466468716122 74.09590518291309) bank110868 +110869 POINT(41.628700241950895 74.03594019735452) bank110869 +110870 POINT(40.70796510407145 74.64452224635804) bank110870 +110871 POINT(39.746076585381516 73.73670091366593) bank110871 +110872 POINT(40.41790036040278 73.61011060916833) bank110872 +110873 POINT(40.63580484421509 74.37175071590596) bank110873 +110874 POINT(40.59536296608612 73.4464742597169) bank110874 +110875 POINT(40.74324559173583 74.3854398573262) bank110875 +110876 POINT(40.699449389258994 73.16914306782704) bank110876 +110877 POINT(40.53760024888554 74.03819762422306) bank110877 +110878 POINT(41.53116718062224 73.87475288147154) bank110878 +110879 POINT(40.345620084456094 73.56943635229004) bank110879 +110880 POINT(39.9247640431469 73.84123280895054) bank110880 +110881 POINT(39.90832202584634 74.28564118620294) bank110881 +110882 POINT(39.94446348226927 74.63248792422863) bank110882 +110883 POINT(41.45684298940703 73.2551578496335) bank110883 +110884 POINT(40.011352175616 73.18687340958337) bank110884 +110885 POINT(40.30335414510369 74.0200310647968) bank110885 +110886 POINT(40.919466198906534 73.01817020830751) bank110886 +110887 POINT(41.51136597768205 74.54537484673361) bank110887 +110888 POINT(41.39935498504151 73.27443656014748) bank110888 +110889 POINT(41.45223283963786 73.27665457186752) bank110889 +110890 POINT(39.77620534652478 73.72838934794947) bank110890 +110891 POINT(39.74657436360111 73.20805401073552) bank110891 +110892 POINT(39.867846943946724 73.88899374625659) bank110892 +110893 POINT(40.249385972986786 74.30242901712771) bank110893 +110894 POINT(41.38706592283542 73.75687514493649) bank110894 +110895 POINT(41.57215125518912 74.43406641404377) bank110895 +110896 POINT(41.6669429874743 73.24224526908677) bank110896 +110897 POINT(40.1887045119086 73.5997336051586) bank110897 +110898 POINT(39.830579555665445 74.70574353379213) bank110898 +110899 POINT(40.54459888870367 74.27061225121491) bank110899 +110900 POINT(39.835521897319836 73.83803366416329) bank110900 +110901 POINT(40.54536244564597 74.51038638011792) bank110901 +110902 POINT(40.696228353342576 73.5824678934411) bank110902 +110903 POINT(41.51557007413847 73.80983316406657) bank110903 +110904 POINT(41.095591607285975 74.17187794766606) bank110904 +110905 POINT(41.184517291894224 74.94420999873961) bank110905 +110906 POINT(39.867941688788044 74.29271362684969) bank110906 +110907 POINT(40.45751275941422 74.4863790691719) bank110907 +110908 POINT(40.5846718854893 73.14204525976272) bank110908 +110909 POINT(41.376748719099595 73.07378964673731) bank110909 +110910 POINT(40.3141359708531 74.30321528373281) bank110910 +110911 POINT(41.424726244539094 74.26957428113064) bank110911 +110912 POINT(40.429752948129696 74.99063335634513) bank110912 +110913 POINT(40.24981760948539 74.9405941362117) bank110913 +110914 POINT(40.73194788016164 74.80888576867754) bank110914 +110915 POINT(41.33475485203854 73.75086894736063) bank110915 +110916 POINT(40.10247399831901 73.9223372048716) bank110916 +110917 POINT(41.43833495332006 73.33873334867889) bank110917 +110918 POINT(41.4320263525084 73.3161604584267) bank110918 +110919 POINT(41.50615801891482 73.27947991473144) bank110919 +110920 POINT(40.850752379610825 73.26911621094168) bank110920 +110921 POINT(40.114117370561516 74.02514986257472) bank110921 +110922 POINT(40.37776286199191 73.53849204751258) bank110922 +110923 POINT(41.13091247165926 74.90781138072303) bank110923 +110924 POINT(40.59238921479398 73.61063332374874) bank110924 +110925 POINT(39.79682035290012 74.34946108358527) bank110925 +110926 POINT(39.99152756844635 73.36716574927935) bank110926 +110927 POINT(40.6447789379124 74.73884632115318) bank110927 +110928 POINT(40.62676628085448 73.67497884567914) bank110928 +110929 POINT(40.61618804532407 74.38217439522634) bank110929 +110930 POINT(41.55692426864128 73.63173856557943) bank110930 +110931 POINT(40.80293307468018 74.21818362509109) bank110931 +110932 POINT(41.694678123557004 73.7642604129119) bank110932 +110933 POINT(40.83669968474158 73.68578068727201) bank110933 +110934 POINT(41.328219650904366 74.44545658280927) bank110934 +110935 POINT(39.82681428065242 73.51882317528315) bank110935 +110936 POINT(41.382110634484214 73.67264001917798) bank110936 +110937 POINT(40.73421831991592 73.21501643812064) bank110937 +110938 POINT(41.140913465251835 74.55665602950982) bank110938 +110939 POINT(40.32934074067583 73.44967274238677) bank110939 +110940 POINT(41.621579605816635 73.83350713697794) bank110940 +110941 POINT(40.356958580834245 74.95362230543543) bank110941 +110942 POINT(40.76370508507534 73.98841658479472) bank110942 +110943 POINT(41.165430425204676 74.13105603474725) bank110943 +110944 POINT(40.527181879236316 74.13006179156208) bank110944 +110945 POINT(40.180792322702416 74.03876876643018) bank110945 +110946 POINT(41.21579777593633 74.23137548337179) bank110946 +110947 POINT(41.37044083217491 73.43206569703703) bank110947 +110948 POINT(41.67652920980233 74.49406061541424) bank110948 +110949 POINT(40.87348555517879 73.98109999823107) bank110949 +110950 POINT(41.05864720514981 73.4160685243325) bank110950 +110951 POINT(41.51204656950524 73.13732248133086) bank110951 +110952 POINT(41.53732112054189 74.96760828643383) bank110952 +110953 POINT(41.53606772548136 73.20349174093141) bank110953 +110954 POINT(40.363127837760004 74.30296744045619) bank110954 +110955 POINT(39.9370524793298 73.32761162043691) bank110955 +110956 POINT(41.031749125612 74.541223011724) bank110956 +110957 POINT(40.96760442758497 73.08497998212951) bank110957 +110958 POINT(40.762716022497486 74.46282164423225) bank110958 +110959 POINT(40.78381031403979 74.54177853738601) bank110959 +110960 POINT(39.96567236515359 74.40607714189015) bank110960 +110961 POINT(41.016114883244185 73.96888441683733) bank110961 +110962 POINT(41.66448317247176 73.0628826868972) bank110962 +110963 POINT(40.422424499847025 73.68314558640883) bank110963 +110964 POINT(40.37320933286715 73.60122716546783) bank110964 +110965 POINT(41.321274444916384 73.78953863184606) bank110965 +110966 POINT(41.19813070706422 74.63739511094062) bank110966 +110967 POINT(40.58762355586536 74.26274392730055) bank110967 +110968 POINT(41.43436762975239 74.98540464782229) bank110968 +110969 POINT(41.69773460549064 73.9835771273559) bank110969 +110970 POINT(40.62811563886883 74.73094257451251) bank110970 +110971 POINT(40.86973022395241 73.46286620177884) bank110971 +110972 POINT(39.72645432130483 74.11103236407904) bank110972 +110973 POINT(39.96109263495615 73.5912149238977) bank110973 +110974 POINT(40.84317809207018 73.78665480556593) bank110974 +110975 POINT(41.69052537033271 73.60984896641327) bank110975 +110976 POINT(39.81530502045331 73.74243168481259) bank110976 +110977 POINT(40.031715492702205 73.84475387560734) bank110977 +110978 POINT(39.879099905456954 73.02805677618066) bank110978 +110979 POINT(41.20300880284548 74.01479178912702) bank110979 +110980 POINT(39.83054060343379 74.61605122372235) bank110980 +110981 POINT(40.27156972767381 73.81783013905137) bank110981 +110982 POINT(39.91777447207486 74.35069109071719) bank110982 +110983 POINT(40.763169378642516 73.4049627463365) bank110983 +110984 POINT(40.18353326771356 73.83710384018333) bank110984 +110985 POINT(41.47628474208129 74.84290285800888) bank110985 +110986 POINT(41.185587075371615 73.82332201192497) bank110986 +110987 POINT(39.838875262839494 74.75994308931928) bank110987 +110988 POINT(40.16235633923135 74.03898047708319) bank110988 +110989 POINT(40.07909095990284 73.94268526213088) bank110989 +110990 POINT(41.5802267472715 74.82353850393012) bank110990 +110991 POINT(40.5981455598601 74.46501065322727) bank110991 +110992 POINT(40.780240337964685 74.60554318030135) bank110992 +110993 POINT(41.117241331746406 74.00286708128515) bank110993 +110994 POINT(40.90192626081442 73.22869150023152) bank110994 +110995 POINT(41.550756835506306 73.21034567117712) bank110995 +110996 POINT(40.17775528651508 74.04323444831157) bank110996 +110997 POINT(40.327410531748605 73.07413190475486) bank110997 +110998 POINT(41.5650552544904 73.68968944756185) bank110998 +110999 POINT(40.908043317289525 74.47228201684582) bank110999 +111000 POINT(41.16879282667155 73.26218582582905) bank111000 +111001 POINT(39.731666167316035 74.21230979326215) bank111001 +111002 POINT(40.617866123422225 73.13868587992832) bank111002 +111003 POINT(41.11097213497379 73.82252762727062) bank111003 +111004 POINT(40.14949574111212 74.10408925804693) bank111004 +111005 POINT(41.570820103767886 73.48732100753071) bank111005 +111006 POINT(41.39271950342719 74.29319672350607) bank111006 +111007 POINT(39.965093669492155 74.80948434754006) bank111007 +111008 POINT(41.42814543264197 73.84304149878055) bank111008 +111009 POINT(40.089184013924346 73.24223627453468) bank111009 +111010 POINT(41.22098042597273 73.28020626795492) bank111010 +111011 POINT(41.67664103937847 73.47725313392007) bank111011 +111012 POINT(41.04852317720262 74.68773561096589) bank111012 +111013 POINT(40.049794688253264 74.80740093865332) bank111013 +111014 POINT(40.75982645657971 73.21787375211855) bank111014 +111015 POINT(39.85725580616476 73.85754081299102) bank111015 +111016 POINT(40.52675439866495 74.48407172830096) bank111016 +111017 POINT(39.91933171416132 74.375727134177) bank111017 +111018 POINT(39.937935671146015 74.81585592519109) bank111018 +111019 POINT(40.94007268117182 73.51862430947628) bank111019 +111020 POINT(40.234260607456974 73.8063713055849) bank111020 +111021 POINT(40.90472681623428 74.25059222250621) bank111021 +111022 POINT(39.72342761695301 74.01611298265125) bank111022 +111023 POINT(40.7780917722882 73.71333814478807) bank111023 +111024 POINT(40.70154975886078 74.85640577573888) bank111024 +111025 POINT(41.26081989612488 73.00986386261637) bank111025 +111026 POINT(41.49409013188399 74.0708433139062) bank111026 +111027 POINT(41.56121408096558 74.11331367611606) bank111027 +111028 POINT(40.19023857804838 74.48309037316689) bank111028 +111029 POINT(41.57765432463038 74.32368935953073) bank111029 +111030 POINT(40.55817766698659 73.69637001010291) bank111030 +111031 POINT(41.63040435950363 73.30631398686057) bank111031 +111032 POINT(39.968512129248694 73.34619202692201) bank111032 +111033 POINT(41.17679073950673 73.68795224890347) bank111033 +111034 POINT(39.911915397131665 74.53948457894462) bank111034 +111035 POINT(41.38183829167191 73.61969148347387) bank111035 +111036 POINT(39.86478472494022 74.6652346938046) bank111036 +111037 POINT(40.75380002312527 74.00434390462318) bank111037 +111038 POINT(40.57379560761957 73.2331463578103) bank111038 +111039 POINT(41.49978906499926 74.31196439726286) bank111039 +111040 POINT(41.28692638236024 73.18955521880994) bank111040 +111041 POINT(40.71596821848655 73.33087995043773) bank111041 +111042 POINT(40.87804030902392 73.11273822398614) bank111042 +111043 POINT(40.86999343139943 73.55157266159482) bank111043 +111044 POINT(40.5161258379018 73.73963519238961) bank111044 +111045 POINT(39.982532463246265 74.78652446317192) bank111045 +111046 POINT(40.9101233758681 74.19138325374456) bank111046 +111047 POINT(40.78593407030708 73.86829423889874) bank111047 +111048 POINT(40.49237409773903 73.94836513313518) bank111048 +111049 POINT(40.8988123286599 73.50850361166904) bank111049 +111050 POINT(40.23638488531846 73.50126090980964) bank111050 +111051 POINT(40.533619524960464 73.61382672916892) bank111051 +111052 POINT(41.3259431434744 73.3722154749122) bank111052 +111053 POINT(40.46601536530657 74.8358943153173) bank111053 +111054 POINT(41.377255446137504 74.83267279407738) bank111054 +111055 POINT(40.267686766509854 74.3734483107704) bank111055 +111056 POINT(40.52248944108943 73.73853366395169) bank111056 +111057 POINT(41.29893410276981 74.9225290108715) bank111057 +111058 POINT(40.712954462624985 74.03269955135245) bank111058 +111059 POINT(40.01149128297557 74.13524780399527) bank111059 +111060 POINT(40.81392404046899 73.1058711630998) bank111060 +111061 POINT(41.57704429602019 73.03109178636402) bank111061 +111062 POINT(39.80581209807877 73.32005735083695) bank111062 +111063 POINT(40.72958048823727 73.95876808949471) bank111063 +111064 POINT(40.13642240529366 74.3662552200465) bank111064 +111065 POINT(39.83229578070413 74.58976857887374) bank111065 +111066 POINT(40.478764665879844 73.41804274976276) bank111066 +111067 POINT(41.16918255329638 73.88018179172231) bank111067 +111068 POINT(41.003796800408445 73.61185311717497) bank111068 +111069 POINT(41.08917551934828 73.0254701070561) bank111069 +111070 POINT(41.65722506705035 74.7738423144343) bank111070 +111071 POINT(41.02185626686818 74.84705725907854) bank111071 +111072 POINT(39.8535533421311 73.66048401324302) bank111072 +111073 POINT(41.68461153596632 73.80952126587165) bank111073 +111074 POINT(41.68732371554621 74.10544455395443) bank111074 +111075 POINT(40.97456147991705 74.42759155784003) bank111075 +111076 POINT(40.036390294652456 74.97124272086273) bank111076 +111077 POINT(40.97047654547223 74.22214691995539) bank111077 +111078 POINT(39.75915681882693 74.62114366055789) bank111078 +111079 POINT(41.16815808034065 73.32873714668261) bank111079 +111080 POINT(40.03126957308828 73.84457549435305) bank111080 +111081 POINT(41.525014889241554 73.88724065594707) bank111081 +111082 POINT(40.804166777912535 73.79499597299028) bank111082 +111083 POINT(40.131762144800724 73.63005365811264) bank111083 +111084 POINT(40.68112408233308 73.55294624873135) bank111084 +111085 POINT(39.96397448708486 73.11295494080473) bank111085 +111086 POINT(41.1902361288006 74.52224261236306) bank111086 +111087 POINT(40.437700483786344 74.80212881632316) bank111087 +111088 POINT(39.805339396171654 74.73277705639211) bank111088 +111089 POINT(40.64379180558562 73.57838165230918) bank111089 +111090 POINT(40.65987535660944 74.16984667080169) bank111090 +111091 POINT(40.51198769836918 74.40555050768697) bank111091 +111092 POINT(39.77203664987531 74.6323110581097) bank111092 +111093 POINT(39.98230104533277 74.87594565525498) bank111093 +111094 POINT(40.540778445424976 73.9932345155147) bank111094 +111095 POINT(41.16520544641369 74.9486087070177) bank111095 +111096 POINT(40.4979138826243 73.65306373147436) bank111096 +111097 POINT(39.88351492275562 73.02266024856797) bank111097 +111098 POINT(39.82005086354191 74.30008320918328) bank111098 +111099 POINT(41.36652659465802 73.0065029275863) bank111099 +111100 POINT(40.33333363782931 73.74004726727149) bank111100 +111101 POINT(41.547789155557055 73.93111524913353) bank111101 +111102 POINT(40.58206708760073 73.66740286213754) bank111102 +111103 POINT(39.72750686738897 74.57394637058577) bank111103 +111104 POINT(41.10436179749735 73.7972106168244) bank111104 +111105 POINT(40.80453520659382 74.84010183922058) bank111105 +111106 POINT(40.4873073191284 73.0335822965426) bank111106 +111107 POINT(39.888075442098604 73.11096212983901) bank111107 +111108 POINT(40.42831770873803 74.70344643593636) bank111108 +111109 POINT(41.101701695532846 74.6163673935231) bank111109 +111110 POINT(41.310902367122566 73.19604618982233) bank111110 +111111 POINT(40.87839337372417 74.96370735996241) bank111111 +111112 POINT(40.18099254631028 74.03710688830053) bank111112 +111113 POINT(40.67838224844479 74.06877287624955) bank111113 +111114 POINT(40.33071783039484 74.63719950745205) bank111114 +111115 POINT(40.843732417711294 73.05027219576652) bank111115 +111116 POINT(41.11280785734971 74.76710611507264) bank111116 +111117 POINT(41.10506439599092 73.14273298712855) bank111117 +111118 POINT(41.528593702736245 74.92887235697974) bank111118 +111119 POINT(39.857868889616626 73.54244771486297) bank111119 +111120 POINT(40.88446067237727 73.97438937622528) bank111120 +111121 POINT(41.09169774144288 73.5020571584174) bank111121 +111122 POINT(39.81296474135 74.59282135551955) bank111122 +111123 POINT(41.26206519236696 74.84164456192302) bank111123 +111124 POINT(41.08212391482166 74.41725481538462) bank111124 +111125 POINT(40.511620808126295 74.74394910893407) bank111125 +111126 POINT(41.570650973575404 73.94350660446959) bank111126 +111127 POINT(40.62597878903567 74.62435323471757) bank111127 +111128 POINT(41.26413846820648 74.76372901219484) bank111128 +111129 POINT(41.331237868495904 73.59610780374425) bank111129 +111130 POINT(41.43598333201444 73.96901834649405) bank111130 +111131 POINT(40.969808977681296 73.13726908080852) bank111131 +111132 POINT(40.05150471938814 74.77139433042286) bank111132 +111133 POINT(40.728755530486445 74.42947699675555) bank111133 +111134 POINT(40.00105674983918 73.26309841437417) bank111134 +111135 POINT(40.562710390985345 73.96258848396083) bank111135 +111136 POINT(40.33262259024062 73.14906036916217) bank111136 +111137 POINT(41.24898163753569 74.02115948718563) bank111137 +111138 POINT(39.938441670266585 74.40899433808694) bank111138 +111139 POINT(40.98440520735455 73.49694442049878) bank111139 +111140 POINT(40.44836735827967 74.63782463642961) bank111140 +111141 POINT(39.87841560579423 74.01023569495445) bank111141 +111142 POINT(40.625881102002744 74.49671553382929) bank111142 +111143 POINT(40.02957906681193 73.18734792433769) bank111143 +111144 POINT(41.38685194331453 74.37964605950756) bank111144 +111145 POINT(39.74777441058073 73.83911704195532) bank111145 +111146 POINT(41.48749820832367 73.0274330145505) bank111146 +111147 POINT(40.892822788748305 73.04052852205601) bank111147 +111148 POINT(40.66772370845205 73.73600655567878) bank111148 +111149 POINT(40.61272578557584 74.48073643262576) bank111149 +111150 POINT(40.90332322587076 73.83374227477225) bank111150 +111151 POINT(41.24202614594964 74.84373907313838) bank111151 +111152 POINT(41.433383347269334 74.15800985843445) bank111152 +111153 POINT(40.11843009177868 73.8509815055148) bank111153 +111154 POINT(40.632165115400035 73.75899095509678) bank111154 +111155 POINT(39.8582031321194 73.87046620170345) bank111155 +111156 POINT(40.48667222979923 74.20435383287943) bank111156 +111157 POINT(40.95887465712009 74.0865910013913) bank111157 +111158 POINT(40.353488936868715 73.4295989800636) bank111158 +111159 POINT(40.31846553066499 74.67738467623589) bank111159 +111160 POINT(40.45858923187462 73.6705408568443) bank111160 +111161 POINT(39.864408867715305 73.17919368077155) bank111161 +111162 POINT(40.40330713242473 74.03703463693506) bank111162 +111163 POINT(39.73704244601081 74.24660695575174) bank111163 +111164 POINT(39.81902347231312 74.11007843913394) bank111164 +111165 POINT(40.63673866093842 73.63303828288583) bank111165 +111166 POINT(40.761971027841845 73.67445806571158) bank111166 +111167 POINT(40.89033745310263 73.459857900066) bank111167 +111168 POINT(41.17963728435598 74.9363248072772) bank111168 +111169 POINT(39.74529408408011 74.76279248278338) bank111169 +111170 POINT(41.3577094650423 74.06605226238327) bank111170 +111171 POINT(40.757106162991406 73.30552470784991) bank111171 +111172 POINT(40.137037678494444 73.05155497725912) bank111172 +111173 POINT(41.29282058974025 73.32527377939917) bank111173 +111174 POINT(39.82840830091205 73.67337428292647) bank111174 +111175 POINT(40.668584028037785 73.72164990669657) bank111175 +111176 POINT(40.036019728456665 73.72626694617087) bank111176 +111177 POINT(41.135338250103295 74.42819460371406) bank111177 +111178 POINT(40.37694200975833 74.32469720920302) bank111178 +111179 POINT(40.369985889849005 73.81562605558868) bank111179 +111180 POINT(39.93095700381785 74.3950529752884) bank111180 +111181 POINT(40.0849838159239 74.23018242304961) bank111181 +111182 POINT(41.018958191648146 73.8239059032046) bank111182 +111183 POINT(40.4919768452745 74.35283248911952) bank111183 +111184 POINT(40.9214999472649 74.22189445002164) bank111184 +111185 POINT(39.76778632109198 74.85707045676604) bank111185 +111186 POINT(41.10397624780483 73.70826578021251) bank111186 +111187 POINT(41.49220129567153 73.39629418482822) bank111187 +111188 POINT(40.96242608839918 74.88950478557493) bank111188 +111189 POINT(41.616615171325776 74.99216804868378) bank111189 +111190 POINT(40.32800130589426 74.93251375272652) bank111190 +111191 POINT(39.80756260134354 73.08796039371657) bank111191 +111192 POINT(41.24108021472567 74.37821631288688) bank111192 +111193 POINT(40.32972013963012 74.08442710872902) bank111193 +111194 POINT(40.093036833011574 74.91087869977908) bank111194 +111195 POINT(40.5541812545065 74.20434346226867) bank111195 +111196 POINT(41.117906435823095 73.43804647749776) bank111196 +111197 POINT(40.06833076740134 74.19903019348199) bank111197 +111198 POINT(41.1434870409712 73.42004669420419) bank111198 +111199 POINT(40.14579678847719 74.54235235626908) bank111199 +111200 POINT(41.51384662355056 74.94032916223343) bank111200 +111201 POINT(40.59326586753352 74.18798411482086) bank111201 +111202 POINT(40.96028487332247 73.74714490969916) bank111202 +111203 POINT(40.73616231352954 74.02093316342541) bank111203 +111204 POINT(40.4865880932444 73.15281977653954) bank111204 +111205 POINT(40.93264771513791 73.84621134062905) bank111205 +111206 POINT(41.0896097453925 73.29702610761805) bank111206 +111207 POINT(40.2651672642074 74.62348834653992) bank111207 +111208 POINT(41.323947691796874 73.46648986338687) bank111208 +111209 POINT(39.98525399885559 73.8765313553579) bank111209 +111210 POINT(41.613963281030756 73.21969044642859) bank111210 +111211 POINT(41.45540559691396 74.73313807119169) bank111211 +111212 POINT(41.193515301618184 73.83707696749751) bank111212 +111213 POINT(40.61011694518392 74.99895802247133) bank111213 +111214 POINT(40.1349252637177 73.31800723294877) bank111214 +111215 POINT(40.18003083189629 73.91955609127834) bank111215 +111216 POINT(40.82080713143425 74.97987288464763) bank111216 +111217 POINT(41.55886861162817 73.98551340232245) bank111217 +111218 POINT(40.30487851461237 73.62184390984186) bank111218 +111219 POINT(41.013729271297734 73.80937713728255) bank111219 +111220 POINT(40.417855108316566 74.41524114339025) bank111220 +111221 POINT(39.78318106892478 74.59277045179967) bank111221 +111222 POINT(39.91781177616506 73.38817127805898) bank111222 +111223 POINT(40.340173202793494 74.26941444254707) bank111223 +111224 POINT(41.12276853270371 73.85892748752774) bank111224 +111225 POINT(41.15439020543006 73.45823901505618) bank111225 +111226 POINT(40.2886667487323 74.06999862110739) bank111226 +111227 POINT(40.95414260004865 74.16493955905221) bank111227 +111228 POINT(41.26122509983625 73.10849641343485) bank111228 +111229 POINT(40.14027387060538 74.7831918471116) bank111229 +111230 POINT(41.04395209736511 74.84565420024535) bank111230 +111231 POINT(41.037400518081384 74.56548293466093) bank111231 +111232 POINT(41.65013568226458 74.97621971162398) bank111232 +111233 POINT(41.25394718442961 73.14028988423787) bank111233 +111234 POINT(40.99506911720252 74.16266455763082) bank111234 +111235 POINT(40.061713560241515 73.82062500455578) bank111235 +111236 POINT(41.57001662110825 73.40161281072466) bank111236 +111237 POINT(40.411505372146465 73.51414000313098) bank111237 +111238 POINT(39.80866752861445 73.1888465564632) bank111238 +111239 POINT(40.442163770017544 73.94730219211813) bank111239 +111240 POINT(40.73521870165498 73.390431040086) bank111240 +111241 POINT(40.67936260462484 74.58884106685227) bank111241 +111242 POINT(41.129755024454845 73.25226335767928) bank111242 +111243 POINT(41.470244620935624 73.38939793946449) bank111243 +111244 POINT(41.50977403284637 73.68676211605084) bank111244 +111245 POINT(39.968903060631206 74.52382350717359) bank111245 +111246 POINT(40.54045417177585 73.0511957359668) bank111246 +111247 POINT(40.45419648216491 73.11101780486257) bank111247 +111248 POINT(41.63488654875518 73.79188662332533) bank111248 +111249 POINT(40.955873526944224 73.19781173674005) bank111249 +111250 POINT(40.4788726300167 74.8911525986235) bank111250 +111251 POINT(41.70555049975555 74.17924896886231) bank111251 +111252 POINT(40.644921684250036 74.31756313509133) bank111252 +111253 POINT(40.15034561698583 73.00868564289017) bank111253 +111254 POINT(40.44810280538946 73.70507421792196) bank111254 +111255 POINT(41.32301139406361 74.78670937986551) bank111255 +111256 POINT(40.92801007812247 73.57333543190553) bank111256 +111257 POINT(40.0936318380243 74.4666110419592) bank111257 +111258 POINT(40.08889477863404 73.9907159145076) bank111258 +111259 POINT(40.07480911949562 74.32852806066707) bank111259 +111260 POINT(40.0009533372262 73.41642563679791) bank111260 +111261 POINT(40.85606846120108 73.11584409595092) bank111261 +111262 POINT(40.86956669077186 74.39203982142166) bank111262 +111263 POINT(39.78056703519888 74.81206770471684) bank111263 +111264 POINT(40.00529745662377 74.7510292147186) bank111264 +111265 POINT(41.61723424170428 73.97855220102193) bank111265 +111266 POINT(40.11153310227926 73.11523437896365) bank111266 +111267 POINT(40.346748888790295 74.395592985827) bank111267 +111268 POINT(41.602613045349045 74.32706561911422) bank111268 +111269 POINT(39.934956225216 73.01229550961273) bank111269 +111270 POINT(41.28967076023489 73.8210419844044) bank111270 +111271 POINT(41.40875525764931 73.76537121815443) bank111271 +111272 POINT(40.92749087267714 73.76804371160154) bank111272 +111273 POINT(41.526363215602785 73.00757878299017) bank111273 +111274 POINT(41.26781994096189 73.79326230658477) bank111274 +111275 POINT(41.39172259863973 74.77604718399465) bank111275 +111276 POINT(40.14510003137391 73.25859816625362) bank111276 +111277 POINT(40.10167534595172 73.8186416219349) bank111277 +111278 POINT(41.30835059398005 73.16212951585818) bank111278 +111279 POINT(41.10413812487463 73.43413234910999) bank111279 +111280 POINT(40.104798721714914 74.35429898981995) bank111280 +111281 POINT(40.74596134309561 74.65834488712034) bank111281 +111282 POINT(41.64504650829339 73.57433677159737) bank111282 +111283 POINT(41.39097546480443 73.68024152937475) bank111283 +111284 POINT(40.717801368398334 73.29649565768443) bank111284 +111285 POINT(40.887280909471585 74.07876441220343) bank111285 +111286 POINT(39.942021466208686 74.43817838034941) bank111286 +111287 POINT(41.39969615955405 73.91706151597197) bank111287 +111288 POINT(40.4174235926226 73.01637286479911) bank111288 +111289 POINT(40.10427183789337 73.53070445227912) bank111289 +111290 POINT(40.52162458285974 74.05498704867435) bank111290 +111291 POINT(41.20986911615673 73.59650895866321) bank111291 +111292 POINT(40.11489647467324 74.697510127274) bank111292 +111293 POINT(41.52960222532673 74.48193024018711) bank111293 +111294 POINT(40.97919699463323 73.5377929614721) bank111294 +111295 POINT(40.207036725715945 74.1781370112457) bank111295 +111296 POINT(41.438825125224454 73.03242512267602) bank111296 +111297 POINT(40.77801414822554 74.79194709625158) bank111297 +111298 POINT(40.57693736084961 73.29537834916329) bank111298 +111299 POINT(41.0899273141367 74.5994402085758) bank111299 +111300 POINT(41.22803835932117 74.86246989952153) bank111300 +111301 POINT(41.52048956252068 73.78804002753793) bank111301 +111302 POINT(41.668462677101715 74.18593883631797) bank111302 +111303 POINT(40.55485802037629 74.76309355960713) bank111303 +111304 POINT(41.22136686618266 74.77682495135203) bank111304 +111305 POINT(41.541955520129974 74.52314991615778) bank111305 +111306 POINT(41.57449007772066 73.014709996437) bank111306 +111307 POINT(41.14592381911323 73.1708765711904) bank111307 +111308 POINT(40.383628908329186 73.86011135362159) bank111308 +111309 POINT(40.745536617477 73.723593810819) bank111309 +111310 POINT(40.189989702712026 73.45165421370467) bank111310 +111311 POINT(40.71763394333553 74.74857563707131) bank111311 +111312 POINT(41.09119270590247 74.2234777448614) bank111312 +111313 POINT(39.99618653029096 74.76058437607567) bank111313 +111314 POINT(40.42018346791415 73.60649738269794) bank111314 +111315 POINT(41.05672425232606 73.58279039832045) bank111315 +111316 POINT(39.93231310651301 74.05719730772468) bank111316 +111317 POINT(40.36096433180213 73.25716865138479) bank111317 +111318 POINT(39.829464431856024 74.33489291463272) bank111318 +111319 POINT(39.82477541710706 74.06715044565041) bank111319 +111320 POINT(40.41510308419744 73.18193013694902) bank111320 +111321 POINT(39.957979634602346 74.99985351162654) bank111321 +111322 POINT(41.574379382050004 74.10531917113175) bank111322 +111323 POINT(40.43727316760652 73.96261221811802) bank111323 +111324 POINT(40.53672659590207 73.93592111633127) bank111324 +111325 POINT(40.621636235721404 73.98590653851267) bank111325 +111326 POINT(41.63040173771486 73.16786256929127) bank111326 +111327 POINT(41.54824725411915 73.80451145250638) bank111327 +111328 POINT(40.228326221007826 73.67278739636527) bank111328 +111329 POINT(41.580988323975866 73.10222520894078) bank111329 +111330 POINT(40.07974795184253 74.69128685740198) bank111330 +111331 POINT(40.0775395573678 74.82770126208601) bank111331 +111332 POINT(41.50948917478737 74.04537676297132) bank111332 +111333 POINT(40.923784207989634 74.45870664855268) bank111333 +111334 POINT(40.254881562498255 73.16511628942602) bank111334 +111335 POINT(40.46480622511598 73.65520105066818) bank111335 +111336 POINT(41.286218915434105 73.6468607153111) bank111336 +111337 POINT(41.45131628559594 73.58960813465502) bank111337 +111338 POINT(40.445705621613925 73.2960498960074) bank111338 +111339 POINT(39.90069514687862 73.88753759696023) bank111339 +111340 POINT(40.5501774352437 73.73157860249131) bank111340 +111341 POINT(40.39869330100182 74.11111873151192) bank111341 +111342 POINT(41.710822628968025 73.76690037031008) bank111342 +111343 POINT(40.78989351694709 74.06287676959863) bank111343 +111344 POINT(40.1991494881853 74.05315010707571) bank111344 +111345 POINT(41.15365315926637 73.79368275417073) bank111345 +111346 POINT(40.71308893624973 74.89983702751486) bank111346 +111347 POINT(40.202224977559425 74.70111812984275) bank111347 +111348 POINT(40.65996844816712 73.41986958011739) bank111348 +111349 POINT(40.46167145957414 74.01086063193844) bank111349 +111350 POINT(41.21070999904736 74.8688022056105) bank111350 +111351 POINT(40.07753218017157 73.91984303569247) bank111351 +111352 POINT(41.65184224937017 74.6498602675287) bank111352 +111353 POINT(40.682027770553674 74.05947920453072) bank111353 +111354 POINT(40.61309876719303 73.31286584811303) bank111354 +111355 POINT(39.94993636124509 74.65468316507575) bank111355 +111356 POINT(40.05472576776403 74.69697469232666) bank111356 +111357 POINT(39.92239327730095 73.30506315237639) bank111357 +111358 POINT(41.43994121636621 73.05500895589878) bank111358 +111359 POINT(39.994546903142826 74.53525004109152) bank111359 +111360 POINT(41.344377793435214 73.0695861315761) bank111360 +111361 POINT(40.88006170862397 73.51008631500154) bank111361 +111362 POINT(41.061025240124515 74.5358508043849) bank111362 +111363 POINT(41.26221391283898 74.11296695196675) bank111363 +111364 POINT(41.427931506190774 74.37558862938897) bank111364 +111365 POINT(40.288233473606425 73.80502407370008) bank111365 +111366 POINT(39.76811121410657 73.27441866564737) bank111366 +111367 POINT(40.96607472171351 74.39348823816631) bank111367 +111368 POINT(39.82764444108142 73.49981493175645) bank111368 +111369 POINT(41.512487268932524 74.80961441024802) bank111369 +111370 POINT(41.01280944426485 74.11497832058657) bank111370 +111371 POINT(40.92060946873804 74.13141351439336) bank111371 +111372 POINT(39.95704006107808 73.22487968163222) bank111372 +111373 POINT(40.893978414247385 74.94440741954539) bank111373 +111374 POINT(41.434644726757035 74.22674684895726) bank111374 +111375 POINT(40.32885880017402 74.7383420434894) bank111375 +111376 POINT(41.029370298122764 73.26567321508472) bank111376 +111377 POINT(39.73368802847312 73.30022770073495) bank111377 +111378 POINT(40.097131870266864 74.68819673991065) bank111378 +111379 POINT(40.190369721025554 73.11282394317172) bank111379 +111380 POINT(39.79738311311674 74.91971162836025) bank111380 +111381 POINT(40.39306455993384 73.75536607237206) bank111381 +111382 POINT(39.92777932656474 73.49583767580268) bank111382 +111383 POINT(40.324280543268024 74.1798637725497) bank111383 +111384 POINT(40.29970944608209 74.04596420061685) bank111384 +111385 POINT(40.659049765657436 73.15621111629588) bank111385 +111386 POINT(40.45590427060445 74.8456935191932) bank111386 +111387 POINT(40.27819923439123 74.291054065458) bank111387 +111388 POINT(41.5914355839322 74.15202231729823) bank111388 +111389 POINT(40.300168875877205 73.34117515826624) bank111389 +111390 POINT(40.812661952170096 74.01826173556067) bank111390 +111391 POINT(39.927429799446216 73.67072768935378) bank111391 +111392 POINT(41.66648002503883 73.77171953839908) bank111392 +111393 POINT(39.81352422198974 73.75640926905903) bank111393 +111394 POINT(40.36585358071719 73.93777521995341) bank111394 +111395 POINT(41.31043979790965 74.13395360948279) bank111395 +111396 POINT(40.659855698256436 73.76749109357897) bank111396 +111397 POINT(41.56691889858572 74.96896058056883) bank111397 +111398 POINT(40.118079247195084 74.27116930315012) bank111398 +111399 POINT(40.92707626005041 73.7481705848735) bank111399 +111400 POINT(40.31644416536178 74.6784251710724) bank111400 +111401 POINT(40.84439793427642 74.08626143721958) bank111401 +111402 POINT(41.38130809121467 74.63795759775631) bank111402 +111403 POINT(39.85294697599216 74.4579616147485) bank111403 +111404 POINT(40.67168462752446 74.44690135154991) bank111404 +111405 POINT(40.14919453106029 73.008400989379) bank111405 +111406 POINT(40.12049068317781 74.70192141644938) bank111406 +111407 POINT(40.20678016802566 74.95088514963436) bank111407 +111408 POINT(40.13348425143505 73.93160940497837) bank111408 +111409 POINT(40.84992700626596 74.61381024944077) bank111409 +111410 POINT(40.40005880509175 73.99087451377625) bank111410 +111411 POINT(41.08003772600756 73.85539237117042) bank111411 +111412 POINT(41.55454061969515 74.70249667926211) bank111412 +111413 POINT(40.532270517854144 73.94714597657374) bank111413 +111414 POINT(41.48008654846395 74.9222956054603) bank111414 +111415 POINT(41.05277616811252 73.60970432360871) bank111415 +111416 POINT(40.34327405939139 73.87169915266958) bank111416 +111417 POINT(41.57935648492502 74.87483202400621) bank111417 +111418 POINT(40.668838259458006 73.32945526356508) bank111418 +111419 POINT(41.69170383091235 73.49140309095955) bank111419 +111420 POINT(40.43951565407341 73.34087216635902) bank111420 +111421 POINT(39.76627776426784 74.60020405791131) bank111421 +111422 POINT(40.28790551485543 73.09036739073457) bank111422 +111423 POINT(40.69539061963638 74.13032248064856) bank111423 +111424 POINT(41.42307608757529 74.32522148737817) bank111424 +111425 POINT(41.410076522835546 74.57402551246061) bank111425 +111426 POINT(41.15574231973255 74.41761246152895) bank111426 +111427 POINT(41.095318117940174 74.60221350092282) bank111427 +111428 POINT(40.7223859080069 73.79650322875744) bank111428 +111429 POINT(40.02876695364088 73.054375583531) bank111429 +111430 POINT(40.38662631979572 73.53109222972272) bank111430 +111431 POINT(41.63609803599166 73.64531726373312) bank111431 +111432 POINT(40.343600186726235 73.91476287931751) bank111432 +111433 POINT(40.668611595940625 73.55576919300604) bank111433 +111434 POINT(39.99738605034329 73.30065700514614) bank111434 +111435 POINT(39.916168524337024 73.43604825341106) bank111435 +111436 POINT(40.96847627730263 73.3133569218552) bank111436 +111437 POINT(41.1510145102388 74.28026344948512) bank111437 +111438 POINT(41.26926771408865 73.15556552925332) bank111438 +111439 POINT(39.91628492338869 73.37269390196289) bank111439 +111440 POINT(41.13417344623732 74.27445085676025) bank111440 +111441 POINT(40.307968277281404 73.29478100755627) bank111441 +111442 POINT(41.710615701415 74.8995994996567) bank111442 +111443 POINT(40.41477580796637 74.30540573777873) bank111443 +111444 POINT(39.715893534678834 74.24334570209342) bank111444 +111445 POINT(39.85766455076498 73.73905145568013) bank111445 +111446 POINT(40.91438939200232 74.15194180554141) bank111446 +111447 POINT(39.76453339075345 73.89881939882187) bank111447 +111448 POINT(41.575483828588844 74.95592639397998) bank111448 +111449 POINT(40.63258956282879 74.91976943271978) bank111449 +111450 POINT(41.44452090396357 73.42640831115823) bank111450 +111451 POINT(41.40243003153477 73.44239387557299) bank111451 +111452 POINT(40.34391754882236 74.80255084056373) bank111452 +111453 POINT(40.3212393541414 74.69696962553866) bank111453 +111454 POINT(40.19039110997061 74.48065297021867) bank111454 +111455 POINT(40.45383264110833 73.19928616818478) bank111455 +111456 POINT(40.242052477928695 74.72254975900738) bank111456 +111457 POINT(40.3060123512455 73.82603674859539) bank111457 +111458 POINT(40.86322943194536 74.22620792249695) bank111458 +111459 POINT(39.90842678739164 74.1058705961103) bank111459 +111460 POINT(40.983650881579706 73.83566007052511) bank111460 +111461 POINT(41.01942239559838 74.29228975842715) bank111461 +111462 POINT(41.66969502363578 74.43890199056641) bank111462 +111463 POINT(40.01973614958775 73.83497656700928) bank111463 +111464 POINT(40.41563989062519 74.76296905680687) bank111464 +111465 POINT(41.525908299625115 74.65931174378976) bank111465 +111466 POINT(40.170037012231596 74.42574633930043) bank111466 +111467 POINT(41.21991258615642 73.12902157440737) bank111467 +111468 POINT(39.83377508859293 73.18191047807929) bank111468 +111469 POINT(40.43218363097686 73.57850105603173) bank111469 +111470 POINT(40.39789342970238 73.71371572572328) bank111470 +111471 POINT(40.65776985604977 74.9999179709049) bank111471 +111472 POINT(40.45364755071417 73.29107748319956) bank111472 +111473 POINT(40.240734089552525 73.48364605093185) bank111473 +111474 POINT(39.960034901309875 73.51600850372093) bank111474 +111475 POINT(40.307287957439506 74.3622339157612) bank111475 +111476 POINT(40.80087029594544 73.11812844776722) bank111476 +111477 POINT(41.42659011773109 73.34907244989883) bank111477 +111478 POINT(41.5813673207638 74.82735011021299) bank111478 +111479 POINT(39.72906408854837 73.89218211175825) bank111479 +111480 POINT(40.82661074347268 73.2965206124918) bank111480 +111481 POINT(41.32184868946433 74.61629396790163) bank111481 +111482 POINT(41.12335675644766 74.27506792188997) bank111482 +111483 POINT(41.2326972896236 74.27340169061857) bank111483 +111484 POINT(41.68123530902256 73.8308970046751) bank111484 +111485 POINT(41.520172132454604 73.58716281741448) bank111485 +111486 POINT(40.58611608113206 73.42890475209187) bank111486 +111487 POINT(41.41195575856823 73.72433187784053) bank111487 +111488 POINT(41.51419427455069 73.3896250798107) bank111488 +111489 POINT(39.98871794723469 74.64657511536234) bank111489 +111490 POINT(40.090047335128354 73.73018485433396) bank111490 +111491 POINT(39.90947807655433 74.14069145690902) bank111491 +111492 POINT(40.29932733531732 74.86801003139244) bank111492 +111493 POINT(40.86771597869237 74.58355097721484) bank111493 +111494 POINT(40.433148856149195 73.79835060116567) bank111494 +111495 POINT(40.83788419713857 74.08258332031562) bank111495 +111496 POINT(40.93704260648486 73.16605187678034) bank111496 +111497 POINT(40.59433823210181 74.35090841475335) bank111497 +111498 POINT(41.46768110406091 73.6866532918858) bank111498 +111499 POINT(40.03164803268914 74.97042791142572) bank111499 +111500 POINT(40.594264352631306 73.51235767096007) bank111500 +111501 POINT(40.70671876989466 74.39137879287973) bank111501 +111502 POINT(40.02589914510966 73.55850478701095) bank111502 +111503 POINT(41.2643304930458 73.13472452671802) bank111503 +111504 POINT(41.42736901507481 73.04452528020657) bank111504 +111505 POINT(41.63165670789017 73.61963212574146) bank111505 +111506 POINT(39.718249705927846 74.38089812189413) bank111506 +111507 POINT(41.02717054566048 73.65776070136337) bank111507 +111508 POINT(41.688426710910846 74.4816620344486) bank111508 +111509 POINT(39.865475961473315 74.12907162380223) bank111509 +111510 POINT(40.780018908538075 74.62731843793675) bank111510 +111511 POINT(40.03709231868434 73.61356723113002) bank111511 +111512 POINT(41.514933712473 74.01902245001486) bank111512 +111513 POINT(40.451826085604345 73.66288440657466) bank111513 +111514 POINT(41.538412655850976 74.95056749791148) bank111514 +111515 POINT(40.11000770232437 73.58365428684857) bank111515 +111516 POINT(40.136695707903336 74.90700903182318) bank111516 +111517 POINT(40.988029918044546 74.96721382646382) bank111517 +111518 POINT(41.57803078570344 73.23675634274734) bank111518 +111519 POINT(40.18986091912347 73.20199799797638) bank111519 +111520 POINT(40.77651068469483 74.19106348685587) bank111520 +111521 POINT(39.76906326968433 74.64777730627999) bank111521 +111522 POINT(40.15852711154057 73.99893921772598) bank111522 +111523 POINT(41.02395222676036 73.47842900150744) bank111523 +111524 POINT(41.44536264981622 73.71582686758936) bank111524 +111525 POINT(40.12163536025586 74.88639256826013) bank111525 +111526 POINT(39.92916274916403 74.43996967688611) bank111526 +111527 POINT(40.591619245614154 73.7199625354466) bank111527 +111528 POINT(40.74396075509856 74.25079363257292) bank111528 +111529 POINT(40.76678361175076 74.75518239186542) bank111529 +111530 POINT(39.7508722959645 73.20199236020395) bank111530 +111531 POINT(41.066359104961236 74.42615843182634) bank111531 +111532 POINT(40.50097397813342 73.3491400496719) bank111532 +111533 POINT(40.118765719803605 73.67759773345085) bank111533 +111534 POINT(40.47238911420798 74.16113022636453) bank111534 +111535 POINT(40.04246406110533 73.01650367457127) bank111535 +111536 POINT(40.37998891839889 74.85736106502289) bank111536 +111537 POINT(40.583882275027264 74.01281565805576) bank111537 +111538 POINT(40.18896735260709 74.50124757273933) bank111538 +111539 POINT(41.323702404948186 74.5819471470252) bank111539 +111540 POINT(41.59868241454371 74.9648484978308) bank111540 +111541 POINT(41.0982596616138 74.75402554767929) bank111541 +111542 POINT(41.05299368496219 73.63559237846972) bank111542 +111543 POINT(40.899095348675765 73.78922118353452) bank111543 +111544 POINT(40.184066588225335 73.81166722763594) bank111544 +111545 POINT(40.23719533511657 73.9638200664885) bank111545 +111546 POINT(40.28457601585555 74.00141595161584) bank111546 +111547 POINT(41.66494323769929 74.95438347371315) bank111547 +111548 POINT(40.18326771908198 74.5959838433717) bank111548 +111549 POINT(40.73943681500452 74.42462596925962) bank111549 +111550 POINT(40.27189001286708 73.65653373006964) bank111550 +111551 POINT(39.79049932510672 74.70263194928329) bank111551 +111552 POINT(40.31187188908552 74.57897923593441) bank111552 +111553 POINT(41.0152633622321 74.42866562235726) bank111553 +111554 POINT(41.09925156482179 74.46777457266522) bank111554 +111555 POINT(40.67076969175089 74.67964972513604) bank111555 +111556 POINT(40.27974388457219 73.05828264274308) bank111556 +111557 POINT(39.89819672744668 73.38961957208035) bank111557 +111558 POINT(39.89242252812661 74.72244089121432) bank111558 +111559 POINT(40.85193048135933 74.97362235535691) bank111559 +111560 POINT(39.830131435494344 73.85366257415603) bank111560 +111561 POINT(39.83077008663462 74.06768657809405) bank111561 +111562 POINT(40.51339789166779 73.70033441511113) bank111562 +111563 POINT(41.33846837592118 74.64505147427822) bank111563 +111564 POINT(41.64701345845818 73.96647190226574) bank111564 +111565 POINT(41.555058730935286 73.1351698511767) bank111565 +111566 POINT(41.50826885822274 74.0005834833058) bank111566 +111567 POINT(41.15835547407733 74.41515013071017) bank111567 +111568 POINT(40.38364744555793 74.63357378782969) bank111568 +111569 POINT(40.59527726630465 73.19808980220296) bank111569 +111570 POINT(41.25181541621695 73.50463924976292) bank111570 +111571 POINT(41.37376131580185 74.1937524633707) bank111571 +111572 POINT(41.48150373347071 73.52546818178888) bank111572 +111573 POINT(40.19630160179013 73.22366533326736) bank111573 +111574 POINT(40.891206733719585 73.17559952157274) bank111574 +111575 POINT(40.93491283390402 73.59331954077359) bank111575 +111576 POINT(39.78993310954938 74.76953143501657) bank111576 +111577 POINT(40.12185055244569 74.8714751513118) bank111577 +111578 POINT(41.56131124937464 74.91240951973266) bank111578 +111579 POINT(40.03966995175412 73.40545801119855) bank111579 +111580 POINT(40.82466302838242 73.99785209122943) bank111580 +111581 POINT(41.22590817307133 73.75110928764443) bank111581 +111582 POINT(41.67419717887734 73.34444836196995) bank111582 +111583 POINT(40.16458192567485 74.47126824300099) bank111583 +111584 POINT(39.97029417042431 73.21988691469427) bank111584 +111585 POINT(41.553508186956606 73.16463797605823) bank111585 +111586 POINT(40.40762903272087 73.46697317866794) bank111586 +111587 POINT(40.56027189872777 74.32101542912245) bank111587 +111588 POINT(41.57823449968262 73.26738669957167) bank111588 +111589 POINT(40.19266098658457 74.4227316130735) bank111589 +111590 POINT(40.10372654760543 74.9468929570825) bank111590 +111591 POINT(40.94199348408279 74.685015320682) bank111591 +111592 POINT(40.16386484301503 73.48861642343111) bank111592 +111593 POINT(41.42522080958655 74.81539285227) bank111593 +111594 POINT(40.46810265491122 74.77875851583987) bank111594 +111595 POINT(41.0108692692814 74.70314910486316) bank111595 +111596 POINT(40.30040228331164 73.07017459650986) bank111596 +111597 POINT(40.879423497330585 74.3340223051752) bank111597 +111598 POINT(41.11586775707576 74.55185722917041) bank111598 +111599 POINT(41.18929678046353 73.02318349588643) bank111599 +111600 POINT(41.30055881570377 74.36388201918724) bank111600 +111601 POINT(40.29566175981501 74.698122107247) bank111601 +111602 POINT(40.79892887493584 73.87178019438012) bank111602 +111603 POINT(41.30871561776972 74.14817925768858) bank111603 +111604 POINT(41.57988733736187 74.51292543157629) bank111604 +111605 POINT(40.10851285774841 74.61668403672024) bank111605 +111606 POINT(39.869097957170204 74.36029556089657) bank111606 +111607 POINT(41.48201869553812 74.0356897760401) bank111607 +111608 POINT(40.60380034722016 74.84572091042142) bank111608 +111609 POINT(40.19028327475463 74.378535529587) bank111609 +111610 POINT(39.83460447594953 73.84361794480813) bank111610 +111611 POINT(40.52561277470958 73.44957428267264) bank111611 +111612 POINT(41.65285431845797 74.5893640697344) bank111612 +111613 POINT(40.55374377034368 74.58567275758442) bank111613 +111614 POINT(40.68564026485213 74.40831708896928) bank111614 +111615 POINT(40.679306480602655 73.63087041084817) bank111615 +111616 POINT(41.6911797609818 73.93478864801335) bank111616 +111617 POINT(40.993625224700715 73.74365405128958) bank111617 +111618 POINT(40.61935400297868 74.77677205408119) bank111618 +111619 POINT(40.74883320893916 74.02818183303337) bank111619 +111620 POINT(39.84947867687024 73.0562813859219) bank111620 +111621 POINT(41.48444655464993 74.72178059597331) bank111621 +111622 POINT(41.46786352168176 73.99905403174905) bank111622 +111623 POINT(40.862130011818905 73.93851995032179) bank111623 +111624 POINT(39.985085327294556 73.63791862613193) bank111624 +111625 POINT(41.58788806210274 73.1342820501524) bank111625 +111626 POINT(40.61033940700415 73.15225678273076) bank111626 +111627 POINT(40.12455141934188 73.58227193902783) bank111627 +111628 POINT(40.954167534471445 73.14037349650611) bank111628 +111629 POINT(41.296950923742045 73.29347648726457) bank111629 +111630 POINT(41.30521646041396 73.58518797180298) bank111630 +111631 POINT(40.512132422074316 74.37191008518201) bank111631 +111632 POINT(41.03908582301585 74.5808669419994) bank111632 +111633 POINT(40.58726764712571 74.29801385570175) bank111633 +111634 POINT(40.88225477016408 74.14744058892384) bank111634 +111635 POINT(41.272203665609865 73.06700569575857) bank111635 +111636 POINT(41.52664276348087 73.45712173837597) bank111636 +111637 POINT(40.88834854647521 74.75183224962548) bank111637 +111638 POINT(41.322863791549345 73.18421736548994) bank111638 +111639 POINT(40.46684058534155 73.39216097802469) bank111639 +111640 POINT(39.90790510583506 74.04061271303738) bank111640 +111641 POINT(39.92393727770674 74.23920210609435) bank111641 +111642 POINT(40.77707608717505 73.7486114277388) bank111642 +111643 POINT(41.56694314062817 73.92995473156957) bank111643 +111644 POINT(41.64392774933255 73.38455305961249) bank111644 +111645 POINT(40.815601129320946 73.26705664691573) bank111645 +111646 POINT(39.72739221366695 74.06755148052088) bank111646 +111647 POINT(39.92963181010199 74.5341258754139) bank111647 +111648 POINT(40.371842654901975 73.47542339784705) bank111648 +111649 POINT(39.96766980621966 73.9889489483654) bank111649 +111650 POINT(40.48579105058417 73.41268293079048) bank111650 +111651 POINT(40.81912438609017 74.51632839998284) bank111651 +111652 POINT(40.56359298638105 74.28928165802994) bank111652 +111653 POINT(41.157069001181235 73.8588947231456) bank111653 +111654 POINT(41.65047027975227 74.30634449428464) bank111654 +111655 POINT(41.362387085552456 74.12212754124577) bank111655 +111656 POINT(40.36972637666276 73.51501049569177) bank111656 +111657 POINT(41.496866771289504 73.24308339270013) bank111657 +111658 POINT(41.00837435702942 73.5178528409576) bank111658 +111659 POINT(41.00130927540347 73.07917526882346) bank111659 +111660 POINT(41.084713044600434 74.77405143641066) bank111660 +111661 POINT(41.13532847366227 74.09605761353554) bank111661 +111662 POINT(40.85751028167681 74.01338396775031) bank111662 +111663 POINT(40.97061818029183 73.78846123045957) bank111663 +111664 POINT(41.148434304533616 74.41387517846844) bank111664 +111665 POINT(41.003690495114014 73.79336926820166) bank111665 +111666 POINT(41.001607317085224 73.87922747949182) bank111666 +111667 POINT(40.40258986963257 73.49206330730894) bank111667 +111668 POINT(40.13386553835862 73.27401904306717) bank111668 +111669 POINT(40.94020344810364 73.09542174468757) bank111669 +111670 POINT(40.79883518308777 74.65192658575758) bank111670 +111671 POINT(39.845548135975356 74.76349360887039) bank111671 +111672 POINT(41.02269787105039 74.6754078975885) bank111672 +111673 POINT(40.292497559087984 74.99021468134805) bank111673 +111674 POINT(40.19361713228078 74.84383850024675) bank111674 +111675 POINT(39.724571626304765 74.41685430976003) bank111675 +111676 POINT(39.785211983198906 73.01532923783171) bank111676 +111677 POINT(41.45699319813924 74.40913883813737) bank111677 +111678 POINT(41.14899476757229 73.09966654932494) bank111678 +111679 POINT(40.90074476073437 73.79186126227883) bank111679 +111680 POINT(41.60353732927152 73.22720829371774) bank111680 +111681 POINT(40.7483045484489 74.99411582077411) bank111681 +111682 POINT(40.439618574774435 74.95427391338598) bank111682 +111683 POINT(40.451769831860354 73.41120494240529) bank111683 +111684 POINT(40.129010967910546 73.10797131675079) bank111684 +111685 POINT(39.89363295227336 73.57299441818128) bank111685 +111686 POINT(39.82711036041553 73.37653581482165) bank111686 +111687 POINT(41.59175478702321 73.91050924889561) bank111687 +111688 POINT(40.228787022287285 74.62620679016935) bank111688 +111689 POINT(40.87993369609165 73.81327632024585) bank111689 +111690 POINT(41.28846018522466 74.79907555977594) bank111690 +111691 POINT(41.3627703950383 73.71688436646846) bank111691 +111692 POINT(41.00046132020565 73.54128491388052) bank111692 +111693 POINT(39.74803898787977 74.25921310897238) bank111693 +111694 POINT(39.71820358497927 74.51704589644022) bank111694 +111695 POINT(41.2491937077417 73.02255956742121) bank111695 +111696 POINT(41.60448048783843 73.89617063563627) bank111696 +111697 POINT(40.80367167199226 74.65496374074496) bank111697 +111698 POINT(40.42443653401778 73.04243730349012) bank111698 +111699 POINT(40.50659426768207 73.71705064021234) bank111699 +111700 POINT(41.64726968384258 74.41049489741644) bank111700 +111701 POINT(41.437859450713674 73.19217220415815) bank111701 +111702 POINT(40.588433238266326 73.31642102273268) bank111702 +111703 POINT(40.26044515948455 74.76552986086826) bank111703 +111704 POINT(40.93812445830359 74.28455704749834) bank111704 +111705 POINT(40.245807258226925 73.32805912775294) bank111705 +111706 POINT(41.220963443002965 73.16745183729645) bank111706 +111707 POINT(40.88720166867293 73.32090460360399) bank111707 +111708 POINT(40.96003601202843 73.29666193956285) bank111708 +111709 POINT(41.32332578539894 74.54304622877483) bank111709 +111710 POINT(41.12553256373477 74.64622540640491) bank111710 +111711 POINT(40.66269931478955 74.31746939029493) bank111711 +111712 POINT(40.254375988699 74.6926309072002) bank111712 +111713 POINT(41.4094387599326 73.1005503600735) bank111713 +111714 POINT(41.691596579500995 74.86808436818004) bank111714 +111715 POINT(41.383340793433455 73.59076620897355) bank111715 +111716 POINT(40.66892767625585 73.7106702946938) bank111716 +111717 POINT(40.01122092086087 74.95542194180004) bank111717 +111718 POINT(39.774719263843124 73.62625594687091) bank111718 +111719 POINT(40.86906028685648 73.82939917946631) bank111719 +111720 POINT(41.52038168010021 74.00648900379912) bank111720 +111721 POINT(41.088163499862375 74.8203601037444) bank111721 +111722 POINT(40.33336280180735 74.59008874870344) bank111722 +111723 POINT(39.92813771516736 74.39367022036457) bank111723 +111724 POINT(40.84300448109191 73.56078514561983) bank111724 +111725 POINT(40.42813252589059 74.92099967459107) bank111725 +111726 POINT(41.26699658148882 73.67693704270268) bank111726 +111727 POINT(40.11048318839609 73.98100905760275) bank111727 +111728 POINT(40.49441202382679 74.61287528638091) bank111728 +111729 POINT(40.38260729132349 74.99916720427032) bank111729 +111730 POINT(41.04780827057448 74.47525207497623) bank111730 +111731 POINT(41.652122450491404 73.76583094662602) bank111731 +111732 POINT(40.12470610611739 73.2779804927327) bank111732 +111733 POINT(41.15245841082726 73.5291908904002) bank111733 +111734 POINT(40.85387669790106 73.5546013728334) bank111734 +111735 POINT(41.25786925529288 74.2159061953166) bank111735 +111736 POINT(41.55672011826634 73.02525178580612) bank111736 +111737 POINT(41.239676708846744 73.17019155761555) bank111737 +111738 POINT(40.6218424755437 73.09514230270824) bank111738 +111739 POINT(40.51697295231478 74.84181343537071) bank111739 +111740 POINT(40.155816634743104 73.43199365014257) bank111740 +111741 POINT(41.62164198653055 74.15233330214558) bank111741 +111742 POINT(40.234990517467516 74.09551897327225) bank111742 +111743 POINT(40.3652980007906 73.27873539752687) bank111743 +111744 POINT(39.836597858948934 73.35726839091208) bank111744 +111745 POINT(41.437006238829106 74.0916142946727) bank111745 +111746 POINT(40.74111683370443 73.7347823886628) bank111746 +111747 POINT(41.00086634110048 73.37661476486089) bank111747 +111748 POINT(41.70168119036208 73.81911505264323) bank111748 +111749 POINT(40.31540251335667 74.49548494134544) bank111749 +111750 POINT(41.64067488519512 74.45351710318474) bank111750 +111751 POINT(41.33596367480119 74.60287561318643) bank111751 +111752 POINT(41.409334881805464 73.18799547088959) bank111752 +111753 POINT(39.87837568569382 74.04952177099868) bank111753 +111754 POINT(39.930995319542966 73.16985922253166) bank111754 +111755 POINT(39.97993061849087 73.24392427544522) bank111755 +111756 POINT(40.56151468317619 74.52170613209762) bank111756 +111757 POINT(39.84669661809573 73.53635495005177) bank111757 +111758 POINT(39.92020377099836 73.11088673532167) bank111758 +111759 POINT(41.29587260402543 73.89395363805555) bank111759 +111760 POINT(41.36957487722538 74.9123162713753) bank111760 +111761 POINT(40.14482273537734 74.0182735928783) bank111761 +111762 POINT(41.00540370685563 74.86180547268211) bank111762 +111763 POINT(40.7814452873462 74.44243665353062) bank111763 +111764 POINT(40.04131858494306 73.81479594539351) bank111764 +111765 POINT(39.915047614444205 74.16422107510816) bank111765 +111766 POINT(41.20245812848443 73.69182715463619) bank111766 +111767 POINT(40.57921870768985 73.53799873405377) bank111767 +111768 POINT(40.63260732258363 73.11318538823285) bank111768 +111769 POINT(40.889546445449746 73.44149046753236) bank111769 +111770 POINT(39.94883714368428 74.19567691042127) bank111770 +111771 POINT(40.12301786131054 73.93670278307346) bank111771 +111772 POINT(39.9297545937693 74.22897165153407) bank111772 +111773 POINT(41.646008948352446 74.35103867665559) bank111773 +111774 POINT(41.589602413866494 74.84754027204862) bank111774 +111775 POINT(40.70842377468079 73.58880534762858) bank111775 +111776 POINT(39.779696212328304 73.5163372858756) bank111776 +111777 POINT(40.97488273453648 73.67162769422589) bank111777 +111778 POINT(41.04867945136162 73.26435688458737) bank111778 +111779 POINT(41.336447872503726 74.77257526763458) bank111779 +111780 POINT(40.40138779915787 74.30336566658133) bank111780 +111781 POINT(41.01380355151459 74.58870397604234) bank111781 +111782 POINT(41.57213877979985 73.34614896636498) bank111782 +111783 POINT(41.454374005662324 73.22614663130062) bank111783 +111784 POINT(41.542991759172125 74.51245621401895) bank111784 +111785 POINT(41.249128529874085 74.003889950212) bank111785 +111786 POINT(41.36729871089609 74.59094326543867) bank111786 +111787 POINT(40.16523400740425 74.04365524266215) bank111787 +111788 POINT(40.51392439908253 73.50594412967332) bank111788 +111789 POINT(40.709782552847116 74.44790799934519) bank111789 +111790 POINT(39.97494520536016 73.12517851506269) bank111790 +111791 POINT(39.943793251510975 73.27806422186315) bank111791 +111792 POINT(41.43074505126939 74.19250089055488) bank111792 +111793 POINT(39.97710040402602 73.02813644026745) bank111793 +111794 POINT(40.88928418073377 74.84372449983819) bank111794 +111795 POINT(40.50832356729427 74.01011068268649) bank111795 +111796 POINT(40.80900719266838 74.00139634476585) bank111796 +111797 POINT(40.50070115319816 73.95956061215941) bank111797 +111798 POINT(40.64469245310402 74.447070918149) bank111798 +111799 POINT(40.45848186531336 73.02300285969183) bank111799 +111800 POINT(40.12035751969672 74.59784374505311) bank111800 +111801 POINT(39.75295859141827 74.17851134646837) bank111801 +111802 POINT(41.3572292825252 73.15749559939368) bank111802 +111803 POINT(40.39792647268188 73.06996025733916) bank111803 +111804 POINT(41.64899311525341 73.33846808802033) bank111804 +111805 POINT(40.76763950571974 73.25760705010522) bank111805 +111806 POINT(40.48595815554347 74.09461484596505) bank111806 +111807 POINT(41.38116046005754 73.29692977581553) bank111807 +111808 POINT(41.32960756873315 74.91377642248874) bank111808 +111809 POINT(40.912746189906194 73.88654992149688) bank111809 +111810 POINT(41.14334237146331 73.33569760540533) bank111810 +111811 POINT(40.80932702329883 73.78778712536675) bank111811 +111812 POINT(40.55034471451235 74.94434246211209) bank111812 +111813 POINT(40.97561791445287 73.83086473423141) bank111813 +111814 POINT(40.13668379104355 74.5255185056255) bank111814 +111815 POINT(40.50437186556285 73.79010534655744) bank111815 +111816 POINT(40.46658085330899 73.13787831048334) bank111816 +111817 POINT(39.80372400433603 74.89201299303625) bank111817 +111818 POINT(41.60005693382816 73.56382630954393) bank111818 +111819 POINT(40.73468050962435 73.0825792235868) bank111819 +111820 POINT(40.60591637679547 74.94313865359166) bank111820 +111821 POINT(40.65558551796908 74.41694308975066) bank111821 +111822 POINT(40.02755507930047 73.97463836570623) bank111822 +111823 POINT(40.95926148081792 74.63920746111123) bank111823 +111824 POINT(41.328509339843706 73.76107516660065) bank111824 +111825 POINT(40.64232590146206 73.86308613475356) bank111825 +111826 POINT(40.226206039439106 73.58602909464618) bank111826 +111827 POINT(40.68110880379602 73.40391460921681) bank111827 +111828 POINT(40.85917224737834 74.25586845451335) bank111828 +111829 POINT(40.67783563628491 73.69992654577253) bank111829 +111830 POINT(41.62030222203106 74.16727840792963) bank111830 +111831 POINT(41.685920588979826 73.91139781721769) bank111831 +111832 POINT(39.827457072590704 74.85358544816329) bank111832 +111833 POINT(40.75932351174835 73.2051671925171) bank111833 +111834 POINT(41.28472649986557 73.88765773502135) bank111834 +111835 POINT(41.04597053235491 73.68721726689387) bank111835 +111836 POINT(41.57419204588505 73.5468034750435) bank111836 +111837 POINT(41.33739757577062 74.06017377546061) bank111837 +111838 POINT(39.95506772235027 74.4297030154907) bank111838 +111839 POINT(40.75922182215718 74.74290694280629) bank111839 +111840 POINT(41.69704955123902 73.43292949984621) bank111840 +111841 POINT(40.720411506161625 74.98708063063482) bank111841 +111842 POINT(40.407417290496554 73.37021682651961) bank111842 +111843 POINT(41.081534217134624 73.52853863930069) bank111843 +111844 POINT(41.1049483333458 74.30961381860264) bank111844 +111845 POINT(41.54676601855404 73.00935271113656) bank111845 +111846 POINT(41.69074998902323 74.8879171069802) bank111846 +111847 POINT(41.23989369379451 74.22047260999285) bank111847 +111848 POINT(41.060386099092064 73.52035264274333) bank111848 +111849 POINT(40.1548043152965 74.2814464401308) bank111849 +111850 POINT(39.96709895739563 74.73065305471839) bank111850 +111851 POINT(40.36074447558332 73.39313785974666) bank111851 +111852 POINT(40.96611340672303 73.88673761458384) bank111852 +111853 POINT(39.7393287790745 73.46113382133139) bank111853 +111854 POINT(40.81212244107544 74.91804933908998) bank111854 +111855 POINT(40.02584348086671 74.30094590866881) bank111855 +111856 POINT(39.90432592699774 73.79031302537601) bank111856 +111857 POINT(40.18454903386363 74.53861801902661) bank111857 +111858 POINT(40.60730026542256 74.40402544767024) bank111858 +111859 POINT(40.43771390266736 73.6187959439418) bank111859 +111860 POINT(40.638154553626556 73.9322961010844) bank111860 +111861 POINT(39.99458404670293 74.65467822496561) bank111861 +111862 POINT(40.06421833738958 73.75653489889265) bank111862 +111863 POINT(41.361885971634244 73.39016174504174) bank111863 +111864 POINT(41.17883350150434 73.4645483139773) bank111864 +111865 POINT(40.08677717936517 73.18684346033699) bank111865 +111866 POINT(40.8766678413294 73.74184729132885) bank111866 +111867 POINT(41.419116668237834 73.64045184874799) bank111867 +111868 POINT(41.5595383063465 73.405361048461) bank111868 +111869 POINT(41.34736381231284 73.17837910216016) bank111869 +111870 POINT(40.813362060560266 74.72758807909179) bank111870 +111871 POINT(40.19034437613904 74.29403126943902) bank111871 +111872 POINT(40.182335596170134 73.09045317217411) bank111872 +111873 POINT(41.022654944486874 73.4396293927798) bank111873 +111874 POINT(41.260028283856904 73.03646914481256) bank111874 +111875 POINT(40.53144535411079 74.39902922759049) bank111875 +111876 POINT(41.132864721361166 73.98051519731322) bank111876 +111877 POINT(41.18358185315604 73.93111958562503) bank111877 +111878 POINT(40.8472672212397 73.72749848598104) bank111878 +111879 POINT(41.2650481853601 73.22391689979258) bank111879 +111880 POINT(41.065324777751265 74.59835265438623) bank111880 +111881 POINT(41.3270624798342 73.50428323174656) bank111881 +111882 POINT(40.079247930642026 74.59391140480774) bank111882 +111883 POINT(40.64612322143566 73.32576609503367) bank111883 +111884 POINT(41.11684157086556 73.69675905862583) bank111884 +111885 POINT(40.56038874596652 74.23789866930875) bank111885 +111886 POINT(41.685769193054625 74.53681461847897) bank111886 +111887 POINT(40.73320847322072 73.80537376775847) bank111887 +111888 POINT(40.98400563397226 74.87433495305983) bank111888 +111889 POINT(41.69523800966042 73.5919837927404) bank111889 +111890 POINT(40.357412674456306 73.77326988314208) bank111890 +111891 POINT(39.771518444774046 73.53744429153038) bank111891 +111892 POINT(41.65590468868272 73.63989380633649) bank111892 +111893 POINT(40.58104556831737 73.17595758025372) bank111893 +111894 POINT(40.657684921441096 74.06954270419753) bank111894 +111895 POINT(40.74568850557927 74.86373875319143) bank111895 +111896 POINT(40.886149564720085 74.3670196042568) bank111896 +111897 POINT(41.23416775027202 74.4819281036407) bank111897 +111898 POINT(40.537941301055916 74.83147007017982) bank111898 +111899 POINT(41.115651213941184 73.66880563993475) bank111899 +111900 POINT(41.22617989293048 73.1567736950752) bank111900 +111901 POINT(40.033915011930816 74.11857625396283) bank111901 +111902 POINT(41.60415876833676 73.44512876073597) bank111902 +111903 POINT(40.889695974341876 73.97804413427046) bank111903 +111904 POINT(41.33153050347764 74.20298287295844) bank111904 +111905 POINT(41.256024881324265 73.51201499519001) bank111905 +111906 POINT(40.3798687135551 74.8853086814264) bank111906 +111907 POINT(40.70049714549427 74.66774824874405) bank111907 +111908 POINT(40.232569716490346 73.04946696352756) bank111908 +111909 POINT(39.86088275609833 74.46327411727786) bank111909 +111910 POINT(40.83933298231426 74.05317184808756) bank111910 +111911 POINT(40.11505585714458 73.49781260446512) bank111911 +111912 POINT(40.483258400233396 73.3010471510671) bank111912 +111913 POINT(39.8150533216141 73.63902035332492) bank111913 +111914 POINT(40.82273954641448 73.64694710380378) bank111914 +111915 POINT(40.067486342109675 74.80721667619915) bank111915 +111916 POINT(40.02244938053992 74.90748598599427) bank111916 +111917 POINT(40.17782311976134 74.6530643986746) bank111917 +111918 POINT(40.43595058213328 74.61154683089183) bank111918 +111919 POINT(40.23108804497215 73.54742127562768) bank111919 +111920 POINT(41.66916907649167 73.65143741046234) bank111920 +111921 POINT(39.84669314555714 74.2323417847511) bank111921 +111922 POINT(41.13580726864336 73.49149527988723) bank111922 +111923 POINT(39.849588718718714 74.27590199670847) bank111923 +111924 POINT(40.152188970653675 73.57741933200289) bank111924 +111925 POINT(41.074327308957926 73.33148976436999) bank111925 +111926 POINT(41.14008564077684 73.28308933256112) bank111926 +111927 POINT(39.97581753996951 73.92759680037935) bank111927 +111928 POINT(39.86443757766389 73.43679764390639) bank111928 +111929 POINT(40.622712530235276 74.56205738795464) bank111929 +111930 POINT(40.530633140726955 74.73379703664405) bank111930 +111931 POINT(41.429305320154434 73.35976595805377) bank111931 +111932 POINT(40.87794947367573 73.9128156381673) bank111932 +111933 POINT(40.18592731169259 74.69061579818134) bank111933 +111934 POINT(40.45548699550392 74.25236247869786) bank111934 +111935 POINT(40.596646045523556 73.28921537417781) bank111935 +111936 POINT(40.74703587304522 74.26105773425095) bank111936 +111937 POINT(40.5901642378752 73.61304091774184) bank111937 +111938 POINT(41.48546197293341 73.89983434907249) bank111938 +111939 POINT(40.533783716054636 73.74843447337366) bank111939 +111940 POINT(40.96779133741096 74.65295430698606) bank111940 +111941 POINT(41.332173362949185 74.45162647648105) bank111941 +111942 POINT(40.41491509372578 73.21979015030804) bank111942 +111943 POINT(40.84611276139059 74.67268535242812) bank111943 +111944 POINT(39.717941553225266 73.89793466129375) bank111944 +111945 POINT(41.34617927355506 73.97222859415837) bank111945 +111946 POINT(39.84960651899701 74.11878051926482) bank111946 +111947 POINT(41.04946111068837 74.75112003521195) bank111947 +111948 POINT(40.938992312134665 73.71605644977637) bank111948 +111949 POINT(40.071533918131905 73.35275880614564) bank111949 +111950 POINT(40.645393811103624 74.89547538894398) bank111950 +111951 POINT(40.4841164043604 74.05380675160202) bank111951 +111952 POINT(41.376678060511296 73.70458546846436) bank111952 +111953 POINT(41.13767658319192 74.42105295350487) bank111953 +111954 POINT(40.63894012423112 74.32493408978569) bank111954 +111955 POINT(40.07437934727696 74.72433316371921) bank111955 +111956 POINT(39.76453395108399 73.19001276484065) bank111956 +111957 POINT(40.180828926030294 73.31987710471195) bank111957 +111958 POINT(41.66319001660922 74.15944979613973) bank111958 +111959 POINT(41.28557730659685 74.7811797938074) bank111959 +111960 POINT(41.00336199430763 74.71689100089559) bank111960 +111961 POINT(40.5086279863337 74.42669004041406) bank111961 +111962 POINT(41.20861803143778 73.25578826148276) bank111962 +111963 POINT(39.83346299130524 73.33827278553801) bank111963 +111964 POINT(41.04661026487422 73.07249720898618) bank111964 +111965 POINT(39.95409096914782 74.12332311759504) bank111965 +111966 POINT(41.42536425634177 73.47049701631363) bank111966 +111967 POINT(40.50374795738852 73.36336587047892) bank111967 +111968 POINT(40.27117721635885 73.94460056657663) bank111968 +111969 POINT(40.27269284392248 74.33658432527645) bank111969 +111970 POINT(39.81132130860336 73.81294237665233) bank111970 +111971 POINT(41.57971748882259 74.62803101923971) bank111971 +111972 POINT(41.22271384593032 74.89981434212024) bank111972 +111973 POINT(40.802589259863865 73.69451446200664) bank111973 +111974 POINT(39.88826545639605 74.87414158505395) bank111974 +111975 POINT(39.94543333161426 74.79489683137624) bank111975 +111976 POINT(41.534621649203046 74.27260512701625) bank111976 +111977 POINT(41.59853393585922 74.95421413740684) bank111977 +111978 POINT(40.92775615986789 73.20122710350877) bank111978 +111979 POINT(39.82529361311373 74.85332425853031) bank111979 +111980 POINT(41.255496171316885 74.47317064637325) bank111980 +111981 POINT(41.538187298940315 73.8205297106453) bank111981 +111982 POINT(39.935852720402906 74.01275222431813) bank111982 +111983 POINT(39.81019575307429 73.02153171805043) bank111983 +111984 POINT(40.56433482094407 74.58041486667003) bank111984 +111985 POINT(41.351496881990734 73.10955074096198) bank111985 +111986 POINT(39.808458866575506 73.00898622001613) bank111986 +111987 POINT(41.578291591783255 74.248735867258) bank111987 +111988 POINT(41.25896414141615 73.6377517673825) bank111988 +111989 POINT(40.41537967338599 73.9001452934606) bank111989 +111990 POINT(40.575025708674495 74.52546248700783) bank111990 +111991 POINT(40.893074644099094 74.38330682908695) bank111991 +111992 POINT(41.28196181162531 74.71625927830985) bank111992 +111993 POINT(40.19118969286031 74.8951083348502) bank111993 +111994 POINT(41.51743863347626 74.5634223239413) bank111994 +111995 POINT(41.20366418720518 73.43680145031804) bank111995 +111996 POINT(41.357076361694574 74.82640129051246) bank111996 +111997 POINT(39.8302720650527 75.00526200131908) bank111997 +111998 POINT(40.09668671077074 73.24143384807546) bank111998 +111999 POINT(41.65282960961657 74.55735917805949) bank111999 +112000 POINT(40.63854953765974 73.74056324959191) bank112000 +112001 POINT(40.32697672559082 74.3153500928539) bank112001 +112002 POINT(39.87051133777432 74.6404912486782) bank112002 +112003 POINT(41.04097697188836 74.59796744497821) bank112003 +112004 POINT(40.235059349322036 74.6534789697211) bank112004 +112005 POINT(41.44152566950284 74.45100444242074) bank112005 +112006 POINT(40.46502895054833 73.05438925917039) bank112006 +112007 POINT(40.144543266155104 73.95370957847098) bank112007 +112008 POINT(40.93911007536085 73.7005858630391) bank112008 +112009 POINT(41.03116692925224 74.53231325644465) bank112009 +112010 POINT(41.698779420393464 74.0856342331718) bank112010 +112011 POINT(39.77311647800258 73.43656741322118) bank112011 +112012 POINT(39.7492280924059 73.49084575796854) bank112012 +112013 POINT(41.34124774192517 74.90948281174784) bank112013 +112014 POINT(39.7164757955219 74.11721959481484) bank112014 +112015 POINT(41.09585445312761 74.35668058226435) bank112015 +112016 POINT(40.95314410886905 74.1780401165545) bank112016 +112017 POINT(41.61655635274611 74.48444941200206) bank112017 +112018 POINT(40.990246800501026 73.15847840896473) bank112018 +112019 POINT(40.6865685033736 73.23824916937427) bank112019 +112020 POINT(40.2223013578109 74.88579459845941) bank112020 +112021 POINT(41.137536849857334 73.45554762687699) bank112021 +112022 POINT(39.760213389062486 73.90100196710135) bank112022 +112023 POINT(40.01854361129632 73.58053297419487) bank112023 +112024 POINT(41.6686880453022 73.1156743036457) bank112024 +112025 POINT(39.867246973032195 73.41077414760967) bank112025 +112026 POINT(40.95612485613605 74.89583438392033) bank112026 +112027 POINT(40.999414683700266 74.62515281980156) bank112027 +112028 POINT(40.60445351108429 73.63214144746308) bank112028 +112029 POINT(41.6933903543708 74.86523157455363) bank112029 +112030 POINT(41.23249188032885 74.67061292314916) bank112030 +112031 POINT(41.10466900370646 73.69571343659196) bank112031 +112032 POINT(41.05466159621113 73.87961074283841) bank112032 +112033 POINT(41.29883175946355 73.69222018511755) bank112033 +112034 POINT(39.815421407863866 74.39184261711065) bank112034 +112035 POINT(39.72516602690778 73.1525408357786) bank112035 +112036 POINT(40.98582573986047 74.4531707760117) bank112036 +112037 POINT(39.96185096613391 74.79545115491231) bank112037 +112038 POINT(40.807911870127306 73.36231197079536) bank112038 +112039 POINT(41.56795884376753 73.20098528582145) bank112039 +112040 POINT(41.658312515234506 74.74329358332753) bank112040 +112041 POINT(40.56196327425112 74.82462750351166) bank112041 +112042 POINT(41.0673312269192 73.99490493010096) bank112042 +112043 POINT(41.33113076306286 74.36899087397619) bank112043 +112044 POINT(40.95497851567072 74.36445314511543) bank112044 +112045 POINT(40.2911632081768 73.73512875504674) bank112045 +112046 POINT(40.15018195696906 73.34510746637822) bank112046 +112047 POINT(40.102846620108096 74.37726237743821) bank112047 +112048 POINT(41.552717445728106 73.43773122031546) bank112048 +112049 POINT(39.936112115858236 73.27044506021848) bank112049 +112050 POINT(39.995325816170755 73.09791854137545) bank112050 +112051 POINT(40.672171057873776 73.05770032884764) bank112051 +112052 POINT(41.00075482928884 74.34620156916536) bank112052 +112053 POINT(39.88742455550779 74.76092455293063) bank112053 +112054 POINT(40.62328486027546 73.50674141731976) bank112054 +112055 POINT(40.83095518136575 73.09061262007403) bank112055 +112056 POINT(41.14735553543667 73.34605391218156) bank112056 +112057 POINT(40.14312232786798 74.48333809033315) bank112057 +112058 POINT(40.59786623816608 74.84170396573467) bank112058 +112059 POINT(40.47390478721623 74.9486802128828) bank112059 +112060 POINT(40.74334062196547 74.83108059283374) bank112060 +112061 POINT(39.776213298706935 73.7869254965622) bank112061 +112062 POINT(41.44792479632954 73.85793939783268) bank112062 +112063 POINT(40.178651468577456 74.01866592200125) bank112063 +112064 POINT(41.4838558724847 74.02521609366637) bank112064 +112065 POINT(40.80471993515598 73.73164496194138) bank112065 +112066 POINT(40.99985891171028 74.6164161888545) bank112066 +112067 POINT(40.34410267926598 73.92294436099347) bank112067 +112068 POINT(40.135768995090636 74.53776652238496) bank112068 +112069 POINT(40.617304221022934 73.16698487688241) bank112069 +112070 POINT(40.711489316983965 74.82861577133025) bank112070 +112071 POINT(40.147585384464705 74.1513899880893) bank112071 +112072 POINT(41.69269550760618 73.5901315848744) bank112072 +112073 POINT(40.85133653547849 74.88204278952088) bank112073 +112074 POINT(41.144966459236166 73.7153167126164) bank112074 +112075 POINT(40.43056723146323 73.34139271785133) bank112075 +112076 POINT(39.750769097945465 74.68100556299167) bank112076 +112077 POINT(40.296876535126096 73.89946782621102) bank112077 +112078 POINT(40.06181122140136 73.06674154052466) bank112078 +112079 POINT(41.57944902360628 73.87737566306026) bank112079 +112080 POINT(40.46588050709454 73.32148200019924) bank112080 +112081 POINT(41.69458142493411 74.66997991151041) bank112081 +112082 POINT(40.91132862326728 74.80545031391193) bank112082 +112083 POINT(40.486937901108796 74.16914323296123) bank112083 +112084 POINT(40.160605823863996 73.01368377691463) bank112084 +112085 POINT(39.791995738561084 73.47811161861715) bank112085 +112086 POINT(40.8630661520903 74.3913482407229) bank112086 +112087 POINT(40.068701888752614 74.70442834562644) bank112087 +112088 POINT(41.132757133127235 74.96843863158213) bank112088 +112089 POINT(40.247495127922726 74.47123743055232) bank112089 +112090 POINT(40.794211094032754 73.18753160716054) bank112090 +112091 POINT(40.717235832480995 73.9031536346828) bank112091 +112092 POINT(41.37452043718337 74.70616589733132) bank112092 +112093 POINT(39.872298035574296 74.80112707281252) bank112093 +112094 POINT(41.06685369591433 73.67964706886148) bank112094 +112095 POINT(39.81158505400402 74.45518773285671) bank112095 +112096 POINT(41.60694717846168 74.76198439980217) bank112096 +112097 POINT(41.5129389567762 74.15098311721675) bank112097 +112098 POINT(40.87870645999373 73.55044764868556) bank112098 +112099 POINT(40.411272585303266 74.87844987702904) bank112099 +112100 POINT(39.93857094120556 74.81379148285002) bank112100 +112101 POINT(40.07449722244077 74.02821097267201) bank112101 +112102 POINT(40.45639721663134 73.01374348529717) bank112102 +112103 POINT(41.53276142696998 73.50596960882193) bank112103 +112104 POINT(40.05763427043878 73.41930558267342) bank112104 +112105 POINT(40.75704613517733 73.6714800179761) bank112105 +112106 POINT(40.7255690123865 73.33647200371966) bank112106 +112107 POINT(39.76977166122965 74.91799552518015) bank112107 +112108 POINT(40.088165114945966 73.04586248551801) bank112108 +112109 POINT(41.216253327180524 74.24788928657316) bank112109 +112110 POINT(41.446784762348535 74.32169033324804) bank112110 +112111 POINT(40.02926659353286 74.98493422004358) bank112111 +112112 POINT(40.51315568646715 74.34373807158647) bank112112 +112113 POINT(39.81644207670438 74.44354502041233) bank112113 +112114 POINT(40.37151159489024 74.03745474355566) bank112114 +112115 POINT(41.604080704853736 74.0086565947785) bank112115 +112116 POINT(41.442791361542945 74.32949503256708) bank112116 +112117 POINT(40.23652857101925 73.62083794775141) bank112117 +112118 POINT(39.81497397277455 74.01951330794684) bank112118 +112119 POINT(41.42494848473402 73.07620547897437) bank112119 +112120 POINT(40.21379500240689 73.10148571928816) bank112120 +112121 POINT(40.47987091344784 73.04971212951044) bank112121 +112122 POINT(39.87631341335985 73.34040382213168) bank112122 +112123 POINT(40.46992423432277 74.96631951068261) bank112123 +112124 POINT(41.31145116251551 74.74812908524072) bank112124 +112125 POINT(41.070006706974354 74.04763010291389) bank112125 +112126 POINT(40.86202242590564 74.58375065600748) bank112126 +112127 POINT(41.18097651386697 74.9872133277703) bank112127 +112128 POINT(40.27443797753982 74.90654276316504) bank112128 +112129 POINT(41.57856531317541 74.66160578834179) bank112129 +112130 POINT(40.76974667557086 73.6979435620167) bank112130 +112131 POINT(39.813946781605594 74.56217845431813) bank112131 +112132 POINT(41.679856469641294 74.31927829167576) bank112132 +112133 POINT(40.738916712668704 73.53769962382417) bank112133 +112134 POINT(41.67911647178244 73.84710152562899) bank112134 +112135 POINT(40.527397273983276 74.0400412732501) bank112135 +112136 POINT(40.915262756444086 74.40726354674518) bank112136 +112137 POINT(40.434982109726455 74.94801243729223) bank112137 +112138 POINT(40.228258980690626 73.2780109759367) bank112138 +112139 POINT(40.264223256097985 73.39530266549974) bank112139 +112140 POINT(41.558264290114614 74.46933359059318) bank112140 +112141 POINT(40.598320635414446 74.76058773584411) bank112141 +112142 POINT(40.55330041787231 74.3985834615563) bank112142 +112143 POINT(41.59710884003707 74.20413729581912) bank112143 +112144 POINT(40.0013370989034 74.50716190149363) bank112144 +112145 POINT(41.15017712675025 74.04885291811956) bank112145 +112146 POINT(41.189599350608994 74.62866608463506) bank112146 +112147 POINT(40.796288398012635 73.19821246958044) bank112147 +112148 POINT(41.31923347214836 73.30183825204158) bank112148 +112149 POINT(40.00319035054878 73.21343769144684) bank112149 +112150 POINT(40.36212142625396 73.86841816660659) bank112150 +112151 POINT(40.21413490533224 74.3869436351381) bank112151 +112152 POINT(40.15368928691992 73.13424187811934) bank112152 +112153 POINT(40.39007066634001 73.32050322462088) bank112153 +112154 POINT(40.1336494112888 74.7987698107243) bank112154 +112155 POINT(40.71443031389375 73.43195125185832) bank112155 +112156 POINT(40.57923249760497 73.67117860632794) bank112156 +112157 POINT(40.45671706687077 74.759490912802) bank112157 +112158 POINT(40.329959912465554 74.58135783772362) bank112158 +112159 POINT(40.431496426986875 73.41953732414923) bank112159 +112160 POINT(40.899047161753785 73.42392029609789) bank112160 +112161 POINT(41.527161357526346 74.06487954099671) bank112161 +112162 POINT(41.64847265260278 73.93714800785652) bank112162 +112163 POINT(41.099763601257166 73.02743814538262) bank112163 +112164 POINT(41.36349598607049 73.77607275594127) bank112164 +112165 POINT(40.528220297698965 74.74731229744552) bank112165 +112166 POINT(40.348704387910985 73.76707739385569) bank112166 +112167 POINT(39.89432135269684 74.13137506816642) bank112167 +112168 POINT(40.63679127143947 73.54993271287637) bank112168 +112169 POINT(41.02467773176044 74.55619185868629) bank112169 +112170 POINT(40.462545310432795 73.39657787691168) bank112170 +112171 POINT(41.07683294157975 73.82339971064117) bank112171 +112172 POINT(39.97318396665179 74.06243825801643) bank112172 +112173 POINT(41.424059423302666 74.91875353228542) bank112173 +112174 POINT(41.64598238573479 74.87115233274434) bank112174 +112175 POINT(40.12829152381531 73.52831020516234) bank112175 +112176 POINT(41.0600477312428 73.89176559012175) bank112176 +112177 POINT(40.50474009570082 73.15028527528803) bank112177 +112178 POINT(41.34326229072277 74.92361769244943) bank112178 +112179 POINT(41.69425222014413 74.4214553960886) bank112179 +112180 POINT(40.91271615103418 73.61324207477193) bank112180 +112181 POINT(40.80886889932151 73.45389343542423) bank112181 +112182 POINT(40.21428441174307 74.74747755278639) bank112182 +112183 POINT(41.40899368616172 74.18329984743536) bank112183 +112184 POINT(41.11690254026022 73.13751834137477) bank112184 +112185 POINT(39.964769501831555 74.56554827498618) bank112185 +112186 POINT(41.681746620179354 73.72357254354296) bank112186 +112187 POINT(40.8901566389967 73.98325667458288) bank112187 +112188 POINT(41.34104770071837 74.52907405839383) bank112188 +112189 POINT(40.41883024489809 73.9176512835335) bank112189 +112190 POINT(40.43694862987284 73.6562417548673) bank112190 +112191 POINT(41.47224378589255 74.21946090824855) bank112191 +112192 POINT(39.98064365342928 73.26366642736436) bank112192 +112193 POINT(39.734575184242765 74.0192130459246) bank112193 +112194 POINT(41.5478335860266 73.71482089266792) bank112194 +112195 POINT(41.16072218936519 74.72276848055039) bank112195 +112196 POINT(40.46668860894738 74.87682371690701) bank112196 +112197 POINT(41.63785239465512 73.1904250275294) bank112197 +112198 POINT(41.33128483886832 74.08122708435033) bank112198 +112199 POINT(40.058322386405365 73.85610369781081) bank112199 +112200 POINT(40.95963948653613 74.75287864991708) bank112200 +112201 POINT(41.02765641739644 73.95904535609074) bank112201 +112202 POINT(40.43979442850342 74.57174322263745) bank112202 +112203 POINT(40.767125565480924 73.13990766137248) bank112203 +112204 POINT(41.12663620265877 74.14302863048773) bank112204 +112205 POINT(40.65550047501827 73.36891641280985) bank112205 +112206 POINT(41.53100020657991 73.2934918701309) bank112206 +112207 POINT(40.029237727000336 74.3397140813185) bank112207 +112208 POINT(40.59767238127856 73.71418889061198) bank112208 +112209 POINT(40.67992663402475 73.6898083594321) bank112209 +112210 POINT(39.98402488175765 73.6017713931313) bank112210 +112211 POINT(41.077645206817515 74.99039445987626) bank112211 +112212 POINT(39.75946512869495 73.15777964742021) bank112212 +112213 POINT(40.532501242361015 73.2186722432065) bank112213 +112214 POINT(40.47021826262871 74.04881124161676) bank112214 +112215 POINT(40.21074193480635 73.79561547985203) bank112215 +112216 POINT(40.80090326850186 73.67346590880106) bank112216 +112217 POINT(40.260919535874486 74.35087922807497) bank112217 +112218 POINT(39.92211739338229 73.9650250566359) bank112218 +112219 POINT(41.59685748276133 74.98726577096075) bank112219 +112220 POINT(40.415356776888814 73.61618717845178) bank112220 +112221 POINT(41.68823779721984 74.17128489580035) bank112221 +112222 POINT(40.88525122081868 73.02282335096531) bank112222 +112223 POINT(40.560590529830115 74.87742900310256) bank112223 +112224 POINT(40.21601085202487 73.78682435606497) bank112224 +112225 POINT(39.929626040535815 73.06513835871276) bank112225 +112226 POINT(39.90405717735791 74.90950503877917) bank112226 +112227 POINT(40.50093292676815 73.91354769779711) bank112227 +112228 POINT(39.72041826370667 74.9756203769298) bank112228 +112229 POINT(40.72829797436834 74.5403728519872) bank112229 +112230 POINT(41.6594519618645 73.9940862978976) bank112230 +112231 POINT(40.30459176075297 74.81354754039758) bank112231 +112232 POINT(40.10020132520517 73.59028331708512) bank112232 +112233 POINT(39.73964992276382 74.3666104442322) bank112233 +112234 POINT(40.275515694765375 74.22332521026826) bank112234 +112235 POINT(41.120460265713156 73.38664408828447) bank112235 +112236 POINT(40.30602319738987 74.19690755194915) bank112236 +112237 POINT(40.07601885995068 74.83399105969518) bank112237 +112238 POINT(40.27015258874612 73.63478211044593) bank112238 +112239 POINT(40.48347522935158 73.48201819618797) bank112239 +112240 POINT(41.04305122190749 73.25589485567613) bank112240 +112241 POINT(40.87401147744197 73.29683818361369) bank112241 +112242 POINT(41.58066322816586 73.30797237247883) bank112242 +112243 POINT(40.23906124522161 74.30685381919075) bank112243 +112244 POINT(41.35219116756859 73.22230065703972) bank112244 +112245 POINT(41.017108096177864 74.60384157043228) bank112245 +112246 POINT(41.61249500835486 73.13816518904338) bank112246 +112247 POINT(39.984977618510584 74.52768869766703) bank112247 +112248 POINT(41.571762801463194 74.61597058136488) bank112248 +112249 POINT(40.72826708515626 73.26511912852537) bank112249 +112250 POINT(41.53499098835818 74.03326647542158) bank112250 +112251 POINT(39.80535610080493 74.07615496984519) bank112251 +112252 POINT(39.90817513975982 74.34501542620728) bank112252 +112253 POINT(40.43312149123481 73.63502798819385) bank112253 +112254 POINT(39.766158252769415 73.82471751066355) bank112254 +112255 POINT(40.72713292670563 73.76212156342913) bank112255 +112256 POINT(40.28806782103412 73.92865649990031) bank112256 +112257 POINT(40.78454768132248 74.4669026389706) bank112257 +112258 POINT(40.70688824402369 73.28628531819831) bank112258 +112259 POINT(41.201155075997114 74.82651686233802) bank112259 +112260 POINT(40.7734325918463 74.69620229463112) bank112260 +112261 POINT(39.72989943170323 73.80163731852788) bank112261 +112262 POINT(41.261902669149244 74.6376433443937) bank112262 +112263 POINT(40.634518817202846 74.1735554545372) bank112263 +112264 POINT(39.85428511984161 74.14945864444253) bank112264 +112265 POINT(39.74597196656931 74.5366639612622) bank112265 +112266 POINT(40.16717953884645 74.52684926277199) bank112266 +112267 POINT(40.1506348834443 73.237702521559) bank112267 +112268 POINT(41.55487930112662 73.17300869540584) bank112268 +112269 POINT(40.710933101944335 73.09444024211756) bank112269 +112270 POINT(41.03492685496586 74.9368542112875) bank112270 +112271 POINT(40.9825488664178 73.72981937278682) bank112271 +112272 POINT(40.07822413138576 73.35690748962823) bank112272 +112273 POINT(41.63260543486429 74.13072828145495) bank112273 +112274 POINT(40.900027890038956 74.28832841051117) bank112274 +112275 POINT(40.96863533991505 73.11076546403778) bank112275 +112276 POINT(40.700919336861475 74.5818260078939) bank112276 +112277 POINT(41.2184413566773 74.84251845896956) bank112277 +112278 POINT(40.54469666815174 73.43759620687798) bank112278 +112279 POINT(40.85573145738547 74.55000234112129) bank112279 +112280 POINT(40.20511601767934 74.48880472613003) bank112280 +112281 POINT(41.0352077367294 73.96094349332104) bank112281 +112282 POINT(41.03483124841255 73.57055861303648) bank112282 +112283 POINT(41.37021316140607 74.81012015550067) bank112283 +112284 POINT(39.98552899552503 74.95495922037766) bank112284 +112285 POINT(40.152279782260734 73.63949541557116) bank112285 +112286 POINT(40.60879286039194 74.72967741158479) bank112286 +112287 POINT(41.64254670804237 74.08211437477844) bank112287 +112288 POINT(40.28740738544236 74.96443828059326) bank112288 +112289 POINT(40.417702910546126 74.66619208705187) bank112289 +112290 POINT(39.883844444180546 73.24257249268484) bank112290 +112291 POINT(41.19587122079825 74.3667921945331) bank112291 +112292 POINT(40.47596952726492 73.06918089583965) bank112292 +112293 POINT(40.86368870531305 73.69004540430676) bank112293 +112294 POINT(41.08461035077931 73.69466919153062) bank112294 +112295 POINT(39.734479397128055 74.43205830515821) bank112295 +112296 POINT(39.776509230680205 74.48903207246444) bank112296 +112297 POINT(40.501853568693065 73.99076222398024) bank112297 +112298 POINT(40.39231638445135 73.78105649499956) bank112298 +112299 POINT(41.083106156544865 74.72798533034873) bank112299 +112300 POINT(40.80900090805369 73.41013447698796) bank112300 +112301 POINT(40.98224282913083 74.07064792530417) bank112301 +112302 POINT(41.3444348006355 73.8495722522503) bank112302 +112303 POINT(40.555263945892506 73.21195999001083) bank112303 +112304 POINT(40.03133939873064 73.84271437634574) bank112304 +112305 POINT(40.63132157702115 74.05747121535522) bank112305 +112306 POINT(40.27031089041433 74.38675633684527) bank112306 +112307 POINT(40.44769281976326 74.134459411279) bank112307 +112308 POINT(40.15845036863314 74.8290148753694) bank112308 +112309 POINT(41.25374504839648 73.70181489936788) bank112309 +112310 POINT(39.97276339934942 73.17002800128179) bank112310 +112311 POINT(40.487138540146844 73.60908401187436) bank112311 +112312 POINT(40.02953830307198 73.60748917496728) bank112312 +112313 POINT(40.135411039326556 74.4077495641526) bank112313 +112314 POINT(40.36238998704969 74.19992503431446) bank112314 +112315 POINT(40.41080751709369 73.15335931540176) bank112315 +112316 POINT(40.038081689182164 74.96623931122417) bank112316 +112317 POINT(40.03510742888209 73.81172347752641) bank112317 +112318 POINT(41.36439749645606 74.67632895769455) bank112318 +112319 POINT(39.80898358857655 74.8158261195436) bank112319 +112320 POINT(40.24241768521064 73.91627238002258) bank112320 +112321 POINT(41.254158907578145 74.6786133419669) bank112321 +112322 POINT(40.56268066113834 74.54279614654214) bank112322 +112323 POINT(41.16736797847257 73.46603837690931) bank112323 +112324 POINT(40.119422486178074 73.23518364934885) bank112324 +112325 POINT(39.8346909571321 73.94878846281098) bank112325 +112326 POINT(41.63692459476826 74.57110838876598) bank112326 +112327 POINT(40.504305921921485 73.99457876140337) bank112327 +112328 POINT(41.06924774573262 73.19373133095723) bank112328 +112329 POINT(40.505051060221795 73.74210048689005) bank112329 +112330 POINT(40.29757292334994 74.95370557834326) bank112330 +112331 POINT(40.19039737364156 73.23877230451603) bank112331 +112332 POINT(40.79673630463353 73.90527807437914) bank112332 +112333 POINT(40.42599946598045 74.90503941896407) bank112333 +112334 POINT(41.18758222652832 74.9375050465112) bank112334 +112335 POINT(40.62308387900455 73.00898757051611) bank112335 +112336 POINT(40.16787601995176 73.80655458374144) bank112336 +112337 POINT(40.966105691297315 74.32122689418986) bank112337 +112338 POINT(40.92647279590943 74.19928986988573) bank112338 +112339 POINT(40.45534086928972 74.76434871355725) bank112339 +112340 POINT(41.31027433328444 73.83905771740493) bank112340 +112341 POINT(40.82161789203471 73.78328761502952) bank112341 +112342 POINT(40.34216283577483 74.49094579168074) bank112342 +112343 POINT(39.861314730697984 74.33665018672089) bank112343 +112344 POINT(40.15737281019945 73.5666091439235) bank112344 +112345 POINT(40.3387110556731 74.00242107150896) bank112345 +112346 POINT(41.59198674972932 73.11341512461036) bank112346 +112347 POINT(41.28623622525654 74.50091952435152) bank112347 +112348 POINT(40.91437356140997 74.84581615758039) bank112348 +112349 POINT(41.06098152042627 73.86272931193079) bank112349 +112350 POINT(40.33657110819206 74.50445706767496) bank112350 +112351 POINT(41.70007782619166 73.86613764122544) bank112351 +112352 POINT(40.85730598424922 74.78581129515928) bank112352 +112353 POINT(40.42840155115309 73.15283814482129) bank112353 +112354 POINT(39.784389919500654 74.49802861093242) bank112354 +112355 POINT(40.412908450370075 74.11244051084451) bank112355 +112356 POINT(39.732055641003726 74.4089161642389) bank112356 +112357 POINT(40.42442778443144 74.27489875488675) bank112357 +112358 POINT(41.02248312754508 74.15965197790032) bank112358 +112359 POINT(41.53291886450282 74.17354555311734) bank112359 +112360 POINT(41.21552661514972 74.5486364592933) bank112360 +112361 POINT(40.52537878146231 73.24059620767191) bank112361 +112362 POINT(41.00133522559384 73.69520494492973) bank112362 +112363 POINT(41.029889295215966 74.29974489449194) bank112363 +112364 POINT(39.74032392616859 74.92421322742257) bank112364 +112365 POINT(41.20341000770228 73.31209071980018) bank112365 +112366 POINT(41.52933959205819 73.75111129900684) bank112366 +112367 POINT(40.6253759377304 74.39594501105827) bank112367 +112368 POINT(41.51015517952702 73.90869932310143) bank112368 +112369 POINT(40.76998738892029 74.91436775233318) bank112369 +112370 POINT(40.14287010579966 74.66737469598371) bank112370 +112371 POINT(40.277615512650996 73.37209641466146) bank112371 +112372 POINT(40.07038739690652 73.2897947916584) bank112372 +112373 POINT(40.00509800499272 73.80403552781884) bank112373 +112374 POINT(39.852051204089506 73.96195094654337) bank112374 +112375 POINT(41.18944657679947 74.300657535164) bank112375 +112376 POINT(40.77415898362593 74.52649321866976) bank112376 +112377 POINT(41.59834734944919 73.39106265183892) bank112377 +112378 POINT(41.41542191256727 73.59001021993124) bank112378 +112379 POINT(40.61271637796211 73.6359967641783) bank112379 +112380 POINT(39.73545842887185 73.92141353991731) bank112380 +112381 POINT(41.60137370028328 73.44425211354961) bank112381 +112382 POINT(39.8929914927196 74.71396615939186) bank112382 +112383 POINT(41.1496250359446 74.79195635920576) bank112383 +112384 POINT(39.89862622369036 73.82028981720619) bank112384 +112385 POINT(41.65592786889966 74.96672579951458) bank112385 +112386 POINT(41.65268950072955 73.69588621616596) bank112386 +112387 POINT(39.79397643898156 74.79878868461995) bank112387 +112388 POINT(39.75220760902012 73.60391920687937) bank112388 +112389 POINT(40.01985748950102 74.14440679715466) bank112389 +112390 POINT(40.75196861391452 73.35255665207859) bank112390 +112391 POINT(40.30274759527851 73.71050503814695) bank112391 +112392 POINT(39.890686397336836 74.89126201285715) bank112392 +112393 POINT(40.54338009344594 74.20357359034878) bank112393 +112394 POINT(39.978615132994165 73.54743971089778) bank112394 +112395 POINT(39.795593347761304 74.80257513199791) bank112395 +112396 POINT(39.9911087526045 73.75289396608467) bank112396 +112397 POINT(41.25534665125987 74.76508012773039) bank112397 +112398 POINT(39.80083051352091 73.77305237207008) bank112398 +112399 POINT(41.63615517945281 74.63288751010566) bank112399 +112400 POINT(40.3158514300557 73.62211070350592) bank112400 +112401 POINT(41.089522147289074 73.45744158749292) bank112401 +112402 POINT(39.82913055592563 73.65291379252595) bank112402 +112403 POINT(39.843222317605246 73.15550841553626) bank112403 +112404 POINT(40.095469566639615 73.26090505928545) bank112404 +112405 POINT(40.698148511948034 73.94974643081635) bank112405 +112406 POINT(40.85931232530566 74.1659173953794) bank112406 +112407 POINT(40.21567471527923 73.59877375284334) bank112407 +112408 POINT(40.096081660878966 74.22414631538614) bank112408 +112409 POINT(40.567812659490414 74.04857624516711) bank112409 +112410 POINT(39.78857860264138 73.27511484103395) bank112410 +112411 POINT(40.563957876819565 74.71931223589979) bank112411 +112412 POINT(40.70272653870622 73.38618177990666) bank112412 +112413 POINT(41.04140238869079 74.76407287713401) bank112413 +112414 POINT(40.06066309801282 74.58954300310482) bank112414 +112415 POINT(41.62614979411145 73.98013526443775) bank112415 +112416 POINT(40.52325484068191 74.67409424315363) bank112416 +112417 POINT(40.838136738676326 74.0635430304566) bank112417 +112418 POINT(41.349144197096784 73.28302818765901) bank112418 +112419 POINT(40.097791807907704 73.32219924152119) bank112419 +112420 POINT(40.79199383084729 73.58266574526935) bank112420 +112421 POINT(41.378778993757834 74.46704399418218) bank112421 +112422 POINT(41.21429163760044 73.83502955547424) bank112422 +112423 POINT(40.47207880275005 74.12330381020348) bank112423 +112424 POINT(40.846709054635916 74.62133838062296) bank112424 +112425 POINT(39.75245199230846 74.69657427066623) bank112425 +112426 POINT(41.12417718155786 73.54124913496031) bank112426 +112427 POINT(40.63966974672316 74.8496877719023) bank112427 +112428 POINT(40.08520578818281 73.60318972664757) bank112428 +112429 POINT(39.98598201091391 74.07535864159655) bank112429 +112430 POINT(41.7101358360144 73.2337413387852) bank112430 +112431 POINT(40.17356593415989 73.26499523759092) bank112431 +112432 POINT(41.326734195119535 74.90481998857385) bank112432 +112433 POINT(41.261839161848116 73.07985349131962) bank112433 +112434 POINT(40.789385180773664 73.9903785449746) bank112434 +112435 POINT(40.591901003504844 74.43458556124156) bank112435 +112436 POINT(40.278353214930775 73.11302893967145) bank112436 +112437 POINT(41.06773329848608 74.87292491089444) bank112437 +112438 POINT(40.49151839757908 74.96053483252601) bank112438 +112439 POINT(41.241562720315784 73.72454900050528) bank112439 +112440 POINT(39.75435644698819 74.26324598789654) bank112440 +112441 POINT(40.277598787309344 73.32446288862363) bank112441 +112442 POINT(41.65154549311247 73.24684294563407) bank112442 +112443 POINT(41.04573181021431 73.90590912880522) bank112443 +112444 POINT(41.11052100563556 73.06143091793659) bank112444 +112445 POINT(40.553712238926295 74.11573201134114) bank112445 +112446 POINT(40.651124193938934 73.60253190339671) bank112446 +112447 POINT(39.97013192012639 74.67262307012135) bank112447 +112448 POINT(40.250293248254266 73.67034936295468) bank112448 +112449 POINT(40.64103286980101 73.63494581298616) bank112449 +112450 POINT(40.03969871796135 74.31473253242415) bank112450 +112451 POINT(41.17810780742999 73.46912229249827) bank112451 +112452 POINT(41.25476388241358 74.40389773211112) bank112452 +112453 POINT(40.260892236148045 74.83733537332638) bank112453 +112454 POINT(40.37037672594023 74.94268996063013) bank112454 +112455 POINT(41.458029203757825 74.82737879992082) bank112455 +112456 POINT(39.99144850816199 74.3141380673522) bank112456 +112457 POINT(39.88376964696147 74.71254342372468) bank112457 +112458 POINT(40.89661700394934 73.89052554634597) bank112458 +112459 POINT(40.42292690362178 74.72842198562195) bank112459 +112460 POINT(40.676865984132384 74.14977543410214) bank112460 +112461 POINT(40.63275183160616 74.51347797595473) bank112461 +112462 POINT(41.06048324188741 74.08860356754079) bank112462 +112463 POINT(41.70164670258753 74.05966125448305) bank112463 +112464 POINT(41.17126246869584 73.33489677245191) bank112464 +112465 POINT(40.263452583345206 74.44919151493696) bank112465 +112466 POINT(41.3470702213071 73.05389065449765) bank112466 +112467 POINT(41.694264246873495 74.4941735526327) bank112467 +112468 POINT(41.436048029823404 74.32301436476291) bank112468 +112469 POINT(40.91411039460536 74.75671967689497) bank112469 +112470 POINT(40.725161140442104 74.57588981182566) bank112470 +112471 POINT(40.7556669824443 74.42528865259678) bank112471 +112472 POINT(40.186842517320045 73.97317047451027) bank112472 +112473 POINT(41.364110186184995 73.16030942916198) bank112473 +112474 POINT(41.03990888375716 73.36533936577888) bank112474 +112475 POINT(40.66898974071992 74.68773208115003) bank112475 +112476 POINT(40.82528816407275 73.69771799519575) bank112476 +112477 POINT(40.865074137468376 74.32710029070752) bank112477 +112478 POINT(40.60615886091636 73.33545647490722) bank112478 +112479 POINT(40.26978030097608 74.18896403096166) bank112479 +112480 POINT(39.76274742049323 74.16219757430109) bank112480 +112481 POINT(41.18341385251689 74.01301977288531) bank112481 +112482 POINT(40.564680037980416 73.83705226767218) bank112482 +112483 POINT(40.04021460890403 74.50033764904843) bank112483 +112484 POINT(40.54774659208938 74.45719803340461) bank112484 +112485 POINT(39.74772433375471 74.47641840181699) bank112485 +112486 POINT(40.860401663013064 73.11843203361268) bank112486 +112487 POINT(39.94200574622937 73.46397073661454) bank112487 +112488 POINT(41.706020014129656 73.62082069600591) bank112488 +112489 POINT(40.428774435545776 73.76061075188292) bank112489 +112490 POINT(41.09404986843028 74.11429326447248) bank112490 +112491 POINT(40.65072100214304 74.77716366240341) bank112491 +112492 POINT(41.23027843706454 74.95141874024797) bank112492 +112493 POINT(41.17280506407098 73.54245374969895) bank112493 +112494 POINT(41.2019753982077 73.65408145873498) bank112494 +112495 POINT(40.58186530261855 74.68104973876925) bank112495 +112496 POINT(41.55833656551602 73.57469984535386) bank112496 +112497 POINT(40.053089178749836 74.68839827046547) bank112497 +112498 POINT(40.167480532569975 74.6070776894853) bank112498 +112499 POINT(40.06865789618232 73.55676986812492) bank112499 +112500 POINT(41.076310771913384 74.36264582268792) bank112500 +112501 POINT(39.727496861631174 73.65547232165693) bank112501 +112502 POINT(40.48104432836555 73.22578653343572) bank112502 +112503 POINT(41.65662230105704 73.1522678017091) bank112503 +112504 POINT(40.353986711200754 73.49569562730453) bank112504 +112505 POINT(40.55685053171406 73.07357959886308) bank112505 +112506 POINT(40.17087792864745 73.44582235836481) bank112506 +112507 POINT(40.311720078341004 74.59339612548895) bank112507 +112508 POINT(41.47723589190067 73.52064739796559) bank112508 +112509 POINT(40.9643840003453 74.86578861674076) bank112509 +112510 POINT(41.00037647165156 74.47374246652163) bank112510 +112511 POINT(41.49288434839734 73.75407854353483) bank112511 +112512 POINT(40.30427524743597 73.78739431820176) bank112512 +112513 POINT(41.5691931855727 74.79341847273386) bank112513 +112514 POINT(41.62732121231888 73.42754720038482) bank112514 +112515 POINT(39.7468951269284 73.46772681652931) bank112515 +112516 POINT(40.56599752439307 74.17983773000275) bank112516 +112517 POINT(40.39798119037337 74.0111834242579) bank112517 +112518 POINT(40.93479902300985 74.29882521326593) bank112518 +112519 POINT(40.49448228607001 74.01955669155872) bank112519 +112520 POINT(41.48606391607762 74.01526612670648) bank112520 +112521 POINT(40.53605017445591 74.15052857540327) bank112521 +112522 POINT(39.73801760034989 74.6935837513893) bank112522 +112523 POINT(41.435022242366074 74.16152520723807) bank112523 +112524 POINT(39.82260711209675 74.51351425640418) bank112524 +112525 POINT(40.94665562108906 74.62322663370279) bank112525 +112526 POINT(39.814346114114734 74.06366474290738) bank112526 +112527 POINT(41.652253391788214 73.5613955320989) bank112527 +112528 POINT(40.5493865781302 74.86922021868922) bank112528 +112529 POINT(40.091445939850644 73.70342849647601) bank112529 +112530 POINT(41.05479632808085 75.00516764083677) bank112530 +112531 POINT(40.437924730149405 73.87630240048) bank112531 +112532 POINT(40.792999466998545 74.08910863117337) bank112532 +112533 POINT(39.9471111242511 74.02801016873681) bank112533 +112534 POINT(40.699101932675724 74.66944535314511) bank112534 +112535 POINT(41.26963955850691 74.92573100315391) bank112535 +112536 POINT(41.6020138452776 73.36858620908575) bank112536 +112537 POINT(40.0162703969621 73.82779526178888) bank112537 +112538 POINT(39.887080407491425 73.49982558169768) bank112538 +112539 POINT(41.69314140090365 73.15169100865538) bank112539 +112540 POINT(40.81651673092048 73.27570098548671) bank112540 +112541 POINT(40.98959114863401 73.53964770718198) bank112541 +112542 POINT(40.01632822040869 74.1393670261958) bank112542 +112543 POINT(41.62305801018327 73.70905807099551) bank112543 +112544 POINT(40.13912656883652 73.49420752410103) bank112544 +112545 POINT(41.598227365005165 73.18121276838035) bank112545 +112546 POINT(39.95745000822258 74.01044646097918) bank112546 +112547 POINT(40.66215431237377 74.40426742374719) bank112547 +112548 POINT(41.31361715327708 73.14336810849866) bank112548 +112549 POINT(41.68223021854939 74.63051132111477) bank112549 +112550 POINT(40.83463401247178 73.40382296995531) bank112550 +112551 POINT(40.53322186640089 74.48354725422075) bank112551 +112552 POINT(40.641219260045844 73.09448995225239) bank112552 +112553 POINT(41.23185289959299 73.79412874908742) bank112553 +112554 POINT(41.41210016488939 74.04289017416535) bank112554 +112555 POINT(39.71580309153061 74.75946148072188) bank112555 +112556 POINT(40.46679311379059 73.99258931531887) bank112556 +112557 POINT(41.64430424463077 73.13840477329367) bank112557 +112558 POINT(41.0673313474576 73.34635306158135) bank112558 +112559 POINT(40.939454199525905 74.31953753025307) bank112559 +112560 POINT(40.305193891589546 74.76950717699731) bank112560 +112561 POINT(40.75510074966294 74.39138450731623) bank112561 +112562 POINT(39.87042868895131 74.50243773743838) bank112562 +112563 POINT(40.5908212257946 73.46966306039849) bank112563 +112564 POINT(40.96651404062194 74.65027710079944) bank112564 +112565 POINT(40.21362355616553 74.06872333414024) bank112565 +112566 POINT(40.208033240753274 73.4550896155259) bank112566 +112567 POINT(41.306469199682205 74.40552944918988) bank112567 +112568 POINT(40.99100863673866 74.15321668364732) bank112568 +112569 POINT(41.689096766262566 73.86558954582453) bank112569 +112570 POINT(41.437302931272015 74.0665385354339) bank112570 +112571 POINT(41.578129774840356 74.15744067519472) bank112571 +112572 POINT(40.940566944132 73.3419746704687) bank112572 +112573 POINT(40.30992154344661 74.7665085619541) bank112573 +112574 POINT(40.56593883341604 74.8838354737589) bank112574 +112575 POINT(41.6022381747233 74.68425876151338) bank112575 +112576 POINT(41.41559520618788 73.58470793448399) bank112576 +112577 POINT(40.54308731844985 74.76420703451815) bank112577 +112578 POINT(40.48445510536049 73.11337577855902) bank112578 +112579 POINT(39.93314489659714 73.41298105652113) bank112579 +112580 POINT(40.12723594089563 74.69720516763898) bank112580 +112581 POINT(41.399777485153095 73.08333497528646) bank112581 +112582 POINT(40.900304122882275 74.5338265001286) bank112582 +112583 POINT(40.70922002696965 74.3289072684496) bank112583 +112584 POINT(41.348109898148806 74.8751310408911) bank112584 +112585 POINT(40.77237948721454 73.89618713882132) bank112585 +112586 POINT(40.43929840902055 74.58276336255906) bank112586 +112587 POINT(39.914599142900975 74.89844899845889) bank112587 +112588 POINT(41.629833883511694 73.23783595642155) bank112588 +112589 POINT(41.061620035735096 73.43498731924568) bank112589 +112590 POINT(40.582645799785496 73.2242607414212) bank112590 +112591 POINT(40.363201425421146 73.25819982191479) bank112591 +112592 POINT(41.45399215622272 73.17676354837589) bank112592 +112593 POINT(40.424953005670055 74.93732260268041) bank112593 +112594 POINT(39.92007986338587 74.81932681933175) bank112594 +112595 POINT(40.04391468342253 74.85519619371462) bank112595 +112596 POINT(41.389238860723026 74.31785805567891) bank112596 +112597 POINT(41.71109633626925 74.68742857807906) bank112597 +112598 POINT(40.13038719567731 74.47819764588323) bank112598 +112599 POINT(40.85189790061141 74.25933829488217) bank112599 +112600 POINT(40.76674213365012 73.24903792454631) bank112600 +112601 POINT(41.24123294041306 73.84418349307276) bank112601 +112602 POINT(39.99824427781922 73.38136628325317) bank112602 +112603 POINT(40.52700414062301 73.57839751382859) bank112603 +112604 POINT(40.045585675063045 73.3016951899167) bank112604 +112605 POINT(40.68484521196303 74.60444636898379) bank112605 +112606 POINT(40.02663697183217 74.43524223345725) bank112606 +112607 POINT(41.00944359545893 74.58869149197886) bank112607 +112608 POINT(41.59082207513225 74.95639433384756) bank112608 +112609 POINT(41.496777961831675 74.36521699556212) bank112609 +112610 POINT(41.09509749312734 73.50704107081091) bank112610 +112611 POINT(41.221355553206784 74.76088307661642) bank112611 +112612 POINT(40.35595166332474 73.27561014664265) bank112612 +112613 POINT(40.14798013734998 73.57054979291729) bank112613 +112614 POINT(39.78981774561101 75.00212576716508) bank112614 +112615 POINT(41.20693914161003 74.00836785608666) bank112615 +112616 POINT(39.977653483304 73.32422312288938) bank112616 +112617 POINT(41.05592953568076 73.05273284050908) bank112617 +112618 POINT(40.27622901143244 73.43918883300306) bank112618 +112619 POINT(40.74784508561023 73.17208476386017) bank112619 +112620 POINT(40.5632630750347 73.45340370825387) bank112620 +112621 POINT(39.96333864888197 73.8748320003402) bank112621 +112622 POINT(40.60526627553189 74.73119056139393) bank112622 +112623 POINT(40.29123939194832 74.51434590674926) bank112623 +112624 POINT(41.69990909611675 74.87190657757037) bank112624 +112625 POINT(39.91657085040024 74.01319137711343) bank112625 +112626 POINT(40.97846943339345 74.78830638331402) bank112626 +112627 POINT(41.67055004599604 74.15828769291592) bank112627 +112628 POINT(40.556253052110534 74.4162419876183) bank112628 +112629 POINT(39.7961265915327 73.58500179587477) bank112629 +112630 POINT(40.01475322643679 73.97816425327028) bank112630 +112631 POINT(41.6042419894373 73.22509081534847) bank112631 +112632 POINT(41.361274333247 73.8929309886961) bank112632 +112633 POINT(40.0654846915897 74.72745050425992) bank112633 +112634 POINT(39.90604974772517 73.77788439605197) bank112634 +112635 POINT(40.67334530500968 74.14688424154191) bank112635 +112636 POINT(41.3853935549693 73.01035510823522) bank112636 +112637 POINT(40.29308627146759 73.2188400340166) bank112637 +112638 POINT(40.07902985192698 74.36310082988349) bank112638 +112639 POINT(39.82498324322488 74.23558473075465) bank112639 +112640 POINT(39.76478723978994 73.79241216695672) bank112640 +112641 POINT(41.41730428293569 73.85625434144153) bank112641 +112642 POINT(41.57926273966085 74.8708124552836) bank112642 +112643 POINT(40.844557552906494 73.49148864102106) bank112643 +112644 POINT(40.559369452288614 73.31104583344143) bank112644 +112645 POINT(40.98084509515114 74.71175261929508) bank112645 +112646 POINT(41.37207753132026 74.57444431074696) bank112646 +112647 POINT(40.77527250535856 74.16613840388801) bank112647 +112648 POINT(39.99385940481214 74.80453174932165) bank112648 +112649 POINT(41.490363473831025 74.3469568708009) bank112649 +112650 POINT(40.16423382035171 73.30563405311639) bank112650 +112651 POINT(40.77298442074017 73.47638573499391) bank112651 +112652 POINT(40.67672146632502 74.34370688531172) bank112652 +112653 POINT(40.77016535858423 74.28495171482858) bank112653 +112654 POINT(40.73990739636881 73.4473047979089) bank112654 +112655 POINT(40.325210826286046 74.66460992203395) bank112655 +112656 POINT(41.24692546472432 74.3512345708007) bank112656 +112657 POINT(39.90636942317607 73.62900125048169) bank112657 +112658 POINT(39.947753794739256 73.65310324091506) bank112658 +112659 POINT(40.68463629270876 74.30582123984733) bank112659 +112660 POINT(39.803607845759814 74.87957667754594) bank112660 +112661 POINT(40.60565652898198 74.56809350889571) bank112661 +112662 POINT(41.61051761473942 73.02330196869623) bank112662 +112663 POINT(40.62044612604238 73.10749003415532) bank112663 +112664 POINT(41.520108069959136 74.45069685459498) bank112664 +112665 POINT(40.72599773517331 74.45230223268055) bank112665 +112666 POINT(40.18593247112232 74.32085197950305) bank112666 +112667 POINT(40.14757186122658 74.09305444033393) bank112667 +112668 POINT(41.226783440883956 74.4664567289454) bank112668 +112669 POINT(40.63771525779969 73.14462264396488) bank112669 +112670 POINT(41.28521284120417 74.71292255133179) bank112670 +112671 POINT(41.64618328207625 73.25889624821514) bank112671 +112672 POINT(40.862629329857796 73.51014308171233) bank112672 +112673 POINT(40.83872030326147 74.39855749971197) bank112673 +112674 POINT(40.21952173924519 73.46344132430106) bank112674 +112675 POINT(40.393782332901274 74.82413650841988) bank112675 +112676 POINT(41.327929468952334 74.6415012399767) bank112676 +112677 POINT(40.08494518073784 74.06389097725136) bank112677 +112678 POINT(41.602139352501396 74.15477436859699) bank112678 +112679 POINT(41.29860617048855 73.89177747461973) bank112679 +112680 POINT(39.75544498176842 74.9647309358951) bank112680 +112681 POINT(40.96529778690018 73.26512774874024) bank112681 +112682 POINT(41.26434779532703 73.1408815413747) bank112682 +112683 POINT(40.27620189095044 73.04794567139034) bank112683 +112684 POINT(40.45968730852709 73.59934800081233) bank112684 +112685 POINT(41.4832747296012 74.17277187363267) bank112685 +112686 POINT(41.00325497879724 74.07722778870256) bank112686 +112687 POINT(40.686300755855605 74.3011190302714) bank112687 +112688 POINT(40.898259023784895 74.58499400216515) bank112688 +112689 POINT(40.46967828074449 73.28870505576829) bank112689 +112690 POINT(40.87431037052149 73.28837368181779) bank112690 +112691 POINT(40.64952831092187 73.75190436818333) bank112691 +112692 POINT(40.600157676497965 73.95842024789512) bank112692 +112693 POINT(39.939997646404635 74.53366831148755) bank112693 +112694 POINT(40.134831746691034 73.40297865441903) bank112694 +112695 POINT(40.852928007690316 73.10507780071856) bank112695 +112696 POINT(41.07914042341248 74.88232104344372) bank112696 +112697 POINT(41.246646573873065 73.36815077312336) bank112697 +112698 POINT(40.45257715377548 73.53729102148799) bank112698 +112699 POINT(40.791490635379894 74.51290658614849) bank112699 +112700 POINT(40.81703634655183 74.23655357602631) bank112700 +112701 POINT(41.4702420566696 74.51227500067412) bank112701 +112702 POINT(40.00000827510491 73.07465734756374) bank112702 +112703 POINT(40.7170392452805 74.09561092209684) bank112703 +112704 POINT(39.861463459443364 73.53412251298282) bank112704 +112705 POINT(40.17852938292139 74.44316984597862) bank112705 +112706 POINT(41.01732939895406 74.25133117438732) bank112706 +112707 POINT(41.525797734152356 73.56934842925844) bank112707 +112708 POINT(41.2423031712831 73.42657596392637) bank112708 +112709 POINT(40.23497861873865 73.39256073042101) bank112709 +112710 POINT(40.228156607931204 73.51527524327808) bank112710 +112711 POINT(40.56931706787973 73.28896145454733) bank112711 +112712 POINT(41.560231069907786 74.26844318767407) bank112712 +112713 POINT(41.545053218255745 73.53934833141595) bank112713 +112714 POINT(40.21978445755777 74.78346665222328) bank112714 +112715 POINT(40.03821670615 74.7486560918009) bank112715 +112716 POINT(41.54603859636543 74.98204537430155) bank112716 +112717 POINT(40.14576455920969 74.58952495022847) bank112717 +112718 POINT(40.712075419117774 73.09322463282804) bank112718 +112719 POINT(41.40582266906695 74.06283133757475) bank112719 +112720 POINT(41.52218607303082 74.71366424658005) bank112720 +112721 POINT(41.45192532711895 73.60318891123667) bank112721 +112722 POINT(41.42564789624325 74.0220861530778) bank112722 +112723 POINT(40.22860272307879 74.18231211877207) bank112723 +112724 POINT(41.573351233509165 73.29929086900378) bank112724 +112725 POINT(41.143325458687805 73.19509938143567) bank112725 +112726 POINT(40.270825399745696 73.523914046563) bank112726 +112727 POINT(40.4109264169004 74.89836281732784) bank112727 +112728 POINT(41.4043010795528 73.61021185648849) bank112728 +112729 POINT(41.22089954794562 74.59251353574308) bank112729 +112730 POINT(41.698343880283666 73.55020060786268) bank112730 +112731 POINT(40.12038792217154 73.5365396309155) bank112731 +112732 POINT(40.6099679455003 73.01736133905612) bank112732 +112733 POINT(41.016357618696205 73.94407567800778) bank112733 +112734 POINT(41.174282754382396 73.01677702775083) bank112734 +112735 POINT(39.774515790830336 73.59242330954008) bank112735 +112736 POINT(39.79761439993469 74.6126200316931) bank112736 +112737 POINT(41.155443281081546 74.61844421468066) bank112737 +112738 POINT(40.62914223306446 73.36468427049888) bank112738 +112739 POINT(40.227155321991 73.17767637364673) bank112739 +112740 POINT(40.346058071067624 74.28130650595406) bank112740 +112741 POINT(40.48718814926726 74.77531879167691) bank112741 +112742 POINT(41.21997708138566 73.49428095123719) bank112742 +112743 POINT(41.048811288361264 73.12488035577545) bank112743 +112744 POINT(40.580575860292036 73.45724968936457) bank112744 +112745 POINT(40.89924809383102 73.6476445588248) bank112745 +112746 POINT(41.04512154293791 74.45910961516785) bank112746 +112747 POINT(41.150425013874525 74.78549653740018) bank112747 +112748 POINT(40.9700140261708 74.68935987838915) bank112748 +112749 POINT(40.45243186202574 74.97833018262175) bank112749 +112750 POINT(39.95616746359688 74.15820420671663) bank112750 +112751 POINT(41.161924854663326 73.85881003937074) bank112751 +112752 POINT(41.20620477237445 73.42991389344259) bank112752 +112753 POINT(39.869179471709906 74.12035215056632) bank112753 +112754 POINT(41.456685099300344 74.26032721718033) bank112754 +112755 POINT(41.067515911252784 74.21184665698341) bank112755 +112756 POINT(40.79439415611735 74.06697305303719) bank112756 +112757 POINT(39.91133417919286 73.57959197283037) bank112757 +112758 POINT(40.374219303138595 74.06609687209402) bank112758 +112759 POINT(41.159835604194775 73.73598713913904) bank112759 +112760 POINT(41.06950711959027 73.54800365751447) bank112760 +112761 POINT(40.78253675169897 74.92899942176018) bank112761 +112762 POINT(41.368300165784866 73.30861353733214) bank112762 +112763 POINT(40.680265322844456 74.76965329461473) bank112763 +112764 POINT(40.7015915139882 74.82067308735985) bank112764 +112765 POINT(41.58609643730265 74.08536024319942) bank112765 +112766 POINT(39.97111952396913 73.83242279665593) bank112766 +112767 POINT(41.429338481146445 73.93691000701646) bank112767 +112768 POINT(39.77837087760624 74.07066790839352) bank112768 +112769 POINT(41.48740522574888 74.96642604671314) bank112769 +112770 POINT(41.034516045934645 74.47028018824737) bank112770 +112771 POINT(41.58998612153924 73.42866351119267) bank112771 +112772 POINT(41.1323394413664 74.19289517940447) bank112772 +112773 POINT(39.87016220745624 74.62050503667223) bank112773 +112774 POINT(41.04250329901231 73.452516845289) bank112774 +112775 POINT(41.12969099638594 73.86444409569108) bank112775 +112776 POINT(41.44473681698815 73.64481840437605) bank112776 +112777 POINT(40.892994401636884 73.06338105012638) bank112777 +112778 POINT(39.743878382445686 74.05686599392733) bank112778 +112779 POINT(41.07008137444781 73.44337323542501) bank112779 +112780 POINT(40.337694942142384 74.46458055012522) bank112780 +112781 POINT(40.07389327116944 75.00283966556474) bank112781 +112782 POINT(40.86347903001248 74.02190228958315) bank112782 +112783 POINT(40.41372603157957 74.6714619561522) bank112783 +112784 POINT(39.893418892819575 73.48648182464343) bank112784 +112785 POINT(39.95830597991245 74.32437149829914) bank112785 +112786 POINT(41.429821759494324 74.97624193223508) bank112786 +112787 POINT(40.527675658214655 74.70528660100612) bank112787 +112788 POINT(40.31061578007227 73.01405262213129) bank112788 +112789 POINT(41.090443106638816 73.16284542061447) bank112789 +112790 POINT(40.31724670581886 73.7985077819538) bank112790 +112791 POINT(40.634112993501674 73.0134635350521) bank112791 +112792 POINT(41.70454880100529 74.03091662629392) bank112792 +112793 POINT(39.85680374150867 73.3610462032239) bank112793 +112794 POINT(40.595360055952256 74.6691083748129) bank112794 +112795 POINT(40.05967806115048 74.28450760708246) bank112795 +112796 POINT(40.024086648953414 73.21888307552254) bank112796 +112797 POINT(40.84470905911736 73.24289716085909) bank112797 +112798 POINT(40.520481646802786 74.5523953875923) bank112798 +112799 POINT(39.73820199395797 74.72522509146643) bank112799 +112800 POINT(40.89686454368171 74.72336149138376) bank112800 +112801 POINT(40.1459458821131 74.63096005121564) bank112801 +112802 POINT(41.12402756220211 74.08330035641721) bank112802 +112803 POINT(41.71096994155631 74.03178665951579) bank112803 +112804 POINT(40.82249354925657 74.11067137887426) bank112804 +112805 POINT(41.19769565372849 73.45767538431348) bank112805 +112806 POINT(39.822342642186435 74.57544364495088) bank112806 +112807 POINT(40.95593295108357 74.28032849577582) bank112807 +112808 POINT(40.93751133822094 74.79695492716607) bank112808 +112809 POINT(41.155483121753285 73.35319745690207) bank112809 +112810 POINT(41.411914040865504 73.24384757709625) bank112810 +112811 POINT(40.554521096392115 74.22502517746258) bank112811 +112812 POINT(40.45613685634718 73.77187955343686) bank112812 +112813 POINT(40.210682700619714 73.0892840364867) bank112813 +112814 POINT(39.830468861982546 74.65077796657482) bank112814 +112815 POINT(41.6685467676483 74.78554988874109) bank112815 +112816 POINT(41.24349260854431 73.62323935597945) bank112816 +112817 POINT(41.54826686999473 73.04979792150428) bank112817 +112818 POINT(40.095780966398436 73.4250754735004) bank112818 +112819 POINT(41.50735632917175 73.33871538670631) bank112819 +112820 POINT(40.57335864801755 74.86847693089341) bank112820 +112821 POINT(39.93465932268641 74.78509296081899) bank112821 +112822 POINT(39.94697870354986 73.55005975603014) bank112822 +112823 POINT(39.922795164823064 73.1433053018276) bank112823 +112824 POINT(41.32333954893585 73.52876160005742) bank112824 +112825 POINT(40.48356959919971 74.98004661936369) bank112825 +112826 POINT(39.93399297787114 73.0271788750403) bank112826 +112827 POINT(40.461968359557346 74.76557673188056) bank112827 +112828 POINT(39.976941744602485 74.57078697106049) bank112828 +112829 POINT(40.98974650964948 73.565116910288) bank112829 +112830 POINT(40.53017208134952 73.0381185201709) bank112830 +112831 POINT(40.70693470916687 73.825000619347) bank112831 +112832 POINT(39.9355256889268 73.9464941786084) bank112832 +112833 POINT(41.32071376221615 74.24854334395646) bank112833 +112834 POINT(41.306231962392395 74.44255572200721) bank112834 +112835 POINT(41.45092396956163 73.11532125097366) bank112835 +112836 POINT(41.359536903193955 73.43689827945195) bank112836 +112837 POINT(40.233288012429675 73.63250198202853) bank112837 +112838 POINT(40.027791783816454 73.76006797244023) bank112838 +112839 POINT(39.98766300688882 73.71515825034865) bank112839 +112840 POINT(40.59338732326094 73.78796443735557) bank112840 +112841 POINT(40.7811820646457 74.73927321975007) bank112841 +112842 POINT(41.176920779712376 73.52186812789422) bank112842 +112843 POINT(41.40267744257285 73.17362569848753) bank112843 +112844 POINT(40.401210676320176 73.40855643662594) bank112844 +112845 POINT(41.48834652109759 73.3268058348554) bank112845 +112846 POINT(39.96015975718918 73.74158528306882) bank112846 +112847 POINT(40.66987410718616 73.60996956394771) bank112847 +112848 POINT(39.73003300627368 74.36061816885639) bank112848 +112849 POINT(41.49221553174451 74.33955477213573) bank112849 +112850 POINT(41.48262303784188 74.8678298772098) bank112850 +112851 POINT(39.721313444494676 74.36336844109053) bank112851 +112852 POINT(41.086504154599204 73.98126594673735) bank112852 +112853 POINT(39.938633861659525 74.34820102963322) bank112853 +112854 POINT(41.44472534150954 74.9528562250168) bank112854 +112855 POINT(41.60635111871071 74.6018800456661) bank112855 +112856 POINT(40.38434843637242 74.72633024039945) bank112856 +112857 POINT(41.33448455930026 74.49101199235923) bank112857 +112858 POINT(41.23681905893039 74.32875613698856) bank112858 +112859 POINT(39.94656192896126 74.9846224348029) bank112859 +112860 POINT(40.67832642322357 74.62857544219018) bank112860 +112861 POINT(41.06886961373787 73.34468570514335) bank112861 +112862 POINT(40.605471770450485 73.49196755836232) bank112862 +112863 POINT(40.38131680896072 74.8770789495743) bank112863 +112864 POINT(40.24754135669626 74.22171303556341) bank112864 +112865 POINT(41.32096340920963 74.22326517937606) bank112865 +112866 POINT(40.79343745255734 74.99512260206215) bank112866 +112867 POINT(41.64554820263384 74.5873387519561) bank112867 +112868 POINT(40.88132466968152 74.0570312304714) bank112868 +112869 POINT(41.10573972564373 74.27552097405587) bank112869 +112870 POINT(40.938737869922505 74.01839508907428) bank112870 +112871 POINT(40.43310834893075 73.21204298772415) bank112871 +112872 POINT(41.44296832918289 73.01895635601903) bank112872 +112873 POINT(40.68685765479514 74.3218960017069) bank112873 +112874 POINT(41.256671544300886 73.16748810455518) bank112874 +112875 POINT(40.689420247966 74.55462511451749) bank112875 +112876 POINT(41.49878377358307 74.70305724948027) bank112876 +112877 POINT(40.53256472976999 73.90301143538859) bank112877 +112878 POINT(41.61948027899099 74.13484987677117) bank112878 +112879 POINT(40.18695863699838 74.52794120445849) bank112879 +112880 POINT(40.42103692045772 73.943665065244) bank112880 +112881 POINT(39.81376921444911 74.56450830665314) bank112881 +112882 POINT(39.86689203414302 74.4828087639118) bank112882 +112883 POINT(39.917156037287384 74.48089093499955) bank112883 +112884 POINT(40.926469230752225 73.17918765258491) bank112884 +112885 POINT(41.461583538488995 74.27473226992414) bank112885 +112886 POINT(39.77750488114721 74.85787628230112) bank112886 +112887 POINT(41.094476144479394 74.19282526201313) bank112887 +112888 POINT(40.220152987396794 73.33204093855822) bank112888 +112889 POINT(41.40808159782161 73.24359050175015) bank112889 +112890 POINT(40.447704996709206 73.87433780298679) bank112890 +112891 POINT(41.42760629634935 74.36419456297047) bank112891 +112892 POINT(40.548537697052275 73.53958313279023) bank112892 +112893 POINT(41.17299599659562 74.2564633098688) bank112893 +112894 POINT(40.92936593493516 74.536259308752) bank112894 +112895 POINT(40.24585095502699 74.99607020072614) bank112895 +112896 POINT(40.33857164331632 74.05289224804436) bank112896 +112897 POINT(40.292147631324376 74.75120986066561) bank112897 +112898 POINT(41.1385141266662 74.91299516564024) bank112898 +112899 POINT(40.1940069244701 74.56265918885224) bank112899 +112900 POINT(40.51767619037776 73.13633422020291) bank112900 +112901 POINT(40.35157881735182 73.37011811321713) bank112901 +112902 POINT(40.95563950569323 74.90991052341158) bank112902 +112903 POINT(41.66746355292572 73.3091311578822) bank112903 +112904 POINT(40.74558978306196 74.01331168234499) bank112904 +112905 POINT(40.539633254275 73.45109598337842) bank112905 +112906 POINT(41.66362751399406 74.52807756382154) bank112906 +112907 POINT(40.28897746357096 74.46020805126429) bank112907 +112908 POINT(40.19918901388455 74.24698056629211) bank112908 +112909 POINT(40.24124567334588 74.71766427455196) bank112909 +112910 POINT(41.3552520915022 73.46163304058321) bank112910 +112911 POINT(40.26538862037315 74.05975008431389) bank112911 +112912 POINT(40.823201327672194 73.15575399285188) bank112912 +112913 POINT(41.169485237864045 73.99447397357618) bank112913 +112914 POINT(41.25852494709163 73.84829083289755) bank112914 +112915 POINT(40.601688191988416 73.95448238780357) bank112915 +112916 POINT(41.00106748544773 74.60357673943771) bank112916 +112917 POINT(41.25812728207266 73.29610586983294) bank112917 +112918 POINT(41.59286170129696 74.64626217114485) bank112918 +112919 POINT(41.53475166243348 73.09156096477403) bank112919 +112920 POINT(41.54174814493775 73.54112899275702) bank112920 +112921 POINT(40.45943356466065 73.08496263805408) bank112921 +112922 POINT(40.92083269917649 74.76277670528755) bank112922 +112923 POINT(39.71959633192425 74.34168970680929) bank112923 +112924 POINT(39.9464573323493 74.75848753158864) bank112924 +112925 POINT(39.85017961354032 73.62456386416267) bank112925 +112926 POINT(41.69426778072707 73.72683726754032) bank112926 +112927 POINT(41.185654709891324 73.13732103129816) bank112927 +112928 POINT(39.848451967809915 73.24469278529943) bank112928 +112929 POINT(40.43817041987662 73.94446478636455) bank112929 +112930 POINT(40.57868327985921 73.22463205343948) bank112930 +112931 POINT(40.428784600033644 74.26769613479823) bank112931 +112932 POINT(40.4877544899222 73.18593397407102) bank112932 +112933 POINT(41.37754613190717 74.96364662751616) bank112933 +112934 POINT(40.73622841776598 73.15914352226218) bank112934 +112935 POINT(40.74070255622684 74.17683290792225) bank112935 +112936 POINT(41.13152902068082 74.8765801417172) bank112936 +112937 POINT(41.6719498864837 73.55403526492621) bank112937 +112938 POINT(39.98120041374582 74.59623494069675) bank112938 +112939 POINT(39.74063785319189 73.74340470101761) bank112939 +112940 POINT(39.838046780355214 73.24756158933346) bank112940 +112941 POINT(39.942727397419596 73.37253622877512) bank112941 +112942 POINT(40.03115865552976 74.1237553678435) bank112942 +112943 POINT(41.52215117357052 74.5220983542014) bank112943 +112944 POINT(41.44116254209966 73.82277636865238) bank112944 +112945 POINT(40.08082160880829 73.18234806712954) bank112945 +112946 POINT(41.14955618376456 73.76157347120308) bank112946 +112947 POINT(39.75163634368116 73.19253133195717) bank112947 +112948 POINT(39.94086887927322 74.5308633094866) bank112948 +112949 POINT(40.77044847178798 74.01438372372881) bank112949 +112950 POINT(40.29035777193807 74.72695224244741) bank112950 +112951 POINT(41.293183433551775 73.45508173004043) bank112951 +112952 POINT(39.816730643504386 73.9492289868807) bank112952 +112953 POINT(39.873250075676815 74.84059369074411) bank112953 +112954 POINT(41.2686710886208 74.11374180710237) bank112954 +112955 POINT(41.558880916131635 73.88819343179372) bank112955 +112956 POINT(40.45326402996302 73.51003119788155) bank112956 +112957 POINT(41.4782023733132 74.704157247199) bank112957 +112958 POINT(40.85945620658049 73.33991310704022) bank112958 +112959 POINT(40.32081599307361 74.75930858104512) bank112959 +112960 POINT(39.95594597764306 74.11809887385456) bank112960 +112961 POINT(41.4440463918433 74.86714424411487) bank112961 +112962 POINT(40.97679776762167 74.343872575544) bank112962 +112963 POINT(39.95232476751219 74.69774843312922) bank112963 +112964 POINT(41.02657432427284 74.79465019812397) bank112964 +112965 POINT(40.904298269142856 73.81207506337721) bank112965 +112966 POINT(40.46332419082193 74.49860578967896) bank112966 +112967 POINT(39.98999483623963 74.4565376262062) bank112967 +112968 POINT(41.168112167922374 73.20666485999683) bank112968 +112969 POINT(40.7923648449446 73.43761754819575) bank112969 +112970 POINT(41.12301863756779 73.38167766075318) bank112970 +112971 POINT(40.4298437869988 73.99454441332307) bank112971 +112972 POINT(40.83520083575323 74.5897034417985) bank112972 +112973 POINT(41.66810998137926 73.03011912704447) bank112973 +112974 POINT(41.711782518917225 73.58127589537789) bank112974 +112975 POINT(40.11165624381007 73.96911181556368) bank112975 +112976 POINT(40.670222788919965 74.16688475541135) bank112976 +112977 POINT(41.45318138792319 73.4007330005504) bank112977 +112978 POINT(40.67107544948838 73.16405125669995) bank112978 +112979 POINT(40.76046064319195 74.71165111256266) bank112979 +112980 POINT(40.88551608641505 73.85494571099015) bank112980 +112981 POINT(39.807905009103116 73.17869667296172) bank112981 +112982 POINT(40.067811492004616 73.9573895881651) bank112982 +112983 POINT(39.720657416967875 74.22983579789818) bank112983 +112984 POINT(40.26116684316487 73.03748756662355) bank112984 +112985 POINT(41.37962952466509 73.8229032799354) bank112985 +112986 POINT(41.08689673574019 74.6806846283549) bank112986 +112987 POINT(41.12435939698315 74.39147479625872) bank112987 +112988 POINT(41.015919019764915 73.96974429438247) bank112988 +112989 POINT(40.3242900555588 74.48412318880354) bank112989 +112990 POINT(40.38889537235376 74.30939286463146) bank112990 +112991 POINT(40.01749606062567 74.67119234360516) bank112991 +112992 POINT(40.521834260028825 73.45551079527341) bank112992 +112993 POINT(40.93070909006696 73.99959309470498) bank112993 +112994 POINT(39.719608792117086 73.88459236599462) bank112994 +112995 POINT(40.171849079309595 73.45275643369352) bank112995 +112996 POINT(40.33523849696506 73.18898911981596) bank112996 +112997 POINT(41.58620040032143 74.29853018300868) bank112997 +112998 POINT(41.43112332295073 73.65213867676205) bank112998 +112999 POINT(39.96757424838861 74.90299720638886) bank112999 +113000 POINT(40.64702530792899 73.30115502485693) bank113000 +113001 POINT(40.650662909631855 74.86578925250961) bank113001 +113002 POINT(40.92503085641349 73.53036798149724) bank113002 +113003 POINT(40.71746482987423 73.4467464717387) bank113003 +113004 POINT(40.404816864179054 74.77862027342108) bank113004 +113005 POINT(39.731821148671685 73.8617192476168) bank113005 +113006 POINT(41.46372621692018 74.70935669241959) bank113006 +113007 POINT(40.7509256246742 73.66418311713991) bank113007 +113008 POINT(41.23931368074965 74.8975525936242) bank113008 +113009 POINT(41.1800376266644 74.01947275127775) bank113009 +113010 POINT(40.183423464927046 73.00786125438319) bank113010 +113011 POINT(40.95813373486068 74.55839553339209) bank113011 +113012 POINT(41.168011007841145 74.920670718652) bank113012 +113013 POINT(40.64370891643769 74.97204514679693) bank113013 +113014 POINT(41.683413178425816 74.75897615525814) bank113014 +113015 POINT(41.40207039918885 73.7266873867707) bank113015 +113016 POINT(40.96432265431326 74.49430337204755) bank113016 +113017 POINT(40.55686399516861 73.22756475615306) bank113017 +113018 POINT(41.682109637340055 74.00137297461718) bank113018 +113019 POINT(40.362293929314006 74.25401241428708) bank113019 +113020 POINT(40.13479269857842 74.20347606818333) bank113020 +113021 POINT(39.8616093143064 73.97496069157117) bank113021 +113022 POINT(41.70297575238129 73.53791517356373) bank113022 +113023 POINT(41.30863423427355 73.36423113954231) bank113023 +113024 POINT(40.72211256156257 74.99419304497142) bank113024 +113025 POINT(41.706688157543745 73.9236553081196) bank113025 +113026 POINT(41.25351296799687 74.34698642971816) bank113026 +113027 POINT(39.94331668319852 73.16825518942571) bank113027 +113028 POINT(39.98680199270352 74.91587931258289) bank113028 +113029 POINT(41.67181779644049 74.9751892915656) bank113029 +113030 POINT(40.13901807268915 73.57885626939782) bank113030 +113031 POINT(40.145841123344745 73.68266695151338) bank113031 +113032 POINT(41.35140317548148 73.40834963242318) bank113032 +113033 POINT(40.37025622023247 73.29718152483717) bank113033 +113034 POINT(40.809616866988605 74.7876202240549) bank113034 +113035 POINT(41.630052255257525 73.7467411074621) bank113035 +113036 POINT(40.992285021755734 73.77653050704397) bank113036 +113037 POINT(41.47984142228915 74.8162052773023) bank113037 +113038 POINT(41.145970064302 73.01237547230531) bank113038 +113039 POINT(40.79770151432273 74.59905230296721) bank113039 +113040 POINT(40.477288462830515 73.84102734138486) bank113040 +113041 POINT(41.41473165158056 74.73112633289824) bank113041 +113042 POINT(40.866023705787235 73.46726792669322) bank113042 +113043 POINT(39.9567491394054 73.24175848786146) bank113043 +113044 POINT(40.61451508619591 73.19617874207245) bank113044 +113045 POINT(41.01555394463267 73.38629611148617) bank113045 +113046 POINT(40.72683659710103 74.40053043530314) bank113046 +113047 POINT(39.99504876425593 74.77040926928774) bank113047 +113048 POINT(40.31271473578934 74.04706895998551) bank113048 +113049 POINT(41.16444390313789 74.02583208529182) bank113049 +113050 POINT(40.84311397106542 74.22190123404991) bank113050 +113051 POINT(41.62521996869291 73.79474472364572) bank113051 +113052 POINT(41.29046217837603 73.7001894814138) bank113052 +113053 POINT(41.690854199947196 74.213847510804) bank113053 +113054 POINT(41.193227342585686 74.44082117979974) bank113054 +113055 POINT(41.22725020547276 73.87643604211593) bank113055 +113056 POINT(41.21839756187574 73.81957612625352) bank113056 +113057 POINT(41.39194114751161 74.67381225385404) bank113057 +113058 POINT(40.62211451601487 73.08325643502197) bank113058 +113059 POINT(41.43795874344837 73.74081202653477) bank113059 +113060 POINT(41.15813865500432 74.5301454466768) bank113060 +113061 POINT(41.42283469063927 74.57139428382698) bank113061 +113062 POINT(40.58378176613164 74.1031186600563) bank113062 +113063 POINT(40.37698197111501 74.79541820941031) bank113063 +113064 POINT(41.20397564628809 74.21448379707188) bank113064 +113065 POINT(39.90093458374082 74.66303725111729) bank113065 +113066 POINT(41.008590642022156 74.3666657317641) bank113066 +113067 POINT(41.48660907449381 73.70899822837048) bank113067 +113068 POINT(39.815803204882926 73.80675067582258) bank113068 +113069 POINT(40.12618959739834 74.008822292773) bank113069 +113070 POINT(39.716061514449564 73.68999075491142) bank113070 +113071 POINT(41.057378035820484 74.17937812558105) bank113071 +113072 POINT(40.15080803672801 74.41048098627715) bank113072 +113073 POINT(40.05014671781983 74.97733735797604) bank113073 +113074 POINT(41.10794448600857 73.67375980501217) bank113074 +113075 POINT(41.67865040755512 74.26367857591349) bank113075 +113076 POINT(40.3894541803694 73.87091874415007) bank113076 +113077 POINT(40.588739272248745 73.05678293690956) bank113077 +113078 POINT(40.376079104261514 74.21906549536828) bank113078 +113079 POINT(40.81737979256138 73.25674790251108) bank113079 +113080 POINT(40.88747905029197 73.59771341347877) bank113080 +113081 POINT(41.047690445941576 74.81602896455205) bank113081 +113082 POINT(40.333650456549904 74.47534700864446) bank113082 +113083 POINT(41.26495081453181 73.97795068100126) bank113083 +113084 POINT(40.84227419718785 74.86254928384507) bank113084 +113085 POINT(41.237075752458686 74.2739867049941) bank113085 +113086 POINT(40.86662535958759 74.32035287974527) bank113086 +113087 POINT(39.970819247338525 73.24392583465804) bank113087 +113088 POINT(41.002387778057525 74.9570009835515) bank113088 +113089 POINT(39.73124464081705 73.17881662411614) bank113089 +113090 POINT(40.20711900112897 74.7784941151493) bank113090 +113091 POINT(41.00863216803973 73.8295534295814) bank113091 +113092 POINT(40.32975570559661 74.37382510955813) bank113092 +113093 POINT(40.174978765424456 73.2832500502296) bank113093 +113094 POINT(41.22541580659838 74.5457929137685) bank113094 +113095 POINT(39.854166697783235 73.92932314704665) bank113095 +113096 POINT(39.84098788624585 73.10847827842356) bank113096 +113097 POINT(41.11847741586305 74.97196542127863) bank113097 +113098 POINT(40.063551766682046 74.10532267242121) bank113098 +113099 POINT(41.46396398397285 74.94858507059661) bank113099 +113100 POINT(40.66529507218786 75.00047708083834) bank113100 +113101 POINT(39.97132769570768 74.54703596732601) bank113101 +113102 POINT(40.65360723783748 74.6519447954494) bank113102 +113103 POINT(39.976935632776424 74.78317871362786) bank113103 +113104 POINT(40.0431728917523 73.75489245814214) bank113104 +113105 POINT(40.381316651655276 73.39458922300285) bank113105 +113106 POINT(41.55277551450743 73.0236887571562) bank113106 +113107 POINT(40.46387152443284 73.16037476491414) bank113107 +113108 POINT(41.203244243901125 73.93357468725807) bank113108 +113109 POINT(40.10903053099164 73.34572394735629) bank113109 +113110 POINT(40.427619546165566 73.75981373531337) bank113110 +113111 POINT(41.66517174547677 74.53165134701723) bank113111 +113112 POINT(41.65651014373555 74.16843807046999) bank113112 +113113 POINT(40.998755409613196 73.60809609136489) bank113113 +113114 POINT(41.12803417226042 73.03378521853247) bank113114 +113115 POINT(39.95633826701406 73.21710569316011) bank113115 +113116 POINT(40.9356905286568 73.39761415046313) bank113116 +113117 POINT(40.800218500674426 74.22847742370348) bank113117 +113118 POINT(41.20276919062152 74.28831060591993) bank113118 +113119 POINT(39.84893803441151 73.21134432089143) bank113119 +113120 POINT(40.022513756854174 73.21014900043104) bank113120 +113121 POINT(40.41777738208742 73.56902854399083) bank113121 +113122 POINT(41.11301883070122 73.92408130746134) bank113122 +113123 POINT(40.36491568465762 74.82982079334514) bank113123 +113124 POINT(39.95787881873352 74.78566215345383) bank113124 +113125 POINT(40.11304811731016 73.18044785551218) bank113125 +113126 POINT(41.50746959817515 73.15036957586379) bank113126 +113127 POINT(40.864632741695125 73.19097501095204) bank113127 +113128 POINT(40.813255428031354 74.54659472421328) bank113128 +113129 POINT(40.943050045255745 74.85131659831946) bank113129 +113130 POINT(40.55344563884799 73.39016359442361) bank113130 +113131 POINT(41.46114782281843 74.2937009430764) bank113131 +113132 POINT(41.53396618688891 74.11621021564017) bank113132 +113133 POINT(41.648905222612264 74.49495643535181) bank113133 +113134 POINT(40.65366185224711 73.59645253961472) bank113134 +113135 POINT(40.59718851032768 73.03352600180152) bank113135 +113136 POINT(40.049079630581865 74.56660475480301) bank113136 +113137 POINT(39.783141941062354 73.07568766464871) bank113137 +113138 POINT(41.244436826842836 73.7437992752191) bank113138 +113139 POINT(40.58738047361102 73.82094607438182) bank113139 +113140 POINT(40.07167531059746 74.51222930873016) bank113140 +113141 POINT(41.19189326315744 73.81776001630774) bank113141 +113142 POINT(41.204342860428824 74.3813939823726) bank113142 +113143 POINT(40.141603055745904 74.17613273436824) bank113143 +113144 POINT(40.147264061115706 74.66256593034642) bank113144 +113145 POINT(41.31051278433582 74.24070430709097) bank113145 +113146 POINT(40.395065679213005 74.11519414858226) bank113146 +113147 POINT(41.6531017338255 73.60471052722949) bank113147 +113148 POINT(40.30869015732961 74.53450384830049) bank113148 +113149 POINT(41.08065044984056 73.20939846433764) bank113149 +113150 POINT(40.68745914327947 74.65534741706662) bank113150 +113151 POINT(39.767320647020725 73.29236570101682) bank113151 +113152 POINT(41.20855372229867 73.2117039969744) bank113152 +113153 POINT(41.20436315650926 73.99718892887898) bank113153 +113154 POINT(41.707961496896274 74.69923334420275) bank113154 +113155 POINT(41.358658871588716 73.31260440496729) bank113155 +113156 POINT(41.41671991893341 75.0040717474699) bank113156 +113157 POINT(40.04051387087244 73.98148620687893) bank113157 +113158 POINT(40.073708300508926 74.72491750689353) bank113158 +113159 POINT(40.615431880105895 74.40080785571685) bank113159 +113160 POINT(40.88772645954913 74.57872204581412) bank113160 +113161 POINT(41.46579878268598 73.2981204721869) bank113161 +113162 POINT(40.49369473695491 73.65902005253335) bank113162 +113163 POINT(41.646256773118836 74.58753635342921) bank113163 +113164 POINT(40.02147636274702 74.80412393068909) bank113164 +113165 POINT(40.697741007848414 74.535364394522) bank113165 +113166 POINT(40.114084919834575 73.65885461582488) bank113166 +113167 POINT(40.42154774692139 73.09651088114492) bank113167 +113168 POINT(41.1862431353706 74.4858593338365) bank113168 +113169 POINT(40.704843831629326 74.96720935894021) bank113169 +113170 POINT(40.45085487008239 74.45610773151849) bank113170 +113171 POINT(41.58896870799658 74.78510058904614) bank113171 +113172 POINT(40.3501886315646 74.29383838233846) bank113172 +113173 POINT(40.05139496180536 73.19625271548135) bank113173 +113174 POINT(40.15277396725835 74.42945795354623) bank113174 +113175 POINT(40.06750597167018 74.98822905785771) bank113175 +113176 POINT(40.1215481358293 74.60152126896254) bank113176 +113177 POINT(39.81990332483036 73.80885326613058) bank113177 +113178 POINT(41.484695369330986 74.01264362342818) bank113178 +113179 POINT(40.823625001462105 74.43443414931478) bank113179 +113180 POINT(39.82633263652992 73.58557089832257) bank113180 +113181 POINT(40.50341550636321 73.57731204877383) bank113181 +113182 POINT(40.10520987806296 73.89392083820366) bank113182 +113183 POINT(41.145954894138 73.42114860965931) bank113183 +113184 POINT(40.08537850039819 73.49916811637544) bank113184 +113185 POINT(40.2780451800384 73.64479436761528) bank113185 +113186 POINT(40.94498511913318 74.11724211558489) bank113186 +113187 POINT(41.484583604282086 73.67704163096644) bank113187 +113188 POINT(40.50554609703412 73.42414545251297) bank113188 +113189 POINT(40.203886513376546 73.14416260958791) bank113189 +113190 POINT(41.09106631751571 74.34186639194175) bank113190 +113191 POINT(41.52987133875851 73.39801948631047) bank113191 +113192 POINT(40.02782729501893 73.95363453487) bank113192 +113193 POINT(40.60791275444486 74.64135242976855) bank113193 +113194 POINT(39.81641564649595 74.66049305250827) bank113194 +113195 POINT(40.44637943229935 73.67417918906787) bank113195 +113196 POINT(40.598912668207625 73.89568715629864) bank113196 +113197 POINT(40.66176284712831 74.06223125617677) bank113197 +113198 POINT(40.1521936442088 73.98902110747476) bank113198 +113199 POINT(40.88106002544313 74.19561106876266) bank113199 +113200 POINT(39.774083618219265 73.09956441303598) bank113200 +113201 POINT(41.54633409044363 74.37124720041173) bank113201 +113202 POINT(41.45307300171778 73.96219287834774) bank113202 +113203 POINT(40.33248447888686 74.34970890073595) bank113203 +113204 POINT(41.61638099144405 74.32607304458757) bank113204 +113205 POINT(40.85843025159022 73.55208844233223) bank113205 +113206 POINT(40.03948144149541 73.50507390576588) bank113206 +113207 POINT(40.574915649463776 74.06730647403431) bank113207 +113208 POINT(40.2100891251429 73.90327797100538) bank113208 +113209 POINT(41.181077288094315 73.1617938315579) bank113209 +113210 POINT(41.585096863550724 74.2615126761856) bank113210 +113211 POINT(40.85622981113313 74.76584402203554) bank113211 +113212 POINT(41.45285702363049 73.69300052807604) bank113212 +113213 POINT(40.66196544613322 74.89358521320449) bank113213 +113214 POINT(40.3198690399606 73.83585379456824) bank113214 +113215 POINT(41.35494997991757 73.31841672965949) bank113215 +113216 POINT(40.27890296883659 73.08641410111805) bank113216 +113217 POINT(41.459755171506856 73.22564491505108) bank113217 +113218 POINT(39.74338612053323 73.93664633419661) bank113218 +113219 POINT(39.90671950888005 73.57409582066032) bank113219 +113220 POINT(40.506131298802714 74.6172786168775) bank113220 +113221 POINT(40.363236048057146 74.28519127731359) bank113221 +113222 POINT(40.0630147506902 74.23386132240928) bank113222 +113223 POINT(41.13314062562616 73.04449343273271) bank113223 +113224 POINT(40.79717456122527 74.4135537196402) bank113224 +113225 POINT(40.2381284558927 74.84119665073949) bank113225 +113226 POINT(41.4957148526739 74.6139851453502) bank113226 +113227 POINT(40.24009221818923 74.60700803016296) bank113227 +113228 POINT(40.17816912113525 74.88936759818169) bank113228 +113229 POINT(40.72952874048154 73.41159971431502) bank113229 +113230 POINT(41.40602991212532 74.7850954486612) bank113230 +113231 POINT(40.86855472389647 73.88159799669954) bank113231 +113232 POINT(40.247130848918665 74.72645074397214) bank113232 +113233 POINT(41.42592898740901 74.5111171602115) bank113233 +113234 POINT(41.1103142726999 73.65784140378176) bank113234 +113235 POINT(40.84056346781509 74.96831418545673) bank113235 +113236 POINT(40.711152757093984 73.11161284123304) bank113236 +113237 POINT(39.960436972171436 73.87817095552359) bank113237 +113238 POINT(41.26778444253213 74.76612613786283) bank113238 +113239 POINT(40.43982826492056 74.00243029664189) bank113239 +113240 POINT(41.23788043452042 74.50370934544439) bank113240 +113241 POINT(40.12287205206301 73.50023498687693) bank113241 +113242 POINT(39.935754803680055 74.93388809004276) bank113242 +113243 POINT(41.377628190642156 73.73079666273847) bank113243 +113244 POINT(40.088830102312045 73.58435027369252) bank113244 +113245 POINT(39.95178379879614 74.69191448037212) bank113245 +113246 POINT(40.773457465610825 73.42180883435078) bank113246 +113247 POINT(41.697953035428675 73.80011550655954) bank113247 +113248 POINT(41.41438488532035 73.55026303995146) bank113248 +113249 POINT(40.95840560312831 73.16054197199345) bank113249 +113250 POINT(40.00224848754232 73.71629425822958) bank113250 +113251 POINT(40.98718757167508 73.75645030038511) bank113251 +113252 POINT(40.64465194209269 73.69435799042617) bank113252 +113253 POINT(40.97335632003054 73.0618350889487) bank113253 +113254 POINT(40.12529264482494 74.06059897135664) bank113254 +113255 POINT(39.93909503398572 73.35960658324893) bank113255 +113256 POINT(40.81442813127708 73.35412961258379) bank113256 +113257 POINT(40.77166966008074 73.37398787577041) bank113257 +113258 POINT(40.785438198561906 73.79568058630855) bank113258 +113259 POINT(41.22151575205805 74.98492206508911) bank113259 +113260 POINT(40.537783716508194 74.1516702290813) bank113260 +113261 POINT(40.15345088365885 73.4653423972134) bank113261 +113262 POINT(41.55511585560181 73.07694629032848) bank113262 +113263 POINT(40.51461989721502 73.52443482861638) bank113263 +113264 POINT(40.04351967720842 74.26181443435432) bank113264 +113265 POINT(41.39519503423738 74.33878282656761) bank113265 +113266 POINT(41.029713885330246 73.93970484607303) bank113266 +113267 POINT(40.7360023277543 73.56515637462674) bank113267 +113268 POINT(41.52086321749883 73.05185506671377) bank113268 +113269 POINT(41.55018366141427 73.14416947791426) bank113269 +113270 POINT(41.244526838898786 74.15645145093282) bank113270 +113271 POINT(40.8637403569433 74.68973896071171) bank113271 +113272 POINT(40.32419102234733 74.57332472652473) bank113272 +113273 POINT(41.03746632184656 74.41411929477611) bank113273 +113274 POINT(41.46254701065594 73.87092416482399) bank113274 +113275 POINT(39.99232352272074 73.26595366175286) bank113275 +113276 POINT(41.59594943128308 74.05347029308041) bank113276 +113277 POINT(41.290486101681985 74.60889314287628) bank113277 +113278 POINT(41.06248605649395 74.4534376018962) bank113278 +113279 POINT(39.718262984249634 74.05875246273246) bank113279 +113280 POINT(40.56449730067956 73.84385246032386) bank113280 +113281 POINT(40.769287584942525 73.79879210150294) bank113281 +113282 POINT(40.67630498975579 73.60612423135949) bank113282 +113283 POINT(40.962968633637814 74.65963670001955) bank113283 +113284 POINT(41.32986691834678 74.07469983711194) bank113284 +113285 POINT(39.82548435115418 74.58464046704505) bank113285 +113286 POINT(39.80410847891241 73.11459352451402) bank113286 +113287 POINT(40.070622446487576 73.89528203831127) bank113287 +113288 POINT(41.351371812331635 73.72402503311802) bank113288 +113289 POINT(41.51353594709045 73.32898243281171) bank113289 +113290 POINT(40.46713148425289 73.94509849075992) bank113290 +113291 POINT(41.05251999281845 74.66226662699165) bank113291 +113292 POINT(41.06114974763584 74.13162938830996) bank113292 +113293 POINT(40.55084877151336 74.5006921358741) bank113293 +113294 POINT(40.56200200277772 73.58329891215719) bank113294 +113295 POINT(40.8963123388406 74.5805313792358) bank113295 +113296 POINT(40.41265966922392 74.05534607048095) bank113296 +113297 POINT(40.36667140061587 73.35981112741703) bank113297 +113298 POINT(40.5489484026019 74.64458491273663) bank113298 +113299 POINT(40.495907463834556 74.15594871462936) bank113299 +113300 POINT(40.64815364844419 73.2892540210676) bank113300 +113301 POINT(41.14716489393995 74.438441729078) bank113301 +113302 POINT(40.02674745446225 73.11248342750976) bank113302 +113303 POINT(40.612621027545046 74.39131053102925) bank113303 +113304 POINT(40.76020357962119 74.22525636257527) bank113304 +113305 POINT(40.44056681915713 73.3987252285775) bank113305 +113306 POINT(41.56879774699743 73.5094034856789) bank113306 +113307 POINT(41.35931393612058 74.24945321599766) bank113307 +113308 POINT(40.71475783647285 74.1332882470088) bank113308 +113309 POINT(40.743613317877454 73.9558345081782) bank113309 +113310 POINT(40.04576956251289 73.93818535043695) bank113310 +113311 POINT(40.67032257096594 74.52253619333504) bank113311 +113312 POINT(39.80205188042354 74.97098770446482) bank113312 +113313 POINT(41.45665333093617 74.20525819177055) bank113313 +113314 POINT(39.92265833459789 74.45613610885509) bank113314 +113315 POINT(41.49252180900752 74.29362269508826) bank113315 +113316 POINT(41.45548967299739 73.44728545896744) bank113316 +113317 POINT(41.20505603117333 73.3856348804671) bank113317 +113318 POINT(39.92749520947541 73.41133183380002) bank113318 +113319 POINT(40.023481553504716 74.96399042115625) bank113319 +113320 POINT(40.65869621650473 74.2112290126806) bank113320 +113321 POINT(41.0605750497929 74.22339527347143) bank113321 +113322 POINT(40.34750336020933 74.25599463943642) bank113322 +113323 POINT(40.978281729673775 73.58017593939549) bank113323 +113324 POINT(40.20766259103617 73.13300958095192) bank113324 +113325 POINT(40.90719224327862 73.95837261085312) bank113325 +113326 POINT(41.6022778949872 74.4527969253842) bank113326 +113327 POINT(39.815682789143096 74.771545550904) bank113327 +113328 POINT(40.67701950387962 74.7099605920748) bank113328 +113329 POINT(40.73402917105899 73.8243706697554) bank113329 +113330 POINT(41.41072652335928 74.63815789414357) bank113330 +113331 POINT(41.557246470199665 73.90695011353779) bank113331 +113332 POINT(41.19138249921763 73.4610414961129) bank113332 +113333 POINT(40.19442579694479 74.66388627235693) bank113333 +113334 POINT(39.96771994599396 74.76422609701358) bank113334 +113335 POINT(40.70369901258038 73.22054628739836) bank113335 +113336 POINT(40.30329196239436 73.25034340543444) bank113336 +113337 POINT(39.753743934899816 73.52624510870798) bank113337 +113338 POINT(39.783283035819025 74.58460245055768) bank113338 +113339 POINT(39.769056571730445 74.57225646954772) bank113339 +113340 POINT(41.63930108384556 74.93215187026829) bank113340 +113341 POINT(40.89175018001342 73.80269872396387) bank113341 +113342 POINT(40.95416139923117 74.89437720594935) bank113342 +113343 POINT(40.351676828724514 74.68391373799855) bank113343 +113344 POINT(39.73366096410799 73.67423954534624) bank113344 +113345 POINT(41.435049138551186 73.56938382663134) bank113345 +113346 POINT(41.0382585907609 73.57686411615857) bank113346 +113347 POINT(40.10835041099699 73.59795739715794) bank113347 +113348 POINT(39.935819378204144 74.69405923295372) bank113348 +113349 POINT(40.933128507404234 73.02170164640033) bank113349 +113350 POINT(41.640470471714366 73.82617709807882) bank113350 +113351 POINT(41.47931729234883 73.33749354537755) bank113351 +113352 POINT(40.214348177041906 73.83728997064308) bank113352 +113353 POINT(41.458001606849514 74.08835791600062) bank113353 +113354 POINT(41.37959732065746 73.7696221249995) bank113354 +113355 POINT(40.981902216740195 74.53731783466249) bank113355 +113356 POINT(41.522644864568335 73.32388097690965) bank113356 +113357 POINT(40.25286286990842 74.53967721460255) bank113357 +113358 POINT(41.21492756671811 73.35976475461845) bank113358 +113359 POINT(41.282214593749345 73.5082261394258) bank113359 +113360 POINT(40.28118996878098 74.17746123108482) bank113360 +113361 POINT(40.04762858991357 74.26666773689286) bank113361 +113362 POINT(40.52278529297285 73.2406388409201) bank113362 +113363 POINT(40.542302210410014 73.75469059461966) bank113363 +113364 POINT(39.988049693451075 73.37176701408887) bank113364 +113365 POINT(41.163721382951465 74.74742898379507) bank113365 +113366 POINT(40.94341864272107 73.08709812766912) bank113366 +113367 POINT(40.47242110425367 74.66648197954177) bank113367 +113368 POINT(40.35659683812073 73.26490260340529) bank113368 +113369 POINT(41.4867849118251 73.44598985664693) bank113369 +113370 POINT(40.71333496971897 74.75483501988408) bank113370 +113371 POINT(39.92425353116239 73.56572989161673) bank113371 +113372 POINT(40.78520961724899 73.39418817298423) bank113372 +113373 POINT(40.42287808850309 74.36078042588419) bank113373 +113374 POINT(41.06691988611515 74.4305048094238) bank113374 +113375 POINT(40.72180737002366 73.64651894402233) bank113375 +113376 POINT(41.302613648554825 73.31108908841894) bank113376 +113377 POINT(41.540254423768474 73.5953361803065) bank113377 +113378 POINT(41.014736807413364 73.86710805573168) bank113378 +113379 POINT(41.68044438273861 73.0561336360356) bank113379 +113380 POINT(41.33220409995629 74.73993906915399) bank113380 +113381 POINT(39.97914772035634 74.06188288199002) bank113381 +113382 POINT(39.93753329071441 74.82004753800437) bank113382 +113383 POINT(41.71068496380027 74.25409485869551) bank113383 +113384 POINT(41.55084914742439 74.70993639125574) bank113384 +113385 POINT(41.481298701985075 73.73126168539164) bank113385 +113386 POINT(40.31604480532742 74.25169222084409) bank113386 +113387 POINT(40.199977698249825 74.98049110981282) bank113387 +113388 POINT(40.58927833828582 74.03109770599622) bank113388 +113389 POINT(41.607736802258486 74.89870955993713) bank113389 +113390 POINT(39.985055194127796 73.62015873053306) bank113390 +113391 POINT(40.551188658797564 74.6186310578399) bank113391 +113392 POINT(40.10498180153293 74.46888644643397) bank113392 +113393 POINT(40.40621954666885 73.20337076407438) bank113393 +113394 POINT(40.153468183482005 74.06525221735812) bank113394 +113395 POINT(41.69228840145365 74.81465909308018) bank113395 +113396 POINT(41.64430780093966 73.7070391600118) bank113396 +113397 POINT(40.88252447512118 73.15073772994832) bank113397 +113398 POINT(40.96494738182981 74.39860741398508) bank113398 +113399 POINT(39.85845365394998 74.11621675128453) bank113399 +113400 POINT(41.66843030927265 73.05435989934595) bank113400 +113401 POINT(40.857146413297855 74.30014452667363) bank113401 +113402 POINT(41.35383850137516 74.13732119734439) bank113402 +113403 POINT(40.31632257462157 73.07038948061609) bank113403 +113404 POINT(40.097309131900296 73.78227275604823) bank113404 +113405 POINT(39.81373121789932 73.99983876737016) bank113405 +113406 POINT(41.11480434336525 73.28192877693952) bank113406 +113407 POINT(39.98787565326875 74.73740198399003) bank113407 +113408 POINT(40.18298190588047 74.98709508661439) bank113408 +113409 POINT(41.05336388947869 74.37155916545093) bank113409 +113410 POINT(41.45406748294607 74.12776980562312) bank113410 +113411 POINT(40.450037746986276 73.33785557433823) bank113411 +113412 POINT(39.95362229403403 74.30123403949638) bank113412 +113413 POINT(40.99270497206947 74.66557751923209) bank113413 +113414 POINT(41.58999055150957 73.31077707506911) bank113414 +113415 POINT(40.25703038146303 74.50724931442983) bank113415 +113416 POINT(41.4036381513284 73.64609093087313) bank113416 +113417 POINT(40.186043554343996 74.14568930498042) bank113417 +113418 POINT(41.36736818540706 74.93296618994476) bank113418 +113419 POINT(40.73159971228423 73.96806953453948) bank113419 +113420 POINT(40.40151708394426 73.97982118796584) bank113420 +113421 POINT(40.406696342062105 73.5572641757385) bank113421 +113422 POINT(40.13060089836306 73.16918763210744) bank113422 +113423 POINT(41.60733694552672 73.68141465121006) bank113423 +113424 POINT(40.72320879892333 74.35397558158289) bank113424 +113425 POINT(40.01157883444009 74.54844106778003) bank113425 +113426 POINT(41.60432267462563 73.08864159232493) bank113426 +113427 POINT(41.57330274837386 74.23234072709117) bank113427 +113428 POINT(40.94932082318469 73.52877165680694) bank113428 +113429 POINT(39.959341231347274 74.92561404145172) bank113429 +113430 POINT(40.20330899581071 74.62067067114945) bank113430 +113431 POINT(41.10890844649184 74.28898712199772) bank113431 +113432 POINT(41.37718047044235 73.78627452479746) bank113432 +113433 POINT(41.55943178976521 73.90036007422924) bank113433 +113434 POINT(41.10753083385739 73.2300258042203) bank113434 +113435 POINT(41.48895911191386 73.61563551868518) bank113435 +113436 POINT(40.52033384338866 74.92239249932548) bank113436 +113437 POINT(40.36243798392542 73.3511716686523) bank113437 +113438 POINT(40.2179255317567 74.90707969327639) bank113438 +113439 POINT(40.51128517857421 73.5290005608246) bank113439 +113440 POINT(39.88210690094501 74.42266532763303) bank113440 +113441 POINT(40.545503679743405 73.15272901127949) bank113441 +113442 POINT(41.05201591428388 73.9303911030981) bank113442 +113443 POINT(40.34788780163787 74.54003062831048) bank113443 +113444 POINT(41.279696364458026 73.14517507700836) bank113444 +113445 POINT(40.38183765814289 73.2893124214203) bank113445 +113446 POINT(40.37396277427341 73.04027944075496) bank113446 +113447 POINT(40.87274086270266 73.77027917099299) bank113447 +113448 POINT(40.19582620982571 74.96865668931315) bank113448 +113449 POINT(40.94496262378069 74.86498271066279) bank113449 +113450 POINT(39.89227191804839 73.594536890673) bank113450 +113451 POINT(40.769494109366036 74.0520214849346) bank113451 +113452 POINT(40.06346554380051 74.54713459055733) bank113452 +113453 POINT(40.86798610780467 73.85219916631408) bank113453 +113454 POINT(41.512263366680976 73.6562223076393) bank113454 +113455 POINT(41.085135285615685 73.25175715178682) bank113455 +113456 POINT(40.830065120418986 74.10456319271039) bank113456 +113457 POINT(39.767941697531306 74.62313191306588) bank113457 +113458 POINT(39.8006232426337 74.88030079443527) bank113458 +113459 POINT(41.10941268395472 73.14249903016314) bank113459 +113460 POINT(40.77155307132114 74.45851269592343) bank113460 +113461 POINT(41.10522821422774 73.34053442160912) bank113461 +113462 POINT(40.20483724219523 74.62706501367704) bank113462 +113463 POINT(41.262554597828924 73.02796468032506) bank113463 +113464 POINT(39.90386068548248 73.66458879418573) bank113464 +113465 POINT(41.19198298737633 73.61413131531619) bank113465 +113466 POINT(41.36026015549062 74.63584148983456) bank113466 +113467 POINT(40.88961341373794 73.17841531555965) bank113467 +113468 POINT(39.938285637799005 74.11027356493953) bank113468 +113469 POINT(40.5994574076475 73.58056380937352) bank113469 +113470 POINT(40.67553951444401 74.39250430663344) bank113470 +113471 POINT(40.82326145698199 74.6267074868835) bank113471 +113472 POINT(40.34258558061448 73.30028643533446) bank113472 +113473 POINT(40.85042113724289 74.50359570187595) bank113473 +113474 POINT(40.33589457921696 73.02248265041263) bank113474 +113475 POINT(39.79529673336747 73.73677200972115) bank113475 +113476 POINT(40.693134951305126 73.26813867104806) bank113476 +113477 POINT(39.971829119868374 74.87771352026579) bank113477 +113478 POINT(40.66661535032331 74.99117974671424) bank113478 +113479 POINT(39.77535085452559 74.18609525876309) bank113479 +113480 POINT(40.25694231580754 73.09638093223809) bank113480 +113481 POINT(40.91721669914411 73.62705753973404) bank113481 +113482 POINT(40.5693121233416 73.9490881009662) bank113482 +113483 POINT(41.226413242529794 74.03287795888131) bank113483 +113484 POINT(40.16414286812745 74.65883259636495) bank113484 +113485 POINT(41.673547161990655 74.02278436363196) bank113485 +113486 POINT(41.17841085645529 74.20504281159091) bank113486 +113487 POINT(40.85481640328912 73.68952577170157) bank113487 +113488 POINT(40.35826989152464 73.17920132241771) bank113488 +113489 POINT(40.615540394639815 73.2575989522848) bank113489 +113490 POINT(40.671619092162004 74.85297367880669) bank113490 +113491 POINT(40.840831859485675 73.71583551968087) bank113491 +113492 POINT(40.45854449640573 73.4437987559001) bank113492 +113493 POINT(41.10546437034416 74.74414259537372) bank113493 +113494 POINT(40.06286982868886 74.51503730939078) bank113494 +113495 POINT(40.18192172040162 73.50595044857049) bank113495 +113496 POINT(40.70112243548146 73.40686297353163) bank113496 +113497 POINT(39.79492091970188 73.06992239309066) bank113497 +113498 POINT(40.84102069219013 74.21270523901839) bank113498 +113499 POINT(41.42762114839203 74.96637271703682) bank113499 +113500 POINT(40.26833655850749 74.97877277636064) bank113500 +113501 POINT(39.87910416972708 73.43315443499824) bank113501 +113502 POINT(40.063362466659285 73.9984716885473) bank113502 +113503 POINT(40.71449204446248 73.58072134588177) bank113503 +113504 POINT(41.382431928703085 73.0834026211841) bank113504 +113505 POINT(39.841625678889606 73.5594315365672) bank113505 +113506 POINT(39.90905648158626 74.17857498704345) bank113506 +113507 POINT(41.07162232970231 73.39310381011359) bank113507 +113508 POINT(40.54175635152766 73.1572659892075) bank113508 +113509 POINT(40.36925287328781 74.41226186282773) bank113509 +113510 POINT(39.828348179848575 74.67482805550955) bank113510 +113511 POINT(40.337701723956904 74.77990351394618) bank113511 +113512 POINT(39.9079199846971 73.45939460071615) bank113512 +113513 POINT(39.94060239126492 74.15472196095993) bank113513 +113514 POINT(41.19675481978556 74.80732388628186) bank113514 +113515 POINT(40.15443059030929 73.87362484261193) bank113515 +113516 POINT(41.047180576883186 73.76832060199672) bank113516 +113517 POINT(40.02722878272983 74.45714285312091) bank113517 +113518 POINT(40.1395908566591 73.34289515986596) bank113518 +113519 POINT(40.08463869678445 74.64958195931997) bank113519 +113520 POINT(40.69474639655805 74.44992281764715) bank113520 +113521 POINT(40.91657694783543 73.5947645058678) bank113521 +113522 POINT(40.38230266269111 74.22623440461902) bank113522 +113523 POINT(40.38961704206294 73.52442205887344) bank113523 +113524 POINT(41.049404143823644 73.53488540017545) bank113524 +113525 POINT(39.97035906937912 74.88403438017666) bank113525 +113526 POINT(41.67019996460304 73.39750879810596) bank113526 +113527 POINT(39.82821643054713 73.59558344428301) bank113527 +113528 POINT(41.67141988299118 73.02663949768696) bank113528 +113529 POINT(40.97164682134473 73.51759112926666) bank113529 +113530 POINT(40.010262415657344 74.93283594063253) bank113530 +113531 POINT(41.20688746135815 73.59487876907971) bank113531 +113532 POINT(40.05917163055664 74.48254043096631) bank113532 +113533 POINT(41.69323942271096 73.41071951022003) bank113533 +113534 POINT(40.0765630084959 74.2514788352295) bank113534 +113535 POINT(40.33977016611786 74.81820073257049) bank113535 +113536 POINT(40.91141745291602 73.28168598676916) bank113536 +113537 POINT(40.16552403605238 74.56501109006791) bank113537 +113538 POINT(39.95015391652694 74.11058437242839) bank113538 +113539 POINT(39.76684337495141 73.690717074291) bank113539 +113540 POINT(41.23878117995895 73.33805777065169) bank113540 +113541 POINT(41.42080973616393 73.15223445663275) bank113541 +113542 POINT(39.978146877989204 73.0961503603854) bank113542 +113543 POINT(40.707800255034684 74.81351519153843) bank113543 +113544 POINT(41.35794075263104 73.65922577439049) bank113544 +113545 POINT(40.859889423425 74.15733890088018) bank113545 +113546 POINT(40.902157750161564 74.83472345944652) bank113546 +113547 POINT(41.67946734396013 74.13662372754433) bank113547 +113548 POINT(40.915085209421726 74.38020204097141) bank113548 +113549 POINT(41.52463273405013 74.93252158025321) bank113549 +113550 POINT(41.58961200882723 73.99307919182621) bank113550 +113551 POINT(40.34660742013956 73.68106514860206) bank113551 +113552 POINT(39.87990146878392 73.40084565463022) bank113552 +113553 POINT(41.476937443088026 74.0548604509463) bank113553 +113554 POINT(40.2421861361701 73.30887512683906) bank113554 +113555 POINT(41.685042553191956 73.70881885287329) bank113555 +113556 POINT(40.30199329169602 73.68474937809722) bank113556 +113557 POINT(40.81283232684819 74.96759271207473) bank113557 +113558 POINT(41.24938500203674 74.94359588438486) bank113558 +113559 POINT(40.33833899930133 73.02780774367352) bank113559 +113560 POINT(41.25678945997026 74.63041185106825) bank113560 +113561 POINT(40.88343148024262 74.90830125859576) bank113561 +113562 POINT(40.80826423471899 74.02355603988687) bank113562 +113563 POINT(40.3474708208623 74.23800198223269) bank113563 +113564 POINT(41.591891237712225 74.80955530181188) bank113564 +113565 POINT(40.67842574514914 73.18965119692564) bank113565 +113566 POINT(41.1047733623565 74.04186267164724) bank113566 +113567 POINT(40.49667945119718 74.48168562572258) bank113567 +113568 POINT(40.20690378406762 73.86737631595645) bank113568 +113569 POINT(40.378484726289415 74.62735534566892) bank113569 +113570 POINT(40.5726027223988 73.59262594920105) bank113570 +113571 POINT(40.15919051826146 74.90148365615862) bank113571 +113572 POINT(40.367093682865956 73.85188796941871) bank113572 +113573 POINT(41.085430101252285 74.39926096149925) bank113573 +113574 POINT(41.455385561210164 73.92499770614269) bank113574 +113575 POINT(41.54673026234281 74.91768875882225) bank113575 +113576 POINT(41.053023401622156 74.62649670426869) bank113576 +113577 POINT(40.304079858928695 74.67378272937529) bank113577 +113578 POINT(40.86197469808486 73.96474581830796) bank113578 +113579 POINT(41.13179988974799 74.36524567173473) bank113579 +113580 POINT(39.84612808308518 74.12142683883673) bank113580 +113581 POINT(40.60002997461283 74.361252063252) bank113581 +113582 POINT(39.98727871530647 73.72703602930176) bank113582 +113583 POINT(39.95036923560914 74.95985452793754) bank113583 +113584 POINT(40.987968732408824 74.6042409052682) bank113584 +113585 POINT(41.164041954485015 74.68841182428925) bank113585 +113586 POINT(39.98484364464711 73.44793174458906) bank113586 +113587 POINT(41.371543205089296 73.60067551516546) bank113587 +113588 POINT(40.07628614721735 73.40541105240713) bank113588 +113589 POINT(40.330445522867805 73.0488588392357) bank113589 +113590 POINT(41.67674612748094 74.24402005109553) bank113590 +113591 POINT(40.7677434740209 74.05366700451394) bank113591 +113592 POINT(40.26940565440915 73.45718284670755) bank113592 +113593 POINT(40.630720799056036 74.49166557880832) bank113593 +113594 POINT(40.87511094985638 73.84255624562795) bank113594 +113595 POINT(40.73657230232854 73.06150891791604) bank113595 +113596 POINT(40.23350050429074 74.76647785634627) bank113596 +113597 POINT(40.841556837649016 74.73166354623858) bank113597 +113598 POINT(39.79708946511542 73.95038014207698) bank113598 +113599 POINT(40.63451420349259 73.52105270962737) bank113599 +113600 POINT(41.107369274729926 74.92367522336957) bank113600 +113601 POINT(40.22978678101564 73.2550068362614) bank113601 +113602 POINT(41.18004729363929 74.40896847254793) bank113602 +113603 POINT(41.494885609798835 73.14938400586598) bank113603 +113604 POINT(41.24081564182344 74.02454916420929) bank113604 +113605 POINT(41.189731661633765 73.49256092388674) bank113605 +113606 POINT(40.41398376952397 73.17318763617786) bank113606 +113607 POINT(41.042776002751914 74.89928598894899) bank113607 +113608 POINT(40.595824154364735 74.51455556884076) bank113608 +113609 POINT(40.35326676467033 73.32392272703608) bank113609 +113610 POINT(40.190953766408896 74.69104285745435) bank113610 +113611 POINT(40.45827540658011 73.98268687518514) bank113611 +113612 POINT(41.55689091999784 73.91977761111596) bank113612 +113613 POINT(39.96520536255691 74.07992036316472) bank113613 +113614 POINT(39.93488129283544 73.97893835259366) bank113614 +113615 POINT(40.79608171293407 74.8652103737989) bank113615 +113616 POINT(40.933726219143125 73.16796773141589) bank113616 +113617 POINT(41.60587190630087 74.13496501109528) bank113617 +113618 POINT(40.36215903388176 74.81130925367387) bank113618 +113619 POINT(40.171864257687076 73.3272579023188) bank113619 +113620 POINT(40.46571921705742 73.47902785053272) bank113620 +113621 POINT(40.78152540982946 73.74168790137865) bank113621 +113622 POINT(40.413508218976354 74.7103851633742) bank113622 +113623 POINT(40.44390070865857 74.89810574305156) bank113623 +113624 POINT(39.725539611389934 74.10740462108515) bank113624 +113625 POINT(41.492334742376684 73.60724355154439) bank113625 +113626 POINT(39.88871380316768 73.58289203973449) bank113626 +113627 POINT(39.91916787487527 74.5277706069152) bank113627 +113628 POINT(41.62233241737802 74.94617435070175) bank113628 +113629 POINT(40.673589257508254 73.15119088865407) bank113629 +113630 POINT(39.79632044477362 73.72620485235167) bank113630 +113631 POINT(40.761162874856666 74.48874780219681) bank113631 +113632 POINT(40.93034327285247 74.19652005081996) bank113632 +113633 POINT(40.62593740245022 74.99000586813837) bank113633 +113634 POINT(40.481391211624356 73.10204273072975) bank113634 +113635 POINT(41.09100260635497 74.23697256990157) bank113635 +113636 POINT(41.27243359829545 74.22760020278974) bank113636 +113637 POINT(40.858530781106644 73.05948031472082) bank113637 +113638 POINT(40.50873600889696 74.52934609959506) bank113638 +113639 POINT(40.095193572151686 74.29224715181599) bank113639 +113640 POINT(41.50152206160098 74.14035197507151) bank113640 +113641 POINT(41.000651539980986 73.15777266121123) bank113641 +113642 POINT(41.03859373203827 73.28211257794688) bank113642 +113643 POINT(39.85086469472542 73.03669739777864) bank113643 +113644 POINT(40.88255217173445 74.54179386290804) bank113644 +113645 POINT(39.77032078655856 73.71418719874087) bank113645 +113646 POINT(41.35603386134703 73.5206192405963) bank113646 +113647 POINT(40.96986894948784 73.51434927736683) bank113647 +113648 POINT(39.80702458281345 74.97044942168048) bank113648 +113649 POINT(41.611206814506566 74.4771668419019) bank113649 +113650 POINT(41.40992617080621 74.52961726167378) bank113650 +113651 POINT(40.90318833972419 74.95414477550896) bank113651 +113652 POINT(39.83995109021941 73.9745325468274) bank113652 +113653 POINT(40.83707504199641 73.64042491205424) bank113653 +113654 POINT(40.92335593827411 73.10171862179253) bank113654 +113655 POINT(39.74497804949756 73.35073504049815) bank113655 +113656 POINT(40.025673226809914 73.62003184497019) bank113656 +113657 POINT(40.18785006716228 73.67021907331751) bank113657 +113658 POINT(40.640242311603934 73.07100658529083) bank113658 +113659 POINT(41.23108832882816 73.8698259279336) bank113659 +113660 POINT(40.33277889194061 73.61996927099567) bank113660 +113661 POINT(40.47441501075812 73.98797599785642) bank113661 +113662 POINT(41.46620223802233 75.00122992448493) bank113662 +113663 POINT(41.36641101147093 73.55834523876847) bank113663 +113664 POINT(40.310768464350545 73.05500191442029) bank113664 +113665 POINT(39.738661667263244 74.02069672214701) bank113665 +113666 POINT(40.29615384570135 74.4266532696777) bank113666 +113667 POINT(41.38967271903661 73.4401386086239) bank113667 +113668 POINT(40.026539354598704 73.12576497415597) bank113668 +113669 POINT(41.03200576849501 73.36465051161906) bank113669 +113670 POINT(40.27387367074969 73.58922474992447) bank113670 +113671 POINT(41.06037550143867 74.1954254576419) bank113671 +113672 POINT(39.73089933472904 73.24707738542867) bank113672 +113673 POINT(41.67895608588258 73.92785179528258) bank113673 +113674 POINT(39.79707231646353 73.8388527474028) bank113674 +113675 POINT(39.74651177944217 74.8607050224169) bank113675 +113676 POINT(40.545453198574506 73.37185389037624) bank113676 +113677 POINT(40.219768870037 73.15771116874353) bank113677 +113678 POINT(41.00066344889475 74.78701762422678) bank113678 +113679 POINT(40.11146800760885 74.44828733419786) bank113679 +113680 POINT(40.01780597483003 73.30981917505076) bank113680 +113681 POINT(40.867554597054195 73.14399781157137) bank113681 +113682 POINT(40.50751994366742 74.24951362219834) bank113682 +113683 POINT(39.91986808650466 74.86076736419481) bank113683 +113684 POINT(39.84741530694972 73.98181149004661) bank113684 +113685 POINT(41.49340767778016 73.96355187967652) bank113685 +113686 POINT(40.385972410739214 74.54507014883917) bank113686 +113687 POINT(41.284305592027955 73.7417709151138) bank113687 +113688 POINT(41.6149616267593 74.00532477500238) bank113688 +113689 POINT(40.22996815607809 73.0367232635643) bank113689 +113690 POINT(41.43926470452009 73.94713007572274) bank113690 +113691 POINT(40.50056973696818 74.99747835775072) bank113691 +113692 POINT(41.439139488831444 74.67553143695572) bank113692 +113693 POINT(41.61145879142185 73.3443315041994) bank113693 +113694 POINT(41.19986740693838 73.8643600664831) bank113694 +113695 POINT(40.28110594862794 74.80269728147618) bank113695 +113696 POINT(41.051038962369645 73.24399071711305) bank113696 +113697 POINT(41.208504314485644 74.45148790635598) bank113697 +113698 POINT(40.88108954527535 73.95022301888025) bank113698 +113699 POINT(40.11344043618152 73.68791099011527) bank113699 +113700 POINT(41.449740394235654 73.88909541401058) bank113700 +113701 POINT(40.017101297649006 74.1200242950646) bank113701 +113702 POINT(41.56186483522514 73.14603218338254) bank113702 +113703 POINT(41.591269309979545 73.94274118882426) bank113703 +113704 POINT(41.412409263140525 74.12759915288723) bank113704 +113705 POINT(40.87965973317735 74.977443412673) bank113705 +113706 POINT(40.72696895430685 74.4626113429574) bank113706 +113707 POINT(39.832687573125185 73.25035519399212) bank113707 +113708 POINT(40.78857876322725 73.59388199777939) bank113708 +113709 POINT(39.84321033915657 73.0526713029422) bank113709 +113710 POINT(41.6714925907151 73.61246427869476) bank113710 +113711 POINT(40.69738390883428 73.86138870343922) bank113711 +113712 POINT(41.32125869025538 74.93752415375197) bank113712 +113713 POINT(40.540300001390236 74.40459144748583) bank113713 +113714 POINT(41.61108380717342 74.81909482106055) bank113714 +113715 POINT(41.12790599284544 73.99719873968185) bank113715 +113716 POINT(40.24425235187162 73.6718264874342) bank113716 +113717 POINT(41.044605937041645 74.38800202554894) bank113717 +113718 POINT(39.98964834443767 73.11217946151618) bank113718 +113719 POINT(41.537183816277235 74.12581731471434) bank113719 +113720 POINT(40.65847980899929 74.45655692891049) bank113720 +113721 POINT(40.91785603264602 73.8307410665682) bank113721 +113722 POINT(39.90843920192332 73.29204548310102) bank113722 +113723 POINT(39.97327965859286 73.24685818189039) bank113723 +113724 POINT(39.71944606786307 73.74983557418365) bank113724 +113725 POINT(40.726086643827145 73.34376608610144) bank113725 +113726 POINT(40.674656574546475 74.54539741703282) bank113726 +113727 POINT(40.567908489186685 73.20469840323854) bank113727 +113728 POINT(41.28435072768668 74.6753248406402) bank113728 +113729 POINT(40.773737965900025 74.66710715269072) bank113729 +113730 POINT(41.11807405803507 73.97791060433254) bank113730 +113731 POINT(40.163477931177574 74.17847493626603) bank113731 +113732 POINT(41.69510159291597 74.5253743004258) bank113732 +113733 POINT(41.075986760261515 73.87591778868) bank113733 +113734 POINT(41.67215426148922 74.17329437882107) bank113734 +113735 POINT(40.680760602892995 74.95292982642856) bank113735 +113736 POINT(40.692931829778274 74.4761008269439) bank113736 +113737 POINT(40.673089629854374 74.19127546776413) bank113737 +113738 POINT(40.684046823367694 73.17262203661711) bank113738 +113739 POINT(41.149631498985165 73.99309483792359) bank113739 +113740 POINT(39.92254885129336 74.55863712952625) bank113740 +113741 POINT(41.61957198684106 73.86455403217681) bank113741 +113742 POINT(40.23251860873632 73.95173590362182) bank113742 +113743 POINT(40.72860888158008 73.50163116934294) bank113743 +113744 POINT(40.46704941492532 73.4757089805045) bank113744 +113745 POINT(40.02452351331672 73.28802019287211) bank113745 +113746 POINT(40.52095698013332 74.98781232990757) bank113746 +113747 POINT(41.036660494392834 73.26343765782303) bank113747 +113748 POINT(40.65977540882335 73.43709305551651) bank113748 +113749 POINT(40.84557949592667 73.40763705269046) bank113749 +113750 POINT(41.611911742467 73.59792681863821) bank113750 +113751 POINT(39.94353138476994 74.01224615153926) bank113751 +113752 POINT(40.043409809762686 74.73244454749097) bank113752 +113753 POINT(40.884130635692316 73.97036837250246) bank113753 +113754 POINT(40.107430571187734 73.96114938638465) bank113754 +113755 POINT(41.25618704212012 73.98226921744124) bank113755 +113756 POINT(40.56081179457326 74.43012219971382) bank113756 +113757 POINT(41.62190175967441 74.22283097050432) bank113757 +113758 POINT(41.09725096768659 74.56811433044777) bank113758 +113759 POINT(40.5310472917816 74.99876820195163) bank113759 +113760 POINT(41.02783108352952 73.99993196383053) bank113760 +113761 POINT(39.7538387866777 74.55923396980228) bank113761 +113762 POINT(41.00128580792072 73.9957354753669) bank113762 +113763 POINT(40.30594855935219 74.32571298603057) bank113763 +113764 POINT(40.48928887083463 73.7509359708118) bank113764 +113765 POINT(40.617659264255714 74.69557452827206) bank113765 +113766 POINT(40.22472708698618 73.91520653058329) bank113766 +113767 POINT(39.843541537126455 74.7520346432146) bank113767 +113768 POINT(41.203404290009644 73.35867955918226) bank113768 +113769 POINT(41.21339421280006 74.31238080447933) bank113769 +113770 POINT(41.16209387516805 74.2884269419803) bank113770 +113771 POINT(41.25982636540285 73.96448682887294) bank113771 +113772 POINT(40.86763649058596 74.51611099314597) bank113772 +113773 POINT(40.528307883959265 73.55056303077308) bank113773 +113774 POINT(40.44521573809095 74.0214549033069) bank113774 +113775 POINT(40.058411964862024 73.330795685584) bank113775 +113776 POINT(41.32062982952266 74.44211814347787) bank113776 +113777 POINT(40.594627198629766 74.16712361499926) bank113777 +113778 POINT(41.67849796787663 74.71763921432937) bank113778 +113779 POINT(40.7974141997676 73.5516605699274) bank113779 +113780 POINT(39.774931978273735 74.96839181700992) bank113780 +113781 POINT(40.431582935316 74.30898413097178) bank113781 +113782 POINT(40.7882815036458 73.2225908101171) bank113782 +113783 POINT(41.348134803579505 74.84037620048555) bank113783 +113784 POINT(41.18019093894621 74.82713467980298) bank113784 +113785 POINT(41.35610455830943 73.96920796294215) bank113785 +113786 POINT(40.129298410500105 74.82268259934766) bank113786 +113787 POINT(40.07558015516285 73.7846585830557) bank113787 +113788 POINT(39.82436483138683 73.29240166370812) bank113788 +113789 POINT(40.52684470314735 73.26059659991051) bank113789 +113790 POINT(41.41125547605852 73.74675968649733) bank113790 +113791 POINT(41.02044650991799 73.78705362073788) bank113791 +113792 POINT(40.338521924579155 74.49314413932127) bank113792 +113793 POINT(39.859537561716365 74.72704975390887) bank113793 +113794 POINT(41.35119351116298 73.14101152990497) bank113794 +113795 POINT(39.840005066784705 73.55679891657509) bank113795 +113796 POINT(41.43962688288209 73.1848187425875) bank113796 +113797 POINT(40.46132968718648 74.8258981675836) bank113797 +113798 POINT(40.0557681170864 73.98716160285511) bank113798 +113799 POINT(39.848675882418505 73.1410246851394) bank113799 +113800 POINT(40.668648619395746 74.68670822129936) bank113800 +113801 POINT(41.40574814759503 73.28768696479993) bank113801 +113802 POINT(40.60736072838198 74.99236548653384) bank113802 +113803 POINT(40.59673606311934 74.32888176070426) bank113803 +113804 POINT(40.765264143948826 74.60615660852783) bank113804 +113805 POINT(39.81748996880233 73.68448479172393) bank113805 +113806 POINT(40.94730233982802 74.81868585397719) bank113806 +113807 POINT(40.16381578655453 73.02908758276155) bank113807 +113808 POINT(41.49483130361054 73.98797943850711) bank113808 +113809 POINT(41.329197176880854 74.80956039979306) bank113809 +113810 POINT(41.0866117821938 74.0984354988851) bank113810 +113811 POINT(41.56704602438336 73.1181469955634) bank113811 +113812 POINT(40.41124670447593 74.33898737892369) bank113812 +113813 POINT(40.10632117713359 73.16590080095511) bank113813 +113814 POINT(40.6343018990256 74.55837964942751) bank113814 +113815 POINT(40.62532716203861 73.060637428409) bank113815 +113816 POINT(41.29444722274052 73.75696999440437) bank113816 +113817 POINT(40.8439705002178 74.71899409914029) bank113817 +113818 POINT(41.49355892629577 73.26285823045795) bank113818 +113819 POINT(41.29109731053156 74.11099130734763) bank113819 +113820 POINT(41.436198089115024 74.98327192792554) bank113820 +113821 POINT(39.97694640501473 73.84839755564184) bank113821 +113822 POINT(41.16557588411883 74.93839041752145) bank113822 +113823 POINT(41.422462125827956 74.53087291339156) bank113823 +113824 POINT(41.241137802517606 73.87231427286784) bank113824 +113825 POINT(39.74725405858683 73.98901152159365) bank113825 +113826 POINT(39.81595476081188 73.63517434302548) bank113826 +113827 POINT(40.90632099125233 74.96941886400847) bank113827 +113828 POINT(40.46098365132646 73.51755878330471) bank113828 +113829 POINT(40.3300445215302 73.91967919956669) bank113829 +113830 POINT(40.00480870023667 74.71711586358758) bank113830 +113831 POINT(40.48068285091942 73.1256003144351) bank113831 +113832 POINT(41.443526508959266 74.91025560566267) bank113832 +113833 POINT(40.44103241970779 74.54499842727536) bank113833 +113834 POINT(39.773319422358206 74.71571462927639) bank113834 +113835 POINT(41.21497584549004 73.89986815131078) bank113835 +113836 POINT(41.70123099501827 74.02265403903095) bank113836 +113837 POINT(40.34323638435015 74.60363181402315) bank113837 +113838 POINT(41.09342659329012 74.28712301943601) bank113838 +113839 POINT(41.706577758255435 74.2877583784516) bank113839 +113840 POINT(40.37011169652487 74.48204136287453) bank113840 +113841 POINT(40.23984118455451 74.7090817548533) bank113841 +113842 POINT(40.45236077812618 73.17845368889726) bank113842 +113843 POINT(40.108995992443184 73.59435638070916) bank113843 +113844 POINT(40.09865684217078 74.54337289662142) bank113844 +113845 POINT(40.29242558510599 73.72288232380801) bank113845 +113846 POINT(40.513116262531796 74.9155233809226) bank113846 +113847 POINT(40.059690042591086 73.10488364148573) bank113847 +113848 POINT(40.80384495909071 74.52108971118783) bank113848 +113849 POINT(39.769484571568135 74.69079106003096) bank113849 +113850 POINT(40.13561850991152 74.30736322227652) bank113850 +113851 POINT(41.08508311603662 74.1832053220204) bank113851 +113852 POINT(40.53575137535717 73.48210970262309) bank113852 +113853 POINT(41.145616536384956 73.14157986440112) bank113853 +113854 POINT(40.91115278301311 73.8977685223773) bank113854 +113855 POINT(40.63978327340035 74.59047740132328) bank113855 +113856 POINT(40.12275494132411 74.48155315734691) bank113856 +113857 POINT(40.05611734253155 73.8573722239515) bank113857 +113858 POINT(41.23500049248816 74.69783650549707) bank113858 +113859 POINT(40.76026395670936 74.1796237249954) bank113859 +113860 POINT(41.484360238058 73.23551249896767) bank113860 +113861 POINT(40.89001640104744 74.78204828348494) bank113861 +113862 POINT(41.584026597298546 73.93458572190426) bank113862 +113863 POINT(40.87124870811072 74.6512630676061) bank113863 +113864 POINT(40.35053635418376 73.33014357143314) bank113864 +113865 POINT(40.92549866418339 74.9298811434829) bank113865 +113866 POINT(40.819697678938454 73.4361867713079) bank113866 +113867 POINT(41.50895777073651 73.43784287049164) bank113867 +113868 POINT(40.37651510045689 74.73158128200674) bank113868 +113869 POINT(40.36706529649758 73.70135960548807) bank113869 +113870 POINT(41.14723806868398 74.2829264156634) bank113870 +113871 POINT(40.7902433365826 74.28881356638242) bank113871 +113872 POINT(39.96344453075213 73.08487470969273) bank113872 +113873 POINT(41.538145403048425 73.20029842955472) bank113873 +113874 POINT(40.51966608146975 74.09834963509162) bank113874 +113875 POINT(40.16924049674638 73.12906831186444) bank113875 +113876 POINT(40.832577769874014 74.19916301435633) bank113876 +113877 POINT(40.31437052482647 74.17557350856958) bank113877 +113878 POINT(39.75228525693209 73.99702670877743) bank113878 +113879 POINT(40.758125672486116 74.68533111204879) bank113879 +113880 POINT(40.098126295469235 73.65995309896215) bank113880 +113881 POINT(40.542626412194046 73.24289739229495) bank113881 +113882 POINT(39.880331388942736 74.00106404929822) bank113882 +113883 POINT(39.83637335939944 74.68434609532687) bank113883 +113884 POINT(40.29916264336647 74.78801062954267) bank113884 +113885 POINT(41.58985542313307 73.0686104555607) bank113885 +113886 POINT(40.79428631968077 73.58011545009117) bank113886 +113887 POINT(41.520434010137805 73.5261529175604) bank113887 +113888 POINT(39.962117011238156 73.11879462747305) bank113888 +113889 POINT(41.0999682420083 73.03220490956467) bank113889 +113890 POINT(41.21273028458454 73.8719017572295) bank113890 +113891 POINT(41.08052387694219 73.02170762557985) bank113891 +113892 POINT(40.800362529347574 74.14162110466046) bank113892 +113893 POINT(39.95393042534916 73.73111828558696) bank113893 +113894 POINT(41.28054285734097 73.41583341343058) bank113894 +113895 POINT(41.678973767534345 73.81945984471324) bank113895 +113896 POINT(41.12915959325239 74.29906691995748) bank113896 +113897 POINT(39.76041484520157 73.37325818687285) bank113897 +113898 POINT(41.328458264730926 73.24182657290649) bank113898 +113899 POINT(40.730303924992675 73.87909767867964) bank113899 +113900 POINT(39.861598189705695 74.25228272265778) bank113900 +113901 POINT(40.519855717737116 74.2458073801768) bank113901 +113902 POINT(40.65283561308698 74.24395028918246) bank113902 +113903 POINT(41.56448712506652 73.42187761154514) bank113903 +113904 POINT(40.89396845606653 73.81027704918331) bank113904 +113905 POINT(40.045649372025615 73.04196854706669) bank113905 +113906 POINT(41.09951580576489 73.44507867444328) bank113906 +113907 POINT(41.501754870395516 74.02805508756117) bank113907 +113908 POINT(41.587301691719375 73.08310338516532) bank113908 +113909 POINT(39.76952166625414 73.37732883014026) bank113909 +113910 POINT(41.24381162578562 74.85025539615953) bank113910 +113911 POINT(41.083625885178314 74.40567389772772) bank113911 +113912 POINT(40.264258104445084 73.64665621998647) bank113912 +113913 POINT(41.18511431001717 74.36564940578988) bank113913 +113914 POINT(40.61113849881397 74.87642299478456) bank113914 +113915 POINT(40.233408305695 74.77567844966907) bank113915 +113916 POINT(39.84712742154277 74.205606131445) bank113916 +113917 POINT(40.54995642475569 73.16476056565612) bank113917 +113918 POINT(39.77699727593311 74.60071645255175) bank113918 +113919 POINT(41.04647612152004 74.24027669599658) bank113919 +113920 POINT(39.84908834391352 74.38603520582056) bank113920 +113921 POINT(41.20485977549067 74.13429016781605) bank113921 +113922 POINT(40.414719667338446 73.1404994862369) bank113922 +113923 POINT(40.26292377360752 73.23101772961971) bank113923 +113924 POINT(41.39622131192082 73.7543588986556) bank113924 +113925 POINT(39.826808249159946 73.8913223052688) bank113925 +113926 POINT(41.62354568435019 74.22030073309918) bank113926 +113927 POINT(39.921487321105474 74.0498831434532) bank113927 +113928 POINT(40.01159290014382 73.14044935146167) bank113928 +113929 POINT(41.29124569973405 74.66565718160435) bank113929 +113930 POINT(40.75695578324953 74.74845810140657) bank113930 +113931 POINT(40.33054408847096 74.29305772024537) bank113931 +113932 POINT(39.94742667703172 74.8638291575095) bank113932 +113933 POINT(40.819063170556774 73.25077465534935) bank113933 +113934 POINT(40.138025313444984 74.62008453297804) bank113934 +113935 POINT(40.02429663737156 73.86885124611838) bank113935 +113936 POINT(40.80851824958382 73.15237692186074) bank113936 +113937 POINT(40.09796283386946 74.91066418709964) bank113937 +113938 POINT(40.01232468108778 74.15748400051245) bank113938 +113939 POINT(41.27626708818564 74.39618774546554) bank113939 +113940 POINT(40.05393019533522 74.36727532691566) bank113940 +113941 POINT(40.70005969421863 74.18665297620738) bank113941 +113942 POINT(41.393405254325685 74.42069905769614) bank113942 +113943 POINT(40.402216487888225 73.79160136039636) bank113943 +113944 POINT(40.223276422204194 73.54560574159204) bank113944 +113945 POINT(41.431291763188916 74.55222645964528) bank113945 +113946 POINT(41.00962096228828 74.05377331033192) bank113946 +113947 POINT(41.54725881294071 74.81975870902365) bank113947 +113948 POINT(40.55946041446302 74.31405989966105) bank113948 +113949 POINT(41.509008096335556 74.99108115401597) bank113949 +113950 POINT(40.291690213048824 74.46355228150037) bank113950 +113951 POINT(41.16890396352791 74.90874598468042) bank113951 +113952 POINT(40.70431257002883 74.54495809675376) bank113952 +113953 POINT(40.80466249796603 74.49263433858394) bank113953 +113954 POINT(40.44412839862604 73.79562893112299) bank113954 +113955 POINT(40.72859158068467 74.5292326330962) bank113955 +113956 POINT(40.731893720162944 73.54884341231568) bank113956 +113957 POINT(40.682829060621316 73.386061644904) bank113957 +113958 POINT(40.16664254413518 73.94761337240877) bank113958 +113959 POINT(41.07336607941278 73.91877020822972) bank113959 +113960 POINT(40.14908492405923 73.50699225956896) bank113960 +113961 POINT(40.904894416249654 74.49376601903438) bank113961 +113962 POINT(40.12019417997583 73.17406322203153) bank113962 +113963 POINT(41.69916905573249 74.57771690055498) bank113963 +113964 POINT(41.685386952386565 73.9953454810802) bank113964 +113965 POINT(40.89644140109744 74.18193745317906) bank113965 +113966 POINT(39.87526944920328 74.9184024286254) bank113966 +113967 POINT(41.21407466360345 73.46773512006344) bank113967 +113968 POINT(40.99514776488693 73.28001416907433) bank113968 +113969 POINT(40.001949969539986 73.03956552979506) bank113969 +113970 POINT(40.984298547571434 73.7117604917842) bank113970 +113971 POINT(40.6926573499309 73.29113359066302) bank113971 +113972 POINT(41.36272256633448 74.03067297307994) bank113972 +113973 POINT(40.32013626304344 73.93626196106685) bank113973 +113974 POINT(40.249217253031325 73.69715609190884) bank113974 +113975 POINT(41.41640904098583 74.42844084337027) bank113975 +113976 POINT(39.84321775929522 73.71651526014477) bank113976 +113977 POINT(41.29603115338587 74.32103971118723) bank113977 +113978 POINT(40.05473106740275 73.11922551446489) bank113978 +113979 POINT(41.534228269384954 74.18574582504375) bank113979 +113980 POINT(40.64228247810557 74.9681896974907) bank113980 +113981 POINT(40.90332133675432 73.91118479875514) bank113981 +113982 POINT(40.076584643753094 74.12974968843133) bank113982 +113983 POINT(39.75353449855267 74.164138916937) bank113983 +113984 POINT(41.30204629049136 74.72525313550754) bank113984 +113985 POINT(40.4157378648566 74.48680218517217) bank113985 +113986 POINT(40.384330388361484 73.64244044206147) bank113986 +113987 POINT(41.17091542569251 73.93651060871234) bank113987 +113988 POINT(39.87061109788022 73.77218879520875) bank113988 +113989 POINT(41.026460368250305 73.99171584035074) bank113989 +113990 POINT(41.471369536426025 73.22174055287749) bank113990 +113991 POINT(41.36530276306846 73.18353727776649) bank113991 +113992 POINT(39.75756218638275 74.73696020302755) bank113992 +113993 POINT(41.5946117177634 74.83852524020132) bank113993 +113994 POINT(40.85714967283344 74.24874142927204) bank113994 +113995 POINT(40.665258582318046 73.52550755594187) bank113995 +113996 POINT(39.8943815718825 74.8867911761111) bank113996 +113997 POINT(40.935167496215094 73.55847705167977) bank113997 +113998 POINT(41.460356145989294 74.72534771882172) bank113998 +113999 POINT(39.83206808849218 73.87857466900763) bank113999 +114000 POINT(40.21646143730653 73.99788581071655) bank114000 +114001 POINT(40.537420741187276 74.86538606493122) bank114001 +114002 POINT(39.967542658929936 74.42452609736769) bank114002 +114003 POINT(40.41370545605286 73.4832607552288) bank114003 +114004 POINT(41.09788428571787 74.2949205096418) bank114004 +114005 POINT(41.36776935275905 73.83929039780595) bank114005 +114006 POINT(40.58743570249604 74.74005942043412) bank114006 +114007 POINT(40.953347593971216 74.94990930270644) bank114007 +114008 POINT(40.38535346251805 74.61421658825532) bank114008 +114009 POINT(39.98585196357974 73.63884471051846) bank114009 +114010 POINT(40.63521970624325 73.39686573010134) bank114010 +114011 POINT(40.69237532887113 74.92269193459776) bank114011 +114012 POINT(41.32106071950987 74.8298534796809) bank114012 +114013 POINT(41.31948317492404 73.08045727040816) bank114013 +114014 POINT(39.73130708512865 74.68241786489287) bank114014 +114015 POINT(41.63135648466876 73.10047865582074) bank114015 +114016 POINT(39.751020122458065 73.58721563724424) bank114016 +114017 POINT(40.6115040282642 74.59720548156461) bank114017 +114018 POINT(40.31904951794743 74.45320376617829) bank114018 +114019 POINT(40.83014451203278 74.00966666897394) bank114019 +114020 POINT(41.0643232343863 74.45549857695613) bank114020 +114021 POINT(40.766409919106906 73.66715856843194) bank114021 +114022 POINT(41.67722729378572 74.8714520457612) bank114022 +114023 POINT(41.69211848255612 73.19941061432249) bank114023 +114024 POINT(39.733229039835706 74.89755938573417) bank114024 +114025 POINT(40.91511110022898 73.92612134690192) bank114025 +114026 POINT(40.570414101843994 74.38046940299951) bank114026 +114027 POINT(41.473565439818806 73.60576739401564) bank114027 +114028 POINT(41.63734163841973 74.70354311230341) bank114028 +114029 POINT(40.76645053706386 74.07630995754315) bank114029 +114030 POINT(40.749436077855975 73.60152553445157) bank114030 +114031 POINT(40.2053462785978 73.73295482553132) bank114031 +114032 POINT(39.86022202442942 74.91748057375777) bank114032 +114033 POINT(41.55988407184582 73.83617885356212) bank114033 +114034 POINT(41.32514001054475 74.24757232090741) bank114034 +114035 POINT(40.40119202171632 73.950959530316) bank114035 +114036 POINT(41.290023638989496 74.14397978383937) bank114036 +114037 POINT(41.26772322435415 73.50712758966816) bank114037 +114038 POINT(41.4996013423903 74.5804549764546) bank114038 +114039 POINT(41.00277709755413 74.05630505968763) bank114039 +114040 POINT(40.61688355718751 73.48639698226829) bank114040 +114041 POINT(40.75948895612385 73.59869933416181) bank114041 +114042 POINT(40.11340567747241 73.00828202542729) bank114042 +114043 POINT(40.14547050139483 73.43542151735765) bank114043 +114044 POINT(40.114001331343864 74.4346175249123) bank114044 +114045 POINT(40.157941839917626 73.95284384360605) bank114045 +114046 POINT(40.562700263772136 73.53218949731144) bank114046 +114047 POINT(40.9003410845572 74.7182661654314) bank114047 +114048 POINT(40.31925107221211 73.86529533222428) bank114048 +114049 POINT(41.36950242265712 73.83587154628201) bank114049 +114050 POINT(40.05002203370627 73.56119147389673) bank114050 +114051 POINT(40.01843067334602 73.01760704248936) bank114051 +114052 POINT(40.18816010725927 73.87130541783206) bank114052 +114053 POINT(41.28173913917785 74.24471949029848) bank114053 +114054 POINT(41.684514229483945 74.05236363991388) bank114054 +114055 POINT(41.69572087672642 73.329796406548) bank114055 +114056 POINT(39.95594877727524 73.79593163431221) bank114056 +114057 POINT(41.14709330072976 74.21358321834192) bank114057 +114058 POINT(41.64486571110392 73.65688381031964) bank114058 +114059 POINT(40.954304366355295 73.18710684652532) bank114059 +114060 POINT(40.95965489813353 74.38862836600053) bank114060 +114061 POINT(41.563028447642 74.30667911482647) bank114061 +114062 POINT(39.95942588731658 74.44705068034197) bank114062 +114063 POINT(40.72459716276276 74.4439495007753) bank114063 +114064 POINT(41.54156597942968 74.84296219638037) bank114064 +114065 POINT(41.59065770097315 74.04442964366723) bank114065 +114066 POINT(39.94693827987985 73.7084352412241) bank114066 +114067 POINT(40.894624228852564 74.00942250453569) bank114067 +114068 POINT(40.75618340517193 73.46118099443233) bank114068 +114069 POINT(40.05476325541657 74.98244923736036) bank114069 +114070 POINT(41.463746502585266 74.11022854872155) bank114070 +114071 POINT(41.467229464024335 73.84887203472367) bank114071 +114072 POINT(41.22399739538102 73.22816279685067) bank114072 +114073 POINT(40.585156737428214 73.57808766351957) bank114073 +114074 POINT(40.343175680738 73.3554761601035) bank114074 +114075 POINT(40.50418815756017 73.62946071471222) bank114075 +114076 POINT(41.14256003891533 73.08209332534686) bank114076 +114077 POINT(41.51450429460947 74.42637996900784) bank114077 +114078 POINT(41.70186274641394 74.80488312061804) bank114078 +114079 POINT(41.31504303616621 73.84107873172341) bank114079 +114080 POINT(39.8127392980176 73.46673734107657) bank114080 +114081 POINT(40.80784731570264 75.00550823138542) bank114081 +114082 POINT(40.7718201589763 74.97136414168902) bank114082 +114083 POINT(40.56469130687586 74.0762750907394) bank114083 +114084 POINT(41.53278113725416 74.31466693653431) bank114084 +114085 POINT(40.63244174854371 73.24942245185804) bank114085 +114086 POINT(40.89737708851582 73.25142734849199) bank114086 +114087 POINT(41.65792998234526 74.3021955729663) bank114087 +114088 POINT(39.945708727457024 74.00921121968186) bank114088 +114089 POINT(41.14990259453937 74.72533038226321) bank114089 +114090 POINT(41.06422362360653 74.79861545203633) bank114090 +114091 POINT(40.93054246601817 73.14636801643815) bank114091 +114092 POINT(40.352159259290296 73.12619211123966) bank114092 +114093 POINT(40.12390390719971 74.65411395510151) bank114093 +114094 POINT(41.56943014496033 73.29695046713837) bank114094 +114095 POINT(41.69966404242819 73.53004500852308) bank114095 +114096 POINT(39.83008146138252 73.04299132811536) bank114096 +114097 POINT(40.57404479122752 73.32823274983602) bank114097 +114098 POINT(39.86769976638104 73.24372144906108) bank114098 +114099 POINT(40.204929557903455 74.36645487302023) bank114099 +114100 POINT(41.6609421640425 73.84875260026587) bank114100 +114101 POINT(40.2826328450747 73.84241916174228) bank114101 +114102 POINT(40.1984306414139 73.03546730587114) bank114102 +114103 POINT(40.387087242212 73.03911400459855) bank114103 +114104 POINT(39.760628821072196 74.79926617038006) bank114104 +114105 POINT(39.97418360556181 73.25238270989432) bank114105 +114106 POINT(40.26380733409554 73.42837384109477) bank114106 +114107 POINT(39.911972957086874 73.02538929162831) bank114107 +114108 POINT(40.80347858200814 73.79675818874244) bank114108 +114109 POINT(40.73350671254251 74.28273175448213) bank114109 +114110 POINT(40.664813328192864 74.4116040968053) bank114110 +114111 POINT(40.50567830981279 73.57614674772104) bank114111 +114112 POINT(40.37796844192131 73.79671998360934) bank114112 +114113 POINT(40.60308181261903 74.26089718641855) bank114113 +114114 POINT(41.21205617727781 74.14205122339017) bank114114 +114115 POINT(39.98573225008864 73.74160109348597) bank114115 +114116 POINT(40.12463713421662 73.43613315272566) bank114116 +114117 POINT(41.06008733805931 74.08478564757166) bank114117 +114118 POINT(39.80536676495182 73.07598665204625) bank114118 +114119 POINT(40.20227071648075 73.12236219024852) bank114119 +114120 POINT(40.902539123906955 74.33331201285088) bank114120 +114121 POINT(40.11106672388055 74.11008208586956) bank114121 +114122 POINT(40.15964191707191 73.45215882214495) bank114122 +114123 POINT(41.69961700888611 73.22406288734904) bank114123 +114124 POINT(41.589325975500735 74.09971164640999) bank114124 +114125 POINT(40.62172573318096 73.88959592061792) bank114125 +114126 POINT(40.82112274364208 73.47246333333553) bank114126 +114127 POINT(41.07153318825023 74.72095951569501) bank114127 +114128 POINT(39.82586132517265 73.62526994886619) bank114128 +114129 POINT(40.20689425351902 74.13755892515701) bank114129 +114130 POINT(41.46274047995217 73.10917531962616) bank114130 +114131 POINT(41.20518198769184 74.13066298391206) bank114131 +114132 POINT(40.21920182812395 73.53674453261834) bank114132 +114133 POINT(41.05533864539866 74.73451852072459) bank114133 +114134 POINT(39.744362256384996 73.93934481578617) bank114134 +114135 POINT(41.433021082245894 74.88729124013551) bank114135 +114136 POINT(39.736368317111435 74.85459331714831) bank114136 +114137 POINT(40.78588986044677 74.04782597136239) bank114137 +114138 POINT(40.55472153160067 73.99657626774095) bank114138 +114139 POINT(41.657423558372756 74.13945040613969) bank114139 +114140 POINT(41.25150981530565 74.46956918734087) bank114140 +114141 POINT(39.786895124275325 74.05202148065004) bank114141 +114142 POINT(41.06093678039025 73.84663314998939) bank114142 +114143 POINT(40.66879475266209 74.68447105426577) bank114143 +114144 POINT(40.69327015368364 74.17684674936733) bank114144 +114145 POINT(40.957951101480774 74.01044319252063) bank114145 +114146 POINT(41.48235451879022 74.33513245422229) bank114146 +114147 POINT(41.37105578113986 73.6606925853299) bank114147 +114148 POINT(40.979355611525506 73.98122490993146) bank114148 +114149 POINT(39.96306632595907 73.7866397219195) bank114149 +114150 POINT(40.145123177257766 73.27417514659575) bank114150 +114151 POINT(40.15646023451924 74.14251589145776) bank114151 +114152 POINT(41.207774596151054 73.641285474459) bank114152 +114153 POINT(40.05462693830465 73.33338991105761) bank114153 +114154 POINT(39.861308279210384 73.09163212956324) bank114154 +114155 POINT(40.03199452616684 74.2928523314953) bank114155 +114156 POINT(40.755807318900324 73.39887314471241) bank114156 +114157 POINT(41.062763410439636 74.70684468359023) bank114157 +114158 POINT(40.01661892854365 73.41013769095386) bank114158 +114159 POINT(40.70679955616827 74.40402465772384) bank114159 +114160 POINT(40.38759649383445 74.46010953178958) bank114160 +114161 POINT(40.67024272177262 73.5538919411188) bank114161 +114162 POINT(41.07167274726566 73.15431019724313) bank114162 +114163 POINT(41.57758076083542 74.74608007579103) bank114163 +114164 POINT(39.95477443904695 73.41949389324904) bank114164 +114165 POINT(41.474070552418624 74.71873289681668) bank114165 +114166 POINT(39.78289198734376 73.19341172712701) bank114166 +114167 POINT(40.15753985402834 74.76406144969279) bank114167 +114168 POINT(40.72831295190505 74.03028866785016) bank114168 +114169 POINT(41.59343530041186 73.15667692811768) bank114169 +114170 POINT(41.14131338268673 74.71525822482573) bank114170 +114171 POINT(41.34895786504318 74.72186512443595) bank114171 +114172 POINT(40.513370891628085 73.54691439581336) bank114172 +114173 POINT(41.04288092105128 73.68465176968623) bank114173 +114174 POINT(40.98851349331853 73.17015092609198) bank114174 +114175 POINT(40.60129299200784 73.96888266338983) bank114175 +114176 POINT(40.444253400843515 74.01776310434416) bank114176 +114177 POINT(41.534686814365685 73.16650720501802) bank114177 +114178 POINT(40.158048922012256 74.36705971017683) bank114178 +114179 POINT(40.42803415794855 74.66910547529203) bank114179 +114180 POINT(41.40650251512352 73.56756221900402) bank114180 +114181 POINT(40.78509765345549 74.54107084354919) bank114181 +114182 POINT(40.25133220344076 74.05571940295157) bank114182 +114183 POINT(40.40519663470618 74.74022603770347) bank114183 +114184 POINT(40.699069370834195 73.16844411560976) bank114184 +114185 POINT(40.63727928538038 73.48098457494781) bank114185 +114186 POINT(40.200581415685164 73.38081849868124) bank114186 +114187 POINT(41.19480866544009 74.56624930314912) bank114187 +114188 POINT(40.93026603220055 73.57229656087601) bank114188 +114189 POINT(39.99757387474047 73.36537479356987) bank114189 +114190 POINT(40.568483957798854 74.75152717794509) bank114190 +114191 POINT(39.90485918379901 73.43936229278975) bank114191 +114192 POINT(40.48574374088767 74.85112915512151) bank114192 +114193 POINT(40.68077768342872 73.35503649898281) bank114193 +114194 POINT(41.526630914253445 74.10297826206353) bank114194 +114195 POINT(41.438782328533485 74.69295908861329) bank114195 +114196 POINT(39.81086396034647 73.5288732973584) bank114196 +114197 POINT(40.529932627791155 73.05980623620317) bank114197 +114198 POINT(40.97198803591799 74.31780153357454) bank114198 +114199 POINT(40.91508913994787 74.0761906197836) bank114199 +114200 POINT(40.78311265322616 73.69656910827747) bank114200 +114201 POINT(39.975720780653305 74.41855108799342) bank114201 +114202 POINT(41.39140546628552 74.73702876312025) bank114202 +114203 POINT(40.304401663765084 73.44727861962576) bank114203 +114204 POINT(40.04621498740236 74.91040871055098) bank114204 +114205 POINT(41.64213065138164 74.52721812440605) bank114205 +114206 POINT(40.448676672438374 74.83791028565898) bank114206 +114207 POINT(40.82008430819436 74.60237797837732) bank114207 +114208 POINT(41.302413801423036 74.21002812500933) bank114208 +114209 POINT(40.48950688608385 74.8803924740598) bank114209 +114210 POINT(40.90853770613856 73.10949123245526) bank114210 +114211 POINT(40.48589554561141 74.56963836302597) bank114211 +114212 POINT(41.49858390208946 74.23377625531725) bank114212 +114213 POINT(41.41335799046111 74.81871808710979) bank114213 +114214 POINT(41.26176207420025 73.65034126120403) bank114214 +114215 POINT(39.85049744021782 73.29375226351318) bank114215 +114216 POINT(40.46461411564498 73.22006293427556) bank114216 +114217 POINT(40.13651333127716 73.47065149050616) bank114217 +114218 POINT(41.547807488441805 73.02782356768378) bank114218 +114219 POINT(40.8853968363575 73.14132559001138) bank114219 +114220 POINT(41.005579348969206 73.287708001565) bank114220 +114221 POINT(40.612541895095276 74.61034190004908) bank114221 +114222 POINT(40.003195666491166 74.64836721215515) bank114222 +114223 POINT(41.38235685594348 74.75492222391132) bank114223 +114224 POINT(40.07086972953361 74.55920978586735) bank114224 +114225 POINT(41.07068939794496 74.86077272879751) bank114225 +114226 POINT(40.967847473721015 73.42649769140506) bank114226 +114227 POINT(40.48397134950979 73.8974237608128) bank114227 +114228 POINT(41.583698513114534 73.6435824569047) bank114228 +114229 POINT(39.97337845727856 73.0283145074469) bank114229 +114230 POINT(40.357392043468806 74.57162833826598) bank114230 +114231 POINT(40.40572427847045 73.50172796118275) bank114231 +114232 POINT(40.69727214526152 74.83860157700155) bank114232 +114233 POINT(41.088621728785796 73.35500173318586) bank114233 +114234 POINT(39.75240171626642 74.04777184461749) bank114234 +114235 POINT(40.9584599764994 73.42135411836223) bank114235 +114236 POINT(40.224545609853756 73.91458808724853) bank114236 +114237 POINT(41.08285758933879 74.55615729403551) bank114237 +114238 POINT(40.21309990837015 73.65008516217571) bank114238 +114239 POINT(40.159229523757105 74.28121431143991) bank114239 +114240 POINT(40.92623314442427 74.0284539119401) bank114240 +114241 POINT(40.758293026223946 73.31578113601255) bank114241 +114242 POINT(39.80864327520786 74.81670295338472) bank114242 +114243 POINT(39.780114843521766 74.37468392194245) bank114243 +114244 POINT(40.86779557414591 73.75139832445932) bank114244 +114245 POINT(40.23822754069708 74.08282852025596) bank114245 +114246 POINT(40.67532390403595 73.14351677755391) bank114246 +114247 POINT(40.787780213928365 74.1614194527885) bank114247 +114248 POINT(39.749880684039454 74.8112500237117) bank114248 +114249 POINT(41.118636774929165 74.2725866407456) bank114249 +114250 POINT(41.21322754277909 74.71072338357297) bank114250 +114251 POINT(39.73626182362312 73.18489583329294) bank114251 +114252 POINT(40.220022980165965 74.5779489304525) bank114252 +114253 POINT(40.0451728589633 73.27633259859) bank114253 +114254 POINT(41.53129912023325 74.30094377415158) bank114254 +114255 POINT(40.591494377605166 74.4975138461413) bank114255 +114256 POINT(41.0600778980727 73.68116099806397) bank114256 +114257 POINT(40.23912875501527 74.10817717837219) bank114257 +114258 POINT(41.64821029173364 74.22941763685628) bank114258 +114259 POINT(41.384875786572216 74.05472341524018) bank114259 +114260 POINT(40.72584526279219 73.80749418401942) bank114260 +114261 POINT(39.82352244309417 74.89129172648539) bank114261 +114262 POINT(41.17579791088174 73.08533862753028) bank114262 +114263 POINT(40.16344637643832 74.2353189438702) bank114263 +114264 POINT(41.364961663135205 74.38022134876536) bank114264 +114265 POINT(40.48373392716581 73.00939978815809) bank114265 +114266 POINT(40.85949379003481 73.10478327704375) bank114266 +114267 POINT(39.97235108638078 73.7735849198129) bank114267 +114268 POINT(40.650865212121374 73.33620713857552) bank114268 +114269 POINT(41.3188378894933 74.25653161524208) bank114269 +114270 POINT(40.15921185490108 73.88130259051334) bank114270 +114271 POINT(41.34655537658134 74.98178205006309) bank114271 +114272 POINT(41.097474116464525 74.65355196912513) bank114272 +114273 POINT(40.08081235078365 73.11987475929008) bank114273 +114274 POINT(41.20013895548005 74.24210863261368) bank114274 +114275 POINT(40.68212321627359 73.66467770175977) bank114275 +114276 POINT(40.43021903568255 74.38702929615468) bank114276 +114277 POINT(40.64772290487635 73.92250310277304) bank114277 +114278 POINT(41.12671134663385 74.16105657852317) bank114278 +114279 POINT(41.28912719261107 73.85304308196233) bank114279 +114280 POINT(40.60195163112098 74.61211507515597) bank114280 +114281 POINT(40.8547832108379 73.13323230920454) bank114281 +114282 POINT(39.897242960657955 73.07429497771021) bank114282 +114283 POINT(40.73522604599821 73.80078722119919) bank114283 +114284 POINT(41.0478356567072 73.55724653204197) bank114284 +114285 POINT(40.17244539890907 73.61472052260223) bank114285 +114286 POINT(40.84431451815172 73.97207397502405) bank114286 +114287 POINT(40.409604714808076 74.11812735695872) bank114287 +114288 POINT(39.81621510463899 74.5361692717478) bank114288 +114289 POINT(40.52551876380058 74.78835247683207) bank114289 +114290 POINT(39.79757993045672 74.99375924333452) bank114290 +114291 POINT(39.86810868249312 74.23595319625498) bank114291 +114292 POINT(40.73021008137395 73.3083770254348) bank114292 +114293 POINT(40.716519776006386 74.16655144923975) bank114293 +114294 POINT(40.41653759924412 73.82848888664402) bank114294 +114295 POINT(40.89518345734571 73.8584262892209) bank114295 +114296 POINT(40.37975775161016 74.22413672365896) bank114296 +114297 POINT(39.89789765021288 74.79311303082467) bank114297 +114298 POINT(41.60059955206482 74.06181840000198) bank114298 +114299 POINT(40.26699727570306 73.313587327377) bank114299 +114300 POINT(41.11375976674883 74.64226597882896) bank114300 +114301 POINT(41.36755673322066 74.13369521985518) bank114301 +114302 POINT(41.229425536326644 73.09339441520444) bank114302 +114303 POINT(39.7161678188174 73.96308392265414) bank114303 +114304 POINT(40.76341937098045 74.856924460514) bank114304 +114305 POINT(41.38187546756088 73.18481206208374) bank114305 +114306 POINT(40.48968194870817 73.25393838524391) bank114306 +114307 POINT(40.602338986307316 74.49145344038342) bank114307 +114308 POINT(41.343848802557275 74.61454565016163) bank114308 +114309 POINT(40.39092469863888 73.3287450940527) bank114309 +114310 POINT(39.776907726718136 74.12480957303278) bank114310 +114311 POINT(41.58959528271869 74.64292107139251) bank114311 +114312 POINT(41.324495061610826 73.5809254597791) bank114312 +114313 POINT(40.26061079277166 73.07488371053049) bank114313 +114314 POINT(41.15697253834845 73.84577241815104) bank114314 +114315 POINT(41.11402489175203 73.52188839673184) bank114315 +114316 POINT(40.28807811382452 74.55821189920812) bank114316 +114317 POINT(41.17575135750996 73.07529248514817) bank114317 +114318 POINT(40.06314080499778 73.72100130159431) bank114318 +114319 POINT(40.81599673345758 74.01515641326304) bank114319 +114320 POINT(40.21896874030259 74.13736237711949) bank114320 +114321 POINT(41.511669685840644 73.36277008912823) bank114321 +114322 POINT(40.49589239320976 73.79884044171156) bank114322 +114323 POINT(41.236324191282186 73.3334405092989) bank114323 +114324 POINT(39.92986924340793 73.12236258601357) bank114324 +114325 POINT(40.26829642854951 74.66474736947107) bank114325 +114326 POINT(40.814742688556066 73.44901705640788) bank114326 +114327 POINT(41.037424639996686 73.68186569746126) bank114327 +114328 POINT(40.95419949878245 74.43727777759166) bank114328 +114329 POINT(41.121950594074406 74.94684029960332) bank114329 +114330 POINT(39.89669943173698 73.23968641122502) bank114330 +114331 POINT(40.881992341413536 74.64896600715713) bank114331 +114332 POINT(41.533119672090194 73.33352101269976) bank114332 +114333 POINT(40.12965117972697 74.37017000685842) bank114333 +114334 POINT(40.229951372575854 74.26450685005445) bank114334 +114335 POINT(40.21402220130002 74.13997455242654) bank114335 +114336 POINT(40.54638053290938 73.98853475067237) bank114336 +114337 POINT(40.58945432080881 73.33217857769019) bank114337 +114338 POINT(40.60526402452228 74.73346037837159) bank114338 +114339 POINT(40.83313514509971 74.26647974331391) bank114339 +114340 POINT(40.648904610744395 73.86562324839772) bank114340 +114341 POINT(39.86473532901417 73.37006424053476) bank114341 +114342 POINT(41.07850554803835 74.7681896902352) bank114342 +114343 POINT(40.42418806835518 74.1606237036827) bank114343 +114344 POINT(41.47556769657886 73.75877332935669) bank114344 +114345 POINT(40.14927770557687 74.8035417956194) bank114345 +114346 POINT(40.224935489738144 74.06986280390254) bank114346 +114347 POINT(40.743232613260815 73.06834691376793) bank114347 +114348 POINT(39.83023699147346 74.90580064197347) bank114348 +114349 POINT(40.74058760854101 74.37327767301842) bank114349 +114350 POINT(41.25587131192233 73.5409687664485) bank114350 +114351 POINT(40.707741486174235 74.17491375712983) bank114351 +114352 POINT(41.13220285132723 73.61468472806092) bank114352 +114353 POINT(41.56270747888303 74.37717125667989) bank114353 +114354 POINT(40.5462063147881 74.20104901995016) bank114354 +114355 POINT(41.706246392279475 73.75919376101345) bank114355 +114356 POINT(39.984233801680105 73.05533397258137) bank114356 +114357 POINT(40.12258019321463 73.67729355379024) bank114357 +114358 POINT(41.49163477842044 74.40081651114461) bank114358 +114359 POINT(41.460470050777694 73.07984276162914) bank114359 +114360 POINT(39.77477957374035 74.07416334513204) bank114360 +114361 POINT(41.115725533427536 74.90779991890653) bank114361 +114362 POINT(41.44942773536606 73.6515577895351) bank114362 +114363 POINT(39.95229298835066 73.16028238618303) bank114363 +114364 POINT(41.010315204802744 74.590665084735) bank114364 +114365 POINT(40.40299621845517 73.55169961692664) bank114365 +114366 POINT(40.97107032898751 73.77189594409109) bank114366 +114367 POINT(40.77976102937825 73.69312437070015) bank114367 +114368 POINT(40.933523809791595 74.78647491181815) bank114368 +114369 POINT(40.94542017091023 74.90736487614484) bank114369 +114370 POINT(40.872316811060216 74.01159165364756) bank114370 +114371 POINT(40.22592120514913 73.55443520605519) bank114371 +114372 POINT(41.29306996418007 74.44046449871827) bank114372 +114373 POINT(39.73042744054543 74.92693511725835) bank114373 +114374 POINT(40.39779936729958 74.32557143600965) bank114374 +114375 POINT(41.152943149997675 73.55792377591021) bank114375 +114376 POINT(40.57768767277284 73.99704143035052) bank114376 +114377 POINT(41.449824133290804 74.5516025564323) bank114377 +114378 POINT(40.26221724923216 74.43771830349336) bank114378 +114379 POINT(41.632800694633644 74.70575069106118) bank114379 +114380 POINT(40.12206285293953 74.52473064274574) bank114380 +114381 POINT(40.08942977587533 73.47211040945703) bank114381 +114382 POINT(41.39113253861274 73.9372380012324) bank114382 +114383 POINT(41.20768054716899 74.55668505906931) bank114383 +114384 POINT(40.63585813788211 73.17542172405916) bank114384 +114385 POINT(41.48675485030365 74.9040950018584) bank114385 +114386 POINT(40.65566213375153 74.46234280061809) bank114386 +114387 POINT(40.87110052417492 73.03335560069708) bank114387 +114388 POINT(40.463257643698576 73.72423120474978) bank114388 +114389 POINT(39.72926003431407 74.0595655355592) bank114389 +114390 POINT(40.665402270900906 73.21870987930551) bank114390 +114391 POINT(41.62277770238883 73.1511593061408) bank114391 +114392 POINT(40.164887950484044 73.95647819098895) bank114392 +114393 POINT(41.589217949206066 74.29135884651602) bank114393 +114394 POINT(40.18664241786333 74.60963409159052) bank114394 +114395 POINT(40.66092361095767 74.11334548352347) bank114395 +114396 POINT(40.79288647548428 73.65545415902295) bank114396 +114397 POINT(39.93681445417977 74.12740323338326) bank114397 +114398 POINT(41.03309221691532 74.60204152325954) bank114398 +114399 POINT(41.37807036051354 73.84787845393373) bank114399 +114400 POINT(40.5419033236193 74.19979968848595) bank114400 +114401 POINT(40.7619481650558 74.81833397101167) bank114401 +114402 POINT(40.947666229894985 74.60668376973996) bank114402 +114403 POINT(41.29752721348107 74.3378491312619) bank114403 +114404 POINT(41.70306281732252 74.76659744890804) bank114404 +114405 POINT(41.4124548459186 74.2389230585916) bank114405 +114406 POINT(41.41467384660968 74.51915381673518) bank114406 +114407 POINT(40.45354260214863 74.00998680070899) bank114407 +114408 POINT(41.64911430907175 74.99207661286592) bank114408 +114409 POINT(40.1266861654657 73.27932134594637) bank114409 +114410 POINT(39.74005944203416 73.9834340344262) bank114410 +114411 POINT(40.724323264322905 74.22491909344028) bank114411 +114412 POINT(40.4269785287215 74.33790475344313) bank114412 +114413 POINT(39.95959407281566 73.37406552795953) bank114413 +114414 POINT(40.55879695072103 73.45811435119226) bank114414 +114415 POINT(39.76143478623014 74.1607805263748) bank114415 +114416 POINT(40.622317156974226 73.37934640857375) bank114416 +114417 POINT(41.06661510107407 73.85301829507537) bank114417 +114418 POINT(40.61070339981981 73.01979877500997) bank114418 +114419 POINT(41.47263049461094 73.11843460918661) bank114419 +114420 POINT(40.253495927915694 73.02909551901477) bank114420 +114421 POINT(40.023652578383825 74.32372825299622) bank114421 +114422 POINT(39.76271245599477 73.01087519925278) bank114422 +114423 POINT(40.81083731365492 74.06507580421926) bank114423 +114424 POINT(41.51120819967751 73.83173957564802) bank114424 +114425 POINT(40.247630481970695 74.14100724281964) bank114425 +114426 POINT(41.50412883020203 74.20811217103473) bank114426 +114427 POINT(40.35369064661223 74.85507917773951) bank114427 +114428 POINT(41.55558006027757 73.02367963106751) bank114428 +114429 POINT(39.98559682764735 74.80669031742576) bank114429 +114430 POINT(41.606674309220764 74.59265686740207) bank114430 +114431 POINT(40.642584952366754 74.9406319308138) bank114431 +114432 POINT(40.89350360901079 73.4865974488) bank114432 +114433 POINT(40.36934922958367 74.39067700058773) bank114433 +114434 POINT(40.16348832174687 73.00856311253253) bank114434 +114435 POINT(40.754643095391636 73.44266933236032) bank114435 +114436 POINT(41.69851835710981 74.3579815610342) bank114436 +114437 POINT(41.66879579714898 73.16099942537359) bank114437 +114438 POINT(40.17074426054162 73.17013541281815) bank114438 +114439 POINT(39.91732411243485 74.52329870361959) bank114439 +114440 POINT(40.35668309008802 73.15381803222073) bank114440 +114441 POINT(41.16175345766305 73.23651635377234) bank114441 +114442 POINT(41.16248124506688 73.32904072300911) bank114442 +114443 POINT(39.74882283689428 74.90176807740832) bank114443 +114444 POINT(40.70517020171023 74.43371236267072) bank114444 +114445 POINT(40.442814533557424 73.94053244947177) bank114445 +114446 POINT(40.826735376311554 74.12340886395496) bank114446 +114447 POINT(40.01619458237237 74.79263473190797) bank114447 +114448 POINT(41.702651976495346 74.82404542158382) bank114448 +114449 POINT(40.48944966354931 73.80744596568) bank114449 +114450 POINT(41.23746078448057 74.51226390914114) bank114450 +114451 POINT(40.09522637770377 73.9781616151513) bank114451 +114452 POINT(41.349311392338834 73.46832375339858) bank114452 +114453 POINT(40.31383792775432 74.25826006947196) bank114453 +114454 POINT(40.44301795858558 73.52086220548335) bank114454 +114455 POINT(41.495144490154516 74.57908820322258) bank114455 +114456 POINT(41.37821193970433 73.1892982220428) bank114456 +114457 POINT(39.84967838578497 73.51666781892467) bank114457 +114458 POINT(40.60477426145118 74.93832842955024) bank114458 +114459 POINT(40.765110990941906 74.8423219361051) bank114459 +114460 POINT(41.568840632840256 74.47126479951993) bank114460 +114461 POINT(41.6883196917983 74.08003606234261) bank114461 +114462 POINT(41.616700322467565 74.773492415271) bank114462 +114463 POINT(39.97604573712078 73.75682252927743) bank114463 +114464 POINT(40.563808763580354 74.37839501790737) bank114464 +114465 POINT(40.131919350391875 73.25299647463888) bank114465 +114466 POINT(40.038018990439056 73.21575128748567) bank114466 +114467 POINT(40.059205674059825 74.34994565663725) bank114467 +114468 POINT(41.5706617765786 74.61919601175558) bank114468 +114469 POINT(40.38401658856165 73.27655226629292) bank114469 +114470 POINT(41.531960488846295 73.9815396146704) bank114470 +114471 POINT(40.41147442868417 74.5526656058359) bank114471 +114472 POINT(41.15127950639935 74.76748721731592) bank114472 +114473 POINT(41.31798064987632 73.53614725282179) bank114473 +114474 POINT(39.87043339974818 74.34673006410722) bank114474 +114475 POINT(41.6634934555201 73.39712667747885) bank114475 +114476 POINT(40.89521211532245 74.08298929853117) bank114476 +114477 POINT(40.02857757752442 73.39256530161308) bank114477 +114478 POINT(40.829532388581434 74.13662043141385) bank114478 +114479 POINT(40.024542444132734 74.9696329331317) bank114479 +114480 POINT(39.934222981406016 74.8266060156425) bank114480 +114481 POINT(39.84723523301617 73.66742692915659) bank114481 +114482 POINT(40.46314557785914 73.77635209523228) bank114482 +114483 POINT(40.502556955290345 73.59026653248664) bank114483 +114484 POINT(40.004446234668904 74.80065847395267) bank114484 +114485 POINT(39.78218374720175 74.65672286385308) bank114485 +114486 POINT(40.79206654035475 74.06056542368599) bank114486 +114487 POINT(41.42304417645837 74.9872301068921) bank114487 +114488 POINT(41.02493272906174 74.14141114252259) bank114488 +114489 POINT(41.2810980309003 73.06828671826759) bank114489 +114490 POINT(40.80192101905903 73.86525571340208) bank114490 +114491 POINT(40.84245832205422 73.86292975695105) bank114491 +114492 POINT(40.78603532740758 73.82415906395143) bank114492 +114493 POINT(41.12735020619848 74.21498425787387) bank114493 +114494 POINT(39.90100728378716 74.34713635673454) bank114494 +114495 POINT(40.76923441876966 74.30488210002454) bank114495 +114496 POINT(40.59920285654658 74.1273443714207) bank114496 +114497 POINT(39.791655209424995 74.56986060779246) bank114497 +114498 POINT(40.034547920009814 74.43694472825673) bank114498 +114499 POINT(41.07740456415178 73.429579231345) bank114499 +114500 POINT(41.506544687235575 74.23671984177818) bank114500 +114501 POINT(39.85509147525784 73.16613073221069) bank114501 +114502 POINT(40.368564892112424 74.73386397986452) bank114502 +114503 POINT(40.26294846853831 74.71962407588818) bank114503 +114504 POINT(40.21629789307912 74.21389523067249) bank114504 +114505 POINT(39.98943444167378 73.29096517952104) bank114505 +114506 POINT(41.24496144728455 74.33885733876882) bank114506 +114507 POINT(41.656729122469415 74.50309137058281) bank114507 +114508 POINT(40.040161598100795 73.63778160210433) bank114508 +114509 POINT(40.98838858130243 73.58473933466786) bank114509 +114510 POINT(40.35793746438384 74.4091379515679) bank114510 +114511 POINT(40.09365323624273 73.26607559207109) bank114511 +114512 POINT(41.45919623239006 74.94581265469624) bank114512 +114513 POINT(40.96985336302866 73.77739054351576) bank114513 +114514 POINT(41.04210538886559 74.25912371546639) bank114514 +114515 POINT(40.653056310899686 74.86754548793574) bank114515 +114516 POINT(41.09465299294766 74.81237520073836) bank114516 +114517 POINT(40.36818762943738 73.5514442203591) bank114517 +114518 POINT(39.790748463739305 73.94558067680354) bank114518 +114519 POINT(41.209518574599336 73.72828843531983) bank114519 +114520 POINT(41.41656242492419 74.9972140214582) bank114520 +114521 POINT(39.73783302813116 73.65107881114629) bank114521 +114522 POINT(40.75030703848799 73.5018282434721) bank114522 +114523 POINT(39.968626809129916 73.23632507108077) bank114523 +114524 POINT(40.490400647918655 73.40044779035547) bank114524 +114525 POINT(40.31032512356254 74.31423839260799) bank114525 +114526 POINT(40.535946542105684 73.90078094934489) bank114526 +114527 POINT(41.19718012326868 74.122135011731) bank114527 +114528 POINT(40.701864665582654 74.50636018419691) bank114528 +114529 POINT(41.17092689375547 73.63489834894241) bank114529 +114530 POINT(41.34165263873028 74.81855882420757) bank114530 +114531 POINT(40.53817088396107 73.99072827266046) bank114531 +114532 POINT(41.22132969371794 73.96414917855081) bank114532 +114533 POINT(41.282478160466276 74.9762558742058) bank114533 +114534 POINT(40.12052530086755 74.65848797235691) bank114534 +114535 POINT(41.25368364716094 74.0959627254568) bank114535 +114536 POINT(40.00365923761425 73.68472039130529) bank114536 +114537 POINT(39.88052256935249 73.84937898615622) bank114537 +114538 POINT(41.22517546159392 74.60555748256358) bank114538 +114539 POINT(41.01823259387136 74.81334979916544) bank114539 +114540 POINT(39.86908595119885 73.43339239560517) bank114540 +114541 POINT(41.5121829532772 74.88483813101114) bank114541 +114542 POINT(40.61067863946556 74.90546328229279) bank114542 +114543 POINT(41.656093547842524 74.47784443215994) bank114543 +114544 POINT(41.712265702224535 74.20738195203405) bank114544 +114545 POINT(40.082046855456404 73.2197627996374) bank114545 +114546 POINT(41.33628754815679 74.21275865172821) bank114546 +114547 POINT(40.50582898646604 73.77762202814776) bank114547 +114548 POINT(40.008867883441845 74.30517345989908) bank114548 +114549 POINT(40.21382909912862 73.81155077159855) bank114549 +114550 POINT(41.24105899144814 74.24279276046089) bank114550 +114551 POINT(40.737333367788295 73.55894288151977) bank114551 +114552 POINT(41.492218793485435 73.33716134577898) bank114552 +114553 POINT(39.931232214892006 74.75394982108092) bank114553 +114554 POINT(40.46210297255749 73.75613864887893) bank114554 +114555 POINT(41.173253369499484 73.637766660806) bank114555 +114556 POINT(40.57789391573155 74.07851136384323) bank114556 +114557 POINT(41.704009534006595 74.47926319349988) bank114557 +114558 POINT(40.47306835797347 73.99620269479377) bank114558 +114559 POINT(40.424964207501134 73.69503397966132) bank114559 +114560 POINT(40.44809600215288 73.2185383060596) bank114560 +114561 POINT(40.507819130713216 74.88524131686955) bank114561 +114562 POINT(40.05044713618027 73.713829358705) bank114562 +114563 POINT(40.53036953380543 74.33180329308797) bank114563 +114564 POINT(41.29453801515657 73.71574893245936) bank114564 +114565 POINT(41.59313183645972 74.02373950489343) bank114565 +114566 POINT(41.5711957964936 74.40939460570313) bank114566 +114567 POINT(41.36881001162608 73.70743887295002) bank114567 +114568 POINT(40.18412609465101 73.78732666866964) bank114568 +114569 POINT(41.40902544465792 74.35394082403928) bank114569 +114570 POINT(40.86966141324877 74.82502955373151) bank114570 +114571 POINT(39.7862597016456 74.0480782539353) bank114571 +114572 POINT(40.109380227815485 74.85088508825278) bank114572 +114573 POINT(40.358726521470864 74.39535331257332) bank114573 +114574 POINT(40.89592489282474 73.09898708106877) bank114574 +114575 POINT(40.01697556773975 74.39828157263459) bank114575 +114576 POINT(41.13772374210839 74.45481687546666) bank114576 +114577 POINT(41.268321673789615 74.51058704482325) bank114577 +114578 POINT(40.19270405908104 73.89332378417204) bank114578 +114579 POINT(40.046076172784396 74.87208218312996) bank114579 +114580 POINT(41.09630744542075 73.40352687068477) bank114580 +114581 POINT(40.50031742719341 73.77650973159257) bank114581 +114582 POINT(41.18951555604112 73.56769198336062) bank114582 +114583 POINT(41.25459677239908 74.20851477202459) bank114583 +114584 POINT(40.294641534940716 73.0658663031795) bank114584 +114585 POINT(40.47819002685386 74.39709289696727) bank114585 +114586 POINT(40.0056534124939 73.83742454996761) bank114586 +114587 POINT(41.32457072645289 74.26877623756245) bank114587 +114588 POINT(41.24977684101381 73.02117625752304) bank114588 +114589 POINT(40.39077979282343 74.6372264073364) bank114589 +114590 POINT(40.18713510162506 74.11569151795776) bank114590 +114591 POINT(40.01565180069153 73.7909446442972) bank114591 +114592 POINT(39.98989455642156 74.62509956360313) bank114592 +114593 POINT(40.55536740659377 74.00009903015662) bank114593 +114594 POINT(41.624661659669755 74.23282919936317) bank114594 +114595 POINT(40.91927164638718 74.4365853809074) bank114595 +114596 POINT(40.33031239236408 73.56506453960536) bank114596 +114597 POINT(41.40833820039578 73.72680002311401) bank114597 +114598 POINT(41.6977194417407 73.36760101901288) bank114598 +114599 POINT(40.517710682108046 73.77917556823567) bank114599 +114600 POINT(40.3046486548801 74.3698948490666) bank114600 +114601 POINT(41.04587520316405 73.61547635868092) bank114601 +114602 POINT(41.24817193524505 74.54650984920103) bank114602 +114603 POINT(41.61575106805173 74.97216537501694) bank114603 +114604 POINT(39.79807748082368 74.57578794909156) bank114604 +114605 POINT(40.36582891741661 74.02973703896552) bank114605 +114606 POINT(41.39808857288668 74.43126239498959) bank114606 +114607 POINT(41.59634455421757 74.68980514341166) bank114607 +114608 POINT(40.77419243074119 73.41705980538998) bank114608 +114609 POINT(40.70723762420394 73.44889814842182) bank114609 +114610 POINT(40.727323677854926 74.64775393339357) bank114610 +114611 POINT(41.55263689559723 73.57615675685012) bank114611 +114612 POINT(40.076603606677764 74.36365101370006) bank114612 +114613 POINT(41.39523160898172 73.95137896022482) bank114613 +114614 POINT(40.280524046677904 74.67183870879029) bank114614 +114615 POINT(40.72590477796335 74.11874772997686) bank114615 +114616 POINT(40.06876038783426 74.08978502355563) bank114616 +114617 POINT(40.94869486668056 73.1409397956305) bank114617 +114618 POINT(40.1029039966233 73.2027718623743) bank114618 +114619 POINT(39.95355200870654 74.2248276893325) bank114619 +114620 POINT(41.03105859509045 74.98230399373551) bank114620 +114621 POINT(41.63679151228793 74.63736895240879) bank114621 +114622 POINT(40.180884237427456 73.9186024637209) bank114622 +114623 POINT(40.8353871516766 73.17354172053116) bank114623 +114624 POINT(40.776613324865245 73.19035521520243) bank114624 +114625 POINT(41.15569774988098 73.42052459849458) bank114625 +114626 POINT(40.03492895644794 74.59501218429705) bank114626 +114627 POINT(39.8953599462739 73.89243381426621) bank114627 +114628 POINT(40.18315434853563 73.36361717007664) bank114628 +114629 POINT(40.36297124976011 73.19863744673103) bank114629 +114630 POINT(41.054826465956964 74.75403817915623) bank114630 +114631 POINT(41.44151891146066 73.8777174429631) bank114631 +114632 POINT(40.32153339804687 74.53788832706688) bank114632 +114633 POINT(40.650311429352605 73.78985532525738) bank114633 +114634 POINT(40.85068407282665 73.21729495152488) bank114634 +114635 POINT(40.30127885181127 73.58855267880134) bank114635 +114636 POINT(40.06991535665837 73.0662800850942) bank114636 +114637 POINT(41.10747911921927 73.52013585105458) bank114637 +114638 POINT(41.20749223521355 73.21631108123218) bank114638 +114639 POINT(41.63427608543972 74.3632683691255) bank114639 +114640 POINT(40.004713839720154 73.12888217629057) bank114640 +114641 POINT(40.98990886407424 74.0740723878303) bank114641 +114642 POINT(41.635112844357415 74.49938268758147) bank114642 +114643 POINT(41.056958634478434 73.37484332449071) bank114643 +114644 POINT(41.349907313726995 73.09362665296989) bank114644 +114645 POINT(41.58900394620825 74.21526114806764) bank114645 +114646 POINT(41.14296505917971 73.42387882050049) bank114646 +114647 POINT(41.228801485072715 74.24192449482017) bank114647 +114648 POINT(39.77110204154227 73.59176580736577) bank114648 +114649 POINT(40.264849597677284 73.7194762736712) bank114649 +114650 POINT(40.37983272179193 74.23364138186746) bank114650 +114651 POINT(39.89795285592388 74.89802965731727) bank114651 +114652 POINT(40.63449571334491 73.07936885352609) bank114652 +114653 POINT(40.8908187621088 74.22012963630283) bank114653 +114654 POINT(40.31221340698472 73.92393020970844) bank114654 +114655 POINT(40.35728895539897 73.42780462638929) bank114655 +114656 POINT(41.29388460441504 74.55567891078728) bank114656 +114657 POINT(39.93015240346004 73.90518351320762) bank114657 +114658 POINT(40.59633534015599 73.63185253034044) bank114658 +114659 POINT(41.583848036980655 73.95268828629152) bank114659 +114660 POINT(39.88949846947821 74.81092869737769) bank114660 +114661 POINT(40.927021886951444 74.11798922535336) bank114661 +114662 POINT(40.88689920684399 73.93348104141617) bank114662 +114663 POINT(40.507256313830375 73.80221688463082) bank114663 +114664 POINT(40.93483807365075 73.15696728286647) bank114664 +114665 POINT(40.12373860336629 74.66192135819365) bank114665 +114666 POINT(40.8776895634601 73.6996342441247) bank114666 +114667 POINT(40.84069090243112 74.09165868068786) bank114667 +114668 POINT(40.13982123882251 73.32959204604924) bank114668 +114669 POINT(41.52483120122658 74.80070785198981) bank114669 +114670 POINT(40.54158111543473 74.71845873688683) bank114670 +114671 POINT(41.40204060622051 74.4189544179413) bank114671 +114672 POINT(39.76783929841571 74.43906961290551) bank114672 +114673 POINT(40.92317718183355 74.48628345153541) bank114673 +114674 POINT(40.565264862828805 73.65543083989009) bank114674 +114675 POINT(40.17364409636829 73.57465421778912) bank114675 +114676 POINT(40.962663903456274 73.40494249772155) bank114676 +114677 POINT(40.61465107977096 74.13944366188451) bank114677 +114678 POINT(40.55859208623488 73.28547402986328) bank114678 +114679 POINT(40.81017016610261 74.25703243635554) bank114679 +114680 POINT(40.54794917058711 74.65832019744359) bank114680 +114681 POINT(40.70202259969099 73.81575746666621) bank114681 +114682 POINT(40.85835752239585 74.21951140907514) bank114682 +114683 POINT(40.46194328040067 74.94045837612494) bank114683 +114684 POINT(40.25057704155796 74.54883021720931) bank114684 +114685 POINT(39.86183169222661 74.21561218918409) bank114685 +114686 POINT(41.53970437383077 73.81703542648556) bank114686 +114687 POINT(41.14602323975108 74.12468477527818) bank114687 +114688 POINT(40.64862518711454 74.26525203262416) bank114688 +114689 POINT(41.55471374846588 74.8503951849748) bank114689 +114690 POINT(41.35628353706951 74.40271256041873) bank114690 +114691 POINT(39.96015648817125 73.4713431228316) bank114691 +114692 POINT(40.13053980508741 73.9843867769483) bank114692 +114693 POINT(39.845341002688855 74.17659158243006) bank114693 +114694 POINT(41.032368817224175 73.83533228536683) bank114694 +114695 POINT(41.67655298295349 73.4598222269065) bank114695 +114696 POINT(40.640578240595765 73.71275832114829) bank114696 +114697 POINT(40.823910894581395 74.02259747854863) bank114697 +114698 POINT(40.244987413455554 73.9421145631462) bank114698 +114699 POINT(40.66855699256285 74.49902421758823) bank114699 +114700 POINT(40.854063188677834 73.10018484292138) bank114700 +114701 POINT(40.016758009581935 74.4851041201836) bank114701 +114702 POINT(40.59711680636409 73.91706023832838) bank114702 +114703 POINT(39.81598038872801 74.80279634815231) bank114703 +114704 POINT(40.86366658204501 74.35880320387221) bank114704 +114705 POINT(41.470851277934 73.45193007653207) bank114705 +114706 POINT(40.86442034366123 74.3818477123962) bank114706 +114707 POINT(41.09466866553016 74.09979297739356) bank114707 +114708 POINT(39.927127922942105 74.23078610327362) bank114708 +114709 POINT(40.020989470728566 73.69436796247282) bank114709 +114710 POINT(39.99412603032455 74.0776784999268) bank114710 +114711 POINT(40.663967853393466 74.0031768820214) bank114711 +114712 POINT(40.89421200344599 74.50915045593591) bank114712 +114713 POINT(40.43728947254451 73.29382403144471) bank114713 +114714 POINT(39.94499181032771 73.79787969536964) bank114714 +114715 POINT(39.98174678279918 74.31637582795588) bank114715 +114716 POINT(40.326136010191036 73.08851290525399) bank114716 +114717 POINT(41.38843130519368 74.14332095783162) bank114717 +114718 POINT(40.77575142105905 74.77328675034856) bank114718 +114719 POINT(40.9254979455419 74.38512919124453) bank114719 +114720 POINT(40.73319600884304 74.24091410840683) bank114720 +114721 POINT(41.10230295466402 73.82049439255482) bank114721 +114722 POINT(40.75693837797561 74.21140281280273) bank114722 +114723 POINT(41.00787104580552 73.64639626949452) bank114723 +114724 POINT(40.1516260876889 73.75558288230181) bank114724 +114725 POINT(41.48390051134582 73.63356153017429) bank114725 +114726 POINT(40.11812814209574 74.45238835163363) bank114726 +114727 POINT(41.5685946865654 73.24700311033463) bank114727 +114728 POINT(40.021698398972354 74.20564951291367) bank114728 +114729 POINT(40.78797591279367 73.31123135681403) bank114729 +114730 POINT(40.92055528688446 73.88854993983514) bank114730 +114731 POINT(41.70063954324872 73.701244591274) bank114731 +114732 POINT(39.80282657520915 74.8226007684129) bank114732 +114733 POINT(41.21873745458007 73.5711611266415) bank114733 +114734 POINT(40.17666293158523 73.80531630345705) bank114734 +114735 POINT(41.54387104796293 74.68397398281806) bank114735 +114736 POINT(39.791673703805436 74.9272487325951) bank114736 +114737 POINT(39.97511365198718 74.44806244110761) bank114737 +114738 POINT(39.909448453779014 74.38889429519703) bank114738 +114739 POINT(39.92598951029068 73.50678797710833) bank114739 +114740 POINT(40.20046076340184 74.83387972104033) bank114740 +114741 POINT(40.073065736300215 74.12653315671642) bank114741 +114742 POINT(40.16634511937755 74.2730402600793) bank114742 +114743 POINT(40.96976509522513 74.63352689481447) bank114743 +114744 POINT(41.53815355093996 74.60506042603834) bank114744 +114745 POINT(41.65005022378046 73.43408949505215) bank114745 +114746 POINT(40.3992752224776 73.57339181886944) bank114746 +114747 POINT(40.67535481881066 73.46046999548203) bank114747 +114748 POINT(40.457743143314694 74.37249976611116) bank114748 +114749 POINT(41.604467845526464 73.47000789702777) bank114749 +114750 POINT(40.092289404580356 74.50995722435387) bank114750 +114751 POINT(41.54170782675234 73.85267266491282) bank114751 +114752 POINT(39.76542888094801 74.72647868122544) bank114752 +114753 POINT(40.79866354680161 74.8146553514988) bank114753 +114754 POINT(39.89795868779911 73.83735741766925) bank114754 +114755 POINT(40.0345730560075 73.86092302958569) bank114755 +114756 POINT(41.03273754861407 73.19551696934174) bank114756 +114757 POINT(40.032210811904186 74.89767569246904) bank114757 +114758 POINT(40.440944637936816 74.17669489272366) bank114758 +114759 POINT(40.95654565297875 73.48853189471087) bank114759 +114760 POINT(41.36821104236643 74.23072423765356) bank114760 +114761 POINT(41.51748415460847 74.48577015229158) bank114761 +114762 POINT(41.53009878338462 73.78035654168863) bank114762 +114763 POINT(41.24673965752153 74.44702108092144) bank114763 +114764 POINT(40.85359309608777 74.26251697908167) bank114764 +114765 POINT(41.22736603776417 73.83120518861779) bank114765 +114766 POINT(40.85043497713355 73.24161211421078) bank114766 +114767 POINT(41.07161624479085 73.39369672647832) bank114767 +114768 POINT(40.57948288119444 74.6108272301736) bank114768 +114769 POINT(41.462203339770525 74.29840379894064) bank114769 +114770 POINT(41.25609075147156 74.63144330750957) bank114770 +114771 POINT(41.52346654796325 73.67811273851754) bank114771 +114772 POINT(41.561338934221425 73.16669244772352) bank114772 +114773 POINT(41.526578285858356 74.72041322159194) bank114773 +114774 POINT(40.72604144010082 74.08877389764749) bank114774 +114775 POINT(40.23427545002999 73.28107503184896) bank114775 +114776 POINT(40.022539117983264 74.25279186085753) bank114776 +114777 POINT(41.596083223163916 74.51775848313633) bank114777 +114778 POINT(40.2883899345079 74.62183141406938) bank114778 +114779 POINT(41.514624885161375 74.47720899758936) bank114779 +114780 POINT(40.323282466174696 73.75326433620721) bank114780 +114781 POINT(40.31442308919377 73.08038360086012) bank114781 +114782 POINT(40.03204993273748 73.9691842175472) bank114782 +114783 POINT(41.57528730740074 73.6018857446904) bank114783 +114784 POINT(41.62738212059758 73.16721177444985) bank114784 +114785 POINT(41.53064841383795 73.0141182213683) bank114785 +114786 POINT(41.30487508842772 73.53747440952945) bank114786 +114787 POINT(39.958217980440935 74.48701646824605) bank114787 +114788 POINT(39.84864741734596 73.06344019307565) bank114788 +114789 POINT(40.616349367642705 73.96352708878986) bank114789 +114790 POINT(40.408168453740494 74.0855876205735) bank114790 +114791 POINT(39.96593604877026 73.19320582704982) bank114791 +114792 POINT(40.78531232080513 73.67706854276966) bank114792 +114793 POINT(40.552395760764085 74.74992105595787) bank114793 +114794 POINT(41.49149765885822 73.19667165287956) bank114794 +114795 POINT(41.53363327559769 74.15497968543255) bank114795 +114796 POINT(40.744822512827774 74.21249603065156) bank114796 +114797 POINT(41.02091939458214 73.65506196833219) bank114797 +114798 POINT(40.07714847240026 73.60466928086188) bank114798 +114799 POINT(41.006668364422936 74.94785330414412) bank114799 +114800 POINT(40.91831909248887 74.90790515770203) bank114800 +114801 POINT(40.49825881466199 74.96488404416577) bank114801 +114802 POINT(41.50458568534738 74.76177411036227) bank114802 +114803 POINT(40.711127354849665 73.08427732841601) bank114803 +114804 POINT(40.94533287388174 74.19987442657612) bank114804 +114805 POINT(39.76004882370355 74.04400238988329) bank114805 +114806 POINT(40.451478968611944 74.62822810735969) bank114806 +114807 POINT(40.325782016297985 73.13119698401712) bank114807 +114808 POINT(40.925160760826614 74.08293326444765) bank114808 +114809 POINT(41.20175605066569 74.0724082347684) bank114809 +114810 POINT(41.12165882493693 73.79602471464644) bank114810 +114811 POINT(39.72388752239607 74.37785971842717) bank114811 +114812 POINT(41.55008656716299 74.62974065286225) bank114812 +114813 POINT(41.44295539953032 74.19455018723916) bank114813 +114814 POINT(40.01449136367497 73.50566653210205) bank114814 +114815 POINT(39.97008553911864 73.97130758046885) bank114815 +114816 POINT(41.59040596424125 74.60798218463437) bank114816 +114817 POINT(40.145674617140926 73.95497022617471) bank114817 +114818 POINT(40.47281300460631 74.79172703759805) bank114818 +114819 POINT(40.77115512097948 73.67255057993364) bank114819 +114820 POINT(41.67713603343497 73.39434821031901) bank114820 +114821 POINT(40.70116009398212 73.38948019633483) bank114821 +114822 POINT(40.08989504857488 73.245095730545) bank114822 +114823 POINT(40.659170525680665 73.75106637399004) bank114823 +114824 POINT(40.85771661265031 74.92060247185658) bank114824 +114825 POINT(40.96600205206912 74.19625841176253) bank114825 +114826 POINT(41.692733212435755 74.38867936242033) bank114826 +114827 POINT(40.00186800106784 74.90885217347518) bank114827 +114828 POINT(39.8188134038807 73.66735827550131) bank114828 +114829 POINT(40.69833527767275 73.27339571223123) bank114829 +114830 POINT(41.2126397266517 74.73547170276001) bank114830 +114831 POINT(40.25643129476318 73.61851393912518) bank114831 +114832 POINT(40.19796395139218 73.16649261599689) bank114832 +114833 POINT(39.95888976836556 73.0716799731392) bank114833 +114834 POINT(40.58296656297408 73.10753140124292) bank114834 +114835 POINT(40.93493550416959 73.63932029076506) bank114835 +114836 POINT(41.36426406105696 73.73616145858001) bank114836 +114837 POINT(40.924521958381874 73.76038914766684) bank114837 +114838 POINT(39.865092551496915 73.45596230906423) bank114838 +114839 POINT(41.54891250647809 73.33632134929398) bank114839 +114840 POINT(40.387484028859085 73.73905193408703) bank114840 +114841 POINT(40.78096560998121 73.70775630080634) bank114841 +114842 POINT(39.762775088521856 73.72205762682593) bank114842 +114843 POINT(40.47363797275907 74.28445926738978) bank114843 +114844 POINT(40.755154789730604 73.48837472378803) bank114844 +114845 POINT(40.87187936298931 73.15680958261407) bank114845 +114846 POINT(40.320445812697834 74.29144316966303) bank114846 +114847 POINT(40.51286616916821 73.37743487051499) bank114847 +114848 POINT(39.76992619351145 74.35383084675189) bank114848 +114849 POINT(40.3114663593363 73.59471884463872) bank114849 +114850 POINT(40.63566112273776 73.90565725392447) bank114850 +114851 POINT(39.947553886316484 74.63484990977128) bank114851 +114852 POINT(41.60393136865096 74.34069953897024) bank114852 +114853 POINT(41.33387250240142 74.26005000729117) bank114853 +114854 POINT(39.966682381869205 74.84338626569489) bank114854 +114855 POINT(40.198695415374026 74.05917547018029) bank114855 +114856 POINT(40.38963035708403 73.8675368562985) bank114856 +114857 POINT(39.96865322664023 73.65797982811148) bank114857 +114858 POINT(39.94543173137768 73.84251224601756) bank114858 +114859 POINT(40.19135485562627 73.4417243460565) bank114859 +114860 POINT(41.66588596939146 74.92829323977588) bank114860 +114861 POINT(40.99800904382276 73.47859196378072) bank114861 +114862 POINT(40.69961106252454 74.97954242527682) bank114862 +114863 POINT(41.1324626370688 73.21973639740695) bank114863 +114864 POINT(39.76737380216013 73.64002207797586) bank114864 +114865 POINT(40.27176711203327 74.70466691634) bank114865 +114866 POINT(40.957055948798526 74.75310455011186) bank114866 +114867 POINT(41.55133831473106 73.50458198329379) bank114867 +114868 POINT(39.88071209348666 74.75470361556458) bank114868 +114869 POINT(41.67628656593738 73.13757899159472) bank114869 +114870 POINT(40.91580534824929 74.8215013803262) bank114870 +114871 POINT(39.9538815560738 73.31937319227957) bank114871 +114872 POINT(40.59247518063762 74.65885622014636) bank114872 +114873 POINT(40.387739017951276 74.81328299088611) bank114873 +114874 POINT(41.227283877058696 74.60611773528306) bank114874 +114875 POINT(40.88083655946635 74.05690003015397) bank114875 +114876 POINT(40.50353517931123 73.88805491291261) bank114876 +114877 POINT(39.750132098544135 73.66573767265075) bank114877 +114878 POINT(40.4667269506413 73.16613078221886) bank114878 +114879 POINT(41.05276031806388 74.55817793180441) bank114879 +114880 POINT(39.790764751858596 74.31980037815045) bank114880 +114881 POINT(41.60006549082962 74.09125980595554) bank114881 +114882 POINT(40.17255159528571 73.94350197255902) bank114882 +114883 POINT(40.551468465399154 73.77534448042884) bank114883 +114884 POINT(40.40454680027351 74.00920704341516) bank114884 +114885 POINT(40.21762286753889 73.48131784601448) bank114885 +114886 POINT(40.41523221907459 74.72627220211268) bank114886 +114887 POINT(41.26930892152593 74.49873964699752) bank114887 +114888 POINT(40.32143745445824 73.1335551929563) bank114888 +114889 POINT(41.2873406814947 74.51637389896037) bank114889 +114890 POINT(40.324194071661665 74.28201736331454) bank114890 +114891 POINT(41.305611389094814 73.11891110680683) bank114891 +114892 POINT(41.525133312782415 74.79588539964361) bank114892 +114893 POINT(40.02503691028756 73.57331457157517) bank114893 +114894 POINT(40.27130918267432 74.46798085487733) bank114894 +114895 POINT(40.019005306836235 73.03861794500823) bank114895 +114896 POINT(41.34535461829411 73.4726849602177) bank114896 +114897 POINT(41.15803637919031 74.70887596320841) bank114897 +114898 POINT(41.65156555478573 73.08063078101928) bank114898 +114899 POINT(39.93090780276174 73.94009289864896) bank114899 +114900 POINT(40.65201898973016 74.22995804737712) bank114900 +114901 POINT(39.77280577895319 73.67938419732786) bank114901 +114902 POINT(40.609016941094005 74.24073644006602) bank114902 +114903 POINT(39.800887559169844 74.96280428968326) bank114903 +114904 POINT(40.37269405534838 74.62943308595858) bank114904 +114905 POINT(39.89286617255222 74.69699838526971) bank114905 +114906 POINT(40.768762079906004 74.22453877160662) bank114906 +114907 POINT(41.093578990408744 74.93785357080259) bank114907 +114908 POINT(39.869741794007375 74.42140894332002) bank114908 +114909 POINT(39.80299416640666 74.4987873816753) bank114909 +114910 POINT(41.43056070984194 74.47566325215774) bank114910 +114911 POINT(39.82474357804376 73.70629622072202) bank114911 +114912 POINT(41.228876090576286 73.23054424521214) bank114912 +114913 POINT(41.53028162694597 73.46698557809859) bank114913 +114914 POINT(41.41318459723263 74.268381880576) bank114914 +114915 POINT(40.348202137989496 73.81020205593279) bank114915 +114916 POINT(40.76226262593994 73.82847694594557) bank114916 +114917 POINT(41.67308876185484 73.23369590909698) bank114917 +114918 POINT(40.07061770165152 73.54754262508192) bank114918 +114919 POINT(40.7634939776718 74.56706167517858) bank114919 +114920 POINT(40.313346734132956 74.47197005059498) bank114920 +114921 POINT(40.04885362309493 74.71317123474662) bank114921 +114922 POINT(40.47695197078527 74.99730853016516) bank114922 +114923 POINT(40.391024385417616 73.68993962079287) bank114923 +114924 POINT(40.866374386945054 74.20119193284569) bank114924 +114925 POINT(41.53622590057232 74.06885010791493) bank114925 +114926 POINT(40.79440109972851 74.72458944988294) bank114926 +114927 POINT(41.28355517880158 74.91245463890837) bank114927 +114928 POINT(40.427186821017735 74.38300355509179) bank114928 +114929 POINT(41.357184167120764 74.82329450625656) bank114929 +114930 POINT(40.75940640904482 74.22834113300785) bank114930 +114931 POINT(40.01202462402726 73.03995074622641) bank114931 +114932 POINT(41.55153137641768 74.2757038007181) bank114932 +114933 POINT(41.69445431893197 73.59681139022557) bank114933 +114934 POINT(40.68309682748024 74.02157311507375) bank114934 +114935 POINT(41.36489239324478 74.96561074424635) bank114935 +114936 POINT(41.433388621637754 74.94185112195136) bank114936 +114937 POINT(40.72989572828831 73.90222146157839) bank114937 +114938 POINT(41.640178429111785 73.37507298144247) bank114938 +114939 POINT(40.729921411276564 73.89180849144643) bank114939 +114940 POINT(41.106422860198364 74.5729913589791) bank114940 +114941 POINT(40.68001086101112 74.21894585102947) bank114941 +114942 POINT(41.0640418584344 74.6233786228782) bank114942 +114943 POINT(41.623115896368596 74.6094168510755) bank114943 +114944 POINT(40.18704506113951 74.48034396293409) bank114944 +114945 POINT(41.656052271081315 73.96364266209105) bank114945 +114946 POINT(41.01585074261668 74.1411620518985) bank114946 +114947 POINT(41.67782708113036 74.24195480339162) bank114947 +114948 POINT(41.18232267006338 74.98729377532473) bank114948 +114949 POINT(41.19049264379483 74.47004203403537) bank114949 +114950 POINT(41.424388029633185 73.29380263837268) bank114950 +114951 POINT(40.665001076680724 73.15377735484851) bank114951 +114952 POINT(41.42450304830589 73.55593742169381) bank114952 +114953 POINT(41.66041439241337 74.21950777283752) bank114953 +114954 POINT(41.64593746401837 74.66415509076137) bank114954 +114955 POINT(40.36293504453124 74.62403044190481) bank114955 +114956 POINT(40.576082183333774 73.17839787864747) bank114956 +114957 POINT(39.78854296327562 73.30150751327433) bank114957 +114958 POINT(40.818300939422386 74.5766247979735) bank114958 +114959 POINT(39.966038157711175 74.19109667963923) bank114959 +114960 POINT(40.01878771355998 73.81901829342176) bank114960 +114961 POINT(41.53924189933013 74.59441804796013) bank114961 +114962 POINT(40.92081373258037 73.91405388325227) bank114962 +114963 POINT(40.58129658900723 74.02023577618775) bank114963 +114964 POINT(39.87421268780791 73.45852836298165) bank114964 +114965 POINT(40.255345725854276 74.1825124317859) bank114965 +114966 POINT(40.29310533689014 73.73041299562362) bank114966 +114967 POINT(40.08100623952219 74.37852476798885) bank114967 +114968 POINT(41.6255754521258 74.08486344212528) bank114968 +114969 POINT(40.87369368702007 73.4947499652851) bank114969 +114970 POINT(41.586163396225544 74.6662817378916) bank114970 +114971 POINT(41.47356183565716 74.14921153425507) bank114971 +114972 POINT(40.59168926088263 74.79476097910799) bank114972 +114973 POINT(40.800602771805295 73.21181779188704) bank114973 +114974 POINT(41.18751129436368 73.70146426869663) bank114974 +114975 POINT(41.26511585908915 74.67085973379338) bank114975 +114976 POINT(40.20983546857351 73.06473157589285) bank114976 +114977 POINT(40.11418749643201 73.18250166559484) bank114977 +114978 POINT(40.65777343675502 73.66825751435346) bank114978 +114979 POINT(40.81533919889599 74.43129000143792) bank114979 +114980 POINT(39.83828647752954 74.6435745295879) bank114980 +114981 POINT(39.94356475496594 74.73500667720609) bank114981 +114982 POINT(40.51393452880092 74.7658984499165) bank114982 +114983 POINT(40.83739040662337 74.79507500302189) bank114983 +114984 POINT(40.554309141813974 73.79832015911111) bank114984 +114985 POINT(40.04101314725426 73.38913583776343) bank114985 +114986 POINT(40.64147356759755 74.97093341808448) bank114986 +114987 POINT(41.14901003024468 74.92776804843471) bank114987 +114988 POINT(39.994550453765925 74.41080302701306) bank114988 +114989 POINT(40.91069150300567 74.41051879965569) bank114989 +114990 POINT(39.76897796983916 74.15188539870482) bank114990 +114991 POINT(41.21908896571072 74.25793362527386) bank114991 +114992 POINT(41.49275565138194 73.52589309277766) bank114992 +114993 POINT(40.37063323780034 73.92819719669336) bank114993 +114994 POINT(41.55269030698513 74.19861958598204) bank114994 +114995 POINT(40.54069435142197 73.78574752990782) bank114995 +114996 POINT(40.34518106336063 74.24903654768936) bank114996 +114997 POINT(40.68681503991784 74.12377082517293) bank114997 +114998 POINT(40.22714868826962 74.18807081415524) bank114998 +114999 POINT(41.204875651897616 74.90274233907088) bank114999 +115000 POINT(41.3998561008331 73.98424801572601) bank115000 +115001 POINT(40.39889350502803 73.7484754745861) bank115001 +115002 POINT(40.532042611204574 74.61974047554095) bank115002 +115003 POINT(40.215129960147706 74.33086885171716) bank115003 +115004 POINT(41.62215502218593 73.3151754046382) bank115004 +115005 POINT(40.36067797381998 73.70306297142565) bank115005 +115006 POINT(40.78372642076335 73.20495898837173) bank115006 +115007 POINT(40.13053828584589 74.98619027442601) bank115007 +115008 POINT(40.73595186765357 74.15398374566873) bank115008 +115009 POINT(40.1882848355218 74.54784493737407) bank115009 +115010 POINT(40.082154061853245 74.36253865622051) bank115010 +115011 POINT(40.45533577580318 74.50984638432142) bank115011 +115012 POINT(39.93942700229678 73.51518392271711) bank115012 +115013 POINT(40.376268310076156 73.6139032207047) bank115013 +115014 POINT(40.406829609952865 74.17344026689543) bank115014 +115015 POINT(40.21301063512388 73.34598952177633) bank115015 +115016 POINT(39.82173355562697 73.18980841932526) bank115016 +115017 POINT(41.19740630612579 74.18145500406256) bank115017 +115018 POINT(40.74829361000301 74.34916456029816) bank115018 +115019 POINT(39.934206808321925 74.82293620375737) bank115019 +115020 POINT(40.0329630448913 73.61396426571234) bank115020 +115021 POINT(41.12328103240625 74.08990415738272) bank115021 +115022 POINT(40.30333059409497 74.1151584610323) bank115022 +115023 POINT(40.880543791470416 73.33697995437149) bank115023 +115024 POINT(40.014802430271416 74.00903739924573) bank115024 +115025 POINT(40.03759438586777 74.69630816090203) bank115025 +115026 POINT(39.96361915283113 73.99816861657688) bank115026 +115027 POINT(40.0397232605384 73.78008416392927) bank115027 +115028 POINT(41.07538889911944 74.65422133242255) bank115028 +115029 POINT(39.98209254611652 74.72702810530355) bank115029 +115030 POINT(40.19209094351138 73.72317607984945) bank115030 +115031 POINT(40.474922734541074 74.43239469191545) bank115031 +115032 POINT(39.776327601572234 73.32042503508262) bank115032 +115033 POINT(39.82702581065859 74.33798610720375) bank115033 +115034 POINT(39.874664327974024 73.89408410504805) bank115034 +115035 POINT(41.324170482993004 74.13282518638624) bank115035 +115036 POINT(40.01819870054514 73.9086829711509) bank115036 +115037 POINT(41.21480813363912 73.44305117624957) bank115037 +115038 POINT(41.70536298368912 74.06598145971857) bank115038 +115039 POINT(41.25641720600964 73.97702834994713) bank115039 +115040 POINT(39.87704091275396 73.19031540441429) bank115040 +115041 POINT(40.88529606385372 74.58598257313955) bank115041 +115042 POINT(39.99297514138756 73.70793279453696) bank115042 +115043 POINT(40.8429928476528 73.18467789633068) bank115043 +115044 POINT(40.8813096180655 74.35151707553462) bank115044 +115045 POINT(39.786322496102834 73.94151244165218) bank115045 +115046 POINT(40.63356582403382 73.23736436011788) bank115046 +115047 POINT(40.63684953054382 74.61305225706913) bank115047 +115048 POINT(40.88083291147488 74.36489439138288) bank115048 +115049 POINT(41.7092731489075 74.27661557809115) bank115049 +115050 POINT(40.095235679052266 73.250412869527) bank115050 +115051 POINT(40.82631695947963 74.64485288400357) bank115051 +115052 POINT(40.22800413042096 74.18188338614593) bank115052 +115053 POINT(39.93624251961773 74.71210426499779) bank115053 +115054 POINT(41.53455227546922 74.78736880284107) bank115054 +115055 POINT(40.226144610384175 74.33358972663174) bank115055 +115056 POINT(40.38758615902185 74.0999123350101) bank115056 +115057 POINT(41.657234646784474 74.9076869456131) bank115057 +115058 POINT(40.443008020320626 73.35199125011842) bank115058 +115059 POINT(41.34536043537612 74.80789038916744) bank115059 +115060 POINT(41.64138625739136 74.84845311768225) bank115060 +115061 POINT(41.7110975766294 73.21425920329693) bank115061 +115062 POINT(40.87628152494039 73.09544874272767) bank115062 +115063 POINT(40.42918296072129 73.11729232175969) bank115063 +115064 POINT(41.49758168099667 74.258651405125) bank115064 +115065 POINT(41.09834243310994 74.07228530016657) bank115065 +115066 POINT(40.5183233799536 74.7390739033597) bank115066 +115067 POINT(40.16244719329272 73.99319250357095) bank115067 +115068 POINT(40.68393314222411 73.04830354666531) bank115068 +115069 POINT(40.15657817705042 74.22717685470339) bank115069 +115070 POINT(40.049756653425646 73.8061775455361) bank115070 +115071 POINT(41.705795467477685 74.3094028944012) bank115071 +115072 POINT(41.54129244491962 73.91001385293521) bank115072 +115073 POINT(40.15098958174791 73.5524038287333) bank115073 +115074 POINT(40.34940854470604 73.95446900741192) bank115074 +115075 POINT(41.68764120439004 73.28014563852804) bank115075 +115076 POINT(40.391624972455425 73.90706784877908) bank115076 +115077 POINT(39.94703327925331 73.71258697540722) bank115077 +115078 POINT(41.07417158832834 74.04812882244502) bank115078 +115079 POINT(39.86855585322552 73.0328151454995) bank115079 +115080 POINT(41.29181052122259 73.8936644095047) bank115080 +115081 POINT(39.861482206116065 73.0506929999918) bank115081 +115082 POINT(41.260753219918364 74.74880834295365) bank115082 +115083 POINT(41.26037881847838 73.6943566705113) bank115083 +115084 POINT(39.7160656333552 74.49256066474929) bank115084 +115085 POINT(40.811620465920186 74.70258350915256) bank115085 +115086 POINT(40.70511459108482 73.24897405170871) bank115086 +115087 POINT(39.71660173657725 73.81905984795311) bank115087 +115088 POINT(40.92680343091489 73.92924564715204) bank115088 +115089 POINT(40.71455756995811 73.57406127452305) bank115089 +115090 POINT(39.72949093646282 73.6317015355817) bank115090 +115091 POINT(41.292751490584386 73.40093230627168) bank115091 +115092 POINT(40.6156273249461 74.99661571666265) bank115092 +115093 POINT(40.36049201105073 73.82263976337417) bank115093 +115094 POINT(40.53495315623646 74.249679236346) bank115094 +115095 POINT(41.00009245710119 73.96833637408015) bank115095 +115096 POINT(40.34292054712933 74.0548135927014) bank115096 +115097 POINT(40.31010058357882 73.38435177292789) bank115097 +115098 POINT(39.93049678068995 73.77923342672446) bank115098 +115099 POINT(39.92120269866959 73.40922324014147) bank115099 +115100 POINT(40.44243129369334 73.33083560518998) bank115100 +115101 POINT(39.9594736569743 74.65992899338961) bank115101 +115102 POINT(40.194347983517 74.48662527355823) bank115102 +115103 POINT(39.87424978329866 74.3045530848769) bank115103 +115104 POINT(39.85374897303446 73.64437770187023) bank115104 +115105 POINT(41.6583477345449 73.58406127237502) bank115105 +115106 POINT(40.198767900999684 74.04443777871843) bank115106 +115107 POINT(40.283704229739385 74.2732810747916) bank115107 +115108 POINT(40.50536429261364 74.29178191992466) bank115108 +115109 POINT(40.43506058375981 74.09553367457953) bank115109 +115110 POINT(39.824796980716116 74.49614971912901) bank115110 +115111 POINT(41.58404440575102 73.44017330800173) bank115111 +115112 POINT(40.34426467322356 74.1049429990276) bank115112 +115113 POINT(40.434805624165534 74.43123575734158) bank115113 +115114 POINT(40.20058281623165 73.50581497363159) bank115114 +115115 POINT(41.44151876596332 74.80808307328819) bank115115 +115116 POINT(40.80102516417389 73.0119209481033) bank115116 +115117 POINT(41.02777965887025 74.08474746896408) bank115117 +115118 POINT(39.96196957333084 73.8350470975914) bank115118 +115119 POINT(40.26593985503954 73.53643533509864) bank115119 +115120 POINT(40.481815495511064 74.56480895810574) bank115120 +115121 POINT(39.729469202737555 73.33481393567158) bank115121 +115122 POINT(41.083528253310384 74.20027622598668) bank115122 +115123 POINT(41.54156572549926 73.14468551718014) bank115123 +115124 POINT(41.59452856378324 74.43072565788911) bank115124 +115125 POINT(40.46195778970137 74.45779144869473) bank115125 +115126 POINT(39.926886427583675 73.86187298340806) bank115126 +115127 POINT(40.90494403481099 74.38854715360917) bank115127 +115128 POINT(40.12627303624673 73.57075058402683) bank115128 +115129 POINT(40.70295726385498 73.32481753217353) bank115129 +115130 POINT(40.186439572436306 74.11085389846181) bank115130 +115131 POINT(39.87676540740053 74.58337937966846) bank115131 +115132 POINT(39.982580052743984 74.17446510747656) bank115132 +115133 POINT(40.90934704488037 74.62940623741888) bank115133 +115134 POINT(40.58387647612207 73.1364612656292) bank115134 +115135 POINT(41.60544039011632 74.8496367083032) bank115135 +115136 POINT(41.08836058234564 74.36402544670558) bank115136 +115137 POINT(41.646834558981745 74.48692365818981) bank115137 +115138 POINT(41.062507861231865 74.61670939693525) bank115138 +115139 POINT(40.44433193356871 74.70559352700074) bank115139 +115140 POINT(40.67157063631392 74.5012668890562) bank115140 +115141 POINT(39.91091889013427 73.36820824849218) bank115141 +115142 POINT(40.49785047308473 74.20350827803281) bank115142 +115143 POINT(40.464378054289305 74.69395523517136) bank115143 +115144 POINT(41.487880908954196 74.60622364195699) bank115144 +115145 POINT(40.24470444499093 73.83087782143818) bank115145 +115146 POINT(40.716091553266416 73.0258465293926) bank115146 +115147 POINT(40.09218871107045 73.32853874479328) bank115147 +115148 POINT(40.61450535459954 73.79849505657526) bank115148 +115149 POINT(39.95088428140943 74.1827085102241) bank115149 +115150 POINT(40.92685550076004 73.95656400049774) bank115150 +115151 POINT(40.888795979548874 73.28777613869299) bank115151 +115152 POINT(41.45195888221899 74.35608323153491) bank115152 +115153 POINT(40.984196546147984 73.04930091463633) bank115153 +115154 POINT(41.45344501686604 73.26697743185245) bank115154 +115155 POINT(39.89698449228961 74.94150877273059) bank115155 +115156 POINT(39.797118339245465 74.03739094499768) bank115156 +115157 POINT(39.94034120732781 73.27730956942277) bank115157 +115158 POINT(40.27647759758924 74.55233286601569) bank115158 +115159 POINT(41.60550714180589 73.46900526730236) bank115159 +115160 POINT(41.34652057898035 73.59758326640303) bank115160 +115161 POINT(41.25221212794617 73.42747472857073) bank115161 +115162 POINT(41.28703034515243 73.46823673091949) bank115162 +115163 POINT(40.02875499622 74.08768738992242) bank115163 +115164 POINT(40.55705842620028 73.8226067416048) bank115164 +115165 POINT(41.178064303165364 74.46547643668117) bank115165 +115166 POINT(40.80298231360195 74.70424558421496) bank115166 +115167 POINT(41.45236414644759 74.31126517569248) bank115167 +115168 POINT(40.77542235024535 73.58889130402811) bank115168 +115169 POINT(39.84427129472345 74.60491140546154) bank115169 +115170 POINT(41.2648750037447 74.25621133746347) bank115170 +115171 POINT(40.66434499631383 73.51855265263717) bank115171 +115172 POINT(40.688976903391996 74.71297276868177) bank115172 +115173 POINT(40.29311690318816 74.27271915893532) bank115173 +115174 POINT(41.04986755673144 74.8365983620748) bank115174 +115175 POINT(40.23705438978389 73.601566741418) bank115175 +115176 POINT(41.26755818279969 73.29161973307478) bank115176 +115177 POINT(40.02788411603775 74.9334862623742) bank115177 +115178 POINT(41.089329839435536 74.9579851911509) bank115178 +115179 POINT(40.65931175042587 73.5920117902181) bank115179 +115180 POINT(40.03385423148506 74.46162206432992) bank115180 +115181 POINT(40.98627848842548 73.0477440062667) bank115181 +115182 POINT(40.43746779920453 73.96252183463771) bank115182 +115183 POINT(40.38567057405832 74.72362446249315) bank115183 +115184 POINT(40.13018204844025 74.18083497553756) bank115184 +115185 POINT(41.499356347162845 73.10392616925164) bank115185 +115186 POINT(40.17194704444698 74.16426305859348) bank115186 +115187 POINT(41.01835917789045 74.29996777034508) bank115187 +115188 POINT(41.046456451483415 73.92659141036239) bank115188 +115189 POINT(40.25088238173258 74.65315302520705) bank115189 +115190 POINT(40.11567737389717 74.23980390053718) bank115190 +115191 POINT(39.88891280243095 74.26432633581481) bank115191 +115192 POINT(39.95120088434815 74.831371488188) bank115192 +115193 POINT(40.05427838987044 73.38833637301066) bank115193 +115194 POINT(40.73730121648763 73.28495559866731) bank115194 +115195 POINT(40.17538377736071 74.3203922054252) bank115195 +115196 POINT(41.42846919098413 73.09217250830704) bank115196 +115197 POINT(39.78545567977253 74.03270172962348) bank115197 +115198 POINT(40.46340500795618 73.53676679171402) bank115198 +115199 POINT(41.34792266355131 73.50313194572823) bank115199 +115200 POINT(41.36160646523425 73.79472410374018) bank115200 +115201 POINT(40.344107289358256 74.90024794891843) bank115201 +115202 POINT(40.01728437892335 73.73171467401409) bank115202 +115203 POINT(40.691857456406645 74.4198479138266) bank115203 +115204 POINT(41.682272361140036 74.84300164111873) bank115204 +115205 POINT(41.503318124946944 73.23714142220933) bank115205 +115206 POINT(41.43834597357639 73.36848663084984) bank115206 +115207 POINT(40.26350774879779 73.71847698316724) bank115207 +115208 POINT(40.710994517587096 73.67189873546214) bank115208 +115209 POINT(39.82949740381267 73.84783014065275) bank115209 +115210 POINT(41.46926685313031 73.20326550871108) bank115210 +115211 POINT(40.53354812040925 74.50958750026419) bank115211 +115212 POINT(40.2919452606378 73.5186032362854) bank115212 +115213 POINT(40.4150411497652 74.32125291815075) bank115213 +115214 POINT(39.71306100490674 73.31074940331399) bank115214 +115215 POINT(40.70066771482366 73.74356742465338) bank115215 +115216 POINT(41.23621019470241 73.76305698668686) bank115216 +115217 POINT(41.188738085203056 74.53659593066344) bank115217 +115218 POINT(40.24856081913374 73.50619177937475) bank115218 +115219 POINT(40.12284540905264 74.68789791120477) bank115219 +115220 POINT(41.32848634330145 73.1704027461984) bank115220 +115221 POINT(40.40389489198189 74.50875392565851) bank115221 +115222 POINT(41.24843435394645 73.7708720794562) bank115222 +115223 POINT(39.86456231704085 74.05707007082715) bank115223 +115224 POINT(40.0339257558066 74.08072894078012) bank115224 +115225 POINT(41.09097012146906 74.88490440110274) bank115225 +115226 POINT(41.12501774989962 73.28218409874995) bank115226 +115227 POINT(41.29782044201983 73.09893567200967) bank115227 +115228 POINT(39.84749992817765 73.71690540369617) bank115228 +115229 POINT(41.135068155135606 73.74013537168148) bank115229 +115230 POINT(41.69106791118099 74.69789690407792) bank115230 +115231 POINT(41.606371721080315 74.23756163744252) bank115231 +115232 POINT(40.32043795907395 73.64246692038603) bank115232 +115233 POINT(40.73364646834288 73.05957853554591) bank115233 +115234 POINT(40.34744852197433 73.27376780551585) bank115234 +115235 POINT(39.943177490748724 73.23795276326514) bank115235 +115236 POINT(41.3094506492475 74.57729619016445) bank115236 +115237 POINT(40.176254085005226 73.70437899221224) bank115237 +115238 POINT(40.86176411758315 73.57341592642648) bank115238 +115239 POINT(40.817448718896806 73.46988654748507) bank115239 +115240 POINT(41.38801624232282 73.17042185749465) bank115240 +115241 POINT(40.85849894541512 73.61124667908803) bank115241 +115242 POINT(40.5587346991786 73.93815259574988) bank115242 +115243 POINT(40.62848078485353 74.69419191189837) bank115243 +115244 POINT(39.90583214995578 73.88217488462435) bank115244 +115245 POINT(40.42293102008771 73.06452826711686) bank115245 +115246 POINT(41.01039389947032 73.6775067166469) bank115246 +115247 POINT(40.64491739371217 74.26900928057545) bank115247 +115248 POINT(40.733366833131264 73.3310316569256) bank115248 +115249 POINT(40.200255985959046 74.23546482422556) bank115249 +115250 POINT(41.17989886008947 73.63063387154482) bank115250 +115251 POINT(40.31495294733851 74.03544118586517) bank115251 +115252 POINT(41.60323888210393 74.87411068889728) bank115252 +115253 POINT(40.94922522066908 74.21890051993525) bank115253 +115254 POINT(41.669243769546185 74.64533623125051) bank115254 +115255 POINT(40.05419093156136 73.41455773760845) bank115255 +115256 POINT(41.56307211546934 74.17226787098375) bank115256 +115257 POINT(40.049595452564816 73.35743980209382) bank115257 +115258 POINT(41.00614533733546 73.01301555176691) bank115258 +115259 POINT(41.425240892373516 74.0465383904533) bank115259 +115260 POINT(41.558920382684164 74.13390603003687) bank115260 +115261 POINT(41.1727353167679 74.00073843019564) bank115261 +115262 POINT(41.58572262856958 73.58816502421) bank115262 +115263 POINT(41.19621100379617 74.26182836041252) bank115263 +115264 POINT(39.897113273816394 74.47599548031201) bank115264 +115265 POINT(40.86986808704389 74.58906192482867) bank115265 +115266 POINT(39.95542204369704 73.93440033397933) bank115266 +115267 POINT(40.285038019232026 74.14569054907923) bank115267 +115268 POINT(40.250127823824194 74.68819264368133) bank115268 +115269 POINT(39.72851347580636 73.60693421546809) bank115269 +115270 POINT(41.13006758311025 74.7186666432188) bank115270 +115271 POINT(41.01641708887135 74.74902698768425) bank115271 +115272 POINT(41.60325153000609 73.1658781199383) bank115272 +115273 POINT(41.546140804063164 74.10301127999165) bank115273 +115274 POINT(39.96863201736204 73.7821041890941) bank115274 +115275 POINT(41.59622483378258 73.59589288981562) bank115275 +115276 POINT(40.77151370674547 74.33018992716488) bank115276 +115277 POINT(41.61719182821821 73.989150744217) bank115277 +115278 POINT(40.20306228050741 74.84069459149528) bank115278 +115279 POINT(40.82131208691371 74.02571311506591) bank115279 +115280 POINT(41.26407907989751 74.19584236441867) bank115280 +115281 POINT(41.433403816369506 73.27192768948238) bank115281 +115282 POINT(40.61637820287232 74.73034246398682) bank115282 +115283 POINT(40.08724239027549 73.51364823768613) bank115283 +115284 POINT(40.71225950724218 74.2514226892343) bank115284 +115285 POINT(40.387423307398635 73.41212901866383) bank115285 +115286 POINT(40.140041466121495 73.24442901928781) bank115286 +115287 POINT(40.43437913839955 74.206811592518) bank115287 +115288 POINT(41.312903697496054 73.21449997746902) bank115288 +115289 POINT(40.43936085615676 74.42367105241784) bank115289 +115290 POINT(40.72093154273485 74.91931816595078) bank115290 +115291 POINT(41.62529799421046 73.64946897167044) bank115291 +115292 POINT(41.31316228682549 74.29465851634254) bank115292 +115293 POINT(40.138341107709444 73.89255477533385) bank115293 +115294 POINT(40.084930151825944 74.57237340647731) bank115294 +115295 POINT(41.70889023410163 73.4378806327453) bank115295 +115296 POINT(40.2046428897586 74.39617571778385) bank115296 +115297 POINT(41.19538860136835 73.93916306874105) bank115297 +115298 POINT(40.96102539130986 74.44311961996456) bank115298 +115299 POINT(41.387937125934414 73.45977370757804) bank115299 +115300 POINT(41.43380445739078 74.01471716419506) bank115300 +115301 POINT(39.72147641184011 74.07626370434667) bank115301 +115302 POINT(40.65440256740866 73.37941765631206) bank115302 +115303 POINT(40.009843518959784 74.11113706659276) bank115303 +115304 POINT(40.800155084316906 74.89316129424526) bank115304 +115305 POINT(40.68899700108178 74.01972151513912) bank115305 +115306 POINT(40.05581904276003 74.05251515024065) bank115306 +115307 POINT(41.61067089490454 74.8398742434579) bank115307 +115308 POINT(40.87748981590425 73.87125134965109) bank115308 +115309 POINT(40.208493926710226 74.82786406573624) bank115309 +115310 POINT(41.52968038178912 74.73155473350126) bank115310 +115311 POINT(39.891506545428435 74.68852448560645) bank115311 +115312 POINT(40.41783200702632 73.5549855848238) bank115312 +115313 POINT(41.290260824838924 73.11055605414775) bank115313 +115314 POINT(40.702461563160256 74.23524310580802) bank115314 +115315 POINT(40.97568471437796 73.97245160656071) bank115315 +115316 POINT(40.3486758657329 74.108480380705) bank115316 +115317 POINT(40.916298320893496 74.76194277048496) bank115317 +115318 POINT(40.82906477275837 74.0052543562349) bank115318 +115319 POINT(41.08687898690222 74.55881348940744) bank115319 +115320 POINT(41.24749341139952 74.83559169441574) bank115320 +115321 POINT(41.15877889300152 73.33802110224727) bank115321 +115322 POINT(40.620156079266664 73.19940252538854) bank115322 +115323 POINT(40.30991691676799 74.77828209808371) bank115323 +115324 POINT(40.91388487445132 73.034959214375) bank115324 +115325 POINT(40.065463588331866 74.10382929161185) bank115325 +115326 POINT(41.22623479859953 74.13869095300113) bank115326 +115327 POINT(41.66494893000262 74.84290941056656) bank115327 +115328 POINT(39.768824453333345 73.81458790284384) bank115328 +115329 POINT(40.41336453742906 74.51064997654055) bank115329 +115330 POINT(39.76646806015003 74.17863178699328) bank115330 +115331 POINT(41.028629859728795 73.62478290296806) bank115331 +115332 POINT(41.57660786507998 74.36314282744947) bank115332 +115333 POINT(39.96326568860724 73.88876775469687) bank115333 +115334 POINT(40.297289393351605 73.5057687844472) bank115334 +115335 POINT(41.56984193016746 73.79574935381584) bank115335 +115336 POINT(40.72275982604833 74.07256535457964) bank115336 +115337 POINT(41.13593939618642 73.13945556101791) bank115337 +115338 POINT(41.38934929534041 74.26926787466698) bank115338 +115339 POINT(40.57692405216605 74.34427450256757) bank115339 +115340 POINT(40.265518233555966 74.48732760658245) bank115340 +115341 POINT(40.58533976107175 74.53345450216237) bank115341 +115342 POINT(41.5301170496064 73.18639718737393) bank115342 +115343 POINT(39.76452718208309 73.67235036302075) bank115343 +115344 POINT(40.342444346200075 73.40375384181517) bank115344 +115345 POINT(41.34009518482286 74.80157970964002) bank115345 +115346 POINT(40.4863012613456 74.63441075704091) bank115346 +115347 POINT(40.482060350893796 74.97356172409444) bank115347 +115348 POINT(39.96916223386406 73.50730181068937) bank115348 +115349 POINT(39.92208025188875 74.91196852711519) bank115349 +115350 POINT(41.31047685539159 73.09275184933632) bank115350 +115351 POINT(39.84592746641803 73.7240898309035) bank115351 +115352 POINT(40.875443953258234 73.37242727680241) bank115352 +115353 POINT(41.38950368177978 74.44310346564606) bank115353 +115354 POINT(41.660699319497056 74.38725951892918) bank115354 +115355 POINT(40.531117390141034 74.34007629585766) bank115355 +115356 POINT(40.59694410957244 73.33542505946488) bank115356 +115357 POINT(41.04308733095415 73.70644488315888) bank115357 +115358 POINT(40.51982899195482 74.31188191084567) bank115358 +115359 POINT(41.04003454852421 74.81130753077059) bank115359 +115360 POINT(41.09101343071784 74.07177064487828) bank115360 +115361 POINT(39.92587407772082 74.15994110197741) bank115361 +115362 POINT(41.36558311282694 73.88864491792138) bank115362 +115363 POINT(39.92930717520127 73.46995122032563) bank115363 +115364 POINT(41.3802107209102 75.00552055950708) bank115364 +115365 POINT(39.988348544334364 73.82031034633981) bank115365 +115366 POINT(40.6807804804331 73.17479583527374) bank115366 +115367 POINT(41.32397149314983 74.39447176252158) bank115367 +115368 POINT(39.977033839806595 74.15323441212375) bank115368 +115369 POINT(40.11306859359242 74.32407390036255) bank115369 +115370 POINT(41.29226243450713 74.90949915026631) bank115370 +115371 POINT(40.99383881442882 73.14430097520862) bank115371 +115372 POINT(40.59737707847241 73.6775766407237) bank115372 +115373 POINT(41.10698089744526 73.87721360924472) bank115373 +115374 POINT(41.5912756109916 74.66239407649664) bank115374 +115375 POINT(41.62607830561004 74.55066590510002) bank115375 +115376 POINT(39.72267534094346 73.50198778533681) bank115376 +115377 POINT(40.23283072068312 74.90574618063084) bank115377 +115378 POINT(40.26898284519592 73.91546624420722) bank115378 +115379 POINT(40.37419218064578 74.43401666450939) bank115379 +115380 POINT(40.14300053336525 74.7103545865215) bank115380 +115381 POINT(41.688104481339394 73.93040785976805) bank115381 +115382 POINT(41.01954006897941 73.21051109242562) bank115382 +115383 POINT(41.553343157392916 73.05845647493231) bank115383 +115384 POINT(41.07813579214651 74.42886179253372) bank115384 +115385 POINT(39.77917006209282 74.64711409120085) bank115385 +115386 POINT(41.333812194110315 74.7681920582378) bank115386 +115387 POINT(40.34859314846925 74.83319735230423) bank115387 +115388 POINT(40.29574569741837 74.68443188223807) bank115388 +115389 POINT(40.84921344262947 73.33710348514627) bank115389 +115390 POINT(40.125237267119495 74.99514889540046) bank115390 +115391 POINT(40.5920973614403 74.97692755178744) bank115391 +115392 POINT(40.93202216636976 74.43831134710156) bank115392 +115393 POINT(40.131909555878906 73.52352325831136) bank115393 +115394 POINT(40.542936393031546 73.92988998776723) bank115394 +115395 POINT(41.53107784999235 74.29233232455934) bank115395 +115396 POINT(40.42317460184566 73.32375605414322) bank115396 +115397 POINT(40.174103583378525 73.95884606133733) bank115397 +115398 POINT(41.03813753917198 74.52325338217958) bank115398 +115399 POINT(41.54189263266767 73.80974291891806) bank115399 +115400 POINT(40.02816691681907 73.5799769433785) bank115400 +115401 POINT(40.81131518488242 74.86198317646583) bank115401 +115402 POINT(40.84564713361329 74.83380916085154) bank115402 +115403 POINT(40.21172756239744 73.53718805946347) bank115403 +115404 POINT(40.51047453720843 73.49389660808596) bank115404 +115405 POINT(40.927449449336834 73.31458006363037) bank115405 +115406 POINT(40.87641971680417 75.00383030447975) bank115406 +115407 POINT(40.211457454886585 73.00759823588443) bank115407 +115408 POINT(39.98561916157587 73.65574982235239) bank115408 +115409 POINT(40.072273093392944 73.80975631760826) bank115409 +115410 POINT(39.75624042431046 74.50974629486697) bank115410 +115411 POINT(41.390708382710145 73.67879829094953) bank115411 +115412 POINT(41.69083023422649 73.64697684477979) bank115412 +115413 POINT(39.76662298672723 74.81777108639052) bank115413 +115414 POINT(40.758764338168184 74.4101734841258) bank115414 +115415 POINT(39.76827905323867 74.06438407340062) bank115415 +115416 POINT(41.30682732115075 73.78507475748935) bank115416 +115417 POINT(40.70693090242175 74.7622392162617) bank115417 +115418 POINT(40.518746682070656 73.08684970997804) bank115418 +115419 POINT(40.23588861243035 73.44102673367756) bank115419 +115420 POINT(40.49722216888278 74.40667691781799) bank115420 +115421 POINT(40.354819139086004 73.70347972504113) bank115421 +115422 POINT(40.86158709378541 74.24849796034874) bank115422 +115423 POINT(40.26153948162341 74.8656458862062) bank115423 +115424 POINT(40.71254933502409 73.65396283358355) bank115424 +115425 POINT(40.380977862876726 73.98478328914638) bank115425 +115426 POINT(39.91231096883999 74.07673555485972) bank115426 +115427 POINT(40.60677347221539 74.69218524830939) bank115427 +115428 POINT(40.613286198415615 74.40706959885291) bank115428 +115429 POINT(41.633715768152854 74.74212710899349) bank115429 +115430 POINT(41.36075988716542 74.965311438479) bank115430 +115431 POINT(40.00286275055106 73.8415708036777) bank115431 +115432 POINT(40.5595272072664 73.863856030372) bank115432 +115433 POINT(40.77301529957883 74.75201905603656) bank115433 +115434 POINT(40.262869784241204 74.9141934158259) bank115434 +115435 POINT(39.85760499558589 73.69883126811774) bank115435 +115436 POINT(41.137728842636456 73.13013867006147) bank115436 +115437 POINT(41.187405707871434 73.14111703053558) bank115437 +115438 POINT(41.05324200146768 73.62711232989403) bank115438 +115439 POINT(41.21695663930307 74.03768529881933) bank115439 +115440 POINT(41.369544309138355 74.7370960827042) bank115440 +115441 POINT(40.027663413131386 74.25345030437613) bank115441 +115442 POINT(39.819547760593316 73.71153369386778) bank115442 +115443 POINT(41.54617363478271 74.19997371889365) bank115443 +115444 POINT(40.8205208320461 73.09025808644111) bank115444 +115445 POINT(41.50325988387005 74.57060125743757) bank115445 +115446 POINT(41.37904979420997 74.06066222396102) bank115446 +115447 POINT(40.96327990322737 73.85893624706803) bank115447 +115448 POINT(41.67317747185622 74.999273353116) bank115448 +115449 POINT(41.2049529638831 74.70266591242243) bank115449 +115450 POINT(40.85109124819142 73.45379644861511) bank115450 +115451 POINT(40.04287848223063 74.43085277752162) bank115451 +115452 POINT(41.60546179292564 73.17009023513542) bank115452 +115453 POINT(41.2876498144264 73.77458846536852) bank115453 +115454 POINT(41.00703075332714 73.72851969754835) bank115454 +115455 POINT(40.10755205348378 73.90233540538065) bank115455 +115456 POINT(40.90619598055399 74.78699180276864) bank115456 +115457 POINT(41.253683382980135 74.87345189412231) bank115457 +115458 POINT(40.41368485150046 73.41108888654566) bank115458 +115459 POINT(41.24017049608256 73.1797912412406) bank115459 +115460 POINT(41.28123686151898 73.47008677728525) bank115460 +115461 POINT(41.59194301353946 74.05067094373666) bank115461 +115462 POINT(39.944602369675636 74.02436000937536) bank115462 +115463 POINT(40.09017204195715 73.25789034860131) bank115463 +115464 POINT(41.34728149140468 73.60213110547082) bank115464 +115465 POINT(39.884320596331015 74.63684638110934) bank115465 +115466 POINT(40.38217078439275 74.51272635655732) bank115466 +115467 POINT(41.32969636321308 73.38409037926148) bank115467 +115468 POINT(40.32764844231688 74.23820673865781) bank115468 +115469 POINT(39.92232334502576 74.084122159391) bank115469 +115470 POINT(39.81548491362704 73.85542334878858) bank115470 +115471 POINT(40.2248874393504 74.44519856812258) bank115471 +115472 POINT(39.80428559463803 73.76339113463293) bank115472 +115473 POINT(40.793782915375125 73.44809936336316) bank115473 +115474 POINT(39.86755347013596 74.3281269995657) bank115474 +115475 POINT(40.565789669353535 73.23062358881195) bank115475 +115476 POINT(39.78862578685714 74.76436242639664) bank115476 +115477 POINT(39.74409742726933 74.09472878985127) bank115477 +115478 POINT(40.92559297167328 74.76804018453852) bank115478 +115479 POINT(40.1092834150787 74.98176558928571) bank115479 +115480 POINT(41.551827259591114 73.2669908046789) bank115480 +115481 POINT(39.86184876771707 74.59685980666038) bank115481 +115482 POINT(41.01767593085193 74.61397676535582) bank115482 +115483 POINT(41.20975424176681 73.34388248941472) bank115483 +115484 POINT(41.61641166405075 74.36969059232273) bank115484 +115485 POINT(39.71479535613959 74.09083840000176) bank115485 +115486 POINT(40.36055797806161 73.86644529809354) bank115486 +115487 POINT(40.18862351913492 73.36777386277554) bank115487 +115488 POINT(40.47121859238278 73.89131568120655) bank115488 +115489 POINT(40.30622543544437 74.71891213973691) bank115489 +115490 POINT(41.511187114375836 73.03565051757748) bank115490 +115491 POINT(41.187202101880544 74.59274098239824) bank115491 +115492 POINT(40.50851851050633 74.60774154342762) bank115492 +115493 POINT(40.2404449730699 74.78094284096635) bank115493 +115494 POINT(40.56595437692448 73.17704421071093) bank115494 +115495 POINT(40.397065060563804 73.78702907678496) bank115495 +115496 POINT(41.67518675494916 73.09121714075366) bank115496 +115497 POINT(41.05562460768734 74.94907821527407) bank115497 +115498 POINT(41.144897515796416 74.38773751576) bank115498 +115499 POINT(41.61948442972864 74.63370387492982) bank115499 +115500 POINT(41.44472982900656 73.51492974206298) bank115500 +115501 POINT(41.05595935108173 74.059659037934) bank115501 +115502 POINT(41.070530670043965 73.06360122223941) bank115502 +115503 POINT(41.34599443566215 73.85040681784427) bank115503 +115504 POINT(41.15931834573886 74.4149593144436) bank115504 +115505 POINT(40.040924407090664 74.66508967190826) bank115505 +115506 POINT(40.46295197173217 74.31433993084741) bank115506 +115507 POINT(41.1853364251566 74.55965912113291) bank115507 +115508 POINT(40.23766151306213 74.57501968368642) bank115508 +115509 POINT(41.08786495580383 74.79134825961867) bank115509 +115510 POINT(40.410118823061616 74.31452910973536) bank115510 +115511 POINT(40.33047018733181 73.51937469374583) bank115511 +115512 POINT(41.074751201848976 73.70189635818723) bank115512 +115513 POINT(41.514316980258876 74.30471435203727) bank115513 +115514 POINT(39.87996040923534 74.07948854868656) bank115514 +115515 POINT(40.68776844048103 74.12067995367994) bank115515 +115516 POINT(40.69037517150779 73.87924325488513) bank115516 +115517 POINT(41.04591478033527 73.98479033352766) bank115517 +115518 POINT(40.339593233133506 73.05298773395504) bank115518 +115519 POINT(41.4634841997583 74.32334639059252) bank115519 +115520 POINT(40.786948937007644 73.02004288909026) bank115520 +115521 POINT(41.20290227052062 73.17969215817104) bank115521 +115522 POINT(40.56114284647525 74.09634345097913) bank115522 +115523 POINT(40.10446385422879 74.11104507874805) bank115523 +115524 POINT(40.04985140578664 74.00485769229773) bank115524 +115525 POINT(40.978459591116234 74.72631888975903) bank115525 +115526 POINT(41.31176623808594 74.54667472992519) bank115526 +115527 POINT(40.355732714220665 74.34573430567065) bank115527 +115528 POINT(40.604876269085814 74.8692802329116) bank115528 +115529 POINT(39.98673253713131 74.97007960720212) bank115529 +115530 POINT(40.1810074240576 73.8235553834067) bank115530 +115531 POINT(40.920585195741054 73.63080072681021) bank115531 +115532 POINT(40.97085650191409 74.66975921184617) bank115532 +115533 POINT(40.09757984204869 74.44096473121715) bank115533 +115534 POINT(40.10891913830365 73.17314737584884) bank115534 +115535 POINT(39.74200737461467 74.13665549821759) bank115535 +115536 POINT(41.0203569948405 74.96469023567161) bank115536 +115537 POINT(40.35386581342721 73.02020419228359) bank115537 +115538 POINT(40.99344288797502 74.60522353560158) bank115538 +115539 POINT(40.90910104269619 74.47798799582296) bank115539 +115540 POINT(39.79519871437452 73.43746555493762) bank115540 +115541 POINT(40.314577974169566 74.17266330149683) bank115541 +115542 POINT(41.12834200364905 73.1203136467267) bank115542 +115543 POINT(40.05514262639865 73.25962382896518) bank115543 +115544 POINT(40.85628819311754 73.41518156859588) bank115544 +115545 POINT(40.786985491102904 73.03107805122939) bank115545 +115546 POINT(41.293566722020884 74.00134211552015) bank115546 +115547 POINT(41.231816734895254 73.16858171041274) bank115547 +115548 POINT(40.74578919862955 73.85302162765815) bank115548 +115549 POINT(40.00537908814572 74.50297182278126) bank115549 +115550 POINT(41.092291174498285 73.43000361982749) bank115550 +115551 POINT(40.37757735120672 74.87307326373585) bank115551 +115552 POINT(41.18365602776744 74.68872111932279) bank115552 +115553 POINT(41.06218739998277 73.61998940047738) bank115553 +115554 POINT(41.58433421839431 74.18196464515815) bank115554 +115555 POINT(41.207966315439776 74.60791446144574) bank115555 +115556 POINT(41.452792788065864 73.8966088261649) bank115556 +115557 POINT(40.356621581199924 74.80595359891086) bank115557 +115558 POINT(40.607104237818376 74.07375380288588) bank115558 +115559 POINT(41.03380207732431 73.08281099690483) bank115559 +115560 POINT(40.204520142009 74.50549181804874) bank115560 +115561 POINT(40.329035873442535 73.00706663936376) bank115561 +115562 POINT(39.73491648709036 73.67609382072912) bank115562 +115563 POINT(40.100396794799785 73.66248823630835) bank115563 +115564 POINT(40.69643221655354 73.48076280173366) bank115564 +115565 POINT(40.97391042993587 74.74632915886842) bank115565 +115566 POINT(39.96148186267104 73.71735458890551) bank115566 +115567 POINT(40.88162500137101 73.92381024537823) bank115567 +115568 POINT(41.65387678261911 73.31550622997139) bank115568 +115569 POINT(40.37035141011756 73.97247647013211) bank115569 +115570 POINT(41.19039060445415 73.08344655337231) bank115570 +115571 POINT(40.1219140705672 74.87938399099963) bank115571 +115572 POINT(41.70222664987991 74.33848534059145) bank115572 +115573 POINT(41.663659957489095 74.9182681045095) bank115573 +115574 POINT(40.07927473632063 73.2860291388919) bank115574 +115575 POINT(41.39194260310896 74.7605668405146) bank115575 +115576 POINT(40.59971270866116 74.01970622740605) bank115576 +115577 POINT(41.27485542333184 74.9906550385359) bank115577 +115578 POINT(41.2675962141149 73.46234137018229) bank115578 +115579 POINT(39.76443219709775 73.82707393585301) bank115579 +115580 POINT(40.83116097000267 74.9410434612719) bank115580 +115581 POINT(40.31498235808297 73.28795243019619) bank115581 +115582 POINT(40.955704695463375 74.68814409641485) bank115582 +115583 POINT(39.829955301822466 73.97528673781179) bank115583 +115584 POINT(40.323809229688116 73.55952434613195) bank115584 +115585 POINT(41.383869284691905 73.05955126077863) bank115585 +115586 POINT(40.151854183002754 74.52411701773835) bank115586 +115587 POINT(40.98436668117007 73.3678492416538) bank115587 +115588 POINT(40.85901691466474 73.3998306934069) bank115588 +115589 POINT(40.684168134531106 74.15126172355411) bank115589 +115590 POINT(40.875118762739355 73.42712490125588) bank115590 +115591 POINT(40.225499679121874 74.74784629159183) bank115591 +115592 POINT(40.79362410681115 73.17485471462686) bank115592 +115593 POINT(41.683414900042806 74.55234032612475) bank115593 +115594 POINT(41.16059542274158 73.60456866976288) bank115594 +115595 POINT(41.055641781531826 74.9516383435499) bank115595 +115596 POINT(39.89483102051261 74.70417244430742) bank115596 +115597 POINT(40.21128009108731 74.18130446327766) bank115597 +115598 POINT(41.60586704825166 73.87195529437766) bank115598 +115599 POINT(39.73175179207138 74.2058220577302) bank115599 +115600 POINT(40.2446150554253 73.62439698767251) bank115600 +115601 POINT(41.33450923187577 74.54287225915148) bank115601 +115602 POINT(40.05314615231475 73.68041494628113) bank115602 +115603 POINT(41.67697618730816 74.44208092854696) bank115603 +115604 POINT(41.25724498242375 73.0135385656112) bank115604 +115605 POINT(41.61004376119232 74.09062141381652) bank115605 +115606 POINT(39.9289964512796 74.65126519931651) bank115606 +115607 POINT(41.45759431154009 73.46052480888513) bank115607 +115608 POINT(40.53735722622109 73.69990715174725) bank115608 +115609 POINT(40.61539337503259 74.1435050357298) bank115609 +115610 POINT(40.468775280102484 73.19327642786011) bank115610 +115611 POINT(41.16987018216481 73.13052647442835) bank115611 +115612 POINT(41.33407344637187 74.03145662265028) bank115612 +115613 POINT(40.70110218992094 73.30106185262493) bank115613 +115614 POINT(39.83395850421703 74.16168765757251) bank115614 +115615 POINT(40.01514981981998 73.96051330639895) bank115615 +115616 POINT(41.25779543704507 74.55755326256198) bank115616 +115617 POINT(41.42040691438172 73.52187668028213) bank115617 +115618 POINT(40.87091491135542 74.8207130126914) bank115618 +115619 POINT(40.82407765412585 73.48518830857353) bank115619 +115620 POINT(40.06798386212023 73.6764581856067) bank115620 +115621 POINT(39.97772615402735 73.43037155756038) bank115621 +115622 POINT(39.930121486798235 74.9017552940362) bank115622 +115623 POINT(39.8170344162585 74.39459159895397) bank115623 +115624 POINT(40.29529059223876 74.0087688168685) bank115624 +115625 POINT(40.762282228967706 73.17784604334783) bank115625 +115626 POINT(39.76674063367875 73.65879893656033) bank115626 +115627 POINT(41.305486824876894 73.52302929881142) bank115627 +115628 POINT(41.46189548328862 73.88451191731062) bank115628 +115629 POINT(40.65607116849229 73.61376012993814) bank115629 +115630 POINT(40.34210322210554 73.07298769902823) bank115630 +115631 POINT(39.72241182860074 73.74820676034072) bank115631 +115632 POINT(39.84721257852718 73.7304257195469) bank115632 +115633 POINT(40.52409777773477 73.21144601349961) bank115633 +115634 POINT(40.69618711285767 73.68030296647711) bank115634 +115635 POINT(41.4545807352409 74.78238470237102) bank115635 +115636 POINT(41.57660508348852 74.76567031463729) bank115636 +115637 POINT(39.76162870569971 73.2821017597127) bank115637 +115638 POINT(40.966177603443434 73.53975398303226) bank115638 +115639 POINT(40.33494603780856 74.0911440742212) bank115639 +115640 POINT(40.14765629880303 73.49711402454241) bank115640 +115641 POINT(41.199795851061474 74.02007782056754) bank115641 +115642 POINT(39.72753036326638 74.11227060948326) bank115642 +115643 POINT(40.01122647043577 73.11399088333428) bank115643 +115644 POINT(39.92657779483663 74.18132778934213) bank115644 +115645 POINT(41.29705644261992 74.6238911051914) bank115645 +115646 POINT(40.27151575759717 73.16802007238492) bank115646 +115647 POINT(39.83832589666014 74.84527104840396) bank115647 +115648 POINT(40.573778615237394 74.7600499792213) bank115648 +115649 POINT(40.29034297460202 74.66973014982814) bank115649 +115650 POINT(40.68381670831701 73.4178812549181) bank115650 +115651 POINT(40.96983337028211 73.89513617257025) bank115651 +115652 POINT(39.81663428630941 74.87409213348198) bank115652 +115653 POINT(41.62670108180196 73.85324130058267) bank115653 +115654 POINT(41.48001712899846 74.95649420913364) bank115654 +115655 POINT(40.12055874152261 74.89433433199389) bank115655 +115656 POINT(40.06414758366521 74.31278561394608) bank115656 +115657 POINT(40.52051196431969 73.58696772623784) bank115657 +115658 POINT(40.709509617837256 73.66979664828726) bank115658 +115659 POINT(40.341967690642115 74.05362487561828) bank115659 +115660 POINT(41.24491422359454 73.55312609135548) bank115660 +115661 POINT(41.48700456398954 74.636625843669) bank115661 +115662 POINT(40.46357403957275 73.4634174449226) bank115662 +115663 POINT(40.91073030091573 74.59022178806002) bank115663 +115664 POINT(40.824140521790746 73.65596876069569) bank115664 +115665 POINT(41.43662416167997 73.2710405841004) bank115665 +115666 POINT(40.379564125305464 73.43274107935821) bank115666 +115667 POINT(40.752788405084885 73.27480469825959) bank115667 +115668 POINT(41.49092831290688 74.60365624629156) bank115668 +115669 POINT(41.28822552371658 74.99442424388732) bank115669 +115670 POINT(40.341712413566825 74.33809749548846) bank115670 +115671 POINT(40.09415784313672 73.14761184616471) bank115671 +115672 POINT(41.02529947337634 74.05897283633335) bank115672 +115673 POINT(40.07968978510037 74.08564598472276) bank115673 +115674 POINT(40.52694243285708 74.49546546216955) bank115674 +115675 POINT(39.817416647088784 74.07170366187601) bank115675 +115676 POINT(41.610032485495324 74.87082765154793) bank115676 +115677 POINT(40.36024775411041 73.88500647006562) bank115677 +115678 POINT(41.22431786456891 74.70019718168618) bank115678 +115679 POINT(40.86908124738465 74.07242628823855) bank115679 +115680 POINT(40.930748263028626 74.93007501431862) bank115680 +115681 POINT(39.78594055418315 73.48318663823655) bank115681 +115682 POINT(40.77691672684042 74.62344189967621) bank115682 +115683 POINT(40.54177924830846 73.57017670629438) bank115683 +115684 POINT(41.51842489584855 73.50911989768747) bank115684 +115685 POINT(40.761014142658425 73.51798715036158) bank115685 +115686 POINT(40.781602512907575 74.84300769912079) bank115686 +115687 POINT(40.21057724357066 73.43742298473138) bank115687 +115688 POINT(39.74900457584329 73.7178333042322) bank115688 +115689 POINT(40.42854918897384 74.3581425549655) bank115689 +115690 POINT(39.98583352482283 73.6658391872477) bank115690 +115691 POINT(40.03795883813322 73.19460821729899) bank115691 +115692 POINT(40.64208273649515 73.58469103241175) bank115692 +115693 POINT(40.90340457566869 73.10903101403532) bank115693 +115694 POINT(41.61822008997137 73.35093176680857) bank115694 +115695 POINT(41.02934457241401 73.5612108912056) bank115695 +115696 POINT(41.22583618653741 74.15572320129871) bank115696 +115697 POINT(40.80440576508896 73.5798937292743) bank115697 +115698 POINT(40.21383774802441 73.1469008054667) bank115698 +115699 POINT(40.82860215998377 73.18322458081583) bank115699 +115700 POINT(39.80306872182323 73.12088291540357) bank115700 +115701 POINT(41.256678489892565 74.42614560733483) bank115701 +115702 POINT(40.689522755842475 73.85021974205934) bank115702 +115703 POINT(41.23177196528583 74.88621434560764) bank115703 +115704 POINT(41.171036285549015 74.14456619795027) bank115704 +115705 POINT(40.185020426013836 73.41012774673375) bank115705 +115706 POINT(40.55512709954251 73.62615471680962) bank115706 +115707 POINT(39.916174919413486 74.41835154734142) bank115707 +115708 POINT(40.67495522813116 74.16859165088238) bank115708 +115709 POINT(40.13885216787874 74.61445999491973) bank115709 +115710 POINT(40.259034925661375 74.67419742127645) bank115710 +115711 POINT(40.55508830093996 74.93099965008838) bank115711 +115712 POINT(41.132901955868256 73.1083661110324) bank115712 +115713 POINT(39.972384638983705 73.57054077855886) bank115713 +115714 POINT(41.6379073271344 74.4157848609885) bank115714 +115715 POINT(40.14136271598424 74.98856043469902) bank115715 +115716 POINT(40.42137633932651 73.84043157089275) bank115716 +115717 POINT(41.31822326544505 74.15431117182814) bank115717 +115718 POINT(40.33897490428243 74.72536482821904) bank115718 +115719 POINT(41.19899959246818 73.05682636647198) bank115719 +115720 POINT(40.73996441852191 74.7163548135635) bank115720 +115721 POINT(41.62520345628211 73.49538125399415) bank115721 +115722 POINT(40.14273441931748 74.92508653667655) bank115722 +115723 POINT(40.09050591776678 74.10758993786288) bank115723 +115724 POINT(40.329067809506455 74.82270949645421) bank115724 +115725 POINT(40.99091753112426 74.34104886987882) bank115725 +115726 POINT(39.85617689331666 74.34239369760479) bank115726 +115727 POINT(40.15398561607129 74.1804200292333) bank115727 +115728 POINT(41.0684733333616 73.11402604045638) bank115728 +115729 POINT(40.50809569651876 73.85009589004193) bank115729 +115730 POINT(41.35340354410282 74.45520328676903) bank115730 +115731 POINT(41.21664322395946 73.72469665302634) bank115731 +115732 POINT(40.19598864650164 73.889843351386) bank115732 +115733 POINT(40.94676148866273 73.93282353780553) bank115733 +115734 POINT(39.82359489174474 73.13440318099964) bank115734 +115735 POINT(40.20100269748074 74.94788414872086) bank115735 +115736 POINT(40.793803168208264 73.90674819984456) bank115736 +115737 POINT(41.63496887996508 74.45825032049913) bank115737 +115738 POINT(40.951565042380324 75.00394051107496) bank115738 +115739 POINT(41.48509384120342 73.45432041258462) bank115739 +115740 POINT(41.45019953801656 74.7436029583837) bank115740 +115741 POINT(40.067051531994835 73.69218507571496) bank115741 +115742 POINT(40.04064156559726 74.20240532750829) bank115742 +115743 POINT(41.11636538364305 73.14556874001117) bank115743 +115744 POINT(40.94566985576733 73.02506359997723) bank115744 +115745 POINT(40.49652350532681 73.70235212347309) bank115745 +115746 POINT(40.5526680333786 73.59701655198928) bank115746 +115747 POINT(40.29919977816686 73.58528258107684) bank115747 +115748 POINT(40.78108446282828 74.35198664021263) bank115748 +115749 POINT(40.51067639674756 74.63858423970908) bank115749 +115750 POINT(40.3939663013311 74.30490299993909) bank115750 +115751 POINT(41.620825472835904 74.08954148294018) bank115751 +115752 POINT(40.498663124847326 73.67619590321603) bank115752 +115753 POINT(40.69174009793308 74.87193227197315) bank115753 +115754 POINT(41.41964822176638 73.54974459193292) bank115754 +115755 POINT(40.63858592624814 74.3147662985471) bank115755 +115756 POINT(39.83346177296658 73.24187494642969) bank115756 +115757 POINT(40.37110588987249 74.8062799510705) bank115757 +115758 POINT(40.148888977143166 74.63701539323219) bank115758 +115759 POINT(40.931909232996716 73.73861630316036) bank115759 +115760 POINT(41.628154276252715 74.57621833326715) bank115760 +115761 POINT(41.35906703356807 73.48059134350905) bank115761 +115762 POINT(40.02628754280375 74.33501915459163) bank115762 +115763 POINT(40.39452539934469 73.65361270258781) bank115763 +115764 POINT(41.51036533282109 74.18083606688889) bank115764 +115765 POINT(40.231230295813084 74.26381597112812) bank115765 +115766 POINT(41.474894981297226 75.0006656935312) bank115766 +115767 POINT(39.80389452558266 73.69135771154279) bank115767 +115768 POINT(41.274496220785984 74.34706257146921) bank115768 +115769 POINT(41.245165543728824 73.27577554725089) bank115769 +115770 POINT(41.2997199941981 73.54927248328285) bank115770 +115771 POINT(40.56784503882592 74.50606741123316) bank115771 +115772 POINT(40.95477155692071 74.65206519201826) bank115772 +115773 POINT(41.17996902323461 74.81041842738595) bank115773 +115774 POINT(40.190950382635386 74.10941655700134) bank115774 +115775 POINT(41.22763780721956 74.63942356863559) bank115775 +115776 POINT(41.296893254726655 73.7826230847908) bank115776 +115777 POINT(40.6284221433444 73.24569650479378) bank115777 +115778 POINT(41.3768472506021 73.33442247949931) bank115778 +115779 POINT(41.150588841346675 74.75778442375395) bank115779 +115780 POINT(41.3095005907375 73.61751129158348) bank115780 +115781 POINT(41.04639915349689 73.12136925841119) bank115781 +115782 POINT(40.29331095605393 73.97305471116545) bank115782 +115783 POINT(40.55447209791434 74.09654965014815) bank115783 +115784 POINT(40.013690773280565 74.27453995507304) bank115784 +115785 POINT(39.7185278018868 74.44972077376548) bank115785 +115786 POINT(40.4753856682706 73.35783913242949) bank115786 +115787 POINT(40.0786128667807 73.26593864005021) bank115787 +115788 POINT(41.57468554550116 74.6638932698857) bank115788 +115789 POINT(40.94626427494431 74.0242539231283) bank115789 +115790 POINT(40.03347568326939 73.82505291232192) bank115790 +115791 POINT(41.48185844318225 74.43609084428685) bank115791 +115792 POINT(39.71842397960792 74.77561549085763) bank115792 +115793 POINT(40.64828712016643 74.19829678652674) bank115793 +115794 POINT(40.48061359377034 74.6137994328202) bank115794 +115795 POINT(39.831273045380264 74.05378588105955) bank115795 +115796 POINT(40.12139143155677 73.03047475333453) bank115796 +115797 POINT(40.82021302431416 73.13253996613562) bank115797 +115798 POINT(41.242928569706784 73.97993655836748) bank115798 +115799 POINT(40.75760173758756 73.48645520932497) bank115799 +115800 POINT(40.654989439815985 73.37591093156323) bank115800 +115801 POINT(39.82254099254514 74.55091634974448) bank115801 +115802 POINT(40.60575082784451 74.055418872481) bank115802 +115803 POINT(41.37906064390637 73.06662400641343) bank115803 +115804 POINT(40.92688247528394 74.81014591877106) bank115804 +115805 POINT(40.35543925296762 73.81762123287405) bank115805 +115806 POINT(41.205872697679126 74.7259182505723) bank115806 +115807 POINT(40.967540404592036 74.87826256305398) bank115807 +115808 POINT(41.32621880384428 73.83185319752586) bank115808 +115809 POINT(40.08922014903396 74.79497279638119) bank115809 +115810 POINT(40.04586828630082 73.95663204669937) bank115810 +115811 POINT(40.418563342637555 73.27362176050525) bank115811 +115812 POINT(40.0319816033342 73.37423640081087) bank115812 +115813 POINT(40.263963951302145 74.11026369046972) bank115813 +115814 POINT(39.731749518687074 74.73197204868836) bank115814 +115815 POINT(39.8844110057605 74.14867956014301) bank115815 +115816 POINT(40.21278699073124 73.36390212883074) bank115816 +115817 POINT(41.31038129202327 74.75604938010011) bank115817 +115818 POINT(41.64578194505059 74.46439982955839) bank115818 +115819 POINT(40.302945515166115 73.36101721682586) bank115819 +115820 POINT(40.19361484895547 74.99316858297834) bank115820 +115821 POINT(40.711893648284075 73.62320518304355) bank115821 +115822 POINT(40.19080465520634 74.97650102131395) bank115822 +115823 POINT(39.71755078447392 74.37417596906403) bank115823 +115824 POINT(40.66377365133193 74.24080630692326) bank115824 +115825 POINT(41.439543017426445 74.58000836298122) bank115825 +115826 POINT(41.541615798145706 73.9860185706768) bank115826 +115827 POINT(41.521350096766845 74.09689432340399) bank115827 +115828 POINT(41.00872135607765 74.03764615549677) bank115828 +115829 POINT(39.97399854707909 74.37956263268626) bank115829 +115830 POINT(41.45836424715237 73.77312497308235) bank115830 +115831 POINT(40.75095238164505 74.10228843159067) bank115831 +115832 POINT(40.24431134126647 74.92862649991244) bank115832 +115833 POINT(40.909950973172236 74.23991146612917) bank115833 +115834 POINT(40.95038391730398 73.23874542705313) bank115834 +115835 POINT(40.47226124151229 74.96610698110861) bank115835 +115836 POINT(40.97114131880919 73.88906079717148) bank115836 +115837 POINT(40.46946185709379 74.4732348756292) bank115837 +115838 POINT(40.25477519963686 73.1424746755836) bank115838 +115839 POINT(41.17064619634939 74.60830080347816) bank115839 +115840 POINT(41.242104526427205 74.85279906734854) bank115840 +115841 POINT(39.996152350008124 73.77009984816095) bank115841 +115842 POINT(39.88174255631995 74.96262438439591) bank115842 +115843 POINT(40.90138925503626 73.54028723458956) bank115843 +115844 POINT(41.463088346289396 74.16638962049328) bank115844 +115845 POINT(40.734109784002534 73.41681022833828) bank115845 +115846 POINT(40.342776740967665 73.39875723883216) bank115846 +115847 POINT(40.66066774323929 74.04122348579882) bank115847 +115848 POINT(40.76802656412124 73.77109621325498) bank115848 +115849 POINT(41.55954512700478 73.81043212970651) bank115849 +115850 POINT(41.608707047605925 74.6207674387571) bank115850 +115851 POINT(40.58649302978971 73.93836322256762) bank115851 +115852 POINT(40.66030714972378 73.47874556075752) bank115852 +115853 POINT(41.53422409515899 74.35524412893542) bank115853 +115854 POINT(40.226389617324976 73.53884164236257) bank115854 +115855 POINT(39.887981722527094 73.54021392835226) bank115855 +115856 POINT(41.55356283717324 73.49116339028438) bank115856 +115857 POINT(40.286294978964136 74.27804488182952) bank115857 +115858 POINT(41.49741550511971 74.15723405460096) bank115858 +115859 POINT(40.630673378943904 74.78011007321273) bank115859 +115860 POINT(40.81325027027183 74.68515670541304) bank115860 +115861 POINT(41.403969598754884 74.25244709772473) bank115861 +115862 POINT(41.458669789043576 73.58117588691526) bank115862 +115863 POINT(39.85189660360552 74.5713839484012) bank115863 +115864 POINT(41.009131890253066 74.84154078400913) bank115864 +115865 POINT(40.53762439705156 73.75704359576123) bank115865 +115866 POINT(40.43441686497249 74.42989578530536) bank115866 +115867 POINT(39.739618206703994 73.43114538033925) bank115867 +115868 POINT(39.7385994683249 74.29821233817056) bank115868 +115869 POINT(40.35123184485418 74.78498527481493) bank115869 +115870 POINT(41.644601891460745 74.46629731472802) bank115870 +115871 POINT(39.744305016348 74.82619159588693) bank115871 +115872 POINT(40.37277884427982 73.62914139888296) bank115872 +115873 POINT(41.3814050863308 74.54825076023565) bank115873 +115874 POINT(41.47191203196544 74.5396169901004) bank115874 +115875 POINT(41.103629494470574 73.76488724639509) bank115875 +115876 POINT(40.433928611451286 74.06137852522093) bank115876 +115877 POINT(40.12228220169245 74.24748289554131) bank115877 +115878 POINT(40.823972034464845 74.35367518415698) bank115878 +115879 POINT(40.60599669782952 73.84392790310518) bank115879 +115880 POINT(40.571623045243705 73.09302318438073) bank115880 +115881 POINT(39.769796281172596 73.10774261348793) bank115881 +115882 POINT(41.65020878326123 73.25333243759385) bank115882 +115883 POINT(40.97112909825711 74.83202875842797) bank115883 +115884 POINT(40.059711063691765 74.01158500906935) bank115884 +115885 POINT(41.609930620207834 73.60697323273966) bank115885 +115886 POINT(41.128080089270185 74.47208625967309) bank115886 +115887 POINT(40.63707740849348 74.99631540504235) bank115887 +115888 POINT(40.09180417301754 73.75177703707853) bank115888 +115889 POINT(40.0194909071055 74.37915362028512) bank115889 +115890 POINT(39.97228908941624 74.67760085368948) bank115890 +115891 POINT(40.4058244563274 73.40531346521246) bank115891 +115892 POINT(40.458736133807314 74.00449002489977) bank115892 +115893 POINT(41.25806500070854 73.74075414580453) bank115893 +115894 POINT(40.41218181424436 74.22531550948467) bank115894 +115895 POINT(40.28790525117125 74.0033942683521) bank115895 +115896 POINT(40.59721642550919 73.08203603684134) bank115896 +115897 POINT(40.347615730063644 73.41379108450026) bank115897 +115898 POINT(40.529581173195375 73.15935422954118) bank115898 +115899 POINT(41.09492783840452 73.21047292439646) bank115899 +115900 POINT(40.79677567393592 73.07858194401146) bank115900 +115901 POINT(40.10471036543243 74.58080252966512) bank115901 +115902 POINT(41.35466367518447 74.17353383932019) bank115902 +115903 POINT(41.42897254714354 73.09200502130027) bank115903 +115904 POINT(40.58544959592293 74.1480687568462) bank115904 +115905 POINT(40.5137282133656 74.74797196919839) bank115905 +115906 POINT(40.44781051619698 74.45122852631441) bank115906 +115907 POINT(40.26928784574758 73.84560054807457) bank115907 +115908 POINT(41.63737574329077 74.51284778097904) bank115908 +115909 POINT(40.79614696296836 73.51367498649992) bank115909 +115910 POINT(41.56931228988563 74.94832018840569) bank115910 +115911 POINT(40.01901590098071 73.67161164881662) bank115911 +115912 POINT(41.22052774224293 74.43815333837317) bank115912 +115913 POINT(41.01570826005792 74.90537919406628) bank115913 +115914 POINT(39.74725864530451 74.08794974052246) bank115914 +115915 POINT(41.36452609264965 73.48512308584921) bank115915 +115916 POINT(39.98315978461648 73.84524464501614) bank115916 +115917 POINT(39.998189876342444 74.66904214096387) bank115917 +115918 POINT(41.70470573285609 73.59479963441703) bank115918 +115919 POINT(40.33030389797077 74.71244545206933) bank115919 +115920 POINT(40.32250477102528 73.99428054516304) bank115920 +115921 POINT(41.57835902831321 73.79631571445375) bank115921 +115922 POINT(40.14955808773129 73.34324485601427) bank115922 +115923 POINT(40.40134968867394 73.83839195671123) bank115923 +115924 POINT(40.7465887603055 73.79984782220856) bank115924 +115925 POINT(40.64042067453322 74.10648889131153) bank115925 +115926 POINT(41.4152415166777 74.02035764199535) bank115926 +115927 POINT(41.0200681955338 74.37663844250422) bank115927 +115928 POINT(39.90538189234728 73.19873803053252) bank115928 +115929 POINT(40.20856393528989 74.82126775441432) bank115929 +115930 POINT(41.595793851812346 73.94602483112438) bank115930 +115931 POINT(40.43463064097961 73.39920122006626) bank115931 +115932 POINT(41.18399083128995 73.7785733941524) bank115932 +115933 POINT(39.88245064948888 73.99822922838196) bank115933 +115934 POINT(40.72705388414201 73.13270040337882) bank115934 +115935 POINT(40.262818907957 74.44970095480076) bank115935 +115936 POINT(41.33176620721308 73.03126081222331) bank115936 +115937 POINT(41.34752705849279 73.96380054928774) bank115937 +115938 POINT(40.41528343042907 74.90611433399776) bank115938 +115939 POINT(40.09276319248561 73.81100960860434) bank115939 +115940 POINT(41.35219713268126 73.08563276033608) bank115940 +115941 POINT(40.22120820210798 74.50373682523254) bank115941 +115942 POINT(40.27590487371827 74.1276931163039) bank115942 +115943 POINT(39.889959453623284 74.03109305287039) bank115943 +115944 POINT(41.287201371172394 74.29298104564495) bank115944 +115945 POINT(41.34303249085387 74.86837655145106) bank115945 +115946 POINT(39.815680127275144 74.0093197008113) bank115946 +115947 POINT(40.18464173987021 74.17597392651086) bank115947 +115948 POINT(40.35864017400581 74.98978696003796) bank115948 +115949 POINT(40.926531716903355 73.71157454115834) bank115949 +115950 POINT(41.526026498750404 74.671829738591) bank115950 +115951 POINT(41.536447276518956 73.17444031997864) bank115951 +115952 POINT(39.98768583918487 74.53013182511684) bank115952 +115953 POINT(39.957759244285064 73.72050293075095) bank115953 +115954 POINT(40.78510193998979 74.64660176624189) bank115954 +115955 POINT(40.813219662467105 73.36964525966562) bank115955 +115956 POINT(40.32213773460293 73.87754301441929) bank115956 +115957 POINT(41.13960021112082 73.80985294857248) bank115957 +115958 POINT(40.92943328981558 74.69572098596677) bank115958 +115959 POINT(41.282917725222916 74.49949216111966) bank115959 +115960 POINT(40.56345923054356 74.74510329294604) bank115960 +115961 POINT(40.70494205579407 73.11141143679158) bank115961 +115962 POINT(40.70308216227424 73.48039148673512) bank115962 +115963 POINT(40.293685969349234 74.20485471651416) bank115963 +115964 POINT(40.894082352212315 74.09215803409417) bank115964 +115965 POINT(40.485739588752665 74.85447309793592) bank115965 +115966 POINT(41.61718751990521 73.07213537822798) bank115966 +115967 POINT(40.80701232787071 73.49414023158839) bank115967 +115968 POINT(40.10770009150028 73.58556977374425) bank115968 +115969 POINT(41.15483724857798 73.0438721953878) bank115969 +115970 POINT(41.415448560776866 74.44115815526425) bank115970 +115971 POINT(40.78448530334113 73.75364061691313) bank115971 +115972 POINT(41.21117412029689 73.62800449590637) bank115972 +115973 POINT(40.56735147271858 73.73665170085566) bank115973 +115974 POINT(39.81073465493456 73.81784363805383) bank115974 +115975 POINT(41.352299923251174 73.91565506396856) bank115975 +115976 POINT(41.163887251455954 74.65060366469218) bank115976 +115977 POINT(40.29623792951173 73.18885828860574) bank115977 +115978 POINT(40.62480962263154 73.09952186603617) bank115978 +115979 POINT(40.66112236456433 73.78887615304131) bank115979 +115980 POINT(41.10442535315715 73.96749102785972) bank115980 +115981 POINT(41.64523132960401 74.74776810603488) bank115981 +115982 POINT(41.12588563505674 74.68315740143939) bank115982 +115983 POINT(40.08240463849022 74.91406250462299) bank115983 +115984 POINT(40.55567515645621 74.21036850275561) bank115984 +115985 POINT(39.74214566508738 74.99818402425029) bank115985 +115986 POINT(39.75323397916299 73.8382498848559) bank115986 +115987 POINT(41.1764546714664 74.28310474163317) bank115987 +115988 POINT(40.423755687204114 74.92711992109885) bank115988 +115989 POINT(40.257238660177364 74.42005591451351) bank115989 +115990 POINT(41.301865414624075 74.46088986112395) bank115990 +115991 POINT(40.19563782098007 74.75084626620726) bank115991 +115992 POINT(41.347064347794685 74.99148289676204) bank115992 +115993 POINT(41.12401254933789 73.5682865222013) bank115993 +115994 POINT(41.69848677483609 73.26354842314691) bank115994 +115995 POINT(40.15056044147308 74.88300572634085) bank115995 +115996 POINT(41.430579727234814 73.16588753610267) bank115996 +115997 POINT(41.69795163554759 74.68839879751421) bank115997 +115998 POINT(39.76975184095762 74.68766219314622) bank115998 +115999 POINT(40.35096258518588 74.35936089234836) bank115999 +116000 POINT(41.09637391726206 74.50206788032578) bank116000 +116001 POINT(39.96913491238193 73.4228110567304) bank116001 +116002 POINT(41.25583754551152 73.67787085036608) bank116002 +116003 POINT(39.897128155818706 73.8069281236649) bank116003 +116004 POINT(41.270193404874334 74.85922893339419) bank116004 +116005 POINT(41.545999040800595 73.48189088401274) bank116005 +116006 POINT(40.24939807768622 74.17783190655554) bank116006 +116007 POINT(40.48595926152508 74.15503058969834) bank116007 +116008 POINT(41.46811312700987 74.07797604522015) bank116008 +116009 POINT(41.475034367476255 74.58431411579788) bank116009 +116010 POINT(40.523578735004556 74.32239629855007) bank116010 +116011 POINT(40.964540722265085 73.67258539062016) bank116011 +116012 POINT(41.401090119244536 73.47523860838838) bank116012 +116013 POINT(40.53379127221619 73.50426051568006) bank116013 +116014 POINT(40.016795329675446 73.11404377420776) bank116014 +116015 POINT(41.20155651484111 73.11461869203458) bank116015 +116016 POINT(41.1313200268745 73.46267628125887) bank116016 +116017 POINT(40.89274240852512 73.76056983839355) bank116017 +116018 POINT(40.910058784434796 73.11492072429) bank116018 +116019 POINT(39.964408807049765 73.58120905696654) bank116019 +116020 POINT(40.16093503845496 73.79525360092724) bank116020 +116021 POINT(40.074870914768994 73.20151238427184) bank116021 +116022 POINT(40.075875794736 74.55492135769343) bank116022 +116023 POINT(41.44589593285913 73.13996155709914) bank116023 +116024 POINT(41.52278771584571 74.84092176357436) bank116024 +116025 POINT(41.62472895503643 73.83956299728362) bank116025 +116026 POINT(40.51360647544876 73.24767653100125) bank116026 +116027 POINT(40.576959209870466 74.18513737373306) bank116027 +116028 POINT(40.9540693583529 73.38119101284968) bank116028 +116029 POINT(39.75489463100146 74.38667012974713) bank116029 +116030 POINT(40.53055087104623 73.02435923911585) bank116030 +116031 POINT(40.406494071802264 74.97243881203319) bank116031 +116032 POINT(40.65497418309054 73.4803862598991) bank116032 +116033 POINT(41.42425175419976 74.6380534790898) bank116033 +116034 POINT(40.74458172124511 73.1408254941851) bank116034 +116035 POINT(41.015001480812046 73.78848108276541) bank116035 +116036 POINT(41.06681179889665 73.31474608080745) bank116036 +116037 POINT(41.51559032237301 74.88453755363177) bank116037 +116038 POINT(40.005882877485924 73.18304643745945) bank116038 +116039 POINT(40.59581696070429 73.99023793064654) bank116039 +116040 POINT(41.163087752750016 73.38082367877927) bank116040 +116041 POINT(39.73383052111064 74.43627224248266) bank116041 +116042 POINT(40.44086504584645 73.04342440344381) bank116042 +116043 POINT(40.281985950227025 74.8653960774597) bank116043 +116044 POINT(40.717398863249656 74.76855905897702) bank116044 +116045 POINT(41.3142878310607 73.76641051447221) bank116045 +116046 POINT(40.914902659608764 73.30689926446045) bank116046 +116047 POINT(40.28766620469005 73.23941171576543) bank116047 +116048 POINT(40.2207150551868 74.98266474034749) bank116048 +116049 POINT(39.93906637570743 74.87384849331127) bank116049 +116050 POINT(41.201859364216084 74.5943889792089) bank116050 +116051 POINT(41.61188312511601 73.19386491988402) bank116051 +116052 POINT(40.91890060291384 73.63008977134287) bank116052 +116053 POINT(41.00973005260701 74.89569098075197) bank116053 +116054 POINT(40.06505682583784 73.93223381063115) bank116054 +116055 POINT(40.59102434857096 73.09867338522342) bank116055 +116056 POINT(40.03799458157536 73.21533452674448) bank116056 +116057 POINT(41.18884974452339 73.44447341761126) bank116057 +116058 POINT(40.63334644889977 73.18607119537728) bank116058 +116059 POINT(41.4259347613813 73.33847683298093) bank116059 +116060 POINT(39.787444230128784 74.6906325925566) bank116060 +116061 POINT(41.498987541393234 73.85443174346132) bank116061 +116062 POINT(41.42459051692454 73.57463384642836) bank116062 +116063 POINT(41.58154056128796 73.64243283797019) bank116063 +116064 POINT(40.633633779771294 73.17806165763838) bank116064 +116065 POINT(41.69428852473906 74.79121041526807) bank116065 +116066 POINT(41.54497930488468 74.26457504568612) bank116066 +116067 POINT(39.757362259419494 73.85095257292612) bank116067 +116068 POINT(40.108246909756765 74.79807198317917) bank116068 +116069 POINT(41.004050456816856 73.53520394634931) bank116069 +116070 POINT(40.57641514252021 73.71370150348706) bank116070 +116071 POINT(40.54570022897755 74.86056597421972) bank116071 +116072 POINT(41.16717577259055 73.25799192445167) bank116072 +116073 POINT(41.20170982981847 73.31202678471304) bank116073 +116074 POINT(40.533336158127675 74.53965604485485) bank116074 +116075 POINT(41.536808509615 74.74966821958364) bank116075 +116076 POINT(39.902989319931066 73.80119934203967) bank116076 +116077 POINT(41.006472708177704 74.22344865392571) bank116077 +116078 POINT(40.382058485079604 73.69097696946538) bank116078 +116079 POINT(40.529057077394626 73.69515474765613) bank116079 +116080 POINT(40.824152937170204 73.46039666099831) bank116080 +116081 POINT(41.494693905265414 73.59365779659964) bank116081 +116082 POINT(41.139398891130014 74.00054596095484) bank116082 +116083 POINT(41.2566258458809 74.55135821986275) bank116083 +116084 POINT(40.9692318128999 73.37611115673255) bank116084 +116085 POINT(40.614164399507985 74.23127285589699) bank116085 +116086 POINT(40.69390725869039 73.10208199282935) bank116086 +116087 POINT(40.593261572128384 74.62420285358587) bank116087 +116088 POINT(40.781596845754564 74.92439280380847) bank116088 +116089 POINT(40.224664972566075 74.82220212182582) bank116089 +116090 POINT(40.322503225494486 74.66167492833752) bank116090 +116091 POINT(41.17518826139122 73.04504991189233) bank116091 +116092 POINT(39.756870711994225 73.9921112531914) bank116092 +116093 POINT(41.06444501996425 74.40644478641457) bank116093 +116094 POINT(40.690612860753724 73.90955551972715) bank116094 +116095 POINT(41.199672816030045 74.8515293945247) bank116095 +116096 POINT(41.39133280593586 73.93533661205673) bank116096 +116097 POINT(40.83806218678363 74.74607935500731) bank116097 +116098 POINT(40.5678267711228 74.91107812731592) bank116098 +116099 POINT(41.00281835592548 74.37268591650131) bank116099 +116100 POINT(40.59647730486235 74.17117195354881) bank116100 +116101 POINT(39.87453890302184 73.33396455628572) bank116101 +116102 POINT(39.73804590323036 74.32202289579607) bank116102 +116103 POINT(41.62400966170172 73.65317915929768) bank116103 +116104 POINT(41.52131455022142 74.28345060638841) bank116104 +116105 POINT(40.61927529803518 73.94012363561295) bank116105 +116106 POINT(40.34134468562017 73.35720136236836) bank116106 +116107 POINT(41.41441004040799 73.3940673026583) bank116107 +116108 POINT(40.88524047621275 74.36658129652838) bank116108 +116109 POINT(40.388712553478825 74.59008763140876) bank116109 +116110 POINT(41.67722904922008 74.73343874120768) bank116110 +116111 POINT(40.884977344412164 73.53594805670345) bank116111 +116112 POINT(40.27963578877586 74.54127372097678) bank116112 +116113 POINT(40.31684605514144 74.36101556302755) bank116113 +116114 POINT(41.0857309810272 74.81585711576425) bank116114 +116115 POINT(41.34620221495956 74.74423057302964) bank116115 +116116 POINT(41.42350742382403 74.33438355671971) bank116116 +116117 POINT(40.97613615996351 74.41438445920471) bank116117 +116118 POINT(40.56992246948688 74.00010828663855) bank116118 +116119 POINT(39.99025780524397 73.40283686311713) bank116119 +116120 POINT(40.470874577148926 73.64313385659358) bank116120 +116121 POINT(40.15287972083272 74.82846229183603) bank116121 +116122 POINT(41.42672084225203 73.36553049318186) bank116122 +116123 POINT(40.7656146272887 73.65016331878151) bank116123 +116124 POINT(41.54042577755165 74.14817270104447) bank116124 +116125 POINT(41.60513875024163 74.6013833994836) bank116125 +116126 POINT(41.506219598080094 74.64774245953382) bank116126 +116127 POINT(40.636309351953 73.91086558495559) bank116127 +116128 POINT(41.138486223233365 74.93754713154537) bank116128 +116129 POINT(41.711578217489894 74.84356662820888) bank116129 +116130 POINT(40.651663227405486 73.73504352647123) bank116130 +116131 POINT(40.82289569839844 73.9866412533154) bank116131 +116132 POINT(41.331241438832656 73.19188154663694) bank116132 +116133 POINT(39.77966682197878 73.70950451155635) bank116133 +116134 POINT(41.47532636473329 74.81946276880413) bank116134 +116135 POINT(40.032841547755595 74.778020492034) bank116135 +116136 POINT(40.68736788130847 74.33760069115154) bank116136 +116137 POINT(41.49256567934694 74.08429426909277) bank116137 +116138 POINT(40.80774611447166 73.6090968099144) bank116138 +116139 POINT(40.72962534465096 73.8549881459163) bank116139 +116140 POINT(39.799865149682226 73.46322756915924) bank116140 +116141 POINT(40.835169430876334 73.32675924630728) bank116141 +116142 POINT(41.57446033091881 74.95874324767615) bank116142 +116143 POINT(40.8764997677581 74.0546939052069) bank116143 +116144 POINT(39.7995903058329 74.27331153382138) bank116144 +116145 POINT(40.473628787355985 74.1227238741853) bank116145 +116146 POINT(41.274503899280575 74.0737280909308) bank116146 +116147 POINT(39.916259531949976 73.09535768440793) bank116147 +116148 POINT(40.46543689288059 73.23286493868217) bank116148 +116149 POINT(40.4573688237482 73.9331371321766) bank116149 +116150 POINT(41.257806331308686 73.72587259681228) bank116150 +116151 POINT(41.221161599463116 74.49200981417644) bank116151 +116152 POINT(41.30288437559218 74.40204066198037) bank116152 +116153 POINT(41.41007869090491 73.05865927526963) bank116153 +116154 POINT(40.1205851173136 74.7551269127094) bank116154 +116155 POINT(41.55362045995327 73.86255140917366) bank116155 +116156 POINT(41.64482532794857 74.55486432474129) bank116156 +116157 POINT(40.5226838574622 74.2793439303499) bank116157 +116158 POINT(41.597569470371404 73.75316374438002) bank116158 +116159 POINT(41.66823175223987 73.62777879947775) bank116159 +116160 POINT(41.571635736675645 73.2823577125167) bank116160 +116161 POINT(40.09868286449383 74.09118058964849) bank116161 +116162 POINT(40.75166888710039 73.24351940534474) bank116162 +116163 POINT(39.81926037098715 74.80837582071187) bank116163 +116164 POINT(39.71331212216097 73.50935757693355) bank116164 +116165 POINT(39.87405154597414 73.50056516888043) bank116165 +116166 POINT(41.121553366799304 73.50162040544329) bank116166 +116167 POINT(39.85346444056176 74.41704672593062) bank116167 +116168 POINT(40.51234493903779 73.27468126885677) bank116168 +116169 POINT(40.859087561350805 73.41385908169187) bank116169 +116170 POINT(40.77306284513026 74.60532045384006) bank116170 +116171 POINT(40.8991052714109 73.84040581901236) bank116171 +116172 POINT(41.651866844274416 73.92355246397697) bank116172 +116173 POINT(41.131296373851775 74.3198474239517) bank116173 +116174 POINT(39.90771265500179 73.51418671614931) bank116174 +116175 POINT(41.00681164311104 73.74853365951378) bank116175 +116176 POINT(40.45105749903691 74.55187015327643) bank116176 +116177 POINT(40.867465834350234 73.31239216364725) bank116177 +116178 POINT(41.64322398458164 73.2754363552067) bank116178 +116179 POINT(40.92105287314594 73.68836327004534) bank116179 +116180 POINT(39.76067816903782 73.4477843571141) bank116180 +116181 POINT(40.929494428067784 74.68877702758896) bank116181 +116182 POINT(40.267354551449195 73.22247721061316) bank116182 +116183 POINT(40.49928079843209 73.02993154372768) bank116183 +116184 POINT(41.41971714143807 73.3386450816159) bank116184 +116185 POINT(41.34829568494572 73.47747781596976) bank116185 +116186 POINT(41.20476171948723 74.25549507280272) bank116186 +116187 POINT(40.404632558611794 74.73851511200111) bank116187 +116188 POINT(41.15214611202814 73.38153633588003) bank116188 +116189 POINT(41.69199430124177 73.55991479199461) bank116189 +116190 POINT(40.02549037217212 73.0771050338375) bank116190 +116191 POINT(41.10995860836661 74.33905363930675) bank116191 +116192 POINT(39.903084637262516 74.97942898113136) bank116192 +116193 POINT(41.106826859935474 73.98232233432624) bank116193 +116194 POINT(41.112724304883855 73.03670292377906) bank116194 +116195 POINT(40.55974743080173 74.4510747080633) bank116195 +116196 POINT(39.81373003532138 73.99305726911373) bank116196 +116197 POINT(40.90608289143689 74.42035795583897) bank116197 +116198 POINT(40.714318675739996 74.4892837378997) bank116198 +116199 POINT(41.33336914557011 73.17257646253987) bank116199 +116200 POINT(39.85426437601337 74.00935623125419) bank116200 +116201 POINT(41.123073484128696 73.68788083330156) bank116201 +116202 POINT(39.84857356926063 74.5463737313986) bank116202 +116203 POINT(41.259659349404174 73.80758159618522) bank116203 +116204 POINT(41.63624731379437 73.55637457200926) bank116204 +116205 POINT(40.937477297658816 74.21654740564372) bank116205 +116206 POINT(40.272782747721834 74.21800413464017) bank116206 +116207 POINT(40.641254658588736 73.51308823863292) bank116207 +116208 POINT(40.680529267293444 74.36226977402421) bank116208 +116209 POINT(41.301929911777506 73.8278622582441) bank116209 +116210 POINT(41.48320808145522 73.56508522146689) bank116210 +116211 POINT(41.001111801203685 74.84370134545746) bank116211 +116212 POINT(39.867615171988405 74.99198653743613) bank116212 +116213 POINT(40.198476805634655 73.42199939628706) bank116213 +116214 POINT(40.74253877674116 74.56161403567104) bank116214 +116215 POINT(40.26200806952694 74.43493377371185) bank116215 +116216 POINT(40.54160673702704 73.5329336102793) bank116216 +116217 POINT(39.882385086320234 74.38545734180452) bank116217 +116218 POINT(41.383288848335546 73.17243701337337) bank116218 +116219 POINT(40.53390791155784 73.87677243035908) bank116219 +116220 POINT(40.13315341070357 74.5725396108577) bank116220 +116221 POINT(41.1735404653078 73.1761961840067) bank116221 +116222 POINT(40.00786211526005 73.88645037139095) bank116222 +116223 POINT(41.066759954478144 74.75384646209149) bank116223 +116224 POINT(40.93059223680467 74.44675992256388) bank116224 +116225 POINT(40.77144939906484 73.09690507656703) bank116225 +116226 POINT(40.995174969095544 73.03958705662914) bank116226 +116227 POINT(39.878523045061876 73.0348856258025) bank116227 +116228 POINT(41.182667731397686 74.63822145686844) bank116228 +116229 POINT(40.18632957104179 73.69050745338416) bank116229 +116230 POINT(40.74500569062577 74.34099033839728) bank116230 +116231 POINT(40.050215117643354 74.93572989689673) bank116231 +116232 POINT(40.983114155294246 73.69958405920038) bank116232 +116233 POINT(40.892380247576924 74.23869246377669) bank116233 +116234 POINT(39.803723708562586 73.75083759361135) bank116234 +116235 POINT(39.769757302338185 73.68721184571709) bank116235 +116236 POINT(39.91642744062896 73.42455169385315) bank116236 +116237 POINT(40.7901076145471 74.24454175135888) bank116237 +116238 POINT(41.20513962126368 74.73239010116552) bank116238 +116239 POINT(40.99162451086818 73.55513613547242) bank116239 +116240 POINT(40.64876011413206 73.2412403903927) bank116240 +116241 POINT(40.850851593833994 73.17228920863951) bank116241 +116242 POINT(40.345059571134506 73.96089450942449) bank116242 +116243 POINT(41.110089544897846 74.89204562115515) bank116243 +116244 POINT(39.85140923056929 74.50738215096288) bank116244 +116245 POINT(40.958880890677776 73.67079172431185) bank116245 +116246 POINT(40.72199305342569 73.9868758739963) bank116246 +116247 POINT(40.159674928490055 73.27145276294608) bank116247 +116248 POINT(39.989580563209216 73.1702757453449) bank116248 +116249 POINT(39.81299452891116 74.47660064409213) bank116249 +116250 POINT(40.11886399335671 74.33521294036018) bank116250 +116251 POINT(41.67521610661815 73.62720814873292) bank116251 +116252 POINT(41.46414820804765 73.86805374681195) bank116252 +116253 POINT(40.81095885521714 74.84106414870071) bank116253 +116254 POINT(41.70435104575648 73.6021298322045) bank116254 +116255 POINT(40.46208973295884 74.7929743595386) bank116255 +116256 POINT(40.135428171023264 73.04664033377841) bank116256 +116257 POINT(41.175454315133614 73.32022265970032) bank116257 +116258 POINT(40.65940986181032 73.73068143581612) bank116258 +116259 POINT(40.17445546833399 73.51410625414786) bank116259 +116260 POINT(41.21480512507786 73.10117157517966) bank116260 +116261 POINT(41.19983594811538 73.6544710654065) bank116261 +116262 POINT(40.93649224395981 73.17802057123018) bank116262 +116263 POINT(40.82784512583842 74.06548243364333) bank116263 +116264 POINT(41.31941401510385 74.36668300356881) bank116264 +116265 POINT(40.84858467260695 73.32747592726459) bank116265 +116266 POINT(40.816635513588416 74.13377871553269) bank116266 +116267 POINT(41.06793608656594 73.39757671301747) bank116267 +116268 POINT(40.827776624322006 74.080966969206) bank116268 +116269 POINT(41.32987899919154 73.20889760361781) bank116269 +116270 POINT(39.92738951997516 73.46650935085782) bank116270 +116271 POINT(41.44873954655642 74.34108887491165) bank116271 +116272 POINT(40.12749443560329 74.4629926170733) bank116272 +116273 POINT(41.013850169796484 74.3811054697321) bank116273 +116274 POINT(40.68132215760212 74.60805475309301) bank116274 +116275 POINT(39.76327196773258 73.60591018287299) bank116275 +116276 POINT(40.53434286204751 73.47250918118151) bank116276 +116277 POINT(40.32888290723894 73.9880074700568) bank116277 +116278 POINT(41.334547881591966 74.55880423315412) bank116278 +116279 POINT(40.81824694649748 74.4746456659907) bank116279 +116280 POINT(39.90196783709676 74.89041816254185) bank116280 +116281 POINT(41.2270825277714 74.21629085629691) bank116281 +116282 POINT(41.07831127994946 73.72813173023877) bank116282 +116283 POINT(39.73424000798604 73.40683118721383) bank116283 +116284 POINT(40.52075382887803 74.13987701872512) bank116284 +116285 POINT(41.24655296174572 74.67705277248254) bank116285 +116286 POINT(40.10639362216288 74.0127697420703) bank116286 +116287 POINT(40.94890765357894 73.54012427962158) bank116287 +116288 POINT(40.93308211282665 73.133382983699) bank116288 +116289 POINT(40.14012770447291 73.49456713035471) bank116289 +116290 POINT(40.958197733053495 74.48593992874686) bank116290 +116291 POINT(39.738513034524516 74.30125778919943) bank116291 +116292 POINT(41.264673738578445 73.1142477332354) bank116292 +116293 POINT(40.415991255430896 74.51351038492774) bank116293 +116294 POINT(39.91209148147164 74.71636319160568) bank116294 +116295 POINT(40.98680114601166 74.2351874332292) bank116295 +116296 POINT(39.870361331192235 73.11598722470652) bank116296 +116297 POINT(40.76486756315933 74.18104766957376) bank116297 +116298 POINT(40.117585181372064 73.90080918636266) bank116298 +116299 POINT(40.73610944343482 73.77768300468105) bank116299 +116300 POINT(40.39858368636189 74.70541744735036) bank116300 +116301 POINT(41.483398483847964 74.47534137897837) bank116301 +116302 POINT(40.613311253339035 74.7162517513821) bank116302 +116303 POINT(40.38568160162055 73.0496025705793) bank116303 +116304 POINT(41.55665022859206 74.28652212528391) bank116304 +116305 POINT(40.85990292565039 74.11482268146794) bank116305 +116306 POINT(40.44451637927659 73.55082741938064) bank116306 +116307 POINT(40.784447449530106 73.45540498445281) bank116307 +116308 POINT(41.251174888511606 74.72338326124574) bank116308 +116309 POINT(41.59858247509524 74.67447195930775) bank116309 +116310 POINT(41.6918850468256 73.71256069730748) bank116310 +116311 POINT(41.268450263239565 73.83701068664627) bank116311 +116312 POINT(40.6188443071273 73.252982236793) bank116312 +116313 POINT(41.12988680463503 73.68361410159238) bank116313 +116314 POINT(41.67168777475734 73.93110144072284) bank116314 +116315 POINT(40.13627501285921 73.42206135851089) bank116315 +116316 POINT(40.86317878231952 73.98811688947409) bank116316 +116317 POINT(41.32308684147617 73.58558158054498) bank116317 +116318 POINT(40.83302858647485 73.3002246627187) bank116318 +116319 POINT(40.2141527751415 73.90214162424682) bank116319 +116320 POINT(41.5152988080268 74.03050927751848) bank116320 +116321 POINT(39.74739942605904 73.03802049519547) bank116321 +116322 POINT(41.07747078463329 74.96762547360545) bank116322 +116323 POINT(40.330947562067735 73.72484690204602) bank116323 +116324 POINT(40.80705828138174 74.9953479872523) bank116324 +116325 POINT(40.9522980338452 74.81385502722868) bank116325 +116326 POINT(41.171426305571835 74.29148091805119) bank116326 +116327 POINT(41.51236340093411 73.35185182437438) bank116327 +116328 POINT(39.87005544061567 73.74893355086742) bank116328 +116329 POINT(40.69924571320257 74.07308084651996) bank116329 +116330 POINT(40.70309914018019 73.11134138921824) bank116330 +116331 POINT(39.946219711952054 74.99614526089076) bank116331 +116332 POINT(40.2871651807202 73.73077954637616) bank116332 +116333 POINT(40.1311373821355 74.9728222545823) bank116333 +116334 POINT(40.0374507912987 74.80698147565427) bank116334 +116335 POINT(40.32190076198494 73.37334504674158) bank116335 +116336 POINT(39.98181182176193 73.78853108046484) bank116336 +116337 POINT(40.80084045127302 73.69925486741506) bank116337 +116338 POINT(40.49301009986396 73.88952379533747) bank116338 +116339 POINT(41.59246946122496 74.11518906810056) bank116339 +116340 POINT(41.362732543334204 73.69659629770344) bank116340 +116341 POINT(40.89190161551596 73.7360276152369) bank116341 +116342 POINT(39.934392011801464 75.00490414626992) bank116342 +116343 POINT(40.60776583671714 74.1170091117428) bank116343 +116344 POINT(41.607730436926346 74.75586126317685) bank116344 +116345 POINT(39.777750833181194 74.42659919909337) bank116345 +116346 POINT(41.45700345346913 73.75589303999148) bank116346 +116347 POINT(41.21901332990243 73.9745743791139) bank116347 +116348 POINT(41.06455628421912 75.00020623754091) bank116348 +116349 POINT(40.247078883055146 73.16010630949884) bank116349 +116350 POINT(41.55898192039918 74.39281253720283) bank116350 +116351 POINT(41.7077015085196 73.91659473781398) bank116351 +116352 POINT(40.44261898187201 73.67379138891083) bank116352 +116353 POINT(41.20534275031503 73.67902903669071) bank116353 +116354 POINT(40.71174987741985 73.51619578742573) bank116354 +116355 POINT(40.083240673637285 73.40575822738545) bank116355 +116356 POINT(40.5859086083739 74.2396668498538) bank116356 +116357 POINT(40.676708875925755 73.83731188036462) bank116357 +116358 POINT(40.61187443820217 73.87485746843636) bank116358 +116359 POINT(41.33232238035922 74.42106135977532) bank116359 +116360 POINT(41.59690125152064 74.69270900815084) bank116360 +116361 POINT(41.19204078858686 74.60776595221373) bank116361 +116362 POINT(40.16877767809563 73.23947098009941) bank116362 +116363 POINT(41.21065940609807 73.09126889860934) bank116363 +116364 POINT(40.48126564069344 73.73686423252009) bank116364 +116365 POINT(41.52250315642312 73.90024758372101) bank116365 +116366 POINT(39.85128577912954 74.12824884888983) bank116366 +116367 POINT(40.74186225150066 73.59374357514758) bank116367 +116368 POINT(40.3254600739077 74.5956326581943) bank116368 +116369 POINT(40.72561581125674 73.08899889724508) bank116369 +116370 POINT(40.96930017119558 74.79143088985037) bank116370 +116371 POINT(39.85926443743388 73.83804557913662) bank116371 +116372 POINT(40.66828396106449 74.48363303985849) bank116372 +116373 POINT(39.77651965328112 74.9202011640374) bank116373 +116374 POINT(40.0706092151925 74.12734789320369) bank116374 +116375 POINT(40.244078005938235 74.477619979486) bank116375 +116376 POINT(40.06795610322774 74.34142247247938) bank116376 +116377 POINT(41.5516558357398 73.31193172235173) bank116377 +116378 POINT(40.259877035291325 73.22926277014302) bank116378 +116379 POINT(40.977797011155275 73.12088379993592) bank116379 +116380 POINT(40.84392348989276 74.34404805625704) bank116380 +116381 POINT(40.11660868201651 74.26432808887527) bank116381 +116382 POINT(39.87983062484685 73.57056055952359) bank116382 +116383 POINT(41.35977775593227 73.93723836596742) bank116383 +116384 POINT(40.332243478901276 74.20253094282938) bank116384 +116385 POINT(40.22950693595319 73.91355851796527) bank116385 +116386 POINT(41.53287619155694 73.57660849978951) bank116386 +116387 POINT(40.482551882689286 74.5896555280089) bank116387 +116388 POINT(41.4611197716878 73.449019748085) bank116388 +116389 POINT(41.41933414040654 73.81894307647794) bank116389 +116390 POINT(40.86613297846149 74.57749423538083) bank116390 +116391 POINT(39.9876407579073 73.81915273150591) bank116391 +116392 POINT(40.973182739610074 74.10411642645374) bank116392 +116393 POINT(40.846070361457166 74.19870055025663) bank116393 +116394 POINT(41.69982474131998 73.67081401303852) bank116394 +116395 POINT(40.27420879970242 73.39208047054943) bank116395 +116396 POINT(40.17066837785705 73.97239707319038) bank116396 +116397 POINT(39.780040071848276 74.62349398319026) bank116397 +116398 POINT(41.04380000165658 73.41127345021978) bank116398 +116399 POINT(40.32576606224288 73.3813283251149) bank116399 +116400 POINT(40.907845389448745 73.3052131952861) bank116400 +116401 POINT(40.56222253081007 74.65490429955062) bank116401 +116402 POINT(40.310174797817695 73.27972928465331) bank116402 +116403 POINT(41.19746747803013 73.28315338296736) bank116403 +116404 POINT(40.67618507684192 74.06983010400386) bank116404 +116405 POINT(40.23317192556102 74.64421908110802) bank116405 +116406 POINT(39.78536284571254 73.81410072361761) bank116406 +116407 POINT(40.922532561109165 74.17858825114607) bank116407 +116408 POINT(40.516783839890635 73.2355588464551) bank116408 +116409 POINT(40.028972716199824 73.9728055215491) bank116409 +116410 POINT(39.78735407158428 74.4496088546799) bank116410 +116411 POINT(41.13954338162379 73.80198603717122) bank116411 +116412 POINT(40.620219500346884 73.14363802128057) bank116412 +116413 POINT(40.48187899608041 73.51696518806312) bank116413 +116414 POINT(39.9400564944492 73.44560367023195) bank116414 +116415 POINT(40.84596549818193 73.76862605566234) bank116415 +116416 POINT(41.44890455843117 74.80738539881197) bank116416 +116417 POINT(41.24018646542661 73.2578533380684) bank116417 +116418 POINT(39.85107760144049 74.75861713745111) bank116418 +116419 POINT(40.62598660511564 74.86576367268036) bank116419 +116420 POINT(40.281075843203276 73.24534937890607) bank116420 +116421 POINT(39.920910333024096 74.32837951682039) bank116421 +116422 POINT(41.631213487317886 73.69827805146384) bank116422 +116423 POINT(40.18088049460419 74.1159612865404) bank116423 +116424 POINT(40.10153969219016 73.91221169824578) bank116424 +116425 POINT(40.70832359547926 73.16629456105468) bank116425 +116426 POINT(41.56559002069328 73.59283773777398) bank116426 +116427 POINT(40.00573895398551 74.28376454099649) bank116427 +116428 POINT(40.161072359269006 74.20996407677279) bank116428 +116429 POINT(41.46099142138821 74.62383154202082) bank116429 +116430 POINT(40.72810071463108 74.48869733164113) bank116430 +116431 POINT(41.078926345519555 73.30793861114647) bank116431 +116432 POINT(40.86907125043595 73.16984583346569) bank116432 +116433 POINT(41.68619505054768 74.99665894211182) bank116433 +116434 POINT(41.34816312101982 73.46086071334179) bank116434 +116435 POINT(40.46205364735976 74.31386836101862) bank116435 +116436 POINT(41.59083438103666 74.76833686195982) bank116436 +116437 POINT(40.12754185075227 74.20426861333084) bank116437 +116438 POINT(40.63141480946049 73.67419868944305) bank116438 +116439 POINT(40.54789925071352 73.03212468545854) bank116439 +116440 POINT(40.333335390622786 73.27491609465967) bank116440 +116441 POINT(41.51265600978966 74.73469818437921) bank116441 +116442 POINT(41.037699481115574 73.64544097819866) bank116442 +116443 POINT(40.534485806623984 74.7384752657925) bank116443 +116444 POINT(40.604660539569124 73.8323395330128) bank116444 +116445 POINT(40.73278142530194 73.8951406591672) bank116445 +116446 POINT(40.88316718558826 73.33705487680312) bank116446 +116447 POINT(39.84901569325278 74.02992234862562) bank116447 +116448 POINT(41.00823714109418 74.10097557462743) bank116448 +116449 POINT(41.53514804679151 74.8317470173019) bank116449 +116450 POINT(41.28497761521795 74.64927630555731) bank116450 +116451 POINT(41.47417141340594 74.06993146812552) bank116451 +116452 POINT(40.823134851545504 73.24433482736565) bank116452 +116453 POINT(40.37094909391461 74.96380435013067) bank116453 +116454 POINT(39.958572731207326 74.85137593727066) bank116454 +116455 POINT(40.30837968774091 73.23126889755548) bank116455 +116456 POINT(39.75177591326803 74.6048593175624) bank116456 +116457 POINT(41.396607267481954 73.14412527774357) bank116457 +116458 POINT(40.52775280149285 73.07615212687914) bank116458 +116459 POINT(41.233124643533394 73.4271782803744) bank116459 +116460 POINT(41.613510228650455 74.72442229664598) bank116460 +116461 POINT(40.38552682834964 73.55174940131535) bank116461 +116462 POINT(41.523136280001935 73.5287635882437) bank116462 +116463 POINT(40.32702640406109 73.51359539604864) bank116463 +116464 POINT(40.716534464649584 73.06207059764631) bank116464 +116465 POINT(41.04571840293804 74.3649189316132) bank116465 +116466 POINT(40.76535245055223 73.57295807159721) bank116466 +116467 POINT(40.78056117792849 73.60381346824906) bank116467 +116468 POINT(41.68605626313238 73.7178160361934) bank116468 +116469 POINT(40.05990528573377 73.78920592425095) bank116469 +116470 POINT(40.75933513018515 74.84473068038507) bank116470 +116471 POINT(41.00158984172762 73.96866285898506) bank116471 +116472 POINT(41.1622136607069 73.71353481267585) bank116472 +116473 POINT(40.30318845028907 74.9094758639695) bank116473 +116474 POINT(41.623005721447306 73.04747007843685) bank116474 +116475 POINT(41.169745635694085 73.63084003500107) bank116475 +116476 POINT(41.074276698411246 74.88605035829852) bank116476 +116477 POINT(40.343994530314006 73.94005338420303) bank116477 +116478 POINT(39.87784297603518 74.81403872654202) bank116478 +116479 POINT(41.45395965513386 74.89397871219379) bank116479 +116480 POINT(40.92936979763089 73.48978216152673) bank116480 +116481 POINT(39.90245320906217 74.55045578008597) bank116481 +116482 POINT(40.04849367325046 74.09834003396938) bank116482 +116483 POINT(41.00056395599397 74.49901024650666) bank116483 +116484 POINT(41.25737612558158 73.99003967573624) bank116484 +116485 POINT(40.12314568416346 74.34309276971773) bank116485 +116486 POINT(41.51138705885565 74.25837366275962) bank116486 +116487 POINT(40.68318795348858 73.01184143223992) bank116487 +116488 POINT(41.182613217326676 73.29913617092656) bank116488 +116489 POINT(39.99385453247089 73.13292326182655) bank116489 +116490 POINT(41.609304178825695 74.92801907149048) bank116490 +116491 POINT(41.53579372678826 73.74113560039028) bank116491 +116492 POINT(40.449191665226465 73.75625164264643) bank116492 +116493 POINT(41.241523137781165 74.24792934878322) bank116493 +116494 POINT(40.24161376233213 74.04515359394169) bank116494 +116495 POINT(40.374744859574506 74.1917588756678) bank116495 +116496 POINT(41.6357626570305 73.73763569132358) bank116496 +116497 POINT(40.340961343058964 73.86437161895725) bank116497 +116498 POINT(39.758340882830424 73.88201527581019) bank116498 +116499 POINT(41.328973186439924 73.72100289281435) bank116499 +116500 POINT(40.443917788069164 73.93139662251215) bank116500 +116501 POINT(40.07897152698646 74.811020303656) bank116501 +116502 POINT(40.89579160836046 74.23127924875881) bank116502 +116503 POINT(40.88012402470548 73.92414617826314) bank116503 +116504 POINT(40.633150825240705 74.98984337710039) bank116504 +116505 POINT(41.35903521838296 74.43221163047669) bank116505 +116506 POINT(41.54307705346814 74.26468761642712) bank116506 +116507 POINT(41.16088162651643 74.9815125839269) bank116507 +116508 POINT(40.19967188822412 74.90401394830394) bank116508 +116509 POINT(41.23019095444772 74.08754943413601) bank116509 +116510 POINT(41.18679506791982 73.38220308752166) bank116510 +116511 POINT(40.16505300052236 73.50915740939321) bank116511 +116512 POINT(41.08549691751675 74.7773998058778) bank116512 +116513 POINT(41.08881705733579 73.29931341032726) bank116513 +116514 POINT(41.40964195185623 73.5473871803108) bank116514 +116515 POINT(39.91315403403899 73.47968556385828) bank116515 +116516 POINT(41.26979437204876 73.85868095009624) bank116516 +116517 POINT(41.32710070203922 74.83886132187486) bank116517 +116518 POINT(41.12955483783534 74.07392789400768) bank116518 +116519 POINT(40.441342177771645 74.46672325687163) bank116519 +116520 POINT(39.750414890701954 73.9637058536442) bank116520 +116521 POINT(39.96423571204329 73.99334330854929) bank116521 +116522 POINT(41.339464623561014 74.22328355799232) bank116522 +116523 POINT(40.42330242011423 74.16110996640856) bank116523 +116524 POINT(41.180470060202914 73.88095567263125) bank116524 +116525 POINT(41.231092496073494 73.10047049989312) bank116525 +116526 POINT(39.95429403924064 74.57751138917139) bank116526 +116527 POINT(40.26062435415281 73.53985112769102) bank116527 +116528 POINT(39.85101664169958 73.67601480579394) bank116528 +116529 POINT(41.10580890795067 74.67332098996559) bank116529 +116530 POINT(40.97910152629154 73.2179686670946) bank116530 +116531 POINT(41.25297190252171 74.10457086720734) bank116531 +116532 POINT(40.332052337001834 74.18620768757623) bank116532 +116533 POINT(41.65841126255981 73.48473336986049) bank116533 +116534 POINT(41.29847047807489 73.32045247986048) bank116534 +116535 POINT(41.19457841238762 73.83593320826348) bank116535 +116536 POINT(40.3255456839231 73.89454014928612) bank116536 +116537 POINT(40.12616632998962 73.86944527145812) bank116537 +116538 POINT(41.29010803905996 74.16492518083957) bank116538 +116539 POINT(40.72578853269772 73.80515523684758) bank116539 +116540 POINT(40.54774779007119 74.80096212575727) bank116540 +116541 POINT(40.49608998633833 73.3094284672343) bank116541 +116542 POINT(40.90143775716628 74.0987478101922) bank116542 +116543 POINT(40.071347669057516 73.68807668438095) bank116543 +116544 POINT(40.70701377345162 74.35538698729991) bank116544 +116545 POINT(40.702014234875634 73.04239536879206) bank116545 +116546 POINT(40.27771889981273 73.829000106365) bank116546 +116547 POINT(41.34936949858077 73.8694550080028) bank116547 +116548 POINT(39.883627753252355 73.39050779322999) bank116548 +116549 POINT(40.788659205206926 74.14855817292418) bank116549 +116550 POINT(41.458291324552036 73.61898152360263) bank116550 +116551 POINT(39.9958565414079 73.17620567439745) bank116551 +116552 POINT(41.409818505153765 73.5547861977492) bank116552 +116553 POINT(41.243008070978306 73.3704804830928) bank116553 +116554 POINT(39.92287166078352 74.4552923013411) bank116554 +116555 POINT(41.534969899622894 73.72843500143799) bank116555 +116556 POINT(41.50837345535176 74.86186049374459) bank116556 +116557 POINT(40.21548270303779 73.99228635017776) bank116557 +116558 POINT(40.23083299244162 73.81837652311293) bank116558 +116559 POINT(41.567296375307826 74.38760672165408) bank116559 +116560 POINT(40.979046792634186 74.13881796525501) bank116560 +116561 POINT(41.55394371486348 73.10034479353882) bank116561 +116562 POINT(41.210583280558154 73.61237866590396) bank116562 +116563 POINT(41.70371746134039 74.57250108990877) bank116563 +116564 POINT(41.24218317349769 73.49733802154616) bank116564 +116565 POINT(40.502055477843214 73.26445712477069) bank116565 +116566 POINT(40.333989440114664 74.67670452892933) bank116566 +116567 POINT(40.419078939487804 74.94674855740269) bank116567 +116568 POINT(39.78849482309847 74.39154610588288) bank116568 +116569 POINT(41.69536588555509 74.72059626544397) bank116569 +116570 POINT(40.81539198290648 73.17348735985918) bank116570 +116571 POINT(39.871353211506275 73.24689617756708) bank116571 +116572 POINT(39.75691461021361 73.2767066480505) bank116572 +116573 POINT(41.04646687115627 73.93796521512117) bank116573 +116574 POINT(40.65878629938564 74.84297462263245) bank116574 +116575 POINT(40.688692208367456 74.37538199804222) bank116575 +116576 POINT(40.966575175678166 73.73408016552293) bank116576 +116577 POINT(41.317865638608154 74.37629303748314) bank116577 +116578 POINT(41.052411932112214 73.87422286207915) bank116578 +116579 POINT(40.53915444265343 74.23962562869026) bank116579 +116580 POINT(40.94806068422189 73.01919310673713) bank116580 +116581 POINT(39.75544321882795 74.44992610005266) bank116581 +116582 POINT(40.39539312063233 74.21018026611554) bank116582 +116583 POINT(39.805575536472354 73.74776073842791) bank116583 +116584 POINT(40.8748152240124 73.61047157921955) bank116584 +116585 POINT(40.005517629526096 73.40972293084756) bank116585 +116586 POINT(39.828914103245964 73.0906236442774) bank116586 +116587 POINT(39.782763935024015 74.46905908789567) bank116587 +116588 POINT(40.149041412420495 73.71887989796551) bank116588 +116589 POINT(41.37763867042549 73.72798431920491) bank116589 +116590 POINT(39.949863356781314 74.48214585110901) bank116590 +116591 POINT(41.28818937317996 73.44064912859038) bank116591 +116592 POINT(40.18368773791417 73.23700166692421) bank116592 +116593 POINT(40.07691644465042 73.81809384797322) bank116593 +116594 POINT(40.735988233906255 74.13735850134239) bank116594 +116595 POINT(40.54256610830381 74.54947750158964) bank116595 +116596 POINT(39.91100655996046 73.67350613734551) bank116596 +116597 POINT(41.3810643079013 73.47336383197053) bank116597 +116598 POINT(40.166534046737574 73.92227560708234) bank116598 +116599 POINT(41.32829394758164 73.69336764157367) bank116599 +116600 POINT(40.82560393696035 74.03000629574939) bank116600 +116601 POINT(40.93227625412556 74.55171836749841) bank116601 +116602 POINT(41.67519283156023 74.49630327654563) bank116602 +116603 POINT(41.1116163661733 73.51895862494676) bank116603 +116604 POINT(40.684397588469466 73.84940525528171) bank116604 +116605 POINT(40.78659248341009 73.08154184071184) bank116605 +116606 POINT(41.481713354242856 74.0475400648139) bank116606 +116607 POINT(40.42458185752417 74.80539989715916) bank116607 +116608 POINT(40.50889262306454 73.05655531862756) bank116608 +116609 POINT(40.36170929182447 73.85784541123941) bank116609 +116610 POINT(40.88482287260442 73.92611871460586) bank116610 +116611 POINT(40.59713822909703 74.83335508164578) bank116611 +116612 POINT(40.0999210706941 74.32556477105634) bank116612 +116613 POINT(41.05453618169708 74.60970200848261) bank116613 +116614 POINT(40.765151991981774 74.620820015386) bank116614 +116615 POINT(41.27433300253741 74.61857482961183) bank116615 +116616 POINT(40.9997796516642 74.62010754307565) bank116616 +116617 POINT(40.516774841404036 73.37962654120669) bank116617 +116618 POINT(40.01824618252827 74.29380928441014) bank116618 +116619 POINT(39.81542139178927 73.17688314788208) bank116619 +116620 POINT(40.56477057696543 73.42376615807996) bank116620 +116621 POINT(41.25429727688076 73.55759574569741) bank116621 +116622 POINT(40.21470051412636 73.98200926628517) bank116622 +116623 POINT(40.58219055438819 74.74701365635663) bank116623 +116624 POINT(40.78489847748308 73.92116613281857) bank116624 +116625 POINT(40.88098449061057 74.52225657783877) bank116625 +116626 POINT(40.84797321222688 74.71925036848188) bank116626 +116627 POINT(40.1382310294446 73.13201650190771) bank116627 +116628 POINT(39.723256284397834 73.58749320662304) bank116628 +116629 POINT(40.17119810118131 74.221376780962) bank116629 +116630 POINT(40.60277383657609 73.31736503963417) bank116630 +116631 POINT(40.59158080401192 73.51024476506488) bank116631 +116632 POINT(41.52874302985649 74.71826803337832) bank116632 +116633 POINT(41.65779947241019 73.92995009502168) bank116633 +116634 POINT(39.83708021188283 74.92435490977078) bank116634 +116635 POINT(40.324572363097516 74.91009748733755) bank116635 +116636 POINT(41.01581217958079 74.38533733995844) bank116636 +116637 POINT(41.17646237776888 73.38878817014685) bank116637 +116638 POINT(40.582205465382735 73.09067954018133) bank116638 +116639 POINT(40.450108693949176 73.0399635604408) bank116639 +116640 POINT(40.44724758322302 73.4589319516528) bank116640 +116641 POINT(41.66397191639091 73.25908476872644) bank116641 +116642 POINT(40.677464848265 73.8260565144594) bank116642 +116643 POINT(39.977294411416345 73.17082289789805) bank116643 +116644 POINT(41.10686075953334 73.39834801605699) bank116644 +116645 POINT(40.17097355432525 74.6438640186445) bank116645 +116646 POINT(41.60281963278884 73.45531759808779) bank116646 +116647 POINT(40.14098274025707 73.7695880984566) bank116647 +116648 POINT(39.804612877757634 74.9506829201911) bank116648 +116649 POINT(40.32777224925165 74.73994495306793) bank116649 +116650 POINT(41.69300412959759 74.26064906102333) bank116650 +116651 POINT(41.53744616646744 73.36462697821612) bank116651 +116652 POINT(40.88848692452018 74.51193387014588) bank116652 +116653 POINT(40.801285420144374 74.4979117180887) bank116653 +116654 POINT(39.82127886933604 74.27910714778885) bank116654 +116655 POINT(40.39986620592415 74.13101059396631) bank116655 +116656 POINT(40.21691315417181 73.62758158998406) bank116656 +116657 POINT(40.396213585298504 74.39816855950654) bank116657 +116658 POINT(40.59498976883718 73.85252519425637) bank116658 +116659 POINT(41.249588292804255 73.38519888290202) bank116659 +116660 POINT(40.432726715243824 74.02354940069715) bank116660 +116661 POINT(41.628484252836664 74.20304061528063) bank116661 +116662 POINT(41.212873075923575 74.95762705724941) bank116662 +116663 POINT(41.11913564692152 73.13036791718541) bank116663 +116664 POINT(40.24824299355386 74.77307865717485) bank116664 +116665 POINT(40.70854766284476 74.81435012084779) bank116665 +116666 POINT(40.50940773175667 73.42042115707919) bank116666 +116667 POINT(41.49743744507299 73.55700715748996) bank116667 +116668 POINT(40.92704393146055 74.61737760054955) bank116668 +116669 POINT(41.35985637667498 73.40751556165742) bank116669 +116670 POINT(40.08349989407866 73.39845173557119) bank116670 +116671 POINT(40.65359825518629 74.92947943154853) bank116671 +116672 POINT(40.975098984425344 73.48915227219356) bank116672 +116673 POINT(40.55197006183331 73.60825469438483) bank116673 +116674 POINT(41.64456837636138 73.63332774302432) bank116674 +116675 POINT(41.32882123524186 73.88571383808063) bank116675 +116676 POINT(39.818786593310335 73.63022352962136) bank116676 +116677 POINT(41.26266333510052 73.44088644623932) bank116677 +116678 POINT(40.40755724503687 73.53575470680538) bank116678 +116679 POINT(39.93390809491694 74.57376235158651) bank116679 +116680 POINT(40.82331058064814 74.15440300109377) bank116680 +116681 POINT(40.006681711970025 73.260873051698) bank116681 +116682 POINT(40.96892616651081 74.67093812612843) bank116682 +116683 POINT(40.51255169105692 73.82227947964749) bank116683 +116684 POINT(40.37202393580849 74.86107060733997) bank116684 +116685 POINT(41.559775348772654 73.3154225176329) bank116685 +116686 POINT(41.580741958598686 73.9416709914855) bank116686 +116687 POINT(40.028639974134954 74.83354535506682) bank116687 +116688 POINT(41.06044033485371 73.02693467087428) bank116688 +116689 POINT(40.464124260748854 73.88186599979181) bank116689 +116690 POINT(39.83552057042354 74.45151462549345) bank116690 +116691 POINT(40.260562918559096 73.76215642239298) bank116691 +116692 POINT(40.88600600659927 74.357253308035) bank116692 +116693 POINT(40.27589898035089 74.7726493155446) bank116693 +116694 POINT(40.00840646174406 73.09004771454156) bank116694 +116695 POINT(40.064517000036815 74.93545625471097) bank116695 +116696 POINT(41.02393901503103 74.64164901959839) bank116696 +116697 POINT(41.338663977244686 73.90000006220136) bank116697 +116698 POINT(41.463826115417106 74.60267747153995) bank116698 +116699 POINT(40.595015239066974 73.0218136658245) bank116699 +116700 POINT(40.3855315102025 73.15054397127898) bank116700 +116701 POINT(39.73078564792822 74.89531042338874) bank116701 +116702 POINT(41.025955437675314 74.64583282891789) bank116702 +116703 POINT(40.40407321595265 73.893391484183) bank116703 +116704 POINT(40.47098527597924 74.74541335038015) bank116704 +116705 POINT(41.125989415876354 73.27263955832922) bank116705 +116706 POINT(41.627295410197156 74.3080774527948) bank116706 +116707 POINT(41.217722061141835 74.17834252757898) bank116707 +116708 POINT(41.49815092633494 74.93792840842707) bank116708 +116709 POINT(40.16332499235455 73.74364281232987) bank116709 +116710 POINT(41.118245152733856 73.22072128401587) bank116710 +116711 POINT(40.406386214191826 74.07135557318185) bank116711 +116712 POINT(40.18974132975751 74.37602553674898) bank116712 +116713 POINT(39.78824572268096 74.31795477582361) bank116713 +116714 POINT(40.62995093037508 73.92396278439013) bank116714 +116715 POINT(40.042856620090454 73.51592592583725) bank116715 +116716 POINT(39.8047445908803 74.8212037575831) bank116716 +116717 POINT(41.711450574240985 73.92996951373598) bank116717 +116718 POINT(39.80799313052762 73.16074158467735) bank116718 +116719 POINT(40.15823054153928 73.24440422205402) bank116719 +116720 POINT(41.62162650804517 73.18260310359109) bank116720 +116721 POINT(41.421668332702964 74.41232633880695) bank116721 +116722 POINT(41.26685864637163 74.43361068757677) bank116722 +116723 POINT(40.47201393420212 74.54307985927504) bank116723 +116724 POINT(40.63790794708526 73.38092339491797) bank116724 +116725 POINT(41.65944931628287 74.149757365331) bank116725 +116726 POINT(41.38807306517813 74.61901301953357) bank116726 +116727 POINT(40.69184422650522 73.67836812292104) bank116727 +116728 POINT(41.37956518033402 73.91743014654081) bank116728 +116729 POINT(41.26869706797419 74.26361206258835) bank116729 +116730 POINT(40.06843075115247 74.39824604339206) bank116730 +116731 POINT(39.90851031809145 73.56969848152302) bank116731 +116732 POINT(39.79811261767728 73.60497257715517) bank116732 +116733 POINT(40.64968518602047 74.57783147442437) bank116733 +116734 POINT(39.84146855809342 74.56737951059684) bank116734 +116735 POINT(40.50642210042602 73.7854747555924) bank116735 +116736 POINT(41.33540456136433 73.14487494763647) bank116736 +116737 POINT(41.602667330828446 74.65899333885334) bank116737 +116738 POINT(41.091915124044824 73.9575914115122) bank116738 +116739 POINT(40.7444952861238 73.10967744963934) bank116739 +116740 POINT(40.50840823752157 73.5332804460654) bank116740 +116741 POINT(41.59891556652614 74.94866788391724) bank116741 +116742 POINT(40.425312625895515 73.82209615668091) bank116742 +116743 POINT(40.76938090173564 74.34576606569799) bank116743 +116744 POINT(40.94252915820601 73.91951258235903) bank116744 +116745 POINT(41.03356408848792 74.74012251455453) bank116745 +116746 POINT(39.801401852070256 74.13082523512307) bank116746 +116747 POINT(41.48880172872209 74.70985112991985) bank116747 +116748 POINT(40.12565313902236 74.98224049555894) bank116748 +116749 POINT(40.796184563878334 74.51057911655752) bank116749 +116750 POINT(41.444200225092224 73.25216360659579) bank116750 +116751 POINT(40.79529273828308 74.23387118390332) bank116751 +116752 POINT(40.58592828901763 74.37403066681918) bank116752 +116753 POINT(40.29816640905797 74.781430353641) bank116753 +116754 POINT(41.125384358317724 74.73928429095659) bank116754 +116755 POINT(41.146486704245454 74.36858090721896) bank116755 +116756 POINT(41.352040779826865 73.07351283022331) bank116756 +116757 POINT(40.54940714084675 73.10180997197503) bank116757 +116758 POINT(40.71292527510467 74.2733321367703) bank116758 +116759 POINT(40.68332366779569 73.22502238073224) bank116759 +116760 POINT(40.1621694607041 74.79497571123959) bank116760 +116761 POINT(41.57187557780561 73.23278722517634) bank116761 +116762 POINT(41.036436255679654 73.14727674598107) bank116762 +116763 POINT(40.542518210260745 74.45016251784224) bank116763 +116764 POINT(40.852433142789245 73.47972192475285) bank116764 +116765 POINT(40.27283402016359 73.65915327541039) bank116765 +116766 POINT(40.911201534746596 74.35327479457426) bank116766 +116767 POINT(40.516838703876076 73.01726823432674) bank116767 +116768 POINT(40.54093624428141 74.34404399308059) bank116768 +116769 POINT(40.1494038915312 73.11173176877162) bank116769 +116770 POINT(39.827317333108155 74.23676939424057) bank116770 +116771 POINT(41.6930964190517 74.19355953189438) bank116771 +116772 POINT(41.66584227302454 73.20539132650838) bank116772 +116773 POINT(40.114466401118385 74.78754462881597) bank116773 +116774 POINT(39.71934180482094 74.88652745204135) bank116774 +116775 POINT(39.83165458534216 74.78537905758667) bank116775 +116776 POINT(39.72806955159312 74.81140022599187) bank116776 +116777 POINT(40.3399975558841 73.90560130271317) bank116777 +116778 POINT(40.78961173884073 73.17053485154736) bank116778 +116779 POINT(41.2320811279333 73.70474051757436) bank116779 +116780 POINT(40.25874939363938 73.97489975143479) bank116780 +116781 POINT(39.779233107360845 73.56892844683736) bank116781 +116782 POINT(41.12806718278941 74.2009654244862) bank116782 +116783 POINT(41.18958562321541 74.96523430636645) bank116783 +116784 POINT(39.878334143188546 74.1746125144464) bank116784 +116785 POINT(41.67151266870115 74.29728079720279) bank116785 +116786 POINT(40.343722908301984 74.02086271853514) bank116786 +116787 POINT(40.031670967098314 74.05631917039231) bank116787 +116788 POINT(41.387474371716316 74.78057843196413) bank116788 +116789 POINT(41.24149633457316 74.8486600324199) bank116789 +116790 POINT(40.03563426140277 74.09189871572593) bank116790 +116791 POINT(40.62867383085646 73.79491496126236) bank116791 +116792 POINT(40.90060710515169 73.02551521602253) bank116792 +116793 POINT(40.590567277405995 74.75093035415054) bank116793 +116794 POINT(40.86075351287339 74.32999193598735) bank116794 +116795 POINT(41.27162885610943 74.75941413364276) bank116795 +116796 POINT(41.34765590976107 73.79703089089031) bank116796 +116797 POINT(41.04134611651594 73.38647650990433) bank116797 +116798 POINT(40.5037600705031 73.34948968508274) bank116798 +116799 POINT(40.50763446861533 73.68755657935313) bank116799 +116800 POINT(39.71548901085062 74.2843571961799) bank116800 +116801 POINT(41.00366395393023 73.17466881105088) bank116801 +116802 POINT(40.003521120151085 74.95243313610554) bank116802 +116803 POINT(41.06443712153066 74.48549135775791) bank116803 +116804 POINT(41.35447061828097 74.54798347986086) bank116804 +116805 POINT(41.07852808519511 73.43592275696011) bank116805 +116806 POINT(40.36618461602153 73.85134793341416) bank116806 +116807 POINT(40.13968969986579 73.08400306231529) bank116807 +116808 POINT(41.64831479493451 73.34595486083983) bank116808 +116809 POINT(39.89483136066555 73.51952205255778) bank116809 +116810 POINT(41.00583380359031 73.71533134079283) bank116810 +116811 POINT(41.08582222726923 74.19724777895036) bank116811 +116812 POINT(40.22121101488222 73.05446387750263) bank116812 +116813 POINT(41.13240066185177 74.16699448920343) bank116813 +116814 POINT(41.198714041452284 74.8330659108371) bank116814 +116815 POINT(40.93802323043817 73.03102745864933) bank116815 +116816 POINT(40.140049164927824 74.79228528007748) bank116816 +116817 POINT(41.0525674213221 74.9797759197391) bank116817 +116818 POINT(40.82780173613874 73.38156086459475) bank116818 +116819 POINT(40.07942467759498 74.07852858016832) bank116819 +116820 POINT(41.67381961070021 74.67813227469587) bank116820 +116821 POINT(40.812222441443105 73.15496303373955) bank116821 +116822 POINT(40.22602352281542 73.71449876567664) bank116822 +116823 POINT(41.41039301831342 73.19584029907128) bank116823 +116824 POINT(40.54226672388703 74.93712424811312) bank116824 +116825 POINT(40.35239379880308 74.67163716108261) bank116825 +116826 POINT(39.82953273014756 74.61758123446197) bank116826 +116827 POINT(41.07454299980619 74.37086451702821) bank116827 +116828 POINT(41.336703177986735 73.4300216022237) bank116828 +116829 POINT(40.097310011343204 74.39105246214183) bank116829 +116830 POINT(41.537885172591864 74.83177489987376) bank116830 +116831 POINT(41.64688734394122 73.49453252421668) bank116831 +116832 POINT(40.71017661480057 73.58409759433358) bank116832 +116833 POINT(41.614396034490326 73.145525100139) bank116833 +116834 POINT(41.15752221712039 74.83870410837996) bank116834 +116835 POINT(39.95194565310364 74.53208621782167) bank116835 +116836 POINT(40.95061963281996 74.79050272839795) bank116836 +116837 POINT(41.49397108882766 74.45586872822155) bank116837 +116838 POINT(39.89160959296155 73.42571398294714) bank116838 +116839 POINT(41.7057391998604 74.91837889411877) bank116839 +116840 POINT(41.32979500880837 73.65121338168622) bank116840 +116841 POINT(40.3245031267915 73.5669724608705) bank116841 +116842 POINT(40.83669955061588 74.31379214939842) bank116842 +116843 POINT(41.346421816841456 73.09402628761006) bank116843 +116844 POINT(39.823379071100824 74.43580270708381) bank116844 +116845 POINT(41.10845127620941 74.42378982338694) bank116845 +116846 POINT(40.01069829649131 73.61474644970556) bank116846 +116847 POINT(41.19439269809662 73.95341736596748) bank116847 +116848 POINT(40.35944633846457 73.00997961422367) bank116848 +116849 POINT(40.61502341209993 74.93383383919556) bank116849 +116850 POINT(40.82473791729095 74.10745643598888) bank116850 +116851 POINT(40.87238082103387 73.37147301766863) bank116851 +116852 POINT(39.83357980035414 73.4581800698946) bank116852 +116853 POINT(41.27994697743157 73.95552923630476) bank116853 +116854 POINT(40.74383730675785 74.01154166116663) bank116854 +116855 POINT(40.69936254802439 74.15535291034622) bank116855 +116856 POINT(40.093633260597194 74.7344443002944) bank116856 +116857 POINT(41.25776856032381 74.35983108030585) bank116857 +116858 POINT(41.63262313196149 74.67351353918215) bank116858 +116859 POINT(40.1506633142717 73.99091450436039) bank116859 +116860 POINT(41.47811198598129 74.8257072957704) bank116860 +116861 POINT(39.988332337053144 73.69321337192818) bank116861 +116862 POINT(39.96715836369464 73.59644401033177) bank116862 +116863 POINT(41.50061282476298 74.33075049556011) bank116863 +116864 POINT(40.45546744701573 74.78901981119147) bank116864 +116865 POINT(40.94422978303203 73.519223658575) bank116865 +116866 POINT(40.01452319263312 73.64177669700689) bank116866 +116867 POINT(40.471987511463844 74.53912240584725) bank116867 +116868 POINT(40.56278334457268 73.73839317650918) bank116868 +116869 POINT(39.901767750284904 73.43351207883295) bank116869 +116870 POINT(41.15597954027259 73.4486721873291) bank116870 +116871 POINT(40.07313209655516 74.29581791603697) bank116871 +116872 POINT(41.20709222871578 74.20656926068277) bank116872 +116873 POINT(39.775136073084035 73.08666344047506) bank116873 +116874 POINT(39.86590653376769 74.37910708761208) bank116874 +116875 POINT(40.694355238518824 74.46536896836946) bank116875 +116876 POINT(40.661890993602725 73.86680133666489) bank116876 +116877 POINT(39.82427214956704 73.80961448247463) bank116877 +116878 POINT(41.19438975148083 74.50363600310358) bank116878 +116879 POINT(40.558213325758985 73.32661809444457) bank116879 +116880 POINT(39.84129671496021 73.9680988780694) bank116880 +116881 POINT(40.683855534161346 74.13184271132768) bank116881 +116882 POINT(39.89240863441568 74.55385928648356) bank116882 +116883 POINT(41.47209506423426 73.255168388932) bank116883 +116884 POINT(39.92641719853429 74.6165477103644) bank116884 +116885 POINT(41.362697954549446 73.96249708462616) bank116885 +116886 POINT(40.66773470104589 73.88870573617919) bank116886 +116887 POINT(40.0193642627244 74.58086039095744) bank116887 +116888 POINT(41.30092512750249 74.61144989902421) bank116888 +116889 POINT(40.99188470103988 74.0294906658905) bank116889 +116890 POINT(39.718159490498465 73.61149783455569) bank116890 +116891 POINT(40.255081217939235 74.18684513428909) bank116891 +116892 POINT(41.495403557469174 73.08349018802573) bank116892 +116893 POINT(40.44075529036482 73.24024944869821) bank116893 +116894 POINT(40.48588211824947 73.95145550382492) bank116894 +116895 POINT(40.01356169673778 73.02288095730053) bank116895 +116896 POINT(41.17848061996639 73.1720720383428) bank116896 +116897 POINT(40.17723338474131 74.03298972666988) bank116897 +116898 POINT(40.26116354574578 73.61422378322816) bank116898 +116899 POINT(40.09059720730763 74.37225700053597) bank116899 +116900 POINT(40.28600733375672 73.47948096540821) bank116900 +116901 POINT(40.81372076469803 74.29001055466021) bank116901 +116902 POINT(41.11996260625371 74.7795381855842) bank116902 +116903 POINT(41.37869252939801 74.32636750960216) bank116903 +116904 POINT(40.050616987128706 73.14282043419968) bank116904 +116905 POINT(41.48630522757155 74.1709971640856) bank116905 +116906 POINT(41.04308706258021 74.55298892616631) bank116906 +116907 POINT(40.95860386611901 74.9634903119049) bank116907 +116908 POINT(41.41364124976539 74.1849816665242) bank116908 +116909 POINT(41.0971898483854 74.1174591637641) bank116909 +116910 POINT(41.68771751620956 73.98205413841536) bank116910 +116911 POINT(40.82232558896434 74.09426814984053) bank116911 +116912 POINT(40.45269239944606 74.7482170762851) bank116912 +116913 POINT(40.73382817447419 74.78475569957189) bank116913 +116914 POINT(40.301922212358846 73.71358514066995) bank116914 +116915 POINT(41.31249152170786 74.87767076370123) bank116915 +116916 POINT(39.92646105230233 73.7568404496646) bank116916 +116917 POINT(41.26163431663037 73.61151607167632) bank116917 +116918 POINT(41.353292323545524 74.41653875781861) bank116918 +116919 POINT(41.147232466146576 74.12613222534738) bank116919 +116920 POINT(39.76306179207812 73.1073967796064) bank116920 +116921 POINT(40.368977041025495 73.26854837054967) bank116921 +116922 POINT(40.09406239657258 74.84194603884983) bank116922 +116923 POINT(40.48379253827789 73.64260583479846) bank116923 +116924 POINT(40.377284351921816 73.1796391440112) bank116924 +116925 POINT(40.68402027019368 74.42587248935774) bank116925 +116926 POINT(40.700435922484665 73.26757452804965) bank116926 +116927 POINT(41.23870201472639 74.81319950177671) bank116927 +116928 POINT(40.421544516398725 74.32171165449503) bank116928 +116929 POINT(41.14065838985278 73.92499544995347) bank116929 +116930 POINT(39.94649556983448 73.49231991885057) bank116930 +116931 POINT(39.848156079476645 73.39041542881232) bank116931 +116932 POINT(39.82990761173561 74.35693763144774) bank116932 +116933 POINT(41.5153655424098 74.7091702243054) bank116933 +116934 POINT(39.75269158444685 74.11635327598074) bank116934 +116935 POINT(41.51062625778321 74.35394979789413) bank116935 +116936 POINT(41.33012945482007 74.96704163074149) bank116936 +116937 POINT(41.14697820113224 73.15230792050748) bank116937 +116938 POINT(40.117427697088885 74.7982568364183) bank116938 +116939 POINT(41.20535715039925 74.65685807921341) bank116939 +116940 POINT(40.622572683397856 74.42532545432076) bank116940 +116941 POINT(40.94377786059375 73.81230692401336) bank116941 +116942 POINT(40.23000201372079 73.88801608281847) bank116942 +116943 POINT(41.64873535122545 73.40223729578673) bank116943 +116944 POINT(41.319209491375254 73.5589674156822) bank116944 +116945 POINT(40.560158893914725 74.76332956986589) bank116945 +116946 POINT(41.585048965101606 73.5721143039932) bank116946 +116947 POINT(41.479732637161526 73.375822151794) bank116947 +116948 POINT(40.14614139719902 74.63770385672704) bank116948 +116949 POINT(39.88694069861964 73.97601761676178) bank116949 +116950 POINT(41.34671625572018 73.48189319241183) bank116950 +116951 POINT(41.45766420310607 74.64590203417427) bank116951 +116952 POINT(39.91292025969735 73.06528380497322) bank116952 +116953 POINT(40.437436094285275 73.06811359158912) bank116953 +116954 POINT(41.047631900118894 73.13435611831183) bank116954 +116955 POINT(40.61445942954529 74.00251122472756) bank116955 +116956 POINT(41.01318965794793 74.05811105629503) bank116956 +116957 POINT(41.351964560810295 74.6142095403576) bank116957 +116958 POINT(41.68563966792833 73.9729550828691) bank116958 +116959 POINT(40.11937083739993 73.01903805222133) bank116959 +116960 POINT(41.24056398412228 74.39350687001594) bank116960 +116961 POINT(39.991008615845956 74.5803988718547) bank116961 +116962 POINT(41.34727599675194 74.89895764799947) bank116962 +116963 POINT(40.209350794832545 73.79020847907759) bank116963 +116964 POINT(39.77594734003495 74.93085033139505) bank116964 +116965 POINT(39.99452913379607 74.53144052485737) bank116965 +116966 POINT(40.806409562245285 73.29638289887433) bank116966 +116967 POINT(40.58323697867534 74.09920641388283) bank116967 +116968 POINT(40.82436995544732 73.84307393825067) bank116968 +116969 POINT(41.697765943887866 73.22048045503996) bank116969 +116970 POINT(41.36179524495663 73.55114251921563) bank116970 +116971 POINT(41.06890028282812 73.35464134864588) bank116971 +116972 POINT(40.59820590879479 74.60458842791708) bank116972 +116973 POINT(40.09566705625039 74.79696825145182) bank116973 +116974 POINT(40.167578967985776 74.91875470273018) bank116974 +116975 POINT(40.11201184075689 73.50617054682729) bank116975 +116976 POINT(40.8826664653257 74.176345969932) bank116976 +116977 POINT(40.02023738837371 74.12188502221322) bank116977 +116978 POINT(39.95774916104185 74.41780795741195) bank116978 +116979 POINT(39.85411414153334 73.79271387622005) bank116979 +116980 POINT(39.8161043738906 73.32722560010603) bank116980 +116981 POINT(41.293851085586475 74.62897459707969) bank116981 +116982 POINT(40.036330663598015 74.5815503025627) bank116982 +116983 POINT(39.889039442453914 73.1590677434416) bank116983 +116984 POINT(41.446762303583746 73.57749455618642) bank116984 +116985 POINT(40.23790074912276 74.41822067732002) bank116985 +116986 POINT(41.13518059077847 73.63651693231158) bank116986 +116987 POINT(40.567663712592896 73.71518372269954) bank116987 +116988 POINT(40.89804537541357 74.38254948786135) bank116988 +116989 POINT(40.77241632282694 73.48819706643404) bank116989 +116990 POINT(41.684549428420084 75.00521436672707) bank116990 +116991 POINT(40.49084072041813 74.46221181812626) bank116991 +116992 POINT(41.3722914174927 73.08066746702977) bank116992 +116993 POINT(40.16568499451071 73.12763943958315) bank116993 +116994 POINT(40.525086275277495 73.52944756972016) bank116994 +116995 POINT(40.91357287515241 74.6336422953047) bank116995 +116996 POINT(41.316487945573805 74.19077353949999) bank116996 +116997 POINT(40.23076586606476 74.31355948628146) bank116997 +116998 POINT(41.44377215480806 74.32758988294133) bank116998 +116999 POINT(41.68024620594136 73.5959977782912) bank116999 +117000 POINT(40.38442955563647 74.31931131363305) bank117000 +117001 POINT(39.97828561817307 73.88023984777067) bank117001 +117002 POINT(40.29702653823904 73.15995225525373) bank117002 +117003 POINT(40.12381723425164 74.94607262269486) bank117003 +117004 POINT(39.98452383870079 73.08115789676485) bank117004 +117005 POINT(39.95807717150623 73.371422722459) bank117005 +117006 POINT(39.849080626604895 73.94925939112876) bank117006 +117007 POINT(41.01838059931965 73.03657912684385) bank117007 +117008 POINT(39.782145086959034 73.57454626432488) bank117008 +117009 POINT(41.346344544440086 73.3695066429651) bank117009 +117010 POINT(40.90112638373153 74.2290518600628) bank117010 +117011 POINT(39.737086907945596 74.24037961953547) bank117011 +117012 POINT(41.688154651951336 74.70561924455316) bank117012 +117013 POINT(39.8428277854218 73.77061378990089) bank117013 +117014 POINT(41.163908321822795 73.71124464635255) bank117014 +117015 POINT(40.67745645805731 73.52555767154993) bank117015 +117016 POINT(41.60077781239638 73.13701450677084) bank117016 +117017 POINT(40.24644198326124 73.0476287029141) bank117017 +117018 POINT(40.166032710648864 73.36107983330959) bank117018 +117019 POINT(41.33306171142933 73.05816678402701) bank117019 +117020 POINT(40.35321223142788 73.57561899180789) bank117020 +117021 POINT(39.90942989756784 73.58452589888856) bank117021 +117022 POINT(39.870424797366546 73.40516331703307) bank117022 +117023 POINT(40.03547777026944 74.26512028720501) bank117023 +117024 POINT(40.924556558994524 73.88290319943269) bank117024 +117025 POINT(40.012907924976005 73.23358825133569) bank117025 +117026 POINT(39.80032244418206 74.29054598333627) bank117026 +117027 POINT(40.36842125254773 74.44572113377986) bank117027 +117028 POINT(40.13716466633952 74.99208660988138) bank117028 +117029 POINT(40.527464352289016 74.0869866985147) bank117029 +117030 POINT(40.69916300371975 73.3161168635357) bank117030 +117031 POINT(40.179628787666076 74.69532009186179) bank117031 +117032 POINT(40.13077725351087 73.39822261339295) bank117032 +117033 POINT(41.403765308944045 73.21382974142698) bank117033 +117034 POINT(41.496584187692335 73.21106838469518) bank117034 +117035 POINT(41.11394203249715 74.92684907212029) bank117035 +117036 POINT(40.142935491481204 74.3186885295873) bank117036 +117037 POINT(40.20155338144119 74.80012554874448) bank117037 +117038 POINT(39.840351293364165 74.02829423362174) bank117038 +117039 POINT(41.638517344070074 73.81237813898463) bank117039 +117040 POINT(40.106840063366754 73.8224477003393) bank117040 +117041 POINT(41.40867875214094 73.48071346548973) bank117041 +117042 POINT(41.2954083992649 73.75357844110677) bank117042 +117043 POINT(40.17931873342927 73.50031762287092) bank117043 +117044 POINT(40.10777031164346 74.92719835211098) bank117044 +117045 POINT(41.627779254347914 73.29412120948926) bank117045 +117046 POINT(41.06288421292467 74.84555813707924) bank117046 +117047 POINT(40.49219273769517 74.091377713323) bank117047 +117048 POINT(40.82159338837585 73.44823182761905) bank117048 +117049 POINT(40.07029624285155 73.28477448882262) bank117049 +117050 POINT(40.19321348545667 74.56532870491468) bank117050 +117051 POINT(40.4423359919712 73.98968448328995) bank117051 +117052 POINT(41.32987331716213 74.87301420459104) bank117052 +117053 POINT(40.03823808890921 74.6550378138793) bank117053 +117054 POINT(41.02324182496372 73.85341472394089) bank117054 +117055 POINT(40.299372309843946 74.6386269711814) bank117055 +117056 POINT(40.15057656257322 74.18758244943173) bank117056 +117057 POINT(40.90160381801175 74.32200522841165) bank117057 +117058 POINT(40.75706178784336 74.68061303539382) bank117058 +117059 POINT(41.23512641358749 73.23743584194277) bank117059 +117060 POINT(40.07623995906148 73.28438803173744) bank117060 +117061 POINT(40.36155134465503 73.58930729588525) bank117061 +117062 POINT(39.908356671256286 74.0659979566802) bank117062 +117063 POINT(41.47732767812268 74.9431429916348) bank117063 +117064 POINT(41.05256008113256 73.35940483272536) bank117064 +117065 POINT(40.517997006970475 74.10957986074874) bank117065 +117066 POINT(41.50366205699451 74.05211292260945) bank117066 +117067 POINT(41.32078438504367 73.16354879493367) bank117067 +117068 POINT(41.1278562271254 74.28400805489652) bank117068 +117069 POINT(40.79529377990159 74.37070491441617) bank117069 +117070 POINT(39.943467205206325 74.91952629223171) bank117070 +117071 POINT(40.523797578465214 73.86520896102306) bank117071 +117072 POINT(40.594061161666794 74.49367622118334) bank117072 +117073 POINT(41.404109846635784 74.60009104546745) bank117073 +117074 POINT(40.34656340120845 74.99399706974161) bank117074 +117075 POINT(40.04063042134355 73.8273109072989) bank117075 +117076 POINT(40.583939203082075 73.99281277120649) bank117076 +117077 POINT(41.03192069855829 73.66622996939894) bank117077 +117078 POINT(40.81282184961938 73.14048598112082) bank117078 +117079 POINT(40.464764402201084 73.2781805855888) bank117079 +117080 POINT(41.412053575533584 74.96595877748847) bank117080 +117081 POINT(41.02411982608454 74.4934982049625) bank117081 +117082 POINT(40.0006810950238 73.54091169608398) bank117082 +117083 POINT(41.243304027699736 73.83274794452014) bank117083 +117084 POINT(40.473582097474484 73.60669696302911) bank117084 +117085 POINT(41.06891730765929 73.5240963352744) bank117085 +117086 POINT(40.71329171905583 73.09671735813357) bank117086 +117087 POINT(40.066159677338256 73.50293141541168) bank117087 +117088 POINT(40.71610151558688 74.41298021768874) bank117088 +117089 POINT(40.15055248826536 73.77123296017226) bank117089 +117090 POINT(41.21051875007569 73.5593727134651) bank117090 +117091 POINT(41.56905603296627 74.70888655464258) bank117091 +117092 POINT(41.62537907124245 73.30054744208935) bank117092 +117093 POINT(41.14998655824031 74.57743165387758) bank117093 +117094 POINT(40.60447148115983 74.90215257653733) bank117094 +117095 POINT(41.05760680397854 74.87749329744081) bank117095 +117096 POINT(40.33109636461446 73.26638571432564) bank117096 +117097 POINT(40.57083401767151 73.69609689212481) bank117097 +117098 POINT(39.78511553536617 74.25006494191732) bank117098 +117099 POINT(40.822727119318806 74.19306405740123) bank117099 +117100 POINT(41.61817923847259 74.29308239768534) bank117100 +117101 POINT(40.4227089964126 74.53358694522873) bank117101 +117102 POINT(39.99179306025304 73.25975280939812) bank117102 +117103 POINT(39.80279268561788 74.25233106217836) bank117103 +117104 POINT(39.7805377165635 74.20495236074346) bank117104 +117105 POINT(41.4911007931891 74.72545591159614) bank117105 +117106 POINT(40.38905458387847 74.38089043182765) bank117106 +117107 POINT(40.10906466068096 74.54097042242738) bank117107 +117108 POINT(41.142313736588605 73.48001269954239) bank117108 +117109 POINT(40.07860535759377 74.50994424186297) bank117109 +117110 POINT(40.05116394989926 74.61678824991475) bank117110 +117111 POINT(40.17590821351506 73.95479983943692) bank117111 +117112 POINT(40.85681582172559 73.44851222639744) bank117112 +117113 POINT(40.13365037745117 73.58002627362809) bank117113 +117114 POINT(40.32337521112613 74.48232735111141) bank117114 +117115 POINT(40.94897629986765 73.06022601083495) bank117115 +117116 POINT(41.49786183676536 73.46962992842668) bank117116 +117117 POINT(40.867896765182415 73.51510334009377) bank117117 +117118 POINT(41.70117753878973 74.04629371036097) bank117118 +117119 POINT(40.76550682156161 73.00822395523822) bank117119 +117120 POINT(40.94752435619842 73.26639207713816) bank117120 +117121 POINT(41.045585791492655 73.97554546861062) bank117121 +117122 POINT(40.74295856695696 73.49307066916005) bank117122 +117123 POINT(40.72450322620144 74.18183232870771) bank117123 +117124 POINT(40.15716032600688 73.20617313221462) bank117124 +117125 POINT(40.44731472548258 73.32343782836243) bank117125 +117126 POINT(40.45464280115206 74.61326548596476) bank117126 +117127 POINT(41.56356704418125 73.78040969228358) bank117127 +117128 POINT(41.17080757519804 74.1167702027426) bank117128 +117129 POINT(39.891167266510685 73.03573593502905) bank117129 +117130 POINT(40.778592652380546 73.82735368210155) bank117130 +117131 POINT(40.518379221966285 74.87513680752005) bank117131 +117132 POINT(40.474603425848855 74.95827356086194) bank117132 +117133 POINT(40.788714457368506 74.9976068632893) bank117133 +117134 POINT(39.9743670945491 74.84975702890434) bank117134 +117135 POINT(40.52673435139986 73.23068476429324) bank117135 +117136 POINT(40.05845849730472 73.45951600385101) bank117136 +117137 POINT(39.76493172267732 73.9172975753575) bank117137 +117138 POINT(40.273264712642884 73.60962259437446) bank117138 +117139 POINT(40.228896755113354 73.86998994954692) bank117139 +117140 POINT(41.02850698769433 74.66393271126749) bank117140 +117141 POINT(41.65883171304523 74.5035855474481) bank117141 +117142 POINT(41.5045688550479 74.83280839946732) bank117142 +117143 POINT(41.296046091700354 74.03648826813657) bank117143 +117144 POINT(41.052683819129705 73.99307678254095) bank117144 +117145 POINT(40.01266551325378 74.69024316936498) bank117145 +117146 POINT(41.31337064373287 74.0009765849652) bank117146 +117147 POINT(41.107680587483074 74.7734402420442) bank117147 +117148 POINT(39.78612870879743 73.05503244721177) bank117148 +117149 POINT(41.38574300404354 74.64391938396984) bank117149 +117150 POINT(40.655800974706985 74.18190215091872) bank117150 +117151 POINT(40.31441368360648 74.54475250365459) bank117151 +117152 POINT(40.31559605205234 73.83129717977519) bank117152 +117153 POINT(39.741756898891175 74.42234081110206) bank117153 +117154 POINT(41.415721190095006 74.34249910606282) bank117154 +117155 POINT(39.722472875024366 73.92527382150749) bank117155 +117156 POINT(39.97412458150834 73.14170429055746) bank117156 +117157 POINT(40.766806303217066 73.4346424362611) bank117157 +117158 POINT(40.9530291483208 74.38407946075428) bank117158 +117159 POINT(40.351062424861965 74.75689389701358) bank117159 +117160 POINT(41.12725634145782 73.89527139132049) bank117160 +117161 POINT(40.81136810965185 74.23481330766236) bank117161 +117162 POINT(40.91858335251074 73.4787759270338) bank117162 +117163 POINT(40.17090752990481 73.31774635048771) bank117163 +117164 POINT(40.22654584826352 74.96524637654127) bank117164 +117165 POINT(41.422447274453184 73.54438938984536) bank117165 +117166 POINT(40.21145007673228 74.18517515620454) bank117166 +117167 POINT(40.52509371083694 73.88130276403612) bank117167 +117168 POINT(39.952778836442846 73.24591169122958) bank117168 +117169 POINT(41.492172240105226 73.84248648960649) bank117169 +117170 POINT(40.109016659288955 74.50164463212994) bank117170 +117171 POINT(39.756565557410696 73.85497844946269) bank117171 +117172 POINT(39.72399448671238 74.07411935498966) bank117172 +117173 POINT(41.19087854246835 73.39587218828098) bank117173 +117174 POINT(40.8285300626612 74.40535139189721) bank117174 +117175 POINT(41.50624629101423 73.48388771714112) bank117175 +117176 POINT(41.44542785128975 73.80458221410852) bank117176 +117177 POINT(39.83695211239339 74.04669977014875) bank117177 +117178 POINT(40.208733908861156 73.24913929875406) bank117178 +117179 POINT(39.77291107689727 74.73700187729915) bank117179 +117180 POINT(41.199929623392606 74.60420310173286) bank117180 +117181 POINT(41.287049570349616 74.0544648002528) bank117181 +117182 POINT(40.307251061129676 73.07687786385242) bank117182 +117183 POINT(40.39100605065019 73.11715025035708) bank117183 +117184 POINT(40.322788665628565 73.72841920242826) bank117184 +117185 POINT(40.62399404666299 73.68784745215619) bank117185 +117186 POINT(40.58737945443178 74.4343197732771) bank117186 +117187 POINT(39.99027749575699 74.6400555163104) bank117187 +117188 POINT(40.44860924102784 73.59561359099035) bank117188 +117189 POINT(40.08088963228075 73.57140030151099) bank117189 +117190 POINT(40.218706132611 74.81032334657466) bank117190 +117191 POINT(41.18689132129182 73.99756253810571) bank117191 +117192 POINT(40.52167106846215 74.61384200492404) bank117192 +117193 POINT(41.71186235984805 73.21241505884423) bank117193 +117194 POINT(40.72266895775151 74.74195907214583) bank117194 +117195 POINT(40.13163417739755 73.55475842036809) bank117195 +117196 POINT(40.46120403689608 74.50998206302579) bank117196 +117197 POINT(40.9387619382731 73.22913776179195) bank117197 +117198 POINT(39.96281647191784 74.13837093973608) bank117198 +117199 POINT(40.191796226045476 74.85194605485007) bank117199 +117200 POINT(40.20282763032559 74.22294548439923) bank117200 +117201 POINT(41.170501676671776 73.93875871737461) bank117201 +117202 POINT(40.87450632343915 74.4815982909915) bank117202 +117203 POINT(40.50669363195002 74.33122364338068) bank117203 +117204 POINT(41.59252395628151 73.30290683658622) bank117204 +117205 POINT(41.361657513992554 73.83134845603016) bank117205 +117206 POINT(41.059277804823594 73.89079205039403) bank117206 +117207 POINT(40.19257757401974 74.81710819701763) bank117207 +117208 POINT(40.04722350885587 73.15746263121045) bank117208 +117209 POINT(40.609015421983045 73.5661403342746) bank117209 +117210 POINT(41.580014328700926 73.86386958144773) bank117210 +117211 POINT(40.91539359043525 73.23867520779906) bank117211 +117212 POINT(41.53612865947484 74.98936752818975) bank117212 +117213 POINT(41.58581353790941 74.58050981539141) bank117213 +117214 POINT(41.51598515292295 74.28446798337903) bank117214 +117215 POINT(40.53828362473206 73.33283510567699) bank117215 +117216 POINT(40.41225245047974 74.99307940802318) bank117216 +117217 POINT(40.84829786915937 74.51412154995903) bank117217 +117218 POINT(41.186339184600705 74.55478959946286) bank117218 +117219 POINT(40.36107238904168 74.54133809592113) bank117219 +117220 POINT(41.04177460692374 74.0926027843794) bank117220 +117221 POINT(40.43593244514231 74.0816998522723) bank117221 +117222 POINT(41.03408254201977 74.36807400804013) bank117222 +117223 POINT(41.30049900120488 74.68716113337754) bank117223 +117224 POINT(41.34159580691057 74.64516127332693) bank117224 +117225 POINT(40.98817927880316 73.67428130713847) bank117225 +117226 POINT(40.348135206360695 73.83998466326751) bank117226 +117227 POINT(40.11370438250508 74.23388623958618) bank117227 +117228 POINT(41.28542870708931 73.52944243180121) bank117228 +117229 POINT(40.857046691256706 73.89058707182322) bank117229 +117230 POINT(40.10204467679264 74.60645512707757) bank117230 +117231 POINT(41.07180683916406 74.80281527012122) bank117231 +117232 POINT(40.994124960915016 73.40330111532045) bank117232 +117233 POINT(40.08031629805037 74.95319443147038) bank117233 +117234 POINT(41.04379751298663 73.54789807358394) bank117234 +117235 POINT(40.05318737261598 74.76475118537802) bank117235 +117236 POINT(39.910689186914595 74.86825999371754) bank117236 +117237 POINT(40.45337803645478 74.33868289920088) bank117237 +117238 POINT(40.007235163602566 73.54693714654348) bank117238 +117239 POINT(41.52149735312031 73.08768555143158) bank117239 +117240 POINT(40.50360517220087 73.7689397124312) bank117240 +117241 POINT(39.946915336443 73.1094312457089) bank117241 +117242 POINT(40.67157030062783 74.32263582017802) bank117242 +117243 POINT(40.99357366808864 73.3676747196029) bank117243 +117244 POINT(40.20003991135026 73.68315308312519) bank117244 +117245 POINT(40.002953395736185 73.22376011004883) bank117245 +117246 POINT(41.40451332140212 74.61260691205635) bank117246 +117247 POINT(40.94990195996304 74.18436944938523) bank117247 +117248 POINT(41.020818593981254 73.52265717755068) bank117248 +117249 POINT(39.75050922353796 74.37363968792579) bank117249 +117250 POINT(40.626044063100665 74.44761615034292) bank117250 +117251 POINT(40.332764380946074 73.4941011283634) bank117251 +117252 POINT(39.7908674045168 73.47447743591644) bank117252 +117253 POINT(40.99489064594855 74.67482898816328) bank117253 +117254 POINT(41.04964787556196 74.28683537766646) bank117254 +117255 POINT(39.86898889304907 74.48267257248749) bank117255 +117256 POINT(41.21800279063396 74.39986050836221) bank117256 +117257 POINT(40.64919831189674 74.25614346285758) bank117257 +117258 POINT(41.330909878973706 73.35111667469222) bank117258 +117259 POINT(40.96879964734825 74.10601522626413) bank117259 +117260 POINT(41.50188205468156 73.31369656311361) bank117260 +117261 POINT(39.996054297385285 73.84611916667117) bank117261 +117262 POINT(39.79783366854384 73.00632775971225) bank117262 +117263 POINT(41.03389254840472 73.30812800444562) bank117263 +117264 POINT(41.2529340114408 73.9533814868722) bank117264 +117265 POINT(40.32363926032717 73.45020808218936) bank117265 +117266 POINT(40.42000416569626 74.43811497934132) bank117266 +117267 POINT(40.32027459005643 73.40601529927808) bank117267 +117268 POINT(40.19493597385393 73.02605602739139) bank117268 +117269 POINT(40.112297306110655 73.6880407311516) bank117269 +117270 POINT(41.301548949201376 73.01235147286648) bank117270 +117271 POINT(40.60706798447626 74.16157682167506) bank117271 +117272 POINT(40.0793896652906 73.07822757646855) bank117272 +117273 POINT(40.673226562132754 73.9077832363767) bank117273 +117274 POINT(40.37994391556882 73.94976935662122) bank117274 +117275 POINT(41.101592399361216 74.04740837593437) bank117275 +117276 POINT(41.187961553754135 74.7353708787447) bank117276 +117277 POINT(39.798584056160884 73.8563319009197) bank117277 +117278 POINT(39.760063451170936 74.4436721142248) bank117278 +117279 POINT(41.38966574314006 74.59944533823928) bank117279 +117280 POINT(41.34026329560801 73.81939868109825) bank117280 +117281 POINT(39.80216263456819 74.58305162698895) bank117281 +117282 POINT(39.79736813492579 74.97692699819957) bank117282 +117283 POINT(39.98100965485725 73.21985561052325) bank117283 +117284 POINT(41.5119086156586 73.28847070259393) bank117284 +117285 POINT(41.37532825121173 74.27121935973177) bank117285 +117286 POINT(40.10331419804351 73.73070263396072) bank117286 +117287 POINT(41.51265976774847 73.23898663204048) bank117287 +117288 POINT(40.116706888982485 73.73068118064498) bank117288 +117289 POINT(41.15937267024052 74.84992406589657) bank117289 +117290 POINT(40.59763339083129 74.69362174800753) bank117290 +117291 POINT(40.254841449744525 74.90980654610914) bank117291 +117292 POINT(40.973075348724244 74.99337153549745) bank117292 +117293 POINT(41.06615865742164 74.96541463544102) bank117293 +117294 POINT(39.9369725639789 74.47782556426678) bank117294 +117295 POINT(39.91234904073415 73.36822523172866) bank117295 +117296 POINT(41.28592983225122 74.025651528827) bank117296 +117297 POINT(40.942573693846974 74.02716492721251) bank117297 +117298 POINT(41.203833452686176 73.52131369529039) bank117298 +117299 POINT(40.97174522393831 74.43474405968566) bank117299 +117300 POINT(40.38483443559734 74.97530435783757) bank117300 +117301 POINT(40.64697256596113 73.1476062949982) bank117301 +117302 POINT(40.11512018769287 73.3865382169456) bank117302 +117303 POINT(40.44164376447599 74.54359599368674) bank117303 +117304 POINT(40.42406614882279 74.99993691410607) bank117304 +117305 POINT(41.00656311911934 74.44761235329533) bank117305 +117306 POINT(41.344271767576295 73.93140890894269) bank117306 +117307 POINT(40.458290279004075 73.98320240263902) bank117307 +117308 POINT(41.09678811244623 73.80371998166223) bank117308 +117309 POINT(40.66846492730369 74.28331365733101) bank117309 +117310 POINT(40.83131675560788 74.22664355000111) bank117310 +117311 POINT(40.60985969124888 74.00902380387548) bank117311 +117312 POINT(41.054258473689956 74.53646433496417) bank117312 +117313 POINT(40.733360114709704 74.9511503357305) bank117313 +117314 POINT(40.97528137950434 74.87732168925766) bank117314 +117315 POINT(41.38061655190255 74.39505112116622) bank117315 +117316 POINT(41.65466492625342 73.16210592490467) bank117316 +117317 POINT(40.507981485761384 74.28494467266228) bank117317 +117318 POINT(40.8696176850628 74.69369119888454) bank117318 +117319 POINT(40.2389110562988 73.21867296942857) bank117319 +117320 POINT(40.65705341605844 73.68343443616995) bank117320 +117321 POINT(41.11154701572761 74.09823517660972) bank117321 +117322 POINT(40.319848069264296 74.70730664527458) bank117322 +117323 POINT(39.83411985539375 74.55514565818622) bank117323 +117324 POINT(40.39333337099496 74.69971669612549) bank117324 +117325 POINT(40.666600985126834 74.10253174208233) bank117325 +117326 POINT(40.73922770584298 73.38575984251754) bank117326 +117327 POINT(40.358828819378154 73.68329831888092) bank117327 +117328 POINT(40.823846202938725 73.0932766107415) bank117328 +117329 POINT(40.24409310147609 73.6488312004819) bank117329 +117330 POINT(40.85912660400127 73.23593958307531) bank117330 +117331 POINT(40.20483705816345 73.2826054368188) bank117331 +117332 POINT(39.857981689111305 73.76160201619196) bank117332 +117333 POINT(40.7860085143615 73.07492902459859) bank117333 +117334 POINT(39.87646102691707 74.71749762701113) bank117334 +117335 POINT(40.48126319572489 74.06501082075398) bank117335 +117336 POINT(40.70711174938601 73.30281265869019) bank117336 +117337 POINT(40.410404065133434 74.05654785987679) bank117337 +117338 POINT(40.0277840408201 73.21220061879) bank117338 +117339 POINT(40.49266261980672 73.40278469358513) bank117339 +117340 POINT(41.10062256873821 73.8543320408227) bank117340 +117341 POINT(41.59853771601154 73.884030046531) bank117341 +117342 POINT(41.513266190180865 73.33674459989281) bank117342 +117343 POINT(40.29294991292317 73.44085264565736) bank117343 +117344 POINT(41.10490465958212 74.64081219015758) bank117344 +117345 POINT(40.40927111745957 73.98755245618062) bank117345 +117346 POINT(41.54499237851597 74.78851206937757) bank117346 +117347 POINT(41.31635073321077 73.62822884199963) bank117347 +117348 POINT(40.85560763644994 74.00657968533487) bank117348 +117349 POINT(40.62729201062265 73.26869664999397) bank117349 +117350 POINT(40.597833550677166 74.2719002535805) bank117350 +117351 POINT(41.31860031291241 74.79837458752813) bank117351 +117352 POINT(41.53976576270928 74.02489902830062) bank117352 +117353 POINT(40.76030361497097 74.35697837903601) bank117353 +117354 POINT(40.61462454376581 74.34035779602728) bank117354 +117355 POINT(40.54619943099574 74.71037049034733) bank117355 +117356 POINT(40.83175693877878 74.6906308028427) bank117356 +117357 POINT(41.10648202864092 74.02425535106875) bank117357 +117358 POINT(40.70794342992416 74.58022100843714) bank117358 +117359 POINT(39.73163641570718 73.9326294289673) bank117359 +117360 POINT(40.50021110479419 74.21797024138029) bank117360 +117361 POINT(39.84174461567931 74.58140104755982) bank117361 +117362 POINT(40.31295418340753 74.71317116230041) bank117362 +117363 POINT(40.70762385332806 73.04715196463557) bank117363 +117364 POINT(41.46147893577755 74.93052163757018) bank117364 +117365 POINT(40.813074514711815 73.8599776356539) bank117365 +117366 POINT(40.44167481984861 73.7377200141804) bank117366 +117367 POINT(39.857055282765685 74.03235322251223) bank117367 +117368 POINT(40.41809443783767 74.38040419209817) bank117368 +117369 POINT(41.08284639315679 73.0713699915627) bank117369 +117370 POINT(39.99729257779151 73.81256812823928) bank117370 +117371 POINT(40.51270308922135 73.48357162085667) bank117371 +117372 POINT(40.52444849386377 74.8929887341389) bank117372 +117373 POINT(40.63527040894183 73.0226100911944) bank117373 +117374 POINT(40.431338964884894 74.09213799688925) bank117374 +117375 POINT(40.477482286927234 73.45800009285027) bank117375 +117376 POINT(40.82616577722095 74.53785113363836) bank117376 +117377 POINT(41.052300525753786 74.7267837121818) bank117377 +117378 POINT(40.751836511499306 73.4906853708216) bank117378 +117379 POINT(41.55725523656099 74.38205931499793) bank117379 +117380 POINT(41.16329028339577 73.16404235264514) bank117380 +117381 POINT(41.52818378577975 74.27851145215907) bank117381 +117382 POINT(41.09802416266554 73.07854947036387) bank117382 +117383 POINT(40.195593484556625 73.37529566067207) bank117383 +117384 POINT(40.788799251715126 74.6680791980912) bank117384 +117385 POINT(40.77281714537749 74.8825574366472) bank117385 +117386 POINT(39.874504734999604 74.73613904511798) bank117386 +117387 POINT(40.39728890727612 73.18358008794762) bank117387 +117388 POINT(41.50717669761337 74.34126869017042) bank117388 +117389 POINT(40.09275778102237 74.95611677730508) bank117389 +117390 POINT(39.84802597572129 74.81964495614227) bank117390 +117391 POINT(40.90623322056786 74.22054129279327) bank117391 +117392 POINT(41.15399526490909 73.53897249227063) bank117392 +117393 POINT(41.459170527125316 73.14411055208998) bank117393 +117394 POINT(41.70939503433558 74.76114974722867) bank117394 +117395 POINT(40.76330395327983 74.69030636210123) bank117395 +117396 POINT(41.27307030493811 73.11692022293764) bank117396 +117397 POINT(39.78079730736108 74.60426522332575) bank117397 +117398 POINT(41.56433157499489 74.96334002571895) bank117398 +117399 POINT(40.86249366207325 73.61384939342469) bank117399 +117400 POINT(39.88286320574774 73.89932446256944) bank117400 +117401 POINT(39.80410237145033 74.7815760517245) bank117401 +117402 POINT(41.31940699579557 73.83692002121428) bank117402 +117403 POINT(41.272238683128705 74.71694646036013) bank117403 +117404 POINT(40.13496209313594 73.10016195872456) bank117404 +117405 POINT(40.59468692677606 73.53554991561592) bank117405 +117406 POINT(41.374077457456075 73.85986734313009) bank117406 +117407 POINT(41.35262396540999 74.37207456670549) bank117407 +117408 POINT(40.270614228978616 74.44626650968979) bank117408 +117409 POINT(41.611597326251065 74.75390166622361) bank117409 +117410 POINT(41.679506184675674 74.03687301535119) bank117410 +117411 POINT(40.47310895894999 74.91673274411315) bank117411 +117412 POINT(40.6407963388363 74.21568863428048) bank117412 +117413 POINT(41.333588047940054 74.8943833435593) bank117413 +117414 POINT(39.90717072684923 74.40140596883094) bank117414 +117415 POINT(39.74831772905411 73.05238767074034) bank117415 +117416 POINT(41.06932975318125 73.50606218412157) bank117416 +117417 POINT(41.1599867008029 73.78792549649008) bank117417 +117418 POINT(40.500783977445835 74.4182769135053) bank117418 +117419 POINT(40.86605617194632 73.46103718714842) bank117419 +117420 POINT(41.54576898302139 73.96691539853741) bank117420 +117421 POINT(39.79255459799172 74.02903898090847) bank117421 +117422 POINT(40.650389120932466 74.55268493245944) bank117422 +117423 POINT(40.84756723608649 74.1233860558471) bank117423 +117424 POINT(41.42899631159092 73.81770019341893) bank117424 +117425 POINT(40.44695482232772 74.54627189329274) bank117425 +117426 POINT(40.138252931415295 74.79927912998758) bank117426 +117427 POINT(41.66352969913048 73.65658701579102) bank117427 +117428 POINT(39.94777227935202 73.28819484605033) bank117428 +117429 POINT(41.68055015142276 73.44294049310275) bank117429 +117430 POINT(39.90719963151046 74.78154200353221) bank117430 +117431 POINT(41.15934947182819 74.03101897494034) bank117431 +117432 POINT(40.00389393192168 74.4972036425536) bank117432 +117433 POINT(40.168094320849804 73.38692258913234) bank117433 +117434 POINT(41.165743138683226 74.79437537787015) bank117434 +117435 POINT(41.071408258016184 73.97795268626892) bank117435 +117436 POINT(41.382331985333565 74.61945538108426) bank117436 +117437 POINT(39.870656342142084 73.07240970293115) bank117437 +117438 POINT(39.93298174035357 73.65903189862108) bank117438 +117439 POINT(40.70040824892346 73.56127954607537) bank117439 +117440 POINT(41.40652295599923 73.62634492844366) bank117440 +117441 POINT(39.77957126147474 74.8098296733274) bank117441 +117442 POINT(40.12732671406066 73.48554405643647) bank117442 +117443 POINT(39.960907110981765 73.618992853496) bank117443 +117444 POINT(40.40979545711265 74.3310375730074) bank117444 +117445 POINT(39.74095643710011 73.56934736099636) bank117445 +117446 POINT(40.767751871407846 74.17977935616895) bank117446 +117447 POINT(40.26588971193411 74.2178100834595) bank117447 +117448 POINT(40.125452345128984 73.45650950639877) bank117448 +117449 POINT(40.71100155456157 73.06967687535021) bank117449 +117450 POINT(41.32444597157084 73.37826467220508) bank117450 +117451 POINT(40.93332042399835 74.91286991123145) bank117451 +117452 POINT(41.473288915352974 74.04626306562025) bank117452 +117453 POINT(41.516055336632384 74.64067703274429) bank117453 +117454 POINT(39.954241436793346 74.52334773484557) bank117454 +117455 POINT(40.95368925070592 73.33704851193444) bank117455 +117456 POINT(40.78313818062465 74.02749486370371) bank117456 +117457 POINT(41.122299657414935 73.14809810042972) bank117457 +117458 POINT(40.30289274327471 74.82127972377386) bank117458 +117459 POINT(40.45287723203246 74.57174964497912) bank117459 +117460 POINT(40.478595735857596 73.49765788614444) bank117460 +117461 POINT(40.10693985477948 74.81532770498383) bank117461 +117462 POINT(40.43359942584106 74.10740495512027) bank117462 +117463 POINT(40.11321123924429 73.62779054035619) bank117463 +117464 POINT(41.46743180423229 73.05446507658539) bank117464 +117465 POINT(40.903702000696185 73.53949909430169) bank117465 +117466 POINT(40.78020054493924 73.43253714019156) bank117466 +117467 POINT(39.77346383520045 74.2230578823473) bank117467 +117468 POINT(40.882641035197736 73.31154157790637) bank117468 +117469 POINT(41.100637178886224 73.07880554362183) bank117469 +117470 POINT(40.19074028867964 74.5966940239669) bank117470 +117471 POINT(40.121227261717166 74.84245061813742) bank117471 +117472 POINT(40.61906113079657 74.95193721768017) bank117472 +117473 POINT(40.890286057385836 73.27791646174795) bank117473 +117474 POINT(40.4411894743235 74.73181254537934) bank117474 +117475 POINT(41.012891180024326 73.10441309993476) bank117475 +117476 POINT(39.91550794503205 74.20227142143978) bank117476 +117477 POINT(41.32604129645089 73.95650740395385) bank117477 +117478 POINT(41.648079495456365 74.62046486701921) bank117478 +117479 POINT(40.763770526287985 74.99260381856757) bank117479 +117480 POINT(41.0015236643787 73.05198800591876) bank117480 +117481 POINT(41.03212635929634 74.85139583984484) bank117481 +117482 POINT(41.46694999286828 74.07199346316527) bank117482 +117483 POINT(40.43355967206122 74.80916259793383) bank117483 +117484 POINT(40.72520386485124 73.29591286903145) bank117484 +117485 POINT(40.96891814110975 73.2264919991747) bank117485 +117486 POINT(40.91528796319195 74.9480949757166) bank117486 +117487 POINT(41.09165115658678 73.67888116344277) bank117487 +117488 POINT(41.36262348784232 74.8502652202886) bank117488 +117489 POINT(40.17431756438318 74.1896682868598) bank117489 +117490 POINT(40.93546140637023 73.387020581153) bank117490 +117491 POINT(39.893129401589434 73.15468432917133) bank117491 +117492 POINT(39.8735208576545 74.97637105650973) bank117492 +117493 POINT(40.09010587349601 73.05085754301291) bank117493 +117494 POINT(41.61006059781301 74.86721944545995) bank117494 +117495 POINT(40.18497135829242 74.5029198275118) bank117495 +117496 POINT(40.65923424123459 74.78072812038103) bank117496 +117497 POINT(40.398293026553254 73.08413100869886) bank117497 +117498 POINT(41.31418067579808 73.41403582955229) bank117498 +117499 POINT(39.92994864682132 74.76571015600268) bank117499 +117500 POINT(39.94385954587012 73.59349213107984) bank117500 +117501 POINT(40.9743220342335 73.49030475523348) bank117501 +117502 POINT(41.673295596567286 74.46728433873668) bank117502 +117503 POINT(40.68285168323375 74.77042548399696) bank117503 +117504 POINT(41.26705740285709 74.39286226662881) bank117504 +117505 POINT(40.90862329683132 73.1991217167143) bank117505 +117506 POINT(41.44649989741966 74.11397250935323) bank117506 +117507 POINT(39.959555878255685 74.94818789492003) bank117507 +117508 POINT(40.73549375331439 73.57831506682064) bank117508 +117509 POINT(40.12478835194325 73.0522240743535) bank117509 +117510 POINT(40.65861859240908 73.16180200852008) bank117510 +117511 POINT(41.24292441678053 73.36680454768752) bank117511 +117512 POINT(39.836139172296 73.87558707240592) bank117512 +117513 POINT(41.71084113226114 73.59457904068226) bank117513 +117514 POINT(39.86815491817597 73.14493167165654) bank117514 +117515 POINT(41.3803709398842 73.228544042359) bank117515 +117516 POINT(40.62762133324528 74.79369845112095) bank117516 +117517 POINT(41.26883099148255 74.30477540790348) bank117517 +117518 POINT(39.88188393570291 73.26074503980308) bank117518 +117519 POINT(41.102353359009854 73.35369696819338) bank117519 +117520 POINT(39.99754021108157 74.62047354753253) bank117520 +117521 POINT(40.76275670990102 73.76362315457689) bank117521 +117522 POINT(41.4413651930339 73.06276541108467) bank117522 +117523 POINT(41.24514687720239 73.15414867492227) bank117523 +117524 POINT(40.132124711698225 74.76132540433346) bank117524 +117525 POINT(40.9209020497829 74.37146400810063) bank117525 +117526 POINT(40.898237705172804 74.64810327652982) bank117526 +117527 POINT(40.77898157982906 73.64660312445682) bank117527 +117528 POINT(40.804400252054755 74.43587357552647) bank117528 +117529 POINT(39.82350517509143 73.98441682627514) bank117529 +117530 POINT(41.60862114244145 73.31579050759724) bank117530 +117531 POINT(41.297950511653276 74.74608057172982) bank117531 +117532 POINT(39.99739584143216 73.73592589685441) bank117532 +117533 POINT(39.97085123506976 74.47877107863651) bank117533 +117534 POINT(40.08829075003497 73.08691040948587) bank117534 +117535 POINT(40.3303939004328 73.52061006486895) bank117535 +117536 POINT(40.69753345771492 74.08828904670801) bank117536 +117537 POINT(40.66410831151315 73.65696403764368) bank117537 +117538 POINT(40.52960446822723 74.55283182238315) bank117538 +117539 POINT(41.566252289743176 73.01770443811392) bank117539 +117540 POINT(40.10346691706604 74.95503429536618) bank117540 +117541 POINT(40.357841331687546 73.58130012056145) bank117541 +117542 POINT(40.72034531914569 73.68555768152736) bank117542 +117543 POINT(40.36798008294502 73.07591945594598) bank117543 +117544 POINT(41.66229348601738 73.721667928764) bank117544 +117545 POINT(40.26792312556897 73.06890748555652) bank117545 +117546 POINT(41.3604957018625 74.26907800780306) bank117546 +117547 POINT(40.00283892647909 73.98024735224459) bank117547 +117548 POINT(40.0625076540962 74.32088422975761) bank117548 +117549 POINT(40.108480655759315 73.37800222499028) bank117549 +117550 POINT(41.33037367354841 74.67412472085253) bank117550 +117551 POINT(41.68487111335482 74.01794780023259) bank117551 +117552 POINT(41.502249102332996 74.89633821327578) bank117552 +117553 POINT(40.63921126345981 74.68283502914018) bank117553 +117554 POINT(39.7627725777727 73.60232806388912) bank117554 +117555 POINT(41.309712542724064 73.83554676325315) bank117555 +117556 POINT(40.03708142149027 74.9538421201138) bank117556 +117557 POINT(41.37495542468119 74.98229808248473) bank117557 +117558 POINT(40.489468857961874 74.19844549250577) bank117558 +117559 POINT(41.371456030234185 74.92569690123149) bank117559 +117560 POINT(40.860217934582096 74.52015785144978) bank117560 +117561 POINT(39.779276525299245 74.97366572791894) bank117561 +117562 POINT(40.420392350954316 74.58523217146845) bank117562 +117563 POINT(41.173575264476355 74.21036010253252) bank117563 +117564 POINT(40.546710255436736 74.36851791477535) bank117564 +117565 POINT(41.693277130291136 73.95025028145469) bank117565 +117566 POINT(40.43949957619675 73.13940125580548) bank117566 +117567 POINT(41.62031469363431 73.50890254390654) bank117567 +117568 POINT(39.95676759877591 73.64452613612865) bank117568 +117569 POINT(40.903269989369925 74.3780879432644) bank117569 +117570 POINT(40.03628785480656 73.98849451202354) bank117570 +117571 POINT(40.29893926638048 74.43002368433594) bank117571 +117572 POINT(41.19355572485757 74.63020814049206) bank117572 +117573 POINT(41.3256230423815 74.39375526891327) bank117573 +117574 POINT(41.42121728455571 73.33266177914281) bank117574 +117575 POINT(40.114318079474316 73.30363465161307) bank117575 +117576 POINT(40.53517493501069 73.55305131683136) bank117576 +117577 POINT(40.45735183787984 73.18745173294471) bank117577 +117578 POINT(41.69884868989946 73.93058841257215) bank117578 +117579 POINT(41.577261697282815 73.85242091884008) bank117579 +117580 POINT(39.82037512659716 74.07184414945588) bank117580 +117581 POINT(41.47534355465487 74.24351184848005) bank117581 +117582 POINT(39.76628804627345 74.68550965514872) bank117582 +117583 POINT(39.95258596212262 74.47231991527076) bank117583 +117584 POINT(40.63501073838166 74.30738236761604) bank117584 +117585 POINT(40.90933202010085 74.25378099892221) bank117585 +117586 POINT(39.830391940866804 74.02867359364869) bank117586 +117587 POINT(40.742124388709684 73.8151526176168) bank117587 +117588 POINT(40.04595498152968 73.62313387534938) bank117588 +117589 POINT(41.22741930394157 74.95892202956166) bank117589 +117590 POINT(40.99865330526151 73.8191400317127) bank117590 +117591 POINT(40.654479584305925 74.71338231533534) bank117591 +117592 POINT(41.20687439718131 74.73917692062773) bank117592 +117593 POINT(41.69921786591737 73.94842418929724) bank117593 +117594 POINT(40.921053364322105 73.67176108874264) bank117594 +117595 POINT(41.352497664389695 73.29698014119023) bank117595 +117596 POINT(40.40727066093099 74.6610998675737) bank117596 +117597 POINT(40.880369309826065 74.22138457550929) bank117597 +117598 POINT(39.92904274150222 74.73276375383324) bank117598 +117599 POINT(40.512116122640556 74.44532951693638) bank117599 +117600 POINT(40.77338096770438 73.67117729332554) bank117600 +117601 POINT(40.289315735296206 73.28027750517293) bank117601 +117602 POINT(40.00578283490383 73.3515337805619) bank117602 +117603 POINT(40.80096562984582 74.68583313275558) bank117603 +117604 POINT(40.03059036010842 74.1159113106987) bank117604 +117605 POINT(41.310552881890885 74.77235748264111) bank117605 +117606 POINT(40.1516355294765 74.29116623343587) bank117606 +117607 POINT(39.8947435593671 73.75337915813773) bank117607 +117608 POINT(40.04550795876341 74.35652973780184) bank117608 +117609 POINT(40.8802166772341 74.99211983173768) bank117609 +117610 POINT(40.79566779889203 73.72640995317713) bank117610 +117611 POINT(39.74638181101621 74.6694251256538) bank117611 +117612 POINT(39.770737688529856 74.14761043520758) bank117612 +117613 POINT(40.350015888592495 74.35289097937924) bank117613 +117614 POINT(41.64238992998025 73.19034411787159) bank117614 +117615 POINT(40.82581829335054 74.23989951563239) bank117615 +117616 POINT(41.37990788109097 73.6237177713684) bank117616 +117617 POINT(40.242370968723094 74.27907656252285) bank117617 +117618 POINT(40.40546335072585 73.52300893587511) bank117618 +117619 POINT(40.512592943404286 73.02654440058888) bank117619 +117620 POINT(40.2553907188146 73.05839084184475) bank117620 +117621 POINT(40.79420793705376 74.74296360838129) bank117621 +117622 POINT(40.431703636137414 73.3475952320737) bank117622 +117623 POINT(40.33264931595731 73.5757538919398) bank117623 +117624 POINT(40.73205420160944 74.52816281679847) bank117624 +117625 POINT(41.66481980509834 73.37865434625577) bank117625 +117626 POINT(40.40275341824563 74.53940310848037) bank117626 +117627 POINT(40.100139264447556 73.24060981508076) bank117627 +117628 POINT(40.60860636841121 74.35234574467673) bank117628 +117629 POINT(40.86900853769932 73.57185424611015) bank117629 +117630 POINT(39.83502004826707 73.11706131843556) bank117630 +117631 POINT(41.255425006629835 73.8539542636234) bank117631 +117632 POINT(41.488857577711165 74.88123486766476) bank117632 +117633 POINT(40.3579660808423 74.28798347263904) bank117633 +117634 POINT(40.06287454781186 73.04051870706179) bank117634 +117635 POINT(41.272148505727884 73.54307950680226) bank117635 +117636 POINT(40.36554614262264 74.6997601079825) bank117636 +117637 POINT(40.01044171298619 73.45307051420251) bank117637 +117638 POINT(41.6010579460498 74.2298797690372) bank117638 +117639 POINT(39.920221899563174 73.37937123754989) bank117639 +117640 POINT(41.237191494489586 74.88398222463113) bank117640 +117641 POINT(41.316571333634165 74.1926675718058) bank117641 +117642 POINT(40.77987734284631 74.9550929096103) bank117642 +117643 POINT(39.785330574586204 73.64995679447998) bank117643 +117644 POINT(40.864111544953346 73.84200388371133) bank117644 +117645 POINT(41.70762211135983 74.51124852462698) bank117645 +117646 POINT(41.39021699994647 74.69153651466229) bank117646 +117647 POINT(41.150123353070015 75.00270024807963) bank117647 +117648 POINT(41.64538877826852 73.54965765797625) bank117648 +117649 POINT(40.14307957378721 74.78086725879363) bank117649 +117650 POINT(40.01534060256087 74.24609704197134) bank117650 +117651 POINT(40.82776086094943 74.43603619030046) bank117651 +117652 POINT(39.87590672837333 74.63032163123046) bank117652 +117653 POINT(41.636070359313784 73.03108205393029) bank117653 +117654 POINT(39.887434833797506 74.50757469845608) bank117654 +117655 POINT(41.64064963118574 73.47828453796917) bank117655 +117656 POINT(41.289387944886975 73.02026269826551) bank117656 +117657 POINT(40.796932564449385 74.70415378109212) bank117657 +117658 POINT(41.51029853797824 74.50360288225228) bank117658 +117659 POINT(41.133596579716006 73.89878897856491) bank117659 +117660 POINT(40.9963371602482 73.5151903479181) bank117660 +117661 POINT(39.88171127487487 73.39284298566649) bank117661 +117662 POINT(40.1666578362974 73.28931243310288) bank117662 +117663 POINT(41.684543003321934 73.9279061688426) bank117663 +117664 POINT(41.48256074364373 73.97614748992812) bank117664 +117665 POINT(40.36668725219974 73.74276046271095) bank117665 +117666 POINT(40.580818189926035 73.36796391524354) bank117666 +117667 POINT(39.72182842484406 73.971420850892) bank117667 +117668 POINT(40.052219929633175 74.22245006704864) bank117668 +117669 POINT(40.97833753585204 73.32705099906165) bank117669 +117670 POINT(41.11168389517522 73.52015000234606) bank117670 +117671 POINT(40.092449865182694 74.90112260662123) bank117671 +117672 POINT(39.85721908959866 73.27958868367935) bank117672 +117673 POINT(40.417311146428254 73.21625909077456) bank117673 +117674 POINT(41.15578800253715 74.66451302824383) bank117674 +117675 POINT(41.249166415560225 73.74776329272018) bank117675 +117676 POINT(39.78385428828651 73.29409032052688) bank117676 +117677 POINT(41.00128692405268 73.04156050236816) bank117677 +117678 POINT(40.66535521045453 74.00684433113241) bank117678 +117679 POINT(40.19288308229543 73.73458253543374) bank117679 +117680 POINT(40.50509130813678 73.19568029658677) bank117680 +117681 POINT(39.774163384750956 74.19647290978199) bank117681 +117682 POINT(40.96323306391897 73.21877505002797) bank117682 +117683 POINT(40.667540250956826 73.46084980659727) bank117683 +117684 POINT(41.614210897408604 74.67499192640078) bank117684 +117685 POINT(39.94289541179241 74.24446840583776) bank117685 +117686 POINT(40.036271285884006 73.69056575864703) bank117686 +117687 POINT(40.684955101575056 73.3790615907577) bank117687 +117688 POINT(40.36528702152995 73.95177280387425) bank117688 +117689 POINT(41.41444199920841 73.50427616132825) bank117689 +117690 POINT(40.36775671726103 73.59081360986129) bank117690 +117691 POINT(40.59887455479395 74.39465351829554) bank117691 +117692 POINT(40.475373541194394 73.07293404883914) bank117692 +117693 POINT(41.43398596455895 74.15251957301852) bank117693 +117694 POINT(40.898868794273525 74.0738787577749) bank117694 +117695 POINT(39.89768218652995 73.09702529612508) bank117695 +117696 POINT(39.86092843821316 73.80430778114815) bank117696 +117697 POINT(40.53548217424264 73.20205021116855) bank117697 +117698 POINT(41.09027586810097 74.35598022656477) bank117698 +117699 POINT(41.2743973762425 74.21477026173956) bank117699 +117700 POINT(39.99776760752092 74.85213580528429) bank117700 +117701 POINT(41.17049981265827 74.74057182590074) bank117701 +117702 POINT(41.557820523182144 73.66492540174153) bank117702 +117703 POINT(40.30342831330622 74.71090499468204) bank117703 +117704 POINT(41.459742046226424 74.21730437337939) bank117704 +117705 POINT(40.38053458781225 73.80279034305812) bank117705 +117706 POINT(41.63038906248851 73.74340056160406) bank117706 +117707 POINT(39.967502650597005 73.3284507039614) bank117707 +117708 POINT(40.93654476608849 74.32584919246186) bank117708 +117709 POINT(39.99880011891817 73.63731738634776) bank117709 +117710 POINT(40.55224672375345 74.61268135161323) bank117710 +117711 POINT(41.22076998978057 74.54482657593232) bank117711 +117712 POINT(41.35152019658612 74.39399954688056) bank117712 +117713 POINT(41.58407619713015 74.92516234397087) bank117713 +117714 POINT(40.23882246613021 73.54110830883552) bank117714 +117715 POINT(40.642315139081916 73.58415179461076) bank117715 +117716 POINT(40.74146738715744 74.809961690683) bank117716 +117717 POINT(40.17754184969904 74.94781238737721) bank117717 +117718 POINT(40.414445056399515 73.39232799317931) bank117718 +117719 POINT(40.4768997276241 73.59411424477062) bank117719 +117720 POINT(41.177940543489264 74.33344351628956) bank117720 +117721 POINT(41.1709494526817 74.54092894459154) bank117721 +117722 POINT(39.884812152667614 73.37452109785886) bank117722 +117723 POINT(40.32034286635993 74.72204516419245) bank117723 +117724 POINT(39.846837054716175 74.10906476171843) bank117724 +117725 POINT(41.643140133670116 73.24798163592882) bank117725 +117726 POINT(39.80759686014815 73.46538628699305) bank117726 +117727 POINT(39.965686833248135 73.92990738100883) bank117727 +117728 POINT(40.91892151349255 74.51250559281841) bank117728 +117729 POINT(40.361835100319645 74.51754204319528) bank117729 +117730 POINT(40.17965530400971 74.62094434543151) bank117730 +117731 POINT(39.7276613622471 73.90851311943787) bank117731 +117732 POINT(39.9042620995453 73.41582123634223) bank117732 +117733 POINT(41.315017578662065 74.17895247772411) bank117733 +117734 POINT(40.13145039237234 73.29487917897022) bank117734 +117735 POINT(41.11842972184048 74.10385181207867) bank117735 +117736 POINT(40.79295357527782 73.15769077081976) bank117736 +117737 POINT(41.13617503547624 74.75106829515641) bank117737 +117738 POINT(40.44668817059411 74.65230539325186) bank117738 +117739 POINT(41.04171446513191 73.13808108412842) bank117739 +117740 POINT(40.64322354336903 74.37314896018736) bank117740 +117741 POINT(40.87208434884516 73.15075250425649) bank117741 +117742 POINT(40.68757914105431 74.40084529441229) bank117742 +117743 POINT(40.35618141256106 73.7739620050251) bank117743 +117744 POINT(40.97415609076909 73.8592035326801) bank117744 +117745 POINT(40.76014638830819 73.57794294261852) bank117745 +117746 POINT(40.70735416654154 73.54528320606455) bank117746 +117747 POINT(41.03572143806903 74.33520150803365) bank117747 +117748 POINT(41.517846556172145 73.583551249926) bank117748 +117749 POINT(40.44855171098162 73.17043247806804) bank117749 +117750 POINT(41.63225215317642 73.47631316070384) bank117750 +117751 POINT(40.356381074956204 73.5783918819445) bank117751 +117752 POINT(41.25047662586163 73.29191616272512) bank117752 +117753 POINT(40.68633553264682 73.97365535170127) bank117753 +117754 POINT(41.182840706125525 74.08522643847242) bank117754 +117755 POINT(41.046388752486436 74.76428268952547) bank117755 +117756 POINT(39.901482125175875 74.9306590231186) bank117756 +117757 POINT(40.86251769497013 74.5611787741785) bank117757 +117758 POINT(41.48996175316065 73.608206689138) bank117758 +117759 POINT(41.43017742554573 74.93483107194008) bank117759 +117760 POINT(41.33166348926505 73.87393662145914) bank117760 +117761 POINT(41.116559685859286 73.89335932519221) bank117761 +117762 POINT(41.486334312530055 74.93951798242833) bank117762 +117763 POINT(40.38745159562895 73.13500734677353) bank117763 +117764 POINT(41.4586928618634 73.4160558926746) bank117764 +117765 POINT(41.1457817847404 73.54442350042909) bank117765 +117766 POINT(40.20540685727592 74.48443672613826) bank117766 +117767 POINT(40.72580800676741 73.36769647006344) bank117767 +117768 POINT(41.02568225998458 74.7822087647015) bank117768 +117769 POINT(39.99348895703468 73.8400871987806) bank117769 +117770 POINT(41.08679114915884 74.77156863350869) bank117770 +117771 POINT(41.699082811416766 73.53177937671248) bank117771 +117772 POINT(41.081141765555664 74.33088753078115) bank117772 +117773 POINT(40.88753743573122 73.15947514353533) bank117773 +117774 POINT(40.770828765808204 74.79543519500749) bank117774 +117775 POINT(39.86742180679184 73.96824630427412) bank117775 +117776 POINT(41.03011399086389 74.66118341409253) bank117776 +117777 POINT(41.51592474231515 73.57113533840635) bank117777 +117778 POINT(40.68539328693464 73.8499603530652) bank117778 +117779 POINT(41.07070234933058 74.92223734521426) bank117779 +117780 POINT(40.200276674687544 73.90831756206826) bank117780 +117781 POINT(40.53753244463138 74.8538745192036) bank117781 +117782 POINT(41.370449607039035 74.37192786227311) bank117782 +117783 POINT(41.643049526365246 73.88186452333757) bank117783 +117784 POINT(41.55595828672523 74.81148282529493) bank117784 +117785 POINT(39.76269119200542 73.49053242378432) bank117785 +117786 POINT(41.11950381672332 73.33400771421674) bank117786 +117787 POINT(40.16496112139806 74.84572215318566) bank117787 +117788 POINT(39.82789977720389 74.02581373352606) bank117788 +117789 POINT(40.32708379403251 73.11849060385548) bank117789 +117790 POINT(41.18247930513935 73.63707599721232) bank117790 +117791 POINT(41.157573136924064 74.39774812332037) bank117791 +117792 POINT(40.11587439897622 73.50875813935218) bank117792 +117793 POINT(40.17370143302644 74.6661202949447) bank117793 +117794 POINT(39.72828410634461 74.56047577018637) bank117794 +117795 POINT(41.22782043040704 73.95681673775114) bank117795 +117796 POINT(41.25836939994036 73.45353259577377) bank117796 +117797 POINT(40.26273789407516 74.902923296903) bank117797 +117798 POINT(40.69441398135122 73.56723600487987) bank117798 +117799 POINT(41.68760161398278 74.52636986069813) bank117799 +117800 POINT(40.356452772619704 74.27011381732221) bank117800 +117801 POINT(41.28640857867246 73.52105904450465) bank117801 +117802 POINT(39.862666521093985 73.62451057090735) bank117802 +117803 POINT(41.67966457223773 74.96024182694093) bank117803 +117804 POINT(41.43643453702867 73.14618604242328) bank117804 +117805 POINT(40.402312311481346 74.26130831687712) bank117805 +117806 POINT(40.53252534180964 74.66164728117717) bank117806 +117807 POINT(41.546897621469775 73.00636847470618) bank117807 +117808 POINT(41.37077566924852 73.40450483957333) bank117808 +117809 POINT(39.92584156887267 73.44380005601273) bank117809 +117810 POINT(39.72406307050283 74.31847472959258) bank117810 +117811 POINT(40.29898131329486 73.11254467455899) bank117811 +117812 POINT(41.59523963467493 73.18351780356673) bank117812 +117813 POINT(40.27133212298815 74.0703707041453) bank117813 +117814 POINT(40.49902042037197 74.67158138227272) bank117814 +117815 POINT(40.33272303944555 74.67187322090618) bank117815 +117816 POINT(40.92272803438062 73.10919697132292) bank117816 +117817 POINT(39.87300886191121 74.48237571629451) bank117817 +117818 POINT(40.016334529461474 73.82454076679036) bank117818 +117819 POINT(40.588802217615864 75.00424433232531) bank117819 +117820 POINT(41.02127818820844 74.94504511362902) bank117820 +117821 POINT(40.52631212619433 74.75176850995183) bank117821 +117822 POINT(39.761108185656774 74.49049930751052) bank117822 +117823 POINT(39.88341438195618 73.24975389945709) bank117823 +117824 POINT(40.98737561930155 74.81249781338975) bank117824 +117825 POINT(41.52831848090794 74.32622787583101) bank117825 +117826 POINT(40.137617957430315 73.78195223702824) bank117826 +117827 POINT(40.70395746018639 74.97438056304134) bank117827 +117828 POINT(39.88302419210164 73.71703246454392) bank117828 +117829 POINT(40.473760804774116 74.71279234362845) bank117829 +117830 POINT(40.68553506998292 74.9808888419687) bank117830 +117831 POINT(40.82288547290819 73.41820121268309) bank117831 +117832 POINT(40.808559552740796 74.13999749595857) bank117832 +117833 POINT(40.49520000685272 74.20460363893633) bank117833 +117834 POINT(41.345472333190756 74.4683351740959) bank117834 +117835 POINT(40.888397654203516 73.43545726693854) bank117835 +117836 POINT(41.472992271148854 74.00159588465745) bank117836 +117837 POINT(39.99703539807993 73.74208027933598) bank117837 +117838 POINT(40.72075453082421 74.71597747132947) bank117838 +117839 POINT(41.12618306755326 74.5766955265578) bank117839 +117840 POINT(41.51367423140577 73.67363830500526) bank117840 +117841 POINT(41.49785765426048 74.97261151268052) bank117841 +117842 POINT(40.55158682163282 73.64457514807584) bank117842 +117843 POINT(41.080446450378766 74.5723586261518) bank117843 +117844 POINT(40.67545680252003 73.41509612018606) bank117844 +117845 POINT(41.51327203428317 73.51870432336753) bank117845 +117846 POINT(40.569352152350945 73.35155944055866) bank117846 +117847 POINT(40.39565688998581 74.37368662065552) bank117847 +117848 POINT(41.31459973093435 74.80764218349606) bank117848 +117849 POINT(41.091519004180206 73.04535584188483) bank117849 +117850 POINT(40.458762124636884 73.47727531823048) bank117850 +117851 POINT(39.76307746988595 73.54692012268791) bank117851 +117852 POINT(40.169783912483325 74.28338141543999) bank117852 +117853 POINT(40.77155216014549 74.04463522650579) bank117853 +117854 POINT(40.905918172773426 74.39468093295062) bank117854 +117855 POINT(41.66447298320652 73.52079714705071) bank117855 +117856 POINT(39.93794827834799 73.24431359298333) bank117856 +117857 POINT(40.00481720372572 73.50955126924909) bank117857 +117858 POINT(41.447437253877226 73.21522850512723) bank117858 +117859 POINT(40.46751227077237 73.55316517462258) bank117859 +117860 POINT(40.76622985054626 74.83416437149818) bank117860 +117861 POINT(41.35738198985036 73.18961814029785) bank117861 +117862 POINT(40.6634586407736 74.88422892997407) bank117862 +117863 POINT(41.50385592575167 73.2263374223117) bank117863 +117864 POINT(41.34463587337132 73.48499439256567) bank117864 +117865 POINT(40.77028588144783 74.49061902059063) bank117865 +117866 POINT(41.39441266245545 74.85460238483223) bank117866 +117867 POINT(41.547141210160476 74.49312393076004) bank117867 +117868 POINT(40.07207672767473 73.70793626949288) bank117868 +117869 POINT(40.865002809101945 74.27976502123667) bank117869 +117870 POINT(41.185513872497594 73.46393832582268) bank117870 +117871 POINT(40.212755991516964 73.89163084024116) bank117871 +117872 POINT(40.68808502075504 74.67036281944131) bank117872 +117873 POINT(40.50161341162335 73.08502580196223) bank117873 +117874 POINT(41.64168276939797 74.6325501255401) bank117874 +117875 POINT(40.14521828809024 73.6881482440754) bank117875 +117876 POINT(40.32605302889053 73.18382245445261) bank117876 +117877 POINT(41.66831084319009 74.96573080573559) bank117877 +117878 POINT(40.75956907344887 73.37853162049508) bank117878 +117879 POINT(40.25777846464605 74.13780726641632) bank117879 +117880 POINT(41.58461722808345 74.83778353879445) bank117880 +117881 POINT(41.200655494294324 73.22539066703561) bank117881 +117882 POINT(40.96078880791969 75.00012124016749) bank117882 +117883 POINT(40.11750140219274 73.12263136820405) bank117883 +117884 POINT(40.26803718050646 74.8918870825533) bank117884 +117885 POINT(40.68673927106497 73.15216353904717) bank117885 +117886 POINT(40.99576500441688 73.1567662573149) bank117886 +117887 POINT(40.37545791642402 74.82255978924566) bank117887 +117888 POINT(39.891546624492015 73.43422216770567) bank117888 +117889 POINT(40.70835566381265 73.2095902066225) bank117889 +117890 POINT(41.344031324512 73.49552763412518) bank117890 +117891 POINT(41.63517440825179 73.13622257181339) bank117891 +117892 POINT(41.16834798461742 73.1179721931978) bank117892 +117893 POINT(41.13184160498433 73.6931643542462) bank117893 +117894 POINT(40.92193195298003 74.50081171281938) bank117894 +117895 POINT(40.081608307369265 74.39547618052934) bank117895 +117896 POINT(40.37842287615702 73.76376582886776) bank117896 +117897 POINT(40.20402854537571 73.44972946282886) bank117897 +117898 POINT(41.499717353414965 74.64284936452039) bank117898 +117899 POINT(40.36810157619615 73.49435989196971) bank117899 +117900 POINT(41.66079954069789 74.61750673902161) bank117900 +117901 POINT(40.18062576841435 74.21803620863142) bank117901 +117902 POINT(39.76062403887426 73.55422325185786) bank117902 +117903 POINT(41.260438483419655 74.27312088697491) bank117903 +117904 POINT(40.70674668242683 74.63817022641237) bank117904 +117905 POINT(40.05687754912653 73.61679152115583) bank117905 +117906 POINT(40.50056001783114 73.09748278761751) bank117906 +117907 POINT(40.00136173058054 73.34784164066133) bank117907 +117908 POINT(41.4328955789297 74.74098876464873) bank117908 +117909 POINT(41.48343880165306 73.93428243976719) bank117909 +117910 POINT(41.09727192175584 74.81543443982821) bank117910 +117911 POINT(39.96624642073133 74.24640194309687) bank117911 +117912 POINT(41.62684928092104 73.55082185104797) bank117912 +117913 POINT(41.627986715587404 74.97418570397694) bank117913 +117914 POINT(40.38645503634503 73.31440812904809) bank117914 +117915 POINT(40.16843067020408 73.21825944097499) bank117915 +117916 POINT(40.325195246251624 74.75863783110165) bank117916 +117917 POINT(39.957821995934076 73.68771753733898) bank117917 +117918 POINT(40.25577118626521 74.44015022156744) bank117918 +117919 POINT(40.39261233542253 74.33690780929673) bank117919 +117920 POINT(40.70507727250576 73.13955029361092) bank117920 +117921 POINT(40.64920513544141 73.7792459544187) bank117921 +117922 POINT(40.66970851887206 73.17288209444659) bank117922 +117923 POINT(40.63252597506247 74.92874780046748) bank117923 +117924 POINT(40.32335181898566 73.68936411908915) bank117924 +117925 POINT(40.29945118602147 73.23641569237994) bank117925 +117926 POINT(40.719574523626775 73.64758522324175) bank117926 +117927 POINT(40.37312929730028 74.87667682125824) bank117927 +117928 POINT(40.908183109072894 73.17439926369727) bank117928 +117929 POINT(41.62826068524423 73.68786567417297) bank117929 +117930 POINT(40.17262730204085 74.76928855668417) bank117930 +117931 POINT(40.867414564635446 74.63031105548593) bank117931 +117932 POINT(39.85978838675214 74.57904711504271) bank117932 +117933 POINT(40.233276176017824 74.19043153682121) bank117933 +117934 POINT(40.56380171548227 73.79867535383445) bank117934 +117935 POINT(41.681180450530604 73.33296691346459) bank117935 +117936 POINT(40.78554995528962 74.89782995811268) bank117936 +117937 POINT(40.0904205236822 74.69087687866701) bank117937 +117938 POINT(40.52015859657206 74.73040037319797) bank117938 +117939 POINT(40.22062700511457 74.80536105101187) bank117939 +117940 POINT(41.69964801147579 74.47951730964182) bank117940 +117941 POINT(41.67061207065024 74.39013098847016) bank117941 +117942 POINT(40.20225110550824 73.75559180220479) bank117942 +117943 POINT(41.64042013313701 74.96943513973413) bank117943 +117944 POINT(41.38179152723344 73.302196059575) bank117944 +117945 POINT(40.65558138671737 74.37502200238676) bank117945 +117946 POINT(39.849967197122794 74.24483640490469) bank117946 +117947 POINT(41.45720280303887 73.47377564743161) bank117947 +117948 POINT(40.80681782013761 74.4815238049498) bank117948 +117949 POINT(40.170328494065856 74.1930855508808) bank117949 +117950 POINT(39.97341130868319 73.75645731576732) bank117950 +117951 POINT(41.35615342134175 74.83270358867594) bank117951 +117952 POINT(40.03988753349435 74.36717333431207) bank117952 +117953 POINT(39.81445427184329 74.96201319080294) bank117953 +117954 POINT(41.625022019256235 74.83138554639876) bank117954 +117955 POINT(41.17344568574634 74.3176217751994) bank117955 +117956 POINT(41.62895268496004 74.84553871981717) bank117956 +117957 POINT(40.46983246665779 74.68768165644846) bank117957 +117958 POINT(40.875622205222264 74.65550571533689) bank117958 +117959 POINT(40.27204184155158 73.01695957633913) bank117959 +117960 POINT(41.40797517523553 73.6835612918694) bank117960 +117961 POINT(40.39887393687683 74.35064945956303) bank117961 +117962 POINT(40.86870353358005 74.04856006563604) bank117962 +117963 POINT(40.71118609499158 74.06179198691017) bank117963 +117964 POINT(41.60161367084685 74.06632244843952) bank117964 +117965 POINT(40.40251059763556 74.31597793270271) bank117965 +117966 POINT(41.68919818398349 73.4838050074222) bank117966 +117967 POINT(41.515970026482115 74.22653252429117) bank117967 +117968 POINT(40.334729575791265 74.885066560832) bank117968 +117969 POINT(41.49058347146882 74.35310777434083) bank117969 +117970 POINT(40.404736339471704 74.67458239069065) bank117970 +117971 POINT(39.72567668666838 73.5020128610278) bank117971 +117972 POINT(40.328293624101825 74.9411173264349) bank117972 +117973 POINT(39.95034353481205 74.43597346642383) bank117973 +117974 POINT(39.90408776127542 74.55246286305564) bank117974 +117975 POINT(40.242768905834126 74.98044085417365) bank117975 +117976 POINT(41.15736458098734 74.69419186902675) bank117976 +117977 POINT(40.859548867083355 74.12684743881728) bank117977 +117978 POINT(39.75123879899869 73.67386936477966) bank117978 +117979 POINT(41.25722051360689 74.83171544139192) bank117979 +117980 POINT(41.55840799037757 73.99016302145509) bank117980 +117981 POINT(40.17506204806976 74.37656164789841) bank117981 +117982 POINT(40.14076125081959 73.91642599482196) bank117982 +117983 POINT(41.11510999411543 73.33936094017868) bank117983 +117984 POINT(41.13031602719269 73.78630633172772) bank117984 +117985 POINT(40.69555466970304 73.53017823391983) bank117985 +117986 POINT(41.50483083981462 74.70251383275136) bank117986 +117987 POINT(40.80644026804215 74.7206446993198) bank117987 +117988 POINT(40.437252981597794 73.59854136434406) bank117988 +117989 POINT(41.43254979297841 73.9101397434184) bank117989 +117990 POINT(41.64766576669706 73.72955746727553) bank117990 +117991 POINT(39.96910291658129 74.72152128876581) bank117991 +117992 POINT(40.27008422450717 73.55311767031496) bank117992 +117993 POINT(41.14994424464402 73.91808976634945) bank117993 +117994 POINT(39.98078306259943 73.44781129196316) bank117994 +117995 POINT(40.49947772768196 74.67349804142523) bank117995 +117996 POINT(41.08453451459461 73.99793514851235) bank117996 +117997 POINT(41.233970948781746 73.19060767585167) bank117997 +117998 POINT(40.402860922157785 73.12395570498433) bank117998 +117999 POINT(40.13698670042647 74.30181457987126) bank117999 +118000 POINT(40.270052494402776 74.81735106669463) bank118000 +118001 POINT(40.98182551461961 73.14376619207385) bank118001 +118002 POINT(40.60374225515646 73.79634267520503) bank118002 +118003 POINT(41.46618882653353 74.40342944438858) bank118003 +118004 POINT(41.7073819825731 73.90025787266289) bank118004 +118005 POINT(41.097803156933985 74.31845345156762) bank118005 +118006 POINT(41.255269829861874 74.2401730240425) bank118006 +118007 POINT(40.007277542592405 74.06954231829602) bank118007 +118008 POINT(41.222156363084935 73.1822822186927) bank118008 +118009 POINT(39.82831381946361 74.9822991756509) bank118009 +118010 POINT(40.82870903848015 73.8677338061886) bank118010 +118011 POINT(40.6443580735376 74.2177268798361) bank118011 +118012 POINT(40.33873735808863 73.43724444866632) bank118012 +118013 POINT(40.50271177748642 74.0408473979855) bank118013 +118014 POINT(39.74277823473542 74.56860557195017) bank118014 +118015 POINT(40.928864467669754 74.42261927069917) bank118015 +118016 POINT(40.59969170330135 73.36738074730455) bank118016 +118017 POINT(41.067050700901426 74.50786612861037) bank118017 +118018 POINT(41.13805141111685 73.54721904092867) bank118018 +118019 POINT(39.965743956912135 73.77550570210838) bank118019 +118020 POINT(40.97833216306987 73.91006617825622) bank118020 +118021 POINT(41.682870997502974 74.08188587089872) bank118021 +118022 POINT(40.937338443673205 74.70896718365417) bank118022 +118023 POINT(41.460223071167455 73.05336946035317) bank118023 +118024 POINT(40.10000237225498 73.70026233557954) bank118024 +118025 POINT(41.62220762195159 73.938129273578) bank118025 +118026 POINT(40.11087822508085 74.92685371076848) bank118026 +118027 POINT(41.01513751024192 73.22186649040769) bank118027 +118028 POINT(40.5947638573051 73.90715290263316) bank118028 +118029 POINT(41.40399262453519 73.28802925141207) bank118029 +118030 POINT(39.768558939486965 73.39164228661417) bank118030 +118031 POINT(40.76687508534297 75.00250668839476) bank118031 +118032 POINT(39.9177370234887 73.3848966313948) bank118032 +118033 POINT(40.22785694136132 74.42776950787322) bank118033 +118034 POINT(40.946127812451245 74.59696904685971) bank118034 +118035 POINT(40.39123277101196 74.70978144218512) bank118035 +118036 POINT(40.275189510792664 74.21487680264863) bank118036 +118037 POINT(41.27118354660135 74.46426602201258) bank118037 +118038 POINT(41.23759091855887 74.5933007083036) bank118038 +118039 POINT(40.028701523534366 73.38154180928059) bank118039 +118040 POINT(39.87449650668611 73.64201744221697) bank118040 +118041 POINT(39.82107425820675 74.08390002967576) bank118041 +118042 POINT(41.65202272247621 73.9587156048999) bank118042 +118043 POINT(39.849439455599004 73.52401694895494) bank118043 +118044 POINT(39.87512626889352 73.45685968616553) bank118044 +118045 POINT(40.757636467710235 73.3081268374762) bank118045 +118046 POINT(40.24682813719838 74.28281601842427) bank118046 +118047 POINT(40.07329696979445 74.23761724539057) bank118047 +118048 POINT(40.845604183185074 73.78425355590714) bank118048 +118049 POINT(39.79376377470401 74.41275632428594) bank118049 +118050 POINT(40.35256445828176 73.90258340076736) bank118050 +118051 POINT(40.625068426118276 74.49481505819014) bank118051 +118052 POINT(41.28545085901882 73.74988819350868) bank118052 +118053 POINT(41.507299506430016 73.48595306124045) bank118053 +118054 POINT(41.616972187561764 73.86339080406788) bank118054 +118055 POINT(40.154616209722796 74.96756289485833) bank118055 +118056 POINT(39.96745058617695 74.09294577262298) bank118056 +118057 POINT(41.70286890396446 74.71909360572654) bank118057 +118058 POINT(40.551256981764844 73.18411734941179) bank118058 +118059 POINT(41.35715840252631 73.44366365109677) bank118059 +118060 POINT(40.704791618867425 74.44838538725787) bank118060 +118061 POINT(39.78995784481221 74.4969660054158) bank118061 +118062 POINT(40.14930215982913 74.37001423472817) bank118062 +118063 POINT(39.941575570637376 74.38276885596025) bank118063 +118064 POINT(41.13837258448361 74.45505438419558) bank118064 +118065 POINT(40.07332297592811 73.97687385041841) bank118065 +118066 POINT(39.986484121419814 73.18118146303556) bank118066 +118067 POINT(40.69561390432062 73.94929653446897) bank118067 +118068 POINT(40.761779773963546 73.4027283995008) bank118068 +118069 POINT(40.50969206942634 73.29688789732312) bank118069 +118070 POINT(41.526172563449876 73.24973495990605) bank118070 +118071 POINT(40.77011911706277 74.05445126643735) bank118071 +118072 POINT(40.02424263269818 74.87876795290568) bank118072 +118073 POINT(41.272288142897665 73.07873714493577) bank118073 +118074 POINT(40.287923472262136 74.56062146943628) bank118074 +118075 POINT(40.40789241092526 74.26377953660487) bank118075 +118076 POINT(41.16377979960914 74.15265414001803) bank118076 +118077 POINT(39.93215627880542 73.67740861343493) bank118077 +118078 POINT(40.205910770890085 74.45441905531776) bank118078 +118079 POINT(40.74985072679844 73.1442631888712) bank118079 +118080 POINT(40.15678072249877 74.3637422788575) bank118080 +118081 POINT(40.72431774275238 74.27914563583452) bank118081 +118082 POINT(40.694647260642924 73.38473827338032) bank118082 +118083 POINT(41.26537361163768 74.18356766145706) bank118083 +118084 POINT(40.98051940424639 73.46469791322741) bank118084 +118085 POINT(41.046493930974414 73.4382804822978) bank118085 +118086 POINT(40.16973265236377 73.02431486743025) bank118086 +118087 POINT(40.200526774065644 74.27514999203085) bank118087 +118088 POINT(40.04614451569908 74.80977293941277) bank118088 +118089 POINT(40.81820665933128 73.9153451346743) bank118089 +118090 POINT(40.80403252302763 74.578571031593) bank118090 +118091 POINT(41.13638086896041 74.31428329869308) bank118091 +118092 POINT(40.876276912309656 73.85986887283374) bank118092 +118093 POINT(40.948587566285205 73.5944093978231) bank118093 +118094 POINT(40.822175213396434 74.13999536123457) bank118094 +118095 POINT(40.08087058181748 74.60079429360086) bank118095 +118096 POINT(40.84243736313529 73.41679683315478) bank118096 +118097 POINT(41.57703164169091 73.4220218960705) bank118097 +118098 POINT(41.2863241627654 73.08890774911447) bank118098 +118099 POINT(41.22981840235366 73.12096234068963) bank118099 +118100 POINT(40.90946446627025 73.30395691789282) bank118100 +118101 POINT(41.62715793371998 73.80955718677907) bank118101 +118102 POINT(41.399697445829425 74.71562582711232) bank118102 +118103 POINT(41.64970019682178 74.70959947526393) bank118103 +118104 POINT(40.07371990770973 73.97888244826262) bank118104 +118105 POINT(40.71901098261465 74.05851891982853) bank118105 +118106 POINT(41.64195720698537 73.65124613126177) bank118106 +118107 POINT(41.5845722300286 74.00398185968456) bank118107 +118108 POINT(41.028919272595374 73.87819613999692) bank118108 +118109 POINT(41.50323689190995 74.68957277740854) bank118109 +118110 POINT(41.33176160266035 74.98239222777593) bank118110 +118111 POINT(40.37981452721417 73.75294589311626) bank118111 +118112 POINT(40.33129034302825 73.86993471489197) bank118112 +118113 POINT(40.28615595162292 73.13842277640549) bank118113 +118114 POINT(41.24948504213166 74.63559035969146) bank118114 +118115 POINT(41.20028241390232 74.51882109100096) bank118115 +118116 POINT(40.314928375971654 74.84871858609276) bank118116 +118117 POINT(40.80135689666113 73.96355572261368) bank118117 +118118 POINT(40.06187694924443 74.44149859019757) bank118118 +118119 POINT(40.36966982198665 74.91548693935948) bank118119 +118120 POINT(41.34611287206632 73.11378177048869) bank118120 +118121 POINT(39.76462502999952 73.7624530722907) bank118121 +118122 POINT(40.517586073046054 74.8449021384782) bank118122 +118123 POINT(40.9459345447208 73.80831688779683) bank118123 +118124 POINT(40.45837239894596 73.3752226734181) bank118124 +118125 POINT(40.64054675070295 74.63889341258539) bank118125 +118126 POINT(40.87674145235272 73.04849994577437) bank118126 +118127 POINT(39.78554797318247 74.26153210777696) bank118127 +118128 POINT(40.46558459823262 73.52019331264638) bank118128 +118129 POINT(40.59296863083836 73.46330465105855) bank118129 +118130 POINT(41.71238425870955 74.11934643232472) bank118130 +118131 POINT(41.679260369613004 74.74765929078406) bank118131 +118132 POINT(40.41424110482303 74.46708620021282) bank118132 +118133 POINT(41.468441775863 74.8246907405681) bank118133 +118134 POINT(40.608367346663634 73.90592122673286) bank118134 +118135 POINT(40.364595969663654 74.42936021022689) bank118135 +118136 POINT(40.949281542551496 73.34654697673861) bank118136 +118137 POINT(41.649592532575426 73.31809698713515) bank118137 +118138 POINT(40.10405945856873 73.03696011856731) bank118138 +118139 POINT(41.23273881603891 73.13974671455418) bank118139 +118140 POINT(41.48031107460598 74.59280570410641) bank118140 +118141 POINT(40.807715198345676 74.14540426338364) bank118141 +118142 POINT(40.17649576804141 74.03274826102441) bank118142 +118143 POINT(40.256800916668965 73.7264716792207) bank118143 +118144 POINT(39.967278753397075 74.16184638159976) bank118144 +118145 POINT(41.20934078016698 73.08669597163707) bank118145 +118146 POINT(40.34410518758508 73.49881417804676) bank118146 +118147 POINT(40.89152959825343 73.59714061526002) bank118147 +118148 POINT(39.78092614731077 74.70323161999788) bank118148 +118149 POINT(41.34970683892646 73.05952243416266) bank118149 +118150 POINT(41.25614287284043 73.46470584016167) bank118150 +118151 POINT(41.113109214041344 74.63760284319211) bank118151 +118152 POINT(40.751739453359015 73.33090116004122) bank118152 +118153 POINT(41.59376954519241 74.66715098214821) bank118153 +118154 POINT(41.08782610712245 73.52449900910725) bank118154 +118155 POINT(41.01370190726523 74.25875228699547) bank118155 +118156 POINT(41.48023717181476 73.09666037359719) bank118156 +118157 POINT(40.968906620844166 74.79858212203607) bank118157 +118158 POINT(40.11173642766502 73.81028097331182) bank118158 +118159 POINT(40.927048354505835 74.93052567064812) bank118159 +118160 POINT(40.04291915078716 74.5740397800533) bank118160 +118161 POINT(40.83628343958514 74.23985806426201) bank118161 +118162 POINT(41.17267826113733 73.97567427072113) bank118162 +118163 POINT(41.373679378899354 73.88489386850436) bank118163 +118164 POINT(39.727338789929334 74.59062756097983) bank118164 +118165 POINT(41.00984607846027 73.59120978879396) bank118165 +118166 POINT(40.89695181655789 74.20591246746088) bank118166 +118167 POINT(41.59643684046406 73.78507538676034) bank118167 +118168 POINT(41.14648020163716 74.288528531332) bank118168 +118169 POINT(40.005985601870975 74.48495727456651) bank118169 +118170 POINT(41.093335294725364 74.29283009199654) bank118170 +118171 POINT(40.15951492490654 74.98123711392799) bank118171 +118172 POINT(39.72060726463269 73.58160253314732) bank118172 +118173 POINT(40.569455362487446 74.48861115884215) bank118173 +118174 POINT(40.24215810589887 73.61117955237671) bank118174 +118175 POINT(41.10147852469101 74.05374066528503) bank118175 +118176 POINT(41.5578280512325 73.43763111054238) bank118176 +118177 POINT(40.708807999220994 73.82757309281678) bank118177 +118178 POINT(41.617984536255754 74.96844107735421) bank118178 +118179 POINT(40.980476344850096 73.17565451480351) bank118179 +118180 POINT(41.68042081142749 73.02425190717229) bank118180 +118181 POINT(41.19246751287719 73.44010325190068) bank118181 +118182 POINT(39.760537858895354 73.93236602400007) bank118182 +118183 POINT(41.119506325702595 74.76167170515963) bank118183 +118184 POINT(41.50606301793389 74.7667521853752) bank118184 +118185 POINT(41.2142984844807 73.76803106788398) bank118185 +118186 POINT(40.159655393019854 74.24581666772829) bank118186 +118187 POINT(39.97216172161646 73.4795070423522) bank118187 +118188 POINT(41.49768744509534 74.0506382740978) bank118188 +118189 POINT(40.60149752061337 74.60066044417582) bank118189 +118190 POINT(41.27866753971081 74.69632356212234) bank118190 +118191 POINT(41.46449737145769 73.40220595589685) bank118191 +118192 POINT(40.326029624151055 74.91913483261706) bank118192 +118193 POINT(40.37081768683925 74.56605483898441) bank118193 +118194 POINT(39.936137616085425 74.65877033920216) bank118194 +118195 POINT(41.68452005735434 73.20374554854344) bank118195 +118196 POINT(40.16858107282709 73.59277123438531) bank118196 +118197 POINT(40.926690380659856 74.19587114714118) bank118197 +118198 POINT(40.30163098841376 73.63690965802277) bank118198 +118199 POINT(40.951687315495526 74.67202994613922) bank118199 +118200 POINT(41.51957435577746 73.88693059091723) bank118200 +118201 POINT(40.35522546588895 73.03684096334618) bank118201 +118202 POINT(41.21112119849087 73.21403961693123) bank118202 +118203 POINT(40.47170792790796 73.39093018147678) bank118203 +118204 POINT(40.327490278227806 73.45886009307226) bank118204 +118205 POINT(40.07940685365571 73.87106043162275) bank118205 +118206 POINT(39.85184227106188 73.91741894815777) bank118206 +118207 POINT(40.609265643811625 74.28514654391438) bank118207 +118208 POINT(40.36663753916086 73.33038882399593) bank118208 +118209 POINT(41.064275610583834 73.59478729869397) bank118209 +118210 POINT(40.07027206247769 73.01203264431685) bank118210 +118211 POINT(40.99466743050991 74.8692028524675) bank118211 +118212 POINT(39.73961126362421 74.46827451580333) bank118212 +118213 POINT(41.10865563552174 74.44309890144932) bank118213 +118214 POINT(40.527745449671414 73.33283593007336) bank118214 +118215 POINT(39.95469642876805 74.94124897288995) bank118215 +118216 POINT(40.91100912420879 74.51278639894105) bank118216 +118217 POINT(39.93751525441973 74.9710101978573) bank118217 +118218 POINT(40.96846705108093 74.29192419060657) bank118218 +118219 POINT(39.719042689309 73.65059050963048) bank118219 +118220 POINT(41.03541148327811 73.5633679155444) bank118220 +118221 POINT(40.36413287784072 73.80950886734152) bank118221 +118222 POINT(39.85779578807125 74.746627853881) bank118222 +118223 POINT(40.15022858819485 74.98353999052816) bank118223 +118224 POINT(40.53008880089145 73.0154775358476) bank118224 +118225 POINT(39.99476330883526 73.0212102869517) bank118225 +118226 POINT(41.55678507892164 74.87310658936258) bank118226 +118227 POINT(40.13128478459214 74.14242507310715) bank118227 +118228 POINT(40.21712858148259 73.141369786191) bank118228 +118229 POINT(40.922391077524566 74.80508623016794) bank118229 +118230 POINT(39.88724952395217 74.92992872040365) bank118230 +118231 POINT(39.87850217607097 74.66345907279783) bank118231 +118232 POINT(41.67601570502976 73.68213467307405) bank118232 +118233 POINT(41.00435660965601 74.20114610747598) bank118233 +118234 POINT(40.894256428244006 73.97203435100694) bank118234 +118235 POINT(40.372995239425414 73.2186708476083) bank118235 +118236 POINT(40.79407441035826 73.40083296924053) bank118236 +118237 POINT(40.61004823393806 74.28624841989532) bank118237 +118238 POINT(40.43265519281877 73.59277746025772) bank118238 +118239 POINT(39.873912464338055 73.19046136968149) bank118239 +118240 POINT(41.13367697316593 74.19403609170092) bank118240 +118241 POINT(40.94233063952084 74.78951531644793) bank118241 +118242 POINT(40.35057322434766 73.90978580584206) bank118242 +118243 POINT(41.472110278586094 73.21483744457716) bank118243 +118244 POINT(40.501618305000974 73.8214037343582) bank118244 +118245 POINT(40.2805416372016 74.95400712358612) bank118245 +118246 POINT(41.35250470837388 73.96850921945436) bank118246 +118247 POINT(41.32083426508787 73.99410623304531) bank118247 +118248 POINT(39.72480468724708 74.6059179666998) bank118248 +118249 POINT(41.60594930840614 73.15258598106942) bank118249 +118250 POINT(41.63689908684987 73.75561768736257) bank118250 +118251 POINT(41.427375040010546 74.6634451068661) bank118251 +118252 POINT(40.95354187786619 73.58541653296555) bank118252 +118253 POINT(41.16293904855774 73.14320415103806) bank118253 +118254 POINT(40.31725380328517 74.47935567227839) bank118254 +118255 POINT(40.45554851231582 74.23992208125976) bank118255 +118256 POINT(41.19960897194544 74.96348090018742) bank118256 +118257 POINT(41.47803792784122 74.77243492843981) bank118257 +118258 POINT(40.61105375768302 74.56230068820746) bank118258 +118259 POINT(40.186713715262385 73.70965078688121) bank118259 +118260 POINT(40.1874612142983 73.93549542737814) bank118260 +118261 POINT(41.66233953555326 73.54949339090903) bank118261 +118262 POINT(40.54336556907806 73.99880053655347) bank118262 +118263 POINT(40.90307590671976 74.24938087176666) bank118263 +118264 POINT(39.933889454042195 73.52319335310429) bank118264 +118265 POINT(41.212880828270656 73.8242511631757) bank118265 +118266 POINT(39.930856855886354 73.65638169328905) bank118266 +118267 POINT(41.12863606204993 73.10176108791529) bank118267 +118268 POINT(41.63108921852903 73.87436615581404) bank118268 +118269 POINT(40.494607723554594 73.17071410238019) bank118269 +118270 POINT(39.93013326690927 73.15660095223309) bank118270 +118271 POINT(41.08204129110945 73.59125661454347) bank118271 +118272 POINT(40.98686327865008 73.31522054622774) bank118272 +118273 POINT(41.50113341737175 74.17574025010772) bank118273 +118274 POINT(41.45010360217469 73.60128824040846) bank118274 +118275 POINT(41.10231426323293 73.70919618107695) bank118275 +118276 POINT(41.28849330712844 74.44314791778608) bank118276 +118277 POINT(41.21995382108403 73.44708009248541) bank118277 +118278 POINT(39.89326652907206 73.65398381876334) bank118278 +118279 POINT(41.06872960492586 74.4739265769372) bank118279 +118280 POINT(40.737990148066224 74.22820793979824) bank118280 +118281 POINT(41.70982042777431 74.77107176042965) bank118281 +118282 POINT(40.598941585082 74.41345546488948) bank118282 +118283 POINT(40.87236571613806 73.87916383779573) bank118283 +118284 POINT(40.57961427706274 73.4760103423756) bank118284 +118285 POINT(40.962307105578425 73.18057141806916) bank118285 +118286 POINT(40.16694997721857 74.802701453837) bank118286 +118287 POINT(41.21315821418988 73.61850596896372) bank118287 +118288 POINT(40.04180797734198 74.03334616740783) bank118288 +118289 POINT(41.248824214301415 74.89257711449397) bank118289 +118290 POINT(41.277677181570375 74.9714381711882) bank118290 +118291 POINT(41.07399039074816 73.90355547295287) bank118291 +118292 POINT(40.28044148462824 73.13662584151423) bank118292 +118293 POINT(40.75312806626891 73.48086157615056) bank118293 +118294 POINT(39.76651821136063 73.17326540292022) bank118294 +118295 POINT(41.048645699013576 73.34186913130168) bank118295 +118296 POINT(40.43499721460096 73.34203523823402) bank118296 +118297 POINT(41.414035527995125 73.7928731817285) bank118297 +118298 POINT(41.54663283664607 74.65692481297367) bank118298 +118299 POINT(41.34209897247434 73.27267426947017) bank118299 +118300 POINT(41.12693218133961 73.46727248597743) bank118300 +118301 POINT(40.32023268216762 73.46667595449975) bank118301 +118302 POINT(40.19169376105728 74.69547693158198) bank118302 +118303 POINT(40.912759207450236 74.0331161584025) bank118303 +118304 POINT(39.743351908652095 74.85159793626066) bank118304 +118305 POINT(41.051449162501825 74.13392958566705) bank118305 +118306 POINT(40.30669348844947 74.39285335169245) bank118306 +118307 POINT(40.611387538679836 73.47674000173754) bank118307 +118308 POINT(39.800817393920646 73.584209549294) bank118308 +118309 POINT(40.963344730330064 74.79026600433502) bank118309 +118310 POINT(40.64484762618643 74.91909265071139) bank118310 +118311 POINT(40.8937667381777 73.50398922791108) bank118311 +118312 POINT(40.57628554337171 74.16280260646266) bank118312 +118313 POINT(41.11423972254311 73.22500366765068) bank118313 +118314 POINT(41.17805681400539 73.9735334920461) bank118314 +118315 POINT(41.046466207899954 74.67124407626473) bank118315 +118316 POINT(39.7896900764891 73.27734732249056) bank118316 +118317 POINT(40.48897621830173 74.26737167297016) bank118317 +118318 POINT(41.559263902345734 73.15875529173292) bank118318 +118319 POINT(41.29450797040564 73.50270423026787) bank118319 +118320 POINT(40.43025785485586 74.91795908288124) bank118320 +118321 POINT(40.4922991790356 73.44757258385881) bank118321 +118322 POINT(40.034945569953976 73.93371515655902) bank118322 +118323 POINT(41.582481266100125 74.25419858264192) bank118323 +118324 POINT(41.00629744751976 74.75806939029185) bank118324 +118325 POINT(41.59988025607908 73.15056394301675) bank118325 +118326 POINT(40.82557835215131 74.8612666212275) bank118326 +118327 POINT(39.856561186754675 74.28228051228095) bank118327 +118328 POINT(40.568656091305165 74.59113925373848) bank118328 +118329 POINT(40.44829228569624 73.51655862373876) bank118329 +118330 POINT(41.155192674156766 73.72237349406151) bank118330 +118331 POINT(39.74141912826134 74.3987261283926) bank118331 +118332 POINT(40.126244942811795 73.21992310441492) bank118332 +118333 POINT(40.75525604664013 73.33520111742901) bank118333 +118334 POINT(40.746539460931686 73.60988308801194) bank118334 +118335 POINT(40.21564288673409 73.07677365350034) bank118335 +118336 POINT(40.30606100359361 74.6810947231539) bank118336 +118337 POINT(41.154972606323916 73.56877887324728) bank118337 +118338 POINT(40.84797114074869 74.96464940637817) bank118338 +118339 POINT(40.209291306791954 74.98451266734212) bank118339 +118340 POINT(40.96345356702856 73.96614011668473) bank118340 +118341 POINT(40.3652556084351 73.26012209867379) bank118341 +118342 POINT(39.721900097681804 74.48350364184238) bank118342 +118343 POINT(41.41496526239458 74.51359563082693) bank118343 +118344 POINT(41.099226120893874 73.63173774904035) bank118344 +118345 POINT(40.78629932643233 73.01178196278737) bank118345 +118346 POINT(41.5524929723095 73.42376053749116) bank118346 +118347 POINT(40.47957156394222 73.61852786388486) bank118347 +118348 POINT(39.94961141873483 74.25600466911911) bank118348 +118349 POINT(40.31637843395285 73.6947990600284) bank118349 +118350 POINT(40.56045086806277 73.61924344180414) bank118350 +118351 POINT(40.63478174135581 73.15120657560422) bank118351 +118352 POINT(40.26180397690111 74.43269785499508) bank118352 +118353 POINT(40.56858509600578 74.5337109755058) bank118353 +118354 POINT(39.87679481791036 73.71319633914688) bank118354 +118355 POINT(40.00873906677631 74.11507733662698) bank118355 +118356 POINT(40.12869916185217 74.71642241865244) bank118356 +118357 POINT(41.18451199053047 74.70154449694466) bank118357 +118358 POINT(40.50457387022102 73.93908255776265) bank118358 +118359 POINT(40.62733575428578 73.80101713783645) bank118359 +118360 POINT(40.30680626169926 74.80531024766793) bank118360 +118361 POINT(40.95109218911014 73.18509947518969) bank118361 +118362 POINT(41.3876312950395 73.55705714324353) bank118362 +118363 POINT(40.12190711255883 73.50402190856319) bank118363 +118364 POINT(40.19571081507927 74.851760553551) bank118364 +118365 POINT(40.664019063536024 74.61164464165464) bank118365 +118366 POINT(41.28861405228885 73.61556752031339) bank118366 +118367 POINT(39.71446039944271 74.4228413544892) bank118367 +118368 POINT(41.02876189634557 73.03490348855485) bank118368 +118369 POINT(41.539921377946115 74.7691603382142) bank118369 +118370 POINT(39.98287063798907 73.64953162408887) bank118370 +118371 POINT(41.22278088518966 74.15775554504317) bank118371 +118372 POINT(41.46496680801701 73.37234757243893) bank118372 +118373 POINT(40.29001023036611 73.0935538311737) bank118373 +118374 POINT(40.974687358972915 73.7436035350655) bank118374 +118375 POINT(40.9687339279158 74.33011289874466) bank118375 +118376 POINT(41.45725481994744 73.47483296753828) bank118376 +118377 POINT(39.89541898882451 73.14574658295729) bank118377 +118378 POINT(40.470588208213634 73.6364546961465) bank118378 +118379 POINT(40.7439397786062 74.26636165156518) bank118379 +118380 POINT(40.588999684783595 73.37779005982217) bank118380 +118381 POINT(40.84014616566143 74.01265827304076) bank118381 +118382 POINT(41.136544744713646 74.87905866957291) bank118382 +118383 POINT(39.8687335795617 74.39345012163393) bank118383 +118384 POINT(41.38742703757338 74.5293570022783) bank118384 +118385 POINT(41.36989916762441 74.44471262578541) bank118385 +118386 POINT(40.20833304197314 73.50561987853345) bank118386 +118387 POINT(39.98069995538121 74.50342352855712) bank118387 +118388 POINT(39.86242125570665 74.83225211623798) bank118388 +118389 POINT(41.56542495200375 73.84601127984233) bank118389 +118390 POINT(41.24141865771512 74.47549654083068) bank118390 +118391 POINT(40.51686935517916 73.5618708035139) bank118391 +118392 POINT(41.56325177499758 74.46722921280062) bank118392 +118393 POINT(39.93322831720268 73.54060914453949) bank118393 +118394 POINT(41.36081803689876 74.00640353421741) bank118394 +118395 POINT(40.653054931647674 74.02315927240741) bank118395 +118396 POINT(41.665380937773385 73.44866049364721) bank118396 +118397 POINT(40.901080852873896 74.5391279451795) bank118397 +118398 POINT(40.75455250554394 74.45786576607411) bank118398 +118399 POINT(41.33244612838282 74.32435770921826) bank118399 +118400 POINT(40.51756229072216 74.31677154307363) bank118400 +118401 POINT(40.57187002031859 74.72918676657875) bank118401 +118402 POINT(39.9758291138961 74.0461161927891) bank118402 +118403 POINT(39.94211781047786 74.18440654267866) bank118403 +118404 POINT(40.91222796598 74.50288515495765) bank118404 +118405 POINT(40.293699770520256 74.09194802958395) bank118405 +118406 POINT(39.892981270365304 74.98454680859656) bank118406 +118407 POINT(41.24484940023458 74.17433300482499) bank118407 +118408 POINT(40.034380822780136 74.25942454630575) bank118408 +118409 POINT(41.557012872429276 73.35230287828634) bank118409 +118410 POINT(40.59334250986795 74.55459572121097) bank118410 +118411 POINT(40.066486046180295 73.98349958629683) bank118411 +118412 POINT(41.71270350228289 73.40420261492304) bank118412 +118413 POINT(41.27561214275366 73.11881374021002) bank118413 +118414 POINT(40.53230925170202 74.58697725532458) bank118414 +118415 POINT(41.546738094479764 74.22146160116397) bank118415 +118416 POINT(41.61828039120599 74.26685343711493) bank118416 +118417 POINT(41.380636200898664 73.4366326444815) bank118417 +118418 POINT(41.09699577108038 74.235812836435) bank118418 +118419 POINT(40.48737892640319 74.38433330604065) bank118419 +118420 POINT(40.95213826195451 73.34891693338228) bank118420 +118421 POINT(40.667066481588314 73.80339769517747) bank118421 +118422 POINT(41.43320983793392 73.40319413603811) bank118422 +118423 POINT(41.39771119703802 74.56283206128137) bank118423 +118424 POINT(40.02095940603582 73.20105979889635) bank118424 +118425 POINT(41.05815075668555 73.19668869772613) bank118425 +118426 POINT(41.20726453449522 74.83913334681488) bank118426 +118427 POINT(40.55991478589451 73.69846721012149) bank118427 +118428 POINT(41.299960650293336 73.12959207376049) bank118428 +118429 POINT(39.85400830303891 74.88656983485458) bank118429 +118430 POINT(40.396874675698385 74.87905031503605) bank118430 +118431 POINT(41.12309240661662 73.6685856345643) bank118431 +118432 POINT(41.021773909856115 73.2622723471373) bank118432 +118433 POINT(40.43278122132875 74.1601910986738) bank118433 +118434 POINT(39.75611663569145 73.64401247503436) bank118434 +118435 POINT(40.56609821931505 74.33002815339806) bank118435 +118436 POINT(41.493190954312695 73.37716268876402) bank118436 +118437 POINT(41.10887388894376 74.70661598387623) bank118437 +118438 POINT(39.92052102757354 73.8958048184054) bank118438 +118439 POINT(41.179877917898054 73.88620083222783) bank118439 +118440 POINT(39.98337852137313 74.70061564707856) bank118440 +118441 POINT(39.869285649068765 73.48136323743866) bank118441 +118442 POINT(41.615061457057855 73.74184516747866) bank118442 +118443 POINT(41.27651814836469 73.48985016436664) bank118443 +118444 POINT(40.90132890799768 73.835167342132) bank118444 +118445 POINT(39.76249318820039 74.6205489676816) bank118445 +118446 POINT(40.73573463184874 73.05581264327242) bank118446 +118447 POINT(41.45373741336112 73.21340495103219) bank118447 +118448 POINT(41.14542112873143 74.95571983738799) bank118448 +118449 POINT(41.262974485386984 74.30780831146825) bank118449 +118450 POINT(39.79931742952355 74.20393086025615) bank118450 +118451 POINT(40.312609773872225 73.61936529732263) bank118451 +118452 POINT(41.28697549561811 74.4414154840377) bank118452 +118453 POINT(41.55159332746932 74.56910779506866) bank118453 +118454 POINT(40.861182551284834 73.9279373687641) bank118454 +118455 POINT(41.419499875942364 73.77926985139692) bank118455 +118456 POINT(40.92070056067532 73.07796285983625) bank118456 +118457 POINT(41.11412053379732 73.42644676355945) bank118457 +118458 POINT(41.40584434905452 74.84316732892148) bank118458 +118459 POINT(40.02399244724456 73.48508417779621) bank118459 +118460 POINT(39.98711684345569 74.00473457131096) bank118460 +118461 POINT(40.12853748842377 73.8480818565572) bank118461 +118462 POINT(41.57904304329458 74.52089998808498) bank118462 +118463 POINT(41.1681665899833 74.41751500769782) bank118463 +118464 POINT(39.86184046284756 74.70821545050748) bank118464 +118465 POINT(41.53224220489691 73.8863908836019) bank118465 +118466 POINT(40.02961448341627 73.22761667141464) bank118466 +118467 POINT(41.40058483312277 74.21477799902269) bank118467 +118468 POINT(40.103962292125814 73.91604453512119) bank118468 +118469 POINT(40.01311773364303 74.47957349243599) bank118469 +118470 POINT(39.81508060319762 74.66693710458519) bank118470 +118471 POINT(40.85542202703732 73.37892844631146) bank118471 +118472 POINT(39.899661335115546 73.07682593426841) bank118472 +118473 POINT(39.85453319933431 74.36880161157565) bank118473 +118474 POINT(40.68008511835579 74.22696639163851) bank118474 +118475 POINT(41.12385496654079 74.57016647714727) bank118475 +118476 POINT(41.10662161125297 73.29626686852914) bank118476 +118477 POINT(40.459464940911325 74.28187542352916) bank118477 +118478 POINT(40.9150096423668 73.07511637425024) bank118478 +118479 POINT(40.506098297678655 74.31861826797747) bank118479 +118480 POINT(40.52911875547713 73.48451224835422) bank118480 +118481 POINT(40.30636403005086 74.700784610159) bank118481 +118482 POINT(39.884787556417606 74.57380978855244) bank118482 +118483 POINT(41.650675748893136 73.86259235990008) bank118483 +118484 POINT(40.35111974338915 73.8705631060162) bank118484 +118485 POINT(40.61689188975426 73.52588888525533) bank118485 +118486 POINT(41.0392516368102 73.50579512097916) bank118486 +118487 POINT(41.36533643480675 74.4160228476394) bank118487 +118488 POINT(40.68304322089895 74.20932745480852) bank118488 +118489 POINT(39.72546890131215 74.13594492407286) bank118489 +118490 POINT(40.000396551671585 74.4848492696138) bank118490 +118491 POINT(41.65687066203441 74.26501676872151) bank118491 +118492 POINT(40.558739013493145 73.18693746325978) bank118492 +118493 POINT(40.36820496147938 74.42024494893943) bank118493 +118494 POINT(41.3667742423059 74.76252427404036) bank118494 +118495 POINT(39.77353569007803 73.74894587102182) bank118495 +118496 POINT(41.48377242311491 74.21347745679961) bank118496 +118497 POINT(41.24423711206747 74.20794356813497) bank118497 +118498 POINT(39.8254537723511 73.65875127173678) bank118498 +118499 POINT(41.64254857440465 73.92823029216152) bank118499 +118500 POINT(41.67160259511597 73.28301985415634) bank118500 +118501 POINT(41.253609077999776 74.04649552126214) bank118501 +118502 POINT(39.73415865479068 74.42519366477997) bank118502 +118503 POINT(41.36147089454191 74.63087890348515) bank118503 +118504 POINT(40.734928905088374 74.38441682428149) bank118504 +118505 POINT(41.65505427491654 74.12609762284416) bank118505 +118506 POINT(40.15837301839664 73.64723317138899) bank118506 +118507 POINT(41.18799110983953 73.18671487904884) bank118507 +118508 POINT(41.46324832095793 73.30722096400277) bank118508 +118509 POINT(40.54194981513142 74.23272609636271) bank118509 +118510 POINT(41.02018133639369 74.25110645060148) bank118510 +118511 POINT(41.63560588533007 73.12766685565467) bank118511 +118512 POINT(39.89628046619994 73.62667793112293) bank118512 +118513 POINT(39.99325416435933 73.07746827119854) bank118513 +118514 POINT(41.13082525779086 73.33706141958768) bank118514 +118515 POINT(41.15747560932913 73.03564658954711) bank118515 +118516 POINT(40.39517877651699 73.40307657673713) bank118516 +118517 POINT(40.100967150865465 74.8752734245018) bank118517 +118518 POINT(40.27106718409513 74.74823348730968) bank118518 +118519 POINT(41.63203720387941 73.7891278155383) bank118519 +118520 POINT(41.17469715579968 74.91323097633705) bank118520 +118521 POINT(40.734502037811126 74.9251747372769) bank118521 +118522 POINT(39.980078849001245 73.7090353246574) bank118522 +118523 POINT(39.751759933736864 74.12652331090223) bank118523 +118524 POINT(40.309147867996295 74.2422590291233) bank118524 +118525 POINT(41.53820060454812 73.14852400029177) bank118525 +118526 POINT(40.06709273221041 73.82747486423982) bank118526 +118527 POINT(40.385596187197365 73.72587377344539) bank118527 +118528 POINT(40.455728568325476 74.26234629071392) bank118528 +118529 POINT(40.94138213506339 74.89662287364906) bank118529 +118530 POINT(41.05764697998986 74.32942015206685) bank118530 +118531 POINT(41.677793937351254 74.77522926144691) bank118531 +118532 POINT(41.65848338206663 73.74496350766547) bank118532 +118533 POINT(39.74064865982819 73.64434493906555) bank118533 +118534 POINT(41.00042010379728 74.57620491103859) bank118534 +118535 POINT(41.056475784299025 73.59708511381234) bank118535 +118536 POINT(40.84363775047843 73.11845785696934) bank118536 +118537 POINT(40.807192714888636 74.46863298883667) bank118537 +118538 POINT(39.80010127740223 74.20939524864173) bank118538 +118539 POINT(41.280329312073995 73.05659351490726) bank118539 +118540 POINT(41.13234458860338 73.20658328159338) bank118540 +118541 POINT(41.00332538480734 73.66301723478693) bank118541 +118542 POINT(40.447390230096374 73.73178377966849) bank118542 +118543 POINT(40.97346273400288 74.64163608515643) bank118543 +118544 POINT(40.381359614539065 73.32732639447902) bank118544 +118545 POINT(41.490169018990684 73.29538716339964) bank118545 +118546 POINT(41.11795722765932 73.05769160428626) bank118546 +118547 POINT(41.04605853987643 73.40277965083135) bank118547 +118548 POINT(41.259274776530255 73.97126756820651) bank118548 +118549 POINT(39.78326938045154 73.6171431079334) bank118549 +118550 POINT(41.187891581311106 73.04951575054298) bank118550 +118551 POINT(39.813826797491394 74.05072326256723) bank118551 +118552 POINT(40.37829613050665 74.82847105494088) bank118552 +118553 POINT(40.412396538712834 74.67110367142998) bank118553 +118554 POINT(40.09202283235386 73.16410867029867) bank118554 +118555 POINT(40.84687334786861 73.07997562337522) bank118555 +118556 POINT(39.83534107700746 73.75036576282663) bank118556 +118557 POINT(39.8858132469894 73.79764607053114) bank118557 +118558 POINT(41.44674418538821 73.37137181229524) bank118558 +118559 POINT(40.70102632791974 73.29441206941739) bank118559 +118560 POINT(40.54950421128372 73.99141535860039) bank118560 +118561 POINT(40.8048350142558 74.77710611245085) bank118561 +118562 POINT(39.997011135751535 73.38431010686243) bank118562 +118563 POINT(40.39760743590524 73.3457420018106) bank118563 +118564 POINT(40.99238904143266 74.62965720928268) bank118564 +118565 POINT(40.30973672026464 74.39380910992975) bank118565 +118566 POINT(40.00430238372227 74.73138868698521) bank118566 +118567 POINT(41.57106789817871 74.0462475250084) bank118567 +118568 POINT(41.1611265272517 74.33025326438681) bank118568 +118569 POINT(40.47473894651268 74.40229404942534) bank118569 +118570 POINT(40.89963852908256 74.2383225703877) bank118570 +118571 POINT(40.22342737198364 74.20374713235225) bank118571 +118572 POINT(41.64456920468049 73.84654074070075) bank118572 +118573 POINT(40.24423731471385 73.01589443761326) bank118573 +118574 POINT(41.35890682680543 73.84040668787192) bank118574 +118575 POINT(40.736060024478206 74.90602982807906) bank118575 +118576 POINT(39.744477726375706 73.79786151333404) bank118576 +118577 POINT(41.01223231619592 74.8951812125748) bank118577 +118578 POINT(41.16125844420105 73.14192866077501) bank118578 +118579 POINT(39.961041125772944 74.75630502904565) bank118579 +118580 POINT(41.206024905946435 75.00083713379806) bank118580 +118581 POINT(40.149501323180715 73.37140612151107) bank118581 +118582 POINT(40.92904032588654 73.8147172064009) bank118582 +118583 POINT(40.10000163964245 74.83430984599559) bank118583 +118584 POINT(41.4499500886998 74.56546890868076) bank118584 +118585 POINT(41.112287078985 73.88210431861425) bank118585 +118586 POINT(40.093819652109985 73.84231662254004) bank118586 +118587 POINT(41.209512969073366 74.84432577421961) bank118587 +118588 POINT(41.170532806631826 74.62560458939407) bank118588 +118589 POINT(40.08685081590674 74.87683790468992) bank118589 +118590 POINT(41.24115880578584 73.90337378943491) bank118590 +118591 POINT(40.34528627979165 73.89943001988347) bank118591 +118592 POINT(40.22171963733258 73.8974066070588) bank118592 +118593 POINT(40.6066058851179 73.03483016506155) bank118593 +118594 POINT(40.87057236425317 74.20379645028974) bank118594 +118595 POINT(40.51954624529923 74.91487170618672) bank118595 +118596 POINT(39.957619685121784 73.04477629628775) bank118596 +118597 POINT(41.33609724806748 73.77381831657542) bank118597 +118598 POINT(39.98371597631003 74.79009884893816) bank118598 +118599 POINT(41.3379189303403 74.08879565769087) bank118599 +118600 POINT(41.26474483364697 73.41366818899576) bank118600 +118601 POINT(41.21393491013447 74.19802975738209) bank118601 +118602 POINT(39.96141337659058 73.73721305195528) bank118602 +118603 POINT(40.107283246651775 74.76797949465085) bank118603 +118604 POINT(40.654649470464655 73.94290378280795) bank118604 +118605 POINT(40.930013700331344 73.02223164067938) bank118605 +118606 POINT(41.15475362834826 73.73787785909052) bank118606 +118607 POINT(40.992878126467225 74.50767137571671) bank118607 +118608 POINT(39.92062381123569 73.42501997357631) bank118608 +118609 POINT(41.30770803198598 74.3864536134589) bank118609 +118610 POINT(40.53440440439819 73.5916759380718) bank118610 +118611 POINT(40.92102198190676 74.49224787225195) bank118611 +118612 POINT(41.45132772273727 74.51895536209696) bank118612 +118613 POINT(39.776313555909105 73.61259477352124) bank118613 +118614 POINT(41.32916107069001 73.96734186537469) bank118614 +118615 POINT(40.956829057287685 73.17076309559843) bank118615 +118616 POINT(41.126787935605854 74.51838448686698) bank118616 +118617 POINT(41.67333002814477 74.85077898490167) bank118617 +118618 POINT(40.90238913698147 74.89256396679748) bank118618 +118619 POINT(41.61406871303407 74.59495081397876) bank118619 +118620 POINT(40.55875670342094 74.00975176951577) bank118620 +118621 POINT(40.15429967320559 74.16201199215183) bank118621 +118622 POINT(41.36471034576659 73.40397718068326) bank118622 +118623 POINT(41.66393687795444 74.52670260808048) bank118623 +118624 POINT(41.57373087464821 73.76232121541649) bank118624 +118625 POINT(40.75037374633475 73.75179607983782) bank118625 +118626 POINT(39.86707490315608 74.67871387611525) bank118626 +118627 POINT(39.72214599112226 73.67400696534371) bank118627 +118628 POINT(41.36124251250007 73.64764490669756) bank118628 +118629 POINT(40.260363751268066 73.26331906489985) bank118629 +118630 POINT(40.8754593217861 73.71591289828618) bank118630 +118631 POINT(40.67136438342392 74.00657558672305) bank118631 +118632 POINT(40.075676375048666 73.7259428869357) bank118632 +118633 POINT(41.62800654088724 74.2122805904542) bank118633 +118634 POINT(39.793199672823604 73.84785415491042) bank118634 +118635 POINT(40.585605451205026 73.7863014376223) bank118635 +118636 POINT(40.10729308840904 73.0985549091461) bank118636 +118637 POINT(41.13615582199898 73.04446904271188) bank118637 +118638 POINT(39.92908698813558 73.23964904170153) bank118638 +118639 POINT(41.337792506904776 74.07248940527592) bank118639 +118640 POINT(41.652508341804065 74.41121924718304) bank118640 +118641 POINT(41.070229832367545 73.4346502989235) bank118641 +118642 POINT(40.605851173245476 73.95525385467519) bank118642 +118643 POINT(41.12120914630072 74.49708261572502) bank118643 +118644 POINT(41.58225837400138 74.57045873238218) bank118644 +118645 POINT(40.54849320793384 73.27612320599619) bank118645 +118646 POINT(40.73146878153612 73.24062947669844) bank118646 +118647 POINT(41.28630592657272 73.35978103316202) bank118647 +118648 POINT(41.331452545458035 73.99241804033223) bank118648 +118649 POINT(39.85916624804871 74.51468599269302) bank118649 +118650 POINT(41.29841657425119 73.56737539494848) bank118650 +118651 POINT(41.29031975860246 74.82903908225629) bank118651 +118652 POINT(40.09879427811295 73.41886522406068) bank118652 +118653 POINT(41.36724757571358 73.75450482075016) bank118653 +118654 POINT(40.940932614212535 74.8698622106148) bank118654 +118655 POINT(40.44469593739767 73.36019024056804) bank118655 +118656 POINT(40.658240561932075 73.97007320133115) bank118656 +118657 POINT(40.849061688051485 73.70888207547274) bank118657 +118658 POINT(40.08446448351183 74.73988428007935) bank118658 +118659 POINT(41.18628042807764 74.84921949373773) bank118659 +118660 POINT(40.206745363227185 73.07470586671495) bank118660 +118661 POINT(40.175403573167884 73.53459201511471) bank118661 +118662 POINT(39.7683061984632 73.73822109518369) bank118662 +118663 POINT(40.92456498285163 74.27342271262678) bank118663 +118664 POINT(41.14747832095721 74.39111200040848) bank118664 +118665 POINT(41.255378965054746 73.88000112530241) bank118665 +118666 POINT(40.05380503948223 73.58799786329772) bank118666 +118667 POINT(41.30942970624282 74.4957364877993) bank118667 +118668 POINT(40.24090598684832 74.66290665156514) bank118668 +118669 POINT(39.88017093695341 74.87368937270116) bank118669 +118670 POINT(40.34010115111709 73.65989666597687) bank118670 +118671 POINT(41.22785599011841 74.05909874661276) bank118671 +118672 POINT(41.28893478021422 74.6587427810015) bank118672 +118673 POINT(40.7028389123851 73.67599779182477) bank118673 +118674 POINT(40.51526879458871 73.98012357588561) bank118674 +118675 POINT(40.60163628959784 74.48201242543706) bank118675 +118676 POINT(41.464424368997555 74.4444767274577) bank118676 +118677 POINT(40.73866983605293 74.30311394964528) bank118677 +118678 POINT(41.28486476988472 74.34699733670094) bank118678 +118679 POINT(40.3514676043528 73.78775141188082) bank118679 +118680 POINT(39.74148311780434 74.72686807452723) bank118680 +118681 POINT(41.00471642170502 74.93178533790199) bank118681 +118682 POINT(40.142382570479846 73.26706077540511) bank118682 +118683 POINT(39.75568558041215 73.42089801062531) bank118683 +118684 POINT(40.53651638851981 74.66863032162031) bank118684 +118685 POINT(41.53838438369498 74.74671745426801) bank118685 +118686 POINT(40.083466400734274 73.50527203577954) bank118686 +118687 POINT(40.88266629813428 74.43820270287709) bank118687 +118688 POINT(40.6831111475356 74.45717759133716) bank118688 +118689 POINT(39.7260772623449 74.45564495160065) bank118689 +118690 POINT(40.24103126420986 74.99741776616895) bank118690 +118691 POINT(39.74627602943749 74.0258672625851) bank118691 +118692 POINT(40.08689720753458 74.82772452727646) bank118692 +118693 POINT(39.94016603581459 74.54726361243354) bank118693 +118694 POINT(40.527623170971175 74.64796916663471) bank118694 +118695 POINT(40.87229176325008 73.7104183519383) bank118695 +118696 POINT(40.86047878366256 73.58368668067267) bank118696 +118697 POINT(40.30163831365464 74.6274624471172) bank118697 +118698 POINT(39.80329944591952 73.43159813303173) bank118698 +118699 POINT(41.09333887706841 73.10440359848116) bank118699 +118700 POINT(41.10746615340117 73.75059149869917) bank118700 +118701 POINT(40.741595945210186 73.09747525029886) bank118701 +118702 POINT(40.861563497455066 73.17149079750551) bank118702 +118703 POINT(40.895758220384 74.3893200210273) bank118703 +118704 POINT(40.87799720798739 73.19886261065223) bank118704 +118705 POINT(41.27847597444341 74.40681005936185) bank118705 +118706 POINT(39.90624580754529 74.73856781845905) bank118706 +118707 POINT(41.5253243601029 74.07193092285493) bank118707 +118708 POINT(39.86196544167605 73.57780855287588) bank118708 +118709 POINT(40.071515712934385 74.9220377952354) bank118709 +118710 POINT(40.61462219606082 74.01742421899294) bank118710 +118711 POINT(41.344210719756525 74.65399431109677) bank118711 +118712 POINT(40.202027024648956 73.53210140807174) bank118712 +118713 POINT(39.969922729925436 73.92196181592706) bank118713 +118714 POINT(40.20276792038062 74.57435591526932) bank118714 +118715 POINT(40.86618858506867 73.82813333025474) bank118715 +118716 POINT(41.00450108572234 73.92875561221572) bank118716 +118717 POINT(40.57601041350097 74.23492666035658) bank118717 +118718 POINT(41.61399342830452 74.97108489003827) bank118718 +118719 POINT(39.88349277565271 73.90524610165008) bank118719 +118720 POINT(40.623435911051956 74.466512880253) bank118720 +118721 POINT(41.086728208536485 73.03205638186834) bank118721 +118722 POINT(39.97293826462951 74.35459784294228) bank118722 +118723 POINT(39.93149582442633 74.32794853174505) bank118723 +118724 POINT(40.563766123186774 74.86282218383951) bank118724 +118725 POINT(40.9283300367756 74.3840790335128) bank118725 +118726 POINT(40.299443534486855 73.89125040050793) bank118726 +118727 POINT(40.192739230150124 74.57615554234569) bank118727 +118728 POINT(40.673408718458695 73.49193941589888) bank118728 +118729 POINT(39.88665295518039 74.46565188545334) bank118729 +118730 POINT(41.14114150589562 74.77820121379013) bank118730 +118731 POINT(39.847881071974506 74.09592973952488) bank118731 +118732 POINT(40.9461552131038 73.87877045104239) bank118732 +118733 POINT(39.75593224870123 74.84910046287784) bank118733 +118734 POINT(41.654128272017886 74.53626242448749) bank118734 +118735 POINT(40.32939527815909 74.31588936597363) bank118735 +118736 POINT(40.17998937547274 74.96734500405113) bank118736 +118737 POINT(40.948110453109784 74.88680054888927) bank118737 +118738 POINT(41.14699232148174 74.76258972833237) bank118738 +118739 POINT(40.79240231820076 73.99465841700312) bank118739 +118740 POINT(40.867596112212794 74.49547386199126) bank118740 +118741 POINT(39.920843034140624 74.25894760762972) bank118741 +118742 POINT(40.94749161031113 73.13812925045907) bank118742 +118743 POINT(41.23566834385909 73.75106915482561) bank118743 +118744 POINT(41.2032542264568 73.0306884551428) bank118744 +118745 POINT(40.684940929547345 74.66341961043264) bank118745 +118746 POINT(40.84828230507159 74.10674672086698) bank118746 +118747 POINT(40.95670775479043 73.92328898766549) bank118747 +118748 POINT(41.52992577083998 73.97296264638358) bank118748 +118749 POINT(41.401204248741294 74.7161663507558) bank118749 +118750 POINT(40.39203014088851 74.88216080142293) bank118750 +118751 POINT(41.54396132764014 74.04018506057064) bank118751 +118752 POINT(41.3212462879366 73.73131974938386) bank118752 +118753 POINT(40.759105389082954 74.46571109158451) bank118753 +118754 POINT(39.81457895148818 73.78197421821106) bank118754 +118755 POINT(40.100592853335804 74.0945329696416) bank118755 +118756 POINT(39.86634759219624 74.13892166445629) bank118756 +118757 POINT(39.801545716072525 74.65404999707042) bank118757 +118758 POINT(40.089555108987426 74.19931680970681) bank118758 +118759 POINT(40.3233848444583 74.61760943034807) bank118759 +118760 POINT(41.479734053127004 73.08244242081024) bank118760 +118761 POINT(40.70973527125771 73.13411188681953) bank118761 +118762 POINT(39.933827116714546 73.07772769640499) bank118762 +118763 POINT(40.20207546981025 73.43498425627523) bank118763 +118764 POINT(39.8162632226731 73.7408922386567) bank118764 +118765 POINT(40.174350599608374 73.85448399963808) bank118765 +118766 POINT(41.46311163218307 74.03531885332244) bank118766 +118767 POINT(41.68890733550131 73.99088738689731) bank118767 +118768 POINT(41.043461395901765 74.7915513095588) bank118768 +118769 POINT(40.201266431507456 74.80338581001854) bank118769 +118770 POINT(39.72569404882515 73.88981591202374) bank118770 +118771 POINT(40.88942428446164 73.56149764100681) bank118771 +118772 POINT(40.93998876669418 73.50118152888265) bank118772 +118773 POINT(41.28493668727896 73.42293304825269) bank118773 +118774 POINT(39.997889866794864 73.94232721703068) bank118774 +118775 POINT(39.959344016692036 73.5683901286002) bank118775 +118776 POINT(40.31110386255953 73.40245197102288) bank118776 +118777 POINT(40.72967663698899 74.50113337535385) bank118777 +118778 POINT(41.0291802448127 73.98597577854441) bank118778 +118779 POINT(40.00881627535774 73.82033805404923) bank118779 +118780 POINT(40.17291824121951 74.47403011669995) bank118780 +118781 POINT(40.03203070191169 74.7634815178879) bank118781 +118782 POINT(40.13819479005843 74.12782794602694) bank118782 +118783 POINT(41.15362428437632 73.6410929880335) bank118783 +118784 POINT(39.79496419995942 74.0829354164114) bank118784 +118785 POINT(41.06926724043578 74.86657693963762) bank118785 +118786 POINT(40.24249660748348 73.95551038369828) bank118786 +118787 POINT(39.717151638896624 73.39086235359214) bank118787 +118788 POINT(40.540201931336426 74.93780526420349) bank118788 +118789 POINT(41.114526634615714 73.3723331841175) bank118789 +118790 POINT(40.30102928784239 73.37517682603013) bank118790 +118791 POINT(41.40935550646986 73.20257240267736) bank118791 +118792 POINT(40.72761471399489 73.45472697597265) bank118792 +118793 POINT(40.65132303137765 74.63411070715794) bank118793 +118794 POINT(40.36566034296708 73.74600500524686) bank118794 +118795 POINT(40.96527569108746 73.12841631909085) bank118795 +118796 POINT(41.71124670312318 74.77366017083854) bank118796 +118797 POINT(40.569429016470416 74.43180091790407) bank118797 +118798 POINT(41.03973075653262 73.01076449840623) bank118798 +118799 POINT(41.293828629118316 74.94033265217216) bank118799 +118800 POINT(40.71117791229859 73.4749943417084) bank118800 +118801 POINT(40.307688825967425 73.7909662101549) bank118801 +118802 POINT(39.815216535138426 73.08168545487402) bank118802 +118803 POINT(40.13215010044419 73.28458483749907) bank118803 +118804 POINT(41.68156385037186 74.12180102728706) bank118804 +118805 POINT(40.1225772022923 73.06842300818568) bank118805 +118806 POINT(40.432153141521795 73.72250230591668) bank118806 +118807 POINT(41.66649070005866 73.2391970757329) bank118807 +118808 POINT(40.38801114039777 73.6577210494958) bank118808 +118809 POINT(40.311744132297 74.72464993766258) bank118809 +118810 POINT(41.35640046568423 73.09037478717795) bank118810 +118811 POINT(40.469026089895564 73.75314624508984) bank118811 +118812 POINT(41.04953351379992 73.23393891562802) bank118812 +118813 POINT(40.909965200950744 74.94163987913637) bank118813 +118814 POINT(40.65604793475003 74.39779991114429) bank118814 +118815 POINT(40.999989425009616 73.57020292083791) bank118815 +118816 POINT(40.199270153138436 73.10568956727114) bank118816 +118817 POINT(40.68551989230723 74.24860112844944) bank118817 +118818 POINT(40.62765813969448 74.1691791690125) bank118818 +118819 POINT(41.49427376450535 73.06560075683562) bank118819 +118820 POINT(39.82956505774901 74.98920449041842) bank118820 +118821 POINT(40.22293345347955 73.4269770004987) bank118821 +118822 POINT(41.08172919475236 74.8330240206106) bank118822 +118823 POINT(40.12982985366774 74.06409994946736) bank118823 +118824 POINT(39.983978123538805 73.31752670293228) bank118824 +118825 POINT(39.87070528777291 73.20160302122615) bank118825 +118826 POINT(41.63768540609753 73.78621869813755) bank118826 +118827 POINT(40.41133045160778 74.52762622318521) bank118827 +118828 POINT(40.12481921307773 74.38749705731752) bank118828 +118829 POINT(40.413329331835534 74.42073221990302) bank118829 +118830 POINT(40.545159104619465 73.59807330618234) bank118830 +118831 POINT(40.49255856493284 74.3562198653511) bank118831 +118832 POINT(40.26106633614497 74.03122126205679) bank118832 +118833 POINT(41.38991557436217 73.53385286880409) bank118833 +118834 POINT(40.29551554194145 74.93382811593678) bank118834 +118835 POINT(41.65542464802804 74.40083746569825) bank118835 +118836 POINT(41.178473073136125 73.01061927157646) bank118836 +118837 POINT(40.96790771536621 73.16244628678915) bank118837 +118838 POINT(40.670766358950395 73.24397973496022) bank118838 +118839 POINT(41.37316558127775 74.66850218962011) bank118839 +118840 POINT(41.52221455092292 73.0400546675214) bank118840 +118841 POINT(40.25758012652482 74.08913439167218) bank118841 +118842 POINT(41.42086996940421 74.95037496514936) bank118842 +118843 POINT(41.63615622041889 74.9805464386297) bank118843 +118844 POINT(41.05116053230802 73.13502485102369) bank118844 +118845 POINT(41.024018341153294 74.86681554272192) bank118845 +118846 POINT(40.61045628887296 74.77214417213187) bank118846 +118847 POINT(41.7014574130809 74.56602503799803) bank118847 +118848 POINT(40.73317518687427 74.59922114807556) bank118848 +118849 POINT(40.44821185731788 74.51077764516964) bank118849 +118850 POINT(40.548975628960875 73.30224050761325) bank118850 +118851 POINT(41.16053514806524 73.56390092091019) bank118851 +118852 POINT(40.37971986647011 73.95942721513181) bank118852 +118853 POINT(40.18495626637831 73.40996524743329) bank118853 +118854 POINT(40.164023208371056 74.9009894539442) bank118854 +118855 POINT(40.140972899271475 73.75534872389795) bank118855 +118856 POINT(40.793249921557795 74.1647671390126) bank118856 +118857 POINT(41.68475851077719 74.67532528779259) bank118857 +118858 POINT(40.80323847784193 73.50706812320517) bank118858 +118859 POINT(40.079440285400956 73.8254043635609) bank118859 +118860 POINT(40.332666519839094 73.47645566100194) bank118860 +118861 POINT(41.34762434804095 74.3401133503394) bank118861 +118862 POINT(39.92870753636944 73.4425651831407) bank118862 +118863 POINT(40.2025159719804 73.02611675817555) bank118863 +118864 POINT(40.99163622742281 74.2043334453836) bank118864 +118865 POINT(41.15280100219705 73.57115480198081) bank118865 +118866 POINT(41.26190589838142 74.82335438848389) bank118866 +118867 POINT(39.9623765912494 74.0784709392224) bank118867 +118868 POINT(41.563781746986045 74.6597646771371) bank118868 +118869 POINT(40.08043078086901 74.26116971591686) bank118869 +118870 POINT(39.737719410893945 73.56878366235838) bank118870 +118871 POINT(40.94903432938563 73.80920305972168) bank118871 +118872 POINT(40.86994507872431 73.66414989609122) bank118872 +118873 POINT(41.33442130229974 74.14652472490693) bank118873 +118874 POINT(39.907640087811906 73.05915308726036) bank118874 +118875 POINT(40.559756047248044 73.20212705724072) bank118875 +118876 POINT(40.79393396406159 73.62072609991543) bank118876 +118877 POINT(41.56439792405414 74.3124064905208) bank118877 +118878 POINT(40.766524478462635 74.60663545292535) bank118878 +118879 POINT(41.447754931170884 73.77520866906944) bank118879 +118880 POINT(40.70991059512132 75.00124120616825) bank118880 +118881 POINT(41.11706884782492 74.74099377742556) bank118881 +118882 POINT(39.737691252699776 74.60210460167283) bank118882 +118883 POINT(40.13662777331297 73.014667462506) bank118883 +118884 POINT(41.36505864108332 73.06704936083035) bank118884 +118885 POINT(40.51035588821481 74.58685516365676) bank118885 +118886 POINT(40.787838534072016 74.84247417182026) bank118886 +118887 POINT(40.53592968158565 73.42774707305742) bank118887 +118888 POINT(40.49118625932266 74.87398310354176) bank118888 +118889 POINT(39.937013640194316 73.83912977605563) bank118889 +118890 POINT(40.673146726949135 74.09365735166693) bank118890 +118891 POINT(40.71780383232367 73.3874459021918) bank118891 +118892 POINT(40.788598634206735 73.77817658794385) bank118892 +118893 POINT(41.17377250487911 73.5214984938084) bank118893 +118894 POINT(41.030812942221516 74.40723881522032) bank118894 +118895 POINT(40.46149151068013 73.90208610365885) bank118895 +118896 POINT(40.55162516239217 74.37011143255174) bank118896 +118897 POINT(41.07435015594212 73.33382925129585) bank118897 +118898 POINT(41.50940213901366 74.32714271189673) bank118898 +118899 POINT(40.079677667166536 73.5816649928052) bank118899 +118900 POINT(40.46930365295949 73.52934406267022) bank118900 +118901 POINT(40.62584904101843 73.19329688404865) bank118901 +118902 POINT(41.34637204844651 74.68701562121477) bank118902 +118903 POINT(40.24806774255333 73.58431080999173) bank118903 +118904 POINT(39.92741771449109 73.34486279516263) bank118904 +118905 POINT(41.460108459254485 74.92006449666293) bank118905 +118906 POINT(40.91108078510912 74.34060679056462) bank118906 +118907 POINT(40.26614145499004 73.98480466320356) bank118907 +118908 POINT(40.341027836439636 74.35280413740216) bank118908 +118909 POINT(39.87416734579047 73.07883782046673) bank118909 +118910 POINT(40.738846470057545 74.12434296364437) bank118910 +118911 POINT(40.29837231373593 74.81377043954348) bank118911 +118912 POINT(41.59481103536287 73.50822140353361) bank118912 +118913 POINT(41.24095063762333 74.62225022089925) bank118913 +118914 POINT(40.326121132136386 73.68208832856854) bank118914 +118915 POINT(40.366317209320044 74.66103566752882) bank118915 +118916 POINT(40.16037112238408 74.51501143728875) bank118916 +118917 POINT(40.34562945820196 73.68493165606995) bank118917 +118918 POINT(41.40483282530304 74.5353893636204) bank118918 +118919 POINT(41.1439902863943 73.12296303698372) bank118919 +118920 POINT(41.61011461033125 74.108169618128) bank118920 +118921 POINT(41.117659863528495 73.70081957239738) bank118921 +118922 POINT(40.88204232538152 74.21426144160523) bank118922 +118923 POINT(41.48989685349077 74.59481689058246) bank118923 +118924 POINT(41.63069522330757 74.99021385787168) bank118924 +118925 POINT(41.25347195630582 74.72552515482408) bank118925 +118926 POINT(39.831469720911215 73.24885468178512) bank118926 +118927 POINT(39.750048243911756 73.06536981722977) bank118927 +118928 POINT(40.28299715591092 73.71698529314335) bank118928 +118929 POINT(41.32510710952836 74.97712314196822) bank118929 +118930 POINT(41.06039255142073 73.88894689915686) bank118930 +118931 POINT(39.75532737908244 74.73701736300264) bank118931 +118932 POINT(40.04149879229288 73.95555405576077) bank118932 +118933 POINT(41.2578995965511 74.35065013919403) bank118933 +118934 POINT(40.999874769105816 73.81328727999446) bank118934 +118935 POINT(40.68728909428664 74.94467909178795) bank118935 +118936 POINT(40.36702842630773 73.60135598959714) bank118936 +118937 POINT(40.802460902900066 73.71933099348476) bank118937 +118938 POINT(39.90283375295458 73.4256916356273) bank118938 +118939 POINT(39.751643897158715 74.9142849866021) bank118939 +118940 POINT(41.26839564707181 73.79589645340987) bank118940 +118941 POINT(39.88074319040818 74.6177170131393) bank118941 +118942 POINT(39.7575173259867 74.63980462623388) bank118942 +118943 POINT(39.76334404504602 73.76197474537365) bank118943 +118944 POINT(39.96668488073934 73.20246842393794) bank118944 +118945 POINT(40.951595850935476 73.21874331347978) bank118945 +118946 POINT(41.2316153821133 73.36635993704206) bank118946 +118947 POINT(41.533988609873646 74.36252094057701) bank118947 +118948 POINT(40.717772413486976 74.24494081712693) bank118948 +118949 POINT(40.1083699969803 73.68596412515865) bank118949 +118950 POINT(40.63054759454213 73.3805675859624) bank118950 +118951 POINT(40.29176040434436 74.21222352168992) bank118951 +118952 POINT(40.04216138978821 74.54279016137005) bank118952 +118953 POINT(40.839291053343416 73.63066924652888) bank118953 +118954 POINT(40.30422688114886 74.2610431649393) bank118954 +118955 POINT(40.62017784506323 74.21871061105513) bank118955 +118956 POINT(40.887462895710186 74.0238286495123) bank118956 +118957 POINT(40.147965817724334 73.41774827860952) bank118957 +118958 POINT(40.650830601570256 73.60605543342655) bank118958 +118959 POINT(40.283094500087564 73.95891872389817) bank118959 +118960 POINT(39.945108342190814 73.83715227489252) bank118960 +118961 POINT(40.548437299144545 73.9986015877541) bank118961 +118962 POINT(41.486043014544556 73.4760773443704) bank118962 +118963 POINT(40.87320344695411 73.23405191340346) bank118963 +118964 POINT(40.1682243114538 74.3995279685886) bank118964 +118965 POINT(40.13496793592894 74.62925626320775) bank118965 +118966 POINT(40.70973809094068 73.01178134658247) bank118966 +118967 POINT(41.61516830487234 73.2726923501418) bank118967 +118968 POINT(39.73988527491672 74.465537707021) bank118968 +118969 POINT(41.105998946584315 74.81690573674489) bank118969 +118970 POINT(39.98807527028773 73.93482116245077) bank118970 +118971 POINT(40.1128743854246 73.63471204005526) bank118971 +118972 POINT(40.39181070450438 74.21364946048413) bank118972 +118973 POINT(41.227141439837226 73.23125516217047) bank118973 +118974 POINT(40.32778078864475 74.56903872850917) bank118974 +118975 POINT(40.99220713319288 73.62583171711601) bank118975 +118976 POINT(41.62174283984072 73.84001048943517) bank118976 +118977 POINT(40.06205514903751 73.07609313419837) bank118977 +118978 POINT(40.66825761907899 74.02954948625299) bank118978 +118979 POINT(41.132345756460744 74.03628579267586) bank118979 +118980 POINT(40.969822893484796 74.6555679639394) bank118980 +118981 POINT(41.308342001006366 74.50670849074113) bank118981 +118982 POINT(41.223631765042974 73.64448294898457) bank118982 +118983 POINT(41.480601780662006 73.21798061406857) bank118983 +118984 POINT(40.3017155684887 74.25152398839724) bank118984 +118985 POINT(40.609491173453954 73.28563126994625) bank118985 +118986 POINT(40.96972721176402 74.27130760167871) bank118986 +118987 POINT(41.452586222921475 73.98186263378176) bank118987 +118988 POINT(41.18859539532074 73.76340547213925) bank118988 +118989 POINT(40.267024493852965 74.11755220348923) bank118989 +118990 POINT(41.24438124625436 74.50855657141594) bank118990 +118991 POINT(40.89185602182809 74.47160728726023) bank118991 +118992 POINT(41.17875185594414 73.45529735662768) bank118992 +118993 POINT(39.78664939380287 73.70321640861242) bank118993 +118994 POINT(40.39849906253633 73.40342001943993) bank118994 +118995 POINT(40.53734504200087 74.70526171323245) bank118995 +118996 POINT(40.61669261677399 74.67116572616413) bank118996 +118997 POINT(41.51016638127385 74.38981542730288) bank118997 +118998 POINT(40.18250874916003 74.06843523075979) bank118998 +118999 POINT(40.917175917550324 74.73005333873898) bank118999 +119000 POINT(39.95245149494186 74.31100008544779) bank119000 +119001 POINT(41.12526207269614 73.28372297825119) bank119001 +119002 POINT(39.79994322236824 73.74980323624284) bank119002 +119003 POINT(40.49985993345842 73.02945081659558) bank119003 +119004 POINT(40.934609781447136 74.72635004191825) bank119004 +119005 POINT(39.90043502239367 73.49930607048375) bank119005 +119006 POINT(40.41117117037908 74.24166272184841) bank119006 +119007 POINT(39.884381099884415 74.94169618871204) bank119007 +119008 POINT(41.50557498352942 74.10344682180681) bank119008 +119009 POINT(40.49752106833191 74.57234218681032) bank119009 +119010 POINT(40.4384764416775 73.54412908450149) bank119010 +119011 POINT(39.76977015653971 73.82094598719584) bank119011 +119012 POINT(40.70983769228832 73.74482779064535) bank119012 +119013 POINT(41.588549032959435 74.87073288551379) bank119013 +119014 POINT(40.28905738791363 74.4411335164932) bank119014 +119015 POINT(40.654144546186224 73.1836003918088) bank119015 +119016 POINT(41.26105504716595 73.71873640054521) bank119016 +119017 POINT(41.28457801387016 73.75816365748499) bank119017 +119018 POINT(41.212763672706224 74.16270681583748) bank119018 +119019 POINT(41.62333012857371 74.52340937623144) bank119019 +119020 POINT(41.6538773339253 73.67605564052347) bank119020 +119021 POINT(41.374991735655534 73.92001494632086) bank119021 +119022 POINT(40.54004599509331 74.84680529345916) bank119022 +119023 POINT(40.0969227280352 74.63619581015824) bank119023 +119024 POINT(41.26940494649942 74.94381783079642) bank119024 +119025 POINT(41.25654009240694 74.05845175907125) bank119025 +119026 POINT(41.509789875010746 74.19022643738) bank119026 +119027 POINT(40.91172107708465 73.6942560518676) bank119027 +119028 POINT(40.79304204347717 74.11362010152521) bank119028 +119029 POINT(40.52743018538286 74.98825815946736) bank119029 +119030 POINT(40.745956544389 73.83173017840801) bank119030 +119031 POINT(41.37855151153409 74.93043827473909) bank119031 +119032 POINT(41.44732252144776 73.47273442726556) bank119032 +119033 POINT(41.554548105500274 74.62793099686107) bank119033 +119034 POINT(40.017182186306194 73.75310449505199) bank119034 +119035 POINT(40.95881888384438 73.32366544081192) bank119035 +119036 POINT(41.6279177619915 73.85897600119763) bank119036 +119037 POINT(41.39106609316201 74.00890744354668) bank119037 +119038 POINT(41.330021637455374 73.98617773271815) bank119038 +119039 POINT(39.826813416699586 73.46902272779617) bank119039 +119040 POINT(40.47310332504451 73.22073433184958) bank119040 +119041 POINT(39.722943084473805 73.97098965122815) bank119041 +119042 POINT(40.209506623479506 74.06645741049844) bank119042 +119043 POINT(40.40147040126047 74.46183614878915) bank119043 +119044 POINT(40.22606226840994 74.77719297325683) bank119044 +119045 POINT(40.65023647217323 74.80950264463338) bank119045 +119046 POINT(41.28903968025924 73.06490698660448) bank119046 +119047 POINT(41.363919597147415 73.20766981409507) bank119047 +119048 POINT(40.08289324021565 73.21160374772988) bank119048 +119049 POINT(39.97264302236894 73.6333103933611) bank119049 +119050 POINT(40.35621115298638 73.84639133258383) bank119050 +119051 POINT(40.345823182296165 73.82140872888505) bank119051 +119052 POINT(41.10958510697411 73.3815654164417) bank119052 +119053 POINT(40.92100880044556 73.01545942457122) bank119053 +119054 POINT(40.53318005291412 73.90530713542026) bank119054 +119055 POINT(41.45251549330527 73.88796886205661) bank119055 +119056 POINT(40.37132733151944 74.87186283586438) bank119056 +119057 POINT(40.87511933367133 73.4335550685185) bank119057 +119058 POINT(41.2263881620585 74.84842334860289) bank119058 +119059 POINT(40.321408048218174 74.95163963456051) bank119059 +119060 POINT(41.141013330650566 74.73450851169494) bank119060 +119061 POINT(39.80014710044744 73.5679233613019) bank119061 +119062 POINT(40.42379021816871 73.96895508130552) bank119062 +119063 POINT(40.94976538939177 74.66575433383662) bank119063 +119064 POINT(41.270782643622205 74.47073400176029) bank119064 +119065 POINT(40.394825702470555 73.51458643729619) bank119065 +119066 POINT(41.69220947518571 74.3251760283432) bank119066 +119067 POINT(39.87014047988785 74.00280042713108) bank119067 +119068 POINT(40.781815402042334 73.22435715674573) bank119068 +119069 POINT(41.315097204939136 74.75071723125849) bank119069 +119070 POINT(40.37109118655217 74.79187372726196) bank119070 +119071 POINT(39.849158160969594 73.85968488843689) bank119071 +119072 POINT(40.92788380822508 73.02238306168555) bank119072 +119073 POINT(40.69864148795831 74.42631914109582) bank119073 +119074 POINT(41.582286204956525 73.55942994524332) bank119074 +119075 POINT(40.08647176416649 74.15319920434712) bank119075 +119076 POINT(40.06964518716219 74.54848162993531) bank119076 +119077 POINT(40.51250125694173 73.80599034508646) bank119077 +119078 POINT(41.28916745631585 73.2493700917868) bank119078 +119079 POINT(40.96700360831992 73.80173601322451) bank119079 +119080 POINT(40.58085891510031 74.75340864200422) bank119080 +119081 POINT(40.729192649360876 74.35700122894139) bank119081 +119082 POINT(40.9842568083677 73.13744156385805) bank119082 +119083 POINT(40.673601977052 74.99636401399783) bank119083 +119084 POINT(40.54599208469743 74.43959076154842) bank119084 +119085 POINT(40.473991663001556 73.78984156421501) bank119085 +119086 POINT(40.776695892742055 74.13252601019471) bank119086 +119087 POINT(41.36949450919491 74.68504757825762) bank119087 +119088 POINT(41.0913042011035 73.7913606603559) bank119088 +119089 POINT(40.151483781613905 73.69189798418996) bank119089 +119090 POINT(40.54769187152 74.47808203201632) bank119090 +119091 POINT(41.514426425084885 74.5607872703821) bank119091 +119092 POINT(40.07161148474149 74.01217547258565) bank119092 +119093 POINT(40.739897319440026 74.26845812483963) bank119093 +119094 POINT(41.28769887250544 73.2626833373342) bank119094 +119095 POINT(40.386214798179864 74.59461199830731) bank119095 +119096 POINT(40.73123708002075 73.80158684047036) bank119096 +119097 POINT(41.09995473408832 73.29272074906814) bank119097 +119098 POINT(41.25287421776337 73.85817017043108) bank119098 +119099 POINT(40.644011591466196 73.95350329258042) bank119099 +119100 POINT(41.41374438353248 73.4070098948842) bank119100 +119101 POINT(41.274449990231496 73.98973047283795) bank119101 +119102 POINT(39.7694474591109 74.87262494947066) bank119102 +119103 POINT(41.20349269519641 73.58479564218709) bank119103 +119104 POINT(40.96880167665854 75.00128945339316) bank119104 +119105 POINT(39.75212389028086 74.62919571631963) bank119105 +119106 POINT(40.19140940806282 74.43817828180437) bank119106 +119107 POINT(39.97489999089315 73.60090050524781) bank119107 +119108 POINT(41.1736974549756 74.31306404714535) bank119108 +119109 POINT(41.20483865079036 74.63995701052684) bank119109 +119110 POINT(40.00799866138592 73.78312683663151) bank119110 +119111 POINT(41.4821035652087 74.26645720630492) bank119111 +119112 POINT(40.173442825411 73.85101786669999) bank119112 +119113 POINT(40.78370575659182 73.76063218120034) bank119113 +119114 POINT(39.80049411752711 74.99534937722918) bank119114 +119115 POINT(40.87280343543843 73.04527053811141) bank119115 +119116 POINT(40.956870975894795 74.45516977338424) bank119116 +119117 POINT(41.66969707154954 74.16312622077035) bank119117 +119118 POINT(39.73240353089109 73.08175310330078) bank119118 +119119 POINT(39.92798580691011 73.24148001148932) bank119119 +119120 POINT(39.91905029833446 74.25212309001847) bank119120 +119121 POINT(40.59678586357824 73.30053686995197) bank119121 +119122 POINT(39.75789694204091 73.51912755018726) bank119122 +119123 POINT(40.46255891399415 74.40681279930841) bank119123 +119124 POINT(39.81609934309819 73.59059867964389) bank119124 +119125 POINT(41.12769476098209 74.81338361947513) bank119125 +119126 POINT(40.68487267478319 73.47848363128475) bank119126 +119127 POINT(40.92510992377659 73.35631644798656) bank119127 +119128 POINT(40.9968500747613 74.79479754390138) bank119128 +119129 POINT(40.50748378206194 74.39730211981107) bank119129 +119130 POINT(41.60845399211473 74.62357772599896) bank119130 +119131 POINT(40.88762607682922 73.27005103127388) bank119131 +119132 POINT(41.16790903259622 74.8810570877365) bank119132 +119133 POINT(40.90684960433118 73.10309271288715) bank119133 +119134 POINT(40.425721301816665 73.73001800947314) bank119134 +119135 POINT(41.27552508126123 74.95699601053373) bank119135 +119136 POINT(41.35603628316614 73.25585986045492) bank119136 +119137 POINT(40.13884840681471 73.42006258926621) bank119137 +119138 POINT(41.2068781589901 74.72224288390345) bank119138 +119139 POINT(41.550187226562215 74.38879119764029) bank119139 +119140 POINT(41.204706736690625 73.32700375943736) bank119140 +119141 POINT(41.144267515606955 73.05914930518881) bank119141 +119142 POINT(41.52386213405034 74.44184858828028) bank119142 +119143 POINT(40.065507714005726 73.69347350528844) bank119143 +119144 POINT(39.81727304412129 74.15796199191607) bank119144 +119145 POINT(41.006527730728926 73.11257219637828) bank119145 +119146 POINT(40.51910339672268 73.45266002344172) bank119146 +119147 POINT(40.55758294573338 73.50096888990556) bank119147 +119148 POINT(41.6574940269082 74.89355813414628) bank119148 +119149 POINT(40.344621175719794 74.2684721184939) bank119149 +119150 POINT(40.04703074592754 74.36939784631275) bank119150 +119151 POINT(40.74377275096651 73.45120050619308) bank119151 +119152 POINT(40.905156498893064 73.6207662931147) bank119152 +119153 POINT(41.16049912674037 74.72312492027821) bank119153 +119154 POINT(39.86797843941914 73.73468955664805) bank119154 +119155 POINT(41.35553913867234 74.83348934486813) bank119155 +119156 POINT(41.350493413733055 73.49589248141974) bank119156 +119157 POINT(40.153471857906425 74.26329940936135) bank119157 +119158 POINT(41.25189866216548 74.42045953617792) bank119158 +119159 POINT(40.43297577515364 74.10578566397126) bank119159 +119160 POINT(40.14865164198897 74.36870364527589) bank119160 +119161 POINT(40.36051169375599 73.9190428026451) bank119161 +119162 POINT(41.3475209742358 74.84227141566846) bank119162 +119163 POINT(40.33642480370337 73.44571723651154) bank119163 +119164 POINT(40.988943243915585 74.84792556743542) bank119164 +119165 POINT(39.720461367633064 73.19776006514513) bank119165 +119166 POINT(40.36354524098216 73.51403297866952) bank119166 +119167 POINT(41.303930267584995 74.30382168997198) bank119167 +119168 POINT(40.192593652278234 74.38695764663703) bank119168 +119169 POINT(40.54477817195553 74.29449577246281) bank119169 +119170 POINT(41.67425776075187 73.06837055293684) bank119170 +119171 POINT(40.35965852868697 73.24765488146515) bank119171 +119172 POINT(40.02101469270805 73.42837495573026) bank119172 +119173 POINT(40.42546324095793 74.49696130955292) bank119173 +119174 POINT(40.80294360359092 73.27846636593351) bank119174 +119175 POINT(40.452034941431585 74.26815260374025) bank119175 +119176 POINT(40.656363497583826 73.93460957450932) bank119176 +119177 POINT(41.243047224022604 73.78654335853652) bank119177 +119178 POINT(40.551674872374015 74.40866949804229) bank119178 +119179 POINT(41.17894410392143 73.27033881428216) bank119179 +119180 POINT(40.85373668175996 73.29891822154052) bank119180 +119181 POINT(40.59839499952378 74.98743185125045) bank119181 +119182 POINT(40.07774230807879 74.64291385177528) bank119182 +119183 POINT(40.2546173709218 73.94803597848531) bank119183 +119184 POINT(40.64884129011638 73.95550680451154) bank119184 +119185 POINT(41.32904201907912 73.19977275229972) bank119185 +119186 POINT(41.64520699802711 73.94947617635235) bank119186 +119187 POINT(40.098688266808836 73.11366951035139) bank119187 +119188 POINT(40.433892853423586 73.1417393321584) bank119188 +119189 POINT(41.16070556755818 74.8818918258566) bank119189 +119190 POINT(39.96195588927289 73.06538839217309) bank119190 +119191 POINT(40.67674634647596 74.98715936928107) bank119191 +119192 POINT(41.6282410154811 73.99274105467435) bank119192 +119193 POINT(40.72032371503623 73.28933964456284) bank119193 +119194 POINT(40.36414898290912 74.86978659935508) bank119194 +119195 POINT(40.109779926209065 74.1648160742302) bank119195 +119196 POINT(40.18184883258336 74.49669656849501) bank119196 +119197 POINT(40.32517364054097 74.52728669536785) bank119197 +119198 POINT(40.62418215388267 73.09022047100842) bank119198 +119199 POINT(40.71575117225696 73.70118350466193) bank119199 +119200 POINT(40.598791568250945 74.12474600730623) bank119200 +119201 POINT(40.33014182505634 74.6644913138422) bank119201 +119202 POINT(41.40316413857181 74.430759908892) bank119202 +119203 POINT(40.9310372748367 73.53847399582334) bank119203 +119204 POINT(40.09078085577851 73.80430533446298) bank119204 +119205 POINT(40.055507647062896 74.05901836369645) bank119205 +119206 POINT(41.30193174988039 74.37436705371341) bank119206 +119207 POINT(40.23056067705837 73.89232275032907) bank119207 +119208 POINT(40.4500179295806 74.29539957122287) bank119208 +119209 POINT(41.59715187246725 73.72251675406403) bank119209 +119210 POINT(41.104567404972144 74.30174232088855) bank119210 +119211 POINT(41.00766661757647 73.33941588466394) bank119211 +119212 POINT(40.948881889898914 74.6306945326901) bank119212 +119213 POINT(40.42670788728041 73.6944090124665) bank119213 +119214 POINT(39.75556992099899 74.43908534058538) bank119214 +119215 POINT(40.597434878117745 73.13123156254194) bank119215 +119216 POINT(41.02079734363414 73.93072439187051) bank119216 +119217 POINT(40.79499436153695 73.11042506264285) bank119217 +119218 POINT(40.070216895548 73.63404195414127) bank119218 +119219 POINT(41.45575544800798 74.76852949041802) bank119219 +119220 POINT(40.0150147146583 74.20997747197072) bank119220 +119221 POINT(40.892102974009546 73.53831345648803) bank119221 +119222 POINT(40.4737342681967 74.57498701234796) bank119222 +119223 POINT(40.99595094184153 73.28703612622795) bank119223 +119224 POINT(39.88345475147623 74.28882396295454) bank119224 +119225 POINT(39.80280770392614 73.43184552633721) bank119225 +119226 POINT(40.396265878318616 73.98898125001759) bank119226 +119227 POINT(41.39614195818419 73.20950733630353) bank119227 +119228 POINT(39.961858268831215 73.86670090266996) bank119228 +119229 POINT(40.60486075612206 73.41327201500327) bank119229 +119230 POINT(41.42077424482798 73.43070970270465) bank119230 +119231 POINT(40.33493866133114 74.30056130316373) bank119231 +119232 POINT(41.45083215509457 73.10741422782893) bank119232 +119233 POINT(40.83963821420393 74.58152134660408) bank119233 +119234 POINT(39.99388404634324 74.4101728607586) bank119234 +119235 POINT(41.586045989532494 73.28599929397011) bank119235 +119236 POINT(41.66602923009459 74.9472177011261) bank119236 +119237 POINT(41.112247794731154 74.97545686840735) bank119237 +119238 POINT(40.8506075608087 74.50671663833388) bank119238 +119239 POINT(40.8570350542414 74.12914625244585) bank119239 +119240 POINT(41.46054424313336 73.98946175432887) bank119240 +119241 POINT(40.05312587207296 73.45714403180958) bank119241 +119242 POINT(40.31990120949604 74.15338242948789) bank119242 +119243 POINT(40.89496180800975 73.1835391319618) bank119243 +119244 POINT(40.91457418389818 73.85918670237878) bank119244 +119245 POINT(40.5497872455603 73.28368190836964) bank119245 +119246 POINT(39.85619245888922 73.67485954735874) bank119246 +119247 POINT(40.10282129272585 74.30540864894893) bank119247 +119248 POINT(41.53713686428813 74.90917350687883) bank119248 +119249 POINT(40.370343136374586 74.16202139901394) bank119249 +119250 POINT(40.16539828130119 74.36209652560788) bank119250 +119251 POINT(40.31074270128941 73.54811267618109) bank119251 +119252 POINT(41.06789646764661 73.93733194915544) bank119252 +119253 POINT(41.27100204480243 74.77290745953727) bank119253 +119254 POINT(40.20283605082668 74.2463263032884) bank119254 +119255 POINT(41.599305823067624 73.86530847604536) bank119255 +119256 POINT(41.361460514697846 73.02516202064898) bank119256 +119257 POINT(41.669186048302485 73.55135339410431) bank119257 +119258 POINT(41.450896381512635 73.56171183954996) bank119258 +119259 POINT(40.66491585439352 74.79177978660813) bank119259 +119260 POINT(40.63952140447326 74.7981897056029) bank119260 +119261 POINT(40.00038049485241 73.20681249809735) bank119261 +119262 POINT(40.71391138120658 73.09201665434195) bank119262 +119263 POINT(40.357040714237584 73.76774077570379) bank119263 +119264 POINT(40.281506349556864 74.07951110468773) bank119264 +119265 POINT(41.18204220868751 74.46861891137377) bank119265 +119266 POINT(41.68883060000062 73.3496170187788) bank119266 +119267 POINT(41.13243182721863 73.43259387249411) bank119267 +119268 POINT(40.75235373111111 74.08442687645537) bank119268 +119269 POINT(40.581742359499145 74.48408606093348) bank119269 +119270 POINT(41.05727668287478 73.24243261089147) bank119270 +119271 POINT(41.43290433428655 73.05920778404831) bank119271 +119272 POINT(40.48885126246153 74.42904557043882) bank119272 +119273 POINT(40.65420030184608 73.54168331548817) bank119273 +119274 POINT(41.444758753552286 74.17724987165563) bank119274 +119275 POINT(40.08109852071166 74.0431970991237) bank119275 +119276 POINT(41.017746392527506 73.29375064177539) bank119276 +119277 POINT(41.458321228065 73.414628162495) bank119277 +119278 POINT(41.06475573588217 74.82221674706538) bank119278 +119279 POINT(41.3327224167731 73.97747269371727) bank119279 +119280 POINT(41.66809219615435 74.77263055943057) bank119280 +119281 POINT(41.60057089413452 74.49812815147862) bank119281 +119282 POINT(40.32227896234936 74.32850253100969) bank119282 +119283 POINT(40.982743460438726 74.3571118960751) bank119283 +119284 POINT(39.733959356979355 73.26719097479713) bank119284 +119285 POINT(40.77507515082593 74.45717342507943) bank119285 +119286 POINT(40.99306996354608 73.25890511715858) bank119286 +119287 POINT(39.713044821879 73.03669292687836) bank119287 +119288 POINT(39.860066080176 73.25880990892486) bank119288 +119289 POINT(41.13417442782578 73.65747838212648) bank119289 +119290 POINT(41.45626148116547 73.42695897855623) bank119290 +119291 POINT(41.10280420912822 74.97374760907013) bank119291 +119292 POINT(40.72288552445683 74.13186610167794) bank119292 +119293 POINT(41.41246711303583 73.12784869540923) bank119293 +119294 POINT(39.90065760264629 73.58186580389145) bank119294 +119295 POINT(40.570991065673816 74.17730682108846) bank119295 +119296 POINT(41.01554636218881 73.59443947790841) bank119296 +119297 POINT(40.46115244120311 73.64018617746763) bank119297 +119298 POINT(41.051528006793276 73.46991222878944) bank119298 +119299 POINT(39.755128330337776 74.39108346174292) bank119299 +119300 POINT(40.28011127888327 73.61519778581622) bank119300 +119301 POINT(40.91399236742623 74.508914840165) bank119301 +119302 POINT(40.9975116810666 73.91925588085552) bank119302 +119303 POINT(40.0424833124447 73.45780869991653) bank119303 +119304 POINT(40.408866883371346 74.77143641368947) bank119304 +119305 POINT(40.69305097494952 74.77599973261293) bank119305 +119306 POINT(39.903777842589015 74.63881934471462) bank119306 +119307 POINT(40.02471881232953 74.6013737135718) bank119307 +119308 POINT(40.695038221414386 74.61709510275558) bank119308 +119309 POINT(41.16161830853074 74.68088557131284) bank119309 +119310 POINT(40.75739698324878 73.24363037317734) bank119310 +119311 POINT(40.479321346146854 74.13886294041792) bank119311 +119312 POINT(40.466026010994355 73.81707145068151) bank119312 +119313 POINT(41.02499340862274 74.83861669943693) bank119313 +119314 POINT(40.80425105373801 74.0058259384934) bank119314 +119315 POINT(40.97286101358644 74.07822304112601) bank119315 +119316 POINT(41.60901879069712 73.39150116468826) bank119316 +119317 POINT(40.14407207586934 73.36049122441985) bank119317 +119318 POINT(41.229805487837396 73.93518634686856) bank119318 +119319 POINT(40.26426158813425 74.14584218087319) bank119319 +119320 POINT(41.48390830562853 74.7540537317429) bank119320 +119321 POINT(40.577400460881286 73.50288779847135) bank119321 +119322 POINT(40.45168375707634 73.64144195133782) bank119322 +119323 POINT(41.377608520263436 73.68460240246704) bank119323 +119324 POINT(40.99776907736779 73.49115227887833) bank119324 +119325 POINT(40.232533551457124 74.09348754783842) bank119325 +119326 POINT(40.04023279995603 74.49288018359574) bank119326 +119327 POINT(40.73757561250053 74.7416603089038) bank119327 +119328 POINT(40.50128416266633 73.02908375746114) bank119328 +119329 POINT(41.328478205022655 74.66992107920105) bank119329 +119330 POINT(40.38527710248857 74.18796122092246) bank119330 +119331 POINT(40.56120595532661 74.1767221868958) bank119331 +119332 POINT(40.38683390845396 74.29257837129781) bank119332 +119333 POINT(41.16478279221577 73.2201164507568) bank119333 +119334 POINT(39.88029339743764 74.73211523625294) bank119334 +119335 POINT(40.40766279271109 74.3523158715964) bank119335 +119336 POINT(39.89688796979438 73.43638026570683) bank119336 +119337 POINT(40.32149927123944 73.95616440657045) bank119337 +119338 POINT(39.84842166529455 74.88874098131171) bank119338 +119339 POINT(40.557158063353285 74.0145522335024) bank119339 +119340 POINT(40.963829104663645 73.22350515905282) bank119340 +119341 POINT(40.96382247363321 73.52845816972672) bank119341 +119342 POINT(40.457436875035526 74.37684813256071) bank119342 +119343 POINT(40.619677900530405 74.67689914740323) bank119343 +119344 POINT(40.789609429488436 74.2262779166455) bank119344 +119345 POINT(41.13839538770522 73.32677303709032) bank119345 +119346 POINT(40.964104970366755 74.5802341774035) bank119346 +119347 POINT(40.42541137437833 73.99616320881971) bank119347 +119348 POINT(39.79067532899761 73.31717016159207) bank119348 +119349 POINT(40.231105067815 74.37358817812378) bank119349 +119350 POINT(40.13250488276771 74.08380428365791) bank119350 +119351 POINT(41.11285375863526 74.47302656000237) bank119351 +119352 POINT(41.16690759304061 73.74069424964333) bank119352 +119353 POINT(41.20062971105144 74.87744815190933) bank119353 +119354 POINT(41.33302991167782 74.48756394446366) bank119354 +119355 POINT(41.53145011623337 74.7350943139907) bank119355 +119356 POINT(40.56418235327602 74.40685411834308) bank119356 +119357 POINT(41.22067099000892 74.51710045629923) bank119357 +119358 POINT(40.409055486148795 74.78096807267649) bank119358 +119359 POINT(41.301254533532 73.79428952155702) bank119359 +119360 POINT(39.949135277352696 74.36773718675778) bank119360 +119361 POINT(40.187791782168624 73.97609705043666) bank119361 +119362 POINT(39.78298707826007 74.2536843776855) bank119362 +119363 POINT(39.9846199037178 73.97733762249182) bank119363 +119364 POINT(40.938436824300474 73.45539008329754) bank119364 +119365 POINT(39.763627662065375 75.00239252585462) bank119365 +119366 POINT(40.8411523422427 73.72495596680045) bank119366 +119367 POINT(40.11606610676189 74.24690780290477) bank119367 +119368 POINT(41.072960812301105 73.73573682335406) bank119368 +119369 POINT(40.112743474036435 73.58075816833039) bank119369 +119370 POINT(39.87329979078949 73.88944812134757) bank119370 +119371 POINT(41.58302974358555 74.25656543838018) bank119371 +119372 POINT(40.31790931235087 74.50072278818509) bank119372 +119373 POINT(40.43872942088807 73.03615785078117) bank119373 +119374 POINT(40.140360644669364 74.23707707378705) bank119374 +119375 POINT(40.48571180350358 73.85717245305204) bank119375 +119376 POINT(41.111274142298726 73.97517019168414) bank119376 +119377 POINT(40.564870367479884 73.39860985342106) bank119377 +119378 POINT(40.211405398071044 73.80222557420818) bank119378 +119379 POINT(41.6332884218654 73.13350237211014) bank119379 +119380 POINT(41.31495305868617 73.30069501223295) bank119380 +119381 POINT(40.877623689046594 74.12751068119752) bank119381 +119382 POINT(40.61262232173514 73.1702785729051) bank119382 +119383 POINT(41.386263311632334 73.46484186129159) bank119383 +119384 POINT(41.55175727155824 73.96882493982267) bank119384 +119385 POINT(40.79301361817326 73.2892469170587) bank119385 +119386 POINT(40.40118655478926 73.35301889117292) bank119386 +119387 POINT(41.05764492625523 74.69251101279171) bank119387 +119388 POINT(40.26500780998688 74.92695707959389) bank119388 +119389 POINT(39.77097241088567 73.29350885055787) bank119389 +119390 POINT(41.632716577407436 74.32922798230408) bank119390 +119391 POINT(40.444559957299354 74.0396471100925) bank119391 +119392 POINT(41.28489095826483 74.02389425643946) bank119392 +119393 POINT(40.56443255970085 73.03767549189892) bank119393 +119394 POINT(40.7538896616936 74.44447940395455) bank119394 +119395 POINT(41.12746676371716 74.17618415587631) bank119395 +119396 POINT(40.596895320721636 74.4771756142484) bank119396 +119397 POINT(40.76058489837585 74.15985585194997) bank119397 +119398 POINT(40.03691032239949 73.96510546641538) bank119398 +119399 POINT(41.626528016195486 74.76397526990681) bank119399 +119400 POINT(41.21866321797623 73.26339098757973) bank119400 +119401 POINT(41.616262026394125 73.81203628719814) bank119401 +119402 POINT(40.908759192245725 74.62835728090937) bank119402 +119403 POINT(40.13669584794956 74.26996211465956) bank119403 +119404 POINT(40.575688891927264 73.41530050755813) bank119404 +119405 POINT(41.50542225871003 74.12763395780881) bank119405 +119406 POINT(41.588244514484366 74.80262909683547) bank119406 +119407 POINT(41.672414610113194 73.58752427741997) bank119407 +119408 POINT(41.078368797407606 73.25024420211015) bank119408 +119409 POINT(40.44364436429296 74.13420844529657) bank119409 +119410 POINT(40.851056239595216 74.44432687077436) bank119410 +119411 POINT(41.62951207736472 73.06013612506528) bank119411 +119412 POINT(39.93323260454683 73.11306700028865) bank119412 +119413 POINT(40.51530183323069 73.6025779866222) bank119413 +119414 POINT(40.84733329400049 74.75495219573189) bank119414 +119415 POINT(40.57807329246958 73.70406412244373) bank119415 +119416 POINT(40.201223170870406 74.73221244015907) bank119416 +119417 POINT(41.28307747160491 74.73467681519493) bank119417 +119418 POINT(39.84761298675625 73.26768995182522) bank119418 +119419 POINT(40.00011103670532 73.12913872309832) bank119419 +119420 POINT(40.17195837997471 73.96140501884987) bank119420 +119421 POINT(41.002244205938894 73.55180410160233) bank119421 +119422 POINT(40.45898645531627 74.32382892596735) bank119422 +119423 POINT(40.880503376689695 74.48907802162259) bank119423 +119424 POINT(40.518787818809635 73.17771492019934) bank119424 +119425 POINT(40.02555016186699 74.29246563567933) bank119425 +119426 POINT(40.08540375848642 74.8266503788342) bank119426 +119427 POINT(40.223847054920206 73.2433021226911) bank119427 +119428 POINT(40.33229398016337 73.93831937744258) bank119428 +119429 POINT(40.801338713448864 74.59194796210146) bank119429 +119430 POINT(41.08195501581538 74.02994400581595) bank119430 +119431 POINT(40.377280474729446 74.60743713790993) bank119431 +119432 POINT(40.64861921748783 74.3300325173891) bank119432 +119433 POINT(40.423804549623064 74.11273764456365) bank119433 +119434 POINT(40.90623380982792 74.02311573479386) bank119434 +119435 POINT(40.058523153492196 74.52775540815968) bank119435 +119436 POINT(41.198397710693854 73.11214210619391) bank119436 +119437 POINT(40.350782514874886 73.01878016873202) bank119437 +119438 POINT(40.06742653703117 73.62661623830665) bank119438 +119439 POINT(39.966461640550754 74.73794127335913) bank119439 +119440 POINT(39.7427672098222 73.66778595272748) bank119440 +119441 POINT(41.26694684391947 74.62939932290304) bank119441 +119442 POINT(40.1481578362799 75.00243372498092) bank119442 +119443 POINT(41.13496603523076 73.40853432525816) bank119443 +119444 POINT(39.776659020137906 74.99593089741366) bank119444 +119445 POINT(41.28130723894124 74.34965215238739) bank119445 +119446 POINT(40.890914561058246 74.63605959225099) bank119446 +119447 POINT(40.501358378004916 74.2398852195499) bank119447 +119448 POINT(41.517635495240484 74.70267923750936) bank119448 +119449 POINT(40.51427305254461 74.08687830118917) bank119449 +119450 POINT(41.45116193953869 73.79707508893725) bank119450 +119451 POINT(41.21766052600779 73.46983402888881) bank119451 +119452 POINT(41.506062207966664 73.28309107681241) bank119452 +119453 POINT(40.012581601677 74.13481723411867) bank119453 +119454 POINT(40.79720337180501 74.03315809014423) bank119454 +119455 POINT(40.64495904663699 73.19160376623611) bank119455 +119456 POINT(40.087763967094446 73.22836579517885) bank119456 +119457 POINT(39.89258351849381 74.58414197345222) bank119457 +119458 POINT(40.880298430328814 73.26028479345251) bank119458 +119459 POINT(41.28112553592859 73.88653455370766) bank119459 +119460 POINT(39.905055706950684 73.24716929087305) bank119460 +119461 POINT(40.10708861751914 74.04188527686347) bank119461 +119462 POINT(40.578730106070694 73.5436425865137) bank119462 +119463 POINT(39.82169557293673 74.38672495354358) bank119463 +119464 POINT(40.33460207028693 73.13616072495535) bank119464 +119465 POINT(39.88234323627886 73.9091131973283) bank119465 +119466 POINT(40.82615439691012 73.56564817511305) bank119466 +119467 POINT(41.32984331793559 73.1955807523036) bank119467 +119468 POINT(40.34399989470078 73.8367990089276) bank119468 +119469 POINT(39.833979546381265 74.94603216337214) bank119469 +119470 POINT(41.493888476050714 73.0709946388887) bank119470 +119471 POINT(40.06487859453426 73.94047471178092) bank119471 +119472 POINT(41.543828918163804 74.00615914508664) bank119472 +119473 POINT(40.43160455802883 74.85609586136506) bank119473 +119474 POINT(41.48659706890176 73.9757557421124) bank119474 +119475 POINT(40.06289870624811 73.56123138248029) bank119475 +119476 POINT(41.138543140114045 74.78461050476905) bank119476 +119477 POINT(41.495202832198814 74.11850321987995) bank119477 +119478 POINT(40.07535218209173 73.57496313562758) bank119478 +119479 POINT(41.26973063971025 74.21025060154562) bank119479 +119480 POINT(40.106749140208024 73.27266842742307) bank119480 +119481 POINT(41.65362153211718 73.72297497169586) bank119481 +119482 POINT(40.80392866674927 74.9963256640128) bank119482 +119483 POINT(41.21898663040288 74.29887044838794) bank119483 +119484 POINT(41.42534641061845 73.36542793422969) bank119484 +119485 POINT(41.1666928720623 74.6072624950751) bank119485 +119486 POINT(41.24688282689546 74.51262809797255) bank119486 +119487 POINT(41.59476597229768 74.46203410418866) bank119487 +119488 POINT(40.63467304037507 74.29117655223634) bank119488 +119489 POINT(39.80019069848043 74.83146742242388) bank119489 +119490 POINT(39.875939223313125 74.11682303892827) bank119490 +119491 POINT(40.150368196538324 74.9724269039362) bank119491 +119492 POINT(40.535095211695875 73.06921582525862) bank119492 +119493 POINT(41.10385769185981 73.17169393268863) bank119493 +119494 POINT(40.57142913099913 73.56161291779962) bank119494 +119495 POINT(41.22092070171602 73.2278447019218) bank119495 +119496 POINT(41.6759410410399 73.358642092488) bank119496 +119497 POINT(41.63669127188147 74.10550664213214) bank119497 +119498 POINT(39.88453491172763 73.09068545302766) bank119498 +119499 POINT(41.42956900011112 73.82119724056402) bank119499 +119500 POINT(40.06442168455211 73.03280693198542) bank119500 +119501 POINT(40.23434989219887 74.38051507663802) bank119501 +119502 POINT(41.19225186273643 74.35236309593647) bank119502 +119503 POINT(40.909458949657605 73.4929749193936) bank119503 +119504 POINT(39.80383446178926 73.44592457637137) bank119504 +119505 POINT(41.67378658952186 74.173262136719) bank119505 +119506 POINT(39.85512086437215 73.76684349387835) bank119506 +119507 POINT(39.763886289317014 74.564421029088) bank119507 +119508 POINT(41.15283627863466 73.44135544910009) bank119508 +119509 POINT(40.068866696293796 74.42189907720297) bank119509 +119510 POINT(39.89706477654676 73.50314675575834) bank119510 +119511 POINT(39.80753633335235 74.7124910891678) bank119511 +119512 POINT(41.47335337876036 73.4313463705963) bank119512 +119513 POINT(41.302372022996934 73.44169677511924) bank119513 +119514 POINT(41.57496459872296 74.68484826511263) bank119514 +119515 POINT(40.9293619035591 73.55384312230844) bank119515 +119516 POINT(40.059250116915145 74.4914877559107) bank119516 +119517 POINT(40.75808173379065 74.53283725608533) bank119517 +119518 POINT(40.91789319221062 73.08842556888317) bank119518 +119519 POINT(41.44253259826216 73.59652726347001) bank119519 +119520 POINT(41.421638936119976 74.66417250071969) bank119520 +119521 POINT(40.39300897022364 74.03749359259857) bank119521 +119522 POINT(40.28391377186166 74.6470496695398) bank119522 +119523 POINT(41.64636944975547 74.94756092532619) bank119523 +119524 POINT(41.45763958346089 73.69204017278516) bank119524 +119525 POINT(40.612642160463444 74.53073571469834) bank119525 +119526 POINT(40.178104775313585 74.72817227169672) bank119526 +119527 POINT(40.97853851781279 73.2628972090538) bank119527 +119528 POINT(40.38148150644511 73.16982200356081) bank119528 +119529 POINT(41.48238873568836 74.72639009329626) bank119529 +119530 POINT(40.39167496926495 74.99495625382494) bank119530 +119531 POINT(40.82782718864385 74.08034334964665) bank119531 +119532 POINT(39.80897395191857 74.3157279731157) bank119532 +119533 POINT(40.56839547340358 74.54540084347464) bank119533 +119534 POINT(39.886547553529276 73.76673928831079) bank119534 +119535 POINT(40.362230470090815 73.65830136089231) bank119535 +119536 POINT(41.64173942427417 74.7434442318434) bank119536 +119537 POINT(41.28326548348263 73.25168150824021) bank119537 +119538 POINT(40.968048864308216 74.2724766694754) bank119538 +119539 POINT(41.08252133367486 74.10356506784504) bank119539 +119540 POINT(40.79273414936045 74.35088688962723) bank119540 +119541 POINT(41.569691602506666 74.68240275929568) bank119541 +119542 POINT(41.560066238102166 73.00818358237233) bank119542 +119543 POINT(40.552396751468876 74.74069895204644) bank119543 +119544 POINT(39.79701083277418 73.36299924634926) bank119544 +119545 POINT(39.85488747598854 74.24266016435745) bank119545 +119546 POINT(40.64893847657117 73.8473732658658) bank119546 +119547 POINT(40.85345688434549 74.37975479669805) bank119547 +119548 POINT(41.52141572372996 74.04554566387937) bank119548 +119549 POINT(39.86370283170247 73.02526474306646) bank119549 +119550 POINT(41.27441227044481 74.72723047882614) bank119550 +119551 POINT(40.001290701266996 74.47326566291801) bank119551 +119552 POINT(40.785429314885455 73.53253006168627) bank119552 +119553 POINT(39.96130407274299 73.42966412153527) bank119553 +119554 POINT(41.28056800515379 74.15088307620088) bank119554 +119555 POINT(40.04906911752021 73.08223800020016) bank119555 +119556 POINT(41.43107626053058 73.2831421100002) bank119556 +119557 POINT(41.12362943511737 73.16907662995935) bank119557 +119558 POINT(41.70498093166461 74.54723960738366) bank119558 +119559 POINT(41.61020028191371 73.54947169300948) bank119559 +119560 POINT(41.247776090883654 73.5357524880442) bank119560 +119561 POINT(39.952084613076394 73.52634898715446) bank119561 +119562 POINT(40.977508800905326 73.2831591217454) bank119562 +119563 POINT(40.46138722483406 74.67145996158665) bank119563 +119564 POINT(41.447568258778496 73.98269135352554) bank119564 +119565 POINT(40.59523043992226 73.92379618223154) bank119565 +119566 POINT(41.35114571714636 73.93020772453127) bank119566 +119567 POINT(41.61735480785274 74.50959756101523) bank119567 +119568 POINT(39.77509395754001 74.24104465547168) bank119568 +119569 POINT(41.53203339062308 73.92601745832096) bank119569 +119570 POINT(41.64945189279283 74.58632867484471) bank119570 +119571 POINT(40.73867482141134 73.22443001376024) bank119571 +119572 POINT(41.0401645976349 74.82067261954549) bank119572 +119573 POINT(40.301691670985115 74.13200678226195) bank119573 +119574 POINT(41.232620745657435 73.40827510861543) bank119574 +119575 POINT(41.33780839810374 73.86009126782255) bank119575 +119576 POINT(40.51320097569807 74.8053438256177) bank119576 +119577 POINT(41.241594247498156 74.90029443403888) bank119577 +119578 POINT(41.260408485940026 73.57782348312085) bank119578 +119579 POINT(39.89779117685143 73.63019319134017) bank119579 +119580 POINT(40.43716150872642 73.07730501294955) bank119580 +119581 POINT(40.01472940627183 73.29301190854942) bank119581 +119582 POINT(40.242527436265846 74.26869271658462) bank119582 +119583 POINT(40.4372619256968 74.92143196761317) bank119583 +119584 POINT(39.819252424362304 74.92864988564058) bank119584 +119585 POINT(40.05011302295231 73.02011160594597) bank119585 +119586 POINT(41.240212785873865 73.18881847458462) bank119586 +119587 POINT(40.85894184515237 73.6163571976855) bank119587 +119588 POINT(40.305554779916434 74.2560809286679) bank119588 +119589 POINT(40.05944882658907 73.64295655630002) bank119589 +119590 POINT(40.90522658191521 73.67433785191821) bank119590 +119591 POINT(41.13674345091427 74.18086288661527) bank119591 +119592 POINT(39.774114477489285 73.37464224335238) bank119592 +119593 POINT(41.02683941634212 73.27698469623432) bank119593 +119594 POINT(40.91605960281414 74.4628826481375) bank119594 +119595 POINT(40.20762965167147 73.07735650292766) bank119595 +119596 POINT(41.265667502272144 74.12103715429372) bank119596 +119597 POINT(41.014003455829 73.48105667250557) bank119597 +119598 POINT(40.875347821909116 73.95004078443611) bank119598 +119599 POINT(39.955925868846094 74.9122417573769) bank119599 +119600 POINT(40.02591046927186 74.90119422607937) bank119600 +119601 POINT(40.80918195808087 73.7848557328545) bank119601 +119602 POINT(41.53622103173527 74.16546375159072) bank119602 +119603 POINT(41.34567770212102 73.97134414862148) bank119603 +119604 POINT(41.025274101280296 73.71234037256303) bank119604 +119605 POINT(40.54184650729643 73.80465412883778) bank119605 +119606 POINT(40.6941404187012 74.80470432390933) bank119606 +119607 POINT(40.978395721623514 74.94293324141404) bank119607 +119608 POINT(40.49403096912947 73.20417791537841) bank119608 +119609 POINT(40.946368024153195 74.42800228531348) bank119609 +119610 POINT(40.495040946281435 74.82118853968954) bank119610 +119611 POINT(41.39243996904957 74.94780260310723) bank119611 +119612 POINT(41.65204263461166 73.35938709553388) bank119612 +119613 POINT(40.36129116374696 73.43611154088636) bank119613 +119614 POINT(40.17812049449643 73.27738449547678) bank119614 +119615 POINT(41.07458344054641 73.69792461799311) bank119615 +119616 POINT(41.2231940547593 74.7001269443391) bank119616 +119617 POINT(41.01925140301686 73.65726602955752) bank119617 +119618 POINT(40.437834748425104 73.64017823246245) bank119618 +119619 POINT(41.40807576489898 73.66449196110938) bank119619 +119620 POINT(41.017654092009195 74.53978780269601) bank119620 +119621 POINT(40.75921494938521 73.15348913978943) bank119621 +119622 POINT(41.63422422575676 74.38752196820269) bank119622 +119623 POINT(41.12072236563167 73.84853796650954) bank119623 +119624 POINT(41.25508763905225 74.93205138409124) bank119624 +119625 POINT(39.80062661780808 73.77241914568053) bank119625 +119626 POINT(41.385789859807296 73.70645567721962) bank119626 +119627 POINT(40.572772262571114 74.6550303048151) bank119627 +119628 POINT(39.87375960911244 73.77728215167113) bank119628 +119629 POINT(40.20470080385295 73.17593483478747) bank119629 +119630 POINT(40.0338340319777 73.34731486713001) bank119630 +119631 POINT(40.38291467420609 73.19622028928941) bank119631 +119632 POINT(39.74776288158138 74.13102848362097) bank119632 +119633 POINT(40.93576998108203 74.8239802381876) bank119633 +119634 POINT(41.20155188469875 73.17290217871056) bank119634 +119635 POINT(40.835446417175966 73.18919430667809) bank119635 +119636 POINT(40.98117643829127 73.80069019480104) bank119636 +119637 POINT(41.061918046790446 73.53702921902145) bank119637 +119638 POINT(39.91968423025291 73.0618086421809) bank119638 +119639 POINT(41.37906355813615 73.7824753137215) bank119639 +119640 POINT(40.5107593705041 73.92596699299833) bank119640 +119641 POINT(40.439296058526836 73.31721628672594) bank119641 +119642 POINT(40.42497919361241 74.80959694478207) bank119642 +119643 POINT(40.29829575911925 74.8138308704457) bank119643 +119644 POINT(39.72705171551915 74.92446686421839) bank119644 +119645 POINT(40.185826780224886 74.60699228820118) bank119645 +119646 POINT(40.93108146013496 73.94783885631436) bank119646 +119647 POINT(41.04684714512991 73.74050711385672) bank119647 +119648 POINT(40.88864108302787 73.213600277185) bank119648 +119649 POINT(41.48265305912265 73.62286002793147) bank119649 +119650 POINT(41.14491013008794 74.27179857746266) bank119650 +119651 POINT(40.310149843349805 73.41900961175618) bank119651 +119652 POINT(41.33297022295752 73.2033444201805) bank119652 +119653 POINT(41.39172977476176 73.70506295360417) bank119653 +119654 POINT(40.26368353542386 74.06178300823757) bank119654 +119655 POINT(40.27662385681475 74.97447564804193) bank119655 +119656 POINT(41.46274609146173 73.82255887212551) bank119656 +119657 POINT(41.67860562003349 74.45972726610539) bank119657 +119658 POINT(41.38281270493982 74.64187588774698) bank119658 +119659 POINT(40.87925382094744 74.23080428250528) bank119659 +119660 POINT(40.35801088157271 73.84963794270749) bank119660 +119661 POINT(41.60494944131838 74.47423152577635) bank119661 +119662 POINT(39.72477758174238 74.22360659957313) bank119662 +119663 POINT(40.97455276675531 74.43448167373083) bank119663 +119664 POINT(40.56719292189817 74.69370996254953) bank119664 +119665 POINT(40.342684531488935 73.44867110275153) bank119665 +119666 POINT(40.071678824947526 73.46654297789092) bank119666 +119667 POINT(40.09135128935093 74.36700211206315) bank119667 +119668 POINT(40.419089577737196 74.77639381949308) bank119668 +119669 POINT(41.62852356869109 73.74806241679559) bank119669 +119670 POINT(39.970715844452506 73.99072547036856) bank119670 +119671 POINT(40.85670321295007 73.14183327570146) bank119671 +119672 POINT(40.87385222975096 74.48039392742963) bank119672 +119673 POINT(41.388896089507135 74.61950145501979) bank119673 +119674 POINT(41.416553994804566 73.26904199982934) bank119674 +119675 POINT(41.25599221907361 73.18129060863039) bank119675 +119676 POINT(40.65880694733958 73.89696435557674) bank119676 +119677 POINT(41.32044789022412 73.91119593570355) bank119677 +119678 POINT(40.859076711814836 73.40659630725027) bank119678 +119679 POINT(39.72021784345418 74.93380189539788) bank119679 +119680 POINT(41.23366815423284 73.29759012767265) bank119680 +119681 POINT(39.77025547640429 74.4732653994101) bank119681 +119682 POINT(40.140190809940194 74.15008164622301) bank119682 +119683 POINT(39.91603313610878 74.86513733221628) bank119683 +119684 POINT(39.87724416299081 73.40061340595824) bank119684 +119685 POINT(40.61535750980985 73.231178273269) bank119685 +119686 POINT(39.8385446951879 73.10357681767296) bank119686 +119687 POINT(40.50867663271756 73.82372203665338) bank119687 +119688 POINT(39.71284241089004 73.07570556392582) bank119688 +119689 POINT(39.91621661080962 73.25938050411425) bank119689 +119690 POINT(40.933323761006456 73.32762730019749) bank119690 +119691 POINT(41.65266045554092 74.53807747524988) bank119691 +119692 POINT(41.49596049657424 73.00920381050442) bank119692 +119693 POINT(39.927833388046174 75.00514858163669) bank119693 +119694 POINT(40.59459864575245 73.83501912176551) bank119694 +119695 POINT(41.239646824419815 73.34909086776747) bank119695 +119696 POINT(41.5398131931978 74.08209912671592) bank119696 +119697 POINT(40.099348579061235 74.34235734277046) bank119697 +119698 POINT(40.12492605411886 74.9406771467728) bank119698 +119699 POINT(41.709389243655316 74.95504063413915) bank119699 +119700 POINT(41.093272369131824 73.52843614549028) bank119700 +119701 POINT(41.244493203241724 73.47997957529597) bank119701 +119702 POINT(39.94628644292702 74.6240355716941) bank119702 +119703 POINT(41.01286517548645 73.50114156847775) bank119703 +119704 POINT(41.11323481757776 73.46131156156027) bank119704 +119705 POINT(40.932175725705484 74.16783058023537) bank119705 +119706 POINT(40.49036921522017 73.1466158147086) bank119706 +119707 POINT(41.679428251772414 74.15562532522671) bank119707 +119708 POINT(40.59417888883835 74.55495076003304) bank119708 +119709 POINT(41.01818712025953 73.52258284453472) bank119709 +119710 POINT(40.387255845697744 74.17078114043899) bank119710 +119711 POINT(40.12602046926526 74.06269637052075) bank119711 +119712 POINT(40.077540325856845 73.2248672975005) bank119712 +119713 POINT(41.02680815553051 73.209774032856) bank119713 +119714 POINT(40.980003303117016 73.67072978548123) bank119714 +119715 POINT(39.823684850775415 74.07099708621777) bank119715 +119716 POINT(40.639538633504266 74.35209400375902) bank119716 +119717 POINT(40.22010942212106 73.53151799480833) bank119717 +119718 POINT(40.28462053633839 73.78778729235191) bank119718 +119719 POINT(41.496885313136055 74.40207493160183) bank119719 +119720 POINT(41.51329119054412 73.2596670160006) bank119720 +119721 POINT(40.92043269103068 74.25039271781719) bank119721 +119722 POINT(40.07344753221333 73.92338151762503) bank119722 +119723 POINT(41.340946973179555 74.47845859178902) bank119723 +119724 POINT(40.27345098472292 74.92047919619161) bank119724 +119725 POINT(40.38617590334162 73.38312642785043) bank119725 +119726 POINT(40.64094400057583 73.082174747496) bank119726 +119727 POINT(40.37793051051925 73.26127707519821) bank119727 +119728 POINT(40.253449826014936 74.85287301503199) bank119728 +119729 POINT(40.61046537904027 73.65396823594361) bank119729 +119730 POINT(40.97345726145615 74.17406648814497) bank119730 +119731 POINT(40.298894806248974 73.34552342825863) bank119731 +119732 POINT(40.71623436476256 74.91203631358806) bank119732 +119733 POINT(40.387843570011995 73.71874606431665) bank119733 +119734 POINT(40.8324914571585 74.7723552946407) bank119734 +119735 POINT(40.93356269616702 74.78173585553652) bank119735 +119736 POINT(41.259363867557354 74.82123424300939) bank119736 +119737 POINT(41.322626401762335 73.08325081486412) bank119737 +119738 POINT(40.03827448658225 73.11031102546544) bank119738 +119739 POINT(41.18287841824671 73.49613585317373) bank119739 +119740 POINT(40.58166782134362 74.95195324693637) bank119740 +119741 POINT(41.553664175777214 74.98403233486576) bank119741 +119742 POINT(40.3461893102729 73.92622484577116) bank119742 +119743 POINT(41.19699776936445 74.85295259861913) bank119743 +119744 POINT(41.18689687792089 73.87859841854646) bank119744 +119745 POINT(41.57866496832974 73.80185116511421) bank119745 +119746 POINT(40.46323869646918 74.99090682512197) bank119746 +119747 POINT(40.25525932126663 73.0170716672121) bank119747 +119748 POINT(40.74298890255912 73.96170394127591) bank119748 +119749 POINT(40.29604582145093 73.49610014822998) bank119749 +119750 POINT(39.75645943078274 74.46354609248945) bank119750 +119751 POINT(39.789300317120684 74.42799481657642) bank119751 +119752 POINT(39.98455095108629 73.9366992979513) bank119752 +119753 POINT(40.63999795056826 74.14064708135311) bank119753 +119754 POINT(40.217549937708554 74.32166969297954) bank119754 +119755 POINT(39.95666380427395 74.94220933307989) bank119755 +119756 POINT(39.81611641159415 74.17117089018137) bank119756 +119757 POINT(40.20515036596829 73.1396836550946) bank119757 +119758 POINT(41.647885390210064 73.19909023225104) bank119758 +119759 POINT(40.229989644065036 73.24146563508566) bank119759 +119760 POINT(41.32750777068498 73.24007158418317) bank119760 +119761 POINT(41.174177208886285 73.64092213110551) bank119761 +119762 POINT(41.26508272705614 74.12280907856001) bank119762 +119763 POINT(40.598126946171746 74.3063414857215) bank119763 +119764 POINT(40.443702683093555 73.5746100668341) bank119764 +119765 POINT(40.33119961057085 74.80164018537133) bank119765 +119766 POINT(41.48510874886441 73.2426704625637) bank119766 +119767 POINT(40.1635915027948 74.27797448853671) bank119767 +119768 POINT(41.324957972006594 73.79187728678963) bank119768 +119769 POINT(41.04443255846057 73.80815321932151) bank119769 +119770 POINT(40.46084233292545 73.12803472965375) bank119770 +119771 POINT(41.70321560537887 74.39789190336327) bank119771 +119772 POINT(41.606746347208734 74.19150903450405) bank119772 +119773 POINT(41.393120796974046 74.96843327764019) bank119773 +119774 POINT(41.16468130656954 74.29885052171905) bank119774 +119775 POINT(40.58355231579738 73.04354268871703) bank119775 +119776 POINT(39.96676524067008 74.33875544776289) bank119776 +119777 POINT(40.7676606269844 73.5181739426162) bank119777 +119778 POINT(40.20823511003305 74.86328127040179) bank119778 +119779 POINT(40.75073654682815 74.26550061163024) bank119779 +119780 POINT(40.479057776669976 73.60862283495152) bank119780 +119781 POINT(40.95741920050722 73.92702346550199) bank119781 +119782 POINT(41.644323637245584 73.90671779941178) bank119782 +119783 POINT(40.161356852236004 74.513378346548) bank119783 +119784 POINT(40.6207266957515 73.52402806326742) bank119784 +119785 POINT(39.95136499689072 74.86914904606627) bank119785 +119786 POINT(40.84124275274173 74.5495411289056) bank119786 +119787 POINT(41.622691440356384 73.63577961156592) bank119787 +119788 POINT(41.09733944527946 73.16377608807764) bank119788 +119789 POINT(39.92761544141522 73.30729368719314) bank119789 +119790 POINT(40.0215003633688 73.33647940667844) bank119790 +119791 POINT(40.89813913763475 73.27272815694312) bank119791 +119792 POINT(40.4704723388418 73.61518330893098) bank119792 +119793 POINT(41.01470370101029 74.08657481601837) bank119793 +119794 POINT(39.78971661453571 74.3420970250292) bank119794 +119795 POINT(39.85648523764726 74.07936901452783) bank119795 +119796 POINT(40.82997901121755 73.8727924672953) bank119796 +119797 POINT(41.22189063149952 74.26095443606296) bank119797 +119798 POINT(41.543788227537995 73.03819005855337) bank119798 +119799 POINT(40.212288517056535 74.63244646475574) bank119799 +119800 POINT(40.8292229612122 73.51241369160424) bank119800 +119801 POINT(40.47296196164316 73.36812266468361) bank119801 +119802 POINT(41.52408530512493 73.87111458219081) bank119802 +119803 POINT(41.29591045143048 74.30072338272088) bank119803 +119804 POINT(39.74485127287221 73.26974685947418) bank119804 +119805 POINT(41.26310104981521 73.90483462164907) bank119805 +119806 POINT(39.90473210117479 74.50131037323627) bank119806 +119807 POINT(40.92604415530162 73.81097769841702) bank119807 +119808 POINT(40.33632880492049 73.43036526935461) bank119808 +119809 POINT(40.49754429246764 73.34841121543359) bank119809 +119810 POINT(41.439315434760815 74.97517879829235) bank119810 +119811 POINT(41.246028207137776 74.9739297224966) bank119811 +119812 POINT(41.46491528761454 74.31203767579653) bank119812 +119813 POINT(40.55170819355509 74.32328898397391) bank119813 +119814 POINT(41.493482005019786 73.63102224884112) bank119814 +119815 POINT(41.23265878254895 73.45853753339263) bank119815 +119816 POINT(41.21915672284994 73.30455598812536) bank119816 +119817 POINT(40.35512459762174 74.61590523603782) bank119817 +119818 POINT(40.60581757644794 74.35637719493221) bank119818 +119819 POINT(40.16516908308949 73.49856802149583) bank119819 +119820 POINT(40.65801449938779 73.0143063132943) bank119820 +119821 POINT(40.13830756814911 74.73454965084179) bank119821 +119822 POINT(41.25045063985363 73.35880621370025) bank119822 +119823 POINT(41.16512692725867 73.5936839933252) bank119823 +119824 POINT(39.7649092691387 74.6082375219399) bank119824 +119825 POINT(40.87473320379824 73.63632003949562) bank119825 +119826 POINT(40.60328362956906 73.20803013397051) bank119826 +119827 POINT(41.37896121725056 74.3585233426208) bank119827 +119828 POINT(40.73172842790306 74.46661571200589) bank119828 +119829 POINT(40.034018212635246 74.87951895018958) bank119829 +119830 POINT(40.678071410515535 73.33169027350976) bank119830 +119831 POINT(40.809421173264404 74.27585234971403) bank119831 +119832 POINT(41.43483923728705 73.08746021646265) bank119832 +119833 POINT(39.73684195448015 74.3824936978602) bank119833 +119834 POINT(41.10419358003515 73.64634617324309) bank119834 +119835 POINT(40.063179883847425 74.09037204826035) bank119835 +119836 POINT(40.51793959303245 73.92196744691971) bank119836 +119837 POINT(40.344706554648646 73.71836413688122) bank119837 +119838 POINT(40.446932084049074 74.30897736756086) bank119838 +119839 POINT(40.76455298031662 73.14207009996095) bank119839 +119840 POINT(40.55475760786795 73.71220311863715) bank119840 +119841 POINT(40.5543011987275 73.60673322529057) bank119841 +119842 POINT(39.992207926698434 74.70473927275165) bank119842 +119843 POINT(40.3264885586398 73.78203237217528) bank119843 +119844 POINT(40.511204162082436 73.24993224464019) bank119844 +119845 POINT(40.6252319897423 74.7524697163375) bank119845 +119846 POINT(39.78371984387667 74.0531117004088) bank119846 +119847 POINT(41.24115311649423 74.49451125689163) bank119847 +119848 POINT(41.41332452576168 74.04081917314558) bank119848 +119849 POINT(39.79354067027217 73.21193296771177) bank119849 +119850 POINT(41.16635655693156 73.95007314365985) bank119850 +119851 POINT(40.16724710112657 73.75789863286023) bank119851 +119852 POINT(40.95641300120186 74.39211987968862) bank119852 +119853 POINT(40.696169644808535 74.61584026523222) bank119853 +119854 POINT(40.05551196034007 73.45118007008229) bank119854 +119855 POINT(39.97179925669738 74.99442009956408) bank119855 +119856 POINT(41.34331872348275 73.21841561692607) bank119856 +119857 POINT(40.223218801639376 74.33580220481612) bank119857 +119858 POINT(41.26251783001916 74.90084837314824) bank119858 +119859 POINT(41.27873684304397 73.25449080502895) bank119859 +119860 POINT(41.07492534017011 74.25258429589645) bank119860 +119861 POINT(41.65990257684274 74.59977511805349) bank119861 +119862 POINT(40.85092091854558 74.04156466504081) bank119862 +119863 POINT(40.91729778783636 74.08130648195882) bank119863 +119864 POINT(41.59965837012571 73.14324382466124) bank119864 +119865 POINT(39.79821095271407 74.00290513123045) bank119865 +119866 POINT(41.62029752961995 74.2072139708056) bank119866 +119867 POINT(41.51838897524493 73.72691504548298) bank119867 +119868 POINT(41.0729540033092 73.71408323548721) bank119868 +119869 POINT(41.62639724631555 73.90994844241472) bank119869 +119870 POINT(40.38007377781806 73.76418817330519) bank119870 +119871 POINT(40.62365398546517 74.3658148489809) bank119871 +119872 POINT(41.08291569514437 74.58512560601628) bank119872 +119873 POINT(39.98491865690714 74.63052798454227) bank119873 +119874 POINT(40.88833149870872 73.56958074280627) bank119874 +119875 POINT(41.641952764372206 73.1629882201501) bank119875 +119876 POINT(41.27578100086199 73.56476638320927) bank119876 +119877 POINT(40.501922493252444 74.69366129403156) bank119877 +119878 POINT(40.22791179332367 73.78729964872183) bank119878 +119879 POINT(40.76536366276243 73.45550045228885) bank119879 +119880 POINT(40.43883664652753 73.1364825747212) bank119880 +119881 POINT(40.345046411486805 73.64368305023876) bank119881 +119882 POINT(40.126706194912096 74.45878197103536) bank119882 +119883 POINT(40.967954683762855 73.55442744689786) bank119883 +119884 POINT(40.56763064907529 74.31864638427739) bank119884 +119885 POINT(40.46293222760356 74.53861167694959) bank119885 +119886 POINT(41.56430021362125 74.22794691005419) bank119886 +119887 POINT(41.024267020721645 73.23672499576114) bank119887 +119888 POINT(40.676312827988845 73.13297346154707) bank119888 +119889 POINT(40.58963819592697 73.80484085363246) bank119889 +119890 POINT(41.66678074377703 74.24191147479401) bank119890 +119891 POINT(39.97493099351603 74.68629584999516) bank119891 +119892 POINT(41.15688492252668 73.2903719349121) bank119892 +119893 POINT(41.21243063925384 75.00337578274254) bank119893 +119894 POINT(41.01377746986489 73.46428439081103) bank119894 +119895 POINT(39.851089847438566 74.10204576987107) bank119895 +119896 POINT(39.797124461783376 73.15071450698551) bank119896 +119897 POINT(41.34558219800074 74.65044175351991) bank119897 +119898 POINT(40.580522822342274 74.2046951018597) bank119898 +119899 POINT(39.87911363103552 73.71411232448081) bank119899 +119900 POINT(41.04929208042767 73.2943710940442) bank119900 +119901 POINT(40.81857735092713 74.15814032487768) bank119901 +119902 POINT(41.22610456645297 74.5643197895965) bank119902 +119903 POINT(40.02003495714073 74.6322154327864) bank119903 +119904 POINT(40.16090701972445 74.47560378918925) bank119904 +119905 POINT(40.65557348979823 74.74660313309658) bank119905 +119906 POINT(40.3413077770405 74.32968367070875) bank119906 +119907 POINT(40.633208068474815 74.84470159679537) bank119907 +119908 POINT(39.83518722474845 74.49242848151346) bank119908 +119909 POINT(39.92731289877744 74.22142782331802) bank119909 +119910 POINT(41.5916670898283 73.91336052736627) bank119910 +119911 POINT(40.14533413166451 73.5990445745689) bank119911 +119912 POINT(41.66457798311637 73.02613269244861) bank119912 +119913 POINT(40.28163852414495 74.60807663154193) bank119913 +119914 POINT(41.35242071598198 74.6999416231446) bank119914 +119915 POINT(40.89032457246708 74.30194828280767) bank119915 +119916 POINT(40.337381250514625 73.00703571156913) bank119916 +119917 POINT(40.94118331034594 74.36304240561084) bank119917 +119918 POINT(40.517070743354665 73.87197988376047) bank119918 +119919 POINT(40.32048301564281 74.86251671787298) bank119919 +119920 POINT(41.3463339520293 73.8995404032367) bank119920 +119921 POINT(40.58197148371298 74.36906519586022) bank119921 +119922 POINT(40.81594674578831 73.80730239928702) bank119922 +119923 POINT(40.40354417526124 73.21603726031968) bank119923 +119924 POINT(40.616220825355704 74.6915674480929) bank119924 +119925 POINT(40.58284113365092 73.07346225882765) bank119925 +119926 POINT(40.95798014378732 74.84479201008367) bank119926 +119927 POINT(41.69496260119172 73.36889873720332) bank119927 +119928 POINT(40.09304120534225 73.10775147160993) bank119928 +119929 POINT(40.59916954059002 74.84020236454782) bank119929 +119930 POINT(40.084927687181775 74.58120291006483) bank119930 +119931 POINT(41.54771673586474 74.40034079229193) bank119931 +119932 POINT(40.34502100165689 74.0853805422783) bank119932 +119933 POINT(40.11413212475504 74.81608030219357) bank119933 +119934 POINT(40.43528270339088 73.36900586336508) bank119934 +119935 POINT(41.514971882459996 74.56521353642523) bank119935 +119936 POINT(40.066307499009355 73.13750221226753) bank119936 +119937 POINT(41.22138668911474 73.76101207659227) bank119937 +119938 POINT(40.96769687167554 74.53967897748227) bank119938 +119939 POINT(41.15408252799573 73.05854549118999) bank119939 +119940 POINT(40.43850136341357 73.20260382718165) bank119940 +119941 POINT(40.52890285309041 73.18261865976135) bank119941 +119942 POINT(41.48789959460149 74.74887588757215) bank119942 +119943 POINT(40.81842869177973 73.31979658630372) bank119943 +119944 POINT(41.13086655455465 74.6831727215219) bank119944 +119945 POINT(41.69267929278895 74.53266853042544) bank119945 +119946 POINT(40.43484160735558 74.11279846770073) bank119946 +119947 POINT(41.16511237514196 73.12600554653034) bank119947 +119948 POINT(39.74132914885101 73.60793930591875) bank119948 +119949 POINT(39.92029493058444 74.38991813019341) bank119949 +119950 POINT(40.050114329360355 74.5406370695822) bank119950 +119951 POINT(40.99112480293964 73.86049602937324) bank119951 +119952 POINT(41.55895177876575 74.97490905157821) bank119952 +119953 POINT(41.214643228812946 74.70924229383635) bank119953 +119954 POINT(41.483395802023026 74.51401249633285) bank119954 +119955 POINT(40.53884475089653 73.73442082435731) bank119955 +119956 POINT(39.9820238075358 73.2099432828861) bank119956 +119957 POINT(41.07328039111385 74.26820762961259) bank119957 +119958 POINT(40.19465623380988 73.27630445703319) bank119958 +119959 POINT(40.38439659840948 73.15370051281869) bank119959 +119960 POINT(39.952716642929396 73.9141960082359) bank119960 +119961 POINT(40.23687767677094 73.46686023936955) bank119961 +119962 POINT(40.581937527371714 74.14005596343713) bank119962 +119963 POINT(41.67869406320392 73.49136400862628) bank119963 +119964 POINT(41.33158014996328 74.57257865182109) bank119964 +119965 POINT(41.514229618603636 73.23268657363265) bank119965 +119966 POINT(40.70940357184616 73.29432380794928) bank119966 +119967 POINT(40.600596589589074 74.9557076611965) bank119967 +119968 POINT(41.321542313596545 74.28234067558388) bank119968 +119969 POINT(41.05234610080508 73.29186893096616) bank119969 +119970 POINT(39.94990554510975 73.28899620250955) bank119970 +119971 POINT(40.844099252417465 74.48068346722673) bank119971 +119972 POINT(40.52321835824913 73.10383375754449) bank119972 +119973 POINT(40.52611242317827 74.38892132626057) bank119973 +119974 POINT(41.68137088335164 73.1204893683372) bank119974 +119975 POINT(41.08567996953385 73.62028744667145) bank119975 +119976 POINT(39.78384422364418 74.24448692581335) bank119976 +119977 POINT(40.77680068093871 74.75785280962789) bank119977 +119978 POINT(39.74448354372854 74.85921251657489) bank119978 +119979 POINT(40.17569087050907 73.85159281443644) bank119979 +119980 POINT(40.90186977290775 73.36039962375193) bank119980 +119981 POINT(40.3383288175996 74.92675396097177) bank119981 +119982 POINT(39.7188334404498 74.07665084919336) bank119982 +119983 POINT(40.57315719921604 74.17822840171054) bank119983 +119984 POINT(41.62756817361125 74.70357732277046) bank119984 +119985 POINT(40.70804121011335 73.47732615039645) bank119985 +119986 POINT(40.388623602422065 73.0164012594138) bank119986 +119987 POINT(41.051268645443265 73.86622485321635) bank119987 +119988 POINT(40.33474809547962 73.33978770134111) bank119988 +119989 POINT(41.49228150052346 74.59819701114964) bank119989 +119990 POINT(40.10206188500513 73.34739651985761) bank119990 +119991 POINT(40.95002232254188 73.37791847404193) bank119991 +119992 POINT(40.275377691830826 74.1416229120164) bank119992 +119993 POINT(41.43825721759368 74.18682475438999) bank119993 +119994 POINT(40.36002235453505 74.55098242774267) bank119994 +119995 POINT(41.43052559042186 74.6115190880071) bank119995 +119996 POINT(40.55892317575968 74.58743530281961) bank119996 +119997 POINT(39.996876833199785 73.80669916021458) bank119997 +119998 POINT(41.11403284278255 73.31056667030931) bank119998 +119999 POINT(41.63518852466731 74.84057581659971) bank119999 +120000 POINT(41.62391866267893 74.48268027813967) bank120000 +120001 POINT(39.79340809454576 73.54650985260902) bank120001 +120002 POINT(40.55213740643765 73.58987615960129) bank120002 +120003 POINT(41.04562808215275 73.75352409318042) bank120003 +120004 POINT(41.17549091373156 74.82011481204678) bank120004 +120005 POINT(40.12431028657606 73.5051925405625) bank120005 +120006 POINT(40.38235167344462 73.84251453482061) bank120006 +120007 POINT(40.2817839921106 74.27504355250605) bank120007 +120008 POINT(41.26521693614697 74.50994733815972) bank120008 +120009 POINT(40.52155304643109 73.78157465750431) bank120009 +120010 POINT(41.13914938997966 74.43773472611575) bank120010 +120011 POINT(40.70255838884525 73.18131179900708) bank120011 +120012 POINT(41.21567719480031 74.31488619811455) bank120012 +120013 POINT(41.660700963889205 74.12005294381105) bank120013 +120014 POINT(40.660886840943476 73.59579162110833) bank120014 +120015 POINT(40.0778979472352 74.54819742337686) bank120015 +120016 POINT(41.6881154769675 74.17387463595715) bank120016 +120017 POINT(40.80835690358045 74.3244036487505) bank120017 +120018 POINT(41.22669689457907 73.08393470288858) bank120018 +120019 POINT(40.93956489179056 74.41358557033078) bank120019 +120020 POINT(41.5065520950511 74.01165996539326) bank120020 +120021 POINT(40.47967566945469 74.25140533423117) bank120021 +120022 POINT(40.77233127897694 73.60602619767766) bank120022 +120023 POINT(40.151921668300815 74.51994742698588) bank120023 +120024 POINT(40.095033678683606 74.1295418540248) bank120024 +120025 POINT(40.26338793169566 73.65714035741179) bank120025 +120026 POINT(41.55668423177285 74.41088047086889) bank120026 +120027 POINT(39.994312843395136 73.58202494078861) bank120027 +120028 POINT(41.313843508939854 74.89739900061092) bank120028 +120029 POINT(40.828550480748575 74.75568393880475) bank120029 +120030 POINT(41.296788315756956 74.41286802923604) bank120030 +120031 POINT(41.66606811076943 73.74390832297013) bank120031 +120032 POINT(40.93377800091351 74.1357387906361) bank120032 +120033 POINT(40.31774710562086 73.7232905451236) bank120033 +120034 POINT(39.72556198098045 73.01667713863674) bank120034 +120035 POINT(40.787228281525806 74.43592300782935) bank120035 +120036 POINT(39.73256678072471 73.07766313262393) bank120036 +120037 POINT(40.604468731880694 73.16276438796511) bank120037 +120038 POINT(41.06409437408567 73.63807958529758) bank120038 +120039 POINT(40.32562041784947 73.65238360930631) bank120039 +120040 POINT(41.29484841073114 73.15520872392356) bank120040 +120041 POINT(40.998962272404505 73.92392291567478) bank120041 +120042 POINT(40.80587426459446 74.85794115338803) bank120042 +120043 POINT(41.21401603454389 74.18946138931877) bank120043 +120044 POINT(40.69183251516929 74.73135700328679) bank120044 +120045 POINT(40.42406456215803 74.60453194946113) bank120045 +120046 POINT(39.866149903330694 74.76639930709123) bank120046 +120047 POINT(41.49803964372012 73.10044209198537) bank120047 +120048 POINT(41.32839777647533 74.58956982999554) bank120048 +120049 POINT(41.29608508338113 74.62638629002487) bank120049 +120050 POINT(41.3891203644111 74.3300499668054) bank120050 +120051 POINT(40.0676818039788 74.73946608066029) bank120051 +120052 POINT(40.251530539347584 73.45550309537053) bank120052 +120053 POINT(40.89390459729417 74.57059397048094) bank120053 +120054 POINT(41.4282944393795 74.02738487509782) bank120054 +120055 POINT(41.59687552863133 74.7159356016981) bank120055 +120056 POINT(41.62952895327366 74.69521810031337) bank120056 +120057 POINT(40.84701700040586 74.60013041276872) bank120057 +120058 POINT(41.01906794368482 73.20655421781213) bank120058 +120059 POINT(41.40116182770607 74.72579329857804) bank120059 +120060 POINT(40.489335454518 73.13348877759998) bank120060 +120061 POINT(40.501518771561386 73.85134417127534) bank120061 +120062 POINT(40.506019447764594 73.24953196506682) bank120062 +120063 POINT(41.22143905252883 73.05724591810362) bank120063 +120064 POINT(41.43569923584924 73.36465177432488) bank120064 +120065 POINT(41.55754200914051 73.86648868682605) bank120065 +120066 POINT(39.95276492700753 74.91794386418142) bank120066 +120067 POINT(39.95284499315614 73.03738873653239) bank120067 +120068 POINT(39.88697731716367 73.74281161040604) bank120068 +120069 POINT(41.56130264946262 73.60350402479449) bank120069 +120070 POINT(41.053374861380945 74.46627333660047) bank120070 +120071 POINT(40.91302337737483 73.21247783416831) bank120071 +120072 POINT(41.66403182780104 74.22426213772549) bank120072 +120073 POINT(40.02462885943797 73.17042263445428) bank120073 +120074 POINT(41.44565215175809 74.80692168901652) bank120074 +120075 POINT(40.664370538093536 74.95583894363735) bank120075 +120076 POINT(40.8682927386369 73.48390262303882) bank120076 +120077 POINT(40.21400852783831 73.88555703706982) bank120077 +120078 POINT(41.409378281479555 73.45988612265816) bank120078 +120079 POINT(40.56489852450727 73.89097068120867) bank120079 +120080 POINT(41.10229823881813 74.64449832609742) bank120080 +120081 POINT(40.85004222738362 74.52745023162234) bank120081 +120082 POINT(40.19656917169164 74.74169836958667) bank120082 +120083 POINT(41.143612238795086 73.44356177629744) bank120083 +120084 POINT(40.64013279931119 74.61687810996179) bank120084 +120085 POINT(40.66361441740013 73.52793531647411) bank120085 +120086 POINT(41.17423799982933 73.43129906581099) bank120086 +120087 POINT(41.62882504647413 73.25248192177688) bank120087 +120088 POINT(40.461499843522574 74.27945233205706) bank120088 +120089 POINT(40.7176487055511 74.25323111454507) bank120089 +120090 POINT(41.594071202598194 73.17536607850579) bank120090 +120091 POINT(41.41354185504702 73.78214678096599) bank120091 +120092 POINT(41.355666846573946 73.86871591100673) bank120092 +120093 POINT(41.54336041988076 73.15885766206885) bank120093 +120094 POINT(40.14301553035418 74.36634037472244) bank120094 +120095 POINT(40.83728870380918 74.14630156980074) bank120095 +120096 POINT(39.88615756073551 74.76436556844327) bank120096 +120097 POINT(39.7679936231794 74.19330890435039) bank120097 +120098 POINT(40.602240309694814 74.37872367383987) bank120098 +120099 POINT(39.80518813018156 74.25228185837291) bank120099 +120100 POINT(40.3523671936532 74.23306471515235) bank120100 +120101 POINT(40.12974617532471 74.14038451056145) bank120101 +120102 POINT(40.21830931528417 73.56562923782185) bank120102 +120103 POINT(41.38775948536751 73.68305671586046) bank120103 +120104 POINT(39.97490853029471 73.19043453924559) bank120104 +120105 POINT(39.901875113216015 73.70914049806439) bank120105 +120106 POINT(40.56234003140007 73.10652348067103) bank120106 +120107 POINT(40.720689629518986 74.16478182454748) bank120107 +120108 POINT(40.68785284490736 73.9890979401593) bank120108 +120109 POINT(41.15712029295195 74.71654844357327) bank120109 +120110 POINT(40.041000875117376 74.25366859079018) bank120110 +120111 POINT(40.345475625077654 73.35894535196982) bank120111 +120112 POINT(40.2989227228921 73.50111273918502) bank120112 +120113 POINT(41.48194120737166 74.91518781787298) bank120113 +120114 POINT(40.018529871652404 73.40957302918123) bank120114 +120115 POINT(40.234128172902814 73.21795055830653) bank120115 +120116 POINT(40.57832378342428 73.1382007927749) bank120116 +120117 POINT(41.22257937744696 73.96280430953117) bank120117 +120118 POINT(40.9580299378041 74.74229095041815) bank120118 +120119 POINT(41.33798621451484 73.49059460370427) bank120119 +120120 POINT(39.73792371015606 73.74820428637754) bank120120 +120121 POINT(40.87038024793693 73.36942449025499) bank120121 +120122 POINT(40.42487166547387 73.2709926460031) bank120122 +120123 POINT(41.01843190379374 74.55433540383697) bank120123 +120124 POINT(40.019241268834826 74.98479973310161) bank120124 +120125 POINT(41.02984218371828 73.98548895696031) bank120125 +120126 POINT(40.848961769809165 74.44897705393151) bank120126 +120127 POINT(41.12165671136107 73.08013477414417) bank120127 +120128 POINT(39.858073175866814 74.37509018927945) bank120128 +120129 POINT(40.58146839057827 73.75443623464139) bank120129 +120130 POINT(40.77061190226108 74.32729375237275) bank120130 +120131 POINT(39.72499277799615 73.0753440389525) bank120131 +120132 POINT(40.827810714624086 73.56470294616345) bank120132 +120133 POINT(39.85097368782757 74.87452380710802) bank120133 +120134 POINT(41.25109225988499 74.38152476238503) bank120134 +120135 POINT(39.87250888913907 74.76540969280053) bank120135 +120136 POINT(41.06050205545532 73.46558687291143) bank120136 +120137 POINT(40.4420233178167 74.43735001075575) bank120137 +120138 POINT(41.046309087051775 73.64654952412424) bank120138 +120139 POINT(41.29189807201946 74.94956503557644) bank120139 +120140 POINT(40.94204315375665 74.54871969067844) bank120140 +120141 POINT(39.78318807805522 73.01597233156835) bank120141 +120142 POINT(41.12811655877938 73.55915647015522) bank120142 +120143 POINT(41.15056245420528 74.33413473516605) bank120143 +120144 POINT(41.13244307656383 74.18444647836668) bank120144 +120145 POINT(40.47841701183757 73.14576072545823) bank120145 +120146 POINT(40.67611934983559 73.95118010487514) bank120146 +120147 POINT(40.517641299268846 73.62632330348302) bank120147 +120148 POINT(40.821860106328714 74.7143222927961) bank120148 +120149 POINT(41.536737825840355 73.54905136728497) bank120149 +120150 POINT(40.298323041265846 74.55430034194322) bank120150 +120151 POINT(40.72113359248604 74.730867085918) bank120151 +120152 POINT(40.18325513615528 74.57871913258549) bank120152 +120153 POINT(39.974214378271746 74.7689080710542) bank120153 +120154 POINT(40.28052173703853 73.83190795092878) bank120154 +120155 POINT(40.42333143061993 73.84382833764909) bank120155 +120156 POINT(41.30948471606349 74.04473172326044) bank120156 +120157 POINT(40.3461545131506 74.76875454236311) bank120157 +120158 POINT(40.777239858795994 73.81809378681217) bank120158 +120159 POINT(40.32641646542336 74.06199876993676) bank120159 +120160 POINT(40.879712592338954 74.57703337008618) bank120160 +120161 POINT(40.42662690272706 74.61208399507417) bank120161 +120162 POINT(41.27559375026997 74.20981959972956) bank120162 +120163 POINT(41.41450684581807 73.96127500998367) bank120163 +120164 POINT(41.5663428011079 74.0669433398294) bank120164 +120165 POINT(39.749820571673034 74.32495460765986) bank120165 +120166 POINT(40.320134821787434 73.76855767968185) bank120166 +120167 POINT(41.65704960671382 73.33304348945043) bank120167 +120168 POINT(40.22625365967165 73.99312241640827) bank120168 +120169 POINT(41.451524384409666 73.47896590694543) bank120169 +120170 POINT(40.66027747297777 73.23702822593025) bank120170 +120171 POINT(40.65722549533234 74.20222869542872) bank120171 +120172 POINT(40.18267992660774 74.89059892014222) bank120172 +120173 POINT(40.48291813461036 74.52363982936231) bank120173 +120174 POINT(41.59339694724625 74.70436939106682) bank120174 +120175 POINT(40.27460817921448 73.57734426515589) bank120175 +120176 POINT(40.95793893148109 74.41009082310963) bank120176 +120177 POINT(40.26618388755992 74.88239052057725) bank120177 +120178 POINT(41.26911239913786 74.99609464691386) bank120178 +120179 POINT(40.36869497956252 74.43470903233242) bank120179 +120180 POINT(40.77643232191653 74.89720771923805) bank120180 +120181 POINT(39.93978123773324 74.2198244760321) bank120181 +120182 POINT(40.89979496139947 74.89857152401467) bank120182 +120183 POINT(40.49070852318669 73.60193097885215) bank120183 +120184 POINT(41.27856621705209 73.04035369295202) bank120184 +120185 POINT(41.17701362044609 73.65424599130385) bank120185 +120186 POINT(41.58546018672525 73.49183478441725) bank120186 +120187 POINT(40.378987256675764 73.069599503793) bank120187 +120188 POINT(40.829802176488464 73.61597520445237) bank120188 +120189 POINT(41.474903934556046 74.07998841544068) bank120189 +120190 POINT(40.95000967131994 73.08375398562895) bank120190 +120191 POINT(41.211623939265294 74.97418607727256) bank120191 +120192 POINT(40.315182425545125 73.38502119576393) bank120192 +120193 POINT(39.8548442481236 74.92212734865609) bank120193 +120194 POINT(40.07970847676529 74.17596955847036) bank120194 +120195 POINT(40.0981510142817 74.84765295302998) bank120195 +120196 POINT(40.940048148307454 73.75426653497604) bank120196 +120197 POINT(40.42909719538807 73.89604341748863) bank120197 +120198 POINT(40.4117094574427 73.79278425516361) bank120198 +120199 POINT(41.44577503722258 74.56988696097726) bank120199 +120200 POINT(41.665092921929634 73.09979471056285) bank120200 +120201 POINT(39.97080763680951 73.58549719320074) bank120201 +120202 POINT(40.78362837925545 74.10414457035452) bank120202 +120203 POINT(40.54233676371222 74.84003282215082) bank120203 +120204 POINT(41.04019783078382 74.53134614080913) bank120204 +120205 POINT(40.85169966738205 74.85010058300483) bank120205 +120206 POINT(41.445117069979545 74.15042734837888) bank120206 +120207 POINT(40.11209367645739 73.43536076990338) bank120207 +120208 POINT(39.93435316466673 74.97983743964207) bank120208 +120209 POINT(41.04006919137468 74.36897199852159) bank120209 +120210 POINT(41.555890140999004 73.82761448092626) bank120210 +120211 POINT(40.04112368946319 74.36634562036826) bank120211 +120212 POINT(40.29209576449719 73.7894333030829) bank120212 +120213 POINT(39.96200190078484 73.67349313608767) bank120213 +120214 POINT(40.194096358362444 74.6015668935069) bank120214 +120215 POINT(41.368744681354976 74.37104632203213) bank120215 +120216 POINT(39.841671402769016 74.28386895578736) bank120216 +120217 POINT(39.87919174575673 74.04725516623377) bank120217 +120218 POINT(40.171472448887535 73.77244626113615) bank120218 +120219 POINT(41.20566106378948 73.73322291255995) bank120219 +120220 POINT(40.30462414253661 73.65074391532815) bank120220 +120221 POINT(40.83846172434313 73.23128830953263) bank120221 +120222 POINT(39.857329331947014 74.15306397114422) bank120222 +120223 POINT(41.285606659841065 73.99422881178552) bank120223 +120224 POINT(41.00428784338561 74.04658269547303) bank120224 +120225 POINT(40.328397662514924 74.92485165700253) bank120225 +120226 POINT(40.5698187249511 73.36292828132574) bank120226 +120227 POINT(41.70110259267496 74.02671541659008) bank120227 +120228 POINT(39.74249263714915 74.98592616511728) bank120228 +120229 POINT(39.83764778685426 73.44091878500187) bank120229 +120230 POINT(40.7961970313552 74.03242432855284) bank120230 +120231 POINT(40.86292040184257 73.7958773332343) bank120231 +120232 POINT(40.582332352029695 74.57465075215659) bank120232 +120233 POINT(40.8409861068699 73.76141378613075) bank120233 +120234 POINT(41.32704235970659 74.11825506033395) bank120234 +120235 POINT(41.14913949131802 74.09947750402931) bank120235 +120236 POINT(40.88566975590655 74.47243169089971) bank120236 +120237 POINT(40.71798675008612 74.62362512566123) bank120237 +120238 POINT(40.99415573534562 73.83429651666145) bank120238 +120239 POINT(41.30206405141623 73.53526477888963) bank120239 +120240 POINT(41.25613955254033 73.77087611945133) bank120240 +120241 POINT(41.08007253997625 74.13090449325112) bank120241 +120242 POINT(40.04550889175277 73.28188280424862) bank120242 +120243 POINT(40.42815350732689 73.55140706271975) bank120243 +120244 POINT(40.414958859413154 74.20157460926472) bank120244 +120245 POINT(40.829522412836596 73.06788984867318) bank120245 +120246 POINT(41.641244623039306 73.74766111250888) bank120246 +120247 POINT(39.71857405102693 73.8341881959682) bank120247 +120248 POINT(40.98605193537874 73.81548866320645) bank120248 +120249 POINT(41.364445442971146 74.32891664979722) bank120249 +120250 POINT(40.746475016172134 74.8294913999063) bank120250 +120251 POINT(41.33835034002207 74.28853964871035) bank120251 +120252 POINT(40.85812379747699 73.32182099298134) bank120252 +120253 POINT(39.74667651008196 73.69374834964258) bank120253 +120254 POINT(40.10146784926591 74.32089591265792) bank120254 +120255 POINT(40.65256026288335 74.97688670414685) bank120255 +120256 POINT(41.52446349906352 74.71979534894368) bank120256 +120257 POINT(41.39422508523868 73.58743060753051) bank120257 +120258 POINT(39.92878992841283 73.48702234095632) bank120258 +120259 POINT(40.133062700502435 73.67263067884893) bank120259 +120260 POINT(41.11448210081704 73.04028986326767) bank120260 +120261 POINT(41.30055777343269 73.40300683229285) bank120261 +120262 POINT(41.3474344016206 73.49856718574048) bank120262 +120263 POINT(40.6818162197777 74.21425708259035) bank120263 +120264 POINT(41.28066995535832 73.60001364822945) bank120264 +120265 POINT(40.59750999938433 74.8607753802904) bank120265 +120266 POINT(40.082086440375505 74.50490991780585) bank120266 +120267 POINT(39.98730493739053 73.00718988879223) bank120267 +120268 POINT(40.122154565437135 74.25527624356775) bank120268 +120269 POINT(40.47608249969219 74.15167192980523) bank120269 +120270 POINT(40.388629261607264 73.15633585735227) bank120270 +120271 POINT(40.263649088862174 74.95222536648814) bank120271 +120272 POINT(41.01289117693705 73.2092013223555) bank120272 +120273 POINT(40.93965172507214 73.21879790187289) bank120273 +120274 POINT(41.132157098920004 74.6205029347761) bank120274 +120275 POINT(40.43074394398715 73.76728527263502) bank120275 +120276 POINT(41.30733418741153 73.06049494277704) bank120276 +120277 POINT(40.75447369811301 73.52001903966323) bank120277 +120278 POINT(40.74194711359721 74.12075889487794) bank120278 +120279 POINT(39.839378479869204 73.01419351071571) bank120279 +120280 POINT(41.569070965995465 73.0469645740733) bank120280 +120281 POINT(40.76417365286707 73.21912786003925) bank120281 +120282 POINT(41.506674422521485 74.02030791622104) bank120282 +120283 POINT(40.016722857330194 73.77973510944203) bank120283 +120284 POINT(40.172628431362824 74.52086153760493) bank120284 +120285 POINT(41.36177032956896 74.46278587063622) bank120285 +120286 POINT(41.1284555310722 73.66422818423268) bank120286 +120287 POINT(39.926502727266154 73.5108738368711) bank120287 +120288 POINT(41.39889762931514 74.34478863562316) bank120288 +120289 POINT(40.27347559269371 73.78500300564089) bank120289 +120290 POINT(39.8255962247349 73.39998516776755) bank120290 +120291 POINT(40.33136611569524 73.52742531108312) bank120291 +120292 POINT(41.02503328960547 73.74747046888344) bank120292 +120293 POINT(40.49646853584104 73.50257688581812) bank120293 +120294 POINT(41.5886407103686 74.12685103713265) bank120294 +120295 POINT(41.00332602340101 74.04293853713054) bank120295 +120296 POINT(40.94723199201181 73.65974292540257) bank120296 +120297 POINT(41.24128346787746 74.00347124041792) bank120297 +120298 POINT(41.7036181352206 74.61608203843768) bank120298 +120299 POINT(41.10770343546384 74.67504251048888) bank120299 +120300 POINT(41.209498282523214 74.59712943495266) bank120300 +120301 POINT(41.195486787564974 73.67968335963147) bank120301 +120302 POINT(40.35628103040138 74.29127214051867) bank120302 +120303 POINT(41.30753337758923 74.25603276083757) bank120303 +120304 POINT(40.06291343359388 74.11017313520908) bank120304 +120305 POINT(40.951432756896324 73.72634496445471) bank120305 +120306 POINT(39.72280926177278 73.79518040700984) bank120306 +120307 POINT(40.81414212469739 74.77042321096461) bank120307 +120308 POINT(40.57780470417243 73.94708907597659) bank120308 +120309 POINT(40.54223546275089 74.93240711750413) bank120309 +120310 POINT(41.29269819638954 73.41736944466895) bank120310 +120311 POINT(41.35978315489498 73.0605624023497) bank120311 +120312 POINT(39.77886866299774 74.95983170217782) bank120312 +120313 POINT(40.24070382392764 74.55317794354154) bank120313 +120314 POINT(39.92871152900367 73.59218157454774) bank120314 +120315 POINT(40.863777896860775 74.01534436318784) bank120315 +120316 POINT(40.66631752377625 73.94713214857) bank120316 +120317 POINT(39.93436733144734 73.05452840443178) bank120317 +120318 POINT(41.12007958996303 74.97041573082352) bank120318 +120319 POINT(39.834818466585624 73.73219055439651) bank120319 +120320 POINT(40.746713886013005 73.97524127918913) bank120320 +120321 POINT(40.31024641336257 73.18555661541949) bank120321 +120322 POINT(40.59692477653528 73.68419217198739) bank120322 +120323 POINT(39.85173293597436 73.35794563422417) bank120323 +120324 POINT(40.06522618963987 73.32525837390912) bank120324 +120325 POINT(39.81728945171669 73.02958737586093) bank120325 +120326 POINT(40.19715419093525 74.07706209835146) bank120326 +120327 POINT(41.62094590700495 74.20654384065593) bank120327 +120328 POINT(40.62794887378781 74.91108094683209) bank120328 +120329 POINT(41.06477494592098 74.224076238363) bank120329 +120330 POINT(41.36314782421172 74.33986296795365) bank120330 +120331 POINT(39.850102065766905 74.75427087726213) bank120331 +120332 POINT(41.500741639758196 74.17764473318039) bank120332 +120333 POINT(41.00523076401932 73.69862299607058) bank120333 +120334 POINT(40.331911432010635 73.50189559483033) bank120334 +120335 POINT(39.725501982879834 73.47848662185164) bank120335 +120336 POINT(40.36868573213085 73.54318226882026) bank120336 +120337 POINT(40.86438423040646 73.62463979503731) bank120337 +120338 POINT(39.97983448629628 74.61826568040304) bank120338 +120339 POINT(40.47319466456825 74.03355274973019) bank120339 +120340 POINT(41.38806796952957 74.29484129166592) bank120340 +120341 POINT(41.32688155308157 73.41329203099264) bank120341 +120342 POINT(40.51298187000086 73.76163720817945) bank120342 +120343 POINT(39.92814240273864 73.82171708284659) bank120343 +120344 POINT(40.77797115237694 74.33030533307861) bank120344 +120345 POINT(40.83401050982413 73.3785153335431) bank120345 +120346 POINT(41.30883763955331 74.39217789591963) bank120346 +120347 POINT(40.186091942738884 73.74289824094234) bank120347 +120348 POINT(40.84508724824672 74.65376340955116) bank120348 +120349 POINT(40.171296582126516 73.76080523062284) bank120349 +120350 POINT(40.1755394305796 73.12783361389954) bank120350 +120351 POINT(41.02418338251604 73.45597127942428) bank120351 +120352 POINT(40.57837340080856 73.70112545959677) bank120352 +120353 POINT(40.90485540054413 73.75928539372475) bank120353 +120354 POINT(40.37638589114708 73.55436406761895) bank120354 +120355 POINT(40.377222200467536 74.22970651635211) bank120355 +120356 POINT(39.74095631845409 74.43087159531198) bank120356 +120357 POINT(40.191173974489 74.3745865860363) bank120357 +120358 POINT(41.016873878314925 73.79584996250203) bank120358 +120359 POINT(41.46146579210654 73.41385998433576) bank120359 +120360 POINT(40.41639317266512 74.42944021489357) bank120360 +120361 POINT(41.060718653041334 74.39744502260491) bank120361 +120362 POINT(40.0185498137668 74.34121259920582) bank120362 +120363 POINT(41.69571953004803 74.0899515245203) bank120363 +120364 POINT(40.44002676151041 73.23389593147661) bank120364 +120365 POINT(40.99741885923154 73.9934500879546) bank120365 +120366 POINT(41.03637823121185 74.10727994933517) bank120366 +120367 POINT(39.81679383373294 74.6258036919602) bank120367 +120368 POINT(41.40519331211622 74.50309302347492) bank120368 +120369 POINT(40.10211141787824 74.28208222488621) bank120369 +120370 POINT(41.35287737961445 74.507530572215) bank120370 +120371 POINT(41.328483663977394 74.66841708882305) bank120371 +120372 POINT(40.613208175024994 73.49964018812318) bank120372 +120373 POINT(41.07600546184353 74.55245647229592) bank120373 +120374 POINT(40.680550756152066 73.04140110457723) bank120374 +120375 POINT(39.77008506589294 74.68268194816127) bank120375 +120376 POINT(41.61818845253784 73.50744470761309) bank120376 +120377 POINT(41.40900164709003 74.81428510799455) bank120377 +120378 POINT(40.702623304641946 74.73749007342295) bank120378 +120379 POINT(40.32921986120106 74.12629850499994) bank120379 +120380 POINT(40.300427048453194 73.38567803125426) bank120380 +120381 POINT(41.01195695849251 73.75675830495737) bank120381 +120382 POINT(40.066162258654 74.35760653551607) bank120382 +120383 POINT(41.688167203829444 74.65715630428413) bank120383 +120384 POINT(40.749422804909734 74.39252177158764) bank120384 +120385 POINT(41.28457991533535 73.43573621226146) bank120385 +120386 POINT(40.99823294377179 74.49977167044743) bank120386 +120387 POINT(39.94718052357426 74.73251624820337) bank120387 +120388 POINT(41.185965413208194 73.52439930873436) bank120388 +120389 POINT(40.46232879512602 74.26407830803419) bank120389 +120390 POINT(39.75479525446684 74.30325434943674) bank120390 +120391 POINT(39.929659315041064 73.6691525144201) bank120391 +120392 POINT(40.389688698372666 73.1805520165101) bank120392 +120393 POINT(40.76202231117787 73.91014123347799) bank120393 +120394 POINT(39.92705105941339 73.34656558692394) bank120394 +120395 POINT(41.62009190477259 73.6151197426856) bank120395 +120396 POINT(41.69070791890547 73.21333483183527) bank120396 +120397 POINT(39.727600284784955 73.22302397171745) bank120397 +120398 POINT(41.09331569543401 73.95011739154813) bank120398 +120399 POINT(40.70560258464524 73.38673141766287) bank120399 +120400 POINT(40.33524586328847 73.00940856773467) bank120400 +120401 POINT(41.18762656595203 74.79217477120056) bank120401 +120402 POINT(40.907757531776845 73.78963416667699) bank120402 +120403 POINT(39.94214612373443 73.45029910652464) bank120403 +120404 POINT(41.499533379322365 73.27471840268389) bank120404 +120405 POINT(40.65051209015477 73.26778553279536) bank120405 +120406 POINT(40.970041690624726 73.14386486101407) bank120406 +120407 POINT(41.10725228759798 73.3274811325368) bank120407 +120408 POINT(41.42962766866869 74.85911373352769) bank120408 +120409 POINT(40.95244272680825 73.53148833787942) bank120409 +120410 POINT(40.21114655445829 73.62024823913347) bank120410 +120411 POINT(41.03734943687189 74.64388989352719) bank120411 +120412 POINT(41.568764152715204 74.80157453502889) bank120412 +120413 POINT(40.1309950713092 73.22657638742933) bank120413 +120414 POINT(40.487997877219165 74.59405611606348) bank120414 +120415 POINT(41.37644297757808 74.69790437195063) bank120415 +120416 POINT(41.33682221949909 74.83157174427804) bank120416 +120417 POINT(41.33257947481769 74.79862917612323) bank120417 +120418 POINT(41.131169604798224 74.27037916328894) bank120418 +120419 POINT(40.0688712757049 74.19371125792377) bank120419 +120420 POINT(41.46455312209408 74.13770781417345) bank120420 +120421 POINT(41.49496558340861 73.65592497595652) bank120421 +120422 POINT(40.13889999904033 73.91194077029583) bank120422 +120423 POINT(40.13878564356134 74.68697921770303) bank120423 +120424 POINT(39.79804092873081 74.89866557450888) bank120424 +120425 POINT(41.24254815441961 74.53414851992217) bank120425 +120426 POINT(40.53824156638671 74.79290593179546) bank120426 +120427 POINT(40.868237877933375 73.69535274264614) bank120427 +120428 POINT(40.127626467080425 74.04753901739012) bank120428 +120429 POINT(39.89760304332527 74.54774209726297) bank120429 +120430 POINT(39.957885696070385 74.6576388143349) bank120430 +120431 POINT(41.11921271606621 73.15557043889159) bank120431 +120432 POINT(41.10842994360039 74.56287915463811) bank120432 +120433 POINT(41.070352571237414 74.04187446401413) bank120433 +120434 POINT(40.68870014820189 74.1697974515231) bank120434 +120435 POINT(41.18160076152601 74.29681393575716) bank120435 +120436 POINT(40.96186854791674 74.84035101040918) bank120436 +120437 POINT(40.80064737621073 74.1055437328177) bank120437 +120438 POINT(40.736729295228976 73.57252794094143) bank120438 +120439 POINT(41.356439792216065 73.01202749133486) bank120439 +120440 POINT(41.097451407059175 74.65941985853644) bank120440 +120441 POINT(39.750579464587126 73.91547822964455) bank120441 +120442 POINT(40.05836660431883 73.57535909503353) bank120442 +120443 POINT(40.487170056093206 73.08350749115182) bank120443 +120444 POINT(39.73468274917478 74.03208572289438) bank120444 +120445 POINT(39.830950735583635 74.97306138659908) bank120445 +120446 POINT(39.96987448851886 73.41626240969335) bank120446 +120447 POINT(39.73433899193817 73.38771375431828) bank120447 +120448 POINT(40.15010733874315 74.96877903777482) bank120448 +120449 POINT(40.60334038355019 74.63801709066475) bank120449 +120450 POINT(40.322084649385886 74.90062123163656) bank120450 +120451 POINT(41.13596797711559 73.30304392945209) bank120451 +120452 POINT(40.65962946105195 73.03999144945105) bank120452 +120453 POINT(40.0101055699794 73.68722404438373) bank120453 +120454 POINT(41.565742778524246 74.85888611799624) bank120454 +120455 POINT(40.38235330106896 73.8286895165437) bank120455 +120456 POINT(41.7032204804344 74.25114241414393) bank120456 +120457 POINT(39.76799259714743 73.68036092113934) bank120457 +120458 POINT(41.31097423002774 74.84573947186169) bank120458 +120459 POINT(39.779456200025 74.34193763095722) bank120459 +120460 POINT(40.46161403548009 73.3208485461195) bank120460 +120461 POINT(40.85182950953629 73.77091270579517) bank120461 +120462 POINT(41.26786294507269 74.43805704705883) bank120462 +120463 POINT(41.485986450568134 73.87529043003047) bank120463 +120464 POINT(41.246039846331314 73.32126307368456) bank120464 +120465 POINT(41.060926200254244 73.98165044383458) bank120465 +120466 POINT(40.0771807402113 74.13225782885085) bank120466 +120467 POINT(41.576361885506714 73.51558804639791) bank120467 +120468 POINT(40.91304870307155 74.79923152063829) bank120468 +120469 POINT(41.6066854479414 74.21151878039797) bank120469 +120470 POINT(41.23105053717731 73.0977501325153) bank120470 +120471 POINT(40.20031171342801 74.70501777027008) bank120471 +120472 POINT(40.871608769906025 73.78751910820291) bank120472 +120473 POINT(40.04106505727547 73.45750589307163) bank120473 +120474 POINT(39.98127996479351 74.31957448183711) bank120474 +120475 POINT(41.129831210932196 74.89922681027589) bank120475 +120476 POINT(40.012266268770674 74.66532629684879) bank120476 +120477 POINT(41.67867386305201 74.67505036839908) bank120477 +120478 POINT(40.988603505292424 73.77608123114805) bank120478 +120479 POINT(40.65390001537495 74.12269855542132) bank120479 +120480 POINT(41.27638163207835 74.58352614783887) bank120480 +120481 POINT(41.02230596694672 74.89106305061529) bank120481 +120482 POINT(41.046988386663514 73.18294410224074) bank120482 +120483 POINT(41.69850872504264 73.02452803456666) bank120483 +120484 POINT(41.28673739406395 74.69725241805389) bank120484 +120485 POINT(40.26884297186005 74.08743476341769) bank120485 +120486 POINT(41.336889948387935 74.90579872846745) bank120486 +120487 POINT(41.53538533057227 73.2755616542517) bank120487 +120488 POINT(39.82741671427635 73.52917983493373) bank120488 +120489 POINT(40.8985112851499 73.87592449587956) bank120489 +120490 POINT(40.13852889535524 74.5666450285315) bank120490 +120491 POINT(41.41057979689198 74.6323955781045) bank120491 +120492 POINT(40.619401227725035 73.04076414873535) bank120492 +120493 POINT(39.98210037355934 73.47438658599353) bank120493 +120494 POINT(40.45572648459937 73.43346087861528) bank120494 +120495 POINT(40.97247839123249 74.98033747822429) bank120495 +120496 POINT(40.97899953716101 73.83320876422734) bank120496 +120497 POINT(41.15429739309101 73.09461184146456) bank120497 +120498 POINT(40.1301682370514 74.1497583311815) bank120498 +120499 POINT(41.504066043873 73.02015381327595) bank120499 +120500 POINT(39.83768278023504 73.93114089780597) bank120500 +120501 POINT(41.63896924210331 74.36799443351426) bank120501 +120502 POINT(41.603933330631804 74.56896263106378) bank120502 +120503 POINT(40.3361207437141 74.42397708675608) bank120503 +120504 POINT(41.341433035242844 73.5098349849416) bank120504 +120505 POINT(40.1209203555817 74.21813810514905) bank120505 +120506 POINT(40.56386692962959 73.85687128750403) bank120506 +120507 POINT(40.718751959684454 74.41194972448069) bank120507 +120508 POINT(40.37145808620055 73.62304719073425) bank120508 +120509 POINT(40.63633061465399 74.99141861192513) bank120509 +120510 POINT(40.638587746346985 73.81404286631222) bank120510 +120511 POINT(40.67378204553524 74.28435878450323) bank120511 +120512 POINT(40.74513763948561 73.75987439634515) bank120512 +120513 POINT(39.815588774188754 73.55954300464983) bank120513 +120514 POINT(40.92857686342166 73.78549513369728) bank120514 +120515 POINT(40.47628149112328 74.38657237041294) bank120515 +120516 POINT(40.15302216128395 73.10713485893532) bank120516 +120517 POINT(41.42020893715756 74.11275412038316) bank120517 +120518 POINT(40.295354572663406 74.80223738383842) bank120518 +120519 POINT(40.75731060697153 74.77072243585373) bank120519 +120520 POINT(41.06061055615618 74.06677169910266) bank120520 +120521 POINT(40.40189288326262 74.0532579258253) bank120521 +120522 POINT(40.00959188952025 74.71096317121777) bank120522 +120523 POINT(39.90630533037647 74.37561122435378) bank120523 +120524 POINT(40.820311498515345 73.88737847711118) bank120524 +120525 POINT(40.966926489721466 73.52120048171366) bank120525 +120526 POINT(40.47334282588803 73.33703537410213) bank120526 +120527 POINT(39.78433250706341 73.46031530728838) bank120527 +120528 POINT(39.95939967376738 74.5641670749608) bank120528 +120529 POINT(40.21425074711353 73.01079065299135) bank120529 +120530 POINT(39.7218520694587 73.79201158357586) bank120530 +120531 POINT(40.54037705892547 74.59666301572064) bank120531 +120532 POINT(40.16833271295345 73.15839842860265) bank120532 +120533 POINT(41.47410633762721 73.92742434825917) bank120533 +120534 POINT(41.0674794120009 73.50313943316836) bank120534 +120535 POINT(40.99718870986591 74.82962452227275) bank120535 +120536 POINT(41.340304542381915 74.78763827694509) bank120536 +120537 POINT(40.51160135421254 73.23131354191312) bank120537 +120538 POINT(40.980955324666176 74.13361397503306) bank120538 +120539 POINT(41.52836330826872 74.34521916953975) bank120539 +120540 POINT(40.51003253142978 74.22747145339419) bank120540 +120541 POINT(41.36396145349005 74.38577689280342) bank120541 +120542 POINT(40.917132353402394 73.1090395782051) bank120542 +120543 POINT(40.824026998389186 73.49156823341023) bank120543 +120544 POINT(41.02797173049241 74.15854252141187) bank120544 +120545 POINT(40.73227623324703 73.64599649300955) bank120545 +120546 POINT(40.79929825084554 74.96650762468649) bank120546 +120547 POINT(41.29973988395441 73.52228080256478) bank120547 +120548 POINT(41.63779687565029 74.51059616512407) bank120548 +120549 POINT(39.93094163722601 74.89081815733282) bank120549 +120550 POINT(41.68120496210995 73.35325035716055) bank120550 +120551 POINT(40.71998477433097 73.46118385246224) bank120551 +120552 POINT(39.87821147101359 73.22182308600709) bank120552 +120553 POINT(40.988602818469424 74.2168777836513) bank120553 +120554 POINT(41.13143349242686 74.70555254035936) bank120554 +120555 POINT(40.02168223947332 74.43996502929437) bank120555 +120556 POINT(40.31642645621507 73.01918759719592) bank120556 +120557 POINT(40.62374559460242 74.24182404510726) bank120557 +120558 POINT(41.47314266250096 74.7086806192756) bank120558 +120559 POINT(40.31506505171273 74.24446620141468) bank120559 +120560 POINT(41.10918750405713 73.82212567126945) bank120560 +120561 POINT(40.470764099329976 73.70755121561483) bank120561 +120562 POINT(40.971249139890304 73.67083124449502) bank120562 +120563 POINT(40.810704261150896 74.0548305107475) bank120563 +120564 POINT(40.786446662476635 74.99672638188343) bank120564 +120565 POINT(41.45397933785884 73.10370420472631) bank120565 +120566 POINT(40.86748120683089 73.14056134810608) bank120566 +120567 POINT(40.890821734487375 73.18876126075497) bank120567 +120568 POINT(39.912350541146104 74.94030052651645) bank120568 +120569 POINT(40.8372061589781 74.23360679882583) bank120569 +120570 POINT(41.57471618079036 73.86237775849587) bank120570 +120571 POINT(41.56842398306194 74.30826631881646) bank120571 +120572 POINT(40.73332037934756 74.53508816179472) bank120572 +120573 POINT(41.19006454524189 74.15780170892396) bank120573 +120574 POINT(41.049639133966785 74.4905613754407) bank120574 +120575 POINT(40.32582594823413 73.92403954056562) bank120575 +120576 POINT(41.48745291700763 73.68030713733881) bank120576 +120577 POINT(40.86922747265031 73.99542404870148) bank120577 +120578 POINT(40.15938149217663 73.94403973247312) bank120578 +120579 POINT(41.08956352441482 74.52266485847034) bank120579 +120580 POINT(40.7008776628723 74.20594324382267) bank120580 +120581 POINT(40.619543150761544 73.94773137356967) bank120581 +120582 POINT(40.18330972858452 74.02001758354398) bank120582 +120583 POINT(40.08566973651801 74.84469272937609) bank120583 +120584 POINT(40.55349096417621 74.43521957266098) bank120584 +120585 POINT(41.71145939870681 73.45206766514535) bank120585 +120586 POINT(40.96993743454601 74.7744347107425) bank120586 +120587 POINT(41.02088955134689 74.07946465781643) bank120587 +120588 POINT(39.89832198644404 74.76214771970922) bank120588 +120589 POINT(41.668171821724904 74.01478428306311) bank120589 +120590 POINT(41.10660753010692 74.77603187410634) bank120590 +120591 POINT(41.046583139910695 74.91868188284563) bank120591 +120592 POINT(41.025692997277055 74.81484875211908) bank120592 +120593 POINT(40.159355659697816 73.15544108009637) bank120593 +120594 POINT(41.253412682925294 73.98326858873605) bank120594 +120595 POINT(41.40551095890403 73.78048836929939) bank120595 +120596 POINT(40.36729176878717 74.22684474694897) bank120596 +120597 POINT(41.683446585732156 74.53686110979193) bank120597 +120598 POINT(39.95272863463491 74.66076047002042) bank120598 +120599 POINT(40.63261908663226 73.56153604594489) bank120599 +120600 POINT(41.57709766484829 73.25408531892518) bank120600 +120601 POINT(39.88656612504203 74.63018245206784) bank120601 +120602 POINT(40.24716367041189 74.13522430209724) bank120602 +120603 POINT(40.567961435070536 74.12578638241455) bank120603 +120604 POINT(40.62395560410551 74.02631860311507) bank120604 +120605 POINT(40.592681197279745 74.01556026817383) bank120605 +120606 POINT(40.56543723771919 73.44825503129547) bank120606 +120607 POINT(40.87703700275125 73.16944480012909) bank120607 +120608 POINT(41.225138675976886 74.74173191153584) bank120608 +120609 POINT(40.71560101893124 73.67989531305327) bank120609 +120610 POINT(39.835913983265584 73.43987108160424) bank120610 +120611 POINT(40.83183744068368 74.37033699615384) bank120611 +120612 POINT(41.27814276203243 73.15345810145317) bank120612 +120613 POINT(41.52875443570115 74.61397804634683) bank120613 +120614 POINT(40.8718523423434 74.24805416817794) bank120614 +120615 POINT(40.88136715061525 73.23807050805095) bank120615 +120616 POINT(40.687659857705675 74.28711242338268) bank120616 +120617 POINT(40.8878512805055 74.83629767938683) bank120617 +120618 POINT(40.886052246613474 73.70042574983987) bank120618 +120619 POINT(40.45691800897229 74.91170653423475) bank120619 +120620 POINT(41.16270806200498 74.92061863635972) bank120620 +120621 POINT(40.37924723222699 73.26013507001913) bank120621 +120622 POINT(40.18462080403338 74.87849597639116) bank120622 +120623 POINT(41.67378703509472 73.83886450314694) bank120623 +120624 POINT(41.141221166580074 73.84825257960735) bank120624 +120625 POINT(40.37785523602884 74.03941885599563) bank120625 +120626 POINT(40.43063487319144 74.12332597758429) bank120626 +120627 POINT(40.99659851513856 73.41315920022865) bank120627 +120628 POINT(40.222478420544086 74.4352210061015) bank120628 +120629 POINT(40.562521288033324 74.31840797339126) bank120629 +120630 POINT(41.407508981749324 73.4474374713638) bank120630 +120631 POINT(41.515287253121166 73.1220116713495) bank120631 +120632 POINT(40.41049849566383 73.61753824651923) bank120632 +120633 POINT(40.03754197326865 74.61823190184056) bank120633 +120634 POINT(40.53385024267867 74.64763668355742) bank120634 +120635 POINT(40.60198511169136 73.40376691906148) bank120635 +120636 POINT(39.920908486498504 73.26659394718253) bank120636 +120637 POINT(40.7672311647644 74.06010659388282) bank120637 +120638 POINT(40.72250562428216 73.63291063356627) bank120638 +120639 POINT(41.297025334144145 74.15055876570968) bank120639 +120640 POINT(39.92300970788467 73.47273660325237) bank120640 +120641 POINT(40.48503684508579 73.31676473679464) bank120641 +120642 POINT(39.80443619143538 74.98514014232212) bank120642 +120643 POINT(39.74388916949899 73.41276898376958) bank120643 +120644 POINT(40.83621805928678 73.61296758734359) bank120644 +120645 POINT(41.086789946767844 74.23468421008657) bank120645 +120646 POINT(40.489806985913965 73.92838187002529) bank120646 +120647 POINT(39.84766955453607 73.13260245997837) bank120647 +120648 POINT(41.28644420343019 73.68388690377292) bank120648 +120649 POINT(40.92292818655161 73.93476883232512) bank120649 +120650 POINT(39.904847018062256 74.17885418696389) bank120650 +120651 POINT(41.49917817947939 73.31348169787897) bank120651 +120652 POINT(40.433163775759645 73.51741557901228) bank120652 +120653 POINT(41.51299590466523 74.87808607642644) bank120653 +120654 POINT(40.73986920331043 74.7152158940353) bank120654 +120655 POINT(41.55345512547113 74.86475556231144) bank120655 +120656 POINT(40.984839579711185 74.2156587499442) bank120656 +120657 POINT(41.65772582442637 73.9007240154783) bank120657 +120658 POINT(40.744890586527255 73.85424201423007) bank120658 +120659 POINT(40.25679373017925 73.96195871511966) bank120659 +120660 POINT(39.74517682873552 73.96563910344517) bank120660 +120661 POINT(39.97946002573507 73.17091641473642) bank120661 +120662 POINT(41.64242920826769 73.99998230229637) bank120662 +120663 POINT(40.88157947467233 73.28516157102266) bank120663 +120664 POINT(40.18607780116405 73.46548938410587) bank120664 +120665 POINT(41.43308889693522 73.65481102457929) bank120665 +120666 POINT(41.331254288065615 73.20911158181195) bank120666 +120667 POINT(40.6531755748411 73.74765649615375) bank120667 +120668 POINT(39.796358949802 73.44966208868195) bank120668 +120669 POINT(41.07719091488391 74.99015336733817) bank120669 +120670 POINT(41.65107815405468 74.78529511500611) bank120670 +120671 POINT(41.61359100014056 73.12417497074472) bank120671 +120672 POINT(40.854393417537445 74.31145146920261) bank120672 +120673 POINT(40.4617954189657 73.2815316893977) bank120673 +120674 POINT(40.432887468559876 74.81532202481839) bank120674 +120675 POINT(39.84618737984217 73.64561307819281) bank120675 +120676 POINT(41.076294288544744 73.99008692474494) bank120676 +120677 POINT(41.42783068578096 73.22944122146207) bank120677 +120678 POINT(40.96892268731056 74.3987470815538) bank120678 +120679 POINT(40.901560634020235 73.62182891741938) bank120679 +120680 POINT(40.883142034373385 73.8181793954159) bank120680 +120681 POINT(40.08029311244324 74.84721621881475) bank120681 +120682 POINT(40.33510960620987 73.25655246886494) bank120682 +120683 POINT(40.542046552611886 73.83146431844635) bank120683 +120684 POINT(40.59842743877705 74.54023099117626) bank120684 +120685 POINT(40.53452734971049 73.1031241681789) bank120685 +120686 POINT(40.16929892951084 73.18875257899329) bank120686 +120687 POINT(40.46798565354521 74.76951669890754) bank120687 +120688 POINT(40.918046408948314 74.16149599625186) bank120688 +120689 POINT(40.743714343009344 74.79183910934746) bank120689 +120690 POINT(40.79384746045197 74.99404880680211) bank120690 +120691 POINT(41.35519913003627 74.39937767722084) bank120691 +120692 POINT(41.66128082229451 74.53088351551388) bank120692 +120693 POINT(41.165059842148615 74.53389676210917) bank120693 +120694 POINT(40.140480975733766 74.26027980989913) bank120694 +120695 POINT(39.980331545337854 74.01134370342665) bank120695 +120696 POINT(41.053717431323406 74.53065345555828) bank120696 +120697 POINT(41.23885877111069 73.83255645228937) bank120697 +120698 POINT(40.422561940070025 73.54724227073706) bank120698 +120699 POINT(41.67914127424018 73.43524251840316) bank120699 +120700 POINT(41.03907862293811 74.10713848329863) bank120700 +120701 POINT(41.31529284896011 73.35298326261986) bank120701 +120702 POINT(41.0753763549417 73.89528938903256) bank120702 +120703 POINT(40.98697086752766 74.06921025381115) bank120703 +120704 POINT(40.956173175416474 74.7049522297101) bank120704 +120705 POINT(40.493076654488306 73.79599710242903) bank120705 +120706 POINT(40.033175234119945 73.92709537018236) bank120706 +120707 POINT(41.27900664206814 73.63520971535509) bank120707 +120708 POINT(41.25224841933205 73.19052484930357) bank120708 +120709 POINT(40.70547970624675 73.60982844169075) bank120709 +120710 POINT(41.711518316091784 73.71819147692906) bank120710 +120711 POINT(41.47462938878247 73.55037970088824) bank120711 +120712 POINT(41.202747808809114 74.41815715384377) bank120712 +120713 POINT(41.43404923203468 73.94139974207538) bank120713 +120714 POINT(40.48847672209809 74.2771207553548) bank120714 +120715 POINT(41.66956553187678 73.40023344285532) bank120715 +120716 POINT(40.36783297778369 74.66023990151174) bank120716 +120717 POINT(41.28150178623205 73.77606412976964) bank120717 +120718 POINT(41.27290114826381 73.70377670342516) bank120718 +120719 POINT(40.21071365212503 74.88155163401977) bank120719 +120720 POINT(40.03727700773636 74.74005510941315) bank120720 +120721 POINT(39.773614010864975 74.0356159137986) bank120721 +120722 POINT(40.84513523003088 73.8690461712806) bank120722 +120723 POINT(41.387712283335894 74.09965294875082) bank120723 +120724 POINT(41.39795574556013 74.61971121899951) bank120724 +120725 POINT(41.01309856617359 73.29111007097283) bank120725 +120726 POINT(39.758221680103446 74.01089448248429) bank120726 +120727 POINT(40.50107008906308 74.58042343815401) bank120727 +120728 POINT(41.11592178942615 73.4235986490302) bank120728 +120729 POINT(39.745761610343486 74.95317677968349) bank120729 +120730 POINT(39.9617099231831 73.89131736077869) bank120730 +120731 POINT(41.691407734736906 73.35756623777094) bank120731 +120732 POINT(39.73679049231727 73.16664831901682) bank120732 +120733 POINT(41.49727023186996 74.24255856160151) bank120733 +120734 POINT(40.570022935115986 73.90926172589452) bank120734 +120735 POINT(41.120082830614734 74.70545939536534) bank120735 +120736 POINT(41.252086161111464 73.35117223231688) bank120736 +120737 POINT(41.14360034891448 74.04551408833836) bank120737 +120738 POINT(40.22499142842671 74.16822049763161) bank120738 +120739 POINT(40.48961242780687 74.4302923193547) bank120739 +120740 POINT(41.23487356347523 73.20052739228166) bank120740 +120741 POINT(40.592276942347965 73.72977386457397) bank120741 +120742 POINT(40.754685495595986 74.28151484285351) bank120742 +120743 POINT(41.38653329648631 73.46392003348622) bank120743 +120744 POINT(39.91992688657873 74.2428524915434) bank120744 +120745 POINT(40.081705920287455 73.39981036751882) bank120745 +120746 POINT(41.57066628520119 74.9713143639177) bank120746 +120747 POINT(40.35230072942281 74.36691746786845) bank120747 +120748 POINT(41.227378812167906 74.98777701490083) bank120748 +120749 POINT(39.861654346859076 73.7587577118724) bank120749 +120750 POINT(40.46017080533207 74.76754905802055) bank120750 +120751 POINT(41.352786533667384 73.82415806018668) bank120751 +120752 POINT(40.03812548524551 73.96034509179948) bank120752 +120753 POINT(39.915283772699674 74.25840010334686) bank120753 +120754 POINT(39.75817937520904 73.69279219432335) bank120754 +120755 POINT(40.8447575575919 74.72566897813289) bank120755 +120756 POINT(40.651237639895356 73.41904389338659) bank120756 +120757 POINT(40.41291714144935 73.46384490261362) bank120757 +120758 POINT(41.63942744272935 73.88729294634436) bank120758 +120759 POINT(40.849662449383075 73.29627714463508) bank120759 +120760 POINT(40.9778263547679 74.07257783598598) bank120760 +120761 POINT(40.552933318849135 74.63008890242288) bank120761 +120762 POINT(40.052774599540584 73.11695724537196) bank120762 +120763 POINT(39.98894031490401 73.46895290522578) bank120763 +120764 POINT(39.881844803087986 74.073507596509) bank120764 +120765 POINT(41.43197385029798 74.5408912849394) bank120765 +120766 POINT(40.41948302246361 74.51610590400911) bank120766 +120767 POINT(40.3733929665529 74.10544384859841) bank120767 +120768 POINT(40.67880531537088 74.39394627594386) bank120768 +120769 POINT(40.21849297551161 73.08356773760597) bank120769 +120770 POINT(39.74379340436161 73.69497967445726) bank120770 +120771 POINT(39.948550739113756 74.75846768248881) bank120771 +120772 POINT(41.38330739041654 74.10733378787974) bank120772 +120773 POINT(40.934872007590265 73.44697896799158) bank120773 +120774 POINT(40.652294301785666 74.27846001642658) bank120774 +120775 POINT(41.18331153030882 73.7131808156215) bank120775 +120776 POINT(40.04494955812561 74.25940600086788) bank120776 +120777 POINT(41.19895213867211 73.84304510217227) bank120777 +120778 POINT(41.593916669098704 73.41493579948879) bank120778 +120779 POINT(40.46316575464327 73.63758417830829) bank120779 +120780 POINT(40.29228012290228 74.78705662942943) bank120780 +120781 POINT(40.69056113058132 74.37897813096491) bank120781 +120782 POINT(41.18156802484944 73.36023440696371) bank120782 +120783 POINT(40.003221146081586 73.72277217578404) bank120783 +120784 POINT(40.831631055912915 74.40499408256477) bank120784 +120785 POINT(39.87897524846771 73.95889563133278) bank120785 +120786 POINT(41.442804323924065 73.29893914004735) bank120786 +120787 POINT(40.172091476672456 73.19611554638033) bank120787 +120788 POINT(39.90552821549585 73.57700523694275) bank120788 +120789 POINT(40.32651111207678 74.9561898665068) bank120789 +120790 POINT(41.52391792756521 74.91454725552668) bank120790 +120791 POINT(40.59582904292508 74.94538192247659) bank120791 +120792 POINT(39.9340047230811 74.3737655808787) bank120792 +120793 POINT(39.94447721166483 73.88781446642909) bank120793 +120794 POINT(40.94179726193528 74.53176602024372) bank120794 +120795 POINT(40.85905190617424 74.50129442924808) bank120795 +120796 POINT(40.35108157672217 73.4167855856514) bank120796 +120797 POINT(40.329325636681176 73.27050744417394) bank120797 +120798 POINT(39.81672562033598 74.31677562053082) bank120798 +120799 POINT(40.69671570284558 73.85691980112298) bank120799 +120800 POINT(40.36354416883668 74.09972167015646) bank120800 +120801 POINT(40.20666587939868 74.10359000925017) bank120801 +120802 POINT(40.810184571887234 73.26858937045701) bank120802 +120803 POINT(41.309587530910896 73.4395577578116) bank120803 +120804 POINT(41.31228402572651 74.57911228330859) bank120804 +120805 POINT(39.91975586819395 73.79091968829766) bank120805 +120806 POINT(40.15838802744246 73.68196011144529) bank120806 +120807 POINT(40.08063374187063 74.97539996746428) bank120807 +120808 POINT(41.307925462058726 73.74294689012429) bank120808 +120809 POINT(40.59855240617093 74.6051569943298) bank120809 +120810 POINT(41.31497919568633 74.64300648634806) bank120810 +120811 POINT(40.782572134002784 74.43872624634261) bank120811 +120812 POINT(41.03063411994237 73.35997468553585) bank120812 +120813 POINT(41.55413854387617 74.41566255761366) bank120813 +120814 POINT(41.52889440725451 74.96672520649685) bank120814 +120815 POINT(39.73458815384113 74.83706857495153) bank120815 +120816 POINT(41.19845321465033 73.80101313661712) bank120816 +120817 POINT(41.027910092265834 74.64296250490386) bank120817 +120818 POINT(41.03314389900548 73.9426786386093) bank120818 +120819 POINT(40.09988114805108 74.37526639372057) bank120819 +120820 POINT(41.100000011477555 74.91514853021961) bank120820 +120821 POINT(40.55686118290434 73.5903458305445) bank120821 +120822 POINT(40.27481061589852 74.14549604684689) bank120822 +120823 POINT(40.30453090401364 74.56292148851182) bank120823 +120824 POINT(41.622959728663105 73.76816947385848) bank120824 +120825 POINT(39.93730956462284 74.33502308365973) bank120825 +120826 POINT(40.53901734327046 74.09202195118151) bank120826 +120827 POINT(40.770880729325924 73.44323646995623) bank120827 +120828 POINT(41.39132656799978 74.91823488053512) bank120828 +120829 POINT(41.629787735940305 74.19577907789096) bank120829 +120830 POINT(40.86346914213114 74.97321998757833) bank120830 +120831 POINT(40.08256368235401 74.61521345768406) bank120831 +120832 POINT(40.49526970798104 73.92328742260108) bank120832 +120833 POINT(40.23193497745231 74.80522659494936) bank120833 +120834 POINT(40.26095466440773 73.65946362099564) bank120834 +120835 POINT(39.962138642084625 74.32386141717028) bank120835 +120836 POINT(39.91614185251858 74.90132581226142) bank120836 +120837 POINT(41.063328180946954 74.95095354775465) bank120837 +120838 POINT(40.91645405884407 74.63776875921437) bank120838 +120839 POINT(40.88902496059768 73.46835994779754) bank120839 +120840 POINT(39.82990617315436 74.05297735520743) bank120840 +120841 POINT(41.15136818332191 73.80676487566308) bank120841 +120842 POINT(40.24840088111235 74.05137923307085) bank120842 +120843 POINT(41.462858326628634 74.37231847241858) bank120843 +120844 POINT(40.791010642225054 74.34646186612271) bank120844 +120845 POINT(40.978576292005954 74.05718387213824) bank120845 +120846 POINT(41.342878639068786 73.88745525262377) bank120846 +120847 POINT(39.89577079521443 74.41518858435913) bank120847 +120848 POINT(41.50024566719202 73.06147193383435) bank120848 +120849 POINT(41.01247364640574 74.80343842613854) bank120849 +120850 POINT(41.2941984847603 74.85707281295511) bank120850 +120851 POINT(40.89030196616039 74.00656446046386) bank120851 +120852 POINT(39.74345004397444 74.42399151233823) bank120852 +120853 POINT(40.5685943010826 73.2838726992876) bank120853 +120854 POINT(40.67796655008078 74.80260582373138) bank120854 +120855 POINT(39.859529736113224 73.9230480566011) bank120855 +120856 POINT(40.42335965582903 74.41870080220755) bank120856 +120857 POINT(41.07293778852778 73.17380573539765) bank120857 +120858 POINT(39.749700726920324 74.6877177639408) bank120858 +120859 POINT(40.76671726980499 74.50731522263025) bank120859 +120860 POINT(41.518115413703605 74.3752258809533) bank120860 +120861 POINT(40.44131569775263 74.50835250893422) bank120861 +120862 POINT(40.52593480478663 74.34747807703154) bank120862 +120863 POINT(40.44508007319933 74.16214750665449) bank120863 +120864 POINT(40.05005825929781 74.37917435644142) bank120864 +120865 POINT(41.28466962528698 74.65851995834845) bank120865 +120866 POINT(40.05774865086204 73.47553332349285) bank120866 +120867 POINT(40.13621032878184 74.3175382059474) bank120867 +120868 POINT(39.941188498918976 74.55336367237133) bank120868 +120869 POINT(40.61624622193273 73.68230099550999) bank120869 +120870 POINT(40.69082047342186 74.60139242941612) bank120870 +120871 POINT(41.17151210220943 73.962715959848) bank120871 +120872 POINT(40.37373554549127 73.251398566171) bank120872 +120873 POINT(40.79169495946449 74.83161033260471) bank120873 +120874 POINT(40.437568324801305 74.10855034557585) bank120874 +120875 POINT(39.97540430634904 74.58319153180415) bank120875 +120876 POINT(41.270920070469764 74.12256403677078) bank120876 +120877 POINT(40.398956699131986 73.08465859762569) bank120877 +120878 POINT(41.10138863618687 73.62260036327648) bank120878 +120879 POINT(40.531268946557276 73.61296104564482) bank120879 +120880 POINT(41.2111109603792 73.11819615676293) bank120880 +120881 POINT(39.97785116078387 74.7498196028077) bank120881 +120882 POINT(40.06942563916073 73.99699097462039) bank120882 +120883 POINT(41.44197153673963 73.9228727208854) bank120883 +120884 POINT(39.73275619142815 73.85518369297536) bank120884 +120885 POINT(39.85009829122477 74.1199991732314) bank120885 +120886 POINT(40.079465835866266 73.58128710670891) bank120886 +120887 POINT(40.40154922510057 74.46000313034877) bank120887 +120888 POINT(40.14795541965064 74.12374501563716) bank120888 +120889 POINT(41.0689354683937 73.74576397735254) bank120889 +120890 POINT(41.04794787865713 73.78104395903212) bank120890 +120891 POINT(41.24950477933739 73.45451960323234) bank120891 +120892 POINT(41.422922355113904 74.53421365354997) bank120892 +120893 POINT(40.427276387567154 74.96853151409175) bank120893 +120894 POINT(40.21228928492888 74.56109671535697) bank120894 +120895 POINT(40.71638562043281 74.76227782734745) bank120895 +120896 POINT(41.67804379862862 73.53603219391407) bank120896 +120897 POINT(39.87673582424988 74.5437098086138) bank120897 +120898 POINT(40.852722861545644 73.14947784439072) bank120898 +120899 POINT(41.027004584663736 74.69624079708734) bank120899 +120900 POINT(40.780731821033534 73.10905576891709) bank120900 +120901 POINT(41.36557014998521 73.70392985725498) bank120901 +120902 POINT(40.61684362512918 73.48140552475628) bank120902 +120903 POINT(41.612028591923966 74.11634510382821) bank120903 +120904 POINT(41.3332840940437 74.62335862299908) bank120904 +120905 POINT(41.643548900380644 74.39806330154498) bank120905 +120906 POINT(41.561798517618726 74.19675908833747) bank120906 +120907 POINT(40.07981452163078 73.87537402721466) bank120907 +120908 POINT(40.85671874628164 73.6979815962772) bank120908 +120909 POINT(41.43068097143291 73.72694158883492) bank120909 +120910 POINT(41.52391121161651 74.7253993846314) bank120910 +120911 POINT(41.425962671327 73.46170075825754) bank120911 +120912 POINT(41.430527441876684 73.67598517036998) bank120912 +120913 POINT(40.02003502678156 73.88817406452705) bank120913 +120914 POINT(41.21274142851913 73.62235199287362) bank120914 +120915 POINT(40.276848411769066 74.38652052422513) bank120915 +120916 POINT(39.93945871162431 74.3515809684195) bank120916 +120917 POINT(41.426991917639214 73.21315627752986) bank120917 +120918 POINT(40.66619816406944 73.45795753610923) bank120918 +120919 POINT(40.19550714371085 74.77057667319473) bank120919 +120920 POINT(39.82691532725045 73.70487668231931) bank120920 +120921 POINT(39.929228938551304 73.03026019700934) bank120921 +120922 POINT(39.74456407670063 73.73430712944763) bank120922 +120923 POINT(40.21739819323038 74.23131112019713) bank120923 +120924 POINT(40.607590348855446 74.00471773682287) bank120924 +120925 POINT(39.86744716049843 74.2901682465597) bank120925 +120926 POINT(40.719246863104935 73.46788142642248) bank120926 +120927 POINT(39.98546676696049 73.34594906919396) bank120927 +120928 POINT(40.185705977177314 74.50727778087746) bank120928 +120929 POINT(41.60573624816438 73.85766202687634) bank120929 +120930 POINT(41.127892082317075 74.00172504304902) bank120930 +120931 POINT(40.035172633997675 74.46671564332551) bank120931 +120932 POINT(41.64597836804248 73.17397622106546) bank120932 +120933 POINT(40.18613248461879 74.3453398878952) bank120933 +120934 POINT(40.95746070739395 74.90559420470574) bank120934 +120935 POINT(41.5288230159794 73.61091237471048) bank120935 +120936 POINT(41.193817484488484 74.78367989211127) bank120936 +120937 POINT(40.656879797077714 74.94614158183305) bank120937 +120938 POINT(39.85933629437446 74.04875170763488) bank120938 +120939 POINT(40.608238792607075 73.25449560634374) bank120939 +120940 POINT(41.47868221021484 74.33352387828954) bank120940 +120941 POINT(41.065280570342416 74.14743976744295) bank120941 +120942 POINT(41.53163485746476 73.0493920556632) bank120942 +120943 POINT(40.162232514759076 73.77767093617851) bank120943 +120944 POINT(40.172123644418605 74.19385896734312) bank120944 +120945 POINT(40.77707286295763 73.87046164156764) bank120945 +120946 POINT(40.928422889613906 74.9703564918168) bank120946 +120947 POINT(40.2785943329208 73.9234027396035) bank120947 +120948 POINT(41.299800313062704 73.91998262845503) bank120948 +120949 POINT(39.8620349881508 74.34325582457116) bank120949 +120950 POINT(39.85512729251133 74.63822172787413) bank120950 +120951 POINT(41.26582699955493 74.29425587306402) bank120951 +120952 POINT(40.75384625888729 73.15904684005406) bank120952 +120953 POINT(41.10088848328676 73.73805187367005) bank120953 +120954 POINT(40.634205967819646 74.92879011652377) bank120954 +120955 POINT(40.951035956243814 73.74719748136819) bank120955 +120956 POINT(39.749162496514465 73.07730063927535) bank120956 +120957 POINT(40.712153216141026 74.87476627723386) bank120957 +120958 POINT(41.70835898339279 74.64676063809965) bank120958 +120959 POINT(41.053454128856615 73.13063393565729) bank120959 +120960 POINT(41.64308455231138 74.18687625945202) bank120960 +120961 POINT(40.668321781844135 74.16755227945883) bank120961 +120962 POINT(40.454108088728766 74.99427239731634) bank120962 +120963 POINT(41.44202538494276 73.13961420330435) bank120963 +120964 POINT(41.00466736215571 73.85141899446882) bank120964 +120965 POINT(41.27278526501034 74.6150989551832) bank120965 +120966 POINT(39.91558756109023 73.56208739786797) bank120966 +120967 POINT(40.01777375464049 73.13815131801557) bank120967 +120968 POINT(41.25150122423007 74.40307860332275) bank120968 +120969 POINT(41.174977406248566 73.1899279964639) bank120969 +120970 POINT(40.07675588298369 74.84968313341538) bank120970 +120971 POINT(40.5837815055269 73.90325030223323) bank120971 +120972 POINT(41.17210540068989 73.68046308530968) bank120972 +120973 POINT(41.136703554937654 73.35345526851897) bank120973 +120974 POINT(41.513601917106726 74.30945370557262) bank120974 +120975 POINT(41.09458158682701 74.20276660997584) bank120975 +120976 POINT(41.02209363256688 74.58071300491373) bank120976 +120977 POINT(39.92123015339254 74.80864427701393) bank120977 +120978 POINT(40.38629842796445 74.0332915324747) bank120978 +120979 POINT(40.10764614019739 73.49146858552807) bank120979 +120980 POINT(41.13223821182115 74.7050633382383) bank120980 +120981 POINT(40.57609467439026 73.27319240791348) bank120981 +120982 POINT(40.41766490920486 74.03404058930438) bank120982 +120983 POINT(40.844304446822775 74.01633999631308) bank120983 +120984 POINT(40.61001459226528 74.81463985341752) bank120984 +120985 POINT(40.56340202079573 74.82280591735453) bank120985 +120986 POINT(41.355606853047185 73.73825799937819) bank120986 +120987 POINT(40.43470329082266 73.53947121825681) bank120987 +120988 POINT(41.515807749714874 73.67068259195052) bank120988 +120989 POINT(40.04121250829889 73.34188335882395) bank120989 +120990 POINT(40.90597385315597 73.53422603921537) bank120990 +120991 POINT(41.0936706735115 74.76866238212853) bank120991 +120992 POINT(39.74345545078504 74.55606091848372) bank120992 +120993 POINT(41.62152853873674 73.76471705666441) bank120993 +120994 POINT(40.20806867388598 73.22484203496107) bank120994 +120995 POINT(41.35526433920408 73.66352223092706) bank120995 +120996 POINT(39.772474570628034 73.1295229989786) bank120996 +120997 POINT(41.11099358226956 74.13866573724529) bank120997 +120998 POINT(39.93141638026521 74.32195471342443) bank120998 +120999 POINT(40.078250784970045 74.55087505563968) bank120999 +121000 POINT(40.825463059418205 74.16409645595668) bank121000 +121001 POINT(39.923872474303984 73.61151528111752) bank121001 +121002 POINT(39.98802224678259 73.14789641267785) bank121002 +121003 POINT(40.2516812514387 74.59447201462878) bank121003 +121004 POINT(40.46623240888696 74.41463188623378) bank121004 +121005 POINT(40.76541816726652 74.53075110304228) bank121005 +121006 POINT(40.23268676543527 74.44009794275713) bank121006 +121007 POINT(40.83772859940694 73.46264476985104) bank121007 +121008 POINT(41.33474174091188 74.42757859644883) bank121008 +121009 POINT(41.52649799273328 73.3501967320884) bank121009 +121010 POINT(41.166089269279006 74.14593647279757) bank121010 +121011 POINT(41.45378789031216 73.66121572859154) bank121011 +121012 POINT(41.55106671966385 74.36232425715842) bank121012 +121013 POINT(40.107172986445335 73.13149570795284) bank121013 +121014 POINT(41.54814305139826 74.32938495731445) bank121014 +121015 POINT(41.68505781372617 74.81638373682648) bank121015 +121016 POINT(40.71271759211812 73.18112784591419) bank121016 +121017 POINT(41.32459259276044 74.04740658314155) bank121017 +121018 POINT(41.18324472978951 73.39670854321336) bank121018 +121019 POINT(39.72078446672386 74.9430131878903) bank121019 +121020 POINT(40.869672682668394 74.45453061424794) bank121020 +121021 POINT(40.219456910885484 73.34711932647642) bank121021 +121022 POINT(40.320871333245414 73.3042669240926) bank121022 +121023 POINT(40.84155888015383 74.93126395444308) bank121023 +121024 POINT(40.41788648996696 73.36170306433394) bank121024 +121025 POINT(39.843544266051175 74.7183477712194) bank121025 +121026 POINT(40.6691691246021 73.57227411658396) bank121026 +121027 POINT(40.40257433100224 73.42218974114147) bank121027 +121028 POINT(41.47637791082709 74.81777036528855) bank121028 +121029 POINT(40.27691984867935 74.59407719037554) bank121029 +121030 POINT(41.60255819635632 74.91932353513123) bank121030 +121031 POINT(39.80053041549184 74.04940474501724) bank121031 +121032 POINT(41.15981210485093 74.29983236088663) bank121032 +121033 POINT(41.302657913449465 74.58564119483128) bank121033 +121034 POINT(40.84342152414245 74.44796239011704) bank121034 +121035 POINT(40.22306302407663 74.10880341517797) bank121035 +121036 POINT(41.0899693215417 74.3624255572286) bank121036 +121037 POINT(41.65541700995011 73.16358972399973) bank121037 +121038 POINT(40.10940579059221 73.16763471867958) bank121038 +121039 POINT(41.22991291457948 73.79433404104515) bank121039 +121040 POINT(41.60986855188955 73.11634859916778) bank121040 +121041 POINT(40.46396693844255 73.46912553764024) bank121041 +121042 POINT(40.11867273194058 73.28299413819836) bank121042 +121043 POINT(40.73550945934014 74.82891042110985) bank121043 +121044 POINT(41.20846867609234 73.86224376875617) bank121044 +121045 POINT(40.78582434051336 73.72264663397507) bank121045 +121046 POINT(39.719451791923795 73.16696090310747) bank121046 +121047 POINT(40.57807152954072 74.25495675880585) bank121047 +121048 POINT(40.55268786511215 74.35202884493154) bank121048 +121049 POINT(40.559637774892046 73.16173498469749) bank121049 +121050 POINT(40.68013456447448 73.22885852133093) bank121050 +121051 POINT(40.24084306189051 73.8883142419854) bank121051 +121052 POINT(41.1985528701187 73.56397828386896) bank121052 +121053 POINT(41.03609472879968 73.05310135528586) bank121053 +121054 POINT(40.9265530794362 74.97345865853858) bank121054 +121055 POINT(41.24713111508791 73.3530193210811) bank121055 +121056 POINT(40.68223698454188 74.55525484866712) bank121056 +121057 POINT(40.69351568171558 74.46288720432572) bank121057 +121058 POINT(41.406961874213046 74.16350260032078) bank121058 +121059 POINT(41.17262098595514 73.5640220745096) bank121059 +121060 POINT(40.40153120777584 73.22968873194796) bank121060 +121061 POINT(40.52963953588885 73.07944176244757) bank121061 +121062 POINT(40.30246053839691 73.74657692310527) bank121062 +121063 POINT(41.10662744051262 74.16235466076358) bank121063 +121064 POINT(40.22443372683072 74.50577495675212) bank121064 +121065 POINT(40.35685107207114 74.2935651433884) bank121065 +121066 POINT(41.355936889852956 73.77672795051045) bank121066 +121067 POINT(41.09588177948053 74.90156585511599) bank121067 +121068 POINT(40.12694465225844 74.7146960939927) bank121068 +121069 POINT(40.80178263130314 73.06439518242362) bank121069 +121070 POINT(39.991485167716185 73.83838848112582) bank121070 +121071 POINT(41.3134792510742 73.64476610541647) bank121071 +121072 POINT(40.20877784240034 74.27353236169131) bank121072 +121073 POINT(40.64462527019206 74.48326669657395) bank121073 +121074 POINT(41.399790832485465 73.8321089971124) bank121074 +121075 POINT(40.424022613079586 74.64755186640356) bank121075 +121076 POINT(41.499273125356744 73.60418602245423) bank121076 +121077 POINT(41.10215037448856 73.85697138614192) bank121077 +121078 POINT(41.379868559264544 74.48736217990458) bank121078 +121079 POINT(41.30822857704749 73.1959407958416) bank121079 +121080 POINT(40.79599592796898 73.58475284125412) bank121080 +121081 POINT(40.5544213237508 73.99163336427429) bank121081 +121082 POINT(40.760222200367615 74.90985076338978) bank121082 +121083 POINT(40.8361816971416 73.07494948762263) bank121083 +121084 POINT(40.408802518254866 73.37404806982606) bank121084 +121085 POINT(40.680310837233 73.31765287846684) bank121085 +121086 POINT(40.52467363930503 73.90525149151537) bank121086 +121087 POINT(40.43106364549098 73.75419742916388) bank121087 +121088 POINT(40.0766340009177 73.45612263003852) bank121088 +121089 POINT(40.722974144497734 74.02271430885777) bank121089 +121090 POINT(41.423240279735175 74.4371101887788) bank121090 +121091 POINT(41.65846013538418 73.83533575282625) bank121091 +121092 POINT(41.171743891105386 74.08913447325041) bank121092 +121093 POINT(41.45199908191724 73.38708495188973) bank121093 +121094 POINT(41.213763176146905 73.9350178349492) bank121094 +121095 POINT(41.266097664550536 74.7719473864438) bank121095 +121096 POINT(39.86618051844981 73.48877643831028) bank121096 +121097 POINT(40.68212864481306 73.87240445285748) bank121097 +121098 POINT(40.03237852255597 73.14763292526189) bank121098 +121099 POINT(40.51349004315108 74.66552050169967) bank121099 +121100 POINT(41.14099661813713 73.24922059068238) bank121100 +121101 POINT(40.66517427435698 73.86192522230367) bank121101 +121102 POINT(40.264879091901314 74.60417884930266) bank121102 +121103 POINT(40.34536262175011 74.55722109900908) bank121103 +121104 POINT(41.31133853218929 73.33019819086799) bank121104 +121105 POINT(40.68128591449179 74.1841186517061) bank121105 +121106 POINT(40.23270832275722 73.07354547503164) bank121106 +121107 POINT(41.44003052660771 74.79504184056087) bank121107 +121108 POINT(40.64291897965047 74.30351112309833) bank121108 +121109 POINT(41.06275282250804 74.0758637447765) bank121109 +121110 POINT(41.303304188756506 74.98326398298789) bank121110 +121111 POINT(41.22655052818759 74.74002734563493) bank121111 +121112 POINT(40.69307971933593 73.79424513313329) bank121112 +121113 POINT(41.32052174264976 73.74760097123955) bank121113 +121114 POINT(41.63192625976876 74.40039071897996) bank121114 +121115 POINT(40.43349188023623 74.53287616610041) bank121115 +121116 POINT(39.999353772467046 74.89948790325961) bank121116 +121117 POINT(40.329099780487105 73.54929641822505) bank121117 +121118 POINT(41.41380852622739 74.80504587552011) bank121118 +121119 POINT(41.63349491494126 73.26805334569414) bank121119 +121120 POINT(40.75829635033086 73.48068588178812) bank121120 +121121 POINT(40.45853649117821 73.24502854038909) bank121121 +121122 POINT(40.13730431279346 73.19478024371024) bank121122 +121123 POINT(40.39675305945759 74.36339207587702) bank121123 +121124 POINT(40.04120042327341 73.97411522789686) bank121124 +121125 POINT(40.74604944983137 74.41259213489697) bank121125 +121126 POINT(40.04681998128127 74.90396798119792) bank121126 +121127 POINT(41.58454418486268 74.65055301674994) bank121127 +121128 POINT(40.33766692174528 74.56616579331944) bank121128 +121129 POINT(41.17112063038013 74.25235915298258) bank121129 +121130 POINT(41.10495073757681 73.95800272839307) bank121130 +121131 POINT(41.34910975893088 74.26943342631684) bank121131 +121132 POINT(39.89337369800917 74.87533239229508) bank121132 +121133 POINT(39.97720662191626 74.05367854038369) bank121133 +121134 POINT(40.47321702556744 74.61749272864328) bank121134 +121135 POINT(40.11983669614806 73.01668233195481) bank121135 +121136 POINT(41.02578406885218 73.35404359466494) bank121136 +121137 POINT(40.35044896502832 73.60023870208025) bank121137 +121138 POINT(39.811580006873996 74.21930469039894) bank121138 +121139 POINT(41.670362445778686 73.82248670276907) bank121139 +121140 POINT(40.078315330843466 74.78379584748585) bank121140 +121141 POINT(41.33813492248502 74.29992945511242) bank121141 +121142 POINT(41.39709456239909 74.56446064763027) bank121142 +121143 POINT(40.29621268724637 73.12811836531097) bank121143 +121144 POINT(41.11209867569119 73.96295684940527) bank121144 +121145 POINT(41.16597448908282 74.86126569658148) bank121145 +121146 POINT(41.6653208362992 74.32452673115702) bank121146 +121147 POINT(39.80767132541615 74.6736698385591) bank121147 +121148 POINT(40.79584217221725 73.95596889884868) bank121148 +121149 POINT(39.97176024338556 73.03572161288817) bank121149 +121150 POINT(40.78592681360173 74.51441722704013) bank121150 +121151 POINT(40.772141165401635 73.16658182392028) bank121151 +121152 POINT(41.17113783524832 74.75629024140359) bank121152 +121153 POINT(40.76746477612318 74.37935335983524) bank121153 +121154 POINT(41.17427122350933 73.06622716611422) bank121154 +121155 POINT(40.226559216286596 73.57483573914992) bank121155 +121156 POINT(40.19031937529221 74.69749976957903) bank121156 +121157 POINT(40.80448097564604 73.36229897769418) bank121157 +121158 POINT(40.27923257019467 73.11301782780104) bank121158 +121159 POINT(40.52418698074659 73.54927882379684) bank121159 +121160 POINT(41.25469498263078 73.85219359089537) bank121160 +121161 POINT(41.486208099096196 74.97014278358687) bank121161 +121162 POINT(40.34085828701499 74.02238397492806) bank121162 +121163 POINT(40.7079102958588 74.71091698304663) bank121163 +121164 POINT(40.3995174217713 74.22433032794896) bank121164 +121165 POINT(41.316582896618094 74.34927962158122) bank121165 +121166 POINT(39.8588895582745 73.80760124076144) bank121166 +121167 POINT(40.707928525740876 73.6330910835707) bank121167 +121168 POINT(41.537399612161344 73.61060586839295) bank121168 +121169 POINT(39.8492145974508 74.77598666474469) bank121169 +121170 POINT(41.44489718053138 74.49042973353752) bank121170 +121171 POINT(40.52144643949915 73.5136687270069) bank121171 +121172 POINT(41.585944693075284 73.45389406842824) bank121172 +121173 POINT(40.24448812601544 74.71844841877058) bank121173 +121174 POINT(39.9519330062485 74.9438241494108) bank121174 +121175 POINT(41.298756043782845 74.78420728527276) bank121175 +121176 POINT(40.67753372887255 74.03656092575599) bank121176 +121177 POINT(40.27912216881883 73.83770973754282) bank121177 +121178 POINT(40.747561768491344 73.83501238884601) bank121178 +121179 POINT(40.04696258344599 73.88472374930907) bank121179 +121180 POINT(40.01978094398985 74.8713037153278) bank121180 +121181 POINT(39.849839474274134 73.62528126793443) bank121181 +121182 POINT(40.53292440680282 73.21686510099752) bank121182 +121183 POINT(41.46867908622858 74.83889102132328) bank121183 +121184 POINT(40.66119171566827 73.07967352989539) bank121184 +121185 POINT(40.45927198657907 73.09665159787309) bank121185 +121186 POINT(40.48884621144279 73.965086887085) bank121186 +121187 POINT(41.121080400114785 73.91619198400976) bank121187 +121188 POINT(40.66980857487973 73.29464789562152) bank121188 +121189 POINT(40.47348532130028 74.20057795406198) bank121189 +121190 POINT(39.987341169817164 73.86002337496254) bank121190 +121191 POINT(40.86004495867584 73.74669980136845) bank121191 +121192 POINT(40.838046214392286 74.19477738381366) bank121192 +121193 POINT(39.95240855468974 73.05503873427824) bank121193 +121194 POINT(40.63248222627871 74.8099644163387) bank121194 +121195 POINT(40.49797597976233 73.7258542362384) bank121195 +121196 POINT(39.99842994867761 74.17591729356029) bank121196 +121197 POINT(40.30120584996851 73.96209075466594) bank121197 +121198 POINT(40.614248772765755 74.65657348925502) bank121198 +121199 POINT(40.63616482615469 74.7965591176518) bank121199 +121200 POINT(39.768256435621204 73.08941615092068) bank121200 +121201 POINT(40.27737889479762 74.08742509113557) bank121201 +121202 POINT(40.988101707098544 74.28139963012158) bank121202 +121203 POINT(41.08698122620304 74.1984509441331) bank121203 +121204 POINT(41.24038438532646 73.69656597768392) bank121204 +121205 POINT(41.49015301794988 74.90011867620031) bank121205 +121206 POINT(40.57547993583419 74.64609918044748) bank121206 +121207 POINT(40.995788813726605 74.99871630271647) bank121207 +121208 POINT(40.07575602738211 73.8990883801236) bank121208 +121209 POINT(39.9558732040394 74.54135987044499) bank121209 +121210 POINT(39.799747198286376 74.92746145494539) bank121210 +121211 POINT(40.35516154094647 73.77332195950898) bank121211 +121212 POINT(40.315839522659076 74.64840620273986) bank121212 +121213 POINT(39.95690188439056 74.70275726939096) bank121213 +121214 POINT(40.363315165002994 73.47169871541793) bank121214 +121215 POINT(41.51354229244954 73.44402819978812) bank121215 +121216 POINT(39.97957088607877 74.0531891564614) bank121216 +121217 POINT(40.777693382908026 74.242421431829) bank121217 +121218 POINT(40.478576648247405 73.77039756013986) bank121218 +121219 POINT(39.836322136402124 73.41467241971714) bank121219 +121220 POINT(41.51737517379833 73.37503138728596) bank121220 +121221 POINT(41.25269883510682 73.374391503907) bank121221 +121222 POINT(41.14412024725406 73.12244820826758) bank121222 +121223 POINT(40.58309739464377 74.92327230493274) bank121223 +121224 POINT(39.85466237626709 73.50014665558551) bank121224 +121225 POINT(40.26540195140303 74.06437742137696) bank121225 +121226 POINT(39.790818633385925 73.69666988895803) bank121226 +121227 POINT(41.38472400590002 74.60262613880795) bank121227 +121228 POINT(39.82394180006232 74.7194091870511) bank121228 +121229 POINT(40.04678509968631 73.10756774906147) bank121229 +121230 POINT(40.254169491398585 73.77591182127718) bank121230 +121231 POINT(41.328420013509486 74.73119495305566) bank121231 +121232 POINT(41.190272911974375 74.29866326040234) bank121232 +121233 POINT(39.80039322200691 73.2387667436693) bank121233 +121234 POINT(41.649008932507364 73.32010625308075) bank121234 +121235 POINT(41.632012070127104 73.70393660562797) bank121235 +121236 POINT(40.611416318844974 73.48405197784135) bank121236 +121237 POINT(39.79695464201724 74.12046331489492) bank121237 +121238 POINT(41.58264155612971 73.30050265700228) bank121238 +121239 POINT(41.114150433229696 74.57410017370798) bank121239 +121240 POINT(40.94761714217809 74.73571832098818) bank121240 +121241 POINT(39.80311711918943 74.92363283059076) bank121241 +121242 POINT(40.336659335804306 74.41337539172287) bank121242 +121243 POINT(40.75390292348871 73.99097257696003) bank121243 +121244 POINT(41.043468325078216 74.89179118157053) bank121244 +121245 POINT(40.66762047339008 73.34303130240572) bank121245 +121246 POINT(41.19770632678863 74.04944657087738) bank121246 +121247 POINT(41.08223809952496 73.35721618856111) bank121247 +121248 POINT(40.06108556894322 74.66626387086501) bank121248 +121249 POINT(41.099008720160256 74.8401575231434) bank121249 +121250 POINT(39.78813326503624 73.65865788163839) bank121250 +121251 POINT(41.49627231834642 73.96453693471123) bank121251 +121252 POINT(40.988337829045264 74.40441098905359) bank121252 +121253 POINT(40.69836078644129 73.67596043976909) bank121253 +121254 POINT(40.47363569614659 74.27062274835411) bank121254 +121255 POINT(41.67842585727021 74.92343403675348) bank121255 +121256 POINT(40.95601275599768 74.64946101422794) bank121256 +121257 POINT(40.5444319500737 74.56322302917918) bank121257 +121258 POINT(40.09948153803782 73.6937372609084) bank121258 +121259 POINT(41.09656882420865 74.08540824118883) bank121259 +121260 POINT(40.01763317818403 74.96531371385622) bank121260 +121261 POINT(40.42347935300995 74.33338056223958) bank121261 +121262 POINT(41.0011078796217 74.18697127959128) bank121262 +121263 POINT(41.04917783091013 74.28267872058422) bank121263 +121264 POINT(41.68478625324001 73.0875789731896) bank121264 +121265 POINT(40.631846935222335 74.43196174916916) bank121265 +121266 POINT(41.67578851135036 74.72906541022346) bank121266 +121267 POINT(40.922062924300896 73.27593969505848) bank121267 +121268 POINT(40.58618454983678 73.12933587392476) bank121268 +121269 POINT(40.2540948958501 74.35470094989117) bank121269 +121270 POINT(40.76739642086551 74.1589394181025) bank121270 +121271 POINT(41.19574187558605 74.71549748043864) bank121271 +121272 POINT(40.42827556473892 74.3798742145833) bank121272 +121273 POINT(41.229828243344684 73.24153956452052) bank121273 +121274 POINT(40.2815906911119 73.90539707783152) bank121274 +121275 POINT(41.528345156459324 74.99376044242577) bank121275 +121276 POINT(41.394577102606235 74.26718619108193) bank121276 +121277 POINT(41.38901042762542 74.67413798652011) bank121277 +121278 POINT(40.36836776251559 73.39872825713148) bank121278 +121279 POINT(41.050294299057846 73.52505211204316) bank121279 +121280 POINT(40.80217827974867 73.31192804391134) bank121280 +121281 POINT(41.346820984613665 73.53052874486447) bank121281 +121282 POINT(40.99513772368256 74.05510782576728) bank121282 +121283 POINT(40.02504194100735 74.29503651838695) bank121283 +121284 POINT(39.88823038289386 74.82363006218719) bank121284 +121285 POINT(40.102846818332154 74.244970403934) bank121285 +121286 POINT(41.16320315259257 74.93424293442338) bank121286 +121287 POINT(41.32953550604605 73.5653111959531) bank121287 +121288 POINT(41.40755055040578 73.39840637765586) bank121288 +121289 POINT(41.04250436886068 74.16543408714358) bank121289 +121290 POINT(39.76784662055578 74.10004755070699) bank121290 +121291 POINT(41.376693252128945 74.70104742197735) bank121291 +121292 POINT(40.40958964376439 73.4222154095562) bank121292 +121293 POINT(40.55811731075962 73.62600948787585) bank121293 +121294 POINT(40.7720944524014 74.76510698365327) bank121294 +121295 POINT(41.07612038454571 73.59828152905365) bank121295 +121296 POINT(41.14834661134682 73.8844838477587) bank121296 +121297 POINT(40.385189826641096 73.80149263557162) bank121297 +121298 POINT(39.98808510359503 74.90131934482424) bank121298 +121299 POINT(40.0012313363736 74.74910275211809) bank121299 +121300 POINT(40.167079452598436 74.42072714844505) bank121300 +121301 POINT(40.04386960038905 73.56762727019549) bank121301 +121302 POINT(39.79591068309652 73.72635826585457) bank121302 +121303 POINT(40.251904830505 74.89293756589534) bank121303 +121304 POINT(41.47871502442446 73.76584317369988) bank121304 +121305 POINT(40.60744401979568 74.35094679326836) bank121305 +121306 POINT(40.88144255842512 74.13763768732315) bank121306 +121307 POINT(40.85355957456028 73.87426210455178) bank121307 +121308 POINT(40.8505922383796 74.7868253533918) bank121308 +121309 POINT(40.625685250437165 73.67725662136075) bank121309 +121310 POINT(41.342944667961774 74.67482332267927) bank121310 +121311 POINT(40.26335231867847 73.84942519184648) bank121311 +121312 POINT(41.455865076074396 74.70257563810875) bank121312 +121313 POINT(41.186744334852406 73.98156825313843) bank121313 +121314 POINT(39.73001289272955 74.1212481994131) bank121314 +121315 POINT(41.339529871733205 74.75588911906013) bank121315 +121316 POINT(41.70038181159964 73.38557112578576) bank121316 +121317 POINT(40.59531667831833 74.97463100228022) bank121317 +121318 POINT(40.99899167510099 74.4785023901908) bank121318 +121319 POINT(39.98832071467956 73.68925735721815) bank121319 +121320 POINT(39.827483416691486 74.34645385587818) bank121320 +121321 POINT(40.18094803962647 74.84237188884977) bank121321 +121322 POINT(41.62353913793613 74.67903641673767) bank121322 +121323 POINT(40.43019328188974 74.95128782495509) bank121323 +121324 POINT(40.57571617878243 73.57235566184399) bank121324 +121325 POINT(41.169311475894524 74.98983159465952) bank121325 +121326 POINT(40.16356323680227 73.07769667230346) bank121326 +121327 POINT(39.883905928070284 74.44481032189323) bank121327 +121328 POINT(40.80846863415217 73.04601950249183) bank121328 +121329 POINT(40.8947394604199 73.17682652478031) bank121329 +121330 POINT(40.50931834730177 74.34121561252994) bank121330 +121331 POINT(41.197750735904286 73.99605522431332) bank121331 +121332 POINT(40.6427798671279 74.24070932980509) bank121332 +121333 POINT(39.810747520612765 73.75969742817105) bank121333 +121334 POINT(40.586076367279354 73.65603492255157) bank121334 +121335 POINT(40.10276678561571 75.00218672987477) bank121335 +121336 POINT(40.858685292443504 73.77738971495837) bank121336 +121337 POINT(39.844028825551035 74.12050833615793) bank121337 +121338 POINT(40.827139164338114 73.57663533877397) bank121338 +121339 POINT(39.83171639364533 73.88770322538345) bank121339 +121340 POINT(41.049698434548 74.28442771241036) bank121340 +121341 POINT(39.79869990768226 74.6127174073041) bank121341 +121342 POINT(41.08852641954321 73.15136209185916) bank121342 +121343 POINT(40.48322023614766 74.3550790406399) bank121343 +121344 POINT(41.32056649749545 73.32645958366584) bank121344 +121345 POINT(40.91553652704515 74.02665633749751) bank121345 +121346 POINT(41.39073023555307 74.92733814700226) bank121346 +121347 POINT(40.4920484778643 74.9194274388893) bank121347 +121348 POINT(40.21941173639198 73.47979269107508) bank121348 +121349 POINT(39.913790525803186 73.93554195313276) bank121349 +121350 POINT(41.70097106288397 73.23869546148693) bank121350 +121351 POINT(39.87624796162762 74.71144956112842) bank121351 +121352 POINT(41.44364346227258 73.94990486139044) bank121352 +121353 POINT(41.24465198958698 74.74614512978886) bank121353 +121354 POINT(40.103644448120846 74.69150751281332) bank121354 +121355 POINT(41.39772087412685 73.61178515549688) bank121355 +121356 POINT(41.01138173384539 73.08883512546326) bank121356 +121357 POINT(40.12143174312824 74.13077745622306) bank121357 +121358 POINT(40.6222963215182 73.7047014304625) bank121358 +121359 POINT(41.14325216994501 74.31139347587786) bank121359 +121360 POINT(40.52023930153828 73.94157970434378) bank121360 +121361 POINT(41.4567064146517 73.61837835958052) bank121361 +121362 POINT(40.82484464951608 73.14604737962222) bank121362 +121363 POINT(39.95858569885143 73.49112500470093) bank121363 +121364 POINT(41.031138574032916 73.2693381695276) bank121364 +121365 POINT(41.35263772856704 73.3436065741064) bank121365 +121366 POINT(39.92244609310118 74.19784160360068) bank121366 +121367 POINT(41.27829015145297 73.64365622779853) bank121367 +121368 POINT(40.52559583590687 74.34823096457517) bank121368 +121369 POINT(41.46224284421542 73.14238126169667) bank121369 +121370 POINT(40.22830533964674 73.11290175663865) bank121370 +121371 POINT(41.488909093377444 73.48097863470343) bank121371 +121372 POINT(40.96374920617188 73.44094376317913) bank121372 +121373 POINT(41.05193237215823 73.14896567751077) bank121373 +121374 POINT(39.83319778517723 73.31154880696387) bank121374 +121375 POINT(41.17766231841257 73.14703326743803) bank121375 +121376 POINT(40.3726691183881 73.29577244199459) bank121376 +121377 POINT(40.69664830923427 74.01664652027964) bank121377 +121378 POINT(39.94231402654616 73.75061911971326) bank121378 +121379 POINT(40.08292876830058 73.72829746737833) bank121379 +121380 POINT(40.515317497276634 73.40792867678346) bank121380 +121381 POINT(41.59781852199874 74.69922359434003) bank121381 +121382 POINT(41.04357773753327 74.40463774117468) bank121382 +121383 POINT(39.96533306301051 74.4804516849327) bank121383 +121384 POINT(40.94868205061113 73.55739114889535) bank121384 +121385 POINT(41.288302341700515 73.77497085842468) bank121385 +121386 POINT(40.72246283147311 74.55140839777799) bank121386 +121387 POINT(41.09150796341044 73.31678953356904) bank121387 +121388 POINT(40.26311385915227 73.27582582201347) bank121388 +121389 POINT(40.28280622309528 73.44443507152873) bank121389 +121390 POINT(39.75535343988638 73.50632592859135) bank121390 +121391 POINT(39.90860210502669 74.18391758125455) bank121391 +121392 POINT(41.53405602720818 74.25536760811312) bank121392 +121393 POINT(41.272688043548726 74.26197757650363) bank121393 +121394 POINT(40.190779584740945 73.8960853604399) bank121394 +121395 POINT(41.198142471763 73.96888005661022) bank121395 +121396 POINT(40.99693035849224 73.24969836324955) bank121396 +121397 POINT(40.37335538826676 74.92805742841709) bank121397 +121398 POINT(41.61936241307941 74.09836046450881) bank121398 +121399 POINT(40.591059089216444 74.08892518648828) bank121399 +121400 POINT(40.17630842542905 73.5908619668672) bank121400 +121401 POINT(40.4144206770985 73.22019117925987) bank121401 +121402 POINT(41.365190954789234 73.41471257732) bank121402 +121403 POINT(40.17679814278226 73.72879327415865) bank121403 +121404 POINT(39.715292738299524 74.05456563087596) bank121404 +121405 POINT(40.02545052075465 73.27284809527315) bank121405 +121406 POINT(41.64915296136896 73.6625804462958) bank121406 +121407 POINT(41.07819183119529 74.99175865021584) bank121407 +121408 POINT(40.53743025253351 73.6847965465223) bank121408 +121409 POINT(40.21133910427638 74.45685911296638) bank121409 +121410 POINT(39.89017264531962 74.89584398459117) bank121410 +121411 POINT(40.00845122075427 74.32130458268529) bank121411 +121412 POINT(40.691331788378406 73.27936298322543) bank121412 +121413 POINT(41.24948235622902 74.7250588809276) bank121413 +121414 POINT(40.41026271186467 74.14840745824095) bank121414 +121415 POINT(40.936190905968445 74.2209977827994) bank121415 +121416 POINT(41.41625417168989 74.77138876276757) bank121416 +121417 POINT(40.43535471468171 73.22362638012073) bank121417 +121418 POINT(40.95065341400721 73.9930487576427) bank121418 +121419 POINT(40.81315548929463 73.11492954202315) bank121419 +121420 POINT(40.887932636606784 73.50694663493734) bank121420 +121421 POINT(40.63356108075737 74.86025739408966) bank121421 +121422 POINT(41.20764077091491 74.64349494400324) bank121422 +121423 POINT(40.50692083648915 73.55241028090872) bank121423 +121424 POINT(40.2927446417822 73.6948430294103) bank121424 +121425 POINT(40.83163852527241 74.6677731795157) bank121425 +121426 POINT(41.24236947789725 74.16389894118058) bank121426 +121427 POINT(39.77455437539324 73.67140446355478) bank121427 +121428 POINT(40.268866866460954 73.20544662096161) bank121428 +121429 POINT(40.28466178706189 73.11259359137175) bank121429 +121430 POINT(40.33536204911859 74.41240095041161) bank121430 +121431 POINT(40.24856278406632 73.87305686806495) bank121431 +121432 POINT(39.86119645468713 73.35040645957164) bank121432 +121433 POINT(39.78617618066512 73.33793945831876) bank121433 +121434 POINT(40.31893170009769 73.89717823259052) bank121434 +121435 POINT(40.01381336561698 73.58275879735729) bank121435 +121436 POINT(41.56772919705259 73.76402559430271) bank121436 +121437 POINT(40.45408820595467 74.0303287575947) bank121437 +121438 POINT(40.68558587971222 74.39603400082245) bank121438 +121439 POINT(40.606008102730215 74.04837712400732) bank121439 +121440 POINT(40.0340938980393 74.72983640915874) bank121440 +121441 POINT(41.493710073479264 73.44332822443779) bank121441 +121442 POINT(40.65671914472051 73.3513537644503) bank121442 +121443 POINT(40.782732671922226 73.23367671588812) bank121443 +121444 POINT(41.28460853071106 73.91387315460497) bank121444 +121445 POINT(41.612239156155034 74.00963105232886) bank121445 +121446 POINT(40.1325256902221 74.1878551862966) bank121446 +121447 POINT(40.039954983779076 73.73818394081444) bank121447 +121448 POINT(41.68308405895697 73.53569179532299) bank121448 +121449 POINT(41.68690616046414 73.32015646426373) bank121449 +121450 POINT(39.72138440513026 74.90724442127663) bank121450 +121451 POINT(39.905045267981464 74.31922265864372) bank121451 +121452 POINT(40.16460272627962 73.93980109182644) bank121452 +121453 POINT(39.86850577065951 74.92797586620529) bank121453 +121454 POINT(40.42040480873455 73.00898541895957) bank121454 +121455 POINT(41.51620151888877 73.44157493205076) bank121455 +121456 POINT(41.266570081052045 74.75491531733887) bank121456 +121457 POINT(40.172583286165164 73.15739469523734) bank121457 +121458 POINT(40.18507748504477 74.21861409520595) bank121458 +121459 POINT(40.19962855150801 74.07876911233686) bank121459 +121460 POINT(41.21458117092377 74.06688806477379) bank121460 +121461 POINT(41.26306469175753 73.32479375453296) bank121461 +121462 POINT(41.31646606964679 74.84775870742834) bank121462 +121463 POINT(41.22138014624942 73.69388774019409) bank121463 +121464 POINT(39.7688543956003 73.03632684460214) bank121464 +121465 POINT(40.158703112796324 74.07718304158313) bank121465 +121466 POINT(41.51175812098826 74.84213759309011) bank121466 +121467 POINT(41.4647807377099 74.06036163916073) bank121467 +121468 POINT(41.09233928304958 74.35060944150004) bank121468 +121469 POINT(41.499176420391414 73.7278086635314) bank121469 +121470 POINT(39.85934960737379 74.62638542223479) bank121470 +121471 POINT(40.08255291323763 73.52596883929836) bank121471 +121472 POINT(41.518904447514984 73.82111496624125) bank121472 +121473 POINT(40.45563686130781 74.60980908823267) bank121473 +121474 POINT(41.502183937160275 74.55445074601563) bank121474 +121475 POINT(40.90128913370824 74.63394231482961) bank121475 +121476 POINT(39.73136308181941 74.7886457639531) bank121476 +121477 POINT(40.64617025244916 73.36229215965832) bank121477 +121478 POINT(41.096888882613435 73.92173676462495) bank121478 +121479 POINT(40.637821393452505 73.0593738729576) bank121479 +121480 POINT(41.22307541886293 73.36760570009079) bank121480 +121481 POINT(41.7117826839128 74.2725965876953) bank121481 +121482 POINT(40.4380547087198 74.48818082954755) bank121482 +121483 POINT(41.484108960288005 73.61046960010144) bank121483 +121484 POINT(39.999001855896594 73.11065005614869) bank121484 +121485 POINT(41.229908850295274 73.36362051366697) bank121485 +121486 POINT(40.53036889511255 73.05434736936151) bank121486 +121487 POINT(41.36203200894072 74.22956005112954) bank121487 +121488 POINT(40.59819731004888 73.64449616628583) bank121488 +121489 POINT(41.01132578471609 74.44844287484926) bank121489 +121490 POINT(41.31392389462466 74.21558444692388) bank121490 +121491 POINT(39.991378109609116 73.15190970130493) bank121491 +121492 POINT(40.023717350488546 73.92549015989988) bank121492 +121493 POINT(40.539654316101426 73.60048451361988) bank121493 +121494 POINT(40.92606416480555 74.83711279961841) bank121494 +121495 POINT(40.446425769588934 74.82869288307222) bank121495 +121496 POINT(41.15088070752184 73.36440068894062) bank121496 +121497 POINT(40.77822286229214 73.8763482124498) bank121497 +121498 POINT(40.379225990678904 74.16030906062203) bank121498 +121499 POINT(41.4406925209284 74.9606279807492) bank121499 +121500 POINT(40.41043490074039 73.58860825900673) bank121500 +121501 POINT(41.31392388401807 74.95586818305023) bank121501 +121502 POINT(40.7624105108988 74.88986978596797) bank121502 +121503 POINT(41.62606373759379 73.08916470267714) bank121503 +121504 POINT(41.010947344012166 74.83282499787818) bank121504 +121505 POINT(40.47204879634393 73.86668785606916) bank121505 +121506 POINT(41.51191391205355 73.35505828838237) bank121506 +121507 POINT(40.80730695809057 74.98630834374724) bank121507 +121508 POINT(41.671427322090665 73.51112668103546) bank121508 +121509 POINT(40.897096135498764 74.57418544422126) bank121509 +121510 POINT(41.69278398744398 74.96029508443364) bank121510 +121511 POINT(39.88426571161778 74.88319418797363) bank121511 +121512 POINT(41.1755522284983 73.7449834873902) bank121512 +121513 POINT(41.5224168649775 74.06516064095555) bank121513 +121514 POINT(41.70247614139013 73.62603175876215) bank121514 +121515 POINT(40.803770496896945 73.02943884978573) bank121515 +121516 POINT(40.97152077355594 73.72863638099425) bank121516 +121517 POINT(41.06239480708071 74.35852435737216) bank121517 +121518 POINT(41.33129334193137 73.4518031235456) bank121518 +121519 POINT(41.00369765896286 73.14883351569202) bank121519 +121520 POINT(40.70153715377133 73.13180971502386) bank121520 +121521 POINT(40.1699030609461 73.21022280361991) bank121521 +121522 POINT(39.78538067507161 74.7590641793719) bank121522 +121523 POINT(41.551001273215455 74.83028208491854) bank121523 +121524 POINT(41.38867195992542 74.03611224498455) bank121524 +121525 POINT(39.74135097129267 74.00409495585264) bank121525 +121526 POINT(40.04358270355396 73.6381510780756) bank121526 +121527 POINT(40.022725870416664 74.82235224961529) bank121527 +121528 POINT(40.748626783223486 74.81397916895153) bank121528 +121529 POINT(40.20405333071612 74.29019903284477) bank121529 +121530 POINT(40.90769721644882 74.56495028372072) bank121530 +121531 POINT(40.51611926934857 73.30763114022689) bank121531 +121532 POINT(40.148066344439265 74.69297947957892) bank121532 +121533 POINT(41.26407844892544 73.66879591341733) bank121533 +121534 POINT(40.5344527119437 74.61398267724023) bank121534 +121535 POINT(41.32922740780654 74.24867603128803) bank121535 +121536 POINT(40.93523205171793 73.71393835211906) bank121536 +121537 POINT(41.310588979679515 74.05095969581039) bank121537 +121538 POINT(41.20411981102781 73.7403532258134) bank121538 +121539 POINT(40.83956851851792 74.48822422649897) bank121539 +121540 POINT(40.090433526868075 74.31356930585176) bank121540 +121541 POINT(41.440172003026525 73.80207573871355) bank121541 +121542 POINT(40.011039488243846 74.48408962667862) bank121542 +121543 POINT(40.049142436633666 73.08317088593414) bank121543 +121544 POINT(41.02513245572659 74.5263745585708) bank121544 +121545 POINT(41.11427602592651 74.75458802213822) bank121545 +121546 POINT(40.57343265160795 73.19712701480756) bank121546 +121547 POINT(41.54460030952989 74.82163977633486) bank121547 +121548 POINT(41.61630173848599 74.71071337109896) bank121548 +121549 POINT(41.12924072373211 73.67792086477674) bank121549 +121550 POINT(41.46933908237283 74.02086721630185) bank121550 +121551 POINT(40.05496050699714 74.41281697537775) bank121551 +121552 POINT(41.52989078520755 73.88661116188004) bank121552 +121553 POINT(41.3143979838249 73.53514794886361) bank121553 +121554 POINT(41.043414387611634 73.59414359919445) bank121554 +121555 POINT(41.33355826876713 73.44591104571266) bank121555 +121556 POINT(40.249761327566645 73.04013520776587) bank121556 +121557 POINT(39.92915298892558 74.32772665666639) bank121557 +121558 POINT(40.458270666686964 74.6611029395729) bank121558 +121559 POINT(39.94717384083204 74.7729389433293) bank121559 +121560 POINT(40.426576086487344 74.88321290785676) bank121560 +121561 POINT(39.73791668683941 73.91437226600301) bank121561 +121562 POINT(40.00490508553582 73.90834564305584) bank121562 +121563 POINT(40.95992931156656 73.87084930423809) bank121563 +121564 POINT(40.8318475508105 75.00513575350351) bank121564 +121565 POINT(41.29077182717103 73.39816570458824) bank121565 +121566 POINT(41.400221675279745 74.34394342816) bank121566 +121567 POINT(41.24970388084655 73.13613994132578) bank121567 +121568 POINT(40.24349208913169 74.83698261368575) bank121568 +121569 POINT(40.02986970595653 74.8644389627361) bank121569 +121570 POINT(41.26391798985567 74.88470798941233) bank121570 +121571 POINT(41.32680099002168 74.52316092950146) bank121571 +121572 POINT(40.363973859597074 73.97865230903649) bank121572 +121573 POINT(40.00737635586596 73.27177586727935) bank121573 +121574 POINT(40.33883323281867 74.03507156514414) bank121574 +121575 POINT(39.87245887901999 74.14976606553574) bank121575 +121576 POINT(39.80443141883069 74.56684373004326) bank121576 +121577 POINT(41.0259985208138 74.61463425196708) bank121577 +121578 POINT(40.07498379345585 74.23042593730943) bank121578 +121579 POINT(41.651394171956504 73.3459579840395) bank121579 +121580 POINT(41.61568853605213 74.96953063196968) bank121580 +121581 POINT(40.32660602560626 74.4167170909533) bank121581 +121582 POINT(40.045529355565996 74.47386983896004) bank121582 +121583 POINT(40.8932625702415 73.37611326575878) bank121583 +121584 POINT(39.90783779666006 73.65261404069662) bank121584 +121585 POINT(40.789694598254215 74.36214606711492) bank121585 +121586 POINT(39.74359863624827 74.63964432092823) bank121586 +121587 POINT(41.18402580723273 74.32418489227405) bank121587 +121588 POINT(41.17466255954962 74.79683995973886) bank121588 +121589 POINT(40.789697727541515 74.74877590041416) bank121589 +121590 POINT(40.85905040326799 73.31102075234203) bank121590 +121591 POINT(40.53462132953146 73.378699556779) bank121591 +121592 POINT(41.5940129692549 73.87796136020148) bank121592 +121593 POINT(40.79306814514487 74.82129585584991) bank121593 +121594 POINT(41.47444697789312 73.1765383197041) bank121594 +121595 POINT(41.448742465087456 74.06918649386546) bank121595 +121596 POINT(40.111884485528705 74.88100203934204) bank121596 +121597 POINT(40.72259706832876 74.01867596461024) bank121597 +121598 POINT(39.93985116509071 74.40728359905017) bank121598 +121599 POINT(41.23314062215085 74.19122036078467) bank121599 +121600 POINT(41.323993345928706 73.41937177244758) bank121600 +121601 POINT(41.47185981040648 74.08735819010357) bank121601 +121602 POINT(40.56939404012607 74.7740694044671) bank121602 +121603 POINT(41.35221001382347 74.82501429959066) bank121603 +121604 POINT(41.684780959565494 73.39812777184174) bank121604 +121605 POINT(41.16658419100136 74.74285135629806) bank121605 +121606 POINT(40.96870903254609 74.8864768889436) bank121606 +121607 POINT(40.406705564754276 74.92313049831722) bank121607 +121608 POINT(41.1210608231715 74.12163751397884) bank121608 +121609 POINT(40.86612688941381 74.53428159250366) bank121609 +121610 POINT(40.293830237520176 74.48791131416888) bank121610 +121611 POINT(39.87445736248872 73.23181492876944) bank121611 +121612 POINT(40.282339341508724 74.23422051236489) bank121612 +121613 POINT(39.9768334630148 73.422648959422) bank121613 +121614 POINT(39.827593254360885 73.73531974051443) bank121614 +121615 POINT(40.81440922764405 73.59671036687408) bank121615 +121616 POINT(40.77405077552204 73.70555638717332) bank121616 +121617 POINT(41.70592723099861 73.26775741187106) bank121617 +121618 POINT(40.09204765236436 73.99493878756682) bank121618 +121619 POINT(40.030758570915076 73.77267763994051) bank121619 +121620 POINT(39.97693627632581 73.90107922945731) bank121620 +121621 POINT(40.435967092869326 73.30291539756793) bank121621 +121622 POINT(40.46079241456521 73.15998980550633) bank121622 +121623 POINT(41.1227534059704 73.70117629834309) bank121623 +121624 POINT(40.80599413338665 74.34870448890366) bank121624 +121625 POINT(41.26134500707314 74.62782302087805) bank121625 +121626 POINT(41.21898502493455 74.08765616550849) bank121626 +121627 POINT(41.04971416028424 74.2844268844105) bank121627 +121628 POINT(41.089714952998236 74.3327583742347) bank121628 +121629 POINT(40.63718852637765 74.26496953790395) bank121629 +121630 POINT(39.899862308211254 73.38256902684677) bank121630 +121631 POINT(40.96398748940958 73.88156751498681) bank121631 +121632 POINT(40.16824149832527 74.99795830064556) bank121632 +121633 POINT(41.48596933405999 74.2276495016283) bank121633 +121634 POINT(41.34815095564328 73.96240338867031) bank121634 +121635 POINT(40.58768525975366 74.56769633510223) bank121635 +121636 POINT(40.96269797686392 73.33395465573935) bank121636 +121637 POINT(40.01003310334253 74.0182840693745) bank121637 +121638 POINT(41.09719057308358 74.35728822573479) bank121638 +121639 POINT(40.15891864345694 73.20712427817627) bank121639 +121640 POINT(40.00331456704637 73.8843680044269) bank121640 +121641 POINT(40.919642928398396 73.83004678120895) bank121641 +121642 POINT(40.7699991681766 73.72668237667119) bank121642 +121643 POINT(40.429940953916294 74.46996190758797) bank121643 +121644 POINT(40.00204481593669 73.00969396195234) bank121644 +121645 POINT(41.37047394507873 74.07433743011482) bank121645 +121646 POINT(40.86351845934091 73.61412368339596) bank121646 +121647 POINT(40.16770102635314 73.81825515320412) bank121647 +121648 POINT(39.94256381658919 74.48557176520269) bank121648 +121649 POINT(39.77786283031187 73.55688523957365) bank121649 +121650 POINT(40.987363813712406 73.0891012122877) bank121650 +121651 POINT(39.74789487271158 73.32605856779453) bank121651 +121652 POINT(40.58927760317849 73.19459005452941) bank121652 +121653 POINT(41.199674622531525 73.68811080088263) bank121653 +121654 POINT(41.207151829335004 73.59235588442442) bank121654 +121655 POINT(41.3777657728718 74.83478269296428) bank121655 +121656 POINT(40.7133003818064 73.48756891082888) bank121656 +121657 POINT(40.15829832676565 74.2634236147236) bank121657 +121658 POINT(40.714789441370854 73.92291554849514) bank121658 +121659 POINT(40.51037864074337 73.5194373766848) bank121659 +121660 POINT(41.401336084696126 73.11132699413389) bank121660 +121661 POINT(40.18931011616625 73.53300910540236) bank121661 +121662 POINT(41.581587865076926 73.99386129203046) bank121662 +121663 POINT(40.77155349430164 73.48670943639752) bank121663 +121664 POINT(41.138362877410536 74.7685848285484) bank121664 +121665 POINT(40.181824197662735 74.26646476272673) bank121665 +121666 POINT(40.985804958445456 74.56649146623646) bank121666 +121667 POINT(39.85130728018962 74.94882410484806) bank121667 +121668 POINT(41.1918028982182 74.52904101154917) bank121668 +121669 POINT(41.49384476221551 73.29724835065295) bank121669 +121670 POINT(41.21131897513315 74.01240901181107) bank121670 +121671 POINT(41.00311377523643 74.57422924960038) bank121671 +121672 POINT(40.961723582474626 73.7098406357376) bank121672 +121673 POINT(39.91658794552232 74.95378183440417) bank121673 +121674 POINT(40.25689614187761 74.4662062214109) bank121674 +121675 POINT(40.178030260991186 73.21898863670354) bank121675 +121676 POINT(40.22836737178639 74.64051885855301) bank121676 +121677 POINT(41.322286463162634 73.98112558921096) bank121677 +121678 POINT(40.56340997438163 74.1100674317515) bank121678 +121679 POINT(40.458054956922204 73.01423706240666) bank121679 +121680 POINT(41.166823006191095 74.81650759113255) bank121680 +121681 POINT(40.692921365440185 73.61531928256336) bank121681 +121682 POINT(41.347941423542665 73.3802622165385) bank121682 +121683 POINT(40.609473384259445 74.04558302010975) bank121683 +121684 POINT(39.94124586231718 74.73491115582456) bank121684 +121685 POINT(41.153486658362226 74.0673360595982) bank121685 +121686 POINT(40.450199270618086 74.29449695293353) bank121686 +121687 POINT(41.68302370727185 73.5170754776113) bank121687 +121688 POINT(41.638012698904525 74.63562310153087) bank121688 +121689 POINT(41.0188398975659 74.41646311064358) bank121689 +121690 POINT(39.887664360742335 73.03102572712235) bank121690 +121691 POINT(40.49779637193591 74.5114806246181) bank121691 +121692 POINT(41.38614617103668 73.97551758360697) bank121692 +121693 POINT(40.862214036473674 73.80624228992704) bank121693 +121694 POINT(39.89342446379595 74.04859513288636) bank121694 +121695 POINT(40.265753363518215 74.61002259728862) bank121695 +121696 POINT(40.3234075198174 74.16121225373445) bank121696 +121697 POINT(40.31644485933513 74.4338321681602) bank121697 +121698 POINT(41.13094828139693 74.61476475996815) bank121698 +121699 POINT(41.114125080260656 74.00760919099685) bank121699 +121700 POINT(41.000303779629334 73.10524093092502) bank121700 +121701 POINT(41.27870697791304 73.55018685858583) bank121701 +121702 POINT(41.16894817829431 73.97723481918231) bank121702 +121703 POINT(40.570120499464856 74.44062872924383) bank121703 +121704 POINT(41.58483499907995 74.24323773553817) bank121704 +121705 POINT(40.11675322497665 73.99666731259639) bank121705 +121706 POINT(40.178824588819985 73.66067674932759) bank121706 +121707 POINT(39.735188193354624 73.35750251785942) bank121707 +121708 POINT(41.66137629722906 74.88532591792473) bank121708 +121709 POINT(40.40939852173233 74.82685538918686) bank121709 +121710 POINT(40.15576068454302 73.17381891927087) bank121710 +121711 POINT(41.081193142790084 73.87361556008483) bank121711 +121712 POINT(41.427977070565376 73.89558288763865) bank121712 +121713 POINT(41.659361054811335 74.11724257213038) bank121713 +121714 POINT(41.572476199645955 74.53114738440246) bank121714 +121715 POINT(40.886563724228374 73.51067408455147) bank121715 +121716 POINT(39.942297906152206 74.30720195462185) bank121716 +121717 POINT(40.64825860950093 73.46582286743912) bank121717 +121718 POINT(40.50852568605122 73.98789266145683) bank121718 +121719 POINT(40.5421491396263 74.6069816647298) bank121719 +121720 POINT(40.556044814685364 74.47418587920093) bank121720 +121721 POINT(41.26810388253041 73.20596773734788) bank121721 +121722 POINT(40.08763343284154 73.99084454689486) bank121722 +121723 POINT(41.176546887603564 74.71625065251041) bank121723 +121724 POINT(41.45079875559056 73.29503957774641) bank121724 +121725 POINT(41.09633504505189 74.22966454280188) bank121725 +121726 POINT(41.527873559408974 73.42923649556383) bank121726 +121727 POINT(39.82707680174391 74.66924165033011) bank121727 +121728 POINT(40.71661933815038 74.88166181851551) bank121728 +121729 POINT(40.95765332972784 73.15337250715075) bank121729 +121730 POINT(40.8505336918219 74.78731268941867) bank121730 +121731 POINT(41.40668236808548 73.51916236489903) bank121731 +121732 POINT(40.67326776187229 74.30862433456504) bank121732 +121733 POINT(40.74383684940506 73.47508656900554) bank121733 +121734 POINT(41.20540328479476 73.59234976077455) bank121734 +121735 POINT(41.3267549574919 73.10031756804976) bank121735 +121736 POINT(40.9182321732886 73.13542960696743) bank121736 +121737 POINT(40.98561565762451 74.1513165844416) bank121737 +121738 POINT(41.02255004358872 74.69485968917272) bank121738 +121739 POINT(41.549265509928766 74.54628269678614) bank121739 +121740 POINT(40.92157473920527 73.89040221785973) bank121740 +121741 POINT(40.53386715305377 73.49549549048979) bank121741 +121742 POINT(40.94945635627912 74.57776510785463) bank121742 +121743 POINT(41.039010161258716 74.66118916221592) bank121743 +121744 POINT(41.34367957235742 73.62395244238523) bank121744 +121745 POINT(40.352545616893416 73.115674163806) bank121745 +121746 POINT(41.181029829781984 74.199278697455) bank121746 +121747 POINT(40.91753829816498 73.62893362217129) bank121747 +121748 POINT(40.871017674980315 74.85545624454058) bank121748 +121749 POINT(40.568872041878315 73.59414266656592) bank121749 +121750 POINT(39.89424365419419 74.74726891704663) bank121750 +121751 POINT(40.945608616177594 73.10725796671427) bank121751 +121752 POINT(40.64172567208128 74.73010540797668) bank121752 +121753 POINT(40.239119579003 74.98768775386944) bank121753 +121754 POINT(40.613974558572075 74.62096568461934) bank121754 +121755 POINT(40.52146962700856 73.5951392793579) bank121755 +121756 POINT(39.80896864435609 74.33293721503395) bank121756 +121757 POINT(40.37262965220664 74.04206348161135) bank121757 +121758 POINT(41.53646574326865 74.61965894546302) bank121758 +121759 POINT(40.01368557995728 73.18349793208561) bank121759 +121760 POINT(41.58895710046168 73.80453887327468) bank121760 +121761 POINT(41.46404431113629 74.67660981521246) bank121761 +121762 POINT(40.78775728883514 73.03280706762794) bank121762 +121763 POINT(40.69727032443238 74.26260123701094) bank121763 +121764 POINT(41.709497273646406 73.80964922347246) bank121764 +121765 POINT(40.814706026574605 74.03093189392395) bank121765 +121766 POINT(40.43645150005557 73.33648185519466) bank121766 +121767 POINT(40.4773668396204 73.96604231431083) bank121767 +121768 POINT(41.291088739883314 74.12287573210124) bank121768 +121769 POINT(40.1446468361966 74.1055514583494) bank121769 +121770 POINT(41.172968814202434 74.52401775105082) bank121770 +121771 POINT(40.55594486637178 73.64109560203401) bank121771 +121772 POINT(40.62782047950634 74.09916692656957) bank121772 +121773 POINT(39.837794164336266 74.48079372317939) bank121773 +121774 POINT(40.666036675834725 73.71751905335627) bank121774 +121775 POINT(41.06280627461986 73.8167950742228) bank121775 +121776 POINT(41.47770744734606 73.92356450424603) bank121776 +121777 POINT(40.700206304712204 73.53949079585516) bank121777 +121778 POINT(41.629960582733084 74.7933579327892) bank121778 +121779 POINT(40.4075972836108 74.35334874883617) bank121779 +121780 POINT(40.263980616645085 74.30223469823676) bank121780 +121781 POINT(40.83124284988496 73.72499476324218) bank121781 +121782 POINT(40.59813981258716 73.21671315129537) bank121782 +121783 POINT(41.20493421479555 74.5100281812234) bank121783 +121784 POINT(40.015826446863485 74.2301444391608) bank121784 +121785 POINT(40.58776661379392 74.56608173110189) bank121785 +121786 POINT(40.501115245182085 73.5731280253904) bank121786 +121787 POINT(40.266633688918 74.53848630990694) bank121787 +121788 POINT(40.83177715328032 74.92634041550531) bank121788 +121789 POINT(40.36169573370864 74.56996509617196) bank121789 +121790 POINT(40.99220440201658 74.55131589539205) bank121790 +121791 POINT(39.83678659737569 73.05554277845829) bank121791 +121792 POINT(40.635222386917114 74.86100682568325) bank121792 +121793 POINT(41.35902881499138 74.90290215598539) bank121793 +121794 POINT(39.819087857191114 74.44215548477078) bank121794 +121795 POINT(40.071129201686794 74.65427244806588) bank121795 +121796 POINT(41.57352515533141 73.90096122357691) bank121796 +121797 POINT(40.557320831176526 73.21630204679357) bank121797 +121798 POINT(41.38542191556975 74.37140900772036) bank121798 +121799 POINT(40.59153124690892 73.1652293365683) bank121799 +121800 POINT(41.273549968171906 73.44014865092328) bank121800 +121801 POINT(40.810783553071545 73.36233395737668) bank121801 +121802 POINT(40.39336306725118 74.51466732317216) bank121802 +121803 POINT(41.49123562673726 74.5547919875473) bank121803 +121804 POINT(41.47662464072025 74.15477998136001) bank121804 +121805 POINT(40.15232782720409 73.01545625900008) bank121805 +121806 POINT(39.74106836988395 74.04677543843515) bank121806 +121807 POINT(39.83425383163041 73.36615962225905) bank121807 +121808 POINT(40.328932453555566 73.39685601018603) bank121808 +121809 POINT(39.87652256770854 74.3076007160547) bank121809 +121810 POINT(39.73495777359713 73.13187908871674) bank121810 +121811 POINT(40.99985549573042 74.40750850644345) bank121811 +121812 POINT(39.87656719442603 73.61068529507979) bank121812 +121813 POINT(41.304405278713396 74.35613588760515) bank121813 +121814 POINT(41.07357678620973 73.53665298856703) bank121814 +121815 POINT(39.94786719125305 73.92236859165166) bank121815 +121816 POINT(41.628426848273755 73.81063062444991) bank121816 +121817 POINT(41.70326834088266 74.07829147638725) bank121817 +121818 POINT(41.5752691273927 74.11557924338668) bank121818 +121819 POINT(40.85184495076179 74.65672908461953) bank121819 +121820 POINT(40.699710852565694 73.91181089481661) bank121820 +121821 POINT(40.96592572881951 75.00465593438226) bank121821 +121822 POINT(41.02767294275256 74.59632361899328) bank121822 +121823 POINT(41.60992391837284 74.43520102348745) bank121823 +121824 POINT(39.719577645504145 74.73535170527552) bank121824 +121825 POINT(39.71758081889526 73.92916917848376) bank121825 +121826 POINT(41.676992260314826 73.64154350255798) bank121826 +121827 POINT(41.063101296882735 74.01803116042622) bank121827 +121828 POINT(40.49590970715872 74.11109049556526) bank121828 +121829 POINT(41.497412779442435 73.97894970244259) bank121829 +121830 POINT(41.562724238639156 74.65542155033401) bank121830 +121831 POINT(39.72202944663339 74.4462635213255) bank121831 +121832 POINT(41.08105539425302 74.31771288814966) bank121832 +121833 POINT(39.910285269262225 74.40797818201477) bank121833 +121834 POINT(39.85276614767894 74.25920979553477) bank121834 +121835 POINT(41.272346353335344 74.86890856620309) bank121835 +121836 POINT(41.71089850133607 73.01115127571397) bank121836 +121837 POINT(40.680496377859455 73.94217453158686) bank121837 +121838 POINT(41.098185739537726 74.85536734739797) bank121838 +121839 POINT(41.44707604081607 74.43111647885436) bank121839 +121840 POINT(41.236817557598755 74.3069759030754) bank121840 +121841 POINT(40.07621288198364 73.77508913551202) bank121841 +121842 POINT(40.62782211294969 73.79103915057107) bank121842 +121843 POINT(40.84894929371384 73.87745476299911) bank121843 +121844 POINT(40.19186711967871 74.4175343440308) bank121844 +121845 POINT(41.59956990609494 74.23486246018598) bank121845 +121846 POINT(41.22451072993464 74.84976632524965) bank121846 +121847 POINT(40.16720969907878 74.8115929993946) bank121847 +121848 POINT(40.23021515170705 74.36324842603315) bank121848 +121849 POINT(41.04250333916144 74.14185429404995) bank121849 +121850 POINT(40.682921421936605 74.45982320280568) bank121850 +121851 POINT(40.35095177988618 73.53611692349938) bank121851 +121852 POINT(41.23961756216879 73.18011130685893) bank121852 +121853 POINT(40.10299588001319 73.28920106777825) bank121853 +121854 POINT(40.1045240561513 74.03026284687276) bank121854 +121855 POINT(40.553711450941556 74.60051248248507) bank121855 +121856 POINT(41.538151912241915 73.0152198373259) bank121856 +121857 POINT(40.51054325175884 74.67746192024366) bank121857 +121858 POINT(41.599900722508345 74.01569864474907) bank121858 +121859 POINT(40.11441077250937 73.17888109337268) bank121859 +121860 POINT(41.00804231690501 74.45253037148711) bank121860 +121861 POINT(40.2580080622763 73.09506094027941) bank121861 +121862 POINT(40.21080907352616 74.56551206738673) bank121862 +121863 POINT(40.423462484297595 74.03725527257531) bank121863 +121864 POINT(41.44388157547187 74.23256681189989) bank121864 +121865 POINT(40.554302667794765 73.9207474744075) bank121865 +121866 POINT(40.34246990153485 74.46429038426655) bank121866 +121867 POINT(41.632843692182085 74.71317181627396) bank121867 +121868 POINT(39.77603652687677 74.7431306982355) bank121868 +121869 POINT(40.148832167700775 74.91687533145218) bank121869 +121870 POINT(40.11339349198513 74.18038781226771) bank121870 +121871 POINT(40.08790717373485 74.24238810793963) bank121871 +121872 POINT(40.256866799316114 73.15966181368377) bank121872 +121873 POINT(41.046701462908175 74.00956330349085) bank121873 +121874 POINT(40.46606794415627 73.86425387495912) bank121874 +121875 POINT(40.77377694117903 74.63499208758623) bank121875 +121876 POINT(41.065559931722255 74.65112727871433) bank121876 +121877 POINT(40.182164206258385 73.05192680476564) bank121877 +121878 POINT(41.601536899379 74.72293653477809) bank121878 +121879 POINT(41.45123308139686 73.37657952532857) bank121879 +121880 POINT(41.38779484757502 74.952037378815) bank121880 +121881 POINT(41.60266425439763 74.55081557643464) bank121881 +121882 POINT(40.751648515160696 74.23035941386401) bank121882 +121883 POINT(41.60546268908328 74.27743606718865) bank121883 +121884 POINT(40.42108000530327 73.64956233988863) bank121884 +121885 POINT(40.91023403761686 73.5106188276326) bank121885 +121886 POINT(39.76541989840839 74.21377224295092) bank121886 +121887 POINT(39.73501241320193 73.82826646602264) bank121887 +121888 POINT(40.178336034406136 73.91059576358492) bank121888 +121889 POINT(40.56895075820406 73.21013478751634) bank121889 +121890 POINT(41.09638647332371 74.06466292336154) bank121890 +121891 POINT(41.01913951876839 73.29008417455297) bank121891 +121892 POINT(41.25927137861832 74.9672249073846) bank121892 +121893 POINT(41.55366416819368 74.06224758378887) bank121893 +121894 POINT(39.821213009603646 73.86049421298853) bank121894 +121895 POINT(39.97748461701782 73.76264252291635) bank121895 +121896 POINT(41.22977098776783 73.46568183888104) bank121896 +121897 POINT(41.55113289291289 73.8337606549191) bank121897 +121898 POINT(39.80946325421702 74.18432990762979) bank121898 +121899 POINT(40.45857002410459 73.80865329435551) bank121899 +121900 POINT(40.91207581891575 73.19726865820049) bank121900 +121901 POINT(40.835503744043294 73.74796181787757) bank121901 +121902 POINT(41.56635499890859 74.77243929532648) bank121902 +121903 POINT(41.689566927172194 74.4375625924704) bank121903 +121904 POINT(39.80189704598449 73.14701002837926) bank121904 +121905 POINT(41.057137204747185 74.3259129041231) bank121905 +121906 POINT(40.78473067645695 74.90918802315915) bank121906 +121907 POINT(41.45554251829726 73.76754498960486) bank121907 +121908 POINT(41.710424398399724 73.52864101257647) bank121908 +121909 POINT(40.163023097262844 74.63364703805917) bank121909 +121910 POINT(40.29394135937836 74.62183802787766) bank121910 +121911 POINT(40.50749032189455 73.49556777779961) bank121911 +121912 POINT(39.81620984984745 73.07040710180345) bank121912 +121913 POINT(40.725850994668164 73.66706935572061) bank121913 +121914 POINT(41.40392768913606 74.54400118446699) bank121914 +121915 POINT(40.62294471679192 73.16536567155836) bank121915 +121916 POINT(41.27459236094171 74.83808298982619) bank121916 +121917 POINT(40.35906056705311 73.32254587188926) bank121917 +121918 POINT(40.10986574591745 74.69752626037031) bank121918 +121919 POINT(40.541504737315464 74.2786916735045) bank121919 +121920 POINT(40.715498097067986 74.8067786139291) bank121920 +121921 POINT(40.92467365699126 74.3968924427505) bank121921 +121922 POINT(39.886614524424246 73.10580028720977) bank121922 +121923 POINT(40.08208811120102 74.27737026071777) bank121923 +121924 POINT(41.179915986251 74.42727512874329) bank121924 +121925 POINT(40.49079813395956 74.1690337467854) bank121925 +121926 POINT(41.59613757921485 73.41366799070956) bank121926 +121927 POINT(40.939165291081935 74.21905030232463) bank121927 +121928 POINT(41.69881798507017 74.79691284205947) bank121928 +121929 POINT(41.1120354940329 74.39394751222704) bank121929 +121930 POINT(40.96140017707723 73.42112717122139) bank121930 +121931 POINT(39.947136581421276 73.0237564183743) bank121931 +121932 POINT(40.09504756891867 73.71060374002283) bank121932 +121933 POINT(40.305954456461926 74.86204776991437) bank121933 +121934 POINT(40.69623312904879 74.65519905429714) bank121934 +121935 POINT(40.55836976100812 74.211335524453) bank121935 +121936 POINT(41.244441119185645 74.52516759481287) bank121936 +121937 POINT(41.48811239862285 74.7907385679398) bank121937 +121938 POINT(40.30433465137072 74.89479511169169) bank121938 +121939 POINT(41.46091808844692 73.7796111171781) bank121939 +121940 POINT(40.265875963077846 73.36962533809151) bank121940 +121941 POINT(39.99585045040532 73.4071401842114) bank121941 +121942 POINT(40.750399485025056 73.60671416512668) bank121942 +121943 POINT(39.9737450005403 74.82821219433475) bank121943 +121944 POINT(39.92880431623994 73.64251048829152) bank121944 +121945 POINT(41.3653431633543 73.67291662702772) bank121945 +121946 POINT(40.647036675644465 74.6227710509627) bank121946 +121947 POINT(39.78425735044954 73.97138150256974) bank121947 +121948 POINT(40.461091575802044 74.26690484481576) bank121948 +121949 POINT(40.06236458838461 73.05722369202557) bank121949 +121950 POINT(41.325406394541716 73.65246733239135) bank121950 +121951 POINT(41.35232359402904 73.37218854683134) bank121951 +121952 POINT(40.798044691846684 74.69194869183585) bank121952 +121953 POINT(41.363080476972755 73.74268395614543) bank121953 +121954 POINT(40.71694172917509 74.32827800836426) bank121954 +121955 POINT(39.956883362216544 73.11110466452573) bank121955 +121956 POINT(41.67021510661877 73.9282752979303) bank121956 +121957 POINT(41.23013080962882 73.36439664142102) bank121957 +121958 POINT(40.277099420968476 74.91863202296159) bank121958 +121959 POINT(41.145781017205 73.69881556535145) bank121959 +121960 POINT(39.86830266623453 74.88972067827494) bank121960 +121961 POINT(41.214490984867616 73.66053426058029) bank121961 +121962 POINT(40.013717112889694 74.39479405023734) bank121962 +121963 POINT(41.30685717480069 74.1469480617088) bank121963 +121964 POINT(41.29075150716332 74.30632127922297) bank121964 +121965 POINT(39.760371327375836 73.61949842529211) bank121965 +121966 POINT(40.0049133450418 73.85484663319085) bank121966 +121967 POINT(40.88273580082935 74.33092723925246) bank121967 +121968 POINT(40.00401027227508 74.07917672180177) bank121968 +121969 POINT(40.69142311638332 74.33332920217306) bank121969 +121970 POINT(41.534247389533995 73.65664032400737) bank121970 +121971 POINT(40.94948565768245 74.52791873500314) bank121971 +121972 POINT(41.351076970849284 74.72296595771587) bank121972 +121973 POINT(40.64758087015503 74.14908387400183) bank121973 +121974 POINT(40.97499909180141 74.60452187068358) bank121974 +121975 POINT(40.97067582742442 73.61863066165198) bank121975 +121976 POINT(41.18246134427459 74.62440537161459) bank121976 +121977 POINT(40.42700860297921 74.59234834655895) bank121977 +121978 POINT(40.56549269942918 74.31579498296321) bank121978 +121979 POINT(39.94515960459339 73.91816151573457) bank121979 +121980 POINT(40.26730875409425 73.01633914876301) bank121980 +121981 POINT(40.67941387622888 74.55853674145256) bank121981 +121982 POINT(40.10847311742018 74.84889554723567) bank121982 +121983 POINT(40.17887397737201 73.72754379442708) bank121983 +121984 POINT(39.971236817107055 74.48931612564245) bank121984 +121985 POINT(40.175253915953704 74.89216048957375) bank121985 +121986 POINT(39.87978851990247 74.83155907512274) bank121986 +121987 POINT(40.741809363771935 73.98897962314821) bank121987 +121988 POINT(41.39870101050224 74.18101479746416) bank121988 +121989 POINT(40.58684050807139 74.90562817941228) bank121989 +121990 POINT(40.663169434072444 73.33386721358147) bank121990 +121991 POINT(40.22272183647761 73.74759490099315) bank121991 +121992 POINT(40.47940389004905 74.1112247752918) bank121992 +121993 POINT(40.421926972483014 74.33883686219761) bank121993 +121994 POINT(40.5577897165307 73.21340287366974) bank121994 +121995 POINT(41.532298146321864 74.35377086324347) bank121995 +121996 POINT(40.66656932558685 74.9303269756072) bank121996 +121997 POINT(41.297115401120664 73.85878283184002) bank121997 +121998 POINT(40.839339404769895 74.39118622247923) bank121998 +121999 POINT(40.84808560268847 74.28568105957896) bank121999 +122000 POINT(40.398473875700084 74.96747572338208) bank122000 +122001 POINT(39.970761255629824 74.1509575874259) bank122001 +122002 POINT(39.86755049507822 74.93272929367134) bank122002 +122003 POINT(40.25182167843432 73.62239341079017) bank122003 +122004 POINT(40.23715971764651 74.71044831399479) bank122004 +122005 POINT(40.672404881497094 73.44313619785892) bank122005 +122006 POINT(41.39135324949369 73.84825013573081) bank122006 +122007 POINT(40.63852220707428 74.15429265180232) bank122007 +122008 POINT(40.41240484857163 74.63993664590726) bank122008 +122009 POINT(41.60381469027868 74.55243683250282) bank122009 +122010 POINT(41.36258418099425 74.70223937294429) bank122010 +122011 POINT(40.108696646844976 73.91300093145009) bank122011 +122012 POINT(40.811698063609406 74.15781437564767) bank122012 +122013 POINT(40.774294999673096 74.33073966136183) bank122013 +122014 POINT(40.48386961635054 73.05091437459485) bank122014 +122015 POINT(40.29106760236556 73.3669262567581) bank122015 +122016 POINT(39.943581460430565 73.12890143213923) bank122016 +122017 POINT(41.63317772361789 73.11924141276636) bank122017 +122018 POINT(39.723545045970454 73.10166209391991) bank122018 +122019 POINT(41.677819031259894 73.53594147488276) bank122019 +122020 POINT(40.793733571693345 73.37484436099074) bank122020 +122021 POINT(39.85775776339707 74.594066774186) bank122021 +122022 POINT(40.030728726520756 73.98718662034037) bank122022 +122023 POINT(39.752994936093764 74.33923145016342) bank122023 +122024 POINT(40.4881090305222 73.9066072160303) bank122024 +122025 POINT(41.68870013060284 74.40140193393161) bank122025 +122026 POINT(40.791896045578056 73.17789644018495) bank122026 +122027 POINT(40.77287612810439 73.50272560491278) bank122027 +122028 POINT(40.751039933030455 74.75732520135064) bank122028 +122029 POINT(41.62346716934863 73.4564358336389) bank122029 +122030 POINT(40.8907888324604 73.58174258060673) bank122030 +122031 POINT(39.92794786741345 74.05687074234125) bank122031 +122032 POINT(39.845733624352086 73.10640253779368) bank122032 +122033 POINT(40.13114631074891 73.17347149323139) bank122033 +122034 POINT(41.032804363943214 73.00857357469003) bank122034 +122035 POINT(40.54148793378283 74.9459036721672) bank122035 +122036 POINT(40.64856264839896 74.05233415643194) bank122036 +122037 POINT(39.72504148943656 73.0334278448669) bank122037 +122038 POINT(41.24775592132245 73.60675719692958) bank122038 +122039 POINT(40.066413776891 74.35407360096318) bank122039 +122040 POINT(41.26136665643537 73.25897496225822) bank122040 +122041 POINT(40.610918142008714 74.93799339567205) bank122041 +122042 POINT(41.593107278943066 74.58923646506543) bank122042 +122043 POINT(40.52977472356754 73.54093420920822) bank122043 +122044 POINT(41.258347944142784 74.9281970659329) bank122044 +122045 POINT(39.816393772624814 74.87036417986083) bank122045 +122046 POINT(41.1340475619466 73.06166084884339) bank122046 +122047 POINT(40.94691087340212 74.57742516244883) bank122047 +122048 POINT(40.2504923153081 74.30736613023247) bank122048 +122049 POINT(40.531256368519294 73.56895804262572) bank122049 +122050 POINT(41.518415019142076 73.73218241647557) bank122050 +122051 POINT(41.03428457736132 73.23910581160416) bank122051 +122052 POINT(40.03056943066766 74.69285611471805) bank122052 +122053 POINT(41.6190092703287 73.87193122618092) bank122053 +122054 POINT(40.98894887100888 74.94804044332658) bank122054 +122055 POINT(40.92217065956616 73.44017580270516) bank122055 +122056 POINT(41.560494600394 74.81201019037879) bank122056 +122057 POINT(39.75310536969397 73.33898230952002) bank122057 +122058 POINT(41.591309533645976 74.04635088998195) bank122058 +122059 POINT(40.35242679240984 74.38470433491497) bank122059 +122060 POINT(39.86383494252817 73.93383342594096) bank122060 +122061 POINT(41.31998530870776 74.82641376088729) bank122061 +122062 POINT(40.51497077018207 74.60635289035405) bank122062 +122063 POINT(40.181418584780175 74.53630069494032) bank122063 +122064 POINT(40.45544404952905 73.17264385365873) bank122064 +122065 POINT(41.68278142007344 74.76319130199265) bank122065 +122066 POINT(41.08541866922007 73.92310705541666) bank122066 +122067 POINT(39.7390936315644 73.92513019438799) bank122067 +122068 POINT(41.5197896301417 74.89194632145852) bank122068 +122069 POINT(39.95761793064623 74.43780548104692) bank122069 +122070 POINT(40.86540745413241 73.68195709873342) bank122070 +122071 POINT(40.304120387731366 73.50546788478795) bank122071 +122072 POINT(40.96898043247232 74.21122345967134) bank122072 +122073 POINT(41.245008484032894 73.43013580848108) bank122073 +122074 POINT(40.530655415470996 74.75274784976921) bank122074 +122075 POINT(40.58969358883559 73.13945499457203) bank122075 +122076 POINT(41.46535225937936 73.44454941049476) bank122076 +122077 POINT(40.30056125820813 74.05695580325997) bank122077 +122078 POINT(39.74545584983027 73.73699706882309) bank122078 +122079 POINT(40.6861930424719 73.55620051274025) bank122079 +122080 POINT(41.30278966079786 74.45957804042996) bank122080 +122081 POINT(40.819045317902756 74.18062818878127) bank122081 +122082 POINT(40.475745781908046 73.63491638197083) bank122082 +122083 POINT(41.5044896939438 73.44833211341724) bank122083 +122084 POINT(40.93739208006803 73.51431302638612) bank122084 +122085 POINT(41.459180066069166 74.96371540698493) bank122085 +122086 POINT(40.27303149245138 74.79494139292217) bank122086 +122087 POINT(40.0815338670548 74.68819726060924) bank122087 +122088 POINT(39.89246063000795 73.9157357078451) bank122088 +122089 POINT(41.03722829594256 74.04472375000876) bank122089 +122090 POINT(41.34093251093003 74.88288766254404) bank122090 +122091 POINT(40.835634730560315 74.67446146294947) bank122091 +122092 POINT(40.32823054922536 74.92048215114428) bank122092 +122093 POINT(39.990361688826326 73.40750042867063) bank122093 +122094 POINT(40.1796083217601 74.80971059759392) bank122094 +122095 POINT(41.455404391341624 74.38575259790811) bank122095 +122096 POINT(41.686812507760024 74.41944066346099) bank122096 +122097 POINT(41.110301070618156 74.70899426301028) bank122097 +122098 POINT(41.461721910332535 73.40473370299505) bank122098 +122099 POINT(41.644778222671256 74.88587140260564) bank122099 +122100 POINT(41.169657899663896 73.55734515977423) bank122100 +122101 POINT(40.529178237785345 74.96339668060448) bank122101 +122102 POINT(40.83540558649998 73.57539338832812) bank122102 +122103 POINT(40.816891130064775 73.32233970458688) bank122103 +122104 POINT(39.80786181157532 74.7971919618452) bank122104 +122105 POINT(40.37343617710936 74.81205675138114) bank122105 +122106 POINT(40.23119035302287 74.0027586239171) bank122106 +122107 POINT(41.56910415340308 73.61242565419452) bank122107 +122108 POINT(40.63707601577682 74.54141827285144) bank122108 +122109 POINT(40.09929684639705 73.10709063840896) bank122109 +122110 POINT(39.930139842799086 74.29610295322722) bank122110 +122111 POINT(39.972306769090835 73.58039073177355) bank122111 +122112 POINT(41.20411331959947 74.75149088069917) bank122112 +122113 POINT(40.89106352346127 73.51218326912122) bank122113 +122114 POINT(39.96605053818001 73.46333371493763) bank122114 +122115 POINT(41.21064448161631 74.7723562028518) bank122115 +122116 POINT(40.03324787584582 74.08762739533984) bank122116 +122117 POINT(40.60199833425605 73.0645231108405) bank122117 +122118 POINT(40.48910604448603 73.41061636762804) bank122118 +122119 POINT(40.58808207683341 73.26935858989535) bank122119 +122120 POINT(41.52364815113663 73.66126236370094) bank122120 +122121 POINT(41.431328169763006 74.20829431277878) bank122121 +122122 POINT(39.74050337826332 74.45833292405993) bank122122 +122123 POINT(40.886112700524734 73.67403911603573) bank122123 +122124 POINT(40.61926935391607 73.0324760381064) bank122124 +122125 POINT(40.73831100965899 74.50363727379434) bank122125 +122126 POINT(40.13102394275473 74.53191537844211) bank122126 +122127 POINT(40.05276632652347 73.53837886653763) bank122127 +122128 POINT(41.53203580376422 73.68605400954651) bank122128 +122129 POINT(40.23266668703391 73.55588858918614) bank122129 +122130 POINT(40.094813529336484 74.16636831817357) bank122130 +122131 POINT(39.84490806770947 74.68258023969412) bank122131 +122132 POINT(39.724307497649434 74.14990371439255) bank122132 +122133 POINT(40.170607844685286 74.79206896761993) bank122133 +122134 POINT(41.2152129020759 74.54894540451541) bank122134 +122135 POINT(40.57802486505154 74.48621944420059) bank122135 +122136 POINT(39.88760836941017 74.55813493775312) bank122136 +122137 POINT(39.87134462043298 74.05003068406467) bank122137 +122138 POINT(41.68530302774912 73.36225551574833) bank122138 +122139 POINT(41.077089269742316 74.87248245860772) bank122139 +122140 POINT(41.35377388026628 74.61011228983828) bank122140 +122141 POINT(40.64804282417653 73.27804069610426) bank122141 +122142 POINT(40.88189614993897 73.13673809176757) bank122142 +122143 POINT(40.03471618349483 73.61758304712352) bank122143 +122144 POINT(41.04983219174887 73.69875390476666) bank122144 +122145 POINT(39.773996594010725 74.48305940881971) bank122145 +122146 POINT(40.82444200702128 73.75928139062958) bank122146 +122147 POINT(39.793125434127866 73.35497457291338) bank122147 +122148 POINT(39.71552349184974 73.74858379022125) bank122148 +122149 POINT(41.263846857671616 74.8246551720108) bank122149 +122150 POINT(40.930864035555224 74.41770334678078) bank122150 +122151 POINT(41.336280787151246 74.28724143796211) bank122151 +122152 POINT(40.96517062010027 74.34767049606891) bank122152 +122153 POINT(40.13574184602515 73.37402238628596) bank122153 +122154 POINT(40.451384831297545 73.36543150931249) bank122154 +122155 POINT(41.374816436154056 74.72221426304205) bank122155 +122156 POINT(40.945839718725495 73.49946236214511) bank122156 +122157 POINT(40.54291417468393 74.02223296560183) bank122157 +122158 POINT(40.4494491496573 73.28561751655897) bank122158 +122159 POINT(40.206761471797485 74.16045897737688) bank122159 +122160 POINT(40.198890415543794 73.4053839215399) bank122160 +122161 POINT(40.82627401622736 74.99804414884326) bank122161 +122162 POINT(41.419514514107334 74.09979361110885) bank122162 +122163 POINT(41.323381499110276 73.23072198149535) bank122163 +122164 POINT(41.133107201097545 73.3487256852531) bank122164 +122165 POINT(40.99478762509031 74.66518900235073) bank122165 +122166 POINT(40.169388921585075 73.82913411107413) bank122166 +122167 POINT(41.508982749051555 73.87762422947652) bank122167 +122168 POINT(39.7587821052021 74.82890436996705) bank122168 +122169 POINT(40.0074659546933 73.39512999915678) bank122169 +122170 POINT(41.335363661986634 73.60478783893899) bank122170 +122171 POINT(41.200034315817916 74.72430629606428) bank122171 +122172 POINT(39.72039621226859 74.26825112037699) bank122172 +122173 POINT(39.90166785761802 73.58565482528859) bank122173 +122174 POINT(41.43431256468619 73.43517404244162) bank122174 +122175 POINT(41.1522797785234 74.9723743526641) bank122175 +122176 POINT(39.926657762614674 73.04869053271187) bank122176 +122177 POINT(40.802147234614225 73.045642185106) bank122177 +122178 POINT(41.26496621284488 73.29713845369574) bank122178 +122179 POINT(40.552450942265175 73.4231996175121) bank122179 +122180 POINT(41.37203092869294 73.57096268028376) bank122180 +122181 POINT(40.49587121653631 73.81652086137049) bank122181 +122182 POINT(40.584634897338006 73.5688454516761) bank122182 +122183 POINT(41.16881238118501 73.7343520190011) bank122183 +122184 POINT(40.65223883182022 74.29464561318896) bank122184 +122185 POINT(40.54525606966866 73.10491375335711) bank122185 +122186 POINT(40.74721204981634 73.31342856647551) bank122186 +122187 POINT(40.522728351584405 74.26015852219362) bank122187 +122188 POINT(40.156105124157854 73.64670435947382) bank122188 +122189 POINT(39.84236786177667 74.62109341431447) bank122189 +122190 POINT(40.069286469994054 73.06776540664089) bank122190 +122191 POINT(40.98496878718973 73.4234156038758) bank122191 +122192 POINT(41.18667053845481 74.27903566705923) bank122192 +122193 POINT(40.70372359278976 74.5300523614497) bank122193 +122194 POINT(40.0016044677133 73.33639598973836) bank122194 +122195 POINT(41.39280711828532 73.34147540726362) bank122195 +122196 POINT(41.49824399206227 74.07642416639929) bank122196 +122197 POINT(40.016526591556506 73.14407851913973) bank122197 +122198 POINT(40.74978972254066 74.60003327079566) bank122198 +122199 POINT(41.26810329630036 74.08270560469279) bank122199 +122200 POINT(40.567918315310436 74.02251954838819) bank122200 +122201 POINT(41.29699628405114 74.59287241901232) bank122201 +122202 POINT(40.73212664920232 73.28855939621259) bank122202 +122203 POINT(41.60169631515618 74.99228438432387) bank122203 +122204 POINT(39.85194131732628 74.87747003224571) bank122204 +122205 POINT(41.51519853575915 74.53989195798194) bank122205 +122206 POINT(40.72311685579475 73.55855015068755) bank122206 +122207 POINT(41.26059371938063 73.84893823636769) bank122207 +122208 POINT(41.6566785440362 73.28764741990169) bank122208 +122209 POINT(39.88490414858013 74.25980614962515) bank122209 +122210 POINT(41.47672934770376 74.8288177078514) bank122210 +122211 POINT(40.5615735894491 73.91754903055994) bank122211 +122212 POINT(40.440903212588225 73.85628411435465) bank122212 +122213 POINT(39.901547762907825 73.80938907321736) bank122213 +122214 POINT(40.86636113490449 74.24760207148746) bank122214 +122215 POINT(41.40611047280438 73.95084134538591) bank122215 +122216 POINT(40.52957419552475 74.59987488678362) bank122216 +122217 POINT(39.84039272738843 73.32148757175139) bank122217 +122218 POINT(40.6078659473345 73.2892910909594) bank122218 +122219 POINT(41.32934931819589 73.06921304247054) bank122219 +122220 POINT(41.229722029300795 74.40725092176766) bank122220 +122221 POINT(40.0553865867759 73.49502940598303) bank122221 +122222 POINT(41.406158301245945 74.8229333497166) bank122222 +122223 POINT(40.876479259269466 73.0808638231903) bank122223 +122224 POINT(40.49219378863198 73.03597501398622) bank122224 +122225 POINT(39.818432105065774 74.98262820740905) bank122225 +122226 POINT(39.79714358050382 74.92321148113719) bank122226 +122227 POINT(40.76876911220865 74.30822544178004) bank122227 +122228 POINT(40.31117626027471 74.1120557463188) bank122228 +122229 POINT(41.16175537959304 74.29651974019475) bank122229 +122230 POINT(40.32301596824618 74.49120724783286) bank122230 +122231 POINT(41.135939579672126 73.03486584175045) bank122231 +122232 POINT(40.343311043208914 73.70686152960373) bank122232 +122233 POINT(41.459600761564026 74.0937056242729) bank122233 +122234 POINT(41.27761332445303 74.2198913717549) bank122234 +122235 POINT(39.74121607761639 73.87320383077288) bank122235 +122236 POINT(41.498025189040085 74.05067308669668) bank122236 +122237 POINT(41.24828115564935 74.55460289450112) bank122237 +122238 POINT(41.38827267269693 73.31567446062276) bank122238 +122239 POINT(41.39063445726073 73.99374643826927) bank122239 +122240 POINT(41.001955513297695 73.85429190780218) bank122240 +122241 POINT(41.501635443163636 73.98441224886534) bank122241 +122242 POINT(40.35099767152214 73.95803591285527) bank122242 +122243 POINT(40.12815970266702 73.3955434869841) bank122243 +122244 POINT(40.92761014559227 74.08276700174694) bank122244 +122245 POINT(41.052369159119095 74.72642083037803) bank122245 +122246 POINT(40.74341842622775 74.36191750858396) bank122246 +122247 POINT(40.864965150409915 73.20873206827785) bank122247 +122248 POINT(40.74175321761108 74.9925737294818) bank122248 +122249 POINT(39.92318816248019 74.79245809689095) bank122249 +122250 POINT(41.22147004468184 73.4292220141799) bank122250 +122251 POINT(40.67898140483157 74.80704776888436) bank122251 +122252 POINT(40.346414070543496 73.09742392442476) bank122252 +122253 POINT(39.78593521334484 74.57975762744195) bank122253 +122254 POINT(40.07864153693198 73.07696586552673) bank122254 +122255 POINT(40.49342937006814 73.190640955613) bank122255 +122256 POINT(41.233055566471656 73.48431706206681) bank122256 +122257 POINT(41.62153934219486 73.40174240903487) bank122257 +122258 POINT(39.95045437080469 73.07547927808707) bank122258 +122259 POINT(40.4634060669404 74.92162795296986) bank122259 +122260 POINT(40.25737221987157 73.5782473999121) bank122260 +122261 POINT(41.305706001681344 73.21502425691469) bank122261 +122262 POINT(39.949516728835405 74.87122728537295) bank122262 +122263 POINT(40.25305386671119 73.44237772224072) bank122263 +122264 POINT(41.38828001031001 73.91511282389301) bank122264 +122265 POINT(41.40951765121023 74.37012106208815) bank122265 +122266 POINT(39.79777420594148 74.90957233458909) bank122266 +122267 POINT(41.041687759087175 74.57897207171301) bank122267 +122268 POINT(41.10366625462144 73.17771932073518) bank122268 +122269 POINT(40.434216812729055 74.8385286239676) bank122269 +122270 POINT(41.103622177433124 74.81112194072183) bank122270 +122271 POINT(39.96377138930876 74.57366669951658) bank122271 +122272 POINT(40.8125331164137 73.20500497906103) bank122272 +122273 POINT(41.03181495175816 74.98783705181522) bank122273 +122274 POINT(41.11939791025296 74.76255670070111) bank122274 +122275 POINT(41.237265154295216 74.72329336529364) bank122275 +122276 POINT(41.64082433284991 74.39463792627637) bank122276 +122277 POINT(39.96087584234368 74.92471812581618) bank122277 +122278 POINT(40.8979894070324 74.21601421889241) bank122278 +122279 POINT(40.48590662416059 74.94753556576416) bank122279 +122280 POINT(40.656879136199095 73.64323149194621) bank122280 +122281 POINT(40.52934590847666 73.51864884319636) bank122281 +122282 POINT(40.626163945798034 73.10193668207167) bank122282 +122283 POINT(40.03693842332592 73.53072065201378) bank122283 +122284 POINT(40.91827760856542 73.5153455442804) bank122284 +122285 POINT(40.67745732928654 73.9550442660233) bank122285 +122286 POINT(40.951559066819065 73.99982148553451) bank122286 +122287 POINT(39.9383737770402 74.9810578340394) bank122287 +122288 POINT(40.310014525496605 74.3433281831284) bank122288 +122289 POINT(40.21576063610875 74.45249594316692) bank122289 +122290 POINT(41.1418654794765 73.97979860325857) bank122290 +122291 POINT(40.63370060882319 73.52597412085612) bank122291 +122292 POINT(41.599596622398444 73.48886924101295) bank122292 +122293 POINT(40.960490283578366 73.57252959760623) bank122293 +122294 POINT(41.27319011967242 73.84458922088751) bank122294 +122295 POINT(40.46443551292914 74.12950128107956) bank122295 +122296 POINT(40.61906578489394 74.47679464510833) bank122296 +122297 POINT(41.54319022874001 74.77777673593181) bank122297 +122298 POINT(41.0779802409446 73.28878602446746) bank122298 +122299 POINT(40.55412187586643 74.85714944391822) bank122299 +122300 POINT(40.09446525526195 74.78495948466193) bank122300 +122301 POINT(40.85205511292559 73.66473788362204) bank122301 +122302 POINT(40.082176759215244 73.16223415164181) bank122302 +122303 POINT(40.49743481284751 74.05035461016548) bank122303 +122304 POINT(40.301428361218484 73.62922116752254) bank122304 +122305 POINT(40.402422604216426 73.90943772427485) bank122305 +122306 POINT(41.63730039629558 73.87464542650729) bank122306 +122307 POINT(39.88794510579648 73.19595723903662) bank122307 +122308 POINT(39.859435375126345 74.85724389380843) bank122308 +122309 POINT(40.15226028150877 73.43697107654758) bank122309 +122310 POINT(41.654857512592415 73.42020248150483) bank122310 +122311 POINT(41.32631794731335 74.75862455937197) bank122311 +122312 POINT(40.67733352141201 73.53464745950642) bank122312 +122313 POINT(40.750093709964574 73.23334723436876) bank122313 +122314 POINT(40.99038390984071 74.72815571515119) bank122314 +122315 POINT(40.21510060775931 74.20722107462198) bank122315 +122316 POINT(40.275864207571786 73.89721231730981) bank122316 +122317 POINT(40.82514618185787 74.80874924726587) bank122317 +122318 POINT(41.08991816912113 73.02104857649982) bank122318 +122319 POINT(41.41705655230194 73.21322201332117) bank122319 +122320 POINT(40.77206961606571 74.84946631276256) bank122320 +122321 POINT(40.21022431877408 74.15876474752058) bank122321 +122322 POINT(41.70662009071387 73.66978633280084) bank122322 +122323 POINT(40.09304648797905 73.55433647430763) bank122323 +122324 POINT(41.56141534165216 74.51370337623335) bank122324 +122325 POINT(39.79884873782719 74.10010898628153) bank122325 +122326 POINT(40.0533530245676 74.32481487359411) bank122326 +122327 POINT(41.32504961913714 73.21669405827305) bank122327 +122328 POINT(40.628424800895495 73.50650977456992) bank122328 +122329 POINT(41.173623625764904 73.31815083594755) bank122329 +122330 POINT(41.13145179781033 73.8176783376692) bank122330 +122331 POINT(41.215509519938294 73.36096803137232) bank122331 +122332 POINT(41.48032751657075 74.71803509870126) bank122332 +122333 POINT(39.810392963268455 73.24728219630933) bank122333 +122334 POINT(40.51293972799908 73.48520763572425) bank122334 +122335 POINT(41.30694302458722 73.3556806025805) bank122335 +122336 POINT(40.41329045532476 73.71660995468895) bank122336 +122337 POINT(40.561459327928645 73.45493777609089) bank122337 +122338 POINT(40.309564952653446 74.44568465565446) bank122338 +122339 POINT(40.06207040040683 73.79685073522866) bank122339 +122340 POINT(41.255629433740815 73.63470632596506) bank122340 +122341 POINT(41.61962942020248 74.01895859147804) bank122341 +122342 POINT(41.633882133144056 74.98313930970751) bank122342 +122343 POINT(41.59578750366003 73.9970246554895) bank122343 +122344 POINT(40.91673203633813 74.36655675809973) bank122344 +122345 POINT(40.229257187442 73.54723315943602) bank122345 +122346 POINT(41.678153104703156 73.17173522409676) bank122346 +122347 POINT(40.52225983442473 74.587927484632) bank122347 +122348 POINT(39.99363510457769 73.71158669221803) bank122348 +122349 POINT(40.5380778763288 74.85904295000778) bank122349 +122350 POINT(40.26353010105297 73.72513936563051) bank122350 +122351 POINT(40.08345990439804 73.36637036989245) bank122351 +122352 POINT(41.423112851868396 74.31062404988043) bank122352 +122353 POINT(41.2906948638491 74.14355672358518) bank122353 +122354 POINT(39.90902949516854 73.37212787456511) bank122354 +122355 POINT(41.354572866819716 74.77335478915576) bank122355 +122356 POINT(40.5984272974088 73.48208059955141) bank122356 +122357 POINT(41.232559447172555 73.0793521395282) bank122357 +122358 POINT(40.63773518796904 73.73826695103558) bank122358 +122359 POINT(40.78583395041961 73.95216538565148) bank122359 +122360 POINT(40.54549091170289 74.36431911850205) bank122360 +122361 POINT(40.32501470854585 73.8563971765002) bank122361 +122362 POINT(41.30189604065447 74.61984103752958) bank122362 +122363 POINT(40.688342932639806 73.04341740535902) bank122363 +122364 POINT(40.67427501213545 74.71454454981094) bank122364 +122365 POINT(41.3972440381076 73.06110481471961) bank122365 +122366 POINT(40.872614082641476 73.66622560667247) bank122366 +122367 POINT(40.51522420444308 73.60606345619823) bank122367 +122368 POINT(41.09713437252579 74.78771283092395) bank122368 +122369 POINT(40.694385419938875 73.91909721434708) bank122369 +122370 POINT(40.34489172657849 73.75793561468899) bank122370 +122371 POINT(40.09422359237857 74.6465519790354) bank122371 +122372 POINT(41.685291795308665 74.39818114872767) bank122372 +122373 POINT(40.252905020728896 73.71980606618031) bank122373 +122374 POINT(40.755176077106206 74.01290979021071) bank122374 +122375 POINT(41.23897908638283 73.77254947816593) bank122375 +122376 POINT(40.1840389469434 73.36459263420775) bank122376 +122377 POINT(40.3644426580652 74.0000335064838) bank122377 +122378 POINT(40.443099324723825 73.31423631632921) bank122378 +122379 POINT(40.45348427797377 73.72043835416524) bank122379 +122380 POINT(40.55984037505774 73.81375742013603) bank122380 +122381 POINT(41.23868284104004 73.96189220916048) bank122381 +122382 POINT(41.06556027440131 73.81301430001739) bank122382 +122383 POINT(40.589412535945456 74.94749336159852) bank122383 +122384 POINT(39.949752540371065 74.70829148893381) bank122384 +122385 POINT(41.68042981121269 73.44596237601121) bank122385 +122386 POINT(41.0404404474816 73.79183052078662) bank122386 +122387 POINT(39.73817284997672 74.63285640664495) bank122387 +122388 POINT(40.175582752401915 74.4567994267732) bank122388 +122389 POINT(40.76285524421861 73.93006933863991) bank122389 +122390 POINT(39.904420653444305 74.38074964338391) bank122390 +122391 POINT(40.42902571213689 74.12420678876018) bank122391 +122392 POINT(40.39155351764026 73.81849674151562) bank122392 +122393 POINT(40.7545541573323 74.75951337182588) bank122393 +122394 POINT(40.41422447622417 74.52208755337732) bank122394 +122395 POINT(40.62721273243697 73.19498195893965) bank122395 +122396 POINT(40.138441762637406 74.41324869990594) bank122396 +122397 POINT(40.82124343683365 73.37750872212715) bank122397 +122398 POINT(39.820723242383714 74.31811770380823) bank122398 +122399 POINT(41.58188493100215 74.87729189590569) bank122399 +122400 POINT(40.48225607907053 73.42298372945581) bank122400 +122401 POINT(40.90336221315263 74.1392425635763) bank122401 +122402 POINT(40.89556726748154 73.44265601280807) bank122402 +122403 POINT(40.99679854510943 73.27362234249752) bank122403 +122404 POINT(41.503828865985454 74.22017891545481) bank122404 +122405 POINT(40.81581813625954 73.88481858289553) bank122405 +122406 POINT(39.77477520752683 73.63882855103424) bank122406 +122407 POINT(39.95354973677659 73.07976573426384) bank122407 +122408 POINT(40.23524615472773 73.17109520563346) bank122408 +122409 POINT(40.23187966565475 74.64584894752677) bank122409 +122410 POINT(41.1980940495036 74.74681042538904) bank122410 +122411 POINT(39.73124392549968 73.49545175301658) bank122411 +122412 POINT(40.174905896379485 73.89421392573763) bank122412 +122413 POINT(40.12255159376386 74.32862396225062) bank122413 +122414 POINT(41.1740138702019 73.26377710113182) bank122414 +122415 POINT(40.44714204141523 74.90610481894477) bank122415 +122416 POINT(39.89679900678808 74.44713063620571) bank122416 +122417 POINT(40.58849973097923 73.20709885458591) bank122417 +122418 POINT(40.19902968259871 74.44434479179625) bank122418 +122419 POINT(40.887075815278635 74.01596826465858) bank122419 +122420 POINT(40.09514503523788 73.60969193909558) bank122420 +122421 POINT(40.68797844019169 73.2658146256552) bank122421 +122422 POINT(40.14817550793381 74.1732404040666) bank122422 +122423 POINT(40.0280846593555 73.01733351358718) bank122423 +122424 POINT(40.47085091321577 74.0186522085188) bank122424 +122425 POINT(41.6072676194949 73.96153458648153) bank122425 +122426 POINT(40.298747982487164 74.5440207039205) bank122426 +122427 POINT(40.65055316042192 73.59781026804393) bank122427 +122428 POINT(40.89591750902441 73.5320304435259) bank122428 +122429 POINT(40.57938620443154 73.60706773906885) bank122429 +122430 POINT(41.33486546119367 74.07781105483544) bank122430 +122431 POINT(39.79288285809944 73.43268606560451) bank122431 +122432 POINT(41.407716926303834 74.00873674203305) bank122432 +122433 POINT(39.85687240288 74.06420636175218) bank122433 +122434 POINT(41.66987594356949 74.98197341859041) bank122434 +122435 POINT(41.4408731573918 74.67839854545439) bank122435 +122436 POINT(40.29726075687751 74.5638982148727) bank122436 +122437 POINT(40.889203364741974 74.43839529359248) bank122437 +122438 POINT(40.3718657550268 74.31463955538752) bank122438 +122439 POINT(41.70502383815609 74.47090868660612) bank122439 +122440 POINT(40.88636692024668 74.61987744624366) bank122440 +122441 POINT(40.53179217369859 73.3986164284503) bank122441 +122442 POINT(40.18850853828302 73.75192172245887) bank122442 +122443 POINT(40.23423908049022 73.94961880013119) bank122443 +122444 POINT(40.24829288698439 73.52862056648016) bank122444 +122445 POINT(41.163957601724846 73.60078680492688) bank122445 +122446 POINT(41.6196419981013 74.8560287255735) bank122446 +122447 POINT(40.358314737929746 74.29429996664837) bank122447 +122448 POINT(40.438102457960774 74.20236929174202) bank122448 +122449 POINT(40.661366834212224 73.24411432934674) bank122449 +122450 POINT(41.42087139990539 73.58089211316036) bank122450 +122451 POINT(41.05587144918505 74.2013253390235) bank122451 +122452 POINT(40.84540467218766 74.33636837349408) bank122452 +122453 POINT(40.81526220010833 74.75752717041524) bank122453 +122454 POINT(40.09439168625618 73.01187858515677) bank122454 +122455 POINT(41.493384407116366 74.1641628398339) bank122455 +122456 POINT(40.86526325226312 74.3640692387015) bank122456 +122457 POINT(40.44053375227772 73.70289982454113) bank122457 +122458 POINT(40.05509238977692 74.3509442080352) bank122458 +122459 POINT(40.77005930775 73.11917698789598) bank122459 +122460 POINT(40.34918814369819 74.34213410731394) bank122460 +122461 POINT(40.610813339350884 74.69676279615643) bank122461 +122462 POINT(41.2739694987194 73.9585298039233) bank122462 +122463 POINT(41.40069352346975 74.68901543737897) bank122463 +122464 POINT(39.95765399298923 73.05159942696697) bank122464 +122465 POINT(40.2597761879117 73.05565403290366) bank122465 +122466 POINT(40.423568664233066 74.59778253259829) bank122466 +122467 POINT(39.77100172308905 74.06775650918773) bank122467 +122468 POINT(40.30692192170108 74.18470585338443) bank122468 +122469 POINT(41.46459647654263 74.63274400265738) bank122469 +122470 POINT(40.366810207964186 74.81629897634416) bank122470 +122471 POINT(41.215316002735065 74.4620613665567) bank122471 +122472 POINT(40.38253027909737 73.85046006156477) bank122472 +122473 POINT(41.54511717577009 73.7139156547819) bank122473 +122474 POINT(41.40322644141495 73.06266026784768) bank122474 +122475 POINT(39.926208514372895 74.26501989474696) bank122475 +122476 POINT(40.27221581646015 73.75684778857229) bank122476 +122477 POINT(40.635964709699536 74.08059542588664) bank122477 +122478 POINT(41.450122368346214 74.13535956742065) bank122478 +122479 POINT(41.59633261612406 74.1294957050925) bank122479 +122480 POINT(41.34353064709858 74.43961029145385) bank122480 +122481 POINT(41.398676319863505 73.7685265090317) bank122481 +122482 POINT(40.071117103726785 74.64700067386406) bank122482 +122483 POINT(41.14681073748788 73.9441138520485) bank122483 +122484 POINT(40.904726960974294 74.485641451724) bank122484 +122485 POINT(39.76827216680593 74.26979090464624) bank122485 +122486 POINT(40.355623998289886 74.16592533802681) bank122486 +122487 POINT(39.77112563978198 73.02390803148462) bank122487 +122488 POINT(41.07297940918491 74.44952592773457) bank122488 +122489 POINT(40.35419454834566 73.81576887701469) bank122489 +122490 POINT(39.76494233407906 73.43531858402844) bank122490 +122491 POINT(39.97149902979875 74.48922104713273) bank122491 +122492 POINT(41.332368148803106 73.41940774343797) bank122492 +122493 POINT(40.62868413038534 73.43089208981422) bank122493 +122494 POINT(41.10642092009526 73.12304213257656) bank122494 +122495 POINT(40.13984891939489 73.75932483479916) bank122495 +122496 POINT(41.149041403432754 74.7915028949802) bank122496 +122497 POINT(41.53646255190576 74.47942644611621) bank122497 +122498 POINT(40.085744040284524 73.33938346651203) bank122498 +122499 POINT(40.721497666724964 74.2742122505737) bank122499 +122500 POINT(40.09429911216743 74.74455515530607) bank122500 +122501 POINT(40.881708187068824 74.54781001677695) bank122501 +122502 POINT(40.33163107965346 74.65424364325563) bank122502 +122503 POINT(41.37686013740051 73.95004320209503) bank122503 +122504 POINT(41.59265915663071 74.62395962560105) bank122504 +122505 POINT(41.116612595868794 73.84725119519628) bank122505 +122506 POINT(39.85366705086915 73.56537273234154) bank122506 +122507 POINT(41.620173444462424 73.80813375685543) bank122507 +122508 POINT(41.49312180613656 73.8164488392452) bank122508 +122509 POINT(40.08747746182516 73.62141557354406) bank122509 +122510 POINT(40.77612871461504 74.75672964834396) bank122510 +122511 POINT(39.90175202787777 73.02532104374855) bank122511 +122512 POINT(39.80778836480669 73.0406343515781) bank122512 +122513 POINT(41.05096659836434 73.77622911278212) bank122513 +122514 POINT(41.01476964291141 74.88687616962638) bank122514 +122515 POINT(40.79141132441896 73.41170861007932) bank122515 +122516 POINT(40.42219113389827 73.13591369199594) bank122516 +122517 POINT(40.26890168822435 73.42117957233532) bank122517 +122518 POINT(40.415326609062916 74.59433058819097) bank122518 +122519 POINT(41.42644778744367 74.82077144502672) bank122519 +122520 POINT(40.296085114520984 74.35145538237965) bank122520 +122521 POINT(41.449202263786766 73.49985305003142) bank122521 +122522 POINT(39.8058507839019 73.72043176343176) bank122522 +122523 POINT(40.27858383789515 74.50163683956961) bank122523 +122524 POINT(41.57421759553001 74.30833794384854) bank122524 +122525 POINT(41.503096385051904 74.68365812266805) bank122525 +122526 POINT(41.11505704200291 73.40790103045998) bank122526 +122527 POINT(40.41937859289677 74.72621681380683) bank122527 +122528 POINT(40.976347184368436 73.86498857227527) bank122528 +122529 POINT(40.65665166050955 74.87744701650125) bank122529 +122530 POINT(41.49561699757613 74.84895681728646) bank122530 +122531 POINT(40.10452568288425 73.19296240575113) bank122531 +122532 POINT(41.43081502624942 74.32244560376172) bank122532 +122533 POINT(40.363685929758475 73.35609005816801) bank122533 +122534 POINT(40.76267292973731 74.22844336635268) bank122534 +122535 POINT(39.95681957308393 74.07193596690743) bank122535 +122536 POINT(40.59772590047178 73.34375064520512) bank122536 +122537 POINT(40.70375552594901 73.81923848520506) bank122537 +122538 POINT(40.90891527188204 74.86343995917531) bank122538 +122539 POINT(40.56726581103797 73.68061225043468) bank122539 +122540 POINT(39.96740742843721 73.88686097440576) bank122540 +122541 POINT(40.978469025849265 74.19356261338194) bank122541 +122542 POINT(40.9913443895726 73.52109882420085) bank122542 +122543 POINT(40.73581826378718 74.74293516111545) bank122543 +122544 POINT(40.0889832828615 73.80269754517073) bank122544 +122545 POINT(40.08937252065895 73.09576001615439) bank122545 +122546 POINT(40.71762965602633 74.29964843369689) bank122546 +122547 POINT(39.94378923513725 74.95386044108388) bank122547 +122548 POINT(41.70346437283694 74.97046649179796) bank122548 +122549 POINT(40.22624034637963 74.85578805179584) bank122549 +122550 POINT(40.99462138863885 73.5479348101487) bank122550 +122551 POINT(41.00106881662527 73.18219453478687) bank122551 +122552 POINT(41.62176444866541 73.07530450035625) bank122552 +122553 POINT(41.05027951439147 73.13166492028081) bank122553 +122554 POINT(40.353399645022485 74.70056204375634) bank122554 +122555 POINT(40.032555527589736 74.94053312890604) bank122555 +122556 POINT(41.3614909995749 73.27893566214219) bank122556 +122557 POINT(40.04396056251572 73.91071267373323) bank122557 +122558 POINT(41.65921722190415 73.19501632477817) bank122558 +122559 POINT(41.310490369762164 73.34505559278058) bank122559 +122560 POINT(40.76486784001997 73.57027326408614) bank122560 +122561 POINT(41.42693201421135 73.93398413710162) bank122561 +122562 POINT(40.62297247584423 74.501802983999) bank122562 +122563 POINT(40.73011426129765 74.11161554623943) bank122563 +122564 POINT(40.322981764590594 74.34002340545119) bank122564 +122565 POINT(41.650103678932894 73.27285927813936) bank122565 +122566 POINT(41.275124533016374 73.11292494743286) bank122566 +122567 POINT(40.74483919867936 73.881971837627) bank122567 +122568 POINT(40.51831741834559 74.5860969267097) bank122568 +122569 POINT(41.06584478843101 74.38095976780502) bank122569 +122570 POINT(41.021470169418535 74.65388544755727) bank122570 +122571 POINT(40.62378794556528 74.74614026113481) bank122571 +122572 POINT(39.89329213035813 73.21034372163878) bank122572 +122573 POINT(40.37206563038454 73.62744777200359) bank122573 +122574 POINT(39.90805790544147 73.63010083961919) bank122574 +122575 POINT(41.34565562288008 74.67545618699253) bank122575 +122576 POINT(40.9559241538465 73.46798471680704) bank122576 +122577 POINT(41.37085570790928 74.06502970457034) bank122577 +122578 POINT(39.93051705212265 74.33026553688656) bank122578 +122579 POINT(40.49668215357137 73.3616926303109) bank122579 +122580 POINT(41.37034625108691 73.9572228034719) bank122580 +122581 POINT(40.52416414180329 74.35906880173111) bank122581 +122582 POINT(40.277023039665174 74.86524234349083) bank122582 +122583 POINT(40.77098798336225 73.5624349228455) bank122583 +122584 POINT(40.27676136346252 74.2561721958863) bank122584 +122585 POINT(40.08914693654248 74.18318681553585) bank122585 +122586 POINT(39.76939449795441 74.67468507690602) bank122586 +122587 POINT(40.03472002304932 74.3381791447518) bank122587 +122588 POINT(40.184602024789385 73.30227434274852) bank122588 +122589 POINT(40.04310790582746 74.48360109694202) bank122589 +122590 POINT(39.8251393301589 73.81029442199814) bank122590 +122591 POINT(40.62979947822675 74.68606024122968) bank122591 +122592 POINT(40.071726302560805 73.43210267874917) bank122592 +122593 POINT(41.6471943813805 74.00736645529688) bank122593 +122594 POINT(40.140711535590185 74.1931736868205) bank122594 +122595 POINT(41.20311929651998 73.04316334874247) bank122595 +122596 POINT(39.74810703716713 74.0182631044218) bank122596 +122597 POINT(39.864068535399994 74.6989536614007) bank122597 +122598 POINT(41.20000706012053 74.20522270744999) bank122598 +122599 POINT(39.929083013180936 74.06010815417991) bank122599 +122600 POINT(40.716974316972426 74.0300949147046) bank122600 +122601 POINT(40.60619882397464 74.60598704164138) bank122601 +122602 POINT(40.0515858076193 74.9528407513162) bank122602 +122603 POINT(41.086459095052966 74.16969689567567) bank122603 +122604 POINT(40.35752463557573 73.74335836189434) bank122604 +122605 POINT(40.31507507110998 73.14630609504309) bank122605 +122606 POINT(41.41129555829571 74.81462229919147) bank122606 +122607 POINT(40.76883191477562 74.87039514261463) bank122607 +122608 POINT(41.12350531914207 73.86367433554332) bank122608 +122609 POINT(41.582930208664976 73.08871612407071) bank122609 +122610 POINT(40.80399593845713 74.02956876431739) bank122610 +122611 POINT(40.98582527862881 73.27540030397545) bank122611 +122612 POINT(40.9994096263726 73.6602247770425) bank122612 +122613 POINT(41.671944102886236 73.10555155240381) bank122613 +122614 POINT(41.01140450022692 74.27581892525708) bank122614 +122615 POINT(40.378393401767156 73.15157667231551) bank122615 +122616 POINT(40.09619094635845 74.81130891907998) bank122616 +122617 POINT(40.46642476073375 73.39565747655469) bank122617 +122618 POINT(40.21020801281482 73.29012921236021) bank122618 +122619 POINT(40.227333189244796 74.00732393848486) bank122619 +122620 POINT(40.86426688292252 73.82835578561237) bank122620 +122621 POINT(41.41148976557778 74.5387311372416) bank122621 +122622 POINT(40.34470260307447 73.20382313394421) bank122622 +122623 POINT(41.02447742943461 73.81652797586948) bank122623 +122624 POINT(40.65052080659686 74.14239795325325) bank122624 +122625 POINT(39.80301501060914 74.01227366883926) bank122625 +122626 POINT(41.59376284878815 73.25146750240914) bank122626 +122627 POINT(39.75572913222909 73.52094656280991) bank122627 +122628 POINT(40.82438777645136 74.77602669916239) bank122628 +122629 POINT(40.23802435244794 74.49220572453544) bank122629 +122630 POINT(41.42429353746053 73.23994213916951) bank122630 +122631 POINT(39.79161259305925 74.5854478034299) bank122631 +122632 POINT(41.14774677969785 73.66296821149456) bank122632 +122633 POINT(41.69234034324334 73.04149991713209) bank122633 +122634 POINT(40.60363647002078 74.21649089026158) bank122634 +122635 POINT(40.166685970431 73.05692926047853) bank122635 +122636 POINT(40.68986205167545 74.81654245726841) bank122636 +122637 POINT(41.31425510201524 74.69522370791678) bank122637 +122638 POINT(39.88612690430797 73.04347064516035) bank122638 +122639 POINT(39.72734877840159 74.01880092456707) bank122639 +122640 POINT(41.05271212716235 74.87043385421588) bank122640 +122641 POINT(41.633453176179415 73.36407185921408) bank122641 +122642 POINT(41.42203491035714 73.27885877257656) bank122642 +122643 POINT(40.32886045573139 73.02882373874311) bank122643 +122644 POINT(40.7923588668261 73.83966088384378) bank122644 +122645 POINT(41.52586565714477 74.12512114798447) bank122645 +122646 POINT(40.90684371538736 74.06959015686311) bank122646 +122647 POINT(41.53208745326463 74.41054974035323) bank122647 +122648 POINT(41.65194157604024 74.34486749410821) bank122648 +122649 POINT(41.39220481712773 74.75276118588793) bank122649 +122650 POINT(41.26937783523607 73.70819025932573) bank122650 +122651 POINT(41.22726396082455 73.23315654483002) bank122651 +122652 POINT(39.97450067788857 73.52036892562559) bank122652 +122653 POINT(40.426149489056755 73.06104358708836) bank122653 +122654 POINT(40.759233241110906 74.39324841273942) bank122654 +122655 POINT(41.24940575300119 74.90096865717696) bank122655 +122656 POINT(39.8579543085872 74.62921688495489) bank122656 +122657 POINT(40.69992186008941 73.43172506769031) bank122657 +122658 POINT(40.501497464029114 74.57973567079026) bank122658 +122659 POINT(40.28951924653523 73.19506240712296) bank122659 +122660 POINT(40.454441384364905 74.16792836884898) bank122660 +122661 POINT(41.56190103127069 73.58822632486162) bank122661 +122662 POINT(40.10557458704989 74.21392048650759) bank122662 +122663 POINT(41.644920528238984 74.32950485389118) bank122663 +122664 POINT(41.02247872198657 73.27778239301743) bank122664 +122665 POINT(39.85292497238012 73.87165436698095) bank122665 +122666 POINT(40.553864344875514 73.85445964286673) bank122666 +122667 POINT(39.75038968910461 74.27873149820628) bank122667 +122668 POINT(41.44800479670012 74.29268397049351) bank122668 +122669 POINT(40.81437494787864 73.92937609550673) bank122669 +122670 POINT(40.48841093778315 74.44330539594641) bank122670 +122671 POINT(40.6372765540376 74.88979765063095) bank122671 +122672 POINT(40.833979129680664 73.45178516702818) bank122672 +122673 POINT(41.15413662428081 74.36492321935033) bank122673 +122674 POINT(41.64260107062984 73.41239878151703) bank122674 +122675 POINT(41.66294248242401 73.53612606791995) bank122675 +122676 POINT(41.016757531110365 73.29596373795044) bank122676 +122677 POINT(40.56380609396945 74.70500234767141) bank122677 +122678 POINT(40.486089130804004 73.09460155546103) bank122678 +122679 POINT(41.009910810485266 74.81485706761298) bank122679 +122680 POINT(41.63766096882915 74.0499733691268) bank122680 +122681 POINT(40.78789515122702 74.1852156802788) bank122681 +122682 POINT(41.71134598135241 73.3284176436756) bank122682 +122683 POINT(41.70390499703073 74.07862955528074) bank122683 +122684 POINT(40.68841431039008 74.04101606134681) bank122684 +122685 POINT(41.37986138780892 73.2597128079899) bank122685 +122686 POINT(40.499885447544045 74.86405719633896) bank122686 +122687 POINT(41.31615322971066 74.3943891355425) bank122687 +122688 POINT(41.18430814569587 74.85742141238755) bank122688 +122689 POINT(41.213810944444205 73.01788677429882) bank122689 +122690 POINT(40.936003740427296 73.90006343073654) bank122690 +122691 POINT(41.3567273487686 73.14236257142237) bank122691 +122692 POINT(41.299256197302405 73.98383153470627) bank122692 +122693 POINT(41.3829312255329 74.64839203289307) bank122693 +122694 POINT(41.26792272076749 74.21790254408505) bank122694 +122695 POINT(40.72315279474077 73.26996690387818) bank122695 +122696 POINT(41.64096189275455 74.47360254090734) bank122696 +122697 POINT(39.80332946726783 74.16133615023267) bank122697 +122698 POINT(40.46016059783318 73.11140694556282) bank122698 +122699 POINT(41.00994084383172 74.65370672488575) bank122699 +122700 POINT(39.71475861862721 73.49415075327337) bank122700 +122701 POINT(39.91395276841161 73.95178134700767) bank122701 +122702 POINT(40.91719371685497 73.40818650154058) bank122702 +122703 POINT(41.01638959272544 74.08427060486957) bank122703 +122704 POINT(41.67815728400131 73.46392230276889) bank122704 +122705 POINT(41.17003405342474 73.59057993709297) bank122705 +122706 POINT(40.58285660176949 73.73123526916523) bank122706 +122707 POINT(39.78287958759508 74.91321100299807) bank122707 +122708 POINT(40.446132479344925 74.01354673477704) bank122708 +122709 POINT(40.16512052893621 73.66853336464891) bank122709 +122710 POINT(40.504095670325285 74.04814327096223) bank122710 +122711 POINT(41.54391325159788 74.54271813428437) bank122711 +122712 POINT(40.84401077906805 73.06951986308) bank122712 +122713 POINT(40.457733802886196 73.57133272090434) bank122713 +122714 POINT(40.34797719051455 74.71378828586484) bank122714 +122715 POINT(41.03396654024525 73.25019593316718) bank122715 +122716 POINT(40.64084526338053 74.25373126540552) bank122716 +122717 POINT(41.082773180986386 74.5376256428792) bank122717 +122718 POINT(40.86764802864076 73.07815718653518) bank122718 +122719 POINT(39.92306053689648 73.83337829840501) bank122719 +122720 POINT(41.347718362685164 74.15878786963982) bank122720 +122721 POINT(41.27962448801084 74.73611622618651) bank122721 +122722 POINT(40.02950002860347 73.8025116630397) bank122722 +122723 POINT(41.450146501054554 74.27959787350453) bank122723 +122724 POINT(40.23382883532277 74.84343537003167) bank122724 +122725 POINT(41.393867875658295 74.77896812387567) bank122725 +122726 POINT(40.500306454848506 73.84618628365675) bank122726 +122727 POINT(40.294944261411096 73.68827673153756) bank122727 +122728 POINT(40.10297223418206 74.55697789494808) bank122728 +122729 POINT(40.66200953943514 74.62586437179836) bank122729 +122730 POINT(41.06603263410974 74.09502104861893) bank122730 +122731 POINT(40.79561179561248 74.67695508550551) bank122731 +122732 POINT(39.79797134534573 73.27569182126862) bank122732 +122733 POINT(40.757432496078174 73.58716110455002) bank122733 +122734 POINT(40.13459507386026 74.41063850360577) bank122734 +122735 POINT(41.33405604450432 74.2242462803017) bank122735 +122736 POINT(41.450764021643224 74.60970311827602) bank122736 +122737 POINT(41.243386025023604 74.46418682102004) bank122737 +122738 POINT(40.809954304628135 73.14892811363458) bank122738 +122739 POINT(39.91090892405231 73.14666488809985) bank122739 +122740 POINT(41.60700969673415 74.02206075400306) bank122740 +122741 POINT(40.53974146269641 74.35701272656617) bank122741 +122742 POINT(40.83135058641787 73.81236618527237) bank122742 +122743 POINT(41.53313575157005 73.87580479754881) bank122743 +122744 POINT(41.286754085026026 73.88729114130331) bank122744 +122745 POINT(41.47998509030725 73.9574369223332) bank122745 +122746 POINT(40.61651641150804 73.90324174955136) bank122746 +122747 POINT(40.56533352292056 74.75605100778758) bank122747 +122748 POINT(40.828731835619344 74.73635045253036) bank122748 +122749 POINT(39.71926676557701 74.90853299845162) bank122749 +122750 POINT(41.107678961136145 74.20735894156664) bank122750 +122751 POINT(41.15651655746314 74.55021077440291) bank122751 +122752 POINT(40.4051041565579 73.66740807690167) bank122752 +122753 POINT(41.52127685031967 73.32105188296316) bank122753 +122754 POINT(41.66418632556466 74.9235500770233) bank122754 +122755 POINT(41.2214474739062 73.32338319729631) bank122755 +122756 POINT(41.39738987217106 74.5370288363613) bank122756 +122757 POINT(40.75102749869847 73.0119226383265) bank122757 +122758 POINT(41.081197209336814 73.7559381396977) bank122758 +122759 POINT(41.00549537918229 74.37472168541069) bank122759 +122760 POINT(41.58124196703084 73.95920063383105) bank122760 +122761 POINT(41.37040773345591 73.06089055779907) bank122761 +122762 POINT(41.65357603190095 74.99814177430672) bank122762 +122763 POINT(40.713810096461714 74.83846441006018) bank122763 +122764 POINT(41.12821319218585 73.80156472548869) bank122764 +122765 POINT(41.29421586063997 74.18577823172816) bank122765 +122766 POINT(40.02324521640671 74.03846065027356) bank122766 +122767 POINT(40.008030339147204 74.15536803157393) bank122767 +122768 POINT(41.08994442786105 74.35445630194228) bank122768 +122769 POINT(40.893602595470334 73.56464438001498) bank122769 +122770 POINT(39.888632185946726 74.37116199412453) bank122770 +122771 POINT(40.77485384627483 74.61246481343102) bank122771 +122772 POINT(41.22637113471689 74.00154379768307) bank122772 +122773 POINT(41.365179668691056 74.53906336595125) bank122773 +122774 POINT(40.741480591234215 73.22318939019446) bank122774 +122775 POINT(40.734086359184666 74.0601876634064) bank122775 +122776 POINT(40.10764527954348 74.07919273341122) bank122776 +122777 POINT(41.710993905603694 73.24995694048056) bank122777 +122778 POINT(40.738506901272 73.64692304378075) bank122778 +122779 POINT(41.29822064408618 74.38044584290799) bank122779 +122780 POINT(40.306092054154306 74.86068022347867) bank122780 +122781 POINT(40.0669222572604 74.8958700537308) bank122781 +122782 POINT(41.24324033343525 73.09144480232665) bank122782 +122783 POINT(39.73924343816585 73.83110507712108) bank122783 +122784 POINT(40.494254086987254 74.05601340941587) bank122784 +122785 POINT(40.69971650304234 74.1726549104726) bank122785 +122786 POINT(41.555083166067114 74.39666392160436) bank122786 +122787 POINT(41.0783713325189 73.25472430103439) bank122787 +122788 POINT(41.01421285466226 73.89252647242114) bank122788 +122789 POINT(41.1131761007447 74.41821570152321) bank122789 +122790 POINT(41.31769815949733 74.62113100496894) bank122790 +122791 POINT(40.67177628941186 74.1661296646813) bank122791 +122792 POINT(41.55663569794635 74.24693060923602) bank122792 +122793 POINT(40.34456213558442 73.42673678462339) bank122793 +122794 POINT(40.11714424644842 73.97021503060544) bank122794 +122795 POINT(41.16838939943395 74.83884187883599) bank122795 +122796 POINT(40.098926721552054 73.38326917180751) bank122796 +122797 POINT(40.14187692798432 74.97304195392827) bank122797 +122798 POINT(41.365874050809886 74.37762172435545) bank122798 +122799 POINT(39.75741768010689 73.85721659302205) bank122799 +122800 POINT(40.58033842467071 73.03064579286254) bank122800 +122801 POINT(40.64481421584465 73.23989998792166) bank122801 +122802 POINT(41.650332892307134 73.19866079108944) bank122802 +122803 POINT(40.18751903073023 73.21558799313367) bank122803 +122804 POINT(40.25794535357784 74.72116651423256) bank122804 +122805 POINT(41.03069570978969 74.79274739052649) bank122805 +122806 POINT(41.09132300992003 74.56254788897907) bank122806 +122807 POINT(40.49549645154626 74.7318956524001) bank122807 +122808 POINT(39.994575999358716 74.02763227486727) bank122808 +122809 POINT(41.418125647850545 73.19140403086703) bank122809 +122810 POINT(40.65527021852372 74.45207136067218) bank122810 +122811 POINT(40.73679894058704 73.595804188368) bank122811 +122812 POINT(40.46233524515613 73.77776416273441) bank122812 +122813 POINT(40.95861984246278 73.6820490011465) bank122813 +122814 POINT(40.0806224339873 74.57786356957185) bank122814 +122815 POINT(41.182165580644 74.44072557832213) bank122815 +122816 POINT(40.54973069615415 73.57881453023356) bank122816 +122817 POINT(41.00303131549082 73.98230690285395) bank122817 +122818 POINT(41.27471985768568 74.22761406096605) bank122818 +122819 POINT(39.91307355199151 73.09803820627454) bank122819 +122820 POINT(40.51446270108708 74.9603366948326) bank122820 +122821 POINT(41.13636372279322 73.54044043320641) bank122821 +122822 POINT(40.1938265937147 74.42266238392364) bank122822 +122823 POINT(40.4677357305944 74.56753527094439) bank122823 +122824 POINT(41.33368949570932 73.37311195815947) bank122824 +122825 POINT(40.56227884332766 74.10396734737621) bank122825 +122826 POINT(41.34761751877441 73.94804773648337) bank122826 +122827 POINT(41.08229804451197 73.65953270768209) bank122827 +122828 POINT(41.17550236556155 73.09188514766694) bank122828 +122829 POINT(41.256045555184336 74.30624173100047) bank122829 +122830 POINT(40.42343523584401 74.90841138566029) bank122830 +122831 POINT(41.31547923776043 74.98083333069498) bank122831 +122832 POINT(39.98711824762632 74.50864487470126) bank122832 +122833 POINT(40.33154973402881 73.6695487686619) bank122833 +122834 POINT(39.85470906602108 74.87652495574225) bank122834 +122835 POINT(40.620950944914014 74.01882239251857) bank122835 +122836 POINT(40.935330498641285 74.54214844426764) bank122836 +122837 POINT(40.368135237161376 73.71483477503784) bank122837 +122838 POINT(40.96229945166857 74.38111384138061) bank122838 +122839 POINT(41.6288039592453 74.17384674911143) bank122839 +122840 POINT(40.92138268753751 74.57726002585548) bank122840 +122841 POINT(40.72367539427083 73.18505335214776) bank122841 +122842 POINT(41.61880270274483 73.60184755322688) bank122842 +122843 POINT(41.298438819012716 73.43289250483544) bank122843 +122844 POINT(41.046488438141694 73.66429272121819) bank122844 +122845 POINT(41.27493268029889 74.00523418279668) bank122845 +122846 POINT(40.966888911374866 74.87255630755905) bank122846 +122847 POINT(40.952382740236665 74.30109956677533) bank122847 +122848 POINT(40.32435944412877 74.15957817925539) bank122848 +122849 POINT(40.83918496496632 73.79247555726913) bank122849 +122850 POINT(41.28094144906865 73.13682324147544) bank122850 +122851 POINT(41.3826293182454 75.00083372440304) bank122851 +122852 POINT(40.9016156550873 74.68459099713692) bank122852 +122853 POINT(40.962988973008926 73.79527183131985) bank122853 +122854 POINT(40.20983291321518 74.88052202738568) bank122854 +122855 POINT(40.22994004641004 73.83017929352076) bank122855 +122856 POINT(41.426299684388525 74.73424495356932) bank122856 +122857 POINT(41.38498003399401 74.79456834008114) bank122857 +122858 POINT(39.729011587988495 74.52643434939782) bank122858 +122859 POINT(41.32761831241529 74.78496430299316) bank122859 +122860 POINT(40.33453394175224 74.1888814422237) bank122860 +122861 POINT(39.79217372518591 73.87058697090666) bank122861 +122862 POINT(40.714219863775995 74.33679420339288) bank122862 +122863 POINT(40.23851576981644 74.25959543737721) bank122863 +122864 POINT(41.101253211704154 73.20116637736147) bank122864 +122865 POINT(39.8345244737858 74.20211705119209) bank122865 +122866 POINT(40.36517528558123 74.53266505474657) bank122866 +122867 POINT(40.34066610796004 74.5696005391088) bank122867 +122868 POINT(41.01750255067815 74.67536701858576) bank122868 +122869 POINT(40.137053483631995 73.95755488711346) bank122869 +122870 POINT(40.84249799326735 73.85711587314923) bank122870 +122871 POINT(40.33961611232323 74.84703234008305) bank122871 +122872 POINT(39.93034961028236 74.21328706992621) bank122872 +122873 POINT(40.73122538330473 73.44713540234137) bank122873 +122874 POINT(40.84008448045994 73.89770295012165) bank122874 +122875 POINT(41.67958781831498 73.14948439942602) bank122875 +122876 POINT(40.04498328852463 73.28918167918727) bank122876 +122877 POINT(41.148549345953384 73.45521616608993) bank122877 +122878 POINT(41.68384056482206 73.07156991950366) bank122878 +122879 POINT(40.24389460788096 74.23696616189287) bank122879 +122880 POINT(40.95776528168914 73.90332142822277) bank122880 +122881 POINT(40.40989404896689 73.91476832298744) bank122881 +122882 POINT(41.36355879588764 74.75963461783915) bank122882 +122883 POINT(40.98944913713757 73.37382543209861) bank122883 +122884 POINT(39.7481671182806 74.82256260408975) bank122884 +122885 POINT(41.172044279410436 74.31610184077533) bank122885 +122886 POINT(40.36385443867405 73.21203314728022) bank122886 +122887 POINT(39.717482512563 74.96552381991638) bank122887 +122888 POINT(39.831796105189554 74.13759225005283) bank122888 +122889 POINT(40.00908388689703 74.40679120128858) bank122889 +122890 POINT(40.90353514992623 73.27761382144989) bank122890 +122891 POINT(40.33016846033528 73.53863693062041) bank122891 +122892 POINT(41.30176939051595 73.45517333058628) bank122892 +122893 POINT(39.87200747601236 73.73123357743717) bank122893 +122894 POINT(41.513718606086506 73.14206618728848) bank122894 +122895 POINT(41.674563219183895 73.15287037912314) bank122895 +122896 POINT(40.366584309165 73.20820660065849) bank122896 +122897 POINT(40.36559676768441 74.13111509372497) bank122897 +122898 POINT(40.49239938567952 73.37325526016794) bank122898 +122899 POINT(41.258860365869054 73.77563851857956) bank122899 +122900 POINT(40.43496005733455 74.60447834865921) bank122900 +122901 POINT(41.4569794696424 73.20267197144541) bank122901 +122902 POINT(41.510104887752526 74.28150718399357) bank122902 +122903 POINT(39.88018358654495 74.07030303232506) bank122903 +122904 POINT(41.17988585274501 73.14935367595798) bank122904 +122905 POINT(41.28707463325195 74.42530870820394) bank122905 +122906 POINT(41.12029383820217 74.89564960648214) bank122906 +122907 POINT(41.4654148707345 74.8879594844944) bank122907 +122908 POINT(41.40346461429084 74.68016364936076) bank122908 +122909 POINT(40.02594238905521 73.32078114318459) bank122909 +122910 POINT(40.790909033088845 73.94539205993014) bank122910 +122911 POINT(40.70414623055559 73.99436189989667) bank122911 +122912 POINT(40.401024845560116 74.06564179533255) bank122912 +122913 POINT(41.3984548465246 73.10140538460143) bank122913 +122914 POINT(41.01023097532155 73.22901145970798) bank122914 +122915 POINT(40.019819561909024 74.13964457074385) bank122915 +122916 POINT(40.882495930994516 74.44043037954013) bank122916 +122917 POINT(40.52339796280924 73.08580687032062) bank122917 +122918 POINT(41.088004253378536 74.66154985998763) bank122918 +122919 POINT(40.90286695584157 73.08090251432108) bank122919 +122920 POINT(40.86463199528588 73.49925272847807) bank122920 +122921 POINT(39.84356124875724 73.10491429229421) bank122921 +122922 POINT(40.726519272499345 73.49305248603737) bank122922 +122923 POINT(40.80452209690691 74.78662465637096) bank122923 +122924 POINT(40.358400432062375 74.16335269908663) bank122924 +122925 POINT(41.51392480100783 74.23674621060229) bank122925 +122926 POINT(41.02008350947261 73.94417716835534) bank122926 +122927 POINT(40.644756915923104 73.1764940369759) bank122927 +122928 POINT(40.91504972534917 73.32076367990501) bank122928 +122929 POINT(40.25173054237229 74.54825887717777) bank122929 +122930 POINT(40.61485817651496 74.28725109270017) bank122930 +122931 POINT(40.51279606287098 74.72561092313208) bank122931 +122932 POINT(41.474173781057104 74.25547629196632) bank122932 +122933 POINT(39.80019679029791 73.49349949032622) bank122933 +122934 POINT(40.17617310857184 73.4883732502147) bank122934 +122935 POINT(41.69155877474263 73.72810695922459) bank122935 +122936 POINT(39.86200320344285 73.47471922842585) bank122936 +122937 POINT(41.499025429660264 74.38247450201524) bank122937 +122938 POINT(40.85181306651459 73.00629754953887) bank122938 +122939 POINT(39.983017333409634 74.90197556353515) bank122939 +122940 POINT(40.41596569041579 73.37741377836642) bank122940 +122941 POINT(41.273346753966486 73.27566605721849) bank122941 +122942 POINT(41.197764735809656 73.46387836609203) bank122942 +122943 POINT(40.22966060962305 74.2483150790131) bank122943 +122944 POINT(40.31107421915886 74.48317173813516) bank122944 +122945 POINT(41.05568882936233 74.5415556072718) bank122945 +122946 POINT(41.318426665983424 73.14435364971557) bank122946 +122947 POINT(41.32861200319233 73.65398845512652) bank122947 +122948 POINT(39.99826413777856 74.707650886453) bank122948 +122949 POINT(40.358781100537506 73.26304769901307) bank122949 +122950 POINT(40.37697929244804 74.76213018168042) bank122950 +122951 POINT(40.03483270892183 74.59381023604388) bank122951 +122952 POINT(40.71563140924166 73.29052553306497) bank122952 +122953 POINT(41.291896184884 73.24227899894544) bank122953 +122954 POINT(40.46085118144034 73.56922375002617) bank122954 +122955 POINT(41.69197702679141 74.5786782520628) bank122955 +122956 POINT(39.88795548279151 73.36270243563723) bank122956 +122957 POINT(40.51266955899201 73.74692507485958) bank122957 +122958 POINT(40.325099727990704 73.56903586509037) bank122958 +122959 POINT(40.88778861202619 73.29754145297932) bank122959 +122960 POINT(41.05845110112223 73.87035469258387) bank122960 +122961 POINT(40.699916289592124 73.46050380134133) bank122961 +122962 POINT(39.84987261494276 73.63988714516316) bank122962 +122963 POINT(40.70079694977252 74.33731452532501) bank122963 +122964 POINT(40.25862163733693 73.82457508701195) bank122964 +122965 POINT(41.412914574301155 73.44839280215453) bank122965 +122966 POINT(41.158640502091345 74.79513733663971) bank122966 +122967 POINT(40.55467583833897 73.60360321333943) bank122967 +122968 POINT(40.28855384337398 73.64185403162075) bank122968 +122969 POINT(41.16610926701663 73.34705997416842) bank122969 +122970 POINT(41.18327862297536 74.4008669901838) bank122970 +122971 POINT(40.1694794115184 74.53018887574373) bank122971 +122972 POINT(40.01035938285543 74.64729058425101) bank122972 +122973 POINT(40.817369174360586 74.70832538015426) bank122973 +122974 POINT(39.715798665291445 73.16912432667388) bank122974 +122975 POINT(41.28179414302119 73.22098928219557) bank122975 +122976 POINT(39.727231158667934 73.51197568033054) bank122976 +122977 POINT(40.952894542517164 74.07698276579106) bank122977 +122978 POINT(40.120574675653096 74.30608593985009) bank122978 +122979 POINT(40.317408253140826 74.15097281150253) bank122979 +122980 POINT(40.77312043850113 74.81725037611305) bank122980 +122981 POINT(40.58736291145825 74.46225140135867) bank122981 +122982 POINT(40.57162620882015 73.60112770821775) bank122982 +122983 POINT(41.474601710235866 74.13031591441927) bank122983 +122984 POINT(41.14903884986764 73.45314315822792) bank122984 +122985 POINT(41.305565666080085 73.97782837317169) bank122985 +122986 POINT(40.38587790078188 73.20668888057286) bank122986 +122987 POINT(39.73079377608997 74.86958227995514) bank122987 +122988 POINT(41.02498945933242 74.3741669768938) bank122988 +122989 POINT(40.64985061274451 74.36983344813555) bank122989 +122990 POINT(40.73795802278465 74.41024697430005) bank122990 +122991 POINT(39.884370424921066 74.24227111978455) bank122991 +122992 POINT(39.7756024271904 74.25858580556513) bank122992 +122993 POINT(41.09803032336252 73.86857080478144) bank122993 +122994 POINT(40.62049403007031 73.7987202573181) bank122994 +122995 POINT(40.770998025962776 74.62048382103642) bank122995 +122996 POINT(39.75869067188474 74.385354585684) bank122996 +122997 POINT(40.06354103622772 74.88354931027017) bank122997 +122998 POINT(40.22941216771206 73.93760345638707) bank122998 +122999 POINT(40.02900692911797 73.15762947501126) bank122999 +123000 POINT(39.79779463411626 73.66693147912929) bank123000 +123001 POINT(41.052372194472866 74.11513376346255) bank123001 +123002 POINT(39.77284440509613 74.13798274754934) bank123002 +123003 POINT(41.405157721797 73.96520156340559) bank123003 +123004 POINT(40.81080517633717 73.23297718036267) bank123004 +123005 POINT(39.75999059312467 74.5953515771451) bank123005 +123006 POINT(40.82491583956371 74.56365063373984) bank123006 +123007 POINT(40.09908734697205 73.291123914543) bank123007 +123008 POINT(39.80784919176303 73.9781954510322) bank123008 +123009 POINT(39.90494279837502 73.10829076885231) bank123009 +123010 POINT(40.562684529886376 74.10002533637001) bank123010 +123011 POINT(40.78505318323041 74.63129296566001) bank123011 +123012 POINT(39.81496758189877 74.32138394464725) bank123012 +123013 POINT(40.58337584357508 74.6252921894543) bank123013 +123014 POINT(40.425771034357645 74.13056007693599) bank123014 +123015 POINT(41.03642619136459 73.62950525911594) bank123015 +123016 POINT(41.42392847053203 73.45337126708363) bank123016 +123017 POINT(39.9616102002484 73.13596201052913) bank123017 +123018 POINT(40.477250682784785 74.1755810258879) bank123018 +123019 POINT(40.59126441046259 73.88128245514346) bank123019 +123020 POINT(39.98151928187216 74.76391307292337) bank123020 +123021 POINT(41.38216881707865 73.1071448499639) bank123021 +123022 POINT(39.8035817281373 74.16020698657597) bank123022 +123023 POINT(40.18066469271521 73.30318582173273) bank123023 +123024 POINT(39.79568212641477 73.35598603636906) bank123024 +123025 POINT(40.71336436793856 73.96584247597983) bank123025 +123026 POINT(40.4206969193313 73.59917340340212) bank123026 +123027 POINT(40.11810711183835 73.28561776820982) bank123027 +123028 POINT(39.75281235141227 73.22701444506278) bank123028 +123029 POINT(41.21436615155518 74.936452719192) bank123029 +123030 POINT(41.05677596184693 74.67409620557697) bank123030 +123031 POINT(40.76551319521592 74.70201873672582) bank123031 +123032 POINT(41.26671346701275 74.34453751023567) bank123032 +123033 POINT(41.63479776447187 73.48489374286177) bank123033 +123034 POINT(41.277787623266846 74.54576946000297) bank123034 +123035 POINT(40.41393243069809 74.4469500260559) bank123035 +123036 POINT(41.44728354415509 73.35653867378716) bank123036 +123037 POINT(41.29595078855225 73.63037637950158) bank123037 +123038 POINT(41.4005875924127 74.56002909461647) bank123038 +123039 POINT(41.62972178822274 73.6917348870989) bank123039 +123040 POINT(40.907856244631965 73.52019724327549) bank123040 +123041 POINT(41.32558489028045 73.84939224603619) bank123041 +123042 POINT(41.43396358285462 73.92058310846171) bank123042 +123043 POINT(40.51100182896362 73.4652590835261) bank123043 +123044 POINT(40.01440325035756 73.8091123480692) bank123044 +123045 POINT(40.157047576505796 74.99295268371931) bank123045 +123046 POINT(39.916900008264165 74.45954190135208) bank123046 +123047 POINT(40.556648271700304 73.73549826781267) bank123047 +123048 POINT(39.92541579917689 73.19859062834466) bank123048 +123049 POINT(41.183328648740854 74.35476469494047) bank123049 +123050 POINT(40.51565414272391 73.98112910054779) bank123050 +123051 POINT(40.54974408982463 74.70817845302598) bank123051 +123052 POINT(40.98067540733407 74.75907726668329) bank123052 +123053 POINT(41.70407706591071 73.67463787541948) bank123053 +123054 POINT(39.80415113864275 73.55802285273313) bank123054 +123055 POINT(40.574443337938106 74.55226025184153) bank123055 +123056 POINT(39.730343162240935 74.95744181829603) bank123056 +123057 POINT(40.16280969010039 74.13753572971585) bank123057 +123058 POINT(41.31937743803342 74.11904151343799) bank123058 +123059 POINT(39.880726367449896 74.36930026222669) bank123059 +123060 POINT(40.965635767954616 74.09678340048258) bank123060 +123061 POINT(41.13413764233856 74.81553634674064) bank123061 +123062 POINT(40.78340083196126 74.4189720783247) bank123062 +123063 POINT(40.3081837173891 73.89608676449109) bank123063 +123064 POINT(39.72390780723303 74.1085434541395) bank123064 +123065 POINT(40.571629226506936 74.09378579263563) bank123065 +123066 POINT(41.22125330366344 73.8127757010689) bank123066 +123067 POINT(40.80576651340153 74.0783919351009) bank123067 +123068 POINT(40.321911346436664 74.78381744604569) bank123068 +123069 POINT(40.482071549745996 73.07575566548982) bank123069 +123070 POINT(41.485861981164575 74.73947558284466) bank123070 +123071 POINT(41.570542333086806 74.22908171407451) bank123071 +123072 POINT(41.18061743282918 74.55496106404186) bank123072 +123073 POINT(40.23606287287154 74.34027561584811) bank123073 +123074 POINT(40.68313138418014 73.33861987819951) bank123074 +123075 POINT(40.175232362315064 74.40041659424764) bank123075 +123076 POINT(41.293040525927786 73.26034405907889) bank123076 +123077 POINT(41.03262069346323 73.52660169805203) bank123077 +123078 POINT(40.34166023825694 73.24357107874295) bank123078 +123079 POINT(40.62378068873178 73.11834920307075) bank123079 +123080 POINT(40.49356195775552 74.78424465287989) bank123080 +123081 POINT(41.61132427973027 74.767904502985) bank123081 +123082 POINT(40.03052035022734 74.35865405445998) bank123082 +123083 POINT(40.34745810934135 73.69985246491288) bank123083 +123084 POINT(40.218228238942885 74.95279268423643) bank123084 +123085 POINT(41.51708557673382 73.88265745294383) bank123085 +123086 POINT(39.765399986480986 74.08753338361063) bank123086 +123087 POINT(40.765187388097935 73.59098526634756) bank123087 +123088 POINT(41.0328955581742 74.8606956091814) bank123088 +123089 POINT(41.51610997401719 74.17472451986445) bank123089 +123090 POINT(40.039795278405066 73.94712707044033) bank123090 +123091 POINT(40.91231199163586 74.76476529777258) bank123091 +123092 POINT(40.630487326125134 73.71478312538807) bank123092 +123093 POINT(40.015497512455745 74.9838075757645) bank123093 +123094 POINT(39.78527934297526 74.41057972619265) bank123094 +123095 POINT(41.65406647256605 73.91635941535152) bank123095 +123096 POINT(40.509447471321565 73.34313591166048) bank123096 +123097 POINT(41.29026634597626 73.94835047901906) bank123097 +123098 POINT(41.66532417681298 73.15832357519209) bank123098 +123099 POINT(41.631498490052614 74.79671445657463) bank123099 +123100 POINT(40.630548685172286 73.63454404529602) bank123100 +123101 POINT(40.64451500509389 74.3803134603524) bank123101 +123102 POINT(40.09615736000036 73.56296048418339) bank123102 +123103 POINT(39.73735887054642 73.39097770143901) bank123103 +123104 POINT(40.66120292884143 74.44681414295279) bank123104 +123105 POINT(40.30208567495158 73.09675258898801) bank123105 +123106 POINT(40.90079577511011 74.74360473606035) bank123106 +123107 POINT(41.245360938433116 74.23186333485441) bank123107 +123108 POINT(39.83639636707956 74.5382314303177) bank123108 +123109 POINT(41.241287883503624 73.25029497438233) bank123109 +123110 POINT(40.68191642447195 73.16779075033352) bank123110 +123111 POINT(41.595807655325146 73.44961463274566) bank123111 +123112 POINT(41.64107872910463 73.06045506255714) bank123112 +123113 POINT(41.087684579501946 74.68485428872056) bank123113 +123114 POINT(41.18059939039777 74.21307666180029) bank123114 +123115 POINT(41.29652396170325 73.4015779777824) bank123115 +123116 POINT(39.780031687912086 73.88388849848177) bank123116 +123117 POINT(41.57697219861627 73.720411668603) bank123117 +123118 POINT(41.51523226752658 74.9286741302341) bank123118 +123119 POINT(40.33708061096747 73.67366138051364) bank123119 +123120 POINT(41.35274391467316 73.72790424549666) bank123120 +123121 POINT(41.569245785327986 73.80417869099317) bank123121 +123122 POINT(40.66038141424003 74.88198773764209) bank123122 +123123 POINT(40.38795335949195 73.78525563802818) bank123123 +123124 POINT(40.08680605608607 73.72852412094899) bank123124 +123125 POINT(41.19703575892331 74.15995586889636) bank123125 +123126 POINT(40.43812248165825 73.63395614401797) bank123126 +123127 POINT(40.04552089883897 74.29996742266506) bank123127 +123128 POINT(40.84208767752956 74.98989481446425) bank123128 +123129 POINT(40.140846619246624 73.59035241226914) bank123129 +123130 POINT(40.95613076200524 74.45181124883831) bank123130 +123131 POINT(41.02679067933306 73.233232280085) bank123131 +123132 POINT(40.03766870329367 74.70782759310319) bank123132 +123133 POINT(40.02151287829779 73.32064032408192) bank123133 +123134 POINT(40.41318483068982 73.88169834430893) bank123134 +123135 POINT(41.11344233294233 74.0560366288568) bank123135 +123136 POINT(40.60172382326297 74.48208573158999) bank123136 +123137 POINT(40.68819231065131 73.65260148030029) bank123137 +123138 POINT(41.119170128757645 74.63240422700379) bank123138 +123139 POINT(39.72006938832714 73.69593929841521) bank123139 +123140 POINT(40.54039257817041 73.45474174568184) bank123140 +123141 POINT(40.2489261735618 73.26534081948822) bank123141 +123142 POINT(41.63906566144157 74.39661569525768) bank123142 +123143 POINT(39.85765891851345 74.22275928326815) bank123143 +123144 POINT(39.86158023582295 74.48278446626811) bank123144 +123145 POINT(40.61141471562588 73.43396638173603) bank123145 +123146 POINT(41.0656271707731 73.28695056148464) bank123146 +123147 POINT(39.893361818324124 74.14198302546924) bank123147 +123148 POINT(40.336218192425534 73.70611718891541) bank123148 +123149 POINT(40.20760361061403 74.28317433534563) bank123149 +123150 POINT(41.16132261694795 73.88327975323331) bank123150 +123151 POINT(39.93354272435572 74.33742391348424) bank123151 +123152 POINT(40.1136747497349 73.39948798824362) bank123152 +123153 POINT(39.86723202351067 73.96710807779738) bank123153 +123154 POINT(40.405406777888196 73.67770195481604) bank123154 +123155 POINT(40.636193399556745 73.61694371438331) bank123155 +123156 POINT(40.82941575610479 73.3591038483769) bank123156 +123157 POINT(41.64647787354839 73.33663796274287) bank123157 +123158 POINT(40.788196915636824 73.04829371384639) bank123158 +123159 POINT(40.655602310437764 74.82516590533618) bank123159 +123160 POINT(39.817547738779524 74.42152788085201) bank123160 +123161 POINT(39.838133910561474 74.07222200016722) bank123161 +123162 POINT(41.139422608278004 73.0660145880049) bank123162 +123163 POINT(41.33748388695793 73.85655462914782) bank123163 +123164 POINT(40.3616224100047 74.85331528381096) bank123164 +123165 POINT(40.98064052889322 74.3546713805204) bank123165 +123166 POINT(40.44454688239774 73.21405116087186) bank123166 +123167 POINT(41.1674788219688 74.21790380608428) bank123167 +123168 POINT(41.53684490166121 73.47224498773656) bank123168 +123169 POINT(40.891156937326095 74.4701938605452) bank123169 +123170 POINT(41.24659646286693 73.94443756711816) bank123170 +123171 POINT(40.31013147829681 74.7920103487567) bank123171 +123172 POINT(40.76207905904192 74.04169271491293) bank123172 +123173 POINT(40.22867286639534 74.52803468805752) bank123173 +123174 POINT(40.8846081977885 73.15851797027406) bank123174 +123175 POINT(40.73073800958532 73.02101112534649) bank123175 +123176 POINT(41.61760223311377 74.55289966255026) bank123176 +123177 POINT(40.181095696401385 73.0423997523936) bank123177 +123178 POINT(40.45433555195664 74.63286966006514) bank123178 +123179 POINT(40.00298947416725 73.67046178660314) bank123179 +123180 POINT(41.43114205640458 73.55520271129835) bank123180 +123181 POINT(39.88477360982649 73.18112193086279) bank123181 +123182 POINT(40.902638370345294 74.4071834751045) bank123182 +123183 POINT(40.27815991574741 74.7394551401686) bank123183 +123184 POINT(39.96453948501365 73.8731904144077) bank123184 +123185 POINT(40.71285951617274 74.51852032214697) bank123185 +123186 POINT(39.971238092748266 73.66169721871373) bank123186 +123187 POINT(40.18567320245621 74.13477697952833) bank123187 +123188 POINT(39.84747582749247 74.55437492216221) bank123188 +123189 POINT(39.912842097351515 73.54941850525556) bank123189 +123190 POINT(40.91644590637447 74.21139293547814) bank123190 +123191 POINT(40.29745775348839 73.02428639635376) bank123191 +123192 POINT(40.609602150583356 74.86364243351979) bank123192 +123193 POINT(39.867572823297685 73.60671083050522) bank123193 +123194 POINT(41.008760126161064 74.88457399737112) bank123194 +123195 POINT(41.1273806131669 73.92774176720351) bank123195 +123196 POINT(41.21599971058203 74.46674505424305) bank123196 +123197 POINT(41.06966690267871 73.8181330063322) bank123197 +123198 POINT(41.06388238705845 73.686265984465) bank123198 +123199 POINT(41.272638442422284 73.73771459870905) bank123199 +123200 POINT(41.52661976563928 73.48820540246115) bank123200 +123201 POINT(41.484754073545325 74.31588846452895) bank123201 +123202 POINT(40.059321085364644 73.23294753495546) bank123202 +123203 POINT(40.049108000775 73.91455798782445) bank123203 +123204 POINT(41.22404064070335 73.52035338893509) bank123204 +123205 POINT(40.2751482915698 74.38063494186973) bank123205 +123206 POINT(40.700694105477425 74.21847702374332) bank123206 +123207 POINT(40.02492996200555 74.0196245719373) bank123207 +123208 POINT(40.758138703802636 74.51300565031177) bank123208 +123209 POINT(41.33463862992632 74.46552888252414) bank123209 +123210 POINT(41.485442873824866 73.86578814813709) bank123210 +123211 POINT(40.88128656446054 73.5389368765257) bank123211 +123212 POINT(40.279335517599925 73.91891355495848) bank123212 +123213 POINT(41.28624393772592 73.08872667337097) bank123213 +123214 POINT(41.29743917784551 73.41267329073504) bank123214 +123215 POINT(41.01053092649014 74.42730703968932) bank123215 +123216 POINT(40.68540908934208 74.44522185094657) bank123216 +123217 POINT(40.81087466651513 73.6419994723885) bank123217 +123218 POINT(40.476692272743215 74.06393621938615) bank123218 +123219 POINT(40.792081894996194 73.97134702671963) bank123219 +123220 POINT(40.151507498987336 74.21045053553242) bank123220 +123221 POINT(41.11756827199919 74.3986996022157) bank123221 +123222 POINT(41.134509057710204 73.63179690300399) bank123222 +123223 POINT(40.07510387820938 73.28579054720207) bank123223 +123224 POINT(41.5004183582353 74.43396512215448) bank123224 +123225 POINT(41.10898383524562 73.56682641131314) bank123225 +123226 POINT(40.89595816629299 73.75860264835867) bank123226 +123227 POINT(41.070784335899276 73.54968812846828) bank123227 +123228 POINT(41.01309156987802 74.1679342575913) bank123228 +123229 POINT(41.28961687557123 74.96333192183772) bank123229 +123230 POINT(40.771920570514126 74.93611899733897) bank123230 +123231 POINT(41.07101522720047 74.07354501450936) bank123231 +123232 POINT(40.39962674423801 74.72410805160625) bank123232 +123233 POINT(40.1631764291069 74.49402382639133) bank123233 +123234 POINT(40.824100051689875 74.89914653013571) bank123234 +123235 POINT(40.75547038753779 74.02900825914087) bank123235 +123236 POINT(41.418641144247104 74.50410996835653) bank123236 +123237 POINT(39.90543324834661 73.22694494211163) bank123237 +123238 POINT(39.724143645060586 74.55043774934714) bank123238 +123239 POINT(40.07317040568657 74.21368456066024) bank123239 +123240 POINT(41.521046862718144 74.66451718751192) bank123240 +123241 POINT(40.38967068967838 73.24161617939764) bank123241 +123242 POINT(40.04132775293874 74.64133042374499) bank123242 +123243 POINT(40.28833130584975 74.94695029923787) bank123243 +123244 POINT(40.16566608753647 74.20086856581739) bank123244 +123245 POINT(40.19429009765883 74.80743888057891) bank123245 +123246 POINT(41.077514279425436 73.68825954749857) bank123246 +123247 POINT(41.23968950227021 74.69447741578624) bank123247 +123248 POINT(39.921526154530916 74.73385173514633) bank123248 +123249 POINT(40.73415604133471 74.07693009994965) bank123249 +123250 POINT(41.5358293853184 74.7967965899106) bank123250 +123251 POINT(40.87964578122406 75.00288524242613) bank123251 +123252 POINT(40.221572678049704 74.30515384649395) bank123252 +123253 POINT(41.197778173013994 74.40171866893317) bank123253 +123254 POINT(40.93454929343064 74.93150418340498) bank123254 +123255 POINT(41.709104005119755 74.97180674236574) bank123255 +123256 POINT(39.80553988182753 74.74552122569767) bank123256 +123257 POINT(40.58915744836877 73.58103325933621) bank123257 +123258 POINT(41.28718374058904 73.52672018250774) bank123258 +123259 POINT(41.70865452862669 74.86439818321603) bank123259 +123260 POINT(40.525149831048 73.14059191916269) bank123260 +123261 POINT(41.59908716096882 73.95542776233258) bank123261 +123262 POINT(41.30500047154977 74.09997388178023) bank123262 +123263 POINT(41.15247073318882 74.71108806192929) bank123263 +123264 POINT(41.32719848879467 73.70155660513868) bank123264 +123265 POINT(40.88233610290048 73.21362061829274) bank123265 +123266 POINT(41.51040887528842 74.47606012682013) bank123266 +123267 POINT(40.974299913244224 73.01750112413792) bank123267 +123268 POINT(40.03305226573187 74.21676010464249) bank123268 +123269 POINT(40.305407810094096 74.5014985125508) bank123269 +123270 POINT(41.43802735805971 73.1760057125973) bank123270 +123271 POINT(40.13135839247022 73.41059087241047) bank123271 +123272 POINT(41.21354871825378 74.80688005928704) bank123272 +123273 POINT(40.62842134980675 73.430720026036) bank123273 +123274 POINT(40.88811720820542 73.23489321864997) bank123274 +123275 POINT(39.846281647822124 74.12073974703878) bank123275 +123276 POINT(40.768196243993025 73.80289214492112) bank123276 +123277 POINT(40.63137887942463 74.11833241630016) bank123277 +123278 POINT(40.191359134902655 73.28090391254075) bank123278 +123279 POINT(40.324292336527726 73.65891502373931) bank123279 +123280 POINT(40.14018616064165 74.97309890013307) bank123280 +123281 POINT(40.7720821326158 74.08340565403279) bank123281 +123282 POINT(41.13006192202945 73.15849175863454) bank123282 +123283 POINT(41.19374266961132 73.77459606797933) bank123283 +123284 POINT(40.18722944932042 74.50617913078594) bank123284 +123285 POINT(41.21440307239165 73.02199777303387) bank123285 +123286 POINT(40.86677122574603 74.65647827909166) bank123286 +123287 POINT(40.70928584276281 73.92086397563257) bank123287 +123288 POINT(40.821715496730825 74.78870135538561) bank123288 +123289 POINT(39.90612253412179 74.24410569813088) bank123289 +123290 POINT(40.38245410647936 74.52852498317867) bank123290 +123291 POINT(41.23446136235027 73.89834389923573) bank123291 +123292 POINT(40.679311259595465 74.70269294863344) bank123292 +123293 POINT(41.67005768887107 74.15300244714871) bank123293 +123294 POINT(40.322694554334326 73.39841124110995) bank123294 +123295 POINT(40.2948094112655 73.59354203176098) bank123295 +123296 POINT(41.196826028006065 74.13811754354869) bank123296 +123297 POINT(41.17069856800082 73.26835884869374) bank123297 +123298 POINT(39.921908998639694 74.70103819843419) bank123298 +123299 POINT(41.27805942245095 74.43416891220517) bank123299 +123300 POINT(40.702367982807935 74.95278212498758) bank123300 +123301 POINT(40.049673652327094 74.65992852633308) bank123301 +123302 POINT(39.7562209375095 74.77790752145157) bank123302 +123303 POINT(41.70858432957252 74.25932325853321) bank123303 +123304 POINT(40.44578252834441 73.88842529198094) bank123304 +123305 POINT(40.694179947948605 73.34121825506102) bank123305 +123306 POINT(40.126081510307074 74.29908275127102) bank123306 +123307 POINT(40.08421450588926 73.40853605607577) bank123307 +123308 POINT(40.34694864995502 73.47935940700872) bank123308 +123309 POINT(41.685165620034134 74.37570120447072) bank123309 +123310 POINT(41.069355220094444 74.13905189953519) bank123310 +123311 POINT(41.61297643668841 73.3977204400035) bank123311 +123312 POINT(41.33527474473177 73.14360063577463) bank123312 +123313 POINT(40.602985019520325 74.20145766988055) bank123313 +123314 POINT(41.63808432575767 73.99319371700851) bank123314 +123315 POINT(40.12954144186992 74.264237091562) bank123315 +123316 POINT(40.53329404842599 74.45252132925712) bank123316 +123317 POINT(40.731484240327596 74.0291231647924) bank123317 +123318 POINT(40.63921522935473 73.29143664082223) bank123318 +123319 POINT(40.67474604807895 74.27775383148838) bank123319 +123320 POINT(39.912041135306666 74.21035753554156) bank123320 +123321 POINT(41.55369106686487 74.43158702369357) bank123321 +123322 POINT(41.58794082595606 74.81506865437821) bank123322 +123323 POINT(39.76498625536497 74.51811914191015) bank123323 +123324 POINT(40.674642110915485 74.62799365894604) bank123324 +123325 POINT(41.4465195312727 74.22770758043326) bank123325 +123326 POINT(39.91119368876322 74.87019597901518) bank123326 +123327 POINT(41.39848206592005 73.35946446771895) bank123327 +123328 POINT(40.780548395969824 74.65915440822084) bank123328 +123329 POINT(40.88200727489988 73.72300595076712) bank123329 +123330 POINT(40.76100564343149 73.40074441208085) bank123330 +123331 POINT(40.982203902250305 73.2492680425343) bank123331 +123332 POINT(40.90148939772394 73.69577346187329) bank123332 +123333 POINT(40.53759047613079 74.3905832469066) bank123333 +123334 POINT(41.68781334212008 74.74048223117384) bank123334 +123335 POINT(40.82068454769963 73.00877237090779) bank123335 +123336 POINT(40.84376490818545 73.94062257235771) bank123336 +123337 POINT(41.293833314480224 74.87812218191291) bank123337 +123338 POINT(40.216848432146726 73.12703441160325) bank123338 +123339 POINT(40.10397685975591 74.20840459876558) bank123339 +123340 POINT(40.686563955641475 74.45768009438446) bank123340 +123341 POINT(39.80846104693608 73.052709939651) bank123341 +123342 POINT(40.82342777224654 74.94543422027951) bank123342 +123343 POINT(41.342586687306124 73.3926583827002) bank123343 +123344 POINT(39.92277199668967 74.877184222353) bank123344 +123345 POINT(40.017883345403234 74.194015144638) bank123345 +123346 POINT(41.2339298513948 73.05476318016912) bank123346 +123347 POINT(40.38544963847703 73.22558216398468) bank123347 +123348 POINT(39.802544616448195 73.1225846238726) bank123348 +123349 POINT(40.569456640485676 73.26190328920036) bank123349 +123350 POINT(39.908584793545856 74.26525316182115) bank123350 +123351 POINT(40.659477659401745 73.4107882925214) bank123351 +123352 POINT(40.204386470738605 73.31624694017262) bank123352 +123353 POINT(40.723296873763985 74.99648239870697) bank123353 +123354 POINT(40.08005343403717 73.35494227202295) bank123354 +123355 POINT(39.72185377529812 74.56912290736993) bank123355 +123356 POINT(40.41756358948007 73.91756899009104) bank123356 +123357 POINT(40.46761043565112 74.18281459101573) bank123357 +123358 POINT(41.429936353903415 74.90253271037207) bank123358 +123359 POINT(40.982212289847034 74.25669487378863) bank123359 +123360 POINT(41.000429424356284 74.96571821224336) bank123360 +123361 POINT(40.63892936399192 73.71058978051215) bank123361 +123362 POINT(40.628992194885384 74.03349302824658) bank123362 +123363 POINT(41.63089153027298 73.06980475288483) bank123363 +123364 POINT(41.692236848295295 73.27371098083867) bank123364 +123365 POINT(40.867504679665124 74.86777295636057) bank123365 +123366 POINT(41.475854652429554 74.67272730956365) bank123366 +123367 POINT(39.79646374437905 73.13970090851842) bank123367 +123368 POINT(41.22003350529346 73.71396035673989) bank123368 +123369 POINT(41.23179880321959 74.30809655136285) bank123369 +123370 POINT(41.4358596031642 74.79062884782016) bank123370 +123371 POINT(41.577476151721854 73.72231294354997) bank123371 +123372 POINT(40.10254346102224 74.03891910955176) bank123372 +123373 POINT(41.45146646518869 74.32263341115573) bank123373 +123374 POINT(39.967791251983975 73.18059978737521) bank123374 +123375 POINT(39.8820298002366 73.7755302817621) bank123375 +123376 POINT(40.28548274573944 74.44270299041452) bank123376 +123377 POINT(41.598424204604754 73.56358986579963) bank123377 +123378 POINT(41.45395266300365 73.71338775295777) bank123378 +123379 POINT(40.066239504956684 73.96726954707722) bank123379 +123380 POINT(40.55322012316513 74.08989351251354) bank123380 +123381 POINT(41.51640578845892 73.67274587506436) bank123381 +123382 POINT(41.00133888469996 73.60007767377127) bank123382 +123383 POINT(41.48294359411975 74.61077517940467) bank123383 +123384 POINT(40.180366620643234 73.98861272323445) bank123384 +123385 POINT(41.28899238100855 73.74211578548092) bank123385 +123386 POINT(41.584083127742524 73.53073173129908) bank123386 +123387 POINT(39.74353115644871 74.54782490393615) bank123387 +123388 POINT(41.64737467545384 73.67902193862368) bank123388 +123389 POINT(39.92616143860054 74.56322518678623) bank123389 +123390 POINT(41.03368608960263 73.39965204571999) bank123390 +123391 POINT(41.06550371853612 74.60223249211079) bank123391 +123392 POINT(40.47691282772323 73.07788771982253) bank123392 +123393 POINT(40.75700689528802 74.56773377049612) bank123393 +123394 POINT(41.41102190345964 73.61543412976411) bank123394 +123395 POINT(40.22149191384674 74.19272814830967) bank123395 +123396 POINT(41.06579170078139 73.93607808861738) bank123396 +123397 POINT(41.247719691166616 73.12777099170661) bank123397 +123398 POINT(41.09468699227007 73.34923175314654) bank123398 +123399 POINT(39.8505952587555 73.40077700978999) bank123399 +123400 POINT(40.955327931638436 74.63913410455156) bank123400 +123401 POINT(40.92570141804814 73.94567183000365) bank123401 +123402 POINT(40.222102700926115 73.02783950231971) bank123402 +123403 POINT(41.60809241981915 74.82883579109613) bank123403 +123404 POINT(41.64650191422494 74.80375655363521) bank123404 +123405 POINT(40.36358590956298 74.31768725442973) bank123405 +123406 POINT(40.319810274549404 74.25194084415664) bank123406 +123407 POINT(39.9568241654328 73.70155651599168) bank123407 +123408 POINT(41.50023643260686 74.3593474012683) bank123408 +123409 POINT(40.74466964421628 73.28582171024243) bank123409 +123410 POINT(40.5040680874346 74.35692172861445) bank123410 +123411 POINT(41.13877513233503 73.17526506888832) bank123411 +123412 POINT(40.974519649797905 73.30771524960248) bank123412 +123413 POINT(40.972110011150335 74.3320100468685) bank123413 +123414 POINT(40.11037777856059 73.21788541717292) bank123414 +123415 POINT(40.10349111919744 73.43652147272982) bank123415 +123416 POINT(41.448263392973125 73.11717643370692) bank123416 +123417 POINT(41.60284843760127 74.67426572527759) bank123417 +123418 POINT(39.79607229851943 74.72443209232183) bank123418 +123419 POINT(40.53317198715478 74.03835242278694) bank123419 +123420 POINT(40.52596584378835 73.04503345402416) bank123420 +123421 POINT(41.54072320239574 73.53191140388901) bank123421 +123422 POINT(41.50310070798316 73.34905846706489) bank123422 +123423 POINT(40.3950167361006 73.2305793327392) bank123423 +123424 POINT(40.064865151611514 73.06790808149125) bank123424 +123425 POINT(41.17332229193059 73.77960103993922) bank123425 +123426 POINT(39.74363601137798 73.08147029793454) bank123426 +123427 POINT(40.173575308677194 73.31487918810005) bank123427 +123428 POINT(41.165103972354885 74.77456845303021) bank123428 +123429 POINT(41.0492427233198 74.27277049731002) bank123429 +123430 POINT(39.76479325991218 73.54832123712744) bank123430 +123431 POINT(40.91697088484937 74.73379903564283) bank123431 +123432 POINT(40.52902784394668 74.31481142990098) bank123432 +123433 POINT(39.888171545272606 74.26786486201895) bank123433 +123434 POINT(40.17423546942934 74.30733108002988) bank123434 +123435 POINT(40.17563350281512 73.47979936096502) bank123435 +123436 POINT(41.26766895255029 73.80148828585489) bank123436 +123437 POINT(40.20380872197517 74.01796659031655) bank123437 +123438 POINT(40.91939170531025 73.36331081013836) bank123438 +123439 POINT(39.834419739460195 73.16871897116813) bank123439 +123440 POINT(40.77292678707469 74.41311102805153) bank123440 +123441 POINT(40.17332771058179 73.19227380154358) bank123441 +123442 POINT(40.695212576197456 73.2433614667803) bank123442 +123443 POINT(40.10696959636735 74.50447019839115) bank123443 +123444 POINT(41.034745998719174 74.03640699977652) bank123444 +123445 POINT(40.90870310992144 74.4838421757294) bank123445 +123446 POINT(40.14075874890173 73.87951823976819) bank123446 +123447 POINT(40.61184210453815 73.57037279260055) bank123447 +123448 POINT(41.027767573699094 73.39372439830647) bank123448 +123449 POINT(40.88353244836787 74.51281424340645) bank123449 +123450 POINT(41.24893578840675 73.61519083980657) bank123450 +123451 POINT(40.47724799000095 73.09334195304847) bank123451 +123452 POINT(40.90068722862403 74.59420544926846) bank123452 +123453 POINT(41.28981045269579 74.96560564322778) bank123453 +123454 POINT(39.73496208888292 73.10463102964027) bank123454 +123455 POINT(40.247685942134936 74.6530961634321) bank123455 +123456 POINT(39.72131653492073 74.20479118018682) bank123456 +123457 POINT(40.31628197755624 74.58393414238817) bank123457 +123458 POINT(39.79549681608492 73.07692297105086) bank123458 +123459 POINT(40.222801769005706 74.03681242093477) bank123459 +123460 POINT(40.088670554403826 74.6983260355462) bank123460 +123461 POINT(40.95020129182425 73.10367974593353) bank123461 +123462 POINT(41.315840112896964 73.08258874933455) bank123462 +123463 POINT(40.45248149487415 73.63693160609446) bank123463 +123464 POINT(41.42865147000886 73.62354996687475) bank123464 +123465 POINT(40.2286361900678 73.38854900330814) bank123465 +123466 POINT(41.15252833496493 74.2311601106701) bank123466 +123467 POINT(41.68271484422907 73.42658438578134) bank123467 +123468 POINT(39.770687911182996 74.80667144851304) bank123468 +123469 POINT(41.19821033247657 74.40274110831444) bank123469 +123470 POINT(41.320176728596266 74.37015515226086) bank123470 +123471 POINT(40.07219780029259 73.02539227128217) bank123471 +123472 POINT(39.75737506742109 74.53123479612765) bank123472 +123473 POINT(41.024140243955415 73.1359828095307) bank123473 +123474 POINT(40.05623012322515 73.07890263989756) bank123474 +123475 POINT(39.7925281322912 74.92330735995678) bank123475 +123476 POINT(40.044020215596056 73.77243418771339) bank123476 +123477 POINT(41.20165873976739 73.70006680809895) bank123477 +123478 POINT(40.20288397773949 74.02738368893355) bank123478 +123479 POINT(39.84429590576763 74.64823024573654) bank123479 +123480 POINT(40.63599787737391 73.4291252485445) bank123480 +123481 POINT(41.12239522395977 74.79647618674215) bank123481 +123482 POINT(40.99874060789843 74.92824319154884) bank123482 +123483 POINT(41.67025581774638 73.34002984994693) bank123483 +123484 POINT(40.887806603328514 74.23067033575745) bank123484 +123485 POINT(40.41195616651806 74.57095159725316) bank123485 +123486 POINT(40.18450007307367 74.53138688125418) bank123486 +123487 POINT(40.9633123663484 74.6764810362788) bank123487 +123488 POINT(40.18971160963583 73.80536554799359) bank123488 +123489 POINT(40.73512782550047 73.21410161478033) bank123489 +123490 POINT(41.44122840091649 74.0433162308813) bank123490 +123491 POINT(40.27544740709224 74.685127694724) bank123491 +123492 POINT(40.416188274160774 73.24206283082164) bank123492 +123493 POINT(39.97353366487013 74.43688326593346) bank123493 +123494 POINT(41.18870318078225 73.28514709242876) bank123494 +123495 POINT(40.162487321301725 74.22966068857548) bank123495 +123496 POINT(40.05522967488943 73.52620411207069) bank123496 +123497 POINT(41.59240062338235 73.68527286969302) bank123497 +123498 POINT(40.71002114832842 74.74302024681403) bank123498 +123499 POINT(41.152231508188 74.78778954937103) bank123499 +123500 POINT(40.61650839113465 74.91353791028726) bank123500 +123501 POINT(41.56644243707511 74.48532320133044) bank123501 +123502 POINT(40.48549196166703 74.08328924306622) bank123502 +123503 POINT(41.33100653429947 73.97504380790748) bank123503 +123504 POINT(39.714277186321596 73.96041826772547) bank123504 +123505 POINT(40.085774453588414 73.6156418565563) bank123505 +123506 POINT(41.278433073032296 73.16697541135831) bank123506 +123507 POINT(41.204250366878995 75.00489882591489) bank123507 +123508 POINT(41.1687767383441 74.93765894715807) bank123508 +123509 POINT(39.77575623660117 74.24390244707027) bank123509 +123510 POINT(40.461524177319326 73.19960215696278) bank123510 +123511 POINT(40.75198098904898 73.48392824198662) bank123511 +123512 POINT(41.06305228913236 74.9867189390419) bank123512 +123513 POINT(39.92975989455513 73.02848116363563) bank123513 +123514 POINT(40.541535557432596 73.96451884311223) bank123514 +123515 POINT(41.70560362027392 73.13368743354187) bank123515 +123516 POINT(41.430284944357645 73.87338677853522) bank123516 +123517 POINT(41.272107384001934 74.25961643825708) bank123517 +123518 POINT(40.34070855808059 74.66503932115394) bank123518 +123519 POINT(41.42458738427405 74.45502116988398) bank123519 +123520 POINT(41.41647453413212 73.37304893233087) bank123520 +123521 POINT(40.645649695863355 73.22579188064715) bank123521 +123522 POINT(41.16141076040962 73.83820195781595) bank123522 +123523 POINT(41.36582433150272 73.78667192589562) bank123523 +123524 POINT(41.24055695046961 74.45107356901565) bank123524 +123525 POINT(40.46136838541154 73.9367763652349) bank123525 +123526 POINT(40.69513668818896 74.82013335885567) bank123526 +123527 POINT(40.322474991757986 74.42836862639575) bank123527 +123528 POINT(40.090465954201754 74.31765775688241) bank123528 +123529 POINT(41.09229323352151 74.43809501354517) bank123529 +123530 POINT(40.71378418759949 74.72533614730091) bank123530 +123531 POINT(41.03918525423492 74.02095904307988) bank123531 +123532 POINT(41.25408566279543 74.63924875773318) bank123532 +123533 POINT(41.27749368854377 73.9760902051125) bank123533 +123534 POINT(40.745804176217 73.5933556969373) bank123534 +123535 POINT(40.56977311804447 73.05262338308057) bank123535 +123536 POINT(40.33787400009089 73.85815619078346) bank123536 +123537 POINT(40.75206609438163 73.37098337226155) bank123537 +123538 POINT(39.767098553862894 75.00247487121014) bank123538 +123539 POINT(40.610685583900825 74.89293742824371) bank123539 +123540 POINT(40.726402101830615 74.45999709176448) bank123540 +123541 POINT(41.09029535366543 74.48291519556481) bank123541 +123542 POINT(40.42314538057703 73.09894245561651) bank123542 +123543 POINT(39.99355057981394 73.70113981098598) bank123543 +123544 POINT(39.94912674780753 74.5393607576521) bank123544 +123545 POINT(40.65708582641859 73.45898752885964) bank123545 +123546 POINT(41.026402114421515 74.77636745215462) bank123546 +123547 POINT(40.86657711708852 74.7012650978969) bank123547 +123548 POINT(41.27261527061726 74.25166755733248) bank123548 +123549 POINT(39.72015743060169 74.43022739479541) bank123549 +123550 POINT(39.97242305474907 73.70471175445395) bank123550 +123551 POINT(40.40583515411261 73.99461901464737) bank123551 +123552 POINT(41.52973873535278 74.34559181897838) bank123552 +123553 POINT(40.481921757257375 73.67905871927918) bank123553 +123554 POINT(40.98693245068033 73.8912270079618) bank123554 +123555 POINT(40.10834012394064 73.69946642162242) bank123555 +123556 POINT(41.41314579250398 74.04410389372585) bank123556 +123557 POINT(40.34985671607423 73.87037527123809) bank123557 +123558 POINT(39.993196883587615 74.21857131905455) bank123558 +123559 POINT(39.749673105657116 73.5597763674742) bank123559 +123560 POINT(41.251705470661854 73.4382481933333) bank123560 +123561 POINT(40.71801595560383 73.37463363128556) bank123561 +123562 POINT(41.26835626496402 74.68622158993796) bank123562 +123563 POINT(40.761787626308006 73.96273251818864) bank123563 +123564 POINT(40.72021550636059 74.98410942311382) bank123564 +123565 POINT(40.266632624756255 74.76652503423743) bank123565 +123566 POINT(41.354590340576635 74.18559159780793) bank123566 +123567 POINT(40.57540359828053 73.88554284787686) bank123567 +123568 POINT(39.96262658056429 74.41169482038637) bank123568 +123569 POINT(41.6248780613395 74.06251082949126) bank123569 +123570 POINT(41.58168275514877 73.16602551441855) bank123570 +123571 POINT(40.834269658346074 74.1348729700219) bank123571 +123572 POINT(41.48152094835782 73.50671180098136) bank123572 +123573 POINT(41.648614858179904 74.67776571918962) bank123573 +123574 POINT(40.133920661068935 73.68491106498287) bank123574 +123575 POINT(39.84905494016686 73.03568317330543) bank123575 +123576 POINT(41.63518407604153 73.29964712881197) bank123576 +123577 POINT(41.62608138752127 73.0344023881066) bank123577 +123578 POINT(40.75228898541738 74.92765037650167) bank123578 +123579 POINT(40.3777701332349 74.25804759675448) bank123579 +123580 POINT(41.13084514973593 73.79088797565691) bank123580 +123581 POINT(41.4440342521727 73.74901323531095) bank123581 +123582 POINT(40.04890477411115 73.16268721087236) bank123582 +123583 POINT(40.24594043395553 73.20789904152414) bank123583 +123584 POINT(40.313616283585255 74.90565151304702) bank123584 +123585 POINT(40.97400936957587 74.6028016932752) bank123585 +123586 POINT(41.11799338008062 73.24153917629118) bank123586 +123587 POINT(41.273668316013335 73.54553876605323) bank123587 +123588 POINT(40.91828387515713 73.54408217434002) bank123588 +123589 POINT(39.931344171895624 73.59477964372606) bank123589 +123590 POINT(41.548690395863844 73.9998684451173) bank123590 +123591 POINT(41.52365018494737 73.79983021815349) bank123591 +123592 POINT(40.975620212223134 73.30829654837882) bank123592 +123593 POINT(41.381388775225304 74.49439842673996) bank123593 +123594 POINT(40.48755249093494 74.34978796453585) bank123594 +123595 POINT(40.85887352253795 74.45986195647087) bank123595 +123596 POINT(41.003172089584346 73.4591623037474) bank123596 +123597 POINT(41.06670076121655 74.1857579663441) bank123597 +123598 POINT(40.1395391791455 73.12257889651394) bank123598 +123599 POINT(41.03808188475578 73.867537632802) bank123599 +123600 POINT(41.3072827827588 74.18734720672614) bank123600 +123601 POINT(40.620626092735236 74.3609137326523) bank123601 +123602 POINT(40.53614460704726 74.04753583410191) bank123602 +123603 POINT(40.47979288080546 73.57373999730767) bank123603 +123604 POINT(40.28572848551415 74.83381982341855) bank123604 +123605 POINT(40.39391151148523 74.9313335123554) bank123605 +123606 POINT(41.31061681234541 74.27940992206855) bank123606 +123607 POINT(40.987445240137895 73.47051547908195) bank123607 +123608 POINT(41.20890695341337 74.6189829356878) bank123608 +123609 POINT(40.07697780050324 73.93708011713393) bank123609 +123610 POINT(41.40923129831007 74.33652977477044) bank123610 +123611 POINT(40.82876977614723 74.87379959882821) bank123611 +123612 POINT(41.194657069565864 74.45640807847695) bank123612 +123613 POINT(40.49502440361345 74.90684461777832) bank123613 +123614 POINT(40.60041637726073 73.37176026969377) bank123614 +123615 POINT(39.794866330916676 74.8661040622391) bank123615 +123616 POINT(41.53201479619847 73.00722676597105) bank123616 +123617 POINT(41.05124263622571 74.85028003042945) bank123617 +123618 POINT(41.2686761821597 73.37630272222636) bank123618 +123619 POINT(40.04276929079191 74.82545966070316) bank123619 +123620 POINT(41.33582259482393 73.28760773407313) bank123620 +123621 POINT(40.237145775346164 73.73851037413989) bank123621 +123622 POINT(41.49250069139518 73.29064850588075) bank123622 +123623 POINT(40.95052206229207 73.58114787918689) bank123623 +123624 POINT(41.18743817658524 74.48295161782691) bank123624 +123625 POINT(41.60695948555796 74.25976732062561) bank123625 +123626 POINT(41.47489713479662 74.88928162075662) bank123626 +123627 POINT(41.685560676648684 74.08355284669734) bank123627 +123628 POINT(39.996636951601126 73.93603331717331) bank123628 +123629 POINT(41.236741662118014 74.95568302513374) bank123629 +123630 POINT(40.270753101671886 73.92024030952611) bank123630 +123631 POINT(40.78501024170106 73.41312510493411) bank123631 +123632 POINT(41.62058858440086 74.79755888368001) bank123632 +123633 POINT(40.714379085465296 73.9768649527187) bank123633 +123634 POINT(41.39299100320782 73.2165569730609) bank123634 +123635 POINT(39.849218465224254 73.6962396853377) bank123635 +123636 POINT(40.972372659967405 75.00345139573763) bank123636 +123637 POINT(39.82681886447934 73.88706069489979) bank123637 +123638 POINT(40.88641531849073 73.8900871252612) bank123638 +123639 POINT(39.98207815834745 73.35647441943114) bank123639 +123640 POINT(41.281989181668614 73.36935688066696) bank123640 +123641 POINT(40.77293071486326 73.98008266589062) bank123641 +123642 POINT(40.19658572544906 73.8281736848579) bank123642 +123643 POINT(39.893711880008894 73.20412679699011) bank123643 +123644 POINT(41.06022437769983 74.69279137606232) bank123644 +123645 POINT(40.55464318370825 73.4594759250622) bank123645 +123646 POINT(41.640685845526 73.68163237927305) bank123646 +123647 POINT(40.23105072293591 74.6788419956888) bank123647 +123648 POINT(41.328031858159385 73.1408813949489) bank123648 +123649 POINT(40.03691933132257 74.86205517090681) bank123649 +123650 POINT(40.26570795529464 73.15125074052307) bank123650 +123651 POINT(41.685058894651185 74.30322442541993) bank123651 +123652 POINT(40.30543447254029 74.52463505836346) bank123652 +123653 POINT(41.459492611567796 74.39133490336359) bank123653 +123654 POINT(40.64997741043705 74.74366298749118) bank123654 +123655 POINT(41.519853042155695 73.25983103735683) bank123655 +123656 POINT(40.45167293321037 73.83531011511324) bank123656 +123657 POINT(40.26556363072693 74.88096412588689) bank123657 +123658 POINT(41.45786424555035 73.10355955134699) bank123658 +123659 POINT(41.680560213013834 73.3153569209615) bank123659 +123660 POINT(40.0384230068335 74.92171242537607) bank123660 +123661 POINT(40.14801139178756 73.44708269944168) bank123661 +123662 POINT(40.68854234015406 74.11132595807828) bank123662 +123663 POINT(40.52849367222062 73.08698870258114) bank123663 +123664 POINT(40.44518815996677 74.72920948169869) bank123664 +123665 POINT(40.05710694782865 73.12422062095743) bank123665 +123666 POINT(40.49298402820469 73.22202280619514) bank123666 +123667 POINT(41.36066229798388 73.21054704729183) bank123667 +123668 POINT(39.71665451589916 73.7948093794062) bank123668 +123669 POINT(40.67964215606228 74.44392464652535) bank123669 +123670 POINT(40.85403309092122 74.46822649169223) bank123670 +123671 POINT(39.91875292677641 73.99944955197104) bank123671 +123672 POINT(40.799912554663585 74.37486502938435) bank123672 +123673 POINT(40.91068963106667 74.05006429610724) bank123673 +123674 POINT(41.21836016996087 73.9709293644406) bank123674 +123675 POINT(39.86087345405967 74.54645064595927) bank123675 +123676 POINT(41.50219975853642 73.47474467793273) bank123676 +123677 POINT(41.662290212105376 73.98236382942723) bank123677 +123678 POINT(41.63686302654399 74.5376713888834) bank123678 +123679 POINT(40.837235301405855 74.61712935958944) bank123679 +123680 POINT(40.01745499986036 74.94054746645601) bank123680 +123681 POINT(39.88671583994879 74.141992276959) bank123681 +123682 POINT(40.17976717526147 73.98736760372724) bank123682 +123683 POINT(40.87186145754182 74.66419668148485) bank123683 +123684 POINT(41.07420896028059 73.38352692001205) bank123684 +123685 POINT(40.39544248365196 73.90419897679172) bank123685 +123686 POINT(40.020035845534025 73.82333602441248) bank123686 +123687 POINT(41.43610812884767 73.38919919879186) bank123687 +123688 POINT(40.34720531131032 73.95678345625937) bank123688 +123689 POINT(41.63504161812822 73.1060851963984) bank123689 +123690 POINT(40.06169511479501 73.65825605548511) bank123690 +123691 POINT(39.874728223687875 73.53398409015394) bank123691 +123692 POINT(40.378727355880294 74.72755739648882) bank123692 +123693 POINT(40.18403133740253 74.19489487272864) bank123693 +123694 POINT(40.74767133650755 74.41202943970927) bank123694 +123695 POINT(41.570190983999176 73.5216543587601) bank123695 +123696 POINT(39.805973442765804 74.54467258205013) bank123696 +123697 POINT(40.98668530177998 74.42992478806916) bank123697 +123698 POINT(41.24032830768845 74.20118239797515) bank123698 +123699 POINT(41.67489026748183 74.64387651465859) bank123699 +123700 POINT(40.536227740353084 74.34673221171255) bank123700 +123701 POINT(40.727791039948364 74.54027698938322) bank123701 +123702 POINT(41.209858766022535 73.125331442499) bank123702 +123703 POINT(41.05610291969245 74.05961048579277) bank123703 +123704 POINT(39.88494806937701 74.33301411912956) bank123704 +123705 POINT(41.6659703410347 73.04552010566245) bank123705 +123706 POINT(41.689787937008305 74.78419695743116) bank123706 +123707 POINT(40.72804071102515 74.78390164172522) bank123707 +123708 POINT(41.442271056447794 73.0475181166609) bank123708 +123709 POINT(41.16297718983947 73.71934785572041) bank123709 +123710 POINT(40.56300241660123 74.3133378264822) bank123710 +123711 POINT(40.455460081090855 73.84597254981429) bank123711 +123712 POINT(40.65017462955087 74.56729580508141) bank123712 +123713 POINT(41.07394359762861 74.48729953684919) bank123713 +123714 POINT(40.90453811939203 73.93108982017856) bank123714 +123715 POINT(40.671550685873015 73.53074730251981) bank123715 +123716 POINT(41.49353871322511 73.71293941914638) bank123716 +123717 POINT(40.209736934246756 74.52554485185843) bank123717 +123718 POINT(40.68340918944476 74.01967490919245) bank123718 +123719 POINT(41.0124042297538 74.61616646638777) bank123719 +123720 POINT(40.86228579750919 73.43608839915173) bank123720 +123721 POINT(41.13939332735972 73.98057939989255) bank123721 +123722 POINT(40.66798589677051 73.41774185333804) bank123722 +123723 POINT(40.78167991636768 73.80517175242622) bank123723 +123724 POINT(41.545561526862045 73.2606200885454) bank123724 +123725 POINT(39.90642973430237 74.1360430690023) bank123725 +123726 POINT(41.610970376532336 74.86231609136954) bank123726 +123727 POINT(41.40244075930475 73.33552899584753) bank123727 +123728 POINT(40.57394223407306 74.79736977298488) bank123728 +123729 POINT(40.97106358413023 74.13798551025646) bank123729 +123730 POINT(41.46139566733332 73.58823006233852) bank123730 +123731 POINT(41.42149668676649 74.44322876289306) bank123731 +123732 POINT(39.963374694686514 73.50793576068018) bank123732 +123733 POINT(41.31729351367598 74.30253307113195) bank123733 +123734 POINT(41.37487215249921 73.69984400568981) bank123734 +123735 POINT(39.97153509076982 74.86255813793545) bank123735 +123736 POINT(41.442987080272516 73.62696609511094) bank123736 +123737 POINT(40.49858692705273 73.22129394535169) bank123737 +123738 POINT(41.51507804363454 73.8487663608704) bank123738 +123739 POINT(40.95090283456743 74.47095785992047) bank123739 +123740 POINT(41.22920507677972 74.18098275851328) bank123740 +123741 POINT(40.28290214259748 73.6925045782903) bank123741 +123742 POINT(40.59271977066378 73.62562728689706) bank123742 +123743 POINT(40.01734796573817 73.44781133843958) bank123743 +123744 POINT(40.05186002412842 74.54736172795079) bank123744 +123745 POINT(39.998933797170416 74.46865795398605) bank123745 +123746 POINT(39.90206876916122 74.58203139189402) bank123746 +123747 POINT(40.48640784027393 74.84219731217057) bank123747 +123748 POINT(39.86610365753427 74.49711500873569) bank123748 +123749 POINT(40.7705070125418 74.27258121927605) bank123749 +123750 POINT(40.57204862162949 74.31085818394611) bank123750 +123751 POINT(40.6093956344916 73.06058686963368) bank123751 +123752 POINT(40.23890161667993 73.62499786119847) bank123752 +123753 POINT(39.939307085440966 74.19651619043191) bank123753 +123754 POINT(41.36778050310712 74.07212677857441) bank123754 +123755 POINT(41.02911885554048 74.60544873539718) bank123755 +123756 POINT(40.260671785086565 73.31938791703752) bank123756 +123757 POINT(41.41714966842381 74.79080480387951) bank123757 +123758 POINT(40.856919396248095 74.36829986876033) bank123758 +123759 POINT(40.29292880645391 73.46310373049785) bank123759 +123760 POINT(40.66461724015352 74.95947609953245) bank123760 +123761 POINT(40.66343782010796 74.85328997551417) bank123761 +123762 POINT(40.82018562723501 73.56836184659399) bank123762 +123763 POINT(40.867201994372486 74.61985772990664) bank123763 +123764 POINT(41.31828399897707 73.58679631636002) bank123764 +123765 POINT(40.10591389544289 74.89804011506462) bank123765 +123766 POINT(41.2665982655238 73.1410520048284) bank123766 +123767 POINT(40.10979772686201 74.68756680793744) bank123767 +123768 POINT(40.64524110721739 74.37787756612096) bank123768 +123769 POINT(41.08836741144526 73.23244302607094) bank123769 +123770 POINT(40.49956922157469 74.45423253441008) bank123770 +123771 POINT(40.159013355637725 74.08070583425132) bank123771 +123772 POINT(39.88181647423852 74.93886501089733) bank123772 +123773 POINT(40.955073433768526 73.98186067572793) bank123773 +123774 POINT(40.07045083430057 73.98108803389032) bank123774 +123775 POINT(40.858522447722066 73.53296906099311) bank123775 +123776 POINT(39.71687411033425 73.9896662018026) bank123776 +123777 POINT(40.71863287762469 73.15895364483943) bank123777 +123778 POINT(40.53303677804096 73.90045606160514) bank123778 +123779 POINT(39.747031979800504 73.14254861643828) bank123779 +123780 POINT(41.020801253253836 73.52621748410797) bank123780 +123781 POINT(40.458752104849296 74.13391509255524) bank123781 +123782 POINT(39.768879206173224 73.65453795977442) bank123782 +123783 POINT(40.427943875796146 74.85412467099667) bank123783 +123784 POINT(39.93294272774001 73.1687309621806) bank123784 +123785 POINT(40.98879007624416 74.14190827828119) bank123785 +123786 POINT(40.09660878275622 73.75583741301199) bank123786 +123787 POINT(41.68583700972554 74.02559922872288) bank123787 +123788 POINT(39.951584678045954 73.94602435107535) bank123788 +123789 POINT(40.28167187930764 73.99721772868399) bank123789 +123790 POINT(41.176548243709554 74.67174350277614) bank123790 +123791 POINT(40.34952758330562 73.30637169799502) bank123791 +123792 POINT(39.90551427549554 73.40576967971161) bank123792 +123793 POINT(41.60234889437071 74.7458645474926) bank123793 +123794 POINT(41.008829199671226 74.78129144814845) bank123794 +123795 POINT(41.00533168098892 74.53131045620948) bank123795 +123796 POINT(40.07005708382961 74.86067110539591) bank123796 +123797 POINT(41.52409595210668 74.95789235771338) bank123797 +123798 POINT(40.026281575770334 74.4013203317307) bank123798 +123799 POINT(41.66701642340991 73.42511323071444) bank123799 +123800 POINT(40.14309797578313 74.08883828782771) bank123800 +123801 POINT(41.27044908975805 74.3953536737278) bank123801 +123802 POINT(41.59798109304366 74.81572195797658) bank123802 +123803 POINT(40.312858911371556 73.58414666250425) bank123803 +123804 POINT(40.41315252637738 73.4126201699797) bank123804 +123805 POINT(41.671419810017056 74.45800030949957) bank123805 +123806 POINT(40.64805900761549 74.42251724868343) bank123806 +123807 POINT(39.78140230567077 74.31907391736708) bank123807 +123808 POINT(41.65939538339012 73.52054885424079) bank123808 +123809 POINT(41.03210639035812 74.56079498814513) bank123809 +123810 POINT(40.071630085489026 74.70756565995933) bank123810 +123811 POINT(41.05859606250564 73.02747822077755) bank123811 +123812 POINT(40.77951534218973 74.22263837829257) bank123812 +123813 POINT(41.007359539248554 74.65366211853156) bank123813 +123814 POINT(41.3450391309685 73.84159702660337) bank123814 +123815 POINT(39.897917032248955 73.78862174268619) bank123815 +123816 POINT(40.82955846133724 73.0466445619923) bank123816 +123817 POINT(41.33856488522233 74.81702747862236) bank123817 +123818 POINT(41.52185599589624 73.47438826200943) bank123818 +123819 POINT(40.50533588467565 73.64301882313215) bank123819 +123820 POINT(41.19306326157317 73.48487973756656) bank123820 +123821 POINT(40.48310311022946 73.88797856078992) bank123821 +123822 POINT(41.43293692513249 73.96853777378378) bank123822 +123823 POINT(41.23543585735155 73.95437164203322) bank123823 +123824 POINT(41.36934840753996 74.8964423174524) bank123824 +123825 POINT(40.2776513568245 74.5403935282356) bank123825 +123826 POINT(40.96974047619708 73.17585565913419) bank123826 +123827 POINT(41.12927974697646 73.1275059948187) bank123827 +123828 POINT(40.73803338337797 73.96652716713923) bank123828 +123829 POINT(40.99171303538111 73.87970666867788) bank123829 +123830 POINT(41.685410802270624 73.98816328617349) bank123830 +123831 POINT(39.758565678541856 73.05488654675608) bank123831 +123832 POINT(40.83938254674766 73.25486714698431) bank123832 +123833 POINT(40.45297488165344 73.34977233582593) bank123833 +123834 POINT(41.66628815751333 73.32256323059366) bank123834 +123835 POINT(41.19350284512792 73.98397130425623) bank123835 +123836 POINT(41.15597143428352 74.67594493242149) bank123836 +123837 POINT(40.02938693151847 74.55453693235519) bank123837 +123838 POINT(40.8015592231806 74.64423185887668) bank123838 +123839 POINT(40.622749153421545 73.82419300249822) bank123839 +123840 POINT(40.03653012772734 73.74281932240001) bank123840 +123841 POINT(41.705952573592604 73.42984258998128) bank123841 +123842 POINT(40.81801266529756 74.50763720409213) bank123842 +123843 POINT(41.328016042467304 74.10032985226842) bank123843 +123844 POINT(41.48486229178361 74.35167200074588) bank123844 +123845 POINT(39.817241358155876 74.27339130629791) bank123845 +123846 POINT(41.39997421420418 73.07797332958788) bank123846 +123847 POINT(41.127547379753416 73.91109030023127) bank123847 +123848 POINT(40.305410610017795 73.7799905307055) bank123848 +123849 POINT(39.7415537665181 73.163346971429) bank123849 +123850 POINT(40.35795144629439 74.77393322256123) bank123850 +123851 POINT(40.88793998963984 73.35502513775988) bank123851 +123852 POINT(40.91209835247151 74.45940729392518) bank123852 +123853 POINT(40.161620006951146 73.8125817778044) bank123853 +123854 POINT(41.322098010292734 74.64916115775596) bank123854 +123855 POINT(41.631350178654884 74.55526025159939) bank123855 +123856 POINT(39.98589311422192 74.03868445382489) bank123856 +123857 POINT(40.49728748418133 73.1684775670735) bank123857 +123858 POINT(41.0995055678848 74.11717582795583) bank123858 +123859 POINT(40.53371734315903 73.93323724209364) bank123859 +123860 POINT(40.79572911742133 73.1240226851935) bank123860 +123861 POINT(39.768706950252145 73.06558392988929) bank123861 +123862 POINT(40.57985776835864 73.70797572859458) bank123862 +123863 POINT(40.4507578946898 74.58022556098551) bank123863 +123864 POINT(40.29038677634297 74.1317399210626) bank123864 +123865 POINT(41.44133082697985 73.18511392203499) bank123865 +123866 POINT(39.96341407769938 74.7812799479213) bank123866 +123867 POINT(41.47021992964599 73.80087574718259) bank123867 +123868 POINT(40.24354238154312 73.06473484156236) bank123868 +123869 POINT(41.70610902765737 74.7732731124896) bank123869 +123870 POINT(41.463855876188106 73.52334946112006) bank123870 +123871 POINT(41.66035282766167 74.38945847221854) bank123871 +123872 POINT(41.60908485439921 74.23107126668238) bank123872 +123873 POINT(40.18934636419951 73.92214930066662) bank123873 +123874 POINT(41.422926371829774 74.16773238387543) bank123874 +123875 POINT(41.12772390370508 74.80122505301438) bank123875 +123876 POINT(40.113359043865806 74.74912621210783) bank123876 +123877 POINT(40.313444600543264 74.61057530013096) bank123877 +123878 POINT(39.74620893898607 74.47634013154463) bank123878 +123879 POINT(40.066608774587095 74.9387502949298) bank123879 +123880 POINT(40.8757249519123 73.18487831294497) bank123880 +123881 POINT(40.136552741084714 73.67202431849152) bank123881 +123882 POINT(41.59704196458401 73.41985604592386) bank123882 +123883 POINT(39.87394622949333 73.13205439875381) bank123883 +123884 POINT(41.46242599684505 74.4216494512697) bank123884 +123885 POINT(39.733219286901615 74.37362958034015) bank123885 +123886 POINT(41.710693335004024 74.84211132037234) bank123886 +123887 POINT(40.55376168996523 73.26308609298981) bank123887 +123888 POINT(39.71578907679728 73.96811198167316) bank123888 +123889 POINT(40.16241906017322 74.03182863282906) bank123889 +123890 POINT(41.68812911805575 74.59219980921607) bank123890 +123891 POINT(40.649541948208125 74.02799109150894) bank123891 +123892 POINT(41.64079236991236 73.75642688385105) bank123892 +123893 POINT(39.869971622381506 74.83609580304311) bank123893 +123894 POINT(41.09623818879397 73.3643723895935) bank123894 +123895 POINT(41.197364653640776 74.08309086749135) bank123895 +123896 POINT(39.87969879961873 74.92339409709955) bank123896 +123897 POINT(40.48122726379885 73.02784085848305) bank123897 +123898 POINT(40.3933885125893 74.03326270007021) bank123898 +123899 POINT(40.79177203096855 74.84066565891722) bank123899 +123900 POINT(40.30627854761012 74.27642209134734) bank123900 +123901 POINT(41.050680060209324 73.04176356274792) bank123901 +123902 POINT(40.55499044952348 75.00025805153588) bank123902 +123903 POINT(40.13960984005903 73.25232238894304) bank123903 +123904 POINT(40.808441847580404 74.40464938749878) bank123904 +123905 POINT(39.76679103967622 73.04547000057809) bank123905 +123906 POINT(39.90026901017565 74.03401633561857) bank123906 +123907 POINT(40.75726817611694 73.36948243693634) bank123907 +123908 POINT(40.55921043213535 73.84160702258539) bank123908 +123909 POINT(40.554908275171 73.33258548439777) bank123909 +123910 POINT(40.911349957816135 74.68778417045651) bank123910 +123911 POINT(40.56548412319151 74.70156470049123) bank123911 +123912 POINT(40.975475120018295 74.12798323826651) bank123912 +123913 POINT(41.14027099049754 73.5909563290835) bank123913 +123914 POINT(41.363461650336156 74.58906030579) bank123914 +123915 POINT(39.72909080062145 73.53653453652545) bank123915 +123916 POINT(40.29602834683323 73.76268689946023) bank123916 +123917 POINT(41.20971853593927 73.28021274491543) bank123917 +123918 POINT(41.23579612678482 74.50673623985136) bank123918 +123919 POINT(39.80397323731406 74.4489154094449) bank123919 +123920 POINT(40.89885656881344 73.03925554900943) bank123920 +123921 POINT(40.40955507398505 74.58931207289537) bank123921 +123922 POINT(40.56387431424019 74.30126575089416) bank123922 +123923 POINT(40.62056650748831 73.9830722182024) bank123923 +123924 POINT(39.75918500403861 73.59354913400944) bank123924 +123925 POINT(40.32688049284609 73.25912840062084) bank123925 +123926 POINT(40.05168011017813 74.24653925412449) bank123926 +123927 POINT(40.97647018589601 74.16224999532517) bank123927 +123928 POINT(40.2413133341516 74.9054860796983) bank123928 +123929 POINT(39.89322695562003 74.03636525438534) bank123929 +123930 POINT(40.924354052617524 74.47251526179924) bank123930 +123931 POINT(41.51900560163053 73.29930757919428) bank123931 +123932 POINT(41.46627642904737 74.53905376728224) bank123932 +123933 POINT(41.24541574252038 74.76924584028744) bank123933 +123934 POINT(40.971902796801274 73.17365471806187) bank123934 +123935 POINT(40.09594522297837 74.33275337560478) bank123935 +123936 POINT(41.05403054977143 73.19479588707817) bank123936 +123937 POINT(41.064143226610895 74.03233795865376) bank123937 +123938 POINT(41.406138757557024 73.83563690368759) bank123938 +123939 POINT(41.17662492673828 74.08627148485903) bank123939 +123940 POINT(39.76491919248296 73.38883414651039) bank123940 +123941 POINT(39.815952332945045 74.54858011000476) bank123941 +123942 POINT(41.70521998570496 74.42851313760914) bank123942 +123943 POINT(40.570311263369 73.26167420070175) bank123943 +123944 POINT(41.00928619104458 73.14758234731922) bank123944 +123945 POINT(40.24092615104107 74.80843948990139) bank123945 +123946 POINT(41.0228337441199 73.54717612119134) bank123946 +123947 POINT(40.74693352305927 73.38213444053375) bank123947 +123948 POINT(40.145666359701444 73.60892431022346) bank123948 +123949 POINT(40.66756610742668 74.83769664429474) bank123949 +123950 POINT(40.76447150703303 73.01218007128368) bank123950 +123951 POINT(41.30645508905805 74.1369785030809) bank123951 +123952 POINT(40.64751935031576 73.65988108849844) bank123952 +123953 POINT(41.22547561501785 73.69004669725172) bank123953 +123954 POINT(40.92998068062018 74.34246663747783) bank123954 +123955 POINT(39.92354356652278 74.88757926822264) bank123955 +123956 POINT(40.27224595734986 73.13344711323506) bank123956 +123957 POINT(40.55887519846088 73.02777884983664) bank123957 +123958 POINT(40.46901379710217 73.50206409795116) bank123958 +123959 POINT(40.25868312965184 73.86356262012406) bank123959 +123960 POINT(41.22285411427848 74.22440991930429) bank123960 +123961 POINT(41.67561485442457 73.88096286100223) bank123961 +123962 POINT(40.09751786769727 73.21536621942779) bank123962 +123963 POINT(40.57928435519135 74.94323215247185) bank123963 +123964 POINT(40.19297111348692 73.19620790808297) bank123964 +123965 POINT(40.806475115006776 74.29148788163461) bank123965 +123966 POINT(40.08532279948979 73.23649291928788) bank123966 +123967 POINT(41.25687829202531 73.01967365893519) bank123967 +123968 POINT(40.669551635508405 74.25316110480809) bank123968 +123969 POINT(39.97432020172525 74.25048735357645) bank123969 +123970 POINT(40.18024707323433 73.10316759996483) bank123970 +123971 POINT(41.0468070450418 74.61208681016554) bank123971 +123972 POINT(40.53141731295219 74.4110186104963) bank123972 +123973 POINT(40.3786459724742 74.03431789883263) bank123973 +123974 POINT(40.10553990989906 74.53892569476294) bank123974 +123975 POINT(41.314194099584604 74.54644023614816) bank123975 +123976 POINT(41.475644949465725 74.5303095010376) bank123976 +123977 POINT(39.93385961939335 74.75568363297545) bank123977 +123978 POINT(41.31743237097954 74.6205992673584) bank123978 +123979 POINT(41.47454030006407 74.67579207477137) bank123979 +123980 POINT(41.363325873600594 73.10034758645526) bank123980 +123981 POINT(39.722565674488465 74.57391858778516) bank123981 +123982 POINT(40.14440566324537 73.66308343166847) bank123982 +123983 POINT(40.387777576334116 73.635319422355) bank123983 +123984 POINT(41.43217629680904 74.31326621466235) bank123984 +123985 POINT(39.75628527942623 74.67002894716182) bank123985 +123986 POINT(41.352580361955354 74.1450916202821) bank123986 +123987 POINT(40.521634288489444 74.49028946883611) bank123987 +123988 POINT(40.814952664100225 73.51779212186175) bank123988 +123989 POINT(40.7206038038829 73.8923059314122) bank123989 +123990 POINT(39.74249476409172 73.23462952803041) bank123990 +123991 POINT(40.28717326086809 74.13461508679462) bank123991 +123992 POINT(41.5723526322944 74.13477965585722) bank123992 +123993 POINT(40.6930311652196 73.36143810954464) bank123993 +123994 POINT(40.339355655910154 73.43943485054682) bank123994 +123995 POINT(40.25105842474502 73.02027055220326) bank123995 +123996 POINT(40.88481684869479 73.99424570290346) bank123996 +123997 POINT(40.61988417033801 74.52056739368001) bank123997 +123998 POINT(40.9206206968686 73.90818722118507) bank123998 +123999 POINT(40.879908470753826 73.20414418949203) bank123999 +124000 POINT(41.67133084917809 73.50287594506975) bank124000 +124001 POINT(41.333410913052724 74.76048230320805) bank124001 +124002 POINT(41.11664850366639 73.49934718878869) bank124002 +124003 POINT(40.968748347956726 73.95592765951405) bank124003 +124004 POINT(41.1423708608855 74.4226497347365) bank124004 +124005 POINT(40.17018269774388 73.75589913429816) bank124005 +124006 POINT(41.10394990076291 73.3683001785234) bank124006 +124007 POINT(39.98225579762879 74.82740417750155) bank124007 +124008 POINT(41.68717652533235 73.16840418795448) bank124008 +124009 POINT(39.93394486125565 74.03106440589598) bank124009 +124010 POINT(39.84194828102847 74.24092796995677) bank124010 +124011 POINT(40.06124978840219 73.90842312682689) bank124011 +124012 POINT(39.93120521681702 73.12895977388608) bank124012 +124013 POINT(40.94713724018342 73.39169353202833) bank124013 +124014 POINT(41.446419551607555 74.09631530151057) bank124014 +124015 POINT(40.988936829872756 73.33553742109284) bank124015 +124016 POINT(40.09792562933482 74.29494337188112) bank124016 +124017 POINT(39.78545581956546 73.28809442535959) bank124017 +124018 POINT(41.0981943709752 74.56795375022817) bank124018 +124019 POINT(40.726981236363855 74.58156195330342) bank124019 +124020 POINT(40.60979550577011 73.96779515576843) bank124020 +124021 POINT(40.30082724199617 73.86947937643427) bank124021 +124022 POINT(39.86678471969872 73.30065193206731) bank124022 +124023 POINT(41.54936840404933 74.15856252500839) bank124023 +124024 POINT(39.79776019931012 74.10735069704366) bank124024 +124025 POINT(40.931354517458956 73.46826007923876) bank124025 +124026 POINT(41.66894366873487 74.84864390819511) bank124026 +124027 POINT(41.02585916562444 74.63961125990906) bank124027 +124028 POINT(41.399019068157784 74.23321958466384) bank124028 +124029 POINT(41.61950459015558 73.58084647163558) bank124029 +124030 POINT(39.88721122838066 73.24043354319295) bank124030 +124031 POINT(40.884558164140756 73.6813667589349) bank124031 +124032 POINT(41.195716426335366 74.47741372908155) bank124032 +124033 POINT(41.02837222422632 73.31169246529294) bank124033 +124034 POINT(41.32249464799652 74.18832538534896) bank124034 +124035 POINT(40.292833862326766 73.56840561879048) bank124035 +124036 POINT(40.197258261995465 74.77704599491156) bank124036 +124037 POINT(40.643628785045586 73.51948649272198) bank124037 +124038 POINT(41.663081443748354 73.85227200030502) bank124038 +124039 POINT(40.74501848641013 73.60414510881441) bank124039 +124040 POINT(41.69416447557975 73.05140671924285) bank124040 +124041 POINT(39.89104317261103 74.21681821724165) bank124041 +124042 POINT(40.38319587322012 73.6935247143347) bank124042 +124043 POINT(40.25352239891438 73.85506435135642) bank124043 +124044 POINT(41.526820651818696 73.27716643482306) bank124044 +124045 POINT(40.57932948556586 73.24235618467613) bank124045 +124046 POINT(40.08232569783265 74.99428169738621) bank124046 +124047 POINT(40.13417369521081 74.10072393228874) bank124047 +124048 POINT(41.29628076995153 73.82677927253162) bank124048 +124049 POINT(40.844731958522615 74.77671032733569) bank124049 +124050 POINT(40.526139943130424 74.89707614896362) bank124050 +124051 POINT(39.87427731581384 74.1720804864681) bank124051 +124052 POINT(41.288615631917295 74.95701484931112) bank124052 +124053 POINT(41.46544199475963 74.34823715789673) bank124053 +124054 POINT(40.49245108209882 73.79682632098553) bank124054 +124055 POINT(40.17150745201803 73.40380971952713) bank124055 +124056 POINT(40.841536425484335 74.51390473664426) bank124056 +124057 POINT(39.92437794568108 73.10994433313496) bank124057 +124058 POINT(40.84834815294497 73.67035340992395) bank124058 +124059 POINT(39.824725872807576 74.45280645931639) bank124059 +124060 POINT(41.28838038845927 73.70159465213722) bank124060 +124061 POINT(40.76508026627838 74.92373185462269) bank124061 +124062 POINT(39.878865816392256 73.08353702475819) bank124062 +124063 POINT(41.463837351767 74.62819978822621) bank124063 +124064 POINT(39.85864641416356 73.15060367990085) bank124064 +124065 POINT(40.38671047128742 74.78566389476316) bank124065 +124066 POINT(39.82031706399773 74.4067655307562) bank124066 +124067 POINT(40.140692232904364 74.46425734398302) bank124067 +124068 POINT(41.68163493328911 73.14026135103201) bank124068 +124069 POINT(40.12245198818906 74.2543214343956) bank124069 +124070 POINT(41.00398029700984 74.74924535204178) bank124070 +124071 POINT(40.8314076079584 73.52611379808225) bank124071 +124072 POINT(41.63919414545251 73.30017662713153) bank124072 +124073 POINT(40.60869668164873 74.07430171917555) bank124073 +124074 POINT(40.40182418805068 74.93519273713109) bank124074 +124075 POINT(40.21700033198934 74.4615471641657) bank124075 +124076 POINT(40.140204362188584 73.8663135778611) bank124076 +124077 POINT(39.827043170815195 73.38824113482787) bank124077 +124078 POINT(40.594121047001074 73.76343440272818) bank124078 +124079 POINT(41.53262900572772 74.1113668555911) bank124079 +124080 POINT(39.728569007258535 74.3132105457444) bank124080 +124081 POINT(39.87229418644679 74.02313739819495) bank124081 +124082 POINT(41.22487016868247 73.87282149070622) bank124082 +124083 POINT(40.95230176352066 74.43704218141161) bank124083 +124084 POINT(41.28798070432413 73.15923271655694) bank124084 +124085 POINT(41.136150402858924 73.86422650271756) bank124085 +124086 POINT(41.450810117341746 74.4432662347122) bank124086 +124087 POINT(40.89917947504042 73.57723252253972) bank124087 +124088 POINT(40.260689291948644 74.69812082465351) bank124088 +124089 POINT(40.382865265965464 73.03193095134058) bank124089 +124090 POINT(41.653175612133936 73.55929161654812) bank124090 +124091 POINT(40.68719036988576 73.20409237121225) bank124091 +124092 POINT(40.81446854435054 74.2818551700473) bank124092 +124093 POINT(40.606084368943804 74.0055413519154) bank124093 +124094 POINT(40.98951049162925 74.48820203370852) bank124094 +124095 POINT(41.23914582217994 74.99366475108745) bank124095 +124096 POINT(39.71631751077919 73.55064226001348) bank124096 +124097 POINT(40.608326111363674 73.37342746257323) bank124097 +124098 POINT(40.91540824274176 73.71550823608561) bank124098 +124099 POINT(40.89025162570127 74.93197389926064) bank124099 +124100 POINT(40.79021323496913 73.08606551573607) bank124100 +124101 POINT(40.08203266487633 73.21025997597198) bank124101 +124102 POINT(41.11150560369329 73.79876311547936) bank124102 +124103 POINT(40.41867415095808 73.64881152846095) bank124103 +124104 POINT(41.15290111761355 73.41170623275372) bank124104 +124105 POINT(40.49600288980027 74.27193344138357) bank124105 +124106 POINT(41.60387184938261 73.9583039324573) bank124106 +124107 POINT(41.01675578323975 74.37686284845866) bank124107 +124108 POINT(40.27881490797293 73.69338821312469) bank124108 +124109 POINT(41.64270186871717 73.95073439664931) bank124109 +124110 POINT(40.8195599294943 73.83698369486902) bank124110 +124111 POINT(40.63080850852995 73.9800750991083) bank124111 +124112 POINT(40.78316496882964 73.68608878794136) bank124112 +124113 POINT(40.47017092584136 74.41480832992944) bank124113 +124114 POINT(41.46537711379196 73.9068454588632) bank124114 +124115 POINT(40.30796798265312 73.45755257162536) bank124115 +124116 POINT(40.38670543627468 74.53403528779776) bank124116 +124117 POINT(39.932637484777395 73.76972462364077) bank124117 +124118 POINT(40.92424467055387 74.41221350058247) bank124118 +124119 POINT(41.511235874206484 73.92696090827545) bank124119 +124120 POINT(41.5216471712576 73.60123778162665) bank124120 +124121 POINT(41.13555740566352 74.31895953482886) bank124121 +124122 POINT(40.00016938866788 73.152763891968) bank124122 +124123 POINT(40.8962235233838 73.13079150572531) bank124123 +124124 POINT(39.81313580543641 73.11852793133993) bank124124 +124125 POINT(40.92749340813059 73.90363394364239) bank124125 +124126 POINT(40.07724441325959 73.02489884185036) bank124126 +124127 POINT(40.16383520327347 74.29688018082057) bank124127 +124128 POINT(40.91407409131963 73.92269735636613) bank124128 +124129 POINT(40.00714944792822 73.48457818901828) bank124129 +124130 POINT(41.50396365397368 74.14777757199471) bank124130 +124131 POINT(41.49764866639965 74.26902700441607) bank124131 +124132 POINT(40.24888236203881 74.83868708293265) bank124132 +124133 POINT(41.446073827671775 73.08737492668124) bank124133 +124134 POINT(40.54395001119367 73.12234202945214) bank124134 +124135 POINT(41.4638018717775 74.39732124245229) bank124135 +124136 POINT(40.44900482854067 73.81284727162534) bank124136 +124137 POINT(41.47858947553784 73.36158979492735) bank124137 +124138 POINT(39.898792927259755 74.22302149080872) bank124138 +124139 POINT(40.700582839505074 73.65345949756762) bank124139 +124140 POINT(40.379765293161256 74.28820627601387) bank124140 +124141 POINT(40.826037331951724 73.84344879885838) bank124141 +124142 POINT(41.230124197954446 73.0938433316311) bank124142 +124143 POINT(41.31495857999918 74.1174071314463) bank124143 +124144 POINT(41.23210990543099 73.29291868396338) bank124144 +124145 POINT(41.39661239462925 74.00158092528066) bank124145 +124146 POINT(41.33863304841921 73.63064741341444) bank124146 +124147 POINT(40.37388416031305 74.40569185916983) bank124147 +124148 POINT(40.848054265657055 74.70428175969357) bank124148 +124149 POINT(40.659560770817535 73.24339619173581) bank124149 +124150 POINT(40.158429912996255 73.53618720999611) bank124150 +124151 POINT(40.69309692264071 74.72247357189609) bank124151 +124152 POINT(41.11254276659639 74.85168182377959) bank124152 +124153 POINT(41.25647764895198 73.90278498700287) bank124153 +124154 POINT(40.713956992081734 74.59019282540172) bank124154 +124155 POINT(40.59799022055883 74.32873372566547) bank124155 +124156 POINT(39.94384699462405 74.47397482955107) bank124156 +124157 POINT(40.28171756723438 74.42242263665659) bank124157 +124158 POINT(40.40256181410984 73.30013483191766) bank124158 +124159 POINT(40.76492806878812 73.20951120202123) bank124159 +124160 POINT(40.985314376619534 74.32313813831811) bank124160 +124161 POINT(41.02905448558049 74.06862950314162) bank124161 +124162 POINT(41.534068774244886 74.69613250255081) bank124162 +124163 POINT(39.811675821594555 73.90678524948572) bank124163 +124164 POINT(39.88136969505891 73.91688599526323) bank124164 +124165 POINT(40.64128250313782 74.64104086932063) bank124165 +124166 POINT(40.40489078924854 74.34438802336386) bank124166 +124167 POINT(40.99390243311596 74.70165131946054) bank124167 +124168 POINT(41.67417115876584 74.80841716287124) bank124168 +124169 POINT(40.52913487629399 74.07803867405099) bank124169 +124170 POINT(40.16254349132104 74.63029468563205) bank124170 +124171 POINT(39.87214669435432 73.34886652319379) bank124171 +124172 POINT(41.39170004423995 73.1916142280084) bank124172 +124173 POINT(41.62326550959329 73.31702782308678) bank124173 +124174 POINT(41.50464742529432 74.09989000970397) bank124174 +124175 POINT(40.93366094865592 74.58582954942763) bank124175 +124176 POINT(39.729897108321005 74.15394735514889) bank124176 +124177 POINT(40.3444605755877 74.85863735999382) bank124177 +124178 POINT(39.76997619369706 74.45435142088594) bank124178 +124179 POINT(40.82046214364817 74.3654869033513) bank124179 +124180 POINT(39.78364838078451 73.29287528924824) bank124180 +124181 POINT(40.241359152864746 74.54702202491924) bank124181 +124182 POINT(40.90209878634341 73.3646575008958) bank124182 +124183 POINT(40.98631344514925 74.53421902663237) bank124183 +124184 POINT(40.944577547874296 74.56365873966038) bank124184 +124185 POINT(40.067010780122835 73.32872126699546) bank124185 +124186 POINT(40.59390121322207 73.86751726643449) bank124186 +124187 POINT(40.16428806806778 73.81972154757204) bank124187 +124188 POINT(41.092116595058535 74.5907821826313) bank124188 +124189 POINT(40.87956995270802 74.2437529782814) bank124189 +124190 POINT(40.96835907212557 74.02637210711767) bank124190 +124191 POINT(40.30112809829833 74.399782195267) bank124191 +124192 POINT(40.97630147631905 74.56886871402227) bank124192 +124193 POINT(39.80915521930274 74.21135245367884) bank124193 +124194 POINT(41.499274382929784 74.22818172965205) bank124194 +124195 POINT(40.424739878931405 74.6928728966122) bank124195 +124196 POINT(40.12585139792374 74.37063076316862) bank124196 +124197 POINT(40.680072199027315 73.40710353556499) bank124197 +124198 POINT(40.84119913328088 74.78438770039533) bank124198 +124199 POINT(39.7661302287056 74.48633790457329) bank124199 +124200 POINT(40.41647376783673 74.10715256997028) bank124200 +124201 POINT(40.52681558232149 73.49256692464738) bank124201 +124202 POINT(41.30938260612622 73.98390314125604) bank124202 +124203 POINT(40.63650033661261 74.97216808133756) bank124203 +124204 POINT(40.99129581727695 74.05109503536838) bank124204 +124205 POINT(40.89624210602025 74.5016369317344) bank124205 +124206 POINT(41.279533938123265 73.10432459335718) bank124206 +124207 POINT(41.394906691524554 73.26135519383745) bank124207 +124208 POINT(40.03915228607822 73.23226712464545) bank124208 +124209 POINT(39.750827380431275 73.50763657494079) bank124209 +124210 POINT(40.18872457051419 75.00281387186232) bank124210 +124211 POINT(41.52899191089409 73.13940482733597) bank124211 +124212 POINT(39.71721692358767 74.4419681399261) bank124212 +124213 POINT(41.5308594848257 73.88085242018164) bank124213 +124214 POINT(41.50832703339137 73.74504225308621) bank124214 +124215 POINT(40.89014850742811 73.41532597838591) bank124215 +124216 POINT(39.94805976284556 73.27782801143091) bank124216 +124217 POINT(41.4457260958567 74.6341148301741) bank124217 +124218 POINT(40.789514950540266 74.03702492627386) bank124218 +124219 POINT(40.56937687835691 74.97866828487285) bank124219 +124220 POINT(40.32111738556144 73.93763175566647) bank124220 +124221 POINT(41.09422514442669 73.4998173130696) bank124221 +124222 POINT(40.23313835616791 73.26634914594528) bank124222 +124223 POINT(40.56373251986382 74.55124411122887) bank124223 +124224 POINT(40.715786185034375 73.9995934423039) bank124224 +124225 POINT(41.15812007804084 74.13671550801463) bank124225 +124226 POINT(41.11614130990167 74.59507644667373) bank124226 +124227 POINT(40.46722119105349 73.0756667394426) bank124227 +124228 POINT(41.29043024585791 73.24881276474258) bank124228 +124229 POINT(40.596089818944606 73.4230833829731) bank124229 +124230 POINT(40.621774164402076 73.71121732177136) bank124230 +124231 POINT(40.46333307246114 73.4182478845305) bank124231 +124232 POINT(40.48976401330135 74.99179061904363) bank124232 +124233 POINT(40.33827211088759 74.74049760050244) bank124233 +124234 POINT(40.69741174930713 74.56306224741571) bank124234 +124235 POINT(41.147658148521025 74.47161557690835) bank124235 +124236 POINT(40.54359819055719 73.4573779748804) bank124236 +124237 POINT(41.363996294934694 73.31722944317106) bank124237 +124238 POINT(40.47503421685792 74.07646893201992) bank124238 +124239 POINT(39.857638431659915 73.43562771899619) bank124239 +124240 POINT(40.282413467942106 74.9310175767321) bank124240 +124241 POINT(40.88269271228528 74.92252761211627) bank124241 +124242 POINT(41.64739333254078 74.51741173353787) bank124242 +124243 POINT(40.79736914142326 73.56068064646317) bank124243 +124244 POINT(39.976575250315896 74.55872123150425) bank124244 +124245 POINT(40.30486872574824 73.38102369970905) bank124245 +124246 POINT(40.660963688411755 74.34228099731168) bank124246 +124247 POINT(40.2708061831353 73.11205080757026) bank124247 +124248 POINT(40.92105659558481 73.81606981951641) bank124248 +124249 POINT(40.41151948561078 74.38887370549863) bank124249 +124250 POINT(40.042997445118374 74.66664637617717) bank124250 +124251 POINT(40.048880069537965 74.79648484178682) bank124251 +124252 POINT(40.71061171621278 74.64530741432074) bank124252 +124253 POINT(41.705415777363726 74.20597635494937) bank124253 +124254 POINT(39.89454851910453 74.38569190761959) bank124254 +124255 POINT(41.27839264223653 73.99300122602315) bank124255 +124256 POINT(41.39945687497915 74.45225724267574) bank124256 +124257 POINT(40.18601744246777 74.46666695577646) bank124257 +124258 POINT(40.14509256182593 74.50186298064527) bank124258 +124259 POINT(39.89250658662339 73.43384736659904) bank124259 +124260 POINT(40.17839952992599 73.22645945508593) bank124260 +124261 POINT(39.9741877558404 74.51180599963033) bank124261 +124262 POINT(40.13061729953791 73.393353356357) bank124262 +124263 POINT(40.93175547569398 74.55991445333737) bank124263 +124264 POINT(40.57152592781196 74.93772448749672) bank124264 +124265 POINT(41.37132365050205 74.65070669759909) bank124265 +124266 POINT(40.41902867254279 73.08552784801421) bank124266 +124267 POINT(40.821437716523384 73.53272564750559) bank124267 +124268 POINT(41.657244995895 74.19967366446524) bank124268 +124269 POINT(41.38626743348495 73.47978081329539) bank124269 +124270 POINT(41.31411193939152 73.67435819714456) bank124270 +124271 POINT(41.505293368564146 73.80164975522102) bank124271 +124272 POINT(41.62442082661754 73.78717141412292) bank124272 +124273 POINT(40.8632116102805 74.19380031343366) bank124273 +124274 POINT(39.952597557089575 73.99528297678309) bank124274 +124275 POINT(41.66482532438857 73.85498761659828) bank124275 +124276 POINT(40.73670158383818 74.37465182275086) bank124276 +124277 POINT(39.81069278441713 74.49815471305747) bank124277 +124278 POINT(39.833393639271634 74.25254327422283) bank124278 +124279 POINT(40.74423428429085 73.3762485781011) bank124279 +124280 POINT(40.61641353539532 74.0048733390374) bank124280 +124281 POINT(40.65630348206432 73.95211551677936) bank124281 +124282 POINT(41.69945385844739 74.48186689402874) bank124282 +124283 POINT(41.100617595035075 73.80646542762605) bank124283 +124284 POINT(39.99898529434277 73.76411490396065) bank124284 +124285 POINT(41.48908616466455 73.84109208858027) bank124285 +124286 POINT(40.312243945693815 73.48115411521643) bank124286 +124287 POINT(41.134918644210565 73.19295908151832) bank124287 +124288 POINT(41.10230059474575 74.90441470432569) bank124288 +124289 POINT(40.12014275702024 73.25504727695109) bank124289 +124290 POINT(40.05841216629962 73.74324900334142) bank124290 +124291 POINT(41.675996333115634 73.26860469498824) bank124291 +124292 POINT(40.54232320762956 74.01662364268672) bank124292 +124293 POINT(40.19925420323923 74.06052269239274) bank124293 +124294 POINT(41.532181527473504 74.35845614025666) bank124294 +124295 POINT(39.77684538442469 73.42205178720236) bank124295 +124296 POINT(40.04846001608174 73.56200559593462) bank124296 +124297 POINT(40.47821638102451 74.72488448840004) bank124297 +124298 POINT(41.42360687861569 73.62716142386107) bank124298 +124299 POINT(41.20682935880928 74.08203722357814) bank124299 +124300 POINT(40.225432822110314 74.28285906065686) bank124300 +124301 POINT(41.58572162881461 73.02860704366394) bank124301 +124302 POINT(40.54555376910014 73.85962590752824) bank124302 +124303 POINT(41.107312466022876 73.36238395704748) bank124303 +124304 POINT(41.66507608310736 73.48704246840155) bank124304 +124305 POINT(41.46126881099577 74.54951384939258) bank124305 +124306 POINT(40.63678187527752 74.40653767825074) bank124306 +124307 POINT(39.8763782489812 73.42007738790888) bank124307 +124308 POINT(40.23288746526002 73.07710168961286) bank124308 +124309 POINT(40.01891419999777 73.31717105258875) bank124309 +124310 POINT(40.53774010484272 73.53501908876994) bank124310 +124311 POINT(40.56369673528542 73.57952567290793) bank124311 +124312 POINT(40.37757928736938 73.79383586288604) bank124312 +124313 POINT(41.51040987020181 73.99948880821863) bank124313 +124314 POINT(41.47805180336175 73.3323028228397) bank124314 +124315 POINT(39.71593094082629 74.97395228734155) bank124315 +124316 POINT(40.043483778730135 74.59731759655777) bank124316 +124317 POINT(41.40652004165393 74.08720296900371) bank124317 +124318 POINT(40.60696987084047 74.73811557940465) bank124318 +124319 POINT(40.070397886231895 73.52299932765848) bank124319 +124320 POINT(40.93029079170543 74.12486639950647) bank124320 +124321 POINT(40.20752776672744 73.91010718594279) bank124321 +124322 POINT(41.1458362319028 73.24787078301723) bank124322 +124323 POINT(41.13998221792637 74.54462891088363) bank124323 +124324 POINT(41.19320944240195 74.29941348274552) bank124324 +124325 POINT(40.85276976060684 74.61149251646712) bank124325 +124326 POINT(40.844851863420374 73.05259454464539) bank124326 +124327 POINT(40.650114244825424 74.36243667178809) bank124327 +124328 POINT(40.09522769133045 74.19383323622284) bank124328 +124329 POINT(40.94344000204657 74.11479306775206) bank124329 +124330 POINT(40.336473815232 74.46397389920067) bank124330 +124331 POINT(40.825030666466574 74.13372981836272) bank124331 +124332 POINT(40.45475607352099 73.26866600618281) bank124332 +124333 POINT(39.802265333081934 73.94184419718636) bank124333 +124334 POINT(40.20456705263017 74.89467913032975) bank124334 +124335 POINT(39.786011309784215 74.51366676085172) bank124335 +124336 POINT(40.72929206118554 73.43155119196386) bank124336 +124337 POINT(41.58338491507239 73.87447925617748) bank124337 +124338 POINT(40.89560195195477 73.9442729491736) bank124338 +124339 POINT(41.26844883027939 74.6026180433986) bank124339 +124340 POINT(40.383442105808065 74.08201286406722) bank124340 +124341 POINT(41.17785992572938 74.37846863900242) bank124341 +124342 POINT(39.95672145343885 73.27774897725227) bank124342 +124343 POINT(40.72500246538683 73.48797360332469) bank124343 +124344 POINT(40.686151556966145 74.6229354203963) bank124344 +124345 POINT(41.34152446997781 74.48881673361589) bank124345 +124346 POINT(41.282544121842115 73.09953715822074) bank124346 +124347 POINT(40.90859827591013 73.80417174560911) bank124347 +124348 POINT(40.401621251282634 73.16511246517979) bank124348 +124349 POINT(40.174439508208266 73.56745174768184) bank124349 +124350 POINT(39.86630198728479 73.3935039387108) bank124350 +124351 POINT(41.25656874734843 74.5198501558078) bank124351 +124352 POINT(40.41790007080778 74.76315981104388) bank124352 +124353 POINT(41.05879850042395 74.9614448922388) bank124353 +124354 POINT(41.248200638512536 73.07862116608285) bank124354 +124355 POINT(40.62732019893712 74.49137278250775) bank124355 +124356 POINT(40.8418221572836 73.89531422382424) bank124356 +124357 POINT(41.375871137869815 74.08425344290616) bank124357 +124358 POINT(40.06034180247848 73.43090599377535) bank124358 +124359 POINT(41.03853164575784 74.95511333722946) bank124359 +124360 POINT(41.56293674170196 73.8453837870564) bank124360 +124361 POINT(41.549027446469914 74.37662256626803) bank124361 +124362 POINT(40.133163805294835 73.54759309884737) bank124362 +124363 POINT(41.10847061275868 73.85966716729037) bank124363 +124364 POINT(41.704702438934845 73.75724460348351) bank124364 +124365 POINT(40.83127889846965 73.89829429681336) bank124365 +124366 POINT(39.852596436616984 74.13056388540895) bank124366 +124367 POINT(41.020991403462546 74.07514984623741) bank124367 +124368 POINT(39.72923493181829 74.58555904025943) bank124368 +124369 POINT(40.211802612476525 74.18574782029442) bank124369 +124370 POINT(40.82109500822324 73.06211485819291) bank124370 +124371 POINT(40.15265653951635 74.54656684831627) bank124371 +124372 POINT(41.678406500444126 73.64605998482146) bank124372 +124373 POINT(40.163413079307745 74.80972960248214) bank124373 +124374 POINT(41.55041053089709 74.88930029770854) bank124374 +124375 POINT(40.45526180522335 73.36191332036503) bank124375 +124376 POINT(40.34737065464246 74.46991472422695) bank124376 +124377 POINT(40.19428594889328 73.09294446992995) bank124377 +124378 POINT(40.109095187711915 74.05848383357659) bank124378 +124379 POINT(41.55602354447678 74.99878875833427) bank124379 +124380 POINT(40.64239484104156 74.96977575192726) bank124380 +124381 POINT(40.20077169066196 73.19705186213965) bank124381 +124382 POINT(41.25168538582505 73.08514700777886) bank124382 +124383 POINT(41.70694672926748 74.62144079118897) bank124383 +124384 POINT(40.86874510528816 74.88980735825983) bank124384 +124385 POINT(41.331419158868464 74.15844511557759) bank124385 +124386 POINT(41.10835795965973 74.91251167790851) bank124386 +124387 POINT(40.21141063301358 73.76444967531856) bank124387 +124388 POINT(41.17842340762043 73.34244725928005) bank124388 +124389 POINT(40.35465162598377 73.62032883042254) bank124389 +124390 POINT(40.691848702706274 73.60255909465464) bank124390 +124391 POINT(41.68834986920152 73.93446575693079) bank124391 +124392 POINT(40.48160463653796 74.27659243241578) bank124392 +124393 POINT(40.65167073712942 74.01619651362907) bank124393 +124394 POINT(40.454021806085564 74.06247103013436) bank124394 +124395 POINT(40.30368906011266 73.96058611043617) bank124395 +124396 POINT(41.136571731368676 73.38642263565012) bank124396 +124397 POINT(41.297650211453565 73.3107835936076) bank124397 +124398 POINT(39.75040180154406 74.49861028403653) bank124398 +124399 POINT(40.94272546338879 74.04923454268045) bank124399 +124400 POINT(40.659646616877254 73.32688108732917) bank124400 +124401 POINT(41.39281736844436 73.72366760050085) bank124401 +124402 POINT(41.47443683132635 74.37638603923406) bank124402 +124403 POINT(39.76240240724036 74.52861962209494) bank124403 +124404 POINT(40.00050397981884 74.29808718454495) bank124404 +124405 POINT(40.177642682618 74.2520530572932) bank124405 +124406 POINT(40.447743242177324 73.99922432524306) bank124406 +124407 POINT(39.73237822505961 74.94117820159879) bank124407 +124408 POINT(40.09955805955517 73.24059002739675) bank124408 +124409 POINT(39.92302900293513 73.76614569500254) bank124409 +124410 POINT(39.93402665212717 73.50223904796806) bank124410 +124411 POINT(41.307197099753715 73.5396729213016) bank124411 +124412 POINT(41.5683515612401 73.79993626035944) bank124412 +124413 POINT(40.09296453358719 73.24800559174543) bank124413 +124414 POINT(41.59489588175339 74.77586383176673) bank124414 +124415 POINT(39.71994683897367 73.15226101384427) bank124415 +124416 POINT(39.76186486478337 73.40451265394175) bank124416 +124417 POINT(39.99076918322356 73.73125733680327) bank124417 +124418 POINT(41.70321746174074 74.17220698673657) bank124418 +124419 POINT(41.10664053793794 73.46098046927308) bank124419 +124420 POINT(41.679386687744135 73.5896086242083) bank124420 +124421 POINT(40.706191456321974 74.1640377030482) bank124421 +124422 POINT(41.70022234731982 73.53509256792599) bank124422 +124423 POINT(41.65686466492416 73.25498551241586) bank124423 +124424 POINT(40.2977888659823 73.56669948790267) bank124424 +124425 POINT(39.90079371510419 73.70483356938018) bank124425 +124426 POINT(41.18899723312435 74.90540703416801) bank124426 +124427 POINT(41.33737284142415 73.94878296175138) bank124427 +124428 POINT(40.40754777992674 74.48053416839755) bank124428 +124429 POINT(41.57778876818816 74.89977884901529) bank124429 +124430 POINT(41.1633899495262 74.77464182508972) bank124430 +124431 POINT(40.50499627918122 73.46487713663542) bank124431 +124432 POINT(40.35976380697692 73.67214199949548) bank124432 +124433 POINT(41.257522863179815 74.04285145916755) bank124433 +124434 POINT(40.06612363304633 74.89867167652949) bank124434 +124435 POINT(40.193570089949915 73.15938546778891) bank124435 +124436 POINT(40.50211581320617 73.46376451092469) bank124436 +124437 POINT(40.28464976869467 74.16986556759473) bank124437 +124438 POINT(41.35446363070679 73.0387332189987) bank124438 +124439 POINT(41.22665906351742 73.01810768478595) bank124439 +124440 POINT(41.59114696834559 74.00484463507338) bank124440 +124441 POINT(40.893966988927225 74.95226444249842) bank124441 +124442 POINT(40.466230889999736 73.85612363941209) bank124442 +124443 POINT(40.45667881982689 74.11719528509805) bank124443 +124444 POINT(40.98806738333079 74.38055525951367) bank124444 +124445 POINT(40.527490498970046 73.69873892049732) bank124445 +124446 POINT(41.47049771747923 74.448858821879) bank124446 +124447 POINT(40.0361508254148 74.00644160614272) bank124447 +124448 POINT(41.26315539816152 74.47378240654498) bank124448 +124449 POINT(40.32410225496108 73.39358636597021) bank124449 +124450 POINT(39.92313379907548 74.26326944753632) bank124450 +124451 POINT(41.175159003432775 74.30581508780448) bank124451 +124452 POINT(39.99652433857131 74.75155873256398) bank124452 +124453 POINT(41.10142915953835 74.23273329573594) bank124453 +124454 POINT(40.88378350475183 74.4276603758299) bank124454 +124455 POINT(40.20425612922234 74.82804722503849) bank124455 +124456 POINT(41.266188512315765 73.70781114143053) bank124456 +124457 POINT(40.3608214604161 74.3428141339097) bank124457 +124458 POINT(41.45472452884825 74.99927153188298) bank124458 +124459 POINT(41.51287904231553 73.18734365205046) bank124459 +124460 POINT(40.359528518709155 73.09461697341202) bank124460 +124461 POINT(41.26104789821046 74.76301644079648) bank124461 +124462 POINT(41.2192585752922 74.60384338385263) bank124462 +124463 POINT(40.89723657152889 73.659300352079) bank124463 +124464 POINT(41.11323748272702 73.7647608700024) bank124464 +124465 POINT(40.22156224814854 74.65733406679443) bank124465 +124466 POINT(40.753020678007395 73.67244822516514) bank124466 +124467 POINT(41.6271147045353 73.55493597299586) bank124467 +124468 POINT(40.28978249708101 73.72947948754536) bank124468 +124469 POINT(40.62434590147692 74.49497133672374) bank124469 +124470 POINT(41.17558842987277 74.22289183093028) bank124470 +124471 POINT(40.14888066810498 73.24049777191308) bank124471 +124472 POINT(40.51516547854358 74.22311740698342) bank124472 +124473 POINT(40.000251445097895 74.08319787586927) bank124473 +124474 POINT(41.18796849855798 73.39465766993393) bank124474 +124475 POINT(40.41224047639752 74.08310069296718) bank124475 +124476 POINT(40.53383877554905 74.29574053382431) bank124476 +124477 POINT(41.402498877067565 73.72157972692479) bank124477 +124478 POINT(40.861007804075 74.6279573061763) bank124478 +124479 POINT(41.25760160165232 74.50233621609098) bank124479 +124480 POINT(40.6846821009455 74.07504522694137) bank124480 +124481 POINT(39.77283262504021 74.0655826752685) bank124481 +124482 POINT(41.488956267541624 74.62370453227997) bank124482 +124483 POINT(39.96197842443624 74.77591500517629) bank124483 +124484 POINT(41.340094615741926 73.50906631831103) bank124484 +124485 POINT(39.73434112016376 73.71836790300834) bank124485 +124486 POINT(41.02822136737283 73.2188973982385) bank124486 +124487 POINT(39.94217941966169 73.1800318032978) bank124487 +124488 POINT(40.80331564160924 73.30373639236164) bank124488 +124489 POINT(40.5986423413038 74.25752089209415) bank124489 +124490 POINT(39.895799473118174 74.46120243743086) bank124490 +124491 POINT(41.00410340604702 74.35010913908013) bank124491 +124492 POINT(40.1056755008634 74.14270684906035) bank124492 +124493 POINT(40.626316638534384 74.85762196088805) bank124493 +124494 POINT(41.53942238942301 73.83543006936043) bank124494 +124495 POINT(40.73639405695648 73.50691321937548) bank124495 +124496 POINT(39.900709652767056 73.86593629575174) bank124496 +124497 POINT(40.52444162921358 73.17025554310057) bank124497 +124498 POINT(40.116025137779594 73.48999655907836) bank124498 +124499 POINT(40.43158767377267 74.08731363252538) bank124499 +124500 POINT(41.045748268873375 73.65672846116033) bank124500 +124501 POINT(39.85029209157991 74.82067463000321) bank124501 +124502 POINT(41.43405916162731 74.52360304253293) bank124502 +124503 POINT(40.93242419722068 74.84794495493655) bank124503 +124504 POINT(40.14163343484274 73.26600026099277) bank124504 +124505 POINT(40.84715297575508 74.08423412090345) bank124505 +124506 POINT(40.055281073200916 74.71835501337256) bank124506 +124507 POINT(41.46805075484089 73.111422275288) bank124507 +124508 POINT(40.815894587876464 73.29219037975433) bank124508 +124509 POINT(41.29964725323145 73.22783339261275) bank124509 +124510 POINT(40.91849850106372 74.7863919084612) bank124510 +124511 POINT(41.23312377737915 74.44484078946145) bank124511 +124512 POINT(41.60978762455518 73.86534870557131) bank124512 +124513 POINT(40.9634721332457 73.45614850438483) bank124513 +124514 POINT(39.818620123104225 73.79891654256598) bank124514 +124515 POINT(41.56545964061873 74.71957452528794) bank124515 +124516 POINT(39.95623441008696 73.42973046595367) bank124516 +124517 POINT(40.483226059810335 74.97027687448184) bank124517 +124518 POINT(39.805561190168184 74.7373238216908) bank124518 +124519 POINT(40.82252328884705 73.68229725345307) bank124519 +124520 POINT(40.80691974600317 73.7410787029383) bank124520 +124521 POINT(41.483797709046044 74.04274187578007) bank124521 +124522 POINT(41.53167246954234 73.5266365067807) bank124522 +124523 POINT(41.1312216081737 73.77262624395179) bank124523 +124524 POINT(40.95354026095001 73.99045329451492) bank124524 +124525 POINT(39.99403154531014 74.17208016139965) bank124525 +124526 POINT(40.79529633941025 73.42712547491303) bank124526 +124527 POINT(41.58710667311844 73.78835209414834) bank124527 +124528 POINT(40.811741855491704 74.33577206719029) bank124528 +124529 POINT(41.4695081367182 74.40064740292662) bank124529 +124530 POINT(41.03369211545622 74.47437185063329) bank124530 +124531 POINT(40.62364120652677 74.30932574569495) bank124531 +124532 POINT(40.66983283240897 73.92363666782232) bank124532 +124533 POINT(39.7815346537316 73.08497740433305) bank124533 +124534 POINT(40.673170518603285 74.3128573725315) bank124534 +124535 POINT(40.06827883810829 73.29497938007134) bank124535 +124536 POINT(40.511987833163026 74.09322017407575) bank124536 +124537 POINT(41.27285775692206 74.58764842211555) bank124537 +124538 POINT(41.34293799601172 73.51663111538547) bank124538 +124539 POINT(40.75133370018472 74.46099042633422) bank124539 +124540 POINT(40.91727078061886 74.69567392788831) bank124540 +124541 POINT(41.393597353756334 73.05815132807844) bank124541 +124542 POINT(40.73271675163172 74.1001667803044) bank124542 +124543 POINT(40.72307510160267 74.29999874904574) bank124543 +124544 POINT(39.80207569812637 73.42116577154458) bank124544 +124545 POINT(39.871246439635605 74.30833480517789) bank124545 +124546 POINT(40.11362772096943 73.07801418169754) bank124546 +124547 POINT(40.737328832348744 74.0202897321922) bank124547 +124548 POINT(40.946350518137955 74.58440592786943) bank124548 +124549 POINT(40.74975043442359 73.04798912581546) bank124549 +124550 POINT(40.43896935625642 74.02652783708676) bank124550 +124551 POINT(40.06984696116358 73.27846514630801) bank124551 +124552 POINT(40.21997832735721 74.35539565466628) bank124552 +124553 POINT(41.198218170161056 73.71425844218717) bank124553 +124554 POINT(40.48831646494198 73.82736667657294) bank124554 +124555 POINT(41.388018386721086 74.505887438525) bank124555 +124556 POINT(40.21884792919929 74.14807179066274) bank124556 +124557 POINT(40.75560896051988 73.3659951681604) bank124557 +124558 POINT(40.93481457076677 73.65443530864309) bank124558 +124559 POINT(41.300885017910296 73.07944159999528) bank124559 +124560 POINT(41.33665723990424 74.25020432902153) bank124560 +124561 POINT(40.65071446698499 74.61608886741487) bank124561 +124562 POINT(40.68491989241976 74.26478792285548) bank124562 +124563 POINT(41.48919731164628 74.49492316807591) bank124563 +124564 POINT(41.38215806287261 73.34154653036018) bank124564 +124565 POINT(41.649498565657716 74.5903586259883) bank124565 +124566 POINT(40.43212702564091 74.56042124741117) bank124566 +124567 POINT(41.55546840734535 73.67330414651215) bank124567 +124568 POINT(41.23842828109164 73.34644155473005) bank124568 +124569 POINT(41.24370787347933 74.83985451990043) bank124569 +124570 POINT(40.45520630550495 74.76566909026657) bank124570 +124571 POINT(40.846224751925966 73.56358437655001) bank124571 +124572 POINT(41.27932902730153 74.3653875376064) bank124572 +124573 POINT(39.84637845387357 73.23960668603675) bank124573 +124574 POINT(40.38440649942577 73.73119397411853) bank124574 +124575 POINT(40.80412705784265 74.44183087585715) bank124575 +124576 POINT(39.95370226638981 74.56351008324282) bank124576 +124577 POINT(41.61082756851126 73.752012683958) bank124577 +124578 POINT(40.271894366939364 73.89422561887704) bank124578 +124579 POINT(40.66911134086164 73.54764565003352) bank124579 +124580 POINT(40.553273308226615 73.76804490022248) bank124580 +124581 POINT(40.5024247124795 74.29226381606881) bank124581 +124582 POINT(41.07493397552125 73.68814674450597) bank124582 +124583 POINT(40.1669858822815 73.13852747919931) bank124583 +124584 POINT(41.270481453055034 74.81152444851945) bank124584 +124585 POINT(41.51346040118563 74.525177482564) bank124585 +124586 POINT(40.07079463830091 74.21803608684549) bank124586 +124587 POINT(41.43706325975228 73.03832203904352) bank124587 +124588 POINT(40.08485413700582 74.67260673853765) bank124588 +124589 POINT(40.26051330850041 74.02193976372835) bank124589 +124590 POINT(41.2874571775061 73.62516234911693) bank124590 +124591 POINT(40.22696979826645 73.7973345399847) bank124591 +124592 POINT(39.92015590103457 74.76220957941266) bank124592 +124593 POINT(40.51992383206665 73.45701474382274) bank124593 +124594 POINT(41.18747005487497 73.77929135459733) bank124594 +124595 POINT(40.79985485117476 74.4697822122098) bank124595 +124596 POINT(39.792038703778026 74.05238510735474) bank124596 +124597 POINT(41.605289307055884 73.38105858913111) bank124597 +124598 POINT(40.570255753955806 73.2551359478338) bank124598 +124599 POINT(40.41726951738183 73.05872893410039) bank124599 +124600 POINT(40.2669609721819 74.03723759696499) bank124600 +124601 POINT(40.74113337316761 73.49341751051567) bank124601 +124602 POINT(41.055781847616714 73.39171478554877) bank124602 +124603 POINT(39.9815278440398 74.21808220365729) bank124603 +124604 POINT(40.14434627566668 73.73256402938918) bank124604 +124605 POINT(41.130333117875985 74.66983539133442) bank124605 +124606 POINT(41.56717014594417 74.78272956674162) bank124606 +124607 POINT(40.14277783111935 74.21160943711128) bank124607 +124608 POINT(41.30308557365473 73.23866819599365) bank124608 +124609 POINT(41.01731226133567 74.90498328688477) bank124609 +124610 POINT(40.64362409926993 73.49761680825213) bank124610 +124611 POINT(41.34298297847664 74.20526191236206) bank124611 +124612 POINT(41.53911460235967 73.94254785308416) bank124612 +124613 POINT(40.61681293065465 74.90503133591974) bank124613 +124614 POINT(41.09847579071908 74.79845539696959) bank124614 +124615 POINT(41.476891120289416 74.57353302888332) bank124615 +124616 POINT(41.07058611226623 74.83768354430529) bank124616 +124617 POINT(40.74748271607217 74.0319833072413) bank124617 +124618 POINT(41.44828597554205 74.69565021107287) bank124618 +124619 POINT(41.218126620104975 74.27276332379162) bank124619 +124620 POINT(41.382540020509815 74.22113264109575) bank124620 +124621 POINT(40.17420029865013 73.46540791042432) bank124621 +124622 POINT(39.91449430151959 73.18080378779742) bank124622 +124623 POINT(40.19048694297564 74.13894370751525) bank124623 +124624 POINT(39.9541986917746 74.68582431829662) bank124624 +124625 POINT(40.844029140924405 73.91760242009703) bank124625 +124626 POINT(39.746712467977886 73.95364949094215) bank124626 +124627 POINT(39.9820588358598 73.02873222627036) bank124627 +124628 POINT(40.084918304412966 74.81522046215834) bank124628 +124629 POINT(40.63315351061836 73.9429459884436) bank124629 +124630 POINT(40.46215890850546 74.13145103000794) bank124630 +124631 POINT(40.04979110903433 73.55627564124846) bank124631 +124632 POINT(39.761802614700514 74.23114893259353) bank124632 +124633 POINT(40.88700887481557 74.42051055607305) bank124633 +124634 POINT(41.21675500136503 73.18799178427231) bank124634 +124635 POINT(39.937744550344334 73.5310978430456) bank124635 +124636 POINT(40.288180311647274 73.85432888657677) bank124636 +124637 POINT(40.55422267308373 73.36504370243661) bank124637 +124638 POINT(40.54513368338366 74.12955012125424) bank124638 +124639 POINT(41.57837600076997 73.46003361791874) bank124639 +124640 POINT(40.06100793592744 74.08850857382474) bank124640 +124641 POINT(41.70096047441744 73.1945596047605) bank124641 +124642 POINT(40.28705648224892 74.84233193079176) bank124642 +124643 POINT(40.6153036906623 73.41918891662256) bank124643 +124644 POINT(39.958036294965 73.47989576049503) bank124644 +124645 POINT(39.79367072296659 73.03669683550702) bank124645 +124646 POINT(41.47975873643129 74.27447225335484) bank124646 +124647 POINT(39.842863480858355 73.80052189762525) bank124647 +124648 POINT(40.4812318124865 74.51498406353389) bank124648 +124649 POINT(40.91547373627867 74.8409421388748) bank124649 +124650 POINT(40.5798862685562 73.02957983878072) bank124650 +124651 POINT(41.20087069752252 74.98112760249968) bank124651 +124652 POINT(39.72248056432418 73.27898971872858) bank124652 +124653 POINT(40.64810452068039 73.95344407968209) bank124653 +124654 POINT(40.608527619849596 73.83663663304169) bank124654 +124655 POINT(40.77857323817546 74.30362628630256) bank124655 +124656 POINT(41.165356496837546 73.8855410323477) bank124656 +124657 POINT(39.86085916656362 74.97111921765479) bank124657 +124658 POINT(40.31914996555898 74.92113492800316) bank124658 +124659 POINT(40.464960919072006 74.86627203366952) bank124659 +124660 POINT(41.22969174160887 74.62434844300215) bank124660 +124661 POINT(41.07614991554199 73.40330774771098) bank124661 +124662 POINT(40.29087579135777 73.29367167373208) bank124662 +124663 POINT(40.495599478971286 74.29143174424358) bank124663 +124664 POINT(40.93064440939087 74.85313503121176) bank124664 +124665 POINT(40.33837705212056 74.34567916373415) bank124665 +124666 POINT(40.143568272204526 73.40965461905576) bank124666 +124667 POINT(39.98629415244003 74.4454336870629) bank124667 +124668 POINT(40.622257947228015 74.99792029252896) bank124668 +124669 POINT(40.427513891076046 74.94227465795711) bank124669 +124670 POINT(40.86785217486656 74.9538591594918) bank124670 +124671 POINT(40.556584600021985 73.42126179757666) bank124671 +124672 POINT(41.65396107249623 73.82617838798687) bank124672 +124673 POINT(40.625280274204215 73.2897188259505) bank124673 +124674 POINT(40.25274803235039 73.66942182406085) bank124674 +124675 POINT(40.14614729170231 73.98331993908229) bank124675 +124676 POINT(41.10757750851693 74.89991912612506) bank124676 +124677 POINT(41.36700331144781 73.7158728899063) bank124677 +124678 POINT(40.76967419768139 74.39248333299194) bank124678 +124679 POINT(40.13354153901832 73.02698139501857) bank124679 +124680 POINT(40.84925115539643 73.59975439247674) bank124680 +124681 POINT(41.16100199576097 73.01482260031288) bank124681 +124682 POINT(39.96974012126161 73.71575394926595) bank124682 +124683 POINT(39.98544756012927 73.09342707894282) bank124683 +124684 POINT(40.787173107610485 73.01694172419226) bank124684 +124685 POINT(41.152244418606735 73.41163387387358) bank124685 +124686 POINT(39.96511732686451 74.49835670473989) bank124686 +124687 POINT(41.648531505129064 73.57248261722941) bank124687 +124688 POINT(39.791139069386475 73.35977963987395) bank124688 +124689 POINT(40.15964821514926 73.12052796382645) bank124689 +124690 POINT(40.19559977603426 74.635352576806) bank124690 +124691 POINT(41.31763155306667 74.50180146314885) bank124691 +124692 POINT(40.75016320999041 73.94891150341876) bank124692 +124693 POINT(40.01815619759387 74.39510930522329) bank124693 +124694 POINT(39.75521617325419 73.99518650097697) bank124694 +124695 POINT(41.677330107446274 74.20027421160381) bank124695 +124696 POINT(40.078926101622386 74.70559506592917) bank124696 +124697 POINT(41.56966597250173 73.09238989051286) bank124697 +124698 POINT(39.84316228371899 74.46364191724835) bank124698 +124699 POINT(40.72942583309792 74.85176777017999) bank124699 +124700 POINT(40.80343884882422 73.31659364056112) bank124700 +124701 POINT(41.02173993454213 73.57185489810894) bank124701 +124702 POINT(40.691541708294785 73.93319418544115) bank124702 +124703 POINT(40.52233309849946 74.53252274089827) bank124703 +124704 POINT(41.360556811141706 73.30490334678974) bank124704 +124705 POINT(41.17715627450005 73.81064840271455) bank124705 +124706 POINT(41.0949553141848 74.50599660995624) bank124706 +124707 POINT(40.900088892025245 73.98232130561522) bank124707 +124708 POINT(39.779781602603464 74.49442822358175) bank124708 +124709 POINT(40.10318299954188 74.35381717145518) bank124709 +124710 POINT(40.8016765546838 73.5438262065259) bank124710 +124711 POINT(40.989510387633686 73.95984236087175) bank124711 +124712 POINT(40.45589015280227 74.43832197929162) bank124712 +124713 POINT(40.61588899494645 74.7280872997391) bank124713 +124714 POINT(41.237257059700944 74.74189717325427) bank124714 +124715 POINT(39.833219029705035 73.617317256425) bank124715 +124716 POINT(41.6457654357871 73.83742873833738) bank124716 +124717 POINT(39.89622887232558 73.53219420255932) bank124717 +124718 POINT(41.32171045690773 73.78591141803268) bank124718 +124719 POINT(41.49019505302984 73.64720306621922) bank124719 +124720 POINT(40.70860257329937 73.30494187535993) bank124720 +124721 POINT(40.296116698387905 73.16407543012144) bank124721 +124722 POINT(40.016746399656235 73.94675403548692) bank124722 +124723 POINT(41.53231015011638 73.926502597386) bank124723 +124724 POINT(40.54828704727073 74.73199065264596) bank124724 +124725 POINT(40.76431316594919 73.5994295122895) bank124725 +124726 POINT(40.959018855196376 73.83715799507826) bank124726 +124727 POINT(40.18171674857865 73.32480528485036) bank124727 +124728 POINT(40.920155910668804 73.46707650066094) bank124728 +124729 POINT(41.173025793432636 74.29150819457513) bank124729 +124730 POINT(40.266821254691244 74.88305636861013) bank124730 +124731 POINT(39.95783294080936 73.57806316191576) bank124731 +124732 POINT(39.74288690574939 74.99959225338849) bank124732 +124733 POINT(40.46523595097403 74.46386017928963) bank124733 +124734 POINT(40.84903758256622 74.28964026550672) bank124734 +124735 POINT(40.10605815483745 73.36922588540773) bank124735 +124736 POINT(41.663432531772166 74.97200905162165) bank124736 +124737 POINT(40.42436097565369 73.81601799711758) bank124737 +124738 POINT(40.745630693596155 74.43607810172008) bank124738 +124739 POINT(40.11089254797026 73.23075326839508) bank124739 +124740 POINT(40.08356276860418 74.96873322534566) bank124740 +124741 POINT(39.918965965859435 74.28826026240931) bank124741 +124742 POINT(39.76067661139935 74.75756915164916) bank124742 +124743 POINT(40.48425714245729 74.81983372911495) bank124743 +124744 POINT(41.02270067618323 74.41284358791188) bank124744 +124745 POINT(40.59863586094215 74.96526641622353) bank124745 +124746 POINT(40.141358732431875 73.88765179206739) bank124746 +124747 POINT(40.70983504275915 73.26448926278692) bank124747 +124748 POINT(39.782198358490035 74.42057657693523) bank124748 +124749 POINT(40.91049206261566 73.45362483223624) bank124749 +124750 POINT(41.60285656118894 74.98795842737326) bank124750 +124751 POINT(41.22375302332207 74.63410411313042) bank124751 +124752 POINT(40.814840019195 73.47911501674189) bank124752 +124753 POINT(40.647197409844196 73.77132960081111) bank124753 +124754 POINT(40.07112797216971 74.5868672796794) bank124754 +124755 POINT(41.35986396447485 74.04837893701597) bank124755 +124756 POINT(40.862442200226845 73.03902929498328) bank124756 +124757 POINT(40.03400901517806 74.40094367211002) bank124757 +124758 POINT(40.022421456113 74.10586031867682) bank124758 +124759 POINT(41.27050035262651 73.05906387638531) bank124759 +124760 POINT(40.068113779804584 73.58899241958929) bank124760 +124761 POINT(39.94876859993934 73.32241100688043) bank124761 +124762 POINT(40.940351713565626 74.62437929877204) bank124762 +124763 POINT(40.51211985215957 73.20839118776591) bank124763 +124764 POINT(40.15473134126759 74.16082156914325) bank124764 +124765 POINT(40.36457630662268 73.25863275898939) bank124765 +124766 POINT(40.57201278458963 74.67331711440096) bank124766 +124767 POINT(39.86167864757185 74.8849174801041) bank124767 +124768 POINT(40.464131681764826 73.64628674678818) bank124768 +124769 POINT(40.08806177891087 73.57561619765254) bank124769 +124770 POINT(41.50615923330721 73.72262810138344) bank124770 +124771 POINT(40.61579154678648 73.44000467742437) bank124771 +124772 POINT(40.55329198640387 73.13076204157458) bank124772 +124773 POINT(39.92030193008975 74.05716214329412) bank124773 +124774 POINT(41.00545015190752 73.39346856503657) bank124774 +124775 POINT(39.84535233909141 73.24168080572383) bank124775 +124776 POINT(40.662657599738395 74.55928254848928) bank124776 +124777 POINT(40.40567909172146 74.27249196569645) bank124777 +124778 POINT(40.067659883882996 73.96294566418534) bank124778 +124779 POINT(41.114433269489744 74.44274233656292) bank124779 +124780 POINT(41.38123035446487 74.0128210997682) bank124780 +124781 POINT(41.45250792906544 73.61094820723991) bank124781 +124782 POINT(39.71814994418196 74.58149336275896) bank124782 +124783 POINT(40.37088590855107 74.20975320704963) bank124783 +124784 POINT(40.11340246062191 73.02887198521756) bank124784 +124785 POINT(41.525799217340605 74.9503868680136) bank124785 +124786 POINT(41.35313555624283 74.92342005935056) bank124786 +124787 POINT(40.06536729639775 74.93869457263732) bank124787 +124788 POINT(40.06650884166244 74.63764051960804) bank124788 +124789 POINT(39.78100935170281 73.37785556560054) bank124789 +124790 POINT(40.13296061642876 73.08999230586943) bank124790 +124791 POINT(39.80290836503967 74.18132360771818) bank124791 +124792 POINT(40.52026173420479 73.31718299028377) bank124792 +124793 POINT(41.13495657565798 73.55585426020865) bank124793 +124794 POINT(40.42862367402774 73.50698433481585) bank124794 +124795 POINT(40.501546924592226 73.1430132642772) bank124795 +124796 POINT(40.44704953139368 74.45798856072068) bank124796 +124797 POINT(40.83784109852089 73.44402980801132) bank124797 +124798 POINT(39.87114207391017 74.22165926247764) bank124798 +124799 POINT(40.25428651862733 74.53462645326537) bank124799 +124800 POINT(40.08740099057077 73.1956796837361) bank124800 +124801 POINT(41.63609615583552 74.7075030581192) bank124801 +124802 POINT(40.08666048576777 73.17953402341551) bank124802 +124803 POINT(41.06409681766739 74.91066137924233) bank124803 +124804 POINT(40.79642871619289 74.64587285567517) bank124804 +124805 POINT(39.86475121092409 73.19749354691218) bank124805 +124806 POINT(40.69556358937991 73.51732919603079) bank124806 +124807 POINT(40.69900161403855 74.30615719006595) bank124807 +124808 POINT(40.21004286594044 73.73195082704262) bank124808 +124809 POINT(40.911758287695655 73.59708221422311) bank124809 +124810 POINT(41.65611751913545 74.16492229370554) bank124810 +124811 POINT(40.63517250651534 74.15818555805954) bank124811 +124812 POINT(40.28285928164112 74.74615866694768) bank124812 +124813 POINT(41.44890840310831 73.3158496198758) bank124813 +124814 POINT(41.6083772390187 73.660747913616) bank124814 +124815 POINT(40.642971265866656 74.61317072372735) bank124815 +124816 POINT(41.19160639908602 74.29093432202012) bank124816 +124817 POINT(40.884993461393826 74.77156230554836) bank124817 +124818 POINT(40.79549804295621 73.97396079572005) bank124818 +124819 POINT(39.95541392891355 74.05131093551822) bank124819 +124820 POINT(41.051713742298645 73.19452613651674) bank124820 +124821 POINT(39.86528882269771 73.8219493370657) bank124821 +124822 POINT(40.0088232676153 74.33635646646249) bank124822 +124823 POINT(41.18534146206409 73.81582799280383) bank124823 +124824 POINT(40.94810802457629 73.10210812164948) bank124824 +124825 POINT(40.89141627767483 74.42052938369356) bank124825 +124826 POINT(39.843871848836585 74.72569349284295) bank124826 +124827 POINT(41.205379764727596 74.58605625920902) bank124827 +124828 POINT(41.07251401696715 73.32293704081319) bank124828 +124829 POINT(41.689831677316114 74.28435886350765) bank124829 +124830 POINT(41.309943509455955 74.39422844076662) bank124830 +124831 POINT(41.45245471149614 74.55333498631073) bank124831 +124832 POINT(40.460890838171366 73.94942885553628) bank124832 +124833 POINT(41.131317679026324 73.58721794306624) bank124833 +124834 POINT(40.88581263087848 73.96288424793809) bank124834 +124835 POINT(41.25091752087773 73.09380010132531) bank124835 +124836 POINT(40.658984898002316 73.49009532682732) bank124836 +124837 POINT(39.982896889192325 73.7236706492006) bank124837 +124838 POINT(40.58178778152639 74.2165461784838) bank124838 +124839 POINT(40.39806406322627 74.73203043219911) bank124839 +124840 POINT(41.14454390113122 73.07032561899555) bank124840 +124841 POINT(40.53009758082314 73.4755409077165) bank124841 +124842 POINT(41.04385063044863 74.31098943965493) bank124842 +124843 POINT(41.018243182799424 73.83707573452565) bank124843 +124844 POINT(39.82135305944406 73.29354365116731) bank124844 +124845 POINT(40.08653105496509 74.34966624740014) bank124845 +124846 POINT(40.2608542629358 74.53424928270158) bank124846 +124847 POINT(41.34130618102603 74.28574956770127) bank124847 +124848 POINT(41.06696681108503 74.16018464353112) bank124848 +124849 POINT(40.63866231482945 73.58931045547575) bank124849 +124850 POINT(41.17808264888951 73.44612472124724) bank124850 +124851 POINT(41.02229710003564 74.76170876283187) bank124851 +124852 POINT(39.891989819216754 74.0400217652386) bank124852 +124853 POINT(41.12449441608767 73.03949346926221) bank124853 +124854 POINT(40.44587111721269 73.02645660959001) bank124854 +124855 POINT(40.156442257184636 73.190929131493) bank124855 +124856 POINT(41.08776178878603 73.267427996667) bank124856 +124857 POINT(40.71605885146715 74.02438951902148) bank124857 +124858 POINT(40.808871804628836 74.26388454661715) bank124858 +124859 POINT(40.55870995410808 74.8412381508882) bank124859 +124860 POINT(39.733138363542785 74.0453943110713) bank124860 +124861 POINT(40.30588106149666 74.87423994049877) bank124861 +124862 POINT(40.71922286177252 74.93008752531821) bank124862 +124863 POINT(40.30237289728202 73.7389758757304) bank124863 +124864 POINT(41.620812389425815 73.29571940626619) bank124864 +124865 POINT(39.961382699150406 74.35260992964193) bank124865 +124866 POINT(40.68247131483351 74.73694256184913) bank124866 +124867 POINT(40.58911505820402 74.446333587836) bank124867 +124868 POINT(41.59187551684561 73.74840725513093) bank124868 +124869 POINT(41.598226073441566 73.9417063675517) bank124869 +124870 POINT(40.553083681422876 74.35324668431383) bank124870 +124871 POINT(40.20278448005146 74.50286904026441) bank124871 +124872 POINT(40.25547350343425 73.75220537862182) bank124872 +124873 POINT(41.096954517443116 74.03026450042388) bank124873 +124874 POINT(40.33417694978208 73.71892909113166) bank124874 +124875 POINT(40.20359574689516 73.52940669962045) bank124875 +124876 POINT(40.27700011033702 74.26208386800806) bank124876 +124877 POINT(41.370966531349744 74.40965572694101) bank124877 +124878 POINT(41.60620131252554 74.31730123027758) bank124878 +124879 POINT(41.42670211197408 73.83461698297845) bank124879 +124880 POINT(39.73420441133509 74.72330682250602) bank124880 +124881 POINT(41.31252205448971 73.29345778687446) bank124881 +124882 POINT(41.211707494731414 73.34476085789981) bank124882 +124883 POINT(40.067506447070244 74.49592291916528) bank124883 +124884 POINT(40.77217400381638 74.54851194432598) bank124884 +124885 POINT(40.67567153118425 73.58298046146048) bank124885 +124886 POINT(41.34115280027478 73.04504922604156) bank124886 +124887 POINT(41.092473093646824 73.03095014811743) bank124887 +124888 POINT(40.62580284771164 74.49586725375669) bank124888 +124889 POINT(41.401731319981415 73.07894096295698) bank124889 +124890 POINT(39.996409504584705 73.09033758169359) bank124890 +124891 POINT(41.53116941159391 73.92830163325901) bank124891 +124892 POINT(40.224428072682834 74.78423331676716) bank124892 +124893 POINT(41.530319077487455 73.6164994215451) bank124893 +124894 POINT(40.831268464330684 73.03563026934675) bank124894 +124895 POINT(40.86897903769482 74.84537580978231) bank124895 +124896 POINT(41.02354716769906 74.51038203061) bank124896 +124897 POINT(40.0389400369254 74.42262467693487) bank124897 +124898 POINT(40.250996440826235 74.78768454808743) bank124898 +124899 POINT(41.21723156538749 73.38142430356676) bank124899 +124900 POINT(41.647625455210985 74.7868541124378) bank124900 +124901 POINT(40.96309163606749 73.57482488813837) bank124901 +124902 POINT(41.48090153141169 74.08786314355353) bank124902 +124903 POINT(40.210036701827164 74.13071934330405) bank124903 +124904 POINT(40.888914585498 73.52199352525997) bank124904 +124905 POINT(40.521464617602525 73.99506712560539) bank124905 +124906 POINT(39.85638728823386 74.05681418489267) bank124906 +124907 POINT(40.47547623744607 74.38168309155891) bank124907 +124908 POINT(40.23152955020935 73.7214062567644) bank124908 +124909 POINT(40.77413062354453 74.09698967992027) bank124909 +124910 POINT(40.28306958850161 73.97817291362342) bank124910 +124911 POINT(40.2395323670861 73.96748506935349) bank124911 +124912 POINT(40.2530568930478 74.51218340854106) bank124912 +124913 POINT(40.89019573740987 74.49278223827827) bank124913 +124914 POINT(40.457631467846426 74.81436179777437) bank124914 +124915 POINT(40.41232770568273 74.7569148700956) bank124915 +124916 POINT(40.23017219657714 74.8216263649848) bank124916 +124917 POINT(40.7957645970924 74.6233026747559) bank124917 +124918 POINT(40.46633396625292 73.27503010835687) bank124918 +124919 POINT(41.62153494427539 74.10290220255617) bank124919 +124920 POINT(39.88056204905479 73.44112336585458) bank124920 +124921 POINT(40.25251880203954 73.26905382697888) bank124921 +124922 POINT(41.25291725574537 73.84350894831881) bank124922 +124923 POINT(40.98512194306784 74.46040920539313) bank124923 +124924 POINT(40.32703544246591 75.00183671511763) bank124924 +124925 POINT(40.376349598151236 74.8459046766738) bank124925 +124926 POINT(40.19353355252246 74.78008521997263) bank124926 +124927 POINT(41.094016929241604 74.8962730473171) bank124927 +124928 POINT(40.34633381078823 73.1353817097246) bank124928 +124929 POINT(40.04048197952291 73.37844180987976) bank124929 +124930 POINT(41.32916075985758 74.5916809541891) bank124930 +124931 POINT(41.28403961735903 74.44642505404985) bank124931 +124932 POINT(40.55082504631587 74.94992609690811) bank124932 +124933 POINT(40.73304161613795 73.18736401054511) bank124933 +124934 POINT(41.42280124876122 73.61540347421194) bank124934 +124935 POINT(40.42568389538327 73.64203450878324) bank124935 +124936 POINT(41.254311990472736 73.48241210442065) bank124936 +124937 POINT(41.3048599654653 73.0432687177285) bank124937 +124938 POINT(40.87515598295673 74.66342348004945) bank124938 +124939 POINT(40.86266246409452 74.59533721474499) bank124939 +124940 POINT(40.84728340676871 74.31019283503623) bank124940 +124941 POINT(39.83022516909597 73.07660354231355) bank124941 +124942 POINT(40.702686941223774 74.17520222459412) bank124942 +124943 POINT(40.83560519643263 74.41958501193496) bank124943 +124944 POINT(39.86869874020058 74.94211204258507) bank124944 +124945 POINT(40.741910203023956 74.5614806967179) bank124945 +124946 POINT(41.167100056602045 73.18242055154722) bank124946 +124947 POINT(41.54016105533287 73.60557789011767) bank124947 +124948 POINT(40.901406778473785 74.69378224024985) bank124948 +124949 POINT(40.77107280278389 74.5548242782911) bank124949 +124950 POINT(40.712765244636216 73.50564925043136) bank124950 +124951 POINT(39.792896749618386 73.50145909759158) bank124951 +124952 POINT(39.869506651061556 73.3533479276954) bank124952 +124953 POINT(39.99294695653703 73.45290176239634) bank124953 +124954 POINT(40.44559536528703 74.34304708879499) bank124954 +124955 POINT(40.430045412302206 73.27987368709672) bank124955 +124956 POINT(40.72212452284026 74.2604388513345) bank124956 +124957 POINT(40.11511464355986 74.82506228978143) bank124957 +124958 POINT(40.82325949529717 74.60814669542069) bank124958 +124959 POINT(40.38740036082067 74.19665603439078) bank124959 +124960 POINT(40.82927530930337 74.74825372095084) bank124960 +124961 POINT(41.630718595379136 73.18361523276893) bank124961 +124962 POINT(40.09641704790038 74.1027575664694) bank124962 +124963 POINT(39.754869019861225 73.95461959107828) bank124963 +124964 POINT(41.20900594591515 73.44612391045416) bank124964 +124965 POINT(39.79450358791154 73.34609439720032) bank124965 +124966 POINT(40.07115923111114 73.52552400803596) bank124966 +124967 POINT(40.48322915639424 73.74562694789438) bank124967 +124968 POINT(40.83050005209238 73.99839928466702) bank124968 +124969 POINT(40.94255699990208 74.92431524766445) bank124969 +124970 POINT(40.864511020721054 73.87154171462444) bank124970 +124971 POINT(40.09869881253517 74.70941125001393) bank124971 +124972 POINT(41.65801523190499 74.6341886582348) bank124972 +124973 POINT(40.39663989970028 74.2694060022733) bank124973 +124974 POINT(40.84332558172777 73.57986398427158) bank124974 +124975 POINT(39.807091983669785 74.81216493095471) bank124975 +124976 POINT(40.91430551904972 73.04055936964096) bank124976 +124977 POINT(40.8455432946739 74.02014902226719) bank124977 +124978 POINT(41.45263193384316 74.0012769196542) bank124978 +124979 POINT(41.60587978818407 73.45826214130285) bank124979 +124980 POINT(40.276906407765544 74.69603564789158) bank124980 +124981 POINT(40.73705947410019 73.63514472254676) bank124981 +124982 POINT(40.48547981843353 73.78963363344153) bank124982 +124983 POINT(40.834018407528035 74.24785165204248) bank124983 +124984 POINT(40.349229930971646 73.99252101176708) bank124984 +124985 POINT(40.730127128470585 74.31434729074277) bank124985 +124986 POINT(39.978119508902935 73.4364650434855) bank124986 +124987 POINT(40.51497878533346 74.55325368378958) bank124987 +124988 POINT(41.26209610912292 74.87509963683065) bank124988 +124989 POINT(40.02768627396592 73.98049526531253) bank124989 +124990 POINT(40.11295860977801 74.04055772394945) bank124990 +124991 POINT(41.0591617268447 73.69048645888168) bank124991 +124992 POINT(40.354945404679974 74.53172190989824) bank124992 +124993 POINT(41.697899373250465 73.8388060335721) bank124993 +124994 POINT(41.23532108754174 73.82790787005885) bank124994 +124995 POINT(40.35744674030265 74.01016440878698) bank124995 +124996 POINT(41.331723100468324 74.27763217264098) bank124996 +124997 POINT(40.295206002051735 73.22854952068275) bank124997 +124998 POINT(40.77583871516976 74.13262460792947) bank124998 +124999 POINT(41.55848457292225 74.73740167928584) bank124999 +125000 POINT(40.83640683053994 74.1190968160547) bank125000 +125001 POINT(41.01647320994736 73.57525259665395) bank125001 +125002 POINT(40.050443027233996 74.49823107607858) bank125002 +125003 POINT(40.13154374847389 74.69828853346297) bank125003 +125004 POINT(40.34476883095699 74.76968366754824) bank125004 +125005 POINT(40.39227249213313 74.55053611987903) bank125005 +125006 POINT(40.67546059140784 73.86525832197854) bank125006 +125007 POINT(39.7733390304752 73.03373526374457) bank125007 +125008 POINT(41.397655163019685 73.16636054348369) bank125008 +125009 POINT(41.68830208740272 74.3100665589342) bank125009 +125010 POINT(39.98201665578495 73.6554480648461) bank125010 +125011 POINT(41.537961679663475 73.79836977474059) bank125011 +125012 POINT(39.76118405681009 74.4704127217313) bank125012 +125013 POINT(41.06919020200413 73.94862342318972) bank125013 +125014 POINT(41.42836944918138 74.04594704753563) bank125014 +125015 POINT(41.12541826303051 73.37848882363232) bank125015 +125016 POINT(40.9840300378359 74.25481089532809) bank125016 +125017 POINT(41.45838731784251 73.14816649628281) bank125017 +125018 POINT(41.480365097373756 73.42148361040815) bank125018 +125019 POINT(40.44625242088917 74.39974048419758) bank125019 +125020 POINT(40.01937366194443 74.42143973572307) bank125020 +125021 POINT(41.47880444654436 74.60053899102121) bank125021 +125022 POINT(40.54728573663316 73.5229020050077) bank125022 +125023 POINT(41.31414669645457 74.08501095692732) bank125023 +125024 POINT(40.224066348084364 74.02899650516122) bank125024 +125025 POINT(41.17726371984746 74.86903152367208) bank125025 +125026 POINT(40.05215887461674 73.88030449007786) bank125026 +125027 POINT(40.8343592463415 73.02539014977526) bank125027 +125028 POINT(40.250095580729486 73.42951186579404) bank125028 +125029 POINT(40.44718331723871 73.22020886258889) bank125029 +125030 POINT(41.31237533666538 74.49057310585235) bank125030 +125031 POINT(40.37355925146006 74.73120106915945) bank125031 +125032 POINT(39.82549962271498 74.89345801283162) bank125032 +125033 POINT(40.651378201386585 73.03268849105807) bank125033 +125034 POINT(41.261819120273024 74.9842515743176) bank125034 +125035 POINT(41.48891011275521 74.56360428473849) bank125035 +125036 POINT(40.40897243368579 73.76163609251812) bank125036 +125037 POINT(40.51021596732398 74.6267275147653) bank125037 +125038 POINT(41.14636553393312 73.1957261978805) bank125038 +125039 POINT(40.504162974562824 74.9828114557678) bank125039 +125040 POINT(41.34700261898296 74.22710092085464) bank125040 +125041 POINT(39.88986817723855 73.3432618562567) bank125041 +125042 POINT(40.4580935313528 73.14796661327384) bank125042 +125043 POINT(40.76606248795721 74.70248909788182) bank125043 +125044 POINT(40.5755289169404 73.32170170971166) bank125044 +125045 POINT(40.836923008764856 74.28427822825202) bank125045 +125046 POINT(41.24163336114867 74.75499576543534) bank125046 +125047 POINT(39.80932482257323 73.07225330429107) bank125047 +125048 POINT(40.654114675186335 74.94704459626618) bank125048 +125049 POINT(40.71370192547032 73.46839530143276) bank125049 +125050 POINT(40.801981219289 73.21885451700992) bank125050 +125051 POINT(41.08344355206962 73.62830636698318) bank125051 +125052 POINT(41.400041408501956 74.34386044778515) bank125052 +125053 POINT(40.96977776351098 73.03390149705628) bank125053 +125054 POINT(41.623667026044224 73.37383003769057) bank125054 +125055 POINT(40.433141894983564 74.22224787555396) bank125055 +125056 POINT(40.75553870156856 74.30774331924141) bank125056 +125057 POINT(40.772669348193816 74.90250084462153) bank125057 +125058 POINT(40.827903500167416 73.45311055138838) bank125058 +125059 POINT(39.942795980525275 73.48940244786881) bank125059 +125060 POINT(39.90985361100144 73.73400351689581) bank125060 +125061 POINT(40.352053373639556 74.0201098232098) bank125061 +125062 POINT(40.824815689933175 73.0358676801334) bank125062 +125063 POINT(40.2593632587723 74.00280848468428) bank125063 +125064 POINT(39.8777181692944 74.55193634501629) bank125064 +125065 POINT(40.262030424956826 73.57759604581832) bank125065 +125066 POINT(40.83150197505333 73.90725765238398) bank125066 +125067 POINT(40.84879409185818 73.32060441286718) bank125067 +125068 POINT(40.68076174869428 73.01803002761687) bank125068 +125069 POINT(39.72509546187111 73.26479022519382) bank125069 +125070 POINT(40.44758129456196 74.71997600248166) bank125070 +125071 POINT(41.024503252397636 74.3919618718773) bank125071 +125072 POINT(41.6378030385254 73.8345093070568) bank125072 +125073 POINT(41.494422329844895 74.22702325111149) bank125073 +125074 POINT(41.393255282492014 74.15346404267879) bank125074 +125075 POINT(40.41049823979789 73.5171034384504) bank125075 +125076 POINT(40.13420696158962 73.40278503194975) bank125076 +125077 POINT(40.59855759245319 74.42136973936631) bank125077 +125078 POINT(39.911623847793294 73.83673801668161) bank125078 +125079 POINT(41.66163154431022 73.81953900960269) bank125079 +125080 POINT(40.79329328498642 73.94622392461704) bank125080 +125081 POINT(41.6592871267658 73.78984402661116) bank125081 +125082 POINT(40.81936020926284 73.22042466144626) bank125082 +125083 POINT(40.727998209434524 73.8687901226156) bank125083 +125084 POINT(40.889541939130936 74.91913508668648) bank125084 +125085 POINT(40.262316005313025 73.89618517086703) bank125085 +125086 POINT(41.51942021091247 74.13522353932991) bank125086 +125087 POINT(40.036478790665115 74.20398029593552) bank125087 +125088 POINT(39.79158114298091 74.77769725473375) bank125088 +125089 POINT(40.07088788742967 73.16519793072403) bank125089 +125090 POINT(39.741805573365006 74.42821384737947) bank125090 +125091 POINT(39.80669786342157 73.37596176305216) bank125091 +125092 POINT(40.38923144580785 73.28821859634924) bank125092 +125093 POINT(39.88643114569063 73.17518615959322) bank125093 +125094 POINT(40.373104711614815 73.57942730031357) bank125094 +125095 POINT(40.58795035294758 74.62260392076037) bank125095 +125096 POINT(39.849598163960636 74.09361760681259) bank125096 +125097 POINT(40.48394597338195 74.90922359184607) bank125097 +125098 POINT(40.58564095915503 73.49388823230812) bank125098 +125099 POINT(39.835775529051546 73.74119759692653) bank125099 +125100 POINT(40.91917385356022 73.34268885899164) bank125100 +125101 POINT(40.00555173513743 73.21155290074469) bank125101 +125102 POINT(40.3355452367392 73.71324574798604) bank125102 +125103 POINT(40.99625246255469 74.91367418339075) bank125103 +125104 POINT(40.221433925493955 74.95229247396925) bank125104 +125105 POINT(41.495838585972024 73.23690377744786) bank125105 +125106 POINT(40.272833734957956 74.42161159538404) bank125106 +125107 POINT(41.15970533540506 74.96104099194253) bank125107 +125108 POINT(41.20414904875353 73.26618590511215) bank125108 +125109 POINT(40.40000275977614 73.55121386912153) bank125109 +125110 POINT(39.74444837274285 73.52325775827282) bank125110 +125111 POINT(40.8830021332556 73.73397244634232) bank125111 +125112 POINT(39.86194755699625 73.7161197964431) bank125112 +125113 POINT(40.482434116086075 73.94569750167939) bank125113 +125114 POINT(40.23668056238952 73.60625685902022) bank125114 +125115 POINT(40.64809309931621 74.51025159859334) bank125115 +125116 POINT(40.97150201531371 73.02726261842433) bank125116 +125117 POINT(40.24424695702318 74.89454840479543) bank125117 +125118 POINT(41.21604902788932 74.88418084073325) bank125118 +125119 POINT(40.44689390000214 73.98777717281061) bank125119 +125120 POINT(40.085507199972746 74.96020338309786) bank125120 +125121 POINT(41.346657897117375 74.28189975667127) bank125121 +125122 POINT(40.789694738875276 73.33558923713997) bank125122 +125123 POINT(40.93706862668996 74.0595794087787) bank125123 +125124 POINT(41.60468402338955 73.95185858119291) bank125124 +125125 POINT(40.325296772707105 73.0543155932048) bank125125 +125126 POINT(40.896470800902414 74.27915702898197) bank125126 +125127 POINT(40.60421318704659 74.15541229567452) bank125127 +125128 POINT(41.26568896982165 74.05827618316029) bank125128 +125129 POINT(39.93504566537638 73.99721801034975) bank125129 +125130 POINT(39.81058994676648 74.54683303122985) bank125130 +125131 POINT(39.863767210992116 73.87596623527043) bank125131 +125132 POINT(40.21991028578351 73.78417333157354) bank125132 +125133 POINT(40.78615011637544 73.36422632235873) bank125133 +125134 POINT(41.35625443007288 74.30652583282786) bank125134 +125135 POINT(39.718360105634716 73.1270052427151) bank125135 +125136 POINT(39.89666958248165 74.73819553564773) bank125136 +125137 POINT(41.50947768719981 73.93230135968972) bank125137 +125138 POINT(40.45609619391252 73.11836006848908) bank125138 +125139 POINT(41.1755685651181 73.21315436229557) bank125139 +125140 POINT(39.97174653248514 73.95489527081237) bank125140 +125141 POINT(39.76377907873146 74.55919892645704) bank125141 +125142 POINT(40.64866455883077 73.88856360780984) bank125142 +125143 POINT(41.22438466149648 74.1081423005618) bank125143 +125144 POINT(41.09577889268409 73.53244955817007) bank125144 +125145 POINT(41.49611515258115 74.70818850657724) bank125145 +125146 POINT(40.74533676486805 73.92838796454873) bank125146 +125147 POINT(39.93422284221556 73.09390794610289) bank125147 +125148 POINT(40.3236111329017 74.82502059676212) bank125148 +125149 POINT(40.087261869970526 73.88034649902309) bank125149 +125150 POINT(39.92881825012841 74.59711564780456) bank125150 +125151 POINT(41.6406360442473 73.67539335130526) bank125151 +125152 POINT(40.06080187901105 73.45664498438097) bank125152 +125153 POINT(39.73373151776924 73.12518781680714) bank125153 +125154 POINT(41.63403319583053 74.00884377340918) bank125154 +125155 POINT(40.32177335356084 74.5509188109561) bank125155 +125156 POINT(41.2252896610094 73.25730962229429) bank125156 +125157 POINT(40.05249555328732 74.94052233569583) bank125157 +125158 POINT(41.640190272050134 73.03288134865952) bank125158 +125159 POINT(40.66948173998529 75.00177673412905) bank125159 +125160 POINT(41.02720230694589 73.68733114842318) bank125160 +125161 POINT(40.676629620421124 73.9080745727197) bank125161 +125162 POINT(40.002558768028756 74.89215240583964) bank125162 +125163 POINT(40.6849714287083 73.67355768105655) bank125163 +125164 POINT(40.25910197306584 74.12460397828794) bank125164 +125165 POINT(40.67994252316406 73.37839735452329) bank125165 +125166 POINT(40.05328106628583 74.9915462973089) bank125166 +125167 POINT(41.19539156875512 73.19988754085554) bank125167 +125168 POINT(41.17675794783938 73.14123898065147) bank125168 +125169 POINT(40.275368683909846 73.6553468620024) bank125169 +125170 POINT(41.34660527847857 73.79491798036027) bank125170 +125171 POINT(40.11402199793077 73.81109264411731) bank125171 +125172 POINT(41.107780780288735 74.17933932581309) bank125172 +125173 POINT(39.81203370889115 74.14377810309844) bank125173 +125174 POINT(39.958627851531546 74.48758042343067) bank125174 +125175 POINT(40.311020765043416 73.5766194813366) bank125175 +125176 POINT(41.46594993289872 73.03318980590169) bank125176 +125177 POINT(41.067733603748984 74.66551377099756) bank125177 +125178 POINT(40.87238515121126 73.77917019900893) bank125178 +125179 POINT(40.75208397295478 73.89434365382411) bank125179 +125180 POINT(40.85980577946802 73.15787435457608) bank125180 +125181 POINT(40.531874402169436 74.27173462401797) bank125181 +125182 POINT(41.7005384097259 73.65477595491151) bank125182 +125183 POINT(39.771287559925945 74.51173705679366) bank125183 +125184 POINT(41.37577111909489 73.19751133222206) bank125184 +125185 POINT(41.54285254252919 74.39013168835321) bank125185 +125186 POINT(40.09428659851822 73.30926353941676) bank125186 +125187 POINT(40.81162540042516 74.64466571021276) bank125187 +125188 POINT(41.14984467356145 74.26164978598123) bank125188 +125189 POINT(41.51046910941956 74.89308459483384) bank125189 +125190 POINT(39.87549478971356 74.84551518434698) bank125190 +125191 POINT(40.54237243771914 73.44192414027248) bank125191 +125192 POINT(40.01557017029047 73.98315018536246) bank125192 +125193 POINT(40.16643068557037 73.32230353192509) bank125193 +125194 POINT(40.20235515943625 74.75907335010258) bank125194 +125195 POINT(40.726540555725364 73.03193512903593) bank125195 +125196 POINT(40.84466380715595 73.43180232543287) bank125196 +125197 POINT(40.32904573317562 74.67680760768985) bank125197 +125198 POINT(40.944945298871716 73.91270847429178) bank125198 +125199 POINT(40.454140031840026 73.47921963096037) bank125199 +125200 POINT(40.951435697787694 73.4553545132193) bank125200 +125201 POINT(41.17735373424691 74.18059719969831) bank125201 +125202 POINT(41.30845157686733 74.66472877918237) bank125202 +125203 POINT(39.78354729890074 74.19146305501567) bank125203 +125204 POINT(41.37859053022465 73.64683441730814) bank125204 +125205 POINT(40.825020539183086 73.27117919361538) bank125205 +125206 POINT(40.63746332311921 73.94848034514813) bank125206 +125207 POINT(41.59416719333858 74.39841850584354) bank125207 +125208 POINT(40.11489700098348 74.63247187511799) bank125208 +125209 POINT(41.406084133798366 74.66737036244477) bank125209 +125210 POINT(40.2854380178708 73.93541207523286) bank125210 +125211 POINT(40.93150202131981 73.47310481688541) bank125211 +125212 POINT(40.25735826253307 73.15912279125128) bank125212 +125213 POINT(41.45938375469819 73.3580693873362) bank125213 +125214 POINT(40.854482183744835 74.96887634604903) bank125214 +125215 POINT(41.2763479561245 74.68412554407048) bank125215 +125216 POINT(40.039432238429136 73.62138953191432) bank125216 +125217 POINT(40.30280858045566 73.61972307840962) bank125217 +125218 POINT(40.04503550031874 74.42925677415478) bank125218 +125219 POINT(40.28965323279023 73.09146596574953) bank125219 +125220 POINT(40.29695094337422 74.03145546939976) bank125220 +125221 POINT(41.41282556211848 73.97873264374714) bank125221 +125222 POINT(40.30401275108966 74.5311501018139) bank125222 +125223 POINT(40.37773781277509 73.14574823128416) bank125223 +125224 POINT(40.75462584236668 74.9713891523601) bank125224 +125225 POINT(39.74334300618439 73.35378438879363) bank125225 +125226 POINT(41.375077677902915 74.48801567217008) bank125226 +125227 POINT(41.104821644491 73.34965448766954) bank125227 +125228 POINT(41.2743039730307 73.34013879866234) bank125228 +125229 POINT(39.716212490120576 73.32722503675812) bank125229 +125230 POINT(40.17036766474903 74.07076932783585) bank125230 +125231 POINT(40.44064643805009 74.0132782619238) bank125231 +125232 POINT(40.27200644495355 74.5168266232985) bank125232 +125233 POINT(39.96684063725308 73.72860195880443) bank125233 +125234 POINT(41.383936292627034 73.17456899959129) bank125234 +125235 POINT(40.8928756589542 74.66938876780597) bank125235 +125236 POINT(39.84411822597004 74.34562298356971) bank125236 +125237 POINT(39.956610871723235 74.56539314322892) bank125237 +125238 POINT(40.714747917267076 74.05209893381154) bank125238 +125239 POINT(39.76038212115772 73.72627899608392) bank125239 +125240 POINT(40.27301313420079 74.0046827464002) bank125240 +125241 POINT(40.79533989684922 73.41061898054737) bank125241 +125242 POINT(41.05309738020476 74.2381440859029) bank125242 +125243 POINT(40.216737319249575 73.37491795912433) bank125243 +125244 POINT(41.58544746126149 73.03971375894052) bank125244 +125245 POINT(41.152080759975306 74.52850740177739) bank125245 +125246 POINT(39.79998443119892 73.44936065390893) bank125246 +125247 POINT(40.49821440835916 74.00760658919836) bank125247 +125248 POINT(40.478801372504144 73.24623771541575) bank125248 +125249 POINT(39.87418049653019 73.60324041878981) bank125249 +125250 POINT(41.00624952195119 73.5814278010621) bank125250 +125251 POINT(40.78514143650325 75.00135574419043) bank125251 +125252 POINT(41.57193432126558 74.61016938740923) bank125252 +125253 POINT(41.23032673247619 73.44568381776081) bank125253 +125254 POINT(40.5357050382546 73.45093761859869) bank125254 +125255 POINT(41.14420291792841 74.16920284340996) bank125255 +125256 POINT(40.63792391238187 74.16674196842307) bank125256 +125257 POINT(41.078579602914836 74.92940981855266) bank125257 +125258 POINT(40.63105405860166 73.13046506513007) bank125258 +125259 POINT(40.7459924841914 74.52682467565346) bank125259 +125260 POINT(40.267023753471086 73.57433656095314) bank125260 +125261 POINT(40.61784400106529 73.52368184397696) bank125261 +125262 POINT(40.14903952836072 73.02355231503876) bank125262 +125263 POINT(40.293598646203506 73.10177370555817) bank125263 +125264 POINT(39.88761151390674 73.35175072588761) bank125264 +125265 POINT(40.97520015572126 74.02213339015145) bank125265 +125266 POINT(40.5225590314492 74.80514546525106) bank125266 +125267 POINT(39.756998748764936 73.54814235142102) bank125267 +125268 POINT(41.650984052029656 74.97111484424342) bank125268 +125269 POINT(40.93029698704412 74.7123883089573) bank125269 +125270 POINT(41.03297672252136 74.61608821395477) bank125270 +125271 POINT(40.380695214488924 73.44840412022003) bank125271 +125272 POINT(41.339712586981484 73.60671886280286) bank125272 +125273 POINT(41.26781881462801 74.90786464091578) bank125273 +125274 POINT(39.86086989605143 73.02003684043481) bank125274 +125275 POINT(41.23941472997809 73.20422412535716) bank125275 +125276 POINT(40.53630013053064 74.5729548004228) bank125276 +125277 POINT(40.04440561667266 74.263811374611) bank125277 +125278 POINT(40.2340397020604 74.48435241681211) bank125278 +125279 POINT(41.18787578218266 74.2507499329359) bank125279 +125280 POINT(40.748094389492614 73.23881332826438) bank125280 +125281 POINT(40.10927813033979 74.26439838331034) bank125281 +125282 POINT(41.62293972420248 73.84852414370096) bank125282 +125283 POINT(40.01220266390235 74.50354008457893) bank125283 +125284 POINT(41.66010139534748 74.02917940083621) bank125284 +125285 POINT(41.4182746201332 74.98335514934831) bank125285 +125286 POINT(41.59697536164486 73.86407054861513) bank125286 +125287 POINT(39.797207387008484 74.31657740546386) bank125287 +125288 POINT(40.041011813933274 73.93878757884612) bank125288 +125289 POINT(40.11576607469133 74.69006083314721) bank125289 +125290 POINT(41.33454520283677 73.39992866062316) bank125290 +125291 POINT(40.871754127728515 74.35831510748166) bank125291 +125292 POINT(39.77948797415126 74.59859704056177) bank125292 +125293 POINT(40.968483176116045 73.9625576401991) bank125293 +125294 POINT(40.43801992998455 74.78849633231306) bank125294 +125295 POINT(39.755993393385694 73.99343397084074) bank125295 +125296 POINT(40.188740658687145 74.81423614681043) bank125296 +125297 POINT(39.99134029656649 74.31099357537741) bank125297 +125298 POINT(40.524633827980665 74.97103452603673) bank125298 +125299 POINT(41.370161722447264 74.40213764004984) bank125299 +125300 POINT(40.06893970708783 73.6614238039391) bank125300 +125301 POINT(40.91660464686055 73.58184874189133) bank125301 +125302 POINT(40.55787004881673 74.39489051458453) bank125302 +125303 POINT(39.93309191811117 73.2673454434545) bank125303 +125304 POINT(41.46496920623241 74.55003526760633) bank125304 +125305 POINT(40.9821221235957 73.47236245125161) bank125305 +125306 POINT(40.35910360758456 73.62285588600693) bank125306 +125307 POINT(41.45872168329549 73.60964008289143) bank125307 +125308 POINT(40.58610324725813 74.36305791538663) bank125308 +125309 POINT(40.76729177875552 74.67625018388281) bank125309 +125310 POINT(40.88279125107612 73.3081140204166) bank125310 +125311 POINT(40.56089399531964 74.1093762230687) bank125311 +125312 POINT(41.234377344851765 73.94612592026301) bank125312 +125313 POINT(40.01104572368035 73.31490129597142) bank125313 +125314 POINT(40.06338349794789 74.97014605595852) bank125314 +125315 POINT(41.641155887950106 73.11328343606843) bank125315 +125316 POINT(41.52187981114102 73.63876252182078) bank125316 +125317 POINT(40.674087477030405 74.28823737012567) bank125317 +125318 POINT(39.85592301095409 74.14287322847265) bank125318 +125319 POINT(39.80888747989672 73.72406581306242) bank125319 +125320 POINT(41.62670705012467 74.11145758488686) bank125320 +125321 POINT(41.70193365457128 74.01891482843638) bank125321 +125322 POINT(41.153424666766384 73.72481184138742) bank125322 +125323 POINT(40.930950917211916 73.24927473080928) bank125323 +125324 POINT(39.84780281098074 74.66703133124231) bank125324 +125325 POINT(40.42955080940195 73.51181955995456) bank125325 +125326 POINT(40.08419713744825 73.49042879023351) bank125326 +125327 POINT(39.78232455985181 73.66687873979113) bank125327 +125328 POINT(40.576638090055944 74.40797811163287) bank125328 +125329 POINT(41.19311335314566 73.2325211888463) bank125329 +125330 POINT(41.564334560744236 74.19023972050051) bank125330 +125331 POINT(41.07869214509024 74.85242637618477) bank125331 +125332 POINT(39.9474778004453 73.8550233710424) bank125332 +125333 POINT(39.8916924978477 73.46676732681021) bank125333 +125334 POINT(40.80142960944594 73.4038625564504) bank125334 +125335 POINT(40.39904354789853 73.98071962529896) bank125335 +125336 POINT(39.85502983018534 74.6032866120066) bank125336 +125337 POINT(41.184763526313766 73.26327791134806) bank125337 +125338 POINT(39.76521864462708 73.24980554978056) bank125338 +125339 POINT(40.54812647564508 74.39961836178078) bank125339 +125340 POINT(39.835295345841914 74.22455275027065) bank125340 +125341 POINT(41.06822053597642 74.73184958914308) bank125341 +125342 POINT(40.71404103239209 73.06687198821881) bank125342 +125343 POINT(40.206558412218875 73.66851926895934) bank125343 +125344 POINT(41.616214950469335 73.39104976103944) bank125344 +125345 POINT(41.476579267027226 73.30364735545028) bank125345 +125346 POINT(41.21746218117214 74.90719116831094) bank125346 +125347 POINT(40.395441297465325 73.52372201204122) bank125347 +125348 POINT(40.19710926097947 74.03487004600647) bank125348 +125349 POINT(41.27173121367592 74.54270177328843) bank125349 +125350 POINT(40.44200577994381 73.02899314176916) bank125350 +125351 POINT(41.116135486476274 74.88152642137955) bank125351 +125352 POINT(39.77584810598322 73.4287741589282) bank125352 +125353 POINT(41.49391031650285 74.50972653583706) bank125353 +125354 POINT(40.44285561919116 73.32902859072608) bank125354 +125355 POINT(40.053443221201086 74.95618644310484) bank125355 +125356 POINT(40.30798905530296 73.8966805807495) bank125356 +125357 POINT(40.45725545569768 74.35333634754305) bank125357 +125358 POINT(41.38737776707211 74.53471556966834) bank125358 +125359 POINT(41.02814343180876 73.83014834852919) bank125359 +125360 POINT(40.36348829992865 74.84762603714843) bank125360 +125361 POINT(40.25579742909128 74.9636794291038) bank125361 +125362 POINT(39.7794085327943 74.94589200899185) bank125362 +125363 POINT(41.16058777703614 74.89193847202793) bank125363 +125364 POINT(40.45062707010854 74.92602183680633) bank125364 +125365 POINT(41.14913350970883 73.41179754163053) bank125365 +125366 POINT(39.962138066456795 74.4660756695822) bank125366 +125367 POINT(41.082168689960085 74.10540457943313) bank125367 +125368 POINT(39.803012036111824 73.39268460157017) bank125368 +125369 POINT(39.784425440875495 73.36701869199486) bank125369 +125370 POINT(41.656740493631574 74.06374590301485) bank125370 +125371 POINT(40.066183827768235 73.74574978217063) bank125371 +125372 POINT(40.288365431284134 74.10401383197821) bank125372 +125373 POINT(40.00996846069419 74.46702933020106) bank125373 +125374 POINT(41.633898287526954 74.37717643587868) bank125374 +125375 POINT(41.38425007325999 73.03118308008658) bank125375 +125376 POINT(40.76939546110863 73.7729071223299) bank125376 +125377 POINT(40.5527148388513 74.29117849867298) bank125377 +125378 POINT(41.017545934847284 74.73946930577789) bank125378 +125379 POINT(39.75594188292142 73.881609919184) bank125379 +125380 POINT(40.90372474117973 73.26680045696105) bank125380 +125381 POINT(41.623925726527176 73.60941955781121) bank125381 +125382 POINT(41.61984578725525 73.95826054679286) bank125382 +125383 POINT(40.36756327123757 74.45660263641581) bank125383 +125384 POINT(41.304755582305766 73.57296441512194) bank125384 +125385 POINT(40.778309720704044 73.07179754578713) bank125385 +125386 POINT(40.48810872486674 73.0799409939364) bank125386 +125387 POINT(40.50079628687533 73.19922254924066) bank125387 +125388 POINT(40.07049011848856 73.85404623173947) bank125388 +125389 POINT(41.28121697423025 74.87851346098039) bank125389 +125390 POINT(40.44272400439687 73.22804686718848) bank125390 +125391 POINT(40.932241502523 74.52423830364101) bank125391 +125392 POINT(40.04981335660185 73.07031963192446) bank125392 +125393 POINT(40.269373023480554 73.13402380084766) bank125393 +125394 POINT(40.76329639089249 74.9177797610606) bank125394 +125395 POINT(40.71903540690649 73.4344381973596) bank125395 +125396 POINT(40.04609546386614 74.38667852688903) bank125396 +125397 POINT(40.889629594189984 74.35232008261755) bank125397 +125398 POINT(41.55599339872579 73.23340518358656) bank125398 +125399 POINT(40.96701882235983 73.4292853980495) bank125399 +125400 POINT(41.06050993360985 73.43081686105796) bank125400 +125401 POINT(41.2256918975684 74.34317944947946) bank125401 +125402 POINT(39.94362133714645 73.84435723667225) bank125402 +125403 POINT(39.971314988114024 73.87683247638894) bank125403 +125404 POINT(41.53529007240352 73.28293423488262) bank125404 +125405 POINT(41.163738127480194 73.69847812027635) bank125405 +125406 POINT(41.00988728420211 74.56586260008879) bank125406 +125407 POINT(40.04148450382937 74.95671569321192) bank125407 +125408 POINT(40.87130374841146 74.81136944044134) bank125408 +125409 POINT(39.757747898307606 73.76019967891033) bank125409 +125410 POINT(41.1291074297768 74.05905270867967) bank125410 +125411 POINT(40.421479223673764 73.33492542428095) bank125411 +125412 POINT(39.99135820235546 73.12124906275285) bank125412 +125413 POINT(40.27556700642905 74.90279772701373) bank125413 +125414 POINT(39.81211612075019 73.91193425788644) bank125414 +125415 POINT(41.188840845434456 73.06325847588164) bank125415 +125416 POINT(40.191569895247355 73.7929691241072) bank125416 +125417 POINT(39.80679544956954 74.18395763488671) bank125417 +125418 POINT(39.93660670840653 73.35932656897282) bank125418 +125419 POINT(40.319709578661325 73.55232534016923) bank125419 +125420 POINT(40.265147586009164 74.76110485312947) bank125420 +125421 POINT(40.82928658722397 73.80566724407291) bank125421 +125422 POINT(41.5827844302538 74.7376913285814) bank125422 +125423 POINT(40.432297221784076 73.7538149426091) bank125423 +125424 POINT(40.05706877229685 74.97460919185644) bank125424 +125425 POINT(40.91322812618902 74.69443794431783) bank125425 +125426 POINT(40.0741005799673 74.92566080955767) bank125426 +125427 POINT(40.83636208678887 74.74270450212254) bank125427 +125428 POINT(40.13950266298799 74.67973082610072) bank125428 +125429 POINT(41.02858763563985 74.3279723630026) bank125429 +125430 POINT(40.30902985780453 74.63158448689143) bank125430 +125431 POINT(41.358184325468315 74.17346554749228) bank125431 +125432 POINT(41.40638172237056 73.57761976593939) bank125432 +125433 POINT(40.640982719668884 73.99834594123756) bank125433 +125434 POINT(40.362738709162905 74.15976397226788) bank125434 +125435 POINT(41.460613259224736 73.69830998394993) bank125435 +125436 POINT(40.51684324665085 73.18722076664359) bank125436 +125437 POINT(40.37369468547409 74.14693315192112) bank125437 +125438 POINT(40.09109231052642 73.7152286034898) bank125438 +125439 POINT(41.57842967522891 73.43444081751268) bank125439 +125440 POINT(40.25076874836335 73.52548651227492) bank125440 +125441 POINT(41.71061969464122 73.67337848037106) bank125441 +125442 POINT(40.09912044971006 73.95306583794468) bank125442 +125443 POINT(40.890207842069444 74.29833545746239) bank125443 +125444 POINT(40.92260582608978 74.05197355058151) bank125444 +125445 POINT(40.89245589266779 73.59504545285803) bank125445 +125446 POINT(40.609686195820125 74.56014952207109) bank125446 +125447 POINT(41.108340491607855 73.25389557236717) bank125447 +125448 POINT(40.76601294592193 74.31108552158929) bank125448 +125449 POINT(41.69493720618083 74.48091255300334) bank125449 +125450 POINT(40.40991014226772 73.52176873236432) bank125450 +125451 POINT(39.89862807570728 74.56269738188735) bank125451 +125452 POINT(40.57640219946245 73.21210464317895) bank125452 +125453 POINT(40.624938404376735 74.01507969425576) bank125453 +125454 POINT(41.61389810838201 74.15453943196653) bank125454 +125455 POINT(40.17799099172078 73.38560928975333) bank125455 +125456 POINT(41.34100314457601 73.10120294492481) bank125456 +125457 POINT(40.759157645519835 74.52294675632236) bank125457 +125458 POINT(41.31442583835199 74.70596520670709) bank125458 +125459 POINT(41.24978190713737 73.79088129153986) bank125459 +125460 POINT(41.30158532958546 73.76723914951985) bank125460 +125461 POINT(41.391324878779436 74.80874056728624) bank125461 +125462 POINT(40.60898948583231 74.43857372948015) bank125462 +125463 POINT(40.46430681072404 73.95234411874904) bank125463 +125464 POINT(41.63607658593911 74.97133359259217) bank125464 +125465 POINT(40.353261190203355 73.36085317053421) bank125465 +125466 POINT(39.80786632830404 74.2336255719519) bank125466 +125467 POINT(41.23398538185861 74.82713009028058) bank125467 +125468 POINT(41.39718856267505 73.86937119088645) bank125468 +125469 POINT(41.140326870602024 73.01228707925664) bank125469 +125470 POINT(40.294503084294014 74.3600913570233) bank125470 +125471 POINT(41.27107749657816 73.20619375614267) bank125471 +125472 POINT(41.42134315582776 74.86081469360855) bank125472 +125473 POINT(41.343545553911355 73.88288441893808) bank125473 +125474 POINT(41.09047575493337 74.39547477298788) bank125474 +125475 POINT(41.4018425141883 73.75212179637491) bank125475 +125476 POINT(40.93270937308402 74.13735654202391) bank125476 +125477 POINT(40.88756957253402 73.9670040691435) bank125477 +125478 POINT(41.5952115908208 73.63990511451719) bank125478 +125479 POINT(39.988060416220236 73.84960256545124) bank125479 +125480 POINT(40.62959470567911 74.70458287708561) bank125480 +125481 POINT(40.18249655776387 74.38880093978689) bank125481 +125482 POINT(40.173453889742326 73.95683913169563) bank125482 +125483 POINT(41.26035920028488 74.09015977464622) bank125483 +125484 POINT(41.06531972636533 73.63298720682279) bank125484 +125485 POINT(39.95082025324919 73.68562551764184) bank125485 +125486 POINT(41.13896108794057 74.53164692808875) bank125486 +125487 POINT(40.22916372789446 74.94463103610659) bank125487 +125488 POINT(41.596409653872904 74.34730865032549) bank125488 +125489 POINT(40.42019989401769 73.63266015503665) bank125489 +125490 POINT(41.34451087643691 73.73103043308686) bank125490 +125491 POINT(40.017420191475594 74.86493117314348) bank125491 +125492 POINT(41.120880197000645 74.21437702317397) bank125492 +125493 POINT(40.69106507321762 74.06981797817059) bank125493 +125494 POINT(40.11830911063964 73.53429092093975) bank125494 +125495 POINT(40.44498940481228 73.22998822909481) bank125495 +125496 POINT(41.45929336661526 74.4098596740015) bank125496 +125497 POINT(41.55229343229236 74.89401627364255) bank125497 +125498 POINT(40.797964844036414 73.31889603269957) bank125498 +125499 POINT(41.49849081859664 74.82989532034739) bank125499 +125500 POINT(41.084086012576975 73.82877202466165) bank125500 +125501 POINT(40.6352513793005 74.67076253718638) bank125501 +125502 POINT(41.52704712898856 74.27365875319796) bank125502 +125503 POINT(40.30547840235072 73.91716291545616) bank125503 +125504 POINT(41.39665984162357 74.12239019010876) bank125504 +125505 POINT(40.855610810348516 73.20616885905079) bank125505 +125506 POINT(40.1593280633512 73.53815101568449) bank125506 +125507 POINT(40.95879460903148 74.68437021452317) bank125507 +125508 POINT(41.29073662474702 74.89091325089717) bank125508 +125509 POINT(41.61669942259178 74.72610587813399) bank125509 +125510 POINT(40.72274416783466 74.85246081941261) bank125510 +125511 POINT(39.945416114231904 73.4641772462229) bank125511 +125512 POINT(41.07443197147038 73.95366329088048) bank125512 +125513 POINT(41.59883543747723 74.59802777887228) bank125513 +125514 POINT(40.71496800923285 74.81920611045443) bank125514 +125515 POINT(41.06184328479156 74.82841982110047) bank125515 +125516 POINT(40.793926522269004 73.37927078289188) bank125516 +125517 POINT(40.73971698968291 73.05799146979804) bank125517 +125518 POINT(40.566920941285744 73.6934937305429) bank125518 +125519 POINT(40.258978442676266 73.72899695513456) bank125519 +125520 POINT(40.93245365164164 74.44165734143874) bank125520 +125521 POINT(41.07305841231262 74.57194310987279) bank125521 +125522 POINT(39.82959262486021 73.35717626520112) bank125522 +125523 POINT(39.870714580542604 74.80140495998451) bank125523 +125524 POINT(39.990263130970305 73.5325207575221) bank125524 +125525 POINT(40.16015228024462 74.21547636215223) bank125525 +125526 POINT(41.59996601384935 73.32354904519804) bank125526 +125527 POINT(40.98672388741231 73.03416415920272) bank125527 +125528 POINT(39.76618182980494 73.47129555996553) bank125528 +125529 POINT(41.69178399956919 74.38306160802684) bank125529 +125530 POINT(40.46517261010986 73.56618149811837) bank125530 +125531 POINT(40.289812300654134 73.3763685297182) bank125531 +125532 POINT(40.383714432478065 73.69036310165275) bank125532 +125533 POINT(40.439776665950454 74.48298547216858) bank125533 +125534 POINT(39.94891290384557 74.16299423659304) bank125534 +125535 POINT(39.84064569673094 74.39976235902535) bank125535 +125536 POINT(40.01471131525862 73.86016484086295) bank125536 +125537 POINT(41.27524949601065 74.73214915116989) bank125537 +125538 POINT(40.832966191188085 74.38089364273756) bank125538 +125539 POINT(40.660278736656096 74.66833050832231) bank125539 +125540 POINT(40.19976178951951 73.61219835375353) bank125540 +125541 POINT(41.09473137188961 74.13590319354226) bank125541 +125542 POINT(40.831240659127985 73.84338934335082) bank125542 +125543 POINT(40.07602334087087 74.83802845561395) bank125543 +125544 POINT(40.658347388126536 74.30109406233258) bank125544 +125545 POINT(39.79927833436962 74.27064569275488) bank125545 +125546 POINT(41.13444111027114 73.21181333602875) bank125546 +125547 POINT(39.778239078627735 74.99932434452224) bank125547 +125548 POINT(40.58002812250369 73.79048375150113) bank125548 +125549 POINT(40.22654337327565 74.87867152093412) bank125549 +125550 POINT(40.56285503843426 73.56865072613571) bank125550 +125551 POINT(40.57200539165638 73.09653603031184) bank125551 +125552 POINT(40.35946604380402 74.83810181061952) bank125552 +125553 POINT(39.721086099885106 73.14893676711992) bank125553 +125554 POINT(41.30503499957832 74.35230027223642) bank125554 +125555 POINT(40.08398714840611 74.97040819311695) bank125555 +125556 POINT(41.29750710806389 74.80193221392138) bank125556 +125557 POINT(40.82009230047579 74.21098857824893) bank125557 +125558 POINT(39.91791864114483 73.11334638302675) bank125558 +125559 POINT(41.52855073805575 74.18835343102917) bank125559 +125560 POINT(41.256832104179374 73.67611184266177) bank125560 +125561 POINT(40.82762209659264 73.33549510246552) bank125561 +125562 POINT(41.597908527545805 74.66943683752636) bank125562 +125563 POINT(41.39939394119634 74.9777672545974) bank125563 +125564 POINT(40.55119895089033 74.95986550330389) bank125564 +125565 POINT(41.208052599394605 74.35745124947434) bank125565 +125566 POINT(41.101637927730714 74.99623678191341) bank125566 +125567 POINT(41.34094951783986 74.48929748268112) bank125567 +125568 POINT(41.462618763171484 74.74797350825787) bank125568 +125569 POINT(39.71789060255514 74.98011117776278) bank125569 +125570 POINT(41.27123176888832 74.8785802119204) bank125570 +125571 POINT(41.300385886124175 74.1893784391051) bank125571 +125572 POINT(41.32874232085129 73.72133850298599) bank125572 +125573 POINT(40.59828738107134 74.67264815412568) bank125573 +125574 POINT(40.167948341776764 74.56989133592194) bank125574 +125575 POINT(39.93307143767515 74.66018815733904) bank125575 +125576 POINT(41.21877279747297 74.79828424318913) bank125576 +125577 POINT(40.379354500992385 74.40120882782911) bank125577 +125578 POINT(40.31648263407783 74.65530022319851) bank125578 +125579 POINT(41.06283876805805 73.66723582380082) bank125579 +125580 POINT(40.07690670584397 74.63945585397289) bank125580 +125581 POINT(40.517567483110184 73.564091114746) bank125581 +125582 POINT(40.540952922536405 73.22608175689824) bank125582 +125583 POINT(40.5213391571909 74.12777263448824) bank125583 +125584 POINT(41.15615552281239 74.34102993249458) bank125584 +125585 POINT(41.04692945723072 74.06096764651896) bank125585 +125586 POINT(40.65935810289336 74.83157718519008) bank125586 +125587 POINT(40.16233028686419 73.75642892904587) bank125587 +125588 POINT(40.26632206283998 73.85968015799979) bank125588 +125589 POINT(41.041885223719255 73.26534298663918) bank125589 +125590 POINT(40.32868488626349 73.0347075921809) bank125590 +125591 POINT(41.09527643576459 74.39919361256625) bank125591 +125592 POINT(41.03168049922012 74.31594877728033) bank125592 +125593 POINT(40.65018258093332 73.7000940083052) bank125593 +125594 POINT(40.35442254191123 73.39559192188224) bank125594 +125595 POINT(39.7159030606163 73.60271653817631) bank125595 +125596 POINT(41.36324418915167 74.47317265994948) bank125596 +125597 POINT(39.91567302721591 73.90530284720536) bank125597 +125598 POINT(40.58695410252215 73.45207044434594) bank125598 +125599 POINT(40.9517377757461 73.99645962945905) bank125599 +125600 POINT(41.09325653370744 74.12216711517087) bank125600 +125601 POINT(40.427001617371616 74.2664764900129) bank125601 +125602 POINT(40.89438791805699 73.64811933024308) bank125602 +125603 POINT(40.52890237366816 73.65889540217123) bank125603 +125604 POINT(40.439685456266545 73.46460629857326) bank125604 +125605 POINT(39.923153713257975 73.67056665563202) bank125605 +125606 POINT(41.34668021485857 74.52174406913969) bank125606 +125607 POINT(40.62789380341345 74.06313647460865) bank125607 +125608 POINT(40.30906188005269 74.69649312257003) bank125608 +125609 POINT(41.304217117907704 74.80006896722476) bank125609 +125610 POINT(39.82369554390925 74.33398248843825) bank125610 +125611 POINT(41.23615075684647 73.90199495754939) bank125611 +125612 POINT(40.064698929183294 73.73277340718894) bank125612 +125613 POINT(40.9382226771177 74.17043802257704) bank125613 +125614 POINT(40.94639533965494 74.74245622030243) bank125614 +125615 POINT(41.51585736172436 73.91313738230542) bank125615 +125616 POINT(40.087347862289015 74.4810556126405) bank125616 +125617 POINT(39.865708493090736 74.49184509300613) bank125617 +125618 POINT(40.197190019906074 74.51188584854762) bank125618 +125619 POINT(40.32201104708608 73.86977805630819) bank125619 +125620 POINT(40.746422045960394 74.6034619078608) bank125620 +125621 POINT(39.746851399552725 73.06585163758929) bank125621 +125622 POINT(40.72427082222151 74.36225141081053) bank125622 +125623 POINT(41.41712773875557 73.95121082031501) bank125623 +125624 POINT(39.95439151550398 74.20125217469307) bank125624 +125625 POINT(40.52376240563067 73.53473031473135) bank125625 +125626 POINT(40.80927876402715 73.57831074838374) bank125626 +125627 POINT(41.69041051220807 74.0883644900709) bank125627 +125628 POINT(40.58075179135121 73.81095454153272) bank125628 +125629 POINT(41.0223804769692 73.12269179447387) bank125629 +125630 POINT(40.483279707656926 73.43715238218404) bank125630 +125631 POINT(41.19474568132362 74.8294276893418) bank125631 +125632 POINT(41.49972860502125 74.45922264962127) bank125632 +125633 POINT(41.408588141196866 74.56651192097564) bank125633 +125634 POINT(40.520546116599306 74.84848190377531) bank125634 +125635 POINT(40.767668713434325 73.89492933383086) bank125635 +125636 POINT(41.69182757854663 73.46742055296811) bank125636 +125637 POINT(40.03788879593327 74.70463643025013) bank125637 +125638 POINT(40.54593715288142 73.23556093391419) bank125638 +125639 POINT(41.32583556399385 73.91705930523358) bank125639 +125640 POINT(39.844716432778334 73.55358852504918) bank125640 +125641 POINT(40.62832737733701 74.87273323955205) bank125641 +125642 POINT(40.98711158708559 74.52467532186841) bank125642 +125643 POINT(41.339309429835446 73.91746263259128) bank125643 +125644 POINT(41.55826957775549 74.35657257950965) bank125644 +125645 POINT(41.569123263203586 74.26801621282698) bank125645 +125646 POINT(39.97573044772493 74.71640654186221) bank125646 +125647 POINT(41.536766927633074 73.74590850182337) bank125647 +125648 POINT(40.769334733649714 73.40649634780432) bank125648 +125649 POINT(41.697460165817745 73.32039776660227) bank125649 +125650 POINT(41.71263077044705 74.53263290819363) bank125650 +125651 POINT(40.35003034434344 73.54176034536484) bank125651 +125652 POINT(41.49629837333322 74.91533430510587) bank125652 +125653 POINT(40.93431262966072 73.81581987809717) bank125653 +125654 POINT(40.03382239838472 74.73369924289972) bank125654 +125655 POINT(40.39367880146747 73.14696036876772) bank125655 +125656 POINT(40.03287666785913 73.18153653943118) bank125656 +125657 POINT(40.624577137495706 74.77116408974523) bank125657 +125658 POINT(39.83719895403717 74.21353319891452) bank125658 +125659 POINT(40.08927716708054 74.40586981816314) bank125659 +125660 POINT(41.326062919570795 74.72782260654678) bank125660 +125661 POINT(39.84717226709814 73.52913685401073) bank125661 +125662 POINT(40.13696080990894 73.24251154547993) bank125662 +125663 POINT(41.559920339577644 74.01852352340573) bank125663 +125664 POINT(41.13994058673453 74.19861337409117) bank125664 +125665 POINT(40.63318676531816 73.97278564731106) bank125665 +125666 POINT(39.87987299252247 73.1768552873331) bank125666 +125667 POINT(40.32043490291036 74.39266676761237) bank125667 +125668 POINT(41.13637460157059 74.21137547798546) bank125668 +125669 POINT(40.45058447664972 73.3126769996865) bank125669 +125670 POINT(40.74639700790715 73.65920246941613) bank125670 +125671 POINT(40.62085314360415 73.12042498003647) bank125671 +125672 POINT(39.96457779556836 73.24628949346035) bank125672 +125673 POINT(39.73047398430139 73.34536546832582) bank125673 +125674 POINT(39.75041930042602 73.166090808485) bank125674 +125675 POINT(41.064118955833024 73.64396399337339) bank125675 +125676 POINT(39.859242335187375 74.75254018718427) bank125676 +125677 POINT(41.12887027911302 74.78571167623137) bank125677 +125678 POINT(40.11991335544488 74.88092000878784) bank125678 +125679 POINT(39.820029370071126 74.21472166158739) bank125679 +125680 POINT(41.35259037715394 74.67333966670786) bank125680 +125681 POINT(40.020278439090056 73.18665362667141) bank125681 +125682 POINT(40.86263107866717 73.90825790908642) bank125682 +125683 POINT(41.032180837700686 74.56874371897985) bank125683 +125684 POINT(40.967159522403676 73.61052385731116) bank125684 +125685 POINT(41.33742249929804 74.6426908126548) bank125685 +125686 POINT(40.82522494288114 74.54798618395228) bank125686 +125687 POINT(39.818636526431746 73.27027876714995) bank125687 +125688 POINT(41.0365632783065 74.80076082198961) bank125688 +125689 POINT(41.26659694970868 74.24840924161344) bank125689 +125690 POINT(39.91683082451503 73.27114019229327) bank125690 +125691 POINT(41.437350305772284 74.8295568617553) bank125691 +125692 POINT(41.32578280703938 73.84151406644239) bank125692 +125693 POINT(40.50116127945912 73.06625697152256) bank125693 +125694 POINT(41.656850465826096 73.69737677329148) bank125694 +125695 POINT(40.83952969889599 74.6965069102301) bank125695 +125696 POINT(41.23020451395405 73.93324844020171) bank125696 +125697 POINT(40.520518547735705 73.27417837057773) bank125697 +125698 POINT(39.86874002398233 73.2875594794574) bank125698 +125699 POINT(40.69002295726882 73.37675700381124) bank125699 +125700 POINT(40.35371822376499 73.65717684238125) bank125700 +125701 POINT(40.563842372494626 73.11453017969156) bank125701 +125702 POINT(40.899997961356284 74.06582583724419) bank125702 +125703 POINT(40.19952102291236 74.49246401856819) bank125703 +125704 POINT(41.035673094154575 73.55448350684846) bank125704 +125705 POINT(41.36960344887562 74.58890708876064) bank125705 +125706 POINT(41.34515106872468 74.13005734943522) bank125706 +125707 POINT(40.05655851257831 73.40526536952152) bank125707 +125708 POINT(41.512974321732 74.10912890345915) bank125708 +125709 POINT(41.14670658894669 73.26050153995192) bank125709 +125710 POINT(41.117767145449534 73.50656251637399) bank125710 +125711 POINT(41.06600814136005 73.13514820612873) bank125711 +125712 POINT(40.419175285745595 74.84830951188437) bank125712 +125713 POINT(41.605696743834976 73.54600472373832) bank125713 +125714 POINT(40.37663216677955 74.8101257518745) bank125714 +125715 POINT(40.61266179855029 73.65557063837285) bank125715 +125716 POINT(41.46611851986512 74.49466816449635) bank125716 +125717 POINT(40.47706752027769 73.64087191136038) bank125717 +125718 POINT(41.08672537628889 73.16261225871119) bank125718 +125719 POINT(40.39462431345897 73.75074221251583) bank125719 +125720 POINT(40.31655280249421 74.82710609623949) bank125720 +125721 POINT(41.601649139472336 74.12950796952282) bank125721 +125722 POINT(40.1916940427645 74.56193314294065) bank125722 +125723 POINT(41.70324907966775 73.08165087891867) bank125723 +125724 POINT(41.698048501542964 74.97814749842613) bank125724 +125725 POINT(41.143177373231566 73.65343296112488) bank125725 +125726 POINT(40.27626276051827 74.88673003683954) bank125726 +125727 POINT(40.83556522979857 73.73416798309506) bank125727 +125728 POINT(41.54839973527766 74.9441116010117) bank125728 +125729 POINT(39.90427662894304 74.63790132101718) bank125729 +125730 POINT(41.586790843539774 73.66210676749314) bank125730 +125731 POINT(40.52448752097346 74.62234430752895) bank125731 +125732 POINT(41.50477833063357 74.9132811045601) bank125732 +125733 POINT(40.3329174346416 74.02447978064846) bank125733 +125734 POINT(41.16358258473763 73.96630734323628) bank125734 +125735 POINT(41.705584298676655 73.88712563465654) bank125735 +125736 POINT(40.36296941627935 74.0981499952994) bank125736 +125737 POINT(40.24059495092625 73.95780238218818) bank125737 +125738 POINT(40.156293873261674 74.42304749328936) bank125738 +125739 POINT(41.09881032165289 73.22193478497447) bank125739 +125740 POINT(41.58007699409876 73.45322938217183) bank125740 +125741 POINT(41.454885938130246 73.11915521540585) bank125741 +125742 POINT(40.75194598066286 74.77481056294847) bank125742 +125743 POINT(40.59718069371225 74.06157204515871) bank125743 +125744 POINT(41.13459309662103 73.28292411633221) bank125744 +125745 POINT(40.923328673201766 73.62133076240605) bank125745 +125746 POINT(41.076130533800715 74.79693501858888) bank125746 +125747 POINT(41.69161743523807 74.43813464124396) bank125747 +125748 POINT(41.60832957124801 73.0915968044127) bank125748 +125749 POINT(41.12890928596623 73.87135817277286) bank125749 +125750 POINT(39.87471393061307 73.47715714713058) bank125750 +125751 POINT(41.38490044518658 73.80765841439616) bank125751 +125752 POINT(40.776675758772754 74.08814064788957) bank125752 +125753 POINT(40.81848614477337 73.66403060810136) bank125753 +125754 POINT(39.92230939360726 74.58358266411518) bank125754 +125755 POINT(41.26047823115829 74.67651786090109) bank125755 +125756 POINT(39.84084470525681 73.59001484379904) bank125756 +125757 POINT(40.23768519792359 73.68239044708041) bank125757 +125758 POINT(40.64072277366415 74.56631912703293) bank125758 +125759 POINT(39.747119560808166 73.95477175778169) bank125759 +125760 POINT(41.69394294517213 74.29098174116669) bank125760 +125761 POINT(40.990022679760145 74.52019819088399) bank125761 +125762 POINT(39.99138301711484 73.07575545683605) bank125762 +125763 POINT(40.733778003270544 73.78506463375219) bank125763 +125764 POINT(40.427618861507895 74.1488456842775) bank125764 +125765 POINT(41.55771159311015 74.07450417997694) bank125765 +125766 POINT(40.312024562093015 73.97782199142219) bank125766 +125767 POINT(39.727325089400125 74.02779229365248) bank125767 +125768 POINT(40.92776811242228 74.46397639990442) bank125768 +125769 POINT(39.88568153889931 73.49217685748366) bank125769 +125770 POINT(40.62363523789272 73.2443266373852) bank125770 +125771 POINT(39.76404624082625 73.3903246755937) bank125771 +125772 POINT(41.247564518900205 74.980859176111) bank125772 +125773 POINT(41.286255536256434 73.82155244833189) bank125773 +125774 POINT(41.2317597494373 74.36630383508188) bank125774 +125775 POINT(40.87615734656716 74.14909323056901) bank125775 +125776 POINT(41.550604068097655 74.39424047399973) bank125776 +125777 POINT(41.19477671173158 73.51979856101129) bank125777 +125778 POINT(40.516200849528275 74.03557243458893) bank125778 +125779 POINT(41.27950362165705 73.46588100927985) bank125779 +125780 POINT(41.31726310198551 74.24771646545022) bank125780 +125781 POINT(40.32449377257982 73.4391695210148) bank125781 +125782 POINT(41.23745203954836 73.8037682549698) bank125782 +125783 POINT(39.84381465474443 74.92503694952526) bank125783 +125784 POINT(41.047513447692914 74.72600024038694) bank125784 +125785 POINT(39.940187990875565 74.1420887660537) bank125785 +125786 POINT(41.013861265508595 73.54347063316519) bank125786 +125787 POINT(39.82469088910903 73.4992647751713) bank125787 +125788 POINT(40.44448521523265 74.74507426796086) bank125788 +125789 POINT(39.98709254598016 74.15588060158841) bank125789 +125790 POINT(39.913481040182795 73.26158618953177) bank125790 +125791 POINT(40.17900714608757 74.08377678682109) bank125791 +125792 POINT(41.28467456794933 74.86968638999156) bank125792 +125793 POINT(40.63662042164161 74.66274726002159) bank125793 +125794 POINT(40.59726931223845 73.73786967575441) bank125794 +125795 POINT(41.46724520246469 73.69140528794273) bank125795 +125796 POINT(39.85993086149126 74.38873249252495) bank125796 +125797 POINT(40.332009074151344 73.74234770693889) bank125797 +125798 POINT(40.90159440465531 73.19801747324021) bank125798 +125799 POINT(40.568855535793226 73.00933432680648) bank125799 +125800 POINT(39.83161646989382 74.44749571471944) bank125800 +125801 POINT(41.64989017473605 74.00603460432943) bank125801 +125802 POINT(41.04328592238462 74.1833951530147) bank125802 +125803 POINT(40.98256781488912 73.47885669260477) bank125803 +125804 POINT(40.579110363511795 73.7492881129126) bank125804 +125805 POINT(41.330437948921 73.41585853948322) bank125805 +125806 POINT(40.05652798571009 74.5752044692334) bank125806 +125807 POINT(39.929982168432616 74.52577478743329) bank125807 +125808 POINT(41.50377183839624 73.32818820116215) bank125808 +125809 POINT(41.675429165171074 73.29207428977585) bank125809 +125810 POINT(40.35873350717583 73.58331393122218) bank125810 +125811 POINT(39.92851833812864 73.45128265830624) bank125811 +125812 POINT(41.4101690799754 73.21347399825382) bank125812 +125813 POINT(40.02059409073396 73.34984360517338) bank125813 +125814 POINT(41.04963387872771 74.41679711000806) bank125814 +125815 POINT(41.536570922506186 74.46801134221207) bank125815 +125816 POINT(40.36447272473985 73.60761525702952) bank125816 +125817 POINT(40.371959201303966 73.99273241730447) bank125817 +125818 POINT(41.13432179055431 73.88660574981665) bank125818 +125819 POINT(40.575950156744334 73.33447435778939) bank125819 +125820 POINT(40.07278776722828 74.05401993187867) bank125820 +125821 POINT(40.86481015429293 73.9519470127511) bank125821 +125822 POINT(40.77005823715905 74.15656818234147) bank125822 +125823 POINT(41.06214510492828 74.09191504737659) bank125823 +125824 POINT(41.63011384056898 73.92483608668888) bank125824 +125825 POINT(40.800575246639156 74.95919030864609) bank125825 +125826 POINT(40.90453439828238 74.35693445581136) bank125826 +125827 POINT(41.69112316886175 73.04472448721803) bank125827 +125828 POINT(41.53510414076319 74.64024817563725) bank125828 +125829 POINT(40.049350475048726 73.60290839455713) bank125829 +125830 POINT(41.39511440163297 73.59030632521701) bank125830 +125831 POINT(39.77085629327654 73.87893390773489) bank125831 +125832 POINT(40.220235446342215 74.24933996108854) bank125832 +125833 POINT(41.005257263185804 74.12324487163288) bank125833 +125834 POINT(41.562891514359734 74.64399037952477) bank125834 +125835 POINT(39.85067287177699 73.43398081143064) bank125835 +125836 POINT(40.61649079474512 74.46669760486851) bank125836 +125837 POINT(40.82041571563059 73.99825948190865) bank125837 +125838 POINT(40.02366424316753 73.24026292111874) bank125838 +125839 POINT(41.024029476469096 74.2680372373029) bank125839 +125840 POINT(41.28786300307923 74.97475777040117) bank125840 +125841 POINT(40.78828887921004 74.03317680401351) bank125841 +125842 POINT(40.30640438812919 73.46927874190223) bank125842 +125843 POINT(41.048751738975405 74.70705080400532) bank125843 +125844 POINT(41.32538570445135 73.67341354105608) bank125844 +125845 POINT(41.207679122353724 73.79383597952456) bank125845 +125846 POINT(40.64964314894359 73.13478716993133) bank125846 +125847 POINT(41.19807492019129 74.96899397884121) bank125847 +125848 POINT(39.752308580977335 74.68317217276677) bank125848 +125849 POINT(40.428276031130665 73.21514941336022) bank125849 +125850 POINT(41.59669406699704 74.52419222811749) bank125850 +125851 POINT(40.797395086839536 74.0811254635709) bank125851 +125852 POINT(40.84572623187532 73.40509831692536) bank125852 +125853 POINT(39.79034265096606 73.64542907139935) bank125853 +125854 POINT(40.13544331295521 73.69361819664176) bank125854 +125855 POINT(40.80398851495162 73.84047150703539) bank125855 +125856 POINT(40.280355071364454 73.26666127699475) bank125856 +125857 POINT(41.13770577605916 73.34971789280884) bank125857 +125858 POINT(41.568577296541626 73.9783722000067) bank125858 +125859 POINT(41.50312185700042 73.75840316949078) bank125859 +125860 POINT(41.655835900480724 73.85804579026616) bank125860 +125861 POINT(40.684119880883635 73.12713621445977) bank125861 +125862 POINT(40.85454246588069 73.62074234086572) bank125862 +125863 POINT(40.309605596187794 74.95237312612853) bank125863 +125864 POINT(39.76080934265417 74.83488280854108) bank125864 +125865 POINT(41.706946660604174 74.96907478981393) bank125865 +125866 POINT(39.748393882870914 74.01888833720342) bank125866 +125867 POINT(40.445118914213914 74.20811465870453) bank125867 +125868 POINT(40.060022610017654 73.7820620275791) bank125868 +125869 POINT(40.62176527139104 73.0937175800238) bank125869 +125870 POINT(41.04012227793013 74.87449896638354) bank125870 +125871 POINT(39.82350748636261 73.94159531178302) bank125871 +125872 POINT(40.910401602981125 74.62880244517163) bank125872 +125873 POINT(40.956284613856596 74.94021467172266) bank125873 +125874 POINT(39.72412820022858 74.98670800565124) bank125874 +125875 POINT(41.20847745578206 74.3391020315299) bank125875 +125876 POINT(39.72387716210091 73.29762694968466) bank125876 +125877 POINT(40.58975978026088 73.20179758274055) bank125877 +125878 POINT(41.35392973044995 73.92268562501003) bank125878 +125879 POINT(40.53500080923233 73.55237043547605) bank125879 +125880 POINT(41.08963623412605 73.91908140361271) bank125880 +125881 POINT(41.20824262800252 73.80674333281775) bank125881 +125882 POINT(40.13243735950153 74.33282210675682) bank125882 +125883 POINT(40.72754211637856 73.90013933389938) bank125883 +125884 POINT(40.14994880577777 74.60898583307498) bank125884 +125885 POINT(41.576206201914154 74.97430524784264) bank125885 +125886 POINT(41.078379924611866 73.79780837980674) bank125886 +125887 POINT(41.264351750943895 73.17728365129577) bank125887 +125888 POINT(40.683739485660524 73.63080129756815) bank125888 +125889 POINT(41.65794912883087 73.82684755999324) bank125889 +125890 POINT(40.496495502189674 73.24671013814098) bank125890 +125891 POINT(41.133882134829356 74.98251372076817) bank125891 +125892 POINT(40.69486902383894 73.13089268937777) bank125892 +125893 POINT(39.85349627140614 74.42671466754229) bank125893 +125894 POINT(40.25220753608086 73.96883443000682) bank125894 +125895 POINT(41.615943501736474 73.85203280686873) bank125895 +125896 POINT(41.63409442646035 73.67089927809229) bank125896 +125897 POINT(41.30912360040296 74.15729201244358) bank125897 +125898 POINT(41.64770794577352 73.82575004058347) bank125898 +125899 POINT(41.14038947112076 73.8098146377672) bank125899 +125900 POINT(40.94071901669363 73.95732321276158) bank125900 +125901 POINT(40.90305295306418 74.22192564469506) bank125901 +125902 POINT(41.21181041976308 74.08396801772874) bank125902 +125903 POINT(40.53806958515351 74.83592824910455) bank125903 +125904 POINT(40.60351709558994 73.87286655280943) bank125904 +125905 POINT(40.39395007574669 73.9522714716044) bank125905 +125906 POINT(39.75213586203372 74.86767752558484) bank125906 +125907 POINT(39.797087588154255 74.76338731525112) bank125907 +125908 POINT(41.69571839071131 73.19232275631309) bank125908 +125909 POINT(41.08779225584454 74.93305683485686) bank125909 +125910 POINT(41.1397565528677 74.2872261390663) bank125910 +125911 POINT(41.48732374753669 73.77013672965028) bank125911 +125912 POINT(39.80824245334519 74.21560685124992) bank125912 +125913 POINT(40.056568625449636 73.57879819757694) bank125913 +125914 POINT(40.48511514135671 73.37805431933704) bank125914 +125915 POINT(41.52886162329485 73.8598071592378) bank125915 +125916 POINT(40.30839298495542 74.15779208882599) bank125916 +125917 POINT(39.9743125842067 74.20052213907152) bank125917 +125918 POINT(39.96438552003925 74.7785826884164) bank125918 +125919 POINT(40.81081777788095 74.6926509133526) bank125919 +125920 POINT(40.861311467752444 74.0574730720542) bank125920 +125921 POINT(39.79403097318146 73.23842480115225) bank125921 +125922 POINT(40.59552814500221 74.45118975659139) bank125922 +125923 POINT(41.328866028537234 74.83354127114212) bank125923 +125924 POINT(40.34267805968754 74.55517127450821) bank125924 +125925 POINT(40.393915175534545 74.37153046331302) bank125925 +125926 POINT(41.30387619174444 74.95922689383185) bank125926 +125927 POINT(41.17917611246009 74.57841363970293) bank125927 +125928 POINT(41.5347091792702 74.85554514768519) bank125928 +125929 POINT(41.37901351849914 74.72874466084937) bank125929 +125930 POINT(40.63452410035293 73.92771960290652) bank125930 +125931 POINT(40.277045532916446 73.52896856513892) bank125931 +125932 POINT(41.34747078642872 73.19711640108495) bank125932 +125933 POINT(40.21642911113508 74.37961271174854) bank125933 +125934 POINT(41.122910120687465 74.06043064272038) bank125934 +125935 POINT(41.23300626149375 73.85397976162194) bank125935 +125936 POINT(40.399558092644426 73.80361638476522) bank125936 +125937 POINT(41.1136655723064 74.29540864174948) bank125937 +125938 POINT(40.49299499138906 73.61756831327656) bank125938 +125939 POINT(40.2596718499288 74.19086239765305) bank125939 +125940 POINT(41.15779995052723 73.86178257183361) bank125940 +125941 POINT(41.58045278251817 73.37594750894108) bank125941 +125942 POINT(39.99381912829104 73.57577952217355) bank125942 +125943 POINT(41.70845456754404 73.3426525336552) bank125943 +125944 POINT(41.5243899573129 74.06073782719298) bank125944 +125945 POINT(40.81415834603846 73.87478807405799) bank125945 +125946 POINT(40.037618666196536 74.8651491175688) bank125946 +125947 POINT(41.392530848765965 73.09276997292837) bank125947 +125948 POINT(40.5251370119423 74.28310956218117) bank125948 +125949 POINT(40.854702017447934 73.78682690733125) bank125949 +125950 POINT(39.892786513196306 74.65821164246968) bank125950 +125951 POINT(40.70471923158975 73.76968838427086) bank125951 +125952 POINT(41.156730121970746 74.43419413470959) bank125952 +125953 POINT(39.855877699434664 73.26041409803226) bank125953 +125954 POINT(40.21103277518343 73.00846371973002) bank125954 +125955 POINT(40.48845962974423 74.59238017714583) bank125955 +125956 POINT(40.617236094691435 74.57890545141571) bank125956 +125957 POINT(40.248468489119354 74.47627047297055) bank125957 +125958 POINT(41.18722059989674 73.95143341290174) bank125958 +125959 POINT(40.48092938783034 73.26691859758361) bank125959 +125960 POINT(40.620210426977835 74.78025084424429) bank125960 +125961 POINT(40.876938516017155 73.3073786757312) bank125961 +125962 POINT(41.65471507739289 73.42409387584877) bank125962 +125963 POINT(40.00277232288416 74.68925929418228) bank125963 +125964 POINT(40.810351542641364 73.42107933084242) bank125964 +125965 POINT(41.20051317332473 73.6815092378365) bank125965 +125966 POINT(41.315738762038244 74.89053772331377) bank125966 +125967 POINT(41.16238727660838 73.72259081450451) bank125967 +125968 POINT(40.3500432672512 74.48718203712043) bank125968 +125969 POINT(39.9515705177837 73.09332303473464) bank125969 +125970 POINT(41.58405234891072 74.58324954572022) bank125970 +125971 POINT(40.634571225705486 73.31878136861489) bank125971 +125972 POINT(41.548673611204805 73.81113416259026) bank125972 +125973 POINT(41.67453377096162 74.80550872731915) bank125973 +125974 POINT(41.287024708676384 74.7685518469607) bank125974 +125975 POINT(40.07608844623505 73.12139564501338) bank125975 +125976 POINT(41.63548086751309 73.03944348580305) bank125976 +125977 POINT(40.55140386621546 74.15546448015817) bank125977 +125978 POINT(40.8355004480551 74.55279931305454) bank125978 +125979 POINT(40.496176341003775 73.93273377291716) bank125979 +125980 POINT(40.56714273369057 74.19412458915274) bank125980 +125981 POINT(41.35003402195874 73.79608335494956) bank125981 +125982 POINT(41.15621195247036 73.55328462427346) bank125982 +125983 POINT(40.7038892397855 73.07207900324302) bank125983 +125984 POINT(40.65954544084594 74.08844343302917) bank125984 +125985 POINT(41.26540268678535 73.25151544528093) bank125985 +125986 POINT(40.0435159184338 74.17697559807735) bank125986 +125987 POINT(40.422067569567446 73.21424644977695) bank125987 +125988 POINT(40.17767429895713 73.24541938095177) bank125988 +125989 POINT(40.5976835510518 74.49360583757945) bank125989 +125990 POINT(40.07763100912114 73.17719156866809) bank125990 +125991 POINT(40.699608200772595 73.78396506763038) bank125991 +125992 POINT(39.82652114380129 73.8606496970869) bank125992 +125993 POINT(39.872021199009446 74.92870040671355) bank125993 +125994 POINT(40.144624669481914 73.29038109926766) bank125994 +125995 POINT(41.25868623424759 73.81481703383145) bank125995 +125996 POINT(40.507132183005226 74.62803699115908) bank125996 +125997 POINT(41.54094165360636 73.02333590834137) bank125997 +125998 POINT(41.35212188686847 74.80567775839621) bank125998 +125999 POINT(39.825508760913536 73.01514687184262) bank125999 +126000 POINT(41.484604176401035 74.75696909779161) bank126000 +126001 POINT(41.337744940923734 73.69231958552881) bank126001 +126002 POINT(40.866657597339085 74.17995704192587) bank126002 +126003 POINT(41.27028453863157 74.99729682381292) bank126003 +126004 POINT(41.298698676460226 73.77936531715005) bank126004 +126005 POINT(41.6646522362303 73.5553496273028) bank126005 +126006 POINT(40.955837400737586 74.02489896031058) bank126006 +126007 POINT(39.93775123944002 73.77171686902318) bank126007 +126008 POINT(39.846480660210275 74.49800983201418) bank126008 +126009 POINT(41.40780801224605 73.37113151695223) bank126009 +126010 POINT(41.14680837595001 74.3717062050825) bank126010 +126011 POINT(40.58346885237517 73.28675500717036) bank126011 +126012 POINT(41.4523182663119 73.31276499113814) bank126012 +126013 POINT(41.27147926578994 75.00315116409531) bank126013 +126014 POINT(41.47314829943922 73.1317996663585) bank126014 +126015 POINT(40.15806831385746 74.32579129444528) bank126015 +126016 POINT(40.74058368172919 73.70446063503955) bank126016 +126017 POINT(40.941193130160464 73.95084466719783) bank126017 +126018 POINT(40.644696958773636 73.06413568971395) bank126018 +126019 POINT(41.27846194480929 73.26831940364696) bank126019 +126020 POINT(40.98480020397763 74.92134502483202) bank126020 +126021 POINT(41.367256839915704 73.0427359246428) bank126021 +126022 POINT(41.353953933857234 73.37040069521561) bank126022 +126023 POINT(40.334865818232835 74.75008569514763) bank126023 +126024 POINT(41.50482537031961 74.3519957081107) bank126024 +126025 POINT(41.59679153191092 73.63083654744642) bank126025 +126026 POINT(41.5477925134496 74.86588782891242) bank126026 +126027 POINT(40.221016594637355 74.01410050810105) bank126027 +126028 POINT(39.72547337929993 74.58064848550735) bank126028 +126029 POINT(40.02550660323705 73.54576057729442) bank126029 +126030 POINT(40.45578697622554 73.44022425309679) bank126030 +126031 POINT(39.85909415234735 73.20860657130784) bank126031 +126032 POINT(39.8845684177709 73.57492544671815) bank126032 +126033 POINT(40.77788415238719 74.51445258849829) bank126033 +126034 POINT(40.33312995035228 73.75570368469225) bank126034 +126035 POINT(40.51136578888168 74.12989078630648) bank126035 +126036 POINT(40.712237487509135 73.56137745579917) bank126036 +126037 POINT(41.50827231448459 74.12441955758615) bank126037 +126038 POINT(41.31354648804857 73.22136643302892) bank126038 +126039 POINT(41.19631594473981 74.30002927052749) bank126039 +126040 POINT(41.23352302285088 73.33018348197864) bank126040 +126041 POINT(41.20216739255994 73.44312680423403) bank126041 +126042 POINT(39.7964576440141 74.32173439869213) bank126042 +126043 POINT(40.1827794456656 73.3004022485583) bank126043 +126044 POINT(41.09890822219323 73.6663745838645) bank126044 +126045 POINT(41.113808063390316 74.65076562944053) bank126045 +126046 POINT(41.542997922039056 73.3497130205499) bank126046 +126047 POINT(39.90926486901526 74.80063317327219) bank126047 +126048 POINT(41.447331735344626 73.36056919637049) bank126048 +126049 POINT(40.61156992393184 73.99746795686977) bank126049 +126050 POINT(40.22172425039005 74.33151511683391) bank126050 +126051 POINT(40.63393107841938 74.23315720792756) bank126051 +126052 POINT(41.052409843882934 73.05173935589995) bank126052 +126053 POINT(40.97265424088105 73.19514314840562) bank126053 +126054 POINT(40.83826198342225 74.44717634268666) bank126054 +126055 POINT(40.71152165797286 73.68296009088327) bank126055 +126056 POINT(41.6695930185355 74.1898434872475) bank126056 +126057 POINT(40.88186896494696 74.5011405462462) bank126057 +126058 POINT(41.32831289211419 74.06318940979428) bank126058 +126059 POINT(40.08694293481862 74.66513182466556) bank126059 +126060 POINT(40.611348111762574 73.54886461093899) bank126060 +126061 POINT(41.57652936100133 74.14324329557664) bank126061 +126062 POINT(39.80822402078686 74.5897020882497) bank126062 +126063 POINT(40.421488736419434 74.81560568580277) bank126063 +126064 POINT(41.25566518997061 74.26307425833217) bank126064 +126065 POINT(40.130875980399495 74.18976588252212) bank126065 +126066 POINT(40.12416004826724 73.25159133401625) bank126066 +126067 POINT(41.41694507198426 74.87621093841541) bank126067 +126068 POINT(40.80390180971295 73.01497280610286) bank126068 +126069 POINT(41.00732133484502 74.3687830796724) bank126069 +126070 POINT(41.71078093305094 74.7784554558088) bank126070 +126071 POINT(41.14425165859708 73.53619544713469) bank126071 +126072 POINT(41.29739784010736 74.65276286172771) bank126072 +126073 POINT(41.67636803238371 73.41208658424776) bank126073 +126074 POINT(39.77117687906907 74.94174183590039) bank126074 +126075 POINT(41.68498729454762 74.30520601136212) bank126075 +126076 POINT(40.253967933831326 73.6278446131099) bank126076 +126077 POINT(41.583334763897106 74.96878430864336) bank126077 +126078 POINT(41.21865416078185 73.21815737639172) bank126078 +126079 POINT(41.02955457532278 73.95172307221483) bank126079 +126080 POINT(41.07476103009954 73.78911186484304) bank126080 +126081 POINT(40.19936929854823 74.43996821009982) bank126081 +126082 POINT(40.201606908074325 74.75287000204244) bank126082 +126083 POINT(40.706789903065506 73.2764633155218) bank126083 +126084 POINT(40.62520489705286 74.7033138784619) bank126084 +126085 POINT(39.7286439670964 73.48731730681851) bank126085 +126086 POINT(40.353293604153826 74.38896165900273) bank126086 +126087 POINT(41.226965652039304 74.0480989718604) bank126087 +126088 POINT(40.178181572768764 73.25813542398512) bank126088 +126089 POINT(40.045870333104524 73.17728409548229) bank126089 +126090 POINT(40.77598884370241 74.38605257417288) bank126090 +126091 POINT(40.31849367844327 73.22633256509843) bank126091 +126092 POINT(40.26724687252302 73.785504677107) bank126092 +126093 POINT(41.66446989143698 73.26972331758991) bank126093 +126094 POINT(40.134190459404806 74.24369327252369) bank126094 +126095 POINT(41.132533287461754 73.6469907886778) bank126095 +126096 POINT(40.354762852188664 73.30403624487327) bank126096 +126097 POINT(40.87508186874662 73.09843009741472) bank126097 +126098 POINT(39.738032696592896 74.49922367580281) bank126098 +126099 POINT(39.71292243138063 74.69263774652674) bank126099 +126100 POINT(41.02147557410293 73.64315427425964) bank126100 +126101 POINT(41.579040346052864 73.84200545789297) bank126101 +126102 POINT(40.753127037977436 74.00861776693522) bank126102 +126103 POINT(40.42975058506298 73.78146770403187) bank126103 +126104 POINT(40.83113106030571 73.5668005834708) bank126104 +126105 POINT(40.102015800480864 73.67627035521494) bank126105 +126106 POINT(41.613706691107865 73.75952966216865) bank126106 +126107 POINT(40.80891060519777 74.61627502642665) bank126107 +126108 POINT(39.75771683137652 73.23784820491666) bank126108 +126109 POINT(40.581754846645104 73.78062754402974) bank126109 +126110 POINT(41.13209568866661 73.44079794753502) bank126110 +126111 POINT(40.87638681967344 73.05176520908476) bank126111 +126112 POINT(39.8663202363526 74.22432745070859) bank126112 +126113 POINT(40.758881656247524 73.24840397297314) bank126113 +126114 POINT(40.50372757622443 74.8953293263265) bank126114 +126115 POINT(40.26890337633396 73.9283137352195) bank126115 +126116 POINT(40.796346848891226 73.63578067558305) bank126116 +126117 POINT(41.44961687064836 73.80660147343765) bank126117 +126118 POINT(41.33503649493242 74.29997209914842) bank126118 +126119 POINT(41.111146058188915 74.32166605608325) bank126119 +126120 POINT(39.93197475347407 73.79247527822554) bank126120 +126121 POINT(40.26701017758842 73.21338597075304) bank126121 +126122 POINT(40.38102278982707 74.30781854578993) bank126122 +126123 POINT(40.4936966009646 73.83961763936854) bank126123 +126124 POINT(41.467671337221105 73.01566526232187) bank126124 +126125 POINT(41.28049205384662 73.33502352695257) bank126125 +126126 POINT(41.16829929188974 73.05632321675122) bank126126 +126127 POINT(41.68711892950174 74.96533095265578) bank126127 +126128 POINT(40.13334776257497 73.57629241054522) bank126128 +126129 POINT(41.13581173192524 74.1265041659974) bank126129 +126130 POINT(40.5225779715759 74.54300278532725) bank126130 +126131 POINT(41.612132677712374 74.56050603536224) bank126131 +126132 POINT(40.6217360910891 74.86368411385814) bank126132 +126133 POINT(40.97592666757044 74.28121376407006) bank126133 +126134 POINT(40.5466406451059 73.62169280661453) bank126134 +126135 POINT(41.647284557529254 73.45570876486155) bank126135 +126136 POINT(40.81695207310031 74.06569884120943) bank126136 +126137 POINT(39.8523730946421 73.58588355066263) bank126137 +126138 POINT(40.38478120423784 74.86742948670503) bank126138 +126139 POINT(40.661639329213784 74.4046757190438) bank126139 +126140 POINT(41.32893597287122 74.46693955564915) bank126140 +126141 POINT(41.52066025333388 74.23681707297989) bank126141 +126142 POINT(39.8107781958503 74.85012189388617) bank126142 +126143 POINT(41.077789917458084 73.63888179500788) bank126143 +126144 POINT(40.20794823382655 73.57425547392235) bank126144 +126145 POINT(40.73035674954285 74.14884860580776) bank126145 +126146 POINT(40.1407166940303 73.98153989661566) bank126146 +126147 POINT(41.1890994753662 74.39686374099935) bank126147 +126148 POINT(39.72211679505873 74.90329531731403) bank126148 +126149 POINT(40.55093015956053 74.1576861608819) bank126149 +126150 POINT(41.05896776246848 73.14510661155366) bank126150 +126151 POINT(41.09979505277575 73.97884140897924) bank126151 +126152 POINT(40.7173273241721 74.90716094477861) bank126152 +126153 POINT(40.54959510193773 74.62685034086314) bank126153 +126154 POINT(41.05309660964085 73.6830618989716) bank126154 +126155 POINT(40.06437313137475 73.94733924391096) bank126155 +126156 POINT(40.266289671382985 74.64681987861073) bank126156 +126157 POINT(41.28730011332638 74.66694193718943) bank126157 +126158 POINT(40.33880573632653 74.1518100768276) bank126158 +126159 POINT(40.995587884715874 73.8607334157628) bank126159 +126160 POINT(39.755788459401515 73.19654911133068) bank126160 +126161 POINT(40.527344363658436 73.42732142642048) bank126161 +126162 POINT(40.87068828668528 74.15555284053588) bank126162 +126163 POINT(39.97536005196254 73.37794911003408) bank126163 +126164 POINT(40.82662333157043 73.83093481323748) bank126164 +126165 POINT(40.95980304366152 74.19050983640776) bank126165 +126166 POINT(41.7001237423361 73.90955294402038) bank126166 +126167 POINT(40.74293057319434 73.17807228592285) bank126167 +126168 POINT(41.57609441050937 73.96714709563834) bank126168 +126169 POINT(41.39310508068904 73.21824454964381) bank126169 +126170 POINT(41.316651860813366 74.05465307723159) bank126170 +126171 POINT(40.090153610684695 73.52461733968197) bank126171 +126172 POINT(40.696760688548885 73.60154067949786) bank126172 +126173 POINT(41.46437713106988 74.18971026768236) bank126173 +126174 POINT(40.269209952783655 74.01239005777937) bank126174 +126175 POINT(39.79383799666451 74.57909345803738) bank126175 +126176 POINT(41.68281293488142 73.45367138345375) bank126176 +126177 POINT(39.801752355117735 74.14136379829802) bank126177 +126178 POINT(40.22170834421222 74.34756875215149) bank126178 +126179 POINT(40.13594054576011 73.58456981967262) bank126179 +126180 POINT(39.99431266730777 74.6950057640266) bank126180 +126181 POINT(40.407539861151385 73.90868478446583) bank126181 +126182 POINT(40.01802616154731 74.40244904661625) bank126182 +126183 POINT(40.15882958660584 74.67634405324168) bank126183 +126184 POINT(41.13939659723755 73.9246732408263) bank126184 +126185 POINT(40.262070347834175 74.7748312954714) bank126185 +126186 POINT(41.05753792137029 73.00866315225824) bank126186 +126187 POINT(40.44815144005144 73.3095986278087) bank126187 +126188 POINT(41.698478224246216 74.5842452900958) bank126188 +126189 POINT(40.7085696125308 74.35050221322963) bank126189 +126190 POINT(40.404332746245124 73.94524642956883) bank126190 +126191 POINT(40.89749306787215 73.54238375497535) bank126191 +126192 POINT(41.25401070927856 74.7696889537775) bank126192 +126193 POINT(41.210645687211844 74.04990037781597) bank126193 +126194 POINT(41.52173578031034 73.26440415269713) bank126194 +126195 POINT(41.031459432477334 74.07639088884375) bank126195 +126196 POINT(41.554117324787626 73.41183063343784) bank126196 +126197 POINT(40.92564319912416 73.83239248631693) bank126197 +126198 POINT(40.80062618785708 73.36418153941574) bank126198 +126199 POINT(40.57868120350878 73.37574678312569) bank126199 +126200 POINT(41.35165147710562 73.40296911734025) bank126200 +126201 POINT(39.79040247124776 74.02522350751954) bank126201 +126202 POINT(40.75428776239098 74.81372260598627) bank126202 +126203 POINT(41.031236133597154 74.14655147430629) bank126203 +126204 POINT(40.63589382029467 74.30451987486907) bank126204 +126205 POINT(40.10221005154697 73.95467899083059) bank126205 +126206 POINT(41.457037602296246 73.85404443483499) bank126206 +126207 POINT(41.552870087050884 74.93607775089843) bank126207 +126208 POINT(40.2871046945839 74.84399875815917) bank126208 +126209 POINT(39.878953656599904 74.7007766500125) bank126209 +126210 POINT(40.61309015792891 74.06992441742865) bank126210 +126211 POINT(40.876497668821166 74.9651203761809) bank126211 +126212 POINT(40.63194336616908 73.9492097408495) bank126212 +126213 POINT(39.80608653279587 73.10988614192702) bank126213 +126214 POINT(40.61530109666144 73.8822680629069) bank126214 +126215 POINT(41.35539406061828 73.3095382220169) bank126215 +126216 POINT(40.953654110446436 73.8460945792985) bank126216 +126217 POINT(41.006452439921446 73.71409001533047) bank126217 +126218 POINT(39.72812093625735 74.39332207897273) bank126218 +126219 POINT(40.733028243636355 73.15648459776361) bank126219 +126220 POINT(40.853318584536666 74.96858334357407) bank126220 +126221 POINT(40.33406065585163 73.24114326298114) bank126221 +126222 POINT(41.70958206226392 74.64266946140941) bank126222 +126223 POINT(40.55976131826727 74.13989460078867) bank126223 +126224 POINT(40.701979358561104 73.74594888727468) bank126224 +126225 POINT(41.2446843697423 73.88587604629129) bank126225 +126226 POINT(41.16496038534021 74.37519365718103) bank126226 +126227 POINT(40.6028197664694 74.21533271003108) bank126227 +126228 POINT(40.0096155284442 74.32355943794738) bank126228 +126229 POINT(40.591003387679734 73.63559676900421) bank126229 +126230 POINT(41.48667108622379 73.00720646824523) bank126230 +126231 POINT(41.65078604056852 74.73760384615152) bank126231 +126232 POINT(41.05259349578563 74.32549419185642) bank126232 +126233 POINT(41.55410583807112 73.21975666576382) bank126233 +126234 POINT(39.92703403189917 74.32674459655743) bank126234 +126235 POINT(40.012039698998734 73.28672791024695) bank126235 +126236 POINT(40.304582209827956 73.5496938795289) bank126236 +126237 POINT(40.861167501946326 74.67163612543888) bank126237 +126238 POINT(41.431519870216015 73.44634863362347) bank126238 +126239 POINT(40.98368126756114 74.99389297990433) bank126239 +126240 POINT(39.75485685910685 74.16946455061327) bank126240 +126241 POINT(41.173055813941 73.05595862749185) bank126241 +126242 POINT(39.84068232404452 74.59079806132056) bank126242 +126243 POINT(39.98127059529137 74.25191285836907) bank126243 +126244 POINT(40.545167716652024 74.19234774967003) bank126244 +126245 POINT(41.35471849278008 73.39685426144337) bank126245 +126246 POINT(40.08629050165872 74.67694330289838) bank126246 +126247 POINT(40.121092911127086 74.84850669517297) bank126247 +126248 POINT(40.77255912188456 74.0946031731998) bank126248 +126249 POINT(40.90935537867013 74.04758890681414) bank126249 +126250 POINT(40.4468069948993 74.31522189799858) bank126250 +126251 POINT(40.13793940042917 73.57472786314491) bank126251 +126252 POINT(39.74022489518747 73.88292629826964) bank126252 +126253 POINT(40.23226804608194 73.04203759477586) bank126253 +126254 POINT(40.079357611436166 74.47782530641666) bank126254 +126255 POINT(39.94490358157081 74.37600917700699) bank126255 +126256 POINT(41.17211405510093 73.82887159200035) bank126256 +126257 POINT(40.73226243398649 74.93983860613483) bank126257 +126258 POINT(40.902673689915304 73.86160472750842) bank126258 +126259 POINT(41.09570622541652 73.13288534480333) bank126259 +126260 POINT(40.023170307167014 74.22172878925444) bank126260 +126261 POINT(40.842386849161 73.40987374072205) bank126261 +126262 POINT(40.07120898093273 74.89580804459064) bank126262 +126263 POINT(40.983158142663335 73.87214167892859) bank126263 +126264 POINT(40.79683346795187 73.55114920975706) bank126264 +126265 POINT(41.13511183457765 74.75281293320012) bank126265 +126266 POINT(41.16019061468614 74.88843119917512) bank126266 +126267 POINT(41.10183681878356 74.38535300734172) bank126267 +126268 POINT(40.583758698187026 74.80942444634331) bank126268 +126269 POINT(39.90430358126792 74.21595864283343) bank126269 +126270 POINT(40.908168043675765 73.86534825494266) bank126270 +126271 POINT(40.7611042251301 73.1985576626907) bank126271 +126272 POINT(41.0183575878533 74.58140619142823) bank126272 +126273 POINT(39.932460028825524 73.3171120225796) bank126273 +126274 POINT(41.53386978614761 74.55611694555438) bank126274 +126275 POINT(41.1095412249322 74.79169087141638) bank126275 +126276 POINT(41.564601251175816 73.1567888621924) bank126276 +126277 POINT(39.78763830806775 73.75643309896672) bank126277 +126278 POINT(40.76721996474515 73.64618804517512) bank126278 +126279 POINT(40.34126868274914 74.46316348860212) bank126279 +126280 POINT(40.83727338797819 74.11855750656883) bank126280 +126281 POINT(40.874049077737794 74.45400951147009) bank126281 +126282 POINT(41.076314501622214 73.21765098219483) bank126282 +126283 POINT(40.87434884349242 74.13827803797874) bank126283 +126284 POINT(40.7114671274372 74.52908619052371) bank126284 +126285 POINT(41.68762738650202 74.20295284351647) bank126285 +126286 POINT(40.83299379211488 73.95094060125186) bank126286 +126287 POINT(41.655941151807006 74.75588699915723) bank126287 +126288 POINT(41.539345907111866 73.69549548144686) bank126288 +126289 POINT(39.90516557309451 73.78112843777565) bank126289 +126290 POINT(39.93684902913718 73.87862399552604) bank126290 +126291 POINT(40.402038098763065 73.79590103484676) bank126291 +126292 POINT(40.216995989987396 73.80202856368157) bank126292 +126293 POINT(40.95668081117459 74.35693807741794) bank126293 +126294 POINT(40.13394106810973 74.3947939548673) bank126294 +126295 POINT(41.641896071705226 74.67804930926143) bank126295 +126296 POINT(39.923084468936416 74.95489526927172) bank126296 +126297 POINT(41.633898849171395 73.42634092226517) bank126297 +126298 POINT(41.520930143633024 73.45227914857742) bank126298 +126299 POINT(39.80620863062424 73.96460782470936) bank126299 +126300 POINT(40.92091968748342 74.88464405221525) bank126300 +126301 POINT(39.894836129629816 74.33897087020871) bank126301 +126302 POINT(39.99560581345023 73.58822400613931) bank126302 +126303 POINT(39.78743954895605 74.15911582303487) bank126303 +126304 POINT(39.88264191591114 73.39386842713618) bank126304 +126305 POINT(41.51932208742068 74.21142578486203) bank126305 +126306 POINT(41.192434109510025 74.81944025627753) bank126306 +126307 POINT(40.20853373969374 74.46815200534941) bank126307 +126308 POINT(40.3128765153135 73.33207439766562) bank126308 +126309 POINT(39.74089800875536 73.60852836434613) bank126309 +126310 POINT(39.73233512654011 73.61906816116681) bank126310 +126311 POINT(41.66920546866722 74.88449849936565) bank126311 +126312 POINT(39.896945501002044 73.21453308683319) bank126312 +126313 POINT(40.74638151759616 73.12861913889917) bank126313 +126314 POINT(40.558251048949714 74.62431467098082) bank126314 +126315 POINT(40.648785724815134 73.63726995427153) bank126315 +126316 POINT(41.18350584919621 74.0308521486781) bank126316 +126317 POINT(39.95902149620237 74.91558041539848) bank126317 +126318 POINT(40.7585144593382 73.73875568901255) bank126318 +126319 POINT(39.96163596650554 74.08512955628466) bank126319 +126320 POINT(39.763617122157285 74.49926081681204) bank126320 +126321 POINT(40.537447325004706 73.22862328270955) bank126321 +126322 POINT(40.74062641975678 74.01736686078938) bank126322 +126323 POINT(40.563200832273616 74.17093911763095) bank126323 +126324 POINT(40.21701948690638 73.57831186086601) bank126324 +126325 POINT(41.51851273832655 73.26946491475763) bank126325 +126326 POINT(39.748254541567555 74.26068702409623) bank126326 +126327 POINT(41.63589397082418 74.89591553260561) bank126327 +126328 POINT(40.83828620975756 73.83292824404758) bank126328 +126329 POINT(41.088931474402635 74.57352999136295) bank126329 +126330 POINT(40.34736945258674 74.63366639213056) bank126330 +126331 POINT(39.9369832644858 73.26225565930899) bank126331 +126332 POINT(40.085693590163956 74.96534513562715) bank126332 +126333 POINT(40.87408134067759 74.5478171574583) bank126333 +126334 POINT(40.429921201378264 73.27434211410436) bank126334 +126335 POINT(41.40119399824819 73.47210033441223) bank126335 +126336 POINT(41.281040997847704 73.4958333842762) bank126336 +126337 POINT(39.90978259231012 73.30741091530703) bank126337 +126338 POINT(41.10588075446379 73.91113523747298) bank126338 +126339 POINT(40.06346533815318 73.6206938546508) bank126339 +126340 POINT(40.24983459297684 73.50017253306646) bank126340 +126341 POINT(40.872551995520304 74.81954319284019) bank126341 +126342 POINT(41.349821498833194 74.03773714234778) bank126342 +126343 POINT(41.41992594523861 73.46180099060223) bank126343 +126344 POINT(40.293630784464725 73.12844848574808) bank126344 +126345 POINT(41.04804388308941 74.8929767245678) bank126345 +126346 POINT(41.27002602195941 74.23220228801395) bank126346 +126347 POINT(40.958587866529356 73.47329676622576) bank126347 +126348 POINT(39.79432700682913 73.75971970564483) bank126348 +126349 POINT(41.06994289766091 73.56588550868754) bank126349 +126350 POINT(41.090606213489146 73.7816568820602) bank126350 +126351 POINT(41.14566564209848 73.69519030431444) bank126351 +126352 POINT(40.31462046381065 74.71251561950434) bank126352 +126353 POINT(40.30266903637588 73.14806024142526) bank126353 +126354 POINT(40.69074070947262 74.91620015291642) bank126354 +126355 POINT(41.58005771504806 74.44007076575801) bank126355 +126356 POINT(41.513617032155864 73.5636617018007) bank126356 +126357 POINT(39.91557426650012 73.57707936837318) bank126357 +126358 POINT(40.99351916380237 73.93110116225843) bank126358 +126359 POINT(41.48111981115033 73.08359468464512) bank126359 +126360 POINT(39.880508961325354 74.3116178750459) bank126360 +126361 POINT(40.60239444726371 73.82689321055527) bank126361 +126362 POINT(40.017432355294495 74.49697821727466) bank126362 +126363 POINT(41.431841075407014 74.6845767950581) bank126363 +126364 POINT(40.71029077592254 74.85575910175103) bank126364 +126365 POINT(39.75353892764454 74.90967277267055) bank126365 +126366 POINT(39.967965607951044 74.74896792941888) bank126366 +126367 POINT(41.46117298674017 73.58869594408404) bank126367 +126368 POINT(41.49300823607334 73.82975671395388) bank126368 +126369 POINT(40.15368601424517 74.18662014248372) bank126369 +126370 POINT(40.53885694048168 74.28257342356825) bank126370 +126371 POINT(40.730538477019515 74.71084104512121) bank126371 +126372 POINT(41.646216475290544 74.95366755688855) bank126372 +126373 POINT(40.76248239377726 73.92684532716845) bank126373 +126374 POINT(40.236491574140985 74.23388592322657) bank126374 +126375 POINT(40.325718914503845 73.2802132681378) bank126375 +126376 POINT(39.969222680419264 74.44924335860004) bank126376 +126377 POINT(40.84469454158255 73.73144838631559) bank126377 +126378 POINT(41.28857189918971 74.55023047946622) bank126378 +126379 POINT(41.121628772325636 74.43820589305854) bank126379 +126380 POINT(40.966562820578474 73.88490574655697) bank126380 +126381 POINT(40.59066074693667 73.06030350524085) bank126381 +126382 POINT(40.0126156021739 73.82701447599626) bank126382 +126383 POINT(40.95172893236418 74.28693732509407) bank126383 +126384 POINT(40.31199951427869 74.15729968326933) bank126384 +126385 POINT(39.74349558037958 74.33219903593546) bank126385 +126386 POINT(41.67056067832072 73.68631047811728) bank126386 +126387 POINT(41.27252911144685 73.10152897256349) bank126387 +126388 POINT(41.155107608961046 73.04883854427129) bank126388 +126389 POINT(40.00613797479241 74.0213117659735) bank126389 +126390 POINT(40.52255006581535 73.42254567188736) bank126390 +126391 POINT(40.91735422739786 73.04444185523548) bank126391 +126392 POINT(40.92164259816201 73.849437714688) bank126392 +126393 POINT(41.077837138087816 74.65260602240325) bank126393 +126394 POINT(41.29430636040723 73.32481419441456) bank126394 +126395 POINT(40.11497247331459 73.00895795864058) bank126395 +126396 POINT(41.66271801532297 74.88808521057031) bank126396 +126397 POINT(41.64115271811656 73.0259254342999) bank126397 +126398 POINT(40.581620115625824 73.90051622446217) bank126398 +126399 POINT(41.442806865702 74.60531870643032) bank126399 +126400 POINT(41.24083963203719 73.16772592997295) bank126400 +126401 POINT(40.539555342653784 73.2782712867746) bank126401 +126402 POINT(41.12989672675221 73.8137510546597) bank126402 +126403 POINT(40.66801854544693 73.46145718983199) bank126403 +126404 POINT(39.76137744304208 74.71603270813047) bank126404 +126405 POINT(40.23430973119903 73.90931068303331) bank126405 +126406 POINT(41.669778258645685 74.76644832676946) bank126406 +126407 POINT(39.921107602266616 74.88528599914216) bank126407 +126408 POINT(41.054797243814654 74.24034937816954) bank126408 +126409 POINT(41.55224093886236 73.8782303981822) bank126409 +126410 POINT(39.96371293583761 74.96079161868292) bank126410 +126411 POINT(40.69509324290816 74.57203833486172) bank126411 +126412 POINT(40.88668976614421 73.94338310470194) bank126412 +126413 POINT(40.03646844316747 74.3501051415034) bank126413 +126414 POINT(41.22208633872528 74.76720172472903) bank126414 +126415 POINT(40.37957906387718 73.90410112721669) bank126415 +126416 POINT(41.174802416565164 74.31306272036828) bank126416 +126417 POINT(40.505757322697804 74.11591487683698) bank126417 +126418 POINT(41.449447723890074 73.45353834027969) bank126418 +126419 POINT(41.3040975855696 74.43750556597126) bank126419 +126420 POINT(40.777126044956006 73.18836559636844) bank126420 +126421 POINT(41.02614684528259 73.36607175962777) bank126421 +126422 POINT(41.29518650887411 73.08616715745791) bank126422 +126423 POINT(40.14553379335355 73.76608752887711) bank126423 +126424 POINT(39.798297517124425 73.52032282232408) bank126424 +126425 POINT(39.86303130416675 73.32300937679527) bank126425 +126426 POINT(40.230355357123216 74.20297245403847) bank126426 +126427 POINT(40.04710252505951 73.95385616505881) bank126427 +126428 POINT(40.61379730710345 73.91926525492406) bank126428 +126429 POINT(40.162460723927545 74.95267084431367) bank126429 +126430 POINT(41.52945789499047 74.65088226327474) bank126430 +126431 POINT(40.13171903684839 73.33155375354275) bank126431 +126432 POINT(41.282833688808076 73.88812814389338) bank126432 +126433 POINT(39.99282745842304 73.80526295488953) bank126433 +126434 POINT(41.671458189201594 74.86423482882614) bank126434 +126435 POINT(40.621789932301674 73.74347552682029) bank126435 +126436 POINT(40.02766326197 74.92637998468928) bank126436 +126437 POINT(40.67634791295269 74.64322329968736) bank126437 +126438 POINT(40.39639012770923 74.03716735351674) bank126438 +126439 POINT(40.9624521180769 73.40720428026518) bank126439 +126440 POINT(40.74105972130705 73.02133574812892) bank126440 +126441 POINT(41.61144748425093 74.81957652754448) bank126441 +126442 POINT(39.88486999822661 73.39925538580304) bank126442 +126443 POINT(41.1517360018149 74.01419283022366) bank126443 +126444 POINT(39.879182856656456 73.47109035297336) bank126444 +126445 POINT(40.41540377458001 74.87527125726288) bank126445 +126446 POINT(39.79992459116643 73.38388033015622) bank126446 +126447 POINT(39.82305988763197 74.79663106022717) bank126447 +126448 POINT(40.09639410429991 74.63924430833283) bank126448 +126449 POINT(40.623683281416284 74.56926662341266) bank126449 +126450 POINT(41.34815741508072 73.08034923262234) bank126450 +126451 POINT(39.89940996059987 73.9761360870494) bank126451 +126452 POINT(40.675229184149245 73.73253851381858) bank126452 +126453 POINT(40.82439190899319 73.24746984260248) bank126453 +126454 POINT(41.278174780099945 73.38112285982918) bank126454 +126455 POINT(39.81456110238129 73.88654057278856) bank126455 +126456 POINT(40.3419800979602 74.73781195994054) bank126456 +126457 POINT(41.48234363588514 74.05940263433517) bank126457 +126458 POINT(41.691814286322085 74.10361916340545) bank126458 +126459 POINT(40.63834426940754 73.35300833006559) bank126459 +126460 POINT(39.79141285459973 74.26751329195295) bank126460 +126461 POINT(41.26201916043742 73.71355741803146) bank126461 +126462 POINT(41.38431952448525 74.68357449687065) bank126462 +126463 POINT(40.814527254343275 74.51653241207927) bank126463 +126464 POINT(40.22708809847239 73.28293941218777) bank126464 +126465 POINT(39.98470098401284 74.94703894528081) bank126465 +126466 POINT(39.92885061557188 74.26100912364197) bank126466 +126467 POINT(41.34242525141718 74.77355497998231) bank126467 +126468 POINT(40.46252552711681 74.61355229432347) bank126468 +126469 POINT(40.90223898871863 74.88418491676372) bank126469 +126470 POINT(41.14554323711076 73.34034657932315) bank126470 +126471 POINT(40.69554567736386 74.56795649643398) bank126471 +126472 POINT(40.04343773178904 73.1209704637762) bank126472 +126473 POINT(40.76404241338298 73.51144338487269) bank126473 +126474 POINT(40.717203237563005 73.38308325644745) bank126474 +126475 POINT(41.68975109025345 74.41803245390837) bank126475 +126476 POINT(40.22315837636343 74.4829949716118) bank126476 +126477 POINT(41.12573448047358 74.78254628031638) bank126477 +126478 POINT(39.88501690200489 74.66872171899945) bank126478 +126479 POINT(40.16469543585718 74.69014927355853) bank126479 +126480 POINT(41.15543846624905 74.69771010716971) bank126480 +126481 POINT(40.3468158518718 73.96644863858278) bank126481 +126482 POINT(39.8450038884103 73.29355675315495) bank126482 +126483 POINT(40.18097445706099 74.62722359342347) bank126483 +126484 POINT(40.19287541337503 74.09504763257691) bank126484 +126485 POINT(40.605789037248016 73.14530738481388) bank126485 +126486 POINT(41.4269480268389 74.49440862084376) bank126486 +126487 POINT(41.528409295242646 74.92872915717857) bank126487 +126488 POINT(40.906754854832236 73.73428165024961) bank126488 +126489 POINT(39.72040161122239 74.07538569770307) bank126489 +126490 POINT(41.40265007078851 74.96033738242681) bank126490 +126491 POINT(39.93598303579578 73.87519873927928) bank126491 +126492 POINT(39.96311707605637 74.04369918631895) bank126492 +126493 POINT(39.92271142238905 73.87998684357265) bank126493 +126494 POINT(41.37300602088152 73.10032208456359) bank126494 +126495 POINT(39.745935060667435 74.41125961978433) bank126495 +126496 POINT(41.123692546650965 73.7370342377828) bank126496 +126497 POINT(39.924298873963956 73.06607744281794) bank126497 +126498 POINT(39.73487785221001 74.18288120648178) bank126498 +126499 POINT(39.71997291451551 74.65443289308986) bank126499 +126500 POINT(41.56439954323491 73.27798150863933) bank126500 +126501 POINT(40.726976612430235 73.34369939079019) bank126501 +126502 POINT(41.68225363466013 74.89595830701865) bank126502 +126503 POINT(40.71345508244942 73.42533984039254) bank126503 +126504 POINT(41.05443195252606 73.76192565673857) bank126504 +126505 POINT(40.18834306910787 73.04303037917148) bank126505 +126506 POINT(41.57737559006768 73.95851758981637) bank126506 +126507 POINT(40.98733942323578 74.01495187224246) bank126507 +126508 POINT(39.88860074388096 73.01336005467009) bank126508 +126509 POINT(39.877699808043445 73.58764154366318) bank126509 +126510 POINT(40.071493064811555 74.64864710977459) bank126510 +126511 POINT(40.69169195416477 74.96261707536344) bank126511 +126512 POINT(39.754255403737254 73.80119255497898) bank126512 +126513 POINT(41.22282097707573 73.15496154541441) bank126513 +126514 POINT(40.97971529586468 74.85992765552297) bank126514 +126515 POINT(40.93875340010045 74.96928099018116) bank126515 +126516 POINT(40.97794848309907 73.16062517746813) bank126516 +126517 POINT(39.81898480747059 73.486842753085) bank126517 +126518 POINT(40.30020281330871 73.10895207199881) bank126518 +126519 POINT(40.950621649743354 73.26233291569064) bank126519 +126520 POINT(40.001253635871215 73.34216332596375) bank126520 +126521 POINT(40.52786473712446 74.31697283107333) bank126521 +126522 POINT(41.268551449871964 74.3100495602843) bank126522 +126523 POINT(40.28370447140773 73.61136683462784) bank126523 +126524 POINT(40.71366379244332 74.31104211472679) bank126524 +126525 POINT(40.3082616966592 74.09122948898055) bank126525 +126526 POINT(40.83808137426333 73.64459130512874) bank126526 +126527 POINT(39.78306813251342 74.11288019248168) bank126527 +126528 POINT(40.93860225493261 74.62886975024584) bank126528 +126529 POINT(41.04193041004258 73.1668608069499) bank126529 +126530 POINT(40.18193382341496 74.73784180806538) bank126530 +126531 POINT(41.542585940679494 73.99067794862512) bank126531 +126532 POINT(41.26971251074958 74.12711652614676) bank126532 +126533 POINT(40.69640408055167 73.84593028001636) bank126533 +126534 POINT(40.00141260835874 73.38452838624146) bank126534 +126535 POINT(40.46682145196096 74.03267844628881) bank126535 +126536 POINT(40.81874680302476 74.87270103413512) bank126536 +126537 POINT(41.574387840958046 74.70240083268297) bank126537 +126538 POINT(40.18307248408213 74.79040003176107) bank126538 +126539 POINT(40.176547802707525 73.61604894593286) bank126539 +126540 POINT(40.82742442382301 74.93886543665597) bank126540 +126541 POINT(39.87988321597915 73.82803502696427) bank126541 +126542 POINT(39.81528178039069 74.23326288399214) bank126542 +126543 POINT(40.14707019697829 74.21173928799706) bank126543 +126544 POINT(40.69178683651238 74.38882423158505) bank126544 +126545 POINT(40.420066476929534 74.46765504053401) bank126545 +126546 POINT(40.31770555389233 73.14803552498707) bank126546 +126547 POINT(40.80093155491033 74.13610091021802) bank126547 +126548 POINT(41.18794517283075 74.2578870330747) bank126548 +126549 POINT(41.44819043273976 74.94665952682851) bank126549 +126550 POINT(40.57640145903172 74.19458756497778) bank126550 +126551 POINT(40.12008647083543 74.23981357070386) bank126551 +126552 POINT(40.59905169930948 73.32471305893792) bank126552 +126553 POINT(41.525599650954106 74.22108172362448) bank126553 +126554 POINT(40.89012451458228 74.91288225848703) bank126554 +126555 POINT(41.36956211858772 74.36674142479127) bank126555 +126556 POINT(41.342470758348874 73.82766061851386) bank126556 +126557 POINT(40.72813657297523 73.2665719166612) bank126557 +126558 POINT(39.91211041503585 74.15058988633373) bank126558 +126559 POINT(40.936145211942545 74.22324743758438) bank126559 +126560 POINT(40.250790812017634 74.30784167324995) bank126560 +126561 POINT(40.503515499461045 73.88540761888423) bank126561 +126562 POINT(41.288303477082216 73.47271903529324) bank126562 +126563 POINT(40.036543028668206 74.63773172497329) bank126563 +126564 POINT(40.922386272866135 73.90949278338955) bank126564 +126565 POINT(41.22524084696463 73.83804328031152) bank126565 +126566 POINT(40.90343308475682 73.27228592657902) bank126566 +126567 POINT(40.475919552254695 74.26138208699898) bank126567 +126568 POINT(41.17614105271642 74.85188059213459) bank126568 +126569 POINT(41.001311906572425 73.81906006963789) bank126569 +126570 POINT(40.59762640941915 73.17252239919254) bank126570 +126571 POINT(40.42779492185495 73.85305163782616) bank126571 +126572 POINT(40.8743533006333 74.38756308014779) bank126572 +126573 POINT(40.508081808802075 73.35845910577395) bank126573 +126574 POINT(41.60738843072505 74.89645835682254) bank126574 +126575 POINT(40.9122534132911 74.11865485704472) bank126575 +126576 POINT(41.41820661174339 74.23415685430705) bank126576 +126577 POINT(40.19754059209766 74.7766012631872) bank126577 +126578 POINT(39.97771439536488 74.69326918634269) bank126578 +126579 POINT(40.85608791614373 74.10547975238897) bank126579 +126580 POINT(40.4042361053034 74.05472317190016) bank126580 +126581 POINT(40.87247650018252 73.16828357764221) bank126581 +126582 POINT(41.49545637818759 73.80189975958147) bank126582 +126583 POINT(40.96414878034909 73.03415829622256) bank126583 +126584 POINT(40.05402453045984 73.21708996677926) bank126584 +126585 POINT(39.88463351672352 73.16710110696192) bank126585 +126586 POINT(40.67145710389439 73.85489637764601) bank126586 +126587 POINT(41.40385072553587 74.9319849658116) bank126587 +126588 POINT(40.69203374455933 74.2793594390199) bank126588 +126589 POINT(40.906018471190755 73.22211495405789) bank126589 +126590 POINT(41.07526804050062 74.16984549441642) bank126590 +126591 POINT(41.16716116476923 74.96165021842519) bank126591 +126592 POINT(39.8511188242708 74.69426943575549) bank126592 +126593 POINT(40.82174734115626 74.05503587718296) bank126593 +126594 POINT(41.25636115478961 74.81181530800112) bank126594 +126595 POINT(41.37289642053403 73.8171902498562) bank126595 +126596 POINT(40.36462465893875 74.25700268303662) bank126596 +126597 POINT(40.757279176973825 73.1557847170954) bank126597 +126598 POINT(39.9508596931173 73.972798586744) bank126598 +126599 POINT(40.672939705976695 74.0070216001896) bank126599 +126600 POINT(40.62646127929691 74.14850705980055) bank126600 +126601 POINT(40.385891456858275 74.79834240776306) bank126601 +126602 POINT(40.722098170686664 73.71349539103522) bank126602 +126603 POINT(40.68818891288927 73.66998257479933) bank126603 +126604 POINT(41.112684120060734 73.61397135429038) bank126604 +126605 POINT(40.47396461934103 73.21928053543513) bank126605 +126606 POINT(40.87397438589952 73.04964333498826) bank126606 +126607 POINT(41.03377746488122 74.86879501120976) bank126607 +126608 POINT(40.52527392911899 74.88718174358753) bank126608 +126609 POINT(40.779709214041375 74.02757713533788) bank126609 +126610 POINT(41.637491593811106 73.46691031316357) bank126610 +126611 POINT(39.910885806871654 74.91126512266362) bank126611 +126612 POINT(40.89917183192577 74.73414959430943) bank126612 +126613 POINT(41.538968188836115 73.65957962985708) bank126613 +126614 POINT(40.974258420585635 74.29368898625604) bank126614 +126615 POINT(39.906974628975874 74.34554186218413) bank126615 +126616 POINT(40.41223959265282 74.26815768768152) bank126616 +126617 POINT(40.060241290531295 74.79268794531171) bank126617 +126618 POINT(40.233896897587904 74.2200783191833) bank126618 +126619 POINT(41.5355840630145 73.53313311018535) bank126619 +126620 POINT(40.87439929529752 73.19544115030553) bank126620 +126621 POINT(41.30132839221311 73.05907666101501) bank126621 +126622 POINT(41.45434015337263 74.73017018286505) bank126622 +126623 POINT(40.849226947649726 74.03507697609282) bank126623 +126624 POINT(41.60252589566361 74.75406633327795) bank126624 +126625 POINT(40.4651340655745 74.81854985512305) bank126625 +126626 POINT(41.59389893932316 74.32867782502986) bank126626 +126627 POINT(40.425893545270604 73.27702364694812) bank126627 +126628 POINT(41.60913447389359 73.88969553131363) bank126628 +126629 POINT(40.44611198013488 74.40609434298844) bank126629 +126630 POINT(41.68996561351265 73.40375120744898) bank126630 +126631 POINT(41.30398917497735 73.3810766548176) bank126631 +126632 POINT(40.157899998078975 73.24071370454315) bank126632 +126633 POINT(40.73942219927972 74.72038905660266) bank126633 +126634 POINT(40.22533270212268 73.48664086747787) bank126634 +126635 POINT(39.76208388796358 74.51190595548479) bank126635 +126636 POINT(40.10268055784488 74.8171553687701) bank126636 +126637 POINT(39.82351561652425 73.64152312395191) bank126637 +126638 POINT(40.04190213031514 74.1128816628022) bank126638 +126639 POINT(40.886939737708396 74.68084464944774) bank126639 +126640 POINT(40.8624072685024 74.16695673898765) bank126640 +126641 POINT(39.81263230783841 74.50422971885912) bank126641 +126642 POINT(41.24433047273774 73.93458963981065) bank126642 +126643 POINT(39.842809948042834 74.60642784607093) bank126643 +126644 POINT(40.02360285168801 74.28669596675645) bank126644 +126645 POINT(40.35707058628418 74.55788217129957) bank126645 +126646 POINT(40.53392495533928 74.52215942948372) bank126646 +126647 POINT(41.33092975585374 73.32579607705148) bank126647 +126648 POINT(40.74645674405833 74.65349351752575) bank126648 +126649 POINT(39.87675521444574 74.28360953528714) bank126649 +126650 POINT(41.61768812995882 74.53358705112464) bank126650 +126651 POINT(40.20223698811235 73.86122045538536) bank126651 +126652 POINT(40.93649109705772 74.28275243066086) bank126652 +126653 POINT(40.41008370171737 74.06532485429842) bank126653 +126654 POINT(41.60850902244392 74.54932077642314) bank126654 +126655 POINT(39.85586755916513 73.42604087995721) bank126655 +126656 POINT(40.035746626032655 74.87009109242854) bank126656 +126657 POINT(40.807844354895494 73.40594792523859) bank126657 +126658 POINT(41.486653990152035 74.28791895479851) bank126658 +126659 POINT(40.056669495715745 73.66287615690194) bank126659 +126660 POINT(40.509395622016555 73.63530719669326) bank126660 +126661 POINT(39.892765106772515 74.1168233030207) bank126661 +126662 POINT(41.62821891657383 73.48136213895074) bank126662 +126663 POINT(40.19630992239879 73.9129103808844) bank126663 +126664 POINT(40.73422230243105 74.13606254053455) bank126664 +126665 POINT(41.31288481119893 74.63606918652775) bank126665 +126666 POINT(40.05443869017679 73.29509041402403) bank126666 +126667 POINT(41.20464405258154 74.06180249196919) bank126667 +126668 POINT(40.676700618719046 73.21415577289001) bank126668 +126669 POINT(41.27926037842631 74.15660131907704) bank126669 +126670 POINT(41.46385247800317 74.95776248038618) bank126670 +126671 POINT(40.42196187952921 73.4257800322064) bank126671 +126672 POINT(40.110172264463586 74.13937883323993) bank126672 +126673 POINT(40.14173844703564 74.79920631424629) bank126673 +126674 POINT(41.5331855698649 73.58746952211374) bank126674 +126675 POINT(40.423961279232145 74.76787654987723) bank126675 +126676 POINT(39.747676244292585 74.22525875385784) bank126676 +126677 POINT(39.896557704773215 74.06520975048716) bank126677 +126678 POINT(40.156000123941325 74.00863435277724) bank126678 +126679 POINT(39.881394469413166 74.43583575106827) bank126679 +126680 POINT(40.981709479540584 74.88277614645939) bank126680 +126681 POINT(41.571372075449084 74.09170438783981) bank126681 +126682 POINT(41.32495114420879 74.08767873098769) bank126682 +126683 POINT(40.21154915737265 73.15504749046025) bank126683 +126684 POINT(41.21164676908653 73.52251977196337) bank126684 +126685 POINT(41.00612268484469 73.52386690818234) bank126685 +126686 POINT(40.47315038584231 74.18100102988944) bank126686 +126687 POINT(41.11793207317841 74.8400628999717) bank126687 +126688 POINT(39.994278322087624 73.94309331127918) bank126688 +126689 POINT(39.78748695412599 74.98459768252336) bank126689 +126690 POINT(40.06988080846192 74.30794483972115) bank126690 +126691 POINT(41.30587037545799 74.86929268778754) bank126691 +126692 POINT(41.288966354178754 74.68476337510425) bank126692 +126693 POINT(40.978625726695455 73.06094468872982) bank126693 +126694 POINT(41.13805064428398 73.33334481000988) bank126694 +126695 POINT(40.88353462599426 74.23612381804986) bank126695 +126696 POINT(40.84569654170375 73.63606245272275) bank126696 +126697 POINT(40.27084814368607 74.82420481575089) bank126697 +126698 POINT(40.14787831070649 74.437556986961) bank126698 +126699 POINT(40.83139301098977 74.81440526017573) bank126699 +126700 POINT(41.50637844584482 74.46529936634157) bank126700 +126701 POINT(41.68816580339974 73.46226591515786) bank126701 +126702 POINT(39.877169697202994 73.35632350459073) bank126702 +126703 POINT(40.427911550303335 74.50545868505945) bank126703 +126704 POINT(41.649624279734645 73.7649505466417) bank126704 +126705 POINT(41.077637884096895 74.50243357534843) bank126705 +126706 POINT(40.04884287409826 74.61069757363944) bank126706 +126707 POINT(41.60052573017143 73.21396169618471) bank126707 +126708 POINT(41.56289732151861 73.04522898349616) bank126708 +126709 POINT(40.68164618482265 73.03412799690288) bank126709 +126710 POINT(41.117719075067185 73.35302104920805) bank126710 +126711 POINT(41.505947697221586 74.38135682952151) bank126711 +126712 POINT(41.310508841392185 73.26996225714666) bank126712 +126713 POINT(41.257282150105716 73.94186819133093) bank126713 +126714 POINT(39.81809689137612 73.75505210236625) bank126714 +126715 POINT(40.05768253755964 74.29884227445966) bank126715 +126716 POINT(41.65817703541824 73.01697577475124) bank126716 +126717 POINT(40.5688128541548 73.99941818074898) bank126717 +126718 POINT(41.36271468251456 73.48332461542608) bank126718 +126719 POINT(40.832823816221804 74.60581363166474) bank126719 +126720 POINT(40.65563673580148 73.57926022513884) bank126720 +126721 POINT(40.83436919306354 74.9541577252952) bank126721 +126722 POINT(39.76324657125312 73.32248983172887) bank126722 +126723 POINT(41.53470497487359 74.6097551125948) bank126723 +126724 POINT(41.33505050408596 74.83129255916097) bank126724 +126725 POINT(41.285974967983 73.86549398636159) bank126725 +126726 POINT(39.92765624094306 74.47926631925266) bank126726 +126727 POINT(40.89025127457196 74.771849247364) bank126727 +126728 POINT(41.21573011863196 74.58946460426966) bank126728 +126729 POINT(41.626099190134646 73.69012661991083) bank126729 +126730 POINT(40.63057731420015 73.59275507636036) bank126730 +126731 POINT(40.60458097512486 73.67167646867827) bank126731 +126732 POINT(40.1985330315744 74.60005742823184) bank126732 +126733 POINT(40.55253672680451 73.12892593271546) bank126733 +126734 POINT(40.022642636211394 74.31453509393397) bank126734 +126735 POINT(41.1816922788409 74.39519458054984) bank126735 +126736 POINT(40.71879104132033 73.94242346244846) bank126736 +126737 POINT(41.1708065484856 73.49384403849623) bank126737 +126738 POINT(39.86508561523518 74.3990986385303) bank126738 +126739 POINT(41.463202663095906 73.36964178184991) bank126739 +126740 POINT(40.28473046831405 74.4586475703678) bank126740 +126741 POINT(40.718925602270794 74.17012513788103) bank126741 +126742 POINT(40.748474691319245 74.08510302190236) bank126742 +126743 POINT(41.2975768075361 73.63929548688691) bank126743 +126744 POINT(41.029223884710426 74.33282519019245) bank126744 +126745 POINT(41.25934479217556 73.76930109810063) bank126745 +126746 POINT(41.414941066590266 73.91711891143984) bank126746 +126747 POINT(40.4866040799645 73.23899978893155) bank126747 +126748 POINT(40.47694880826248 73.1484130244922) bank126748 +126749 POINT(39.889881585453004 73.76596833194583) bank126749 +126750 POINT(40.516293254626426 74.72519651878581) bank126750 +126751 POINT(41.674524451446594 74.07657929195403) bank126751 +126752 POINT(40.495758987172984 73.44755557605492) bank126752 +126753 POINT(40.19939495864277 74.33106704642394) bank126753 +126754 POINT(40.05617156904286 74.80427090998313) bank126754 +126755 POINT(39.974084285822414 74.66629899007076) bank126755 +126756 POINT(40.868208710493334 73.60352169808911) bank126756 +126757 POINT(39.723264904965426 73.49026866723969) bank126757 +126758 POINT(40.761407448159616 73.56851094516851) bank126758 +126759 POINT(41.57056040914133 73.49947481982815) bank126759 +126760 POINT(39.81573528495079 73.07955884817946) bank126760 +126761 POINT(39.80392924579998 73.25179403097097) bank126761 +126762 POINT(41.57110086549559 73.81531874973992) bank126762 +126763 POINT(40.68980571213149 74.06088262045793) bank126763 +126764 POINT(41.64835907860356 73.91473506863464) bank126764 +126765 POINT(40.033062768120246 73.92757089976804) bank126765 +126766 POINT(40.15068770051907 73.48654424655041) bank126766 +126767 POINT(39.95088688049707 73.8319692409949) bank126767 +126768 POINT(41.29575036194538 74.37787804645302) bank126768 +126769 POINT(41.24437798363486 74.4771818639921) bank126769 +126770 POINT(41.10283564765186 74.20773460978658) bank126770 +126771 POINT(40.19292803108091 74.45581635262032) bank126771 +126772 POINT(41.26177901661636 74.20241093383957) bank126772 +126773 POINT(40.244624429333435 74.51280249157064) bank126773 +126774 POINT(40.645891942804916 74.79255789372469) bank126774 +126775 POINT(41.37765284213698 74.17328857724381) bank126775 +126776 POINT(41.019002598646 74.6941463560505) bank126776 +126777 POINT(40.270799984185736 73.13189080533097) bank126777 +126778 POINT(40.62059416982355 73.42155260146609) bank126778 +126779 POINT(39.95206265922495 73.03082182555012) bank126779 +126780 POINT(41.25784682763912 73.5423464169722) bank126780 +126781 POINT(41.516306163682366 74.60517864893158) bank126781 +126782 POINT(41.08748982964819 73.09967560920921) bank126782 +126783 POINT(40.04272000543298 74.81948143338735) bank126783 +126784 POINT(40.33186335684751 73.71437752818761) bank126784 +126785 POINT(39.790071830691836 73.1393553407619) bank126785 +126786 POINT(40.56009387099044 73.12141997223898) bank126786 +126787 POINT(40.992961774886865 73.86084489686581) bank126787 +126788 POINT(39.85767628634414 73.89697619875479) bank126788 +126789 POINT(40.114004490625064 74.80858006900372) bank126789 +126790 POINT(40.295764471930404 73.87819041943094) bank126790 +126791 POINT(40.19466820674187 74.38444717672903) bank126791 +126792 POINT(40.726019463678895 74.51255183462305) bank126792 +126793 POINT(40.52105086127587 74.43491685109174) bank126793 +126794 POINT(39.776009388486244 73.87112777554385) bank126794 +126795 POINT(39.956625063421264 73.0378097885349) bank126795 +126796 POINT(41.24801852765478 73.37136493418994) bank126796 +126797 POINT(41.278623090901505 74.33980119286642) bank126797 +126798 POINT(40.21874563367706 74.61843329999365) bank126798 +126799 POINT(40.929962794944096 73.28117759395417) bank126799 +126800 POINT(40.77056927800554 73.24819965274679) bank126800 +126801 POINT(41.62363289065192 73.20776488779997) bank126801 +126802 POINT(41.380780786831686 74.46539345153104) bank126802 +126803 POINT(40.61630138707631 74.53705902925594) bank126803 +126804 POINT(41.32659740661365 73.10704704621634) bank126804 +126805 POINT(40.61256658433494 74.12584008949442) bank126805 +126806 POINT(40.46902980196948 73.42205744137614) bank126806 +126807 POINT(40.05990947969132 74.34878269665434) bank126807 +126808 POINT(40.55363798805835 73.40030316734185) bank126808 +126809 POINT(40.158109669302846 74.72760628585958) bank126809 +126810 POINT(40.03299586415205 74.33365811212221) bank126810 +126811 POINT(40.5891579737402 74.67007066462415) bank126811 +126812 POINT(39.8122081378478 73.76569773688743) bank126812 +126813 POINT(40.09898831156735 74.72052988065252) bank126813 +126814 POINT(41.37144115072565 74.89017260183097) bank126814 +126815 POINT(41.50700191688602 73.02534610348573) bank126815 +126816 POINT(40.36610457229919 74.09251217541451) bank126816 +126817 POINT(40.1430813441787 74.98947752188857) bank126817 +126818 POINT(41.62648495534858 74.68032631013938) bank126818 +126819 POINT(41.43472115341183 73.41966694737005) bank126819 +126820 POINT(41.4781254953646 73.9703819282864) bank126820 +126821 POINT(40.68820690725602 73.0571682827276) bank126821 +126822 POINT(40.43813385326232 74.604330335475) bank126822 +126823 POINT(40.191300089335066 73.75025521639841) bank126823 +126824 POINT(40.72650023164294 74.53179813390429) bank126824 +126825 POINT(40.17659130839856 73.92201842289253) bank126825 +126826 POINT(40.03939070987626 74.42368163317788) bank126826 +126827 POINT(41.55208881934085 73.89738996824806) bank126827 +126828 POINT(40.922987228062325 73.99647122651426) bank126828 +126829 POINT(41.47267433262039 73.47783743542408) bank126829 +126830 POINT(39.80346365808244 73.2951207085819) bank126830 +126831 POINT(39.75105296225973 74.4039110169181) bank126831 +126832 POINT(40.17610970202571 74.10179873340158) bank126832 +126833 POINT(40.67239433838469 74.36585932057135) bank126833 +126834 POINT(40.64163602815368 74.56424320850493) bank126834 +126835 POINT(41.59890222556227 73.2482822668214) bank126835 +126836 POINT(39.99621123570366 73.9714498234155) bank126836 +126837 POINT(39.76217117843725 74.91972321988916) bank126837 +126838 POINT(40.05069320272899 74.48929764220154) bank126838 +126839 POINT(41.56848608159374 74.22672954252282) bank126839 +126840 POINT(41.47261647579402 73.31564435106364) bank126840 +126841 POINT(41.705737925134486 73.85977531854174) bank126841 +126842 POINT(41.1109339887885 74.57344854341235) bank126842 +126843 POINT(39.88292796004139 73.36476485104563) bank126843 +126844 POINT(39.99707024213315 74.46708814689559) bank126844 +126845 POINT(40.220630683363524 73.9453662661784) bank126845 +126846 POINT(40.72299794642061 74.0774460979801) bank126846 +126847 POINT(40.16783559044731 73.08588094495873) bank126847 +126848 POINT(41.09495908010806 74.41822506463575) bank126848 +126849 POINT(40.37644120408106 74.87546822782738) bank126849 +126850 POINT(40.89752610872979 73.46032406682137) bank126850 +126851 POINT(41.656833124257325 73.58244711966913) bank126851 +126852 POINT(40.9518166881418 74.641936235491) bank126852 +126853 POINT(40.371097604306755 74.18001050496271) bank126853 +126854 POINT(40.61236044942966 73.5282858265291) bank126854 +126855 POINT(40.921684696253664 74.8917792858893) bank126855 +126856 POINT(40.25991786185616 74.04413328188804) bank126856 +126857 POINT(39.850408619122284 73.61842094704137) bank126857 +126858 POINT(39.927263046767045 74.98695291470806) bank126858 +126859 POINT(41.105298441812614 73.0486824584986) bank126859 +126860 POINT(40.39579557686643 74.81921675905876) bank126860 +126861 POINT(41.166975070232645 73.60573615604002) bank126861 +126862 POINT(41.37259859546707 73.94206989980069) bank126862 +126863 POINT(40.22080474974309 73.71656997597817) bank126863 +126864 POINT(41.38601370974847 73.08027384132929) bank126864 +126865 POINT(41.186754002583115 74.31815717370823) bank126865 +126866 POINT(40.71480646008695 74.92178410516559) bank126866 +126867 POINT(41.34717640867979 73.20083948261008) bank126867 +126868 POINT(41.37470309269106 74.8677965747864) bank126868 +126869 POINT(40.876448826716704 74.90842705381003) bank126869 +126870 POINT(40.68601802818636 73.24545883906471) bank126870 +126871 POINT(39.73544996034218 74.93257978170037) bank126871 +126872 POINT(40.563047942068906 73.98892357604026) bank126872 +126873 POINT(40.811227383725 74.05051826769693) bank126873 +126874 POINT(41.356637864588 73.76292203055603) bank126874 +126875 POINT(40.52261621393746 74.22731131674652) bank126875 +126876 POINT(40.74428671798179 74.25236888326184) bank126876 +126877 POINT(40.36102678195517 73.15447293411098) bank126877 +126878 POINT(40.192834262270985 73.9894811939012) bank126878 +126879 POINT(41.09958566488278 74.03575125956425) bank126879 +126880 POINT(39.762904820779596 73.79208782782162) bank126880 +126881 POINT(40.38036463589602 73.6822849586297) bank126881 +126882 POINT(40.09405460115064 73.7520596566414) bank126882 +126883 POINT(40.22576413583903 74.8849369114333) bank126883 +126884 POINT(39.95536909732888 73.85275469773752) bank126884 +126885 POINT(40.62360154770821 73.36541837461924) bank126885 +126886 POINT(41.35057319165104 73.97584728612225) bank126886 +126887 POINT(40.15459552645344 74.90470544241747) bank126887 +126888 POINT(40.29459648025637 74.99912794804276) bank126888 +126889 POINT(40.570581132880015 74.15375899040072) bank126889 +126890 POINT(41.031980866732496 73.76006187435885) bank126890 +126891 POINT(39.85706785893426 74.94949910627646) bank126891 +126892 POINT(40.984242142459514 74.70534369153921) bank126892 +126893 POINT(40.13004151039565 74.642506662297) bank126893 +126894 POINT(40.9770367061145 74.59675234271296) bank126894 +126895 POINT(40.59410051666557 74.11874781694793) bank126895 +126896 POINT(40.77076894291177 74.5966006153965) bank126896 +126897 POINT(40.999506975177425 74.20448647480585) bank126897 +126898 POINT(41.26485564127655 74.85174635438122) bank126898 +126899 POINT(39.760930594758165 74.81414972578585) bank126899 +126900 POINT(40.746437259433 74.61178269875668) bank126900 +126901 POINT(40.20792990708232 73.11248502518636) bank126901 +126902 POINT(40.37703460750165 73.74024857538959) bank126902 +126903 POINT(41.4973722190892 73.98345822389658) bank126903 +126904 POINT(40.4621660756969 74.43563455913795) bank126904 +126905 POINT(41.16337170986788 73.22479019085975) bank126905 +126906 POINT(40.15416041664802 73.40823204326921) bank126906 +126907 POINT(41.40468655616745 74.08762514621529) bank126907 +126908 POINT(41.44530669427301 73.06368473256553) bank126908 +126909 POINT(40.486224443866064 73.41941426374252) bank126909 +126910 POINT(40.66879694737442 74.42444515090216) bank126910 +126911 POINT(40.78492717660402 73.56742851095355) bank126911 +126912 POINT(39.85504902633052 73.31434138439631) bank126912 +126913 POINT(40.2470255287538 73.22663406754572) bank126913 +126914 POINT(41.206355128213815 73.29238160066956) bank126914 +126915 POINT(39.96584237832337 73.71994637219173) bank126915 +126916 POINT(40.3688546637512 73.09727846339362) bank126916 +126917 POINT(40.93485683933484 74.3170108615231) bank126917 +126918 POINT(40.911413504292724 74.17457454958762) bank126918 +126919 POINT(41.320630997778004 74.165119502225) bank126919 +126920 POINT(40.958185542342065 74.31867498584884) bank126920 +126921 POINT(41.22414608190565 74.12759709814686) bank126921 +126922 POINT(39.894640135187004 74.00389696086611) bank126922 +126923 POINT(40.65438990160014 74.16952168002283) bank126923 +126924 POINT(41.59879154447081 73.02346659012987) bank126924 +126925 POINT(41.39168091840373 74.12233426796088) bank126925 +126926 POINT(39.77791362873511 74.39212504968432) bank126926 +126927 POINT(40.09649997042724 74.00742005559962) bank126927 +126928 POINT(40.779553213491944 74.40810871637392) bank126928 +126929 POINT(40.3801194610699 73.7575986493579) bank126929 +126930 POINT(40.8955838087474 74.741106997132) bank126930 +126931 POINT(40.50342357771286 74.7901459738277) bank126931 +126932 POINT(41.171234046160755 73.0108422056549) bank126932 +126933 POINT(40.255565627869906 74.77024494608372) bank126933 +126934 POINT(40.946137626492 73.93802864941587) bank126934 +126935 POINT(40.65881555313892 73.5991275395804) bank126935 +126936 POINT(40.722022583924044 73.67068039461482) bank126936 +126937 POINT(39.84886892632853 74.69207480894491) bank126937 +126938 POINT(40.28905043597548 73.81115063814349) bank126938 +126939 POINT(40.93388478391869 73.22538812326044) bank126939 +126940 POINT(40.54673938302024 74.24153028532157) bank126940 +126941 POINT(39.76174266600108 73.40557139599788) bank126941 +126942 POINT(40.21973921511861 73.38140389638099) bank126942 +126943 POINT(39.869439413721835 74.64340242228307) bank126943 +126944 POINT(41.12511609676155 73.08870085137349) bank126944 +126945 POINT(41.167625216972375 73.62522034887617) bank126945 +126946 POINT(40.33358478120569 74.87974536893643) bank126946 +126947 POINT(40.961896614937594 74.70138880144543) bank126947 +126948 POINT(41.03392618407547 74.95302089030494) bank126948 +126949 POINT(41.128749824610075 73.59784402385006) bank126949 +126950 POINT(41.18247209027783 74.89055177495646) bank126950 +126951 POINT(40.7271093020974 74.38020480997052) bank126951 +126952 POINT(41.60465317547468 74.19456102043192) bank126952 +126953 POINT(39.75499101688643 73.44173096319297) bank126953 +126954 POINT(41.66414050023936 73.38997924744588) bank126954 +126955 POINT(41.68265990518053 73.91552030151281) bank126955 +126956 POINT(40.97745631210463 73.85913547203471) bank126956 +126957 POINT(40.002824888808355 74.69474934249224) bank126957 +126958 POINT(41.206248130079 73.19417553367278) bank126958 +126959 POINT(40.03245560930322 73.44401066846945) bank126959 +126960 POINT(40.66543026209053 73.38502414987644) bank126960 +126961 POINT(41.64885926259879 74.77825062970163) bank126961 +126962 POINT(40.675424098205355 74.68289971931745) bank126962 +126963 POINT(40.82865393538186 73.68986261130533) bank126963 +126964 POINT(40.70196510119446 73.17714623239166) bank126964 +126965 POINT(40.218229109218235 74.0714444156967) bank126965 +126966 POINT(41.50208031896018 74.67398259570817) bank126966 +126967 POINT(41.27109899579541 73.54267543149341) bank126967 +126968 POINT(40.779039770590735 74.13068629507805) bank126968 +126969 POINT(40.53041197526607 74.51056712824821) bank126969 +126970 POINT(40.527415472633535 74.7096785389672) bank126970 +126971 POINT(39.884603144454545 74.05963821182605) bank126971 +126972 POINT(40.109107614287225 73.73260299553425) bank126972 +126973 POINT(41.12660049017393 74.55144141442267) bank126973 +126974 POINT(40.7931636472415 73.03241972329074) bank126974 +126975 POINT(40.634150736466445 73.59293874872029) bank126975 +126976 POINT(41.576285239958 74.88414680318124) bank126976 +126977 POINT(40.85073511681421 74.52337010887628) bank126977 +126978 POINT(40.17037958176495 74.44260613083529) bank126978 +126979 POINT(41.21198116252778 73.51066960592689) bank126979 +126980 POINT(41.054675107084016 73.55838019014578) bank126980 +126981 POINT(39.98842431488047 74.52003642029574) bank126981 +126982 POINT(41.351726525468706 74.85181116136458) bank126982 +126983 POINT(40.688837591905354 74.70167330033638) bank126983 +126984 POINT(40.920069714545676 74.59329605114763) bank126984 +126985 POINT(40.642691407261495 74.29111114990414) bank126985 +126986 POINT(41.408941787718696 74.23446597607733) bank126986 +126987 POINT(40.837656655330186 73.32397962737635) bank126987 +126988 POINT(41.23729950099547 73.29826285285519) bank126988 +126989 POINT(41.412665666956556 73.91098390856135) bank126989 +126990 POINT(40.18465294756414 74.85893864649584) bank126990 +126991 POINT(39.83040147118746 73.51340502841146) bank126991 +126992 POINT(39.86561465675446 73.79404532629674) bank126992 +126993 POINT(40.724730481469706 73.09341460720508) bank126993 +126994 POINT(39.817386027831525 74.35158084947935) bank126994 +126995 POINT(41.17959813696764 74.04584084940596) bank126995 +126996 POINT(40.944218029242556 73.83023243299989) bank126996 +126997 POINT(39.929259879284196 73.69926857606471) bank126997 +126998 POINT(41.35610371805349 73.16272182687825) bank126998 +126999 POINT(41.231696048194564 74.20261250149842) bank126999 +127000 POINT(40.537867538268834 73.15443352439206) bank127000 +127001 POINT(40.67415066557806 73.71405356342963) bank127001 +127002 POINT(41.36008832865179 73.66119441835227) bank127002 +127003 POINT(40.518218502449294 73.5000301544775) bank127003 +127004 POINT(40.196952598685584 74.21873148043052) bank127004 +127005 POINT(41.35233450873671 73.52367110686154) bank127005 +127006 POINT(39.91850212973912 74.1025099691444) bank127006 +127007 POINT(41.10028773432865 73.2309172277007) bank127007 +127008 POINT(40.31685883155922 74.46291039072156) bank127008 +127009 POINT(39.90572456103492 74.50696849643421) bank127009 +127010 POINT(41.01218051283846 73.71828756204947) bank127010 +127011 POINT(40.85157753216804 74.07927195102367) bank127011 +127012 POINT(40.040522861168206 74.09992331253072) bank127012 +127013 POINT(41.173693971254416 74.27274309477326) bank127013 +127014 POINT(41.497150600362126 73.76254736263019) bank127014 +127015 POINT(41.14176653963707 73.75414783851318) bank127015 +127016 POINT(40.14632053234003 74.1935633830014) bank127016 +127017 POINT(41.52731265341165 74.26227149720276) bank127017 +127018 POINT(40.45426511534209 73.34264865182605) bank127018 +127019 POINT(40.54198915193082 74.39148683083542) bank127019 +127020 POINT(41.291999540654814 74.7149510187498) bank127020 +127021 POINT(41.53099217943145 73.65464252526344) bank127021 +127022 POINT(41.562562472427864 74.79128660012165) bank127022 +127023 POINT(40.91044958609058 74.11021810337678) bank127023 +127024 POINT(40.06370957565159 73.2687070531256) bank127024 +127025 POINT(40.46774690659211 73.37050075386193) bank127025 +127026 POINT(41.46194295901789 74.97186367231407) bank127026 +127027 POINT(40.30362951040711 74.63400892249349) bank127027 +127028 POINT(40.57098559362264 73.3264499348512) bank127028 +127029 POINT(40.921584008393125 74.41571178294453) bank127029 +127030 POINT(39.816144947964766 73.6214496386626) bank127030 +127031 POINT(41.272434542621475 73.00714186064208) bank127031 +127032 POINT(41.54683737899068 74.1823692193851) bank127032 +127033 POINT(40.57295850758687 73.51673885283583) bank127033 +127034 POINT(41.56426470315108 73.12490161820423) bank127034 +127035 POINT(41.26180402377203 74.03854706076089) bank127035 +127036 POINT(39.86776402231084 74.66963717485356) bank127036 +127037 POINT(40.772942266662334 73.78477962741208) bank127037 +127038 POINT(40.01889580308327 73.53240893991013) bank127038 +127039 POINT(41.09508702963254 74.1114013993927) bank127039 +127040 POINT(40.060992455080424 74.4871788674329) bank127040 +127041 POINT(41.22471957360244 74.2462316564074) bank127041 +127042 POINT(40.00474108112548 73.721043071438) bank127042 +127043 POINT(41.227048202341976 73.2851236368276) bank127043 +127044 POINT(41.34185661144553 74.70809604736648) bank127044 +127045 POINT(39.73282163891146 73.61978205309185) bank127045 +127046 POINT(41.40287178879827 73.43771677041642) bank127046 +127047 POINT(39.96674848818027 73.03843931323222) bank127047 +127048 POINT(41.36395265451519 74.68275342293768) bank127048 +127049 POINT(40.01289960742973 73.77883588829157) bank127049 +127050 POINT(39.91779124481465 74.88574026544474) bank127050 +127051 POINT(39.999170219829374 74.33622701647457) bank127051 +127052 POINT(40.40051862657004 73.2183958505345) bank127052 +127053 POINT(40.48835577326866 74.07622321002759) bank127053 +127054 POINT(41.544025490005126 74.73725562770342) bank127054 +127055 POINT(40.84278696716704 73.1493411084076) bank127055 +127056 POINT(41.45055036286826 73.37662975138056) bank127056 +127057 POINT(40.27786344019213 73.41138279755909) bank127057 +127058 POINT(41.07343232585267 73.2916983479963) bank127058 +127059 POINT(40.88063977408637 73.10094121521396) bank127059 +127060 POINT(39.72486648196182 73.81723059791067) bank127060 +127061 POINT(40.01787716927567 73.78918769409252) bank127061 +127062 POINT(40.392691450253096 74.00263102133013) bank127062 +127063 POINT(41.65151624163824 73.90527237425279) bank127063 +127064 POINT(40.143614373111035 74.12792798082354) bank127064 +127065 POINT(40.661977756466676 73.89659319826143) bank127065 +127066 POINT(41.13294862282392 73.73851098871343) bank127066 +127067 POINT(40.76494718331613 73.76333010204439) bank127067 +127068 POINT(40.41088224789766 74.50137466337354) bank127068 +127069 POINT(41.258545756507125 73.55713262765612) bank127069 +127070 POINT(40.766670353600375 74.33689087148) bank127070 +127071 POINT(40.9128956978543 73.47403363439452) bank127071 +127072 POINT(40.07186143256564 74.16576018349298) bank127072 +127073 POINT(40.38286423117733 73.24305206207936) bank127073 +127074 POINT(40.383275857772425 73.63596251636197) bank127074 +127075 POINT(40.31367470690638 73.06232959732844) bank127075 +127076 POINT(40.875525659368996 73.35282917085851) bank127076 +127077 POINT(41.294174205261804 73.72029050335152) bank127077 +127078 POINT(40.332301901347684 74.02903136519849) bank127078 +127079 POINT(39.996212079179095 73.93481330818797) bank127079 +127080 POINT(40.59646618734585 74.72719135656332) bank127080 +127081 POINT(40.71158420885009 74.85583686476112) bank127081 +127082 POINT(40.473008505179024 73.86891912034031) bank127082 +127083 POINT(40.94714024630413 74.2183739309645) bank127083 +127084 POINT(40.14252964318876 73.63763196102013) bank127084 +127085 POINT(40.14121534269139 74.45669579254778) bank127085 +127086 POINT(40.14955072030986 74.75874996546594) bank127086 +127087 POINT(40.6090252807165 73.37173394637423) bank127087 +127088 POINT(39.78462902994614 74.17306258982177) bank127088 +127089 POINT(40.919159124991594 74.00099500662417) bank127089 +127090 POINT(40.93861179405487 74.58277566525469) bank127090 +127091 POINT(41.01334401095911 73.28374759601385) bank127091 +127092 POINT(40.84760578528593 73.28851753638014) bank127092 +127093 POINT(40.89712038099903 74.35824547392465) bank127093 +127094 POINT(40.70718616643639 74.34437561470038) bank127094 +127095 POINT(39.7796204524785 74.05204616035127) bank127095 +127096 POINT(40.55688545411952 74.43932665077836) bank127096 +127097 POINT(39.78647234004042 74.49616067193244) bank127097 +127098 POINT(40.17293373210562 73.81614156795604) bank127098 +127099 POINT(41.50415097470752 74.05124720751368) bank127099 +127100 POINT(40.43155135664093 74.24061036116558) bank127100 +127101 POINT(39.90003813386705 74.042177081474) bank127101 +127102 POINT(41.47045307714209 74.85581139872556) bank127102 +127103 POINT(41.55458248990085 74.3164185619453) bank127103 +127104 POINT(40.20443365333573 74.8223002914349) bank127104 +127105 POINT(41.03265110836476 73.80114139177408) bank127105 +127106 POINT(41.435055915476156 74.46556424700509) bank127106 +127107 POINT(40.448661107094644 73.23399251372359) bank127107 +127108 POINT(40.79036539008589 73.6357464237526) bank127108 +127109 POINT(39.84503497679867 74.48329908103831) bank127109 +127110 POINT(39.742980447162076 74.72051549271299) bank127110 +127111 POINT(40.306621051601795 73.09142225924823) bank127111 +127112 POINT(40.14099687851788 73.52734284374431) bank127112 +127113 POINT(41.56346573214736 73.59675483207363) bank127113 +127114 POINT(41.049583030274846 73.80812901155655) bank127114 +127115 POINT(40.78738333236516 73.77435170557237) bank127115 +127116 POINT(40.774066479858305 73.15579047273971) bank127116 +127117 POINT(39.99132898249154 74.90595938274579) bank127117 +127118 POINT(40.73001504399996 74.40854513125191) bank127118 +127119 POINT(41.466305981110764 73.67973173162466) bank127119 +127120 POINT(41.704479148080296 73.13607266220944) bank127120 +127121 POINT(40.50265247345374 73.99555621211782) bank127121 +127122 POINT(41.134254524616935 74.81152494078336) bank127122 +127123 POINT(39.714671359200054 73.29193340205957) bank127123 +127124 POINT(41.2317403672149 74.26648657513142) bank127124 +127125 POINT(40.66207932488739 74.26582384456512) bank127125 +127126 POINT(41.121485850671014 73.84392616624214) bank127126 +127127 POINT(40.91565200744377 74.85454366615468) bank127127 +127128 POINT(41.07844827913092 73.06522019605498) bank127128 +127129 POINT(41.06334038869654 73.53086944321339) bank127129 +127130 POINT(39.82842129787028 74.41673196397932) bank127130 +127131 POINT(39.75468491581239 74.50481434675879) bank127131 +127132 POINT(40.69848476804012 74.49129044575372) bank127132 +127133 POINT(40.00929983695937 74.6056113184976) bank127133 +127134 POINT(39.93420101134995 74.57150729814931) bank127134 +127135 POINT(41.54152397360589 73.97970276224173) bank127135 +127136 POINT(39.87125419276841 74.33699129190117) bank127136 +127137 POINT(40.492872536829225 73.25076550281591) bank127137 +127138 POINT(40.280373895029456 73.78388935792171) bank127138 +127139 POINT(41.13671552805952 73.16001837366086) bank127139 +127140 POINT(39.773726997931284 73.65196230449624) bank127140 +127141 POINT(40.037048633518204 74.44810900704687) bank127141 +127142 POINT(40.4519338428139 74.73235542275457) bank127142 +127143 POINT(40.4499212693488 74.44472822837281) bank127143 +127144 POINT(41.05562605703258 73.33124259773271) bank127144 +127145 POINT(39.84415104674673 74.98131778110309) bank127145 +127146 POINT(39.79702383788055 73.63004201270283) bank127146 +127147 POINT(41.39812145318595 73.62142064484784) bank127147 +127148 POINT(39.955102805248835 73.37396663411843) bank127148 +127149 POINT(41.47054767207115 74.71287097385016) bank127149 +127150 POINT(41.25816948374785 73.36328030327002) bank127150 +127151 POINT(40.3837818897457 73.66667631905452) bank127151 +127152 POINT(40.775570925899764 74.63476034926943) bank127152 +127153 POINT(40.0063505072311 74.88278155775556) bank127153 +127154 POINT(39.83466207322634 73.38824795593524) bank127154 +127155 POINT(40.21228292912866 74.97753789639837) bank127155 +127156 POINT(40.8061333716983 73.32144114293858) bank127156 +127157 POINT(40.839078772694286 73.13982785580409) bank127157 +127158 POINT(40.33681352308993 74.5257177187061) bank127158 +127159 POINT(40.87595292917594 73.66666942305366) bank127159 +127160 POINT(41.15140444112568 74.4815024776859) bank127160 +127161 POINT(40.17505909828339 73.82266050597325) bank127161 +127162 POINT(40.478803918798164 74.52144333806882) bank127162 +127163 POINT(39.72692391609702 74.28075334657558) bank127163 +127164 POINT(40.63252883494804 74.47158934478739) bank127164 +127165 POINT(41.38913284271452 73.96776462370663) bank127165 +127166 POINT(40.05869510247872 73.4330554301732) bank127166 +127167 POINT(41.316364689042075 74.77743004162097) bank127167 +127168 POINT(39.87321774165842 74.42025769369184) bank127168 +127169 POINT(39.8826902423219 74.3925337285851) bank127169 +127170 POINT(40.29495794296143 74.1786106220371) bank127170 +127171 POINT(41.40317739063055 73.12221429043686) bank127171 +127172 POINT(39.801235118301285 74.76413365933206) bank127172 +127173 POINT(40.04978529008921 73.43800575174514) bank127173 +127174 POINT(40.02607761124049 73.05114480488947) bank127174 +127175 POINT(39.90467174061588 74.15077494607196) bank127175 +127176 POINT(40.66386334580524 74.52583315010268) bank127176 +127177 POINT(40.28017313297704 74.21579244090088) bank127177 +127178 POINT(40.523947602633854 74.26106732330712) bank127178 +127179 POINT(40.735753963882935 73.80222169585916) bank127179 +127180 POINT(40.13002098102164 73.59971682290315) bank127180 +127181 POINT(41.23149081914705 74.18942250988773) bank127181 +127182 POINT(39.95609243014016 73.83231222128038) bank127182 +127183 POINT(41.69535652512048 73.37052253555716) bank127183 +127184 POINT(40.78333072487532 74.6771564693527) bank127184 +127185 POINT(41.283535101225205 73.66066957044714) bank127185 +127186 POINT(41.17552125865212 73.10988895872252) bank127186 +127187 POINT(40.844068434481336 73.92952325307832) bank127187 +127188 POINT(41.485382294151556 73.12546086779389) bank127188 +127189 POINT(40.16730709540869 74.54518628858855) bank127189 +127190 POINT(40.59195302275584 74.19527001661149) bank127190 +127191 POINT(40.94061505009053 74.64748284914735) bank127191 +127192 POINT(39.955536463770095 74.51301180236645) bank127192 +127193 POINT(41.60966703121101 74.75092563763391) bank127193 +127194 POINT(41.11661780468006 73.9628106474599) bank127194 +127195 POINT(40.32884669089566 73.99113464636592) bank127195 +127196 POINT(40.41053647023867 74.07770107528557) bank127196 +127197 POINT(41.59574412344071 73.32821293108776) bank127197 +127198 POINT(41.26106611189073 73.16057113600799) bank127198 +127199 POINT(41.4212444708382 73.92105067718096) bank127199 +127200 POINT(41.17639685095017 74.58372395840607) bank127200 +127201 POINT(40.58468599231206 73.70736415969007) bank127201 +127202 POINT(40.86661669162838 74.39458776110367) bank127202 +127203 POINT(41.38831917410958 73.03273890194573) bank127203 +127204 POINT(40.88184450093142 73.44606637635427) bank127204 +127205 POINT(41.29574682749243 73.66021435873076) bank127205 +127206 POINT(39.99815332495744 74.01579189480714) bank127206 +127207 POINT(40.95363000343742 73.17961840546448) bank127207 +127208 POINT(40.49065226470821 73.85642996738369) bank127208 +127209 POINT(40.75693071546568 74.75770530754825) bank127209 +127210 POINT(40.77464964951214 74.02713051160444) bank127210 +127211 POINT(41.22737524011198 74.25881616174904) bank127211 +127212 POINT(40.142220406147544 73.57154727872076) bank127212 +127213 POINT(40.81529136275255 74.41770880677497) bank127213 +127214 POINT(41.262194769624166 73.59289952451138) bank127214 +127215 POINT(41.04803259599381 73.10870631475365) bank127215 +127216 POINT(41.671517785586694 73.46437350406597) bank127216 +127217 POINT(40.506437836060115 74.20585128532137) bank127217 +127218 POINT(41.63253883732478 73.62075028212942) bank127218 +127219 POINT(41.21303355495318 73.37667328944501) bank127219 +127220 POINT(40.94094369938629 73.25412497758562) bank127220 +127221 POINT(39.81715388023769 74.15083562327176) bank127221 +127222 POINT(41.367280811568044 73.20690864898602) bank127222 +127223 POINT(40.65971372981985 73.48388731579405) bank127223 +127224 POINT(40.95813239040579 73.70329484631195) bank127224 +127225 POINT(39.73869148516755 73.64443546958637) bank127225 +127226 POINT(39.912706834909365 73.85417633754363) bank127226 +127227 POINT(41.05814776456726 73.19198809787325) bank127227 +127228 POINT(39.77906653365256 74.590083269139) bank127228 +127229 POINT(41.08987949950755 74.49751118212158) bank127229 +127230 POINT(40.266958519198226 74.25816620650498) bank127230 +127231 POINT(39.73611541766758 74.13763735138743) bank127231 +127232 POINT(40.88059856636397 74.07272032202172) bank127232 +127233 POINT(40.218532057115965 74.985892730981) bank127233 +127234 POINT(41.000827394227 74.23448680165463) bank127234 +127235 POINT(40.26427786092497 73.22834506593978) bank127235 +127236 POINT(40.892402285880834 73.0672324892396) bank127236 +127237 POINT(40.37451072530028 73.86507548738062) bank127237 +127238 POINT(39.894702672551084 73.01195614319423) bank127238 +127239 POINT(40.13316895464805 73.36599021740554) bank127239 +127240 POINT(39.74903798262106 74.17697949502099) bank127240 +127241 POINT(41.39635925900673 74.06350555973732) bank127241 +127242 POINT(40.6718811389044 74.35577743069334) bank127242 +127243 POINT(40.70436203839113 73.17140899278223) bank127243 +127244 POINT(40.19627787265022 74.63651123877169) bank127244 +127245 POINT(39.979941366618164 73.38870565101843) bank127245 +127246 POINT(40.30084131683247 74.77070949872602) bank127246 +127247 POINT(39.996647349460105 74.91933695574772) bank127247 +127248 POINT(40.4534462137322 74.74587881308766) bank127248 +127249 POINT(40.520945013500935 73.9713704817314) bank127249 +127250 POINT(41.21014166812697 73.61675651433029) bank127250 +127251 POINT(40.368350245017275 73.41237909270322) bank127251 +127252 POINT(41.02225209936741 74.74877378192178) bank127252 +127253 POINT(41.05249057508522 73.09152366978614) bank127253 +127254 POINT(41.699136076101134 74.46226006279375) bank127254 +127255 POINT(41.331622670905574 74.13567479249045) bank127255 +127256 POINT(41.24813329819829 73.62447258473185) bank127256 +127257 POINT(41.19065391757463 74.04665996367567) bank127257 +127258 POINT(41.12812210596658 73.6264484870821) bank127258 +127259 POINT(41.698769946331126 73.628861307474) bank127259 +127260 POINT(40.057411760882815 74.02855645689735) bank127260 +127261 POINT(41.68064291310051 73.64976972644291) bank127261 +127262 POINT(40.50866468715384 75.00023383940977) bank127262 +127263 POINT(40.38444361027777 74.53225271591548) bank127263 +127264 POINT(40.497235492065435 73.74978315852539) bank127264 +127265 POINT(40.76657028502296 73.40683723034334) bank127265 +127266 POINT(40.86842197143584 73.28531429912744) bank127266 +127267 POINT(41.42079827920691 74.22384433036919) bank127267 +127268 POINT(39.90536466871799 73.03863763463372) bank127268 +127269 POINT(41.22402043954254 74.91783847419765) bank127269 +127270 POINT(41.24677509461068 73.2483558210945) bank127270 +127271 POINT(40.839297711731106 74.50758618345921) bank127271 +127272 POINT(40.21540213786756 74.28812925318329) bank127272 +127273 POINT(40.228440773470105 74.43006282290136) bank127273 +127274 POINT(40.33891483000153 74.89072482640401) bank127274 +127275 POINT(40.119833744076956 73.48679149359049) bank127275 +127276 POINT(39.718643467696445 73.0474779257386) bank127276 +127277 POINT(40.34759480684865 74.85246003984818) bank127277 +127278 POINT(41.07587185401092 73.96222803243221) bank127278 +127279 POINT(40.67155967047979 74.74661775494631) bank127279 +127280 POINT(39.93820993964391 74.1603172741567) bank127280 +127281 POINT(40.616721857988324 73.99273544996053) bank127281 +127282 POINT(40.740406966216355 74.0182666361887) bank127282 +127283 POINT(41.00836397666097 74.43222410244552) bank127283 +127284 POINT(40.785373398067165 74.5030932393676) bank127284 +127285 POINT(40.05303438128259 74.80809316571047) bank127285 +127286 POINT(41.516004351036415 74.89200237094853) bank127286 +127287 POINT(40.14392430961806 73.54694888191109) bank127287 +127288 POINT(41.42986076744943 73.2959373297412) bank127288 +127289 POINT(39.73248514741584 74.2484891013752) bank127289 +127290 POINT(40.23420445775734 74.32057537902514) bank127290 +127291 POINT(40.064905305508155 74.07586916136235) bank127291 +127292 POINT(41.039897924519416 74.25375153243459) bank127292 +127293 POINT(41.69020481794661 74.05326013902673) bank127293 +127294 POINT(40.48326227825465 73.32934850464507) bank127294 +127295 POINT(40.48717446669181 74.09488162561543) bank127295 +127296 POINT(40.73787301198677 73.65353672730555) bank127296 +127297 POINT(41.59773067193156 73.84785267728171) bank127297 +127298 POINT(40.689953096265164 74.08727198769897) bank127298 +127299 POINT(40.95832876610547 73.51676981134406) bank127299 +127300 POINT(39.718959497363336 73.81765554498315) bank127300 +127301 POINT(39.762663346530466 74.13194760096825) bank127301 +127302 POINT(40.012731928380134 73.52221217368219) bank127302 +127303 POINT(39.76852632786189 74.89964261294011) bank127303 +127304 POINT(40.82638140974152 74.23495803555927) bank127304 +127305 POINT(40.63727438115125 73.66750413260552) bank127305 +127306 POINT(40.31927837300062 73.24062562434479) bank127306 +127307 POINT(41.63802611214796 74.22871777821639) bank127307 +127308 POINT(40.26505716612761 73.07289908198521) bank127308 +127309 POINT(40.49605493899526 73.68628381846933) bank127309 +127310 POINT(40.962184775185804 74.5344298362109) bank127310 +127311 POINT(40.57980834681194 74.99769646209002) bank127311 +127312 POINT(39.74025383544104 74.03864576214922) bank127312 +127313 POINT(39.93516440395618 73.45384630319435) bank127313 +127314 POINT(40.4640857276316 74.05965392482064) bank127314 +127315 POINT(41.0101324178717 73.23335602460024) bank127315 +127316 POINT(40.723063465297315 74.20162965557104) bank127316 +127317 POINT(41.08146795228003 73.94835952473508) bank127317 +127318 POINT(40.75323244382023 73.78036005426875) bank127318 +127319 POINT(40.99105119710169 74.64611078883362) bank127319 +127320 POINT(41.395236667601246 73.08269907321082) bank127320 +127321 POINT(40.29231654184469 73.50838578366645) bank127321 +127322 POINT(41.441142464642724 74.3078295088706) bank127322 +127323 POINT(41.41025826307651 74.6699286464818) bank127323 +127324 POINT(40.77487381287447 74.71288323600254) bank127324 +127325 POINT(40.67160218111278 74.72270410241508) bank127325 +127326 POINT(40.555136261291494 73.25662070493826) bank127326 +127327 POINT(41.08026400189233 73.09538055859458) bank127327 +127328 POINT(40.8528829412631 74.9812760850995) bank127328 +127329 POINT(40.44129614307403 74.18083880100612) bank127329 +127330 POINT(41.262683527358895 73.09358160705334) bank127330 +127331 POINT(40.55702915855191 73.11081013482197) bank127331 +127332 POINT(41.02780928401977 73.86176256662436) bank127332 +127333 POINT(40.723739508180586 73.59017262275063) bank127333 +127334 POINT(39.88495093505647 73.47049461329604) bank127334 +127335 POINT(41.66574665918296 74.93355583915438) bank127335 +127336 POINT(40.001628117686025 73.17973535289576) bank127336 +127337 POINT(40.78890863722207 73.09702066116839) bank127337 +127338 POINT(40.15900381332461 73.06350014806318) bank127338 +127339 POINT(41.625812679426616 73.22405162343631) bank127339 +127340 POINT(39.97018417170148 73.26135435781933) bank127340 +127341 POINT(39.98214023805011 73.5560541938069) bank127341 +127342 POINT(40.00961695013306 73.99287865601953) bank127342 +127343 POINT(40.856862032364496 73.13449790877068) bank127343 +127344 POINT(40.6260690841153 73.77101842585537) bank127344 +127345 POINT(41.70051973587504 74.86434662964662) bank127345 +127346 POINT(40.714748520647746 74.07730490688606) bank127346 +127347 POINT(41.377762804175724 74.27346303412484) bank127347 +127348 POINT(41.26168252089863 73.7434692917686) bank127348 +127349 POINT(41.36305204819251 73.46358245794879) bank127349 +127350 POINT(40.14036409930945 73.33499047601642) bank127350 +127351 POINT(41.10290541066121 74.8667499600941) bank127351 +127352 POINT(41.200174528681444 73.60123988651618) bank127352 +127353 POINT(40.150896249800326 73.97899828044801) bank127353 +127354 POINT(40.58178508127332 74.46054393298166) bank127354 +127355 POINT(40.438078383935164 74.01349765311367) bank127355 +127356 POINT(41.36671061306585 73.22988803328178) bank127356 +127357 POINT(40.18511520107469 73.31337558290583) bank127357 +127358 POINT(40.681463834779706 74.24882431918401) bank127358 +127359 POINT(40.016291834318686 74.19330851301557) bank127359 +127360 POINT(41.681168292413766 73.6190592597992) bank127360 +127361 POINT(40.52965090448253 73.45056905828342) bank127361 +127362 POINT(39.87660904824239 73.5334863145674) bank127362 +127363 POINT(40.897030596284154 74.8894801705248) bank127363 +127364 POINT(40.03707086245874 73.25790792524643) bank127364 +127365 POINT(39.962683967736055 74.52067328916662) bank127365 +127366 POINT(39.8201353344607 74.94603807101544) bank127366 +127367 POINT(40.639905215741024 74.22016926096988) bank127367 +127368 POINT(40.54469367615729 74.76799826230815) bank127368 +127369 POINT(41.28974313453452 73.19885336850204) bank127369 +127370 POINT(40.95765267699261 73.90337737150642) bank127370 +127371 POINT(40.22253484795001 74.61495544119425) bank127371 +127372 POINT(40.213098306706165 73.83119822675567) bank127372 +127373 POINT(41.57046622634783 74.95870246074257) bank127373 +127374 POINT(39.79415914419151 74.27957443094714) bank127374 +127375 POINT(39.80385230412128 74.36010944968838) bank127375 +127376 POINT(39.856009843447616 73.42216150073688) bank127376 +127377 POINT(41.4507776524974 74.21129472110988) bank127377 +127378 POINT(41.566898421561774 74.71779290945283) bank127378 +127379 POINT(41.69746648338495 73.74418766895263) bank127379 +127380 POINT(41.226162916139764 74.79122012934526) bank127380 +127381 POINT(41.090689790184996 73.18170954207932) bank127381 +127382 POINT(39.973223763557556 74.25708516062544) bank127382 +127383 POINT(41.14942925037516 73.3633310171662) bank127383 +127384 POINT(40.696513204237064 74.55931126420195) bank127384 +127385 POINT(40.815721047237595 73.59313652527709) bank127385 +127386 POINT(40.773924323674244 73.27690671854444) bank127386 +127387 POINT(41.345753163413335 74.7779246558195) bank127387 +127388 POINT(41.09192924925661 73.28216820343786) bank127388 +127389 POINT(39.822485884073444 73.26734011818112) bank127389 +127390 POINT(41.613510938880346 73.97896370395347) bank127390 +127391 POINT(39.93774452603035 73.87240159491513) bank127391 +127392 POINT(41.442411883693026 73.86195681429027) bank127392 +127393 POINT(40.816165680288286 74.93139055909081) bank127393 +127394 POINT(41.25250330311484 73.84046782594913) bank127394 +127395 POINT(41.643338206483634 73.33649114001086) bank127395 +127396 POINT(40.93974293152322 74.92206514646132) bank127396 +127397 POINT(40.83128040152142 74.61220128554191) bank127397 +127398 POINT(41.03862246995521 73.3483993756592) bank127398 +127399 POINT(40.01746023451986 74.42106395095907) bank127399 +127400 POINT(39.71491716845791 73.7295224472484) bank127400 +127401 POINT(41.394627061166474 73.20759513950303) bank127401 +127402 POINT(40.32944586582182 74.74461486658466) bank127402 +127403 POINT(40.31411983906613 74.8033886188501) bank127403 +127404 POINT(41.535032280394255 74.77494673713551) bank127404 +127405 POINT(41.25376910359864 74.75445933778407) bank127405 +127406 POINT(39.97143055731175 74.67917735534562) bank127406 +127407 POINT(39.870143115655935 74.04445522648035) bank127407 +127408 POINT(40.01499713991622 73.32729646566212) bank127408 +127409 POINT(41.187235574914254 74.6451118737461) bank127409 +127410 POINT(41.62379176966476 74.87621848367651) bank127410 +127411 POINT(40.018341989421856 73.59734030507785) bank127411 +127412 POINT(40.21215193575766 74.42874877994431) bank127412 +127413 POINT(39.883817302402385 73.3017893333007) bank127413 +127414 POINT(41.36406406175647 74.48593888323522) bank127414 +127415 POINT(41.43134384668513 73.54466628145474) bank127415 +127416 POINT(40.11242917935678 73.27145386315128) bank127416 +127417 POINT(41.1042985977632 74.66784918035256) bank127417 +127418 POINT(40.4766258432466 74.4246850864903) bank127418 +127419 POINT(39.80327936506296 74.75044796938587) bank127419 +127420 POINT(41.55574039215688 73.71946707098411) bank127420 +127421 POINT(40.03968758099331 73.06040965641692) bank127421 +127422 POINT(39.97879539283902 73.56754376337987) bank127422 +127423 POINT(40.65735008513935 74.61730641538371) bank127423 +127424 POINT(41.25775581710939 73.05936818224376) bank127424 +127425 POINT(40.37449418014518 74.49468671797544) bank127425 +127426 POINT(41.112012843552804 74.49972685093233) bank127426 +127427 POINT(41.39234149886037 74.60712484641164) bank127427 +127428 POINT(41.02959379096906 74.48976775572487) bank127428 +127429 POINT(41.22027217895206 74.62851531334844) bank127429 +127430 POINT(41.11321625043824 74.32547213519916) bank127430 +127431 POINT(41.07111978200938 74.02538548671586) bank127431 +127432 POINT(41.50528811129548 73.33982953557654) bank127432 +127433 POINT(40.22936883936318 74.47948919180094) bank127433 +127434 POINT(39.76742063229926 74.35112111818643) bank127434 +127435 POINT(40.74475448005277 74.65651667361846) bank127435 +127436 POINT(41.31269831243365 73.53356700180163) bank127436 +127437 POINT(39.928529779950814 74.85845443264036) bank127437 +127438 POINT(40.75087804061406 73.49425127546424) bank127438 +127439 POINT(41.689580032992886 73.09884367575265) bank127439 +127440 POINT(41.57928642070373 74.9508286739246) bank127440 +127441 POINT(41.49016838849334 74.74506012579384) bank127441 +127442 POINT(39.86400791596224 73.70878316694728) bank127442 +127443 POINT(40.74750115768262 73.76530793544543) bank127443 +127444 POINT(40.25859713585744 74.15838130494735) bank127444 +127445 POINT(39.856247622310775 73.76700128049693) bank127445 +127446 POINT(40.35396237094091 73.85607201212042) bank127446 +127447 POINT(39.73860944269039 73.6761384860739) bank127447 +127448 POINT(40.692357455432855 73.76378320800761) bank127448 +127449 POINT(39.91892345947238 73.16889872914115) bank127449 +127450 POINT(41.60751426809272 74.87832583206222) bank127450 +127451 POINT(41.23904820798755 74.09368601576766) bank127451 +127452 POINT(39.78803368134089 73.10442192416662) bank127452 +127453 POINT(40.06626493727246 73.85593353443709) bank127453 +127454 POINT(41.0523497220003 73.98498169181158) bank127454 +127455 POINT(41.40231592194997 74.28764050837943) bank127455 +127456 POINT(39.73705798402108 73.85857695643875) bank127456 +127457 POINT(40.14093899405958 73.97272702978259) bank127457 +127458 POINT(41.46806636218047 73.46400937731377) bank127458 +127459 POINT(40.06596282510802 74.96994914199794) bank127459 +127460 POINT(40.82662895553127 74.06667322764524) bank127460 +127461 POINT(41.49674652490117 73.78283616282113) bank127461 +127462 POINT(40.77705608523769 74.74850877691524) bank127462 +127463 POINT(40.45477475914292 74.35106949712393) bank127463 +127464 POINT(40.33774863502432 73.29874231969683) bank127464 +127465 POINT(40.76860391430442 74.94975947274973) bank127465 +127466 POINT(41.35409961686382 73.90790435110792) bank127466 +127467 POINT(40.84573720807769 74.88326792592902) bank127467 +127468 POINT(39.93613778856826 74.98034771955187) bank127468 +127469 POINT(40.58595497672409 74.22322902345444) bank127469 +127470 POINT(41.61954375833909 74.44000973821126) bank127470 +127471 POINT(41.39144383815478 73.2248388913398) bank127471 +127472 POINT(40.43475343140195 74.25706474333913) bank127472 +127473 POINT(39.87148496498277 73.20328371194563) bank127473 +127474 POINT(40.25895579651719 73.70749798084734) bank127474 +127475 POINT(41.01370676173659 74.41051675353194) bank127475 +127476 POINT(40.472158300195666 74.1957675951288) bank127476 +127477 POINT(40.959739658316586 74.96651234913907) bank127477 +127478 POINT(39.933156280715366 74.2390238997284) bank127478 +127479 POINT(39.9465698931495 73.21454274222016) bank127479 +127480 POINT(39.82453474919297 73.1909464049887) bank127480 +127481 POINT(40.304175878542985 73.65490789315662) bank127481 +127482 POINT(39.81608795991526 73.22241479430988) bank127482 +127483 POINT(40.00534814137176 74.10160132926872) bank127483 +127484 POINT(40.49914325380471 74.49708407535489) bank127484 +127485 POINT(40.896187549241816 74.81618796778915) bank127485 +127486 POINT(40.9673998017502 73.94111760273783) bank127486 +127487 POINT(40.83466435886596 74.24309473790338) bank127487 +127488 POINT(40.784869127005834 74.43717082430418) bank127488 +127489 POINT(40.553448462760635 74.83207193394753) bank127489 +127490 POINT(40.3725152804164 74.33801653322148) bank127490 +127491 POINT(41.683288159496094 74.66204740681533) bank127491 +127492 POINT(40.287691272171166 73.60015391644035) bank127492 +127493 POINT(40.10154787001801 74.93334450321115) bank127493 +127494 POINT(40.44268412218396 74.4694857129732) bank127494 +127495 POINT(41.02886711757243 73.84881186770454) bank127495 +127496 POINT(40.087956563507824 74.75366577791446) bank127496 +127497 POINT(39.887690051313214 73.10063804481014) bank127497 +127498 POINT(40.9230873103754 74.45934806389612) bank127498 +127499 POINT(40.962470912817814 73.71275563032536) bank127499 +127500 POINT(40.79663673467065 74.59014635238857) bank127500 +127501 POINT(39.76290516469387 74.69881268309736) bank127501 +127502 POINT(40.63264051468894 73.86265097327679) bank127502 +127503 POINT(40.44952019305018 74.09174273314957) bank127503 +127504 POINT(40.85200776746505 74.73332978079266) bank127504 +127505 POINT(40.80178120251175 74.49313569127251) bank127505 +127506 POINT(40.586572248737625 73.57689599903522) bank127506 +127507 POINT(40.98472252326281 74.80632189368676) bank127507 +127508 POINT(40.68178300786313 74.41327666206163) bank127508 +127509 POINT(40.971999134599784 74.12230426883677) bank127509 +127510 POINT(40.913872010153824 73.82363859334201) bank127510 +127511 POINT(40.094426202751194 73.26565756869304) bank127511 +127512 POINT(40.30037073401314 73.05120853427434) bank127512 +127513 POINT(40.873002134713886 74.27974482767817) bank127513 +127514 POINT(40.01249705387921 73.82522851469909) bank127514 +127515 POINT(40.932646685530734 74.14039588188389) bank127515 +127516 POINT(41.46777926860154 73.77010016035642) bank127516 +127517 POINT(41.58532322498273 73.89144695329404) bank127517 +127518 POINT(40.2151654503519 74.89227706952579) bank127518 +127519 POINT(41.32127722970523 73.67115202333075) bank127519 +127520 POINT(41.625727271354776 74.9646076765618) bank127520 +127521 POINT(39.75084042642196 73.65386137799995) bank127521 +127522 POINT(40.333990984681 73.71003634752054) bank127522 +127523 POINT(41.59638945560439 73.6713606296718) bank127523 +127524 POINT(40.92220131172598 74.64165910979204) bank127524 +127525 POINT(41.1176193396392 74.30460230078523) bank127525 +127526 POINT(40.95048804837677 73.06882891341901) bank127526 +127527 POINT(39.901578977663334 73.58771520270868) bank127527 +127528 POINT(41.65780739743395 74.53234651395265) bank127528 +127529 POINT(40.667286889158554 73.49080382499231) bank127529 +127530 POINT(39.81748203310272 74.03428346754072) bank127530 +127531 POINT(40.474000084597215 73.26320212563327) bank127531 +127532 POINT(40.818371214805545 73.87474638513193) bank127532 +127533 POINT(41.26897169659768 74.48368927758791) bank127533 +127534 POINT(41.359507215228106 73.6348355550042) bank127534 +127535 POINT(40.816125714509006 74.49101587400283) bank127535 +127536 POINT(40.773742426760535 73.77536355847563) bank127536 +127537 POINT(41.25325938706263 73.21188010403233) bank127537 +127538 POINT(40.527774510205404 73.44724235792374) bank127538 +127539 POINT(40.08710383790951 73.0662121018234) bank127539 +127540 POINT(40.17341279199008 74.14707473992608) bank127540 +127541 POINT(41.42767643822382 74.22395230837195) bank127541 +127542 POINT(41.40293077155029 74.7647195457612) bank127542 +127543 POINT(41.33723806607854 73.07097807948809) bank127543 +127544 POINT(40.88829083739561 73.73821452613745) bank127544 +127545 POINT(41.45771550408178 73.37298111617864) bank127545 +127546 POINT(41.214258470734805 73.59297639550768) bank127546 +127547 POINT(40.35568479997541 74.41698754880491) bank127547 +127548 POINT(40.50461187329501 73.04956477296363) bank127548 +127549 POINT(40.220440582027955 73.83162964806358) bank127549 +127550 POINT(40.41065501451596 74.88554864592047) bank127550 +127551 POINT(40.59754545616229 73.46132992181964) bank127551 +127552 POINT(41.448063611478894 74.78400064733992) bank127552 +127553 POINT(39.94844931357878 74.47947618061684) bank127553 +127554 POINT(41.61318445426904 74.55831248257864) bank127554 +127555 POINT(41.4729348911761 73.24518017125735) bank127555 +127556 POINT(40.77696410297371 73.07990851912028) bank127556 +127557 POINT(40.80254158583334 74.02452863454943) bank127557 +127558 POINT(41.36475896849243 73.28737102101886) bank127558 +127559 POINT(40.30954630484628 73.43243250901443) bank127559 +127560 POINT(41.198558028496976 74.86949810595256) bank127560 +127561 POINT(40.23021842292842 73.33798615459335) bank127561 +127562 POINT(41.15152832164801 74.21216535643558) bank127562 +127563 POINT(40.422518269066394 74.66308047587887) bank127563 +127564 POINT(40.99099334407972 74.87302405817297) bank127564 +127565 POINT(40.64556798672615 74.89199194870304) bank127565 +127566 POINT(41.42287902578419 74.48344010119649) bank127566 +127567 POINT(39.88453264270307 74.46393891911272) bank127567 +127568 POINT(41.05896834370602 73.35538464305917) bank127568 +127569 POINT(40.14765073115287 74.5442772925373) bank127569 +127570 POINT(40.827323778049106 73.4781012276668) bank127570 +127571 POINT(40.160747391293896 73.12244467267399) bank127571 +127572 POINT(40.45892702346945 73.6466169109956) bank127572 +127573 POINT(40.86391284063359 74.46539141213486) bank127573 +127574 POINT(41.54507450687283 74.86192138484878) bank127574 +127575 POINT(40.612353928257974 73.37349855515426) bank127575 +127576 POINT(39.991805570621935 74.15764949551742) bank127576 +127577 POINT(40.133979217447184 74.52172081339846) bank127577 +127578 POINT(40.106804020545475 73.79119485244588) bank127578 +127579 POINT(40.16151226190577 74.43351833477003) bank127579 +127580 POINT(41.68760523993054 73.2593578175813) bank127580 +127581 POINT(40.89231528661912 73.42312433639222) bank127581 +127582 POINT(40.59898886189953 73.34905137437488) bank127582 +127583 POINT(41.71199317740939 73.09609145707967) bank127583 +127584 POINT(40.16712947739668 74.29264305362011) bank127584 +127585 POINT(40.159748935111416 73.5013322197662) bank127585 +127586 POINT(40.02559571358978 73.11071856310646) bank127586 +127587 POINT(41.398479672260436 74.21753630621973) bank127587 +127588 POINT(40.751779643515384 74.80693251605432) bank127588 +127589 POINT(39.80981284151502 73.07900521184978) bank127589 +127590 POINT(39.807230012812056 73.21927133543079) bank127590 +127591 POINT(41.40376217850958 73.40645009845686) bank127591 +127592 POINT(40.85871464310349 74.84756772907988) bank127592 +127593 POINT(39.91141869185105 74.20351265965348) bank127593 +127594 POINT(41.323793239542844 73.9316209713744) bank127594 +127595 POINT(40.622896079415064 73.49880682820789) bank127595 +127596 POINT(41.642485488739624 73.13314721792354) bank127596 +127597 POINT(41.524104773270295 73.05662552480585) bank127597 +127598 POINT(39.88527228622014 74.84162937848546) bank127598 +127599 POINT(40.84805515106155 74.8665450932324) bank127599 +127600 POINT(40.87523732720065 73.29628007404139) bank127600 +127601 POINT(41.556221872561444 73.25904307197322) bank127601 +127602 POINT(39.85379679520135 73.86183888465987) bank127602 +127603 POINT(39.75322965219048 73.79739305416057) bank127603 +127604 POINT(41.44104302853967 74.28613224669435) bank127604 +127605 POINT(41.39625375204414 74.5446079833948) bank127605 +127606 POINT(40.7849243480971 74.34034815378568) bank127606 +127607 POINT(40.00365546719131 74.49475727766983) bank127607 +127608 POINT(40.404703416482434 73.66963738597364) bank127608 +127609 POINT(40.63308602253436 74.20270147581846) bank127609 +127610 POINT(40.62498255669941 74.8582869071744) bank127610 +127611 POINT(41.01475181323132 74.92277951905666) bank127611 +127612 POINT(40.26273363779107 74.2236237089094) bank127612 +127613 POINT(41.56122788588137 73.03723503360905) bank127613 +127614 POINT(40.13954063988138 74.21676662275523) bank127614 +127615 POINT(40.52803292161824 74.4004580461824) bank127615 +127616 POINT(41.19632163451948 74.00804391602894) bank127616 +127617 POINT(41.13602134577886 74.08032159275534) bank127617 +127618 POINT(41.01699813003301 74.73540851235386) bank127618 +127619 POINT(41.15486954575696 73.52692174664953) bank127619 +127620 POINT(40.48835283625472 74.54265634186578) bank127620 +127621 POINT(41.01428446165798 73.79460749102645) bank127621 +127622 POINT(39.97689000136532 74.82762971907404) bank127622 +127623 POINT(40.616273956499484 73.2592244989351) bank127623 +127624 POINT(40.71535277108752 74.42191682790487) bank127624 +127625 POINT(39.87245183017925 74.7751797600298) bank127625 +127626 POINT(40.211623258083904 74.46219796190688) bank127626 +127627 POINT(40.68127998325634 74.11463745280211) bank127627 +127628 POINT(41.07601295844103 73.67316567132902) bank127628 +127629 POINT(39.83657189911793 74.05906431347634) bank127629 +127630 POINT(39.920938016002914 74.3543953907433) bank127630 +127631 POINT(41.43293560237393 74.30029602816701) bank127631 +127632 POINT(39.781356247924705 73.90222054869473) bank127632 +127633 POINT(40.76468831554959 74.46697787269403) bank127633 +127634 POINT(40.093545890681625 73.69475480521739) bank127634 +127635 POINT(40.78481418917025 74.50887699832599) bank127635 +127636 POINT(41.61949247964882 74.6831456707776) bank127636 +127637 POINT(40.33049715627605 74.1459775069643) bank127637 +127638 POINT(41.5574972013883 73.37494687459802) bank127638 +127639 POINT(40.528452287697924 73.08415323566791) bank127639 +127640 POINT(40.96327482272812 74.14074564868065) bank127640 +127641 POINT(39.74726756277215 74.776857865834) bank127641 +127642 POINT(39.91049361288759 74.88696378382612) bank127642 +127643 POINT(41.379652708384235 74.31684484451432) bank127643 +127644 POINT(40.25617481648263 73.78473525709136) bank127644 +127645 POINT(41.396823688087785 73.19466405197116) bank127645 +127646 POINT(40.01621016505595 73.77083451474694) bank127646 +127647 POINT(41.54871449692385 74.15114216782304) bank127647 +127648 POINT(40.01635317827187 73.8844048019907) bank127648 +127649 POINT(39.761674619296656 74.70559671192049) bank127649 +127650 POINT(39.85021581671232 74.10663694795804) bank127650 +127651 POINT(41.374968002526394 73.73963230838068) bank127651 +127652 POINT(41.26539859548898 73.65957482740889) bank127652 +127653 POINT(39.77931400616648 73.31890050689609) bank127653 +127654 POINT(40.42083189228267 73.11957965176894) bank127654 +127655 POINT(40.142061205291384 74.40812685920959) bank127655 +127656 POINT(41.51848565884456 74.07219163886379) bank127656 +127657 POINT(40.5595699359451 74.91580114262547) bank127657 +127658 POINT(40.98251354093096 74.67892864744742) bank127658 +127659 POINT(41.16697133504544 73.3757437725008) bank127659 +127660 POINT(39.92582181587236 73.83776782945841) bank127660 +127661 POINT(40.76682340014031 74.51888432243109) bank127661 +127662 POINT(40.99819956330577 74.16421901604004) bank127662 +127663 POINT(41.62431477805649 73.26436953667238) bank127663 +127664 POINT(39.75642012021826 73.17630988879463) bank127664 +127665 POINT(40.37903131743359 73.71898995629431) bank127665 +127666 POINT(41.281742513250734 74.86350851927473) bank127666 +127667 POINT(41.47464803427345 74.72017456433811) bank127667 +127668 POINT(41.395179115027986 74.12854281015395) bank127668 +127669 POINT(41.54412184869802 74.420681343395) bank127669 +127670 POINT(40.174536703026504 74.37744738462015) bank127670 +127671 POINT(41.3627589345807 73.74522880385447) bank127671 +127672 POINT(40.861463178577075 74.93002296552436) bank127672 +127673 POINT(40.88578902395066 73.05478292781973) bank127673 +127674 POINT(41.348349113108384 74.70374683314724) bank127674 +127675 POINT(40.56662477354287 74.77754633100552) bank127675 +127676 POINT(40.570912850675434 73.10041409500612) bank127676 +127677 POINT(40.64005221372758 73.03029752402769) bank127677 +127678 POINT(41.5390470153636 73.48369010490028) bank127678 +127679 POINT(40.72558206168507 74.50726905666055) bank127679 +127680 POINT(40.68689623600849 73.74961682729206) bank127680 +127681 POINT(40.16359132296346 73.15814070629507) bank127681 +127682 POINT(40.05409995246086 73.32585749982746) bank127682 +127683 POINT(41.014942038221236 73.83811464518878) bank127683 +127684 POINT(40.55934437133838 74.69211823803658) bank127684 +127685 POINT(41.50602382579619 74.36582560087028) bank127685 +127686 POINT(41.2294735179172 73.78694457336428) bank127686 +127687 POINT(40.72453432243572 74.61381936878965) bank127687 +127688 POINT(40.839055250922584 73.80972223043248) bank127688 +127689 POINT(40.95129072923412 73.42335629150881) bank127689 +127690 POINT(40.89953187968595 73.1806548651856) bank127690 +127691 POINT(40.202641565028834 74.54535757368573) bank127691 +127692 POINT(41.25479132844963 74.25634142835955) bank127692 +127693 POINT(41.08045392707566 73.95391828671384) bank127693 +127694 POINT(40.54701815468998 73.95416538490996) bank127694 +127695 POINT(41.25966451520937 73.4437192200625) bank127695 +127696 POINT(41.47900578799101 74.05376537995367) bank127696 +127697 POINT(41.54905775571916 74.58834967042166) bank127697 +127698 POINT(40.30346912539959 74.97269109694919) bank127698 +127699 POINT(40.242465978368635 74.31303326040985) bank127699 +127700 POINT(41.294369347366924 74.34670141110063) bank127700 +127701 POINT(40.33237092669343 74.67126200950439) bank127701 +127702 POINT(40.28029323776849 74.19385559550179) bank127702 +127703 POINT(40.93879841355563 74.61520744670575) bank127703 +127704 POINT(40.6006280368588 74.90777549410797) bank127704 +127705 POINT(41.37775653877539 74.6731819668484) bank127705 +127706 POINT(40.45435203312148 74.36629293489389) bank127706 +127707 POINT(41.626372997201905 74.3525063048779) bank127707 +127708 POINT(40.35653839726445 74.32168058393037) bank127708 +127709 POINT(40.438983413709195 73.3621523138652) bank127709 +127710 POINT(40.15353150513629 74.2001699716236) bank127710 +127711 POINT(41.232786264485384 74.79659229348303) bank127711 +127712 POINT(41.70298061584735 73.32345795443631) bank127712 +127713 POINT(41.35619113083545 74.48529261479219) bank127713 +127714 POINT(40.93520169738242 73.91060269719631) bank127714 +127715 POINT(41.24975619029394 73.80240308012104) bank127715 +127716 POINT(41.615537863199656 73.2479311212582) bank127716 +127717 POINT(39.968039460059266 74.22891543157719) bank127717 +127718 POINT(40.28901931271868 73.16572540488637) bank127718 +127719 POINT(41.627437599951094 74.4584405154665) bank127719 +127720 POINT(41.425338289767595 73.40362980763652) bank127720 +127721 POINT(40.225251046330605 73.87829350130787) bank127721 +127722 POINT(39.7318359497969 73.41418298727744) bank127722 +127723 POINT(39.80200912987442 74.61483115367754) bank127723 +127724 POINT(40.530646656234 74.12978510893836) bank127724 +127725 POINT(40.217917485653594 74.11400100858353) bank127725 +127726 POINT(40.78151607076531 73.53758197126612) bank127726 +127727 POINT(39.98220964279383 73.04906575123982) bank127727 +127728 POINT(40.04042271740825 74.50687178657934) bank127728 +127729 POINT(41.37443067895561 74.50729034791225) bank127729 +127730 POINT(40.420223353537416 73.22720854109345) bank127730 +127731 POINT(41.07738958190317 74.53982575794106) bank127731 +127732 POINT(40.74639078271417 74.0930582844152) bank127732 +127733 POINT(41.45682661197106 73.93523322266638) bank127733 +127734 POINT(41.560573749116344 73.24422579739053) bank127734 +127735 POINT(40.791028513783345 74.67636043823485) bank127735 +127736 POINT(41.135204273230485 74.88091563533992) bank127736 +127737 POINT(41.02740905447209 74.5549652669658) bank127737 +127738 POINT(40.043747853702925 74.32520152963272) bank127738 +127739 POINT(40.664114695728415 74.31037170666086) bank127739 +127740 POINT(41.234373556353276 73.13405238492025) bank127740 +127741 POINT(40.551973255472845 74.51070490287051) bank127741 +127742 POINT(40.077370190391306 74.59617119397032) bank127742 +127743 POINT(41.63193795247898 74.06278276295849) bank127743 +127744 POINT(39.93783430740835 74.19951545957103) bank127744 +127745 POINT(41.18573764639545 74.34814511340782) bank127745 +127746 POINT(39.71621664458881 74.93570306229756) bank127746 +127747 POINT(39.91732076902781 74.08128343294365) bank127747 +127748 POINT(41.681664425860205 74.45320607320366) bank127748 +127749 POINT(41.57182666957832 73.60124456685665) bank127749 +127750 POINT(39.98971771508762 73.44213408057455) bank127750 +127751 POINT(40.3977035306883 74.76569135515665) bank127751 +127752 POINT(40.45623828910559 74.92406265607184) bank127752 +127753 POINT(40.4734143734041 73.84549448397043) bank127753 +127754 POINT(40.77578574049311 74.53925994883083) bank127754 +127755 POINT(40.55486513771199 73.40025598304767) bank127755 +127756 POINT(41.26879111358489 74.92265544217946) bank127756 +127757 POINT(41.69349024084848 73.07882767468543) bank127757 +127758 POINT(40.763203805078085 73.82975657398362) bank127758 +127759 POINT(40.7988079039926 73.78468547622214) bank127759 +127760 POINT(39.905345454591334 74.0490507261069) bank127760 +127761 POINT(41.384430247894656 74.56674605346116) bank127761 +127762 POINT(40.56861537860828 73.07797720921798) bank127762 +127763 POINT(40.89048545560082 74.51053905978411) bank127763 +127764 POINT(40.70137932470051 75.00219244924082) bank127764 +127765 POINT(40.545716880869854 73.40941836289085) bank127765 +127766 POINT(40.89018309750664 74.0447772779241) bank127766 +127767 POINT(40.989534410605415 73.32859253406457) bank127767 +127768 POINT(40.26672586706267 73.46556661054876) bank127768 +127769 POINT(39.9824210207381 73.63659687451732) bank127769 +127770 POINT(40.09627908778278 74.06197265160196) bank127770 +127771 POINT(41.63407880285522 74.75843595957592) bank127771 +127772 POINT(40.852489729957064 73.56689712412019) bank127772 +127773 POINT(40.893399802675084 73.79958434858547) bank127773 +127774 POINT(39.994430874679324 73.23317505195227) bank127774 +127775 POINT(39.885196832416305 74.04045095476779) bank127775 +127776 POINT(41.250427295216504 73.32406531053455) bank127776 +127777 POINT(40.97956028335878 74.33644097084064) bank127777 +127778 POINT(39.72153184056187 74.42209641934058) bank127778 +127779 POINT(40.98251139222975 73.98904696532315) bank127779 +127780 POINT(39.949995484752066 73.02225750745822) bank127780 +127781 POINT(41.41017361777751 73.53836531558434) bank127781 +127782 POINT(39.76366162584735 73.55463015835738) bank127782 +127783 POINT(41.581991180497845 74.25454043600932) bank127783 +127784 POINT(39.87320100466398 74.52434654328592) bank127784 +127785 POINT(40.52174196691434 74.28340356536032) bank127785 +127786 POINT(39.8002553915569 73.2647334039885) bank127786 +127787 POINT(40.50132250880089 73.89188714310312) bank127787 +127788 POINT(40.63155384595615 73.6551658094185) bank127788 +127789 POINT(40.05995870418201 74.18299141322862) bank127789 +127790 POINT(40.58341036227926 73.74354083518355) bank127790 +127791 POINT(40.168178859525824 74.6158540240345) bank127791 +127792 POINT(41.48771832122728 73.65380860326295) bank127792 +127793 POINT(40.22380655534251 73.47480700021038) bank127793 +127794 POINT(40.59242499682203 73.18939544582258) bank127794 +127795 POINT(41.10705794420622 74.60258481172868) bank127795 +127796 POINT(40.38818080807636 73.63897151718965) bank127796 +127797 POINT(41.278223247826965 74.96416919595171) bank127797 +127798 POINT(40.72456677657868 74.66132227845581) bank127798 +127799 POINT(40.81449412720108 73.63087991003935) bank127799 +127800 POINT(39.87691853510053 73.28896576335593) bank127800 +127801 POINT(41.115286488470055 74.94807228940802) bank127801 +127802 POINT(40.39312781239975 73.62769674291724) bank127802 +127803 POINT(40.72262163817561 74.92329350277242) bank127803 +127804 POINT(40.8407067197378 73.25766110543022) bank127804 +127805 POINT(40.38036448689648 73.1967560998983) bank127805 +127806 POINT(40.5991614762238 73.62968322808794) bank127806 +127807 POINT(40.56758179763051 73.4379140936447) bank127807 +127808 POINT(41.17746549716502 74.04492741896169) bank127808 +127809 POINT(40.73707805844721 73.72893118088582) bank127809 +127810 POINT(41.553466820448534 74.03903301757519) bank127810 +127811 POINT(40.39128690730506 73.67300301342998) bank127811 +127812 POINT(40.08349592233662 74.87402759393638) bank127812 +127813 POINT(40.42627513465546 73.58233104993552) bank127813 +127814 POINT(40.0895343677122 74.58710409187043) bank127814 +127815 POINT(40.33808546584822 73.1598154551801) bank127815 +127816 POINT(40.48611758000615 74.38449869004262) bank127816 +127817 POINT(40.12936209360966 73.62829722854867) bank127817 +127818 POINT(40.59575256552193 74.52954520913728) bank127818 +127819 POINT(41.688622789540354 74.43982142719739) bank127819 +127820 POINT(41.515297705802055 74.51763225238008) bank127820 +127821 POINT(40.49038609733789 73.64969915985021) bank127821 +127822 POINT(39.96139826674831 74.80542877214384) bank127822 +127823 POINT(41.50169814618734 74.80438529242801) bank127823 +127824 POINT(40.22871283907259 73.71391583582582) bank127824 +127825 POINT(41.506126328861946 73.87264038647521) bank127825 +127826 POINT(41.25100189259068 74.32081415705048) bank127826 +127827 POINT(41.30134675378708 74.24739795957842) bank127827 +127828 POINT(41.25184661688203 74.43796453928046) bank127828 +127829 POINT(39.98526295775483 74.43681266161356) bank127829 +127830 POINT(40.87764334877072 73.95375733399146) bank127830 +127831 POINT(40.84651421271529 73.24818417166226) bank127831 +127832 POINT(40.670558242800624 73.84521109246081) bank127832 +127833 POINT(40.10888724723858 74.7835644037215) bank127833 +127834 POINT(41.13679920010637 74.20082675717741) bank127834 +127835 POINT(40.73314254232857 74.19909541046769) bank127835 +127836 POINT(40.97153198028914 73.70124933414418) bank127836 +127837 POINT(41.10344118973167 73.65108978512713) bank127837 +127838 POINT(40.28657448485126 73.00806163829903) bank127838 +127839 POINT(40.70390102469132 74.25901225394087) bank127839 +127840 POINT(39.95370469601604 73.54059033563472) bank127840 +127841 POINT(40.689572941386054 74.72230514386021) bank127841 +127842 POINT(41.65557022022887 74.38310855714631) bank127842 +127843 POINT(40.4395475869696 73.49839483372737) bank127843 +127844 POINT(41.37174285048601 73.59002509180289) bank127844 +127845 POINT(41.6291615718002 74.86695075335305) bank127845 +127846 POINT(40.36763715616414 73.65743153191977) bank127846 +127847 POINT(40.40179669872611 73.95653313198068) bank127847 +127848 POINT(39.953449810082404 74.80872691360057) bank127848 +127849 POINT(40.001044456573965 75.00556854222344) bank127849 +127850 POINT(41.302914423032405 74.27479349990011) bank127850 +127851 POINT(39.84574353114068 73.90428926492913) bank127851 +127852 POINT(41.682440196848894 73.07181746228285) bank127852 +127853 POINT(40.703061430512136 74.17904626431812) bank127853 +127854 POINT(41.062542792129484 74.40900160118308) bank127854 +127855 POINT(39.879717980149934 73.9742364366252) bank127855 +127856 POINT(40.86680386048577 74.41457156978244) bank127856 +127857 POINT(41.692579772919565 73.39243197496351) bank127857 +127858 POINT(40.38253575337577 74.014619775682) bank127858 +127859 POINT(41.33152036629033 74.65731692655532) bank127859 +127860 POINT(41.65208445220167 73.39852300224061) bank127860 +127861 POINT(41.39003068287739 74.49727716559026) bank127861 +127862 POINT(40.495025684224665 73.74347080329159) bank127862 +127863 POINT(40.68849878422905 74.87542023991358) bank127863 +127864 POINT(41.67350666839861 74.54738900161007) bank127864 +127865 POINT(40.26690194543698 73.77577078009688) bank127865 +127866 POINT(40.492331758913636 73.61243247638376) bank127866 +127867 POINT(40.29830302286107 74.29176089438197) bank127867 +127868 POINT(40.71289063018644 74.34978204495559) bank127868 +127869 POINT(40.49792371928291 73.58800245399472) bank127869 +127870 POINT(39.88684911166809 73.51533728737706) bank127870 +127871 POINT(40.495855750254584 74.57889656458808) bank127871 +127872 POINT(41.60012091432253 73.9690454588856) bank127872 +127873 POINT(40.19437691885491 73.61825946696972) bank127873 +127874 POINT(41.14463463685669 73.63462479594708) bank127874 +127875 POINT(41.638790820561645 74.73821642800628) bank127875 +127876 POINT(41.640450555971526 74.80799720796234) bank127876 +127877 POINT(39.74923912631693 73.54188698933832) bank127877 +127878 POINT(40.62625012759639 74.03142092158373) bank127878 +127879 POINT(40.2753233578209 74.90943346418996) bank127879 +127880 POINT(40.86905237173296 74.3105615741915) bank127880 +127881 POINT(39.920897792333946 74.94163352966532) bank127881 +127882 POINT(41.65735823062114 73.19212502272853) bank127882 +127883 POINT(41.10027813220555 74.01438243706664) bank127883 +127884 POINT(39.76680866912168 73.77581210509123) bank127884 +127885 POINT(40.75558892726091 74.42056392661532) bank127885 +127886 POINT(39.744547828953166 74.30331178959587) bank127886 +127887 POINT(40.97944860010825 74.93544532773319) bank127887 +127888 POINT(40.15478154667291 73.53256107015335) bank127888 +127889 POINT(39.75429982954097 74.57289620456883) bank127889 +127890 POINT(41.109154121765954 73.38766300939184) bank127890 +127891 POINT(41.46083488615172 74.52658353950996) bank127891 +127892 POINT(41.70992013365718 74.38225099099645) bank127892 +127893 POINT(40.19844627789199 73.32930396769525) bank127893 +127894 POINT(40.71910245422589 74.3368477460894) bank127894 +127895 POINT(41.22965102152218 73.69862540783576) bank127895 +127896 POINT(40.23921893450486 73.5297092221679) bank127896 +127897 POINT(39.741634966352905 74.44856077552507) bank127897 +127898 POINT(40.30072133635673 74.58957529255147) bank127898 +127899 POINT(40.79357476654672 73.90865463029061) bank127899 +127900 POINT(41.635946147288685 73.05520003558637) bank127900 +127901 POINT(41.485284274424274 73.55091104333906) bank127901 +127902 POINT(41.363989665748825 73.72803525018874) bank127902 +127903 POINT(40.665167464795104 74.54425493523294) bank127903 +127904 POINT(40.51393921036353 74.37356104430077) bank127904 +127905 POINT(39.99890305956969 73.27723965687589) bank127905 +127906 POINT(41.26899787053995 74.19017361549412) bank127906 +127907 POINT(41.40100825150347 74.59161106879516) bank127907 +127908 POINT(40.25852471375325 74.25388554742952) bank127908 +127909 POINT(40.28202654759819 73.27247454514904) bank127909 +127910 POINT(41.695403114815406 74.9825553241975) bank127910 +127911 POINT(39.78467438881583 73.65465282216651) bank127911 +127912 POINT(40.440463007343496 73.39289133341993) bank127912 +127913 POINT(41.09652951614251 74.84243989818921) bank127913 +127914 POINT(39.88585669062014 73.9977809184987) bank127914 +127915 POINT(40.26427967978176 74.7850027723808) bank127915 +127916 POINT(40.88813869006731 74.96076878608162) bank127916 +127917 POINT(40.47294704148931 73.97671871061502) bank127917 +127918 POINT(41.14571868440713 74.51611681170029) bank127918 +127919 POINT(40.670605246978496 73.28309255131204) bank127919 +127920 POINT(41.61710398563659 73.71578083045559) bank127920 +127921 POINT(39.91186771714221 74.40032301735043) bank127921 +127922 POINT(40.33346134069923 73.61430445603878) bank127922 +127923 POINT(40.99909286976248 74.40083323575672) bank127923 +127924 POINT(41.56176498934985 74.80782962522717) bank127924 +127925 POINT(40.887994936086606 73.13961300293657) bank127925 +127926 POINT(40.02905193478238 73.16862131309632) bank127926 +127927 POINT(40.370456047021435 73.56391775686414) bank127927 +127928 POINT(40.121613947655135 73.03325397374338) bank127928 +127929 POINT(40.536476240939116 73.57532736128474) bank127929 +127930 POINT(41.68282903218295 74.23840643595295) bank127930 +127931 POINT(41.564504917834576 74.29010857396214) bank127931 +127932 POINT(41.05720920095656 73.93219079545408) bank127932 +127933 POINT(40.8737037793497 74.42985927871925) bank127933 +127934 POINT(39.90418510674967 73.87263584433869) bank127934 +127935 POINT(40.56257157885447 73.21850590341374) bank127935 +127936 POINT(40.942141017626525 74.34878173471911) bank127936 +127937 POINT(41.02988503699147 74.19359266449437) bank127937 +127938 POINT(40.98396214552805 74.24866665606959) bank127938 +127939 POINT(39.748846803106574 74.81257813350972) bank127939 +127940 POINT(41.178930690561245 73.48320652128093) bank127940 +127941 POINT(40.288312177874154 73.29068703551901) bank127941 +127942 POINT(40.95727082692152 73.66749171540252) bank127942 +127943 POINT(41.418939787546016 74.78816557188323) bank127943 +127944 POINT(39.78151120790727 74.88028354244963) bank127944 +127945 POINT(41.04707470841487 73.71924223741432) bank127945 +127946 POINT(41.68455513107857 74.38044212405478) bank127946 +127947 POINT(40.98641705630712 74.45854574760888) bank127947 +127948 POINT(40.89455922338201 73.84495470205344) bank127948 +127949 POINT(40.09390454900684 73.66462384319236) bank127949 +127950 POINT(40.83239805190607 74.66383166881022) bank127950 +127951 POINT(40.72636040055223 74.00040299434013) bank127951 +127952 POINT(41.01566027201784 74.1260641792436) bank127952 +127953 POINT(40.45346432618001 74.62080887760901) bank127953 +127954 POINT(40.66052868075017 73.13971751465107) bank127954 +127955 POINT(40.25498011462079 73.6825933035591) bank127955 +127956 POINT(39.779461105823756 74.85287577025859) bank127956 +127957 POINT(41.05111631350843 74.32387570908965) bank127957 +127958 POINT(39.932390727626846 74.39918182438394) bank127958 +127959 POINT(40.48824002890487 73.26381605344969) bank127959 +127960 POINT(41.1234845745945 74.52582163998571) bank127960 +127961 POINT(41.027677499247474 73.90410165467799) bank127961 +127962 POINT(41.67671884655465 74.62174402781977) bank127962 +127963 POINT(41.531683795735916 73.93658219727897) bank127963 +127964 POINT(39.927168681138234 73.13127151488918) bank127964 +127965 POINT(40.720972029277235 73.25381028188745) bank127965 +127966 POINT(41.2701633768959 74.57240050093185) bank127966 +127967 POINT(41.66709575632115 73.85635884810803) bank127967 +127968 POINT(40.89511713714175 74.82706057672101) bank127968 +127969 POINT(40.899349967864794 73.80721374302314) bank127969 +127970 POINT(40.971028764644515 73.83505809970084) bank127970 +127971 POINT(40.96254971732943 73.72319162988579) bank127971 +127972 POINT(41.15977354061197 73.54907668167117) bank127972 +127973 POINT(40.132315497497416 73.34115727197329) bank127973 +127974 POINT(41.6199855139125 74.01160224085612) bank127974 +127975 POINT(40.44279600770756 73.53112825523449) bank127975 +127976 POINT(41.305999962083064 74.46241478349786) bank127976 +127977 POINT(40.8066844308719 74.39496578065237) bank127977 +127978 POINT(40.84125224201245 73.78594088735285) bank127978 +127979 POINT(40.0493350150415 74.65649934971475) bank127979 +127980 POINT(41.70065297344255 73.67283107252017) bank127980 +127981 POINT(40.053541332224555 74.96042652392725) bank127981 +127982 POINT(40.065012844718005 74.73010477002363) bank127982 +127983 POINT(40.453928890102915 74.11823728260117) bank127983 +127984 POINT(41.60643113998103 74.44475773279657) bank127984 +127985 POINT(40.37337289498666 73.03809479491042) bank127985 +127986 POINT(40.915180158632964 74.14606759894714) bank127986 +127987 POINT(40.92108718423198 74.93740404502712) bank127987 +127988 POINT(40.40328198962672 74.36738297339177) bank127988 +127989 POINT(41.290524393604464 73.37777133192985) bank127989 +127990 POINT(40.4920914018436 73.23340661411639) bank127990 +127991 POINT(39.910502028690644 73.79391529426897) bank127991 +127992 POINT(39.99258251475695 73.97748767325439) bank127992 +127993 POINT(40.44304180799489 73.92396344775771) bank127993 +127994 POINT(40.0166598563755 74.49217357565794) bank127994 +127995 POINT(39.94256276336473 74.25178556913069) bank127995 +127996 POINT(40.19656498836496 73.63403695516925) bank127996 +127997 POINT(40.67562035121597 73.77162527646603) bank127997 +127998 POINT(41.416976055383564 73.66094775317427) bank127998 +127999 POINT(41.6304604117221 73.19884995778031) bank127999 +128000 POINT(41.612186132382206 74.11032421680679) bank128000 +128001 POINT(40.5575189755902 74.24509322192604) bank128001 +128002 POINT(39.95285939793989 73.19338613581299) bank128002 +128003 POINT(39.7875529543616 74.71811316199452) bank128003 +128004 POINT(41.236424551319885 73.05244658424408) bank128004 +128005 POINT(40.50742461983816 74.34089406511544) bank128005 +128006 POINT(41.164307552796394 73.64548308266448) bank128006 +128007 POINT(39.84792664375745 73.10287801953105) bank128007 +128008 POINT(40.00685507785394 73.64257692861237) bank128008 +128009 POINT(40.980988841027894 74.65940537933724) bank128009 +128010 POINT(41.49862533124358 73.783985360676) bank128010 +128011 POINT(40.56601385177469 74.4994547469766) bank128011 +128012 POINT(40.53027250093551 74.6198332175877) bank128012 +128013 POINT(41.02716291525465 73.51712124897473) bank128013 +128014 POINT(39.9883290299573 74.77862552216985) bank128014 +128015 POINT(40.78706090500691 74.92054723742503) bank128015 +128016 POINT(39.83054964888322 73.5645526945375) bank128016 +128017 POINT(41.04445048861287 73.04702095541643) bank128017 +128018 POINT(41.7055211558048 73.48927471559311) bank128018 +128019 POINT(41.30444046712546 73.95136630358935) bank128019 +128020 POINT(40.18641519424639 73.18878244587587) bank128020 +128021 POINT(40.4319388575353 74.37085539795328) bank128021 +128022 POINT(40.79743316225001 74.03054290375633) bank128022 +128023 POINT(40.16861357377896 74.06386802810418) bank128023 +128024 POINT(40.68947453352182 74.61523664152811) bank128024 +128025 POINT(40.25954651439194 73.53407299720274) bank128025 +128026 POINT(39.77659561037649 73.77040786119176) bank128026 +128027 POINT(39.774320564965834 74.77224121866843) bank128027 +128028 POINT(41.507171757084365 74.10839308150483) bank128028 +128029 POINT(40.70138264573611 73.92351042830569) bank128029 +128030 POINT(40.05825259396391 74.29679837876165) bank128030 +128031 POINT(40.44796525380963 73.89106467759007) bank128031 +128032 POINT(41.47624122966 74.29145582130327) bank128032 +128033 POINT(40.326854812188394 74.57105805210888) bank128033 +128034 POINT(40.57848606766768 73.28861583846626) bank128034 +128035 POINT(39.83590987880125 74.33418434455778) bank128035 +128036 POINT(40.24548300455839 73.7332169717097) bank128036 +128037 POINT(39.80535184636403 74.48850936283421) bank128037 +128038 POINT(41.16939397496706 74.99946538887477) bank128038 +128039 POINT(40.61101294758896 74.03096260243319) bank128039 +128040 POINT(39.77229225467358 73.30988882086626) bank128040 +128041 POINT(41.11241176617263 73.75430352366956) bank128041 +128042 POINT(40.36478456887201 73.4175637505807) bank128042 +128043 POINT(41.36070545654802 73.36081186688136) bank128043 +128044 POINT(40.6813203847014 73.20432806372786) bank128044 +128045 POINT(41.292585596161864 73.25676532620993) bank128045 +128046 POINT(41.55251485348218 73.5194827170955) bank128046 +128047 POINT(41.447061741885804 74.42958032050159) bank128047 +128048 POINT(41.360607152429075 73.87797185216554) bank128048 +128049 POINT(40.92822612400269 74.68841984245856) bank128049 +128050 POINT(40.97982598692903 73.54475701140686) bank128050 +128051 POINT(40.340792610148185 73.33052231296588) bank128051 +128052 POINT(39.72906009625986 74.87382180998105) bank128052 +128053 POINT(41.48623404732512 73.2041378853971) bank128053 +128054 POINT(41.074830788223984 73.57192252448476) bank128054 +128055 POINT(41.504343599617656 73.93244093406508) bank128055 +128056 POINT(41.50586678565974 73.45033495229809) bank128056 +128057 POINT(41.54146811507196 74.78191110861813) bank128057 +128058 POINT(40.639032966010944 74.95540612235874) bank128058 +128059 POINT(40.387356825383314 74.52008421798197) bank128059 +128060 POINT(41.4866641834247 73.55045354248124) bank128060 +128061 POINT(41.1516960784541 74.9848887303255) bank128061 +128062 POINT(41.01371631201254 74.18774212867118) bank128062 +128063 POINT(41.283388564528536 74.41391416973424) bank128063 +128064 POINT(41.10292148877791 74.27086030266635) bank128064 +128065 POINT(40.60582033295496 73.05918351101924) bank128065 +128066 POINT(41.47077150268272 74.79537812736682) bank128066 +128067 POINT(40.51820740279719 74.48385642214011) bank128067 +128068 POINT(39.86526854991596 74.95425155686321) bank128068 +128069 POINT(40.36013134226114 74.93642372476224) bank128069 +128070 POINT(40.464780561574116 74.15323800567964) bank128070 +128071 POINT(40.820476401116274 74.63334214176476) bank128071 +128072 POINT(40.16704790463646 74.289669189191) bank128072 +128073 POINT(40.80899455642391 73.36026865051375) bank128073 +128074 POINT(41.00479876211314 74.9702222096179) bank128074 +128075 POINT(40.92113326390543 74.05433465530889) bank128075 +128076 POINT(40.92318553183039 73.2813005520166) bank128076 +128077 POINT(40.578709041236245 74.2517837400482) bank128077 +128078 POINT(40.17427430852943 73.61251840517849) bank128078 +128079 POINT(40.2806567325473 73.26281642578587) bank128079 +128080 POINT(40.57810082782277 73.13113987765425) bank128080 +128081 POINT(40.13977976332486 74.16841072754701) bank128081 +128082 POINT(41.27369115747181 74.59515239033081) bank128082 +128083 POINT(41.18288388388919 74.81755792388721) bank128083 +128084 POINT(41.20295028720367 74.02199774277116) bank128084 +128085 POINT(40.24935197051438 74.5931576759312) bank128085 +128086 POINT(40.62424567426116 73.26851916747222) bank128086 +128087 POINT(41.49678555018932 73.31897166810492) bank128087 +128088 POINT(41.03093202550037 74.93434875640118) bank128088 +128089 POINT(41.259086214274376 74.63582116366602) bank128089 +128090 POINT(40.47954207228246 74.780584864401) bank128090 +128091 POINT(40.90447850522542 74.94850067745742) bank128091 +128092 POINT(40.254407423069196 74.5336437858515) bank128092 +128093 POINT(41.22278359490851 73.23126272051918) bank128093 +128094 POINT(40.75768355519452 73.64057595181797) bank128094 +128095 POINT(40.74861894755619 73.94617838297579) bank128095 +128096 POINT(41.142424061798266 73.50770463407258) bank128096 +128097 POINT(40.89721394000874 74.97918948376828) bank128097 +128098 POINT(40.4159184557711 73.46316848429444) bank128098 +128099 POINT(39.942986710381376 73.59069626950638) bank128099 +128100 POINT(40.77166304419168 73.69527698673059) bank128100 +128101 POINT(40.80657452288407 73.293399162517) bank128101 +128102 POINT(39.84320383354936 74.27123552775191) bank128102 +128103 POINT(41.58918825646203 73.51024763513384) bank128103 +128104 POINT(39.9369801599893 73.08244634230876) bank128104 +128105 POINT(40.57284226314371 73.93720741475364) bank128105 +128106 POINT(40.886039290515185 74.73186625926901) bank128106 +128107 POINT(41.32149819245527 73.56082679089766) bank128107 +128108 POINT(41.02724430633722 73.4364471038143) bank128108 +128109 POINT(40.91622792430136 73.38274500764444) bank128109 +128110 POINT(40.26593031619109 73.1014716511143) bank128110 +128111 POINT(40.49836315356616 73.7474367240977) bank128111 +128112 POINT(41.50117424487754 74.53954879010928) bank128112 +128113 POINT(39.87981274034727 73.53306295353762) bank128113 +128114 POINT(41.16974948935054 73.68321546812251) bank128114 +128115 POINT(39.81775976945858 74.47141263980326) bank128115 +128116 POINT(41.43922297660943 73.30199489217678) bank128116 +128117 POINT(41.201492865556254 74.81717734860548) bank128117 +128118 POINT(40.46751566855925 73.42992900031643) bank128118 +128119 POINT(39.90030225618054 74.02366667198606) bank128119 +128120 POINT(40.98214122910469 74.22438449871352) bank128120 +128121 POINT(40.66613545372842 74.83708761335033) bank128121 +128122 POINT(40.55662006420784 73.36189024134259) bank128122 +128123 POINT(40.57081357438734 73.30116547010034) bank128123 +128124 POINT(41.48378944924899 74.15754960745942) bank128124 +128125 POINT(40.994661755504715 73.7660262827635) bank128125 +128126 POINT(40.63794888176812 73.87280290610124) bank128126 +128127 POINT(41.43510013417044 74.68448127647682) bank128127 +128128 POINT(40.918177294999914 73.39497331611189) bank128128 +128129 POINT(40.78884001002375 74.58815278232919) bank128129 +128130 POINT(40.89702409101205 74.46990383317436) bank128130 +128131 POINT(41.48501955735936 74.05183380963001) bank128131 +128132 POINT(40.44717415923264 74.4404294202347) bank128132 +128133 POINT(41.25547198127866 73.90453572777407) bank128133 +128134 POINT(39.967670844331124 73.63615755999693) bank128134 +128135 POINT(41.69651243273535 74.00606328776645) bank128135 +128136 POINT(40.757854387889545 73.887362813522) bank128136 +128137 POINT(41.26942503499083 74.20062358883035) bank128137 +128138 POINT(41.149882243197794 73.2879515403944) bank128138 +128139 POINT(41.27725387151216 73.71110674070702) bank128139 +128140 POINT(40.18320551771705 73.40088106400583) bank128140 +128141 POINT(41.597589198394985 73.20419310137476) bank128141 +128142 POINT(41.29199806820996 74.429360472544) bank128142 +128143 POINT(41.25573697789536 73.06321248655475) bank128143 +128144 POINT(40.36843422068503 74.8954397041719) bank128144 +128145 POINT(41.19341360180325 73.94433863337692) bank128145 +128146 POINT(41.28109437827571 73.80996876079537) bank128146 +128147 POINT(41.47401590150727 74.87863410318717) bank128147 +128148 POINT(41.43497576480281 73.704783351348) bank128148 +128149 POINT(40.338581079705115 74.60945606079062) bank128149 +128150 POINT(40.66946672030285 73.39235960428361) bank128150 +128151 POINT(40.75420350588621 73.03828559353876) bank128151 +128152 POINT(40.96957081240922 74.52658038055019) bank128152 +128153 POINT(40.059898967599075 74.5493695634765) bank128153 +128154 POINT(41.4732491160574 74.19532827010467) bank128154 +128155 POINT(40.64077933582688 74.1531104625487) bank128155 +128156 POINT(40.08471900646095 73.93814363167671) bank128156 +128157 POINT(40.68212451027614 73.61100673407879) bank128157 +128158 POINT(39.87471232649593 73.12315284696075) bank128158 +128159 POINT(40.11168869437975 74.09874841350593) bank128159 +128160 POINT(41.149508154280205 74.04494253665278) bank128160 +128161 POINT(41.2790318113704 73.79162354332573) bank128161 +128162 POINT(39.93174674960786 73.64182245858166) bank128162 +128163 POINT(41.30025729941647 74.61058315047265) bank128163 +128164 POINT(40.647869813301675 73.66095033675593) bank128164 +128165 POINT(40.168943619981704 74.11633608709512) bank128165 +128166 POINT(39.78152124046264 74.92588684689561) bank128166 +128167 POINT(40.620646974790084 73.34345966105151) bank128167 +128168 POINT(41.30107559655927 74.45494415629065) bank128168 +128169 POINT(40.680159235149624 73.42516381702238) bank128169 +128170 POINT(41.23468922750577 73.67539379897723) bank128170 +128171 POINT(40.07710850767724 74.46596660526005) bank128171 +128172 POINT(40.771612604161604 74.565236474606) bank128172 +128173 POINT(40.571633264203825 74.66300706879565) bank128173 +128174 POINT(40.568463733921746 73.60812676264348) bank128174 +128175 POINT(40.29827407420025 74.57827959124793) bank128175 +128176 POINT(39.753133147264876 74.60600549043654) bank128176 +128177 POINT(40.596056199368476 73.44027367994619) bank128177 +128178 POINT(40.95455486961853 73.90267701890588) bank128178 +128179 POINT(39.76146394070806 74.42443698845092) bank128179 +128180 POINT(40.790163468997505 74.26995725954434) bank128180 +128181 POINT(41.40012333776836 73.51226381065104) bank128181 +128182 POINT(40.390382531693774 73.29137108734089) bank128182 +128183 POINT(40.065400512095216 74.58638957797486) bank128183 +128184 POINT(40.9527640916751 73.72055975579931) bank128184 +128185 POINT(39.81392566507396 73.35419364696767) bank128185 +128186 POINT(41.524605152746666 73.68359270682556) bank128186 +128187 POINT(40.32585959327774 74.25436408622157) bank128187 +128188 POINT(40.70858496146514 73.2771675162785) bank128188 +128189 POINT(40.443864119453465 74.34949360086067) bank128189 +128190 POINT(41.03518268102525 73.67702468975128) bank128190 +128191 POINT(40.7516408193986 74.32818760657332) bank128191 +128192 POINT(39.852542946315495 74.63662478061472) bank128192 +128193 POINT(40.20399451697215 73.51094034361174) bank128193 +128194 POINT(40.17733518364018 73.72710531563726) bank128194 +128195 POINT(41.57496395363052 73.61901464663023) bank128195 +128196 POINT(40.893948212004105 73.21897596946964) bank128196 +128197 POINT(39.936710161220105 74.82293235429565) bank128197 +128198 POINT(41.0684803278716 74.45724542658624) bank128198 +128199 POINT(41.68409510051361 74.51076090848892) bank128199 +128200 POINT(40.541296490547865 73.38010590137648) bank128200 +128201 POINT(41.64904913149747 74.00014431733345) bank128201 +128202 POINT(40.84674029663527 73.30619228183554) bank128202 +128203 POINT(40.64210760670958 73.80944244626329) bank128203 +128204 POINT(39.7175715000379 73.21467065099407) bank128204 +128205 POINT(40.17954888572075 74.45956717009312) bank128205 +128206 POINT(41.30001381186959 74.84068627888384) bank128206 +128207 POINT(40.506939135314 73.55287487425196) bank128207 +128208 POINT(40.86399373116522 74.74474949142586) bank128208 +128209 POINT(40.42999249768528 74.02540802340314) bank128209 +128210 POINT(41.5229910683702 73.04483930184078) bank128210 +128211 POINT(41.46129442627425 73.52329260194867) bank128211 +128212 POINT(40.06081941313367 74.23361128741959) bank128212 +128213 POINT(40.18914798700366 74.70207570309344) bank128213 +128214 POINT(41.09394579356701 73.51702880170016) bank128214 +128215 POINT(39.84086023139298 73.06504137561818) bank128215 +128216 POINT(41.59000115700707 73.00853536608045) bank128216 +128217 POINT(41.29011323361372 73.62496341479834) bank128217 +128218 POINT(41.68683326329368 74.88628604209572) bank128218 +128219 POINT(40.593395715431654 73.04828299509215) bank128219 +128220 POINT(41.61705363939181 73.50038922324751) bank128220 +128221 POINT(39.733952607267575 74.13041502881735) bank128221 +128222 POINT(40.59973593842338 74.04097957283835) bank128222 +128223 POINT(41.5350768110214 73.27872532104675) bank128223 +128224 POINT(41.4327990043241 73.16381569355757) bank128224 +128225 POINT(39.83657878352483 73.76132994090669) bank128225 +128226 POINT(39.93647992630147 73.1820570480773) bank128226 +128227 POINT(40.385716203963426 73.89220551281326) bank128227 +128228 POINT(40.19566670537761 74.52009835078326) bank128228 +128229 POINT(41.707394186714495 73.49046469575565) bank128229 +128230 POINT(41.381054932469425 74.4058781150892) bank128230 +128231 POINT(40.83754305804652 74.13710268990378) bank128231 +128232 POINT(40.26300251549809 74.76629734900352) bank128232 +128233 POINT(41.69449357493935 74.25536621395074) bank128233 +128234 POINT(40.82657701130293 74.76488278784402) bank128234 +128235 POINT(41.16883790988007 74.38089580068457) bank128235 +128236 POINT(40.49236625699236 74.87285047749563) bank128236 +128237 POINT(40.85566085242595 73.86549190007088) bank128237 +128238 POINT(40.06801403306004 73.14734693926388) bank128238 +128239 POINT(39.72651710244586 74.20271619779295) bank128239 +128240 POINT(41.69783376897404 73.47417010388588) bank128240 +128241 POINT(40.053099456889846 73.63441135920465) bank128241 +128242 POINT(39.80624268152688 74.79482209126647) bank128242 +128243 POINT(41.66357027325739 74.03043087396003) bank128243 +128244 POINT(40.60839514483803 73.18340479481718) bank128244 +128245 POINT(41.531705864244785 74.27601231045159) bank128245 +128246 POINT(41.18444211286953 73.47454223381112) bank128246 +128247 POINT(40.27873977131088 74.27158504180109) bank128247 +128248 POINT(40.856177545015555 74.04551500164828) bank128248 +128249 POINT(39.78795480534659 74.74076663930992) bank128249 +128250 POINT(40.298145225885754 74.35869828112065) bank128250 +128251 POINT(39.88802886216268 73.5610154932167) bank128251 +128252 POINT(41.40482489096819 73.87355852050489) bank128252 +128253 POINT(41.252844274868856 74.70497255523719) bank128253 +128254 POINT(40.213861471420486 74.86599768898708) bank128254 +128255 POINT(39.8628524028603 74.12724047018351) bank128255 +128256 POINT(40.180305181752864 73.4968337426619) bank128256 +128257 POINT(40.980362645671555 74.28600375734183) bank128257 +128258 POINT(41.58275144060543 73.33203189283805) bank128258 +128259 POINT(40.66048219782291 73.54446922249399) bank128259 +128260 POINT(40.03839432955521 73.81432397533312) bank128260 +128261 POINT(40.2736795181376 73.87087869494219) bank128261 +128262 POINT(41.115395001148975 73.63547973058174) bank128262 +128263 POINT(40.99777260035438 74.1854497266986) bank128263 +128264 POINT(40.091542485798705 74.71090964878245) bank128264 +128265 POINT(40.80024320713479 73.65667186829054) bank128265 +128266 POINT(41.36848857691943 74.77126938300214) bank128266 +128267 POINT(40.117391405262715 73.67445835400008) bank128267 +128268 POINT(39.86175242795823 73.99936680977264) bank128268 +128269 POINT(40.2366696939582 74.70391446583656) bank128269 +128270 POINT(40.811943781135064 73.63977039387626) bank128270 +128271 POINT(41.18449477522837 74.2863190101049) bank128271 +128272 POINT(40.34679242965576 73.78694136993334) bank128272 +128273 POINT(41.66213544454553 73.1056277633502) bank128273 +128274 POINT(40.13465271350899 73.97489887630825) bank128274 +128275 POINT(41.63638332832678 73.16894602704531) bank128275 +128276 POINT(40.19323392178258 73.69900931555053) bank128276 +128277 POINT(40.62877007556 74.82281905394515) bank128277 +128278 POINT(40.684504913940316 74.23797184771365) bank128278 +128279 POINT(40.42100392883506 74.18659234837769) bank128279 +128280 POINT(39.784892495221186 73.15031434271842) bank128280 +128281 POINT(40.9308256446613 73.84281006235507) bank128281 +128282 POINT(39.78132109617008 73.08787244740478) bank128282 +128283 POINT(41.3556666770569 74.67950207183256) bank128283 +128284 POINT(40.91978388275125 73.9746614302927) bank128284 +128285 POINT(39.80939898713364 73.74452340511245) bank128285 +128286 POINT(40.30272932354793 73.42488605486066) bank128286 +128287 POINT(40.24554483463747 74.53388951431879) bank128287 +128288 POINT(40.75531511174784 73.00600881640734) bank128288 +128289 POINT(41.66490297331564 73.86421468945694) bank128289 +128290 POINT(41.48158071367207 74.21418340884647) bank128290 +128291 POINT(40.787050885125815 74.19012441498816) bank128291 +128292 POINT(41.06910289090429 73.78465657662647) bank128292 +128293 POINT(40.089524958853985 73.22159619951346) bank128293 +128294 POINT(41.49350100570626 74.78798641390493) bank128294 +128295 POINT(40.56425320203264 74.16560936023255) bank128295 +128296 POINT(40.51130359994583 74.77320529001035) bank128296 +128297 POINT(41.201274506000544 74.6083520515892) bank128297 +128298 POINT(40.81694692155331 74.64274352595616) bank128298 +128299 POINT(40.179866800489975 73.22444118854078) bank128299 +128300 POINT(41.51983778232269 73.22642931037214) bank128300 +128301 POINT(40.18442814672915 73.82162302695242) bank128301 +128302 POINT(41.09536892862718 74.50820881557573) bank128302 +128303 POINT(41.335169051769086 74.4752900198723) bank128303 +128304 POINT(40.30384175023276 73.03119046471045) bank128304 +128305 POINT(39.93489581810024 74.73007643408786) bank128305 +128306 POINT(40.77713726884289 74.36521452627346) bank128306 +128307 POINT(40.82014185266848 74.34817958174315) bank128307 +128308 POINT(39.892024741412385 73.55337789056263) bank128308 +128309 POINT(39.95620276891691 73.71675176531878) bank128309 +128310 POINT(41.17202063526176 74.5392186147572) bank128310 +128311 POINT(39.86222304668582 74.19801643958358) bank128311 +128312 POINT(40.137309145952976 74.04675869759063) bank128312 +128313 POINT(41.02767842316675 74.05537036203603) bank128313 +128314 POINT(40.005444862184845 73.81233628578694) bank128314 +128315 POINT(40.58117702814746 73.48348886352352) bank128315 +128316 POINT(40.402873383021536 74.61225992089037) bank128316 +128317 POINT(40.459683196394664 73.54528094701331) bank128317 +128318 POINT(40.385911889260655 74.23222669912207) bank128318 +128319 POINT(40.76837170468764 74.79124500568876) bank128319 +128320 POINT(41.33158804270347 74.70925455525293) bank128320 +128321 POINT(40.9685902110183 74.62205051171685) bank128321 +128322 POINT(40.98946482126442 73.99810335548243) bank128322 +128323 POINT(40.42357683704977 74.9530885508732) bank128323 +128324 POINT(40.911309407145296 73.20212966973035) bank128324 +128325 POINT(40.23304020474882 74.25070203364234) bank128325 +128326 POINT(40.659613917597326 74.16128396947079) bank128326 +128327 POINT(41.21346227139837 73.87193917741014) bank128327 +128328 POINT(40.54863106058069 73.42288528880368) bank128328 +128329 POINT(41.55942148624102 74.20639222303059) bank128329 +128330 POINT(41.57286539380567 73.40531275965816) bank128330 +128331 POINT(40.798958110798495 74.86695593032239) bank128331 +128332 POINT(40.68151504243685 73.0427428853846) bank128332 +128333 POINT(41.23708606308183 74.48843932938978) bank128333 +128334 POINT(41.699183449218765 73.29929628302435) bank128334 +128335 POINT(41.20516637658716 74.1885661180886) bank128335 +128336 POINT(39.88659260398509 73.69360809734505) bank128336 +128337 POINT(40.32199446708869 74.76564855049477) bank128337 +128338 POINT(40.594834418733264 73.01794793157978) bank128338 +128339 POINT(40.01116807398692 74.78759711495837) bank128339 +128340 POINT(41.64379854969166 74.4158668071851) bank128340 +128341 POINT(40.069324395951625 73.08579996827564) bank128341 +128342 POINT(41.60802640595385 74.65115529806208) bank128342 +128343 POINT(41.56779272701887 74.13367659165434) bank128343 +128344 POINT(40.94436161275564 74.78656694874783) bank128344 +128345 POINT(40.189054719565966 73.06231641785322) bank128345 +128346 POINT(39.773612075869494 74.41101203181006) bank128346 +128347 POINT(41.37342993748826 74.93187307074254) bank128347 +128348 POINT(40.22650924930085 73.42018172071614) bank128348 +128349 POINT(40.7497561122016 73.7280519451478) bank128349 +128350 POINT(40.17417784715942 73.1912524979475) bank128350 +128351 POINT(41.45716496170105 74.58021362908902) bank128351 +128352 POINT(40.590754550536474 73.76159000829742) bank128352 +128353 POINT(40.19353654956397 73.70988435108605) bank128353 +128354 POINT(41.174538743314706 73.14832015349283) bank128354 +128355 POINT(40.16627850512272 73.96917124923526) bank128355 +128356 POINT(39.716554437999015 74.99320733649085) bank128356 +128357 POINT(40.14105128010046 73.08254310415818) bank128357 +128358 POINT(40.84633428972802 73.6546231879746) bank128358 +128359 POINT(40.32847757550118 74.70795826676667) bank128359 +128360 POINT(41.69837176614752 74.96880961104007) bank128360 +128361 POINT(40.71720540909878 73.73612239805517) bank128361 +128362 POINT(41.20090575555686 73.86111622945117) bank128362 +128363 POINT(40.87905162639636 73.66180518244437) bank128363 +128364 POINT(41.37528573623966 73.30950596869364) bank128364 +128365 POINT(40.07866323170321 74.46800129170667) bank128365 +128366 POINT(40.8972818902761 73.89249148411936) bank128366 +128367 POINT(40.01505959261379 73.23197578852124) bank128367 +128368 POINT(40.528303707571716 74.64001643446845) bank128368 +128369 POINT(41.630184845274904 74.83644959436538) bank128369 +128370 POINT(41.142046326045275 74.0232610314789) bank128370 +128371 POINT(40.64892485331 73.79562213279145) bank128371 +128372 POINT(40.953741277393156 74.41078344259621) bank128372 +128373 POINT(39.94731267604358 73.53737979383104) bank128373 +128374 POINT(40.774989905856636 73.35419613073567) bank128374 +128375 POINT(41.677105036612375 74.58058811633792) bank128375 +128376 POINT(39.74709997040126 74.01010200969242) bank128376 +128377 POINT(40.46183706016428 73.63830042627144) bank128377 +128378 POINT(40.050782717716 73.62128132600836) bank128378 +128379 POINT(39.77948608842206 73.4890913703204) bank128379 +128380 POINT(41.417795278138456 73.1104192554239) bank128380 +128381 POINT(40.830680721015646 73.3289897945535) bank128381 +128382 POINT(41.45797023078948 73.80491018836459) bank128382 +128383 POINT(40.72388432838622 73.45244712177687) bank128383 +128384 POINT(41.27160204881454 74.76146230717634) bank128384 +128385 POINT(39.87078078427127 73.14443504066885) bank128385 +128386 POINT(40.886717669694676 73.60374994991972) bank128386 +128387 POINT(40.217452866671714 74.12452285002905) bank128387 +128388 POINT(41.25111049270881 74.65680094720709) bank128388 +128389 POINT(41.17285757210548 73.70449845249013) bank128389 +128390 POINT(40.723820664927636 73.25018618369123) bank128390 +128391 POINT(40.93143042322276 73.35714354443002) bank128391 +128392 POINT(40.729826644671775 74.7051667329753) bank128392 +128393 POINT(40.284459201824056 74.30491055060881) bank128393 +128394 POINT(41.449459135081206 73.00645049014891) bank128394 +128395 POINT(40.690357480973226 73.05223538483459) bank128395 +128396 POINT(40.549471449592446 74.3554290159186) bank128396 +128397 POINT(41.50473743628858 74.72897608525189) bank128397 +128398 POINT(39.97515997911766 74.38585564809856) bank128398 +128399 POINT(40.02155029955328 73.17595219137309) bank128399 +128400 POINT(40.86555510353581 73.519247801028) bank128400 +128401 POINT(39.76644877104447 73.93670743698648) bank128401 +128402 POINT(40.364067484071775 73.73501315389983) bank128402 +128403 POINT(41.50474515040811 73.68205099278816) bank128403 +128404 POINT(40.041384421008665 73.668914775107) bank128404 +128405 POINT(40.789008150341346 73.08171652297084) bank128405 +128406 POINT(40.52626727036077 73.06952183299578) bank128406 +128407 POINT(40.40954723488429 74.24078340204632) bank128407 +128408 POINT(40.71742722763006 73.10455805759774) bank128408 +128409 POINT(39.914261600551995 74.08932937993568) bank128409 +128410 POINT(41.20951325292478 73.35781572536614) bank128410 +128411 POINT(41.58439090377544 73.46086970754752) bank128411 +128412 POINT(40.97687920268253 73.49640114759337) bank128412 +128413 POINT(39.863019509648595 74.05991132325721) bank128413 +128414 POINT(40.403528113123954 73.81415203770871) bank128414 +128415 POINT(40.32442661159181 73.16712663911264) bank128415 +128416 POINT(41.04944309118238 74.426164385884) bank128416 +128417 POINT(39.81862446879338 73.02715432636194) bank128417 +128418 POINT(40.97737268284593 74.00333287657088) bank128418 +128419 POINT(40.176034501451866 73.36850198014608) bank128419 +128420 POINT(41.338633104154106 74.70769208629815) bank128420 +128421 POINT(41.42955236779462 74.03208602055324) bank128421 +128422 POINT(40.354195355252415 74.61424530351236) bank128422 +128423 POINT(39.89462977783149 74.62290332266645) bank128423 +128424 POINT(40.590942156391016 73.79835686086506) bank128424 +128425 POINT(41.354798623986014 73.54994596289991) bank128425 +128426 POINT(40.986173890324125 74.5795681150798) bank128426 +128427 POINT(39.902103025945316 74.0005884474619) bank128427 +128428 POINT(41.64662857565003 74.71096552158585) bank128428 +128429 POINT(40.97250734231193 74.71510905219442) bank128429 +128430 POINT(39.804676381675435 74.91427020867107) bank128430 +128431 POINT(39.91296657934397 73.38848124265306) bank128431 +128432 POINT(40.83549264166799 73.38802382744484) bank128432 +128433 POINT(41.475142067047116 73.47671877405757) bank128433 +128434 POINT(40.518622022828815 73.66072657872202) bank128434 +128435 POINT(40.58686771721948 73.650463056649) bank128435 +128436 POINT(40.34991037656398 74.80635551346711) bank128436 +128437 POINT(40.594549938018055 73.05115674569346) bank128437 +128438 POINT(40.74368191166546 73.73497823475091) bank128438 +128439 POINT(41.157792017162706 74.83195734353892) bank128439 +128440 POINT(39.954308732544156 73.44781710843934) bank128440 +128441 POINT(40.70886229293245 74.36513974432847) bank128441 +128442 POINT(40.133804023770765 74.60793670885126) bank128442 +128443 POINT(40.97829859537567 73.593404006197) bank128443 +128444 POINT(40.339897828736966 73.8284568660694) bank128444 +128445 POINT(39.765183067574334 74.35276565816048) bank128445 +128446 POINT(41.39328129570614 73.90686179308497) bank128446 +128447 POINT(39.84921131358459 73.83815168761497) bank128447 +128448 POINT(40.117650838515324 73.509568825793) bank128448 +128449 POINT(39.948187421862464 74.5084404702338) bank128449 +128450 POINT(39.92553509588094 73.5788632308421) bank128450 +128451 POINT(41.24198540662375 74.72573094863148) bank128451 +128452 POINT(40.62077740656946 74.79149091884292) bank128452 +128453 POINT(40.86744947592845 73.67511646482039) bank128453 +128454 POINT(40.68930999427599 74.06437440246036) bank128454 +128455 POINT(40.444896715046156 74.3076780354739) bank128455 +128456 POINT(40.75301853049147 73.73165104230542) bank128456 +128457 POINT(41.22175639213273 73.7637297591806) bank128457 +128458 POINT(40.86904570031186 73.54562729773795) bank128458 +128459 POINT(40.90516032886874 74.01095926640669) bank128459 +128460 POINT(41.15429219693014 74.11336544789287) bank128460 +128461 POINT(39.81030033919179 74.44939000352693) bank128461 +128462 POINT(40.48107763970224 74.37242499302647) bank128462 +128463 POINT(40.386361579087826 75.00067231159342) bank128463 +128464 POINT(40.1481444987942 74.66047923784451) bank128464 +128465 POINT(40.6600243788521 73.78970935119713) bank128465 +128466 POINT(41.43286704527935 74.32715645090516) bank128466 +128467 POINT(39.74318906837385 74.89844297661324) bank128467 +128468 POINT(39.96489093154413 73.44092865528337) bank128468 +128469 POINT(40.73888920015446 74.8123617866765) bank128469 +128470 POINT(40.67057156470527 73.12984943333379) bank128470 +128471 POINT(40.195401771220546 74.28838328521013) bank128471 +128472 POINT(39.997460847338346 73.01456398988084) bank128472 +128473 POINT(41.484842710396094 73.32269197717503) bank128473 +128474 POINT(40.732481029718514 73.22801852401957) bank128474 +128475 POINT(41.536955778913864 73.31871557468632) bank128475 +128476 POINT(41.10370267657557 73.41574096546728) bank128476 +128477 POINT(40.792009995715034 74.06108200009052) bank128477 +128478 POINT(40.52619668977788 74.76076467611601) bank128478 +128479 POINT(40.0129895557086 73.28785351447593) bank128479 +128480 POINT(40.654931925820826 73.46485226625751) bank128480 +128481 POINT(40.73924782156222 73.15019484588174) bank128481 +128482 POINT(39.871009086753716 74.15206245909623) bank128482 +128483 POINT(40.280994994222105 73.94851029115136) bank128483 +128484 POINT(39.754331884194116 73.73230162122648) bank128484 +128485 POINT(40.702246592084705 74.23478836463396) bank128485 +128486 POINT(40.01881761733254 73.77222647056391) bank128486 +128487 POINT(41.05252655736851 73.88757822765183) bank128487 +128488 POINT(39.87336380376697 74.114900731473) bank128488 +128489 POINT(40.35836682738017 73.30502121284587) bank128489 +128490 POINT(40.13942336307835 73.73492272759427) bank128490 +128491 POINT(41.508387810013545 73.68035814074439) bank128491 +128492 POINT(39.99316653085342 74.21735896106767) bank128492 +128493 POINT(40.4321450235799 74.25630711702959) bank128493 +128494 POINT(39.91723122243283 74.29011862478947) bank128494 +128495 POINT(41.035807361218616 73.2425052091338) bank128495 +128496 POINT(40.2127787739251 74.46065940294132) bank128496 +128497 POINT(39.75902242431972 74.12361519851031) bank128497 +128498 POINT(40.26930417728923 73.60492474176897) bank128498 +128499 POINT(41.477314829276864 74.65891120018378) bank128499 +128500 POINT(40.56487946988933 74.7940287824153) bank128500 +128501 POINT(40.27620167667969 73.34504606769768) bank128501 +128502 POINT(39.97784759030194 74.79857524594641) bank128502 +128503 POINT(40.062269807595214 73.39496416740177) bank128503 +128504 POINT(41.01714013857701 73.89744716086598) bank128504 +128505 POINT(40.08469367784445 74.94492256176184) bank128505 +128506 POINT(40.879136118480936 73.92694827079579) bank128506 +128507 POINT(40.17041437766669 73.06322905417902) bank128507 +128508 POINT(41.54745037985237 74.65138432806842) bank128508 +128509 POINT(40.74109725293849 74.23564809604656) bank128509 +128510 POINT(39.81470504881388 74.70421821701271) bank128510 +128511 POINT(40.38367586603828 74.4357002795633) bank128511 +128512 POINT(41.320022572780985 74.88476452354972) bank128512 +128513 POINT(40.78736489208761 74.39000394616103) bank128513 +128514 POINT(40.48573128662719 73.72761174674042) bank128514 +128515 POINT(39.798101693455095 73.72387195579869) bank128515 +128516 POINT(40.6300726728011 73.28940517821303) bank128516 +128517 POINT(40.496593386151076 73.25537996715649) bank128517 +128518 POINT(41.62134614669015 74.32424970414114) bank128518 +128519 POINT(41.62408887960881 74.3167870585703) bank128519 +128520 POINT(39.83877409187296 74.77538811499005) bank128520 +128521 POINT(41.03255864633006 73.202157411529) bank128521 +128522 POINT(40.07974882865374 73.55147423117556) bank128522 +128523 POINT(41.53445508775818 74.50524874112027) bank128523 +128524 POINT(40.9918808664423 74.86265042695406) bank128524 +128525 POINT(41.34668298864937 74.81121577852637) bank128525 +128526 POINT(41.68974655858282 73.83938746771514) bank128526 +128527 POINT(41.28982318326871 74.764916403117) bank128527 +128528 POINT(40.81053188580529 73.53781651997168) bank128528 +128529 POINT(41.638320367040016 74.75336848508701) bank128529 +128530 POINT(40.82250798576361 74.48609769542726) bank128530 +128531 POINT(40.471162208216064 73.47737067607761) bank128531 +128532 POINT(40.63316537073978 73.5718353179234) bank128532 +128533 POINT(41.341613556470286 74.24480242886841) bank128533 +128534 POINT(41.00918271793597 73.5854593872009) bank128534 +128535 POINT(41.290817977663316 73.67159035491528) bank128535 +128536 POINT(40.51089602183212 73.75828931937417) bank128536 +128537 POINT(40.92105425755367 73.10422902668176) bank128537 +128538 POINT(41.59411101412369 73.71580454295402) bank128538 +128539 POINT(40.51625431807246 74.18678762897629) bank128539 +128540 POINT(39.78549815731134 74.59611374452122) bank128540 +128541 POINT(40.182346724169044 73.84845589560844) bank128541 +128542 POINT(41.115114403838355 74.97889055630883) bank128542 +128543 POINT(40.99012085157018 73.24444163257893) bank128543 +128544 POINT(39.94568195244665 73.34726648496908) bank128544 +128545 POINT(40.2340170692348 74.36694420077401) bank128545 +128546 POINT(41.398996928585944 74.5286208918444) bank128546 +128547 POINT(41.6898970736294 74.27822316196514) bank128547 +128548 POINT(40.070395396173815 73.3294767262381) bank128548 +128549 POINT(41.11129148307267 74.51476627322909) bank128549 +128550 POINT(40.26629302018988 74.53610022807482) bank128550 +128551 POINT(41.264731302342796 73.89776634510329) bank128551 +128552 POINT(41.66446512984901 73.61487019196505) bank128552 +128553 POINT(40.07182114959479 74.76273795847963) bank128553 +128554 POINT(41.181485144181245 74.35212956285469) bank128554 +128555 POINT(40.91769163204607 74.07965266926786) bank128555 +128556 POINT(39.80979086218877 73.92320141874212) bank128556 +128557 POINT(40.72127062816714 74.20248615757356) bank128557 +128558 POINT(41.66650122257378 73.5534582712095) bank128558 +128559 POINT(39.73432605267523 74.34878366969247) bank128559 +128560 POINT(41.07900265922872 74.62266104867187) bank128560 +128561 POINT(41.177001349562126 73.28199620312073) bank128561 +128562 POINT(40.51654588596839 73.22379886796713) bank128562 +128563 POINT(41.39264045835164 73.48155505801697) bank128563 +128564 POINT(40.60216143346721 73.05244698151296) bank128564 +128565 POINT(40.9345515854556 74.6638298417329) bank128565 +128566 POINT(40.838318727390345 73.2639301383083) bank128566 +128567 POINT(41.41199844098517 74.53606575447627) bank128567 +128568 POINT(41.02376541946878 74.30243241426409) bank128568 +128569 POINT(40.24843724821827 74.55166955657148) bank128569 +128570 POINT(41.02276080084623 73.27617000402785) bank128570 +128571 POINT(40.444542015513484 74.31277866673263) bank128571 +128572 POINT(40.94321406084605 74.06656738399778) bank128572 +128573 POINT(40.08340334810795 74.5955393273807) bank128573 +128574 POINT(41.52774512124383 73.99043771693349) bank128574 +128575 POINT(41.37454418702947 73.58660051605906) bank128575 +128576 POINT(41.003762723651974 73.18043944499126) bank128576 +128577 POINT(41.218704825724814 73.53222045726896) bank128577 +128578 POINT(40.98326038567045 73.94687407611657) bank128578 +128579 POINT(40.761503629956714 73.80080734346303) bank128579 +128580 POINT(40.118796066278385 74.78077721956878) bank128580 +128581 POINT(41.411173106276394 74.8823137705751) bank128581 +128582 POINT(40.87233967171887 74.7038374969158) bank128582 +128583 POINT(40.21426386954215 73.52374545326275) bank128583 +128584 POINT(40.50770387908997 73.83994052818908) bank128584 +128585 POINT(40.74836851847154 74.36356595601377) bank128585 +128586 POINT(40.570074584555535 73.13415439961322) bank128586 +128587 POINT(39.730349356286105 74.4632545591501) bank128587 +128588 POINT(41.44665780324889 74.37141770386759) bank128588 +128589 POINT(40.20654788319419 73.45353367157084) bank128589 +128590 POINT(41.65127910077009 74.62060376080862) bank128590 +128591 POINT(40.114180311216344 74.91272438431956) bank128591 +128592 POINT(39.8116672616439 74.61403661626132) bank128592 +128593 POINT(39.990194735330874 73.92694710388193) bank128593 +128594 POINT(40.609316151281945 73.51198370501187) bank128594 +128595 POINT(41.11972330880658 74.77962986822166) bank128595 +128596 POINT(41.09982998684325 73.54421799532676) bank128596 +128597 POINT(39.910501277683714 73.30738680259495) bank128597 +128598 POINT(41.5005219631345 73.36679672959873) bank128598 +128599 POINT(40.90273538260365 73.6739710498913) bank128599 +128600 POINT(41.00345709346983 73.57446335645218) bank128600 +128601 POINT(39.76970468922832 73.42863171482668) bank128601 +128602 POINT(40.11522419817782 73.69531562096037) bank128602 +128603 POINT(40.09964938029743 73.39571850310767) bank128603 +128604 POINT(39.8249620736253 73.51980320311144) bank128604 +128605 POINT(40.008588624263524 73.80949161599702) bank128605 +128606 POINT(41.16653789390007 74.94878828820352) bank128606 +128607 POINT(40.27239135764878 73.46786082489784) bank128607 +128608 POINT(41.66280018744321 73.49546255141794) bank128608 +128609 POINT(41.21551014518087 74.09840248457206) bank128609 +128610 POINT(41.61128910908758 74.36849844535791) bank128610 +128611 POINT(41.53908417789514 73.07306408119459) bank128611 +128612 POINT(41.55994151334084 73.64692906153111) bank128612 +128613 POINT(40.739336308403544 73.687578183131) bank128613 +128614 POINT(41.630750871003784 74.56287125749292) bank128614 +128615 POINT(40.913590411204055 73.91034516011965) bank128615 +128616 POINT(40.02192682282221 73.61792395923871) bank128616 +128617 POINT(40.00998416442571 73.77414871788591) bank128617 +128618 POINT(40.215007565117325 74.77886025989636) bank128618 +128619 POINT(41.28890971638501 74.19622019102088) bank128619 +128620 POINT(39.86536147521831 74.91657949824453) bank128620 +128621 POINT(40.55576953432927 74.05363418911456) bank128621 +128622 POINT(41.36386504456676 74.51725068497352) bank128622 +128623 POINT(41.30293233259972 73.29837216912995) bank128623 +128624 POINT(39.89852748934842 73.24225897976396) bank128624 +128625 POINT(41.48083806531021 73.89249785916715) bank128625 +128626 POINT(41.40202448108944 73.0482623830361) bank128626 +128627 POINT(40.96212419499939 74.88612908615475) bank128627 +128628 POINT(41.07961676174895 73.73858502735405) bank128628 +128629 POINT(40.22706661554164 73.11802313565506) bank128629 +128630 POINT(40.13597893708282 74.04400382194011) bank128630 +128631 POINT(39.996013732652585 74.2844705761785) bank128631 +128632 POINT(41.21816029088689 74.72880594011431) bank128632 +128633 POINT(40.77339916083575 73.96547342263534) bank128633 +128634 POINT(40.170409542663336 74.53116628406617) bank128634 +128635 POINT(40.1424085803026 74.86405211932839) bank128635 +128636 POINT(40.91017616405617 73.36457488280845) bank128636 +128637 POINT(41.47333349724687 73.39976431468457) bank128637 +128638 POINT(39.906177703742024 74.64399742138036) bank128638 +128639 POINT(41.22363585589504 74.67503305348063) bank128639 +128640 POINT(39.78792443187263 73.89440049629661) bank128640 +128641 POINT(40.07314600751121 74.10751788123075) bank128641 +128642 POINT(41.616526454311725 73.88370332529065) bank128642 +128643 POINT(40.31291402897054 73.80212598746122) bank128643 +128644 POINT(41.52697577918052 73.16797107505027) bank128644 +128645 POINT(39.8414225432254 74.61768259243924) bank128645 +128646 POINT(41.570041572558075 74.46194154885875) bank128646 +128647 POINT(41.59879544512925 73.13713626522295) bank128647 +128648 POINT(41.22325207066773 73.10549006090031) bank128648 +128649 POINT(41.60851093586206 74.0126310506733) bank128649 +128650 POINT(40.526166737336226 73.38340357736288) bank128650 +128651 POINT(40.93396514913664 74.23946676774314) bank128651 +128652 POINT(40.16719495178656 74.18086131629147) bank128652 +128653 POINT(40.94709518273825 73.64388712522918) bank128653 +128654 POINT(40.27735067508872 74.66672616927195) bank128654 +128655 POINT(40.74215933571716 74.56230406455062) bank128655 +128656 POINT(41.681063078844836 74.17819685008716) bank128656 +128657 POINT(41.007407070339355 74.50047654604982) bank128657 +128658 POINT(40.25705971487731 74.2714491853641) bank128658 +128659 POINT(41.461842646863495 73.30764727991597) bank128659 +128660 POINT(41.08036219531775 74.09256849873272) bank128660 +128661 POINT(40.00261080083656 73.90101885937685) bank128661 +128662 POINT(41.08134864021294 73.92989328766564) bank128662 +128663 POINT(39.93557228531046 74.34359419548652) bank128663 +128664 POINT(40.812160239779935 73.06415029955005) bank128664 +128665 POINT(40.43411410256903 73.57641259576619) bank128665 +128666 POINT(40.55085165952153 74.12452763735934) bank128666 +128667 POINT(41.17270172772857 73.7392658539145) bank128667 +128668 POINT(40.00411773628713 73.75350446474835) bank128668 +128669 POINT(40.13659920004116 73.93246186947658) bank128669 +128670 POINT(40.2966960468949 73.2132384699948) bank128670 +128671 POINT(40.88633397049579 73.85508343904715) bank128671 +128672 POINT(41.5299505213814 73.3011060144042) bank128672 +128673 POINT(40.49886011306784 73.41285443204559) bank128673 +128674 POINT(41.02282499012305 74.66581328058751) bank128674 +128675 POINT(40.77670308000363 74.14481929133561) bank128675 +128676 POINT(41.270424524869 73.76247823206837) bank128676 +128677 POINT(41.139191114355825 74.50790539924515) bank128677 +128678 POINT(41.253155843183535 74.16740848715322) bank128678 +128679 POINT(41.300170278712976 74.83560957414788) bank128679 +128680 POINT(40.70179980136467 73.34426047545763) bank128680 +128681 POINT(40.94587454250763 74.15995702496477) bank128681 +128682 POINT(39.977109425861435 74.08621640998477) bank128682 +128683 POINT(40.23542714097626 74.29050353239472) bank128683 +128684 POINT(40.82202689001357 74.01645904070594) bank128684 +128685 POINT(41.297602877610736 74.69637017526415) bank128685 +128686 POINT(39.860921794648476 74.23814283045625) bank128686 +128687 POINT(41.03495275604912 73.62970377657187) bank128687 +128688 POINT(40.632593250704886 74.05460773823435) bank128688 +128689 POINT(41.445053828289794 73.77468971436376) bank128689 +128690 POINT(40.63652036432636 74.81268307562678) bank128690 +128691 POINT(40.04043708867559 74.37257596927984) bank128691 +128692 POINT(40.046843759152544 74.47090765928951) bank128692 +128693 POINT(40.16298816733183 74.65923561781969) bank128693 +128694 POINT(41.178826522380994 74.16280298744306) bank128694 +128695 POINT(39.774182692405105 74.35229039239417) bank128695 +128696 POINT(40.618920071901044 74.37501772986025) bank128696 +128697 POINT(40.75296491770947 74.0902033445497) bank128697 +128698 POINT(40.93990187553829 74.82605465649229) bank128698 +128699 POINT(40.47307638031114 74.93880875928345) bank128699 +128700 POINT(40.46695998744389 74.58475645358381) bank128700 +128701 POINT(41.408556853505104 73.67462288359951) bank128701 +128702 POINT(41.68742559987327 73.03065749859601) bank128702 +128703 POINT(41.17265310193536 74.98082285435319) bank128703 +128704 POINT(40.58212780946143 73.68768522034722) bank128704 +128705 POINT(41.1382136064004 74.15828788404673) bank128705 +128706 POINT(41.62980013593795 73.86655145536959) bank128706 +128707 POINT(39.80048603259233 74.09800123753234) bank128707 +128708 POINT(40.43867118598078 74.49380295960766) bank128708 +128709 POINT(39.77458014249113 73.50749858644903) bank128709 +128710 POINT(40.72811134507498 73.11232461618924) bank128710 +128711 POINT(41.45896518194136 74.17479584590319) bank128711 +128712 POINT(41.66907361072632 73.31840457098106) bank128712 +128713 POINT(41.068685746483595 73.99449097530879) bank128713 +128714 POINT(40.93458854350132 73.58554156167841) bank128714 +128715 POINT(41.2811768892253 73.90427702156303) bank128715 +128716 POINT(39.76468635945158 73.42404006028781) bank128716 +128717 POINT(40.208751730307554 73.05354284226622) bank128717 +128718 POINT(40.67443515807687 73.30951654335705) bank128718 +128719 POINT(40.49464065366172 74.71751511431489) bank128719 +128720 POINT(40.17188502457897 73.7845630068297) bank128720 +128721 POINT(40.64100532964815 73.117577314366) bank128721 +128722 POINT(41.67245622574142 74.13492964040628) bank128722 +128723 POINT(40.06951859067902 74.84723065124483) bank128723 +128724 POINT(41.53583140259339 74.39525044000384) bank128724 +128725 POINT(41.542827236717294 74.85470183792746) bank128725 +128726 POINT(41.70898319969249 74.68377236399623) bank128726 +128727 POINT(40.42782902967254 73.20798395090657) bank128727 +128728 POINT(39.89726630274429 74.85451000911542) bank128728 +128729 POINT(41.194190670509485 73.39993893902475) bank128729 +128730 POINT(41.41993281644487 73.49468469844535) bank128730 +128731 POINT(40.30947658147843 73.54632508151194) bank128731 +128732 POINT(41.118404016168235 73.45196548070507) bank128732 +128733 POINT(39.94797246023507 73.15568831992819) bank128733 +128734 POINT(41.11845227099282 74.65981118792644) bank128734 +128735 POINT(40.44177003813594 73.35349074778816) bank128735 +128736 POINT(40.49829139353967 74.75532066630332) bank128736 +128737 POINT(39.722954434477415 74.56623062809246) bank128737 +128738 POINT(41.5552509871084 74.09969846289776) bank128738 +128739 POINT(41.65970745283122 74.2969999581989) bank128739 +128740 POINT(40.8100209458369 74.26383868227757) bank128740 +128741 POINT(40.59282543144844 74.76118695733787) bank128741 +128742 POINT(40.5855495142784 73.7868602144766) bank128742 +128743 POINT(40.10908863534454 73.49787991915389) bank128743 +128744 POINT(39.836721793724024 73.22910820643078) bank128744 +128745 POINT(41.54208570705268 74.59705375392821) bank128745 +128746 POINT(39.871790434240104 73.0697386699628) bank128746 +128747 POINT(39.75325769383962 73.80036052466811) bank128747 +128748 POINT(39.96172544987818 73.8156436580314) bank128748 +128749 POINT(40.09473482137552 73.04886967723863) bank128749 +128750 POINT(40.844205494790245 74.61394627584832) bank128750 +128751 POINT(41.34044003027763 73.09697832125131) bank128751 +128752 POINT(40.94453512618815 74.94390139026497) bank128752 +128753 POINT(40.901739334086834 74.4209588768733) bank128753 +128754 POINT(40.26718355128009 73.39131321723957) bank128754 +128755 POINT(40.538870528564416 74.81364691481072) bank128755 +128756 POINT(40.439233399094995 73.82980047626766) bank128756 +128757 POINT(40.137330789977774 73.84836959818757) bank128757 +128758 POINT(40.1485513552768 73.04388852685152) bank128758 +128759 POINT(39.77372112241921 73.75546324193732) bank128759 +128760 POINT(39.92044424857762 74.97640755263008) bank128760 +128761 POINT(41.49115203955982 74.61066814266813) bank128761 +128762 POINT(41.04285643488879 73.89390987348177) bank128762 +128763 POINT(40.32330381031331 73.21047086071776) bank128763 +128764 POINT(40.13558278655792 74.55265726424646) bank128764 +128765 POINT(40.49464415909446 73.33280828113158) bank128765 +128766 POINT(40.61027809622421 73.82606324176808) bank128766 +128767 POINT(40.97743135293378 74.72788250127932) bank128767 +128768 POINT(40.590664809979614 73.13215054350748) bank128768 +128769 POINT(40.481775497615786 74.76909715585143) bank128769 +128770 POINT(40.52150901532777 74.05052224348495) bank128770 +128771 POINT(40.629412564062726 73.53202224369316) bank128771 +128772 POINT(40.28921191847801 73.69015747539639) bank128772 +128773 POINT(40.98953394986165 73.06688595593391) bank128773 +128774 POINT(39.91016488494706 73.11873234103673) bank128774 +128775 POINT(40.50440297395968 73.3583384674916) bank128775 +128776 POINT(41.46328743077303 73.3821654642582) bank128776 +128777 POINT(40.33022470523221 74.87869702235174) bank128777 +128778 POINT(40.91055979265755 74.49385720068706) bank128778 +128779 POINT(40.08064139754624 74.2441290637034) bank128779 +128780 POINT(41.69115709351344 73.60808910889143) bank128780 +128781 POINT(40.1643849442003 73.22287165935401) bank128781 +128782 POINT(39.970395146863105 74.37573906510745) bank128782 +128783 POINT(39.98736690887072 73.14247984622901) bank128783 +128784 POINT(41.625819172311935 74.04749992259877) bank128784 +128785 POINT(40.7181175635983 74.75579540858637) bank128785 +128786 POINT(39.99429732950802 74.02865539829494) bank128786 +128787 POINT(40.93039721723577 74.93068701658858) bank128787 +128788 POINT(41.69687344051294 74.41805709751843) bank128788 +128789 POINT(39.80941875357217 73.5655583418947) bank128789 +128790 POINT(39.717081694751066 74.1081840484245) bank128790 +128791 POINT(40.76849619677381 73.90217780344327) bank128791 +128792 POINT(41.61287682756707 73.96507277081143) bank128792 +128793 POINT(40.9843003169212 74.30490188057527) bank128793 +128794 POINT(40.195252493386455 73.22851077977616) bank128794 +128795 POINT(41.02611704034926 74.312296867539) bank128795 +128796 POINT(41.59325514218028 74.68254567463491) bank128796 +128797 POINT(40.026989414417464 74.58905740221857) bank128797 +128798 POINT(41.20369411888779 74.14646608541477) bank128798 +128799 POINT(41.128365094894825 73.50132746528632) bank128799 +128800 POINT(40.78376203508686 74.7236355404429) bank128800 +128801 POINT(39.831180907698354 74.44696597342406) bank128801 +128802 POINT(40.51246654884794 74.25658921568645) bank128802 +128803 POINT(41.47578887612731 74.95195580375439) bank128803 +128804 POINT(39.89093734570331 73.23737929729654) bank128804 +128805 POINT(40.47888512510449 73.25746513890992) bank128805 +128806 POINT(41.70162120022774 73.8083535519909) bank128806 +128807 POINT(41.45897082776312 73.91237004752644) bank128807 +128808 POINT(40.143669487743374 74.48285676004755) bank128808 +128809 POINT(41.580762778755954 73.22614161093618) bank128809 +128810 POINT(40.005859399019656 73.53000298447623) bank128810 +128811 POINT(40.55094049411308 73.94323244625723) bank128811 +128812 POINT(41.35692772158325 74.77443791424538) bank128812 +128813 POINT(40.46532013927591 74.09240220368667) bank128813 +128814 POINT(41.51619004084645 74.4942357465328) bank128814 +128815 POINT(40.029901960789424 74.69610097268173) bank128815 +128816 POINT(40.77929685856142 73.55251775986245) bank128816 +128817 POINT(39.93322791572347 74.20152533326736) bank128817 +128818 POINT(41.39340709485175 74.01712245786304) bank128818 +128819 POINT(41.1995094540826 74.71641549308721) bank128819 +128820 POINT(40.58931756648393 73.98901029938042) bank128820 +128821 POINT(41.527664809858535 73.39942676006345) bank128821 +128822 POINT(39.84689325443494 73.62122332123263) bank128822 +128823 POINT(41.42533884668433 74.84799987498913) bank128823 +128824 POINT(41.32817559674973 74.59031251436664) bank128824 +128825 POINT(39.87491416735981 74.83854864016092) bank128825 +128826 POINT(40.258614046993976 74.26183432332489) bank128826 +128827 POINT(40.27014401548422 74.63605952191408) bank128827 +128828 POINT(41.27453571213936 74.79901544839916) bank128828 +128829 POINT(39.78916360359264 73.82183825641066) bank128829 +128830 POINT(40.036248253807294 74.93336479718977) bank128830 +128831 POINT(41.05818679088354 73.19613944820497) bank128831 +128832 POINT(41.07004695107703 74.74302448443214) bank128832 +128833 POINT(40.03777965226449 74.37300867876156) bank128833 +128834 POINT(40.36463107709852 73.55853695595384) bank128834 +128835 POINT(40.49338423211585 73.75771045855595) bank128835 +128836 POINT(39.86007096757271 73.45936804944895) bank128836 +128837 POINT(40.5466627196662 73.38741992815464) bank128837 +128838 POINT(40.72591382033919 74.95868757423908) bank128838 +128839 POINT(41.62241470694955 74.03496603471291) bank128839 +128840 POINT(41.168512706844766 74.44275016671217) bank128840 +128841 POINT(40.913759940675625 74.61476577264132) bank128841 +128842 POINT(39.73975195410461 73.68372813102344) bank128842 +128843 POINT(39.84361805076875 73.74690796684318) bank128843 +128844 POINT(41.31646159608299 74.80428335120298) bank128844 +128845 POINT(40.80065898293817 73.06131392014754) bank128845 +128846 POINT(39.79845047837633 73.89980378155103) bank128846 +128847 POINT(40.90903916472478 74.00983549272895) bank128847 +128848 POINT(41.65996130954253 73.39955355213674) bank128848 +128849 POINT(41.32662535144115 73.7860837826488) bank128849 +128850 POINT(41.54496269469839 74.00128829599369) bank128850 +128851 POINT(41.612923558388914 73.4931300075731) bank128851 +128852 POINT(40.261660564941295 74.33414397164448) bank128852 +128853 POINT(41.514568603791595 73.54083799639297) bank128853 +128854 POINT(40.53715158627913 74.98678971438484) bank128854 +128855 POINT(40.17141283305449 74.8659901612104) bank128855 +128856 POINT(40.0582006936158 74.53973346594375) bank128856 +128857 POINT(40.71752296516025 74.16059502565479) bank128857 +128858 POINT(41.20240706979496 73.58396811292086) bank128858 +128859 POINT(41.177419765333134 74.21098240431584) bank128859 +128860 POINT(40.348648978322025 73.72994563276697) bank128860 +128861 POINT(40.44885575472227 73.46004009988273) bank128861 +128862 POINT(40.88318296547366 74.90157613371574) bank128862 +128863 POINT(40.37295722792095 73.94189964937118) bank128863 +128864 POINT(40.51179939649524 73.58374333544802) bank128864 +128865 POINT(40.51585060753031 74.65253870582116) bank128865 +128866 POINT(40.58263295856555 73.95654259209404) bank128866 +128867 POINT(41.59432469590029 74.44767277000331) bank128867 +128868 POINT(41.62918793485243 73.94300022860601) bank128868 +128869 POINT(40.84790704515159 73.00679796018586) bank128869 +128870 POINT(40.37803175688617 74.7880178495997) bank128870 +128871 POINT(40.82066093701511 73.95828051189964) bank128871 +128872 POINT(40.49483431884219 74.85887649911838) bank128872 +128873 POINT(41.18914569002093 74.3548457370348) bank128873 +128874 POINT(39.79275139753143 73.74216612280082) bank128874 +128875 POINT(41.32590344255696 73.98379907322598) bank128875 +128876 POINT(41.14775301409384 74.25407295343284) bank128876 +128877 POINT(40.916185608698896 74.12171255857028) bank128877 +128878 POINT(39.8043554778133 73.72130824524774) bank128878 +128879 POINT(39.86126199823861 74.25889384231665) bank128879 +128880 POINT(40.25456417226592 74.35373622714857) bank128880 +128881 POINT(40.777522494840476 74.53641457683567) bank128881 +128882 POINT(41.39594765854778 73.14988113931477) bank128882 +128883 POINT(40.12776837819802 73.87487678087614) bank128883 +128884 POINT(41.13039619752431 73.76779187767386) bank128884 +128885 POINT(41.30661244218077 73.15755305473242) bank128885 +128886 POINT(41.05536899356927 73.06560723191951) bank128886 +128887 POINT(40.14874500408894 74.53432584863485) bank128887 +128888 POINT(41.161513915041546 74.77310079375897) bank128888 +128889 POINT(39.84371230402111 73.90137275711507) bank128889 +128890 POINT(40.625407255125836 73.02350009678827) bank128890 +128891 POINT(41.22344397127241 74.14337758639567) bank128891 +128892 POINT(40.17970536359142 74.6417750934972) bank128892 +128893 POINT(40.330108746562225 73.86215760312038) bank128893 +128894 POINT(40.35169273360725 73.22087188694283) bank128894 +128895 POINT(41.071680106526955 73.92240154568977) bank128895 +128896 POINT(41.2274227597106 74.4116267774807) bank128896 +128897 POINT(40.80048982440916 74.93881835619591) bank128897 +128898 POINT(41.00985313221677 73.47829384578216) bank128898 +128899 POINT(40.65054411900999 73.36788755771495) bank128899 +128900 POINT(40.87562121519803 73.26072452438292) bank128900 +128901 POINT(40.03680432460309 74.69334835943332) bank128901 +128902 POINT(39.79881376509108 74.95917657829885) bank128902 +128903 POINT(40.87609461640727 74.54747671635602) bank128903 +128904 POINT(39.76599019428982 73.29671823030785) bank128904 +128905 POINT(40.818993389461205 73.60158994872069) bank128905 +128906 POINT(41.17090951850421 74.22515813020973) bank128906 +128907 POINT(40.60242715891864 74.40959314706787) bank128907 +128908 POINT(39.99024885727851 73.8709974493488) bank128908 +128909 POINT(41.109452580549465 74.67150730402929) bank128909 +128910 POINT(40.30526836123998 74.23510354068738) bank128910 +128911 POINT(41.60018309060592 73.96510581100496) bank128911 +128912 POINT(40.502378169622396 73.74635291517788) bank128912 +128913 POINT(40.679370502140905 73.01345433465465) bank128913 +128914 POINT(41.40413646985572 73.14124674744316) bank128914 +128915 POINT(40.32418764428054 73.12739686397123) bank128915 +128916 POINT(40.123833374230465 74.63586809073847) bank128916 +128917 POINT(39.77863677818417 74.23850852551989) bank128917 +128918 POINT(39.76814619161333 73.48729943012177) bank128918 +128919 POINT(41.087578507926914 74.18630840426519) bank128919 +128920 POINT(40.21141631207043 73.6262566264598) bank128920 +128921 POINT(40.08355922263461 73.74277841786646) bank128921 +128922 POINT(41.0261975040709 74.48383851430036) bank128922 +128923 POINT(39.98903687235948 74.31147685461336) bank128923 +128924 POINT(41.409372894707474 73.35689458991997) bank128924 +128925 POINT(40.28509133086209 74.61595811479515) bank128925 +128926 POINT(41.24956671251635 73.76166542170152) bank128926 +128927 POINT(41.18404804648852 74.27486714015265) bank128927 +128928 POINT(40.34408889470995 73.36253300729958) bank128928 +128929 POINT(40.14690992563513 73.4150356329665) bank128929 +128930 POINT(41.34104996037691 73.26095199239451) bank128930 +128931 POINT(40.51238441349317 73.09099231129416) bank128931 +128932 POINT(39.864896536084714 74.48921441191987) bank128932 +128933 POINT(41.19134003300643 73.07379894085638) bank128933 +128934 POINT(39.839070586983034 74.17487352515285) bank128934 +128935 POINT(39.92664739992563 73.89003596894428) bank128935 +128936 POINT(41.46493450853464 73.28038504762834) bank128936 +128937 POINT(41.23463782086739 74.42967772019739) bank128937 +128938 POINT(40.12353685452652 74.8347109267608) bank128938 +128939 POINT(41.19726409687719 74.19502481195744) bank128939 +128940 POINT(41.5576605105675 74.48294211919318) bank128940 +128941 POINT(40.87423619667445 74.78011107530784) bank128941 +128942 POINT(41.708771715187446 74.17423970388141) bank128942 +128943 POINT(40.42956078069353 73.58151255153537) bank128943 +128944 POINT(41.027467164690485 74.36903825028723) bank128944 +128945 POINT(39.82543031429881 73.03712853555689) bank128945 +128946 POINT(41.07866720119307 74.04773405541681) bank128946 +128947 POINT(41.014324814169534 74.52176888320902) bank128947 +128948 POINT(40.579749379866925 74.04281380907219) bank128948 +128949 POINT(41.024601834238474 74.76636433047835) bank128949 +128950 POINT(39.83726856223752 74.13798492142058) bank128950 +128951 POINT(41.095141485358624 74.67431306247921) bank128951 +128952 POINT(40.32259529584995 74.03919374861692) bank128952 +128953 POINT(40.7683419987386 73.85498184560186) bank128953 +128954 POINT(40.72702907640627 73.21458072469099) bank128954 +128955 POINT(41.043644710535816 73.510982449096) bank128955 +128956 POINT(41.36389192172727 73.64545278450956) bank128956 +128957 POINT(40.01659303231346 73.68903532543095) bank128957 +128958 POINT(40.272165815864426 73.03787643482336) bank128958 +128959 POINT(41.116668497209304 74.8495866539073) bank128959 +128960 POINT(41.403223998398865 73.88183194502004) bank128960 +128961 POINT(40.60321982190914 74.0774902607443) bank128961 +128962 POINT(41.05036929445222 73.9047873302254) bank128962 +128963 POINT(40.515415474501566 73.60939645302219) bank128963 +128964 POINT(40.82252137093321 73.78977143605172) bank128964 +128965 POINT(41.05033768047589 73.02294075202742) bank128965 +128966 POINT(40.70826831987538 74.34254320218741) bank128966 +128967 POINT(40.52758077104701 73.87120676462118) bank128967 +128968 POINT(41.104383501906575 74.12347970140088) bank128968 +128969 POINT(40.53828549426234 73.91166578878186) bank128969 +128970 POINT(41.44087841769999 73.35699350846185) bank128970 +128971 POINT(39.991875377142726 73.81003276338704) bank128971 +128972 POINT(39.9183886599796 74.49765703218769) bank128972 +128973 POINT(41.59142149147161 73.58357880770096) bank128973 +128974 POINT(40.510828805445804 74.70586256670133) bank128974 +128975 POINT(40.83703083703717 73.46296544760729) bank128975 +128976 POINT(40.21776062984038 74.90187147337414) bank128976 +128977 POINT(41.489562450109446 73.37152667968394) bank128977 +128978 POINT(41.674596087219875 74.59296414021989) bank128978 +128979 POINT(39.90319322227164 73.4129622675471) bank128979 +128980 POINT(39.72526306954164 74.35437586903016) bank128980 +128981 POINT(40.959832014224126 73.11295069103619) bank128981 +128982 POINT(40.773652753109886 73.34536356392067) bank128982 +128983 POINT(40.876114049643405 74.83250589868987) bank128983 +128984 POINT(41.135657553042606 74.30519634999187) bank128984 +128985 POINT(40.81222748043997 73.92596822520633) bank128985 +128986 POINT(40.714414297343225 74.8327299394231) bank128986 +128987 POINT(40.368667379805174 73.15748505084842) bank128987 +128988 POINT(41.13725579848194 74.04476606826898) bank128988 +128989 POINT(41.28237831656746 73.03378346480065) bank128989 +128990 POINT(40.618673386645966 73.99233765804084) bank128990 +128991 POINT(40.56511731341917 74.93079189557973) bank128991 +128992 POINT(39.904997631784646 73.60944280541943) bank128992 +128993 POINT(41.71031676323486 74.20396071720164) bank128993 +128994 POINT(40.92782621853866 74.43837537928215) bank128994 +128995 POINT(41.29194450616648 74.45646767313873) bank128995 +128996 POINT(40.503046449820744 74.04085780745638) bank128996 +128997 POINT(40.22647414770275 73.17653584736782) bank128997 +128998 POINT(39.95870524880481 74.63728828892081) bank128998 +128999 POINT(41.31117555762181 73.86758325398904) bank128999 +129000 POINT(39.73744619367813 73.45307737943887) bank129000 +129001 POINT(40.86661315797713 73.74697700040696) bank129001 +129002 POINT(40.552222196256345 74.38571762728486) bank129002 +129003 POINT(40.637748113420564 74.74897409496464) bank129003 +129004 POINT(41.07407035411219 73.07792438357326) bank129004 +129005 POINT(41.02385048671175 74.71647249386669) bank129005 +129006 POINT(41.36905189784358 73.95328650711326) bank129006 +129007 POINT(41.02022479615524 74.37825954181609) bank129007 +129008 POINT(40.625369023538354 73.80079112438509) bank129008 +129009 POINT(40.878009086828065 73.40928606424785) bank129009 +129010 POINT(39.81370180466644 73.22226137146195) bank129010 +129011 POINT(39.84348827002134 74.94453851370591) bank129011 +129012 POINT(40.92817025129789 73.06052144774152) bank129012 +129013 POINT(40.81208036732735 74.57878400822335) bank129013 +129014 POINT(41.01648083727252 74.01058893099501) bank129014 +129015 POINT(41.65400758224852 73.76230028776916) bank129015 +129016 POINT(40.49947454473817 74.02835884761642) bank129016 +129017 POINT(41.320934219905325 73.92277475301476) bank129017 +129018 POINT(41.0068000480212 73.45278785085257) bank129018 +129019 POINT(40.80530227054658 74.10790702405396) bank129019 +129020 POINT(39.73920294426792 73.999545821718) bank129020 +129021 POINT(41.13313125322502 74.29218403613882) bank129021 +129022 POINT(41.35450972956325 73.80401087220676) bank129022 +129023 POINT(40.394691743331094 73.1462077629119) bank129023 +129024 POINT(40.283465106816934 73.36908506725706) bank129024 +129025 POINT(40.3141035923495 74.04087068130316) bank129025 +129026 POINT(41.31155488319819 73.40866364570373) bank129026 +129027 POINT(40.01067858726374 74.1382912324427) bank129027 +129028 POINT(41.24103531028766 74.96786959086205) bank129028 +129029 POINT(39.95720814412408 73.16332154113664) bank129029 +129030 POINT(40.706898261158045 74.88104477918024) bank129030 +129031 POINT(39.954219741126614 74.73053785803876) bank129031 +129032 POINT(39.82232778713089 74.54670141808079) bank129032 +129033 POINT(39.75457625436892 74.87810981369442) bank129033 +129034 POINT(41.10303143966826 73.13666014958734) bank129034 +129035 POINT(41.48147496575723 73.71231629717656) bank129035 +129036 POINT(41.58794942615948 73.81462925594549) bank129036 +129037 POINT(39.817237227420826 74.99319521170621) bank129037 +129038 POINT(41.37251304434496 74.48103332635873) bank129038 +129039 POINT(39.733546135775065 74.48177495188037) bank129039 +129040 POINT(41.04830850443482 73.9332528220795) bank129040 +129041 POINT(40.825319268194406 74.78853689312143) bank129041 +129042 POINT(41.54171507124323 74.94135308201041) bank129042 +129043 POINT(40.449372409395764 74.40034972215174) bank129043 +129044 POINT(41.37570979571265 74.97790590753543) bank129044 +129045 POINT(41.245907633856646 73.08409135297211) bank129045 +129046 POINT(40.07284148936887 73.49921876343585) bank129046 +129047 POINT(41.34863991088772 74.16105019579327) bank129047 +129048 POINT(41.26631478286187 73.21562332524462) bank129048 +129049 POINT(40.30119130022771 73.69839936660797) bank129049 +129050 POINT(41.36991942571814 74.41274146245563) bank129050 +129051 POINT(41.22397262494964 74.20822702077487) bank129051 +129052 POINT(39.844299993125105 73.55529741652089) bank129052 +129053 POINT(40.353285323001096 74.75461689952687) bank129053 +129054 POINT(40.11432782942932 74.58252467852348) bank129054 +129055 POINT(41.442183913217704 73.76495730394127) bank129055 +129056 POINT(41.064317779195505 73.01176119529373) bank129056 +129057 POINT(40.10402832642563 74.77935277311742) bank129057 +129058 POINT(40.042514996952946 73.43453678985713) bank129058 +129059 POINT(40.45197901688108 73.5132625655564) bank129059 +129060 POINT(41.16644646043296 74.97746740296188) bank129060 +129061 POINT(39.96713789631656 73.12315215349562) bank129061 +129062 POINT(40.33128183161542 73.44174850496977) bank129062 +129063 POINT(41.37070745214048 73.9828225736838) bank129063 +129064 POINT(41.17518497356482 74.85465682989374) bank129064 +129065 POINT(41.46756313716236 73.56542025990944) bank129065 +129066 POINT(40.08386023551821 73.06602296530106) bank129066 +129067 POINT(41.16848536828312 74.06040459905344) bank129067 +129068 POINT(41.2183821559826 74.13937940899294) bank129068 +129069 POINT(40.74194549291243 73.0374154787975) bank129069 +129070 POINT(40.61111234227893 74.60346040502613) bank129070 +129071 POINT(40.04239744993102 73.00961052098634) bank129071 +129072 POINT(40.16960294290688 73.3272148172275) bank129072 +129073 POINT(41.293453764901926 74.94862701265403) bank129073 +129074 POINT(40.965223514383084 74.01746972353547) bank129074 +129075 POINT(40.19862925976275 73.22798127712292) bank129075 +129076 POINT(40.21082516718344 73.6139885774979) bank129076 +129077 POINT(41.49793693486729 73.9163955765449) bank129077 +129078 POINT(41.50819754913395 73.4209407279166) bank129078 +129079 POINT(41.1055109778356 73.2440523019368) bank129079 +129080 POINT(40.08347579573213 73.04062443008186) bank129080 +129081 POINT(41.376645803130685 74.76003651014203) bank129081 +129082 POINT(40.69862586493831 74.38818275685705) bank129082 +129083 POINT(41.6321901417179 73.46205139782664) bank129083 +129084 POINT(41.50825584631803 73.06672702278682) bank129084 +129085 POINT(40.26495050340027 74.03257640566825) bank129085 +129086 POINT(40.49369096236884 74.73822030653204) bank129086 +129087 POINT(40.257061393487795 74.23090944707987) bank129087 +129088 POINT(41.23403627283097 73.93842146593819) bank129088 +129089 POINT(41.633593911479615 74.52168566196669) bank129089 +129090 POINT(41.18946554073919 74.81641536290296) bank129090 +129091 POINT(40.544146273727605 73.28498159405451) bank129091 +129092 POINT(41.105110026895964 74.16289866632447) bank129092 +129093 POINT(40.389186256593966 73.41366954090554) bank129093 +129094 POINT(39.719355728537636 73.79981728877408) bank129094 +129095 POINT(40.49813276132169 73.31418556905975) bank129095 +129096 POINT(40.38016482580798 74.92885657008992) bank129096 +129097 POINT(41.69936456017099 74.95631475860418) bank129097 +129098 POINT(40.16485671775035 74.96359164065086) bank129098 +129099 POINT(40.540385557254254 73.63139697107746) bank129099 +129100 POINT(40.64699410291811 74.05947489104648) bank129100 +129101 POINT(40.70172700426975 73.58227613619117) bank129101 +129102 POINT(40.05990690626944 74.32356506124705) bank129102 +129103 POINT(41.57626420372198 74.66172250401995) bank129103 +129104 POINT(40.16492969199862 74.55668836207352) bank129104 +129105 POINT(41.18696694551246 73.8548433384102) bank129105 +129106 POINT(41.031035828169856 74.00797055996479) bank129106 +129107 POINT(41.669227855292796 74.33865302422618) bank129107 +129108 POINT(40.80874869110576 74.50602901814636) bank129108 +129109 POINT(40.172101479693985 74.02999552740516) bank129109 +129110 POINT(41.490818119366295 74.35677303980461) bank129110 +129111 POINT(41.269824475253344 73.60844855949831) bank129111 +129112 POINT(39.97168452663628 73.694415225576) bank129112 +129113 POINT(40.819131508952715 73.30361879383683) bank129113 +129114 POINT(39.74948363220386 73.19427482189406) bank129114 +129115 POINT(40.40751108616915 74.38084851903363) bank129115 +129116 POINT(40.49576138999631 74.79785051953111) bank129116 +129117 POINT(40.550835292939375 73.50526336190572) bank129117 +129118 POINT(39.961011880764694 74.2151053443858) bank129118 +129119 POINT(41.367668834231175 73.50754467356263) bank129119 +129120 POINT(41.03376666195819 74.74310294892976) bank129120 +129121 POINT(40.825571994363784 73.44043954731654) bank129121 +129122 POINT(40.14331985736653 74.34498428765984) bank129122 +129123 POINT(40.375296783616946 73.69657136674685) bank129123 +129124 POINT(40.96164360787006 73.65781973903081) bank129124 +129125 POINT(41.53148957145565 73.78902551740927) bank129125 +129126 POINT(40.32765058550018 73.46123660192183) bank129126 +129127 POINT(39.980501483038275 74.71635747980879) bank129127 +129128 POINT(39.80200811027025 73.65385762486184) bank129128 +129129 POINT(40.73744278539574 74.92554581813235) bank129129 +129130 POINT(40.812470037865644 73.0515321545749) bank129130 +129131 POINT(40.77660142967256 73.42061417418898) bank129131 +129132 POINT(40.54029214342133 74.07946592664527) bank129132 +129133 POINT(41.698827408314855 73.67435189311186) bank129133 +129134 POINT(39.76824789591636 73.81178149378468) bank129134 +129135 POINT(40.28903751488975 73.4586392574019) bank129135 +129136 POINT(40.02241932920987 74.04620056555223) bank129136 +129137 POINT(41.42837707671541 74.44046013759557) bank129137 +129138 POINT(39.89924072216786 73.58794742144697) bank129138 +129139 POINT(41.57722806481871 75.00135105119907) bank129139 +129140 POINT(41.23875280643316 74.7864965754092) bank129140 +129141 POINT(41.14510894025616 73.98822948764511) bank129141 +129142 POINT(39.75922787060777 73.85542507989109) bank129142 +129143 POINT(40.782041082482515 73.25062181509034) bank129143 +129144 POINT(41.07289392357168 73.53734697291073) bank129144 +129145 POINT(41.06748860192408 73.64265211429719) bank129145 +129146 POINT(40.142408982575276 73.7856302536903) bank129146 +129147 POINT(41.34474317520282 73.65813233737033) bank129147 +129148 POINT(40.61946336115616 74.8215557313434) bank129148 +129149 POINT(40.14348921525218 73.45226434521048) bank129149 +129150 POINT(41.41831688932886 73.06745382079822) bank129150 +129151 POINT(40.809173326455195 74.03155870109093) bank129151 +129152 POINT(40.53949069466924 73.32878459214201) bank129152 +129153 POINT(41.0292878196288 74.28675957869282) bank129153 +129154 POINT(41.423765301036525 73.95326637083527) bank129154 +129155 POINT(40.299372289647366 74.66468485903201) bank129155 +129156 POINT(41.557387691758294 74.12226741903697) bank129156 +129157 POINT(40.550700160447434 73.33609896475383) bank129157 +129158 POINT(40.16056422659657 73.67683446888758) bank129158 +129159 POINT(39.77906234474159 73.08913532278162) bank129159 +129160 POINT(40.2171232045453 73.48326274867266) bank129160 +129161 POINT(40.41196267573764 73.8330872274176) bank129161 +129162 POINT(40.32515210150969 73.04402381331447) bank129162 +129163 POINT(40.3604007180858 73.16499064822372) bank129163 +129164 POINT(39.88790586821058 74.783954720976) bank129164 +129165 POINT(40.229344099235576 73.80004150437281) bank129165 +129166 POINT(39.77937015984066 74.9970783337368) bank129166 +129167 POINT(40.7497542698353 73.47414030969833) bank129167 +129168 POINT(41.585092720693275 74.34280998160916) bank129168 +129169 POINT(40.07868270785788 73.61361782509489) bank129169 +129170 POINT(40.72798540730407 73.01505601924937) bank129170 +129171 POINT(40.83628555860956 73.88207596344995) bank129171 +129172 POINT(40.17708884392293 73.70600927022018) bank129172 +129173 POINT(41.55481763171686 73.17416418511107) bank129173 +129174 POINT(41.61510634425457 73.59766583720382) bank129174 +129175 POINT(41.14997534354283 74.76220825714637) bank129175 +129176 POINT(40.69872481073653 74.29712557845626) bank129176 +129177 POINT(40.054061944240495 73.29811278524193) bank129177 +129178 POINT(39.85638353550353 73.05681274840423) bank129178 +129179 POINT(41.58804782233225 73.46721162214581) bank129179 +129180 POINT(39.89424078775985 73.57387218442297) bank129180 +129181 POINT(40.73219370210382 74.74331075403778) bank129181 +129182 POINT(41.3702118744489 74.98080081240165) bank129182 +129183 POINT(41.45013712942681 74.98530697708429) bank129183 +129184 POINT(40.298794773544444 74.8678085720984) bank129184 +129185 POINT(40.46147109483128 73.42027314021631) bank129185 +129186 POINT(40.72173502949417 74.16839204971141) bank129186 +129187 POINT(40.63663190336011 74.33945187520472) bank129187 +129188 POINT(41.253546457447676 74.01641330225864) bank129188 +129189 POINT(41.50341615390287 74.90666984823997) bank129189 +129190 POINT(41.49237736533107 74.97190899683741) bank129190 +129191 POINT(40.3627294839686 73.42712828096538) bank129191 +129192 POINT(39.900924339908954 73.0847360078393) bank129192 +129193 POINT(39.77724602810627 74.16832182572793) bank129193 +129194 POINT(40.37696001308385 73.38623283996266) bank129194 +129195 POINT(40.28638256968463 73.64315749610905) bank129195 +129196 POINT(40.64897206500882 74.1209219956163) bank129196 +129197 POINT(40.59668503610569 73.18277464765177) bank129197 +129198 POINT(40.27889968188829 74.82939861343796) bank129198 +129199 POINT(40.79284047868546 74.16450232384068) bank129199 +129200 POINT(40.11760891295707 73.11426858642496) bank129200 +129201 POINT(40.461575095559446 73.06311941432851) bank129201 +129202 POINT(40.56968269907102 73.75502706220668) bank129202 +129203 POINT(41.14560823470063 74.93123293645061) bank129203 +129204 POINT(40.95111614835515 73.04003235978821) bank129204 +129205 POINT(41.240186976198466 74.16092053690154) bank129205 +129206 POINT(41.2231645416819 73.61312970335281) bank129206 +129207 POINT(40.041379649524366 74.98272263594886) bank129207 +129208 POINT(40.78582418260513 73.6685852392891) bank129208 +129209 POINT(39.76500759096081 74.39445648061614) bank129209 +129210 POINT(40.00658012726706 74.07618308832178) bank129210 +129211 POINT(39.88978019476198 74.2826989837953) bank129211 +129212 POINT(41.09425847197881 74.32024767661268) bank129212 +129213 POINT(41.00008173422182 74.2093853093351) bank129213 +129214 POINT(40.81975284830688 74.08569521921747) bank129214 +129215 POINT(40.243781636067055 74.3006624609933) bank129215 +129216 POINT(39.98167646440251 73.83137606774544) bank129216 +129217 POINT(39.74604149024537 73.49463687972334) bank129217 +129218 POINT(41.28712271479927 74.53493266177023) bank129218 +129219 POINT(39.92434338667143 73.97024372207686) bank129219 +129220 POINT(41.151377574485494 73.19414805389978) bank129220 +129221 POINT(40.275923875689735 74.03896045253636) bank129221 +129222 POINT(39.75518450628412 73.32412375365458) bank129222 +129223 POINT(41.3732127700727 74.02111829941198) bank129223 +129224 POINT(40.79900703627209 74.98866362032501) bank129224 +129225 POINT(41.019582086575475 74.01683197357188) bank129225 +129226 POINT(40.19519316496197 74.08977809912281) bank129226 +129227 POINT(40.30377804281991 74.4794450348812) bank129227 +129228 POINT(40.40287252454073 74.00673965427684) bank129228 +129229 POINT(39.932404620541995 73.60289207782522) bank129229 +129230 POINT(40.61450735186782 73.14093444682948) bank129230 +129231 POINT(39.78239043203355 73.91557911677535) bank129231 +129232 POINT(39.79093692792679 73.19058739513328) bank129232 +129233 POINT(39.94661557489014 73.69110698266097) bank129233 +129234 POINT(41.56937004448888 74.95893362273725) bank129234 +129235 POINT(39.89683775058704 73.60389619367707) bank129235 +129236 POINT(40.26978378505776 73.52379529642397) bank129236 +129237 POINT(40.03631431700951 74.80133043297168) bank129237 +129238 POINT(39.93649981367118 74.56111232986682) bank129238 +129239 POINT(39.93908553148835 73.60321286114757) bank129239 +129240 POINT(39.90893902887751 74.93327097412585) bank129240 +129241 POINT(40.34677488963811 74.04505653843401) bank129241 +129242 POINT(40.378537614240784 73.28019156500945) bank129242 +129243 POINT(41.30676983821642 73.27052273508423) bank129243 +129244 POINT(40.57776019730207 73.9824841136331) bank129244 +129245 POINT(40.466374493949324 74.06425394710404) bank129245 +129246 POINT(41.2380728780278 73.75394027348545) bank129246 +129247 POINT(40.66500279181092 74.29610748520406) bank129247 +129248 POINT(40.57133617147589 73.17463555550282) bank129248 +129249 POINT(40.055889718989505 73.18152484768062) bank129249 +129250 POINT(41.007363365852655 74.43129684297148) bank129250 +129251 POINT(40.80958172232431 74.71177170446175) bank129251 +129252 POINT(41.04916417097132 73.11192818217546) bank129252 +129253 POINT(41.52581837690685 73.88116837546931) bank129253 +129254 POINT(40.00516893366019 73.50868524025019) bank129254 +129255 POINT(41.36907420664111 73.40263760743736) bank129255 +129256 POINT(39.80685485433831 74.36125639597428) bank129256 +129257 POINT(40.40847364124279 74.24540239636731) bank129257 +129258 POINT(41.65316090164637 74.24232511367619) bank129258 +129259 POINT(41.33137633099454 74.7283067163429) bank129259 +129260 POINT(39.897732988046066 73.10591883478135) bank129260 +129261 POINT(40.837494888357725 73.79348006282666) bank129261 +129262 POINT(41.161195546354506 73.77239544872742) bank129262 +129263 POINT(40.603499451231514 73.77815441550693) bank129263 +129264 POINT(40.832221089328414 73.18410860202214) bank129264 +129265 POINT(41.673067782687475 73.91169663788736) bank129265 +129266 POINT(40.48458825216154 73.83210537343297) bank129266 +129267 POINT(41.711813638225934 73.59211821562519) bank129267 +129268 POINT(39.79621608912775 73.2554256188685) bank129268 +129269 POINT(40.19571815507482 74.9981306135732) bank129269 +129270 POINT(41.706388559560416 74.96969267614445) bank129270 +129271 POINT(40.34075579687943 73.17054290855917) bank129271 +129272 POINT(40.44518069914647 74.95231056347535) bank129272 +129273 POINT(40.97112092940674 74.00661282556892) bank129273 +129274 POINT(39.92394893262269 74.28092008089241) bank129274 +129275 POINT(41.03961556307693 74.90322185361707) bank129275 +129276 POINT(41.6860054719241 74.06128863516322) bank129276 +129277 POINT(41.315564823897816 73.59522985123323) bank129277 +129278 POINT(40.92741195186288 74.05219209972255) bank129278 +129279 POINT(41.627663600001334 73.8859901891199) bank129279 +129280 POINT(40.50870374230252 74.79242279730205) bank129280 +129281 POINT(40.31413348560009 73.04386754615923) bank129281 +129282 POINT(40.70127907027245 74.8783350828814) bank129282 +129283 POINT(40.98174006095707 73.47899229300528) bank129283 +129284 POINT(40.920928683142115 73.48441163737984) bank129284 +129285 POINT(40.75811597319279 73.70314105702124) bank129285 +129286 POINT(41.3517711980564 74.47208001103554) bank129286 +129287 POINT(41.35596328836082 74.34313326369133) bank129287 +129288 POINT(40.61880756795624 74.7755989634743) bank129288 +129289 POINT(40.673231688893914 73.61668761085366) bank129289 +129290 POINT(40.83184004696193 73.11253203423071) bank129290 +129291 POINT(39.78639219104156 73.56388161807062) bank129291 +129292 POINT(40.36873559483801 73.04547505794406) bank129292 +129293 POINT(40.71116960721814 74.8816129757203) bank129293 +129294 POINT(40.77041704171646 73.15928474067421) bank129294 +129295 POINT(40.88384551469816 73.99270681765414) bank129295 +129296 POINT(40.86658631917912 74.25118704811096) bank129296 +129297 POINT(39.79309083260626 73.00881813168968) bank129297 +129298 POINT(41.21044777285024 74.30599888542608) bank129298 +129299 POINT(40.10938720009033 73.47196668660226) bank129299 +129300 POINT(40.83091912835524 74.28169074020269) bank129300 +129301 POINT(40.44748519701779 73.43988823900783) bank129301 +129302 POINT(40.1087226714916 74.7659671113752) bank129302 +129303 POINT(40.69859070206874 73.64186175202707) bank129303 +129304 POINT(41.710003658004396 73.47082283622476) bank129304 +129305 POINT(41.005846703381245 73.76188322438159) bank129305 +129306 POINT(40.52537838086934 74.65753489967388) bank129306 +129307 POINT(41.315744326174915 74.6777986555339) bank129307 +129308 POINT(40.83922202518722 73.59690260866977) bank129308 +129309 POINT(39.75583430789081 74.26336812630657) bank129309 +129310 POINT(40.9565299095135 74.33197263795658) bank129310 +129311 POINT(41.26347913381319 73.24615276068673) bank129311 +129312 POINT(40.48843862046003 74.1314047361543) bank129312 +129313 POINT(40.66389249044062 73.33295771926001) bank129313 +129314 POINT(40.15944102068887 74.61064815991105) bank129314 +129315 POINT(41.17636584424 74.3584001699525) bank129315 +129316 POINT(41.47040018022826 74.0105543888672) bank129316 +129317 POINT(41.11354140452679 73.73885848272688) bank129317 +129318 POINT(40.64571838214703 73.14852349691391) bank129318 +129319 POINT(41.67715999442663 73.61008850368802) bank129319 +129320 POINT(40.03705882132828 74.17176128046258) bank129320 +129321 POINT(40.30932408814428 74.47336477031787) bank129321 +129322 POINT(40.85872407106234 73.5561625171216) bank129322 +129323 POINT(41.290922709820606 73.92583068900848) bank129323 +129324 POINT(41.130829302030094 73.58198374608612) bank129324 +129325 POINT(40.7375693982658 74.19507722618344) bank129325 +129326 POINT(41.11324020371689 74.4738112784379) bank129326 +129327 POINT(40.31295536577871 74.98950578114454) bank129327 +129328 POINT(39.84190239804458 74.19760731720415) bank129328 +129329 POINT(40.31132393454307 73.42584217331377) bank129329 +129330 POINT(40.71241567538457 74.99598700246736) bank129330 +129331 POINT(41.308732353679964 74.14629456567306) bank129331 +129332 POINT(41.32870215035686 73.01548663408596) bank129332 +129333 POINT(41.20774519854135 74.96130661406326) bank129333 +129334 POINT(40.76426860327388 74.2177999748626) bank129334 +129335 POINT(41.69631485540428 73.22433732978025) bank129335 +129336 POINT(41.47420824405026 73.198181940215) bank129336 +129337 POINT(39.83580829167678 74.44670587874852) bank129337 +129338 POINT(40.90121329693034 73.12416993702236) bank129338 +129339 POINT(41.15473534271536 74.5491971842899) bank129339 +129340 POINT(39.73511340321071 74.82383816891661) bank129340 +129341 POINT(40.2188467390262 74.49331301928625) bank129341 +129342 POINT(40.6747072470294 74.4379085067095) bank129342 +129343 POINT(41.527710086437814 74.09633363831232) bank129343 +129344 POINT(41.2308089110836 73.41774090867233) bank129344 +129345 POINT(41.650228744058836 73.8692427736182) bank129345 +129346 POINT(39.86357564522291 74.06386729499945) bank129346 +129347 POINT(41.019360122475895 74.92243989372577) bank129347 +129348 POINT(41.333907762922955 73.19698167038955) bank129348 +129349 POINT(41.23655779175863 74.18772635560927) bank129349 +129350 POINT(41.57835609040481 74.42483827147603) bank129350 +129351 POINT(39.99657822565632 73.01594129592041) bank129351 +129352 POINT(40.5851534772799 74.41663723799964) bank129352 +129353 POINT(40.32437071614435 74.04072501307742) bank129353 +129354 POINT(40.82606311801548 73.05876099133918) bank129354 +129355 POINT(41.4187195299118 73.61761795026813) bank129355 +129356 POINT(40.1531532945906 73.60604060695266) bank129356 +129357 POINT(40.32939385720654 73.0642171708882) bank129357 +129358 POINT(41.308670568889 73.49340806378262) bank129358 +129359 POINT(41.32623797999166 74.32640657965413) bank129359 +129360 POINT(39.7335423927223 74.14457178030042) bank129360 +129361 POINT(39.96142545571566 73.50056396140806) bank129361 +129362 POINT(41.40589845829211 74.24933137335873) bank129362 +129363 POINT(41.644975100904766 73.43638972212268) bank129363 +129364 POINT(40.419880475058015 74.2780292327383) bank129364 +129365 POINT(41.48560622853852 74.00359721811095) bank129365 +129366 POINT(40.82616060447059 74.5640877121542) bank129366 +129367 POINT(40.557625305382395 74.56040106656576) bank129367 +129368 POINT(40.012286048609624 74.6134802092282) bank129368 +129369 POINT(41.06541928365132 74.38427581761877) bank129369 +129370 POINT(40.106178952015405 73.91567104969606) bank129370 +129371 POINT(41.58107387819418 73.28489448071052) bank129371 +129372 POINT(39.84551411503939 74.7973159991293) bank129372 +129373 POINT(40.93640765200729 73.46297448692388) bank129373 +129374 POINT(41.00497293565183 74.59073240032687) bank129374 +129375 POINT(40.413048950863356 74.24513925983425) bank129375 +129376 POINT(40.488223016095745 73.7085664263267) bank129376 +129377 POINT(39.88907715009236 74.63319926232722) bank129377 +129378 POINT(39.99434624745681 73.47045639513382) bank129378 +129379 POINT(41.108761498860844 74.24037481110979) bank129379 +129380 POINT(41.5214886907019 73.76105380513228) bank129380 +129381 POINT(39.71733080767675 74.83682341514755) bank129381 +129382 POINT(40.36290236954543 73.97087657554925) bank129382 +129383 POINT(41.21806896482718 74.84254305151451) bank129383 +129384 POINT(41.61307488896872 74.02461587767951) bank129384 +129385 POINT(40.42418669388661 73.97025367524591) bank129385 +129386 POINT(41.36182817687598 73.3201923154709) bank129386 +129387 POINT(41.10913594221455 74.72925439019087) bank129387 +129388 POINT(41.52654873299306 74.60385011521514) bank129388 +129389 POINT(41.44197251761873 74.98286914519679) bank129389 +129390 POINT(40.48763255868141 73.77011564494893) bank129390 +129391 POINT(40.100271803557185 73.50871754084925) bank129391 +129392 POINT(40.199959750998275 74.29267212544259) bank129392 +129393 POINT(41.55169626511773 74.50115504648451) bank129393 +129394 POINT(40.47821595527739 73.59490059342939) bank129394 +129395 POINT(41.68339210496989 74.69433928440888) bank129395 +129396 POINT(41.58106593040192 73.5885332070924) bank129396 +129397 POINT(40.22938416548205 74.096818380765) bank129397 +129398 POINT(40.44304287397052 74.02105187539964) bank129398 +129399 POINT(41.02890103065015 74.45379911536492) bank129399 +129400 POINT(41.22753210246428 73.7974413368844) bank129400 +129401 POINT(39.82454675759382 73.60048367697777) bank129401 +129402 POINT(41.53250547508038 73.08488814595276) bank129402 +129403 POINT(40.733684111399 73.81133649434078) bank129403 +129404 POINT(40.58213395952135 73.92880752357938) bank129404 +129405 POINT(41.03841176877359 73.35685621168963) bank129405 +129406 POINT(41.335236610799576 74.2990181067115) bank129406 +129407 POINT(40.18828948057894 73.66010149647629) bank129407 +129408 POINT(40.142693809355826 74.99360066548857) bank129408 +129409 POINT(41.44219091583047 74.21542607563923) bank129409 +129410 POINT(40.780735108987194 74.25741915699895) bank129410 +129411 POINT(40.13351652850516 74.9535960702085) bank129411 +129412 POINT(40.16157561200936 73.91962256412089) bank129412 +129413 POINT(41.510550025037716 74.34337629279986) bank129413 +129414 POINT(41.201140519110375 73.41937632129483) bank129414 +129415 POINT(40.117143443489965 73.39224055932738) bank129415 +129416 POINT(40.99742020928126 74.13123340386107) bank129416 +129417 POINT(39.75355894263188 73.3800116299103) bank129417 +129418 POINT(41.66102706370936 73.46347853081946) bank129418 +129419 POINT(40.40018819466658 73.59811188261554) bank129419 +129420 POINT(40.18411009272385 73.64967295301271) bank129420 +129421 POINT(41.38059807557165 74.28393212328518) bank129421 +129422 POINT(41.296482661352584 73.89800177275721) bank129422 +129423 POINT(40.606112042828514 74.29584383510655) bank129423 +129424 POINT(40.737293890028866 73.61375105343362) bank129424 +129425 POINT(40.40046173260504 73.82989002085615) bank129425 +129426 POINT(41.06013732943353 73.29745663221813) bank129426 +129427 POINT(40.60131217746146 73.68323944698665) bank129427 +129428 POINT(40.53106414741644 73.94342164735592) bank129428 +129429 POINT(41.20632282349198 74.4838914211054) bank129429 +129430 POINT(40.80202027938195 74.23738707772313) bank129430 +129431 POINT(40.269942689387705 73.31743225794125) bank129431 +129432 POINT(40.095495156425294 74.60637770098286) bank129432 +129433 POINT(40.63154544589437 74.97368824692943) bank129433 +129434 POINT(40.55338891168779 73.31085829837562) bank129434 +129435 POINT(40.55415614600094 73.19692582955031) bank129435 +129436 POINT(39.98538500194634 73.03678896989211) bank129436 +129437 POINT(40.95295590565787 74.5122601006608) bank129437 +129438 POINT(39.78216229042461 74.78332055888193) bank129438 +129439 POINT(41.32892809303826 74.3211035835499) bank129439 +129440 POINT(41.63825591200715 73.58371134193452) bank129440 +129441 POINT(39.73960550284859 73.70111019533081) bank129441 +129442 POINT(40.85926726614162 74.58783858780575) bank129442 +129443 POINT(40.10959679636219 73.85391475605543) bank129443 +129444 POINT(41.41478662241191 73.45212026460905) bank129444 +129445 POINT(39.845602337342 74.90868236899473) bank129445 +129446 POINT(40.221997401372676 73.95512361905352) bank129446 +129447 POINT(40.21163366197027 73.31794519196666) bank129447 +129448 POINT(40.98668240336396 74.57564332262511) bank129448 +129449 POINT(40.749523007834696 73.5651405886068) bank129449 +129450 POINT(39.724736523666465 73.18832820269117) bank129450 +129451 POINT(41.25410213536102 73.15674199703588) bank129451 +129452 POINT(41.71234334472404 73.32732118175358) bank129452 +129453 POINT(40.143140465835216 74.23480870929329) bank129453 +129454 POINT(40.204862892462636 74.40880548895413) bank129454 +129455 POINT(41.12175620628866 74.6787225559128) bank129455 +129456 POINT(40.743402232350924 73.8130463810638) bank129456 +129457 POINT(39.871257868185786 74.78718139487032) bank129457 +129458 POINT(41.70747332408078 73.10002251624789) bank129458 +129459 POINT(40.29176051912209 74.10677797195956) bank129459 +129460 POINT(40.95732668745333 74.59254634075465) bank129460 +129461 POINT(41.29433356998054 74.62124857695449) bank129461 +129462 POINT(40.84463852656373 74.91952210681666) bank129462 +129463 POINT(40.96162190435986 74.92103957454445) bank129463 +129464 POINT(40.45179272958239 74.13107931023077) bank129464 +129465 POINT(41.5314731801563 74.88879124424761) bank129465 +129466 POINT(40.910221185478804 73.3312348135617) bank129466 +129467 POINT(41.55762959473121 74.1095958855409) bank129467 +129468 POINT(41.39489861679134 74.95564080566619) bank129468 +129469 POINT(39.94690334908857 74.37338905282286) bank129469 +129470 POINT(41.63651865066917 73.42088158969351) bank129470 +129471 POINT(41.21017079396539 74.01926782295928) bank129471 +129472 POINT(40.0612134327311 73.1465578842541) bank129472 +129473 POINT(39.79160623625925 73.90271361940465) bank129473 +129474 POINT(41.01060690086908 74.9479138459866) bank129474 +129475 POINT(40.08528698203917 74.52382378372197) bank129475 +129476 POINT(41.39949730956777 74.88967768609619) bank129476 +129477 POINT(40.15282569043273 73.80279487214911) bank129477 +129478 POINT(39.81022267598943 73.45294775368231) bank129478 +129479 POINT(40.15551327510967 74.0412178375049) bank129479 +129480 POINT(41.50178174522879 74.86047888840699) bank129480 +129481 POINT(40.94720579431396 73.65400079930029) bank129481 +129482 POINT(40.80672691656628 74.90883707367443) bank129482 +129483 POINT(41.63149488919224 74.2001796137959) bank129483 +129484 POINT(40.58425092297201 74.48018479763276) bank129484 +129485 POINT(41.354963797654996 73.00734758578294) bank129485 +129486 POINT(41.63313315431653 74.99839941607061) bank129486 +129487 POINT(40.02837162395105 73.19578782892111) bank129487 +129488 POINT(39.83961080721265 73.20812080240644) bank129488 +129489 POINT(40.02315009451385 74.42332240542228) bank129489 +129490 POINT(39.80282542014221 74.23346310903952) bank129490 +129491 POINT(40.952065243829 74.02297565179985) bank129491 +129492 POINT(41.32469064483091 74.42635776921972) bank129492 +129493 POINT(41.113311582888684 74.14304009613703) bank129493 +129494 POINT(40.83535560788257 73.56125007510356) bank129494 +129495 POINT(39.71452931312224 74.95274720926926) bank129495 +129496 POINT(40.201629389975935 74.37979180639483) bank129496 +129497 POINT(41.374633749202125 74.53024941693614) bank129497 +129498 POINT(41.45089980470123 73.62004392796946) bank129498 +129499 POINT(41.1390084024112 73.09197532046693) bank129499 +129500 POINT(41.64175565302281 74.04538609342706) bank129500 +129501 POINT(41.224048883264956 73.65263815975297) bank129501 +129502 POINT(41.26360982888753 73.37588447803824) bank129502 +129503 POINT(41.14073886313571 73.35067122362138) bank129503 +129504 POINT(39.87603341342033 74.26062759047643) bank129504 +129505 POINT(41.152375734204185 74.27854260006416) bank129505 +129506 POINT(39.731798848291525 73.71625916203529) bank129506 +129507 POINT(41.46188071479874 74.0424935539519) bank129507 +129508 POINT(40.80459828593987 74.34885169112962) bank129508 +129509 POINT(40.054239598255336 73.203396155877) bank129509 +129510 POINT(41.66526445717682 73.51635309789876) bank129510 +129511 POINT(40.448811703241425 73.95229569754653) bank129511 +129512 POINT(40.33646328571148 73.49620853678574) bank129512 +129513 POINT(39.76396807899224 73.91498398706226) bank129513 +129514 POINT(41.53000046513953 74.49095038197588) bank129514 +129515 POINT(41.46016681985676 73.14170989557476) bank129515 +129516 POINT(41.043738894216375 73.52677331684336) bank129516 +129517 POINT(39.88823080386232 74.53779942637134) bank129517 +129518 POINT(40.63540872042042 74.05722641837416) bank129518 +129519 POINT(41.6066663375578 74.80448269349434) bank129519 +129520 POINT(41.35165899754752 73.92725448435803) bank129520 +129521 POINT(40.81500349547729 74.93369165860658) bank129521 +129522 POINT(40.85898882595091 74.74583575708257) bank129522 +129523 POINT(41.526781577324485 74.15818061542434) bank129523 +129524 POINT(40.2137044866937 74.30881371306296) bank129524 +129525 POINT(40.92796356435518 73.34254129223162) bank129525 +129526 POINT(40.70795789556874 73.06869449114203) bank129526 +129527 POINT(40.237469246501476 73.40752558453461) bank129527 +129528 POINT(39.84638230298396 73.9379645627484) bank129528 +129529 POINT(40.61617135302421 73.23703631347615) bank129529 +129530 POINT(41.53983352673437 73.34436354439632) bank129530 +129531 POINT(40.742040775826254 73.55764975114742) bank129531 +129532 POINT(40.42203322354884 73.93341743252458) bank129532 +129533 POINT(41.16037836204615 74.22453452368853) bank129533 +129534 POINT(40.917267940292675 74.27328589318334) bank129534 +129535 POINT(40.88254820062728 74.96858965061934) bank129535 +129536 POINT(41.06567091912918 73.64788829257407) bank129536 +129537 POINT(40.2816037380157 74.96597653652107) bank129537 +129538 POINT(41.57042768402002 74.95260160344209) bank129538 +129539 POINT(39.934823394800176 73.6876848969878) bank129539 +129540 POINT(40.306190367653855 73.64131770447051) bank129540 +129541 POINT(41.58751908337405 74.43526188770456) bank129541 +129542 POINT(40.08411066677051 74.93558168492015) bank129542 +129543 POINT(41.57281780690917 74.96629690165881) bank129543 +129544 POINT(39.79869235245297 74.67839100824423) bank129544 +129545 POINT(41.59926856017915 73.38317847654525) bank129545 +129546 POINT(40.629750397796585 73.00678730360453) bank129546 +129547 POINT(40.38179671551002 73.23806926058354) bank129547 +129548 POINT(41.546660780539405 74.1384932667679) bank129548 +129549 POINT(39.90036255054449 73.0171489317614) bank129549 +129550 POINT(39.99165448208416 73.37767769891818) bank129550 +129551 POINT(40.21715940042363 74.28599837317695) bank129551 +129552 POINT(41.565144434343836 74.68910037143117) bank129552 +129553 POINT(41.63187316985859 73.23109136262981) bank129553 +129554 POINT(41.1630236978253 73.29129503589459) bank129554 +129555 POINT(41.23103461071486 73.56489938476233) bank129555 +129556 POINT(40.47840471165199 73.92453729884289) bank129556 +129557 POINT(40.31843213887437 74.98991557007226) bank129557 +129558 POINT(40.35545583400309 74.26982146953345) bank129558 +129559 POINT(40.03046209823839 74.15497751419366) bank129559 +129560 POINT(40.11229111180523 74.83901998853938) bank129560 +129561 POINT(41.554369359851734 73.24373725371952) bank129561 +129562 POINT(39.78276718148048 74.20231196746711) bank129562 +129563 POINT(40.77319058477798 73.52994178466191) bank129563 +129564 POINT(40.04029380161015 74.6069610022174) bank129564 +129565 POINT(40.75543548745012 73.38663471798472) bank129565 +129566 POINT(41.56272334425627 73.81961396032008) bank129566 +129567 POINT(41.34586654281798 73.18598626222222) bank129567 +129568 POINT(40.84726606297595 74.45143559395652) bank129568 +129569 POINT(41.00673766279648 74.13292026942622) bank129569 +129570 POINT(41.15900838239904 73.92408993489286) bank129570 +129571 POINT(39.72892279979463 73.70903948620028) bank129571 +129572 POINT(41.452105684059156 74.48739209134672) bank129572 +129573 POINT(40.501195421810955 74.97817444884613) bank129573 +129574 POINT(40.803780369543915 74.1839371439508) bank129574 +129575 POINT(41.65778746420294 73.59162908002071) bank129575 +129576 POINT(40.72478663069492 74.63857522363975) bank129576 +129577 POINT(41.61376210585935 74.16095331074351) bank129577 +129578 POINT(40.78539931799574 73.78185153318266) bank129578 +129579 POINT(40.990647023498376 73.88859000017376) bank129579 +129580 POINT(41.637213670649984 74.70473520984976) bank129580 +129581 POINT(40.90401499271632 74.68139893676553) bank129581 +129582 POINT(41.41852179487017 73.72146734964774) bank129582 +129583 POINT(39.776353936393015 74.74733453069196) bank129583 +129584 POINT(39.72394526289889 74.38610797814943) bank129584 +129585 POINT(40.54900236904736 74.12318330978447) bank129585 +129586 POINT(40.586438698823386 73.0224954803749) bank129586 +129587 POINT(40.915223024271164 73.7964133938533) bank129587 +129588 POINT(39.75388213941681 74.66781443254904) bank129588 +129589 POINT(41.00497270652505 74.64273284397079) bank129589 +129590 POINT(40.65944954143008 73.90309711702606) bank129590 +129591 POINT(40.883152480180776 74.06707279022473) bank129591 +129592 POINT(41.49097083504635 73.08257942575874) bank129592 +129593 POINT(40.19785012223548 73.45814480273792) bank129593 +129594 POINT(41.70211706448678 73.37115303460104) bank129594 +129595 POINT(40.08562956655385 73.72319633756045) bank129595 +129596 POINT(41.43404705771215 74.92369342955949) bank129596 +129597 POINT(40.06717603895041 74.24163147594459) bank129597 +129598 POINT(40.03831080460766 73.35965563849024) bank129598 +129599 POINT(41.05481070186121 74.25350199601849) bank129599 +129600 POINT(39.75029857163275 74.9263265859834) bank129600 +129601 POINT(40.99568489446832 73.23789273725076) bank129601 +129602 POINT(41.44901723109243 73.91025244587568) bank129602 +129603 POINT(41.46882455929406 73.14267359832974) bank129603 +129604 POINT(39.87133140559821 73.54742518643172) bank129604 +129605 POINT(40.192502639406115 74.79226352219217) bank129605 +129606 POINT(41.011871574974265 74.31899600024728) bank129606 +129607 POINT(41.310769246884234 74.52549191179587) bank129607 +129608 POINT(40.54227283705452 74.81988799100867) bank129608 +129609 POINT(39.86108262812813 74.62223002987325) bank129609 +129610 POINT(41.688528702701625 74.51493071263103) bank129610 +129611 POINT(41.02187141638424 73.30687292832698) bank129611 +129612 POINT(40.370013263367866 73.77297769272957) bank129612 +129613 POINT(40.969968901921995 74.0687808481059) bank129613 +129614 POINT(40.99561037470631 73.79646572157424) bank129614 +129615 POINT(40.36656184755914 74.20823151374032) bank129615 +129616 POINT(40.80362428104037 73.40903455245433) bank129616 +129617 POINT(40.76522587469788 73.8162856734617) bank129617 +129618 POINT(40.489138447872485 73.44961009349626) bank129618 +129619 POINT(40.70845478464745 73.96374509755115) bank129619 +129620 POINT(40.257262279482404 74.2036527587993) bank129620 +129621 POINT(39.856331016306335 73.3855398326296) bank129621 +129622 POINT(41.46886842979293 73.69004373712096) bank129622 +129623 POINT(40.697178751466296 74.96965664354994) bank129623 +129624 POINT(40.82433367978995 74.4403375191692) bank129624 +129625 POINT(39.73276866696005 73.09884473508147) bank129625 +129626 POINT(40.0899502923242 73.8345661179805) bank129626 +129627 POINT(40.22360569341146 74.69936160749756) bank129627 +129628 POINT(41.42435769348006 74.26393172119) bank129628 +129629 POINT(40.18665033438121 74.22324797969445) bank129629 +129630 POINT(39.87716488894707 74.6780261464436) bank129630 +129631 POINT(39.95621750383101 74.64789062537933) bank129631 +129632 POINT(39.96043852114931 74.44620270951046) bank129632 +129633 POINT(41.110652660137795 73.46290688871262) bank129633 +129634 POINT(40.01569472936785 73.95882214865307) bank129634 +129635 POINT(41.40297975231161 74.931869883274) bank129635 +129636 POINT(40.89676207310186 73.78549069723768) bank129636 +129637 POINT(40.850085264803255 74.51719864539122) bank129637 +129638 POINT(40.85803661129793 73.65994486509493) bank129638 +129639 POINT(39.77861675964002 73.26334897637238) bank129639 +129640 POINT(40.90452993090901 74.25489924946616) bank129640 +129641 POINT(40.31162234705821 73.32980406589928) bank129641 +129642 POINT(40.466682766838694 74.04723392246471) bank129642 +129643 POINT(41.47987437350508 73.59546795002966) bank129643 +129644 POINT(40.92403985860345 74.59761052153442) bank129644 +129645 POINT(41.349773648316024 74.52626831580511) bank129645 +129646 POINT(40.57148331967689 74.93614681256487) bank129646 +129647 POINT(40.59391912875252 74.69097462346305) bank129647 +129648 POINT(41.47830866179388 73.86080842058567) bank129648 +129649 POINT(39.83031589995551 73.19349509290488) bank129649 +129650 POINT(40.18710234963989 73.87794230187023) bank129650 +129651 POINT(41.084275803883344 73.99944748776625) bank129651 +129652 POINT(41.2460347466562 73.82773353799467) bank129652 +129653 POINT(39.82620854097413 73.5660999346896) bank129653 +129654 POINT(40.64560538417535 73.94491194907958) bank129654 +129655 POINT(41.167001390323406 74.56151756723132) bank129655 +129656 POINT(39.93994453633465 74.50055802509841) bank129656 +129657 POINT(40.616479928105264 74.44730732745182) bank129657 +129658 POINT(41.18960595462166 73.08963901915803) bank129658 +129659 POINT(41.10241557006938 73.08156837896502) bank129659 +129660 POINT(41.55873049633509 74.5073537329242) bank129660 +129661 POINT(40.363307532709186 74.85501784351433) bank129661 +129662 POINT(40.421690712761084 74.29326499119928) bank129662 +129663 POINT(40.207685263088905 74.27095221348407) bank129663 +129664 POINT(40.67890432336797 74.70127627995602) bank129664 +129665 POINT(40.19919505279024 74.50480471909479) bank129665 +129666 POINT(40.649714602021454 74.59554948397007) bank129666 +129667 POINT(40.593281301997 74.35748491721932) bank129667 +129668 POINT(40.092104184297234 74.01085878889785) bank129668 +129669 POINT(40.880070591128785 73.31805244124926) bank129669 +129670 POINT(40.92918521688757 73.54291981211435) bank129670 +129671 POINT(39.98880319629099 73.23033183243369) bank129671 +129672 POINT(40.48647135554772 73.10089542387311) bank129672 +129673 POINT(40.39672126455704 74.53447146104398) bank129673 +129674 POINT(40.05778922657473 73.91085641084005) bank129674 +129675 POINT(39.99612460210781 73.71295798088303) bank129675 +129676 POINT(40.70759267606303 74.81214886459063) bank129676 +129677 POINT(41.56299572551784 73.83718326510129) bank129677 +129678 POINT(40.26437197612192 73.09484119370204) bank129678 +129679 POINT(40.153245640590114 74.35144159426326) bank129679 +129680 POINT(41.111759279716935 74.40510730109949) bank129680 +129681 POINT(41.58685992725038 74.02651500717204) bank129681 +129682 POINT(40.914859603254165 73.03648680659315) bank129682 +129683 POINT(40.062164394496605 73.50268347470974) bank129683 +129684 POINT(40.15040021611441 73.8942375478314) bank129684 +129685 POINT(41.14220672258359 73.22348452520546) bank129685 +129686 POINT(40.20385881483436 74.12100390389902) bank129686 +129687 POINT(40.031901313906424 73.66923420879444) bank129687 +129688 POINT(40.039616561834585 74.11806155560281) bank129688 +129689 POINT(40.94987580718544 73.70642754628787) bank129689 +129690 POINT(40.121211089314954 73.1257809834017) bank129690 +129691 POINT(41.05469236534131 74.02485818310655) bank129691 +129692 POINT(39.9621740434952 73.75771782193688) bank129692 +129693 POINT(40.32686649718175 75.00143340371915) bank129693 +129694 POINT(41.59819074673224 73.20869741655912) bank129694 +129695 POINT(40.084813827377054 74.45138392057552) bank129695 +129696 POINT(41.09310372504943 74.97679695878303) bank129696 +129697 POINT(40.294802712803886 74.40865681864602) bank129697 +129698 POINT(41.433835643602926 73.38623798732385) bank129698 +129699 POINT(39.81787998990199 73.08743984836525) bank129699 +129700 POINT(41.12077926590897 74.76262475906354) bank129700 +129701 POINT(40.375506603473774 73.85481951737223) bank129701 +129702 POINT(40.78193451995815 74.95959969542271) bank129702 +129703 POINT(41.04351005417453 73.72070689586496) bank129703 +129704 POINT(40.64762593097582 73.55641033390607) bank129704 +129705 POINT(41.223845059516115 73.46277726939425) bank129705 +129706 POINT(40.42983577182298 73.00877800663528) bank129706 +129707 POINT(41.02272887096763 73.09262731437099) bank129707 +129708 POINT(40.87619680197863 74.41536321064811) bank129708 +129709 POINT(40.37662421411943 74.38720763288468) bank129709 +129710 POINT(40.3970985457817 74.99950712108559) bank129710 +129711 POINT(39.8265930880901 74.90917712679888) bank129711 +129712 POINT(41.56221452461833 74.3726907091083) bank129712 +129713 POINT(40.27950413639233 73.72198109001255) bank129713 +129714 POINT(41.707660265640094 74.89541480208729) bank129714 +129715 POINT(41.70203852468306 73.60991835120771) bank129715 +129716 POINT(41.396464533881314 73.99254046422429) bank129716 +129717 POINT(41.027994495553415 73.81592986445659) bank129717 +129718 POINT(40.655435547937934 73.22453802253321) bank129718 +129719 POINT(41.04171758373047 74.1328333342153) bank129719 +129720 POINT(40.41690987937083 73.39359628670239) bank129720 +129721 POINT(40.463442968377635 73.3569666860198) bank129721 +129722 POINT(39.90610362414365 74.3419019095484) bank129722 +129723 POINT(40.8500011462488 74.33895266587653) bank129723 +129724 POINT(41.42585411903322 74.62147344099719) bank129724 +129725 POINT(41.57318939531989 73.5729086803364) bank129725 +129726 POINT(41.04618184262672 74.99249808589762) bank129726 +129727 POINT(40.01655176942981 73.7570380666868) bank129727 +129728 POINT(40.566872634297155 73.47756728921848) bank129728 +129729 POINT(40.86169644766509 73.99420315496434) bank129729 +129730 POINT(41.58978751012639 74.34407969917518) bank129730 +129731 POINT(40.715580334238325 73.71594225396048) bank129731 +129732 POINT(41.4870447246376 74.13430841087458) bank129732 +129733 POINT(40.6293162891274 73.30368135775794) bank129733 +129734 POINT(40.272723881387165 74.9629258448904) bank129734 +129735 POINT(40.45649586418473 73.72470604728515) bank129735 +129736 POINT(40.28760097745383 74.24919169499661) bank129736 +129737 POINT(40.79916715546596 73.02405743026371) bank129737 +129738 POINT(41.611529746011726 74.2317391101007) bank129738 +129739 POINT(41.24371546310193 74.47812049585481) bank129739 +129740 POINT(39.76674433199513 73.012451980977) bank129740 +129741 POINT(40.582659525202715 74.34480510761998) bank129741 +129742 POINT(40.12439826082618 74.70171077723934) bank129742 +129743 POINT(39.75400433696584 74.76058627706772) bank129743 +129744 POINT(40.92852566760346 73.1656478803286) bank129744 +129745 POINT(39.83424654987973 73.8228193028005) bank129745 +129746 POINT(41.042096307357355 73.84499777702057) bank129746 +129747 POINT(41.34094452231111 73.14819785906123) bank129747 +129748 POINT(41.39273186677599 73.6034314344093) bank129748 +129749 POINT(40.48849916897073 73.85337423746617) bank129749 +129750 POINT(40.39242972413506 74.59588368651194) bank129750 +129751 POINT(41.71088328804271 73.13076897163891) bank129751 +129752 POINT(39.76452363506254 73.71254203563231) bank129752 +129753 POINT(41.423049829611536 73.48291167051721) bank129753 +129754 POINT(40.72964831029746 74.77376960453644) bank129754 +129755 POINT(41.21636670981989 73.8835518118094) bank129755 +129756 POINT(40.65006475078319 73.71396860017344) bank129756 +129757 POINT(40.932717776455384 74.2173785807484) bank129757 +129758 POINT(40.55773976276944 74.69221501641438) bank129758 +129759 POINT(41.68204699783614 73.98892996332401) bank129759 +129760 POINT(40.63317924101938 73.75473508677022) bank129760 +129761 POINT(41.52914697361588 74.4898771242956) bank129761 +129762 POINT(40.6082813901706 74.57579976272582) bank129762 +129763 POINT(41.00563330371243 74.8913024724575) bank129763 +129764 POINT(40.34646715194291 74.36530982781439) bank129764 +129765 POINT(40.761202405638194 74.64442646143748) bank129765 +129766 POINT(41.13976900067317 74.44402324517576) bank129766 +129767 POINT(40.92477109643838 73.68866342155232) bank129767 +129768 POINT(39.895746169917636 74.89048493713275) bank129768 +129769 POINT(40.492514635398855 73.2339541988572) bank129769 +129770 POINT(41.400729868916684 73.45317539239218) bank129770 +129771 POINT(40.226690575204415 73.36161202556237) bank129771 +129772 POINT(40.02903104380228 74.4574269711512) bank129772 +129773 POINT(41.089464559667476 73.82654477610903) bank129773 +129774 POINT(39.98219116391037 74.70487257158774) bank129774 +129775 POINT(40.97682294135681 74.95515329169385) bank129775 +129776 POINT(41.16645109957222 74.98956001871599) bank129776 +129777 POINT(40.2185563691438 74.30951649710474) bank129777 +129778 POINT(40.616698416893854 73.08599099418997) bank129778 +129779 POINT(41.70857847999282 74.67028024679405) bank129779 +129780 POINT(41.37957850031899 73.90635418509747) bank129780 +129781 POINT(40.94174302996091 74.63266805495711) bank129781 +129782 POINT(40.99720945935775 74.12036432609578) bank129782 +129783 POINT(41.15008692689145 74.46305360729959) bank129783 +129784 POINT(41.18362631795366 73.31555597685178) bank129784 +129785 POINT(39.842423664232896 74.64645605427455) bank129785 +129786 POINT(40.59873044066738 74.86933981391856) bank129786 +129787 POINT(40.20861954452774 74.26251880770575) bank129787 +129788 POINT(40.12827731457935 74.5022120295936) bank129788 +129789 POINT(40.549165842244435 73.6400969509627) bank129789 +129790 POINT(40.4364630421353 74.95331282502153) bank129790 +129791 POINT(40.18573531630288 73.28574381420025) bank129791 +129792 POINT(41.50040929331976 73.70178842384942) bank129792 +129793 POINT(39.78684246671465 73.41763203459664) bank129793 +129794 POINT(39.87981122425228 74.34635694661704) bank129794 +129795 POINT(41.65181440830568 73.96324835440483) bank129795 +129796 POINT(41.04314528732677 73.08184962682864) bank129796 +129797 POINT(40.7541326176435 73.60813826370878) bank129797 +129798 POINT(40.94788704006404 73.93126279376993) bank129798 +129799 POINT(41.65488801615504 73.95201015665356) bank129799 +129800 POINT(40.391633386743806 74.67063541046417) bank129800 +129801 POINT(41.13348818918309 73.2197080705588) bank129801 +129802 POINT(41.483433790935024 73.87285821604962) bank129802 +129803 POINT(40.79524309140244 73.95892910926926) bank129803 +129804 POINT(40.262160347165924 74.76383642525215) bank129804 +129805 POINT(40.20759457151958 73.61463518780106) bank129805 +129806 POINT(39.744538533649184 73.87683714346548) bank129806 +129807 POINT(40.82584783552055 74.75365306389047) bank129807 +129808 POINT(40.183846136647354 74.4155136226269) bank129808 +129809 POINT(41.207077623289486 74.8219199271051) bank129809 +129810 POINT(39.81364040648221 74.63183304176835) bank129810 +129811 POINT(41.53171919452297 74.1130984280086) bank129811 +129812 POINT(40.87234571082704 74.8116663062065) bank129812 +129813 POINT(40.7948351363261 74.54485966522329) bank129813 +129814 POINT(40.919444130602855 74.05990426213295) bank129814 +129815 POINT(41.56326160065248 73.53749363010883) bank129815 +129816 POINT(40.72388361793058 74.80286626538798) bank129816 +129817 POINT(40.990670830208956 74.29043818947636) bank129817 +129818 POINT(39.89917637427853 74.53557486976892) bank129818 +129819 POINT(39.83272989628331 73.09704237138378) bank129819 +129820 POINT(40.089979336955764 74.74541556645737) bank129820 +129821 POINT(40.392961709784146 73.80275356907467) bank129821 +129822 POINT(40.081526098419424 74.88669716096358) bank129822 +129823 POINT(40.10794473694401 73.34692973943241) bank129823 +129824 POINT(39.8467983025635 73.50586201480252) bank129824 +129825 POINT(41.71050743512603 74.46125582288678) bank129825 +129826 POINT(39.90999381929991 73.58140679105529) bank129826 +129827 POINT(41.29814456109572 73.25943060751908) bank129827 +129828 POINT(41.41238446402047 73.47566804717528) bank129828 +129829 POINT(40.407217739777124 74.4534664207255) bank129829 +129830 POINT(39.77019684425908 73.12165261004975) bank129830 +129831 POINT(41.369675738658565 74.93931835204509) bank129831 +129832 POINT(40.82038447264386 73.91718821631724) bank129832 +129833 POINT(41.552246558464184 74.9355499870028) bank129833 +129834 POINT(41.433626613098454 74.90673568004594) bank129834 +129835 POINT(39.73388887837583 73.38829736926374) bank129835 +129836 POINT(40.87647248330338 74.31006467546752) bank129836 +129837 POINT(40.13531575988981 74.54158632942973) bank129837 +129838 POINT(41.60145087888339 73.03796244933767) bank129838 +129839 POINT(41.636445296796914 73.08628153221167) bank129839 +129840 POINT(39.89681478909565 73.15317666488173) bank129840 +129841 POINT(41.1694532756717 74.10974843595191) bank129841 +129842 POINT(41.01231999526998 73.47654988727302) bank129842 +129843 POINT(40.941966047393315 74.2059855268569) bank129843 +129844 POINT(40.934246856628114 73.68109970558295) bank129844 +129845 POINT(41.47666913441434 74.48300380500233) bank129845 +129846 POINT(39.720483941380095 74.05670514801947) bank129846 +129847 POINT(39.989324667765715 73.63352412093953) bank129847 +129848 POINT(41.037767972586 73.56196447184787) bank129848 +129849 POINT(40.066796027695496 74.67174440868561) bank129849 +129850 POINT(40.47948069315086 73.50789874998563) bank129850 +129851 POINT(40.311225049793066 73.23826699756872) bank129851 +129852 POINT(41.05850663196259 74.0781712914748) bank129852 +129853 POINT(41.32369321110139 74.77211607006795) bank129853 +129854 POINT(40.295673321471526 73.87737678911738) bank129854 +129855 POINT(41.45020911231666 74.51417957170587) bank129855 +129856 POINT(41.374571379779674 73.1491833161816) bank129856 +129857 POINT(40.42562700097086 74.94493541804374) bank129857 +129858 POINT(41.02362293390231 73.04391705782506) bank129858 +129859 POINT(41.61414882986592 73.08989690317364) bank129859 +129860 POINT(40.21647404847706 73.24199184663902) bank129860 +129861 POINT(40.67016743359193 74.73943570979857) bank129861 +129862 POINT(40.25153270876392 74.976520791971) bank129862 +129863 POINT(41.326313929449526 74.5405793365497) bank129863 +129864 POINT(40.35310538818901 73.42445212788797) bank129864 +129865 POINT(40.97770680888084 74.93392901565447) bank129865 +129866 POINT(41.15480697659942 73.57264750203316) bank129866 +129867 POINT(41.239918363272196 74.06511453607557) bank129867 +129868 POINT(40.15229942181345 74.22398753349754) bank129868 +129869 POINT(41.298670423391435 73.67568266201805) bank129869 +129870 POINT(40.35634304974212 73.89571364949283) bank129870 +129871 POINT(41.2652379775801 74.3140514988542) bank129871 +129872 POINT(40.37224874884611 74.72978405522782) bank129872 +129873 POINT(39.9398807361345 74.85156333438934) bank129873 +129874 POINT(40.137654321380666 73.47052466265421) bank129874 +129875 POINT(40.039235512142646 74.91642480410049) bank129875 +129876 POINT(40.2802165280956 73.76276718642343) bank129876 +129877 POINT(41.576606770922126 74.50581371156295) bank129877 +129878 POINT(40.829062581436105 74.95045111759556) bank129878 +129879 POINT(40.76448533098546 74.00928780515956) bank129879 +129880 POINT(40.471476143146965 74.70684861321645) bank129880 +129881 POINT(41.66852915539504 73.87541876904967) bank129881 +129882 POINT(41.551549522055204 73.41526986685747) bank129882 +129883 POINT(39.90939642137723 74.12057657054261) bank129883 +129884 POINT(40.05672809523308 74.12038317594435) bank129884 +129885 POINT(40.11136655374658 73.74514350538963) bank129885 +129886 POINT(41.1750026291351 74.01133370631437) bank129886 +129887 POINT(40.80301916856199 73.07467494764109) bank129887 +129888 POINT(39.79735722550482 74.32573129607484) bank129888 +129889 POINT(40.86753749044015 73.4174280683723) bank129889 +129890 POINT(41.25592191543969 73.4154929540296) bank129890 +129891 POINT(40.30708228092764 74.7592359275498) bank129891 +129892 POINT(41.29468732099473 74.7659570827345) bank129892 +129893 POINT(40.20556832445783 73.84232061133685) bank129893 +129894 POINT(40.60299832482408 73.47227175950592) bank129894 +129895 POINT(40.8747302145595 74.78140830738076) bank129895 +129896 POINT(39.970636798174944 74.07412242269135) bank129896 +129897 POINT(41.517397892035824 73.93280959186107) bank129897 +129898 POINT(40.27110998315357 74.38276440644654) bank129898 +129899 POINT(40.40973295101365 74.08860564306926) bank129899 +129900 POINT(40.2575570771293 74.26275490810258) bank129900 +129901 POINT(39.99566493163237 73.56842247310503) bank129901 +129902 POINT(41.59760857372979 73.19447921191119) bank129902 +129903 POINT(40.846717702563296 74.80011459729455) bank129903 +129904 POINT(40.80189138363828 74.02280765132356) bank129904 +129905 POINT(40.20801170368297 74.35254502489693) bank129905 +129906 POINT(39.75606185722879 73.90611067155406) bank129906 +129907 POINT(40.169071215637544 73.77101380226107) bank129907 +129908 POINT(40.407253733629894 74.17561803327085) bank129908 +129909 POINT(40.45616566416476 73.06276114431981) bank129909 +129910 POINT(40.76541010727651 74.23577384277047) bank129910 +129911 POINT(41.28187483934413 74.8812032076941) bank129911 +129912 POINT(40.33143070864779 73.67864762727419) bank129912 +129913 POINT(39.95068240427692 73.95708879513826) bank129913 +129914 POINT(40.02593422032292 74.61228699550949) bank129914 +129915 POINT(40.592100284232735 74.69076447045155) bank129915 +129916 POINT(39.99510946243789 73.98235293686454) bank129916 +129917 POINT(41.25157731867196 74.86791337277516) bank129917 +129918 POINT(41.676906692943355 73.63887180103795) bank129918 +129919 POINT(39.8483735135134 73.32403997240921) bank129919 +129920 POINT(40.75666584267167 73.50102097369805) bank129920 +129921 POINT(40.51463501056499 74.53358776139258) bank129921 +129922 POINT(41.57989114208237 73.23051430518605) bank129922 +129923 POINT(40.02093465552778 73.99633172430903) bank129923 +129924 POINT(40.794712659150285 73.98273965205529) bank129924 +129925 POINT(41.03556621782775 73.04012033147282) bank129925 +129926 POINT(40.07520407985613 74.71427348445032) bank129926 +129927 POINT(41.6914191919549 74.34342658257474) bank129927 +129928 POINT(40.36568115686496 74.90972793986623) bank129928 +129929 POINT(41.474034017717926 74.07242553158933) bank129929 +129930 POINT(40.77182786702224 74.19525469488129) bank129930 +129931 POINT(41.47806518688623 74.48872688652983) bank129931 +129932 POINT(40.49539507569312 73.4569057713222) bank129932 +129933 POINT(41.39862417157409 74.7829368585894) bank129933 +129934 POINT(41.70019761469481 73.29279809431392) bank129934 +129935 POINT(40.70179899660345 74.65073418415946) bank129935 +129936 POINT(39.9009980593982 73.54664529327377) bank129936 +129937 POINT(39.731942775285226 73.10589178560755) bank129937 +129938 POINT(39.8944253217588 73.92690886058128) bank129938 +129939 POINT(40.48326599936448 73.33002630783777) bank129939 +129940 POINT(41.122398796576675 73.11139827271008) bank129940 +129941 POINT(40.48583611438792 73.836306186887) bank129941 +129942 POINT(40.43308046642808 74.81632696825383) bank129942 +129943 POINT(40.39711680978949 73.50530431119218) bank129943 +129944 POINT(41.15887537221165 74.48929140286513) bank129944 +129945 POINT(41.658954558265556 74.13239228319287) bank129945 +129946 POINT(41.21223436495228 73.328171033371) bank129946 +129947 POINT(41.09192917924573 73.57951335756778) bank129947 +129948 POINT(40.76917683635547 74.00948271768785) bank129948 +129949 POINT(41.136730562109115 74.23481264643424) bank129949 +129950 POINT(41.49133507771494 74.53873598831666) bank129950 +129951 POINT(41.17027299801066 73.02546631458459) bank129951 +129952 POINT(40.60003547514402 74.44647588585141) bank129952 +129953 POINT(39.95974598596979 74.59517148963097) bank129953 +129954 POINT(39.924864459341556 74.63251381658995) bank129954 +129955 POINT(41.040965190021225 73.23088963696783) bank129955 +129956 POINT(40.03394608341641 74.22918472189419) bank129956 +129957 POINT(40.507189711091364 74.35502380949592) bank129957 +129958 POINT(40.96127569901814 73.80576634566492) bank129958 +129959 POINT(41.4053954769158 73.45223025707895) bank129959 +129960 POINT(41.36139980244253 74.04826422583612) bank129960 +129961 POINT(40.3695476643683 74.83735112566573) bank129961 +129962 POINT(41.19332283946202 73.32680081505016) bank129962 +129963 POINT(41.3407454842349 73.9909811499783) bank129963 +129964 POINT(41.2423237631291 74.46296314968106) bank129964 +129965 POINT(41.35159710013954 73.6103911003771) bank129965 +129966 POINT(40.35521919221544 73.98008413848187) bank129966 +129967 POINT(40.99186272499603 73.24996123094311) bank129967 +129968 POINT(40.80414777740274 74.2789341753813) bank129968 +129969 POINT(41.553006594350016 73.5383766007354) bank129969 +129970 POINT(40.813417931021974 74.30448989259212) bank129970 +129971 POINT(41.3882546521168 74.89526219472556) bank129971 +129972 POINT(41.32059621903925 73.74573075872519) bank129972 +129973 POINT(40.76931525380461 73.55740112554278) bank129973 +129974 POINT(39.85037996197023 74.78170747385933) bank129974 +129975 POINT(40.155685034196125 73.53076196452865) bank129975 +129976 POINT(39.7500544466991 73.50917861990503) bank129976 +129977 POINT(41.610214808972316 73.67810010466233) bank129977 +129978 POINT(41.01076578456871 74.46433059536112) bank129978 +129979 POINT(40.51252272643303 73.80154314787904) bank129979 +129980 POINT(40.471573742495636 74.14259010517588) bank129980 +129981 POINT(41.594666776651565 73.9825823865616) bank129981 +129982 POINT(41.712089822409204 74.10870978852687) bank129982 +129983 POINT(40.946258586131 74.37300687607072) bank129983 +129984 POINT(41.16676946087886 74.74243186481787) bank129984 +129985 POINT(40.446972996667434 74.64435920883585) bank129985 +129986 POINT(41.37692103282298 73.85020041905166) bank129986 +129987 POINT(40.23747966998797 74.20177960687306) bank129987 +129988 POINT(40.01703674117577 73.34450657118335) bank129988 +129989 POINT(40.93653406184117 73.93722613447663) bank129989 +129990 POINT(40.49385842919418 73.49946084006665) bank129990 +129991 POINT(40.661197039755564 74.27762884974256) bank129991 +129992 POINT(40.40388822550119 74.98324468336958) bank129992 +129993 POINT(41.40476856316484 73.33520428240763) bank129993 +129994 POINT(39.97357761432988 74.66883187804697) bank129994 +129995 POINT(39.76365110382397 74.94318421821232) bank129995 +129996 POINT(41.414062429639586 74.84549538611498) bank129996 +129997 POINT(41.54616873602891 74.53266112298189) bank129997 +129998 POINT(41.62640546120484 73.29080168484887) bank129998 +129999 POINT(41.60088597445071 73.13533663538044) bank129999 +130000 POINT(41.094639072606384 73.38655029754517) bank130000 +130001 POINT(41.20508232545659 73.98292529056688) bank130001 +130002 POINT(40.351043417599506 73.27314429565828) bank130002 +130003 POINT(40.23343184457117 73.94183557353378) bank130003 +130004 POINT(41.551010497095014 74.59369874455392) bank130004 +130005 POINT(40.262604044195974 74.53896698121572) bank130005 +130006 POINT(41.19474682141003 73.75680129441209) bank130006 +130007 POINT(41.04263275330833 74.37994801930115) bank130007 +130008 POINT(41.363218926533705 74.18745547194382) bank130008 +130009 POINT(41.52560562254597 74.617939040447) bank130009 +130010 POINT(40.30494606696372 73.3568649992449) bank130010 +130011 POINT(40.27783333398657 74.39677702681756) bank130011 +130012 POINT(41.42540108893371 73.08986773901574) bank130012 +130013 POINT(41.33430073134755 74.45536161704075) bank130013 +130014 POINT(41.439354640950604 74.35145779620031) bank130014 +130015 POINT(39.99852923271923 73.04559002290931) bank130015 +130016 POINT(40.01330384636286 74.62979491286697) bank130016 +130017 POINT(40.61799880914944 74.20842078952172) bank130017 +130018 POINT(40.55145224493979 73.61505356888452) bank130018 +130019 POINT(41.49152992324026 74.08489452201253) bank130019 +130020 POINT(40.424608427737326 73.66970583985646) bank130020 +130021 POINT(40.727137728354286 74.47809666881139) bank130021 +130022 POINT(40.57868911803916 73.47214555039852) bank130022 +130023 POINT(41.642422361836836 73.01522211846013) bank130023 +130024 POINT(41.08783818680802 73.83308688981482) bank130024 +130025 POINT(41.06105889407527 74.03726121630632) bank130025 +130026 POINT(41.625160203167404 73.58641005531895) bank130026 +130027 POINT(41.35839623165128 73.580892107551) bank130027 +130028 POINT(40.41045545063082 74.74538164670159) bank130028 +130029 POINT(40.395304042117886 74.5941990452766) bank130029 +130030 POINT(40.582556985179025 73.476343204534) bank130030 +130031 POINT(39.981737099323794 73.70526135647229) bank130031 +130032 POINT(41.62275464444818 74.32877693649853) bank130032 +130033 POINT(40.039299949500446 74.63191938384627) bank130033 +130034 POINT(40.189605088812016 73.5967723587805) bank130034 +130035 POINT(40.5203887237191 73.66066083684768) bank130035 +130036 POINT(40.096810901895445 73.4234703412292) bank130036 +130037 POINT(40.895303831755996 74.00678935682656) bank130037 +130038 POINT(41.452784203162416 74.10765063955212) bank130038 +130039 POINT(41.24914569674146 73.44704350692369) bank130039 +130040 POINT(40.50589860081367 74.59976154981749) bank130040 +130041 POINT(40.876703609774594 74.55397332564614) bank130041 +130042 POINT(40.18839138002211 73.49939730382401) bank130042 +130043 POINT(41.65247355838221 74.46734686110678) bank130043 +130044 POINT(41.490295862475705 75.0010855108307) bank130044 +130045 POINT(40.371462818169036 74.8823985967706) bank130045 +130046 POINT(40.56934157317014 74.41542432570682) bank130046 +130047 POINT(40.9454239154441 74.50510277030754) bank130047 +130048 POINT(41.362011040537546 73.8829696696908) bank130048 +130049 POINT(39.876212062971554 73.75014668775049) bank130049 +130050 POINT(41.1964317540258 73.06610503074204) bank130050 +130051 POINT(40.27460162110629 74.22189561849288) bank130051 +130052 POINT(40.9252231832123 74.90767164113409) bank130052 +130053 POINT(39.750560531951585 73.23302820016563) bank130053 +130054 POINT(40.28401219632755 74.38171446592958) bank130054 +130055 POINT(41.42817112236182 73.86598722079361) bank130055 +130056 POINT(40.5526970286613 73.22585167234436) bank130056 +130057 POINT(40.53656146553944 73.73356979196157) bank130057 +130058 POINT(40.03623579541071 74.517918880724) bank130058 +130059 POINT(39.87049403735145 73.91970743844968) bank130059 +130060 POINT(41.62900635463258 74.90243324943238) bank130060 +130061 POINT(41.43235701603979 74.49761092487529) bank130061 +130062 POINT(41.460895236649435 74.70448201532845) bank130062 +130063 POINT(40.19917440867027 74.89968348975792) bank130063 +130064 POINT(41.1583874949559 73.07083801987142) bank130064 +130065 POINT(39.8652450519791 74.80073206426951) bank130065 +130066 POINT(40.21259920922827 73.61322378049323) bank130066 +130067 POINT(39.88589198558435 73.33181982109942) bank130067 +130068 POINT(41.488103964690794 74.95462713902367) bank130068 +130069 POINT(41.30796933277651 73.35511316180337) bank130069 +130070 POINT(40.14680851539814 73.06040889562564) bank130070 +130071 POINT(41.07106781087516 73.65844344679888) bank130071 +130072 POINT(41.11783555524204 74.04061991961362) bank130072 +130073 POINT(40.0333300316649 73.63949143458908) bank130073 +130074 POINT(40.020470920555155 74.84863041235235) bank130074 +130075 POINT(40.26286827470417 74.00981765447398) bank130075 +130076 POINT(41.3930697376745 74.45418126156773) bank130076 +130077 POINT(40.160012131407115 74.59595892087518) bank130077 +130078 POINT(41.481989868546094 73.09964554421866) bank130078 +130079 POINT(40.91963923334884 73.67388279035987) bank130079 +130080 POINT(40.16075079123908 73.99506186601087) bank130080 +130081 POINT(39.98573806749901 73.27296747989439) bank130081 +130082 POINT(41.70316577963674 74.58425925537935) bank130082 +130083 POINT(39.93860296835961 74.57878702613007) bank130083 +130084 POINT(40.3900421286285 73.05306642816029) bank130084 +130085 POINT(39.95672852441206 74.89509525203177) bank130085 +130086 POINT(40.071376103934135 73.1236898026925) bank130086 +130087 POINT(41.60089362571027 73.49198396608655) bank130087 +130088 POINT(40.96929800688518 73.36531444882081) bank130088 +130089 POINT(41.582344081091236 73.77216313392317) bank130089 +130090 POINT(40.48249585067687 73.04748582470715) bank130090 +130091 POINT(40.21173938014335 74.07600017223506) bank130091 +130092 POINT(41.15611057577625 74.90725760819946) bank130092 +130093 POINT(41.57629062701762 74.38928495160549) bank130093 +130094 POINT(41.43365578742285 74.90245608981276) bank130094 +130095 POINT(41.51294796163242 73.30695016751355) bank130095 +130096 POINT(40.406686215746134 74.76072377075374) bank130096 +130097 POINT(41.35953032271152 74.75544261284992) bank130097 +130098 POINT(40.118245665419686 74.45823256125612) bank130098 +130099 POINT(40.041834820544715 74.06875007239177) bank130099 +130100 POINT(39.99729879271902 74.2093254887412) bank130100 +130101 POINT(39.75446912400848 74.31416419169685) bank130101 +130102 POINT(41.577240488031876 73.47065808263461) bank130102 +130103 POINT(41.377937957686676 74.93222755635108) bank130103 +130104 POINT(40.684527505331026 74.71150951205023) bank130104 +130105 POINT(40.548128365714845 74.82876239231294) bank130105 +130106 POINT(40.365037344023094 74.68459108629618) bank130106 +130107 POINT(40.519901413917694 73.84520593571193) bank130107 +130108 POINT(40.358921538162534 73.36894670899186) bank130108 +130109 POINT(40.42215514018513 74.63737646432249) bank130109 +130110 POINT(40.299934538074545 73.68869035135552) bank130110 +130111 POINT(40.87654440263987 73.58726492453354) bank130111 +130112 POINT(40.86980509043002 74.36373470343607) bank130112 +130113 POINT(40.32379749484363 73.94377254411272) bank130113 +130114 POINT(41.622259099920974 74.82275287827594) bank130114 +130115 POINT(41.45568905034893 73.49636201334785) bank130115 +130116 POINT(41.414590412435544 74.23566608055287) bank130116 +130117 POINT(39.90493911043591 73.44394519054688) bank130117 +130118 POINT(40.76342472328228 74.34197926032434) bank130118 +130119 POINT(40.27528921342777 73.04687863970827) bank130119 +130120 POINT(40.07883382440375 73.50983288357439) bank130120 +130121 POINT(39.977333286554675 73.33482142296626) bank130121 +130122 POINT(40.006680144654474 74.60539804064267) bank130122 +130123 POINT(39.983504014640026 73.94845227820404) bank130123 +130124 POINT(40.22750727029278 74.79137429639626) bank130124 +130125 POINT(40.53857019111742 74.6887780414063) bank130125 +130126 POINT(40.89355729819596 73.02557052680147) bank130126 +130127 POINT(41.58327840658674 74.4142146944242) bank130127 +130128 POINT(41.685596280108065 73.23329063875536) bank130128 +130129 POINT(39.87713747109168 73.49069393955638) bank130129 +130130 POINT(41.04045257321592 74.55188329440503) bank130130 +130131 POINT(39.88433310026156 74.67956408799364) bank130131 +130132 POINT(40.441228233126154 73.4550950202068) bank130132 +130133 POINT(39.8079485252967 74.03315736832313) bank130133 +130134 POINT(40.717815086652536 74.36424818674332) bank130134 +130135 POINT(41.26172176942586 74.58892237934818) bank130135 +130136 POINT(40.281095173162406 74.50726476507425) bank130136 +130137 POINT(41.07156954696474 74.27759803144728) bank130137 +130138 POINT(40.25216805579052 74.71581052434333) bank130138 +130139 POINT(40.271979933113656 73.2119526469692) bank130139 +130140 POINT(39.99312356224613 74.60317044669848) bank130140 +130141 POINT(41.47057003318533 74.45829340102019) bank130141 +130142 POINT(41.115127276658285 74.87635445289729) bank130142 +130143 POINT(41.03888526046216 73.45866356582626) bank130143 +130144 POINT(40.26162319703766 73.34605545847133) bank130144 +130145 POINT(39.73737060554771 73.55953136688315) bank130145 +130146 POINT(41.4929135476703 74.07074262677376) bank130146 +130147 POINT(40.617107754117924 73.6049146900286) bank130147 +130148 POINT(40.44075201584875 74.34091641834279) bank130148 +130149 POINT(41.46419964103293 74.45051008046921) bank130149 +130150 POINT(41.41587363119717 74.59549358770295) bank130150 +130151 POINT(40.61684709798352 74.72429206943535) bank130151 +130152 POINT(41.7032202802192 73.82045787356213) bank130152 +130153 POINT(40.714128192438984 73.31861144092063) bank130153 +130154 POINT(41.64650436022348 73.56864438519528) bank130154 +130155 POINT(39.854555330987246 74.86164514100881) bank130155 +130156 POINT(41.40556129334679 74.88498067564157) bank130156 +130157 POINT(41.18834012378815 73.91427767596734) bank130157 +130158 POINT(39.71686554948612 73.43726846593052) bank130158 +130159 POINT(39.992411087639915 73.1103805901669) bank130159 +130160 POINT(40.31785783588984 74.55424502477914) bank130160 +130161 POINT(41.601121234091124 74.58969438845065) bank130161 +130162 POINT(39.95575080483374 73.62197420582442) bank130162 +130163 POINT(41.402718237956364 74.75626553126826) bank130163 +130164 POINT(41.42287566387436 73.95187413057265) bank130164 +130165 POINT(40.94376428365042 74.52011597717448) bank130165 +130166 POINT(41.476256577276395 74.71721018603209) bank130166 +130167 POINT(41.524193426875904 73.53666506924586) bank130167 +130168 POINT(39.823348483012396 73.93573449326489) bank130168 +130169 POINT(41.49888343675596 74.99692316749231) bank130169 +130170 POINT(40.403667693729496 74.6912155845504) bank130170 +130171 POINT(41.176831816331514 74.43936136911529) bank130171 +130172 POINT(41.51687579398159 73.90034234290115) bank130172 +130173 POINT(40.742879309686465 73.82104878589804) bank130173 +130174 POINT(41.25793606851494 74.08691879606066) bank130174 +130175 POINT(41.36580983331251 74.11154952487388) bank130175 +130176 POINT(41.540871101614805 74.47338266029259) bank130176 +130177 POINT(41.27494551356226 74.01298661620889) bank130177 +130178 POINT(40.90938136133723 73.97474116252315) bank130178 +130179 POINT(40.77108201564397 73.85568184412878) bank130179 +130180 POINT(40.232675999759984 73.40870476640097) bank130180 +130181 POINT(41.54185674536808 74.29874783436368) bank130181 +130182 POINT(39.8895728684802 74.88176344677996) bank130182 +130183 POINT(40.725963037722856 74.99229150860857) bank130183 +130184 POINT(40.56259049899331 74.0646046919905) bank130184 +130185 POINT(41.62666197276407 73.00830879801107) bank130185 +130186 POINT(40.39022698144159 73.57889967330362) bank130186 +130187 POINT(40.37914369403218 73.90392665222173) bank130187 +130188 POINT(41.5333530328363 73.93079478175703) bank130188 +130189 POINT(41.134969843687195 73.3683736279941) bank130189 +130190 POINT(40.081254304725164 73.49605329650439) bank130190 +130191 POINT(41.32048664173829 73.62666558524083) bank130191 +130192 POINT(41.625182617862 74.82258921899472) bank130192 +130193 POINT(40.62010804693186 74.93641453137862) bank130193 +130194 POINT(40.495918720440116 74.22044300288947) bank130194 +130195 POINT(40.0504588592483 74.65066064963582) bank130195 +130196 POINT(40.90525895648307 73.86812160606398) bank130196 +130197 POINT(40.99428696041443 73.27747216397503) bank130197 +130198 POINT(39.904846098310735 74.34440831766757) bank130198 +130199 POINT(41.45954241614296 73.28016009219779) bank130199 +130200 POINT(40.417229756198104 75.00464371269545) bank130200 +130201 POINT(40.831227894862366 73.9439516246073) bank130201 +130202 POINT(40.9740981368812 74.29404762261193) bank130202 +130203 POINT(41.20093638294242 74.0063833065427) bank130203 +130204 POINT(41.33532162770135 74.66419648423982) bank130204 +130205 POINT(40.30661923058548 73.65035611481555) bank130205 +130206 POINT(39.96179947185382 74.19573088852678) bank130206 +130207 POINT(40.489957802234386 74.77478114048176) bank130207 +130208 POINT(40.27526608832674 74.55749599635793) bank130208 +130209 POINT(40.915241400889514 73.352325001222) bank130209 +130210 POINT(39.78310018332141 74.65675107140713) bank130210 +130211 POINT(41.11310701480381 74.83407680720137) bank130211 +130212 POINT(40.82876054448679 74.42462657758526) bank130212 +130213 POINT(40.77109476487729 74.38272008798955) bank130213 +130214 POINT(40.2749314338534 74.59795923900064) bank130214 +130215 POINT(40.04561370449688 73.74958381187349) bank130215 +130216 POINT(41.35252430532952 73.56223244702443) bank130216 +130217 POINT(40.32051300050952 73.50040613826398) bank130217 +130218 POINT(41.426281856253524 73.79027385269252) bank130218 +130219 POINT(39.79899925770221 74.15395194343527) bank130219 +130220 POINT(40.961160505567875 73.89793577540374) bank130220 +130221 POINT(39.73685232602944 74.6177334067907) bank130221 +130222 POINT(39.97948115335342 73.16599414245506) bank130222 +130223 POINT(40.79355991642413 73.16002787064551) bank130223 +130224 POINT(39.779843702157486 74.61558590018628) bank130224 +130225 POINT(41.3671980334535 73.11319896228908) bank130225 +130226 POINT(41.32243957288673 74.72507784106952) bank130226 +130227 POINT(41.315735996271854 74.35607533248756) bank130227 +130228 POINT(40.01772157812021 74.90293061882718) bank130228 +130229 POINT(41.26751190362926 74.03714927953561) bank130229 +130230 POINT(41.117724016971806 74.02382639070927) bank130230 +130231 POINT(40.72883089225272 74.64038727025577) bank130231 +130232 POINT(40.32664714983008 74.21299841361413) bank130232 +130233 POINT(40.59026320458423 73.80262499746243) bank130233 +130234 POINT(39.829236616103806 74.05794635395374) bank130234 +130235 POINT(40.232165858122535 74.42782821151333) bank130235 +130236 POINT(41.104214893228615 74.14925949433108) bank130236 +130237 POINT(40.7654773019739 73.31813334725864) bank130237 +130238 POINT(40.61649836239668 73.0292422802206) bank130238 +130239 POINT(40.27543894545597 73.43276758942797) bank130239 +130240 POINT(40.8893114571397 74.49095833555577) bank130240 +130241 POINT(41.61947791351342 74.81978456237094) bank130241 +130242 POINT(41.30712175125919 73.1291697086409) bank130242 +130243 POINT(40.25479525794687 73.83805354742113) bank130243 +130244 POINT(40.69964536481435 74.76863392704213) bank130244 +130245 POINT(40.805448187364235 74.89143933758228) bank130245 +130246 POINT(39.80482729167435 74.019177859909) bank130246 +130247 POINT(40.49944284648449 74.9046177755427) bank130247 +130248 POINT(39.794763575845295 73.87959580591689) bank130248 +130249 POINT(41.12022189379531 74.13233543892264) bank130249 +130250 POINT(39.74571437063725 74.83984963084869) bank130250 +130251 POINT(41.596385454711154 74.2755020521538) bank130251 +130252 POINT(39.958637144136155 74.08885435122592) bank130252 +130253 POINT(41.07812706980842 73.95279833038371) bank130253 +130254 POINT(40.999756541930886 74.49843127645242) bank130254 +130255 POINT(41.281732512160396 74.55099166041639) bank130255 +130256 POINT(41.5890209048556 73.45367759779516) bank130256 +130257 POINT(41.165709447224735 74.7867461644929) bank130257 +130258 POINT(40.95010406018766 73.01469616730014) bank130258 +130259 POINT(41.67112860645642 73.16003643361825) bank130259 +130260 POINT(41.365095067498636 73.96902402039194) bank130260 +130261 POINT(40.66834604425178 74.80031768862825) bank130261 +130262 POINT(41.105850662739996 73.3935869708869) bank130262 +130263 POINT(40.457181091245296 74.88645219610628) bank130263 +130264 POINT(39.96225229887223 73.29461743381441) bank130264 +130265 POINT(40.625961883374536 73.2347844085805) bank130265 +130266 POINT(40.93773717445095 74.43356925976403) bank130266 +130267 POINT(40.030530242362076 73.60302120501868) bank130267 +130268 POINT(40.52105435885527 73.72709719918495) bank130268 +130269 POINT(40.29662694950998 74.92474533841673) bank130269 +130270 POINT(40.803872851110384 74.17917551528421) bank130270 +130271 POINT(39.823354369746546 73.6479744762285) bank130271 +130272 POINT(41.421953022678416 74.10656867356208) bank130272 +130273 POINT(39.85023573134249 73.37130156157198) bank130273 +130274 POINT(41.57539987741954 73.88040841140608) bank130274 +130275 POINT(39.77550415613753 73.89121620648797) bank130275 +130276 POINT(39.930911395225785 73.63957731504594) bank130276 +130277 POINT(40.74332022505529 74.72801399839406) bank130277 +130278 POINT(39.879396213536154 74.07456347964958) bank130278 +130279 POINT(40.862376244253575 73.39640736711266) bank130279 +130280 POINT(41.58992114503202 73.82875979480976) bank130280 +130281 POINT(40.95939333380159 73.24667441261323) bank130281 +130282 POINT(39.7292007117172 74.03522040312187) bank130282 +130283 POINT(40.50335561873009 73.97596603984553) bank130283 +130284 POINT(41.489090467308195 74.14446932460238) bank130284 +130285 POINT(40.8320603533651 73.085043148345) bank130285 +130286 POINT(40.362635062200084 74.10035084931968) bank130286 +130287 POINT(40.88319464946526 74.89395881975884) bank130287 +130288 POINT(41.311296314978655 73.82998088207167) bank130288 +130289 POINT(41.45336163568073 73.35201481803016) bank130289 +130290 POINT(40.704495555973224 73.83105814065121) bank130290 +130291 POINT(40.799802450573424 74.93212588500587) bank130291 +130292 POINT(40.86065708030617 73.42202783652823) bank130292 +130293 POINT(41.20066999786539 73.74766524683531) bank130293 +130294 POINT(40.79155844793592 73.0611081938031) bank130294 +130295 POINT(41.474679913965666 74.46792817648914) bank130295 +130296 POINT(40.141507551744766 73.43981264414148) bank130296 +130297 POINT(40.93208132878853 73.41116693445649) bank130297 +130298 POINT(40.91134463079781 73.73590949849456) bank130298 +130299 POINT(39.74053756975077 73.62296350836618) bank130299 +130300 POINT(40.39051238849393 73.23041905055484) bank130300 +130301 POINT(41.67809983085629 73.27099059793527) bank130301 +130302 POINT(40.36418595762293 73.361723198476) bank130302 +130303 POINT(40.25605721985034 74.72984838945251) bank130303 +130304 POINT(41.447440538261056 74.72972870165202) bank130304 +130305 POINT(39.82710880048286 73.96340647569465) bank130305 +130306 POINT(41.10120038922393 74.21361351120095) bank130306 +130307 POINT(41.116354319636464 74.53621915950274) bank130307 +130308 POINT(39.84065340542602 73.71793851722228) bank130308 +130309 POINT(41.30537081961758 73.09764588346289) bank130309 +130310 POINT(41.41912084172631 73.08129618601212) bank130310 +130311 POINT(41.604055744602455 74.65015326580568) bank130311 +130312 POINT(39.839484033810244 74.77620450266247) bank130312 +130313 POINT(40.040484764166024 74.16495209449116) bank130313 +130314 POINT(41.247771510272024 73.73877351386727) bank130314 +130315 POINT(40.268855755680065 73.37216590242062) bank130315 +130316 POINT(41.03641209359773 73.26232553169231) bank130316 +130317 POINT(41.13882125134638 74.11915259448774) bank130317 +130318 POINT(41.03888221335486 73.21779030119977) bank130318 +130319 POINT(39.99998518703759 73.33498975023707) bank130319 +130320 POINT(40.247935445154546 74.46285191259592) bank130320 +130321 POINT(41.34177452371357 73.67742190562902) bank130321 +130322 POINT(40.56698155892781 73.5921480038562) bank130322 +130323 POINT(41.03604876509704 74.87472144028472) bank130323 +130324 POINT(39.86068627551036 74.65427132374124) bank130324 +130325 POINT(39.93751245428591 73.98622465523377) bank130325 +130326 POINT(40.90293825104354 74.83679800135747) bank130326 +130327 POINT(41.28413064199551 74.96176279778587) bank130327 +130328 POINT(40.02102355807152 73.68250083630426) bank130328 +130329 POINT(41.260362071047254 74.92174273071015) bank130329 +130330 POINT(40.74198111760886 73.5713109880855) bank130330 +130331 POINT(40.15853626218467 74.63302800480676) bank130331 +130332 POINT(39.9614956997254 73.06722971020437) bank130332 +130333 POINT(40.355647065986375 74.31180341907314) bank130333 +130334 POINT(39.79619043898685 73.09748598125202) bank130334 +130335 POINT(40.02216272994141 74.16971555908275) bank130335 +130336 POINT(40.23158551884075 73.53111142037297) bank130336 +130337 POINT(40.49095039051206 74.95645563530016) bank130337 +130338 POINT(40.81472183954201 73.60224736917257) bank130338 +130339 POINT(39.96783946550413 74.71614677248462) bank130339 +130340 POINT(40.09710875758806 73.5623638466863) bank130340 +130341 POINT(41.370904482997666 74.39654323883236) bank130341 +130342 POINT(40.77022121867448 73.77901836653582) bank130342 +130343 POINT(41.55074030091646 74.73335621729228) bank130343 +130344 POINT(41.49161659901827 73.7937539575758) bank130344 +130345 POINT(40.43295883752958 74.85285588826713) bank130345 +130346 POINT(41.074005993071026 74.12409776715458) bank130346 +130347 POINT(40.252563533377476 73.41858022449327) bank130347 +130348 POINT(41.13755260365389 73.74359839604932) bank130348 +130349 POINT(40.16813583629772 73.33906519111193) bank130349 +130350 POINT(40.927947563126416 74.49931733357404) bank130350 +130351 POINT(40.279716131615835 73.21428139313184) bank130351 +130352 POINT(40.354907500689045 73.94359794376594) bank130352 +130353 POINT(40.97684782391762 74.10051218159994) bank130353 +130354 POINT(40.34909397274413 74.60313839014069) bank130354 +130355 POINT(40.53010411440505 74.69878813225127) bank130355 +130356 POINT(40.312094867521196 73.32014078856366) bank130356 +130357 POINT(40.75990802997209 73.92370634777514) bank130357 +130358 POINT(41.54272946118249 74.8159097623082) bank130358 +130359 POINT(40.93041314142774 73.80452616785433) bank130359 +130360 POINT(41.30893465919907 73.79981139761173) bank130360 +130361 POINT(40.48572073538999 74.43362216582686) bank130361 +130362 POINT(41.38859350119554 73.73042218165092) bank130362 +130363 POINT(40.6095885085808 74.84214714570052) bank130363 +130364 POINT(39.756579560179546 74.54110581695676) bank130364 +130365 POINT(41.27393620401826 74.29706689162343) bank130365 +130366 POINT(41.09129421766518 73.71838453961087) bank130366 +130367 POINT(39.74887031665134 73.5100857156585) bank130367 +130368 POINT(41.51832614712021 73.30774140589128) bank130368 +130369 POINT(40.67456833313515 73.68370358139094) bank130369 +130370 POINT(39.95910229744758 73.55720293438192) bank130370 +130371 POINT(41.61285958498285 73.0642358326531) bank130371 +130372 POINT(40.11620354008298 74.45257411853687) bank130372 +130373 POINT(40.80999438157635 74.81994206367166) bank130373 +130374 POINT(40.8637487659008 73.17140395557878) bank130374 +130375 POINT(39.76414639554189 74.84880990798511) bank130375 +130376 POINT(39.73318889508676 74.28890152391043) bank130376 +130377 POINT(39.93303720836036 73.86893134322978) bank130377 +130378 POINT(40.71100092509345 74.1665499408364) bank130378 +130379 POINT(40.03767540340978 74.25273394328168) bank130379 +130380 POINT(41.252975459358765 73.50666415255667) bank130380 +130381 POINT(40.02085296027568 74.55905385374682) bank130381 +130382 POINT(40.23816791238753 73.50136015331475) bank130382 +130383 POINT(40.16338603983837 73.13510911250279) bank130383 +130384 POINT(41.153795553126336 73.4028874891475) bank130384 +130385 POINT(40.3731377389682 74.0056039791163) bank130385 +130386 POINT(40.17388413326307 73.33740012016445) bank130386 +130387 POINT(41.31247385053162 73.21828938228583) bank130387 +130388 POINT(40.01984069709755 74.17022275612167) bank130388 +130389 POINT(41.36926371579513 74.66241794044456) bank130389 +130390 POINT(40.80988479634208 73.80289222529892) bank130390 +130391 POINT(40.20028781096745 73.17817521361152) bank130391 +130392 POINT(40.3633481780841 73.70925689498544) bank130392 +130393 POINT(40.905162251460126 73.6426951286868) bank130393 +130394 POINT(41.31208162105317 74.72915420014343) bank130394 +130395 POINT(40.7011569100165 74.5273980658113) bank130395 +130396 POINT(40.664135861321554 73.3715224604011) bank130396 +130397 POINT(41.62120936164604 74.8438255876742) bank130397 +130398 POINT(40.053932252181944 74.57711312300903) bank130398 +130399 POINT(41.26040956291066 73.79370230619365) bank130399 +130400 POINT(41.394010612042514 73.51856170990062) bank130400 +130401 POINT(41.314464509439944 74.2627753501005) bank130401 +130402 POINT(40.05459415562274 73.97335437546568) bank130402 +130403 POINT(41.501630517894704 73.91626754853081) bank130403 +130404 POINT(40.72331259142993 74.94288461630863) bank130404 +130405 POINT(41.54801271168421 74.09051837093249) bank130405 +130406 POINT(40.25638359132624 74.17757756965152) bank130406 +130407 POINT(39.739405360307025 73.17796393351736) bank130407 +130408 POINT(41.208339530517016 74.88216998868273) bank130408 +130409 POINT(41.23509673016125 74.82296836231394) bank130409 +130410 POINT(40.53915503039196 74.82384851810353) bank130410 +130411 POINT(39.782587329039835 74.96570105273366) bank130411 +130412 POINT(40.209331695340396 73.71295479397713) bank130412 +130413 POINT(40.35994263430548 73.36922497818561) bank130413 +130414 POINT(41.3855786854833 74.81314966630151) bank130414 +130415 POINT(39.74263873705899 73.91504893565362) bank130415 +130416 POINT(40.77929558033646 74.13938652285779) bank130416 +130417 POINT(39.98669998663977 74.330258892711) bank130417 +130418 POINT(39.99109105181175 74.40959831133893) bank130418 +130419 POINT(40.41172409547411 74.71020378644417) bank130419 +130420 POINT(41.25759484620101 73.1189996995111) bank130420 +130421 POINT(40.214438882345675 73.74645102274792) bank130421 +130422 POINT(39.729191703490294 74.86240649573978) bank130422 +130423 POINT(40.79731152815002 74.20350387826186) bank130423 +130424 POINT(39.829509373192096 74.51274432995415) bank130424 +130425 POINT(41.51857780420261 74.11235547262659) bank130425 +130426 POINT(40.5846484799232 73.2195585070395) bank130426 +130427 POINT(40.28599268141164 73.24628171252185) bank130427 +130428 POINT(39.95250529415559 73.89384938159692) bank130428 +130429 POINT(39.76846372404008 73.44448537133354) bank130429 +130430 POINT(40.90272529604502 73.10042481485102) bank130430 +130431 POINT(39.72290347338797 73.79408331675786) bank130431 +130432 POINT(39.73767964228593 73.40990593559029) bank130432 +130433 POINT(40.70127068682746 74.36825953475869) bank130433 +130434 POINT(41.005235177100104 73.56620529967856) bank130434 +130435 POINT(40.635709156207774 74.86394329095123) bank130435 +130436 POINT(41.49032063673333 73.89505944220689) bank130436 +130437 POINT(39.75678852778991 73.62897669873912) bank130437 +130438 POINT(41.69739139635812 73.30878282454633) bank130438 +130439 POINT(40.737739862219875 74.709263728043) bank130439 +130440 POINT(40.08344465385626 74.8351398912852) bank130440 +130441 POINT(41.40736137907618 74.37273136420711) bank130441 +130442 POINT(41.34126691775947 74.51670119273693) bank130442 +130443 POINT(41.60340537018121 73.37739242662023) bank130443 +130444 POINT(39.911285019323756 73.48853934815185) bank130444 +130445 POINT(41.380191189797145 73.57690366634134) bank130445 +130446 POINT(41.44804015750213 73.34442833953538) bank130446 +130447 POINT(40.92654885101912 73.34421094451156) bank130447 +130448 POINT(41.26460944295454 74.83680457706564) bank130448 +130449 POINT(40.28692450631046 74.61496712338821) bank130449 +130450 POINT(39.988718985207036 74.41119771629425) bank130450 +130451 POINT(41.55948485418063 73.79847469358108) bank130451 +130452 POINT(41.28597782458971 73.17071971430656) bank130452 +130453 POINT(41.69677820936291 74.44818400011766) bank130453 +130454 POINT(41.01682342010587 73.78452334257207) bank130454 +130455 POINT(40.97453260603892 73.4235896087173) bank130455 +130456 POINT(40.48538777902137 74.6825410434077) bank130456 +130457 POINT(40.27903663377172 74.88923128997497) bank130457 +130458 POINT(41.18286747498076 74.01817167471243) bank130458 +130459 POINT(41.39790006045343 74.70190337312279) bank130459 +130460 POINT(40.59526373774708 73.44628814226901) bank130460 +130461 POINT(39.947495553763616 73.66498110125819) bank130461 +130462 POINT(40.81162570526428 73.4695417330944) bank130462 +130463 POINT(40.56935356328463 73.08870529817607) bank130463 +130464 POINT(39.82578861600929 74.04736420410615) bank130464 +130465 POINT(41.63684027612417 73.88717929548075) bank130465 +130466 POINT(40.04120951608048 74.41420571621616) bank130466 +130467 POINT(40.66618357366978 74.79756421350594) bank130467 +130468 POINT(40.4567040560321 73.37221763073214) bank130468 +130469 POINT(41.39956812854377 74.20097806356526) bank130469 +130470 POINT(40.543934874490745 73.06412938719163) bank130470 +130471 POINT(40.08335816362852 73.06683367234469) bank130471 +130472 POINT(41.14555894704424 74.07752810959606) bank130472 +130473 POINT(40.69849662206037 73.29558135768772) bank130473 +130474 POINT(40.29111673168381 74.80120152287132) bank130474 +130475 POINT(39.840059818724264 73.74887984833381) bank130475 +130476 POINT(40.622950669940046 73.24425649192783) bank130476 +130477 POINT(41.446478911599904 73.52500969194871) bank130477 +130478 POINT(40.723066374766375 74.55323294959186) bank130478 +130479 POINT(40.968792420703856 74.10670740627201) bank130479 +130480 POINT(39.82311375833938 73.47629077267196) bank130480 +130481 POINT(40.863973893953144 74.72803791851807) bank130481 +130482 POINT(39.84437336172916 74.01373030488404) bank130482 +130483 POINT(39.96768771727312 73.75206944305133) bank130483 +130484 POINT(40.65048048363702 74.61273731863926) bank130484 +130485 POINT(39.85063015610983 74.55294499456988) bank130485 +130486 POINT(40.11007879713073 74.67640686372174) bank130486 +130487 POINT(41.439712555778165 74.14286746524381) bank130487 +130488 POINT(41.3037053171994 74.08836365358222) bank130488 +130489 POINT(40.71329093433714 74.94309635566773) bank130489 +130490 POINT(41.44879686987737 73.6470303737054) bank130490 +130491 POINT(40.926862839240904 73.86738976135729) bank130491 +130492 POINT(40.57932787038016 73.94349856908542) bank130492 +130493 POINT(40.596348974586064 74.65773961203752) bank130493 +130494 POINT(40.4498628319592 74.12314696734353) bank130494 +130495 POINT(39.95208048974985 73.63690901291227) bank130495 +130496 POINT(39.83222763455283 74.82689653123474) bank130496 +130497 POINT(40.8731066034302 73.80881359679768) bank130497 +130498 POINT(41.3857547203402 73.60842004722785) bank130498 +130499 POINT(40.227953012637755 73.35558360114338) bank130499 +130500 POINT(41.696695328635144 74.73344374145316) bank130500 +130501 POINT(40.75905347860712 73.68841753726922) bank130501 +130502 POINT(40.19952887449482 74.8757567414102) bank130502 +130503 POINT(41.216092781773064 73.82331700708454) bank130503 +130504 POINT(41.305772351679785 74.30213853271286) bank130504 +130505 POINT(40.86217134234367 73.25954741700174) bank130505 +130506 POINT(41.125817691549216 73.28982088140312) bank130506 +130507 POINT(40.31760471837084 74.64708491819876) bank130507 +130508 POINT(41.349505955147535 73.67690208691695) bank130508 +130509 POINT(39.763639896231105 74.60004155028344) bank130509 +130510 POINT(40.92244570873636 74.47836148580052) bank130510 +130511 POINT(40.225987122690874 74.87169731453338) bank130511 +130512 POINT(40.89245172153545 74.5645933482088) bank130512 +130513 POINT(40.23047943926761 73.60754952409854) bank130513 +130514 POINT(40.457535483013515 73.44513005528248) bank130514 +130515 POINT(39.758579543396365 73.52361170845735) bank130515 +130516 POINT(41.56490006565128 73.91861320739763) bank130516 +130517 POINT(40.8355059417877 73.3973219266006) bank130517 +130518 POINT(41.65470538306732 74.0443223852466) bank130518 +130519 POINT(40.621819524122884 74.25115315092228) bank130519 +130520 POINT(41.39699219016997 74.12608159871083) bank130520 +130521 POINT(40.63234009708505 74.21334071203796) bank130521 +130522 POINT(41.14446244848448 73.14897444189282) bank130522 +130523 POINT(40.37465116241076 74.49843468389595) bank130523 +130524 POINT(40.14835803477552 73.19384398586963) bank130524 +130525 POINT(41.16709644208971 73.44659460467376) bank130525 +130526 POINT(40.2725829378169 74.16263239086291) bank130526 +130527 POINT(41.253279713035894 73.99469279172249) bank130527 +130528 POINT(40.16618244112565 73.68633559150715) bank130528 +130529 POINT(40.2944971546061 74.01740388280024) bank130529 +130530 POINT(41.55819258522676 73.96458811745292) bank130530 +130531 POINT(40.54001629927889 74.44143165863707) bank130531 +130532 POINT(40.12253776772063 74.84876552667103) bank130532 +130533 POINT(40.50256946790891 74.3194131281377) bank130533 +130534 POINT(39.924675238673885 74.14068934349149) bank130534 +130535 POINT(41.121855801851204 73.82761552047845) bank130535 +130536 POINT(40.902134420012835 74.95894349024947) bank130536 +130537 POINT(41.63723624621102 73.28935982064543) bank130537 +130538 POINT(40.529632595492316 74.97918855139505) bank130538 +130539 POINT(40.27290115971496 74.32359537766713) bank130539 +130540 POINT(40.05757343041183 74.94996715095863) bank130540 +130541 POINT(40.81178635170634 73.69697255212253) bank130541 +130542 POINT(40.014635808415626 73.44842855602955) bank130542 +130543 POINT(41.175965356177365 73.71407552850472) bank130543 +130544 POINT(41.574116008514046 73.59324830427697) bank130544 +130545 POINT(39.83639963833396 73.17507114696576) bank130545 +130546 POINT(40.77003912233336 74.36752509747707) bank130546 +130547 POINT(39.79129755437151 74.64743102246535) bank130547 +130548 POINT(41.23879790236581 73.18769191419071) bank130548 +130549 POINT(41.28027300628239 74.60896083641812) bank130549 +130550 POINT(40.10449676037271 73.01970527745196) bank130550 +130551 POINT(40.085358560569645 73.59233011024398) bank130551 +130552 POINT(40.689652598306836 73.61307287646628) bank130552 +130553 POINT(40.935366763484616 74.50554482501633) bank130553 +130554 POINT(40.82046610783123 74.17969531002768) bank130554 +130555 POINT(40.11501652638081 73.03961795650724) bank130555 +130556 POINT(40.83556600797541 73.74666426015744) bank130556 +130557 POINT(40.208590937178364 74.08160431519961) bank130557 +130558 POINT(39.95443007942259 74.04480998965406) bank130558 +130559 POINT(40.04032063441243 74.37290423587106) bank130559 +130560 POINT(40.822383714721134 74.61649949212898) bank130560 +130561 POINT(40.96035605111934 74.06665546887007) bank130561 +130562 POINT(39.73479624727237 73.58132500337827) bank130562 +130563 POINT(40.31957716913666 73.85567547324148) bank130563 +130564 POINT(40.0311930088822 74.29416694287364) bank130564 +130565 POINT(41.28155867616046 74.62838546691806) bank130565 +130566 POINT(40.18114345956177 73.70387453915109) bank130566 +130567 POINT(40.795758358083866 74.45143869098722) bank130567 +130568 POINT(41.15812266190172 73.306351519539) bank130568 +130569 POINT(40.22182643460994 73.38807586090657) bank130569 +130570 POINT(40.858842360735125 74.8785573617394) bank130570 +130571 POINT(39.72496369724544 74.32959945442039) bank130571 +130572 POINT(41.60233680850278 74.53465139942456) bank130572 +130573 POINT(41.4433457346349 73.96753388045434) bank130573 +130574 POINT(40.647006441402624 74.1944803484391) bank130574 +130575 POINT(40.04158464206931 73.83038618197634) bank130575 +130576 POINT(40.85015931869127 73.3309444034184) bank130576 +130577 POINT(39.88707607489378 74.94273673914677) bank130577 +130578 POINT(40.7212024324403 74.86264245377126) bank130578 +130579 POINT(40.1473926388187 73.31690951682597) bank130579 +130580 POINT(40.30747344627683 74.0835903727813) bank130580 +130581 POINT(40.7039904628151 74.79142452485344) bank130581 +130582 POINT(39.928638707511396 73.61326268163985) bank130582 +130583 POINT(39.74520663351209 74.13259300139195) bank130583 +130584 POINT(40.008556999161556 74.22996879273816) bank130584 +130585 POINT(40.38343588494518 73.1552890477626) bank130585 +130586 POINT(41.30934055883629 74.47381207683821) bank130586 +130587 POINT(41.15381177384199 73.9189107905123) bank130587 +130588 POINT(40.07767652599948 74.00582603753014) bank130588 +130589 POINT(41.41740250544237 73.66905058429154) bank130589 +130590 POINT(40.850729659827735 73.13277066888222) bank130590 +130591 POINT(40.67976534876213 73.67238789284512) bank130591 +130592 POINT(41.02407009698968 73.53772222120611) bank130592 +130593 POINT(39.95415967724803 74.09249230568736) bank130593 +130594 POINT(41.08912335496494 73.55518558689259) bank130594 +130595 POINT(39.87889037257887 73.59070846728832) bank130595 +130596 POINT(39.75447172172975 73.95613429821718) bank130596 +130597 POINT(40.802196958135816 73.68923810492338) bank130597 +130598 POINT(40.535074045783034 73.26914391881743) bank130598 +130599 POINT(41.19935454564126 73.99165547788628) bank130599 +130600 POINT(41.63729001238374 74.36547036267928) bank130600 +130601 POINT(40.09826638458133 73.88126722789265) bank130601 +130602 POINT(40.5235874526593 73.59709014063509) bank130602 +130603 POINT(40.13960087637765 73.4213670597339) bank130603 +130604 POINT(40.51030115030746 74.82409449498999) bank130604 +130605 POINT(40.43479913155069 74.39419541079202) bank130605 +130606 POINT(41.331450934959946 74.47369716398632) bank130606 +130607 POINT(40.34056472170302 73.7309271946596) bank130607 +130608 POINT(40.8483495594605 73.99857528444853) bank130608 +130609 POINT(41.61402804084335 73.04487325522369) bank130609 +130610 POINT(40.7177547030194 74.26199963210716) bank130610 +130611 POINT(40.642652921269146 74.6392221970772) bank130611 +130612 POINT(41.10968897528642 74.62091723320898) bank130612 +130613 POINT(41.237187030915564 73.68029623348369) bank130613 +130614 POINT(40.55970366371722 73.08209889025976) bank130614 +130615 POINT(40.16795944708859 74.62941029855877) bank130615 +130616 POINT(41.44084952597399 73.56772729917371) bank130616 +130617 POINT(40.111751332408794 74.52956812197229) bank130617 +130618 POINT(40.155929394043184 74.50802127389724) bank130618 +130619 POINT(39.87638176505602 73.34515785226533) bank130619 +130620 POINT(39.96743156062801 74.35763199651659) bank130620 +130621 POINT(40.92434325671795 74.7939168829687) bank130621 +130622 POINT(39.94581150432358 74.64618164693064) bank130622 +130623 POINT(40.68081969784288 73.13339890212099) bank130623 +130624 POINT(39.93259997685532 73.84151008786432) bank130624 +130625 POINT(40.30995021936603 74.18557180626166) bank130625 +130626 POINT(40.29588197723384 73.0318085328227) bank130626 +130627 POINT(41.54121757888843 73.37461833664875) bank130627 +130628 POINT(40.606153157983265 73.76684570198819) bank130628 +130629 POINT(40.546447079152145 74.44250221904053) bank130629 +130630 POINT(41.04590755432606 74.833472533744) bank130630 +130631 POINT(40.38612468544679 74.45098603309751) bank130631 +130632 POINT(40.09324056325624 74.06746814905968) bank130632 +130633 POINT(39.88163389739602 74.31475210371308) bank130633 +130634 POINT(40.01201107447139 74.81407569777002) bank130634 +130635 POINT(41.50978574292997 74.87899680662187) bank130635 +130636 POINT(41.1805572116724 73.68735168625199) bank130636 +130637 POINT(40.46256795236541 73.38733320089817) bank130637 +130638 POINT(41.02163203971796 74.96235663559115) bank130638 +130639 POINT(40.62020118571377 74.71339478443966) bank130639 +130640 POINT(41.41960134603067 74.27481458406615) bank130640 +130641 POINT(40.434151056418834 73.93924835515536) bank130641 +130642 POINT(41.092395009324555 73.69372731479038) bank130642 +130643 POINT(40.26051432403787 74.29453745396987) bank130643 +130644 POINT(41.27074152601518 74.32174178581435) bank130644 +130645 POINT(40.228629558917845 74.53210163936521) bank130645 +130646 POINT(40.15405203670943 73.11828639780373) bank130646 +130647 POINT(41.07286362938224 74.24798831052608) bank130647 +130648 POINT(40.275105225046445 74.0060740048349) bank130648 +130649 POINT(40.097004706119534 73.7244777888317) bank130649 +130650 POINT(40.80424399188806 74.59486465055879) bank130650 +130651 POINT(40.34895802709538 73.50277053231493) bank130651 +130652 POINT(41.011842199901416 74.52453721328034) bank130652 +130653 POINT(39.72654214537121 74.88666926355964) bank130653 +130654 POINT(40.28217458094276 74.587756033336) bank130654 +130655 POINT(40.65194311206278 74.14173152773964) bank130655 +130656 POINT(40.71073813957511 73.05438429738422) bank130656 +130657 POINT(40.81420047106082 74.99550013055567) bank130657 +130658 POINT(40.45338659551052 74.08994093772208) bank130658 +130659 POINT(40.391369187522336 74.64867791822999) bank130659 +130660 POINT(40.25431777909805 74.73311251960806) bank130660 +130661 POINT(41.24503694284411 74.75973085067926) bank130661 +130662 POINT(41.02308353206197 73.06983280589927) bank130662 +130663 POINT(41.55598523187409 74.45487915286894) bank130663 +130664 POINT(40.157527306842695 74.20879219406038) bank130664 +130665 POINT(40.03628594110667 74.84400938280571) bank130665 +130666 POINT(40.959555098537045 74.60177477904011) bank130666 +130667 POINT(40.403723804247896 74.42494201569743) bank130667 +130668 POINT(41.3866699058346 73.10501997028751) bank130668 +130669 POINT(41.09036184478038 73.32862237956618) bank130669 +130670 POINT(41.03690237824222 73.084800451121) bank130670 +130671 POINT(40.26870990413218 74.44313451938754) bank130671 +130672 POINT(40.47985884491415 74.91561799373227) bank130672 +130673 POINT(41.41928613753474 73.26168081872535) bank130673 +130674 POINT(39.81852316336454 74.15755917881332) bank130674 +130675 POINT(39.81286977882277 73.68331888683547) bank130675 +130676 POINT(40.305664867911915 73.202457289106) bank130676 +130677 POINT(40.55805506253777 73.35232914577026) bank130677 +130678 POINT(41.035818738608405 73.56352706046667) bank130678 +130679 POINT(40.82630222979663 73.35293540149787) bank130679 +130680 POINT(40.92995633962427 74.47899769327202) bank130680 +130681 POINT(40.3521486362128 74.8993767858564) bank130681 +130682 POINT(40.75464716906377 74.28202023818504) bank130682 +130683 POINT(39.99743349296323 74.15391695322901) bank130683 +130684 POINT(40.87689845871363 73.34071553359541) bank130684 +130685 POINT(41.49435318748701 74.07945959160281) bank130685 +130686 POINT(41.08776900770125 74.63462139791996) bank130686 +130687 POINT(39.90018644413536 73.86150062154462) bank130687 +130688 POINT(39.72690582207063 73.90780180358477) bank130688 +130689 POINT(40.24135750164911 73.09676760381983) bank130689 +130690 POINT(41.37982485194882 73.01276333515568) bank130690 +130691 POINT(40.57996239545156 74.76163426861761) bank130691 +130692 POINT(41.178905089657945 73.9040627788688) bank130692 +130693 POINT(40.59882260430759 74.48427588063964) bank130693 +130694 POINT(40.01589944697483 73.9091050205016) bank130694 +130695 POINT(40.51465118131934 73.53555587370244) bank130695 +130696 POINT(40.36498507264533 73.26613446559374) bank130696 +130697 POINT(41.053535302061356 74.64883971919481) bank130697 +130698 POINT(39.87810233958263 73.70078995687406) bank130698 +130699 POINT(39.98785366989675 73.11852731840277) bank130699 +130700 POINT(41.55578079075571 73.1713124050956) bank130700 +130701 POINT(40.10613837492654 73.63321565054743) bank130701 +130702 POINT(40.05091878654778 74.78911881030537) bank130702 +130703 POINT(40.016363170816255 73.22648996662433) bank130703 +130704 POINT(41.090560045789736 73.83241804375409) bank130704 +130705 POINT(41.647470674578926 73.71722115199701) bank130705 +130706 POINT(40.00410951699714 73.75403208864262) bank130706 +130707 POINT(40.18902001480254 73.9177260956231) bank130707 +130708 POINT(41.50730233626452 73.26166109970549) bank130708 +130709 POINT(41.369550806251006 73.05126502884235) bank130709 +130710 POINT(41.56990898328515 73.55810723222359) bank130710 +130711 POINT(40.72984079314553 74.29768958036655) bank130711 +130712 POINT(41.01865976477731 74.74152656863748) bank130712 +130713 POINT(39.88085262481309 74.458844285523) bank130713 +130714 POINT(41.34686395074663 73.04240667532424) bank130714 +130715 POINT(41.47630631595023 74.35320190404448) bank130715 +130716 POINT(41.46606774628699 73.81556897078018) bank130716 +130717 POINT(41.09375228188208 74.92848178412561) bank130717 +130718 POINT(40.377271693693 74.59404889802562) bank130718 +130719 POINT(40.94566638873627 74.07520651096654) bank130719 +130720 POINT(41.5409953212442 73.58923538352711) bank130720 +130721 POINT(39.95255793403015 74.10352364058683) bank130721 +130722 POINT(41.18734081508516 73.3684233976953) bank130722 +130723 POINT(39.88098550917122 74.22122633053011) bank130723 +130724 POINT(41.02168531326014 73.03282525723571) bank130724 +130725 POINT(41.326568153145274 74.79815673914786) bank130725 +130726 POINT(40.33973049011811 74.62749476734739) bank130726 +130727 POINT(41.54899290809364 74.25852863444467) bank130727 +130728 POINT(40.12334441760348 74.22557744564168) bank130728 +130729 POINT(41.183127391510276 73.81720465944404) bank130729 +130730 POINT(40.281577000390655 74.42269273901883) bank130730 +130731 POINT(40.424315320616394 73.00997006136654) bank130731 +130732 POINT(41.20776002136223 74.67393641246807) bank130732 +130733 POINT(41.56952645350047 73.68753447164625) bank130733 +130734 POINT(39.74714242581156 73.17431871400086) bank130734 +130735 POINT(39.7805681694882 74.83273267926012) bank130735 +130736 POINT(40.89627496568577 73.41227854067567) bank130736 +130737 POINT(41.6007367634913 73.18978752144541) bank130737 +130738 POINT(39.89674803606683 74.28692516831215) bank130738 +130739 POINT(41.3803728591818 73.3538812065397) bank130739 +130740 POINT(39.87083527788357 74.61739475289575) bank130740 +130741 POINT(40.24257717605259 73.36935483973741) bank130741 +130742 POINT(41.19214993401049 73.73796272992543) bank130742 +130743 POINT(41.57607955372283 74.24936571787893) bank130743 +130744 POINT(41.31007653469267 74.66700035342879) bank130744 +130745 POINT(40.74769444874259 74.92520531176712) bank130745 +130746 POINT(39.74564441192365 73.1451292841316) bank130746 +130747 POINT(41.6304523652313 73.66403449136645) bank130747 +130748 POINT(40.52225080257881 73.33996270557815) bank130748 +130749 POINT(40.56991719343126 73.60579544081808) bank130749 +130750 POINT(39.87538755890468 73.03053535558952) bank130750 +130751 POINT(39.937309181498385 73.68362691211654) bank130751 +130752 POINT(40.54632023055126 74.09081824737068) bank130752 +130753 POINT(40.30262438050781 73.52891755003101) bank130753 +130754 POINT(40.56492828813691 73.89642650409847) bank130754 +130755 POINT(40.62304102419028 73.4992351662058) bank130755 +130756 POINT(40.72668219216578 74.96718925870987) bank130756 +130757 POINT(40.7075090885035 74.2622388734742) bank130757 +130758 POINT(40.76675806304313 73.68608830393316) bank130758 +130759 POINT(40.9903121599997 74.2580731122147) bank130759 +130760 POINT(41.007675864474194 73.82312571167812) bank130760 +130761 POINT(40.56815298215759 74.8545071475259) bank130761 +130762 POINT(40.267439666932944 73.97576822802665) bank130762 +130763 POINT(41.036136007277726 74.72720962034019) bank130763 +130764 POINT(39.912656752647855 73.51271637241928) bank130764 +130765 POINT(40.435546770258235 74.51276927436187) bank130765 +130766 POINT(41.65687182416548 73.98293225388845) bank130766 +130767 POINT(41.38394715912884 74.9600539871932) bank130767 +130768 POINT(41.01494261950393 73.34380918947708) bank130768 +130769 POINT(40.81621180667511 73.02822893949067) bank130769 +130770 POINT(41.06448986855988 73.6429038451579) bank130770 +130771 POINT(40.48700978936099 73.91070670686557) bank130771 +130772 POINT(40.24169045093863 74.59491392475232) bank130772 +130773 POINT(40.02044570395892 73.05535369161653) bank130773 +130774 POINT(41.52728714512644 74.89453302273363) bank130774 +130775 POINT(41.01347738734785 74.3172448095015) bank130775 +130776 POINT(40.63029958776729 73.8357020912889) bank130776 +130777 POINT(41.218204932611194 73.83177132426778) bank130777 +130778 POINT(40.60119509415151 73.09550677948002) bank130778 +130779 POINT(40.99435112827792 74.14012258582848) bank130779 +130780 POINT(40.517548317001115 74.46178892228814) bank130780 +130781 POINT(41.21853465582483 73.16485180067241) bank130781 +130782 POINT(39.98719381068602 73.96362978941987) bank130782 +130783 POINT(41.55310334753059 73.17555959628552) bank130783 +130784 POINT(41.56908189344213 73.86085310696507) bank130784 +130785 POINT(41.11817489124789 73.23716033612675) bank130785 +130786 POINT(41.19256049667233 74.5078753438321) bank130786 +130787 POINT(39.99952911143396 74.88329154235645) bank130787 +130788 POINT(40.993471271353954 73.25046248810847) bank130788 +130789 POINT(40.974220131371055 73.07452435198854) bank130789 +130790 POINT(41.10825519946267 73.5341239620563) bank130790 +130791 POINT(41.43113918980007 73.3292010709954) bank130791 +130792 POINT(41.56808326002948 73.58444193072648) bank130792 +130793 POINT(41.438590166436114 73.36579424609528) bank130793 +130794 POINT(41.155660500715 73.55882482728191) bank130794 +130795 POINT(40.607918877516326 74.92110893441851) bank130795 +130796 POINT(41.43769452315226 74.54025564844629) bank130796 +130797 POINT(40.796423282161825 74.59797651078708) bank130797 +130798 POINT(40.73175619960619 74.56476032955504) bank130798 +130799 POINT(39.87288044654278 74.29736200182046) bank130799 +130800 POINT(40.709212517140834 73.39172259178147) bank130800 +130801 POINT(41.70398152899659 73.4774176746038) bank130801 +130802 POINT(40.18226506782877 73.13416158699727) bank130802 +130803 POINT(41.052788354027506 74.95344227078675) bank130803 +130804 POINT(40.28307363632158 74.93247747477284) bank130804 +130805 POINT(40.544860044387825 73.37451345558131) bank130805 +130806 POINT(40.21196280243273 74.38928179506233) bank130806 +130807 POINT(40.22067915257642 73.14776799936647) bank130807 +130808 POINT(40.008186116249405 74.52574749651247) bank130808 +130809 POINT(40.02103877137524 74.08735121051058) bank130809 +130810 POINT(40.90607188091297 74.47663709148958) bank130810 +130811 POINT(40.491042611038026 74.46418373665905) bank130811 +130812 POINT(40.393166988390554 74.96834802402225) bank130812 +130813 POINT(41.19434908727009 73.15870729942499) bank130813 +130814 POINT(41.26137806939761 74.98845915007126) bank130814 +130815 POINT(40.08957589402622 74.42575258400886) bank130815 +130816 POINT(40.872407717079774 74.2444917193409) bank130816 +130817 POINT(39.76644336837934 74.76479129725152) bank130817 +130818 POINT(40.25499062362676 73.50487084731819) bank130818 +130819 POINT(40.524897284098316 74.70199103430954) bank130819 +130820 POINT(39.88382120833445 74.22667541768368) bank130820 +130821 POINT(40.92574088425411 74.31611199680216) bank130821 +130822 POINT(41.46051751579195 73.83423251681734) bank130822 +130823 POINT(41.33419263507054 74.5364283716505) bank130823 +130824 POINT(40.26308034224667 73.88790794821858) bank130824 +130825 POINT(41.68584949202713 74.94609121872567) bank130825 +130826 POINT(39.82984456258955 74.68571932503275) bank130826 +130827 POINT(40.54959083499502 73.77626578967265) bank130827 +130828 POINT(41.48111793402974 73.30863643692756) bank130828 +130829 POINT(39.74951508036123 74.0339651857392) bank130829 +130830 POINT(39.84707734700687 73.56123218780243) bank130830 +130831 POINT(41.66436033559538 73.76901116574788) bank130831 +130832 POINT(40.42320912325738 74.81999835855265) bank130832 +130833 POINT(40.096031548356024 73.90470598126458) bank130833 +130834 POINT(40.391353199720605 73.21845007136426) bank130834 +130835 POINT(39.77235710911463 73.40709675615776) bank130835 +130836 POINT(40.04721909284471 73.96524587574866) bank130836 +130837 POINT(40.82212342425207 74.33010832994093) bank130837 +130838 POINT(41.508206002558296 73.63947808707782) bank130838 +130839 POINT(40.36207889104834 74.09962690501268) bank130839 +130840 POINT(40.060839663396905 73.36959909161585) bank130840 +130841 POINT(40.64699159258956 73.52835604677836) bank130841 +130842 POINT(41.216272859624446 73.09534296894289) bank130842 +130843 POINT(40.37998367417272 73.20394456221976) bank130843 +130844 POINT(40.8278371214559 73.11677821583935) bank130844 +130845 POINT(39.7585654434908 74.02374536482756) bank130845 +130846 POINT(40.265256810964814 74.52190944100937) bank130846 +130847 POINT(41.04287129207601 73.53772521103558) bank130847 +130848 POINT(40.25747660881817 73.28255119498613) bank130848 +130849 POINT(40.91053741585572 73.93417949481041) bank130849 +130850 POINT(39.98712791645069 74.27010471478106) bank130850 +130851 POINT(41.31770422102117 74.90492730334185) bank130851 +130852 POINT(41.05932368699865 74.29387164444402) bank130852 +130853 POINT(41.43261956309791 73.04121749835157) bank130853 +130854 POINT(40.386837121029785 74.24160616020401) bank130854 +130855 POINT(39.95116927496585 74.85986394127514) bank130855 +130856 POINT(39.87537586295448 74.98799923977803) bank130856 +130857 POINT(40.903361469169496 74.45799406936916) bank130857 +130858 POINT(40.370065834474104 74.7694412018114) bank130858 +130859 POINT(41.70352501027463 73.27738842881384) bank130859 +130860 POINT(39.879892946191 73.36663355114202) bank130860 +130861 POINT(40.90071170445817 73.83855105254759) bank130861 +130862 POINT(40.89758029629308 74.94937665472429) bank130862 +130863 POINT(41.61394963332452 73.15528696794806) bank130863 +130864 POINT(41.52449551716803 73.09839717794388) bank130864 +130865 POINT(39.93348606836888 74.19101776714433) bank130865 +130866 POINT(40.024645430989466 74.5328865926605) bank130866 +130867 POINT(40.61745923868719 73.8630616421417) bank130867 +130868 POINT(41.52127467634242 73.84802035823908) bank130868 +130869 POINT(39.99683764567312 74.36224874107123) bank130869 +130870 POINT(41.152277115473524 74.1579930151594) bank130870 +130871 POINT(41.25552664665909 74.75590827805091) bank130871 +130872 POINT(40.50420645715716 73.61701066328982) bank130872 +130873 POINT(39.76499968445009 74.7568184393212) bank130873 +130874 POINT(40.81985363002806 73.43375154703708) bank130874 +130875 POINT(41.500694486934194 74.1199627754698) bank130875 +130876 POINT(39.799491809174405 74.55071747377946) bank130876 +130877 POINT(40.96885970447126 74.40399252301326) bank130877 +130878 POINT(40.33961395103471 74.41958248836534) bank130878 +130879 POINT(41.26890761965056 73.03192881125113) bank130879 +130880 POINT(41.41006124987971 74.04813029830132) bank130880 +130881 POINT(39.80879078283912 74.85632577841831) bank130881 +130882 POINT(41.60005400200034 74.15976940134608) bank130882 +130883 POINT(41.23457426976687 73.6006150666412) bank130883 +130884 POINT(39.9808550157414 74.73267964380167) bank130884 +130885 POINT(39.92782806961616 73.65452898081527) bank130885 +130886 POINT(40.12163585647614 73.69558612672536) bank130886 +130887 POINT(40.472595495662574 74.25763895851003) bank130887 +130888 POINT(40.149750248219235 74.3024522867077) bank130888 +130889 POINT(41.2036504603987 73.25870825200985) bank130889 +130890 POINT(40.64437827069545 74.30093499608674) bank130890 +130891 POINT(41.630336946843876 74.12047240967745) bank130891 +130892 POINT(41.67735518139814 74.72586601294691) bank130892 +130893 POINT(41.66219636499551 73.40940714511865) bank130893 +130894 POINT(41.02593118271039 74.83345674809357) bank130894 +130895 POINT(40.51993697774937 74.2864238252951) bank130895 +130896 POINT(40.981498306018565 74.93350090862113) bank130896 +130897 POINT(39.928124818983044 73.93276909116895) bank130897 +130898 POINT(40.81033108137754 74.47420161478469) bank130898 +130899 POINT(40.909335240832455 73.02173878744131) bank130899 +130900 POINT(40.09510434856102 74.06740081690594) bank130900 +130901 POINT(41.59296331874304 73.71871679227087) bank130901 +130902 POINT(40.38827795060666 74.63983499895329) bank130902 +130903 POINT(40.780306317114196 73.25298730133245) bank130903 +130904 POINT(40.64433439560504 74.63591783955414) bank130904 +130905 POINT(41.455721468869456 73.36966388600608) bank130905 +130906 POINT(40.36760960719897 74.92460950428271) bank130906 +130907 POINT(41.47799519404231 73.54655157614715) bank130907 +130908 POINT(41.123452217038164 73.61360113454262) bank130908 +130909 POINT(41.222611778984735 73.6972590710867) bank130909 +130910 POINT(40.735912198329835 74.29657048633071) bank130910 +130911 POINT(40.564924480231845 74.98773767506107) bank130911 +130912 POINT(39.885399993095675 73.4590227952227) bank130912 +130913 POINT(41.375494011339306 74.44256132620475) bank130913 +130914 POINT(40.16599880366063 73.91504673911119) bank130914 +130915 POINT(40.47223089811137 74.23005140974013) bank130915 +130916 POINT(40.41018137338056 74.70373036670587) bank130916 +130917 POINT(40.030949658510515 74.83096415582868) bank130917 +130918 POINT(40.753103309680355 74.29079693890131) bank130918 +130919 POINT(39.84026377378979 74.24347478309181) bank130919 +130920 POINT(40.575172945565626 74.91023019833318) bank130920 +130921 POINT(40.30802380672331 73.05027259618248) bank130921 +130922 POINT(40.87965120685831 74.75608096560697) bank130922 +130923 POINT(40.06451876249387 74.76371791336514) bank130923 +130924 POINT(40.071168112553224 73.68424236829405) bank130924 +130925 POINT(40.39610101364204 73.20278031917005) bank130925 +130926 POINT(41.488689424909225 74.70676798354997) bank130926 +130927 POINT(41.41284931052011 74.82556198250205) bank130927 +130928 POINT(40.1574028433329 74.95117916589358) bank130928 +130929 POINT(41.076394021695805 74.22084518840286) bank130929 +130930 POINT(41.449141437066324 74.72970758427093) bank130930 +130931 POINT(41.677100144349275 74.89441387808711) bank130931 +130932 POINT(39.78738339123246 74.67580885484061) bank130932 +130933 POINT(40.81944875294481 74.05486259420418) bank130933 +130934 POINT(40.20205649674619 74.68473319242996) bank130934 +130935 POINT(40.08322650725458 73.66753907673763) bank130935 +130936 POINT(40.66947648430281 73.63249390978217) bank130936 +130937 POINT(41.1202883091969 74.19167408137783) bank130937 +130938 POINT(39.94191654198711 74.50272746645213) bank130938 +130939 POINT(41.64727219754287 73.7532245324445) bank130939 +130940 POINT(41.016063110907545 74.67844648488271) bank130940 +130941 POINT(39.8179429302576 73.56388439574597) bank130941 +130942 POINT(39.981757017681666 73.66287546479666) bank130942 +130943 POINT(41.690303677600035 73.81687661310609) bank130943 +130944 POINT(40.752455539859014 73.17256019389353) bank130944 +130945 POINT(41.20442729583278 73.6093256911805) bank130945 +130946 POINT(40.24692332340568 73.65803229934167) bank130946 +130947 POINT(40.17704927211872 73.11326965836055) bank130947 +130948 POINT(40.00033577094044 73.80746182655678) bank130948 +130949 POINT(40.929503777305555 74.40263644910011) bank130949 +130950 POINT(41.228601024442476 74.56543238749516) bank130950 +130951 POINT(41.32402385615713 74.0406020664485) bank130951 +130952 POINT(41.277833092893175 74.89031260109846) bank130952 +130953 POINT(40.26907909065325 73.22329099148602) bank130953 +130954 POINT(40.333115079770764 74.5089104543479) bank130954 +130955 POINT(41.08047709608368 74.27820617643965) bank130955 +130956 POINT(39.808413578147935 74.78867955833734) bank130956 +130957 POINT(40.09858022931874 73.2124536079549) bank130957 +130958 POINT(41.53136016697906 74.60630747964424) bank130958 +130959 POINT(39.81008246800862 73.97123646577383) bank130959 +130960 POINT(41.346605572729786 74.01529731096284) bank130960 +130961 POINT(41.31531471015884 74.00121625736986) bank130961 +130962 POINT(40.381947693037944 74.38543365296937) bank130962 +130963 POINT(41.26840026029539 73.20030979432039) bank130963 +130964 POINT(41.18339455533251 73.77117403722615) bank130964 +130965 POINT(40.483445321879685 74.7383498793307) bank130965 +130966 POINT(40.62065170735317 74.54713433050611) bank130966 +130967 POINT(40.35739004915603 74.247675032872) bank130967 +130968 POINT(40.5389927204488 74.1157376027553) bank130968 +130969 POINT(41.51491937049579 73.01294992636574) bank130969 +130970 POINT(41.1360652577753 73.0234788160966) bank130970 +130971 POINT(41.41001753041828 73.29410914195165) bank130971 +130972 POINT(41.2398275583944 73.37955022193384) bank130972 +130973 POINT(40.51286424336995 74.497230467545) bank130973 +130974 POINT(40.41112811123979 73.89771013914115) bank130974 +130975 POINT(40.83210366570695 73.25049831829871) bank130975 +130976 POINT(39.947585503888995 73.33212860377402) bank130976 +130977 POINT(40.76003472353477 73.5561452426136) bank130977 +130978 POINT(41.04632147196003 73.1908427014103) bank130978 +130979 POINT(41.607959591521094 74.95028022071465) bank130979 +130980 POINT(41.1390740744048 74.51045928051296) bank130980 +130981 POINT(41.28250148202915 73.30117336587224) bank130981 +130982 POINT(40.157242862727834 73.42982196924075) bank130982 +130983 POINT(40.98017593477502 74.25613533126496) bank130983 +130984 POINT(41.21673201146933 74.2277040854415) bank130984 +130985 POINT(40.21100250079058 74.97530430182461) bank130985 +130986 POINT(41.687993296313614 74.35158291827781) bank130986 +130987 POINT(41.64145132555064 73.93378494712407) bank130987 +130988 POINT(41.62010585280849 74.45011939674603) bank130988 +130989 POINT(40.72425082528392 74.72811336049952) bank130989 +130990 POINT(41.468416908637614 74.82331635472234) bank130990 +130991 POINT(40.84959895323997 73.98390684403209) bank130991 +130992 POINT(40.92630196569385 73.72745790917249) bank130992 +130993 POINT(40.46514667610479 73.0730087298453) bank130993 +130994 POINT(41.161283384384284 74.24811634190995) bank130994 +130995 POINT(39.73254892701433 73.06251508204166) bank130995 +130996 POINT(41.541133447051884 73.91692399173148) bank130996 +130997 POINT(40.46913353659509 74.80049962319421) bank130997 +130998 POINT(41.35809249936589 73.83321770818586) bank130998 +130999 POINT(40.72917594924132 73.18544054167793) bank130999 +131000 POINT(40.16487410211767 73.22443781348437) bank131000 +131001 POINT(41.47230503539447 74.96381316644658) bank131001 +131002 POINT(39.93773365964658 73.93429283388888) bank131002 +131003 POINT(40.90966012180084 74.18714554909012) bank131003 +131004 POINT(41.2903700383709 74.71594740448215) bank131004 +131005 POINT(40.2590939259201 74.95152270981382) bank131005 +131006 POINT(40.892491629141425 73.56797994425239) bank131006 +131007 POINT(40.00901248624501 73.55810815135317) bank131007 +131008 POINT(40.03301135485458 73.0473524353278) bank131008 +131009 POINT(40.233536631941824 73.09767279716502) bank131009 +131010 POINT(41.498669687235115 74.06304769277908) bank131010 +131011 POINT(40.51768928585629 73.48566401497992) bank131011 +131012 POINT(39.81745535351018 74.33176487369113) bank131012 +131013 POINT(40.550464665040586 73.81068191927874) bank131013 +131014 POINT(40.71714827336682 73.47102813231982) bank131014 +131015 POINT(41.11259327495238 73.48232635839462) bank131015 +131016 POINT(41.02031130026754 73.49608701395422) bank131016 +131017 POINT(40.35097162672538 74.30990957334626) bank131017 +131018 POINT(41.41775846157457 74.99671292016073) bank131018 +131019 POINT(41.40202949897617 73.51208344854935) bank131019 +131020 POINT(41.10344215018098 74.08236740184435) bank131020 +131021 POINT(40.20813338318371 74.07010844221269) bank131021 +131022 POINT(40.49560884210756 74.24801325843528) bank131022 +131023 POINT(40.149106164980736 73.64186897607001) bank131023 +131024 POINT(41.458987025468424 73.76023802714127) bank131024 +131025 POINT(41.64490248066365 74.96998692830647) bank131025 +131026 POINT(40.73730474294474 73.49477622897508) bank131026 +131027 POINT(40.30311926424257 73.6918818109219) bank131027 +131028 POINT(39.97265027347642 73.99788335202983) bank131028 +131029 POINT(40.10065535383032 73.16309004588967) bank131029 +131030 POINT(39.781262357247776 74.76882253332263) bank131030 +131031 POINT(41.67264725238317 73.92101953193315) bank131031 +131032 POINT(41.25868867142501 73.4413255125325) bank131032 +131033 POINT(41.37063488675641 74.79032207737218) bank131033 +131034 POINT(41.70203453461888 74.32301551584975) bank131034 +131035 POINT(40.249049827837354 74.69627258253027) bank131035 +131036 POINT(41.062948847164776 74.78419912904884) bank131036 +131037 POINT(41.177573474616004 73.26715446113158) bank131037 +131038 POINT(40.09749331218808 73.602555386618) bank131038 +131039 POINT(40.14396644832943 73.77104078566161) bank131039 +131040 POINT(39.907720161134684 73.13286853215425) bank131040 +131041 POINT(40.00206705377737 73.33151889252149) bank131041 +131042 POINT(41.598387917832355 74.66195597932213) bank131042 +131043 POINT(41.710245234255595 74.13117056858258) bank131043 +131044 POINT(40.07511662856857 74.79847865874794) bank131044 +131045 POINT(40.06160230581882 74.00769426614674) bank131045 +131046 POINT(41.10732693770034 74.78218701790945) bank131046 +131047 POINT(40.18679285530108 74.88724226811593) bank131047 +131048 POINT(41.32903456286764 74.95907715889665) bank131048 +131049 POINT(41.03493574622325 74.7620251449037) bank131049 +131050 POINT(41.202829962776946 74.61586370231375) bank131050 +131051 POINT(41.282546023038634 73.64860073693403) bank131051 +131052 POINT(41.33769508632217 74.57440168354879) bank131052 +131053 POINT(39.8847306784279 74.00172417139231) bank131053 +131054 POINT(40.55857106338438 73.4304684982281) bank131054 +131055 POINT(40.075042752547546 74.14707399356054) bank131055 +131056 POINT(40.75737766161408 74.72073099866635) bank131056 +131057 POINT(41.45617161288267 74.96918499917363) bank131057 +131058 POINT(40.28346933833626 73.8659892655048) bank131058 +131059 POINT(40.17074208557389 74.23910469797352) bank131059 +131060 POINT(40.902008842395794 73.97917882190542) bank131060 +131061 POINT(40.45775121826434 74.57114494766117) bank131061 +131062 POINT(40.599273776348255 74.39780156258091) bank131062 +131063 POINT(40.46705240472268 73.1127835284423) bank131063 +131064 POINT(39.740070119236094 73.19695305722321) bank131064 +131065 POINT(40.55352150477533 73.41313520346448) bank131065 +131066 POINT(40.2076526790144 74.51017771726143) bank131066 +131067 POINT(41.1139616339294 73.47697870292629) bank131067 +131068 POINT(40.628041835907375 73.04794650265173) bank131068 +131069 POINT(40.26204955801633 73.52877409096152) bank131069 +131070 POINT(40.040494743351545 73.81355771730473) bank131070 +131071 POINT(41.49144925605494 74.19608003475491) bank131071 +131072 POINT(41.2262891403991 74.76654203287566) bank131072 +131073 POINT(40.87027446644199 73.58770919762998) bank131073 +131074 POINT(40.283854905400304 74.74573510796212) bank131074 +131075 POINT(40.57922556267911 74.4027036803856) bank131075 +131076 POINT(40.38238612308314 74.83650037187752) bank131076 +131077 POINT(41.07884130279311 73.06220771210299) bank131077 +131078 POINT(39.78862733238822 74.49520937226609) bank131078 +131079 POINT(40.95145245317961 74.03145833682746) bank131079 +131080 POINT(41.28610971933343 73.27869796206399) bank131080 +131081 POINT(41.29193744482427 74.82329921831062) bank131081 +131082 POINT(41.40426651374101 74.84623588493638) bank131082 +131083 POINT(39.88447660653075 73.46681724867325) bank131083 +131084 POINT(40.45530989468244 74.01260258827311) bank131084 +131085 POINT(41.17230147321923 74.79422714240643) bank131085 +131086 POINT(40.09641965373648 73.26534269301914) bank131086 +131087 POINT(41.03771340680273 74.36707200695767) bank131087 +131088 POINT(39.838607645694445 74.34881935169444) bank131088 +131089 POINT(40.177501774569464 73.26408275497377) bank131089 +131090 POINT(40.27919616169257 74.08927265352357) bank131090 +131091 POINT(41.62882708715908 73.25415663171076) bank131091 +131092 POINT(40.28733049104332 74.05459650000408) bank131092 +131093 POINT(40.35841715517331 73.66227488670296) bank131093 +131094 POINT(40.242197031453706 73.99131658646523) bank131094 +131095 POINT(41.622415998350235 74.76283720105283) bank131095 +131096 POINT(40.26879017056315 73.88244865095605) bank131096 +131097 POINT(40.42945754528051 74.66303329996516) bank131097 +131098 POINT(40.03746883797495 74.25083859758618) bank131098 +131099 POINT(39.97156043568512 74.87250036869295) bank131099 +131100 POINT(40.28870503186546 73.25091797362819) bank131100 +131101 POINT(40.340789997573786 74.63853107393837) bank131101 +131102 POINT(40.55240445430897 73.36061398954979) bank131102 +131103 POINT(40.01393855874616 74.35732443675057) bank131103 +131104 POINT(39.95378039412581 73.61812709816296) bank131104 +131105 POINT(40.30924701286463 73.49732776355064) bank131105 +131106 POINT(40.83917849841148 73.42880571243512) bank131106 +131107 POINT(41.678667131570386 73.54464542500142) bank131107 +131108 POINT(41.43626962771177 74.6858494572605) bank131108 +131109 POINT(40.265063351198826 74.64037831773247) bank131109 +131110 POINT(40.16419826062511 74.84756262750233) bank131110 +131111 POINT(41.02590744595949 73.21753408767151) bank131111 +131112 POINT(41.10774299322008 73.39275480060091) bank131112 +131113 POINT(40.44500725498591 74.51071382960214) bank131113 +131114 POINT(40.810031000677405 74.43798758570888) bank131114 +131115 POINT(40.49573201211196 73.73984869281804) bank131115 +131116 POINT(40.55433700958618 74.42954635731806) bank131116 +131117 POINT(40.52631181104362 74.0638383602699) bank131117 +131118 POINT(40.70144526218967 73.25931042503915) bank131118 +131119 POINT(40.123622865113646 73.6559212257082) bank131119 +131120 POINT(40.347785539291166 74.74729409106459) bank131120 +131121 POINT(40.459644230350804 74.95801123375561) bank131121 +131122 POINT(39.85504999068979 74.24556370391947) bank131122 +131123 POINT(40.620875387869226 73.15205169280182) bank131123 +131124 POINT(40.05013948827811 74.01064438705531) bank131124 +131125 POINT(40.86871384620561 74.40731592852883) bank131125 +131126 POINT(41.31582728452602 74.34922110843236) bank131126 +131127 POINT(40.57485529008267 74.62376500204986) bank131127 +131128 POINT(41.43367408294151 73.89592090882684) bank131128 +131129 POINT(41.3611505233484 73.93926063587269) bank131129 +131130 POINT(39.758348960699834 73.73668177627125) bank131130 +131131 POINT(40.369268482595054 74.86901455273988) bank131131 +131132 POINT(41.529417050070386 73.10907870865455) bank131132 +131133 POINT(41.302897654493364 73.46077957409065) bank131133 +131134 POINT(40.390907631119255 73.97021597950737) bank131134 +131135 POINT(40.59481469183182 73.42389629629041) bank131135 +131136 POINT(41.1690162762496 73.37403524392099) bank131136 +131137 POINT(40.412286835990656 73.57073385387588) bank131137 +131138 POINT(40.91762276898562 74.35803593194005) bank131138 +131139 POINT(40.750295880639946 74.01938585590314) bank131139 +131140 POINT(41.55892254487526 74.32144212968308) bank131140 +131141 POINT(41.67835341564627 74.56011961664304) bank131141 +131142 POINT(40.050320877531384 73.1300932813477) bank131142 +131143 POINT(40.22298545833387 74.94463678532458) bank131143 +131144 POINT(40.14121645330321 73.07058311540696) bank131144 +131145 POINT(40.573044002902535 73.89147195097854) bank131145 +131146 POINT(41.29105945921897 73.24841267268256) bank131146 +131147 POINT(40.41834625983428 74.82385453032474) bank131147 +131148 POINT(39.83117472019715 73.78545936669694) bank131148 +131149 POINT(40.7192399508049 73.48970859499249) bank131149 +131150 POINT(40.07337430304782 74.6645706748533) bank131150 +131151 POINT(40.13313221585424 73.39278850393697) bank131151 +131152 POINT(40.59431317081813 73.01305254707258) bank131152 +131153 POINT(39.82102167348449 73.07314767893065) bank131153 +131154 POINT(40.806594409955984 74.10912831952467) bank131154 +131155 POINT(41.57539214366068 74.48256384743887) bank131155 +131156 POINT(40.893041206927364 74.2180733057918) bank131156 +131157 POINT(41.4028647339403 73.98071034628374) bank131157 +131158 POINT(40.69489663799784 73.1761197303809) bank131158 +131159 POINT(40.32218552583049 74.20079972001743) bank131159 +131160 POINT(41.13592779991042 74.84326290115082) bank131160 +131161 POINT(41.71186151400936 73.05789230559722) bank131161 +131162 POINT(40.99951688192559 74.10655821200483) bank131162 +131163 POINT(40.66614675704418 74.59688540103666) bank131163 +131164 POINT(40.19243934501465 74.11779990146087) bank131164 +131165 POINT(40.08287846976299 73.36789618829583) bank131165 +131166 POINT(40.88070439110298 73.93120104147451) bank131166 +131167 POINT(40.726435923483514 74.6335033926711) bank131167 +131168 POINT(39.908643007620235 73.37981498810828) bank131168 +131169 POINT(40.48891541110171 74.36875217751141) bank131169 +131170 POINT(41.26609277836799 74.9997921499696) bank131170 +131171 POINT(40.64757618802801 74.76952815235224) bank131171 +131172 POINT(41.36177328968542 73.05023837346073) bank131172 +131173 POINT(41.26435502262889 73.91362878001947) bank131173 +131174 POINT(41.43300069262857 73.0153347792516) bank131174 +131175 POINT(40.130278434211185 74.04520654636521) bank131175 +131176 POINT(40.87519858270509 73.28172819107385) bank131176 +131177 POINT(41.5371575105418 73.37606371242647) bank131177 +131178 POINT(41.50104619574131 74.86771835172094) bank131178 +131179 POINT(41.06902937876509 73.47092128969382) bank131179 +131180 POINT(41.16316791377083 73.80202726848228) bank131180 +131181 POINT(40.343986315173886 74.65048345382888) bank131181 +131182 POINT(41.077103751556905 73.28265947137844) bank131182 +131183 POINT(40.55163322766318 73.91801374766011) bank131183 +131184 POINT(40.86001902855411 74.94694218418427) bank131184 +131185 POINT(41.07910203545304 73.31727544998166) bank131185 +131186 POINT(41.40083703283958 73.9275491500978) bank131186 +131187 POINT(40.365408105728456 74.43578071481402) bank131187 +131188 POINT(39.93385676863451 74.33463120122569) bank131188 +131189 POINT(40.98313887102033 74.59097172687444) bank131189 +131190 POINT(41.43325940703333 74.74220621774376) bank131190 +131191 POINT(39.989049481128895 73.70601260950784) bank131191 +131192 POINT(41.454899091106164 73.36776500646239) bank131192 +131193 POINT(39.796976516961436 74.82029362177013) bank131193 +131194 POINT(41.503895602025686 74.19137741662433) bank131194 +131195 POINT(40.49620647748414 73.78730023974518) bank131195 +131196 POINT(40.54389447422479 73.6452279469069) bank131196 +131197 POINT(41.1509639064859 73.5017173248655) bank131197 +131198 POINT(41.40751878098734 74.11184480814964) bank131198 +131199 POINT(40.35499508943991 74.83094013136724) bank131199 +131200 POINT(40.23757996780562 73.76850338138394) bank131200 +131201 POINT(40.66016768803694 73.34846868777241) bank131201 +131202 POINT(40.59801300392451 74.13170321460517) bank131202 +131203 POINT(40.13211946205784 73.42491042156169) bank131203 +131204 POINT(41.672199192605724 74.60395703153011) bank131204 +131205 POINT(41.47279826537832 74.52177421458175) bank131205 +131206 POINT(39.92275553722017 74.0410632695825) bank131206 +131207 POINT(41.20604296757115 73.9003854090967) bank131207 +131208 POINT(39.846451804677336 73.34482255114766) bank131208 +131209 POINT(40.27389608006368 73.87376150789432) bank131209 +131210 POINT(40.892458395562734 74.1470531785732) bank131210 +131211 POINT(39.9592133628937 73.65155005831798) bank131211 +131212 POINT(40.73937700692723 73.97830839361272) bank131212 +131213 POINT(40.38421882742331 73.60913612128049) bank131213 +131214 POINT(41.54267821598746 74.74007360483692) bank131214 +131215 POINT(41.20427367466174 73.86676391775426) bank131215 +131216 POINT(40.82751171606164 73.3130261519841) bank131216 +131217 POINT(40.90155322928432 73.95508982356509) bank131217 +131218 POINT(39.80564531996543 73.31968256753787) bank131218 +131219 POINT(41.65041561019466 74.71632350902588) bank131219 +131220 POINT(40.75867191831062 74.23005002804089) bank131220 +131221 POINT(41.308941990272416 73.75594960306736) bank131221 +131222 POINT(40.57805683805932 73.83765328442058) bank131222 +131223 POINT(40.81319232190577 74.39029051260667) bank131223 +131224 POINT(41.435529145978556 74.40277388112742) bank131224 +131225 POINT(41.33429805179406 74.99183648506418) bank131225 +131226 POINT(40.54687103519003 73.69850458323283) bank131226 +131227 POINT(41.26520729387648 73.66727365401185) bank131227 +131228 POINT(39.928222898103286 74.12805928545092) bank131228 +131229 POINT(41.682457891036556 73.69611018507563) bank131229 +131230 POINT(40.014624935818055 74.88051019898646) bank131230 +131231 POINT(40.862618471255026 74.72856646494235) bank131231 +131232 POINT(40.506458741979635 73.28821976105814) bank131232 +131233 POINT(41.533411071104304 74.55033721775938) bank131233 +131234 POINT(39.739562457333214 74.91016607052329) bank131234 +131235 POINT(39.76243918547129 73.09453559363273) bank131235 +131236 POINT(41.60818269918597 73.57749660338692) bank131236 +131237 POINT(39.71764809513872 73.83785380907158) bank131237 +131238 POINT(41.114030943207645 74.6947016374762) bank131238 +131239 POINT(41.32467168116977 74.8030537459539) bank131239 +131240 POINT(41.65286528061528 73.87649149106632) bank131240 +131241 POINT(40.62514826400026 74.72492110661588) bank131241 +131242 POINT(40.13471842894519 74.08475447332859) bank131242 +131243 POINT(41.12919815528802 74.17654750626741) bank131243 +131244 POINT(39.810608424600254 74.36622069289886) bank131244 +131245 POINT(40.11942041268168 74.61802437752559) bank131245 +131246 POINT(41.506835566113175 73.20028917330825) bank131246 +131247 POINT(40.31681742298504 73.37320955186055) bank131247 +131248 POINT(41.170142510592676 74.01574384354271) bank131248 +131249 POINT(41.71021727756088 74.00350018923672) bank131249 +131250 POINT(40.23599040667637 73.15576464501204) bank131250 +131251 POINT(39.8252495390815 73.49458949544339) bank131251 +131252 POINT(40.29413713193891 73.59746881693289) bank131252 +131253 POINT(41.65028073275819 74.65667179697986) bank131253 +131254 POINT(41.65054996853796 74.86267242127433) bank131254 +131255 POINT(40.100400012908295 73.92622667293311) bank131255 +131256 POINT(40.97601687230221 74.32452209365785) bank131256 +131257 POINT(40.94960536237963 74.6137092642614) bank131257 +131258 POINT(40.91909238213128 74.9944780416515) bank131258 +131259 POINT(40.948443779205604 73.82976564283344) bank131259 +131260 POINT(39.98442063206797 74.95330713897428) bank131260 +131261 POINT(41.19212604893285 74.83615064331413) bank131261 +131262 POINT(40.062204657680034 73.39736759753049) bank131262 +131263 POINT(41.42890670257682 73.44883491252558) bank131263 +131264 POINT(40.56653565650573 73.61709835818291) bank131264 +131265 POINT(40.13874287617604 73.59993204012166) bank131265 +131266 POINT(40.10485443159356 74.39725901627979) bank131266 +131267 POINT(40.79227748775567 74.36539581679685) bank131267 +131268 POINT(39.79113133295317 74.30123418342764) bank131268 +131269 POINT(40.75947140198137 74.50901654316597) bank131269 +131270 POINT(40.06651031291122 74.88333297713594) bank131270 +131271 POINT(40.08466253100902 74.06175799109873) bank131271 +131272 POINT(41.571753500762085 73.85297039037106) bank131272 +131273 POINT(40.76677499190263 73.17227844048203) bank131273 +131274 POINT(40.30314967257557 74.32677115587246) bank131274 +131275 POINT(40.138808859164655 74.90356678215423) bank131275 +131276 POINT(39.919988049590906 74.60650530752726) bank131276 +131277 POINT(40.55385552630371 73.57127748412496) bank131277 +131278 POINT(40.07749007545435 74.02575892724276) bank131278 +131279 POINT(40.1909577502998 73.38601809593303) bank131279 +131280 POINT(41.71237701257132 73.11304278273239) bank131280 +131281 POINT(40.018128785956876 73.53294927307748) bank131281 +131282 POINT(39.7307321760375 73.3025379217856) bank131282 +131283 POINT(41.6175418401992 74.52550507142352) bank131283 +131284 POINT(39.89535346420247 74.2179636763306) bank131284 +131285 POINT(40.40783921539197 73.13841065604106) bank131285 +131286 POINT(40.80988493805904 73.861643004634) bank131286 +131287 POINT(40.66653065387094 74.54170606174301) bank131287 +131288 POINT(41.19492146978757 73.97135697563532) bank131288 +131289 POINT(41.30718005340514 74.45721243134854) bank131289 +131290 POINT(40.56200259329741 74.08313916201246) bank131290 +131291 POINT(40.772883498736064 74.81248068780545) bank131291 +131292 POINT(40.195434233258055 73.32496704643881) bank131292 +131293 POINT(40.77446581202653 73.42823326673582) bank131293 +131294 POINT(41.184904400908444 73.58831888374384) bank131294 +131295 POINT(39.796679485928266 73.25526911708084) bank131295 +131296 POINT(40.1978151685218 73.41592471003622) bank131296 +131297 POINT(41.49604588054704 74.80430310251123) bank131297 +131298 POINT(41.238185936279145 74.3532188615208) bank131298 +131299 POINT(40.322821760149495 73.83821555390075) bank131299 +131300 POINT(41.06031387890456 74.730557673725) bank131300 +131301 POINT(40.713105240999816 74.07904212921662) bank131301 +131302 POINT(40.472087018273676 74.09315423544426) bank131302 +131303 POINT(40.38208789922136 74.31182446184985) bank131303 +131304 POINT(41.25302130602181 73.35614613863075) bank131304 +131305 POINT(40.75814176608962 73.95544310188399) bank131305 +131306 POINT(40.29498108937177 73.82844276967302) bank131306 +131307 POINT(41.10252461305025 74.66285251423582) bank131307 +131308 POINT(40.81919139347494 74.07463192627893) bank131308 +131309 POINT(40.773288770416265 73.1608439723273) bank131309 +131310 POINT(41.65862558458902 73.8574574481152) bank131310 +131311 POINT(41.53268492256746 74.94696054738276) bank131311 +131312 POINT(39.760317659584814 74.18799997741317) bank131312 +131313 POINT(41.39903449032953 73.28537342361494) bank131313 +131314 POINT(41.07176635171172 74.35696974059721) bank131314 +131315 POINT(40.61064769480393 73.52501331746826) bank131315 +131316 POINT(41.42710881901039 73.50262583484428) bank131316 +131317 POINT(41.55048779672388 74.59508391401654) bank131317 +131318 POINT(40.231295340205406 74.79869668325175) bank131318 +131319 POINT(41.31491826061542 74.31718279884174) bank131319 +131320 POINT(40.031162970183146 73.4896620280236) bank131320 +131321 POINT(40.580079966626485 73.56576870905424) bank131321 +131322 POINT(41.5379174194641 74.50810406395723) bank131322 +131323 POINT(39.91408296279488 73.96794305157364) bank131323 +131324 POINT(40.19046915024085 73.10736650465385) bank131324 +131325 POINT(39.890369168272485 74.58868717518533) bank131325 +131326 POINT(40.65072876118391 73.50624729209686) bank131326 +131327 POINT(41.019386074090555 73.47993598011244) bank131327 +131328 POINT(40.17601494718942 73.61147096289318) bank131328 +131329 POINT(40.13604027769503 73.68151817468815) bank131329 +131330 POINT(41.040243093294926 73.52613544094685) bank131330 +131331 POINT(41.3850004554726 74.4580811904205) bank131331 +131332 POINT(40.313547006193225 74.21952569518368) bank131332 +131333 POINT(40.66145361125373 73.44789761976433) bank131333 +131334 POINT(40.9433192995688 74.70012431375301) bank131334 +131335 POINT(40.96379620129956 73.92067060247417) bank131335 +131336 POINT(39.741683667976076 74.71125787935262) bank131336 +131337 POINT(39.92537502722344 74.0792836705474) bank131337 +131338 POINT(41.31770837167382 74.7359198392994) bank131338 +131339 POINT(39.91150538747539 74.75722968339991) bank131339 +131340 POINT(39.77691537416916 73.43314603937338) bank131340 +131341 POINT(39.95195731161303 73.77384486476006) bank131341 +131342 POINT(40.11200157690249 73.09224489626214) bank131342 +131343 POINT(40.81282778818495 74.10617351306321) bank131343 +131344 POINT(41.13808583496489 74.65076519520095) bank131344 +131345 POINT(40.28743244086827 74.31170151251607) bank131345 +131346 POINT(40.25129616236077 74.84227883213073) bank131346 +131347 POINT(41.558278769427 74.69370326766418) bank131347 +131348 POINT(39.74542408289004 74.26421923714565) bank131348 +131349 POINT(40.917363934242495 74.53182815100811) bank131349 +131350 POINT(41.19954331820374 73.43608686550306) bank131350 +131351 POINT(39.87359096495915 74.73073299765413) bank131351 +131352 POINT(40.6702058199601 73.47390369694288) bank131352 +131353 POINT(40.893034514357375 74.06337215132022) bank131353 +131354 POINT(41.62757679824182 73.35675559816309) bank131354 +131355 POINT(40.42983632532539 74.8576647366806) bank131355 +131356 POINT(40.89698364015051 74.19217664106391) bank131356 +131357 POINT(40.878777034169865 73.62209851110379) bank131357 +131358 POINT(41.49075352180919 73.13666262246956) bank131358 +131359 POINT(41.16404719277907 74.72862028093434) bank131359 +131360 POINT(41.592537889351114 73.37231344124103) bank131360 +131361 POINT(40.6970211455926 73.17214010853252) bank131361 +131362 POINT(41.55403345626572 73.13576625040217) bank131362 +131363 POINT(41.69057001982591 74.76268238658827) bank131363 +131364 POINT(41.07586974646907 74.90599926459177) bank131364 +131365 POINT(41.17470243153925 74.87956329550889) bank131365 +131366 POINT(41.263216305491945 74.54936937467177) bank131366 +131367 POINT(41.30334041650207 73.81538535879584) bank131367 +131368 POINT(39.94365927128382 74.04661086078096) bank131368 +131369 POINT(39.79705983894141 74.90331419069987) bank131369 +131370 POINT(40.45576312456741 73.11228180719169) bank131370 +131371 POINT(41.23912068761948 73.19870561880352) bank131371 +131372 POINT(40.955938752284275 73.68922063612139) bank131372 +131373 POINT(40.35257333118178 74.89563993873458) bank131373 +131374 POINT(41.29846078611294 74.6785932269504) bank131374 +131375 POINT(40.47996713316397 73.54259743856511) bank131375 +131376 POINT(41.47046713117357 73.51588759821688) bank131376 +131377 POINT(39.973903834050866 74.04629396509468) bank131377 +131378 POINT(40.302355566854466 74.79169478315679) bank131378 +131379 POINT(40.16591704227201 74.03687760617665) bank131379 +131380 POINT(41.579084204562065 73.9248400817777) bank131380 +131381 POINT(40.73477751510715 74.31016228110992) bank131381 +131382 POINT(40.95242889871478 73.99494830512134) bank131382 +131383 POINT(40.32766118098642 74.50844914466559) bank131383 +131384 POINT(40.54520867168223 73.52300544948258) bank131384 +131385 POINT(40.38586963291692 74.01803101848722) bank131385 +131386 POINT(40.97636562508493 73.47675836221099) bank131386 +131387 POINT(40.549922748856204 74.6662276966249) bank131387 +131388 POINT(41.49030290571391 74.39636075376147) bank131388 +131389 POINT(40.40991795491923 74.69980691904746) bank131389 +131390 POINT(40.340492078180176 74.19489088117408) bank131390 +131391 POINT(40.05804595234058 74.38040838803819) bank131391 +131392 POINT(41.54415284358411 73.42812224071844) bank131392 +131393 POINT(41.08046357484921 73.66760980821772) bank131393 +131394 POINT(41.13235000107622 74.84137698326029) bank131394 +131395 POINT(41.599898754317316 73.31167600357456) bank131395 +131396 POINT(40.10699433983864 73.69476719712013) bank131396 +131397 POINT(41.543406935186646 74.87674343657481) bank131397 +131398 POINT(41.09271749089451 73.29087203303101) bank131398 +131399 POINT(40.861597710375186 73.76279657052605) bank131399 +131400 POINT(41.29066218207961 73.55957387819029) bank131400 +131401 POINT(41.32083717226605 74.00382974076155) bank131401 +131402 POINT(41.28493683903009 73.92854103587862) bank131402 +131403 POINT(39.90632235913649 73.05208714570445) bank131403 +131404 POINT(40.20584463569463 73.13554738061804) bank131404 +131405 POINT(40.72720404912407 74.30474540508028) bank131405 +131406 POINT(40.879454401522885 74.20552105296963) bank131406 +131407 POINT(40.24458034893244 74.8043709193104) bank131407 +131408 POINT(41.53874434233974 74.19003364682007) bank131408 +131409 POINT(39.91795905741874 73.93932034807396) bank131409 +131410 POINT(40.504780175827 74.31962080100331) bank131410 +131411 POINT(40.7583354240582 73.21022767216469) bank131411 +131412 POINT(40.095072730137375 74.25626391943733) bank131412 +131413 POINT(40.251424364696625 74.95432793983781) bank131413 +131414 POINT(40.68551891015955 73.75655178969414) bank131414 +131415 POINT(39.75365723575681 74.04583110441864) bank131415 +131416 POINT(40.870792681144046 73.48706350492216) bank131416 +131417 POINT(40.24258804371318 74.5601053494433) bank131417 +131418 POINT(40.38265618894397 73.54780285292746) bank131418 +131419 POINT(40.27943642954242 74.74842989344495) bank131419 +131420 POINT(40.645984869224655 74.39981023772232) bank131420 +131421 POINT(41.608840181733555 73.89157642662701) bank131421 +131422 POINT(40.960668404985284 73.31487915360306) bank131422 +131423 POINT(40.16015060950282 74.3352321922869) bank131423 +131424 POINT(41.59925580127514 74.41633365910519) bank131424 +131425 POINT(40.75780538464934 74.58230880032659) bank131425 +131426 POINT(40.98977912563823 73.26204023561091) bank131426 +131427 POINT(41.40474991869267 73.66086795016838) bank131427 +131428 POINT(41.07579735960255 73.84601671377258) bank131428 +131429 POINT(41.4825576178687 74.05196157092696) bank131429 +131430 POINT(41.23510352236335 74.90433058322655) bank131430 +131431 POINT(39.92894370449689 73.60683209496777) bank131431 +131432 POINT(40.47310736020757 74.64474183802969) bank131432 +131433 POINT(40.9105837658465 73.90241172506292) bank131433 +131434 POINT(40.80990923608244 73.87212342945875) bank131434 +131435 POINT(39.8898466269482 73.90916611355283) bank131435 +131436 POINT(41.576889427045444 74.21615811850364) bank131436 +131437 POINT(41.54861907635264 73.10676720995667) bank131437 +131438 POINT(41.02745424029374 74.07132547195917) bank131438 +131439 POINT(41.149730655435555 74.48011636782775) bank131439 +131440 POINT(39.7832028928338 74.01437368288816) bank131440 +131441 POINT(40.38273358476496 74.72401632548691) bank131441 +131442 POINT(41.434960460911924 75.00105903128095) bank131442 +131443 POINT(41.11641094527234 73.82303821723856) bank131443 +131444 POINT(40.84839801517391 74.76613136580463) bank131444 +131445 POINT(41.42415909963371 73.40009215050743) bank131445 +131446 POINT(39.99064464503192 73.14120249086805) bank131446 +131447 POINT(39.72716336712499 73.09536465509065) bank131447 +131448 POINT(40.05771309858636 74.9745248354406) bank131448 +131449 POINT(41.516270030663264 74.9886172634187) bank131449 +131450 POINT(40.25579055615619 73.5580892387523) bank131450 +131451 POINT(40.937689022376134 73.21263351152997) bank131451 +131452 POINT(40.72355941541001 74.69099139030926) bank131452 +131453 POINT(41.12440020267217 73.81610448010859) bank131453 +131454 POINT(40.359296349325426 74.16919078930279) bank131454 +131455 POINT(39.77011831513404 74.99771395383301) bank131455 +131456 POINT(41.71019685258914 74.8891846727449) bank131456 +131457 POINT(39.840340641047874 73.3728359292664) bank131457 +131458 POINT(40.70556120723789 74.57084813210348) bank131458 +131459 POINT(39.76021189540172 73.29526441292131) bank131459 +131460 POINT(41.31232189860871 73.69177103454265) bank131460 +131461 POINT(40.40292631185012 73.68146092063762) bank131461 +131462 POINT(40.267974307613265 74.4618472018476) bank131462 +131463 POINT(41.591531278672434 73.9588100394194) bank131463 +131464 POINT(40.25900667667236 74.6853259649389) bank131464 +131465 POINT(40.54099075684238 73.71323122670596) bank131465 +131466 POINT(39.97330739484135 74.35148083020603) bank131466 +131467 POINT(40.387895635759016 74.44388056706477) bank131467 +131468 POINT(41.672254315659444 73.67554346030539) bank131468 +131469 POINT(41.586596235677206 74.44074074311611) bank131469 +131470 POINT(41.40919512351444 74.75051436113519) bank131470 +131471 POINT(41.57764340839476 74.23726628828045) bank131471 +131472 POINT(41.44681324198598 73.56938117715367) bank131472 +131473 POINT(41.54770873694274 73.73620613349831) bank131473 +131474 POINT(40.03924207836498 74.59532013966583) bank131474 +131475 POINT(40.72303026874482 73.6022092920669) bank131475 +131476 POINT(39.71346949529662 73.26778952259554) bank131476 +131477 POINT(41.23050170629571 74.75264131989218) bank131477 +131478 POINT(41.19124034167473 74.23915281016737) bank131478 +131479 POINT(40.31008373546986 73.19462647267022) bank131479 +131480 POINT(41.403571835332635 73.31650996804917) bank131480 +131481 POINT(41.33090836010675 74.16997028661883) bank131481 +131482 POINT(41.272447714374835 73.2548922890826) bank131482 +131483 POINT(41.184995519537345 74.30697068971978) bank131483 +131484 POINT(39.94679407346553 74.5883722940569) bank131484 +131485 POINT(40.04664012227707 74.05317443400068) bank131485 +131486 POINT(41.02212795011427 73.73079127126485) bank131486 +131487 POINT(40.350370991039206 74.53374754558685) bank131487 +131488 POINT(40.22972066752706 73.02859653512051) bank131488 +131489 POINT(40.85391502024094 73.09538082692785) bank131489 +131490 POINT(41.13250648811002 73.12950240782969) bank131490 +131491 POINT(40.15340070101954 73.73695421381828) bank131491 +131492 POINT(41.656972971421645 73.18225523915748) bank131492 +131493 POINT(40.66821988158234 74.82238521938022) bank131493 +131494 POINT(41.328898645755935 73.7973857550783) bank131494 +131495 POINT(39.830010998425 74.49058018545236) bank131495 +131496 POINT(40.832253486614896 73.39045848077498) bank131496 +131497 POINT(40.696727694048874 73.22502138552247) bank131497 +131498 POINT(41.236759619102905 73.42902932311512) bank131498 +131499 POINT(40.53835513701726 73.98844766785112) bank131499 +131500 POINT(40.6038492339804 73.17129922854889) bank131500 +131501 POINT(41.18138471713473 73.24492275904457) bank131501 +131502 POINT(40.320389612910596 74.25968634547931) bank131502 +131503 POINT(41.36004657461907 73.21426643381739) bank131503 +131504 POINT(40.05059732810618 73.94794961144228) bank131504 +131505 POINT(41.42442401456908 73.58292924805575) bank131505 +131506 POINT(41.16545077197079 73.20278779981986) bank131506 +131507 POINT(40.01049964439399 73.35815546753194) bank131507 +131508 POINT(41.529600307998834 74.30365450126483) bank131508 +131509 POINT(41.04582857896127 74.83906811817033) bank131509 +131510 POINT(40.32025473793813 74.310193141964) bank131510 +131511 POINT(40.48517741473797 73.33081826692924) bank131511 +131512 POINT(40.522692486717375 73.46730317576345) bank131512 +131513 POINT(41.467128556805726 74.76143286217909) bank131513 +131514 POINT(41.63815979054601 73.32960380616171) bank131514 +131515 POINT(40.5517392754955 73.0873608145149) bank131515 +131516 POINT(40.45696246273559 74.21664231135698) bank131516 +131517 POINT(40.847074335590186 74.4019765031999) bank131517 +131518 POINT(40.25769019642043 74.47272142229409) bank131518 +131519 POINT(41.08079885754176 73.38967985280775) bank131519 +131520 POINT(41.26303273783657 73.68253249176102) bank131520 +131521 POINT(41.65177646097142 74.07126437311331) bank131521 +131522 POINT(41.48738189463784 74.71231766354249) bank131522 +131523 POINT(40.4573151613645 74.59019307898454) bank131523 +131524 POINT(41.20951378366642 73.84827435084111) bank131524 +131525 POINT(41.084982524942454 74.83459722032832) bank131525 +131526 POINT(40.92746701272896 74.9453893643917) bank131526 +131527 POINT(40.46576064589196 74.51547686934786) bank131527 +131528 POINT(39.88606220181252 73.91514259325581) bank131528 +131529 POINT(41.180500328455544 74.86560616922651) bank131529 +131530 POINT(39.91031976544099 74.39377932138107) bank131530 +131531 POINT(40.22768906079667 73.45076602539422) bank131531 +131532 POINT(41.55479526232398 74.57363289737418) bank131532 +131533 POINT(40.24879663261139 74.52870637058889) bank131533 +131534 POINT(40.38427278042124 74.42880138023446) bank131534 +131535 POINT(41.456595831568634 74.23997549402412) bank131535 +131536 POINT(40.776510497532236 74.46711015223413) bank131536 +131537 POINT(40.20669185466902 74.05910235075454) bank131537 +131538 POINT(41.64877815159754 73.04544555181972) bank131538 +131539 POINT(39.98122984829722 74.42360963147867) bank131539 +131540 POINT(40.61073673002723 73.56163686961318) bank131540 +131541 POINT(41.15324191639781 73.26244800566006) bank131541 +131542 POINT(40.29346616815721 74.12279577511228) bank131542 +131543 POINT(40.24453726653426 73.16138967956176) bank131543 +131544 POINT(40.62789772765261 73.45578103428129) bank131544 +131545 POINT(40.12488417999614 74.11379171843178) bank131545 +131546 POINT(41.13339756793116 74.20120035223694) bank131546 +131547 POINT(40.81935688726714 73.96878705870121) bank131547 +131548 POINT(40.45683628044102 73.73933181718506) bank131548 +131549 POINT(41.570139598585804 74.71337580387176) bank131549 +131550 POINT(40.77938638066611 74.53434929349683) bank131550 +131551 POINT(41.361509730286656 73.07978210568437) bank131551 +131552 POINT(39.919973041006564 73.89027764715576) bank131552 +131553 POINT(41.006421235338976 73.97657861459847) bank131553 +131554 POINT(40.23123697633891 74.19473514969886) bank131554 +131555 POINT(41.25858402784276 74.30634285684) bank131555 +131556 POINT(41.50936105788135 73.74839113971406) bank131556 +131557 POINT(40.874975225331454 73.5827699889664) bank131557 +131558 POINT(40.69810792285553 74.47479643783353) bank131558 +131559 POINT(40.43504247767879 74.50006481069325) bank131559 +131560 POINT(40.16074709102577 74.00373174354291) bank131560 +131561 POINT(41.47927111074886 73.09631424737184) bank131561 +131562 POINT(41.50773949405344 74.33849478800336) bank131562 +131563 POINT(41.48838122947638 74.70112316992814) bank131563 +131564 POINT(40.55338680321 74.48447522129165) bank131564 +131565 POINT(41.44228667233763 73.66105210704265) bank131565 +131566 POINT(39.731919121425314 73.61156840380207) bank131566 +131567 POINT(40.3005200917747 74.48884286264847) bank131567 +131568 POINT(40.54991397613009 73.124286206201) bank131568 +131569 POINT(40.27898055076074 74.61672001517051) bank131569 +131570 POINT(40.05265397219751 74.09123676387456) bank131570 +131571 POINT(41.38173062806678 74.93544848355603) bank131571 +131572 POINT(40.496264398656095 74.02687754430552) bank131572 +131573 POINT(41.42018871980524 74.11353276354589) bank131573 +131574 POINT(39.950182196767074 73.33201284574393) bank131574 +131575 POINT(41.69007579515154 73.272294110187) bank131575 +131576 POINT(41.00896476966016 73.94275614858469) bank131576 +131577 POINT(40.202557167371296 73.2813337522254) bank131577 +131578 POINT(40.644836689766926 73.61940331530667) bank131578 +131579 POINT(40.056478209972276 73.32944638808827) bank131579 +131580 POINT(39.85119772815859 74.31460101300561) bank131580 +131581 POINT(39.73496262477205 74.06368386517029) bank131581 +131582 POINT(39.94773460836608 75.00017955011697) bank131582 +131583 POINT(41.571938630572625 73.92824469003827) bank131583 +131584 POINT(41.21060450053179 73.41527609837314) bank131584 +131585 POINT(39.90300054925141 74.73415003188423) bank131585 +131586 POINT(41.558691235094805 74.00764807705075) bank131586 +131587 POINT(40.80964291495828 74.2870879344161) bank131587 +131588 POINT(39.84928765968994 73.45431574896533) bank131588 +131589 POINT(40.20186272698246 74.23315120638881) bank131589 +131590 POINT(39.71301640729743 74.28502587747623) bank131590 +131591 POINT(39.87936016221469 74.23770049290898) bank131591 +131592 POINT(40.07285189238634 73.84512189476592) bank131592 +131593 POINT(40.99383209251753 74.91389347264034) bank131593 +131594 POINT(41.342805942453175 73.4621606629367) bank131594 +131595 POINT(39.74198247642377 74.41243866363835) bank131595 +131596 POINT(41.418791351642355 74.36316234882759) bank131596 +131597 POINT(40.661833307124056 73.86520121293441) bank131597 +131598 POINT(41.69204943280301 73.56452374833755) bank131598 +131599 POINT(41.34887962218841 74.38334598435223) bank131599 +131600 POINT(40.658678666908784 74.41777083459407) bank131600 +131601 POINT(40.053752199596424 73.90811645318193) bank131601 +131602 POINT(41.13275895437372 73.24341522486438) bank131602 +131603 POINT(41.66213568469102 74.35234739711129) bank131603 +131604 POINT(40.14251841422479 73.93643220604288) bank131604 +131605 POINT(41.27796945239236 74.15714622621029) bank131605 +131606 POINT(40.32703742538893 73.60863977108427) bank131606 +131607 POINT(40.93924116880544 73.42058466379129) bank131607 +131608 POINT(40.123787012941634 74.97864522542388) bank131608 +131609 POINT(40.614477863938795 73.15134982832821) bank131609 +131610 POINT(40.44099781859283 74.24382089708726) bank131610 +131611 POINT(39.92722822515985 73.4191294360193) bank131611 +131612 POINT(41.27088842292248 73.9703120224864) bank131612 +131613 POINT(40.61088666442382 74.81125565753871) bank131613 +131614 POINT(41.241244906898444 73.15174300212477) bank131614 +131615 POINT(40.28419191424472 74.87418448290575) bank131615 +131616 POINT(41.15414351773636 73.33531471451367) bank131616 +131617 POINT(40.34387719391629 73.23838700888386) bank131617 +131618 POINT(40.23201624665685 74.00987422606043) bank131618 +131619 POINT(40.168387190089184 74.01003565587953) bank131619 +131620 POINT(40.35647457557456 74.76050806025992) bank131620 +131621 POINT(39.71677688840411 73.21744790654827) bank131621 +131622 POINT(40.7986533947455 73.88404864550117) bank131622 +131623 POINT(39.872094825910665 73.23296825692559) bank131623 +131624 POINT(39.73151858089108 73.51962563309372) bank131624 +131625 POINT(41.20217095438415 73.14831861068657) bank131625 +131626 POINT(39.88183270904067 74.56419380493806) bank131626 +131627 POINT(40.922769530400565 74.86055021897737) bank131627 +131628 POINT(41.665847666624124 73.57232266999975) bank131628 +131629 POINT(40.936507390254384 73.68563376914216) bank131629 +131630 POINT(41.22223267289114 74.88522656107808) bank131630 +131631 POINT(40.70987060308088 73.90642874421502) bank131631 +131632 POINT(40.441608240819434 73.37926796496961) bank131632 +131633 POINT(40.739074257948545 73.97574734163294) bank131633 +131634 POINT(41.46563307151993 74.59844632582926) bank131634 +131635 POINT(39.95441672102396 73.36179748701002) bank131635 +131636 POINT(40.17883433580474 73.24354856177438) bank131636 +131637 POINT(41.4898638687668 73.15519039649698) bank131637 +131638 POINT(39.71837550534423 73.30505277833105) bank131638 +131639 POINT(41.49636180754681 74.10288856147791) bank131639 +131640 POINT(41.41059737461585 73.83254351090058) bank131640 +131641 POINT(41.60817339074265 73.60861856239688) bank131641 +131642 POINT(40.593467126161805 74.70892503294458) bank131642 +131643 POINT(41.52599121133132 73.82375930137825) bank131643 +131644 POINT(40.79872160154622 73.05748137259987) bank131644 +131645 POINT(40.51075276422781 73.7036954599613) bank131645 +131646 POINT(40.61189649683595 74.62008570962634) bank131646 +131647 POINT(41.03536855862681 74.93524206512778) bank131647 +131648 POINT(40.07136859231374 73.14595816903287) bank131648 +131649 POINT(39.83884556447939 73.80649682028462) bank131649 +131650 POINT(39.95544076247017 74.94065287091377) bank131650 +131651 POINT(40.09835735059502 74.1362581611492) bank131651 +131652 POINT(41.658558355194025 73.61379710143757) bank131652 +131653 POINT(39.72666251586558 74.85016675881268) bank131653 +131654 POINT(39.75261189855505 73.74753060698745) bank131654 +131655 POINT(41.39221045935565 73.285296536801) bank131655 +131656 POINT(40.61632465052052 73.07140954368016) bank131656 +131657 POINT(41.690679302536324 74.53048462273584) bank131657 +131658 POINT(40.20543479151124 74.33660099273844) bank131658 +131659 POINT(40.428888780062366 74.66224092116563) bank131659 +131660 POINT(41.57146005347041 73.63462766249147) bank131660 +131661 POINT(41.55317011055998 74.52826485494403) bank131661 +131662 POINT(39.81447717850452 74.00894323117355) bank131662 +131663 POINT(40.04392140476813 74.1713785771591) bank131663 +131664 POINT(41.375875361529204 74.91858433146975) bank131664 +131665 POINT(40.54730393069163 74.85918174441395) bank131665 +131666 POINT(41.564189119292706 74.79374523492785) bank131666 +131667 POINT(40.29717320491234 73.89814748484768) bank131667 +131668 POINT(40.66438925468195 73.19688792822218) bank131668 +131669 POINT(41.44721961763804 74.95496078441862) bank131669 +131670 POINT(40.95248994860613 73.25040381277627) bank131670 +131671 POINT(41.099954328245694 73.09719000872741) bank131671 +131672 POINT(40.296539250992915 73.1098093388629) bank131672 +131673 POINT(41.261646979502764 74.94336409085435) bank131673 +131674 POINT(40.853487334377505 74.60542945271185) bank131674 +131675 POINT(40.97586656061533 73.75886554380519) bank131675 +131676 POINT(40.285891360215224 73.65493796429932) bank131676 +131677 POINT(40.073558518349856 75.0001068331256) bank131677 +131678 POINT(40.22649732613485 73.04621825040988) bank131678 +131679 POINT(39.82576916609883 74.10866780422627) bank131679 +131680 POINT(40.93261878914247 73.16629028626362) bank131680 +131681 POINT(39.926632995302135 74.72581562559908) bank131681 +131682 POINT(41.712100776258474 73.05305362746311) bank131682 +131683 POINT(40.63369125243155 74.42780325713127) bank131683 +131684 POINT(40.16527088845853 74.97027191135568) bank131684 +131685 POINT(40.065834755164644 73.56401859292338) bank131685 +131686 POINT(39.85077273803691 74.53457523674261) bank131686 +131687 POINT(40.92195145391018 74.8709180503281) bank131687 +131688 POINT(39.87242962296552 74.63559712999131) bank131688 +131689 POINT(41.067361509634274 74.65541653960992) bank131689 +131690 POINT(40.46363471711689 73.87352026201694) bank131690 +131691 POINT(40.25747014960994 73.03787380841739) bank131691 +131692 POINT(39.87923020672359 73.92701772011137) bank131692 +131693 POINT(40.08878602226545 73.15396524819944) bank131693 +131694 POINT(41.21558476379068 74.76043694479768) bank131694 +131695 POINT(39.85254282393803 73.9515571551345) bank131695 +131696 POINT(39.7700027515426 74.88621309768976) bank131696 +131697 POINT(40.02810285235433 73.53130935560895) bank131697 +131698 POINT(41.161346188672454 73.68651506938554) bank131698 +131699 POINT(40.7698713864633 74.06014258103377) bank131699 +131700 POINT(41.15669398799707 74.6814785042582) bank131700 +131701 POINT(41.27196907255406 74.56319870353802) bank131701 +131702 POINT(40.03222265470508 73.39616225361037) bank131702 +131703 POINT(40.362139511396364 74.9936575307978) bank131703 +131704 POINT(40.560369114528 74.33576027076093) bank131704 +131705 POINT(41.064367201593626 74.42763737635543) bank131705 +131706 POINT(40.594642882459176 73.61018614820101) bank131706 +131707 POINT(41.68008785673027 74.15824428178495) bank131707 +131708 POINT(40.26130178273353 73.3158776379515) bank131708 +131709 POINT(40.19341028155265 74.63933076570054) bank131709 +131710 POINT(41.64786668240727 74.23408623761894) bank131710 +131711 POINT(40.43202011142962 73.29042139681543) bank131711 +131712 POINT(39.92993463281892 73.77173759358371) bank131712 +131713 POINT(41.163231259629825 74.28756343927986) bank131713 +131714 POINT(40.64669635125538 74.76473441838405) bank131714 +131715 POINT(40.7870557828988 74.0191911500923) bank131715 +131716 POINT(41.36217996969936 74.37159076635031) bank131716 +131717 POINT(41.69916100890563 73.98140994972167) bank131717 +131718 POINT(41.52847985635234 73.56372003273755) bank131718 +131719 POINT(40.485481830235294 73.31453939489172) bank131719 +131720 POINT(41.47318157575145 74.28352627331765) bank131720 +131721 POINT(41.39060699506276 74.82227562161751) bank131721 +131722 POINT(41.711022905258076 74.37285976549116) bank131722 +131723 POINT(41.451146683513144 74.33346564982271) bank131723 +131724 POINT(41.224070696002514 73.31278826391345) bank131724 +131725 POINT(40.434610122571776 73.79276673404829) bank131725 +131726 POINT(41.35651260302332 74.71029399534676) bank131726 +131727 POINT(40.6492079326719 73.95205811712863) bank131727 +131728 POINT(41.2998398303915 73.87064869008722) bank131728 +131729 POINT(40.49258956419132 73.32117611704287) bank131729 +131730 POINT(40.31673755128857 73.13524209720028) bank131730 +131731 POINT(41.35861311069452 73.95388404477818) bank131731 +131732 POINT(40.342683084908124 73.71133486959849) bank131732 +131733 POINT(40.63519329130519 74.07809186723267) bank131733 +131734 POINT(41.00127486452195 74.79971049009723) bank131734 +131735 POINT(40.44918883050404 73.94781316359992) bank131735 +131736 POINT(41.52834881465246 73.97149104678287) bank131736 +131737 POINT(39.997704495862614 73.23399810020103) bank131737 +131738 POINT(41.3525620068076 73.94848691272651) bank131738 +131739 POINT(41.61604324863473 73.8800820174445) bank131739 +131740 POINT(40.519652056802805 73.43998058424394) bank131740 +131741 POINT(40.263453184603485 73.11312897129613) bank131741 +131742 POINT(40.354204661174286 73.51446756357637) bank131742 +131743 POINT(41.66595839109368 74.80332265356459) bank131743 +131744 POINT(40.87211621426788 73.80861053834978) bank131744 +131745 POINT(41.40001723324406 73.43155541921627) bank131745 +131746 POINT(39.87007082238239 74.981590124304) bank131746 +131747 POINT(41.08442435811334 74.75297207035922) bank131747 +131748 POINT(39.90281733492075 74.96061681221624) bank131748 +131749 POINT(41.4655016063999 73.31495224277843) bank131749 +131750 POINT(40.25927925583413 74.40376979360308) bank131750 +131751 POINT(40.396463568924425 74.32260940840081) bank131751 +131752 POINT(39.768817204341566 74.95379519790707) bank131752 +131753 POINT(40.82517151643251 74.1113717570042) bank131753 +131754 POINT(41.4670837730282 74.93594519446789) bank131754 +131755 POINT(41.50122453940731 73.57833271172645) bank131755 +131756 POINT(40.02316206211234 73.33873967990876) bank131756 +131757 POINT(40.917827508744246 73.46240117305116) bank131757 +131758 POINT(40.26631879805119 74.74200309930856) bank131758 +131759 POINT(40.39123747804859 74.49898493364178) bank131759 +131760 POINT(40.44256294931428 73.2031014490049) bank131760 +131761 POINT(40.62908463213127 74.50421681216807) bank131761 +131762 POINT(41.27446738354935 73.47959526112396) bank131762 +131763 POINT(41.29594139811386 73.92663069090314) bank131763 +131764 POINT(40.17626996251793 74.11701788666261) bank131764 +131765 POINT(39.959344849527184 74.02521197373282) bank131765 +131766 POINT(40.02504204772776 74.71480956027519) bank131766 +131767 POINT(40.53368478604603 74.4369085363534) bank131767 +131768 POINT(40.02255800602637 73.48197103169193) bank131768 +131769 POINT(40.48650199386646 73.08744395918475) bank131769 +131770 POINT(40.699750355462946 73.38291455433362) bank131770 +131771 POINT(40.899152383549875 73.1630257125226) bank131771 +131772 POINT(40.24637749584808 74.32672111257935) bank131772 +131773 POINT(40.13936367531595 73.10477139882117) bank131773 +131774 POINT(41.571921943519285 73.308699762473) bank131774 +131775 POINT(41.130835213709304 74.94123144872833) bank131775 +131776 POINT(39.991349684015134 74.74620582207025) bank131776 +131777 POINT(41.308343856788284 74.18407464499974) bank131777 +131778 POINT(40.562548971279085 74.41148891706476) bank131778 +131779 POINT(40.663412409584396 73.05649099533966) bank131779 +131780 POINT(40.71186275606085 74.78649114781324) bank131780 +131781 POINT(41.43141742059874 74.83231857409871) bank131781 +131782 POINT(41.548932293758604 73.3692139712677) bank131782 +131783 POINT(39.77459496500752 73.64725413959732) bank131783 +131784 POINT(40.481864184645644 73.28707166673092) bank131784 +131785 POINT(41.11859847626052 73.99542607415178) bank131785 +131786 POINT(40.025586271921455 73.83289342674861) bank131786 +131787 POINT(40.05574967100044 74.55203959377408) bank131787 +131788 POINT(41.388172975174534 73.20487678344237) bank131788 +131789 POINT(41.48907205135407 73.26725035785049) bank131789 +131790 POINT(39.89320467777394 74.81037602666126) bank131790 +131791 POINT(41.63087899244717 74.00479492751649) bank131791 +131792 POINT(40.81338901137524 73.59372378864694) bank131792 +131793 POINT(41.02538450961537 74.4916926831948) bank131793 +131794 POINT(40.40299608476305 74.12629050414643) bank131794 +131795 POINT(41.39380386006283 73.92257297336745) bank131795 +131796 POINT(40.47483992173909 74.31211236483814) bank131796 +131797 POINT(40.18079284427983 74.35521756177602) bank131797 +131798 POINT(41.398206669574776 73.3389089481499) bank131798 +131799 POINT(41.46064399711793 73.24606971364322) bank131799 +131800 POINT(41.15187340367833 74.95249316210173) bank131800 +131801 POINT(40.807267507523264 73.22092142547423) bank131801 +131802 POINT(40.400047374040696 73.11238783371127) bank131802 +131803 POINT(40.628231195996804 73.53173311053932) bank131803 +131804 POINT(39.81967385845612 73.82587172926748) bank131804 +131805 POINT(40.7840676964822 73.47329473745344) bank131805 +131806 POINT(40.65044933271072 73.63916083778852) bank131806 +131807 POINT(41.248140867462595 73.91187476363885) bank131807 +131808 POINT(40.34779834293028 73.72706342439662) bank131808 +131809 POINT(41.35815697670209 74.25015049199074) bank131809 +131810 POINT(40.71452841288805 74.4098257826286) bank131810 +131811 POINT(40.00314378727652 73.86911195734777) bank131811 +131812 POINT(41.06655402975003 73.86474230487045) bank131812 +131813 POINT(41.250297815443815 74.68868925788077) bank131813 +131814 POINT(40.544550641368204 74.15052213626713) bank131814 +131815 POINT(40.81184853970171 73.61516639844865) bank131815 +131816 POINT(41.145763236574794 73.69776579421155) bank131816 +131817 POINT(40.862760337701694 74.46132367073582) bank131817 +131818 POINT(40.894915160430344 74.2000122421979) bank131818 +131819 POINT(40.05922235363077 74.15507606353273) bank131819 +131820 POINT(41.51435121988024 73.55755448627747) bank131820 +131821 POINT(41.40573265827051 73.04172133127041) bank131821 +131822 POINT(41.35336418997899 73.53073952645784) bank131822 +131823 POINT(41.13988191796782 74.53923072019096) bank131823 +131824 POINT(39.78331788032011 74.89778397347126) bank131824 +131825 POINT(39.87477641054935 73.23229771357799) bank131825 +131826 POINT(41.448507153735264 73.63214521449454) bank131826 +131827 POINT(40.0032037428568 73.91468409348856) bank131827 +131828 POINT(41.106184388368824 73.88347604856384) bank131828 +131829 POINT(39.85359476075185 74.58417957565119) bank131829 +131830 POINT(40.969782936089395 74.26992064272007) bank131830 +131831 POINT(41.33429302887144 74.01668546352613) bank131831 +131832 POINT(40.87522318457975 73.87155196107584) bank131832 +131833 POINT(41.04318957565515 74.8231767434963) bank131833 +131834 POINT(39.713074167075945 73.83175256600053) bank131834 +131835 POINT(41.568003485414046 74.17625173322081) bank131835 +131836 POINT(41.50373601892992 73.27790174879927) bank131836 +131837 POINT(40.83064795090019 74.84511366092079) bank131837 +131838 POINT(41.173023927760966 73.3370490817802) bank131838 +131839 POINT(40.473221500173224 74.70490301135213) bank131839 +131840 POINT(39.921495171639116 74.14769986293766) bank131840 +131841 POINT(41.22739182824704 74.69879722237249) bank131841 +131842 POINT(41.372287718082276 73.26087123077545) bank131842 +131843 POINT(40.91325713561449 73.82396681752577) bank131843 +131844 POINT(41.08281579095806 73.44099113885501) bank131844 +131845 POINT(40.59569165081663 73.28143431558604) bank131845 +131846 POINT(40.162924355132176 74.3225277616481) bank131846 +131847 POINT(40.69099550747466 73.65680670673241) bank131847 +131848 POINT(41.35971838563343 73.6227696181031) bank131848 +131849 POINT(40.884575482003115 74.68709548763677) bank131849 +131850 POINT(41.05318934676318 74.65068451657001) bank131850 +131851 POINT(40.002910506827995 74.31226038150359) bank131851 +131852 POINT(40.088629644401145 74.4889320912551) bank131852 +131853 POINT(40.45091267683408 73.96106243539586) bank131853 +131854 POINT(41.583504941306465 73.97606097279917) bank131854 +131855 POINT(40.23233323422374 73.61712748030278) bank131855 +131856 POINT(40.078735671083905 74.9891401704279) bank131856 +131857 POINT(41.30717333228394 74.32386768932086) bank131857 +131858 POINT(40.59927448709839 73.80739865038214) bank131858 +131859 POINT(40.99135759665913 74.24126457132934) bank131859 +131860 POINT(40.20894524434826 74.63250083580134) bank131860 +131861 POINT(41.54857287337807 73.12402907494696) bank131861 +131862 POINT(40.56134064335691 74.13255404445218) bank131862 +131863 POINT(40.59304258448135 74.22030482406868) bank131863 +131864 POINT(40.50045739672619 73.65212856542017) bank131864 +131865 POINT(40.14086973114895 74.81595023937707) bank131865 +131866 POINT(39.87796528148986 73.39030824289502) bank131866 +131867 POINT(40.77487086895319 73.19488334238368) bank131867 +131868 POINT(41.38668481532167 73.93307561606848) bank131868 +131869 POINT(40.67022254085387 74.62864764545212) bank131869 +131870 POINT(40.94554708336204 73.20207204649166) bank131870 +131871 POINT(40.92053357970533 74.36594262118649) bank131871 +131872 POINT(39.97734521426807 73.93110032511363) bank131872 +131873 POINT(39.78989011946646 73.50011425150906) bank131873 +131874 POINT(40.294131222442466 74.76730449340442) bank131874 +131875 POINT(40.24950577605091 73.89932032042499) bank131875 +131876 POINT(41.180056779755034 73.87346040145592) bank131876 +131877 POINT(40.02550051592257 74.09669396902768) bank131877 +131878 POINT(40.45977034882973 74.6017467059474) bank131878 +131879 POINT(41.56609953954458 73.6143731579476) bank131879 +131880 POINT(40.80939038681775 74.942783973008) bank131880 +131881 POINT(39.8286810481558 74.43467604529653) bank131881 +131882 POINT(40.05665738865317 73.35618322812873) bank131882 +131883 POINT(39.97945041484323 74.44020284554547) bank131883 +131884 POINT(41.12843621874456 74.5194821292276) bank131884 +131885 POINT(40.58416237927141 74.72640062709277) bank131885 +131886 POINT(40.1537538495702 73.02559773066248) bank131886 +131887 POINT(40.35646124792206 73.69946300789496) bank131887 +131888 POINT(41.30641475340217 73.73753991908343) bank131888 +131889 POINT(39.762027162721836 74.9975640929487) bank131889 +131890 POINT(40.78206393647509 74.52532945755522) bank131890 +131891 POINT(41.60722029874557 73.02027055096379) bank131891 +131892 POINT(40.220433607460265 74.92883972984097) bank131892 +131893 POINT(40.651725563193054 73.97478161515144) bank131893 +131894 POINT(40.919296024063904 74.50849993039763) bank131894 +131895 POINT(41.229631636088186 74.72661670448278) bank131895 +131896 POINT(40.568714919358854 73.58301072086488) bank131896 +131897 POINT(40.23994687967172 73.19506901710075) bank131897 +131898 POINT(39.82457783142178 73.03429176261758) bank131898 +131899 POINT(40.291748501429794 74.41087755307079) bank131899 +131900 POINT(40.9473230678864 73.06270019866422) bank131900 +131901 POINT(40.07488499608038 73.95622276537424) bank131901 +131902 POINT(41.15270709200094 73.89229768244279) bank131902 +131903 POINT(41.305939840520594 74.55716711710464) bank131903 +131904 POINT(41.54481753623845 74.55076078877211) bank131904 +131905 POINT(39.91801437360558 74.74428389821767) bank131905 +131906 POINT(41.143013881673355 73.19012408454246) bank131906 +131907 POINT(40.82961314555895 74.61896440774454) bank131907 +131908 POINT(39.878024279167185 73.61084955798837) bank131908 +131909 POINT(40.00911340253644 73.23161044367072) bank131909 +131910 POINT(40.446190415218 74.34240612903822) bank131910 +131911 POINT(40.955462249989075 74.67050011399752) bank131911 +131912 POINT(40.34655617432076 74.44051729448533) bank131912 +131913 POINT(39.919104842966135 74.15985717898221) bank131913 +131914 POINT(39.83335030491826 73.21447382439864) bank131914 +131915 POINT(41.44711306219711 74.148647984591) bank131915 +131916 POINT(41.69765559584983 73.74093473254811) bank131916 +131917 POINT(40.5541628319954 73.28700945671012) bank131917 +131918 POINT(39.936380833267684 73.75907931204435) bank131918 +131919 POINT(39.7165081483982 74.00579127554344) bank131919 +131920 POINT(40.02124960014147 74.94843143091948) bank131920 +131921 POINT(39.831386001359775 74.29345223124216) bank131921 +131922 POINT(39.900344691515585 74.85778543891709) bank131922 +131923 POINT(40.38604744931702 74.64046787129722) bank131923 +131924 POINT(40.256228406231585 73.60375054606948) bank131924 +131925 POINT(40.98497095258573 73.39451074079732) bank131925 +131926 POINT(40.782603574531926 73.79629351161617) bank131926 +131927 POINT(40.836837849751745 74.56437447702294) bank131927 +131928 POINT(41.67331792899276 74.1452850814913) bank131928 +131929 POINT(40.176616617042896 73.89687752082921) bank131929 +131930 POINT(40.56642996159995 74.3474841228678) bank131930 +131931 POINT(40.89809093457023 74.87087475869832) bank131931 +131932 POINT(41.478992804028756 74.84060907827376) bank131932 +131933 POINT(40.64932006176579 74.57552606458803) bank131933 +131934 POINT(41.21497781003266 73.60549618959254) bank131934 +131935 POINT(40.132380354312446 73.72914335532731) bank131935 +131936 POINT(40.744172405213604 73.09318911279664) bank131936 +131937 POINT(41.462705196094866 74.4100819488855) bank131937 +131938 POINT(41.07898324203576 74.2743127736445) bank131938 +131939 POINT(40.44541932514625 73.1322677556487) bank131939 +131940 POINT(41.4539465737578 73.01015736903321) bank131940 +131941 POINT(41.036665252080404 73.1046419142444) bank131941 +131942 POINT(40.375104388291895 74.00223397345516) bank131942 +131943 POINT(39.82488929284478 74.10150251011775) bank131943 +131944 POINT(40.40846655817067 74.56328031908382) bank131944 +131945 POINT(40.42923040331252 73.13464771920462) bank131945 +131946 POINT(39.89320284776897 74.3895945658405) bank131946 +131947 POINT(40.59133974164249 74.02355060176274) bank131947 +131948 POINT(40.56422348032132 73.47611838362837) bank131948 +131949 POINT(39.816051279405826 74.1629369603884) bank131949 +131950 POINT(40.85625055724446 74.91100714227379) bank131950 +131951 POINT(40.974347576602156 74.0980189398944) bank131951 +131952 POINT(41.32826914837522 74.10621135401577) bank131952 +131953 POINT(40.932282431406556 73.09595009569868) bank131953 +131954 POINT(40.28288485349567 73.92925228443853) bank131954 +131955 POINT(40.5678687553768 73.43203930892396) bank131955 +131956 POINT(40.29881783584018 74.5868676842024) bank131956 +131957 POINT(40.33344947778432 74.60856455854749) bank131957 +131958 POINT(40.83262395002856 73.82864570009609) bank131958 +131959 POINT(40.64792376474168 73.74750213640033) bank131959 +131960 POINT(41.12754704374134 73.62611173224121) bank131960 +131961 POINT(40.81150818552175 74.4481100120119) bank131961 +131962 POINT(40.47938583290162 74.100430270365) bank131962 +131963 POINT(39.828760629606954 73.79883562242296) bank131963 +131964 POINT(40.84956770324694 74.18969206353611) bank131964 +131965 POINT(40.66292712365378 73.86481213101541) bank131965 +131966 POINT(41.35931114989274 73.98740318336614) bank131966 +131967 POINT(40.02868904752781 74.47185526812856) bank131967 +131968 POINT(41.2070511377036 73.43346786094354) bank131968 +131969 POINT(41.33193414652827 74.65109233128722) bank131969 +131970 POINT(40.80578750259022 74.68681278294568) bank131970 +131971 POINT(41.48610885739282 74.71163980322024) bank131971 +131972 POINT(41.413304797331776 74.4286455993643) bank131972 +131973 POINT(40.90885514315097 73.25439317667639) bank131973 +131974 POINT(40.665979277231024 73.77232357482406) bank131974 +131975 POINT(41.703787724785954 73.68080588130918) bank131975 +131976 POINT(41.632036023870135 73.52322217053687) bank131976 +131977 POINT(41.520588723675125 74.85530606855238) bank131977 +131978 POINT(41.104580256237035 73.05418240707291) bank131978 +131979 POINT(41.19640006847585 74.85790253988533) bank131979 +131980 POINT(40.999662419217664 74.43332923759438) bank131980 +131981 POINT(41.49158673018747 73.57237533238491) bank131981 +131982 POINT(39.858376678182466 74.96979607858387) bank131982 +131983 POINT(40.708415554105414 73.69747430096353) bank131983 +131984 POINT(39.82669465686638 74.61887682645943) bank131984 +131985 POINT(40.163187015486166 73.042621834723) bank131985 +131986 POINT(40.33091661915348 73.13936285852233) bank131986 +131987 POINT(39.74769892135507 73.28806723166244) bank131987 +131988 POINT(40.47139414953256 73.83767284908707) bank131988 +131989 POINT(40.335493118529094 73.82925276628175) bank131989 +131990 POINT(39.92712041813815 73.6619352958176) bank131990 +131991 POINT(40.922639962710036 74.3356234457603) bank131991 +131992 POINT(40.06056484362158 73.69262853248233) bank131992 +131993 POINT(40.305279621049806 74.15421207614037) bank131993 +131994 POINT(40.670472706420504 74.41513897968629) bank131994 +131995 POINT(40.71401964880136 73.05306383535692) bank131995 +131996 POINT(40.16543210533671 74.96894054795902) bank131996 +131997 POINT(41.274930900665346 74.14160827065945) bank131997 +131998 POINT(40.78944513217848 74.54085707809112) bank131998 +131999 POINT(40.68577217555042 73.59362590797696) bank131999 +132000 POINT(41.608482569577696 73.455436700488) bank132000 +132001 POINT(41.60783699674907 73.86970837168737) bank132001 +132002 POINT(41.67242806415883 74.99797559693589) bank132002 +132003 POINT(41.17003073435705 74.22946865808204) bank132003 +132004 POINT(40.88084872787148 73.12770242539398) bank132004 +132005 POINT(39.729077969469856 74.04662658972192) bank132005 +132006 POINT(40.681420116450354 74.5350397266096) bank132006 +132007 POINT(40.66191972616928 73.70230020439587) bank132007 +132008 POINT(40.32305230086545 74.89820148517975) bank132008 +132009 POINT(41.19505181606333 73.96164932110273) bank132009 +132010 POINT(40.521797013882384 73.7327544991042) bank132010 +132011 POINT(40.61927619763154 74.77867326191594) bank132011 +132012 POINT(40.21920307465062 73.50612295320201) bank132012 +132013 POINT(39.89717663680682 73.61125540750234) bank132013 +132014 POINT(40.470532374216454 74.11410381429052) bank132014 +132015 POINT(41.21385017564322 73.94099100760461) bank132015 +132016 POINT(40.94518839150887 73.01603013313988) bank132016 +132017 POINT(40.14117625040265 73.11763285921461) bank132017 +132018 POINT(40.37849201160141 74.24320613033944) bank132018 +132019 POINT(41.35340027958723 73.4454772826809) bank132019 +132020 POINT(40.39287517427506 74.46896794990101) bank132020 +132021 POINT(41.582580039907505 74.75239113910425) bank132021 +132022 POINT(39.782352393914046 73.7632660605635) bank132022 +132023 POINT(41.25481835643385 74.54047836577521) bank132023 +132024 POINT(40.906958845649555 74.99682158205052) bank132024 +132025 POINT(41.15364350537131 74.92398057373444) bank132025 +132026 POINT(41.352451500069165 74.75889346517205) bank132026 +132027 POINT(41.08027978329584 74.50530533635342) bank132027 +132028 POINT(40.65408396641909 74.8803864581611) bank132028 +132029 POINT(41.41612466840449 74.52111104705544) bank132029 +132030 POINT(41.31701624425217 74.0853139618258) bank132030 +132031 POINT(39.90315681648112 73.94669241154496) bank132031 +132032 POINT(39.79333161689297 73.26842440754821) bank132032 +132033 POINT(41.69920189690772 74.25361505523675) bank132033 +132034 POINT(39.73762592874351 73.7858923351973) bank132034 +132035 POINT(40.67117689311466 74.35272714565758) bank132035 +132036 POINT(40.67538677552544 74.77688346876828) bank132036 +132037 POINT(41.69610939589187 73.74033099771358) bank132037 +132038 POINT(39.78734099100717 73.4426767344218) bank132038 +132039 POINT(39.91951009098243 74.80942140529329) bank132039 +132040 POINT(40.839464540233124 74.11534253381959) bank132040 +132041 POINT(40.464883003675986 73.66847103186537) bank132041 +132042 POINT(41.585178998720174 74.05711597891519) bank132042 +132043 POINT(40.180794736533045 73.60863663984965) bank132043 +132044 POINT(39.96294487783399 73.23014485995319) bank132044 +132045 POINT(40.12242405447769 73.25800941984994) bank132045 +132046 POINT(40.632366601484975 74.09130273537104) bank132046 +132047 POINT(40.859658037376306 74.10608788037487) bank132047 +132048 POINT(40.07727386784758 74.18222522879813) bank132048 +132049 POINT(39.775169617264936 74.63917759204313) bank132049 +132050 POINT(39.87246064388416 73.46112710467699) bank132050 +132051 POINT(40.60932132816852 73.61653579727944) bank132051 +132052 POINT(41.664685443813426 73.42975441270659) bank132052 +132053 POINT(41.414756899252616 73.38629392647759) bank132053 +132054 POINT(41.48431820989549 73.76573864455638) bank132054 +132055 POINT(40.03192171875279 74.56587903263471) bank132055 +132056 POINT(41.53875410503322 74.08490285970964) bank132056 +132057 POINT(41.418085010959125 74.809310989289) bank132057 +132058 POINT(40.49408817318735 74.5988813117611) bank132058 +132059 POINT(41.67696629419378 74.4332800625341) bank132059 +132060 POINT(41.10184850540699 73.76874122471858) bank132060 +132061 POINT(40.23056484917982 74.85545202492737) bank132061 +132062 POINT(40.31007080966179 73.10550543464976) bank132062 +132063 POINT(39.714929870586204 73.05161651387753) bank132063 +132064 POINT(39.73452000252189 73.01765084945043) bank132064 +132065 POINT(40.94444861510052 73.8832978588165) bank132065 +132066 POINT(40.87677741125013 74.35294748114919) bank132066 +132067 POINT(40.51539334087628 74.4445106703778) bank132067 +132068 POINT(39.84431484432639 73.59668742100881) bank132068 +132069 POINT(40.09727214031164 73.19229647997122) bank132069 +132070 POINT(40.56991590720557 73.73053830655691) bank132070 +132071 POINT(40.82198479602301 74.71263629491229) bank132071 +132072 POINT(39.825509541883534 74.38241675429832) bank132072 +132073 POINT(41.26456783954161 74.1553006912478) bank132073 +132074 POINT(39.717405108234935 73.14257429802568) bank132074 +132075 POINT(40.810520848444924 74.45587392615097) bank132075 +132076 POINT(40.23530133911924 73.64184728028505) bank132076 +132077 POINT(40.07838536260797 74.58449752358112) bank132077 +132078 POINT(40.010011898118435 73.53585186031263) bank132078 +132079 POINT(41.30532875379367 73.90405146250464) bank132079 +132080 POINT(41.081355618618375 74.76781129653295) bank132080 +132081 POINT(40.767276265994184 73.84575026479965) bank132081 +132082 POINT(41.534794387520535 73.4483642636544) bank132082 +132083 POINT(41.37188460892728 74.05366329782515) bank132083 +132084 POINT(41.4208566636547 74.74124902401817) bank132084 +132085 POINT(41.68019493482386 73.07007771993585) bank132085 +132086 POINT(41.43908569006131 74.67934200678144) bank132086 +132087 POINT(39.77049125216556 74.20992196888118) bank132087 +132088 POINT(40.25560678083466 74.65347647139806) bank132088 +132089 POINT(41.3081105584638 74.28105252205148) bank132089 +132090 POINT(41.02055421629404 73.37909423134298) bank132090 +132091 POINT(41.12725295084396 73.8704387879063) bank132091 +132092 POINT(40.84319018741016 74.3857530238171) bank132092 +132093 POINT(40.858728100768836 73.1838805301288) bank132093 +132094 POINT(40.232482803530644 74.62573482463507) bank132094 +132095 POINT(40.93011284006936 74.50665222774605) bank132095 +132096 POINT(39.95738585661357 74.96702131714827) bank132096 +132097 POINT(40.29738261888348 74.83638893136981) bank132097 +132098 POINT(41.6156784391583 74.93492596798048) bank132098 +132099 POINT(40.61892551845485 73.07317794616256) bank132099 +132100 POINT(40.261268648401234 73.10558370098272) bank132100 +132101 POINT(41.24991718573834 74.3451063142038) bank132101 +132102 POINT(39.793333467247265 73.87142612026348) bank132102 +132103 POINT(40.29828674038345 73.36356228062557) bank132103 +132104 POINT(40.94732735662308 73.18537040339368) bank132104 +132105 POINT(39.745885828952765 74.63464619921602) bank132105 +132106 POINT(41.28578941937133 73.30606696727762) bank132106 +132107 POINT(39.72749210466236 73.73648080314582) bank132107 +132108 POINT(40.48246748714382 73.40430255730293) bank132108 +132109 POINT(40.97439915636961 73.29681304534343) bank132109 +132110 POINT(41.36721168670216 73.71599182963965) bank132110 +132111 POINT(41.44844585759356 74.80593882922169) bank132111 +132112 POINT(40.8266720001674 73.87557976809447) bank132112 +132113 POINT(40.91568142018036 74.78627659504836) bank132113 +132114 POINT(40.00378221836532 74.74818598041847) bank132114 +132115 POINT(40.60723823917618 73.8809157327687) bank132115 +132116 POINT(41.186781802532145 73.68737105240731) bank132116 +132117 POINT(39.959340929703 74.68822641529827) bank132117 +132118 POINT(41.00507658360318 73.51823555668663) bank132118 +132119 POINT(40.9166564171029 73.32105023164866) bank132119 +132120 POINT(41.65640967200287 74.51681713063607) bank132120 +132121 POINT(40.900708776436254 73.16518928680998) bank132121 +132122 POINT(41.09595826578865 74.92684812388117) bank132122 +132123 POINT(40.6265384472999 74.8163706422568) bank132123 +132124 POINT(41.15709500686853 74.69642619550181) bank132124 +132125 POINT(40.59577653140368 74.40955963590953) bank132125 +132126 POINT(39.791426760625875 74.84913118266643) bank132126 +132127 POINT(41.587530375790436 73.09863125198278) bank132127 +132128 POINT(40.926306175502006 74.79041960918073) bank132128 +132129 POINT(41.20712064024416 74.41464931178999) bank132129 +132130 POINT(41.08265083911625 73.45291375541325) bank132130 +132131 POINT(39.76632307337644 74.4144559192857) bank132131 +132132 POINT(41.00816483643134 73.12928719412537) bank132132 +132133 POINT(41.119557202428645 73.76907440697173) bank132133 +132134 POINT(40.650087922790945 73.0795902808975) bank132134 +132135 POINT(41.39427515774899 74.3994573587018) bank132135 +132136 POINT(40.19862309766405 73.51408006573843) bank132136 +132137 POINT(39.96253755933036 73.1764063599137) bank132137 +132138 POINT(39.809469361653 74.50724217956174) bank132138 +132139 POINT(41.057873025478 73.88019117495554) bank132139 +132140 POINT(41.21873127215042 73.15539892882137) bank132140 +132141 POINT(40.27568580963175 73.58047323985572) bank132141 +132142 POINT(40.52821634892659 73.83918720728742) bank132142 +132143 POINT(40.38395316719431 75.00103955081642) bank132143 +132144 POINT(40.84397839857074 73.29701120978979) bank132144 +132145 POINT(40.71797228622523 74.45161022669603) bank132145 +132146 POINT(41.24603875807729 73.14893083511684) bank132146 +132147 POINT(41.39270708878378 74.68934326698195) bank132147 +132148 POINT(41.670787662001565 74.44204496526814) bank132148 +132149 POINT(40.555465636383715 74.36954878032226) bank132149 +132150 POINT(41.056824462097914 73.57636447990268) bank132150 +132151 POINT(41.46249080777306 73.52706328477493) bank132151 +132152 POINT(40.649545894203655 74.26485203485737) bank132152 +132153 POINT(41.574143004328455 74.22099799351564) bank132153 +132154 POINT(41.345407836766405 74.28177281720045) bank132154 +132155 POINT(41.13451340152054 74.8522507330704) bank132155 +132156 POINT(39.77863258725164 73.36693868980295) bank132156 +132157 POINT(40.569241862069774 73.51137385837686) bank132157 +132158 POINT(41.01809390356466 74.77025996636701) bank132158 +132159 POINT(40.472672323461104 73.55537769080567) bank132159 +132160 POINT(41.06830322312007 73.28161860370479) bank132160 +132161 POINT(40.053050608050775 73.56569232619687) bank132161 +132162 POINT(39.91484703482174 73.8938295580261) bank132162 +132163 POINT(41.362103175767515 74.5319976731313) bank132163 +132164 POINT(40.047584517047284 73.87764023256891) bank132164 +132165 POINT(40.216099589588474 73.91317660222022) bank132165 +132166 POINT(39.914864057511714 74.773737418024) bank132166 +132167 POINT(40.33667705044324 74.86383972403684) bank132167 +132168 POINT(40.38198160870634 73.49516327031414) bank132168 +132169 POINT(39.98696272796896 74.02703097632998) bank132169 +132170 POINT(41.11015270401898 74.04035990728264) bank132170 +132171 POINT(41.70914980164384 73.11655474133727) bank132171 +132172 POINT(40.42415404105386 73.3506309858981) bank132172 +132173 POINT(40.492891850161776 73.09231405467169) bank132173 +132174 POINT(40.86226922250749 74.2085082967548) bank132174 +132175 POINT(40.45452785573695 74.46812296575544) bank132175 +132176 POINT(39.89581537054052 74.8637218287431) bank132176 +132177 POINT(39.74534911660726 74.28955675248903) bank132177 +132178 POINT(40.71483446380639 74.64248243655463) bank132178 +132179 POINT(41.64543203747134 73.97605227435245) bank132179 +132180 POINT(40.680160828378945 73.16241206020496) bank132180 +132181 POINT(40.9258038366354 74.11947915267673) bank132181 +132182 POINT(39.72885472779094 74.59432557007015) bank132182 +132183 POINT(40.08061126944825 73.37716389568385) bank132183 +132184 POINT(39.76259032481462 74.20870128116687) bank132184 +132185 POINT(40.93279398674128 73.25696323475343) bank132185 +132186 POINT(40.5783804244423 74.99976618316262) bank132186 +132187 POINT(41.04853192869328 74.95309796334321) bank132187 +132188 POINT(41.55921424969768 73.43578932652566) bank132188 +132189 POINT(41.096851836093926 74.99311638423383) bank132189 +132190 POINT(40.26104758350428 74.70671459349289) bank132190 +132191 POINT(40.303788436854774 73.86230819809902) bank132191 +132192 POINT(40.56039867300305 74.30779378659638) bank132192 +132193 POINT(40.65784502855052 73.1095039283021) bank132193 +132194 POINT(40.03526602113957 73.41462482612702) bank132194 +132195 POINT(41.307860490321666 74.82151341051873) bank132195 +132196 POINT(40.33999216647743 73.27671097515432) bank132196 +132197 POINT(39.91118517707279 74.82687072959727) bank132197 +132198 POINT(41.501242087618266 73.06900831740941) bank132198 +132199 POINT(41.56193200553726 74.73113629324624) bank132199 +132200 POINT(41.01344722994274 74.31277067577928) bank132200 +132201 POINT(41.35311658005614 73.86375018947622) bank132201 +132202 POINT(40.29447705877324 73.45258008803616) bank132202 +132203 POINT(40.36693346309913 74.27772833321248) bank132203 +132204 POINT(40.347892823353924 73.7512395173848) bank132204 +132205 POINT(40.05151264520002 73.90109825154684) bank132205 +132206 POINT(41.3610640889796 74.78465463705578) bank132206 +132207 POINT(40.66259422627586 74.268311798967) bank132207 +132208 POINT(40.07407478082864 74.09350526354723) bank132208 +132209 POINT(41.23675084426319 73.06244706758801) bank132209 +132210 POINT(40.3931747836936 73.05791541569178) bank132210 +132211 POINT(41.20575022742052 73.20440270585398) bank132211 +132212 POINT(41.085678312175496 74.67424309635608) bank132212 +132213 POINT(41.056643803647354 73.62279417173336) bank132213 +132214 POINT(41.653277706429144 73.51090200412656) bank132214 +132215 POINT(40.325586410057774 73.4273065680519) bank132215 +132216 POINT(41.47416984447492 73.9669374627492) bank132216 +132217 POINT(41.356131336492176 73.12372374395596) bank132217 +132218 POINT(39.878559617302706 73.90998498698264) bank132218 +132219 POINT(40.67799359189148 74.50084174022581) bank132219 +132220 POINT(40.71676289702808 74.19488719167366) bank132220 +132221 POINT(39.978031947361146 74.317956085586) bank132221 +132222 POINT(41.21052599705365 74.1777574396046) bank132222 +132223 POINT(40.06675102896121 73.88069096849337) bank132223 +132224 POINT(41.28849557459168 74.06443073720376) bank132224 +132225 POINT(40.804501196167564 74.17928245874955) bank132225 +132226 POINT(40.996807441134415 73.37003834264253) bank132226 +132227 POINT(41.01984606343729 74.34778920547754) bank132227 +132228 POINT(40.26462723082582 73.59245062392388) bank132228 +132229 POINT(39.87258974129262 74.25778161566998) bank132229 +132230 POINT(40.01359008579174 74.0043160529003) bank132230 +132231 POINT(40.426266199347666 73.40052823912423) bank132231 +132232 POINT(41.49050858064396 73.26219398937971) bank132232 +132233 POINT(40.38714759092061 73.181455702136) bank132233 +132234 POINT(41.591262071598244 73.32631761582454) bank132234 +132235 POINT(40.2355516435558 74.02375495221195) bank132235 +132236 POINT(40.283398669168434 74.47454454417702) bank132236 +132237 POINT(41.6380279183471 73.38993693463668) bank132237 +132238 POINT(41.33602905294489 73.96590131788376) bank132238 +132239 POINT(40.53966929478705 73.72599148459531) bank132239 +132240 POINT(41.32132784360214 74.24465929035053) bank132240 +132241 POINT(40.417880938143234 73.2045740820554) bank132241 +132242 POINT(40.71653055705219 73.42197561980862) bank132242 +132243 POINT(40.62449547317367 74.56709557772133) bank132243 +132244 POINT(41.298046509806255 73.58489973058906) bank132244 +132245 POINT(40.28667966776909 73.88077089028076) bank132245 +132246 POINT(40.613765413697145 74.75560205310285) bank132246 +132247 POINT(40.6843660068165 73.84023655476469) bank132247 +132248 POINT(40.11270708409438 73.46694594743738) bank132248 +132249 POINT(40.744140287566346 73.84089828513872) bank132249 +132250 POINT(40.73287208764563 73.22049039150212) bank132250 +132251 POINT(41.30928694269866 74.65940401514388) bank132251 +132252 POINT(41.15490501657802 73.96656109673711) bank132252 +132253 POINT(39.8690163526687 73.97474191444338) bank132253 +132254 POINT(40.294815287406855 74.98304941725308) bank132254 +132255 POINT(41.52548952701505 74.62400398653305) bank132255 +132256 POINT(41.26123014695429 73.13089225892921) bank132256 +132257 POINT(40.121434724258975 74.53556350515294) bank132257 +132258 POINT(40.72748234702967 74.25453041723117) bank132258 +132259 POINT(41.1827783559809 73.39724628515823) bank132259 +132260 POINT(40.3784314214175 74.89312235921528) bank132260 +132261 POINT(40.74261966184612 73.40270146440916) bank132261 +132262 POINT(39.93696649018 73.60197567297674) bank132262 +132263 POINT(41.090725234320416 73.95920611291848) bank132263 +132264 POINT(40.778244361457205 73.39192942348063) bank132264 +132265 POINT(41.44620711430443 73.83583228000927) bank132265 +132266 POINT(40.83331706063932 73.65976061009786) bank132266 +132267 POINT(40.03787042469938 74.69677018801853) bank132267 +132268 POINT(40.062990272414375 74.46041087113922) bank132268 +132269 POINT(39.91460131390357 74.20552467387142) bank132269 +132270 POINT(40.69264128068717 73.68131535600678) bank132270 +132271 POINT(41.134186922775854 74.45377135688115) bank132271 +132272 POINT(40.30410302030293 74.27207050781269) bank132272 +132273 POINT(41.021474927890594 73.30205286063855) bank132273 +132274 POINT(41.24821081673424 74.5460260772021) bank132274 +132275 POINT(40.924894383786054 73.8455026755036) bank132275 +132276 POINT(39.840225335642316 74.65174931175302) bank132276 +132277 POINT(40.74898071001074 74.86303034845834) bank132277 +132278 POINT(41.369773339770326 74.40504690324283) bank132278 +132279 POINT(41.52813689300278 73.70445848819841) bank132279 +132280 POINT(40.710892732034125 74.12500240822008) bank132280 +132281 POINT(41.62474792349734 73.52080863396394) bank132281 +132282 POINT(41.00904058243005 73.17045768217557) bank132282 +132283 POINT(41.08021560919358 73.14173487573743) bank132283 +132284 POINT(40.35713409428146 74.94397115014445) bank132284 +132285 POINT(40.1759394900571 74.32292298109049) bank132285 +132286 POINT(40.62370053172804 74.4410546049527) bank132286 +132287 POINT(41.62962053536459 73.40817823137765) bank132287 +132288 POINT(40.98482787300685 74.14229131396074) bank132288 +132289 POINT(39.95681565658923 73.91412395674243) bank132289 +132290 POINT(40.575571606585996 73.98692554396405) bank132290 +132291 POINT(39.835286556225554 74.24052833309263) bank132291 +132292 POINT(41.571045593690826 74.76935733801929) bank132292 +132293 POINT(40.44561925286365 74.04899942183907) bank132293 +132294 POINT(41.03608410202642 74.03729341326991) bank132294 +132295 POINT(40.47547517155988 74.10634922367336) bank132295 +132296 POINT(41.594513050382616 73.62556650644477) bank132296 +132297 POINT(41.36596449860978 74.84805596155111) bank132297 +132298 POINT(41.058396853678616 73.77350220580801) bank132298 +132299 POINT(40.203565476484876 74.397993247314) bank132299 +132300 POINT(41.701567429363024 74.57297218253885) bank132300 +132301 POINT(39.90855937885865 73.81272484995577) bank132301 +132302 POINT(40.916810368089564 73.6239624217891) bank132302 +132303 POINT(41.68957029819158 74.87337504853662) bank132303 +132304 POINT(40.09990760992351 73.46013392617989) bank132304 +132305 POINT(40.95756089680719 73.26951294541176) bank132305 +132306 POINT(40.319563739749164 74.80702963484752) bank132306 +132307 POINT(41.63085810720936 74.0491598806325) bank132307 +132308 POINT(41.64703048115111 73.332177643772) bank132308 +132309 POINT(41.222415602430296 74.28370565919424) bank132309 +132310 POINT(39.821907332774316 73.70479822661038) bank132310 +132311 POINT(40.66937167014364 73.65029986050669) bank132311 +132312 POINT(39.86467832702319 73.15532415083491) bank132312 +132313 POINT(41.14131222370066 74.60813804416404) bank132313 +132314 POINT(41.21571774234084 74.48222692848631) bank132314 +132315 POINT(39.84142919129029 73.85326976050293) bank132315 +132316 POINT(41.69058414168602 74.5094516578159) bank132316 +132317 POINT(40.5243724209029 73.93835926007671) bank132317 +132318 POINT(40.55968250935094 74.0469059010674) bank132318 +132319 POINT(39.82237588215685 74.01264906296997) bank132319 +132320 POINT(39.81730600194279 74.95948066680626) bank132320 +132321 POINT(41.57384597298951 73.4322866115621) bank132321 +132322 POINT(40.503072763210554 73.21657800625246) bank132322 +132323 POINT(40.83067817249811 73.76766929904034) bank132323 +132324 POINT(40.52065911070577 74.41547111711915) bank132324 +132325 POINT(39.79601658905704 74.1348143608998) bank132325 +132326 POINT(41.305091165475424 74.76830016219112) bank132326 +132327 POINT(41.168330097256856 73.45636439651152) bank132327 +132328 POINT(41.29536976860197 73.25575781349802) bank132328 +132329 POINT(40.28089740212873 74.58040380385096) bank132329 +132330 POINT(40.09576036780387 74.36873167024623) bank132330 +132331 POINT(40.05290303629253 73.18078581888741) bank132331 +132332 POINT(40.17064602332623 73.00603982037542) bank132332 +132333 POINT(39.90081618052743 73.06661986236702) bank132333 +132334 POINT(40.93529370751761 75.0052148389074) bank132334 +132335 POINT(40.621085497567684 73.05528064527981) bank132335 +132336 POINT(39.85285368526893 73.63398270892354) bank132336 +132337 POINT(40.688531058851844 73.19443858650997) bank132337 +132338 POINT(41.46271059137553 74.84914415566809) bank132338 +132339 POINT(40.63450659235636 74.21761311145065) bank132339 +132340 POINT(40.824411261804855 74.50802142358877) bank132340 +132341 POINT(39.85285319880025 73.84366907979957) bank132341 +132342 POINT(40.896862852910864 73.21916702349365) bank132342 +132343 POINT(40.1814884993794 73.6401623851169) bank132343 +132344 POINT(40.1262911375787 73.95663060832779) bank132344 +132345 POINT(41.643393556014544 73.31569569342507) bank132345 +132346 POINT(41.32629336975718 74.8880931592467) bank132346 +132347 POINT(41.052270794486915 73.89572652096719) bank132347 +132348 POINT(40.69305452014478 73.69755944102374) bank132348 +132349 POINT(40.41282158051457 73.22217311907472) bank132349 +132350 POINT(40.048154299506855 74.59383123790664) bank132350 +132351 POINT(40.37276207556082 74.36290442575152) bank132351 +132352 POINT(40.99810130114829 74.79692956731982) bank132352 +132353 POINT(39.81054251599125 73.17445194065245) bank132353 +132354 POINT(40.16893832253596 74.05223129839328) bank132354 +132355 POINT(41.61198626995308 73.18393692602521) bank132355 +132356 POINT(40.1307110163581 74.82434153459593) bank132356 +132357 POINT(40.78967580345572 73.83014939266062) bank132357 +132358 POINT(41.63681183565346 73.91911475292699) bank132358 +132359 POINT(41.040442217188534 74.12203173335097) bank132359 +132360 POINT(40.76072139955686 74.1702034542621) bank132360 +132361 POINT(40.159575475602786 73.58151953906928) bank132361 +132362 POINT(41.36528349595967 73.83511133042) bank132362 +132363 POINT(41.08750354052165 73.46477089606468) bank132363 +132364 POINT(40.16718495013479 73.76820670613166) bank132364 +132365 POINT(40.732215417049154 74.40308367751126) bank132365 +132366 POINT(41.51726154108585 74.23898547394599) bank132366 +132367 POINT(40.70568799316166 74.2024685168034) bank132367 +132368 POINT(40.70206687737386 73.47394883083527) bank132368 +132369 POINT(40.55779435471235 74.34763588376659) bank132369 +132370 POINT(41.59866152006002 74.09431652947193) bank132370 +132371 POINT(39.867091596155326 73.47120634240827) bank132371 +132372 POINT(40.07016925136934 73.82351901076005) bank132372 +132373 POINT(40.20191705925859 74.39428283496308) bank132373 +132374 POINT(40.18152356983043 73.17973532399591) bank132374 +132375 POINT(40.69123162413169 74.3359459511275) bank132375 +132376 POINT(41.61499714786365 73.3539710430218) bank132376 +132377 POINT(40.384765425238875 73.27720394871595) bank132377 +132378 POINT(41.10500057766093 73.69716143415727) bank132378 +132379 POINT(40.25828517037713 74.61559937960102) bank132379 +132380 POINT(41.00678938445509 73.5310989898614) bank132380 +132381 POINT(41.323397095302 74.63876919101462) bank132381 +132382 POINT(40.18461634571753 73.47361690879694) bank132382 +132383 POINT(40.20310864411467 74.84332689609504) bank132383 +132384 POINT(41.03081619936365 74.60952672082028) bank132384 +132385 POINT(40.72192617062193 73.43055679518983) bank132385 +132386 POINT(40.31879233636209 74.98744269307916) bank132386 +132387 POINT(39.97932784247745 73.6898899051803) bank132387 +132388 POINT(41.25539725562108 74.00356583915813) bank132388 +132389 POINT(40.41154939915285 74.74356467697017) bank132389 +132390 POINT(40.31398871976067 73.50824526466522) bank132390 +132391 POINT(41.47212937792484 74.76983362459983) bank132391 +132392 POINT(40.43138488540725 74.39578515036095) bank132392 +132393 POINT(41.531793268100714 73.09084836847282) bank132393 +132394 POINT(41.28457447348257 74.64034668506454) bank132394 +132395 POINT(40.658692707968996 73.8040564557887) bank132395 +132396 POINT(40.630884648737045 74.55035051350177) bank132396 +132397 POINT(40.610875225926904 74.31224677253087) bank132397 +132398 POINT(40.605075815879545 73.24135694297894) bank132398 +132399 POINT(41.126317752305674 73.49648182455977) bank132399 +132400 POINT(40.51651447431195 73.5232124002771) bank132400 +132401 POINT(39.98660967473394 74.1310001096508) bank132401 +132402 POINT(39.95090542426884 74.81148809662226) bank132402 +132403 POINT(40.11330631719147 74.07346824625533) bank132403 +132404 POINT(40.2375075184101 73.71048557879824) bank132404 +132405 POINT(40.82439354461098 73.96548679625381) bank132405 +132406 POINT(40.644099250343466 74.19972145993911) bank132406 +132407 POINT(41.192852001887914 73.23298232432516) bank132407 +132408 POINT(40.79037106478103 73.67356843099459) bank132408 +132409 POINT(40.91346561325073 74.07883863367374) bank132409 +132410 POINT(41.3326609139252 74.39373353101624) bank132410 +132411 POINT(39.79610032823112 74.19445613639841) bank132411 +132412 POINT(40.08591121593122 74.99816568816081) bank132412 +132413 POINT(40.83869193980876 73.12463686772432) bank132413 +132414 POINT(40.61177745379376 73.37362124194222) bank132414 +132415 POINT(41.359918548015294 74.41360015966346) bank132415 +132416 POINT(41.153870326324274 73.9151389868826) bank132416 +132417 POINT(41.4839070651181 73.01325687785963) bank132417 +132418 POINT(40.56143006393346 74.7304518783635) bank132418 +132419 POINT(40.77286288933807 74.24603778334526) bank132419 +132420 POINT(41.11900020227835 73.26047864185341) bank132420 +132421 POINT(39.80959328784061 74.86081098424548) bank132421 +132422 POINT(41.67673604884448 74.60499596964799) bank132422 +132423 POINT(40.19953004158644 74.70181727699782) bank132423 +132424 POINT(40.87976500005037 73.62291827161101) bank132424 +132425 POINT(39.961547251805456 74.03855103127187) bank132425 +132426 POINT(40.19353971541798 73.79222285146908) bank132426 +132427 POINT(40.92776084382205 74.68375123372768) bank132427 +132428 POINT(40.2510203710349 74.5215022307634) bank132428 +132429 POINT(40.10951343330793 74.09386151265721) bank132429 +132430 POINT(40.97928379977648 73.06297433391474) bank132430 +132431 POINT(40.32574252273945 74.43109659201056) bank132431 +132432 POINT(40.378533654459694 74.56509358480862) bank132432 +132433 POINT(40.12950234512702 74.28826044986724) bank132433 +132434 POINT(40.804229658766616 73.01529712927892) bank132434 +132435 POINT(40.73379479528682 73.17849337223369) bank132435 +132436 POINT(41.49528576396905 74.11681439952996) bank132436 +132437 POINT(39.759913581628865 73.46078012194964) bank132437 +132438 POINT(39.98620813780164 73.0749703107027) bank132438 +132439 POINT(40.101246320966965 74.35735646415856) bank132439 +132440 POINT(39.909617477214894 74.88066118082111) bank132440 +132441 POINT(41.38201309552371 74.92679135163998) bank132441 +132442 POINT(39.79011201302894 73.21177366462008) bank132442 +132443 POINT(40.8277535944001 73.31338746447246) bank132443 +132444 POINT(41.449207318969755 73.35322366725742) bank132444 +132445 POINT(40.9681563457215 73.15223784630601) bank132445 +132446 POINT(39.87508070733833 73.79058619419028) bank132446 +132447 POINT(41.421926162833465 74.82455680830378) bank132447 +132448 POINT(41.14892614715726 73.34661252170345) bank132448 +132449 POINT(39.931066855702696 73.6907367168818) bank132449 +132450 POINT(39.83369113538156 73.27293840695542) bank132450 +132451 POINT(41.5313255724572 74.53282651043135) bank132451 +132452 POINT(40.50754330902509 73.36475946967015) bank132452 +132453 POINT(41.11931350071921 74.8367921089715) bank132453 +132454 POINT(41.667132829974506 74.51027981184733) bank132454 +132455 POINT(40.09445904970534 73.17239411622289) bank132455 +132456 POINT(41.51768814064074 73.6190892410969) bank132456 +132457 POINT(40.07060571950801 73.11787004930278) bank132457 +132458 POINT(41.4781388719668 74.25522761252705) bank132458 +132459 POINT(41.26865604863266 74.69712032400174) bank132459 +132460 POINT(40.24677570433005 73.4624890169967) bank132460 +132461 POINT(40.01808194545372 73.85582664182128) bank132461 +132462 POINT(39.95224057436905 74.72660652053447) bank132462 +132463 POINT(41.60788670908031 73.82690599530176) bank132463 +132464 POINT(41.241990155569525 74.36207444322245) bank132464 +132465 POINT(40.803820695694014 73.79802176985531) bank132465 +132466 POINT(39.77080346268663 73.78964474606175) bank132466 +132467 POINT(41.27095937927571 74.57831414026803) bank132467 +132468 POINT(40.70810274492378 73.88925521867988) bank132468 +132469 POINT(40.422521277327014 73.01490304797086) bank132469 +132470 POINT(40.44947450313744 73.57759828326039) bank132470 +132471 POINT(40.04676052589811 73.18065709802941) bank132471 +132472 POINT(40.24100481206898 73.1105538034344) bank132472 +132473 POINT(40.477880964972414 73.25064017166972) bank132473 +132474 POINT(40.195578816936205 74.53993580402721) bank132474 +132475 POINT(41.32224474626515 73.61552428180734) bank132475 +132476 POINT(40.30444388102655 74.50194303917046) bank132476 +132477 POINT(41.48031444684185 74.47521296547616) bank132477 +132478 POINT(40.433569146106386 73.44045788300801) bank132478 +132479 POINT(39.97456894606296 73.5612324121202) bank132479 +132480 POINT(39.99662537412043 73.62936334193904) bank132480 +132481 POINT(40.38153537607695 74.22460287664708) bank132481 +132482 POINT(41.58205421400144 73.0230383517782) bank132482 +132483 POINT(40.25485930632174 73.03729376108777) bank132483 +132484 POINT(41.52437992039653 73.34454066293637) bank132484 +132485 POINT(41.229966766220926 73.28450952332523) bank132485 +132486 POINT(40.8510872584149 73.01979396295842) bank132486 +132487 POINT(41.19415976904459 74.63053056378702) bank132487 +132488 POINT(41.056196862672735 73.75394061488262) bank132488 +132489 POINT(41.338664732941645 73.92571258778707) bank132489 +132490 POINT(40.80394527933796 73.15527476141702) bank132490 +132491 POINT(39.98639657680033 73.65056199722737) bank132491 +132492 POINT(41.22944757323481 74.388793712062) bank132492 +132493 POINT(39.92000466207903 74.27107433220911) bank132493 +132494 POINT(40.907543369022164 73.66170591499412) bank132494 +132495 POINT(41.61550488265338 74.77936020632293) bank132495 +132496 POINT(39.75789877822107 73.40224339407882) bank132496 +132497 POINT(41.583708468671794 74.01162560334339) bank132497 +132498 POINT(40.34601975389868 74.97098435710085) bank132498 +132499 POINT(40.511941250229086 73.6945816927574) bank132499 +132500 POINT(39.887492425304146 73.76410993519241) bank132500 +132501 POINT(41.62070318324681 74.80467328644879) bank132501 +132502 POINT(41.708804270269546 73.53068839301714) bank132502 +132503 POINT(40.09989511793665 74.95535485580737) bank132503 +132504 POINT(40.987122287822416 73.46558002422182) bank132504 +132505 POINT(40.021869323789225 74.23260834471726) bank132505 +132506 POINT(41.42592715678868 73.37180733989703) bank132506 +132507 POINT(40.56266007558145 73.59040327482994) bank132507 +132508 POINT(41.1005387398811 73.90001742255302) bank132508 +132509 POINT(40.817415009498305 74.57833484227815) bank132509 +132510 POINT(40.88187425225246 74.14959616566287) bank132510 +132511 POINT(40.749802648791864 73.54541254922053) bank132511 +132512 POINT(40.83470479424321 74.81203856465702) bank132512 +132513 POINT(41.40079867364381 73.5455211479803) bank132513 +132514 POINT(41.10674540957972 74.89703667335249) bank132514 +132515 POINT(40.239962434058484 74.26321512855591) bank132515 +132516 POINT(40.20800489602613 73.43717368240942) bank132516 +132517 POINT(41.21787164140707 74.9222733006288) bank132517 +132518 POINT(39.93910242117617 73.45574439809293) bank132518 +132519 POINT(41.31273869725966 73.64209381450532) bank132519 +132520 POINT(39.745659442749 74.39685311908156) bank132520 +132521 POINT(39.85985373529805 73.28200243275113) bank132521 +132522 POINT(41.65752046945503 74.19415959436563) bank132522 +132523 POINT(40.58375428419181 74.85073647774564) bank132523 +132524 POINT(40.96276836864056 74.69895277011885) bank132524 +132525 POINT(41.17857668875157 74.72073989548468) bank132525 +132526 POINT(40.69476534124879 73.16979193486611) bank132526 +132527 POINT(41.64197894019245 74.80137657520554) bank132527 +132528 POINT(40.840858259529284 73.34411300033966) bank132528 +132529 POINT(40.48958598914315 74.00821560966574) bank132529 +132530 POINT(40.17239735580023 73.39707530109136) bank132530 +132531 POINT(41.65541976849691 73.73607505726783) bank132531 +132532 POINT(40.616172090682184 73.27202248638186) bank132532 +132533 POINT(40.69472545214893 74.9310414589665) bank132533 +132534 POINT(39.81360301664316 74.50364622294744) bank132534 +132535 POINT(41.263993725358254 73.32544064402936) bank132535 +132536 POINT(41.683880580105885 74.48180639218124) bank132536 +132537 POINT(40.896762910343554 74.90086308514476) bank132537 +132538 POINT(39.77828633049482 74.65795777887189) bank132538 +132539 POINT(40.154676854539524 74.49941916431646) bank132539 +132540 POINT(40.564042156815326 74.08066781295126) bank132540 +132541 POINT(39.89022863440207 74.76179421256529) bank132541 +132542 POINT(40.552281575811236 74.86524656841965) bank132542 +132543 POINT(40.195576277815164 74.43668033902927) bank132543 +132544 POINT(40.596268719659975 74.92149929590857) bank132544 +132545 POINT(41.30049503568858 74.24195060742335) bank132545 +132546 POINT(40.680346908886584 73.14753236165444) bank132546 +132547 POINT(41.00414321598806 73.34257340902087) bank132547 +132548 POINT(40.8240227314063 73.0147395449951) bank132548 +132549 POINT(41.0898613212432 74.28627399056238) bank132549 +132550 POINT(41.4594469210798 73.85505035116815) bank132550 +132551 POINT(41.06789279371902 73.04112111631844) bank132551 +132552 POINT(41.6067563346339 74.74670397513368) bank132552 +132553 POINT(40.36167657147778 73.58367751839283) bank132553 +132554 POINT(40.95815656221922 74.10696753449642) bank132554 +132555 POINT(41.45624727728513 74.94147453688565) bank132555 +132556 POINT(41.0535632312509 73.5138378182319) bank132556 +132557 POINT(40.455948400938844 74.35849853832103) bank132557 +132558 POINT(40.62313835815026 73.26354283088352) bank132558 +132559 POINT(40.89168759885559 74.12843486902386) bank132559 +132560 POINT(40.10294470670842 73.37169016670072) bank132560 +132561 POINT(40.29639983127671 73.59508650311729) bank132561 +132562 POINT(40.06205045125896 73.90064072331488) bank132562 +132563 POINT(41.32766599856383 73.51150580067025) bank132563 +132564 POINT(40.28253772894508 73.18236737583858) bank132564 +132565 POINT(39.99556195215731 74.42485483548829) bank132565 +132566 POINT(39.74730695299991 74.83050013583583) bank132566 +132567 POINT(40.26750128229922 73.80957523354192) bank132567 +132568 POINT(40.7127923914215 73.07140424037985) bank132568 +132569 POINT(40.31035150609193 73.25178045158889) bank132569 +132570 POINT(39.9469699275386 74.07312875481185) bank132570 +132571 POINT(41.12381495636066 73.40828827107907) bank132571 +132572 POINT(41.07886593487262 74.24507592173966) bank132572 +132573 POINT(41.32450838639279 74.35080932250116) bank132573 +132574 POINT(41.67599200488859 74.03637206660022) bank132574 +132575 POINT(40.05145748620619 74.76157101610089) bank132575 +132576 POINT(40.82354433707852 74.51766444111962) bank132576 +132577 POINT(41.37870877400604 73.66489581957595) bank132577 +132578 POINT(40.87517018283447 73.55291336652428) bank132578 +132579 POINT(41.21878925744479 73.3794094589011) bank132579 +132580 POINT(41.33134692307359 74.11734312263026) bank132580 +132581 POINT(40.46841950355066 74.72656287608257) bank132581 +132582 POINT(41.181222316204895 74.49543351202931) bank132582 +132583 POINT(41.5791177042384 73.38039386124576) bank132583 +132584 POINT(40.03088639390603 73.10506880776124) bank132584 +132585 POINT(41.022472936975106 73.23959481056728) bank132585 +132586 POINT(40.87946747037083 74.35736058465541) bank132586 +132587 POINT(40.4966560464073 73.74531815487916) bank132587 +132588 POINT(40.96334033009875 73.81418445958892) bank132588 +132589 POINT(41.05520248703806 74.50815136587322) bank132589 +132590 POINT(41.536308255983776 74.25910600688007) bank132590 +132591 POINT(40.41242649462267 73.04231001044707) bank132591 +132592 POINT(40.90815014731273 74.85975148145623) bank132592 +132593 POINT(40.28522951125159 73.76403906555963) bank132593 +132594 POINT(40.75436575394768 74.04623457832024) bank132594 +132595 POINT(41.11456711628714 73.10582257364868) bank132595 +132596 POINT(39.939790510414234 74.67947041160049) bank132596 +132597 POINT(40.24751542400181 73.65220670145624) bank132597 +132598 POINT(41.142085918634926 74.80113108465679) bank132598 +132599 POINT(41.452252985237806 73.62186483512146) bank132599 +132600 POINT(40.18112181697521 74.33201313468842) bank132600 +132601 POINT(39.735151230616886 74.18658692906736) bank132601 +132602 POINT(41.56667626007089 74.35104466113407) bank132602 +132603 POINT(40.884347333009266 74.17809838079818) bank132603 +132604 POINT(40.186207290396496 73.43016045984378) bank132604 +132605 POINT(41.26951472564224 73.98481037702851) bank132605 +132606 POINT(41.17980266259498 74.22369172048818) bank132606 +132607 POINT(40.03584514106122 73.24706707447076) bank132607 +132608 POINT(40.53265066595887 73.72172207318133) bank132608 +132609 POINT(41.515913448408185 73.9927003493883) bank132609 +132610 POINT(40.16667885179999 74.82661679823002) bank132610 +132611 POINT(40.1494163946573 73.06880407727171) bank132611 +132612 POINT(40.00184882710016 73.06771379565477) bank132612 +132613 POINT(41.392592602082914 73.08911533537916) bank132613 +132614 POINT(40.67461334330243 73.72275838164526) bank132614 +132615 POINT(41.08998652642774 73.36875618822671) bank132615 +132616 POINT(40.08321321838532 74.62992053135628) bank132616 +132617 POINT(41.257061745939616 73.7486092110071) bank132617 +132618 POINT(40.3875592602845 73.43631306478483) bank132618 +132619 POINT(40.064612347385655 73.61519565527128) bank132619 +132620 POINT(40.43194059326403 74.3086834803648) bank132620 +132621 POINT(40.2792007368776 73.61993089306003) bank132621 +132622 POINT(39.835823785494846 74.43476117435348) bank132622 +132623 POINT(41.5351503086042 73.2941152218733) bank132623 +132624 POINT(40.45774604511744 74.71182114276485) bank132624 +132625 POINT(40.839225600959494 73.31409401915917) bank132625 +132626 POINT(41.28217607375017 74.00071651151438) bank132626 +132627 POINT(40.511583353294064 74.52126239744074) bank132627 +132628 POINT(40.91662312219266 74.28532236333159) bank132628 +132629 POINT(41.18065155301959 74.81798324430449) bank132629 +132630 POINT(41.23902414453532 74.7681633139882) bank132630 +132631 POINT(41.461794123596505 73.96983372091336) bank132631 +132632 POINT(40.915176796740084 74.60159439109889) bank132632 +132633 POINT(41.308566927179136 73.56408498692895) bank132633 +132634 POINT(40.90093133582419 73.70355721309004) bank132634 +132635 POINT(40.99922088285698 74.61711911024301) bank132635 +132636 POINT(39.83229978928701 73.14220064899952) bank132636 +132637 POINT(40.4910370490587 73.29137166722015) bank132637 +132638 POINT(40.02195423666564 74.37168090234312) bank132638 +132639 POINT(40.83617117088633 74.62409818913743) bank132639 +132640 POINT(40.58416720893919 74.82520645053604) bank132640 +132641 POINT(41.439300372271866 74.44132909778888) bank132641 +132642 POINT(40.15794372163911 73.50856996104363) bank132642 +132643 POINT(40.32567501694513 73.83065021858826) bank132643 +132644 POINT(40.77970798925957 73.32164494463598) bank132644 +132645 POINT(41.26267949222926 74.50582360832053) bank132645 +132646 POINT(40.973065441752524 74.97066056060305) bank132646 +132647 POINT(40.644550060695494 73.29374333931186) bank132647 +132648 POINT(40.976888495432526 73.83450149071439) bank132648 +132649 POINT(41.149283149962805 73.39464867315205) bank132649 +132650 POINT(39.873165105195945 74.62443877386526) bank132650 +132651 POINT(41.0911981813048 73.62719531077724) bank132651 +132652 POINT(40.991662478797245 74.0843417691625) bank132652 +132653 POINT(41.26190844287165 73.17708168813529) bank132653 +132654 POINT(39.7624689239628 74.24720297462089) bank132654 +132655 POINT(40.64016799395831 74.49457485403575) bank132655 +132656 POINT(40.169050564979486 74.56080231497651) bank132656 +132657 POINT(40.04093004209302 73.62936405653555) bank132657 +132658 POINT(40.88637805893934 73.73866651555959) bank132658 +132659 POINT(40.95514370127739 73.45356948286594) bank132659 +132660 POINT(41.45696394325101 73.11494788043706) bank132660 +132661 POINT(41.69010556185785 74.19368084022318) bank132661 +132662 POINT(40.19961428548034 74.50605980714325) bank132662 +132663 POINT(40.01910430895668 73.4177028022501) bank132663 +132664 POINT(41.69312693723184 73.56564591459328) bank132664 +132665 POINT(40.25726366329578 74.58634884115669) bank132665 +132666 POINT(40.481354357298855 74.72288483306289) bank132666 +132667 POINT(41.52878071460254 73.17318259382822) bank132667 +132668 POINT(40.21521076095561 73.13659041204723) bank132668 +132669 POINT(41.28589197799769 74.35595981511395) bank132669 +132670 POINT(41.23785799566855 74.64049983968233) bank132670 +132671 POINT(41.46771026530964 73.42402177836301) bank132671 +132672 POINT(41.20029328227875 73.41119826023603) bank132672 +132673 POINT(41.70916088797894 74.19216769967805) bank132673 +132674 POINT(39.96794209523864 73.32576038296958) bank132674 +132675 POINT(40.29153337028882 74.05278558782572) bank132675 +132676 POINT(40.92998770836765 73.60154390380248) bank132676 +132677 POINT(39.750277202926895 73.17440670795267) bank132677 +132678 POINT(41.48287366995756 74.59154147464044) bank132678 +132679 POINT(40.109958982376014 74.07479439625887) bank132679 +132680 POINT(41.14874686204051 73.31420592025724) bank132680 +132681 POINT(40.95849854377596 74.34783807247372) bank132681 +132682 POINT(41.536688806206314 74.21550522204063) bank132682 +132683 POINT(40.41601729280268 73.56731908550394) bank132683 +132684 POINT(41.633268228271355 74.49491993533346) bank132684 +132685 POINT(41.29064391217203 73.10101857152799) bank132685 +132686 POINT(40.32176612976849 74.8623453716788) bank132686 +132687 POINT(40.027838649509896 74.01896198402046) bank132687 +132688 POINT(40.47502407519016 73.54063251959771) bank132688 +132689 POINT(39.74838460908839 73.17632158570213) bank132689 +132690 POINT(40.557914225180404 74.7191394152018) bank132690 +132691 POINT(41.00137749637788 74.27061990512463) bank132691 +132692 POINT(40.75876464646269 73.18302876601099) bank132692 +132693 POINT(40.15549733252399 74.23213846499668) bank132693 +132694 POINT(39.838277181882006 73.46982678193534) bank132694 +132695 POINT(40.05269510410296 74.55381298579154) bank132695 +132696 POINT(41.4173159505617 74.61089401812451) bank132696 +132697 POINT(39.81033620489037 74.80315395979525) bank132697 +132698 POINT(40.726757903372004 74.07462384420222) bank132698 +132699 POINT(40.54500858084138 74.82146589145287) bank132699 +132700 POINT(39.83427697615207 74.72187829780806) bank132700 +132701 POINT(39.99774497082674 74.24649392730586) bank132701 +132702 POINT(40.48813518738357 74.25685105830233) bank132702 +132703 POINT(41.28258526091356 73.42758427759838) bank132703 +132704 POINT(40.907289044904275 74.30599766382294) bank132704 +132705 POINT(41.53789573299846 73.95537562671566) bank132705 +132706 POINT(41.151738841029456 73.0394025575308) bank132706 +132707 POINT(41.51128335729191 73.18480287166267) bank132707 +132708 POINT(41.71257831959022 74.93919151649789) bank132708 +132709 POINT(41.24447967397589 74.16833674830697) bank132709 +132710 POINT(41.26429916615604 73.54521762432452) bank132710 +132711 POINT(41.203944157979244 73.90745097155103) bank132711 +132712 POINT(41.177203334289985 74.79216908788621) bank132712 +132713 POINT(40.26949030586047 73.46687550513262) bank132713 +132714 POINT(40.56772463002759 74.717440531434) bank132714 +132715 POINT(39.74196564353238 73.06354442378714) bank132715 +132716 POINT(41.11670974291342 73.46052369454928) bank132716 +132717 POINT(41.52553946416335 73.87541176465501) bank132717 +132718 POINT(41.10772615324455 73.36444011221955) bank132718 +132719 POINT(39.76127810395968 74.72213463852839) bank132719 +132720 POINT(39.73365929132396 74.83890705192766) bank132720 +132721 POINT(40.58994373529394 74.8109423608273) bank132721 +132722 POINT(39.833369366714656 74.63635615768175) bank132722 +132723 POINT(40.395333204284945 73.07573186403764) bank132723 +132724 POINT(40.46690550670617 74.62571749359637) bank132724 +132725 POINT(40.60626172313706 73.43220020537369) bank132725 +132726 POINT(40.079108863469955 73.20601971442983) bank132726 +132727 POINT(40.83465591282436 73.88367264059737) bank132727 +132728 POINT(40.67883673316352 74.86738413723049) bank132728 +132729 POINT(41.39408540709983 74.34974099810992) bank132729 +132730 POINT(40.76069188449275 73.42535592630766) bank132730 +132731 POINT(41.34435622115781 74.572298585497) bank132731 +132732 POINT(39.72692216501289 74.92072259948162) bank132732 +132733 POINT(39.8845858753996 73.76013726344033) bank132733 +132734 POINT(41.674896651164076 73.3675572974735) bank132734 +132735 POINT(41.583621477708405 74.18495054952247) bank132735 +132736 POINT(40.42947679726636 74.60994770202001) bank132736 +132737 POINT(40.68619158815799 73.08841344155768) bank132737 +132738 POINT(40.98901910208489 73.54834036980972) bank132738 +132739 POINT(40.367178749491124 74.82739941728967) bank132739 +132740 POINT(40.170576339392994 74.4657888083565) bank132740 +132741 POINT(40.345183756583786 74.27349824953964) bank132741 +132742 POINT(39.819064416391605 73.56164296551239) bank132742 +132743 POINT(41.01903392516391 74.22679227613337) bank132743 +132744 POINT(41.278901430929 73.89354570680551) bank132744 +132745 POINT(41.688844591514176 74.18938023754767) bank132745 +132746 POINT(40.15104575710768 74.63556960018708) bank132746 +132747 POINT(41.03401717328558 74.58714279243728) bank132747 +132748 POINT(40.04983734371702 74.05787255449587) bank132748 +132749 POINT(39.79605518515323 73.07321098985823) bank132749 +132750 POINT(40.00698391966516 74.92312536049752) bank132750 +132751 POINT(40.441638605735 74.09423344143819) bank132751 +132752 POINT(40.50137278702836 74.17568794623955) bank132752 +132753 POINT(41.22823521482525 73.88040299592542) bank132753 +132754 POINT(41.296691728130284 74.32005854730967) bank132754 +132755 POINT(40.75198647254143 74.39093797705269) bank132755 +132756 POINT(40.32693733444502 74.28029913498969) bank132756 +132757 POINT(40.41998556854447 74.28838761530882) bank132757 +132758 POINT(40.57181058041983 73.25700516295318) bank132758 +132759 POINT(40.98697689909735 73.4243654627156) bank132759 +132760 POINT(41.24356932837785 73.07823681781713) bank132760 +132761 POINT(40.72794983220026 74.84968670447475) bank132761 +132762 POINT(41.058256537479664 73.38589116576401) bank132762 +132763 POINT(40.54797307638948 73.83947555110335) bank132763 +132764 POINT(40.80236637309553 73.70332710013972) bank132764 +132765 POINT(41.235710802421075 74.54997391523277) bank132765 +132766 POINT(39.89226326120671 74.36642585522176) bank132766 +132767 POINT(40.992134492680854 74.97664489407491) bank132767 +132768 POINT(40.94812288468144 73.035329481981) bank132768 +132769 POINT(39.915332150511105 74.16640630701572) bank132769 +132770 POINT(41.230597810518255 74.3057833953447) bank132770 +132771 POINT(39.86181145650383 74.63960097828436) bank132771 +132772 POINT(39.932498752176876 73.28757583507281) bank132772 +132773 POINT(40.19509645879878 73.34429015467698) bank132773 +132774 POINT(41.46967441767268 74.8053354127191) bank132774 +132775 POINT(39.78797342663558 74.31953457568876) bank132775 +132776 POINT(40.054604393006514 74.99276195881059) bank132776 +132777 POINT(40.45377750151825 74.70199845348783) bank132777 +132778 POINT(40.04302977865411 73.90424074598421) bank132778 +132779 POINT(40.86695846410978 73.77014189830204) bank132779 +132780 POINT(39.8417744356384 73.86995818440772) bank132780 +132781 POINT(39.95881329047433 73.71072443050305) bank132781 +132782 POINT(40.759355741486715 73.74671394467141) bank132782 +132783 POINT(41.16230025350573 73.7770092408251) bank132783 +132784 POINT(40.99187727708355 74.80269022256235) bank132784 +132785 POINT(40.86066792337542 74.19095269308752) bank132785 +132786 POINT(40.93383005729339 74.4917563556239) bank132786 +132787 POINT(41.31369848857695 73.59442564892935) bank132787 +132788 POINT(41.52795724031767 74.14638730830004) bank132788 +132789 POINT(40.40962041315551 74.6058483615609) bank132789 +132790 POINT(40.93875945117094 73.81259977766463) bank132790 +132791 POINT(40.255957802345776 73.82126148409253) bank132791 +132792 POINT(40.599675234116816 74.42973460768117) bank132792 +132793 POINT(41.423847233710035 74.46530262744149) bank132793 +132794 POINT(41.624611358458 73.72565708914853) bank132794 +132795 POINT(40.595326484635095 73.4524262833448) bank132795 +132796 POINT(40.949573549161 74.54192170160145) bank132796 +132797 POINT(41.65502208690807 74.01627378805384) bank132797 +132798 POINT(41.65281587226 74.99465759049613) bank132798 +132799 POINT(41.53418854423901 74.68731174415059) bank132799 +132800 POINT(40.3200520886846 73.20124939084884) bank132800 +132801 POINT(41.17299416010665 73.89132714523652) bank132801 +132802 POINT(40.17927855409641 73.51849854537036) bank132802 +132803 POINT(39.959750431133116 74.23402364434584) bank132803 +132804 POINT(40.85656560230658 74.27741402074406) bank132804 +132805 POINT(41.03476776485695 73.23524135325658) bank132805 +132806 POINT(40.54756745982631 73.06497290705332) bank132806 +132807 POINT(41.10034524291671 74.92612500847711) bank132807 +132808 POINT(40.75822910531391 73.91546493761392) bank132808 +132809 POINT(40.9110484048396 73.34794856124505) bank132809 +132810 POINT(40.839074528114146 74.32737200148183) bank132810 +132811 POINT(41.07183811348919 73.33196490689971) bank132811 +132812 POINT(39.797538672832246 73.23140540677078) bank132812 +132813 POINT(40.3354104885542 74.85559896201703) bank132813 +132814 POINT(39.76828599055961 73.07411714018434) bank132814 +132815 POINT(39.89086695994342 74.17300457873606) bank132815 +132816 POINT(40.97647996531962 74.10135374686881) bank132816 +132817 POINT(40.70021186127694 73.74062775866578) bank132817 +132818 POINT(41.4471947004655 74.89261656897469) bank132818 +132819 POINT(41.38920593177046 74.05110884738394) bank132819 +132820 POINT(41.473259872577486 73.2385598501777) bank132820 +132821 POINT(40.17159385252706 73.31485497019698) bank132821 +132822 POINT(40.549901946724496 73.78935599527706) bank132822 +132823 POINT(40.79287432085707 73.53690787047165) bank132823 +132824 POINT(40.19543816157762 73.51311778685489) bank132824 +132825 POINT(40.94086731636336 74.49429395141628) bank132825 +132826 POINT(41.425041380000195 73.55033150830877) bank132826 +132827 POINT(41.23688166246163 73.01315071459747) bank132827 +132828 POINT(40.28543798367913 74.78289597190899) bank132828 +132829 POINT(39.91784189424683 73.50954763254285) bank132829 +132830 POINT(41.36163810431575 74.3345737170026) bank132830 +132831 POINT(41.00346384712333 74.95596891157662) bank132831 +132832 POINT(40.05241834473762 73.54363538349062) bank132832 +132833 POINT(41.340213363176176 73.16006030929272) bank132833 +132834 POINT(40.88120620636989 74.81619734457925) bank132834 +132835 POINT(40.90334481934741 74.18947935004186) bank132835 +132836 POINT(41.3022076727415 74.31932028486803) bank132836 +132837 POINT(41.00172431237844 74.70541588435947) bank132837 +132838 POINT(40.505265970366366 74.36391177678559) bank132838 +132839 POINT(41.539381746059725 73.3492143474686) bank132839 +132840 POINT(41.437791799392656 74.9432815864464) bank132840 +132841 POINT(41.333291559728444 74.55291267703142) bank132841 +132842 POINT(40.623072311863126 74.67037957234042) bank132842 +132843 POINT(41.572073192576994 74.39723299845521) bank132843 +132844 POINT(41.63464055075096 74.3118082702359) bank132844 +132845 POINT(40.12183888272208 74.20114328251259) bank132845 +132846 POINT(41.33340600944541 74.06149001146822) bank132846 +132847 POINT(41.569280129586524 74.80891580151156) bank132847 +132848 POINT(41.44691988740227 73.61702893868706) bank132848 +132849 POINT(41.25750459148675 74.63881591207821) bank132849 +132850 POINT(39.97512214667481 73.2467074523578) bank132850 +132851 POINT(41.20266727716954 74.63140654131686) bank132851 +132852 POINT(40.86518581610591 74.48120967498936) bank132852 +132853 POINT(40.2343774886683 74.26842981589924) bank132853 +132854 POINT(40.77734876775323 74.92540919661482) bank132854 +132855 POINT(41.043436226863896 74.13349818385473) bank132855 +132856 POINT(40.18341422109121 73.51152113893649) bank132856 +132857 POINT(40.49673347050914 73.83350843455126) bank132857 +132858 POINT(40.88630253571589 74.95383431034884) bank132858 +132859 POINT(41.4011748190527 74.81933767990722) bank132859 +132860 POINT(40.67499881742138 73.5428247888213) bank132860 +132861 POINT(40.50290931185702 74.65335130500509) bank132861 +132862 POINT(41.45533617529481 74.34163427747613) bank132862 +132863 POINT(40.60943554786257 74.15505414633972) bank132863 +132864 POINT(40.53194936039582 74.13932787943997) bank132864 +132865 POINT(41.187422759257466 73.28373346068979) bank132865 +132866 POINT(39.95739417471498 73.4069825358475) bank132866 +132867 POINT(41.23280988696361 73.37909547549704) bank132867 +132868 POINT(40.567148373377435 73.99502888706677) bank132868 +132869 POINT(40.904323390331825 74.76671277285462) bank132869 +132870 POINT(40.64194028878981 74.90133574791918) bank132870 +132871 POINT(40.92824607824443 73.39224176172591) bank132871 +132872 POINT(41.60101973514606 73.12814153899191) bank132872 +132873 POINT(40.45885415777629 74.76522256404103) bank132873 +132874 POINT(40.160125722654975 74.42465712291839) bank132874 +132875 POINT(40.89293689630161 73.20300221798976) bank132875 +132876 POINT(41.059006098560126 74.33576314274273) bank132876 +132877 POINT(41.543479421232014 73.89869646769421) bank132877 +132878 POINT(40.109524077689585 74.33099544403338) bank132878 +132879 POINT(39.7840947437426 73.29710068559018) bank132879 +132880 POINT(40.833251763626635 74.63891368417923) bank132880 +132881 POINT(41.25398248938934 74.66437425051242) bank132881 +132882 POINT(39.88081117483954 73.44834626589837) bank132882 +132883 POINT(40.78419419617484 73.58344127944122) bank132883 +132884 POINT(41.04600987160313 74.93334288061672) bank132884 +132885 POINT(41.67825043289447 73.07056478623096) bank132885 +132886 POINT(40.96408311114827 73.44669526722325) bank132886 +132887 POINT(40.22785187260401 73.06596282399869) bank132887 +132888 POINT(40.41013246403545 74.57993317801127) bank132888 +132889 POINT(39.71470482968408 73.37460927445542) bank132889 +132890 POINT(40.346436963660686 73.84566279033386) bank132890 +132891 POINT(41.69919016355834 73.4072991223683) bank132891 +132892 POINT(40.48881098471289 73.54429382195502) bank132892 +132893 POINT(40.18212953584854 73.31653093058912) bank132893 +132894 POINT(41.42612312369744 73.98761584634656) bank132894 +132895 POINT(40.00664578633598 73.48645825519905) bank132895 +132896 POINT(40.03643041623756 73.42689893927002) bank132896 +132897 POINT(40.403637880401924 73.81833082784816) bank132897 +132898 POINT(40.72009026765641 74.54600448313883) bank132898 +132899 POINT(40.40205748775568 73.3761065414631) bank132899 +132900 POINT(39.814060079075055 73.47036819625104) bank132900 +132901 POINT(39.75397922362972 73.41107864905084) bank132901 +132902 POINT(39.88370377753934 73.49092360022898) bank132902 +132903 POINT(40.23710824684492 73.31030849840063) bank132903 +132904 POINT(41.01361055875843 73.15771698747619) bank132904 +132905 POINT(40.15144299034524 73.9518129273637) bank132905 +132906 POINT(41.660141251128636 74.50915149311875) bank132906 +132907 POINT(41.22321574670796 73.06356398065253) bank132907 +132908 POINT(41.66460804810149 73.42773033261156) bank132908 +132909 POINT(40.0859544168069 74.63322583507446) bank132909 +132910 POINT(40.35640127942154 73.14710735173546) bank132910 +132911 POINT(39.78050178585732 74.63274328790656) bank132911 +132912 POINT(40.02994122889944 73.73304254972123) bank132912 +132913 POINT(40.25764664226392 73.58650796855169) bank132913 +132914 POINT(40.26244970598824 74.75878302196227) bank132914 +132915 POINT(40.0972805184804 73.48378744530662) bank132915 +132916 POINT(41.62513381117371 74.52852707570788) bank132916 +132917 POINT(40.94590132756893 73.5350287275442) bank132917 +132918 POINT(39.998734673914576 74.86480435651377) bank132918 +132919 POINT(39.868442556733015 74.52821524800594) bank132919 +132920 POINT(41.15279116972675 74.84554637133937) bank132920 +132921 POINT(41.16930299555622 73.32430904676494) bank132921 +132922 POINT(41.39816692198159 73.35057397746044) bank132922 +132923 POINT(41.57247952282173 74.07772211467444) bank132923 +132924 POINT(40.22322463869082 73.05975703204454) bank132924 +132925 POINT(39.91651544197777 74.55090570930405) bank132925 +132926 POINT(40.30434178378647 74.48355347779032) bank132926 +132927 POINT(40.54012922500358 74.55879467427944) bank132927 +132928 POINT(41.50403040092871 74.27958465121593) bank132928 +132929 POINT(40.56197897381103 73.51684998607804) bank132929 +132930 POINT(39.733106808370685 73.27387788288526) bank132930 +132931 POINT(40.679009793013094 73.29212420877293) bank132931 +132932 POINT(41.4343928980721 74.9787122925498) bank132932 +132933 POINT(41.466048984093774 73.58334209883841) bank132933 +132934 POINT(40.37590667975981 73.44753303539642) bank132934 +132935 POINT(41.58037954577074 74.89658277088718) bank132935 +132936 POINT(41.40206787728665 73.64707266284658) bank132936 +132937 POINT(41.31026573756896 73.0507174900017) bank132937 +132938 POINT(40.84220922091871 73.3478203246714) bank132938 +132939 POINT(40.882261462032844 73.52911471323138) bank132939 +132940 POINT(40.130326821315336 73.52759190164664) bank132940 +132941 POINT(41.35491183782309 73.26570136503106) bank132941 +132942 POINT(40.55159118720346 73.41351961916791) bank132942 +132943 POINT(41.412702246415314 74.8464070695034) bank132943 +132944 POINT(40.63948427052722 74.01462773970967) bank132944 +132945 POINT(41.68341504223283 73.53846941653234) bank132945 +132946 POINT(39.874622576145335 73.84125792606788) bank132946 +132947 POINT(41.56051080030274 74.22138983040857) bank132947 +132948 POINT(39.863021783637485 73.80412502062947) bank132948 +132949 POINT(41.35289942153676 74.099254002146) bank132949 +132950 POINT(40.55485373391334 73.15306223712606) bank132950 +132951 POINT(40.600395706860056 73.80180446013838) bank132951 +132952 POINT(39.97028658292155 74.11167052098722) bank132952 +132953 POINT(40.367441129822886 73.57714148442413) bank132953 +132954 POINT(40.53649146401452 73.50448397370744) bank132954 +132955 POINT(41.485098577398794 74.19294204429778) bank132955 +132956 POINT(41.12287875690059 74.1740676361305) bank132956 +132957 POINT(40.67632707882978 74.55010044157025) bank132957 +132958 POINT(40.678932991695746 74.11412168842205) bank132958 +132959 POINT(40.60117162139551 73.48579924769258) bank132959 +132960 POINT(40.626160582307676 74.90755045548849) bank132960 +132961 POINT(40.38629301217712 73.10264061457116) bank132961 +132962 POINT(41.199907806538256 74.79606719602732) bank132962 +132963 POINT(41.53597613059389 73.06684324296953) bank132963 +132964 POINT(41.07247825724252 74.66459842042711) bank132964 +132965 POINT(39.85945762379168 74.51991002231755) bank132965 +132966 POINT(40.232663836863175 73.84999099761527) bank132966 +132967 POINT(41.211723998129855 74.60655251125448) bank132967 +132968 POINT(40.76591660280341 74.83254802223463) bank132968 +132969 POINT(40.30403784477496 74.9310065505651) bank132969 +132970 POINT(41.46087170091015 75.00312750394696) bank132970 +132971 POINT(40.275823850627575 73.70455822766132) bank132971 +132972 POINT(41.458794211147186 74.25673803895597) bank132972 +132973 POINT(41.63593478095631 74.59248089424831) bank132973 +132974 POINT(41.27226190981544 74.34729642194552) bank132974 +132975 POINT(41.44112520888573 73.74388595208754) bank132975 +132976 POINT(41.314057832604604 73.66270084409835) bank132976 +132977 POINT(41.413278163734475 74.82755471836941) bank132977 +132978 POINT(41.22013344110351 73.16050298668179) bank132978 +132979 POINT(40.62510137886135 73.27032010897565) bank132979 +132980 POINT(40.99266228865359 73.27365803108036) bank132980 +132981 POINT(40.79555586485959 74.86924237468884) bank132981 +132982 POINT(40.897737958293604 74.01895916412697) bank132982 +132983 POINT(41.056134304909456 73.40630065076887) bank132983 +132984 POINT(41.31321555553609 74.09038329286614) bank132984 +132985 POINT(41.33128155304704 74.20297306227717) bank132985 +132986 POINT(39.91650176018063 73.23254082342962) bank132986 +132987 POINT(40.03966674199624 73.57871036158133) bank132987 +132988 POINT(40.45308967038424 74.22524465976377) bank132988 +132989 POINT(40.74702448288199 73.26374886968944) bank132989 +132990 POINT(40.778981699764515 75.00158188520925) bank132990 +132991 POINT(41.164941958569095 74.39071914440842) bank132991 +132992 POINT(40.99457508594364 73.36667127501244) bank132992 +132993 POINT(40.02414939328985 73.71387838177908) bank132993 +132994 POINT(40.54562427365695 73.46745111834451) bank132994 +132995 POINT(40.757990412675554 73.0368851263951) bank132995 +132996 POINT(41.5859923877781 73.42296559744925) bank132996 +132997 POINT(40.53544368339053 73.52642739200434) bank132997 +132998 POINT(41.4431406117885 74.19461368094503) bank132998 +132999 POINT(41.40263410141331 73.93916650598923) bank132999 +133000 POINT(39.88631606064158 73.84157094774879) bank133000 +133001 POINT(40.41785688562214 75.00592106967392) bank133001 +133002 POINT(40.095651993337896 73.10147936111528) bank133002 +133003 POINT(40.54543484665066 73.10704572444975) bank133003 +133004 POINT(40.70983094010389 73.74215696868073) bank133004 +133005 POINT(40.6453133917919 74.74860099706191) bank133005 +133006 POINT(40.393378893593564 73.32091901365479) bank133006 +133007 POINT(40.93675126024103 73.63835908631879) bank133007 +133008 POINT(41.2465660531734 73.33720528787374) bank133008 +133009 POINT(41.19646579303997 73.79004118193427) bank133009 +133010 POINT(40.11641767192975 73.10255251486558) bank133010 +133011 POINT(40.06096653215164 74.43492632818946) bank133011 +133012 POINT(39.718562941840844 73.43661967225121) bank133012 +133013 POINT(41.03755155111599 74.94028023135422) bank133013 +133014 POINT(41.57904074827267 73.38231944956922) bank133014 +133015 POINT(40.75131855826736 73.62018646065668) bank133015 +133016 POINT(40.301007563061106 73.85170173313406) bank133016 +133017 POINT(40.87804289947008 74.39445553165908) bank133017 +133018 POINT(41.64465985261601 73.18095325026678) bank133018 +133019 POINT(41.03769233370388 73.92052725155564) bank133019 +133020 POINT(40.51714657266399 73.96810432230139) bank133020 +133021 POINT(40.41628397363323 73.18288937035966) bank133021 +133022 POINT(40.70414399972357 73.41543783408639) bank133022 +133023 POINT(40.04324205413964 74.5733937696013) bank133023 +133024 POINT(40.598385007433556 74.09716104615251) bank133024 +133025 POINT(41.457442855511616 74.90822828307508) bank133025 +133026 POINT(40.15839196624261 74.96506935862078) bank133026 +133027 POINT(41.62251735887868 74.4773758656164) bank133027 +133028 POINT(39.984626730590506 73.83169246127719) bank133028 +133029 POINT(39.992496515736484 74.72911393271376) bank133029 +133030 POINT(40.27505710740391 74.1652309317891) bank133030 +133031 POINT(40.22047353093831 73.61343637449035) bank133031 +133032 POINT(39.822823682110176 73.92682271719526) bank133032 +133033 POINT(40.00508585021764 73.91756499903707) bank133033 +133034 POINT(40.025889810990826 73.57860989718344) bank133034 +133035 POINT(40.88568323262767 74.55015670966213) bank133035 +133036 POINT(40.368558768590326 74.25065317545854) bank133036 +133037 POINT(40.99135209247575 73.96358358822289) bank133037 +133038 POINT(40.92750466182256 74.40249578879765) bank133038 +133039 POINT(41.711504546399624 73.56717058162933) bank133039 +133040 POINT(40.43032081039801 74.97700160108475) bank133040 +133041 POINT(40.80370707781006 73.7293880852505) bank133041 +133042 POINT(41.09916383990624 74.382144362768) bank133042 +133043 POINT(41.04644765545488 74.71342603400385) bank133043 +133044 POINT(40.893008853927526 74.16687299749549) bank133044 +133045 POINT(40.49631179683499 74.30867546531874) bank133045 +133046 POINT(40.63569891666616 73.43175447501056) bank133046 +133047 POINT(40.68939021094215 74.14491961912402) bank133047 +133048 POINT(40.703352306864204 74.07206449293817) bank133048 +133049 POINT(41.000717943337015 74.89148011292407) bank133049 +133050 POINT(40.53863515486463 73.58692827718055) bank133050 +133051 POINT(40.527074853742995 74.78848585144472) bank133051 +133052 POINT(41.30906560458963 73.47003161747405) bank133052 +133053 POINT(40.49322234967864 74.8173625901373) bank133053 +133054 POINT(40.65870823859265 74.42316291267362) bank133054 +133055 POINT(40.616837516964786 74.46190115655197) bank133055 +133056 POINT(40.29920370377045 74.4741673976311) bank133056 +133057 POINT(41.08778897338134 74.77770414501843) bank133057 +133058 POINT(39.97042015297447 74.1910713263463) bank133058 +133059 POINT(39.80919049965497 73.85132359823005) bank133059 +133060 POINT(39.94831243322339 74.7075330677246) bank133060 +133061 POINT(40.90592731988979 74.44086543679123) bank133061 +133062 POINT(41.60516579242356 74.75401326173585) bank133062 +133063 POINT(39.843464451462836 74.58089573266135) bank133063 +133064 POINT(40.64435661870396 73.90243136078375) bank133064 +133065 POINT(40.26288678809801 74.38466579459842) bank133065 +133066 POINT(39.929334076350294 73.99910353561346) bank133066 +133067 POINT(40.7241262581547 74.74461011966433) bank133067 +133068 POINT(40.49647940551714 74.75414417841746) bank133068 +133069 POINT(41.2023400693435 74.67215779254082) bank133069 +133070 POINT(39.8764855693574 74.41408002652656) bank133070 +133071 POINT(40.9267874626327 74.24250500564982) bank133071 +133072 POINT(41.564308022106 73.87333202883978) bank133072 +133073 POINT(39.880927067920126 73.82661880980962) bank133073 +133074 POINT(40.59742053990393 74.67301093353025) bank133074 +133075 POINT(41.60725744409855 74.0207599028325) bank133075 +133076 POINT(40.90892529833272 74.96691436089239) bank133076 +133077 POINT(40.9487589076547 74.91863814732649) bank133077 +133078 POINT(39.894045202023435 74.3645378189202) bank133078 +133079 POINT(40.595427735794644 74.34075674189796) bank133079 +133080 POINT(41.504451889253886 73.2353139159802) bank133080 +133081 POINT(40.872039737141435 73.29802941292976) bank133081 +133082 POINT(41.228603490696386 73.53522194229608) bank133082 +133083 POINT(41.12979777410208 73.31830756642935) bank133083 +133084 POINT(40.69189079722433 73.57069956617138) bank133084 +133085 POINT(41.40084010191099 73.65402598855773) bank133085 +133086 POINT(39.82307314056147 74.56389509308939) bank133086 +133087 POINT(40.26244658492705 74.1032355583449) bank133087 +133088 POINT(41.54672703459987 74.83580541934298) bank133088 +133089 POINT(39.79354426600579 73.43938824014727) bank133089 +133090 POINT(40.52662701063392 73.58934254200658) bank133090 +133091 POINT(41.205497229400954 73.86189435445425) bank133091 +133092 POINT(41.08586615142707 74.37800622467735) bank133092 +133093 POINT(41.582292235884665 73.76150890908337) bank133093 +133094 POINT(41.06226721984591 74.72400151600499) bank133094 +133095 POINT(39.809953209412896 73.64851929507873) bank133095 +133096 POINT(41.47634515692512 73.25357580353547) bank133096 +133097 POINT(41.279581888239356 74.11882700294466) bank133097 +133098 POINT(40.9945336369409 74.82567042465429) bank133098 +133099 POINT(41.270625596569474 74.28322173924303) bank133099 +133100 POINT(40.19278910029323 74.40844029301076) bank133100 +133101 POINT(40.870752351975064 74.47102656067192) bank133101 +133102 POINT(39.75266865937589 74.1809406785647) bank133102 +133103 POINT(41.431527412561515 74.18156675192043) bank133103 +133104 POINT(39.96782744744553 73.54924550471786) bank133104 +133105 POINT(40.64873628594234 74.79926460245336) bank133105 +133106 POINT(40.588733121099175 73.19920474411266) bank133106 +133107 POINT(40.25754208374395 74.90544859559877) bank133107 +133108 POINT(41.43412543970413 73.75539238677227) bank133108 +133109 POINT(41.33010447252832 74.07707112357576) bank133109 +133110 POINT(40.57305021735323 73.97471101545703) bank133110 +133111 POINT(40.48385725721789 73.61783579586103) bank133111 +133112 POINT(41.23487542326839 74.5779398899846) bank133112 +133113 POINT(40.68015586706161 73.59771837382935) bank133113 +133114 POINT(40.06225130716775 74.90171007413207) bank133114 +133115 POINT(41.33731200651356 73.16365980070232) bank133115 +133116 POINT(40.777435373444575 74.60022775549893) bank133116 +133117 POINT(40.66539526942639 73.17080539908164) bank133117 +133118 POINT(40.97162981298596 73.22863525797679) bank133118 +133119 POINT(41.448775295583 73.0146838954479) bank133119 +133120 POINT(40.7257594727733 73.93841259597305) bank133120 +133121 POINT(40.24529554928696 74.41119730253956) bank133121 +133122 POINT(39.817875142649655 74.13330399394853) bank133122 +133123 POINT(40.94884817374063 74.66432202303929) bank133123 +133124 POINT(40.90808099143585 74.14516002185594) bank133124 +133125 POINT(41.062492685102725 74.75982737191464) bank133125 +133126 POINT(40.18081986492403 73.50558032831998) bank133126 +133127 POINT(41.39019101963943 73.06598523190289) bank133127 +133128 POINT(40.49999616050134 74.14258177079252) bank133128 +133129 POINT(39.72688262156603 73.50308038863099) bank133129 +133130 POINT(41.411512558615065 74.09745297457748) bank133130 +133131 POINT(41.240692394412825 74.1066416209457) bank133131 +133132 POINT(40.23067540840225 73.16689869982945) bank133132 +133133 POINT(40.05114100803295 74.3881463602566) bank133133 +133134 POINT(39.97785221463523 73.2075663070605) bank133134 +133135 POINT(41.16782849780028 73.27870542807425) bank133135 +133136 POINT(40.90677897945172 73.8695860771271) bank133136 +133137 POINT(40.154937737414606 74.28123220608522) bank133137 +133138 POINT(40.607304718690116 73.06085877152728) bank133138 +133139 POINT(40.98299360076556 74.80046438475533) bank133139 +133140 POINT(41.66058642211916 73.1856349456845) bank133140 +133141 POINT(40.71241703481323 73.43987382637613) bank133141 +133142 POINT(40.17965428133501 74.6040901338996) bank133142 +133143 POINT(40.14269749575589 74.59482291775439) bank133143 +133144 POINT(40.379864102997274 73.3866457228139) bank133144 +133145 POINT(41.097490037936005 73.57895248401913) bank133145 +133146 POINT(40.8700270897713 73.78180831235122) bank133146 +133147 POINT(40.91163810265564 73.91421507742714) bank133147 +133148 POINT(41.222933599150814 74.2260093943686) bank133148 +133149 POINT(40.75479897575043 73.30472476113775) bank133149 +133150 POINT(41.26836677436771 73.46542362612152) bank133150 +133151 POINT(41.28726114398428 74.64316394225588) bank133151 +133152 POINT(41.20438303612448 73.45413482479648) bank133152 +133153 POINT(40.047155494899386 73.37416539762646) bank133153 +133154 POINT(40.75628148642175 73.81900721238122) bank133154 +133155 POINT(40.58966400898162 73.30887201565294) bank133155 +133156 POINT(41.53050511473755 73.41592070084901) bank133156 +133157 POINT(39.795276478782554 73.96733343105136) bank133157 +133158 POINT(40.14329798229364 73.24625330463176) bank133158 +133159 POINT(41.05712779278026 73.1991043802935) bank133159 +133160 POINT(40.47156900055288 73.17173526154757) bank133160 +133161 POINT(39.78495629836591 73.07298943898701) bank133161 +133162 POINT(40.96300427183728 73.97841964416793) bank133162 +133163 POINT(41.20024840711095 74.78455130265031) bank133163 +133164 POINT(40.232948706935105 73.03650168182892) bank133164 +133165 POINT(40.862149216056416 73.03655938408082) bank133165 +133166 POINT(40.5679372558925 74.15732975235555) bank133166 +133167 POINT(40.44650101035774 74.4594654108493) bank133167 +133168 POINT(41.588040514446256 74.15072299657623) bank133168 +133169 POINT(40.04911607209878 74.77744592634716) bank133169 +133170 POINT(39.972898212580674 74.28281046757996) bank133170 +133171 POINT(40.89231163323398 73.49187020960257) bank133171 +133172 POINT(40.770396096162024 73.36537664521188) bank133172 +133173 POINT(39.797314485056695 73.47891576422842) bank133173 +133174 POINT(39.95282051107498 74.50530098626498) bank133174 +133175 POINT(41.09899018979699 73.87791885135333) bank133175 +133176 POINT(40.938900546652626 73.89784006690266) bank133176 +133177 POINT(39.73408241450134 74.52815393182667) bank133177 +133178 POINT(39.784867963598444 73.83961189669749) bank133178 +133179 POINT(41.35958890370061 74.0576300836159) bank133179 +133180 POINT(41.363791156518715 74.41665555685782) bank133180 +133181 POINT(39.74135521128336 74.24736932923332) bank133181 +133182 POINT(40.122980108647845 74.75420274552039) bank133182 +133183 POINT(39.95991875194321 74.41846396594654) bank133183 +133184 POINT(40.12961193392326 74.4420362484339) bank133184 +133185 POINT(40.23745491510886 73.29858612178596) bank133185 +133186 POINT(39.7945566511468 74.2708354665926) bank133186 +133187 POINT(41.55948694475142 73.62294913963052) bank133187 +133188 POINT(39.73904578146909 74.26774806791974) bank133188 +133189 POINT(39.92322445433288 74.48301639148794) bank133189 +133190 POINT(40.36213049453998 74.4951339370616) bank133190 +133191 POINT(39.865381931526436 73.98694644517077) bank133191 +133192 POINT(40.80467579045723 74.75159077758994) bank133192 +133193 POINT(41.303556072627416 74.31745563027849) bank133193 +133194 POINT(41.251449196378495 73.63619646019878) bank133194 +133195 POINT(41.36531155461014 74.44321765028691) bank133195 +133196 POINT(40.8573453457575 74.16605845738957) bank133196 +133197 POINT(40.33320113099364 74.5795165070463) bank133197 +133198 POINT(41.02462919516409 74.8388515696981) bank133198 +133199 POINT(41.242083734746274 74.23540841892782) bank133199 +133200 POINT(40.08342889533978 74.18444133481358) bank133200 +133201 POINT(41.48773507599548 74.8166326928063) bank133201 +133202 POINT(40.28137195657111 74.24445537134095) bank133202 +133203 POINT(41.653004538603305 73.62079871750575) bank133203 +133204 POINT(41.47150511822277 74.02934890254282) bank133204 +133205 POINT(41.63191693509186 74.06025690790136) bank133205 +133206 POINT(40.32149581766951 73.03836606976995) bank133206 +133207 POINT(40.762032596228295 73.321912168238) bank133207 +133208 POINT(40.701998148995834 74.60511789387179) bank133208 +133209 POINT(40.70382182008541 74.45718405610143) bank133209 +133210 POINT(40.34959161633727 73.50983904338351) bank133210 +133211 POINT(40.80280952365522 73.2853561664249) bank133211 +133212 POINT(40.5309092475631 74.49786282019869) bank133212 +133213 POINT(41.32853519645625 74.03560622752114) bank133213 +133214 POINT(41.63145674417103 73.26044206621833) bank133214 +133215 POINT(40.8177830278448 73.1092825362757) bank133215 +133216 POINT(39.80715602700024 74.16894263800027) bank133216 +133217 POINT(40.061138805519036 73.06855087685031) bank133217 +133218 POINT(40.80540399919522 74.67212106474447) bank133218 +133219 POINT(40.578820467714465 74.86202850610285) bank133219 +133220 POINT(40.248863711456295 74.75738430323815) bank133220 +133221 POINT(40.90492661511224 74.16540331760564) bank133221 +133222 POINT(41.18788380469455 73.76078000915473) bank133222 +133223 POINT(41.686008789358716 74.23081582015233) bank133223 +133224 POINT(39.97007143904716 73.862536554891) bank133224 +133225 POINT(41.67528166163553 73.82386547054413) bank133225 +133226 POINT(39.76453408059939 73.71030741941298) bank133226 +133227 POINT(40.14892561124557 73.37725771561153) bank133227 +133228 POINT(40.181307614466306 73.97068885869074) bank133228 +133229 POINT(40.53889401831731 73.16626813671807) bank133229 +133230 POINT(41.334793537329446 73.8334917011258) bank133230 +133231 POINT(40.28731602056503 73.71793276057718) bank133231 +133232 POINT(41.04512602952877 73.17963554341654) bank133232 +133233 POINT(39.9177274753773 73.21664899372746) bank133233 +133234 POINT(41.131851054779276 74.46802455878316) bank133234 +133235 POINT(40.02010314964583 74.27879008281796) bank133235 +133236 POINT(40.60877907798435 74.79919150455727) bank133236 +133237 POINT(40.73339574601243 73.54422090872177) bank133237 +133238 POINT(39.94459225190833 73.82501958122205) bank133238 +133239 POINT(39.979139103333885 74.03148053246795) bank133239 +133240 POINT(40.62452847052621 74.27400632193473) bank133240 +133241 POINT(40.042670125079646 73.94633918649097) bank133241 +133242 POINT(40.1773102583258 74.27552955593713) bank133242 +133243 POINT(40.77354078458971 74.18335129775224) bank133243 +133244 POINT(40.42699476086874 73.07208839591254) bank133244 +133245 POINT(40.043929193059014 74.10631687161603) bank133245 +133246 POINT(41.28494063517521 74.3448314686838) bank133246 +133247 POINT(40.45253082734898 74.9627248096433) bank133247 +133248 POINT(40.4544083656019 73.95986708283414) bank133248 +133249 POINT(40.62323466948209 73.21497331219075) bank133249 +133250 POINT(41.04067279772248 74.11107812276704) bank133250 +133251 POINT(39.95237552176278 73.56489163263367) bank133251 +133252 POINT(40.596724615185295 74.40326332126952) bank133252 +133253 POINT(40.31134953872453 73.91605213525578) bank133253 +133254 POINT(40.95983519831015 73.98516169434666) bank133254 +133255 POINT(40.74177779568047 74.6131286796393) bank133255 +133256 POINT(41.55874145566061 73.17556221581826) bank133256 +133257 POINT(39.87038015501563 73.45586452148115) bank133257 +133258 POINT(40.95961847180107 73.22545567571642) bank133258 +133259 POINT(41.60930081907198 73.19826643749616) bank133259 +133260 POINT(40.55017328219834 73.63418268254529) bank133260 +133261 POINT(40.522622795920796 74.19877415763297) bank133261 +133262 POINT(40.80480673254295 73.54501458866373) bank133262 +133263 POINT(39.910979197651216 73.38133267589677) bank133263 +133264 POINT(40.17210064763651 74.88681084378064) bank133264 +133265 POINT(41.42080279608515 73.38204467892228) bank133265 +133266 POINT(40.00640929286105 74.16331468662372) bank133266 +133267 POINT(39.93180746801976 73.13601753677213) bank133267 +133268 POINT(41.04437825315133 74.92230158312438) bank133268 +133269 POINT(40.15872585116477 74.02654666492899) bank133269 +133270 POINT(40.05686740935409 74.91956633192297) bank133270 +133271 POINT(39.86912279873538 73.59275225315146) bank133271 +133272 POINT(39.81588875156475 74.86704975904425) bank133272 +133273 POINT(40.62107394063725 74.90955925367602) bank133273 +133274 POINT(41.03759969589717 74.63848876567832) bank133274 +133275 POINT(40.2600848359828 73.22190977760276) bank133275 +133276 POINT(40.41193576419553 73.66304155503624) bank133276 +133277 POINT(40.94411241391932 73.69661445212778) bank133277 +133278 POINT(40.009349710655314 74.0694096812223) bank133278 +133279 POINT(40.789547812248536 74.02984183574215) bank133279 +133280 POINT(41.509044960435205 74.34786176774203) bank133280 +133281 POINT(41.149776087429075 74.65357669028268) bank133281 +133282 POINT(40.64132380303094 73.7567844877247) bank133282 +133283 POINT(40.28446649330114 74.02095988065224) bank133283 +133284 POINT(40.98591059364946 74.17716335454917) bank133284 +133285 POINT(40.27715591899947 73.67771295799183) bank133285 +133286 POINT(40.1279197424818 74.35250007750868) bank133286 +133287 POINT(40.938766094117845 73.30908220324679) bank133287 +133288 POINT(41.056828985388144 73.46454419262973) bank133288 +133289 POINT(40.20249090884855 73.02482614568493) bank133289 +133290 POINT(41.475702606469426 73.04524481303969) bank133290 +133291 POINT(41.06788407708388 73.64912389248538) bank133291 +133292 POINT(40.06937218650045 74.1926023796087) bank133292 +133293 POINT(40.69709899191449 74.46545710525528) bank133293 +133294 POINT(40.42598264167757 74.58038745554474) bank133294 +133295 POINT(40.38500489363911 73.99418234859705) bank133295 +133296 POINT(39.85913684387735 73.4963039457686) bank133296 +133297 POINT(39.78090408678928 73.93627999485828) bank133297 +133298 POINT(40.22617261560322 74.38294224257494) bank133298 +133299 POINT(40.41206522394136 74.13517459886639) bank133299 +133300 POINT(40.56508890419023 73.69305882863516) bank133300 +133301 POINT(40.77636721833039 73.04962306018861) bank133301 +133302 POINT(40.0861773268046 73.19797433021891) bank133302 +133303 POINT(40.8119984462406 74.95982637480417) bank133303 +133304 POINT(41.34132242407825 73.03622148126767) bank133304 +133305 POINT(41.67384142844785 73.26776687045546) bank133305 +133306 POINT(41.51948314869701 73.50727223107961) bank133306 +133307 POINT(41.50155921742262 73.48411520754217) bank133307 +133308 POINT(40.846259169980684 73.3735558896334) bank133308 +133309 POINT(40.6107897272897 73.54352739893254) bank133309 +133310 POINT(41.61573486954711 73.35961555118891) bank133310 +133311 POINT(41.16358013318287 74.36326806468358) bank133311 +133312 POINT(40.48523453175542 73.98284921165038) bank133312 +133313 POINT(41.514271505743864 73.6007298768457) bank133313 +133314 POINT(40.50182945729148 74.70604261441055) bank133314 +133315 POINT(40.197814196880216 74.86783878908346) bank133315 +133316 POINT(41.113777476307654 73.61260154225913) bank133316 +133317 POINT(40.6550608326995 73.40135081006572) bank133317 +133318 POINT(40.25794987052564 74.02397693232122) bank133318 +133319 POINT(41.314611400446694 74.44679965469622) bank133319 +133320 POINT(40.80299598428352 74.94387839968482) bank133320 +133321 POINT(40.576970070581346 73.24821771237664) bank133321 +133322 POINT(39.87180870529823 74.67481178187722) bank133322 +133323 POINT(40.010929734714146 73.06462467036283) bank133323 +133324 POINT(40.46892379406934 74.95318748400834) bank133324 +133325 POINT(41.69144010888481 73.77259068588451) bank133325 +133326 POINT(40.55674664651882 73.35027977097157) bank133326 +133327 POINT(40.924249815776285 74.14453489993834) bank133327 +133328 POINT(40.77142530226759 73.11806364991409) bank133328 +133329 POINT(40.31265196637829 74.9266839380488) bank133329 +133330 POINT(41.37190158991069 73.388326667335) bank133330 +133331 POINT(41.036608968770935 74.75409309091762) bank133331 +133332 POINT(40.98780077776992 74.1297822076042) bank133332 +133333 POINT(41.194949380279716 75.00579719256244) bank133333 +133334 POINT(40.72410989149724 74.16708476279837) bank133334 +133335 POINT(41.333538669407815 73.1319771345109) bank133335 +133336 POINT(41.17093154153134 74.08081668933072) bank133336 +133337 POINT(40.61442574745096 73.34240446285945) bank133337 +133338 POINT(41.66978534592575 73.65498922714961) bank133338 +133339 POINT(41.277060811598865 73.52500173462843) bank133339 +133340 POINT(39.821119333171914 73.9195413995072) bank133340 +133341 POINT(39.94570182159857 74.57996588830684) bank133341 +133342 POINT(40.08451466230427 74.48919468917339) bank133342 +133343 POINT(41.06114499554277 73.41832600663037) bank133343 +133344 POINT(39.90111073255852 74.19405559132406) bank133344 +133345 POINT(39.856640113488325 73.21876188305558) bank133345 +133346 POINT(41.09613415454221 74.76708279226946) bank133346 +133347 POINT(40.13396482912141 73.14387009822447) bank133347 +133348 POINT(40.40556539974488 73.02210040460703) bank133348 +133349 POINT(41.03841842258241 73.0657833378374) bank133349 +133350 POINT(40.47444728631881 73.45471309312467) bank133350 +133351 POINT(40.1093133527172 74.77113812136388) bank133351 +133352 POINT(39.94141645864632 74.87423734932784) bank133352 +133353 POINT(40.36130605353664 73.21561312740342) bank133353 +133354 POINT(40.887272597623124 73.24670891048373) bank133354 +133355 POINT(39.96605443282054 73.59475527394603) bank133355 +133356 POINT(40.783209466368774 73.67939220096339) bank133356 +133357 POINT(39.854571567241905 74.2628488860368) bank133357 +133358 POINT(40.68178976471496 73.66643299582545) bank133358 +133359 POINT(41.625120101057654 74.7772985990996) bank133359 +133360 POINT(41.01280863693602 74.66267943664594) bank133360 +133361 POINT(40.798812334492716 73.56931534310834) bank133361 +133362 POINT(40.937343503744245 73.75407925150859) bank133362 +133363 POINT(39.99179133028273 74.49201179728304) bank133363 +133364 POINT(40.57037802140784 73.9380705968605) bank133364 +133365 POINT(41.18101065639706 73.0717667767157) bank133365 +133366 POINT(40.38612564097967 73.7714084664609) bank133366 +133367 POINT(41.10450931291141 73.92798867935453) bank133367 +133368 POINT(41.47606480858072 74.97765623073059) bank133368 +133369 POINT(40.44428660833888 74.83702038760036) bank133369 +133370 POINT(41.28818668080291 74.81672604329256) bank133370 +133371 POINT(41.22039654194701 74.38092998342097) bank133371 +133372 POINT(39.97551049755674 73.16044606351298) bank133372 +133373 POINT(40.369686981885025 73.89379817474739) bank133373 +133374 POINT(39.94200865530387 74.10552105220826) bank133374 +133375 POINT(41.09447189743471 74.31791882189559) bank133375 +133376 POINT(39.810051833964636 74.2312432381116) bank133376 +133377 POINT(40.47824009663126 74.84981060389117) bank133377 +133378 POINT(40.1321047943177 73.32778344778875) bank133378 +133379 POINT(39.78105629928659 73.75633048751622) bank133379 +133380 POINT(41.479436559051145 73.27172123390194) bank133380 +133381 POINT(40.25307567471477 73.37880690440774) bank133381 +133382 POINT(40.79986545153108 73.1981438483186) bank133382 +133383 POINT(41.48963771058972 73.85221218792631) bank133383 +133384 POINT(40.684745969950605 74.88808268201414) bank133384 +133385 POINT(41.61097230367194 74.01958462115024) bank133385 +133386 POINT(41.186022824195575 74.3499715619058) bank133386 +133387 POINT(40.134157335238896 73.80428133497215) bank133387 +133388 POINT(39.9994059989444 74.69917502108208) bank133388 +133389 POINT(41.60248054988432 73.637960165172) bank133389 +133390 POINT(40.784024729516105 74.341206110204) bank133390 +133391 POINT(40.84022323090076 74.55315148834215) bank133391 +133392 POINT(40.988198548436046 73.06758807801326) bank133392 +133393 POINT(41.1120230558058 74.70906952955077) bank133393 +133394 POINT(41.00044720993183 74.3765149951638) bank133394 +133395 POINT(41.54327160446508 74.37473920963097) bank133395 +133396 POINT(40.593183225263125 73.22114879716581) bank133396 +133397 POINT(41.334430925857376 74.01452485720357) bank133397 +133398 POINT(40.724599197103316 73.73536742696943) bank133398 +133399 POINT(40.45728015905074 73.70757515598216) bank133399 +133400 POINT(40.77188425019504 73.09331689728677) bank133400 +133401 POINT(39.763228413515215 73.67030139772595) bank133401 +133402 POINT(41.68433500950811 73.52579302166984) bank133402 +133403 POINT(41.63732098201021 73.28105529309725) bank133403 +133404 POINT(40.80276273254316 74.64269910364928) bank133404 +133405 POINT(40.282891563720106 74.30604893136679) bank133405 +133406 POINT(39.97494330432466 74.6035068415458) bank133406 +133407 POINT(39.75540351005787 74.6672102015828) bank133407 +133408 POINT(40.04333759467086 73.04624113914716) bank133408 +133409 POINT(40.42620246205014 74.08771588665809) bank133409 +133410 POINT(39.791853418347436 73.77388616086395) bank133410 +133411 POINT(40.51786722944986 74.6310463885302) bank133411 +133412 POINT(40.695192936952445 73.50331408608032) bank133412 +133413 POINT(41.555661415015386 73.9065903258657) bank133413 +133414 POINT(40.60205952069028 75.00445780883722) bank133414 +133415 POINT(40.32223754862791 73.32998848441964) bank133415 +133416 POINT(40.87320760263244 74.36284853410892) bank133416 +133417 POINT(41.30479772261755 74.9442116256536) bank133417 +133418 POINT(40.0773629097282 74.39447981192875) bank133418 +133419 POINT(41.389130368253866 74.59942103884899) bank133419 +133420 POINT(40.923631276205995 74.11678537200109) bank133420 +133421 POINT(41.0776287087048 73.31196902526239) bank133421 +133422 POINT(39.975653155556 74.49720070210793) bank133422 +133423 POINT(41.159004241972326 73.17253401579727) bank133423 +133424 POINT(41.41806766723521 74.7588218899647) bank133424 +133425 POINT(41.54677544458176 73.01912272061702) bank133425 +133426 POINT(40.99527731359352 74.60788443850329) bank133426 +133427 POINT(39.934833723905584 74.57770792720737) bank133427 +133428 POINT(40.57100681794135 73.6581628388932) bank133428 +133429 POINT(39.71869274011594 74.03534748036634) bank133429 +133430 POINT(41.63755023023673 74.13620038730446) bank133430 +133431 POINT(41.65942576473745 73.97652930918503) bank133431 +133432 POINT(41.12760467762841 73.80467282089838) bank133432 +133433 POINT(41.66059823122657 73.44143151485817) bank133433 +133434 POINT(40.210971607559046 73.26789094209846) bank133434 +133435 POINT(41.3067464675255 74.01641300579881) bank133435 +133436 POINT(39.83224707906651 73.19944859471616) bank133436 +133437 POINT(40.57526048233661 73.37536911489774) bank133437 +133438 POINT(40.532174210211465 74.33842734588363) bank133438 +133439 POINT(41.61874182041968 73.82236371976326) bank133439 +133440 POINT(41.220089380868934 74.74987431004404) bank133440 +133441 POINT(39.95068356449491 74.25923633724192) bank133441 +133442 POINT(40.03263856513673 74.57938515983571) bank133442 +133443 POINT(41.45533239441825 73.10567585338583) bank133443 +133444 POINT(41.138127587895625 74.1924155511604) bank133444 +133445 POINT(40.6281743292141 74.2323093248579) bank133445 +133446 POINT(40.27680491232105 75.00021054971535) bank133446 +133447 POINT(40.630546426395284 74.71036864228759) bank133447 +133448 POINT(41.01558335626005 73.73191631369527) bank133448 +133449 POINT(41.390491158302446 74.37606477425437) bank133449 +133450 POINT(41.36160185782915 73.6746572583494) bank133450 +133451 POINT(40.79941321683208 74.22504667544929) bank133451 +133452 POINT(40.92117053212971 73.54293496504398) bank133452 +133453 POINT(39.861183158796585 74.7976302737076) bank133453 +133454 POINT(41.467121931939865 74.54342104163442) bank133454 +133455 POINT(41.46082501746693 74.83184251276127) bank133455 +133456 POINT(40.584133431669024 74.17272168446128) bank133456 +133457 POINT(39.969909733448446 74.61335092958282) bank133457 +133458 POINT(40.648329369147376 74.69719422596494) bank133458 +133459 POINT(40.67698256260047 74.79959582968665) bank133459 +133460 POINT(40.430276557673785 73.71840732455252) bank133460 +133461 POINT(41.0046286435247 74.99369727444008) bank133461 +133462 POINT(40.58417192782372 74.39529746794929) bank133462 +133463 POINT(40.70664552614662 74.54061585085402) bank133463 +133464 POINT(40.28680399870499 74.2064408803719) bank133464 +133465 POINT(40.65573127131068 74.29867295362784) bank133465 +133466 POINT(40.83348556186803 73.33142069051951) bank133466 +133467 POINT(41.09165637919529 74.88240228752711) bank133467 +133468 POINT(41.534710999293814 74.45375964675979) bank133468 +133469 POINT(40.41345788572559 74.44896521801654) bank133469 +133470 POINT(41.61692668673601 73.65808182112792) bank133470 +133471 POINT(40.36615638321151 73.74778609601138) bank133471 +133472 POINT(40.53301302676494 74.94447652578279) bank133472 +133473 POINT(40.31376953913002 73.45636718478279) bank133473 +133474 POINT(41.17103852886987 74.73548501435481) bank133474 +133475 POINT(41.66998345441741 73.89585271534466) bank133475 +133476 POINT(41.44657193363494 73.03006519847847) bank133476 +133477 POINT(40.01289686860457 73.77577128497484) bank133477 +133478 POINT(40.913585292934776 73.8773321250014) bank133478 +133479 POINT(40.103596602862325 73.446542274346) bank133479 +133480 POINT(40.90836456979981 73.07275555063104) bank133480 +133481 POINT(40.05301304816619 73.82711182182229) bank133481 +133482 POINT(40.52641168053814 74.60311218437748) bank133482 +133483 POINT(41.68941082250837 73.42092363775552) bank133483 +133484 POINT(40.31741211336433 73.31346615707594) bank133484 +133485 POINT(41.27512011924157 74.49315811096194) bank133485 +133486 POINT(40.26441944180658 74.01119779140734) bank133486 +133487 POINT(40.08604562796705 74.82881355559739) bank133487 +133488 POINT(41.47408107796765 73.32257947906079) bank133488 +133489 POINT(41.27244777947772 73.79726356296406) bank133489 +133490 POINT(41.24305533867698 73.98477854871355) bank133490 +133491 POINT(40.93066855072162 73.87958660530963) bank133491 +133492 POINT(41.414318747916795 74.29330774712807) bank133492 +133493 POINT(41.33443204145056 73.79688989117483) bank133493 +133494 POINT(40.28374473201926 74.76190058158323) bank133494 +133495 POINT(40.358012333222945 73.26894265489585) bank133495 +133496 POINT(40.65437276538447 73.38511088800425) bank133496 +133497 POINT(40.9221523966965 73.73944929806662) bank133497 +133498 POINT(39.78216455851043 74.30915565096291) bank133498 +133499 POINT(40.801895126609644 73.05985567332341) bank133499 +133500 POINT(40.15904551996319 74.56477309816093) bank133500 +133501 POINT(41.13170776402158 73.93771716962323) bank133501 +133502 POINT(40.422141729836014 73.5275489387255) bank133502 +133503 POINT(41.33956373426224 74.82998571166219) bank133503 +133504 POINT(41.3223565454599 73.06304416015355) bank133504 +133505 POINT(40.86271535032916 74.87220583601243) bank133505 +133506 POINT(41.28067675555934 74.94570334966117) bank133506 +133507 POINT(40.52833411034683 73.22510057690106) bank133507 +133508 POINT(39.93741017073425 74.17579752062582) bank133508 +133509 POINT(40.54867891213232 74.17441305102486) bank133509 +133510 POINT(41.495468163933644 73.11099605788935) bank133510 +133511 POINT(41.11788416091425 73.70917583190068) bank133511 +133512 POINT(40.92459222165193 74.96008788387886) bank133512 +133513 POINT(40.8906702820307 73.65822385543598) bank133513 +133514 POINT(41.16508964907409 74.82206845002005) bank133514 +133515 POINT(41.358439756292604 73.7150352935412) bank133515 +133516 POINT(40.76680471647384 74.9472785222116) bank133516 +133517 POINT(41.540429458432996 73.41013535798112) bank133517 +133518 POINT(40.87678505059001 73.12488496721282) bank133518 +133519 POINT(40.71029953167862 74.18734588082509) bank133519 +133520 POINT(41.70860048480451 74.58598967594686) bank133520 +133521 POINT(41.485987574213844 73.2604789875316) bank133521 +133522 POINT(40.59690861991367 74.64709464274046) bank133522 +133523 POINT(40.308867437684405 74.11867005490832) bank133523 +133524 POINT(40.12950445165697 73.44929504602584) bank133524 +133525 POINT(40.087936950279115 74.44684514706181) bank133525 +133526 POINT(41.66390857282985 74.9749072087604) bank133526 +133527 POINT(40.11939338255798 74.82454346049192) bank133527 +133528 POINT(41.14804750207797 73.30135857047131) bank133528 +133529 POINT(39.89570043640975 74.36141965827629) bank133529 +133530 POINT(41.46218736530995 73.06391708797034) bank133530 +133531 POINT(41.15365987839597 74.10177937339682) bank133531 +133532 POINT(41.361332883158184 74.51639931607673) bank133532 +133533 POINT(41.30649636645146 73.58427274943375) bank133533 +133534 POINT(41.03403702137517 74.52635201287603) bank133534 +133535 POINT(39.71985220532346 73.00679893121566) bank133535 +133536 POINT(40.92410636513893 73.93919599899431) bank133536 +133537 POINT(39.76588649389347 73.30479996695175) bank133537 +133538 POINT(40.48490202047801 74.73668604805724) bank133538 +133539 POINT(41.10626711632625 74.64045101602295) bank133539 +133540 POINT(40.80726376464968 73.8116959469977) bank133540 +133541 POINT(41.596493489176176 73.11416021893153) bank133541 +133542 POINT(40.635125622404054 73.09566039980305) bank133542 +133543 POINT(41.16863558260243 73.48444691878127) bank133543 +133544 POINT(39.89476838566412 73.64170104720444) bank133544 +133545 POINT(41.50429081289748 74.93479045270789) bank133545 +133546 POINT(41.401129188743425 74.84126136368675) bank133546 +133547 POINT(41.58478435219448 74.41520943903532) bank133547 +133548 POINT(41.34066455094712 73.47803499548075) bank133548 +133549 POINT(41.514196068641795 74.8617890294979) bank133549 +133550 POINT(39.891817593120706 73.92794581392887) bank133550 +133551 POINT(40.74584818289354 73.37889230589067) bank133551 +133552 POINT(40.42528806949164 74.97722620320232) bank133552 +133553 POINT(40.830222229392035 73.32768548501876) bank133553 +133554 POINT(41.23216002790042 73.65643290311563) bank133554 +133555 POINT(40.41558673226509 73.15489906475808) bank133555 +133556 POINT(40.61377543535245 73.52985353626097) bank133556 +133557 POINT(41.64326147611806 74.8676210084226) bank133557 +133558 POINT(40.597760714880756 73.32684653942356) bank133558 +133559 POINT(41.673914949381206 74.78635716482127) bank133559 +133560 POINT(39.96808631192369 74.86631685009688) bank133560 +133561 POINT(41.412484092758994 74.80571867425839) bank133561 +133562 POINT(40.57251220731424 73.9334950354605) bank133562 +133563 POINT(40.576006607967926 74.26957686437973) bank133563 +133564 POINT(39.720777043419055 73.88722873977653) bank133564 +133565 POINT(41.513844233370286 74.85204154706742) bank133565 +133566 POINT(40.18333542587269 73.85972342708031) bank133566 +133567 POINT(41.54814225021844 73.70853298883002) bank133567 +133568 POINT(40.79282325915208 74.75501482034863) bank133568 +133569 POINT(40.16736775926453 74.05658658592195) bank133569 +133570 POINT(40.02304539919766 74.75058000826331) bank133570 +133571 POINT(41.20764140507802 74.72253165372527) bank133571 +133572 POINT(41.205843507102244 73.90553928416045) bank133572 +133573 POINT(40.5507528756552 74.06074276076211) bank133573 +133574 POINT(40.92454735969989 74.03903692978511) bank133574 +133575 POINT(41.072950598928564 74.50640060489422) bank133575 +133576 POINT(41.13694688650994 73.76989308995229) bank133576 +133577 POINT(40.015448924611135 73.77421915742318) bank133577 +133578 POINT(40.611834300669265 74.82042470226516) bank133578 +133579 POINT(41.07912077903747 73.24769861369374) bank133579 +133580 POINT(39.99922954588426 73.85255564699504) bank133580 +133581 POINT(40.04257491395345 74.2182036265504) bank133581 +133582 POINT(41.42599184550983 74.74263392262873) bank133582 +133583 POINT(39.732335007059106 73.10291702049075) bank133583 +133584 POINT(41.04173793318615 74.82540818056503) bank133584 +133585 POINT(41.129505075974166 73.99939273465318) bank133585 +133586 POINT(40.643489278503516 74.70469070748845) bank133586 +133587 POINT(41.61689681116781 74.01563624673113) bank133587 +133588 POINT(40.0768221812104 73.28919996451734) bank133588 +133589 POINT(41.32071922897088 74.88705538923644) bank133589 +133590 POINT(40.97411707304534 73.78853667403011) bank133590 +133591 POINT(40.11625250388932 73.55084953009677) bank133591 +133592 POINT(40.79338116148702 74.75526316643109) bank133592 +133593 POINT(40.15648930148355 73.18608556852297) bank133593 +133594 POINT(41.094101229572864 74.69464976017034) bank133594 +133595 POINT(40.52149235267347 73.5945923900462) bank133595 +133596 POINT(40.339308212153384 74.12613874211928) bank133596 +133597 POINT(40.788094429831304 74.85930100575219) bank133597 +133598 POINT(40.41269112195645 73.71147152836853) bank133598 +133599 POINT(40.29510093166434 73.53573577054537) bank133599 +133600 POINT(41.544922160707785 74.45186784287195) bank133600 +133601 POINT(40.81141178222155 74.86690377606271) bank133601 +133602 POINT(39.90842621652219 73.30361162116748) bank133602 +133603 POINT(40.933046721446665 73.83311129194905) bank133603 +133604 POINT(40.64029763413887 74.72137501465366) bank133604 +133605 POINT(39.808769230993654 74.19877607355932) bank133605 +133606 POINT(40.403187593514794 73.94430639845848) bank133606 +133607 POINT(39.772469639324555 73.57779078314877) bank133607 +133608 POINT(41.14403988245757 73.90800567061608) bank133608 +133609 POINT(40.98124003502322 74.706450340869) bank133609 +133610 POINT(40.93546197877562 74.12047080756686) bank133610 +133611 POINT(40.53049157150447 74.08192726766612) bank133611 +133612 POINT(41.29467089472582 73.98175192078564) bank133612 +133613 POINT(41.10367201521394 74.7723672835613) bank133613 +133614 POINT(41.611353214062866 73.43440140841936) bank133614 +133615 POINT(41.133372099172576 74.00977130871668) bank133615 +133616 POINT(40.05240756503092 73.21666087782239) bank133616 +133617 POINT(41.19346520349984 74.66449166212196) bank133617 +133618 POINT(40.40905985714773 73.05292288385522) bank133618 +133619 POINT(41.63534396318156 74.94515305846966) bank133619 +133620 POINT(41.545759425946585 73.76507002778295) bank133620 +133621 POINT(40.43567354859068 73.61038108828774) bank133621 +133622 POINT(40.99118707016606 73.61650471459065) bank133622 +133623 POINT(41.37846223111098 73.1749001377929) bank133623 +133624 POINT(39.8679009207048 73.63304051028328) bank133624 +133625 POINT(40.25050219573736 73.58526998328351) bank133625 +133626 POINT(40.21809826203158 73.84143186694364) bank133626 +133627 POINT(40.39272134932172 73.28866309024136) bank133627 +133628 POINT(41.10969547549238 73.36083981574194) bank133628 +133629 POINT(41.29540991297069 73.1151868241261) bank133629 +133630 POINT(41.118450179333436 73.56966672005292) bank133630 +133631 POINT(40.890764029618936 73.35705600780925) bank133631 +133632 POINT(40.57319057840615 74.36102475323005) bank133632 +133633 POINT(41.385421479633926 74.6992357832825) bank133633 +133634 POINT(41.37319821943194 74.30591919493271) bank133634 +133635 POINT(41.43474291442569 74.00220853960086) bank133635 +133636 POINT(41.229216285434234 74.37602894756346) bank133636 +133637 POINT(41.42540213891297 74.98126357439135) bank133637 +133638 POINT(41.1659974388849 74.63988018782666) bank133638 +133639 POINT(40.52852973630611 73.81265105553828) bank133639 +133640 POINT(41.64333231140843 73.12958752200237) bank133640 +133641 POINT(40.18525060794662 73.89539804823458) bank133641 +133642 POINT(39.999646084411225 74.14066880614722) bank133642 +133643 POINT(41.03196117160228 73.49704671119574) bank133643 +133644 POINT(39.76264619445564 74.34766332746055) bank133644 +133645 POINT(41.5166380244491 73.94269642076155) bank133645 +133646 POINT(39.80194100372881 74.29830981585259) bank133646 +133647 POINT(40.76642382913672 74.48464006998049) bank133647 +133648 POINT(40.70428265010338 74.58999784735104) bank133648 +133649 POINT(40.02776089270993 73.34181373351849) bank133649 +133650 POINT(41.642014632300324 74.88804956342659) bank133650 +133651 POINT(40.51838645203456 73.09337904022048) bank133651 +133652 POINT(41.30552592571863 73.1472250591663) bank133652 +133653 POINT(40.323547538658694 75.00246748184422) bank133653 +133654 POINT(40.12088266063614 73.34661537281843) bank133654 +133655 POINT(41.2339575934641 73.12782581000216) bank133655 +133656 POINT(40.15908052822868 74.50464567975635) bank133656 +133657 POINT(41.21781736132472 74.99631432947959) bank133657 +133658 POINT(41.588862523375425 74.73746571971232) bank133658 +133659 POINT(39.78498504785276 73.96672342625362) bank133659 +133660 POINT(40.35217719671633 74.67576519170692) bank133660 +133661 POINT(40.45188833713666 74.55023184278406) bank133661 +133662 POINT(40.20943536145847 73.51290456689462) bank133662 +133663 POINT(40.55823950995174 74.62787920747674) bank133663 +133664 POINT(40.01423342745756 73.32729944986355) bank133664 +133665 POINT(40.10053387251461 74.35071569859996) bank133665 +133666 POINT(40.935045983882794 74.74088763447148) bank133666 +133667 POINT(41.64918032385002 73.16277786542135) bank133667 +133668 POINT(41.66080500603409 74.54447342059888) bank133668 +133669 POINT(41.543975151201415 74.05803826572354) bank133669 +133670 POINT(40.602834062731276 73.87568514617504) bank133670 +133671 POINT(40.47589362846691 74.3686909122681) bank133671 +133672 POINT(41.647418821153806 74.1553170598974) bank133672 +133673 POINT(39.97666116139248 73.47166194464357) bank133673 +133674 POINT(40.95865080926807 74.40773362880104) bank133674 +133675 POINT(41.440951244884324 74.49086892797501) bank133675 +133676 POINT(40.80944183512168 74.40124557325916) bank133676 +133677 POINT(39.77555228517499 74.66545227658862) bank133677 +133678 POINT(41.451182122249335 74.33836394932771) bank133678 +133679 POINT(40.94882917558432 74.85505290636894) bank133679 +133680 POINT(39.93532873143188 74.77407316479547) bank133680 +133681 POINT(41.32390269953525 73.19532468799648) bank133681 +133682 POINT(41.00278177812052 74.01824855696077) bank133682 +133683 POINT(40.52676648702897 73.39100356085116) bank133683 +133684 POINT(40.83874136290194 74.56755640595632) bank133684 +133685 POINT(39.7283026231389 74.34648154939039) bank133685 +133686 POINT(40.406924479045074 74.17736403401297) bank133686 +133687 POINT(40.278543400045635 74.62584844285642) bank133687 +133688 POINT(40.28819735039123 74.84728759027219) bank133688 +133689 POINT(39.757620396428656 74.68349806160415) bank133689 +133690 POINT(41.65340763135435 74.17940771877967) bank133690 +133691 POINT(40.57790963463314 74.25357906867639) bank133691 +133692 POINT(39.73805996950171 73.16091052921263) bank133692 +133693 POINT(40.89863036881244 73.31678520415167) bank133693 +133694 POINT(40.28871148721292 74.0294201820164) bank133694 +133695 POINT(40.554237162087226 74.87095858507658) bank133695 +133696 POINT(41.65869435649643 73.98653054781533) bank133696 +133697 POINT(40.251542131586206 73.46741538988591) bank133697 +133698 POINT(40.72746493472194 73.15853929641223) bank133698 +133699 POINT(40.54750555574743 74.26570678285925) bank133699 +133700 POINT(40.84837633530169 74.81211039728088) bank133700 +133701 POINT(41.55857492037233 74.49324485922862) bank133701 +133702 POINT(40.55723408474149 74.0268512188208) bank133702 +133703 POINT(39.77900404376509 74.68118692739564) bank133703 +133704 POINT(40.679207422788444 74.52414309003163) bank133704 +133705 POINT(41.67206269632697 74.26462462910573) bank133705 +133706 POINT(41.38379881241104 73.46183256794262) bank133706 +133707 POINT(40.72060562820309 73.02425825317232) bank133707 +133708 POINT(40.216679729712546 74.13235718760806) bank133708 +133709 POINT(41.349003329660405 73.28523883956777) bank133709 +133710 POINT(39.91418474068597 74.08611384807138) bank133710 +133711 POINT(40.83094103619224 74.54832551809456) bank133711 +133712 POINT(39.919874661241785 73.19397365286734) bank133712 +133713 POINT(40.03096944120712 74.12509919469153) bank133713 +133714 POINT(39.766638990380585 73.07347002754238) bank133714 +133715 POINT(40.194517622533766 73.93054319494186) bank133715 +133716 POINT(41.5353795252981 73.86423823844318) bank133716 +133717 POINT(39.73096812605399 74.84463475836402) bank133717 +133718 POINT(41.44904366212452 74.15042056898301) bank133718 +133719 POINT(41.28120084386787 74.15622063339029) bank133719 +133720 POINT(41.39643828335087 75.0020119958902) bank133720 +133721 POINT(40.79727886966557 73.22104867712844) bank133721 +133722 POINT(39.89186696827161 74.10429640250258) bank133722 +133723 POINT(40.3540843974181 73.09058033349625) bank133723 +133724 POINT(40.37811865076229 73.06939697858505) bank133724 +133725 POINT(40.883328409048175 73.14322567477653) bank133725 +133726 POINT(41.386920326936185 73.92401355051408) bank133726 +133727 POINT(39.76088592213881 73.82206001265224) bank133727 +133728 POINT(41.01078692076937 73.84100898940798) bank133728 +133729 POINT(40.20595449590744 74.39812188062079) bank133729 +133730 POINT(40.519954691563875 73.19346049575793) bank133730 +133731 POINT(40.33578197340045 73.91953353950014) bank133731 +133732 POINT(41.534751835375296 73.65352663673207) bank133732 +133733 POINT(41.26616274323723 74.651444517971) bank133733 +133734 POINT(40.18398422600302 74.37567959573128) bank133734 +133735 POINT(39.871040511942645 74.6032805919027) bank133735 +133736 POINT(40.25680817819261 73.84957956694204) bank133736 +133737 POINT(40.33025533313048 74.03830167428488) bank133737 +133738 POINT(39.87523521161258 73.68739059475206) bank133738 +133739 POINT(41.32350483194912 73.32053476090468) bank133739 +133740 POINT(40.7672511682817 74.92799322889731) bank133740 +133741 POINT(41.39655738854387 73.11367612047803) bank133741 +133742 POINT(40.906192912398275 73.87833027639925) bank133742 +133743 POINT(41.488035774717765 73.63764978716114) bank133743 +133744 POINT(40.38097639469332 73.50971313773644) bank133744 +133745 POINT(40.349640062493 74.75802087129337) bank133745 +133746 POINT(40.9750151056791 74.59521697040073) bank133746 +133747 POINT(40.00507536861457 74.11980431795405) bank133747 +133748 POINT(41.35648945889809 73.29926099796609) bank133748 +133749 POINT(40.78089163852757 74.09476500881357) bank133749 +133750 POINT(40.92420841119885 73.3548852318308) bank133750 +133751 POINT(40.8079483907343 73.5068716230811) bank133751 +133752 POINT(40.66849985904032 74.38647373075663) bank133752 +133753 POINT(41.4023417980028 73.45902826443566) bank133753 +133754 POINT(41.09874039985806 74.23310816071435) bank133754 +133755 POINT(41.66320754479919 74.01105481366051) bank133755 +133756 POINT(40.46567532234776 73.62723226925704) bank133756 +133757 POINT(39.93304217789937 74.32418146949645) bank133757 +133758 POINT(40.28745282151837 73.00661463786209) bank133758 +133759 POINT(40.7608953015601 74.72040825160585) bank133759 +133760 POINT(41.2059294750809 74.03398468161195) bank133760 +133761 POINT(40.22763037304529 74.61384695499379) bank133761 +133762 POINT(41.3416321142162 73.54649342285974) bank133762 +133763 POINT(40.422647169257594 73.58620245213896) bank133763 +133764 POINT(41.369387811711384 74.59014830006407) bank133764 +133765 POINT(39.88987635869603 74.71326379041497) bank133765 +133766 POINT(41.12553768374574 73.74631453440419) bank133766 +133767 POINT(41.40477019455851 74.96648990695627) bank133767 +133768 POINT(41.184271524320096 74.17221259028794) bank133768 +133769 POINT(40.488270133306436 74.49859267762254) bank133769 +133770 POINT(40.318962219517175 73.55675097878698) bank133770 +133771 POINT(40.346673087536374 74.60337537943991) bank133771 +133772 POINT(41.27396378776604 74.59723482263924) bank133772 +133773 POINT(41.32338806121447 73.26210591845242) bank133773 +133774 POINT(41.253101350218984 74.93353716880083) bank133774 +133775 POINT(40.75720957999088 74.02068330295678) bank133775 +133776 POINT(39.802610032083955 73.3092907357991) bank133776 +133777 POINT(39.82803732608746 74.97217487449798) bank133777 +133778 POINT(40.45592702006692 74.11738772659284) bank133778 +133779 POINT(41.55504148026176 74.87928567509543) bank133779 +133780 POINT(40.469588391417865 73.33512364568554) bank133780 +133781 POINT(39.822862832269834 73.21426203514172) bank133781 +133782 POINT(40.00117888594855 73.58546266939334) bank133782 +133783 POINT(40.07980181051345 73.22591197249893) bank133783 +133784 POINT(40.7146486808036 73.10127517833224) bank133784 +133785 POINT(40.27481087444484 73.74922808161635) bank133785 +133786 POINT(40.366917145022505 73.04786343350138) bank133786 +133787 POINT(41.05659845572999 74.34367057629315) bank133787 +133788 POINT(41.32790340690647 73.06319966299588) bank133788 +133789 POINT(39.88632928160207 74.39860443472236) bank133789 +133790 POINT(41.32099291821569 74.47078304837615) bank133790 +133791 POINT(40.405394358347806 73.92651695353521) bank133791 +133792 POINT(41.662046905526665 73.62054206724954) bank133792 +133793 POINT(40.20869402515159 74.60268612614662) bank133793 +133794 POINT(39.813600495703625 73.51543140321364) bank133794 +133795 POINT(41.17611658298144 73.99746438294196) bank133795 +133796 POINT(41.326060099764256 74.1461277425742) bank133796 +133797 POINT(40.88745784698479 74.28880671093941) bank133797 +133798 POINT(41.116411754330436 74.38982682989244) bank133798 +133799 POINT(40.59741188441453 73.17879785737514) bank133799 +133800 POINT(41.62972030960521 74.72881057060484) bank133800 +133801 POINT(39.78026832287639 74.6716717941258) bank133801 +133802 POINT(40.95954962709598 73.14038926529737) bank133802 +133803 POINT(39.7626689055463 73.12855589598814) bank133803 +133804 POINT(40.51010856225535 74.0048652607659) bank133804 +133805 POINT(41.582318113572455 73.47094502283502) bank133805 +133806 POINT(41.34780975314858 73.57331981520863) bank133806 +133807 POINT(40.61187149130363 74.42063835740645) bank133807 +133808 POINT(40.40097561669264 73.39901743358008) bank133808 +133809 POINT(40.142692887731855 73.86198063386578) bank133809 +133810 POINT(40.187171887285416 73.2733214044307) bank133810 +133811 POINT(40.87489047109811 73.83434926141666) bank133811 +133812 POINT(40.20281578793476 74.99253223750786) bank133812 +133813 POINT(41.16274944771173 74.96734877794273) bank133813 +133814 POINT(40.71077884061 73.21505602114173) bank133814 +133815 POINT(39.96749885621721 73.96977158321701) bank133815 +133816 POINT(39.991326087039646 73.1617255494369) bank133816 +133817 POINT(40.353059786361115 73.16266760125349) bank133817 +133818 POINT(39.744207400649245 74.3712146308633) bank133818 +133819 POINT(40.96719732194997 73.35724592541258) bank133819 +133820 POINT(40.927268819736014 73.71778882576946) bank133820 +133821 POINT(41.24367074460155 74.9029586107697) bank133821 +133822 POINT(40.32574180258339 73.9643526281119) bank133822 +133823 POINT(40.11288541055714 73.32255585702706) bank133823 +133824 POINT(40.965229096083554 73.1190240253597) bank133824 +133825 POINT(40.3612623854395 73.62090046347602) bank133825 +133826 POINT(40.77351481560471 73.57973387586419) bank133826 +133827 POINT(41.568155668052135 74.82532813662833) bank133827 +133828 POINT(40.456703576814384 74.9942116541454) bank133828 +133829 POINT(40.975569484026266 73.30451318303443) bank133829 +133830 POINT(40.32401316295982 74.7729334620381) bank133830 +133831 POINT(40.81510686059041 73.37550790331439) bank133831 +133832 POINT(41.58403976505914 74.30430903281339) bank133832 +133833 POINT(40.43693492960909 74.78401150539831) bank133833 +133834 POINT(40.214986618179836 73.6249328558481) bank133834 +133835 POINT(40.85131089752246 74.09703864756544) bank133835 +133836 POINT(41.67413541756603 74.08301573338917) bank133836 +133837 POINT(40.81458486484673 74.79680031190023) bank133837 +133838 POINT(41.325395599327265 73.01600721990312) bank133838 +133839 POINT(41.52684041682427 74.98592781287446) bank133839 +133840 POINT(40.7155165219074 73.18185388332303) bank133840 +133841 POINT(41.15747481104749 73.10910884558528) bank133841 +133842 POINT(40.86014967400715 73.86771341689678) bank133842 +133843 POINT(40.05294717499295 73.50235537542127) bank133843 +133844 POINT(40.22059037749159 73.14571037463953) bank133844 +133845 POINT(40.32503092240636 73.89065311677007) bank133845 +133846 POINT(41.02056134695859 74.5515021951171) bank133846 +133847 POINT(40.06763867384565 74.65123176930939) bank133847 +133848 POINT(40.416214403207285 74.98032642816194) bank133848 +133849 POINT(40.764348816659925 73.72091545394375) bank133849 +133850 POINT(40.127090018874576 73.59264913509935) bank133850 +133851 POINT(41.38414260051558 74.84874471772295) bank133851 +133852 POINT(41.16578744930146 73.76794320559968) bank133852 +133853 POINT(39.77216897923254 73.07502350764875) bank133853 +133854 POINT(41.20754922026872 73.12909512461889) bank133854 +133855 POINT(40.01891863634229 74.67844562495368) bank133855 +133856 POINT(41.06577699798573 74.60317849467069) bank133856 +133857 POINT(40.39352061864003 73.1961277031189) bank133857 +133858 POINT(41.22464026811097 74.16028486269056) bank133858 +133859 POINT(40.266082206986844 74.40213044202818) bank133859 +133860 POINT(41.16781959017291 73.0644738401222) bank133860 +133861 POINT(40.99067561379052 74.355338667476) bank133861 +133862 POINT(40.971557889335784 74.59854762031716) bank133862 +133863 POINT(41.35019859634589 73.0680192290573) bank133863 +133864 POINT(40.59892529956638 73.18448428093845) bank133864 +133865 POINT(40.57583638148542 73.14008115795514) bank133865 +133866 POINT(41.61316370084133 74.4698927295966) bank133866 +133867 POINT(40.52581471711739 74.90300233804572) bank133867 +133868 POINT(41.27982737602742 74.07707539976323) bank133868 +133869 POINT(40.308807174847594 74.53351425045827) bank133869 +133870 POINT(40.9547656756591 74.58328243100408) bank133870 +133871 POINT(40.33843944106009 73.48120947911637) bank133871 +133872 POINT(41.413946634488155 74.20102659301598) bank133872 +133873 POINT(40.83949145692647 74.88016883717093) bank133873 +133874 POINT(40.845888524753924 73.96901406520526) bank133874 +133875 POINT(41.297547957562614 73.64242699376524) bank133875 +133876 POINT(40.93078252842257 74.75784027253626) bank133876 +133877 POINT(39.71908758804619 74.07453941776538) bank133877 +133878 POINT(40.03573174072585 73.88965112475974) bank133878 +133879 POINT(40.69649019756961 74.2551473468373) bank133879 +133880 POINT(40.76925559753287 74.3316725775685) bank133880 +133881 POINT(41.3209988882104 73.34516623591904) bank133881 +133882 POINT(41.58575980395453 74.81614436864106) bank133882 +133883 POINT(40.40246481574376 73.83429922511104) bank133883 +133884 POINT(40.6073340366572 73.3380314694202) bank133884 +133885 POINT(40.01477971424123 73.94568159506326) bank133885 +133886 POINT(40.30190482170145 73.24773279058728) bank133886 +133887 POINT(40.6850983909189 73.22007890292612) bank133887 +133888 POINT(40.04569234239789 73.94130365070761) bank133888 +133889 POINT(40.102227499128176 73.60204877086585) bank133889 +133890 POINT(41.01857984479438 73.04625070362238) bank133890 +133891 POINT(41.472137950170314 74.07858684104923) bank133891 +133892 POINT(40.60088875700869 74.66737200058283) bank133892 +133893 POINT(40.63441506458731 74.87623673103495) bank133893 +133894 POINT(40.68573042068548 74.88112645913981) bank133894 +133895 POINT(40.402281689258466 73.21830261971128) bank133895 +133896 POINT(40.14097764082306 74.87922152928864) bank133896 +133897 POINT(40.69896987634383 73.18498793502587) bank133897 +133898 POINT(39.7714856691438 74.44661466758204) bank133898 +133899 POINT(40.50353987162095 73.02496700629897) bank133899 +133900 POINT(41.42135833612387 74.88659286482118) bank133900 +133901 POINT(41.329319637861964 73.72509220535535) bank133901 +133902 POINT(41.25911467596772 73.59763469507186) bank133902 +133903 POINT(41.197828021536566 74.58885843990105) bank133903 +133904 POINT(40.882994642444864 74.98437457977879) bank133904 +133905 POINT(41.27500336023628 74.2932759247244) bank133905 +133906 POINT(39.752668327679785 73.25669323346673) bank133906 +133907 POINT(39.97814134002706 74.53694982120932) bank133907 +133908 POINT(40.87974783505074 74.53718815525235) bank133908 +133909 POINT(40.45560328125659 73.92236487808154) bank133909 +133910 POINT(41.39944080750773 73.5793706925611) bank133910 +133911 POINT(39.98516782110851 74.40895905678805) bank133911 +133912 POINT(41.32654409430933 74.57542773156678) bank133912 +133913 POINT(41.396831976998115 73.11948168698571) bank133913 +133914 POINT(40.667579422692874 73.68255274187551) bank133914 +133915 POINT(40.99911707549478 73.6029724692783) bank133915 +133916 POINT(41.29395731840346 73.94255067843179) bank133916 +133917 POINT(41.70619019791043 74.22660846789655) bank133917 +133918 POINT(41.332303108119106 73.8245571705097) bank133918 +133919 POINT(39.77128299284994 74.44129802694593) bank133919 +133920 POINT(39.758364608409394 73.03571024329763) bank133920 +133921 POINT(40.66422026674501 73.49775712625583) bank133921 +133922 POINT(41.0061643702605 74.73973416583335) bank133922 +133923 POINT(40.812228388429745 73.8465398765341) bank133923 +133924 POINT(41.04254626286603 74.17144891247845) bank133924 +133925 POINT(40.380770464860205 74.91101732134712) bank133925 +133926 POINT(41.598568711759576 73.03778531351888) bank133926 +133927 POINT(41.44844246286971 73.71731225643393) bank133927 +133928 POINT(40.65048879147205 74.06928730340921) bank133928 +133929 POINT(39.86720646731297 74.55788775555062) bank133929 +133930 POINT(40.014775375235956 73.79743985245898) bank133930 +133931 POINT(40.70517520738881 74.99260449335375) bank133931 +133932 POINT(39.71566669210748 73.51059701390461) bank133932 +133933 POINT(40.00304232900324 73.38170351084412) bank133933 +133934 POINT(40.66253493355973 74.27242821494148) bank133934 +133935 POINT(41.09184756782095 73.09482731716471) bank133935 +133936 POINT(41.1595621637884 73.49948216073379) bank133936 +133937 POINT(40.55206600537317 74.50364769533468) bank133937 +133938 POINT(41.17277505504998 74.85145200615283) bank133938 +133939 POINT(39.79095790743301 73.10486857610381) bank133939 +133940 POINT(39.972429619393765 73.08795652781015) bank133940 +133941 POINT(40.965852206849306 74.20426069635937) bank133941 +133942 POINT(39.94707014664929 74.37682801768487) bank133942 +133943 POINT(39.804121371027016 73.95064051702546) bank133943 +133944 POINT(40.26057112099696 73.74587115557995) bank133944 +133945 POINT(40.633260170364395 74.43351139331804) bank133945 +133946 POINT(40.02418591759047 73.20397012227052) bank133946 +133947 POINT(41.11338294504215 74.18806489486435) bank133947 +133948 POINT(40.833011588059264 73.784195287364) bank133948 +133949 POINT(40.73461723306703 74.01997166023908) bank133949 +133950 POINT(40.13993792714846 73.06492952340884) bank133950 +133951 POINT(40.00970147524922 73.37017713514702) bank133951 +133952 POINT(40.45787228088676 74.84030153735765) bank133952 +133953 POINT(40.789467282972815 73.54328296335838) bank133953 +133954 POINT(40.682484472764 73.13504961628132) bank133954 +133955 POINT(40.5839043468543 74.45859233948104) bank133955 +133956 POINT(41.361480786418184 73.43008594668241) bank133956 +133957 POINT(39.737554986472325 73.0105056913158) bank133957 +133958 POINT(40.46670092938576 74.02385854908552) bank133958 +133959 POINT(40.71778844265093 73.30642022571588) bank133959 +133960 POINT(40.27534217496796 73.864074422803) bank133960 +133961 POINT(40.99119922734695 75.00472401944712) bank133961 +133962 POINT(41.51917798215575 73.90066379624216) bank133962 +133963 POINT(39.7362327072991 73.72435163334596) bank133963 +133964 POINT(40.31820432710953 73.6899996590705) bank133964 +133965 POINT(40.77646251851191 73.90715622915509) bank133965 +133966 POINT(40.439836174751534 73.24624685100478) bank133966 +133967 POINT(41.520709420114166 74.97806428880169) bank133967 +133968 POINT(40.99235061294687 73.09475179699986) bank133968 +133969 POINT(40.07885676236573 74.34539919717284) bank133969 +133970 POINT(41.21291184730779 74.89684322515852) bank133970 +133971 POINT(40.72049857024463 74.06186054744715) bank133971 +133972 POINT(40.81169210300389 74.53997705493643) bank133972 +133973 POINT(39.716447983560535 73.80179956647592) bank133973 +133974 POINT(40.2757050259199 74.48645223209793) bank133974 +133975 POINT(41.27817174371444 74.14282329463893) bank133975 +133976 POINT(41.08411051257402 74.06462583147743) bank133976 +133977 POINT(40.74053740594892 73.53185569829465) bank133977 +133978 POINT(40.12256555883906 74.75249103808937) bank133978 +133979 POINT(40.19476749298421 74.95317524839554) bank133979 +133980 POINT(40.56776013174987 73.39521238985779) bank133980 +133981 POINT(41.28519530777895 73.84510492543265) bank133981 +133982 POINT(39.811476425885175 74.09642509925261) bank133982 +133983 POINT(41.12844788379049 73.6892650203306) bank133983 +133984 POINT(41.27959050587473 74.45505416006823) bank133984 +133985 POINT(41.2119059400498 73.3573568121283) bank133985 +133986 POINT(41.11573781900182 73.71026249712106) bank133986 +133987 POINT(39.82586713183868 74.53080147100752) bank133987 +133988 POINT(40.56003727740637 73.15057591355892) bank133988 +133989 POINT(41.67112446146072 74.28813601472629) bank133989 +133990 POINT(40.38305820125865 74.70809912863176) bank133990 +133991 POINT(40.52186626150193 74.72064033617288) bank133991 +133992 POINT(39.730702980958874 73.91106557892559) bank133992 +133993 POINT(39.90639730092127 73.61100560958884) bank133993 +133994 POINT(40.96227944009931 73.10480847655835) bank133994 +133995 POINT(40.18222796151292 74.8408107869165) bank133995 +133996 POINT(40.972103186326976 73.03009714780201) bank133996 +133997 POINT(41.13275987503949 73.71967253518108) bank133997 +133998 POINT(40.47208575513861 73.72181567657864) bank133998 +133999 POINT(41.46344860840492 74.17545710242791) bank133999 +134000 POINT(39.95737012669371 73.73931184144958) bank134000 +134001 POINT(40.764336768886736 74.29878340397286) bank134001 +134002 POINT(40.51706997531331 73.85903339643117) bank134002 +134003 POINT(41.11413703915027 73.01997215407852) bank134003 +134004 POINT(39.87407713353426 73.49629589618596) bank134004 +134005 POINT(40.67638834840013 73.6800765048269) bank134005 +134006 POINT(41.37787956956636 74.9492994323925) bank134006 +134007 POINT(40.27480236558424 74.10460486601056) bank134007 +134008 POINT(40.578164795747405 74.25407390749044) bank134008 +134009 POINT(41.49615359411743 73.97332765041091) bank134009 +134010 POINT(40.28997501032129 73.47323713219552) bank134010 +134011 POINT(40.15539914579367 73.42214956379055) bank134011 +134012 POINT(40.81685416438926 73.739145685681) bank134012 +134013 POINT(40.99877883424671 74.99141326192401) bank134013 +134014 POINT(41.42996596166475 74.76042657691) bank134014 +134015 POINT(41.36777448643756 73.09295474803503) bank134015 +134016 POINT(41.25968910480729 74.95266752394267) bank134016 +134017 POINT(41.270935285759116 73.82416300017591) bank134017 +134018 POINT(41.11217048910901 74.59976978887494) bank134018 +134019 POINT(40.614626933818656 74.98359534943042) bank134019 +134020 POINT(40.05693569201574 73.72972376881845) bank134020 +134021 POINT(40.17633507217841 74.73558131338889) bank134021 +134022 POINT(40.43954620210921 74.14094278527529) bank134022 +134023 POINT(40.06688153478419 74.35812007288025) bank134023 +134024 POINT(40.57526859037206 73.70322958991264) bank134024 +134025 POINT(39.951225706225834 73.36581325229827) bank134025 +134026 POINT(41.36106810325226 73.61440911469023) bank134026 +134027 POINT(40.50489311545705 73.73175328593622) bank134027 +134028 POINT(40.63697840007775 73.35942132596136) bank134028 +134029 POINT(41.0919171306421 74.66003752945068) bank134029 +134030 POINT(41.31706580499997 74.6932391731493) bank134030 +134031 POINT(40.3919922897913 74.14533387537642) bank134031 +134032 POINT(40.50487549133748 74.43078038367221) bank134032 +134033 POINT(40.138528190125726 73.56610100089505) bank134033 +134034 POINT(40.5956944084665 74.81048880977924) bank134034 +134035 POINT(40.3353664854319 74.92025221665462) bank134035 +134036 POINT(41.51923795939718 73.41519759709679) bank134036 +134037 POINT(40.31682594397609 74.3768667820778) bank134037 +134038 POINT(40.57623464537761 74.75203482803569) bank134038 +134039 POINT(41.304153752761216 74.86564253968344) bank134039 +134040 POINT(40.87308611595865 73.03254454018898) bank134040 +134041 POINT(40.84380130656314 73.47742032053812) bank134041 +134042 POINT(40.60734229979313 73.31267630489783) bank134042 +134043 POINT(40.3352152664755 74.39726582169033) bank134043 +134044 POINT(41.374963279129034 73.33222544205854) bank134044 +134045 POINT(40.5984202545662 73.49710530831427) bank134045 +134046 POINT(40.93838402994044 73.29615324923415) bank134046 +134047 POINT(40.702391675180415 73.7050631298527) bank134047 +134048 POINT(40.837900839726714 73.03638715060873) bank134048 +134049 POINT(40.93032484288626 73.36856443482678) bank134049 +134050 POINT(41.39110437164303 73.51560275972147) bank134050 +134051 POINT(40.71733457269191 73.74837863999993) bank134051 +134052 POINT(40.24861498067844 74.2887285169042) bank134052 +134053 POINT(40.62374181856627 74.59911373656317) bank134053 +134054 POINT(41.68102392101602 73.60785558326641) bank134054 +134055 POINT(40.01551280535699 73.75381719356322) bank134055 +134056 POINT(39.7558848900959 74.20420043212981) bank134056 +134057 POINT(40.39682082188958 73.96261989721448) bank134057 +134058 POINT(40.03561777405844 74.09669824378665) bank134058 +134059 POINT(40.98984477591976 74.29964350168044) bank134059 +134060 POINT(40.191650422425056 74.52582355560548) bank134060 +134061 POINT(40.12800335248635 74.76343173493107) bank134061 +134062 POINT(40.5156250089983 73.43026848583028) bank134062 +134063 POINT(40.246309585005605 74.01041313117335) bank134063 +134064 POINT(40.07353592543596 74.13217586554948) bank134064 +134065 POINT(41.69237778645524 73.1619476952694) bank134065 +134066 POINT(41.4484370957345 73.90612249952709) bank134066 +134067 POINT(40.031725350177766 73.57243921891946) bank134067 +134068 POINT(41.65222170297233 73.49263199350673) bank134068 +134069 POINT(40.5128234809504 73.78926009273565) bank134069 +134070 POINT(40.036934804116626 74.28042068706799) bank134070 +134071 POINT(41.28237870034999 73.74007372392569) bank134071 +134072 POINT(40.11939580478026 73.48251299785058) bank134072 +134073 POINT(40.343568142476705 73.03226817348762) bank134073 +134074 POINT(39.80713794697265 73.21972877331876) bank134074 +134075 POINT(39.71908813511341 74.46825370441326) bank134075 +134076 POINT(40.975777059764724 73.16685071578075) bank134076 +134077 POINT(40.56122865526571 73.15402881297484) bank134077 +134078 POINT(41.594721995956476 73.25242066980526) bank134078 +134079 POINT(39.82599170739377 74.42405710381455) bank134079 +134080 POINT(41.493676582782484 74.77419328774512) bank134080 +134081 POINT(39.90107721003066 73.1082299536685) bank134081 +134082 POINT(40.863573720486755 74.19930307051021) bank134082 +134083 POINT(40.29825418659692 74.69719963575574) bank134083 +134084 POINT(40.19020630382338 73.78110908816207) bank134084 +134085 POINT(41.64504086810886 74.69357668487852) bank134085 +134086 POINT(40.67164767796314 73.69054133582648) bank134086 +134087 POINT(41.26141901395967 73.60961193765823) bank134087 +134088 POINT(41.651402934133074 73.0666513948362) bank134088 +134089 POINT(41.65137015635036 74.31244846922972) bank134089 +134090 POINT(40.18219255119537 73.7596220328308) bank134090 +134091 POINT(41.51903255041025 73.031489677077) bank134091 +134092 POINT(41.01760907493058 73.35433937381623) bank134092 +134093 POINT(41.62712321324304 74.08000634972773) bank134093 +134094 POINT(40.62176615821212 74.8428227145931) bank134094 +134095 POINT(39.79017981505879 74.95816224798313) bank134095 +134096 POINT(40.378855940801095 73.05763405310941) bank134096 +134097 POINT(39.814590131037015 73.05837292731351) bank134097 +134098 POINT(40.836492076837274 74.11588182091342) bank134098 +134099 POINT(41.23913845110489 73.59528526428804) bank134099 +134100 POINT(40.82420939274464 73.4931279387082) bank134100 +134101 POINT(40.79002604777094 73.8642613378308) bank134101 +134102 POINT(40.03615625325581 73.7462385773065) bank134102 +134103 POINT(39.82496643908358 73.61838720125544) bank134103 +134104 POINT(41.08772132470436 73.61488307953343) bank134104 +134105 POINT(41.37432380626717 73.88510408161434) bank134105 +134106 POINT(41.19823016588851 74.85655032541578) bank134106 +134107 POINT(39.85922541353622 73.36256233764502) bank134107 +134108 POINT(40.78240352617708 74.87506031375744) bank134108 +134109 POINT(40.010574665024 74.90055934162099) bank134109 +134110 POINT(41.48043082097235 73.04960917166733) bank134110 +134111 POINT(41.29730713460512 73.52977057665248) bank134111 +134112 POINT(40.96518625485548 73.18715484975084) bank134112 +134113 POINT(40.50174979700036 73.88365632492814) bank134113 +134114 POINT(41.69021395682574 73.99352536220847) bank134114 +134115 POINT(40.322570491454094 74.66723878800472) bank134115 +134116 POINT(41.046158429897694 74.41253269017021) bank134116 +134117 POINT(39.82262813542114 74.2876005380868) bank134117 +134118 POINT(40.85589793914251 73.48952457902242) bank134118 +134119 POINT(39.803982399349415 74.64593923655491) bank134119 +134120 POINT(40.16789237958776 74.64662396286255) bank134120 +134121 POINT(41.66397648199998 74.25010120675996) bank134121 +134122 POINT(39.85381904718032 74.3564097850001) bank134122 +134123 POINT(40.44078595561382 73.06080059599644) bank134123 +134124 POINT(41.17447388645701 74.18513142973835) bank134124 +134125 POINT(40.77123724186349 73.53700383620021) bank134125 +134126 POINT(40.97480989066648 74.48640016606731) bank134126 +134127 POINT(40.55711048423187 74.0070973379662) bank134127 +134128 POINT(41.61215906109395 73.70724917203698) bank134128 +134129 POINT(39.97631879112541 74.18704383234486) bank134129 +134130 POINT(40.0167857020473 74.33077456772793) bank134130 +134131 POINT(40.43667464450593 74.36853953596355) bank134131 +134132 POINT(40.822598325741765 73.60904148370037) bank134132 +134133 POINT(40.42229792315373 73.61544480216462) bank134133 +134134 POINT(40.313260873770616 74.42655528172882) bank134134 +134135 POINT(41.256916275005175 74.04159178142002) bank134135 +134136 POINT(40.29249856649377 74.82897637854856) bank134136 +134137 POINT(41.40179898990185 73.85290768096517) bank134137 +134138 POINT(40.30842175264308 73.34581160209518) bank134138 +134139 POINT(41.39571770485967 74.13765757484128) bank134139 +134140 POINT(40.04151009375596 73.2735706664382) bank134140 +134141 POINT(39.75523661500828 73.31798857034553) bank134141 +134142 POINT(40.13547115327857 74.70997723436712) bank134142 +134143 POINT(41.00408637661988 74.95636060704253) bank134143 +134144 POINT(40.54641617667045 74.44705360651838) bank134144 +134145 POINT(40.9205216063661 73.75357640055606) bank134145 +134146 POINT(41.1333152946434 73.61190764304186) bank134146 +134147 POINT(39.97421769906297 73.54181928403611) bank134147 +134148 POINT(40.73285346202698 73.8338287476256) bank134148 +134149 POINT(41.2844941092047 74.66608674169915) bank134149 +134150 POINT(40.71791471879077 74.60984581024387) bank134150 +134151 POINT(40.777620235221 73.44650637504029) bank134151 +134152 POINT(41.6880984058773 73.66586860272079) bank134152 +134153 POINT(41.53644299148287 74.39483509548026) bank134153 +134154 POINT(40.9345062112312 74.48000739397419) bank134154 +134155 POINT(39.93373873704989 73.52958462333604) bank134155 +134156 POINT(40.98669653612868 73.64157408921744) bank134156 +134157 POINT(40.558266965592146 74.68236774474835) bank134157 +134158 POINT(39.807955725005094 73.88635890324153) bank134158 +134159 POINT(41.45783494497203 74.74130247095131) bank134159 +134160 POINT(41.31057047034957 73.81622593240522) bank134160 +134161 POINT(40.687139760410616 73.75660051982297) bank134161 +134162 POINT(39.98638159583487 73.08603447252428) bank134162 +134163 POINT(40.10768324449969 74.9824707447695) bank134163 +134164 POINT(39.95002470533749 73.24253160996884) bank134164 +134165 POINT(40.16314789948206 74.7855799998123) bank134165 +134166 POINT(40.42700863414973 74.71584580617665) bank134166 +134167 POINT(41.01136319068062 74.02278721258413) bank134167 +134168 POINT(40.22523898068782 73.75784619494668) bank134168 +134169 POINT(39.757019554938054 74.26907174137025) bank134169 +134170 POINT(40.77075393387627 74.10423440568425) bank134170 +134171 POINT(41.421195274489065 74.53978691861856) bank134171 +134172 POINT(40.27446466817866 74.40590017489015) bank134172 +134173 POINT(40.44222014133208 73.84738507072083) bank134173 +134174 POINT(39.83170678654384 73.45988024598233) bank134174 +134175 POINT(41.3194071623214 74.29302460452435) bank134175 +134176 POINT(41.64183760986305 74.05266925098533) bank134176 +134177 POINT(41.32726360663704 74.53533477753189) bank134177 +134178 POINT(41.596251360111744 73.57317892583822) bank134178 +134179 POINT(39.96404336205041 74.80166487651564) bank134179 +134180 POINT(40.21944830909464 74.25617638969611) bank134180 +134181 POINT(40.58336677512385 74.31399915426202) bank134181 +134182 POINT(40.37543231987109 73.99713773316357) bank134182 +134183 POINT(40.79697531101342 74.82314103956614) bank134183 +134184 POINT(40.859360769172035 74.73181277194142) bank134184 +134185 POINT(39.84424215332149 73.49381778931308) bank134185 +134186 POINT(41.285166588064065 73.5389994745371) bank134186 +134187 POINT(40.63151701299004 73.32433642709626) bank134187 +134188 POINT(39.83627054987533 74.20126394102613) bank134188 +134189 POINT(40.791072331445484 74.61462232773452) bank134189 +134190 POINT(41.66077300691905 74.52052116643424) bank134190 +134191 POINT(39.74662374023316 73.28172902899651) bank134191 +134192 POINT(40.104631419348244 74.81928374130553) bank134192 +134193 POINT(40.733890777410224 73.33782410180916) bank134193 +134194 POINT(40.90864128739584 74.17721097804255) bank134194 +134195 POINT(40.10412598026158 73.95456182180226) bank134195 +134196 POINT(39.80982541321744 74.50408871895306) bank134196 +134197 POINT(40.40233206401773 73.36796415005237) bank134197 +134198 POINT(40.19956471384437 74.07395404016303) bank134198 +134199 POINT(40.38617365392741 73.96123308067739) bank134199 +134200 POINT(41.65379064220495 74.35729024559052) bank134200 +134201 POINT(39.782165024343385 73.86525677348325) bank134201 +134202 POINT(41.535629668682354 73.43774192389023) bank134202 +134203 POINT(41.0944073046735 74.74144657174152) bank134203 +134204 POINT(39.97242431381093 73.90583488935427) bank134204 +134205 POINT(40.134296983843214 74.91485350591493) bank134205 +134206 POINT(41.403405367634235 74.31008890684384) bank134206 +134207 POINT(40.26102686146894 74.08203510553808) bank134207 +134208 POINT(40.839742525943166 74.17738228432249) bank134208 +134209 POINT(40.476153764042905 73.60534390562916) bank134209 +134210 POINT(41.19105864822359 73.04650758406939) bank134210 +134211 POINT(39.91908663323404 73.03061341120551) bank134211 +134212 POINT(40.17693891947127 74.42864802754684) bank134212 +134213 POINT(40.42283465717775 73.05043782372849) bank134213 +134214 POINT(40.22378769156667 74.26115881052382) bank134214 +134215 POINT(40.77719703303547 73.09110065726692) bank134215 +134216 POINT(40.20561094056359 73.74486088096228) bank134216 +134217 POINT(40.4707542924953 74.97350438547274) bank134217 +134218 POINT(41.518601030588066 74.19242261126837) bank134218 +134219 POINT(39.87133698586896 74.32662206994199) bank134219 +134220 POINT(40.38116640245397 74.69449124411861) bank134220 +134221 POINT(40.301392564344866 74.01454463341913) bank134221 +134222 POINT(41.278848345472866 74.30115830532564) bank134222 +134223 POINT(40.66324933383143 74.17235054309855) bank134223 +134224 POINT(41.62768951084239 74.35029033163606) bank134224 +134225 POINT(41.28798672522983 74.1414203487819) bank134225 +134226 POINT(41.09220808581354 73.12977262354083) bank134226 +134227 POINT(40.544295925673026 73.48800529568035) bank134227 +134228 POINT(40.175296416589056 73.51307914686721) bank134228 +134229 POINT(41.6418188139601 74.91576291026261) bank134229 +134230 POINT(40.68188537949508 74.87755093195602) bank134230 +134231 POINT(40.67603189714 73.74430979724777) bank134231 +134232 POINT(40.71229228524218 74.91437753725351) bank134232 +134233 POINT(39.79525177922945 74.41912948217417) bank134233 +134234 POINT(41.29488422409732 73.01629382352115) bank134234 +134235 POINT(40.070263984892286 74.8992890788942) bank134235 +134236 POINT(40.943199610548525 73.92884076134847) bank134236 +134237 POINT(41.7054662258604 74.32969433968626) bank134237 +134238 POINT(41.18545387133666 74.2838842754593) bank134238 +134239 POINT(40.474287265612695 74.02246664323857) bank134239 +134240 POINT(40.564105687671876 74.73334258493813) bank134240 +134241 POINT(40.77250719957424 73.97947439730206) bank134241 +134242 POINT(39.91760869660424 73.09175149431181) bank134242 +134243 POINT(40.27493756152487 74.7185877149421) bank134243 +134244 POINT(40.816217217617336 73.96664322655771) bank134244 +134245 POINT(41.097655064883135 74.18812121816799) bank134245 +134246 POINT(41.275570719090304 74.20729658172391) bank134246 +134247 POINT(39.72997274474381 73.51743186936055) bank134247 +134248 POINT(40.69293279997723 73.31649745249194) bank134248 +134249 POINT(40.505587183840326 74.16897742466686) bank134249 +134250 POINT(39.72806029749421 74.93803201707827) bank134250 +134251 POINT(40.154337564949294 73.98887097827345) bank134251 +134252 POINT(40.57601303868763 74.8479375488643) bank134252 +134253 POINT(40.75443401198968 73.60042974656727) bank134253 +134254 POINT(41.28263064319676 73.90296790303488) bank134254 +134255 POINT(40.19230021105472 74.15826669999409) bank134255 +134256 POINT(41.558394929752694 74.26365195553898) bank134256 +134257 POINT(39.91739516172269 74.25971882828975) bank134257 +134258 POINT(40.04070593102373 74.73861547688486) bank134258 +134259 POINT(41.05035530081559 73.21819890741084) bank134259 +134260 POINT(40.082531806798684 74.04975382585073) bank134260 +134261 POINT(41.4231633046833 73.67539491439442) bank134261 +134262 POINT(41.53082964915004 73.32999210571626) bank134262 +134263 POINT(41.20171375628531 73.49815766239531) bank134263 +134264 POINT(40.29554798360587 74.60777833891983) bank134264 +134265 POINT(40.62757113031402 74.19317132245396) bank134265 +134266 POINT(39.73096550626752 74.6036084356333) bank134266 +134267 POINT(41.32707167252444 74.20252447931004) bank134267 +134268 POINT(40.02882943921833 73.18461609872482) bank134268 +134269 POINT(40.75873876020585 73.37233557544904) bank134269 +134270 POINT(40.6234649672873 73.3139857785695) bank134270 +134271 POINT(40.62606298047079 74.02333962426715) bank134271 +134272 POINT(39.98996904899879 73.22886317845828) bank134272 +134273 POINT(40.988125824659804 74.73111220654592) bank134273 +134274 POINT(41.214076844305374 74.03454093895894) bank134274 +134275 POINT(40.07456438959043 73.06409973404246) bank134275 +134276 POINT(39.97852694817803 74.60629265067824) bank134276 +134277 POINT(40.76415765713886 73.55713568546693) bank134277 +134278 POINT(40.829357423964034 74.13989259746678) bank134278 +134279 POINT(40.05393432817161 74.62782753271878) bank134279 +134280 POINT(41.22675859487707 74.8996149866785) bank134280 +134281 POINT(41.1858338489881 73.29990714069866) bank134281 +134282 POINT(41.18447392850455 73.2508771039154) bank134282 +134283 POINT(41.262714073037884 74.12589732656927) bank134283 +134284 POINT(39.74123674679666 73.69852593068671) bank134284 +134285 POINT(41.2977179426964 74.3329604649454) bank134285 +134286 POINT(40.81381987810411 74.05608815270152) bank134286 +134287 POINT(41.03666664482861 73.65899551136623) bank134287 +134288 POINT(40.76202011676117 74.46891992433166) bank134288 +134289 POINT(41.46364630366688 73.90211206008257) bank134289 +134290 POINT(40.391441682359705 73.19895508107012) bank134290 +134291 POINT(40.15628439489335 73.55693497585511) bank134291 +134292 POINT(40.04226768813347 74.06184899771155) bank134292 +134293 POINT(40.126200601829986 73.51920034221928) bank134293 +134294 POINT(40.829927681401074 74.34474267359955) bank134294 +134295 POINT(40.24997599393241 73.40504229832291) bank134295 +134296 POINT(41.22417687629834 74.34951010717768) bank134296 +134297 POINT(41.19816722485689 73.13469913141046) bank134297 +134298 POINT(40.908241954455086 74.31864621318964) bank134298 +134299 POINT(40.54543842043323 74.20612072355672) bank134299 +134300 POINT(40.37771910048314 73.6802859404714) bank134300 +134301 POINT(41.14343449867874 73.20629976630023) bank134301 +134302 POINT(41.38405571444664 74.53171394130587) bank134302 +134303 POINT(39.927917438688354 74.94444199612161) bank134303 +134304 POINT(41.30053091251662 73.10873567296781) bank134304 +134305 POINT(39.77713013276637 74.6626552071057) bank134305 +134306 POINT(40.937014228434236 73.67009075373157) bank134306 +134307 POINT(41.265489424488585 74.37258393990538) bank134307 +134308 POINT(41.38792610562194 73.71589181450652) bank134308 +134309 POINT(40.53920158886429 74.03757568962034) bank134309 +134310 POINT(41.289414131248996 74.92342576566998) bank134310 +134311 POINT(40.89255766787173 74.25451785365354) bank134311 +134312 POINT(40.13969532384449 73.84239449183899) bank134312 +134313 POINT(40.53788072300471 74.21339472643766) bank134313 +134314 POINT(40.89631318408567 73.36009532207426) bank134314 +134315 POINT(39.78085596637359 73.17306666221187) bank134315 +134316 POINT(41.22938571125222 73.62284438152628) bank134316 +134317 POINT(40.51509156924729 74.45343002936697) bank134317 +134318 POINT(40.8405066129443 74.65411996829373) bank134318 +134319 POINT(40.30694462046091 73.78153187944412) bank134319 +134320 POINT(39.91296919035352 73.82171933642309) bank134320 +134321 POINT(41.2092555732208 74.69474497061886) bank134321 +134322 POINT(41.20635418638057 74.60141051145212) bank134322 +134323 POINT(39.9782740584842 74.6821696687158) bank134323 +134324 POINT(41.58059868726032 73.25901770574461) bank134324 +134325 POINT(41.66049700454474 73.30659494486005) bank134325 +134326 POINT(40.381943149379964 74.4517112178679) bank134326 +134327 POINT(40.46950258941642 74.68319111803667) bank134327 +134328 POINT(41.13712408279879 74.74306975113656) bank134328 +134329 POINT(41.00721966901705 73.91092309262245) bank134329 +134330 POINT(40.03777013685256 74.67776953172415) bank134330 +134331 POINT(40.67485812212836 73.06513930400642) bank134331 +134332 POINT(40.73687398250144 74.31448873437314) bank134332 +134333 POINT(40.097175853431416 74.77779509402366) bank134333 +134334 POINT(40.249787110867054 74.76945113915636) bank134334 +134335 POINT(40.39378786340438 73.34632319795968) bank134335 +134336 POINT(40.234375071678734 74.00120584899484) bank134336 +134337 POINT(39.729106131735065 73.9967133475268) bank134337 +134338 POINT(40.34509541141697 74.75962213580124) bank134338 +134339 POINT(39.786271868639346 74.31264752596631) bank134339 +134340 POINT(40.84729861315164 74.41339420860643) bank134340 +134341 POINT(41.271662942316986 73.59078947798807) bank134341 +134342 POINT(40.3593816242108 73.1352216219729) bank134342 +134343 POINT(40.07248966361328 73.54931243230722) bank134343 +134344 POINT(40.13082319960525 73.65844968671783) bank134344 +134345 POINT(40.68155683184544 74.5650017104295) bank134345 +134346 POINT(41.22252461494967 74.68558809726291) bank134346 +134347 POINT(40.78428379734252 73.63219784790668) bank134347 +134348 POINT(41.448824331107254 73.94562172425006) bank134348 +134349 POINT(41.167548655593556 73.23451707165495) bank134349 +134350 POINT(40.848092093393504 73.51041792776869) bank134350 +134351 POINT(41.49419222746148 73.82502351070127) bank134351 +134352 POINT(41.30017259316564 74.55282868695197) bank134352 +134353 POINT(40.240228449963624 73.96935864613751) bank134353 +134354 POINT(40.0872332259476 74.76187212343827) bank134354 +134355 POINT(41.32157999802592 74.40060737487063) bank134355 +134356 POINT(39.77028765107553 73.7555216599873) bank134356 +134357 POINT(40.96349346542125 73.6003181402032) bank134357 +134358 POINT(40.344526578145796 74.76147937034158) bank134358 +134359 POINT(41.113527596994665 73.0871073256854) bank134359 +134360 POINT(41.46733148701374 74.42060933445143) bank134360 +134361 POINT(40.76564248147695 74.59550573802724) bank134361 +134362 POINT(41.161238698013605 73.41892217540068) bank134362 +134363 POINT(39.85009294375385 73.51473411212883) bank134363 +134364 POINT(40.42391695275103 73.88421879649759) bank134364 +134365 POINT(41.28138926024934 73.45274537113569) bank134365 +134366 POINT(41.296219281761815 74.68305140542941) bank134366 +134367 POINT(39.96573035234193 74.45875157974547) bank134367 +134368 POINT(40.026074845482604 73.77720634979897) bank134368 +134369 POINT(41.62811496325364 73.86738247337215) bank134369 +134370 POINT(39.93287790067326 73.12740291549103) bank134370 +134371 POINT(41.54549869439858 74.66313567973101) bank134371 +134372 POINT(40.971312275726284 73.15050725017686) bank134372 +134373 POINT(41.081844629852874 74.47493583009587) bank134373 +134374 POINT(40.93727658156685 73.4634837102824) bank134374 +134375 POINT(41.11350868392983 73.16792333351665) bank134375 +134376 POINT(41.42076414088454 74.3569754206196) bank134376 +134377 POINT(41.38472611880679 73.09057872430775) bank134377 +134378 POINT(39.8673169672447 74.44289395820914) bank134378 +134379 POINT(41.488821354039835 73.62884939249739) bank134379 +134380 POINT(40.806621431907374 74.32602067198152) bank134380 +134381 POINT(39.80436582686584 73.16700281831375) bank134381 +134382 POINT(40.87732180063395 73.24800525607358) bank134382 +134383 POINT(40.24656153770956 73.04631297209201) bank134383 +134384 POINT(40.133561350317656 73.51592719367129) bank134384 +134385 POINT(40.724465961602256 74.35097461028883) bank134385 +134386 POINT(40.83316625719402 73.073566755295) bank134386 +134387 POINT(41.08488555645975 73.35697335128066) bank134387 +134388 POINT(40.864197834290955 73.27558829238212) bank134388 +134389 POINT(40.22982669171392 73.13248552355698) bank134389 +134390 POINT(40.866463744034554 73.26267875354803) bank134390 +134391 POINT(40.80307928897306 74.90780834598124) bank134391 +134392 POINT(40.5726334382619 73.1484287557838) bank134392 +134393 POINT(40.22580925463248 74.69185282165486) bank134393 +134394 POINT(40.00697742828621 74.5042307870721) bank134394 +134395 POINT(41.08706114830045 73.63551254430378) bank134395 +134396 POINT(40.65445209788908 74.57749257570698) bank134396 +134397 POINT(40.774489557192084 73.7066583559432) bank134397 +134398 POINT(40.324210298926154 73.56083205715106) bank134398 +134399 POINT(41.49706697557578 73.15746951996839) bank134399 +134400 POINT(41.34469800047867 73.27828611962654) bank134400 +134401 POINT(40.100004136583344 74.50751897056128) bank134401 +134402 POINT(39.974714054382396 73.0152754631997) bank134402 +134403 POINT(40.41733079384868 74.87901264236618) bank134403 +134404 POINT(41.091506437323545 73.63318725867778) bank134404 +134405 POINT(41.381200863377366 74.25906231238572) bank134405 +134406 POINT(40.292021952062754 74.93771693247295) bank134406 +134407 POINT(41.17233299312453 74.5647253927257) bank134407 +134408 POINT(39.8616321855378 73.91420908127624) bank134408 +134409 POINT(40.92714470017999 73.51160622723481) bank134409 +134410 POINT(39.92138392980441 73.7900570700802) bank134410 +134411 POINT(40.014253892222 74.77820935336436) bank134411 +134412 POINT(40.42269787353195 73.36609351909819) bank134412 +134413 POINT(41.45045441731893 73.62891666563355) bank134413 +134414 POINT(41.55438974470671 74.27342688286765) bank134414 +134415 POINT(40.09155391974178 74.44395008233005) bank134415 +134416 POINT(40.72956861318774 74.0644541946335) bank134416 +134417 POINT(39.789405878004395 74.30736846054141) bank134417 +134418 POINT(41.52261433228689 74.27889864316212) bank134418 +134419 POINT(41.534102700446525 73.7457801509918) bank134419 +134420 POINT(40.089698502243635 73.73602586224317) bank134420 +134421 POINT(40.77157276446683 73.01672054487477) bank134421 +134422 POINT(40.89839698242067 74.3143312476258) bank134422 +134423 POINT(41.06104664055135 74.73328422342234) bank134423 +134424 POINT(40.674448031899615 73.85212747310271) bank134424 +134425 POINT(40.247196951569784 73.977815351921) bank134425 +134426 POINT(40.12029816714064 73.59842401794349) bank134426 +134427 POINT(40.94061930815141 73.13920653945016) bank134427 +134428 POINT(41.503482493304645 74.32810591758803) bank134428 +134429 POINT(41.17392083838932 73.10081841769504) bank134429 +134430 POINT(40.809944280313935 73.03324032388402) bank134430 +134431 POINT(41.121608641500835 73.77685331115393) bank134431 +134432 POINT(40.97937507761837 74.19415813089991) bank134432 +134433 POINT(39.96717174182863 74.48130128064173) bank134433 +134434 POINT(40.315084568621415 73.74952557675756) bank134434 +134435 POINT(40.483201940235894 74.39077679814959) bank134435 +134436 POINT(40.30728755675934 74.350134891082) bank134436 +134437 POINT(40.7966556046508 74.81157031250545) bank134437 +134438 POINT(41.06257346289181 73.11412534290196) bank134438 +134439 POINT(39.77102302944886 73.89158918470366) bank134439 +134440 POINT(41.23578484422236 73.15574541676256) bank134440 +134441 POINT(40.02619549951822 74.10632690009616) bank134441 +134442 POINT(39.85039511444673 74.4214204633257) bank134442 +134443 POINT(40.436048672048614 74.6929846151568) bank134443 +134444 POINT(39.99297351309187 73.69119617691452) bank134444 +134445 POINT(41.351398627989 74.73970059405107) bank134445 +134446 POINT(40.753171814769544 73.09227420006006) bank134446 +134447 POINT(41.7038024703439 74.91098784945524) bank134447 +134448 POINT(41.59561598790798 74.73437747468252) bank134448 +134449 POINT(40.250960212809915 74.68575866165781) bank134449 +134450 POINT(39.911700866180034 73.04690590943318) bank134450 +134451 POINT(40.77121087209449 74.66988679007277) bank134451 +134452 POINT(40.762836709355874 73.54170374775319) bank134452 +134453 POINT(39.891262106017344 73.66945908832525) bank134453 +134454 POINT(41.11540396516409 74.68331309532492) bank134454 +134455 POINT(41.31940728809208 74.60563149822013) bank134455 +134456 POINT(41.05768444677829 73.58061336602908) bank134456 +134457 POINT(40.7414871493459 73.42551183580495) bank134457 +134458 POINT(40.940997647189626 74.68686276836324) bank134458 +134459 POINT(41.65497631825304 74.06972616443734) bank134459 +134460 POINT(40.422670460615535 74.02604676824738) bank134460 +134461 POINT(40.48940940026427 73.91543180225698) bank134461 +134462 POINT(40.47322710467522 74.07224317249906) bank134462 +134463 POINT(40.24642926309072 73.83586757451245) bank134463 +134464 POINT(41.292939668433334 73.15320277711812) bank134464 +134465 POINT(41.6460178693137 74.51824441767776) bank134465 +134466 POINT(40.79376816593175 73.33267372906653) bank134466 +134467 POINT(40.886239872242676 74.44498194649938) bank134467 +134468 POINT(41.43800282663075 74.43207452435004) bank134468 +134469 POINT(40.29549778226349 73.41575299704034) bank134469 +134470 POINT(40.99779631041532 73.4702731217234) bank134470 +134471 POINT(40.70073256141214 73.93716774109153) bank134471 +134472 POINT(41.08188807812575 73.62267555875032) bank134472 +134473 POINT(41.152114878997644 73.77663713624158) bank134473 +134474 POINT(40.08493862762564 73.33711110009487) bank134474 +134475 POINT(39.8735255385839 74.31131228733982) bank134475 +134476 POINT(40.49861470851581 74.72842176802749) bank134476 +134477 POINT(39.98949918706816 73.39829273664748) bank134477 +134478 POINT(39.84485668670874 73.52282935873427) bank134478 +134479 POINT(40.91801558162856 74.45931773090165) bank134479 +134480 POINT(41.24093960549775 74.31571698000795) bank134480 +134481 POINT(41.26317931154536 74.11927537247185) bank134481 +134482 POINT(41.63039470034253 73.34831147494084) bank134482 +134483 POINT(41.58228943203942 74.92777072087347) bank134483 +134484 POINT(41.43804866310935 74.83674679547639) bank134484 +134485 POINT(40.91489337735191 74.7396090024262) bank134485 +134486 POINT(40.52597343978698 73.2590182315674) bank134486 +134487 POINT(40.67930342893034 73.61557329824007) bank134487 +134488 POINT(41.62345634865933 74.48450301052848) bank134488 +134489 POINT(40.14506559839387 73.50318600065943) bank134489 +134490 POINT(41.628613359425145 73.91989911400997) bank134490 +134491 POINT(41.12845210199574 73.32583397553421) bank134491 +134492 POINT(41.06852299649469 74.6605973685144) bank134492 +134493 POINT(39.77126855455034 74.29200674166265) bank134493 +134494 POINT(41.523172448417974 73.71391434344658) bank134494 +134495 POINT(40.950893598974474 73.45868427811781) bank134495 +134496 POINT(40.80843244756097 73.11343885878311) bank134496 +134497 POINT(40.787131200158996 73.05922386266008) bank134497 +134498 POINT(40.71043767572135 74.45895645432192) bank134498 +134499 POINT(41.422582299495794 73.73372972843181) bank134499 +134500 POINT(40.43767659497243 74.61268992401692) bank134500 +134501 POINT(40.10805380481004 74.76045073881649) bank134501 +134502 POINT(41.48819069085179 74.44631612322088) bank134502 +134503 POINT(39.79472073938141 73.70532629357727) bank134503 +134504 POINT(40.78288628680052 73.83921356769869) bank134504 +134505 POINT(41.712048654660485 73.19040312889584) bank134505 +134506 POINT(39.90127848217089 74.85313860114421) bank134506 +134507 POINT(40.60321019112804 74.79039092241857) bank134507 +134508 POINT(41.45380779291511 74.47030901827195) bank134508 +134509 POINT(40.04044964731348 73.27663890960844) bank134509 +134510 POINT(40.79406173937552 74.09932271252869) bank134510 +134511 POINT(41.215211721182186 73.3060407867816) bank134511 +134512 POINT(40.73227047132524 73.65769710476813) bank134512 +134513 POINT(39.98097336361916 73.1955651636996) bank134513 +134514 POINT(40.19093429290582 74.15625776729729) bank134514 +134515 POINT(39.99012720725179 74.77240629734196) bank134515 +134516 POINT(40.43076781998132 73.49123658627991) bank134516 +134517 POINT(40.554895753779554 73.06151055063914) bank134517 +134518 POINT(40.33129088762446 73.52461740426172) bank134518 +134519 POINT(40.64451064473836 73.6572709740995) bank134519 +134520 POINT(41.407986790108694 74.8666596850023) bank134520 +134521 POINT(40.489402280853085 74.26669580614679) bank134521 +134522 POINT(40.21149927104704 73.42750614431117) bank134522 +134523 POINT(39.732958086129365 73.87547842463685) bank134523 +134524 POINT(40.716128632432735 73.05075001670986) bank134524 +134525 POINT(40.046156534005895 73.57726262797921) bank134525 +134526 POINT(39.992051548306975 73.91510453379915) bank134526 +134527 POINT(40.7761828570742 74.7823127119988) bank134527 +134528 POINT(40.87576994910843 73.3555620050008) bank134528 +134529 POINT(40.20453904426298 73.63643441491213) bank134529 +134530 POINT(41.59446959150203 74.85186005162673) bank134530 +134531 POINT(41.50210446418509 73.34749306143772) bank134531 +134532 POINT(41.50206350937831 73.69643734852511) bank134532 +134533 POINT(40.375982299162985 73.83915873332224) bank134533 +134534 POINT(41.685621447557416 73.55149119180508) bank134534 +134535 POINT(39.73905032797789 73.6135134309991) bank134535 +134536 POINT(40.74153327829729 74.74520163540141) bank134536 +134537 POINT(39.77623182601196 74.27323944091691) bank134537 +134538 POINT(40.01232352888063 73.14336038294499) bank134538 +134539 POINT(40.221052235595735 74.75124844112008) bank134539 +134540 POINT(41.13889542912608 74.3339088993385) bank134540 +134541 POINT(41.36084949624421 73.88698955374218) bank134541 +134542 POINT(40.96918990866521 74.36227957930362) bank134542 +134543 POINT(40.288197529500486 73.79904594642503) bank134543 +134544 POINT(41.69392750379106 74.02639174547934) bank134544 +134545 POINT(40.03023701660109 74.30946954802052) bank134545 +134546 POINT(40.25788146258741 74.85385449907352) bank134546 +134547 POINT(40.42708154766124 73.7411569499962) bank134547 +134548 POINT(40.77340299909504 74.58083124552392) bank134548 +134549 POINT(40.351283947630336 73.1611620155233) bank134549 +134550 POINT(41.34497660746532 74.23557200938568) bank134550 +134551 POINT(41.22913541116409 73.88314287890068) bank134551 +134552 POINT(41.225871320419316 73.88568029793545) bank134552 +134553 POINT(40.06626574808045 74.33023280300034) bank134553 +134554 POINT(40.33344137307852 74.73508468772303) bank134554 +134555 POINT(40.45062094428269 73.67454916868125) bank134555 +134556 POINT(40.36793159429672 74.90720381630611) bank134556 +134557 POINT(40.55447670771737 74.60115808191316) bank134557 +134558 POINT(39.86484092564495 74.26537342356292) bank134558 +134559 POINT(41.40725158940711 74.65020681584677) bank134559 +134560 POINT(40.766866350090055 73.33421149492564) bank134560 +134561 POINT(40.70047939393163 73.95383614459784) bank134561 +134562 POINT(40.33300698265145 73.53698250831023) bank134562 +134563 POINT(41.70049771137844 74.81448150139157) bank134563 +134564 POINT(39.95271623338386 74.61165390909872) bank134564 +134565 POINT(41.3282703519047 74.82219505651793) bank134565 +134566 POINT(40.69204970196176 74.38089801603843) bank134566 +134567 POINT(41.383922842824916 74.26858446395904) bank134567 +134568 POINT(40.99739296865269 74.43129448742052) bank134568 +134569 POINT(41.162524015694075 74.91162476184161) bank134569 +134570 POINT(40.813638813872174 74.93238500713022) bank134570 +134571 POINT(41.099105082951006 73.4275069219648) bank134571 +134572 POINT(39.76094970205684 73.84855260445315) bank134572 +134573 POINT(39.80592940536336 74.92979675556201) bank134573 +134574 POINT(40.97760928061749 73.66897386830789) bank134574 +134575 POINT(41.702125112439845 73.10092383332125) bank134575 +134576 POINT(39.93699193030943 73.42659217326865) bank134576 +134577 POINT(40.17329567925063 74.44667571007282) bank134577 +134578 POINT(40.10717366219952 74.0755153290955) bank134578 +134579 POINT(40.47138989021041 73.38561513507759) bank134579 +134580 POINT(41.21418072340447 73.70942003577316) bank134580 +134581 POINT(41.334887093716304 73.73582532143205) bank134581 +134582 POINT(40.386306889802775 73.72008842334395) bank134582 +134583 POINT(40.62736767181482 73.95587215420963) bank134583 +134584 POINT(39.876723031367696 74.15907782821887) bank134584 +134585 POINT(41.55279437648315 74.19443522073031) bank134585 +134586 POINT(39.71340666966819 74.79197079118673) bank134586 +134587 POINT(40.96771920376623 73.21295943941838) bank134587 +134588 POINT(40.35029774858045 74.29992144413652) bank134588 +134589 POINT(40.41733616692651 73.63155115006971) bank134589 +134590 POINT(40.83160043521992 74.61941430760389) bank134590 +134591 POINT(41.22699871420356 74.82975455767073) bank134591 +134592 POINT(41.53647634993262 73.93103170576248) bank134592 +134593 POINT(40.40320279749094 73.7517513814244) bank134593 +134594 POINT(41.10471765749264 73.1292546584482) bank134594 +134595 POINT(40.54058467231958 74.64137931085425) bank134595 +134596 POINT(40.32898466061752 74.91418205047468) bank134596 +134597 POINT(40.424289274176076 74.36301437983437) bank134597 +134598 POINT(41.01776891969272 74.45633131498987) bank134598 +134599 POINT(40.43746309427558 73.68309238648166) bank134599 +134600 POINT(41.51703646483526 75.00509114614155) bank134600 +134601 POINT(41.09896733520108 74.44363897159184) bank134601 +134602 POINT(40.32379493041632 73.35164158632419) bank134602 +134603 POINT(41.509634650122265 73.18605398724311) bank134603 +134604 POINT(40.74300853680341 73.90651673611102) bank134604 +134605 POINT(41.17163317847808 74.16028538364282) bank134605 +134606 POINT(41.365965078488564 74.57651348748954) bank134606 +134607 POINT(41.236609790222516 74.57934868830456) bank134607 +134608 POINT(41.22339307154348 73.85075868814864) bank134608 +134609 POINT(41.41782095472397 73.60252873537719) bank134609 +134610 POINT(39.94894982837086 73.58106470691034) bank134610 +134611 POINT(41.43881642083961 74.74089219511595) bank134611 +134612 POINT(40.23086022715678 73.68071615729637) bank134612 +134613 POINT(41.32828184922126 73.46247317426764) bank134613 +134614 POINT(41.103950659930625 73.5528395301623) bank134614 +134615 POINT(41.35771168194887 74.64215383076147) bank134615 +134616 POINT(40.17653690150716 74.51291610420377) bank134616 +134617 POINT(40.391048041705616 73.69412397093743) bank134617 +134618 POINT(40.326418972347675 74.77518407250501) bank134618 +134619 POINT(40.091075079282454 73.89908005789441) bank134619 +134620 POINT(40.73512993240287 74.86882991367355) bank134620 +134621 POINT(41.66658414417592 73.57213079142859) bank134621 +134622 POINT(41.10248439806974 74.45287771797013) bank134622 +134623 POINT(40.813002732764204 74.81109013095063) bank134623 +134624 POINT(41.09934518461365 73.2900174711159) bank134624 +134625 POINT(40.739637429185535 73.6281536837166) bank134625 +134626 POINT(40.494734191603214 74.04196963052347) bank134626 +134627 POINT(41.527659562734826 73.40295119316605) bank134627 +134628 POINT(40.30577141870921 74.99614462496773) bank134628 +134629 POINT(41.04757948053065 73.85290729810073) bank134629 +134630 POINT(41.05726585823132 73.30750349369184) bank134630 +134631 POINT(40.992432456071704 74.85955664603112) bank134631 +134632 POINT(41.2833164645107 73.66296711128626) bank134632 +134633 POINT(40.11306614148356 73.17881068619411) bank134633 +134634 POINT(41.25371841670229 74.90121101955638) bank134634 +134635 POINT(41.150012373518166 73.62831937673539) bank134635 +134636 POINT(39.97679415778475 74.3060918502273) bank134636 +134637 POINT(41.60159666016443 74.86324765137388) bank134637 +134638 POINT(41.019851700245106 73.11220985756361) bank134638 +134639 POINT(40.693884424535625 74.73703762893614) bank134639 +134640 POINT(40.9654208237662 73.24280004077494) bank134640 +134641 POINT(41.53567058420521 73.60642957170609) bank134641 +134642 POINT(41.1415834674583 73.97715122843452) bank134642 +134643 POINT(39.873915303997464 74.51362691642122) bank134643 +134644 POINT(41.27688051256247 73.39269602825846) bank134644 +134645 POINT(41.341599406513176 74.23036988022886) bank134645 +134646 POINT(41.68307157650216 73.08313656682596) bank134646 +134647 POINT(40.328988244793635 73.50765521770961) bank134647 +134648 POINT(39.99126081203783 73.61759923096238) bank134648 +134649 POINT(39.959465480202006 73.41256826549561) bank134649 +134650 POINT(40.884393163532124 74.27102475109751) bank134650 +134651 POINT(41.69190994545347 74.9712225439707) bank134651 +134652 POINT(41.118514711019934 73.1044119673027) bank134652 +134653 POINT(40.158574664475985 73.58956420402257) bank134653 +134654 POINT(40.4982032441834 74.05598341377896) bank134654 +134655 POINT(40.07665551634281 74.13727195628097) bank134655 +134656 POINT(40.810268522601326 74.90171464476182) bank134656 +134657 POINT(40.73747961377518 73.986019415455) bank134657 +134658 POINT(39.99551901566542 73.15033289246537) bank134658 +134659 POINT(40.11381596026853 73.12874665060616) bank134659 +134660 POINT(41.38369856785375 74.76007957019769) bank134660 +134661 POINT(40.502054508355386 74.85631305675007) bank134661 +134662 POINT(41.25791767363779 73.30413327399263) bank134662 +134663 POINT(40.32097588651621 73.8698012657397) bank134663 +134664 POINT(41.570390243638755 73.57072359368537) bank134664 +134665 POINT(39.84227659867601 73.44126328500397) bank134665 +134666 POINT(40.77175182710758 74.64006660431937) bank134666 +134667 POINT(41.05235258681634 74.56114243374002) bank134667 +134668 POINT(41.43507357180172 73.12735479160597) bank134668 +134669 POINT(39.90556211781826 74.39368446222454) bank134669 +134670 POINT(40.36617898808319 74.3854374232298) bank134670 +134671 POINT(40.15663212790502 73.42145330066187) bank134671 +134672 POINT(41.49438776799243 74.04653286387129) bank134672 +134673 POINT(40.70806235320847 74.5390394948191) bank134673 +134674 POINT(40.15472431909428 73.64459217045001) bank134674 +134675 POINT(41.45177673251526 73.87756023163611) bank134675 +134676 POINT(40.18826257327695 73.29371591618185) bank134676 +134677 POINT(39.96409021607863 74.43978404121239) bank134677 +134678 POINT(41.53234807361985 74.10429399835114) bank134678 +134679 POINT(41.047064130350215 73.67961674068299) bank134679 +134680 POINT(39.75358628252939 73.22806554945083) bank134680 +134681 POINT(39.93542788497729 73.64672660778871) bank134681 +134682 POINT(40.820551225067575 74.7685999905939) bank134682 +134683 POINT(39.776129802902716 73.15750787996899) bank134683 +134684 POINT(40.196548170764984 74.0607055473434) bank134684 +134685 POINT(41.16267624436462 74.43127239539214) bank134685 +134686 POINT(40.88269371036641 73.45109526826838) bank134686 +134687 POINT(40.88440340744323 73.55125913600695) bank134687 +134688 POINT(41.3672821196131 74.86781033152214) bank134688 +134689 POINT(41.66812236492889 74.678161296433) bank134689 +134690 POINT(40.02477339701913 73.06044280134441) bank134690 +134691 POINT(40.431009075313895 73.17689392394534) bank134691 +134692 POINT(41.66665379777815 73.04282332783222) bank134692 +134693 POINT(41.16712815933992 73.8071573628568) bank134693 +134694 POINT(41.349056846029484 73.79666990393865) bank134694 +134695 POINT(41.20987737651433 73.9736442933148) bank134695 +134696 POINT(41.11095717033426 73.08943603522323) bank134696 +134697 POINT(41.542238952596456 73.107887640525) bank134697 +134698 POINT(41.305850860786606 74.86625123527085) bank134698 +134699 POINT(40.39506262064654 73.64361878315773) bank134699 +134700 POINT(40.030913150313644 73.90080390563406) bank134700 +134701 POINT(41.51176219700215 74.54590107885092) bank134701 +134702 POINT(40.637999218061196 74.94862084897143) bank134702 +134703 POINT(40.464014235950415 73.89347028839566) bank134703 +134704 POINT(40.12443605978919 73.69777834473793) bank134704 +134705 POINT(39.76377777343157 74.02072564899002) bank134705 +134706 POINT(40.34844050609211 74.52927760064625) bank134706 +134707 POINT(41.58543808817875 73.28225817726188) bank134707 +134708 POINT(41.0151173072923 74.63530246920033) bank134708 +134709 POINT(40.39218022361921 74.89966669407683) bank134709 +134710 POINT(40.75540644439107 73.8371960381559) bank134710 +134711 POINT(41.52618898305056 73.69904264485375) bank134711 +134712 POINT(41.70576377867114 74.60819656864805) bank134712 +134713 POINT(41.6330310857565 74.72247540938469) bank134713 +134714 POINT(41.59321778060749 74.78279376667793) bank134714 +134715 POINT(41.58368423486989 74.88568306230272) bank134715 +134716 POINT(40.675669587142494 74.45470743824232) bank134716 +134717 POINT(39.7147402732806 74.62735708551659) bank134717 +134718 POINT(40.132463592258404 73.6019184928523) bank134718 +134719 POINT(39.833363862215826 74.81227531053518) bank134719 +134720 POINT(39.79870334338672 73.35524152327648) bank134720 +134721 POINT(40.24670415248561 74.5003456179338) bank134721 +134722 POINT(40.34209316952484 73.66968879361303) bank134722 +134723 POINT(40.796288733739395 73.22529482934279) bank134723 +134724 POINT(41.018225206630284 74.55756112032022) bank134724 +134725 POINT(41.087544063953786 73.5644436152889) bank134725 +134726 POINT(41.45173451256461 73.39792034297858) bank134726 +134727 POINT(40.97985090002062 74.34722303852317) bank134727 +134728 POINT(39.90150157991467 73.75456056128576) bank134728 +134729 POINT(41.100226343098484 73.82474319215424) bank134729 +134730 POINT(41.21839981998125 73.31907511604052) bank134730 +134731 POINT(40.01963828231708 73.9472407671552) bank134731 +134732 POINT(41.43877093188933 73.59275338827207) bank134732 +134733 POINT(40.66309090533023 74.6107309386949) bank134733 +134734 POINT(40.28121436495397 74.50502604648237) bank134734 +134735 POINT(40.65696188384383 74.04562577093793) bank134735 +134736 POINT(40.604188438787595 74.68975207946937) bank134736 +134737 POINT(39.750536285405445 74.73251401482645) bank134737 +134738 POINT(40.93578423225577 74.27365240600855) bank134738 +134739 POINT(41.38359007493227 74.49385648761097) bank134739 +134740 POINT(40.523390996814165 74.68494233819146) bank134740 +134741 POINT(40.0356240396812 73.79270220041747) bank134741 +134742 POINT(41.142837259805766 74.56437219107727) bank134742 +134743 POINT(41.10579355745553 74.074585643194) bank134743 +134744 POINT(39.99973626093164 73.36173898865657) bank134744 +134745 POINT(40.97983342299644 73.85584421237368) bank134745 +134746 POINT(40.76331829061268 73.06290929047834) bank134746 +134747 POINT(41.092005740416106 73.61780710359106) bank134747 +134748 POINT(40.087648438212675 74.27433117838837) bank134748 +134749 POINT(39.94885685208722 74.56318972761903) bank134749 +134750 POINT(41.180167749806714 74.3250425738649) bank134750 +134751 POINT(40.19430859915657 73.43110462558755) bank134751 +134752 POINT(40.56720812096192 73.43563106081606) bank134752 +134753 POINT(39.873375676426456 74.43450606401797) bank134753 +134754 POINT(40.36812417091742 74.2227929758902) bank134754 +134755 POINT(40.62319261569421 73.75321158367825) bank134755 +134756 POINT(41.392349496805906 74.54193442898273) bank134756 +134757 POINT(40.49396397343087 74.03065788549277) bank134757 +134758 POINT(39.953180554277054 74.03654740568322) bank134758 +134759 POINT(39.74008996804251 74.58424069216184) bank134759 +134760 POINT(41.22865218253372 74.06784153995999) bank134760 +134761 POINT(40.996960450866574 73.66240033214852) bank134761 +134762 POINT(40.49793084566133 74.33669002711756) bank134762 +134763 POINT(40.09301477638262 74.63031107086407) bank134763 +134764 POINT(40.092136947735774 74.48599686503533) bank134764 +134765 POINT(40.98970681516868 73.54294680808201) bank134765 +134766 POINT(41.388885369883965 73.28416485897557) bank134766 +134767 POINT(40.26557401868546 73.96134736111176) bank134767 +134768 POINT(40.126322724426 74.05407805194235) bank134768 +134769 POINT(40.37675915966311 73.04514051210626) bank134769 +134770 POINT(41.27238548376788 73.35467378274882) bank134770 +134771 POINT(40.91082570872863 74.67193213615177) bank134771 +134772 POINT(41.68462682587848 74.95304417844866) bank134772 +134773 POINT(41.432855370881455 74.71792065181687) bank134773 +134774 POINT(40.94002090361651 74.09072799160892) bank134774 +134775 POINT(41.250764481697864 74.15937878566628) bank134775 +134776 POINT(41.6811067344332 73.74663350599286) bank134776 +134777 POINT(41.54532391825676 74.6855692338676) bank134777 +134778 POINT(41.67272231055683 73.75319169589933) bank134778 +134779 POINT(40.205891399399256 73.21497077629505) bank134779 +134780 POINT(40.226320658991334 74.56780646770997) bank134780 +134781 POINT(41.08399532401905 73.49742997279124) bank134781 +134782 POINT(41.33253492772575 73.85614540758257) bank134782 +134783 POINT(40.89851583532672 74.93558442066475) bank134783 +134784 POINT(41.25439980215477 74.16439717555284) bank134784 +134785 POINT(39.77483780071417 73.3305051050532) bank134785 +134786 POINT(39.87709736336749 74.66871242743058) bank134786 +134787 POINT(41.009986501308276 73.62366907307131) bank134787 +134788 POINT(41.14317580883976 74.90336601313057) bank134788 +134789 POINT(41.47008650203563 73.3061834268168) bank134789 +134790 POINT(40.29101461114573 74.31660183672514) bank134790 +134791 POINT(40.651651811007035 74.45247615380507) bank134791 +134792 POINT(40.69142630732864 73.20173670665454) bank134792 +134793 POINT(40.007351983861405 73.35908845152588) bank134793 +134794 POINT(39.91452334790464 74.153025956519) bank134794 +134795 POINT(41.005441105330256 74.61444999359918) bank134795 +134796 POINT(39.76893335065255 73.11802928505364) bank134796 +134797 POINT(41.43827190500434 74.67032334048112) bank134797 +134798 POINT(41.245795663353306 73.48763464834221) bank134798 +134799 POINT(40.13983988801488 73.76677933942645) bank134799 +134800 POINT(40.0971932905814 74.52206198034933) bank134800 +134801 POINT(40.886476559465464 73.39917292879781) bank134801 +134802 POINT(39.98044292629452 74.61439403224945) bank134802 +134803 POINT(40.22022701731561 74.71890788931685) bank134803 +134804 POINT(39.999890008157664 73.46019057325735) bank134804 +134805 POINT(40.299597066331756 74.24531639723318) bank134805 +134806 POINT(41.49754118521877 73.27627790820941) bank134806 +134807 POINT(41.54401595799641 74.79962301080272) bank134807 +134808 POINT(41.176053614758835 74.6875279261447) bank134808 +134809 POINT(41.40706532277915 73.48406122497232) bank134809 +134810 POINT(39.96146799070047 74.54169184924163) bank134810 +134811 POINT(40.71337534000711 74.56037998098307) bank134811 +134812 POINT(41.17062110401964 73.79812241647485) bank134812 +134813 POINT(39.89492278167572 73.72882091877798) bank134813 +134814 POINT(40.662216179868004 73.33978874360268) bank134814 +134815 POINT(41.568703856900534 74.51330532793796) bank134815 +134816 POINT(40.5281243846481 73.1667604754372) bank134816 +134817 POINT(39.819421744733816 74.6040390928121) bank134817 +134818 POINT(39.98654981152851 74.66318537318377) bank134818 +134819 POINT(40.637360834999754 74.8037842951015) bank134819 +134820 POINT(40.06782415204723 74.01823143585335) bank134820 +134821 POINT(41.078019819779854 73.95062888129316) bank134821 +134822 POINT(41.54057424644063 74.47771281557091) bank134822 +134823 POINT(41.55853554450499 73.27940741008013) bank134823 +134824 POINT(40.53261972581587 74.44548391637741) bank134824 +134825 POINT(39.95293207410405 74.56254147655278) bank134825 +134826 POINT(40.54851106883035 74.08713797953018) bank134826 +134827 POINT(41.25251117386704 74.54070863135695) bank134827 +134828 POINT(40.58021049636883 74.77033058511829) bank134828 +134829 POINT(40.74557603788911 74.69534820004166) bank134829 +134830 POINT(41.11403644764663 74.0628290665298) bank134830 +134831 POINT(40.72541145183134 74.78287139566183) bank134831 +134832 POINT(40.260797469967265 74.31795759986576) bank134832 +134833 POINT(41.56000086259093 73.36843918813062) bank134833 +134834 POINT(40.728988906805995 74.4208531994249) bank134834 +134835 POINT(41.534848617409125 74.37208350338311) bank134835 +134836 POINT(40.64825129805638 73.35434898297302) bank134836 +134837 POINT(40.64075568537645 74.44531814409675) bank134837 +134838 POINT(41.03196563377659 73.0259288930132) bank134838 +134839 POINT(40.36054676285123 73.74776265184492) bank134839 +134840 POINT(39.88517838111557 74.3481682462475) bank134840 +134841 POINT(41.27515466728779 74.3052510998036) bank134841 +134842 POINT(40.266327935417515 73.36699721278039) bank134842 +134843 POINT(40.608623715993254 73.7389542756776) bank134843 +134844 POINT(39.859444888179894 74.25201284274723) bank134844 +134845 POINT(40.09422199510192 74.7924246811446) bank134845 +134846 POINT(40.013635416553 73.48769596921504) bank134846 +134847 POINT(41.53129789647762 73.53920284109529) bank134847 +134848 POINT(40.29211320567877 74.54898449679052) bank134848 +134849 POINT(40.20575535179539 74.94289262655747) bank134849 +134850 POINT(40.498287946175054 73.31330573944358) bank134850 +134851 POINT(40.121377257811496 74.97173490171069) bank134851 +134852 POINT(41.07062764308135 73.73493548869345) bank134852 +134853 POINT(40.206352360870966 73.66976394608858) bank134853 +134854 POINT(41.56648730273911 74.79704976256059) bank134854 +134855 POINT(41.08578401419966 73.50061059146596) bank134855 +134856 POINT(40.43486186578837 73.8855507530186) bank134856 +134857 POINT(41.35678780170734 74.37347535140181) bank134857 +134858 POINT(41.022787473657075 73.20601065075016) bank134858 +134859 POINT(41.67703750019392 74.75875624411496) bank134859 +134860 POINT(40.7128442144101 74.21380799311619) bank134860 +134861 POINT(40.6157998206409 73.2699417769742) bank134861 +134862 POINT(41.37093515050092 73.84176307044457) bank134862 +134863 POINT(41.04772151743828 74.55103611451) bank134863 +134864 POINT(39.7871508947416 73.03368796533255) bank134864 +134865 POINT(40.09424237009583 74.44998855786854) bank134865 +134866 POINT(40.564655312292835 73.32629117658581) bank134866 +134867 POINT(39.77555085823862 73.42699411531254) bank134867 +134868 POINT(40.06368515954727 74.68129663103664) bank134868 +134869 POINT(40.96429106700166 74.18330369909538) bank134869 +134870 POINT(41.27595360180148 73.03033085200718) bank134870 +134871 POINT(40.33090321062219 73.53794951000077) bank134871 +134872 POINT(40.113715818415805 74.30610495087065) bank134872 +134873 POINT(39.8574322549956 74.96101822731377) bank134873 +134874 POINT(41.49435950380064 74.64769216009094) bank134874 +134875 POINT(40.47714984288666 73.10991395518262) bank134875 +134876 POINT(40.11826306147837 73.16839990730895) bank134876 +134877 POINT(40.72483543293156 73.0735904193996) bank134877 +134878 POINT(41.66630506257032 73.96621602407234) bank134878 +134879 POINT(40.42645226871049 74.71014807715163) bank134879 +134880 POINT(39.82580477377711 74.12522600823348) bank134880 +134881 POINT(40.88894361457052 74.16735943786794) bank134881 +134882 POINT(40.897355840711 73.37019162546724) bank134882 +134883 POINT(41.38660595150507 74.7371256015772) bank134883 +134884 POINT(40.36818519189251 74.2620379696737) bank134884 +134885 POINT(41.29151105446082 74.23718810904367) bank134885 +134886 POINT(41.47789563147604 73.97819795249639) bank134886 +134887 POINT(41.25748434787058 73.52763373489086) bank134887 +134888 POINT(40.30176708582411 74.61693724494316) bank134888 +134889 POINT(40.77654345005823 74.70949227803861) bank134889 +134890 POINT(41.18263395759058 73.24181089407367) bank134890 +134891 POINT(39.96869634735706 74.37199595843602) bank134891 +134892 POINT(41.48199435694986 73.55492665670849) bank134892 +134893 POINT(40.14363826164357 74.29178908803772) bank134893 +134894 POINT(40.63594795555918 73.93390437776903) bank134894 +134895 POINT(39.914587856812545 73.29643138501554) bank134895 +134896 POINT(41.09568829068687 73.78108696567118) bank134896 +134897 POINT(41.278297960791654 74.4855398384851) bank134897 +134898 POINT(41.062741357517645 74.91239389627954) bank134898 +134899 POINT(41.43097719451898 74.88084560041692) bank134899 +134900 POINT(41.50975257796386 73.25290604974064) bank134900 +134901 POINT(40.36270449423894 73.04197485316563) bank134901 +134902 POINT(41.61217358489192 73.29428859206514) bank134902 +134903 POINT(40.73771741321859 73.55335479097205) bank134903 +134904 POINT(40.63128630170198 73.74451266093291) bank134904 +134905 POINT(40.290015345792405 73.2235647733724) bank134905 +134906 POINT(39.99493003679897 73.30131192872075) bank134906 +134907 POINT(39.79272092941899 74.13917848607278) bank134907 +134908 POINT(41.113233471696425 74.14336247282894) bank134908 +134909 POINT(40.36834759744565 74.04752046375344) bank134909 +134910 POINT(39.93750701569737 74.71682181646801) bank134910 +134911 POINT(40.6308487375623 74.01389545108134) bank134911 +134912 POINT(40.9657865191853 73.22484362973884) bank134912 +134913 POINT(40.52217574676097 74.08919756651308) bank134913 +134914 POINT(41.025413575873465 74.57628589440309) bank134914 +134915 POINT(41.601192553174705 73.9891138226139) bank134915 +134916 POINT(40.89208573985889 74.57219797753261) bank134916 +134917 POINT(39.77753734347488 73.23784823838955) bank134917 +134918 POINT(40.93471359295365 74.99598603203937) bank134918 +134919 POINT(40.37375207866436 73.39263654522665) bank134919 +134920 POINT(40.5098558759946 74.44144294292276) bank134920 +134921 POINT(41.344594371143145 73.7345566426729) bank134921 +134922 POINT(40.15040114065329 74.6869738080179) bank134922 +134923 POINT(40.38488515341551 73.00618754176027) bank134923 +134924 POINT(41.52952030585459 74.12714076924522) bank134924 +134925 POINT(41.13217307192759 74.02029477709733) bank134925 +134926 POINT(41.095972876785844 73.3368582335762) bank134926 +134927 POINT(39.803846967861354 74.66743475349804) bank134927 +134928 POINT(39.809141878154705 73.69363835880782) bank134928 +134929 POINT(40.376977793753355 73.98353479150707) bank134929 +134930 POINT(41.15490850983209 73.95900319972651) bank134930 +134931 POINT(41.49414327425965 74.1430978065479) bank134931 +134932 POINT(40.502009533284536 74.98230364528986) bank134932 +134933 POINT(41.46848075348332 73.25781467801093) bank134933 +134934 POINT(40.54552480252923 74.85614513404171) bank134934 +134935 POINT(40.75210331979405 74.36939053109664) bank134935 +134936 POINT(40.29262680495172 74.50014917438779) bank134936 +134937 POINT(40.5134657374063 73.65841453209448) bank134937 +134938 POINT(40.179790936650484 73.73365122471776) bank134938 +134939 POINT(41.44698743266565 73.29799304827822) bank134939 +134940 POINT(39.851574614420585 74.21672589312882) bank134940 +134941 POINT(41.64363518927155 73.78660449853037) bank134941 +134942 POINT(40.99837213739189 73.7380176079864) bank134942 +134943 POINT(40.50547647135093 74.4555491746549) bank134943 +134944 POINT(40.43702808580924 74.67607363976207) bank134944 +134945 POINT(40.26507131428006 73.02670999781458) bank134945 +134946 POINT(41.498748692072965 73.7297434225974) bank134946 +134947 POINT(40.996515179182886 73.96461786457034) bank134947 +134948 POINT(39.96991108549172 74.28971337199087) bank134948 +134949 POINT(41.00832005133255 73.08662134900513) bank134949 +134950 POINT(40.46502609284975 74.198526898559) bank134950 +134951 POINT(39.97987550694185 73.67783422780909) bank134951 +134952 POINT(39.98354018317165 74.217161470195) bank134952 +134953 POINT(40.31095644607077 73.24387560285373) bank134953 +134954 POINT(40.57979321191871 73.37713421490066) bank134954 +134955 POINT(41.286864628866084 74.44787318620284) bank134955 +134956 POINT(39.99272900512877 73.61713077411056) bank134956 +134957 POINT(40.09993589546098 74.90623065039954) bank134957 +134958 POINT(39.81594345499701 74.73196298741672) bank134958 +134959 POINT(40.464026314473564 74.13968994117513) bank134959 +134960 POINT(40.83602679173299 73.09534993408931) bank134960 +134961 POINT(40.49441505093605 74.9069758065241) bank134961 +134962 POINT(40.93374245882169 73.45720140264827) bank134962 +134963 POINT(40.81556264131312 73.97423136602534) bank134963 +134964 POINT(40.99088386179631 74.85286785951958) bank134964 +134965 POINT(40.48863692812229 74.91172472770859) bank134965 +134966 POINT(40.81288444152686 73.25925831470498) bank134966 +134967 POINT(40.55267807871574 74.67153719851983) bank134967 +134968 POINT(41.464348191045666 73.38603505791068) bank134968 +134969 POINT(41.643718907331795 74.33963061116204) bank134969 +134970 POINT(40.94251563905885 73.01283139507065) bank134970 +134971 POINT(41.416984575450904 74.13981640627824) bank134971 +134972 POINT(40.685481668940156 74.03870399099034) bank134972 +134973 POINT(40.9145854219531 74.9557903929739) bank134973 +134974 POINT(41.66731038844523 73.39237755366813) bank134974 +134975 POINT(40.0508758030419 74.65372623135106) bank134975 +134976 POINT(40.10349770951622 74.14880259562115) bank134976 +134977 POINT(41.43267942684697 73.9382504776307) bank134977 +134978 POINT(41.13509412861735 74.25588386341192) bank134978 +134979 POINT(40.70257820280222 74.56416509845704) bank134979 +134980 POINT(41.08800968217825 73.03393542142538) bank134980 +134981 POINT(40.38953536397676 74.86759671214324) bank134981 +134982 POINT(39.87146813181811 73.08478656233414) bank134982 +134983 POINT(41.03665616412742 74.1436092807015) bank134983 +134984 POINT(41.05107633453176 74.4052727107718) bank134984 +134985 POINT(41.440351068967715 73.53452779764423) bank134985 +134986 POINT(41.124103446364295 73.77010600166984) bank134986 +134987 POINT(41.196421231828246 73.74823712256486) bank134987 +134988 POINT(40.50640595812815 74.78540485519427) bank134988 +134989 POINT(40.617104603137044 74.22596069430746) bank134989 +134990 POINT(41.48459797172241 73.90339602716998) bank134990 +134991 POINT(41.32935411267696 73.9099416784242) bank134991 +134992 POINT(41.51428344798088 73.03272547789683) bank134992 +134993 POINT(40.23124610829785 73.98837884617232) bank134993 +134994 POINT(41.07788621706398 74.50179933603985) bank134994 +134995 POINT(39.95692533526654 74.19290009171962) bank134995 +134996 POINT(40.147497133590434 73.08773399440163) bank134996 +134997 POINT(41.63151753779384 73.41136244823984) bank134997 +134998 POINT(41.16146911613617 73.22740654754224) bank134998 +134999 POINT(41.37702322653532 74.05710089803587) bank134999 +135000 POINT(40.56160990666093 74.57909608116132) bank135000 +135001 POINT(39.86461417718956 73.78749183466812) bank135001 +135002 POINT(39.85544560764738 73.62816959068708) bank135002 +135003 POINT(39.72797591191585 73.04042717912567) bank135003 +135004 POINT(41.28908730730112 74.59506153332086) bank135004 +135005 POINT(40.29787111487935 74.98571579295307) bank135005 +135006 POINT(41.453382392315 74.74670339894257) bank135006 +135007 POINT(40.5534046301944 73.53602386439336) bank135007 +135008 POINT(41.52976127839523 74.61575790973791) bank135008 +135009 POINT(39.89747684461309 73.56485587246286) bank135009 +135010 POINT(41.11324698599543 73.35868564490524) bank135010 +135011 POINT(41.670162911726315 73.86033705661019) bank135011 +135012 POINT(40.796001646223466 74.74988781832086) bank135012 +135013 POINT(40.97560829139441 73.56149999784071) bank135013 +135014 POINT(40.81935592327731 73.03556137008162) bank135014 +135015 POINT(41.422320757218145 74.25280016682106) bank135015 +135016 POINT(39.81712228961508 74.73979943549618) bank135016 +135017 POINT(40.781065686359234 73.61841431966597) bank135017 +135018 POINT(41.4199538017997 75.00373070746778) bank135018 +135019 POINT(40.036133295980655 73.11493527824838) bank135019 +135020 POINT(39.860450922131136 74.83278597052431) bank135020 +135021 POINT(39.72046424562972 73.99621925334138) bank135021 +135022 POINT(40.4420184378939 74.05269533992055) bank135022 +135023 POINT(40.25930422777804 73.0473199230435) bank135023 +135024 POINT(41.687090699119196 73.73790521336649) bank135024 +135025 POINT(41.40675949685078 73.52241974035329) bank135025 +135026 POINT(40.37051327327151 73.87662077517689) bank135026 +135027 POINT(40.96071610060916 73.5206729096338) bank135027 +135028 POINT(40.94264303390839 74.73200474362166) bank135028 +135029 POINT(41.188254832280556 73.06788258902837) bank135029 +135030 POINT(41.34767629826675 73.05734202902379) bank135030 +135031 POINT(40.16576121186941 73.81890329838298) bank135031 +135032 POINT(41.44664565952899 73.52245550533264) bank135032 +135033 POINT(40.65892752930479 74.83783216086943) bank135033 +135034 POINT(39.86698462281708 74.495321720927) bank135034 +135035 POINT(39.941689158788854 73.09155215864385) bank135035 +135036 POINT(41.68807352777553 73.46331621083637) bank135036 +135037 POINT(40.79468963863422 74.3312056728366) bank135037 +135038 POINT(41.58817723365738 74.39560648066279) bank135038 +135039 POINT(40.064340882604974 73.16739204548573) bank135039 +135040 POINT(41.50493788292332 74.90256896306408) bank135040 +135041 POINT(40.75142843173444 73.31800953059094) bank135041 +135042 POINT(39.9564564473236 73.43816869495228) bank135042 +135043 POINT(40.1731798201578 74.89470902759548) bank135043 +135044 POINT(41.594720891448375 74.65894917287874) bank135044 +135045 POINT(41.016542370688384 74.89474996295183) bank135045 +135046 POINT(40.07837739993695 73.20195578550819) bank135046 +135047 POINT(41.27493706343357 73.23062398827473) bank135047 +135048 POINT(40.802605916989776 74.89710126948049) bank135048 +135049 POINT(41.40336819192416 73.60905761851944) bank135049 +135050 POINT(41.49332537351915 73.0989080036202) bank135050 +135051 POINT(41.01419857814925 74.81760607717398) bank135051 +135052 POINT(40.89613828922039 73.0234612452518) bank135052 +135053 POINT(39.806043097411404 74.28692768256472) bank135053 +135054 POINT(40.26756487857679 73.97607321527752) bank135054 +135055 POINT(40.163853339208295 73.57745216705256) bank135055 +135056 POINT(41.57909776229285 73.9057038995499) bank135056 +135057 POINT(40.51294305743814 73.62110741365285) bank135057 +135058 POINT(41.389993433642346 74.14391836912792) bank135058 +135059 POINT(39.78589082954267 73.54719183457694) bank135059 +135060 POINT(40.85470159357598 73.28210648171107) bank135060 +135061 POINT(40.10368667632459 73.49930651534667) bank135061 +135062 POINT(40.45198937535937 73.95381479922524) bank135062 +135063 POINT(40.020346728111804 74.40533302979644) bank135063 +135064 POINT(40.970679438894884 73.14704166896712) bank135064 +135065 POINT(41.59316498658213 74.6685093072555) bank135065 +135066 POINT(40.7429884770845 73.32662657160209) bank135066 +135067 POINT(40.006129986039504 73.78662417641902) bank135067 +135068 POINT(39.99567063674761 73.73384600823171) bank135068 +135069 POINT(40.296236443626384 73.9128698905863) bank135069 +135070 POINT(39.873291583307704 73.77084641848565) bank135070 +135071 POINT(41.23622985524532 73.94186890872619) bank135071 +135072 POINT(41.48981020907388 73.54173481282938) bank135072 +135073 POINT(41.382285728422005 74.70978608331875) bank135073 +135074 POINT(40.821692953957104 74.2975695880474) bank135074 +135075 POINT(40.36275626925137 74.65870409556682) bank135075 +135076 POINT(41.17911444450366 74.49410245872123) bank135076 +135077 POINT(40.903514822480034 73.3006738117156) bank135077 +135078 POINT(41.30906269132447 74.46483928950386) bank135078 +135079 POINT(41.625844175975416 73.85646344032912) bank135079 +135080 POINT(40.16877170286679 74.13949456419881) bank135080 +135081 POINT(40.57829508556216 73.93662878105613) bank135081 +135082 POINT(39.74099812918989 74.84872643535634) bank135082 +135083 POINT(40.37604063998425 73.4269839768527) bank135083 +135084 POINT(40.02811367427434 73.19140905525524) bank135084 +135085 POINT(41.43011467017478 73.647326557055) bank135085 +135086 POINT(41.03887868410292 73.57200842952606) bank135086 +135087 POINT(39.73066426955026 74.67332618760555) bank135087 +135088 POINT(39.90129467871552 73.7765543912611) bank135088 +135089 POINT(41.0233835861305 74.10105123740985) bank135089 +135090 POINT(40.01694902693265 73.06657327139204) bank135090 +135091 POINT(41.43982017090722 74.33111521904496) bank135091 +135092 POINT(40.552355124667564 74.2803665653256) bank135092 +135093 POINT(39.90376497678314 74.74288047820905) bank135093 +135094 POINT(41.34107685071782 73.10378326241573) bank135094 +135095 POINT(41.587660942461596 73.2700420895865) bank135095 +135096 POINT(41.12996960523735 73.10655688185622) bank135096 +135097 POINT(40.84293406824006 73.91707713951521) bank135097 +135098 POINT(41.5469209078874 73.06683234695251) bank135098 +135099 POINT(41.4555302487087 74.4268206755417) bank135099 +135100 POINT(39.90651121612919 74.3711543553154) bank135100 +135101 POINT(41.03953878647632 74.86351160302763) bank135101 +135102 POINT(40.04607036691812 74.41797660929545) bank135102 +135103 POINT(41.23767611736148 73.84590090324467) bank135103 +135104 POINT(40.28729179197183 73.80137105465735) bank135104 +135105 POINT(41.27162854392248 73.5466027975884) bank135105 +135106 POINT(41.369459787009994 74.85522063858852) bank135106 +135107 POINT(39.816190367139534 74.00488062044928) bank135107 +135108 POINT(40.18699171083011 74.43321750511707) bank135108 +135109 POINT(41.69290800000786 73.8524947553207) bank135109 +135110 POINT(41.24958848909271 74.13080309670138) bank135110 +135111 POINT(40.756356379554035 73.63109552400742) bank135111 +135112 POINT(40.23669690220416 74.2391022341101) bank135112 +135113 POINT(41.04713476397337 74.97632803486817) bank135113 +135114 POINT(40.6321260359019 73.20768071688184) bank135114 +135115 POINT(40.34975123228358 74.78978636949903) bank135115 +135116 POINT(41.378239539638464 73.4176908001044) bank135116 +135117 POINT(41.57886005524851 73.12278866498718) bank135117 +135118 POINT(40.6456913552214 73.03164295526517) bank135118 +135119 POINT(40.91350152578437 73.16137093731479) bank135119 +135120 POINT(40.73787826149468 74.59841403868445) bank135120 +135121 POINT(40.92818956523447 74.17916920071492) bank135121 +135122 POINT(41.205397894052616 74.96682855287183) bank135122 +135123 POINT(40.1752072291874 74.59129559828435) bank135123 +135124 POINT(41.60974887354803 74.69245932883078) bank135124 +135125 POINT(40.05220844255385 74.75620167473616) bank135125 +135126 POINT(40.02674134921286 74.566216820115) bank135126 +135127 POINT(39.78332872532349 74.04997492577077) bank135127 +135128 POINT(41.5185468750412 73.19085337264315) bank135128 +135129 POINT(40.469817629031894 74.16487052177543) bank135129 +135130 POINT(40.326291512293345 73.2298161939448) bank135130 +135131 POINT(41.22648818266181 73.67317363167761) bank135131 +135132 POINT(40.619522189972386 73.46094156282321) bank135132 +135133 POINT(41.13265573318284 73.25350783726712) bank135133 +135134 POINT(39.93788634660768 74.01134139683623) bank135134 +135135 POINT(40.942456321857264 74.17067498883364) bank135135 +135136 POINT(41.66487556304398 73.21933131582831) bank135136 +135137 POINT(40.74710008122797 74.60789475833674) bank135137 +135138 POINT(41.52308275728076 73.62390597728172) bank135138 +135139 POINT(40.485849903847125 74.71800624848463) bank135139 +135140 POINT(39.96924634438976 74.44517727742252) bank135140 +135141 POINT(39.98732743663508 74.82503972522925) bank135141 +135142 POINT(40.2565275429011 73.28373402722393) bank135142 +135143 POINT(40.838373692535754 73.92010064580215) bank135143 +135144 POINT(40.35278383802809 73.29462650160309) bank135144 +135145 POINT(40.886865909495725 74.68861459696552) bank135145 +135146 POINT(40.95170399115203 73.6010104967113) bank135146 +135147 POINT(40.52039387070065 74.58673728003112) bank135147 +135148 POINT(41.5687788074363 73.21161688096477) bank135148 +135149 POINT(40.333142564970565 74.73563733453456) bank135149 +135150 POINT(40.45511208514691 74.40491851013722) bank135150 +135151 POINT(40.654426853873865 74.20394705911535) bank135151 +135152 POINT(41.50602436442409 73.08282298895283) bank135152 +135153 POINT(39.937549939022176 74.75565082372356) bank135153 +135154 POINT(40.43858147656715 73.19739679524606) bank135154 +135155 POINT(40.17340685127381 74.0895901173161) bank135155 +135156 POINT(41.354041535320256 74.64398069750268) bank135156 +135157 POINT(39.74285776863912 74.29387008809952) bank135157 +135158 POINT(40.63089693008256 73.17575742924437) bank135158 +135159 POINT(39.9991337095646 73.62222806003972) bank135159 +135160 POINT(39.774710649168085 74.55791927271932) bank135160 +135161 POINT(39.8965354195775 74.94468969803088) bank135161 +135162 POINT(39.86680787983679 73.93512282510925) bank135162 +135163 POINT(40.12196870323884 74.81611367928872) bank135163 +135164 POINT(40.329349571802716 74.61590470084062) bank135164 +135165 POINT(40.744428186948895 74.26616444476936) bank135165 +135166 POINT(40.97960855885581 73.9370629138138) bank135166 +135167 POINT(40.41073117723436 74.01720852628155) bank135167 +135168 POINT(39.81911675518059 73.48696809793891) bank135168 +135169 POINT(40.64391490547989 74.13528507794562) bank135169 +135170 POINT(40.503299159162694 74.02535543340933) bank135170 +135171 POINT(40.86760586622245 74.76308231807724) bank135171 +135172 POINT(40.844956483018535 74.82946342166882) bank135172 +135173 POINT(40.957353699664125 73.20587620631579) bank135173 +135174 POINT(39.98167166046598 73.63847321673845) bank135174 +135175 POINT(41.396887754329455 74.24595020715078) bank135175 +135176 POINT(40.32482767649053 73.19739496132894) bank135176 +135177 POINT(40.791786229190166 73.54371941735123) bank135177 +135178 POINT(41.281572859417565 73.43792021206224) bank135178 +135179 POINT(40.15845423138853 74.63199418013275) bank135179 +135180 POINT(41.5192633416137 74.78194041012688) bank135180 +135181 POINT(40.856527250007936 73.98468639412556) bank135181 +135182 POINT(41.634236545043294 73.29890017082926) bank135182 +135183 POINT(39.71549769834854 74.53678814537349) bank135183 +135184 POINT(39.92302136468426 74.66881865840938) bank135184 +135185 POINT(41.176054254919336 73.37606813104193) bank135185 +135186 POINT(41.1284184124391 73.31793566672283) bank135186 +135187 POINT(40.62326470010682 74.06132600155249) bank135187 +135188 POINT(40.31282108964006 73.8848169632501) bank135188 +135189 POINT(41.57111293251558 74.90108050674974) bank135189 +135190 POINT(40.136292681341885 74.82575304996752) bank135190 +135191 POINT(40.119641421413235 73.46755075146599) bank135191 +135192 POINT(41.01274363312932 74.6689891487432) bank135192 +135193 POINT(41.16266284918386 74.43810611384458) bank135193 +135194 POINT(40.31918458600473 74.22478642930176) bank135194 +135195 POINT(39.86482895447916 73.98064765168738) bank135195 +135196 POINT(41.1245757423485 74.93799713412683) bank135196 +135197 POINT(41.21797323751044 74.07348651099676) bank135197 +135198 POINT(40.727553468898826 74.05715280058011) bank135198 +135199 POINT(41.30101066376051 73.24110844331335) bank135199 +135200 POINT(39.84608182752429 74.698068195223) bank135200 +135201 POINT(40.228638792034516 73.78772081882039) bank135201 +135202 POINT(41.39658400952959 73.08243934786383) bank135202 +135203 POINT(41.688243333239825 73.17367103428748) bank135203 +135204 POINT(41.1202696110515 73.82413007029525) bank135204 +135205 POINT(40.96444170514012 73.52031716951896) bank135205 +135206 POINT(40.35088354619028 74.14002179892145) bank135206 +135207 POINT(40.19522954627166 73.79686343950692) bank135207 +135208 POINT(40.647703192573545 73.53439055956804) bank135208 +135209 POINT(40.499559939457136 73.42275723868323) bank135209 +135210 POINT(40.475486451057314 73.79942786988646) bank135210 +135211 POINT(40.35093969328068 73.35547052953069) bank135211 +135212 POINT(41.468813901518544 73.21061924684267) bank135212 +135213 POINT(40.1266158253925 73.24222835139204) bank135213 +135214 POINT(41.69448413940947 74.45183371464441) bank135214 +135215 POINT(40.76623881768865 73.03546452783597) bank135215 +135216 POINT(40.3825134794785 73.05424972637118) bank135216 +135217 POINT(41.2350058551952 74.22570325478142) bank135217 +135218 POINT(40.34424812138104 73.08967052317426) bank135218 +135219 POINT(39.82436593006035 73.82576434411857) bank135219 +135220 POINT(39.980667738072334 74.21082398277959) bank135220 +135221 POINT(40.4389032948358 73.77202365253014) bank135221 +135222 POINT(39.87113163240977 73.2259770513117) bank135222 +135223 POINT(39.75022498432972 74.2080211660079) bank135223 +135224 POINT(40.608543736662824 74.255641027889) bank135224 +135225 POINT(40.811326880848995 73.3848366184992) bank135225 +135226 POINT(40.151848323525265 73.45584404029316) bank135226 +135227 POINT(41.65454224850086 74.20998694121404) bank135227 +135228 POINT(41.3140454803901 73.21162776883033) bank135228 +135229 POINT(40.68069343805548 74.7470476292221) bank135229 +135230 POINT(39.989456153603385 74.99505233722387) bank135230 +135231 POINT(40.86584755188794 74.35699000947609) bank135231 +135232 POINT(41.293192712663306 74.19068286033271) bank135232 +135233 POINT(40.01784245784537 73.65857136864878) bank135233 +135234 POINT(39.72843528647566 73.91626021613811) bank135234 +135235 POINT(41.25983400129818 73.33853542263722) bank135235 +135236 POINT(40.17458822202224 74.41222754900888) bank135236 +135237 POINT(39.926721331716585 73.22413597472786) bank135237 +135238 POINT(39.834669860360854 74.21201993960628) bank135238 +135239 POINT(41.60675110080803 74.49830768418568) bank135239 +135240 POINT(39.78224956756669 73.15102468508286) bank135240 +135241 POINT(41.27386850409397 73.90029792782003) bank135241 +135242 POINT(40.14821916677553 74.82286403396832) bank135242 +135243 POINT(40.44070204933587 73.2739378835787) bank135243 +135244 POINT(41.63027453350355 74.14736032695505) bank135244 +135245 POINT(41.28216696751775 73.30463855892512) bank135245 +135246 POINT(40.17178915978168 73.66792962986598) bank135246 +135247 POINT(40.920405095763954 74.08069025142923) bank135247 +135248 POINT(41.691637741018766 74.02498737173163) bank135248 +135249 POINT(41.66639183852932 73.54272505815723) bank135249 +135250 POINT(40.170098640735795 74.54351677702333) bank135250 +135251 POINT(39.86032529426491 73.72594570435957) bank135251 +135252 POINT(41.564987569815976 73.094706906192) bank135252 +135253 POINT(41.61454843769051 74.67620382078134) bank135253 +135254 POINT(40.30143670155446 73.97030824962005) bank135254 +135255 POINT(40.712175626379214 74.20833398321403) bank135255 +135256 POINT(39.713219441511846 74.0670390700386) bank135256 +135257 POINT(40.739212868225536 73.62028951797133) bank135257 +135258 POINT(39.719332934824685 73.04986387266983) bank135258 +135259 POINT(40.63677690745503 74.5969245577166) bank135259 +135260 POINT(40.883700733666174 74.65552692968926) bank135260 +135261 POINT(40.28201479320094 73.17579062233199) bank135261 +135262 POINT(41.57757382721289 73.78049934619774) bank135262 +135263 POINT(41.30185610918618 73.13138684354895) bank135263 +135264 POINT(39.80128542362829 73.89656071945518) bank135264 +135265 POINT(40.2888754429279 73.68054327216103) bank135265 +135266 POINT(41.672028290543494 73.04289352802778) bank135266 +135267 POINT(41.400086404180925 74.80206331707274) bank135267 +135268 POINT(40.02791694826131 74.81637701616928) bank135268 +135269 POINT(41.56632817129733 74.08126716880108) bank135269 +135270 POINT(40.75998202804793 74.91351763972166) bank135270 +135271 POINT(40.44956061477749 73.82387981272566) bank135271 +135272 POINT(40.22697374595738 73.95733594064869) bank135272 +135273 POINT(41.2705056938357 74.67050449169614) bank135273 +135274 POINT(41.43470708180846 73.95303302054722) bank135274 +135275 POINT(41.44190716763512 74.69084116343655) bank135275 +135276 POINT(41.524225606088606 73.01284536945136) bank135276 +135277 POINT(41.165785970399355 74.21709175642711) bank135277 +135278 POINT(39.85785366892849 74.76279606525655) bank135278 +135279 POINT(41.37710220018541 73.31352177976638) bank135279 +135280 POINT(40.59806553727304 74.25985862852531) bank135280 +135281 POINT(40.12371824842128 74.60066383900894) bank135281 +135282 POINT(40.1203240677548 74.31178697176439) bank135282 +135283 POINT(41.041569176552144 74.99351427408727) bank135283 +135284 POINT(40.30284864889521 73.30953023532365) bank135284 +135285 POINT(39.82145912088109 73.36382525255364) bank135285 +135286 POINT(40.97864151762977 74.21970724530648) bank135286 +135287 POINT(41.51960866973242 74.30084080844865) bank135287 +135288 POINT(40.90927501152307 74.91317944488199) bank135288 +135289 POINT(40.518247060064105 74.65419109008236) bank135289 +135290 POINT(40.00289097099737 74.42188584674872) bank135290 +135291 POINT(40.112878801446705 73.5649758791027) bank135291 +135292 POINT(40.48831746876007 74.76465250301514) bank135292 +135293 POINT(40.6731889924797 74.32663705845347) bank135293 +135294 POINT(41.15237251874918 74.61348146086843) bank135294 +135295 POINT(41.29202127795835 74.18786509978656) bank135295 +135296 POINT(39.772791520741585 74.37416726869009) bank135296 +135297 POINT(41.529250957977034 73.8611462441388) bank135297 +135298 POINT(39.86444314858192 74.36881247929479) bank135298 +135299 POINT(40.33634371379085 73.37786055298885) bank135299 +135300 POINT(40.74101427407517 73.13294802989115) bank135300 +135301 POINT(40.25747447925269 73.31938153587289) bank135301 +135302 POINT(40.704818881591464 74.90082675701466) bank135302 +135303 POINT(40.88220948116069 73.4687918238998) bank135303 +135304 POINT(40.20340018958514 73.82872727059892) bank135304 +135305 POINT(40.48985173573182 74.6826750657777) bank135305 +135306 POINT(41.56109302920575 74.0364828829759) bank135306 +135307 POINT(41.089976430521844 73.39561957619966) bank135307 +135308 POINT(41.48308398085754 73.17987668323252) bank135308 +135309 POINT(40.74138164365363 73.90189932523202) bank135309 +135310 POINT(41.41544928522555 73.29313164985493) bank135310 +135311 POINT(41.623846963895666 74.82240855625619) bank135311 +135312 POINT(40.17815031637834 73.94376589923738) bank135312 +135313 POINT(41.13351657066489 73.59554521552103) bank135313 +135314 POINT(40.875167999868786 74.6410848274577) bank135314 +135315 POINT(40.188077685283986 73.60936267342403) bank135315 +135316 POINT(41.21330313330492 73.66840876323216) bank135316 +135317 POINT(41.0901833024994 74.15451998042356) bank135317 +135318 POINT(41.424539871047976 73.20321246601688) bank135318 +135319 POINT(41.30851463189206 74.29965839263897) bank135319 +135320 POINT(41.07765595194409 73.10881486143136) bank135320 +135321 POINT(41.711932432230014 74.06240228674503) bank135321 +135322 POINT(41.04093384626182 73.86229673634467) bank135322 +135323 POINT(40.28581970199793 73.45141659751664) bank135323 +135324 POINT(40.06040201503108 73.46739225328153) bank135324 +135325 POINT(40.555179804735914 73.63722621148845) bank135325 +135326 POINT(40.6435896654936 74.78204299735708) bank135326 +135327 POINT(40.04473844910154 73.85900335792438) bank135327 +135328 POINT(40.20877683190688 73.20653385896297) bank135328 +135329 POINT(40.52615023265051 73.7926789759869) bank135329 +135330 POINT(40.21858078934331 74.63393039827415) bank135330 +135331 POINT(40.627095561881625 74.78690379703772) bank135331 +135332 POINT(40.16718998468384 73.6658983571917) bank135332 +135333 POINT(41.19399397312861 73.41150949900779) bank135333 +135334 POINT(41.415557202951426 73.09354981513661) bank135334 +135335 POINT(41.40613056739135 74.89516684333634) bank135335 +135336 POINT(41.53449696809284 73.5562601481633) bank135336 +135337 POINT(40.62537463720397 73.39099816211042) bank135337 +135338 POINT(39.73977450491277 73.9852717209746) bank135338 +135339 POINT(41.693754294639085 73.57613618268176) bank135339 +135340 POINT(41.1569045316449 73.66054137346714) bank135340 +135341 POINT(41.01015121819261 74.71495976086628) bank135341 +135342 POINT(40.04248431462122 73.74770198028223) bank135342 +135343 POINT(41.08708268368668 74.66538474378278) bank135343 +135344 POINT(40.38788551349159 74.41048538552579) bank135344 +135345 POINT(40.887106662944895 74.13676946945449) bank135345 +135346 POINT(41.57818493875473 74.60149856411063) bank135346 +135347 POINT(41.67071849129441 73.11056551602387) bank135347 +135348 POINT(41.24176989163212 73.77190294222413) bank135348 +135349 POINT(40.41105811193223 73.70723167544085) bank135349 +135350 POINT(41.044660578758574 73.68482131617756) bank135350 +135351 POINT(41.30958261048648 74.79088136182804) bank135351 +135352 POINT(41.25080597295506 73.2711263007619) bank135352 +135353 POINT(41.22163771467558 73.13959713936929) bank135353 +135354 POINT(39.84206879922023 73.62958340641862) bank135354 +135355 POINT(41.38091069442433 73.86898003106103) bank135355 +135356 POINT(41.244947387046544 73.18579281181542) bank135356 +135357 POINT(41.69468356954991 73.45230906975574) bank135357 +135358 POINT(40.77655289836384 74.37653858262738) bank135358 +135359 POINT(41.46435319063058 74.77215597897002) bank135359 +135360 POINT(41.44771988490896 74.6525756413255) bank135360 +135361 POINT(40.750521134809325 74.2535897609825) bank135361 +135362 POINT(41.524693699262 74.42403301593967) bank135362 +135363 POINT(39.98464365011147 74.6531050662393) bank135363 +135364 POINT(41.1456934618928 73.23818854697218) bank135364 +135365 POINT(40.98148519556035 74.75586630334504) bank135365 +135366 POINT(41.23086850777354 74.40146016513476) bank135366 +135367 POINT(40.48961770008943 74.71704349322391) bank135367 +135368 POINT(40.96061251350276 74.42794774594846) bank135368 +135369 POINT(40.165294612796 74.78555806157144) bank135369 +135370 POINT(41.683660762530565 73.09358627137945) bank135370 +135371 POINT(40.28018802865311 73.91574271978136) bank135371 +135372 POINT(40.425683703242846 74.89921877288687) bank135372 +135373 POINT(39.9500629649835 73.71395745069712) bank135373 +135374 POINT(41.50240647927915 73.3469168230464) bank135374 +135375 POINT(41.312091703749886 74.71816505130138) bank135375 +135376 POINT(40.62606714303121 74.79540711309015) bank135376 +135377 POINT(39.91740343629035 73.91072934850534) bank135377 +135378 POINT(40.13579343698803 74.84408506278831) bank135378 +135379 POINT(40.206649542928766 73.67823437427327) bank135379 +135380 POINT(41.05476483401754 73.47391553634819) bank135380 +135381 POINT(41.335355400942866 74.61782608530558) bank135381 +135382 POINT(40.597894501885236 73.31534542675489) bank135382 +135383 POINT(40.08287917182533 74.952467621238) bank135383 +135384 POINT(40.94123640616842 74.36175748566366) bank135384 +135385 POINT(41.023684902943586 74.93563652046673) bank135385 +135386 POINT(40.69676168240995 73.83109538756142) bank135386 +135387 POINT(39.952590968736025 73.94433228712735) bank135387 +135388 POINT(40.22448713604782 74.84636099268117) bank135388 +135389 POINT(41.16542289325242 73.10210716945643) bank135389 +135390 POINT(40.5112093325205 74.23950717790625) bank135390 +135391 POINT(41.45845589651386 74.98232540075342) bank135391 +135392 POINT(40.177006100368274 74.15629023928894) bank135392 +135393 POINT(39.79154025755756 74.84138754053933) bank135393 +135394 POINT(40.07606092552011 73.87720004713009) bank135394 +135395 POINT(40.97131481352962 74.80370000320339) bank135395 +135396 POINT(40.344678532601954 73.95789778967368) bank135396 +135397 POINT(39.960322799092566 73.19804279402037) bank135397 +135398 POINT(40.16540302499207 74.1502406858515) bank135398 +135399 POINT(41.105162298804444 73.93374268289031) bank135399 +135400 POINT(40.91037654955097 73.14845256631733) bank135400 +135401 POINT(41.007718265566865 73.72531203834983) bank135401 +135402 POINT(39.82230323052853 74.75251061105644) bank135402 +135403 POINT(41.3760055720914 73.24630568752214) bank135403 +135404 POINT(41.08276254287694 74.34783646415478) bank135404 +135405 POINT(40.891820240587975 74.24912161480395) bank135405 +135406 POINT(41.058673130752894 74.78930187683083) bank135406 +135407 POINT(41.68049442624207 74.05998081746321) bank135407 +135408 POINT(40.97732455959726 73.94965833865817) bank135408 +135409 POINT(41.16703335715988 74.70642933755158) bank135409 +135410 POINT(41.41106116411131 73.02334135351907) bank135410 +135411 POINT(40.03534499419588 74.18103408031655) bank135411 +135412 POINT(40.98683104528222 73.68458669949634) bank135412 +135413 POINT(40.223146658560395 74.21785801075383) bank135413 +135414 POINT(41.57462525159312 74.9633289386074) bank135414 +135415 POINT(41.30018425691844 74.65073710636166) bank135415 +135416 POINT(40.12368813116295 73.51596343689629) bank135416 +135417 POINT(40.29352868916895 73.56025149206621) bank135417 +135418 POINT(39.97675220591777 73.35963986210741) bank135418 +135419 POINT(40.39748127155099 74.30066268815786) bank135419 +135420 POINT(41.512323706791825 74.82348502210009) bank135420 +135421 POINT(40.307523860275005 73.37425931351913) bank135421 +135422 POINT(40.480720889588085 74.1390287898495) bank135422 +135423 POINT(40.353777087762644 73.80882775146) bank135423 +135424 POINT(41.446873581240055 73.50836071095067) bank135424 +135425 POINT(40.06434759156161 74.29194710989188) bank135425 +135426 POINT(40.130518559886546 74.79565733978295) bank135426 +135427 POINT(41.472347033244255 73.53932111350856) bank135427 +135428 POINT(41.56283647967983 73.00959803365612) bank135428 +135429 POINT(39.90667948993172 73.96646877798084) bank135429 +135430 POINT(41.235587961085066 73.67037792962547) bank135430 +135431 POINT(40.224725658818436 74.52569554841772) bank135431 +135432 POINT(40.7907463052348 73.39940916396078) bank135432 +135433 POINT(40.79974740015917 73.64031371659955) bank135433 +135434 POINT(40.72181489177482 74.32314636919126) bank135434 +135435 POINT(39.95644311933734 74.77684362189062) bank135435 +135436 POINT(40.042840539456634 74.33526840025029) bank135436 +135437 POINT(39.81196109707852 74.38864831213544) bank135437 +135438 POINT(40.587021288422726 73.91843290463417) bank135438 +135439 POINT(41.25523647475514 73.17553771910342) bank135439 +135440 POINT(41.086151137220526 73.4962900982419) bank135440 +135441 POINT(40.792060085448185 74.10136399530082) bank135441 +135442 POINT(41.67807742807349 74.73157159694829) bank135442 +135443 POINT(39.90636068665434 74.28530751559548) bank135443 +135444 POINT(41.47609617417357 73.46228633160032) bank135444 +135445 POINT(40.319576599733196 74.61680760290649) bank135445 +135446 POINT(41.53256068527408 74.39485584396431) bank135446 +135447 POINT(41.15962510109984 73.7433634156492) bank135447 +135448 POINT(41.508138958665285 74.93776707819407) bank135448 +135449 POINT(41.57029403907603 73.75182514370505) bank135449 +135450 POINT(40.340053571613396 73.1080245543428) bank135450 +135451 POINT(40.75118164442644 73.60912273533991) bank135451 +135452 POINT(41.42309946436775 74.48145119105205) bank135452 +135453 POINT(39.90717987334186 73.12681016599304) bank135453 +135454 POINT(40.04415115939066 74.41742994751863) bank135454 +135455 POINT(39.79900601172504 74.63067482803854) bank135455 +135456 POINT(41.680683612547945 74.73766069224122) bank135456 +135457 POINT(40.190861784259745 73.81474441321336) bank135457 +135458 POINT(39.88886714098392 73.97492151060123) bank135458 +135459 POINT(40.95620661717527 73.41841225395981) bank135459 +135460 POINT(40.03076614185556 74.83959579360922) bank135460 +135461 POINT(41.59639572639775 74.78005716723126) bank135461 +135462 POINT(39.90534151265187 73.04717084455912) bank135462 +135463 POINT(40.35021815639445 73.92839604992778) bank135463 +135464 POINT(40.115588557569055 73.98278379995514) bank135464 +135465 POINT(40.56865545020071 74.97126160546678) bank135465 +135466 POINT(40.10161698114426 74.45058167237781) bank135466 +135467 POINT(41.46211403521022 74.32920421179152) bank135467 +135468 POINT(40.85845979249547 73.96555421409673) bank135468 +135469 POINT(41.02750150907822 74.57389926614172) bank135469 +135470 POINT(40.11430685541917 74.3681591089648) bank135470 +135471 POINT(40.19655874297172 73.94689095924784) bank135471 +135472 POINT(40.89420468050013 73.28019103135023) bank135472 +135473 POINT(40.54768391237264 74.40449735763582) bank135473 +135474 POINT(39.92413701370613 74.2639695813626) bank135474 +135475 POINT(40.44153045103449 74.72838780125048) bank135475 +135476 POINT(40.06413820372476 73.9416260402929) bank135476 +135477 POINT(41.12921500957162 74.7055892126658) bank135477 +135478 POINT(40.677982219579604 74.54043113482739) bank135478 +135479 POINT(41.097857538361 74.59585524412894) bank135479 +135480 POINT(40.318880894425924 74.13557816348691) bank135480 +135481 POINT(41.14116596597026 74.89246001205109) bank135481 +135482 POINT(40.91880544269919 74.81848511670788) bank135482 +135483 POINT(40.8339326879996 74.53524678860018) bank135483 +135484 POINT(40.453358206596285 73.04480664726266) bank135484 +135485 POINT(41.45024099602615 74.209421346837) bank135485 +135486 POINT(41.083385866998015 74.7883418486858) bank135486 +135487 POINT(40.24236330885861 73.37731711854883) bank135487 +135488 POINT(41.013325679484765 73.60041094977773) bank135488 +135489 POINT(41.24978784252406 73.03447294153483) bank135489 +135490 POINT(39.85729828833725 74.56422061652248) bank135490 +135491 POINT(41.376897688607116 73.44756335304439) bank135491 +135492 POINT(40.13706014446595 73.10262110647446) bank135492 +135493 POINT(41.36774005721817 73.61799329371046) bank135493 +135494 POINT(41.34900423822718 74.23755796838533) bank135494 +135495 POINT(41.18122104463451 73.22364829362871) bank135495 +135496 POINT(40.45139900864363 73.38282973018113) bank135496 +135497 POINT(40.70866261144231 74.17446063771425) bank135497 +135498 POINT(40.83381866473542 73.38462809377549) bank135498 +135499 POINT(39.73704788884029 74.52091046194273) bank135499 +135500 POINT(39.828168111049784 74.06036038946773) bank135500 +135501 POINT(40.71015909719572 73.90969363023221) bank135501 +135502 POINT(40.62562500210032 73.1083460946432) bank135502 +135503 POINT(40.64770333943665 73.0760495427303) bank135503 +135504 POINT(41.3901217366645 74.58338154219275) bank135504 +135505 POINT(40.02530937928542 73.19861204265659) bank135505 +135506 POINT(40.4120647376403 74.41141482030962) bank135506 +135507 POINT(41.394667477999064 74.3946409125703) bank135507 +135508 POINT(40.925255002261984 73.6404313208976) bank135508 +135509 POINT(41.05861974056876 73.80920464603443) bank135509 +135510 POINT(40.7734120615783 73.50017943848916) bank135510 +135511 POINT(40.361268871555815 73.15118658270808) bank135511 +135512 POINT(40.547157976348444 74.00765368207261) bank135512 +135513 POINT(41.4896703514384 73.50321770423378) bank135513 +135514 POINT(40.0072979831269 73.15766477615807) bank135514 +135515 POINT(41.629993710098404 74.59482761008961) bank135515 +135516 POINT(41.27719636271259 73.93331425426315) bank135516 +135517 POINT(40.96742115899763 73.19075087634202) bank135517 +135518 POINT(41.70058479519034 74.84900164312216) bank135518 +135519 POINT(40.671857888330365 74.35832330644759) bank135519 +135520 POINT(39.72871584213892 73.02736990246879) bank135520 +135521 POINT(40.70539819474404 73.03249523257809) bank135521 +135522 POINT(40.342149636388854 74.32287156475546) bank135522 +135523 POINT(41.25719355046062 74.3041830492513) bank135523 +135524 POINT(41.02565570452529 74.32533432837144) bank135524 +135525 POINT(39.83384942011111 73.69932671304892) bank135525 +135526 POINT(41.00010993785825 74.1873250054281) bank135526 +135527 POINT(40.584833523713044 73.97451115513721) bank135527 +135528 POINT(40.40852436157828 74.45853202987426) bank135528 +135529 POINT(39.9136752224851 74.15509434744635) bank135529 +135530 POINT(40.69599255584288 74.74087297154227) bank135530 +135531 POINT(41.15667295098061 74.70445832570253) bank135531 +135532 POINT(41.047777056527615 74.70801661934682) bank135532 +135533 POINT(41.578851483357525 73.81034372380844) bank135533 +135534 POINT(41.38052649580126 73.03087891517957) bank135534 +135535 POINT(40.1542111212453 74.05559718503329) bank135535 +135536 POINT(39.76564514532216 73.90258194249928) bank135536 +135537 POINT(40.366895088152845 73.04971985936812) bank135537 +135538 POINT(40.56343412564388 73.08415893117508) bank135538 +135539 POINT(41.28632979566894 73.3925275769085) bank135539 +135540 POINT(39.7890262282659 74.65650485207185) bank135540 +135541 POINT(40.95059316295781 73.25258402203434) bank135541 +135542 POINT(40.84780568403255 74.76702946332539) bank135542 +135543 POINT(39.71464584401999 74.30796810526472) bank135543 +135544 POINT(39.775909972175256 74.5672872293392) bank135544 +135545 POINT(40.17443556499145 73.01699626431422) bank135545 +135546 POINT(40.595718749317996 74.50437036135996) bank135546 +135547 POINT(40.952628978946485 73.62138606314645) bank135547 +135548 POINT(41.073604905445144 73.68831272396189) bank135548 +135549 POINT(40.074848909231626 74.98313461871591) bank135549 +135550 POINT(40.53162336129176 73.17362951503826) bank135550 +135551 POINT(41.28788796802216 73.38969034152264) bank135551 +135552 POINT(41.42924248064686 74.66559434154043) bank135552 +135553 POINT(41.660654811221036 74.67432106857136) bank135553 +135554 POINT(41.102689450478465 73.11971262557066) bank135554 +135555 POINT(41.03644190061386 74.37691147279139) bank135555 +135556 POINT(39.86714936687473 73.43466879953961) bank135556 +135557 POINT(41.22892814517968 74.49128816249214) bank135557 +135558 POINT(40.77930963416125 74.4445946459357) bank135558 +135559 POINT(41.650006883746066 73.50995144985737) bank135559 +135560 POINT(39.81203270062219 73.42353048347056) bank135560 +135561 POINT(41.635554109463065 74.57058336895184) bank135561 +135562 POINT(39.86537321771751 73.5496613022188) bank135562 +135563 POINT(40.76764833485249 74.3218089568871) bank135563 +135564 POINT(41.31247506966937 73.27242657489879) bank135564 +135565 POINT(40.45646581203267 73.72596999219711) bank135565 +135566 POINT(41.22520148734604 74.16050169466222) bank135566 +135567 POINT(39.823798922064896 74.82867759246676) bank135567 +135568 POINT(40.25591516118617 74.25967735465505) bank135568 +135569 POINT(41.03555731238148 74.54983362579115) bank135569 +135570 POINT(41.38715742069992 73.71702349815314) bank135570 +135571 POINT(40.11057781507982 74.08887612007523) bank135571 +135572 POINT(41.48093747398771 74.86740254791694) bank135572 +135573 POINT(40.37139506769388 73.99951980803486) bank135573 +135574 POINT(40.972296446640826 73.63856996524201) bank135574 +135575 POINT(40.98186467410604 73.30242877493815) bank135575 +135576 POINT(40.75639786701838 74.37944951489203) bank135576 +135577 POINT(39.92472960749667 73.90624551991348) bank135577 +135578 POINT(39.728164066007956 74.0162637094651) bank135578 +135579 POINT(39.99704427556918 74.08329693625615) bank135579 +135580 POINT(40.67245328961709 74.67903540087434) bank135580 +135581 POINT(41.36172102355353 74.93883559660874) bank135581 +135582 POINT(40.009614278816244 74.82647767625633) bank135582 +135583 POINT(41.38044794566631 73.49209529962407) bank135583 +135584 POINT(40.89697316326673 74.07964814163255) bank135584 +135585 POINT(40.92113983210378 73.79299324379599) bank135585 +135586 POINT(41.36514608917925 73.81191477145427) bank135586 +135587 POINT(40.95388221743698 74.99587660186555) bank135587 +135588 POINT(40.355572314807425 73.87685783558483) bank135588 +135589 POINT(41.054448225964336 74.4106284857383) bank135589 +135590 POINT(41.68743542980072 74.65571871690369) bank135590 +135591 POINT(40.082594099744846 74.64493071518315) bank135591 +135592 POINT(40.88165968853947 73.09030506272813) bank135592 +135593 POINT(40.73191968216504 74.28126277111922) bank135593 +135594 POINT(39.88307086826652 73.61908593139086) bank135594 +135595 POINT(39.83739106524141 73.61045081346592) bank135595 +135596 POINT(40.39451402879255 73.31040558675603) bank135596 +135597 POINT(39.7423614340814 74.24784053031065) bank135597 +135598 POINT(41.07361243346444 74.51817017138728) bank135598 +135599 POINT(41.36426512108549 73.37855042492083) bank135599 +135600 POINT(39.93565414327613 73.90832920967848) bank135600 +135601 POINT(41.37363836625077 74.7247205580586) bank135601 +135602 POINT(40.14983688062755 73.08102192282448) bank135602 +135603 POINT(39.782073330462204 74.16341200284059) bank135603 +135604 POINT(41.537678808409396 73.3283899243912) bank135604 +135605 POINT(40.84404771044882 73.69382072595906) bank135605 +135606 POINT(40.7489871849855 74.72484119213308) bank135606 +135607 POINT(41.60153174314722 74.22960544907849) bank135607 +135608 POINT(40.44311650643518 74.49060362392578) bank135608 +135609 POINT(40.308983290066365 73.59978470867667) bank135609 +135610 POINT(40.178569954265384 74.86308082382203) bank135610 +135611 POINT(39.88115575059239 74.97428307115587) bank135611 +135612 POINT(40.56846890909556 73.15886182527197) bank135612 +135613 POINT(41.329086476865974 74.37685915867311) bank135613 +135614 POINT(40.172397199403896 74.02216920944845) bank135614 +135615 POINT(40.03218370858719 73.47958172435717) bank135615 +135616 POINT(41.283345113847396 74.35999191427096) bank135616 +135617 POINT(40.83088838709991 74.78343701417398) bank135617 +135618 POINT(40.20403182442865 73.48926392161464) bank135618 +135619 POINT(40.02989608933404 74.42107080082954) bank135619 +135620 POINT(40.43623923351867 73.29642568241462) bank135620 +135621 POINT(39.97687377597935 73.27163595365852) bank135621 +135622 POINT(40.86238209996991 73.92220501512745) bank135622 +135623 POINT(40.519232298188086 73.32774315088412) bank135623 +135624 POINT(39.912862645525585 74.12925843362817) bank135624 +135625 POINT(40.92092453892925 74.79087754929004) bank135625 +135626 POINT(40.680689792178654 74.68710851673082) bank135626 +135627 POINT(41.4044430089868 74.50767480087221) bank135627 +135628 POINT(40.3730682757804 73.2751408165976) bank135628 +135629 POINT(40.34120656165203 73.80422566230247) bank135629 +135630 POINT(40.88711761581081 74.70764229062202) bank135630 +135631 POINT(39.939645622549946 74.82900933036467) bank135631 +135632 POINT(41.20916026050151 74.88632178737821) bank135632 +135633 POINT(41.430471889530565 74.75428325318764) bank135633 +135634 POINT(40.93904215571073 74.33847988872505) bank135634 +135635 POINT(40.48609695017752 74.86581597287125) bank135635 +135636 POINT(39.8161672802384 74.87526817938866) bank135636 +135637 POINT(40.253303300078244 73.38619110737625) bank135637 +135638 POINT(40.104397259810575 74.6697662152455) bank135638 +135639 POINT(40.456993598123326 74.82752771331914) bank135639 +135640 POINT(40.93952827996016 73.03095202623159) bank135640 +135641 POINT(41.03156353791649 74.66111351750035) bank135641 +135642 POINT(40.18286679787654 73.17091423648453) bank135642 +135643 POINT(39.844140400995975 73.3961825036084) bank135643 +135644 POINT(39.889978815316255 74.94549082033731) bank135644 +135645 POINT(40.843985594918834 74.7466066405279) bank135645 +135646 POINT(41.05260788074179 74.11977147363996) bank135646 +135647 POINT(40.74926425862759 73.69298021178766) bank135647 +135648 POINT(41.509101129569245 73.49386816130698) bank135648 +135649 POINT(39.747418752113475 74.48904516906168) bank135649 +135650 POINT(39.847253965642494 73.90490485908428) bank135650 +135651 POINT(40.92029640599031 74.08592364500808) bank135651 +135652 POINT(40.466364757572634 73.9939141899486) bank135652 +135653 POINT(40.16695327282859 74.08402228857717) bank135653 +135654 POINT(40.39876708572973 74.5641998595263) bank135654 +135655 POINT(41.522379842536836 73.86494505170445) bank135655 +135656 POINT(41.282284929180385 73.74255031153304) bank135656 +135657 POINT(41.05751677126769 74.57179383318122) bank135657 +135658 POINT(40.73791102061676 73.10673359092746) bank135658 +135659 POINT(40.54560281381529 73.6730219165407) bank135659 +135660 POINT(40.78010696906365 73.9083741149682) bank135660 +135661 POINT(40.610836457515404 74.702930031323) bank135661 +135662 POINT(40.36978843386511 74.83870202295358) bank135662 +135663 POINT(41.33125400664166 73.55728974734762) bank135663 +135664 POINT(40.93518662022417 74.58198325062567) bank135664 +135665 POINT(40.52093536147876 74.13808092344483) bank135665 +135666 POINT(41.40669230554193 73.58675167715865) bank135666 +135667 POINT(41.52553014474303 73.92223189030881) bank135667 +135668 POINT(41.18359039026935 74.85129934610774) bank135668 +135669 POINT(40.530435504997456 74.34671551576156) bank135669 +135670 POINT(39.845732431210436 74.2380593858087) bank135670 +135671 POINT(41.57054096588495 73.48485002282082) bank135671 +135672 POINT(40.8705652506261 74.96156563760815) bank135672 +135673 POINT(41.68227080013301 73.57936122505927) bank135673 +135674 POINT(39.89665147280499 74.23896077843548) bank135674 +135675 POINT(39.983976172983674 74.16209548592826) bank135675 +135676 POINT(40.10393406654532 74.6955787992962) bank135676 +135677 POINT(41.3689609111453 74.53685895433784) bank135677 +135678 POINT(40.37627677445688 73.56747003728772) bank135678 +135679 POINT(41.07120652342581 73.35577628377294) bank135679 +135680 POINT(40.01616955697951 74.51178443014284) bank135680 +135681 POINT(40.74160437295032 74.48766571200301) bank135681 +135682 POINT(40.92123172297321 73.43404598866238) bank135682 +135683 POINT(40.56240690354059 73.03217961185129) bank135683 +135684 POINT(39.84218514269263 73.65861293691813) bank135684 +135685 POINT(40.4578346723488 74.60131868406191) bank135685 +135686 POINT(39.720025419736565 73.54631793028172) bank135686 +135687 POINT(41.55207182842569 73.21747709506852) bank135687 +135688 POINT(41.33447309823266 74.84221640127662) bank135688 +135689 POINT(41.343089484879435 74.47462593991946) bank135689 +135690 POINT(40.45677284340685 74.61896672236387) bank135690 +135691 POINT(41.23889214382077 74.02485167233021) bank135691 +135692 POINT(40.32279520012638 74.99417799921929) bank135692 +135693 POINT(40.48480929262243 74.18822495162588) bank135693 +135694 POINT(41.26908643865602 74.89138233630909) bank135694 +135695 POINT(40.45513131606606 74.79592862129967) bank135695 +135696 POINT(39.91297388882323 73.28771812311815) bank135696 +135697 POINT(40.96670245178893 74.84840671092681) bank135697 +135698 POINT(40.64547993722367 73.3227730702087) bank135698 +135699 POINT(40.24718353684057 73.38406034222287) bank135699 +135700 POINT(41.70731091049487 74.27510056910742) bank135700 +135701 POINT(40.797646747730234 74.7084967548397) bank135701 +135702 POINT(41.13374475542068 73.59957165514) bank135702 +135703 POINT(40.61022153294656 74.70291002901578) bank135703 +135704 POINT(41.34195384876287 73.0840638030165) bank135704 +135705 POINT(40.98399606725272 74.86815846122714) bank135705 +135706 POINT(41.19870763425688 73.45025953001053) bank135706 +135707 POINT(40.14262205431405 74.83015935578602) bank135707 +135708 POINT(41.35120545476621 74.58388942281778) bank135708 +135709 POINT(40.34782747394045 73.97417334639854) bank135709 +135710 POINT(40.81159293816291 74.5707261372453) bank135710 +135711 POINT(39.95145752805697 74.74754172680724) bank135711 +135712 POINT(39.811159893886725 74.11960505988252) bank135712 +135713 POINT(41.43743375874625 74.52150433268326) bank135713 +135714 POINT(41.30101400459465 73.7475543288046) bank135714 +135715 POINT(40.23264306901596 73.18970550146) bank135715 +135716 POINT(40.318369415411546 74.74116283270936) bank135716 +135717 POINT(40.46079657669856 73.59807584377597) bank135717 +135718 POINT(40.13467503621662 74.65133048233163) bank135718 +135719 POINT(40.59949450275818 73.09719031919289) bank135719 +135720 POINT(40.31392678017929 74.3673860525185) bank135720 +135721 POINT(39.71605958578048 73.49035927837195) bank135721 +135722 POINT(41.359776950175636 73.18983470821814) bank135722 +135723 POINT(41.03314909817362 74.60180047009024) bank135723 +135724 POINT(40.368317766082875 73.16524832320074) bank135724 +135725 POINT(40.37439315619056 73.01195295088657) bank135725 +135726 POINT(40.05066349222683 73.41598940302367) bank135726 +135727 POINT(41.202696418289726 74.39623044662252) bank135727 +135728 POINT(39.83950888330138 73.94219073002601) bank135728 +135729 POINT(41.28204546052103 74.51414978980078) bank135729 +135730 POINT(40.5089035169925 74.88302891186608) bank135730 +135731 POINT(39.923236786954085 73.10155109727245) bank135731 +135732 POINT(40.02234934453335 74.93891554114168) bank135732 +135733 POINT(40.364790467091176 74.33625469180095) bank135733 +135734 POINT(40.00430100406625 73.3898064590034) bank135734 +135735 POINT(40.34949754111887 74.153905728184) bank135735 +135736 POINT(40.19097011533747 74.97065146261407) bank135736 +135737 POINT(40.39354294799533 74.17296865086549) bank135737 +135738 POINT(40.1610203476132 74.91363690352493) bank135738 +135739 POINT(41.25054717173894 74.6714376986528) bank135739 +135740 POINT(40.89356187123094 74.16484526094519) bank135740 +135741 POINT(41.483461656362344 74.41842829571274) bank135741 +135742 POINT(41.66307522848125 73.69437907973568) bank135742 +135743 POINT(41.59885930373714 73.70163959475421) bank135743 +135744 POINT(40.18931864268763 73.36976245479886) bank135744 +135745 POINT(41.08084275776555 74.30169640245214) bank135745 +135746 POINT(40.669231267059 73.02879611360179) bank135746 +135747 POINT(41.06718444804645 74.76346697062087) bank135747 +135748 POINT(39.85072610171958 73.29361789863925) bank135748 +135749 POINT(40.70296368059896 73.14659294416899) bank135749 +135750 POINT(40.53982255819774 74.6237117057603) bank135750 +135751 POINT(40.90811217586659 74.07236840884367) bank135751 +135752 POINT(40.99605441322181 73.44108710261003) bank135752 +135753 POINT(41.51736438849095 73.80520406181346) bank135753 +135754 POINT(40.53545781939288 74.59796028657902) bank135754 +135755 POINT(41.24509766541949 74.97943356446885) bank135755 +135756 POINT(41.4528407269523 73.73407207194488) bank135756 +135757 POINT(41.55874850711776 74.63971434151172) bank135757 +135758 POINT(40.90776799716352 74.29793216763414) bank135758 +135759 POINT(40.7496195011267 74.9877024257089) bank135759 +135760 POINT(41.09560663995616 73.78020771898414) bank135760 +135761 POINT(41.423679868484136 73.18406000112792) bank135761 +135762 POINT(41.57664936556437 74.62804371833148) bank135762 +135763 POINT(40.40800577906511 73.37231272729045) bank135763 +135764 POINT(39.74971667956732 74.66223455940602) bank135764 +135765 POINT(41.70657462437931 74.1830940118649) bank135765 +135766 POINT(39.83309566761894 74.49147246038534) bank135766 +135767 POINT(39.98150344430429 73.10289729062255) bank135767 +135768 POINT(40.76608710125519 73.65604469947299) bank135768 +135769 POINT(40.58842795239611 74.49552152707484) bank135769 +135770 POINT(41.53567720484182 74.12886491905698) bank135770 +135771 POINT(41.301400440221286 74.41464667381983) bank135771 +135772 POINT(40.645821730239426 74.71441473860708) bank135772 +135773 POINT(41.688434678643766 74.83757625141624) bank135773 +135774 POINT(41.10000293194005 74.37038076605572) bank135774 +135775 POINT(41.10910435132246 73.7621674984034) bank135775 +135776 POINT(40.45737801352254 74.52461989325116) bank135776 +135777 POINT(40.96143274774308 74.90334819785105) bank135777 +135778 POINT(41.336243732079545 74.48886378537472) bank135778 +135779 POINT(40.054963714929464 73.81761423387822) bank135779 +135780 POINT(41.108941139983926 73.57815771531135) bank135780 +135781 POINT(40.930507473274425 74.27572030178946) bank135781 +135782 POINT(40.44433457929223 74.79054142659193) bank135782 +135783 POINT(41.02866577541435 74.27395987324519) bank135783 +135784 POINT(39.99856759925944 73.3062787690036) bank135784 +135785 POINT(40.30822427004786 73.16782982051816) bank135785 +135786 POINT(39.84484346009123 73.64967408357651) bank135786 +135787 POINT(39.927106439157 73.03483964480384) bank135787 +135788 POINT(41.33020245930177 73.43988233298832) bank135788 +135789 POINT(40.14801402169033 73.74610126264123) bank135789 +135790 POINT(39.725340135321424 74.59386454934608) bank135790 +135791 POINT(41.5638411005144 74.41565023984907) bank135791 +135792 POINT(41.38464979888793 74.60725458916706) bank135792 +135793 POINT(41.6379394331599 73.64346785048507) bank135793 +135794 POINT(40.50912876028006 73.21089289599591) bank135794 +135795 POINT(40.58947930377867 73.28718869824993) bank135795 +135796 POINT(40.04648987372203 74.51509474651272) bank135796 +135797 POINT(40.97522338692746 74.6041393469944) bank135797 +135798 POINT(39.92903965238045 74.28767063614337) bank135798 +135799 POINT(40.0568636567716 74.59588325915459) bank135799 +135800 POINT(40.7974694416798 73.14150308431356) bank135800 +135801 POINT(39.76358637937225 74.10735125073158) bank135801 +135802 POINT(39.97827790829324 74.49886954037154) bank135802 +135803 POINT(40.981055771151254 73.5240682778242) bank135803 +135804 POINT(41.24638086021491 74.66604037626152) bank135804 +135805 POINT(41.54165983586743 73.13902885222558) bank135805 +135806 POINT(40.693149606826715 74.82998552733763) bank135806 +135807 POINT(40.77725467788297 75.00355797546115) bank135807 +135808 POINT(40.51083236323085 73.77276909412598) bank135808 +135809 POINT(41.15746264372653 73.5610279555588) bank135809 +135810 POINT(39.71534633135821 74.4933358052448) bank135810 +135811 POINT(40.06683241307775 73.40565931879584) bank135811 +135812 POINT(40.113979583536725 73.26565520502324) bank135812 +135813 POINT(41.06344397870712 73.56271715390886) bank135813 +135814 POINT(40.608246734264014 73.4317452695038) bank135814 +135815 POINT(39.96024654090689 74.72854482567796) bank135815 +135816 POINT(40.71241126849483 74.13624964389838) bank135816 +135817 POINT(40.81388877827014 74.29135178539292) bank135817 +135818 POINT(41.46856753928718 73.67573724599163) bank135818 +135819 POINT(41.198350048100195 73.46188380654311) bank135819 +135820 POINT(41.616458137811605 74.84430161972817) bank135820 +135821 POINT(40.51323780645589 73.95822245159441) bank135821 +135822 POINT(41.602213363503324 74.52485915366782) bank135822 +135823 POINT(40.30479864089917 73.26916962287231) bank135823 +135824 POINT(40.79212274229647 73.15444709376604) bank135824 +135825 POINT(40.276964105281465 73.48077931278915) bank135825 +135826 POINT(41.00729284013495 74.32688099190185) bank135826 +135827 POINT(39.9515260434699 74.69979694732467) bank135827 +135828 POINT(41.12414373736079 74.64817657282306) bank135828 +135829 POINT(40.15217840562115 74.73248195293716) bank135829 +135830 POINT(41.09556703220991 73.80830038960667) bank135830 +135831 POINT(39.93635808650286 73.07995562895965) bank135831 +135832 POINT(40.57354414739547 73.94892090601314) bank135832 +135833 POINT(41.0451023189194 73.36036613441424) bank135833 +135834 POINT(41.223816065829396 73.71289963561362) bank135834 +135835 POINT(41.08500516804645 73.9243936976324) bank135835 +135836 POINT(40.95610635527638 73.4364344453003) bank135836 +135837 POINT(40.27483401311534 74.73074683326817) bank135837 +135838 POINT(39.996403456862126 74.25213263040648) bank135838 +135839 POINT(41.116689639826085 74.58580937790379) bank135839 +135840 POINT(41.05639779970369 73.49480059927782) bank135840 +135841 POINT(40.62059165647907 73.11415221328193) bank135841 +135842 POINT(39.827203501061014 74.82190255717637) bank135842 +135843 POINT(40.1221922819729 73.21727575000058) bank135843 +135844 POINT(40.54070769490684 73.10729478307611) bank135844 +135845 POINT(39.94523381232736 74.91685556829083) bank135845 +135846 POINT(41.342165722575466 73.51300328717622) bank135846 +135847 POINT(41.64609042752417 74.29236051728932) bank135847 +135848 POINT(40.7292938136825 74.87257427257019) bank135848 +135849 POINT(41.609464114858966 74.00978187954823) bank135849 +135850 POINT(40.5179613657849 74.00818843983753) bank135850 +135851 POINT(39.99352285833037 74.13072021854583) bank135851 +135852 POINT(40.30517398031836 73.91010790582123) bank135852 +135853 POINT(40.29351882076244 73.07290637046098) bank135853 +135854 POINT(41.51948120072249 73.47273461020653) bank135854 +135855 POINT(41.6808060632081 73.05144434660193) bank135855 +135856 POINT(40.93990308183015 74.23096903957371) bank135856 +135857 POINT(40.95422521915774 74.88935043165617) bank135857 +135858 POINT(40.31623002057078 74.73407808575867) bank135858 +135859 POINT(40.491087099210674 74.3882407557337) bank135859 +135860 POINT(40.734614475444936 73.00761538848316) bank135860 +135861 POINT(40.80909209102157 74.90617345053245) bank135861 +135862 POINT(40.54604453047727 73.55510020632961) bank135862 +135863 POINT(40.81507806035028 74.3758979196809) bank135863 +135864 POINT(41.56112287364349 74.66199777570529) bank135864 +135865 POINT(40.76598833860022 74.56163487455362) bank135865 +135866 POINT(39.875831512799685 74.13305979312342) bank135866 +135867 POINT(40.810507745398766 73.35376754110774) bank135867 +135868 POINT(41.68909299756726 73.93181774474208) bank135868 +135869 POINT(41.69243928939952 73.78275893681014) bank135869 +135870 POINT(40.63225809555052 73.8090058179915) bank135870 +135871 POINT(39.96193610706702 73.60448546546169) bank135871 +135872 POINT(40.749329321197855 73.80131306437318) bank135872 +135873 POINT(40.55002183182442 73.38363491017107) bank135873 +135874 POINT(39.77296104945605 73.1803074225754) bank135874 +135875 POINT(41.644221906396155 74.04021035362864) bank135875 +135876 POINT(41.38924152934537 74.9823616084756) bank135876 +135877 POINT(40.18867302049209 74.33318771270774) bank135877 +135878 POINT(40.03060749585311 74.0288752970455) bank135878 +135879 POINT(39.988112356561416 74.92396190067065) bank135879 +135880 POINT(40.00233636569767 73.59614353812847) bank135880 +135881 POINT(41.62465859777652 74.21419261745875) bank135881 +135882 POINT(41.5295866487507 74.18304460421767) bank135882 +135883 POINT(40.52512211217322 74.85301139600516) bank135883 +135884 POINT(40.496976054024515 73.36588756770003) bank135884 +135885 POINT(39.90230955350764 74.22102583932136) bank135885 +135886 POINT(41.21121369434945 74.47919458353736) bank135886 +135887 POINT(40.49873228130494 73.90265552016203) bank135887 +135888 POINT(40.51096121079742 73.65025142283334) bank135888 +135889 POINT(40.72342486950057 74.87221698837344) bank135889 +135890 POINT(39.89880531328795 73.85078854232599) bank135890 +135891 POINT(41.45934920392567 74.95715336258962) bank135891 +135892 POINT(39.85124678355035 73.5253145382259) bank135892 +135893 POINT(40.58833271516444 74.06208681887337) bank135893 +135894 POINT(41.373873194212 74.70218503569271) bank135894 +135895 POINT(40.42090231312375 74.72971479179388) bank135895 +135896 POINT(41.209643046345874 74.89946953755975) bank135896 +135897 POINT(40.957369142794 73.63866043333101) bank135897 +135898 POINT(40.06254841798095 74.72642391856785) bank135898 +135899 POINT(41.5720290656892 74.93214504875382) bank135899 +135900 POINT(40.05115513147102 75.00008812440001) bank135900 +135901 POINT(40.64865941844671 73.5130964608397) bank135901 +135902 POINT(40.99323305280648 73.58104363983126) bank135902 +135903 POINT(41.4324536930222 74.46791729903624) bank135903 +135904 POINT(40.526045619811626 73.9867111978817) bank135904 +135905 POINT(41.10901200055849 73.71535348037636) bank135905 +135906 POINT(41.176545378776304 73.76197923281067) bank135906 +135907 POINT(41.46204704638812 74.28364063668946) bank135907 +135908 POINT(41.293152770845005 73.25350904917595) bank135908 +135909 POINT(41.19563155284593 74.55940961419091) bank135909 +135910 POINT(39.743547273859114 73.68344133160456) bank135910 +135911 POINT(41.08288291323063 74.78299400742804) bank135911 +135912 POINT(41.08029911648025 74.53970482321182) bank135912 +135913 POINT(39.97514277833725 73.71046481205825) bank135913 +135914 POINT(40.90940250833868 74.69632166259444) bank135914 +135915 POINT(40.74616412847279 73.84160326569591) bank135915 +135916 POINT(40.08331281762394 73.6127322075539) bank135916 +135917 POINT(40.94742795943294 74.13892196849599) bank135917 +135918 POINT(40.55601476575608 74.70930557908397) bank135918 +135919 POINT(40.348719039698445 73.47664372050664) bank135919 +135920 POINT(39.9848715938525 74.83639525346933) bank135920 +135921 POINT(41.036415269927204 73.62832030413769) bank135921 +135922 POINT(41.6659749974711 74.43848188623849) bank135922 +135923 POINT(41.134391471933824 73.76601750745697) bank135923 +135924 POINT(40.35783631604228 74.20612272102193) bank135924 +135925 POINT(41.2971937343047 74.44541226715647) bank135925 +135926 POINT(40.801727207449204 73.9791666923157) bank135926 +135927 POINT(39.74650041571611 73.95957557895677) bank135927 +135928 POINT(41.50288710572594 73.81335417993975) bank135928 +135929 POINT(41.4840038906386 74.34436463201276) bank135929 +135930 POINT(41.52192630700837 74.90012082381952) bank135930 +135931 POINT(40.68300881457674 74.49960002914545) bank135931 +135932 POINT(40.311311234276864 74.50635121512366) bank135932 +135933 POINT(40.524938461550256 74.90362829776619) bank135933 +135934 POINT(40.435243156077284 73.75578370038372) bank135934 +135935 POINT(39.77626478614088 73.79768244262469) bank135935 +135936 POINT(39.76890434554502 74.42264977981608) bank135936 +135937 POINT(40.563780513707194 73.42635589794503) bank135937 +135938 POINT(41.60071176731766 73.71373017345219) bank135938 +135939 POINT(40.07454698562742 73.28199911580877) bank135939 +135940 POINT(40.34345432772726 74.69219054832718) bank135940 +135941 POINT(40.904802617557564 73.73135894724716) bank135941 +135942 POINT(40.77235832609 74.55008822641969) bank135942 +135943 POINT(41.711243857995655 73.46957029441467) bank135943 +135944 POINT(40.33798408283619 73.65836555071202) bank135944 +135945 POINT(41.443092440402296 74.79594779135623) bank135945 +135946 POINT(40.60026017210883 74.10817154783372) bank135946 +135947 POINT(39.9286540266897 74.93710506787232) bank135947 +135948 POINT(39.82246215812176 74.63259169908162) bank135948 +135949 POINT(41.534113740315384 73.34280699789403) bank135949 +135950 POINT(40.67486600635062 73.13028528727133) bank135950 +135951 POINT(39.79370393075009 73.51302236998372) bank135951 +135952 POINT(41.30970008361879 74.21490032549046) bank135952 +135953 POINT(40.44012293592948 73.36645039895663) bank135953 +135954 POINT(39.825890188498825 74.19106667877963) bank135954 +135955 POINT(40.13076292490112 73.88844598190727) bank135955 +135956 POINT(40.247097058717344 73.41740873668728) bank135956 +135957 POINT(40.201227299975194 73.26086922554671) bank135957 +135958 POINT(40.007073103490356 73.33355441000404) bank135958 +135959 POINT(40.81089377194153 74.2463431674941) bank135959 +135960 POINT(40.59947108723186 74.9992202352742) bank135960 +135961 POINT(40.65001846283905 73.92965978097646) bank135961 +135962 POINT(39.752201111720844 74.15660946694314) bank135962 +135963 POINT(41.69176167363311 73.88472948635821) bank135963 +135964 POINT(40.34647382733199 73.86465872824769) bank135964 +135965 POINT(40.734382175580095 74.48136995702173) bank135965 +135966 POINT(40.0403999042704 74.72582787052663) bank135966 +135967 POINT(40.683010086656594 74.46577407890292) bank135967 +135968 POINT(40.99437883767885 73.62013809633254) bank135968 +135969 POINT(39.9789660790473 74.38789885612151) bank135969 +135970 POINT(39.88068703877314 73.24395142378486) bank135970 +135971 POINT(40.02430740820124 73.67609336300548) bank135971 +135972 POINT(40.41692426729322 74.79509221213266) bank135972 +135973 POINT(40.14466867069703 74.0047843080904) bank135973 +135974 POINT(41.660682930728726 74.35747571082943) bank135974 +135975 POINT(41.244805407830825 74.4346935482749) bank135975 +135976 POINT(40.87693616197045 74.49057699483429) bank135976 +135977 POINT(40.2346391575225 74.85250529650483) bank135977 +135978 POINT(40.235866918876546 73.46224428968655) bank135978 +135979 POINT(40.260710735579195 74.43805210484044) bank135979 +135980 POINT(40.96446501733211 73.25842697799702) bank135980 +135981 POINT(40.744522169208174 74.85518359309727) bank135981 +135982 POINT(41.68898971744052 74.9459889645228) bank135982 +135983 POINT(41.13238213909433 73.29680750454048) bank135983 +135984 POINT(40.88335263637366 73.1762294852062) bank135984 +135985 POINT(41.05422840752253 74.89569882899335) bank135985 +135986 POINT(39.840133631163404 74.00401625216186) bank135986 +135987 POINT(41.347529763983445 73.59346202219719) bank135987 +135988 POINT(40.86638940965303 74.98853703283216) bank135988 +135989 POINT(40.919460476739886 73.90241271377313) bank135989 +135990 POINT(40.59103627024588 73.56503398308672) bank135990 +135991 POINT(41.36933605790575 73.1557137724339) bank135991 +135992 POINT(39.75493788561374 74.05642959213371) bank135992 +135993 POINT(41.63706342882511 74.17692397515492) bank135993 +135994 POINT(40.69213291101008 74.18700666202552) bank135994 +135995 POINT(40.67007795531484 73.44978561289943) bank135995 +135996 POINT(41.58801668314094 74.00114402778586) bank135996 +135997 POINT(41.15270666939551 73.56338257995728) bank135997 +135998 POINT(40.95657958859576 74.06604747162659) bank135998 +135999 POINT(41.69525338752522 74.4246090502512) bank135999 +136000 POINT(41.35722918459365 73.89640223143292) bank136000 +136001 POINT(41.555643842953394 73.8291762357394) bank136001 +136002 POINT(40.42221020994046 74.59430348240589) bank136002 +136003 POINT(41.09071612246979 74.43420422539418) bank136003 +136004 POINT(40.316083651677104 74.04222287730812) bank136004 +136005 POINT(40.118135999971635 73.93454527043372) bank136005 +136006 POINT(41.199702125392335 74.88877180218034) bank136006 +136007 POINT(41.34816098369966 73.25449675008797) bank136007 +136008 POINT(41.06929196706318 73.42826848195206) bank136008 +136009 POINT(40.15712542951192 74.19042079155994) bank136009 +136010 POINT(41.68596610429457 73.05374796753901) bank136010 +136011 POINT(41.38370490030922 74.56211274369443) bank136011 +136012 POINT(40.531633795182366 74.20716164644544) bank136012 +136013 POINT(41.709593966147665 74.57617510365237) bank136013 +136014 POINT(39.97859755043251 74.94764172939743) bank136014 +136015 POINT(41.47569761418101 73.68126423101083) bank136015 +136016 POINT(40.567846434072116 74.63970155474045) bank136016 +136017 POINT(41.52276946561059 73.6629593702765) bank136017 +136018 POINT(41.238428877545054 74.39503426476331) bank136018 +136019 POINT(41.56240703703743 73.64886561633186) bank136019 +136020 POINT(39.800458456282286 74.74363091665407) bank136020 +136021 POINT(40.71452143996109 74.77832047334772) bank136021 +136022 POINT(41.0509712412667 74.18626551659472) bank136022 +136023 POINT(40.06567417238502 74.67620874584007) bank136023 +136024 POINT(40.11401827276969 74.1152773417846) bank136024 +136025 POINT(41.550951382642644 74.72087687636312) bank136025 +136026 POINT(40.419129574625956 73.36975998524439) bank136026 +136027 POINT(41.61889436222018 73.92777284370167) bank136027 +136028 POINT(40.85837090367834 74.02667897556934) bank136028 +136029 POINT(40.5281563686352 74.80230380960363) bank136029 +136030 POINT(41.40925870174544 73.26911291077751) bank136030 +136031 POINT(41.32202379365948 74.6688922780348) bank136031 +136032 POINT(40.30940029500296 73.22837215007738) bank136032 +136033 POINT(41.65260054208039 74.96512209346982) bank136033 +136034 POINT(40.274992784221425 73.11028182024525) bank136034 +136035 POINT(40.53530625720219 73.73077292805368) bank136035 +136036 POINT(39.86707165807305 73.78485702730211) bank136036 +136037 POINT(41.62805174291281 74.44267447976236) bank136037 +136038 POINT(40.50862392422857 73.62570159662297) bank136038 +136039 POINT(40.04390837183735 74.8228348952201) bank136039 +136040 POINT(41.31441601606622 74.19945792803732) bank136040 +136041 POINT(40.677706647835805 75.00245081627959) bank136041 +136042 POINT(41.324115728121 73.23589837999629) bank136042 +136043 POINT(41.03140686903717 73.96751896076886) bank136043 +136044 POINT(40.98075161319849 73.17185903534418) bank136044 +136045 POINT(41.57687210974643 73.64553031675403) bank136045 +136046 POINT(41.667046463652554 73.81347661205693) bank136046 +136047 POINT(40.26079594676183 74.68741490097501) bank136047 +136048 POINT(41.41991240446452 74.39677054652768) bank136048 +136049 POINT(40.35494449114983 73.8685332706255) bank136049 +136050 POINT(40.73819247905976 73.7354999648065) bank136050 +136051 POINT(41.63282911026803 73.1788894916117) bank136051 +136052 POINT(40.52590684322304 73.30413194210456) bank136052 +136053 POINT(40.307364645567404 73.39410636748308) bank136053 +136054 POINT(41.53648881514417 74.15619425859929) bank136054 +136055 POINT(40.151076553591864 74.57902001907199) bank136055 +136056 POINT(39.75560502931376 73.871744323151) bank136056 +136057 POINT(41.3196859199112 73.94585050273729) bank136057 +136058 POINT(39.8249632202828 73.5783737435834) bank136058 +136059 POINT(40.98667126683497 73.92263912972679) bank136059 +136060 POINT(41.26943462364726 74.40977882606718) bank136060 +136061 POINT(40.56394806347221 74.24979465625272) bank136061 +136062 POINT(40.892111948148795 73.12921724945892) bank136062 +136063 POINT(40.25507683642018 74.76031645117588) bank136063 +136064 POINT(41.06590737818651 73.08741925991424) bank136064 +136065 POINT(41.581031572347285 74.91058737034953) bank136065 +136066 POINT(41.33997241535393 74.7067314582248) bank136066 +136067 POINT(40.17831999155605 73.5433311564528) bank136067 +136068 POINT(41.66442464177067 74.43277252362218) bank136068 +136069 POINT(41.23778589671217 74.06941739801306) bank136069 +136070 POINT(40.32513598900372 73.1028083233482) bank136070 +136071 POINT(40.063174172634476 74.09911467945729) bank136071 +136072 POINT(40.578783939614155 74.49567045110318) bank136072 +136073 POINT(41.39021132432779 73.64787979364516) bank136073 +136074 POINT(41.62668280107552 73.20631617833087) bank136074 +136075 POINT(41.46078041255945 73.08972222678807) bank136075 +136076 POINT(40.79768196103065 74.50921263902737) bank136076 +136077 POINT(40.24291772514229 74.03015865673902) bank136077 +136078 POINT(40.8327896821314 73.80937909202996) bank136078 +136079 POINT(40.57184747668589 74.52430760695574) bank136079 +136080 POINT(41.02764030606 74.11978554967264) bank136080 +136081 POINT(39.90009087314929 73.88564611306623) bank136081 +136082 POINT(41.47686076863383 73.15908206091545) bank136082 +136083 POINT(40.73113311296099 74.97073509453453) bank136083 +136084 POINT(40.558432281055886 73.07428638214942) bank136084 +136085 POINT(40.77128466432455 73.59454727636049) bank136085 +136086 POINT(40.35432611338168 73.2733490602036) bank136086 +136087 POINT(41.313464224214286 74.09365944534122) bank136087 +136088 POINT(41.59791466224654 73.90026414709386) bank136088 +136089 POINT(40.522524987368186 74.43613559006373) bank136089 +136090 POINT(40.35372253048149 74.56983043693556) bank136090 +136091 POINT(41.26318987321218 73.65359551155865) bank136091 +136092 POINT(40.27040716013563 73.11688172256325) bank136092 +136093 POINT(40.21654013791772 73.66502749734461) bank136093 +136094 POINT(41.04891400117377 73.27217543527676) bank136094 +136095 POINT(41.30010237937667 73.32437297362264) bank136095 +136096 POINT(39.98683415611464 73.3206070894495) bank136096 +136097 POINT(40.43074966650023 73.11585389921369) bank136097 +136098 POINT(41.52820182719011 73.76412898077078) bank136098 +136099 POINT(40.32871562349347 73.84744296851169) bank136099 +136100 POINT(40.859913220416544 73.36609055476416) bank136100 +136101 POINT(40.36543418915269 73.12679624645605) bank136101 +136102 POINT(40.62767526063591 73.77722553149223) bank136102 +136103 POINT(40.39357549913906 74.23531320512231) bank136103 +136104 POINT(40.24821743027639 73.6484207741608) bank136104 +136105 POINT(40.73359855872314 73.43633943699913) bank136105 +136106 POINT(41.45138142775917 73.20630645771573) bank136106 +136107 POINT(39.74095435345016 74.63665761373232) bank136107 +136108 POINT(40.939888437614954 74.42900862346585) bank136108 +136109 POINT(40.398112931736776 74.9834926449135) bank136109 +136110 POINT(41.418607914531684 74.16160502887739) bank136110 +136111 POINT(40.651524200658706 74.69413250107445) bank136111 +136112 POINT(41.558478432287394 73.25525771148399) bank136112 +136113 POINT(40.4772729650027 73.50593161570166) bank136113 +136114 POINT(40.63285027222874 74.92800487159788) bank136114 +136115 POINT(41.69130433401477 73.76629613033076) bank136115 +136116 POINT(39.99372793443707 74.54650470095926) bank136116 +136117 POINT(41.69776759097408 73.01145266587197) bank136117 +136118 POINT(41.05670650460103 73.11510241194091) bank136118 +136119 POINT(40.68427303364092 73.97701850407488) bank136119 +136120 POINT(41.09496815496455 74.22443612737892) bank136120 +136121 POINT(40.85017073988366 73.27317785249545) bank136121 +136122 POINT(39.93938495380351 74.18722023938834) bank136122 +136123 POINT(40.40907080145606 73.81640627680888) bank136123 +136124 POINT(40.586368104611054 74.42152970090791) bank136124 +136125 POINT(40.32983367441872 74.98834575680625) bank136125 +136126 POINT(40.354208811429366 74.38528674138732) bank136126 +136127 POINT(41.589827519792486 74.08906903519208) bank136127 +136128 POINT(40.68463310011237 73.32131118789135) bank136128 +136129 POINT(39.95241061468998 73.92359614295842) bank136129 +136130 POINT(40.73678096841028 73.419598316876) bank136130 +136131 POINT(40.59804424358545 74.76537178489923) bank136131 +136132 POINT(40.02649462776607 73.99264512273793) bank136132 +136133 POINT(41.51313282520919 73.15320647573662) bank136133 +136134 POINT(39.969251842886855 73.79097296109629) bank136134 +136135 POINT(40.27458947861066 73.9277940542893) bank136135 +136136 POINT(39.91793219937544 74.5970085926674) bank136136 +136137 POINT(40.976016434208326 73.50798145746451) bank136137 +136138 POINT(41.2420713298193 74.9974321049827) bank136138 +136139 POINT(41.69173865990361 74.4357651809522) bank136139 +136140 POINT(39.940387911810554 74.49184256048915) bank136140 +136141 POINT(41.13662069632262 74.695938675958) bank136141 +136142 POINT(40.13440805738814 74.0789830383324) bank136142 +136143 POINT(40.71486880031911 73.78967964682828) bank136143 +136144 POINT(41.669158975651946 73.0998728501644) bank136144 +136145 POINT(40.41559413773647 74.42713222473756) bank136145 +136146 POINT(40.96247672537873 74.41340085460877) bank136146 +136147 POINT(40.42522790985833 74.58415976949584) bank136147 +136148 POINT(41.26644855532543 73.21402556670415) bank136148 +136149 POINT(40.19370786268124 74.80788236500798) bank136149 +136150 POINT(41.636373368759344 73.17688118247136) bank136150 +136151 POINT(40.30818596849176 74.0836823379594) bank136151 +136152 POINT(39.91328151122432 74.13066452440687) bank136152 +136153 POINT(40.233195982224245 74.6008940997348) bank136153 +136154 POINT(41.2289025389677 74.36977006125441) bank136154 +136155 POINT(40.24561434428752 73.17263737426886) bank136155 +136156 POINT(40.55308003840027 73.02742972674811) bank136156 +136157 POINT(40.61960643263652 74.23433871309727) bank136157 +136158 POINT(41.67317885555711 73.7026040997221) bank136158 +136159 POINT(40.691124761548906 73.13365645935293) bank136159 +136160 POINT(41.32890960364448 73.78102053901173) bank136160 +136161 POINT(40.499630816131024 73.46465493298534) bank136161 +136162 POINT(40.026126272310584 73.21653089725771) bank136162 +136163 POINT(40.38346508090219 73.06081163186536) bank136163 +136164 POINT(41.491436221139026 74.45578076421663) bank136164 +136165 POINT(40.784075494829644 74.5106484445873) bank136165 +136166 POINT(40.46641777351229 74.03705642189537) bank136166 +136167 POINT(40.28376749867667 75.0023166689307) bank136167 +136168 POINT(40.990797242633896 73.97535997889467) bank136168 +136169 POINT(39.740653620358074 73.72608962535192) bank136169 +136170 POINT(40.22887555993875 73.42231316643206) bank136170 +136171 POINT(39.795756137191255 74.75462949288844) bank136171 +136172 POINT(41.283410645968075 73.33018827988239) bank136172 +136173 POINT(40.27684402211039 73.57997461500848) bank136173 +136174 POINT(40.76090097264252 74.59850204447955) bank136174 +136175 POINT(39.71647692409852 74.95983660064458) bank136175 +136176 POINT(41.01339275250969 74.08704287703414) bank136176 +136177 POINT(40.72673645695028 73.90097818879055) bank136177 +136178 POINT(41.075496176483085 74.50447600751171) bank136178 +136179 POINT(41.35001154822751 74.34040819885773) bank136179 +136180 POINT(41.530237996861906 73.61768230238494) bank136180 +136181 POINT(41.10421440715796 73.87858215064455) bank136181 +136182 POINT(41.55079971508126 73.2897429415792) bank136182 +136183 POINT(40.74918749405944 73.96699543227822) bank136183 +136184 POINT(41.632782253561956 74.20443188665006) bank136184 +136185 POINT(39.76674226571727 73.84552587953698) bank136185 +136186 POINT(41.11079226040548 74.16360834353826) bank136186 +136187 POINT(41.49828230563564 74.43933249572055) bank136187 +136188 POINT(39.74935982830155 73.72483898083581) bank136188 +136189 POINT(41.27732199258987 74.82958781892661) bank136189 +136190 POINT(39.984483399011914 73.62783121637888) bank136190 +136191 POINT(39.76482324794126 73.18750589848274) bank136191 +136192 POINT(41.35326947014561 73.8104475116706) bank136192 +136193 POINT(41.6653033914402 74.33282558463574) bank136193 +136194 POINT(41.50622958938878 74.92519842827292) bank136194 +136195 POINT(40.60258040883564 73.14165671360117) bank136195 +136196 POINT(40.129246514123025 73.15626490337256) bank136196 +136197 POINT(40.751175968790214 74.60268752307894) bank136197 +136198 POINT(40.8852645769362 73.24505505226942) bank136198 +136199 POINT(39.80111564922655 74.84853386338892) bank136199 +136200 POINT(40.863752881651266 73.26430811152726) bank136200 +136201 POINT(40.41106048130761 73.93961189599433) bank136201 +136202 POINT(40.713070729323036 74.21086597498324) bank136202 +136203 POINT(40.31631785808721 73.32520702340895) bank136203 +136204 POINT(40.84320629404016 74.09070734799884) bank136204 +136205 POINT(41.45191092469502 73.6475382145299) bank136205 +136206 POINT(39.971778285205815 74.77243340950788) bank136206 +136207 POINT(41.58528356055336 74.22641994641663) bank136207 +136208 POINT(40.002993585602056 74.89658390489053) bank136208 +136209 POINT(40.43256180009159 73.13218939953586) bank136209 +136210 POINT(40.70742720321738 74.11221634872408) bank136210 +136211 POINT(40.85375698149155 74.8439441404144) bank136211 +136212 POINT(40.90913915585819 73.91699865055185) bank136212 +136213 POINT(40.44703298451866 74.22858969134566) bank136213 +136214 POINT(41.646419905109 73.88073228777269) bank136214 +136215 POINT(40.65518415066529 74.54024208663915) bank136215 +136216 POINT(40.340819409937296 74.87435618712381) bank136216 +136217 POINT(40.2513285775031 74.32047587544925) bank136217 +136218 POINT(41.136997726871186 74.86427217082293) bank136218 +136219 POINT(40.359631890493205 74.04782152872933) bank136219 +136220 POINT(41.56656157107911 74.08179037400416) bank136220 +136221 POINT(41.40032030507842 73.34762073561141) bank136221 +136222 POINT(40.82757567804127 75.00585022803541) bank136222 +136223 POINT(40.14228902699949 74.89757606598751) bank136223 +136224 POINT(39.7982613992811 74.34070037941534) bank136224 +136225 POINT(40.413549688295156 73.74284746504212) bank136225 +136226 POINT(40.46124143165824 74.8752222971478) bank136226 +136227 POINT(41.28913863889846 73.68918949810164) bank136227 +136228 POINT(41.311052336384606 73.7243858457402) bank136228 +136229 POINT(39.85215974716016 73.10063890499231) bank136229 +136230 POINT(41.69715862224087 73.8496931892896) bank136230 +136231 POINT(40.00613452286995 74.87490392755649) bank136231 +136232 POINT(40.64403918603182 74.31721286267938) bank136232 +136233 POINT(40.05677978563994 74.03106390972935) bank136233 +136234 POINT(40.75952304705415 74.26909597632803) bank136234 +136235 POINT(40.67033416235232 74.38973541578174) bank136235 +136236 POINT(39.99191020603232 74.5346663443991) bank136236 +136237 POINT(39.797352302319645 73.1103007048144) bank136237 +136238 POINT(40.952047081645254 73.40664106917377) bank136238 +136239 POINT(40.0142149492339 73.25372165587446) bank136239 +136240 POINT(40.94152724093333 73.60190811914936) bank136240 +136241 POINT(40.602877259584865 74.76187426330705) bank136241 +136242 POINT(40.52668722004122 73.6237742703204) bank136242 +136243 POINT(41.06014926689916 74.35178187862681) bank136243 +136244 POINT(40.07222712520756 74.01836948697904) bank136244 +136245 POINT(40.37320499682618 73.96741731325476) bank136245 +136246 POINT(40.693621822866476 73.22891131279539) bank136246 +136247 POINT(40.99155153127929 73.83133171791319) bank136247 +136248 POINT(40.17143536952194 74.5535768042121) bank136248 +136249 POINT(40.192219400291044 73.34369240203341) bank136249 +136250 POINT(41.69960401996357 73.68875757455899) bank136250 +136251 POINT(41.503576926083554 73.23557063299741) bank136251 +136252 POINT(40.83631694145923 73.88024846800808) bank136252 +136253 POINT(39.99022985621416 74.89809146957906) bank136253 +136254 POINT(40.359377936006815 73.43086958322051) bank136254 +136255 POINT(40.44785674007905 74.72113194687716) bank136255 +136256 POINT(41.235747098703094 73.77593253210578) bank136256 +136257 POINT(40.83397484484078 74.76684632776993) bank136257 +136258 POINT(41.69338601127752 74.84834189621596) bank136258 +136259 POINT(40.506404403961504 73.40387445347386) bank136259 +136260 POINT(40.0116601286075 73.02040388811395) bank136260 +136261 POINT(40.11359056285536 74.83398346689437) bank136261 +136262 POINT(41.00745100803095 74.82744310677991) bank136262 +136263 POINT(39.813092327420605 73.89535436022275) bank136263 +136264 POINT(41.00270999573962 74.63531063501607) bank136264 +136265 POINT(40.88039437517814 73.4041639281388) bank136265 +136266 POINT(40.40573848058061 74.61918412188041) bank136266 +136267 POINT(40.577918676916546 74.35436903766862) bank136267 +136268 POINT(41.552089398947864 74.27375476174124) bank136268 +136269 POINT(40.49832879568979 74.63600825328784) bank136269 +136270 POINT(40.64453006766868 74.65987246844357) bank136270 +136271 POINT(40.760987257681435 73.69270175666338) bank136271 +136272 POINT(40.75383956122202 74.57863704513426) bank136272 +136273 POINT(39.8811053137831 74.2145539539456) bank136273 +136274 POINT(40.449152104527954 73.39568551138393) bank136274 +136275 POINT(40.665404916180904 73.91467646883298) bank136275 +136276 POINT(41.28253713406188 73.24534973619411) bank136276 +136277 POINT(40.866764793652436 73.13168810953228) bank136277 +136278 POINT(39.76659830176781 74.83136045562499) bank136278 +136279 POINT(40.17064111687064 74.68792463146836) bank136279 +136280 POINT(41.18575220622341 73.32355306387734) bank136280 +136281 POINT(40.56997535557062 74.39439379250828) bank136281 +136282 POINT(39.90471593814744 74.67022636602877) bank136282 +136283 POINT(40.908393599688175 74.43962457899443) bank136283 +136284 POINT(41.27490300801814 73.04692759598001) bank136284 +136285 POINT(41.481288559766284 74.9567237517818) bank136285 +136286 POINT(41.38063122786883 73.83271275797492) bank136286 +136287 POINT(40.735969973933685 73.42531994869287) bank136287 +136288 POINT(39.779446474421256 73.84043538758347) bank136288 +136289 POINT(41.62389003072772 73.46317088956901) bank136289 +136290 POINT(40.95507153030622 73.45709271567877) bank136290 +136291 POINT(40.366745186521044 73.26194703962858) bank136291 +136292 POINT(39.74119699544632 73.68816007355618) bank136292 +136293 POINT(39.78084007830698 74.26614333635028) bank136293 +136294 POINT(41.186082705618475 73.65082279366051) bank136294 +136295 POINT(41.57739259213765 73.99562409803649) bank136295 +136296 POINT(40.14989520391575 73.1726278675659) bank136296 +136297 POINT(40.51583275936284 74.80101286797428) bank136297 +136298 POINT(41.56213614664319 74.63510756730696) bank136298 +136299 POINT(40.601830331705514 74.13554143712062) bank136299 +136300 POINT(41.10898488592876 73.43412933432747) bank136300 +136301 POINT(40.315246627037055 74.10195021771524) bank136301 +136302 POINT(40.86214010417547 74.38542122035267) bank136302 +136303 POINT(40.10125934873206 74.92443560156956) bank136303 +136304 POINT(39.982511886669 74.3546729842172) bank136304 +136305 POINT(41.29716663194574 74.53559756062525) bank136305 +136306 POINT(40.69135273874396 74.02359795377018) bank136306 +136307 POINT(40.14342336691423 75.00327196459332) bank136307 +136308 POINT(41.180039085120086 74.0882690415871) bank136308 +136309 POINT(40.382568792923585 74.19888432843501) bank136309 +136310 POINT(41.4133916470831 73.93566203268865) bank136310 +136311 POINT(41.26689768805522 73.32284150381075) bank136311 +136312 POINT(41.239702343340056 73.83113250186639) bank136312 +136313 POINT(40.93687899461996 73.54788164616535) bank136313 +136314 POINT(39.80876396572595 74.66339787018278) bank136314 +136315 POINT(41.575416247895824 74.50487861894206) bank136315 +136316 POINT(41.56097340751099 73.95213735308978) bank136316 +136317 POINT(40.18861277702957 74.44271464232439) bank136317 +136318 POINT(40.80303604492713 74.87189860403922) bank136318 +136319 POINT(41.49262759090402 73.67156778165668) bank136319 +136320 POINT(41.080629865172874 74.89877323528363) bank136320 +136321 POINT(40.38203732413789 73.76349619371959) bank136321 +136322 POINT(39.86301929557002 74.79572827273614) bank136322 +136323 POINT(40.58383502685985 73.11999927055366) bank136323 +136324 POINT(40.78262504129857 74.59406204633655) bank136324 +136325 POINT(40.60723874326021 74.02759961371054) bank136325 +136326 POINT(41.24633408351945 74.12255000042877) bank136326 +136327 POINT(41.6222408123295 73.7034844025745) bank136327 +136328 POINT(39.96983988476621 74.32495342688576) bank136328 +136329 POINT(41.38405137606849 74.05616973456812) bank136329 +136330 POINT(39.89650919937997 74.61057018194755) bank136330 +136331 POINT(40.704044455205356 73.28943060551164) bank136331 +136332 POINT(40.063938135799205 73.13510862251901) bank136332 +136333 POINT(40.899795932889184 73.04419227961415) bank136333 +136334 POINT(41.13020540149696 74.05723936933879) bank136334 +136335 POINT(40.78104634958097 74.82638586530632) bank136335 +136336 POINT(40.04454422330342 73.10629947644861) bank136336 +136337 POINT(41.68069442723574 74.53770128557089) bank136337 +136338 POINT(41.21132172027416 74.23647551234805) bank136338 +136339 POINT(41.5274319737816 73.93439548368954) bank136339 +136340 POINT(39.92894061211074 74.71198937652258) bank136340 +136341 POINT(40.29558214266026 73.75107926339355) bank136341 +136342 POINT(39.76341974071837 74.373202365501) bank136342 +136343 POINT(39.78080981565168 74.644930704693) bank136343 +136344 POINT(40.58080535883335 74.64947224209847) bank136344 +136345 POINT(41.46645661522092 74.7781584466993) bank136345 +136346 POINT(40.13261201074393 73.69544368456025) bank136346 +136347 POINT(40.70432212127334 73.72109935551933) bank136347 +136348 POINT(41.06152706455408 74.55563660745693) bank136348 +136349 POINT(40.23940009246551 73.35414502853668) bank136349 +136350 POINT(41.21412354380943 74.87257741786306) bank136350 +136351 POINT(40.42804445078536 73.79315964618968) bank136351 +136352 POINT(41.464784259427056 73.98255603789326) bank136352 +136353 POINT(41.56662313419719 73.04275012583639) bank136353 +136354 POINT(39.7691649299805 74.20356126296426) bank136354 +136355 POINT(40.88853261444599 73.61828721901574) bank136355 +136356 POINT(40.613329415745575 74.40512678049036) bank136356 +136357 POINT(39.807808410006125 73.0856901368731) bank136357 +136358 POINT(41.43738151852419 73.99037101982086) bank136358 +136359 POINT(41.43003802473046 73.13450133605616) bank136359 +136360 POINT(40.74842244227685 74.89686398835131) bank136360 +136361 POINT(40.360465975412694 73.98335168466515) bank136361 +136362 POINT(40.26836088826974 74.24154511871414) bank136362 +136363 POINT(39.853603500939364 74.97461699510662) bank136363 +136364 POINT(40.953175801335775 73.35073317396989) bank136364 +136365 POINT(40.25113323259887 73.24610215943646) bank136365 +136366 POINT(40.432813328737744 74.1774998142339) bank136366 +136367 POINT(41.22729361904197 74.86600365378817) bank136367 +136368 POINT(41.40842976990894 73.10163557710237) bank136368 +136369 POINT(40.488443844223134 73.53281733158417) bank136369 +136370 POINT(40.45297661590613 73.55688409141737) bank136370 +136371 POINT(40.98495248547842 74.74538385936239) bank136371 +136372 POINT(39.966398710878494 73.26216465699963) bank136372 +136373 POINT(40.48653157257702 74.3592280135814) bank136373 +136374 POINT(40.65761039017042 74.73716295947966) bank136374 +136375 POINT(41.55071675710502 73.53442295908694) bank136375 +136376 POINT(40.21647221990139 73.3706290182327) bank136376 +136377 POINT(40.71250962103987 74.29076238676932) bank136377 +136378 POINT(41.23162077432106 73.16031308096659) bank136378 +136379 POINT(40.34754370448823 74.08823673757333) bank136379 +136380 POINT(40.45533487221768 74.82710362990304) bank136380 +136381 POINT(41.096591481273265 74.97423916831141) bank136381 +136382 POINT(39.85484872784862 74.03513669584444) bank136382 +136383 POINT(40.29765044170517 74.81324211858359) bank136383 +136384 POINT(41.09407869133413 73.94596947661076) bank136384 +136385 POINT(40.865425617684345 73.49357835215254) bank136385 +136386 POINT(39.778548387670895 73.80724453089607) bank136386 +136387 POINT(41.094960209632916 73.47318136802703) bank136387 +136388 POINT(40.84672670685884 74.68753809889193) bank136388 +136389 POINT(40.60560348621796 73.10599690497449) bank136389 +136390 POINT(41.46050946034906 74.66943948885299) bank136390 +136391 POINT(41.510553005166976 73.3199170206199) bank136391 +136392 POINT(40.57744554234499 73.50704581435717) bank136392 +136393 POINT(40.48810666901781 73.40971400443861) bank136393 +136394 POINT(40.659382234940715 73.6407904359666) bank136394 +136395 POINT(41.24546711935036 73.03323856522438) bank136395 +136396 POINT(41.31998534740796 73.56344266713896) bank136396 +136397 POINT(40.2777594447364 73.20154036476623) bank136397 +136398 POINT(41.65711125078181 73.24867085915815) bank136398 +136399 POINT(41.011890745577986 73.72730049701939) bank136399 +136400 POINT(40.48998100893075 73.1734585711004) bank136400 +136401 POINT(39.76541685927588 73.95837580012622) bank136401 +136402 POINT(41.210162016524926 73.2979233428599) bank136402 +136403 POINT(41.14887210663065 73.11948814487661) bank136403 +136404 POINT(41.344866995286274 73.37745386436694) bank136404 +136405 POINT(40.97551614680376 74.94574390121704) bank136405 +136406 POINT(41.42152308130756 74.86040115506077) bank136406 +136407 POINT(41.30757083925596 74.25132955879123) bank136407 +136408 POINT(39.855981877629524 73.53332674483758) bank136408 +136409 POINT(40.45825785896054 73.56791856220255) bank136409 +136410 POINT(40.45496487273989 74.74720264616263) bank136410 +136411 POINT(39.80471261690657 73.96574271206516) bank136411 +136412 POINT(41.36080365836342 73.43878583908153) bank136412 +136413 POINT(40.42772755051 73.62680891516986) bank136413 +136414 POINT(41.28152278555212 73.22921321986193) bank136414 +136415 POINT(39.740691971168204 73.20951674614871) bank136415 +136416 POINT(41.25940023021609 74.16386883306026) bank136416 +136417 POINT(41.13363349706551 73.43472635870913) bank136417 +136418 POINT(39.87556883047393 73.56065065751734) bank136418 +136419 POINT(40.5246041521247 73.67689468759653) bank136419 +136420 POINT(40.54298550219434 74.88009531511355) bank136420 +136421 POINT(41.20000879951335 74.02704554271197) bank136421 +136422 POINT(39.81560925566851 74.79423385409544) bank136422 +136423 POINT(39.72777476269563 73.73239072509298) bank136423 +136424 POINT(40.85142514986754 73.01838364137511) bank136424 +136425 POINT(41.352908005783775 73.22540240059634) bank136425 +136426 POINT(40.691412073085665 73.16264962834855) bank136426 +136427 POINT(41.366497316570836 73.93714740943743) bank136427 +136428 POINT(39.79647551993553 73.74970050540611) bank136428 +136429 POINT(40.91168719185896 74.97316941350041) bank136429 +136430 POINT(41.17847965657779 74.98805089398347) bank136430 +136431 POINT(39.857290879216215 74.30799814266985) bank136431 +136432 POINT(40.16873397923612 73.72119828714936) bank136432 +136433 POINT(41.017519657802396 74.60904938511278) bank136433 +136434 POINT(41.08655267414445 74.1537112506276) bank136434 +136435 POINT(40.00383767106827 74.62279264874734) bank136435 +136436 POINT(40.274626372382556 74.82524049759577) bank136436 +136437 POINT(41.0663257872941 74.57708973688054) bank136437 +136438 POINT(40.34288540812268 73.45415751439498) bank136438 +136439 POINT(41.226559337652766 73.42742282556527) bank136439 +136440 POINT(40.02722982361246 73.35613079343479) bank136440 +136441 POINT(40.29439682848089 74.32979061950842) bank136441 +136442 POINT(41.253337032588036 74.23867150605255) bank136442 +136443 POINT(39.76320212602615 73.43949541754606) bank136443 +136444 POINT(40.283745597497656 74.38627815324523) bank136444 +136445 POINT(40.81332404099625 73.44975050433104) bank136445 +136446 POINT(40.94425183521985 74.1314815527153) bank136446 +136447 POINT(40.22413544131163 73.249662027075) bank136447 +136448 POINT(40.1890023465589 73.56525619407962) bank136448 +136449 POINT(40.2654929509275 73.02166327820515) bank136449 +136450 POINT(41.520101312500955 73.7895523056079) bank136450 +136451 POINT(39.768090904653164 73.29784204625435) bank136451 +136452 POINT(40.20946033382204 73.11044100460333) bank136452 +136453 POINT(41.390159999033095 74.1324600317893) bank136453 +136454 POINT(40.43307608892516 73.60534111759864) bank136454 +136455 POINT(41.44942108182896 73.79047780540475) bank136455 +136456 POINT(40.848539522514315 73.82812439765742) bank136456 +136457 POINT(39.82456975880035 73.84842314955043) bank136457 +136458 POINT(40.21386830563129 74.82143384553366) bank136458 +136459 POINT(40.95702025405616 73.72084560509575) bank136459 +136460 POINT(41.46974942433465 73.86227527038605) bank136460 +136461 POINT(41.690173688479916 73.40963581091624) bank136461 +136462 POINT(41.660154707065814 73.23251207271463) bank136462 +136463 POINT(40.23949600663232 74.00738133352681) bank136463 +136464 POINT(41.32035332140275 73.35873605160222) bank136464 +136465 POINT(40.56004086177606 74.37974566454635) bank136465 +136466 POINT(41.66136682956153 74.53510878646125) bank136466 +136467 POINT(41.16521069755047 73.72924158924735) bank136467 +136468 POINT(41.422057155887536 73.7449255961603) bank136468 +136469 POINT(41.491253333706 73.14131301448631) bank136469 +136470 POINT(41.11296300873582 74.56102835175066) bank136470 +136471 POINT(41.49016645803433 73.26088520748154) bank136471 +136472 POINT(40.344175610950785 74.25065495846559) bank136472 +136473 POINT(39.72680497930082 73.20343784768629) bank136473 +136474 POINT(41.41368142153388 74.43894620555717) bank136474 +136475 POINT(41.12338854672573 74.48620717601592) bank136475 +136476 POINT(40.299315655096315 73.98045637917994) bank136476 +136477 POINT(40.17365667712662 74.48242781085365) bank136477 +136478 POINT(40.850349107731546 73.02474140606238) bank136478 +136479 POINT(41.122921780743084 74.07530025261987) bank136479 +136480 POINT(40.96408174934004 74.2579025756905) bank136480 +136481 POINT(41.24831357937131 73.36156665536934) bank136481 +136482 POINT(39.84619647637864 73.57197167716471) bank136482 +136483 POINT(40.68821931554728 73.15054557330578) bank136483 +136484 POINT(39.760486197770675 74.06132983340352) bank136484 +136485 POINT(41.240844874401574 74.05753854987141) bank136485 +136486 POINT(40.63313447917903 73.65659346726589) bank136486 +136487 POINT(40.28734399732542 74.78951182178179) bank136487 +136488 POINT(41.67563114446255 74.35373246911608) bank136488 +136489 POINT(41.01056976338592 74.36610212056895) bank136489 +136490 POINT(40.44027006878295 74.26291385281321) bank136490 +136491 POINT(40.50708358478943 73.37363593603453) bank136491 +136492 POINT(41.34287595956836 73.20770437190268) bank136492 +136493 POINT(39.86354774975695 74.00727871338746) bank136493 +136494 POINT(40.55463901958156 73.92237049840426) bank136494 +136495 POINT(41.667471154643636 73.65314119451357) bank136495 +136496 POINT(40.94637706182305 74.4083707956249) bank136496 +136497 POINT(39.82088474908584 74.04397693181527) bank136497 +136498 POINT(41.54444793249385 74.58634030410421) bank136498 +136499 POINT(40.70170678021402 73.97987045199969) bank136499 +136500 POINT(40.58053297497247 74.1150769907723) bank136500 +136501 POINT(41.015020797052955 74.72318583749392) bank136501 +136502 POINT(41.046157214497676 73.72739960492237) bank136502 +136503 POINT(40.001543443915594 73.37991530456773) bank136503 +136504 POINT(41.690423601147145 73.32793985326657) bank136504 +136505 POINT(40.20969245272116 73.17638534340166) bank136505 +136506 POINT(41.11585147958038 73.85267589094857) bank136506 +136507 POINT(40.09408398554598 73.00907753019688) bank136507 +136508 POINT(40.75699920614826 73.5889452529486) bank136508 +136509 POINT(41.57262224591548 74.09655140793156) bank136509 +136510 POINT(40.721107695375444 74.87483158830445) bank136510 +136511 POINT(40.54956533802131 73.60419348694535) bank136511 +136512 POINT(40.699961179174636 74.30875588941615) bank136512 +136513 POINT(39.76036015955457 73.72790348826295) bank136513 +136514 POINT(39.901139551177 73.24741500689706) bank136514 +136515 POINT(40.34378356385056 74.48592209765552) bank136515 +136516 POINT(41.564703170966396 74.57772822197883) bank136516 +136517 POINT(41.638354732625615 73.18138089985615) bank136517 +136518 POINT(41.641594445449876 73.0930804694454) bank136518 +136519 POINT(40.1285509333614 74.9572237588422) bank136519 +136520 POINT(41.080329156539094 73.32370429882137) bank136520 +136521 POINT(40.91987040245502 74.38211683534212) bank136521 +136522 POINT(40.316695626175836 73.88358989292968) bank136522 +136523 POINT(40.17580595968718 74.3260005275414) bank136523 +136524 POINT(41.48881150083683 73.46498588683072) bank136524 +136525 POINT(39.84600001764296 73.69439333828024) bank136525 +136526 POINT(39.79914981124964 73.99890178171495) bank136526 +136527 POINT(40.38975275980467 73.60406641184474) bank136527 +136528 POINT(39.732247364069 74.51643472933378) bank136528 +136529 POINT(40.25582690959446 73.19721144434857) bank136529 +136530 POINT(41.404896958893886 74.75964943751615) bank136530 +136531 POINT(41.062155489401164 74.45302734143768) bank136531 +136532 POINT(40.62858292342647 73.25091969465709) bank136532 +136533 POINT(41.181321396444375 73.612894271815) bank136533 +136534 POINT(41.6318217515498 73.96117365414896) bank136534 +136535 POINT(41.54639692144568 73.98007579436168) bank136535 +136536 POINT(41.49178709378023 73.24381480203186) bank136536 +136537 POINT(40.837791712588555 74.92004954424667) bank136537 +136538 POINT(41.37589618014217 74.14900014534737) bank136538 +136539 POINT(39.73953566547833 74.31334240102328) bank136539 +136540 POINT(39.748280201975014 73.55604345798264) bank136540 +136541 POINT(40.081476467368596 74.68642052704013) bank136541 +136542 POINT(40.69079010178936 73.27140125138826) bank136542 +136543 POINT(40.425267000847576 74.85078004553854) bank136543 +136544 POINT(40.57343623728015 73.22035566752359) bank136544 +136545 POINT(40.51300505854819 74.79038354398512) bank136545 +136546 POINT(41.618194364539015 73.6729431716778) bank136546 +136547 POINT(41.42593395632062 74.18936030826369) bank136547 +136548 POINT(40.39543544864962 74.50925455925227) bank136548 +136549 POINT(39.715028482605454 74.70492839292535) bank136549 +136550 POINT(39.89442168038104 74.95386788435606) bank136550 +136551 POINT(41.69577653771228 74.63603243940163) bank136551 +136552 POINT(41.4778728402805 74.09315915275072) bank136552 +136553 POINT(39.74122710901153 74.04984936030509) bank136553 +136554 POINT(39.9963960021578 74.98396017395963) bank136554 +136555 POINT(40.48998330239249 74.93507473914171) bank136555 +136556 POINT(40.273022362606085 74.50123749699497) bank136556 +136557 POINT(39.94011460842263 73.473247342115) bank136557 +136558 POINT(40.32498640281477 73.9059341487674) bank136558 +136559 POINT(40.03267307135711 74.98711820820407) bank136559 +136560 POINT(41.17546629089454 74.33567799523513) bank136560 +136561 POINT(41.00535771740037 73.8072952107521) bank136561 +136562 POINT(41.3233704628808 73.41720899169498) bank136562 +136563 POINT(40.528877827673675 73.12260014505422) bank136563 +136564 POINT(39.99191856223292 73.81466848380926) bank136564 +136565 POINT(39.84193063091517 74.90640240885413) bank136565 +136566 POINT(40.05280776277148 74.8972621665531) bank136566 +136567 POINT(40.99029736096891 74.8260258607899) bank136567 +136568 POINT(39.89794961928573 74.49643917448877) bank136568 +136569 POINT(40.56065583905445 73.27292658988137) bank136569 +136570 POINT(41.60422115635018 75.00082391391318) bank136570 +136571 POINT(40.12347185722773 73.53528403776848) bank136571 +136572 POINT(40.09722969172676 74.24804589502237) bank136572 +136573 POINT(41.6083354303225 73.06896716171286) bank136573 +136574 POINT(39.77671694975721 73.25921904225683) bank136574 +136575 POINT(41.70506299934098 74.53270632910836) bank136575 +136576 POINT(40.73274132830826 74.59055167721212) bank136576 +136577 POINT(40.34621153170188 74.20445718465933) bank136577 +136578 POINT(41.61387604924408 73.60096236684562) bank136578 +136579 POINT(40.06444810885702 73.16450607913123) bank136579 +136580 POINT(40.16376654451876 73.64567860150645) bank136580 +136581 POINT(40.51369271999779 74.47555424881823) bank136581 +136582 POINT(40.03170981995219 73.01033060963847) bank136582 +136583 POINT(40.317576710178116 73.18060936705973) bank136583 +136584 POINT(41.21401865602656 74.93097674745053) bank136584 +136585 POINT(41.00982200493775 73.10502245300657) bank136585 +136586 POINT(39.93164566820047 73.31728228174212) bank136586 +136587 POINT(40.459832833729934 73.23047971855017) bank136587 +136588 POINT(40.10572582941718 73.36787771218667) bank136588 +136589 POINT(41.33664933536608 74.83190121134683) bank136589 +136590 POINT(41.185877297327146 74.20263302678045) bank136590 +136591 POINT(40.51559361056168 74.05431958721337) bank136591 +136592 POINT(40.60495680851445 74.05604502539214) bank136592 +136593 POINT(41.11112879044355 74.81113608468972) bank136593 +136594 POINT(41.10692305929829 74.27600251050491) bank136594 +136595 POINT(40.22300793080015 74.08648054500802) bank136595 +136596 POINT(40.65979149545985 73.53318033963558) bank136596 +136597 POINT(41.0617076578158 74.93139853505107) bank136597 +136598 POINT(40.52075630655447 73.93936568500149) bank136598 +136599 POINT(40.524655772746854 74.55618970189445) bank136599 +136600 POINT(39.945611140790696 73.60651125898168) bank136600 +136601 POINT(41.1159654726796 74.7012025865795) bank136601 +136602 POINT(41.639745671787765 74.3391608775621) bank136602 +136603 POINT(40.24794307475404 74.24512755927275) bank136603 +136604 POINT(40.28360550319282 73.95021221117129) bank136604 +136605 POINT(40.29744738175721 73.98966696580486) bank136605 +136606 POINT(40.23137997624371 73.37424828469608) bank136606 +136607 POINT(40.970299590711946 73.59635713999337) bank136607 +136608 POINT(40.39459809947623 73.56436342190877) bank136608 +136609 POINT(41.478448311870686 73.28537203588121) bank136609 +136610 POINT(41.54727317005959 74.88581955991674) bank136610 +136611 POINT(40.016301344594986 73.54489024723573) bank136611 +136612 POINT(40.0194954421179 73.05620434144782) bank136612 +136613 POINT(41.384213476967226 73.35907243541602) bank136613 +136614 POINT(40.925004512606584 74.32846142485236) bank136614 +136615 POINT(40.914642968315874 73.05012644282414) bank136615 +136616 POINT(39.71444196715206 74.79363627317049) bank136616 +136617 POINT(40.25320066280482 74.6773270619063) bank136617 +136618 POINT(40.016546132810035 73.42769241201816) bank136618 +136619 POINT(41.58850026496163 73.38538784416025) bank136619 +136620 POINT(39.833545664124905 74.87107891017386) bank136620 +136621 POINT(40.498392517849666 73.71754282757774) bank136621 +136622 POINT(39.87483813013466 73.37918241415699) bank136622 +136623 POINT(41.45900477901814 74.33153291446959) bank136623 +136624 POINT(40.94627345629029 74.0880078679382) bank136624 +136625 POINT(40.62801048085701 74.92717453086944) bank136625 +136626 POINT(40.261113795392184 73.67633825951954) bank136626 +136627 POINT(41.20790129065958 73.13032785337379) bank136627 +136628 POINT(41.636096274073296 74.75220672288587) bank136628 +136629 POINT(40.86715984469096 74.9257699563787) bank136629 +136630 POINT(41.682443646442046 73.7157695325146) bank136630 +136631 POINT(40.96755676495831 74.37451920360472) bank136631 +136632 POINT(41.50818963248656 74.40809261987752) bank136632 +136633 POINT(40.77185940498699 73.40105323601551) bank136633 +136634 POINT(40.16760978593382 74.7721083065413) bank136634 +136635 POINT(40.440120936728064 74.55730277873946) bank136635 +136636 POINT(40.603141961727125 74.83116679273081) bank136636 +136637 POINT(41.5019607167243 74.03911724624062) bank136637 +136638 POINT(41.16307574608197 73.74825643895879) bank136638 +136639 POINT(40.38970490208951 74.31483323942786) bank136639 +136640 POINT(40.2535917896892 74.43598353752319) bank136640 +136641 POINT(40.60608344850839 73.50788984704705) bank136641 +136642 POINT(40.71016584964218 73.84629947154168) bank136642 +136643 POINT(40.31476089907575 74.53499714295417) bank136643 +136644 POINT(40.188684775450405 74.89406719635791) bank136644 +136645 POINT(40.82787214086191 74.34461709913806) bank136645 +136646 POINT(40.19873221350758 74.65777953517689) bank136646 +136647 POINT(39.984344519876224 74.44218697766374) bank136647 +136648 POINT(40.32653242089366 73.94764318194918) bank136648 +136649 POINT(39.848343059754605 73.91058249461095) bank136649 +136650 POINT(41.623339715732996 74.34978912076589) bank136650 +136651 POINT(41.23643328317419 73.73413870226852) bank136651 +136652 POINT(40.14746255480015 73.56905559172174) bank136652 +136653 POINT(41.12642026221312 73.35234170284731) bank136653 +136654 POINT(41.444366429633995 74.6467028085432) bank136654 +136655 POINT(41.33397218322925 73.96494601938694) bank136655 +136656 POINT(41.36707605438691 73.50864658414365) bank136656 +136657 POINT(40.84395696551493 73.80111508728564) bank136657 +136658 POINT(40.10663622484128 73.03580818084461) bank136658 +136659 POINT(40.80437415005727 74.59708644827283) bank136659 +136660 POINT(41.63847047785411 73.45645095161102) bank136660 +136661 POINT(40.24427543637094 73.12665283837791) bank136661 +136662 POINT(41.16824232995337 73.9745312461239) bank136662 +136663 POINT(41.666148887611676 74.3318615409039) bank136663 +136664 POINT(40.45694712151446 73.19685921981454) bank136664 +136665 POINT(41.10445335620756 74.51289038388343) bank136665 +136666 POINT(40.3611312283468 74.13312961101619) bank136666 +136667 POINT(39.83874511944992 74.85080492500443) bank136667 +136668 POINT(41.592062775320656 74.36579642760348) bank136668 +136669 POINT(40.600603988637545 73.40902402224826) bank136669 +136670 POINT(40.038587835430654 74.85249745229153) bank136670 +136671 POINT(40.84437988044237 74.97570045971072) bank136671 +136672 POINT(40.977915007423505 73.75592319689052) bank136672 +136673 POINT(40.25673253287954 73.60631034350052) bank136673 +136674 POINT(40.89102263516043 74.10409670279317) bank136674 +136675 POINT(41.00525169156419 74.97705735314403) bank136675 +136676 POINT(39.72125167440126 74.5993079124284) bank136676 +136677 POINT(40.41914906462111 73.70332392071606) bank136677 +136678 POINT(41.5730056997765 73.76520099417291) bank136678 +136679 POINT(40.75653189688446 73.9002425479632) bank136679 +136680 POINT(40.773096794740454 74.76857615199665) bank136680 +136681 POINT(40.135081916518665 73.46443258124829) bank136681 +136682 POINT(41.3262148636562 73.34275893495428) bank136682 +136683 POINT(40.06887540396232 74.79626512645964) bank136683 +136684 POINT(40.943710051226624 74.19999170350411) bank136684 +136685 POINT(39.9486450891604 74.96748420465246) bank136685 +136686 POINT(40.29363441005232 74.35374890925837) bank136686 +136687 POINT(40.88158903168954 73.74554531409187) bank136687 +136688 POINT(41.178096407990886 73.52656396070083) bank136688 +136689 POINT(40.1235882669352 73.70504191011507) bank136689 +136690 POINT(41.35679456118457 73.2288674704784) bank136690 +136691 POINT(41.08141055301752 73.01734775569352) bank136691 +136692 POINT(41.41664958786264 73.54146446310386) bank136692 +136693 POINT(41.5253028717796 74.12291451291448) bank136693 +136694 POINT(41.219939186175665 74.15251583230555) bank136694 +136695 POINT(41.22041997454073 73.52409634665544) bank136695 +136696 POINT(41.30169759961138 73.3706500403976) bank136696 +136697 POINT(41.16762660395789 73.99225236012106) bank136697 +136698 POINT(41.411451535405156 74.70823170885865) bank136698 +136699 POINT(39.85586629010308 73.78910547207161) bank136699 +136700 POINT(40.68088303468113 73.70844012025324) bank136700 +136701 POINT(40.828593635073915 74.95677371005658) bank136701 +136702 POINT(40.49805347674974 73.9375210705855) bank136702 +136703 POINT(40.07557895642008 73.43417002119874) bank136703 +136704 POINT(39.95311056314511 74.10776871502266) bank136704 +136705 POINT(41.13072857171826 73.36244123141262) bank136705 +136706 POINT(41.556123857905874 74.66493370562613) bank136706 +136707 POINT(40.288925607705025 73.006714089597) bank136707 +136708 POINT(40.20572916756075 73.44438056268237) bank136708 +136709 POINT(41.43660048925681 73.86567417108557) bank136709 +136710 POINT(41.21090430852009 74.6663443500068) bank136710 +136711 POINT(40.44857564405293 73.46630250186278) bank136711 +136712 POINT(41.38048399271375 74.85937194579896) bank136712 +136713 POINT(40.33533309236465 73.22647389564384) bank136713 +136714 POINT(41.12472840975989 73.96590896313958) bank136714 +136715 POINT(40.67750330600389 74.84065011363776) bank136715 +136716 POINT(40.80651770496358 73.2932364246868) bank136716 +136717 POINT(40.28820071734657 74.25613153075548) bank136717 +136718 POINT(39.77498524198413 73.27662676935871) bank136718 +136719 POINT(41.07732649803803 73.19736541326576) bank136719 +136720 POINT(40.312641332468644 74.96631153190742) bank136720 +136721 POINT(40.993158621624126 74.69617253558269) bank136721 +136722 POINT(40.41128174053201 73.3268618465267) bank136722 +136723 POINT(41.082067069767355 73.87511268815544) bank136723 +136724 POINT(39.80625983080403 73.1718407573521) bank136724 +136725 POINT(41.24105185968746 73.12206984170574) bank136725 +136726 POINT(41.651799249741224 73.3247766272085) bank136726 +136727 POINT(41.56600031849802 73.75133193762764) bank136727 +136728 POINT(40.234272233367406 74.50533856608466) bank136728 +136729 POINT(40.067425416184214 73.27931948082603) bank136729 +136730 POINT(41.622528832246 74.12234599150787) bank136730 +136731 POINT(39.788775236162344 73.50978428185007) bank136731 +136732 POINT(40.48536881910397 74.00899952103514) bank136732 +136733 POINT(41.14754045983326 74.79748375975436) bank136733 +136734 POINT(39.73487940548498 73.81564381229701) bank136734 +136735 POINT(41.034895525624776 74.0241534133398) bank136735 +136736 POINT(39.82751134444453 74.30374201929858) bank136736 +136737 POINT(40.374427599119464 74.34074603116221) bank136737 +136738 POINT(39.7510852941401 74.92731802160856) bank136738 +136739 POINT(40.057870899338425 74.80197411689133) bank136739 +136740 POINT(40.13865106667652 73.46374080234743) bank136740 +136741 POINT(41.3319214435015 73.02352426593752) bank136741 +136742 POINT(41.37116694695756 73.39494774409285) bank136742 +136743 POINT(40.961586235281 74.96711372183813) bank136743 +136744 POINT(41.49825270692282 73.90777810408889) bank136744 +136745 POINT(39.900046934265475 74.21040228754298) bank136745 +136746 POINT(41.207333001166 73.59711396224532) bank136746 +136747 POINT(41.2264001064896 73.58758362633047) bank136747 +136748 POINT(40.98681702909527 73.26157974086193) bank136748 +136749 POINT(40.33298574530174 73.41980396446397) bank136749 +136750 POINT(41.62365344169251 73.22174095388202) bank136750 +136751 POINT(39.871303202890154 74.7807942284468) bank136751 +136752 POINT(41.686271686561604 73.08863901594947) bank136752 +136753 POINT(39.92102155599477 74.57851343188202) bank136753 +136754 POINT(40.734388466183525 73.18077220491381) bank136754 +136755 POINT(40.17372260477504 74.10687377378947) bank136755 +136756 POINT(39.94701683554538 74.03027731174753) bank136756 +136757 POINT(39.846877052727685 74.6501715021127) bank136757 +136758 POINT(40.26304504589399 73.52979071882793) bank136758 +136759 POINT(39.98439046174623 74.32479839275187) bank136759 +136760 POINT(41.100698590550444 73.2139940510032) bank136760 +136761 POINT(40.38160846860086 73.83591525865536) bank136761 +136762 POINT(40.81303490299549 73.27746200911025) bank136762 +136763 POINT(40.05671144035482 73.67865435466098) bank136763 +136764 POINT(40.93854084008479 74.339850634269) bank136764 +136765 POINT(40.340794026292926 73.91749196128498) bank136765 +136766 POINT(40.97458817181442 74.65431291531519) bank136766 +136767 POINT(40.82417072431369 73.2617337610303) bank136767 +136768 POINT(40.72517849207879 74.38737915152394) bank136768 +136769 POINT(40.95031458854178 73.04267338035238) bank136769 +136770 POINT(40.44789129448704 74.4193883562618) bank136770 +136771 POINT(40.56793652656761 74.51861732852068) bank136771 +136772 POINT(40.157647841735816 74.7910969021404) bank136772 +136773 POINT(40.14355666153888 74.71370141597228) bank136773 +136774 POINT(41.189455517309895 74.78952352860881) bank136774 +136775 POINT(41.674475805398565 74.16111886761469) bank136775 +136776 POINT(40.43071225983383 73.41537663594119) bank136776 +136777 POINT(40.86801991051267 74.11306764510867) bank136777 +136778 POINT(39.940765915963745 73.36168631297136) bank136778 +136779 POINT(40.3238739667414 73.49821707336076) bank136779 +136780 POINT(40.654165481749665 73.11414352133717) bank136780 +136781 POINT(41.33200077155053 74.59110029636875) bank136781 +136782 POINT(40.89535223275221 73.01452037298863) bank136782 +136783 POINT(40.335582072867794 73.05092315291267) bank136783 +136784 POINT(40.174546372006894 73.95258279328279) bank136784 +136785 POINT(41.05221966055248 74.91532617887948) bank136785 +136786 POINT(41.06364373200311 74.85338631682826) bank136786 +136787 POINT(40.75255702866745 74.19796416261003) bank136787 +136788 POINT(41.36239114188024 74.2071734618051) bank136788 +136789 POINT(39.85556137411803 74.62422582301252) bank136789 +136790 POINT(40.839006008107965 73.13203960830795) bank136790 +136791 POINT(40.90356238775932 74.52606441037966) bank136791 +136792 POINT(40.11304342999879 73.67277374671153) bank136792 +136793 POINT(41.36324024890561 74.10276737973874) bank136793 +136794 POINT(40.22617484466162 74.82442242241102) bank136794 +136795 POINT(39.98318420158127 73.5709032457758) bank136795 +136796 POINT(40.84577418007225 74.98433004069848) bank136796 +136797 POINT(41.13498304646029 74.57419528144214) bank136797 +136798 POINT(41.22260681672869 74.43590858997418) bank136798 +136799 POINT(41.410179823761446 73.33705826364357) bank136799 +136800 POINT(40.88467598005544 73.74597652107387) bank136800 +136801 POINT(40.02158577691567 73.26647798840989) bank136801 +136802 POINT(41.00941774360361 73.84191253876537) bank136802 +136803 POINT(40.44445289488111 73.07713855823188) bank136803 +136804 POINT(41.02958342213716 73.06360935922623) bank136804 +136805 POINT(39.768279470179785 73.89728948694015) bank136805 +136806 POINT(40.022102648116366 73.10641859277344) bank136806 +136807 POINT(41.21045499413069 74.75106645577016) bank136807 +136808 POINT(39.720224065917705 73.66404694363149) bank136808 +136809 POINT(41.55494196820976 74.36135342532131) bank136809 +136810 POINT(39.767295552001194 74.25774927840405) bank136810 +136811 POINT(41.21441319301933 73.39600620355809) bank136811 +136812 POINT(41.32842979693832 74.4129673208283) bank136812 +136813 POINT(40.20706209932604 73.71204572743757) bank136813 +136814 POINT(41.00748862967681 74.74826224184181) bank136814 +136815 POINT(40.24074791865154 74.91977957562209) bank136815 +136816 POINT(41.58879050884214 73.89004085571167) bank136816 +136817 POINT(40.274644999416864 74.51688954462992) bank136817 +136818 POINT(41.513686241076314 74.57893040260193) bank136818 +136819 POINT(40.62986528435375 74.8552003728222) bank136819 +136820 POINT(41.48507890219302 74.69856375544798) bank136820 +136821 POINT(40.52017068022852 74.716835191256) bank136821 +136822 POINT(40.82437234411549 74.68968979101847) bank136822 +136823 POINT(40.90152881078694 74.33019575471779) bank136823 +136824 POINT(41.38642994320436 74.7839649605518) bank136824 +136825 POINT(39.94954774081318 73.36390126653902) bank136825 +136826 POINT(40.695146894092254 74.49059789809415) bank136826 +136827 POINT(40.84222590108891 74.80427440516446) bank136827 +136828 POINT(40.57874591186371 73.27645699338227) bank136828 +136829 POINT(40.331934587301404 73.67475940495716) bank136829 +136830 POINT(40.810427907071144 74.12887222669985) bank136830 +136831 POINT(40.6641117046935 73.67700506017212) bank136831 +136832 POINT(40.7147136708621 74.64051524205476) bank136832 +136833 POINT(41.31661829720812 74.50780581364629) bank136833 +136834 POINT(40.97272887314812 74.07467704591431) bank136834 +136835 POINT(41.32804118951215 74.05886951093262) bank136835 +136836 POINT(39.75761883018248 73.11185651426464) bank136836 +136837 POINT(41.228804793539375 73.5569903403787) bank136837 +136838 POINT(40.385744004712706 73.3878259360609) bank136838 +136839 POINT(39.87026272739874 74.46591897593227) bank136839 +136840 POINT(41.59124757625194 74.38994970123453) bank136840 +136841 POINT(40.705193314168724 74.18465111536474) bank136841 +136842 POINT(41.712517293752896 73.8408608502044) bank136842 +136843 POINT(39.91869174885404 73.31086334838947) bank136843 +136844 POINT(39.71864814475584 74.96430812789556) bank136844 +136845 POINT(39.938703745261385 74.95948393808894) bank136845 +136846 POINT(40.84154881139864 74.36840615962835) bank136846 +136847 POINT(41.631390953449326 73.50613140316533) bank136847 +136848 POINT(40.108531081358606 74.20779050730005) bank136848 +136849 POINT(40.077907529171796 74.91990105035116) bank136849 +136850 POINT(41.039318222172845 74.74255754452109) bank136850 +136851 POINT(41.61391102224612 74.06941736014132) bank136851 +136852 POINT(41.53888167054459 73.36007942571709) bank136852 +136853 POINT(41.00018905084726 73.84585096326892) bank136853 +136854 POINT(40.56966192526485 74.39673476107954) bank136854 +136855 POINT(40.08575114735364 74.43253172872565) bank136855 +136856 POINT(40.53104857737598 74.72304599994764) bank136856 +136857 POINT(41.08455515993351 73.72992113057869) bank136857 +136858 POINT(40.30022271763605 73.50866371458297) bank136858 +136859 POINT(41.10203914040411 74.77161359257607) bank136859 +136860 POINT(40.78164344292021 73.63580973971608) bank136860 +136861 POINT(40.367184044016625 73.61463850418937) bank136861 +136862 POINT(40.67240857595362 74.6032918819112) bank136862 +136863 POINT(41.59644700471149 74.01578686638045) bank136863 +136864 POINT(40.814066127257355 74.11890657942382) bank136864 +136865 POINT(39.744506140131264 73.5507304689491) bank136865 +136866 POINT(41.51344048411719 73.14390624122848) bank136866 +136867 POINT(41.16471827555451 73.99903640469823) bank136867 +136868 POINT(40.10373394400298 74.38610016679864) bank136868 +136869 POINT(39.8724515457212 73.49779263862203) bank136869 +136870 POINT(41.151218074793306 73.6862843668321) bank136870 +136871 POINT(41.589193901322155 73.15320954783368) bank136871 +136872 POINT(40.434539040842665 73.01633039836682) bank136872 +136873 POINT(40.16865422719869 73.28964636242365) bank136873 +136874 POINT(40.1837118850056 74.1790167883463) bank136874 +136875 POINT(40.85163948763011 74.47210183125001) bank136875 +136876 POINT(40.43128318944147 74.035257834599) bank136876 +136877 POINT(41.163687006463974 73.10068726378587) bank136877 +136878 POINT(39.96644060264302 73.90417229059908) bank136878 +136879 POINT(39.870994253847684 73.2038712459565) bank136879 +136880 POINT(41.094835992878224 74.75936472265391) bank136880 +136881 POINT(40.439029653989465 74.72035875875928) bank136881 +136882 POINT(40.708684095864015 74.89597194562475) bank136882 +136883 POINT(39.725307473544916 74.73587058546778) bank136883 +136884 POINT(40.87612683855786 74.55063217979479) bank136884 +136885 POINT(41.07722508916224 74.30203338650561) bank136885 +136886 POINT(41.348667782242345 74.76838145358133) bank136886 +136887 POINT(41.160932710370304 73.95266096070766) bank136887 +136888 POINT(40.94681009478406 73.07822668781121) bank136888 +136889 POINT(39.92013833831925 73.98022954776772) bank136889 +136890 POINT(41.40577625099368 73.45226615112428) bank136890 +136891 POINT(40.09892884168591 74.2778838377889) bank136891 +136892 POINT(41.20403193214237 73.24165140487443) bank136892 +136893 POINT(40.1025001431732 73.45168269368568) bank136893 +136894 POINT(41.32969696553638 73.60004088731503) bank136894 +136895 POINT(40.583384833524825 74.51739562732686) bank136895 +136896 POINT(39.778743574619064 73.96807852903348) bank136896 +136897 POINT(41.25296182068434 74.9920902874178) bank136897 +136898 POINT(40.6863583002411 73.26293530760714) bank136898 +136899 POINT(41.24915167178427 74.58279591450419) bank136899 +136900 POINT(40.04698299120992 74.69218859281548) bank136900 +136901 POINT(41.55957348923993 74.43630488319435) bank136901 +136902 POINT(39.871899777478944 74.69116660436184) bank136902 +136903 POINT(39.7783669309609 74.5164309171026) bank136903 +136904 POINT(41.33073851549379 73.56090173910069) bank136904 +136905 POINT(41.188959543864385 73.63412850809296) bank136905 +136906 POINT(41.08556972383577 73.38594547785154) bank136906 +136907 POINT(41.138324156946226 74.45870981494006) bank136907 +136908 POINT(40.718528100343946 73.08765706043926) bank136908 +136909 POINT(41.4756655168418 74.16374019569652) bank136909 +136910 POINT(41.36976941825984 74.56841547072321) bank136910 +136911 POINT(40.95104210117446 73.37663046416041) bank136911 +136912 POINT(40.19027586709457 73.3427595927278) bank136912 +136913 POINT(40.971379768764756 73.05263827939372) bank136913 +136914 POINT(39.88648330094381 73.36322094016488) bank136914 +136915 POINT(40.29025950651897 73.93231509877614) bank136915 +136916 POINT(40.22014844930446 73.3962402289915) bank136916 +136917 POINT(40.9454841317776 74.44310750546347) bank136917 +136918 POINT(40.03583970172455 73.17112524084978) bank136918 +136919 POINT(40.23482271446032 73.66314995069649) bank136919 +136920 POINT(41.28304038154299 73.19487140186664) bank136920 +136921 POINT(41.43473508745868 74.68304619148591) bank136921 +136922 POINT(41.25515819515855 74.20867517825322) bank136922 +136923 POINT(40.01264486605558 74.40740364255983) bank136923 +136924 POINT(40.73421697739092 74.89067229053434) bank136924 +136925 POINT(40.03157075806308 74.62356354207469) bank136925 +136926 POINT(40.11659075554696 74.13195862391446) bank136926 +136927 POINT(39.721744063106975 74.41414150973984) bank136927 +136928 POINT(40.578593856195134 73.98008742258261) bank136928 +136929 POINT(41.544159398922375 73.8628525381614) bank136929 +136930 POINT(41.36318787167611 74.36066020116034) bank136930 +136931 POINT(40.07655550319479 73.4752820739047) bank136931 +136932 POINT(40.8293916926867 73.3736587792921) bank136932 +136933 POINT(40.44404305645596 73.17381997207292) bank136933 +136934 POINT(40.21400585959011 73.31511437410857) bank136934 +136935 POINT(40.950377319736255 74.66795698149164) bank136935 +136936 POINT(41.35184467042721 74.80819902866332) bank136936 +136937 POINT(41.431837794770786 73.87936384896705) bank136937 +136938 POINT(41.43556177465312 73.62379020182254) bank136938 +136939 POINT(40.76243315075426 74.21813040978279) bank136939 +136940 POINT(40.777386921505475 74.49440488211926) bank136940 +136941 POINT(40.97448119844953 74.20036634637121) bank136941 +136942 POINT(40.39694699422963 73.8188016979792) bank136942 +136943 POINT(40.524560640722136 73.49765193028125) bank136943 +136944 POINT(41.67965567428396 73.84799598381713) bank136944 +136945 POINT(40.08609836702313 73.58691806910376) bank136945 +136946 POINT(41.57697885737651 73.4826462176078) bank136946 +136947 POINT(40.38946651421692 73.12613272037882) bank136947 +136948 POINT(41.53376017056814 74.75366526115522) bank136948 +136949 POINT(40.064126782651705 73.73999057262026) bank136949 +136950 POINT(41.446656540917054 74.33082428768478) bank136950 +136951 POINT(41.25356588588407 73.24818621508956) bank136951 +136952 POINT(41.70854805216128 74.19481448601897) bank136952 +136953 POINT(40.05905194591523 74.02403804086768) bank136953 +136954 POINT(40.88445305393887 73.73466672596116) bank136954 +136955 POINT(39.719564627367795 74.2093006244213) bank136955 +136956 POINT(39.921287682696125 73.15745222537758) bank136956 +136957 POINT(41.42099179009733 73.23029367229623) bank136957 +136958 POINT(41.35321606865035 74.23799350141744) bank136958 +136959 POINT(40.6517977785164 73.56971318965599) bank136959 +136960 POINT(40.3867976507509 74.06623328583494) bank136960 +136961 POINT(40.805656779230695 73.07664747382536) bank136961 +136962 POINT(40.76609446686386 73.60034753513253) bank136962 +136963 POINT(39.976431424103836 73.88091279824025) bank136963 +136964 POINT(40.1516419417057 73.21860412055493) bank136964 +136965 POINT(41.221321701600544 74.14573727149542) bank136965 +136966 POINT(41.63597008831938 73.39697558162209) bank136966 +136967 POINT(40.42456164355604 73.94319123068652) bank136967 +136968 POINT(41.64572116078644 74.23425195200792) bank136968 +136969 POINT(39.82879447469233 73.73194832014119) bank136969 +136970 POINT(40.36575206599029 73.9742891215472) bank136970 +136971 POINT(40.87489204587192 74.16199198993645) bank136971 +136972 POINT(40.181422031317986 73.77150245442499) bank136972 +136973 POINT(41.33693869153012 74.77382112047671) bank136973 +136974 POINT(40.454903487326604 74.97011889300741) bank136974 +136975 POINT(40.563970557424334 74.03569534134338) bank136975 +136976 POINT(40.61057343075684 73.26415047977653) bank136976 +136977 POINT(39.80049645218304 74.3583392439429) bank136977 +136978 POINT(41.31624282988219 74.43994883852667) bank136978 +136979 POINT(40.79740174820257 74.65273832474035) bank136979 +136980 POINT(41.654343509897565 74.52234239220579) bank136980 +136981 POINT(41.66889172801443 74.38641444678645) bank136981 +136982 POINT(39.921851758392606 74.04713625987621) bank136982 +136983 POINT(41.182714340507424 73.63391239636631) bank136983 +136984 POINT(41.20923490704767 74.25163023161402) bank136984 +136985 POINT(40.99790603949178 74.41987168811411) bank136985 +136986 POINT(40.2911081117797 74.4532678948697) bank136986 +136987 POINT(41.44035492586601 74.66194715175085) bank136987 +136988 POINT(40.16873791149759 73.70196727034245) bank136988 +136989 POINT(40.966493066423276 73.77146774587686) bank136989 +136990 POINT(40.18423556257223 74.44908356380117) bank136990 +136991 POINT(40.90449266303173 74.75156249363455) bank136991 +136992 POINT(41.661031541651454 74.41832413483213) bank136992 +136993 POINT(41.54901279736822 73.23796088244531) bank136993 +136994 POINT(41.02390849866349 74.66025912076323) bank136994 +136995 POINT(40.64596350367792 74.51018415366171) bank136995 +136996 POINT(41.16176562550753 73.48199374551271) bank136996 +136997 POINT(40.26683069569828 74.64515560749999) bank136997 +136998 POINT(39.956940254892665 73.32544089817804) bank136998 +136999 POINT(40.152100537984964 74.45458314226288) bank136999 +137000 POINT(41.11082615994949 74.01337342020425) bank137000 +137001 POINT(41.53622565462029 73.42518748268496) bank137001 +137002 POINT(41.441085904760016 73.64791998511309) bank137002 +137003 POINT(41.67835354437652 74.13631326340605) bank137003 +137004 POINT(41.66482148451158 74.33825628880159) bank137004 +137005 POINT(41.69400962610458 73.71522074233124) bank137005 +137006 POINT(40.11780102877082 73.02396051092116) bank137006 +137007 POINT(40.88143978729879 74.30486722036098) bank137007 +137008 POINT(41.20758932986558 74.96393326753788) bank137008 +137009 POINT(40.60054509840932 73.63525286809976) bank137009 +137010 POINT(40.31268573313669 73.77090999820027) bank137010 +137011 POINT(40.36076115713067 74.94364368747172) bank137011 +137012 POINT(41.53776957075943 74.63994683578555) bank137012 +137013 POINT(41.282367946078224 74.37272082971441) bank137013 +137014 POINT(41.17227399512326 74.94918030310969) bank137014 +137015 POINT(41.50198218203245 73.02222401725082) bank137015 +137016 POINT(40.60626593252124 73.20506884024032) bank137016 +137017 POINT(40.47063005662777 74.19229469995801) bank137017 +137018 POINT(40.05161295056198 74.77096182600539) bank137018 +137019 POINT(41.39952578914603 74.81455318831065) bank137019 +137020 POINT(40.18060921649862 74.94246288238851) bank137020 +137021 POINT(40.65027639404707 74.30612642010556) bank137021 +137022 POINT(39.773376478069366 74.8719493429784) bank137022 +137023 POINT(41.257770665551256 73.96241388580367) bank137023 +137024 POINT(40.98538300491077 73.26872008683537) bank137024 +137025 POINT(41.54342452702625 73.91908548966794) bank137025 +137026 POINT(40.02328294797521 73.21296578458865) bank137026 +137027 POINT(40.371981926202224 74.23493829507274) bank137027 +137028 POINT(40.598350308902255 73.41896527623436) bank137028 +137029 POINT(40.37995148795447 73.08808186682705) bank137029 +137030 POINT(40.06711958067037 74.42831718615238) bank137030 +137031 POINT(41.20035847004618 73.47451748752285) bank137031 +137032 POINT(41.06280272519603 74.20226261767452) bank137032 +137033 POINT(39.88369448157122 74.00368055354905) bank137033 +137034 POINT(41.6329225330359 74.58771117111743) bank137034 +137035 POINT(41.04334132367222 74.10450027862248) bank137035 +137036 POINT(40.247107561894865 73.30228639895331) bank137036 +137037 POINT(41.539694028106325 74.64807481147251) bank137037 +137038 POINT(40.06297791543841 73.78850299328084) bank137038 +137039 POINT(40.5191097785754 73.65673744506533) bank137039 +137040 POINT(41.33290116032916 74.13443914234703) bank137040 +137041 POINT(40.11137607549121 73.63385065640624) bank137041 +137042 POINT(40.35623716335493 74.59699058180041) bank137042 +137043 POINT(41.42223954155936 73.48718531340131) bank137043 +137044 POINT(41.408479448303744 74.63789946670872) bank137044 +137045 POINT(40.621953524859755 73.57661269358073) bank137045 +137046 POINT(40.13610023269909 74.21510464053145) bank137046 +137047 POINT(41.201280731777246 74.08386782660568) bank137047 +137048 POINT(40.97380602177905 74.25149720392224) bank137048 +137049 POINT(40.5328560066833 74.92888399340245) bank137049 +137050 POINT(40.923762550281296 73.87862854070111) bank137050 +137051 POINT(39.77990047175101 74.02086044995855) bank137051 +137052 POINT(40.92181811808699 74.96895139176398) bank137052 +137053 POINT(40.37291538475185 74.96350277468463) bank137053 +137054 POINT(39.87308433716833 73.0431068552846) bank137054 +137055 POINT(40.673344763535546 73.60148386818449) bank137055 +137056 POINT(41.41873283446135 73.26007801723841) bank137056 +137057 POINT(40.06241055930464 74.80218231568783) bank137057 +137058 POINT(41.25040771193959 74.51848869288818) bank137058 +137059 POINT(41.37805236789827 73.4110489914159) bank137059 +137060 POINT(41.27708181859119 73.04394463414789) bank137060 +137061 POINT(41.53107070329102 74.95267956403849) bank137061 +137062 POINT(40.65195098452789 73.58858788161923) bank137062 +137063 POINT(39.979332647491425 74.84169270799038) bank137063 +137064 POINT(41.42835000233452 74.63552470702884) bank137064 +137065 POINT(41.367112489296375 73.99301823026346) bank137065 +137066 POINT(41.41026267460772 74.67750555674981) bank137066 +137067 POINT(40.675459281075554 73.699496324042) bank137067 +137068 POINT(41.62560357797558 73.95427932835463) bank137068 +137069 POINT(40.86101794414288 73.06510420385995) bank137069 +137070 POINT(40.12215409783549 73.28951412522592) bank137070 +137071 POINT(40.02290826949291 73.19198104744231) bank137071 +137072 POINT(40.690154747255995 74.72275111443244) bank137072 +137073 POINT(40.37331996707536 74.11983699156788) bank137073 +137074 POINT(41.29187756003818 74.87285638096483) bank137074 +137075 POINT(40.57843478956849 73.80742892496563) bank137075 +137076 POINT(41.6423609341275 74.76185233135136) bank137076 +137077 POINT(41.59853872504374 74.51585323314634) bank137077 +137078 POINT(40.744312048298724 74.22575500481507) bank137078 +137079 POINT(41.204426054405204 74.12299046356023) bank137079 +137080 POINT(41.009325358795394 74.920222935486) bank137080 +137081 POINT(41.53243182881521 74.57035257552822) bank137081 +137082 POINT(41.50724619059864 74.49057711081628) bank137082 +137083 POINT(40.93841480217569 74.18811235088415) bank137083 +137084 POINT(41.02308946968386 73.68006664681826) bank137084 +137085 POINT(40.3678763170869 73.46699960178941) bank137085 +137086 POINT(39.901106215849374 73.36619125949478) bank137086 +137087 POINT(41.345072936787595 73.74691489295124) bank137087 +137088 POINT(40.377056679176555 74.40644729173006) bank137088 +137089 POINT(41.34112727934058 74.22946159979618) bank137089 +137090 POINT(39.72538169481739 73.34184936448783) bank137090 +137091 POINT(41.345544454039995 73.43069802150929) bank137091 +137092 POINT(40.03281891747688 73.64191469617036) bank137092 +137093 POINT(40.13176362943676 73.89661628913825) bank137093 +137094 POINT(40.46125558433087 74.63542434134777) bank137094 +137095 POINT(41.121000660486125 74.21198402852629) bank137095 +137096 POINT(40.56186474980412 73.37512066509696) bank137096 +137097 POINT(40.81717728438195 73.12506732933421) bank137097 +137098 POINT(40.72891518685935 74.98002891371756) bank137098 +137099 POINT(40.47709983973033 73.93721098508034) bank137099 +137100 POINT(39.809700358475276 74.58033334193499) bank137100 +137101 POINT(41.43192732424668 73.2252764756479) bank137101 +137102 POINT(40.34368809360744 74.67842334543327) bank137102 +137103 POINT(41.09325239108947 73.9200119386894) bank137103 +137104 POINT(41.45938285592919 74.72996370431758) bank137104 +137105 POINT(40.56179969117353 74.2460485711981) bank137105 +137106 POINT(40.21466675067529 73.23245620005686) bank137106 +137107 POINT(41.29747012230646 73.44955264893771) bank137107 +137108 POINT(39.995385995886785 74.78381065365) bank137108 +137109 POINT(40.17833334905626 74.80515507329444) bank137109 +137110 POINT(40.49424333102135 73.88234518660377) bank137110 +137111 POINT(40.24267295157131 73.52555806451986) bank137111 +137112 POINT(41.38532343068481 74.8804794618502) bank137112 +137113 POINT(39.9422043832873 74.95198022418471) bank137113 +137114 POINT(40.227498363865564 73.8005432667805) bank137114 +137115 POINT(39.964216896751864 74.21531811448953) bank137115 +137116 POINT(40.80245728631675 73.0791537066177) bank137116 +137117 POINT(39.81057427599298 73.35954250504327) bank137117 +137118 POINT(39.816384590412625 74.3790961381317) bank137118 +137119 POINT(41.25936459881674 74.17463106219991) bank137119 +137120 POINT(41.568164887965736 73.8281915391771) bank137120 +137121 POINT(40.71646555099651 73.66537215370845) bank137121 +137122 POINT(40.80276627929648 73.69104559994824) bank137122 +137123 POINT(41.41827019516795 74.26276852070052) bank137123 +137124 POINT(39.96825940323225 73.358364409097) bank137124 +137125 POINT(40.62180875936794 74.10022531124844) bank137125 +137126 POINT(41.391728618745795 73.93166713497861) bank137126 +137127 POINT(40.67394889355348 74.42258945705706) bank137127 +137128 POINT(40.475359352551486 74.36591904053941) bank137128 +137129 POINT(41.59049903180344 73.89099561354196) bank137129 +137130 POINT(41.52353022745098 74.38275105018961) bank137130 +137131 POINT(41.32994934179705 73.19206467098591) bank137131 +137132 POINT(41.051526239267965 73.0821777534293) bank137132 +137133 POINT(40.796024084916695 74.03973088109838) bank137133 +137134 POINT(40.78617381009613 73.92361408147558) bank137134 +137135 POINT(40.36312015586007 73.72466859117107) bank137135 +137136 POINT(40.194591891745624 74.55124156908298) bank137136 +137137 POINT(41.24333493750539 73.9976515202617) bank137137 +137138 POINT(41.62364185400297 74.14698319795167) bank137138 +137139 POINT(40.33448966640343 74.89673182627443) bank137139 +137140 POINT(40.621516069551646 73.05886879117962) bank137140 +137141 POINT(41.7044228068194 74.9385654339021) bank137141 +137142 POINT(40.58098812884333 73.91969965252831) bank137142 +137143 POINT(41.57053540586478 73.14273387933258) bank137143 +137144 POINT(40.03496627953541 74.56854207474359) bank137144 +137145 POINT(41.28460057217046 74.23604644775955) bank137145 +137146 POINT(39.80444679943438 73.82475796547186) bank137146 +137147 POINT(40.93544402427192 73.72347647048804) bank137147 +137148 POINT(41.13294789770547 73.87239558362562) bank137148 +137149 POINT(40.91077596265124 73.30610630565727) bank137149 +137150 POINT(40.511576351756325 73.03437123407673) bank137150 +137151 POINT(40.009303079738174 74.84319579651502) bank137151 +137152 POINT(40.04698393535264 74.8853531769576) bank137152 +137153 POINT(41.086754758094465 74.2858004489092) bank137153 +137154 POINT(40.898684302792574 74.82920733919065) bank137154 +137155 POINT(39.93017728255025 73.40403351296911) bank137155 +137156 POINT(41.31305560125496 73.32632270423184) bank137156 +137157 POINT(40.069720789193255 74.85268372325338) bank137157 +137158 POINT(40.50545333507992 74.54747565117084) bank137158 +137159 POINT(40.16150047202543 74.87451713925148) bank137159 +137160 POINT(40.862761539627286 74.15346662317937) bank137160 +137161 POINT(41.03063169313938 73.95168199488378) bank137161 +137162 POINT(40.74680435559974 73.15695384517052) bank137162 +137163 POINT(39.91490168951669 74.773634862286) bank137163 +137164 POINT(39.991685777526364 74.10696316316786) bank137164 +137165 POINT(40.63073195426831 74.45481410323218) bank137165 +137166 POINT(41.43709836604374 73.5831258238587) bank137166 +137167 POINT(40.91493673479818 73.37419396910131) bank137167 +137168 POINT(41.25455030361602 73.9186306165567) bank137168 +137169 POINT(39.787607862960535 73.26480810106452) bank137169 +137170 POINT(40.36713333834184 73.94800895786062) bank137170 +137171 POINT(40.859084648791516 74.81251020682865) bank137171 +137172 POINT(40.987824522059896 74.13506696874626) bank137172 +137173 POINT(41.45896193358876 73.54368483206439) bank137173 +137174 POINT(40.233165529071336 73.824419156285) bank137174 +137175 POINT(40.70235081363967 74.82102984667617) bank137175 +137176 POINT(40.97653834083625 73.30757026707253) bank137176 +137177 POINT(41.1446182846613 73.7132713526461) bank137177 +137178 POINT(40.70150937354647 74.85891564744055) bank137178 +137179 POINT(40.41692882599635 74.7223206170723) bank137179 +137180 POINT(40.662698143434866 74.94538484710122) bank137180 +137181 POINT(40.78665460312595 73.82430866209465) bank137181 +137182 POINT(41.37390128241651 73.31912773261324) bank137182 +137183 POINT(39.79508370813816 74.86209979367315) bank137183 +137184 POINT(39.99318243792296 74.1752610429383) bank137184 +137185 POINT(40.783583732394554 74.12729931578079) bank137185 +137186 POINT(41.26073032450596 74.86198967073831) bank137186 +137187 POINT(40.96027123252705 73.86423884306748) bank137187 +137188 POINT(41.02588958811094 74.27369496405476) bank137188 +137189 POINT(41.23011853486986 74.4360485010266) bank137189 +137190 POINT(41.211373876340645 73.41960983890435) bank137190 +137191 POINT(40.774771374833776 73.29663663424763) bank137191 +137192 POINT(41.44399017402619 74.80147896852291) bank137192 +137193 POINT(39.76363124772997 73.20495000024799) bank137193 +137194 POINT(39.90732775730052 73.34042466933357) bank137194 +137195 POINT(41.24046073553911 74.64052317035312) bank137195 +137196 POINT(41.25540035433378 74.92496548080479) bank137196 +137197 POINT(40.11412325867269 73.20662171489954) bank137197 +137198 POINT(40.73682614106851 74.62009462872403) bank137198 +137199 POINT(39.731411517198865 73.05391741029537) bank137199 +137200 POINT(40.82408159957603 73.14213648737342) bank137200 +137201 POINT(39.8251474767943 73.67707642241844) bank137201 +137202 POINT(41.65742721266453 73.640340926929) bank137202 +137203 POINT(40.78730605283707 73.01204869509318) bank137203 +137204 POINT(40.07891175264476 73.44063471305165) bank137204 +137205 POINT(41.28478990774074 74.6729936640833) bank137205 +137206 POINT(41.40791303460751 74.36357868176356) bank137206 +137207 POINT(39.973442052030535 74.38758371634081) bank137207 +137208 POINT(39.71398599496963 74.6593111540378) bank137208 +137209 POINT(40.6140316632947 74.65556289122802) bank137209 +137210 POINT(41.18291966616605 74.46356174168929) bank137210 +137211 POINT(40.331879685367944 74.94077330378217) bank137211 +137212 POINT(41.214690110938385 73.64018141850003) bank137212 +137213 POINT(39.77983017542669 73.09226221238504) bank137213 +137214 POINT(40.681940800608636 73.27918534480925) bank137214 +137215 POINT(40.17731433417037 73.17703043337085) bank137215 +137216 POINT(40.33210993727432 74.1991916522382) bank137216 +137217 POINT(40.68087842702471 73.25873948929639) bank137217 +137218 POINT(41.15769466138215 73.11419710475019) bank137218 +137219 POINT(40.930664131265594 73.92745798868711) bank137219 +137220 POINT(39.743385652870195 73.62463528315074) bank137220 +137221 POINT(40.78221453276205 74.99423099632546) bank137221 +137222 POINT(41.415627928341515 73.45411750515223) bank137222 +137223 POINT(40.855150186144904 74.87168468023343) bank137223 +137224 POINT(40.1770008575434 73.57791859297525) bank137224 +137225 POINT(40.61254782984959 74.06150272837071) bank137225 +137226 POINT(40.90808120263681 73.73019851445768) bank137226 +137227 POINT(41.325355184800564 74.1723643885232) bank137227 +137228 POINT(41.538932451849135 74.55593100968453) bank137228 +137229 POINT(40.56393660758482 73.51642284460331) bank137229 +137230 POINT(41.199123064427084 74.5625111719717) bank137230 +137231 POINT(41.448343198659934 74.24637320074355) bank137231 +137232 POINT(41.01707329812869 73.8804734679901) bank137232 +137233 POINT(39.909572882652924 73.64504935702954) bank137233 +137234 POINT(41.15083074631934 73.0797972590228) bank137234 +137235 POINT(40.70816375185371 74.90532096174748) bank137235 +137236 POINT(40.14906167379948 74.55124233023733) bank137236 +137237 POINT(39.83517464209676 73.4249599819713) bank137237 +137238 POINT(40.136828547916586 74.59508399193088) bank137238 +137239 POINT(40.36417690288935 73.51829871096035) bank137239 +137240 POINT(40.17336767787972 74.91818242109183) bank137240 +137241 POINT(40.35092864657174 74.05733429956977) bank137241 +137242 POINT(39.988719911396814 74.73741761682678) bank137242 +137243 POINT(40.770032303787815 74.2526160196517) bank137243 +137244 POINT(39.85750921172767 73.37879767132361) bank137244 +137245 POINT(40.63143850700587 74.00093985990164) bank137245 +137246 POINT(39.963072619735456 74.30350449588782) bank137246 +137247 POINT(40.04214456944993 73.37735957232682) bank137247 +137248 POINT(39.83155580648298 73.58745006203294) bank137248 +137249 POINT(41.60727657891371 74.7232138139964) bank137249 +137250 POINT(41.50011566473101 73.76221624486013) bank137250 +137251 POINT(41.26537432597498 73.22125177036749) bank137251 +137252 POINT(39.92880370155561 73.62345936009437) bank137252 +137253 POINT(40.11548915650896 73.0342781506698) bank137253 +137254 POINT(41.47228754714954 73.63797193873604) bank137254 +137255 POINT(39.78202780211504 73.03962712848131) bank137255 +137256 POINT(40.12235050141595 73.25796710624496) bank137256 +137257 POINT(40.77591791963166 74.50868658904068) bank137257 +137258 POINT(40.02366534419564 73.90768610795793) bank137258 +137259 POINT(39.96989295220418 74.03719561760195) bank137259 +137260 POINT(40.68252975734715 74.6637866269339) bank137260 +137261 POINT(40.06225019264361 73.48795090233321) bank137261 +137262 POINT(41.457699390127274 73.99720707637765) bank137262 +137263 POINT(41.661758875826635 73.21854525938508) bank137263 +137264 POINT(41.03896679846437 74.94399905320687) bank137264 +137265 POINT(40.326397455179496 73.6650635063759) bank137265 +137266 POINT(41.5405921625515 73.53456385310935) bank137266 +137267 POINT(41.605374594119 73.8988174410411) bank137267 +137268 POINT(41.31197110702712 74.79721293321683) bank137268 +137269 POINT(40.4750664329255 73.432852060248) bank137269 +137270 POINT(41.55193506059387 74.84565815312115) bank137270 +137271 POINT(41.23394178695241 73.38167154522898) bank137271 +137272 POINT(41.61949796673394 74.13281949158308) bank137272 +137273 POINT(40.706292927381796 73.09399824463115) bank137273 +137274 POINT(41.44682522941791 73.11784558312426) bank137274 +137275 POINT(40.70482751096771 73.14281206775325) bank137275 +137276 POINT(41.52428329237418 74.74193515857371) bank137276 +137277 POINT(40.753835560245804 73.24997645289156) bank137277 +137278 POINT(39.95062078094594 74.45077842837976) bank137278 +137279 POINT(40.59127585134051 73.485640569868) bank137279 +137280 POINT(40.41364422236996 74.57206038158148) bank137280 +137281 POINT(40.15077550694347 74.11049825196092) bank137281 +137282 POINT(40.3398946915756 73.98598300996045) bank137282 +137283 POINT(41.50913436235969 73.82872662413094) bank137283 +137284 POINT(40.848371955230796 74.72885764201772) bank137284 +137285 POINT(40.049377461069 73.53236434552407) bank137285 +137286 POINT(41.69856736315721 73.8888680705007) bank137286 +137287 POINT(41.15244194451049 73.76810804065961) bank137287 +137288 POINT(41.34953779547189 73.43061328947815) bank137288 +137289 POINT(40.36200241182585 73.24595943955906) bank137289 +137290 POINT(40.66678591214326 73.91368287515306) bank137290 +137291 POINT(39.94850521770558 74.24519622507687) bank137291 +137292 POINT(40.180865142328706 73.31919370655218) bank137292 +137293 POINT(41.01712579271853 74.71368405358214) bank137293 +137294 POINT(41.56889292519045 73.97487632546299) bank137294 +137295 POINT(39.71872116509461 74.0898997786071) bank137295 +137296 POINT(40.09493994659301 74.37509926437794) bank137296 +137297 POINT(41.35019972523096 74.4500892717404) bank137297 +137298 POINT(39.99068476577247 73.59593337732) bank137298 +137299 POINT(40.623758321937586 73.1444048123152) bank137299 +137300 POINT(40.32878755135021 74.24685691566883) bank137300 +137301 POINT(40.640067283844715 74.03594780696035) bank137301 +137302 POINT(40.12250849497555 74.44384810972674) bank137302 +137303 POINT(40.347856365816725 74.32386170741728) bank137303 +137304 POINT(40.02706714210193 74.54664021365703) bank137304 +137305 POINT(40.265072992225726 73.28214835748592) bank137305 +137306 POINT(41.0832213510377 73.3421972064332) bank137306 +137307 POINT(41.36340015184204 73.24188975276209) bank137307 +137308 POINT(41.17434993772296 73.3200200019238) bank137308 +137309 POINT(40.370715114259696 73.79194623940637) bank137309 +137310 POINT(41.65812681224211 73.71677986852052) bank137310 +137311 POINT(41.174479685805906 74.02658938514422) bank137311 +137312 POINT(41.06614899318273 74.79403757065347) bank137312 +137313 POINT(40.08227671538513 73.52180373661147) bank137313 +137314 POINT(40.73861842184191 74.62577616426611) bank137314 +137315 POINT(40.5269573734191 74.23027200984461) bank137315 +137316 POINT(40.75800147691337 74.45422637025484) bank137316 +137317 POINT(39.94889928523843 73.02611563107017) bank137317 +137318 POINT(39.99840427624923 73.14587354287302) bank137318 +137319 POINT(40.80310414101214 73.2327871185176) bank137319 +137320 POINT(41.29842758236576 74.04833533047284) bank137320 +137321 POINT(40.86288359703518 74.79226015764795) bank137321 +137322 POINT(40.76296419602696 74.13561390332784) bank137322 +137323 POINT(39.89525408567846 73.96363939270577) bank137323 +137324 POINT(41.69903420247899 74.1169252894292) bank137324 +137325 POINT(41.256443296572186 74.85723219656184) bank137325 +137326 POINT(40.45926992598235 74.86367815692262) bank137326 +137327 POINT(40.47931372957139 74.4328998965983) bank137327 +137328 POINT(40.380308548040674 74.45656086630905) bank137328 +137329 POINT(40.10668779887893 73.86011890694361) bank137329 +137330 POINT(41.296818564307095 73.8963760028369) bank137330 +137331 POINT(41.40794674733611 74.73301463086752) bank137331 +137332 POINT(41.06556971758783 73.06391480425272) bank137332 +137333 POINT(39.88288664157516 74.82715750969243) bank137333 +137334 POINT(40.872133090917856 73.61857549122627) bank137334 +137335 POINT(39.7595243602556 74.10681404735426) bank137335 +137336 POINT(41.03316588734321 73.52879717274946) bank137336 +137337 POINT(39.89662391747183 73.14208135517804) bank137337 +137338 POINT(41.13309294906838 74.3199674020417) bank137338 +137339 POINT(41.323290697403664 73.49239622395109) bank137339 +137340 POINT(41.512487080112884 73.78620640935048) bank137340 +137341 POINT(40.28067871817203 73.81540489634716) bank137341 +137342 POINT(41.18843925629431 74.05049871665948) bank137342 +137343 POINT(41.50385316178782 74.58840293289036) bank137343 +137344 POINT(40.48901198560472 73.18930556160066) bank137344 +137345 POINT(40.328805365057896 73.17900988513335) bank137345 +137346 POINT(40.65658680890004 73.12289335707625) bank137346 +137347 POINT(41.69900638758469 74.52698532119607) bank137347 +137348 POINT(41.19465558745991 73.91044023050995) bank137348 +137349 POINT(41.44527548937038 74.85340126458078) bank137349 +137350 POINT(39.9783304858265 73.49204351483668) bank137350 +137351 POINT(39.8277426451034 73.66547867191747) bank137351 +137352 POINT(40.10713456228439 74.19456304827776) bank137352 +137353 POINT(41.34353606151227 74.40294256613929) bank137353 +137354 POINT(40.32866413099405 73.52863952229089) bank137354 +137355 POINT(41.481784821038055 74.91790205363147) bank137355 +137356 POINT(40.48155009650286 73.19297448541356) bank137356 +137357 POINT(41.51398534351857 73.37923631348862) bank137357 +137358 POINT(39.754279498659145 74.89901729103022) bank137358 +137359 POINT(40.49714241337519 74.75406176024599) bank137359 +137360 POINT(40.81831444379171 74.25711008894535) bank137360 +137361 POINT(40.328649470181446 73.08959644261567) bank137361 +137362 POINT(40.02804550347047 73.88284426214862) bank137362 +137363 POINT(41.069377502220846 73.5259657065881) bank137363 +137364 POINT(40.61874855986931 73.66086987199449) bank137364 +137365 POINT(40.394595529801386 73.71110184825432) bank137365 +137366 POINT(41.342253342434475 74.67063214187047) bank137366 +137367 POINT(40.4743930290094 74.15726507923873) bank137367 +137368 POINT(40.575585825729895 74.15437846580527) bank137368 +137369 POINT(39.92511695712492 73.59275311987186) bank137369 +137370 POINT(41.12213602686212 73.91176162619881) bank137370 +137371 POINT(39.99922539292315 74.72365190606648) bank137371 +137372 POINT(41.12884888800175 74.06883016341169) bank137372 +137373 POINT(40.297613893170414 73.58563884897876) bank137373 +137374 POINT(40.60830660553293 74.45534251661539) bank137374 +137375 POINT(40.10623040437566 73.57003197460702) bank137375 +137376 POINT(41.70747342320948 73.50270271208448) bank137376 +137377 POINT(40.14796628021997 73.00728088922965) bank137377 +137378 POINT(40.715210844398165 73.13310140880424) bank137378 +137379 POINT(40.64517984129025 74.5183342315149) bank137379 +137380 POINT(40.61857423230857 74.11559847299776) bank137380 +137381 POINT(40.04630298329995 74.45139086748185) bank137381 +137382 POINT(40.682023856321294 74.78939252692595) bank137382 +137383 POINT(40.10863912827127 74.66547722352367) bank137383 +137384 POINT(39.86710363593929 74.44153600440573) bank137384 +137385 POINT(40.17895057967699 74.71183865844604) bank137385 +137386 POINT(40.635725872240215 73.82882272332284) bank137386 +137387 POINT(40.85851061720042 74.68452638182477) bank137387 +137388 POINT(41.271163703308325 74.42078056700414) bank137388 +137389 POINT(40.34021543778682 73.77436339940539) bank137389 +137390 POINT(41.3377204906332 74.96890768023715) bank137390 +137391 POINT(41.47381255933027 73.46715176480046) bank137391 +137392 POINT(39.85844583625163 73.31272568222487) bank137392 +137393 POINT(41.30484245039522 73.22113438200876) bank137393 +137394 POINT(40.72646430748627 73.22516185961648) bank137394 +137395 POINT(41.64694093342976 73.86579755997707) bank137395 +137396 POINT(41.0052214821879 74.35944423976547) bank137396 +137397 POINT(40.44005713162227 74.97091702634712) bank137397 +137398 POINT(40.134861027985124 73.26791602766082) bank137398 +137399 POINT(40.75243758594289 73.90033924543759) bank137399 +137400 POINT(41.27193808525613 74.8896654736946) bank137400 +137401 POINT(40.18537316083213 74.13525930114785) bank137401 +137402 POINT(40.98270829635727 74.94356460633266) bank137402 +137403 POINT(40.22421467962927 74.29498098504381) bank137403 +137404 POINT(40.39472581934233 74.01320531254589) bank137404 +137405 POINT(40.72541378625441 73.52044430577227) bank137405 +137406 POINT(41.21302876322941 74.70322979137339) bank137406 +137407 POINT(39.96148631611462 74.7349884673109) bank137407 +137408 POINT(40.72366628035706 74.72662559709995) bank137408 +137409 POINT(40.05824310286252 74.80318881215466) bank137409 +137410 POINT(40.565694637172285 73.30191395856338) bank137410 +137411 POINT(41.36886838540697 73.82634256569493) bank137411 +137412 POINT(41.283575438421934 74.94690263463676) bank137412 +137413 POINT(40.75351144399656 73.47433901185232) bank137413 +137414 POINT(40.19012253271112 74.499085442176) bank137414 +137415 POINT(40.558403477277125 74.49846047623821) bank137415 +137416 POINT(40.901575823298344 74.07777365219164) bank137416 +137417 POINT(40.46453030344084 74.16737926215214) bank137417 +137418 POINT(41.658494795838024 73.31713152348085) bank137418 +137419 POINT(41.1676544070628 73.82101407949528) bank137419 +137420 POINT(41.162863643472456 74.71932965997595) bank137420 +137421 POINT(41.626647486062154 74.89247834898964) bank137421 +137422 POINT(41.55011581097351 74.97971154812932) bank137422 +137423 POINT(41.597503829085525 73.06027614133117) bank137423 +137424 POINT(40.21743518255208 73.92832482478812) bank137424 +137425 POINT(41.5065257189174 74.69399210555443) bank137425 +137426 POINT(40.33978860842901 73.21273624951075) bank137426 +137427 POINT(40.76198066071571 74.38240269453578) bank137427 +137428 POINT(40.47362463530354 74.74409221088526) bank137428 +137429 POINT(40.57452259746359 73.83112827084597) bank137429 +137430 POINT(40.819423207625356 73.67328808360347) bank137430 +137431 POINT(40.86582129297914 73.3073774981602) bank137431 +137432 POINT(40.30579772511821 74.9356352697129) bank137432 +137433 POINT(41.23969875937225 73.36395442861016) bank137433 +137434 POINT(41.18822671694524 73.94957810171829) bank137434 +137435 POINT(40.479957106818546 74.71338965779792) bank137435 +137436 POINT(40.42173187607786 73.84576558521239) bank137436 +137437 POINT(40.60542555279444 74.38715454559758) bank137437 +137438 POINT(39.89916049606431 74.10149361411267) bank137438 +137439 POINT(40.393041008519965 73.83889976126629) bank137439 +137440 POINT(39.791640199661025 74.00379848321226) bank137440 +137441 POINT(40.99373025464011 73.68393752263721) bank137441 +137442 POINT(40.21780781455757 73.41242161272895) bank137442 +137443 POINT(39.73663342443972 74.00935921377649) bank137443 +137444 POINT(39.7586311090427 73.31564283450022) bank137444 +137445 POINT(39.81864837338979 74.20721240161042) bank137445 +137446 POINT(41.61182359420578 74.19091156926284) bank137446 +137447 POINT(40.40494143257966 74.6640759943827) bank137447 +137448 POINT(40.83954590154619 74.15909348478733) bank137448 +137449 POINT(39.828920125268645 73.73645422498517) bank137449 +137450 POINT(40.30902064935335 74.85459821424587) bank137450 +137451 POINT(41.38333261014714 73.83939451460554) bank137451 +137452 POINT(41.05215553429403 75.00587115988638) bank137452 +137453 POINT(40.28094969231147 73.15940817478752) bank137453 +137454 POINT(40.37602634534632 74.79002006945258) bank137454 +137455 POINT(41.47536260425056 73.11016090440279) bank137455 +137456 POINT(41.00762118727018 74.32511737062694) bank137456 +137457 POINT(41.02309283148116 73.36671009594554) bank137457 +137458 POINT(39.8869063499992 73.89787243534033) bank137458 +137459 POINT(40.176275808728526 73.47861843549029) bank137459 +137460 POINT(40.44450622148203 73.0490359084378) bank137460 +137461 POINT(40.08299014716899 73.7806602351032) bank137461 +137462 POINT(41.57702418211582 74.0856474299919) bank137462 +137463 POINT(40.34449955828979 73.08685823019286) bank137463 +137464 POINT(40.535327012992695 73.13255688773805) bank137464 +137465 POINT(39.94838572764067 73.96153849322629) bank137465 +137466 POINT(40.75097593813112 74.90147552610156) bank137466 +137467 POINT(40.605952732194496 73.75611827943783) bank137467 +137468 POINT(40.58950440883984 73.66611924849879) bank137468 +137469 POINT(41.555540572487814 74.38257132849587) bank137469 +137470 POINT(40.037722479256935 74.49641423215236) bank137470 +137471 POINT(39.7460580473962 74.19185147720977) bank137471 +137472 POINT(41.4295263739473 74.79089765421222) bank137472 +137473 POINT(40.16160738349583 73.80754772727595) bank137473 +137474 POINT(40.32502451448372 74.08589720371027) bank137474 +137475 POINT(40.96533558291842 74.53881814624309) bank137475 +137476 POINT(39.713307662298874 73.1626776810415) bank137476 +137477 POINT(40.37196654414464 74.97613375993512) bank137477 +137478 POINT(39.95144028973811 73.03088430624332) bank137478 +137479 POINT(40.59476420085884 73.08238912148738) bank137479 +137480 POINT(41.44690006148346 74.09994976628474) bank137480 +137481 POINT(40.03882612171714 74.10959607909167) bank137481 +137482 POINT(40.05743530386327 73.68837975002074) bank137482 +137483 POINT(40.96726258046326 74.80079497255782) bank137483 +137484 POINT(40.65296019220116 74.98642089770564) bank137484 +137485 POINT(40.750981525789356 74.01394587838102) bank137485 +137486 POINT(41.574964749798525 74.41376579885572) bank137486 +137487 POINT(40.05674467426832 74.71674762975661) bank137487 +137488 POINT(40.0160081046133 74.54241898006335) bank137488 +137489 POINT(40.315325565616675 74.45231597647064) bank137489 +137490 POINT(40.49664311215408 73.66510255078005) bank137490 +137491 POINT(41.699093719356895 73.45739380127907) bank137491 +137492 POINT(40.38846591771405 74.41672970550334) bank137492 +137493 POINT(41.41647245184356 74.15800600108597) bank137493 +137494 POINT(41.371352080600175 74.91167887301228) bank137494 +137495 POINT(40.607591903449645 74.67316205231711) bank137495 +137496 POINT(40.39659426293972 73.28533857381507) bank137496 +137497 POINT(41.5739018063482 73.23509138505828) bank137497 +137498 POINT(40.79057305644934 73.46311895184508) bank137498 +137499 POINT(40.09127716722826 74.4457983230243) bank137499 +137500 POINT(40.34637264618343 74.43888236343992) bank137500 +137501 POINT(40.613317088687936 73.19535676569615) bank137501 +137502 POINT(39.95196582729207 74.52305061534233) bank137502 +137503 POINT(40.496380740103525 74.328848225997) bank137503 +137504 POINT(41.653207764541634 73.10536713528124) bank137504 +137505 POINT(40.80585530131943 74.77977760369953) bank137505 +137506 POINT(40.140095160656124 74.58027371689761) bank137506 +137507 POINT(40.566677054141714 74.28302784251457) bank137507 +137508 POINT(40.42462342109624 73.7738342000574) bank137508 +137509 POINT(40.42088872426685 74.82991513337684) bank137509 +137510 POINT(40.09092658590204 74.76594898890652) bank137510 +137511 POINT(40.68159585999405 73.89684198599988) bank137511 +137512 POINT(40.72788011248656 74.22215185783341) bank137512 +137513 POINT(40.07807354257531 73.03951037930591) bank137513 +137514 POINT(40.807884432575534 73.14210957041546) bank137514 +137515 POINT(40.283713322523724 73.98860648269829) bank137515 +137516 POINT(40.69305896404149 73.83909894682759) bank137516 +137517 POINT(40.32024097055111 73.82220885198078) bank137517 +137518 POINT(40.277410024799146 74.15547180350403) bank137518 +137519 POINT(41.51288227414764 73.20896150881606) bank137519 +137520 POINT(39.816034668601134 73.54993093992822) bank137520 +137521 POINT(39.83489919539376 74.08154509429879) bank137521 +137522 POINT(39.86765702711708 73.57595355530916) bank137522 +137523 POINT(40.37751374633922 73.50127807587323) bank137523 +137524 POINT(41.29445033930938 73.97193407730117) bank137524 +137525 POINT(41.08100661123853 73.40838896955188) bank137525 +137526 POINT(40.225569021123405 74.25605490223575) bank137526 +137527 POINT(41.153983295172516 73.16423737966421) bank137527 +137528 POINT(41.54175885826066 73.10889897930255) bank137528 +137529 POINT(39.99447724540712 74.04045535951187) bank137529 +137530 POINT(40.08936669675904 74.8432265528003) bank137530 +137531 POINT(41.00738624519964 73.02904760304367) bank137531 +137532 POINT(39.95047701083711 73.04920360666782) bank137532 +137533 POINT(40.67977064245219 74.94173639693965) bank137533 +137534 POINT(40.29959653397676 73.16126274433007) bank137534 +137535 POINT(40.33813912526648 73.5425565193276) bank137535 +137536 POINT(40.79992605317509 74.5760551148999) bank137536 +137537 POINT(40.72058529803195 74.98689505984798) bank137537 +137538 POINT(40.69204299039029 73.662043423475) bank137538 +137539 POINT(41.187131873783514 74.45459835845843) bank137539 +137540 POINT(40.66254572056201 73.6966481590012) bank137540 +137541 POINT(41.63296781844184 73.0605760837281) bank137541 +137542 POINT(39.7561919230547 73.02426445620432) bank137542 +137543 POINT(40.366515497681945 74.6594052161346) bank137543 +137544 POINT(40.08125771497969 74.93718392405113) bank137544 +137545 POINT(40.91444435620659 73.77984960591739) bank137545 +137546 POINT(40.17290475075239 74.1143059632982) bank137546 +137547 POINT(41.46018277472573 74.07092836185723) bank137547 +137548 POINT(40.36954764442088 73.34415929749035) bank137548 +137549 POINT(40.69479064799413 73.23852843118533) bank137549 +137550 POINT(41.12889344287279 74.4350596884448) bank137550 +137551 POINT(40.99050892589343 73.30487453022744) bank137551 +137552 POINT(41.37464628673922 73.45756527136608) bank137552 +137553 POINT(40.35259219683266 74.03223145151215) bank137553 +137554 POINT(40.113545119984764 74.75307853446866) bank137554 +137555 POINT(40.6233979557616 74.76625000155923) bank137555 +137556 POINT(40.41499928095664 73.65906378454743) bank137556 +137557 POINT(41.607035007517 73.38721770762199) bank137557 +137558 POINT(41.11769011120246 73.50592396248452) bank137558 +137559 POINT(41.6156163066849 73.647131878153) bank137559 +137560 POINT(41.23462159025827 73.69017506391761) bank137560 +137561 POINT(41.45222294231532 74.16653206656238) bank137561 +137562 POINT(40.71179598286241 73.17436215861814) bank137562 +137563 POINT(40.85250844464804 73.07878198957029) bank137563 +137564 POINT(40.676663264295925 74.58502862610288) bank137564 +137565 POINT(40.70641500060881 74.34196445716945) bank137565 +137566 POINT(41.42173259335729 73.81085916451278) bank137566 +137567 POINT(41.5710125303341 74.93539616356041) bank137567 +137568 POINT(40.13003394964215 73.4168218224939) bank137568 +137569 POINT(40.421468444569314 73.58567716902246) bank137569 +137570 POINT(41.00955448345259 73.59190857033339) bank137570 +137571 POINT(40.99763890196474 74.16408084111693) bank137571 +137572 POINT(41.341731097303914 74.8471468029348) bank137572 +137573 POINT(41.12583304790718 74.21252128109101) bank137573 +137574 POINT(40.59035943592523 74.00264706088511) bank137574 +137575 POINT(41.063035855350634 73.10672217661981) bank137575 +137576 POINT(40.8540856386636 74.30545894874749) bank137576 +137577 POINT(40.097825989145164 73.62805109937764) bank137577 +137578 POINT(41.673052794294904 74.7263583380829) bank137578 +137579 POINT(41.131840135682445 73.06667606403998) bank137579 +137580 POINT(40.97987223528731 74.15557729650719) bank137580 +137581 POINT(40.30697618324402 73.89402065958741) bank137581 +137582 POINT(40.5408163482457 74.39203299501337) bank137582 +137583 POINT(41.38315485459835 73.58144428980334) bank137583 +137584 POINT(41.187792114695014 74.04792615632017) bank137584 +137585 POINT(40.366432918967476 74.37589607879733) bank137585 +137586 POINT(41.289999219093545 74.82446700353803) bank137586 +137587 POINT(40.96733124525755 73.59921402692827) bank137587 +137588 POINT(40.50595087938566 73.95730930062014) bank137588 +137589 POINT(40.70237512760235 73.92166808502908) bank137589 +137590 POINT(41.20723805535483 73.62977667958047) bank137590 +137591 POINT(40.39669234670857 73.37752112188289) bank137591 +137592 POINT(40.263825991365955 74.80923610738508) bank137592 +137593 POINT(40.701122988119195 74.86506285867199) bank137593 +137594 POINT(40.41004724314141 73.39075419556107) bank137594 +137595 POINT(40.62234057643654 74.80220224976361) bank137595 +137596 POINT(41.462003553069444 74.17702263377504) bank137596 +137597 POINT(41.02011127183008 73.69125799753301) bank137597 +137598 POINT(40.57453816851709 73.03893002257325) bank137598 +137599 POINT(39.930993363543784 73.77360529952021) bank137599 +137600 POINT(40.213850564277806 74.02042014487812) bank137600 +137601 POINT(41.43126406194935 74.8479998312506) bank137601 +137602 POINT(40.74748418157773 73.16735232476084) bank137602 +137603 POINT(41.37933719274358 73.7020221176175) bank137603 +137604 POINT(39.945934758325265 74.64924189789484) bank137604 +137605 POINT(39.876526888263776 73.35231758741891) bank137605 +137606 POINT(39.98853032922285 73.52173349605236) bank137606 +137607 POINT(39.79317810559465 74.5842549400587) bank137607 +137608 POINT(41.48399254781686 74.39319088067057) bank137608 +137609 POINT(41.41124470311498 73.66311932344746) bank137609 +137610 POINT(41.42146613109028 74.92366429083155) bank137610 +137611 POINT(39.885974369177674 73.83303329795207) bank137611 +137612 POINT(40.67506578092082 74.63112075025714) bank137612 +137613 POINT(40.629340925343556 73.22165907286026) bank137613 +137614 POINT(41.52036016403464 74.92801172241244) bank137614 +137615 POINT(41.234872667523256 73.04081447918158) bank137615 +137616 POINT(39.98650118881428 74.89971551785742) bank137616 +137617 POINT(40.536327900050374 74.13475966045426) bank137617 +137618 POINT(39.85900725154305 73.43424297003419) bank137618 +137619 POINT(40.236310770116575 73.03365204280388) bank137619 +137620 POINT(40.49029957132974 74.48073340141515) bank137620 +137621 POINT(39.94993303419672 74.00024879615714) bank137621 +137622 POINT(40.3006224863125 74.30512644189727) bank137622 +137623 POINT(40.20381419465441 74.0621747111409) bank137623 +137624 POINT(40.385467864044536 73.90175995404503) bank137624 +137625 POINT(40.127073294479196 73.08377584479338) bank137625 +137626 POINT(41.44752376507641 74.8032325374306) bank137626 +137627 POINT(41.53377702275631 73.33844696214607) bank137627 +137628 POINT(40.50170261364307 74.00359033240088) bank137628 +137629 POINT(40.48971978072943 74.5416978541252) bank137629 +137630 POINT(40.10810688983256 74.19150913767746) bank137630 +137631 POINT(40.245057515672165 73.30354834202002) bank137631 +137632 POINT(40.05550756039939 73.27779824909541) bank137632 +137633 POINT(40.95131938325429 73.92515791269062) bank137633 +137634 POINT(41.114090746596766 73.13955333206663) bank137634 +137635 POINT(40.14576537257904 74.06001735712182) bank137635 +137636 POINT(40.11742643374401 74.37542690006883) bank137636 +137637 POINT(40.21173509877836 73.90754927883762) bank137637 +137638 POINT(40.837833166356674 73.1535939810808) bank137638 +137639 POINT(39.9474586384721 74.56187667037386) bank137639 +137640 POINT(39.883210591555226 74.75756773196072) bank137640 +137641 POINT(40.85587776752389 74.04737434929152) bank137641 +137642 POINT(41.545510624141954 74.5511103968031) bank137642 +137643 POINT(41.70770161158293 74.82920462425413) bank137643 +137644 POINT(40.43939852807184 74.43467384915432) bank137644 +137645 POINT(40.83382917955023 74.13649499617894) bank137645 +137646 POINT(40.793439230691625 74.33622465583392) bank137646 +137647 POINT(40.71982373060651 73.08167729425305) bank137647 +137648 POINT(41.21370048131716 73.296302684361) bank137648 +137649 POINT(39.909613640409326 74.02294314414476) bank137649 +137650 POINT(41.633479366457564 74.37536649823423) bank137650 +137651 POINT(41.13806986774857 74.1995805920325) bank137651 +137652 POINT(41.59599096608277 74.72438150709058) bank137652 +137653 POINT(40.16683106934811 73.01566454129988) bank137653 +137654 POINT(40.95987324047897 74.70719558550743) bank137654 +137655 POINT(39.85110094571977 73.66607931457254) bank137655 +137656 POINT(41.035307573654826 74.25372920667836) bank137656 +137657 POINT(39.80262674677799 73.12605198682859) bank137657 +137658 POINT(40.92083885138276 73.88608299619658) bank137658 +137659 POINT(39.739212564491915 73.65078163263959) bank137659 +137660 POINT(39.77191991337658 73.897461288951) bank137660 +137661 POINT(41.06913523990836 74.05894810670917) bank137661 +137662 POINT(40.43800321242256 73.52956748648032) bank137662 +137663 POINT(40.22236803362824 74.64038799318354) bank137663 +137664 POINT(39.87991848372248 74.83204474605232) bank137664 +137665 POINT(41.492940417840174 73.95543987229726) bank137665 +137666 POINT(39.78527479942372 74.100217476531) bank137666 +137667 POINT(40.12883773304529 74.76354937660601) bank137667 +137668 POINT(40.31567849956008 73.55445374396463) bank137668 +137669 POINT(40.7476773755229 73.16872609434863) bank137669 +137670 POINT(41.11553846261036 74.77394813926816) bank137670 +137671 POINT(41.07802131227666 74.00757015451296) bank137671 +137672 POINT(40.393100113465685 73.76311831384018) bank137672 +137673 POINT(41.392218536769484 74.54768661582392) bank137673 +137674 POINT(39.92451838976235 74.2021288324305) bank137674 +137675 POINT(41.158853869598666 73.0162776193649) bank137675 +137676 POINT(39.83448297392369 73.71059862353228) bank137676 +137677 POINT(40.96789307757913 74.20002447992948) bank137677 +137678 POINT(41.07372240944557 74.42986689379835) bank137678 +137679 POINT(40.81646291642333 73.76842974741662) bank137679 +137680 POINT(40.015616724209075 74.04051356952593) bank137680 +137681 POINT(40.688391771061376 74.41895644948652) bank137681 +137682 POINT(40.91266036929946 73.28718964140259) bank137682 +137683 POINT(40.43617362467809 73.54741147886443) bank137683 +137684 POINT(40.202710249554144 73.69210067203646) bank137684 +137685 POINT(41.56345488637605 73.8015181316016) bank137685 +137686 POINT(41.54356262090778 73.29426387710834) bank137686 +137687 POINT(39.72125943105739 73.86197445882334) bank137687 +137688 POINT(40.626756754328355 74.91418742458117) bank137688 +137689 POINT(40.090547394882684 74.06877765207821) bank137689 +137690 POINT(41.023197985482525 73.31859295304362) bank137690 +137691 POINT(40.519464388641325 74.48057173156177) bank137691 +137692 POINT(40.044413710167824 73.45663659425655) bank137692 +137693 POINT(41.56645942008202 73.16406266106137) bank137693 +137694 POINT(41.419024635009244 73.2748166797993) bank137694 +137695 POINT(40.795483699892635 74.14111677187607) bank137695 +137696 POINT(40.8349906451952 74.42863842345898) bank137696 +137697 POINT(40.58022426793793 74.31936415325238) bank137697 +137698 POINT(41.67078703222649 73.59073999245791) bank137698 +137699 POINT(41.346938805041994 74.34921526816504) bank137699 +137700 POINT(41.42037235988317 74.19860316665032) bank137700 +137701 POINT(41.29178436829532 73.71382386634089) bank137701 +137702 POINT(40.406142648949256 73.54509641911528) bank137702 +137703 POINT(39.93905951327658 73.83429010542203) bank137703 +137704 POINT(40.6730679871996 73.88728141417592) bank137704 +137705 POINT(39.93667481041144 74.36436861546072) bank137705 +137706 POINT(40.18644854596223 74.70489105888875) bank137706 +137707 POINT(40.89317226361906 73.20058707107097) bank137707 +137708 POINT(40.35467118442043 74.85710795190319) bank137708 +137709 POINT(41.29512622131351 73.31400703376347) bank137709 +137710 POINT(39.916290589904015 74.2684469804112) bank137710 +137711 POINT(41.27796633836457 73.9936791460556) bank137711 +137712 POINT(39.92242924288584 74.44504655672375) bank137712 +137713 POINT(41.24517764709204 74.77911198772239) bank137713 +137714 POINT(41.511805400019355 74.42508710325504) bank137714 +137715 POINT(41.06998277153455 73.74432349763141) bank137715 +137716 POINT(40.67249965371181 73.05864711635431) bank137716 +137717 POINT(41.075049539301055 73.5610551494168) bank137717 +137718 POINT(41.52393228902998 74.5515876433568) bank137718 +137719 POINT(40.191990851779856 73.80879539533376) bank137719 +137720 POINT(40.166439435254986 74.04837060732638) bank137720 +137721 POINT(41.57754864490213 74.63998903144454) bank137721 +137722 POINT(40.17302665851854 74.85294469552977) bank137722 +137723 POINT(40.96391263957134 73.55399346680151) bank137723 +137724 POINT(40.89544948699184 73.0090679445546) bank137724 +137725 POINT(40.46654936762413 74.42631877520593) bank137725 +137726 POINT(40.417009355346806 74.67922967366748) bank137726 +137727 POINT(40.961958778910564 74.58434234947583) bank137727 +137728 POINT(41.098580352346325 73.74777632576442) bank137728 +137729 POINT(41.35207327932794 74.71425162291047) bank137729 +137730 POINT(39.990866330097596 73.93113903040181) bank137730 +137731 POINT(41.51078848077388 73.64091972755246) bank137731 +137732 POINT(40.066597398932906 74.96923927177814) bank137732 +137733 POINT(41.420080521190236 73.12250312114602) bank137733 +137734 POINT(39.85853798497723 73.60113426551305) bank137734 +137735 POINT(40.90226973409827 73.1733106503678) bank137735 +137736 POINT(41.69347245224262 73.67972106978895) bank137736 +137737 POINT(41.03742991236049 73.43856927796328) bank137737 +137738 POINT(39.978768779600294 73.7707424297412) bank137738 +137739 POINT(40.93490002554374 74.94620884963706) bank137739 +137740 POINT(40.89945466236633 74.73006711751536) bank137740 +137741 POINT(40.79385016985256 73.82373228827588) bank137741 +137742 POINT(40.77174542953715 73.7636861797676) bank137742 +137743 POINT(40.88304592060154 73.99730226340348) bank137743 +137744 POINT(40.07313301896371 73.22921495957374) bank137744 +137745 POINT(41.530921355735366 73.009613589106) bank137745 +137746 POINT(40.028660716855775 74.86304812362964) bank137746 +137747 POINT(40.685082579107934 74.27726182734557) bank137747 +137748 POINT(41.10352349886115 74.95532395889275) bank137748 +137749 POINT(40.74283510991757 73.13239419873574) bank137749 +137750 POINT(41.49986351979449 74.54059744031044) bank137750 +137751 POINT(40.33940826619006 74.22048529114541) bank137751 +137752 POINT(40.815507703702785 73.57312871192673) bank137752 +137753 POINT(40.97969153606727 73.31803762925736) bank137753 +137754 POINT(40.828933623777175 73.21622069805002) bank137754 +137755 POINT(41.40085710418477 74.46886505015823) bank137755 +137756 POINT(41.20528503731047 74.6219871159324) bank137756 +137757 POINT(39.83159348803921 73.78639975928776) bank137757 +137758 POINT(40.15006865675293 73.25914706714136) bank137758 +137759 POINT(39.925181055765634 73.41444835342615) bank137759 +137760 POINT(40.581128051868355 73.79063802010243) bank137760 +137761 POINT(40.075446574878086 74.1592661691899) bank137761 +137762 POINT(40.340524612015734 74.61008384577146) bank137762 +137763 POINT(40.885577733898174 74.16272333760172) bank137763 +137764 POINT(41.61127718115568 73.5227720580822) bank137764 +137765 POINT(40.75053510031107 73.50094097322855) bank137765 +137766 POINT(40.16436676212187 73.11162442392704) bank137766 +137767 POINT(40.55589244881621 74.36897593048117) bank137767 +137768 POINT(39.844934640412355 73.87577822363377) bank137768 +137769 POINT(40.87508317186746 73.72002744454417) bank137769 +137770 POINT(39.994600443155996 73.14035982557176) bank137770 +137771 POINT(39.91440358819229 73.91794476688047) bank137771 +137772 POINT(41.20201657909562 74.19949688913101) bank137772 +137773 POINT(41.52852876597731 74.51786244194447) bank137773 +137774 POINT(39.978281520355075 74.91752194759363) bank137774 +137775 POINT(41.130124169025066 73.66105483439442) bank137775 +137776 POINT(40.46483912012073 74.8686716552448) bank137776 +137777 POINT(41.532684635832396 73.28951618048305) bank137777 +137778 POINT(41.59066662628784 74.1122332051207) bank137778 +137779 POINT(41.12991269525761 73.25596072462446) bank137779 +137780 POINT(41.09737163215819 74.15690677222587) bank137780 +137781 POINT(41.50384559459519 73.99393380893412) bank137781 +137782 POINT(41.336449544073524 74.99080212249328) bank137782 +137783 POINT(41.619880740604486 74.7866551840782) bank137783 +137784 POINT(40.26742164358962 74.67254886162183) bank137784 +137785 POINT(41.334793108814495 74.48479671953204) bank137785 +137786 POINT(39.856591718330606 73.92429044251831) bank137786 +137787 POINT(40.37540829710964 74.54341585468306) bank137787 +137788 POINT(40.10160435663664 74.47058920811297) bank137788 +137789 POINT(39.78266378671583 74.070057346079) bank137789 +137790 POINT(39.745119140465135 73.81637324807168) bank137790 +137791 POINT(40.43216513749506 74.5071847123249) bank137791 +137792 POINT(41.3945592507425 75.00396565901366) bank137792 +137793 POINT(40.19637933134615 73.79197908560347) bank137793 +137794 POINT(40.43898128623146 73.34859458426897) bank137794 +137795 POINT(40.12215764098901 73.70338699329676) bank137795 +137796 POINT(41.07382800525962 73.323127096449) bank137796 +137797 POINT(40.136085046790654 73.11289317173086) bank137797 +137798 POINT(40.06290564550171 74.4559511786889) bank137798 +137799 POINT(40.20208691870484 73.14364622982521) bank137799 +137800 POINT(39.90473434366795 73.52343423043126) bank137800 +137801 POINT(40.305997864367974 74.9126324215576) bank137801 +137802 POINT(40.817450144513145 74.8890127182481) bank137802 +137803 POINT(40.69001016893194 74.43356993960676) bank137803 +137804 POINT(40.28208218523235 73.88711837496935) bank137804 +137805 POINT(40.40117762638549 74.3535239249108) bank137805 +137806 POINT(40.748284863072854 73.12191778526586) bank137806 +137807 POINT(41.2004995173062 73.18422559296164) bank137807 +137808 POINT(40.23612321641673 74.30471683425762) bank137808 +137809 POINT(40.70363251796632 73.633849276377) bank137809 +137810 POINT(40.89475907155079 73.59174331796996) bank137810 +137811 POINT(41.11313202319351 74.84049540965054) bank137811 +137812 POINT(39.94173496799972 74.3025515703607) bank137812 +137813 POINT(40.66026971017746 74.69832392931275) bank137813 +137814 POINT(40.904549718553625 74.10102880217678) bank137814 +137815 POINT(40.830323414724745 73.14922399476006) bank137815 +137816 POINT(39.93443583599826 73.59258933505828) bank137816 +137817 POINT(40.3937580799959 73.39599103757911) bank137817 +137818 POINT(40.31101809890464 74.85556914643008) bank137818 +137819 POINT(40.095248120866515 74.0370744310097) bank137819 +137820 POINT(39.965705923688525 73.80265763013499) bank137820 +137821 POINT(40.93222417533389 73.88671610628214) bank137821 +137822 POINT(41.65140058386183 74.31274787656571) bank137822 +137823 POINT(40.88615328621343 73.18758379122225) bank137823 +137824 POINT(40.226032072565914 73.59389708717127) bank137824 +137825 POINT(41.06940180026934 74.55912808087174) bank137825 +137826 POINT(40.36737958545399 74.01497424334102) bank137826 +137827 POINT(41.02463869136672 74.02213948382476) bank137827 +137828 POINT(40.01721701154293 73.06183751663882) bank137828 +137829 POINT(41.19988799446464 73.76423613000462) bank137829 +137830 POINT(40.20534136134452 73.88286157540176) bank137830 +137831 POINT(39.98178955347186 73.33051548825343) bank137831 +137832 POINT(40.606273830570586 74.31754263580271) bank137832 +137833 POINT(40.1331868096216 73.96957180349035) bank137833 +137834 POINT(39.927598921346785 73.80288078707761) bank137834 +137835 POINT(41.35504337146459 73.43143057852275) bank137835 +137836 POINT(41.418826470029494 73.21242984039597) bank137836 +137837 POINT(39.9072009715749 73.69061522300058) bank137837 +137838 POINT(41.58419349157241 73.94192626633894) bank137838 +137839 POINT(40.479558834387944 74.72564726031828) bank137839 +137840 POINT(40.719944751048686 73.91816233485277) bank137840 +137841 POINT(40.24861642552615 74.29868241941321) bank137841 +137842 POINT(41.68280007534718 74.92912507399797) bank137842 +137843 POINT(39.76716812977058 74.7626930727158) bank137843 +137844 POINT(41.15673117328629 74.26769160169974) bank137844 +137845 POINT(40.62833553071084 74.67445805886487) bank137845 +137846 POINT(41.270207180420286 73.55773777201516) bank137846 +137847 POINT(39.87740376941576 74.94195230655541) bank137847 +137848 POINT(39.915189102025906 74.21707084104912) bank137848 +137849 POINT(41.67616832252973 74.14381825175958) bank137849 +137850 POINT(39.7486164971872 74.12894945873228) bank137850 +137851 POINT(40.761764195281465 74.06829099928902) bank137851 +137852 POINT(40.28556971633681 73.775848144873) bank137852 +137853 POINT(41.29452164901585 74.49193718505593) bank137853 +137854 POINT(41.37956144065608 74.0294873914134) bank137854 +137855 POINT(41.5441791231053 74.37149065788532) bank137855 +137856 POINT(40.6251185363776 74.78857014672538) bank137856 +137857 POINT(41.320137295898256 74.54081124623998) bank137857 +137858 POINT(40.47395414393904 73.03277378065927) bank137858 +137859 POINT(41.35621548968324 74.2056594062624) bank137859 +137860 POINT(40.29223766208686 73.78015116142498) bank137860 +137861 POINT(41.325475641971494 73.49164757577597) bank137861 +137862 POINT(41.289480863437205 73.18085431069363) bank137862 +137863 POINT(40.78741286020713 74.31653096921897) bank137863 +137864 POINT(40.18374351788612 74.1727463704752) bank137864 +137865 POINT(41.161977242487104 73.90101413136864) bank137865 +137866 POINT(40.697884418689135 74.63346664306115) bank137866 +137867 POINT(41.07094854345191 73.19624620551548) bank137867 +137868 POINT(41.47518300132886 73.12822895988974) bank137868 +137869 POINT(40.934823407155925 73.47426223396995) bank137869 +137870 POINT(41.110904335819114 73.92355626987175) bank137870 +137871 POINT(41.53875467597098 73.22421383119469) bank137871 +137872 POINT(40.678377966092484 73.41384645447945) bank137872 +137873 POINT(40.263646041289604 74.16923717547397) bank137873 +137874 POINT(41.29458726703101 73.91087139153794) bank137874 +137875 POINT(41.03839506054999 73.83268252707666) bank137875 +137876 POINT(40.49931014938293 74.3515866987479) bank137876 +137877 POINT(41.48891330366035 73.73526103439657) bank137877 +137878 POINT(40.06561262294897 73.9715158918321) bank137878 +137879 POINT(40.757687416578484 73.2836212855398) bank137879 +137880 POINT(41.268807515191654 74.13183737152764) bank137880 +137881 POINT(40.8032044909481 73.80928739535976) bank137881 +137882 POINT(40.67918184479074 73.6253146673207) bank137882 +137883 POINT(40.15329625739381 74.41771519117849) bank137883 +137884 POINT(40.375247803978 73.49595770806805) bank137884 +137885 POINT(41.36102977092651 74.02602956458166) bank137885 +137886 POINT(41.099667121343174 73.358389157081) bank137886 +137887 POINT(40.43223213292877 73.56556047319154) bank137887 +137888 POINT(41.155499985268555 74.29478862300991) bank137888 +137889 POINT(40.409091172705 73.46349429905456) bank137889 +137890 POINT(40.6894023261042 74.16779397857728) bank137890 +137891 POINT(40.238414948688806 73.36797722134106) bank137891 +137892 POINT(41.12201097910412 73.61098742823395) bank137892 +137893 POINT(40.39103246959071 74.03355448899778) bank137893 +137894 POINT(41.64363235699217 73.98231891866772) bank137894 +137895 POINT(40.59300428158673 74.87651586772203) bank137895 +137896 POINT(40.101392749127456 73.55573059312758) bank137896 +137897 POINT(41.5072390821498 74.75252115302682) bank137897 +137898 POINT(41.55131165511481 74.56128669485098) bank137898 +137899 POINT(40.987830848402616 73.52634581509783) bank137899 +137900 POINT(40.884553085799276 74.97239026897125) bank137900 +137901 POINT(40.8893563457852 73.89338034692943) bank137901 +137902 POINT(40.969692874387746 74.3542998044828) bank137902 +137903 POINT(40.472335254464845 73.82323268961646) bank137903 +137904 POINT(41.09281740678306 74.92364157548394) bank137904 +137905 POINT(40.49203013924142 74.29432789647379) bank137905 +137906 POINT(40.70055713599287 73.30704213918855) bank137906 +137907 POINT(39.77898082616055 74.65567494167382) bank137907 +137908 POINT(41.45682053863784 74.02834831902739) bank137908 +137909 POINT(40.44854119285106 73.23871509911632) bank137909 +137910 POINT(41.07474309319286 74.95685771307728) bank137910 +137911 POINT(41.514875205908176 74.83855695627521) bank137911 +137912 POINT(40.4777168246231 73.22028998934083) bank137912 +137913 POINT(40.76769340247687 74.07521531566671) bank137913 +137914 POINT(41.136581744246854 74.83523173963279) bank137914 +137915 POINT(41.31979833621523 73.5508631238601) bank137915 +137916 POINT(39.89674694506962 74.55132156432818) bank137916 +137917 POINT(39.947020697879395 74.55838631623415) bank137917 +137918 POINT(39.770838796839264 73.61889062470266) bank137918 +137919 POINT(40.78281979073855 73.0212815047387) bank137919 +137920 POINT(40.92489888652915 74.81126887417042) bank137920 +137921 POINT(40.134187920892174 74.54656166790882) bank137921 +137922 POINT(41.67238195092551 73.36792825288589) bank137922 +137923 POINT(40.082612616353714 74.92079602329186) bank137923 +137924 POINT(39.93217799024592 73.9453287241357) bank137924 +137925 POINT(39.81891677245226 74.87993249177731) bank137925 +137926 POINT(41.399434686407204 74.8225512532874) bank137926 +137927 POINT(40.15418618123676 73.17729654823788) bank137927 +137928 POINT(41.198956556746516 73.67405244789542) bank137928 +137929 POINT(39.770958799226115 74.27190792217726) bank137929 +137930 POINT(40.881715709225254 74.81081089801853) bank137930 +137931 POINT(40.60642718579367 74.73061190540112) bank137931 +137932 POINT(39.82931942497994 73.75651123581089) bank137932 +137933 POINT(40.534592053133686 73.07350572451438) bank137933 +137934 POINT(40.1152811176442 73.55747828930205) bank137934 +137935 POINT(40.942438058291984 74.68452429298908) bank137935 +137936 POINT(39.86785903292025 74.97063907422324) bank137936 +137937 POINT(39.839267691501576 74.4091371577213) bank137937 +137938 POINT(39.791561709966 73.54357951014714) bank137938 +137939 POINT(40.87381260544156 74.78223154777325) bank137939 +137940 POINT(40.08932214711231 74.33246453103644) bank137940 +137941 POINT(41.33014765564317 73.23516964801244) bank137941 +137942 POINT(40.81322433780259 74.77796303143747) bank137942 +137943 POINT(40.85721991360939 74.95116093869949) bank137943 +137944 POINT(41.47577563341115 73.54028266921638) bank137944 +137945 POINT(40.870283802194265 73.84465426207493) bank137945 +137946 POINT(40.81824516616551 73.51574699105797) bank137946 +137947 POINT(40.21717191603762 73.2159738597407) bank137947 +137948 POINT(40.5674182905424 73.93865378706268) bank137948 +137949 POINT(41.073994557630776 73.04337317344964) bank137949 +137950 POINT(40.58021677096381 74.43538279465605) bank137950 +137951 POINT(41.45091093098902 74.2721986558567) bank137951 +137952 POINT(40.23601500917214 73.84278205121242) bank137952 +137953 POINT(41.21178655366802 73.83731957515568) bank137953 +137954 POINT(41.48451532050646 74.71972680895875) bank137954 +137955 POINT(40.81028576463957 73.24944644279908) bank137955 +137956 POINT(40.47016721226741 73.56738420368845) bank137956 +137957 POINT(41.68653412669821 74.42430980340684) bank137957 +137958 POINT(41.34503150397819 73.06316185076068) bank137958 +137959 POINT(40.659486332660016 73.98258116333268) bank137959 +137960 POINT(39.746124558801505 74.88852396078786) bank137960 +137961 POINT(40.299702913717326 74.4981399063562) bank137961 +137962 POINT(40.53481904783469 73.95670039757712) bank137962 +137963 POINT(41.53074939749785 74.87279878339875) bank137963 +137964 POINT(39.81884337440117 73.53801757287955) bank137964 +137965 POINT(40.554916877272554 73.63979716509161) bank137965 +137966 POINT(39.84694020224742 74.44178293673363) bank137966 +137967 POINT(39.903954579408975 74.97330935835593) bank137967 +137968 POINT(40.55350476222059 74.00312682054518) bank137968 +137969 POINT(40.00637863582207 73.59753069265396) bank137969 +137970 POINT(41.320647963807396 74.99887384752738) bank137970 +137971 POINT(41.5703040833399 73.82238044950611) bank137971 +137972 POINT(40.47637513778198 74.43283032876037) bank137972 +137973 POINT(41.324162510262106 73.20828129396564) bank137973 +137974 POINT(40.005453470920735 73.90017657820916) bank137974 +137975 POINT(40.23701942202916 74.21171585490823) bank137975 +137976 POINT(39.85514964953051 74.32884754239029) bank137976 +137977 POINT(41.465771698332325 73.63014750366291) bank137977 +137978 POINT(40.355022959792876 73.14724032779753) bank137978 +137979 POINT(40.40910677907221 73.40283932936418) bank137979 +137980 POINT(40.70836206057942 74.38787836436474) bank137980 +137981 POINT(41.659725752212836 74.13733760832675) bank137981 +137982 POINT(41.57407002531558 73.8270012188531) bank137982 +137983 POINT(39.78982706574701 74.59699652810284) bank137983 +137984 POINT(41.01462056860463 73.45672267322513) bank137984 +137985 POINT(40.137462932770426 74.67615982539918) bank137985 +137986 POINT(41.010194594122545 73.96741284619569) bank137986 +137987 POINT(41.01766525711098 73.5927370876969) bank137987 +137988 POINT(41.608808469029896 74.02600986222129) bank137988 +137989 POINT(41.07472316903981 74.83249528329749) bank137989 +137990 POINT(40.42510364481599 73.36863808276921) bank137990 +137991 POINT(41.30758213716034 73.06388491392518) bank137991 +137992 POINT(40.6746273759314 73.50276137732529) bank137992 +137993 POINT(39.854034641946235 74.14974140369081) bank137993 +137994 POINT(41.64081410234633 74.15393846300536) bank137994 +137995 POINT(40.85591249922035 73.39531677661074) bank137995 +137996 POINT(41.701550239225966 73.84552009276186) bank137996 +137997 POINT(41.34094319826737 74.62522197859967) bank137997 +137998 POINT(40.40155304148527 73.56141418041939) bank137998 +137999 POINT(40.09407286095454 73.29555216786042) bank137999 +138000 POINT(40.52332573481325 73.35471769030649) bank138000 +138001 POINT(39.80572566930894 73.13506809745651) bank138001 +138002 POINT(40.072112393981456 73.42620116617131) bank138002 +138003 POINT(41.65803355229303 73.30131373335304) bank138003 +138004 POINT(41.137993443830425 74.17187222034543) bank138004 +138005 POINT(40.43138467787151 73.84048051022246) bank138005 +138006 POINT(41.31754847657005 74.74236549637028) bank138006 +138007 POINT(39.90308462065492 74.70684368153874) bank138007 +138008 POINT(40.04401793466784 74.78723797329054) bank138008 +138009 POINT(40.89072746527206 74.07314231228312) bank138009 +138010 POINT(41.70339507035477 73.44340446697316) bank138010 +138011 POINT(39.748169083353645 74.2373054574394) bank138011 +138012 POINT(40.54495641471732 73.31886228286571) bank138012 +138013 POINT(41.39503355586454 74.69134148824814) bank138013 +138014 POINT(41.03503292733775 74.46408223801726) bank138014 +138015 POINT(40.06805268857832 73.82577828806473) bank138015 +138016 POINT(40.656207250292745 73.57368215415056) bank138016 +138017 POINT(40.34334787301766 74.07070719001497) bank138017 +138018 POINT(40.482092068718366 74.47894470601504) bank138018 +138019 POINT(40.31508915008991 74.6205122987814) bank138019 +138020 POINT(39.91603395662502 74.42885414701742) bank138020 +138021 POINT(40.71468158546985 74.66174279002026) bank138021 +138022 POINT(40.714896641562206 73.09924083240251) bank138022 +138023 POINT(40.523615898145785 74.48926542608852) bank138023 +138024 POINT(40.29812608549421 73.70885575290691) bank138024 +138025 POINT(40.09730125411058 73.44017216703662) bank138025 +138026 POINT(41.60857033482274 73.64169219621209) bank138026 +138027 POINT(41.143666063796324 73.01223337816339) bank138027 +138028 POINT(41.2566545530716 74.30164205202836) bank138028 +138029 POINT(40.38713987845101 73.04404149411816) bank138029 +138030 POINT(40.98086278020868 73.40718106634561) bank138030 +138031 POINT(40.06503657050089 74.19557775325353) bank138031 +138032 POINT(39.887016128510176 74.15133077745055) bank138032 +138033 POINT(41.052832368813235 74.96944615725633) bank138033 +138034 POINT(41.21152310649179 73.85931958555244) bank138034 +138035 POINT(41.65555337618858 73.52699356778517) bank138035 +138036 POINT(41.30230621101169 74.84907307785957) bank138036 +138037 POINT(40.25504275394486 74.8515234796138) bank138037 +138038 POINT(40.310973034543544 73.80235277928126) bank138038 +138039 POINT(41.249134808136645 74.82972248460935) bank138039 +138040 POINT(41.43007723096717 73.41243227680353) bank138040 +138041 POINT(41.49605119939325 73.90106327124033) bank138041 +138042 POINT(41.47309808855582 73.59159855421852) bank138042 +138043 POINT(41.36728832499368 73.49985479092592) bank138043 +138044 POINT(39.851063966045736 73.85317963413544) bank138044 +138045 POINT(41.6669286705462 73.36673879423464) bank138045 +138046 POINT(39.94366849897795 73.9860749660417) bank138046 +138047 POINT(39.96555581089696 73.09589677795466) bank138047 +138048 POINT(40.3309720480857 74.60351996135098) bank138048 +138049 POINT(39.78367952576084 73.41648301201913) bank138049 +138050 POINT(40.91808550376156 73.5429470065267) bank138050 +138051 POINT(41.19308066310156 73.98626538422415) bank138051 +138052 POINT(41.1559473869426 73.49361168638107) bank138052 +138053 POINT(41.05130238669145 73.67961050497988) bank138053 +138054 POINT(39.75292303286002 73.57244119080346) bank138054 +138055 POINT(40.56946723388318 74.06241567528971) bank138055 +138056 POINT(40.8819490987757 73.16720678984707) bank138056 +138057 POINT(40.25421086100836 73.78979402834223) bank138057 +138058 POINT(41.22639088676362 74.92470153349132) bank138058 +138059 POINT(40.934945760939684 73.15412753465733) bank138059 +138060 POINT(41.44201300572728 73.72371555319508) bank138060 +138061 POINT(41.22599576191794 74.78355176670718) bank138061 +138062 POINT(40.5465274823786 73.48404371263061) bank138062 +138063 POINT(41.575090636652135 74.04068430294187) bank138063 +138064 POINT(39.98188212640584 73.66286686468702) bank138064 +138065 POINT(40.641941460400446 73.64942490513243) bank138065 +138066 POINT(40.435990550844075 73.26816855790659) bank138066 +138067 POINT(41.460473119007275 73.94967326146217) bank138067 +138068 POINT(40.00816936994406 73.21006959891746) bank138068 +138069 POINT(40.29635253138481 73.216898085664) bank138069 +138070 POINT(39.968689223366155 74.7377538328175) bank138070 +138071 POINT(39.850282500415815 74.14653291452606) bank138071 +138072 POINT(39.799464440251 74.46093728086655) bank138072 +138073 POINT(40.72237207599323 74.78063678671701) bank138073 +138074 POINT(40.25649515783147 74.07620271286979) bank138074 +138075 POINT(40.61338740584578 74.1690719769307) bank138075 +138076 POINT(40.58284810841956 73.74185773618464) bank138076 +138077 POINT(40.37002462449351 73.48184365054912) bank138077 +138078 POINT(40.217628228504005 74.52012458179729) bank138078 +138079 POINT(41.13141968555384 75.00204855027035) bank138079 +138080 POINT(41.41435608416812 73.81508090015804) bank138080 +138081 POINT(40.11552142623045 74.5079231539676) bank138081 +138082 POINT(39.84426758188639 73.09609545095189) bank138082 +138083 POINT(40.54810084157313 73.80710688989087) bank138083 +138084 POINT(40.531981748417415 73.06385269344956) bank138084 +138085 POINT(41.61242652796849 74.58401007922976) bank138085 +138086 POINT(40.94849751380946 73.05694096067683) bank138086 +138087 POINT(41.090305175778646 74.42319492979946) bank138087 +138088 POINT(41.51967327407958 74.28832875693406) bank138088 +138089 POINT(40.877403374419195 74.54833554234528) bank138089 +138090 POINT(40.95586017333616 74.15241908334572) bank138090 +138091 POINT(40.61773501011742 73.51502194849118) bank138091 +138092 POINT(40.16347811336453 74.7383310447995) bank138092 +138093 POINT(40.0016577214527 74.56259840478097) bank138093 +138094 POINT(40.5558524374118 73.25518068001827) bank138094 +138095 POINT(41.395344147004145 73.39022012192217) bank138095 +138096 POINT(41.0661903590302 74.9250420075167) bank138096 +138097 POINT(41.47587129627008 74.83383163556321) bank138097 +138098 POINT(40.234831943148 73.44674349541064) bank138098 +138099 POINT(40.00133313297225 74.2293366499694) bank138099 +138100 POINT(40.63400804994165 73.95604968082084) bank138100 +138101 POINT(41.63473883563312 73.87147391476687) bank138101 +138102 POINT(40.85725222678787 74.87489269370943) bank138102 +138103 POINT(39.89815833211782 74.53972993890788) bank138103 +138104 POINT(39.80744753102335 73.89417795852486) bank138104 +138105 POINT(40.361959042821944 73.82110221332626) bank138105 +138106 POINT(40.46218713019862 73.50296423218464) bank138106 +138107 POINT(40.234529012900964 73.2329322518248) bank138107 +138108 POINT(40.8901760100688 74.99034489237822) bank138108 +138109 POINT(40.88246780621753 73.41436144653154) bank138109 +138110 POINT(40.35508759438051 73.23580975955835) bank138110 +138111 POINT(41.272263116459 74.17786594373135) bank138111 +138112 POINT(40.591434349760505 74.53372686012645) bank138112 +138113 POINT(40.25777506506084 74.35093720691997) bank138113 +138114 POINT(40.9429810344362 74.43019754885918) bank138114 +138115 POINT(39.7990778565561 74.64397069553993) bank138115 +138116 POINT(40.23217726511736 73.20785079609563) bank138116 +138117 POINT(40.660553241836475 74.88605860660358) bank138117 +138118 POINT(41.40255441448069 74.89446810960712) bank138118 +138119 POINT(39.948977933501055 73.26639958543868) bank138119 +138120 POINT(39.89183158931978 73.90326476323234) bank138120 +138121 POINT(40.10766858038809 73.16806574840078) bank138121 +138122 POINT(40.91416222058072 73.91987148639076) bank138122 +138123 POINT(41.16230595372473 74.57149662460205) bank138123 +138124 POINT(41.43415051946949 73.2909207448858) bank138124 +138125 POINT(40.66354781904896 74.33219991591206) bank138125 +138126 POINT(40.928636040919 73.37332742378997) bank138126 +138127 POINT(40.115898564419716 73.75178027162434) bank138127 +138128 POINT(40.67098557789669 74.11614962941567) bank138128 +138129 POINT(41.17749232006946 73.53385639486464) bank138129 +138130 POINT(40.78952165101563 73.32412290320767) bank138130 +138131 POINT(40.3405608704812 74.11234963182969) bank138131 +138132 POINT(41.03972300187559 73.0693109631149) bank138132 +138133 POINT(41.46231366302346 73.99612183140714) bank138133 +138134 POINT(40.91426566780659 73.65635612955501) bank138134 +138135 POINT(40.51667314215832 73.54937934203981) bank138135 +138136 POINT(40.014217855892284 74.53889743638199) bank138136 +138137 POINT(41.378092765356236 73.36454216927648) bank138137 +138138 POINT(40.3935229528936 74.86693659250984) bank138138 +138139 POINT(39.828058475229945 74.61660527037523) bank138139 +138140 POINT(40.31839606388954 74.75870609349333) bank138140 +138141 POINT(41.59625341336227 74.20287602724117) bank138141 +138142 POINT(39.97748410704877 73.78657175179914) bank138142 +138143 POINT(40.74739388606611 74.25024468651806) bank138143 +138144 POINT(41.24409457663986 73.90698606373934) bank138144 +138145 POINT(40.45099677691018 73.74222962399504) bank138145 +138146 POINT(40.450030356605154 74.86264124818527) bank138146 +138147 POINT(41.54520103987516 74.03950751889668) bank138147 +138148 POINT(41.47183883865503 73.51963628403824) bank138148 +138149 POINT(40.78398499775997 74.91101624994835) bank138149 +138150 POINT(41.46784623442132 73.25418816030117) bank138150 +138151 POINT(40.22495302699148 73.03245542505672) bank138151 +138152 POINT(40.75366802074751 74.53934770139101) bank138152 +138153 POINT(39.80065864018291 73.26316223731439) bank138153 +138154 POINT(39.94942756544118 73.24989825190205) bank138154 +138155 POINT(40.56107171938702 73.57432116216162) bank138155 +138156 POINT(41.038225453162866 73.64521842331385) bank138156 +138157 POINT(39.96800082288039 74.94026641594367) bank138157 +138158 POINT(40.542662931396805 73.77533256690486) bank138158 +138159 POINT(39.73727977361781 74.99927087689659) bank138159 +138160 POINT(39.920663418488445 74.9940910486812) bank138160 +138161 POINT(40.841320538176035 73.13755327193172) bank138161 +138162 POINT(41.18624830542626 74.65671492769015) bank138162 +138163 POINT(40.949762031282006 73.46800152734558) bank138163 +138164 POINT(39.952010195076646 73.91446602809816) bank138164 +138165 POINT(40.22794014760596 74.82629811814658) bank138165 +138166 POINT(41.29212401097019 74.31652332788182) bank138166 +138167 POINT(41.47663592114429 73.15133912564076) bank138167 +138168 POINT(41.41968078495894 74.98088828566709) bank138168 +138169 POINT(40.497573828993836 73.52710564732092) bank138169 +138170 POINT(39.912651088245404 74.18375123387608) bank138170 +138171 POINT(40.118733348757104 73.11676131190421) bank138171 +138172 POINT(39.87224523343055 73.23578848436723) bank138172 +138173 POINT(40.95979580321287 74.48129300968372) bank138173 +138174 POINT(40.74602037065154 73.7366641852504) bank138174 +138175 POINT(40.137459343769244 73.7143497680936) bank138175 +138176 POINT(40.3381570869064 73.45853894607386) bank138176 +138177 POINT(39.738658638342606 74.07221986441132) bank138177 +138178 POINT(40.485134360897774 73.802667437874) bank138178 +138179 POINT(41.23513853317695 73.78693446946745) bank138179 +138180 POINT(40.51753273088073 73.88295925173703) bank138180 +138181 POINT(41.004049184271814 74.41888152921533) bank138181 +138182 POINT(41.450297923771295 73.85804910663595) bank138182 +138183 POINT(40.6396027281442 74.3440978834171) bank138183 +138184 POINT(40.15794460998002 74.34706405854415) bank138184 +138185 POINT(40.97147761487184 73.83869849717215) bank138185 +138186 POINT(41.576204510485915 73.39289014841947) bank138186 +138187 POINT(40.10198638728734 73.68275925284865) bank138187 +138188 POINT(40.96524795612997 73.78529013785311) bank138188 +138189 POINT(41.343377940221316 74.89754543691514) bank138189 +138190 POINT(39.72140032334217 73.61663541405784) bank138190 +138191 POINT(41.11071907179752 74.6948673595799) bank138191 +138192 POINT(39.77274237686082 73.17475863902278) bank138192 +138193 POINT(40.081025082779455 73.77972502024951) bank138193 +138194 POINT(40.568849413466985 74.13015439526426) bank138194 +138195 POINT(40.70886006026154 73.50438741302786) bank138195 +138196 POINT(41.623046391741234 73.53396926077465) bank138196 +138197 POINT(40.65608881865846 74.10764418672623) bank138197 +138198 POINT(39.88918925171528 73.1026356603627) bank138198 +138199 POINT(40.53199670562681 73.68570665857642) bank138199 +138200 POINT(41.030590224756516 73.99260247649713) bank138200 +138201 POINT(41.481129998979505 73.6514529713463) bank138201 +138202 POINT(41.36059637453833 73.31750287922225) bank138202 +138203 POINT(40.790104268376005 73.88553149933662) bank138203 +138204 POINT(39.97212960517564 74.76182944964876) bank138204 +138205 POINT(40.13299137479963 74.59284817228676) bank138205 +138206 POINT(40.2090453620585 74.16348927139128) bank138206 +138207 POINT(40.10778614976114 73.37095142282976) bank138207 +138208 POINT(41.07129481309561 74.32413355806032) bank138208 +138209 POINT(40.75715548906997 73.83053310095933) bank138209 +138210 POINT(40.332466407812085 74.26176880768865) bank138210 +138211 POINT(40.26557947311722 74.52324422159865) bank138211 +138212 POINT(40.486297219154224 74.32501107727826) bank138212 +138213 POINT(41.45113257495886 74.4392852848654) bank138213 +138214 POINT(40.98802526013791 73.59394942499219) bank138214 +138215 POINT(40.58364202834559 74.21103929919185) bank138215 +138216 POINT(40.16496582598402 74.76224934061909) bank138216 +138217 POINT(39.81904712208357 73.06067264968053) bank138217 +138218 POINT(39.74459041992649 73.03902013681257) bank138218 +138219 POINT(41.20293745488286 74.01152659237302) bank138219 +138220 POINT(41.43309862289547 74.77947053459778) bank138220 +138221 POINT(40.20909795888247 73.56272346123028) bank138221 +138222 POINT(39.78508138915216 73.46183691913905) bank138222 +138223 POINT(41.14629543259448 74.85552412171585) bank138223 +138224 POINT(39.90544579330927 73.51086563784565) bank138224 +138225 POINT(40.06448655860749 74.14500552369817) bank138225 +138226 POINT(41.59147037500523 74.04511208727912) bank138226 +138227 POINT(40.84355711784261 73.93682472664158) bank138227 +138228 POINT(40.737469397721476 74.58817454070163) bank138228 +138229 POINT(41.42259958221464 74.42467641275653) bank138229 +138230 POINT(39.81821345389424 73.82068862278885) bank138230 +138231 POINT(41.13505618592847 74.60577772145209) bank138231 +138232 POINT(41.47476545322863 74.45041215634613) bank138232 +138233 POINT(40.504413008342844 74.98261187633722) bank138233 +138234 POINT(41.22490065698975 75.00521866492467) bank138234 +138235 POINT(41.30123805476049 73.25059639744109) bank138235 +138236 POINT(39.98037834676028 74.73695370827113) bank138236 +138237 POINT(40.92544643525965 74.83214972296061) bank138237 +138238 POINT(39.972653572315956 74.76741114986801) bank138238 +138239 POINT(41.185728748505795 74.97724072763036) bank138239 +138240 POINT(41.631703213038634 73.29089896640008) bank138240 +138241 POINT(41.62450854013638 74.46433563014239) bank138241 +138242 POINT(40.221193300188716 73.69482670716378) bank138242 +138243 POINT(41.64220494802261 73.86430525406688) bank138243 +138244 POINT(40.802919059002846 74.20210668565652) bank138244 +138245 POINT(40.33786820539762 73.60699096561564) bank138245 +138246 POINT(39.899426989339595 73.11420092566091) bank138246 +138247 POINT(41.52460010216395 73.87037903019038) bank138247 +138248 POINT(39.75784727435092 73.45618788362106) bank138248 +138249 POINT(40.71449082352513 74.38837322770797) bank138249 +138250 POINT(41.67903191125762 73.66240050977412) bank138250 +138251 POINT(39.9825096819215 73.24744862500728) bank138251 +138252 POINT(39.92798605649897 73.87934798173075) bank138252 +138253 POINT(40.36678936574614 73.37652443517885) bank138253 +138254 POINT(40.85836136890192 74.93137746441765) bank138254 +138255 POINT(40.36975559888181 74.5900762180268) bank138255 +138256 POINT(39.810059829111864 73.10940534881782) bank138256 +138257 POINT(40.44966557773317 74.0579463437007) bank138257 +138258 POINT(41.188857887881255 74.502881157604) bank138258 +138259 POINT(40.360377476395556 74.2125315757896) bank138259 +138260 POINT(39.79464293666376 73.10341555958303) bank138260 +138261 POINT(40.82314450255862 74.52551140111837) bank138261 +138262 POINT(39.893847500127336 73.28644572277983) bank138262 +138263 POINT(41.18015152963525 73.14310800456758) bank138263 +138264 POINT(40.53685917271101 74.59012791639455) bank138264 +138265 POINT(40.39702147322175 74.8262976250176) bank138265 +138266 POINT(41.11953591539013 74.12985115917192) bank138266 +138267 POINT(40.2767777879381 73.09169651393702) bank138267 +138268 POINT(40.85949436893085 74.87059470079225) bank138268 +138269 POINT(40.140196672321544 74.11645805475544) bank138269 +138270 POINT(41.15752830527474 73.90050471571023) bank138270 +138271 POINT(41.20851789568989 73.33969818113322) bank138271 +138272 POINT(39.77975018148821 74.87467248342023) bank138272 +138273 POINT(40.22566103674306 74.61542230717068) bank138273 +138274 POINT(40.622523277684216 74.99487540424279) bank138274 +138275 POINT(41.49241103408195 73.19176806280014) bank138275 +138276 POINT(39.94010591436716 74.37131418769988) bank138276 +138277 POINT(41.35994634860667 74.65067273293334) bank138277 +138278 POINT(40.60508627171102 74.41711972833922) bank138278 +138279 POINT(41.505325686080056 74.48833262060309) bank138279 +138280 POINT(40.216727705096744 73.8007285316515) bank138280 +138281 POINT(39.84539034196822 74.63137768167323) bank138281 +138282 POINT(40.47577910870593 73.46864893143133) bank138282 +138283 POINT(41.0788653269878 74.34044170343141) bank138283 +138284 POINT(40.6410499635805 73.87240468875738) bank138284 +138285 POINT(40.29121484356351 74.476936705083) bank138285 +138286 POINT(41.130168624347554 74.2187344187585) bank138286 +138287 POINT(41.55047961640271 73.11590276839244) bank138287 +138288 POINT(41.31627239411519 74.4567864022625) bank138288 +138289 POINT(40.697242098193456 73.23244041993148) bank138289 +138290 POINT(40.043145899650185 74.58200010680503) bank138290 +138291 POINT(40.48775520712989 73.05105562741902) bank138291 +138292 POINT(40.03087441670796 73.48406415686807) bank138292 +138293 POINT(40.04011127657378 73.1720070989702) bank138293 +138294 POINT(39.86508641993149 74.96361566864941) bank138294 +138295 POINT(40.101460373360325 74.50182725403634) bank138295 +138296 POINT(39.77991199245205 74.5527149018527) bank138296 +138297 POINT(41.19139940525691 74.23172447224793) bank138297 +138298 POINT(41.65690918462479 73.91364092792197) bank138298 +138299 POINT(40.19015016335209 74.59259365226364) bank138299 +138300 POINT(40.121407125739346 74.6359801414483) bank138300 +138301 POINT(41.389071545189665 74.66476127873902) bank138301 +138302 POINT(41.25204050607127 73.69672080422818) bank138302 +138303 POINT(41.09342372042933 73.06214589975146) bank138303 +138304 POINT(39.96895633992174 73.42915865902815) bank138304 +138305 POINT(40.40914675361752 73.68630200846667) bank138305 +138306 POINT(41.11204316079212 73.36568032488391) bank138306 +138307 POINT(40.14522551393467 74.72020109710779) bank138307 +138308 POINT(40.92175512154856 74.72629697772295) bank138308 +138309 POINT(39.78915041351455 74.82390838630192) bank138309 +138310 POINT(40.25753234308032 73.12629584010004) bank138310 +138311 POINT(39.83657334083891 73.6780855523313) bank138311 +138312 POINT(41.56074820428873 74.00702906613652) bank138312 +138313 POINT(41.47201762599245 73.41826229404282) bank138313 +138314 POINT(41.52913811104709 74.4957576526459) bank138314 +138315 POINT(41.033759807145465 74.17096343862347) bank138315 +138316 POINT(40.201409605832616 73.18720672893399) bank138316 +138317 POINT(39.97746806588602 74.40440009664415) bank138317 +138318 POINT(41.2863550532429 74.3979894235378) bank138318 +138319 POINT(40.48761539875098 74.76386835410995) bank138319 +138320 POINT(39.96021483567492 74.12960547172486) bank138320 +138321 POINT(40.558202891914966 74.28965660426266) bank138321 +138322 POINT(40.921056109982075 73.49700356228846) bank138322 +138323 POINT(39.966410243328106 74.41684369516994) bank138323 +138324 POINT(40.58151503354482 74.73927092129107) bank138324 +138325 POINT(40.52861050570557 73.2842380349239) bank138325 +138326 POINT(40.671818856274 74.45673734804024) bank138326 +138327 POINT(39.94482695335414 74.83084873399339) bank138327 +138328 POINT(41.60593458116396 73.34352280948977) bank138328 +138329 POINT(41.24741176128764 73.96164452694603) bank138329 +138330 POINT(40.11862245175749 74.76294430775482) bank138330 +138331 POINT(40.343180234026505 74.20254766332926) bank138331 +138332 POINT(40.28500799822088 73.63532031224558) bank138332 +138333 POINT(40.851092739215844 73.47740910420907) bank138333 +138334 POINT(41.664895878213756 74.2148683598718) bank138334 +138335 POINT(39.85687365924825 73.38566879239274) bank138335 +138336 POINT(41.24225606361716 74.5250052025574) bank138336 +138337 POINT(40.71957099085145 73.86335130895071) bank138337 +138338 POINT(40.55853025765765 74.01001482234467) bank138338 +138339 POINT(40.90410396981056 73.14365402816549) bank138339 +138340 POINT(39.74269405240353 74.3783684946611) bank138340 +138341 POINT(40.444739473041885 73.72511131618565) bank138341 +138342 POINT(40.235703421415586 74.88153277430654) bank138342 +138343 POINT(40.553934451202906 73.9737609423641) bank138343 +138344 POINT(40.72422500854663 73.71644396820486) bank138344 +138345 POINT(41.625375927264685 73.49434872935976) bank138345 +138346 POINT(40.40416804745621 73.69707205977096) bank138346 +138347 POINT(41.0616062805589 74.36527401199419) bank138347 +138348 POINT(40.93995089443633 74.26796920720928) bank138348 +138349 POINT(40.98530879949997 73.8259762156183) bank138349 +138350 POINT(40.861619885541984 74.53925059338312) bank138350 +138351 POINT(41.4219809260098 73.09582188210065) bank138351 +138352 POINT(40.65282353041033 73.72937777926808) bank138352 +138353 POINT(41.30887988303691 74.3911950142137) bank138353 +138354 POINT(40.49175193212368 73.43357084214327) bank138354 +138355 POINT(41.50793789333865 74.20522961687533) bank138355 +138356 POINT(40.69903566919966 75.00013387321307) bank138356 +138357 POINT(41.04486816543778 74.70425132469597) bank138357 +138358 POINT(40.554769977405165 74.44963737024675) bank138358 +138359 POINT(41.3039647023514 74.61882480386365) bank138359 +138360 POINT(40.655745168894896 74.5020597042194) bank138360 +138361 POINT(40.406664491156874 74.61691808001508) bank138361 +138362 POINT(41.69700832370687 73.51899769647939) bank138362 +138363 POINT(39.80258115834728 73.76800947048861) bank138363 +138364 POINT(41.1983288200015 74.32743831291302) bank138364 +138365 POINT(40.46222492814568 73.76946778272529) bank138365 +138366 POINT(40.48578034002192 74.38235065747195) bank138366 +138367 POINT(40.8676234248057 73.36104151665073) bank138367 +138368 POINT(39.99956573447858 74.3181898344832) bank138368 +138369 POINT(40.7128232038904 73.27682276689481) bank138369 +138370 POINT(40.43945966922035 73.4538918066506) bank138370 +138371 POINT(41.657948258134404 73.54892809344366) bank138371 +138372 POINT(41.44967146975214 74.33670938683089) bank138372 +138373 POINT(39.98536556455287 73.42555469329301) bank138373 +138374 POINT(41.377597172535985 73.57535388743857) bank138374 +138375 POINT(39.98979653885424 73.47480453382121) bank138375 +138376 POINT(40.56032354919007 73.39826710928578) bank138376 +138377 POINT(41.652971198425405 74.81905698590094) bank138377 +138378 POINT(40.0602326684166 74.6933111032356) bank138378 +138379 POINT(40.25824664492686 73.98963393305496) bank138379 +138380 POINT(40.30140199841513 74.718553526931) bank138380 +138381 POINT(40.49617486850289 74.6862700302831) bank138381 +138382 POINT(40.18559926733145 74.65534066765832) bank138382 +138383 POINT(41.01935176369359 74.74077585092142) bank138383 +138384 POINT(40.16864504155457 74.35050673213738) bank138384 +138385 POINT(41.25627786213607 73.69185040139615) bank138385 +138386 POINT(39.98641527848297 73.88446324606859) bank138386 +138387 POINT(40.91332956804258 74.22354651801187) bank138387 +138388 POINT(41.02157959478557 73.40157146828135) bank138388 +138389 POINT(39.89995557017431 73.7994197224518) bank138389 +138390 POINT(40.37347674101105 73.69806501909966) bank138390 +138391 POINT(41.44066280190061 75.00307533057462) bank138391 +138392 POINT(39.94479493268712 73.63333628554122) bank138392 +138393 POINT(41.440935460623365 73.38334222886051) bank138393 +138394 POINT(40.79627674246985 74.1646078774027) bank138394 +138395 POINT(41.60712757418798 74.53332307617833) bank138395 +138396 POINT(40.774355601783604 73.9138887780534) bank138396 +138397 POINT(41.20328556229405 74.15029664558966) bank138397 +138398 POINT(41.421818725224924 73.4921718923287) bank138398 +138399 POINT(40.804173159421275 73.64158274027614) bank138399 +138400 POINT(40.649914275343356 74.53375379824163) bank138400 +138401 POINT(40.15678661059466 73.85255236225709) bank138401 +138402 POINT(40.85486417505007 73.08768390153156) bank138402 +138403 POINT(40.95760412311677 74.54418727180636) bank138403 +138404 POINT(41.26084565572678 74.29910908166771) bank138404 +138405 POINT(39.91262463510058 73.97493295230366) bank138405 +138406 POINT(40.08381690956891 74.12543313669887) bank138406 +138407 POINT(41.679590631358785 73.91629739485576) bank138407 +138408 POINT(41.031575325707685 74.25204739043895) bank138408 +138409 POINT(40.834329436730386 73.89383471056604) bank138409 +138410 POINT(40.76677838624779 73.47936362353477) bank138410 +138411 POINT(41.22438506173921 73.78430368682109) bank138411 +138412 POINT(41.27494538639549 73.57358135179332) bank138412 +138413 POINT(41.08402343879132 74.22524794523044) bank138413 +138414 POINT(41.25355669411268 74.8538910308977) bank138414 +138415 POINT(41.5774730374267 74.75473443062066) bank138415 +138416 POINT(40.78874470519196 74.16110188362036) bank138416 +138417 POINT(40.46848498181998 73.5993036282328) bank138417 +138418 POINT(41.68767060376554 74.28233059088849) bank138418 +138419 POINT(40.988010257900896 74.4604268047141) bank138419 +138420 POINT(40.5447173458567 73.85005008920417) bank138420 +138421 POINT(41.63203936648356 74.68179825656628) bank138421 +138422 POINT(40.93118286065076 73.71123431433) bank138422 +138423 POINT(40.56407856927269 73.4502091029406) bank138423 +138424 POINT(41.34843713974264 74.31001797286777) bank138424 +138425 POINT(41.38929488763766 73.03796293149543) bank138425 +138426 POINT(40.13351830869676 73.25395508072839) bank138426 +138427 POINT(41.094871318165005 73.84759465423076) bank138427 +138428 POINT(41.367476216438305 74.39662074477566) bank138428 +138429 POINT(39.92247275906379 74.47032016078929) bank138429 +138430 POINT(40.13418221218301 73.2836392044237) bank138430 +138431 POINT(40.460674339090474 73.79378205702771) bank138431 +138432 POINT(41.60926368910928 73.6707031585323) bank138432 +138433 POINT(41.10757212849751 74.42958880046729) bank138433 +138434 POINT(41.59525819677349 73.23072975606888) bank138434 +138435 POINT(41.185335934442136 73.5447958819374) bank138435 +138436 POINT(40.90029547629505 73.24488671663165) bank138436 +138437 POINT(40.660777414481785 73.06557541436577) bank138437 +138438 POINT(41.18847382108904 73.32897630003822) bank138438 +138439 POINT(40.52662678594559 74.8136671887544) bank138439 +138440 POINT(40.72176055678295 73.92956233693067) bank138440 +138441 POINT(41.63894919046805 74.11479933089393) bank138441 +138442 POINT(41.14106131126712 73.98611985822944) bank138442 +138443 POINT(40.20471698871893 74.75076372900773) bank138443 +138444 POINT(40.411675962062354 74.01868503475143) bank138444 +138445 POINT(41.0483460342866 73.59250641719855) bank138445 +138446 POINT(40.64055479403359 74.29716999877571) bank138446 +138447 POINT(41.367495045223585 74.64530309888215) bank138447 +138448 POINT(41.5908975397267 73.36384942323177) bank138448 +138449 POINT(41.256162139907026 73.01201723402697) bank138449 +138450 POINT(41.25731123889719 74.69318342602236) bank138450 +138451 POINT(40.40894263165695 73.47972994218134) bank138451 +138452 POINT(40.23868940202314 73.70050635089562) bank138452 +138453 POINT(40.85897318685795 73.70598200011693) bank138453 +138454 POINT(40.14950151315667 74.02365141026067) bank138454 +138455 POINT(39.94553480232361 73.80026318481399) bank138455 +138456 POINT(41.399509561271664 74.04957683658378) bank138456 +138457 POINT(41.45751384640261 74.64654586656437) bank138457 +138458 POINT(41.52453386679526 74.68250806084481) bank138458 +138459 POINT(41.18969585913332 74.63025436171517) bank138459 +138460 POINT(39.75316674775395 74.93828958171062) bank138460 +138461 POINT(39.92279784243822 73.19375892627266) bank138461 +138462 POINT(40.90394774158449 74.87194495657704) bank138462 +138463 POINT(40.27247315884377 73.20006124170799) bank138463 +138464 POINT(40.102709869049754 73.22188896115827) bank138464 +138465 POINT(40.69818131197162 73.03171164190839) bank138465 +138466 POINT(39.81695404057884 73.97627254091981) bank138466 +138467 POINT(41.552243389356114 74.28750001495678) bank138467 +138468 POINT(41.50357683284246 74.5087473743771) bank138468 +138469 POINT(40.90801643552781 74.07563869722716) bank138469 +138470 POINT(40.80199028959686 74.58261940325558) bank138470 +138471 POINT(39.84763807895924 73.99735244814615) bank138471 +138472 POINT(40.42852980273142 74.00115429743133) bank138472 +138473 POINT(40.56012298293852 74.791690133615) bank138473 +138474 POINT(40.43033402108757 74.05228301335106) bank138474 +138475 POINT(40.50336030775675 73.52308026600207) bank138475 +138476 POINT(40.4652518974022 74.07688678926301) bank138476 +138477 POINT(40.74649263692183 74.7704354261116) bank138477 +138478 POINT(41.12441849238016 73.76718219909452) bank138478 +138479 POINT(40.860136851928 74.3344882400002) bank138479 +138480 POINT(39.91811325061637 73.88690153443348) bank138480 +138481 POINT(40.73339387385069 74.46705603145263) bank138481 +138482 POINT(40.51914591826076 74.18458054199729) bank138482 +138483 POINT(40.81995420968686 73.91206779158928) bank138483 +138484 POINT(40.62440372757016 74.86492985087325) bank138484 +138485 POINT(41.07088089919946 73.7951696467657) bank138485 +138486 POINT(41.3897479632152 73.92782791766341) bank138486 +138487 POINT(41.27951316035954 73.54211330083292) bank138487 +138488 POINT(40.385914938396404 74.79654054710649) bank138488 +138489 POINT(40.930841238110105 73.93943588219238) bank138489 +138490 POINT(41.60574857090297 74.76604745707708) bank138490 +138491 POINT(41.18568376901148 73.72867661237306) bank138491 +138492 POINT(41.14295641186838 73.38241309210032) bank138492 +138493 POINT(40.54960537058951 73.00642313958235) bank138493 +138494 POINT(40.90361968549286 73.73188989651634) bank138494 +138495 POINT(40.17947766701132 74.67345989324959) bank138495 +138496 POINT(40.19228734437239 73.953163169926) bank138496 +138497 POINT(40.21516419274848 74.12144221317641) bank138497 +138498 POINT(41.275189623092956 73.7824189442514) bank138498 +138499 POINT(40.87217151628101 74.08039416278508) bank138499 +138500 POINT(41.53768967267432 73.99082275362568) bank138500 +138501 POINT(40.49296372932647 73.48214361826003) bank138501 +138502 POINT(41.55805850185537 73.26808182601582) bank138502 +138503 POINT(41.01947216441111 73.5826408060559) bank138503 +138504 POINT(41.585641139449216 74.55333674630259) bank138504 +138505 POINT(40.14584502644511 73.69146592341926) bank138505 +138506 POINT(39.957650957841075 74.14146725540674) bank138506 +138507 POINT(40.532213369148884 73.52663367723741) bank138507 +138508 POINT(41.54472825043318 73.33860609957074) bank138508 +138509 POINT(40.79405200366475 74.02950245389712) bank138509 +138510 POINT(41.63225372567553 74.7227934225324) bank138510 +138511 POINT(40.77010139083153 73.72862910426355) bank138511 +138512 POINT(40.34730659192735 74.13559272194601) bank138512 +138513 POINT(39.990908961550275 73.96933181745561) bank138513 +138514 POINT(40.3707443242303 74.10924785664496) bank138514 +138515 POINT(41.540999819154074 74.10963803732857) bank138515 +138516 POINT(41.64300270184237 73.45266070723257) bank138516 +138517 POINT(39.98475338756616 73.64721441058738) bank138517 +138518 POINT(39.77142526558384 73.76739539962637) bank138518 +138519 POINT(39.81508152574872 73.33347279235674) bank138519 +138520 POINT(40.91957657171623 73.15163707665413) bank138520 +138521 POINT(39.81078884461721 74.24443976591567) bank138521 +138522 POINT(39.918354697537396 73.50510581809587) bank138522 +138523 POINT(39.970857485206444 73.59526583487741) bank138523 +138524 POINT(41.47808517557939 73.81742655874527) bank138524 +138525 POINT(39.86712138110307 74.447400949679) bank138525 +138526 POINT(41.19819007920993 74.43117358158811) bank138526 +138527 POINT(40.70190079503192 74.26481689379834) bank138527 +138528 POINT(41.608651697617525 73.14149909924907) bank138528 +138529 POINT(40.479753228811596 74.50394639292567) bank138529 +138530 POINT(41.613043736350555 73.22548844841656) bank138530 +138531 POINT(41.236618861900844 74.53902384836019) bank138531 +138532 POINT(41.19476557173899 74.26992315766093) bank138532 +138533 POINT(41.1623416965406 74.89015550596797) bank138533 +138534 POINT(41.46181614972443 73.134576884126) bank138534 +138535 POINT(39.90168589007828 74.28076842705941) bank138535 +138536 POINT(41.505506967144505 73.76352929200323) bank138536 +138537 POINT(41.487551089077805 74.6012608067256) bank138537 +138538 POINT(41.447792626202435 74.92648858353255) bank138538 +138539 POINT(41.27080421685042 73.5212849584667) bank138539 +138540 POINT(39.97364274950736 74.52109287318632) bank138540 +138541 POINT(40.46187997565443 74.84477446287742) bank138541 +138542 POINT(41.4843978646507 73.65034282484145) bank138542 +138543 POINT(41.054007568097376 73.3971550296949) bank138543 +138544 POINT(41.47246101047019 74.05440860633074) bank138544 +138545 POINT(39.759010207530096 74.0457775012654) bank138545 +138546 POINT(40.53674992717699 74.4026232308741) bank138546 +138547 POINT(40.71431407549707 74.51684017810149) bank138547 +138548 POINT(40.69862824092821 73.2028100687986) bank138548 +138549 POINT(39.86325251576413 73.50182131700404) bank138549 +138550 POINT(41.4780749946809 74.74733426063942) bank138550 +138551 POINT(41.07337863404078 73.9687372152466) bank138551 +138552 POINT(41.14012350676883 74.54409789728534) bank138552 +138553 POINT(40.15061026559531 73.6545114310112) bank138553 +138554 POINT(40.59031887488789 74.27134205504642) bank138554 +138555 POINT(41.11617388264877 73.6955732748932) bank138555 +138556 POINT(39.91396441561792 73.16468935979444) bank138556 +138557 POINT(39.768556420870304 74.59633762537383) bank138557 +138558 POINT(39.80667714084117 73.96700832239571) bank138558 +138559 POINT(41.50369214526156 73.94716129098951) bank138559 +138560 POINT(40.045161267919624 73.3603514598256) bank138560 +138561 POINT(39.76283978182945 73.96129685506806) bank138561 +138562 POINT(40.55099238535318 74.93337075201745) bank138562 +138563 POINT(41.122101403932426 74.62632657036002) bank138563 +138564 POINT(40.47389294707662 73.95013991887477) bank138564 +138565 POINT(41.441013076420866 74.71536166513802) bank138565 +138566 POINT(39.871435933646474 73.47883925750725) bank138566 +138567 POINT(39.757845483505946 74.81603233706332) bank138567 +138568 POINT(41.49622157716113 74.07461204682826) bank138568 +138569 POINT(40.83366064364315 73.30160567220483) bank138569 +138570 POINT(41.43125405155645 73.3318915025316) bank138570 +138571 POINT(40.237968054311196 74.35807634600553) bank138571 +138572 POINT(41.18840453005621 73.81633855758763) bank138572 +138573 POINT(40.788311443883174 74.53965049957165) bank138573 +138574 POINT(40.54578733774984 73.1675212129289) bank138574 +138575 POINT(40.28213614007153 73.07374285458717) bank138575 +138576 POINT(39.74312749124567 73.1654908893148) bank138576 +138577 POINT(41.23612152678491 74.23484567261829) bank138577 +138578 POINT(40.36401942309104 74.77247483671393) bank138578 +138579 POINT(39.81215275297252 74.58592235593916) bank138579 +138580 POINT(39.8060440309732 74.35670242887032) bank138580 +138581 POINT(39.84092383497317 74.93960302864052) bank138581 +138582 POINT(41.05430801491674 74.72106495620214) bank138582 +138583 POINT(40.16464236384668 74.95243911652388) bank138583 +138584 POINT(41.101511235788614 73.11588067360796) bank138584 +138585 POINT(41.14219492458184 73.16581852501679) bank138585 +138586 POINT(40.09696237073936 74.39619748719208) bank138586 +138587 POINT(40.33858954898473 74.1722002132039) bank138587 +138588 POINT(40.182857042554325 74.44688890465132) bank138588 +138589 POINT(40.11129346142556 74.1188690543856) bank138589 +138590 POINT(40.319347466350294 73.1565325968827) bank138590 +138591 POINT(40.90282025548518 73.22836368581325) bank138591 +138592 POINT(41.63167541369746 73.31140894663979) bank138592 +138593 POINT(41.67382575164405 73.96301226245059) bank138593 +138594 POINT(39.77005630932906 74.20161853953614) bank138594 +138595 POINT(41.30639354145331 74.2390805634204) bank138595 +138596 POINT(40.73238859102848 74.81099681531542) bank138596 +138597 POINT(41.46782307497266 73.27906217262189) bank138597 +138598 POINT(39.8119891174952 74.74171787118168) bank138598 +138599 POINT(41.14473565502846 73.68964465988702) bank138599 +138600 POINT(40.39297353875732 74.81675560733434) bank138600 +138601 POINT(40.494703438238076 73.32646024103045) bank138601 +138602 POINT(40.63127339980033 74.18310049777732) bank138602 +138603 POINT(40.160181770167185 73.55928838756428) bank138603 +138604 POINT(41.324133526344944 73.31929475542567) bank138604 +138605 POINT(40.16792760070337 73.72271917274226) bank138605 +138606 POINT(40.8817216489823 74.31546103130522) bank138606 +138607 POINT(40.91420471235707 74.32253165555062) bank138607 +138608 POINT(40.97829180660177 73.38547829094134) bank138608 +138609 POINT(41.376211750883364 74.66789911807646) bank138609 +138610 POINT(41.264542265300555 73.70042146719308) bank138610 +138611 POINT(41.23484611596612 74.77571142487238) bank138611 +138612 POINT(40.511991668894886 73.19894609021156) bank138612 +138613 POINT(40.97909478178181 74.81944958272788) bank138613 +138614 POINT(41.350029220385 73.92794169556018) bank138614 +138615 POINT(40.636824695701165 74.9896688505279) bank138615 +138616 POINT(39.83423418259879 73.56758464215224) bank138616 +138617 POINT(40.085085239723774 74.080701532295) bank138617 +138618 POINT(40.698319803109335 73.69839645873698) bank138618 +138619 POINT(40.95022162771277 73.9201924704598) bank138619 +138620 POINT(41.03318231468609 73.51168303506658) bank138620 +138621 POINT(39.98232438925359 74.32098665597712) bank138621 +138622 POINT(41.201922649806335 73.74472224673781) bank138622 +138623 POINT(39.957814401189054 74.39212415249912) bank138623 +138624 POINT(41.37510972923178 73.4144712772336) bank138624 +138625 POINT(40.33420062037427 74.89468209349226) bank138625 +138626 POINT(40.99333060012871 74.08780204231147) bank138626 +138627 POINT(41.69959925730095 73.5343493490078) bank138627 +138628 POINT(41.68931703450705 74.0315462184344) bank138628 +138629 POINT(40.26107231246012 73.71513711301229) bank138629 +138630 POINT(39.79450434387801 74.01253718973368) bank138630 +138631 POINT(41.57106230567275 74.43594600552058) bank138631 +138632 POINT(41.41134077761307 74.59901710210399) bank138632 +138633 POINT(40.359561784798885 73.07015114299222) bank138633 +138634 POINT(39.72546133833218 74.4470045805081) bank138634 +138635 POINT(41.39191610997447 73.95401590593245) bank138635 +138636 POINT(39.77840647558403 73.21234581876547) bank138636 +138637 POINT(40.25484869713496 74.04329588074073) bank138637 +138638 POINT(39.91261046912445 74.48022527762512) bank138638 +138639 POINT(40.610834336304514 73.23157572597685) bank138639 +138640 POINT(40.382344195347336 73.92908398543379) bank138640 +138641 POINT(40.65545373731033 73.85804663078673) bank138641 +138642 POINT(40.08824896928717 73.60419333913718) bank138642 +138643 POINT(41.28852635910679 73.42661058676443) bank138643 +138644 POINT(40.90500474357315 73.95867456278603) bank138644 +138645 POINT(41.62342899390097 74.43555094959585) bank138645 +138646 POINT(40.84695370362414 73.09524496601809) bank138646 +138647 POINT(40.03131539103525 74.84547921071714) bank138647 +138648 POINT(40.178934609018555 74.70259158227766) bank138648 +138649 POINT(40.33402517120293 74.64115669431654) bank138649 +138650 POINT(41.06946306330124 74.96189334886742) bank138650 +138651 POINT(40.314152300770374 73.65054552629809) bank138651 +138652 POINT(40.361649717535265 73.56574171480914) bank138652 +138653 POINT(40.727092273131895 73.52390690443795) bank138653 +138654 POINT(40.33614346876116 73.60835064558722) bank138654 +138655 POINT(41.4075878413101 73.14386614090975) bank138655 +138656 POINT(40.7388926549982 74.06774906719903) bank138656 +138657 POINT(40.83738643158203 74.33737958334386) bank138657 +138658 POINT(41.35723332169632 74.64146537101128) bank138658 +138659 POINT(39.769623466901734 74.26678810654687) bank138659 +138660 POINT(40.99183910771203 73.7837743357407) bank138660 +138661 POINT(40.74498137039557 74.27059023581462) bank138661 +138662 POINT(40.005872779663754 73.48894889890484) bank138662 +138663 POINT(40.2879812206781 73.9116812203784) bank138663 +138664 POINT(39.897974674248076 73.05186052793798) bank138664 +138665 POINT(40.40226562256829 74.96369948000007) bank138665 +138666 POINT(41.27576569738889 73.0807591698341) bank138666 +138667 POINT(41.07466531995626 73.77113533689577) bank138667 +138668 POINT(40.92441994062529 73.94656507883755) bank138668 +138669 POINT(40.85206939552913 74.69673617367437) bank138669 +138670 POINT(39.747634377435475 74.01709229519325) bank138670 +138671 POINT(40.24810308081091 73.58882312606205) bank138671 +138672 POINT(40.31291686880073 73.32863853047553) bank138672 +138673 POINT(40.748236252487565 73.4035517543799) bank138673 +138674 POINT(41.121460054865125 73.9947950438257) bank138674 +138675 POINT(40.34606476920933 73.74189643995447) bank138675 +138676 POINT(40.03275601699936 73.75732220117409) bank138676 +138677 POINT(39.938301810082436 74.21814669462405) bank138677 +138678 POINT(40.628764332230446 75.00078995768231) bank138678 +138679 POINT(40.094061248709785 74.04946981672455) bank138679 +138680 POINT(40.911568349835115 73.2048272176299) bank138680 +138681 POINT(39.816624137617886 74.36032355683551) bank138681 +138682 POINT(41.541898818282775 73.7492433828143) bank138682 +138683 POINT(40.85819179322753 73.57135378292556) bank138683 +138684 POINT(40.96202376432647 73.78664685085518) bank138684 +138685 POINT(40.619768447847186 73.66007373585617) bank138685 +138686 POINT(40.08963898311723 73.28148464737328) bank138686 +138687 POINT(41.16733701497275 73.85785812182394) bank138687 +138688 POINT(41.613897398591455 73.5947949004609) bank138688 +138689 POINT(40.5418794932692 74.97975092029769) bank138689 +138690 POINT(40.73458030097867 74.0342287389375) bank138690 +138691 POINT(41.37139946137381 74.85606792005544) bank138691 +138692 POINT(40.797171836598366 74.33444140434945) bank138692 +138693 POINT(41.02705357094699 74.93024508902191) bank138693 +138694 POINT(40.73224452200489 74.11577722307159) bank138694 +138695 POINT(41.043825561512975 74.44338616835113) bank138695 +138696 POINT(41.13010722289342 73.40934401339987) bank138696 +138697 POINT(40.912019869091395 74.62323985091338) bank138697 +138698 POINT(39.96537153471117 73.32659881943135) bank138698 +138699 POINT(40.29599395648957 74.3103448669393) bank138699 +138700 POINT(40.1518978589715 74.4196359091767) bank138700 +138701 POINT(40.10702683393641 74.87046342157981) bank138701 +138702 POINT(39.85526594905421 73.20112705865141) bank138702 +138703 POINT(41.51924846643595 74.29079418983382) bank138703 +138704 POINT(41.21974204500529 74.75284128022385) bank138704 +138705 POINT(41.28035625496313 74.57375501998793) bank138705 +138706 POINT(41.59721009094562 73.91420685798802) bank138706 +138707 POINT(41.167497197104296 73.25520587206327) bank138707 +138708 POINT(39.86518439880954 74.42196588998269) bank138708 +138709 POINT(41.65396217879029 73.76411365352554) bank138709 +138710 POINT(40.85434115722538 73.26297784469848) bank138710 +138711 POINT(41.608899970905064 73.93050599195158) bank138711 +138712 POINT(41.07344385490889 74.87244061132641) bank138712 +138713 POINT(41.49973789811142 73.57802872686362) bank138713 +138714 POINT(40.8603132090014 74.02648419257555) bank138714 +138715 POINT(41.351035397188205 73.4574574913356) bank138715 +138716 POINT(39.72814258583564 74.62171738418363) bank138716 +138717 POINT(40.37204087824311 73.52679204916144) bank138717 +138718 POINT(40.6360773619637 74.57111906885555) bank138718 +138719 POINT(39.81966022636797 74.23706624843288) bank138719 +138720 POINT(40.44527784991938 74.21237146204066) bank138720 +138721 POINT(40.50325310509874 73.85772023586352) bank138721 +138722 POINT(40.42263826761537 74.35903764173608) bank138722 +138723 POINT(41.17944044692418 73.50335292072309) bank138723 +138724 POINT(41.21640427206933 74.20865900314486) bank138724 +138725 POINT(40.40219528300193 73.85342041722778) bank138725 +138726 POINT(40.90840948623109 74.37480747111108) bank138726 +138727 POINT(40.09978938004481 73.63942784867146) bank138727 +138728 POINT(41.289644503637675 74.57748941405012) bank138728 +138729 POINT(41.45914777925894 74.81362177786585) bank138729 +138730 POINT(39.77748701259474 73.80205374175421) bank138730 +138731 POINT(41.45469749559062 73.3890654157781) bank138731 +138732 POINT(39.7907142419038 74.26111005012915) bank138732 +138733 POINT(40.59511516849318 73.36370023961106) bank138733 +138734 POINT(41.14918658902403 74.27676137785059) bank138734 +138735 POINT(40.865695874683134 73.45574129886647) bank138735 +138736 POINT(40.578825464759625 74.80109180012765) bank138736 +138737 POINT(40.596793745279655 73.50254949344215) bank138737 +138738 POINT(40.37596007588473 73.77166706566159) bank138738 +138739 POINT(41.53045767666416 73.05428751158324) bank138739 +138740 POINT(40.07158509662526 73.43917072472328) bank138740 +138741 POINT(40.478359853959546 74.1128023424409) bank138741 +138742 POINT(40.43885238099646 73.3233387397301) bank138742 +138743 POINT(40.91767916949136 74.14921048372102) bank138743 +138744 POINT(40.289404946148615 73.21033930610905) bank138744 +138745 POINT(41.503798253321 73.27788053493646) bank138745 +138746 POINT(41.146097379768335 73.6376090118258) bank138746 +138747 POINT(41.51075741108639 73.40302710828469) bank138747 +138748 POINT(41.341034335060385 74.37035151910906) bank138748 +138749 POINT(40.19684953812926 73.34735159320881) bank138749 +138750 POINT(41.151830410663884 74.95723690030302) bank138750 +138751 POINT(39.81470602223614 74.00365798513626) bank138751 +138752 POINT(39.976160754710655 73.1759852403835) bank138752 +138753 POINT(40.41689026951203 73.76591426980983) bank138753 +138754 POINT(41.10819597218843 73.53374624830957) bank138754 +138755 POINT(40.74657258135122 73.70805258810412) bank138755 +138756 POINT(40.526845846398786 74.36567081583743) bank138756 +138757 POINT(40.774003609210226 73.97000203493741) bank138757 +138758 POINT(40.49561247267304 73.42539082287611) bank138758 +138759 POINT(40.93369984228157 73.67472190000016) bank138759 +138760 POINT(40.750939545554886 73.9170761361117) bank138760 +138761 POINT(41.60865882115016 73.57089561105239) bank138761 +138762 POINT(39.95127927738752 73.73185444905779) bank138762 +138763 POINT(39.858892562742874 73.01921589051813) bank138763 +138764 POINT(39.94382441662453 74.21578764542468) bank138764 +138765 POINT(40.197782603051856 73.41114920347698) bank138765 +138766 POINT(41.12518382298952 74.24828198672363) bank138766 +138767 POINT(40.64409138618794 73.64634698596305) bank138767 +138768 POINT(40.287802637216906 73.55239608881614) bank138768 +138769 POINT(40.920425271740555 73.26111296473456) bank138769 +138770 POINT(40.50581911402465 74.04878112628295) bank138770 +138771 POINT(40.93562739238311 74.79202588712148) bank138771 +138772 POINT(39.772821141662135 73.86321344933853) bank138772 +138773 POINT(40.41286336790237 74.42771335289734) bank138773 +138774 POINT(40.90020068388492 74.09207772577636) bank138774 +138775 POINT(39.80190557293543 73.39250727608774) bank138775 +138776 POINT(40.205721137699186 74.4749056215075) bank138776 +138777 POINT(40.80546987804757 73.1337612070821) bank138777 +138778 POINT(41.6812857239028 74.82434246266263) bank138778 +138779 POINT(39.81057932805721 73.33479312326577) bank138779 +138780 POINT(40.49632985057429 74.23215011195224) bank138780 +138781 POINT(41.28202843036135 73.11415670253538) bank138781 +138782 POINT(40.73198589052871 74.54852132520588) bank138782 +138783 POINT(41.2431491130219 73.71151100562393) bank138783 +138784 POINT(40.0083948716906 74.50038188572182) bank138784 +138785 POINT(41.68351314284159 74.67419859265082) bank138785 +138786 POINT(40.600411850309825 74.28034245212478) bank138786 +138787 POINT(41.196851074904586 74.53299842028012) bank138787 +138788 POINT(40.211659566527594 74.13915257042432) bank138788 +138789 POINT(40.27825374135167 73.11838567535872) bank138789 +138790 POINT(40.66999485976821 73.632168716767) bank138790 +138791 POINT(41.43252358619713 73.7327115555939) bank138791 +138792 POINT(40.74402038082284 74.45965543067432) bank138792 +138793 POINT(40.14666604199146 74.98774680187877) bank138793 +138794 POINT(41.547591271797806 73.76867580716213) bank138794 +138795 POINT(40.97241114683152 73.85269542173677) bank138795 +138796 POINT(41.06603036088575 73.23508608069614) bank138796 +138797 POINT(40.65154197287442 73.35118274199505) bank138797 +138798 POINT(41.17694552025811 73.27057572671984) bank138798 +138799 POINT(40.91810754179867 74.06820510328019) bank138799 +138800 POINT(40.75830004611963 73.9808627274222) bank138800 +138801 POINT(39.83827568953537 73.93376748422932) bank138801 +138802 POINT(41.25951744499147 74.32242082472857) bank138802 +138803 POINT(41.35724650182111 73.87727088984707) bank138803 +138804 POINT(41.61150970392274 75.00043763750662) bank138804 +138805 POINT(40.50086543101662 74.58037147474214) bank138805 +138806 POINT(40.73329970591859 74.56637615585245) bank138806 +138807 POINT(40.54761763877045 74.0790906615458) bank138807 +138808 POINT(41.38102064485254 73.57530226214811) bank138808 +138809 POINT(40.877563057919 73.31469974769276) bank138809 +138810 POINT(41.16607903936931 74.6235404223595) bank138810 +138811 POINT(40.193465700328794 74.19796668049146) bank138811 +138812 POINT(40.00796620993057 74.23404847694559) bank138812 +138813 POINT(40.16468114666881 73.37248898017957) bank138813 +138814 POINT(41.4737464693575 74.73153182878893) bank138814 +138815 POINT(39.91042635125968 73.53274104028803) bank138815 +138816 POINT(41.254991356855335 74.44300333244757) bank138816 +138817 POINT(41.31465988942101 74.23896808472321) bank138817 +138818 POINT(40.65696748182823 73.77189875222183) bank138818 +138819 POINT(40.68789054972528 73.4718076772341) bank138819 +138820 POINT(40.29105224960708 74.68700788229003) bank138820 +138821 POINT(41.20225820590326 74.12632604470178) bank138821 +138822 POINT(40.182856904332354 73.65977945821083) bank138822 +138823 POINT(40.070719296659895 74.5166037025811) bank138823 +138824 POINT(41.22744928038638 73.63393171175883) bank138824 +138825 POINT(41.170465889183404 74.68346440010086) bank138825 +138826 POINT(40.7624245914774 73.38923412825758) bank138826 +138827 POINT(40.42543646740654 73.03596558131831) bank138827 +138828 POINT(40.34681877664015 73.18284956237581) bank138828 +138829 POINT(41.1633157047924 74.07255500077571) bank138829 +138830 POINT(40.416725659253046 73.36505895166312) bank138830 +138831 POINT(40.645283786832785 74.71366083620411) bank138831 +138832 POINT(40.54737867068685 74.79211255077449) bank138832 +138833 POINT(41.04536881636121 74.35271028349254) bank138833 +138834 POINT(40.046774588263446 74.6946108276684) bank138834 +138835 POINT(40.67772671634667 73.9496048283715) bank138835 +138836 POINT(41.071585070495985 73.72639057892123) bank138836 +138837 POINT(40.8113677995985 73.95094250573378) bank138837 +138838 POINT(40.60805934380473 73.17286631325689) bank138838 +138839 POINT(40.22048733245335 74.66758840230662) bank138839 +138840 POINT(40.644583420747665 74.6505176804317) bank138840 +138841 POINT(40.09759775216968 73.70854971638282) bank138841 +138842 POINT(41.048059629438924 73.83062692615907) bank138842 +138843 POINT(41.259039939204584 74.80008590386474) bank138843 +138844 POINT(40.854911840623515 73.54266581835498) bank138844 +138845 POINT(40.07998904650561 74.01405727011627) bank138845 +138846 POINT(40.97466198355724 73.42220446509903) bank138846 +138847 POINT(40.054502319182646 73.64415155502617) bank138847 +138848 POINT(40.58859567341689 74.35614388754657) bank138848 +138849 POINT(40.54672183735949 74.19168632347647) bank138849 +138850 POINT(41.00459820698273 73.6870786538066) bank138850 +138851 POINT(40.522206233229696 74.18035469182708) bank138851 +138852 POINT(41.202912160863015 74.6316840832441) bank138852 +138853 POINT(41.08707740714923 74.33494585447364) bank138853 +138854 POINT(40.00633522026368 73.12513982204162) bank138854 +138855 POINT(40.05456149669434 73.85232524946555) bank138855 +138856 POINT(40.94866960573662 74.03077755159244) bank138856 +138857 POINT(41.15910831280337 73.16819983580379) bank138857 +138858 POINT(40.47686607784435 74.49960062254466) bank138858 +138859 POINT(41.223956703307934 74.61394603518147) bank138859 +138860 POINT(40.76628747244018 73.66136519796567) bank138860 +138861 POINT(39.941049782076945 73.72304632023399) bank138861 +138862 POINT(41.08114066125241 74.02320962414163) bank138862 +138863 POINT(40.4759854407443 73.51351479801265) bank138863 +138864 POINT(40.00144605385652 74.6299920987626) bank138864 +138865 POINT(40.985540804757804 74.2993766944156) bank138865 +138866 POINT(41.53454268913896 73.3726619185961) bank138866 +138867 POINT(41.1408650926257 74.56624113798946) bank138867 +138868 POINT(41.01368662628355 74.08644408728668) bank138868 +138869 POINT(40.52471326159013 73.3613755026342) bank138869 +138870 POINT(41.696768421958915 74.80604531994642) bank138870 +138871 POINT(40.33986274855553 74.77503499409485) bank138871 +138872 POINT(40.051729417015515 74.31246098712342) bank138872 +138873 POINT(41.44375873154197 74.24356328815678) bank138873 +138874 POINT(40.207509685027205 73.31313655797268) bank138874 +138875 POINT(40.70612532725395 74.83595136148452) bank138875 +138876 POINT(41.32698532628225 73.16219574942278) bank138876 +138877 POINT(40.08876300446171 73.44090258499234) bank138877 +138878 POINT(40.05568210592046 74.25559123286133) bank138878 +138879 POINT(41.204928486945356 74.30636740299119) bank138879 +138880 POINT(40.72939595507381 74.9248813559752) bank138880 +138881 POINT(40.788525066731864 74.98727334292403) bank138881 +138882 POINT(40.53671116915541 73.53562482327024) bank138882 +138883 POINT(40.104024643110556 73.88861727094226) bank138883 +138884 POINT(40.97498512606342 74.07552826279945) bank138884 +138885 POINT(40.82804347385159 74.27806873666886) bank138885 +138886 POINT(39.7401991174108 74.677006119567) bank138886 +138887 POINT(40.78817424366499 73.80944209714649) bank138887 +138888 POINT(40.20700588666062 73.45065657617829) bank138888 +138889 POINT(41.03032073873513 74.69060599952617) bank138889 +138890 POINT(41.11524976152014 73.11611865707341) bank138890 +138891 POINT(41.53414082028222 74.14375814750396) bank138891 +138892 POINT(40.29335863021906 74.19338451144885) bank138892 +138893 POINT(40.189566794943325 74.68828126764667) bank138893 +138894 POINT(40.09316647991621 74.41100820838201) bank138894 +138895 POINT(39.799850470472634 74.9371113419161) bank138895 +138896 POINT(41.081337902686066 73.11940498912388) bank138896 +138897 POINT(40.49403649043259 74.63592848278505) bank138897 +138898 POINT(39.975944146231235 74.17366204689954) bank138898 +138899 POINT(41.309991479828994 73.25611990865119) bank138899 +138900 POINT(40.208614883659166 73.52825942265052) bank138900 +138901 POINT(40.363949676477084 74.4733954469322) bank138901 +138902 POINT(40.142836637004464 73.46042822267664) bank138902 +138903 POINT(40.55078174739333 74.82319503990202) bank138903 +138904 POINT(41.601174894743764 74.00326516410472) bank138904 +138905 POINT(40.312309683879384 73.77344761409557) bank138905 +138906 POINT(41.604864997498865 74.02011243035642) bank138906 +138907 POINT(41.33851311738864 74.11530805787089) bank138907 +138908 POINT(41.35940582560731 73.91928329754278) bank138908 +138909 POINT(40.49928788157146 73.04027161622011) bank138909 +138910 POINT(40.9586762015998 73.27397623507429) bank138910 +138911 POINT(41.09320830310067 74.06865180896226) bank138911 +138912 POINT(40.674634578135205 73.46473726718423) bank138912 +138913 POINT(40.395755968568984 74.28925127774122) bank138913 +138914 POINT(41.66107793512159 73.03008521161881) bank138914 +138915 POINT(40.348635878039865 73.90201100804333) bank138915 +138916 POINT(40.78855723983803 74.6941951365987) bank138916 +138917 POINT(40.936666362137046 74.49967341939575) bank138917 +138918 POINT(39.783831136296186 73.46241764770224) bank138918 +138919 POINT(41.07951632018049 73.9086169158887) bank138919 +138920 POINT(40.713990221110116 74.89710076733232) bank138920 +138921 POINT(40.96508182234185 74.92825203111023) bank138921 +138922 POINT(41.49060860982975 73.08325737415404) bank138922 +138923 POINT(41.27483362351952 74.47462607970697) bank138923 +138924 POINT(40.87959556082696 73.56266422789737) bank138924 +138925 POINT(40.23396965925169 74.32797814468798) bank138925 +138926 POINT(39.81060126363183 73.0371507348457) bank138926 +138927 POINT(41.39468743678864 74.43103123010873) bank138927 +138928 POINT(41.26007066292911 73.68210496821597) bank138928 +138929 POINT(40.239455346587434 74.78321246920034) bank138929 +138930 POINT(41.33578431242898 74.22800329092853) bank138930 +138931 POINT(41.08299847931147 73.1405728731837) bank138931 +138932 POINT(40.31509138200191 73.77682650562572) bank138932 +138933 POINT(40.183894701504876 74.9636249121654) bank138933 +138934 POINT(40.653830291170905 73.30641081130727) bank138934 +138935 POINT(40.960845826933024 73.19647372654953) bank138935 +138936 POINT(41.415285854067825 73.68388171918804) bank138936 +138937 POINT(40.988202169086755 74.57366023732325) bank138937 +138938 POINT(41.46513199951195 74.16293876974365) bank138938 +138939 POINT(40.89988191521155 74.09675022248781) bank138939 +138940 POINT(40.96790038944106 74.74300685499257) bank138940 +138941 POINT(40.05386292942786 73.35522901285717) bank138941 +138942 POINT(40.58658181514427 73.95385701626368) bank138942 +138943 POINT(39.79254200849265 74.53574451103516) bank138943 +138944 POINT(40.72772036820822 74.95338949278718) bank138944 +138945 POINT(40.02779655216001 73.19433496605367) bank138945 +138946 POINT(41.23047598701546 73.06905392383234) bank138946 +138947 POINT(40.884124586694384 73.46095781449753) bank138947 +138948 POINT(39.847343605086486 74.98005736466192) bank138948 +138949 POINT(40.15719883230484 74.72116368984628) bank138949 +138950 POINT(40.36667785891757 73.07271983407418) bank138950 +138951 POINT(39.72571806565949 73.33329983436026) bank138951 +138952 POINT(40.401797542557205 73.58406639627972) bank138952 +138953 POINT(40.46472260421315 73.16318244239227) bank138953 +138954 POINT(41.01620331257239 73.78542961344353) bank138954 +138955 POINT(39.92048188095109 73.26078321159355) bank138955 +138956 POINT(40.51361613498168 74.04070442442996) bank138956 +138957 POINT(40.564393730417606 74.66294532377785) bank138957 +138958 POINT(41.28218174649407 73.37900181044623) bank138958 +138959 POINT(40.983035876047886 74.46792641879743) bank138959 +138960 POINT(39.82969994807325 74.41232051432594) bank138960 +138961 POINT(41.05041237938014 73.82913861525041) bank138961 +138962 POINT(41.56386385977695 73.71367196883752) bank138962 +138963 POINT(41.414614624891804 74.67466655783218) bank138963 +138964 POINT(40.269582743536375 74.7365667368999) bank138964 +138965 POINT(41.31092813399824 74.48028301189306) bank138965 +138966 POINT(41.547260575106385 74.44864822538807) bank138966 +138967 POINT(40.72475606720543 74.81996430244767) bank138967 +138968 POINT(41.1853733364434 73.47544561298068) bank138968 +138969 POINT(41.478943117505416 74.87665281395076) bank138969 +138970 POINT(40.08344289667907 73.47190248507088) bank138970 +138971 POINT(41.35327886394999 74.467232776277) bank138971 +138972 POINT(40.22011650449973 73.88001828792238) bank138972 +138973 POINT(40.442301195585294 73.44888027213105) bank138973 +138974 POINT(40.927892112084876 74.3729602218729) bank138974 +138975 POINT(40.005072377435916 73.35094335162273) bank138975 +138976 POINT(41.217417609384434 74.6923981152936) bank138976 +138977 POINT(41.036909308606845 74.79475673083364) bank138977 +138978 POINT(40.46387293880154 73.61955282316995) bank138978 +138979 POINT(41.09418988749709 74.07762796232163) bank138979 +138980 POINT(40.96717080260298 73.9775282739447) bank138980 +138981 POINT(41.33112848109645 73.50922222403966) bank138981 +138982 POINT(41.35323930836179 73.79821338703852) bank138982 +138983 POINT(40.228542847877904 74.80026145680851) bank138983 +138984 POINT(40.74493303952203 73.02686914651603) bank138984 +138985 POINT(41.609882996454694 73.53765166450123) bank138985 +138986 POINT(39.728802534132186 73.05230387507422) bank138986 +138987 POINT(40.63332427383209 73.85474453116952) bank138987 +138988 POINT(40.79649868065224 74.8500539847128) bank138988 +138989 POINT(40.43043100637726 74.77923250393637) bank138989 +138990 POINT(40.285622519568335 73.34242755581387) bank138990 +138991 POINT(41.208408967993904 74.2446169269274) bank138991 +138992 POINT(40.542532671628976 74.69909394379881) bank138992 +138993 POINT(40.731680828010454 74.13509475619107) bank138993 +138994 POINT(41.44858929736154 74.5149266010674) bank138994 +138995 POINT(41.4274144069268 74.55124643431883) bank138995 +138996 POINT(39.8796679741227 73.27093451501186) bank138996 +138997 POINT(40.094535572561064 73.53128332419402) bank138997 +138998 POINT(41.55847388465549 73.01789306195414) bank138998 +138999 POINT(40.454968627527414 74.16864848079746) bank138999 +139000 POINT(40.61148349177644 74.67532726937573) bank139000 +139001 POINT(40.46242836742785 74.78918781810248) bank139001 +139002 POINT(41.09326277762302 74.81846675128693) bank139002 +139003 POINT(41.11509650314566 73.17634675903231) bank139003 +139004 POINT(40.364757868545645 73.55107628063651) bank139004 +139005 POINT(40.797825842155014 73.14836445423846) bank139005 +139006 POINT(41.310821782319984 74.35638373002703) bank139006 +139007 POINT(40.779714817189806 73.94209655418553) bank139007 +139008 POINT(41.24995311727481 74.69309776834005) bank139008 +139009 POINT(40.937296150800684 73.67239944706921) bank139009 +139010 POINT(40.69303429335549 74.94551242217845) bank139010 +139011 POINT(40.36477719802993 74.12459219488588) bank139011 +139012 POINT(39.90898374046716 73.23964705422978) bank139012 +139013 POINT(40.836838484361294 74.94441960258797) bank139013 +139014 POINT(40.843419774426515 73.99677262215039) bank139014 +139015 POINT(40.69938395823269 73.96496671830238) bank139015 +139016 POINT(41.30071384735597 74.87855750945812) bank139016 +139017 POINT(40.13175221075076 73.58271067064554) bank139017 +139018 POINT(40.57509689117445 73.61583347170715) bank139018 +139019 POINT(39.82812724728799 73.03551386237868) bank139019 +139020 POINT(41.115788935566954 73.93413651426143) bank139020 +139021 POINT(40.21003421081469 73.93298845984242) bank139021 +139022 POINT(41.37462492340791 74.07175045705254) bank139022 +139023 POINT(39.88363284795451 73.96824292669163) bank139023 +139024 POINT(41.21955293392662 73.60846326696422) bank139024 +139025 POINT(41.04521834172722 73.2518114989484) bank139025 +139026 POINT(40.23849310032296 73.15371250475441) bank139026 +139027 POINT(41.551883583141226 74.78785525972893) bank139027 +139028 POINT(39.74270998350708 73.6480669965296) bank139028 +139029 POINT(40.7497803609408 73.49264615048234) bank139029 +139030 POINT(40.08925018957258 74.69679953170214) bank139030 +139031 POINT(41.33975546111296 73.29730192839402) bank139031 +139032 POINT(41.310609903095155 73.3046933862243) bank139032 +139033 POINT(40.53822371331131 73.94998839774522) bank139033 +139034 POINT(41.60006092614644 73.52841777920167) bank139034 +139035 POINT(41.163112892107534 73.91729482308345) bank139035 +139036 POINT(40.63815111100033 74.7531374156717) bank139036 +139037 POINT(39.76154533989682 74.39756748381328) bank139037 +139038 POINT(41.24188056271615 74.59371941711207) bank139038 +139039 POINT(40.01102195680231 74.26740284912924) bank139039 +139040 POINT(40.11097974325273 73.97070388483621) bank139040 +139041 POINT(40.63759999471378 74.76419659674328) bank139041 +139042 POINT(39.83002075865077 74.05689785444386) bank139042 +139043 POINT(41.39665885618211 74.93886067406719) bank139043 +139044 POINT(40.95768108331949 73.7952147260035) bank139044 +139045 POINT(40.44215374728161 73.90521156475076) bank139045 +139046 POINT(40.18485105775011 74.71015417791763) bank139046 +139047 POINT(40.06177117726325 73.12455347163254) bank139047 +139048 POINT(40.02625302429828 74.703276314191) bank139048 +139049 POINT(40.20397312283564 73.77552958642661) bank139049 +139050 POINT(39.84343227397085 74.92085243547268) bank139050 +139051 POINT(40.24691840317033 73.33834149161865) bank139051 +139052 POINT(40.47397462354062 74.22561168583012) bank139052 +139053 POINT(41.346658431115095 73.65838311548924) bank139053 +139054 POINT(41.391084459101315 73.42972526999185) bank139054 +139055 POINT(41.53972927787686 73.64774717771733) bank139055 +139056 POINT(41.44577017211405 73.76556576442535) bank139056 +139057 POINT(40.75725015744382 73.9874558585022) bank139057 +139058 POINT(40.986031012409036 74.86877722026365) bank139058 +139059 POINT(39.94840956355818 74.86493277396293) bank139059 +139060 POINT(41.35128243863943 73.15517939764428) bank139060 +139061 POINT(40.73720294683504 73.56311789542438) bank139061 +139062 POINT(41.6203310034315 73.8286735819202) bank139062 +139063 POINT(41.57116681315189 73.23507256218774) bank139063 +139064 POINT(41.19553369558545 73.81476783731398) bank139064 +139065 POINT(39.71676229705606 73.65887268571093) bank139065 +139066 POINT(40.95463641461095 74.81038695314773) bank139066 +139067 POINT(40.127425753091245 73.79064911945461) bank139067 +139068 POINT(40.07527932497454 73.48223962765877) bank139068 +139069 POINT(40.59204117520925 73.2229310163237) bank139069 +139070 POINT(40.319771558089116 74.69239813904815) bank139070 +139071 POINT(40.29121046806353 74.60202780043385) bank139071 +139072 POINT(40.161257388747664 74.92892655288158) bank139072 +139073 POINT(40.6953377774707 74.67019929571973) bank139073 +139074 POINT(39.75421251149745 73.25626838153826) bank139074 +139075 POINT(40.638062996723214 74.31413549506037) bank139075 +139076 POINT(39.87299398016731 74.16613296322745) bank139076 +139077 POINT(40.40480230755507 74.14810322674637) bank139077 +139078 POINT(40.878576214111675 75.00287274740401) bank139078 +139079 POINT(40.50576999837981 73.78191864805201) bank139079 +139080 POINT(40.82346187390708 74.32909740748111) bank139080 +139081 POINT(40.14353969577363 73.08912280604143) bank139081 +139082 POINT(41.61268325968721 73.82639543195206) bank139082 +139083 POINT(39.72732303379234 74.63910625114363) bank139083 +139084 POINT(41.501908808466005 73.88286662155032) bank139084 +139085 POINT(40.03015793857425 74.24770458964205) bank139085 +139086 POINT(40.29254965299863 74.65711583901398) bank139086 +139087 POINT(39.91999956666298 74.38525391566874) bank139087 +139088 POINT(40.6761294074288 73.87122729429902) bank139088 +139089 POINT(41.372778789930976 74.77466268431392) bank139089 +139090 POINT(41.22475994303429 74.8767953235467) bank139090 +139091 POINT(40.48182539519561 73.97481940501764) bank139091 +139092 POINT(41.589579780971604 74.22914102037007) bank139092 +139093 POINT(41.47539993114191 73.21462354762032) bank139093 +139094 POINT(41.25089934955336 74.81347922290726) bank139094 +139095 POINT(39.93429967134228 74.13398971935605) bank139095 +139096 POINT(39.78254356458964 74.49056463594042) bank139096 +139097 POINT(39.948670490289594 74.68089386442071) bank139097 +139098 POINT(40.175735097318444 75.00305748201055) bank139098 +139099 POINT(40.03654605107913 74.88653506640638) bank139099 +139100 POINT(40.6093228273202 74.78481409003511) bank139100 +139101 POINT(40.49690501785688 74.08352802076806) bank139101 +139102 POINT(40.471210630077294 73.7680277665186) bank139102 +139103 POINT(41.099395868475234 73.95510992816772) bank139103 +139104 POINT(40.8563223045703 73.86842107776944) bank139104 +139105 POINT(39.995346527424026 73.12357360778424) bank139105 +139106 POINT(40.14539397753055 74.20541780572208) bank139106 +139107 POINT(40.9171556145258 74.47652693882475) bank139107 +139108 POINT(40.49341909024225 73.37925747991622) bank139108 +139109 POINT(40.44341244486862 74.86124099105002) bank139109 +139110 POINT(41.23904059202254 74.45833675262946) bank139110 +139111 POINT(41.47542919784155 73.60340448689605) bank139111 +139112 POINT(40.41309102125764 74.16838069207807) bank139112 +139113 POINT(40.57572570736017 73.70716114704973) bank139113 +139114 POINT(41.34452996144723 73.27967442032407) bank139114 +139115 POINT(40.752079791786585 74.30111863771089) bank139115 +139116 POINT(41.675324473887805 73.48325217121638) bank139116 +139117 POINT(39.912520202002774 74.74449597029614) bank139117 +139118 POINT(40.94294007429807 73.27014585180075) bank139118 +139119 POINT(41.319269679246396 74.85608866603218) bank139119 +139120 POINT(40.7908087396405 73.2275716596461) bank139120 +139121 POINT(41.48347215653382 73.23791159086542) bank139121 +139122 POINT(41.456969797366774 73.66085368536005) bank139122 +139123 POINT(41.57472200590916 73.20941857103251) bank139123 +139124 POINT(40.28117412974439 73.0123384035713) bank139124 +139125 POINT(40.67822095102395 74.27125128277028) bank139125 +139126 POINT(40.21843640550753 74.11097408346181) bank139126 +139127 POINT(39.82110800927715 74.2436259395164) bank139127 +139128 POINT(41.42117281473109 73.0191919802882) bank139128 +139129 POINT(40.16935160582298 73.73416970615136) bank139129 +139130 POINT(40.12436571733157 73.18448299337078) bank139130 +139131 POINT(41.28141529138089 74.5059793560212) bank139131 +139132 POINT(39.736094611207676 73.18092542112304) bank139132 +139133 POINT(40.71590540350214 73.11239936368057) bank139133 +139134 POINT(41.23349933192587 73.40488273197056) bank139134 +139135 POINT(41.14003500166764 73.89910515918888) bank139135 +139136 POINT(40.261586656160006 74.82211683390007) bank139136 +139137 POINT(40.09352405526966 74.73454167372442) bank139137 +139138 POINT(41.192469459479675 74.47389154828109) bank139138 +139139 POINT(40.701583376992566 73.88691465669127) bank139139 +139140 POINT(41.65275790981259 73.33175830647335) bank139140 +139141 POINT(40.214548877441835 73.71711437258381) bank139141 +139142 POINT(39.83501958357972 73.32318624871566) bank139142 +139143 POINT(40.35424373068958 73.10098717431062) bank139143 +139144 POINT(40.760980791775346 74.46900244446442) bank139144 +139145 POINT(41.46760070996142 73.66822762434006) bank139145 +139146 POINT(40.37865153286894 74.55719711799784) bank139146 +139147 POINT(41.6266249570432 74.65275011552663) bank139147 +139148 POINT(39.810941362777925 74.96319548614002) bank139148 +139149 POINT(39.96829764597178 74.4479066741732) bank139149 +139150 POINT(41.08523673213702 73.5583969350574) bank139150 +139151 POINT(39.848265773646155 74.44803461654467) bank139151 +139152 POINT(40.603728778314704 74.51934545852482) bank139152 +139153 POINT(40.80294038221254 73.74201956396553) bank139153 +139154 POINT(39.83203653002397 74.3102659626071) bank139154 +139155 POINT(40.43361089453904 74.77815920391589) bank139155 +139156 POINT(40.17053922317415 74.15029226091691) bank139156 +139157 POINT(40.65205088768145 74.09796868418567) bank139157 +139158 POINT(40.6255177023088 73.42097985900702) bank139158 +139159 POINT(41.310558796609854 73.3958616753064) bank139159 +139160 POINT(40.54061731004479 74.08187045953981) bank139160 +139161 POINT(39.94538177399636 73.45334308459758) bank139161 +139162 POINT(41.519040248642064 73.04194101650619) bank139162 +139163 POINT(39.75826198892369 74.86510382866706) bank139163 +139164 POINT(40.384448202573246 74.37022289285002) bank139164 +139165 POINT(41.48177950909677 73.31609691453862) bank139165 +139166 POINT(40.49634337057182 73.26163967037225) bank139166 +139167 POINT(40.174106168221236 74.0603288076686) bank139167 +139168 POINT(40.20321089399275 73.77837075380585) bank139168 +139169 POINT(39.943860744721185 74.2444668049727) bank139169 +139170 POINT(40.568886523524846 74.4459020869632) bank139170 +139171 POINT(40.64930954701364 74.96017039212988) bank139171 +139172 POINT(41.387381712080284 74.56979293992381) bank139172 +139173 POINT(40.82578560710068 73.69401167848963) bank139173 +139174 POINT(39.75863361652058 74.53476993486923) bank139174 +139175 POINT(40.99642407790296 73.99835799847963) bank139175 +139176 POINT(39.73597452940433 73.09994335548615) bank139176 +139177 POINT(40.8462031774532 73.27945098295362) bank139177 +139178 POINT(41.38914290970618 74.73779714397614) bank139178 +139179 POINT(41.137529645801465 74.1937258410879) bank139179 +139180 POINT(41.11760933436701 73.20203862855257) bank139180 +139181 POINT(40.18183451157908 74.30229351416253) bank139181 +139182 POINT(41.64314007513375 74.01386281904885) bank139182 +139183 POINT(41.44708264015977 73.27526895417509) bank139183 +139184 POINT(41.05639651779327 74.79683854929944) bank139184 +139185 POINT(41.293997651197444 74.44270872564555) bank139185 +139186 POINT(40.55869419345245 74.42802513730092) bank139186 +139187 POINT(40.442349410319316 74.09937690646902) bank139187 +139188 POINT(41.25108157989904 74.19232641206854) bank139188 +139189 POINT(41.18308470135942 74.56257710011545) bank139189 +139190 POINT(40.83138676509298 74.28691306951076) bank139190 +139191 POINT(39.78003055102429 73.554475947574) bank139191 +139192 POINT(39.95816212170005 74.40837952651682) bank139192 +139193 POINT(39.876977886324056 74.17926562776738) bank139193 +139194 POINT(40.55928429656537 74.30047719030145) bank139194 +139195 POINT(40.482741639720146 74.92368099521137) bank139195 +139196 POINT(39.730101260869915 74.06849291975769) bank139196 +139197 POINT(40.14302944613903 74.85331017261073) bank139197 +139198 POINT(41.11304977626209 73.80780677169763) bank139198 +139199 POINT(40.317606240330214 74.04009180071841) bank139199 +139200 POINT(40.888639244238995 74.4698317989409) bank139200 +139201 POINT(40.741155074592726 74.2971699498328) bank139201 +139202 POINT(41.61048769996779 73.04742563525046) bank139202 +139203 POINT(41.6636460346514 73.62870912911241) bank139203 +139204 POINT(40.76498264909377 74.87548179829508) bank139204 +139205 POINT(41.12270658731227 73.68320813725956) bank139205 +139206 POINT(41.307248540983686 74.12422548279314) bank139206 +139207 POINT(40.33028087808199 74.39347838854543) bank139207 +139208 POINT(40.29226860945998 73.08750876629968) bank139208 +139209 POINT(40.41861098796383 73.8474300156044) bank139209 +139210 POINT(39.930603874529794 73.57583928079632) bank139210 +139211 POINT(40.11676680712638 74.46950040214276) bank139211 +139212 POINT(41.46801254434733 74.49204004432035) bank139212 +139213 POINT(39.85480666751971 73.62158747035849) bank139213 +139214 POINT(40.05134663441842 73.61885857656632) bank139214 +139215 POINT(41.15478217621563 74.48147896475993) bank139215 +139216 POINT(40.47436437255996 74.08064959347867) bank139216 +139217 POINT(39.82713703476429 74.24302127608956) bank139217 +139218 POINT(41.05407640361207 74.35910499869412) bank139218 +139219 POINT(41.13548111216975 73.63164854047801) bank139219 +139220 POINT(40.601906573793144 73.7677443162632) bank139220 +139221 POINT(40.840201789119384 74.64779252818832) bank139221 +139222 POINT(39.92549066883839 74.6980785280191) bank139222 +139223 POINT(40.87313824515729 73.80501679013068) bank139223 +139224 POINT(39.89673505015015 73.44670053972955) bank139224 +139225 POINT(40.08699067422668 73.68112218392622) bank139225 +139226 POINT(41.60263087109442 73.64392427472646) bank139226 +139227 POINT(41.04407602028006 73.73095924915681) bank139227 +139228 POINT(41.606363441662666 74.55306359205738) bank139228 +139229 POINT(40.15006046856423 74.62799963094055) bank139229 +139230 POINT(40.823542107827286 73.92676952696695) bank139230 +139231 POINT(40.74842749667435 73.55509733782988) bank139231 +139232 POINT(40.07247913545658 73.11243091327931) bank139232 +139233 POINT(41.210600451245746 74.5989618367014) bank139233 +139234 POINT(40.5957505439496 74.48522001741718) bank139234 +139235 POINT(41.094669400365234 73.45658405551787) bank139235 +139236 POINT(40.918934827629805 74.10625350331723) bank139236 +139237 POINT(41.40381095659135 73.75239306211856) bank139237 +139238 POINT(41.366568980596654 73.99511197191553) bank139238 +139239 POINT(40.91260689725353 74.40791373198779) bank139239 +139240 POINT(41.27137727511695 74.31903340045174) bank139240 +139241 POINT(40.771857759610384 74.3105532424824) bank139241 +139242 POINT(40.98309440942771 73.19240686872327) bank139242 +139243 POINT(41.4577943428077 74.29139048233307) bank139243 +139244 POINT(39.77164110104047 73.81304965741107) bank139244 +139245 POINT(40.76916099223192 73.31501219934502) bank139245 +139246 POINT(41.55113151400053 74.40373332783662) bank139246 +139247 POINT(39.88871567612444 74.35619050408653) bank139247 +139248 POINT(40.10636208699704 74.85086985602123) bank139248 +139249 POINT(41.7001651307938 73.14680003245122) bank139249 +139250 POINT(40.059154451906416 73.21903101521848) bank139250 +139251 POINT(40.08233243756143 74.0976304990629) bank139251 +139252 POINT(39.77385536232072 73.3982211175399) bank139252 +139253 POINT(40.162491472899305 73.34138568993708) bank139253 +139254 POINT(40.33687603355973 74.95752339208316) bank139254 +139255 POINT(40.71470987526837 74.67868310396179) bank139255 +139256 POINT(41.292251895125254 73.62673903671647) bank139256 +139257 POINT(41.56539236330895 74.5064592154888) bank139257 +139258 POINT(41.07151626904239 74.41279792703098) bank139258 +139259 POINT(39.9867510968788 73.95941417685832) bank139259 +139260 POINT(41.41301951935306 74.21857589647588) bank139260 +139261 POINT(40.22752000409132 73.577775408472) bank139261 +139262 POINT(41.454107961151685 74.77215268635118) bank139262 +139263 POINT(41.418893251163865 73.72004709181856) bank139263 +139264 POINT(40.12362670187689 74.81200608400879) bank139264 +139265 POINT(40.993330241965914 73.70560952671957) bank139265 +139266 POINT(41.16365175859197 73.42448927973832) bank139266 +139267 POINT(40.329240550273845 74.43701584560952) bank139267 +139268 POINT(40.706057657970725 73.8948483463056) bank139268 +139269 POINT(41.48226126776153 73.75573399602419) bank139269 +139270 POINT(40.17918430989924 74.23943647395852) bank139270 +139271 POINT(40.29947170405752 74.20826113827565) bank139271 +139272 POINT(40.79472319625214 74.26586271055857) bank139272 +139273 POINT(40.96102657156518 74.49729433372548) bank139273 +139274 POINT(40.440280008079206 73.93068732313841) bank139274 +139275 POINT(41.522297658401776 73.9365773006851) bank139275 +139276 POINT(41.60012511495475 73.35997806133972) bank139276 +139277 POINT(41.206120379334 73.19520092530617) bank139277 +139278 POINT(40.17405313824691 74.63334451353366) bank139278 +139279 POINT(40.183486526382644 73.8906019846166) bank139279 +139280 POINT(40.443097517692436 73.52749727682948) bank139280 +139281 POINT(40.09582363373385 74.96978858688362) bank139281 +139282 POINT(41.634174749226716 74.86027281470227) bank139282 +139283 POINT(40.500376957220965 73.29607801032526) bank139283 +139284 POINT(41.162952908222294 74.51453571748202) bank139284 +139285 POINT(41.574076292004534 73.02799001749543) bank139285 +139286 POINT(40.4559570120245 74.79309553842097) bank139286 +139287 POINT(40.131502870516634 73.60982479045681) bank139287 +139288 POINT(40.35381409652052 74.9557020747729) bank139288 +139289 POINT(40.06421500334247 73.5121544028411) bank139289 +139290 POINT(39.741736685098076 73.23350069949713) bank139290 +139291 POINT(41.05938898462911 73.78571038361848) bank139291 +139292 POINT(40.180327044265084 74.08254675667703) bank139292 +139293 POINT(40.80958980981287 74.85982603447442) bank139293 +139294 POINT(41.219267361518874 74.02538955441054) bank139294 +139295 POINT(41.41127003442931 73.22196196240313) bank139295 +139296 POINT(40.15511109887433 74.76862797229015) bank139296 +139297 POINT(40.5401534741727 73.17529877370517) bank139297 +139298 POINT(40.23297282081016 73.28572240078371) bank139298 +139299 POINT(39.885502100664944 74.37817305674166) bank139299 +139300 POINT(41.52559153455163 74.075492135407) bank139300 +139301 POINT(40.70522911231422 73.78953254238536) bank139301 +139302 POINT(41.42046000639883 73.84424920233714) bank139302 +139303 POINT(39.71655988308792 74.59539039298011) bank139303 +139304 POINT(40.7899537366275 73.74488307588925) bank139304 +139305 POINT(40.65068788246633 73.40898772407621) bank139305 +139306 POINT(39.8798528283878 73.51900353882053) bank139306 +139307 POINT(40.39780578294188 73.30507830549998) bank139307 +139308 POINT(41.28574874283307 73.74760853885643) bank139308 +139309 POINT(41.6451436322199 74.27476964115934) bank139309 +139310 POINT(40.15130919020997 74.49866798677353) bank139310 +139311 POINT(40.89491626115919 74.01898232166441) bank139311 +139312 POINT(40.7639499866291 74.25860940665947) bank139312 +139313 POINT(39.81029544221384 73.52639594046954) bank139313 +139314 POINT(41.17497207468179 74.92188403583606) bank139314 +139315 POINT(40.43507604263199 73.90774321176326) bank139315 +139316 POINT(41.64466214501834 74.58496005657737) bank139316 +139317 POINT(41.116429744355955 74.24089485231042) bank139317 +139318 POINT(40.562229688971385 73.62842174061693) bank139318 +139319 POINT(39.912448473059186 73.95803419544212) bank139319 +139320 POINT(41.079224546037366 74.6710226954824) bank139320 +139321 POINT(41.052936189908536 73.27085003999561) bank139321 +139322 POINT(41.65377688802997 73.93770618386154) bank139322 +139323 POINT(40.92353466621146 74.59623859028456) bank139323 +139324 POINT(39.96577192984603 74.6435000741361) bank139324 +139325 POINT(40.66500949014958 74.68185766273145) bank139325 +139326 POINT(40.189798829413384 73.48468583383519) bank139326 +139327 POINT(40.138674050769254 73.75722204136454) bank139327 +139328 POINT(41.470204966548636 73.65208751542153) bank139328 +139329 POINT(40.69473759961562 73.66446396626182) bank139329 +139330 POINT(39.92518673789948 74.20742073905187) bank139330 +139331 POINT(41.09374120407405 74.81465244184726) bank139331 +139332 POINT(39.995725129308056 73.06469670153704) bank139332 +139333 POINT(40.436386606059735 73.92734284568301) bank139333 +139334 POINT(40.78000951989716 73.57711057420067) bank139334 +139335 POINT(40.23316648955722 73.60657755086545) bank139335 +139336 POINT(41.05942587610957 73.03606567973861) bank139336 +139337 POINT(41.48752674872995 74.94812314695466) bank139337 +139338 POINT(41.35921876940446 74.7125842775567) bank139338 +139339 POINT(40.16344918450685 74.60324885789595) bank139339 +139340 POINT(41.11161816625844 74.01054446734088) bank139340 +139341 POINT(41.258775334436386 74.54759276005025) bank139341 +139342 POINT(41.52163731576888 73.18563371190452) bank139342 +139343 POINT(41.18397298058639 74.42650328755974) bank139343 +139344 POINT(41.123432781895815 73.1125174779371) bank139344 +139345 POINT(41.20705786472102 74.64478507454135) bank139345 +139346 POINT(40.01775988208064 73.38937452965023) bank139346 +139347 POINT(41.4903165081592 74.85693788242203) bank139347 +139348 POINT(40.62423541962899 74.94270696470006) bank139348 +139349 POINT(40.633928379708834 73.17361671640073) bank139349 +139350 POINT(40.032644134741616 73.77728580323772) bank139350 +139351 POINT(40.05711212826946 73.24650043153365) bank139351 +139352 POINT(40.34226733430704 74.16411087990689) bank139352 +139353 POINT(39.930396289131735 74.08549717295494) bank139353 +139354 POINT(40.29149538276777 73.61421327936391) bank139354 +139355 POINT(40.5408580864987 73.45350717561752) bank139355 +139356 POINT(39.97886021481493 74.2471556873766) bank139356 +139357 POINT(40.40162919496994 74.54481010895394) bank139357 +139358 POINT(41.69014491042371 74.07589408500544) bank139358 +139359 POINT(40.2278638699269 74.8464205568531) bank139359 +139360 POINT(41.57292140792613 74.70062322732014) bank139360 +139361 POINT(41.44161742765933 74.73433212349275) bank139361 +139362 POINT(41.60555666522217 74.03027805116055) bank139362 +139363 POINT(41.251588073537356 74.99845834388788) bank139363 +139364 POINT(41.62478562241042 73.56423714036686) bank139364 +139365 POINT(40.18402878220628 74.12248698887255) bank139365 +139366 POINT(40.43125724806286 74.13905432234851) bank139366 +139367 POINT(40.44822353370744 74.35472884640942) bank139367 +139368 POINT(41.03751824761816 74.96525089304578) bank139368 +139369 POINT(40.89174621414954 73.18976529318414) bank139369 +139370 POINT(41.16122804264705 74.85769083475729) bank139370 +139371 POINT(39.74456221043258 73.42935325186062) bank139371 +139372 POINT(40.8522081420557 73.84711627804818) bank139372 +139373 POINT(40.53339826790887 73.72669593993801) bank139373 +139374 POINT(41.57052351132769 74.65909143697971) bank139374 +139375 POINT(40.81730968556661 73.26187731447399) bank139375 +139376 POINT(40.022608792167844 73.29774121139587) bank139376 +139377 POINT(40.968592654372 73.44877092727432) bank139377 +139378 POINT(40.6283298781648 73.5978302635504) bank139378 +139379 POINT(39.71315461019078 74.0054922611525) bank139379 +139380 POINT(40.13484309815168 74.67233117727693) bank139380 +139381 POINT(41.05456032366016 74.88358419271249) bank139381 +139382 POINT(39.95353781224948 73.66387739854846) bank139382 +139383 POINT(40.77186167422895 74.85794668166935) bank139383 +139384 POINT(40.566988304922845 73.58045912493877) bank139384 +139385 POINT(41.0064313214118 73.02288532788168) bank139385 +139386 POINT(40.02734934824086 73.27077864064258) bank139386 +139387 POINT(40.03659660750552 74.67145092851506) bank139387 +139388 POINT(41.42692753040975 74.44011029957343) bank139388 +139389 POINT(40.73511097783517 74.76462281135285) bank139389 +139390 POINT(41.671682447545756 73.95774845879069) bank139390 +139391 POINT(40.117759701898386 74.13259930402829) bank139391 +139392 POINT(41.46625185214084 74.66181886986402) bank139392 +139393 POINT(41.52673859744763 73.16581794315275) bank139393 +139394 POINT(40.87550959988982 73.00717706525268) bank139394 +139395 POINT(39.74719715205821 74.60180639147687) bank139395 +139396 POINT(40.53322489581411 73.93663561655609) bank139396 +139397 POINT(41.6192695197494 73.07217660748847) bank139397 +139398 POINT(40.20058273414668 74.3121123754926) bank139398 +139399 POINT(41.452643040748235 73.32944413326545) bank139399 +139400 POINT(39.89589180573909 73.91294588609368) bank139400 +139401 POINT(41.23170949801595 73.60557419132545) bank139401 +139402 POINT(41.15969997239612 73.02936700197041) bank139402 +139403 POINT(39.82349250835871 73.472550160855) bank139403 +139404 POINT(40.26754117001261 74.89145226125822) bank139404 +139405 POINT(40.55523073047951 74.1417528412325) bank139405 +139406 POINT(39.832519972730886 73.19129319195059) bank139406 +139407 POINT(41.56094029541717 74.38163834737078) bank139407 +139408 POINT(40.02358720640772 74.37435167244372) bank139408 +139409 POINT(41.05558544433016 73.03706459581633) bank139409 +139410 POINT(40.97206063644894 74.37890765554138) bank139410 +139411 POINT(41.513311834814324 74.5277880467622) bank139411 +139412 POINT(40.20815076899808 73.02776530814623) bank139412 +139413 POINT(40.82436730662576 74.33177361334307) bank139413 +139414 POINT(39.918469587077624 74.97533715168328) bank139414 +139415 POINT(40.95348130170879 73.57220241847635) bank139415 +139416 POINT(40.85050774021088 73.45615407289465) bank139416 +139417 POINT(41.10741607743535 74.27965600784482) bank139417 +139418 POINT(40.49992823186139 73.02527078033991) bank139418 +139419 POINT(39.80235603610323 74.46235356521403) bank139419 +139420 POINT(39.807660782040884 73.24492718542967) bank139420 +139421 POINT(40.23974303240309 73.14282976423982) bank139421 +139422 POINT(40.38014406454225 74.20381308090619) bank139422 +139423 POINT(40.45841316292593 74.60704236772239) bank139423 +139424 POINT(41.50263571989603 74.08452209375427) bank139424 +139425 POINT(40.79869542655259 73.86026333062419) bank139425 +139426 POINT(40.418758411516585 74.66331060293733) bank139426 +139427 POINT(41.628178824822406 74.77555297782446) bank139427 +139428 POINT(41.6359110988189 73.6014224364386) bank139428 +139429 POINT(39.90999748793565 73.77977729467983) bank139429 +139430 POINT(39.83304191167078 73.33480289857727) bank139430 +139431 POINT(40.29497820299302 74.81013711878522) bank139431 +139432 POINT(40.04503340090988 73.66655514147327) bank139432 +139433 POINT(40.3771845837657 73.87029423553628) bank139433 +139434 POINT(40.71554371727722 74.30232025286489) bank139434 +139435 POINT(41.52777492442808 73.36690106188668) bank139435 +139436 POINT(40.46081202482093 74.19303163238939) bank139436 +139437 POINT(39.76406078976493 74.16916823765193) bank139437 +139438 POINT(41.217429185304304 74.46627066186902) bank139438 +139439 POINT(40.16353918365444 73.77138843049914) bank139439 +139440 POINT(41.043889677554155 74.50963682326773) bank139440 +139441 POINT(41.62331511509344 74.0640511872493) bank139441 +139442 POINT(39.9453225525095 74.51979572406664) bank139442 +139443 POINT(39.87152496912571 74.3172585559461) bank139443 +139444 POINT(40.79411630631525 73.66174554549686) bank139444 +139445 POINT(40.6019373786141 74.46417037657432) bank139445 +139446 POINT(41.25841196349905 73.47411139661327) bank139446 +139447 POINT(41.66823307234768 73.14892936201633) bank139447 +139448 POINT(40.86187597034048 74.77097772531067) bank139448 +139449 POINT(40.92822154461457 74.46925215250504) bank139449 +139450 POINT(41.20352897954873 74.07058135116016) bank139450 +139451 POINT(41.340511756472786 74.86518963108719) bank139451 +139452 POINT(40.340491556464215 73.234601996495) bank139452 +139453 POINT(41.67439446147405 74.60420168803117) bank139453 +139454 POINT(39.97975126281286 74.39418327314759) bank139454 +139455 POINT(39.760632754782776 74.10842628914064) bank139455 +139456 POINT(41.562149465181655 74.95291463057002) bank139456 +139457 POINT(40.973220982296326 73.15348543541282) bank139457 +139458 POINT(41.395206663553985 74.03539084740422) bank139458 +139459 POINT(40.62111739528648 74.21862239081811) bank139459 +139460 POINT(41.12734515028084 73.4594382120514) bank139460 +139461 POINT(41.027162838397054 73.11878983305769) bank139461 +139462 POINT(41.06388823022417 73.80038734171471) bank139462 +139463 POINT(41.30821855025309 73.82259081144227) bank139463 +139464 POINT(39.98107673335066 74.06665048512112) bank139464 +139465 POINT(40.47030757254063 74.67289663792468) bank139465 +139466 POINT(39.90764224230692 74.49388306770295) bank139466 +139467 POINT(41.11444177506078 73.18265585842524) bank139467 +139468 POINT(41.46076396641038 74.90985963908787) bank139468 +139469 POINT(40.082408872795305 74.36874181559452) bank139469 +139470 POINT(40.0459778757337 74.9205559061524) bank139470 +139471 POINT(40.569029628326675 74.4595781057988) bank139471 +139472 POINT(40.428459882509244 73.24482632147506) bank139472 +139473 POINT(40.673788980996605 73.69240057663824) bank139473 +139474 POINT(41.00629025315358 74.37040499511909) bank139474 +139475 POINT(40.656213339158015 74.14697393945463) bank139475 +139476 POINT(40.43908268594125 74.92330445001875) bank139476 +139477 POINT(41.036865849670185 74.83864806206847) bank139477 +139478 POINT(41.2128155302061 73.90019417714903) bank139478 +139479 POINT(40.77138657820558 74.3256484851399) bank139479 +139480 POINT(41.07453980482789 73.76437235906108) bank139480 +139481 POINT(39.84436996305796 73.3459469354574) bank139481 +139482 POINT(41.146602689584924 73.19306954984944) bank139482 +139483 POINT(40.12858773427832 73.67841158349472) bank139483 +139484 POINT(40.52397499801529 74.57570922075828) bank139484 +139485 POINT(40.116761965200766 74.83418962351662) bank139485 +139486 POINT(39.79981425034762 74.5278205312095) bank139486 +139487 POINT(40.0422219190287 73.4630708555659) bank139487 +139488 POINT(40.44235857942868 74.20338342550563) bank139488 +139489 POINT(41.57443212595588 73.05215286733777) bank139489 +139490 POINT(40.31340839138375 74.53673506531489) bank139490 +139491 POINT(39.95254513333839 74.33279934649889) bank139491 +139492 POINT(40.846124689488526 74.44135577779522) bank139492 +139493 POINT(41.04184907700284 74.67276857574134) bank139493 +139494 POINT(39.92067954259233 73.11525589768083) bank139494 +139495 POINT(40.91477226439226 73.60274972281312) bank139495 +139496 POINT(40.868293774144156 73.92606948077476) bank139496 +139497 POINT(40.213210461544485 74.56430240926841) bank139497 +139498 POINT(41.518714618209856 74.46325395579821) bank139498 +139499 POINT(40.69126958713477 73.51085060451295) bank139499 +139500 POINT(41.1968812566548 74.66488366777578) bank139500 +139501 POINT(40.9984216903504 73.85385707465002) bank139501 +139502 POINT(41.431528779338656 75.00131026243407) bank139502 +139503 POINT(39.89990923584453 74.5030603654754) bank139503 +139504 POINT(40.311553362631834 74.59232425273828) bank139504 +139505 POINT(41.067494054838924 73.27190774662488) bank139505 +139506 POINT(40.25006797869001 73.22168418282577) bank139506 +139507 POINT(40.16476150516098 74.54778756895412) bank139507 +139508 POINT(39.96706374507011 73.95857865861127) bank139508 +139509 POINT(40.63307092773915 74.41346658938274) bank139509 +139510 POINT(39.970815954061756 74.54722626345531) bank139510 +139511 POINT(41.67334958694159 74.70539319348889) bank139511 +139512 POINT(41.63146469459865 74.26316383051565) bank139512 +139513 POINT(40.043422992523276 74.96019711463303) bank139513 +139514 POINT(40.65325532724821 74.56826266252048) bank139514 +139515 POINT(40.52874617399325 74.0265970623609) bank139515 +139516 POINT(41.63182353562068 74.66923082909504) bank139516 +139517 POINT(41.32487224056831 74.86692444538222) bank139517 +139518 POINT(40.97528044175432 74.76871904201556) bank139518 +139519 POINT(39.92985733160554 73.29844866595663) bank139519 +139520 POINT(40.12156364130113 74.02144204676051) bank139520 +139521 POINT(41.39369360900853 73.88986752277094) bank139521 +139522 POINT(41.02386542534353 73.43907576544878) bank139522 +139523 POINT(41.13084419059767 74.79761147485029) bank139523 +139524 POINT(41.71107849675916 73.66406143016279) bank139524 +139525 POINT(40.93514722788025 74.19439613733556) bank139525 +139526 POINT(39.987570939684886 73.93675748842246) bank139526 +139527 POINT(41.476287726997334 73.3373095689314) bank139527 +139528 POINT(40.088076188549785 74.33150322501642) bank139528 +139529 POINT(41.00706823990726 74.12048985078387) bank139529 +139530 POINT(40.72242769720728 74.2447490094673) bank139530 +139531 POINT(41.574864508628785 73.3526879925073) bank139531 +139532 POINT(41.20400837471515 74.63686964740297) bank139532 +139533 POINT(40.745753911704256 73.35204476489538) bank139533 +139534 POINT(39.88220061779282 73.2711515502726) bank139534 +139535 POINT(40.87465175358482 73.92803630622878) bank139535 +139536 POINT(41.30407897392362 74.2074750679695) bank139536 +139537 POINT(39.731059569096075 74.33602422050079) bank139537 +139538 POINT(41.36503449826095 73.3566380063763) bank139538 +139539 POINT(39.99407378732663 74.14812680440117) bank139539 +139540 POINT(40.2431553815546 74.16208063543272) bank139540 +139541 POINT(41.211694587485574 74.5267663158844) bank139541 +139542 POINT(41.188387018368466 73.19288255644145) bank139542 +139543 POINT(40.81524460357017 73.98641334741959) bank139543 +139544 POINT(40.09489909710686 74.42925942087183) bank139544 +139545 POINT(40.17032176679705 74.30233475998493) bank139545 +139546 POINT(40.35422728873846 73.11443826587812) bank139546 +139547 POINT(41.07281991309068 73.18918859167736) bank139547 +139548 POINT(39.939916648019114 74.92499344518423) bank139548 +139549 POINT(41.568028419036054 73.50104799485281) bank139549 +139550 POINT(40.04965349177166 73.84070686321375) bank139550 +139551 POINT(40.733118012676194 74.50507679785156) bank139551 +139552 POINT(40.63684939054445 73.59269637740053) bank139552 +139553 POINT(40.93623068656109 73.99895973327456) bank139553 +139554 POINT(41.18266883477516 73.48993281072191) bank139554 +139555 POINT(40.561446112383635 73.23662246720757) bank139555 +139556 POINT(41.41524841689474 74.05178563303943) bank139556 +139557 POINT(40.88796155001266 74.42320727580119) bank139557 +139558 POINT(40.39206263003333 73.29024228561055) bank139558 +139559 POINT(40.61252512928386 74.1537445416473) bank139559 +139560 POINT(40.17269674027602 73.62207193976485) bank139560 +139561 POINT(40.101515172690306 73.79998608376373) bank139561 +139562 POINT(40.370211509989154 73.94847823185025) bank139562 +139563 POINT(41.110999494289466 74.87372237918342) bank139563 +139564 POINT(39.924366674106196 73.7179138316435) bank139564 +139565 POINT(40.10495771815676 73.81130026210148) bank139565 +139566 POINT(40.944980445663035 74.69704402439189) bank139566 +139567 POINT(40.84118148521532 73.84466527799123) bank139567 +139568 POINT(40.83811779697746 74.32026747918711) bank139568 +139569 POINT(40.737353901579425 74.14563998544027) bank139569 +139570 POINT(39.89110406451878 74.89973024872442) bank139570 +139571 POINT(41.683830017145354 73.17646633298168) bank139571 +139572 POINT(40.01783453084806 73.5278407785436) bank139572 +139573 POINT(41.5062129121952 73.75941601581289) bank139573 +139574 POINT(40.749190348129076 73.64212353226596) bank139574 +139575 POINT(40.692445557534136 73.91210967360466) bank139575 +139576 POINT(41.592457199509454 74.63164784111508) bank139576 +139577 POINT(40.70316863983846 73.06755738601372) bank139577 +139578 POINT(40.75869828435073 73.70047803275541) bank139578 +139579 POINT(40.694970759267584 73.62658553030002) bank139579 +139580 POINT(40.875542706466724 73.91904583588489) bank139580 +139581 POINT(40.54196669330562 74.47408025745995) bank139581 +139582 POINT(41.567778388777384 73.29565843187477) bank139582 +139583 POINT(40.04900454200687 73.68232597285805) bank139583 +139584 POINT(41.08564572916384 74.13511099445134) bank139584 +139585 POINT(40.40605345420423 73.35129773877378) bank139585 +139586 POINT(40.974370557005436 74.37410273224664) bank139586 +139587 POINT(40.053543586357264 74.5798822830606) bank139587 +139588 POINT(41.366442108863 74.22067403728676) bank139588 +139589 POINT(41.01696269363494 74.95553880447962) bank139589 +139590 POINT(41.10972738389174 74.53122842381475) bank139590 +139591 POINT(39.847409720740465 73.92117399848246) bank139591 +139592 POINT(41.39998263554248 74.09149162185398) bank139592 +139593 POINT(40.48363103828843 73.90862254452576) bank139593 +139594 POINT(40.86339084120252 74.75175766355332) bank139594 +139595 POINT(41.708500545938016 73.25761058493522) bank139595 +139596 POINT(41.52526806129254 74.55024047157409) bank139596 +139597 POINT(41.28259550687767 73.56896520971496) bank139597 +139598 POINT(39.714903258909004 74.53244886394742) bank139598 +139599 POINT(40.84842160436275 74.25537801708218) bank139599 +139600 POINT(41.66248283502123 73.3030721891906) bank139600 +139601 POINT(40.09912760835558 74.1202901511032) bank139601 +139602 POINT(40.240069663827455 73.89473556833238) bank139602 +139603 POINT(41.29613872346559 73.8832901943703) bank139603 +139604 POINT(40.54374117148087 73.36379061150868) bank139604 +139605 POINT(40.891328363812185 74.84884900552046) bank139605 +139606 POINT(39.96107631379804 74.79123319134119) bank139606 +139607 POINT(40.45894713960948 73.15131371340257) bank139607 +139608 POINT(41.66018915641496 74.85301568655136) bank139608 +139609 POINT(40.78261246129413 74.26590124149585) bank139609 +139610 POINT(40.214406835674524 73.54221607387763) bank139610 +139611 POINT(39.76790223214892 73.91870901815795) bank139611 +139612 POINT(40.47504421255246 73.03206270141698) bank139612 +139613 POINT(40.198122667962345 74.7539592278412) bank139613 +139614 POINT(40.161868125074456 73.87903132921839) bank139614 +139615 POINT(40.59622448269923 73.4696577515422) bank139615 +139616 POINT(40.642098876277025 73.71564407117103) bank139616 +139617 POINT(39.777635414654625 73.5775118751276) bank139617 +139618 POINT(40.63949705827709 74.52900340979878) bank139618 +139619 POINT(41.36195198142539 73.72625212187425) bank139619 +139620 POINT(40.903113204486544 74.56119755214537) bank139620 +139621 POINT(40.26075612923221 74.32062174352673) bank139621 +139622 POINT(41.037828210040566 73.32403255436721) bank139622 +139623 POINT(41.23077550182049 74.53048373970239) bank139623 +139624 POINT(39.729894448761925 73.23696701138977) bank139624 +139625 POINT(41.054642116721105 74.18873469904678) bank139625 +139626 POINT(40.500882537720344 74.49798938897469) bank139626 +139627 POINT(40.43407807866912 73.38704262226666) bank139627 +139628 POINT(39.86601620229604 74.97425764995627) bank139628 +139629 POINT(41.141760002198744 74.29482666263931) bank139629 +139630 POINT(41.15497097700247 74.16246861370499) bank139630 +139631 POINT(39.86185439658006 74.9501178350578) bank139631 +139632 POINT(40.630170518588145 74.9416327915897) bank139632 +139633 POINT(40.127520072534935 73.59649901205617) bank139633 +139634 POINT(40.54450723643246 73.08872436203123) bank139634 +139635 POINT(41.36919837974915 74.06463841937034) bank139635 +139636 POINT(40.216810478877335 74.89841573195876) bank139636 +139637 POINT(39.92569108446588 74.71487317510578) bank139637 +139638 POINT(40.078587724345816 74.47518742989357) bank139638 +139639 POINT(40.57333279239076 74.51251417174306) bank139639 +139640 POINT(41.004296348429214 73.01003603183763) bank139640 +139641 POINT(39.77958434795474 73.92313880152636) bank139641 +139642 POINT(40.348348133536234 73.21709946168835) bank139642 +139643 POINT(40.62050317544964 74.84270342617808) bank139643 +139644 POINT(40.85028395058359 74.74700362003627) bank139644 +139645 POINT(40.572957936886226 74.08695972530847) bank139645 +139646 POINT(41.392969770118896 74.39130792193467) bank139646 +139647 POINT(40.164349421227726 74.68131047954694) bank139647 +139648 POINT(39.9822906250886 73.13988700406037) bank139648 +139649 POINT(41.663481360887005 73.8570379703091) bank139649 +139650 POINT(39.9702340166665 74.56654761261602) bank139650 +139651 POINT(40.96306064289612 73.48997934145154) bank139651 +139652 POINT(40.31172404008992 73.05708410829534) bank139652 +139653 POINT(40.98131697081043 74.65897330948918) bank139653 +139654 POINT(41.404258588216976 74.31653232682005) bank139654 +139655 POINT(40.58420160827805 74.09740736125251) bank139655 +139656 POINT(40.22103278377354 74.35964903317895) bank139656 +139657 POINT(40.24122861204877 74.7724876199621) bank139657 +139658 POINT(41.63249575781328 73.25971223551073) bank139658 +139659 POINT(40.73576267104263 73.96454174638549) bank139659 +139660 POINT(40.31727927195296 73.2260391723377) bank139660 +139661 POINT(40.92205373227598 74.9000877393272) bank139661 +139662 POINT(41.06482240241016 74.00791076422266) bank139662 +139663 POINT(41.30148839112164 74.20975796582798) bank139663 +139664 POINT(40.56029912370776 73.12715984594517) bank139664 +139665 POINT(40.113152238199476 74.86308641181694) bank139665 +139666 POINT(40.12807311803255 74.51757676612033) bank139666 +139667 POINT(40.068695117393666 74.416244011057) bank139667 +139668 POINT(40.34192045231249 74.17081405792658) bank139668 +139669 POINT(41.25153559658386 74.62481166180412) bank139669 +139670 POINT(40.01429662433046 73.0278638765822) bank139670 +139671 POINT(39.894375700862554 73.73536430085659) bank139671 +139672 POINT(41.6417081945623 74.17432720325193) bank139672 +139673 POINT(41.37915567255067 74.34907510459718) bank139673 +139674 POINT(41.16941643198863 73.67933676396053) bank139674 +139675 POINT(41.30673689276372 74.98885993700017) bank139675 +139676 POINT(40.89216149797327 74.5178473015691) bank139676 +139677 POINT(40.371220938371586 73.26192077643843) bank139677 +139678 POINT(40.18712739319454 74.26724759506703) bank139678 +139679 POINT(39.87147749742689 73.49951305545235) bank139679 +139680 POINT(41.47229395364773 74.30605491637483) bank139680 +139681 POINT(40.93418743266042 74.23354509196515) bank139681 +139682 POINT(41.542241310645416 74.65785510178857) bank139682 +139683 POINT(39.768716052554126 74.06861691231325) bank139683 +139684 POINT(41.61416388330935 74.40544266569103) bank139684 +139685 POINT(39.76058031583841 74.46349695893473) bank139685 +139686 POINT(40.16476589416784 74.95889430331238) bank139686 +139687 POINT(40.71029010303708 74.81148983913111) bank139687 +139688 POINT(40.46724195598467 74.11514824326943) bank139688 +139689 POINT(41.68148120325994 74.41251311947613) bank139689 +139690 POINT(41.35608687750852 74.81041760369035) bank139690 +139691 POINT(40.829472865168775 74.51519624683341) bank139691 +139692 POINT(41.44936658955407 74.90301294011583) bank139692 +139693 POINT(41.32697531668167 74.49364063947156) bank139693 +139694 POINT(40.0527410349984 74.8263223979693) bank139694 +139695 POINT(40.39465553172657 74.79932491927676) bank139695 +139696 POINT(40.035498209114614 73.17025940273534) bank139696 +139697 POINT(40.95442212481263 74.02320426535178) bank139697 +139698 POINT(41.673980084469655 73.09889715648927) bank139698 +139699 POINT(41.703984179475924 74.20930875282646) bank139699 +139700 POINT(39.897299282956816 73.27878104097404) bank139700 +139701 POINT(40.31474033621049 74.84822816276936) bank139701 +139702 POINT(41.02397259919099 73.93527574069911) bank139702 +139703 POINT(39.93472408586405 73.26077412748913) bank139703 +139704 POINT(40.953136023864424 74.8291375049633) bank139704 +139705 POINT(39.74704857712477 74.02640511817806) bank139705 +139706 POINT(40.94012798035964 74.46400623423003) bank139706 +139707 POINT(40.27479506638363 74.0988862068155) bank139707 +139708 POINT(40.789159772520584 73.22682355287868) bank139708 +139709 POINT(39.84815572118904 74.10142936179383) bank139709 +139710 POINT(41.06064206904645 74.74671533024456) bank139710 +139711 POINT(41.506783300312435 73.4166025104463) bank139711 +139712 POINT(41.46110061343601 74.41795335423984) bank139712 +139713 POINT(40.2773778133713 74.2767871079364) bank139713 +139714 POINT(40.81701607809247 74.63489197007985) bank139714 +139715 POINT(40.322926463083 75.00512398918555) bank139715 +139716 POINT(39.9752396252191 74.06369935316873) bank139716 +139717 POINT(41.560363243324446 73.85123663336748) bank139717 +139718 POINT(39.87085010584879 74.47532767105166) bank139718 +139719 POINT(41.511949224945454 74.95405249802681) bank139719 +139720 POINT(40.87060131948549 73.57681415469483) bank139720 +139721 POINT(41.68936645990936 73.93795884004194) bank139721 +139722 POINT(40.89581465722434 73.25402839315278) bank139722 +139723 POINT(41.66675196703204 74.29439823320858) bank139723 +139724 POINT(40.48655617047186 74.46746609049718) bank139724 +139725 POINT(39.818248356136564 74.98880431292261) bank139725 +139726 POINT(40.96680176275871 74.28047118713886) bank139726 +139727 POINT(41.48562386603734 74.17814951977138) bank139727 +139728 POINT(40.43371056383909 74.94169978532828) bank139728 +139729 POINT(41.63520268573822 73.9047650449868) bank139729 +139730 POINT(40.78273631071985 74.31030661241677) bank139730 +139731 POINT(40.91056307635775 73.53024669153285) bank139731 +139732 POINT(39.883284243083835 74.73752928571767) bank139732 +139733 POINT(40.399654962838234 74.27850142155056) bank139733 +139734 POINT(41.00547309109151 74.4155624485212) bank139734 +139735 POINT(40.633556063269495 73.5653686449516) bank139735 +139736 POINT(40.05742148041827 74.61426868952887) bank139736 +139737 POINT(41.564399142147366 74.81724776879055) bank139737 +139738 POINT(41.67076521045006 74.27854198786278) bank139738 +139739 POINT(40.26510972244475 74.01331462482548) bank139739 +139740 POINT(41.17951925940929 73.83195562269468) bank139740 +139741 POINT(39.97050694969368 74.74236402684548) bank139741 +139742 POINT(40.31684048551375 73.96967609302513) bank139742 +139743 POINT(40.18897568981301 73.95327486649298) bank139743 +139744 POINT(40.95465918638602 73.7624269792504) bank139744 +139745 POINT(41.60517328688757 73.39962425856645) bank139745 +139746 POINT(41.267064335311616 74.83704389588682) bank139746 +139747 POINT(40.695508080414086 73.24320702546147) bank139747 +139748 POINT(40.02585764842947 73.2875808940384) bank139748 +139749 POINT(40.245299209572686 74.33394947266636) bank139749 +139750 POINT(40.3900245314019 73.15656502784985) bank139750 +139751 POINT(40.96940198959551 73.98298842948944) bank139751 +139752 POINT(40.67509478089876 74.80295697447276) bank139752 +139753 POINT(39.76162305164029 73.27583396746493) bank139753 +139754 POINT(39.953782674820246 74.5851043070428) bank139754 +139755 POINT(39.952326761176494 73.13835758498259) bank139755 +139756 POINT(41.45423208411586 73.44783384647937) bank139756 +139757 POINT(41.20895082812688 74.15119422921973) bank139757 +139758 POINT(40.05100357958963 74.52331581304702) bank139758 +139759 POINT(40.42739073475295 74.38623061689849) bank139759 +139760 POINT(41.47472347229342 73.20331864613166) bank139760 +139761 POINT(39.92044257023577 74.71661805917745) bank139761 +139762 POINT(41.43309270897632 74.38348953370739) bank139762 +139763 POINT(41.43989834341262 73.74757132737213) bank139763 +139764 POINT(40.45849745604793 74.4002629768468) bank139764 +139765 POINT(40.38765767306431 74.20388303740768) bank139765 +139766 POINT(41.4237820105161 73.20138743495971) bank139766 +139767 POINT(40.42407649910371 74.4869797121336) bank139767 +139768 POINT(41.5169646670292 73.3709021643526) bank139768 +139769 POINT(41.0407746841538 74.99489836662973) bank139769 +139770 POINT(39.90052845364099 73.2588287030276) bank139770 +139771 POINT(40.28816000149834 74.8717878678916) bank139771 +139772 POINT(41.60543183362369 74.77757984561423) bank139772 +139773 POINT(41.701750849987825 73.91981208106863) bank139773 +139774 POINT(40.976656969767994 74.72692954428763) bank139774 +139775 POINT(41.69023842865565 73.68194409150692) bank139775 +139776 POINT(41.6305114733577 73.89938725402384) bank139776 +139777 POINT(41.450592988037144 74.38209988720179) bank139777 +139778 POINT(41.476485434841585 74.4968114025441) bank139778 +139779 POINT(41.45164565820111 74.71690733925098) bank139779 +139780 POINT(41.36456312842826 73.63820664631412) bank139780 +139781 POINT(41.26527902514049 73.72109971866283) bank139781 +139782 POINT(41.61658570242095 74.75618776118228) bank139782 +139783 POINT(41.134273893701504 73.26933764752948) bank139783 +139784 POINT(41.255773134541336 74.82587176007807) bank139784 +139785 POINT(39.846076041178335 73.89039946660654) bank139785 +139786 POINT(40.62073483908016 74.12907807714076) bank139786 +139787 POINT(41.116714390116115 74.49407855997083) bank139787 +139788 POINT(39.98380296849782 74.82405063201936) bank139788 +139789 POINT(41.68928177164283 74.64731786284402) bank139789 +139790 POINT(41.55015266294004 74.09080738627023) bank139790 +139791 POINT(41.10307943777604 73.8638746508642) bank139791 +139792 POINT(39.99706846681026 74.78475512196209) bank139792 +139793 POINT(41.328915488616744 74.28082265735623) bank139793 +139794 POINT(41.17332101524305 74.68629858266637) bank139794 +139795 POINT(39.80951715845975 73.34753711624268) bank139795 +139796 POINT(41.56217123620002 73.77632727099372) bank139796 +139797 POINT(39.744429070934 74.97802947162015) bank139797 +139798 POINT(41.04868393122898 74.36887788417633) bank139798 +139799 POINT(41.31709546980424 74.80006489707317) bank139799 +139800 POINT(40.65977987925533 74.7118971909585) bank139800 +139801 POINT(39.806046290110245 73.61967601232581) bank139801 +139802 POINT(40.86827265435136 73.28473655086106) bank139802 +139803 POINT(39.89720364611769 73.15436107541504) bank139803 +139804 POINT(40.40216822017169 73.85260595444419) bank139804 +139805 POINT(41.21414994370171 74.36762278624403) bank139805 +139806 POINT(39.894088546835135 74.15977350906924) bank139806 +139807 POINT(40.49553400027217 73.29465746415069) bank139807 +139808 POINT(39.81100100639825 73.6822029493685) bank139808 +139809 POINT(41.269582942908805 73.88800763897396) bank139809 +139810 POINT(40.51226767545397 73.14578080100873) bank139810 +139811 POINT(40.56742168624024 74.80681765608283) bank139811 +139812 POINT(41.11791081410642 74.3529474399697) bank139812 +139813 POINT(40.163707543454855 73.38652134947259) bank139813 +139814 POINT(40.04524831992564 73.16405618688758) bank139814 +139815 POINT(41.31605989249237 74.09393313377929) bank139815 +139816 POINT(40.43667477987988 73.39015344014454) bank139816 +139817 POINT(41.01203526550772 73.28296063259971) bank139817 +139818 POINT(41.46045620381934 73.72505941207238) bank139818 +139819 POINT(41.61985054851022 73.03393979727132) bank139819 +139820 POINT(41.335144075053115 74.43552854419245) bank139820 +139821 POINT(41.64073496486381 74.20905221563179) bank139821 +139822 POINT(40.14432157366068 74.05000430506935) bank139822 +139823 POINT(40.58680391270975 73.55290023022012) bank139823 +139824 POINT(40.39058360417727 73.12161286604578) bank139824 +139825 POINT(41.676894199564714 74.3318589849847) bank139825 +139826 POINT(39.86218459618321 74.99381684648391) bank139826 +139827 POINT(40.5148396379028 74.97115873333622) bank139827 +139828 POINT(41.276565461150625 74.84164388678161) bank139828 +139829 POINT(39.887675102603886 73.3997976844905) bank139829 +139830 POINT(41.08313136853927 74.53467894749198) bank139830 +139831 POINT(41.652519082221275 73.54260335985619) bank139831 +139832 POINT(40.86456268380326 74.52499379896373) bank139832 +139833 POINT(40.04886096100738 74.3229242313525) bank139833 +139834 POINT(39.76020004640191 73.94595155817868) bank139834 +139835 POINT(40.952384486035555 73.61751307386497) bank139835 +139836 POINT(41.638729232508446 74.91528492251754) bank139836 +139837 POINT(41.07478943704047 73.84318175670039) bank139837 +139838 POINT(41.01249134269192 74.02937243948861) bank139838 +139839 POINT(40.28073212022488 74.99362175633166) bank139839 +139840 POINT(40.83944879178125 74.31859431250982) bank139840 +139841 POINT(40.539535327731755 74.11054472167658) bank139841 +139842 POINT(41.250003131124 73.38996278826411) bank139842 +139843 POINT(40.16126742885933 73.9586359628192) bank139843 +139844 POINT(39.87973591043234 74.59460000643264) bank139844 +139845 POINT(40.14547133129652 73.64049918862796) bank139845 +139846 POINT(40.54760674771431 73.57321308195885) bank139846 +139847 POINT(40.53921232338182 74.93333927694532) bank139847 +139848 POINT(41.30262808564904 73.1579852789155) bank139848 +139849 POINT(41.419793750104446 74.27849084481043) bank139849 +139850 POINT(41.31544672514608 73.94620528527118) bank139850 +139851 POINT(39.80956502714024 74.7943985553037) bank139851 +139852 POINT(41.50129441344453 74.1675004580934) bank139852 +139853 POINT(40.14361280880013 73.0696684287696) bank139853 +139854 POINT(40.1019540238012 73.73277566537658) bank139854 +139855 POINT(40.08910713460766 74.88469675782191) bank139855 +139856 POINT(41.19941917148268 74.47315194546016) bank139856 +139857 POINT(40.694003897142544 73.21594661605148) bank139857 +139858 POINT(41.09911952829554 73.76014203236966) bank139858 +139859 POINT(40.746205275632356 74.24235827314851) bank139859 +139860 POINT(39.97842317384367 73.51388567188552) bank139860 +139861 POINT(40.60145450152302 73.33735440968216) bank139861 +139862 POINT(40.83220093237386 74.56908373016788) bank139862 +139863 POINT(41.08404645507119 73.76054322965142) bank139863 +139864 POINT(41.184163112254254 74.04396655627677) bank139864 +139865 POINT(41.200837812579174 73.54122671205924) bank139865 +139866 POINT(40.465344748679414 74.9919459336228) bank139866 +139867 POINT(39.83741638598554 73.03748582008811) bank139867 +139868 POINT(39.90250424069033 73.31743885478451) bank139868 +139869 POINT(40.498150006984254 74.41860232142574) bank139869 +139870 POINT(39.94326331498544 74.04818474477145) bank139870 +139871 POINT(40.142160706921516 73.36325193160029) bank139871 +139872 POINT(41.61498871381761 74.46596270269178) bank139872 +139873 POINT(40.284266536239805 74.13060040084135) bank139873 +139874 POINT(40.777802615614995 73.16314411003806) bank139874 +139875 POINT(40.26799408001444 74.22767907377039) bank139875 +139876 POINT(40.15038622506351 73.87745124367083) bank139876 +139877 POINT(41.239644827936935 74.32038904769526) bank139877 +139878 POINT(40.92006720717916 74.64764721154177) bank139878 +139879 POINT(39.97344537744563 74.0351501474246) bank139879 +139880 POINT(40.315042501041084 74.60999245983571) bank139880 +139881 POINT(40.90793773959336 73.02501562219283) bank139881 +139882 POINT(41.34749914122564 74.4603434829227) bank139882 +139883 POINT(40.08894886774738 73.06082133381544) bank139883 +139884 POINT(40.560855518299526 74.95313573618824) bank139884 +139885 POINT(39.899971392975985 73.40865365598214) bank139885 +139886 POINT(41.56724437809664 73.13677057196546) bank139886 +139887 POINT(41.06005090849866 74.83522187859217) bank139887 +139888 POINT(40.623629341738614 74.44784598139087) bank139888 +139889 POINT(41.213180922138505 73.27743992663011) bank139889 +139890 POINT(40.04140342172782 73.20338898982274) bank139890 +139891 POINT(40.450818133470044 73.60527916913476) bank139891 +139892 POINT(41.005303735233795 74.60563859660441) bank139892 +139893 POINT(40.93286556096017 73.82255050976924) bank139893 +139894 POINT(40.12779450469291 73.14207462035986) bank139894 +139895 POINT(41.21639143855339 74.01470202110066) bank139895 +139896 POINT(41.02463196775455 73.13763918592083) bank139896 +139897 POINT(40.266646232843314 73.37855446384492) bank139897 +139898 POINT(40.823156983929394 73.7453934189266) bank139898 +139899 POINT(40.95049461800979 73.73189310877534) bank139899 +139900 POINT(41.12976297671001 74.8202267751139) bank139900 +139901 POINT(40.265503090505646 74.63811484685104) bank139901 +139902 POINT(41.3635197661485 74.925249394067) bank139902 +139903 POINT(40.264568106644106 73.13553413543698) bank139903 +139904 POINT(41.442853452758214 74.9260066373769) bank139904 +139905 POINT(40.750157431994445 74.15360235631898) bank139905 +139906 POINT(41.006174464497185 73.00649350266588) bank139906 +139907 POINT(40.330221000242105 74.37551850681689) bank139907 +139908 POINT(40.33239893012029 74.8263461272168) bank139908 +139909 POINT(40.01873781365236 74.75594036944265) bank139909 +139910 POINT(40.06740560123241 73.08856952291508) bank139910 +139911 POINT(41.57318281813802 73.09023615776664) bank139911 +139912 POINT(41.461176340548896 74.78325397245102) bank139912 +139913 POINT(39.99939881193606 73.65382184134182) bank139913 +139914 POINT(41.009213063527014 73.35580258620372) bank139914 +139915 POINT(40.43822509229614 73.3499533462066) bank139915 +139916 POINT(41.04294143413633 74.42494269375217) bank139916 +139917 POINT(41.45165554959756 74.44623805206783) bank139917 +139918 POINT(41.134302736979954 74.32310859193822) bank139918 +139919 POINT(41.18879919423964 74.2932405103782) bank139919 +139920 POINT(40.437515936166825 73.28798375604661) bank139920 +139921 POINT(39.73141300245047 73.75357799427744) bank139921 +139922 POINT(40.10547369675364 74.79936684757544) bank139922 +139923 POINT(41.137421131671175 74.42429123339438) bank139923 +139924 POINT(40.33462637296697 74.80036578605628) bank139924 +139925 POINT(41.397613730352155 74.57699896307341) bank139925 +139926 POINT(41.42354157405755 74.79196353386001) bank139926 +139927 POINT(41.52588088777161 74.19085629514463) bank139927 +139928 POINT(40.26152389857135 74.01874032062746) bank139928 +139929 POINT(41.643402091710186 74.36732295247126) bank139929 +139930 POINT(41.2917996584937 74.25584763335976) bank139930 +139931 POINT(39.87760205405366 73.78933480977138) bank139931 +139932 POINT(40.30363649464879 74.90629725808115) bank139932 +139933 POINT(40.63373641446159 73.06677973230136) bank139933 +139934 POINT(40.33540136471487 74.10339075643434) bank139934 +139935 POINT(41.67441918825329 73.1326281675746) bank139935 +139936 POINT(40.984685670500184 73.31951037922505) bank139936 +139937 POINT(41.50394639415547 74.46036380757825) bank139937 +139938 POINT(40.88782313715289 73.8265389234012) bank139938 +139939 POINT(41.14329516416734 74.42671869004714) bank139939 +139940 POINT(40.69472734356742 73.49557657542209) bank139940 +139941 POINT(41.00893075390655 73.52966855254348) bank139941 +139942 POINT(40.906728842996785 73.9144017809468) bank139942 +139943 POINT(40.93954404049901 73.90551129901787) bank139943 +139944 POINT(40.4900389718368 74.65752792241246) bank139944 +139945 POINT(41.419389031953386 74.06854997342535) bank139945 +139946 POINT(40.80518039626154 74.57082750244449) bank139946 +139947 POINT(41.309378371148114 74.98651493696693) bank139947 +139948 POINT(41.68742819425089 73.71320117804248) bank139948 +139949 POINT(40.117769384813506 74.30377863377127) bank139949 +139950 POINT(41.6788852657632 73.03304504146102) bank139950 +139951 POINT(41.109983926945816 74.64471715438955) bank139951 +139952 POINT(39.72708898290044 74.22858852212804) bank139952 +139953 POINT(40.35844511607647 74.50244896395655) bank139953 +139954 POINT(40.837046477241536 74.1271311311938) bank139954 +139955 POINT(40.8102258414961 74.76988561336574) bank139955 +139956 POINT(40.60749779966078 74.13083744978059) bank139956 +139957 POINT(40.4144526223522 74.36640905749947) bank139957 +139958 POINT(40.28821615322574 74.379099433053) bank139958 +139959 POINT(40.58596521286419 73.7684590972918) bank139959 +139960 POINT(40.30159610920116 74.31304483001303) bank139960 +139961 POINT(41.53277356898592 74.5321705703484) bank139961 +139962 POINT(39.915012558307 73.78496888452402) bank139962 +139963 POINT(41.33954441768313 74.56635533480706) bank139963 +139964 POINT(41.14650445017728 74.6259878413771) bank139964 +139965 POINT(39.881033148440565 74.0229224321855) bank139965 +139966 POINT(40.463725728528296 73.60872994032292) bank139966 +139967 POINT(40.2672978626238 74.44279307054506) bank139967 +139968 POINT(40.00784876298906 73.8524581846778) bank139968 +139969 POINT(40.51291285602031 73.99707711424642) bank139969 +139970 POINT(40.53701829338306 73.0636434436495) bank139970 +139971 POINT(40.646199127428 74.37525828406022) bank139971 +139972 POINT(40.113663140143046 73.30226528654269) bank139972 +139973 POINT(39.73142403660843 73.95302609135352) bank139973 +139974 POINT(41.22693722055185 74.01472208158948) bank139974 +139975 POINT(40.09747370652002 73.5539770523088) bank139975 +139976 POINT(41.44222290727648 73.77075122418303) bank139976 +139977 POINT(40.22070698350721 74.05079297555287) bank139977 +139978 POINT(40.30675435443231 74.0767155424172) bank139978 +139979 POINT(39.768005566035605 74.68070159187124) bank139979 +139980 POINT(40.577674138252306 74.57429092076038) bank139980 +139981 POINT(41.360658022514976 74.65358542721268) bank139981 +139982 POINT(40.44173277147653 74.21781605267694) bank139982 +139983 POINT(40.402396271869904 74.10794379017479) bank139983 +139984 POINT(41.687361578378 73.67977732960901) bank139984 +139985 POINT(40.88445842208815 73.75623085760158) bank139985 +139986 POINT(40.230181459122775 73.821659519044) bank139986 +139987 POINT(40.08527476927123 73.21969856920663) bank139987 +139988 POINT(41.58412235782175 73.7968032989165) bank139988 +139989 POINT(40.41027296991641 73.83088122963831) bank139989 +139990 POINT(41.00137230928793 74.26327287335997) bank139990 +139991 POINT(41.404693113951495 73.84968905840098) bank139991 +139992 POINT(40.50852582384542 74.4446643619057) bank139992 +139993 POINT(41.414311492900396 74.50144380849747) bank139993 +139994 POINT(39.75673840412681 73.48146073901812) bank139994 +139995 POINT(41.02342617718419 74.01718430608791) bank139995 +139996 POINT(41.35551624835984 73.51479747955828) bank139996 +139997 POINT(40.014516760736086 73.86711476593203) bank139997 +139998 POINT(40.383447603075396 73.76722515754506) bank139998 +139999 POINT(41.431873448545716 73.09002735586151) bank139999 +140000 POINT(41.3312083045388 74.19868353627508) bank140000 +140001 POINT(41.135157686070656 73.82767123807567) bank140001 +140002 POINT(41.00782072468167 74.14212295102479) bank140002 +140003 POINT(40.29320809655316 73.52412290593534) bank140003 +140004 POINT(40.084802356554135 73.79138510408514) bank140004 +140005 POINT(40.07032696882151 74.86303419401426) bank140005 +140006 POINT(40.511883378980286 74.67250664204013) bank140006 +140007 POINT(41.159804557448794 74.0425258909364) bank140007 +140008 POINT(39.941846095748076 73.35377213196243) bank140008 +140009 POINT(40.52328993185037 74.5709098145808) bank140009 +140010 POINT(40.00551086453203 73.4544764873395) bank140010 +140011 POINT(40.08017083331206 73.59018660138324) bank140011 +140012 POINT(40.81240331980479 73.79173585657846) bank140012 +140013 POINT(40.07887578374266 74.9874420666441) bank140013 +140014 POINT(40.42612167213512 73.77356279115436) bank140014 +140015 POINT(39.794047802306395 73.98413385458103) bank140015 +140016 POINT(40.0595083843677 74.72535061251021) bank140016 +140017 POINT(41.13752122855085 74.4813324688016) bank140017 +140018 POINT(40.76224546100002 73.66647185442407) bank140018 +140019 POINT(41.14192330968958 74.21078881604465) bank140019 +140020 POINT(41.62840822224229 73.30179739286861) bank140020 +140021 POINT(41.228971750110105 73.67828245064914) bank140021 +140022 POINT(39.936579478947294 74.57937352367757) bank140022 +140023 POINT(41.41824996332769 74.32440543460265) bank140023 +140024 POINT(41.424138997658005 73.5757010451017) bank140024 +140025 POINT(40.85657549886984 73.34705982928085) bank140025 +140026 POINT(39.78722787711269 73.97272494986157) bank140026 +140027 POINT(41.41204023873395 74.05022720104175) bank140027 +140028 POINT(40.27110091426556 74.34799157230941) bank140028 +140029 POINT(40.94190204365086 73.58157922885874) bank140029 +140030 POINT(41.21518119877435 73.2486622982142) bank140030 +140031 POINT(41.08224003313975 73.48863891541046) bank140031 +140032 POINT(40.664930971643216 74.69709384864454) bank140032 +140033 POINT(40.90348965805411 73.5783476885368) bank140033 +140034 POINT(40.502437970907614 73.7276913972191) bank140034 +140035 POINT(40.93847312139791 73.07448823024953) bank140035 +140036 POINT(40.17305715391201 74.23009605703614) bank140036 +140037 POINT(40.639345633457566 74.2732800629525) bank140037 +140038 POINT(39.72598394572474 73.86957813122685) bank140038 +140039 POINT(40.311049464268464 73.47356919286145) bank140039 +140040 POINT(41.59291863144999 74.8097057738638) bank140040 +140041 POINT(39.88427388796159 73.88736947400574) bank140041 +140042 POINT(39.85588812414393 73.15018794351852) bank140042 +140043 POINT(40.90965069050684 73.139220225423) bank140043 +140044 POINT(41.26588244721767 73.29588255470338) bank140044 +140045 POINT(40.21114716738542 74.29537523470239) bank140045 +140046 POINT(40.22096010131453 73.2627277315584) bank140046 +140047 POINT(40.40290761132297 73.13264376865956) bank140047 +140048 POINT(40.43498631644212 74.78901471895044) bank140048 +140049 POINT(41.27369358734965 73.9722392187838) bank140049 +140050 POINT(40.48532943641155 74.70614301181362) bank140050 +140051 POINT(41.33522742698824 73.66792176674734) bank140051 +140052 POINT(41.13963205147094 74.45798212818084) bank140052 +140053 POINT(41.41318115127781 74.20561943852725) bank140053 +140054 POINT(39.92534872683358 73.33629611432134) bank140054 +140055 POINT(40.89333416061928 74.96588086725072) bank140055 +140056 POINT(40.53332458589519 73.33393607851573) bank140056 +140057 POINT(40.01738686361999 74.91307885742859) bank140057 +140058 POINT(39.82547374407798 74.49141393622183) bank140058 +140059 POINT(40.0063680483375 73.46506001784863) bank140059 +140060 POINT(41.26886810758977 73.83371683846987) bank140060 +140061 POINT(41.32566216026292 74.13579345617937) bank140061 +140062 POINT(39.94525650739658 74.06691110979807) bank140062 +140063 POINT(40.79970569950732 74.64566812026266) bank140063 +140064 POINT(41.34293308924167 73.68169615129347) bank140064 +140065 POINT(41.50138342960056 74.79188789938246) bank140065 +140066 POINT(40.559057184140244 74.35651527859866) bank140066 +140067 POINT(40.941566354332636 74.61447085419603) bank140067 +140068 POINT(40.912725297750235 74.18743671300658) bank140068 +140069 POINT(40.25805018907627 73.36480888277816) bank140069 +140070 POINT(40.10734942810889 74.50662707582516) bank140070 +140071 POINT(41.3398441986969 73.49774743824828) bank140071 +140072 POINT(40.932697458056495 74.101797119638) bank140072 +140073 POINT(40.84142518971348 73.9241401257391) bank140073 +140074 POINT(40.53912872557734 73.91943494414438) bank140074 +140075 POINT(41.05073519709004 74.25865344322433) bank140075 +140076 POINT(40.34780448773235 74.223044367605) bank140076 +140077 POINT(41.66832659144647 73.77203930268587) bank140077 +140078 POINT(41.22757874318129 73.18585275297734) bank140078 +140079 POINT(39.950364434477926 73.93699655890549) bank140079 +140080 POINT(40.44015910549761 73.45520611588005) bank140080 +140081 POINT(40.849192105554486 74.56474152068527) bank140081 +140082 POINT(41.618003706338214 73.06564877335244) bank140082 +140083 POINT(41.14426241007631 73.60857940765226) bank140083 +140084 POINT(39.904376271206935 73.73338370942128) bank140084 +140085 POINT(41.26285557475644 73.86780797129259) bank140085 +140086 POINT(39.71577828589165 74.03227170941405) bank140086 +140087 POINT(40.717784119954366 74.88044652448124) bank140087 +140088 POINT(40.7519049768774 74.39439088908969) bank140088 +140089 POINT(40.15453503171122 74.45938142991535) bank140089 +140090 POINT(39.76530207704703 73.62009345747866) bank140090 +140091 POINT(40.31638816383501 73.94690431596358) bank140091 +140092 POINT(40.01322609708881 73.14137922947786) bank140092 +140093 POINT(40.77934617422036 73.60900323253932) bank140093 +140094 POINT(40.95246577178471 73.58845265615439) bank140094 +140095 POINT(41.35884199634031 74.49354446184323) bank140095 +140096 POINT(40.19458122501267 74.95524348276982) bank140096 +140097 POINT(40.86810850191585 73.23086250343852) bank140097 +140098 POINT(40.145349542985194 74.83304872284421) bank140098 +140099 POINT(40.91790073455286 74.32337912076875) bank140099 +140100 POINT(40.38090279869902 73.36234796576841) bank140100 +140101 POINT(41.30247984125736 74.83301083814948) bank140101 +140102 POINT(40.162177229322666 73.39308929088975) bank140102 +140103 POINT(40.883307346536306 73.57741734810466) bank140103 +140104 POINT(41.08197458847022 73.40133937138461) bank140104 +140105 POINT(41.068503257692626 74.03633184316813) bank140105 +140106 POINT(41.11768926397946 73.98480755165596) bank140106 +140107 POINT(41.52168766073817 74.53439593908821) bank140107 +140108 POINT(40.151670668982206 73.13433820893003) bank140108 +140109 POINT(40.239941103632034 73.95830855875798) bank140109 +140110 POINT(41.561800556951184 74.78429368614816) bank140110 +140111 POINT(41.608209980164595 74.84094775718957) bank140111 +140112 POINT(41.282975549008654 74.81551817948875) bank140112 +140113 POINT(39.74063506988853 74.50747208468727) bank140113 +140114 POINT(41.46620934596981 73.63244298566585) bank140114 +140115 POINT(40.867592322203976 73.93578914043285) bank140115 +140116 POINT(40.351470382293115 74.74068663267059) bank140116 +140117 POINT(41.30256148962242 74.78244303981717) bank140117 +140118 POINT(40.12437680000509 73.79059463750549) bank140118 +140119 POINT(40.251781560443334 74.37113560633036) bank140119 +140120 POINT(40.063843736133784 73.66937387978106) bank140120 +140121 POINT(41.08026335371577 73.08724904615767) bank140121 +140122 POINT(40.79789510576988 73.72461787738254) bank140122 +140123 POINT(39.8129443823977 73.6053463279427) bank140123 +140124 POINT(39.71410723786925 73.71103440738393) bank140124 +140125 POINT(41.67231007530984 74.47715761484565) bank140125 +140126 POINT(40.43272276038747 73.85533927704094) bank140126 +140127 POINT(39.86717581777768 74.79211943011678) bank140127 +140128 POINT(41.60938800495375 74.47853220147763) bank140128 +140129 POINT(41.64321234266615 74.70458795180221) bank140129 +140130 POINT(41.02353721336096 74.18300252343222) bank140130 +140131 POINT(40.819040379952185 73.89743779726439) bank140131 +140132 POINT(40.69454765865458 73.68269799075371) bank140132 +140133 POINT(40.77252938245451 73.6780604159726) bank140133 +140134 POINT(41.01139806916784 73.46450463441475) bank140134 +140135 POINT(41.528170761711145 74.08549902623885) bank140135 +140136 POINT(41.17206767290248 74.74897946524021) bank140136 +140137 POINT(41.3401615187505 73.31227458695253) bank140137 +140138 POINT(40.628117285292525 73.78685525928623) bank140138 +140139 POINT(39.893609098305845 73.63754733381631) bank140139 +140140 POINT(40.42397330350573 74.50135100353097) bank140140 +140141 POINT(40.68459884829147 74.1210754063656) bank140141 +140142 POINT(41.559663183643615 73.6648335804391) bank140142 +140143 POINT(40.11660893779398 74.82625058829204) bank140143 +140144 POINT(41.28714489481062 74.62967563025997) bank140144 +140145 POINT(39.95086548978918 73.28530327426134) bank140145 +140146 POINT(39.76842402595565 74.72911385754469) bank140146 +140147 POINT(40.94948715080665 74.63811239000538) bank140147 +140148 POINT(41.09700995232469 74.96645451315466) bank140148 +140149 POINT(41.037656976092975 74.17731115958567) bank140149 +140150 POINT(40.75137435612717 73.5118191635826) bank140150 +140151 POINT(41.62139650235711 74.58226871935763) bank140151 +140152 POINT(39.827011243956335 73.75945242556651) bank140152 +140153 POINT(41.513780981636216 73.86207404368353) bank140153 +140154 POINT(40.042780873314925 74.81304170639697) bank140154 +140155 POINT(39.96883002128696 73.63463800807688) bank140155 +140156 POINT(40.30459965507127 73.38163790349199) bank140156 +140157 POINT(41.30154981958666 73.43445071113312) bank140157 +140158 POINT(40.964719365340514 74.45697449847808) bank140158 +140159 POINT(39.870990217283016 74.52482265638842) bank140159 +140160 POINT(40.8349948119977 74.8223876360971) bank140160 +140161 POINT(40.85079524014757 74.7904492749585) bank140161 +140162 POINT(40.16360005748501 73.67309328147437) bank140162 +140163 POINT(39.806104382210826 74.01150452688431) bank140163 +140164 POINT(41.689326698525726 73.89859929668476) bank140164 +140165 POINT(40.737966503067824 73.51067478120574) bank140165 +140166 POINT(41.58059707068734 73.38209865164154) bank140166 +140167 POINT(40.759720240296794 74.5133334510489) bank140167 +140168 POINT(41.05725657766725 73.52206100408027) bank140168 +140169 POINT(40.65983561397182 74.84752642838275) bank140169 +140170 POINT(41.390186188157195 74.93425169786057) bank140170 +140171 POINT(39.719336067165386 73.49682678170498) bank140171 +140172 POINT(40.39621731598431 73.84843539026814) bank140172 +140173 POINT(41.500953897061656 73.80325462203254) bank140173 +140174 POINT(41.551455642070444 74.50309695425268) bank140174 +140175 POINT(40.167727659255306 74.31179774660089) bank140175 +140176 POINT(41.00208865866008 74.14074981091872) bank140176 +140177 POINT(40.80047444450837 73.83720292962035) bank140177 +140178 POINT(41.12981745838076 73.21506053439371) bank140178 +140179 POINT(40.9691863344391 74.6427818008093) bank140179 +140180 POINT(40.163096841053104 74.91661090706863) bank140180 +140181 POINT(41.20004365378434 73.63797232517894) bank140181 +140182 POINT(41.21362620279295 73.47674127251125) bank140182 +140183 POINT(39.96143925813912 73.90435844632566) bank140183 +140184 POINT(41.05842041529015 73.0081270416437) bank140184 +140185 POINT(41.035914983890436 73.53809392864798) bank140185 +140186 POINT(40.416159796686294 74.07766236329068) bank140186 +140187 POINT(41.37925072943293 74.94894436469234) bank140187 +140188 POINT(40.173060100942585 73.30536716272132) bank140188 +140189 POINT(41.48238819590839 74.09962059618246) bank140189 +140190 POINT(41.55600302940381 73.62220014266872) bank140190 +140191 POINT(40.31909357055303 73.72165666595579) bank140191 +140192 POINT(41.13219288942706 74.56628632884575) bank140192 +140193 POINT(41.46984310205863 74.55222847199578) bank140193 +140194 POINT(40.24883245896956 74.19635244828913) bank140194 +140195 POINT(40.636121154889295 74.44915334686466) bank140195 +140196 POINT(40.57903998807296 73.7927682561953) bank140196 +140197 POINT(40.71903133438952 74.09079074720972) bank140197 +140198 POINT(40.20589316523781 73.49158467738152) bank140198 +140199 POINT(41.65900288887197 74.22223093096932) bank140199 +140200 POINT(41.33992115598046 74.26393555697331) bank140200 +140201 POINT(40.94089236701038 73.55462684272095) bank140201 +140202 POINT(40.64792635733141 73.59811951314957) bank140202 +140203 POINT(41.57433110710821 73.63507690148715) bank140203 +140204 POINT(39.872383242055335 74.05753721065727) bank140204 +140205 POINT(41.01095806920937 73.85943051702895) bank140205 +140206 POINT(40.70531326213763 74.18132890786406) bank140206 +140207 POINT(41.392344573271686 73.68126667462023) bank140207 +140208 POINT(41.239098722118214 74.37936270181724) bank140208 +140209 POINT(41.658658932766905 73.3282666695055) bank140209 +140210 POINT(41.50163420093772 74.23422645201103) bank140210 +140211 POINT(41.54097699040258 73.14988342399015) bank140211 +140212 POINT(41.63436499488364 74.16013876914468) bank140212 +140213 POINT(41.23249766292199 73.26701756344674) bank140213 +140214 POINT(40.0577779607985 74.35617069016004) bank140214 +140215 POINT(41.45001582309024 74.42772295446451) bank140215 +140216 POINT(41.70338969130976 73.92560298079025) bank140216 +140217 POINT(40.561708329250216 74.61207285007086) bank140217 +140218 POINT(41.308948320232005 74.21890752475271) bank140218 +140219 POINT(40.2118390799441 73.27632061013783) bank140219 +140220 POINT(41.66270413596416 74.02996002100792) bank140220 +140221 POINT(40.34080932257159 74.09270543087548) bank140221 +140222 POINT(39.81028332642058 74.00190434820469) bank140222 +140223 POINT(41.58415446979894 73.35372737160553) bank140223 +140224 POINT(40.08472755978726 73.8770145674627) bank140224 +140225 POINT(41.131035034893536 73.31960831847657) bank140225 +140226 POINT(40.28658912998244 73.53595684357421) bank140226 +140227 POINT(40.281662032300645 74.00890545321151) bank140227 +140228 POINT(40.379332088837494 74.53104964358997) bank140228 +140229 POINT(39.76536442442422 74.82525684541872) bank140229 +140230 POINT(41.50128739306478 73.59434300730996) bank140230 +140231 POINT(41.49205563637797 74.90168775050353) bank140231 +140232 POINT(41.04409618813872 73.744446386845) bank140232 +140233 POINT(41.33995126426433 74.02303511865534) bank140233 +140234 POINT(39.735882755218384 74.46369302133024) bank140234 +140235 POINT(40.52170407221701 73.13975321898111) bank140235 +140236 POINT(40.26527727908838 73.52375183595134) bank140236 +140237 POINT(41.01011039729033 73.83790757142262) bank140237 +140238 POINT(40.659510937685255 74.38443925126818) bank140238 +140239 POINT(39.906564756347386 74.53794860272704) bank140239 +140240 POINT(41.17653207555682 74.50350553115287) bank140240 +140241 POINT(40.60775880149169 74.38922028830262) bank140241 +140242 POINT(41.47094548791994 73.77213602451963) bank140242 +140243 POINT(40.09695240590166 73.03768333546631) bank140243 +140244 POINT(41.23026256170577 73.59410415433405) bank140244 +140245 POINT(40.63842315790469 74.82935804956783) bank140245 +140246 POINT(39.93159817121447 73.8626975018192) bank140246 +140247 POINT(40.82692792346279 73.05008396486055) bank140247 +140248 POINT(40.03745708088211 73.47123161560765) bank140248 +140249 POINT(41.70940913733697 74.13643394301029) bank140249 +140250 POINT(40.83618808913172 73.89923769611933) bank140250 +140251 POINT(39.75006543623693 73.48964037685819) bank140251 +140252 POINT(41.43224819434676 73.12328403607532) bank140252 +140253 POINT(40.23701200097951 73.7783184483654) bank140253 +140254 POINT(41.29926604294671 74.60972771314484) bank140254 +140255 POINT(41.26896269477582 73.32722124464006) bank140255 +140256 POINT(41.494980821425564 73.4381497582363) bank140256 +140257 POINT(40.30635762125275 73.592952223075) bank140257 +140258 POINT(41.01162943024414 74.30921936563101) bank140258 +140259 POINT(40.23505763076318 73.22549858264868) bank140259 +140260 POINT(40.81870430542581 74.87498995503934) bank140260 +140261 POINT(41.110916322474 73.87816958835234) bank140261 +140262 POINT(40.65672699286133 74.06129159311692) bank140262 +140263 POINT(39.71697169629072 73.36059658978402) bank140263 +140264 POINT(40.29904281291719 73.25842770505919) bank140264 +140265 POINT(41.44411907153024 74.00065798915087) bank140265 +140266 POINT(41.03153632848566 74.3473340862854) bank140266 +140267 POINT(40.45913066288287 73.22209091320741) bank140267 +140268 POINT(41.53177174443004 74.86133513469579) bank140268 +140269 POINT(40.144655046777636 74.25762467622413) bank140269 +140270 POINT(41.1619335148394 74.66112276310693) bank140270 +140271 POINT(40.186141579538486 73.77920651769504) bank140271 +140272 POINT(40.136528241138265 73.65939604301843) bank140272 +140273 POINT(40.07098690932609 74.93838843433774) bank140273 +140274 POINT(41.65942176324044 74.10943253141994) bank140274 +140275 POINT(39.7138482176338 73.03984362260756) bank140275 +140276 POINT(40.53083548778122 74.98427123247829) bank140276 +140277 POINT(40.25081054295403 73.54754895379865) bank140277 +140278 POINT(41.46813621256374 73.59046288500356) bank140278 +140279 POINT(39.92544866304353 74.3896346639324) bank140279 +140280 POINT(41.65934414407348 73.49849516546237) bank140280 +140281 POINT(41.442846589133346 73.42861381301576) bank140281 +140282 POINT(40.43696493310796 73.89752789056409) bank140282 +140283 POINT(40.525655484492134 73.45870232028302) bank140283 +140284 POINT(40.489200207440255 73.22980780427613) bank140284 +140285 POINT(41.579279227191726 73.06739005618961) bank140285 +140286 POINT(41.18512449820336 73.07670016168562) bank140286 +140287 POINT(40.69112112996949 74.96567761631323) bank140287 +140288 POINT(40.66945700997432 74.98927508780197) bank140288 +140289 POINT(41.27086149674153 73.1802158475515) bank140289 +140290 POINT(40.744795507783984 74.53576334671139) bank140290 +140291 POINT(40.80886389182663 74.7101577240333) bank140291 +140292 POINT(40.0296643857805 74.65899251944694) bank140292 +140293 POINT(41.59686573641455 74.81972884576217) bank140293 +140294 POINT(41.08159333108785 74.23250836127531) bank140294 +140295 POINT(41.43796944147185 74.58296104245352) bank140295 +140296 POINT(39.845141937714324 74.38125686844191) bank140296 +140297 POINT(41.5690464020498 73.06557372617333) bank140297 +140298 POINT(40.201092106421505 74.786794944588) bank140298 +140299 POINT(40.185891043279184 74.61597990297025) bank140299 +140300 POINT(40.124086060461686 74.36690593733256) bank140300 +140301 POINT(40.58956902357922 74.57431634009404) bank140301 +140302 POINT(40.93730515451415 73.84940678303882) bank140302 +140303 POINT(40.79976329963971 73.31505143436819) bank140303 +140304 POINT(41.129746784325604 74.61600457992908) bank140304 +140305 POINT(41.60899861579788 73.52463842304418) bank140305 +140306 POINT(41.049509627398145 73.85487268501659) bank140306 +140307 POINT(40.4312052873467 74.07740185281762) bank140307 +140308 POINT(40.47274322731824 73.3046090257674) bank140308 +140309 POINT(41.48117550543362 73.05742432698877) bank140309 +140310 POINT(41.3428141391811 73.77201793406715) bank140310 +140311 POINT(40.5806390865252 73.48433658738335) bank140311 +140312 POINT(40.043262008013016 73.32071999942936) bank140312 +140313 POINT(41.57203281241208 74.25860480606126) bank140313 +140314 POINT(41.61757349836677 74.21302589340299) bank140314 +140315 POINT(41.26409151967893 74.9687355267241) bank140315 +140316 POINT(40.82795467499489 73.16351999369448) bank140316 +140317 POINT(39.82618928562867 73.89945543346634) bank140317 +140318 POINT(40.00466983656076 73.87249977160465) bank140318 +140319 POINT(39.78113740390914 73.75682389081351) bank140319 +140320 POINT(40.61723999968836 74.71391083315041) bank140320 +140321 POINT(40.132158449812266 73.61135724199252) bank140321 +140322 POINT(39.85550132953679 73.58349256521039) bank140322 +140323 POINT(41.419224196501865 74.20960356508574) bank140323 +140324 POINT(40.06252751246859 73.3260898888422) bank140324 +140325 POINT(41.55852155132278 74.25346220210098) bank140325 +140326 POINT(41.300424790532695 74.28534089976523) bank140326 +140327 POINT(39.988185087246535 73.03347449566908) bank140327 +140328 POINT(40.94404867766868 74.73204557789461) bank140328 +140329 POINT(40.65111738270231 73.21148498580229) bank140329 +140330 POINT(41.38065223212061 73.97426890053738) bank140330 +140331 POINT(40.41868793419472 74.1899292454386) bank140331 +140332 POINT(41.30912758382944 74.64640647125829) bank140332 +140333 POINT(41.61578105266377 74.84448580687418) bank140333 +140334 POINT(41.22696117393236 73.65175233623815) bank140334 +140335 POINT(39.79000896222952 73.85163521534503) bank140335 +140336 POINT(40.86263717728199 73.58918034496367) bank140336 +140337 POINT(39.713270036300166 74.91071246667283) bank140337 +140338 POINT(39.80985435046115 74.72753636359965) bank140338 +140339 POINT(40.99574927699111 73.36023734765449) bank140339 +140340 POINT(41.27688068916293 74.5200618246278) bank140340 +140341 POINT(40.96223755190483 73.24242412811894) bank140341 +140342 POINT(41.22629489963192 73.09555687941896) bank140342 +140343 POINT(40.40647408670849 74.83751870568234) bank140343 +140344 POINT(39.8135748691953 73.8865819125575) bank140344 +140345 POINT(40.767811527227614 74.9988972200332) bank140345 +140346 POINT(40.598081455424236 74.79146937450682) bank140346 +140347 POINT(41.62237620594457 73.50288904965859) bank140347 +140348 POINT(41.45067621183376 73.11583187086822) bank140348 +140349 POINT(41.2974398695227 74.05448282377024) bank140349 +140350 POINT(41.222389279482044 74.57507004694308) bank140350 +140351 POINT(39.72147354580352 73.26911163444669) bank140351 +140352 POINT(40.39659022221256 73.65914341703228) bank140352 +140353 POINT(39.72970373331476 73.68788672651564) bank140353 +140354 POINT(41.13727641212718 74.5307113356044) bank140354 +140355 POINT(41.04187910730533 73.17918889127326) bank140355 +140356 POINT(40.19710685831976 74.76653373535864) bank140356 +140357 POINT(41.31966066504844 74.62169805871281) bank140357 +140358 POINT(40.519880118789594 73.91491602032583) bank140358 +140359 POINT(40.30969654090961 74.98670704435462) bank140359 +140360 POINT(41.09896950618264 73.02074996240876) bank140360 +140361 POINT(40.108984547169406 73.65167910040651) bank140361 +140362 POINT(40.13164584724608 74.42899229800253) bank140362 +140363 POINT(40.56637939109268 73.45889964448443) bank140363 +140364 POINT(39.97777553618663 73.33662056035926) bank140364 +140365 POINT(41.6768280225951 74.80820907125478) bank140365 +140366 POINT(41.39269645403172 74.32547006580404) bank140366 +140367 POINT(39.82295605999955 73.6266014771795) bank140367 +140368 POINT(40.43346165897242 74.03845356993816) bank140368 +140369 POINT(41.0775074011231 73.16600269426313) bank140369 +140370 POINT(41.34871059646588 74.2434869152699) bank140370 +140371 POINT(41.16058069661151 73.75739415318564) bank140371 +140372 POINT(40.10402665384724 74.94452085260484) bank140372 +140373 POINT(41.3722225829702 73.36870585644421) bank140373 +140374 POINT(40.097988400839 74.19334233147627) bank140374 +140375 POINT(40.43579669752051 73.65473575091475) bank140375 +140376 POINT(40.67506346004638 73.73584858289942) bank140376 +140377 POINT(39.758145243286144 74.90250360533398) bank140377 +140378 POINT(41.20919494450988 73.40099318150237) bank140378 +140379 POINT(40.35094166898557 74.41408176920922) bank140379 +140380 POINT(40.42824308380587 73.9092972619053) bank140380 +140381 POINT(40.06007657897273 73.56980817717726) bank140381 +140382 POINT(40.38109904173887 73.31488862087895) bank140382 +140383 POINT(40.2695997228988 73.9862138597102) bank140383 +140384 POINT(40.80687174842673 74.80918378305765) bank140384 +140385 POINT(41.54071816822453 73.65077445510002) bank140385 +140386 POINT(40.67414842140647 73.75687778276057) bank140386 +140387 POINT(41.032745662277975 73.65226058789331) bank140387 +140388 POINT(41.17531262742718 73.62912914571693) bank140388 +140389 POINT(41.33562221587728 73.66448274415747) bank140389 +140390 POINT(41.24614625016553 74.88383864696367) bank140390 +140391 POINT(40.46983710181233 74.28859515711378) bank140391 +140392 POINT(40.92165420360913 74.69270463348268) bank140392 +140393 POINT(39.71833574322348 73.16139866350387) bank140393 +140394 POINT(41.18483402718629 74.47325199861814) bank140394 +140395 POINT(41.044933021114396 73.22330051321893) bank140395 +140396 POINT(39.87567960572116 73.76060788187955) bank140396 +140397 POINT(40.635395632449544 73.48548206011564) bank140397 +140398 POINT(41.2467096016052 74.95065242265974) bank140398 +140399 POINT(40.15972383312108 74.33402441317959) bank140399 +140400 POINT(41.30024750926344 74.41106374046535) bank140400 +140401 POINT(41.400192373047375 73.54044497526999) bank140401 +140402 POINT(40.50518733999559 73.07913184142082) bank140402 +140403 POINT(41.710915257646064 74.93773178812621) bank140403 +140404 POINT(41.15576479962344 73.60304613882009) bank140404 +140405 POINT(41.497919389417206 73.78211808355681) bank140405 +140406 POINT(41.269974394163526 73.76413448179262) bank140406 +140407 POINT(41.10552081135404 73.70178495947872) bank140407 +140408 POINT(41.30860578762457 73.94201691819484) bank140408 +140409 POINT(40.8603479398154 74.35167602095105) bank140409 +140410 POINT(40.434637092134224 74.31069785129328) bank140410 +140411 POINT(41.17321426684156 74.17005895695387) bank140411 +140412 POINT(41.58810398418057 74.83355064043675) bank140412 +140413 POINT(41.60917155379625 73.75214938832247) bank140413 +140414 POINT(41.63931265338546 73.92361401283412) bank140414 +140415 POINT(39.947676873791806 74.91985902376075) bank140415 +140416 POINT(40.39016115672875 74.21975174445578) bank140416 +140417 POINT(40.751204777324865 74.71300961603393) bank140417 +140418 POINT(40.333915647097356 73.33956307312211) bank140418 +140419 POINT(40.802522193730546 73.62714243271337) bank140419 +140420 POINT(40.278331311593305 74.64782408319913) bank140420 +140421 POINT(40.213853763468585 74.17880218938214) bank140421 +140422 POINT(40.36359593603654 73.50930131139161) bank140422 +140423 POINT(40.12891423249485 73.8107365430606) bank140423 +140424 POINT(41.651345304185526 74.02096100041965) bank140424 +140425 POINT(40.265615080348816 73.69300246055829) bank140425 +140426 POINT(41.12551476793425 73.47364060604966) bank140426 +140427 POINT(40.39866367013338 73.16358860666023) bank140427 +140428 POINT(40.091873618161515 74.13163017254277) bank140428 +140429 POINT(40.26929827051166 73.43461067257726) bank140429 +140430 POINT(41.30851276253152 74.76451090951288) bank140430 +140431 POINT(40.52590431495147 74.70095646539512) bank140431 +140432 POINT(40.98618712559393 73.9700220530186) bank140432 +140433 POINT(40.26258571307952 73.4191672945302) bank140433 +140434 POINT(40.396829186255104 73.38393456979712) bank140434 +140435 POINT(40.36741888046422 73.01204116151526) bank140435 +140436 POINT(41.04695323584229 74.6770384063222) bank140436 +140437 POINT(39.87825860987457 73.35268911248721) bank140437 +140438 POINT(40.335326133104275 73.83024217932564) bank140438 +140439 POINT(39.81010136915308 74.19309786824176) bank140439 +140440 POINT(41.13736398712093 73.66822129660888) bank140440 +140441 POINT(40.26579551987228 73.97506131682304) bank140441 +140442 POINT(41.491734179473426 73.39179559450868) bank140442 +140443 POINT(40.40126640824406 73.51189411049332) bank140443 +140444 POINT(40.953006691196904 74.32527488566936) bank140444 +140445 POINT(41.04668236367262 74.793830844118) bank140445 +140446 POINT(39.99313989253777 73.26778954309007) bank140446 +140447 POINT(39.977661468960456 73.78743860144958) bank140447 +140448 POINT(40.9479840770201 73.40504557581025) bank140448 +140449 POINT(40.82721789119157 74.9799620902136) bank140449 +140450 POINT(41.61823703095868 73.74224699519812) bank140450 +140451 POINT(40.4461243428941 73.22486402589055) bank140451 +140452 POINT(41.27120823494546 74.67307626271945) bank140452 +140453 POINT(41.098052985569055 73.21808776052812) bank140453 +140454 POINT(41.3112695797913 74.53770091508014) bank140454 +140455 POINT(40.4117912648687 73.77449706149908) bank140455 +140456 POINT(40.32969409228145 73.31434175414472) bank140456 +140457 POINT(41.08926400567666 74.80480498366744) bank140457 +140458 POINT(40.92888362411874 74.5307613588239) bank140458 +140459 POINT(39.91002396614407 73.79317817443356) bank140459 +140460 POINT(41.39124463813602 73.2683559640837) bank140460 +140461 POINT(41.23553855980261 73.79513458167858) bank140461 +140462 POINT(41.12968181986378 74.48379421332099) bank140462 +140463 POINT(40.3559065139317 74.53314253542855) bank140463 +140464 POINT(41.34650024653462 73.17128658732831) bank140464 +140465 POINT(40.59263255327305 73.2278127200805) bank140465 +140466 POINT(40.82106344168123 74.41019670531804) bank140466 +140467 POINT(41.12014041261159 74.24161785103932) bank140467 +140468 POINT(40.38838831269009 73.61883859617275) bank140468 +140469 POINT(40.563776487469866 73.37226262893) bank140469 +140470 POINT(40.07011232636175 74.60273181441347) bank140470 +140471 POINT(40.14539167511674 73.93087760768356) bank140471 +140472 POINT(40.1576854336134 73.5873420723795) bank140472 +140473 POINT(41.47944716726574 74.42544778687605) bank140473 +140474 POINT(40.568165156245016 74.24669585158833) bank140474 +140475 POINT(39.87137819577122 73.93122467903868) bank140475 +140476 POINT(39.9102042971178 73.8478864338597) bank140476 +140477 POINT(40.11517539329318 73.04855170720171) bank140477 +140478 POINT(41.51548652530526 73.75972240308023) bank140478 +140479 POINT(40.93242169054023 75.0022018856343) bank140479 +140480 POINT(41.471249937986364 74.71927846342304) bank140480 +140481 POINT(41.212540254374346 73.23920872906743) bank140481 +140482 POINT(40.35695508601785 74.44780178430595) bank140482 +140483 POINT(39.85488133402909 73.67622356269123) bank140483 +140484 POINT(40.728593204961875 74.18651117707275) bank140484 +140485 POINT(40.314812543632065 73.9736720724488) bank140485 +140486 POINT(41.257021569514215 73.43846510615533) bank140486 +140487 POINT(39.830671227824745 73.95757652188132) bank140487 +140488 POINT(40.17900152656737 73.61462103750647) bank140488 +140489 POINT(39.724064802855025 74.52377589088731) bank140489 +140490 POINT(41.650660207070146 74.8265463975717) bank140490 +140491 POINT(41.202820186092644 74.73646307190594) bank140491 +140492 POINT(40.40553663178811 74.85268316430042) bank140492 +140493 POINT(40.76830436945357 74.53837341815748) bank140493 +140494 POINT(41.4698362668751 74.97741960379385) bank140494 +140495 POINT(41.0910553345537 73.33330271970674) bank140495 +140496 POINT(41.11541768064784 74.74272348473788) bank140496 +140497 POINT(40.22284231579122 74.49858197816813) bank140497 +140498 POINT(41.585615552914696 74.52168265570116) bank140498 +140499 POINT(40.614881179678065 74.67826817478469) bank140499 +140500 POINT(41.368081984475 74.46015626336153) bank140500 +140501 POINT(40.57055993015402 74.41016729915371) bank140501 +140502 POINT(40.97630910249837 74.63077546924153) bank140502 +140503 POINT(40.68732058514483 74.61774077695596) bank140503 +140504 POINT(40.33321418081955 73.0705344222234) bank140504 +140505 POINT(40.951727333462614 74.77300170651161) bank140505 +140506 POINT(40.24813303577438 73.38089292501449) bank140506 +140507 POINT(39.81353357426686 74.41248499003969) bank140507 +140508 POINT(41.076586003038216 73.3636125809987) bank140508 +140509 POINT(40.291508289132615 73.79340160588339) bank140509 +140510 POINT(40.458469533719395 74.46743824786523) bank140510 +140511 POINT(40.04638403759517 73.63044774221238) bank140511 +140512 POINT(39.82954488505587 74.81470681930176) bank140512 +140513 POINT(41.21156966218448 74.95370502865767) bank140513 +140514 POINT(41.422953322659126 73.80138558892725) bank140514 +140515 POINT(40.3082297099811 73.89769186683588) bank140515 +140516 POINT(41.68219594815915 73.96962568017562) bank140516 +140517 POINT(40.45797894498149 74.53010509513977) bank140517 +140518 POINT(39.83716021964489 73.98964660321771) bank140518 +140519 POINT(40.60713654489503 74.06619929894718) bank140519 +140520 POINT(39.71945350345723 74.48677561528025) bank140520 +140521 POINT(40.059477885922306 74.05889886733122) bank140521 +140522 POINT(41.282691547131165 74.58836613166964) bank140522 +140523 POINT(40.22258333850435 74.6279614563932) bank140523 +140524 POINT(40.5899702476171 73.36547217215445) bank140524 +140525 POINT(41.33273031318824 73.28513072362185) bank140525 +140526 POINT(41.705764390703735 73.75660066695866) bank140526 +140527 POINT(40.582592259229585 73.63611874893128) bank140527 +140528 POINT(40.391589654212424 74.41418076592448) bank140528 +140529 POINT(39.89042827384154 74.96425300173726) bank140529 +140530 POINT(40.59117655298362 73.92158461024216) bank140530 +140531 POINT(41.000615150190605 74.57510166224955) bank140531 +140532 POINT(41.37303557425593 74.78277873745114) bank140532 +140533 POINT(39.99978143369686 74.9650115486775) bank140533 +140534 POINT(41.218459668132745 74.15717243650069) bank140534 +140535 POINT(40.0976762449675 73.26333484812034) bank140535 +140536 POINT(41.04995107669105 74.84042370006377) bank140536 +140537 POINT(41.19172413788114 73.34430614796386) bank140537 +140538 POINT(39.78989495261858 74.98798783790879) bank140538 +140539 POINT(40.303578850774805 74.08648979775731) bank140539 +140540 POINT(41.56312492114228 74.27299953964629) bank140540 +140541 POINT(41.66213033061488 74.78291698562637) bank140541 +140542 POINT(39.744687785135575 73.37102546414569) bank140542 +140543 POINT(41.649627162978874 74.38340831655174) bank140543 +140544 POINT(39.84938327237922 73.8881488799876) bank140544 +140545 POINT(40.443777519395766 74.1434681706689) bank140545 +140546 POINT(40.82381779617725 74.53368193480004) bank140546 +140547 POINT(40.11137832480025 74.85826541912942) bank140547 +140548 POINT(40.17822524125721 73.03189722490725) bank140548 +140549 POINT(40.17122996088052 73.4397562782079) bank140549 +140550 POINT(40.085956416822896 74.0307051039859) bank140550 +140551 POINT(39.96836849096321 74.4665912175632) bank140551 +140552 POINT(40.023394487381026 74.21149493812885) bank140552 +140553 POINT(41.35380990309562 73.63086528072259) bank140553 +140554 POINT(41.70249853937753 74.0223808422208) bank140554 +140555 POINT(40.52119399304038 74.34503056794244) bank140555 +140556 POINT(41.2706038821046 73.22123393885066) bank140556 +140557 POINT(40.1787226007021 74.23231663272823) bank140557 +140558 POINT(41.3444748190773 74.90280030167992) bank140558 +140559 POINT(40.68007525989758 73.36837967495808) bank140559 +140560 POINT(41.47232941747533 74.39547048113153) bank140560 +140561 POINT(40.268753020998325 73.95099920387996) bank140561 +140562 POINT(40.38383069985266 74.61087323259156) bank140562 +140563 POINT(40.04053823656238 73.59376444016765) bank140563 +140564 POINT(41.4156930061419 74.39081817429843) bank140564 +140565 POINT(40.71873911520035 74.97801282258422) bank140565 +140566 POINT(41.50318752686844 74.35159483086319) bank140566 +140567 POINT(40.86041007727247 73.3216708140368) bank140567 +140568 POINT(40.76450120097482 73.43018561039014) bank140568 +140569 POINT(39.90242267713281 74.26177918747615) bank140569 +140570 POINT(41.21876247832668 74.97369338443805) bank140570 +140571 POINT(39.99969504163902 74.4934339975352) bank140571 +140572 POINT(39.769191924335544 74.7615930620593) bank140572 +140573 POINT(41.455334751492586 73.54900945249412) bank140573 +140574 POINT(41.3853234757104 73.36612440578726) bank140574 +140575 POINT(40.33491765583188 73.55153501118907) bank140575 +140576 POINT(41.22878900251859 74.52217584614833) bank140576 +140577 POINT(40.73558177714132 74.81209217518581) bank140577 +140578 POINT(41.5665603926227 74.3783984213872) bank140578 +140579 POINT(41.09623915985254 73.21617963498622) bank140579 +140580 POINT(40.388593631672414 74.54100065418156) bank140580 +140581 POINT(40.93215990688089 73.03729336697762) bank140581 +140582 POINT(40.8622321739592 73.86035609566692) bank140582 +140583 POINT(39.966026075737894 73.08637724090872) bank140583 +140584 POINT(41.59144311158909 74.42402508904445) bank140584 +140585 POINT(40.261585207354365 73.46897998021916) bank140585 +140586 POINT(40.7803607211405 74.4407352161765) bank140586 +140587 POINT(40.28535174557479 73.14434422039592) bank140587 +140588 POINT(39.929118008970235 74.69641570748443) bank140588 +140589 POINT(39.92619681073284 74.67428175924492) bank140589 +140590 POINT(40.27121910642794 73.03938144696967) bank140590 +140591 POINT(40.61441122746702 73.51914614307768) bank140591 +140592 POINT(40.248021136713156 74.47958011565161) bank140592 +140593 POINT(41.34509634977949 73.9384205860894) bank140593 +140594 POINT(39.9226447319331 73.8905206939288) bank140594 +140595 POINT(40.14325100311741 74.4795718276018) bank140595 +140596 POINT(40.10882322519453 73.13768711206899) bank140596 +140597 POINT(40.365594838334346 73.48653406264584) bank140597 +140598 POINT(40.65549289880938 74.12895328801089) bank140598 +140599 POINT(39.78629763419209 73.54565067496587) bank140599 +140600 POINT(41.26749616354884 73.14179572151546) bank140600 +140601 POINT(40.686667215907384 74.37556849907273) bank140601 +140602 POINT(40.401369503356094 73.8481130257723) bank140602 +140603 POINT(40.481405014094065 74.60746614739142) bank140603 +140604 POINT(39.99167307576956 73.41597675920308) bank140604 +140605 POINT(39.91341249997398 74.09069660000796) bank140605 +140606 POINT(40.238828300296554 74.25235625946873) bank140606 +140607 POINT(39.991512608820976 74.5217925933885) bank140607 +140608 POINT(40.99088135585909 73.45661788689107) bank140608 +140609 POINT(40.78743020977362 74.21632570679579) bank140609 +140610 POINT(40.99336906550243 74.21733235809162) bank140610 +140611 POINT(40.23502182087548 74.53973525013849) bank140611 +140612 POINT(40.03521316308747 73.04030942572095) bank140612 +140613 POINT(40.38390401920689 73.54543178710028) bank140613 +140614 POINT(40.84479560928339 74.44680433339987) bank140614 +140615 POINT(40.89883402742786 73.14906511603365) bank140615 +140616 POINT(40.76389530646807 73.30469079105565) bank140616 +140617 POINT(40.83858353534046 73.79062971421683) bank140617 +140618 POINT(41.01359379855166 74.0857889396868) bank140618 +140619 POINT(40.04673168888235 74.80968857233356) bank140619 +140620 POINT(39.89922645166021 73.7236780032418) bank140620 +140621 POINT(41.57110450641828 74.58211108978709) bank140621 +140622 POINT(41.6077319615374 74.36448639168223) bank140622 +140623 POINT(40.73786120618314 74.24077175306361) bank140623 +140624 POINT(39.95635712034142 73.38045020428305) bank140624 +140625 POINT(40.17531419549399 73.60195616017769) bank140625 +140626 POINT(41.687012797547695 73.83670139994038) bank140626 +140627 POINT(41.12208952715889 74.53713109765005) bank140627 +140628 POINT(40.33450634583336 73.58383185884053) bank140628 +140629 POINT(41.359077523027594 73.3173548204211) bank140629 +140630 POINT(41.035436019861855 74.43272516709114) bank140630 +140631 POINT(40.510147929255865 74.46555806913335) bank140631 +140632 POINT(40.707117558462436 74.11026496080639) bank140632 +140633 POINT(40.187415583226745 74.73381223768179) bank140633 +140634 POINT(39.976180235241046 73.48532260475793) bank140634 +140635 POINT(41.1863364545427 73.12149134907494) bank140635 +140636 POINT(40.72978369770744 73.50329541212335) bank140636 +140637 POINT(39.7568258155296 74.92894212908081) bank140637 +140638 POINT(41.5099138535662 74.71122356152681) bank140638 +140639 POINT(40.395066383895504 73.9531483995978) bank140639 +140640 POINT(39.76907070177698 73.38571298507436) bank140640 +140641 POINT(40.21581816303449 73.3949784040773) bank140641 +140642 POINT(40.90517613394399 73.02733178531611) bank140642 +140643 POINT(41.26252586468329 74.72915571728159) bank140643 +140644 POINT(39.73196108563109 74.8252148587736) bank140644 +140645 POINT(40.80978264388986 74.40837682232755) bank140645 +140646 POINT(40.49638730792738 73.56996189088788) bank140646 +140647 POINT(40.26849876738134 73.43141160651486) bank140647 +140648 POINT(40.770279207595195 74.70407883474684) bank140648 +140649 POINT(41.687356865579154 73.48944160927773) bank140649 +140650 POINT(39.721293142667776 73.0188596633142) bank140650 +140651 POINT(40.122968212147164 74.13636307096627) bank140651 +140652 POINT(40.871368654679316 74.00259964710192) bank140652 +140653 POINT(41.53337832932453 73.72437671438786) bank140653 +140654 POINT(41.08565471230178 74.88755868986446) bank140654 +140655 POINT(39.77459656415159 73.48597010166569) bank140655 +140656 POINT(41.589729038100245 74.76399749368227) bank140656 +140657 POINT(41.058599460029065 73.77881087216187) bank140657 +140658 POINT(39.71524441378908 73.86461458486298) bank140658 +140659 POINT(41.44878589101237 74.29847878428251) bank140659 +140660 POINT(40.712903370554166 74.05198038146804) bank140660 +140661 POINT(40.89488964967327 73.76029094697212) bank140661 +140662 POINT(41.36505070069693 74.36748656027488) bank140662 +140663 POINT(40.970172148041826 74.60688300413484) bank140663 +140664 POINT(40.353574061951925 73.87133459351872) bank140664 +140665 POINT(40.20561396454514 73.56313723831886) bank140665 +140666 POINT(39.838631218404274 73.38045473725337) bank140666 +140667 POINT(40.326915545802215 74.08570477818675) bank140667 +140668 POINT(41.421344991709475 74.50756207709804) bank140668 +140669 POINT(40.42796151407492 73.77602438015214) bank140669 +140670 POINT(40.00979747274627 74.49367733055483) bank140670 +140671 POINT(41.49776138776713 73.81378695443046) bank140671 +140672 POINT(40.399778044367814 73.53756862758938) bank140672 +140673 POINT(41.22114303665837 74.95774407757035) bank140673 +140674 POINT(40.65182457409408 74.33318919136264) bank140674 +140675 POINT(40.71841153972954 74.64713876512643) bank140675 +140676 POINT(40.54965919420329 73.45524232805592) bank140676 +140677 POINT(41.42706769248267 74.4065148929914) bank140677 +140678 POINT(41.366865535123715 73.43672061472039) bank140678 +140679 POINT(40.163063611808525 74.34736519175357) bank140679 +140680 POINT(41.24017244861407 73.59604641638848) bank140680 +140681 POINT(40.93084713249623 74.19460957454675) bank140681 +140682 POINT(41.16063447553452 73.46650190191959) bank140682 +140683 POINT(41.486296264082064 74.02716469618673) bank140683 +140684 POINT(40.91532827291166 73.25428536901153) bank140684 +140685 POINT(40.33913018656393 73.47449774017764) bank140685 +140686 POINT(40.9121466783131 73.33453769129159) bank140686 +140687 POINT(40.05350394960141 74.32485941017963) bank140687 +140688 POINT(41.53289343283488 73.80928096494587) bank140688 +140689 POINT(40.35677364633866 73.3066743693053) bank140689 +140690 POINT(40.45089727841895 73.40561538618354) bank140690 +140691 POINT(40.95725687530551 73.90044262054195) bank140691 +140692 POINT(41.17555634684293 73.49326745221762) bank140692 +140693 POINT(40.951391688260195 74.51704729586494) bank140693 +140694 POINT(40.60739763633591 73.91462425544981) bank140694 +140695 POINT(40.81763253941405 73.22348559498033) bank140695 +140696 POINT(40.230376783997656 74.85798573831093) bank140696 +140697 POINT(39.8940262046896 73.68107389582697) bank140697 +140698 POINT(41.35016664000515 73.16847510369092) bank140698 +140699 POINT(41.53342320672852 73.57424609419824) bank140699 +140700 POINT(39.83064154890869 74.6292044766031) bank140700 +140701 POINT(40.244815803446336 74.12607245053283) bank140701 +140702 POINT(40.620941611319346 73.01251220517656) bank140702 +140703 POINT(40.55388532272606 74.8877599114504) bank140703 +140704 POINT(39.90118183279008 74.17134498328268) bank140704 +140705 POINT(40.79731555666326 73.70058057514427) bank140705 +140706 POINT(40.54914994961241 74.96418657476976) bank140706 +140707 POINT(40.92994181066555 74.04138831118168) bank140707 +140708 POINT(40.28285609704499 73.64610608920663) bank140708 +140709 POINT(41.13725820484583 74.06708936559629) bank140709 +140710 POINT(40.70398113329463 73.9847719823493) bank140710 +140711 POINT(40.9837491232462 73.61191760645873) bank140711 +140712 POINT(40.99369758599083 73.0943043131377) bank140712 +140713 POINT(40.11087852705368 73.29106075768665) bank140713 +140714 POINT(40.47849491599565 73.77089701679387) bank140714 +140715 POINT(40.6776288655398 74.47339040439138) bank140715 +140716 POINT(41.244993571466374 73.95132347828768) bank140716 +140717 POINT(40.93752065115495 73.77736458463778) bank140717 +140718 POINT(41.62139443874302 73.81998270241043) bank140718 +140719 POINT(40.56529221463536 73.44678450611526) bank140719 +140720 POINT(41.368120798401925 74.80013620996012) bank140720 +140721 POINT(40.3558919375394 74.65340523440493) bank140721 +140722 POINT(40.46483148033331 73.38087580507177) bank140722 +140723 POINT(40.927661640285564 73.87063043647959) bank140723 +140724 POINT(40.96972657199339 73.80401783648112) bank140724 +140725 POINT(41.513423545129875 73.26097162157532) bank140725 +140726 POINT(40.01047865753223 74.85614516919208) bank140726 +140727 POINT(40.693531328092234 74.64117982515955) bank140727 +140728 POINT(40.94835846570164 73.32694890941194) bank140728 +140729 POINT(39.869192001275906 73.86923726755154) bank140729 +140730 POINT(40.57786308889452 74.91683483383247) bank140730 +140731 POINT(39.82499077703264 74.4843092766135) bank140731 +140732 POINT(40.713939593036905 74.0820732360354) bank140732 +140733 POINT(39.963620970870814 73.76630914791119) bank140733 +140734 POINT(40.85677827241152 74.60102134568156) bank140734 +140735 POINT(40.531435435330124 73.51537758719327) bank140735 +140736 POINT(40.56772989328051 73.66015078086804) bank140736 +140737 POINT(40.00563360316978 73.32201094037542) bank140737 +140738 POINT(40.258752705345586 73.28739947766854) bank140738 +140739 POINT(40.66397677667005 73.05565871498524) bank140739 +140740 POINT(40.15701688140662 74.20112328815995) bank140740 +140741 POINT(41.33183014563838 74.08384195592161) bank140741 +140742 POINT(40.62945863833692 73.13787907219543) bank140742 +140743 POINT(40.6798780200264 73.70875292625796) bank140743 +140744 POINT(40.26175842495547 74.32123368007142) bank140744 +140745 POINT(41.08008310150807 73.89358248965696) bank140745 +140746 POINT(41.49957421102941 73.52550078334188) bank140746 +140747 POINT(41.47900661382405 73.48437951964522) bank140747 +140748 POINT(40.127018137963 73.91030538070204) bank140748 +140749 POINT(40.1035893888423 74.08854800693801) bank140749 +140750 POINT(40.148736167838805 73.1780957941623) bank140750 +140751 POINT(40.56045431589637 73.63775698253781) bank140751 +140752 POINT(40.920883260066894 73.28807200538428) bank140752 +140753 POINT(41.29612234150998 74.63515893304378) bank140753 +140754 POINT(40.96324639176372 73.86223186802444) bank140754 +140755 POINT(39.8625528956969 74.90387673612854) bank140755 +140756 POINT(40.713964281154645 74.73628132553306) bank140756 +140757 POINT(40.927893923064445 74.88223577714562) bank140757 +140758 POINT(41.39924043610049 74.81475467224067) bank140758 +140759 POINT(40.7538082679111 74.31086611297225) bank140759 +140760 POINT(40.145618678570116 73.7235737820425) bank140760 +140761 POINT(41.652923347481064 74.12758591838475) bank140761 +140762 POINT(40.72676851026083 74.59447025539674) bank140762 +140763 POINT(40.610087232840655 74.12811925863221) bank140763 +140764 POINT(39.89686171925075 73.49929434107418) bank140764 +140765 POINT(41.168389541114394 74.06667832655937) bank140765 +140766 POINT(39.74317870957845 74.4507181014032) bank140766 +140767 POINT(41.311780337633245 74.58301867001481) bank140767 +140768 POINT(41.000047414234544 73.11164624759729) bank140768 +140769 POINT(40.9431178983417 73.60996342023385) bank140769 +140770 POINT(39.89583424987042 73.11951739644326) bank140770 +140771 POINT(40.27287980820333 73.57378736622576) bank140771 +140772 POINT(40.11726866685334 73.32115338388364) bank140772 +140773 POINT(41.674023764826394 73.11368497019211) bank140773 +140774 POINT(39.95153393272075 73.22347699232694) bank140774 +140775 POINT(41.61717925495837 73.65501920535823) bank140775 +140776 POINT(40.71008418746047 73.13215622646618) bank140776 +140777 POINT(40.54393173375013 74.16078505815163) bank140777 +140778 POINT(40.87689524026393 73.69547492783265) bank140778 +140779 POINT(40.041976606688586 73.5015930846918) bank140779 +140780 POINT(40.65556354896924 73.83262709284138) bank140780 +140781 POINT(39.78213689770519 73.31256111813701) bank140781 +140782 POINT(41.35112175699262 73.81201427426731) bank140782 +140783 POINT(40.31413196628323 74.3422162040876) bank140783 +140784 POINT(40.67399093767414 73.48324838583592) bank140784 +140785 POINT(40.15655805138975 73.3714512946307) bank140785 +140786 POINT(41.11133692192303 74.28502202036192) bank140786 +140787 POINT(40.826198689731704 74.07104035656627) bank140787 +140788 POINT(39.99334179941667 73.0111909220081) bank140788 +140789 POINT(40.09344987334366 74.92786284572716) bank140789 +140790 POINT(40.68410532482057 73.49736588720431) bank140790 +140791 POINT(41.70504796133541 74.07709954254659) bank140791 +140792 POINT(41.70462218260984 73.7487747829568) bank140792 +140793 POINT(41.46497231018326 74.33297641756967) bank140793 +140794 POINT(40.690090911327474 74.41682051584809) bank140794 +140795 POINT(40.69157114181507 74.38222303510658) bank140795 +140796 POINT(40.88602381083963 73.26212522785497) bank140796 +140797 POINT(41.27468880936058 73.30774627758397) bank140797 +140798 POINT(40.72076404518937 74.33703935831652) bank140798 +140799 POINT(40.4095455556465 73.16061140194657) bank140799 +140800 POINT(41.46726727877967 74.46850422368841) bank140800 +140801 POINT(41.57574320158153 73.72318901689361) bank140801 +140802 POINT(40.148792109310044 74.10782737830073) bank140802 +140803 POINT(40.509431242851214 73.82896145783879) bank140803 +140804 POINT(39.92607459784846 73.12936134916245) bank140804 +140805 POINT(40.97704781924068 74.09810952913713) bank140805 +140806 POINT(40.46340546480823 73.18904212606756) bank140806 +140807 POINT(40.53342075962588 74.59626967062808) bank140807 +140808 POINT(40.551541692272174 74.96427266755724) bank140808 +140809 POINT(41.354127831518944 73.09649066324272) bank140809 +140810 POINT(40.19168560776774 74.98825586548327) bank140810 +140811 POINT(40.7965721139312 73.85779247573275) bank140811 +140812 POINT(40.331004839544924 74.78566854568967) bank140812 +140813 POINT(40.72538370504296 73.39849695589673) bank140813 +140814 POINT(39.791936764782164 74.42798764958081) bank140814 +140815 POINT(41.17329454650385 73.90409555980214) bank140815 +140816 POINT(41.2719092825981 73.4680049036283) bank140816 +140817 POINT(41.281068314803264 74.63430014703745) bank140817 +140818 POINT(39.92989232249633 74.41242142967332) bank140818 +140819 POINT(40.50119770132161 73.20975715730373) bank140819 +140820 POINT(41.2587214090525 74.39625330408592) bank140820 +140821 POINT(40.07245798277439 73.9469890865587) bank140821 +140822 POINT(39.942612350841685 73.67239519229456) bank140822 +140823 POINT(40.85042630466327 73.41821990400862) bank140823 +140824 POINT(40.7733836157162 74.66381761735096) bank140824 +140825 POINT(41.26087309888161 74.30767040109629) bank140825 +140826 POINT(39.76488967922893 74.6882923989929) bank140826 +140827 POINT(40.93499304263079 74.22710449394222) bank140827 +140828 POINT(40.01706426663079 73.49010954295102) bank140828 +140829 POINT(41.09962134766232 74.83033211465583) bank140829 +140830 POINT(40.31519614322215 74.49107936922405) bank140830 +140831 POINT(41.48285992423362 73.42928579930994) bank140831 +140832 POINT(40.184544397150994 73.04266060622072) bank140832 +140833 POINT(40.95934634306488 74.6576976416362) bank140833 +140834 POINT(41.10069987443381 74.57385169121301) bank140834 +140835 POINT(41.274018211917216 74.71016094997697) bank140835 +140836 POINT(41.39231464356904 73.87269248093503) bank140836 +140837 POINT(40.31409595874978 74.13378148136704) bank140837 +140838 POINT(40.95523060812253 73.63340784212986) bank140838 +140839 POINT(40.69891738092988 73.77806120999489) bank140839 +140840 POINT(41.609010753259696 73.01358942782596) bank140840 +140841 POINT(40.679831051874274 74.9428115454134) bank140841 +140842 POINT(40.538147346412536 73.15713037384069) bank140842 +140843 POINT(41.099754261043856 74.2603923634392) bank140843 +140844 POINT(41.39449903955818 74.72581085894187) bank140844 +140845 POINT(39.976135366251356 73.41546195713794) bank140845 +140846 POINT(40.475575016323575 73.9358860671938) bank140846 +140847 POINT(40.415447411411506 73.21897262945033) bank140847 +140848 POINT(41.61713254740636 73.608188673295) bank140848 +140849 POINT(40.492226155413995 73.00986829840207) bank140849 +140850 POINT(40.324114556094365 73.27996905950367) bank140850 +140851 POINT(41.18122131913124 73.88030767776974) bank140851 +140852 POINT(41.67189396162245 73.53666585646278) bank140852 +140853 POINT(40.02045517518609 74.01036414755457) bank140853 +140854 POINT(40.216666653926644 73.04468723965161) bank140854 +140855 POINT(40.72899773113202 73.5736253784988) bank140855 +140856 POINT(40.957110850555914 73.81422342437989) bank140856 +140857 POINT(40.10929923442484 74.49030773339057) bank140857 +140858 POINT(41.36264257346999 74.09054406702833) bank140858 +140859 POINT(41.12332056734348 74.35247638950581) bank140859 +140860 POINT(39.88934125005476 73.37273348109677) bank140860 +140861 POINT(40.789335583088246 73.81620836978148) bank140861 +140862 POINT(41.42247917212093 73.42180666512344) bank140862 +140863 POINT(39.77758791820666 73.51959403541491) bank140863 +140864 POINT(40.22368827519603 74.6113141054862) bank140864 +140865 POINT(40.02839918991868 73.30787354676228) bank140865 +140866 POINT(41.41834077416602 73.26217855671305) bank140866 +140867 POINT(40.615896387822914 73.71166121930428) bank140867 +140868 POINT(39.85470563429438 74.79661745028723) bank140868 +140869 POINT(40.612612131316546 73.78967339452566) bank140869 +140870 POINT(41.58927485655979 73.85326621001494) bank140870 +140871 POINT(40.92023185538964 74.21682374488834) bank140871 +140872 POINT(39.960898178645515 74.09419760287746) bank140872 +140873 POINT(40.09947332369449 73.24400963460599) bank140873 +140874 POINT(41.337933129062506 74.62639136178089) bank140874 +140875 POINT(39.788796208941804 73.92201759796995) bank140875 +140876 POINT(39.862967162719656 74.85583965779573) bank140876 +140877 POINT(41.54357667216189 74.09336863178056) bank140877 +140878 POINT(40.221851697358744 74.63727100873025) bank140878 +140879 POINT(41.695411058702206 73.56273092177382) bank140879 +140880 POINT(39.91182842201207 74.07931451515475) bank140880 +140881 POINT(39.93972430359824 74.2564675278261) bank140881 +140882 POINT(41.470772270492084 73.94869924946276) bank140882 +140883 POINT(40.00092106351721 73.80199330665216) bank140883 +140884 POINT(41.226748742188526 74.54199121586731) bank140884 +140885 POINT(41.07038784093835 73.96276870348333) bank140885 +140886 POINT(41.06870885240597 73.74138626797925) bank140886 +140887 POINT(39.732029444358346 74.03295830601431) bank140887 +140888 POINT(40.36040146976469 74.94929353889829) bank140888 +140889 POINT(41.520797687309354 74.82589769359521) bank140889 +140890 POINT(40.93469874658407 73.03672005232313) bank140890 +140891 POINT(41.39552121001046 74.8759987171781) bank140891 +140892 POINT(39.80402740963962 73.83459143567494) bank140892 +140893 POINT(40.5715370652633 74.78620913675694) bank140893 +140894 POINT(40.66295820897772 74.70492229622738) bank140894 +140895 POINT(41.254730032406385 74.12700584241928) bank140895 +140896 POINT(40.571275770946784 74.16751271912372) bank140896 +140897 POINT(40.64488724737685 73.7828931276816) bank140897 +140898 POINT(40.035414313099544 74.48756689644006) bank140898 +140899 POINT(39.80959458531717 73.30510847217471) bank140899 +140900 POINT(39.90070160251508 74.54772441327427) bank140900 +140901 POINT(40.252199330249034 73.64896995795871) bank140901 +140902 POINT(41.10090261127369 73.90474327673944) bank140902 +140903 POINT(39.94244721119068 73.15086636422889) bank140903 +140904 POINT(40.049677474012206 73.85387752844879) bank140904 +140905 POINT(40.95727349186513 73.47082777302107) bank140905 +140906 POINT(41.25461309741561 74.96235802391996) bank140906 +140907 POINT(41.4645659334579 74.94962098106217) bank140907 +140908 POINT(40.319441120006374 73.27060727494748) bank140908 +140909 POINT(40.77189970429564 74.07969931244315) bank140909 +140910 POINT(40.548732883440486 73.24329217643879) bank140910 +140911 POINT(40.4032476368648 73.26019147175197) bank140911 +140912 POINT(40.65569281613779 74.90171131910103) bank140912 +140913 POINT(40.02205365123972 73.66287910987772) bank140913 +140914 POINT(40.380589974668574 73.53221220155731) bank140914 +140915 POINT(40.83393362227843 73.40302179694038) bank140915 +140916 POINT(41.49915862393196 74.97416516554101) bank140916 +140917 POINT(40.62102845366927 73.64846900237225) bank140917 +140918 POINT(41.10314511285867 74.35228909214852) bank140918 +140919 POINT(39.804880691295374 74.80719210311989) bank140919 +140920 POINT(41.30501298888886 74.02638377860033) bank140920 +140921 POINT(40.304800613732716 73.24955681822514) bank140921 +140922 POINT(41.71085731181304 74.87839270136402) bank140922 +140923 POINT(40.76275954829842 74.43605599656088) bank140923 +140924 POINT(39.99607660635279 74.66526001604952) bank140924 +140925 POINT(40.87854654712139 73.4455554393112) bank140925 +140926 POINT(41.10923622224662 73.0643444558003) bank140926 +140927 POINT(39.78722976293215 74.44555609053266) bank140927 +140928 POINT(41.43569038865078 74.0471704775717) bank140928 +140929 POINT(40.113394065857065 74.85158608684912) bank140929 +140930 POINT(41.40842233337654 74.95541238356985) bank140930 +140931 POINT(41.375547807201606 74.69209616596106) bank140931 +140932 POINT(40.04374388589263 74.75330589713936) bank140932 +140933 POINT(41.45975793384897 73.17387460196046) bank140933 +140934 POINT(40.677500115544575 73.5718293167362) bank140934 +140935 POINT(40.66488176023972 74.06131220297748) bank140935 +140936 POINT(41.36289969268691 73.51107775030651) bank140936 +140937 POINT(40.14552501782392 74.4235407576733) bank140937 +140938 POINT(41.03560040478625 74.91185266679325) bank140938 +140939 POINT(40.55877818477469 73.43405593571202) bank140939 +140940 POINT(41.645983524673994 74.51789536206478) bank140940 +140941 POINT(41.054580561597994 73.88515688308657) bank140941 +140942 POINT(40.07735442505848 73.01368762627726) bank140942 +140943 POINT(41.2139451462661 73.40245049603293) bank140943 +140944 POINT(41.38289551560327 74.7100335510269) bank140944 +140945 POINT(39.940113120701554 74.01487223946289) bank140945 +140946 POINT(41.669456937605915 73.26624003978522) bank140946 +140947 POINT(40.782792204503366 74.09472990177552) bank140947 +140948 POINT(41.04540694858627 74.73200622099705) bank140948 +140949 POINT(39.80922738697819 73.51681933594088) bank140949 +140950 POINT(40.38816565198578 74.37115363681245) bank140950 +140951 POINT(40.47768684473226 74.7731085322096) bank140951 +140952 POINT(40.99692150982929 74.68497971069937) bank140952 +140953 POINT(40.23459517490309 74.89360071288395) bank140953 +140954 POINT(40.112064392136226 74.38655618898545) bank140954 +140955 POINT(40.19802484462388 73.40431552873534) bank140955 +140956 POINT(39.8869079526445 74.58788961587148) bank140956 +140957 POINT(40.42388469712017 74.87241010537623) bank140957 +140958 POINT(41.37295340727366 73.80835778393774) bank140958 +140959 POINT(39.84335455184353 73.99605277044253) bank140959 +140960 POINT(41.518715986442054 74.52812394141473) bank140960 +140961 POINT(39.990371466353615 73.82784981021753) bank140961 +140962 POINT(40.38090166957723 73.84894126514227) bank140962 +140963 POINT(40.037307612114034 74.44700405549655) bank140963 +140964 POINT(40.99739891067164 74.61184314898992) bank140964 +140965 POINT(40.39128782031044 74.63112845042545) bank140965 +140966 POINT(40.79816763528933 74.27678946513184) bank140966 +140967 POINT(40.140489457250965 74.98531116577848) bank140967 +140968 POINT(41.20083281227859 74.16839192584271) bank140968 +140969 POINT(40.872346847648366 73.71514201162859) bank140969 +140970 POINT(39.916811464880546 73.75173537492746) bank140970 +140971 POINT(39.74468548800409 74.22944292944459) bank140971 +140972 POINT(40.65674010433985 74.78345218873298) bank140972 +140973 POINT(40.42596121435023 74.71724008345258) bank140973 +140974 POINT(40.20445576486056 73.13208879542717) bank140974 +140975 POINT(40.601993086211976 74.4340590988029) bank140975 +140976 POINT(41.16098970198111 74.05789400120001) bank140976 +140977 POINT(40.25650614789191 74.37993163613446) bank140977 +140978 POINT(40.906433557676465 74.98871678158129) bank140978 +140979 POINT(40.96302205116615 74.19765887943788) bank140979 +140980 POINT(40.34309134505448 73.99559298645185) bank140980 +140981 POINT(40.11967800777671 73.57066437182404) bank140981 +140982 POINT(40.164484705043826 73.67550052027332) bank140982 +140983 POINT(40.27327247183871 74.14730020895047) bank140983 +140984 POINT(41.20353834850835 74.60577630163169) bank140984 +140985 POINT(39.81551729348982 74.22820549070927) bank140985 +140986 POINT(40.67827322536436 73.27430153896583) bank140986 +140987 POINT(40.610435392080674 73.2791626460787) bank140987 +140988 POINT(40.68415138826085 73.28650227586289) bank140988 +140989 POINT(40.912396905752544 73.36445817680064) bank140989 +140990 POINT(39.88502105763452 74.30761868883774) bank140990 +140991 POINT(39.94208431430276 73.33964848663837) bank140991 +140992 POINT(41.602732577874626 74.31136081010553) bank140992 +140993 POINT(40.152945269020165 74.34138082253622) bank140993 +140994 POINT(41.48743213528693 74.42234711827929) bank140994 +140995 POINT(40.07282670932232 74.78928415166894) bank140995 +140996 POINT(41.44155484806259 74.51524366737316) bank140996 +140997 POINT(41.62378236062272 73.27179054595634) bank140997 +140998 POINT(39.87116711505961 73.75373518826646) bank140998 +140999 POINT(40.76249126001621 74.02589239585829) bank140999 +141000 POINT(40.671769175790196 74.2073787598065) bank141000 +141001 POINT(41.28674630443439 74.87383119352232) bank141001 +141002 POINT(40.258264415910155 73.81076740481278) bank141002 +141003 POINT(40.00638070101001 73.34481033799197) bank141003 +141004 POINT(41.310507650282105 74.02682578226711) bank141004 +141005 POINT(41.38489263776633 73.02590076715283) bank141005 +141006 POINT(40.809115753607244 74.7042567325942) bank141006 +141007 POINT(40.639371744173005 73.5465019111359) bank141007 +141008 POINT(40.155209193474434 74.73296644023928) bank141008 +141009 POINT(40.458771888138706 74.20066018695132) bank141009 +141010 POINT(40.13105829758149 74.18034043497194) bank141010 +141011 POINT(40.49881485248984 74.48391405511356) bank141011 +141012 POINT(40.54821306584311 74.4188762632572) bank141012 +141013 POINT(41.572737384885976 73.76717269780892) bank141013 +141014 POINT(40.422856035657574 74.77905113523227) bank141014 +141015 POINT(39.96759565119339 73.59539580415026) bank141015 +141016 POINT(39.99031198265368 73.88478471689959) bank141016 +141017 POINT(41.19002800639714 73.87258353528152) bank141017 +141018 POINT(41.19672039077358 74.8461224787578) bank141018 +141019 POINT(41.68311918545813 73.60171014814986) bank141019 +141020 POINT(40.13644421673773 73.01897230468684) bank141020 +141021 POINT(39.771281758047074 73.15683171955367) bank141021 +141022 POINT(41.41166988622293 74.3494239613452) bank141022 +141023 POINT(40.50139386046925 74.08236040973685) bank141023 +141024 POINT(41.568209129802554 73.13056758300166) bank141024 +141025 POINT(41.35233741149643 74.10407675620361) bank141025 +141026 POINT(41.688552315421035 73.76398762831718) bank141026 +141027 POINT(40.16667322896588 73.43742038029109) bank141027 +141028 POINT(41.01662687978849 74.01535265197349) bank141028 +141029 POINT(41.20983727281192 73.44885406601152) bank141029 +141030 POINT(40.544428233268306 73.44109139920353) bank141030 +141031 POINT(40.46393897016199 74.83631402131427) bank141031 +141032 POINT(41.4189584665197 73.57535005860748) bank141032 +141033 POINT(41.56291795056342 73.74193600396657) bank141033 +141034 POINT(40.47666674652962 73.47012771189408) bank141034 +141035 POINT(40.54759645466448 74.215545281589) bank141035 +141036 POINT(40.688077275474186 73.60972950125051) bank141036 +141037 POINT(40.35988812089747 74.33833248780397) bank141037 +141038 POINT(40.946550107395005 74.15649240404983) bank141038 +141039 POINT(40.51037427615997 74.29502656006716) bank141039 +141040 POINT(41.32118686996864 74.70836746186487) bank141040 +141041 POINT(41.34209948949852 74.56007380886395) bank141041 +141042 POINT(40.16892009344369 73.73114116813689) bank141042 +141043 POINT(40.92023231297642 73.79183034833497) bank141043 +141044 POINT(40.62664894845615 74.70504683237478) bank141044 +141045 POINT(41.71070355013482 73.12292798999704) bank141045 +141046 POINT(39.97612512688589 73.2605300935679) bank141046 +141047 POINT(39.878798648286654 74.02018261509875) bank141047 +141048 POINT(40.91443478624767 74.50241489746485) bank141048 +141049 POINT(40.93118091517684 73.637467666291) bank141049 +141050 POINT(40.57344202863924 74.7696396732152) bank141050 +141051 POINT(40.53343534896389 74.82072226302111) bank141051 +141052 POINT(41.6469841597132 74.92453161806648) bank141052 +141053 POINT(41.229822407615686 73.68088575026799) bank141053 +141054 POINT(41.46406954188622 74.04274196028247) bank141054 +141055 POINT(41.00195286569224 74.71024106496874) bank141055 +141056 POINT(41.034621131759344 73.33514410515762) bank141056 +141057 POINT(41.16771802318556 73.39442209525153) bank141057 +141058 POINT(41.39153646932624 74.98388148779294) bank141058 +141059 POINT(40.77542248427394 73.04092845028883) bank141059 +141060 POINT(40.939727219458014 74.00412127207123) bank141060 +141061 POINT(41.090772948805466 73.61544516989476) bank141061 +141062 POINT(40.428955935009505 73.01658532775383) bank141062 +141063 POINT(40.610430492925424 73.2578600666591) bank141063 +141064 POINT(41.5439888341424 73.13435732061082) bank141064 +141065 POINT(40.75345127284586 74.84476614163734) bank141065 +141066 POINT(40.47055001688848 74.91654649506597) bank141066 +141067 POINT(39.882079876858974 74.05249267622007) bank141067 +141068 POINT(40.09164728191381 74.10515741818287) bank141068 +141069 POINT(40.08651979041959 73.7838013757893) bank141069 +141070 POINT(40.73007555181153 73.78526381022716) bank141070 +141071 POINT(41.3289547522378 73.06072329831014) bank141071 +141072 POINT(41.057025759142554 74.11621447844443) bank141072 +141073 POINT(41.258933976628924 73.62597468597058) bank141073 +141074 POINT(40.76057803077203 73.74660670702815) bank141074 +141075 POINT(40.69992793413919 74.9708412728376) bank141075 +141076 POINT(41.0456485749747 73.3436084710073) bank141076 +141077 POINT(39.82398926406745 73.76693433065505) bank141077 +141078 POINT(41.56630546416913 74.59210802043243) bank141078 +141079 POINT(40.192092674226004 73.27170339034497) bank141079 +141080 POINT(40.43888687848825 73.44745982948614) bank141080 +141081 POINT(40.937407890199964 73.58105038675771) bank141081 +141082 POINT(40.23757553225292 73.28445332505578) bank141082 +141083 POINT(41.25645570064775 73.20826118269332) bank141083 +141084 POINT(40.30573312741809 74.22641889380473) bank141084 +141085 POINT(40.31489951674283 74.0836932316431) bank141085 +141086 POINT(40.22968168005701 74.61198944363512) bank141086 +141087 POINT(40.986122736750495 73.32687921408142) bank141087 +141088 POINT(39.82494812849492 74.2649123954279) bank141088 +141089 POINT(40.43229411706795 74.71551173690546) bank141089 +141090 POINT(40.95595913219554 73.79427396427815) bank141090 +141091 POINT(40.21197442523313 74.92312000375043) bank141091 +141092 POINT(40.98084647132664 73.74196902384172) bank141092 +141093 POINT(39.91778642256849 74.33473346933678) bank141093 +141094 POINT(40.013052715598484 73.57070697302123) bank141094 +141095 POINT(41.14384695160266 74.6553828613232) bank141095 +141096 POINT(41.073884854257436 73.16621052434657) bank141096 +141097 POINT(41.175649018642375 73.05080380229386) bank141097 +141098 POINT(40.09600890032264 73.48075914557121) bank141098 +141099 POINT(40.85091164407131 73.44752344581208) bank141099 +141100 POINT(41.7081240243509 74.96215526992466) bank141100 +141101 POINT(40.96417112103215 74.59216535750195) bank141101 +141102 POINT(40.37001852780689 73.26915837204878) bank141102 +141103 POINT(40.514200614109825 73.99338121444387) bank141103 +141104 POINT(40.3046351196116 74.97019051917661) bank141104 +141105 POINT(41.02541707311381 74.94023293490467) bank141105 +141106 POINT(41.490325652204284 73.09027257229545) bank141106 +141107 POINT(41.24997832394058 74.35241719945799) bank141107 +141108 POINT(39.882305403408076 73.56504902140107) bank141108 +141109 POINT(41.48408128554359 74.19492480485567) bank141109 +141110 POINT(40.43466526398916 74.52628625830766) bank141110 +141111 POINT(40.13684603613431 74.01159919193739) bank141111 +141112 POINT(40.39532056691597 73.50895587134426) bank141112 +141113 POINT(41.397461576131164 73.34510213340992) bank141113 +141114 POINT(40.02521042372209 73.1345457316187) bank141114 +141115 POINT(40.91922817963345 73.03878308670066) bank141115 +141116 POINT(39.96617268163996 73.4104252715629) bank141116 +141117 POINT(40.13284327600488 73.68677480063505) bank141117 +141118 POINT(41.55028763164969 73.39159763875065) bank141118 +141119 POINT(40.82792060560055 73.13948497428842) bank141119 +141120 POINT(40.28776541590141 73.37620707357931) bank141120 +141121 POINT(41.597964675259476 73.31906693941858) bank141121 +141122 POINT(40.21310313850104 74.4016884654) bank141122 +141123 POINT(39.81041609773796 73.74479890450708) bank141123 +141124 POINT(40.99237196318864 73.37230880810596) bank141124 +141125 POINT(40.832849839445174 74.12854999632873) bank141125 +141126 POINT(41.02185437121641 74.64752420930172) bank141126 +141127 POINT(41.01424555932405 73.56704698534658) bank141127 +141128 POINT(41.084769963359896 73.76562495217807) bank141128 +141129 POINT(41.303252974755054 73.29254315956125) bank141129 +141130 POINT(40.693961585308195 74.95061314228536) bank141130 +141131 POINT(41.19845454926721 73.48612604084232) bank141131 +141132 POINT(40.63220246627852 74.43528905560552) bank141132 +141133 POINT(41.29735997743078 74.122672355256) bank141133 +141134 POINT(40.35345686757808 73.34196582776909) bank141134 +141135 POINT(40.28630272335275 74.78365036270252) bank141135 +141136 POINT(41.1398049367433 73.60094592836025) bank141136 +141137 POINT(41.6637625708082 74.86001146779323) bank141137 +141138 POINT(41.01093873258887 73.53320859121519) bank141138 +141139 POINT(39.94935327454924 74.41841417853459) bank141139 +141140 POINT(41.546159216666254 74.17099770904463) bank141140 +141141 POINT(40.70669423011905 73.43583167151624) bank141141 +141142 POINT(40.517596022966245 73.13184991232583) bank141142 +141143 POINT(40.20453592717138 73.14225433521766) bank141143 +141144 POINT(39.72316344695118 73.86313120711667) bank141144 +141145 POINT(41.33298792391796 73.33852400436358) bank141145 +141146 POINT(39.97729222777559 74.6528912645319) bank141146 +141147 POINT(41.337593562276034 74.88709043551599) bank141147 +141148 POINT(40.20953280965755 73.82628568321377) bank141148 +141149 POINT(41.51001673760489 73.80901791971306) bank141149 +141150 POINT(40.719485121957405 74.82722659221717) bank141150 +141151 POINT(40.52339606739542 73.40972867748603) bank141151 +141152 POINT(40.046731064690235 73.8090797538816) bank141152 +141153 POINT(40.074729560258575 74.06633110059093) bank141153 +141154 POINT(41.63429026452607 74.99340705623474) bank141154 +141155 POINT(40.36674356121976 73.62244890973697) bank141155 +141156 POINT(39.96611135266102 73.50208611491124) bank141156 +141157 POINT(41.367477670956504 74.54222679101709) bank141157 +141158 POINT(41.60198055540161 73.42206525720945) bank141158 +141159 POINT(40.49996454392326 73.06005067247494) bank141159 +141160 POINT(39.73518701782194 73.73523036490461) bank141160 +141161 POINT(41.238072095688956 73.75763793106775) bank141161 +141162 POINT(41.30999395880348 73.37460846849099) bank141162 +141163 POINT(40.08147947672014 73.49286785103368) bank141163 +141164 POINT(39.80974604716886 74.45903229074135) bank141164 +141165 POINT(41.118808751891834 74.51169147107635) bank141165 +141166 POINT(41.0800200120911 73.31678406117615) bank141166 +141167 POINT(40.81401628483226 74.77072395258624) bank141167 +141168 POINT(40.37298385017086 74.63184468041386) bank141168 +141169 POINT(41.61840537167892 73.28541320781696) bank141169 +141170 POINT(41.29905355144708 73.94777441518804) bank141170 +141171 POINT(40.13494996196747 74.73133216772614) bank141171 +141172 POINT(39.7469951955568 73.44139021338552) bank141172 +141173 POINT(41.155032834376875 73.43737169013382) bank141173 +141174 POINT(41.54477910330548 74.88703590109327) bank141174 +141175 POINT(41.238800100250046 73.70452508330604) bank141175 +141176 POINT(41.62468324367546 74.98259570162904) bank141176 +141177 POINT(41.691313251488225 74.92740185046607) bank141177 +141178 POINT(41.548026953637596 74.16468091826178) bank141178 +141179 POINT(40.34593397943639 73.34726509049094) bank141179 +141180 POINT(40.313671399948696 73.14228989262025) bank141180 +141181 POINT(40.81533862993566 73.44102253438595) bank141181 +141182 POINT(40.77573149512474 74.47663284403488) bank141182 +141183 POINT(41.596987592694006 74.14520538286321) bank141183 +141184 POINT(39.79801141722966 73.16806031326932) bank141184 +141185 POINT(41.046761564882964 74.89432871635711) bank141185 +141186 POINT(40.43118485977788 73.88226829864308) bank141186 +141187 POINT(40.40438538336097 74.19240789262065) bank141187 +141188 POINT(39.92933028033398 73.37064205101608) bank141188 +141189 POINT(41.33785569017434 75.004837037582) bank141189 +141190 POINT(40.816078071590674 74.34781470601168) bank141190 +141191 POINT(40.97108077133664 74.74982822990177) bank141191 +141192 POINT(40.856111679127 74.08498585688899) bank141192 +141193 POINT(40.39608969229237 73.46805857764069) bank141193 +141194 POINT(41.14003691020979 74.1251990263246) bank141194 +141195 POINT(40.713439391441035 73.55511923527446) bank141195 +141196 POINT(40.329776550530134 74.7019798263117) bank141196 +141197 POINT(41.24132783014549 74.97846917695469) bank141197 +141198 POINT(41.263014638421055 74.09348514296528) bank141198 +141199 POINT(40.26588404827617 73.76286782034909) bank141199 +141200 POINT(40.28527309405619 74.11493062753145) bank141200 +141201 POINT(41.28156087115959 74.07587745249609) bank141201 +141202 POINT(39.712946083191476 74.17342321973918) bank141202 +141203 POINT(40.56172898761902 74.90752218161387) bank141203 +141204 POINT(40.222114915076 74.24593201673181) bank141204 +141205 POINT(40.1839680320777 73.11152312489888) bank141205 +141206 POINT(39.88997535337226 74.89784224664854) bank141206 +141207 POINT(40.86537085332539 73.60951678604556) bank141207 +141208 POINT(41.46849776461711 73.44483651213643) bank141208 +141209 POINT(39.79838026539442 73.27518529904218) bank141209 +141210 POINT(40.118916756945 73.60167150677862) bank141210 +141211 POINT(41.50192568755796 74.72060104660707) bank141211 +141212 POINT(41.12375827464564 73.43308660022369) bank141212 +141213 POINT(39.80471085500893 73.37695428249147) bank141213 +141214 POINT(39.97073222554607 73.00992335790545) bank141214 +141215 POINT(39.78547538631737 74.41363714929625) bank141215 +141216 POINT(40.75861520757265 74.5134561576794) bank141216 +141217 POINT(40.602869202495675 74.56430112135442) bank141217 +141218 POINT(41.51200909818245 73.29904069253502) bank141218 +141219 POINT(41.54071486701546 73.47561562502528) bank141219 +141220 POINT(40.97271560882272 74.26317838296609) bank141220 +141221 POINT(40.75288743628596 74.77295549038884) bank141221 +141222 POINT(40.790647947274415 74.25444844975516) bank141222 +141223 POINT(41.36714936904333 74.0503289478722) bank141223 +141224 POINT(40.15483036536426 73.19581348669297) bank141224 +141225 POINT(40.77296094581941 73.73767713991877) bank141225 +141226 POINT(40.70550363876037 74.06771374594142) bank141226 +141227 POINT(40.47380985714288 73.54537633095975) bank141227 +141228 POINT(41.37341218976958 74.16432073779909) bank141228 +141229 POINT(40.065583117069075 74.27607623621306) bank141229 +141230 POINT(41.399180600171334 73.16366693519248) bank141230 +141231 POINT(41.665004606900176 74.14373941836836) bank141231 +141232 POINT(41.23720128283312 73.57034962888852) bank141232 +141233 POINT(40.11282309135671 73.11742412556643) bank141233 +141234 POINT(40.141870804934676 74.34460871763991) bank141234 +141235 POINT(40.387271458330865 74.12144176303977) bank141235 +141236 POINT(39.7858262058366 74.60436809741108) bank141236 +141237 POINT(40.27114799825434 74.12165665359653) bank141237 +141238 POINT(39.9511273941774 73.11409437044553) bank141238 +141239 POINT(39.82284703277132 74.75403845413152) bank141239 +141240 POINT(40.23305184483867 73.90768806707962) bank141240 +141241 POINT(40.326032138202535 73.93011166932706) bank141241 +141242 POINT(41.05354519526838 73.47725676054513) bank141242 +141243 POINT(41.429466987919184 73.06840710983363) bank141243 +141244 POINT(41.393987218865036 73.43079196139492) bank141244 +141245 POINT(41.53888095358767 73.53322816273983) bank141245 +141246 POINT(41.53906043034266 74.18422934225754) bank141246 +141247 POINT(39.88168989234374 73.03573943127424) bank141247 +141248 POINT(40.50151680781275 74.40846889704726) bank141248 +141249 POINT(40.695222529039725 73.5564177751909) bank141249 +141250 POINT(41.473622803338245 73.13753983820352) bank141250 +141251 POINT(40.70619074045167 73.92173812043958) bank141251 +141252 POINT(40.13335948186167 73.21952914073792) bank141252 +141253 POINT(40.94161780869923 73.7341450267542) bank141253 +141254 POINT(41.71026292794229 73.05035189863301) bank141254 +141255 POINT(41.399902218992544 73.73700842293248) bank141255 +141256 POINT(40.776569760664806 73.15938213295053) bank141256 +141257 POINT(39.934577771470146 73.58387756762096) bank141257 +141258 POINT(40.46354779358287 73.8215479226304) bank141258 +141259 POINT(41.41867644430351 73.13100216764028) bank141259 +141260 POINT(40.95846536360606 74.46866433546072) bank141260 +141261 POINT(39.78690559373044 74.7099261759731) bank141261 +141262 POINT(40.74960069592728 73.09825927534933) bank141262 +141263 POINT(40.38526960479478 74.17222369120158) bank141263 +141264 POINT(41.47182819156191 73.1880239190749) bank141264 +141265 POINT(40.28617395012124 74.06852067183736) bank141265 +141266 POINT(40.58712218675797 74.22401540301244) bank141266 +141267 POINT(41.63119449763441 74.16276014664506) bank141267 +141268 POINT(40.30682858169666 74.11565416248021) bank141268 +141269 POINT(41.63509494060899 73.22265871293058) bank141269 +141270 POINT(40.585887145631425 73.32616838643573) bank141270 +141271 POINT(39.91534215792009 73.04736014057899) bank141271 +141272 POINT(41.634839375519746 74.12716967590535) bank141272 +141273 POINT(40.90723067191079 74.56060206308021) bank141273 +141274 POINT(41.62286658955429 74.18590917342958) bank141274 +141275 POINT(40.7984927667696 73.53998975661013) bank141275 +141276 POINT(41.62646265555296 73.68891517902432) bank141276 +141277 POINT(40.28877797653397 74.77617566333977) bank141277 +141278 POINT(41.254691590817636 74.87005098332884) bank141278 +141279 POINT(40.547511198961764 73.29523052618795) bank141279 +141280 POINT(40.63118160351278 73.32014856486015) bank141280 +141281 POINT(40.43998296782086 74.73532011484959) bank141281 +141282 POINT(40.2835739902774 74.94632566119579) bank141282 +141283 POINT(40.7313784692383 74.07819239245525) bank141283 +141284 POINT(40.59969108396431 73.17815594762588) bank141284 +141285 POINT(41.56786465258874 74.33740806037328) bank141285 +141286 POINT(39.88623102128643 73.7666113943463) bank141286 +141287 POINT(40.03789641383651 73.43728923462974) bank141287 +141288 POINT(39.98626011190931 73.85081398126879) bank141288 +141289 POINT(40.76887720546221 74.75747726212279) bank141289 +141290 POINT(40.01596218845792 74.31261543437276) bank141290 +141291 POINT(40.06076517781606 74.21325897949222) bank141291 +141292 POINT(39.87974615432608 74.21024228160535) bank141292 +141293 POINT(40.74586632993202 73.92713948474844) bank141293 +141294 POINT(40.39696330821699 73.01777770239231) bank141294 +141295 POINT(40.19577247930681 73.43517458124896) bank141295 +141296 POINT(41.41276411204061 74.31683449001646) bank141296 +141297 POINT(39.9635268570578 74.74499634128365) bank141297 +141298 POINT(41.330613052421036 73.80467560986237) bank141298 +141299 POINT(41.43107489480196 73.77142953605546) bank141299 +141300 POINT(40.62657380909858 73.12820310558834) bank141300 +141301 POINT(40.804581816651535 74.3956535254346) bank141301 +141302 POINT(40.50797586395201 73.32302089051187) bank141302 +141303 POINT(39.95043694603003 74.12935185655661) bank141303 +141304 POINT(41.520687856114 74.64661605122316) bank141304 +141305 POINT(40.6013978179992 74.68221221396536) bank141305 +141306 POINT(40.16015435259609 73.38691389575696) bank141306 +141307 POINT(40.610269595394904 74.29053221628699) bank141307 +141308 POINT(40.61175562953492 73.23227461433532) bank141308 +141309 POINT(40.57627209953998 74.36886568812243) bank141309 +141310 POINT(40.429582257881066 74.979322900702) bank141310 +141311 POINT(40.85357534691223 74.07668873038493) bank141311 +141312 POINT(41.33731297988118 73.35126664840615) bank141312 +141313 POINT(40.68558646881637 74.08259514459215) bank141313 +141314 POINT(40.019378239904526 74.83177454023702) bank141314 +141315 POINT(41.090706195790844 74.36503283760294) bank141315 +141316 POINT(40.733403005883304 73.30374351219966) bank141316 +141317 POINT(41.02615575089341 74.06990443810699) bank141317 +141318 POINT(40.45787836566383 73.62403729807714) bank141318 +141319 POINT(40.67393732141928 73.5832418475845) bank141319 +141320 POINT(41.29164603279224 73.125936179902) bank141320 +141321 POINT(41.6867097568793 73.7539412894748) bank141321 +141322 POINT(41.15536664191843 74.89251207354513) bank141322 +141323 POINT(40.47297683314179 73.21282015011441) bank141323 +141324 POINT(41.471255640651364 73.48941874962391) bank141324 +141325 POINT(39.97438141682701 74.58618534171782) bank141325 +141326 POINT(40.29426940678868 73.55333123726932) bank141326 +141327 POINT(40.250373161402514 74.63543757081987) bank141327 +141328 POINT(41.711165913410966 74.82139693704333) bank141328 +141329 POINT(41.39039664508805 74.75597049042305) bank141329 +141330 POINT(41.4351824906462 74.07187911249633) bank141330 +141331 POINT(41.66823576300801 74.142508229689) bank141331 +141332 POINT(40.85770483792433 74.82028597818476) bank141332 +141333 POINT(39.94598614583548 74.87899409193987) bank141333 +141334 POINT(40.32847130538297 74.12926466631379) bank141334 +141335 POINT(41.47389622796256 74.87141599777802) bank141335 +141336 POINT(40.77195088719476 73.41361097115568) bank141336 +141337 POINT(39.913700367287035 74.57776405292944) bank141337 +141338 POINT(40.21057612360477 74.76214732235066) bank141338 +141339 POINT(39.953605654751414 74.66609322793268) bank141339 +141340 POINT(40.73459702461514 74.27352682735155) bank141340 +141341 POINT(41.129682393107956 73.93741107359041) bank141341 +141342 POINT(41.40715138277685 74.12370834405544) bank141342 +141343 POINT(40.25230806805414 73.85023647261136) bank141343 +141344 POINT(40.20879912688181 73.73342470801741) bank141344 +141345 POINT(40.84195482585147 74.3922078822693) bank141345 +141346 POINT(40.76130213270204 74.84155518293466) bank141346 +141347 POINT(40.63429804177137 73.24653162375648) bank141347 +141348 POINT(40.69785815308662 73.94119261846414) bank141348 +141349 POINT(40.50026189761751 74.5713907763115) bank141349 +141350 POINT(41.29647943849473 74.65685291171528) bank141350 +141351 POINT(41.58891173506194 73.28879705416117) bank141351 +141352 POINT(40.35676994628816 74.57220278806899) bank141352 +141353 POINT(40.98265293406941 73.24511525195264) bank141353 +141354 POINT(40.68144653517767 73.6333875054535) bank141354 +141355 POINT(39.74599718549482 74.58804212787287) bank141355 +141356 POINT(40.655798737271695 74.18565515667343) bank141356 +141357 POINT(40.52660897071056 73.08201080150049) bank141357 +141358 POINT(39.989319968289024 74.28452070645528) bank141358 +141359 POINT(40.57139705234193 73.02345180930631) bank141359 +141360 POINT(41.01015005587945 74.9023377399892) bank141360 +141361 POINT(41.383746435282276 73.76816961700239) bank141361 +141362 POINT(40.07633619817776 74.4096651385222) bank141362 +141363 POINT(40.48085703903966 74.00442366526829) bank141363 +141364 POINT(40.27697446579189 73.57632918622205) bank141364 +141365 POINT(41.17411557581105 74.50001507180082) bank141365 +141366 POINT(41.63387298124675 74.88882988913903) bank141366 +141367 POINT(40.617108355037075 74.06570217957774) bank141367 +141368 POINT(40.59946078016816 74.63147337957136) bank141368 +141369 POINT(39.9760382690598 74.31870428807932) bank141369 +141370 POINT(40.51819190744979 73.88179939575635) bank141370 +141371 POINT(41.30739442593787 74.8227678464723) bank141371 +141372 POINT(41.02901973042914 74.90242170463542) bank141372 +141373 POINT(41.64777421475143 73.37082122428288) bank141373 +141374 POINT(41.45493177255068 73.13279043304266) bank141374 +141375 POINT(40.08872666642942 74.6322566910014) bank141375 +141376 POINT(39.78327787161002 73.27331523990823) bank141376 +141377 POINT(39.93941029175862 74.09605600282491) bank141377 +141378 POINT(41.238735717892844 73.6255721040153) bank141378 +141379 POINT(41.35118864621703 74.95869597982676) bank141379 +141380 POINT(40.55979540864681 73.39139905980177) bank141380 +141381 POINT(39.86836255090176 73.24459033051373) bank141381 +141382 POINT(39.88109352711622 74.13628934741365) bank141382 +141383 POINT(40.02472760246103 74.35644269930926) bank141383 +141384 POINT(40.70557391483675 73.67848821943501) bank141384 +141385 POINT(40.146447289443266 73.9349084814145) bank141385 +141386 POINT(41.52674721706798 74.28757100449481) bank141386 +141387 POINT(39.970037790097344 73.66724042361652) bank141387 +141388 POINT(41.63555856755418 74.69634265854769) bank141388 +141389 POINT(40.28677687559886 73.75893442497802) bank141389 +141390 POINT(40.73823770134641 73.97479370011412) bank141390 +141391 POINT(39.86919638508507 74.03210547988826) bank141391 +141392 POINT(40.541036417633535 74.31496273398798) bank141392 +141393 POINT(39.92040508290687 74.05817349166716) bank141393 +141394 POINT(40.16802617757011 73.32030539666866) bank141394 +141395 POINT(40.83886673127738 73.44122771144595) bank141395 +141396 POINT(39.751924754456674 73.97051137751056) bank141396 +141397 POINT(41.67637615330573 74.89568874445371) bank141397 +141398 POINT(41.21777446832604 73.94388677655684) bank141398 +141399 POINT(39.8428373802819 73.29192890472297) bank141399 +141400 POINT(40.20497691715647 74.40847991100178) bank141400 +141401 POINT(40.586966473256844 73.7282922735235) bank141401 +141402 POINT(40.61139806606154 73.84500007317003) bank141402 +141403 POINT(40.952644602825295 73.28700904430849) bank141403 +141404 POINT(41.144017266829394 73.30478788185042) bank141404 +141405 POINT(40.3052585337557 74.22956982152701) bank141405 +141406 POINT(40.7999667076227 74.28348603927998) bank141406 +141407 POINT(39.926219897753505 73.43678245048645) bank141407 +141408 POINT(41.66641967438868 74.74069630384713) bank141408 +141409 POINT(39.891587089973505 73.88368941850251) bank141409 +141410 POINT(40.327191633306356 73.49252297681969) bank141410 +141411 POINT(40.92215111485342 73.79489307028992) bank141411 +141412 POINT(40.97183882401293 74.52032955526755) bank141412 +141413 POINT(39.79423066621594 74.12820369981932) bank141413 +141414 POINT(40.366472214235706 74.79557516435253) bank141414 +141415 POINT(41.11571111409413 74.84157119848338) bank141415 +141416 POINT(40.1868376285376 73.74813417312706) bank141416 +141417 POINT(40.29440117551799 73.339092364594) bank141417 +141418 POINT(40.92198963062206 74.68636378790681) bank141418 +141419 POINT(40.43942303306785 73.5078134599282) bank141419 +141420 POINT(40.608374587698556 73.9228179714765) bank141420 +141421 POINT(40.61634517286717 74.66419779093503) bank141421 +141422 POINT(40.10856636877554 74.3426818227693) bank141422 +141423 POINT(41.2856282624861 73.81381319568212) bank141423 +141424 POINT(40.36064495245313 74.52862988263641) bank141424 +141425 POINT(41.22747089743223 74.19322775712705) bank141425 +141426 POINT(40.711646015145 74.12338273448474) bank141426 +141427 POINT(39.92906178743624 73.07214806922869) bank141427 +141428 POINT(40.00573150987488 73.01809112522486) bank141428 +141429 POINT(39.76846692580704 73.32278355542762) bank141429 +141430 POINT(40.88085382156739 74.44621008302603) bank141430 +141431 POINT(40.928676586930344 73.3409453780677) bank141431 +141432 POINT(40.67209410392708 74.78843824836665) bank141432 +141433 POINT(40.52893865907806 74.7856344156562) bank141433 +141434 POINT(39.75345604266616 73.54014306755614) bank141434 +141435 POINT(41.63591387497834 74.5246042775327) bank141435 +141436 POINT(40.05732337848519 73.11782644048319) bank141436 +141437 POINT(40.664481266379525 74.53435411470515) bank141437 +141438 POINT(40.06538993992697 73.21460525790876) bank141438 +141439 POINT(40.536292767372245 73.29943127191339) bank141439 +141440 POINT(40.95019890415586 74.11889938637194) bank141440 +141441 POINT(40.19304148479002 74.41811703108182) bank141441 +141442 POINT(39.7505292923091 73.85260727807278) bank141442 +141443 POINT(41.07353012387485 73.6955299616453) bank141443 +141444 POINT(40.37441223528734 74.03199418986664) bank141444 +141445 POINT(39.71446292160338 74.72145926712075) bank141445 +141446 POINT(41.27126179467181 74.58605846520483) bank141446 +141447 POINT(41.078129205639684 74.29943157608382) bank141447 +141448 POINT(40.39909579783447 74.1481611115479) bank141448 +141449 POINT(40.419595379391374 74.01213467370147) bank141449 +141450 POINT(39.89056306383426 73.44934775991334) bank141450 +141451 POINT(40.09579095312787 74.13179814663444) bank141451 +141452 POINT(41.61427630450395 74.50677778596561) bank141452 +141453 POINT(39.75410349474124 74.908248244519) bank141453 +141454 POINT(39.78996550473774 74.51995306555995) bank141454 +141455 POINT(41.113111967113404 74.91977671913665) bank141455 +141456 POINT(40.578199480419606 74.85145609927017) bank141456 +141457 POINT(41.475823795198615 73.24275103647025) bank141457 +141458 POINT(39.7877306897643 74.80129994152718) bank141458 +141459 POINT(40.716558343116745 74.55040810597754) bank141459 +141460 POINT(40.91628033245438 74.78062253912773) bank141460 +141461 POINT(40.347968911048724 73.89789621621952) bank141461 +141462 POINT(40.90666702645544 74.79602642316179) bank141462 +141463 POINT(40.74489413714707 73.81962027207913) bank141463 +141464 POINT(41.701233421730976 73.40217124524747) bank141464 +141465 POINT(40.27313368787262 74.89920986254894) bank141465 +141466 POINT(41.126603723038734 73.41070105624568) bank141466 +141467 POINT(40.68515029806972 73.6788351348691) bank141467 +141468 POINT(40.19731268239183 73.2598695840861) bank141468 +141469 POINT(40.456313879549114 73.04195276644732) bank141469 +141470 POINT(40.1103049256633 73.79424716782854) bank141470 +141471 POINT(41.05623137757342 74.53475719624507) bank141471 +141472 POINT(41.36624355819795 73.58844367399533) bank141472 +141473 POINT(41.629794031920426 75.00294513205623) bank141473 +141474 POINT(40.36740064573755 73.60182795946626) bank141474 +141475 POINT(39.87453389722844 74.71249320822919) bank141475 +141476 POINT(40.60715007637294 73.53029897726067) bank141476 +141477 POINT(41.46472146871215 74.19031075823189) bank141477 +141478 POINT(41.363474004082995 74.0407701561567) bank141478 +141479 POINT(40.60208383935725 73.35430318557482) bank141479 +141480 POINT(41.126218807201745 73.86584181961715) bank141480 +141481 POINT(41.360053696302195 74.99063095332735) bank141481 +141482 POINT(40.75447504247074 74.74004566657042) bank141482 +141483 POINT(41.085209203106515 73.66667366794984) bank141483 +141484 POINT(41.4638212113831 73.10595432191595) bank141484 +141485 POINT(40.25307376711426 73.02719464947768) bank141485 +141486 POINT(40.076596442402675 73.28263308479396) bank141486 +141487 POINT(41.168543244015694 74.5147433867871) bank141487 +141488 POINT(41.11452691154964 74.19181641189742) bank141488 +141489 POINT(40.38075444743755 74.17248795973585) bank141489 +141490 POINT(40.84814936406036 74.58833135370158) bank141490 +141491 POINT(40.16487970272482 74.63473959752062) bank141491 +141492 POINT(41.05269471840135 74.66708220635711) bank141492 +141493 POINT(40.08852339960235 74.93176305848222) bank141493 +141494 POINT(41.48850392006834 74.30245684453493) bank141494 +141495 POINT(41.076427262541 74.6043591710988) bank141495 +141496 POINT(40.77613955294862 74.71402600003334) bank141496 +141497 POINT(40.772110173911585 74.85146698962225) bank141497 +141498 POINT(41.3429138349449 74.93278987275667) bank141498 +141499 POINT(41.187166720757666 73.36148568862208) bank141499 +141500 POINT(40.90734128097049 74.91821617516929) bank141500 +141501 POINT(41.49305161917832 74.93531128689473) bank141501 +141502 POINT(41.554673748222335 74.79189337411168) bank141502 +141503 POINT(40.13899624947639 73.86895863516155) bank141503 +141504 POINT(41.69857481812103 73.73934632751637) bank141504 +141505 POINT(41.58568172272889 73.61308963638274) bank141505 +141506 POINT(39.99045779009472 73.41851125587944) bank141506 +141507 POINT(39.78697483578248 73.14528604041404) bank141507 +141508 POINT(40.596089301820015 73.76969347123115) bank141508 +141509 POINT(41.30324147021889 73.85261182329893) bank141509 +141510 POINT(41.0777074618417 74.79835537339179) bank141510 +141511 POINT(40.227432578599185 74.64121548590816) bank141511 +141512 POINT(41.599170924789995 74.30912257343009) bank141512 +141513 POINT(41.002566922080526 73.97437995632794) bank141513 +141514 POINT(40.567218795602045 73.4635659223927) bank141514 +141515 POINT(41.45522200978066 73.14477316879464) bank141515 +141516 POINT(40.20804992208972 73.11967096642103) bank141516 +141517 POINT(40.07554428959776 74.98946987418307) bank141517 +141518 POINT(40.06408205241473 73.78557652805956) bank141518 +141519 POINT(39.829952008342815 74.09375829137628) bank141519 +141520 POINT(40.35516953817992 74.21182280821705) bank141520 +141521 POINT(39.87545911030114 74.13032251840015) bank141521 +141522 POINT(40.773252633496746 74.96613891590322) bank141522 +141523 POINT(40.2183307248789 73.69725072384153) bank141523 +141524 POINT(40.58799319479596 73.31239017191433) bank141524 +141525 POINT(40.40097891185947 73.95056416829681) bank141525 +141526 POINT(41.689824182073984 73.59335802750479) bank141526 +141527 POINT(41.28171879624687 74.2198483983422) bank141527 +141528 POINT(41.408267793293675 74.2981203238145) bank141528 +141529 POINT(39.98830778227969 74.93765351206419) bank141529 +141530 POINT(40.49358492522445 73.54669303669246) bank141530 +141531 POINT(40.735733217209365 74.09741814180437) bank141531 +141532 POINT(40.12598838075583 74.72396800511889) bank141532 +141533 POINT(39.89966271972079 73.56183334752738) bank141533 +141534 POINT(40.45718012587242 74.94496603154376) bank141534 +141535 POINT(40.43085681558618 74.46721588222046) bank141535 +141536 POINT(39.73465691959585 74.132249596339) bank141536 +141537 POINT(39.77904005339308 74.49918929473314) bank141537 +141538 POINT(40.73550858218502 74.67257436284564) bank141538 +141539 POINT(40.42027201775381 73.6093969889612) bank141539 +141540 POINT(40.335935722325615 73.297207183801) bank141540 +141541 POINT(40.86694570026464 74.8946182063977) bank141541 +141542 POINT(39.87148246039476 73.4603803445184) bank141542 +141543 POINT(41.498647019097014 74.3291012954901) bank141543 +141544 POINT(40.74436965598566 74.95964094642292) bank141544 +141545 POINT(40.561333582850814 73.37079207959115) bank141545 +141546 POINT(41.17409198753413 73.56267593460169) bank141546 +141547 POINT(40.061398301074064 73.73491281527076) bank141547 +141548 POINT(41.50632542528403 74.75907918204213) bank141548 +141549 POINT(40.624933839473435 73.23919037294134) bank141549 +141550 POINT(40.807373221391046 74.88416752913521) bank141550 +141551 POINT(39.999003949509905 74.49052663392978) bank141551 +141552 POINT(41.121561445634995 74.39927340167883) bank141552 +141553 POINT(39.9621422087002 73.77854013997545) bank141553 +141554 POINT(40.25757024775083 73.21878231293154) bank141554 +141555 POINT(41.63201634558264 74.98011023726475) bank141555 +141556 POINT(40.53506697731821 73.32612239396451) bank141556 +141557 POINT(40.59850597394119 73.3773415235024) bank141557 +141558 POINT(40.714733139516206 73.39920297714706) bank141558 +141559 POINT(41.45595877132019 74.76675887339694) bank141559 +141560 POINT(39.73424293537356 74.6131859811952) bank141560 +141561 POINT(40.338853171717204 73.68686881531599) bank141561 +141562 POINT(40.81928667382321 74.8027433787643) bank141562 +141563 POINT(40.94943012938776 74.57770556243655) bank141563 +141564 POINT(40.06102009229505 73.44261355421574) bank141564 +141565 POINT(40.7988404031365 73.01259456723) bank141565 +141566 POINT(39.77944283760878 74.43512894689273) bank141566 +141567 POINT(40.40137746305964 73.97883024757188) bank141567 +141568 POINT(40.61908826600863 73.26058103192972) bank141568 +141569 POINT(41.641497826245704 73.15748530796581) bank141569 +141570 POINT(41.025583504729795 73.20677455090421) bank141570 +141571 POINT(40.254589855773496 74.10363632762656) bank141571 +141572 POINT(41.64037774352476 74.047860564933) bank141572 +141573 POINT(40.87142179083862 74.77117623869142) bank141573 +141574 POINT(39.904171940765394 74.98338757055417) bank141574 +141575 POINT(40.33976427574488 74.85592266201016) bank141575 +141576 POINT(40.58419171687549 74.67592126904744) bank141576 +141577 POINT(40.09550721805465 73.0908154296029) bank141577 +141578 POINT(40.821464820795306 74.01781960639558) bank141578 +141579 POINT(40.17575761331419 74.0505425470884) bank141579 +141580 POINT(40.091073003808845 74.4956202647688) bank141580 +141581 POINT(40.666343912558396 74.02170608150493) bank141581 +141582 POINT(39.92762449256119 74.95163768608359) bank141582 +141583 POINT(41.69073362146712 73.29096419283192) bank141583 +141584 POINT(40.98359547452403 73.48013348961723) bank141584 +141585 POINT(41.22064433964637 73.09975614037127) bank141585 +141586 POINT(40.82826921509589 74.51496227221145) bank141586 +141587 POINT(39.783812346231905 74.18357120406316) bank141587 +141588 POINT(39.74857940583618 74.14262388827665) bank141588 +141589 POINT(40.45455208774622 74.57375727091622) bank141589 +141590 POINT(41.247378176411374 73.82814309771821) bank141590 +141591 POINT(41.24238134424436 73.67066049563402) bank141591 +141592 POINT(40.52140774673338 73.277228522777) bank141592 +141593 POINT(40.647619539463705 73.14874147757578) bank141593 +141594 POINT(39.97188329451985 73.57751755862942) bank141594 +141595 POINT(40.11580721150657 74.24908763884015) bank141595 +141596 POINT(40.28493189961412 73.29213422608117) bank141596 +141597 POINT(41.13101827628606 74.70573216488627) bank141597 +141598 POINT(40.69026907310316 74.86344211557167) bank141598 +141599 POINT(40.545458839503084 73.45364024658153) bank141599 +141600 POINT(39.78908911197869 74.67604289732431) bank141600 +141601 POINT(41.41741390009716 74.35492576227944) bank141601 +141602 POINT(40.040265181751614 73.12881054139615) bank141602 +141603 POINT(39.8514903939129 74.65016406136132) bank141603 +141604 POINT(40.346333375120025 74.50563145033978) bank141604 +141605 POINT(41.59582570836795 74.23778222269304) bank141605 +141606 POINT(41.41038375462242 73.70405711369644) bank141606 +141607 POINT(41.299622497508444 73.92115773113449) bank141607 +141608 POINT(40.17335488428882 74.72273419005049) bank141608 +141609 POINT(40.412701990680155 74.56792207168905) bank141609 +141610 POINT(41.29532196789228 73.63024811997752) bank141610 +141611 POINT(40.712037645027166 74.86258061710289) bank141611 +141612 POINT(41.47570218518728 73.83182988629787) bank141612 +141613 POINT(41.371237142592456 73.01007667354837) bank141613 +141614 POINT(40.59424929154075 74.87027284045757) bank141614 +141615 POINT(41.70640140752597 74.21061670058995) bank141615 +141616 POINT(41.39052127369809 73.29683691311858) bank141616 +141617 POINT(41.23274492329328 73.69167703645026) bank141617 +141618 POINT(40.307648260004804 74.89452031846828) bank141618 +141619 POINT(41.23402826626519 74.99259741400938) bank141619 +141620 POINT(40.93251874829541 73.6030265233934) bank141620 +141621 POINT(41.55914912906068 74.77560945966847) bank141621 +141622 POINT(40.27746273507259 74.48645892317165) bank141622 +141623 POINT(40.349679328017 73.35147246367235) bank141623 +141624 POINT(40.916378310079146 74.96757094544546) bank141624 +141625 POINT(40.60550501634199 73.75284396286858) bank141625 +141626 POINT(40.81174068599192 74.86629358773708) bank141626 +141627 POINT(41.64425352779248 73.77883806330975) bank141627 +141628 POINT(40.2989609136541 74.50004608965389) bank141628 +141629 POINT(39.76596238984019 74.42490798267073) bank141629 +141630 POINT(40.4657137368407 73.07040110816298) bank141630 +141631 POINT(40.84254894992586 74.9617278220087) bank141631 +141632 POINT(40.600637327120154 74.51761761562692) bank141632 +141633 POINT(41.15087396506397 74.3337821778269) bank141633 +141634 POINT(40.81165833039686 74.12188778157602) bank141634 +141635 POINT(40.16805343123639 74.3590626527413) bank141635 +141636 POINT(41.61240226314003 74.02099950374271) bank141636 +141637 POINT(40.841009635661905 73.71102448430793) bank141637 +141638 POINT(41.54298017974493 73.6486766827891) bank141638 +141639 POINT(39.834044692808526 74.803117926886) bank141639 +141640 POINT(40.92652386862115 74.25922089344236) bank141640 +141641 POINT(40.82986266399723 74.16100720128456) bank141641 +141642 POINT(41.499095388408335 74.93256869251618) bank141642 +141643 POINT(41.69111356571355 73.22894873686097) bank141643 +141644 POINT(40.07292442102036 74.46555456150223) bank141644 +141645 POINT(41.08956360509338 74.43777546439523) bank141645 +141646 POINT(40.342263617382905 73.4196706535782) bank141646 +141647 POINT(41.29695423488821 73.91009427176523) bank141647 +141648 POINT(39.73702407150347 73.3864782109781) bank141648 +141649 POINT(41.36968987094885 73.34606265074933) bank141649 +141650 POINT(40.75547801478248 74.80605822640047) bank141650 +141651 POINT(40.13428939616589 73.99684912859358) bank141651 +141652 POINT(39.83796578282208 74.74304672390753) bank141652 +141653 POINT(40.07320557257166 73.11190602709966) bank141653 +141654 POINT(41.53270731123707 73.75649133568385) bank141654 +141655 POINT(39.94669940976161 74.97670866252082) bank141655 +141656 POINT(40.31831452095547 73.04990246852826) bank141656 +141657 POINT(40.9581346582428 74.87115018797884) bank141657 +141658 POINT(41.454015737068964 73.0715525002029) bank141658 +141659 POINT(40.41660376574914 73.85933633562077) bank141659 +141660 POINT(41.5801337802694 73.78020782713865) bank141660 +141661 POINT(40.3959655305182 74.57265422215767) bank141661 +141662 POINT(40.279024275584845 74.62849397566339) bank141662 +141663 POINT(40.338632396943616 73.5278808134283) bank141663 +141664 POINT(40.65489005990202 74.74143727720728) bank141664 +141665 POINT(40.63546224852971 74.39775414797977) bank141665 +141666 POINT(40.400177192765476 73.52909453408832) bank141666 +141667 POINT(41.65119234033854 73.99994299345535) bank141667 +141668 POINT(40.00696032606207 73.29284189088888) bank141668 +141669 POINT(40.125663772908766 73.27373984454327) bank141669 +141670 POINT(41.1896015863442 73.90445907276626) bank141670 +141671 POINT(41.597538790054685 73.87672767384134) bank141671 +141672 POINT(39.886780636616855 73.21191962041082) bank141672 +141673 POINT(40.40923539663776 74.4301584077736) bank141673 +141674 POINT(40.08521947425096 73.68663827923307) bank141674 +141675 POINT(40.968326726576485 74.65248155571828) bank141675 +141676 POINT(40.47335120271507 73.00730659972032) bank141676 +141677 POINT(40.45887521153775 74.85171675311838) bank141677 +141678 POINT(40.99009481234735 74.81685238191027) bank141678 +141679 POINT(40.886624519464796 73.69634116612721) bank141679 +141680 POINT(40.16761550271203 74.77084773867294) bank141680 +141681 POINT(40.24546311285746 73.75033235074451) bank141681 +141682 POINT(39.98630010954839 74.76460563270867) bank141682 +141683 POINT(40.76730605881117 73.57114745670378) bank141683 +141684 POINT(41.59636006264712 73.12477003103214) bank141684 +141685 POINT(40.86790725617044 73.91471333731383) bank141685 +141686 POINT(41.232105219490535 73.88020102896826) bank141686 +141687 POINT(40.117850862712984 74.8809776702993) bank141687 +141688 POINT(41.02866665571479 74.04042416852121) bank141688 +141689 POINT(40.837209953567694 73.509866367822) bank141689 +141690 POINT(40.093342142085554 74.56328352962652) bank141690 +141691 POINT(39.86988965992743 73.13498558271172) bank141691 +141692 POINT(40.851210214603846 73.25663787589056) bank141692 +141693 POINT(41.15618559045377 74.77006801435095) bank141693 +141694 POINT(40.34710991859399 74.18572045934397) bank141694 +141695 POINT(40.940656017876414 74.92195092837756) bank141695 +141696 POINT(40.787330493340825 74.20598503820301) bank141696 +141697 POINT(41.319803978414235 73.23578373442412) bank141697 +141698 POINT(39.91211099890415 73.23103564652007) bank141698 +141699 POINT(39.958291961770186 75.00176803827621) bank141699 +141700 POINT(39.921396425179616 74.0048860935027) bank141700 +141701 POINT(40.66855175966672 73.4799550246935) bank141701 +141702 POINT(41.354719218878984 74.98024197902033) bank141702 +141703 POINT(41.490022519793754 73.66405929667259) bank141703 +141704 POINT(41.516122330231866 74.93123838234008) bank141704 +141705 POINT(40.47543062107104 73.96439889214228) bank141705 +141706 POINT(40.26986967451767 73.80918465367901) bank141706 +141707 POINT(41.604212386861036 73.8691659291267) bank141707 +141708 POINT(40.058317151624706 74.53574921734611) bank141708 +141709 POINT(41.2040481099864 74.51309382266612) bank141709 +141710 POINT(40.520140310356 73.15869731937657) bank141710 +141711 POINT(41.010425789030336 74.92451028970541) bank141711 +141712 POINT(40.186236593590436 73.78125849620479) bank141712 +141713 POINT(41.15950557416817 74.41340748192061) bank141713 +141714 POINT(41.02030222647347 74.43025807001315) bank141714 +141715 POINT(41.252284253795956 74.85246694269296) bank141715 +141716 POINT(40.85814315273117 73.21274815281369) bank141716 +141717 POINT(41.16817617377107 73.86778058013434) bank141717 +141718 POINT(39.77291619020978 74.43616578830911) bank141718 +141719 POINT(41.06299639287097 74.67336367354666) bank141719 +141720 POINT(40.3646205689675 73.23797497248282) bank141720 +141721 POINT(41.158305493776474 74.93187562710125) bank141721 +141722 POINT(40.61661025004801 73.8356858312048) bank141722 +141723 POINT(40.689778463829015 73.43822619124549) bank141723 +141724 POINT(40.930857038667796 74.24444898944762) bank141724 +141725 POINT(41.32729357613926 74.75613374890958) bank141725 +141726 POINT(40.50076784799141 73.9573159711271) bank141726 +141727 POINT(40.278518505678065 73.45807997559847) bank141727 +141728 POINT(40.39898362902273 73.51954313154332) bank141728 +141729 POINT(41.105013570361876 74.10374727953398) bank141729 +141730 POINT(40.72807744850421 73.4349586873913) bank141730 +141731 POINT(41.256980926729824 74.93688051243473) bank141731 +141732 POINT(41.177287699436725 73.22925707159865) bank141732 +141733 POINT(40.40168103449176 73.5068178545421) bank141733 +141734 POINT(40.129168760052785 74.14780655734408) bank141734 +141735 POINT(40.5783000478823 74.15423211611454) bank141735 +141736 POINT(41.438259302749806 74.57238400447156) bank141736 +141737 POINT(41.122249330558525 74.24101745025987) bank141737 +141738 POINT(41.690219214883086 74.44638482227519) bank141738 +141739 POINT(40.436378506392735 73.18489424466195) bank141739 +141740 POINT(40.765604744505524 73.12420938148624) bank141740 +141741 POINT(41.56438436616375 74.98263248998552) bank141741 +141742 POINT(41.35798814573885 73.93798251016788) bank141742 +141743 POINT(41.40799846966236 73.3929151897569) bank141743 +141744 POINT(40.858150224188456 74.91430388672879) bank141744 +141745 POINT(41.43580765077724 74.29120417499811) bank141745 +141746 POINT(40.64796346278611 74.32777386499579) bank141746 +141747 POINT(40.79675199565973 74.42836102995064) bank141747 +141748 POINT(40.717809663348525 73.49718296874212) bank141748 +141749 POINT(40.69577392296233 73.86844797575071) bank141749 +141750 POINT(41.12511297982257 74.05737562870196) bank141750 +141751 POINT(40.84838326922293 74.56639650297443) bank141751 +141752 POINT(41.381989620646834 74.35840715150078) bank141752 +141753 POINT(41.01716377265407 73.14120731716771) bank141753 +141754 POINT(40.44363540558996 74.7636565099269) bank141754 +141755 POINT(40.584360258998686 74.95399475406381) bank141755 +141756 POINT(41.59951664734999 73.28550391199585) bank141756 +141757 POINT(40.47544366017916 74.61309588922204) bank141757 +141758 POINT(40.03103319175206 74.39607021604054) bank141758 +141759 POINT(40.504649226029095 73.69210054831007) bank141759 +141760 POINT(41.690370682911436 73.9187930415998) bank141760 +141761 POINT(40.96465196277219 74.24349600234714) bank141761 +141762 POINT(41.596820555936475 73.00615812619718) bank141762 +141763 POINT(41.68751648458783 74.02891933434645) bank141763 +141764 POINT(41.68820596208883 73.74545713376057) bank141764 +141765 POINT(40.23114955095198 74.8718889859241) bank141765 +141766 POINT(40.45940287780225 73.82811662186658) bank141766 +141767 POINT(40.020902079175904 73.28611508903745) bank141767 +141768 POINT(40.97300898632092 73.7037192722446) bank141768 +141769 POINT(41.15174566713338 74.28193285724083) bank141769 +141770 POINT(41.166708612866664 73.74828241562501) bank141770 +141771 POINT(41.420798772167025 74.5784581727458) bank141771 +141772 POINT(39.99531541041922 73.39668401733051) bank141772 +141773 POINT(40.61960913952036 74.23263821322877) bank141773 +141774 POINT(41.71233921182788 73.04266232224201) bank141774 +141775 POINT(41.418788853009694 73.47487710390355) bank141775 +141776 POINT(41.49331116279829 74.192384654785) bank141776 +141777 POINT(41.014250230800116 74.28821538897672) bank141777 +141778 POINT(41.05543670675246 73.33087215661604) bank141778 +141779 POINT(40.15971525939192 73.2765854635067) bank141779 +141780 POINT(41.33468328066007 74.18112183057187) bank141780 +141781 POINT(41.47361804448135 73.58942267995276) bank141781 +141782 POINT(41.59965190881568 74.82098716521) bank141782 +141783 POINT(39.91096074627767 73.19305723247518) bank141783 +141784 POINT(40.12810610857768 74.52065957649849) bank141784 +141785 POINT(41.391320225712434 73.44697142987275) bank141785 +141786 POINT(41.674309676065675 73.87915093910931) bank141786 +141787 POINT(39.77704711409812 74.67528249105726) bank141787 +141788 POINT(40.00159287485892 73.87780353859183) bank141788 +141789 POINT(40.629596606636106 74.552579641438) bank141789 +141790 POINT(41.339004141477126 74.99374314969008) bank141790 +141791 POINT(40.89076851221428 73.18874630461234) bank141791 +141792 POINT(40.48798101203091 74.9326793437938) bank141792 +141793 POINT(41.552792843444486 73.36404158994335) bank141793 +141794 POINT(41.02593489218782 73.08909294504602) bank141794 +141795 POINT(40.69623306287748 73.65821435203462) bank141795 +141796 POINT(40.39609149530084 74.49751344865297) bank141796 +141797 POINT(40.1795710180674 74.02982179142684) bank141797 +141798 POINT(39.930899785530215 73.29251711654928) bank141798 +141799 POINT(40.509326086547645 74.94537555565786) bank141799 +141800 POINT(40.54334124131597 74.5766689441984) bank141800 +141801 POINT(39.85958071520997 74.11665685638098) bank141801 +141802 POINT(40.591830523589884 74.51411777988187) bank141802 +141803 POINT(41.63231426996353 74.64026368707725) bank141803 +141804 POINT(41.47628152052446 73.67219711274586) bank141804 +141805 POINT(41.58328913077085 74.11019999110782) bank141805 +141806 POINT(39.90985155518433 73.4086640121563) bank141806 +141807 POINT(41.241506935102976 74.08058668479937) bank141807 +141808 POINT(40.91981358838969 73.61682473069536) bank141808 +141809 POINT(40.11068387615732 74.25746761015486) bank141809 +141810 POINT(40.92409983053359 73.50018503764525) bank141810 +141811 POINT(40.15316581542071 74.24468092924675) bank141811 +141812 POINT(40.12784547369643 73.27177307516499) bank141812 +141813 POINT(41.59158952708648 73.85856577796912) bank141813 +141814 POINT(41.376803679590815 74.96436784474531) bank141814 +141815 POINT(40.63489908157504 74.15028992671313) bank141815 +141816 POINT(39.78766030908217 74.16549689008855) bank141816 +141817 POINT(41.61576818802526 74.85162711385728) bank141817 +141818 POINT(40.22914329461688 73.99439691431705) bank141818 +141819 POINT(40.26700840855158 73.10398172652462) bank141819 +141820 POINT(41.650424996278744 74.09500839674726) bank141820 +141821 POINT(40.65027815158132 74.67182491441443) bank141821 +141822 POINT(40.92570402843579 74.98773517641239) bank141822 +141823 POINT(41.54481112734682 73.04063838501564) bank141823 +141824 POINT(40.096175099324434 73.08005840160855) bank141824 +141825 POINT(39.871183977389684 73.1099958502316) bank141825 +141826 POINT(41.643950992862855 74.5834250530608) bank141826 +141827 POINT(40.80638140674048 73.5798162170403) bank141827 +141828 POINT(40.76165550409867 73.22482366943365) bank141828 +141829 POINT(40.58509879557643 74.18402253906882) bank141829 +141830 POINT(40.60007496654659 73.5880407147571) bank141830 +141831 POINT(40.583945719643744 73.53521880057704) bank141831 +141832 POINT(40.91673995926166 74.27015100021198) bank141832 +141833 POINT(40.62500613648352 74.80991049046503) bank141833 +141834 POINT(40.79396226167427 74.63187903413564) bank141834 +141835 POINT(41.34393973304085 74.56189150684928) bank141835 +141836 POINT(39.77409771836904 73.03021488036944) bank141836 +141837 POINT(40.93922052979656 74.5676776526908) bank141837 +141838 POINT(41.580210962257105 74.88968138528276) bank141838 +141839 POINT(41.45879174432642 74.46804729179136) bank141839 +141840 POINT(40.94909984055415 74.55898503531571) bank141840 +141841 POINT(41.094344027667994 73.01792935211675) bank141841 +141842 POINT(41.25462068226211 73.25751677454826) bank141842 +141843 POINT(41.312585756560864 73.59091402650236) bank141843 +141844 POINT(41.344717319358374 74.04710485673591) bank141844 +141845 POINT(41.62884600135694 73.06098874117879) bank141845 +141846 POINT(41.262132613380544 73.66891812515637) bank141846 +141847 POINT(41.320005873230734 74.41725924744561) bank141847 +141848 POINT(40.75999911113595 74.91887951633151) bank141848 +141849 POINT(40.6146046916677 73.8148359104356) bank141849 +141850 POINT(40.14655359306592 73.63333384937549) bank141850 +141851 POINT(40.53682505892169 73.33893152492354) bank141851 +141852 POINT(40.465553464864044 74.26285427255934) bank141852 +141853 POINT(41.00936019413214 74.81070983672971) bank141853 +141854 POINT(41.2256895622285 74.92128606894045) bank141854 +141855 POINT(40.270894480556315 73.1008306823353) bank141855 +141856 POINT(40.3563027340538 74.19018668869022) bank141856 +141857 POINT(40.40352899844163 73.44620277016271) bank141857 +141858 POINT(40.048691146378744 74.04335763362025) bank141858 +141859 POINT(41.60752501772058 73.88230362768877) bank141859 +141860 POINT(40.81818820482728 74.21615495532505) bank141860 +141861 POINT(40.39679142090985 74.81464082685027) bank141861 +141862 POINT(40.32327130729346 73.94757419281775) bank141862 +141863 POINT(40.09287574570084 74.25025431083324) bank141863 +141864 POINT(39.73548774941617 73.40516182456659) bank141864 +141865 POINT(40.227574289292974 74.56657757704126) bank141865 +141866 POINT(40.30139314703076 73.43086353308983) bank141866 +141867 POINT(39.862107787943835 74.21645542479487) bank141867 +141868 POINT(39.79277590987165 74.44196935839162) bank141868 +141869 POINT(40.454478364736325 74.36186381661648) bank141869 +141870 POINT(40.29626088540914 74.53791459207464) bank141870 +141871 POINT(40.543873155249536 73.19700831367452) bank141871 +141872 POINT(40.07704209975097 74.8627752435299) bank141872 +141873 POINT(39.86285569946287 73.16011261750467) bank141873 +141874 POINT(41.09991258083504 74.61060758681542) bank141874 +141875 POINT(40.21448878803167 74.01846909148568) bank141875 +141876 POINT(41.47157969143431 74.80672051390897) bank141876 +141877 POINT(41.06118329642317 73.43323864550818) bank141877 +141878 POINT(40.10256962293126 74.58813039611098) bank141878 +141879 POINT(39.73586139106162 74.21513802694086) bank141879 +141880 POINT(39.85630984912319 73.86272011095207) bank141880 +141881 POINT(41.46331955783173 73.39448665408456) bank141881 +141882 POINT(40.638334040040064 73.74344099575234) bank141882 +141883 POINT(39.808545773512904 73.95055704403167) bank141883 +141884 POINT(40.637573275345886 74.4420056114284) bank141884 +141885 POINT(39.787810234287456 73.5335223005221) bank141885 +141886 POINT(39.848455375517936 74.12040036746801) bank141886 +141887 POINT(41.62703519376601 73.49650667889533) bank141887 +141888 POINT(40.54132012476953 74.34105998951995) bank141888 +141889 POINT(40.52291638419609 74.58871547978971) bank141889 +141890 POINT(40.73712779140289 73.84844845416912) bank141890 +141891 POINT(40.295996548428505 73.8557675042418) bank141891 +141892 POINT(41.18075621864192 73.55060135182194) bank141892 +141893 POINT(40.97198960028116 73.14396756905734) bank141893 +141894 POINT(41.253489113726374 73.44767162658823) bank141894 +141895 POINT(41.208828824461605 73.88586645748241) bank141895 +141896 POINT(40.77432183015758 73.74024766336451) bank141896 +141897 POINT(40.274076714015784 74.46556786641145) bank141897 +141898 POINT(40.06100592941255 73.67294791681405) bank141898 +141899 POINT(39.89180664980881 74.89859847198706) bank141899 +141900 POINT(41.04686064344855 73.47864418252833) bank141900 +141901 POINT(40.48344495146384 74.5475732061246) bank141901 +141902 POINT(41.661066421682335 73.63586810063582) bank141902 +141903 POINT(41.36271659581201 74.37347415793022) bank141903 +141904 POINT(40.67132946297474 74.91868707257635) bank141904 +141905 POINT(40.0407016849002 74.51782453511042) bank141905 +141906 POINT(40.029254905896266 74.62766732024329) bank141906 +141907 POINT(41.15873045979048 74.82036907207005) bank141907 +141908 POINT(40.21879842512664 73.25282386892351) bank141908 +141909 POINT(40.831134447011145 73.95382906302504) bank141909 +141910 POINT(41.2550376613432 73.03020944790791) bank141910 +141911 POINT(41.315646133915415 74.97398531726826) bank141911 +141912 POINT(41.16472264569965 74.14149278556465) bank141912 +141913 POINT(41.53226690632798 74.15155412961693) bank141913 +141914 POINT(41.079230668120914 73.59738866378011) bank141914 +141915 POINT(41.092260154302174 74.95202271188884) bank141915 +141916 POINT(39.935543812982694 74.13985163669807) bank141916 +141917 POINT(39.890952914352056 73.35373748860167) bank141917 +141918 POINT(40.63989473746531 74.048582204626) bank141918 +141919 POINT(40.70014233297077 73.79686259402398) bank141919 +141920 POINT(40.836339150141086 74.68291626533237) bank141920 +141921 POINT(39.75141932609608 74.68311361099595) bank141921 +141922 POINT(39.84929343296712 74.03877342558951) bank141922 +141923 POINT(39.844655084614985 74.11428214831797) bank141923 +141924 POINT(40.55906034149182 74.17334754550077) bank141924 +141925 POINT(41.16145073852135 74.74950325042475) bank141925 +141926 POINT(40.82546123888933 73.25823033962149) bank141926 +141927 POINT(40.70329418761076 74.54103238597436) bank141927 +141928 POINT(40.64478984000704 74.12108559705476) bank141928 +141929 POINT(40.19761488960828 73.18712603229979) bank141929 +141930 POINT(40.77836638106382 74.14046097891062) bank141930 +141931 POINT(41.455405783259586 73.45529178207588) bank141931 +141932 POINT(39.96431221528113 73.68770312695325) bank141932 +141933 POINT(41.15120654021194 73.64821486649844) bank141933 +141934 POINT(39.855596332522936 73.72704654817092) bank141934 +141935 POINT(40.18243466169068 74.94315213829944) bank141935 +141936 POINT(39.99550504011733 74.85802280949312) bank141936 +141937 POINT(40.407126689156904 74.7578496451097) bank141937 +141938 POINT(40.594268035047 73.37842161949477) bank141938 +141939 POINT(40.48158283394036 74.83169824029343) bank141939 +141940 POINT(40.00889726082971 74.88497229405517) bank141940 +141941 POINT(40.45719762697493 74.66887058323948) bank141941 +141942 POINT(41.24693633039691 74.20656803007898) bank141942 +141943 POINT(39.803796771432566 73.94731656213281) bank141943 +141944 POINT(39.90071637800498 74.97061368770068) bank141944 +141945 POINT(40.152764672498414 73.49286540844096) bank141945 +141946 POINT(41.62898997160071 73.8064230495131) bank141946 +141947 POINT(40.92863825125767 73.40267369193397) bank141947 +141948 POINT(41.544892060843324 73.30431756906138) bank141948 +141949 POINT(41.703010263652224 74.03649355277854) bank141949 +141950 POINT(40.171242539919746 74.72944920061269) bank141950 +141951 POINT(41.37294765351892 74.90522503869533) bank141951 +141952 POINT(41.31196263569913 74.98524868285146) bank141952 +141953 POINT(41.16912388024065 74.72379923892427) bank141953 +141954 POINT(39.771925863616964 73.36328069395087) bank141954 +141955 POINT(39.80637779981189 74.18294862620964) bank141955 +141956 POINT(40.26988366857938 73.35905227946448) bank141956 +141957 POINT(41.02513266919389 74.63505728696174) bank141957 +141958 POINT(41.382474701550194 73.59168219170502) bank141958 +141959 POINT(40.17606693393007 74.56643628492651) bank141959 +141960 POINT(40.11228663352876 74.20493251943728) bank141960 +141961 POINT(40.76149480464794 74.1033073987588) bank141961 +141962 POINT(40.59759915210807 73.0524359031779) bank141962 +141963 POINT(40.89503185089486 73.29938639000842) bank141963 +141964 POINT(41.25694890166029 74.7151284251112) bank141964 +141965 POINT(41.065985286634124 74.76583369619262) bank141965 +141966 POINT(41.240701086573246 74.60308965836882) bank141966 +141967 POINT(40.197236624729484 74.11704829670808) bank141967 +141968 POINT(40.126923770850766 74.13305411066953) bank141968 +141969 POINT(39.96965522584038 73.96017206992877) bank141969 +141970 POINT(41.43884057753807 73.72973543648709) bank141970 +141971 POINT(40.342648286514304 74.86078864201085) bank141971 +141972 POINT(41.09624394334145 74.53644805550049) bank141972 +141973 POINT(41.365444262126964 73.25089334303325) bank141973 +141974 POINT(40.53239731526554 73.93124103431127) bank141974 +141975 POINT(40.62203629170208 74.56483223048495) bank141975 +141976 POINT(41.288159132210524 74.02122912273802) bank141976 +141977 POINT(40.117546953937705 73.95911099078306) bank141977 +141978 POINT(40.318407846811624 74.68451891145902) bank141978 +141979 POINT(40.81579621471329 75.00409784016925) bank141979 +141980 POINT(41.019334905331846 74.00008690881742) bank141980 +141981 POINT(41.6699757526826 73.69878982285303) bank141981 +141982 POINT(41.467536026690915 73.4301055227199) bank141982 +141983 POINT(41.54344092839885 73.04192624106679) bank141983 +141984 POINT(40.98912778189421 74.33940658310397) bank141984 +141985 POINT(40.91518347902858 73.04304588111897) bank141985 +141986 POINT(40.25531558534512 74.14189327723336) bank141986 +141987 POINT(40.11384733137423 74.67453374666376) bank141987 +141988 POINT(41.300131036805425 74.54172661556998) bank141988 +141989 POINT(41.26079139215963 74.50936422356133) bank141989 +141990 POINT(40.27691558622198 73.46867009793023) bank141990 +141991 POINT(41.45943207444567 74.45066717587054) bank141991 +141992 POINT(40.01394501607424 73.28614223081296) bank141992 +141993 POINT(41.41342434505631 74.23917467980489) bank141993 +141994 POINT(41.36623093755867 73.25782081358226) bank141994 +141995 POINT(40.90551691496445 73.2855625592081) bank141995 +141996 POINT(40.268350348320794 74.4323345250372) bank141996 +141997 POINT(40.638910744946195 74.85894965669974) bank141997 +141998 POINT(40.17277111863641 74.66900877222957) bank141998 +141999 POINT(41.607947630052806 73.38504356631738) bank141999 +142000 POINT(40.00460485374237 73.53060451893641) bank142000 +142001 POINT(41.22522449598044 73.40071679131617) bank142001 +142002 POINT(40.15645136225079 73.52642678607984) bank142002 +142003 POINT(40.17072207446875 73.04457765735674) bank142003 +142004 POINT(40.43855240299742 73.62675859020314) bank142004 +142005 POINT(40.89872715868712 74.69752820159567) bank142005 +142006 POINT(40.68742148833362 74.79637668139648) bank142006 +142007 POINT(41.41928083674673 74.52640331891564) bank142007 +142008 POINT(40.06546533128695 73.64515884880124) bank142008 +142009 POINT(41.16509175854831 74.4307113071611) bank142009 +142010 POINT(40.707916892835165 73.60564676948522) bank142010 +142011 POINT(40.83683988444589 73.85131412584472) bank142011 +142012 POINT(41.331612151944846 73.81723984306363) bank142012 +142013 POINT(40.46735251829077 74.26181061389839) bank142013 +142014 POINT(40.08019255535016 73.94318652486875) bank142014 +142015 POINT(41.30701129985766 74.0279663369022) bank142015 +142016 POINT(41.48830973592344 73.5266111686816) bank142016 +142017 POINT(40.62439486042193 73.3940643805123) bank142017 +142018 POINT(41.61235301834022 74.51044485913977) bank142018 +142019 POINT(40.96736225988349 74.52675383651848) bank142019 +142020 POINT(40.89092339340202 74.0860995894206) bank142020 +142021 POINT(41.333449518922656 73.6242147771567) bank142021 +142022 POINT(41.63273700781974 74.68667563655313) bank142022 +142023 POINT(41.459111884499 73.25776499497039) bank142023 +142024 POINT(41.185160884994815 74.13000501897248) bank142024 +142025 POINT(41.12023614325204 73.2096120530167) bank142025 +142026 POINT(41.666735060947914 74.76380241995687) bank142026 +142027 POINT(41.65820455748587 73.61152684700359) bank142027 +142028 POINT(41.12045382186105 74.59540888668663) bank142028 +142029 POINT(41.31343068568313 73.11189492893106) bank142029 +142030 POINT(41.09365612831356 74.40887059977874) bank142030 +142031 POINT(39.96025842443389 74.73314295759731) bank142031 +142032 POINT(40.293981268423366 74.97539441521997) bank142032 +142033 POINT(40.64267787055695 74.84795441796251) bank142033 +142034 POINT(41.60514828243995 73.92735668032839) bank142034 +142035 POINT(41.56767854784674 74.43173423708339) bank142035 +142036 POINT(41.709320329691124 73.52193938600355) bank142036 +142037 POINT(40.10807205726244 74.84002147465245) bank142037 +142038 POINT(40.626427108907876 74.84909273903334) bank142038 +142039 POINT(39.77433111852776 73.20163963456231) bank142039 +142040 POINT(40.14089771410458 75.00484485982813) bank142040 +142041 POINT(40.82648839482094 73.87965970371967) bank142041 +142042 POINT(40.979991197203674 74.80072110804444) bank142042 +142043 POINT(39.847081978099844 74.89125310018828) bank142043 +142044 POINT(40.42709689411776 73.97124106384325) bank142044 +142045 POINT(41.63953138273469 73.72351318631054) bank142045 +142046 POINT(41.06724958256493 73.71027669092152) bank142046 +142047 POINT(41.33331885217119 74.30812140542652) bank142047 +142048 POINT(41.12666411327051 73.70934014355835) bank142048 +142049 POINT(41.505527518698116 74.74448144200738) bank142049 +142050 POINT(41.68131921031367 73.83012495949217) bank142050 +142051 POINT(40.912438930707374 74.17461180282366) bank142051 +142052 POINT(41.12772086093028 73.93031835978229) bank142052 +142053 POINT(40.02655156808015 73.51598639542782) bank142053 +142054 POINT(41.67211787684399 73.27829057682439) bank142054 +142055 POINT(41.344741013009106 73.62150375129093) bank142055 +142056 POINT(41.0388647540822 73.67585900966235) bank142056 +142057 POINT(41.41629497972485 74.17922552480753) bank142057 +142058 POINT(40.53724400285162 74.12787454579045) bank142058 +142059 POINT(40.54883364360838 73.49594649478661) bank142059 +142060 POINT(41.47304352233773 74.22762930366476) bank142060 +142061 POINT(40.04523749498233 74.2314807542571) bank142061 +142062 POINT(40.868331314043935 74.16160187647515) bank142062 +142063 POINT(41.59953034220631 74.24467330927912) bank142063 +142064 POINT(39.7498920075648 74.27069281772553) bank142064 +142065 POINT(40.08690439760375 73.73424687972722) bank142065 +142066 POINT(40.947294372275046 74.53141594600024) bank142066 +142067 POINT(40.99728481445476 74.22336735883025) bank142067 +142068 POINT(40.31463617247775 73.9364357186131) bank142068 +142069 POINT(41.0622645261775 74.49432201161727) bank142069 +142070 POINT(41.5274519836058 73.82798032383877) bank142070 +142071 POINT(41.05426791790126 73.36623198641583) bank142071 +142072 POINT(39.81282700480415 73.39468511638891) bank142072 +142073 POINT(40.77021867906343 74.78522815383496) bank142073 +142074 POINT(40.15504526372987 74.08373087754569) bank142074 +142075 POINT(40.85557007151252 74.5390167857547) bank142075 +142076 POINT(40.08119149525997 73.3381975849905) bank142076 +142077 POINT(40.77746938545803 73.95967143479452) bank142077 +142078 POINT(41.23751114222698 73.23664331663068) bank142078 +142079 POINT(40.11870376577823 73.81759228796298) bank142079 +142080 POINT(40.30982113571246 73.79737569368042) bank142080 +142081 POINT(40.01805430329411 73.91800000724336) bank142081 +142082 POINT(39.96577347141413 74.7079240663481) bank142082 +142083 POINT(39.88228139711371 74.64085623320449) bank142083 +142084 POINT(41.657946088272695 73.26915373911795) bank142084 +142085 POINT(40.59707933139713 73.920320556807) bank142085 +142086 POINT(41.65401945940883 74.67726244689909) bank142086 +142087 POINT(41.25262714464529 73.42205764780886) bank142087 +142088 POINT(39.790826535488684 74.74723417213244) bank142088 +142089 POINT(41.307554254886306 73.88623394332156) bank142089 +142090 POINT(39.783010273845285 73.27828109960875) bank142090 +142091 POINT(40.38637529426962 74.11742173659718) bank142091 +142092 POINT(40.937545902460215 74.77376881846715) bank142092 +142093 POINT(41.47120170321076 74.75841270633134) bank142093 +142094 POINT(40.60698570101646 74.04582472203225) bank142094 +142095 POINT(40.48487518892973 74.93116532030272) bank142095 +142096 POINT(40.58188723576955 73.91166997336234) bank142096 +142097 POINT(40.0880247409762 73.96864293576368) bank142097 +142098 POINT(41.440325729229926 74.49727618342311) bank142098 +142099 POINT(39.88045518479032 74.02692829797651) bank142099 +142100 POINT(40.91620568805254 74.09149850662348) bank142100 +142101 POINT(40.79214811913537 73.14047134030587) bank142101 +142102 POINT(41.09983837265203 74.68895791403243) bank142102 +142103 POINT(41.16864519298717 73.97769193281825) bank142103 +142104 POINT(41.30298345837065 74.53974585829148) bank142104 +142105 POINT(40.624176328804566 74.31716213648926) bank142105 +142106 POINT(41.06970735682204 74.64508647897792) bank142106 +142107 POINT(40.038823942292495 74.20490091504608) bank142107 +142108 POINT(41.51956873464672 74.23736073672218) bank142108 +142109 POINT(41.01483188961571 73.31243456076392) bank142109 +142110 POINT(40.32077760472098 73.49964963092545) bank142110 +142111 POINT(39.93131519216217 74.58389597983165) bank142111 +142112 POINT(40.025785923286676 73.42451444638617) bank142112 +142113 POINT(40.940904601666865 73.41036053817565) bank142113 +142114 POINT(40.00309803792976 74.03239916512713) bank142114 +142115 POINT(40.76082130625462 74.07893096306236) bank142115 +142116 POINT(40.21535469201121 73.41800859275536) bank142116 +142117 POINT(40.59643878914813 73.18526168705087) bank142117 +142118 POINT(41.149926054743894 74.96385626584738) bank142118 +142119 POINT(41.63349827078472 74.4710669543223) bank142119 +142120 POINT(41.34870135449579 74.38796803411196) bank142120 +142121 POINT(40.18400617573459 74.48195017967043) bank142121 +142122 POINT(40.43492662885878 73.11333785751305) bank142122 +142123 POINT(40.32628139508755 74.05171083290308) bank142123 +142124 POINT(39.94497054466175 73.42510973608539) bank142124 +142125 POINT(39.85533932025035 73.70875351623576) bank142125 +142126 POINT(41.37533355162561 73.0667961097646) bank142126 +142127 POINT(40.954308163922605 73.19368259738415) bank142127 +142128 POINT(40.4329743703192 73.14380000448622) bank142128 +142129 POINT(41.70912213742991 74.55296265527447) bank142129 +142130 POINT(41.42670505297431 73.11517355506535) bank142130 +142131 POINT(41.24134192442047 74.55166398477502) bank142131 +142132 POINT(40.216291778717 73.08116631147642) bank142132 +142133 POINT(40.34089243069289 74.89119872201614) bank142133 +142134 POINT(40.36166547376981 73.48974964440846) bank142134 +142135 POINT(41.13035983236811 73.8782611227961) bank142135 +142136 POINT(41.235628158272306 74.72992407661494) bank142136 +142137 POINT(41.301896167475945 73.00676555434016) bank142137 +142138 POINT(40.67364964751838 73.98342847536435) bank142138 +142139 POINT(40.19344489021137 73.47552398286416) bank142139 +142140 POINT(41.38268107888898 74.03529980507811) bank142140 +142141 POINT(40.90622877780402 73.31066996572773) bank142141 +142142 POINT(40.79176538284458 74.90172356936448) bank142142 +142143 POINT(40.894327369793984 74.35883723112585) bank142143 +142144 POINT(41.19004658025314 74.6478390753261) bank142144 +142145 POINT(41.52558187857825 74.3151471030008) bank142145 +142146 POINT(40.25983767405569 73.81693296612845) bank142146 +142147 POINT(40.97717819550703 73.31987003122035) bank142147 +142148 POINT(41.629249517863684 74.67267595772775) bank142148 +142149 POINT(40.70139635310578 73.38088872701626) bank142149 +142150 POINT(41.121002175355656 74.45641156151224) bank142150 +142151 POINT(40.32879904254134 74.71817457847524) bank142151 +142152 POINT(40.531391568740474 73.24437582334234) bank142152 +142153 POINT(39.83596997257315 73.39988995521921) bank142153 +142154 POINT(39.90032884704171 74.23942028453828) bank142154 +142155 POINT(41.12499969426211 73.14920679361758) bank142155 +142156 POINT(41.667369332514824 73.4419599269825) bank142156 +142157 POINT(41.34197408280244 73.13366111887585) bank142157 +142158 POINT(41.32231539533926 74.16103846953291) bank142158 +142159 POINT(40.24767990380027 73.61408831911258) bank142159 +142160 POINT(40.0349350278121 73.93107442431916) bank142160 +142161 POINT(41.57068081041794 74.70180566636998) bank142161 +142162 POINT(41.504409737489105 74.14585193826329) bank142162 +142163 POINT(40.71784195313454 73.92277917551577) bank142163 +142164 POINT(40.797919123436444 73.46174342564751) bank142164 +142165 POINT(39.879038902238776 73.7910976719296) bank142165 +142166 POINT(39.96324288810488 74.25147756363145) bank142166 +142167 POINT(40.479543852224865 73.60590204758547) bank142167 +142168 POINT(41.32435196633449 74.86508599592466) bank142168 +142169 POINT(40.7256107566666 73.79755110946145) bank142169 +142170 POINT(40.836023455644764 74.34482445675206) bank142170 +142171 POINT(40.237133566208506 74.62620415002594) bank142171 +142172 POINT(41.180293395801 74.63152184144215) bank142172 +142173 POINT(40.60398729712864 74.46289690621997) bank142173 +142174 POINT(39.848131627741836 74.15000377966682) bank142174 +142175 POINT(40.268642976702814 73.91262750908565) bank142175 +142176 POINT(40.91694386353373 73.49808433926728) bank142176 +142177 POINT(40.31675673733084 74.00316594743065) bank142177 +142178 POINT(41.59736768028937 74.53624109899046) bank142178 +142179 POINT(40.109512363649465 74.89627645732287) bank142179 +142180 POINT(41.450423139230374 74.77559661422511) bank142180 +142181 POINT(40.7011550367415 74.2653130266308) bank142181 +142182 POINT(40.3371477362717 74.13741290205058) bank142182 +142183 POINT(41.496619582426554 73.57724060141591) bank142183 +142184 POINT(40.20252589285377 73.01105162351193) bank142184 +142185 POINT(39.925306737682305 74.23513298877332) bank142185 +142186 POINT(40.07690785939618 74.97987537125051) bank142186 +142187 POINT(40.18086068553653 74.77748283914808) bank142187 +142188 POINT(40.6324294077256 73.7831082062361) bank142188 +142189 POINT(40.88959083354165 73.95417625115641) bank142189 +142190 POINT(40.410156118846736 73.53289612626374) bank142190 +142191 POINT(40.727670813540875 74.60538869873446) bank142191 +142192 POINT(41.56889682076369 74.10626621335737) bank142192 +142193 POINT(40.29015282864646 74.19074489417042) bank142193 +142194 POINT(40.353824651259444 74.29110806035143) bank142194 +142195 POINT(40.25625339480691 74.00429123698099) bank142195 +142196 POINT(40.548952975810884 73.95948821334756) bank142196 +142197 POINT(41.34179228967614 73.345796431304) bank142197 +142198 POINT(39.99131843183974 73.75190017123508) bank142198 +142199 POINT(41.33531200740243 73.32870455203965) bank142199 +142200 POINT(41.4919791198516 73.8790041426273) bank142200 +142201 POINT(41.21499830063688 74.55683345854426) bank142201 +142202 POINT(40.83258202229481 73.39952421671192) bank142202 +142203 POINT(41.4152740923664 73.10494169702775) bank142203 +142204 POINT(41.476240343579455 73.4527992034575) bank142204 +142205 POINT(40.04028577487725 74.59535386899843) bank142205 +142206 POINT(41.03316132009983 74.738086129512) bank142206 +142207 POINT(39.900345748772004 73.93137190299583) bank142207 +142208 POINT(40.62285483357819 74.77158721635254) bank142208 +142209 POINT(39.91438117498797 74.82467107641546) bank142209 +142210 POINT(41.17857410918078 74.39774747305218) bank142210 +142211 POINT(41.21630165702659 74.2130873553035) bank142211 +142212 POINT(40.27598534369914 73.39216010569739) bank142212 +142213 POINT(41.197162319820606 73.68000186854842) bank142213 +142214 POINT(40.55127231568933 74.15218100575696) bank142214 +142215 POINT(41.6329657794083 74.1874545029492) bank142215 +142216 POINT(41.08629110433915 73.43731003142837) bank142216 +142217 POINT(40.10509314854567 73.80110292179916) bank142217 +142218 POINT(40.51998454575348 73.0449627537136) bank142218 +142219 POINT(40.861389833451 73.6650186497917) bank142219 +142220 POINT(40.67135909783739 73.11517819074948) bank142220 +142221 POINT(40.644078236640404 73.50271281368009) bank142221 +142222 POINT(39.846963709421196 73.95296843406939) bank142222 +142223 POINT(40.89649831029666 74.16572764402841) bank142223 +142224 POINT(41.236160846911936 74.87453454256033) bank142224 +142225 POINT(39.86161681841441 74.43419399994426) bank142225 +142226 POINT(40.09623095420663 74.06456584106084) bank142226 +142227 POINT(40.02663520974786 74.04064390058473) bank142227 +142228 POINT(41.05347022729473 74.4736642880179) bank142228 +142229 POINT(41.244018104817435 73.91957383047851) bank142229 +142230 POINT(40.00356853519457 74.31499670325867) bank142230 +142231 POINT(41.55726059357488 74.38356085823463) bank142231 +142232 POINT(39.78854390216257 74.30150048002395) bank142232 +142233 POINT(39.75243744698965 74.93316245063714) bank142233 +142234 POINT(41.19501306694005 73.2767901203934) bank142234 +142235 POINT(41.60021696235888 73.69519106001826) bank142235 +142236 POINT(39.85582303111984 74.72921237846981) bank142236 +142237 POINT(40.2507004632347 74.61008345328952) bank142237 +142238 POINT(41.4612229903496 74.65972092951218) bank142238 +142239 POINT(41.04578572602591 74.9332807314618) bank142239 +142240 POINT(40.64878109668289 74.15961545411207) bank142240 +142241 POINT(40.59656435100693 73.22126924564338) bank142241 +142242 POINT(40.89253700436412 74.65431124100826) bank142242 +142243 POINT(40.54583780387881 73.38142700694553) bank142243 +142244 POINT(41.1084760381495 73.16026736082067) bank142244 +142245 POINT(39.77664831744642 74.49241315815115) bank142245 +142246 POINT(41.09406793543341 73.60795055342659) bank142246 +142247 POINT(40.69248058326863 74.56236692404042) bank142247 +142248 POINT(40.43345323536124 73.36356850428243) bank142248 +142249 POINT(40.161011297387525 73.1135748142777) bank142249 +142250 POINT(40.013743934369565 74.33662293056159) bank142250 +142251 POINT(39.727449216046395 74.68561279880305) bank142251 +142252 POINT(41.05469187583926 74.23101082420715) bank142252 +142253 POINT(40.1518786658964 73.34026659738628) bank142253 +142254 POINT(39.76861389374187 73.18925193714495) bank142254 +142255 POINT(40.841090228891446 74.8749096198029) bank142255 +142256 POINT(40.08031358141172 73.60071442112525) bank142256 +142257 POINT(40.57303778966932 73.55169883421016) bank142257 +142258 POINT(40.60398104015339 73.28400675842205) bank142258 +142259 POINT(39.92747601955163 73.66391246937458) bank142259 +142260 POINT(39.76115290996323 74.19267277287746) bank142260 +142261 POINT(40.44661786990982 74.63019003142318) bank142261 +142262 POINT(40.813679530546004 73.23003448300622) bank142262 +142263 POINT(40.55993521594638 73.25748358815385) bank142263 +142264 POINT(40.13870986147314 74.88182002041403) bank142264 +142265 POINT(40.81289445855472 74.46824753113927) bank142265 +142266 POINT(40.331894963624954 74.09880097712154) bank142266 +142267 POINT(40.64980269220705 75.00032776201986) bank142267 +142268 POINT(40.723725989570056 74.37124527670976) bank142268 +142269 POINT(40.59531182621033 73.8458768986933) bank142269 +142270 POINT(41.40203385680451 74.6017111466544) bank142270 +142271 POINT(41.58389669697596 74.56787763045405) bank142271 +142272 POINT(39.74063285913714 74.58158431530916) bank142272 +142273 POINT(39.73635180152011 73.08132312552651) bank142273 +142274 POINT(39.95378927633993 74.31346057363602) bank142274 +142275 POINT(40.796807361253734 74.03040731267487) bank142275 +142276 POINT(41.407725815068616 74.33619103039764) bank142276 +142277 POINT(41.55649553821481 73.59696780086009) bank142277 +142278 POINT(40.17725655445058 73.8390247306477) bank142278 +142279 POINT(40.220291545148 73.50121760514618) bank142279 +142280 POINT(40.23870883775851 73.54164536009732) bank142280 +142281 POINT(40.87498493054306 73.02999061485203) bank142281 +142282 POINT(41.389596238690885 73.3506053149054) bank142282 +142283 POINT(40.456841198673686 73.70781607251403) bank142283 +142284 POINT(40.85754390588216 73.6872661108419) bank142284 +142285 POINT(40.30111979925661 74.05066245796657) bank142285 +142286 POINT(39.901006342358876 74.42299616081795) bank142286 +142287 POINT(40.42039909653342 74.64726229444695) bank142287 +142288 POINT(41.402091714657494 73.60701754238941) bank142288 +142289 POINT(39.750776121704234 74.59227658022833) bank142289 +142290 POINT(41.39184899036239 74.69940274720533) bank142290 +142291 POINT(40.89456111113127 73.06078058135576) bank142291 +142292 POINT(41.526189599939556 73.80048877011662) bank142292 +142293 POINT(40.60889229351961 73.73546434244614) bank142293 +142294 POINT(40.14898283729693 73.60460137034012) bank142294 +142295 POINT(40.2842351822166 73.21096723188695) bank142295 +142296 POINT(41.533048483044944 73.85586793695906) bank142296 +142297 POINT(40.82962068760988 74.47508200359906) bank142297 +142298 POINT(39.862510599406036 74.18458846147227) bank142298 +142299 POINT(41.688241413744166 74.55327537580646) bank142299 +142300 POINT(41.07070386640997 74.17381293858615) bank142300 +142301 POINT(40.102313343625404 73.70259711996522) bank142301 +142302 POINT(40.111549221816865 74.99009265862836) bank142302 +142303 POINT(40.3856281762955 73.31596318546707) bank142303 +142304 POINT(41.514881733805815 73.42620393658267) bank142304 +142305 POINT(40.238321396425654 74.35846178410002) bank142305 +142306 POINT(40.61603751917744 74.77214933348984) bank142306 +142307 POINT(39.79041340631012 73.19931716954761) bank142307 +142308 POINT(41.135673337231545 73.49847334624815) bank142308 +142309 POINT(40.377018379982815 73.50181918615934) bank142309 +142310 POINT(40.561759348587714 73.50512771267456) bank142310 +142311 POINT(41.08391165473178 74.91024094866991) bank142311 +142312 POINT(40.20199596190897 73.17947288209268) bank142312 +142313 POINT(41.51758964397565 73.59093803465561) bank142313 +142314 POINT(40.72069378558275 73.06458520429348) bank142314 +142315 POINT(41.67665676419277 73.62762424987962) bank142315 +142316 POINT(40.61076619808196 73.94646298776446) bank142316 +142317 POINT(40.08934671312041 73.87965231661308) bank142317 +142318 POINT(41.097302082826744 73.55443112965207) bank142318 +142319 POINT(40.00275430838033 73.69719672556475) bank142319 +142320 POINT(40.63375043975691 74.20979276515455) bank142320 +142321 POINT(39.87911230552221 73.41555724564058) bank142321 +142322 POINT(40.93956374962909 73.3245810011504) bank142322 +142323 POINT(39.86167379458745 74.75722510804796) bank142323 +142324 POINT(40.895339104540774 73.76489831976907) bank142324 +142325 POINT(41.609332358512006 74.88155615505039) bank142325 +142326 POINT(39.967508300524266 74.23041433189584) bank142326 +142327 POINT(41.44895850811848 73.82371745184226) bank142327 +142328 POINT(41.372469856385905 73.41317325250563) bank142328 +142329 POINT(41.110342253358496 73.85267554968708) bank142329 +142330 POINT(39.95915433170031 74.20197105746749) bank142330 +142331 POINT(39.828993395670565 73.22781697395074) bank142331 +142332 POINT(40.367633073443315 74.09473577679874) bank142332 +142333 POINT(40.497545807024906 73.06238480387113) bank142333 +142334 POINT(41.601601315472905 74.4229329205663) bank142334 +142335 POINT(41.513101373425286 73.46332952747326) bank142335 +142336 POINT(39.81711916760632 74.46113181450963) bank142336 +142337 POINT(40.89123070173831 74.85156097721526) bank142337 +142338 POINT(40.557141981393755 74.68448547607802) bank142338 +142339 POINT(40.71065773405743 73.58307363319976) bank142339 +142340 POINT(41.021171728473284 73.9757669731004) bank142340 +142341 POINT(40.51084820548247 74.17978702270476) bank142341 +142342 POINT(39.84655913493723 74.26628476465858) bank142342 +142343 POINT(40.426443850540046 73.6204690062761) bank142343 +142344 POINT(41.361770995273474 73.47300201857354) bank142344 +142345 POINT(40.9350271838448 74.7307783087286) bank142345 +142346 POINT(40.57797590151134 73.78156326569538) bank142346 +142347 POINT(40.84514887818096 75.00408630101933) bank142347 +142348 POINT(40.37809907310745 73.48154951101134) bank142348 +142349 POINT(39.98595301258691 73.57257125250362) bank142349 +142350 POINT(40.79033469147635 73.52053352283735) bank142350 +142351 POINT(41.5062088242741 74.4703987965487) bank142351 +142352 POINT(41.06055311452257 74.41261978884032) bank142352 +142353 POINT(40.01869546349098 73.83644204348623) bank142353 +142354 POINT(40.88252312620539 73.46421953910253) bank142354 +142355 POINT(40.04150665566594 74.89667290977636) bank142355 +142356 POINT(41.40781041222236 73.52870423276352) bank142356 +142357 POINT(39.93617118568721 73.97015781002939) bank142357 +142358 POINT(41.63637798368323 73.04558870064345) bank142358 +142359 POINT(41.09700666032276 74.84027088358243) bank142359 +142360 POINT(40.715349684405496 73.96385963025482) bank142360 +142361 POINT(40.690341255027086 73.1930896423366) bank142361 +142362 POINT(40.933992043830614 73.67908095670556) bank142362 +142363 POINT(41.3461530657823 74.53122229561203) bank142363 +142364 POINT(40.97629205443519 74.98484164636335) bank142364 +142365 POINT(40.1061838004763 74.44297018627132) bank142365 +142366 POINT(40.82665004688375 74.29246329649177) bank142366 +142367 POINT(39.801844981698245 74.64159021715419) bank142367 +142368 POINT(41.48028718736294 74.6151693584258) bank142368 +142369 POINT(40.52142603320626 73.27759075335467) bank142369 +142370 POINT(39.78779394929425 73.33511096642899) bank142370 +142371 POINT(41.45175936392958 74.21366433683082) bank142371 +142372 POINT(41.33129306183476 74.34881731573896) bank142372 +142373 POINT(40.29780223966341 73.27932131315117) bank142373 +142374 POINT(40.77285385852848 73.76514468902025) bank142374 +142375 POINT(40.98589471434075 73.11568477074199) bank142375 +142376 POINT(41.30806196907604 74.02765216562612) bank142376 +142377 POINT(40.15261370032356 73.10642776767783) bank142377 +142378 POINT(40.58683292883087 74.64108978338449) bank142378 +142379 POINT(40.767266937842294 73.71550890371141) bank142379 +142380 POINT(41.085733263434776 73.85386942057329) bank142380 +142381 POINT(40.21306913528992 73.98378333657384) bank142381 +142382 POINT(41.330572954411004 74.90565342077049) bank142382 +142383 POINT(40.39873646842677 74.32821838079182) bank142383 +142384 POINT(40.03384885373933 73.12346610539791) bank142384 +142385 POINT(41.64729064432549 73.29311002690386) bank142385 +142386 POINT(41.64127196352725 74.30623464631687) bank142386 +142387 POINT(40.38531709819984 73.16998276982407) bank142387 +142388 POINT(39.997963337996175 75.0034297167023) bank142388 +142389 POINT(41.669686005670506 74.09155405443843) bank142389 +142390 POINT(40.492978317219276 74.22603962029002) bank142390 +142391 POINT(39.8285402277247 73.89086651580097) bank142391 +142392 POINT(39.75871262966733 73.1663277999704) bank142392 +142393 POINT(40.367387049446194 73.82713601743802) bank142393 +142394 POINT(40.960923471619 73.85544692461247) bank142394 +142395 POINT(39.84723444611506 74.69679992997386) bank142395 +142396 POINT(41.64323480273403 73.57535789494608) bank142396 +142397 POINT(40.68391704718255 74.6978038087162) bank142397 +142398 POINT(39.73801840108611 74.60796641269391) bank142398 +142399 POINT(39.98233897110171 74.6190664121629) bank142399 +142400 POINT(39.9176328978586 74.35031588337982) bank142400 +142401 POINT(40.22673688545185 74.92127704641497) bank142401 +142402 POINT(40.23736301454663 74.12906073442473) bank142402 +142403 POINT(41.28230571438473 74.18507283682982) bank142403 +142404 POINT(41.48065005255276 73.26532343788983) bank142404 +142405 POINT(40.775116151175794 73.18743346115369) bank142405 +142406 POINT(40.18084587936657 73.3069851391638) bank142406 +142407 POINT(40.87364371372581 74.43607610461613) bank142407 +142408 POINT(40.07253616633668 74.84941050089918) bank142408 +142409 POINT(40.72820511889422 74.45901382405853) bank142409 +142410 POINT(40.52822078478559 73.81764195958498) bank142410 +142411 POINT(40.680074507800214 73.41652168435054) bank142411 +142412 POINT(39.93175556240519 74.68610792869592) bank142412 +142413 POINT(40.87231895283963 74.30199067482613) bank142413 +142414 POINT(40.88355261564016 74.88703428746366) bank142414 +142415 POINT(40.85646691820876 73.7470553534991) bank142415 +142416 POINT(40.37799125107645 74.64987906616363) bank142416 +142417 POINT(40.77396113858311 73.17544549435546) bank142417 +142418 POINT(40.076779237590195 74.12156746596956) bank142418 +142419 POINT(40.28980770203304 74.4376827478938) bank142419 +142420 POINT(41.57804020892853 73.93387200069729) bank142420 +142421 POINT(40.975119024288766 74.39872261491777) bank142421 +142422 POINT(41.27911763218783 74.97059857951454) bank142422 +142423 POINT(41.671463039827756 74.4029494347947) bank142423 +142424 POINT(40.992294407168785 73.426565225615) bank142424 +142425 POINT(39.82792873159127 74.52321481236956) bank142425 +142426 POINT(40.30986272162401 74.64687378782666) bank142426 +142427 POINT(40.020677243095285 74.12387726499168) bank142427 +142428 POINT(40.57701219880671 73.84143413901158) bank142428 +142429 POINT(39.89391346186186 74.94710232220046) bank142429 +142430 POINT(41.33158589880414 74.38064936465668) bank142430 +142431 POINT(40.41428539117938 73.5584129152499) bank142431 +142432 POINT(40.11520451650802 74.18586665771899) bank142432 +142433 POINT(41.18003776380606 74.71457087554741) bank142433 +142434 POINT(40.12504126396287 73.65653717738957) bank142434 +142435 POINT(39.845787606882894 74.12452671335754) bank142435 +142436 POINT(41.595000881559514 73.32296804280563) bank142436 +142437 POINT(40.17626915954694 73.51449128941924) bank142437 +142438 POINT(41.06495659201375 73.88494720484917) bank142438 +142439 POINT(41.5092799184349 74.15883441393576) bank142439 +142440 POINT(41.39000886697911 73.64641945367514) bank142440 +142441 POINT(40.00864587436546 73.2164490609846) bank142441 +142442 POINT(40.27111350512698 74.16616127753011) bank142442 +142443 POINT(39.94510126818719 74.1650021888726) bank142443 +142444 POINT(39.75607987738292 74.30329742791292) bank142444 +142445 POINT(39.97881876723369 74.4239345777005) bank142445 +142446 POINT(41.182667615169834 74.50591109205223) bank142446 +142447 POINT(40.4816680779946 73.59370861753966) bank142447 +142448 POINT(40.444614065890185 73.10866998395102) bank142448 +142449 POINT(40.200373481308944 74.12810464981725) bank142449 +142450 POINT(39.89084484732709 74.92133322905009) bank142450 +142451 POINT(40.483817988774476 73.36533779190832) bank142451 +142452 POINT(40.056013910449444 73.29454660917204) bank142452 +142453 POINT(40.37665734117737 74.52636497632683) bank142453 +142454 POINT(40.50230081307874 74.01696564378096) bank142454 +142455 POINT(40.31826726335224 73.18481787278849) bank142455 +142456 POINT(41.28350150891399 73.61168838179888) bank142456 +142457 POINT(41.34202062665858 74.57370722808538) bank142457 +142458 POINT(40.50899003790487 73.25320880643517) bank142458 +142459 POINT(40.125248518393384 74.10698185918268) bank142459 +142460 POINT(40.89586431880456 73.67894973469645) bank142460 +142461 POINT(41.46063469032895 73.70930553659348) bank142461 +142462 POINT(40.68380128967435 73.86342715605497) bank142462 +142463 POINT(41.28127134864358 73.95280701444771) bank142463 +142464 POINT(40.75251442611392 73.88918183453505) bank142464 +142465 POINT(40.76017887553352 74.98187026615625) bank142465 +142466 POINT(41.454961225789575 74.5978916568049) bank142466 +142467 POINT(40.221233764667176 74.25357054175154) bank142467 +142468 POINT(39.80350923000043 73.68707397576807) bank142468 +142469 POINT(39.89747473192933 73.03848049852823) bank142469 +142470 POINT(40.40584617579259 74.04729485359813) bank142470 +142471 POINT(40.67903783850925 73.8457175945875) bank142471 +142472 POINT(41.54734069300957 74.21422623888597) bank142472 +142473 POINT(41.56829775908204 73.0480201746674) bank142473 +142474 POINT(39.77163341872121 74.47221550230007) bank142474 +142475 POINT(41.17447656838421 74.30608125707124) bank142475 +142476 POINT(41.18261846849604 74.01768587023139) bank142476 +142477 POINT(41.109748049702524 74.82124373510193) bank142477 +142478 POINT(40.87685044507552 73.66747706235562) bank142478 +142479 POINT(40.895634365600884 74.16939865445445) bank142479 +142480 POINT(40.6553687322948 73.09402612120104) bank142480 +142481 POINT(39.927577114123594 73.54485127297725) bank142481 +142482 POINT(39.894405009694296 73.39602340860533) bank142482 +142483 POINT(40.22062372636261 74.42877451544679) bank142483 +142484 POINT(40.47328428224477 74.25057959408751) bank142484 +142485 POINT(40.89609762193996 74.08145249702999) bank142485 +142486 POINT(39.83963744008751 73.71056494856703) bank142486 +142487 POINT(39.93505579284403 74.25165922853611) bank142487 +142488 POINT(39.88419774180714 74.10873913792548) bank142488 +142489 POINT(41.661777141248514 74.74356792252814) bank142489 +142490 POINT(41.440225048819926 74.83108038972061) bank142490 +142491 POINT(40.667386927388634 74.72492317108313) bank142491 +142492 POINT(40.089513201103465 74.7048086013181) bank142492 +142493 POINT(40.50833187900051 74.90185590673676) bank142493 +142494 POINT(41.69536871961805 74.35878772458362) bank142494 +142495 POINT(41.162505094450225 74.7578372003977) bank142495 +142496 POINT(41.283820938844364 73.4558312022333) bank142496 +142497 POINT(41.44423515429867 74.90016552714715) bank142497 +142498 POINT(40.82202071728726 73.16636513807137) bank142498 +142499 POINT(40.48481066288206 74.22241926207576) bank142499 +142500 POINT(40.181778482214774 73.37650638707083) bank142500 +142501 POINT(40.23917605120347 73.22166284872414) bank142501 +142502 POINT(41.152554213964976 73.40471497241532) bank142502 +142503 POINT(41.163942570183245 74.94642137987364) bank142503 +142504 POINT(40.942526307120445 74.23501218826291) bank142504 +142505 POINT(41.39754851485415 74.92736584792674) bank142505 +142506 POINT(41.20934071810138 74.19542263823986) bank142506 +142507 POINT(41.34745916582198 73.3882697216129) bank142507 +142508 POINT(41.23497030351753 73.8151065566626) bank142508 +142509 POINT(41.003926710795405 73.45976036281313) bank142509 +142510 POINT(40.52131844808855 74.61285667104181) bank142510 +142511 POINT(40.662796461702726 74.96480670127542) bank142511 +142512 POINT(40.09888178776631 74.41071370384485) bank142512 +142513 POINT(39.92248559041411 73.42345394526117) bank142513 +142514 POINT(40.26537263878287 73.19405044239392) bank142514 +142515 POINT(40.66586095987221 73.16975389123346) bank142515 +142516 POINT(39.94393860655498 73.03268690364996) bank142516 +142517 POINT(40.98143613727266 74.54297451945114) bank142517 +142518 POINT(40.0160345927546 74.69563367027844) bank142518 +142519 POINT(40.07194629711296 74.77195255265237) bank142519 +142520 POINT(41.24752721376126 74.01060309083012) bank142520 +142521 POINT(40.181962200379004 74.99778946539743) bank142521 +142522 POINT(40.162071587936126 74.16681652159104) bank142522 +142523 POINT(40.7265221788591 73.08381122708929) bank142523 +142524 POINT(39.847451847192055 74.32121339517205) bank142524 +142525 POINT(39.87440908468964 74.95810294607905) bank142525 +142526 POINT(40.27185870803175 74.59002800239877) bank142526 +142527 POINT(41.189122944724126 73.09371298106898) bank142527 +142528 POINT(40.586855573933526 74.89363794795264) bank142528 +142529 POINT(41.366912863352404 74.44558970865596) bank142529 +142530 POINT(40.98431548612221 73.23083812459117) bank142530 +142531 POINT(40.25080404173252 73.03526121053834) bank142531 +142532 POINT(40.662053100966595 74.6329414713954) bank142532 +142533 POINT(40.52322563174529 74.30846493166345) bank142533 +142534 POINT(40.40912913465349 74.73372952295232) bank142534 +142535 POINT(40.87127062751531 74.20616675785593) bank142535 +142536 POINT(41.508375747649296 74.38522356980451) bank142536 +142537 POINT(39.76509977102391 73.92255058625072) bank142537 +142538 POINT(41.426938068404674 73.69480344192824) bank142538 +142539 POINT(39.95225350284912 74.91433908039969) bank142539 +142540 POINT(40.582185214316745 73.16633868080304) bank142540 +142541 POINT(41.27426749719837 73.339465298764) bank142541 +142542 POINT(41.38396233602342 74.32350866395375) bank142542 +142543 POINT(41.63061509825377 73.35267018003282) bank142543 +142544 POINT(41.226279942486045 73.92763717232287) bank142544 +142545 POINT(41.026261958643616 74.00250438340966) bank142545 +142546 POINT(41.2290842994412 74.0870277623341) bank142546 +142547 POINT(39.89737387899744 73.51387080420464) bank142547 +142548 POINT(41.06527027625003 73.76097562918687) bank142548 +142549 POINT(41.41322547982398 74.93869209003823) bank142549 +142550 POINT(40.63489413649722 73.78605886905567) bank142550 +142551 POINT(41.255878337848 73.97584326758471) bank142551 +142552 POINT(41.210520832172605 74.43784891361024) bank142552 +142553 POINT(40.807951405939676 73.92610743575021) bank142553 +142554 POINT(39.88048606055988 73.5963823155654) bank142554 +142555 POINT(41.21304327672792 74.94187766916399) bank142555 +142556 POINT(40.093715325104895 73.57612992217197) bank142556 +142557 POINT(39.85087610668649 73.2977388403588) bank142557 +142558 POINT(40.17329012359171 74.5439191261902) bank142558 +142559 POINT(41.705343149654 73.86249787665739) bank142559 +142560 POINT(41.38716046945353 73.97207937135094) bank142560 +142561 POINT(41.14867126008932 74.38979320824379) bank142561 +142562 POINT(40.593748475425286 74.6252761846054) bank142562 +142563 POINT(39.93287500725606 74.46805963439333) bank142563 +142564 POINT(39.91561409306554 73.61134283610367) bank142564 +142565 POINT(41.65431789217027 74.91501079615519) bank142565 +142566 POINT(41.08810056769554 73.82768383216447) bank142566 +142567 POINT(40.407889976976875 73.8539657145882) bank142567 +142568 POINT(39.83444654089444 74.37436385691926) bank142568 +142569 POINT(41.092864455899225 74.53968784783042) bank142569 +142570 POINT(41.031081198937656 73.90856727658809) bank142570 +142571 POINT(40.81162319106579 74.0561916156452) bank142571 +142572 POINT(39.98904288770748 73.50614937182426) bank142572 +142573 POINT(41.58295347532127 74.22063835920393) bank142573 +142574 POINT(41.676525437048056 73.53427753749186) bank142574 +142575 POINT(40.3260772555042 74.11699168871256) bank142575 +142576 POINT(40.622334006736665 73.97880522499194) bank142576 +142577 POINT(41.20243571896767 73.64528495956314) bank142577 +142578 POINT(40.774760130324076 74.66618529678128) bank142578 +142579 POINT(40.54280007031949 74.63141915362884) bank142579 +142580 POINT(40.62504481256893 73.76880420793502) bank142580 +142581 POINT(41.68658501486358 73.97917292311077) bank142581 +142582 POINT(40.113286486270056 74.07164389098519) bank142582 +142583 POINT(41.588922322012344 73.92011973263469) bank142583 +142584 POINT(39.735527738201064 74.63969741042142) bank142584 +142585 POINT(40.43224033157708 74.35334467157152) bank142585 +142586 POINT(41.09687976730864 74.69872828065945) bank142586 +142587 POINT(41.2973712576518 73.13419787223435) bank142587 +142588 POINT(40.62808021837312 74.95547629469033) bank142588 +142589 POINT(39.819803356291224 73.08410186709874) bank142589 +142590 POINT(41.0807536168518 73.74602123674667) bank142590 +142591 POINT(40.53121457312938 73.72893179525934) bank142591 +142592 POINT(40.58466682567176 74.56779859431849) bank142592 +142593 POINT(39.903201050166786 74.78971995103123) bank142593 +142594 POINT(40.45097141991442 74.00391343317511) bank142594 +142595 POINT(40.04403784102985 73.4484205189164) bank142595 +142596 POINT(40.099905467660264 73.07723873835164) bank142596 +142597 POINT(40.53921898337237 74.87830437603498) bank142597 +142598 POINT(41.68108873790024 73.75777026146456) bank142598 +142599 POINT(41.15272175111044 74.90889246910082) bank142599 +142600 POINT(41.137492904516975 73.46162494540002) bank142600 +142601 POINT(39.949791678741946 74.47708154733289) bank142601 +142602 POINT(41.42689098443782 74.6161038007154) bank142602 +142603 POINT(40.5547523099695 74.08134987857567) bank142603 +142604 POINT(40.34459564140373 73.37469564082113) bank142604 +142605 POINT(41.34918189947662 73.72914466994047) bank142605 +142606 POINT(40.43510778123307 74.97289482147666) bank142606 +142607 POINT(40.48086197529019 73.50376528186284) bank142607 +142608 POINT(39.78762465494924 74.20956332743702) bank142608 +142609 POINT(40.673398653404774 73.07822591127143) bank142609 +142610 POINT(41.266597855608914 73.33828640113094) bank142610 +142611 POINT(39.88909015452925 73.61933095593368) bank142611 +142612 POINT(41.64165050170571 74.90346675056846) bank142612 +142613 POINT(41.36747028734795 74.62880291092634) bank142613 +142614 POINT(41.558048894020445 73.4145423047587) bank142614 +142615 POINT(40.11470633041942 74.10615514865556) bank142615 +142616 POINT(40.06579810313152 73.99329351236844) bank142616 +142617 POINT(40.22895290973275 74.92009897375495) bank142617 +142618 POINT(41.30594982213648 74.93710707149108) bank142618 +142619 POINT(40.60976792514767 74.32423957260502) bank142619 +142620 POINT(40.81731091527097 73.1610720590644) bank142620 +142621 POINT(40.6669838098593 74.28663581554329) bank142621 +142622 POINT(41.50524881445303 74.91194516328824) bank142622 +142623 POINT(41.29422818199411 73.22923306531014) bank142623 +142624 POINT(40.55963336633861 73.92672721012174) bank142624 +142625 POINT(40.016923999835896 73.77474088085033) bank142625 +142626 POINT(39.797356314496525 74.29687522308207) bank142626 +142627 POINT(39.89875542634189 73.56769428199478) bank142627 +142628 POINT(41.17198686075494 74.84489532074645) bank142628 +142629 POINT(41.295538958673205 74.45554897908009) bank142629 +142630 POINT(39.92538777641299 73.41093278983926) bank142630 +142631 POINT(41.44053804490983 73.77266597824803) bank142631 +142632 POINT(40.165275663361285 74.82693873463067) bank142632 +142633 POINT(40.660305657140974 74.0642683187429) bank142633 +142634 POINT(40.19690019742695 74.8315430806091) bank142634 +142635 POINT(41.08127368971031 74.28017122382664) bank142635 +142636 POINT(41.632061413578505 73.86837263709766) bank142636 +142637 POINT(40.693757024498304 74.37971769168558) bank142637 +142638 POINT(41.66775197158936 73.83370783167601) bank142638 +142639 POINT(40.56379265322754 74.87507463604237) bank142639 +142640 POINT(39.984514582313615 73.58849463129398) bank142640 +142641 POINT(40.825162494911076 74.83038048593347) bank142641 +142642 POINT(41.42583420556496 74.0407895469946) bank142642 +142643 POINT(40.501797423354255 74.08020891363049) bank142643 +142644 POINT(40.83028990370251 73.21963774728488) bank142644 +142645 POINT(41.395915036178515 74.28742788056073) bank142645 +142646 POINT(40.551567059619465 73.74297698371817) bank142646 +142647 POINT(39.747244946643654 73.94105870812838) bank142647 +142648 POINT(40.90179076511165 73.32434503112496) bank142648 +142649 POINT(41.529459789751456 74.86609177908757) bank142649 +142650 POINT(41.44776510632506 73.32104665513849) bank142650 +142651 POINT(40.60979443448991 74.42894436123045) bank142651 +142652 POINT(40.20844322633284 73.0638295936775) bank142652 +142653 POINT(40.095809068508586 73.47286869817256) bank142653 +142654 POINT(41.49756258935105 74.58613222551479) bank142654 +142655 POINT(40.153477277594384 74.9938996246247) bank142655 +142656 POINT(40.83568297698846 74.38023742157095) bank142656 +142657 POINT(40.03389197051756 73.46985558739507) bank142657 +142658 POINT(39.91570697433474 73.13131619860744) bank142658 +142659 POINT(40.35480484102252 73.56058896032046) bank142659 +142660 POINT(40.24563909424687 74.52049129388244) bank142660 +142661 POINT(41.406854437319055 73.31773703967431) bank142661 +142662 POINT(40.300585920800685 73.3175289570214) bank142662 +142663 POINT(41.58833911439885 74.62851529004415) bank142663 +142664 POINT(40.916781683722014 73.5962973990142) bank142664 +142665 POINT(40.7230089982205 73.92217479301033) bank142665 +142666 POINT(40.35314524947202 73.72853793584653) bank142666 +142667 POINT(41.52565411771865 73.77124868647067) bank142667 +142668 POINT(41.05916503036486 74.1753401058478) bank142668 +142669 POINT(41.20108759306017 73.46152637951084) bank142669 +142670 POINT(39.92829302838154 74.0232854913442) bank142670 +142671 POINT(39.72564717328162 73.96653715912004) bank142671 +142672 POINT(41.03657704753537 74.09378945838344) bank142672 +142673 POINT(40.154445973903634 74.46628053490066) bank142673 +142674 POINT(39.767076376363796 74.56483982511797) bank142674 +142675 POINT(39.808268746048086 73.14729615831153) bank142675 +142676 POINT(40.051557011858655 74.35828952678142) bank142676 +142677 POINT(40.67738583663915 74.13300051853926) bank142677 +142678 POINT(40.36978995343235 74.76336503709743) bank142678 +142679 POINT(39.91605479504355 74.65149429211236) bank142679 +142680 POINT(40.15657595222524 73.74652885788865) bank142680 +142681 POINT(40.29931369540123 73.90854536268827) bank142681 +142682 POINT(39.93150846924725 73.40348473223317) bank142682 +142683 POINT(39.91661916769589 74.39388201283975) bank142683 +142684 POINT(40.899252133349194 73.42551251679703) bank142684 +142685 POINT(40.78996780153726 73.54001260514433) bank142685 +142686 POINT(40.16398994612603 73.36105332837586) bank142686 +142687 POINT(39.78362889082054 74.01677223404627) bank142687 +142688 POINT(41.04411296531182 73.67468353733504) bank142688 +142689 POINT(41.232815396530185 74.15242088573929) bank142689 +142690 POINT(41.472388136589046 73.913178355778) bank142690 +142691 POINT(40.847160154473855 74.38672357459481) bank142691 +142692 POINT(40.255360889064754 73.67999769482586) bank142692 +142693 POINT(39.829988984604384 73.83390198941068) bank142693 +142694 POINT(40.0213687747742 73.06819902019829) bank142694 +142695 POINT(41.526501204579546 74.93427188282067) bank142695 +142696 POINT(41.48414098377964 74.71346617366576) bank142696 +142697 POINT(40.94957305653991 74.6902608140172) bank142697 +142698 POINT(41.4990851168886 74.32621852450633) bank142698 +142699 POINT(40.14587366466601 73.61465591931602) bank142699 +142700 POINT(40.39379470660267 74.91966492205916) bank142700 +142701 POINT(41.051427276118254 73.63860252379536) bank142701 +142702 POINT(41.37541099556877 73.17832509519717) bank142702 +142703 POINT(41.69551420880273 73.40759249507525) bank142703 +142704 POINT(41.19870391986377 73.78206178191306) bank142704 +142705 POINT(41.075915167098394 74.17423601524118) bank142705 +142706 POINT(39.75438954265239 74.8110682871971) bank142706 +142707 POINT(41.0161717761432 74.33630279229628) bank142707 +142708 POINT(39.9993151309596 73.4860460831409) bank142708 +142709 POINT(39.81745746472699 74.12794987221768) bank142709 +142710 POINT(40.33752740103267 73.33968078694635) bank142710 +142711 POINT(40.73683571625902 73.08507495334591) bank142711 +142712 POINT(41.17860624211363 74.1454040194956) bank142712 +142713 POINT(40.045630545582405 74.23064527542537) bank142713 +142714 POINT(39.97977425995268 73.76903336841669) bank142714 +142715 POINT(40.59848636568925 74.8160576215371) bank142715 +142716 POINT(40.77193160165632 74.29283602538408) bank142716 +142717 POINT(41.10621109328732 73.12806289185974) bank142717 +142718 POINT(40.41799345803151 74.9256396968495) bank142718 +142719 POINT(41.43090269185492 74.94455344205625) bank142719 +142720 POINT(40.73467438682055 74.63829614419784) bank142720 +142721 POINT(41.70149171061761 73.27802391130939) bank142721 +142722 POINT(39.809251906867345 73.82343788783) bank142722 +142723 POINT(40.00049260624884 73.2413466160108) bank142723 +142724 POINT(39.71529439102612 74.40804756091522) bank142724 +142725 POINT(40.11164824407146 74.62970489939973) bank142725 +142726 POINT(39.86405016611492 74.33377550652729) bank142726 +142727 POINT(39.77461479070391 74.09177548079725) bank142727 +142728 POINT(40.45370378168676 74.31021781142066) bank142728 +142729 POINT(40.55919900242828 73.93780828107181) bank142729 +142730 POINT(41.64564593296941 73.74334558147184) bank142730 +142731 POINT(40.18765463137848 74.20701166135237) bank142731 +142732 POINT(41.417204084250564 74.10758892615682) bank142732 +142733 POINT(41.3252642868258 74.27510636082079) bank142733 +142734 POINT(40.793723143925966 73.54623658573915) bank142734 +142735 POINT(40.79596936022325 74.11081179966519) bank142735 +142736 POINT(40.733455311424656 73.11779895815744) bank142736 +142737 POINT(40.7298463227536 73.47490893472725) bank142737 +142738 POINT(40.001478369675205 74.692250512862) bank142738 +142739 POINT(41.083645321161896 73.76953704668279) bank142739 +142740 POINT(41.49140362267643 73.77300696348041) bank142740 +142741 POINT(41.64245629731319 74.33126110094675) bank142741 +142742 POINT(40.20087406110842 73.985005183735) bank142742 +142743 POINT(40.8863717679175 74.28676460432376) bank142743 +142744 POINT(39.86181383345754 74.81212997998725) bank142744 +142745 POINT(40.697940178412686 73.49028108113947) bank142745 +142746 POINT(39.731579628590026 73.27120973903745) bank142746 +142747 POINT(39.89705916767501 74.27413460997619) bank142747 +142748 POINT(41.40768267532039 73.31780958596464) bank142748 +142749 POINT(41.12672117487318 73.33640407240163) bank142749 +142750 POINT(40.43015078930802 73.60910941520744) bank142750 +142751 POINT(40.52620087019824 73.86821398500649) bank142751 +142752 POINT(40.25197301579845 73.55006065989664) bank142752 +142753 POINT(41.472945563711 74.4932067322729) bank142753 +142754 POINT(41.13431244874482 73.37228118061658) bank142754 +142755 POINT(40.68972064164125 73.1945410490501) bank142755 +142756 POINT(41.4470390778242 73.22135271371486) bank142756 +142757 POINT(40.976978615226805 73.13726019392918) bank142757 +142758 POINT(39.86667596766573 74.20499768478007) bank142758 +142759 POINT(40.88763653113503 73.89881312673883) bank142759 +142760 POINT(41.527799489143696 74.69385488731679) bank142760 +142761 POINT(41.18594308897014 74.97956626879615) bank142761 +142762 POINT(40.776736577521014 74.80389725298566) bank142762 +142763 POINT(39.90365360575653 73.10283925753524) bank142763 +142764 POINT(41.14826345138762 74.80638568071416) bank142764 +142765 POINT(40.494481975531784 74.75028080815545) bank142765 +142766 POINT(40.879108597932834 74.34381337960102) bank142766 +142767 POINT(39.75444537278544 73.43152465535125) bank142767 +142768 POINT(40.265449733939455 73.40705626608904) bank142768 +142769 POINT(40.16544917845994 73.41516434670574) bank142769 +142770 POINT(41.11675846719766 74.00419444999662) bank142770 +142771 POINT(41.30455912365997 74.37401148388663) bank142771 +142772 POINT(41.4983360580148 73.59047983761722) bank142772 +142773 POINT(41.114900644237046 73.23408064271612) bank142773 +142774 POINT(41.35560150062861 73.31937171451987) bank142774 +142775 POINT(41.27408401744234 74.88979315238726) bank142775 +142776 POINT(39.997482330270245 74.3195991042309) bank142776 +142777 POINT(41.30712495107201 73.57313744346114) bank142777 +142778 POINT(40.663120571627005 73.09287388847807) bank142778 +142779 POINT(40.01000002358821 74.64852251792405) bank142779 +142780 POINT(40.243258778943755 74.06414628931438) bank142780 +142781 POINT(41.3312266430555 73.3821334493205) bank142781 +142782 POINT(40.2789573766564 74.3229133281545) bank142782 +142783 POINT(40.57702215437334 74.46419638994891) bank142783 +142784 POINT(41.49200207156777 73.2533554131497) bank142784 +142785 POINT(40.473811251442406 73.92426694530066) bank142785 +142786 POINT(39.737753775541655 73.26503617058589) bank142786 +142787 POINT(41.687232048176504 73.42634450676229) bank142787 +142788 POINT(41.65114493255694 73.44331973538651) bank142788 +142789 POINT(40.635951459621516 74.77600271260226) bank142789 +142790 POINT(40.50695688146604 74.43195494213688) bank142790 +142791 POINT(41.05926222956523 73.78054128915446) bank142791 +142792 POINT(41.04123799997549 74.29370622666296) bank142792 +142793 POINT(40.88535596054801 73.44959657957175) bank142793 +142794 POINT(41.00293148456899 73.0101722515434) bank142794 +142795 POINT(41.48872915721714 74.89163502434492) bank142795 +142796 POINT(40.50101107326128 73.84352302379871) bank142796 +142797 POINT(41.39542736609062 74.31982482638709) bank142797 +142798 POINT(41.1797536533197 74.11244547514029) bank142798 +142799 POINT(40.64030504564954 74.43643916044458) bank142799 +142800 POINT(40.04947132904103 74.65116751472608) bank142800 +142801 POINT(41.11624411457797 74.57061293536397) bank142801 +142802 POINT(40.467152927263975 73.52251278345749) bank142802 +142803 POINT(40.61204543734827 73.50739655297431) bank142803 +142804 POINT(41.608924031296816 74.02199342269807) bank142804 +142805 POINT(41.084750217740115 74.74485265687194) bank142805 +142806 POINT(40.09340172742367 74.28284824533058) bank142806 +142807 POINT(40.093496589158846 74.76469115053376) bank142807 +142808 POINT(40.733316341540686 74.96401367889663) bank142808 +142809 POINT(40.53764319496938 74.98669501482294) bank142809 +142810 POINT(39.83339338748017 73.81692713237562) bank142810 +142811 POINT(39.94917513766032 73.06960784000566) bank142811 +142812 POINT(40.54386065602594 73.52207945064865) bank142812 +142813 POINT(41.33219890458926 73.30927554078087) bank142813 +142814 POINT(40.864531681314695 73.18546240708338) bank142814 +142815 POINT(40.592764648167176 73.73091853897773) bank142815 +142816 POINT(40.16660187894313 73.05667517397188) bank142816 +142817 POINT(40.04617878455357 74.95149321812201) bank142817 +142818 POINT(40.68975894180325 74.22041558358673) bank142818 +142819 POINT(39.85609911216087 73.51641701814755) bank142819 +142820 POINT(41.68882197208563 73.4745486735647) bank142820 +142821 POINT(39.728809521255805 73.71918499504376) bank142821 +142822 POINT(41.658862478373656 73.5256217897034) bank142822 +142823 POINT(40.066322008706244 74.94972961029029) bank142823 +142824 POINT(41.041286159425425 73.30659998780914) bank142824 +142825 POINT(41.011620832524656 73.00829148446917) bank142825 +142826 POINT(41.675916691542895 73.33341567346467) bank142826 +142827 POINT(40.385526293199064 74.72767914218399) bank142827 +142828 POINT(40.933225753002425 73.23232052822732) bank142828 +142829 POINT(41.53871328171298 73.82204465541443) bank142829 +142830 POINT(39.77135749237931 74.79237445176007) bank142830 +142831 POINT(41.60795739912742 74.61160082995774) bank142831 +142832 POINT(41.12735471533882 74.17896477998397) bank142832 +142833 POINT(40.50212383597122 74.56262308518569) bank142833 +142834 POINT(39.80607252251085 73.66575734100974) bank142834 +142835 POINT(41.27558880063512 73.92932553027755) bank142835 +142836 POINT(40.213987803371126 74.50243817853617) bank142836 +142837 POINT(41.544084112861924 74.55655448843117) bank142837 +142838 POINT(39.88923399464709 73.93150651332928) bank142838 +142839 POINT(40.221724958124334 73.92040456657963) bank142839 +142840 POINT(40.59555116372097 74.22756548804053) bank142840 +142841 POINT(41.29411332696585 74.66687575520399) bank142841 +142842 POINT(40.204321375715814 73.94520418745195) bank142842 +142843 POINT(39.79662443461886 73.81259499273135) bank142843 +142844 POINT(40.93294104822579 73.72853716034741) bank142844 +142845 POINT(40.18743671019986 73.42878569815919) bank142845 +142846 POINT(40.316627634211564 73.79896087154677) bank142846 +142847 POINT(40.7794761882964 73.65308767361229) bank142847 +142848 POINT(41.22507801871041 74.41572704223374) bank142848 +142849 POINT(40.02453515186254 74.97061211034891) bank142849 +142850 POINT(40.610828139258274 74.09373893872527) bank142850 +142851 POINT(39.74139470299815 74.18673312924966) bank142851 +142852 POINT(39.731304710686885 73.34556677554353) bank142852 +142853 POINT(40.46877858158702 73.68389482886455) bank142853 +142854 POINT(39.754327734040814 73.66548909088462) bank142854 +142855 POINT(39.89000634050104 74.13429291090571) bank142855 +142856 POINT(41.243583358302104 73.75935260702299) bank142856 +142857 POINT(39.73669487586478 74.9195804171785) bank142857 +142858 POINT(40.44224458506836 74.0856483737079) bank142858 +142859 POINT(40.6428552003475 73.77906753909502) bank142859 +142860 POINT(41.17721124954687 74.6049780190693) bank142860 +142861 POINT(41.62119260117204 73.98299238440103) bank142861 +142862 POINT(40.41146539822444 73.99961614797851) bank142862 +142863 POINT(41.5227025117853 73.82264195510506) bank142863 +142864 POINT(40.333202354492705 74.65466639804937) bank142864 +142865 POINT(39.78135148110387 74.02768565542164) bank142865 +142866 POINT(39.94971307280182 73.36111613117671) bank142866 +142867 POINT(40.16800240608517 73.00814464464129) bank142867 +142868 POINT(39.74076958346739 74.15298297528756) bank142868 +142869 POINT(39.81416798562153 74.62042849471528) bank142869 +142870 POINT(40.47959442620312 73.47939143544181) bank142870 +142871 POINT(39.77835508384289 74.33350897105541) bank142871 +142872 POINT(39.80370399366547 74.77041593714482) bank142872 +142873 POINT(41.53945268568979 73.5706956932209) bank142873 +142874 POINT(40.60202305296659 74.28320542640878) bank142874 +142875 POINT(40.710993875432735 74.55764793887943) bank142875 +142876 POINT(41.68517459936577 74.38633282367) bank142876 +142877 POINT(41.416612392672086 73.79030496032823) bank142877 +142878 POINT(40.8348695747407 74.33099685407119) bank142878 +142879 POINT(40.91366805545628 73.93775984507316) bank142879 +142880 POINT(40.60318523473405 73.39218759415134) bank142880 +142881 POINT(40.55587730737573 74.97363941841776) bank142881 +142882 POINT(39.90688717796897 74.09569417708815) bank142882 +142883 POINT(40.02660480620539 74.78545814390155) bank142883 +142884 POINT(39.842674824069796 74.00322239890127) bank142884 +142885 POINT(40.7026594967353 74.86753841439491) bank142885 +142886 POINT(40.450717768340425 73.19678880516742) bank142886 +142887 POINT(41.648582979929756 74.81832118856296) bank142887 +142888 POINT(39.71839076584112 73.24780772625166) bank142888 +142889 POINT(41.390729882679395 73.139771855987) bank142889 +142890 POINT(41.132269568095566 74.90501988749794) bank142890 +142891 POINT(40.54090567159755 73.44059369882623) bank142891 +142892 POINT(40.22906646871299 74.01544979815523) bank142892 +142893 POINT(40.9565747207207 74.64075632080963) bank142893 +142894 POINT(39.955420175869556 74.39282451028761) bank142894 +142895 POINT(41.140272771229306 73.35839954068265) bank142895 +142896 POINT(40.95766451831839 74.11238472723439) bank142896 +142897 POINT(40.66740208851527 73.10730242293779) bank142897 +142898 POINT(41.20144559486057 73.6372214359229) bank142898 +142899 POINT(40.047659655540656 73.57684806017784) bank142899 +142900 POINT(40.59545438728883 73.11207512206333) bank142900 +142901 POINT(41.555571118677335 73.01628022238067) bank142901 +142902 POINT(39.86972326138951 74.09706721591638) bank142902 +142903 POINT(40.963913478685825 73.76071173503638) bank142903 +142904 POINT(41.26672115217427 74.56405311342913) bank142904 +142905 POINT(41.66625940346503 74.02217587922739) bank142905 +142906 POINT(40.03187187699318 73.30298700317005) bank142906 +142907 POINT(40.52449192126104 74.32380135365734) bank142907 +142908 POINT(41.68889311720054 74.11326923316106) bank142908 +142909 POINT(41.659907978461135 74.09509164210544) bank142909 +142910 POINT(40.07249306793952 73.95916085836174) bank142910 +142911 POINT(41.3966947239321 74.54846143310472) bank142911 +142912 POINT(40.67146616217131 74.93746349331397) bank142912 +142913 POINT(40.96404251087765 74.57411333098115) bank142913 +142914 POINT(41.11088546449242 73.46921816974115) bank142914 +142915 POINT(41.61867284288328 73.70201683902594) bank142915 +142916 POINT(39.98554577937315 74.94392450906489) bank142916 +142917 POINT(41.001256599572464 73.52525362836971) bank142917 +142918 POINT(41.345162957792205 74.91577102701143) bank142918 +142919 POINT(41.56184672857123 74.19950568035519) bank142919 +142920 POINT(41.20985785098963 74.62374883236293) bank142920 +142921 POINT(41.68761188720012 74.90564201337277) bank142921 +142922 POINT(40.55943403615807 74.73122710238269) bank142922 +142923 POINT(39.960312495051376 73.61999755778271) bank142923 +142924 POINT(41.51845823974161 74.99858170340924) bank142924 +142925 POINT(41.0007076631482 73.2501023660066) bank142925 +142926 POINT(41.686466298638315 74.22724685587947) bank142926 +142927 POINT(39.72603966115226 73.76341722883808) bank142927 +142928 POINT(40.331830818731724 73.88128004098668) bank142928 +142929 POINT(41.353571797814446 73.18870709960257) bank142929 +142930 POINT(40.637770461796954 74.40662468547853) bank142930 +142931 POINT(40.7995058356206 74.95058769951288) bank142931 +142932 POINT(40.203630195849186 74.41042420530718) bank142932 +142933 POINT(39.824691858190626 74.19277296513748) bank142933 +142934 POINT(41.24011646103287 73.246739939183) bank142934 +142935 POINT(40.26517610793024 73.64444043631073) bank142935 +142936 POINT(40.86592407084953 73.79255601350164) bank142936 +142937 POINT(40.24339676110004 73.17981447927188) bank142937 +142938 POINT(40.24113755921741 74.10174501312177) bank142938 +142939 POINT(41.04948659261791 74.20031403997271) bank142939 +142940 POINT(40.74996395497168 73.85076180794391) bank142940 +142941 POINT(39.89148592318206 74.83117851143894) bank142941 +142942 POINT(39.86295370641481 73.86595714812984) bank142942 +142943 POINT(41.268372863940414 74.60711360193767) bank142943 +142944 POINT(41.240083134726554 74.61106073123614) bank142944 +142945 POINT(41.423190449326256 73.8555163759234) bank142945 +142946 POINT(39.81664446528123 74.70139401602135) bank142946 +142947 POINT(40.820814970996985 73.09706743323805) bank142947 +142948 POINT(41.68052935221623 74.74592369063072) bank142948 +142949 POINT(40.213706085205054 73.45077212069333) bank142949 +142950 POINT(41.316295019609804 73.77098287444961) bank142950 +142951 POINT(40.79089222669673 74.84661896444321) bank142951 +142952 POINT(41.6430300373842 73.11995361369075) bank142952 +142953 POINT(41.389184338736186 73.6502362584317) bank142953 +142954 POINT(40.83887214263232 74.89899203344366) bank142954 +142955 POINT(40.52503525254465 74.78034207688637) bank142955 +142956 POINT(40.85482100057938 74.07884495705406) bank142956 +142957 POINT(40.83455353485924 73.33673011475153) bank142957 +142958 POINT(41.61727087389276 73.80246383251621) bank142958 +142959 POINT(40.36638030205305 74.90120738885314) bank142959 +142960 POINT(41.12412165976786 74.59358050874046) bank142960 +142961 POINT(39.714228390953195 74.10800047953275) bank142961 +142962 POINT(40.24153265102818 73.34210749673538) bank142962 +142963 POINT(41.04589810115935 73.55888853712733) bank142963 +142964 POINT(40.60953197872137 73.02443170670041) bank142964 +142965 POINT(40.287045665783616 74.81505545889512) bank142965 +142966 POINT(41.10086817384117 74.62152534944696) bank142966 +142967 POINT(39.89515039422081 74.10502251630142) bank142967 +142968 POINT(40.073493563836955 74.7045912203731) bank142968 +142969 POINT(41.217220153398074 73.79272758860805) bank142969 +142970 POINT(41.20665596645478 73.4300841423191) bank142970 +142971 POINT(40.274084434019464 74.54337193633367) bank142971 +142972 POINT(40.0300951746501 73.23221129694193) bank142972 +142973 POINT(40.158848679170305 74.42348001279245) bank142973 +142974 POINT(40.950769024114095 73.71618151435632) bank142974 +142975 POINT(41.47738150652227 73.59114422604449) bank142975 +142976 POINT(40.124854463408646 73.3688735490566) bank142976 +142977 POINT(40.48161102163657 73.54783043281851) bank142977 +142978 POINT(41.23429287889809 73.38896931946131) bank142978 +142979 POINT(40.70028385075822 74.49722605528446) bank142979 +142980 POINT(41.226959264776404 74.18803591599975) bank142980 +142981 POINT(41.19857479871013 74.1136706690841) bank142981 +142982 POINT(40.616827457687705 73.39289549782197) bank142982 +142983 POINT(40.08816933642972 75.00146126851222) bank142983 +142984 POINT(41.54744209556989 73.87384899007631) bank142984 +142985 POINT(40.949793078917345 74.6795885915003) bank142985 +142986 POINT(40.9081265397897 74.99549784065735) bank142986 +142987 POINT(40.03829313983726 74.25585444689884) bank142987 +142988 POINT(40.688887029155204 74.65473593107843) bank142988 +142989 POINT(41.43666253816351 74.62283658904722) bank142989 +142990 POINT(41.570587804314634 73.4889459356) bank142990 +142991 POINT(41.52582212412662 73.76044532920851) bank142991 +142992 POINT(41.32288684743338 73.99621160214062) bank142992 +142993 POINT(40.78258531674592 73.32456552407837) bank142993 +142994 POINT(39.95738093171434 73.31335878185286) bank142994 +142995 POINT(39.76065000510011 74.8564200085322) bank142995 +142996 POINT(41.60138437869594 74.3153568499841) bank142996 +142997 POINT(40.99146881184582 73.37302882282242) bank142997 +142998 POINT(41.67741533422012 74.58007168509658) bank142998 +142999 POINT(40.644430920596406 73.03705935457376) bank142999 +143000 POINT(40.836037483800645 74.21352030007873) bank143000 +143001 POINT(39.76717101650585 74.08699613142527) bank143001 +143002 POINT(40.31943693129732 74.43314990126463) bank143002 +143003 POINT(41.1419062953418 74.20741177129807) bank143003 +143004 POINT(39.73697143395592 73.67985313117164) bank143004 +143005 POINT(41.317727664537635 74.98541234676905) bank143005 +143006 POINT(40.39441025828033 74.71130271847576) bank143006 +143007 POINT(41.64456497005606 73.68297011318703) bank143007 +143008 POINT(39.75480217487553 74.8419530388508) bank143008 +143009 POINT(41.1664956498618 74.67560294160444) bank143009 +143010 POINT(40.62932829299635 74.73006474063521) bank143010 +143011 POINT(41.32670207562211 73.53979544253146) bank143011 +143012 POINT(41.16832323947188 74.60396442816382) bank143012 +143013 POINT(40.841994169904154 74.93162822859696) bank143013 +143014 POINT(40.48338118151845 73.86452501165294) bank143014 +143015 POINT(40.06683687671176 74.6951913287332) bank143015 +143016 POINT(41.42266719318519 73.26746476596703) bank143016 +143017 POINT(40.64829494964113 74.60813700169112) bank143017 +143018 POINT(40.69429790891981 74.65169120033895) bank143018 +143019 POINT(41.67443183023922 73.80248304427491) bank143019 +143020 POINT(41.56062766591799 74.2775344206875) bank143020 +143021 POINT(39.980364832957186 73.80298075033457) bank143021 +143022 POINT(40.64915705581492 74.53453922153196) bank143022 +143023 POINT(39.84718324105037 74.32814968716735) bank143023 +143024 POINT(40.81939461908148 73.88006128103073) bank143024 +143025 POINT(40.21574778877908 74.52146914443394) bank143025 +143026 POINT(40.48369873354537 73.38844864073451) bank143026 +143027 POINT(41.55170904003231 73.62658858686872) bank143027 +143028 POINT(40.733453758685464 74.82898692599686) bank143028 +143029 POINT(41.40393498691001 74.66751779993437) bank143029 +143030 POINT(40.72490370713764 74.84289340934818) bank143030 +143031 POINT(40.97477572150463 73.4326748487622) bank143031 +143032 POINT(41.40129607491362 74.1277807576289) bank143032 +143033 POINT(41.59757228296668 74.45521050903406) bank143033 +143034 POINT(41.59026763867091 73.28767331744109) bank143034 +143035 POINT(40.89837149955687 74.90379003347209) bank143035 +143036 POINT(40.52213573546145 73.72937311499898) bank143036 +143037 POINT(40.60934873051412 74.76824577656825) bank143037 +143038 POINT(40.167616845716125 74.67758777267652) bank143038 +143039 POINT(40.641636110391026 73.60777842628472) bank143039 +143040 POINT(40.416293932263855 73.26780078847904) bank143040 +143041 POINT(41.318055094542004 74.42264196262687) bank143041 +143042 POINT(41.48464324578664 74.4631869880665) bank143042 +143043 POINT(40.85931987915331 74.54996719410472) bank143043 +143044 POINT(40.03690925912009 73.04844497947614) bank143044 +143045 POINT(41.32969429397822 73.98353117471208) bank143045 +143046 POINT(41.49616132378977 73.96765291496803) bank143046 +143047 POINT(41.26981840980645 74.0390544488209) bank143047 +143048 POINT(39.80258584453885 73.85985648373928) bank143048 +143049 POINT(40.202672710627915 73.11317651203991) bank143049 +143050 POINT(40.70107525887734 74.13267912141616) bank143050 +143051 POINT(41.49515842947611 74.61759373339396) bank143051 +143052 POINT(39.83266221434182 74.50238095637125) bank143052 +143053 POINT(40.42193644149295 73.75325017597262) bank143053 +143054 POINT(41.16681014158771 73.50535508030826) bank143054 +143055 POINT(41.608273864696706 74.51963965454806) bank143055 +143056 POINT(41.54724776498016 73.83605357449333) bank143056 +143057 POINT(40.265252859419526 74.11870493431984) bank143057 +143058 POINT(40.3553636151329 74.05550637555257) bank143058 +143059 POINT(41.272241432781335 73.66510789458542) bank143059 +143060 POINT(40.83443301803966 73.66765372921796) bank143060 +143061 POINT(41.329662276072746 74.89688769285671) bank143061 +143062 POINT(40.62725240238425 73.73110731822781) bank143062 +143063 POINT(40.25098609458119 74.86517310945841) bank143063 +143064 POINT(41.40478506891518 73.07365035399911) bank143064 +143065 POINT(40.27092724836502 74.05793652613706) bank143065 +143066 POINT(40.600231920810685 73.3299990383125) bank143066 +143067 POINT(41.68870559541234 73.75489203096646) bank143067 +143068 POINT(40.588878482714286 74.42892850378745) bank143068 +143069 POINT(40.246752966122564 73.48723966929695) bank143069 +143070 POINT(40.027212175148435 73.88130985985852) bank143070 +143071 POINT(41.14045909066009 74.5576981486771) bank143071 +143072 POINT(41.03438723963991 74.87334229085731) bank143072 +143073 POINT(39.73043573635475 74.76983199232141) bank143073 +143074 POINT(41.24778939781726 74.74503657727377) bank143074 +143075 POINT(40.92084709155326 74.96234406048478) bank143075 +143076 POINT(41.19411945819061 73.79447874691668) bank143076 +143077 POINT(39.78108073171485 73.69910555947324) bank143077 +143078 POINT(41.26402424669559 73.63248320522011) bank143078 +143079 POINT(41.63366336289621 73.10147506094819) bank143079 +143080 POINT(40.10557879295989 73.6362755961486) bank143080 +143081 POINT(40.58377854079283 74.81773343163837) bank143081 +143082 POINT(40.62695038361247 73.35409311187105) bank143082 +143083 POINT(40.829555782384304 74.98594901300555) bank143083 +143084 POINT(39.820136113556906 73.01516794755308) bank143084 +143085 POINT(40.590977000831174 73.01818234452378) bank143085 +143086 POINT(40.72151741498503 73.65195041722224) bank143086 +143087 POINT(40.31463291110096 73.07178527531447) bank143087 +143088 POINT(40.11073868927845 74.289088471544) bank143088 +143089 POINT(39.98905669838796 73.10922412047317) bank143089 +143090 POINT(41.431068361765604 73.4468213774944) bank143090 +143091 POINT(40.93760161064438 73.17539227586714) bank143091 +143092 POINT(41.66572701655206 74.43362103212152) bank143092 +143093 POINT(40.77612922218836 73.90381004381022) bank143093 +143094 POINT(40.91498098689685 73.11296662610994) bank143094 +143095 POINT(40.65217172538474 74.43115971363694) bank143095 +143096 POINT(40.16006026280404 75.00123328946403) bank143096 +143097 POINT(41.014776093092365 73.01481284504405) bank143097 +143098 POINT(41.09845960339728 73.30742650885841) bank143098 +143099 POINT(39.81807624152927 74.18613075735634) bank143099 +143100 POINT(40.60944260210201 74.28836524654383) bank143100 +143101 POINT(40.31833188939107 74.28194540728421) bank143101 +143102 POINT(39.8618043828115 73.25360039421192) bank143102 +143103 POINT(41.3320949938194 74.57888789392305) bank143103 +143104 POINT(41.408423684885065 74.35810745426372) bank143104 +143105 POINT(41.67643507696305 73.55449157160665) bank143105 +143106 POINT(41.55044783787727 73.5158155798596) bank143106 +143107 POINT(40.34192782601723 74.44253160562924) bank143107 +143108 POINT(40.90960894947444 74.39840611455364) bank143108 +143109 POINT(41.31363834700848 73.01710278179108) bank143109 +143110 POINT(41.634472734303046 73.42770973990068) bank143110 +143111 POINT(40.979984576701945 74.45429936728566) bank143111 +143112 POINT(41.48160228194981 74.36706235852398) bank143112 +143113 POINT(41.60028929297297 73.22625994957728) bank143113 +143114 POINT(40.12524282713753 73.2412039850527) bank143114 +143115 POINT(40.25774532059482 73.05035769079272) bank143115 +143116 POINT(41.28948362399054 73.21835860703247) bank143116 +143117 POINT(41.0053704583294 74.22044793886724) bank143117 +143118 POINT(40.70801584442746 73.05393615358902) bank143118 +143119 POINT(40.42121759785907 74.41096735424453) bank143119 +143120 POINT(40.89645798849994 74.27050551774423) bank143120 +143121 POINT(40.61011584697836 74.63089380581903) bank143121 +143122 POINT(40.337921486660264 73.56018790162142) bank143122 +143123 POINT(40.43161608597198 73.87097065112975) bank143123 +143124 POINT(40.84054094520833 73.22443417498127) bank143124 +143125 POINT(40.406940135144566 74.83126348754155) bank143125 +143126 POINT(41.2624796099286 73.35706428427336) bank143126 +143127 POINT(41.68868081276013 74.78105649565437) bank143127 +143128 POINT(40.29884616883367 74.81430261031304) bank143128 +143129 POINT(40.04428600703104 73.45061706767208) bank143129 +143130 POINT(41.3370347739357 74.79686591480122) bank143130 +143131 POINT(41.2807806961822 74.60809273217455) bank143131 +143132 POINT(40.44722783176031 74.26618476671399) bank143132 +143133 POINT(41.038022333251234 73.83866765068215) bank143133 +143134 POINT(40.19227450245062 74.44274275778727) bank143134 +143135 POINT(40.40931470132686 73.80433129116358) bank143135 +143136 POINT(40.056773144508036 74.7943047603425) bank143136 +143137 POINT(41.207746374956784 74.1899740720688) bank143137 +143138 POINT(40.13026808160429 73.17593461811443) bank143138 +143139 POINT(40.27539786218328 73.31921344331134) bank143139 +143140 POINT(39.76228686644032 73.74132184938102) bank143140 +143141 POINT(41.315866641371656 74.69818344017747) bank143141 +143142 POINT(40.5945121216514 74.561571388565) bank143142 +143143 POINT(39.746259541736464 74.4148829002269) bank143143 +143144 POINT(41.473615077155536 74.28190273558587) bank143144 +143145 POINT(39.87934486393006 74.29764623025173) bank143145 +143146 POINT(41.63762565507825 73.88715790280267) bank143146 +143147 POINT(40.58432095126107 74.14139807843172) bank143147 +143148 POINT(40.60594462122132 73.67761757277805) bank143148 +143149 POINT(39.94476724856927 73.87040365395885) bank143149 +143150 POINT(40.058695725949555 74.24308735526392) bank143150 +143151 POINT(41.441300735013954 74.03389133129087) bank143151 +143152 POINT(40.64149417755443 74.69791388472133) bank143152 +143153 POINT(40.28623152889917 73.14584931484201) bank143153 +143154 POINT(39.794336083545524 73.79030165470789) bank143154 +143155 POINT(40.28092510409646 74.6015657734498) bank143155 +143156 POINT(40.157478184409804 73.86854062149422) bank143156 +143157 POINT(40.53452077947177 74.85610134092768) bank143157 +143158 POINT(39.83222290807664 73.9288659222205) bank143158 +143159 POINT(39.94513370277025 74.33001789331149) bank143159 +143160 POINT(40.49887545655183 73.56443301890033) bank143160 +143161 POINT(41.34194961579008 73.37559511793405) bank143161 +143162 POINT(40.9435470099696 74.63213567073733) bank143162 +143163 POINT(40.24936761499191 74.17523169032582) bank143163 +143164 POINT(41.17554201036199 73.6107855014507) bank143164 +143165 POINT(40.979286310530135 74.03629551646675) bank143165 +143166 POINT(41.24188708368836 74.34240027457622) bank143166 +143167 POINT(41.12779276337921 73.16748592217314) bank143167 +143168 POINT(40.15197754918042 73.94886943430058) bank143168 +143169 POINT(40.54761895792791 73.87524927158776) bank143169 +143170 POINT(41.20970143562649 73.70808939489541) bank143170 +143171 POINT(41.516600331248334 74.89979311657517) bank143171 +143172 POINT(40.80776306740901 73.41161320868102) bank143172 +143173 POINT(39.92743638327831 74.0092607748894) bank143173 +143174 POINT(40.762644415888445 73.44389137603886) bank143174 +143175 POINT(41.48608133917964 74.4117538622368) bank143175 +143176 POINT(41.12106033584639 73.31580690743398) bank143176 +143177 POINT(41.06800238593695 74.01678648627612) bank143177 +143178 POINT(41.264038987295876 74.49401716813551) bank143178 +143179 POINT(40.16460258589362 74.77904086590539) bank143179 +143180 POINT(40.35776763702326 74.78913606463738) bank143180 +143181 POINT(41.13767899761345 74.2791167932544) bank143181 +143182 POINT(41.374484681993934 74.93918873646923) bank143182 +143183 POINT(40.002508982337595 74.30120171154446) bank143183 +143184 POINT(40.79516106294975 74.64312089803954) bank143184 +143185 POINT(40.97331909376062 74.1102257808645) bank143185 +143186 POINT(41.009774266249764 74.31247049403763) bank143186 +143187 POINT(41.22695130270515 74.29912782265455) bank143187 +143188 POINT(41.00248677145028 74.1105597639168) bank143188 +143189 POINT(41.38609922719583 73.48548323045564) bank143189 +143190 POINT(41.60622435917409 73.32107775301004) bank143190 +143191 POINT(40.69735305055528 74.61444552477735) bank143191 +143192 POINT(40.95552196150086 73.29122787378209) bank143192 +143193 POINT(41.27730268066738 73.6370253949286) bank143193 +143194 POINT(39.7440839541526 73.65895613312244) bank143194 +143195 POINT(40.74265339316457 73.52069596228226) bank143195 +143196 POINT(40.53167700368887 73.55048984189361) bank143196 +143197 POINT(41.18324601587237 74.50815954239219) bank143197 +143198 POINT(40.70067294597246 74.08948224295875) bank143198 +143199 POINT(41.50838108777801 74.5143062850801) bank143199 +143200 POINT(41.5175962176942 73.94865361489512) bank143200 +143201 POINT(41.60621297758057 73.97107173100808) bank143201 +143202 POINT(41.69101679768087 74.50920795631856) bank143202 +143203 POINT(41.5243222422894 74.12580942152806) bank143203 +143204 POINT(40.65364664766798 73.19120191790593) bank143204 +143205 POINT(40.54153548069522 74.54337763915976) bank143205 +143206 POINT(39.75721792548442 73.19106647820253) bank143206 +143207 POINT(40.141388153627496 74.44672707960902) bank143207 +143208 POINT(40.12432317674362 74.96821120602021) bank143208 +143209 POINT(40.670328635157794 74.66590676614975) bank143209 +143210 POINT(41.56948082234412 74.16475516220216) bank143210 +143211 POINT(40.60875632121948 73.96980746189547) bank143211 +143212 POINT(39.99690692370898 74.81416404842632) bank143212 +143213 POINT(41.17058435010881 74.98112966206632) bank143213 +143214 POINT(40.33804313869652 74.85830034685571) bank143214 +143215 POINT(40.754898457215916 73.55436127026854) bank143215 +143216 POINT(40.058480428611766 74.55679482877878) bank143216 +143217 POINT(41.146445127841716 73.99709940983453) bank143217 +143218 POINT(41.34135310781195 73.78002197372766) bank143218 +143219 POINT(40.178530436191686 73.98064874882236) bank143219 +143220 POINT(39.892924060003715 74.76271548886518) bank143220 +143221 POINT(41.01291193828074 73.3874447869317) bank143221 +143222 POINT(40.03891617087193 73.82069203457624) bank143222 +143223 POINT(39.86167423535943 73.39800160979136) bank143223 +143224 POINT(40.145754447730944 73.39785631899063) bank143224 +143225 POINT(40.56343473097461 74.1275294970391) bank143225 +143226 POINT(39.821300105001036 74.36982449923552) bank143226 +143227 POINT(40.896487130935625 73.48554489231769) bank143227 +143228 POINT(40.91607503275547 73.89436773121466) bank143228 +143229 POINT(41.3851184921189 73.41842813629944) bank143229 +143230 POINT(40.0901523974651 73.34471382973219) bank143230 +143231 POINT(40.42953889089011 74.26500141957762) bank143231 +143232 POINT(41.40376104084587 73.1771989657996) bank143232 +143233 POINT(40.731359342697196 74.61471569155725) bank143233 +143234 POINT(41.47421365147651 74.68932260273274) bank143234 +143235 POINT(41.673601396833504 73.35901358926505) bank143235 +143236 POINT(41.405607320165366 74.5797308258195) bank143236 +143237 POINT(41.288728820192695 73.52948189952616) bank143237 +143238 POINT(40.23642992107075 73.75597650288162) bank143238 +143239 POINT(40.553384700937265 73.56728718606057) bank143239 +143240 POINT(41.451678481123665 74.24436835402717) bank143240 +143241 POINT(39.93873719965782 73.83490632057877) bank143241 +143242 POINT(40.33806596101152 73.31956669673703) bank143242 +143243 POINT(41.024478403936975 73.90263324649808) bank143243 +143244 POINT(40.74020740519185 74.37219657250776) bank143244 +143245 POINT(41.48443262400488 74.77406780649557) bank143245 +143246 POINT(39.76962840838297 73.51636880893052) bank143246 +143247 POINT(41.67243056878299 74.63093828371785) bank143247 +143248 POINT(39.97242700489768 73.90796237545796) bank143248 +143249 POINT(40.58361393504448 73.98677016767151) bank143249 +143250 POINT(41.08374862900834 74.85101801382115) bank143250 +143251 POINT(41.59609710921365 74.91914165475443) bank143251 +143252 POINT(41.234095554162934 74.66973625880593) bank143252 +143253 POINT(40.29520206424348 73.71334700222062) bank143253 +143254 POINT(41.34149838912552 74.3347282560181) bank143254 +143255 POINT(40.925322085443334 73.26055917808031) bank143255 +143256 POINT(40.545173661129716 73.06905620220579) bank143256 +143257 POINT(40.927297087398195 73.73230142847342) bank143257 +143258 POINT(40.010622746787064 73.10369486111232) bank143258 +143259 POINT(41.139426255266116 74.16599124272614) bank143259 +143260 POINT(40.71390892601721 73.49470995679194) bank143260 +143261 POINT(40.63090965187518 73.20269420070923) bank143261 +143262 POINT(40.874019789229706 73.20999176089772) bank143262 +143263 POINT(40.52052593518897 74.84667072492691) bank143263 +143264 POINT(39.81000707029963 73.3169545658422) bank143264 +143265 POINT(40.919303056957354 73.38677332155918) bank143265 +143266 POINT(40.82315260543746 73.80991468170136) bank143266 +143267 POINT(40.10497007540931 73.97025319732438) bank143267 +143268 POINT(39.7259772519742 74.98641641979792) bank143268 +143269 POINT(41.33501486708788 74.94802090903744) bank143269 +143270 POINT(40.66960424522313 74.95167543555877) bank143270 +143271 POINT(40.36485146628064 74.56292393690745) bank143271 +143272 POINT(40.02584975319025 73.39859082505703) bank143272 +143273 POINT(39.75688464599135 74.8430158877885) bank143273 +143274 POINT(41.5985851970094 74.73109291756052) bank143274 +143275 POINT(41.60233428473885 73.50484457320202) bank143275 +143276 POINT(40.76556073230164 74.83801325349843) bank143276 +143277 POINT(41.05406242207952 73.30283176108081) bank143277 +143278 POINT(39.990888929478636 74.07639571709962) bank143278 +143279 POINT(40.67642803453114 73.28442719823373) bank143279 +143280 POINT(40.64438803012351 74.63460262950956) bank143280 +143281 POINT(40.671999342242785 74.56120463175525) bank143281 +143282 POINT(40.022779802995444 74.90374026265121) bank143282 +143283 POINT(41.088678112938645 73.66696704253698) bank143283 +143284 POINT(41.14837670297795 74.65995345156735) bank143284 +143285 POINT(40.278500136491445 73.2114419616207) bank143285 +143286 POINT(40.79851817585259 74.97114099175404) bank143286 +143287 POINT(40.91943002101051 73.06318123043447) bank143287 +143288 POINT(39.81887009280562 74.96881360423767) bank143288 +143289 POINT(41.18733564110707 73.46297558359157) bank143289 +143290 POINT(40.91733348037015 73.55277885807848) bank143290 +143291 POINT(41.518430282551115 73.90834786788912) bank143291 +143292 POINT(39.77373152022666 74.27127381500455) bank143292 +143293 POINT(39.7729107089047 73.858899146644) bank143293 +143294 POINT(39.7141434843757 73.7219064593661) bank143294 +143295 POINT(41.29073333669916 74.5587393030078) bank143295 +143296 POINT(40.37212211706866 74.75052180010397) bank143296 +143297 POINT(39.94381987960145 73.59903240502508) bank143297 +143298 POINT(40.34753442630068 73.0964818832655) bank143298 +143299 POINT(41.023724389741496 74.78094201495136) bank143299 +143300 POINT(41.42682908057557 73.5723212920223) bank143300 +143301 POINT(41.466120958021946 73.74597918848347) bank143301 +143302 POINT(41.36635384558598 73.8195405612379) bank143302 +143303 POINT(41.66302271593101 74.0871690985569) bank143303 +143304 POINT(41.09627228909622 74.90729928228073) bank143304 +143305 POINT(41.50910737326289 73.41822227248315) bank143305 +143306 POINT(41.093356481384504 74.92388871593914) bank143306 +143307 POINT(41.6013779349785 74.38749679595412) bank143307 +143308 POINT(39.97787919791324 74.73841214801143) bank143308 +143309 POINT(41.6313801240641 74.9527161953062) bank143309 +143310 POINT(39.99301586915264 74.30960824593575) bank143310 +143311 POINT(41.33731690785463 74.38633772292906) bank143311 +143312 POINT(39.898847499721676 74.03122130919768) bank143312 +143313 POINT(40.54036238662307 73.24881829228087) bank143313 +143314 POINT(40.662717301918285 74.20395639382761) bank143314 +143315 POINT(40.33398908068494 74.21588325057034) bank143315 +143316 POINT(39.782719502778455 73.98589894765078) bank143316 +143317 POINT(41.093422676768505 73.81510245169719) bank143317 +143318 POINT(39.95136327099867 73.43453923577974) bank143318 +143319 POINT(40.42033511939972 73.76226858607727) bank143319 +143320 POINT(40.12323067768862 74.88057105042066) bank143320 +143321 POINT(40.64049548238482 74.9238850740486) bank143321 +143322 POINT(40.9829792307408 73.71683430364506) bank143322 +143323 POINT(41.64819482443048 74.37579030476394) bank143323 +143324 POINT(40.060471134750884 74.94093201611581) bank143324 +143325 POINT(41.20624978564812 73.41271627717134) bank143325 +143326 POINT(41.53506522807064 74.45043199023567) bank143326 +143327 POINT(40.6308902631431 74.73411803313279) bank143327 +143328 POINT(40.833261186050976 73.10998145522535) bank143328 +143329 POINT(40.978986886556946 74.15528794407254) bank143329 +143330 POINT(39.879834007362774 73.31854969794868) bank143330 +143331 POINT(41.287144649338586 73.19999935711263) bank143331 +143332 POINT(40.604493519976174 73.82149057708222) bank143332 +143333 POINT(40.88775693128996 73.5606478805786) bank143333 +143334 POINT(40.308774155833895 73.49898076246717) bank143334 +143335 POINT(39.86531731022364 73.4952503511267) bank143335 +143336 POINT(40.26091251134553 74.48119815027266) bank143336 +143337 POINT(40.97357484531763 73.8540264572322) bank143337 +143338 POINT(39.94034181549334 74.28670652405674) bank143338 +143339 POINT(40.57047802138288 73.9680131095113) bank143339 +143340 POINT(40.90136667892017 74.3881213556597) bank143340 +143341 POINT(41.7100207668814 73.63750429901313) bank143341 +143342 POINT(41.64580612610063 73.32076442102704) bank143342 +143343 POINT(40.78234489956106 74.42630715698489) bank143343 +143344 POINT(40.93616411319654 73.6600451758665) bank143344 +143345 POINT(41.196642819105826 74.04768053823501) bank143345 +143346 POINT(40.88819226849579 73.75937053743897) bank143346 +143347 POINT(39.90704988623557 74.10425184106447) bank143347 +143348 POINT(41.64606733338141 73.4128591066275) bank143348 +143349 POINT(41.13960165446478 74.81656764691571) bank143349 +143350 POINT(40.58895395733298 73.86567785472916) bank143350 +143351 POINT(41.08651085492854 74.11916537159478) bank143351 +143352 POINT(40.52546957293137 74.37987938097628) bank143352 +143353 POINT(40.47765865144916 74.53007575201485) bank143353 +143354 POINT(39.78287348044247 74.67356158457788) bank143354 +143355 POINT(39.80909602886279 73.93060393339036) bank143355 +143356 POINT(39.956376243634075 74.07427075509536) bank143356 +143357 POINT(40.56743272362895 73.40893592756686) bank143357 +143358 POINT(41.09550849862781 74.10814234374584) bank143358 +143359 POINT(40.26441160820473 74.2776229344964) bank143359 +143360 POINT(41.64352133876323 73.7443667072356) bank143360 +143361 POINT(41.017785900223124 74.51803103720476) bank143361 +143362 POINT(40.25333557929821 73.48192427841128) bank143362 +143363 POINT(40.551828148465674 74.7418726979) bank143363 +143364 POINT(41.199824671916275 73.26973220096089) bank143364 +143365 POINT(39.82179703219996 74.43671801042655) bank143365 +143366 POINT(40.30344534197373 73.82502609814244) bank143366 +143367 POINT(40.381620522153796 73.62076270288672) bank143367 +143368 POINT(41.704410943259234 73.34743493240786) bank143368 +143369 POINT(41.63437872328363 73.75354415332836) bank143369 +143370 POINT(41.235469642332525 73.6981671438598) bank143370 +143371 POINT(40.46775497801874 74.06056919398937) bank143371 +143372 POINT(39.76607635088395 73.54573412233276) bank143372 +143373 POINT(40.31623391098604 74.17662816119585) bank143373 +143374 POINT(41.28614771938917 73.11529389824348) bank143374 +143375 POINT(40.86346195842217 74.08987685597657) bank143375 +143376 POINT(40.691256102303 73.53448808652584) bank143376 +143377 POINT(40.011684033525924 74.59108017375752) bank143377 +143378 POINT(40.50855139653284 73.56506709522847) bank143378 +143379 POINT(41.110953336044886 74.75356106131561) bank143379 +143380 POINT(41.1395093924257 73.67836503167132) bank143380 +143381 POINT(41.423318572907455 74.22977861238587) bank143381 +143382 POINT(41.55011269337719 74.97959753782956) bank143382 +143383 POINT(40.456635861440056 73.88778040938065) bank143383 +143384 POINT(41.66739639658272 74.15722176581315) bank143384 +143385 POINT(39.80850934284886 73.08303125824568) bank143385 +143386 POINT(41.45481866823451 74.6692368516091) bank143386 +143387 POINT(39.95756514691091 74.08055517035211) bank143387 +143388 POINT(40.612384678836214 74.01763068295334) bank143388 +143389 POINT(41.41558092532357 73.88078098826051) bank143389 +143390 POINT(40.321843164392924 74.37115155493882) bank143390 +143391 POINT(40.61759943623596 73.3445876594245) bank143391 +143392 POINT(40.0943760660418 73.38163025923218) bank143392 +143393 POINT(41.60812515502262 74.04613289563777) bank143393 +143394 POINT(40.78385258064844 73.08725057193215) bank143394 +143395 POINT(40.07993180687692 74.34938749921231) bank143395 +143396 POINT(40.86554788365308 73.65451105501349) bank143396 +143397 POINT(40.05857969215335 73.8639546175781) bank143397 +143398 POINT(41.41145668362274 74.02128377831413) bank143398 +143399 POINT(40.77964795500003 73.72185326833396) bank143399 +143400 POINT(41.2550730497602 73.55239963455617) bank143400 +143401 POINT(40.494858706859134 73.88225703198061) bank143401 +143402 POINT(39.876517040106634 73.49519546570981) bank143402 +143403 POINT(41.1752400763732 74.96070543206014) bank143403 +143404 POINT(40.60775996209836 73.5538463819433) bank143404 +143405 POINT(40.388204929336695 73.94187575275377) bank143405 +143406 POINT(39.884387539449285 73.63298324339006) bank143406 +143407 POINT(41.50809078422685 73.82476951060775) bank143407 +143408 POINT(39.732504463371846 73.7034340006003) bank143408 +143409 POINT(41.374132163019915 73.23191260213888) bank143409 +143410 POINT(40.1260704823192 74.51810204118368) bank143410 +143411 POINT(40.43766164022081 74.18322426260204) bank143411 +143412 POINT(40.806747082724485 73.85746099542354) bank143412 +143413 POINT(41.09957514396305 73.14355338227399) bank143413 +143414 POINT(41.27914091549035 73.83777148084405) bank143414 +143415 POINT(39.7523420195269 73.75722188879168) bank143415 +143416 POINT(41.3710119200557 74.44083611729964) bank143416 +143417 POINT(40.65239741154041 73.1310649079808) bank143417 +143418 POINT(40.620736679115694 74.15847772816288) bank143418 +143419 POINT(39.87773620710452 74.96016631856742) bank143419 +143420 POINT(40.6843816830648 74.22221597325198) bank143420 +143421 POINT(41.447579358571716 74.19877951725167) bank143421 +143422 POINT(39.82127328924875 74.28752084092008) bank143422 +143423 POINT(41.64288723359874 74.08184979280588) bank143423 +143424 POINT(40.75296108863593 73.48296196356624) bank143424 +143425 POINT(40.754247026068725 73.68961145215555) bank143425 +143426 POINT(39.72124845655312 73.32470038805391) bank143426 +143427 POINT(41.17033140210145 74.65558592491865) bank143427 +143428 POINT(39.89688169873123 74.80214281597947) bank143428 +143429 POINT(40.41198862427187 73.52126744878555) bank143429 +143430 POINT(39.77763086316497 73.31830540541998) bank143430 +143431 POINT(41.360581093130854 73.46315239219054) bank143431 +143432 POINT(40.82997172778565 73.20830870644177) bank143432 +143433 POINT(39.82487022546957 73.97258632039896) bank143433 +143434 POINT(41.24824948962177 74.66403583537748) bank143434 +143435 POINT(40.26199270277117 73.67473557138865) bank143435 +143436 POINT(40.033081944340324 73.04291377272828) bank143436 +143437 POINT(41.01022022803452 74.35898594896832) bank143437 +143438 POINT(40.14689222692954 73.7168256009978) bank143438 +143439 POINT(40.06064224860693 73.75356508290695) bank143439 +143440 POINT(41.08648532069874 73.76931243946919) bank143440 +143441 POINT(40.83609800828002 73.287921441591) bank143441 +143442 POINT(39.73252306537585 73.0654849035472) bank143442 +143443 POINT(40.54684202290612 73.7620115759832) bank143443 +143444 POINT(39.99335006246882 74.92431405599432) bank143444 +143445 POINT(40.97698257450753 73.03977742898192) bank143445 +143446 POINT(40.66163670786512 73.4028694176183) bank143446 +143447 POINT(41.06001267471758 74.67611575719961) bank143447 +143448 POINT(40.13530737236056 74.50410289390857) bank143448 +143449 POINT(40.85582785567092 74.03383204918583) bank143449 +143450 POINT(41.18994444186076 73.99103657145496) bank143450 +143451 POINT(40.56692235019074 74.03220204018766) bank143451 +143452 POINT(41.44486286953181 74.8801851771094) bank143452 +143453 POINT(41.523043778995664 74.75837309379372) bank143453 +143454 POINT(40.15971520632963 73.87161056769487) bank143454 +143455 POINT(40.97948637107317 74.11085784766206) bank143455 +143456 POINT(40.11925636177516 73.63164625586137) bank143456 +143457 POINT(40.3503523247926 74.48948305815448) bank143457 +143458 POINT(40.521512182630865 73.23246471786298) bank143458 +143459 POINT(41.296979860191925 74.02774893341427) bank143459 +143460 POINT(41.193243203882 74.47933092068945) bank143460 +143461 POINT(41.67779515335499 73.05410201968667) bank143461 +143462 POINT(41.16226275577378 74.83310282079424) bank143462 +143463 POINT(40.946778976892816 74.72205317672272) bank143463 +143464 POINT(39.97420738053963 73.14311261690631) bank143464 +143465 POINT(39.86133418102799 74.25218070456059) bank143465 +143466 POINT(40.32534969253362 74.9361075371677) bank143466 +143467 POINT(39.75657055601225 73.08345205125948) bank143467 +143468 POINT(40.25618672130707 74.3761639780987) bank143468 +143469 POINT(39.884033050183405 74.92268596331255) bank143469 +143470 POINT(41.385399813633214 73.85667804679619) bank143470 +143471 POINT(40.22972787140041 73.48338572614608) bank143471 +143472 POINT(40.98071767120068 73.16712136107773) bank143472 +143473 POINT(40.20523491334416 73.09535910784) bank143473 +143474 POINT(41.400372653485626 74.5649769420972) bank143474 +143475 POINT(40.75203435094816 74.6740826051965) bank143475 +143476 POINT(41.266161974548865 74.06281374216087) bank143476 +143477 POINT(39.84392019350382 73.33784797047701) bank143477 +143478 POINT(41.234813675223585 73.1695926949335) bank143478 +143479 POINT(40.01560353866293 73.33348945815374) bank143479 +143480 POINT(40.05979992946047 73.13597270907474) bank143480 +143481 POINT(41.254442736717735 74.41257158944012) bank143481 +143482 POINT(39.8753971782704 73.30042040455616) bank143482 +143483 POINT(41.55750456686293 73.02174217510048) bank143483 +143484 POINT(41.45521864001105 74.53837545735887) bank143484 +143485 POINT(40.59647598752026 73.73187117005533) bank143485 +143486 POINT(40.303358756427215 73.06972019788353) bank143486 +143487 POINT(40.19674616763272 74.14285951498624) bank143487 +143488 POINT(39.841644665781985 74.24839818487467) bank143488 +143489 POINT(39.71904870658291 73.11527497881974) bank143489 +143490 POINT(39.77829503227195 74.12049705972127) bank143490 +143491 POINT(40.11053719395259 74.79675914905916) bank143491 +143492 POINT(40.35438514539139 73.00632851228755) bank143492 +143493 POINT(40.7292643440234 74.58423759125385) bank143493 +143494 POINT(40.208889970159206 73.86570190103537) bank143494 +143495 POINT(39.810147321507614 73.33028793908954) bank143495 +143496 POINT(40.85444930949013 73.24847657847351) bank143496 +143497 POINT(40.49009490124663 73.05545484614541) bank143497 +143498 POINT(41.38966944495412 73.39910733844057) bank143498 +143499 POINT(41.25638746377186 74.11091716150993) bank143499 +143500 POINT(40.15125664447191 74.35930813707203) bank143500 +143501 POINT(39.71603705897254 74.91294669942116) bank143501 +143502 POINT(40.1970059130572 74.1001937142843) bank143502 +143503 POINT(41.28460213352971 74.12985351769628) bank143503 +143504 POINT(41.09945959214324 73.2967278505506) bank143504 +143505 POINT(41.04440276152082 74.40736132802527) bank143505 +143506 POINT(41.62865832536945 74.520589643486) bank143506 +143507 POINT(40.59854334072442 73.20719569184232) bank143507 +143508 POINT(41.344512774726745 74.43822481339667) bank143508 +143509 POINT(40.70653030692667 74.8142734773868) bank143509 +143510 POINT(39.72169097188179 73.90747071907313) bank143510 +143511 POINT(41.2084549652941 74.87380193595763) bank143511 +143512 POINT(40.693774600650926 74.02593217861653) bank143512 +143513 POINT(41.66154254226011 74.5641398509925) bank143513 +143514 POINT(40.48205286078359 73.93792401107825) bank143514 +143515 POINT(40.57696754938694 74.86617293376045) bank143515 +143516 POINT(40.215074098376896 74.24653013799089) bank143516 +143517 POINT(40.609743995565054 73.46978630814544) bank143517 +143518 POINT(41.10916134853146 73.08392960976423) bank143518 +143519 POINT(40.059837206405945 73.50404610941271) bank143519 +143520 POINT(39.97156795955615 73.5495165027086) bank143520 +143521 POINT(39.76252180361634 73.61873426940734) bank143521 +143522 POINT(40.59051789439533 75.002322450098) bank143522 +143523 POINT(41.539897766859575 74.1316818026072) bank143523 +143524 POINT(40.842679380890466 73.85438604467886) bank143524 +143525 POINT(40.735866865082414 73.62560355960544) bank143525 +143526 POINT(40.269172487632794 74.06207861087509) bank143526 +143527 POINT(40.114044551604124 74.61190471688539) bank143527 +143528 POINT(39.82499680226333 73.19590443375589) bank143528 +143529 POINT(40.431795794333816 73.06917580904444) bank143529 +143530 POINT(40.64237363891317 74.82512594219487) bank143530 +143531 POINT(41.019023962570735 73.44306648766153) bank143531 +143532 POINT(40.395304052686804 73.70880168918593) bank143532 +143533 POINT(39.99528215774836 74.83036369348275) bank143533 +143534 POINT(40.264809144716466 73.33445963650476) bank143534 +143535 POINT(40.674265528289226 74.24416236696923) bank143535 +143536 POINT(40.097197139669696 73.79269043060421) bank143536 +143537 POINT(41.56620852311075 74.24007398692586) bank143537 +143538 POINT(41.1919503432199 74.02880093085987) bank143538 +143539 POINT(41.29219373364882 74.85696943898527) bank143539 +143540 POINT(39.85147668428056 74.7252567381826) bank143540 +143541 POINT(39.84844591163673 73.23588763606857) bank143541 +143542 POINT(40.45343165017906 73.82907677446505) bank143542 +143543 POINT(40.2849861413262 73.01859495313423) bank143543 +143544 POINT(40.79988961457673 73.02673866610795) bank143544 +143545 POINT(41.575350732243535 73.59038982631944) bank143545 +143546 POINT(41.57228080773796 74.18538278827197) bank143546 +143547 POINT(41.6945080443954 74.02758027319825) bank143547 +143548 POINT(39.95417681224418 73.69298526453474) bank143548 +143549 POINT(40.85511469450549 74.36906356890539) bank143549 +143550 POINT(41.518393590356425 74.21770122760357) bank143550 +143551 POINT(41.676847277261196 73.45044123488455) bank143551 +143552 POINT(40.787828201342975 73.74138667689053) bank143552 +143553 POINT(40.70512081225555 73.96671483898976) bank143553 +143554 POINT(41.42330783685525 73.79611548059775) bank143554 +143555 POINT(41.55431778277878 73.78563626966594) bank143555 +143556 POINT(41.34089382706502 73.41263440444989) bank143556 +143557 POINT(41.48321727243805 74.4762427914003) bank143557 +143558 POINT(40.437953962193355 73.46359691550572) bank143558 +143559 POINT(41.70033477582814 73.69963816932793) bank143559 +143560 POINT(39.85331642086338 74.73594236396534) bank143560 +143561 POINT(40.72131418794756 74.0552423616154) bank143561 +143562 POINT(40.35853847887588 73.66883215362304) bank143562 +143563 POINT(40.28174268378882 74.95092236815846) bank143563 +143564 POINT(39.82466903542492 74.37591939535005) bank143564 +143565 POINT(39.92025622818393 73.56674855733257) bank143565 +143566 POINT(39.94998370254535 74.21446250719094) bank143566 +143567 POINT(41.63431781475432 73.24546464564811) bank143567 +143568 POINT(41.56861041419824 73.36808391696829) bank143568 +143569 POINT(41.03336236636976 73.06217794904244) bank143569 +143570 POINT(41.54877801921464 73.95012817687945) bank143570 +143571 POINT(40.0934417437563 74.23351892235938) bank143571 +143572 POINT(40.97703366710888 73.74185152355786) bank143572 +143573 POINT(40.56681468249788 73.67008330408645) bank143573 +143574 POINT(40.4863560164315 73.98728922249104) bank143574 +143575 POINT(40.53321749039626 74.50988845375237) bank143575 +143576 POINT(40.23295429516677 74.2875008440308) bank143576 +143577 POINT(39.95593597240071 74.32666961603563) bank143577 +143578 POINT(41.105925498463236 74.64173935411084) bank143578 +143579 POINT(40.590869793428524 74.0303565345896) bank143579 +143580 POINT(40.01509026104676 74.94650873376581) bank143580 +143581 POINT(39.97474742601978 74.38446103298452) bank143581 +143582 POINT(41.39353817990655 73.45868900968799) bank143582 +143583 POINT(41.0112372601411 74.28382510253492) bank143583 +143584 POINT(40.10555446477776 73.78845673980207) bank143584 +143585 POINT(41.102426996987994 75.00510367983235) bank143585 +143586 POINT(41.25930165468596 73.44897863166823) bank143586 +143587 POINT(40.49461780113773 74.27219891962496) bank143587 +143588 POINT(39.87966945753541 73.71603746577992) bank143588 +143589 POINT(40.571355361026086 74.4919118190326) bank143589 +143590 POINT(40.58916328087629 74.09697952523838) bank143590 +143591 POINT(39.94142624086835 73.82493905148459) bank143591 +143592 POINT(41.44850237834378 73.69761296285209) bank143592 +143593 POINT(39.78808245105602 74.44509466131014) bank143593 +143594 POINT(41.061753614967174 73.25893743270288) bank143594 +143595 POINT(39.90485630104026 74.15902975379296) bank143595 +143596 POINT(41.346024848736896 74.12674946112614) bank143596 +143597 POINT(40.62044438202354 73.50655778448483) bank143597 +143598 POINT(41.48098195958474 74.79002163823837) bank143598 +143599 POINT(40.069695000641495 74.5139474465972) bank143599 +143600 POINT(40.75035390044224 73.12908871792574) bank143600 +143601 POINT(40.0384448743648 74.62716418325373) bank143601 +143602 POINT(40.319524136528486 74.29886333536602) bank143602 +143603 POINT(40.70815065377439 73.12141900690767) bank143603 +143604 POINT(40.48551148834984 73.31111195225176) bank143604 +143605 POINT(40.38796344809205 73.64357245839297) bank143605 +143606 POINT(41.63587451941378 74.04286324881967) bank143606 +143607 POINT(41.291953712301286 74.86590856303476) bank143607 +143608 POINT(40.79497761858316 74.70473215435482) bank143608 +143609 POINT(40.791159231313 74.3445976492682) bank143609 +143610 POINT(39.90627639473871 73.55313181153788) bank143610 +143611 POINT(41.4158069509455 74.47064187422595) bank143611 +143612 POINT(39.99748586711551 73.54319698344209) bank143612 +143613 POINT(41.3047303603396 73.59449125555484) bank143613 +143614 POINT(40.50864662423335 73.2783680686741) bank143614 +143615 POINT(40.46550883121646 73.67251421167691) bank143615 +143616 POINT(40.986272511685904 74.73262725833446) bank143616 +143617 POINT(41.340630941286754 74.54807719330017) bank143617 +143618 POINT(39.84473112894313 73.43630268442345) bank143618 +143619 POINT(41.45669405237253 73.67322739754437) bank143619 +143620 POINT(40.455464051789924 73.94397434922764) bank143620 +143621 POINT(40.022854993962454 73.74676284775444) bank143621 +143622 POINT(40.448889775988015 74.65172842562018) bank143622 +143623 POINT(39.82482336109233 74.03829946202828) bank143623 +143624 POINT(40.17734408377174 73.85597791575597) bank143624 +143625 POINT(41.64459469010165 74.47478426398773) bank143625 +143626 POINT(40.64770272864599 74.00489707923771) bank143626 +143627 POINT(40.315492013316 74.16995078478638) bank143627 +143628 POINT(39.84139677981845 73.4259021595092) bank143628 +143629 POINT(41.15847156549704 73.07294201004694) bank143629 +143630 POINT(40.63702676379612 74.3159628536602) bank143630 +143631 POINT(41.137425979554784 74.84954752232989) bank143631 +143632 POINT(41.26810050897872 73.67895387612816) bank143632 +143633 POINT(41.41032564675993 73.62758302360501) bank143633 +143634 POINT(40.80178819173633 73.69826201526278) bank143634 +143635 POINT(40.58794732778788 74.81927471488366) bank143635 +143636 POINT(40.95681444707254 73.68175025764211) bank143636 +143637 POINT(41.51194378039486 73.11979918429785) bank143637 +143638 POINT(40.27482362639722 74.46473062637993) bank143638 +143639 POINT(39.81948557675912 73.59983163384936) bank143639 +143640 POINT(41.67655418931205 74.29223354054423) bank143640 +143641 POINT(39.890313401555495 74.50497925087852) bank143641 +143642 POINT(39.88382710365471 74.46111353250579) bank143642 +143643 POINT(41.03372774024501 73.65745972808186) bank143643 +143644 POINT(40.65048500788739 74.43146600709873) bank143644 +143645 POINT(40.56362028010642 73.13569668289655) bank143645 +143646 POINT(40.98908834998672 74.8785652135135) bank143646 +143647 POINT(39.80905876373105 74.93277091374576) bank143647 +143648 POINT(39.75414959384345 74.80956303017823) bank143648 +143649 POINT(39.910506921099376 73.18061310433296) bank143649 +143650 POINT(40.80248256100356 74.43532769386154) bank143650 +143651 POINT(39.98494887552929 74.42793987771421) bank143651 +143652 POINT(39.818431577146946 74.18314402130841) bank143652 +143653 POINT(41.153545927194756 73.01746109261985) bank143653 +143654 POINT(41.09945312688758 73.68625001445783) bank143654 +143655 POINT(40.120447420692756 73.7754421858523) bank143655 +143656 POINT(40.920735283498985 73.98656309463017) bank143656 +143657 POINT(40.10413740969768 74.08807180492974) bank143657 +143658 POINT(40.0392712524948 74.15798811143382) bank143658 +143659 POINT(40.458287754117634 73.06478786329005) bank143659 +143660 POINT(41.00111226326486 74.29533030859538) bank143660 +143661 POINT(39.851316648966076 74.35547172073863) bank143661 +143662 POINT(40.61089463513689 74.35355664855055) bank143662 +143663 POINT(40.26459237618656 74.43476707023486) bank143663 +143664 POINT(40.353200017653585 73.51230856025165) bank143664 +143665 POINT(41.10892275207989 74.6943166462007) bank143665 +143666 POINT(39.75158947257803 73.24944251707566) bank143666 +143667 POINT(39.77652630184315 73.91175182215291) bank143667 +143668 POINT(39.731492039729716 73.08911646457085) bank143668 +143669 POINT(40.85685629392737 73.63748909899222) bank143669 +143670 POINT(40.965087787506455 74.49734360973349) bank143670 +143671 POINT(39.83836944153401 73.16200400794494) bank143671 +143672 POINT(41.23736810532829 74.08757997082068) bank143672 +143673 POINT(39.75349304230011 73.02719074810813) bank143673 +143674 POINT(41.7029836108659 73.80215072675777) bank143674 +143675 POINT(41.13682804671953 74.16387005511844) bank143675 +143676 POINT(41.588005466724944 73.26317109059026) bank143676 +143677 POINT(40.873476590465835 73.45771911037095) bank143677 +143678 POINT(40.28634741148455 73.86054070276656) bank143678 +143679 POINT(41.41761425759236 74.66681816762352) bank143679 +143680 POINT(40.368557599116095 74.91227606573929) bank143680 +143681 POINT(39.73576194795404 73.11545854025252) bank143681 +143682 POINT(41.06012233935843 74.59094364957899) bank143682 +143683 POINT(40.99179931691746 73.28716696934977) bank143683 +143684 POINT(41.685245791705555 73.72709411071874) bank143684 +143685 POINT(41.21059091616797 73.39044733120075) bank143685 +143686 POINT(40.58283723064212 73.05409547439974) bank143686 +143687 POINT(40.64738227183576 73.78393837295836) bank143687 +143688 POINT(40.000205718404516 74.83083361385236) bank143688 +143689 POINT(39.780617011087166 73.55905969791604) bank143689 +143690 POINT(39.99631285003576 74.42774076937067) bank143690 +143691 POINT(39.944651284590144 73.60918760784872) bank143691 +143692 POINT(41.27572391351037 74.93598945560815) bank143692 +143693 POINT(41.43082188078996 73.42910139433602) bank143693 +143694 POINT(40.3063162796132 74.03466712810956) bank143694 +143695 POINT(41.03987843052801 74.11654931427022) bank143695 +143696 POINT(40.00315115370644 74.23197988015488) bank143696 +143697 POINT(39.80420741603642 73.21466003209198) bank143697 +143698 POINT(41.55317176364609 73.43838048614448) bank143698 +143699 POINT(40.89688961574607 74.84362278384505) bank143699 +143700 POINT(41.48839345313739 74.00257934341828) bank143700 +143701 POINT(41.48667015750932 74.40929952844417) bank143701 +143702 POINT(40.378848888828266 74.50801731178372) bank143702 +143703 POINT(41.05880797468359 74.80547047189158) bank143703 +143704 POINT(40.441145099424524 73.03081262053841) bank143704 +143705 POINT(41.646518734261115 73.51248658070709) bank143705 +143706 POINT(40.876487953368276 74.93387555191863) bank143706 +143707 POINT(41.32334293727494 74.83433403954022) bank143707 +143708 POINT(40.3935418884621 74.52144348389005) bank143708 +143709 POINT(40.0535540933304 73.17596187035038) bank143709 +143710 POINT(41.01198840939788 73.35225214484448) bank143710 +143711 POINT(40.55364590980267 74.63261877255542) bank143711 +143712 POINT(41.67082498698848 73.20997769311917) bank143712 +143713 POINT(40.8484710179466 73.80038972042539) bank143713 +143714 POINT(40.78750881393269 74.21762501707371) bank143714 +143715 POINT(40.7772127583178 74.16130752065334) bank143715 +143716 POINT(41.03356924333987 73.20332135531426) bank143716 +143717 POINT(41.12503189489864 73.38524562473879) bank143717 +143718 POINT(40.75994879624782 74.08654047990707) bank143718 +143719 POINT(39.813077303632184 73.14199615582093) bank143719 +143720 POINT(40.08414208327427 74.16302082154775) bank143720 +143721 POINT(41.369335585237415 73.66932860693909) bank143721 +143722 POINT(40.93401946358659 74.12746521612367) bank143722 +143723 POINT(40.29173575120655 73.54476789186238) bank143723 +143724 POINT(40.38333485759896 73.21124541481913) bank143724 +143725 POINT(40.940109751909084 73.17468529275018) bank143725 +143726 POINT(40.58137938510237 73.31430620756962) bank143726 +143727 POINT(41.34039440584045 74.16398841629845) bank143727 +143728 POINT(41.29275354266036 73.30985244023738) bank143728 +143729 POINT(41.325553651012285 74.24824597870459) bank143729 +143730 POINT(40.51429725236365 74.44551117297473) bank143730 +143731 POINT(41.26246766292047 73.52412566200614) bank143731 +143732 POINT(40.77762505972657 74.12210981166665) bank143732 +143733 POINT(40.58149215301371 74.07241919054294) bank143733 +143734 POINT(40.55740278405732 73.14718643772711) bank143734 +143735 POINT(40.54832764835393 73.03651859604133) bank143735 +143736 POINT(40.42785964055551 74.29650332141735) bank143736 +143737 POINT(40.139633291505376 74.05119424640618) bank143737 +143738 POINT(41.60663378752802 74.28980925680182) bank143738 +143739 POINT(41.56654229971511 74.51020330526202) bank143739 +143740 POINT(41.60900797238034 74.96909357324652) bank143740 +143741 POINT(40.525350862665476 73.85401685039909) bank143741 +143742 POINT(40.35578237616384 73.24466009533025) bank143742 +143743 POINT(39.883298351172755 74.62674095883757) bank143743 +143744 POINT(41.44317180360227 73.10285710916341) bank143744 +143745 POINT(41.282322259748504 73.51556952191275) bank143745 +143746 POINT(41.35965063298848 74.15322387981102) bank143746 +143747 POINT(40.86916958745405 73.32707346942932) bank143747 +143748 POINT(39.71503330455012 74.62711958925894) bank143748 +143749 POINT(40.551892484470216 74.29558204041155) bank143749 +143750 POINT(40.68157800652192 73.07449427084526) bank143750 +143751 POINT(40.36049710852865 73.31373355210701) bank143751 +143752 POINT(40.518144504865404 74.19462629548755) bank143752 +143753 POINT(40.07204953897263 74.85555835406952) bank143753 +143754 POINT(41.39099286924518 74.88820564780347) bank143754 +143755 POINT(41.41310900519126 74.61465858857305) bank143755 +143756 POINT(41.33188521425373 73.14650189885543) bank143756 +143757 POINT(41.171733269329295 73.81486914997156) bank143757 +143758 POINT(40.69583824820298 74.35844890638553) bank143758 +143759 POINT(39.89821541323992 74.58115795023748) bank143759 +143760 POINT(39.78147695949866 73.38317783533395) bank143760 +143761 POINT(40.39655718315922 73.72115357739708) bank143761 +143762 POINT(40.0474369356951 73.17149805987852) bank143762 +143763 POINT(39.78435708812204 74.04550419999075) bank143763 +143764 POINT(40.000117252338434 74.51786896277505) bank143764 +143765 POINT(41.22427713930042 74.5934112186276) bank143765 +143766 POINT(40.37973940619957 74.72636444577823) bank143766 +143767 POINT(40.466307297425686 73.65292286476087) bank143767 +143768 POINT(41.627412811612245 74.26205616654907) bank143768 +143769 POINT(40.53465238142757 74.1405499585578) bank143769 +143770 POINT(40.16051380422417 73.90032737723268) bank143770 +143771 POINT(41.25237978875956 74.25835609277487) bank143771 +143772 POINT(41.160974644566494 74.6554666523742) bank143772 +143773 POINT(41.60958819148969 73.44780155946115) bank143773 +143774 POINT(41.67473532361949 75.00509360763557) bank143774 +143775 POINT(40.65515906079259 74.96471322851718) bank143775 +143776 POINT(40.0855477895036 74.26972877295832) bank143776 +143777 POINT(40.687931136820694 73.06049500815672) bank143777 +143778 POINT(41.56355183551346 74.17084529275796) bank143778 +143779 POINT(41.50425522864866 73.71935312858484) bank143779 +143780 POINT(41.157658402856136 73.52739567540169) bank143780 +143781 POINT(40.37078369802653 74.95290735882173) bank143781 +143782 POINT(39.8289155673744 73.4187495844852) bank143782 +143783 POINT(40.501253316281634 74.32170318908544) bank143783 +143784 POINT(40.38055580285156 73.90293348524817) bank143784 +143785 POINT(41.49479064854326 74.0393187865282) bank143785 +143786 POINT(39.76761363785833 73.02229929619779) bank143786 +143787 POINT(40.39803669303043 74.117319089176) bank143787 +143788 POINT(41.22258349506517 73.95222174530284) bank143788 +143789 POINT(41.460784686484374 73.4019208701292) bank143789 +143790 POINT(41.613585520075574 73.44725166556114) bank143790 +143791 POINT(41.18567943100511 74.96479855066654) bank143791 +143792 POINT(39.78511742024464 73.18615513071194) bank143792 +143793 POINT(41.502855887672 73.434824870358) bank143793 +143794 POINT(41.234154632705504 73.2973104608231) bank143794 +143795 POINT(41.44923178581871 74.75086204209454) bank143795 +143796 POINT(41.07508189679485 73.57137864597162) bank143796 +143797 POINT(41.292930996659344 74.65070226731652) bank143797 +143798 POINT(40.900226750583485 74.52812486652077) bank143798 +143799 POINT(40.688842567859595 73.71758693247604) bank143799 +143800 POINT(40.483750488569115 73.02691353509464) bank143800 +143801 POINT(41.395180315453906 73.3750228934607) bank143801 +143802 POINT(40.47388323741656 74.54032665482916) bank143802 +143803 POINT(41.33208444426302 74.26516393487718) bank143803 +143804 POINT(41.18395044162404 74.3934087153528) bank143804 +143805 POINT(41.2636117705775 73.96679217806583) bank143805 +143806 POINT(40.41989721883209 73.26738175312093) bank143806 +143807 POINT(40.13723565062079 74.79518523830752) bank143807 +143808 POINT(40.085648390098115 73.93519995499268) bank143808 +143809 POINT(40.658577630947974 73.19170036048337) bank143809 +143810 POINT(41.40471407845754 73.45325591034137) bank143810 +143811 POINT(40.73137347185992 73.41516081159182) bank143811 +143812 POINT(39.94788212031126 73.21501041356031) bank143812 +143813 POINT(40.076223274642466 73.7463579162622) bank143813 +143814 POINT(40.94036108379031 73.75998211355734) bank143814 +143815 POINT(40.463292374511795 74.41482589009824) bank143815 +143816 POINT(40.886145807492525 74.4301057981907) bank143816 +143817 POINT(41.37413027553069 73.22991656037114) bank143817 +143818 POINT(40.70160194037343 73.20948318212339) bank143818 +143819 POINT(40.55780821068157 74.06536690963026) bank143819 +143820 POINT(40.727941382729384 73.29749801827684) bank143820 +143821 POINT(40.753126749926125 74.22449663933351) bank143821 +143822 POINT(41.40180476780226 73.33340586264035) bank143822 +143823 POINT(41.29897994206654 74.29395281623604) bank143823 +143824 POINT(40.91744391765 73.63230462913938) bank143824 +143825 POINT(40.38931237899528 74.02600386737299) bank143825 +143826 POINT(40.98685015121744 73.83192994127917) bank143826 +143827 POINT(40.41507864917376 74.3307189253883) bank143827 +143828 POINT(41.165747809026584 74.31511877456828) bank143828 +143829 POINT(41.01429454634523 74.36223325625637) bank143829 +143830 POINT(41.48495150978798 73.7318835482244) bank143830 +143831 POINT(40.932856964022804 73.2434070988201) bank143831 +143832 POINT(41.50865617698601 73.89893445537966) bank143832 +143833 POINT(40.11906044319106 74.67914425042308) bank143833 +143834 POINT(41.4126916518937 74.94579864026056) bank143834 +143835 POINT(40.65478419958916 73.39498460583916) bank143835 +143836 POINT(40.917696596163786 73.24315016586972) bank143836 +143837 POINT(41.638425061633406 74.93153506980477) bank143837 +143838 POINT(40.65163818644172 74.61153001202393) bank143838 +143839 POINT(39.7894578888345 73.68053882406015) bank143839 +143840 POINT(39.723196210874754 73.89034219961587) bank143840 +143841 POINT(40.96843560200914 74.05656678192632) bank143841 +143842 POINT(39.76160179191825 74.17520916472606) bank143842 +143843 POINT(40.102926618741016 73.07359560142247) bank143843 +143844 POINT(41.28943681059874 73.71571358221176) bank143844 +143845 POINT(39.95566094429829 74.86562554532043) bank143845 +143846 POINT(41.57036144835879 73.62739234633317) bank143846 +143847 POINT(40.041131583266804 74.47035528936635) bank143847 +143848 POINT(39.792447394891106 74.79500024169838) bank143848 +143849 POINT(40.24146696223518 73.50831571172166) bank143849 +143850 POINT(40.12790228656231 74.84063138144506) bank143850 +143851 POINT(40.558600476519345 73.59365742807418) bank143851 +143852 POINT(39.7377264710282 74.81977657707208) bank143852 +143853 POINT(41.00101776246613 74.40988219311592) bank143853 +143854 POINT(40.9489382752037 74.19129582503189) bank143854 +143855 POINT(40.624747156325775 73.0113123522393) bank143855 +143856 POINT(40.428082408986825 74.95576075597805) bank143856 +143857 POINT(40.69571908684118 74.03305533954925) bank143857 +143858 POINT(40.60985709885108 74.5433581347621) bank143858 +143859 POINT(40.7929066857941 73.80037058613703) bank143859 +143860 POINT(40.23188882597113 73.50385201632112) bank143860 +143861 POINT(39.77328857147477 74.51257394895637) bank143861 +143862 POINT(39.801991668961776 74.85442539228951) bank143862 +143863 POINT(41.15311736135433 73.73772861750804) bank143863 +143864 POINT(40.83498079323029 74.8707171201661) bank143864 +143865 POINT(40.9544628613674 74.92679462638678) bank143865 +143866 POINT(39.844055002453345 73.66318172788543) bank143866 +143867 POINT(40.07611751219257 74.69636438116225) bank143867 +143868 POINT(41.375739439334566 74.46381609158344) bank143868 +143869 POINT(40.75134329095874 73.30678597317302) bank143869 +143870 POINT(41.22753659761858 73.26531844426957) bank143870 +143871 POINT(40.452600785913226 73.1812528924981) bank143871 +143872 POINT(39.821985365544805 74.99536969716736) bank143872 +143873 POINT(40.499256332217854 73.98361632407524) bank143873 +143874 POINT(41.131878647206364 73.59091584566772) bank143874 +143875 POINT(41.632229129657226 74.4938668437348) bank143875 +143876 POINT(41.12844304663932 74.66320501004209) bank143876 +143877 POINT(40.263646918104946 74.96086312346243) bank143877 +143878 POINT(41.60192541630538 74.17404634722747) bank143878 +143879 POINT(41.447691224383334 73.66630259615144) bank143879 +143880 POINT(41.296891993022044 73.67225704079313) bank143880 +143881 POINT(41.60052611257769 74.75821254514331) bank143881 +143882 POINT(41.57531067354851 73.47787366682554) bank143882 +143883 POINT(41.32459176282866 73.98342882227311) bank143883 +143884 POINT(40.93844336581245 73.04234862275042) bank143884 +143885 POINT(40.02375025062554 74.56809518710935) bank143885 +143886 POINT(39.85901630051468 73.46426058111918) bank143886 +143887 POINT(41.041165192095654 73.88371763592154) bank143887 +143888 POINT(40.183356802387806 73.248811411201) bank143888 +143889 POINT(40.737982984195604 74.67949002789737) bank143889 +143890 POINT(41.63412319284113 73.9069084093586) bank143890 +143891 POINT(40.73282077747337 73.05598972380324) bank143891 +143892 POINT(39.98463484648414 74.06632348485728) bank143892 +143893 POINT(41.113422477067175 73.10842418011487) bank143893 +143894 POINT(40.82981332565994 74.4349372109205) bank143894 +143895 POINT(40.175861818131935 73.81328021807315) bank143895 +143896 POINT(41.459555623723816 74.40590699542078) bank143896 +143897 POINT(40.80839134636891 73.70934565854691) bank143897 +143898 POINT(40.417268687776584 74.59944371314286) bank143898 +143899 POINT(41.136396373152635 74.83303846507174) bank143899 +143900 POINT(39.886252859464875 74.29819814131305) bank143900 +143901 POINT(40.400431870448536 73.6787765193366) bank143901 +143902 POINT(40.466015677369036 73.90365818095057) bank143902 +143903 POINT(40.87070807678372 73.0944413417317) bank143903 +143904 POINT(41.089080409941964 74.31994082184576) bank143904 +143905 POINT(40.41710323214761 73.70116294616808) bank143905 +143906 POINT(40.6780513187656 73.5513938104514) bank143906 +143907 POINT(41.21582224597393 74.899425776977) bank143907 +143908 POINT(40.93221032158882 74.11616380745376) bank143908 +143909 POINT(41.273781372545166 74.88974810467676) bank143909 +143910 POINT(40.91889660092281 74.12958312729322) bank143910 +143911 POINT(41.4843913810243 74.69847274753171) bank143911 +143912 POINT(41.09422255528933 74.05823521055885) bank143912 +143913 POINT(41.39958903660497 73.44733011808273) bank143913 +143914 POINT(40.51444973409386 73.93708226612142) bank143914 +143915 POINT(41.46689859588047 73.09892331749563) bank143915 +143916 POINT(40.110134417432334 74.0553709833655) bank143916 +143917 POINT(40.519662159097805 74.70414835183993) bank143917 +143918 POINT(40.03714369723189 74.72945427641068) bank143918 +143919 POINT(39.74206379371482 74.82466438605155) bank143919 +143920 POINT(40.937601858509396 74.2141819302991) bank143920 +143921 POINT(40.42484369173784 74.70802536121676) bank143921 +143922 POINT(41.20056831028945 74.18752025310478) bank143922 +143923 POINT(41.61510342514415 74.76546415575284) bank143923 +143924 POINT(40.024676016221086 74.98226798070442) bank143924 +143925 POINT(39.83726094246361 73.97877791039184) bank143925 +143926 POINT(41.25999923372247 73.20122430674162) bank143926 +143927 POINT(40.295161956293015 74.04033209897646) bank143927 +143928 POINT(41.55814786657061 73.3507753049797) bank143928 +143929 POINT(40.01744124704399 73.19749347071128) bank143929 +143930 POINT(40.56771654853826 74.13636400642888) bank143930 +143931 POINT(41.04980625983449 73.95083755842062) bank143931 +143932 POINT(41.46786586537546 74.45987214600622) bank143932 +143933 POINT(40.73416582390571 74.29569263077838) bank143933 +143934 POINT(41.172745272040046 74.25361470263644) bank143934 +143935 POINT(40.35877515796512 74.09700394664428) bank143935 +143936 POINT(40.73163307303637 73.73426656693037) bank143936 +143937 POINT(41.33176379903166 74.08217790587788) bank143937 +143938 POINT(40.1524259726499 74.51820860816113) bank143938 +143939 POINT(40.70946731954017 73.4983963298789) bank143939 +143940 POINT(41.10305864610116 73.19637554726741) bank143940 +143941 POINT(40.61836287283574 74.42675098785683) bank143941 +143942 POINT(40.44980607018061 74.59953873750533) bank143942 +143943 POINT(40.59621453352494 73.33161186904488) bank143943 +143944 POINT(41.12957675410061 73.27321301380788) bank143944 +143945 POINT(41.164952293882045 73.31313947566585) bank143945 +143946 POINT(41.21406873430673 74.49639582671873) bank143946 +143947 POINT(39.99087610745946 74.36403863943278) bank143947 +143948 POINT(40.879387818309596 73.60380570759767) bank143948 +143949 POINT(40.20747711186696 73.64214601926594) bank143949 +143950 POINT(40.84552767845844 74.91675284177104) bank143950 +143951 POINT(41.63627890911278 73.04719118596729) bank143951 +143952 POINT(41.028095353332546 74.82146507121632) bank143952 +143953 POINT(41.175548913998036 73.77430105238344) bank143953 +143954 POINT(40.632140673316435 74.70211256909501) bank143954 +143955 POINT(41.13754973065985 73.51332209492354) bank143955 +143956 POINT(40.18859601057299 73.8771026487922) bank143956 +143957 POINT(41.41534178035494 74.88501834176206) bank143957 +143958 POINT(41.36702886231142 73.10225392237636) bank143958 +143959 POINT(40.15320998627396 73.19948379289649) bank143959 +143960 POINT(40.20074158308647 74.48284932236824) bank143960 +143961 POINT(40.29213714969942 74.7018203581075) bank143961 +143962 POINT(41.28294348485926 74.1570336147871) bank143962 +143963 POINT(40.84047937562274 73.59957015917617) bank143963 +143964 POINT(39.844932796091 73.7351247933898) bank143964 +143965 POINT(40.66548013506629 73.36208345153875) bank143965 +143966 POINT(40.63267999934794 74.68464836580688) bank143966 +143967 POINT(40.174252017758995 74.41729090511049) bank143967 +143968 POINT(39.76372404867576 74.60346855791381) bank143968 +143969 POINT(39.75692373488141 73.6899714441675) bank143969 +143970 POINT(41.39737971697551 73.82903770007599) bank143970 +143971 POINT(41.05278230028674 73.96976968770186) bank143971 +143972 POINT(40.757456397233724 73.41957845235517) bank143972 +143973 POINT(40.15469314921295 73.96006221324967) bank143973 +143974 POINT(40.00281348670362 74.2307067375415) bank143974 +143975 POINT(41.36925723328845 73.56227660437828) bank143975 +143976 POINT(40.30415837117685 73.16892449153065) bank143976 +143977 POINT(41.09720127981553 73.48392654422544) bank143977 +143978 POINT(40.5351394438625 74.92768772116304) bank143978 +143979 POINT(39.91754260866291 73.75621491591703) bank143979 +143980 POINT(40.12582313816219 73.08433556824875) bank143980 +143981 POINT(40.87525057567362 74.13789957873648) bank143981 +143982 POINT(41.40730093173532 74.79920530865371) bank143982 +143983 POINT(41.21239402633807 74.23138673162268) bank143983 +143984 POINT(41.24460404137352 73.67110151117838) bank143984 +143985 POINT(40.42456001772258 73.31844833389687) bank143985 +143986 POINT(40.58369832911317 74.7367900515681) bank143986 +143987 POINT(41.32344625957204 74.99839258053296) bank143987 +143988 POINT(39.93096906458463 74.50935134551641) bank143988 +143989 POINT(41.53706559124961 74.61959525813901) bank143989 +143990 POINT(41.3335417711635 73.58300927786755) bank143990 +143991 POINT(41.07872379963517 73.92413934037332) bank143991 +143992 POINT(39.99557186291204 73.29224132566036) bank143992 +143993 POINT(41.530979610603396 73.36216709568248) bank143993 +143994 POINT(40.66001769252827 74.04887134705926) bank143994 +143995 POINT(40.325576925431584 73.23817382491498) bank143995 +143996 POINT(40.92969176889937 73.11139151385576) bank143996 +143997 POINT(41.21563781141386 74.21788861629891) bank143997 +143998 POINT(40.568762649214406 74.35783302712503) bank143998 +143999 POINT(40.95271062784651 73.75171350666204) bank143999 +144000 POINT(41.58737951948654 74.0332689820385) bank144000 +144001 POINT(40.67900053324405 74.57169214008054) bank144001 +144002 POINT(40.12677269367004 73.27718274139028) bank144002 +144003 POINT(40.35788878724758 73.65170385656883) bank144003 +144004 POINT(40.63123687456206 74.9662515177279) bank144004 +144005 POINT(41.382369974265984 74.07819885301923) bank144005 +144006 POINT(40.88238539254324 74.05155220836234) bank144006 +144007 POINT(41.508030807738784 74.4306076449444) bank144007 +144008 POINT(41.45885833332091 73.36579723655812) bank144008 +144009 POINT(41.30117329110487 73.34596678946765) bank144009 +144010 POINT(40.78959454147431 74.21199632712164) bank144010 +144011 POINT(39.967870798450704 73.02204365358632) bank144011 +144012 POINT(41.26317260455057 73.18800801593405) bank144012 +144013 POINT(40.01797774033433 74.36662639573801) bank144013 +144014 POINT(40.65973782108332 73.83972213022815) bank144014 +144015 POINT(40.24131294579112 74.56381521930963) bank144015 +144016 POINT(40.33263893185175 74.54121183663692) bank144016 +144017 POINT(40.45073676245118 74.88494683784846) bank144017 +144018 POINT(40.930369392991054 73.46736699894265) bank144018 +144019 POINT(41.520174509946635 73.41311628612104) bank144019 +144020 POINT(40.951998384539564 73.39938110033849) bank144020 +144021 POINT(41.51146156041736 73.334556070313) bank144021 +144022 POINT(40.54009458221715 73.70060812023509) bank144022 +144023 POINT(41.358855896723476 74.17926303898486) bank144023 +144024 POINT(39.7138941334521 73.62115396540342) bank144024 +144025 POINT(40.31491796408929 73.4296006192418) bank144025 +144026 POINT(40.32395729840205 73.16056090895803) bank144026 +144027 POINT(41.35160054971562 73.68473697949347) bank144027 +144028 POINT(41.27270507668445 74.0614069752939) bank144028 +144029 POINT(41.70562933917397 74.82743573976772) bank144029 +144030 POINT(40.56452759497309 73.67820112297801) bank144030 +144031 POINT(40.59097489758781 73.72063163459028) bank144031 +144032 POINT(41.12491554486458 73.71138791931045) bank144032 +144033 POINT(40.482122761975276 74.74351491845803) bank144033 +144034 POINT(41.63070539465719 74.06649934262693) bank144034 +144035 POINT(40.34000715031576 74.0525619812414) bank144035 +144036 POINT(39.92695414578772 73.23069590958387) bank144036 +144037 POINT(40.08613492333969 74.40302960447298) bank144037 +144038 POINT(40.33423085943994 73.01547872480627) bank144038 +144039 POINT(41.6121210054623 73.8079258806602) bank144039 +144040 POINT(41.61958590610616 73.31121031911364) bank144040 +144041 POINT(41.02066946262192 74.54753813156313) bank144041 +144042 POINT(40.56196571088672 74.2742722050682) bank144042 +144043 POINT(40.090386082654334 73.53399034738803) bank144043 +144044 POINT(40.683122535858786 73.33387321466155) bank144044 +144045 POINT(39.986265360869886 73.04464001636117) bank144045 +144046 POINT(39.89090885690875 73.78537671762098) bank144046 +144047 POINT(40.7419664739546 74.00810317401691) bank144047 +144048 POINT(39.774138017782974 73.23280030569174) bank144048 +144049 POINT(39.95143359605999 74.12135692142833) bank144049 +144050 POINT(39.80774540112795 73.73189458801974) bank144050 +144051 POINT(40.711883721429615 74.50102471650682) bank144051 +144052 POINT(40.63024370396906 74.05456280125507) bank144052 +144053 POINT(39.73159098432543 73.13841186821018) bank144053 +144054 POINT(40.87418733640017 74.61904112782864) bank144054 +144055 POINT(40.66241765406916 74.87493107111409) bank144055 +144056 POINT(40.68052821590232 73.43460354207576) bank144056 +144057 POINT(40.760603667687995 74.38253346453303) bank144057 +144058 POINT(40.46112511716869 74.40215209932514) bank144058 +144059 POINT(40.80427021745313 74.93181958351586) bank144059 +144060 POINT(41.02104224219397 73.97463498528171) bank144060 +144061 POINT(40.82043936966138 73.2284260157918) bank144061 +144062 POINT(41.11957124367886 73.54527375975837) bank144062 +144063 POINT(40.189829330216824 74.35739063278004) bank144063 +144064 POINT(40.90324977800223 73.79542463972513) bank144064 +144065 POINT(41.48303219223439 73.80529839183113) bank144065 +144066 POINT(41.48451881479318 74.83677940369978) bank144066 +144067 POINT(40.55702643968643 73.93401665577912) bank144067 +144068 POINT(40.41198057898649 73.31909120024426) bank144068 +144069 POINT(40.62654182775767 73.28748526941364) bank144069 +144070 POINT(41.12895993738393 73.65611384879605) bank144070 +144071 POINT(40.56691608107768 73.57830285756698) bank144071 +144072 POINT(40.161143102090975 73.60748887406174) bank144072 +144073 POINT(41.268354632006655 73.45846351147732) bank144073 +144074 POINT(41.642021584695605 73.80774411941178) bank144074 +144075 POINT(40.23849613079985 74.46712992355604) bank144075 +144076 POINT(39.9414158766302 74.75139429766979) bank144076 +144077 POINT(40.558376630888034 73.66529895631736) bank144077 +144078 POINT(41.17135508865026 73.34506139940541) bank144078 +144079 POINT(40.396504971361395 73.80092946863708) bank144079 +144080 POINT(40.592179461450854 73.65298877946546) bank144080 +144081 POINT(40.83162227489838 73.86260139566961) bank144081 +144082 POINT(40.32749774459123 74.55779893697773) bank144082 +144083 POINT(40.527662029285786 73.43335449405905) bank144083 +144084 POINT(40.32302774522817 73.2245150300769) bank144084 +144085 POINT(41.381349043538066 73.99017728148021) bank144085 +144086 POINT(40.735438546805646 74.80812496972732) bank144086 +144087 POINT(40.797852619082754 73.09974174259767) bank144087 +144088 POINT(40.56522217107831 73.65142825729667) bank144088 +144089 POINT(39.95922655782265 73.75062116155162) bank144089 +144090 POINT(41.49579424264934 73.6877224238804) bank144090 +144091 POINT(40.536362422697344 73.87490924434202) bank144091 +144092 POINT(41.52110387427684 74.05386760851768) bank144092 +144093 POINT(39.84264571498783 73.30632597012111) bank144093 +144094 POINT(40.76967483449772 74.78297404799127) bank144094 +144095 POINT(41.687705524991244 73.84506842430106) bank144095 +144096 POINT(39.82022399020344 74.96836170988017) bank144096 +144097 POINT(41.13097575525276 74.77942802648414) bank144097 +144098 POINT(41.451697899192595 74.89306808658696) bank144098 +144099 POINT(41.37860219939042 73.26194203368505) bank144099 +144100 POINT(40.950911073377526 74.31064478628875) bank144100 +144101 POINT(40.11356200539561 74.80097419042153) bank144101 +144102 POINT(41.17120398713771 73.03630962248616) bank144102 +144103 POINT(40.587726150515756 73.34087575948355) bank144103 +144104 POINT(41.52163332600197 74.45244775220932) bank144104 +144105 POINT(41.547174451420254 74.69766073897145) bank144105 +144106 POINT(40.084982033231036 73.3571113168836) bank144106 +144107 POINT(41.173499977210064 73.31866732980687) bank144107 +144108 POINT(41.252817906217956 74.36651064280262) bank144108 +144109 POINT(40.27614617290816 74.08823366953855) bank144109 +144110 POINT(40.809074858468065 73.7341777643236) bank144110 +144111 POINT(41.71063303771965 74.67838448980974) bank144111 +144112 POINT(40.93072105015893 73.12483730177384) bank144112 +144113 POINT(40.295190398148314 73.40427268975881) bank144113 +144114 POINT(40.03805069718065 74.3595579876928) bank144114 +144115 POINT(40.355497645432365 73.17890463618025) bank144115 +144116 POINT(40.11949787024732 73.14337737916782) bank144116 +144117 POINT(39.867941679166556 74.88333246650693) bank144117 +144118 POINT(40.51889360058163 73.46097066066419) bank144118 +144119 POINT(41.49302253183384 73.22621861264184) bank144119 +144120 POINT(39.9993149395359 73.19376696749234) bank144120 +144121 POINT(41.34822261564669 73.93569517645973) bank144121 +144122 POINT(40.426445388977434 73.81028586458555) bank144122 +144123 POINT(41.472801575008816 73.11328626075276) bank144123 +144124 POINT(41.510717752665876 73.45326776989025) bank144124 +144125 POINT(40.84855138795778 74.16093724717341) bank144125 +144126 POINT(40.46427573970044 73.45577861352005) bank144126 +144127 POINT(40.691025351199244 74.4532266957623) bank144127 +144128 POINT(41.01056241464898 74.18030701556539) bank144128 +144129 POINT(41.50296072421145 74.19778825019364) bank144129 +144130 POINT(41.35974615337006 74.73366478461982) bank144130 +144131 POINT(40.93729837682647 74.87382381638238) bank144131 +144132 POINT(41.134842887729015 74.53326988956871) bank144132 +144133 POINT(41.23014051603398 74.95679383258813) bank144133 +144134 POINT(41.04343948045962 73.41808060142756) bank144134 +144135 POINT(41.315923009484266 73.15611546756992) bank144135 +144136 POINT(41.23787585443271 73.90130696257944) bank144136 +144137 POINT(40.65118065873525 74.3521431131345) bank144137 +144138 POINT(39.83397160653142 73.81577398979749) bank144138 +144139 POINT(39.86526704544998 73.77797875978376) bank144139 +144140 POINT(40.30092620054922 74.3924132598192) bank144140 +144141 POINT(41.09170590507637 73.55949539066452) bank144141 +144142 POINT(39.928630670967806 73.96190974629923) bank144142 +144143 POINT(40.84259356677556 74.54525756311119) bank144143 +144144 POINT(41.075566869214235 74.88004404591165) bank144144 +144145 POINT(41.368298241566926 74.95705506576286) bank144145 +144146 POINT(40.35641332458836 74.96685909699292) bank144146 +144147 POINT(40.97081077259474 73.17878408661035) bank144147 +144148 POINT(41.47151627169589 74.3618062070894) bank144148 +144149 POINT(40.357740352492485 74.18030808735375) bank144149 +144150 POINT(40.35984336786935 74.65688291892829) bank144150 +144151 POINT(40.86577108802935 74.59982958054195) bank144151 +144152 POINT(41.470723591360986 74.12385905103105) bank144152 +144153 POINT(41.54052976939872 74.74236905095191) bank144153 +144154 POINT(40.23307893558468 73.89843356558771) bank144154 +144155 POINT(39.81679719321614 73.53462501268496) bank144155 +144156 POINT(40.193502731793046 73.45358551471317) bank144156 +144157 POINT(40.66357990621342 73.83346852513836) bank144157 +144158 POINT(39.871081577233525 73.20639528152829) bank144158 +144159 POINT(39.94967236293067 73.52176823682201) bank144159 +144160 POINT(41.492588079602136 73.57664223742631) bank144160 +144161 POINT(40.832795507380624 74.04023676559501) bank144161 +144162 POINT(41.70485472427985 73.9249608565931) bank144162 +144163 POINT(41.35143576228644 73.60858083392775) bank144163 +144164 POINT(40.45980597146368 73.95880190087708) bank144164 +144165 POINT(40.02114240251304 73.14958983798363) bank144165 +144166 POINT(41.15138763967472 73.69765886876779) bank144166 +144167 POINT(40.80482418070947 73.55883847371332) bank144167 +144168 POINT(41.22726770224574 74.95998557147375) bank144168 +144169 POINT(41.35267046930417 74.49534419715768) bank144169 +144170 POINT(39.77033178455483 73.07829284973968) bank144170 +144171 POINT(40.32019091155311 73.14153431996857) bank144171 +144172 POINT(40.06014461785929 74.157386068066) bank144172 +144173 POINT(39.982452165770084 73.31993941863482) bank144173 +144174 POINT(40.983118807252694 73.64211716375237) bank144174 +144175 POINT(40.51892614774719 73.06608422770006) bank144175 +144176 POINT(40.711989055224024 74.50749108176043) bank144176 +144177 POINT(40.251527161937624 74.5224135610349) bank144177 +144178 POINT(40.33643253973433 73.9959913486716) bank144178 +144179 POINT(41.54165463933796 73.33411024425601) bank144179 +144180 POINT(40.638298323125326 74.44482813042887) bank144180 +144181 POINT(40.88460178899194 74.48829064015605) bank144181 +144182 POINT(41.32526333645468 73.39362964682445) bank144182 +144183 POINT(41.68836819632621 74.69409740193781) bank144183 +144184 POINT(40.23067882069645 74.53508135075788) bank144184 +144185 POINT(40.137104436596864 74.61717503399788) bank144185 +144186 POINT(41.699679672634126 74.55583753858163) bank144186 +144187 POINT(40.65396429854137 73.33986648728303) bank144187 +144188 POINT(40.12180765521765 74.43308346119808) bank144188 +144189 POINT(39.99510546290612 74.26707405120281) bank144189 +144190 POINT(41.23580942702876 73.63130155333607) bank144190 +144191 POINT(41.06721988788866 73.7940922997526) bank144191 +144192 POINT(40.489805661603356 74.29670608825204) bank144192 +144193 POINT(40.72609629167342 73.65126118479431) bank144193 +144194 POINT(40.3285535605536 74.77165930884873) bank144194 +144195 POINT(41.04842945368234 73.32850535203072) bank144195 +144196 POINT(41.4854610777935 73.27195073081486) bank144196 +144197 POINT(40.09223903351464 73.5491791176613) bank144197 +144198 POINT(39.89913403838006 74.09150763223788) bank144198 +144199 POINT(39.714471936869096 74.89855525321464) bank144199 +144200 POINT(40.767788361569714 73.99183093005301) bank144200 +144201 POINT(40.29089049389705 73.81514440363418) bank144201 +144202 POINT(41.28690366797558 73.99071767539267) bank144202 +144203 POINT(40.04809353761699 73.53934261858934) bank144203 +144204 POINT(40.89943626876365 74.2786800580084) bank144204 +144205 POINT(40.9535562284489 74.34631140407798) bank144205 +144206 POINT(39.795257336289865 74.66721770929303) bank144206 +144207 POINT(41.033732969214974 73.33485729202282) bank144207 +144208 POINT(40.80435940908803 74.58316803016719) bank144208 +144209 POINT(40.97617835658796 74.14628640794052) bank144209 +144210 POINT(41.23887880297877 73.54602197520725) bank144210 +144211 POINT(40.399476982868975 74.62466968365295) bank144211 +144212 POINT(40.86703594025928 74.28318510281647) bank144212 +144213 POINT(40.7463824354384 73.99052333737694) bank144213 +144214 POINT(39.9136631788463 74.15390465787897) bank144214 +144215 POINT(40.48045762650279 73.87570461302735) bank144215 +144216 POINT(40.321423856986264 73.70387529369415) bank144216 +144217 POINT(39.85378743716469 74.1553684974957) bank144217 +144218 POINT(40.89931463219476 73.6562126809288) bank144218 +144219 POINT(40.679536260677665 73.21705266530083) bank144219 +144220 POINT(40.56021543982625 74.4528575070328) bank144220 +144221 POINT(40.7415378517572 73.99533100001433) bank144221 +144222 POINT(39.91886015350079 73.85889361298172) bank144222 +144223 POINT(40.010207713382705 74.24847610650178) bank144223 +144224 POINT(41.31143005811952 74.22240066534037) bank144224 +144225 POINT(41.387789216728855 74.9763461359501) bank144225 +144226 POINT(41.25342654181115 74.99620162774299) bank144226 +144227 POINT(39.74456018644638 74.34853279922598) bank144227 +144228 POINT(40.11875209539544 73.25882949784389) bank144228 +144229 POINT(41.61231145503709 73.58723660781746) bank144229 +144230 POINT(40.94754575262773 74.77713330976206) bank144230 +144231 POINT(40.334982148832864 73.76675874434737) bank144231 +144232 POINT(41.18610169538991 74.93917602759072) bank144232 +144233 POINT(40.685225247686624 73.87745445144255) bank144233 +144234 POINT(41.23785521234922 74.95804602498785) bank144234 +144235 POINT(41.19147237831272 73.25235427657685) bank144235 +144236 POINT(40.65505676859978 74.06189836369481) bank144236 +144237 POINT(41.5753591443245 74.81696186008814) bank144237 +144238 POINT(40.43092478724324 73.42495587656849) bank144238 +144239 POINT(39.88913682569772 73.33885132997945) bank144239 +144240 POINT(41.21196922062348 74.404371162949) bank144240 +144241 POINT(41.15690716490118 73.16159615065827) bank144241 +144242 POINT(41.24836928394935 73.93343892851675) bank144242 +144243 POINT(41.66964197266766 73.57093309144528) bank144243 +144244 POINT(39.71596314584861 73.52013468867533) bank144244 +144245 POINT(41.483000351500905 74.53677153399022) bank144245 +144246 POINT(40.818487840740254 74.59046457627227) bank144246 +144247 POINT(40.44193840249519 73.72047916793919) bank144247 +144248 POINT(41.04007356312267 73.57313130516903) bank144248 +144249 POINT(40.505730086043776 74.81901395938453) bank144249 +144250 POINT(40.74987123057848 74.57751077229817) bank144250 +144251 POINT(41.52789716255199 74.00934174035491) bank144251 +144252 POINT(40.64358608938031 74.23294367498264) bank144252 +144253 POINT(40.5730799032577 73.50541144936402) bank144253 +144254 POINT(41.400052166247505 73.74891006722646) bank144254 +144255 POINT(41.60765140379016 74.6289585661778) bank144255 +144256 POINT(41.61495174879904 73.06655245230264) bank144256 +144257 POINT(40.31268019478811 74.48265774854205) bank144257 +144258 POINT(39.94893028472077 73.22205284107007) bank144258 +144259 POINT(39.72243774383488 73.22252344155062) bank144259 +144260 POINT(39.95951159636644 73.93255853182126) bank144260 +144261 POINT(40.14498331697171 73.50128363118789) bank144261 +144262 POINT(41.629013291040984 73.83140950362187) bank144262 +144263 POINT(41.616167513228405 73.33287251172173) bank144263 +144264 POINT(41.31899287583975 74.78465886126807) bank144264 +144265 POINT(41.154014771499035 74.29090651506249) bank144265 +144266 POINT(40.32880250304982 73.54114644009404) bank144266 +144267 POINT(40.26118837477769 74.66454239071304) bank144267 +144268 POINT(40.61733659385037 73.16569027325811) bank144268 +144269 POINT(41.25351580355771 73.9336436993138) bank144269 +144270 POINT(39.99569030765755 73.1444412122095) bank144270 +144271 POINT(41.080256566871036 74.88284207481902) bank144271 +144272 POINT(40.73764831159326 74.84348124306328) bank144272 +144273 POINT(41.194061925975085 73.12222439651701) bank144273 +144274 POINT(40.85520067705993 74.6634990237808) bank144274 +144275 POINT(40.59276312911124 74.07583190409498) bank144275 +144276 POINT(40.53394123106156 73.17717270017542) bank144276 +144277 POINT(40.426594661225174 73.73077863735152) bank144277 +144278 POINT(41.63399363128829 73.18887588453671) bank144278 +144279 POINT(40.49454374306514 74.22968932116262) bank144279 +144280 POINT(41.48868271868009 74.90043538905142) bank144280 +144281 POINT(39.71688014338922 73.84610896295553) bank144281 +144282 POINT(41.435864712257896 73.65031257672337) bank144282 +144283 POINT(40.60352542621813 74.13268856472861) bank144283 +144284 POINT(41.53759397736735 73.53209557902694) bank144284 +144285 POINT(40.03689926985229 74.81538556511572) bank144285 +144286 POINT(41.0504038086575 74.85806494916714) bank144286 +144287 POINT(41.65948801217681 74.73298047209683) bank144287 +144288 POINT(39.89155439237391 74.09901318721829) bank144288 +144289 POINT(41.6183875452844 73.5346774239659) bank144289 +144290 POINT(41.05063429130475 73.8045877712813) bank144290 +144291 POINT(40.06221059402843 73.65990103742162) bank144291 +144292 POINT(39.959775728251834 74.02970952295426) bank144292 +144293 POINT(40.56330998356816 73.30962576561927) bank144293 +144294 POINT(40.331690173031724 74.36026085099509) bank144294 +144295 POINT(40.69377500826023 74.3886346327824) bank144295 +144296 POINT(41.31538429253288 73.36048429747754) bank144296 +144297 POINT(40.49599968517017 73.32169463530431) bank144297 +144298 POINT(40.34170245810284 74.58409011146544) bank144298 +144299 POINT(39.934623254494426 73.13888302448466) bank144299 +144300 POINT(41.199804782365234 74.8745332684994) bank144300 +144301 POINT(40.09090441907364 74.76171098673792) bank144301 +144302 POINT(41.38680216874007 74.35997643787026) bank144302 +144303 POINT(40.27006759737735 73.18898875147255) bank144303 +144304 POINT(40.31763635280027 73.2634015152923) bank144304 +144305 POINT(40.90346465868456 74.45865353099694) bank144305 +144306 POINT(41.651397965923145 74.42969839821923) bank144306 +144307 POINT(39.77750071205578 74.78630877055757) bank144307 +144308 POINT(40.17344542765354 74.36994523448888) bank144308 +144309 POINT(40.53743266110021 74.08945820557518) bank144309 +144310 POINT(41.49003457766609 74.45722898680593) bank144310 +144311 POINT(40.5808991080201 73.69414235662329) bank144311 +144312 POINT(41.596765033910835 74.26198794135004) bank144312 +144313 POINT(39.75439035671825 73.57064972024583) bank144313 +144314 POINT(41.04260539153071 73.12895577608774) bank144314 +144315 POINT(41.481986786321 74.81519510650165) bank144315 +144316 POINT(40.23919917479917 73.9438579919289) bank144316 +144317 POINT(39.81984444377707 73.66633048288264) bank144317 +144318 POINT(40.746354679220815 73.58264209110345) bank144318 +144319 POINT(39.82895354902233 73.07180649266795) bank144319 +144320 POINT(41.381719874371576 73.52892115717553) bank144320 +144321 POINT(40.262273993468284 73.02479976907773) bank144321 +144322 POINT(41.13818855219812 73.12860626172007) bank144322 +144323 POINT(39.88439776396305 74.39226004335906) bank144323 +144324 POINT(39.78743485529683 73.49669231450822) bank144324 +144325 POINT(40.562453619794404 74.61590947994301) bank144325 +144326 POINT(41.414552007164644 74.1750718719479) bank144326 +144327 POINT(41.084335673798876 73.13634410589582) bank144327 +144328 POINT(41.03106638851043 73.32701211799659) bank144328 +144329 POINT(41.194500068065 73.77810165507155) bank144329 +144330 POINT(40.14984975628835 73.52535367861577) bank144330 +144331 POINT(40.74704453579064 74.88765781522997) bank144331 +144332 POINT(39.754118799398846 74.9996068039292) bank144332 +144333 POINT(41.23630424395238 74.23085728904815) bank144333 +144334 POINT(40.539609827179554 74.49901215389785) bank144334 +144335 POINT(39.99972368992652 74.91597092606744) bank144335 +144336 POINT(41.248505345276776 73.49461617821373) bank144336 +144337 POINT(41.46877517744267 73.63892151877994) bank144337 +144338 POINT(40.61140806663324 74.1663039748404) bank144338 +144339 POINT(41.18320338664795 73.6892811241167) bank144339 +144340 POINT(40.20493401961475 73.22156587667419) bank144340 +144341 POINT(40.723448231890046 74.34541856894616) bank144341 +144342 POINT(39.75648023008018 74.64547892099064) bank144342 +144343 POINT(41.3675876487721 73.524060369411) bank144343 +144344 POINT(40.23815404225524 74.80209075732287) bank144344 +144345 POINT(40.32604942648139 74.20048334913112) bank144345 +144346 POINT(40.290060922305365 74.60419167019299) bank144346 +144347 POINT(41.60144368212013 74.3868446908328) bank144347 +144348 POINT(39.97389137933709 73.14684807024604) bank144348 +144349 POINT(40.778842296196366 74.74099145655931) bank144349 +144350 POINT(40.68150335618274 74.06208197754025) bank144350 +144351 POINT(40.94733157534557 74.9762185301485) bank144351 +144352 POINT(40.67379126621249 73.63057949885984) bank144352 +144353 POINT(41.60158582160473 73.9449916579105) bank144353 +144354 POINT(41.53385959202712 74.93833938704633) bank144354 +144355 POINT(39.94112782899956 73.40336519787753) bank144355 +144356 POINT(40.83649545229042 73.44243249559447) bank144356 +144357 POINT(39.92993254947279 74.87101341525366) bank144357 +144358 POINT(40.94746998661451 73.57865121646284) bank144358 +144359 POINT(41.39031783872154 73.65604386270817) bank144359 +144360 POINT(41.24885828462779 73.49719042800479) bank144360 +144361 POINT(41.67423279847141 74.56638138789799) bank144361 +144362 POINT(40.4157667768378 74.33089760378694) bank144362 +144363 POINT(40.72048217143371 74.86426756687864) bank144363 +144364 POINT(41.561486815436275 73.45775489703476) bank144364 +144365 POINT(39.92846897455551 73.22730365914363) bank144365 +144366 POINT(39.71656971357763 73.9995205001906) bank144366 +144367 POINT(41.50816540413812 73.48320831012977) bank144367 +144368 POINT(40.961108106858575 74.57821611102928) bank144368 +144369 POINT(41.40442276304885 74.74933031585891) bank144369 +144370 POINT(40.40338787147403 73.56870948825986) bank144370 +144371 POINT(41.346863952615145 74.88859618362137) bank144371 +144372 POINT(41.194243751941116 74.58947004939947) bank144372 +144373 POINT(40.87907149114504 73.15301552801465) bank144373 +144374 POINT(40.24395969479404 73.44264050953883) bank144374 +144375 POINT(39.99433634891343 73.45571464513654) bank144375 +144376 POINT(41.35667794339908 74.98879230088242) bank144376 +144377 POINT(40.063407006489825 73.48470301495588) bank144377 +144378 POINT(40.01028695166733 74.92378935323215) bank144378 +144379 POINT(40.19987409651528 74.29055903588876) bank144379 +144380 POINT(40.23375556481988 73.924319941088) bank144380 +144381 POINT(40.96764337535632 74.52018639347283) bank144381 +144382 POINT(41.428824916494214 73.81186239746924) bank144382 +144383 POINT(40.61094773323896 74.1369308683348) bank144383 +144384 POINT(41.155286281376476 73.00750837516398) bank144384 +144385 POINT(41.36650950735243 74.37071151965247) bank144385 +144386 POINT(41.524923027597765 73.04453005340999) bank144386 +144387 POINT(40.39491187002349 73.46425979928125) bank144387 +144388 POINT(39.95666867139563 73.75690968739681) bank144388 +144389 POINT(40.83090902697008 74.49264730458205) bank144389 +144390 POINT(39.83501421133297 73.54341379187368) bank144390 +144391 POINT(41.554451423355296 73.87345367396608) bank144391 +144392 POINT(41.588280400325345 74.47786902681976) bank144392 +144393 POINT(41.35640942988214 74.73982829757755) bank144393 +144394 POINT(40.19374630416927 74.40034814704921) bank144394 +144395 POINT(41.36032031723515 73.8716760311565) bank144395 +144396 POINT(39.838296254928196 73.55696787788294) bank144396 +144397 POINT(41.256256983022304 73.87764404348285) bank144397 +144398 POINT(40.87016277818613 74.6286498169662) bank144398 +144399 POINT(41.576389929785925 73.82165697449537) bank144399 +144400 POINT(40.8325799851876 73.04394878977354) bank144400 +144401 POINT(40.42767835985174 74.29500484898061) bank144401 +144402 POINT(40.46551169806548 73.76279549988736) bank144402 +144403 POINT(41.388323144841664 74.12653855889677) bank144403 +144404 POINT(39.931725454145706 74.73725907540364) bank144404 +144405 POINT(40.70866073775696 73.86207314571688) bank144405 +144406 POINT(41.2099776094878 74.92490857768382) bank144406 +144407 POINT(40.57191907680803 73.94439293645142) bank144407 +144408 POINT(39.916061509544015 73.79462571992737) bank144408 +144409 POINT(41.40606861477094 74.8053619295731) bank144409 +144410 POINT(41.46555029324371 74.4730592345367) bank144410 +144411 POINT(41.4816339987477 74.465048997949) bank144411 +144412 POINT(41.62274901871847 73.74811508347868) bank144412 +144413 POINT(40.94468929897734 73.61056842049243) bank144413 +144414 POINT(41.48972520731102 73.6285290845164) bank144414 +144415 POINT(39.85384510736884 74.63039206706182) bank144415 +144416 POINT(39.77626535629191 73.2718882203505) bank144416 +144417 POINT(40.98371492090213 73.0877565145966) bank144417 +144418 POINT(40.82572214088413 74.560746155513) bank144418 +144419 POINT(41.590250673609454 74.43738007469203) bank144419 +144420 POINT(40.6674543254452 73.82061991174739) bank144420 +144421 POINT(41.355700740527936 74.8192450298559) bank144421 +144422 POINT(41.43224291954618 74.18314919217762) bank144422 +144423 POINT(40.03913438448103 73.04296277148272) bank144423 +144424 POINT(39.77409896530348 74.81572351591774) bank144424 +144425 POINT(40.016319098849365 73.10686378250364) bank144425 +144426 POINT(40.68660128326043 74.83356252519306) bank144426 +144427 POINT(41.05365699686037 74.8247606423817) bank144427 +144428 POINT(39.99895367536647 73.35511910347233) bank144428 +144429 POINT(40.611291396427376 73.07555054917054) bank144429 +144430 POINT(40.588898245458225 73.344918038148) bank144430 +144431 POINT(40.26684575866755 73.05161979185075) bank144431 +144432 POINT(41.59514688306371 74.58881983779325) bank144432 +144433 POINT(41.062264090607904 74.75748948854647) bank144433 +144434 POINT(39.95702426561256 74.18772445906318) bank144434 +144435 POINT(40.86532708625084 74.55313864564138) bank144435 +144436 POINT(40.593363270154754 73.89736487797997) bank144436 +144437 POINT(40.54516347837238 73.69638876206007) bank144437 +144438 POINT(40.44121969797675 74.1189490758247) bank144438 +144439 POINT(41.183753861588436 74.63546641109042) bank144439 +144440 POINT(41.7026861455398 74.75800042501044) bank144440 +144441 POINT(40.14153826851247 74.84768730562429) bank144441 +144442 POINT(41.27050482516035 73.42077954712668) bank144442 +144443 POINT(40.36934295274335 74.93277749973531) bank144443 +144444 POINT(41.4524829097299 73.61294816828027) bank144444 +144445 POINT(41.17743360492824 73.04096243186181) bank144445 +144446 POINT(40.684392176390396 73.13598435219954) bank144446 +144447 POINT(41.45794611594901 74.97266672204539) bank144447 +144448 POINT(40.39603411269679 74.88341651282296) bank144448 +144449 POINT(41.05230652113951 74.59334493359037) bank144449 +144450 POINT(41.28014442994771 73.19925922342904) bank144450 +144451 POINT(40.344233304121296 74.09438829988925) bank144451 +144452 POINT(40.645069181573405 73.74156917142663) bank144452 +144453 POINT(41.31561399805589 73.40961482132381) bank144453 +144454 POINT(39.98524314349672 73.70634092176645) bank144454 +144455 POINT(41.37401952851916 74.4201140480252) bank144455 +144456 POINT(40.110434983693125 73.4247540663532) bank144456 +144457 POINT(40.09235021218484 73.96308172668397) bank144457 +144458 POINT(41.11279682005909 74.89915985109404) bank144458 +144459 POINT(40.04614564169672 74.8326796287364) bank144459 +144460 POINT(41.12149362647557 74.2889058110978) bank144460 +144461 POINT(39.79427763908147 73.36396152621954) bank144461 +144462 POINT(40.90996400613154 73.2843824680768) bank144462 +144463 POINT(41.048616711104955 74.67372850495322) bank144463 +144464 POINT(39.85264046861734 73.82393569259492) bank144464 +144465 POINT(40.38219940946574 73.54103939955698) bank144465 +144466 POINT(41.560604723284285 74.55904323676813) bank144466 +144467 POINT(40.202778579372335 73.00763225022656) bank144467 +144468 POINT(40.50297976010183 74.18837686491678) bank144468 +144469 POINT(40.94800605744034 73.30332799405106) bank144469 +144470 POINT(40.75172719690115 73.66669271255441) bank144470 +144471 POINT(41.70463547359153 74.69442076209641) bank144471 +144472 POINT(40.33844154865813 74.33036334240029) bank144472 +144473 POINT(40.28651666182203 73.35068954011153) bank144473 +144474 POINT(40.539283433979016 73.27166739342988) bank144474 +144475 POINT(41.45434885999491 74.13323137210907) bank144475 +144476 POINT(41.095464572393695 74.10243140717924) bank144476 +144477 POINT(41.708815357371094 73.11603184554188) bank144477 +144478 POINT(40.04961623015955 73.21655800488588) bank144478 +144479 POINT(40.7685815952339 73.86344806332777) bank144479 +144480 POINT(40.15295340979961 73.0432675028279) bank144480 +144481 POINT(41.270353534252735 73.08920474165151) bank144481 +144482 POINT(41.60756170531047 73.23386074706974) bank144482 +144483 POINT(40.30270668604507 73.51601262791215) bank144483 +144484 POINT(41.432711337732265 74.66806326575154) bank144484 +144485 POINT(40.80834133703704 74.92854204421162) bank144485 +144486 POINT(41.09135922317 74.42772226359209) bank144486 +144487 POINT(40.72908799718991 73.93002150831069) bank144487 +144488 POINT(40.55425908478065 74.92997424496696) bank144488 +144489 POINT(41.428540923644015 74.58758154545639) bank144489 +144490 POINT(40.66559750610115 74.749100949076) bank144490 +144491 POINT(39.9334444719833 73.36606641757066) bank144491 +144492 POINT(40.95668687322651 73.45814442687237) bank144492 +144493 POINT(40.11590292310348 73.67993956577615) bank144493 +144494 POINT(40.5913135107253 74.54352077874742) bank144494 +144495 POINT(40.59919852297907 73.54368042310597) bank144495 +144496 POINT(40.194202568209846 73.22669368079325) bank144496 +144497 POINT(40.3828465400693 73.49030075097461) bank144497 +144498 POINT(41.156389418602 73.95734411450206) bank144498 +144499 POINT(41.0464133460941 74.2392741371423) bank144499 +144500 POINT(41.118899005781756 74.70525341160445) bank144500 +144501 POINT(40.368508035252745 74.816739346227) bank144501 +144502 POINT(41.483860752042844 73.60854551165941) bank144502 +144503 POINT(40.75306472415272 73.02859137241566) bank144503 +144504 POINT(40.45443837861109 73.7373826107968) bank144504 +144505 POINT(40.00864649903039 74.37103242233019) bank144505 +144506 POINT(40.34566161444944 73.81627398859325) bank144506 +144507 POINT(40.90209269434265 73.68587400622741) bank144507 +144508 POINT(40.90592953808592 73.4769957637906) bank144508 +144509 POINT(39.93291083544897 73.10596415313468) bank144509 +144510 POINT(41.18421329960069 73.97659498702839) bank144510 +144511 POINT(41.407257273138214 73.9639284049557) bank144511 +144512 POINT(41.28616592520011 73.72123581955776) bank144512 +144513 POINT(40.58761476531145 74.00925739954982) bank144513 +144514 POINT(40.573130628346036 74.87866935125044) bank144514 +144515 POINT(39.983103162557924 73.33920550248017) bank144515 +144516 POINT(39.97878853273839 74.83440163707807) bank144516 +144517 POINT(40.57782822794271 73.83250111661462) bank144517 +144518 POINT(40.77477402035111 74.04987483895752) bank144518 +144519 POINT(40.5922458555302 74.71210358611425) bank144519 +144520 POINT(41.58690461530944 73.88763847051739) bank144520 +144521 POINT(40.848680385736124 74.16577257365303) bank144521 +144522 POINT(40.840542282559525 73.43991221029471) bank144522 +144523 POINT(41.38447139163955 74.15702579044941) bank144523 +144524 POINT(41.098790510365994 73.52889190521606) bank144524 +144525 POINT(41.104507748134694 73.64255565386884) bank144525 +144526 POINT(39.947765704403324 74.45867924765435) bank144526 +144527 POINT(41.295580260872036 74.1300235303681) bank144527 +144528 POINT(39.906615341031454 73.38101443909274) bank144528 +144529 POINT(41.1460636217229 74.99902293242162) bank144529 +144530 POINT(41.525788965870944 74.7595958887604) bank144530 +144531 POINT(40.34215947605907 74.15999144156999) bank144531 +144532 POINT(41.45910839378527 74.36585335787701) bank144532 +144533 POINT(40.22877605785899 74.75740862245843) bank144533 +144534 POINT(40.864592684275834 74.85294199273599) bank144534 +144535 POINT(40.1948056420463 73.4261470464551) bank144535 +144536 POINT(41.59467687129301 74.92757571327179) bank144536 +144537 POINT(41.021723243042835 74.97416351235692) bank144537 +144538 POINT(40.31662238549361 73.03593847579462) bank144538 +144539 POINT(41.21893362814444 73.12490579800712) bank144539 +144540 POINT(40.16492150403314 74.69007995347084) bank144540 +144541 POINT(41.625758706487865 73.51993548938599) bank144541 +144542 POINT(40.726650613782915 74.48345552605119) bank144542 +144543 POINT(41.46605296331138 73.98365781196523) bank144543 +144544 POINT(41.243244427579114 73.98086067351376) bank144544 +144545 POINT(40.501281828544954 74.64271748630007) bank144545 +144546 POINT(40.864858216877444 73.10491513126385) bank144546 +144547 POINT(40.379558187170886 74.11715099674238) bank144547 +144548 POINT(41.37886932048492 73.6224007960221) bank144548 +144549 POINT(41.4271120935714 73.5728776264441) bank144549 +144550 POINT(41.14347971198419 73.4040701163691) bank144550 +144551 POINT(41.61771119518518 74.08735482961477) bank144551 +144552 POINT(41.493919195427274 73.87488102686875) bank144552 +144553 POINT(41.67890204633086 74.89668515190668) bank144553 +144554 POINT(41.29977751043201 73.04375027174005) bank144554 +144555 POINT(39.75417330259462 73.47255482044129) bank144555 +144556 POINT(40.65829209088583 73.3420577446202) bank144556 +144557 POINT(40.05040135988594 74.04696909531945) bank144557 +144558 POINT(39.72161561163801 74.89887002718802) bank144558 +144559 POINT(40.61395098904892 73.82664032737678) bank144559 +144560 POINT(40.49931366240152 73.13820702926463) bank144560 +144561 POINT(40.73689654759313 73.20921866790303) bank144561 +144562 POINT(40.70172424471908 74.32242682437253) bank144562 +144563 POINT(40.429980657679025 74.54689425848173) bank144563 +144564 POINT(41.1035262177285 73.33295750626046) bank144564 +144565 POINT(41.492720208948164 73.47284753756975) bank144565 +144566 POINT(41.06074297489018 74.62685816390989) bank144566 +144567 POINT(39.751258374572345 74.30720810647794) bank144567 +144568 POINT(40.596501247416874 74.02989342611698) bank144568 +144569 POINT(41.67696258494178 74.9705279337368) bank144569 +144570 POINT(40.650203415403055 73.35791275335419) bank144570 +144571 POINT(40.944453580819015 73.40689180041738) bank144571 +144572 POINT(41.24431724279357 73.00708817054957) bank144572 +144573 POINT(40.88518264375986 74.84078860196463) bank144573 +144574 POINT(39.98625994653539 73.19423718318312) bank144574 +144575 POINT(41.66510841300022 74.27364910648086) bank144575 +144576 POINT(40.97241872921256 74.81627198168819) bank144576 +144577 POINT(41.65803922260169 74.97928654746038) bank144577 +144578 POINT(41.079822594945156 73.99597720467101) bank144578 +144579 POINT(41.07220707266097 74.70774057529393) bank144579 +144580 POINT(40.99233295716391 74.41447693947615) bank144580 +144581 POINT(40.270864723779795 73.07844069069846) bank144581 +144582 POINT(41.33612894606807 74.32170655339526) bank144582 +144583 POINT(40.9035798927482 73.02145572406869) bank144583 +144584 POINT(40.307164188073116 73.75540993532191) bank144584 +144585 POINT(40.890615282045964 74.13196843093182) bank144585 +144586 POINT(41.46428778816595 73.10756073213805) bank144586 +144587 POINT(40.08885557797679 74.90288234054673) bank144587 +144588 POINT(41.13573965983127 74.70137336038042) bank144588 +144589 POINT(40.41182882979754 73.08658101886469) bank144589 +144590 POINT(41.65968782346391 73.40212642389793) bank144590 +144591 POINT(40.404807784515455 73.93719823318314) bank144591 +144592 POINT(40.39530039985546 73.33961400401965) bank144592 +144593 POINT(40.30905136384273 73.62389359329819) bank144593 +144594 POINT(40.50410672840046 73.15578928817068) bank144594 +144595 POINT(41.42969771987117 73.3008550787046) bank144595 +144596 POINT(40.38556003587231 74.11049088431498) bank144596 +144597 POINT(39.74300114875158 73.39681061215131) bank144597 +144598 POINT(41.09085721076844 74.49744525283059) bank144598 +144599 POINT(40.95023239913123 74.52613973618597) bank144599 +144600 POINT(40.90066975162094 73.03184441736174) bank144600 +144601 POINT(40.028816696497344 73.54304551422096) bank144601 +144602 POINT(41.538058977002514 73.24338237976401) bank144602 +144603 POINT(41.1213342650052 74.68481036909654) bank144603 +144604 POINT(40.00209698090591 73.62140105951354) bank144604 +144605 POINT(40.439789123142205 73.44300893967498) bank144605 +144606 POINT(41.665357285148836 74.92830825613473) bank144606 +144607 POINT(40.2732531468569 74.98101458518836) bank144607 +144608 POINT(41.65549129344814 73.26223300413317) bank144608 +144609 POINT(40.214975693127904 74.9057982680749) bank144609 +144610 POINT(39.88114476081835 73.46273797741176) bank144610 +144611 POINT(40.746107876476415 74.24798431019444) bank144611 +144612 POINT(40.83279749678875 73.47815391476836) bank144612 +144613 POINT(41.4368807615223 74.24663571485252) bank144613 +144614 POINT(40.135821857205336 73.88142844215517) bank144614 +144615 POINT(40.95878991190066 74.9757749910766) bank144615 +144616 POINT(40.80414665853132 73.6744562235998) bank144616 +144617 POINT(41.64701271717616 73.66352158745416) bank144617 +144618 POINT(41.44234189042446 74.53594374617947) bank144618 +144619 POINT(40.73326847846989 74.09898703418138) bank144619 +144620 POINT(41.6008241804308 73.81812239771598) bank144620 +144621 POINT(40.66389358338287 74.23265931685654) bank144621 +144622 POINT(41.077814919306356 74.98729641295847) bank144622 +144623 POINT(40.99378620633795 74.87924156512247) bank144623 +144624 POINT(41.31965038163944 74.05460213996646) bank144624 +144625 POINT(39.80455766321338 73.9018780932299) bank144625 +144626 POINT(40.29390871997518 73.24848838978096) bank144626 +144627 POINT(40.829225286555385 74.26194059794094) bank144627 +144628 POINT(40.87266128808327 74.89624668965858) bank144628 +144629 POINT(40.564562671656105 73.49232117001918) bank144629 +144630 POINT(40.70034579955619 74.80425899804827) bank144630 +144631 POINT(40.542927310558866 74.9473150560783) bank144631 +144632 POINT(41.5382988436019 74.17673313768947) bank144632 +144633 POINT(40.18479806675718 73.92927117990669) bank144633 +144634 POINT(39.87865207157722 74.58693876083478) bank144634 +144635 POINT(40.43252367787957 73.49063730561154) bank144635 +144636 POINT(41.30655215062408 74.85833481641576) bank144636 +144637 POINT(41.54014546079945 74.66244606586558) bank144637 +144638 POINT(40.218874180414645 74.0202252863496) bank144638 +144639 POINT(39.90257951005269 74.34405872332003) bank144639 +144640 POINT(41.13546275779586 73.89247380766818) bank144640 +144641 POINT(39.976544700880275 73.03455329187629) bank144641 +144642 POINT(41.24030497644131 73.43637736011401) bank144642 +144643 POINT(39.95546925436253 74.38730031458768) bank144643 +144644 POINT(40.83068083617612 73.36907024326047) bank144644 +144645 POINT(40.00064537795423 73.3242150824542) bank144645 +144646 POINT(40.484423438426795 73.6997299211569) bank144646 +144647 POINT(40.15524062296605 74.09942313947116) bank144647 +144648 POINT(41.482465283367574 74.70041248094283) bank144648 +144649 POINT(40.24238124182752 74.45662987502887) bank144649 +144650 POINT(41.549836701759546 74.05955872267567) bank144650 +144651 POINT(40.850270782268204 73.14205754849688) bank144651 +144652 POINT(40.809642823895146 73.72761554904643) bank144652 +144653 POINT(41.64227832917122 74.74871883878959) bank144653 +144654 POINT(41.244247417751254 73.25442793301548) bank144654 +144655 POINT(41.04143530056449 75.00370197146468) bank144655 +144656 POINT(40.76810917666185 73.76036658809424) bank144656 +144657 POINT(41.45531366660684 74.97459628276566) bank144657 +144658 POINT(40.070443898453576 74.94126602817363) bank144658 +144659 POINT(40.11146615508748 74.02767179717117) bank144659 +144660 POINT(41.33520471241675 74.11836196479464) bank144660 +144661 POINT(40.14064200245655 73.88360643264896) bank144661 +144662 POINT(39.72533058615685 73.39625744857118) bank144662 +144663 POINT(41.02782825073324 74.53995678311439) bank144663 +144664 POINT(39.84838120189647 73.26883650253018) bank144664 +144665 POINT(41.072914412548336 73.37000123113707) bank144665 +144666 POINT(40.56954416779197 73.39953632927758) bank144666 +144667 POINT(41.580392508991615 74.98987324746382) bank144667 +144668 POINT(39.996166715527075 73.99327749345784) bank144668 +144669 POINT(41.45704910453565 74.21833793346816) bank144669 +144670 POINT(41.60691608797919 73.80882482565062) bank144670 +144671 POINT(41.24496784996834 73.37001315688455) bank144671 +144672 POINT(40.47995772598935 74.09248161706078) bank144672 +144673 POINT(40.95902165911633 73.87474473837229) bank144673 +144674 POINT(41.216668231703125 73.01850830642864) bank144674 +144675 POINT(39.74406465346913 74.26267905784609) bank144675 +144676 POINT(41.09602220112706 73.8165096366885) bank144676 +144677 POINT(39.851105725550944 74.26422253870324) bank144677 +144678 POINT(40.20437895066241 73.87860344834183) bank144678 +144679 POINT(41.63988041664167 74.62073330103213) bank144679 +144680 POINT(41.286691465717226 74.4822732721088) bank144680 +144681 POINT(40.70840810611417 74.52220940148793) bank144681 +144682 POINT(40.57310479870891 74.43351894265253) bank144682 +144683 POINT(40.49205498924158 74.23431232194137) bank144683 +144684 POINT(40.45409880564968 74.74642588148389) bank144684 +144685 POINT(40.73128865875701 74.28187480692839) bank144685 +144686 POINT(41.657158241983325 74.61628057350725) bank144686 +144687 POINT(41.601071248263445 74.96190879623103) bank144687 +144688 POINT(40.709248736467174 74.3080788430892) bank144688 +144689 POINT(39.78324061536739 73.70779355184358) bank144689 +144690 POINT(40.337960646488945 74.77413696719096) bank144690 +144691 POINT(40.36400146537337 74.57028952919418) bank144691 +144692 POINT(41.315570472568936 73.91535577843271) bank144692 +144693 POINT(41.441263880828515 74.47666033445451) bank144693 +144694 POINT(40.113138677622324 73.31866169545336) bank144694 +144695 POINT(41.13637638226045 74.68966703911671) bank144695 +144696 POINT(41.42212168096925 74.6910844356191) bank144696 +144697 POINT(41.34478677563378 74.22006295575888) bank144697 +144698 POINT(40.42019847138955 74.52598714636935) bank144698 +144699 POINT(41.20275140216241 74.1463150925603) bank144699 +144700 POINT(39.80951230325714 73.82292584308202) bank144700 +144701 POINT(41.480205430214866 73.48422428476216) bank144701 +144702 POINT(39.812636458494715 74.74344699359952) bank144702 +144703 POINT(40.62652107481392 73.06973606211248) bank144703 +144704 POINT(40.74918337440656 73.08009744274192) bank144704 +144705 POINT(40.62673476164167 73.10355907971866) bank144705 +144706 POINT(40.4985014308704 73.06294612909691) bank144706 +144707 POINT(40.010284019932996 74.92036483730206) bank144707 +144708 POINT(40.53663258704809 74.05859523207965) bank144708 +144709 POINT(40.08350078317744 73.13479410034228) bank144709 +144710 POINT(40.840221519261284 73.7207036159163) bank144710 +144711 POINT(39.8195204368694 73.30226428198269) bank144711 +144712 POINT(41.68942420527637 74.34017164690638) bank144712 +144713 POINT(40.004281711985584 74.14283720060108) bank144713 +144714 POINT(41.15426819991693 73.57009795896022) bank144714 +144715 POINT(41.35337577739491 73.16086200320537) bank144715 +144716 POINT(41.15158947267468 74.16023739969218) bank144716 +144717 POINT(39.726038793466394 74.82855477129982) bank144717 +144718 POINT(40.76596808911967 74.86414527001054) bank144718 +144719 POINT(40.878658701513785 74.19741259933467) bank144719 +144720 POINT(40.80213487441657 73.66298762429855) bank144720 +144721 POINT(40.840226469580216 73.21069501137808) bank144721 +144722 POINT(41.19273119427099 74.398249686165) bank144722 +144723 POINT(40.61293006963526 74.92063167130374) bank144723 +144724 POINT(41.280830958535745 73.8968533850076) bank144724 +144725 POINT(40.44777608116484 74.26549920511904) bank144725 +144726 POINT(40.23311914856004 73.02157733989374) bank144726 +144727 POINT(41.635182732198686 73.22242925818796) bank144727 +144728 POINT(41.4232271782719 73.96025851376491) bank144728 +144729 POINT(40.6846725372133 73.55297656393012) bank144729 +144730 POINT(39.77886303650626 73.9841941866665) bank144730 +144731 POINT(41.15468029163402 73.79118936899182) bank144731 +144732 POINT(41.41739388562216 73.4781371535882) bank144732 +144733 POINT(40.57490839696295 74.32689081735403) bank144733 +144734 POINT(40.12840444571426 74.51737112002509) bank144734 +144735 POINT(41.17112538891748 74.09439066186113) bank144735 +144736 POINT(40.439720599977036 74.34752894394711) bank144736 +144737 POINT(40.36694788941218 74.07918841962659) bank144737 +144738 POINT(40.028863025852985 74.70954692002472) bank144738 +144739 POINT(40.800784782295075 73.44641133169887) bank144739 +144740 POINT(40.85072758314562 73.11807771861423) bank144740 +144741 POINT(41.00700913634112 74.62387190810206) bank144741 +144742 POINT(41.19283149539518 73.49572846149418) bank144742 +144743 POINT(41.39640955960885 73.36497160814201) bank144743 +144744 POINT(41.089756978454005 74.10653157087403) bank144744 +144745 POINT(41.19735701633649 74.19226712376546) bank144745 +144746 POINT(41.45539843205371 74.73108316787676) bank144746 +144747 POINT(41.561961680714354 73.29319844773127) bank144747 +144748 POINT(39.828419469810065 74.5642728589232) bank144748 +144749 POINT(41.70435032634971 73.90702703969784) bank144749 +144750 POINT(40.9944786462427 73.12000828600982) bank144750 +144751 POINT(41.45563386705873 73.44333821297724) bank144751 +144752 POINT(40.53983169236007 74.13304315162097) bank144752 +144753 POINT(41.02353909613898 73.90612385074292) bank144753 +144754 POINT(39.970123889900236 74.01124635053266) bank144754 +144755 POINT(41.31252035272811 74.17237151773074) bank144755 +144756 POINT(41.484167696691806 74.70775553885943) bank144756 +144757 POINT(41.668649949394975 74.15325208766784) bank144757 +144758 POINT(40.91484860204108 73.74947515013639) bank144758 +144759 POINT(39.74173740593382 73.12782714590548) bank144759 +144760 POINT(40.52456386999976 74.66916812484784) bank144760 +144761 POINT(41.077694767212805 74.82044747459157) bank144761 +144762 POINT(39.75451037370034 74.77694577822925) bank144762 +144763 POINT(41.3009871478451 73.05717923998557) bank144763 +144764 POINT(41.23107431497719 74.65833262900676) bank144764 +144765 POINT(40.023873294976994 74.76028276709219) bank144765 +144766 POINT(41.42043224628814 74.80648407766193) bank144766 +144767 POINT(41.29036107069395 73.82541663592421) bank144767 +144768 POINT(40.228778232791065 73.1834911431859) bank144768 +144769 POINT(40.23983076497599 74.43103465571672) bank144769 +144770 POINT(40.19365324958246 73.37588906879964) bank144770 +144771 POINT(41.47504849897602 74.18786925146948) bank144771 +144772 POINT(40.350945173707565 73.02570892678041) bank144772 +144773 POINT(41.708148359775414 74.55868285848919) bank144773 +144774 POINT(40.080928142559046 73.20326688152997) bank144774 +144775 POINT(40.85782135608024 73.86955184895601) bank144775 +144776 POINT(40.22498995792362 74.05616235258573) bank144776 +144777 POINT(41.10508760529683 73.36927550488936) bank144777 +144778 POINT(40.9070783873143 73.52153932773717) bank144778 +144779 POINT(41.11234289563505 73.41726682091776) bank144779 +144780 POINT(40.955193579688164 73.06219756859578) bank144780 +144781 POINT(40.49739539381731 74.82012762395597) bank144781 +144782 POINT(41.682091794722055 73.13987711592513) bank144782 +144783 POINT(41.27576996285787 73.28297028077425) bank144783 +144784 POINT(41.6689349396743 74.34432418741352) bank144784 +144785 POINT(40.893216616949736 73.80610698858973) bank144785 +144786 POINT(40.38733703799629 74.00426991684286) bank144786 +144787 POINT(40.23864106080956 74.0530984290124) bank144787 +144788 POINT(41.16511018641033 73.56877448399662) bank144788 +144789 POINT(41.53358276418889 74.13857872711455) bank144789 +144790 POINT(40.37945305170444 74.04882383068761) bank144790 +144791 POINT(40.2547284160433 74.42607032870168) bank144791 +144792 POINT(41.22940159069512 74.21187734155045) bank144792 +144793 POINT(40.35926085868789 74.36386293638766) bank144793 +144794 POINT(40.469746342278256 74.93311321303814) bank144794 +144795 POINT(39.97134621161853 74.85030725868279) bank144795 +144796 POINT(39.82909527171389 73.16289233920314) bank144796 +144797 POINT(40.61279559987413 74.99897010609033) bank144797 +144798 POINT(41.678336058821195 73.49162097160571) bank144798 +144799 POINT(41.59067903830472 74.1887338198842) bank144799 +144800 POINT(41.30460650646402 74.77313871679144) bank144800 +144801 POINT(40.77782313522696 73.68600750792969) bank144801 +144802 POINT(41.27586918946978 74.01630250656723) bank144802 +144803 POINT(41.517443306344106 73.8535472528854) bank144803 +144804 POINT(40.415562084578674 73.86808858125731) bank144804 +144805 POINT(40.87303297995819 73.87007188692232) bank144805 +144806 POINT(41.68908672378876 73.98269401089676) bank144806 +144807 POINT(40.54066760648748 73.45972533093614) bank144807 +144808 POINT(39.71470549766223 73.89730247899688) bank144808 +144809 POINT(40.64314868030638 74.03942188075732) bank144809 +144810 POINT(41.644790021855634 73.19010977059133) bank144810 +144811 POINT(40.52645778047051 74.38594332551382) bank144811 +144812 POINT(40.38763865342764 74.45304910421542) bank144812 +144813 POINT(41.45769135780156 73.63061318147906) bank144813 +144814 POINT(40.090015298947215 74.75248495736912) bank144814 +144815 POINT(41.31319851587228 74.02824875347466) bank144815 +144816 POINT(40.24980169224937 74.60147136761384) bank144816 +144817 POINT(39.7966399576039 74.9713738008941) bank144817 +144818 POINT(39.78743333136341 73.00664703359232) bank144818 +144819 POINT(41.11231155245839 74.42867333571535) bank144819 +144820 POINT(39.89816928136989 73.48556273772103) bank144820 +144821 POINT(40.33789585186839 73.10585664803243) bank144821 +144822 POINT(40.94115163573452 74.10031535567138) bank144822 +144823 POINT(41.57078462939375 74.07498073030139) bank144823 +144824 POINT(40.3606679337497 73.2914220686594) bank144824 +144825 POINT(40.07177068890943 73.97643418546488) bank144825 +144826 POINT(40.668662341229314 73.73748720535492) bank144826 +144827 POINT(41.482880047172884 73.88124579259818) bank144827 +144828 POINT(41.10913767840104 73.8231926807347) bank144828 +144829 POINT(40.76948642998497 73.11764886558058) bank144829 +144830 POINT(40.14716695326074 73.58412187885833) bank144830 +144831 POINT(40.799829821977035 73.99761937405741) bank144831 +144832 POINT(40.65751377516687 74.95060903970716) bank144832 +144833 POINT(40.057736032219154 74.19054448232562) bank144833 +144834 POINT(40.22472277970181 73.84557338857479) bank144834 +144835 POINT(41.56320801443631 74.3240693162859) bank144835 +144836 POINT(39.770067262372336 73.51330523175059) bank144836 +144837 POINT(40.15650169540816 73.04704153748774) bank144837 +144838 POINT(41.119730510787434 73.90836152993553) bank144838 +144839 POINT(40.85029102171572 73.2676947961483) bank144839 +144840 POINT(39.768803615084984 73.47236548443496) bank144840 +144841 POINT(40.423117922656175 73.54725885085314) bank144841 +144842 POINT(41.37733436577769 74.14343632043656) bank144842 +144843 POINT(39.87882844397856 73.74509291556564) bank144843 +144844 POINT(40.025469698856256 74.05955638402592) bank144844 +144845 POINT(41.439652017873726 73.22436554800268) bank144845 +144846 POINT(39.88192312401812 73.94493618969214) bank144846 +144847 POINT(40.209123009133094 73.04553743888312) bank144847 +144848 POINT(40.90995133732848 74.89864687733342) bank144848 +144849 POINT(39.85138717967712 74.98567862381874) bank144849 +144850 POINT(40.98574692572243 74.0243670883848) bank144850 +144851 POINT(40.09968211988124 73.95271090273184) bank144851 +144852 POINT(40.966536078485504 74.50700675287601) bank144852 +144853 POINT(40.37626848748068 73.86205974520222) bank144853 +144854 POINT(40.47997593269221 74.52537408927222) bank144854 +144855 POINT(40.38069038125055 73.89963504264294) bank144855 +144856 POINT(41.36543583065037 73.27026973718891) bank144856 +144857 POINT(41.290492643514845 73.62214225090729) bank144857 +144858 POINT(39.82949989146099 73.465413589191) bank144858 +144859 POINT(39.802838345908604 74.81734890776626) bank144859 +144860 POINT(39.74179595765407 73.62593598477793) bank144860 +144861 POINT(41.51286734560748 74.70997221657628) bank144861 +144862 POINT(40.39160065156481 73.7511018891946) bank144862 +144863 POINT(40.910122458448775 74.43142129015287) bank144863 +144864 POINT(40.58202693219767 73.06980004145169) bank144864 +144865 POINT(40.176629146430145 74.39687126514218) bank144865 +144866 POINT(39.75688304477927 73.92118688772221) bank144866 +144867 POINT(40.109415795421306 73.98009350784895) bank144867 +144868 POINT(40.70930335283081 73.08079602376368) bank144868 +144869 POINT(40.16962684929846 74.52142712027096) bank144869 +144870 POINT(40.69363435247969 74.37448986087419) bank144870 +144871 POINT(40.739595852002225 74.18566986324632) bank144871 +144872 POINT(40.79529054850506 74.72581747102113) bank144872 +144873 POINT(41.02381830330988 74.36872634710889) bank144873 +144874 POINT(40.60917383777082 73.32492479568002) bank144874 +144875 POINT(40.058288885671395 73.33475825100858) bank144875 +144876 POINT(40.768333334867556 74.90158344954597) bank144876 +144877 POINT(40.24593187479555 73.73720561676039) bank144877 +144878 POINT(41.0961667654593 73.13743671938705) bank144878 +144879 POINT(41.01833428401032 73.36585102410513) bank144879 +144880 POINT(41.55681621643371 74.46488498774164) bank144880 +144881 POINT(41.55737062167774 73.4245530040069) bank144881 +144882 POINT(40.118219204224296 74.79474042889429) bank144882 +144883 POINT(39.90643824025929 73.13178381524013) bank144883 +144884 POINT(41.417430255875985 73.3724179873715) bank144884 +144885 POINT(41.20248286544672 74.21243473190715) bank144885 +144886 POINT(39.92922666742031 74.76130421291926) bank144886 +144887 POINT(41.44984805407115 73.4777040629156) bank144887 +144888 POINT(41.21139057412455 74.60831779171194) bank144888 +144889 POINT(40.35765729324524 73.1294847267995) bank144889 +144890 POINT(41.59266956345756 73.3406637539886) bank144890 +144891 POINT(40.962437162259384 73.92464017168905) bank144891 +144892 POINT(40.27993180366255 74.70508170832647) bank144892 +144893 POINT(41.47866139963987 74.40210634317214) bank144893 +144894 POINT(39.86132571918855 73.19294580044475) bank144894 +144895 POINT(40.065933028632365 74.53072968886661) bank144895 +144896 POINT(39.888652518458166 74.42282730786951) bank144896 +144897 POINT(41.11287265164405 73.67631648751576) bank144897 +144898 POINT(40.44467963960666 74.80702128758986) bank144898 +144899 POINT(41.62643843647137 73.94702626200346) bank144899 +144900 POINT(40.34317739816666 74.52886789890326) bank144900 +144901 POINT(39.82890966806367 74.27656665387958) bank144901 +144902 POINT(40.29758311786487 73.81360086828266) bank144902 +144903 POINT(40.073734751159975 74.92951585902485) bank144903 +144904 POINT(39.792583542388364 74.37161581988423) bank144904 +144905 POINT(40.61834709348521 73.62843267306485) bank144905 +144906 POINT(40.587917026727304 74.90787877467947) bank144906 +144907 POINT(40.076733882170146 73.66948281098672) bank144907 +144908 POINT(40.05129793804457 74.99464156310354) bank144908 +144909 POINT(40.91503565077199 74.65573243453842) bank144909 +144910 POINT(41.36840207427533 74.34336956220048) bank144910 +144911 POINT(41.20067810478798 73.96086716677306) bank144911 +144912 POINT(41.2397117581834 74.63230685994486) bank144912 +144913 POINT(41.38390901218917 74.21104293271225) bank144913 +144914 POINT(40.90533234344306 74.38837050223441) bank144914 +144915 POINT(41.48472940214934 74.19041174311897) bank144915 +144916 POINT(40.8949767772979 74.27252259974789) bank144916 +144917 POINT(39.7300960527586 73.05676603293223) bank144917 +144918 POINT(40.58499723145869 74.56660606502267) bank144918 +144919 POINT(40.640084218707734 74.62659595244621) bank144919 +144920 POINT(40.32109429737169 74.79628751932205) bank144920 +144921 POINT(40.63720263054767 73.2424403783982) bank144921 +144922 POINT(41.233743211306994 73.81217217879245) bank144922 +144923 POINT(41.15480825723682 73.50998342863154) bank144923 +144924 POINT(40.34018336936992 73.06975897582512) bank144924 +144925 POINT(40.25506053348952 73.04002335342932) bank144925 +144926 POINT(40.75652283878099 73.66067247199425) bank144926 +144927 POINT(41.128686507885455 73.97943764271724) bank144927 +144928 POINT(40.38327481011208 73.75251757912666) bank144928 +144929 POINT(40.428759125400354 73.82184181721756) bank144929 +144930 POINT(39.847897329805164 73.9695137536111) bank144930 +144931 POINT(41.49901532034208 74.19618051059136) bank144931 +144932 POINT(41.07490402429618 74.64422099190008) bank144932 +144933 POINT(41.42493493737394 73.65357140423785) bank144933 +144934 POINT(40.78194281437391 74.96705301208722) bank144934 +144935 POINT(40.26693404395556 74.90447536176366) bank144935 +144936 POINT(41.48948596968283 74.63133767077677) bank144936 +144937 POINT(40.362708453657795 73.47164377156746) bank144937 +144938 POINT(40.80245885645483 73.66853687152908) bank144938 +144939 POINT(40.540270052409134 73.69602972179307) bank144939 +144940 POINT(41.274661081730606 73.6857989154818) bank144940 +144941 POINT(40.07134054386005 74.05130646465813) bank144941 +144942 POINT(41.542968027264976 73.7666431014548) bank144942 +144943 POINT(41.08782933286714 74.07328481306531) bank144943 +144944 POINT(41.68530080362697 74.16521180788051) bank144944 +144945 POINT(41.37393765283369 74.0087809961043) bank144945 +144946 POINT(41.08862786945645 73.79071100085959) bank144946 +144947 POINT(41.55133365897517 74.03373434076673) bank144947 +144948 POINT(40.551053322348956 74.9789825718572) bank144948 +144949 POINT(39.956865495363516 73.65720720878062) bank144949 +144950 POINT(40.719731009585 73.00950202001022) bank144950 +144951 POINT(40.9544486373583 73.8270089141071) bank144951 +144952 POINT(41.22060812439858 74.57503377522936) bank144952 +144953 POINT(40.040333452519036 73.12873362862918) bank144953 +144954 POINT(41.3352836618913 74.54203811622142) bank144954 +144955 POINT(39.771002152241934 74.45006343819672) bank144955 +144956 POINT(41.580722405654825 74.07244014560654) bank144956 +144957 POINT(40.74786192230408 73.3059263923325) bank144957 +144958 POINT(39.88816382006718 73.221742912318) bank144958 +144959 POINT(40.54927073745845 73.09013174104625) bank144959 +144960 POINT(41.517452919938634 74.33544348249072) bank144960 +144961 POINT(40.33965562632892 74.05923252127032) bank144961 +144962 POINT(41.4999722778489 74.2514951925315) bank144962 +144963 POINT(39.9743014644888 73.05526557883067) bank144963 +144964 POINT(40.936919902827945 73.50687980666075) bank144964 +144965 POINT(40.99980013715809 73.9631266136115) bank144965 +144966 POINT(40.59396457058907 74.16106967902772) bank144966 +144967 POINT(41.459207164882315 74.41085243636131) bank144967 +144968 POINT(40.3644310355451 74.37429126254823) bank144968 +144969 POINT(41.489790712835074 73.36874472559349) bank144969 +144970 POINT(40.691146327205665 73.83131660558401) bank144970 +144971 POINT(40.72391165404886 73.62674014800508) bank144971 +144972 POINT(40.907237291471326 74.21684307523144) bank144972 +144973 POINT(39.79375350000875 74.75760683756978) bank144973 +144974 POINT(40.42683425225401 73.74831176697508) bank144974 +144975 POINT(40.255014125905596 73.0885610799862) bank144975 +144976 POINT(41.55860121555913 74.4948187634854) bank144976 +144977 POINT(40.21250068634176 73.86334988217338) bank144977 +144978 POINT(41.0205223320003 74.69304058286608) bank144978 +144979 POINT(40.51603864302827 74.17561480648362) bank144979 +144980 POINT(41.41635697359398 74.45184224600226) bank144980 +144981 POINT(39.75941951067552 74.17146714693162) bank144981 +144982 POINT(41.53832188635022 74.90291083521305) bank144982 +144983 POINT(40.276958736344284 74.00837301225364) bank144983 +144984 POINT(41.06865088698957 73.76092748709378) bank144984 +144985 POINT(41.44212214563889 73.6435767710007) bank144985 +144986 POINT(41.600999876416175 73.51875676562265) bank144986 +144987 POINT(41.04456333814337 74.61105888806888) bank144987 +144988 POINT(41.10521266291253 74.33308906661675) bank144988 +144989 POINT(40.25156578119949 74.0588863253398) bank144989 +144990 POINT(39.943491449628915 73.51308800704939) bank144990 +144991 POINT(41.16945615627718 74.51641238342732) bank144991 +144992 POINT(40.018825647549114 73.83956647463447) bank144992 +144993 POINT(39.84921553269523 74.62308511607193) bank144993 +144994 POINT(40.7166229856466 74.57085085711446) bank144994 +144995 POINT(41.61104736793338 73.49513352837955) bank144995 +144996 POINT(41.11103063936559 74.1648955340342) bank144996 +144997 POINT(39.87129524204188 73.056333158025) bank144997 +144998 POINT(40.961938166730704 74.12691236141461) bank144998 +144999 POINT(40.19185865217939 73.71818010646443) bank144999 +145000 POINT(40.406558162949395 73.92336343117921) bank145000 +145001 POINT(41.17024170131331 73.42914119169195) bank145001 +145002 POINT(40.58831926765768 73.0760027825685) bank145002 +145003 POINT(40.307930335007576 73.1534182695298) bank145003 +145004 POINT(41.57344996485851 73.53444905230069) bank145004 +145005 POINT(41.026955868313344 74.38370032915599) bank145005 +145006 POINT(40.817720793262815 74.03892168726092) bank145006 +145007 POINT(40.990939664887506 74.83379936871304) bank145007 +145008 POINT(41.35079655386013 73.49404729786446) bank145008 +145009 POINT(40.726725288420184 74.73427561018859) bank145009 +145010 POINT(40.8097512124851 74.03634207359731) bank145010 +145011 POINT(40.25317780251464 74.2183819176322) bank145011 +145012 POINT(40.44364667210661 74.91060247434491) bank145012 +145013 POINT(40.96558931857839 74.33137410657783) bank145013 +145014 POINT(40.10659407931235 73.41311386692593) bank145014 +145015 POINT(41.66178520984482 74.15448421831286) bank145015 +145016 POINT(40.04609004818666 74.33148025875086) bank145016 +145017 POINT(41.36865729576537 74.89619277187779) bank145017 +145018 POINT(40.8942710223509 73.52338939876351) bank145018 +145019 POINT(40.86166558681248 74.1560753899846) bank145019 +145020 POINT(40.2940212333279 73.16590440948796) bank145020 +145021 POINT(40.053148374433555 74.4025717125552) bank145021 +145022 POINT(40.82611437121523 74.31808300915137) bank145022 +145023 POINT(40.54125277199006 74.99670891031127) bank145023 +145024 POINT(40.40690155795586 73.15823452362906) bank145024 +145025 POINT(40.23732690348837 74.49281670748394) bank145025 +145026 POINT(40.864427490522 73.63197723641689) bank145026 +145027 POINT(41.55649409587409 74.22180969214519) bank145027 +145028 POINT(39.92421804359755 74.84267495282597) bank145028 +145029 POINT(40.728724337222445 73.83321379638241) bank145029 +145030 POINT(40.65186146626715 73.73468924690614) bank145030 +145031 POINT(40.21472511512735 73.20277464758105) bank145031 +145032 POINT(40.742294330619764 73.02348890941539) bank145032 +145033 POINT(40.73519183513064 73.32468195305705) bank145033 +145034 POINT(41.09546692369862 73.75903047739354) bank145034 +145035 POINT(41.638741462569854 74.43215826152857) bank145035 +145036 POINT(40.33989771199469 74.74986682916123) bank145036 +145037 POINT(41.45446515612164 74.76588145350102) bank145037 +145038 POINT(41.35611576144015 73.52446190284833) bank145038 +145039 POINT(40.395222583092014 73.1542028245128) bank145039 +145040 POINT(41.39790298064671 73.37597725617417) bank145040 +145041 POINT(39.854860246026476 74.76443844089887) bank145041 +145042 POINT(40.83053961951646 73.772852953824) bank145042 +145043 POINT(39.80912883888644 73.1770999548617) bank145043 +145044 POINT(41.22824617526318 74.40109063757926) bank145044 +145045 POINT(41.085904196953926 73.10203846175953) bank145045 +145046 POINT(41.16051147345106 73.39707516411262) bank145046 +145047 POINT(40.16845955912477 73.7303771030934) bank145047 +145048 POINT(41.548579643738066 74.92998050965305) bank145048 +145049 POINT(41.36328815234574 74.0117085847731) bank145049 +145050 POINT(40.07282627221768 73.56977356813994) bank145050 +145051 POINT(39.97619719242248 73.176926781569) bank145051 +145052 POINT(41.332453415802505 74.99689375745596) bank145052 +145053 POINT(39.74932185740422 73.4584871587784) bank145053 +145054 POINT(41.34143523394019 74.14915488856764) bank145054 +145055 POINT(40.07342773967272 73.17526578785436) bank145055 +145056 POINT(41.57738436240185 73.51143830531595) bank145056 +145057 POINT(40.262621684749824 73.03267606828743) bank145057 +145058 POINT(40.507090779681 74.56893384225273) bank145058 +145059 POINT(40.95458420075629 74.27728130668646) bank145059 +145060 POINT(41.492692389239785 73.77160634062754) bank145060 +145061 POINT(41.2003256795406 73.88127377821809) bank145061 +145062 POINT(40.618949351172795 73.32894615134745) bank145062 +145063 POINT(40.172635525694766 74.35856976545628) bank145063 +145064 POINT(40.64331140486278 74.65686417774822) bank145064 +145065 POINT(39.72333056949659 73.34145889914173) bank145065 +145066 POINT(40.12119538517907 73.88017454408343) bank145066 +145067 POINT(40.637905331018004 73.95498648139538) bank145067 +145068 POINT(41.058616996029684 73.66076644527062) bank145068 +145069 POINT(40.8248105585409 73.6380738166687) bank145069 +145070 POINT(41.67506702778604 73.58256141618257) bank145070 +145071 POINT(40.329694511255134 73.60852947579298) bank145071 +145072 POINT(40.52050250832313 74.37168983366443) bank145072 +145073 POINT(39.83769865010387 73.38562377468223) bank145073 +145074 POINT(41.3654539067253 74.23479251580915) bank145074 +145075 POINT(40.19558646453867 74.78627875016076) bank145075 +145076 POINT(40.543310775819705 74.68114230942994) bank145076 +145077 POINT(40.073174930867374 74.2838414827905) bank145077 +145078 POINT(41.44778324287399 73.1746244407861) bank145078 +145079 POINT(40.96243754289623 74.15166289202318) bank145079 +145080 POINT(40.50356933418618 73.05927872277802) bank145080 +145081 POINT(41.5825912294489 73.02355823056219) bank145081 +145082 POINT(39.983603800153546 73.89108830798355) bank145082 +145083 POINT(41.68670060238412 74.84655907355527) bank145083 +145084 POINT(41.71194517664085 74.87358909923107) bank145084 +145085 POINT(40.06793927151701 73.64082166961843) bank145085 +145086 POINT(40.65879995552167 74.49603665426247) bank145086 +145087 POINT(41.42821885104016 74.4061573817116) bank145087 +145088 POINT(40.69840770532799 73.12535734998018) bank145088 +145089 POINT(40.96446526768109 73.74371966882923) bank145089 +145090 POINT(40.466965037621 73.68337357770125) bank145090 +145091 POINT(40.97239948516841 74.04516171415807) bank145091 +145092 POINT(40.4463306255798 73.95619315577177) bank145092 +145093 POINT(41.154991532887884 73.66565131108914) bank145093 +145094 POINT(40.74908443590321 73.82831089442189) bank145094 +145095 POINT(41.21926771899171 74.1102968205053) bank145095 +145096 POINT(41.32540179646118 74.52298442257215) bank145096 +145097 POINT(40.16701649299405 74.7269098814993) bank145097 +145098 POINT(40.630729146113055 73.8486934222483) bank145098 +145099 POINT(40.31949285473164 73.33584611630842) bank145099 +145100 POINT(40.082451295818856 73.55350100734775) bank145100 +145101 POINT(40.71169764505858 73.4759411669109) bank145101 +145102 POINT(40.81914741842406 74.88972049039953) bank145102 +145103 POINT(40.47923817278829 73.99958619447091) bank145103 +145104 POINT(40.45079381661369 74.73115089958733) bank145104 +145105 POINT(40.249971673345414 73.65625777748771) bank145105 +145106 POINT(39.91602223152672 74.18900454790747) bank145106 +145107 POINT(40.803106525766594 73.81841865311844) bank145107 +145108 POINT(40.20251335931909 74.97348597085818) bank145108 +145109 POINT(39.76034731028967 74.50402741008908) bank145109 +145110 POINT(40.6302255892832 74.43564565888853) bank145110 +145111 POINT(40.44248092813805 73.83747910189071) bank145111 +145112 POINT(41.33302822833298 73.7746816919771) bank145112 +145113 POINT(40.033261989300456 73.62750285311256) bank145113 +145114 POINT(39.85680817267517 73.75100784971103) bank145114 +145115 POINT(40.09267316202938 73.75516146579939) bank145115 +145116 POINT(39.75539951964746 74.26445835898593) bank145116 +145117 POINT(40.636093821737184 73.70220003914791) bank145117 +145118 POINT(40.52412953037438 74.62540654083232) bank145118 +145119 POINT(41.01880334598819 74.66486862612145) bank145119 +145120 POINT(41.47890807594637 73.6440804867347) bank145120 +145121 POINT(40.966882288637244 74.34044587998041) bank145121 +145122 POINT(40.113621902997245 73.13933508965697) bank145122 +145123 POINT(41.03618184791388 74.0204039810437) bank145123 +145124 POINT(41.695375233313186 73.11229224640417) bank145124 +145125 POINT(40.500579687530866 73.74006279064814) bank145125 +145126 POINT(40.66665240245413 73.84513086316016) bank145126 +145127 POINT(39.720175454538754 74.68678414295655) bank145127 +145128 POINT(41.117628918329125 73.68676226049948) bank145128 +145129 POINT(39.778488096054424 74.42157782680079) bank145129 +145130 POINT(41.31640898944556 73.33468230506537) bank145130 +145131 POINT(40.12039034377822 73.98373768601478) bank145131 +145132 POINT(40.10940862309629 74.49022502066133) bank145132 +145133 POINT(40.66412609844011 73.1329471462806) bank145133 +145134 POINT(40.02190677005374 73.56894517303215) bank145134 +145135 POINT(40.29772719550693 73.16986216602298) bank145135 +145136 POINT(40.65110436611262 74.99914619144538) bank145136 +145137 POINT(41.38395380315494 74.84781508209839) bank145137 +145138 POINT(41.34644634254395 74.42855881642345) bank145138 +145139 POINT(41.4713561901961 73.87449825088265) bank145139 +145140 POINT(40.761186843944 74.84458783107988) bank145140 +145141 POINT(41.042607128771714 73.48936739798462) bank145141 +145142 POINT(40.33738800952132 73.92932517014889) bank145142 +145143 POINT(41.10780579850987 73.98577856356658) bank145143 +145144 POINT(40.19610973034213 73.99406537511658) bank145144 +145145 POINT(40.27311755255167 74.63996641499836) bank145145 +145146 POINT(40.6813394381644 73.05688075499138) bank145146 +145147 POINT(40.22979198030297 74.96925045070132) bank145147 +145148 POINT(39.969735168474394 74.2346893871498) bank145148 +145149 POINT(40.62008981954308 73.48558895230987) bank145149 +145150 POINT(41.7024525352542 73.36085924017952) bank145150 +145151 POINT(40.89139966615417 73.41601378693741) bank145151 +145152 POINT(41.31919728581344 73.68560277589552) bank145152 +145153 POINT(40.05462390310003 74.188538793313) bank145153 +145154 POINT(40.49055910805705 74.6809400178225) bank145154 +145155 POINT(40.207383355578585 73.0796024990674) bank145155 +145156 POINT(41.110305396588004 74.02173796849574) bank145156 +145157 POINT(39.855829705060415 74.65301452435955) bank145157 +145158 POINT(40.84481158095155 74.8209740752163) bank145158 +145159 POINT(41.399831168183404 74.43934412715932) bank145159 +145160 POINT(41.66027099700578 73.33569884041388) bank145160 +145161 POINT(40.52000479080484 74.42757393549628) bank145161 +145162 POINT(40.716867026708506 74.13163717441414) bank145162 +145163 POINT(41.480817477111295 74.3619004711338) bank145163 +145164 POINT(40.01798407516994 74.98085918544122) bank145164 +145165 POINT(40.08026936405987 73.85687728766159) bank145165 +145166 POINT(41.19993644159098 73.80795567442674) bank145166 +145167 POINT(41.07364047610079 74.81960531577951) bank145167 +145168 POINT(41.68035470191568 74.56746385606372) bank145168 +145169 POINT(40.89882052515201 74.86285713735913) bank145169 +145170 POINT(40.10478126963998 74.56770581576494) bank145170 +145171 POINT(41.42812833172769 74.81101942272639) bank145171 +145172 POINT(40.37072245677969 74.07833144884664) bank145172 +145173 POINT(41.485221428751785 73.0892624885681) bank145173 +145174 POINT(41.4046437727422 74.1047957999735) bank145174 +145175 POINT(40.624792276953265 73.12414320829421) bank145175 +145176 POINT(41.19498848233231 73.87154008422871) bank145176 +145177 POINT(41.3838368469578 74.90145134650172) bank145177 +145178 POINT(40.84715219301307 73.44519530257199) bank145178 +145179 POINT(41.01311553012728 73.24529603724648) bank145179 +145180 POINT(40.90762373703513 73.11863537556579) bank145180 +145181 POINT(39.77345486272236 74.37482050555646) bank145181 +145182 POINT(41.27014734614214 73.18382325855649) bank145182 +145183 POINT(41.53777814073685 74.1931285588238) bank145183 +145184 POINT(41.03125153831977 74.84924662559602) bank145184 +145185 POINT(40.27560861382155 74.32438671997392) bank145185 +145186 POINT(41.21615989678136 73.11674938184606) bank145186 +145187 POINT(40.99634127522473 73.65369665003287) bank145187 +145188 POINT(41.515704402938056 74.91567970342425) bank145188 +145189 POINT(40.65108054552819 73.56174619879424) bank145189 +145190 POINT(40.560334832109696 73.70417320732417) bank145190 +145191 POINT(41.29765377303852 73.17261024981055) bank145191 +145192 POINT(39.839031567309355 73.52021562849407) bank145192 +145193 POINT(40.28322515267674 74.21750847242444) bank145193 +145194 POINT(41.555373435629065 73.7479705374988) bank145194 +145195 POINT(41.21659985168545 73.91247740184468) bank145195 +145196 POINT(40.98382841372219 74.43598478323841) bank145196 +145197 POINT(40.55728335340192 73.43631703578666) bank145197 +145198 POINT(39.927176638893826 74.40841026358977) bank145198 +145199 POINT(40.89823487717337 74.11081346446912) bank145199 +145200 POINT(41.28195907385867 74.33396176977251) bank145200 +145201 POINT(40.76548342469488 73.90294215678287) bank145201 +145202 POINT(40.18831190830732 73.75677692094077) bank145202 +145203 POINT(40.17628698646363 73.12904253748886) bank145203 +145204 POINT(40.71954074905125 74.33468620782301) bank145204 +145205 POINT(41.084241715123355 73.78793586948412) bank145205 +145206 POINT(41.15410527567191 74.15122467507804) bank145206 +145207 POINT(40.62538688278606 73.6626951118471) bank145207 +145208 POINT(40.01098786231372 74.97256951566202) bank145208 +145209 POINT(40.461358524754296 74.1057358345397) bank145209 +145210 POINT(40.92491694841621 73.24716638926162) bank145210 +145211 POINT(39.76679336260333 73.71822419904603) bank145211 +145212 POINT(40.313317151460446 73.30867242190855) bank145212 +145213 POINT(40.1384452670575 73.66439052767758) bank145213 +145214 POINT(40.904830517491035 73.84719354161885) bank145214 +145215 POINT(41.26698526207302 73.85613285175634) bank145215 +145216 POINT(41.65979104772419 74.27279889893667) bank145216 +145217 POINT(40.27952501739443 73.89639541313908) bank145217 +145218 POINT(39.838547654468826 74.65333837932366) bank145218 +145219 POINT(40.9608861310736 73.14375635995786) bank145219 +145220 POINT(41.65722166330991 74.05831017399956) bank145220 +145221 POINT(41.13652003743695 73.27630459723063) bank145221 +145222 POINT(41.43372335074035 73.16618556134249) bank145222 +145223 POINT(41.353403258225434 74.65850778309729) bank145223 +145224 POINT(40.725094144001595 74.37458957893362) bank145224 +145225 POINT(40.44742704977185 74.35288764584453) bank145225 +145226 POINT(40.177830140260355 74.78924855593243) bank145226 +145227 POINT(40.77738923913243 74.327094387683) bank145227 +145228 POINT(39.75771304725547 74.18935589703504) bank145228 +145229 POINT(41.01906080842731 74.01870640167374) bank145229 +145230 POINT(40.39513999400366 74.78197387626116) bank145230 +145231 POINT(40.092992065786426 74.59018074834714) bank145231 +145232 POINT(40.942289661116824 74.14827170926057) bank145232 +145233 POINT(41.084081037730506 73.67538625771722) bank145233 +145234 POINT(40.48307096934673 74.86462460091708) bank145234 +145235 POINT(41.08007819647874 74.78841741823021) bank145235 +145236 POINT(41.159261739471 73.49268065769505) bank145236 +145237 POINT(40.23113880431286 73.60788315569697) bank145237 +145238 POINT(41.168429234266775 73.80335746909316) bank145238 +145239 POINT(39.90043611506544 73.54025639066765) bank145239 +145240 POINT(40.54105684585106 73.76562406450525) bank145240 +145241 POINT(39.953178080921276 73.9848665041316) bank145241 +145242 POINT(39.91914118979932 73.72261631980567) bank145242 +145243 POINT(39.82751964432157 73.45117450934353) bank145243 +145244 POINT(39.79169627647146 74.27771476074348) bank145244 +145245 POINT(41.70043551414674 74.9822449087067) bank145245 +145246 POINT(40.78632171058564 73.32666829202962) bank145246 +145247 POINT(41.19207809694385 73.76708565659119) bank145247 +145248 POINT(41.621618449391704 73.29642683396278) bank145248 +145249 POINT(41.12033283045594 74.94713441587479) bank145249 +145250 POINT(40.40960142259235 73.2172142643175) bank145250 +145251 POINT(39.87643767209672 74.75681803342782) bank145251 +145252 POINT(40.482191454982164 73.9058271490373) bank145252 +145253 POINT(40.86689487491715 73.4034839687081) bank145253 +145254 POINT(40.92311909007795 73.68143304300425) bank145254 +145255 POINT(40.172283021104555 74.47209268118758) bank145255 +145256 POINT(40.472653289400334 74.33794247115428) bank145256 +145257 POINT(40.67378755932569 74.76452766772496) bank145257 +145258 POINT(41.54324766969513 74.71685497769248) bank145258 +145259 POINT(40.38386776787969 73.97518157622517) bank145259 +145260 POINT(41.15925118679059 73.89789128912983) bank145260 +145261 POINT(40.260225242456215 74.7498733945381) bank145261 +145262 POINT(41.03346362487781 73.59937910481578) bank145262 +145263 POINT(41.53775658719364 73.39523312837464) bank145263 +145264 POINT(41.33626577303834 74.18595934048128) bank145264 +145265 POINT(39.780341117559246 74.64863973225155) bank145265 +145266 POINT(39.889192802354806 74.89843436369486) bank145266 +145267 POINT(40.94677627526076 73.88285477922803) bank145267 +145268 POINT(40.68144660876883 74.97875108910658) bank145268 +145269 POINT(40.813127880786446 74.75799790535109) bank145269 +145270 POINT(40.778144857877955 73.33262347579021) bank145270 +145271 POINT(40.57721241105253 74.94970901890653) bank145271 +145272 POINT(41.04916088989193 73.12258353108146) bank145272 +145273 POINT(40.267041986676844 73.1663113183011) bank145273 +145274 POINT(41.54383613813735 73.29232606887112) bank145274 +145275 POINT(40.4325739108898 74.43713661317382) bank145275 +145276 POINT(40.29027374946807 74.1432281481113) bank145276 +145277 POINT(40.28283924532406 74.23110944946313) bank145277 +145278 POINT(41.493735192518976 73.74432523947772) bank145278 +145279 POINT(41.64444621948956 74.83882806991281) bank145279 +145280 POINT(40.25562988752176 73.48488743153533) bank145280 +145281 POINT(41.25992504082168 74.92652030024344) bank145281 +145282 POINT(39.884791122549835 74.33475420694235) bank145282 +145283 POINT(40.08938206312913 74.65682118388328) bank145283 +145284 POINT(40.00801150856612 73.0398043941099) bank145284 +145285 POINT(41.23488712764423 74.56983944563709) bank145285 +145286 POINT(40.98125495861309 74.89732682975122) bank145286 +145287 POINT(41.50665311620464 73.94166116515353) bank145287 +145288 POINT(39.88180699379445 73.94294048695396) bank145288 +145289 POINT(40.76155525526029 73.49760687786696) bank145289 +145290 POINT(40.03920520822408 73.96825064606226) bank145290 +145291 POINT(40.9218995959593 74.08239001238466) bank145291 +145292 POINT(40.85621634070039 74.38041584551108) bank145292 +145293 POINT(41.22044965850003 74.463604507009) bank145293 +145294 POINT(41.15377977876342 74.43888219409489) bank145294 +145295 POINT(41.10356987463809 73.0493399280012) bank145295 +145296 POINT(41.39204833240543 74.80588047919238) bank145296 +145297 POINT(41.63523202294505 73.74093354329152) bank145297 +145298 POINT(40.03630543615125 73.16069373012007) bank145298 +145299 POINT(40.04008437421994 74.43554770723748) bank145299 +145300 POINT(40.771446256351936 73.89888985367743) bank145300 +145301 POINT(41.58185300111245 73.03757458441615) bank145301 +145302 POINT(40.051143544914666 74.671756818632) bank145302 +145303 POINT(40.30910303547223 74.0223533371493) bank145303 +145304 POINT(41.693615017312 73.2690416415039) bank145304 +145305 POINT(41.669280879242265 74.02891582602636) bank145305 +145306 POINT(41.70013367480187 74.30698516092599) bank145306 +145307 POINT(41.70135069681509 74.77313496554156) bank145307 +145308 POINT(40.08805536924803 73.86390504936607) bank145308 +145309 POINT(41.36031578551506 73.33613176981979) bank145309 +145310 POINT(40.66375293671374 73.06339994580749) bank145310 +145311 POINT(41.49586573203479 73.79898196094082) bank145311 +145312 POINT(40.03914843694029 74.66335724562241) bank145312 +145313 POINT(40.72101245477075 74.62932730468874) bank145313 +145314 POINT(40.52063645140534 74.1728047039665) bank145314 +145315 POINT(40.120330558009755 74.7946672716631) bank145315 +145316 POINT(41.38437867259949 74.15982033458562) bank145316 +145317 POINT(40.03048088244254 73.39246833646035) bank145317 +145318 POINT(39.940139694087954 74.14604985608061) bank145318 +145319 POINT(40.996196120955595 73.9618705443537) bank145319 +145320 POINT(39.998942028359025 74.61909395185741) bank145320 +145321 POINT(39.904728079907144 74.92213312902695) bank145321 +145322 POINT(41.557091861373046 74.20308643578431) bank145322 +145323 POINT(40.87843897573419 74.18641851495349) bank145323 +145324 POINT(40.848349574404125 73.40526249923991) bank145324 +145325 POINT(41.03602421893556 74.11751777623493) bank145325 +145326 POINT(41.543535788072546 73.33800334333719) bank145326 +145327 POINT(40.58077586486846 74.6924529811538) bank145327 +145328 POINT(40.37888269359254 73.22947164576347) bank145328 +145329 POINT(41.49465781840129 74.93025711881643) bank145329 +145330 POINT(40.701901893385 74.17647397721912) bank145330 +145331 POINT(40.31959670294439 74.40326430983481) bank145331 +145332 POINT(39.855216252459314 74.62974836258574) bank145332 +145333 POINT(41.3699749792725 73.08732889273695) bank145333 +145334 POINT(40.066069176931414 74.35340481489847) bank145334 +145335 POINT(40.21606169862927 74.36386365691052) bank145335 +145336 POINT(39.87599219964085 74.62349112470429) bank145336 +145337 POINT(40.73580239062916 74.35375205703087) bank145337 +145338 POINT(41.439583502034964 74.20493567595355) bank145338 +145339 POINT(39.775315455234185 73.89429799580684) bank145339 +145340 POINT(40.07832047308793 73.01087281156425) bank145340 +145341 POINT(39.82924692713475 73.87706117756139) bank145341 +145342 POINT(40.86802229530613 73.29200962646867) bank145342 +145343 POINT(39.90666472180015 74.31426588802748) bank145343 +145344 POINT(40.335129564541845 73.17985355563432) bank145344 +145345 POINT(40.08263200893138 74.48753445753398) bank145345 +145346 POINT(40.562227470043275 74.27172048334401) bank145346 +145347 POINT(40.15434127930737 74.27540153347695) bank145347 +145348 POINT(39.83691727706885 74.10171128085746) bank145348 +145349 POINT(41.23223590778756 73.94209964280856) bank145349 +145350 POINT(40.998628980561136 74.11080467740827) bank145350 +145351 POINT(41.095418419536585 74.2246970099425) bank145351 +145352 POINT(40.427109795098914 73.17954513691832) bank145352 +145353 POINT(39.76928792637439 74.59242462561642) bank145353 +145354 POINT(41.01593633934429 74.007492199475) bank145354 +145355 POINT(41.46429991401151 74.6117749515213) bank145355 +145356 POINT(39.88933075326597 73.3049745774412) bank145356 +145357 POINT(41.02365919140704 74.97780332046189) bank145357 +145358 POINT(41.067580658993755 74.44519599170313) bank145358 +145359 POINT(41.08442903641128 74.66050402705497) bank145359 +145360 POINT(40.131150768570784 73.85919622406784) bank145360 +145361 POINT(41.187392581827325 73.37881373033161) bank145361 +145362 POINT(40.35644641573559 73.06146616812448) bank145362 +145363 POINT(41.65983533032116 73.87310927981184) bank145363 +145364 POINT(40.489593992630695 74.73940039186071) bank145364 +145365 POINT(40.48365619404314 73.667100086134) bank145365 +145366 POINT(41.25612899726726 74.15873842855216) bank145366 +145367 POINT(41.4835088484955 74.61536204938184) bank145367 +145368 POINT(40.31940864686387 74.05790624696384) bank145368 +145369 POINT(40.08512480181181 73.76466118895355) bank145369 +145370 POINT(40.53115918798584 74.47897640092427) bank145370 +145371 POINT(40.25588487509697 73.89900914639996) bank145371 +145372 POINT(41.27764421933457 73.11486577992108) bank145372 +145373 POINT(41.410766342183166 74.29247482783971) bank145373 +145374 POINT(40.224235191547294 73.42985009961437) bank145374 +145375 POINT(41.26533578620405 74.88901762103752) bank145375 +145376 POINT(41.38393107863958 73.42237646110479) bank145376 +145377 POINT(40.43796377575043 74.8250438536477) bank145377 +145378 POINT(41.66720248006219 74.86042062541979) bank145378 +145379 POINT(41.417362380649024 73.46173182245131) bank145379 +145380 POINT(41.52293600524076 74.99380511585257) bank145380 +145381 POINT(40.48875929388758 73.36639518046049) bank145381 +145382 POINT(40.9949403257048 73.4764096029236) bank145382 +145383 POINT(41.10445047753975 73.44323853999364) bank145383 +145384 POINT(41.50421530545872 74.80845251131058) bank145384 +145385 POINT(40.01891826364674 73.21304539750635) bank145385 +145386 POINT(40.76815389206459 74.91719942308382) bank145386 +145387 POINT(40.34912233357942 74.74335639890056) bank145387 +145388 POINT(39.9594390201266 73.78740730630828) bank145388 +145389 POINT(40.34266850423966 74.6224066810738) bank145389 +145390 POINT(41.10991741300436 73.45793011385344) bank145390 +145391 POINT(40.8180686418655 73.53766899876578) bank145391 +145392 POINT(41.02674902273138 73.94779083325857) bank145392 +145393 POINT(40.8448662532836 73.91153336522997) bank145393 +145394 POINT(40.380832447086085 74.93362919813625) bank145394 +145395 POINT(40.72944294840727 73.61070397815139) bank145395 +145396 POINT(39.71532382542159 74.04448155435074) bank145396 +145397 POINT(40.174234450668315 73.79278324019515) bank145397 +145398 POINT(41.50312063053788 73.79862594374758) bank145398 +145399 POINT(40.75491855341705 74.65088866014482) bank145399 +145400 POINT(40.96350745402685 74.42409242103933) bank145400 +145401 POINT(41.095954784559105 73.37142575327806) bank145401 +145402 POINT(40.0139671680235 74.73918398231389) bank145402 +145403 POINT(41.126765929089714 74.29182796425975) bank145403 +145404 POINT(40.460600145863886 73.78275231345123) bank145404 +145405 POINT(39.929743675906444 74.22392620501465) bank145405 +145406 POINT(41.244225907209525 73.42767435704035) bank145406 +145407 POINT(41.65778068297217 74.1674142475679) bank145407 +145408 POINT(40.915689417786055 74.22320981721151) bank145408 +145409 POINT(39.922449540810305 74.98447695615359) bank145409 +145410 POINT(40.52029127864913 74.66018322456885) bank145410 +145411 POINT(40.48813279284294 73.23044165556016) bank145411 +145412 POINT(41.16189918299955 74.6617154207644) bank145412 +145413 POINT(40.7857214755891 74.50389712024914) bank145413 +145414 POINT(40.08425425469752 74.7733266278121) bank145414 +145415 POINT(40.16138675976345 73.58227494084443) bank145415 +145416 POINT(40.85310234238762 74.05418197865316) bank145416 +145417 POINT(40.332370485670104 73.66374431090034) bank145417 +145418 POINT(41.62707075829243 74.74363767595953) bank145418 +145419 POINT(40.22016325425816 73.94891517947242) bank145419 +145420 POINT(40.14907864825194 74.95735749398011) bank145420 +145421 POINT(39.84669087984247 74.11188786594552) bank145421 +145422 POINT(41.66587212545582 74.86006746398269) bank145422 +145423 POINT(40.498275349256375 74.53745747968915) bank145423 +145424 POINT(40.84491421943998 74.2556444822265) bank145424 +145425 POINT(40.934047911290875 73.97897398426032) bank145425 +145426 POINT(41.6968093581653 73.7594291490731) bank145426 +145427 POINT(41.59628397030369 74.97232847362788) bank145427 +145428 POINT(40.010984876428154 74.1008337434793) bank145428 +145429 POINT(41.496966901986454 73.89027284443094) bank145429 +145430 POINT(40.56663590894211 74.48603159341658) bank145430 +145431 POINT(40.94334695833108 73.52818875768016) bank145431 +145432 POINT(39.8819099043191 73.0085025368051) bank145432 +145433 POINT(40.300015561829596 73.33523224533427) bank145433 +145434 POINT(41.578938287567716 74.32662790981541) bank145434 +145435 POINT(41.602600369897914 73.2664124509729) bank145435 +145436 POINT(40.619527518262586 73.76224795035938) bank145436 +145437 POINT(40.39611752025041 73.35343396657633) bank145437 +145438 POINT(40.96309286909143 74.55040456594723) bank145438 +145439 POINT(40.68981836881288 73.22094101288528) bank145439 +145440 POINT(40.219911587257066 74.55604742118143) bank145440 +145441 POINT(40.31043976681987 73.0906770607363) bank145441 +145442 POINT(41.616070620085495 74.09288414655629) bank145442 +145443 POINT(41.38730503632327 73.45050847574038) bank145443 +145444 POINT(41.55931263739325 73.44254633385876) bank145444 +145445 POINT(39.771336411611735 73.23806635934892) bank145445 +145446 POINT(39.7820651607407 73.08247303055734) bank145446 +145447 POINT(40.346204938739874 73.90336143866246) bank145447 +145448 POINT(41.03033023929767 73.2738810149736) bank145448 +145449 POINT(39.71694140466951 74.83879769684903) bank145449 +145450 POINT(40.90800453158726 74.84359054079114) bank145450 +145451 POINT(41.54561076141011 73.94315362266728) bank145451 +145452 POINT(41.011577740918064 74.29146122133069) bank145452 +145453 POINT(40.98278544322916 73.98756039876652) bank145453 +145454 POINT(41.44293744166739 73.19807317387983) bank145454 +145455 POINT(39.788259865570225 73.13815562258745) bank145455 +145456 POINT(41.27718957934241 74.9852396359317) bank145456 +145457 POINT(40.952376596405614 73.26812905980103) bank145457 +145458 POINT(41.63864686738588 74.69088114384263) bank145458 +145459 POINT(40.2856225167258 73.97984977445812) bank145459 +145460 POINT(41.29744264571805 74.97855295400774) bank145460 +145461 POINT(40.531717193085264 74.20805708291184) bank145461 +145462 POINT(40.82794629182373 73.43551448036338) bank145462 +145463 POINT(40.07190891994516 73.46304518938967) bank145463 +145464 POINT(40.07520763328947 74.32999940969945) bank145464 +145465 POINT(40.53077557897437 73.67240267031748) bank145465 +145466 POINT(41.48999315922701 74.58478534174763) bank145466 +145467 POINT(41.48505514217573 73.07969536335787) bank145467 +145468 POINT(41.67308272348759 74.34293142458814) bank145468 +145469 POINT(41.004342763630206 73.34530138411938) bank145469 +145470 POINT(40.860891640115334 73.17628523522973) bank145470 +145471 POINT(41.11133595195432 73.28078617131645) bank145471 +145472 POINT(40.776578147243136 74.15437755474159) bank145472 +145473 POINT(40.34919641972071 73.82136983733248) bank145473 +145474 POINT(41.3720054599964 73.7013467157649) bank145474 +145475 POINT(41.403389266513265 74.98099491556358) bank145475 +145476 POINT(39.86133064325664 73.13471061882535) bank145476 +145477 POINT(41.02847779287591 73.08349909447867) bank145477 +145478 POINT(40.84215956069745 74.533184681144) bank145478 +145479 POINT(41.29858725448897 74.1080017943584) bank145479 +145480 POINT(40.25611482443665 74.94628371022701) bank145480 +145481 POINT(41.70510357059012 74.70884945532462) bank145481 +145482 POINT(39.751049132482535 73.69871821097777) bank145482 +145483 POINT(41.58483467980267 74.70087595561093) bank145483 +145484 POINT(40.61768610214624 73.57541904898794) bank145484 +145485 POINT(40.03865500840452 73.84044128474015) bank145485 +145486 POINT(41.427846288874875 73.47109854590721) bank145486 +145487 POINT(40.02986256020003 73.82326010954908) bank145487 +145488 POINT(41.609935845385905 73.28908571985045) bank145488 +145489 POINT(40.39303101388675 74.16747107269461) bank145489 +145490 POINT(41.41205785379249 73.22475118630474) bank145490 +145491 POINT(40.54253082631668 73.92968382560616) bank145491 +145492 POINT(40.12331211124535 73.10986691317915) bank145492 +145493 POINT(40.64525931889074 73.04272681238353) bank145493 +145494 POINT(40.353698572856544 74.96463926504097) bank145494 +145495 POINT(40.41504257094293 73.78178131315644) bank145495 +145496 POINT(41.609243059156526 73.40585465441576) bank145496 +145497 POINT(40.46988135765989 73.34755821505381) bank145497 +145498 POINT(40.8960447989063 74.74060516775172) bank145498 +145499 POINT(40.75965752615924 73.64502365707659) bank145499 +145500 POINT(40.57159740213886 74.4666795744664) bank145500 +145501 POINT(41.28330520667036 74.12739125029479) bank145501 +145502 POINT(40.22956288253746 73.2238507431773) bank145502 +145503 POINT(41.12551794359261 73.98657817863108) bank145503 +145504 POINT(40.61365251137457 73.48639153770394) bank145504 +145505 POINT(40.50598400965423 74.7214076271998) bank145505 +145506 POINT(40.250739414450614 74.88815510663672) bank145506 +145507 POINT(40.7711410541377 74.82651816455284) bank145507 +145508 POINT(41.29860737579851 74.11462096242843) bank145508 +145509 POINT(39.90875640925813 74.11361715602588) bank145509 +145510 POINT(40.61001011786692 73.6682713814698) bank145510 +145511 POINT(40.47811744088601 73.07307765746825) bank145511 +145512 POINT(41.034815469566986 73.23729475061666) bank145512 +145513 POINT(40.49554739015484 73.73221797995319) bank145513 +145514 POINT(40.46021236393473 74.49234662308915) bank145514 +145515 POINT(41.58965530423168 73.61801790087327) bank145515 +145516 POINT(40.9948795617419 75.000967039647) bank145516 +145517 POINT(40.91648457252921 73.45837732780913) bank145517 +145518 POINT(39.92652065540322 73.0276340074792) bank145518 +145519 POINT(40.56082458084705 75.00319548940162) bank145519 +145520 POINT(41.284925257104746 74.06796199545411) bank145520 +145521 POINT(41.035957646786194 73.5521459826443) bank145521 +145522 POINT(40.576450210778546 73.17047595463174) bank145522 +145523 POINT(40.96876491712115 73.45621682247523) bank145523 +145524 POINT(41.227712206807496 74.19524300576124) bank145524 +145525 POINT(41.056540125234704 74.96204025960864) bank145525 +145526 POINT(39.90845596107463 74.52074038941231) bank145526 +145527 POINT(41.701083307770986 74.01111575437454) bank145527 +145528 POINT(40.28083734202212 73.29865129446496) bank145528 +145529 POINT(40.822673060018445 74.38762013077562) bank145529 +145530 POINT(39.89100931852702 74.22806526132273) bank145530 +145531 POINT(40.96457228082739 74.65695222293549) bank145531 +145532 POINT(40.64791207161442 74.05711623313866) bank145532 +145533 POINT(39.9694344052439 73.35003664718634) bank145533 +145534 POINT(41.32724671129298 73.69499780790024) bank145534 +145535 POINT(39.965692459935546 74.8631150691379) bank145535 +145536 POINT(40.42180245784552 74.61796338726613) bank145536 +145537 POINT(40.52393464762743 73.75059685195598) bank145537 +145538 POINT(40.38150646154099 73.41937435228239) bank145538 +145539 POINT(40.06012263388678 73.43685008187673) bank145539 +145540 POINT(40.45242567464561 74.61828926835081) bank145540 +145541 POINT(40.45878903047206 74.72551845968711) bank145541 +145542 POINT(39.71901000167819 73.94534536376328) bank145542 +145543 POINT(41.02151217249987 73.33350774258403) bank145543 +145544 POINT(39.97260985914873 73.64077881036334) bank145544 +145545 POINT(41.25064880492906 73.48067532350161) bank145545 +145546 POINT(40.52140287590112 74.9615717684238) bank145546 +145547 POINT(40.14673822457363 73.95639663814478) bank145547 +145548 POINT(41.530141872087405 73.30795365300877) bank145548 +145549 POINT(40.35220788351235 73.16014839571888) bank145549 +145550 POINT(39.95335087626195 74.08243421784289) bank145550 +145551 POINT(40.451989370276735 73.36470051894693) bank145551 +145552 POINT(41.612116997295466 73.72006056670452) bank145552 +145553 POINT(40.35368167686538 73.91128475885132) bank145553 +145554 POINT(40.67959229299454 73.97528103763551) bank145554 +145555 POINT(40.7458124778298 73.22901130735175) bank145555 +145556 POINT(41.5783690398187 74.23349418140936) bank145556 +145557 POINT(41.2578339025484 74.08988096774203) bank145557 +145558 POINT(41.45212234803916 74.02290906231102) bank145558 +145559 POINT(40.440663170824365 73.58346012562819) bank145559 +145560 POINT(39.88989763832724 73.02748830948548) bank145560 +145561 POINT(40.00876487778059 73.12527389055882) bank145561 +145562 POINT(40.098556381176245 73.38340845208799) bank145562 +145563 POINT(41.494969624759136 73.68231553190135) bank145563 +145564 POINT(41.22632894532631 74.6102337989288) bank145564 +145565 POINT(40.322754894898644 73.99051296355553) bank145565 +145566 POINT(41.68217162743536 74.43253390768302) bank145566 +145567 POINT(41.389558515049856 73.10273246318762) bank145567 +145568 POINT(40.752707540278735 74.18973049560663) bank145568 +145569 POINT(39.961943331564356 74.09205782134539) bank145569 +145570 POINT(41.42848703754102 74.17981436766837) bank145570 +145571 POINT(40.79117071558218 73.2125059672796) bank145571 +145572 POINT(40.23604014408444 73.75291745543711) bank145572 +145573 POINT(41.335119217188655 73.99083317434769) bank145573 +145574 POINT(40.16648419663418 74.91614831751299) bank145574 +145575 POINT(40.29336516015444 74.0899283216516) bank145575 +145576 POINT(39.71444483414854 73.02493206522436) bank145576 +145577 POINT(40.94236450786923 74.9962848996553) bank145577 +145578 POINT(40.74858384401179 73.25264591597121) bank145578 +145579 POINT(40.052807692787496 74.66166701112998) bank145579 +145580 POINT(40.60813461927491 74.88192987192913) bank145580 +145581 POINT(40.95595289845501 74.32359128302994) bank145581 +145582 POINT(40.361589674598704 73.4637628663072) bank145582 +145583 POINT(41.18912117579716 73.95597309189355) bank145583 +145584 POINT(41.58287913673386 73.10545820160628) bank145584 +145585 POINT(40.04341879200053 74.37312483618518) bank145585 +145586 POINT(41.345959130037414 74.81894020301203) bank145586 +145587 POINT(40.02509445557146 73.73251958774603) bank145587 +145588 POINT(40.12375619814729 73.07764233477336) bank145588 +145589 POINT(40.16239282990861 74.30364693879564) bank145589 +145590 POINT(40.45406038566029 73.15370561944778) bank145590 +145591 POINT(40.64659196749944 73.12962624376125) bank145591 +145592 POINT(41.26301823269115 74.40977199127882) bank145592 +145593 POINT(40.55716424502991 73.90846572554105) bank145593 +145594 POINT(40.40001748244076 74.37889857647194) bank145594 +145595 POINT(41.38017099972518 74.73385028549032) bank145595 +145596 POINT(40.55557394200748 74.73519149174383) bank145596 +145597 POINT(40.724573052161475 74.73048002883363) bank145597 +145598 POINT(40.745452779451355 73.88584358510694) bank145598 +145599 POINT(41.58348000560451 74.59307096022722) bank145599 +145600 POINT(40.13497804808912 74.91541123017244) bank145600 +145601 POINT(40.935554074915125 73.00728347907274) bank145601 +145602 POINT(40.73547562973839 74.45570092278831) bank145602 +145603 POINT(40.14587951982613 74.17665748300817) bank145603 +145604 POINT(39.72258828861286 74.90835329470842) bank145604 +145605 POINT(39.949880468012125 73.87653624059647) bank145605 +145606 POINT(40.087484010874 74.43100355190872) bank145606 +145607 POINT(40.935810169807446 74.9347638370995) bank145607 +145608 POINT(41.545319471199605 73.18420263801683) bank145608 +145609 POINT(40.93032641292291 74.75088187002284) bank145609 +145610 POINT(41.60944040841616 74.35701838586476) bank145610 +145611 POINT(40.04688172347704 74.34807863594409) bank145611 +145612 POINT(40.13027904280025 74.07524248486241) bank145612 +145613 POINT(40.2326945276769 74.48619744470318) bank145613 +145614 POINT(40.41848355604095 73.58721718321573) bank145614 +145615 POINT(41.28256460834013 73.83740814352218) bank145615 +145616 POINT(41.0337655002808 74.88053643663898) bank145616 +145617 POINT(40.0045616602854 74.28302845277993) bank145617 +145618 POINT(41.21933828733762 74.23495421541199) bank145618 +145619 POINT(40.41994621614159 74.61218664609129) bank145619 +145620 POINT(41.68378676857882 73.57080353147886) bank145620 +145621 POINT(40.31682425831116 74.3133442819685) bank145621 +145622 POINT(41.63938097109458 74.97608193874436) bank145622 +145623 POINT(40.69412395658159 73.24581384883584) bank145623 +145624 POINT(40.37524645819836 73.58622005196928) bank145624 +145625 POINT(40.21317734284851 73.38540673186645) bank145625 +145626 POINT(41.264888944304225 73.66064574695721) bank145626 +145627 POINT(41.154346410445235 73.74277520999887) bank145627 +145628 POINT(41.507443204845984 73.29528399343705) bank145628 +145629 POINT(41.32139127005135 74.16006541588212) bank145629 +145630 POINT(41.03986614575499 73.45732978366924) bank145630 +145631 POINT(41.168742776329786 74.05765439528076) bank145631 +145632 POINT(40.70593397413489 73.60965350051384) bank145632 +145633 POINT(40.30941419367523 74.79986330942704) bank145633 +145634 POINT(40.334754526813285 73.83886002797566) bank145634 +145635 POINT(41.30356900068795 73.3677532875128) bank145635 +145636 POINT(40.96936696141302 74.39327235607408) bank145636 +145637 POINT(40.18471595623596 74.79502751791483) bank145637 +145638 POINT(40.5360619750481 74.88462467570996) bank145638 +145639 POINT(40.82577389874338 74.4535473167742) bank145639 +145640 POINT(39.88013104211896 73.10322184327407) bank145640 +145641 POINT(40.92908244932625 74.08338460644427) bank145641 +145642 POINT(40.46738723057943 73.36795472542542) bank145642 +145643 POINT(41.567048551104904 74.11392831847152) bank145643 +145644 POINT(39.87371637275269 73.61188785443288) bank145644 +145645 POINT(40.74486805327974 74.15859118018798) bank145645 +145646 POINT(40.04207532928665 74.48458866198564) bank145646 +145647 POINT(41.03894946782665 74.07379966696345) bank145647 +145648 POINT(41.09296613709022 74.52328062629782) bank145648 +145649 POINT(40.00859329390295 74.96617209718015) bank145649 +145650 POINT(39.980754584269334 74.35487153913985) bank145650 +145651 POINT(40.410981579534585 73.52753750246644) bank145651 +145652 POINT(40.526836633535886 73.81755938137972) bank145652 +145653 POINT(41.1602661142614 74.24047275482222) bank145653 +145654 POINT(39.9868621054553 74.28469581077843) bank145654 +145655 POINT(39.75160550567844 73.4327942235684) bank145655 +145656 POINT(40.28954712075368 73.68061276528589) bank145656 +145657 POINT(41.32571768431592 74.78500045973092) bank145657 +145658 POINT(40.51121496488834 73.10436170647739) bank145658 +145659 POINT(40.46612145891118 73.38399127218123) bank145659 +145660 POINT(40.46163471616588 74.37805682225932) bank145660 +145661 POINT(40.37531747443573 73.94038315943338) bank145661 +145662 POINT(39.8172387180898 73.13303361365428) bank145662 +145663 POINT(39.77861636791853 73.66793210048154) bank145663 +145664 POINT(40.18254354613697 74.99875998624056) bank145664 +145665 POINT(40.906523324988 74.5230447684958) bank145665 +145666 POINT(40.94132598790855 74.90134880569161) bank145666 +145667 POINT(40.3977230851151 73.38075213207783) bank145667 +145668 POINT(40.90851569797134 73.39373927061415) bank145668 +145669 POINT(40.575819073350274 73.26020488117145) bank145669 +145670 POINT(41.59734940404414 74.63165984466302) bank145670 +145671 POINT(41.191162128476044 74.17594100263577) bank145671 +145672 POINT(41.534370541863275 74.22377233611999) bank145672 +145673 POINT(39.782737014087374 73.10223491299452) bank145673 +145674 POINT(40.04561004570621 74.73372989677517) bank145674 +145675 POINT(40.043794674278395 74.31217226713562) bank145675 +145676 POINT(40.35287455629966 74.55217088902702) bank145676 +145677 POINT(40.499541413633786 74.23286431288513) bank145677 +145678 POINT(40.61651717953043 74.55187629880665) bank145678 +145679 POINT(40.1848824260171 74.91421856012742) bank145679 +145680 POINT(39.91704650292059 73.23679592828674) bank145680 +145681 POINT(40.14259745779518 73.06678294347829) bank145681 +145682 POINT(39.9240963361702 73.75411652809765) bank145682 +145683 POINT(39.85247171814821 73.5366607770719) bank145683 +145684 POINT(40.15946796786741 74.95755986305609) bank145684 +145685 POINT(40.434031541824105 74.7791481213163) bank145685 +145686 POINT(41.14918421014421 74.84103554279729) bank145686 +145687 POINT(40.47369217412036 73.37001346691943) bank145687 +145688 POINT(39.90808801571039 73.99832381884531) bank145688 +145689 POINT(41.535139332862464 73.95384025158324) bank145689 +145690 POINT(41.58103561589287 74.5496495481924) bank145690 +145691 POINT(41.625250012848355 73.56912561912986) bank145691 +145692 POINT(40.83310012653877 74.50101046128736) bank145692 +145693 POINT(41.153557655229484 73.01615657589304) bank145693 +145694 POINT(41.58341044203245 74.63161255867305) bank145694 +145695 POINT(41.299571590972256 73.47388058940598) bank145695 +145696 POINT(41.25864892069165 74.52929972912675) bank145696 +145697 POINT(40.625394583591735 73.29230330790887) bank145697 +145698 POINT(40.269098332909515 74.47571304293325) bank145698 +145699 POINT(41.70450796889381 73.60750274118679) bank145699 +145700 POINT(40.15958775283871 73.15808896081376) bank145700 +145701 POINT(40.337876644731445 73.54142356124952) bank145701 +145702 POINT(40.747456322851356 74.14633492018552) bank145702 +145703 POINT(40.72502958515362 73.24873161873835) bank145703 +145704 POINT(39.89508002025923 73.50491743472891) bank145704 +145705 POINT(41.037999938017485 74.14984017150051) bank145705 +145706 POINT(41.13814400195344 73.88616899770982) bank145706 +145707 POINT(41.50115702491989 73.77925632964293) bank145707 +145708 POINT(40.92357290489364 73.208477815099) bank145708 +145709 POINT(40.44521596130966 74.79355041185221) bank145709 +145710 POINT(40.0963647435431 73.02368238393359) bank145710 +145711 POINT(40.977890729433554 73.23354276813083) bank145711 +145712 POINT(41.6350051883791 73.83158481295955) bank145712 +145713 POINT(41.634175734336125 73.3559007382245) bank145713 +145714 POINT(41.648528122847225 73.3986005290704) bank145714 +145715 POINT(40.9065080459245 73.59171234356717) bank145715 +145716 POINT(39.902757501043745 74.2481617720661) bank145716 +145717 POINT(40.973216502957854 74.82239776481482) bank145717 +145718 POINT(39.9413586695473 73.65905912373914) bank145718 +145719 POINT(40.1516623727065 74.08675124990947) bank145719 +145720 POINT(39.85270071491542 74.54566212998087) bank145720 +145721 POINT(40.06077332233906 74.60045532361549) bank145721 +145722 POINT(39.85286880427951 73.97303459040685) bank145722 +145723 POINT(41.505804469707144 74.74646628277074) bank145723 +145724 POINT(41.344262061171115 73.76656324423186) bank145724 +145725 POINT(41.6359714073972 73.53095642160457) bank145725 +145726 POINT(41.09618653742807 74.25974792958968) bank145726 +145727 POINT(41.69806139514382 74.66739677644426) bank145727 +145728 POINT(41.70724890506314 74.43639852302167) bank145728 +145729 POINT(41.062852699542304 73.3043829532881) bank145729 +145730 POINT(40.06789615857526 73.30183494549159) bank145730 +145731 POINT(39.89364766255243 74.18701015002242) bank145731 +145732 POINT(41.291482206387165 74.04176409704209) bank145732 +145733 POINT(40.75968118540366 73.98326991960077) bank145733 +145734 POINT(41.0166642243796 74.45140581509226) bank145734 +145735 POINT(39.81144215583488 73.59525052591152) bank145735 +145736 POINT(40.171818396816754 73.1024582955428) bank145736 +145737 POINT(41.57538976226236 73.91082627151312) bank145737 +145738 POINT(40.961611251488655 74.35203847101198) bank145738 +145739 POINT(40.136744035769574 73.73853100298349) bank145739 +145740 POINT(40.00188562060752 73.52308518629404) bank145740 +145741 POINT(39.89394563318793 73.87893224286633) bank145741 +145742 POINT(41.38851492621497 73.26841406985855) bank145742 +145743 POINT(40.40752057683547 73.51587875605429) bank145743 +145744 POINT(40.250447914704004 74.29510602842934) bank145744 +145745 POINT(40.005918380983765 73.0133358266004) bank145745 +145746 POINT(40.457826378220894 73.59224713634958) bank145746 +145747 POINT(40.99639525734744 74.177454711249) bank145747 +145748 POINT(39.979982630223674 73.67436038196001) bank145748 +145749 POINT(40.25781512088089 73.61709701985637) bank145749 +145750 POINT(41.00168438013435 74.22564935764778) bank145750 +145751 POINT(39.734000374141814 73.8006207256483) bank145751 +145752 POINT(40.72148035312907 74.00808684205636) bank145752 +145753 POINT(41.304864622075016 74.77996858010071) bank145753 +145754 POINT(40.83901071479001 73.73584409899797) bank145754 +145755 POINT(41.250308308505495 73.2070843055252) bank145755 +145756 POINT(40.64022811858726 73.5957122549863) bank145756 +145757 POINT(39.80320474185113 74.48849566901129) bank145757 +145758 POINT(40.985769805845386 73.67563717795296) bank145758 +145759 POINT(40.48188112344785 73.17889120586435) bank145759 +145760 POINT(40.608680715759895 73.5513734146662) bank145760 +145761 POINT(39.738544166425804 73.43715089902838) bank145761 +145762 POINT(40.72068429172067 73.27113310429567) bank145762 +145763 POINT(41.545351846770544 74.46712763878553) bank145763 +145764 POINT(41.13191993371677 73.27233851927095) bank145764 +145765 POINT(40.44359834849328 73.32601661965018) bank145765 +145766 POINT(39.83915232455894 73.5842324526494) bank145766 +145767 POINT(40.226344336964615 73.86635358359553) bank145767 +145768 POINT(41.44053711982261 74.88160126869798) bank145768 +145769 POINT(41.06831066692596 74.45762587098768) bank145769 +145770 POINT(39.91809859548641 73.09160044231777) bank145770 +145771 POINT(39.74044605871696 74.91876337688883) bank145771 +145772 POINT(39.81967792151427 73.5872797423366) bank145772 +145773 POINT(41.57762886368013 74.81207350590978) bank145773 +145774 POINT(41.36503792007723 73.18547619037987) bank145774 +145775 POINT(41.3644339616617 74.86652937420368) bank145775 +145776 POINT(41.25169278982525 74.04801786693336) bank145776 +145777 POINT(41.44924016459996 74.14574386895276) bank145777 +145778 POINT(41.23572473904999 74.13236246486316) bank145778 +145779 POINT(40.962530452714745 74.89972188663737) bank145779 +145780 POINT(41.40125268559153 73.9865332242747) bank145780 +145781 POINT(39.71858741218967 73.93531129839262) bank145781 +145782 POINT(39.80745093635975 74.41187146366646) bank145782 +145783 POINT(40.12947482734423 74.74276430372576) bank145783 +145784 POINT(40.790178939524445 74.2759095839089) bank145784 +145785 POINT(40.261583198140556 74.63155077587777) bank145785 +145786 POINT(41.49052508910398 73.47155327713749) bank145786 +145787 POINT(41.077392770771674 74.3649220252749) bank145787 +145788 POINT(40.59618031793419 74.94452983212375) bank145788 +145789 POINT(41.180587183738645 73.27964359407524) bank145789 +145790 POINT(41.508846213944175 73.53395883730698) bank145790 +145791 POINT(40.004366133615726 73.3190506012948) bank145791 +145792 POINT(40.17426952546137 73.10037819763363) bank145792 +145793 POINT(39.930200502072786 74.75852547932122) bank145793 +145794 POINT(40.80699075777775 74.06154473520007) bank145794 +145795 POINT(40.75613229815308 73.23139012996032) bank145795 +145796 POINT(41.292956412016196 74.7494188891106) bank145796 +145797 POINT(40.545215174662744 73.35971478671186) bank145797 +145798 POINT(41.16650311783657 74.91000110660077) bank145798 +145799 POINT(40.39165035867635 73.61592444820573) bank145799 +145800 POINT(41.25016616735699 74.62370487859022) bank145800 +145801 POINT(40.64973516312141 74.79639825321128) bank145801 +145802 POINT(40.58936953004672 74.25500850077705) bank145802 +145803 POINT(40.6584344741072 74.81173859380853) bank145803 +145804 POINT(40.230312281225864 74.40327120797136) bank145804 +145805 POINT(40.656941321886485 73.11754133484196) bank145805 +145806 POINT(39.75349534761475 74.87613075676161) bank145806 +145807 POINT(40.91329630097333 73.57565306223711) bank145807 +145808 POINT(39.788971658573104 74.48273040335839) bank145808 +145809 POINT(41.142905447689515 73.84522246951407) bank145809 +145810 POINT(41.353313300319506 74.66643226597954) bank145810 +145811 POINT(40.63320217027834 73.55598865556031) bank145811 +145812 POINT(39.87758913861178 74.49967032226115) bank145812 +145813 POINT(40.04938380913331 73.8871197903521) bank145813 +145814 POINT(40.29526673451167 74.85893856036009) bank145814 +145815 POINT(39.85423255419384 74.1422398033811) bank145815 +145816 POINT(40.45677369724716 73.75463742784248) bank145816 +145817 POINT(40.28880482807009 74.21624482894458) bank145817 +145818 POINT(41.294866997853205 74.08729982748996) bank145818 +145819 POINT(39.8991800514169 74.80382907520072) bank145819 +145820 POINT(40.398513188056064 74.0355013991974) bank145820 +145821 POINT(40.649233019153215 74.30550605573288) bank145821 +145822 POINT(40.37485010387986 74.64552992723411) bank145822 +145823 POINT(40.350544471650146 74.63331620728694) bank145823 +145824 POINT(40.13103365227114 74.96018383948658) bank145824 +145825 POINT(39.88626106588369 74.56401696396796) bank145825 +145826 POINT(41.61254340822335 74.37820207538621) bank145826 +145827 POINT(40.99193963329888 73.71713750533999) bank145827 +145828 POINT(40.69847879156841 73.08716458863668) bank145828 +145829 POINT(40.14128758048771 73.1693387887805) bank145829 +145830 POINT(40.12827935616404 74.19739996811565) bank145830 +145831 POINT(40.22629827123084 73.79017892654181) bank145831 +145832 POINT(40.75741911269307 74.3832509486167) bank145832 +145833 POINT(40.70557312589397 73.15828606616022) bank145833 +145834 POINT(40.24493932947577 73.90979965416496) bank145834 +145835 POINT(40.76449588293574 73.54939486065803) bank145835 +145836 POINT(41.10118909855749 74.05354986178035) bank145836 +145837 POINT(40.74538390107106 73.22806328415871) bank145837 +145838 POINT(40.26049614812612 74.42780556639791) bank145838 +145839 POINT(41.6552854024748 73.70622459789932) bank145839 +145840 POINT(39.83762469250286 74.23736934151552) bank145840 +145841 POINT(39.87369388656084 74.61650250561031) bank145841 +145842 POINT(41.01248700236084 73.14508210651925) bank145842 +145843 POINT(40.41630540512497 73.48499417471066) bank145843 +145844 POINT(40.26031029226438 73.84120681442853) bank145844 +145845 POINT(41.67667500302445 73.53357892113158) bank145845 +145846 POINT(39.730015558483316 74.82268893957058) bank145846 +145847 POINT(40.55301270856883 74.35163912400823) bank145847 +145848 POINT(40.4277888387698 73.99497638067585) bank145848 +145849 POINT(40.53448543639493 73.94107550217052) bank145849 +145850 POINT(41.43274192265033 74.66560132280179) bank145850 +145851 POINT(40.26780165827353 74.47947578201463) bank145851 +145852 POINT(41.429967852190174 73.00713474720962) bank145852 +145853 POINT(40.991089412585346 74.90680722099617) bank145853 +145854 POINT(41.21864275589967 73.65016191094183) bank145854 +145855 POINT(39.89831721723045 74.47943008186634) bank145855 +145856 POINT(39.84905367488519 74.89538898660305) bank145856 +145857 POINT(40.80952114123037 73.55800943686624) bank145857 +145858 POINT(39.79119453662021 74.8103606783468) bank145858 +145859 POINT(41.64470744203566 73.16293663487359) bank145859 +145860 POINT(40.26865914415988 73.59970077362564) bank145860 +145861 POINT(41.586272880147206 73.84690467900205) bank145861 +145862 POINT(41.61358100890728 74.67039408561466) bank145862 +145863 POINT(41.23916350481487 73.26995089999097) bank145863 +145864 POINT(40.54165229493706 73.67459201474342) bank145864 +145865 POINT(40.253048939640536 73.60536341474345) bank145865 +145866 POINT(41.4761284833619 73.7696126384181) bank145866 +145867 POINT(40.10198688644554 74.89167568570741) bank145867 +145868 POINT(41.67595074350287 73.42618230494477) bank145868 +145869 POINT(40.99211082841746 73.81228844192998) bank145869 +145870 POINT(41.39064420545503 74.61593938350704) bank145870 +145871 POINT(40.71432516521229 73.4572051294613) bank145871 +145872 POINT(40.10298197389109 74.43497495916587) bank145872 +145873 POINT(40.65808230801881 73.03092908785734) bank145873 +145874 POINT(40.56276458652157 73.42866316063827) bank145874 +145875 POINT(40.24813409301782 73.33110814589735) bank145875 +145876 POINT(39.94502380006336 73.87986079124659) bank145876 +145877 POINT(41.40894504059231 73.38469562559531) bank145877 +145878 POINT(39.77910062313182 74.31745295510133) bank145878 +145879 POINT(41.046471718731425 74.74675664750195) bank145879 +145880 POINT(39.89710667232334 73.3725678977228) bank145880 +145881 POINT(41.47539947709626 74.04101287027278) bank145881 +145882 POINT(39.81183443795976 73.44343752718281) bank145882 +145883 POINT(41.2783548114947 73.16086279510547) bank145883 +145884 POINT(41.17131813864863 74.99303957031914) bank145884 +145885 POINT(40.510097284191275 74.75382110049297) bank145885 +145886 POINT(40.88977220607926 74.4204137542501) bank145886 +145887 POINT(41.15978078067941 73.28295001757857) bank145887 +145888 POINT(40.08930299123776 73.0159817560254) bank145888 +145889 POINT(41.4788284063482 74.04795414522322) bank145889 +145890 POINT(40.12232013918255 74.81212654607269) bank145890 +145891 POINT(39.76473786958708 73.0875056853908) bank145891 +145892 POINT(40.592568436317144 73.35323511074664) bank145892 +145893 POINT(39.877709239550654 73.336867777384) bank145893 +145894 POINT(40.64394375788012 73.95488840764034) bank145894 +145895 POINT(41.14853543728588 73.30769659003973) bank145895 +145896 POINT(41.29224557615698 74.73639488294047) bank145896 +145897 POINT(39.77141128588271 73.96155544967826) bank145897 +145898 POINT(41.584251040408766 73.39218582186915) bank145898 +145899 POINT(40.4142098877448 74.43644118904655) bank145899 +145900 POINT(41.49024221018606 73.37100637273855) bank145900 +145901 POINT(41.04777354071402 73.24748970880117) bank145901 +145902 POINT(39.91624009027301 74.80709072950823) bank145902 +145903 POINT(40.060225499158385 73.24803621763134) bank145903 +145904 POINT(40.3487118650761 74.26208030746984) bank145904 +145905 POINT(40.28801538581223 74.53441825321546) bank145905 +145906 POINT(40.30500340430115 74.13734456887903) bank145906 +145907 POINT(41.382396691080686 74.72652664123898) bank145907 +145908 POINT(41.28031404397632 74.94442575957581) bank145908 +145909 POINT(41.34401806686402 74.00314009360332) bank145909 +145910 POINT(40.3607223913107 73.33114996550198) bank145910 +145911 POINT(41.61575187473657 73.21318042731698) bank145911 +145912 POINT(40.47877124372292 73.97569949948344) bank145912 +145913 POINT(40.16277502723582 74.98829845560284) bank145913 +145914 POINT(41.60783725671296 73.61947598460438) bank145914 +145915 POINT(41.49017637423829 73.15320411362637) bank145915 +145916 POINT(41.42782117612587 73.86349482385224) bank145916 +145917 POINT(40.28156217996241 73.412576056339) bank145917 +145918 POINT(40.998045090157724 74.06656482726919) bank145918 +145919 POINT(40.7533182174447 73.2373349742498) bank145919 +145920 POINT(40.632296501442895 74.49239524210597) bank145920 +145921 POINT(40.65018463505135 74.06889073621905) bank145921 +145922 POINT(41.66394209396652 74.51376593165125) bank145922 +145923 POINT(41.64979771168567 74.41652725217054) bank145923 +145924 POINT(40.43249665894343 73.72624639448205) bank145924 +145925 POINT(41.289342456711424 73.81137215914279) bank145925 +145926 POINT(40.232742083843014 73.55886977560668) bank145926 +145927 POINT(41.08967957032102 73.72063111743417) bank145927 +145928 POINT(40.84582493783203 74.77077306381199) bank145928 +145929 POINT(40.46869337954456 73.75273566842323) bank145929 +145930 POINT(40.331835404594216 73.18420729979317) bank145930 +145931 POINT(41.21475218397886 74.77202009257371) bank145931 +145932 POINT(41.20698841525141 74.2062743157174) bank145932 +145933 POINT(40.30614329420178 74.85098673565052) bank145933 +145934 POINT(40.31964639789041 74.90585430664424) bank145934 +145935 POINT(39.80372852311047 73.67355774318105) bank145935 +145936 POINT(40.723357235622494 74.09089031453102) bank145936 +145937 POINT(40.85115814988326 74.52065345097637) bank145937 +145938 POINT(40.14131333025914 74.29609917627633) bank145938 +145939 POINT(40.587519660570166 74.03689052186155) bank145939 +145940 POINT(40.060455068244394 74.0397153295198) bank145940 +145941 POINT(40.748793853279594 73.32783781298862) bank145941 +145942 POINT(40.3275867690714 73.88418020378617) bank145942 +145943 POINT(41.349735671771946 74.320797890422) bank145943 +145944 POINT(40.589890796839136 74.37833927294277) bank145944 +145945 POINT(40.2542397866203 74.09806591563301) bank145945 +145946 POINT(39.795249455272064 73.67120480630165) bank145946 +145947 POINT(40.804393394976174 74.17300075192682) bank145947 +145948 POINT(39.85205071969712 73.56397409534492) bank145948 +145949 POINT(40.04789213475191 74.84149800049568) bank145949 +145950 POINT(39.91058053482539 73.8130272807753) bank145950 +145951 POINT(39.83896013254905 74.82867190077019) bank145951 +145952 POINT(40.24460561728614 73.11957800255773) bank145952 +145953 POINT(40.40635063679344 73.46056894985759) bank145953 +145954 POINT(40.59334943675852 73.38812387537581) bank145954 +145955 POINT(40.35598468788596 74.12235149498453) bank145955 +145956 POINT(40.08170579021076 73.03131611789853) bank145956 +145957 POINT(39.982259837087746 73.42920052852533) bank145957 +145958 POINT(40.84579985755052 73.0227892567868) bank145958 +145959 POINT(40.35361637033084 73.57598448790246) bank145959 +145960 POINT(40.10526942211044 74.79237193403661) bank145960 +145961 POINT(41.65358281238652 74.88537760407624) bank145961 +145962 POINT(40.117727194628685 74.81956858040577) bank145962 +145963 POINT(41.42088685574958 74.29446760877343) bank145963 +145964 POINT(40.34231364852904 73.42849193916123) bank145964 +145965 POINT(41.40288076564425 73.0099391645021) bank145965 +145966 POINT(40.28898690184315 73.73682707829305) bank145966 +145967 POINT(39.77035133252241 74.03984585939013) bank145967 +145968 POINT(40.680063550944425 73.19348719348712) bank145968 +145969 POINT(39.83419616126036 73.83410317331375) bank145969 +145970 POINT(39.85526515268821 74.15969326989041) bank145970 +145971 POINT(40.13228583811706 73.78697516666446) bank145971 +145972 POINT(40.172504294826176 73.71976106000946) bank145972 +145973 POINT(40.86716447657846 74.64102519214643) bank145973 +145974 POINT(40.48181558132416 73.61175658761728) bank145974 +145975 POINT(41.338593072766145 73.55962837762453) bank145975 +145976 POINT(40.20501312847166 73.69744247961337) bank145976 +145977 POINT(41.55611300224185 74.22847208066153) bank145977 +145978 POINT(40.99042467506104 74.9473042599549) bank145978 +145979 POINT(40.511500251622664 74.977074498697) bank145979 +145980 POINT(40.572788847472395 73.38104538346019) bank145980 +145981 POINT(40.39781645438901 74.129961911664) bank145981 +145982 POINT(41.43325405233202 73.34503907905254) bank145982 +145983 POINT(41.66899053303375 74.56627440081002) bank145983 +145984 POINT(40.15885646465274 74.90224402168532) bank145984 +145985 POINT(41.27857795867928 73.75083009624284) bank145985 +145986 POINT(40.739319167548196 73.95370356682487) bank145986 +145987 POINT(40.349794906113274 73.80376918666317) bank145987 +145988 POINT(40.32630636961697 73.24539951304436) bank145988 +145989 POINT(39.85744602837631 73.08328872798786) bank145989 +145990 POINT(40.797501852736296 74.1677001874787) bank145990 +145991 POINT(40.600919620278106 74.41340252850668) bank145991 +145992 POINT(41.49203604185606 74.66800783726681) bank145992 +145993 POINT(40.57995285783817 74.90920120846253) bank145993 +145994 POINT(41.689569507864206 73.78397236697734) bank145994 +145995 POINT(40.96183750314156 74.56467303572633) bank145995 +145996 POINT(40.81785385334304 73.44037136698365) bank145996 +145997 POINT(41.207436889568015 74.34309410816645) bank145997 +145998 POINT(40.57053898753499 74.02321311079794) bank145998 +145999 POINT(40.33591557413944 73.7283869145276) bank145999 +146000 POINT(40.097411993206265 73.11678089380676) bank146000 +146001 POINT(40.692183670233355 74.1032587242743) bank146001 +146002 POINT(40.46788475973263 74.77915232883828) bank146002 +146003 POINT(40.46934960690631 73.53956960121333) bank146003 +146004 POINT(39.932656032251295 74.01948782980998) bank146004 +146005 POINT(40.59044717997082 74.04726659361661) bank146005 +146006 POINT(40.57017580109295 73.49949888076203) bank146006 +146007 POINT(40.897194148458695 74.7904475811347) bank146007 +146008 POINT(40.96296107438798 73.3530955679323) bank146008 +146009 POINT(41.00617939853131 74.4563162614521) bank146009 +146010 POINT(40.018061264657014 73.0872394449771) bank146010 +146011 POINT(41.34445661346261 73.32413107092867) bank146011 +146012 POINT(39.82898568325721 73.16778531522561) bank146012 +146013 POINT(39.89815475030972 73.83775427802378) bank146013 +146014 POINT(40.01458432860537 73.31036630130632) bank146014 +146015 POINT(41.61397512952131 74.79079908209762) bank146015 +146016 POINT(41.11314614687695 74.25957413211334) bank146016 +146017 POINT(40.61355828011795 73.46897257449756) bank146017 +146018 POINT(41.20921275242988 74.10130434894097) bank146018 +146019 POINT(39.77757444264769 73.79284904145794) bank146019 +146020 POINT(41.553514634377734 74.33825700373393) bank146020 +146021 POINT(41.584137503128346 74.83018743222085) bank146021 +146022 POINT(41.556523053047 73.55178922044725) bank146022 +146023 POINT(40.18854993037378 73.29008717749548) bank146023 +146024 POINT(39.9142901311156 73.4162687064633) bank146024 +146025 POINT(40.638652537672066 73.63048509192696) bank146025 +146026 POINT(40.48615773576305 74.8902206262215) bank146026 +146027 POINT(41.18400241021218 73.71234078754966) bank146027 +146028 POINT(41.05107244907407 74.02408933516664) bank146028 +146029 POINT(40.67069991450645 74.12627093125974) bank146029 +146030 POINT(40.382629423067456 73.360021842923) bank146030 +146031 POINT(41.68603107755637 74.20903205787117) bank146031 +146032 POINT(40.073061490120786 73.82944410206014) bank146032 +146033 POINT(39.871122685246064 73.52525345877301) bank146033 +146034 POINT(41.705681916477594 73.40072820821318) bank146034 +146035 POINT(39.97132503457211 73.74914263521822) bank146035 +146036 POINT(41.677360762919655 74.7742428056361) bank146036 +146037 POINT(40.196990195676086 73.8198776587371) bank146037 +146038 POINT(41.399043145119975 73.12302122662418) bank146038 +146039 POINT(40.48715404434976 74.15121093930605) bank146039 +146040 POINT(41.54297861133975 73.13441081445136) bank146040 +146041 POINT(40.12355273105412 74.83421045874776) bank146041 +146042 POINT(41.012197107942896 73.23236425525583) bank146042 +146043 POINT(39.78591420857181 73.61423397896424) bank146043 +146044 POINT(41.36059407722687 73.51966006086323) bank146044 +146045 POINT(41.05307641840101 74.08786512736451) bank146045 +146046 POINT(41.46165739846713 74.82630009365283) bank146046 +146047 POINT(40.78129435859623 74.73540254641001) bank146047 +146048 POINT(40.068001039091065 74.58498575538403) bank146048 +146049 POINT(40.89382747357862 73.05257285059949) bank146049 +146050 POINT(39.74281167662085 73.55519246882172) bank146050 +146051 POINT(40.248281446804405 73.98117974976819) bank146051 +146052 POINT(40.508984959519424 73.24109939995661) bank146052 +146053 POINT(39.8080690392131 73.20573261542334) bank146053 +146054 POINT(40.27433242289478 74.82955849991896) bank146054 +146055 POINT(41.29825655622744 73.19712953014253) bank146055 +146056 POINT(40.23772967125511 73.06258757672073) bank146056 +146057 POINT(40.682021730107174 74.6751806527767) bank146057 +146058 POINT(40.457313214260616 73.34608299283164) bank146058 +146059 POINT(40.171904447380584 73.68585300715718) bank146059 +146060 POINT(40.00812641060151 73.07099950876112) bank146060 +146061 POINT(40.31391023034094 74.17526583951131) bank146061 +146062 POINT(41.53176990461976 73.44361416615699) bank146062 +146063 POINT(40.89991921599491 73.74618864273401) bank146063 +146064 POINT(39.86427863701063 74.30873669785821) bank146064 +146065 POINT(40.08542529618002 73.37443423824466) bank146065 +146066 POINT(40.08022444835339 74.22400709482602) bank146066 +146067 POINT(41.71216519352193 73.46203017187416) bank146067 +146068 POINT(40.18219146254535 74.92953930035512) bank146068 +146069 POINT(40.48074127007722 74.68169350642042) bank146069 +146070 POINT(41.26706367109836 74.5337958916545) bank146070 +146071 POINT(40.912891751242974 74.66771954319253) bank146071 +146072 POINT(40.53894976848291 75.00401070894655) bank146072 +146073 POINT(39.796974160806144 74.33046371315575) bank146073 +146074 POINT(39.73784262638479 73.68881485166413) bank146074 +146075 POINT(41.25832996868111 73.7834521249502) bank146075 +146076 POINT(40.5021281426196 73.09505801995479) bank146076 +146077 POINT(41.52861145173058 74.33840042888896) bank146077 +146078 POINT(41.52668065823367 74.42443196162957) bank146078 +146079 POINT(40.08202512617207 74.04360667772013) bank146079 +146080 POINT(40.877266071842776 74.3356866818174) bank146080 +146081 POINT(41.661024655671646 74.6580274492482) bank146081 +146082 POINT(40.75334261489837 73.27029249104476) bank146082 +146083 POINT(39.92579024509688 74.9546843822167) bank146083 +146084 POINT(39.99142006701319 74.9262117905064) bank146084 +146085 POINT(40.1310263979046 73.97627400957111) bank146085 +146086 POINT(41.36516526171664 73.87703014643276) bank146086 +146087 POINT(39.716312653441975 74.53169388805446) bank146087 +146088 POINT(41.045296494007125 73.04755373959821) bank146088 +146089 POINT(41.39925240010083 74.14920350832375) bank146089 +146090 POINT(41.69371092096562 74.19912252607696) bank146090 +146091 POINT(39.79014966814122 74.16340104764508) bank146091 +146092 POINT(41.551741147003696 73.71429727566836) bank146092 +146093 POINT(40.36171674575601 73.85545431299768) bank146093 +146094 POINT(40.64106082500949 73.68754014544893) bank146094 +146095 POINT(40.706560921840165 73.87874305000493) bank146095 +146096 POINT(40.7689018708037 73.09600109263309) bank146096 +146097 POINT(40.378433421262635 74.92354190486721) bank146097 +146098 POINT(40.45178238496426 74.55725769205516) bank146098 +146099 POINT(40.2204148364524 73.7278159663574) bank146099 +146100 POINT(39.71594159701781 73.49918237881712) bank146100 +146101 POINT(40.748049971621846 74.82212837763151) bank146101 +146102 POINT(40.057988839429456 73.50661944154497) bank146102 +146103 POINT(41.11450707899405 74.565526827779) bank146103 +146104 POINT(41.36664572565856 73.69253323900321) bank146104 +146105 POINT(40.904492849052886 73.01651989984882) bank146105 +146106 POINT(41.668235016697864 74.73324331978742) bank146106 +146107 POINT(41.252530875794356 73.8645979389845) bank146107 +146108 POINT(41.36957446009078 73.96946987791534) bank146108 +146109 POINT(40.56242573743797 73.97215985967924) bank146109 +146110 POINT(40.99029800829835 74.77208041970795) bank146110 +146111 POINT(40.31443375919019 74.636478427678) bank146111 +146112 POINT(40.180696991981236 74.84153050788439) bank146112 +146113 POINT(39.826563859066226 74.87891775341896) bank146113 +146114 POINT(41.54042293548525 74.00863931030335) bank146114 +146115 POINT(41.15638033731814 74.57787890143449) bank146115 +146116 POINT(39.895545674893704 74.00091775772833) bank146116 +146117 POINT(41.07949985768761 73.16486048384616) bank146117 +146118 POINT(40.38689594479278 74.96840157995022) bank146118 +146119 POINT(41.34593250394755 74.40194687901491) bank146119 +146120 POINT(41.09568629311166 74.59778269264486) bank146120 +146121 POINT(40.09068493033728 73.98151395423969) bank146121 +146122 POINT(41.02548570754014 74.59129027715912) bank146122 +146123 POINT(40.29034662393703 74.12430186125319) bank146123 +146124 POINT(40.94321664754291 74.64891412412415) bank146124 +146125 POINT(41.38443150274058 74.17638300435279) bank146125 +146126 POINT(40.83062233143303 74.287084128415) bank146126 +146127 POINT(40.952801493079605 74.73930785479328) bank146127 +146128 POINT(40.84654453063627 73.67747601097128) bank146128 +146129 POINT(39.98596968245763 73.51656493727099) bank146129 +146130 POINT(41.52221359086892 73.25999911174863) bank146130 +146131 POINT(40.89587657069795 73.77146708734462) bank146131 +146132 POINT(39.83377904062563 73.59257307728785) bank146132 +146133 POINT(40.634842052886654 73.41915095191976) bank146133 +146134 POINT(40.274064339150684 73.29779304603389) bank146134 +146135 POINT(39.74335224676612 74.07445478755096) bank146135 +146136 POINT(40.49228002762586 73.74561583768846) bank146136 +146137 POINT(40.1837817457876 73.3689782538833) bank146137 +146138 POINT(40.1896598193411 74.02606095787914) bank146138 +146139 POINT(40.743759338088324 74.05598749563863) bank146139 +146140 POINT(39.95578496724554 73.48099881623163) bank146140 +146141 POINT(41.513160702188785 73.48905989378518) bank146141 +146142 POINT(41.54135619084049 73.71595298202564) bank146142 +146143 POINT(40.206645233512916 73.67867383239305) bank146143 +146144 POINT(40.986779658406206 73.89061773088689) bank146144 +146145 POINT(40.97671421218481 74.39769045821488) bank146145 +146146 POINT(41.36612276146387 73.42730287722098) bank146146 +146147 POINT(40.23618487256236 73.4424628255801) bank146147 +146148 POINT(40.99212859426241 74.30571509082522) bank146148 +146149 POINT(40.88972840724599 74.01615780208893) bank146149 +146150 POINT(41.64656000826311 74.55084598738206) bank146150 +146151 POINT(40.013499220832315 74.97494161930786) bank146151 +146152 POINT(40.029103648841634 73.06244373890142) bank146152 +146153 POINT(39.81746219277096 74.95473957750771) bank146153 +146154 POINT(41.65705650960128 74.7551276455475) bank146154 +146155 POINT(41.28173719744824 73.57483780019695) bank146155 +146156 POINT(40.755743686457755 74.89646427886838) bank146156 +146157 POINT(41.232339117095485 74.38986437762247) bank146157 +146158 POINT(39.99715008796255 74.81019013219733) bank146158 +146159 POINT(41.11873479708474 73.63130077379127) bank146159 +146160 POINT(40.65925267149652 73.35025928381879) bank146160 +146161 POINT(40.15759670406313 73.48991429441944) bank146161 +146162 POINT(40.225827613152745 74.62396414052927) bank146162 +146163 POINT(40.12208378934498 73.4575275668582) bank146163 +146164 POINT(40.580946227010145 73.13487805841437) bank146164 +146165 POINT(41.68879687897672 74.87277007063443) bank146165 +146166 POINT(40.99837376906101 74.69175777657692) bank146166 +146167 POINT(40.45206064498809 74.12895111047762) bank146167 +146168 POINT(41.0601304906117 73.18194180322901) bank146168 +146169 POINT(40.77985030693638 74.60302037557233) bank146169 +146170 POINT(40.184374992598016 73.25626226825966) bank146170 +146171 POINT(40.4942951482875 74.86050208815662) bank146171 +146172 POINT(40.927808683836076 73.10345154694176) bank146172 +146173 POINT(40.487000733047005 73.76824288559733) bank146173 +146174 POINT(40.92938959648934 74.00608068867301) bank146174 +146175 POINT(40.11781210513477 74.99825529256515) bank146175 +146176 POINT(41.60125410351739 73.62730135716758) bank146176 +146177 POINT(40.855379154805554 73.98756327042454) bank146177 +146178 POINT(40.650364864333184 74.57857606984567) bank146178 +146179 POINT(41.392572811961905 74.97039602893821) bank146179 +146180 POINT(39.73850217961635 73.36503981372033) bank146180 +146181 POINT(39.99571179642243 73.11300921166192) bank146181 +146182 POINT(39.72215114371876 73.66753290937639) bank146182 +146183 POINT(40.78976017812675 74.40473641611301) bank146183 +146184 POINT(40.78721613875298 73.42546203305668) bank146184 +146185 POINT(40.09317652186526 73.84992993827674) bank146185 +146186 POINT(41.46849823232129 74.57367172156722) bank146186 +146187 POINT(40.513586120094594 73.07773783725175) bank146187 +146188 POINT(41.16145072938277 74.40209946013668) bank146188 +146189 POINT(40.04215579111225 73.26801242119872) bank146189 +146190 POINT(40.23182679912884 73.0659609488148) bank146190 +146191 POINT(41.3350023824799 74.06163180444172) bank146191 +146192 POINT(40.042088971821265 73.0965905316715) bank146192 +146193 POINT(40.32923603360043 73.55714819573687) bank146193 +146194 POINT(39.86370002019165 73.01679707600356) bank146194 +146195 POINT(40.460250604513284 74.18794440838785) bank146195 +146196 POINT(41.648527960464506 73.8386316042712) bank146196 +146197 POINT(40.80210561154246 73.38240610262743) bank146197 +146198 POINT(41.33507118967352 74.99629407754259) bank146198 +146199 POINT(41.48679554171133 74.06037902483234) bank146199 +146200 POINT(40.642935829249616 73.1744147307812) bank146200 +146201 POINT(40.477676865491404 74.9462825952067) bank146201 +146202 POINT(40.759866914083716 74.86672869791562) bank146202 +146203 POINT(40.32851381852891 73.2590911168629) bank146203 +146204 POINT(40.438492776182144 73.31847704969326) bank146204 +146205 POINT(41.635201537066166 74.19997145165956) bank146205 +146206 POINT(40.184227497548875 73.23372036449415) bank146206 +146207 POINT(41.453163287658825 74.33920312563981) bank146207 +146208 POINT(41.33846967838185 74.11340964777958) bank146208 +146209 POINT(41.1748625229386 73.125158704074) bank146209 +146210 POINT(41.58477570296483 74.0645703653449) bank146210 +146211 POINT(41.39882538906521 74.45739486762373) bank146211 +146212 POINT(40.9337747654726 73.05524361628733) bank146212 +146213 POINT(40.79039173262636 73.50915003831891) bank146213 +146214 POINT(41.38012634282478 74.91387741917578) bank146214 +146215 POINT(41.25899466318711 74.88640032447613) bank146215 +146216 POINT(39.9993086801791 74.49506251260536) bank146216 +146217 POINT(41.11788552480863 73.1355715945454) bank146217 +146218 POINT(40.21513959919963 73.34402169090741) bank146218 +146219 POINT(41.19527855740411 73.6405527971672) bank146219 +146220 POINT(41.23099200185206 74.58710330642666) bank146220 +146221 POINT(40.591746067939575 74.22985097979274) bank146221 +146222 POINT(41.33238173051546 73.6190511339257) bank146222 +146223 POINT(40.461132571879105 74.51278351527996) bank146223 +146224 POINT(39.94634720434156 74.75371376034427) bank146224 +146225 POINT(40.62923893447942 73.21199275211482) bank146225 +146226 POINT(40.80708200779564 73.14987860670936) bank146226 +146227 POINT(41.632940172145915 73.19110573956844) bank146227 +146228 POINT(41.33247946773065 74.25521126917063) bank146228 +146229 POINT(40.37197276239053 73.25426870999698) bank146229 +146230 POINT(40.76472534233184 74.2126943800394) bank146230 +146231 POINT(41.12755621923946 73.33738410986712) bank146231 +146232 POINT(40.122385516561756 73.84886504846624) bank146232 +146233 POINT(40.902980232764584 74.22969247858) bank146233 +146234 POINT(39.84479419931665 74.34667703469763) bank146234 +146235 POINT(39.8255690385968 74.69783632215334) bank146235 +146236 POINT(41.19072887367888 74.17490463737998) bank146236 +146237 POINT(41.261750271736666 73.09846770281165) bank146237 +146238 POINT(41.474918867396944 74.94906744491668) bank146238 +146239 POINT(40.754258888974505 73.04899332576265) bank146239 +146240 POINT(40.64853653919489 74.62556124664236) bank146240 +146241 POINT(40.56304205955013 73.52253436620478) bank146241 +146242 POINT(41.46765301544281 73.04182033925431) bank146242 +146243 POINT(40.722013864262436 73.79053202967839) bank146243 +146244 POINT(40.84389677278909 73.73310488320648) bank146244 +146245 POINT(41.17759991332659 74.67033774259089) bank146245 +146246 POINT(39.801685166263674 74.54046850004205) bank146246 +146247 POINT(40.83755409544713 73.91780330380857) bank146247 +146248 POINT(40.77394949142954 75.0049995274607) bank146248 +146249 POINT(41.701354822643 73.9808074562489) bank146249 +146250 POINT(41.38671109084439 74.88079908344605) bank146250 +146251 POINT(41.2289125678639 73.45301714205416) bank146251 +146252 POINT(39.99469309287747 74.97109234481722) bank146252 +146253 POINT(41.55595034449847 74.94254082334389) bank146253 +146254 POINT(40.43598378798507 73.21731308353638) bank146254 +146255 POINT(40.0033772679921 74.4491723039849) bank146255 +146256 POINT(41.356924748974365 73.277490136134) bank146256 +146257 POINT(40.01726169109412 74.07178044770963) bank146257 +146258 POINT(41.598153375419784 73.93774996283014) bank146258 +146259 POINT(41.11855223964222 74.76599403868337) bank146259 +146260 POINT(40.76948097315271 74.09756560142353) bank146260 +146261 POINT(39.973816202648365 74.37771633240841) bank146261 +146262 POINT(40.68343502895552 74.6458554474463) bank146262 +146263 POINT(40.57137930556819 74.9863154842849) bank146263 +146264 POINT(40.553910868277725 73.93845235791271) bank146264 +146265 POINT(40.135808353762044 73.57933711204447) bank146265 +146266 POINT(41.401983813365426 73.67493545995886) bank146266 +146267 POINT(41.454749355740596 74.32850896774606) bank146267 +146268 POINT(41.617839761957995 73.71709678524189) bank146268 +146269 POINT(39.77698323462658 73.72887290324465) bank146269 +146270 POINT(41.182752006219985 73.63796234896617) bank146270 +146271 POINT(40.76289375529117 74.48556792765912) bank146271 +146272 POINT(40.61473707720975 74.83964404787349) bank146272 +146273 POINT(39.73325551762058 74.64102695814947) bank146273 +146274 POINT(40.34047661122494 74.81994398979623) bank146274 +146275 POINT(41.095451305437074 74.0356764209257) bank146275 +146276 POINT(40.07161413910753 73.86310287282579) bank146276 +146277 POINT(40.77752178265544 74.75008831340925) bank146277 +146278 POINT(41.555986232501795 73.7084888214776) bank146278 +146279 POINT(40.6464676334064 74.19558666293653) bank146279 +146280 POINT(40.3871050410253 73.18437243400777) bank146280 +146281 POINT(41.68194224480091 74.73151927715591) bank146281 +146282 POINT(40.735577782419945 74.68054448126517) bank146282 +146283 POINT(40.20680214161289 74.54755957983753) bank146283 +146284 POINT(39.91351990080493 74.45971035484082) bank146284 +146285 POINT(40.851448212746234 74.8118538499344) bank146285 +146286 POINT(40.27891003976378 74.65724724287716) bank146286 +146287 POINT(41.490818209541416 73.67088492291963) bank146287 +146288 POINT(40.22396331743064 73.4262306525313) bank146288 +146289 POINT(41.69555789440076 73.8582688075772) bank146289 +146290 POINT(40.08553907647058 74.87659678959706) bank146290 +146291 POINT(41.091156757655384 74.42807399721568) bank146291 +146292 POINT(40.56112043745111 74.94058493775835) bank146292 +146293 POINT(41.313972866825175 74.76507835772873) bank146293 +146294 POINT(40.87583423852287 73.4774799224263) bank146294 +146295 POINT(39.8199925163554 73.51109307101432) bank146295 +146296 POINT(40.62765063941172 74.88528891733773) bank146296 +146297 POINT(39.92826213410443 73.08616872286065) bank146297 +146298 POINT(40.44994907382773 74.44776678468156) bank146298 +146299 POINT(41.016636031943264 75.00333685210398) bank146299 +146300 POINT(41.24759125281898 73.06499836025351) bank146300 +146301 POINT(40.5537216992406 74.86552954465361) bank146301 +146302 POINT(39.824508629398395 73.3110741153048) bank146302 +146303 POINT(40.141143362862216 74.957584225851) bank146303 +146304 POINT(40.72084352365685 74.91031765599452) bank146304 +146305 POINT(41.610541712756 73.73915491882572) bank146305 +146306 POINT(40.88545045760467 74.87246486015523) bank146306 +146307 POINT(41.701717444320636 73.36481436447899) bank146307 +146308 POINT(41.28539148290693 74.37021394522031) bank146308 +146309 POINT(41.700487408107094 73.9963191051975) bank146309 +146310 POINT(41.67666201862371 73.52896246002045) bank146310 +146311 POINT(41.298598743472695 73.05022005831408) bank146311 +146312 POINT(39.822384116920375 74.53737387342098) bank146312 +146313 POINT(40.97637452399628 73.92101191524033) bank146313 +146314 POINT(41.113994015368746 73.15995650789124) bank146314 +146315 POINT(41.298300838783454 74.64373577090619) bank146315 +146316 POINT(41.3134512844152 74.35052496374038) bank146316 +146317 POINT(41.36448436415642 74.56404298239923) bank146317 +146318 POINT(41.14000537256098 74.72620998742725) bank146318 +146319 POINT(41.012136554090496 73.2125166873692) bank146319 +146320 POINT(40.56610394049747 74.46788923229121) bank146320 +146321 POINT(40.8654815324082 73.03435625456981) bank146321 +146322 POINT(40.28461239138879 73.82849390711803) bank146322 +146323 POINT(40.590161810134894 74.37764012410189) bank146323 +146324 POINT(40.98303782356179 74.39769494924934) bank146324 +146325 POINT(41.1930298280328 73.82135881160184) bank146325 +146326 POINT(41.091453294590025 74.38331892441798) bank146326 +146327 POINT(41.24651552606751 73.42006114725389) bank146327 +146328 POINT(40.04375371331331 74.30754878375583) bank146328 +146329 POINT(41.22790996741847 73.51839135402619) bank146329 +146330 POINT(40.6155252612745 74.20081635256456) bank146330 +146331 POINT(40.80122219819344 73.68328580529149) bank146331 +146332 POINT(40.454707566778566 74.07103037703956) bank146332 +146333 POINT(41.24825572815044 73.85826263452816) bank146333 +146334 POINT(40.614828391070354 74.14457077926693) bank146334 +146335 POINT(40.15512166840395 73.35993024225526) bank146335 +146336 POINT(41.41339747247082 74.89490680322155) bank146336 +146337 POINT(41.503750006791414 74.04649528076295) bank146337 +146338 POINT(41.486035174208574 74.76046316069004) bank146338 +146339 POINT(40.55521868395652 74.71012558456977) bank146339 +146340 POINT(40.249373499140546 73.31551750638027) bank146340 +146341 POINT(40.49083179446711 73.57891993677806) bank146341 +146342 POINT(40.00625610190916 73.70278176354279) bank146342 +146343 POINT(39.820663575296145 73.18248342868421) bank146343 +146344 POINT(40.16700674110214 74.64075063779178) bank146344 +146345 POINT(40.323544893632636 73.67671369951042) bank146345 +146346 POINT(41.67234812540297 74.98656138402897) bank146346 +146347 POINT(41.2913513360308 73.53122420896322) bank146347 +146348 POINT(40.18183492840028 74.47667094153455) bank146348 +146349 POINT(40.04266898885019 74.90303258355509) bank146349 +146350 POINT(40.841623297559316 73.97882147989311) bank146350 +146351 POINT(40.89268782704814 74.10961754659071) bank146351 +146352 POINT(39.991608405138265 74.2369559262477) bank146352 +146353 POINT(40.79591589346298 73.13371850550821) bank146353 +146354 POINT(40.01241628625834 73.37995752137797) bank146354 +146355 POINT(40.5511452739926 73.77510464779145) bank146355 +146356 POINT(40.67299455443234 74.05825102150952) bank146356 +146357 POINT(40.84565150246292 73.29055428049564) bank146357 +146358 POINT(40.589689210546815 73.80207160762328) bank146358 +146359 POINT(40.90674296352108 74.03728682327642) bank146359 +146360 POINT(41.30441789511877 73.62867891991536) bank146360 +146361 POINT(40.82745778313192 73.334593397251) bank146361 +146362 POINT(40.32659651331539 73.56717021969507) bank146362 +146363 POINT(40.379975870946886 74.96821875459051) bank146363 +146364 POINT(40.57974779410357 73.08757760657457) bank146364 +146365 POINT(40.301336176309334 74.01048499342893) bank146365 +146366 POINT(40.943779560580275 74.11454610384338) bank146366 +146367 POINT(41.52465720563821 74.05971899458154) bank146367 +146368 POINT(41.05779199240283 74.80131467263024) bank146368 +146369 POINT(40.15222314681721 73.36286795025946) bank146369 +146370 POINT(39.86314538584098 74.1980565524188) bank146370 +146371 POINT(39.725292617899676 73.71772865424488) bank146371 +146372 POINT(40.248953584268385 74.59480258782108) bank146372 +146373 POINT(40.52076757517802 74.3508288415835) bank146373 +146374 POINT(39.81747238572498 73.87784462484873) bank146374 +146375 POINT(40.91368962962238 74.25392040529111) bank146375 +146376 POINT(40.39243162113563 73.61921655282138) bank146376 +146377 POINT(41.00847891084525 74.91247323402294) bank146377 +146378 POINT(39.79814744280588 74.72090658869216) bank146378 +146379 POINT(40.472629551851746 74.40104170909093) bank146379 +146380 POINT(40.64360368258338 74.60009679593101) bank146380 +146381 POINT(40.781203975561574 74.93455174601479) bank146381 +146382 POINT(40.380634155372384 74.45882458649925) bank146382 +146383 POINT(41.28432411245831 73.23134237895034) bank146383 +146384 POINT(40.441453577626355 74.86912496141038) bank146384 +146385 POINT(41.351792074264296 74.79754108786776) bank146385 +146386 POINT(40.580663120520185 74.56055789964776) bank146386 +146387 POINT(41.30283428425429 74.82617366219438) bank146387 +146388 POINT(41.51161324471689 74.91074925242292) bank146388 +146389 POINT(40.983075777006995 74.41552871519312) bank146389 +146390 POINT(41.154271163410066 74.17332555557749) bank146390 +146391 POINT(40.38690727231522 74.83712637199531) bank146391 +146392 POINT(40.403877878079484 73.8877655143404) bank146392 +146393 POINT(40.80675743858134 73.42429786283452) bank146393 +146394 POINT(40.449978959631046 73.44743466806584) bank146394 +146395 POINT(40.25060123071642 74.09689160978844) bank146395 +146396 POINT(40.508760653361065 74.5289815037155) bank146396 +146397 POINT(41.69619531105476 73.96961880608625) bank146397 +146398 POINT(40.15543422632075 73.97379969820624) bank146398 +146399 POINT(41.45756158878425 73.4383743751895) bank146399 +146400 POINT(41.43573804539614 73.03873086724624) bank146400 +146401 POINT(41.275340587421454 74.91385180591921) bank146401 +146402 POINT(41.29960560406509 73.65326498153398) bank146402 +146403 POINT(40.33371430949855 74.59120726078972) bank146403 +146404 POINT(41.66695491638251 74.23608049631048) bank146404 +146405 POINT(40.08932760906844 74.65217064807105) bank146405 +146406 POINT(40.985644636328686 73.12754049379426) bank146406 +146407 POINT(40.92801642126389 74.00661974359907) bank146407 +146408 POINT(41.42363888421551 74.33183638704533) bank146408 +146409 POINT(41.357172448476696 74.79298119513228) bank146409 +146410 POINT(41.335525955986185 74.5236199553738) bank146410 +146411 POINT(40.302732788476234 74.04723350273416) bank146411 +146412 POINT(40.99143073292068 74.78342061687668) bank146412 +146413 POINT(40.590970176851535 73.02299589013764) bank146413 +146414 POINT(40.08455123901477 74.60633308084707) bank146414 +146415 POINT(39.85119043490724 73.67575067026914) bank146415 +146416 POINT(40.276275175868555 73.00943911792409) bank146416 +146417 POINT(40.10497746436436 73.83126464532965) bank146417 +146418 POINT(40.984288682552034 73.15758602205476) bank146418 +146419 POINT(40.243065552562086 74.5418621899453) bank146419 +146420 POINT(40.11234469366547 73.65408668422224) bank146420 +146421 POINT(40.32856613449356 74.52876344256605) bank146421 +146422 POINT(40.80558757651211 74.04491829618493) bank146422 +146423 POINT(41.13915495929568 73.54179333339293) bank146423 +146424 POINT(41.41076402368886 74.12601871479593) bank146424 +146425 POINT(40.13626932708978 73.45890763959532) bank146425 +146426 POINT(40.121321698192084 74.09548060892624) bank146426 +146427 POINT(39.91301152158104 74.25058592117726) bank146427 +146428 POINT(40.59707232970296 74.61050636332166) bank146428 +146429 POINT(40.95467128323304 74.9140435267665) bank146429 +146430 POINT(40.55868436208866 73.3091178935107) bank146430 +146431 POINT(40.480884012868216 73.3664437472511) bank146431 +146432 POINT(40.795310981783686 73.02884020693602) bank146432 +146433 POINT(40.95172014304919 74.79501520821117) bank146433 +146434 POINT(40.67525443626716 74.77291494983753) bank146434 +146435 POINT(41.36937540300955 74.77867016552338) bank146435 +146436 POINT(40.373860809182005 74.29626906940827) bank146436 +146437 POINT(41.27161986517714 73.21427965221514) bank146437 +146438 POINT(40.100083749398586 74.91239486696628) bank146438 +146439 POINT(40.08192240624857 73.97589381181557) bank146439 +146440 POINT(41.07892319721841 73.17342386393332) bank146440 +146441 POINT(41.20319820797262 73.3123697805421) bank146441 +146442 POINT(40.5601539532066 73.93091030252775) bank146442 +146443 POINT(39.99147814509502 74.54831047178833) bank146443 +146444 POINT(41.648197512294196 74.50241856556889) bank146444 +146445 POINT(39.98639324296303 73.39015402060389) bank146445 +146446 POINT(39.881596244518576 73.52791689929028) bank146446 +146447 POINT(41.53187745623711 74.74306112614632) bank146447 +146448 POINT(40.048979936982775 73.00750084122346) bank146448 +146449 POINT(39.87206731348461 74.4047657632209) bank146449 +146450 POINT(41.39453171602762 74.36543696889481) bank146450 +146451 POINT(40.981591359180115 73.02443127368072) bank146451 +146452 POINT(40.47272510815583 73.2091385419744) bank146452 +146453 POINT(40.83667058324993 74.56155208800708) bank146453 +146454 POINT(40.01448024380329 73.19102841022088) bank146454 +146455 POINT(40.52822634181515 74.62529230763866) bank146455 +146456 POINT(40.06776934286037 74.51100963374914) bank146456 +146457 POINT(40.00132591270779 73.25852265916318) bank146457 +146458 POINT(40.16103944882728 74.74160699056883) bank146458 +146459 POINT(41.51566477352452 74.73011974439103) bank146459 +146460 POINT(40.229560046613265 73.02416247147272) bank146460 +146461 POINT(41.44144551576041 74.2205606685148) bank146461 +146462 POINT(40.57989323302666 73.36466310564421) bank146462 +146463 POINT(40.38481506405383 74.09265310900183) bank146463 +146464 POINT(39.811596361201396 74.30892286306876) bank146464 +146465 POINT(40.65568419527017 73.25889905306502) bank146465 +146466 POINT(41.338616236757986 74.4448165474334) bank146466 +146467 POINT(41.169916339223086 74.76563989025355) bank146467 +146468 POINT(39.75959556678953 73.26671909892406) bank146468 +146469 POINT(40.29069134190748 73.39277289214671) bank146469 +146470 POINT(40.10940597798308 73.03494621615837) bank146470 +146471 POINT(41.54255225676351 73.84279813860027) bank146471 +146472 POINT(41.38928253868369 73.26162677766172) bank146472 +146473 POINT(39.718229843886846 74.69402883349012) bank146473 +146474 POINT(41.63026066759748 74.66555664875047) bank146474 +146475 POINT(41.54547400249446 73.628509783634) bank146475 +146476 POINT(40.59055134601382 74.68626514170452) bank146476 +146477 POINT(40.66176202459913 74.57274900376201) bank146477 +146478 POINT(40.96057712795949 74.17788602740204) bank146478 +146479 POINT(40.12581431390236 74.44484021315216) bank146479 +146480 POINT(40.05701337906292 73.54542176704435) bank146480 +146481 POINT(40.83651290404114 73.13624821168146) bank146481 +146482 POINT(41.15734091854705 74.12160752931779) bank146482 +146483 POINT(41.20279494940417 74.16502590914033) bank146483 +146484 POINT(41.11626076974501 73.83172392305568) bank146484 +146485 POINT(41.49205908777922 73.8290785847343) bank146485 +146486 POINT(40.86164437496358 74.2512184246126) bank146486 +146487 POINT(40.23736607600969 73.31590080839551) bank146487 +146488 POINT(41.44703801595014 74.8957724538099) bank146488 +146489 POINT(41.14694092587927 74.60747123118027) bank146489 +146490 POINT(40.35163923929568 73.95958495579038) bank146490 +146491 POINT(40.399955331856326 73.10080039792275) bank146491 +146492 POINT(40.008969894704876 73.25871164052938) bank146492 +146493 POINT(41.27750270449106 73.45978265986462) bank146493 +146494 POINT(39.98509051474061 73.93548073278596) bank146494 +146495 POINT(41.56491586352314 73.29481175241959) bank146495 +146496 POINT(39.800063179815346 74.88601666146629) bank146496 +146497 POINT(40.61602033583052 74.06100154403765) bank146497 +146498 POINT(40.59205232829784 74.15326922437167) bank146498 +146499 POINT(39.99493458835817 73.53294587008502) bank146499 +146500 POINT(41.21870750035591 74.96846055054499) bank146500 +146501 POINT(41.09462075288459 74.19142081762308) bank146501 +146502 POINT(40.993178275343396 74.71998063723582) bank146502 +146503 POINT(40.841005012459476 74.80200369530809) bank146503 +146504 POINT(40.38511241553512 73.46881328927057) bank146504 +146505 POINT(41.397356267831526 74.96373991325638) bank146505 +146506 POINT(40.27639256516404 73.11050655414248) bank146506 +146507 POINT(40.337774815847325 73.48769715339641) bank146507 +146508 POINT(40.06216119777825 74.46236889823986) bank146508 +146509 POINT(41.28093987670174 74.0057172596352) bank146509 +146510 POINT(39.97693033835305 74.00856078687593) bank146510 +146511 POINT(41.61709695413269 73.71362177800027) bank146511 +146512 POINT(40.96435639927381 74.46543001933952) bank146512 +146513 POINT(41.43917590022616 74.23751128325546) bank146513 +146514 POINT(40.74736606391135 74.0813975134275) bank146514 +146515 POINT(40.87923405927007 74.71254390690189) bank146515 +146516 POINT(40.10453294691905 74.74629256978575) bank146516 +146517 POINT(41.138134897715176 73.76116159288772) bank146517 +146518 POINT(39.88590620569397 73.58023486769942) bank146518 +146519 POINT(41.47582306894079 74.45575267997472) bank146519 +146520 POINT(40.87843857957594 74.95573119746886) bank146520 +146521 POINT(40.66991031795383 73.70950746418296) bank146521 +146522 POINT(40.910603183236425 74.35187549578467) bank146522 +146523 POINT(40.749019700549326 73.18976682493658) bank146523 +146524 POINT(40.77622768200518 74.56856794135406) bank146524 +146525 POINT(40.24118293248879 73.61591596493254) bank146525 +146526 POINT(40.853188720014074 73.89021174321503) bank146526 +146527 POINT(40.89726186891091 74.85566659712373) bank146527 +146528 POINT(40.450446137537895 74.48795246594293) bank146528 +146529 POINT(40.91016383432471 74.80782325372924) bank146529 +146530 POINT(40.672501497904804 73.6722867232544) bank146530 +146531 POINT(40.86081455431429 73.10220249674575) bank146531 +146532 POINT(40.61061342466251 73.62300270388111) bank146532 +146533 POINT(41.65145893821886 74.34175412693794) bank146533 +146534 POINT(39.874478108342046 73.91956515162852) bank146534 +146535 POINT(39.777940275226456 74.10118738952941) bank146535 +146536 POINT(41.24651491629117 73.64017161140448) bank146536 +146537 POINT(41.35224628724404 74.01010157830545) bank146537 +146538 POINT(40.09498236586445 74.60336828732936) bank146538 +146539 POINT(41.27698360904073 74.7217508167378) bank146539 +146540 POINT(39.878148616359965 73.79093883717518) bank146540 +146541 POINT(40.572433369644386 74.67985573438784) bank146541 +146542 POINT(40.98678942160767 73.80076072435472) bank146542 +146543 POINT(40.66968034565303 74.77473924464803) bank146543 +146544 POINT(40.6492877617335 74.84510648524619) bank146544 +146545 POINT(41.54395566975596 74.13585914955087) bank146545 +146546 POINT(40.81844730470328 73.77848502705858) bank146546 +146547 POINT(40.9524819143222 73.31975439172675) bank146547 +146548 POINT(41.18340233797763 74.56697181726524) bank146548 +146549 POINT(41.07217406293169 73.23215446220951) bank146549 +146550 POINT(39.98007804484401 74.36907839251658) bank146550 +146551 POINT(41.55292581485875 74.27388716231245) bank146551 +146552 POINT(41.404054188759865 74.04885169592703) bank146552 +146553 POINT(40.644023607115614 73.04667616174892) bank146553 +146554 POINT(39.82987447374627 74.04569443902786) bank146554 +146555 POINT(40.826823234434045 73.09061525901942) bank146555 +146556 POINT(41.49992303306569 73.7465037515625) bank146556 +146557 POINT(41.643616111436216 73.85967599709923) bank146557 +146558 POINT(41.247892268490645 74.6988746399847) bank146558 +146559 POINT(39.99651014435282 74.3981749746709) bank146559 +146560 POINT(41.620531584264825 74.282142979404) bank146560 +146561 POINT(39.83493865597447 74.45426449298402) bank146561 +146562 POINT(40.08576294793223 73.95023563583449) bank146562 +146563 POINT(41.614102921982166 74.24161538609948) bank146563 +146564 POINT(40.44606108446287 73.58458838325478) bank146564 +146565 POINT(40.14097035543767 73.04908942355479) bank146565 +146566 POINT(39.99167371361432 73.32349429409291) bank146566 +146567 POINT(39.98377452756781 74.99826998143934) bank146567 +146568 POINT(41.1000661679304 74.09292145722793) bank146568 +146569 POINT(40.41627662556567 75.00499393884652) bank146569 +146570 POINT(40.88549971327041 74.68638924856236) bank146570 +146571 POINT(39.84179545461157 74.3328151038932) bank146571 +146572 POINT(41.42509726352274 74.55165905098058) bank146572 +146573 POINT(40.332651902185376 74.3316502426788) bank146573 +146574 POINT(39.89220126909932 74.89279889134852) bank146574 +146575 POINT(40.35040922781859 73.65299799254568) bank146575 +146576 POINT(40.905707934031746 74.67969822235436) bank146576 +146577 POINT(41.62889086462049 74.88486747054047) bank146577 +146578 POINT(41.12012926111205 74.31917180048296) bank146578 +146579 POINT(41.154769931881496 74.03374657593106) bank146579 +146580 POINT(40.26999226377887 73.7269144221345) bank146580 +146581 POINT(39.90445230198279 73.88138130993735) bank146581 +146582 POINT(41.56372187749323 73.08489900601829) bank146582 +146583 POINT(41.38548132114903 73.16969645949617) bank146583 +146584 POINT(40.76876489437848 73.26978359848196) bank146584 +146585 POINT(40.766617162437605 74.18155404914936) bank146585 +146586 POINT(40.87721653373785 74.51900574496229) bank146586 +146587 POINT(40.32678959376237 73.86182213937137) bank146587 +146588 POINT(40.44327978593694 74.65267410728906) bank146588 +146589 POINT(40.032249156004546 74.31275485293634) bank146589 +146590 POINT(41.03914432192777 74.12029543152495) bank146590 +146591 POINT(41.51541908461797 74.4873692988579) bank146591 +146592 POINT(40.4253196468762 73.8949773773617) bank146592 +146593 POINT(41.43169597018551 73.99373663441612) bank146593 +146594 POINT(40.38663704699026 74.49555082678461) bank146594 +146595 POINT(41.24807484212042 73.28990574010051) bank146595 +146596 POINT(40.7945941575218 73.59374793555108) bank146596 +146597 POINT(40.09885857565617 73.7923910445556) bank146597 +146598 POINT(40.98238213852061 74.03271137198274) bank146598 +146599 POINT(41.636815556750776 74.82030560982095) bank146599 +146600 POINT(39.75391442184214 74.22809481974909) bank146600 +146601 POINT(40.49198346178733 73.09137614316947) bank146601 +146602 POINT(41.24778693499251 73.16935727403506) bank146602 +146603 POINT(41.48030060543929 74.74929769196821) bank146603 +146604 POINT(41.51569919124633 74.10874020510859) bank146604 +146605 POINT(41.231683464356145 74.66235861431856) bank146605 +146606 POINT(41.37493918532177 74.06878615839953) bank146606 +146607 POINT(41.10635609384 73.90482673390639) bank146607 +146608 POINT(40.3498329362018 74.99867976579148) bank146608 +146609 POINT(40.82748597387592 74.58337587181988) bank146609 +146610 POINT(40.9704633977085 74.16953154105653) bank146610 +146611 POINT(40.91403947539233 74.24296048564467) bank146611 +146612 POINT(41.216237747590384 74.80398198082835) bank146612 +146613 POINT(40.19801988611858 74.38708132358448) bank146613 +146614 POINT(40.58315069635609 74.03663047252658) bank146614 +146615 POINT(40.08359088644202 73.12863204472477) bank146615 +146616 POINT(40.01163375809255 74.82570998589473) bank146616 +146617 POINT(40.530771807869236 74.98600879762266) bank146617 +146618 POINT(40.24764198984385 74.4724213818671) bank146618 +146619 POINT(41.23327854183505 74.9297170887146) bank146619 +146620 POINT(41.23270223371643 73.17554202933921) bank146620 +146621 POINT(39.9216697001488 74.90710075477845) bank146621 +146622 POINT(40.35687292950149 73.61425020380639) bank146622 +146623 POINT(41.613285547833605 73.11598172363703) bank146623 +146624 POINT(41.08319572287 73.18426348338039) bank146624 +146625 POINT(40.07245962267166 74.02097175327184) bank146625 +146626 POINT(40.90662443788679 74.03548032113646) bank146626 +146627 POINT(39.88735352130027 73.79151798326403) bank146627 +146628 POINT(39.90977820748186 74.6006339871285) bank146628 +146629 POINT(39.811305557125316 73.52999198843663) bank146629 +146630 POINT(41.43391633532047 74.58010827690644) bank146630 +146631 POINT(41.22597846543821 73.50855595567238) bank146631 +146632 POINT(41.126907357629634 74.00010558041764) bank146632 +146633 POINT(39.76269006474654 74.55252844929294) bank146633 +146634 POINT(41.44562886322658 73.09441895030679) bank146634 +146635 POINT(40.45271031872628 73.50238161628283) bank146635 +146636 POINT(40.23554645309645 73.86054971957051) bank146636 +146637 POINT(40.26531427089287 74.0042800839489) bank146637 +146638 POINT(41.330796634159036 74.95407793264951) bank146638 +146639 POINT(41.29814834303275 74.99965621961194) bank146639 +146640 POINT(40.21350229253481 74.78108230626472) bank146640 +146641 POINT(40.88455086268419 73.63599263915647) bank146641 +146642 POINT(41.39745076103988 73.93339318805243) bank146642 +146643 POINT(41.57079894187857 73.20401663185089) bank146643 +146644 POINT(40.867538047057366 73.65716657809155) bank146644 +146645 POINT(40.92510505762903 73.91604099731654) bank146645 +146646 POINT(40.08447417586578 74.03769568546757) bank146646 +146647 POINT(40.52768620294704 73.83836780695538) bank146647 +146648 POINT(40.13541651229411 74.63976711835657) bank146648 +146649 POINT(41.23968344343259 74.23238787522874) bank146649 +146650 POINT(41.52497627423768 74.77883157460455) bank146650 +146651 POINT(40.86557951062691 73.14678574122811) bank146651 +146652 POINT(41.470201456191944 74.23942285765061) bank146652 +146653 POINT(40.831741018100445 73.90600331028789) bank146653 +146654 POINT(40.679832941943936 74.89156191715172) bank146654 +146655 POINT(39.98269232777075 74.71266550117252) bank146655 +146656 POINT(40.93582218274647 74.2040993845603) bank146656 +146657 POINT(40.105808181699985 73.88701432140059) bank146657 +146658 POINT(39.95667951011338 73.53800043926967) bank146658 +146659 POINT(39.74848826718539 73.93708890915771) bank146659 +146660 POINT(40.63340624179941 74.7262809564528) bank146660 +146661 POINT(40.97382031720076 73.4358946098633) bank146661 +146662 POINT(40.40832762201124 74.53963979230744) bank146662 +146663 POINT(41.078768108540515 73.48509145195685) bank146663 +146664 POINT(41.04253868332793 74.1815453314442) bank146664 +146665 POINT(39.7838093174828 73.09629460351577) bank146665 +146666 POINT(41.28363377428126 74.51974145910629) bank146666 +146667 POINT(40.30318407781288 73.57864595503705) bank146667 +146668 POINT(40.19762527995265 73.79226250695464) bank146668 +146669 POINT(41.225144441342586 73.15310831016697) bank146669 +146670 POINT(40.867190845328686 74.5424522057173) bank146670 +146671 POINT(40.30264696287618 73.49763250709984) bank146671 +146672 POINT(40.5001860474952 74.75751593537626) bank146672 +146673 POINT(41.70941279136578 73.56712114689375) bank146673 +146674 POINT(40.41203457924556 74.36945118164978) bank146674 +146675 POINT(41.662944754781435 74.45169439830318) bank146675 +146676 POINT(41.59998487762711 73.10193212990744) bank146676 +146677 POINT(41.08011846087124 74.21276855102589) bank146677 +146678 POINT(41.161422619089585 73.47514503639776) bank146678 +146679 POINT(40.18501023277413 74.99519381385835) bank146679 +146680 POINT(41.51875802417871 74.52334653447383) bank146680 +146681 POINT(40.630158175393994 73.31751443931388) bank146681 +146682 POINT(41.55464996288511 73.6637855095808) bank146682 +146683 POINT(41.477910607161284 73.32368141864885) bank146683 +146684 POINT(41.613137522165445 74.9042347516507) bank146684 +146685 POINT(40.62464763098781 73.07572068751725) bank146685 +146686 POINT(40.31528274298416 73.66240239679566) bank146686 +146687 POINT(41.497888113903976 73.08844338085528) bank146687 +146688 POINT(41.08254392253655 74.9850850076858) bank146688 +146689 POINT(40.88711615280875 74.25921540186027) bank146689 +146690 POINT(41.63734393991091 73.99561203988125) bank146690 +146691 POINT(41.10937014873115 74.42155781608466) bank146691 +146692 POINT(40.504216894859056 74.23504056906118) bank146692 +146693 POINT(40.747585090688254 74.87557737983934) bank146693 +146694 POINT(40.342184527798885 74.52219815380049) bank146694 +146695 POINT(41.306469435505726 73.80537348054085) bank146695 +146696 POINT(40.99769193844002 73.25404900151327) bank146696 +146697 POINT(41.55051965210313 73.76941942602646) bank146697 +146698 POINT(41.12509970035026 74.68115200490853) bank146698 +146699 POINT(40.76265827262748 74.90004865383521) bank146699 +146700 POINT(39.846555170767715 73.01455118921724) bank146700 +146701 POINT(40.420785467255754 74.128045801247) bank146701 +146702 POINT(40.970882120374334 74.87221935240369) bank146702 +146703 POINT(41.563560247802556 74.7685577749011) bank146703 +146704 POINT(40.64488240549478 73.2122209329606) bank146704 +146705 POINT(41.53342210175241 73.26707664923866) bank146705 +146706 POINT(40.71008311610769 74.37757092668758) bank146706 +146707 POINT(39.960081733750705 73.96909237676385) bank146707 +146708 POINT(40.10109692330752 73.56531482034299) bank146708 +146709 POINT(40.85378600068456 73.00948917755007) bank146709 +146710 POINT(41.29486433794478 74.51618181913123) bank146710 +146711 POINT(41.12187433585068 74.89764584715685) bank146711 +146712 POINT(41.70082807273629 73.99093413432503) bank146712 +146713 POINT(41.046279003289584 73.05420664037037) bank146713 +146714 POINT(39.852062037521684 74.37560541305166) bank146714 +146715 POINT(40.2775433298211 73.7205146410131) bank146715 +146716 POINT(40.91483010416455 74.24315683581929) bank146716 +146717 POINT(40.41603707161767 74.2052293930222) bank146717 +146718 POINT(40.0802341180264 74.27986652469161) bank146718 +146719 POINT(40.97776974599147 73.3337810753504) bank146719 +146720 POINT(41.14098948064222 73.32629097262175) bank146720 +146721 POINT(40.84740623334704 74.16955707161483) bank146721 +146722 POINT(39.93703795307431 73.05467818593267) bank146722 +146723 POINT(40.81581120205172 74.6043999004364) bank146723 +146724 POINT(40.07132841160811 73.07226784477898) bank146724 +146725 POINT(41.250164182066314 74.96392445614612) bank146725 +146726 POINT(40.41465444854421 73.86131051515966) bank146726 +146727 POINT(41.45276756554033 74.1226913453563) bank146727 +146728 POINT(41.11519038478418 74.43263854976078) bank146728 +146729 POINT(40.23622925826721 73.77033398131027) bank146729 +146730 POINT(40.12488605854883 73.56752994511643) bank146730 +146731 POINT(39.820123020743466 73.52944413359363) bank146731 +146732 POINT(40.738521189871975 74.51669174345808) bank146732 +146733 POINT(40.916235991740216 73.20581589399707) bank146733 +146734 POINT(40.92031854309742 73.7188939271269) bank146734 +146735 POINT(41.546027590115514 74.41019638689043) bank146735 +146736 POINT(40.332865185258896 74.47201354973441) bank146736 +146737 POINT(41.41142289428617 74.00722755785281) bank146737 +146738 POINT(40.08845953670739 74.92684576384706) bank146738 +146739 POINT(41.3675387018846 74.15743623667656) bank146739 +146740 POINT(40.71792927341122 73.8234729364378) bank146740 +146741 POINT(40.09330064565401 73.58542901693934) bank146741 +146742 POINT(40.96104619457044 74.49076448036622) bank146742 +146743 POINT(40.001867689693036 73.57664425375243) bank146743 +146744 POINT(40.42044367774223 74.32686169473047) bank146744 +146745 POINT(40.98367900421775 73.72905146554616) bank146745 +146746 POINT(41.23293987874211 74.34831099354115) bank146746 +146747 POINT(41.61039141288829 74.45066964033833) bank146747 +146748 POINT(39.860981982526674 74.42525068906315) bank146748 +146749 POINT(40.09831510294355 74.87039605803973) bank146749 +146750 POINT(41.677474216516984 73.7342287981711) bank146750 +146751 POINT(39.75979589864197 74.9643618652036) bank146751 +146752 POINT(39.88213345531015 74.9922574419689) bank146752 +146753 POINT(40.79726266397861 74.99416614672919) bank146753 +146754 POINT(41.2808075598019 73.65293885874554) bank146754 +146755 POINT(41.17544290491515 74.34039547294572) bank146755 +146756 POINT(41.14466365128636 74.10574465172108) bank146756 +146757 POINT(41.08944271744192 73.17807418261714) bank146757 +146758 POINT(40.28839786098178 73.43995358541358) bank146758 +146759 POINT(41.44656541269727 73.5105360105068) bank146759 +146760 POINT(40.50300381611941 74.11965230291707) bank146760 +146761 POINT(40.65289273499623 73.52741966453173) bank146761 +146762 POINT(40.957729831750946 73.09941083357897) bank146762 +146763 POINT(39.93235280045504 73.34842329180728) bank146763 +146764 POINT(40.07586075248092 73.04146448384569) bank146764 +146765 POINT(41.04157322921165 73.65469474227521) bank146765 +146766 POINT(40.40654052136915 73.43437517337436) bank146766 +146767 POINT(41.451173171918136 74.118820621141) bank146767 +146768 POINT(41.63913875184923 73.66989823385944) bank146768 +146769 POINT(40.63440352960069 74.73824065973432) bank146769 +146770 POINT(41.699953225666995 74.60869570707457) bank146770 +146771 POINT(40.19808795984443 74.11143918839979) bank146771 +146772 POINT(39.922159404772934 73.31958229904329) bank146772 +146773 POINT(41.35540558412242 74.15841710797281) bank146773 +146774 POINT(40.82517597304087 73.62862081161262) bank146774 +146775 POINT(40.614496836563745 74.27436067167909) bank146775 +146776 POINT(40.182799686302054 74.99922050656605) bank146776 +146777 POINT(41.46376456144399 74.52663999763708) bank146777 +146778 POINT(40.1912459258632 74.33150920669031) bank146778 +146779 POINT(41.3325533741536 74.80201386725595) bank146779 +146780 POINT(39.9053022715203 73.78868057209361) bank146780 +146781 POINT(41.49936643048001 74.93016550190879) bank146781 +146782 POINT(40.54644155673928 74.17245317178009) bank146782 +146783 POINT(40.49735954559053 74.92418203772138) bank146783 +146784 POINT(41.3580456705658 74.99439125273388) bank146784 +146785 POINT(39.799725427306456 74.2268153285899) bank146785 +146786 POINT(40.288255417982086 73.06253207942467) bank146786 +146787 POINT(40.29586930242159 73.35140560247456) bank146787 +146788 POINT(40.8862207804564 73.06249937010686) bank146788 +146789 POINT(40.05019154776837 73.44002520451654) bank146789 +146790 POINT(39.91337225944252 74.94298796161364) bank146790 +146791 POINT(41.56071684142347 73.89880087837979) bank146791 +146792 POINT(40.80004943972287 73.5646716909126) bank146792 +146793 POINT(40.828533807149824 73.05437533082358) bank146793 +146794 POINT(41.25932964559417 73.90045320034307) bank146794 +146795 POINT(39.77770556343219 74.61879627510757) bank146795 +146796 POINT(40.44614816657669 74.28130034679792) bank146796 +146797 POINT(40.59466375893011 73.59826717090104) bank146797 +146798 POINT(39.86988126751062 74.66093490330931) bank146798 +146799 POINT(39.938338638459484 74.35819302155768) bank146799 +146800 POINT(39.794227419084336 74.61518275381182) bank146800 +146801 POINT(41.39497121147426 73.24890976918934) bank146801 +146802 POINT(40.82455315999277 73.95228708120223) bank146802 +146803 POINT(39.7414845552287 74.32204939359141) bank146803 +146804 POINT(39.95757304882686 73.11686107812255) bank146804 +146805 POINT(40.36842953342287 73.71145716655457) bank146805 +146806 POINT(41.66886180639626 74.12799004218627) bank146806 +146807 POINT(41.545386562088446 73.07098470644824) bank146807 +146808 POINT(39.93498938083445 73.82754438184419) bank146808 +146809 POINT(41.46757339203621 73.70576715396994) bank146809 +146810 POINT(40.00661220180713 74.5418985026606) bank146810 +146811 POINT(39.90259097082922 73.16260530768011) bank146811 +146812 POINT(40.03926151442242 74.1846616089462) bank146812 +146813 POINT(40.72841980121382 73.20871878415694) bank146813 +146814 POINT(40.474198368085816 73.92201756315505) bank146814 +146815 POINT(40.36881608021256 73.02876239021047) bank146815 +146816 POINT(39.970998330236164 73.97890545728369) bank146816 +146817 POINT(41.107819097914124 73.9723774565052) bank146817 +146818 POINT(41.14790576439692 73.41285336309292) bank146818 +146819 POINT(40.654213487409095 74.0255907563843) bank146819 +146820 POINT(40.017833207223866 74.02335485924569) bank146820 +146821 POINT(40.77776012674153 73.97856426245274) bank146821 +146822 POINT(39.89638733863055 73.69223947319719) bank146822 +146823 POINT(41.12167960622607 74.26100549728451) bank146823 +146824 POINT(40.12280799959879 74.11933387325337) bank146824 +146825 POINT(41.45357376698961 73.53340879269176) bank146825 +146826 POINT(41.401874905281126 74.70841127347985) bank146826 +146827 POINT(41.57856329068557 74.87079084354363) bank146827 +146828 POINT(40.316542535015685 74.37317670174829) bank146828 +146829 POINT(40.428863575923 73.46746954117575) bank146829 +146830 POINT(41.49138198329711 73.57601560038812) bank146830 +146831 POINT(41.3928856160075 73.81807104173572) bank146831 +146832 POINT(40.30975035153966 74.48159227229695) bank146832 +146833 POINT(40.43722141786437 74.99416842364315) bank146833 +146834 POINT(41.39209227816971 73.3509582773879) bank146834 +146835 POINT(40.82756258422077 74.71352339057708) bank146835 +146836 POINT(40.89409719682397 74.13664510370658) bank146836 +146837 POINT(41.6369702221578 74.92653611857857) bank146837 +146838 POINT(41.270248849965505 74.67715073465897) bank146838 +146839 POINT(41.40063956304932 74.71587581918537) bank146839 +146840 POINT(39.803717837533384 74.858503245411) bank146840 +146841 POINT(41.06712433188369 73.22673306543773) bank146841 +146842 POINT(41.052733984339866 74.34490713212416) bank146842 +146843 POINT(40.81770468953234 74.9745741411227) bank146843 +146844 POINT(41.42661983682785 73.24761407845678) bank146844 +146845 POINT(40.03510619429445 74.62889124282675) bank146845 +146846 POINT(40.44127170862128 74.31324671351538) bank146846 +146847 POINT(41.537916636373076 73.63557692634669) bank146847 +146848 POINT(41.377551395051974 73.4916041642037) bank146848 +146849 POINT(40.04905837042213 73.21326871747445) bank146849 +146850 POINT(41.43369965756483 73.31321449015007) bank146850 +146851 POINT(41.63489085351915 74.1085766829148) bank146851 +146852 POINT(40.68985325718651 73.04190651421358) bank146852 +146853 POINT(39.996356791458986 74.5417771359671) bank146853 +146854 POINT(41.65993370670639 73.35159298459767) bank146854 +146855 POINT(41.038700254477696 74.05982353133057) bank146855 +146856 POINT(41.416778087575175 73.38948086463239) bank146856 +146857 POINT(40.23190701765866 74.36613861213527) bank146857 +146858 POINT(40.57254753267945 73.04220001975371) bank146858 +146859 POINT(41.24549245298013 73.95512496887399) bank146859 +146860 POINT(41.51879198215558 73.44557800064888) bank146860 +146861 POINT(40.37567230590196 73.30587924280596) bank146861 +146862 POINT(40.55698326495437 74.08083535855532) bank146862 +146863 POINT(40.90973160123504 73.81204597334563) bank146863 +146864 POINT(40.730406373686506 73.20472987918727) bank146864 +146865 POINT(41.65937068666764 74.15369847940386) bank146865 +146866 POINT(41.61641358531066 74.45561278466586) bank146866 +146867 POINT(40.42253120131394 73.91593573104615) bank146867 +146868 POINT(41.5710041674892 74.50050060597559) bank146868 +146869 POINT(40.76981183109679 73.53662644360747) bank146869 +146870 POINT(41.555021692280796 74.78543385534407) bank146870 +146871 POINT(40.221212543265864 73.85225252918822) bank146871 +146872 POINT(39.73033836294607 73.72790524872387) bank146872 +146873 POINT(40.010998583799974 73.73765597725036) bank146873 +146874 POINT(40.14780313727062 73.77036740756192) bank146874 +146875 POINT(40.23873378457697 73.96751701572292) bank146875 +146876 POINT(40.25193591464283 73.24657965899031) bank146876 +146877 POINT(39.74537432485486 73.4771314997949) bank146877 +146878 POINT(41.21549502343324 73.59812669750693) bank146878 +146879 POINT(39.914006017799885 74.40450467839176) bank146879 +146880 POINT(40.23405285126329 74.18333684803622) bank146880 +146881 POINT(41.177102016088114 73.39829427233187) bank146881 +146882 POINT(40.76134626529683 74.88571542177544) bank146882 +146883 POINT(40.602278031254734 74.27964818708429) bank146883 +146884 POINT(40.37959667656113 73.18409928588973) bank146884 +146885 POINT(39.76416411567143 74.76753660044592) bank146885 +146886 POINT(39.79252202922947 74.39897457776443) bank146886 +146887 POINT(40.819773881933706 73.16802486903231) bank146887 +146888 POINT(40.04346558259565 74.27702407373219) bank146888 +146889 POINT(41.56195336399028 73.8385555083744) bank146889 +146890 POINT(41.18226209906836 74.17229127439205) bank146890 +146891 POINT(39.79945852690563 74.14011088546958) bank146891 +146892 POINT(41.14122677046729 73.27304365743763) bank146892 +146893 POINT(41.04297131092865 73.58957752843631) bank146893 +146894 POINT(39.751357871006626 73.93393615150876) bank146894 +146895 POINT(39.81415489968462 73.50381692750508) bank146895 +146896 POINT(39.750451143338566 73.14114345483121) bank146896 +146897 POINT(40.31986886202192 74.13194914105794) bank146897 +146898 POINT(40.20921304752992 74.40345240791946) bank146898 +146899 POINT(41.31023649820324 73.6923264277211) bank146899 +146900 POINT(41.005655770539995 74.8409399992612) bank146900 +146901 POINT(39.714567845597145 74.4850826927897) bank146901 +146902 POINT(39.82345080214423 74.6341541450629) bank146902 +146903 POINT(40.585133078252525 73.14057325872548) bank146903 +146904 POINT(40.969337994309576 73.04072463834927) bank146904 +146905 POINT(41.56929832349761 74.77619320519551) bank146905 +146906 POINT(40.03042353921741 74.52404741554136) bank146906 +146907 POINT(41.53393854993305 73.39616121478137) bank146907 +146908 POINT(40.00244165778508 74.58210098615757) bank146908 +146909 POINT(41.314760790510896 74.886547921579) bank146909 +146910 POINT(40.536124751514045 74.6938746873288) bank146910 +146911 POINT(40.20392458561037 74.99750160521799) bank146911 +146912 POINT(41.23090720600614 73.15724501080645) bank146912 +146913 POINT(40.656051346571495 73.47999624478186) bank146913 +146914 POINT(40.574866182403944 73.53380162572924) bank146914 +146915 POINT(39.93601774114482 74.77002053247142) bank146915 +146916 POINT(41.62144437598263 74.21966741909525) bank146916 +146917 POINT(40.608623597284705 73.611203857728) bank146917 +146918 POINT(40.860376452434764 74.97983207299903) bank146918 +146919 POINT(40.35190122264346 73.0641609151757) bank146919 +146920 POINT(40.21446028654065 73.73431234277741) bank146920 +146921 POINT(41.46881361778551 74.00034330227551) bank146921 +146922 POINT(41.11798416637716 73.84314560832891) bank146922 +146923 POINT(40.32312389387164 74.8608776455484) bank146923 +146924 POINT(40.24626900879907 74.16034521051571) bank146924 +146925 POINT(40.52010980689278 73.97267822464099) bank146925 +146926 POINT(41.233225205088324 73.910628812123) bank146926 +146927 POINT(39.805417100641165 73.85220884601672) bank146927 +146928 POINT(40.05293655779851 73.32314837569322) bank146928 +146929 POINT(39.83001227742098 73.1970274199225) bank146929 +146930 POINT(39.774525370609126 74.55875663361611) bank146930 +146931 POINT(41.07363615901251 74.54016528210133) bank146931 +146932 POINT(40.97016390712449 74.7474873082551) bank146932 +146933 POINT(41.38825960232535 73.41713772850083) bank146933 +146934 POINT(40.78350785553136 74.84043452406227) bank146934 +146935 POINT(41.283080340593735 73.0412961838489) bank146935 +146936 POINT(40.37766339382071 74.20988576757733) bank146936 +146937 POINT(40.02592657507942 74.01648584365057) bank146937 +146938 POINT(41.14322426426064 73.60547145966369) bank146938 +146939 POINT(40.04680949934317 73.70988255654413) bank146939 +146940 POINT(40.519214264831646 74.40289419094911) bank146940 +146941 POINT(39.72462571347721 73.6882820556567) bank146941 +146942 POINT(41.65869844084743 73.73074097481702) bank146942 +146943 POINT(41.01566250749956 73.99268598126268) bank146943 +146944 POINT(40.77002278889419 74.07304361364854) bank146944 +146945 POINT(41.45896876435311 74.36461049999488) bank146945 +146946 POINT(40.214611186548126 74.44923549651607) bank146946 +146947 POINT(40.791348377999086 74.23274879552855) bank146947 +146948 POINT(41.390642850801626 74.05788572014697) bank146948 +146949 POINT(41.5904283217114 73.07564501248012) bank146949 +146950 POINT(41.626024666873604 74.4753458987712) bank146950 +146951 POINT(40.23344893310023 74.9934800366985) bank146951 +146952 POINT(40.59037372617039 73.92689716944679) bank146952 +146953 POINT(40.75232979644744 74.58618474971416) bank146953 +146954 POINT(41.616635894987034 73.2192092563198) bank146954 +146955 POINT(41.48214714754821 73.18443118559294) bank146955 +146956 POINT(40.3196774330123 73.64439776577296) bank146956 +146957 POINT(39.88811917933254 74.65518255255971) bank146957 +146958 POINT(41.367087542764025 74.3980864854732) bank146958 +146959 POINT(40.44371943559636 73.87923968279866) bank146959 +146960 POINT(39.78938127207211 74.28055352736695) bank146960 +146961 POINT(41.11467258625576 73.38730014619713) bank146961 +146962 POINT(41.2414403609253 74.44643554731574) bank146962 +146963 POINT(40.789999409625224 73.9067404697619) bank146963 +146964 POINT(40.347319746778496 73.85620995535905) bank146964 +146965 POINT(39.81613294387223 74.78123079041318) bank146965 +146966 POINT(41.63088852028993 73.83329832170863) bank146966 +146967 POINT(41.63915598653689 74.58313620535581) bank146967 +146968 POINT(39.99722798945955 74.73075456765196) bank146968 +146969 POINT(41.25837678878237 73.2681720861138) bank146969 +146970 POINT(39.85882173039765 73.18356188052125) bank146970 +146971 POINT(40.74101857476925 73.63591285514954) bank146971 +146972 POINT(39.788885823117646 73.13646234670708) bank146972 +146973 POINT(40.03725761336094 73.48121370894505) bank146973 +146974 POINT(39.75541381128787 73.10719281644069) bank146974 +146975 POINT(41.48876906378813 73.64204226958178) bank146975 +146976 POINT(40.606623976640954 73.4509855643598) bank146976 +146977 POINT(40.85968415104465 74.01677126441774) bank146977 +146978 POINT(41.36219426696809 73.41206559741823) bank146978 +146979 POINT(40.061147638875326 74.75847797216456) bank146979 +146980 POINT(39.9321853044286 73.76209630566073) bank146980 +146981 POINT(40.948582717692446 74.20185267934544) bank146981 +146982 POINT(40.41823606797688 73.54826235935707) bank146982 +146983 POINT(41.2662766869677 73.8990615613573) bank146983 +146984 POINT(40.17099760414667 74.5996911153106) bank146984 +146985 POINT(40.226661474881325 74.23884745532445) bank146985 +146986 POINT(40.426303303533395 74.86160525690062) bank146986 +146987 POINT(41.47182366180054 74.71596966746638) bank146987 +146988 POINT(41.4190537516275 73.95107157663323) bank146988 +146989 POINT(40.718026763712736 74.6087917674092) bank146989 +146990 POINT(41.14446552890835 73.9896388935391) bank146990 +146991 POINT(40.808069395454154 74.69268281477525) bank146991 +146992 POINT(40.976170900419454 73.1476903622425) bank146992 +146993 POINT(41.49569128168346 74.68612941564058) bank146993 +146994 POINT(40.98010868174608 74.43402825448491) bank146994 +146995 POINT(40.05328589847355 73.03465350983876) bank146995 +146996 POINT(41.12317078849981 74.1402270021843) bank146996 +146997 POINT(41.544361659133926 73.68161623086954) bank146997 +146998 POINT(40.40397106040562 74.08955077340623) bank146998 +146999 POINT(39.99773011676834 73.88358625757992) bank146999 +147000 POINT(40.445273271049466 73.66641000297392) bank147000 +147001 POINT(41.444041989375826 73.10964216464268) bank147001 +147002 POINT(40.587492647851725 73.81251128020946) bank147002 +147003 POINT(40.02204585783623 74.2043350348028) bank147003 +147004 POINT(40.04631800563662 74.0090334619636) bank147004 +147005 POINT(41.24361514017819 73.72522898185524) bank147005 +147006 POINT(40.1232696419032 73.67263345173228) bank147006 +147007 POINT(41.68473781078089 73.21999497936753) bank147007 +147008 POINT(40.1175094446658 74.51165513895529) bank147008 +147009 POINT(40.21724359896179 73.44065490430933) bank147009 +147010 POINT(39.809149222342924 74.834702345995) bank147010 +147011 POINT(40.560348405218505 74.86950111945191) bank147011 +147012 POINT(39.902447533705114 74.32421455435363) bank147012 +147013 POINT(40.44841257050696 73.0141589441563) bank147013 +147014 POINT(40.15347769887179 73.55433278852955) bank147014 +147015 POINT(40.39991555591944 73.77108625280684) bank147015 +147016 POINT(39.925558990531485 74.24500435800583) bank147016 +147017 POINT(39.74020059945255 73.60459283481768) bank147017 +147018 POINT(41.212803729921355 73.88178454312072) bank147018 +147019 POINT(41.395846664813384 73.39091880934156) bank147019 +147020 POINT(40.92065012981168 73.13321371402624) bank147020 +147021 POINT(41.08493591719465 74.3646051331461) bank147021 +147022 POINT(40.086647688970956 74.3890783233127) bank147022 +147023 POINT(41.09193709435998 73.63116031340543) bank147023 +147024 POINT(40.7706030296669 74.5165973377967) bank147024 +147025 POINT(40.672785325836806 74.38570343756565) bank147025 +147026 POINT(39.74702241777374 73.93769607177917) bank147026 +147027 POINT(41.16333894047827 74.05909608305197) bank147027 +147028 POINT(41.573980740399676 73.08266817839198) bank147028 +147029 POINT(41.27452602512209 74.12558939397992) bank147029 +147030 POINT(41.35413989199506 74.02437442518477) bank147030 +147031 POINT(41.71228477217573 74.08033077014407) bank147031 +147032 POINT(40.73364420836243 73.66928143973813) bank147032 +147033 POINT(41.35006503386315 73.8772707770745) bank147033 +147034 POINT(40.942244252134564 73.96089972509445) bank147034 +147035 POINT(41.100269731062404 74.89091282264455) bank147035 +147036 POINT(40.34440930396575 74.22956316457294) bank147036 +147037 POINT(41.216711904825985 73.36721221634797) bank147037 +147038 POINT(41.12972739856256 74.45451792523757) bank147038 +147039 POINT(41.625329009709205 74.32692911858726) bank147039 +147040 POINT(40.809619070700315 74.64207471252034) bank147040 +147041 POINT(40.501617222089656 74.74969789063377) bank147041 +147042 POINT(41.04652385639971 73.60871079705568) bank147042 +147043 POINT(40.161844873140815 74.53726141022078) bank147043 +147044 POINT(41.32867415803027 74.824830119959) bank147044 +147045 POINT(41.32934532808259 74.61187938638103) bank147045 +147046 POINT(40.37158101022433 73.65874817617545) bank147046 +147047 POINT(40.480221118809176 74.09005957802441) bank147047 +147048 POINT(40.056488218150875 73.08601060494433) bank147048 +147049 POINT(41.01761599190764 74.9567641890743) bank147049 +147050 POINT(40.785461383272064 74.35721854330376) bank147050 +147051 POINT(40.547934147329585 74.43839315421475) bank147051 +147052 POINT(40.889449829433175 73.05320615298481) bank147052 +147053 POINT(41.192844839210075 74.88021543377084) bank147053 +147054 POINT(39.749631110251045 74.77155372750515) bank147054 +147055 POINT(41.407496272206565 74.26525400280151) bank147055 +147056 POINT(41.55707810928906 73.38293341089944) bank147056 +147057 POINT(41.146942837250315 73.22094406089396) bank147057 +147058 POINT(41.389489762143164 74.51232889418395) bank147058 +147059 POINT(41.250339401922425 73.59590971290116) bank147059 +147060 POINT(41.35820758096746 74.15880050952674) bank147060 +147061 POINT(39.93930168058309 74.19019893614342) bank147061 +147062 POINT(40.37634197281972 74.28291583458142) bank147062 +147063 POINT(40.50499872260064 74.27593238081005) bank147063 +147064 POINT(40.48872773176783 74.99775334231924) bank147064 +147065 POINT(40.31532452927489 74.97877030284293) bank147065 +147066 POINT(41.41625843418833 74.41127032639633) bank147066 +147067 POINT(41.44506797156655 74.0590691686567) bank147067 +147068 POINT(40.982049004427495 74.47766801242315) bank147068 +147069 POINT(41.530636443061425 74.73650808802908) bank147069 +147070 POINT(40.885927321645084 73.93823600898307) bank147070 +147071 POINT(40.033154716876496 74.74393555999947) bank147071 +147072 POINT(40.5818265708244 73.86235131946664) bank147072 +147073 POINT(40.228246620629534 73.11365344717508) bank147073 +147074 POINT(40.79633333277413 74.3182103253905) bank147074 +147075 POINT(39.880481821921094 73.84591700420489) bank147075 +147076 POINT(39.95508793448924 73.67337668649968) bank147076 +147077 POINT(41.57212483206253 74.62658402835856) bank147077 +147078 POINT(41.15323562486556 73.3120616329081) bank147078 +147079 POINT(40.497705267171995 74.13265249578035) bank147079 +147080 POINT(39.739863627448585 74.68311003077528) bank147080 +147081 POINT(40.255256559182364 73.53703581021429) bank147081 +147082 POINT(41.45543052534082 73.99817781438533) bank147082 +147083 POINT(40.99115071504825 73.52999053712148) bank147083 +147084 POINT(41.33067880720098 74.02800669031696) bank147084 +147085 POINT(39.99232559026689 74.94508259014982) bank147085 +147086 POINT(41.337258374298365 73.58690790688681) bank147086 +147087 POINT(40.98426245180098 73.01526100127737) bank147087 +147088 POINT(40.073625557471196 73.30802906743634) bank147088 +147089 POINT(41.29262302466969 73.13469021666968) bank147089 +147090 POINT(40.3173641412901 74.10825988172375) bank147090 +147091 POINT(39.71536917439209 74.16699360157725) bank147091 +147092 POINT(39.91366078348118 74.47422612314651) bank147092 +147093 POINT(40.78460747409868 73.61465600280323) bank147093 +147094 POINT(39.9325144675826 74.01518391850459) bank147094 +147095 POINT(41.45635370623049 74.77243643673916) bank147095 +147096 POINT(40.60046198008551 74.81541048733345) bank147096 +147097 POINT(40.86755478441319 74.66491249538592) bank147097 +147098 POINT(41.00779999169879 74.58282865096237) bank147098 +147099 POINT(40.42328038261955 73.45767353277947) bank147099 +147100 POINT(40.15359012453543 73.698748562979) bank147100 +147101 POINT(40.58675272708947 74.60899074000709) bank147101 +147102 POINT(39.820306727891975 74.57755503472144) bank147102 +147103 POINT(39.94004616295907 74.94812903267022) bank147103 +147104 POINT(40.31843769015882 74.09276941245419) bank147104 +147105 POINT(41.07500465964266 73.23007531483152) bank147105 +147106 POINT(41.23098218732182 74.39421883114774) bank147106 +147107 POINT(41.58767470854784 74.78820068570086) bank147107 +147108 POINT(40.73316001569658 74.62829431982155) bank147108 +147109 POINT(40.07959650452272 74.3545260074313) bank147109 +147110 POINT(40.43995102786684 74.43806294506459) bank147110 +147111 POINT(39.83659379288814 73.35454395354516) bank147111 +147112 POINT(41.64459096523602 74.34611755401187) bank147112 +147113 POINT(40.34325591376184 74.5961532166107) bank147113 +147114 POINT(40.3990710002733 73.21876956549626) bank147114 +147115 POINT(40.362378849238965 74.46391431434587) bank147115 +147116 POINT(40.098509337040475 74.28387880559336) bank147116 +147117 POINT(40.27653397153189 73.10413668467542) bank147117 +147118 POINT(40.246027432439554 73.98417613579505) bank147118 +147119 POINT(40.006500078939915 74.74322126247628) bank147119 +147120 POINT(41.27903440843317 74.4065489496141) bank147120 +147121 POINT(40.382380782643345 73.7852498788635) bank147121 +147122 POINT(39.86288969393223 73.92473508443207) bank147122 +147123 POINT(41.47162138590494 74.06846468526194) bank147123 +147124 POINT(41.285804141195655 74.55525455279417) bank147124 +147125 POINT(40.5174030823323 73.50569318163674) bank147125 +147126 POINT(41.564837680523446 73.01433321941141) bank147126 +147127 POINT(40.04091569435146 74.08283538508918) bank147127 +147128 POINT(41.32718714537134 74.15174066066376) bank147128 +147129 POINT(41.44050314458441 73.67374258774072) bank147129 +147130 POINT(40.96740334533863 73.01160268480461) bank147130 +147131 POINT(40.09285380938812 74.17275143505222) bank147131 +147132 POINT(40.168946871554155 73.28961796828874) bank147132 +147133 POINT(40.67097497568272 73.55713301011345) bank147133 +147134 POINT(40.07971825272627 74.58798383488127) bank147134 +147135 POINT(41.221927219509446 74.978771077781) bank147135 +147136 POINT(41.43133853996558 73.8170294283941) bank147136 +147137 POINT(40.180138577119735 73.44186081190296) bank147137 +147138 POINT(41.35392727381662 74.91940568886518) bank147138 +147139 POINT(41.298679855021135 73.26527707036747) bank147139 +147140 POINT(40.22142601599265 73.0870880370487) bank147140 +147141 POINT(39.810592355479805 74.33811632370991) bank147141 +147142 POINT(41.120175699805515 74.90374242094266) bank147142 +147143 POINT(41.40280558005485 73.39925430588914) bank147143 +147144 POINT(41.293543406644766 73.42664869192186) bank147144 +147145 POINT(40.513858956030134 73.05975340228315) bank147145 +147146 POINT(40.409426563961205 73.19215462624858) bank147146 +147147 POINT(41.24078406515968 74.22826805274103) bank147147 +147148 POINT(39.80413173946704 73.00897424667755) bank147148 +147149 POINT(40.30106364009355 74.38447803937005) bank147149 +147150 POINT(40.001126071024736 73.8327229981327) bank147150 +147151 POINT(39.8576741497427 74.4266534603628) bank147151 +147152 POINT(40.01081949409766 74.56524385888433) bank147152 +147153 POINT(41.03886567601455 74.5626548496091) bank147153 +147154 POINT(39.82526595104675 73.42089499617255) bank147154 +147155 POINT(40.173997701270906 74.9078006042882) bank147155 +147156 POINT(40.61469751935581 73.26053326715596) bank147156 +147157 POINT(40.816121131337425 74.22939844885744) bank147157 +147158 POINT(40.99130670318516 73.81113193442044) bank147158 +147159 POINT(40.25335425205914 73.59435892649593) bank147159 +147160 POINT(40.39039787423728 73.65252645577495) bank147160 +147161 POINT(39.919929026869674 73.92793346276959) bank147161 +147162 POINT(41.5134082635498 74.59222433566524) bank147162 +147163 POINT(41.639602657662955 74.08284895541705) bank147163 +147164 POINT(40.77059428337299 74.70810008963802) bank147164 +147165 POINT(40.74867331330918 73.67601205566123) bank147165 +147166 POINT(40.95719210845214 74.89166639383355) bank147166 +147167 POINT(41.56246224062187 73.85880806766323) bank147167 +147168 POINT(41.64857065660795 74.25687052968108) bank147168 +147169 POINT(40.39200778993069 73.49468406714178) bank147169 +147170 POINT(40.885641304411244 74.82838431397238) bank147170 +147171 POINT(40.87436006695696 74.92339563207464) bank147171 +147172 POINT(40.167878445037324 74.44086607217048) bank147172 +147173 POINT(40.44494677344885 74.78672802272726) bank147173 +147174 POINT(40.94412277454355 73.7022081885225) bank147174 +147175 POINT(39.917421399119775 74.64808881611168) bank147175 +147176 POINT(40.6070231198003 73.0468954633085) bank147176 +147177 POINT(40.12110016470258 74.32605464904321) bank147177 +147178 POINT(40.176050183276935 74.72065841971805) bank147178 +147179 POINT(41.62615656752322 74.99981280091035) bank147179 +147180 POINT(39.92445388115257 74.84275251223202) bank147180 +147181 POINT(40.519638468069715 73.22167678789462) bank147181 +147182 POINT(40.17696857288678 73.1825543678193) bank147182 +147183 POINT(41.62695784843674 74.1074064606663) bank147183 +147184 POINT(40.51322545245558 73.07021648964518) bank147184 +147185 POINT(41.3190876889742 73.31531878344059) bank147185 +147186 POINT(39.88644536017867 74.97808217927016) bank147186 +147187 POINT(40.943965241574425 73.85927259551478) bank147187 +147188 POINT(40.254698405866776 73.0978980822426) bank147188 +147189 POINT(40.89487732432745 73.09717951013916) bank147189 +147190 POINT(41.29828955324972 73.90960195965108) bank147190 +147191 POINT(41.284692603040185 73.6745594675426) bank147191 +147192 POINT(41.115525613605115 73.07209599543427) bank147192 +147193 POINT(41.33542301865917 73.91796337874418) bank147193 +147194 POINT(41.573258145015515 74.36781093749728) bank147194 +147195 POINT(41.408790816098076 73.31355870308886) bank147195 +147196 POINT(40.92248451602377 74.2363834681857) bank147196 +147197 POINT(39.83988351669976 74.84069318676312) bank147197 +147198 POINT(39.741419088341374 73.02092751460151) bank147198 +147199 POINT(39.822776092369104 73.71587539872843) bank147199 +147200 POINT(40.75553807773201 74.35541211384427) bank147200 +147201 POINT(40.68881312776779 73.87215057563121) bank147201 +147202 POINT(41.47391751851503 74.43288875703645) bank147202 +147203 POINT(41.1965171449617 73.82905090036596) bank147203 +147204 POINT(40.972407890128835 73.10260916251006) bank147204 +147205 POINT(39.905286476765404 73.19701085862395) bank147205 +147206 POINT(41.701942954778566 74.93870500037062) bank147206 +147207 POINT(41.484416008613614 74.17010126563085) bank147207 +147208 POINT(41.40500690177151 74.30969916998515) bank147208 +147209 POINT(40.91135078952263 74.32789366078612) bank147209 +147210 POINT(39.8564639477209 73.69987073814988) bank147210 +147211 POINT(40.48688479652018 73.93329603218767) bank147211 +147212 POINT(41.258236614545794 73.63111878495069) bank147212 +147213 POINT(40.12175907494758 73.07079626247362) bank147213 +147214 POINT(41.38719475759261 73.82079059625782) bank147214 +147215 POINT(39.978099659321096 74.10354822985184) bank147215 +147216 POINT(41.60795856659174 74.50883789540012) bank147216 +147217 POINT(40.77215118969908 74.85264412858393) bank147217 +147218 POINT(40.28577261514641 74.19458171126756) bank147218 +147219 POINT(41.097929270569864 74.18012346252108) bank147219 +147220 POINT(40.583183658211176 73.45510481275343) bank147220 +147221 POINT(40.82863247915402 74.43400461572743) bank147221 +147222 POINT(41.033458193093026 74.96138614422641) bank147222 +147223 POINT(41.442716385571735 74.66537475362007) bank147223 +147224 POINT(41.578664784223605 74.71639747767837) bank147224 +147225 POINT(39.92996252337337 73.39359708041468) bank147225 +147226 POINT(40.371890695940564 73.34515953980498) bank147226 +147227 POINT(41.440191024861676 73.5353559418267) bank147227 +147228 POINT(39.74624111139943 74.01075573177633) bank147228 +147229 POINT(39.82783029427678 74.24433314302277) bank147229 +147230 POINT(39.79935581737366 73.0332191346179) bank147230 +147231 POINT(41.679861888388885 73.78672700914048) bank147231 +147232 POINT(40.82504379646379 74.19507448655591) bank147232 +147233 POINT(40.2001129960874 74.56415991378486) bank147233 +147234 POINT(39.87550840746212 73.9995711141665) bank147234 +147235 POINT(39.9585841918879 74.84316388159434) bank147235 +147236 POINT(41.674137827209066 74.97711892873049) bank147236 +147237 POINT(40.57689545659096 74.9350809676485) bank147237 +147238 POINT(40.49475598590973 73.39128452028895) bank147238 +147239 POINT(41.08690541183677 73.37398943135537) bank147239 +147240 POINT(40.643550718977174 73.54253869491886) bank147240 +147241 POINT(40.62046593602727 74.28588453502675) bank147241 +147242 POINT(40.72577212244585 73.13093106655039) bank147242 +147243 POINT(40.4996875867883 73.50738022760977) bank147243 +147244 POINT(40.77924220988787 74.0993070526382) bank147244 +147245 POINT(39.89581053629795 74.15138078864338) bank147245 +147246 POINT(39.96930475214936 73.5594292991067) bank147246 +147247 POINT(41.54339909639951 74.36201893686109) bank147247 +147248 POINT(40.101774197994146 73.70174116456327) bank147248 +147249 POINT(41.25353113780111 73.82060532488649) bank147249 +147250 POINT(40.488368952369136 73.72702976140954) bank147250 +147251 POINT(41.63682005736779 74.41427331695883) bank147251 +147252 POINT(41.296935161253764 74.47083547678666) bank147252 +147253 POINT(41.089067529060486 73.65977462111695) bank147253 +147254 POINT(41.565571979379776 74.58209116655293) bank147254 +147255 POINT(40.565301893625644 73.99310991613319) bank147255 +147256 POINT(40.00430151214915 73.92810334827112) bank147256 +147257 POINT(41.393919279990016 73.89858252934349) bank147257 +147258 POINT(39.79437803330301 74.96845743309936) bank147258 +147259 POINT(40.22948416817853 73.21614851567847) bank147259 +147260 POINT(40.61770585543949 74.68482926815328) bank147260 +147261 POINT(40.80636880587749 73.57834674689951) bank147261 +147262 POINT(40.97318147544455 74.16578666265885) bank147262 +147263 POINT(39.99020987419975 74.34410195768915) bank147263 +147264 POINT(40.47546115604836 73.79900588886105) bank147264 +147265 POINT(39.98794819228502 74.00540382942694) bank147265 +147266 POINT(41.36182511737804 74.99074035670084) bank147266 +147267 POINT(40.24443855523755 73.14055986942829) bank147267 +147268 POINT(39.757386642822176 73.71306905135489) bank147268 +147269 POINT(40.79168575915636 73.38540443221038) bank147269 +147270 POINT(41.06142830735274 74.99023383834322) bank147270 +147271 POINT(41.56553083331073 73.73716777153082) bank147271 +147272 POINT(39.79194425416568 74.94440614475178) bank147272 +147273 POINT(40.59374694428705 74.85601159087582) bank147273 +147274 POINT(40.91921331177585 74.93455463977693) bank147274 +147275 POINT(40.18579001436765 74.59491114106113) bank147275 +147276 POINT(40.07601092012962 74.04223043609957) bank147276 +147277 POINT(39.9743717955566 73.97976391093968) bank147277 +147278 POINT(41.16727786421693 73.1568399471678) bank147278 +147279 POINT(40.73177383934255 73.31116846072612) bank147279 +147280 POINT(39.73019565053785 73.86930519549345) bank147280 +147281 POINT(41.32200965578474 74.15447332313079) bank147281 +147282 POINT(41.42122898964872 74.58712525428639) bank147282 +147283 POINT(41.586975961437645 73.53389553721216) bank147283 +147284 POINT(40.70512391950917 74.0605782016416) bank147284 +147285 POINT(41.65259101168137 74.68656913776522) bank147285 +147286 POINT(40.99117654125272 73.85204032568345) bank147286 +147287 POINT(41.67743990671859 74.81118406547289) bank147287 +147288 POINT(40.40748821199876 74.43015403884402) bank147288 +147289 POINT(41.35548536614488 73.33751233302036) bank147289 +147290 POINT(41.21987443805828 73.13753347526551) bank147290 +147291 POINT(39.781600231747554 74.42263460730102) bank147291 +147292 POINT(40.315703784802004 74.98019970153628) bank147292 +147293 POINT(40.999993206111476 73.3728910686056) bank147293 +147294 POINT(39.813940344390744 74.478451839249) bank147294 +147295 POINT(40.011488218236444 73.15673239079285) bank147295 +147296 POINT(40.60897220176669 74.69988673487222) bank147296 +147297 POINT(40.07106616471481 74.83819780010995) bank147297 +147298 POINT(41.182863800973486 73.98078083931604) bank147298 +147299 POINT(41.380531372199044 74.85960192138896) bank147299 +147300 POINT(40.323802026903806 73.20326324078619) bank147300 +147301 POINT(40.141908104882425 73.4676779151213) bank147301 +147302 POINT(40.48542791010042 73.66425756824596) bank147302 +147303 POINT(40.81266721606598 73.79377729883372) bank147303 +147304 POINT(40.73137756282155 74.07385579465355) bank147304 +147305 POINT(40.1410995947811 73.07340469143162) bank147305 +147306 POINT(39.95432822604591 73.89753775610443) bank147306 +147307 POINT(41.63911169707875 74.01484951982337) bank147307 +147308 POINT(41.56661534361178 73.2014157984221) bank147308 +147309 POINT(40.316717629151185 74.40947841156571) bank147309 +147310 POINT(40.15296232930419 74.78131520247385) bank147310 +147311 POINT(40.494008645931004 73.02748933593702) bank147311 +147312 POINT(40.77050143765138 73.97078167589237) bank147312 +147313 POINT(40.62733376404248 74.8668554363883) bank147313 +147314 POINT(40.61006483067014 73.99119623026243) bank147314 +147315 POINT(41.06018148464668 73.44674519728704) bank147315 +147316 POINT(40.64589610289059 74.1446666165746) bank147316 +147317 POINT(41.26459503118999 73.01284926816405) bank147317 +147318 POINT(41.33923765140047 74.14219997375604) bank147318 +147319 POINT(40.015589547347034 74.75944331730052) bank147319 +147320 POINT(40.9786583896236 74.40486953370878) bank147320 +147321 POINT(41.1651999855764 73.22845841395193) bank147321 +147322 POINT(40.03007671872939 74.66034594119444) bank147322 +147323 POINT(40.44860661935971 73.8662161434294) bank147323 +147324 POINT(39.8221981733954 73.93228508243155) bank147324 +147325 POINT(41.537225431155626 74.5668395329416) bank147325 +147326 POINT(41.66651019504937 74.41213195997213) bank147326 +147327 POINT(41.680485245594646 74.84994465614001) bank147327 +147328 POINT(40.06072236792355 74.7879914058097) bank147328 +147329 POINT(41.00694635496176 74.40930674460864) bank147329 +147330 POINT(39.71678861377543 74.61553755169875) bank147330 +147331 POINT(41.157474738543584 73.3812073314637) bank147331 +147332 POINT(41.433701528270085 73.8217906264568) bank147332 +147333 POINT(41.44062948511446 74.26292088723707) bank147333 +147334 POINT(40.273969704293464 74.31813091582899) bank147334 +147335 POINT(41.63305956076069 73.62451320248668) bank147335 +147336 POINT(41.64692852536115 74.33971221588826) bank147336 +147337 POINT(40.47122408339086 73.68675645300807) bank147337 +147338 POINT(41.14977325997202 73.93283275403678) bank147338 +147339 POINT(41.00368187895783 74.82712585720714) bank147339 +147340 POINT(41.260221040898344 73.90157614662685) bank147340 +147341 POINT(41.03169063150132 73.68900791268723) bank147341 +147342 POINT(41.246537746496905 74.39827698681863) bank147342 +147343 POINT(40.97531282320733 73.38439723861) bank147343 +147344 POINT(41.189509341831545 74.52126031593318) bank147344 +147345 POINT(41.302714066761816 74.38992853536448) bank147345 +147346 POINT(41.150721601184046 73.1574956219721) bank147346 +147347 POINT(40.598650297801335 73.73949054794069) bank147347 +147348 POINT(41.03610528847732 74.5868394303898) bank147348 +147349 POINT(40.18924044496633 74.11990368358859) bank147349 +147350 POINT(40.08880988148466 74.88379908516292) bank147350 +147351 POINT(41.11563741905289 73.79872499829345) bank147351 +147352 POINT(40.39994812699404 73.68798359780733) bank147352 +147353 POINT(40.479603612375854 73.17365738907391) bank147353 +147354 POINT(40.66840329949799 73.29013024867344) bank147354 +147355 POINT(40.92894263699259 73.32426880060007) bank147355 +147356 POINT(40.46633824313771 74.10295429612718) bank147356 +147357 POINT(40.71047266183946 73.33920241675798) bank147357 +147358 POINT(41.299542479775745 74.945932132031) bank147358 +147359 POINT(41.56538850243289 73.92725885327415) bank147359 +147360 POINT(40.6517254736026 74.22592564376332) bank147360 +147361 POINT(41.59261193207803 73.15583615745567) bank147361 +147362 POINT(40.972682403604324 73.89804966199652) bank147362 +147363 POINT(41.29856099810519 74.60322712169008) bank147363 +147364 POINT(40.538158222186404 74.76379104765844) bank147364 +147365 POINT(40.403302783765355 75.00524682315263) bank147365 +147366 POINT(41.36400812616325 73.45610687574023) bank147366 +147367 POINT(40.71464970732194 73.793361887825) bank147367 +147368 POINT(39.82515479311724 74.51332466674198) bank147368 +147369 POINT(40.093912058725316 74.79615844153443) bank147369 +147370 POINT(41.368549860127565 73.04152278210555) bank147370 +147371 POINT(41.565898763066144 73.24699139100747) bank147371 +147372 POINT(41.33653902048184 74.15165607668763) bank147372 +147373 POINT(41.56866989675457 74.06415737475551) bank147373 +147374 POINT(40.14214042741622 74.06885701982623) bank147374 +147375 POINT(40.54752322543437 74.83406610913747) bank147375 +147376 POINT(41.26753666590519 73.8232923833866) bank147376 +147377 POINT(40.02720506586255 74.77152996073127) bank147377 +147378 POINT(40.56440188633386 74.1224270549939) bank147378 +147379 POINT(41.09072191038783 74.35258522860764) bank147379 +147380 POINT(40.72098242720447 74.38379759619971) bank147380 +147381 POINT(41.71088493190161 73.32452754783053) bank147381 +147382 POINT(40.20985014993325 75.00218201301901) bank147382 +147383 POINT(40.19731922941814 73.56166056200804) bank147383 +147384 POINT(40.53830395945628 73.76258902090403) bank147384 +147385 POINT(41.333432771977236 74.35597612307471) bank147385 +147386 POINT(39.943565193096084 74.62882747887153) bank147386 +147387 POINT(40.697091344829005 74.07933648124681) bank147387 +147388 POINT(40.167914819856 73.55581972235858) bank147388 +147389 POINT(40.532251277400405 74.66611267975803) bank147389 +147390 POINT(40.49489823172027 73.79319395161755) bank147390 +147391 POINT(40.217249091700396 74.4072733490755) bank147391 +147392 POINT(41.11327121823041 74.47061131096983) bank147392 +147393 POINT(41.17252554847881 74.48374106983275) bank147393 +147394 POINT(40.57683636696209 73.2761211464248) bank147394 +147395 POINT(41.44215154193065 73.8205154989212) bank147395 +147396 POINT(40.279629687242334 73.04515666728446) bank147396 +147397 POINT(39.8926469722757 73.50145219443417) bank147397 +147398 POINT(40.42122127113333 73.14573231101448) bank147398 +147399 POINT(40.679682450679 73.05162427738759) bank147399 +147400 POINT(40.77687309159884 74.91146946291974) bank147400 +147401 POINT(40.96195192595815 74.16671767763275) bank147401 +147402 POINT(39.8171301628834 74.25290505830328) bank147402 +147403 POINT(41.6139240444217 73.30119653456389) bank147403 +147404 POINT(41.57371022898123 74.61939954765195) bank147404 +147405 POINT(40.511154817361856 74.55239313638599) bank147405 +147406 POINT(40.85575009705536 73.57418195479731) bank147406 +147407 POINT(40.99592686259569 73.40049320143363) bank147407 +147408 POINT(41.12571517247622 73.78682772866051) bank147408 +147409 POINT(41.581605007894716 73.35547834946216) bank147409 +147410 POINT(40.340113703721414 74.19605873145507) bank147410 +147411 POINT(41.46504741537474 74.85434720801328) bank147411 +147412 POINT(40.57558445213278 74.38438224608613) bank147412 +147413 POINT(40.136287992957875 73.39978950698097) bank147413 +147414 POINT(40.29144718412236 73.88233571892566) bank147414 +147415 POINT(39.885444593631235 73.62575406392996) bank147415 +147416 POINT(41.600555841176316 74.19266905169937) bank147416 +147417 POINT(40.1729947463336 74.42603143924705) bank147417 +147418 POINT(40.548496705534525 74.86103578465703) bank147418 +147419 POINT(41.12510976352635 73.72319270195852) bank147419 +147420 POINT(41.16527208072613 73.03763483842758) bank147420 +147421 POINT(40.761483684832385 74.99757114918225) bank147421 +147422 POINT(41.51576992873535 73.12754935976459) bank147422 +147423 POINT(40.03060267247744 74.11682960164661) bank147423 +147424 POINT(40.28207052062989 73.15414326529805) bank147424 +147425 POINT(39.84677823380286 74.29911034659534) bank147425 +147426 POINT(39.80938523651784 74.6545300882064) bank147426 +147427 POINT(41.06935072122283 73.26721198236609) bank147427 +147428 POINT(40.74163721465393 74.34565234608266) bank147428 +147429 POINT(40.32256103578763 74.81925312240843) bank147429 +147430 POINT(41.608612196071256 74.33928477432384) bank147430 +147431 POINT(40.21049325773901 73.5752136372028) bank147431 +147432 POINT(39.732214589802176 73.8329172633539) bank147432 +147433 POINT(41.49426390142181 73.34059516138362) bank147433 +147434 POINT(40.10926498555361 74.92594395755697) bank147434 +147435 POINT(40.55823362405505 74.01231590876587) bank147435 +147436 POINT(40.53983513507588 74.20382363545866) bank147436 +147437 POINT(40.78216653248187 73.91853539546432) bank147437 +147438 POINT(40.347475574599024 74.12470287511036) bank147438 +147439 POINT(40.787909383559985 73.89682649824289) bank147439 +147440 POINT(41.049096879809184 73.48512541410315) bank147440 +147441 POINT(40.721172166391526 73.54153126880563) bank147441 +147442 POINT(41.331131883210475 74.82482422100215) bank147442 +147443 POINT(40.63745943538015 75.00527773154258) bank147443 +147444 POINT(40.38569162134406 73.46649149096467) bank147444 +147445 POINT(40.846635759949244 73.04019937809392) bank147445 +147446 POINT(40.97940132157409 73.54305424946068) bank147446 +147447 POINT(40.21354780098783 74.55372995560047) bank147447 +147448 POINT(41.637674094064664 73.45850045459284) bank147448 +147449 POINT(41.614604473799886 74.94755303138794) bank147449 +147450 POINT(41.25443580559928 74.99836045252441) bank147450 +147451 POINT(40.38623044724227 73.14012427823916) bank147451 +147452 POINT(41.63475468092091 73.99936800900859) bank147452 +147453 POINT(40.05972024162718 73.49485084317458) bank147453 +147454 POINT(40.04874992056615 74.84756761466781) bank147454 +147455 POINT(39.77302702916177 74.66223194635127) bank147455 +147456 POINT(40.61328057394153 74.65252403828293) bank147456 +147457 POINT(40.619075924412556 74.46622255175005) bank147457 +147458 POINT(41.04019681309025 74.25119820207686) bank147458 +147459 POINT(41.58575733585367 74.42492910683369) bank147459 +147460 POINT(41.66094671166912 74.82981185508645) bank147460 +147461 POINT(40.817217175816076 74.56004518362872) bank147461 +147462 POINT(40.66722997919116 73.53754110520215) bank147462 +147463 POINT(39.845290836812275 73.27232678797934) bank147463 +147464 POINT(41.56842758858201 74.17606242764101) bank147464 +147465 POINT(40.30207324264564 73.82737002874605) bank147465 +147466 POINT(41.683712540636655 74.45462473352924) bank147466 +147467 POINT(41.58084628583143 74.93065280727527) bank147467 +147468 POINT(40.643938821993494 74.74366319789338) bank147468 +147469 POINT(41.565048759999584 74.85557492843252) bank147469 +147470 POINT(40.434461921347435 74.0887007387592) bank147470 +147471 POINT(41.01016394776701 73.68303144030706) bank147471 +147472 POINT(41.55069783961304 74.43820095867804) bank147472 +147473 POINT(39.92216273587914 73.2560667178412) bank147473 +147474 POINT(40.48516708354912 73.79510834611162) bank147474 +147475 POINT(39.88470707133693 73.94419500435728) bank147475 +147476 POINT(41.19809182934274 73.44822349254585) bank147476 +147477 POINT(40.72490832166567 74.48391140260577) bank147477 +147478 POINT(40.659593908260895 74.58670306401828) bank147478 +147479 POINT(41.36845199824516 73.46202905009532) bank147479 +147480 POINT(41.58324870783381 73.6325700296715) bank147480 +147481 POINT(41.56731208133029 74.2042720472225) bank147481 +147482 POINT(40.253826409234776 74.39175628471465) bank147482 +147483 POINT(40.45537768237605 74.84729391075237) bank147483 +147484 POINT(40.463463604162726 74.8230762431457) bank147484 +147485 POINT(41.060860710716355 73.24112892612916) bank147485 +147486 POINT(40.668722448195226 74.11181665541727) bank147486 +147487 POINT(39.9512793835286 74.1877695704038) bank147487 +147488 POINT(40.17195994414637 74.05283819694363) bank147488 +147489 POINT(40.94748532936566 74.28517107398264) bank147489 +147490 POINT(40.183036212193436 74.66210499184953) bank147490 +147491 POINT(40.25406159574837 73.59199965128398) bank147491 +147492 POINT(41.662557107826906 74.24878174542192) bank147492 +147493 POINT(40.573100170789516 74.50251179948853) bank147493 +147494 POINT(40.34448543893487 73.16955433508892) bank147494 +147495 POINT(40.471481324349746 73.22008890753408) bank147495 +147496 POINT(40.77825164387894 74.80674172215257) bank147496 +147497 POINT(40.379267430437594 73.50022591250365) bank147497 +147498 POINT(41.021671596469794 73.10643003004083) bank147498 +147499 POINT(40.07695754184022 73.111884073566) bank147499 +147500 POINT(41.376179545249805 74.31285824955576) bank147500 +147501 POINT(40.64801133628354 74.79265373137605) bank147501 +147502 POINT(40.57263721324829 74.79303311283466) bank147502 +147503 POINT(41.31777513085737 74.91846088474391) bank147503 +147504 POINT(41.366595577365715 73.28612273643051) bank147504 +147505 POINT(41.18632273342326 73.62257806112802) bank147505 +147506 POINT(41.30486579033825 73.39123101276886) bank147506 +147507 POINT(41.35567120271061 74.9639677056455) bank147507 +147508 POINT(41.355910238128125 73.60910558297826) bank147508 +147509 POINT(40.23949027653814 74.51252905468638) bank147509 +147510 POINT(41.061942776024665 74.46853637170177) bank147510 +147511 POINT(40.96452467506013 73.35590021003449) bank147511 +147512 POINT(40.432870537578985 74.7873209427082) bank147512 +147513 POINT(40.24526543159082 73.83632657270586) bank147513 +147514 POINT(39.821530004599346 74.79223643763162) bank147514 +147515 POINT(40.99434764891916 74.01082890261449) bank147515 +147516 POINT(41.06805458779772 74.19746164628562) bank147516 +147517 POINT(40.743337518779164 73.68845604009503) bank147517 +147518 POINT(40.69778927786491 74.13729596724471) bank147518 +147519 POINT(40.45534181616382 74.10242707044279) bank147519 +147520 POINT(40.9007187989774 73.55355205399005) bank147520 +147521 POINT(41.56817243402448 74.11183008832771) bank147521 +147522 POINT(40.93570564316364 74.29399473093444) bank147522 +147523 POINT(40.765287770698436 73.56593451989721) bank147523 +147524 POINT(40.01359734617046 74.17861615194565) bank147524 +147525 POINT(40.90319852167917 74.9344505890852) bank147525 +147526 POINT(41.587055699794185 74.82883416765918) bank147526 +147527 POINT(39.85564416288871 73.31135569181549) bank147527 +147528 POINT(41.68538294936041 74.87540682912869) bank147528 +147529 POINT(41.44428444344051 74.765167816569) bank147529 +147530 POINT(40.83089586689991 74.4136070579143) bank147530 +147531 POINT(41.25814003898837 73.5457460544233) bank147531 +147532 POINT(40.3883898013132 74.1370105858865) bank147532 +147533 POINT(40.13300527787 73.13813260471066) bank147533 +147534 POINT(41.16445145099208 74.4504347122224) bank147534 +147535 POINT(41.56447031286501 73.09925224619506) bank147535 +147536 POINT(40.792962536931235 73.49548288493517) bank147536 +147537 POINT(39.936078593292564 73.16044422910092) bank147537 +147538 POINT(40.030920245321276 73.08565519060284) bank147538 +147539 POINT(40.46295910125877 74.04347437507772) bank147539 +147540 POINT(40.281908232771144 73.51149357950108) bank147540 +147541 POINT(40.33073526152167 74.02805592888934) bank147541 +147542 POINT(40.324919554465566 73.96172977240697) bank147542 +147543 POINT(40.0172199273888 74.90749516677124) bank147543 +147544 POINT(41.41147571382327 74.94968197734876) bank147544 +147545 POINT(40.61053799147905 74.61461858287417) bank147545 +147546 POINT(40.4862266086731 74.33960107248808) bank147546 +147547 POINT(40.95047848224055 73.29139487844212) bank147547 +147548 POINT(41.01640533990069 74.68819427644159) bank147548 +147549 POINT(39.987781449419444 74.19581722502774) bank147549 +147550 POINT(41.396022820568724 74.16945692559729) bank147550 +147551 POINT(41.3817319436041 74.5780675864367) bank147551 +147552 POINT(41.3575418821078 74.70328543903125) bank147552 +147553 POINT(40.3513892623103 74.97306761252473) bank147553 +147554 POINT(41.544485722993095 73.65553718370882) bank147554 +147555 POINT(41.076659693732644 74.015527971792) bank147555 +147556 POINT(40.66217273694624 74.37635878427733) bank147556 +147557 POINT(40.592303338189154 74.45041893529778) bank147557 +147558 POINT(40.641618960812245 74.61059165485145) bank147558 +147559 POINT(41.5152397152389 73.31557787305134) bank147559 +147560 POINT(40.632606145094094 73.86968352510513) bank147560 +147561 POINT(40.11026235311717 74.70053168312647) bank147561 +147562 POINT(41.51213439288639 73.79296080260335) bank147562 +147563 POINT(40.173514576926536 74.24919330043056) bank147563 +147564 POINT(40.54869656320538 73.44836373111819) bank147564 +147565 POINT(41.68542553966627 73.14225371537856) bank147565 +147566 POINT(39.816632952534015 74.09543765821857) bank147566 +147567 POINT(41.41045946883137 73.06921213122033) bank147567 +147568 POINT(40.069507842062656 73.35392095839283) bank147568 +147569 POINT(40.19697761996822 73.39978844907728) bank147569 +147570 POINT(41.32054158909708 74.12099062214973) bank147570 +147571 POINT(40.60111850626011 73.61240478843138) bank147571 +147572 POINT(39.86501920530976 73.99225611416936) bank147572 +147573 POINT(41.711038093046184 74.75024257804016) bank147573 +147574 POINT(41.24618670753057 73.874604615285) bank147574 +147575 POINT(40.30116919592932 73.11995039119427) bank147575 +147576 POINT(40.47436076949731 74.15484365343559) bank147576 +147577 POINT(41.35074713147241 74.56033572856263) bank147577 +147578 POINT(39.844548051727905 74.12064941278722) bank147578 +147579 POINT(41.28054250316798 73.5548178100139) bank147579 +147580 POINT(41.16019802769973 74.09747057399821) bank147580 +147581 POINT(40.24331590814966 74.1678965453232) bank147581 +147582 POINT(41.38957765507 74.08460948123259) bank147582 +147583 POINT(40.54271935524437 74.07808465200023) bank147583 +147584 POINT(41.14790526117184 73.50832710593143) bank147584 +147585 POINT(40.44822006180235 74.55468616841263) bank147585 +147586 POINT(40.093874290477366 74.69822148144736) bank147586 +147587 POINT(39.973722437181465 74.09858195225672) bank147587 +147588 POINT(41.3606572983973 73.8015430817755) bank147588 +147589 POINT(40.498577416556664 73.38373786449759) bank147589 +147590 POINT(39.98143028414409 74.88806429828007) bank147590 +147591 POINT(40.30541938249989 74.03671364984793) bank147591 +147592 POINT(39.75846161490051 74.40871159219577) bank147592 +147593 POINT(41.66761441024855 74.49467418113169) bank147593 +147594 POINT(40.672966738439534 73.93601393258622) bank147594 +147595 POINT(41.00140421218171 74.97519648572838) bank147595 +147596 POINT(40.51559746181914 73.90772927037348) bank147596 +147597 POINT(40.819586222334216 74.23732776803897) bank147597 +147598 POINT(40.773223333789346 73.37607768712107) bank147598 +147599 POINT(41.35897442098699 73.58972742801203) bank147599 +147600 POINT(40.91287327391243 73.56812228010978) bank147600 +147601 POINT(41.40075178654817 74.4196456230241) bank147601 +147602 POINT(40.86400832340196 74.25435896267096) bank147602 +147603 POINT(40.83077781380059 73.40781939082126) bank147603 +147604 POINT(40.38084163177785 73.79109935415553) bank147604 +147605 POINT(39.863635657705764 73.6998483851759) bank147605 +147606 POINT(41.21377584735346 74.78519469291626) bank147606 +147607 POINT(41.53935029419334 74.10839018286359) bank147607 +147608 POINT(40.227610057444565 73.37386034827747) bank147608 +147609 POINT(41.10564289999628 74.29944833300465) bank147609 +147610 POINT(39.721999632363115 73.57424105284208) bank147610 +147611 POINT(41.60263440483716 73.67244675926419) bank147611 +147612 POINT(40.504834139827665 73.46320521920694) bank147612 +147613 POINT(39.908037969394485 73.08555612937877) bank147613 +147614 POINT(40.11417196570499 74.59021229691196) bank147614 +147615 POINT(40.3761086863134 73.78617737713469) bank147615 +147616 POINT(41.32317508546847 73.2835965429972) bank147616 +147617 POINT(41.30624458221252 74.10446838083938) bank147617 +147618 POINT(40.11037178144521 74.60011547841006) bank147618 +147619 POINT(40.07820801310249 73.4893767464533) bank147619 +147620 POINT(41.477449343147 73.81575970526875) bank147620 +147621 POINT(40.79711077653777 73.23502761029175) bank147621 +147622 POINT(40.92893431373794 74.40502610315424) bank147622 +147623 POINT(41.117682481704605 73.27113559678223) bank147623 +147624 POINT(40.93589403599739 73.78443913765355) bank147624 +147625 POINT(39.802413489786005 73.413582482566) bank147625 +147626 POINT(40.48975778666939 74.86549753848298) bank147626 +147627 POINT(39.74815945132708 73.2125656237556) bank147627 +147628 POINT(40.74408312771784 73.06490367151567) bank147628 +147629 POINT(40.67867578091341 74.2215744727513) bank147629 +147630 POINT(40.342574366786664 74.89839729019448) bank147630 +147631 POINT(41.011793256396686 73.93737681768212) bank147631 +147632 POINT(40.83506940748885 74.86337487021422) bank147632 +147633 POINT(41.678149282154095 73.38227221255232) bank147633 +147634 POINT(40.09130338551845 73.73528680733655) bank147634 +147635 POINT(40.76749908556954 73.19152393350912) bank147635 +147636 POINT(40.98947965380464 74.00803106274253) bank147636 +147637 POINT(41.35160014829899 73.79556185281857) bank147637 +147638 POINT(41.58358412934121 74.8488231855901) bank147638 +147639 POINT(40.8440614241553 74.63690975179485) bank147639 +147640 POINT(40.81960132586908 73.37732527723435) bank147640 +147641 POINT(39.92052033946006 73.83769908475182) bank147641 +147642 POINT(41.6864678456469 74.64101105192296) bank147642 +147643 POINT(39.82054646268426 74.67718020671411) bank147643 +147644 POINT(40.0722735567369 73.40568781244986) bank147644 +147645 POINT(40.31312732673759 74.40405736385351) bank147645 +147646 POINT(40.386381788385336 74.56970501867868) bank147646 +147647 POINT(40.64454367238344 73.7420422576839) bank147647 +147648 POINT(41.00514198487719 74.07992411614173) bank147648 +147649 POINT(41.358209028452 73.28243720876347) bank147649 +147650 POINT(40.6968469217112 73.07050977726166) bank147650 +147651 POINT(41.29024696554339 74.51982941446478) bank147651 +147652 POINT(40.997874556461646 73.27920792667658) bank147652 +147653 POINT(39.887655348103 74.51017529769126) bank147653 +147654 POINT(41.47294688547393 73.49653966573044) bank147654 +147655 POINT(41.51874533315197 73.5323292697065) bank147655 +147656 POINT(41.65280007567705 74.42768466282435) bank147656 +147657 POINT(40.30973892891602 73.59856538408148) bank147657 +147658 POINT(41.36594031961837 73.47228477350197) bank147658 +147659 POINT(40.707020487173025 73.61621281138024) bank147659 +147660 POINT(41.57779983550178 73.04604845278682) bank147660 +147661 POINT(39.73543243365994 74.24614605419221) bank147661 +147662 POINT(41.088925181955965 74.7499496041379) bank147662 +147663 POINT(41.351767506622366 74.01437856334545) bank147663 +147664 POINT(40.475686942315676 74.55325610645322) bank147664 +147665 POINT(40.91059723499907 75.00447847205996) bank147665 +147666 POINT(39.71586033315144 74.20556565384237) bank147666 +147667 POINT(41.18811415353934 74.72215093615733) bank147667 +147668 POINT(39.93106871469971 74.31449062342432) bank147668 +147669 POINT(41.6126232556208 74.85792800513897) bank147669 +147670 POINT(40.11347230384681 73.20025078489701) bank147670 +147671 POINT(41.14349770629472 74.29211234550725) bank147671 +147672 POINT(40.336244737576344 73.90192229445447) bank147672 +147673 POINT(40.571154174716085 73.94630979516863) bank147673 +147674 POINT(40.982672673213905 73.20087254109022) bank147674 +147675 POINT(41.1788137272671 74.51588936435718) bank147675 +147676 POINT(39.99328507379041 74.37482864231193) bank147676 +147677 POINT(39.908694911164716 74.4124293448384) bank147677 +147678 POINT(39.71340532121827 74.41536554319332) bank147678 +147679 POINT(41.213818590223404 73.62554874248808) bank147679 +147680 POINT(41.15976629571514 74.64291594080274) bank147680 +147681 POINT(40.21855840205033 74.17349935875511) bank147681 +147682 POINT(40.748864668989775 74.77035847681715) bank147682 +147683 POINT(41.16265840437719 74.85503886343967) bank147683 +147684 POINT(39.92360311204647 74.22147857924276) bank147684 +147685 POINT(41.246374015370584 74.9716283575352) bank147685 +147686 POINT(41.502156608622 73.09345326405517) bank147686 +147687 POINT(41.27059291832885 73.9852633297468) bank147687 +147688 POINT(40.31988421401526 73.1634992364074) bank147688 +147689 POINT(40.63251677447364 74.88100564179088) bank147689 +147690 POINT(39.76483101879369 73.1294687951868) bank147690 +147691 POINT(40.25811566880694 74.2484006849133) bank147691 +147692 POINT(40.502947422001355 73.55609947364132) bank147692 +147693 POINT(39.89165710185038 73.38828258428417) bank147693 +147694 POINT(41.067428033625035 74.95044891797652) bank147694 +147695 POINT(40.26224099423311 74.69216204529393) bank147695 +147696 POINT(40.60073088309945 74.87407252423121) bank147696 +147697 POINT(39.9435111692673 74.85763098727939) bank147697 +147698 POINT(40.35280636056577 73.34385788452325) bank147698 +147699 POINT(41.24501375984057 73.9564729831689) bank147699 +147700 POINT(39.76985536107036 74.59612280659914) bank147700 +147701 POINT(40.941112365560464 74.23062772348119) bank147701 +147702 POINT(39.852851257545325 74.34612660679856) bank147702 +147703 POINT(40.26681642151566 73.7473262235931) bank147703 +147704 POINT(40.154420157533146 73.70836614887936) bank147704 +147705 POINT(41.50265907944495 74.9981157952914) bank147705 +147706 POINT(41.212199790808825 73.10549181916593) bank147706 +147707 POINT(41.439133245319475 73.65210633248657) bank147707 +147708 POINT(40.99516672976901 73.75773547942585) bank147708 +147709 POINT(41.390612275511934 74.23112913624237) bank147709 +147710 POINT(41.49123746266582 74.17046930746628) bank147710 +147711 POINT(41.61042106870126 74.65395191414645) bank147711 +147712 POINT(41.44052327409195 74.14932070711089) bank147712 +147713 POINT(40.57684749784548 74.98340736838256) bank147713 +147714 POINT(40.15518671326749 74.78653543290267) bank147714 +147715 POINT(40.87803027510763 73.15871543700062) bank147715 +147716 POINT(41.07905911921678 74.63344646604375) bank147716 +147717 POINT(40.88306458063963 74.34563256734629) bank147717 +147718 POINT(40.519058597053174 74.9827464189924) bank147718 +147719 POINT(40.54113023638411 74.76320119429998) bank147719 +147720 POINT(40.29149140123751 74.53835556992594) bank147720 +147721 POINT(40.94564698436863 74.29231119796039) bank147721 +147722 POINT(41.14645635170648 73.24835469686587) bank147722 +147723 POINT(39.81874855537 74.43234306157812) bank147723 +147724 POINT(40.23702869506039 74.27141033841191) bank147724 +147725 POINT(39.82644930769068 73.72735366091918) bank147725 +147726 POINT(40.45388954181092 74.75605080432635) bank147726 +147727 POINT(41.35911123640746 74.09983777575653) bank147727 +147728 POINT(40.735254155374285 73.8829037538935) bank147728 +147729 POINT(41.44710880246022 74.97933861192743) bank147729 +147730 POINT(40.82063571891785 74.10170501273562) bank147730 +147731 POINT(41.25188476336739 73.52814868962254) bank147731 +147732 POINT(41.31471430281931 74.63564883564015) bank147732 +147733 POINT(40.327262217489555 73.09722583490205) bank147733 +147734 POINT(41.69773160648652 73.7468487101714) bank147734 +147735 POINT(40.068006270218056 73.40520276083132) bank147735 +147736 POINT(39.76418081037624 74.97022469679294) bank147736 +147737 POINT(40.495541257658076 73.60072843740708) bank147737 +147738 POINT(40.60552851008733 74.77115244605903) bank147738 +147739 POINT(40.73127213318482 74.93838425972736) bank147739 +147740 POINT(40.56304391250244 74.97445949266253) bank147740 +147741 POINT(40.82260973323658 73.75883114870224) bank147741 +147742 POINT(41.67602955183018 73.55288355237053) bank147742 +147743 POINT(41.24059729990248 73.72998939889705) bank147743 +147744 POINT(40.20678496369881 73.34496986941083) bank147744 +147745 POINT(40.2856791140316 73.60521174524871) bank147745 +147746 POINT(40.443290993290915 74.2276968615323) bank147746 +147747 POINT(41.004857295149314 74.4349146087042) bank147747 +147748 POINT(41.652885167212084 74.58754932304238) bank147748 +147749 POINT(40.33813103642218 74.95605568764613) bank147749 +147750 POINT(39.83913655877691 74.37095195936573) bank147750 +147751 POINT(39.927466175764486 74.4169789722475) bank147751 +147752 POINT(41.71258669154384 74.24990888767174) bank147752 +147753 POINT(41.24243947220652 73.2133399457979) bank147753 +147754 POINT(40.90406804972214 73.65030532895672) bank147754 +147755 POINT(40.92585633639166 73.14167615315438) bank147755 +147756 POINT(39.72524042991943 73.4907814708193) bank147756 +147757 POINT(40.85858521457098 73.26212785539269) bank147757 +147758 POINT(41.483369872338116 74.04539416932417) bank147758 +147759 POINT(41.22835949099871 74.30094809792617) bank147759 +147760 POINT(40.00997249382371 74.88780706868295) bank147760 +147761 POINT(40.32632790329529 74.8870785513179) bank147761 +147762 POINT(41.10711606670153 73.43639092746646) bank147762 +147763 POINT(40.62420555540401 73.18652236683397) bank147763 +147764 POINT(41.56900159342197 74.65481790630608) bank147764 +147765 POINT(39.88359053062688 74.78263718504822) bank147765 +147766 POINT(40.09685237207638 74.24877480604614) bank147766 +147767 POINT(41.09861324171243 73.51564992977896) bank147767 +147768 POINT(39.89236688263219 74.36559790816466) bank147768 +147769 POINT(41.70961238091412 74.57730213386453) bank147769 +147770 POINT(41.555770839743516 73.36708044362682) bank147770 +147771 POINT(40.92907350434119 74.80799971337746) bank147771 +147772 POINT(40.139851226017015 75.00143147277359) bank147772 +147773 POINT(40.38578146958981 73.50671213348305) bank147773 +147774 POINT(40.88993934888707 73.02006100079288) bank147774 +147775 POINT(41.40355569073339 74.41385454177623) bank147775 +147776 POINT(40.81490619802446 73.2001685894492) bank147776 +147777 POINT(40.249017354531546 73.03880384741016) bank147777 +147778 POINT(40.95533235086262 74.20544661003176) bank147778 +147779 POINT(40.856839781932955 73.19959574753796) bank147779 +147780 POINT(41.172148210985064 73.27526340750993) bank147780 +147781 POINT(40.01266712907979 73.96083357731897) bank147781 +147782 POINT(39.79602942727193 74.69563451899074) bank147782 +147783 POINT(40.59449051901781 73.5418701608705) bank147783 +147784 POINT(40.01707996076719 74.20108495750952) bank147784 +147785 POINT(40.07886436998426 73.70031825874929) bank147785 +147786 POINT(40.265175506345976 74.76618410845448) bank147786 +147787 POINT(40.59624241713035 74.00329335595464) bank147787 +147788 POINT(41.06642440684657 74.45629382461192) bank147788 +147789 POINT(40.08330359706149 73.81832015786127) bank147789 +147790 POINT(40.948727068660666 74.9997501483441) bank147790 +147791 POINT(41.09064386179499 73.06943468409483) bank147791 +147792 POINT(41.68301688423088 73.04106390074797) bank147792 +147793 POINT(41.30299814544106 73.62955151723897) bank147793 +147794 POINT(41.622786855773555 74.62934398176164) bank147794 +147795 POINT(40.54365117166258 74.56461363770961) bank147795 +147796 POINT(39.73670694427616 74.2532806810975) bank147796 +147797 POINT(40.74788413570529 74.24138368676381) bank147797 +147798 POINT(41.52353910725927 74.6430363247402) bank147798 +147799 POINT(39.868159512476 74.53418596655884) bank147799 +147800 POINT(40.92839558153579 74.67896029585314) bank147800 +147801 POINT(39.85429985977721 73.59925689407112) bank147801 +147802 POINT(40.094008524669434 73.57642182425118) bank147802 +147803 POINT(40.75500243110285 73.29987119146436) bank147803 +147804 POINT(41.49594825892477 73.43631771549462) bank147804 +147805 POINT(40.7037113412369 73.57030164887745) bank147805 +147806 POINT(40.75790542193093 74.65443646490202) bank147806 +147807 POINT(41.37536026364604 74.84991149930711) bank147807 +147808 POINT(41.48855183657128 73.2665733383201) bank147808 +147809 POINT(41.63773017807532 74.98456401690814) bank147809 +147810 POINT(40.31326438138934 74.53755238149814) bank147810 +147811 POINT(41.40127948217996 73.34769789936443) bank147811 +147812 POINT(40.12468492610417 74.05903204231788) bank147812 +147813 POINT(41.175488543523414 73.87933071510467) bank147813 +147814 POINT(40.452843115828834 74.10818770373865) bank147814 +147815 POINT(40.099794023731654 74.991013834574) bank147815 +147816 POINT(40.64965620633129 74.95429306453893) bank147816 +147817 POINT(41.29616373594587 73.07348934685325) bank147817 +147818 POINT(40.18137862297699 73.94019177482987) bank147818 +147819 POINT(41.385234076632564 73.58815904371885) bank147819 +147820 POINT(40.26782351067005 73.69627395149921) bank147820 +147821 POINT(40.32418899711933 74.53997112245311) bank147821 +147822 POINT(39.97401898763283 74.1648344525145) bank147822 +147823 POINT(41.532348794555446 73.62381291567975) bank147823 +147824 POINT(40.64336117488253 74.50454619738461) bank147824 +147825 POINT(40.781955609116075 73.1502667092295) bank147825 +147826 POINT(40.42439939227628 73.91891635109916) bank147826 +147827 POINT(41.104520016297364 73.85610842800314) bank147827 +147828 POINT(40.63890962578531 73.99205447012694) bank147828 +147829 POINT(41.67438445046203 74.03069748007451) bank147829 +147830 POINT(41.27841934145155 74.67542675774459) bank147830 +147831 POINT(41.37923937360895 73.75133926017908) bank147831 +147832 POINT(40.33727334680722 74.24218296145017) bank147832 +147833 POINT(41.15207139453944 73.33115958563987) bank147833 +147834 POINT(41.12368815000268 73.93621823060734) bank147834 +147835 POINT(41.23251254590835 74.85991898620158) bank147835 +147836 POINT(40.29214479688118 73.28004988624622) bank147836 +147837 POINT(41.06323024511265 74.6506815828018) bank147837 +147838 POINT(41.30132749026718 74.19680573270333) bank147838 +147839 POINT(40.85333500164464 73.54001615609124) bank147839 +147840 POINT(41.09255417249813 74.0032279870866) bank147840 +147841 POINT(40.20456191896683 73.8558470846487) bank147841 +147842 POINT(41.079837725682864 74.94069725080841) bank147842 +147843 POINT(40.996151121079336 73.0528455321468) bank147843 +147844 POINT(40.995879380376934 73.34232732936898) bank147844 +147845 POINT(40.90434469947768 74.0598607950889) bank147845 +147846 POINT(40.075645248898766 73.61597209013892) bank147846 +147847 POINT(40.70621345697716 73.18580689612394) bank147847 +147848 POINT(41.40916862169496 73.76311591498721) bank147848 +147849 POINT(40.787454151906026 74.7410696994555) bank147849 +147850 POINT(40.786239522350066 74.38794225479046) bank147850 +147851 POINT(40.27862565220358 74.85926803943728) bank147851 +147852 POINT(40.891068577596926 73.43833038906068) bank147852 +147853 POINT(39.769145195728356 74.12966773239901) bank147853 +147854 POINT(41.181952154162865 73.59172540460209) bank147854 +147855 POINT(41.46061031623289 73.68656928573695) bank147855 +147856 POINT(40.624486064533755 73.64616611885461) bank147856 +147857 POINT(41.08008904627903 73.9363388213137) bank147857 +147858 POINT(39.80732208474489 74.95318919538335) bank147858 +147859 POINT(40.31834788705247 73.05820370877835) bank147859 +147860 POINT(39.75436201833192 74.13368934345252) bank147860 +147861 POINT(41.10581430661819 73.59373740832585) bank147861 +147862 POINT(41.26969770874078 74.33601935739767) bank147862 +147863 POINT(40.83175851427718 73.26198757958534) bank147863 +147864 POINT(41.2256253809896 73.89415718431908) bank147864 +147865 POINT(40.95879030136471 73.16711121449529) bank147865 +147866 POINT(41.32889275393026 73.06949008351776) bank147866 +147867 POINT(41.01426447992005 74.87547276832866) bank147867 +147868 POINT(40.333427725144176 73.40057512219083) bank147868 +147869 POINT(41.52784088274594 74.65085077166098) bank147869 +147870 POINT(40.1237664464817 74.50679348169467) bank147870 +147871 POINT(39.87755928307291 73.8335291540325) bank147871 +147872 POINT(39.920134194811396 75.0019488457632) bank147872 +147873 POINT(41.62140247752993 73.84104690886439) bank147873 +147874 POINT(40.040458669084266 73.46002239863266) bank147874 +147875 POINT(40.58572923016982 74.4558227829496) bank147875 +147876 POINT(40.56117352397214 73.47413223668698) bank147876 +147877 POINT(40.771889910570856 73.37271295381693) bank147877 +147878 POINT(39.80904989701044 73.73301597385257) bank147878 +147879 POINT(40.74744539768335 73.1347947346945) bank147879 +147880 POINT(39.78315648036148 73.99342815244881) bank147880 +147881 POINT(40.545800297452864 73.41802710651463) bank147881 +147882 POINT(40.77538427537166 74.9012871107099) bank147882 +147883 POINT(41.56348634081871 73.35464031093063) bank147883 +147884 POINT(40.18156128465736 74.00597439730115) bank147884 +147885 POINT(40.63274677105055 73.99988439238693) bank147885 +147886 POINT(40.70859313395513 73.94707794582554) bank147886 +147887 POINT(41.385609208839924 74.73256943197515) bank147887 +147888 POINT(41.444846734727655 73.4572110687545) bank147888 +147889 POINT(40.35500840230001 73.99613334006894) bank147889 +147890 POINT(40.62825666384759 74.31482276227887) bank147890 +147891 POINT(40.08844456296248 74.84866981369179) bank147891 +147892 POINT(39.86594048532797 74.04638505583196) bank147892 +147893 POINT(40.03131834915554 73.18430286565437) bank147893 +147894 POINT(40.6966263422119 74.28501807399968) bank147894 +147895 POINT(40.98055435862624 73.41132525801042) bank147895 +147896 POINT(41.04131277279022 74.6866266852521) bank147896 +147897 POINT(41.00371104352601 73.75501216636526) bank147897 +147898 POINT(40.958631051761074 74.99557809594033) bank147898 +147899 POINT(41.70490950205547 73.49929497841697) bank147899 +147900 POINT(41.164086127464564 74.81685423626723) bank147900 +147901 POINT(40.476918913209765 73.01707786936711) bank147901 +147902 POINT(41.50417393038512 73.9761865388645) bank147902 +147903 POINT(40.24892039446705 73.49195275121963) bank147903 +147904 POINT(41.530310550987544 73.73134398766175) bank147904 +147905 POINT(40.75632240499235 74.80362252282602) bank147905 +147906 POINT(41.582581152994415 74.26674814719311) bank147906 +147907 POINT(40.41925611416033 74.81278946497774) bank147907 +147908 POINT(40.36134945150802 74.17141110018156) bank147908 +147909 POINT(41.494069328683956 73.71883888660815) bank147909 +147910 POINT(41.17244850827458 74.48589214471195) bank147910 +147911 POINT(41.046874985962376 73.35741391503377) bank147911 +147912 POINT(41.48434616413949 73.90473904388605) bank147912 +147913 POINT(41.10972876828451 73.78046596806738) bank147913 +147914 POINT(40.36747484688142 73.23304553811256) bank147914 +147915 POINT(41.69013607829222 74.38918131010617) bank147915 +147916 POINT(39.7402300186999 73.74184742155487) bank147916 +147917 POINT(40.35982657106763 74.03108585738566) bank147917 +147918 POINT(40.854767665985 73.97497527198601) bank147918 +147919 POINT(41.36259329560368 74.41253274100829) bank147919 +147920 POINT(39.76460660096522 73.44387441103854) bank147920 +147921 POINT(39.744105961921875 74.45573959952584) bank147921 +147922 POINT(40.575254493137734 74.1483060135687) bank147922 +147923 POINT(41.682674482948876 74.64271726269534) bank147923 +147924 POINT(41.64342804746013 74.77941860977758) bank147924 +147925 POINT(41.464129582154314 73.31387642700147) bank147925 +147926 POINT(40.510864713939355 74.36590760413651) bank147926 +147927 POINT(40.095687344100924 74.42540281444248) bank147927 +147928 POINT(40.074426955679925 73.23722086968463) bank147928 +147929 POINT(41.2735927411507 74.94368233706598) bank147929 +147930 POINT(41.112712016474575 73.02584813422341) bank147930 +147931 POINT(41.02718805226002 73.52739877552) bank147931 +147932 POINT(40.129275897670034 73.6456045438739) bank147932 +147933 POINT(41.57810907575477 73.27757013941027) bank147933 +147934 POINT(39.93582933530102 73.08687133275163) bank147934 +147935 POINT(41.03670850169104 74.20883427026908) bank147935 +147936 POINT(41.02998825288256 74.93248878281389) bank147936 +147937 POINT(40.452342262485864 73.69295594431306) bank147937 +147938 POINT(40.21003007022539 73.37173827155972) bank147938 +147939 POINT(40.521009009086136 74.18774237345015) bank147939 +147940 POINT(41.46452041207882 74.58752017976053) bank147940 +147941 POINT(40.48757957542187 74.16845202119212) bank147941 +147942 POINT(40.43182586320345 73.08247896163704) bank147942 +147943 POINT(39.799880531365176 74.9039972694324) bank147943 +147944 POINT(40.877534620229525 74.66123590500084) bank147944 +147945 POINT(41.19391011905923 74.87372663101404) bank147945 +147946 POINT(40.65568227653837 74.89310214224075) bank147946 +147947 POINT(40.47481244777209 74.68284020499705) bank147947 +147948 POINT(40.14335040482843 74.17429427740122) bank147948 +147949 POINT(41.619101169395215 73.74414464620449) bank147949 +147950 POINT(41.6113079963886 73.45422711524411) bank147950 +147951 POINT(41.18822649363886 73.95707504183234) bank147951 +147952 POINT(41.58057994080479 73.59518985362396) bank147952 +147953 POINT(40.20580149916724 74.90970264699438) bank147953 +147954 POINT(39.88169025756024 73.57996619304403) bank147954 +147955 POINT(39.85979557683727 73.63048011286716) bank147955 +147956 POINT(40.9444010084021 74.29805278756517) bank147956 +147957 POINT(39.91682965433444 73.0360643695146) bank147957 +147958 POINT(40.18374864682201 74.5898866418455) bank147958 +147959 POINT(41.433190856213386 74.4699897432472) bank147959 +147960 POINT(40.522496070733595 73.17689302992747) bank147960 +147961 POINT(41.297998891369105 73.34641230424398) bank147961 +147962 POINT(40.59386766668689 73.15623094622934) bank147962 +147963 POINT(41.625306571975415 73.90645319108208) bank147963 +147964 POINT(39.88341918185672 74.2899645285219) bank147964 +147965 POINT(40.292679438347506 73.73252239392875) bank147965 +147966 POINT(41.08148113126437 74.4007676480796) bank147966 +147967 POINT(39.82701234034692 73.44815021325509) bank147967 +147968 POINT(41.03123806376125 74.45396733695405) bank147968 +147969 POINT(41.252674343056086 74.71348140157312) bank147969 +147970 POINT(40.273473530390696 73.64790681478762) bank147970 +147971 POINT(41.2544934131776 73.62427207186205) bank147971 +147972 POINT(41.507102088189825 74.53910512360883) bank147972 +147973 POINT(41.19774418288647 74.42326472504348) bank147973 +147974 POINT(41.553680374296924 73.58103846542362) bank147974 +147975 POINT(40.43919744764758 74.82537385521762) bank147975 +147976 POINT(41.62340657206337 73.1857452987362) bank147976 +147977 POINT(40.95129788164981 73.92222909920547) bank147977 +147978 POINT(39.93440836297383 73.72385666521922) bank147978 +147979 POINT(41.697221806530145 73.92906538203363) bank147979 +147980 POINT(41.55616825572707 74.97270175353295) bank147980 +147981 POINT(40.087484912780596 74.87717375668697) bank147981 +147982 POINT(41.037776643567824 73.73176824276541) bank147982 +147983 POINT(41.23225356589503 74.25675453906234) bank147983 +147984 POINT(41.23532759502878 73.8042675647202) bank147984 +147985 POINT(41.44324855061184 73.13506607006263) bank147985 +147986 POINT(40.18797433698689 73.98301703194528) bank147986 +147987 POINT(40.53911897610385 73.3913490525082) bank147987 +147988 POINT(41.063743402320846 74.15189268468043) bank147988 +147989 POINT(40.33724442626744 73.78183186276515) bank147989 +147990 POINT(40.75582016667017 74.71025444242939) bank147990 +147991 POINT(41.58915186977343 74.66672127143866) bank147991 +147992 POINT(40.40740474985204 73.26996424124985) bank147992 +147993 POINT(41.00550441097396 73.75669027604151) bank147993 +147994 POINT(41.20783774730602 73.70920071693867) bank147994 +147995 POINT(40.37298283061896 74.0211744893464) bank147995 +147996 POINT(41.09728727595475 74.84694549484679) bank147996 +147997 POINT(39.99531742764568 73.4143597373891) bank147997 +147998 POINT(41.276059263251966 73.76514607623564) bank147998 +147999 POINT(40.57340724908682 74.09580557946627) bank147999 +148000 POINT(41.319967617532924 74.72951976084839) bank148000 +148001 POINT(41.056350454779064 73.78445485031172) bank148001 +148002 POINT(40.99649328906226 74.13841970843433) bank148002 +148003 POINT(40.32704610892681 74.29564680947965) bank148003 +148004 POINT(40.005112669445495 73.96818721813226) bank148004 +148005 POINT(39.98792384357081 74.53457852616334) bank148005 +148006 POINT(40.05383468439677 73.5398432616337) bank148006 +148007 POINT(39.92120954897889 74.00404596244414) bank148007 +148008 POINT(41.15880235967781 73.19165415630717) bank148008 +148009 POINT(41.38161662570605 74.61459014380371) bank148009 +148010 POINT(39.77617057476486 74.72304847309253) bank148010 +148011 POINT(41.70018474591011 74.92732817023112) bank148011 +148012 POINT(40.93975828888837 74.4814470498386) bank148012 +148013 POINT(39.98200883367195 73.16577048134522) bank148013 +148014 POINT(41.44252471662439 73.22745814904354) bank148014 +148015 POINT(40.821027665780065 74.01047663539555) bank148015 +148016 POINT(41.30975148001646 74.88695826265771) bank148016 +148017 POINT(41.104186678349244 74.18474768790274) bank148017 +148018 POINT(41.20061443784239 73.84886530366302) bank148018 +148019 POINT(41.103053390248206 73.26548889398191) bank148019 +148020 POINT(41.60016257658486 74.86653451495148) bank148020 +148021 POINT(40.09379123878032 73.1376554569505) bank148021 +148022 POINT(41.14559105314102 73.56357957987153) bank148022 +148023 POINT(40.84483959910464 73.43518778778636) bank148023 +148024 POINT(41.082368216436834 74.4879928761274) bank148024 +148025 POINT(40.45634814526984 73.09298663939059) bank148025 +148026 POINT(41.63820566329266 73.4783781931383) bank148026 +148027 POINT(40.769893715831515 74.93449453405155) bank148027 +148028 POINT(40.98323908538334 74.31296201110587) bank148028 +148029 POINT(40.43563112303284 74.837432938596) bank148029 +148030 POINT(40.730944695200996 74.44092919863138) bank148030 +148031 POINT(40.627216482294465 74.67131457502656) bank148031 +148032 POINT(40.51421204493007 74.90579087684871) bank148032 +148033 POINT(40.82069980970754 74.65040151073777) bank148033 +148034 POINT(40.33838257431005 73.74427160686885) bank148034 +148035 POINT(41.38832975226945 73.13750234965516) bank148035 +148036 POINT(40.21416066814934 74.23578391862586) bank148036 +148037 POINT(41.60527601206388 74.29192064837834) bank148037 +148038 POINT(40.37027502142808 74.33314336286455) bank148038 +148039 POINT(40.71392896053594 74.34175383984405) bank148039 +148040 POINT(40.38721930798626 73.59645732707429) bank148040 +148041 POINT(40.39223687302569 74.36440152590554) bank148041 +148042 POINT(40.473694723746554 73.01801850911814) bank148042 +148043 POINT(40.65846293831084 73.89556637370418) bank148043 +148044 POINT(41.29340801029852 74.9185040560628) bank148044 +148045 POINT(40.75574696263438 74.50772997198094) bank148045 +148046 POINT(41.32363814657956 74.7974884424192) bank148046 +148047 POINT(40.80171838758392 73.08130515140161) bank148047 +148048 POINT(40.54981437577099 73.8353786816605) bank148048 +148049 POINT(41.270429899826155 74.94588590036552) bank148049 +148050 POINT(40.444823353503665 73.60660280166397) bank148050 +148051 POINT(41.66408205906728 74.10551601241154) bank148051 +148052 POINT(41.29002912712345 74.1348536738994) bank148052 +148053 POINT(40.997694469256224 74.34272698380505) bank148053 +148054 POINT(40.25751242097669 74.60232843802399) bank148054 +148055 POINT(40.65376777826827 74.70706709214102) bank148055 +148056 POINT(41.31591810395659 73.6846570313115) bank148056 +148057 POINT(41.646322405451706 73.77127773491473) bank148057 +148058 POINT(41.017071915435096 74.91123087088424) bank148058 +148059 POINT(40.217200885646214 73.22190141500543) bank148059 +148060 POINT(40.131187425068596 74.68688695041274) bank148060 +148061 POINT(40.60609795500037 74.74077330116116) bank148061 +148062 POINT(41.392444073487496 73.3748239363057) bank148062 +148063 POINT(41.323597362461925 74.69559120627402) bank148063 +148064 POINT(41.19801413994683 73.04640245574485) bank148064 +148065 POINT(40.957778566515515 74.89283193466099) bank148065 +148066 POINT(39.99558845174897 73.70323972970388) bank148066 +148067 POINT(39.88863047968772 74.16808427490358) bank148067 +148068 POINT(41.412067950187826 74.29686198196887) bank148068 +148069 POINT(39.84901101584391 73.87791007195122) bank148069 +148070 POINT(40.218758063456576 74.7621838620291) bank148070 +148071 POINT(40.077891532535695 73.99323508104234) bank148071 +148072 POINT(40.54203567141315 74.35355985839067) bank148072 +148073 POINT(41.57004643686091 74.67371004148228) bank148073 +148074 POINT(39.83878776983292 74.75814927651157) bank148074 +148075 POINT(41.20886391089112 74.88473057152162) bank148075 +148076 POINT(40.506489456585804 74.90760444136883) bank148076 +148077 POINT(39.85794714991565 73.21196014819344) bank148077 +148078 POINT(41.075492998193596 74.16470968142727) bank148078 +148079 POINT(39.74418419884966 74.15454271787343) bank148079 +148080 POINT(40.16044300647043 73.14070434428982) bank148080 +148081 POINT(39.86029881328446 74.0443759553032) bank148081 +148082 POINT(41.48600473660822 74.93934789171742) bank148082 +148083 POINT(40.82702385805018 74.98790718132592) bank148083 +148084 POINT(40.51407038451416 74.33818944547666) bank148084 +148085 POINT(39.75043049813851 74.83671422349808) bank148085 +148086 POINT(41.38715182448056 73.48620883595443) bank148086 +148087 POINT(39.900327573882386 73.06228052050744) bank148087 +148088 POINT(41.20464697633633 73.4399271777096) bank148088 +148089 POINT(40.98883423787594 74.94538730135788) bank148089 +148090 POINT(40.58341248067311 74.97019285744658) bank148090 +148091 POINT(39.8887351754356 74.9314373653088) bank148091 +148092 POINT(39.951658408890005 74.00722369776538) bank148092 +148093 POINT(41.227395330778755 73.99505562794413) bank148093 +148094 POINT(41.360621414536745 73.65023904569503) bank148094 +148095 POINT(41.550090296067445 73.02938665222109) bank148095 +148096 POINT(40.44916986396908 73.8139542615068) bank148096 +148097 POINT(40.08330095900481 74.50267906922063) bank148097 +148098 POINT(40.1642239572373 74.07286059851326) bank148098 +148099 POINT(40.89135956504137 73.3226883761623) bank148099 +148100 POINT(40.48537500460732 73.40553472002719) bank148100 +148101 POINT(41.71039473777695 74.23853214612963) bank148101 +148102 POINT(39.788023673123426 74.61309259613435) bank148102 +148103 POINT(40.17240339718153 73.56718037830423) bank148103 +148104 POINT(40.4590942038948 74.70211777612714) bank148104 +148105 POINT(40.420832840763616 73.62976420684909) bank148105 +148106 POINT(39.77344272966645 73.87002531491652) bank148106 +148107 POINT(40.465301575244595 73.17720682596992) bank148107 +148108 POINT(40.06074859506328 73.08463464300438) bank148108 +148109 POINT(39.87384203193377 74.26799765066822) bank148109 +148110 POINT(40.870823401469416 74.04634434023751) bank148110 +148111 POINT(40.71078909225848 74.07584902414649) bank148111 +148112 POINT(41.16461923108948 73.9867229340113) bank148112 +148113 POINT(40.755099980794576 74.87057571271707) bank148113 +148114 POINT(41.32987963703571 74.96265214222896) bank148114 +148115 POINT(40.73023252082441 74.60522335803533) bank148115 +148116 POINT(40.75875972703914 74.22205636263149) bank148116 +148117 POINT(39.72719037924768 74.63730052978674) bank148117 +148118 POINT(40.99073525959178 73.84113421057299) bank148118 +148119 POINT(41.66922679184985 74.73336599706512) bank148119 +148120 POINT(41.42833363574303 74.74209263082618) bank148120 +148121 POINT(41.542381645905024 74.97464333294953) bank148121 +148122 POINT(39.838951202569135 74.98387540787621) bank148122 +148123 POINT(39.84968148222469 73.98260017568225) bank148123 +148124 POINT(41.45343956291947 73.16161444498408) bank148124 +148125 POINT(40.680813998566805 74.15013027694988) bank148125 +148126 POINT(39.95948888417445 73.42433492610414) bank148126 +148127 POINT(40.10961056917561 73.01096848561805) bank148127 +148128 POINT(40.063561623663375 74.33502760935197) bank148128 +148129 POINT(39.80566685077878 74.12693766656515) bank148129 +148130 POINT(40.91639393438339 73.29366773482063) bank148130 +148131 POINT(39.98469690955337 73.46294193927683) bank148131 +148132 POINT(40.22048220836713 74.8663665835039) bank148132 +148133 POINT(40.301886633979414 73.28044518846204) bank148133 +148134 POINT(40.44276340973866 73.54499982953041) bank148134 +148135 POINT(40.152271793382575 74.6884515215259) bank148135 +148136 POINT(41.01694507346561 74.55421418736606) bank148136 +148137 POINT(41.66370069543168 74.56357637544008) bank148137 +148138 POINT(41.50964303222902 74.44104675701118) bank148138 +148139 POINT(40.92921058237465 73.28924088004827) bank148139 +148140 POINT(41.189427133816416 73.2842805639438) bank148140 +148141 POINT(41.04292080174866 74.6758217770962) bank148141 +148142 POINT(41.08151280517196 73.18913807133562) bank148142 +148143 POINT(41.46034814768926 74.99644639375587) bank148143 +148144 POINT(41.305123437470726 74.04088713232156) bank148144 +148145 POINT(41.408233083999086 74.99296703696055) bank148145 +148146 POINT(41.125498589267 74.69663086415959) bank148146 +148147 POINT(41.64474218669126 74.7112257077388) bank148147 +148148 POINT(40.36800433831662 74.00567996788153) bank148148 +148149 POINT(41.432695988342466 73.96488423670202) bank148149 +148150 POINT(40.36116537500216 73.695599002664) bank148150 +148151 POINT(39.72124630460283 74.67965237276466) bank148151 +148152 POINT(40.13847442196296 73.38839398344365) bank148152 +148153 POINT(40.86875399928492 73.05989783155003) bank148153 +148154 POINT(40.05676600217569 74.72632931862438) bank148154 +148155 POINT(41.32939558881822 74.57534921441915) bank148155 +148156 POINT(40.35911458067111 74.58452918421393) bank148156 +148157 POINT(40.32608873519251 73.38489126795355) bank148157 +148158 POINT(40.45059811120077 73.2078800400738) bank148158 +148159 POINT(41.55172982786538 74.70653167411504) bank148159 +148160 POINT(40.61415987221193 74.44583389776211) bank148160 +148161 POINT(40.85422788521583 73.63473185597921) bank148161 +148162 POINT(41.66556394139642 73.88812291415408) bank148162 +148163 POINT(41.350528124599556 73.54155524549091) bank148163 +148164 POINT(41.27456978631903 74.79504767304795) bank148164 +148165 POINT(40.38804134440418 74.87571079465381) bank148165 +148166 POINT(40.93290843567366 74.29773844598503) bank148166 +148167 POINT(40.515789491390315 74.38510518126299) bank148167 +148168 POINT(40.95873591047821 73.41014295184206) bank148168 +148169 POINT(41.516497285447976 73.38147446876754) bank148169 +148170 POINT(41.16092377413043 73.15926391849219) bank148170 +148171 POINT(41.65743715453276 74.52931721077204) bank148171 +148172 POINT(40.20719758300571 73.25261021751942) bank148172 +148173 POINT(40.16596336317791 74.81180302321519) bank148173 +148174 POINT(39.764228511752 73.84878426958022) bank148174 +148175 POINT(40.918719615847685 74.69484582260519) bank148175 +148176 POINT(40.685472781076456 74.86757208270319) bank148176 +148177 POINT(41.03525688905834 74.45520237087881) bank148177 +148178 POINT(41.070848901630654 74.11080881845847) bank148178 +148179 POINT(41.34164892212414 74.80656729126501) bank148179 +148180 POINT(41.612748858392514 73.78683392690792) bank148180 +148181 POINT(41.191459416572044 74.44903622185909) bank148181 +148182 POINT(41.56873099571807 74.55508809355554) bank148182 +148183 POINT(40.8806620922178 74.5131397329912) bank148183 +148184 POINT(40.641047085972204 74.88569625055635) bank148184 +148185 POINT(41.127457003034515 74.63235556330461) bank148185 +148186 POINT(41.32241450035725 74.33130262092253) bank148186 +148187 POINT(40.87989671836021 74.34232688623102) bank148187 +148188 POINT(41.31850077405693 74.16925659866486) bank148188 +148189 POINT(41.39980077303102 73.27121076980944) bank148189 +148190 POINT(40.207850616174525 74.97768008613991) bank148190 +148191 POINT(39.74020106607554 73.30248183140401) bank148191 +148192 POINT(40.018468887609586 74.81051657463942) bank148192 +148193 POINT(40.9309832939055 73.0339514902608) bank148193 +148194 POINT(39.87148637292037 73.2134953990553) bank148194 +148195 POINT(41.404689414567635 73.62272779733746) bank148195 +148196 POINT(41.171500101470954 74.98186863751623) bank148196 +148197 POINT(39.71792665176149 74.39612646552747) bank148197 +148198 POINT(41.4146486803651 74.94107441956986) bank148198 +148199 POINT(39.78795582173049 73.80924253071589) bank148199 +148200 POINT(40.40641089550858 74.75321255086438) bank148200 +148201 POINT(41.233503511606884 73.82674300357701) bank148201 +148202 POINT(39.89008689926982 73.26419906834796) bank148202 +148203 POINT(40.81851677619073 73.68002624121584) bank148203 +148204 POINT(41.29410578800359 73.53322249779278) bank148204 +148205 POINT(40.114814518610586 74.59702798021232) bank148205 +148206 POINT(40.11199499275427 74.02894141142278) bank148206 +148207 POINT(40.46101414110194 74.86245730593896) bank148207 +148208 POINT(41.649909575252416 73.39957009141814) bank148208 +148209 POINT(41.36803007946019 73.62131936057227) bank148209 +148210 POINT(41.66013126018993 73.59245665943655) bank148210 +148211 POINT(40.35410516990805 73.5141078748956) bank148211 +148212 POINT(41.38108152972623 73.5538535301505) bank148212 +148213 POINT(41.29032169687174 73.36662098189313) bank148213 +148214 POINT(40.12383153178916 74.46372772100364) bank148214 +148215 POINT(40.55746832943811 73.01388569924785) bank148215 +148216 POINT(40.051614200992546 74.44808282200219) bank148216 +148217 POINT(39.81195925623248 74.18590303949655) bank148217 +148218 POINT(39.961960950517884 74.81058542798361) bank148218 +148219 POINT(39.95316356129959 74.50712791310546) bank148219 +148220 POINT(39.89474989139563 74.47849356147275) bank148220 +148221 POINT(40.883024789329056 74.83604890503854) bank148221 +148222 POINT(40.28352342736779 74.01992663663725) bank148222 +148223 POINT(41.01925444774517 74.0255849106573) bank148223 +148224 POINT(40.48906927233133 74.67158233277176) bank148224 +148225 POINT(39.91022647241917 74.38512598412971) bank148225 +148226 POINT(41.24595000725967 74.2073483489026) bank148226 +148227 POINT(40.1734332779887 74.71571276840811) bank148227 +148228 POINT(41.57368179168118 73.83452080170822) bank148228 +148229 POINT(41.55747475899004 73.32620145809604) bank148229 +148230 POINT(39.77684491147441 74.33229965544908) bank148230 +148231 POINT(39.871675097231524 73.9131319594849) bank148231 +148232 POINT(40.28779458920527 73.09330721466976) bank148232 +148233 POINT(41.329027196402066 73.6824001889785) bank148233 +148234 POINT(41.67242529100987 73.95594820128515) bank148234 +148235 POINT(41.388930773906395 74.24081685717495) bank148235 +148236 POINT(41.09963561593247 74.36287370623131) bank148236 +148237 POINT(39.891806490905296 74.61729683650944) bank148237 +148238 POINT(40.44914073827961 73.1343467275688) bank148238 +148239 POINT(41.062048087298905 73.47493224938239) bank148239 +148240 POINT(39.795680669975084 73.99002796413204) bank148240 +148241 POINT(40.78231251044282 73.25925204230329) bank148241 +148242 POINT(40.5640476439625 73.90365060996885) bank148242 +148243 POINT(40.261637654619655 74.79509420062014) bank148243 +148244 POINT(39.9124064267264 74.67253194646808) bank148244 +148245 POINT(40.02199642288062 73.7532966596561) bank148245 +148246 POINT(40.49079147124222 74.58661844639715) bank148246 +148247 POINT(40.34022632921656 74.90366607930837) bank148247 +148248 POINT(41.26066162750577 74.55377247730135) bank148248 +148249 POINT(40.518035270579716 73.46910498802183) bank148249 +148250 POINT(40.0295256762936 73.2928385853044) bank148250 +148251 POINT(41.30478539541773 74.72639543430309) bank148251 +148252 POINT(40.03368943762129 73.05143659943855) bank148252 +148253 POINT(41.60877624767688 74.05160091323178) bank148253 +148254 POINT(41.446336924283585 73.26249298345454) bank148254 +148255 POINT(40.46486479069854 73.31393362238036) bank148255 +148256 POINT(41.60123783369629 74.4321313605808) bank148256 +148257 POINT(40.49335302443009 74.2494878485186) bank148257 +148258 POINT(40.76003568077276 74.95767573593027) bank148258 +148259 POINT(41.01185733486005 73.17320968229507) bank148259 +148260 POINT(40.95040076215646 74.30165805355406) bank148260 +148261 POINT(39.79796283182116 73.65189354891581) bank148261 +148262 POINT(40.32039373929959 73.69000109711845) bank148262 +148263 POINT(40.25162972258498 74.62972019723895) bank148263 +148264 POINT(40.11015379711408 73.49479027865605) bank148264 +148265 POINT(41.14756977014885 74.87135772671526) bank148265 +148266 POINT(41.60537121767027 73.09213009993586) bank148266 +148267 POINT(41.610331147974094 73.58611231756153) bank148267 +148268 POINT(40.91012638509932 73.75443090183347) bank148268 +148269 POINT(40.19921235737312 73.05972402809532) bank148269 +148270 POINT(41.63528808410619 74.53873561516066) bank148270 +148271 POINT(40.41952837108931 73.95849835956005) bank148271 +148272 POINT(41.58260891872587 74.55652744810317) bank148272 +148273 POINT(40.390641439862506 73.82728569246316) bank148273 +148274 POINT(41.62176754141716 74.21773891867348) bank148274 +148275 POINT(40.44875761515984 73.90281230896441) bank148275 +148276 POINT(41.521902757409244 73.06265409439061) bank148276 +148277 POINT(40.29355525764442 73.65724068702038) bank148277 +148278 POINT(41.37416021705409 74.02455763434112) bank148278 +148279 POINT(40.919157253325345 73.293432623146) bank148279 +148280 POINT(41.37434837064021 74.37891156815292) bank148280 +148281 POINT(41.10081364154558 73.14275525188003) bank148281 +148282 POINT(40.903368616051694 73.92472890081504) bank148282 +148283 POINT(41.12807646982862 73.55445706675219) bank148283 +148284 POINT(40.18945601456139 74.63900976269574) bank148284 +148285 POINT(41.267533090458485 74.67701697395982) bank148285 +148286 POINT(40.09285856574264 73.56459359036509) bank148286 +148287 POINT(41.422136926360146 74.35450877596378) bank148287 +148288 POINT(40.8707219341482 73.74810785780274) bank148288 +148289 POINT(40.75483604735405 74.85095617222076) bank148289 +148290 POINT(41.444630070335734 73.35512172131195) bank148290 +148291 POINT(39.95824795622481 74.31233699752406) bank148291 +148292 POINT(40.481755516026524 73.54137199172301) bank148292 +148293 POINT(40.83094442014458 74.2344839159136) bank148293 +148294 POINT(40.73398804614325 74.88782018853045) bank148294 +148295 POINT(39.996689653639635 74.97986131675617) bank148295 +148296 POINT(40.33517651382642 74.9265355389367) bank148296 +148297 POINT(41.18898547362017 73.83005057809498) bank148297 +148298 POINT(40.69503625968374 73.75916465717214) bank148298 +148299 POINT(41.06436714073631 73.42233145035446) bank148299 +148300 POINT(40.141407298130865 73.09938371996206) bank148300 +148301 POINT(40.4109555496953 74.43271817154604) bank148301 +148302 POINT(40.54351579964699 73.41697443678949) bank148302 +148303 POINT(40.172764718390205 73.85263874348477) bank148303 +148304 POINT(40.87511997630296 74.55842202610094) bank148304 +148305 POINT(40.577757405149725 74.22865628169816) bank148305 +148306 POINT(40.057941189180376 73.83243698378912) bank148306 +148307 POINT(39.845397211267006 73.46312159300118) bank148307 +148308 POINT(40.76699347282749 73.65956447071822) bank148308 +148309 POINT(40.98122076556274 74.9200967353057) bank148309 +148310 POINT(41.22969660391753 74.67350662440353) bank148310 +148311 POINT(40.9056899521291 74.56938670543373) bank148311 +148312 POINT(41.21922424059245 73.34121722936237) bank148312 +148313 POINT(40.579635799816785 74.90159233911717) bank148313 +148314 POINT(40.60131477093495 73.65941749982429) bank148314 +148315 POINT(41.33052659188927 74.61214283144774) bank148315 +148316 POINT(40.857048782746205 73.38887550625708) bank148316 +148317 POINT(39.929115868220215 73.02495500654204) bank148317 +148318 POINT(41.42029928131461 74.57276771822815) bank148318 +148319 POINT(40.584515278206744 73.52596995750866) bank148319 +148320 POINT(40.588148575990786 74.80869603111054) bank148320 +148321 POINT(41.03419541578648 73.84616933705952) bank148321 +148322 POINT(41.23624992632422 74.34681902778267) bank148322 +148323 POINT(41.111351568334804 74.71250025133949) bank148323 +148324 POINT(39.77406757783738 74.18862285478632) bank148324 +148325 POINT(40.6862947379269 73.94414607545777) bank148325 +148326 POINT(40.89546418587507 74.58915745545632) bank148326 +148327 POINT(40.45406824272679 74.68920411048396) bank148327 +148328 POINT(40.73914509872033 74.62927800389068) bank148328 +148329 POINT(41.46140695287062 74.03123187352952) bank148329 +148330 POINT(39.72407774119507 74.44862777517835) bank148330 +148331 POINT(39.95535750641119 73.20800547683334) bank148331 +148332 POINT(40.01790465835296 73.16628251125968) bank148332 +148333 POINT(41.70956486861718 74.4592222536303) bank148333 +148334 POINT(41.158099219495696 74.2255462845179) bank148334 +148335 POINT(41.36585135591651 74.0460420568699) bank148335 +148336 POINT(40.28045809314517 74.25223954605538) bank148336 +148337 POINT(41.20844675983552 74.58214147861388) bank148337 +148338 POINT(40.962394177882224 73.87255267762579) bank148338 +148339 POINT(39.88618422090715 73.1507546577044) bank148339 +148340 POINT(41.35841433061689 73.8475597378843) bank148340 +148341 POINT(40.331322025766475 73.34572427349441) bank148341 +148342 POINT(40.28598204990857 74.7705024961028) bank148342 +148343 POINT(41.4037779212871 74.46544566766848) bank148343 +148344 POINT(40.83664535632651 74.5487894282374) bank148344 +148345 POINT(40.35029260189407 74.15166330732139) bank148345 +148346 POINT(40.87257915346306 73.06220808912961) bank148346 +148347 POINT(39.79803728699731 73.5089835444942) bank148347 +148348 POINT(40.411631854603776 73.04299199939238) bank148348 +148349 POINT(40.602869149088534 74.39307015856824) bank148349 +148350 POINT(39.98436114236272 74.27594087451277) bank148350 +148351 POINT(41.60724282472685 74.06962316345884) bank148351 +148352 POINT(39.788778808440775 74.14706109386655) bank148352 +148353 POINT(40.236701283114655 74.07340096402764) bank148353 +148354 POINT(41.33463251984963 74.67413000014835) bank148354 +148355 POINT(41.45329309022104 73.90762333329644) bank148355 +148356 POINT(40.66637836014791 73.61002858033515) bank148356 +148357 POINT(40.65308007445431 74.76661120584062) bank148357 +148358 POINT(41.58469903826879 74.78914870092333) bank148358 +148359 POINT(40.30994728402807 73.25209364075805) bank148359 +148360 POINT(41.63132224017035 74.9480364653393) bank148360 +148361 POINT(40.673754668447074 74.38825261190279) bank148361 +148362 POINT(40.492628418515984 73.8731292389335) bank148362 +148363 POINT(40.517037048228694 73.26261822264416) bank148363 +148364 POINT(41.52507994359435 73.94735382179644) bank148364 +148365 POINT(40.47036466548482 73.32429813613712) bank148365 +148366 POINT(41.12953676792903 73.34109830162366) bank148366 +148367 POINT(41.41722777953174 73.4887605314941) bank148367 +148368 POINT(39.830242452060546 74.44397172436557) bank148368 +148369 POINT(41.237871533102705 73.20907757963695) bank148369 +148370 POINT(40.00079975405249 73.17038739239064) bank148370 +148371 POINT(40.35225056137145 74.71619469643285) bank148371 +148372 POINT(41.5271492898856 73.68426294744697) bank148372 +148373 POINT(40.08261622431792 74.50971829332941) bank148373 +148374 POINT(40.823995112199405 73.5715790574898) bank148374 +148375 POINT(40.43909695091788 74.93968025428363) bank148375 +148376 POINT(40.108518093641536 73.24179631325916) bank148376 +148377 POINT(40.5213586363525 74.05519043910779) bank148377 +148378 POINT(40.654035412796404 73.23990332962273) bank148378 +148379 POINT(41.184518667221944 73.65410393452647) bank148379 +148380 POINT(41.05621034055222 74.65656055916459) bank148380 +148381 POINT(41.5700857924141 74.07280796764493) bank148381 +148382 POINT(41.03684352353548 73.19818584986413) bank148382 +148383 POINT(40.22020086254433 73.65934888938438) bank148383 +148384 POINT(41.13197566857144 74.9496163325184) bank148384 +148385 POINT(41.25144213530882 73.11881814937401) bank148385 +148386 POINT(39.98955164190587 73.57080414952516) bank148386 +148387 POINT(40.24881828168214 74.28054775465745) bank148387 +148388 POINT(41.36294449307422 74.31925314368257) bank148388 +148389 POINT(41.30245914874836 73.98432347437168) bank148389 +148390 POINT(41.4070874312893 73.31462275821312) bank148390 +148391 POINT(40.28151869188436 74.13046337297705) bank148391 +148392 POINT(41.55722573798043 73.60573975658794) bank148392 +148393 POINT(40.098785800280524 73.86779965628561) bank148393 +148394 POINT(40.80799038459595 74.98025339325477) bank148394 +148395 POINT(41.54288807760764 73.54800566530513) bank148395 +148396 POINT(41.38897367178753 73.67630972646265) bank148396 +148397 POINT(40.8158369129372 73.08914067306038) bank148397 +148398 POINT(41.23189583800454 73.21463795858992) bank148398 +148399 POINT(41.50427117650331 74.81243512578456) bank148399 +148400 POINT(40.95085066406626 74.46008044166082) bank148400 +148401 POINT(40.237706220143096 73.5895792291015) bank148401 +148402 POINT(41.388258090275734 74.81672944223862) bank148402 +148403 POINT(39.7385345082465 74.70343002131953) bank148403 +148404 POINT(41.51873004941332 73.52509099036077) bank148404 +148405 POINT(40.0424354586044 73.62901016313963) bank148405 +148406 POINT(40.7509530859764 73.30447387070251) bank148406 +148407 POINT(39.95271412072045 74.70917098851906) bank148407 +148408 POINT(40.67562315036473 73.21659577491398) bank148408 +148409 POINT(40.189182057265256 73.87121486470684) bank148409 +148410 POINT(40.696557507128375 74.75795884418694) bank148410 +148411 POINT(39.95531414540504 73.68429985377884) bank148411 +148412 POINT(40.35125378659348 73.07225597387375) bank148412 +148413 POINT(39.755857993728554 73.36388413338919) bank148413 +148414 POINT(40.90476602163586 73.42950626050731) bank148414 +148415 POINT(41.62485761326202 74.73719170330176) bank148415 +148416 POINT(41.35031896906886 74.27734289250604) bank148416 +148417 POINT(40.158626583086345 74.33584044209171) bank148417 +148418 POINT(40.04441134543566 74.47306290989786) bank148418 +148419 POINT(40.76139348629095 73.35806291847848) bank148419 +148420 POINT(41.21729217889805 73.67873335436215) bank148420 +148421 POINT(41.253885045217196 73.41813025775069) bank148421 +148422 POINT(40.84767810532281 74.98141876801857) bank148422 +148423 POINT(41.441985136559154 74.82765341954416) bank148423 +148424 POINT(40.57600200687361 74.55796605332407) bank148424 +148425 POINT(40.53836146711715 73.74164561620172) bank148425 +148426 POINT(40.72783601448175 74.41056760513901) bank148426 +148427 POINT(41.00795932910005 74.77702308537809) bank148427 +148428 POINT(40.48657418800866 74.88928154542342) bank148428 +148429 POINT(41.11054504270873 74.20686541219823) bank148429 +148430 POINT(41.32135366166996 73.64325057634355) bank148430 +148431 POINT(39.91368346456473 74.65328525550687) bank148431 +148432 POINT(39.88535400347774 73.03526374820302) bank148432 +148433 POINT(39.88441806238812 73.41061317169445) bank148433 +148434 POINT(40.73444456051671 73.398158648193) bank148434 +148435 POINT(39.95789926091196 74.19990052527416) bank148435 +148436 POINT(41.28484176355197 73.58512946658281) bank148436 +148437 POINT(40.03324239863472 73.49261563659962) bank148437 +148438 POINT(40.55881214806568 73.136403117197) bank148438 +148439 POINT(39.83085847356199 73.80438762079716) bank148439 +148440 POINT(40.66554957275912 74.12534684142791) bank148440 +148441 POINT(41.45536254283363 73.41282871618017) bank148441 +148442 POINT(40.2915028737092 73.46919585197557) bank148442 +148443 POINT(40.157713172779566 73.89168664082626) bank148443 +148444 POINT(40.21598599793993 73.76444766228362) bank148444 +148445 POINT(40.39312316781888 74.54118813398489) bank148445 +148446 POINT(41.207571233190144 73.0212358004281) bank148446 +148447 POINT(40.35417108774381 74.6602819700461) bank148447 +148448 POINT(41.18698708635291 74.13207964748159) bank148448 +148449 POINT(40.84681571618635 73.82611988739393) bank148449 +148450 POINT(39.983308411195054 74.75593228653335) bank148450 +148451 POINT(40.13448253544302 73.76944629575196) bank148451 +148452 POINT(41.67150085002262 74.50303245120095) bank148452 +148453 POINT(40.57667541224868 73.52533174040418) bank148453 +148454 POINT(39.81985528416882 73.46054560444612) bank148454 +148455 POINT(40.138145014379326 73.27191673060152) bank148455 +148456 POINT(40.49951561010312 73.77870984983589) bank148456 +148457 POINT(40.92916118402352 74.2567288337672) bank148457 +148458 POINT(39.81476895583429 74.16761689567339) bank148458 +148459 POINT(40.54464936287518 73.73877894805744) bank148459 +148460 POINT(40.95166239762481 74.60311598325734) bank148460 +148461 POINT(39.8667281707496 74.52772886810673) bank148461 +148462 POINT(40.86597401773153 74.0958141376214) bank148462 +148463 POINT(39.98167473905469 73.71711552575836) bank148463 +148464 POINT(40.213533015213756 73.33228498550184) bank148464 +148465 POINT(40.59931810386807 73.59814696413663) bank148465 +148466 POINT(40.196787874758634 74.61217751229658) bank148466 +148467 POINT(41.389123090584604 73.27652669333993) bank148467 +148468 POINT(40.90864908178989 73.33066291735456) bank148468 +148469 POINT(41.47634161099221 74.3784280202399) bank148469 +148470 POINT(40.72956154918893 74.18815036399778) bank148470 +148471 POINT(41.21745382051105 73.90400699371203) bank148471 +148472 POINT(40.76759065485557 74.9433242468847) bank148472 +148473 POINT(40.62126830798068 73.97055494887339) bank148473 +148474 POINT(41.057730213926774 73.84428020931861) bank148474 +148475 POINT(39.80139320086472 73.25547068621422) bank148475 +148476 POINT(41.078058021225075 74.1184661519635) bank148476 +148477 POINT(40.57795761279227 74.42468357249254) bank148477 +148478 POINT(39.804992225786016 73.53567693223894) bank148478 +148479 POINT(41.28629285752654 73.68912577493629) bank148479 +148480 POINT(41.494166094705136 74.2545887127561) bank148480 +148481 POINT(39.994100127334356 73.77361815784819) bank148481 +148482 POINT(39.82319937192287 74.98614032081116) bank148482 +148483 POINT(40.36743807210459 73.80622422772336) bank148483 +148484 POINT(41.14942938469137 74.87220274134118) bank148484 +148485 POINT(40.54073384756065 73.47254905369236) bank148485 +148486 POINT(40.82465388513515 73.68684161760548) bank148486 +148487 POINT(40.32736773520828 73.44346116859013) bank148487 +148488 POINT(40.059182605672916 73.50353727738558) bank148488 +148489 POINT(40.958156627605575 73.54975484938208) bank148489 +148490 POINT(40.92874045505741 74.94348415346565) bank148490 +148491 POINT(39.82045988693795 73.2150932004813) bank148491 +148492 POINT(41.58613661044494 74.03924806584908) bank148492 +148493 POINT(40.21424292720979 74.6862683768358) bank148493 +148494 POINT(41.345763523922685 73.58182427531727) bank148494 +148495 POINT(41.55112827753298 73.62451655220602) bank148495 +148496 POINT(41.40018721953614 73.33897156482058) bank148496 +148497 POINT(40.60548340780102 74.74231927464658) bank148497 +148498 POINT(39.85477948741634 73.88664633648496) bank148498 +148499 POINT(41.196626632040775 74.95986713118855) bank148499 +148500 POINT(40.7793701748958 74.22116038083347) bank148500 +148501 POINT(40.50667110439651 74.53374416295655) bank148501 +148502 POINT(39.718599105793615 73.34742152613202) bank148502 +148503 POINT(41.16009801415964 73.10572555932036) bank148503 +148504 POINT(40.93972205997883 74.71251737863356) bank148504 +148505 POINT(41.477343964713256 74.70202253313646) bank148505 +148506 POINT(41.30290916188778 74.5780501925497) bank148506 +148507 POINT(40.617709472922144 73.78167002577977) bank148507 +148508 POINT(40.204498307908196 74.79759168883595) bank148508 +148509 POINT(40.98581490637035 74.34657620761212) bank148509 +148510 POINT(40.43087893072327 73.24974637938413) bank148510 +148511 POINT(40.28407337879758 74.88844582233499) bank148511 +148512 POINT(41.67741348159746 73.39216451898275) bank148512 +148513 POINT(41.058275842545356 73.91854780967398) bank148513 +148514 POINT(41.08237181101414 73.15148405245054) bank148514 +148515 POINT(40.06195706084775 73.99244419235185) bank148515 +148516 POINT(40.69023516362149 73.15616501734532) bank148516 +148517 POINT(40.02523558107209 74.14361193436713) bank148517 +148518 POINT(41.33108865077771 73.33270552838351) bank148518 +148519 POINT(40.99378949537274 73.43933485509281) bank148519 +148520 POINT(40.33060910035031 73.60528582753142) bank148520 +148521 POINT(40.58198819057943 74.49918441044294) bank148521 +148522 POINT(39.78796520356714 73.0493149548803) bank148522 +148523 POINT(41.35669384478363 73.366637516865) bank148523 +148524 POINT(40.9254202775499 74.05244110416548) bank148524 +148525 POINT(41.622402494590105 74.87968846063413) bank148525 +148526 POINT(40.8154258271049 74.38869366005152) bank148526 +148527 POINT(40.85411981707881 73.16120211878498) bank148527 +148528 POINT(40.95885907246959 73.62747993735617) bank148528 +148529 POINT(41.22595689783464 74.80180242575982) bank148529 +148530 POINT(40.12148723674191 73.31401629545802) bank148530 +148531 POINT(41.31102983269474 73.79227495254585) bank148531 +148532 POINT(40.8761397046158 73.59333545672992) bank148532 +148533 POINT(39.884298285901295 74.8172299035118) bank148533 +148534 POINT(41.47316903128357 74.5536823394727) bank148534 +148535 POINT(40.400919473702594 74.11362885736571) bank148535 +148536 POINT(40.72474492652779 73.70309159817842) bank148536 +148537 POINT(40.20759940704871 74.60947874175302) bank148537 +148538 POINT(40.86723272261923 73.63326429286866) bank148538 +148539 POINT(41.374525795336474 74.55472778223928) bank148539 +148540 POINT(40.627635716388966 74.56328405956931) bank148540 +148541 POINT(41.25032637662639 73.34458930579237) bank148541 +148542 POINT(39.99792251957771 74.99206334818967) bank148542 +148543 POINT(40.69349436650696 73.56957334551204) bank148543 +148544 POINT(40.61567850618016 74.50290451120412) bank148544 +148545 POINT(40.89298103950018 74.5070844929856) bank148545 +148546 POINT(40.79851409954122 74.57635658174888) bank148546 +148547 POINT(40.1745394950585 74.39359935198695) bank148547 +148548 POINT(41.2445294886194 73.03961708709939) bank148548 +148549 POINT(41.24126760127111 73.65482014332994) bank148549 +148550 POINT(40.649050438254505 74.84448550412618) bank148550 +148551 POINT(40.01675110794129 73.8722263725496) bank148551 +148552 POINT(40.515959879569195 73.84077489926487) bank148552 +148553 POINT(41.116356906111605 73.59617241779364) bank148553 +148554 POINT(41.609493044177704 73.8076724456033) bank148554 +148555 POINT(40.71235715444317 74.15098826779028) bank148555 +148556 POINT(40.80212739448482 73.10497970890567) bank148556 +148557 POINT(40.16035338184869 73.18858578144645) bank148557 +148558 POINT(41.395037689886514 74.20459495449593) bank148558 +148559 POINT(40.375636854379415 74.85634677597395) bank148559 +148560 POINT(41.24162705304592 73.66561581957761) bank148560 +148561 POINT(39.93723127355455 74.49369275437842) bank148561 +148562 POINT(40.72346528008277 73.73605828588741) bank148562 +148563 POINT(41.33274579097165 74.310413638674) bank148563 +148564 POINT(40.934270003271884 74.4310496984376) bank148564 +148565 POINT(40.329309441475445 74.18810079954554) bank148565 +148566 POINT(40.35516513957703 73.05760806531289) bank148566 +148567 POINT(41.471857024167676 74.3716859607891) bank148567 +148568 POINT(41.06889150381473 74.84868349555767) bank148568 +148569 POINT(40.533636520238176 73.42489019246766) bank148569 +148570 POINT(41.17311675472965 73.05080399295726) bank148570 +148571 POINT(40.91427418197991 74.82955057275277) bank148571 +148572 POINT(39.80095665497261 73.13221908925503) bank148572 +148573 POINT(41.43849705427156 74.15675512146504) bank148573 +148574 POINT(40.20528225665923 74.69967341196468) bank148574 +148575 POINT(40.11515034828894 74.45616898788128) bank148575 +148576 POINT(41.67123359351258 74.94576451086567) bank148576 +148577 POINT(41.037778784580944 74.81699801451954) bank148577 +148578 POINT(39.713574659785515 74.59955399769939) bank148578 +148579 POINT(39.871505709658685 74.01411075126066) bank148579 +148580 POINT(39.91741045202939 73.38301611901804) bank148580 +148581 POINT(39.92640147002371 73.91191901077482) bank148581 +148582 POINT(40.726444009914346 74.2210022875344) bank148582 +148583 POINT(40.230601147574426 74.2189545056667) bank148583 +148584 POINT(40.53627757630627 74.49239807510536) bank148584 +148585 POINT(41.488568809132815 74.79774706216377) bank148585 +148586 POINT(41.03750699102074 73.67430659500914) bank148586 +148587 POINT(39.84744259000514 74.18774822273521) bank148587 +148588 POINT(41.56828285175153 74.51697715727795) bank148588 +148589 POINT(41.58045243301502 73.95539306591354) bank148589 +148590 POINT(40.651503136236904 73.47948338641366) bank148590 +148591 POINT(41.412813532424664 74.61458539552963) bank148591 +148592 POINT(41.496000199225946 74.13229459435489) bank148592 +148593 POINT(40.98814805219991 74.33576395612054) bank148593 +148594 POINT(40.019390640007124 73.13946222216717) bank148594 +148595 POINT(40.53580999808049 73.42425112816956) bank148595 +148596 POINT(40.92268404804458 74.29926905308614) bank148596 +148597 POINT(40.38280807395905 73.95957926726196) bank148597 +148598 POINT(40.8704392195461 74.14284009333603) bank148598 +148599 POINT(41.17105008409369 74.72952874163737) bank148599 +148600 POINT(40.50577058441661 73.62145446876123) bank148600 +148601 POINT(41.13963805675485 74.40363026466322) bank148601 +148602 POINT(40.910590193741065 74.92567553675258) bank148602 +148603 POINT(39.75879908984977 73.53336929503718) bank148603 +148604 POINT(41.50545319577816 74.97017156996029) bank148604 +148605 POINT(39.74948850154609 74.84909966280406) bank148605 +148606 POINT(40.94860737364648 73.48174971909052) bank148606 +148607 POINT(41.218090573841714 74.55687874869061) bank148607 +148608 POINT(40.93744617485132 74.16357460103396) bank148608 +148609 POINT(41.54360271854276 73.85556598133128) bank148609 +148610 POINT(40.45698584826032 73.96836113555308) bank148610 +148611 POINT(40.20704761512601 74.50214349827208) bank148611 +148612 POINT(40.71218093390731 74.64321596779524) bank148612 +148613 POINT(41.65637081422001 73.84259644490335) bank148613 +148614 POINT(41.62626875202964 73.78452777238141) bank148614 +148615 POINT(40.975392920492475 74.6729496592164) bank148615 +148616 POINT(39.76865701911268 74.13602416752788) bank148616 +148617 POINT(40.12568364484299 73.17902945893272) bank148617 +148618 POINT(41.59084587475001 73.34591306018385) bank148618 +148619 POINT(39.73878730216622 73.98687158128091) bank148619 +148620 POINT(41.704757619850625 73.50008633602472) bank148620 +148621 POINT(41.50852300783607 74.54582429111217) bank148621 +148622 POINT(39.87646328569496 74.02431055100207) bank148622 +148623 POINT(41.67918052226792 73.46247345923076) bank148623 +148624 POINT(40.269426840744494 73.40891475492309) bank148624 +148625 POINT(41.08810655906644 74.29850473875088) bank148625 +148626 POINT(40.03087775517113 74.41551324738263) bank148626 +148627 POINT(41.14784831279033 73.27842883090854) bank148627 +148628 POINT(41.63884143495596 74.04449128781496) bank148628 +148629 POINT(40.20621122893699 73.40943595490155) bank148629 +148630 POINT(40.95611934707597 73.8720699499873) bank148630 +148631 POINT(41.234218932317596 74.57992181329166) bank148631 +148632 POINT(40.33944969689466 73.82230075547086) bank148632 +148633 POINT(40.80057072551999 74.4358314383887) bank148633 +148634 POINT(40.19335425386928 73.35042773669896) bank148634 +148635 POINT(40.96534126347197 73.98252311853325) bank148635 +148636 POINT(40.607730145421044 73.15244523978) bank148636 +148637 POINT(40.33668816201861 74.50700855610947) bank148637 +148638 POINT(39.832200746143535 73.76480610693696) bank148638 +148639 POINT(41.16179203889687 74.72774318191388) bank148639 +148640 POINT(41.258339314785395 74.06548956643365) bank148640 +148641 POINT(40.06850861398358 73.08309821236685) bank148641 +148642 POINT(40.19186993043422 74.95324253871051) bank148642 +148643 POINT(39.88133942136454 73.7635724926667) bank148643 +148644 POINT(41.14179022626083 73.62330381079794) bank148644 +148645 POINT(39.95532094814942 74.98649149780051) bank148645 +148646 POINT(40.270944090776524 73.31734564037079) bank148646 +148647 POINT(40.743064449509596 73.40915009127247) bank148647 +148648 POINT(41.28410607898418 73.5043172801966) bank148648 +148649 POINT(41.69332231412986 73.15194243918982) bank148649 +148650 POINT(41.12290113064064 74.25167944064033) bank148650 +148651 POINT(39.835770095543914 73.97545448164446) bank148651 +148652 POINT(41.6642937305638 74.5622897681112) bank148652 +148653 POINT(40.81527660403406 74.9161010693228) bank148653 +148654 POINT(40.24219557859907 74.72066189960354) bank148654 +148655 POINT(40.856111603936895 73.51923678341166) bank148655 +148656 POINT(40.8918642792861 74.70368747931767) bank148656 +148657 POINT(40.62654728629705 74.33534944287754) bank148657 +148658 POINT(40.27806965491548 74.28908785903523) bank148658 +148659 POINT(41.05620478839647 74.90485351043131) bank148659 +148660 POINT(40.207325674532925 74.7492799255804) bank148660 +148661 POINT(41.46803074714705 73.58735937819121) bank148661 +148662 POINT(40.55502557312138 74.28587187283085) bank148662 +148663 POINT(40.48433596941288 74.32534715313213) bank148663 +148664 POINT(41.68308145439136 73.77113020790769) bank148664 +148665 POINT(40.56848773969578 74.92027916502204) bank148665 +148666 POINT(41.56434912659582 74.46662528929947) bank148666 +148667 POINT(41.50295317177838 73.29263106376386) bank148667 +148668 POINT(40.38855211147225 73.8087230587516) bank148668 +148669 POINT(39.75333947788578 74.64195895236381) bank148669 +148670 POINT(40.81512384603654 73.4079208923173) bank148670 +148671 POINT(39.73521064799772 73.80485755525548) bank148671 +148672 POINT(41.1446684178035 73.8968305888714) bank148672 +148673 POINT(40.02484028207097 74.26215745402332) bank148673 +148674 POINT(39.734857826290515 74.08070048687664) bank148674 +148675 POINT(40.970677128285416 73.58590792389046) bank148675 +148676 POINT(40.28645756331465 73.57931823550598) bank148676 +148677 POINT(41.12326747122936 73.61554062251093) bank148677 +148678 POINT(40.68276492535208 73.35605685935533) bank148678 +148679 POINT(41.43256507724889 74.29870818362552) bank148679 +148680 POINT(41.68617337755885 73.96777189133599) bank148680 +148681 POINT(39.789092915070036 73.26553824467561) bank148681 +148682 POINT(40.46693740818286 73.8652760865293) bank148682 +148683 POINT(40.54307696185413 73.716906061141) bank148683 +148684 POINT(41.341288916915715 73.87192167070579) bank148684 +148685 POINT(40.38815576927288 73.30376336105257) bank148685 +148686 POINT(40.57499216188001 73.04371290846237) bank148686 +148687 POINT(40.831973821144324 74.28655352338598) bank148687 +148688 POINT(41.254720415796015 74.98062340715474) bank148688 +148689 POINT(41.64056536326177 73.94924796597459) bank148689 +148690 POINT(40.83325652772595 73.35184162127678) bank148690 +148691 POINT(40.414490476563095 74.21850712254994) bank148691 +148692 POINT(40.53367321270446 74.69639767153355) bank148692 +148693 POINT(40.5899272624827 73.0107832662799) bank148693 +148694 POINT(40.82182016610334 73.93400835724428) bank148694 +148695 POINT(40.06966056295338 73.81766173487108) bank148695 +148696 POINT(41.4243747878335 74.01413743971813) bank148696 +148697 POINT(40.593303238848186 74.51914938490401) bank148697 +148698 POINT(41.01250239854077 74.00181510052552) bank148698 +148699 POINT(40.795729818705865 73.43541144415362) bank148699 +148700 POINT(41.05312461158782 73.49451823782495) bank148700 +148701 POINT(39.85237531086007 74.41690272168168) bank148701 +148702 POINT(41.455326698695984 74.39037739567436) bank148702 +148703 POINT(40.714435541113744 73.2787112062387) bank148703 +148704 POINT(40.222062690968876 73.86538410261943) bank148704 +148705 POINT(40.53994129549137 74.50033662852753) bank148705 +148706 POINT(39.87278980028497 73.68489242218631) bank148706 +148707 POINT(40.63513280986304 73.1827019564448) bank148707 +148708 POINT(40.49784934550037 73.22837847048214) bank148708 +148709 POINT(40.49051326107527 73.60347953866537) bank148709 +148710 POINT(40.86419471725713 73.16567352423955) bank148710 +148711 POINT(41.4341925547005 73.73435963749519) bank148711 +148712 POINT(41.01324487425966 74.68247311178294) bank148712 +148713 POINT(40.553506654801375 73.57682709912564) bank148713 +148714 POINT(41.320150664896616 73.75262128217105) bank148714 +148715 POINT(41.26579492028528 73.30669547893774) bank148715 +148716 POINT(39.726364671361765 73.57679344780787) bank148716 +148717 POINT(41.597751982112705 73.27441089440836) bank148717 +148718 POINT(40.036852385554084 74.7589208801285) bank148718 +148719 POINT(39.94243344251315 73.28257961864293) bank148719 +148720 POINT(40.36673491780251 73.54565418156128) bank148720 +148721 POINT(40.617702177932586 74.47163940526931) bank148721 +148722 POINT(40.2670099852058 73.04254994385707) bank148722 +148723 POINT(40.13961826549624 73.01762331465322) bank148723 +148724 POINT(39.98585499307173 73.27626324134835) bank148724 +148725 POINT(40.95556609819309 73.60183768577213) bank148725 +148726 POINT(39.903730769936175 74.46090671369465) bank148726 +148727 POINT(41.47873947085966 73.21679756404117) bank148727 +148728 POINT(40.46634889034547 73.06285954037811) bank148728 +148729 POINT(40.54647877718545 74.1930387440776) bank148729 +148730 POINT(40.729313220483185 74.07968925748855) bank148730 +148731 POINT(40.85595666628387 74.1942782157308) bank148731 +148732 POINT(39.7516339668053 73.89044070962862) bank148732 +148733 POINT(41.68721050774422 74.97179932945963) bank148733 +148734 POINT(40.014498857134825 73.3037471629638) bank148734 +148735 POINT(40.12431997819976 73.1869379170452) bank148735 +148736 POINT(41.020117795178216 74.45034085414139) bank148736 +148737 POINT(40.28808797803474 73.80169776327655) bank148737 +148738 POINT(41.37900312683296 73.23889041286425) bank148738 +148739 POINT(39.865366978564325 73.23280841618913) bank148739 +148740 POINT(41.1800434605212 74.05074463391865) bank148740 +148741 POINT(40.39118320097523 74.06911924635625) bank148741 +148742 POINT(40.10640467158823 74.85972485130647) bank148742 +148743 POINT(40.33281847510731 74.98163944063185) bank148743 +148744 POINT(40.72262615688889 73.91971265006853) bank148744 +148745 POINT(40.009336725550256 74.53023035022608) bank148745 +148746 POINT(40.47118630769263 74.52985242309478) bank148746 +148747 POINT(41.29825069358203 74.55441936093814) bank148747 +148748 POINT(41.234709814127 74.60821904568242) bank148748 +148749 POINT(41.42878405456417 74.87040821956785) bank148749 +148750 POINT(41.354891861334806 74.02955523705917) bank148750 +148751 POINT(39.92320662054639 74.36336193021866) bank148751 +148752 POINT(40.91972558425217 73.64886692045098) bank148752 +148753 POINT(41.17645970531341 74.33944370646967) bank148753 +148754 POINT(39.818809797391076 74.15000071597203) bank148754 +148755 POINT(40.70505744401133 73.01705149467634) bank148755 +148756 POINT(41.364086982925684 73.8907517985879) bank148756 +148757 POINT(40.99578547289978 73.9542537254326) bank148757 +148758 POINT(41.42674170860632 73.65242815247794) bank148758 +148759 POINT(40.62164870016798 74.33187328827252) bank148759 +148760 POINT(40.54891184124754 73.57347766240518) bank148760 +148761 POINT(40.43436646071151 74.65456049700087) bank148761 +148762 POINT(40.018086928564074 74.51410887302531) bank148762 +148763 POINT(39.878804162612745 73.77984663154223) bank148763 +148764 POINT(41.58303269172337 74.50853350887246) bank148764 +148765 POINT(39.79774667414774 73.25439807562908) bank148765 +148766 POINT(40.932249554916936 73.39136477176491) bank148766 +148767 POINT(41.13103141829237 74.62356396842341) bank148767 +148768 POINT(39.86783556282016 73.6365479731454) bank148768 +148769 POINT(40.24122865050066 73.53081764209144) bank148769 +148770 POINT(40.46633275957576 74.93083842687527) bank148770 +148771 POINT(40.83436529154612 74.87497848916277) bank148771 +148772 POINT(41.3686756801776 73.59455911510588) bank148772 +148773 POINT(40.21501435078305 73.57182988329687) bank148773 +148774 POINT(40.741007356864834 74.9699638819117) bank148774 +148775 POINT(41.07528486278401 74.39456964396919) bank148775 +148776 POINT(40.52703388240808 73.48768864759134) bank148776 +148777 POINT(40.94076246133826 73.88372656530923) bank148777 +148778 POINT(41.641868847837515 74.40630627084167) bank148778 +148779 POINT(40.577227164717044 73.1888168313447) bank148779 +148780 POINT(41.531354925749575 73.17448618933525) bank148780 +148781 POINT(39.97498074008071 74.4905265400836) bank148781 +148782 POINT(40.223467394181235 74.411044943445) bank148782 +148783 POINT(41.4398254885623 73.52085676388064) bank148783 +148784 POINT(39.85266105469385 73.26300946779496) bank148784 +148785 POINT(41.198361852753514 73.0351573332013) bank148785 +148786 POINT(41.66400000416667 73.71742298283198) bank148786 +148787 POINT(41.049255273883574 74.28439921509371) bank148787 +148788 POINT(41.217265619194286 74.0912386156649) bank148788 +148789 POINT(40.71643412107277 74.8705806267607) bank148789 +148790 POINT(39.820274082601706 73.44795614081508) bank148790 +148791 POINT(40.4919783387152 74.9814049423266) bank148791 +148792 POINT(39.778413097617644 73.17528422498653) bank148792 +148793 POINT(41.16654042346855 74.31403081939617) bank148793 +148794 POINT(41.02733597545811 74.58336409977164) bank148794 +148795 POINT(41.615365932180964 74.26243691258448) bank148795 +148796 POINT(40.823328803126245 73.25597495225222) bank148796 +148797 POINT(39.8783311343015 73.04400911111749) bank148797 +148798 POINT(41.4017972133679 73.13047645132691) bank148798 +148799 POINT(41.016834991981604 74.20462964988965) bank148799 +148800 POINT(40.35962779793064 74.25963096535338) bank148800 +148801 POINT(41.00790055095072 74.93945189851522) bank148801 +148802 POINT(41.560378013747254 73.72331841180305) bank148802 +148803 POINT(40.140891945358455 74.41753250217018) bank148803 +148804 POINT(40.23477497433041 73.43514270770615) bank148804 +148805 POINT(41.1607126721476 73.21949797417868) bank148805 +148806 POINT(39.889663447503516 74.2181837479455) bank148806 +148807 POINT(40.782455372585524 73.05816162354002) bank148807 +148808 POINT(41.58611180111801 73.26977431460554) bank148808 +148809 POINT(41.66162973613524 73.1576745853408) bank148809 +148810 POINT(40.446582807967765 74.87876083850641) bank148810 +148811 POINT(40.67862328202265 74.99658152939804) bank148811 +148812 POINT(41.17660368164909 74.39254394953879) bank148812 +148813 POINT(40.02561921750992 74.53709145448751) bank148813 +148814 POINT(41.38319035141336 74.08894560765063) bank148814 +148815 POINT(41.49072483921814 73.45327671880558) bank148815 +148816 POINT(40.52956405066012 73.9137096487013) bank148816 +148817 POINT(40.89168965176114 73.21780564376174) bank148817 +148818 POINT(40.48402963976535 74.88859012655574) bank148818 +148819 POINT(40.70591290902266 74.16850613199647) bank148819 +148820 POINT(40.95834381202805 73.23559943684228) bank148820 +148821 POINT(40.44216972707759 74.39263504623156) bank148821 +148822 POINT(41.027239665905256 74.93502843058567) bank148822 +148823 POINT(40.25629629792743 74.3847436379046) bank148823 +148824 POINT(40.85460613900082 73.0486808948662) bank148824 +148825 POINT(40.57553027028354 73.08309632930104) bank148825 +148826 POINT(41.71033191303736 74.30275902112676) bank148826 +148827 POINT(39.824162630311726 74.41971308557875) bank148827 +148828 POINT(40.60278844274212 73.04974727943747) bank148828 +148829 POINT(41.01897124361971 74.74429963599448) bank148829 +148830 POINT(39.80256016903869 73.22603091613382) bank148830 +148831 POINT(41.032985058957834 74.25209328621382) bank148831 +148832 POINT(41.06165053988522 73.40389866689591) bank148832 +148833 POINT(40.118824569327735 74.35288025933568) bank148833 +148834 POINT(41.666049847905505 74.60886686602515) bank148834 +148835 POINT(40.97504550161402 73.6253192812975) bank148835 +148836 POINT(41.67813759357529 73.39253066763408) bank148836 +148837 POINT(41.43525021676683 74.80533046855028) bank148837 +148838 POINT(40.67247719455237 74.9312578609956) bank148838 +148839 POINT(40.02934635011956 74.40552593858119) bank148839 +148840 POINT(41.605760824465555 74.3139532943731) bank148840 +148841 POINT(41.32435431684199 74.8385285871532) bank148841 +148842 POINT(41.41243298722537 74.83274297963146) bank148842 +148843 POINT(40.66679252775786 74.81163721501089) bank148843 +148844 POINT(40.88341639486644 73.84501767185175) bank148844 +148845 POINT(39.97034335845197 73.74482901587098) bank148845 +148846 POINT(41.41986579539438 74.76134563779794) bank148846 +148847 POINT(41.00218693038107 74.47792203240444) bank148847 +148848 POINT(39.8893842069062 73.69768952902079) bank148848 +148849 POINT(41.3838578238973 73.12263303236102) bank148849 +148850 POINT(40.42808146518073 73.04857227106255) bank148850 +148851 POINT(40.671167773333565 74.29179701988173) bank148851 +148852 POINT(39.90929555293263 73.75925011037442) bank148852 +148853 POINT(40.357520177826174 74.92920332529168) bank148853 +148854 POINT(40.10641809703735 73.47242534303258) bank148854 +148855 POINT(40.943958113125255 73.2005903630869) bank148855 +148856 POINT(41.103438337980194 74.14502067666126) bank148856 +148857 POINT(40.31555290850336 73.53434762731602) bank148857 +148858 POINT(40.997947736470074 74.9469509319457) bank148858 +148859 POINT(40.26406011642421 74.36941215668588) bank148859 +148860 POINT(40.44710131494359 73.89608486307534) bank148860 +148861 POINT(39.78746433099207 74.29399992515282) bank148861 +148862 POINT(41.073597543102636 73.87674644768025) bank148862 +148863 POINT(41.04285017742545 74.61602244853344) bank148863 +148864 POINT(41.37573317269208 73.68226693936477) bank148864 +148865 POINT(41.15498339692737 74.56422736332794) bank148865 +148866 POINT(40.69272506301387 74.68747026518638) bank148866 +148867 POINT(41.427688741764065 73.13411734293497) bank148867 +148868 POINT(40.134909203134406 73.71840506561914) bank148868 +148869 POINT(40.3276099702841 73.58056432738675) bank148869 +148870 POINT(40.252500361613876 74.986848072767) bank148870 +148871 POINT(40.590928046497325 73.69032390105885) bank148871 +148872 POINT(39.89105085227294 73.3993929045438) bank148872 +148873 POINT(41.53343470872574 73.61282396465072) bank148873 +148874 POINT(39.90733271814942 73.98836348816576) bank148874 +148875 POINT(41.0858503014851 73.58623542075654) bank148875 +148876 POINT(40.612184432711324 74.82744458279035) bank148876 +148877 POINT(41.030674321617774 74.05152648918511) bank148877 +148878 POINT(40.60691059050017 73.3008658762733) bank148878 +148879 POINT(41.45102094026437 74.34691162146673) bank148879 +148880 POINT(39.95280386673348 73.87166826321933) bank148880 +148881 POINT(40.19023969145334 73.84544016718635) bank148881 +148882 POINT(40.400653594624444 74.38831226097325) bank148882 +148883 POINT(41.65562874148949 73.33936316839107) bank148883 +148884 POINT(39.98051015861383 73.76356605703809) bank148884 +148885 POINT(41.154900642324634 73.8089465545633) bank148885 +148886 POINT(40.22671253298945 73.26360654798603) bank148886 +148887 POINT(41.45172004213417 74.85331756171523) bank148887 +148888 POINT(40.03677809340608 74.75469759925414) bank148888 +148889 POINT(40.90082757850555 74.66336137326361) bank148889 +148890 POINT(41.38341356148032 74.27054957110842) bank148890 +148891 POINT(41.235202852161144 74.19704374486481) bank148891 +148892 POINT(40.88498517729851 74.29753854813562) bank148892 +148893 POINT(40.61606799129249 74.21404984974767) bank148893 +148894 POINT(40.30957712666704 74.28439674934872) bank148894 +148895 POINT(40.43064453157069 73.17625735620042) bank148895 +148896 POINT(40.463180411523936 74.7477511495254) bank148896 +148897 POINT(41.30370724839421 74.81557022396392) bank148897 +148898 POINT(41.10174774536747 73.86546289093704) bank148898 +148899 POINT(40.62578160883798 74.03533168714938) bank148899 +148900 POINT(40.900626023109055 74.68541716806834) bank148900 +148901 POINT(40.15346931866424 74.60353784306253) bank148901 +148902 POINT(41.13277617865194 74.08948246259678) bank148902 +148903 POINT(40.2921048649747 73.7331300486194) bank148903 +148904 POINT(40.16964848438603 73.06503299848738) bank148904 +148905 POINT(41.497473709125686 74.91157681044423) bank148905 +148906 POINT(41.616598210929915 74.42279769772283) bank148906 +148907 POINT(40.766056006223145 74.66079864999335) bank148907 +148908 POINT(41.50681108265699 74.64386037630327) bank148908 +148909 POINT(39.87405322636131 74.66005004265504) bank148909 +148910 POINT(40.74137641121893 73.65778596670687) bank148910 +148911 POINT(40.5579484591626 74.36842199674614) bank148911 +148912 POINT(40.0154914061535 74.7838984553833) bank148912 +148913 POINT(40.375388447828854 73.51312081581571) bank148913 +148914 POINT(41.31795614242518 73.15785396680884) bank148914 +148915 POINT(40.044926484030896 73.19208641130287) bank148915 +148916 POINT(40.00175975248603 74.2964090185068) bank148916 +148917 POINT(40.2650954529372 74.88950078231177) bank148917 +148918 POINT(40.38812327081267 73.38984475628222) bank148918 +148919 POINT(40.169547805687706 74.62387628084979) bank148919 +148920 POINT(40.806804554936996 73.51521620252254) bank148920 +148921 POINT(40.317659254539265 73.4769883326858) bank148921 +148922 POINT(40.025015357421964 74.05925279644018) bank148922 +148923 POINT(41.080988054462104 74.0177895940333) bank148923 +148924 POINT(40.96617863435371 74.64219323193626) bank148924 +148925 POINT(40.18907250827332 73.09500939132423) bank148925 +148926 POINT(41.1668678451315 73.69170832724812) bank148926 +148927 POINT(41.53740650875534 73.26910575255614) bank148927 +148928 POINT(40.61154005967421 73.88322727943425) bank148928 +148929 POINT(41.34534677336304 73.97167708857214) bank148929 +148930 POINT(41.361770196149855 73.12108311565841) bank148930 +148931 POINT(39.852363566006595 74.31695188599154) bank148931 +148932 POINT(39.86852537151091 73.15212904550809) bank148932 +148933 POINT(40.94478412488311 74.8956439042215) bank148933 +148934 POINT(40.00340564106556 74.07675714854076) bank148934 +148935 POINT(40.68422129517467 74.82325976068199) bank148935 +148936 POINT(41.54003402935886 74.92504982386988) bank148936 +148937 POINT(41.05639987151529 74.23575378381659) bank148937 +148938 POINT(39.744183746862475 73.06583846808931) bank148938 +148939 POINT(40.05385657898896 73.35619570628756) bank148939 +148940 POINT(41.648855610483835 73.32330453448027) bank148940 +148941 POINT(40.85092150365444 74.0541059021838) bank148941 +148942 POINT(40.0591771552435 73.4098711631012) bank148942 +148943 POINT(40.8912606074272 73.91636240269338) bank148943 +148944 POINT(41.13597134077298 73.76032446467134) bank148944 +148945 POINT(40.80391375860738 74.55919338802215) bank148945 +148946 POINT(41.25644491374014 74.91168207648491) bank148946 +148947 POINT(39.833592062436296 73.96205348441066) bank148947 +148948 POINT(41.4662958221798 73.62913256488692) bank148948 +148949 POINT(39.747721778938896 73.52777967023967) bank148949 +148950 POINT(40.552512268687494 73.1895389393339) bank148950 +148951 POINT(40.710903136010565 74.14195708937578) bank148951 +148952 POINT(40.051358856002594 74.0722876963717) bank148952 +148953 POINT(41.24119572382277 73.75015310297809) bank148953 +148954 POINT(40.422994440542176 73.19345134751585) bank148954 +148955 POINT(40.697256636655126 74.52869024297105) bank148955 +148956 POINT(40.080900583695716 73.25742999816644) bank148956 +148957 POINT(39.966974994052585 74.16765204890064) bank148957 +148958 POINT(41.39774776942262 74.12263459912486) bank148958 +148959 POINT(41.10983720644146 74.84621244985156) bank148959 +148960 POINT(39.99857054016378 74.75343965681223) bank148960 +148961 POINT(40.43579569437846 73.06273521797074) bank148961 +148962 POINT(41.45036951002919 74.96082546109993) bank148962 +148963 POINT(40.25818928928015 73.38517738538208) bank148963 +148964 POINT(40.3561709320112 74.99053589579222) bank148964 +148965 POINT(40.77601337862765 74.96999627681126) bank148965 +148966 POINT(40.013754435942005 74.47662226803666) bank148966 +148967 POINT(41.472716713461125 73.06571278284035) bank148967 +148968 POINT(41.40088956109153 74.23754318221125) bank148968 +148969 POINT(40.83644170455599 73.95117480361378) bank148969 +148970 POINT(40.74845895170242 74.71774139891636) bank148970 +148971 POINT(40.7271201671198 73.59582085853502) bank148971 +148972 POINT(40.94776070231431 73.70738444456795) bank148972 +148973 POINT(39.88329760390109 73.08068606241764) bank148973 +148974 POINT(40.5546250759207 74.9518560847235) bank148974 +148975 POINT(39.85550855858032 73.59942870243974) bank148975 +148976 POINT(40.093543451895684 74.2979313958409) bank148976 +148977 POINT(40.99479520510222 73.18389333602529) bank148977 +148978 POINT(40.56082378546776 74.42481712277069) bank148978 +148979 POINT(39.79773972639354 74.81158933390248) bank148979 +148980 POINT(41.255143532172944 74.1709991026381) bank148980 +148981 POINT(41.52806418805673 73.76597932609573) bank148981 +148982 POINT(39.90456769252668 73.81604399422419) bank148982 +148983 POINT(41.44410309092319 74.37972934094813) bank148983 +148984 POINT(39.83789877122264 73.18552015987116) bank148984 +148985 POINT(40.64472659090461 73.16572503014244) bank148985 +148986 POINT(40.181057547657794 74.29549933030442) bank148986 +148987 POINT(40.296254299416816 74.58333665545476) bank148987 +148988 POINT(41.007964077463335 74.92204168945227) bank148988 +148989 POINT(40.542089390357525 73.4191435852783) bank148989 +148990 POINT(39.98260636273598 73.49286608589767) bank148990 +148991 POINT(40.79091981466286 74.68374446962366) bank148991 +148992 POINT(40.199102480964434 73.51036757165524) bank148992 +148993 POINT(41.652101859070775 73.6134850730634) bank148993 +148994 POINT(39.87238617332586 74.59586620549614) bank148994 +148995 POINT(40.46583475468259 74.93847220904368) bank148995 +148996 POINT(41.06161997025732 74.85802923871206) bank148996 +148997 POINT(39.98637272536387 73.64067511707177) bank148997 +148998 POINT(41.52750968481384 74.3477684503605) bank148998 +148999 POINT(39.770300978420245 73.3041451821156) bank148999 +149000 POINT(40.3317444050395 73.01931643441591) bank149000 +149001 POINT(40.35234956455476 73.10166669961653) bank149001 +149002 POINT(39.745686054533124 73.75717656301981) bank149002 +149003 POINT(40.04548202355571 74.63808678994668) bank149003 +149004 POINT(39.904298559670735 74.33509968609197) bank149004 +149005 POINT(40.30845074973677 73.67209331080004) bank149005 +149006 POINT(40.283027842977724 73.06266820923739) bank149006 +149007 POINT(40.358665178406824 74.73192082742841) bank149007 +149008 POINT(40.0552304076745 73.05678975089897) bank149008 +149009 POINT(39.84497365248995 74.1873107746775) bank149009 +149010 POINT(40.05487911242716 73.39167821775708) bank149010 +149011 POINT(40.48442803571485 74.6435394293043) bank149011 +149012 POINT(39.968781899811454 73.11287668463848) bank149012 +149013 POINT(40.59930674977902 73.98192296338564) bank149013 +149014 POINT(41.38861023844165 73.58349138147514) bank149014 +149015 POINT(40.92115322820213 74.24889670033478) bank149015 +149016 POINT(41.28693689039809 73.26407329525192) bank149016 +149017 POINT(40.73285459652693 73.6898520457883) bank149017 +149018 POINT(40.651935429318435 74.01404791019881) bank149018 +149019 POINT(41.56986783747168 74.72300153720819) bank149019 +149020 POINT(39.954354058971916 73.25371637515299) bank149020 +149021 POINT(41.18274589962079 74.52850784802767) bank149021 +149022 POINT(40.43931061243652 74.94441502167415) bank149022 +149023 POINT(40.1461093528763 74.43434576470558) bank149023 +149024 POINT(41.67448924019496 74.21292313516376) bank149024 +149025 POINT(40.91027661396622 74.5953423890092) bank149025 +149026 POINT(41.13169098672376 74.15156050618056) bank149026 +149027 POINT(40.14412854324173 74.6925311767346) bank149027 +149028 POINT(40.80421322976152 74.34151682084054) bank149028 +149029 POINT(40.97107982691942 74.9143613493735) bank149029 +149030 POINT(40.41243342095687 73.54108539071191) bank149030 +149031 POINT(40.56709623390106 74.04110361758218) bank149031 +149032 POINT(40.024531441837496 74.72690107484259) bank149032 +149033 POINT(41.46840801809069 74.97228590679252) bank149033 +149034 POINT(40.203904932171206 74.08406509848197) bank149034 +149035 POINT(40.56678727540383 73.18599024492586) bank149035 +149036 POINT(41.16962566532794 73.25240777280946) bank149036 +149037 POINT(40.76867937184234 73.52209820150665) bank149037 +149038 POINT(39.84547981021743 74.09512664640701) bank149038 +149039 POINT(40.449158222406744 74.35701854298549) bank149039 +149040 POINT(41.01099162917153 73.72633643418746) bank149040 +149041 POINT(39.929560566381035 74.67149064656363) bank149041 +149042 POINT(40.11921400164729 74.68151886443312) bank149042 +149043 POINT(40.83164631463506 73.02055352609902) bank149043 +149044 POINT(40.831130515719934 74.12982803107538) bank149044 +149045 POINT(39.716596401519304 73.46896183329424) bank149045 +149046 POINT(41.33989219323475 73.44561037151979) bank149046 +149047 POINT(40.363015376641854 73.60168791015295) bank149047 +149048 POINT(40.12341880948939 73.86618608394271) bank149048 +149049 POINT(40.867766084650455 74.90302165213353) bank149049 +149050 POINT(40.21513505056689 73.21946143388625) bank149050 +149051 POINT(41.242898518754075 73.16665351389307) bank149051 +149052 POINT(41.078963656892896 73.34107976829327) bank149052 +149053 POINT(40.748315623070894 73.25380685353969) bank149053 +149054 POINT(40.38132146211009 73.73502439951933) bank149054 +149055 POINT(41.40024501284556 73.1830823994333) bank149055 +149056 POINT(40.74823140508748 73.7244689330992) bank149056 +149057 POINT(40.819077846711274 73.35693514101251) bank149057 +149058 POINT(39.75608507699315 74.34425996047821) bank149058 +149059 POINT(40.27885060725945 74.05966131400946) bank149059 +149060 POINT(40.924259020309634 73.55984377205237) bank149060 +149061 POINT(41.001701063224964 73.93227625446781) bank149061 +149062 POINT(40.5744173587891 74.91868520990214) bank149062 +149063 POINT(40.5036164436836 73.45619080115902) bank149063 +149064 POINT(40.580413259854055 74.83875496068826) bank149064 +149065 POINT(41.29401849894046 74.58092802398386) bank149065 +149066 POINT(40.50125419876759 74.83149257262072) bank149066 +149067 POINT(39.995909816222934 73.77639319941123) bank149067 +149068 POINT(40.990562716446796 74.09064464277412) bank149068 +149069 POINT(39.772260023416884 74.5739018009687) bank149069 +149070 POINT(41.331817315030925 73.57437719003863) bank149070 +149071 POINT(40.40779108370678 73.86974048955454) bank149071 +149072 POINT(40.2929706280618 74.2574984324941) bank149072 +149073 POINT(40.57708004240162 74.05420932180111) bank149073 +149074 POINT(41.475438098878584 74.94578482257876) bank149074 +149075 POINT(39.817332512802 74.18591057075653) bank149075 +149076 POINT(41.35936576941936 74.24969160251896) bank149076 +149077 POINT(40.76293807211573 74.41080527708318) bank149077 +149078 POINT(41.2839279077924 74.01332576806169) bank149078 +149079 POINT(41.12158121754417 73.49002848030126) bank149079 +149080 POINT(40.35542649883352 73.36611323858963) bank149080 +149081 POINT(41.089883208507715 74.42940809289003) bank149081 +149082 POINT(40.425553699552715 74.05812104928991) bank149082 +149083 POINT(39.8490843022938 73.67766044494111) bank149083 +149084 POINT(41.270736335217094 74.77626543100898) bank149084 +149085 POINT(41.499263358028315 73.28861632593761) bank149085 +149086 POINT(41.14179552219927 74.11808554332073) bank149086 +149087 POINT(40.02351101191198 73.36795519724986) bank149087 +149088 POINT(40.13731497713159 73.15174997857335) bank149088 +149089 POINT(40.74264043408083 74.23415331861439) bank149089 +149090 POINT(40.259984478398714 74.67238543412026) bank149090 +149091 POINT(40.681513396372004 74.16518598523226) bank149091 +149092 POINT(41.122482758087905 73.2084257778721) bank149092 +149093 POINT(39.79442841829548 74.95127121639683) bank149093 +149094 POINT(39.83813343934753 73.07323430486522) bank149094 +149095 POINT(39.95689893455925 73.57635021874401) bank149095 +149096 POINT(40.41708393092557 73.51163311817184) bank149096 +149097 POINT(41.498141210806786 73.04392877095535) bank149097 +149098 POINT(39.874898381127196 74.36059260665469) bank149098 +149099 POINT(40.27808449885464 74.20538428339647) bank149099 +149100 POINT(41.011956747503966 74.6667611539964) bank149100 +149101 POINT(40.36621145238776 74.48908780877507) bank149101 +149102 POINT(40.66931788087768 73.3474651143163) bank149102 +149103 POINT(39.766455958202535 73.24291864106203) bank149103 +149104 POINT(41.29853162255391 74.04499643855357) bank149104 +149105 POINT(40.01904412573161 73.77747913729515) bank149105 +149106 POINT(39.95159569241219 74.04848242422716) bank149106 +149107 POINT(40.2327273873166 73.57330485031065) bank149107 +149108 POINT(40.63633062695681 73.87393302492977) bank149108 +149109 POINT(41.211240289569396 74.00407986627364) bank149109 +149110 POINT(40.4569504021412 73.76176386139119) bank149110 +149111 POINT(40.05488435532262 73.47410227142197) bank149111 +149112 POINT(39.8845160547083 73.01398079427871) bank149112 +149113 POINT(40.56946133405187 74.18050553857636) bank149113 +149114 POINT(40.11276368970319 73.42219004721402) bank149114 +149115 POINT(41.31635990997568 74.60301484707661) bank149115 +149116 POINT(41.102332548520536 73.94013557087837) bank149116 +149117 POINT(40.885975370393716 74.38681211611323) bank149117 +149118 POINT(40.549559003977954 73.39387339630865) bank149118 +149119 POINT(40.31888687222446 73.48396124551165) bank149119 +149120 POINT(41.51709946486144 73.0880994792804) bank149120 +149121 POINT(41.376778565292916 73.75455552974962) bank149121 +149122 POINT(40.82849985579955 73.71056243362877) bank149122 +149123 POINT(40.31919410792755 74.52557604136355) bank149123 +149124 POINT(41.585771961588335 73.67653186712816) bank149124 +149125 POINT(41.41308073861699 73.2557056150301) bank149125 +149126 POINT(41.254791760306986 74.04031451897488) bank149126 +149127 POINT(41.29560781519686 73.73867018030198) bank149127 +149128 POINT(41.29239688674787 74.75234260911051) bank149128 +149129 POINT(40.950638108537376 73.83451666070273) bank149129 +149130 POINT(41.226045140536634 73.84188556156975) bank149130 +149131 POINT(40.22288473583367 73.41145503668662) bank149131 +149132 POINT(41.315875354674525 73.94917550717699) bank149132 +149133 POINT(40.75688666959531 74.15746535343276) bank149133 +149134 POINT(40.78810474747481 73.71630466414302) bank149134 +149135 POINT(40.35521799885283 74.2469234800214) bank149135 +149136 POINT(40.6821194327675 74.08310653234396) bank149136 +149137 POINT(40.409324382820984 73.3297250347633) bank149137 +149138 POINT(40.36503077257294 74.04500747954611) bank149138 +149139 POINT(41.05065110374706 74.59071677123531) bank149139 +149140 POINT(41.637606289773665 74.94539218858063) bank149140 +149141 POINT(41.66296418241547 73.56657929208932) bank149141 +149142 POINT(39.944847640495176 73.41191503519552) bank149142 +149143 POINT(41.578928145484426 74.89665251263136) bank149143 +149144 POINT(39.712806022046706 73.62944759234563) bank149144 +149145 POINT(40.08086523552722 73.47199020106393) bank149145 +149146 POINT(40.89522048818381 74.35551383486603) bank149146 +149147 POINT(40.46661721088698 74.18825710108075) bank149147 +149148 POINT(40.24790319153997 73.9581847091911) bank149148 +149149 POINT(41.47212690902061 74.50312888706753) bank149149 +149150 POINT(40.897379354675586 73.06651209150878) bank149150 +149151 POINT(41.06080984942824 73.23699742731424) bank149151 +149152 POINT(39.944160556475325 74.93157807612073) bank149152 +149153 POINT(41.57887893521823 73.94600671937174) bank149153 +149154 POINT(41.16795312319135 74.96923913946308) bank149154 +149155 POINT(41.19927403441678 74.8909033427032) bank149155 +149156 POINT(40.68404976949262 74.78197857400033) bank149156 +149157 POINT(40.67091665917704 73.55350305460988) bank149157 +149158 POINT(40.243827626509116 74.6210780088374) bank149158 +149159 POINT(41.32758656329724 74.05693602506432) bank149159 +149160 POINT(41.4652665590657 74.00511628591144) bank149160 +149161 POINT(40.033828908443 73.95677964479674) bank149161 +149162 POINT(40.8700406885313 74.24034000957795) bank149162 +149163 POINT(40.98653390454083 74.60536868013652) bank149163 +149164 POINT(41.3184466738158 74.53228568264413) bank149164 +149165 POINT(40.39952108259666 74.4908605710552) bank149165 +149166 POINT(41.4539861041081 73.78276768563914) bank149166 +149167 POINT(40.67744193017452 74.82015961176637) bank149167 +149168 POINT(40.490394878519105 73.9117942329523) bank149168 +149169 POINT(39.999162445847816 73.88251299309745) bank149169 +149170 POINT(40.52209752405805 74.41527845945136) bank149170 +149171 POINT(40.09481061458314 73.0956282270088) bank149171 +149172 POINT(39.71885629983416 73.07906308898674) bank149172 +149173 POINT(39.75135858118992 73.33328722015703) bank149173 +149174 POINT(41.53928535988012 74.06482983807815) bank149174 +149175 POINT(41.57232812897418 73.15215168229044) bank149175 +149176 POINT(40.513469129452304 73.68905758596529) bank149176 +149177 POINT(41.659045147402445 73.05806027928094) bank149177 +149178 POINT(41.395614859525516 73.92162566870682) bank149178 +149179 POINT(41.711767733005956 73.10404459540558) bank149179 +149180 POINT(41.65937822339603 73.25344262051794) bank149180 +149181 POINT(40.857820346910586 74.30486755138602) bank149181 +149182 POINT(40.898290272559564 74.14076790186738) bank149182 +149183 POINT(40.57308531972902 73.92042054862512) bank149183 +149184 POINT(40.22297555852433 73.72013721829398) bank149184 +149185 POINT(40.26029487737475 73.97586667533594) bank149185 +149186 POINT(40.947002128033795 74.08080889853976) bank149186 +149187 POINT(41.70594554727061 73.56780092010085) bank149187 +149188 POINT(41.54867876461377 74.00111134698791) bank149188 +149189 POINT(40.1232572767053 74.44383991618704) bank149189 +149190 POINT(39.84055368697148 74.81578792393957) bank149190 +149191 POINT(40.63608308912818 73.46290407619803) bank149191 +149192 POINT(40.159703717148 74.30800371597) bank149192 +149193 POINT(40.69650399397053 73.59345262891497) bank149193 +149194 POINT(41.39601626402219 73.30832417871811) bank149194 +149195 POINT(40.48438159062642 74.33651049121157) bank149195 +149196 POINT(39.90047230063082 74.61784998009234) bank149196 +149197 POINT(40.25291121542176 73.82795930000316) bank149197 +149198 POINT(41.102565247526385 73.76032126774163) bank149198 +149199 POINT(41.662316858358224 74.78127917627315) bank149199 +149200 POINT(40.30902083294551 74.50569539589692) bank149200 +149201 POINT(40.98950204462616 73.81118799583645) bank149201 +149202 POINT(40.97899488516038 73.18453337477277) bank149202 +149203 POINT(39.95943870632036 74.06578566678553) bank149203 +149204 POINT(40.378379254717906 74.44321770298203) bank149204 +149205 POINT(40.7715990778348 74.36552570929922) bank149205 +149206 POINT(40.8110365126298 73.40384620411561) bank149206 +149207 POINT(41.169422672439374 73.06130508072742) bank149207 +149208 POINT(40.382800706588256 74.8851266126069) bank149208 +149209 POINT(40.916587824119034 73.34743627660657) bank149209 +149210 POINT(41.16385787926847 74.63035136253043) bank149210 +149211 POINT(39.835025397714716 74.89148923532923) bank149211 +149212 POINT(41.46817311627298 74.46151852899263) bank149212 +149213 POINT(40.93684176658728 74.91571817881093) bank149213 +149214 POINT(39.94363560295609 73.60553492082344) bank149214 +149215 POINT(39.99744017417086 73.71779152329225) bank149215 +149216 POINT(41.552408622527636 74.7194565673872) bank149216 +149217 POINT(41.54814413149903 74.32349441364688) bank149217 +149218 POINT(41.29850564005623 73.313197899391) bank149218 +149219 POINT(40.797164401310944 73.22203745928671) bank149219 +149220 POINT(41.111871615759135 74.68404622938282) bank149220 +149221 POINT(40.44228641981253 73.89352064547238) bank149221 +149222 POINT(41.43816153806519 73.61732040435987) bank149222 +149223 POINT(41.05429404047164 74.34171201769942) bank149223 +149224 POINT(41.112593323169314 73.39529885422805) bank149224 +149225 POINT(41.05461822734738 74.41598156540525) bank149225 +149226 POINT(40.95589693912088 74.07802105260767) bank149226 +149227 POINT(39.9927493853683 73.79740926662205) bank149227 +149228 POINT(40.933566174564284 74.70650858752036) bank149228 +149229 POINT(40.22390781182572 74.95263397998141) bank149229 +149230 POINT(41.6977576379434 73.37329512081357) bank149230 +149231 POINT(40.591444127786886 73.401795425257) bank149231 +149232 POINT(41.677522866746415 74.69292699486347) bank149232 +149233 POINT(40.55146289071628 73.7322511377578) bank149233 +149234 POINT(40.07908109971748 74.69816286008185) bank149234 +149235 POINT(41.62202687159934 73.79931346093447) bank149235 +149236 POINT(41.62464256224819 74.2346568438497) bank149236 +149237 POINT(41.650905060045986 74.63637779875103) bank149237 +149238 POINT(40.50568161748301 73.59946991565083) bank149238 +149239 POINT(40.318589744979846 73.53763855801132) bank149239 +149240 POINT(40.733838026026795 73.57003609841783) bank149240 +149241 POINT(39.85183485896012 73.52184962300421) bank149241 +149242 POINT(40.271882108965826 74.21161137609542) bank149242 +149243 POINT(41.03196529613235 73.36641584538707) bank149243 +149244 POINT(39.96912690019812 74.85030690180342) bank149244 +149245 POINT(40.93489108204298 74.84194504979024) bank149245 +149246 POINT(40.04087332436749 74.37245063518033) bank149246 +149247 POINT(40.88543117833556 73.70898142808734) bank149247 +149248 POINT(40.3355125692725 73.20098558762933) bank149248 +149249 POINT(41.04851273506636 74.5850190054549) bank149249 +149250 POINT(40.50622502761329 73.48813379234458) bank149250 +149251 POINT(40.226525758189425 73.79638985427172) bank149251 +149252 POINT(41.35343785651513 74.46486588184754) bank149252 +149253 POINT(41.30331938145046 74.02438173488467) bank149253 +149254 POINT(40.98224142833322 74.14582556334274) bank149254 +149255 POINT(41.705439088706434 74.85465735835689) bank149255 +149256 POINT(40.84672076282211 74.2867429515907) bank149256 +149257 POINT(39.729755270643246 74.41033800383718) bank149257 +149258 POINT(41.464075423670856 74.84952678560411) bank149258 +149259 POINT(40.07993191701968 74.27085969845915) bank149259 +149260 POINT(40.51099792387438 74.61683706821815) bank149260 +149261 POINT(40.3924090974771 74.02892035124495) bank149261 +149262 POINT(41.332941299976085 73.43288276262815) bank149262 +149263 POINT(40.24461569582303 73.90495069356228) bank149263 +149264 POINT(41.35978466552481 74.18566077179638) bank149264 +149265 POINT(39.865619193500784 74.31179784593029) bank149265 +149266 POINT(40.813471102508224 74.12342658624225) bank149266 +149267 POINT(40.881194051100685 73.20776414046927) bank149267 +149268 POINT(40.957798872429294 74.19257673524173) bank149268 +149269 POINT(39.83361701658893 73.74590491686814) bank149269 +149270 POINT(41.36471533056776 73.66848490994022) bank149270 +149271 POINT(40.81710039509744 73.02495527380965) bank149271 +149272 POINT(40.62606221716373 74.90958093644059) bank149272 +149273 POINT(40.791796268808 73.22495172014695) bank149273 +149274 POINT(40.36841587618 73.03559990185006) bank149274 +149275 POINT(40.346141460635096 73.2535954054813) bank149275 +149276 POINT(40.43648526540714 73.48544795846713) bank149276 +149277 POINT(40.451527752237766 74.87274884045874) bank149277 +149278 POINT(41.16284581376246 73.310399092377) bank149278 +149279 POINT(40.89032979548198 73.9301444261456) bank149279 +149280 POINT(40.53682825392227 74.48057327169514) bank149280 +149281 POINT(39.744671286494096 73.83098698545264) bank149281 +149282 POINT(40.35782759569297 74.76201926733665) bank149282 +149283 POINT(39.981293737104224 74.84209614432181) bank149283 +149284 POINT(39.9274202551532 73.85358269122676) bank149284 +149285 POINT(40.01974222494718 73.32058401186497) bank149285 +149286 POINT(40.01242924966486 73.7518778400519) bank149286 +149287 POINT(40.471310270134616 74.64500185205509) bank149287 +149288 POINT(40.63466071877383 74.99502981472966) bank149288 +149289 POINT(39.98395941426815 74.93092638773561) bank149289 +149290 POINT(40.75069185270853 73.07522566375712) bank149290 +149291 POINT(41.47238686217538 73.46189993232953) bank149291 +149292 POINT(40.22448430920282 73.28122859945279) bank149292 +149293 POINT(40.30244455417472 74.53192508863592) bank149293 +149294 POINT(40.68634892531153 74.17458897952997) bank149294 +149295 POINT(40.163391705484976 74.24855849863523) bank149295 +149296 POINT(40.239201189398315 74.61648775164251) bank149296 +149297 POINT(39.890040066599354 74.78821868857659) bank149297 +149298 POINT(40.65694079067636 74.61365121668109) bank149298 +149299 POINT(40.16796213809286 73.87521194617702) bank149299 +149300 POINT(40.34716440729094 73.61830669897614) bank149300 +149301 POINT(41.63076396014129 74.94647838895666) bank149301 +149302 POINT(41.50433692827235 73.16756593575025) bank149302 +149303 POINT(41.426802521910716 74.32107483427443) bank149303 +149304 POINT(41.24438541400948 73.74916870769856) bank149304 +149305 POINT(40.91761622933478 74.7837272201719) bank149305 +149306 POINT(40.48079106752483 73.98730806755002) bank149306 +149307 POINT(40.988862993829486 74.499895322743) bank149307 +149308 POINT(41.69311582434505 74.80440265695063) bank149308 +149309 POINT(41.587294073897425 73.6372549724704) bank149309 +149310 POINT(40.90619173418087 73.0216508046276) bank149310 +149311 POINT(41.241653292441036 74.84946524021616) bank149311 +149312 POINT(40.89095155851543 73.3667568724765) bank149312 +149313 POINT(40.15748117323511 73.59340349178994) bank149313 +149314 POINT(40.10138217383759 74.05449971093164) bank149314 +149315 POINT(40.5893171709582 73.8754011613562) bank149315 +149316 POINT(39.93807585437866 73.81924350556419) bank149316 +149317 POINT(39.90690745396543 74.21124933392701) bank149317 +149318 POINT(41.48184526195225 74.05411290337035) bank149318 +149319 POINT(41.58421028192434 73.9986532358807) bank149319 +149320 POINT(40.32476367516058 74.34254858989637) bank149320 +149321 POINT(39.81536101918233 73.40404112411711) bank149321 +149322 POINT(40.896157409130545 74.93529285765536) bank149322 +149323 POINT(41.234489309473325 73.42261769680266) bank149323 +149324 POINT(40.97712691151472 74.51420874942002) bank149324 +149325 POINT(40.889021353461665 73.45265703245812) bank149325 +149326 POINT(41.01058942321467 73.96044813331196) bank149326 +149327 POINT(40.152438569818926 74.37417904604528) bank149327 +149328 POINT(39.818733902434694 73.50591079449708) bank149328 +149329 POINT(40.13410099972811 73.13855617483874) bank149329 +149330 POINT(39.98435427140567 73.81939159558232) bank149330 +149331 POINT(40.31093365677134 73.93281990536704) bank149331 +149332 POINT(41.57022725374883 74.10318148271632) bank149332 +149333 POINT(40.27660947632837 74.80667026937017) bank149333 +149334 POINT(40.4891184507666 73.07350945009904) bank149334 +149335 POINT(41.31315582608154 74.23279338262822) bank149335 +149336 POINT(40.66750405979482 74.43509725550048) bank149336 +149337 POINT(40.06877496085867 74.44748850553923) bank149337 +149338 POINT(40.23212521814222 73.33373539331426) bank149338 +149339 POINT(41.06937855575868 73.22262881765553) bank149339 +149340 POINT(41.71012814350389 74.67238359242495) bank149340 +149341 POINT(40.63127184886025 74.20653089732319) bank149341 +149342 POINT(40.50283388309184 74.83414020863559) bank149342 +149343 POINT(40.196661354882615 73.87654792365586) bank149343 +149344 POINT(40.10419161348793 73.04509645932578) bank149344 +149345 POINT(40.53807915446722 73.61858028461863) bank149345 +149346 POINT(39.811113673372255 74.44302463070063) bank149346 +149347 POINT(40.392857646703334 73.60039885214444) bank149347 +149348 POINT(40.399655626134184 74.65541747690361) bank149348 +149349 POINT(40.83243666875434 74.90794457953464) bank149349 +149350 POINT(40.55195488410349 73.08678182550764) bank149350 +149351 POINT(39.9984306150448 74.81779243589862) bank149351 +149352 POINT(40.15481300088763 74.16907872845073) bank149352 +149353 POINT(39.83467522129482 74.30684994524226) bank149353 +149354 POINT(39.740244173098716 73.44641260691951) bank149354 +149355 POINT(39.87416667280394 73.98879106134643) bank149355 +149356 POINT(40.74351803529429 73.92030867455651) bank149356 +149357 POINT(40.32482248683807 74.71066009296531) bank149357 +149358 POINT(40.502793518492965 74.79661109317081) bank149358 +149359 POINT(40.71546517244199 74.05055459147594) bank149359 +149360 POINT(39.913590414723814 74.65510636786223) bank149360 +149361 POINT(39.813659829628214 74.9948383297405) bank149361 +149362 POINT(40.28185862734462 73.76105326908316) bank149362 +149363 POINT(40.90976189309166 73.2812192597269) bank149363 +149364 POINT(39.92126536114288 74.30314306610752) bank149364 +149365 POINT(40.20677161202948 73.99311645353299) bank149365 +149366 POINT(40.703747217190056 74.67635375911291) bank149366 +149367 POINT(40.31261073209379 73.56639124173351) bank149367 +149368 POINT(41.67560454953271 74.51638029386275) bank149368 +149369 POINT(40.9698674605794 74.6035124195321) bank149369 +149370 POINT(40.864974809878994 73.07805402911697) bank149370 +149371 POINT(40.14086760326313 74.7873195043741) bank149371 +149372 POINT(40.50304609062949 73.2696051933044) bank149372 +149373 POINT(41.288358688849755 73.69335041532042) bank149373 +149374 POINT(41.03069529089113 74.85742554058058) bank149374 +149375 POINT(41.548904443305055 73.76884027623532) bank149375 +149376 POINT(39.826246924110826 73.33317505137744) bank149376 +149377 POINT(40.08873988211251 73.36430174957317) bank149377 +149378 POINT(40.3258453019402 74.56992721242261) bank149378 +149379 POINT(40.39711898248526 73.22441006313773) bank149379 +149380 POINT(39.91509691837778 74.39884498442352) bank149380 +149381 POINT(40.721665789978985 74.71493852586599) bank149381 +149382 POINT(41.52285366627395 74.49814938392953) bank149382 +149383 POINT(40.535454302216344 74.02631429586974) bank149383 +149384 POINT(41.2425896551813 74.9147625039132) bank149384 +149385 POINT(39.724016428010984 73.22223999278805) bank149385 +149386 POINT(41.544845989259585 74.4126187562408) bank149386 +149387 POINT(40.59539502994137 74.37007128656883) bank149387 +149388 POINT(40.5941369404955 73.53108422373028) bank149388 +149389 POINT(40.76025933445473 74.72785464173516) bank149389 +149390 POINT(40.04438876308808 73.57034749287688) bank149390 +149391 POINT(39.79101476033716 73.49484762016789) bank149391 +149392 POINT(41.47103687054696 73.95213607472189) bank149392 +149393 POINT(41.35714519555167 73.91227226919878) bank149393 +149394 POINT(40.29258192022367 74.30830261900309) bank149394 +149395 POINT(40.693940662714844 74.1944898710404) bank149395 +149396 POINT(41.048200016517065 73.05874795841393) bank149396 +149397 POINT(41.50913484761429 73.03596627647804) bank149397 +149398 POINT(40.373967917320876 73.78711021390059) bank149398 +149399 POINT(41.67550866587145 74.87998593600388) bank149399 +149400 POINT(40.183434515397444 73.71428753695184) bank149400 +149401 POINT(40.52805128724125 74.561541941692) bank149401 +149402 POINT(41.1466462532611 73.66149184799715) bank149402 +149403 POINT(41.15097106427568 74.60717958087463) bank149403 +149404 POINT(39.889572183817926 73.31119615737815) bank149404 +149405 POINT(41.636334615824595 73.7926886703023) bank149405 +149406 POINT(40.89421434451133 74.20895469814445) bank149406 +149407 POINT(40.07542962776504 73.3466502487373) bank149407 +149408 POINT(40.51345306074269 73.27894110073505) bank149408 +149409 POINT(41.18299469262605 74.69085340864282) bank149409 +149410 POINT(41.480589187529425 74.88804030914461) bank149410 +149411 POINT(40.96438545433671 73.78418822353113) bank149411 +149412 POINT(41.328084332972445 74.13561170375529) bank149412 +149413 POINT(40.222639667068044 74.65083422445448) bank149413 +149414 POINT(40.78281490018994 74.2512319537541) bank149414 +149415 POINT(40.29988755290127 74.40904664491755) bank149415 +149416 POINT(40.06686668245614 74.16735112175756) bank149416 +149417 POINT(41.41732597554804 74.57336975429524) bank149417 +149418 POINT(41.49553362390909 73.83180510738845) bank149418 +149419 POINT(40.90214163154585 73.26270100431714) bank149419 +149420 POINT(40.7052864504437 74.44734199224274) bank149420 +149421 POINT(39.7773094242769 73.74678424865427) bank149421 +149422 POINT(40.543649667119254 73.15267076663142) bank149422 +149423 POINT(41.24494027257999 73.92351996670345) bank149423 +149424 POINT(41.549317445673914 73.02666908545135) bank149424 +149425 POINT(41.10278066451591 73.7003668661475) bank149425 +149426 POINT(41.179133963270516 74.95345314184698) bank149426 +149427 POINT(40.14297329352613 74.96169233620444) bank149427 +149428 POINT(40.132131401970156 73.48370918672119) bank149428 +149429 POINT(41.6474300107595 74.94106706187843) bank149429 +149430 POINT(40.73482185587716 74.02563791669641) bank149430 +149431 POINT(40.56431150775361 74.38784358822215) bank149431 +149432 POINT(39.90117659233189 73.82242103321197) bank149432 +149433 POINT(41.0566506306755 74.5414369853872) bank149433 +149434 POINT(39.878302012536984 74.81251147792443) bank149434 +149435 POINT(40.01161475593018 73.64610745526255) bank149435 +149436 POINT(40.126389546282454 74.98906628522164) bank149436 +149437 POINT(39.945378988538046 74.5418744278919) bank149437 +149438 POINT(40.64076537927697 73.72136393252794) bank149438 +149439 POINT(40.600170952180264 73.34532740569809) bank149439 +149440 POINT(41.433822328241135 74.29485557707318) bank149440 +149441 POINT(40.903100056598085 74.38473945261273) bank149441 +149442 POINT(40.52733773148765 74.51700420480073) bank149442 +149443 POINT(41.60554728149934 74.03730685840513) bank149443 +149444 POINT(41.27934334401659 74.24455588731563) bank149444 +149445 POINT(41.41470160860633 74.3953436607986) bank149445 +149446 POINT(40.92362214691881 73.88749619333497) bank149446 +149447 POINT(40.704417896330874 73.9320205400963) bank149447 +149448 POINT(40.744992801910044 73.62780025947005) bank149448 +149449 POINT(41.619231556644216 74.41063518470463) bank149449 +149450 POINT(40.906140995520644 74.28424485482836) bank149450 +149451 POINT(41.377801147559836 74.01031211732533) bank149451 +149452 POINT(40.160383303845755 74.0318467632365) bank149452 +149453 POINT(39.81525365390467 74.53522164036738) bank149453 +149454 POINT(40.542068590804966 73.22649371892093) bank149454 +149455 POINT(40.12716712213427 73.60677008218097) bank149455 +149456 POINT(40.31741451919067 74.37864100253223) bank149456 +149457 POINT(39.77160603683504 74.60665572790077) bank149457 +149458 POINT(40.326383355227556 73.66935738955598) bank149458 +149459 POINT(40.77858646392511 74.34060002212574) bank149459 +149460 POINT(41.08990534336883 73.50119721764081) bank149460 +149461 POINT(40.87278977237224 73.9461748811684) bank149461 +149462 POINT(40.39288474412779 74.34806889211411) bank149462 +149463 POINT(39.92498479937694 74.22028262261637) bank149463 +149464 POINT(41.40557611023014 73.54494682329255) bank149464 +149465 POINT(40.29984139561702 74.95544117547479) bank149465 +149466 POINT(40.638792904413066 73.98868114263227) bank149466 +149467 POINT(40.64320479860937 73.9256086039563) bank149467 +149468 POINT(40.47894784210911 74.93723353179041) bank149468 +149469 POINT(40.33972720644249 73.95703817665803) bank149469 +149470 POINT(40.44486951472942 73.44421935905704) bank149470 +149471 POINT(40.60564236439206 73.37773185452757) bank149471 +149472 POINT(40.36735869901979 74.39509228364486) bank149472 +149473 POINT(40.126337786781455 73.1567188339011) bank149473 +149474 POINT(41.57233842077347 74.1961620686857) bank149474 +149475 POINT(39.981801843463955 74.78655510108625) bank149475 +149476 POINT(40.85964883721353 73.33772504393923) bank149476 +149477 POINT(40.80647254724256 74.10472567466412) bank149477 +149478 POINT(40.26798175333494 74.32741420116437) bank149478 +149479 POINT(40.929860296678605 73.10779140084294) bank149479 +149480 POINT(40.4002653966776 73.63122591630639) bank149480 +149481 POINT(40.09091605698501 73.59013047109283) bank149481 +149482 POINT(39.93642783219632 74.17058764288005) bank149482 +149483 POINT(41.23275038330028 73.36362895026382) bank149483 +149484 POINT(40.90968846930111 73.93904126728442) bank149484 +149485 POINT(41.68853730547878 74.3025147641455) bank149485 +149486 POINT(41.18939527410829 74.48130247141934) bank149486 +149487 POINT(41.414108821696814 73.16022722330601) bank149487 +149488 POINT(41.63611181839736 74.01818438208346) bank149488 +149489 POINT(40.540634512709545 73.34810771945575) bank149489 +149490 POINT(40.79570565029052 74.53848398935239) bank149490 +149491 POINT(39.91557036621489 73.43717110465444) bank149491 +149492 POINT(41.64721852168351 74.49263316302067) bank149492 +149493 POINT(40.5433207694024 74.98462850886006) bank149493 +149494 POINT(40.339769154143696 73.64534704438074) bank149494 +149495 POINT(40.17416278779231 74.10249711850534) bank149495 +149496 POINT(41.384720535978786 74.25672631412041) bank149496 +149497 POINT(39.80810495712736 74.40440379736219) bank149497 +149498 POINT(41.31669771227434 74.43346417398475) bank149498 +149499 POINT(40.89714059305134 74.83306146908458) bank149499 +149500 POINT(39.939502202720384 74.30777408188897) bank149500 +149501 POINT(39.871523780546724 73.44889202395923) bank149501 +149502 POINT(41.68635866224446 74.71853656039042) bank149502 +149503 POINT(40.09214681053819 74.73381766117016) bank149503 +149504 POINT(41.00931586490441 73.86337407340429) bank149504 +149505 POINT(40.5996911624308 73.28565279682206) bank149505 +149506 POINT(40.21751100825327 74.53454026522839) bank149506 +149507 POINT(40.85811746575573 73.650803193536) bank149507 +149508 POINT(40.249646955504105 73.56906781364297) bank149508 +149509 POINT(40.36944381226236 74.35711434697808) bank149509 +149510 POINT(41.041371598986004 73.6570857448242) bank149510 +149511 POINT(40.95276055848981 73.04530788890554) bank149511 +149512 POINT(40.15695997703373 73.67464341559428) bank149512 +149513 POINT(41.03257204072082 73.1998445361514) bank149513 +149514 POINT(39.98190886214919 74.5016713967651) bank149514 +149515 POINT(40.90172561695342 74.598498683333) bank149515 +149516 POINT(39.83600098554374 73.49488925973287) bank149516 +149517 POINT(41.091449424080174 73.62014305115008) bank149517 +149518 POINT(40.63875963264852 74.69726364357544) bank149518 +149519 POINT(40.386526900829004 74.08653430693523) bank149519 +149520 POINT(39.90922773947672 73.93153099799927) bank149520 +149521 POINT(40.29431716979871 73.52495509245777) bank149521 +149522 POINT(40.86246207808787 74.21833394398668) bank149522 +149523 POINT(40.04597048414243 73.4740787967688) bank149523 +149524 POINT(41.46721489788128 74.16293578755622) bank149524 +149525 POINT(40.245141607880896 73.73195274623342) bank149525 +149526 POINT(39.916988209968935 73.55373704430343) bank149526 +149527 POINT(40.166846820456044 73.15435476600484) bank149527 +149528 POINT(40.541339327530686 73.3437407668009) bank149528 +149529 POINT(41.485480652536864 74.81478413278884) bank149529 +149530 POINT(41.68690214668254 73.62254057926717) bank149530 +149531 POINT(39.84864002398043 74.44603739903181) bank149531 +149532 POINT(41.15996050944321 73.70091629168611) bank149532 +149533 POINT(40.4419816837565 74.84720289677256) bank149533 +149534 POINT(41.37305053010065 74.54836598604685) bank149534 +149535 POINT(39.960942674088145 74.80082051828546) bank149535 +149536 POINT(40.930789884686135 73.96076498224437) bank149536 +149537 POINT(40.877843604660036 73.11910905671245) bank149537 +149538 POINT(40.534712182854655 74.90906582759136) bank149538 +149539 POINT(41.30711011478141 74.36034336418875) bank149539 +149540 POINT(41.270011856499984 73.91815017792561) bank149540 +149541 POINT(41.578322003081226 73.61374063312981) bank149541 +149542 POINT(41.36052171495887 74.92084664311997) bank149542 +149543 POINT(40.57304125813119 74.20199439243108) bank149543 +149544 POINT(40.98732616747676 74.38149473375391) bank149544 +149545 POINT(40.732710358968546 73.51279503778777) bank149545 +149546 POINT(41.0801396453389 73.01206732662028) bank149546 +149547 POINT(40.475176684064955 74.44118437356173) bank149547 +149548 POINT(40.75930583442325 73.15738196754114) bank149548 +149549 POINT(40.85578179820142 73.70084270545765) bank149549 +149550 POINT(40.31620137779967 74.93428308100093) bank149550 +149551 POINT(40.21343324700213 74.39410693350443) bank149551 +149552 POINT(40.114495426072814 73.89229911638104) bank149552 +149553 POINT(40.035090712947394 74.62831979543392) bank149553 +149554 POINT(41.62162517024221 73.26279478177835) bank149554 +149555 POINT(41.28499788587974 73.71672506981322) bank149555 +149556 POINT(40.08631630490991 73.7402823871247) bank149556 +149557 POINT(40.74194790628909 73.84058068352448) bank149557 +149558 POINT(40.52875564135269 73.73697701947995) bank149558 +149559 POINT(41.42030442696935 73.54617993834354) bank149559 +149560 POINT(40.58929690156375 73.28853709166191) bank149560 +149561 POINT(40.504967934491106 74.62697465700734) bank149561 +149562 POINT(40.19567459165405 73.65934717903255) bank149562 +149563 POINT(39.78919943417291 74.0568996135201) bank149563 +149564 POINT(41.314854263465676 73.97126268524788) bank149564 +149565 POINT(39.808072856259166 74.96298005874884) bank149565 +149566 POINT(41.27813305050837 74.9947288127411) bank149566 +149567 POINT(40.8828786249265 74.23890530605301) bank149567 +149568 POINT(41.07043626171431 73.81702352999065) bank149568 +149569 POINT(39.99706625115377 74.41917791444835) bank149569 +149570 POINT(39.791112460805486 74.57056030173224) bank149570 +149571 POINT(40.894630641049226 73.75358760820144) bank149571 +149572 POINT(41.18334284898453 73.51966638190557) bank149572 +149573 POINT(39.77072406732171 73.70218087805677) bank149573 +149574 POINT(40.960585597718435 74.9334967862716) bank149574 +149575 POINT(41.07048777356119 74.20026634931247) bank149575 +149576 POINT(41.35005218861641 73.58258030719645) bank149576 +149577 POINT(41.054218496074455 74.98299315570422) bank149577 +149578 POINT(41.057892483230155 73.22906155296748) bank149578 +149579 POINT(40.75568741780791 73.25912749169734) bank149579 +149580 POINT(41.69277708453993 74.75954134185066) bank149580 +149581 POINT(40.77648474682464 73.28904312706027) bank149581 +149582 POINT(40.87851629606247 74.11355798600171) bank149582 +149583 POINT(39.93405401416813 73.89055427661344) bank149583 +149584 POINT(39.807532754126555 73.23419786311831) bank149584 +149585 POINT(41.316192307690244 73.79823245374531) bank149585 +149586 POINT(40.56034282613255 74.43373907119481) bank149586 +149587 POINT(39.886973878187945 73.41022285759037) bank149587 +149588 POINT(40.12445779286576 73.61916288509522) bank149588 +149589 POINT(40.85028453530389 73.9294166924131) bank149589 +149590 POINT(40.43846337622105 73.73641847809347) bank149590 +149591 POINT(41.08252423456484 73.197208542991) bank149591 +149592 POINT(41.494178427923515 74.37070697239511) bank149592 +149593 POINT(40.675625493445025 74.01553648329006) bank149593 +149594 POINT(41.22523123864077 73.56889815424944) bank149594 +149595 POINT(41.33355001711084 74.77751678555346) bank149595 +149596 POINT(41.353778333319234 74.37350344686138) bank149596 +149597 POINT(40.29066043121789 73.31358988347733) bank149597 +149598 POINT(41.373395781805044 74.05496106602895) bank149598 +149599 POINT(39.90371779260406 74.4629156525312) bank149599 +149600 POINT(40.899204992429794 73.59575472794212) bank149600 +149601 POINT(40.65192619091712 74.38219493152845) bank149601 +149602 POINT(40.54268638406754 73.930396658091) bank149602 +149603 POINT(41.0977711746459 73.16259090803943) bank149603 +149604 POINT(40.25236408120488 73.77598657850272) bank149604 +149605 POINT(41.197353967532806 74.39174501416038) bank149605 +149606 POINT(40.131669126436265 74.11687910968804) bank149606 +149607 POINT(41.19446554445709 74.8396608488073) bank149607 +149608 POINT(40.171077365282585 73.85197362778739) bank149608 +149609 POINT(41.36964462871717 74.36958689069566) bank149609 +149610 POINT(41.53605734934385 74.50039888139983) bank149610 +149611 POINT(40.19299553905762 73.38871419835439) bank149611 +149612 POINT(41.43865746801416 73.95113627636398) bank149612 +149613 POINT(41.43549595742361 74.34547652130914) bank149613 +149614 POINT(41.106487549412165 74.65714435717241) bank149614 +149615 POINT(40.358831130326855 74.91358445642992) bank149615 +149616 POINT(40.23534513508593 74.321265058476) bank149616 +149617 POINT(40.99717229000131 74.10713715371746) bank149617 +149618 POINT(40.138354584402805 74.71330682669594) bank149618 +149619 POINT(39.736117255223455 73.26708346032167) bank149619 +149620 POINT(39.8175840891274 74.12388228525188) bank149620 +149621 POINT(41.0137630827122 73.08944730284617) bank149621 +149622 POINT(40.329678646949596 73.44076738523349) bank149622 +149623 POINT(41.467362275742744 73.05365817539462) bank149623 +149624 POINT(40.41472877216721 73.40886114989173) bank149624 +149625 POINT(41.24426052900304 73.751144110473) bank149625 +149626 POINT(41.25347025159561 73.12116769865335) bank149626 +149627 POINT(41.21585089898471 73.90722329063763) bank149627 +149628 POINT(39.85016339923522 73.79393913052739) bank149628 +149629 POINT(40.653870473976596 74.76414490791363) bank149629 +149630 POINT(41.364632034357385 73.61530614739536) bank149630 +149631 POINT(41.2233602530133 74.92899124440859) bank149631 +149632 POINT(41.19654985201889 73.13545777586876) bank149632 +149633 POINT(40.158856246532565 73.28918887916039) bank149633 +149634 POINT(40.83071102745187 74.33552874671805) bank149634 +149635 POINT(41.06144182456843 73.32186025071529) bank149635 +149636 POINT(40.6514529443553 74.04990641499832) bank149636 +149637 POINT(40.760925462687524 73.38340318180809) bank149637 +149638 POINT(40.53565327250264 74.21608839305712) bank149638 +149639 POINT(40.569239630166464 73.50038830672649) bank149639 +149640 POINT(40.61032754300521 73.93954521427126) bank149640 +149641 POINT(40.726344290778776 74.86536602304147) bank149641 +149642 POINT(41.40938533180514 74.35133210942058) bank149642 +149643 POINT(40.92989112219006 73.7932263049664) bank149643 +149644 POINT(40.412104798961316 74.71951257881922) bank149644 +149645 POINT(41.51161873984514 73.60773722210178) bank149645 +149646 POINT(39.8439887645066 73.03160745843807) bank149646 +149647 POINT(40.2562976655044 73.38008726346786) bank149647 +149648 POINT(39.87671500715535 73.77590076232684) bank149648 +149649 POINT(41.13209095154047 74.67185729928431) bank149649 +149650 POINT(41.63759160770016 73.73597356853215) bank149650 +149651 POINT(40.069240903232085 73.32853729556254) bank149651 +149652 POINT(41.0257498443359 74.82650703197) bank149652 +149653 POINT(41.0009208763214 74.66632425360389) bank149653 +149654 POINT(41.21824730383087 73.31789277497626) bank149654 +149655 POINT(41.02700191152713 73.94744876290726) bank149655 +149656 POINT(40.786063457373956 74.21724660380033) bank149656 +149657 POINT(40.15102404845012 74.79995499340431) bank149657 +149658 POINT(41.17822032570404 73.04940621618097) bank149658 +149659 POINT(39.780593546713106 73.31355242038194) bank149659 +149660 POINT(40.05313296336017 74.99279019954088) bank149660 +149661 POINT(41.45485757711646 74.61591525529974) bank149661 +149662 POINT(39.93207522261673 73.55189014673547) bank149662 +149663 POINT(40.435791290965305 73.48972634034921) bank149663 +149664 POINT(40.85222272821789 73.39836309169489) bank149664 +149665 POINT(40.12391362695139 73.39275295733495) bank149665 +149666 POINT(41.44417027005602 74.74257426985143) bank149666 +149667 POINT(40.092066167313504 74.36150236734944) bank149667 +149668 POINT(39.98458038672945 73.13322831509473) bank149668 +149669 POINT(41.278012730673176 74.69256807227846) bank149669 +149670 POINT(40.84238356761285 73.8054829178204) bank149670 +149671 POINT(41.27999459677008 74.89984261331432) bank149671 +149672 POINT(39.9165384763937 73.72497944385259) bank149672 +149673 POINT(40.41893971205156 73.59793257557968) bank149673 +149674 POINT(41.458351926748655 74.96284736260282) bank149674 +149675 POINT(40.10721422575344 73.43537210701798) bank149675 +149676 POINT(41.16619624615697 74.78546073795026) bank149676 +149677 POINT(41.03065097397385 74.32829114472719) bank149677 +149678 POINT(40.07730793247345 74.6025540975231) bank149678 +149679 POINT(40.29832041529491 73.96207585396442) bank149679 +149680 POINT(40.5008718466324 73.5919507981916) bank149680 +149681 POINT(41.50963181931435 74.69554777081116) bank149681 +149682 POINT(41.14463702848109 74.07265119317165) bank149682 +149683 POINT(40.5265178542863 73.71778786219987) bank149683 +149684 POINT(41.60069246460348 73.76791567585971) bank149684 +149685 POINT(41.02463161890543 73.81627983399136) bank149685 +149686 POINT(40.83907536941286 73.34124634499344) bank149686 +149687 POINT(39.75959350743287 74.18490889721008) bank149687 +149688 POINT(41.679111279584234 73.0798643681062) bank149688 +149689 POINT(41.4229307602134 74.30476405892702) bank149689 +149690 POINT(41.095274712162656 73.77473902125972) bank149690 +149691 POINT(40.440629351231856 73.49846838956798) bank149691 +149692 POINT(40.56507170340529 74.94120895579293) bank149692 +149693 POINT(41.48946649214813 74.632000880986) bank149693 +149694 POINT(40.98331446132922 73.125682533387) bank149694 +149695 POINT(40.15258601005654 73.84042119665018) bank149695 +149696 POINT(40.80441139760935 73.21587350364226) bank149696 +149697 POINT(41.032159149199444 73.83171179553037) bank149697 +149698 POINT(40.84550498820416 73.47577035825826) bank149698 +149699 POINT(40.75638085691461 74.52642402898192) bank149699 +149700 POINT(40.964112528044545 74.09673633512499) bank149700 +149701 POINT(40.37654016534178 74.86124712755577) bank149701 +149702 POINT(40.02874633692539 74.69489506955121) bank149702 +149703 POINT(41.612385388046704 74.935965265956) bank149703 +149704 POINT(41.66421067776726 73.06203954437477) bank149704 +149705 POINT(40.04021462852955 73.3269546490433) bank149705 +149706 POINT(39.74061546458973 73.56065188591258) bank149706 +149707 POINT(41.46739677681183 74.14565822034444) bank149707 +149708 POINT(40.29713067043473 73.68913669773235) bank149708 +149709 POINT(41.370247048696406 74.0325287644355) bank149709 +149710 POINT(40.88219640765101 74.99667353374042) bank149710 +149711 POINT(40.95855142991547 74.06620518334192) bank149711 +149712 POINT(40.81415834527907 74.2275748775232) bank149712 +149713 POINT(39.97961297017604 73.9094470629681) bank149713 +149714 POINT(41.34587764923413 73.58608536155657) bank149714 +149715 POINT(40.305154477080734 73.63238678530361) bank149715 +149716 POINT(41.486584659897844 74.92060956671342) bank149716 +149717 POINT(41.6565575546743 73.86066850368633) bank149717 +149718 POINT(40.588003267486705 73.43128084306713) bank149718 +149719 POINT(40.634278196696435 73.73646495415454) bank149719 +149720 POINT(40.178072305208 73.8853821071513) bank149720 +149721 POINT(40.96399495040098 73.74884782561006) bank149721 +149722 POINT(41.26085760509088 74.65363112315595) bank149722 +149723 POINT(41.198899158999396 73.75254167730066) bank149723 +149724 POINT(41.07918393625338 73.09779142253166) bank149724 +149725 POINT(41.628727369486455 73.7385087863471) bank149725 +149726 POINT(40.61137053336144 74.28131889747252) bank149726 +149727 POINT(41.685782447801806 73.61580688431317) bank149727 +149728 POINT(41.23325335713073 74.808389599327) bank149728 +149729 POINT(41.002915884363716 73.03911847159394) bank149729 +149730 POINT(40.977424839823684 74.82204039864125) bank149730 +149731 POINT(41.59032477567001 74.20694557690692) bank149731 +149732 POINT(41.41069067154257 74.73181000218533) bank149732 +149733 POINT(40.979710520811935 74.42738528038949) bank149733 +149734 POINT(40.2432691109612 73.50383635118786) bank149734 +149735 POINT(41.26795541477703 73.32176487161885) bank149735 +149736 POINT(40.76501656272908 73.8651439658617) bank149736 +149737 POINT(41.51129254845411 74.64335309052105) bank149737 +149738 POINT(41.59016806205223 74.09893653952196) bank149738 +149739 POINT(39.986272013272156 73.17431717311949) bank149739 +149740 POINT(41.586010384678815 73.35302055349231) bank149740 +149741 POINT(41.04862168329269 74.70937680868367) bank149741 +149742 POINT(39.87859823639741 73.91192135721299) bank149742 +149743 POINT(41.33309656690042 73.36534158850426) bank149743 +149744 POINT(40.990971115350355 74.96860897497196) bank149744 +149745 POINT(41.54458898187739 73.62256869151102) bank149745 +149746 POINT(41.5561928438509 73.53300459632857) bank149746 +149747 POINT(40.449352820383325 73.59120267717253) bank149747 +149748 POINT(39.99251045855696 73.76731117610034) bank149748 +149749 POINT(39.94243589168004 73.92831482350608) bank149749 +149750 POINT(40.09551528541123 74.06166399149961) bank149750 +149751 POINT(39.896186088242786 74.49983538127186) bank149751 +149752 POINT(40.265916810913254 73.31382103598942) bank149752 +149753 POINT(41.70669756074624 73.15929258639937) bank149753 +149754 POINT(41.1576009475689 74.99609481092017) bank149754 +149755 POINT(41.20777436456809 73.4663276569311) bank149755 +149756 POINT(40.803281807310114 73.85965709507661) bank149756 +149757 POINT(41.0121219546038 74.82445483991923) bank149757 +149758 POINT(39.86871317560025 73.22073810971233) bank149758 +149759 POINT(40.152805217364886 74.99684020142794) bank149759 +149760 POINT(40.13471340059569 73.08290235080943) bank149760 +149761 POINT(41.10246204624397 73.81336372234058) bank149761 +149762 POINT(41.091218552597056 74.26409928967126) bank149762 +149763 POINT(40.32025740243926 74.20184872412327) bank149763 +149764 POINT(39.862983254904066 73.27091046813514) bank149764 +149765 POINT(40.197283491523386 73.13303300346458) bank149765 +149766 POINT(41.056683314299285 74.78097934857189) bank149766 +149767 POINT(41.15219425870537 73.38943263152714) bank149767 +149768 POINT(40.16398484918004 74.81595920681474) bank149768 +149769 POINT(41.28632485529208 74.95428386890661) bank149769 +149770 POINT(40.90062252901053 73.38116247640302) bank149770 +149771 POINT(40.9828450521221 73.23687087702285) bank149771 +149772 POINT(40.42491980644784 73.15594348107312) bank149772 +149773 POINT(41.54827960149074 73.44421182613353) bank149773 +149774 POINT(40.13664190741751 73.17536878696222) bank149774 +149775 POINT(40.05259647000816 74.7667555931212) bank149775 +149776 POINT(40.184585598063904 73.28387517212047) bank149776 +149777 POINT(39.831779341133505 74.47445958564175) bank149777 +149778 POINT(40.89004062868508 73.5768364062705) bank149778 +149779 POINT(40.75767012973224 74.8186257886262) bank149779 +149780 POINT(40.65191821340203 73.42322316634662) bank149780 +149781 POINT(40.586758452452536 73.82621223545597) bank149781 +149782 POINT(40.618834107009945 73.65703825616268) bank149782 +149783 POINT(40.87349256733223 74.50984590106647) bank149783 +149784 POINT(40.59076095324439 73.02718978083604) bank149784 +149785 POINT(41.03793028423245 73.37744796419088) bank149785 +149786 POINT(41.71197287201294 74.64665574204568) bank149786 +149787 POINT(40.57482163462227 74.58740041522991) bank149787 +149788 POINT(41.0607573805429 73.9507471534012) bank149788 +149789 POINT(41.18552499501216 73.66793604556042) bank149789 +149790 POINT(40.34521536492653 74.91194900987561) bank149790 +149791 POINT(41.353269361081516 74.41411124603236) bank149791 +149792 POINT(40.288575750547416 74.76964774531127) bank149792 +149793 POINT(40.29419397168397 73.59005227488491) bank149793 +149794 POINT(40.3913774750471 73.74736631283247) bank149794 +149795 POINT(41.420546878866894 74.9577612719749) bank149795 +149796 POINT(40.177414875889326 73.65352375266428) bank149796 +149797 POINT(40.15052165191 73.28400249902353) bank149797 +149798 POINT(41.572598796027535 73.49322775079938) bank149798 +149799 POINT(39.82629846760336 73.78331055003783) bank149799 +149800 POINT(39.92678327419336 74.84860488348511) bank149800 +149801 POINT(39.8511389415091 73.7281994957294) bank149801 +149802 POINT(41.273790540735 74.85685494930136) bank149802 +149803 POINT(40.63301747785835 73.3901263577395) bank149803 +149804 POINT(41.704333195229 73.4823167074175) bank149804 +149805 POINT(39.97281197470171 73.25898512025715) bank149805 +149806 POINT(40.24888233028283 73.93939617569583) bank149806 +149807 POINT(40.21273007337019 73.6075034770946) bank149807 +149808 POINT(40.240987679187725 74.96890408451637) bank149808 +149809 POINT(40.600914699046186 74.9698389444388) bank149809 +149810 POINT(40.55776006477789 74.16689872126572) bank149810 +149811 POINT(41.28124616251197 74.8458771499401) bank149811 +149812 POINT(41.115263111495324 73.2850269494766) bank149812 +149813 POINT(41.136084127731095 73.62354222426461) bank149813 +149814 POINT(39.82480164441649 73.42496660046514) bank149814 +149815 POINT(39.99508839617235 74.79286772573714) bank149815 +149816 POINT(40.33365771394006 74.42647676053787) bank149816 +149817 POINT(39.7419690063748 74.08060224762399) bank149817 +149818 POINT(39.97552686565497 73.57651245431356) bank149818 +149819 POINT(40.22759222071853 74.69911882213044) bank149819 +149820 POINT(40.106008906929404 74.3539553516753) bank149820 +149821 POINT(40.18879009929668 73.98329363666394) bank149821 +149822 POINT(39.7271705300282 73.83279833878973) bank149822 +149823 POINT(39.77721197790005 73.81427862750864) bank149823 +149824 POINT(41.575515145393396 74.08353911396142) bank149824 +149825 POINT(40.076486717651115 74.14166776388528) bank149825 +149826 POINT(41.29427554741859 74.74032798353296) bank149826 +149827 POINT(40.038951824735605 74.96205110407224) bank149827 +149828 POINT(41.50201928232314 74.25696347209403) bank149828 +149829 POINT(40.98765785538519 74.2770572586224) bank149829 +149830 POINT(40.34421228988476 74.11766818914461) bank149830 +149831 POINT(41.35431075203075 74.90444337710987) bank149831 +149832 POINT(41.46331897660888 74.46728372344297) bank149832 +149833 POINT(40.723925135411484 74.51731353354545) bank149833 +149834 POINT(40.83367477836604 73.52135396642892) bank149834 +149835 POINT(41.44037251987133 73.84113680463092) bank149835 +149836 POINT(40.0611039661404 74.46982272981525) bank149836 +149837 POINT(41.13173106728882 73.77314996607916) bank149837 +149838 POINT(40.29673442334602 73.30974809929839) bank149838 +149839 POINT(40.564691981545195 74.07456602138892) bank149839 +149840 POINT(40.29533015002226 74.71940048107503) bank149840 +149841 POINT(41.34997813458858 73.90358337388342) bank149841 +149842 POINT(41.05295683735723 74.26388228038297) bank149842 +149843 POINT(41.62784734748114 73.03373843218854) bank149843 +149844 POINT(40.46676695844185 73.48159289473014) bank149844 +149845 POINT(40.2201881726917 74.79944635728104) bank149845 +149846 POINT(41.53132755427818 73.77166791624958) bank149846 +149847 POINT(40.517136070055614 73.18715206611373) bank149847 +149848 POINT(40.541306410818294 74.40836116924892) bank149848 +149849 POINT(41.22522232576829 74.87258761719687) bank149849 +149850 POINT(41.600029311442995 74.14647383233786) bank149850 +149851 POINT(40.93304803670955 74.015847651151) bank149851 +149852 POINT(40.167877929940204 74.87424991486148) bank149852 +149853 POINT(40.537541715823245 74.56750046822897) bank149853 +149854 POINT(40.803418212843724 74.06749677732823) bank149854 +149855 POINT(40.495949472446576 74.7460736897836) bank149855 +149856 POINT(40.44583792179945 73.15720763863368) bank149856 +149857 POINT(40.66669930510994 73.03739800115606) bank149857 +149858 POINT(40.77816053271819 73.04958131757634) bank149858 +149859 POINT(40.766985046059524 73.9647033178031) bank149859 +149860 POINT(39.71920052783707 73.96126249890219) bank149860 +149861 POINT(41.16423077974049 74.01126389114289) bank149861 +149862 POINT(41.18719361609947 74.29402889770324) bank149862 +149863 POINT(40.48594381697363 74.56692946118082) bank149863 +149864 POINT(41.356574569607865 74.50380219424834) bank149864 +149865 POINT(41.53173786459421 73.24376610992299) bank149865 +149866 POINT(41.559874099680684 73.23524057934294) bank149866 +149867 POINT(39.81874209259849 74.89020491976639) bank149867 +149868 POINT(40.98550813290648 74.29731625227498) bank149868 +149869 POINT(39.984757238777924 73.26759264754203) bank149869 +149870 POINT(40.79597208341836 73.65805875430327) bank149870 +149871 POINT(40.72828772495481 73.74589657544149) bank149871 +149872 POINT(40.34424006206201 73.94153911760749) bank149872 +149873 POINT(41.05848240938002 74.31514643903562) bank149873 +149874 POINT(40.74991541031544 73.81382642839553) bank149874 +149875 POINT(39.85716696109355 74.74266706869798) bank149875 +149876 POINT(40.279210178358255 74.9689718866908) bank149876 +149877 POINT(41.062562720306694 74.64362383539346) bank149877 +149878 POINT(40.02028477313832 74.9719270728423) bank149878 +149879 POINT(41.472917793820024 74.13322271861941) bank149879 +149880 POINT(40.55520009407056 74.77840419750825) bank149880 +149881 POINT(40.8907829536384 74.2978575010603) bank149881 +149882 POINT(41.65108761687116 74.76251831407507) bank149882 +149883 POINT(41.17582455468417 74.40260363237296) bank149883 +149884 POINT(41.18036298053908 74.8607690494381) bank149884 +149885 POINT(39.89516186666654 74.94336562632601) bank149885 +149886 POINT(40.64066587674488 73.63249395967604) bank149886 +149887 POINT(40.09890615906308 74.2286509360134) bank149887 +149888 POINT(39.91885150971648 73.15826858269428) bank149888 +149889 POINT(40.47998159087646 74.41432444065853) bank149889 +149890 POINT(41.156491162212916 74.68779445462141) bank149890 +149891 POINT(41.38136109213018 74.98200226575813) bank149891 +149892 POINT(40.56161228540013 74.43058054770142) bank149892 +149893 POINT(39.7264797629212 73.51024232330435) bank149893 +149894 POINT(41.44716552706578 73.30979006629974) bank149894 +149895 POINT(40.26800299036407 73.024293393755) bank149895 +149896 POINT(40.145372358021035 74.53607450583473) bank149896 +149897 POINT(41.007117935573255 74.23943480962973) bank149897 +149898 POINT(40.50907407091178 74.23334902258476) bank149898 +149899 POINT(41.48272051503704 73.32588143552874) bank149899 +149900 POINT(41.18333813453162 74.53836768884433) bank149900 +149901 POINT(41.09101842480363 74.00786610029786) bank149901 +149902 POINT(40.29306709360164 74.02796651819206) bank149902 +149903 POINT(41.07713698407522 74.57898290977002) bank149903 +149904 POINT(39.93509458273687 73.36107697635461) bank149904 +149905 POINT(40.32620589702383 73.69890317651561) bank149905 +149906 POINT(39.84922437259125 73.86346389064941) bank149906 +149907 POINT(39.727507803184594 73.72673737362277) bank149907 +149908 POINT(40.06125979975474 74.2438973226605) bank149908 +149909 POINT(40.621556720805934 73.54117510701649) bank149909 +149910 POINT(40.525851373681746 74.642331716415) bank149910 +149911 POINT(41.052312827762954 74.6147032462932) bank149911 +149912 POINT(40.17653667489875 74.77781827263308) bank149912 +149913 POINT(40.81512133140897 73.4532574547115) bank149913 +149914 POINT(41.256405509889134 73.46370911227397) bank149914 +149915 POINT(41.05126024382874 74.94831722132884) bank149915 +149916 POINT(39.7499874227432 74.45456146593439) bank149916 +149917 POINT(39.857293692686156 73.94284434322144) bank149917 +149918 POINT(41.1231710063956 73.25310866916983) bank149918 +149919 POINT(40.25536015144048 74.4330718316129) bank149919 +149920 POINT(41.3005015956703 73.6953768235249) bank149920 +149921 POINT(40.36493072151202 73.53866774912515) bank149921 +149922 POINT(40.199985895359035 73.74358319728336) bank149922 +149923 POINT(39.86778527668204 73.22015215439781) bank149923 +149924 POINT(40.04070865300617 73.1368516468023) bank149924 +149925 POINT(40.18928861011086 74.32638570066027) bank149925 +149926 POINT(41.30776306701148 73.74703721472098) bank149926 +149927 POINT(41.13600560170015 73.47135347802991) bank149927 +149928 POINT(40.86589080079966 73.3566734604754) bank149928 +149929 POINT(41.0980705379069 73.820458474663) bank149929 +149930 POINT(39.94661395757054 73.04384460076996) bank149930 +149931 POINT(41.44801468677928 73.0647689977716) bank149931 +149932 POINT(40.209379010801094 73.04893276902818) bank149932 +149933 POINT(39.84959910493033 73.27224498432204) bank149933 +149934 POINT(40.517912398079076 73.73308520166078) bank149934 +149935 POINT(40.54546487124792 74.83162084585564) bank149935 +149936 POINT(41.63341836682594 73.95791365440324) bank149936 +149937 POINT(41.28604906192169 74.87833260069198) bank149937 +149938 POINT(40.98132885428301 73.34134936233596) bank149938 +149939 POINT(41.40896747498363 73.77016462112594) bank149939 +149940 POINT(41.60311927179143 73.48499384160701) bank149940 +149941 POINT(39.81462158031381 73.97351630530773) bank149941 +149942 POINT(41.397329308648175 74.74982281484466) bank149942 +149943 POINT(40.849638304687545 74.04539307299805) bank149943 +149944 POINT(39.8353001868233 73.82037506261553) bank149944 +149945 POINT(40.68756052176921 73.72511063170779) bank149945 +149946 POINT(40.691110143259564 74.07737608431013) bank149946 +149947 POINT(41.227583862026144 73.30847788348093) bank149947 +149948 POINT(41.63034482752357 73.64934891052258) bank149948 +149949 POINT(40.62025980392626 74.05154080910546) bank149949 +149950 POINT(41.019718574935844 73.02188763167449) bank149950 +149951 POINT(39.72792575376995 74.8420131020271) bank149951 +149952 POINT(41.500751630374594 74.07275723429763) bank149952 +149953 POINT(40.963165053122104 74.20766958633533) bank149953 +149954 POINT(40.57443321409885 74.32917820825081) bank149954 +149955 POINT(40.92006875309875 74.99938270774805) bank149955 +149956 POINT(40.962671168401315 73.9109230758771) bank149956 +149957 POINT(41.0298918720467 73.0679078577222) bank149957 +149958 POINT(41.29198703099539 74.49944421250706) bank149958 +149959 POINT(41.46447706973165 73.41504053424582) bank149959 +149960 POINT(41.69948847234987 73.24412787573587) bank149960 +149961 POINT(39.76581808426052 73.88139884885618) bank149961 +149962 POINT(40.06712467841777 73.6616569409007) bank149962 +149963 POINT(41.55416150192372 74.6880085367889) bank149963 +149964 POINT(39.811025084784845 73.32551085361163) bank149964 +149965 POINT(40.58640214078003 74.18864207173313) bank149965 +149966 POINT(40.40893549493762 74.20236206362439) bank149966 +149967 POINT(39.95661581014974 74.37768429515023) bank149967 +149968 POINT(41.70529025826792 74.5198821083096) bank149968 +149969 POINT(41.112767850182564 73.16417416596927) bank149969 +149970 POINT(41.32617270827501 74.27958952363706) bank149970 +149971 POINT(40.15738430963014 74.99962704966543) bank149971 +149972 POINT(40.34394407749085 74.80527651760839) bank149972 +149973 POINT(40.95979616659373 73.10793403593763) bank149973 +149974 POINT(41.628457482376376 74.90537795674145) bank149974 +149975 POINT(40.419914882906504 73.30114904286808) bank149975 +149976 POINT(40.13666568007907 73.20510181884714) bank149976 +149977 POINT(40.11899385551455 74.10920301727148) bank149977 +149978 POINT(39.84488026885202 74.67358836308475) bank149978 +149979 POINT(41.08905535021229 73.4152002652956) bank149979 +149980 POINT(40.80533410281591 74.50663413086369) bank149980 +149981 POINT(40.172496546652994 73.40087331675528) bank149981 +149982 POINT(40.181876348926735 74.44922081983191) bank149982 +149983 POINT(39.780946069302885 73.95136856269089) bank149983 +149984 POINT(40.85990825131201 74.85019948676877) bank149984 +149985 POINT(41.52160158731053 73.49753493648768) bank149985 +149986 POINT(40.8818791642875 74.30739934922117) bank149986 +149987 POINT(40.98472598298627 74.1958054399532) bank149987 +149988 POINT(39.77217235538865 73.06010898308462) bank149988 +149989 POINT(39.86331285771225 74.41423459252357) bank149989 +149990 POINT(39.983832749436495 74.19452273746761) bank149990 +149991 POINT(41.01885674270113 73.56463982035214) bank149991 +149992 POINT(40.40849496690227 74.04010982568033) bank149992 +149993 POINT(40.14362192603418 74.91704580035521) bank149993 +149994 POINT(40.16416261167358 73.95146419996289) bank149994 +149995 POINT(41.19245933328226 74.1571022910978) bank149995 +149996 POINT(40.838183412745515 74.48322248235392) bank149996 +149997 POINT(39.73623027511049 74.69446026931129) bank149997 +149998 POINT(39.86454779770585 74.02838829714895) bank149998 +149999 POINT(40.710242190044454 73.74471135573351) bank149999 +150000 POINT(41.288004963456544 73.6681192015655) bank150000 +150001 POINT(39.86603957431406 73.95844003143422) bank150001 +150002 POINT(40.86916855526476 73.72206901595183) bank150002 +150003 POINT(40.98213003453403 73.71379821197925) bank150003 +150004 POINT(41.67089570580194 74.87112145507152) bank150004 +150005 POINT(41.46121368793418 73.5079593926663) bank150005 +150006 POINT(41.0905418644028 73.58066098743883) bank150006 +150007 POINT(39.87370491267805 73.67421216999587) bank150007 +150008 POINT(41.21576629926685 73.35145218517718) bank150008 +150009 POINT(40.36236996454798 74.3995087474587) bank150009 +150010 POINT(40.3556389360389 74.16745017686202) bank150010 +150011 POINT(41.18550039776749 74.61195509082515) bank150011 +150012 POINT(41.18831930127862 74.23447066875126) bank150012 +150013 POINT(40.34843172831184 74.89538122569836) bank150013 +150014 POINT(41.54656081321426 73.71936102823827) bank150014 +150015 POINT(40.97003720048845 73.43970384652091) bank150015 +150016 POINT(41.15506150420937 73.96063719859265) bank150016 +150017 POINT(41.10624982631433 73.42622552240121) bank150017 +150018 POINT(40.72586868868478 74.20613781916998) bank150018 +150019 POINT(39.72585736244878 74.47296417648441) bank150019 +150020 POINT(40.7851060226464 74.52486924236484) bank150020 +150021 POINT(40.649295928784944 73.05906852000008) bank150021 +150022 POINT(39.91013008523668 74.13009813551342) bank150022 +150023 POINT(40.74041984418891 73.12670060440243) bank150023 +150024 POINT(40.87192412307504 73.64167286905453) bank150024 +150025 POINT(41.2344368862064 73.75743449673959) bank150025 +150026 POINT(40.735693891802384 74.89481961874743) bank150026 +150027 POINT(41.69127977158788 73.92212985885017) bank150027 +150028 POINT(41.63146782778366 74.07450207996429) bank150028 +150029 POINT(39.72639615186171 73.45253887710754) bank150029 +150030 POINT(40.376883776759236 74.0444296475908) bank150030 +150031 POINT(39.84659489441684 73.14546333806568) bank150031 +150032 POINT(41.25685501281794 73.63894258832616) bank150032 +150033 POINT(40.468988147586295 74.1151590168808) bank150033 +150034 POINT(41.044744231495315 74.16161328018907) bank150034 +150035 POINT(41.7088685092702 74.13968434249944) bank150035 +150036 POINT(40.360094841615414 73.01801920604495) bank150036 +150037 POINT(41.25185615775748 73.24932342304938) bank150037 +150038 POINT(40.22846412567162 73.42846051985875) bank150038 +150039 POINT(40.13265633246563 74.85507177090864) bank150039 +150040 POINT(41.49656310018801 74.96029924156112) bank150040 +150041 POINT(41.29654282968252 74.51917445114776) bank150041 +150042 POINT(41.11334712536399 74.30504037064674) bank150042 +150043 POINT(41.34260450214025 74.347039532849) bank150043 +150044 POINT(40.23996905706483 74.69479256898693) bank150044 +150045 POINT(41.33299747440604 73.67273160501792) bank150045 +150046 POINT(40.26867964196505 73.31932930809545) bank150046 +150047 POINT(41.612543882544735 74.88918791382972) bank150047 +150048 POINT(40.62347310416933 74.97327709334829) bank150048 +150049 POINT(41.60212100333211 74.27364307858312) bank150049 +150050 POINT(40.0836148639145 74.17694861016736) bank150050 +150051 POINT(40.731980348407504 74.7099096513572) bank150051 +150052 POINT(41.14865984499203 73.54405532416429) bank150052 +150053 POINT(41.25745071987938 73.33235143425571) bank150053 +150054 POINT(39.75985695969612 74.85781891136372) bank150054 +150055 POINT(41.569401222280945 74.72036067755747) bank150055 +150056 POINT(40.80569217089953 73.38366506649835) bank150056 +150057 POINT(41.642750720323974 73.79810716643186) bank150057 +150058 POINT(40.26954321502563 73.83355004967179) bank150058 +150059 POINT(41.3204513000554 73.99741435217871) bank150059 +150060 POINT(41.001014120334354 73.00802564708363) bank150060 +150061 POINT(41.20174426000979 74.24300573729212) bank150061 +150062 POINT(41.36678434737131 74.2090418300222) bank150062 +150063 POINT(41.34017265253283 74.48618524568604) bank150063 +150064 POINT(40.63161972899402 74.09337533581207) bank150064 +150065 POINT(40.71126152335777 73.50111434063531) bank150065 +150066 POINT(41.534991893250364 74.2636498954318) bank150066 +150067 POINT(40.488691654196536 74.70747508789908) bank150067 +150068 POINT(40.39927533597518 74.32388708638209) bank150068 +150069 POINT(41.04150369295692 74.64307463749533) bank150069 +150070 POINT(40.97638114205728 74.70163193583186) bank150070 +150071 POINT(40.82937276729946 74.04352174943037) bank150071 +150072 POINT(41.05582623824026 73.84382369671317) bank150072 +150073 POINT(41.63566410077332 74.91913536891192) bank150073 +150074 POINT(40.12363289946468 74.9579238085752) bank150074 +150075 POINT(41.68107810905199 74.02889285992327) bank150075 +150076 POINT(40.81940703421982 74.08151575764433) bank150076 +150077 POINT(40.074771441933756 74.83362383115488) bank150077 +150078 POINT(40.560167353112746 73.98075197998739) bank150078 +150079 POINT(40.988079802284766 74.7339846824583) bank150079 +150080 POINT(41.233461248453196 74.3662110036621) bank150080 +150081 POINT(40.70209535456781 73.12125589981736) bank150081 +150082 POINT(39.9774052787662 74.16728636731082) bank150082 +150083 POINT(40.18373227103779 74.98594556372262) bank150083 +150084 POINT(41.15092298753287 73.36334722438403) bank150084 +150085 POINT(41.26567647259373 74.13678701537083) bank150085 +150086 POINT(40.32010642852987 73.28697180291292) bank150086 +150087 POINT(40.493215722092835 73.72642469657312) bank150087 +150088 POINT(41.26571887752633 73.73845287015922) bank150088 +150089 POINT(40.971406651755245 73.33872865020838) bank150089 +150090 POINT(39.96880151473764 74.65122150149793) bank150090 +150091 POINT(41.65367530957687 74.45916986652551) bank150091 +150092 POINT(41.398613179879376 73.37365686218878) bank150092 +150093 POINT(39.85599924277379 74.69565474184206) bank150093 +150094 POINT(40.92875293347635 73.04164258408953) bank150094 +150095 POINT(40.34699769718679 74.60066014155841) bank150095 +150096 POINT(39.83272792874442 73.15926582665777) bank150096 +150097 POINT(41.49516694517702 73.46685293770854) bank150097 +150098 POINT(40.138140393732506 74.73475966140629) bank150098 +150099 POINT(40.30895623287059 74.59103932375835) bank150099 +150100 POINT(40.654853707153755 73.7616696955446) bank150100 +150101 POINT(41.63151888332056 73.09492620689369) bank150101 +150102 POINT(40.492962215395536 73.819062780963) bank150102 +150103 POINT(40.34333181793856 73.85694937297369) bank150103 +150104 POINT(41.10244901428893 74.3444031476361) bank150104 +150105 POINT(41.16159708985135 74.54392760588885) bank150105 +150106 POINT(39.73885689812324 73.09356766836088) bank150106 +150107 POINT(40.3389535187003 74.46983946986417) bank150107 +150108 POINT(40.58000169131703 73.30323601822556) bank150108 +150109 POINT(40.89857506661988 74.83335971931011) bank150109 +150110 POINT(39.777529867770326 74.1579601335783) bank150110 +150111 POINT(41.19461200130728 73.94920904482859) bank150111 +150112 POINT(40.34472550142128 73.77942647518698) bank150112 +150113 POINT(40.27575047667063 74.9226931880457) bank150113 +150114 POINT(41.20810917409494 73.83586267161006) bank150114 +150115 POINT(41.427237927509864 73.38480527695553) bank150115 +150116 POINT(39.841134933749785 73.39288717906743) bank150116 +150117 POINT(39.74556977706781 73.03900662018583) bank150117 +150118 POINT(40.235224647582754 73.90387579313624) bank150118 +150119 POINT(40.04649068625301 74.51085658730324) bank150119 +150120 POINT(40.47593505493366 74.44279904397577) bank150120 +150121 POINT(41.24071624368959 74.40407173940646) bank150121 +150122 POINT(41.34959233793569 74.85023978462499) bank150122 +150123 POINT(40.300683827285155 74.02707413151964) bank150123 +150124 POINT(39.92667742817029 73.46067447653093) bank150124 +150125 POINT(39.878416622382254 74.67043059139951) bank150125 +150126 POINT(40.5354794994745 73.29828181848939) bank150126 +150127 POINT(41.69573689471525 74.15548736397547) bank150127 +150128 POINT(41.54365094753959 74.01505824954107) bank150128 +150129 POINT(41.42567547974807 74.60598648911268) bank150129 +150130 POINT(40.63472811263755 74.09811508607652) bank150130 +150131 POINT(40.5847065300975 73.59812188688366) bank150131 +150132 POINT(41.534568589528256 73.97497451509159) bank150132 +150133 POINT(40.030974517926225 74.91997522738284) bank150133 +150134 POINT(41.238020350751945 73.41861942883038) bank150134 +150135 POINT(40.792304345539264 73.740208321414) bank150135 +150136 POINT(40.6273945487886 73.14465313395071) bank150136 +150137 POINT(41.52607329192902 74.15297403795422) bank150137 +150138 POINT(40.794924742750794 74.28701186514708) bank150138 +150139 POINT(41.08939498292736 73.96937392617826) bank150139 +150140 POINT(40.59712462493821 73.46049414835024) bank150140 +150141 POINT(40.52839728567347 73.75045637424222) bank150141 +150142 POINT(40.354958701547716 73.74838966408012) bank150142 +150143 POINT(40.26373828064846 74.01358455784933) bank150143 +150144 POINT(39.9354846200603 73.9407866859513) bank150144 +150145 POINT(40.80855622543678 74.17225737385725) bank150145 +150146 POINT(41.677535462922506 73.64317913870742) bank150146 +150147 POINT(40.51023606596203 73.8586629296748) bank150147 +150148 POINT(40.72228363983418 74.57150800852362) bank150148 +150149 POINT(41.3917010086858 74.57568483664596) bank150149 +150150 POINT(41.626479419134036 74.77770331671906) bank150150 +150151 POINT(40.45900408412856 74.29130779278914) bank150151 +150152 POINT(40.67479877271431 74.35984981790611) bank150152 +150153 POINT(40.54265882473003 73.35136410280596) bank150153 +150154 POINT(40.73844219961478 74.32771706419499) bank150154 +150155 POINT(40.205655433290474 74.01045889833854) bank150155 +150156 POINT(40.17412779431356 73.71025589640621) bank150156 +150157 POINT(40.915918566784995 74.31166475025329) bank150157 +150158 POINT(40.141585435875214 74.86309890152931) bank150158 +150159 POINT(41.20313347656072 73.58120524394333) bank150159 +150160 POINT(40.39238175285765 74.66949934243429) bank150160 +150161 POINT(40.644057803285186 74.77472760585414) bank150161 +150162 POINT(40.02292210921842 74.99080602605102) bank150162 +150163 POINT(40.47391551110498 74.29467505837383) bank150163 +150164 POINT(39.945700060992166 74.69790459116692) bank150164 +150165 POINT(40.43222415431158 74.9492922227458) bank150165 +150166 POINT(41.66533817110883 73.57704417818668) bank150166 +150167 POINT(39.72509974617006 73.77393607195958) bank150167 +150168 POINT(40.486122039046485 74.83175363031667) bank150168 +150169 POINT(40.489295161714566 73.6177832585129) bank150169 +150170 POINT(39.92490527136432 74.44089861143944) bank150170 +150171 POINT(41.373455972000315 73.86593093270636) bank150171 +150172 POINT(40.500088669725024 73.20112691401364) bank150172 +150173 POINT(41.15738025595381 74.27509898650432) bank150173 +150174 POINT(41.42775204023221 74.87353801664749) bank150174 +150175 POINT(40.09984638041247 73.66384979921314) bank150175 +150176 POINT(39.73197290419621 73.94147658897074) bank150176 +150177 POINT(40.98867685499921 74.89100890535168) bank150177 +150178 POINT(41.39324207437553 73.25123084250947) bank150178 +150179 POINT(41.037328463187286 74.8335558026487) bank150179 +150180 POINT(40.902775135995554 74.90961151073549) bank150180 +150181 POINT(40.38374963570389 74.32466463924405) bank150181 +150182 POINT(40.96476063858311 74.05225122620067) bank150182 +150183 POINT(40.98854514782329 74.499594810956) bank150183 +150184 POINT(41.066942768506664 73.36206765760288) bank150184 +150185 POINT(40.65231118103525 74.8714256855417) bank150185 +150186 POINT(41.66428718749941 74.45898854097034) bank150186 +150187 POINT(41.486674321128945 73.46801709935603) bank150187 +150188 POINT(40.352444720901055 73.23647562497216) bank150188 +150189 POINT(41.54970893162831 74.94295460992635) bank150189 +150190 POINT(40.05937089673073 74.09307660266089) bank150190 +150191 POINT(40.026643392654826 73.59501614263216) bank150191 +150192 POINT(41.313181356452404 74.82266860558876) bank150192 +150193 POINT(40.46969886195786 73.54750630593175) bank150193 +150194 POINT(41.16701199508489 74.5414850465759) bank150194 +150195 POINT(40.99331617270931 73.91627241989318) bank150195 +150196 POINT(40.8563271756723 74.26333988452049) bank150196 +150197 POINT(40.42258113670336 73.16293050768802) bank150197 +150198 POINT(41.09474332721326 74.27702097208164) bank150198 +150199 POINT(40.61797242931375 74.83287361782635) bank150199 +150200 POINT(40.99481236689471 73.18957545918352) bank150200 +150201 POINT(40.218217886746224 73.63282470088258) bank150201 +150202 POINT(39.76343827485249 73.97383913554643) bank150202 +150203 POINT(41.05782771650146 73.36361284589607) bank150203 +150204 POINT(41.67726935553713 74.7907390500269) bank150204 +150205 POINT(40.55578086143178 74.9094370286218) bank150205 +150206 POINT(40.09778113515557 74.11159382749918) bank150206 +150207 POINT(41.0076472853031 73.90260034918471) bank150207 +150208 POINT(41.14586964947859 73.75216816137329) bank150208 +150209 POINT(40.614339023258474 74.09413894079516) bank150209 +150210 POINT(39.882602862837885 73.8018407798266) bank150210 +150211 POINT(41.4046147843725 73.98319534715728) bank150211 +150212 POINT(40.093482846701676 74.10740983014364) bank150212 +150213 POINT(40.33535571651836 73.72075092183458) bank150213 +150214 POINT(40.29013852639929 73.55908026897957) bank150214 +150215 POINT(40.89712569814342 73.29383597378016) bank150215 +150216 POINT(40.25868078755691 73.90904708762248) bank150216 +150217 POINT(41.01719201379732 73.68110719706152) bank150217 +150218 POINT(40.90271858669912 74.1898241038616) bank150218 +150219 POINT(39.75073934149089 74.06210564330223) bank150219 +150220 POINT(40.60082825086772 73.09795873904605) bank150220 +150221 POINT(40.45840812779237 73.53181490945823) bank150221 +150222 POINT(40.307132646163616 73.96810499888043) bank150222 +150223 POINT(41.37575678323524 74.52008539808527) bank150223 +150224 POINT(41.23629221660687 74.46225865849357) bank150224 +150225 POINT(41.13074011558631 73.12578180210387) bank150225 +150226 POINT(40.29051038709046 74.34253419869565) bank150226 +150227 POINT(40.84391230263626 73.86549001840558) bank150227 +150228 POINT(40.107068628252094 73.97162650240526) bank150228 +150229 POINT(39.956759442454384 74.46116723109957) bank150229 +150230 POINT(40.6531480733524 74.25538413044615) bank150230 +150231 POINT(40.42152016341494 73.59429225333245) bank150231 +150232 POINT(41.43130326024402 74.87367617034332) bank150232 +150233 POINT(40.565563085541484 73.08127028845908) bank150233 +150234 POINT(40.29348000129809 73.64829808680477) bank150234 +150235 POINT(41.32807606216617 73.51932622472413) bank150235 +150236 POINT(41.22773815368568 73.97027821391578) bank150236 +150237 POINT(40.61156359278955 73.12853539794126) bank150237 +150238 POINT(41.02248342049254 74.10151232595078) bank150238 +150239 POINT(41.395926051283354 73.47086152213721) bank150239 +150240 POINT(41.53130326836756 74.86276121615634) bank150240 +150241 POINT(40.52961244431431 73.94003701393828) bank150241 +150242 POINT(39.78070843765542 73.01301410700357) bank150242 +150243 POINT(39.834253376345806 73.1578645156267) bank150243 +150244 POINT(39.7506994618114 73.08777399610071) bank150244 +150245 POINT(41.007768933585396 74.48830325918195) bank150245 +150246 POINT(40.35033091203273 74.8335208999899) bank150246 +150247 POINT(41.610735075356594 74.15936073837113) bank150247 +150248 POINT(41.59864338972675 74.15326679943973) bank150248 +150249 POINT(40.75738622323168 73.13062372348585) bank150249 +150250 POINT(40.699695452434064 73.23617845343624) bank150250 +150251 POINT(40.50874836032476 73.36047150718177) bank150251 +150252 POINT(41.69095968928879 73.14746987337345) bank150252 +150253 POINT(41.39373483668553 73.24141846320407) bank150253 +150254 POINT(41.408440709747175 73.79562217137898) bank150254 +150255 POINT(40.79417621367908 74.28490550102407) bank150255 +150256 POINT(41.5604226676037 73.80768459441117) bank150256 +150257 POINT(40.155567133984 73.34555317050001) bank150257 +150258 POINT(41.40412527884248 74.0041650331888) bank150258 +150259 POINT(40.26541937105994 74.21920124373325) bank150259 +150260 POINT(41.17832956074642 74.89964459171708) bank150260 +150261 POINT(40.718480234609835 74.21063614791575) bank150261 +150262 POINT(41.459012513401035 74.96089805545277) bank150262 +150263 POINT(40.95945663360996 73.77862852092706) bank150263 +150264 POINT(41.22481043347837 73.77786568808655) bank150264 +150265 POINT(40.38900154565315 73.62008692664311) bank150265 +150266 POINT(40.13971520072615 73.26216844238459) bank150266 +150267 POINT(40.822245929655 74.69726464685827) bank150267 +150268 POINT(40.1753735365705 74.82247763937814) bank150268 +150269 POINT(40.15366554931683 73.7316917389372) bank150269 +150270 POINT(39.820209845124175 74.9060971932039) bank150270 +150271 POINT(40.76845226223508 74.25785146643429) bank150271 +150272 POINT(41.192196202689935 73.95825722092665) bank150272 +150273 POINT(41.315924624226646 74.74113169063664) bank150273 +150274 POINT(41.34285572659356 73.0171853970883) bank150274 +150275 POINT(40.7354869252808 73.88385296143464) bank150275 +150276 POINT(39.88570086266124 74.46448205884646) bank150276 +150277 POINT(40.86021257749821 74.50316756830381) bank150277 +150278 POINT(40.71171462633551 73.16540297055636) bank150278 +150279 POINT(40.46350735859001 73.32317888666763) bank150279 +150280 POINT(40.784772487418955 74.2933511366679) bank150280 +150281 POINT(40.218123794429964 74.55461866891147) bank150281 +150282 POINT(40.639071416742034 73.75344042280747) bank150282 +150283 POINT(39.852547024330406 73.07149088498105) bank150283 +150284 POINT(39.952409803765825 74.86446685357812) bank150284 +150285 POINT(41.12629146360413 74.35270293819116) bank150285 +150286 POINT(41.13452116089607 73.26648308260344) bank150286 +150287 POINT(41.67550343583267 73.94284122273106) bank150287 +150288 POINT(40.06082359071121 73.54366622742238) bank150288 +150289 POINT(41.55770600771385 74.72875492481757) bank150289 +150290 POINT(40.69524070609627 73.02727148190903) bank150290 +150291 POINT(39.76349650753545 73.98567584238155) bank150291 +150292 POINT(39.7900583224232 73.36982371498888) bank150292 +150293 POINT(41.393771790807044 73.87550785519257) bank150293 +150294 POINT(39.77533695313171 73.93765317783705) bank150294 +150295 POINT(40.52079895055141 74.72954519439334) bank150295 +150296 POINT(40.14141222864362 73.18266111253983) bank150296 +150297 POINT(41.65383059787108 73.50940265494194) bank150297 +150298 POINT(40.012588536879775 73.00714204007544) bank150298 +150299 POINT(40.42477657518701 74.15043859411637) bank150299 +150300 POINT(41.64465733687075 74.86003039839434) bank150300 +150301 POINT(40.36327297514879 74.3209528349254) bank150301 +150302 POINT(41.206364674949185 73.92283962426716) bank150302 +150303 POINT(40.45295697720321 74.88301935758538) bank150303 +150304 POINT(40.26115612256526 73.2076936285356) bank150304 +150305 POINT(41.04348984489749 73.0085682400774) bank150305 +150306 POINT(41.041663253323094 74.91181493604472) bank150306 +150307 POINT(39.92398543828789 74.33912110569959) bank150307 +150308 POINT(41.11663023435603 74.41341288320965) bank150308 +150309 POINT(41.08063593888311 74.46792439765827) bank150309 +150310 POINT(40.08946661694816 74.16321128463632) bank150310 +150311 POINT(41.56161269319909 73.75186456848057) bank150311 +150312 POINT(40.20892424082928 73.79987325237309) bank150312 +150313 POINT(40.70928274435425 74.08161793940907) bank150313 +150314 POINT(41.49184822979091 73.37869898795921) bank150314 +150315 POINT(40.97440835004877 73.46574421244614) bank150315 +150316 POINT(41.68507662114826 73.59111168964091) bank150316 +150317 POINT(41.51222863931707 73.11090824331454) bank150317 +150318 POINT(40.75649655347303 74.9777040575202) bank150318 +150319 POINT(40.09605763858474 73.97820703511842) bank150319 +150320 POINT(41.317108305671695 74.79392657440052) bank150320 +150321 POINT(40.80475536271758 73.58583610074868) bank150321 +150322 POINT(40.217276964012875 74.30478410760098) bank150322 +150323 POINT(40.1333313333268 73.36690059370827) bank150323 +150324 POINT(41.13846939142537 74.68345184153387) bank150324 +150325 POINT(40.47050449193483 73.9357663011476) bank150325 +150326 POINT(39.75204804769281 74.96575719240064) bank150326 +150327 POINT(40.285366370838716 74.33501371427651) bank150327 +150328 POINT(41.116756275738666 73.3292189019433) bank150328 +150329 POINT(41.428135278779756 74.16574757047132) bank150329 +150330 POINT(41.67969235738785 74.35856699804792) bank150330 +150331 POINT(40.69632597931629 74.00529504001014) bank150331 +150332 POINT(39.79284563992039 74.38199394427654) bank150332 +150333 POINT(40.43338090823966 73.18303887113817) bank150333 +150334 POINT(41.200454838775194 73.20110970783007) bank150334 +150335 POINT(39.88891563594836 73.08661290671034) bank150335 +150336 POINT(39.7381364718175 74.42199141404221) bank150336 +150337 POINT(41.39028270510082 73.6256950919007) bank150337 +150338 POINT(40.949868785580975 74.80523030149291) bank150338 +150339 POINT(41.2478061938136 73.44745362702513) bank150339 +150340 POINT(40.54754082484297 73.99602968943023) bank150340 +150341 POINT(40.404827326212676 74.4894711937651) bank150341 +150342 POINT(41.23843710973195 74.40968546253467) bank150342 +150343 POINT(40.177964105589844 74.30618150716865) bank150343 +150344 POINT(40.4551617047226 74.1076726044552) bank150344 +150345 POINT(41.06161396661112 73.1059846014195) bank150345 +150346 POINT(41.31919549098646 74.46765189603971) bank150346 +150347 POINT(41.6876443065277 73.88094674766856) bank150347 +150348 POINT(40.780443059018204 73.90406678034984) bank150348 +150349 POINT(41.10421926084575 74.47725877260741) bank150349 +150350 POINT(40.83275949674633 74.61454220756373) bank150350 +150351 POINT(41.12409371991569 74.59907499599133) bank150351 +150352 POINT(40.746602136911825 74.3094251765611) bank150352 +150353 POINT(40.66624537145577 74.07319073827905) bank150353 +150354 POINT(40.42353340299157 73.72394406025472) bank150354 +150355 POINT(40.64230553170814 73.40770669768497) bank150355 +150356 POINT(39.94775706961168 73.9040865179799) bank150356 +150357 POINT(39.730248794245654 74.07081967038133) bank150357 +150358 POINT(40.293359785538556 74.93541046359358) bank150358 +150359 POINT(41.35205500879376 73.98253192243025) bank150359 +150360 POINT(39.78493505905468 74.01760635906278) bank150360 +150361 POINT(39.89912993112433 73.83823243536663) bank150361 +150362 POINT(40.932773564090596 73.58233494333608) bank150362 +150363 POINT(40.160830766301686 74.77438515306183) bank150363 +150364 POINT(40.68911307128633 73.69799123756307) bank150364 +150365 POINT(40.850215287665016 74.58888329883177) bank150365 +150366 POINT(40.292533907788794 73.17377491098955) bank150366 +150367 POINT(40.89396938527136 74.4593514174044) bank150367 +150368 POINT(39.73433824544755 74.0373256424811) bank150368 +150369 POINT(41.50798195564095 73.04339416400404) bank150369 +150370 POINT(41.14169939373856 74.96988461065034) bank150370 +150371 POINT(39.75741272839137 73.07261718118491) bank150371 +150372 POINT(40.03730948471995 73.11990186115574) bank150372 +150373 POINT(40.12950374088772 74.80099292065583) bank150373 +150374 POINT(40.119449361972364 73.97699738500629) bank150374 +150375 POINT(40.268312958283886 73.27465977050417) bank150375 +150376 POINT(40.756822971838425 74.33744505734384) bank150376 +150377 POINT(39.80569163053898 73.94872049176821) bank150377 +150378 POINT(41.02489952709731 74.22098836327896) bank150378 +150379 POINT(40.104465779744324 73.25745551108017) bank150379 +150380 POINT(40.368676283991064 74.04242785693165) bank150380 +150381 POINT(41.68235707800465 74.35697328582054) bank150381 +150382 POINT(41.4013719247252 73.04886721406233) bank150382 +150383 POINT(40.59919322365454 74.24071514404167) bank150383 +150384 POINT(41.494477842171094 73.18420506131268) bank150384 +150385 POINT(40.787553842752615 74.899112332262) bank150385 +150386 POINT(40.83369910168456 74.46934230370412) bank150386 +150387 POINT(40.004270048334256 74.2117076474034) bank150387 +150388 POINT(40.7345683998678 73.93077494650882) bank150388 +150389 POINT(39.75513899354842 74.58512703325536) bank150389 +150390 POINT(41.4078755320444 74.12900849204487) bank150390 +150391 POINT(41.61018507659668 74.23524325565083) bank150391 +150392 POINT(40.900207294300024 73.9608455512108) bank150392 +150393 POINT(40.2810575990858 74.65106716511676) bank150393 +150394 POINT(41.706533018648024 73.9536035283038) bank150394 +150395 POINT(40.172331120568906 73.20799708598304) bank150395 +150396 POINT(39.951616346691786 73.36699325346348) bank150396 +150397 POINT(39.71854286364611 73.15397513223684) bank150397 +150398 POINT(40.23365762707103 73.633586997313) bank150398 +150399 POINT(40.66127324710516 74.73951237753712) bank150399 +150400 POINT(40.91821973263176 74.61377598093777) bank150400 +150401 POINT(41.139403140947216 74.15451381371834) bank150401 +150402 POINT(39.93084824711518 73.38241658911572) bank150402 +150403 POINT(40.42541761228022 74.75425583596426) bank150403 +150404 POINT(41.41884668248049 73.55243205336416) bank150404 +150405 POINT(40.484099714274656 74.96246302317337) bank150405 +150406 POINT(40.26104803078788 73.24497172390905) bank150406 +150407 POINT(40.56858622064885 73.42937884036506) bank150407 +150408 POINT(41.24817212735446 73.73359087964681) bank150408 +150409 POINT(41.14234184950948 74.4696856522537) bank150409 +150410 POINT(40.65846034732089 74.88329910703237) bank150410 +150411 POINT(41.375397638537635 74.05106165654789) bank150411 +150412 POINT(40.61107795008336 74.27287765861406) bank150412 +150413 POINT(41.3037821767938 73.85366936256878) bank150413 +150414 POINT(40.41127682978051 74.6256619942309) bank150414 +150415 POINT(39.76352148537707 74.80793021336524) bank150415 +150416 POINT(41.40116328515784 73.15119456993158) bank150416 +150417 POINT(40.68926717886818 74.54752011341735) bank150417 +150418 POINT(40.24059685177638 73.6162559007843) bank150418 +150419 POINT(39.90488330861792 74.55349006426187) bank150419 +150420 POINT(40.8623460516868 74.79952476424874) bank150420 +150421 POINT(40.91195268567312 74.67944656916366) bank150421 +150422 POINT(41.09081706175968 74.5163583371391) bank150422 +150423 POINT(40.97770096800943 73.4768014217111) bank150423 +150424 POINT(40.78017085363049 73.48191218097475) bank150424 +150425 POINT(39.75340107167243 74.36520729766812) bank150425 +150426 POINT(40.353791521961625 73.30951482529565) bank150426 +150427 POINT(41.3681689024346 74.54571290426125) bank150427 +150428 POINT(41.30277671304588 74.89345977959853) bank150428 +150429 POINT(39.95266352205749 73.40781069868191) bank150429 +150430 POINT(40.32071383704788 73.18094330658315) bank150430 +150431 POINT(40.37750297428488 74.26219027703718) bank150431 +150432 POINT(41.56923330312161 73.34246258960322) bank150432 +150433 POINT(40.744675484198225 74.22630276282041) bank150433 +150434 POINT(40.22383622007043 73.13562867542684) bank150434 +150435 POINT(41.3937258119616 73.70920044881461) bank150435 +150436 POINT(40.2984054737033 73.47298316477489) bank150436 +150437 POINT(40.27876337580132 74.61534223540713) bank150437 +150438 POINT(40.957671580955086 74.75269271124691) bank150438 +150439 POINT(40.82212047677174 73.17786096833666) bank150439 +150440 POINT(39.824641487466145 74.81964775711646) bank150440 +150441 POINT(40.99833347588147 73.50191138513483) bank150441 +150442 POINT(39.82854990010253 74.4701483616208) bank150442 +150443 POINT(40.73303163147959 74.73994135051379) bank150443 +150444 POINT(40.03110867894981 74.71571447365861) bank150444 +150445 POINT(40.344785910857475 73.07884478494626) bank150445 +150446 POINT(40.87270832588101 74.80025373179657) bank150446 +150447 POINT(40.66385796853817 73.96095168251526) bank150447 +150448 POINT(39.935184360135885 74.73785095019424) bank150448 +150449 POINT(40.92252414622134 73.98972394248682) bank150449 +150450 POINT(41.344846288831754 73.46090033314975) bank150450 +150451 POINT(40.72027403779006 73.69553053545572) bank150451 +150452 POINT(40.005567838891174 74.13131522535807) bank150452 +150453 POINT(40.85520909077992 74.60240272374078) bank150453 +150454 POINT(40.986399892222266 74.87888542829637) bank150454 +150455 POINT(39.882545200896324 73.14684063316636) bank150455 +150456 POINT(40.78704804508089 73.74173553062018) bank150456 +150457 POINT(41.14168517421946 74.50180753625607) bank150457 +150458 POINT(41.649850962263436 74.97628863368206) bank150458 +150459 POINT(41.641691657776654 74.1889272381644) bank150459 +150460 POINT(41.45758395559858 74.60076167433763) bank150460 +150461 POINT(41.13886157290949 74.46285609432555) bank150461 +150462 POINT(40.141010120809256 74.01825768135588) bank150462 +150463 POINT(40.829303869478096 73.80085326077966) bank150463 +150464 POINT(41.50611274125508 74.38456940883908) bank150464 +150465 POINT(41.16462234169503 74.97103243578633) bank150465 +150466 POINT(40.02620923908222 74.1178587866422) bank150466 +150467 POINT(41.153157606817665 74.15931294896681) bank150467 +150468 POINT(39.78221598108563 73.53913963630114) bank150468 +150469 POINT(41.45409679949436 74.49084498649066) bank150469 +150470 POINT(40.269651341519655 74.98925741332638) bank150470 +150471 POINT(40.70523704161543 74.2407046596233) bank150471 +150472 POINT(40.4869544500723 73.0130639909185) bank150472 +150473 POINT(40.22914980881179 74.95566273167458) bank150473 +150474 POINT(40.718890735610465 74.24716058893203) bank150474 +150475 POINT(40.875167234259195 73.15112700921067) bank150475 +150476 POINT(40.783132475124305 74.1557258403234) bank150476 +150477 POINT(40.6361381275575 73.55047811614176) bank150477 +150478 POINT(40.00298645147306 74.0015723561605) bank150478 +150479 POINT(41.349505816285635 74.13270185357553) bank150479 +150480 POINT(41.68151462644283 74.6809208168812) bank150480 +150481 POINT(39.79792001273309 74.64386762353243) bank150481 +150482 POINT(40.787067753444504 73.89412020963452) bank150482 +150483 POINT(39.98579795753481 73.60181700973573) bank150483 +150484 POINT(41.43591303886977 73.52277030569088) bank150484 +150485 POINT(40.84503227177012 73.83027753950917) bank150485 +150486 POINT(41.368614661311845 73.05706761273741) bank150486 +150487 POINT(40.98828300364409 73.14571249686753) bank150487 +150488 POINT(40.66971836682226 73.08311458012997) bank150488 +150489 POINT(40.601223302490865 73.55912351855038) bank150489 +150490 POINT(39.774926609714726 74.6169356505472) bank150490 +150491 POINT(39.843744025989835 73.52543352011696) bank150491 +150492 POINT(41.4788614131097 73.11707559744171) bank150492 +150493 POINT(41.22147714214927 73.94619619523837) bank150493 +150494 POINT(40.53382622711986 74.2800897777523) bank150494 +150495 POINT(40.54958698267563 73.42419911637636) bank150495 +150496 POINT(40.151270120104435 73.28444221955309) bank150496 +150497 POINT(41.03009760133087 73.41177517902125) bank150497 +150498 POINT(39.99382223899155 73.44777983694878) bank150498 +150499 POINT(40.05233809431805 74.28166200798084) bank150499 +150500 POINT(40.45237626842751 74.36360923134225) bank150500 +150501 POINT(40.8341649503773 73.22786261832613) bank150501 +150502 POINT(40.860998895449576 74.78807979814725) bank150502 +150503 POINT(39.84130273782879 74.03560126354077) bank150503 +150504 POINT(41.41427742216177 74.3396498281669) bank150504 +150505 POINT(41.28461047398226 73.46454945412306) bank150505 +150506 POINT(41.50207499212648 74.24626920927929) bank150506 +150507 POINT(39.86543590661229 74.85435680339974) bank150507 +150508 POINT(41.12439124730927 73.97943992849615) bank150508 +150509 POINT(40.521816763509456 73.60274007546563) bank150509 +150510 POINT(40.75835705145771 73.07096665920247) bank150510 +150511 POINT(39.87206807319217 74.77213286933417) bank150511 +150512 POINT(41.3041050879607 73.23477132793776) bank150512 +150513 POINT(41.31484662156766 73.5066962315141) bank150513 +150514 POINT(40.367438617620905 73.73051622002691) bank150514 +150515 POINT(40.31374122579241 73.44610631592552) bank150515 +150516 POINT(39.7899620936924 74.77508882453071) bank150516 +150517 POINT(39.91749861683522 73.94572026801048) bank150517 +150518 POINT(40.31749034551093 74.98518675478374) bank150518 +150519 POINT(40.07402529411774 74.43621855659357) bank150519 +150520 POINT(41.44896868083971 73.34168085960566) bank150520 +150521 POINT(41.50630872477987 74.76196848429267) bank150521 +150522 POINT(39.870361041248294 73.74280115802316) bank150522 +150523 POINT(40.627651412591874 74.83112217625272) bank150523 +150524 POINT(40.908742267161074 74.4870502266632) bank150524 +150525 POINT(40.87820244934373 73.41860886214295) bank150525 +150526 POINT(40.078633974936075 73.34891997278903) bank150526 +150527 POINT(40.84679311090705 74.71638999406295) bank150527 +150528 POINT(40.60918346052121 74.06170131220439) bank150528 +150529 POINT(41.07498370402377 74.0046766984389) bank150529 +150530 POINT(41.57552181516962 74.93591059003913) bank150530 +150531 POINT(40.763148768097516 74.15600352779661) bank150531 +150532 POINT(40.42857509920688 73.78657533135255) bank150532 +150533 POINT(40.14628657692277 74.62114634928132) bank150533 +150534 POINT(40.37711599173837 74.75617972293385) bank150534 +150535 POINT(40.98388149871599 74.83954108048893) bank150535 +150536 POINT(41.55043979476409 73.13032080528795) bank150536 +150537 POINT(40.74566888298868 73.81584320590846) bank150537 +150538 POINT(40.76982877533609 73.74648343915699) bank150538 +150539 POINT(40.56065581559292 73.36929772928261) bank150539 +150540 POINT(41.28778363315752 73.64590152675888) bank150540 +150541 POINT(41.12700531820291 73.86601997179825) bank150541 +150542 POINT(41.24866358613608 74.37575564644811) bank150542 +150543 POINT(41.3830316728374 73.97734108720404) bank150543 +150544 POINT(41.51927965702678 74.1239278820682) bank150544 +150545 POINT(41.16877317551254 74.54321252753421) bank150545 +150546 POINT(40.13569884007227 74.42354599657672) bank150546 +150547 POINT(39.874409662236054 73.32974322908962) bank150547 +150548 POINT(41.39546634506593 74.78112105248096) bank150548 +150549 POINT(40.460995939290136 74.35001969026554) bank150549 +150550 POINT(41.34720900716953 74.40072651680892) bank150550 +150551 POINT(41.66622612250323 73.44869989099948) bank150551 +150552 POINT(39.73674883036648 74.15512858779917) bank150552 +150553 POINT(40.87103615972704 74.92617900878078) bank150553 +150554 POINT(41.392478351588466 73.89964923834631) bank150554 +150555 POINT(40.4736903263738 73.94167158412796) bank150555 +150556 POINT(41.61190418939158 74.46480411283373) bank150556 +150557 POINT(40.482323165220414 74.6822021911127) bank150557 +150558 POINT(41.52510808041587 74.7007857906612) bank150558 +150559 POINT(40.0000456595676 74.43298662888068) bank150559 +150560 POINT(41.31329913150897 73.04647746615724) bank150560 +150561 POINT(40.77875165899441 74.42478587045598) bank150561 +150562 POINT(41.30869589068304 73.65149192070396) bank150562 +150563 POINT(39.757972817959924 73.56833485403247) bank150563 +150564 POINT(39.79149802465137 73.93750279386528) bank150564 +150565 POINT(40.357265990629664 73.86355349613696) bank150565 +150566 POINT(40.28327229716149 73.08609143122221) bank150566 +150567 POINT(41.0624362967223 73.93227821048772) bank150567 +150568 POINT(40.577061702730795 73.28063140617826) bank150568 +150569 POINT(40.466641028954356 74.73765835460803) bank150569 +150570 POINT(39.721014444512434 74.47533741688538) bank150570 +150571 POINT(41.45942228488856 74.11720904970657) bank150571 +150572 POINT(40.34345937353816 74.88604806391119) bank150572 +150573 POINT(40.35156027544283 74.36595300239759) bank150573 +150574 POINT(41.03760480569523 74.77696320340065) bank150574 +150575 POINT(40.370955062707 74.88706116270252) bank150575 +150576 POINT(41.674237521986484 74.03800154994744) bank150576 +150577 POINT(40.88128706008817 74.04475226288396) bank150577 +150578 POINT(39.89478440670406 73.15970902848595) bank150578 +150579 POINT(40.40825941321721 73.02702483871784) bank150579 +150580 POINT(39.77732189979184 74.28994719440324) bank150580 +150581 POINT(40.3707803452077 74.07370650324485) bank150581 +150582 POINT(41.44809846077626 74.00410057385021) bank150582 +150583 POINT(39.824036345256744 73.87421971978016) bank150583 +150584 POINT(41.658595890624554 73.29199335977525) bank150584 +150585 POINT(40.20138170790768 74.79572390790761) bank150585 +150586 POINT(41.178341954461395 74.13846740621265) bank150586 +150587 POINT(39.85263083773805 73.91650210566642) bank150587 +150588 POINT(41.19620257342487 74.7838291138065) bank150588 +150589 POINT(41.10870109261446 73.76485804628307) bank150589 +150590 POINT(41.25034078909765 73.64073404571351) bank150590 +150591 POINT(40.865159278872646 74.15872960240043) bank150591 +150592 POINT(41.080816882423775 74.51466967587838) bank150592 +150593 POINT(40.63536589410731 74.63011692745513) bank150593 +150594 POINT(40.974725682171155 73.48964972065883) bank150594 +150595 POINT(40.12039496995308 73.29265243334706) bank150595 +150596 POINT(41.20333551503713 74.50703545839723) bank150596 +150597 POINT(40.99741322732298 74.43421509605307) bank150597 +150598 POINT(40.62086404625834 73.93346456549712) bank150598 +150599 POINT(40.89188794562968 74.57978017177399) bank150599 +150600 POINT(40.144781242204516 73.34991907256187) bank150600 +150601 POINT(41.47769069282388 74.5096193002045) bank150601 +150602 POINT(39.80725062706725 73.44332065875899) bank150602 +150603 POINT(39.88635417662442 74.42090276584433) bank150603 +150604 POINT(40.636357759976434 74.35220322779142) bank150604 +150605 POINT(41.45775635598417 74.94712759649296) bank150605 +150606 POINT(41.10520366234074 74.02203504192461) bank150606 +150607 POINT(40.9333197943425 73.89223069413707) bank150607 +150608 POINT(40.84158417746049 73.70426300872819) bank150608 +150609 POINT(41.12217579047213 74.62658837334517) bank150609 +150610 POINT(41.659886296300265 73.3229869035058) bank150610 +150611 POINT(40.89984129100076 74.57197795418585) bank150611 +150612 POINT(41.3153277444103 73.20714263108127) bank150612 +150613 POINT(39.9159341061435 74.75912056838496) bank150613 +150614 POINT(41.078946579684384 74.97124017190058) bank150614 +150615 POINT(41.08986952724376 73.04300915396472) bank150615 +150616 POINT(41.59165803143523 73.4351776741346) bank150616 +150617 POINT(41.100571134704154 74.17702223397397) bank150617 +150618 POINT(40.62469190209847 74.30025874027619) bank150618 +150619 POINT(41.6944792509191 74.75317039326092) bank150619 +150620 POINT(40.08380274236306 74.6921491204771) bank150620 +150621 POINT(40.01449975846193 74.06046400842682) bank150621 +150622 POINT(40.398389822744484 74.965030301689) bank150622 +150623 POINT(41.19558875991502 73.33791010858599) bank150623 +150624 POINT(40.20126290983132 74.20341767553684) bank150624 +150625 POINT(40.1247699106334 73.73509240798427) bank150625 +150626 POINT(39.831783237923496 74.93375289884202) bank150626 +150627 POINT(41.29578829245079 74.43457338238561) bank150627 +150628 POINT(40.43514434431157 73.95609153423173) bank150628 +150629 POINT(40.819968048767116 73.58827606999485) bank150629 +150630 POINT(40.61926693619079 74.77454909010535) bank150630 +150631 POINT(41.42543627772222 74.0476789962459) bank150631 +150632 POINT(40.58042965528755 74.43880187398558) bank150632 +150633 POINT(41.106011078151575 74.05422313470848) bank150633 +150634 POINT(40.45999037929204 73.24579020716116) bank150634 +150635 POINT(40.0229361205212 73.90632805283518) bank150635 +150636 POINT(40.78415282557721 73.42009384152234) bank150636 +150637 POINT(40.60436472049401 74.68768587905876) bank150637 +150638 POINT(40.76835869735047 73.36104839302088) bank150638 +150639 POINT(40.47472535387345 75.00037339348543) bank150639 +150640 POINT(40.98063394544703 74.4318998206575) bank150640 +150641 POINT(41.314690278553826 73.13706425698444) bank150641 +150642 POINT(39.78420164773763 74.49570304369787) bank150642 +150643 POINT(41.30603102707372 74.81160508192718) bank150643 +150644 POINT(39.88841653565909 74.5088820526138) bank150644 +150645 POINT(40.685751255559985 74.61575575382719) bank150645 +150646 POINT(40.78958275552308 74.46021545283247) bank150646 +150647 POINT(40.99623036950365 74.78246404825545) bank150647 +150648 POINT(40.78159147706235 73.22183031141464) bank150648 +150649 POINT(41.67079752816431 74.83954559229832) bank150649 +150650 POINT(41.251950157402014 73.71241055922202) bank150650 +150651 POINT(40.93276185532869 73.97971644110203) bank150651 +150652 POINT(40.901154754912504 73.38353484827776) bank150652 +150653 POINT(41.13500783354762 74.46986069670662) bank150653 +150654 POINT(40.95044820224628 73.1059158018465) bank150654 +150655 POINT(40.25568119205278 74.3451760476426) bank150655 +150656 POINT(40.55552655595617 73.65629677716434) bank150656 +150657 POINT(41.05162445311969 74.60112285345248) bank150657 +150658 POINT(41.40915243421372 73.6756946084728) bank150658 +150659 POINT(41.071585212172224 74.77010041809285) bank150659 +150660 POINT(41.29571056468632 74.51067147944615) bank150660 +150661 POINT(40.1050575772554 74.31501790806722) bank150661 +150662 POINT(39.86186219517505 73.86927897392704) bank150662 +150663 POINT(41.27207700813085 73.08263067200986) bank150663 +150664 POINT(40.46642916805734 73.48326233720469) bank150664 +150665 POINT(39.87942120110668 73.54474405004437) bank150665 +150666 POINT(41.32625265730128 73.40718804267802) bank150666 +150667 POINT(41.18852979002149 74.45143080304405) bank150667 +150668 POINT(41.20330895882634 74.6752069841218) bank150668 +150669 POINT(39.849634090432694 73.63248080116055) bank150669 +150670 POINT(40.849508414160844 74.89350648536998) bank150670 +150671 POINT(40.51421507122451 73.04171203168298) bank150671 +150672 POINT(40.23899220553229 73.26986471666856) bank150672 +150673 POINT(39.76845642449502 74.45008804551057) bank150673 +150674 POINT(40.35928748557529 73.1797355631911) bank150674 +150675 POINT(41.53793609977576 74.3356570880581) bank150675 +150676 POINT(41.01517684674462 73.64411561499642) bank150676 +150677 POINT(40.90350728985051 73.30803230948756) bank150677 +150678 POINT(40.20132787856798 73.42589019611137) bank150678 +150679 POINT(40.25680727906234 73.39520729039833) bank150679 +150680 POINT(39.778586319673096 73.1694257841628) bank150680 +150681 POINT(40.536135530478646 73.83495591166619) bank150681 +150682 POINT(40.897197288505176 73.71403062639337) bank150682 +150683 POINT(40.58616593941156 73.70150450509193) bank150683 +150684 POINT(41.59195753616575 74.808312355614) bank150684 +150685 POINT(41.66208112560481 74.65768011957364) bank150685 +150686 POINT(40.76530113873544 74.99708984376625) bank150686 +150687 POINT(41.19913336597856 73.69076001534712) bank150687 +150688 POINT(40.58130370206176 74.0372258767638) bank150688 +150689 POINT(40.35797873740474 73.65793804460068) bank150689 +150690 POINT(40.50944438117092 73.73798183939809) bank150690 +150691 POINT(39.92624978004904 74.5878584721421) bank150691 +150692 POINT(41.36829999470401 74.33864773549396) bank150692 +150693 POINT(40.201076254843066 73.94759746020068) bank150693 +150694 POINT(39.83862116223636 73.15057417531058) bank150694 +150695 POINT(40.48403207911539 74.5233436932408) bank150695 +150696 POINT(40.67111589755333 73.97850704725812) bank150696 +150697 POINT(41.093065280019125 74.19144627353435) bank150697 +150698 POINT(40.54091872217314 74.95285740767275) bank150698 +150699 POINT(40.94063031153657 74.3108091240399) bank150699 +150700 POINT(40.368493793919804 74.05512920803582) bank150700 +150701 POINT(40.68906443461737 74.34489496372835) bank150701 +150702 POINT(40.920453013600614 74.62072693591358) bank150702 +150703 POINT(41.08260001535747 74.94279071419794) bank150703 +150704 POINT(40.84085205407455 74.81402729896938) bank150704 +150705 POINT(40.79255316456617 73.10869790822858) bank150705 +150706 POINT(41.70474639231848 73.75260717010244) bank150706 +150707 POINT(41.313447663413406 74.63096673675165) bank150707 +150708 POINT(39.92146061707294 73.37699514238997) bank150708 +150709 POINT(41.13731629073588 74.33345516167391) bank150709 +150710 POINT(40.50709597188413 74.31067411042449) bank150710 +150711 POINT(40.669758552607675 74.6326108738216) bank150711 +150712 POINT(39.76701655511207 73.93058700253303) bank150712 +150713 POINT(40.09467283558276 73.0899826567626) bank150713 +150714 POINT(40.80254219441508 73.21388652780578) bank150714 +150715 POINT(40.128366817296886 73.46394963431898) bank150715 +150716 POINT(39.820695894543405 74.44180158845234) bank150716 +150717 POINT(41.24412110798176 74.07452519710665) bank150717 +150718 POINT(40.87113377363914 73.09510448866357) bank150718 +150719 POINT(40.25535474674276 73.64444921207948) bank150719 +150720 POINT(41.27046608293956 73.66094633164576) bank150720 +150721 POINT(40.815254068482666 74.90387204899912) bank150721 +150722 POINT(40.397514677715414 74.54841989221006) bank150722 +150723 POINT(41.417339385202595 73.29399642074951) bank150723 +150724 POINT(41.387337419151386 74.47204543660743) bank150724 +150725 POINT(40.82330808287514 74.19994913798665) bank150725 +150726 POINT(40.67351139156177 74.00339116524344) bank150726 +150727 POINT(41.67678693963279 74.06309593177595) bank150727 +150728 POINT(41.05470584588865 73.62167012374768) bank150728 +150729 POINT(40.52009994215048 74.74752750312625) bank150729 +150730 POINT(41.56592202992571 74.94993157788626) bank150730 +150731 POINT(41.235699536243416 74.64833092686936) bank150731 +150732 POINT(41.200440686396426 74.02538988757247) bank150732 +150733 POINT(39.9742957397932 73.92860613570656) bank150733 +150734 POINT(40.80958636674731 73.82802985112909) bank150734 +150735 POINT(39.901795892326625 74.41925863664265) bank150735 +150736 POINT(40.00097019160662 73.3429209756565) bank150736 +150737 POINT(41.08494147958917 74.08512476776973) bank150737 +150738 POINT(40.869128490171654 74.94749027776476) bank150738 +150739 POINT(40.18302126023194 74.3377017350072) bank150739 +150740 POINT(40.48098654763713 74.9873827725497) bank150740 +150741 POINT(41.29835900374524 73.33099711505886) bank150741 +150742 POINT(40.3400097889542 74.15487428746273) bank150742 +150743 POINT(40.356435346254194 73.82058172659728) bank150743 +150744 POINT(40.10074661835789 74.75074844455544) bank150744 +150745 POINT(40.73581027271071 73.3676974592602) bank150745 +150746 POINT(40.59000649371217 74.1881628035368) bank150746 +150747 POINT(41.3790818613313 73.4395162697235) bank150747 +150748 POINT(39.85613727998623 73.40530986756144) bank150748 +150749 POINT(39.799813399059744 73.67069857511284) bank150749 +150750 POINT(40.93345834939881 74.57564679620103) bank150750 +150751 POINT(40.04089126036076 73.55959100577223) bank150751 +150752 POINT(41.01906429165194 73.1510520458437) bank150752 +150753 POINT(39.91453895054326 73.25195445608267) bank150753 +150754 POINT(39.840510916358966 74.15125183210692) bank150754 +150755 POINT(40.65311632914917 73.52757246369399) bank150755 +150756 POINT(41.24287194576919 73.1058047081773) bank150756 +150757 POINT(39.9644195088803 74.56715958605453) bank150757 +150758 POINT(39.8988048274552 74.92528437876207) bank150758 +150759 POINT(40.20188860029246 73.50175742966782) bank150759 +150760 POINT(40.202501716132936 74.61514483347052) bank150760 +150761 POINT(40.03143163267231 73.28503367516126) bank150761 +150762 POINT(40.861668620797445 74.73149581200533) bank150762 +150763 POINT(41.30129045969088 73.00748719170936) bank150763 +150764 POINT(40.16627458656931 73.18911317239036) bank150764 +150765 POINT(39.85898281834149 74.20154210470966) bank150765 +150766 POINT(41.144784087764364 73.10233842956201) bank150766 +150767 POINT(41.45200610366215 73.04686500890101) bank150767 +150768 POINT(40.47236410542594 73.96680353107331) bank150768 +150769 POINT(40.03017896545189 74.55176910779649) bank150769 +150770 POINT(41.02585925784776 74.00316220692372) bank150770 +150771 POINT(41.08298338430191 73.9248131729263) bank150771 +150772 POINT(40.45128198019716 73.38334444903701) bank150772 +150773 POINT(39.953239586354115 73.28977566067663) bank150773 +150774 POINT(41.29716435783317 74.0952766200986) bank150774 +150775 POINT(41.36860310242568 73.07787442989567) bank150775 +150776 POINT(41.43669123078433 73.72165569897948) bank150776 +150777 POINT(40.98924802125833 74.32388358303314) bank150777 +150778 POINT(39.80226661395044 73.17131913809776) bank150778 +150779 POINT(40.62779075768605 74.66310416335796) bank150779 +150780 POINT(40.79875253212867 74.25791094437197) bank150780 +150781 POINT(40.19501929407024 73.93641809032921) bank150781 +150782 POINT(41.10126202052882 74.06243337763131) bank150782 +150783 POINT(40.29101312039789 73.0619555999252) bank150783 +150784 POINT(39.93751413960354 73.86154992219909) bank150784 +150785 POINT(41.507209350188774 73.55947959720395) bank150785 +150786 POINT(40.236545664415395 74.7444191615999) bank150786 +150787 POINT(40.552520508490446 73.73923480340159) bank150787 +150788 POINT(40.12747057139065 74.02125657742971) bank150788 +150789 POINT(41.5613687325272 74.66713813919529) bank150789 +150790 POINT(40.87949843588642 74.89156344170968) bank150790 +150791 POINT(41.33373189819603 73.8327111781761) bank150791 +150792 POINT(40.251445482131075 73.57800974574295) bank150792 +150793 POINT(41.12770445608119 74.472954966684) bank150793 +150794 POINT(40.863837414973375 74.84566954643991) bank150794 +150795 POINT(41.46477332345294 74.82471445061111) bank150795 +150796 POINT(40.797096040066975 73.87463752649592) bank150796 +150797 POINT(41.559261020519514 73.30397238120557) bank150797 +150798 POINT(40.823470441250834 74.28647221197899) bank150798 +150799 POINT(40.970661059681454 73.9133408464648) bank150799 +150800 POINT(40.0815781022543 73.40055176205377) bank150800 +150801 POINT(39.86642753172572 74.765603640269) bank150801 +150802 POINT(40.35512207777043 73.72025704378399) bank150802 +150803 POINT(41.59847971788545 73.44526919376842) bank150803 +150804 POINT(40.78287879306728 74.30323871947212) bank150804 +150805 POINT(40.834376927824174 74.25023146166673) bank150805 +150806 POINT(41.29796623658321 73.56051915865075) bank150806 +150807 POINT(40.17251639717648 74.91552223302115) bank150807 +150808 POINT(40.256442892808245 73.62588687699491) bank150808 +150809 POINT(39.84969334040561 74.96964116486389) bank150809 +150810 POINT(40.707247199591144 73.4148607831862) bank150810 +150811 POINT(40.01980528454083 73.77869928578707) bank150811 +150812 POINT(40.19544264671675 73.36778010165311) bank150812 +150813 POINT(41.46782909973092 74.24906152038015) bank150813 +150814 POINT(40.43000098789059 73.19154920068318) bank150814 +150815 POINT(39.95732057537476 73.51774617048126) bank150815 +150816 POINT(40.43812347515431 73.20196615863098) bank150816 +150817 POINT(41.09134577669149 73.30183333695449) bank150817 +150818 POINT(41.70573159608486 74.8659546318474) bank150818 +150819 POINT(41.39085456856732 73.12815984286026) bank150819 +150820 POINT(40.85001033224403 74.57529666360824) bank150820 +150821 POINT(40.979786443303084 74.74520142562106) bank150821 +150822 POINT(40.34412381660509 74.41568274396238) bank150822 +150823 POINT(40.34680287968196 74.68493675542764) bank150823 +150824 POINT(41.12864902751529 73.45591928319497) bank150824 +150825 POINT(41.36178346643522 73.63017715908994) bank150825 +150826 POINT(41.18095670035718 73.5209471405886) bank150826 +150827 POINT(40.00560751075582 73.53344570989444) bank150827 +150828 POINT(40.15352344002805 73.43462607417777) bank150828 +150829 POINT(41.37679409285619 73.0338791491906) bank150829 +150830 POINT(39.91715361626534 74.1387114390601) bank150830 +150831 POINT(41.06136077713251 74.33189588392803) bank150831 +150832 POINT(40.56317783866263 73.68294196664894) bank150832 +150833 POINT(40.330477223706204 74.00208732877664) bank150833 +150834 POINT(40.97840350593364 73.5844280375086) bank150834 +150835 POINT(40.79587059840822 73.11814716919564) bank150835 +150836 POINT(41.370890475587224 73.99642036409844) bank150836 +150837 POINT(40.47887028829292 74.40934592383714) bank150837 +150838 POINT(40.21318774443198 74.985845201699) bank150838 +150839 POINT(41.634019924150785 74.5653221122059) bank150839 +150840 POINT(41.190823890273236 74.32989545407248) bank150840 +150841 POINT(40.727403885914526 74.51407086952051) bank150841 +150842 POINT(40.4479460652749 74.74076751168941) bank150842 +150843 POINT(41.52217523868884 73.5645937729978) bank150843 +150844 POINT(41.70606498630341 73.6651688325643) bank150844 +150845 POINT(39.895364310611214 74.74928874159049) bank150845 +150846 POINT(41.607786644474004 73.02694595630193) bank150846 +150847 POINT(40.18905387616657 73.38537062172436) bank150847 +150848 POINT(41.35472574505749 73.34946043220101) bank150848 +150849 POINT(40.170057071094305 74.24621202965876) bank150849 +150850 POINT(39.89215807612587 74.04913301594091) bank150850 +150851 POINT(40.97883394630983 73.11994226958181) bank150851 +150852 POINT(41.44932026939947 73.89033450736889) bank150852 +150853 POINT(41.05391760413799 74.409100226294) bank150853 +150854 POINT(40.88249116675225 73.2428836400136) bank150854 +150855 POINT(41.43081142877464 73.29663710667637) bank150855 +150856 POINT(41.69376042136388 74.54559757623379) bank150856 +150857 POINT(41.705050850888775 74.44669650225865) bank150857 +150858 POINT(41.596425179851835 74.33319971530754) bank150858 +150859 POINT(40.92835971977097 74.01972853770958) bank150859 +150860 POINT(41.027857501178936 74.43026656541785) bank150860 +150861 POINT(40.663964252363925 74.24735099409617) bank150861 +150862 POINT(41.51978138876777 73.6675424628923) bank150862 +150863 POINT(40.89957197069211 73.60040916447501) bank150863 +150864 POINT(40.51160493884551 74.21941934679643) bank150864 +150865 POINT(39.99531882139732 74.66684474557468) bank150865 +150866 POINT(41.25489659302809 73.5979156709674) bank150866 +150867 POINT(39.94666331823937 73.04050745476049) bank150867 +150868 POINT(39.91765414815913 73.32841760653095) bank150868 +150869 POINT(41.4185792531811 74.82403296372996) bank150869 +150870 POINT(40.38457933928147 73.88075795150715) bank150870 +150871 POINT(41.396531114469894 74.61294094614902) bank150871 +150872 POINT(41.19478264053841 74.75389437951424) bank150872 +150873 POINT(39.89810621779139 74.20931438899899) bank150873 +150874 POINT(41.62108507660129 74.12416709670423) bank150874 +150875 POINT(41.061452269789164 74.48547111997615) bank150875 +150876 POINT(39.912169392852135 74.71491665893339) bank150876 +150877 POINT(40.32172290550171 74.1600606114952) bank150877 +150878 POINT(41.558820464324235 74.98330519442557) bank150878 +150879 POINT(41.26279116377845 73.81868227264911) bank150879 +150880 POINT(41.65536406226946 74.63487545895583) bank150880 +150881 POINT(39.885641002610576 74.56718133995214) bank150881 +150882 POINT(40.715499121760615 73.35525435541402) bank150882 +150883 POINT(40.53899162901379 74.1178931146263) bank150883 +150884 POINT(41.27682372906552 74.92373485045745) bank150884 +150885 POINT(41.48161942799012 73.47556938816103) bank150885 +150886 POINT(41.2060426390522 74.13153005282162) bank150886 +150887 POINT(40.986474570649236 74.25635599902401) bank150887 +150888 POINT(40.78141248527919 74.80851077164071) bank150888 +150889 POINT(40.098978376234015 74.3687535483875) bank150889 +150890 POINT(40.244202262811356 73.0122987417875) bank150890 +150891 POINT(41.607425998169944 74.5690836464422) bank150891 +150892 POINT(40.948521315708966 73.40135756776858) bank150892 +150893 POINT(41.44526842029964 74.13575027733235) bank150893 +150894 POINT(39.87900080249657 73.84563702915399) bank150894 +150895 POINT(41.10584814357402 73.65864477424697) bank150895 +150896 POINT(41.3860775543171 73.53242082847802) bank150896 +150897 POINT(40.43592628675927 74.36895766710352) bank150897 +150898 POINT(40.28724200375073 73.6929492409651) bank150898 +150899 POINT(41.34868907328222 73.86354619894139) bank150899 +150900 POINT(40.60864648695444 73.54110586598088) bank150900 +150901 POINT(39.82141876067202 73.21910304509507) bank150901 +150902 POINT(41.03720327145288 74.76009176264877) bank150902 +150903 POINT(39.871407074345214 74.1202145601829) bank150903 +150904 POINT(41.37069554943008 73.25868541700204) bank150904 +150905 POINT(40.50208855836552 73.88052538428857) bank150905 +150906 POINT(41.56471917242784 74.44168791248832) bank150906 +150907 POINT(39.73448441228914 74.85342200484367) bank150907 +150908 POINT(41.32689657735632 73.28492441776888) bank150908 +150909 POINT(39.74126412326366 73.09482161260479) bank150909 +150910 POINT(41.05314461962617 74.54023627911961) bank150910 +150911 POINT(40.48921627191731 74.31680456268882) bank150911 +150912 POINT(40.81952505343628 74.37852550611923) bank150912 +150913 POINT(39.86450471413273 73.25492300632146) bank150913 +150914 POINT(40.726205499499315 73.83860657516487) bank150914 +150915 POINT(41.23476601661682 73.61905263230493) bank150915 +150916 POINT(41.42259845049278 73.85120612746995) bank150916 +150917 POINT(40.99109472577736 73.44584022027738) bank150917 +150918 POINT(41.36235895831735 73.627477286585) bank150918 +150919 POINT(41.340163999083565 73.32539559927098) bank150919 +150920 POINT(40.46950689972452 73.26182571274553) bank150920 +150921 POINT(41.448742854469955 73.22367488243898) bank150921 +150922 POINT(40.965595534024224 74.02963317413482) bank150922 +150923 POINT(40.51791492928458 73.99945174011778) bank150923 +150924 POINT(40.314532690065036 74.10518420254941) bank150924 +150925 POINT(41.413359668941666 73.01115060486794) bank150925 +150926 POINT(41.06009098812107 73.13442760567996) bank150926 +150927 POINT(40.660719372425355 74.63522872539703) bank150927 +150928 POINT(41.30395409827874 73.55428488917657) bank150928 +150929 POINT(41.05903988057478 73.87824889053998) bank150929 +150930 POINT(40.426138779773126 74.99699038286901) bank150930 +150931 POINT(39.914119903072844 73.09914022866658) bank150931 +150932 POINT(40.78754853602224 73.95941077480379) bank150932 +150933 POINT(40.64607685489793 74.25221700500232) bank150933 +150934 POINT(41.41015941054817 74.70567769841513) bank150934 +150935 POINT(39.88349553568975 73.71245502266534) bank150935 +150936 POINT(39.923810634372444 74.01744825978734) bank150936 +150937 POINT(39.82955165808032 74.21421771396172) bank150937 +150938 POINT(40.45209801850671 74.04007988404989) bank150938 +150939 POINT(39.95864279767382 74.71250429781185) bank150939 +150940 POINT(40.61022794246476 74.47489332693901) bank150940 +150941 POINT(40.881078470266516 74.85460509972235) bank150941 +150942 POINT(40.60357844551913 74.45736228366785) bank150942 +150943 POINT(41.711926018638884 74.19269662495738) bank150943 +150944 POINT(40.406840391523204 74.58737112833612) bank150944 +150945 POINT(40.162928633271555 74.18476293678948) bank150945 +150946 POINT(40.43117783813734 73.4878319862632) bank150946 +150947 POINT(40.31844096848875 74.87226714418911) bank150947 +150948 POINT(41.6714594166825 73.96359722179778) bank150948 +150949 POINT(40.35563095821482 73.78814635969003) bank150949 +150950 POINT(39.78952465811689 74.36838228381644) bank150950 +150951 POINT(40.17491307311502 73.83812683386628) bank150951 +150952 POINT(41.44905646792793 73.7490070711479) bank150952 +150953 POINT(39.90046119398656 74.16284741913104) bank150953 +150954 POINT(40.04019257191326 73.50888470760208) bank150954 +150955 POINT(40.811505722816406 74.77689244359915) bank150955 +150956 POINT(41.47702887573528 73.26007776952981) bank150956 +150957 POINT(40.37693356950639 74.78686385812975) bank150957 +150958 POINT(40.777038803022656 73.81495394259653) bank150958 +150959 POINT(40.58676965947905 73.96855821595433) bank150959 +150960 POINT(40.67251258160951 74.88937609035943) bank150960 +150961 POINT(41.57828294732383 73.39972621100891) bank150961 +150962 POINT(41.15703141649546 73.96839566766882) bank150962 +150963 POINT(39.7853260960125 73.10993403606017) bank150963 +150964 POINT(40.750556811708286 74.20842470992126) bank150964 +150965 POINT(39.83928330413483 73.63935149563878) bank150965 +150966 POINT(39.917942836983464 74.20553149424902) bank150966 +150967 POINT(40.2944103338064 73.9182639866514) bank150967 +150968 POINT(41.22878925257913 73.25069205380842) bank150968 +150969 POINT(41.64637139272955 73.32124452928173) bank150969 +150970 POINT(40.203149841037124 74.24974831568034) bank150970 +150971 POINT(39.793840660247824 73.35989591293094) bank150971 +150972 POINT(40.203202355918314 74.65707528218863) bank150972 +150973 POINT(41.590284931177 74.28358241142784) bank150973 +150974 POINT(39.734866017995834 74.7697679359817) bank150974 +150975 POINT(41.60879933027516 73.8121564242373) bank150975 +150976 POINT(40.54236946942825 74.74995142471639) bank150976 +150977 POINT(39.83268641104527 73.13486913227459) bank150977 +150978 POINT(39.747955856952984 74.53881732579705) bank150978 +150979 POINT(40.577822977033854 74.42356600019035) bank150979 +150980 POINT(40.424006905270325 73.24354943289987) bank150980 +150981 POINT(40.180194486623634 73.14004210834466) bank150981 +150982 POINT(40.78891530504293 74.16601979405193) bank150982 +150983 POINT(40.83657167385746 74.31343953991812) bank150983 +150984 POINT(41.689421615110476 73.74087854296098) bank150984 +150985 POINT(41.17996668765217 73.81179803407856) bank150985 +150986 POINT(41.60550719797959 73.58882052162812) bank150986 +150987 POINT(41.24438400852443 73.5430431595341) bank150987 +150988 POINT(41.01132254979382 74.42171478280257) bank150988 +150989 POINT(40.94828779269759 74.90872722653222) bank150989 +150990 POINT(39.75167325583514 73.99192119783488) bank150990 +150991 POINT(40.45557563346618 74.91386823789306) bank150991 +150992 POINT(40.51885876786139 73.49337184944109) bank150992 +150993 POINT(40.493253405927746 73.76277546812035) bank150993 +150994 POINT(41.45142620046933 73.9597259308002) bank150994 +150995 POINT(40.9901371709903 74.64482916969558) bank150995 +150996 POINT(40.274125397482884 74.11306668830822) bank150996 +150997 POINT(39.82400594913377 73.83298242588779) bank150997 +150998 POINT(40.10681204536193 73.01746175329227) bank150998 +150999 POINT(40.57292399737335 73.61396775752723) bank150999 +151000 POINT(40.995980626144906 74.86977913723757) bank151000 +151001 POINT(39.85175324511314 74.38793624501754) bank151001 +151002 POINT(40.7911487221381 73.19715727829627) bank151002 +151003 POINT(40.31750627969849 73.71547941155613) bank151003 +151004 POINT(41.50831238662463 73.52052887597385) bank151004 +151005 POINT(40.97114710240276 73.10500703479946) bank151005 +151006 POINT(41.36998525472831 74.60056316652208) bank151006 +151007 POINT(40.25950935570848 74.90245136795785) bank151007 +151008 POINT(40.79315571460246 74.99657056659059) bank151008 +151009 POINT(41.63798205158807 73.18347234583291) bank151009 +151010 POINT(39.89315086335379 74.88546811839404) bank151010 +151011 POINT(40.806049519983375 74.03213860906763) bank151011 +151012 POINT(41.62435728022315 73.98666534469757) bank151012 +151013 POINT(40.191994133471724 73.43688007406584) bank151013 +151014 POINT(40.37461373718368 73.63801470381635) bank151014 +151015 POINT(40.53274190872169 74.51380533194312) bank151015 +151016 POINT(41.141435208158825 73.81678932640361) bank151016 +151017 POINT(41.44704791219087 74.31770499550193) bank151017 +151018 POINT(40.45720826946823 73.48050719028399) bank151018 +151019 POINT(41.44490902617373 74.38682835764135) bank151019 +151020 POINT(40.0286225333767 73.2673504641676) bank151020 +151021 POINT(41.64235837289543 73.09591613059747) bank151021 +151022 POINT(41.67447321187812 74.87542069109327) bank151022 +151023 POINT(40.71489406119763 74.89870286223862) bank151023 +151024 POINT(40.1546213552088 74.88738273468732) bank151024 +151025 POINT(39.859421096282404 74.47771870251306) bank151025 +151026 POINT(39.91622314992101 74.2790252866627) bank151026 +151027 POINT(41.14395998262422 74.46520294366843) bank151027 +151028 POINT(40.87155800188602 74.7756231366214) bank151028 +151029 POINT(41.46241799561342 74.49290586805151) bank151029 +151030 POINT(41.283287235144165 74.55758184371217) bank151030 +151031 POINT(40.045914922709336 73.0237909306203) bank151031 +151032 POINT(40.359805830226016 74.09445244440836) bank151032 +151033 POINT(40.10478966794919 74.70233487439613) bank151033 +151034 POINT(39.85035758335048 73.6054681198296) bank151034 +151035 POINT(39.8934515673848 73.26857196515617) bank151035 +151036 POINT(39.803874642964495 73.88431453567487) bank151036 +151037 POINT(41.35985182622702 73.76121485474475) bank151037 +151038 POINT(39.89702144728617 73.26587037429086) bank151038 +151039 POINT(40.628958718101316 74.20563586362643) bank151039 +151040 POINT(40.611935134961364 74.0949867107782) bank151040 +151041 POINT(41.17240370461691 73.31460225916186) bank151041 +151042 POINT(40.87374723701348 74.99006253096667) bank151042 +151043 POINT(40.70818888671668 74.82206486171823) bank151043 +151044 POINT(41.024517423851215 74.49130635491022) bank151044 +151045 POINT(41.53386832295615 74.79634028339169) bank151045 +151046 POINT(40.9873637774815 73.86185199252557) bank151046 +151047 POINT(39.9352842470777 74.7203763879056) bank151047 +151048 POINT(39.91545584365831 74.25482978856847) bank151048 +151049 POINT(40.39525453676104 74.99855877340873) bank151049 +151050 POINT(40.03600822293396 74.88390780798152) bank151050 +151051 POINT(40.79968272536501 74.68883373521832) bank151051 +151052 POINT(39.77068096548727 73.9567028639731) bank151052 +151053 POINT(39.933477075765914 74.20956257149443) bank151053 +151054 POINT(40.33419199750079 73.446388125876) bank151054 +151055 POINT(40.28035734474793 74.33546743301825) bank151055 +151056 POINT(40.94753640382732 73.15219081733815) bank151056 +151057 POINT(40.36374945670611 73.0251506593779) bank151057 +151058 POINT(41.1934313584408 73.91658742742585) bank151058 +151059 POINT(41.064667411328756 73.36528102445696) bank151059 +151060 POINT(40.036531115317295 74.32558814220083) bank151060 +151061 POINT(40.93102153894624 74.75470975713509) bank151061 +151062 POINT(40.11818229670296 73.53897837204897) bank151062 +151063 POINT(40.794960879668494 74.74818987679274) bank151063 +151064 POINT(40.40827759934124 74.04794189333779) bank151064 +151065 POINT(41.236209221109426 74.43608506485577) bank151065 +151066 POINT(41.185436125054856 74.44731348858987) bank151066 +151067 POINT(41.393964829958406 73.44338434407176) bank151067 +151068 POINT(40.76825592967669 74.90557345122683) bank151068 +151069 POINT(40.387831731100746 74.22681353844763) bank151069 +151070 POINT(41.69528196743808 74.2025984232821) bank151070 +151071 POINT(39.985621097579966 74.30436397037357) bank151071 +151072 POINT(41.550644386389536 74.6627966242436) bank151072 +151073 POINT(41.13620537194763 74.76295700956472) bank151073 +151074 POINT(39.80591778004792 73.39718091978942) bank151074 +151075 POINT(40.88417182244861 73.1127189702458) bank151075 +151076 POINT(41.50064006752017 74.8332211794713) bank151076 +151077 POINT(39.959313547249586 73.17813526240339) bank151077 +151078 POINT(41.46462035701752 74.3187346633009) bank151078 +151079 POINT(40.0869121728923 74.78465426200296) bank151079 +151080 POINT(41.25875376243171 74.26476388341575) bank151080 +151081 POINT(40.63288880023455 73.52161049759499) bank151081 +151082 POINT(40.17479900796334 73.98700247038512) bank151082 +151083 POINT(41.68100633709247 73.40423888018341) bank151083 +151084 POINT(40.656408034604276 74.8232310245249) bank151084 +151085 POINT(40.521845396573674 73.01164646893312) bank151085 +151086 POINT(41.412397138928014 73.52251032238364) bank151086 +151087 POINT(40.68596377503462 74.6533279822339) bank151087 +151088 POINT(40.81011365502881 74.83794715597192) bank151088 +151089 POINT(41.3071162730711 73.49452271429314) bank151089 +151090 POINT(41.1804395497869 74.6137508582846) bank151090 +151091 POINT(40.03051018847419 74.3886545370912) bank151091 +151092 POINT(40.99535395762236 73.1658079519419) bank151092 +151093 POINT(41.6214812530613 74.05599520700693) bank151093 +151094 POINT(41.57556847204383 74.7242622220882) bank151094 +151095 POINT(39.75716391566523 74.59864918849782) bank151095 +151096 POINT(40.85391158376711 74.5485041402404) bank151096 +151097 POINT(39.95935377645665 73.38858706109865) bank151097 +151098 POINT(41.69496604356748 74.60357025088194) bank151098 +151099 POINT(39.97362050769281 73.87631271393943) bank151099 +151100 POINT(41.301461344529955 73.57439744305205) bank151100 +151101 POINT(39.91501014822848 73.01476834067296) bank151101 +151102 POINT(39.805148723378245 74.34689685664674) bank151102 +151103 POINT(41.18560036360699 73.56627468550697) bank151103 +151104 POINT(40.5237613882745 73.27803003708685) bank151104 +151105 POINT(41.37689303601885 73.22825089645914) bank151105 +151106 POINT(41.55638504219677 74.82180898134145) bank151106 +151107 POINT(41.46974456175751 74.85021896644415) bank151107 +151108 POINT(40.573533575371435 74.88824636978042) bank151108 +151109 POINT(41.05785548438727 73.88958313899006) bank151109 +151110 POINT(40.82746611958142 73.52739316079459) bank151110 +151111 POINT(41.44068433028582 74.64164542597022) bank151111 +151112 POINT(40.93055232051159 73.4821456704788) bank151112 +151113 POINT(40.78357752526791 73.88775202547636) bank151113 +151114 POINT(40.31594459971012 74.64454098626916) bank151114 +151115 POINT(41.68848985615161 73.0941246947706) bank151115 +151116 POINT(40.64893914027053 74.53007922377417) bank151116 +151117 POINT(39.92034343158653 73.39910714267079) bank151117 +151118 POINT(41.58204743004242 74.04335962594672) bank151118 +151119 POINT(40.19320279660303 74.9088384519173) bank151119 +151120 POINT(40.14575563833479 74.60502537559104) bank151120 +151121 POINT(41.231830171979055 74.23742532009597) bank151121 +151122 POINT(41.57701084928459 74.07446468221852) bank151122 +151123 POINT(39.94791101941041 74.3533747407018) bank151123 +151124 POINT(41.58913035528714 73.48631968965873) bank151124 +151125 POINT(39.89729458193527 74.17115226035887) bank151125 +151126 POINT(40.2575379976799 74.0453943606111) bank151126 +151127 POINT(41.43908378631582 73.08165342648304) bank151127 +151128 POINT(41.1727599722643 74.3784727039729) bank151128 +151129 POINT(41.12425297486736 73.59075293098172) bank151129 +151130 POINT(40.642076206087445 74.61716963675138) bank151130 +151131 POINT(40.03115784319204 74.6423278072282) bank151131 +151132 POINT(39.81614984382332 73.21219834473936) bank151132 +151133 POINT(41.42413911179166 74.74276734225104) bank151133 +151134 POINT(40.13389410116924 74.96033851389468) bank151134 +151135 POINT(40.22255980210087 73.89132389402668) bank151135 +151136 POINT(40.97943240281864 73.92986902417873) bank151136 +151137 POINT(41.48344840726532 73.23799338190129) bank151137 +151138 POINT(41.05472006119445 73.80286945214384) bank151138 +151139 POINT(40.75333522298057 73.10244923733111) bank151139 +151140 POINT(41.53765588751722 74.06221319090166) bank151140 +151141 POINT(41.4522952550535 73.21929110168105) bank151141 +151142 POINT(40.56997847195879 73.4663615896703) bank151142 +151143 POINT(40.45041388689007 74.78260852113098) bank151143 +151144 POINT(41.14438638955267 73.72682692400555) bank151144 +151145 POINT(41.21858157769734 73.77773032272434) bank151145 +151146 POINT(41.50587814768168 73.24523158571432) bank151146 +151147 POINT(40.29129177691533 74.27316115911474) bank151147 +151148 POINT(41.60809939874096 74.48323612513146) bank151148 +151149 POINT(40.5592588354378 73.70261683176308) bank151149 +151150 POINT(41.28726443619048 73.50615338454926) bank151150 +151151 POINT(39.8858963888867 73.07227324364582) bank151151 +151152 POINT(40.48188524798668 73.27015293857913) bank151152 +151153 POINT(41.23785411578253 74.2182698024078) bank151153 +151154 POINT(40.01904268442495 73.54087487671282) bank151154 +151155 POINT(40.46587837228823 74.01484245867042) bank151155 +151156 POINT(40.34959626005865 73.19049343939963) bank151156 +151157 POINT(41.417652002989556 73.85194105550748) bank151157 +151158 POINT(41.03566080114456 74.35413490250222) bank151158 +151159 POINT(41.31188301939152 73.2895993229546) bank151159 +151160 POINT(40.60738150556615 73.37148071149822) bank151160 +151161 POINT(40.75812507379731 73.40675357324544) bank151161 +151162 POINT(39.9108516211545 73.64665977172768) bank151162 +151163 POINT(41.03614468076613 73.82008320025524) bank151163 +151164 POINT(41.1922766532883 74.83552355769946) bank151164 +151165 POINT(39.77652321890263 73.93962847296991) bank151165 +151166 POINT(41.338922341260925 74.62485598284356) bank151166 +151167 POINT(41.262632618601295 74.90856884301972) bank151167 +151168 POINT(40.772636629435475 73.53733386706934) bank151168 +151169 POINT(41.26696386802164 73.97254116263645) bank151169 +151170 POINT(41.33290880825844 73.18433157153692) bank151170 +151171 POINT(39.9472587478899 74.08979169408934) bank151171 +151172 POINT(40.58409775355204 73.34175880532672) bank151172 +151173 POINT(40.14171694319791 74.9411997945192) bank151173 +151174 POINT(40.99400643046474 73.56850284772584) bank151174 +151175 POINT(39.969679986118486 74.35209220556195) bank151175 +151176 POINT(40.79431544187465 73.26493859801924) bank151176 +151177 POINT(40.680604341083715 74.18549972456876) bank151177 +151178 POINT(41.68089790896439 73.6315001686266) bank151178 +151179 POINT(40.74024747787384 73.92011752247888) bank151179 +151180 POINT(41.18649737123417 74.98080104463914) bank151180 +151181 POINT(40.40202393583254 74.11656638880224) bank151181 +151182 POINT(40.368022143016844 74.25087200724334) bank151182 +151183 POINT(41.04812040140313 73.34371303217341) bank151183 +151184 POINT(40.637345328958986 74.45406456031523) bank151184 +151185 POINT(40.70227674485704 74.61974009612766) bank151185 +151186 POINT(39.954096609836554 74.4509652378299) bank151186 +151187 POINT(41.70027715288845 73.16471999361404) bank151187 +151188 POINT(40.06674315688242 74.37672253188367) bank151188 +151189 POINT(40.38536103475566 73.94810228074545) bank151189 +151190 POINT(40.995459997368286 74.24746790587984) bank151190 +151191 POINT(39.82138581841027 74.05222218970931) bank151191 +151192 POINT(40.66615953179775 73.36858091737103) bank151192 +151193 POINT(40.42159227746551 73.82797997500977) bank151193 +151194 POINT(41.126183146325126 73.4691277499139) bank151194 +151195 POINT(39.82327099145109 74.02726528272564) bank151195 +151196 POINT(41.483944047558566 74.99210796756509) bank151196 +151197 POINT(39.770791529543196 73.13808369101898) bank151197 +151198 POINT(41.35070131505392 73.96140371172645) bank151198 +151199 POINT(40.433080814867616 73.32374326917277) bank151199 +151200 POINT(40.00721546468777 74.82853445873145) bank151200 +151201 POINT(41.11165128315849 73.65082164580937) bank151201 +151202 POINT(40.3828370999044 73.97294899969627) bank151202 +151203 POINT(41.360878935972735 74.28566347156568) bank151203 +151204 POINT(41.50982967115219 74.80555755625777) bank151204 +151205 POINT(41.528662969451595 73.16127438202254) bank151205 +151206 POINT(41.573989542557456 73.08420264792916) bank151206 +151207 POINT(41.29436675309673 73.95368358193141) bank151207 +151208 POINT(40.90373214337465 74.29607120300385) bank151208 +151209 POINT(40.30559980223029 73.77576110502795) bank151209 +151210 POINT(41.51126604482362 74.90715570364995) bank151210 +151211 POINT(40.12530040441484 74.35779239823293) bank151211 +151212 POINT(41.64152443533571 74.36722562922192) bank151212 +151213 POINT(41.429640472347494 74.18425426222397) bank151213 +151214 POINT(41.25793703445088 73.33217523106713) bank151214 +151215 POINT(40.06950359301594 74.4649265002647) bank151215 +151216 POINT(40.05610966371809 74.1626292765254) bank151216 +151217 POINT(40.61847027442885 73.74530099300863) bank151217 +151218 POINT(40.836739326737835 73.36922142363572) bank151218 +151219 POINT(40.89733262285312 74.32016753238798) bank151219 +151220 POINT(39.782880849779836 73.85844034181191) bank151220 +151221 POINT(40.180047890400644 74.27769021441921) bank151221 +151222 POINT(39.918610078457974 74.06304922592588) bank151222 +151223 POINT(40.245197606268135 73.46697128118387) bank151223 +151224 POINT(40.54872793420814 74.5823643305294) bank151224 +151225 POINT(41.01927112748016 73.06486806063828) bank151225 +151226 POINT(40.61473880895911 73.4760457403458) bank151226 +151227 POINT(40.41824068363222 73.67635569479788) bank151227 +151228 POINT(39.761816982613844 73.46780201029608) bank151228 +151229 POINT(39.81232558306902 74.17576635916276) bank151229 +151230 POINT(39.9417746977785 74.8938541330029) bank151230 +151231 POINT(41.57376390988829 73.28843164921592) bank151231 +151232 POINT(40.60894635072543 74.83585352948143) bank151232 +151233 POINT(40.018149353549376 73.15299240432013) bank151233 +151234 POINT(40.878693894170326 74.80170280875933) bank151234 +151235 POINT(41.61999832516413 73.47474893419502) bank151235 +151236 POINT(40.48913018481301 74.1006821245905) bank151236 +151237 POINT(39.89445011421699 73.20270565278462) bank151237 +151238 POINT(41.424276439781934 74.94640457985038) bank151238 +151239 POINT(40.87795694360006 73.44545670789452) bank151239 +151240 POINT(41.6824283451645 73.09723458669131) bank151240 +151241 POINT(39.80531226818345 73.32425974289592) bank151241 +151242 POINT(41.205852386987615 73.30403274058772) bank151242 +151243 POINT(40.847649980176925 74.47875226315824) bank151243 +151244 POINT(39.81933456725085 73.42772536834303) bank151244 +151245 POINT(40.182550988269895 74.01524574711979) bank151245 +151246 POINT(39.961997610403536 74.94307206853144) bank151246 +151247 POINT(41.289169325485304 74.34574646838712) bank151247 +151248 POINT(39.77521452031426 73.08167431883942) bank151248 +151249 POINT(41.13658334242372 73.64694945647405) bank151249 +151250 POINT(40.32601753621301 74.1333565589414) bank151250 +151251 POINT(41.49253655553647 74.49576129267895) bank151251 +151252 POINT(40.544607759839145 73.03897837029024) bank151252 +151253 POINT(40.526611677337414 73.39810119681768) bank151253 +151254 POINT(41.45356491353955 74.35167551309613) bank151254 +151255 POINT(39.989072997643675 73.57317262054552) bank151255 +151256 POINT(40.307546391843076 73.75556763196111) bank151256 +151257 POINT(39.855521331801874 74.4575840091103) bank151257 +151258 POINT(40.220069856564514 74.84303000411558) bank151258 +151259 POINT(40.73103739405471 74.0401213653954) bank151259 +151260 POINT(40.39707562386107 74.35675086633194) bank151260 +151261 POINT(40.521413250249324 73.45942627342562) bank151261 +151262 POINT(41.673147571581765 74.11572618874685) bank151262 +151263 POINT(40.23444358671847 73.3055995339932) bank151263 +151264 POINT(41.61139916127439 73.91329798507665) bank151264 +151265 POINT(41.31413247179505 73.20180878019096) bank151265 +151266 POINT(40.90387892456604 74.55656452458648) bank151266 +151267 POINT(40.40488217894523 73.46585364477208) bank151267 +151268 POINT(41.002022914660195 73.2510109139941) bank151268 +151269 POINT(40.4433256675354 74.26220297051667) bank151269 +151270 POINT(39.83808556656116 73.17548621238694) bank151270 +151271 POINT(41.1734002062066 74.06451638584241) bank151271 +151272 POINT(41.57564404487034 73.45221065142267) bank151272 +151273 POINT(40.35101414961364 73.72776526532111) bank151273 +151274 POINT(40.33984416380903 74.58213496374451) bank151274 +151275 POINT(40.216753602655814 74.2383507665023) bank151275 +151276 POINT(41.334664387257114 74.40900393535027) bank151276 +151277 POINT(41.21906100063501 74.15679322360901) bank151277 +151278 POINT(40.807901845241176 74.15027076442426) bank151278 +151279 POINT(40.83140241869969 74.46583579096584) bank151279 +151280 POINT(41.51099902879884 73.44217318995989) bank151280 +151281 POINT(39.82814852701118 73.38050211883166) bank151281 +151282 POINT(40.28590922155798 73.83197543366836) bank151282 +151283 POINT(41.05771498263965 73.46800295034811) bank151283 +151284 POINT(41.57725381420216 74.69692591679622) bank151284 +151285 POINT(40.096396473058185 73.60767669987591) bank151285 +151286 POINT(40.361226714690495 73.51737044215001) bank151286 +151287 POINT(39.79928013942786 73.25380216324149) bank151287 +151288 POINT(41.187625643308735 74.13428380021627) bank151288 +151289 POINT(41.42785954849725 74.69052426699051) bank151289 +151290 POINT(40.17718998840519 74.73040005015469) bank151290 +151291 POINT(40.24699010058729 74.18149949379844) bank151291 +151292 POINT(40.609922577977 73.26534553679308) bank151292 +151293 POINT(40.01188331815945 74.2064345683161) bank151293 +151294 POINT(41.23290287848854 73.39999928127317) bank151294 +151295 POINT(41.64673239674894 74.87089471674841) bank151295 +151296 POINT(40.22843228454771 73.64101640406253) bank151296 +151297 POINT(41.56174814554025 74.02166290618877) bank151297 +151298 POINT(41.035923374562756 74.43052865050194) bank151298 +151299 POINT(40.759390160817475 74.13502863974138) bank151299 +151300 POINT(41.4384023888496 74.82173268536458) bank151300 +151301 POINT(39.78686624961749 74.51636903109092) bank151301 +151302 POINT(40.24959845769751 73.0338514401517) bank151302 +151303 POINT(40.999679633700715 73.03801254211389) bank151303 +151304 POINT(39.8329035260925 73.8290139878225) bank151304 +151305 POINT(41.29390307639652 73.77344000331603) bank151305 +151306 POINT(41.48435562262848 73.8595312792023) bank151306 +151307 POINT(40.88077936674517 73.46167231476299) bank151307 +151308 POINT(40.99573424908269 73.6089446927527) bank151308 +151309 POINT(39.73867291699964 74.86527409126286) bank151309 +151310 POINT(41.25005858364713 74.48178746830149) bank151310 +151311 POINT(41.11346600736388 74.99952405635788) bank151311 +151312 POINT(40.516867272340825 73.2699813251582) bank151312 +151313 POINT(40.14071046128654 73.19898501354444) bank151313 +151314 POINT(39.95631635814751 73.97163619464997) bank151314 +151315 POINT(40.41765851023149 73.46612458069362) bank151315 +151316 POINT(40.74892526142901 74.72442656452792) bank151316 +151317 POINT(39.798552158699195 73.5404474605397) bank151317 +151318 POINT(40.94462881978668 74.02525024635868) bank151318 +151319 POINT(41.57724403989838 74.26316990600843) bank151319 +151320 POINT(39.87137992520203 74.51723144376375) bank151320 +151321 POINT(40.30581738207109 74.213588659568) bank151321 +151322 POINT(40.957362093290456 74.8364612447156) bank151322 +151323 POINT(40.23338841218797 74.19707150825465) bank151323 +151324 POINT(41.011133570216714 74.31221975047667) bank151324 +151325 POINT(40.911861529529375 74.71164720998146) bank151325 +151326 POINT(41.37705697861145 73.72374722203266) bank151326 +151327 POINT(40.08032572194978 73.85993560804157) bank151327 +151328 POINT(39.98432081232019 74.23552664989029) bank151328 +151329 POINT(39.81164915961141 74.1821557222423) bank151329 +151330 POINT(41.20543303727245 74.64187535945474) bank151330 +151331 POINT(40.61535298734777 73.82183237385246) bank151331 +151332 POINT(40.82807429799188 73.67696115451014) bank151332 +151333 POINT(40.73138433914137 73.26861624780139) bank151333 +151334 POINT(40.11489424005269 73.447795166995) bank151334 +151335 POINT(40.50547243071269 73.87055639011491) bank151335 +151336 POINT(40.995727275635964 73.72925938906332) bank151336 +151337 POINT(39.991168006501894 73.55222171356054) bank151337 +151338 POINT(40.88675305572639 73.63732723185116) bank151338 +151339 POINT(40.532575131940845 74.8014901908091) bank151339 +151340 POINT(41.096836492482964 73.60501337725532) bank151340 +151341 POINT(40.48967415266769 74.94429884974214) bank151341 +151342 POINT(40.48831438415908 74.47598921552907) bank151342 +151343 POINT(41.344553379366225 73.96817478542492) bank151343 +151344 POINT(40.938306697254134 73.28008766932933) bank151344 +151345 POINT(41.518776743632976 73.44317065701661) bank151345 +151346 POINT(40.29152808919518 73.73480754340302) bank151346 +151347 POINT(39.785920949196765 75.00320927395694) bank151347 +151348 POINT(41.2197852245928 73.95372600881245) bank151348 +151349 POINT(39.86212579283435 73.50415990265158) bank151349 +151350 POINT(40.20623587420719 73.06222515716155) bank151350 +151351 POINT(41.34938076148904 74.95564794197648) bank151351 +151352 POINT(41.513027394093214 73.02411777830099) bank151352 +151353 POINT(40.30734470482568 74.25370302050459) bank151353 +151354 POINT(40.15711172443369 74.9947589399052) bank151354 +151355 POINT(41.61195452112304 74.2868626533677) bank151355 +151356 POINT(40.339336153505414 74.05468203334149) bank151356 +151357 POINT(41.039859975530575 74.79925536852873) bank151357 +151358 POINT(41.365338283536275 74.6205688974748) bank151358 +151359 POINT(41.328020636922254 74.56973308214987) bank151359 +151360 POINT(40.707280556010474 73.19586415515343) bank151360 +151361 POINT(41.31766721002642 74.8117298966507) bank151361 +151362 POINT(40.2560863630348 73.2655775103611) bank151362 +151363 POINT(40.00882749316938 73.8139932293819) bank151363 +151364 POINT(39.87280165038891 74.55812134140922) bank151364 +151365 POINT(41.52372782940835 73.55512914446565) bank151365 +151366 POINT(39.99437827732905 74.52131752608463) bank151366 +151367 POINT(40.29341128298207 73.65933276833047) bank151367 +151368 POINT(41.48651312488047 73.43715799338926) bank151368 +151369 POINT(40.80614840104582 73.69499370765004) bank151369 +151370 POINT(40.07623764992755 74.2104811965941) bank151370 +151371 POINT(41.073927099737105 74.37049458777517) bank151371 +151372 POINT(40.20062226497488 74.18231350566367) bank151372 +151373 POINT(40.635637147229 74.42237487594292) bank151373 +151374 POINT(40.10611625862846 73.15542029440708) bank151374 +151375 POINT(41.525548173620386 74.96019982107342) bank151375 +151376 POINT(39.80472500878279 73.58180270981642) bank151376 +151377 POINT(41.014967836964885 74.41105515973695) bank151377 +151378 POINT(39.91439835095931 73.171156714416) bank151378 +151379 POINT(39.8310112213947 73.30525353434551) bank151379 +151380 POINT(41.6919532722763 74.09939227306592) bank151380 +151381 POINT(40.6289462018516 74.48266094043623) bank151381 +151382 POINT(41.034788148003734 73.01890822722808) bank151382 +151383 POINT(41.48218344697826 73.47324208517601) bank151383 +151384 POINT(41.03050092974531 74.61549350231573) bank151384 +151385 POINT(41.340870659733234 73.38116994926413) bank151385 +151386 POINT(41.3019606051166 73.95748375400665) bank151386 +151387 POINT(40.534163476590855 73.2356246880438) bank151387 +151388 POINT(40.08626336709766 74.27894527474876) bank151388 +151389 POINT(41.227757418577475 74.24758158578787) bank151389 +151390 POINT(39.780912421952735 73.72325346566528) bank151390 +151391 POINT(40.2776616265337 74.96041894611479) bank151391 +151392 POINT(41.32385125601546 74.80007697345943) bank151392 +151393 POINT(40.19972989348248 73.81124415216392) bank151393 +151394 POINT(40.47065440559288 74.42201813905402) bank151394 +151395 POINT(41.03486100002494 74.07264436794256) bank151395 +151396 POINT(40.31865973593905 74.52099208765317) bank151396 +151397 POINT(41.648134956977806 73.6117609208718) bank151397 +151398 POINT(41.377962552538065 74.93303955780148) bank151398 +151399 POINT(40.642275417999834 74.79791259607667) bank151399 +151400 POINT(40.57789935922721 73.99774367501573) bank151400 +151401 POINT(41.622604379369406 74.70602375076133) bank151401 +151402 POINT(40.65899865731676 73.37563076656053) bank151402 +151403 POINT(40.76599522835577 74.83139361396363) bank151403 +151404 POINT(41.60443417717517 74.05923941398547) bank151404 +151405 POINT(41.44426400236283 74.37698134641242) bank151405 +151406 POINT(40.19843313276963 73.48510734408259) bank151406 +151407 POINT(41.332074279776094 74.77840841888747) bank151407 +151408 POINT(41.703943237762324 74.64114137160516) bank151408 +151409 POINT(41.461239856206255 74.86178670555631) bank151409 +151410 POINT(40.82863165314426 73.87607490179171) bank151410 +151411 POINT(41.10819750687073 74.86430194565202) bank151411 +151412 POINT(40.56776663538024 73.73394450797785) bank151412 +151413 POINT(40.2771450369938 73.98355879787003) bank151413 +151414 POINT(39.7956474984677 73.24761929125975) bank151414 +151415 POINT(40.558797235533206 74.77186281453672) bank151415 +151416 POINT(40.516841694028265 74.04113320817626) bank151416 +151417 POINT(41.54106200976595 74.6125446344177) bank151417 +151418 POINT(41.657445332056135 73.83470675077615) bank151418 +151419 POINT(40.804523220054875 73.45557307895555) bank151419 +151420 POINT(41.22781121453188 74.86550443307314) bank151420 +151421 POINT(40.35503873901747 74.2418147522145) bank151421 +151422 POINT(40.093248223475335 73.76683651920048) bank151422 +151423 POINT(40.983478575709945 73.55330383895871) bank151423 +151424 POINT(41.06115941862379 74.72539107258486) bank151424 +151425 POINT(40.903202845183515 74.62080498154401) bank151425 +151426 POINT(40.03280905473931 73.31547149599038) bank151426 +151427 POINT(40.282755904487836 73.76578215841022) bank151427 +151428 POINT(41.043735525021305 73.43402110204836) bank151428 +151429 POINT(40.30705527418949 74.8300698144751) bank151429 +151430 POINT(39.90115541918458 73.69467023221598) bank151430 +151431 POINT(40.748733727386906 73.64371037426233) bank151431 +151432 POINT(39.850108668647444 73.30456407096104) bank151432 +151433 POINT(40.7027929081669 74.25660721300383) bank151433 +151434 POINT(40.30075914363292 73.42935493278844) bank151434 +151435 POINT(40.02772169040484 74.62768696874794) bank151435 +151436 POINT(41.5214417753815 73.41634255018782) bank151436 +151437 POINT(40.22458223913674 74.80993863723985) bank151437 +151438 POINT(41.5092722099937 73.6916392928022) bank151438 +151439 POINT(40.711719950233395 74.21240345590445) bank151439 +151440 POINT(40.66729459483467 73.47133570031642) bank151440 +151441 POINT(40.90457758395902 73.89531604140863) bank151441 +151442 POINT(40.81545359231679 74.33869756667772) bank151442 +151443 POINT(40.756222150766426 74.88739288040327) bank151443 +151444 POINT(41.28176614884778 74.26319967581317) bank151444 +151445 POINT(39.822602189058685 74.7574604488886) bank151445 +151446 POINT(40.83129306248699 74.91245731460513) bank151446 +151447 POINT(41.16278439178631 73.20982894367557) bank151447 +151448 POINT(39.92351471513877 74.75286451980222) bank151448 +151449 POINT(40.15226636257257 73.64005096046647) bank151449 +151450 POINT(40.091474120280765 73.6704924177998) bank151450 +151451 POINT(41.04644843598448 73.48004845729899) bank151451 +151452 POINT(40.711691066460915 73.7189837292353) bank151452 +151453 POINT(41.611752073241306 74.08333366052305) bank151453 +151454 POINT(40.961392124378385 74.55772609203777) bank151454 +151455 POINT(40.13446620315724 73.96641879645102) bank151455 +151456 POINT(40.00555403326984 74.72976406028299) bank151456 +151457 POINT(41.15279298768796 73.40202380778106) bank151457 +151458 POINT(40.38659791042143 73.58681419796176) bank151458 +151459 POINT(39.87700692937799 74.63472711542244) bank151459 +151460 POINT(39.85770518799347 74.10864709412371) bank151460 +151461 POINT(39.72719199095847 73.63303258201414) bank151461 +151462 POINT(41.49455890390343 74.10801779544168) bank151462 +151463 POINT(41.1641990352043 74.93685575961923) bank151463 +151464 POINT(40.10387886072483 74.13763162983317) bank151464 +151465 POINT(40.52500500787941 74.0660713403918) bank151465 +151466 POINT(39.956416291797765 74.06445249687967) bank151466 +151467 POINT(39.86222912475073 74.44071989325083) bank151467 +151468 POINT(41.45032584796387 74.84728769698675) bank151468 +151469 POINT(40.06514381387547 74.07909066777574) bank151469 +151470 POINT(39.893552693881475 74.48549115428091) bank151470 +151471 POINT(41.02227473226597 74.78979676089979) bank151471 +151472 POINT(39.96296303641031 73.45119748343747) bank151472 +151473 POINT(41.68611439684284 74.65225213443419) bank151473 +151474 POINT(41.486735761836165 74.40253976318886) bank151474 +151475 POINT(41.40520850291435 74.18469634957862) bank151475 +151476 POINT(40.75709088835592 74.83021122898832) bank151476 +151477 POINT(40.938613795918265 74.03016918979182) bank151477 +151478 POINT(39.72659906154493 73.90860919123106) bank151478 +151479 POINT(41.010253284737885 74.23600547215143) bank151479 +151480 POINT(41.02754372000346 73.21270590203851) bank151480 +151481 POINT(40.59244036768249 74.42731536706326) bank151481 +151482 POINT(40.65722291765414 73.85647931765517) bank151482 +151483 POINT(40.03592942075354 73.28281420572449) bank151483 +151484 POINT(40.567049004822074 73.61164226518274) bank151484 +151485 POINT(41.35747487894252 73.78683845541302) bank151485 +151486 POINT(41.23790685654707 73.24118687546543) bank151486 +151487 POINT(41.37053646990205 74.3025735256827) bank151487 +151488 POINT(40.34844525900365 74.40774715100693) bank151488 +151489 POINT(41.21020169960094 73.9415506056577) bank151489 +151490 POINT(40.165739439681 74.72057331397606) bank151490 +151491 POINT(41.117225517472 74.83855959748918) bank151491 +151492 POINT(41.50078457983915 74.37739801541488) bank151492 +151493 POINT(40.52442726999526 73.2824314244687) bank151493 +151494 POINT(39.95563841061106 73.59432303257535) bank151494 +151495 POINT(41.65609767681886 74.93835043301846) bank151495 +151496 POINT(40.73775083909818 73.48643965048475) bank151496 +151497 POINT(41.05010169234703 73.90539713303875) bank151497 +151498 POINT(40.15601141115113 73.31242749331781) bank151498 +151499 POINT(40.66494606196151 74.55627982987882) bank151499 +151500 POINT(41.64382101746927 74.91866928828745) bank151500 +151501 POINT(40.38686939783371 73.66362150574999) bank151501 +151502 POINT(40.489302621728584 73.68279355208796) bank151502 +151503 POINT(40.50249956877597 73.75337012121622) bank151503 +151504 POINT(39.795615084919234 73.48888979799862) bank151504 +151505 POINT(39.8448722346979 74.42583829776535) bank151505 +151506 POINT(40.04586341005187 73.32910679724282) bank151506 +151507 POINT(41.28062256661043 74.31840269385349) bank151507 +151508 POINT(40.3223355836543 73.52306460208861) bank151508 +151509 POINT(39.804868271909186 74.56835552319221) bank151509 +151510 POINT(39.96331902429523 74.27239703721061) bank151510 +151511 POINT(39.970618705018886 74.22192195763904) bank151511 +151512 POINT(40.25643736864553 74.58397583420549) bank151512 +151513 POINT(41.239793425471746 74.08977900856867) bank151513 +151514 POINT(41.47738461465748 74.43360992922526) bank151514 +151515 POINT(40.878712739685064 74.11384594718388) bank151515 +151516 POINT(41.01345423885976 73.45662064859933) bank151516 +151517 POINT(40.956092628820244 73.20046457767162) bank151517 +151518 POINT(41.13746536751297 74.0428019578652) bank151518 +151519 POINT(40.36593154401006 73.46716533919277) bank151519 +151520 POINT(41.440371576794504 74.57544380644792) bank151520 +151521 POINT(40.63859739383168 73.88668943486446) bank151521 +151522 POINT(40.11504134553034 74.13608231807605) bank151522 +151523 POINT(41.55634707438646 74.81715004876098) bank151523 +151524 POINT(40.10300530450898 74.93338154285092) bank151524 +151525 POINT(40.24622863710854 73.9817108757674) bank151525 +151526 POINT(41.332769385492135 74.34035157054267) bank151526 +151527 POINT(40.70954182033218 74.19677873937806) bank151527 +151528 POINT(40.027464914069334 74.08373668025507) bank151528 +151529 POINT(40.01881329289739 73.70401454510235) bank151529 +151530 POINT(41.44482341025505 74.9402973392863) bank151530 +151531 POINT(40.30548361837541 74.91002850494709) bank151531 +151532 POINT(41.126454219270215 74.12627562432513) bank151532 +151533 POINT(41.41584119246279 74.59557130263049) bank151533 +151534 POINT(41.51322033781014 74.67688872620202) bank151534 +151535 POINT(40.78309502410021 73.8689131713321) bank151535 +151536 POINT(41.29870607458184 73.8793339990562) bank151536 +151537 POINT(40.34371185942679 74.14722641904251) bank151537 +151538 POINT(40.56069711077564 73.17146809010794) bank151538 +151539 POINT(40.911697859582276 73.70016403051463) bank151539 +151540 POINT(41.017623590849595 73.7293847276885) bank151540 +151541 POINT(40.01193928513174 74.5638095858793) bank151541 +151542 POINT(40.45142421953611 74.21506809574116) bank151542 +151543 POINT(39.728213531490994 73.09600837287533) bank151543 +151544 POINT(41.50106237155116 73.81372288340391) bank151544 +151545 POINT(41.57900750605092 74.64321881554305) bank151545 +151546 POINT(40.60477274619996 74.71136466235025) bank151546 +151547 POINT(40.6521364106393 73.28163813137033) bank151547 +151548 POINT(40.184601808526864 74.22415221138618) bank151548 +151549 POINT(40.595785969569015 73.70910398145342) bank151549 +151550 POINT(40.58504667784294 74.64866530496354) bank151550 +151551 POINT(40.7221585101937 73.72825263855796) bank151551 +151552 POINT(41.2788561252325 74.01194214687743) bank151552 +151553 POINT(40.38573296848617 73.2561114363269) bank151553 +151554 POINT(39.90260052595183 73.55677956618882) bank151554 +151555 POINT(41.34177932812051 74.83131609647319) bank151555 +151556 POINT(40.72510657314949 74.31057728163293) bank151556 +151557 POINT(41.392366717191095 73.6685973010184) bank151557 +151558 POINT(41.128672851261605 74.22128708483966) bank151558 +151559 POINT(40.91902065920284 73.01737381728749) bank151559 +151560 POINT(41.60961717240522 74.01869523995084) bank151560 +151561 POINT(41.05334708152183 73.11032006332506) bank151561 +151562 POINT(39.815117634657945 73.33979886767307) bank151562 +151563 POINT(40.220432889901325 73.8291540290019) bank151563 +151564 POINT(40.47088670316054 74.3771036040648) bank151564 +151565 POINT(40.09699420770896 73.34277507805817) bank151565 +151566 POINT(41.1570649454149 73.129041095698) bank151566 +151567 POINT(40.42508641802092 73.89664113163568) bank151567 +151568 POINT(41.026660108493154 74.86949922551622) bank151568 +151569 POINT(41.065896505494855 74.14634498459195) bank151569 +151570 POINT(41.14838224203572 73.8370225489015) bank151570 +151571 POINT(39.95935849363166 73.63759390579371) bank151571 +151572 POINT(40.58502389029128 73.3995032374576) bank151572 +151573 POINT(41.03203754575321 73.03307461832995) bank151573 +151574 POINT(41.706349939410586 74.56556795664808) bank151574 +151575 POINT(39.75842591566721 74.39607277631266) bank151575 +151576 POINT(41.0824869905995 74.97100851389331) bank151576 +151577 POINT(41.65318543123568 73.50207160237188) bank151577 +151578 POINT(41.70825969764715 74.8103636041353) bank151578 +151579 POINT(40.9450583634156 73.05036535369958) bank151579 +151580 POINT(40.84712942937368 73.50480329529456) bank151580 +151581 POINT(40.18461820559823 73.7376626892597) bank151581 +151582 POINT(41.568593697885106 73.57051972607643) bank151582 +151583 POINT(41.35287916438197 74.16677994075069) bank151583 +151584 POINT(40.575304703543175 74.54502723233156) bank151584 +151585 POINT(39.90824772418132 74.12581298639506) bank151585 +151586 POINT(41.03695720738224 74.19532910359976) bank151586 +151587 POINT(41.036096945180105 73.90535132581118) bank151587 +151588 POINT(40.34361590605471 73.220231449215) bank151588 +151589 POINT(40.72918102530766 74.14662666372486) bank151589 +151590 POINT(40.902844323752426 73.57146922987006) bank151590 +151591 POINT(40.296620632494644 74.35078172380767) bank151591 +151592 POINT(40.37285245651868 74.10195740869372) bank151592 +151593 POINT(41.63524655775955 74.43621777802596) bank151593 +151594 POINT(40.24301154886167 74.7726674029068) bank151594 +151595 POINT(41.679073773083665 74.92140471915866) bank151595 +151596 POINT(40.44677197555861 73.41024044379263) bank151596 +151597 POINT(41.118459300175346 73.03673880485374) bank151597 +151598 POINT(40.696330013960896 74.35723712982073) bank151598 +151599 POINT(40.8709522443086 73.99400267070072) bank151599 +151600 POINT(40.080525680588785 73.55267070850158) bank151600 +151601 POINT(40.269263042888255 73.27081209004066) bank151601 +151602 POINT(41.48844640026729 73.48776534442328) bank151602 +151603 POINT(40.13116325999894 73.71046931040054) bank151603 +151604 POINT(40.581073639912695 73.53498756490447) bank151604 +151605 POINT(41.691061320340665 73.14553842225905) bank151605 +151606 POINT(40.370071343136004 73.26152214647182) bank151606 +151607 POINT(40.3444436420674 73.18503509242075) bank151607 +151608 POINT(41.545230006656304 74.10703455759254) bank151608 +151609 POINT(41.662143065659556 73.85849965032409) bank151609 +151610 POINT(40.538213586510395 74.315199914026) bank151610 +151611 POINT(41.57525313126945 74.83400788814613) bank151611 +151612 POINT(40.52360404473059 73.28669095854555) bank151612 +151613 POINT(40.34215850415665 73.82614120551476) bank151613 +151614 POINT(40.898321563799286 74.62910797839945) bank151614 +151615 POINT(40.45063387669546 74.93909549680166) bank151615 +151616 POINT(40.34729137886782 74.25230633802026) bank151616 +151617 POINT(40.66258665441011 74.26270764057287) bank151617 +151618 POINT(41.41754458278979 73.97065962856259) bank151618 +151619 POINT(40.482727722231914 74.03338807271298) bank151619 +151620 POINT(41.01289415630197 74.20930840702951) bank151620 +151621 POINT(40.25410588318048 73.71626438634948) bank151621 +151622 POINT(41.267227065397705 73.48917517368592) bank151622 +151623 POINT(40.61341216228246 73.45475172598279) bank151623 +151624 POINT(40.55519101149801 73.42877795959946) bank151624 +151625 POINT(40.41513235355514 74.47113159706885) bank151625 +151626 POINT(41.213544658778304 74.61062912881255) bank151626 +151627 POINT(40.76043403929253 73.02047764678662) bank151627 +151628 POINT(40.50436533208049 73.79193890192623) bank151628 +151629 POINT(41.078433705695254 74.01378680798734) bank151629 +151630 POINT(40.274813136784836 74.61205446948753) bank151630 +151631 POINT(40.71998394399183 73.28609100904363) bank151631 +151632 POINT(40.9299248480472 73.49958342345694) bank151632 +151633 POINT(40.20345940765632 73.9809019861957) bank151633 +151634 POINT(40.67459340401772 73.16257035166107) bank151634 +151635 POINT(41.674983445518386 73.27193318007504) bank151635 +151636 POINT(40.61316658604264 73.94821745671452) bank151636 +151637 POINT(41.499571856748815 74.75309417358721) bank151637 +151638 POINT(40.39993663246502 73.06705580130965) bank151638 +151639 POINT(40.498223083309774 74.42383686495165) bank151639 +151640 POINT(40.45329322840035 74.0849652794363) bank151640 +151641 POINT(40.49750012607726 74.97430560361548) bank151641 +151642 POINT(40.50296866364261 74.8102128278777) bank151642 +151643 POINT(39.78119794152326 74.85624109229894) bank151643 +151644 POINT(39.836958529308426 74.43505772006452) bank151644 +151645 POINT(41.504035180538985 74.73840617348587) bank151645 +151646 POINT(40.7068925260601 73.7208578849203) bank151646 +151647 POINT(39.7186457185835 74.41059771736865) bank151647 +151648 POINT(41.22841800908449 73.36469991837373) bank151648 +151649 POINT(40.000117755860074 73.7867197036706) bank151649 +151650 POINT(41.59047124198396 74.85582362565573) bank151650 +151651 POINT(39.95783164906952 74.52988916256881) bank151651 +151652 POINT(39.937992418928935 73.4634334897225) bank151652 +151653 POINT(40.80583994914628 73.49645042676292) bank151653 +151654 POINT(41.01728420951276 74.27199900825534) bank151654 +151655 POINT(40.073635606391505 74.03235517906928) bank151655 +151656 POINT(40.690272054873844 73.58371623157133) bank151656 +151657 POINT(41.592943758495686 74.94833365161765) bank151657 +151658 POINT(40.332461665589115 74.92114330534358) bank151658 +151659 POINT(41.334521630729604 74.34462027909035) bank151659 +151660 POINT(39.79964580701045 74.88534168809274) bank151660 +151661 POINT(40.235794854459236 74.2884651278594) bank151661 +151662 POINT(41.31356864304218 73.12502274551481) bank151662 +151663 POINT(41.31275926866748 73.86879761122611) bank151663 +151664 POINT(40.80717556403351 73.46432213867257) bank151664 +151665 POINT(40.76099791203575 73.61545563258274) bank151665 +151666 POINT(40.80881653401611 73.67936165116105) bank151666 +151667 POINT(41.39420702737504 73.37124940812767) bank151667 +151668 POINT(40.6875277937225 73.70551825846042) bank151668 +151669 POINT(40.3476742728623 73.1462632181071) bank151669 +151670 POINT(40.200372940865265 73.14713117521055) bank151670 +151671 POINT(40.20179591280543 73.06777260569345) bank151671 +151672 POINT(41.53149558663609 73.68829103341682) bank151672 +151673 POINT(39.95476581407901 74.53643130421928) bank151673 +151674 POINT(39.935966596634856 74.31728764814113) bank151674 +151675 POINT(39.92179280587606 73.40764379369253) bank151675 +151676 POINT(41.68031230017809 73.04994584535888) bank151676 +151677 POINT(39.86237888851084 74.66964237621542) bank151677 +151678 POINT(41.69469269637095 73.50713231036892) bank151678 +151679 POINT(39.739331953149446 74.68055176507814) bank151679 +151680 POINT(39.77668174908697 74.14518898876686) bank151680 +151681 POINT(41.027882143538314 74.8516442569839) bank151681 +151682 POINT(40.55956462167879 74.2664242156955) bank151682 +151683 POINT(40.234402231549474 73.707651922437) bank151683 +151684 POINT(40.571106767885794 74.78634835463365) bank151684 +151685 POINT(40.14779447564659 74.45693320443479) bank151685 +151686 POINT(40.72625813704343 74.5884281084546) bank151686 +151687 POINT(41.580483589753435 74.99949645568397) bank151687 +151688 POINT(40.154500120738305 73.47749825737054) bank151688 +151689 POINT(41.644122437596266 74.47540707217469) bank151689 +151690 POINT(41.53437468053238 74.13438407094822) bank151690 +151691 POINT(41.67399924514715 73.47940897137487) bank151691 +151692 POINT(40.74295042397033 74.85273878254894) bank151692 +151693 POINT(41.40130683427221 73.57705012416166) bank151693 +151694 POINT(40.97580098587252 74.23111120606592) bank151694 +151695 POINT(40.45983132626314 74.47178884765327) bank151695 +151696 POINT(40.36758850232961 73.88784713054561) bank151696 +151697 POINT(40.80049249387562 74.06548113512028) bank151697 +151698 POINT(40.49228693886495 73.80704542186255) bank151698 +151699 POINT(40.00775115605393 74.63586037754155) bank151699 +151700 POINT(41.43372005648155 73.97696038712023) bank151700 +151701 POINT(40.96226933087398 73.092042590954) bank151701 +151702 POINT(40.64029064174737 73.8422174135854) bank151702 +151703 POINT(39.95912822722866 74.09058638360301) bank151703 +151704 POINT(41.12621184151237 73.76155521047983) bank151704 +151705 POINT(39.89777959915358 74.64575273556252) bank151705 +151706 POINT(40.03166878204149 74.62466657229963) bank151706 +151707 POINT(40.448211612862465 74.47975300171497) bank151707 +151708 POINT(41.684987897268755 73.32391629530194) bank151708 +151709 POINT(40.250954121871466 73.54245358150492) bank151709 +151710 POINT(40.85969916579623 73.01571506192215) bank151710 +151711 POINT(41.466662546230275 74.15275459463382) bank151711 +151712 POINT(41.66510124434519 74.85133188690983) bank151712 +151713 POINT(41.604422579550686 73.42167758592721) bank151713 +151714 POINT(40.68143837919748 73.72219098058991) bank151714 +151715 POINT(41.42223171638282 73.38960728824105) bank151715 +151716 POINT(41.4073338040383 73.30235763857324) bank151716 +151717 POINT(41.03567241669747 74.04055265146802) bank151717 +151718 POINT(41.34790771833847 73.1340841891188) bank151718 +151719 POINT(40.04641612733402 73.02997284045261) bank151719 +151720 POINT(40.67809165374236 74.01467166164377) bank151720 +151721 POINT(41.4466371547544 73.54676419440247) bank151721 +151722 POINT(40.66681391625559 74.44516407644365) bank151722 +151723 POINT(41.68066941476793 74.7373299759454) bank151723 +151724 POINT(41.32832007845981 74.61547572619388) bank151724 +151725 POINT(39.84049195731899 73.99885173482932) bank151725 +151726 POINT(40.4007941157828 73.65319063305412) bank151726 +151727 POINT(40.251810556503635 73.8199939154602) bank151727 +151728 POINT(39.74284862004063 73.22492827212068) bank151728 +151729 POINT(41.30048195061062 74.70031027812917) bank151729 +151730 POINT(40.86951450924933 73.34853588161707) bank151730 +151731 POINT(40.047434516807854 73.40977120171374) bank151731 +151732 POINT(40.701538298518116 74.45101389182824) bank151732 +151733 POINT(41.42944240020053 74.53416882519576) bank151733 +151734 POINT(39.75620958274128 74.96153855086206) bank151734 +151735 POINT(41.23619406970102 73.98024675351031) bank151735 +151736 POINT(41.214898685506135 74.60808683651715) bank151736 +151737 POINT(40.31410602519339 73.65328966645093) bank151737 +151738 POINT(40.880182955405616 73.30674394945798) bank151738 +151739 POINT(39.863560794144284 74.82508800691886) bank151739 +151740 POINT(40.339104733365346 74.00990433168515) bank151740 +151741 POINT(40.446769859438945 73.78235372215433) bank151741 +151742 POINT(41.67000117284136 74.61541464035778) bank151742 +151743 POINT(41.6983921227098 73.73966849380146) bank151743 +151744 POINT(40.53143701472422 74.17361401956306) bank151744 +151745 POINT(39.7442631622317 74.95946761644927) bank151745 +151746 POINT(40.18032815187899 73.42335752866929) bank151746 +151747 POINT(39.971330730971715 74.81984475617165) bank151747 +151748 POINT(40.627576180707884 73.15524343822077) bank151748 +151749 POINT(41.211242216313806 74.2539322896822) bank151749 +151750 POINT(40.021571466991226 74.59951621037062) bank151750 +151751 POINT(40.974313385106065 73.1960673252001) bank151751 +151752 POINT(40.75795900665588 74.00564829303129) bank151752 +151753 POINT(41.32674329426197 73.8932690769598) bank151753 +151754 POINT(41.21022797656606 73.00868273713574) bank151754 +151755 POINT(40.40011012988154 74.24325548878411) bank151755 +151756 POINT(39.892853432730696 73.26537301678128) bank151756 +151757 POINT(40.79100976500077 74.73663594067186) bank151757 +151758 POINT(40.459161144403815 74.84402511840241) bank151758 +151759 POINT(40.25013886477248 73.09663781100906) bank151759 +151760 POINT(40.45137633349094 74.89901785365325) bank151760 +151761 POINT(40.95956331433047 73.16824355179425) bank151761 +151762 POINT(41.11286362999416 74.56608429113183) bank151762 +151763 POINT(41.33957955287091 73.28758041355067) bank151763 +151764 POINT(40.371566979376546 73.36933659835798) bank151764 +151765 POINT(40.70008099224109 74.92610107522219) bank151765 +151766 POINT(40.64040032302411 74.86222898882075) bank151766 +151767 POINT(40.478648247202734 73.18471986982603) bank151767 +151768 POINT(40.18535370440851 74.60454389389346) bank151768 +151769 POINT(39.752292975141394 73.76809560092771) bank151769 +151770 POINT(41.55644006834274 74.68610403922635) bank151770 +151771 POINT(41.64737498728828 73.97038786661716) bank151771 +151772 POINT(40.116599370034805 73.41133536792971) bank151772 +151773 POINT(41.58981556799707 73.81642984329693) bank151773 +151774 POINT(41.019475244007324 74.15161659749478) bank151774 +151775 POINT(41.698329476834445 73.85848209474433) bank151775 +151776 POINT(40.57968882529314 73.0259066776981) bank151776 +151777 POINT(40.28558121654341 73.2207282378699) bank151777 +151778 POINT(40.436449983290586 74.84484504532071) bank151778 +151779 POINT(40.55830673111084 73.79362658109875) bank151779 +151780 POINT(41.489383468449255 73.2244157388412) bank151780 +151781 POINT(40.30581564521408 73.21050781715861) bank151781 +151782 POINT(40.287613095626135 73.9800128905449) bank151782 +151783 POINT(40.80518662912578 74.62496180581732) bank151783 +151784 POINT(40.071092086195605 74.76529669335824) bank151784 +151785 POINT(40.87828482363496 74.05903549395758) bank151785 +151786 POINT(41.040455460822216 73.6344054073897) bank151786 +151787 POINT(41.06421356316999 73.39216748979064) bank151787 +151788 POINT(40.80332781177678 73.04878727911243) bank151788 +151789 POINT(40.380994567687225 74.0848723564318) bank151789 +151790 POINT(40.43645991167796 73.0114422608054) bank151790 +151791 POINT(40.21183590032307 74.41063369925128) bank151791 +151792 POINT(40.56913180492043 74.40901845317863) bank151792 +151793 POINT(41.358782332632046 74.56663440150305) bank151793 +151794 POINT(41.275882924691764 74.1473983629904) bank151794 +151795 POINT(39.86213227815297 74.64878284215035) bank151795 +151796 POINT(39.99973656749597 74.6586873346258) bank151796 +151797 POINT(39.75587155893012 74.76698354803383) bank151797 +151798 POINT(40.65869661645953 73.4271736500744) bank151798 +151799 POINT(40.54410793884814 73.28485129930922) bank151799 +151800 POINT(41.35869140796727 74.78997829228484) bank151800 +151801 POINT(41.1085574805641 73.61026710965237) bank151801 +151802 POINT(40.18541282029599 74.46368316931104) bank151802 +151803 POINT(40.272924897275814 74.94172168894929) bank151803 +151804 POINT(40.020088616656246 74.32974075043607) bank151804 +151805 POINT(40.65749918834972 74.5157184046804) bank151805 +151806 POINT(40.29586330859215 73.0511699905166) bank151806 +151807 POINT(41.024814113520506 74.57599938952471) bank151807 +151808 POINT(41.63239891271736 73.1337721707206) bank151808 +151809 POINT(41.33723386834379 74.52388918000095) bank151809 +151810 POINT(41.23416815379947 74.74345466389846) bank151810 +151811 POINT(40.83614096903045 74.18559549009579) bank151811 +151812 POINT(40.41022579372872 74.40946477171072) bank151812 +151813 POINT(41.339360229929184 74.50741502766697) bank151813 +151814 POINT(40.15353991575026 74.74218303742876) bank151814 +151815 POINT(39.89125274530109 74.4170886848123) bank151815 +151816 POINT(40.69312054716645 73.0836224586393) bank151816 +151817 POINT(41.31439025694672 74.159267956527) bank151817 +151818 POINT(40.594733592084694 73.95205700424856) bank151818 +151819 POINT(40.28531396034792 73.33291693169585) bank151819 +151820 POINT(39.781659072682224 73.09324472934732) bank151820 +151821 POINT(41.07227179026193 74.1749054561568) bank151821 +151822 POINT(41.05208215841068 74.86488249712592) bank151822 +151823 POINT(40.287555139495986 73.77188002032456) bank151823 +151824 POINT(40.50639790418549 74.10596645357617) bank151824 +151825 POINT(40.680562421320786 74.4880767247413) bank151825 +151826 POINT(40.91923267438662 74.47655354753981) bank151826 +151827 POINT(41.20711284836168 74.44682934165148) bank151827 +151828 POINT(40.140643121316074 73.50540500991202) bank151828 +151829 POINT(41.44757919156442 73.11327915083169) bank151829 +151830 POINT(40.44409828983672 73.69171240713484) bank151830 +151831 POINT(40.605106604378314 74.24114362133459) bank151831 +151832 POINT(40.73250707717154 74.27027457023182) bank151832 +151833 POINT(41.60931163422006 73.36536857394945) bank151833 +151834 POINT(40.941759479651914 74.11236685357845) bank151834 +151835 POINT(39.784085466235375 74.45627233119325) bank151835 +151836 POINT(41.6256046725496 74.89437297638194) bank151836 +151837 POINT(41.08036589618441 74.16585909594747) bank151837 +151838 POINT(40.09373974012155 74.78828796525323) bank151838 +151839 POINT(41.553321817707136 74.06030996767909) bank151839 +151840 POINT(40.66130085182046 73.90971169420904) bank151840 +151841 POINT(39.952816478464705 74.19114115612595) bank151841 +151842 POINT(40.55262937945486 74.5658046400092) bank151842 +151843 POINT(41.0302050419006 74.66011434872946) bank151843 +151844 POINT(40.81721611885233 74.33889914781747) bank151844 +151845 POINT(40.555416248286534 73.86621290269407) bank151845 +151846 POINT(41.70846185150833 74.16140030188936) bank151846 +151847 POINT(41.00116520386524 74.37116558544014) bank151847 +151848 POINT(40.11254757554506 73.76341630673463) bank151848 +151849 POINT(41.143709471448936 73.13362468721046) bank151849 +151850 POINT(40.09295209732295 74.25305273593575) bank151850 +151851 POINT(39.93259315993897 73.78511962892227) bank151851 +151852 POINT(40.62290837522144 74.61668268207868) bank151852 +151853 POINT(41.377341279986005 73.46181173898758) bank151853 +151854 POINT(40.268265129531514 73.2022624115359) bank151854 +151855 POINT(39.872744721009475 74.22147466533544) bank151855 +151856 POINT(41.092874434816125 73.45181349656036) bank151856 +151857 POINT(41.52643475971816 73.5158960276386) bank151857 +151858 POINT(40.753484274031294 74.92781659282034) bank151858 +151859 POINT(40.10763034633788 73.14756953752342) bank151859 +151860 POINT(41.286973121937145 74.2816444082132) bank151860 +151861 POINT(40.944474348031164 74.03551554106473) bank151861 +151862 POINT(40.66499010254768 74.51948306273822) bank151862 +151863 POINT(39.97852857689157 74.45687694313442) bank151863 +151864 POINT(40.23181385751089 73.34916343678273) bank151864 +151865 POINT(40.891477176055055 74.41857856016271) bank151865 +151866 POINT(40.8041145596793 73.71848177407958) bank151866 +151867 POINT(40.754083865821165 73.98944538567893) bank151867 +151868 POINT(40.95178393877044 73.65988856060218) bank151868 +151869 POINT(40.02381872126047 74.93649765388838) bank151869 +151870 POINT(41.684539207777355 73.42940575876837) bank151870 +151871 POINT(40.76331742992912 74.7216016602876) bank151871 +151872 POINT(41.517153228144174 73.857052080973) bank151872 +151873 POINT(40.31823301234404 74.33127705572261) bank151873 +151874 POINT(41.33597709575179 73.29645934298577) bank151874 +151875 POINT(40.45719147507234 73.71590197047787) bank151875 +151876 POINT(40.85482222970785 74.42013951986326) bank151876 +151877 POINT(39.97725283043096 73.64086791271686) bank151877 +151878 POINT(40.88437759793614 74.47834368309418) bank151878 +151879 POINT(41.2270798556023 74.77205295050453) bank151879 +151880 POINT(40.640459210460946 74.21606660832927) bank151880 +151881 POINT(40.11454593967296 74.20645686875379) bank151881 +151882 POINT(40.417138568598546 74.31686433054684) bank151882 +151883 POINT(41.36828484290053 73.05734762792679) bank151883 +151884 POINT(40.160494266686975 74.55141074078871) bank151884 +151885 POINT(39.900982783787896 73.68272256189485) bank151885 +151886 POINT(41.68726773951923 74.03629010281762) bank151886 +151887 POINT(39.97095157914321 73.26635290046573) bank151887 +151888 POINT(41.32351083269059 73.31917910709512) bank151888 +151889 POINT(41.573315125745836 73.09105638297173) bank151889 +151890 POINT(40.35316730205061 73.03472776940406) bank151890 +151891 POINT(41.115219018402044 73.55825888614179) bank151891 +151892 POINT(40.49263261263248 73.36163443308018) bank151892 +151893 POINT(41.15327656013313 74.58542846057607) bank151893 +151894 POINT(41.314909716541926 74.85197049805251) bank151894 +151895 POINT(40.463380052521416 73.09827174964744) bank151895 +151896 POINT(40.28340430970746 74.0699671266385) bank151896 +151897 POINT(40.9506356147454 73.96325109289528) bank151897 +151898 POINT(40.81769787855377 74.57755995752274) bank151898 +151899 POINT(40.659325898931726 73.89611748252041) bank151899 +151900 POINT(41.41762056981194 73.0918867956111) bank151900 +151901 POINT(39.752081103961416 73.33105903286327) bank151901 +151902 POINT(41.44462589164482 74.16031110193866) bank151902 +151903 POINT(40.39148476019427 74.91014185004325) bank151903 +151904 POINT(41.0471725323351 73.26202145337872) bank151904 +151905 POINT(40.425047818111274 73.43493812322527) bank151905 +151906 POINT(41.41456125318763 73.95826622431393) bank151906 +151907 POINT(40.29231519281357 74.17045546762698) bank151907 +151908 POINT(40.530721092692865 73.06997657722832) bank151908 +151909 POINT(40.25299261231069 74.09154673576711) bank151909 +151910 POINT(41.70227407712832 74.2371079765867) bank151910 +151911 POINT(41.28025373378178 74.59591854348886) bank151911 +151912 POINT(41.291754408364774 73.46701410480297) bank151912 +151913 POINT(39.99530571188022 74.32734520206968) bank151913 +151914 POINT(40.189179742278746 73.97120662609872) bank151914 +151915 POINT(40.75397560473048 74.54105707954584) bank151915 +151916 POINT(41.332909227905965 74.68681454602607) bank151916 +151917 POINT(40.192873022717855 73.96227456728376) bank151917 +151918 POINT(41.15021397874838 74.89617421807894) bank151918 +151919 POINT(41.41788657897761 74.81195393328157) bank151919 +151920 POINT(40.87630430210768 73.28728776031265) bank151920 +151921 POINT(39.92478933831462 74.12660973700478) bank151921 +151922 POINT(41.095088639636955 73.0395197982484) bank151922 +151923 POINT(41.25434612462047 74.37510039768242) bank151923 +151924 POINT(39.778867145172555 74.48401427817032) bank151924 +151925 POINT(40.15791004270611 74.42747928769897) bank151925 +151926 POINT(41.53982582488673 74.76356548518346) bank151926 +151927 POINT(40.91431397966065 74.75654409550587) bank151927 +151928 POINT(40.37008600347218 73.68214691303636) bank151928 +151929 POINT(41.334525106194356 74.5111528662716) bank151929 +151930 POINT(40.84187584995685 73.2167443760249) bank151930 +151931 POINT(41.317555387405775 73.14727176020529) bank151931 +151932 POINT(41.67853029886275 73.03692526920278) bank151932 +151933 POINT(40.43510854112493 73.42532078544852) bank151933 +151934 POINT(41.10150287695775 73.86511880164188) bank151934 +151935 POINT(40.68031230898551 74.79423449115059) bank151935 +151936 POINT(40.23472800661339 73.36620905905454) bank151936 +151937 POINT(40.73516642956146 73.29102164724696) bank151937 +151938 POINT(41.33842478963496 74.95988508829255) bank151938 +151939 POINT(40.03687679425511 73.91140450437682) bank151939 +151940 POINT(40.43575389635697 73.65094371460059) bank151940 +151941 POINT(40.94055334531975 74.02790270623359) bank151941 +151942 POINT(41.673788480319466 74.67840007234787) bank151942 +151943 POINT(40.55560507897239 73.63315504511452) bank151943 +151944 POINT(39.86211854501198 74.52689984567824) bank151944 +151945 POINT(41.08618412918139 73.92081660040158) bank151945 +151946 POINT(40.2508430485391 73.15317618136119) bank151946 +151947 POINT(41.41011753368853 74.677349473235) bank151947 +151948 POINT(41.48871231885818 74.67464631706024) bank151948 +151949 POINT(39.78219708857419 73.24836810312704) bank151949 +151950 POINT(41.31377079158381 74.23143600265745) bank151950 +151951 POINT(41.12634820623751 73.8523479982046) bank151951 +151952 POINT(39.882204757746834 74.37736503303334) bank151952 +151953 POINT(40.55634529380107 73.72432844395487) bank151953 +151954 POINT(41.38820702892326 74.27873355274366) bank151954 +151955 POINT(40.511854613152124 74.27962748924428) bank151955 +151956 POINT(40.50625984061353 73.36603285860656) bank151956 +151957 POINT(41.52481182438779 74.02157493427767) bank151957 +151958 POINT(40.394893693512124 73.31759178850179) bank151958 +151959 POINT(41.66710531643619 74.83566415484665) bank151959 +151960 POINT(40.12482490182104 74.64531530680794) bank151960 +151961 POINT(41.298173772200215 74.67460868732641) bank151961 +151962 POINT(40.68825665232258 73.73815999316295) bank151962 +151963 POINT(41.552722783516515 73.38689271875052) bank151963 +151964 POINT(40.118046119098224 74.5328626372282) bank151964 +151965 POINT(39.77598438068526 74.11506246782503) bank151965 +151966 POINT(40.75232102123436 74.85107075633606) bank151966 +151967 POINT(39.73169494232956 73.23939361324301) bank151967 +151968 POINT(41.33361840206166 74.7622257889095) bank151968 +151969 POINT(41.58689963826036 73.94413120845907) bank151969 +151970 POINT(39.88367450730605 73.17563597854914) bank151970 +151971 POINT(40.834123988469905 74.1894571297131) bank151971 +151972 POINT(40.634009572751424 74.47641856447287) bank151972 +151973 POINT(41.13202590850553 73.5993161456503) bank151973 +151974 POINT(40.052487362745175 74.07144498542128) bank151974 +151975 POINT(41.48597003392081 73.66319140509445) bank151975 +151976 POINT(40.680456997166466 73.37783001949778) bank151976 +151977 POINT(40.65080688213572 73.81921754957318) bank151977 +151978 POINT(41.15626744129245 74.20846093994827) bank151978 +151979 POINT(41.46835719954423 74.31411523160789) bank151979 +151980 POINT(40.31314048479753 74.19446627305688) bank151980 +151981 POINT(41.390371473826974 74.38183632741155) bank151981 +151982 POINT(40.60982565126565 74.14811602460195) bank151982 +151983 POINT(40.385535158906 73.50186654248799) bank151983 +151984 POINT(41.33843643529167 74.18454068202452) bank151984 +151985 POINT(40.97487268984144 73.77634083655724) bank151985 +151986 POINT(40.240610872318705 73.93178230872387) bank151986 +151987 POINT(40.95746554550764 73.07132025185004) bank151987 +151988 POINT(40.74716046999886 74.50120419181737) bank151988 +151989 POINT(40.40892252004574 73.63868646147981) bank151989 +151990 POINT(40.85976548826781 74.9967216742055) bank151990 +151991 POINT(41.228909365247276 74.50877162339998) bank151991 +151992 POINT(40.32683094559846 73.98604035282816) bank151992 +151993 POINT(39.883614385254454 73.79520654983689) bank151993 +151994 POINT(41.497059570628046 74.88199153855913) bank151994 +151995 POINT(41.304060623199554 73.6258416119237) bank151995 +151996 POINT(41.55017649580658 73.45426157710459) bank151996 +151997 POINT(41.48356519719851 74.84405951564014) bank151997 +151998 POINT(41.49560902410519 74.50985831897184) bank151998 +151999 POINT(40.21243781173851 74.52411965002523) bank151999 +152000 POINT(40.145108855680604 73.80249844494837) bank152000 +152001 POINT(40.625156600362544 73.4087991849779) bank152001 +152002 POINT(41.03560858925096 73.11717600223525) bank152002 +152003 POINT(40.14379302170657 73.11156457278346) bank152003 +152004 POINT(40.365035802706785 74.32975361582865) bank152004 +152005 POINT(41.57754119348815 74.04429702584757) bank152005 +152006 POINT(41.51030133940457 74.31315674940711) bank152006 +152007 POINT(41.599811891376405 74.13246250901587) bank152007 +152008 POINT(40.335532757323705 73.76838768358886) bank152008 +152009 POINT(41.655301916833146 74.24098487905154) bank152009 +152010 POINT(39.863719573684264 73.43464079560651) bank152010 +152011 POINT(40.94741488445681 74.46716825868762) bank152011 +152012 POINT(40.5834758392332 74.37527833491065) bank152012 +152013 POINT(40.52164923798228 73.78714832980712) bank152013 +152014 POINT(41.18580621569185 73.08177786325908) bank152014 +152015 POINT(40.316038842608094 74.66856272696332) bank152015 +152016 POINT(41.68757570981112 74.62025824019801) bank152016 +152017 POINT(41.02836673600229 74.29031247838091) bank152017 +152018 POINT(41.23151223970037 73.200339580605) bank152018 +152019 POINT(41.31635302939005 73.6985906568191) bank152019 +152020 POINT(40.541675056731656 74.16606462141098) bank152020 +152021 POINT(39.77185810078446 73.17032415424411) bank152021 +152022 POINT(41.03658582734883 74.32945829011139) bank152022 +152023 POINT(41.66279004637126 73.24578953239067) bank152023 +152024 POINT(41.2320426622945 74.39789854000435) bank152024 +152025 POINT(40.921457649463015 74.72120921021455) bank152025 +152026 POINT(40.719571400791196 73.79254855800843) bank152026 +152027 POINT(40.282757949195094 73.12791088598864) bank152027 +152028 POINT(39.7577969163366 73.42074841533365) bank152028 +152029 POINT(40.1195450005044 73.59775928447273) bank152029 +152030 POINT(40.50121508451475 73.83825763104217) bank152030 +152031 POINT(41.036216800004425 74.59367054253464) bank152031 +152032 POINT(40.267450933650494 73.01931295179709) bank152032 +152033 POINT(40.69290824315922 74.33126198843887) bank152033 +152034 POINT(41.284316027425014 74.99105015922267) bank152034 +152035 POINT(40.42729556515851 73.42611192446398) bank152035 +152036 POINT(41.014727198606344 74.3206355766547) bank152036 +152037 POINT(40.74490045665792 73.87975611474059) bank152037 +152038 POINT(41.36184369056233 73.44737162827232) bank152038 +152039 POINT(40.67425869408072 74.12707210455255) bank152039 +152040 POINT(40.78135389054746 74.23497172382892) bank152040 +152041 POINT(41.69745733353063 74.18179030758637) bank152041 +152042 POINT(40.419302778778096 73.14431365980423) bank152042 +152043 POINT(39.74564493065089 73.93275179143811) bank152043 +152044 POINT(40.921084103600215 73.6222949819556) bank152044 +152045 POINT(40.78346381532255 73.52277698102465) bank152045 +152046 POINT(40.48761130325353 73.68975145176142) bank152046 +152047 POINT(40.812774921000276 73.68609333300577) bank152047 +152048 POINT(40.813939782006514 74.49676785756031) bank152048 +152049 POINT(40.394462187846464 73.90153507447718) bank152049 +152050 POINT(41.58619057607246 74.33132356058586) bank152050 +152051 POINT(40.07484329944325 73.02250772073255) bank152051 +152052 POINT(41.19509419962478 74.88401709559342) bank152052 +152053 POINT(39.78234805367305 74.74902864981523) bank152053 +152054 POINT(39.833550551731534 74.68285616648875) bank152054 +152055 POINT(40.44190804853974 74.68776052531622) bank152055 +152056 POINT(41.02562443960201 73.14619478616743) bank152056 +152057 POINT(40.40925808446165 74.7178858923774) bank152057 +152058 POINT(40.377520124959176 73.23345817695996) bank152058 +152059 POINT(39.73536682592979 74.72927322296663) bank152059 +152060 POINT(40.85100009773536 74.98434040656913) bank152060 +152061 POINT(40.07273797265777 74.7136865467856) bank152061 +152062 POINT(40.27216467351986 74.7613622600092) bank152062 +152063 POINT(41.36210936123342 74.71417453589595) bank152063 +152064 POINT(40.23403060791318 74.26401395631981) bank152064 +152065 POINT(40.73110372753437 73.77314896383128) bank152065 +152066 POINT(40.79502572635966 74.28355008722022) bank152066 +152067 POINT(40.926639222643914 74.77840917286393) bank152067 +152068 POINT(39.94857974113756 74.3534771029625) bank152068 +152069 POINT(41.62798370483248 73.21707947329365) bank152069 +152070 POINT(39.79988547278128 74.79493868611347) bank152070 +152071 POINT(40.80962043991844 73.58093425338588) bank152071 +152072 POINT(41.06057881802076 73.38835923744779) bank152072 +152073 POINT(41.45397199766612 73.29832520506329) bank152073 +152074 POINT(40.763450729457404 74.99940368685355) bank152074 +152075 POINT(41.00775436015222 73.88077939399835) bank152075 +152076 POINT(40.555994316985824 73.82315238894432) bank152076 +152077 POINT(41.00569467983881 73.74896453385176) bank152077 +152078 POINT(40.71175150526799 73.37130245270569) bank152078 +152079 POINT(40.708664319488385 73.06256460850226) bank152079 +152080 POINT(39.743728788129836 73.77399430601645) bank152080 +152081 POINT(41.22521883706404 73.53887796981432) bank152081 +152082 POINT(40.48945044129714 74.16273674892064) bank152082 +152083 POINT(40.182996168363495 74.77284314470774) bank152083 +152084 POINT(39.805151017525574 73.23040959366456) bank152084 +152085 POINT(41.24434052986895 74.4344152334453) bank152085 +152086 POINT(39.93337068259925 74.44948966012186) bank152086 +152087 POINT(41.51129115626061 74.76339452459173) bank152087 +152088 POINT(40.17984678124506 73.97413920120682) bank152088 +152089 POINT(41.427757368871916 73.7322610131505) bank152089 +152090 POINT(40.537244085609544 74.83066454502529) bank152090 +152091 POINT(41.43960675020338 73.99286948315715) bank152091 +152092 POINT(40.57027077931756 73.64059080957061) bank152092 +152093 POINT(40.34350636866554 74.86884111586332) bank152093 +152094 POINT(40.68650206288528 74.35596018450468) bank152094 +152095 POINT(41.4751891787893 74.24119890426687) bank152095 +152096 POINT(39.93323975385661 74.78510490195659) bank152096 +152097 POINT(41.530787516637815 73.74294622251314) bank152097 +152098 POINT(41.309503594916634 73.82373832016201) bank152098 +152099 POINT(41.3346670545553 73.5089842990377) bank152099 +152100 POINT(41.633009311568145 74.59273492522134) bank152100 +152101 POINT(40.55317512029898 74.073284343564) bank152101 +152102 POINT(39.95713714410535 73.52151373634646) bank152102 +152103 POINT(40.98158821733278 73.40924848065744) bank152103 +152104 POINT(41.63216135802205 73.89665123734827) bank152104 +152105 POINT(40.211994578142665 74.80981511132019) bank152105 +152106 POINT(39.90293945101261 74.53712113124033) bank152106 +152107 POINT(39.87712532914848 74.0901303473625) bank152107 +152108 POINT(39.73664405869734 73.87327337491463) bank152108 +152109 POINT(40.934721796466064 74.92129030170649) bank152109 +152110 POINT(40.82755390604913 74.05830604875045) bank152110 +152111 POINT(40.77354858320909 73.97420246110352) bank152111 +152112 POINT(40.12136273639172 74.66987432638884) bank152112 +152113 POINT(40.5134038042281 73.37081049497942) bank152113 +152114 POINT(40.154164385541236 74.12899163453402) bank152114 +152115 POINT(41.42186628640635 74.89321844008141) bank152115 +152116 POINT(40.58219613610684 74.42454386582214) bank152116 +152117 POINT(40.73295817288261 73.33556199515445) bank152117 +152118 POINT(40.798357745480224 73.99192899335974) bank152118 +152119 POINT(41.060322212010846 74.12596508536188) bank152119 +152120 POINT(41.626788477577335 73.3346862786611) bank152120 +152121 POINT(40.0788764685553 73.41955627500683) bank152121 +152122 POINT(40.39049823973651 74.0523959171911) bank152122 +152123 POINT(40.78096341667698 73.37234428401813) bank152123 +152124 POINT(40.04537488192362 73.18461079882864) bank152124 +152125 POINT(40.87066937250043 73.60292545407273) bank152125 +152126 POINT(39.7980644372132 73.09548344868107) bank152126 +152127 POINT(39.96271842656121 74.88323271203537) bank152127 +152128 POINT(40.69619520290035 74.82396230574548) bank152128 +152129 POINT(40.76599375699037 73.6876898489917) bank152129 +152130 POINT(41.15774026826026 73.69364037223814) bank152130 +152131 POINT(40.38530634012783 74.45302797726916) bank152131 +152132 POINT(41.49892531678219 74.73151538892928) bank152132 +152133 POINT(41.27648258178597 73.7426456210899) bank152133 +152134 POINT(40.97204758793141 74.89612895313776) bank152134 +152135 POINT(41.09525122077484 73.7994900584296) bank152135 +152136 POINT(40.44240992903426 73.52805766686308) bank152136 +152137 POINT(41.34242241270959 74.43347728702962) bank152137 +152138 POINT(40.975316859180175 73.72480382680604) bank152138 +152139 POINT(41.127342132894945 74.98348605280846) bank152139 +152140 POINT(39.783468534964655 73.17522398321556) bank152140 +152141 POINT(40.90483885539211 73.28158089836721) bank152141 +152142 POINT(41.2864422074765 74.9499400944061) bank152142 +152143 POINT(40.04552168107692 74.74127749740454) bank152143 +152144 POINT(40.37343222006162 74.72540047626333) bank152144 +152145 POINT(40.97249466211977 73.19374166640861) bank152145 +152146 POINT(41.5132076755406 73.13397716861314) bank152146 +152147 POINT(41.5195731672806 74.45516067917892) bank152147 +152148 POINT(40.510535931923656 73.85695411473834) bank152148 +152149 POINT(41.69908351894727 74.78428075132672) bank152149 +152150 POINT(41.62717189342792 73.75800646194436) bank152150 +152151 POINT(40.534807150344136 73.4531891498224) bank152151 +152152 POINT(41.48675894417116 73.94641340107452) bank152152 +152153 POINT(40.27150288698667 74.83664127794724) bank152153 +152154 POINT(41.38844529147578 74.84945893204245) bank152154 +152155 POINT(39.8388650511616 74.627418239028) bank152155 +152156 POINT(40.95072955814493 73.82238834671145) bank152156 +152157 POINT(41.18037183404617 73.34145246830893) bank152157 +152158 POINT(39.977198399726475 74.72073717849429) bank152158 +152159 POINT(41.42513037264502 73.91778016851916) bank152159 +152160 POINT(40.55195499032292 74.1380943528174) bank152160 +152161 POINT(39.81589167239232 73.40806756004446) bank152161 +152162 POINT(39.92910878700722 73.39886546065586) bank152162 +152163 POINT(40.50073186822783 73.65072164004002) bank152163 +152164 POINT(40.072585229214894 74.75821011117988) bank152164 +152165 POINT(39.869087815030426 73.70201120669881) bank152165 +152166 POINT(40.80639035732133 74.13571570921805) bank152166 +152167 POINT(41.00031799472086 74.19376531528327) bank152167 +152168 POINT(40.786068971190204 74.94329766137946) bank152168 +152169 POINT(39.861559897818076 74.2229472203867) bank152169 +152170 POINT(41.51867237353126 74.64957254383533) bank152170 +152171 POINT(40.59995708378145 73.1028232315765) bank152171 +152172 POINT(39.932705605525214 74.54232000899646) bank152172 +152173 POINT(41.5600329236281 74.92121634665799) bank152173 +152174 POINT(41.05416754863381 74.44473900077348) bank152174 +152175 POINT(40.19932265528832 74.87322350183197) bank152175 +152176 POINT(40.35598701942662 73.39840516472319) bank152176 +152177 POINT(40.6889378643665 73.74060150442348) bank152177 +152178 POINT(41.39754328046428 73.3451826484113) bank152178 +152179 POINT(39.77257707106019 74.12931566188696) bank152179 +152180 POINT(40.042811889451464 73.18642605445321) bank152180 +152181 POINT(40.628895458028175 73.77792648163211) bank152181 +152182 POINT(39.80891787892914 74.39567573444609) bank152182 +152183 POINT(40.74556723329504 74.6586538120869) bank152183 +152184 POINT(39.88977922356866 74.8422847414304) bank152184 +152185 POINT(40.96445262573058 73.25334280680916) bank152185 +152186 POINT(40.72583522364435 73.98045215181897) bank152186 +152187 POINT(39.870031226620654 74.90436201692943) bank152187 +152188 POINT(41.56428532093384 74.69316190501208) bank152188 +152189 POINT(41.361625737284314 74.41401960649472) bank152189 +152190 POINT(41.62567091621811 73.71326782991613) bank152190 +152191 POINT(40.139236529705364 73.85786852536924) bank152191 +152192 POINT(40.27523084970535 74.02733258217026) bank152192 +152193 POINT(41.208193371261444 74.10312098350268) bank152193 +152194 POINT(39.95504748254452 73.0134439123513) bank152194 +152195 POINT(40.80005176600389 73.95734912345934) bank152195 +152196 POINT(40.69601741216143 73.71215782711964) bank152196 +152197 POINT(40.36430844971525 74.36555346180849) bank152197 +152198 POINT(39.92256493759612 74.40420204107016) bank152198 +152199 POINT(41.32769207344332 73.00642786742387) bank152199 +152200 POINT(40.404715304540794 73.2583889289664) bank152200 +152201 POINT(40.50044831038842 73.95917291141424) bank152201 +152202 POINT(41.50447481697054 73.78945755041909) bank152202 +152203 POINT(40.81808489724708 74.82517391988107) bank152203 +152204 POINT(41.21422728977891 73.25083794433944) bank152204 +152205 POINT(41.332433574597914 73.45167816478231) bank152205 +152206 POINT(41.334820650596164 74.83087056091566) bank152206 +152207 POINT(41.330733231648274 73.81320516117593) bank152207 +152208 POINT(39.939752095582314 74.35205193128161) bank152208 +152209 POINT(41.6515193378834 74.60752412387605) bank152209 +152210 POINT(40.45217893011268 73.38873192436742) bank152210 +152211 POINT(41.53020396100957 74.11561090087726) bank152211 +152212 POINT(41.08005099570274 74.81470974855357) bank152212 +152213 POINT(40.45699817550864 74.1749967228308) bank152213 +152214 POINT(40.88187304973471 74.20777499657976) bank152214 +152215 POINT(39.762202655602835 73.3852576843707) bank152215 +152216 POINT(41.06009515369017 73.12980668127433) bank152216 +152217 POINT(41.195727164485646 73.739262120245) bank152217 +152218 POINT(40.712041028008706 73.40238237524599) bank152218 +152219 POINT(39.738301362091775 74.58655617621501) bank152219 +152220 POINT(41.51994595697893 74.13918559431684) bank152220 +152221 POINT(41.4643444218984 73.38653380407918) bank152221 +152222 POINT(41.20051769447133 73.07533966546151) bank152222 +152223 POINT(40.29928258478068 73.87174252549855) bank152223 +152224 POINT(41.160760133035005 73.20049289598956) bank152224 +152225 POINT(40.36035868980633 74.98969995185313) bank152225 +152226 POINT(39.76968536011668 73.30151564109379) bank152226 +152227 POINT(40.079970306168875 74.48608665741277) bank152227 +152228 POINT(40.11423424701651 73.67015695831819) bank152228 +152229 POINT(41.32463179513967 74.72169790814897) bank152229 +152230 POINT(40.4755755826501 73.05933368711331) bank152230 +152231 POINT(39.860362267610995 74.80703810098588) bank152231 +152232 POINT(40.29198280908977 74.9086434628721) bank152232 +152233 POINT(41.19114851220371 74.90857681837618) bank152233 +152234 POINT(41.63775073754034 73.49965376370915) bank152234 +152235 POINT(40.69636660606464 73.49470689095426) bank152235 +152236 POINT(41.29163184913437 73.00881953006501) bank152236 +152237 POINT(39.74821176069233 74.16681264458553) bank152237 +152238 POINT(40.69485864414848 74.57278281340824) bank152238 +152239 POINT(40.935267309419885 74.81993144964757) bank152239 +152240 POINT(41.36235893010639 73.17813163888749) bank152240 +152241 POINT(40.33288413090611 73.88112374814594) bank152241 +152242 POINT(41.37504198153595 74.2430367047232) bank152242 +152243 POINT(40.78307405907352 73.33438163169211) bank152243 +152244 POINT(40.39023836505121 74.24051651254068) bank152244 +152245 POINT(39.85394677640662 73.41750365356593) bank152245 +152246 POINT(41.348513086273464 74.83684943680737) bank152246 +152247 POINT(40.66950466989077 74.00288268413726) bank152247 +152248 POINT(39.861805661393696 73.0585727454378) bank152248 +152249 POINT(40.54440137904046 74.29766060822003) bank152249 +152250 POINT(40.51958462456392 73.26020754860016) bank152250 +152251 POINT(41.07363993304043 73.33625084650467) bank152251 +152252 POINT(40.42808436421923 74.25064973042136) bank152252 +152253 POINT(40.14838800775036 74.82998932509015) bank152253 +152254 POINT(40.63477046290082 74.03608679702369) bank152254 +152255 POINT(40.31238873083603 74.79963769654184) bank152255 +152256 POINT(40.10677759852802 73.92454866580609) bank152256 +152257 POINT(41.010367464780586 74.26805681550732) bank152257 +152258 POINT(40.572734665352876 73.54670670264257) bank152258 +152259 POINT(41.36279210839278 73.70194274266646) bank152259 +152260 POINT(41.17746789509936 73.34174524544407) bank152260 +152261 POINT(41.189246041454574 73.43348349763411) bank152261 +152262 POINT(40.208897153902555 74.47481351569331) bank152262 +152263 POINT(41.22287038892112 73.59961569863631) bank152263 +152264 POINT(41.33875174914591 74.86151202539601) bank152264 +152265 POINT(41.21770717522657 73.78713886874985) bank152265 +152266 POINT(40.15542206098167 74.65900571502547) bank152266 +152267 POINT(41.53442187458417 74.17336624058196) bank152267 +152268 POINT(40.435592426100925 73.4566824292926) bank152268 +152269 POINT(40.34133549446816 74.06409260149235) bank152269 +152270 POINT(41.09259326384079 73.82863724724474) bank152270 +152271 POINT(40.593326073651625 74.798464820719) bank152271 +152272 POINT(39.71521835885269 74.3174245286079) bank152272 +152273 POINT(40.887555986777095 73.96454522681782) bank152273 +152274 POINT(40.38107392634537 74.9051258164187) bank152274 +152275 POINT(40.996400625692424 73.82971050304768) bank152275 +152276 POINT(41.218860352486324 73.59399518526082) bank152276 +152277 POINT(39.84789668093768 74.96048569520266) bank152277 +152278 POINT(41.42888675197567 73.16526237323407) bank152278 +152279 POINT(40.662815335902074 73.71266828273482) bank152279 +152280 POINT(40.89355645400704 74.38736213377607) bank152280 +152281 POINT(41.665745593392124 74.35563726199901) bank152281 +152282 POINT(41.51440232177998 74.53173693194121) bank152282 +152283 POINT(39.95056056024519 74.58525545677495) bank152283 +152284 POINT(40.135569315720105 74.79757210047077) bank152284 +152285 POINT(40.6012162684266 74.54810879700597) bank152285 +152286 POINT(41.181323885952935 74.30854977120073) bank152286 +152287 POINT(40.568010816754935 74.82380689566283) bank152287 +152288 POINT(39.9461263385265 74.65852007862745) bank152288 +152289 POINT(40.36349287558577 73.98836987535823) bank152289 +152290 POINT(40.9374743096711 74.27638430890019) bank152290 +152291 POINT(39.76198518679454 74.41048093877113) bank152291 +152292 POINT(40.40663714109159 74.48516579202484) bank152292 +152293 POINT(40.66285437794874 74.27605242153682) bank152293 +152294 POINT(41.0851667826422 73.22953710416202) bank152294 +152295 POINT(41.59430677734129 73.6996699045141) bank152295 +152296 POINT(41.58543418810123 73.7364066041229) bank152296 +152297 POINT(41.29594606933399 73.31921201932202) bank152297 +152298 POINT(40.38505025241467 74.49748300176199) bank152298 +152299 POINT(41.12565842854687 73.04837962643109) bank152299 +152300 POINT(41.04713674907327 74.97765344430975) bank152300 +152301 POINT(40.210322513026796 74.09858459720702) bank152301 +152302 POINT(41.41868197774483 73.916163470799) bank152302 +152303 POINT(40.698272515318564 74.1164361631472) bank152303 +152304 POINT(39.87910512191803 74.61574199200628) bank152304 +152305 POINT(40.792481744226066 73.85819001267275) bank152305 +152306 POINT(40.73948908794642 73.64830241453541) bank152306 +152307 POINT(41.52312853751383 73.5365608366915) bank152307 +152308 POINT(41.22759733681329 73.46472252003294) bank152308 +152309 POINT(41.24939373123321 74.612665739702) bank152309 +152310 POINT(41.66538383008938 73.22896677021369) bank152310 +152311 POINT(40.20230325877126 74.85471951186948) bank152311 +152312 POINT(41.217076716892905 74.55959001576029) bank152312 +152313 POINT(40.47646130714345 73.21911283961212) bank152313 +152314 POINT(41.18035751513037 73.73519705022704) bank152314 +152315 POINT(40.92973287258558 74.40510383181282) bank152315 +152316 POINT(41.300073603437305 74.27415129233704) bank152316 +152317 POINT(41.49235110703611 74.59979569239566) bank152317 +152318 POINT(41.39129154428053 74.56215388331137) bank152318 +152319 POINT(40.06945290908618 74.33582443148798) bank152319 +152320 POINT(40.944624666608156 74.28905829222114) bank152320 +152321 POINT(41.301742595596025 73.34818190908618) bank152321 +152322 POINT(40.872002924135586 74.61069274531157) bank152322 +152323 POINT(40.979091157466065 74.18221951975622) bank152323 +152324 POINT(40.62608243447685 73.36439279004816) bank152324 +152325 POINT(39.74802357054225 74.31759378708261) bank152325 +152326 POINT(40.450981670961134 74.67680516657812) bank152326 +152327 POINT(41.5502744044142 74.93645794891916) bank152327 +152328 POINT(39.997016453522946 74.3059718394193) bank152328 +152329 POINT(41.43451710108301 73.49790096341648) bank152329 +152330 POINT(40.68924012459242 74.17141496999771) bank152330 +152331 POINT(41.638980698281486 73.19699680097186) bank152331 +152332 POINT(40.85591890814045 74.71475582177979) bank152332 +152333 POINT(40.19530562261918 73.43295700955795) bank152333 +152334 POINT(40.47798622801276 73.10347298086072) bank152334 +152335 POINT(40.7299181142197 74.82735502406459) bank152335 +152336 POINT(39.89156915583379 73.80587522063908) bank152336 +152337 POINT(40.110652788968665 73.72133310313725) bank152337 +152338 POINT(40.786808984486775 73.70325004449039) bank152338 +152339 POINT(40.14904640370207 74.25996689193472) bank152339 +152340 POINT(41.265681365098374 74.58732423788602) bank152340 +152341 POINT(41.53382529018801 74.28710412759466) bank152341 +152342 POINT(41.43857478870616 74.33551080444818) bank152342 +152343 POINT(40.350547294278584 73.65083208703464) bank152343 +152344 POINT(40.008313964878035 73.09332143710265) bank152344 +152345 POINT(41.03534734014249 73.24750068722224) bank152345 +152346 POINT(39.79319526605341 73.89799388707173) bank152346 +152347 POINT(39.887470045761106 74.39344042086857) bank152347 +152348 POINT(40.47157098339484 74.7956602115005) bank152348 +152349 POINT(40.25038083484796 74.87776658529155) bank152349 +152350 POINT(41.1122576362365 73.83532497715227) bank152350 +152351 POINT(39.97722577387047 74.21174624278943) bank152351 +152352 POINT(40.40152270803599 73.79658789322059) bank152352 +152353 POINT(39.91528721533467 73.22934969081565) bank152353 +152354 POINT(41.337510318628006 73.41031049899448) bank152354 +152355 POINT(40.23513751276402 73.07923962232634) bank152355 +152356 POINT(40.06318615863577 73.37004103395535) bank152356 +152357 POINT(41.7110514704108 73.06489470206263) bank152357 +152358 POINT(41.6208697460976 74.64932021812096) bank152358 +152359 POINT(39.90415970562107 73.72023323258244) bank152359 +152360 POINT(40.55956314458108 73.45891822782532) bank152360 +152361 POINT(40.2360286988174 74.05902252398197) bank152361 +152362 POINT(40.1664726445762 73.78518761182119) bank152362 +152363 POINT(41.56264940824963 73.83243189002006) bank152363 +152364 POINT(40.80310178639772 73.33485174086341) bank152364 +152365 POINT(41.314490022940454 73.1255401699071) bank152365 +152366 POINT(41.14643638829542 73.8175482941675) bank152366 +152367 POINT(40.25211873578193 74.38174511313636) bank152367 +152368 POINT(41.183878814574655 74.15549057982045) bank152368 +152369 POINT(40.92181993356524 73.94169566614305) bank152369 +152370 POINT(41.05305372164706 74.07354027717817) bank152370 +152371 POINT(40.39835855705458 73.61138384146959) bank152371 +152372 POINT(40.35381200993889 73.92128420209129) bank152372 +152373 POINT(40.43011473155425 73.32418642713873) bank152373 +152374 POINT(39.82751569167571 73.67315856733507) bank152374 +152375 POINT(41.594551799918484 73.91384878480677) bank152375 +152376 POINT(40.51967735458473 74.24993697951666) bank152376 +152377 POINT(41.34354510612959 73.25791499791325) bank152377 +152378 POINT(41.17843823648916 73.84183062391293) bank152378 +152379 POINT(39.823854841744975 74.507846145658) bank152379 +152380 POINT(40.89370744053302 74.24127807274813) bank152380 +152381 POINT(40.47854882999472 74.75154950320415) bank152381 +152382 POINT(41.37393795431661 73.81984178932102) bank152382 +152383 POINT(40.493484016937785 74.8021125315357) bank152383 +152384 POINT(40.23441300186915 74.71498560116264) bank152384 +152385 POINT(39.772025404644836 74.43216829667854) bank152385 +152386 POINT(40.87007784100533 73.492865157634) bank152386 +152387 POINT(39.99189531201022 73.19648795834148) bank152387 +152388 POINT(40.82574758101384 73.38960138731976) bank152388 +152389 POINT(40.44938007949051 74.76923538814673) bank152389 +152390 POINT(40.33923134125384 73.360899153858) bank152390 +152391 POINT(41.075301711691864 74.2258619112915) bank152391 +152392 POINT(41.06581090234919 74.31276332413715) bank152392 +152393 POINT(41.01292895134998 73.42350993588819) bank152393 +152394 POINT(40.001532040477784 74.1895715018081) bank152394 +152395 POINT(41.20752309244031 73.07254670311285) bank152395 +152396 POINT(41.40419376480608 73.7501574893517) bank152396 +152397 POINT(40.052503387590804 74.79005847966002) bank152397 +152398 POINT(39.97355789382124 74.35549671455937) bank152398 +152399 POINT(40.50652929270753 74.22084446116372) bank152399 +152400 POINT(39.942830921101105 74.53336712033385) bank152400 +152401 POINT(41.51545652550018 74.15149599459974) bank152401 +152402 POINT(41.549956028381565 73.73504049867722) bank152402 +152403 POINT(40.80404188717224 74.49975262079548) bank152403 +152404 POINT(41.64833598514982 73.16926294516448) bank152404 +152405 POINT(41.643461311553324 74.63990729106577) bank152405 +152406 POINT(40.640127558857216 74.05268348703991) bank152406 +152407 POINT(40.226161165779665 73.9001753997748) bank152407 +152408 POINT(40.614044933532085 73.25908616272596) bank152408 +152409 POINT(40.3252166220121 73.2736433028805) bank152409 +152410 POINT(39.762561363273846 74.70949415038558) bank152410 +152411 POINT(40.71077450827366 73.82440592623111) bank152411 +152412 POINT(39.8599258371096 73.7194670293393) bank152412 +152413 POINT(40.27069364156299 73.13957176316077) bank152413 +152414 POINT(39.9545155355232 74.59773096394476) bank152414 +152415 POINT(41.481954747561026 74.90430708255303) bank152415 +152416 POINT(40.0981416430123 73.97643801627645) bank152416 +152417 POINT(41.34601815303831 74.90378281662286) bank152417 +152418 POINT(40.13074195148167 73.72566640273645) bank152418 +152419 POINT(40.18601874444812 73.92147128916001) bank152419 +152420 POINT(39.75523573577427 73.04295142245769) bank152420 +152421 POINT(39.81269799602099 73.67731880147446) bank152421 +152422 POINT(41.39671999064966 74.62493886504315) bank152422 +152423 POINT(40.348689948039485 74.66553055958784) bank152423 +152424 POINT(41.190703377360755 75.00361609490932) bank152424 +152425 POINT(39.89169348509444 74.45828740316648) bank152425 +152426 POINT(40.91204354142555 74.03406487943191) bank152426 +152427 POINT(40.38491411127874 73.24277834691038) bank152427 +152428 POINT(40.48126012240454 74.50040804853141) bank152428 +152429 POINT(39.824942967030616 73.83648457063654) bank152429 +152430 POINT(39.84863627366304 74.47021595796826) bank152430 +152431 POINT(40.9043857309778 73.50732316258672) bank152431 +152432 POINT(41.524624729486135 74.57807695370427) bank152432 +152433 POINT(41.43086092107368 73.7887745936944) bank152433 +152434 POINT(40.75641810748369 74.84299382393796) bank152434 +152435 POINT(41.06907485269675 73.17258196863257) bank152435 +152436 POINT(41.194601687919224 74.84009453391538) bank152436 +152437 POINT(40.7267730262279 74.33887741985548) bank152437 +152438 POINT(40.32278668605814 74.55905555661175) bank152438 +152439 POINT(40.7643318916081 73.91099973830143) bank152439 +152440 POINT(41.23332426455014 74.61337711494319) bank152440 +152441 POINT(39.82435365946562 73.84516673463804) bank152441 +152442 POINT(41.48374321573492 73.03215812912045) bank152442 +152443 POINT(40.85812192871499 73.27351994674206) bank152443 +152444 POINT(41.19671345242232 73.8389726549456) bank152444 +152445 POINT(41.08482421281811 74.51034635875655) bank152445 +152446 POINT(39.9930591217403 74.71767841686727) bank152446 +152447 POINT(39.91339371276591 74.78204101768954) bank152447 +152448 POINT(40.37316454733307 74.69484119368447) bank152448 +152449 POINT(39.88213244944694 73.03691343607412) bank152449 +152450 POINT(39.854780194827605 73.11665344243083) bank152450 +152451 POINT(40.40660227111072 74.07906268645299) bank152451 +152452 POINT(41.422718045451376 73.02047429733605) bank152452 +152453 POINT(40.8914929813331 74.22336458914363) bank152453 +152454 POINT(41.4526155412184 74.8422402647249) bank152454 +152455 POINT(40.52924240602001 73.13548253775751) bank152455 +152456 POINT(39.77498281632771 74.52830690082816) bank152456 +152457 POINT(40.04733115550686 74.75711066843607) bank152457 +152458 POINT(40.56663487252488 74.06562807206771) bank152458 +152459 POINT(40.819149039777265 73.3536301641854) bank152459 +152460 POINT(39.97717638785746 73.04342718381837) bank152460 +152461 POINT(39.90690349492238 74.20872959058168) bank152461 +152462 POINT(41.132018168922684 74.17258436380376) bank152462 +152463 POINT(40.189550383814314 74.74847461504459) bank152463 +152464 POINT(39.87394836988588 74.89556113602441) bank152464 +152465 POINT(40.68372006414475 74.46896769743174) bank152465 +152466 POINT(40.86441441465596 73.10493488958464) bank152466 +152467 POINT(41.03762592029503 73.78270279273684) bank152467 +152468 POINT(39.958176271584456 74.19280688287519) bank152468 +152469 POINT(40.71088214669116 73.45058926206434) bank152469 +152470 POINT(40.87245028007973 73.58422253287932) bank152470 +152471 POINT(40.16443752283216 73.64218037136106) bank152471 +152472 POINT(41.31670714139064 73.61997224680262) bank152472 +152473 POINT(41.54664355155936 74.614462300883) bank152473 +152474 POINT(41.081369489227626 73.98540398293426) bank152474 +152475 POINT(40.498942261736254 73.03589729865247) bank152475 +152476 POINT(40.54563230663672 74.57664844545869) bank152476 +152477 POINT(40.943325632965085 73.55740905738162) bank152477 +152478 POINT(40.03869663881869 74.92012963408476) bank152478 +152479 POINT(40.71563937376414 74.04043184212861) bank152479 +152480 POINT(41.126721885344544 74.70021829255687) bank152480 +152481 POINT(40.94149165390889 74.53276850511925) bank152481 +152482 POINT(40.04182585053193 74.69422478351235) bank152482 +152483 POINT(40.94189327413655 73.38468479587999) bank152483 +152484 POINT(41.195365958336616 73.54753216622979) bank152484 +152485 POINT(40.24466147585171 74.52059285670728) bank152485 +152486 POINT(40.248802937326545 73.73179827249862) bank152486 +152487 POINT(41.094200215505104 74.14518353439246) bank152487 +152488 POINT(41.150331692360794 74.94446563075708) bank152488 +152489 POINT(41.67216644408309 73.98539087598432) bank152489 +152490 POINT(40.34834190623706 74.07649548794944) bank152490 +152491 POINT(40.74759065390753 73.42651125054195) bank152491 +152492 POINT(39.92619072066833 73.70263741937315) bank152492 +152493 POINT(40.9295890969496 73.86019847709515) bank152493 +152494 POINT(41.316445182945536 73.84743023415507) bank152494 +152495 POINT(41.0076606712356 73.60322495494862) bank152495 +152496 POINT(41.13784290367176 74.7338672091039) bank152496 +152497 POINT(41.18890898138324 73.02394424418249) bank152497 +152498 POINT(40.19382755638904 73.94863940731582) bank152498 +152499 POINT(40.42084498535119 74.40031676983149) bank152499 +152500 POINT(41.42638045217107 73.06457266976118) bank152500 +152501 POINT(40.68023354138539 73.35117767686715) bank152501 +152502 POINT(39.73981478613207 74.17530939838058) bank152502 +152503 POINT(40.208825323227714 73.34479213692755) bank152503 +152504 POINT(40.168979932300886 73.06189332833397) bank152504 +152505 POINT(40.95637534002334 74.06886884757819) bank152505 +152506 POINT(41.10949566176815 74.21487543375537) bank152506 +152507 POINT(40.327020526312936 73.3022248223624) bank152507 +152508 POINT(39.84741210135388 74.15747888209623) bank152508 +152509 POINT(41.66671104333381 74.49967206673898) bank152509 +152510 POINT(40.13872640800544 74.34367645880208) bank152510 +152511 POINT(41.57921139352837 73.33697760879602) bank152511 +152512 POINT(41.58688476875807 73.09963914837505) bank152512 +152513 POINT(40.50393312398508 74.109728921516) bank152513 +152514 POINT(41.40546574004274 74.8303508319006) bank152514 +152515 POINT(40.63113592329498 73.55860379941915) bank152515 +152516 POINT(40.22172170983386 73.24841396406599) bank152516 +152517 POINT(40.57245153097737 74.75326669034767) bank152517 +152518 POINT(39.954037111701695 73.78014248717965) bank152518 +152519 POINT(41.156260652595996 73.92479599759172) bank152519 +152520 POINT(40.83825691440789 73.98409225434759) bank152520 +152521 POINT(40.216246518408866 74.44464325102514) bank152521 +152522 POINT(41.39718190417069 74.29925582567047) bank152522 +152523 POINT(41.168853494504674 73.94631121102235) bank152523 +152524 POINT(40.07702282488614 73.21184981691694) bank152524 +152525 POINT(41.36397258363724 73.12447688259053) bank152525 +152526 POINT(41.011106355296945 73.02777897763848) bank152526 +152527 POINT(39.98964046232246 73.23464880427984) bank152527 +152528 POINT(41.161995578065145 73.46160217938024) bank152528 +152529 POINT(39.75145577382545 73.26875966730753) bank152529 +152530 POINT(41.169900947372575 74.17904664453448) bank152530 +152531 POINT(41.377906802706725 74.03951048849994) bank152531 +152532 POINT(41.45966984136365 74.13263938153577) bank152532 +152533 POINT(40.94350636833891 74.45527233467774) bank152533 +152534 POINT(39.85317402714988 73.35897548302299) bank152534 +152535 POINT(40.62057987135193 73.9670968333971) bank152535 +152536 POINT(40.51878704546054 74.6278478126512) bank152536 +152537 POINT(41.153482125098755 73.2939589914519) bank152537 +152538 POINT(40.87966685985437 74.48682019475196) bank152538 +152539 POINT(40.24634638663658 74.98208994442399) bank152539 +152540 POINT(40.18863013168777 74.0790776728114) bank152540 +152541 POINT(41.611032636024696 74.17618537328028) bank152541 +152542 POINT(40.02226678870681 74.71745826647617) bank152542 +152543 POINT(40.809324666765356 74.95811289118613) bank152543 +152544 POINT(40.60165988472763 74.12415674467013) bank152544 +152545 POINT(40.472220163301586 73.8472783582926) bank152545 +152546 POINT(41.468509916345084 74.36904821198881) bank152546 +152547 POINT(40.70653869060108 73.43654117513405) bank152547 +152548 POINT(41.68657334040775 73.56676916401268) bank152548 +152549 POINT(41.414158158601815 74.40274031063376) bank152549 +152550 POINT(40.956316832430225 74.56673691012175) bank152550 +152551 POINT(40.16207052915143 74.57984101324388) bank152551 +152552 POINT(40.1388935689873 74.97119101620866) bank152552 +152553 POINT(41.080650623270664 73.54023019936946) bank152553 +152554 POINT(41.4641387307494 74.77042438406372) bank152554 +152555 POINT(40.25511295214773 74.0964394203319) bank152555 +152556 POINT(41.26038112036018 74.12319220757406) bank152556 +152557 POINT(41.48446651493652 74.0017082534351) bank152557 +152558 POINT(40.64733545367585 73.85095726664018) bank152558 +152559 POINT(40.25559278137566 74.02543287006627) bank152559 +152560 POINT(40.227893734033415 74.83108351946754) bank152560 +152561 POINT(40.15765129006962 73.84039255590014) bank152561 +152562 POINT(40.201940929790894 73.65736367494497) bank152562 +152563 POINT(41.08658482271008 74.02359349288157) bank152563 +152564 POINT(40.01419329627833 73.25471289697573) bank152564 +152565 POINT(41.066761768269856 74.89942110891018) bank152565 +152566 POINT(40.510296842504935 73.08539495675568) bank152566 +152567 POINT(41.14119644622999 74.75660860085792) bank152567 +152568 POINT(40.739476282851335 73.69919373021166) bank152568 +152569 POINT(41.62797147873118 73.62090663620336) bank152569 +152570 POINT(40.43569014526351 73.16267123499883) bank152570 +152571 POINT(40.311401708671816 74.43573535658648) bank152571 +152572 POINT(41.45125286589805 74.94952303297022) bank152572 +152573 POINT(41.630424062677086 74.11596946961306) bank152573 +152574 POINT(41.05852645280481 74.8150707633863) bank152574 +152575 POINT(41.66573916369969 74.5017720913779) bank152575 +152576 POINT(40.67449465614843 73.18950338372042) bank152576 +152577 POINT(40.28886565893673 73.43428821026092) bank152577 +152578 POINT(41.19532164935599 74.08873432806521) bank152578 +152579 POINT(41.673129893839125 73.95977998980538) bank152579 +152580 POINT(41.234702373017925 74.26721814873346) bank152580 +152581 POINT(40.73662480499836 74.07392695759748) bank152581 +152582 POINT(41.05267034150917 73.87883853945543) bank152582 +152583 POINT(40.623294408238195 73.13809128148146) bank152583 +152584 POINT(41.090979150123474 74.79277861001106) bank152584 +152585 POINT(40.420882185294694 74.02408435461552) bank152585 +152586 POINT(41.0605918592905 73.78794478429718) bank152586 +152587 POINT(41.149743092266284 74.84907690349516) bank152587 +152588 POINT(40.198671529287 74.14803753108148) bank152588 +152589 POINT(39.9942728547693 73.25841255826343) bank152589 +152590 POINT(40.51621537290175 73.83888364155517) bank152590 +152591 POINT(40.74397030517057 74.74403756885292) bank152591 +152592 POINT(41.39926048776122 73.3858893629834) bank152592 +152593 POINT(40.59147749192 74.16920292339185) bank152593 +152594 POINT(39.76965793379532 74.79676958148994) bank152594 +152595 POINT(40.8022459253966 74.45724635829603) bank152595 +152596 POINT(41.00047194326599 74.46736059978896) bank152596 +152597 POINT(41.64248372280746 73.74772599987661) bank152597 +152598 POINT(40.69192632855061 74.09467014212353) bank152598 +152599 POINT(39.874989593195465 73.6021864659223) bank152599 +152600 POINT(39.87939850883523 74.71524518104222) bank152600 +152601 POINT(40.32955101882071 74.01247179753014) bank152601 +152602 POINT(41.01513062191513 74.37847442989278) bank152602 +152603 POINT(40.171147397965484 74.6903255639131) bank152603 +152604 POINT(40.96987778863353 74.34847665636285) bank152604 +152605 POINT(41.24700124232231 73.01558274498267) bank152605 +152606 POINT(40.94658651691219 73.38461128242949) bank152606 +152607 POINT(40.87326503444772 73.53567398760138) bank152607 +152608 POINT(40.92561375592042 74.5577446711084) bank152608 +152609 POINT(41.32122740882982 74.22926239967494) bank152609 +152610 POINT(40.94486229530117 73.61193822139391) bank152610 +152611 POINT(39.84759167244305 74.58713385719949) bank152611 +152612 POINT(40.170705974170986 73.30684325897583) bank152612 +152613 POINT(41.541987069828316 74.54997025260244) bank152613 +152614 POINT(40.38606584423977 74.72298303076596) bank152614 +152615 POINT(41.494700064967205 73.29867810963029) bank152615 +152616 POINT(39.82203910021121 73.04617781469308) bank152616 +152617 POINT(41.126733285726566 73.83431543297658) bank152617 +152618 POINT(40.654775366498214 74.05791998228005) bank152618 +152619 POINT(40.705235699151686 73.77075946118076) bank152619 +152620 POINT(39.97151465978155 73.28740568247282) bank152620 +152621 POINT(40.743503635293905 73.39866733998326) bank152621 +152622 POINT(39.91070211621009 74.91766984333422) bank152622 +152623 POINT(40.747690335130685 74.63610233051783) bank152623 +152624 POINT(39.99447243550533 73.29614853527936) bank152624 +152625 POINT(41.13979736533001 73.86826089942757) bank152625 +152626 POINT(39.739314771692015 73.79816293904705) bank152626 +152627 POINT(40.018523362488054 74.07829539794889) bank152627 +152628 POINT(40.638796664891196 74.5950475365372) bank152628 +152629 POINT(40.83696319002264 74.25918371293233) bank152629 +152630 POINT(41.69633176255039 73.21269429352213) bank152630 +152631 POINT(41.03886588219674 73.95742483460936) bank152631 +152632 POINT(40.384751672396575 73.98694618462822) bank152632 +152633 POINT(41.52084490244031 73.13062468941894) bank152633 +152634 POINT(41.41158941423671 73.10059186220988) bank152634 +152635 POINT(41.33511082123847 74.73348912958465) bank152635 +152636 POINT(40.40226758831791 73.15156984196841) bank152636 +152637 POINT(41.30466546892306 73.43577713618882) bank152637 +152638 POINT(41.147599967664625 74.52383005545356) bank152638 +152639 POINT(41.70391760928523 74.82578377309768) bank152639 +152640 POINT(41.21026754325727 73.09754571813993) bank152640 +152641 POINT(39.9311217909763 73.69218583686424) bank152641 +152642 POINT(40.20757961877982 74.4890181075942) bank152642 +152643 POINT(39.78670377912537 74.56884843228124) bank152643 +152644 POINT(40.90417649752341 74.89342302495368) bank152644 +152645 POINT(41.68765882974807 73.73962530857122) bank152645 +152646 POINT(40.794185573439655 73.07071453235888) bank152646 +152647 POINT(39.88862575826596 74.51867372790338) bank152647 +152648 POINT(40.15322263697462 73.9428974105016) bank152648 +152649 POINT(41.421069757922524 73.58983231692689) bank152649 +152650 POINT(40.595790552864436 73.53663237996244) bank152650 +152651 POINT(40.975711865500486 74.74279385102939) bank152651 +152652 POINT(40.101351180093744 73.43257265445682) bank152652 +152653 POINT(40.020300239910824 73.56066044918462) bank152653 +152654 POINT(40.90255850105785 74.77327728224297) bank152654 +152655 POINT(40.86926483912356 73.40349352142366) bank152655 +152656 POINT(41.46505969126209 73.26651556149385) bank152656 +152657 POINT(41.61342938993734 74.68862492150375) bank152657 +152658 POINT(39.92488469491981 73.641300688604) bank152658 +152659 POINT(40.446112335133684 73.9970318412758) bank152659 +152660 POINT(41.38564372746142 73.91818890248861) bank152660 +152661 POINT(41.24045032829905 74.96272981785997) bank152661 +152662 POINT(41.343376643191036 73.04823917025656) bank152662 +152663 POINT(40.55225920568867 73.50854472650605) bank152663 +152664 POINT(40.17670281394081 73.99175298599162) bank152664 +152665 POINT(39.78848669607718 73.59634929072585) bank152665 +152666 POINT(39.740497595658496 74.0937692842863) bank152666 +152667 POINT(40.44614798471495 74.24647198228769) bank152667 +152668 POINT(39.808294310207025 74.167933925989) bank152668 +152669 POINT(40.42920071075323 74.40742982522247) bank152669 +152670 POINT(39.80762167428553 73.43476781850013) bank152670 +152671 POINT(40.837375323360305 74.7678575288144) bank152671 +152672 POINT(41.03948124107017 73.48085020739218) bank152672 +152673 POINT(41.566774079005505 74.61251523979269) bank152673 +152674 POINT(40.2050484499291 74.27629894855596) bank152674 +152675 POINT(41.566922776852614 74.00835130245105) bank152675 +152676 POINT(41.020437238310926 73.08788010524404) bank152676 +152677 POINT(40.31222489701477 74.24995509463311) bank152677 +152678 POINT(41.475343728653556 73.80858916411722) bank152678 +152679 POINT(40.7316407170241 73.58298662810736) bank152679 +152680 POINT(40.102362228592504 74.02022072244203) bank152680 +152681 POINT(39.86562902056269 73.21326032033863) bank152681 +152682 POINT(40.7354562327812 74.06490667602057) bank152682 +152683 POINT(40.99630524779654 74.61971486006169) bank152683 +152684 POINT(40.51997009885633 74.73645813600035) bank152684 +152685 POINT(41.247615530022955 73.06413567766599) bank152685 +152686 POINT(41.6040288290335 74.64863607920613) bank152686 +152687 POINT(41.186873776044706 73.44081530069114) bank152687 +152688 POINT(40.19500416609057 73.33871352933036) bank152688 +152689 POINT(40.49587954601974 74.44859265729441) bank152689 +152690 POINT(41.62111455935241 73.1780663845454) bank152690 +152691 POINT(40.3979846584668 73.07664880708201) bank152691 +152692 POINT(40.45283074632267 73.56559305493208) bank152692 +152693 POINT(41.37240999943268 73.22559077542932) bank152693 +152694 POINT(40.539005906368494 74.88741266652384) bank152694 +152695 POINT(41.66929169785357 73.26242817661839) bank152695 +152696 POINT(41.61755668003144 73.19793890018305) bank152696 +152697 POINT(41.53910487852561 73.42599720401245) bank152697 +152698 POINT(40.960443377580354 73.5599168241315) bank152698 +152699 POINT(41.37167086073955 73.59786767756277) bank152699 +152700 POINT(39.93909209202977 73.32923950776785) bank152700 +152701 POINT(40.805406370801975 74.11206660333487) bank152701 +152702 POINT(40.15232824800852 73.35777092508594) bank152702 +152703 POINT(40.016702539225996 73.28360349796621) bank152703 +152704 POINT(40.18954236153571 73.14612661309152) bank152704 +152705 POINT(39.86281228620909 74.35637154342862) bank152705 +152706 POINT(41.3945349382314 73.82331637770896) bank152706 +152707 POINT(40.64365686696476 74.05213049711881) bank152707 +152708 POINT(40.900598175416526 74.17061933085964) bank152708 +152709 POINT(39.83652027121936 73.18675445965025) bank152709 +152710 POINT(40.666747566784494 73.04920145629318) bank152710 +152711 POINT(40.66086603480904 73.5090104528658) bank152711 +152712 POINT(41.287847870780496 73.92469817290505) bank152712 +152713 POINT(40.59365103624538 73.13820628847049) bank152713 +152714 POINT(41.475131987299946 74.74424291619431) bank152714 +152715 POINT(39.96698782375704 74.33005772878658) bank152715 +152716 POINT(40.50636901895225 73.22264535038038) bank152716 +152717 POINT(41.07730261099294 74.07946423852141) bank152717 +152718 POINT(41.6693573455886 73.42698121874028) bank152718 +152719 POINT(41.113614914506634 74.40750042546698) bank152719 +152720 POINT(40.44417758170579 74.92496496652538) bank152720 +152721 POINT(41.275182726164665 74.13625729910108) bank152721 +152722 POINT(40.083151746305255 74.0149898216685) bank152722 +152723 POINT(40.27009081137444 73.53144712573325) bank152723 +152724 POINT(40.26222538952823 73.78824960541492) bank152724 +152725 POINT(39.78998031815475 74.20399695372183) bank152725 +152726 POINT(41.54964847171504 74.81785865726896) bank152726 +152727 POINT(40.229304726675345 73.72491466806389) bank152727 +152728 POINT(39.864323899974124 73.81439968096586) bank152728 +152729 POINT(40.51770303039829 73.86579731971483) bank152729 +152730 POINT(40.841849577776024 74.1428728676219) bank152730 +152731 POINT(41.68136841898701 73.52471081879932) bank152731 +152732 POINT(39.74255564611405 74.21114139431111) bank152732 +152733 POINT(40.05927439030865 74.49233146771543) bank152733 +152734 POINT(40.580539170785194 73.10449902290733) bank152734 +152735 POINT(41.099849029076815 73.08243887876962) bank152735 +152736 POINT(40.12980818973085 74.76336783731337) bank152736 +152737 POINT(40.63169182913749 74.50489471942583) bank152737 +152738 POINT(40.99737937628185 74.18019858697888) bank152738 +152739 POINT(40.21993073000977 74.21796303105101) bank152739 +152740 POINT(41.70504460569337 73.43505726264918) bank152740 +152741 POINT(41.2963075637372 73.633039858263) bank152741 +152742 POINT(41.52247419519472 73.91593520496556) bank152742 +152743 POINT(39.93145769164548 74.21956796208356) bank152743 +152744 POINT(41.05119347472975 74.85447540495247) bank152744 +152745 POINT(40.53773511473374 73.71966282858337) bank152745 +152746 POINT(39.8258850567099 73.91276819116098) bank152746 +152747 POINT(39.817210508044056 74.89916152770647) bank152747 +152748 POINT(41.53914932560178 74.9497976245637) bank152748 +152749 POINT(40.26802285845302 74.54584369852721) bank152749 +152750 POINT(40.81910976149219 73.02903121747231) bank152750 +152751 POINT(40.40928475902819 73.84862452741989) bank152751 +152752 POINT(40.883979628087864 74.63822905902867) bank152752 +152753 POINT(40.85283932330858 73.50836352843082) bank152753 +152754 POINT(41.363287772051535 73.02564712205795) bank152754 +152755 POINT(41.5298550999521 73.86560672317482) bank152755 +152756 POINT(40.268237521225096 73.27252239241169) bank152756 +152757 POINT(40.091701952727725 74.1699150428823) bank152757 +152758 POINT(39.945539979243236 74.64070211860857) bank152758 +152759 POINT(40.96616788892454 74.80063783541668) bank152759 +152760 POINT(40.98407963209058 73.79906967865723) bank152760 +152761 POINT(40.05594911591833 73.58585114617581) bank152761 +152762 POINT(40.41166605826233 74.09716198459203) bank152762 +152763 POINT(41.68810883167963 74.0089298375936) bank152763 +152764 POINT(40.86828534540182 74.64697007075142) bank152764 +152765 POINT(41.67284927128357 73.54204344799015) bank152765 +152766 POINT(40.03370370742482 73.53048785061604) bank152766 +152767 POINT(41.45424161371644 74.46402167491368) bank152767 +152768 POINT(39.77736648705629 73.66317352407818) bank152768 +152769 POINT(40.16008208670682 74.06439647644724) bank152769 +152770 POINT(40.87183737339398 74.95497091687118) bank152770 +152771 POINT(41.386157366229156 74.16641384807545) bank152771 +152772 POINT(40.168008443118 74.21693214786839) bank152772 +152773 POINT(40.35674770037748 74.52863913468744) bank152773 +152774 POINT(40.14775681778602 73.56258583413012) bank152774 +152775 POINT(41.04635079940433 73.89948551024487) bank152775 +152776 POINT(41.676933054355686 74.34113974371685) bank152776 +152777 POINT(39.867123373269614 74.05936868288057) bank152777 +152778 POINT(40.80786910516921 73.08652805319564) bank152778 +152779 POINT(41.362894881332664 73.51873495195309) bank152779 +152780 POINT(41.36616717757614 74.05252193210437) bank152780 +152781 POINT(39.91180166095927 73.07290907366752) bank152781 +152782 POINT(41.34437038612338 74.6805827562152) bank152782 +152783 POINT(39.81899787232208 74.42868475412821) bank152783 +152784 POINT(41.0619746482636 74.47681003729564) bank152784 +152785 POINT(40.75995779280983 73.35372826288655) bank152785 +152786 POINT(40.410591659709745 74.99699256323963) bank152786 +152787 POINT(41.482008004410176 73.45702622317646) bank152787 +152788 POINT(41.05293058644611 74.6712939347373) bank152788 +152789 POINT(40.3388097540508 74.74054498510422) bank152789 +152790 POINT(40.819366800048186 73.81283028971521) bank152790 +152791 POINT(40.81407671357432 74.52873021670281) bank152791 +152792 POINT(39.969335185467266 74.00003531550367) bank152792 +152793 POINT(40.27065000122044 73.43215511142265) bank152793 +152794 POINT(40.48203848050239 74.31409447831932) bank152794 +152795 POINT(40.149679391136075 73.70562844114647) bank152795 +152796 POINT(41.640891768810405 73.85513980420211) bank152796 +152797 POINT(41.69566401956435 74.8509955853154) bank152797 +152798 POINT(39.93636360725574 74.84931417781166) bank152798 +152799 POINT(41.10595775613447 74.47070992229789) bank152799 +152800 POINT(39.75374350539801 73.94190423521795) bank152800 +152801 POINT(41.484867811673524 74.76996106898065) bank152801 +152802 POINT(40.538686624438654 73.6358057738632) bank152802 +152803 POINT(39.954545630613794 74.24799501373444) bank152803 +152804 POINT(40.17062827099013 73.14737423872994) bank152804 +152805 POINT(41.3191442266152 73.56764113706355) bank152805 +152806 POINT(41.149994144625175 73.05555612937307) bank152806 +152807 POINT(40.703886996196495 73.50918028442229) bank152807 +152808 POINT(40.29902488016582 74.98535851658063) bank152808 +152809 POINT(40.21121453589028 74.80865096110466) bank152809 +152810 POINT(40.36904922270367 73.04001649690113) bank152810 +152811 POINT(39.98111879033604 74.68667908607372) bank152811 +152812 POINT(40.6977306307956 74.79833917153245) bank152812 +152813 POINT(40.72185043044709 73.30124096418197) bank152813 +152814 POINT(41.4139609513704 74.49821467547063) bank152814 +152815 POINT(39.83866351934193 73.9627331416902) bank152815 +152816 POINT(40.340392495739934 73.81107062137822) bank152816 +152817 POINT(40.42563308437067 73.61589459182264) bank152817 +152818 POINT(39.85458353496993 74.66277021068312) bank152818 +152819 POINT(40.47810132001393 73.04063065976197) bank152819 +152820 POINT(41.12021044402567 73.38013699381574) bank152820 +152821 POINT(40.846547934649834 74.6306328178795) bank152821 +152822 POINT(40.556515750681015 74.70274085143305) bank152822 +152823 POINT(40.55561034148345 73.99652202720002) bank152823 +152824 POINT(41.60161907564946 73.14256073321886) bank152824 +152825 POINT(40.20968210487352 74.75626548484655) bank152825 +152826 POINT(41.414857357383354 74.35457411657723) bank152826 +152827 POINT(40.14564140613596 73.72480799527939) bank152827 +152828 POINT(40.76180027228592 73.50135284959934) bank152828 +152829 POINT(41.26670706826 74.17184659713926) bank152829 +152830 POINT(40.084177129250726 73.99385816828462) bank152830 +152831 POINT(41.017711544943324 73.0578781667963) bank152831 +152832 POINT(40.553898968815865 74.65657532598777) bank152832 +152833 POINT(41.150324240569915 74.16333357779605) bank152833 +152834 POINT(40.91533614397432 74.19417217298403) bank152834 +152835 POINT(40.25179197040053 74.83364389410315) bank152835 +152836 POINT(41.07690566743631 73.56257677645323) bank152836 +152837 POINT(40.18992855149193 74.33312724044633) bank152837 +152838 POINT(40.12323049148751 74.59829423354778) bank152838 +152839 POINT(41.3548873725282 73.93052150520587) bank152839 +152840 POINT(41.04387710651321 73.28499695722104) bank152840 +152841 POINT(41.26561287236065 73.43242643141565) bank152841 +152842 POINT(40.675871861414095 73.22849149433691) bank152842 +152843 POINT(39.867962558130586 73.40934626582376) bank152843 +152844 POINT(40.19399514844744 73.48714361319736) bank152844 +152845 POINT(41.29918981494554 73.2194222681653) bank152845 +152846 POINT(40.888065094684954 73.32342159122877) bank152846 +152847 POINT(40.22499006692073 73.88973950883236) bank152847 +152848 POINT(39.93914829663818 74.3351794952695) bank152848 +152849 POINT(41.090742406262954 74.43905455131159) bank152849 +152850 POINT(41.711663321547306 73.68167174154486) bank152850 +152851 POINT(40.41010569388395 73.52607966227701) bank152851 +152852 POINT(40.103767988151915 74.72397293715662) bank152852 +152853 POINT(40.40713957582139 73.88533369429283) bank152853 +152854 POINT(40.68403220608721 73.00839349132482) bank152854 +152855 POINT(40.760164776143505 74.36190994669847) bank152855 +152856 POINT(41.12459798396934 74.89072551965354) bank152856 +152857 POINT(40.740720453964585 74.47112267790416) bank152857 +152858 POINT(40.20209407438383 73.03626959071332) bank152858 +152859 POINT(40.06055407605998 73.40114097817235) bank152859 +152860 POINT(40.31753298873061 73.41715581025181) bank152860 +152861 POINT(41.511783061118166 74.18342591899967) bank152861 +152862 POINT(40.32045681127166 74.6841338445733) bank152862 +152863 POINT(41.66106945468334 74.87925542431209) bank152863 +152864 POINT(41.58381898691566 73.14387436445708) bank152864 +152865 POINT(40.38024752013589 74.69462234475256) bank152865 +152866 POINT(41.62275790804029 73.01102294740278) bank152866 +152867 POINT(40.929585707508 73.9105655440869) bank152867 +152868 POINT(40.86851281332219 74.36276234680722) bank152868 +152869 POINT(41.26750048456693 73.23212628393864) bank152869 +152870 POINT(40.810354595833026 73.5311710004652) bank152870 +152871 POINT(40.6226127877092 73.13770013685595) bank152871 +152872 POINT(40.2327818432273 73.39134918343545) bank152872 +152873 POINT(41.332680282181116 73.79157273539123) bank152873 +152874 POINT(41.57559533928745 74.52385059735145) bank152874 +152875 POINT(40.85850831035404 73.97302239572947) bank152875 +152876 POINT(41.38936147937781 74.9090718838909) bank152876 +152877 POINT(41.50145489473579 74.66280671984326) bank152877 +152878 POINT(41.51173316270461 73.2370527482273) bank152878 +152879 POINT(40.21179254926883 74.74424072725968) bank152879 +152880 POINT(41.35546748995257 74.26910059030605) bank152880 +152881 POINT(40.33106380390931 73.38377996750866) bank152881 +152882 POINT(40.11768866087967 74.10676478938169) bank152882 +152883 POINT(40.279999901823544 73.73668550115768) bank152883 +152884 POINT(39.90023170717532 73.15857508746987) bank152884 +152885 POINT(40.71134386209409 73.46746907635622) bank152885 +152886 POINT(41.34035298948039 74.06862873393611) bank152886 +152887 POINT(39.741750792841266 74.53133656545255) bank152887 +152888 POINT(40.28123430666024 74.17901088762054) bank152888 +152889 POINT(40.757861768778554 73.11730861195034) bank152889 +152890 POINT(41.37782675086405 73.58160280333023) bank152890 +152891 POINT(41.285616691592416 73.77246981905753) bank152891 +152892 POINT(41.39841551178046 73.72573794878072) bank152892 +152893 POINT(41.060411603668776 74.50781419704306) bank152893 +152894 POINT(40.087648306900896 73.15541267028897) bank152894 +152895 POINT(40.86344598320048 74.05694309034621) bank152895 +152896 POINT(41.65986491744887 73.75494519959472) bank152896 +152897 POINT(41.56557283969161 73.51449838851562) bank152897 +152898 POINT(40.74052147420855 74.7625204341113) bank152898 +152899 POINT(41.3023792411634 74.88544203797754) bank152899 +152900 POINT(39.94024610840989 73.88438202400758) bank152900 +152901 POINT(39.77415181341556 74.44832769419084) bank152901 +152902 POINT(39.95547114349765 74.18878283478432) bank152902 +152903 POINT(41.02127482239588 73.8315276512328) bank152903 +152904 POINT(40.28843487504133 73.50749467769629) bank152904 +152905 POINT(41.0838313731059 74.31029241746518) bank152905 +152906 POINT(41.50154505689516 74.72345051751594) bank152906 +152907 POINT(41.27654086129724 74.87465514209839) bank152907 +152908 POINT(40.168304281587574 73.28603847057026) bank152908 +152909 POINT(40.33314171624029 73.54098967155099) bank152909 +152910 POINT(40.86171133508605 74.40832952756853) bank152910 +152911 POINT(40.71848171620792 74.49889086704005) bank152911 +152912 POINT(40.75117771633888 74.25431445265005) bank152912 +152913 POINT(40.05046572424977 74.3052918401673) bank152913 +152914 POINT(40.31450709534957 74.53546598596425) bank152914 +152915 POINT(40.902245511041706 73.02154251066013) bank152915 +152916 POINT(40.53082151102714 73.13138728585044) bank152916 +152917 POINT(40.756857376246856 74.87392757130515) bank152917 +152918 POINT(41.03684227498105 74.44166015339587) bank152918 +152919 POINT(40.8665091194551 73.91805724542489) bank152919 +152920 POINT(41.59631608981623 73.247015428385) bank152920 +152921 POINT(41.191848670961456 74.82735968346529) bank152921 +152922 POINT(41.55058446390256 74.0894061103991) bank152922 +152923 POINT(40.9741140557845 73.6419334305906) bank152923 +152924 POINT(40.21590284595277 73.63329605767322) bank152924 +152925 POINT(40.74160215403985 74.90487797235369) bank152925 +152926 POINT(39.97106751005638 74.96027067196815) bank152926 +152927 POINT(40.62459481297316 74.0587987494374) bank152927 +152928 POINT(41.1884180292125 73.0677891576759) bank152928 +152929 POINT(40.93266624836044 73.65777295943481) bank152929 +152930 POINT(40.58377366113423 73.72592626654308) bank152930 +152931 POINT(40.59917374749657 73.37424973925056) bank152931 +152932 POINT(40.30530314849898 73.48584744898832) bank152932 +152933 POINT(41.242829446863354 73.48743967718146) bank152933 +152934 POINT(40.308329789640894 73.0509007214327) bank152934 +152935 POINT(40.24925655954786 74.75326507627116) bank152935 +152936 POINT(40.57665481818369 73.23636997804584) bank152936 +152937 POINT(40.83562402430856 74.54033821553308) bank152937 +152938 POINT(39.949799071464454 74.33719374427386) bank152938 +152939 POINT(40.947115622818494 74.0245096037117) bank152939 +152940 POINT(40.23953685371195 74.04706179243836) bank152940 +152941 POINT(39.85188106518558 73.13745003721479) bank152941 +152942 POINT(40.20729981044715 73.95640585505284) bank152942 +152943 POINT(41.32572162437879 73.06035159605055) bank152943 +152944 POINT(40.49853801164302 73.35400521533812) bank152944 +152945 POINT(41.101669877689105 73.53421703112492) bank152945 +152946 POINT(41.108419456174644 74.37035338400443) bank152946 +152947 POINT(40.212689929851386 73.0793117110047) bank152947 +152948 POINT(40.919779808114605 74.57471372912443) bank152948 +152949 POINT(39.844380611270445 74.79583189921601) bank152949 +152950 POINT(39.915182828670204 74.51629517223313) bank152950 +152951 POINT(40.061626951929235 74.82936679400215) bank152951 +152952 POINT(41.039533820815286 74.39017833621389) bank152952 +152953 POINT(39.82537381715601 73.02652905939325) bank152953 +152954 POINT(40.80127681764626 73.23048762532014) bank152954 +152955 POINT(40.615774701442824 73.85395225710229) bank152955 +152956 POINT(40.124461711964905 74.0779496550172) bank152956 +152957 POINT(40.98538965059487 74.85850969329975) bank152957 +152958 POINT(41.33889248314043 74.93816732353287) bank152958 +152959 POINT(40.84895587855945 73.34120738151522) bank152959 +152960 POINT(39.921725283193645 74.66916065189395) bank152960 +152961 POINT(40.2748238735642 73.57085040494735) bank152961 +152962 POINT(41.052144485831214 73.4993341939179) bank152962 +152963 POINT(40.65280657061305 74.97804913924588) bank152963 +152964 POINT(40.86742357980284 74.38221974887253) bank152964 +152965 POINT(40.72056114757975 73.74226656238386) bank152965 +152966 POINT(40.11537933185335 73.11083564000376) bank152966 +152967 POINT(40.26942885982245 74.08551204993343) bank152967 +152968 POINT(40.95015700530298 74.0269567834243) bank152968 +152969 POINT(41.00836854453777 74.54006502690689) bank152969 +152970 POINT(41.20532642051931 74.52620972416558) bank152970 +152971 POINT(40.25212528729247 73.99448871780186) bank152971 +152972 POINT(40.88701003722715 74.48398226312915) bank152972 +152973 POINT(39.71286717702548 73.50957211437411) bank152973 +152974 POINT(40.875243945762115 74.22422569493003) bank152974 +152975 POINT(40.473123301881536 74.1309402023266) bank152975 +152976 POINT(40.254522054490536 73.83158040969079) bank152976 +152977 POINT(41.21186551472819 73.30897891742225) bank152977 +152978 POINT(40.90583008291006 73.25827318573305) bank152978 +152979 POINT(39.926706849379 74.99821104734262) bank152979 +152980 POINT(41.69014036872699 73.75908399389233) bank152980 +152981 POINT(41.33859760292737 73.31081244250832) bank152981 +152982 POINT(40.20362935559261 74.49297373847217) bank152982 +152983 POINT(40.91633473909839 73.76139186458644) bank152983 +152984 POINT(41.078003925934645 73.6258806167313) bank152984 +152985 POINT(41.63348478383179 74.42646675785126) bank152985 +152986 POINT(40.66652257524974 73.59413308980446) bank152986 +152987 POINT(40.636953153115975 74.78599698524596) bank152987 +152988 POINT(39.77364496998763 74.65946928544342) bank152988 +152989 POINT(40.791000022572526 74.8148845032806) bank152989 +152990 POINT(40.90193533056679 74.59005256784886) bank152990 +152991 POINT(41.50673775272896 73.80363190973453) bank152991 +152992 POINT(41.27883790323268 73.24232332951459) bank152992 +152993 POINT(41.561962180623205 73.31272020495022) bank152993 +152994 POINT(41.6940864893006 73.62085171900485) bank152994 +152995 POINT(40.21441967876745 73.01882452842005) bank152995 +152996 POINT(41.158590954130766 74.42226199728128) bank152996 +152997 POINT(41.341356704162585 74.79540906662244) bank152997 +152998 POINT(41.28861376538379 74.22091726591091) bank152998 +152999 POINT(40.082290420534974 73.6635316895906) bank152999 +153000 POINT(41.31516954064471 73.9417863844076) bank153000 +153001 POINT(40.11538411350003 74.12497077143765) bank153001 +153002 POINT(40.17819619056022 73.98990791618105) bank153002 +153003 POINT(40.766538947048446 73.63660582054985) bank153003 +153004 POINT(40.05773742598571 73.23217054103068) bank153004 +153005 POINT(41.54300546668015 74.74472995049456) bank153005 +153006 POINT(41.59990751903207 74.1184194857573) bank153006 +153007 POINT(40.77617830580682 74.90612767479047) bank153007 +153008 POINT(39.94567941830884 74.61380534870165) bank153008 +153009 POINT(41.21201767078178 73.17724803906204) bank153009 +153010 POINT(40.800254214102374 73.28206745652349) bank153010 +153011 POINT(40.00921699308986 73.60031637716315) bank153011 +153012 POINT(39.78487311589868 73.52397071911376) bank153012 +153013 POINT(40.37938161894393 74.76174062382026) bank153013 +153014 POINT(40.20604601931006 73.99495136284838) bank153014 +153015 POINT(41.37552173047048 73.95381403891793) bank153015 +153016 POINT(39.805172947953245 73.80836077598863) bank153016 +153017 POINT(40.50004147704838 73.03091602978355) bank153017 +153018 POINT(41.553472976089104 74.471087115186) bank153018 +153019 POINT(41.145935946346285 73.48004522885864) bank153019 +153020 POINT(39.942956767078215 73.97006036141975) bank153020 +153021 POINT(41.193945067601454 74.96789107773029) bank153021 +153022 POINT(40.7815565680789 74.75799771063389) bank153022 +153023 POINT(40.59432836876559 74.20387784939443) bank153023 +153024 POINT(40.30529487065189 73.21459941405821) bank153024 +153025 POINT(41.120043897281164 74.16602836436414) bank153025 +153026 POINT(39.82635904634689 74.8595406532846) bank153026 +153027 POINT(41.47938222818224 74.40936478713843) bank153027 +153028 POINT(41.1497903671882 74.97816368522888) bank153028 +153029 POINT(40.24191715345961 73.53215273173156) bank153029 +153030 POINT(39.91919265309315 74.36177536756821) bank153030 +153031 POINT(39.92323053217473 74.41652582215971) bank153031 +153032 POINT(40.854267116300385 75.0017416932241) bank153032 +153033 POINT(40.79973627863246 73.27464848083369) bank153033 +153034 POINT(40.60289651043444 73.70986798420459) bank153034 +153035 POINT(41.074150708652176 73.7022151293111) bank153035 +153036 POINT(40.4400189178163 73.91805651140278) bank153036 +153037 POINT(41.203777859855926 73.06845660646877) bank153037 +153038 POINT(40.161250003113445 73.88320583271266) bank153038 +153039 POINT(40.45881162984869 74.43299658803608) bank153039 +153040 POINT(40.67845343603142 74.83963389146344) bank153040 +153041 POINT(39.94544416607843 74.28315704276355) bank153041 +153042 POINT(40.24998953879172 73.02690326252778) bank153042 +153043 POINT(39.72480314438365 73.22237150186882) bank153043 +153044 POINT(41.36505683071842 73.2019679718551) bank153044 +153045 POINT(39.83572551612248 73.14920957181124) bank153045 +153046 POINT(39.79623067659462 73.53380999998421) bank153046 +153047 POINT(40.805476452952185 74.94125139442913) bank153047 +153048 POINT(40.67447054843018 74.83413977364148) bank153048 +153049 POINT(39.94502042984514 73.69322731834255) bank153049 +153050 POINT(40.349977722897236 74.97429241723863) bank153050 +153051 POINT(40.56108574851967 73.55655297640375) bank153051 +153052 POINT(41.386117824933166 73.66245325569137) bank153052 +153053 POINT(40.11578504073225 74.44680744367504) bank153053 +153054 POINT(41.332947350134376 74.57203180833199) bank153054 +153055 POINT(39.8106699812116 74.15658078119174) bank153055 +153056 POINT(40.85846543918878 74.32183355664726) bank153056 +153057 POINT(41.580400592302624 74.55080038126579) bank153057 +153058 POINT(39.74898751385447 74.72362518219938) bank153058 +153059 POINT(40.67829399761049 74.68275096720168) bank153059 +153060 POINT(40.03973038367343 74.15002741083941) bank153060 +153061 POINT(40.07713365759702 74.95326351990634) bank153061 +153062 POINT(41.6411974016909 74.99148977745843) bank153062 +153063 POINT(41.47554680909725 74.73579750212292) bank153063 +153064 POINT(40.25564949369721 74.15532617450778) bank153064 +153065 POINT(40.91520128731287 73.39499442166657) bank153065 +153066 POINT(40.48451435379677 74.42530270971264) bank153066 +153067 POINT(40.813848499605434 74.67801516901173) bank153067 +153068 POINT(40.15031882987971 74.52403231386032) bank153068 +153069 POINT(40.06446045203635 74.99152700950214) bank153069 +153070 POINT(40.44898762065398 74.01067581810122) bank153070 +153071 POINT(40.0351215347795 74.47776679636031) bank153071 +153072 POINT(39.9461492844097 73.25695342991484) bank153072 +153073 POINT(40.50829987687515 73.4034946085123) bank153073 +153074 POINT(40.344059341308814 74.90258326739891) bank153074 +153075 POINT(40.59743599830823 74.57635096657495) bank153075 +153076 POINT(40.12775064895966 74.17559298924083) bank153076 +153077 POINT(41.629542076468894 74.12221824223113) bank153077 +153078 POINT(41.5071189367788 74.40005650463893) bank153078 +153079 POINT(41.567631128554346 74.9684159917839) bank153079 +153080 POINT(40.75659942249845 74.67501705100594) bank153080 +153081 POINT(40.44426692954047 74.71130565336135) bank153081 +153082 POINT(41.115947516700295 74.65691313368653) bank153082 +153083 POINT(40.64438843768618 73.50925645279123) bank153083 +153084 POINT(41.55994055010705 73.3193155134258) bank153084 +153085 POINT(39.76918698055054 73.69762172774979) bank153085 +153086 POINT(40.28356725842295 73.33218760769293) bank153086 +153087 POINT(40.55823547007635 73.04756191746716) bank153087 +153088 POINT(40.639827749137154 73.26068385814773) bank153088 +153089 POINT(41.622168377946956 73.44590917307619) bank153089 +153090 POINT(40.709311000605155 73.89339926818698) bank153090 +153091 POINT(40.5874297229001 73.61808286273582) bank153091 +153092 POINT(41.129861785935645 73.59516110124405) bank153092 +153093 POINT(40.898311769536086 73.65262310864806) bank153093 +153094 POINT(41.31709617144607 73.7900325678835) bank153094 +153095 POINT(41.109836753919424 74.69307730880226) bank153095 +153096 POINT(40.293020145231075 74.94389621652151) bank153096 +153097 POINT(40.85192322406437 74.26519431282078) bank153097 +153098 POINT(40.6554276764747 73.8758068801754) bank153098 +153099 POINT(41.64155773794592 73.85789833739979) bank153099 +153100 POINT(40.08116828095751 73.44934804468639) bank153100 +153101 POINT(39.75576527538928 73.9608150623746) bank153101 +153102 POINT(41.379559968013844 73.2134034295391) bank153102 +153103 POINT(40.7191067663994 74.01746858292096) bank153103 +153104 POINT(40.311035109609406 74.69554128888714) bank153104 +153105 POINT(39.941233168657526 74.3624678545173) bank153105 +153106 POINT(41.10122238915442 73.32448457995025) bank153106 +153107 POINT(41.05705805967556 73.59203039835474) bank153107 +153108 POINT(41.126473290534086 73.46684970497617) bank153108 +153109 POINT(41.56360620700771 73.31058566815722) bank153109 +153110 POINT(40.5294512731443 73.8399330889313) bank153110 +153111 POINT(40.55961121562639 74.25932612628384) bank153111 +153112 POINT(40.11921749946957 74.49990691200813) bank153112 +153113 POINT(41.3551878664967 73.01826575908198) bank153113 +153114 POINT(41.32095299687306 74.60715667314675) bank153114 +153115 POINT(41.548266852312416 74.69227607577412) bank153115 +153116 POINT(40.03887391027168 73.4539648206548) bank153116 +153117 POINT(41.4227979364104 73.90880280584925) bank153117 +153118 POINT(39.85384541348803 73.03340338848241) bank153118 +153119 POINT(41.18644608057352 73.80610281941009) bank153119 +153120 POINT(41.14659928821918 73.9585516937894) bank153120 +153121 POINT(39.96235104807624 74.54499818149068) bank153121 +153122 POINT(39.80975501606602 74.75836081800864) bank153122 +153123 POINT(40.521635690815046 73.19378328969432) bank153123 +153124 POINT(41.04344533046008 74.81164663597157) bank153124 +153125 POINT(39.75175863382088 73.94718640006792) bank153125 +153126 POINT(39.99202004979953 73.4275418606788) bank153126 +153127 POINT(40.85972252023401 73.99371783010044) bank153127 +153128 POINT(41.36603567736312 73.95659900947378) bank153128 +153129 POINT(41.66555197269192 74.65575486181747) bank153129 +153130 POINT(41.4820148750343 74.8943249661578) bank153130 +153131 POINT(39.967218259548794 74.80163653322921) bank153131 +153132 POINT(39.80935400023481 74.10689336056052) bank153132 +153133 POINT(41.248569276900774 73.5718712341037) bank153133 +153134 POINT(41.202357914923624 73.62627337629469) bank153134 +153135 POINT(40.332187847871424 74.92400652892256) bank153135 +153136 POINT(40.71807465586947 73.8950719444296) bank153136 +153137 POINT(40.71915361303375 75.00487545230172) bank153137 +153138 POINT(40.32434112018605 73.21895586384046) bank153138 +153139 POINT(41.16338218895581 73.66579410621921) bank153139 +153140 POINT(39.83920568673904 73.72786876386824) bank153140 +153141 POINT(39.966752630559526 73.71348659163664) bank153141 +153142 POINT(41.40894733129893 73.05430306682882) bank153142 +153143 POINT(40.237184261855155 74.04226674436686) bank153143 +153144 POINT(40.35620045029526 73.88525730854255) bank153144 +153145 POINT(40.14388373893746 73.98753739491364) bank153145 +153146 POINT(41.399329816639664 73.04991318826066) bank153146 +153147 POINT(41.264143358241135 73.82751421220557) bank153147 +153148 POINT(39.752950871369244 74.98305137630933) bank153148 +153149 POINT(39.89299260870389 74.61259037052477) bank153149 +153150 POINT(41.29396831087287 73.03256168546716) bank153150 +153151 POINT(40.024338220906934 74.5599120460206) bank153151 +153152 POINT(40.694729293729154 73.72067277774065) bank153152 +153153 POINT(41.338578583480576 74.62454682716657) bank153153 +153154 POINT(40.69793360252748 74.84986310612717) bank153154 +153155 POINT(40.74436516026118 74.35099169272607) bank153155 +153156 POINT(40.166054335232694 74.33966699533521) bank153156 +153157 POINT(41.402053122314605 74.31439398116589) bank153157 +153158 POINT(40.86301647208893 74.40780711420985) bank153158 +153159 POINT(41.14543352832549 73.77416524508571) bank153159 +153160 POINT(41.28548781142219 73.31711632257262) bank153160 +153161 POINT(40.940899134178196 73.01842666243925) bank153161 +153162 POINT(41.65858185223412 74.97745374382288) bank153162 +153163 POINT(40.891261601767724 74.710152479581) bank153163 +153164 POINT(40.62221806554866 74.1449105661901) bank153164 +153165 POINT(41.11155471214464 73.9063367806569) bank153165 +153166 POINT(40.18328774738843 74.2596902450059) bank153166 +153167 POINT(40.91354314167229 73.68871504261772) bank153167 +153168 POINT(39.75438725543629 74.79977325405542) bank153168 +153169 POINT(39.739664171174965 74.62443101383431) bank153169 +153170 POINT(40.28913615681422 74.32513179942251) bank153170 +153171 POINT(41.55217679615082 74.31084378776285) bank153171 +153172 POINT(41.09041981382268 74.84516116541114) bank153172 +153173 POINT(41.36308046303629 74.89306340562398) bank153173 +153174 POINT(41.58881131291597 74.25839393492586) bank153174 +153175 POINT(39.73095533718311 74.28273632637405) bank153175 +153176 POINT(40.51514717642585 73.5461146795519) bank153176 +153177 POINT(41.64103565042728 74.90183066086347) bank153177 +153178 POINT(40.87544126503953 73.12366817778819) bank153178 +153179 POINT(41.09128263312504 74.33482445702671) bank153179 +153180 POINT(40.30201828958 73.1088526434507) bank153180 +153181 POINT(40.94582903934982 73.7525042319687) bank153181 +153182 POINT(41.30044653893228 74.65850304117951) bank153182 +153183 POINT(39.73525883985784 74.0141149059375) bank153183 +153184 POINT(41.1409462404453 74.99533308748883) bank153184 +153185 POINT(41.03673464561414 74.65242607149972) bank153185 +153186 POINT(41.27214868237329 73.91614889122214) bank153186 +153187 POINT(39.92125074984931 74.38104152151953) bank153187 +153188 POINT(40.42807010488879 74.71078976307473) bank153188 +153189 POINT(41.34534778496611 73.64796071432927) bank153189 +153190 POINT(41.19236475152325 74.76327625366346) bank153190 +153191 POINT(40.919290210209354 73.20127983315828) bank153191 +153192 POINT(41.35446781066059 73.12232420098194) bank153192 +153193 POINT(41.19133551614428 73.78823679239653) bank153193 +153194 POINT(39.83991360676924 74.5097639031238) bank153194 +153195 POINT(41.385651003223224 74.58711431283196) bank153195 +153196 POINT(39.82163345819247 73.79638495782761) bank153196 +153197 POINT(41.055718510236055 73.05543900097571) bank153197 +153198 POINT(40.89459419984809 74.49063862305982) bank153198 +153199 POINT(40.58010227781991 74.34207842035379) bank153199 +153200 POINT(40.18886194286668 74.11523661898461) bank153200 +153201 POINT(41.23893348619302 74.65491325417382) bank153201 +153202 POINT(40.915361489747035 74.7428515474969) bank153202 +153203 POINT(41.386121549395966 74.08033672544148) bank153203 +153204 POINT(40.322489744435416 73.79990624568025) bank153204 +153205 POINT(41.37993898158737 73.7692279366105) bank153205 +153206 POINT(41.58629330974779 74.36065597343507) bank153206 +153207 POINT(41.01866102789966 73.87913498111507) bank153207 +153208 POINT(40.54018989392626 74.0560611434522) bank153208 +153209 POINT(40.27112158554116 73.99368560458666) bank153209 +153210 POINT(41.25908085563816 74.5857437130527) bank153210 +153211 POINT(40.894989901026236 73.70467448982765) bank153211 +153212 POINT(40.053016552090966 74.89384219970808) bank153212 +153213 POINT(40.04088103692123 73.83842076720585) bank153213 +153214 POINT(40.323994710473265 74.25628445264029) bank153214 +153215 POINT(40.524988996343545 73.02799741540294) bank153215 +153216 POINT(41.27030758687552 74.30266312115624) bank153216 +153217 POINT(41.107493552894645 74.50325280741028) bank153217 +153218 POINT(39.7835962823864 74.38701465274394) bank153218 +153219 POINT(40.524760693389524 73.02966657136484) bank153219 +153220 POINT(40.702035664629385 73.51315190728367) bank153220 +153221 POINT(40.27030668263724 73.99658346275567) bank153221 +153222 POINT(41.527398329166644 74.22496792219309) bank153222 +153223 POINT(40.5678996377668 74.07372794587575) bank153223 +153224 POINT(40.83367298002927 73.15356674668269) bank153224 +153225 POINT(41.00343058507162 74.42107374022738) bank153225 +153226 POINT(39.94418770481699 74.22985088429135) bank153226 +153227 POINT(39.956090037683765 73.93271779618681) bank153227 +153228 POINT(40.82402457327446 74.24592803974463) bank153228 +153229 POINT(40.648805689548425 74.61734001918855) bank153229 +153230 POINT(39.9772980437607 74.5950453195164) bank153230 +153231 POINT(40.957282446024685 73.57280348886975) bank153231 +153232 POINT(40.295701382726804 73.88901983435538) bank153232 +153233 POINT(40.65542230078977 74.78114731373485) bank153233 +153234 POINT(41.32048955872074 74.78025466917293) bank153234 +153235 POINT(41.65429476609568 73.0716493138024) bank153235 +153236 POINT(40.577894072469725 73.60053041236341) bank153236 +153237 POINT(40.20366010986029 74.00017416091842) bank153237 +153238 POINT(39.78388730686203 74.50572121236) bank153238 +153239 POINT(41.51655544808458 73.66250058848233) bank153239 +153240 POINT(40.57247957967157 73.40277526502689) bank153240 +153241 POINT(40.55955784557962 73.67835570071647) bank153241 +153242 POINT(40.68007825105221 74.95531719505404) bank153242 +153243 POINT(41.41516240694628 73.13153020647637) bank153243 +153244 POINT(41.56523255802872 73.83337251417954) bank153244 +153245 POINT(40.57559805006335 73.1231416135905) bank153245 +153246 POINT(40.907198360622914 73.52934630928718) bank153246 +153247 POINT(40.046086412630785 74.87597854868999) bank153247 +153248 POINT(41.61610062855975 73.30184329118771) bank153248 +153249 POINT(39.721688592646075 73.00736709069163) bank153249 +153250 POINT(41.13153377073949 73.84877604977066) bank153250 +153251 POINT(40.37416624688676 74.19240301861929) bank153251 +153252 POINT(39.76429320998836 73.93544572822056) bank153252 +153253 POINT(40.74071913276963 73.638748006562) bank153253 +153254 POINT(39.98643037421345 73.12982825699916) bank153254 +153255 POINT(40.80869812517919 74.02833401782118) bank153255 +153256 POINT(39.9054830725885 73.25834286525597) bank153256 +153257 POINT(39.73753867033009 74.00171259725717) bank153257 +153258 POINT(41.09057378069055 74.35588990159987) bank153258 +153259 POINT(41.27412455100864 73.68507851864356) bank153259 +153260 POINT(39.891186668361826 73.69153652319324) bank153260 +153261 POINT(41.6619750005775 74.14020708488104) bank153261 +153262 POINT(39.769790771367596 74.92733266253632) bank153262 +153263 POINT(40.556457888200946 74.94537504798149) bank153263 +153264 POINT(41.37096863641527 73.05896473690817) bank153264 +153265 POINT(40.04965337787825 73.32456745629295) bank153265 +153266 POINT(41.559350562139535 73.59861537378768) bank153266 +153267 POINT(41.60082866509696 73.19548550634431) bank153267 +153268 POINT(41.0646761811338 73.03888040737502) bank153268 +153269 POINT(39.98145584094249 74.72312116914236) bank153269 +153270 POINT(40.25167163906839 73.96672351242346) bank153270 +153271 POINT(40.53201700885874 73.87614499943047) bank153271 +153272 POINT(40.44015783375513 74.98033078849758) bank153272 +153273 POINT(41.37472103734456 74.63623354953936) bank153273 +153274 POINT(39.879914409820934 74.75786258483159) bank153274 +153275 POINT(40.5701817921598 74.34433659425085) bank153275 +153276 POINT(39.9081760431702 73.28682786184638) bank153276 +153277 POINT(40.50185297374559 73.14365335431235) bank153277 +153278 POINT(41.0012402127855 73.065585169435) bank153278 +153279 POINT(41.613148343726536 73.40164759219451) bank153279 +153280 POINT(40.641119036028826 73.90887176249127) bank153280 +153281 POINT(41.18700753652624 74.22939222546708) bank153281 +153282 POINT(41.70245482393982 73.58339905460298) bank153282 +153283 POINT(39.89273772347948 74.13642130559136) bank153283 +153284 POINT(40.94951478368027 74.82300629246717) bank153284 +153285 POINT(40.3665144152366 74.74183130867067) bank153285 +153286 POINT(41.34221871171907 74.20977083163113) bank153286 +153287 POINT(40.05233002390008 73.03655698019493) bank153287 +153288 POINT(40.256406655135486 74.27086410516945) bank153288 +153289 POINT(40.67800021025588 73.17582560796649) bank153289 +153290 POINT(41.11120862777282 74.81045656881396) bank153290 +153291 POINT(41.43262231251288 73.43487157211247) bank153291 +153292 POINT(40.94768545891165 73.84142398384773) bank153292 +153293 POINT(40.42780522692792 73.57881725784053) bank153293 +153294 POINT(40.53210136522686 73.81312235430327) bank153294 +153295 POINT(40.71883328516302 73.97609036925917) bank153295 +153296 POINT(41.380119908571835 74.77199603671627) bank153296 +153297 POINT(41.17303115659123 73.51922064232492) bank153297 +153298 POINT(41.22424437665484 74.98588341716213) bank153298 +153299 POINT(40.24424852809171 74.01366100459454) bank153299 +153300 POINT(41.347195281829606 73.80055035863131) bank153300 +153301 POINT(40.667987584601974 74.63576763924604) bank153301 +153302 POINT(41.0176179794211 73.15100040120315) bank153302 +153303 POINT(40.54499723859099 74.59024723952339) bank153303 +153304 POINT(40.59996852187123 73.59469671260227) bank153304 +153305 POINT(39.88848549025263 73.03012830676369) bank153305 +153306 POINT(40.39596279827612 73.31355151300755) bank153306 +153307 POINT(40.06778182871706 74.08324882561466) bank153307 +153308 POINT(41.189973493398476 74.71894692899123) bank153308 +153309 POINT(39.86165269871477 73.09632648677535) bank153309 +153310 POINT(40.89454692011287 74.17416842347764) bank153310 +153311 POINT(41.126580826725935 73.37389569535239) bank153311 +153312 POINT(41.19410880650783 73.9193740130302) bank153312 +153313 POINT(41.27110536488862 73.66141187192615) bank153313 +153314 POINT(41.12356886337541 74.25679832103522) bank153314 +153315 POINT(41.12810301885159 74.04354845002254) bank153315 +153316 POINT(40.77904398410962 74.90819846250218) bank153316 +153317 POINT(41.67813142954105 73.52483444939215) bank153317 +153318 POINT(40.043977950042205 74.2284654704437) bank153318 +153319 POINT(41.0360348200496 73.92182141051295) bank153319 +153320 POINT(40.33520776057033 73.30631635076514) bank153320 +153321 POINT(41.554727257843815 73.21609906567055) bank153321 +153322 POINT(40.84265748783638 73.61012326775933) bank153322 +153323 POINT(40.95134810204598 73.00804473948853) bank153323 +153324 POINT(40.45782177619262 73.3675381877635) bank153324 +153325 POINT(41.127139571475965 74.88102022115356) bank153325 +153326 POINT(41.46730779200981 73.51480566010281) bank153326 +153327 POINT(41.30534571040821 73.8879916572947) bank153327 +153328 POINT(40.484972889212415 73.40289836977678) bank153328 +153329 POINT(41.168142452093456 73.700156245944) bank153329 +153330 POINT(41.384687110467446 73.99840147259061) bank153330 +153331 POINT(41.408028589414194 74.39829696226468) bank153331 +153332 POINT(41.29889305383904 74.74739092931225) bank153332 +153333 POINT(39.90139214299112 73.98222093642073) bank153333 +153334 POINT(40.32745539023724 74.85634713761947) bank153334 +153335 POINT(40.101403154350514 74.32934086591845) bank153335 +153336 POINT(40.362056890394555 73.92682458330397) bank153336 +153337 POINT(40.89638356641756 74.76972096722658) bank153337 +153338 POINT(41.691488976518606 73.12604960947294) bank153338 +153339 POINT(41.34350562191747 73.48061940942866) bank153339 +153340 POINT(40.05523507869896 74.16985207242256) bank153340 +153341 POINT(41.44766109102887 73.22845737649061) bank153341 +153342 POINT(41.07330092750086 74.37805257996601) bank153342 +153343 POINT(39.76366119738152 73.55275114654434) bank153343 +153344 POINT(39.842107583927685 74.00448093182008) bank153344 +153345 POINT(39.8896035939494 74.07276704281156) bank153345 +153346 POINT(40.75013538941895 73.20018094229593) bank153346 +153347 POINT(40.467500874580836 74.31536967099173) bank153347 +153348 POINT(40.37792761038164 74.19613599099378) bank153348 +153349 POINT(41.26905890471012 74.10702727751197) bank153349 +153350 POINT(41.21847321690623 74.89078582299035) bank153350 +153351 POINT(41.68904421617344 74.80336940098186) bank153351 +153352 POINT(40.864064402816794 74.76000223882323) bank153352 +153353 POINT(41.64112719468277 73.95135148748578) bank153353 +153354 POINT(40.836506012928915 73.5087486052169) bank153354 +153355 POINT(40.810005165752635 73.51275471950679) bank153355 +153356 POINT(40.66406715209346 73.18754898756319) bank153356 +153357 POINT(41.60318150753799 74.5798497601982) bank153357 +153358 POINT(41.119911063316565 73.83461597833283) bank153358 +153359 POINT(40.616722074259535 74.3042730216584) bank153359 +153360 POINT(41.31503229487868 74.62642532977712) bank153360 +153361 POINT(39.852647976599776 73.68548720728985) bank153361 +153362 POINT(40.588088023907844 74.01279262950156) bank153362 +153363 POINT(41.263902243170214 74.65495198799233) bank153363 +153364 POINT(41.052668674880934 74.04463875317529) bank153364 +153365 POINT(41.48276865925869 73.76238547565178) bank153365 +153366 POINT(41.31283031599936 73.18834083703265) bank153366 +153367 POINT(40.00852242307438 73.21555254313859) bank153367 +153368 POINT(41.04891710373282 74.84064970176773) bank153368 +153369 POINT(39.73492289431339 74.24353488072593) bank153369 +153370 POINT(40.17751519550851 74.36777140987547) bank153370 +153371 POINT(39.767791621511336 74.45618109324683) bank153371 +153372 POINT(41.44159707887485 74.0070965325968) bank153372 +153373 POINT(40.16887717148087 74.75711682062149) bank153373 +153374 POINT(39.82491740463624 74.20818518168532) bank153374 +153375 POINT(41.12710587636526 74.87599759745851) bank153375 +153376 POINT(40.88805376017116 74.74381141249793) bank153376 +153377 POINT(40.71621895862544 73.52390738730686) bank153377 +153378 POINT(41.447404646272744 74.9667354485484) bank153378 +153379 POINT(40.8600855625925 74.59419413313073) bank153379 +153380 POINT(41.369411329057975 74.07180174285092) bank153380 +153381 POINT(39.89096827112554 74.8765782993328) bank153381 +153382 POINT(39.83487617161562 74.21720531831072) bank153382 +153383 POINT(41.13021248502227 74.12229042550811) bank153383 +153384 POINT(39.81964758287277 74.23064886403179) bank153384 +153385 POINT(41.14842353979708 73.51406649747553) bank153385 +153386 POINT(41.52926292084274 74.6536053577249) bank153386 +153387 POINT(40.873978433745066 73.5951057336681) bank153387 +153388 POINT(40.52933261122163 73.22968647561444) bank153388 +153389 POINT(40.76903707740665 74.71746989531611) bank153389 +153390 POINT(41.12232345470298 74.69298277243841) bank153390 +153391 POINT(40.21910622378786 74.23321178894042) bank153391 +153392 POINT(41.12986916345841 74.00760036669415) bank153392 +153393 POINT(41.032943993408914 74.5209488477351) bank153393 +153394 POINT(40.62084645248563 74.5449500137264) bank153394 +153395 POINT(40.710560080504294 73.46984988597438) bank153395 +153396 POINT(39.922664629763965 73.81284900871515) bank153396 +153397 POINT(40.58703792729543 73.34209551965151) bank153397 +153398 POINT(39.896208616612604 73.1562974636047) bank153398 +153399 POINT(41.15063614746518 74.41457323233476) bank153399 +153400 POINT(39.82344960142223 74.2800589028789) bank153400 +153401 POINT(40.55966903840359 74.78462947745416) bank153401 +153402 POINT(40.48320004063752 73.33870392065644) bank153402 +153403 POINT(41.083338060219305 73.24797367971955) bank153403 +153404 POINT(40.68868034967736 73.04265610123569) bank153404 +153405 POINT(40.716213877452496 73.54721129483498) bank153405 +153406 POINT(40.622238620590394 73.3155880040797) bank153406 +153407 POINT(40.46908777025452 73.54815022039732) bank153407 +153408 POINT(40.847383212561255 73.68587658826863) bank153408 +153409 POINT(40.49924455391553 73.49366638838401) bank153409 +153410 POINT(40.10809199118537 73.0170498431514) bank153410 +153411 POINT(40.25693592521116 74.93612947293414) bank153411 +153412 POINT(41.269813675587436 73.52794154571612) bank153412 +153413 POINT(40.42948902648629 74.04203471178951) bank153413 +153414 POINT(41.47707961969338 74.49747844482982) bank153414 +153415 POINT(40.463325939600026 73.96668691017128) bank153415 +153416 POINT(40.69969238195307 73.25987557265863) bank153416 +153417 POINT(40.92665579382742 73.41860883606202) bank153417 +153418 POINT(40.27323955350179 73.77376049255442) bank153418 +153419 POINT(41.32063661982133 74.99909561022376) bank153419 +153420 POINT(40.731407049813065 73.20653316243002) bank153420 +153421 POINT(41.04924587379582 73.27842743063673) bank153421 +153422 POINT(41.02043404967669 74.5012062726735) bank153422 +153423 POINT(40.48938833974303 73.53914295309795) bank153423 +153424 POINT(39.87566344461892 74.44788587708855) bank153424 +153425 POINT(41.50116533834068 74.99885706861079) bank153425 +153426 POINT(40.74807625462199 73.77493769102894) bank153426 +153427 POINT(39.76690443608435 73.94710307539566) bank153427 +153428 POINT(41.18114570400691 74.07818126266834) bank153428 +153429 POINT(40.38939847799047 73.28468594827659) bank153429 +153430 POINT(41.00934715677272 74.25483227607744) bank153430 +153431 POINT(40.31064146144122 74.012074366468) bank153431 +153432 POINT(39.807018747381754 74.61908347206685) bank153432 +153433 POINT(40.4561632601499 73.65404908872321) bank153433 +153434 POINT(40.57890677411084 73.3642876266629) bank153434 +153435 POINT(40.43549494239912 73.65482304074159) bank153435 +153436 POINT(40.85524968174835 73.07964343381738) bank153436 +153437 POINT(39.73029052356667 73.64343889897256) bank153437 +153438 POINT(40.986430521146055 74.14078897984992) bank153438 +153439 POINT(40.14241116226515 73.84222232933146) bank153439 +153440 POINT(40.627763095792005 74.16082731705222) bank153440 +153441 POINT(41.71041780054559 74.60417820220128) bank153441 +153442 POINT(41.32549427726019 73.75634095775618) bank153442 +153443 POINT(40.26995458642275 73.80527575021358) bank153443 +153444 POINT(39.76925359434785 73.63128675094946) bank153444 +153445 POINT(40.588398972739874 73.58113486830092) bank153445 +153446 POINT(40.77495354470923 74.814401805986) bank153446 +153447 POINT(41.616073709674 74.29675901568753) bank153447 +153448 POINT(41.27665004780141 73.90042158513296) bank153448 +153449 POINT(41.65506573512725 74.66214577594202) bank153449 +153450 POINT(41.484845378588204 74.07294310539841) bank153450 +153451 POINT(41.331608578801536 74.50042122205208) bank153451 +153452 POINT(41.47683440286078 74.10824928299758) bank153452 +153453 POINT(40.95811083038566 74.6383093893881) bank153453 +153454 POINT(40.78439221465797 73.29207360231936) bank153454 +153455 POINT(39.95388848234804 73.07074066400475) bank153455 +153456 POINT(40.45521461841668 73.92773737374947) bank153456 +153457 POINT(39.9691088991817 74.70894449416554) bank153457 +153458 POINT(40.214640932464135 74.22532102477805) bank153458 +153459 POINT(40.0003111976318 74.23905729326337) bank153459 +153460 POINT(41.52314263456521 74.50345236831508) bank153460 +153461 POINT(41.068598807903584 73.47818431487127) bank153461 +153462 POINT(39.97094855403754 73.89478663913269) bank153462 +153463 POINT(41.43784527645968 74.93098324371059) bank153463 +153464 POINT(39.76893294321602 73.59660786434632) bank153464 +153465 POINT(40.34058979259436 74.49716845637595) bank153465 +153466 POINT(40.8560372808895 74.0035523821378) bank153466 +153467 POINT(39.88959537049547 73.75974479044369) bank153467 +153468 POINT(40.688910816557204 73.30328219876843) bank153468 +153469 POINT(41.461820476371464 74.31228054535306) bank153469 +153470 POINT(41.67441808885355 74.85749885211722) bank153470 +153471 POINT(41.15626453293226 74.70152658622717) bank153471 +153472 POINT(41.153628411723275 73.70348562106619) bank153472 +153473 POINT(40.30422109844868 73.4487135816851) bank153473 +153474 POINT(40.30900733167315 74.0621232278643) bank153474 +153475 POINT(41.0665728287955 74.45550652505538) bank153475 +153476 POINT(39.89228926772622 73.42782597764577) bank153476 +153477 POINT(40.223875913942585 74.32778121925436) bank153477 +153478 POINT(40.03885188654448 73.2659981344284) bank153478 +153479 POINT(40.11088453651439 74.32063383880252) bank153479 +153480 POINT(40.22264180084239 74.90224403555368) bank153480 +153481 POINT(40.8749815604916 74.7627042660567) bank153481 +153482 POINT(40.237035363393886 74.4528821499484) bank153482 +153483 POINT(41.25196227824975 73.28793490915604) bank153483 +153484 POINT(40.707874544316645 73.76003222756202) bank153484 +153485 POINT(40.424743687475676 74.23937362577303) bank153485 +153486 POINT(39.900410417484885 74.84987722656629) bank153486 +153487 POINT(39.75672123806307 73.9739290672265) bank153487 +153488 POINT(40.27494414388906 73.2467034135881) bank153488 +153489 POINT(40.955200433259144 74.78943609822872) bank153489 +153490 POINT(41.62405141027031 73.05793762088534) bank153490 +153491 POINT(40.7301429603348 73.14943318416906) bank153491 +153492 POINT(40.25426942339321 74.75444017430718) bank153492 +153493 POINT(40.72740214229142 74.60061552305754) bank153493 +153494 POINT(40.17307503830139 73.34451878880137) bank153494 +153495 POINT(41.539444276601984 74.84837287202161) bank153495 +153496 POINT(40.194969993284445 73.1608011331603) bank153496 +153497 POINT(40.367842927324794 74.8744987659103) bank153497 +153498 POINT(40.12040328128122 73.01218893144288) bank153498 +153499 POINT(41.566521189813095 73.5165320131226) bank153499 +153500 POINT(41.04172747810331 74.62599353025169) bank153500 +153501 POINT(40.094610864353065 73.1820774119218) bank153501 +153502 POINT(41.00719486488536 74.89047898554071) bank153502 +153503 POINT(39.843855526546626 74.41758649253117) bank153503 +153504 POINT(40.21484535799651 73.8915217509527) bank153504 +153505 POINT(40.64590934567166 74.80499103204339) bank153505 +153506 POINT(39.90649543180401 74.2321212259357) bank153506 +153507 POINT(39.94183571429932 74.87616893870513) bank153507 +153508 POINT(40.30228280566982 73.2771097568901) bank153508 +153509 POINT(41.615267977885935 74.1227960894112) bank153509 +153510 POINT(41.046220976848105 73.12685716086443) bank153510 +153511 POINT(40.25995115675132 74.91562834469298) bank153511 +153512 POINT(40.87748926966287 73.94141477397868) bank153512 +153513 POINT(40.58637154912605 74.91085743074362) bank153513 +153514 POINT(41.10312983513034 73.32380094020995) bank153514 +153515 POINT(40.69696528181804 74.90210534028168) bank153515 +153516 POINT(41.70621147009553 73.78525160459078) bank153516 +153517 POINT(41.277377111090146 73.46857239664656) bank153517 +153518 POINT(41.630266268468404 73.895326886298) bank153518 +153519 POINT(41.70338917363568 73.15693555175017) bank153519 +153520 POINT(40.771690236634136 74.37506086148836) bank153520 +153521 POINT(41.00644814546037 74.99163362984157) bank153521 +153522 POINT(41.490449983945396 73.49770849685547) bank153522 +153523 POINT(40.1679956287349 73.86601224235264) bank153523 +153524 POINT(40.18771116180752 73.14617612367783) bank153524 +153525 POINT(41.41509715658648 73.57517215758756) bank153525 +153526 POINT(40.12847791552798 73.26550983952504) bank153526 +153527 POINT(39.97832928777456 73.6686164608565) bank153527 +153528 POINT(40.37831778843697 73.01210035268629) bank153528 +153529 POINT(40.97123934428729 74.85805702686949) bank153529 +153530 POINT(41.486315627982435 74.99948937938537) bank153530 +153531 POINT(41.098408549289296 74.35449576510511) bank153531 +153532 POINT(40.15197579820013 73.04199310333347) bank153532 +153533 POINT(40.71443326403889 73.79958411008013) bank153533 +153534 POINT(40.54501718814119 73.8037672944889) bank153534 +153535 POINT(41.20338045219487 74.7088394561363) bank153535 +153536 POINT(39.78340426253511 73.78504446584246) bank153536 +153537 POINT(40.33154607414934 73.28106090291392) bank153537 +153538 POINT(41.57422364885925 74.15578530343524) bank153538 +153539 POINT(40.41135220318378 73.20616179724468) bank153539 +153540 POINT(41.700669897312075 73.6832669980244) bank153540 +153541 POINT(40.6647816780075 73.65811980948912) bank153541 +153542 POINT(40.08744011282577 74.1215236729813) bank153542 +153543 POINT(41.49526454843514 74.32373826829922) bank153543 +153544 POINT(40.236191351555036 74.72258684332635) bank153544 +153545 POINT(39.96568303947133 73.90239248493778) bank153545 +153546 POINT(39.922779322278316 74.36240876764877) bank153546 +153547 POINT(39.98620227255313 73.11762033128608) bank153547 +153548 POINT(40.344833424247824 74.14185906662046) bank153548 +153549 POINT(41.35814930924169 74.87051113189698) bank153549 +153550 POINT(39.95370682804991 74.12205200501626) bank153550 +153551 POINT(41.38059182862638 74.80670383907714) bank153551 +153552 POINT(41.510641111235124 73.81112267301242) bank153552 +153553 POINT(41.29037654208889 73.09777241908586) bank153553 +153554 POINT(40.994186292744565 74.73246991690692) bank153554 +153555 POINT(39.78681889543933 73.04253983326534) bank153555 +153556 POINT(40.91713831193097 74.53574342081373) bank153556 +153557 POINT(39.816907339646754 73.28879478174879) bank153557 +153558 POINT(40.69688468722489 74.1478328146947) bank153558 +153559 POINT(40.30086899993707 73.33406667490227) bank153559 +153560 POINT(41.08075801426809 73.04471941913683) bank153560 +153561 POINT(41.648862389529484 74.90279411569036) bank153561 +153562 POINT(40.67067483084393 73.73236964255442) bank153562 +153563 POINT(40.53060740609249 74.67846125742615) bank153563 +153564 POINT(41.593571354585755 73.18479031160423) bank153564 +153565 POINT(40.643285989130135 74.77719075994258) bank153565 +153566 POINT(41.323051803880425 73.0340268112278) bank153566 +153567 POINT(41.0228120360418 73.44654044607337) bank153567 +153568 POINT(41.45365415433356 73.36392806853688) bank153568 +153569 POINT(40.94474025699192 73.89792294225157) bank153569 +153570 POINT(40.652174593490685 74.61591118534467) bank153570 +153571 POINT(40.43897099455714 74.49366241730142) bank153571 +153572 POINT(41.024775244273975 73.8867092241232) bank153572 +153573 POINT(39.9631817656828 73.45532370247811) bank153573 +153574 POINT(40.91572220506126 73.50615470046318) bank153574 +153575 POINT(40.05173697975109 74.50491717179841) bank153575 +153576 POINT(40.66974628468489 73.37569376130715) bank153576 +153577 POINT(39.89425001299403 74.25749143098477) bank153577 +153578 POINT(41.477647542396056 74.8997724498833) bank153578 +153579 POINT(41.31798262718106 73.77562770525672) bank153579 +153580 POINT(39.74880591570382 74.10777203245179) bank153580 +153581 POINT(41.53554308168347 73.51303720733402) bank153581 +153582 POINT(40.61210355408114 73.76846702805994) bank153582 +153583 POINT(40.10555813338019 74.80596746786863) bank153583 +153584 POINT(40.12442869817918 73.80268919826331) bank153584 +153585 POINT(41.23517712509175 73.91969267652563) bank153585 +153586 POINT(40.73548369091084 74.51181902374272) bank153586 +153587 POINT(39.78105968697543 73.519041187114) bank153587 +153588 POINT(39.79805652040089 74.1172231023442) bank153588 +153589 POINT(39.748701050818525 73.50638797272153) bank153589 +153590 POINT(40.82907963067092 73.41082656821862) bank153590 +153591 POINT(41.45801919549824 74.9043061326413) bank153591 +153592 POINT(41.60516426286183 74.49198961825755) bank153592 +153593 POINT(40.29519215013186 74.1138048687115) bank153593 +153594 POINT(40.263450523500765 74.70604299646497) bank153594 +153595 POINT(39.88535905970732 73.2025117137267) bank153595 +153596 POINT(40.90143969665724 74.79745662900187) bank153596 +153597 POINT(40.746954633549755 74.25358250479536) bank153597 +153598 POINT(41.132777831462604 73.81000091977631) bank153598 +153599 POINT(41.470921774378084 74.08456332033319) bank153599 +153600 POINT(41.103562212728626 73.64861030262459) bank153600 +153601 POINT(40.36884242552084 73.5099859376041) bank153601 +153602 POINT(41.539833010685015 73.82185728699184) bank153602 +153603 POINT(41.12157232630693 74.77974506901751) bank153603 +153604 POINT(41.07135867466703 74.04881981408921) bank153604 +153605 POINT(40.95030778587011 73.17854421909617) bank153605 +153606 POINT(41.1113471659557 73.5033522557896) bank153606 +153607 POINT(41.59418201473164 73.68531959487888) bank153607 +153608 POINT(41.50444025001665 73.36815756567886) bank153608 +153609 POINT(40.747447872332934 74.47959533899378) bank153609 +153610 POINT(40.93906585418738 73.82835142142352) bank153610 +153611 POINT(41.20896200224009 74.56577222148951) bank153611 +153612 POINT(41.452479620671994 73.27575135437516) bank153612 +153613 POINT(40.509266134593815 73.64911081364244) bank153613 +153614 POINT(40.841766833776134 74.82124151231443) bank153614 +153615 POINT(40.3022936628409 73.91403500355378) bank153615 +153616 POINT(39.74378160619851 74.01522906961135) bank153616 +153617 POINT(41.685309778130936 73.50706696089235) bank153617 +153618 POINT(40.05542842961453 74.5761019787818) bank153618 +153619 POINT(41.44686513267479 73.12606676829091) bank153619 +153620 POINT(41.46384182334616 73.60540279048341) bank153620 +153621 POINT(41.628376053287134 74.72961875135962) bank153621 +153622 POINT(40.41998661622035 74.37070544541811) bank153622 +153623 POINT(41.648239614472736 73.24889892287152) bank153623 +153624 POINT(39.813718828927676 74.23149612927567) bank153624 +153625 POINT(41.128251953956244 73.85258579170701) bank153625 +153626 POINT(41.58557526981532 74.3152720975853) bank153626 +153627 POINT(41.32256711480126 74.03794133123185) bank153627 +153628 POINT(40.23964534948488 73.55533225497518) bank153628 +153629 POINT(41.074564652066655 73.64607389214767) bank153629 +153630 POINT(40.53128009616234 74.73815252048763) bank153630 +153631 POINT(40.40022192905265 73.3166833859666) bank153631 +153632 POINT(41.21968897651508 73.69278608770053) bank153632 +153633 POINT(40.69073979802763 74.60130317947505) bank153633 +153634 POINT(40.655835724013976 74.32771368242095) bank153634 +153635 POINT(40.755324258011186 74.59342273674896) bank153635 +153636 POINT(41.54233800992219 74.2766273956014) bank153636 +153637 POINT(41.467665155081974 73.81194015473717) bank153637 +153638 POINT(41.6975947152106 73.67460325237532) bank153638 +153639 POINT(41.35411358800535 73.65869596027697) bank153639 +153640 POINT(41.00187782958993 74.66343423685836) bank153640 +153641 POINT(40.330074205007634 73.51905353185337) bank153641 +153642 POINT(41.586032942755935 73.08054643223484) bank153642 +153643 POINT(39.83324390314628 73.17120611869083) bank153643 +153644 POINT(39.72803994161714 73.52019119775021) bank153644 +153645 POINT(41.1788718959785 74.1065189592545) bank153645 +153646 POINT(39.772603569254166 73.57404928083537) bank153646 +153647 POINT(41.70376807294229 73.56839994595437) bank153647 +153648 POINT(40.24798070311766 73.67193455973829) bank153648 +153649 POINT(40.79993012132479 74.96552880193157) bank153649 +153650 POINT(41.0674509776019 73.8089861433579) bank153650 +153651 POINT(40.2460914371145 73.29917439637985) bank153651 +153652 POINT(41.04959171850189 73.75780398171698) bank153652 +153653 POINT(40.769465532841224 74.08648212282908) bank153653 +153654 POINT(40.73067778426835 74.68985531096217) bank153654 +153655 POINT(40.31850671811982 73.18389937252276) bank153655 +153656 POINT(40.66674156464527 74.27852044537875) bank153656 +153657 POINT(39.765464076066884 73.86175617048144) bank153657 +153658 POINT(40.54285644283421 74.6113629106408) bank153658 +153659 POINT(40.480454187988165 73.03427843946746) bank153659 +153660 POINT(41.19994630609623 73.34325794593171) bank153660 +153661 POINT(40.97639432381403 73.1990021588849) bank153661 +153662 POINT(40.22642966270391 74.32136735431352) bank153662 +153663 POINT(40.23866086665513 74.67525338772336) bank153663 +153664 POINT(40.5403492458428 73.44506596979186) bank153664 +153665 POINT(41.608367016495514 73.55834399381624) bank153665 +153666 POINT(40.04949342009235 73.8364030353842) bank153666 +153667 POINT(40.38982894536922 74.35092127784107) bank153667 +153668 POINT(41.68339149756768 74.96619015673171) bank153668 +153669 POINT(39.74991328305012 73.17418194796839) bank153669 +153670 POINT(41.41915979227287 74.02510132883356) bank153670 +153671 POINT(40.29738458299518 73.99229299800396) bank153671 +153672 POINT(39.94701896470195 73.71996336374343) bank153672 +153673 POINT(40.4099743489727 73.4375875383949) bank153673 +153674 POINT(41.65554492827987 74.19337173052719) bank153674 +153675 POINT(40.006758112802196 74.41112842313551) bank153675 +153676 POINT(40.15896472839184 74.73340989713404) bank153676 +153677 POINT(40.44983230559609 74.92154922905797) bank153677 +153678 POINT(40.64879513082254 74.24319993817431) bank153678 +153679 POINT(40.913013271658905 73.13453998245889) bank153679 +153680 POINT(40.84366584283632 74.26406472555539) bank153680 +153681 POINT(40.5791869370367 73.71718516883986) bank153681 +153682 POINT(40.09455855176407 73.99775240727524) bank153682 +153683 POINT(41.03715871001336 73.06199698012708) bank153683 +153684 POINT(40.6771570425878 74.59314682780573) bank153684 +153685 POINT(40.00195328045083 73.84159555415182) bank153685 +153686 POINT(39.801127989600175 74.59428737967653) bank153686 +153687 POINT(40.942026263882255 74.07023117004796) bank153687 +153688 POINT(41.23739137726721 74.52121214959014) bank153688 +153689 POINT(39.71807421751328 73.54221943820318) bank153689 +153690 POINT(39.80680768415537 74.85299203441568) bank153690 +153691 POINT(39.74272318144548 73.5016445194418) bank153691 +153692 POINT(39.89488424028252 74.84256474248559) bank153692 +153693 POINT(40.06856463707782 73.29579098215426) bank153693 +153694 POINT(40.429865823224155 74.09571618753756) bank153694 +153695 POINT(40.2027953072482 73.3696913793239) bank153695 +153696 POINT(40.67903269561933 74.7587989871804) bank153696 +153697 POINT(41.19978554079397 73.96549030488923) bank153697 +153698 POINT(41.25263250359822 73.92389802652404) bank153698 +153699 POINT(41.66136696743565 73.8440420233913) bank153699 +153700 POINT(41.527367771815996 74.36900474109527) bank153700 +153701 POINT(40.76475560167176 74.28544545151345) bank153701 +153702 POINT(40.323867060434736 74.52904639956108) bank153702 +153703 POINT(40.90106613862165 74.10431680806111) bank153703 +153704 POINT(40.274217356404606 73.63534007971587) bank153704 +153705 POINT(40.07423206302946 74.39411063778316) bank153705 +153706 POINT(39.71419969817274 73.8757122745168) bank153706 +153707 POINT(41.644494965046036 73.19849282329444) bank153707 +153708 POINT(41.216567904368986 73.45689714946462) bank153708 +153709 POINT(40.73272790125299 73.93428402000417) bank153709 +153710 POINT(40.81717031323507 73.8213927242202) bank153710 +153711 POINT(40.99083883601298 73.96271255795612) bank153711 +153712 POINT(40.110009655032755 73.2734590820312) bank153712 +153713 POINT(41.257840037786046 74.33353679992375) bank153713 +153714 POINT(41.57254051817694 73.14410269355815) bank153714 +153715 POINT(40.04341776062293 73.68366370605725) bank153715 +153716 POINT(39.86792250173548 73.1668105553855) bank153716 +153717 POINT(41.001932805127424 73.18882991203441) bank153717 +153718 POINT(40.55025826340897 74.56557682615285) bank153718 +153719 POINT(39.76899508863683 74.07516946569658) bank153719 +153720 POINT(39.788467358583716 73.2086388365356) bank153720 +153721 POINT(40.990348479697715 73.24884940185028) bank153721 +153722 POINT(39.869467974104396 73.5578411957137) bank153722 +153723 POINT(40.214736054982204 74.10909624067897) bank153723 +153724 POINT(41.529575217137435 74.77797843966214) bank153724 +153725 POINT(39.947824058582526 74.93770574290187) bank153725 +153726 POINT(40.187456113315385 74.3169430851153) bank153726 +153727 POINT(40.807083388897226 73.93626843056632) bank153727 +153728 POINT(40.0311915173705 73.99979576776269) bank153728 +153729 POINT(41.33361509998294 74.03802245670889) bank153729 +153730 POINT(40.34320603102657 73.98642088563406) bank153730 +153731 POINT(41.38625081604252 73.13440593997643) bank153731 +153732 POINT(41.00509965412072 74.99176051411918) bank153732 +153733 POINT(40.06163600051283 74.78511745038108) bank153733 +153734 POINT(40.36869295888183 73.48511783128554) bank153734 +153735 POINT(39.80339837588311 74.69350776191692) bank153735 +153736 POINT(41.627033273460086 74.71583615318475) bank153736 +153737 POINT(40.23692893028423 74.61939537739013) bank153737 +153738 POINT(40.268343325388 74.1558832894391) bank153738 +153739 POINT(40.83927356064528 74.34118713859378) bank153739 +153740 POINT(40.97995084146376 73.87936827997197) bank153740 +153741 POINT(40.873997623864945 73.72867701233265) bank153741 +153742 POINT(41.64911262321968 74.25751214392203) bank153742 +153743 POINT(39.932356128431984 73.71903351807202) bank153743 +153744 POINT(40.71256328259779 74.02125045200036) bank153744 +153745 POINT(40.44193100568713 73.60802468886399) bank153745 +153746 POINT(41.02573455619967 73.77725292832915) bank153746 +153747 POINT(41.42055612582495 73.3879961770467) bank153747 +153748 POINT(40.7059666262272 73.5114931921379) bank153748 +153749 POINT(41.64578469834425 74.29613234459246) bank153749 +153750 POINT(41.40898551798902 73.57324668345926) bank153750 +153751 POINT(40.674144300595124 73.43016423046247) bank153751 +153752 POINT(41.171033258002055 74.2874618006453) bank153752 +153753 POINT(40.65303211114911 74.36518916121899) bank153753 +153754 POINT(39.856199104272754 74.40788877948417) bank153754 +153755 POINT(41.58000060711068 74.17765981562495) bank153755 +153756 POINT(40.35592376487439 73.93179873003032) bank153756 +153757 POINT(41.25993039071234 74.63751190631602) bank153757 +153758 POINT(40.90778496362009 74.15865390252542) bank153758 +153759 POINT(40.31553035841262 73.76111614116675) bank153759 +153760 POINT(39.81867552691996 73.38549856167596) bank153760 +153761 POINT(41.070288930905654 73.42291688657443) bank153761 +153762 POINT(40.1719739217212 74.08826855729534) bank153762 +153763 POINT(40.151608888462164 73.86021589587786) bank153763 +153764 POINT(40.60367120246113 74.14777865910754) bank153764 +153765 POINT(41.24632193041054 73.07889580138358) bank153765 +153766 POINT(40.700730439294155 73.01389656511179) bank153766 +153767 POINT(39.95800354400434 74.69720379064981) bank153767 +153768 POINT(40.696289431732524 73.71008391909558) bank153768 +153769 POINT(40.02174263125572 73.49883865666867) bank153769 +153770 POINT(40.99157985321922 73.41468012328683) bank153770 +153771 POINT(40.79421745066739 74.83825651124529) bank153771 +153772 POINT(40.808800439190435 74.59502445383993) bank153772 +153773 POINT(40.79051826735944 73.52228715537605) bank153773 +153774 POINT(40.63893965919383 73.11474461058542) bank153774 +153775 POINT(41.162266118354644 73.65731667094576) bank153775 +153776 POINT(39.93906100259886 74.22835142068413) bank153776 +153777 POINT(40.83204619029962 73.38015670911592) bank153777 +153778 POINT(41.43176585988065 74.62194900901555) bank153778 +153779 POINT(40.01777947924091 74.84018267019343) bank153779 +153780 POINT(39.74155999979427 74.52780977008742) bank153780 +153781 POINT(41.59509068805635 73.06512446350183) bank153781 +153782 POINT(41.45563585586243 73.42882038026637) bank153782 +153783 POINT(40.21580344923118 73.67532031943765) bank153783 +153784 POINT(40.56190914563879 73.13136986522515) bank153784 +153785 POINT(40.26327379496852 73.31753469374112) bank153785 +153786 POINT(41.46961576575711 73.39747388819458) bank153786 +153787 POINT(41.35755695753851 73.01445520969928) bank153787 +153788 POINT(40.74233662393411 73.12005279768435) bank153788 +153789 POINT(40.36329792545597 73.8722777627591) bank153789 +153790 POINT(40.63816513040212 73.63946996431031) bank153790 +153791 POINT(41.50214650672328 75.00167519016023) bank153791 +153792 POINT(41.69759709140114 74.85014157778991) bank153792 +153793 POINT(41.0397771736627 74.28657639236476) bank153793 +153794 POINT(40.5115725556756 73.05047343937485) bank153794 +153795 POINT(41.092839408065736 73.65974305532181) bank153795 +153796 POINT(41.21119809888052 73.41703717347994) bank153796 +153797 POINT(41.147401486765446 74.32613764826651) bank153797 +153798 POINT(40.098735790942975 73.45901488007493) bank153798 +153799 POINT(40.15829138187334 73.12612607036839) bank153799 +153800 POINT(41.0035961484573 74.07429632733026) bank153800 +153801 POINT(41.492754270761175 73.30666204787877) bank153801 +153802 POINT(40.480359637463415 73.01975183193811) bank153802 +153803 POINT(40.59686534808143 73.23488563480149) bank153803 +153804 POINT(40.72711347606101 74.46328815937028) bank153804 +153805 POINT(41.63709139861911 73.51936835179953) bank153805 +153806 POINT(40.98773970847517 73.2022807519423) bank153806 +153807 POINT(40.976640520563 74.68117943780376) bank153807 +153808 POINT(41.06609819681312 73.44400957247332) bank153808 +153809 POINT(39.953823071321395 73.30589930721723) bank153809 +153810 POINT(41.166925206073046 73.81888974656344) bank153810 +153811 POINT(40.53872222871196 74.27850440514075) bank153811 +153812 POINT(40.52725328577541 73.83285439472255) bank153812 +153813 POINT(39.85406930486624 73.7176170383188) bank153813 +153814 POINT(41.292246464518605 73.66848612197398) bank153814 +153815 POINT(40.658138542409354 74.17733603980092) bank153815 +153816 POINT(40.535978471211536 73.22008705166581) bank153816 +153817 POINT(40.561764229255715 74.6009524227622) bank153817 +153818 POINT(39.86854179974016 73.97388514962798) bank153818 +153819 POINT(40.34734630282666 74.37304801593818) bank153819 +153820 POINT(40.48102219297455 74.10609349664233) bank153820 +153821 POINT(40.369623174233645 73.50260137319088) bank153821 +153822 POINT(40.943203344808424 74.05893204348489) bank153822 +153823 POINT(39.91786675857635 73.82725160721418) bank153823 +153824 POINT(40.148130665824205 74.62376989513265) bank153824 +153825 POINT(41.63362117365189 74.23617632145746) bank153825 +153826 POINT(39.86912565110623 74.73679076286606) bank153826 +153827 POINT(40.91945324482275 74.05333257721759) bank153827 +153828 POINT(41.40943259903063 73.59167653022072) bank153828 +153829 POINT(41.305235402097075 74.87789074813725) bank153829 +153830 POINT(39.94962898647204 74.84114331325145) bank153830 +153831 POINT(41.293693335301285 74.19523485772245) bank153831 +153832 POINT(40.74460620062204 73.1970746288855) bank153832 +153833 POINT(41.304958566924924 74.26089942493354) bank153833 +153834 POINT(41.640205146843286 73.76082026881204) bank153834 +153835 POINT(41.11666179055938 74.03786743035607) bank153835 +153836 POINT(40.042470739658505 74.72983033184696) bank153836 +153837 POINT(40.07474077536425 73.43886140304953) bank153837 +153838 POINT(40.937021536116134 74.44325075860372) bank153838 +153839 POINT(40.005195427948124 74.17179601438887) bank153839 +153840 POINT(41.21281784040604 73.57656307383726) bank153840 +153841 POINT(41.00604923840319 73.44734191031321) bank153841 +153842 POINT(39.75822664452738 74.12263589461563) bank153842 +153843 POINT(41.665733759252866 73.16338930383581) bank153843 +153844 POINT(39.921630374604796 73.35759196312469) bank153844 +153845 POINT(41.57832271170504 74.7704897602849) bank153845 +153846 POINT(40.06894340275136 74.15377695715831) bank153846 +153847 POINT(40.79018272129679 74.9237483593612) bank153847 +153848 POINT(39.93495532238433 74.70316658151955) bank153848 +153849 POINT(40.952992176737375 73.16597160447223) bank153849 +153850 POINT(41.53265218399372 74.63859429951071) bank153850 +153851 POINT(41.4417828220015 74.62478555669124) bank153851 +153852 POINT(41.10481477884777 74.14302277576459) bank153852 +153853 POINT(41.02520976652651 74.25593593741868) bank153853 +153854 POINT(40.99870255161494 74.4384463099013) bank153854 +153855 POINT(40.791554512203255 73.78715044238726) bank153855 +153856 POINT(41.31648435677297 73.57830250683145) bank153856 +153857 POINT(40.72961306926579 73.70882841365938) bank153857 +153858 POINT(41.52608117628462 73.93062325825473) bank153858 +153859 POINT(41.16605138032232 73.40634756785262) bank153859 +153860 POINT(40.22722060787108 73.4948090516689) bank153860 +153861 POINT(40.344408424838015 74.15711441036366) bank153861 +153862 POINT(41.431642335177386 74.01202894380764) bank153862 +153863 POINT(39.7396850097406 74.120317310217) bank153863 +153864 POINT(40.539240393357744 73.27055862846021) bank153864 +153865 POINT(40.16386542849566 74.85124422341387) bank153865 +153866 POINT(39.94687554371961 73.30092361522044) bank153866 +153867 POINT(39.81298403448633 73.5961373830871) bank153867 +153868 POINT(41.447516727477655 73.84893650511971) bank153868 +153869 POINT(41.03771474000299 74.66840873515459) bank153869 +153870 POINT(40.617165209894125 74.26586287288751) bank153870 +153871 POINT(40.88598370423359 73.3424250399104) bank153871 +153872 POINT(40.0642994748149 73.2845543569573) bank153872 +153873 POINT(41.60594498487736 74.45105469565016) bank153873 +153874 POINT(41.406545519701645 74.33791069444787) bank153874 +153875 POINT(40.49201676144216 73.85711759251525) bank153875 +153876 POINT(40.73493209115002 73.21677698083694) bank153876 +153877 POINT(40.404599941953016 74.49206428189454) bank153877 +153878 POINT(39.836708660815944 74.14367754652405) bank153878 +153879 POINT(41.55387755927341 74.26696015766439) bank153879 +153880 POINT(39.874863095238226 73.48575437773474) bank153880 +153881 POINT(41.66706170003058 73.36994462957156) bank153881 +153882 POINT(40.724822982959886 74.77584607690554) bank153882 +153883 POINT(41.493723255087374 74.99820989782263) bank153883 +153884 POINT(41.69951173781906 74.60744869792097) bank153884 +153885 POINT(41.619354304985954 74.06652939880554) bank153885 +153886 POINT(40.98693614130081 73.43135505259777) bank153886 +153887 POINT(41.14483402777465 73.21052000520515) bank153887 +153888 POINT(41.59118692703486 74.53225076543698) bank153888 +153889 POINT(39.772710478501764 73.43551849626323) bank153889 +153890 POINT(41.526757296550045 73.12241385280731) bank153890 +153891 POINT(41.45774714979794 73.83333233655425) bank153891 +153892 POINT(41.37618446140675 73.0846108470593) bank153892 +153893 POINT(41.117059829269124 74.48652218075803) bank153893 +153894 POINT(41.559483207043215 73.92960497170856) bank153894 +153895 POINT(41.61754309413028 73.43828242521842) bank153895 +153896 POINT(41.70744038676575 73.41347817858306) bank153896 +153897 POINT(40.04627809972133 74.88554701393764) bank153897 +153898 POINT(39.95781706264824 73.96930524623592) bank153898 +153899 POINT(40.89683282321393 73.96449156513556) bank153899 +153900 POINT(39.80192435853504 74.82231296169293) bank153900 +153901 POINT(39.89324175644523 73.71971885117219) bank153901 +153902 POINT(39.9513279376888 74.10135728969377) bank153902 +153903 POINT(40.552262772509906 73.1515146927275) bank153903 +153904 POINT(40.09676857916929 74.33474193271124) bank153904 +153905 POINT(39.8887517081543 74.06222749221504) bank153905 +153906 POINT(40.52647124569219 73.43201867653654) bank153906 +153907 POINT(39.98931349492915 74.4591499388671) bank153907 +153908 POINT(41.590126141806465 74.6082005968729) bank153908 +153909 POINT(41.48452343114609 73.50732168949939) bank153909 +153910 POINT(40.12007922577278 73.62828359305607) bank153910 +153911 POINT(40.83823025153846 73.76366795310214) bank153911 +153912 POINT(40.04170046753247 74.82252726161275) bank153912 +153913 POINT(41.7098871565038 73.19793935850248) bank153913 +153914 POINT(40.82046313566297 73.11312646164572) bank153914 +153915 POINT(41.6475349763256 74.5181640944189) bank153915 +153916 POINT(41.28483456912382 74.84964428413963) bank153916 +153917 POINT(40.327445135999994 74.07778344968014) bank153917 +153918 POINT(41.02884682963455 74.92469101087462) bank153918 +153919 POINT(39.95615114477875 74.27775594163683) bank153919 +153920 POINT(41.19079228474134 73.93290975423686) bank153920 +153921 POINT(40.603931170429995 73.2747328384204) bank153921 +153922 POINT(40.409357282210564 73.59885032944425) bank153922 +153923 POINT(40.38874679739246 74.64169317353691) bank153923 +153924 POINT(40.1931168003574 74.87035138766588) bank153924 +153925 POINT(41.47876652593035 73.57371559326504) bank153925 +153926 POINT(40.73968032541693 73.44938209223918) bank153926 +153927 POINT(41.10685581122356 74.91478574393842) bank153927 +153928 POINT(39.72575063076031 73.67888290272033) bank153928 +153929 POINT(40.57570060820768 73.10995493879285) bank153929 +153930 POINT(41.15015747168327 74.65607073237261) bank153930 +153931 POINT(40.27653813578894 74.24165383297161) bank153931 +153932 POINT(39.965652599759935 74.98416517135526) bank153932 +153933 POINT(41.322986782494006 73.30850166375099) bank153933 +153934 POINT(41.09673093728552 74.55700236141946) bank153934 +153935 POINT(39.74414000466437 74.44619762245269) bank153935 +153936 POINT(39.93914133607339 73.87943101560558) bank153936 +153937 POINT(41.19559092613552 73.47331508080528) bank153937 +153938 POINT(41.64631950992989 73.25763143963437) bank153938 +153939 POINT(41.58993214322542 73.26604338721003) bank153939 +153940 POINT(40.43253757656762 74.78295651390513) bank153940 +153941 POINT(40.56902155042959 74.02260711935679) bank153941 +153942 POINT(41.43026258468752 73.1604366050894) bank153942 +153943 POINT(40.81383750859912 74.39395278980805) bank153943 +153944 POINT(40.17486568159853 74.19545962806556) bank153944 +153945 POINT(40.19702299382998 73.72763493739161) bank153945 +153946 POINT(39.96742535843055 73.97116156771374) bank153946 +153947 POINT(40.764829629751574 74.91538407307723) bank153947 +153948 POINT(40.88110111631912 74.96296140612229) bank153948 +153949 POINT(41.00519448283367 74.89135207180504) bank153949 +153950 POINT(41.390314984831974 73.7873923169622) bank153950 +153951 POINT(40.151969352403135 74.56428792569442) bank153951 +153952 POINT(39.74084622156182 73.37508795875931) bank153952 +153953 POINT(41.53806311565056 73.82350404461157) bank153953 +153954 POINT(39.90746703873744 74.6643725070555) bank153954 +153955 POINT(41.40375143567511 74.27259787200529) bank153955 +153956 POINT(40.25184288933553 74.66314680503523) bank153956 +153957 POINT(41.64363514799644 73.3757452665683) bank153957 +153958 POINT(41.575287017045085 73.79035742269946) bank153958 +153959 POINT(40.340328976805644 73.7321333574718) bank153959 +153960 POINT(40.896809478448084 74.19972412812618) bank153960 +153961 POINT(40.93042599018741 73.09865315583312) bank153961 +153962 POINT(40.15053482714037 74.87761245237995) bank153962 +153963 POINT(41.436056974042465 73.96048108835988) bank153963 +153964 POINT(40.946816925319446 74.13486441180028) bank153964 +153965 POINT(40.92925382132476 73.51128320672206) bank153965 +153966 POINT(41.26203607829787 73.74724327090863) bank153966 +153967 POINT(40.96172411236136 74.48512194812096) bank153967 +153968 POINT(39.80343419032368 74.09464601573549) bank153968 +153969 POINT(39.89953462058223 74.81273472886689) bank153969 +153970 POINT(39.80309454191727 74.00018092664212) bank153970 +153971 POINT(40.47364562722308 73.84704320326968) bank153971 +153972 POINT(39.99038112966059 73.86814587165807) bank153972 +153973 POINT(40.57148922734915 74.16295304445426) bank153973 +153974 POINT(41.50849587359386 74.94128508562879) bank153974 +153975 POINT(41.18783307587816 74.50563506384353) bank153975 +153976 POINT(41.52834447235676 74.01523742522764) bank153976 +153977 POINT(40.21570900346342 74.93018956313313) bank153977 +153978 POINT(40.02616743116495 74.67050859833844) bank153978 +153979 POINT(40.90455618285288 73.65040768415908) bank153979 +153980 POINT(40.775669633226656 73.8318166861716) bank153980 +153981 POINT(40.455114273981856 73.93048420110587) bank153981 +153982 POINT(40.20049838187412 74.87296614888443) bank153982 +153983 POINT(40.802948468620265 74.04142951797716) bank153983 +153984 POINT(40.50722674184597 74.77669506494874) bank153984 +153985 POINT(41.301250992853774 74.63848122992955) bank153985 +153986 POINT(40.63694190715427 74.66637899458138) bank153986 +153987 POINT(41.421190457573516 73.87986755817502) bank153987 +153988 POINT(39.953576962994816 74.41002794518023) bank153988 +153989 POINT(40.83487623865253 73.92023744738573) bank153989 +153990 POINT(41.07658181908346 74.35671228131518) bank153990 +153991 POINT(40.23760476256934 74.07111961476203) bank153991 +153992 POINT(40.00906811680916 73.76382710641104) bank153992 +153993 POINT(41.236649371354154 73.87211581771189) bank153993 +153994 POINT(39.83283002386855 74.42514669419282) bank153994 +153995 POINT(41.07382731328831 74.31884108855722) bank153995 +153996 POINT(40.96395643433888 74.35735466233265) bank153996 +153997 POINT(40.74614118088181 73.66417142058738) bank153997 +153998 POINT(40.678388475120194 74.5637234812366) bank153998 +153999 POINT(40.525421860881785 74.01557099035043) bank153999 +154000 POINT(39.88157189226471 74.2797834217321) bank154000 +154001 POINT(40.775580014540374 74.74811689215527) bank154001 +154002 POINT(41.710517729432766 73.81075822364791) bank154002 +154003 POINT(41.55715328492172 74.81331737583413) bank154003 +154004 POINT(41.06801466918125 74.26759267294952) bank154004 +154005 POINT(39.82789450876486 74.77302664827832) bank154005 +154006 POINT(39.909237506438096 74.24815820076793) bank154006 +154007 POINT(41.20002171348332 73.42139790797863) bank154007 +154008 POINT(41.14467475370546 74.82836179071278) bank154008 +154009 POINT(40.09120889331212 73.86306392331493) bank154009 +154010 POINT(40.33507794000485 74.40945758733781) bank154010 +154011 POINT(41.20172481222172 73.51568272047822) bank154011 +154012 POINT(39.78102687344676 73.11598656598848) bank154012 +154013 POINT(41.31784219623942 74.69663049802993) bank154013 +154014 POINT(39.72597636609373 73.99364423838888) bank154014 +154015 POINT(40.97572253311919 74.53610781560134) bank154015 +154016 POINT(40.27476684143864 73.19132748168458) bank154016 +154017 POINT(41.708052341985095 73.55702609892491) bank154017 +154018 POINT(41.49005476614868 73.14834642425949) bank154018 +154019 POINT(40.53155424182063 74.73876746884763) bank154019 +154020 POINT(40.83229426522093 73.9646306637847) bank154020 +154021 POINT(39.81881747225482 74.88404918881302) bank154021 +154022 POINT(41.348397527200504 73.40121293802945) bank154022 +154023 POINT(39.92101191735407 74.04068925386999) bank154023 +154024 POINT(39.943191026850364 74.32291546026188) bank154024 +154025 POINT(41.675416262116364 74.20647527110371) bank154025 +154026 POINT(40.86121970167502 73.87304575854215) bank154026 +154027 POINT(39.73652173343334 73.77801135028682) bank154027 +154028 POINT(40.944184157755885 74.07753947707573) bank154028 +154029 POINT(41.20750432851316 74.77839552940327) bank154029 +154030 POINT(39.74412611732757 74.80447964077574) bank154030 +154031 POINT(41.6918608571723 74.05420155224479) bank154031 +154032 POINT(40.83723496415158 73.67427691052109) bank154032 +154033 POINT(40.343808157166144 73.7725803915201) bank154033 +154034 POINT(41.3722761650007 73.06958534215974) bank154034 +154035 POINT(39.870120643716504 73.5650093047935) bank154035 +154036 POINT(40.32010341654 73.20031980770601) bank154036 +154037 POINT(40.26375808754296 74.42580839165056) bank154037 +154038 POINT(41.55053767330314 74.8565690620043) bank154038 +154039 POINT(40.501111843953545 73.16162365728222) bank154039 +154040 POINT(40.91003178287676 74.89365958728564) bank154040 +154041 POINT(41.67679106292565 74.64979116162083) bank154041 +154042 POINT(40.616724371425384 74.23913975581496) bank154042 +154043 POINT(40.42016697665946 73.57128205921212) bank154043 +154044 POINT(40.01281771208878 73.43004922653816) bank154044 +154045 POINT(39.84389749708235 73.3695012128793) bank154045 +154046 POINT(40.424860818690654 74.48636146254945) bank154046 +154047 POINT(41.21738475312671 74.54010481301567) bank154047 +154048 POINT(40.233411888828506 74.34928998108002) bank154048 +154049 POINT(41.43825738468038 74.69442228089066) bank154049 +154050 POINT(40.18656310702111 73.88444569501837) bank154050 +154051 POINT(40.90739536661795 74.98121946009864) bank154051 +154052 POINT(41.32886420150055 74.43014248589509) bank154052 +154053 POINT(40.98572779241015 74.49489035087981) bank154053 +154054 POINT(39.80535335660984 73.56469920215208) bank154054 +154055 POINT(41.534694809524936 73.2261962968569) bank154055 +154056 POINT(40.73728583541028 74.49434533706079) bank154056 +154057 POINT(41.436041500509745 74.78773158097549) bank154057 +154058 POINT(40.97489020088971 73.11037543661398) bank154058 +154059 POINT(40.61792171373211 74.99111210234196) bank154059 +154060 POINT(41.1524596105306 73.02336821332263) bank154060 +154061 POINT(40.9514302242145 74.50812107257767) bank154061 +154062 POINT(39.79282998057416 73.6910997805511) bank154062 +154063 POINT(40.98140683903123 74.17529899969111) bank154063 +154064 POINT(39.96459442020546 74.84689468709092) bank154064 +154065 POINT(39.737457391265025 73.73770318110819) bank154065 +154066 POINT(40.872602104706566 74.40949493427459) bank154066 +154067 POINT(40.673922649274246 74.91985774280525) bank154067 +154068 POINT(39.733210176647574 74.09153605631032) bank154068 +154069 POINT(40.37192058050741 74.05850252556876) bank154069 +154070 POINT(41.088775797389935 73.2855844789847) bank154070 +154071 POINT(40.41302482613445 74.36364280496367) bank154071 +154072 POINT(40.135202228226774 74.47057621834706) bank154072 +154073 POINT(39.753367674088224 74.18324685526208) bank154073 +154074 POINT(41.21211643220923 73.28624148488727) bank154074 +154075 POINT(41.002406152774874 73.81403515375273) bank154075 +154076 POINT(41.05318035386302 74.16919081391563) bank154076 +154077 POINT(39.906611743858925 73.17567044398457) bank154077 +154078 POINT(40.79591026175398 74.98742006600766) bank154078 +154079 POINT(41.17743556645093 74.38274624978376) bank154079 +154080 POINT(39.99591474737585 73.61278157381523) bank154080 +154081 POINT(40.14211832955598 74.83330427024414) bank154081 +154082 POINT(40.45794350209841 73.16956946765474) bank154082 +154083 POINT(40.650635522424594 74.04369696333303) bank154083 +154084 POINT(39.738053882224754 74.93020812812712) bank154084 +154085 POINT(40.64730062799348 74.41474576049724) bank154085 +154086 POINT(41.51872169152211 74.20611073579059) bank154086 +154087 POINT(40.26746364327429 73.2501696382862) bank154087 +154088 POINT(40.52317488773485 73.07481194362762) bank154088 +154089 POINT(40.78911129920153 73.98762789601412) bank154089 +154090 POINT(39.73742880550676 73.84358304951225) bank154090 +154091 POINT(41.05689535710124 74.64595489730453) bank154091 +154092 POINT(41.652930249607486 74.50729208915854) bank154092 +154093 POINT(39.97118279066208 74.3560468659301) bank154093 +154094 POINT(40.05474019001811 74.34522162406694) bank154094 +154095 POINT(40.2507079005313 74.1515377237425) bank154095 +154096 POINT(39.7297376880421 73.34880084720767) bank154096 +154097 POINT(40.61859425597892 73.0864476969949) bank154097 +154098 POINT(41.23495679483877 73.07605862344276) bank154098 +154099 POINT(41.43745234434483 73.78463138959151) bank154099 +154100 POINT(41.43308712535502 74.47383249622698) bank154100 +154101 POINT(40.41018400012484 73.54466147574101) bank154101 +154102 POINT(41.2156173686137 73.73609676459782) bank154102 +154103 POINT(41.38066353515917 73.58432939468383) bank154103 +154104 POINT(41.253864146960616 73.22697866630847) bank154104 +154105 POINT(40.83614941479593 74.00146107039589) bank154105 +154106 POINT(40.73671935923235 74.48480345062883) bank154106 +154107 POINT(41.53784740351226 73.87530383152004) bank154107 +154108 POINT(40.30765241767227 74.60467480641015) bank154108 +154109 POINT(40.64567073052586 74.0487468676886) bank154109 +154110 POINT(40.93653197678681 73.40219806307165) bank154110 +154111 POINT(40.3330004359829 73.44586020947742) bank154111 +154112 POINT(40.39588890396031 73.26290617662272) bank154112 +154113 POINT(40.92046041667761 74.16814380347125) bank154113 +154114 POINT(40.914702439288604 74.33109276485999) bank154114 +154115 POINT(39.76877813356383 73.72459003046298) bank154115 +154116 POINT(39.99966657266957 74.54003583375798) bank154116 +154117 POINT(39.941638565685984 74.21503002036265) bank154117 +154118 POINT(40.88438182367397 73.45334982180088) bank154118 +154119 POINT(41.66647211574698 73.63910239419575) bank154119 +154120 POINT(40.133225449667044 73.3748837796764) bank154120 +154121 POINT(41.59255972656747 74.9663894780444) bank154121 +154122 POINT(41.636127033143566 74.12229128728153) bank154122 +154123 POINT(40.74110927768878 73.85905346277555) bank154123 +154124 POINT(41.27030031712342 74.53283563445031) bank154124 +154125 POINT(39.91120373017095 74.44712585606554) bank154125 +154126 POINT(40.610310126053136 73.27426724845587) bank154126 +154127 POINT(40.69245635859998 73.06414114698354) bank154127 +154128 POINT(41.26051706201665 74.38648027220242) bank154128 +154129 POINT(40.39416362618284 74.47042981150098) bank154129 +154130 POINT(41.15613117069607 73.28686837679321) bank154130 +154131 POINT(41.539985045386445 74.83759184412075) bank154131 +154132 POINT(41.01036800759525 74.21522473099066) bank154132 +154133 POINT(40.06429727995887 73.91020027410417) bank154133 +154134 POINT(41.16948174849197 73.47962567123156) bank154134 +154135 POINT(40.20520780809025 73.81696899340629) bank154135 +154136 POINT(41.12679875150478 73.6741567108515) bank154136 +154137 POINT(41.59541105462164 73.01395541744239) bank154137 +154138 POINT(40.17336076654383 74.71868912542637) bank154138 +154139 POINT(41.631257462139445 73.53328513449789) bank154139 +154140 POINT(40.94589142848721 74.86438531117938) bank154140 +154141 POINT(40.34858291731422 73.14192880985087) bank154141 +154142 POINT(39.85000586464958 73.99976504008349) bank154142 +154143 POINT(39.90950260836825 74.11589667554962) bank154143 +154144 POINT(40.63230892142875 73.09539397060854) bank154144 +154145 POINT(40.96751807443018 73.05697314925962) bank154145 +154146 POINT(40.930040532771734 73.90417295179498) bank154146 +154147 POINT(40.458322748217604 73.90813425689609) bank154147 +154148 POINT(40.03548604102733 73.03207279343376) bank154148 +154149 POINT(40.81644193144011 73.05097780862938) bank154149 +154150 POINT(41.30758712467508 74.0526759188446) bank154150 +154151 POINT(39.84616438954155 73.85135169444283) bank154151 +154152 POINT(41.41372400345239 73.74170647857531) bank154152 +154153 POINT(39.915308487142276 73.20865026877055) bank154153 +154154 POINT(39.82741941358869 73.93224832954034) bank154154 +154155 POINT(40.358348024774095 74.24430603865287) bank154155 +154156 POINT(40.7624672175154 73.93893156815413) bank154156 +154157 POINT(40.85899100866462 73.46627028600157) bank154157 +154158 POINT(39.88292480706447 74.16465373016909) bank154158 +154159 POINT(40.561267973890494 74.31959276042058) bank154159 +154160 POINT(41.38708746913253 73.06248536993829) bank154160 +154161 POINT(41.22618606222484 73.02038471287791) bank154161 +154162 POINT(40.27719753027998 74.20223690246826) bank154162 +154163 POINT(41.47201507020941 74.15273849746046) bank154163 +154164 POINT(41.07550139258891 73.30783840082135) bank154164 +154165 POINT(41.205721923446305 74.46156402633432) bank154165 +154166 POINT(40.17382715087091 74.82003091296583) bank154166 +154167 POINT(41.118654312806186 74.03752081421383) bank154167 +154168 POINT(41.03755362802329 74.596436913359) bank154168 +154169 POINT(40.84821635117301 74.81431746285236) bank154169 +154170 POINT(41.382203872543656 73.75535409666904) bank154170 +154171 POINT(40.997482976083354 73.07508132468566) bank154171 +154172 POINT(40.37559363559494 74.70142066610592) bank154172 +154173 POINT(41.30629838510809 73.68470912857858) bank154173 +154174 POINT(40.5862957065725 73.41401908271878) bank154174 +154175 POINT(41.0834273818485 73.55558711279535) bank154175 +154176 POINT(41.28493573544998 75.00447741609682) bank154176 +154177 POINT(41.28057704356606 73.87514839680375) bank154177 +154178 POINT(39.736468168594456 73.86215085620931) bank154178 +154179 POINT(41.309715681979526 73.10771683372033) bank154179 +154180 POINT(40.74961905260259 73.153840658272) bank154180 +154181 POINT(41.39517121719862 73.5482395328528) bank154181 +154182 POINT(40.89243773553512 73.8123116526234) bank154182 +154183 POINT(40.86821956461012 73.37787028166387) bank154183 +154184 POINT(41.615980183858866 74.95306654251631) bank154184 +154185 POINT(41.15193505645991 73.52929104797227) bank154185 +154186 POINT(40.7093470559884 74.29817232141487) bank154186 +154187 POINT(41.45563038102777 74.23596698815425) bank154187 +154188 POINT(40.31647045244028 74.74343255800905) bank154188 +154189 POINT(40.4531861007316 73.50473793596296) bank154189 +154190 POINT(39.79142127778235 73.27268740112812) bank154190 +154191 POINT(41.33165785116084 74.92164987969794) bank154191 +154192 POINT(40.54402146196968 74.81262341994956) bank154192 +154193 POINT(41.27975669154908 73.7242507727027) bank154193 +154194 POINT(39.75593063050938 73.40954776755386) bank154194 +154195 POINT(40.18912148094061 74.20423795417106) bank154195 +154196 POINT(40.65864272995925 73.07404766141241) bank154196 +154197 POINT(40.17586584711468 74.2745557380271) bank154197 +154198 POINT(40.95016567160708 74.99108436411264) bank154198 +154199 POINT(41.13327549104578 74.21912330155243) bank154199 +154200 POINT(41.46289687512005 74.40009659725995) bank154200 +154201 POINT(40.07868162595807 74.21983781208976) bank154201 +154202 POINT(40.69818917251907 73.37347657952833) bank154202 +154203 POINT(40.814868648709876 74.37564788409288) bank154203 +154204 POINT(40.03109110010764 74.01095707794995) bank154204 +154205 POINT(40.87090165524713 73.81356739538504) bank154205 +154206 POINT(40.31638568823706 73.16173193324309) bank154206 +154207 POINT(40.60261502279476 73.72760977580947) bank154207 +154208 POINT(40.462760667453125 73.23912122544519) bank154208 +154209 POINT(39.98089437091553 73.94896995505033) bank154209 +154210 POINT(40.675328030587984 73.41998193274053) bank154210 +154211 POINT(40.18399542258056 73.38250869647439) bank154211 +154212 POINT(41.15664582020593 74.9752338243353) bank154212 +154213 POINT(41.08890224619435 73.52184605177659) bank154213 +154214 POINT(40.039193103004074 73.84645501075286) bank154214 +154215 POINT(41.41640549632165 74.71776078154687) bank154215 +154216 POINT(40.61175131597704 73.16882663615961) bank154216 +154217 POINT(41.0537590104447 74.3904452768123) bank154217 +154218 POINT(40.33779769651499 73.05405677676666) bank154218 +154219 POINT(41.15388012788936 74.72672065561264) bank154219 +154220 POINT(40.46098213891103 74.25133337782128) bank154220 +154221 POINT(40.07747030786561 74.24024887944731) bank154221 +154222 POINT(41.03418240935367 74.20033368348011) bank154222 +154223 POINT(40.31121266702374 73.36047891150136) bank154223 +154224 POINT(39.9864967068437 73.98443939891756) bank154224 +154225 POINT(40.144197107370516 73.16950052369211) bank154225 +154226 POINT(41.514982156001274 73.50544238902437) bank154226 +154227 POINT(41.6901884281643 74.66618474174476) bank154227 +154228 POINT(40.70644364379955 73.77887183885719) bank154228 +154229 POINT(39.8449595921611 73.071211828331) bank154229 +154230 POINT(40.01459109185707 74.90518947958604) bank154230 +154231 POINT(41.60356406318205 74.97836133785239) bank154231 +154232 POINT(41.376064422231295 73.27233982458355) bank154232 +154233 POINT(40.10709566395405 73.13265289738308) bank154233 +154234 POINT(40.31922075211428 74.27053083917579) bank154234 +154235 POINT(40.3195334614002 74.07676616486789) bank154235 +154236 POINT(40.582704133025146 73.72362269753451) bank154236 +154237 POINT(39.9372698393491 74.97797004166581) bank154237 +154238 POINT(40.07126219094805 73.09251345397104) bank154238 +154239 POINT(40.54109140610542 73.54846727882675) bank154239 +154240 POINT(41.444394378661286 74.94254471765504) bank154240 +154241 POINT(41.54018618500652 74.89674068158781) bank154241 +154242 POINT(40.392170710061386 73.33552944142075) bank154242 +154243 POINT(40.2828219416489 73.87266041723015) bank154243 +154244 POINT(41.21574162821544 74.60564849168178) bank154244 +154245 POINT(40.57650790580417 73.50366293439667) bank154245 +154246 POINT(40.372506871387294 73.45814044669797) bank154246 +154247 POINT(40.938994761705516 73.33952981671224) bank154247 +154248 POINT(40.06241557646395 73.75283769794804) bank154248 +154249 POINT(40.41896116171769 73.02085872073573) bank154249 +154250 POINT(40.98220357212476 74.82528271752393) bank154250 +154251 POINT(40.00370539219606 74.22851966052181) bank154251 +154252 POINT(39.981989985315785 73.6208162495046) bank154252 +154253 POINT(40.7702414998839 73.43599932186665) bank154253 +154254 POINT(40.36634336102502 74.3405957233589) bank154254 +154255 POINT(40.8301981811059 73.75678149998686) bank154255 +154256 POINT(40.73067634649323 73.4221613894115) bank154256 +154257 POINT(41.20784594956364 74.49687439092234) bank154257 +154258 POINT(40.64409116022898 74.23480257224885) bank154258 +154259 POINT(39.97600547633145 73.42451857266862) bank154259 +154260 POINT(41.258716259619845 73.48537356802186) bank154260 +154261 POINT(40.34371877199003 73.25234342902291) bank154261 +154262 POINT(40.73757427400217 74.18766445456868) bank154262 +154263 POINT(39.74426979552562 74.9974228539598) bank154263 +154264 POINT(40.823995311026984 74.35946011072461) bank154264 +154265 POINT(41.427993309802815 73.23505479064447) bank154265 +154266 POINT(41.38139072136223 74.60594322483861) bank154266 +154267 POINT(40.97048218244099 73.49074233193186) bank154267 +154268 POINT(40.51191142915327 73.44226028913174) bank154268 +154269 POINT(39.78087926072319 74.18897188372074) bank154269 +154270 POINT(40.553128468276824 73.9147811683333) bank154270 +154271 POINT(39.782398427036085 74.77569533094021) bank154271 +154272 POINT(40.65972559539552 73.32719951612677) bank154272 +154273 POINT(41.028437220899626 74.86744172173223) bank154273 +154274 POINT(40.64374997749082 74.13569324765787) bank154274 +154275 POINT(41.08540979073685 73.7447769807512) bank154275 +154276 POINT(41.53383663879747 74.1143631083245) bank154276 +154277 POINT(39.95195708513299 73.88251882281628) bank154277 +154278 POINT(40.66841917164495 74.28167131298895) bank154278 +154279 POINT(39.997459437092765 74.75307531234573) bank154279 +154280 POINT(40.12801154205822 73.0488868608798) bank154280 +154281 POINT(41.5173380312559 73.79817643151038) bank154281 +154282 POINT(40.45048859575877 73.27874871819887) bank154282 +154283 POINT(40.90578978845341 74.54875998211227) bank154283 +154284 POINT(40.7450422158366 74.18712447824196) bank154284 +154285 POINT(39.94490740971833 73.87709344331189) bank154285 +154286 POINT(39.81878727913285 73.48354162433989) bank154286 +154287 POINT(40.81434744318331 74.29746974198356) bank154287 +154288 POINT(40.628330192538826 73.6623448497949) bank154288 +154289 POINT(40.73537054879358 73.10595415967336) bank154289 +154290 POINT(40.09743850728846 74.62851726336923) bank154290 +154291 POINT(40.445594337528156 74.6840056347619) bank154291 +154292 POINT(41.20905985797105 73.0773295576394) bank154292 +154293 POINT(39.71981918445988 73.73507698143666) bank154293 +154294 POINT(41.63326319055401 74.9574785706749) bank154294 +154295 POINT(40.09867005930415 74.4068141853358) bank154295 +154296 POINT(41.55514212384345 74.58890467087889) bank154296 +154297 POINT(41.34373786040414 74.99115025982485) bank154297 +154298 POINT(41.6742774388501 74.74598708575256) bank154298 +154299 POINT(40.10091554133054 73.40732088554795) bank154299 +154300 POINT(39.866046442747795 74.16077857955804) bank154300 +154301 POINT(40.195033707968925 74.5216687189606) bank154301 +154302 POINT(41.17360458962296 73.72488001258557) bank154302 +154303 POINT(41.32980134217603 73.79501800612566) bank154303 +154304 POINT(41.142111357464 74.02793840365293) bank154304 +154305 POINT(40.233697400570136 73.10991756591386) bank154305 +154306 POINT(41.57566464056627 74.13382376507619) bank154306 +154307 POINT(40.44804977051247 74.1747396842896) bank154307 +154308 POINT(41.44830622678352 74.88230775439699) bank154308 +154309 POINT(40.080912911218746 74.0773469230111) bank154309 +154310 POINT(40.05118737117074 74.59552099690335) bank154310 +154311 POINT(39.74833275343332 73.64948027623689) bank154311 +154312 POINT(41.26440329526046 74.7444537148718) bank154312 +154313 POINT(40.9829177222226 73.53115642268132) bank154313 +154314 POINT(40.44394461301668 73.03790130356829) bank154314 +154315 POINT(41.48587973797962 74.1832350863587) bank154315 +154316 POINT(41.19118995820539 74.61213303790711) bank154316 +154317 POINT(40.892418809053424 73.89090158409253) bank154317 +154318 POINT(40.16655410032371 73.92229213640213) bank154318 +154319 POINT(40.587577472893045 73.99676694052755) bank154319 +154320 POINT(39.81352673958574 73.5624577910511) bank154320 +154321 POINT(39.79884365416113 74.93594664225621) bank154321 +154322 POINT(40.57749030623304 74.16242642788242) bank154322 +154323 POINT(40.4412599012497 74.75012130805293) bank154323 +154324 POINT(40.31071604183741 73.07022362518993) bank154324 +154325 POINT(40.86203720723671 73.68338145728075) bank154325 +154326 POINT(40.61063253849393 73.42879492917145) bank154326 +154327 POINT(40.99226866582331 74.39058995098804) bank154327 +154328 POINT(41.64449756764679 73.60727502524448) bank154328 +154329 POINT(40.71243618657027 74.3737290847266) bank154329 +154330 POINT(41.473311498353695 73.02381615921914) bank154330 +154331 POINT(41.46226608599772 74.7650914299695) bank154331 +154332 POINT(40.34739774407583 73.9521755674071) bank154332 +154333 POINT(39.98244063395415 74.43169329763626) bank154333 +154334 POINT(40.89660991398928 75.00550636000347) bank154334 +154335 POINT(40.7178029963884 73.4178639828282) bank154335 +154336 POINT(40.312469614526776 74.8144786912452) bank154336 +154337 POINT(41.46612436756486 73.47927138092636) bank154337 +154338 POINT(39.858827388785514 73.01161752916435) bank154338 +154339 POINT(41.149191920799595 73.29042953568549) bank154339 +154340 POINT(40.33232772924793 74.18057816119236) bank154340 +154341 POINT(40.246605650754155 73.31236277637956) bank154341 +154342 POINT(41.43451520508568 73.03402886725495) bank154342 +154343 POINT(41.64447813646609 73.74101689270134) bank154343 +154344 POINT(40.396265510163985 74.63521804907916) bank154344 +154345 POINT(40.1198478500306 74.70802574274234) bank154345 +154346 POINT(41.661407938631314 74.36082521101119) bank154346 +154347 POINT(40.923027770064515 73.95613398572388) bank154347 +154348 POINT(39.83530090005078 73.14320825725268) bank154348 +154349 POINT(41.52082972434492 73.53705637423967) bank154349 +154350 POINT(40.796232390914874 74.78310251344027) bank154350 +154351 POINT(39.82842204378907 73.56654407968459) bank154351 +154352 POINT(40.57923736444066 73.49427360113694) bank154352 +154353 POINT(40.26461565829645 74.30229578291829) bank154353 +154354 POINT(40.35269221879971 73.4830497591199) bank154354 +154355 POINT(39.814178395088994 73.3525703255082) bank154355 +154356 POINT(41.70652839638588 73.23026653432127) bank154356 +154357 POINT(40.02478717761 74.51223395905191) bank154357 +154358 POINT(41.688960389550935 74.34612209791219) bank154358 +154359 POINT(40.26762634078154 73.91080131130613) bank154359 +154360 POINT(41.448196797419676 73.18121588472243) bank154360 +154361 POINT(41.31433825204233 74.61900712867343) bank154361 +154362 POINT(41.32739235172146 73.64645103999496) bank154362 +154363 POINT(39.982906080303636 73.23689434823339) bank154363 +154364 POINT(39.91998984542356 73.56174699857577) bank154364 +154365 POINT(40.10240292823096 73.29468752013216) bank154365 +154366 POINT(41.49909133552588 74.21071506586739) bank154366 +154367 POINT(41.50011094764007 73.75493312989856) bank154367 +154368 POINT(40.38053154553102 74.13183761500974) bank154368 +154369 POINT(41.22648197328637 74.80454651700789) bank154369 +154370 POINT(40.7551702020135 74.0447611768465) bank154370 +154371 POINT(40.29975534028038 74.26129480472282) bank154371 +154372 POINT(41.48915174902753 73.75128409272192) bank154372 +154373 POINT(41.129572954184184 74.44298977937946) bank154373 +154374 POINT(39.8534332346539 73.20162594777285) bank154374 +154375 POINT(41.271971609555635 73.09719555911528) bank154375 +154376 POINT(40.86507880532528 73.40647731408502) bank154376 +154377 POINT(40.01229304581948 74.7629405361634) bank154377 +154378 POINT(40.956095973464 74.95283063877466) bank154378 +154379 POINT(40.07461285118265 73.91126998382323) bank154379 +154380 POINT(40.13831099725462 74.87063785099758) bank154380 +154381 POINT(41.218610412743914 73.71509592063502) bank154381 +154382 POINT(39.93420944525385 74.23820364968887) bank154382 +154383 POINT(39.72022318914703 73.13102838211701) bank154383 +154384 POINT(41.21319213915255 73.74454868434526) bank154384 +154385 POINT(41.53098972549125 74.5084820495813) bank154385 +154386 POINT(40.456228629811854 74.30680937688562) bank154386 +154387 POINT(39.86447876207155 74.70337425844792) bank154387 +154388 POINT(41.501377742093005 73.86235100148468) bank154388 +154389 POINT(39.95866612823988 74.8032409707815) bank154389 +154390 POINT(40.30090190204356 73.42155673441518) bank154390 +154391 POINT(41.532957466043115 74.57083661358992) bank154391 +154392 POINT(41.019661133801904 73.37623176874789) bank154392 +154393 POINT(41.315979679559355 73.91847750914648) bank154393 +154394 POINT(39.93184373397794 74.46190133110684) bank154394 +154395 POINT(40.95393363855421 74.39721571046337) bank154395 +154396 POINT(40.643271559766575 74.92744860253741) bank154396 +154397 POINT(41.25390008200555 74.59530491880483) bank154397 +154398 POINT(41.57569373333338 73.15088609293103) bank154398 +154399 POINT(41.52246808966299 74.13956829913032) bank154399 +154400 POINT(41.525475935632535 73.22171580807446) bank154400 +154401 POINT(39.80701996805116 73.43343148349929) bank154401 +154402 POINT(40.617607127337074 74.85076574943729) bank154402 +154403 POINT(40.27668330385094 73.51398271643185) bank154403 +154404 POINT(40.13952267735259 74.5037461300691) bank154404 +154405 POINT(39.816485719276166 73.64377613206968) bank154405 +154406 POINT(40.68308210848698 74.10257696632944) bank154406 +154407 POINT(40.40414724610139 74.79102316592055) bank154407 +154408 POINT(40.55639112253298 74.64815951035564) bank154408 +154409 POINT(41.34211780656402 73.24619550870344) bank154409 +154410 POINT(40.231814264570204 73.16264602559917) bank154410 +154411 POINT(40.74181190535648 74.29150533008537) bank154411 +154412 POINT(40.95050241061893 73.91451297596286) bank154412 +154413 POINT(39.952068212389094 74.62616848217056) bank154413 +154414 POINT(40.36159085470946 74.08081682492521) bank154414 +154415 POINT(41.448024356867165 73.1525268977856) bank154415 +154416 POINT(40.78538539184113 73.26795430074002) bank154416 +154417 POINT(41.41391538260739 74.41971464759925) bank154417 +154418 POINT(41.54236366552244 73.5430959584558) bank154418 +154419 POINT(41.39577750771924 73.37386362769955) bank154419 +154420 POINT(40.68449482091463 73.29870046524327) bank154420 +154421 POINT(40.033334276284265 74.79382691149027) bank154421 +154422 POINT(40.63615604791053 74.94800379031966) bank154422 +154423 POINT(40.52057180758601 73.96977014427748) bank154423 +154424 POINT(41.09632762765997 73.6360493958617) bank154424 +154425 POINT(39.815516113186575 74.37238948243072) bank154425 +154426 POINT(40.5534805483553 74.32933230329472) bank154426 +154427 POINT(41.31273980489466 74.53296248328252) bank154427 +154428 POINT(40.17090760744463 74.53296219655014) bank154428 +154429 POINT(40.286835675610696 74.24228156141697) bank154429 +154430 POINT(40.95348089802546 73.0880212564205) bank154430 +154431 POINT(40.52348403604127 73.68757682846143) bank154431 +154432 POINT(40.4297547095989 74.50444836804492) bank154432 +154433 POINT(41.40443693481092 74.38210941793538) bank154433 +154434 POINT(41.09643712394723 74.1232932576831) bank154434 +154435 POINT(39.744977606057205 73.91494222172314) bank154435 +154436 POINT(41.445887274791936 73.49006061282665) bank154436 +154437 POINT(41.11209944902124 73.81763491312246) bank154437 +154438 POINT(40.477151869777096 73.29926207863681) bank154438 +154439 POINT(41.08876495574307 74.92227183962724) bank154439 +154440 POINT(41.47575446353966 73.19532933869603) bank154440 +154441 POINT(40.94991236512329 73.2887740026821) bank154441 +154442 POINT(40.50075043206461 73.82834417033176) bank154442 +154443 POINT(40.43103730603967 74.07000313576368) bank154443 +154444 POINT(40.709769731812244 73.80601096458672) bank154444 +154445 POINT(40.62277792011555 74.9299635852659) bank154445 +154446 POINT(40.717357089084416 73.92780124149) bank154446 +154447 POINT(40.815977927236695 74.87446790779201) bank154447 +154448 POINT(40.515342949207344 73.28300964838215) bank154448 +154449 POINT(40.537112537683605 74.81755950311347) bank154449 +154450 POINT(40.49082550735031 73.78120280369879) bank154450 +154451 POINT(40.07593489265089 74.52538981221265) bank154451 +154452 POINT(40.76574321217825 73.355664717113) bank154452 +154453 POINT(40.875382941023695 73.62057937668192) bank154453 +154454 POINT(41.60387462106115 74.28610252434072) bank154454 +154455 POINT(41.20447033013579 73.29720175157024) bank154455 +154456 POINT(40.376675772532906 74.30942255502889) bank154456 +154457 POINT(41.57759050014734 73.35607892128255) bank154457 +154458 POINT(41.307006676007056 74.18751135483186) bank154458 +154459 POINT(40.87348751435104 74.78366520727205) bank154459 +154460 POINT(40.07634115577912 73.02211600310285) bank154460 +154461 POINT(39.836800955153755 74.05854673064714) bank154461 +154462 POINT(40.658122177181085 74.69925517724111) bank154462 +154463 POINT(41.571999819533644 73.35296590672304) bank154463 +154464 POINT(41.394111402619934 74.96928505445588) bank154464 +154465 POINT(40.181943522500674 73.28818324810545) bank154465 +154466 POINT(41.58937428108515 74.75700902117141) bank154466 +154467 POINT(40.6899765993383 73.91633732562622) bank154467 +154468 POINT(41.50413872420927 74.81794681166353) bank154468 +154469 POINT(40.486036709916846 73.4460607702433) bank154469 +154470 POINT(41.527093337016815 74.29087382655354) bank154470 +154471 POINT(41.135284677528084 74.57710806173475) bank154471 +154472 POINT(40.94532609798483 73.92238152464806) bank154472 +154473 POINT(40.13170253049867 74.85211276580833) bank154473 +154474 POINT(41.56684433328428 73.55478808455386) bank154474 +154475 POINT(40.48747932074097 73.78630365693769) bank154475 +154476 POINT(41.00683231272124 74.96850720230519) bank154476 +154477 POINT(40.77454910383854 74.86135955283288) bank154477 +154478 POINT(40.70642154949638 73.97530863472927) bank154478 +154479 POINT(39.90242432760004 74.54288274980406) bank154479 +154480 POINT(40.09892858191092 73.72954565297258) bank154480 +154481 POINT(41.220516944846025 74.40122102700091) bank154481 +154482 POINT(40.84720988330529 73.6748369339972) bank154482 +154483 POINT(41.58711124275945 73.95419901419464) bank154483 +154484 POINT(39.81868489647342 74.62929191852626) bank154484 +154485 POINT(41.121389608119756 74.69996243499912) bank154485 +154486 POINT(40.07282595618729 74.50304397529018) bank154486 +154487 POINT(40.618364750170564 73.35931722466572) bank154487 +154488 POINT(40.198785339609614 73.24238822264762) bank154488 +154489 POINT(41.07397940486357 74.31726233488224) bank154489 +154490 POINT(40.61481064339197 74.50861038743948) bank154490 +154491 POINT(40.00587780981464 74.13456366267134) bank154491 +154492 POINT(40.40120992374918 74.1304724682571) bank154492 +154493 POINT(40.63856237700906 74.9790095002625) bank154493 +154494 POINT(39.86249068833941 74.94545431043402) bank154494 +154495 POINT(40.7511864761842 74.3568093005651) bank154495 +154496 POINT(39.98914948997544 74.46165646090174) bank154496 +154497 POINT(41.688890616702395 74.63667286071852) bank154497 +154498 POINT(40.093793485423774 73.36015859004712) bank154498 +154499 POINT(40.241009606248156 74.81704415704898) bank154499 +154500 POINT(41.569816778163315 74.56778820995773) bank154500 +154501 POINT(41.55046151361217 73.5683068251532) bank154501 +154502 POINT(41.23208646507324 74.24595313082904) bank154502 +154503 POINT(40.30364578501262 74.99788493626374) bank154503 +154504 POINT(40.580856123986436 73.06476371450265) bank154504 +154505 POINT(40.744606238591004 74.75703648559765) bank154505 +154506 POINT(41.428368060267054 73.14914302852574) bank154506 +154507 POINT(41.05947209851919 73.45793412093212) bank154507 +154508 POINT(40.30020868495003 73.98195158095787) bank154508 +154509 POINT(41.03290176147641 74.35797966363519) bank154509 +154510 POINT(41.691247851594746 74.02519724665511) bank154510 +154511 POINT(40.728323317682865 73.55173702200427) bank154511 +154512 POINT(40.07415107493618 74.10078629550266) bank154512 +154513 POINT(40.05756403599491 74.47374387202967) bank154513 +154514 POINT(41.249760664220716 73.6445045461057) bank154514 +154515 POINT(41.58274381753764 73.33592926300452) bank154515 +154516 POINT(41.04900030079555 73.2851199285133) bank154516 +154517 POINT(41.29687511406313 73.75944088319032) bank154517 +154518 POINT(40.84078215502339 74.52300366767496) bank154518 +154519 POINT(41.37519691007695 73.48744024693897) bank154519 +154520 POINT(41.66274419489014 73.83812241809626) bank154520 +154521 POINT(40.160916069736146 74.54693036522232) bank154521 +154522 POINT(39.8762740230046 73.53095188484446) bank154522 +154523 POINT(40.641296210524345 74.07534489119898) bank154523 +154524 POINT(41.27125431400385 73.42233655388621) bank154524 +154525 POINT(39.902225092966646 74.37206173089567) bank154525 +154526 POINT(40.64691147243493 74.73396990268728) bank154526 +154527 POINT(40.935586672243154 73.62536053739109) bank154527 +154528 POINT(40.18137726340089 73.07259043565682) bank154528 +154529 POINT(41.22442629412427 73.95929927051533) bank154529 +154530 POINT(39.76081531379662 74.20041819641229) bank154530 +154531 POINT(40.92401598491304 74.34213288258556) bank154531 +154532 POINT(40.38367016250872 73.50790167141655) bank154532 +154533 POINT(41.61729386304195 73.02494094456902) bank154533 +154534 POINT(41.48035451418605 73.23582761906262) bank154534 +154535 POINT(40.070865572154645 73.89880745305528) bank154535 +154536 POINT(41.616913316116175 74.76065143716042) bank154536 +154537 POINT(41.17745660186717 73.07718086415098) bank154537 +154538 POINT(41.603050785475894 73.99609888307933) bank154538 +154539 POINT(39.99858518303083 74.32655411629871) bank154539 +154540 POINT(40.86645439800529 74.12612987532846) bank154540 +154541 POINT(40.726348351942384 73.89083023621761) bank154541 +154542 POINT(40.7588978584261 74.78891374940822) bank154542 +154543 POINT(40.42121321616094 73.48578195617195) bank154543 +154544 POINT(40.639153257625246 73.37254392568293) bank154544 +154545 POINT(40.77565362389279 73.8214646033528) bank154545 +154546 POINT(40.847594002545904 73.96791878712486) bank154546 +154547 POINT(40.928488278469366 73.16733715031744) bank154547 +154548 POINT(40.0605809444926 74.83914642329299) bank154548 +154549 POINT(40.923504837140555 73.3305966160821) bank154549 +154550 POINT(41.54646377247364 74.0365220200686) bank154550 +154551 POINT(40.348962309991514 74.31750995322807) bank154551 +154552 POINT(40.970353916515755 74.80688689672213) bank154552 +154553 POINT(40.04317204222219 73.87465702807908) bank154553 +154554 POINT(40.63227634016739 73.51016797436729) bank154554 +154555 POINT(39.732660547510406 73.77606718270921) bank154555 +154556 POINT(40.605173436951475 73.93772640950971) bank154556 +154557 POINT(40.23783963849595 74.5938604673216) bank154557 +154558 POINT(40.29709684673784 73.97519309648273) bank154558 +154559 POINT(41.541045435561585 74.70300700587296) bank154559 +154560 POINT(39.967210393555604 74.92313698874177) bank154560 +154561 POINT(41.25636504353125 73.34088493238737) bank154561 +154562 POINT(41.00121843274042 73.9453922086879) bank154562 +154563 POINT(39.87402989539388 74.11309693467018) bank154563 +154564 POINT(41.0316077161124 74.7808885567397) bank154564 +154565 POINT(40.04754573928229 73.46007007022894) bank154565 +154566 POINT(41.304588062789584 74.62593830537976) bank154566 +154567 POINT(40.69415360212356 74.75394226034312) bank154567 +154568 POINT(40.37104579994466 74.31919892251778) bank154568 +154569 POINT(41.54905130779235 73.44717944870455) bank154569 +154570 POINT(41.062844933710196 74.94961199991901) bank154570 +154571 POINT(40.2308044201602 74.48575964238199) bank154571 +154572 POINT(40.63004209517512 73.16186083705796) bank154572 +154573 POINT(41.61419880780431 74.41782963033005) bank154573 +154574 POINT(40.918887435849214 73.28173753294793) bank154574 +154575 POINT(40.08654369959802 74.35463349973331) bank154575 +154576 POINT(40.997729436583654 73.78466178738493) bank154576 +154577 POINT(41.38809719682486 74.92488153132356) bank154577 +154578 POINT(41.414455382182524 74.3299416428951) bank154578 +154579 POINT(40.665484374344615 73.9262722024858) bank154579 +154580 POINT(40.19540214672399 74.97951287143927) bank154580 +154581 POINT(41.67068352689572 73.07849524619779) bank154581 +154582 POINT(40.93674225051996 73.16897259175606) bank154582 +154583 POINT(40.18683425274398 73.80559166999915) bank154583 +154584 POINT(41.45381871764329 73.85420417760494) bank154584 +154585 POINT(39.92616445530981 73.55316779825134) bank154585 +154586 POINT(39.830903180618975 74.89102642000113) bank154586 +154587 POINT(40.62680478111477 73.1561036813166) bank154587 +154588 POINT(40.74094452196798 74.17528649598626) bank154588 +154589 POINT(39.99588111359062 73.25887861397932) bank154589 +154590 POINT(39.96460260386418 73.30742127432458) bank154590 +154591 POINT(40.88088750280554 74.76008127348832) bank154591 +154592 POINT(40.03683123694461 74.30509152688744) bank154592 +154593 POINT(40.02180998316891 73.31666298849056) bank154593 +154594 POINT(40.9780159835177 73.73463141528694) bank154594 +154595 POINT(39.87735708847518 74.05089820910253) bank154595 +154596 POINT(41.605053208209135 74.29489040183239) bank154596 +154597 POINT(41.33139400701933 73.02042021384246) bank154597 +154598 POINT(41.16977221179587 73.67238122963954) bank154598 +154599 POINT(41.537254857208495 73.53005085206202) bank154599 +154600 POINT(40.68348847426534 74.7720217500806) bank154600 +154601 POINT(40.731495646168746 73.52603875926191) bank154601 +154602 POINT(40.17708700168006 74.89470711941819) bank154602 +154603 POINT(41.64908821563412 74.22242856538571) bank154603 +154604 POINT(40.74499896739089 74.18184708287845) bank154604 +154605 POINT(40.88348800150002 74.57764008196345) bank154605 +154606 POINT(40.92370516622769 74.99969226548482) bank154606 +154607 POINT(40.053155224079106 74.59975117816207) bank154607 +154608 POINT(40.313509905193094 74.70099950841578) bank154608 +154609 POINT(40.693220378691635 74.17828390933536) bank154609 +154610 POINT(40.042396667094664 73.79864155844092) bank154610 +154611 POINT(40.020618494399784 75.00456378948535) bank154611 +154612 POINT(39.99510119488065 73.73926780732175) bank154612 +154613 POINT(40.031186255651136 73.41667293981308) bank154613 +154614 POINT(40.854428463875074 74.39202678990878) bank154614 +154615 POINT(41.56298878358306 73.13691806622633) bank154615 +154616 POINT(40.13958192123672 74.61478154180458) bank154616 +154617 POINT(41.52335615985423 73.94229957721673) bank154617 +154618 POINT(41.069758824664504 73.225762846901) bank154618 +154619 POINT(40.39373907613042 74.01214123465971) bank154619 +154620 POINT(40.457449182799095 74.88944879944115) bank154620 +154621 POINT(40.67331799888634 73.69038354682851) bank154621 +154622 POINT(41.177767031515764 73.18984356860736) bank154622 +154623 POINT(40.029251379404634 74.91727005129847) bank154623 +154624 POINT(40.702488250027805 74.6427470854424) bank154624 +154625 POINT(40.5482241786054 74.9563837446458) bank154625 +154626 POINT(40.16009156757886 74.84076500427723) bank154626 +154627 POINT(41.42256247502703 73.59572123752503) bank154627 +154628 POINT(40.299001719237545 74.49305174256143) bank154628 +154629 POINT(41.143028448607986 73.62594475147063) bank154629 +154630 POINT(40.69814792038373 73.06932940428773) bank154630 +154631 POINT(41.61054821071218 74.71475069930123) bank154631 +154632 POINT(41.642676542570314 74.96097054211664) bank154632 +154633 POINT(40.75730814095528 74.37938615993589) bank154633 +154634 POINT(41.313446440335895 74.09677296048285) bank154634 +154635 POINT(39.794062737362296 74.47838824663407) bank154635 +154636 POINT(40.08964001531381 74.80668529211131) bank154636 +154637 POINT(41.12765769604839 74.50621903408613) bank154637 +154638 POINT(39.843562514409484 73.47296950720462) bank154638 +154639 POINT(40.419233468323085 73.57010472426279) bank154639 +154640 POINT(40.79208446192358 73.75187174563486) bank154640 +154641 POINT(39.746371193559504 73.76761424475335) bank154641 +154642 POINT(40.04525062835277 74.83147955512388) bank154642 +154643 POINT(41.5583181361141 74.08180547212615) bank154643 +154644 POINT(41.66283487331334 74.44803186666641) bank154644 +154645 POINT(40.125078879934925 74.18524149156919) bank154645 +154646 POINT(40.21996365610174 73.61227695166552) bank154646 +154647 POINT(40.727141132745956 74.44369242679149) bank154647 +154648 POINT(40.2207495181563 73.79584656406527) bank154648 +154649 POINT(41.50340771484893 74.0196376788908) bank154649 +154650 POINT(40.4906335925726 74.60591891944021) bank154650 +154651 POINT(41.66697936392315 74.40522627279586) bank154651 +154652 POINT(40.476669972464556 74.01866635911426) bank154652 +154653 POINT(40.212675971063625 74.70961926052989) bank154653 +154654 POINT(39.924008252080085 74.26488730210554) bank154654 +154655 POINT(41.07700365632052 73.44259412935604) bank154655 +154656 POINT(41.15231541669247 74.14867442365627) bank154656 +154657 POINT(39.85490704084416 73.5915738688643) bank154657 +154658 POINT(40.30320226820239 73.42580854244363) bank154658 +154659 POINT(40.81528273386883 74.72076384178759) bank154659 +154660 POINT(41.638574672946355 73.84903735393323) bank154660 +154661 POINT(40.95264213355793 73.5524222410614) bank154661 +154662 POINT(40.127234297155155 73.43104702257536) bank154662 +154663 POINT(41.544173135078154 73.03995877389487) bank154663 +154664 POINT(39.810646973662145 73.15350641616607) bank154664 +154665 POINT(40.64772628694054 74.83707700788274) bank154665 +154666 POINT(39.76545502429891 73.37538071142592) bank154666 +154667 POINT(40.032391798601566 73.92926005412892) bank154667 +154668 POINT(40.8041500847501 74.84028625154001) bank154668 +154669 POINT(41.30600763857689 73.46269194673087) bank154669 +154670 POINT(40.36588761081166 73.10679273034802) bank154670 +154671 POINT(41.64995651894088 73.11565545888608) bank154671 +154672 POINT(41.48097929586463 73.54408100359285) bank154672 +154673 POINT(41.330636047147934 74.88358901367975) bank154673 +154674 POINT(40.64158180963279 74.7738279377174) bank154674 +154675 POINT(39.91375681717117 74.99126470089763) bank154675 +154676 POINT(40.24118752980217 74.93942396665511) bank154676 +154677 POINT(40.50380051681709 74.93786001688997) bank154677 +154678 POINT(41.57596719109541 74.47683006176894) bank154678 +154679 POINT(41.55238504745751 73.29441016465056) bank154679 +154680 POINT(40.53592092057887 74.31298748419547) bank154680 +154681 POINT(40.12484340577406 74.31276624311555) bank154681 +154682 POINT(41.65678951091861 74.37146381250781) bank154682 +154683 POINT(40.093574543279736 74.85707628234108) bank154683 +154684 POINT(40.4278103066605 73.65590512019863) bank154684 +154685 POINT(41.106402320556256 74.40452964734318) bank154685 +154686 POINT(41.17667590191946 73.39885611628708) bank154686 +154687 POINT(40.13756633645565 73.56162087864257) bank154687 +154688 POINT(40.80080817115099 74.0595892899548) bank154688 +154689 POINT(39.984533213785184 73.07545523443011) bank154689 +154690 POINT(41.45700640286614 73.19422293857475) bank154690 +154691 POINT(40.4670415782712 73.11824082522261) bank154691 +154692 POINT(40.491923335968345 73.8878987127175) bank154692 +154693 POINT(40.723050821514974 73.88761969273547) bank154693 +154694 POINT(40.40602269636062 74.4598392447422) bank154694 +154695 POINT(41.48427018408014 74.26174676992595) bank154695 +154696 POINT(41.37285257932925 73.69445142148615) bank154696 +154697 POINT(39.802828021348695 74.91737086716384) bank154697 +154698 POINT(40.94943234611389 73.23570181498751) bank154698 +154699 POINT(41.00981544664529 74.497308374548) bank154699 +154700 POINT(41.34531970286644 74.30838372817857) bank154700 +154701 POINT(40.49518591127091 74.21835005998848) bank154701 +154702 POINT(40.63619907533845 74.25300408628054) bank154702 +154703 POINT(39.90556351236649 73.26616476460937) bank154703 +154704 POINT(41.18913933323891 74.03068450485695) bank154704 +154705 POINT(41.37743316748485 73.12371781740822) bank154705 +154706 POINT(40.69171762330222 73.25469474016491) bank154706 +154707 POINT(41.60339430550462 73.53252939681668) bank154707 +154708 POINT(40.256839329614444 74.60207388028286) bank154708 +154709 POINT(41.55287960656866 73.12596382652792) bank154709 +154710 POINT(41.00855155369008 74.46297978981539) bank154710 +154711 POINT(40.89199964800452 73.93346839373187) bank154711 +154712 POINT(41.526153050277784 74.0449694061052) bank154712 +154713 POINT(41.6310253961315 74.78502413339959) bank154713 +154714 POINT(39.746732611834695 74.32523250573762) bank154714 +154715 POINT(40.709299631040125 74.70779915768043) bank154715 +154716 POINT(40.42221964824163 74.91415622434081) bank154716 +154717 POINT(40.07009169029433 73.61008710600758) bank154717 +154718 POINT(40.711508227960636 74.48691227027368) bank154718 +154719 POINT(41.24129560971413 73.66082094421216) bank154719 +154720 POINT(40.472739019948655 73.63057471507253) bank154720 +154721 POINT(41.31865021594428 73.60213249364782) bank154721 +154722 POINT(40.517700552494354 73.38492397639698) bank154722 +154723 POINT(41.708364706441174 74.31776826479462) bank154723 +154724 POINT(41.54535466683047 74.28155786894268) bank154724 +154725 POINT(40.359670777626604 73.0826710598086) bank154725 +154726 POINT(41.52719121908926 74.45341031652104) bank154726 +154727 POINT(40.86861718049417 74.28573047948124) bank154727 +154728 POINT(39.727046185417514 73.43605371513222) bank154728 +154729 POINT(41.189843096858226 74.97131409595495) bank154729 +154730 POINT(40.17604317892809 73.7660408561762) bank154730 +154731 POINT(41.5080216807539 73.73475603139535) bank154731 +154732 POINT(40.50514846061164 73.94354870994003) bank154732 +154733 POINT(41.55771054460142 73.41580705551105) bank154733 +154734 POINT(40.75720199581289 73.69358726190463) bank154734 +154735 POINT(40.65763352857403 74.23912507672864) bank154735 +154736 POINT(40.487471058307925 74.49434127184924) bank154736 +154737 POINT(40.566717169986184 74.47752356957899) bank154737 +154738 POINT(39.91721938071964 73.27549422105004) bank154738 +154739 POINT(40.902576325514005 73.26129804916957) bank154739 +154740 POINT(40.06912198943213 73.0315907703303) bank154740 +154741 POINT(40.16069050562478 73.25119486151995) bank154741 +154742 POINT(41.05156064295742 73.23801049796752) bank154742 +154743 POINT(41.517849382376326 73.06186995185168) bank154743 +154744 POINT(39.916006071925096 73.38081682696912) bank154744 +154745 POINT(41.1432900114822 74.68843121873908) bank154745 +154746 POINT(40.624141299612596 74.19394676056051) bank154746 +154747 POINT(41.30251441794651 74.23083191491314) bank154747 +154748 POINT(41.353961349434606 74.22340702836782) bank154748 +154749 POINT(40.44760013963248 73.10949652697519) bank154749 +154750 POINT(39.81398990557618 73.47929126808134) bank154750 +154751 POINT(40.37949060312144 74.7468595364307) bank154751 +154752 POINT(41.317731446578776 74.68944667803493) bank154752 +154753 POINT(41.49265681681752 73.80795517453782) bank154753 +154754 POINT(41.049994794839066 74.60341611858817) bank154754 +154755 POINT(39.8523205785426 74.37771038067929) bank154755 +154756 POINT(39.874299158746595 74.63736735318633) bank154756 +154757 POINT(39.97819206068117 73.82920526518365) bank154757 +154758 POINT(41.07836691383285 74.76106019787683) bank154758 +154759 POINT(40.0397012520352 73.80649987280437) bank154759 +154760 POINT(41.388132067735484 73.70520618945396) bank154760 +154761 POINT(41.6457086922796 73.41487416759736) bank154761 +154762 POINT(41.183407779065746 73.99004047183666) bank154762 +154763 POINT(41.5858613280018 73.92182910138459) bank154763 +154764 POINT(41.67153618063176 73.62257942992139) bank154764 +154765 POINT(41.286137974286675 74.19683790485179) bank154765 +154766 POINT(40.74533289715134 73.11354220610332) bank154766 +154767 POINT(40.200503893743786 74.40227133673172) bank154767 +154768 POINT(40.060913958911705 74.85986552864092) bank154768 +154769 POINT(40.194636563719726 74.30378761737785) bank154769 +154770 POINT(41.477738681073106 73.97771824648767) bank154770 +154771 POINT(41.52835515704115 73.2260130109003) bank154771 +154772 POINT(41.525626651871775 73.05884904855253) bank154772 +154773 POINT(40.73823724067926 74.03632789735171) bank154773 +154774 POINT(41.18466846831861 73.8854574731174) bank154774 +154775 POINT(40.448699543006526 74.76405810153872) bank154775 +154776 POINT(40.305956494741196 73.67008204177455) bank154776 +154777 POINT(40.68703324475951 73.23028954017998) bank154777 +154778 POINT(40.35223304453729 74.66697443534689) bank154778 +154779 POINT(41.07890673212396 74.8602979050528) bank154779 +154780 POINT(40.94970904142361 74.05133418171899) bank154780 +154781 POINT(40.059574283883485 73.21894490545564) bank154781 +154782 POINT(39.750172827528004 73.65087139817496) bank154782 +154783 POINT(41.03188710293116 74.69359043269807) bank154783 +154784 POINT(40.5678870695871 73.81678838348596) bank154784 +154785 POINT(39.85673399485401 73.91167178293883) bank154785 +154786 POINT(41.5025600157274 74.49642292409233) bank154786 +154787 POINT(41.49345922498438 74.90237234975017) bank154787 +154788 POINT(41.041265561254754 74.5700590941724) bank154788 +154789 POINT(41.2409978988102 74.8325527509382) bank154789 +154790 POINT(41.24922223975559 74.54016751037436) bank154790 +154791 POINT(39.765982781476225 73.09609745225032) bank154791 +154792 POINT(41.219525546621675 74.16369891120931) bank154792 +154793 POINT(41.23097464382111 73.15650855054398) bank154793 +154794 POINT(41.228815225928415 73.08598938112743) bank154794 +154795 POINT(40.47046297705035 74.20181143078283) bank154795 +154796 POINT(40.713405808249895 73.87663633531645) bank154796 +154797 POINT(40.261980299137 74.28254351729512) bank154797 +154798 POINT(41.15383955566148 73.3509098368223) bank154798 +154799 POINT(40.64721606243497 73.14668040146356) bank154799 +154800 POINT(41.57976202189005 74.95781597451698) bank154800 +154801 POINT(40.0437113680741 74.35449782921273) bank154801 +154802 POINT(40.33962351413057 73.32652039339013) bank154802 +154803 POINT(39.78220511025226 73.67865889174898) bank154803 +154804 POINT(39.94499609014433 73.93457320914855) bank154804 +154805 POINT(41.049073005275176 74.28624723255277) bank154805 +154806 POINT(41.20056722637851 74.31936431158789) bank154806 +154807 POINT(41.32952220062946 73.09495247660224) bank154807 +154808 POINT(41.64122414608808 73.55194433438493) bank154808 +154809 POINT(40.99401586614383 73.81912361719542) bank154809 +154810 POINT(40.33530461449392 73.62909627633248) bank154810 +154811 POINT(41.1491557802701 73.63069888068154) bank154811 +154812 POINT(40.140274739184186 73.90403789210815) bank154812 +154813 POINT(41.45641657727849 73.76646555022594) bank154813 +154814 POINT(39.777706747785935 74.81393507669314) bank154814 +154815 POINT(40.676231464250066 74.26813957065434) bank154815 +154816 POINT(41.7057936329522 74.64126443602545) bank154816 +154817 POINT(41.16832239249264 73.44425312193079) bank154817 +154818 POINT(39.932389244054654 74.8939273129504) bank154818 +154819 POINT(41.10105693760578 73.39243549388294) bank154819 +154820 POINT(40.81668615231882 73.26710160427393) bank154820 +154821 POINT(40.51230495988488 74.65245419170958) bank154821 +154822 POINT(40.829445478260496 73.13666069172058) bank154822 +154823 POINT(41.43697131357767 74.44817190897878) bank154823 +154824 POINT(40.64058035109409 74.6516995749068) bank154824 +154825 POINT(41.11741550072708 74.93114635822853) bank154825 +154826 POINT(40.36093939235497 75.00235024970748) bank154826 +154827 POINT(41.03571536530388 73.72429304804307) bank154827 +154828 POINT(40.76021204456059 74.27643510443521) bank154828 +154829 POINT(41.171150549837314 73.23694106236724) bank154829 +154830 POINT(40.90949075338437 74.91230660752755) bank154830 +154831 POINT(41.39592529348925 73.6622230763771) bank154831 +154832 POINT(40.15177321673102 74.32452040029492) bank154832 +154833 POINT(40.23794360417436 74.19292502884063) bank154833 +154834 POINT(41.24235198561153 73.59940649309334) bank154834 +154835 POINT(41.682752828364606 74.50379764697212) bank154835 +154836 POINT(40.55088667919701 74.44345856941757) bank154836 +154837 POINT(40.04725482623584 74.54298122827677) bank154837 +154838 POINT(40.50680661532701 74.41827849043703) bank154838 +154839 POINT(40.69306277575988 74.28676789361536) bank154839 +154840 POINT(40.2582508403051 74.20157872852391) bank154840 +154841 POINT(40.663413249694166 73.22006060414871) bank154841 +154842 POINT(40.51067149536838 74.82749485288593) bank154842 +154843 POINT(41.30129750202635 74.61480115546009) bank154843 +154844 POINT(41.64287485449367 74.58145601061491) bank154844 +154845 POINT(41.61085398381556 74.42472660057197) bank154845 +154846 POINT(40.37472288889909 73.88242351590034) bank154846 +154847 POINT(39.94777574313183 74.35383198310235) bank154847 +154848 POINT(41.228567854610134 73.04672458584173) bank154848 +154849 POINT(41.4027877809417 73.06634177546874) bank154849 +154850 POINT(41.30545318098351 73.3834059073601) bank154850 +154851 POINT(41.05569796382329 73.97032841973314) bank154851 +154852 POINT(40.62477458876067 74.80756381228979) bank154852 +154853 POINT(40.05980021965282 73.53068228704053) bank154853 +154854 POINT(40.1345509262048 73.26797769594475) bank154854 +154855 POINT(40.714826792685535 74.58047627398066) bank154855 +154856 POINT(41.13815574259131 74.83950021249757) bank154856 +154857 POINT(40.013458160466215 73.48596100988117) bank154857 +154858 POINT(41.509350195055454 74.75125207584033) bank154858 +154859 POINT(40.41316782170938 73.97311082177696) bank154859 +154860 POINT(40.51673791537268 74.75476684664055) bank154860 +154861 POINT(40.94994656761415 73.24207372310316) bank154861 +154862 POINT(41.447844033978456 74.6609648962006) bank154862 +154863 POINT(40.95110637697486 73.53527973012511) bank154863 +154864 POINT(39.86863844300054 73.90750137693591) bank154864 +154865 POINT(39.852938189787004 73.54652253363953) bank154865 +154866 POINT(41.25545192346618 73.84096986130265) bank154866 +154867 POINT(40.85510000889818 74.1412206370138) bank154867 +154868 POINT(41.39133882818047 74.90922743120802) bank154868 +154869 POINT(40.16163928172095 74.31367489031956) bank154869 +154870 POINT(40.903817945795346 74.03664341113262) bank154870 +154871 POINT(39.75227885130359 74.58699491271881) bank154871 +154872 POINT(41.26331324490086 74.11986019048348) bank154872 +154873 POINT(41.30066940229423 74.68106208064634) bank154873 +154874 POINT(40.02870453778892 73.07986680957877) bank154874 +154875 POINT(41.12952074213603 73.6550172596115) bank154875 +154876 POINT(40.76140182529802 73.8377075210901) bank154876 +154877 POINT(39.98667595119631 74.53571367816272) bank154877 +154878 POINT(39.746553653807325 74.69267736133261) bank154878 +154879 POINT(40.32726128259029 73.16250791202692) bank154879 +154880 POINT(40.76975983516864 73.43008071115695) bank154880 +154881 POINT(41.61152554243291 73.89892638337322) bank154881 +154882 POINT(40.94903447561754 73.95455129454149) bank154882 +154883 POINT(39.94692069614615 73.79397634935191) bank154883 +154884 POINT(41.672843639221924 73.01231547011162) bank154884 +154885 POINT(39.81533295899164 74.56729183948005) bank154885 +154886 POINT(40.380634150506 73.08923895438602) bank154886 +154887 POINT(41.32720631260809 73.39081703815799) bank154887 +154888 POINT(40.79373060944433 73.07137606258404) bank154888 +154889 POINT(41.18176788615647 73.07476849199415) bank154889 +154890 POINT(40.33844957628017 73.09693033875894) bank154890 +154891 POINT(41.2496482145829 74.05388162494769) bank154891 +154892 POINT(40.095853471593436 74.1423593019467) bank154892 +154893 POINT(40.031140916616906 74.5181697397684) bank154893 +154894 POINT(40.173729683133594 73.99903802725578) bank154894 +154895 POINT(41.259147375215626 73.28539180334609) bank154895 +154896 POINT(40.43072184616482 73.90980257157581) bank154896 +154897 POINT(41.16820180104375 74.05100952601869) bank154897 +154898 POINT(41.00179032831164 74.20646500530233) bank154898 +154899 POINT(40.95776794869814 73.09821362496044) bank154899 +154900 POINT(41.67711722087849 74.98690789392101) bank154900 +154901 POINT(40.53007266884455 73.84862199184364) bank154901 +154902 POINT(41.066788549765775 74.68851260697608) bank154902 +154903 POINT(41.597813244475375 74.47296852088303) bank154903 +154904 POINT(41.32853590387661 73.91317527330072) bank154904 +154905 POINT(40.53367599826631 73.63784327920844) bank154905 +154906 POINT(40.46150283972303 73.23292605318906) bank154906 +154907 POINT(41.212761690386344 74.12828653568) bank154907 +154908 POINT(40.53357168387836 73.48585959716307) bank154908 +154909 POINT(40.59295899487177 74.25192961745925) bank154909 +154910 POINT(41.223672481750484 74.66791383296511) bank154910 +154911 POINT(40.03368933666028 74.08558582108047) bank154911 +154912 POINT(39.73277573795175 73.05755977705054) bank154912 +154913 POINT(41.16086160978545 74.36183177871867) bank154913 +154914 POINT(41.22439130155328 74.02214154844008) bank154914 +154915 POINT(40.42171768951856 73.66283750754876) bank154915 +154916 POINT(39.716526726217964 74.34606921088451) bank154916 +154917 POINT(40.48908345702117 74.33992513628705) bank154917 +154918 POINT(39.75895601715096 74.69498866391694) bank154918 +154919 POINT(40.97920316189989 74.98662121253575) bank154919 +154920 POINT(41.05524250303678 73.637825974515) bank154920 +154921 POINT(39.89986877243315 73.70885539640845) bank154921 +154922 POINT(41.03845713727087 73.81485980488083) bank154922 +154923 POINT(40.66691188079105 74.93009643077632) bank154923 +154924 POINT(41.012340414095945 73.21261529452794) bank154924 +154925 POINT(40.593555071135334 73.32358932338802) bank154925 +154926 POINT(40.51951402675232 74.74512748123678) bank154926 +154927 POINT(41.31561845448708 73.69734167906302) bank154927 +154928 POINT(40.830560836690005 74.04783444983545) bank154928 +154929 POINT(40.87869037670019 74.7888148552943) bank154929 +154930 POINT(41.02502759690833 73.99255618411655) bank154930 +154931 POINT(41.70298377723256 73.40964658743397) bank154931 +154932 POINT(40.238848513573124 73.58600966609143) bank154932 +154933 POINT(40.04497118083858 73.79327254743028) bank154933 +154934 POINT(39.87601100262211 73.89090181418443) bank154934 +154935 POINT(40.669766726221575 73.92721168545508) bank154935 +154936 POINT(40.48225528460696 73.13352721332096) bank154936 +154937 POINT(41.24966821764477 74.5351461118159) bank154937 +154938 POINT(40.075275899172176 73.44524215327225) bank154938 +154939 POINT(40.35843672125168 74.05565750573486) bank154939 +154940 POINT(40.03175081909109 74.65245976831098) bank154940 +154941 POINT(40.936117272882 73.52680793408967) bank154941 +154942 POINT(39.749556741968135 73.41358518035267) bank154942 +154943 POINT(41.28055066568448 74.80042497071442) bank154943 +154944 POINT(40.44147531409364 73.99917452312069) bank154944 +154945 POINT(40.94319950936538 74.63705017058766) bank154945 +154946 POINT(41.65331444416268 74.54470881417403) bank154946 +154947 POINT(41.496641069852224 73.80963977521368) bank154947 +154948 POINT(40.47513942358366 73.8849165824138) bank154948 +154949 POINT(41.1774627922945 73.73885173502391) bank154949 +154950 POINT(40.009893170161405 73.15717506443961) bank154950 +154951 POINT(41.10045568750879 73.00724787351066) bank154951 +154952 POINT(39.94011916083785 74.70295766907574) bank154952 +154953 POINT(41.380960987494575 74.23993282489198) bank154953 +154954 POINT(39.85480150368808 74.99681221576274) bank154954 +154955 POINT(40.67211673032894 73.24768417840653) bank154955 +154956 POINT(39.71974147850972 73.4790732767088) bank154956 +154957 POINT(41.5116186932533 73.04128424360337) bank154957 +154958 POINT(39.91812815680967 74.97812600188468) bank154958 +154959 POINT(40.707749210270144 74.58620141585699) bank154959 +154960 POINT(41.49769374643875 74.00215066708404) bank154960 +154961 POINT(40.91990753373447 73.3685217089931) bank154961 +154962 POINT(39.868010030608275 74.10595757215175) bank154962 +154963 POINT(41.23182824147735 74.16002526666287) bank154963 +154964 POINT(41.11332489506614 74.13612590891002) bank154964 +154965 POINT(40.236864399039945 73.24739039636069) bank154965 +154966 POINT(41.595823725449954 74.84572848650002) bank154966 +154967 POINT(40.45024995641731 74.46761239445699) bank154967 +154968 POINT(41.66352389159346 73.7989806530672) bank154968 +154969 POINT(40.12762718116125 73.83941006424186) bank154969 +154970 POINT(40.82710038457608 73.47592190712649) bank154970 +154971 POINT(40.351984452440945 74.3504282751156) bank154971 +154972 POINT(39.73935483109531 74.10198674569487) bank154972 +154973 POINT(40.68659678950762 73.18140718673192) bank154973 +154974 POINT(41.42880949885235 73.38195267707202) bank154974 +154975 POINT(41.49033295515482 73.50617918831945) bank154975 +154976 POINT(39.750795204157995 73.26414406932767) bank154976 +154977 POINT(40.186258830488164 74.8072907941534) bank154977 +154978 POINT(41.471168445511736 73.05907216656364) bank154978 +154979 POINT(41.39729060714852 74.73067377918167) bank154979 +154980 POINT(40.8457908857483 74.66590944917672) bank154980 +154981 POINT(41.37669469475726 74.54362190034786) bank154981 +154982 POINT(39.94737093217733 74.75786386357635) bank154982 +154983 POINT(39.823787639141784 74.26764686200723) bank154983 +154984 POINT(39.74505222507578 73.83114588676847) bank154984 +154985 POINT(40.78355915122296 74.85537202415905) bank154985 +154986 POINT(40.80204841926664 73.15939751065) bank154986 +154987 POINT(40.44068326239147 74.03584272766814) bank154987 +154988 POINT(40.69943020206411 74.96068700525585) bank154988 +154989 POINT(40.75700543136348 74.96270600838284) bank154989 +154990 POINT(41.13131274213668 73.60501122522723) bank154990 +154991 POINT(41.297023544229106 73.56300822239213) bank154991 +154992 POINT(40.608010061897915 74.94822112574744) bank154992 +154993 POINT(40.07346420991416 73.6689010269522) bank154993 +154994 POINT(40.4686282270185 73.9391762637465) bank154994 +154995 POINT(40.558767764401736 73.50456669914297) bank154995 +154996 POINT(40.029286605758415 74.19851634054487) bank154996 +154997 POINT(41.067102779461486 73.55095250109542) bank154997 +154998 POINT(40.79222495922117 74.4758821582839) bank154998 +154999 POINT(41.17415726814401 73.45349912031479) bank154999 +155000 POINT(40.57445871678877 73.03142862770278) bank155000 +155001 POINT(41.532911889757074 74.58892929867174) bank155001 +155002 POINT(41.26302292157392 74.11423970118965) bank155002 +155003 POINT(41.14497928098559 74.86336489814228) bank155003 +155004 POINT(40.165860067968225 74.31643527594144) bank155004 +155005 POINT(41.59254896773885 73.38409037009436) bank155005 +155006 POINT(41.356413498679615 74.86093254851897) bank155006 +155007 POINT(41.632694978582826 74.55231658296908) bank155007 +155008 POINT(39.71798812498848 73.52757736404723) bank155008 +155009 POINT(41.19364355436065 73.5078987534607) bank155009 +155010 POINT(39.98432957237001 74.01124700528983) bank155010 +155011 POINT(41.47606933820679 73.69333318725694) bank155011 +155012 POINT(39.93326775115683 74.04174466685276) bank155012 +155013 POINT(40.10112707438868 74.29838329238733) bank155013 +155014 POINT(39.958959678683414 74.9852186980595) bank155014 +155015 POINT(39.82728165888056 73.49594238969034) bank155015 +155016 POINT(39.936648461544536 74.43403720611737) bank155016 +155017 POINT(40.291344075683575 74.18075541823839) bank155017 +155018 POINT(40.48016597570668 74.64717280866368) bank155018 +155019 POINT(40.81198241675292 74.80705692092268) bank155019 +155020 POINT(41.1394241646454 73.50326628274621) bank155020 +155021 POINT(41.3204571088151 74.57898899664666) bank155021 +155022 POINT(40.55239451904614 74.87198610309886) bank155022 +155023 POINT(40.214452418637 74.09240022518931) bank155023 +155024 POINT(40.47020070164867 73.31836251589426) bank155024 +155025 POINT(40.308141313144716 74.23835497041775) bank155025 +155026 POINT(40.769481369960964 73.06652164387802) bank155026 +155027 POINT(41.59136664475422 74.1115556711768) bank155027 +155028 POINT(40.247347379484815 74.04993087158616) bank155028 +155029 POINT(40.083007975867375 74.84548858583855) bank155029 +155030 POINT(40.693059647805924 74.4729671121503) bank155030 +155031 POINT(40.36237918662521 73.86169084531484) bank155031 +155032 POINT(40.10735248252497 74.6521915953856) bank155032 +155033 POINT(41.082930564867574 73.33750182245113) bank155033 +155034 POINT(40.58881511238632 74.15947533869476) bank155034 +155035 POINT(41.53606012888784 73.76712640049456) bank155035 +155036 POINT(39.77213043261425 73.21901213000983) bank155036 +155037 POINT(40.75793065366924 73.27946385949285) bank155037 +155038 POINT(40.37942353968762 74.03091179408113) bank155038 +155039 POINT(39.714778029552576 74.33573277632387) bank155039 +155040 POINT(41.547117928703685 73.75282283546454) bank155040 +155041 POINT(41.66041760259382 74.76137607054432) bank155041 +155042 POINT(41.15375976071845 73.56793228489971) bank155042 +155043 POINT(40.609097178650046 73.25718389230029) bank155043 +155044 POINT(41.154906476111634 73.54503056695498) bank155044 +155045 POINT(39.73882522738727 74.54395048073054) bank155045 +155046 POINT(40.2777540723865 73.42507697188879) bank155046 +155047 POINT(41.16644125717388 73.02231404943524) bank155047 +155048 POINT(41.56761760274523 74.94546326333568) bank155048 +155049 POINT(40.12930818033108 74.18520500105701) bank155049 +155050 POINT(40.727671302013185 74.88482099266231) bank155050 +155051 POINT(40.7569503019486 74.97537090513784) bank155051 +155052 POINT(40.500137696025206 74.84886704073014) bank155052 +155053 POINT(41.6819567245734 74.0861228940187) bank155053 +155054 POINT(40.14397340416517 74.87233614221832) bank155054 +155055 POINT(41.453599899820865 73.49926595183243) bank155055 +155056 POINT(40.771938782403254 73.7265857188828) bank155056 +155057 POINT(40.13309956073038 73.31305275470065) bank155057 +155058 POINT(39.83495116122699 74.08076374191671) bank155058 +155059 POINT(40.560951606669406 74.02227844553455) bank155059 +155060 POINT(39.998818354373306 74.79385126266853) bank155060 +155061 POINT(40.3340749048968 74.95666079854935) bank155061 +155062 POINT(41.50345267023846 73.17147782639893) bank155062 +155063 POINT(41.31942769681723 74.52086338745373) bank155063 +155064 POINT(40.333212240155675 74.87398814086623) bank155064 +155065 POINT(39.85610810360488 74.32252636043796) bank155065 +155066 POINT(39.78294004953908 73.37219130178877) bank155066 +155067 POINT(40.80578228074862 73.4795039494182) bank155067 +155068 POINT(40.33453285299448 73.89274811596118) bank155068 +155069 POINT(40.463208988605295 74.73465747958889) bank155069 +155070 POINT(40.459599342329064 73.20288825490589) bank155070 +155071 POINT(40.77314830093334 73.22023253941255) bank155071 +155072 POINT(41.022080212056466 73.82061662091145) bank155072 +155073 POINT(40.40289510207765 73.85514194382553) bank155073 +155074 POINT(41.136808811100224 74.41657563237743) bank155074 +155075 POINT(40.704605259767476 74.49991173325989) bank155075 +155076 POINT(39.83828832764355 74.50993205785595) bank155076 +155077 POINT(41.659189476890575 74.3020404541698) bank155077 +155078 POINT(39.97179347000158 74.59479772580484) bank155078 +155079 POINT(40.97171955587563 74.57456776687647) bank155079 +155080 POINT(39.988833204275366 75.00187368206237) bank155080 +155081 POINT(41.42597961909719 74.71328252818098) bank155081 +155082 POINT(40.106254783846076 74.1731794283598) bank155082 +155083 POINT(41.38447959124572 74.97232548405583) bank155083 +155084 POINT(41.33621785123252 73.17263206172346) bank155084 +155085 POINT(40.75292529061009 74.90324239743809) bank155085 +155086 POINT(41.22458882551816 73.85409035981948) bank155086 +155087 POINT(40.122031183376734 74.2278712810965) bank155087 +155088 POINT(41.40071606681207 73.05831937090541) bank155088 +155089 POINT(40.95079224376616 73.0871353631331) bank155089 +155090 POINT(39.78867416653827 73.48945592961034) bank155090 +155091 POINT(41.64155811742192 74.79777568846984) bank155091 +155092 POINT(40.31532846163976 74.70020597464182) bank155092 +155093 POINT(40.259400937722994 73.92702175494995) bank155093 +155094 POINT(40.864670090617324 74.95401871408353) bank155094 +155095 POINT(41.6323853017864 73.7974648609565) bank155095 +155096 POINT(40.55338384812289 73.15353192439048) bank155096 +155097 POINT(40.806081713313084 74.59203737492479) bank155097 +155098 POINT(39.923772633184015 73.87277293954288) bank155098 +155099 POINT(41.60199673008346 74.4694370562823) bank155099 +155100 POINT(41.07689423959889 74.3498528949805) bank155100 +155101 POINT(41.61941388319818 73.89748588981978) bank155101 +155102 POINT(39.94198662999076 73.93602329444035) bank155102 +155103 POINT(40.95955894613723 73.98972563387919) bank155103 +155104 POINT(40.73224223830452 73.36969571897065) bank155104 +155105 POINT(40.40475321450064 74.84126316112855) bank155105 +155106 POINT(39.863552344592044 73.14612670320167) bank155106 +155107 POINT(40.92893439652653 74.64170095787922) bank155107 +155108 POINT(40.11233679434401 73.70229963953796) bank155108 +155109 POINT(41.141047698191336 73.05033535818295) bank155109 +155110 POINT(41.566122332226506 74.92268528443014) bank155110 +155111 POINT(41.30553850163476 73.64272760497923) bank155111 +155112 POINT(40.53296726963181 74.638012111918) bank155112 +155113 POINT(40.51497979849071 74.72011456738562) bank155113 +155114 POINT(40.75966537241928 74.14682755342065) bank155114 +155115 POINT(39.81596302834616 74.91816376393695) bank155115 +155116 POINT(40.452458568464465 73.30005771461315) bank155116 +155117 POINT(40.5625791066178 73.52930034733885) bank155117 +155118 POINT(40.60252189285816 74.707604194843) bank155118 +155119 POINT(40.5573420189752 74.78719236978746) bank155119 +155120 POINT(41.101383905954094 73.0277908819382) bank155120 +155121 POINT(40.31458540183572 73.38494634813779) bank155121 +155122 POINT(41.505113425816376 74.25582172583502) bank155122 +155123 POINT(39.874472197846465 74.72106753386103) bank155123 +155124 POINT(41.432528782973534 74.55345941499918) bank155124 +155125 POINT(39.95682379343412 73.07628865659343) bank155125 +155126 POINT(40.171675152281 73.59760711585145) bank155126 +155127 POINT(40.13752875971467 73.01313143158428) bank155127 +155128 POINT(41.26853339120392 73.02365290046615) bank155128 +155129 POINT(40.453243997956385 74.77529782254878) bank155129 +155130 POINT(39.88163793525637 73.99810074168255) bank155130 +155131 POINT(41.273685263221935 74.7781475089862) bank155131 +155132 POINT(40.112584451789544 73.25438106055624) bank155132 +155133 POINT(40.4528483407028 74.79759788103944) bank155133 +155134 POINT(40.52097975791347 73.778138393854) bank155134 +155135 POINT(41.1964959639936 74.7433540467535) bank155135 +155136 POINT(40.073641627240725 73.12405617796679) bank155136 +155137 POINT(41.269493350947506 73.85105047957907) bank155137 +155138 POINT(41.642809234726165 74.32359228792996) bank155138 +155139 POINT(40.57963290056504 73.69421966217867) bank155139 +155140 POINT(39.720989409503034 73.57425945724371) bank155140 +155141 POINT(40.12583701156897 74.75193869369161) bank155141 +155142 POINT(41.07364669336178 74.52417935663232) bank155142 +155143 POINT(40.39364388111492 74.37515764876858) bank155143 +155144 POINT(40.1752218505675 73.473544954398) bank155144 +155145 POINT(41.421926127218654 73.18001050626782) bank155145 +155146 POINT(41.233912462064104 74.74808596715948) bank155146 +155147 POINT(41.12222104887065 74.03314169959845) bank155147 +155148 POINT(40.788138804297006 73.10166125201283) bank155148 +155149 POINT(41.530690055942856 73.76203970901001) bank155149 +155150 POINT(41.01502283011495 74.47099936052783) bank155150 +155151 POINT(40.704285980841476 73.25756146011625) bank155151 +155152 POINT(41.16706671912755 74.98003161249949) bank155152 +155153 POINT(40.53267900176053 74.58986462121052) bank155153 +155154 POINT(40.62289010583164 74.88957238599639) bank155154 +155155 POINT(40.67994198637899 73.69780658945116) bank155155 +155156 POINT(40.18735007408151 74.84056081531236) bank155156 +155157 POINT(41.587086437286516 74.45293762470946) bank155157 +155158 POINT(40.65985389587196 74.54866447542396) bank155158 +155159 POINT(41.21157009047836 73.54935934475343) bank155159 +155160 POINT(40.83541879147205 73.47734250506329) bank155160 +155161 POINT(41.34342033367501 74.19684082120062) bank155161 +155162 POINT(40.287920561470685 73.55407896521261) bank155162 +155163 POINT(41.192955061539436 73.47195802125994) bank155163 +155164 POINT(40.2085037835472 74.17449601840421) bank155164 +155165 POINT(41.341886137392514 74.23556670247936) bank155165 +155166 POINT(41.116464101765956 73.3520288405235) bank155166 +155167 POINT(41.35633901676264 74.55232940493666) bank155167 +155168 POINT(40.022697064228204 73.01846159884292) bank155168 +155169 POINT(40.999051750123265 74.53944217322979) bank155169 +155170 POINT(40.939082869855014 74.36231269809956) bank155170 +155171 POINT(40.83724918176357 73.57508332953854) bank155171 +155172 POINT(41.545746552684605 74.34251391864753) bank155172 +155173 POINT(41.357234667354064 73.94978246723225) bank155173 +155174 POINT(41.62669225396618 73.77915010769163) bank155174 +155175 POINT(41.63729962932121 74.96245885301846) bank155175 +155176 POINT(39.87941070097935 74.31971070410388) bank155176 +155177 POINT(41.062110901079265 74.37722230659223) bank155177 +155178 POINT(40.52865057638789 74.71252444113371) bank155178 +155179 POINT(41.68937143242734 73.01587396412587) bank155179 +155180 POINT(39.87548910711793 73.68505008096551) bank155180 +155181 POINT(39.884427858482084 74.33865824140823) bank155181 +155182 POINT(41.52338613157708 74.82010865390609) bank155182 +155183 POINT(41.44832389958775 74.7926039993767) bank155183 +155184 POINT(40.56527270744982 73.0575321914843) bank155184 +155185 POINT(40.32707249849469 74.22375106616074) bank155185 +155186 POINT(41.18486735220188 74.52624471957299) bank155186 +155187 POINT(40.60873640956434 73.54019887296728) bank155187 +155188 POINT(40.896019812016405 73.53221098433922) bank155188 +155189 POINT(40.879048319268456 74.37150042921931) bank155189 +155190 POINT(39.9243770885219 74.77160071220074) bank155190 +155191 POINT(40.03879208003477 74.43684208656562) bank155191 +155192 POINT(40.00344266267669 74.33529537216516) bank155192 +155193 POINT(41.42359271736552 73.39446383233329) bank155193 +155194 POINT(41.507317729044445 73.0988540402783) bank155194 +155195 POINT(41.00136324101477 73.34250329022865) bank155195 +155196 POINT(41.40090454631211 73.05939269355846) bank155196 +155197 POINT(40.77335687826596 74.53132323361332) bank155197 +155198 POINT(41.10503146463693 74.57585355352376) bank155198 +155199 POINT(40.27600259578677 74.21191642848177) bank155199 +155200 POINT(40.94851094334321 73.81978034800909) bank155200 +155201 POINT(41.49769849818254 74.49377640576) bank155201 +155202 POINT(41.20712007970297 73.49033665134061) bank155202 +155203 POINT(40.69169270557726 73.0420904151029) bank155203 +155204 POINT(41.22719187202978 73.21718644834073) bank155204 +155205 POINT(41.71214579747033 73.50154003931924) bank155205 +155206 POINT(41.47812304551065 73.37627572804013) bank155206 +155207 POINT(41.14918460244676 74.32929159925584) bank155207 +155208 POINT(40.01632418176565 74.05121521156384) bank155208 +155209 POINT(40.85652806437361 73.34817037396319) bank155209 +155210 POINT(40.573140723635866 73.53747400480145) bank155210 +155211 POINT(41.58629415102129 73.99200110635907) bank155211 +155212 POINT(40.81863562012905 74.2301730982643) bank155212 +155213 POINT(40.12648371713745 74.39738786321529) bank155213 +155214 POINT(41.28123953759921 73.3961622772664) bank155214 +155215 POINT(40.975005949828734 73.48893210976124) bank155215 +155216 POINT(40.91810663491335 74.88294576777425) bank155216 +155217 POINT(41.287416219187804 74.46349702551487) bank155217 +155218 POINT(39.824171257211646 73.10006650932273) bank155218 +155219 POINT(41.4393928434262 74.888674722002) bank155219 +155220 POINT(40.74289939936074 73.24496648530526) bank155220 +155221 POINT(40.97705434470297 73.48911133654524) bank155221 +155222 POINT(41.19362823209601 73.06404820208863) bank155222 +155223 POINT(40.42335482181812 74.15477371056627) bank155223 +155224 POINT(41.01126245248461 74.00506077332209) bank155224 +155225 POINT(41.498658263591956 73.89574151636539) bank155225 +155226 POINT(40.46082584983792 73.18207577196392) bank155226 +155227 POINT(40.492072923818405 74.39420480162819) bank155227 +155228 POINT(41.09417437713876 74.66798781854314) bank155228 +155229 POINT(41.576556497770774 73.96987360558968) bank155229 +155230 POINT(41.18740736351224 73.69168703533761) bank155230 +155231 POINT(41.46589306864467 74.0951218513912) bank155231 +155232 POINT(40.435767569383025 74.15060388864153) bank155232 +155233 POINT(41.294900159312135 74.66520473370004) bank155233 +155234 POINT(40.475280349448745 74.10071809534423) bank155234 +155235 POINT(39.956983991941996 74.08550903982173) bank155235 +155236 POINT(40.288861044779914 74.79260280739487) bank155236 +155237 POINT(41.10177393518681 74.68672828315098) bank155237 +155238 POINT(40.019234192495546 73.21846724381878) bank155238 +155239 POINT(41.21789275175938 74.31796488925261) bank155239 +155240 POINT(41.35879210753602 73.00751784396222) bank155240 +155241 POINT(40.99088381634646 73.26066760201489) bank155241 +155242 POINT(40.695692614729225 73.32235471593714) bank155242 +155243 POINT(41.085831418708615 73.99948525895111) bank155243 +155244 POINT(40.62926021618412 73.70154409796227) bank155244 +155245 POINT(41.706793384114135 74.63611366097916) bank155245 +155246 POINT(41.073015028893614 73.46046307636298) bank155246 +155247 POINT(40.527073904590374 73.28864661420579) bank155247 +155248 POINT(40.476429715281995 73.22398035358304) bank155248 +155249 POINT(41.24756901533007 73.51146868298812) bank155249 +155250 POINT(40.986825205197405 73.51505945845943) bank155250 +155251 POINT(41.370522312924855 74.32166830244344) bank155251 +155252 POINT(40.72728838578901 74.33854718993584) bank155252 +155253 POINT(39.73531903129406 74.47065029861773) bank155253 +155254 POINT(40.99661864327422 74.67365171942755) bank155254 +155255 POINT(41.35310790757727 74.73417891354127) bank155255 +155256 POINT(40.7366461817528 73.89474406963387) bank155256 +155257 POINT(41.30498049060982 73.09216210622269) bank155257 +155258 POINT(40.87818977078508 73.94715077228702) bank155258 +155259 POINT(40.02743167218894 74.79931306598156) bank155259 +155260 POINT(39.9643427879805 73.53038603696565) bank155260 +155261 POINT(40.255548922376555 73.28159249666624) bank155261 +155262 POINT(40.71310455913415 73.03803765277459) bank155262 +155263 POINT(40.32204069063835 74.61423353354357) bank155263 +155264 POINT(41.06651873607041 73.23064111804791) bank155264 +155265 POINT(40.11976764722934 73.49941074582476) bank155265 +155266 POINT(40.41744365994506 74.59803419460668) bank155266 +155267 POINT(39.95487502644338 73.05553009188795) bank155267 +155268 POINT(41.58887898713032 73.45990049515007) bank155268 +155269 POINT(41.45989794020756 74.16331487667094) bank155269 +155270 POINT(40.71856912519164 73.83768017502207) bank155270 +155271 POINT(40.79478562462196 74.05862754552437) bank155271 +155272 POINT(40.537951881730514 74.52680409429291) bank155272 +155273 POINT(40.81765705751437 73.43908931661608) bank155273 +155274 POINT(40.96238905361161 74.16577657551679) bank155274 +155275 POINT(39.92786754405883 73.25230340199347) bank155275 +155276 POINT(41.64059284438989 73.54728035684542) bank155276 +155277 POINT(40.26161115443229 74.08523841387037) bank155277 +155278 POINT(41.68328563211429 73.09263169585896) bank155278 +155279 POINT(41.27604524000662 73.05049857691405) bank155279 +155280 POINT(40.64774240629293 73.21390400520372) bank155280 +155281 POINT(41.15156207827568 73.44717386210554) bank155281 +155282 POINT(41.14959458078823 73.77184566015653) bank155282 +155283 POINT(41.41988480221023 73.70501044414887) bank155283 +155284 POINT(40.2433180661369 74.65576918907745) bank155284 +155285 POINT(40.35162508527906 74.11711211359425) bank155285 +155286 POINT(40.97917192935892 74.19703442262005) bank155286 +155287 POINT(39.788404598843336 73.6482378225852) bank155287 +155288 POINT(40.55031592593336 73.0704035629109) bank155288 +155289 POINT(40.48049877051595 74.15278276211622) bank155289 +155290 POINT(40.323160019644014 73.20804311240919) bank155290 +155291 POINT(40.33958093924029 74.69379296949627) bank155291 +155292 POINT(39.76451059870688 73.08057899764063) bank155292 +155293 POINT(40.96294013896503 73.65818880624931) bank155293 +155294 POINT(40.913053468007725 73.22211870055197) bank155294 +155295 POINT(40.79298257215601 73.35909612498082) bank155295 +155296 POINT(41.3293604972704 73.02045490766224) bank155296 +155297 POINT(41.125634108086174 74.37774093587265) bank155297 +155298 POINT(41.065426678936426 73.47693016326974) bank155298 +155299 POINT(41.082838695862456 74.73423342206985) bank155299 +155300 POINT(39.908891402032474 74.23720365630541) bank155300 +155301 POINT(41.497950974033216 74.7834375962454) bank155301 +155302 POINT(41.548913060787385 73.73522535196425) bank155302 +155303 POINT(40.567308895482945 73.7860806115365) bank155303 +155304 POINT(40.75228483442792 73.07827432023781) bank155304 +155305 POINT(41.09391649056491 74.15961151079868) bank155305 +155306 POINT(40.15572979532154 74.38756653780952) bank155306 +155307 POINT(41.547663270485494 74.58568077615648) bank155307 +155308 POINT(39.9832441225611 73.33911364449934) bank155308 +155309 POINT(40.50394750402468 74.67514141891927) bank155309 +155310 POINT(39.813226511247585 73.29265471609979) bank155310 +155311 POINT(39.946886967272576 74.78281238559485) bank155311 +155312 POINT(41.42230034548425 74.3154069091118) bank155312 +155313 POINT(40.9808121707929 74.3403564565718) bank155313 +155314 POINT(40.589388287370504 74.03259510049395) bank155314 +155315 POINT(40.01849709908567 74.63315927294319) bank155315 +155316 POINT(40.93467853074571 74.10903821543415) bank155316 +155317 POINT(39.939120304985984 74.63012817475864) bank155317 +155318 POINT(41.19408680662468 74.51705660621431) bank155318 +155319 POINT(40.579230157583225 74.5389457730321) bank155319 +155320 POINT(40.59429053490131 74.28659006025185) bank155320 +155321 POINT(40.43956836182448 73.16740416369339) bank155321 +155322 POINT(41.045339921698265 74.31219153066156) bank155322 +155323 POINT(39.922947224305716 74.56017707727443) bank155323 +155324 POINT(40.42930143043476 73.9881389597902) bank155324 +155325 POINT(39.80990378373609 73.83249918962647) bank155325 +155326 POINT(40.21207864444632 73.19394342225164) bank155326 +155327 POINT(40.92043196857968 74.81339481352347) bank155327 +155328 POINT(40.64767466577463 73.50634936688061) bank155328 +155329 POINT(40.26205333882764 74.15253662864178) bank155329 +155330 POINT(40.07456644125005 73.69148278769983) bank155330 +155331 POINT(40.090222602672576 73.14005655262298) bank155331 +155332 POINT(40.281879481731984 74.53505301028471) bank155332 +155333 POINT(41.40949499438759 74.22155242543712) bank155333 +155334 POINT(40.57805206493817 73.45645198370956) bank155334 +155335 POINT(40.595675689158114 74.08268216096067) bank155335 +155336 POINT(41.203048712640786 74.42817256883204) bank155336 +155337 POINT(40.3750547261762 74.42543559899954) bank155337 +155338 POINT(41.12356679502085 74.19284850465019) bank155338 +155339 POINT(41.315271593299904 73.4058423818054) bank155339 +155340 POINT(41.02920288631897 73.17127797015914) bank155340 +155341 POINT(41.49445900003755 74.11510439109821) bank155341 +155342 POINT(41.210723259202766 74.73196142642628) bank155342 +155343 POINT(40.826678609893655 74.47519258224449) bank155343 +155344 POINT(40.21504363183609 73.74741600233779) bank155344 +155345 POINT(40.7411247455952 73.84669822768397) bank155345 +155346 POINT(41.37245860054702 74.04163624878646) bank155346 +155347 POINT(41.00456133952781 74.83166941778585) bank155347 +155348 POINT(41.34009715909676 74.82469972326975) bank155348 +155349 POINT(39.82766334418861 73.12194200702189) bank155349 +155350 POINT(40.21992882089161 74.2913725774819) bank155350 +155351 POINT(40.88738828664752 74.80896126644912) bank155351 +155352 POINT(41.135985247942884 74.3468219441066) bank155352 +155353 POINT(40.833273391559345 74.04555842179741) bank155353 +155354 POINT(39.78703330012892 74.07514143324022) bank155354 +155355 POINT(41.145506931450875 74.90745731545432) bank155355 +155356 POINT(41.44128889955007 74.5721409670125) bank155356 +155357 POINT(41.35504517068071 73.40854348266302) bank155357 +155358 POINT(40.83151286070047 74.55539381615989) bank155358 +155359 POINT(40.09013469609012 74.97293065221598) bank155359 +155360 POINT(41.65796494446058 74.81141432839372) bank155360 +155361 POINT(41.381964526362786 74.75061363706492) bank155361 +155362 POINT(41.419746761934256 74.85800124282996) bank155362 +155363 POINT(41.496293191402394 73.97860677703325) bank155363 +155364 POINT(39.75759339812467 73.81184914801261) bank155364 +155365 POINT(41.28350375009449 73.66325889732664) bank155365 +155366 POINT(41.62577017249985 73.79814847225602) bank155366 +155367 POINT(40.99471761131138 74.78776245015364) bank155367 +155368 POINT(40.450116853824674 73.88115162831836) bank155368 +155369 POINT(39.7245178733354 73.53632526163278) bank155369 +155370 POINT(39.86242555156869 74.73897310826932) bank155370 +155371 POINT(40.838354514128625 73.97604791930375) bank155371 +155372 POINT(40.00749760850567 73.0172571867746) bank155372 +155373 POINT(39.98228509004963 73.51714003143505) bank155373 +155374 POINT(39.91988502632087 73.96933087262568) bank155374 +155375 POINT(40.675347976935896 74.97565852387073) bank155375 +155376 POINT(40.43133795444034 73.20125534606946) bank155376 +155377 POINT(40.74568802923862 74.29010927580383) bank155377 +155378 POINT(41.20092023494021 74.67376462832817) bank155378 +155379 POINT(41.52893242316831 73.23740696613012) bank155379 +155380 POINT(40.64969959500194 74.14518240937534) bank155380 +155381 POINT(41.70229677891214 74.22333378986967) bank155381 +155382 POINT(39.8448991136831 73.37421027063922) bank155382 +155383 POINT(40.82899924831086 73.22664761075599) bank155383 +155384 POINT(40.21916254366082 74.70498272052949) bank155384 +155385 POINT(40.4400311374966 74.98304659724738) bank155385 +155386 POINT(40.49864079948476 74.93323558117675) bank155386 +155387 POINT(39.95333945853575 74.56371765447129) bank155387 +155388 POINT(40.46741686451295 74.75599011829766) bank155388 +155389 POINT(41.423265960097005 74.5313471709095) bank155389 +155390 POINT(39.79672549126045 73.30511563804195) bank155390 +155391 POINT(40.87848355185628 73.58161454566054) bank155391 +155392 POINT(41.09919484527093 73.04118755851746) bank155392 +155393 POINT(39.81635923685977 73.89233778712824) bank155393 +155394 POINT(40.0426444659161 74.46618722063874) bank155394 +155395 POINT(40.83604287203438 73.28653429352687) bank155395 +155396 POINT(40.817179347185814 74.84704635553872) bank155396 +155397 POINT(39.803890609226194 73.7452356896763) bank155397 +155398 POINT(40.101038181688324 74.65707770314177) bank155398 +155399 POINT(41.678258170914276 74.74249002947694) bank155399 +155400 POINT(40.21169291107054 73.13646337805835) bank155400 +155401 POINT(40.07150897848954 74.83200733683314) bank155401 +155402 POINT(40.17682913177603 73.1562186280442) bank155402 +155403 POINT(41.184575279531884 74.91774457098006) bank155403 +155404 POINT(40.93541963350832 73.52816641356199) bank155404 +155405 POINT(39.86430108284335 73.35011416754696) bank155405 +155406 POINT(39.76673940652233 74.02603257203913) bank155406 +155407 POINT(40.91379549048758 73.03661240567106) bank155407 +155408 POINT(41.425198861134234 74.35589365582226) bank155408 +155409 POINT(41.29650671584288 73.18926636024682) bank155409 +155410 POINT(40.68312001935693 73.99613796901961) bank155410 +155411 POINT(40.38442951161691 73.76899525873755) bank155411 +155412 POINT(41.38690430403411 74.6031007946827) bank155412 +155413 POINT(40.89600471402424 74.05328802517855) bank155413 +155414 POINT(40.09851865549385 73.28515444369718) bank155414 +155415 POINT(40.307543012787264 74.9465235963417) bank155415 +155416 POINT(40.659225458288965 73.70257893178129) bank155416 +155417 POINT(41.32001441459507 74.12113399488055) bank155417 +155418 POINT(41.43658794513513 74.70119718639978) bank155418 +155419 POINT(40.1508223911666 73.49740585500136) bank155419 +155420 POINT(41.05092645871251 73.64252852578106) bank155420 +155421 POINT(39.770596699360304 74.93507371299492) bank155421 +155422 POINT(41.418456010813344 74.19611090117475) bank155422 +155423 POINT(40.79676565432877 73.62627846869414) bank155423 +155424 POINT(40.930818314212715 73.4633965244065) bank155424 +155425 POINT(40.17656782769332 74.63916442947382) bank155425 +155426 POINT(41.348170814085144 73.90090649027287) bank155426 +155427 POINT(41.30443142730744 74.43340508308683) bank155427 +155428 POINT(40.56751845602794 74.48766914901408) bank155428 +155429 POINT(41.07102938018872 74.78472566664183) bank155429 +155430 POINT(40.34204910234805 73.44576977521307) bank155430 +155431 POINT(41.3218011564307 74.69939154649731) bank155431 +155432 POINT(41.44901746927917 74.42756484939885) bank155432 +155433 POINT(39.735562654388836 73.36523267224261) bank155433 +155434 POINT(40.59681808058787 73.55771330307057) bank155434 +155435 POINT(40.05429664882774 74.75283220225367) bank155435 +155436 POINT(41.14422297717342 73.74838123843055) bank155436 +155437 POINT(39.766820275455224 73.90138865856026) bank155437 +155438 POINT(41.21769429283915 74.4029092023608) bank155438 +155439 POINT(40.039323084586165 74.64659985639042) bank155439 +155440 POINT(41.562661103437506 73.94816619543353) bank155440 +155441 POINT(40.70274812388405 73.49875633701619) bank155441 +155442 POINT(40.16418465755667 74.47368208762195) bank155442 +155443 POINT(40.703079275441596 73.51773440164685) bank155443 +155444 POINT(39.82234588501398 74.85649956340947) bank155444 +155445 POINT(41.36391206212004 74.4307275256023) bank155445 +155446 POINT(40.787639314198465 73.5729023919646) bank155446 +155447 POINT(40.60539299578315 73.89662617018548) bank155447 +155448 POINT(41.0675681209834 74.51207641961786) bank155448 +155449 POINT(40.19365431007678 74.52285436984145) bank155449 +155450 POINT(41.60722843073492 74.45162524514454) bank155450 +155451 POINT(40.99203492994609 73.35968100672046) bank155451 +155452 POINT(41.17628951583956 73.66097367572416) bank155452 +155453 POINT(40.890627683827574 73.1129124010633) bank155453 +155454 POINT(41.021447345840976 73.70319471890687) bank155454 +155455 POINT(40.87540094794128 74.57549749218543) bank155455 +155456 POINT(40.948028025378086 74.08076677863234) bank155456 +155457 POINT(40.70710893457813 73.29261827453625) bank155457 +155458 POINT(40.7133081616252 73.87934305865252) bank155458 +155459 POINT(40.16264364976705 74.88303916326939) bank155459 +155460 POINT(39.86164127827143 73.93864768876458) bank155460 +155461 POINT(40.880133017216934 73.95816117224518) bank155461 +155462 POINT(40.8277657162147 74.26475504340543) bank155462 +155463 POINT(41.677675500615386 74.18236542719372) bank155463 +155464 POINT(40.09454989992494 73.26886774053378) bank155464 +155465 POINT(41.46581875357773 74.6631206602139) bank155465 +155466 POINT(41.50292032766658 73.34740357689812) bank155466 +155467 POINT(41.4065563550535 73.90864731808733) bank155467 +155468 POINT(41.63852832695163 74.69961443254923) bank155468 +155469 POINT(39.78618294377508 74.78623823525565) bank155469 +155470 POINT(41.6734796078284 73.56264324008049) bank155470 +155471 POINT(40.749020109933376 73.01430001003547) bank155471 +155472 POINT(41.71150982630571 73.24790685844587) bank155472 +155473 POINT(40.688701414701995 73.04138267041131) bank155473 +155474 POINT(41.359117114645834 73.69961705161285) bank155474 +155475 POINT(41.395419336135724 74.42173240525794) bank155475 +155476 POINT(40.21863255609158 73.99216883202985) bank155476 +155477 POINT(41.031924089022596 73.91052115784275) bank155477 +155478 POINT(40.97816775269366 73.06885080697393) bank155478 +155479 POINT(39.945774033005506 73.9057793141) bank155479 +155480 POINT(40.48086077387263 73.78963434979723) bank155480 +155481 POINT(40.14804386490599 73.55675435686082) bank155481 +155482 POINT(40.79231805118097 73.31826434302599) bank155482 +155483 POINT(40.06615812006195 74.55854082810772) bank155483 +155484 POINT(39.766330951918384 74.14337821813172) bank155484 +155485 POINT(40.19771381077172 73.8949980625529) bank155485 +155486 POINT(41.350575791569774 73.25393832443325) bank155486 +155487 POINT(39.92706476782784 74.30360696487772) bank155487 +155488 POINT(40.958462910438776 74.35887889059128) bank155488 +155489 POINT(41.22137069307773 74.83642934392967) bank155489 +155490 POINT(40.76185170567345 74.275851107089) bank155490 +155491 POINT(41.08109456763753 74.16554095177368) bank155491 +155492 POINT(40.644049129577645 74.99683622445967) bank155492 +155493 POINT(40.66890814183144 73.80278684478111) bank155493 +155494 POINT(40.6890870139893 73.73162436280941) bank155494 +155495 POINT(41.10297788562008 73.30677156022273) bank155495 +155496 POINT(41.49658569926044 74.9969388242029) bank155496 +155497 POINT(40.63566600958904 73.0092889013474) bank155497 +155498 POINT(40.23463397462715 74.89720693692603) bank155498 +155499 POINT(40.7889861112287 74.31860541351786) bank155499 +155500 POINT(40.52804611262703 73.0911848953312) bank155500 +155501 POINT(40.97836965139286 73.60899643660771) bank155501 +155502 POINT(40.44338597348044 73.71434067420667) bank155502 +155503 POINT(40.64682128855111 74.39635304890457) bank155503 +155504 POINT(41.2380486610368 74.30071090765146) bank155504 +155505 POINT(40.96403898935768 73.45511018265532) bank155505 +155506 POINT(40.51264184603784 73.5424539600074) bank155506 +155507 POINT(41.679446521309096 74.28947867731539) bank155507 +155508 POINT(40.61204347211856 74.92874370551719) bank155508 +155509 POINT(40.176320166976566 74.03790545701725) bank155509 +155510 POINT(40.53811595848408 74.03025817231419) bank155510 +155511 POINT(40.25812928810482 74.60796697938807) bank155511 +155512 POINT(41.26652359655624 73.88856619447303) bank155512 +155513 POINT(40.84316454665871 74.45934319924422) bank155513 +155514 POINT(39.94806471087968 74.24301573641799) bank155514 +155515 POINT(41.41670803319594 73.78007525344326) bank155515 +155516 POINT(39.76138452826899 74.90835139345944) bank155516 +155517 POINT(39.880091054785574 74.42254991228829) bank155517 +155518 POINT(39.96390786401823 74.8519038425776) bank155518 +155519 POINT(40.25403189424569 73.81956420690676) bank155519 +155520 POINT(41.28526736722679 73.58098740818902) bank155520 +155521 POINT(41.13634365150984 74.0520154219433) bank155521 +155522 POINT(40.01327474049164 74.79292753918722) bank155522 +155523 POINT(40.00330584588044 73.2361232917868) bank155523 +155524 POINT(39.85271235083105 73.17332941355903) bank155524 +155525 POINT(41.69430771741569 73.1577276134948) bank155525 +155526 POINT(41.1318419908072 74.43276499820736) bank155526 +155527 POINT(41.63654831561951 74.30243600632924) bank155527 +155528 POINT(39.88681703401114 73.88766133565656) bank155528 +155529 POINT(39.917409975817684 74.40130376110518) bank155529 +155530 POINT(40.92712043910473 74.1887180063873) bank155530 +155531 POINT(41.277350011631036 73.42744710681166) bank155531 +155532 POINT(41.33463918360021 74.09632419726654) bank155532 +155533 POINT(40.36815063584665 73.82876760080974) bank155533 +155534 POINT(40.76023892759342 74.54914015682961) bank155534 +155535 POINT(40.96275179571619 74.29190019357722) bank155535 +155536 POINT(39.86703576213913 74.47603887765109) bank155536 +155537 POINT(39.73170917024054 74.96034671477776) bank155537 +155538 POINT(41.08598199272656 73.32177142845379) bank155538 +155539 POINT(40.73623386106011 73.90190601404161) bank155539 +155540 POINT(40.97769006972718 74.52135488684557) bank155540 +155541 POINT(40.29762612678518 74.71177934403268) bank155541 +155542 POINT(39.91917719670359 73.4257859089096) bank155542 +155543 POINT(40.18118255087283 74.21589565358987) bank155543 +155544 POINT(40.09731118755001 73.78647379771458) bank155544 +155545 POINT(40.46960564792127 74.67980278455039) bank155545 +155546 POINT(41.26927984871283 74.57900698982874) bank155546 +155547 POINT(41.69592109154573 73.66628886107526) bank155547 +155548 POINT(40.85864092595145 73.32250702411918) bank155548 +155549 POINT(41.45081338928813 74.00800730443981) bank155549 +155550 POINT(41.06846620282322 74.31658183297168) bank155550 +155551 POINT(39.79161284145997 73.61813550681835) bank155551 +155552 POINT(40.86232417271344 73.71472591024431) bank155552 +155553 POINT(41.39091017872875 74.36001150534064) bank155553 +155554 POINT(41.46737099628647 74.50690627567154) bank155554 +155555 POINT(40.864662376981634 73.63916378633121) bank155555 +155556 POINT(40.35049855564623 73.93382562730805) bank155556 +155557 POINT(40.091040216857955 73.05614547963958) bank155557 +155558 POINT(40.39830720548103 74.69431562317007) bank155558 +155559 POINT(40.44517296271377 73.07453818603192) bank155559 +155560 POINT(41.2771115330846 74.49102728342399) bank155560 +155561 POINT(39.90079923094791 73.38904547661583) bank155561 +155562 POINT(40.24721124674781 74.93643622868774) bank155562 +155563 POINT(39.88840091400687 74.85400796951963) bank155563 +155564 POINT(40.52185029971199 74.57246874478425) bank155564 +155565 POINT(39.71705002263429 74.06235451201069) bank155565 +155566 POINT(40.96658073087059 74.65974688886087) bank155566 +155567 POINT(41.64110817904427 73.09679400261602) bank155567 +155568 POINT(40.46732922751126 74.73648965739677) bank155568 +155569 POINT(41.6885801421091 73.33312613718279) bank155569 +155570 POINT(39.97640138853193 74.67701353208012) bank155570 +155571 POINT(40.4386207539445 73.0597461236275) bank155571 +155572 POINT(40.236166154298125 74.21199811151861) bank155572 +155573 POINT(40.6631656256395 74.91755661658324) bank155573 +155574 POINT(40.4632589168546 74.16137636022286) bank155574 +155575 POINT(41.238743772054555 74.08910772379426) bank155575 +155576 POINT(40.295948252320656 73.09354284262696) bank155576 +155577 POINT(40.79849088794772 73.60865728500403) bank155577 +155578 POINT(40.996253670522634 73.3336840111765) bank155578 +155579 POINT(39.7921126277195 73.92934018925122) bank155579 +155580 POINT(41.21565154157235 73.88457179513954) bank155580 +155581 POINT(40.71790911638362 73.85492762827958) bank155581 +155582 POINT(39.81213274025337 74.35196610798249) bank155582 +155583 POINT(40.55093889572792 73.29652907065754) bank155583 +155584 POINT(39.83096388342709 74.45210337056474) bank155584 +155585 POINT(41.128733984504386 74.31689114421815) bank155585 +155586 POINT(41.6813387080179 74.15814950552675) bank155586 +155587 POINT(41.10547570727213 74.25100685613495) bank155587 +155588 POINT(41.5646728538019 74.79133255962414) bank155588 +155589 POINT(40.76833100685545 74.49100325912347) bank155589 +155590 POINT(41.31624825792156 74.50283605070486) bank155590 +155591 POINT(40.20362371249409 74.4684958095079) bank155591 +155592 POINT(39.80111962932678 74.77423310742215) bank155592 +155593 POINT(41.46451261449581 73.39839285491533) bank155593 +155594 POINT(40.20049995160647 73.03686643813191) bank155594 +155595 POINT(41.5608814772684 73.38763304406886) bank155595 +155596 POINT(39.721425846975166 74.87997662233508) bank155596 +155597 POINT(40.78821999837998 74.73349750235904) bank155597 +155598 POINT(41.22724773796218 73.23013498985483) bank155598 +155599 POINT(40.3505447105973 73.69842704439533) bank155599 +155600 POINT(40.97487875532636 74.60826785529423) bank155600 +155601 POINT(40.308890654204 74.69609069383385) bank155601 +155602 POINT(40.99464515028609 73.38268249314325) bank155602 +155603 POINT(41.13332425981574 73.12469717081281) bank155603 +155604 POINT(39.914932914363405 74.17202181438522) bank155604 +155605 POINT(41.10664079116925 73.07371452634753) bank155605 +155606 POINT(41.42444871729198 73.02240614144057) bank155606 +155607 POINT(39.85074812170902 74.34445866745153) bank155607 +155608 POINT(40.96667598628441 73.46089097610795) bank155608 +155609 POINT(41.59198981740484 74.23723089123061) bank155609 +155610 POINT(40.91888106079284 73.10103848667984) bank155610 +155611 POINT(41.65778782675904 73.67419858229627) bank155611 +155612 POINT(40.204444450605685 73.8041556727993) bank155612 +155613 POINT(40.510929132871205 74.28492903105462) bank155613 +155614 POINT(40.19837531066383 73.68306851097606) bank155614 +155615 POINT(40.063333353027616 73.91794069608414) bank155615 +155616 POINT(40.74068585007366 74.41321656540927) bank155616 +155617 POINT(41.55470071957771 74.61908679155934) bank155617 +155618 POINT(40.60567486799027 74.17185390060318) bank155618 +155619 POINT(39.77258686724842 73.23104631283262) bank155619 +155620 POINT(40.347537060328406 73.4683082423404) bank155620 +155621 POINT(40.958628058572145 74.78515661298155) bank155621 +155622 POINT(40.191951838701485 73.51903232566087) bank155622 +155623 POINT(40.46411034861374 73.98374880298552) bank155623 +155624 POINT(40.973803127236174 73.06530349879264) bank155624 +155625 POINT(40.76195881640142 73.99849538970852) bank155625 +155626 POINT(40.31748927947893 73.09427168394335) bank155626 +155627 POINT(40.66523796624146 73.59348800914883) bank155627 +155628 POINT(39.96302696697186 73.82602441307645) bank155628 +155629 POINT(40.02127796876777 74.50822049523178) bank155629 +155630 POINT(40.7830466761533 74.48764794098) bank155630 +155631 POINT(41.530355888067625 73.99551515161414) bank155631 +155632 POINT(41.12634650764421 73.71586598508719) bank155632 +155633 POINT(40.49067507741582 74.81190762105606) bank155633 +155634 POINT(41.57152843809455 73.18079991030203) bank155634 +155635 POINT(41.38381365089226 73.06969730891264) bank155635 +155636 POINT(40.206704148438575 74.85806870202025) bank155636 +155637 POINT(40.10799784623693 73.6803918758343) bank155637 +155638 POINT(40.22553217617899 73.50321015604992) bank155638 +155639 POINT(41.706241722589084 73.5381033419919) bank155639 +155640 POINT(40.54353651665606 73.3782967181905) bank155640 +155641 POINT(40.74598639680617 73.80067698277102) bank155641 +155642 POINT(41.17193169017157 73.50871436030472) bank155642 +155643 POINT(40.24872891831866 74.17488327115584) bank155643 +155644 POINT(39.93910978841073 73.63010085443965) bank155644 +155645 POINT(40.478071322914 73.76746680192076) bank155645 +155646 POINT(41.2177265132509 74.58632900608175) bank155646 +155647 POINT(40.526859048232176 73.86238684686484) bank155647 +155648 POINT(41.67977979479661 73.56363089547003) bank155648 +155649 POINT(40.68732047876625 73.71959075331823) bank155649 +155650 POINT(40.21733947947582 73.89679716000555) bank155650 +155651 POINT(41.32971639063017 74.81590086896372) bank155651 +155652 POINT(41.612790617521114 74.9518814793464) bank155652 +155653 POINT(40.443819333063814 73.49535500333302) bank155653 +155654 POINT(41.482214021634164 74.05328032131794) bank155654 +155655 POINT(41.19163816198644 74.76529320114217) bank155655 +155656 POINT(40.135284582338485 73.18786642744729) bank155656 +155657 POINT(41.06046818127832 73.9857363794324) bank155657 +155658 POINT(41.22782434529872 74.9233042375269) bank155658 +155659 POINT(40.52252734208202 73.69669179077208) bank155659 +155660 POINT(41.228077284238104 74.53382495793636) bank155660 +155661 POINT(41.6005025597127 74.44441724702185) bank155661 +155662 POINT(39.864231213160934 74.84480070785243) bank155662 +155663 POINT(40.036099563014396 74.98021316508274) bank155663 +155664 POINT(40.28148221511476 73.33308373508359) bank155664 +155665 POINT(41.692012946782334 74.24422132992758) bank155665 +155666 POINT(40.526098141570536 74.07855521930249) bank155666 +155667 POINT(40.216551300637136 73.44866396501753) bank155667 +155668 POINT(40.292992861974895 73.50675610293911) bank155668 +155669 POINT(40.72703920854419 73.47243745915401) bank155669 +155670 POINT(41.38211575138485 73.85099656537459) bank155670 +155671 POINT(40.319350810904524 73.65843325318644) bank155671 +155672 POINT(41.21861594509061 74.47268497857675) bank155672 +155673 POINT(40.89702591166392 73.90652785677922) bank155673 +155674 POINT(39.82546172221003 73.53489900888646) bank155674 +155675 POINT(41.01259965341597 74.39761716054731) bank155675 +155676 POINT(40.666701425192656 74.37251853384834) bank155676 +155677 POINT(40.66319815398586 74.30769911442431) bank155677 +155678 POINT(40.19284745198933 73.12226170780491) bank155678 +155679 POINT(40.57364997649462 73.31252694787176) bank155679 +155680 POINT(39.73912018439087 74.51853192257462) bank155680 +155681 POINT(40.86559924160569 73.7076408468167) bank155681 +155682 POINT(40.959218037988975 74.48951302383244) bank155682 +155683 POINT(41.34907844312774 73.61338636492826) bank155683 +155684 POINT(40.07680130992167 74.2438802607227) bank155684 +155685 POINT(40.82795085446238 73.80934527769114) bank155685 +155686 POINT(40.60237508563924 74.7638435476145) bank155686 +155687 POINT(40.19156535851404 74.16334712785492) bank155687 +155688 POINT(40.156851058291366 74.85868629201961) bank155688 +155689 POINT(40.41064044784499 73.11402876086764) bank155689 +155690 POINT(40.651256634422644 74.0776145252108) bank155690 +155691 POINT(40.1564326564142 74.07739353913715) bank155691 +155692 POINT(41.304430275425354 74.49797230030592) bank155692 +155693 POINT(40.57520199239766 74.5930470992156) bank155693 +155694 POINT(40.83451455558789 73.83199622722016) bank155694 +155695 POINT(40.02032239531061 74.13924026001624) bank155695 +155696 POINT(39.82654655693279 73.07839003374163) bank155696 +155697 POINT(40.866443621282045 73.41974082313799) bank155697 +155698 POINT(41.450374771165976 74.57673171805055) bank155698 +155699 POINT(40.631795285356716 74.99864760552678) bank155699 +155700 POINT(40.73905594299987 73.95682005127694) bank155700 +155701 POINT(40.50604886217475 74.6744942890422) bank155701 +155702 POINT(39.749814646239656 73.13177163814713) bank155702 +155703 POINT(39.95684458524159 73.06749038795766) bank155703 +155704 POINT(41.03583145168562 74.27586654568925) bank155704 +155705 POINT(40.99208121742556 73.56936302861345) bank155705 +155706 POINT(41.1834392364354 73.0224699489262) bank155706 +155707 POINT(40.700070232938096 74.41009974184135) bank155707 +155708 POINT(41.2201176899968 73.10511130867738) bank155708 +155709 POINT(39.85428061513249 74.8328243368297) bank155709 +155710 POINT(39.83881939715406 73.1918722209896) bank155710 +155711 POINT(40.962569978676896 74.07239223924391) bank155711 +155712 POINT(41.08901431334037 74.98180541202272) bank155712 +155713 POINT(40.524499254829735 74.16604192324654) bank155713 +155714 POINT(41.28579126412293 74.7786089891668) bank155714 +155715 POINT(40.429988119399106 73.4702192291482) bank155715 +155716 POINT(41.23985530292518 73.49541246201049) bank155716 +155717 POINT(41.540092914412476 73.63427884882461) bank155717 +155718 POINT(40.72559928292067 74.03282307174429) bank155718 +155719 POINT(41.11899564832937 73.78371291129613) bank155719 +155720 POINT(40.06239587590879 73.4978357007251) bank155720 +155721 POINT(39.75710824689631 74.11243136070999) bank155721 +155722 POINT(40.626669860868525 73.90642244640009) bank155722 +155723 POINT(40.997494530865715 74.57761174951908) bank155723 +155724 POINT(41.702461322228885 73.55507052621984) bank155724 +155725 POINT(40.345671471347316 74.37879735511933) bank155725 +155726 POINT(41.16907927203209 73.58296318268455) bank155726 +155727 POINT(39.862244626346516 74.04352077842086) bank155727 +155728 POINT(40.691991310184804 73.49600715670684) bank155728 +155729 POINT(39.79988369467366 73.75351132608722) bank155729 +155730 POINT(40.623826869161 73.80483893385349) bank155730 +155731 POINT(40.82359851957662 73.1037434056459) bank155731 +155732 POINT(41.221368554870615 74.30028180670745) bank155732 +155733 POINT(40.45373259172372 73.36473041912612) bank155733 +155734 POINT(40.701189488597265 74.64597231198316) bank155734 +155735 POINT(40.148920288382854 73.62706481603922) bank155735 +155736 POINT(41.437718283951156 74.13671060764287) bank155736 +155737 POINT(41.27038792516628 73.44405751240747) bank155737 +155738 POINT(40.50582522612046 74.71554722127811) bank155738 +155739 POINT(41.42076750172998 73.8477477052978) bank155739 +155740 POINT(41.69009845742633 74.2204504881033) bank155740 +155741 POINT(40.13149361733855 73.76677058911476) bank155741 +155742 POINT(40.59940502383524 73.43337039146574) bank155742 +155743 POINT(41.27796308516605 74.955898897833) bank155743 +155744 POINT(41.352876819815314 74.29390577093501) bank155744 +155745 POINT(41.46383959523878 73.08135144039308) bank155745 +155746 POINT(39.98729303755306 74.37805742086849) bank155746 +155747 POINT(41.145811688148285 74.13882460902228) bank155747 +155748 POINT(41.03066158091408 74.4532379239815) bank155748 +155749 POINT(40.8305701998714 74.79339808532592) bank155749 +155750 POINT(41.662262333195464 73.86895871914636) bank155750 +155751 POINT(41.320458802684136 74.89060047924931) bank155751 +155752 POINT(40.633200380570344 74.42833895862442) bank155752 +155753 POINT(41.413548903669735 74.25231079951841) bank155753 +155754 POINT(40.8505568233499 74.86013806079627) bank155754 +155755 POINT(41.17581725687532 73.04210994988203) bank155755 +155756 POINT(40.22384498847849 73.00747733112212) bank155756 +155757 POINT(40.19323147998102 74.87468458891611) bank155757 +155758 POINT(41.118832879063575 73.38068622248974) bank155758 +155759 POINT(40.81104403310873 74.04659252654561) bank155759 +155760 POINT(40.128335431824304 74.19235111879114) bank155760 +155761 POINT(40.44610005803276 73.75467208388511) bank155761 +155762 POINT(41.40030232032663 74.24439593686262) bank155762 +155763 POINT(40.511428117375914 74.29740197774892) bank155763 +155764 POINT(40.24829669422285 73.69406134992805) bank155764 +155765 POINT(39.71905590737169 74.01207480337185) bank155765 +155766 POINT(41.52806330740435 73.12662294505088) bank155766 +155767 POINT(39.725479680238166 74.40309612381097) bank155767 +155768 POINT(40.3554717124231 73.99195517167108) bank155768 +155769 POINT(40.90969852473475 73.93060969445504) bank155769 +155770 POINT(39.838382333135264 74.16531584900197) bank155770 +155771 POINT(41.49628915285661 74.7030930837743) bank155771 +155772 POINT(41.37848875681241 74.89087483625916) bank155772 +155773 POINT(41.39978724962578 73.9413328303887) bank155773 +155774 POINT(40.087227860273096 74.155076548778) bank155774 +155775 POINT(41.387041956644886 74.5479766881336) bank155775 +155776 POINT(39.90131598833263 73.77967633125459) bank155776 +155777 POINT(41.565213197123576 73.64696472911822) bank155777 +155778 POINT(40.08852107064321 74.82690829854222) bank155778 +155779 POINT(41.44532649384865 73.94870843225915) bank155779 +155780 POINT(40.95512906989191 74.32968942363547) bank155780 +155781 POINT(40.59112092334211 74.97083901160357) bank155781 +155782 POINT(41.26052774268554 73.08929857610528) bank155782 +155783 POINT(40.89945079821343 74.07765050366162) bank155783 +155784 POINT(41.509623717060194 73.80214960516739) bank155784 +155785 POINT(40.019544847961654 74.89817123415824) bank155785 +155786 POINT(40.138008156048734 74.18550418988879) bank155786 +155787 POINT(40.946652716295866 73.8163833216435) bank155787 +155788 POINT(41.62708022963599 73.00817832745149) bank155788 +155789 POINT(39.79524294138584 73.7395906743451) bank155789 +155790 POINT(41.2593481740484 74.11147240791927) bank155790 +155791 POINT(40.43130739958192 73.44882832802412) bank155791 +155792 POINT(40.17724458387914 73.1105926626033) bank155792 +155793 POINT(39.97728438485019 73.74628269211951) bank155793 +155794 POINT(40.45279216450123 74.07955623627964) bank155794 +155795 POINT(40.29043424061187 73.70136080648538) bank155795 +155796 POINT(41.70024165865921 74.8211234427481) bank155796 +155797 POINT(40.982149147622565 74.12465415000672) bank155797 +155798 POINT(40.684073187578406 74.30688109037344) bank155798 +155799 POINT(40.976838208297934 74.57183493220664) bank155799 +155800 POINT(41.2901166028309 73.25019740189286) bank155800 +155801 POINT(40.46019698815254 73.88816875863834) bank155801 +155802 POINT(41.14091950667311 74.31341641063041) bank155802 +155803 POINT(39.88772918697179 74.53601051223076) bank155803 +155804 POINT(41.28474532255953 74.33183438996008) bank155804 +155805 POINT(40.57204085380246 74.99270955232453) bank155805 +155806 POINT(40.467867341252614 74.35752498562653) bank155806 +155807 POINT(39.77688736303443 74.54296720366571) bank155807 +155808 POINT(40.22534711868615 74.68670544072522) bank155808 +155809 POINT(40.44009565204313 74.78320723320543) bank155809 +155810 POINT(39.90404057818472 74.78064343521744) bank155810 +155811 POINT(40.494450934076994 74.00996381722818) bank155811 +155812 POINT(41.28049257080018 74.79076928360311) bank155812 +155813 POINT(39.97514497220393 74.11539762801337) bank155813 +155814 POINT(41.18632994374254 74.08920206232536) bank155814 +155815 POINT(40.80628205064252 73.38869512545972) bank155815 +155816 POINT(41.499281552454065 73.93366459830735) bank155816 +155817 POINT(41.357601571083634 74.72885735978551) bank155817 +155818 POINT(40.45280986355156 73.67319418610107) bank155818 +155819 POINT(41.008543912432906 73.03398476346193) bank155819 +155820 POINT(40.87040828427004 73.14346558035957) bank155820 +155821 POINT(40.97136814785258 74.97021807742959) bank155821 +155822 POINT(41.30776982202063 73.65229573752289) bank155822 +155823 POINT(41.20355147001392 74.49316218760536) bank155823 +155824 POINT(40.511585978275505 73.62083414327792) bank155824 +155825 POINT(41.04596340783692 74.52891313121756) bank155825 +155826 POINT(41.15363597390674 74.73707512839464) bank155826 +155827 POINT(41.0797100903073 74.04345563334644) bank155827 +155828 POINT(41.01942685164295 73.77941225736616) bank155828 +155829 POINT(40.20782437933025 73.25554067371732) bank155829 +155830 POINT(40.340749236279024 73.50316493820272) bank155830 +155831 POINT(40.87258206995108 73.84087352376035) bank155831 +155832 POINT(40.24998875402478 74.85429223261782) bank155832 +155833 POINT(40.29971176375483 73.62735905354938) bank155833 +155834 POINT(40.354934699904554 74.03992284407394) bank155834 +155835 POINT(40.448899368333294 73.21410982572792) bank155835 +155836 POINT(41.235496983361244 74.88131477586363) bank155836 +155837 POINT(40.11368870659041 74.03934627596703) bank155837 +155838 POINT(40.945621414231425 74.75889158220971) bank155838 +155839 POINT(41.571432718593684 74.22982553689597) bank155839 +155840 POINT(39.96656699196978 73.27190697296295) bank155840 +155841 POINT(41.6186040621466 73.59842519250059) bank155841 +155842 POINT(40.53668490902126 73.53317864152339) bank155842 +155843 POINT(40.176285829158296 74.77339156712955) bank155843 +155844 POINT(40.030982934076256 73.20459161094838) bank155844 +155845 POINT(40.541865073450026 73.93038957508334) bank155845 +155846 POINT(40.65499244510259 74.02499042525545) bank155846 +155847 POINT(39.79548388538718 74.54398714501214) bank155847 +155848 POINT(40.3089745015751 74.24747391048129) bank155848 +155849 POINT(41.23384067687238 74.23601990891032) bank155849 +155850 POINT(41.17654224474051 74.84700789388476) bank155850 +155851 POINT(41.658301749063035 73.56860311793304) bank155851 +155852 POINT(41.05524523119467 74.01307434316773) bank155852 +155853 POINT(41.59782451164088 74.03846894859815) bank155853 +155854 POINT(40.05484461113545 73.29734792467478) bank155854 +155855 POINT(41.42755733257009 73.16129501702625) bank155855 +155856 POINT(40.90982396991593 73.68756815853192) bank155856 +155857 POINT(40.73237045055997 74.47047604957434) bank155857 +155858 POINT(41.60981448446512 74.83902036881614) bank155858 +155859 POINT(40.731061034755925 73.59742759344093) bank155859 +155860 POINT(40.42896117313876 73.47534810995046) bank155860 +155861 POINT(40.566894860160666 74.95682533338244) bank155861 +155862 POINT(41.00349718203214 74.18576345917279) bank155862 +155863 POINT(40.31884459281132 74.78285922711368) bank155863 +155864 POINT(40.37823703290417 74.48754792261943) bank155864 +155865 POINT(40.77380471753519 73.92097093668045) bank155865 +155866 POINT(40.63006323348648 73.80910157845908) bank155866 +155867 POINT(40.28347871550329 73.2177974932132) bank155867 +155868 POINT(41.09852129120994 74.57699613334219) bank155868 +155869 POINT(41.16205364316054 74.24607173085535) bank155869 +155870 POINT(39.93255968918346 74.1749657203743) bank155870 +155871 POINT(40.694795135737 73.80355586237559) bank155871 +155872 POINT(40.192669955110034 73.33885354003904) bank155872 +155873 POINT(40.44103795220392 74.07068447004396) bank155873 +155874 POINT(41.10968716867953 73.50819931523034) bank155874 +155875 POINT(39.791231389113925 73.76423578532587) bank155875 +155876 POINT(40.68670618534027 73.62365035322784) bank155876 +155877 POINT(39.78485614948229 74.19859447009438) bank155877 +155878 POINT(40.744343887107554 73.28806692153411) bank155878 +155879 POINT(41.31300597539041 73.271386210001) bank155879 +155880 POINT(41.43626512911974 74.45597476724954) bank155880 +155881 POINT(40.80326417278311 74.41504315436367) bank155881 +155882 POINT(41.01486471132249 74.37962746752429) bank155882 +155883 POINT(41.493742125617345 73.2535427709204) bank155883 +155884 POINT(39.998249373156966 73.10761040141739) bank155884 +155885 POINT(41.42686482549198 74.13136678769705) bank155885 +155886 POINT(39.845068975568516 74.76600629427901) bank155886 +155887 POINT(39.868986242925224 74.08919156451213) bank155887 +155888 POINT(41.23318586060941 74.56672647558975) bank155888 +155889 POINT(40.126844369169405 74.34772678331088) bank155889 +155890 POINT(40.79606178286821 74.0461626347354) bank155890 +155891 POINT(40.26149704104222 73.33912884925446) bank155891 +155892 POINT(40.503341542126435 74.86468584503223) bank155892 +155893 POINT(40.665515384071206 74.00331105983679) bank155893 +155894 POINT(40.85085513730294 73.69681886643677) bank155894 +155895 POINT(41.17598434010289 74.08553738377647) bank155895 +155896 POINT(40.787987750885435 73.48683821927075) bank155896 +155897 POINT(39.72975965812963 74.63337536096847) bank155897 +155898 POINT(41.16851693705957 74.73960529487921) bank155898 +155899 POINT(41.16401730857641 74.93440601576705) bank155899 +155900 POINT(40.39846727439738 74.86758355372419) bank155900 +155901 POINT(41.15498169739511 73.67834426977319) bank155901 +155902 POINT(39.82048856714539 73.97231349916665) bank155902 +155903 POINT(40.40770345386181 74.59030676352678) bank155903 +155904 POINT(40.615492136484534 73.01265384169936) bank155904 +155905 POINT(41.39379509262288 73.07873898221433) bank155905 +155906 POINT(41.25363133306364 74.26293596278624) bank155906 +155907 POINT(40.52794123487555 73.5373615529865) bank155907 +155908 POINT(41.170342637382205 73.67839113945891) bank155908 +155909 POINT(40.43449979769224 74.88472034309655) bank155909 +155910 POINT(40.71811521373131 73.36289907583827) bank155910 +155911 POINT(40.19029268147865 73.85589288425705) bank155911 +155912 POINT(40.545441539328394 73.85515333639682) bank155912 +155913 POINT(39.75765365597126 73.48353470875868) bank155913 +155914 POINT(41.52883064068825 74.27961700865133) bank155914 +155915 POINT(41.14977709746543 74.83102272496218) bank155915 +155916 POINT(40.43190916300282 73.00845948431345) bank155916 +155917 POINT(40.39732427957828 74.4185216402523) bank155917 +155918 POINT(41.4606889841968 73.13402268914298) bank155918 +155919 POINT(40.39946770908574 73.31342114793742) bank155919 +155920 POINT(41.18283706853295 74.83228594817298) bank155920 +155921 POINT(41.671936131774665 73.98549998682164) bank155921 +155922 POINT(40.56011709521094 73.53695504631445) bank155922 +155923 POINT(40.216671780119725 74.29167461629228) bank155923 +155924 POINT(40.741821779745294 73.50142920009444) bank155924 +155925 POINT(40.157234018856585 73.66065719751305) bank155925 +155926 POINT(40.47931573629606 74.90288002952644) bank155926 +155927 POINT(41.22555518524101 73.45880263729455) bank155927 +155928 POINT(39.94811610452448 74.42283587293747) bank155928 +155929 POINT(41.55991344884267 74.73775160038912) bank155929 +155930 POINT(40.70616290926981 74.07724406701296) bank155930 +155931 POINT(40.157431327852976 73.6779573440754) bank155931 +155932 POINT(41.34063154174477 74.21620295185932) bank155932 +155933 POINT(40.797455050510244 73.2790813924947) bank155933 +155934 POINT(40.42411515629451 74.9006069842837) bank155934 +155935 POINT(40.59112423330931 73.87101840584829) bank155935 +155936 POINT(40.92474168029157 73.79030707937208) bank155936 +155937 POINT(41.296914586578644 74.98678083445073) bank155937 +155938 POINT(40.42255357295677 74.2125688290569) bank155938 +155939 POINT(40.236355491226846 74.43870450408436) bank155939 +155940 POINT(40.90933156068722 74.53202870107543) bank155940 +155941 POINT(40.7267512157179 73.66180235808905) bank155941 +155942 POINT(40.46519714046904 73.067342659222) bank155942 +155943 POINT(40.25769840750786 74.32771859288795) bank155943 +155944 POINT(40.91924567389009 73.72523901148581) bank155944 +155945 POINT(40.87362419693848 74.2033420414196) bank155945 +155946 POINT(41.570738693662605 73.52472468814547) bank155946 +155947 POINT(40.37662252744076 73.73894019739326) bank155947 +155948 POINT(41.656685324227304 74.99766325609359) bank155948 +155949 POINT(40.02924760501916 73.82908304094683) bank155949 +155950 POINT(40.39515101232475 74.47136136522977) bank155950 +155951 POINT(41.35441006588611 73.97941431046885) bank155951 +155952 POINT(40.38482649070324 74.4837111395706) bank155952 +155953 POINT(41.34967782856276 73.61033260348178) bank155953 +155954 POINT(40.58796121794954 74.78214517288492) bank155954 +155955 POINT(40.49413115766533 74.62105188585913) bank155955 +155956 POINT(41.5500827738298 73.32936447402938) bank155956 +155957 POINT(41.0609855522506 73.75281688628478) bank155957 +155958 POINT(39.91128310942535 74.61067354445765) bank155958 +155959 POINT(40.3162160783735 73.98168495734686) bank155959 +155960 POINT(41.38303067334229 74.5767584260481) bank155960 +155961 POINT(40.04126981330657 73.31027118589397) bank155961 +155962 POINT(40.455781155901136 74.16879284987891) bank155962 +155963 POINT(41.58532751613622 73.94490331058131) bank155963 +155964 POINT(39.84770880415768 74.99367001966901) bank155964 +155965 POINT(40.671639226296314 73.52309926888839) bank155965 +155966 POINT(41.444863741449616 74.86854241269393) bank155966 +155967 POINT(40.16815107116867 73.17008434624087) bank155967 +155968 POINT(39.83158198077074 73.62755381087455) bank155968 +155969 POINT(41.051142157850244 73.74115164701061) bank155969 +155970 POINT(40.83925777848381 73.31051824022362) bank155970 +155971 POINT(40.7617296212601 73.5294172032009) bank155971 +155972 POINT(41.508480176786186 74.23798509288655) bank155972 +155973 POINT(40.229095267600975 74.62514934470532) bank155973 +155974 POINT(40.87778801603263 74.89321302999228) bank155974 +155975 POINT(40.753531419882435 73.71052369437237) bank155975 +155976 POINT(40.695342255569884 74.99257591085805) bank155976 +155977 POINT(41.51802344443685 74.77619496661515) bank155977 +155978 POINT(40.733212335102074 74.56922281000932) bank155978 +155979 POINT(40.12224926491722 74.07134547055543) bank155979 +155980 POINT(41.529896516283955 73.15300985462761) bank155980 +155981 POINT(41.47535142818804 74.17153094113227) bank155981 +155982 POINT(41.21002394495651 74.81259983089076) bank155982 +155983 POINT(39.875892975057944 73.82846207191037) bank155983 +155984 POINT(40.80010771730007 74.67108066430983) bank155984 +155985 POINT(39.82772320797972 74.67741644822661) bank155985 +155986 POINT(40.37699368648953 74.36573793092066) bank155986 +155987 POINT(41.54631637964026 73.68437727678241) bank155987 +155988 POINT(41.30003890737355 73.94178874878472) bank155988 +155989 POINT(41.49945606348938 73.33767175399363) bank155989 +155990 POINT(39.974198493727606 74.97032532593194) bank155990 +155991 POINT(40.3044462435859 73.01695702998852) bank155991 +155992 POINT(41.65741371547913 74.10164003205384) bank155992 +155993 POINT(40.127586256259704 74.45686542169696) bank155993 +155994 POINT(40.67871438672148 74.5141777165233) bank155994 +155995 POINT(39.844547935378465 73.1539756141391) bank155995 +155996 POINT(40.32698438956318 73.76559869307707) bank155996 +155997 POINT(40.636663873177056 73.5449535852034) bank155997 +155998 POINT(40.380060283283896 74.01197077491906) bank155998 +155999 POINT(40.47098118555108 73.14562888182537) bank155999 +156000 POINT(40.57641625064907 74.27800979505865) bank156000 +156001 POINT(41.42713850560719 74.19197029324138) bank156001 +156002 POINT(39.833649379999116 74.18029171188579) bank156002 +156003 POINT(41.61166151074802 73.08034222665688) bank156003 +156004 POINT(40.5994202568191 73.82703511532883) bank156004 +156005 POINT(40.821644547585215 74.3864863256862) bank156005 +156006 POINT(40.31707223060733 73.46443133064933) bank156006 +156007 POINT(41.00380137014933 74.58865820419756) bank156007 +156008 POINT(40.29128361431197 74.35292029905528) bank156008 +156009 POINT(40.63124163298341 73.22720788025885) bank156009 +156010 POINT(40.28554357311725 73.76572280590794) bank156010 +156011 POINT(41.546290666800026 73.37020456266085) bank156011 +156012 POINT(40.024504366287466 73.20499021800183) bank156012 +156013 POINT(41.358118010378924 73.67506863217918) bank156013 +156014 POINT(40.506339732240136 74.58674339640866) bank156014 +156015 POINT(40.07093196100454 74.82509406374794) bank156015 +156016 POINT(40.88953926553292 74.03215744446982) bank156016 +156017 POINT(40.72934241008535 73.74150511201091) bank156017 +156018 POINT(40.93840978166377 74.95629219110981) bank156018 +156019 POINT(41.30125076207714 73.18737947657284) bank156019 +156020 POINT(39.87606426802248 73.14893782965926) bank156020 +156021 POINT(40.54976515608918 74.34678589962641) bank156021 +156022 POINT(39.868067009187364 74.78389671999639) bank156022 +156023 POINT(40.890369304482235 73.69559493637308) bank156023 +156024 POINT(41.298383235863554 73.97983454875822) bank156024 +156025 POINT(39.756684824650996 74.10060246218002) bank156025 +156026 POINT(39.8982857823457 74.87141202742421) bank156026 +156027 POINT(41.3628885319282 74.3475466344309) bank156027 +156028 POINT(41.626661552259144 73.4798117618438) bank156028 +156029 POINT(41.186312603466746 73.10768464451151) bank156029 +156030 POINT(39.84255784200467 73.1478838313192) bank156030 +156031 POINT(40.44660982918364 73.86642862234302) bank156031 +156032 POINT(40.87450346150125 74.63070229933258) bank156032 +156033 POINT(41.17415773376495 73.96099201594487) bank156033 +156034 POINT(40.66476600056593 73.64001592468757) bank156034 +156035 POINT(41.280238526955905 73.07743294251199) bank156035 +156036 POINT(41.488186620565074 74.36258285194482) bank156036 +156037 POINT(39.910880608561314 74.89498915032905) bank156037 +156038 POINT(41.42302477115644 74.1631460958427) bank156038 +156039 POINT(41.55654020465615 73.02677936783721) bank156039 +156040 POINT(41.50455162705368 73.66160642664799) bank156040 +156041 POINT(41.38485967664097 74.50887203775588) bank156041 +156042 POINT(40.55157674567406 74.85306883622451) bank156042 +156043 POINT(40.45239427476568 74.53328305105114) bank156043 +156044 POINT(41.27766569553645 74.58159833691222) bank156044 +156045 POINT(40.03745635262223 73.08662270118354) bank156045 +156046 POINT(40.883532697885734 74.62247278023162) bank156046 +156047 POINT(40.36503966998671 73.09890044256615) bank156047 +156048 POINT(39.91682780129826 73.49801808327736) bank156048 +156049 POINT(41.38067044548283 73.66912992796347) bank156049 +156050 POINT(40.33451843258062 73.87133483692563) bank156050 +156051 POINT(40.97075563592817 73.38442276049442) bank156051 +156052 POINT(41.08242910831766 74.62948210926504) bank156052 +156053 POINT(41.367985714471125 73.82620075142266) bank156053 +156054 POINT(40.13871933640733 73.46782261265066) bank156054 +156055 POINT(40.59203603007497 74.0919286596276) bank156055 +156056 POINT(40.859305235881884 74.14184962152176) bank156056 +156057 POINT(39.76455736223581 74.14397722452279) bank156057 +156058 POINT(41.04525895993384 74.90064924336167) bank156058 +156059 POINT(40.70449158238343 74.80665368128643) bank156059 +156060 POINT(39.80694701263771 74.44035196390988) bank156060 +156061 POINT(40.37661699725264 74.86178924310033) bank156061 +156062 POINT(40.31115590854733 74.82443573764591) bank156062 +156063 POINT(40.62161196125252 73.84575217924072) bank156063 +156064 POINT(40.10629484858236 74.5926279126053) bank156064 +156065 POINT(40.46706678873169 73.51517688993215) bank156065 +156066 POINT(41.639830954114274 73.79883390160636) bank156066 +156067 POINT(41.62123090694904 73.77303354004385) bank156067 +156068 POINT(41.0644063982858 74.59395610338964) bank156068 +156069 POINT(41.57638197524156 73.15271508873418) bank156069 +156070 POINT(40.03179923023699 74.7885414465047) bank156070 +156071 POINT(41.508991962903515 74.00931178558189) bank156071 +156072 POINT(41.04134266373847 74.83299531103313) bank156072 +156073 POINT(41.45140960214266 74.61121748673311) bank156073 +156074 POINT(40.78137205178346 73.56927223664769) bank156074 +156075 POINT(41.119493487785775 74.68489291933828) bank156075 +156076 POINT(40.06732429349454 73.81329388601662) bank156076 +156077 POINT(40.09320101255392 74.59802567958216) bank156077 +156078 POINT(40.0341622102272 74.00893338173458) bank156078 +156079 POINT(41.70459811351628 74.20377802273919) bank156079 +156080 POINT(39.82866834932965 73.73100076630578) bank156080 +156081 POINT(39.8435482515255 73.99985583145441) bank156081 +156082 POINT(41.10598467326222 73.04818099020794) bank156082 +156083 POINT(41.65867216094733 73.14881083003512) bank156083 +156084 POINT(40.764838178635905 73.30220911739221) bank156084 +156085 POINT(40.931376302498826 74.20289972728496) bank156085 +156086 POINT(41.131005680096266 74.95563018725632) bank156086 +156087 POINT(40.01636369310178 74.50541387866214) bank156087 +156088 POINT(39.75116526299971 73.23899097315233) bank156088 +156089 POINT(41.20605204165456 74.37090394673974) bank156089 +156090 POINT(39.87340240318042 73.26436691286054) bank156090 +156091 POINT(40.811968863272654 73.129230941303) bank156091 +156092 POINT(40.51802108952184 73.08229135530217) bank156092 +156093 POINT(40.208145916498864 74.81387440126777) bank156093 +156094 POINT(41.53044684269979 74.54088491567329) bank156094 +156095 POINT(40.97799716840454 73.92318228864772) bank156095 +156096 POINT(39.96816412493682 73.41016710956968) bank156096 +156097 POINT(40.98436002462124 74.50500026760467) bank156097 +156098 POINT(41.24149276477831 74.69543148165103) bank156098 +156099 POINT(40.785792173788565 74.81511832008242) bank156099 +156100 POINT(41.393063552172066 74.43107736864883) bank156100 +156101 POINT(40.71228884102743 74.09009107645416) bank156101 +156102 POINT(41.19021152829096 74.68752245656982) bank156102 +156103 POINT(40.37087366718797 74.74996646394952) bank156103 +156104 POINT(41.32149977616288 73.42360769461209) bank156104 +156105 POINT(40.28274398910908 73.24459177685439) bank156105 +156106 POINT(40.44574238003361 73.27114738044078) bank156106 +156107 POINT(39.80347926901725 73.48237329205341) bank156107 +156108 POINT(40.59657868134266 73.99201813285919) bank156108 +156109 POINT(40.72734903079389 73.7524496948681) bank156109 +156110 POINT(40.414674905944956 73.65902035281239) bank156110 +156111 POINT(40.12556964163417 74.45114571190005) bank156111 +156112 POINT(41.26181922126924 73.89119088135612) bank156112 +156113 POINT(40.52435266915083 74.30094301723845) bank156113 +156114 POINT(41.174643106284236 74.92685745240951) bank156114 +156115 POINT(41.59569105668306 73.25697838144734) bank156115 +156116 POINT(40.119009471382114 73.2132265156643) bank156116 +156117 POINT(40.70064254313381 73.96191119222536) bank156117 +156118 POINT(41.53996238637917 73.34601584363145) bank156118 +156119 POINT(39.88561869480462 73.47948176247175) bank156119 +156120 POINT(40.6633509468082 74.49171102477752) bank156120 +156121 POINT(41.03628611519734 73.32247974271095) bank156121 +156122 POINT(41.61727388681769 73.40399379219583) bank156122 +156123 POINT(41.13381407575438 74.3866998049782) bank156123 +156124 POINT(40.20753086247037 73.92413051890078) bank156124 +156125 POINT(41.266642828744565 73.87250914949296) bank156125 +156126 POINT(41.39369098487921 74.2385522746142) bank156126 +156127 POINT(41.146263217302085 74.44261066611318) bank156127 +156128 POINT(41.17375925794276 74.1919670403199) bank156128 +156129 POINT(40.46059020101487 73.40033984266225) bank156129 +156130 POINT(40.49678013536281 74.10142846524107) bank156130 +156131 POINT(41.343566171776494 74.48594821029411) bank156131 +156132 POINT(40.40751982993595 74.97884490487236) bank156132 +156133 POINT(41.228611321191046 73.67421203220617) bank156133 +156134 POINT(40.58486754833121 73.6370045307264) bank156134 +156135 POINT(40.75164206165864 74.54444822143574) bank156135 +156136 POINT(40.77425393285768 73.67527929170377) bank156136 +156137 POINT(40.01121791534369 73.14715439133758) bank156137 +156138 POINT(40.09838058144027 74.62984243825619) bank156138 +156139 POINT(41.290898387350445 74.84662663140502) bank156139 +156140 POINT(41.49897157683574 73.59945286086884) bank156140 +156141 POINT(40.03947156728996 74.74768403794415) bank156141 +156142 POINT(40.98946464596343 74.78251970328984) bank156142 +156143 POINT(41.39758270511606 74.54179466060259) bank156143 +156144 POINT(41.29096135766536 73.7810709798792) bank156144 +156145 POINT(41.51524485213812 74.88976318484043) bank156145 +156146 POINT(40.08264216488397 73.90918495977913) bank156146 +156147 POINT(40.43222728057398 74.15080722217219) bank156147 +156148 POINT(40.71204215768111 73.55891742160249) bank156148 +156149 POINT(41.63629634114215 74.67553623271769) bank156149 +156150 POINT(41.12117754784708 73.83727834194777) bank156150 +156151 POINT(41.419348049308404 73.24515103906954) bank156151 +156152 POINT(40.717966687020635 74.17754661988624) bank156152 +156153 POINT(39.96043028264623 73.19192673567275) bank156153 +156154 POINT(40.914563471603955 73.81640963730543) bank156154 +156155 POINT(40.033906852668295 73.8751491101793) bank156155 +156156 POINT(40.60727719642793 73.95400330421464) bank156156 +156157 POINT(40.082822934657756 73.01688759624987) bank156157 +156158 POINT(39.97990800450964 74.72797440915303) bank156158 +156159 POINT(40.57802424049333 73.54891384504838) bank156159 +156160 POINT(41.42082896989103 73.97310512818804) bank156160 +156161 POINT(41.672084750403 74.74066007580505) bank156161 +156162 POINT(41.168862356255 73.57687604922243) bank156162 +156163 POINT(41.56897971496766 73.76925917459566) bank156163 +156164 POINT(40.72510683148597 73.40893996236352) bank156164 +156165 POINT(39.71951613949457 73.17156299290575) bank156165 +156166 POINT(40.567178517736735 73.68621467574063) bank156166 +156167 POINT(41.447162200802744 74.29717506208655) bank156167 +156168 POINT(40.96576253791501 74.21771485412714) bank156168 +156169 POINT(40.806039053723126 73.12726235297063) bank156169 +156170 POINT(40.512260288710266 74.18249427292226) bank156170 +156171 POINT(40.385025921935245 73.39148107955656) bank156171 +156172 POINT(40.59814024004695 73.27880560644586) bank156172 +156173 POINT(41.31783983520506 74.64303324859429) bank156173 +156174 POINT(39.7527690358979 73.23079416446697) bank156174 +156175 POINT(41.37700103716835 74.15067752382373) bank156175 +156176 POINT(40.38397015709858 74.1995534246044) bank156176 +156177 POINT(40.681941684907756 74.26804737286616) bank156177 +156178 POINT(40.607993761520326 73.31915846264275) bank156178 +156179 POINT(40.613289743322035 74.02933031249643) bank156179 +156180 POINT(40.52301358666611 74.7723099765768) bank156180 +156181 POINT(41.19309325365703 73.27251176205394) bank156181 +156182 POINT(41.3443932954366 73.6822142526664) bank156182 +156183 POINT(41.22608168180379 74.90038927365775) bank156183 +156184 POINT(40.40558724839411 73.69720284179651) bank156184 +156185 POINT(40.558125290001605 73.99797445826707) bank156185 +156186 POINT(40.70687338515345 74.83747442024936) bank156186 +156187 POINT(41.64340863723508 74.90197680935893) bank156187 +156188 POINT(40.62895600667781 73.48433638825253) bank156188 +156189 POINT(41.31103937819288 73.28111993546828) bank156189 +156190 POINT(40.47198090795204 74.37441563514932) bank156190 +156191 POINT(41.3014910975293 74.98694156353775) bank156191 +156192 POINT(40.129725736478264 74.03257755775343) bank156192 +156193 POINT(40.819395416356095 74.63013495267231) bank156193 +156194 POINT(40.291636713099805 74.71766602601166) bank156194 +156195 POINT(40.93131666274368 73.5217893900931) bank156195 +156196 POINT(39.75309903971092 74.40353222194042) bank156196 +156197 POINT(40.22198428271521 74.68492253436759) bank156197 +156198 POINT(41.61687951262809 73.07409374641135) bank156198 +156199 POINT(40.53109467983609 74.44286081180019) bank156199 +156200 POINT(41.37756051955594 73.70401052812095) bank156200 +156201 POINT(40.21680896673146 74.51796636892628) bank156201 +156202 POINT(41.20778851665449 74.04088213472052) bank156202 +156203 POINT(40.76254521887292 73.63181674164413) bank156203 +156204 POINT(41.20549902443611 74.25346339999517) bank156204 +156205 POINT(41.08449748024948 73.95549595576881) bank156205 +156206 POINT(41.08956990774828 73.53351164504136) bank156206 +156207 POINT(41.129063746927706 73.04302226443103) bank156207 +156208 POINT(39.95584678900051 73.78811204163108) bank156208 +156209 POINT(40.72302793035258 73.17198448192998) bank156209 +156210 POINT(41.53806946373814 74.27152351963755) bank156210 +156211 POINT(40.25368104680433 74.0100074777134) bank156211 +156212 POINT(40.43069291256705 74.4816632536145) bank156212 +156213 POINT(40.515086823875976 74.73769880181636) bank156213 +156214 POINT(41.512335565708604 74.13958467165342) bank156214 +156215 POINT(40.681609583231776 73.56119537637552) bank156215 +156216 POINT(41.0292213105145 74.9718469005433) bank156216 +156217 POINT(40.538632170927244 74.04401557446006) bank156217 +156218 POINT(41.58284305966112 74.35999374034017) bank156218 +156219 POINT(40.54014023279194 74.46903085694753) bank156219 +156220 POINT(40.66239244722904 74.03465544473532) bank156220 +156221 POINT(40.504501413582034 73.38463558838177) bank156221 +156222 POINT(40.952105224182645 73.17692946639112) bank156222 +156223 POINT(40.034726490046545 74.19961432188919) bank156223 +156224 POINT(40.87037872352344 73.29571135849622) bank156224 +156225 POINT(39.72712272456942 74.16195798549458) bank156225 +156226 POINT(40.658497570705904 73.20650335191289) bank156226 +156227 POINT(41.364949343579035 74.76398947142336) bank156227 +156228 POINT(40.611953319688766 74.3311516110356) bank156228 +156229 POINT(41.32710570222618 74.22770108465876) bank156229 +156230 POINT(41.28744209751486 74.00637444695761) bank156230 +156231 POINT(40.868692000976 73.21645122558365) bank156231 +156232 POINT(40.626238567117404 73.70025369340348) bank156232 +156233 POINT(40.05175949944335 74.9626942967343) bank156233 +156234 POINT(41.425782395437224 73.09129357032629) bank156234 +156235 POINT(40.13802438508035 74.81688093277592) bank156235 +156236 POINT(40.68711669146703 74.13035276416625) bank156236 +156237 POINT(41.1489644580876 73.40326466394838) bank156237 +156238 POINT(41.258654012090155 73.43799984509486) bank156238 +156239 POINT(40.40777274726427 73.61120790461749) bank156239 +156240 POINT(41.63602318735184 73.31008701107338) bank156240 +156241 POINT(41.10168831413658 73.65285530289572) bank156241 +156242 POINT(41.29753120188315 73.79692124694121) bank156242 +156243 POINT(41.662131908172015 73.78095802314603) bank156243 +156244 POINT(40.54856063150071 74.63031582434995) bank156244 +156245 POINT(39.85051066301026 73.73979132202831) bank156245 +156246 POINT(40.78279904920533 73.5021627886624) bank156246 +156247 POINT(41.58623559780211 74.829790686185) bank156247 +156248 POINT(41.52097276157029 74.77724336284165) bank156248 +156249 POINT(40.986095979821116 73.33919112856013) bank156249 +156250 POINT(41.39664194494613 73.85492981209968) bank156250 +156251 POINT(39.99527815362336 74.05451323984006) bank156251 +156252 POINT(41.223686890891436 73.25834512731565) bank156252 +156253 POINT(39.977711320836974 73.14728318145713) bank156253 +156254 POINT(40.6212635027965 74.66823884554708) bank156254 +156255 POINT(40.47082257566907 74.44062534798236) bank156255 +156256 POINT(40.01078360258282 73.41356199231579) bank156256 +156257 POINT(40.59713750624393 73.77092243537584) bank156257 +156258 POINT(41.16515789201239 73.63832816931708) bank156258 +156259 POINT(40.436491886931236 74.44054020140011) bank156259 +156260 POINT(39.7686921676407 74.12632442902041) bank156260 +156261 POINT(40.67903590284284 73.10103880815197) bank156261 +156262 POINT(40.01140897315361 74.9700166226976) bank156262 +156263 POINT(41.318084003655514 74.2480677009814) bank156263 +156264 POINT(40.30650279210585 73.25721316726232) bank156264 +156265 POINT(40.314963828933045 73.05845069633567) bank156265 +156266 POINT(39.94112329292936 73.33914151906349) bank156266 +156267 POINT(40.75198698414734 74.52602678037309) bank156267 +156268 POINT(40.40201938403381 74.54112192506003) bank156268 +156269 POINT(41.162257690027175 74.26249161219909) bank156269 +156270 POINT(40.31793399759492 74.57521289500505) bank156270 +156271 POINT(41.70012538636427 74.74663008110593) bank156271 +156272 POINT(41.27939058496886 73.791918354234) bank156272 +156273 POINT(40.89272595034105 74.87632241447388) bank156273 +156274 POINT(40.21731266708298 73.08281317990212) bank156274 +156275 POINT(40.319895726761935 73.1145832527901) bank156275 +156276 POINT(41.17765044983948 74.82443641918864) bank156276 +156277 POINT(40.085004622159175 74.10483445025989) bank156277 +156278 POINT(39.77793928947041 73.52695762816201) bank156278 +156279 POINT(41.42968779042575 73.54336915619096) bank156279 +156280 POINT(41.19045804041952 74.51922642694845) bank156280 +156281 POINT(39.97487007202143 74.55921972756356) bank156281 +156282 POINT(40.047830351136376 74.37929381192951) bank156282 +156283 POINT(40.79727181021453 73.88599062248944) bank156283 +156284 POINT(40.52491147157942 73.17107982428743) bank156284 +156285 POINT(40.986193873602275 74.75429496197762) bank156285 +156286 POINT(39.81580753134611 73.64880506135573) bank156286 +156287 POINT(40.872258165582714 73.29806186734952) bank156287 +156288 POINT(41.49921307987363 74.77046974550714) bank156288 +156289 POINT(40.01997565676988 74.88708056225282) bank156289 +156290 POINT(41.39983853132778 74.18971725210405) bank156290 +156291 POINT(40.82510085205961 74.38891585028574) bank156291 +156292 POINT(40.709711465598126 73.43448167226623) bank156292 +156293 POINT(41.62870774271289 74.57004683816723) bank156293 +156294 POINT(40.613339990858655 73.29027312680715) bank156294 +156295 POINT(41.0347817958522 73.3371036707399) bank156295 +156296 POINT(40.18134122586564 73.79466352687609) bank156296 +156297 POINT(41.63954976494979 73.52739662198292) bank156297 +156298 POINT(40.166280256167276 73.5622818201408) bank156298 +156299 POINT(40.98624502721788 74.56004928974212) bank156299 +156300 POINT(39.80551298521007 73.27167186847191) bank156300 +156301 POINT(40.996927561047116 73.11063432128985) bank156301 +156302 POINT(40.53130312012055 74.47354666250207) bank156302 +156303 POINT(40.835017205119684 74.22580179963737) bank156303 +156304 POINT(40.731137216308525 73.8897319867159) bank156304 +156305 POINT(41.426447594952805 74.55420819453533) bank156305 +156306 POINT(39.920499991231566 73.12657746073401) bank156306 +156307 POINT(41.69428688488447 73.45793884760286) bank156307 +156308 POINT(41.038274386731025 74.84631773276033) bank156308 +156309 POINT(40.900819786651816 74.34969167401354) bank156309 +156310 POINT(40.458803646154536 73.58712012052086) bank156310 +156311 POINT(41.12826569808948 73.92427055405408) bank156311 +156312 POINT(40.61967645790491 74.20474126598026) bank156312 +156313 POINT(41.44997346032711 74.36865402644621) bank156313 +156314 POINT(40.972758359820176 73.16086738182302) bank156314 +156315 POINT(40.293648540155594 73.21734500493532) bank156315 +156316 POINT(39.83347768720825 74.0564815516292) bank156316 +156317 POINT(41.399567907671475 74.42714654983025) bank156317 +156318 POINT(40.05526006725863 73.26655151776515) bank156318 +156319 POINT(40.086896171982964 74.30182508650066) bank156319 +156320 POINT(41.210328403529154 73.49533578668397) bank156320 +156321 POINT(41.05953200654984 74.50632653137902) bank156321 +156322 POINT(41.48364042792779 74.40925362862879) bank156322 +156323 POINT(40.72259786115117 74.10944839831667) bank156323 +156324 POINT(40.85232927127327 73.27231750115726) bank156324 +156325 POINT(40.60150746192363 74.32334440433841) bank156325 +156326 POINT(41.625752913567354 73.34966761863807) bank156326 +156327 POINT(40.819773664386226 74.90034581706675) bank156327 +156328 POINT(41.17600713982849 73.89452465885094) bank156328 +156329 POINT(39.88544968350812 74.93340748224666) bank156329 +156330 POINT(40.66368136801796 74.91916605076531) bank156330 +156331 POINT(41.350165949478146 74.17602671451525) bank156331 +156332 POINT(40.27558654470004 74.98423499132996) bank156332 +156333 POINT(39.82268752535327 74.46999357139713) bank156333 +156334 POINT(40.54005721869591 74.16394341050939) bank156334 +156335 POINT(40.01829749307403 73.72090220086301) bank156335 +156336 POINT(40.42501783680312 74.7389122776664) bank156336 +156337 POINT(39.85321355361107 73.60541350340104) bank156337 +156338 POINT(40.33085641876126 74.4780804632868) bank156338 +156339 POINT(40.006834854675176 73.62479018847927) bank156339 +156340 POINT(41.56433264831443 74.5248279968722) bank156340 +156341 POINT(41.20085566020593 73.43228711923892) bank156341 +156342 POINT(40.513656952686915 73.00963088259755) bank156342 +156343 POINT(40.57801761290584 74.10688042016274) bank156343 +156344 POINT(39.9116974521758 73.34372330877389) bank156344 +156345 POINT(40.86021216142675 74.07571643392939) bank156345 +156346 POINT(39.94403212287881 74.40636267182316) bank156346 +156347 POINT(40.49533172459514 73.98049727846325) bank156347 +156348 POINT(40.81877767263883 74.78354255917654) bank156348 +156349 POINT(40.896995532259616 74.25371785547178) bank156349 +156350 POINT(41.30096663260022 74.85228298549275) bank156350 +156351 POINT(39.98422953988233 74.94893518887629) bank156351 +156352 POINT(40.05829550087919 74.46479316513717) bank156352 +156353 POINT(40.81685032123223 73.5025667974136) bank156353 +156354 POINT(40.54901720007959 73.9211368998548) bank156354 +156355 POINT(41.11238534345939 73.57695137889965) bank156355 +156356 POINT(40.80110399044317 74.9210682062875) bank156356 +156357 POINT(41.38757557977627 74.112029590974) bank156357 +156358 POINT(41.033280243834405 73.47134563828044) bank156358 +156359 POINT(41.24664362259623 73.51234628767418) bank156359 +156360 POINT(39.74525914968623 73.50150566470468) bank156360 +156361 POINT(41.21620775144519 73.47003819757681) bank156361 +156362 POINT(40.0780252934486 73.19257180046733) bank156362 +156363 POINT(40.364319991936256 74.82050499980477) bank156363 +156364 POINT(41.54326780847345 74.81774584904886) bank156364 +156365 POINT(41.3298720389115 73.8381752985976) bank156365 +156366 POINT(40.80690574236889 74.31431963655035) bank156366 +156367 POINT(40.126646160631665 73.73706325127041) bank156367 +156368 POINT(40.02353725187958 73.29435588426027) bank156368 +156369 POINT(41.05166198755571 73.57456103947453) bank156369 +156370 POINT(41.40435867982281 74.48037187833147) bank156370 +156371 POINT(41.659261889301725 74.09119190891886) bank156371 +156372 POINT(40.74331492231618 74.25435671597242) bank156372 +156373 POINT(40.014932169862064 74.40466373651928) bank156373 +156374 POINT(40.59269065496996 73.45409605105557) bank156374 +156375 POINT(40.157665078442776 74.1467606271244) bank156375 +156376 POINT(40.10008047145136 73.88937598416177) bank156376 +156377 POINT(40.128877610700634 73.02036681024042) bank156377 +156378 POINT(40.92226024733201 73.51533054475894) bank156378 +156379 POINT(41.57673745830171 74.80459478929879) bank156379 +156380 POINT(40.81474582868856 73.61682073937607) bank156380 +156381 POINT(40.367886580872536 74.60497760694057) bank156381 +156382 POINT(39.90980019900289 73.19896525859609) bank156382 +156383 POINT(39.78311801873444 74.98493597326744) bank156383 +156384 POINT(41.39617425914207 74.62645407159563) bank156384 +156385 POINT(41.48791181079642 74.94391918778106) bank156385 +156386 POINT(40.77944515734388 73.50311802572718) bank156386 +156387 POINT(39.91206864494021 74.72962420510505) bank156387 +156388 POINT(40.756644427723295 74.38672554591605) bank156388 +156389 POINT(40.96003450111131 73.39700064930491) bank156389 +156390 POINT(40.89009519036004 74.62859257447336) bank156390 +156391 POINT(40.843641194407084 73.99034617751494) bank156391 +156392 POINT(40.691097621362836 73.67378065648953) bank156392 +156393 POINT(41.41434785317385 74.52847061474131) bank156393 +156394 POINT(41.70412433945726 73.13858537419857) bank156394 +156395 POINT(41.288967438061114 73.74242003942902) bank156395 +156396 POINT(40.36910973902331 74.07373271297881) bank156396 +156397 POINT(40.76972125378788 73.55288832406838) bank156397 +156398 POINT(41.05481551234343 73.51683305898605) bank156398 +156399 POINT(40.84692590137895 74.12905430482235) bank156399 +156400 POINT(40.13882598118174 74.90791106525167) bank156400 +156401 POINT(40.6730607650929 73.03546444734303) bank156401 +156402 POINT(41.18796400937742 73.47028321863863) bank156402 +156403 POINT(41.39971023550706 74.67737077134804) bank156403 +156404 POINT(41.68424125882216 73.78094673993621) bank156404 +156405 POINT(39.9734907876934 74.68364477424248) bank156405 +156406 POINT(40.41627260801987 74.98643208236768) bank156406 +156407 POINT(40.45632436838336 74.70430814225297) bank156407 +156408 POINT(40.81456444947107 74.20542230319407) bank156408 +156409 POINT(40.18033739894252 74.82229405195704) bank156409 +156410 POINT(40.342202054425556 73.03023487154799) bank156410 +156411 POINT(40.59509512385597 73.51257893798763) bank156411 +156412 POINT(41.209448249598736 73.19845982458354) bank156412 +156413 POINT(39.93056975267637 74.45133544531714) bank156413 +156414 POINT(40.70649984672809 73.40017280948611) bank156414 +156415 POINT(40.09134685670188 74.7927854006269) bank156415 +156416 POINT(41.51114794668594 74.49245700349549) bank156416 +156417 POINT(39.92595822763002 74.21615511421732) bank156417 +156418 POINT(39.86617243067449 73.13928393354772) bank156418 +156419 POINT(40.977344009065604 74.73481125053988) bank156419 +156420 POINT(41.51206638394469 74.35399875503145) bank156420 +156421 POINT(41.0826322537081 73.13249040016879) bank156421 +156422 POINT(41.272266556169804 74.94586368656879) bank156422 +156423 POINT(40.48302734333424 73.87483157769273) bank156423 +156424 POINT(41.41651697323725 73.16103668988252) bank156424 +156425 POINT(39.84127493101482 74.1809356133417) bank156425 +156426 POINT(40.95509117596001 74.50439067008122) bank156426 +156427 POINT(41.53664275581641 73.17003244247738) bank156427 +156428 POINT(40.63812812649472 74.89330597356091) bank156428 +156429 POINT(40.01392772346375 74.77241020600336) bank156429 +156430 POINT(40.942216167658735 74.73253611881411) bank156430 +156431 POINT(39.800922538327434 74.05246285973023) bank156431 +156432 POINT(41.14670712200309 73.56373945429266) bank156432 +156433 POINT(41.604656051207954 74.5945079391652) bank156433 +156434 POINT(40.23374314151783 74.282779338285) bank156434 +156435 POINT(41.66997703607308 73.91970716368854) bank156435 +156436 POINT(39.72090998595028 73.64184044879534) bank156436 +156437 POINT(40.72802623806119 74.01367656996777) bank156437 +156438 POINT(40.343238572300265 73.89638383455078) bank156438 +156439 POINT(41.566705769839636 73.58500018416329) bank156439 +156440 POINT(41.6466074358321 74.10425720587338) bank156440 +156441 POINT(40.308395194606 74.98340450657275) bank156441 +156442 POINT(41.362183072231005 74.2221835685634) bank156442 +156443 POINT(40.75486511264648 74.78541860382971) bank156443 +156444 POINT(41.02431310245132 74.04871547492887) bank156444 +156445 POINT(41.44285737674255 73.54058016242044) bank156445 +156446 POINT(41.658772491794515 74.81209196151758) bank156446 +156447 POINT(40.41036802292852 74.57363673692649) bank156447 +156448 POINT(40.10523725292015 74.64770115653157) bank156448 +156449 POINT(41.66037223209285 73.35240075535938) bank156449 +156450 POINT(41.596591001113616 73.92476485526187) bank156450 +156451 POINT(41.05796364149483 73.55580139475681) bank156451 +156452 POINT(40.55862058150242 74.19343927679355) bank156452 +156453 POINT(41.19982247479439 73.26858205617002) bank156453 +156454 POINT(41.461251256360846 73.66323796153216) bank156454 +156455 POINT(41.475328561517486 73.3386382715798) bank156455 +156456 POINT(40.62326391189033 73.80541674926013) bank156456 +156457 POINT(40.503154254723306 73.47878741881179) bank156457 +156458 POINT(40.37384547914063 73.2783161339888) bank156458 +156459 POINT(39.951395592302205 74.66699358766142) bank156459 +156460 POINT(40.12466089866078 73.585244620662) bank156460 +156461 POINT(40.48649871384115 74.78714396138514) bank156461 +156462 POINT(40.014479601890685 73.84426448753314) bank156462 +156463 POINT(41.66079805713318 74.62526375494723) bank156463 +156464 POINT(39.78184320073479 74.2576875899498) bank156464 +156465 POINT(40.59095527841974 74.98898047461336) bank156465 +156466 POINT(41.1286325625673 74.03595408590292) bank156466 +156467 POINT(41.495460495058 74.0326520374036) bank156467 +156468 POINT(40.92186990589433 74.88348624503364) bank156468 +156469 POINT(40.60941024424374 74.97757876078656) bank156469 +156470 POINT(39.79462292756783 74.49710107520099) bank156470 +156471 POINT(41.35781188297206 74.70667016334069) bank156471 +156472 POINT(41.037433397441994 74.4469271446245) bank156472 +156473 POINT(40.47659874649639 74.87542727674447) bank156473 +156474 POINT(41.668124267274045 73.77930228627669) bank156474 +156475 POINT(41.63013268875054 74.40646247612018) bank156475 +156476 POINT(41.17795782610356 74.19807537377817) bank156476 +156477 POINT(40.05105892468441 74.6685846083484) bank156477 +156478 POINT(41.644595930846194 74.4156088055219) bank156478 +156479 POINT(40.30525848368959 74.59082767190411) bank156479 +156480 POINT(39.733868759170264 74.54559225324888) bank156480 +156481 POINT(40.19513283345075 73.37035701165229) bank156481 +156482 POINT(39.972041506841116 73.29239906012029) bank156482 +156483 POINT(39.886011528132116 73.55139462143957) bank156483 +156484 POINT(40.84062817787387 74.47085991030548) bank156484 +156485 POINT(40.843537801829285 74.32527776057546) bank156485 +156486 POINT(40.54193452996664 74.40461485530003) bank156486 +156487 POINT(41.602793558232946 73.0810392895983) bank156487 +156488 POINT(41.504106748226846 74.03995149842439) bank156488 +156489 POINT(40.55780518941538 74.51311552864952) bank156489 +156490 POINT(39.84758955098487 73.78003447739455) bank156490 +156491 POINT(40.46342626250458 74.85151372267356) bank156491 +156492 POINT(40.87784928191268 74.02340330240436) bank156492 +156493 POINT(40.61738394137236 74.37839172498082) bank156493 +156494 POINT(41.137534381080734 73.2251374274331) bank156494 +156495 POINT(41.24242639741186 74.68809680420836) bank156495 +156496 POINT(39.743411702620065 73.63340627244291) bank156496 +156497 POINT(40.539543842315624 74.74325802018028) bank156497 +156498 POINT(40.89168432812927 74.59742372910938) bank156498 +156499 POINT(39.73176941098145 73.24315302966049) bank156499 +156500 POINT(40.26783164440157 74.64353041029811) bank156500 +156501 POINT(40.573488310403484 74.06001859855171) bank156501 +156502 POINT(39.94002330271104 73.53231541660655) bank156502 +156503 POINT(41.02632948249351 73.73527167400205) bank156503 +156504 POINT(40.35835459643394 74.57384211063543) bank156504 +156505 POINT(40.30165925785326 74.29345260035815) bank156505 +156506 POINT(40.05979071016624 73.2701878424038) bank156506 +156507 POINT(40.59171172932869 74.69037532915468) bank156507 +156508 POINT(41.0908508963684 73.55620325714256) bank156508 +156509 POINT(40.306037035685044 74.84424813853298) bank156509 +156510 POINT(39.92110682055211 74.17071170550841) bank156510 +156511 POINT(39.87854557921659 73.65429676342609) bank156511 +156512 POINT(41.30054793377277 73.93989774811779) bank156512 +156513 POINT(40.92490965826546 74.93742184318107) bank156513 +156514 POINT(39.95906715638004 73.98590725493032) bank156514 +156515 POINT(41.54339651209363 73.66966373960875) bank156515 +156516 POINT(41.34349765212214 73.23379575416169) bank156516 +156517 POINT(40.882627134226205 74.39810718569909) bank156517 +156518 POINT(41.7054662066392 74.28759935391037) bank156518 +156519 POINT(41.22599462542602 74.45241179662915) bank156519 +156520 POINT(40.51819616559682 73.64827573408184) bank156520 +156521 POINT(40.8561949991478 74.81197132570206) bank156521 +156522 POINT(40.10680140757163 74.803045896027) bank156522 +156523 POINT(40.81142026559725 74.19365716574023) bank156523 +156524 POINT(40.90571580091167 74.21811650360652) bank156524 +156525 POINT(40.78124116291755 74.13490585282192) bank156525 +156526 POINT(41.59642604837704 74.22970861815134) bank156526 +156527 POINT(40.48503230151051 73.25159961629147) bank156527 +156528 POINT(39.839125717281675 73.11115174171887) bank156528 +156529 POINT(39.990950373321894 73.04712600972817) bank156529 +156530 POINT(40.57889578690973 74.84091927728836) bank156530 +156531 POINT(40.45939904321793 73.36542519875084) bank156531 +156532 POINT(41.39166282845112 73.36625055727993) bank156532 +156533 POINT(41.696029851640276 74.7903070674343) bank156533 +156534 POINT(39.75274130706462 73.95050963931665) bank156534 +156535 POINT(40.518459134166655 73.02741127198641) bank156535 +156536 POINT(40.139751184522105 74.76744317640625) bank156536 +156537 POINT(40.74123423462794 73.59201282492648) bank156537 +156538 POINT(39.918075217455964 74.19965966823422) bank156538 +156539 POINT(40.36760866245253 73.75826146836253) bank156539 +156540 POINT(40.94302951217582 74.7097984657278) bank156540 +156541 POINT(41.295000367506496 73.87227474118811) bank156541 +156542 POINT(40.22445122616581 74.95498888012766) bank156542 +156543 POINT(41.22036496975535 74.53138769594234) bank156543 +156544 POINT(41.08760905370595 73.05110498696936) bank156544 +156545 POINT(40.663778714748716 74.24029434918491) bank156545 +156546 POINT(39.750953127330654 74.07242778414475) bank156546 +156547 POINT(39.7359212505593 74.10771144884608) bank156547 +156548 POINT(41.519734899631516 74.7843365657765) bank156548 +156549 POINT(40.155067676473905 73.04409425582229) bank156549 +156550 POINT(41.14532999154311 74.53912090197875) bank156550 +156551 POINT(41.04796961761575 73.97469439915422) bank156551 +156552 POINT(40.57272482486673 74.05131919101804) bank156552 +156553 POINT(40.641616744812836 74.74098578439647) bank156553 +156554 POINT(41.14694621365473 73.53320026948722) bank156554 +156555 POINT(41.37583939116149 73.72078516528327) bank156555 +156556 POINT(40.82557081488495 74.82532387565958) bank156556 +156557 POINT(40.096618686643666 73.78615116369066) bank156557 +156558 POINT(40.854637773857924 73.2643719872504) bank156558 +156559 POINT(40.798968743538396 73.06867021810308) bank156559 +156560 POINT(40.973103186203765 74.63611461002826) bank156560 +156561 POINT(40.739182447312466 74.37551732755081) bank156561 +156562 POINT(41.30702197274849 74.0083200866539) bank156562 +156563 POINT(41.55238706612786 75.00292224981635) bank156563 +156564 POINT(41.17940185374172 73.27556465554338) bank156564 +156565 POINT(41.150122953462834 73.81190566192124) bank156565 +156566 POINT(39.87387715461393 73.43172853016695) bank156566 +156567 POINT(39.819244584485766 74.39996145709044) bank156567 +156568 POINT(41.67709777320174 73.31779067411672) bank156568 +156569 POINT(40.75061827854428 74.73975105377569) bank156569 +156570 POINT(41.22938554143448 73.96579941977747) bank156570 +156571 POINT(41.62809751642941 73.48480542230566) bank156571 +156572 POINT(41.66609317371559 74.32484771023758) bank156572 +156573 POINT(40.032572557444894 74.92203284942754) bank156573 +156574 POINT(41.27064357000198 74.29799997370226) bank156574 +156575 POINT(41.116731327916725 74.71314682341784) bank156575 +156576 POINT(40.497994328906465 74.94062811955001) bank156576 +156577 POINT(40.049716230419 73.21069847244016) bank156577 +156578 POINT(40.62992050713645 73.01509956804792) bank156578 +156579 POINT(40.47181766030566 74.08457627245758) bank156579 +156580 POINT(40.65070761284116 74.34106250871052) bank156580 +156581 POINT(41.69523592140598 74.87817964044953) bank156581 +156582 POINT(40.603010416228315 73.64325884431568) bank156582 +156583 POINT(41.129063537518284 74.82897808174265) bank156583 +156584 POINT(41.65557434907012 73.7197643074005) bank156584 +156585 POINT(40.673478833729675 74.10588343608032) bank156585 +156586 POINT(40.19313786993102 73.34445194361932) bank156586 +156587 POINT(40.21657151467931 74.82977845801695) bank156587 +156588 POINT(41.575633143380756 74.49463125611322) bank156588 +156589 POINT(40.08355112708283 74.84300583689485) bank156589 +156590 POINT(40.34670920138645 74.58548465508179) bank156590 +156591 POINT(41.52580017281835 73.77219978920975) bank156591 +156592 POINT(41.442111761228425 73.94337793188693) bank156592 +156593 POINT(40.45258906801645 73.70896460951673) bank156593 +156594 POINT(41.705834448999546 74.90459126606471) bank156594 +156595 POINT(40.145619404680666 74.44048300343903) bank156595 +156596 POINT(40.89660236028341 74.28115491950055) bank156596 +156597 POINT(41.022553840246054 73.762638583845) bank156597 +156598 POINT(41.109025194456834 74.15402021752396) bank156598 +156599 POINT(41.082094512106046 74.2856244224758) bank156599 +156600 POINT(40.72030335813724 74.54876133821844) bank156600 +156601 POINT(41.69666749655524 74.13813916086798) bank156601 +156602 POINT(40.777293141301755 74.1236438071332) bank156602 +156603 POINT(39.75071949578029 73.71709410155786) bank156603 +156604 POINT(40.45461573047257 74.2500931606087) bank156604 +156605 POINT(40.082872695407616 73.88863902505129) bank156605 +156606 POINT(40.3770119114838 73.04962850129989) bank156606 +156607 POINT(40.93206533303214 74.55502345180149) bank156607 +156608 POINT(39.83274382765814 74.75418109185728) bank156608 +156609 POINT(40.74518375427906 73.91979453711289) bank156609 +156610 POINT(40.75914709232098 73.94364167494057) bank156610 +156611 POINT(41.33193921155689 73.8850669379023) bank156611 +156612 POINT(40.32458960779769 74.1984803706027) bank156612 +156613 POINT(39.832463815323074 73.56982623173083) bank156613 +156614 POINT(40.469051350743726 73.88365995987341) bank156614 +156615 POINT(41.450822048259454 73.39667196713644) bank156615 +156616 POINT(40.13604183576262 73.67668389697323) bank156616 +156617 POINT(40.18702717775657 73.27368298455568) bank156617 +156618 POINT(40.576375446963915 73.52626883666409) bank156618 +156619 POINT(40.30897877539959 74.03653329016316) bank156619 +156620 POINT(40.387635504671906 73.84674963220878) bank156620 +156621 POINT(40.15108286979991 74.11684447456082) bank156621 +156622 POINT(40.49250936888635 74.08974660420179) bank156622 +156623 POINT(41.466748106802754 74.34858403926695) bank156623 +156624 POINT(40.07519573052644 74.63759071992082) bank156624 +156625 POINT(40.61449251009812 74.31158964936118) bank156625 +156626 POINT(41.64163940767202 74.47914410629977) bank156626 +156627 POINT(40.51052001909191 73.07663804081227) bank156627 +156628 POINT(40.30875799904937 73.3515522569412) bank156628 +156629 POINT(41.217127169644016 74.8575177889091) bank156629 +156630 POINT(41.69000993966938 73.2125921904162) bank156630 +156631 POINT(40.6467248129262 74.08761697537682) bank156631 +156632 POINT(41.56148043146875 73.8467986903275) bank156632 +156633 POINT(40.61792117420909 73.58388287339139) bank156633 +156634 POINT(40.761744708774 75.00545348614307) bank156634 +156635 POINT(39.987301016079094 74.93987728741236) bank156635 +156636 POINT(39.97306497940973 73.50852261266107) bank156636 +156637 POINT(40.7769592050719 74.97224949791014) bank156637 +156638 POINT(40.120287757260925 73.82649016906257) bank156638 +156639 POINT(39.975675964809355 73.89080668304227) bank156639 +156640 POINT(40.44388884044096 74.17763071044821) bank156640 +156641 POINT(39.80430701895557 74.93433286597698) bank156641 +156642 POINT(40.563565539826925 73.66810769677204) bank156642 +156643 POINT(40.16353253458539 74.05189930966496) bank156643 +156644 POINT(41.11954960377556 74.27191538100553) bank156644 +156645 POINT(39.95187469371591 73.11468602641959) bank156645 +156646 POINT(40.029755317016516 73.65024740966115) bank156646 +156647 POINT(41.45270978088171 74.63348059056837) bank156647 +156648 POINT(40.412261090927885 73.95719734168777) bank156648 +156649 POINT(41.089695089744055 74.84343236733841) bank156649 +156650 POINT(40.13100831992345 73.59797070905921) bank156650 +156651 POINT(41.03718339726759 73.1516186949828) bank156651 +156652 POINT(40.87080438368844 73.02593528304469) bank156652 +156653 POINT(40.08129021278769 73.77214704172096) bank156653 +156654 POINT(40.21247480809939 74.38094100359336) bank156654 +156655 POINT(41.546115939063775 74.97220038142146) bank156655 +156656 POINT(39.86509746849876 73.04697149318231) bank156656 +156657 POINT(40.25086414876445 74.82588462117795) bank156657 +156658 POINT(41.00250170621286 73.30662214134452) bank156658 +156659 POINT(41.51324104955207 74.0491957587913) bank156659 +156660 POINT(40.17836494496491 73.03687720010772) bank156660 +156661 POINT(40.240738911938955 73.28748949700119) bank156661 +156662 POINT(41.26903777812936 73.12417871193597) bank156662 +156663 POINT(39.83684161815678 73.3843806169313) bank156663 +156664 POINT(39.96889710543995 73.1127003575586) bank156664 +156665 POINT(41.39398033585299 73.23202330167135) bank156665 +156666 POINT(39.95644169978087 73.1327871481744) bank156666 +156667 POINT(40.35280802640649 73.55927167410587) bank156667 +156668 POINT(41.49830752118618 74.17068571995624) bank156668 +156669 POINT(40.27986382589434 73.63204199876077) bank156669 +156670 POINT(39.76059670310396 74.4453429782964) bank156670 +156671 POINT(41.06841761131706 74.55783663073905) bank156671 +156672 POINT(39.98561102733528 73.73393823965465) bank156672 +156673 POINT(41.66958082859765 74.11137368363984) bank156673 +156674 POINT(40.981757932940276 73.13438637780749) bank156674 +156675 POINT(40.23170738194683 74.59666907870908) bank156675 +156676 POINT(40.973384254768874 73.14274152325264) bank156676 +156677 POINT(41.36193318506144 74.28057231323517) bank156677 +156678 POINT(41.34519922825985 73.87395551536804) bank156678 +156679 POINT(40.19110025387164 73.41734485074403) bank156679 +156680 POINT(40.15589679387121 74.34097535823847) bank156680 +156681 POINT(41.09488673438885 74.21546896571968) bank156681 +156682 POINT(40.52659997515717 73.18885004778807) bank156682 +156683 POINT(41.41523603248798 74.75980206363889) bank156683 +156684 POINT(40.31511411683963 73.3343634421808) bank156684 +156685 POINT(40.244155259407755 73.14409606228249) bank156685 +156686 POINT(41.062701991937274 74.08559428821819) bank156686 +156687 POINT(40.506050187176605 74.70841325620368) bank156687 +156688 POINT(39.86797980170499 74.51301326854399) bank156688 +156689 POINT(41.14897128971345 74.29061353766747) bank156689 +156690 POINT(39.83814209804487 74.27793816084183) bank156690 +156691 POINT(41.08503709681983 74.66433980037529) bank156691 +156692 POINT(41.11139200681236 74.79919525582805) bank156692 +156693 POINT(40.232946710902056 73.8277037015104) bank156693 +156694 POINT(40.16328470602673 73.46214120939206) bank156694 +156695 POINT(41.435117533251486 74.12334500820488) bank156695 +156696 POINT(40.588554574889955 74.44873994030155) bank156696 +156697 POINT(41.65540921924022 73.50549195852012) bank156697 +156698 POINT(41.55773015391379 74.83983553092868) bank156698 +156699 POINT(41.62563499938082 74.15259034830923) bank156699 +156700 POINT(40.35030631722233 74.71639974273431) bank156700 +156701 POINT(40.98889318721286 74.89690623228532) bank156701 +156702 POINT(41.06552918108603 73.14880081896813) bank156702 +156703 POINT(41.67534282770931 74.20036427307569) bank156703 +156704 POINT(41.351272866632335 74.46857653403498) bank156704 +156705 POINT(40.56045937766154 74.19885304226554) bank156705 +156706 POINT(40.580001079142185 74.98059203639659) bank156706 +156707 POINT(40.798427525819534 73.22250726548685) bank156707 +156708 POINT(40.83082841746036 74.98245593238599) bank156708 +156709 POINT(40.77105565211353 73.15214033753037) bank156709 +156710 POINT(41.57338581343052 73.24803928218785) bank156710 +156711 POINT(41.6848734969933 74.47072952236717) bank156711 +156712 POINT(39.91003714965759 74.71069519044691) bank156712 +156713 POINT(40.58661924928067 74.44279902404894) bank156713 +156714 POINT(40.6872234054221 74.593269246112) bank156714 +156715 POINT(40.39199185967063 74.09355573704009) bank156715 +156716 POINT(39.742555392165585 74.34152355348209) bank156716 +156717 POINT(40.72450700573022 74.1973449363062) bank156717 +156718 POINT(41.60296247857805 74.63308312026483) bank156718 +156719 POINT(40.473184108315294 73.19196652913959) bank156719 +156720 POINT(40.7121813366171 74.55532241806806) bank156720 +156721 POINT(40.92504786049034 74.52497143446946) bank156721 +156722 POINT(40.52091479752069 74.95765241271921) bank156722 +156723 POINT(40.622889070849816 74.59803986172433) bank156723 +156724 POINT(41.174865538280265 74.75098090190319) bank156724 +156725 POINT(40.34841718854989 73.54799853007583) bank156725 +156726 POINT(40.75995425531414 73.7871250537235) bank156726 +156727 POINT(41.314058812716496 73.04289015873326) bank156727 +156728 POINT(40.6677547058585 74.2128773895734) bank156728 +156729 POINT(39.81010076639285 74.67429051448205) bank156729 +156730 POINT(41.15831922993949 73.73570104733517) bank156730 +156731 POINT(40.42528671389359 74.02891901342213) bank156731 +156732 POINT(41.52378399299285 73.06767338695148) bank156732 +156733 POINT(40.82878153313439 74.47368278383654) bank156733 +156734 POINT(41.23131957600184 74.72311445210403) bank156734 +156735 POINT(39.77115500966323 73.02860046486683) bank156735 +156736 POINT(39.90405443702404 74.23608641483706) bank156736 +156737 POINT(39.93126770421719 74.18048644993942) bank156737 +156738 POINT(41.189915964212744 74.59595749008037) bank156738 +156739 POINT(41.097155568189436 74.64433470955798) bank156739 +156740 POINT(41.077766475690154 73.55314090330012) bank156740 +156741 POINT(39.77045639152336 74.78911951929096) bank156741 +156742 POINT(40.25975985794042 73.280873623606) bank156742 +156743 POINT(39.79271474903456 74.91570515893687) bank156743 +156744 POINT(40.9352633712534 74.58751902282147) bank156744 +156745 POINT(40.17444230102339 73.23980719208244) bank156745 +156746 POINT(40.520950558501916 73.10338558483755) bank156746 +156747 POINT(41.338485026499725 74.95893228369674) bank156747 +156748 POINT(41.51508436376479 74.96360486614014) bank156748 +156749 POINT(41.462845717387395 73.31616787938796) bank156749 +156750 POINT(41.619234122597085 73.91121591088863) bank156750 +156751 POINT(40.6274891023542 73.69015984967095) bank156751 +156752 POINT(40.55982091648636 73.18578391940832) bank156752 +156753 POINT(40.73504757252917 74.32421635373012) bank156753 +156754 POINT(41.632671574229335 73.88499057935705) bank156754 +156755 POINT(40.001268860359495 74.45677928267456) bank156755 +156756 POINT(41.102035526989 74.34154919534141) bank156756 +156757 POINT(41.52238030892685 74.73150149458566) bank156757 +156758 POINT(40.38910624255851 73.02991166085847) bank156758 +156759 POINT(39.83529531802225 74.62418286484056) bank156759 +156760 POINT(41.668232653146134 73.85170762581316) bank156760 +156761 POINT(41.12131587244773 73.54116370234104) bank156761 +156762 POINT(40.48474423249425 73.133933592263) bank156762 +156763 POINT(41.19181271645061 74.32441324177235) bank156763 +156764 POINT(40.41612096584559 74.40867454800174) bank156764 +156765 POINT(40.626760926327854 73.31438264589414) bank156765 +156766 POINT(40.951761574636755 73.46272806190309) bank156766 +156767 POINT(41.15569728202149 73.1425287958269) bank156767 +156768 POINT(41.66498782507757 73.48508533156465) bank156768 +156769 POINT(39.79573404403715 73.13390481589511) bank156769 +156770 POINT(40.37501972881316 74.98693806154446) bank156770 +156771 POINT(40.99529244380622 74.24021564187777) bank156771 +156772 POINT(40.39499603139023 73.36435591129823) bank156772 +156773 POINT(40.20608439179803 74.26669223421212) bank156773 +156774 POINT(41.27145904237916 74.07032192917545) bank156774 +156775 POINT(40.58778536611689 74.53971437283802) bank156775 +156776 POINT(39.788734716852865 73.49415348286615) bank156776 +156777 POINT(40.02425604069926 73.24740320536621) bank156777 +156778 POINT(41.50172587146089 74.79226000039496) bank156778 +156779 POINT(41.28526866676014 74.42956507757462) bank156779 +156780 POINT(41.69183294131715 73.71048639032908) bank156780 +156781 POINT(40.1589958487512 74.74284049069745) bank156781 +156782 POINT(40.43638081308323 73.60416634861575) bank156782 +156783 POINT(40.13761360627662 74.11219319918773) bank156783 +156784 POINT(40.0884799026826 73.58086871476836) bank156784 +156785 POINT(40.46467411971683 73.62946346651198) bank156785 +156786 POINT(41.000218242985355 73.01471245615863) bank156786 +156787 POINT(40.11415529509353 73.27717180962567) bank156787 +156788 POINT(41.22659245028983 73.8827886196372) bank156788 +156789 POINT(40.84366627597892 73.24311817386005) bank156789 +156790 POINT(40.40644618749637 73.2664775460274) bank156790 +156791 POINT(41.53593032750821 74.40410811384751) bank156791 +156792 POINT(39.7914253193591 73.05071526368505) bank156792 +156793 POINT(41.35408445489507 73.59697301209114) bank156793 +156794 POINT(39.73241480134708 74.03924262688385) bank156794 +156795 POINT(40.501543265827365 73.44243854076966) bank156795 +156796 POINT(41.110770678783496 73.08527831782185) bank156796 +156797 POINT(41.27973407499524 74.31960777022542) bank156797 +156798 POINT(40.18209838800776 74.62629583351088) bank156798 +156799 POINT(40.759171098173745 73.7855293175177) bank156799 +156800 POINT(40.15694449464541 74.27053505871707) bank156800 +156801 POINT(40.929099796027245 74.23761166983635) bank156801 +156802 POINT(40.72729448802536 73.66135734212861) bank156802 +156803 POINT(39.86892531860264 74.97857344860061) bank156803 +156804 POINT(40.195770591242415 73.73345836390408) bank156804 +156805 POINT(41.43784499522345 73.03698541423026) bank156805 +156806 POINT(41.207667827861826 74.93521374582629) bank156806 +156807 POINT(41.55013484744489 74.86854174535169) bank156807 +156808 POINT(41.52210553930571 74.83038682843912) bank156808 +156809 POINT(41.150688597393554 73.82072871975193) bank156809 +156810 POINT(41.681645274496 74.2887148998293) bank156810 +156811 POINT(40.55897933272943 73.40795043357726) bank156811 +156812 POINT(40.871046392907644 74.79318011732714) bank156812 +156813 POINT(40.212686277134544 74.29380389280595) bank156813 +156814 POINT(41.32554707430434 74.75641807617143) bank156814 +156815 POINT(40.04837777559862 73.66674280036639) bank156815 +156816 POINT(40.23958209513615 73.42527604247138) bank156816 +156817 POINT(40.89260664112538 74.60835568732246) bank156817 +156818 POINT(40.65648730867373 73.27180244443088) bank156818 +156819 POINT(41.279298005822625 73.1055954733274) bank156819 +156820 POINT(40.63594961190667 74.28770173654264) bank156820 +156821 POINT(40.25945271606623 74.21396009680136) bank156821 +156822 POINT(41.605381300895885 74.73965672016244) bank156822 +156823 POINT(39.755356019884424 73.55841006165633) bank156823 +156824 POINT(40.45345557704902 74.00533132716237) bank156824 +156825 POINT(39.96820215477674 74.59379395069418) bank156825 +156826 POINT(40.54118157027478 73.86865244064577) bank156826 +156827 POINT(40.843016691952705 73.96356894273407) bank156827 +156828 POINT(41.31565130220409 73.12094204947138) bank156828 +156829 POINT(41.61863730370746 73.60602628730415) bank156829 +156830 POINT(40.45440843721778 74.58936811562596) bank156830 +156831 POINT(40.300789992201885 74.22257679051789) bank156831 +156832 POINT(39.72786430562035 73.93260473820256) bank156832 +156833 POINT(39.79211003343491 74.94156816748895) bank156833 +156834 POINT(40.62705254492341 74.53095008935647) bank156834 +156835 POINT(41.247429783590135 74.95838317690966) bank156835 +156836 POINT(39.766118359979245 74.46989095933672) bank156836 +156837 POINT(41.4378860092777 74.44540179520193) bank156837 +156838 POINT(41.49442490052318 73.38905157153062) bank156838 +156839 POINT(40.084451424357155 73.14195677033638) bank156839 +156840 POINT(41.513524087725166 73.75059684316668) bank156840 +156841 POINT(41.35310772503711 74.94691165633745) bank156841 +156842 POINT(40.805903042961496 74.06911086072853) bank156842 +156843 POINT(41.43780400099779 73.81323824721913) bank156843 +156844 POINT(40.3014655149273 73.69318098269618) bank156844 +156845 POINT(41.0995043930232 74.9794543134777) bank156845 +156846 POINT(40.95709910260998 74.92289569521822) bank156846 +156847 POINT(40.940214194381305 74.31564079302203) bank156847 +156848 POINT(41.23650587010981 73.64749069291499) bank156848 +156849 POINT(41.25092818250167 73.41658409350802) bank156849 +156850 POINT(41.23471355435366 74.81394154708526) bank156850 +156851 POINT(41.07872459444204 73.88135310666027) bank156851 +156852 POINT(40.73203795517782 74.56038636272277) bank156852 +156853 POINT(40.463022270475655 74.4519478576487) bank156853 +156854 POINT(40.598455233060434 74.24284074659406) bank156854 +156855 POINT(39.948389136286366 73.51472611070476) bank156855 +156856 POINT(39.79865406071221 73.83418803751152) bank156856 +156857 POINT(41.557875248664146 73.93489673672269) bank156857 +156858 POINT(40.47449249029731 74.89691639430387) bank156858 +156859 POINT(40.37940414163236 74.32396203483857) bank156859 +156860 POINT(41.23985194646823 73.65492007544842) bank156860 +156861 POINT(40.89057846124361 73.1399109829308) bank156861 +156862 POINT(39.90181710020521 74.80458399318101) bank156862 +156863 POINT(41.289801479279916 74.08906200227241) bank156863 +156864 POINT(41.531502610979 73.3476532223969) bank156864 +156865 POINT(40.02009293781359 73.66719334662601) bank156865 +156866 POINT(39.95852475025476 73.74923880916397) bank156866 +156867 POINT(39.83760717638855 74.80676688966625) bank156867 +156868 POINT(39.81632220623345 74.89359990410522) bank156868 +156869 POINT(41.41109075233269 73.45249956759069) bank156869 +156870 POINT(41.59041765663239 73.01792841456891) bank156870 +156871 POINT(41.38788001014762 73.42626571824128) bank156871 +156872 POINT(40.89575650736434 74.74656000991394) bank156872 +156873 POINT(40.55241186625297 74.67869234477533) bank156873 +156874 POINT(40.19917923387197 74.34241408382309) bank156874 +156875 POINT(39.81272709184791 74.48261932336146) bank156875 +156876 POINT(40.4722522871602 74.26942088283438) bank156876 +156877 POINT(41.33800255170322 74.93432236010439) bank156877 +156878 POINT(41.33626037638982 74.30474309669513) bank156878 +156879 POINT(40.556519692030726 74.76732278153345) bank156879 +156880 POINT(41.473115084364025 73.04318998511944) bank156880 +156881 POINT(40.76334637872925 74.49272702492287) bank156881 +156882 POINT(40.0861312066028 74.70339485880929) bank156882 +156883 POINT(41.58548304835241 73.26235946101556) bank156883 +156884 POINT(40.03590577530774 74.75763220479722) bank156884 +156885 POINT(41.6170196855035 74.19785971058937) bank156885 +156886 POINT(40.365107566810515 73.84972645542474) bank156886 +156887 POINT(41.41206270580901 73.36958837896528) bank156887 +156888 POINT(40.2165743869174 73.82180363932072) bank156888 +156889 POINT(39.81264790302115 73.65538941081894) bank156889 +156890 POINT(39.81797901886973 74.73812825885729) bank156890 +156891 POINT(41.18685319240832 73.64093243891138) bank156891 +156892 POINT(40.20120656573436 74.7038070893685) bank156892 +156893 POINT(41.47477897009099 74.74470039045852) bank156893 +156894 POINT(40.19955933076615 74.89361324813227) bank156894 +156895 POINT(40.52026926449004 73.78709702264298) bank156895 +156896 POINT(39.72286458100397 74.83545571511523) bank156896 +156897 POINT(41.45843660113066 74.13420756319296) bank156897 +156898 POINT(40.35889141140499 73.60561122429219) bank156898 +156899 POINT(40.523602373643975 74.73970686021244) bank156899 +156900 POINT(41.15408652838815 74.97536144654798) bank156900 +156901 POINT(40.36743616500658 73.02861195757743) bank156901 +156902 POINT(41.43582395742592 74.63156927750597) bank156902 +156903 POINT(40.96726014141738 74.34014824089473) bank156903 +156904 POINT(39.96506668141008 73.63400276913889) bank156904 +156905 POINT(40.191602961804186 73.40312800810273) bank156905 +156906 POINT(40.809298266813705 74.29351785822558) bank156906 +156907 POINT(40.346089122233565 73.20107041465828) bank156907 +156908 POINT(41.54433005566318 73.78432197103518) bank156908 +156909 POINT(40.45960681863615 73.90857935477898) bank156909 +156910 POINT(39.92090022181627 73.99344078854162) bank156910 +156911 POINT(40.65968785234629 74.76298042642219) bank156911 +156912 POINT(41.698312246899675 73.51440794818251) bank156912 +156913 POINT(40.563084978616466 73.6342186471162) bank156913 +156914 POINT(40.36803569846513 73.31748313883118) bank156914 +156915 POINT(39.92400526141934 73.64563915922952) bank156915 +156916 POINT(41.31094465964018 74.85927842986759) bank156916 +156917 POINT(39.90028406611562 74.3149053842085) bank156917 +156918 POINT(40.59983076370554 73.34313910195782) bank156918 +156919 POINT(40.83043390526418 74.60355144907503) bank156919 +156920 POINT(41.16502931038272 74.43748387091867) bank156920 +156921 POINT(41.010729816523266 73.47982602991006) bank156921 +156922 POINT(40.73142900148878 74.04985440912961) bank156922 +156923 POINT(40.40876954728182 73.16513112476454) bank156923 +156924 POINT(41.108165716360446 74.58922780954893) bank156924 +156925 POINT(40.598268967810995 74.25694009851622) bank156925 +156926 POINT(39.931869419429184 74.68289623748876) bank156926 +156927 POINT(40.56712662876695 73.85417995842333) bank156927 +156928 POINT(41.63222616616575 74.7575656989101) bank156928 +156929 POINT(40.50115933035692 74.95638162905247) bank156929 +156930 POINT(40.879181485753726 74.85711239154926) bank156930 +156931 POINT(40.84180545747188 73.33705852258004) bank156931 +156932 POINT(41.26899139830688 74.00169805122114) bank156932 +156933 POINT(39.90799304491921 74.09888385693496) bank156933 +156934 POINT(39.77906074786981 74.1982716823368) bank156934 +156935 POINT(41.51436136238852 74.49841263099532) bank156935 +156936 POINT(41.55094470701002 73.99020945674243) bank156936 +156937 POINT(40.942238794237845 73.99908382518089) bank156937 +156938 POINT(40.47675887975384 74.2993811231166) bank156938 +156939 POINT(41.29565352695315 73.28777479709764) bank156939 +156940 POINT(40.36644575504325 73.83084960074824) bank156940 +156941 POINT(41.05855085660047 74.86611957383745) bank156941 +156942 POINT(40.36360549796653 73.30879532428808) bank156942 +156943 POINT(40.003517541934066 73.47161428309403) bank156943 +156944 POINT(41.15972430757174 74.1112149925667) bank156944 +156945 POINT(40.25870457024716 74.30493124739691) bank156945 +156946 POINT(41.30318760070152 74.61792518557981) bank156946 +156947 POINT(41.19564760295401 74.96333069504533) bank156947 +156948 POINT(41.183690413032686 74.71672988294743) bank156948 +156949 POINT(41.71237701906057 74.22060699867451) bank156949 +156950 POINT(40.55636093212844 73.04160749469202) bank156950 +156951 POINT(40.56578715233197 74.31946341562342) bank156951 +156952 POINT(39.9100579887381 74.14314666202868) bank156952 +156953 POINT(39.86866794597274 74.64988616801065) bank156953 +156954 POINT(40.42863625440705 73.09273925905599) bank156954 +156955 POINT(41.69758191270814 74.05316563805816) bank156955 +156956 POINT(40.952413362609846 73.5633543021983) bank156956 +156957 POINT(40.31650250805621 73.3005660382467) bank156957 +156958 POINT(41.18671839210199 73.55931960341933) bank156958 +156959 POINT(41.623016112891925 73.50777443027437) bank156959 +156960 POINT(41.220841033305916 74.31120469131781) bank156960 +156961 POINT(39.92775264049506 74.48668125801962) bank156961 +156962 POINT(40.49674238645861 74.88460681877321) bank156962 +156963 POINT(40.182234585469395 74.23423446468408) bank156963 +156964 POINT(41.07958006252003 73.36875044889057) bank156964 +156965 POINT(40.558823619372596 74.59740753253246) bank156965 +156966 POINT(40.92372637977792 74.12638335527616) bank156966 +156967 POINT(41.009318333084764 74.66863958515658) bank156967 +156968 POINT(40.1764404650265 74.92648607600346) bank156968 +156969 POINT(41.15769234350031 74.51264548806273) bank156969 +156970 POINT(40.59202337980564 73.5732888119027) bank156970 +156971 POINT(40.784598528198174 73.20032308872368) bank156971 +156972 POINT(41.155799489673285 74.52531235593129) bank156972 +156973 POINT(40.58411253626926 74.3343920599947) bank156973 +156974 POINT(41.474899405700654 73.32890564965663) bank156974 +156975 POINT(39.86784456902605 74.48483467647512) bank156975 +156976 POINT(40.84543908699631 74.96280477824708) bank156976 +156977 POINT(40.52639599243922 73.89350886959849) bank156977 +156978 POINT(40.94899984720291 74.80091673183753) bank156978 +156979 POINT(40.23475048603715 74.21128115824766) bank156979 +156980 POINT(39.79518140652976 74.28926960724236) bank156980 +156981 POINT(40.250777188166204 74.8817955690339) bank156981 +156982 POINT(39.98737299008523 73.64297162113586) bank156982 +156983 POINT(40.930045359872 73.49483029746216) bank156983 +156984 POINT(41.09219143579817 73.13447473576852) bank156984 +156985 POINT(40.549406628187015 73.6794222711031) bank156985 +156986 POINT(41.05593412946226 73.52856213355076) bank156986 +156987 POINT(39.82808515584708 74.97557339830274) bank156987 +156988 POINT(40.21743468181959 74.96405100507805) bank156988 +156989 POINT(39.759084981159006 74.72389602163737) bank156989 +156990 POINT(41.367282943287066 74.44240329588708) bank156990 +156991 POINT(39.98535503622333 73.4755630308789) bank156991 +156992 POINT(41.09106530468982 73.30457152038362) bank156992 +156993 POINT(40.118020408932075 73.06676349000072) bank156993 +156994 POINT(40.80547834332191 73.50023915377108) bank156994 +156995 POINT(40.910720131538284 73.66404098072415) bank156995 +156996 POINT(41.08807829206154 73.0533382953395) bank156996 +156997 POINT(40.53926606095914 74.32663381956023) bank156997 +156998 POINT(41.40448711214132 74.05326938280979) bank156998 +156999 POINT(40.315790006775885 73.48267508848677) bank156999 +157000 POINT(40.8516530095451 74.66947320791316) bank157000 +157001 POINT(40.12831560168157 74.7258875576106) bank157001 +157002 POINT(41.321178200657926 74.2700602376961) bank157002 +157003 POINT(41.36082593541357 73.91000274018225) bank157003 +157004 POINT(40.50528062799264 73.2095409739874) bank157004 +157005 POINT(40.172107238033924 73.3127568081851) bank157005 +157006 POINT(40.1806554929937 73.67014706648531) bank157006 +157007 POINT(39.83507245804363 74.43003124335897) bank157007 +157008 POINT(40.19694036397132 74.78565161326844) bank157008 +157009 POINT(41.38993261251951 74.30379888053511) bank157009 +157010 POINT(40.452726276485464 73.09400037325068) bank157010 +157011 POINT(40.78463178840843 74.36253172919774) bank157011 +157012 POINT(40.23502460149285 73.12842919964918) bank157012 +157013 POINT(40.72949602383681 74.87837867914341) bank157013 +157014 POINT(40.730844496898776 74.93458474132605) bank157014 +157015 POINT(41.32372071820659 74.66100246422647) bank157015 +157016 POINT(41.622462289918275 73.55008490292097) bank157016 +157017 POINT(40.35217972761121 74.46904263415769) bank157017 +157018 POINT(40.006643881062985 73.26344846857404) bank157018 +157019 POINT(41.060635200785995 73.01805517813958) bank157019 +157020 POINT(41.21549602448332 74.34254990381004) bank157020 +157021 POINT(39.782126381897214 73.69064909144792) bank157021 +157022 POINT(41.076797371915085 74.87654550704826) bank157022 +157023 POINT(40.94650520968213 74.1328144874485) bank157023 +157024 POINT(40.24580467074075 74.92474537006306) bank157024 +157025 POINT(40.29194094976369 73.17305551753238) bank157025 +157026 POINT(40.70512199388608 74.92926048049557) bank157026 +157027 POINT(39.800252731205774 74.97718370866119) bank157027 +157028 POINT(41.23942997169451 73.42271405763626) bank157028 +157029 POINT(41.548943377481024 74.25873699612383) bank157029 +157030 POINT(40.82852855959219 74.50993716136465) bank157030 +157031 POINT(41.526803097374255 73.01964463064371) bank157031 +157032 POINT(40.61011181281207 74.39536168136489) bank157032 +157033 POINT(40.0729304522821 74.07692784715847) bank157033 +157034 POINT(41.055090759195096 73.91284643590107) bank157034 +157035 POINT(40.62775419579011 73.56033609298325) bank157035 +157036 POINT(39.98710216791243 74.55413472492481) bank157036 +157037 POINT(40.61411904059857 73.7082151416616) bank157037 +157038 POINT(40.82009237674834 74.9305593901159) bank157038 +157039 POINT(40.154612833258334 73.16722626694502) bank157039 +157040 POINT(40.81510986951455 73.8280328547278) bank157040 +157041 POINT(41.51678396789566 74.59698673506499) bank157041 +157042 POINT(40.703289182279555 74.38787193043214) bank157042 +157043 POINT(40.67707615463264 73.04762607762443) bank157043 +157044 POINT(41.50005376102985 74.898822388836) bank157044 +157045 POINT(40.301291579110384 74.4217859825648) bank157045 +157046 POINT(39.91170194643848 73.54729840402733) bank157046 +157047 POINT(41.09135476942428 74.58007579316713) bank157047 +157048 POINT(40.39673463589623 73.50308019029535) bank157048 +157049 POINT(41.164355103334785 74.01034951706083) bank157049 +157050 POINT(40.49210238636164 74.56433821751652) bank157050 +157051 POINT(40.62429033151047 74.1107394733996) bank157051 +157052 POINT(40.59494476625114 74.93203869563081) bank157052 +157053 POINT(40.27158321541914 73.17899067072015) bank157053 +157054 POINT(41.40739875594527 73.44056469114149) bank157054 +157055 POINT(41.099265161774134 74.26189225446954) bank157055 +157056 POINT(40.171775513233946 73.21687629287268) bank157056 +157057 POINT(41.44608459761952 74.15109142711543) bank157057 +157058 POINT(40.46304756249417 74.6080598736249) bank157058 +157059 POINT(41.48909280060551 74.43543952916706) bank157059 +157060 POINT(39.93194053615502 73.30581361816178) bank157060 +157061 POINT(40.9745614997583 73.23516028985631) bank157061 +157062 POINT(41.20700606002223 74.01646302747763) bank157062 +157063 POINT(39.74237751835824 73.91004710553209) bank157063 +157064 POINT(41.302444944184586 74.97208015986426) bank157064 +157065 POINT(40.32555422912389 74.4796817005222) bank157065 +157066 POINT(40.240992845318566 74.28671456114536) bank157066 +157067 POINT(40.43668189773646 74.82031277686453) bank157067 +157068 POINT(39.75654806390253 74.43410894777094) bank157068 +157069 POINT(39.72969908160141 73.52343488900162) bank157069 +157070 POINT(39.77498831077886 73.28321087245227) bank157070 +157071 POINT(40.76414733675003 73.38814173373096) bank157071 +157072 POINT(40.986582599777066 74.9770144698433) bank157072 +157073 POINT(39.90039570152063 74.37321518652269) bank157073 +157074 POINT(40.18659538587284 73.82422509772577) bank157074 +157075 POINT(40.987230552708304 74.0279734036279) bank157075 +157076 POINT(41.49062776429688 74.05752791519174) bank157076 +157077 POINT(40.11289298777355 74.67143483062242) bank157077 +157078 POINT(40.62103446044825 74.56386425996372) bank157078 +157079 POINT(40.18739928525257 74.06315066139881) bank157079 +157080 POINT(40.470990863637 73.20270906116039) bank157080 +157081 POINT(41.585880447658546 74.42217654017689) bank157081 +157082 POINT(40.65600252349409 74.37356278810408) bank157082 +157083 POINT(39.77836075739515 74.44405808294077) bank157083 +157084 POINT(40.779607339625684 74.2482495628234) bank157084 +157085 POINT(41.05664744111754 74.69510817939275) bank157085 +157086 POINT(41.273139677624705 73.35912127835768) bank157086 +157087 POINT(40.65136176523561 73.28644364446811) bank157087 +157088 POINT(39.751858391287655 74.72046473042933) bank157088 +157089 POINT(40.060527603784166 73.89484872798391) bank157089 +157090 POINT(41.344462936517104 73.90406590783917) bank157090 +157091 POINT(39.899450956739095 73.1089291600718) bank157091 +157092 POINT(41.34262818005049 74.21632225276348) bank157092 +157093 POINT(40.1206565432418 73.47477066347514) bank157093 +157094 POINT(41.338863789333104 74.62386856613139) bank157094 +157095 POINT(40.35230723049063 74.78385523262156) bank157095 +157096 POINT(41.380913657761795 74.92012982470438) bank157096 +157097 POINT(40.29665020830535 73.9887605115161) bank157097 +157098 POINT(41.517197568660656 74.84210983424805) bank157098 +157099 POINT(39.80436047629333 74.03418814158404) bank157099 +157100 POINT(40.284178840701536 73.34394079135363) bank157100 +157101 POINT(41.535771296401506 74.34988611713335) bank157101 +157102 POINT(40.991582429678566 74.90312080735636) bank157102 +157103 POINT(39.86848984750958 73.10836772553516) bank157103 +157104 POINT(41.49402300390992 74.87955081768959) bank157104 +157105 POINT(40.21876378012102 74.36788539374052) bank157105 +157106 POINT(40.326373650010076 73.41664451511603) bank157106 +157107 POINT(41.575451874553366 73.77153342512631) bank157107 +157108 POINT(40.610194690083155 74.26032499419141) bank157108 +157109 POINT(40.559104637438864 74.72970556009702) bank157109 +157110 POINT(41.48661803979805 74.61080865695288) bank157110 +157111 POINT(40.789764373939654 73.45795430338501) bank157111 +157112 POINT(41.69235043856289 73.31357529054033) bank157112 +157113 POINT(40.011337546093436 74.8987031090702) bank157113 +157114 POINT(41.1550256567236 73.07408462480723) bank157114 +157115 POINT(41.397548336492996 74.64527358795524) bank157115 +157116 POINT(40.383945195104324 73.35984282647617) bank157116 +157117 POINT(41.70231693488532 73.10624978265402) bank157117 +157118 POINT(40.52377062014644 73.22336806683417) bank157118 +157119 POINT(40.04404524489782 74.04224316000555) bank157119 +157120 POINT(40.89248278853083 73.11032378796794) bank157120 +157121 POINT(41.68275305685609 73.55771507494549) bank157121 +157122 POINT(40.772271514802355 74.15144569753171) bank157122 +157123 POINT(39.99919633467496 74.37501209214066) bank157123 +157124 POINT(40.55537511578968 73.26437446938135) bank157124 +157125 POINT(40.430154829033526 73.16985999215129) bank157125 +157126 POINT(40.060987344894926 74.80254705552893) bank157126 +157127 POINT(39.979247972880664 73.4491706441464) bank157127 +157128 POINT(39.848896826869804 74.36409639391279) bank157128 +157129 POINT(41.213704707953276 73.92363587565758) bank157129 +157130 POINT(40.151539831028835 73.0941408821063) bank157130 +157131 POINT(40.423985821768774 73.91358024827348) bank157131 +157132 POINT(40.94180735263298 73.43248431957292) bank157132 +157133 POINT(40.922651009130114 73.14311264451814) bank157133 +157134 POINT(40.484708574459646 73.58897387477374) bank157134 +157135 POINT(40.40390924105029 73.53283152784276) bank157135 +157136 POINT(41.40195647406737 73.877762646134) bank157136 +157137 POINT(40.059646939871 73.65881647533283) bank157137 +157138 POINT(39.85181839946833 73.9875287735986) bank157138 +157139 POINT(41.25476542617647 73.82734106793194) bank157139 +157140 POINT(40.524796676645856 73.75846224376866) bank157140 +157141 POINT(40.432830592918975 74.91925524245724) bank157141 +157142 POINT(40.055924822381215 73.72408747076948) bank157142 +157143 POINT(40.84222031791484 74.8907752902904) bank157143 +157144 POINT(41.47139822469947 73.71868781491722) bank157144 +157145 POINT(40.45547966292645 74.54348975478344) bank157145 +157146 POINT(41.308583524754724 74.1447891891378) bank157146 +157147 POINT(40.79591579712931 73.79999388471145) bank157147 +157148 POINT(40.20581826982451 74.01847047326105) bank157148 +157149 POINT(41.67439549665244 74.47626759561165) bank157149 +157150 POINT(41.07902358375361 73.11322915655234) bank157150 +157151 POINT(40.6278131563413 73.80644994595458) bank157151 +157152 POINT(41.348459179354656 73.58826462543331) bank157152 +157153 POINT(40.153509707003636 74.94540126742797) bank157153 +157154 POINT(41.31513762620422 73.32412162106502) bank157154 +157155 POINT(39.75037928120879 74.86741886040444) bank157155 +157156 POINT(41.38285535152458 74.099632354733) bank157156 +157157 POINT(40.87721735486442 73.23714160551317) bank157157 +157158 POINT(41.061476612845695 73.95486801713052) bank157158 +157159 POINT(40.67838451514945 74.2584125308909) bank157159 +157160 POINT(40.69126379079077 73.0757359450777) bank157160 +157161 POINT(40.79414647716973 73.85243969954026) bank157161 +157162 POINT(40.84305901591259 73.23627137699793) bank157162 +157163 POINT(41.53331532429075 73.56206504045005) bank157163 +157164 POINT(40.36590183061988 73.01927198797104) bank157164 +157165 POINT(40.61555757527907 73.13445665582216) bank157165 +157166 POINT(39.820844515954896 74.25004896576387) bank157166 +157167 POINT(40.02783001141766 73.06853444873909) bank157167 +157168 POINT(41.04054129148882 74.1790094725234) bank157168 +157169 POINT(41.327249035216894 74.36764316921376) bank157169 +157170 POINT(41.505742402470965 73.51488012553422) bank157170 +157171 POINT(40.11164539333288 74.35233384350592) bank157171 +157172 POINT(40.24940931407886 75.00059443375208) bank157172 +157173 POINT(41.09453540745027 74.42860826131698) bank157173 +157174 POINT(41.45431483133918 73.47560025554272) bank157174 +157175 POINT(41.42848577111592 74.60025766426574) bank157175 +157176 POINT(40.694876270380306 74.79265533719266) bank157176 +157177 POINT(40.82222229641283 73.81797913192018) bank157177 +157178 POINT(41.17129225954238 73.99767724222231) bank157178 +157179 POINT(41.00417893152502 73.40718657605288) bank157179 +157180 POINT(40.244612611420074 73.56316290280654) bank157180 +157181 POINT(41.374424904325984 74.45911124293336) bank157181 +157182 POINT(41.25397249651118 73.28935347532502) bank157182 +157183 POINT(39.9071242291542 74.43372422055293) bank157183 +157184 POINT(41.48908004700839 73.90084973441982) bank157184 +157185 POINT(40.12042959949701 74.79265711471095) bank157185 +157186 POINT(40.56415552735282 74.2871841599803) bank157186 +157187 POINT(39.81798357392012 73.170466843275) bank157187 +157188 POINT(41.630526756282286 73.89710109872686) bank157188 +157189 POINT(39.77140779399098 74.8535322856857) bank157189 +157190 POINT(41.4129769859574 74.72846702794244) bank157190 +157191 POINT(40.16332526037681 74.19054488265479) bank157191 +157192 POINT(41.70272139310838 73.18025932679825) bank157192 +157193 POINT(40.77088922494419 73.46824965263886) bank157193 +157194 POINT(40.196270649601004 74.04240032851605) bank157194 +157195 POINT(41.571244889471444 73.91674821111854) bank157195 +157196 POINT(40.8081693328436 74.34550635310647) bank157196 +157197 POINT(40.440903664337874 73.64238070948353) bank157197 +157198 POINT(39.77873846856586 74.20471928784731) bank157198 +157199 POINT(41.254395381144356 74.83927576468824) bank157199 +157200 POINT(41.14048484581103 73.6444955290805) bank157200 +157201 POINT(40.35970498337526 73.6686113580282) bank157201 +157202 POINT(40.19195545790749 73.4369438520491) bank157202 +157203 POINT(40.010812734452784 74.80916240122168) bank157203 +157204 POINT(39.91732715745823 73.23161828519305) bank157204 +157205 POINT(41.6533397511992 74.70986488277455) bank157205 +157206 POINT(39.88711597083327 73.66931440789538) bank157206 +157207 POINT(39.8694642517725 73.66265646194836) bank157207 +157208 POINT(39.970566823439796 74.79005468405944) bank157208 +157209 POINT(40.48422136450279 74.73240709754147) bank157209 +157210 POINT(40.025957209505066 74.69653268945608) bank157210 +157211 POINT(41.05982903999941 73.79817133011393) bank157211 +157212 POINT(41.39475023879842 73.63816447786051) bank157212 +157213 POINT(39.87857602441851 73.40301006504119) bank157213 +157214 POINT(41.48477992097909 73.36347164656277) bank157214 +157215 POINT(40.92441344422595 74.53229756166122) bank157215 +157216 POINT(40.95021490601802 73.30242342171138) bank157216 +157217 POINT(41.68281832357925 73.79460408997757) bank157217 +157218 POINT(39.90345774127472 73.50839772479063) bank157218 +157219 POINT(39.984746605923064 73.98935091208176) bank157219 +157220 POINT(41.56544101017814 74.92778732153222) bank157220 +157221 POINT(40.055162494108224 73.53009648448514) bank157221 +157222 POINT(40.89914592997721 74.83816492995365) bank157222 +157223 POINT(41.57019384715468 74.03713287202204) bank157223 +157224 POINT(40.90063488753739 74.44317576619314) bank157224 +157225 POINT(39.993414915922024 74.50774013355091) bank157225 +157226 POINT(40.167576254917506 73.78426172164355) bank157226 +157227 POINT(40.358577395493654 74.47650750299857) bank157227 +157228 POINT(40.46762648514145 73.3458201670818) bank157228 +157229 POINT(40.33800027606452 74.8216643897764) bank157229 +157230 POINT(40.11738418740347 73.27823934700248) bank157230 +157231 POINT(40.93696529085901 74.21247150928752) bank157231 +157232 POINT(40.85502389424194 73.34360934071685) bank157232 +157233 POINT(40.90429826167165 74.28527734561868) bank157233 +157234 POINT(39.921553514456875 73.04788277415622) bank157234 +157235 POINT(40.54710560881076 74.17529655895555) bank157235 +157236 POINT(41.31184881189854 73.35407407714172) bank157236 +157237 POINT(41.36974300473946 74.32717271054143) bank157237 +157238 POINT(39.77032588501536 73.24166864316702) bank157238 +157239 POINT(40.695925946337056 73.80827721534747) bank157239 +157240 POINT(40.1795876794927 73.32997007965764) bank157240 +157241 POINT(41.2023059590763 74.86122606492995) bank157241 +157242 POINT(41.2340110412448 74.39172340726104) bank157242 +157243 POINT(40.706684887978376 73.67582783203015) bank157243 +157244 POINT(41.55356840155609 74.00764397761466) bank157244 +157245 POINT(40.031059537827296 74.54409715784365) bank157245 +157246 POINT(41.20616029062213 73.27240105260033) bank157246 +157247 POINT(40.08972239253446 74.35601653204637) bank157247 +157248 POINT(41.46205673905884 74.46294475827608) bank157248 +157249 POINT(39.80363453104099 73.46906500208168) bank157249 +157250 POINT(40.26157699363488 74.60862114264113) bank157250 +157251 POINT(39.858268642689666 74.0700355368219) bank157251 +157252 POINT(39.983721919576716 73.56168052326701) bank157252 +157253 POINT(39.826583788032366 74.52979750606254) bank157253 +157254 POINT(40.284008025405114 73.89964474078994) bank157254 +157255 POINT(40.48486238097092 73.77101403121502) bank157255 +157256 POINT(40.66345096578988 74.23578431572135) bank157256 +157257 POINT(40.27203857471905 73.28218824000942) bank157257 +157258 POINT(40.15997378050387 74.80337634079032) bank157258 +157259 POINT(40.329412556206684 73.13623672819112) bank157259 +157260 POINT(41.632114494700886 74.1309875172914) bank157260 +157261 POINT(40.40707084705006 74.76725886588285) bank157261 +157262 POINT(39.92378279173135 74.61094773238408) bank157262 +157263 POINT(40.48515976086162 74.98171618905809) bank157263 +157264 POINT(40.16769473888607 74.39934325610828) bank157264 +157265 POINT(39.882243656520636 73.90339341063407) bank157265 +157266 POINT(41.701386620121326 74.1703686614377) bank157266 +157267 POINT(40.589993205491055 74.01613516523376) bank157267 +157268 POINT(41.411942612947506 74.61400181152656) bank157268 +157269 POINT(41.37816334660352 74.05760027588957) bank157269 +157270 POINT(40.85874185418814 74.69982023450903) bank157270 +157271 POINT(41.18815697609334 74.5785783922606) bank157271 +157272 POINT(41.10592802270576 73.75470749495574) bank157272 +157273 POINT(41.60886354311012 73.09519078367695) bank157273 +157274 POINT(41.494299760482626 73.70986147251894) bank157274 +157275 POINT(41.151037963101594 73.39752809086532) bank157275 +157276 POINT(41.01216533740727 74.90837042829126) bank157276 +157277 POINT(40.57201948118755 73.75953115161447) bank157277 +157278 POINT(39.88157950119018 74.58531373987596) bank157278 +157279 POINT(41.316526012902735 73.1317101798742) bank157279 +157280 POINT(40.50838695830359 74.15518836055266) bank157280 +157281 POINT(40.144052520348566 73.51823099699799) bank157281 +157282 POINT(41.11521258306615 74.28677320634378) bank157282 +157283 POINT(41.01512770073099 74.93283311366203) bank157283 +157284 POINT(40.215167497597065 74.376299367131) bank157284 +157285 POINT(40.20929698442557 74.27440880950795) bank157285 +157286 POINT(41.19413558801575 73.18366536131307) bank157286 +157287 POINT(39.9529786614405 73.34789159763518) bank157287 +157288 POINT(41.12838247500124 74.89921787226508) bank157288 +157289 POINT(40.63832708526395 73.43479600859129) bank157289 +157290 POINT(40.74057074860235 74.41799806516956) bank157290 +157291 POINT(41.00872591876656 73.91494919892365) bank157291 +157292 POINT(41.567870973076275 74.45483459312264) bank157292 +157293 POINT(39.83173520552522 74.87001726234452) bank157293 +157294 POINT(39.86479358704017 73.68665168125632) bank157294 +157295 POINT(40.046970372882775 73.10205038343916) bank157295 +157296 POINT(40.72647911792669 73.32943193713) bank157296 +157297 POINT(41.442333390241984 73.42316567319155) bank157297 +157298 POINT(40.20682513119253 74.21395989999047) bank157298 +157299 POINT(41.14354399701068 73.86414805560632) bank157299 +157300 POINT(40.958751863436504 74.23992227148679) bank157300 +157301 POINT(40.82212936677875 73.23487063095696) bank157301 +157302 POINT(41.681686141300254 74.64353647501422) bank157302 +157303 POINT(39.95610660214613 74.06066690297857) bank157303 +157304 POINT(40.62174583093417 73.32724586415308) bank157304 +157305 POINT(40.657520735272676 74.25365160213975) bank157305 +157306 POINT(40.25945010730501 73.36881050060904) bank157306 +157307 POINT(41.55215401529598 74.67398917504632) bank157307 +157308 POINT(40.65698837879967 74.5295559082169) bank157308 +157309 POINT(40.721929155500966 74.4230254965451) bank157309 +157310 POINT(41.65895949221057 74.70700723012455) bank157310 +157311 POINT(41.2549658655286 73.41632684418317) bank157311 +157312 POINT(40.66051964530188 74.67205432051674) bank157312 +157313 POINT(40.91929916615748 74.89034273699077) bank157313 +157314 POINT(40.31117345047821 73.77404325784552) bank157314 +157315 POINT(41.453601888170674 74.377485582358) bank157315 +157316 POINT(41.655738231423015 74.40238592756539) bank157316 +157317 POINT(41.273199046486525 74.64001671925145) bank157317 +157318 POINT(40.69059490983054 74.78996370333155) bank157318 +157319 POINT(41.122568237658996 74.8885915474316) bank157319 +157320 POINT(41.09435906114177 74.65160552597558) bank157320 +157321 POINT(41.16499104988828 73.94760272050443) bank157321 +157322 POINT(40.75829670541121 74.80270662119135) bank157322 +157323 POINT(41.35131935831976 73.96551196858117) bank157323 +157324 POINT(40.822305518005315 73.21149063641016) bank157324 +157325 POINT(39.948290005852314 74.07684324608113) bank157325 +157326 POINT(41.3271530984702 73.93078356449122) bank157326 +157327 POINT(39.77481750205831 73.40092627001633) bank157327 +157328 POINT(41.49787756632639 74.40379501665596) bank157328 +157329 POINT(39.89153020987794 73.86684057079643) bank157329 +157330 POINT(41.03882261621337 74.11334508265416) bank157330 +157331 POINT(41.07927389740937 73.25128443459731) bank157331 +157332 POINT(41.29704952627686 73.36020266173105) bank157332 +157333 POINT(40.742500501114996 73.69736575644836) bank157333 +157334 POINT(41.29239404401886 73.8405908656735) bank157334 +157335 POINT(40.08315622094227 74.75755373454213) bank157335 +157336 POINT(39.907968983052314 73.82592114534391) bank157336 +157337 POINT(40.58931889687953 73.75705448714182) bank157337 +157338 POINT(41.36172437855331 73.61531569605168) bank157338 +157339 POINT(40.762084807045575 74.21067683278422) bank157339 +157340 POINT(41.63986660463298 73.24764339039305) bank157340 +157341 POINT(39.80768852679921 73.78486085150055) bank157341 +157342 POINT(40.427382816805604 73.50964049091563) bank157342 +157343 POINT(41.55291719765846 73.06958798853914) bank157343 +157344 POINT(41.36727655664355 74.57275144787984) bank157344 +157345 POINT(40.73036927146123 73.46651969078894) bank157345 +157346 POINT(40.71943066447545 74.39964945124656) bank157346 +157347 POINT(40.82247273161428 74.40701674155892) bank157347 +157348 POINT(41.438760000697215 74.6392386119785) bank157348 +157349 POINT(41.68737124672552 73.5084605806194) bank157349 +157350 POINT(40.136720466359016 74.9570316309614) bank157350 +157351 POINT(41.12393889042394 74.45752059928924) bank157351 +157352 POINT(40.61441730023137 73.09027910864678) bank157352 +157353 POINT(41.06966937781597 74.16691271462813) bank157353 +157354 POINT(40.20794990460865 74.82235140065268) bank157354 +157355 POINT(41.47382456921519 74.65841813120886) bank157355 +157356 POINT(39.76099771533931 74.33958778557441) bank157356 +157357 POINT(40.138029064797266 73.58180832567396) bank157357 +157358 POINT(40.30886588369825 73.82354910622377) bank157358 +157359 POINT(41.6298398731779 73.4595093588005) bank157359 +157360 POINT(40.27918255600481 74.4998180554056) bank157360 +157361 POINT(41.562476993093576 74.83200136109004) bank157361 +157362 POINT(41.28959882556593 74.36419958464508) bank157362 +157363 POINT(41.50959253348468 74.63231173893725) bank157363 +157364 POINT(41.21770554188412 74.99943932921201) bank157364 +157365 POINT(41.64453902148782 74.33888729253546) bank157365 +157366 POINT(40.93773118203823 73.91570792453277) bank157366 +157367 POINT(41.293011242313504 74.38757885690143) bank157367 +157368 POINT(39.9205654027063 74.12836300593183) bank157368 +157369 POINT(40.87622344548124 73.90157597836217) bank157369 +157370 POINT(40.6934506990321 73.86703167528601) bank157370 +157371 POINT(39.93142410453455 73.25385948795241) bank157371 +157372 POINT(40.60554629268691 73.5010480570724) bank157372 +157373 POINT(39.845616284765974 73.98001296899356) bank157373 +157374 POINT(41.573957295458115 73.43769748474192) bank157374 +157375 POINT(39.95790124438642 74.09920977964012) bank157375 +157376 POINT(40.2357229814168 75.00051332863998) bank157376 +157377 POINT(41.190932848759886 73.93806989387018) bank157377 +157378 POINT(41.5513860785715 74.33343778906251) bank157378 +157379 POINT(40.765861772554 74.22501610395163) bank157379 +157380 POINT(40.01427292606683 74.11604171070547) bank157380 +157381 POINT(40.202962090943224 74.75637974682046) bank157381 +157382 POINT(40.6504469408486 73.74025284343327) bank157382 +157383 POINT(40.90897515991414 74.24361682462091) bank157383 +157384 POINT(41.48745209949953 74.6043323784353) bank157384 +157385 POINT(40.07774333592983 74.46877189458007) bank157385 +157386 POINT(41.19858620830199 74.45517606116206) bank157386 +157387 POINT(41.657340514212194 73.09212760788212) bank157387 +157388 POINT(41.42034574190548 74.11662410393184) bank157388 +157389 POINT(40.9532726815576 73.1290218498505) bank157389 +157390 POINT(40.20297808577873 74.2121813833414) bank157390 +157391 POINT(41.26740774601019 74.8346742026969) bank157391 +157392 POINT(40.52539772883859 73.37022211281685) bank157392 +157393 POINT(41.65582061247356 74.52758710911368) bank157393 +157394 POINT(41.21969214493566 74.35409874253595) bank157394 +157395 POINT(40.382975089434794 73.93964402166503) bank157395 +157396 POINT(40.58102021354962 74.33017119417525) bank157396 +157397 POINT(39.8083425816963 74.19724383748644) bank157397 +157398 POINT(41.130927052605955 74.56175385406105) bank157398 +157399 POINT(40.53290560872865 73.97255974852006) bank157399 +157400 POINT(41.45207299691388 73.94350675172907) bank157400 +157401 POINT(40.160596735747944 74.85211777507617) bank157401 +157402 POINT(39.727627994355345 73.0959580158407) bank157402 +157403 POINT(39.82773620464707 74.55979908020475) bank157403 +157404 POINT(41.16911859607619 73.5290308691537) bank157404 +157405 POINT(40.337595453026566 74.62283018674759) bank157405 +157406 POINT(40.428277796475705 74.09549977245447) bank157406 +157407 POINT(41.296933770125385 74.66593028432773) bank157407 +157408 POINT(40.817120919374325 74.96336114939429) bank157408 +157409 POINT(40.059400471727 74.4828502464513) bank157409 +157410 POINT(41.36730181622779 74.6571268580642) bank157410 +157411 POINT(41.58882072562915 74.08281695786943) bank157411 +157412 POINT(41.307031836348386 73.54077448165674) bank157412 +157413 POINT(39.87096676054212 73.52728405547022) bank157413 +157414 POINT(40.55506787728499 74.20895579531322) bank157414 +157415 POINT(41.49464865004362 73.92535101221794) bank157415 +157416 POINT(40.78566553509141 73.13176686816652) bank157416 +157417 POINT(40.39161485487399 73.07960047243171) bank157417 +157418 POINT(40.14232374662535 74.68312851080736) bank157418 +157419 POINT(40.59258623695951 74.46780594715881) bank157419 +157420 POINT(41.301044154274095 74.53350604573112) bank157420 +157421 POINT(40.94123583824157 73.0824738880455) bank157421 +157422 POINT(40.169596652640315 74.42115413812432) bank157422 +157423 POINT(41.50942905093366 74.73873212317332) bank157423 +157424 POINT(39.97342308636317 74.40127723319156) bank157424 +157425 POINT(41.53054735323816 73.57908403145105) bank157425 +157426 POINT(40.24810461667988 73.78679496423995) bank157426 +157427 POINT(41.35817234448385 74.60128771674792) bank157427 +157428 POINT(41.52730372004069 73.63880972672598) bank157428 +157429 POINT(40.89594335637962 73.56746138727766) bank157429 +157430 POINT(40.822304304062335 73.79455908677828) bank157430 +157431 POINT(41.68339081177515 74.67195386231502) bank157431 +157432 POINT(41.117667484561146 73.02777269788143) bank157432 +157433 POINT(40.59031278989131 74.29988763360271) bank157433 +157434 POINT(40.37009220117738 73.66166674960378) bank157434 +157435 POINT(40.74941972838444 74.05715211013157) bank157435 +157436 POINT(41.59585410469065 74.37718886255247) bank157436 +157437 POINT(39.92653975163286 73.36108640918826) bank157437 +157438 POINT(40.74315551894629 73.16936605038504) bank157438 +157439 POINT(39.81564940072619 74.75953991347977) bank157439 +157440 POINT(40.789495443092484 73.17662479836432) bank157440 +157441 POINT(40.03443105420511 73.51873326586501) bank157441 +157442 POINT(40.461573821962254 74.56782363425053) bank157442 +157443 POINT(40.20988385319063 74.72729169325076) bank157443 +157444 POINT(41.39015064494576 74.69846743917006) bank157444 +157445 POINT(40.49780563530528 73.6556031816718) bank157445 +157446 POINT(40.4614333756079 73.1809671448672) bank157446 +157447 POINT(41.43617690205144 73.52584742658938) bank157447 +157448 POINT(40.465324302525914 73.0841205203414) bank157448 +157449 POINT(40.34532331910067 73.32098902939792) bank157449 +157450 POINT(41.23032238250997 74.483964429575) bank157450 +157451 POINT(41.54587223922295 74.73258620873958) bank157451 +157452 POINT(40.44081045662326 74.34637977968784) bank157452 +157453 POINT(40.45602096512799 74.74766041640993) bank157453 +157454 POINT(41.402300464657 74.09925536776674) bank157454 +157455 POINT(40.77071628103245 74.75515773889197) bank157455 +157456 POINT(40.10013004327629 74.88455276178337) bank157456 +157457 POINT(40.41110740267131 74.64888470447926) bank157457 +157458 POINT(40.418046122745764 73.62169120410732) bank157458 +157459 POINT(40.7409709568735 73.96173361756678) bank157459 +157460 POINT(41.281670161803724 74.43012218943454) bank157460 +157461 POINT(40.3594016841917 74.86124100725644) bank157461 +157462 POINT(41.57886692306262 73.05063634317503) bank157462 +157463 POINT(41.02280705667313 74.3961254321517) bank157463 +157464 POINT(40.24813772131325 74.29948471914518) bank157464 +157465 POINT(41.040564048836885 73.0799268719451) bank157465 +157466 POINT(40.58051368558423 73.8046788515654) bank157466 +157467 POINT(39.71914136844788 74.76158322814075) bank157467 +157468 POINT(40.56434953126655 74.40183207827047) bank157468 +157469 POINT(41.526016055109594 73.4496211420897) bank157469 +157470 POINT(40.06731621153638 74.61543124427592) bank157470 +157471 POINT(40.373483122440916 74.83498047897329) bank157471 +157472 POINT(40.10304131574481 73.26502879713996) bank157472 +157473 POINT(40.3207949152481 73.25065927330039) bank157473 +157474 POINT(39.897420490569694 73.61338931055086) bank157474 +157475 POINT(40.23627872690263 74.77874845911964) bank157475 +157476 POINT(39.78102639385118 73.00839796858693) bank157476 +157477 POINT(40.388811459875456 73.99012526353636) bank157477 +157478 POINT(40.44034059927546 74.82962579168856) bank157478 +157479 POINT(40.467872479864084 74.80911697474863) bank157479 +157480 POINT(41.49947968307805 73.3663581160759) bank157480 +157481 POINT(40.51756010339089 74.3480620391283) bank157481 +157482 POINT(41.39528683394566 73.85479737049853) bank157482 +157483 POINT(40.70430126794821 73.14748281149481) bank157483 +157484 POINT(40.944293518677135 74.63740215374051) bank157484 +157485 POINT(41.446607233313344 74.76388400665759) bank157485 +157486 POINT(39.86409873927169 73.4755327847149) bank157486 +157487 POINT(40.747771744128464 74.14142466479953) bank157487 +157488 POINT(40.18939563770724 73.94434036215992) bank157488 +157489 POINT(40.132188949526366 74.69951866302489) bank157489 +157490 POINT(40.02046593613035 73.9620769632683) bank157490 +157491 POINT(41.54500384972154 74.08776464533939) bank157491 +157492 POINT(40.99686327639135 74.32711787331574) bank157492 +157493 POINT(40.377945041892 74.10213164096666) bank157493 +157494 POINT(40.927884744249425 73.41937686532148) bank157494 +157495 POINT(40.75855403611469 73.43799243797297) bank157495 +157496 POINT(40.667389386296946 73.06868866602518) bank157496 +157497 POINT(41.46518444300318 74.38567296008424) bank157497 +157498 POINT(39.92510887329953 74.21279301916901) bank157498 +157499 POINT(41.357813076789846 73.31010354049285) bank157499 +157500 POINT(41.22035748410287 74.61268905011104) bank157500 +157501 POINT(40.92942027835224 73.99290508425997) bank157501 +157502 POINT(41.41621851748739 74.66041064531777) bank157502 +157503 POINT(40.816066551278425 74.10974440855541) bank157503 +157504 POINT(40.46463941823758 73.38932735227044) bank157504 +157505 POINT(41.06730953716675 74.90565077155459) bank157505 +157506 POINT(39.958877715952745 74.94874515464315) bank157506 +157507 POINT(39.71352292597992 74.10049155516607) bank157507 +157508 POINT(40.89082563169562 73.04540286661226) bank157508 +157509 POINT(40.94016740389517 73.09813702521492) bank157509 +157510 POINT(41.53087885705959 74.7263269644526) bank157510 +157511 POINT(40.5657883597886 74.74926299750251) bank157511 +157512 POINT(39.85662944535545 73.47571935300779) bank157512 +157513 POINT(41.52994227284135 74.08795011245876) bank157513 +157514 POINT(40.65787445167721 73.58067379907646) bank157514 +157515 POINT(40.16394770154929 74.72277224551425) bank157515 +157516 POINT(40.60106678253583 73.69844458766471) bank157516 +157517 POINT(41.14190722035731 74.24545053703366) bank157517 +157518 POINT(40.05975858249322 74.19900180645236) bank157518 +157519 POINT(40.61331911073545 73.67254201867092) bank157519 +157520 POINT(41.31770504841596 74.5808981128035) bank157520 +157521 POINT(41.4860073484646 73.99144893660333) bank157521 +157522 POINT(40.452497403524355 73.08129316483947) bank157522 +157523 POINT(40.67250827434179 74.69712353821805) bank157523 +157524 POINT(41.27542905155763 74.94618351692895) bank157524 +157525 POINT(40.32834017051733 74.09650023290571) bank157525 +157526 POINT(41.63380319591583 74.81448376851088) bank157526 +157527 POINT(40.770201723089116 73.20955343443879) bank157527 +157528 POINT(40.93594342959058 73.62784189984964) bank157528 +157529 POINT(40.22646913290046 74.42907706219628) bank157529 +157530 POINT(39.96457915204193 73.20917838518226) bank157530 +157531 POINT(40.38508382947787 73.33186377376936) bank157531 +157532 POINT(39.9570481255505 73.87117043727139) bank157532 +157533 POINT(39.88511473983979 74.2550479802092) bank157533 +157534 POINT(40.09526554596188 74.93437225355058) bank157534 +157535 POINT(40.802850547999746 74.27203193663624) bank157535 +157536 POINT(41.217244024360696 73.3662375655382) bank157536 +157537 POINT(40.165974262891766 73.52469418290195) bank157537 +157538 POINT(41.654991201628164 73.01477762746894) bank157538 +157539 POINT(41.294260461906134 73.5407785661896) bank157539 +157540 POINT(40.30126484603281 73.15994653289647) bank157540 +157541 POINT(40.682667738175034 74.84371509987632) bank157541 +157542 POINT(40.82797945795856 73.25953345621477) bank157542 +157543 POINT(41.01621747393089 74.5127137611825) bank157543 +157544 POINT(40.04886504698705 74.11401168768053) bank157544 +157545 POINT(41.185125552965104 73.28901270059556) bank157545 +157546 POINT(41.23754371790005 74.8120559083007) bank157546 +157547 POINT(39.83320806087136 74.3862161801421) bank157547 +157548 POINT(40.679002763322515 74.19894044405372) bank157548 +157549 POINT(40.03052155078471 74.73311062737564) bank157549 +157550 POINT(41.52630861725308 73.42065255092758) bank157550 +157551 POINT(40.32363678600411 74.34220599645671) bank157551 +157552 POINT(41.255454912314946 74.15160208865517) bank157552 +157553 POINT(40.47660376733054 74.35612450317588) bank157553 +157554 POINT(41.64685846290019 74.16921515654576) bank157554 +157555 POINT(41.149480739012425 74.10382287802815) bank157555 +157556 POINT(39.90478945204011 74.50083450806413) bank157556 +157557 POINT(41.67723448289265 73.99510522987029) bank157557 +157558 POINT(39.75505595290544 73.51491721662245) bank157558 +157559 POINT(41.170853866879476 73.10030786480807) bank157559 +157560 POINT(41.454025253663566 73.61228355660892) bank157560 +157561 POINT(41.60339800615216 73.25028476885687) bank157561 +157562 POINT(39.7770230945347 74.8668175956685) bank157562 +157563 POINT(40.49240393781286 74.01314387742968) bank157563 +157564 POINT(40.95340945681511 73.74825378804479) bank157564 +157565 POINT(40.53219108854284 73.35404529873811) bank157565 +157566 POINT(39.817554293665715 74.90478380602262) bank157566 +157567 POINT(40.47853822163039 73.77699212546494) bank157567 +157568 POINT(41.63627501588157 74.32806826963834) bank157568 +157569 POINT(40.27340869780842 74.83953754268778) bank157569 +157570 POINT(41.35271207099085 73.99273970794809) bank157570 +157571 POINT(40.976621514675635 74.63979374240935) bank157571 +157572 POINT(39.82750411007 73.81338928677012) bank157572 +157573 POINT(39.98108161234825 73.26218897438832) bank157573 +157574 POINT(40.15399800337642 73.4836065161814) bank157574 +157575 POINT(41.11110800242542 73.06128299292912) bank157575 +157576 POINT(41.346805064457826 74.97268561276003) bank157576 +157577 POINT(41.53647891661829 73.09712348306905) bank157577 +157578 POINT(41.60226317871839 74.37269301097668) bank157578 +157579 POINT(40.7432965255888 73.69292862369115) bank157579 +157580 POINT(40.78424563995757 73.14206017185492) bank157580 +157581 POINT(40.44817872742512 74.01004256892531) bank157581 +157582 POINT(40.166630315579376 74.81153970908525) bank157582 +157583 POINT(39.92910409538861 73.3889584184744) bank157583 +157584 POINT(39.81090991308136 74.17039531785395) bank157584 +157585 POINT(40.552544638039386 74.68101119720207) bank157585 +157586 POINT(39.72075362409311 74.29883593753718) bank157586 +157587 POINT(40.58597087183296 73.23183911093668) bank157587 +157588 POINT(41.47110517886028 73.9011933940763) bank157588 +157589 POINT(40.57927700301334 74.84453061134755) bank157589 +157590 POINT(40.31009452360266 74.04995413653172) bank157590 +157591 POINT(40.100824096928626 73.58502959331753) bank157591 +157592 POINT(41.21445215166774 73.65906690759292) bank157592 +157593 POINT(40.7064551967272 74.28823803453463) bank157593 +157594 POINT(39.832642051918974 74.98290269782113) bank157594 +157595 POINT(40.683366184502 74.51462356371539) bank157595 +157596 POINT(39.73056324424699 73.62856787675021) bank157596 +157597 POINT(40.76197794443874 73.93099353857555) bank157597 +157598 POINT(41.24987654059697 74.23746781333641) bank157598 +157599 POINT(40.104085437511806 73.47863850193917) bank157599 +157600 POINT(40.954175629346054 73.02197540823127) bank157600 +157601 POINT(39.80342200015807 74.36345614682912) bank157601 +157602 POINT(41.03539997360282 73.94136334591425) bank157602 +157603 POINT(40.61328429425587 73.06144187788482) bank157603 +157604 POINT(39.80416823641461 74.72023002663803) bank157604 +157605 POINT(40.75566530684753 73.0514542498517) bank157605 +157606 POINT(41.03879057971159 73.83747915911763) bank157606 +157607 POINT(41.15227798072113 74.4255858555704) bank157607 +157608 POINT(40.8513920012512 73.68134385510464) bank157608 +157609 POINT(40.18278513910309 74.80018202182846) bank157609 +157610 POINT(40.79690854875368 73.1503220540097) bank157610 +157611 POINT(41.1143397925028 73.43758359978871) bank157611 +157612 POINT(40.860716103048944 74.75166305798037) bank157612 +157613 POINT(41.36261415660944 73.9223911977539) bank157613 +157614 POINT(40.911256177568696 74.15394286993003) bank157614 +157615 POINT(40.61811035212925 74.80391517831042) bank157615 +157616 POINT(40.3540018548331 73.73693596634072) bank157616 +157617 POINT(39.97033387085322 73.55188963871576) bank157617 +157618 POINT(40.85342565372032 73.54803946451652) bank157618 +157619 POINT(40.99239265150754 74.01070319330395) bank157619 +157620 POINT(40.22438996887221 73.68957692163444) bank157620 +157621 POINT(41.06357777235998 73.914808848625) bank157621 +157622 POINT(39.901057290724985 74.69094753321009) bank157622 +157623 POINT(40.97851377958028 73.95058019744057) bank157623 +157624 POINT(40.51578815191158 74.10706604951532) bank157624 +157625 POINT(40.55821571689434 74.35539637100419) bank157625 +157626 POINT(41.26148995586261 74.25303949659784) bank157626 +157627 POINT(41.13166115611998 74.79080033084514) bank157627 +157628 POINT(40.33365940271382 74.96760456763113) bank157628 +157629 POINT(40.78277342711868 74.97080927577898) bank157629 +157630 POINT(41.62989384199088 73.37160310558617) bank157630 +157631 POINT(39.92273356933855 74.01579509136577) bank157631 +157632 POINT(39.8975425646048 73.67180755500578) bank157632 +157633 POINT(41.17252504925104 73.10894687061258) bank157633 +157634 POINT(40.24381954349174 74.14946801741897) bank157634 +157635 POINT(41.32603761784894 73.09991503650353) bank157635 +157636 POINT(41.50455508915179 73.0385125795084) bank157636 +157637 POINT(40.47970851942204 74.73664811102216) bank157637 +157638 POINT(40.20521004522393 73.21630650352917) bank157638 +157639 POINT(40.36468818759667 74.6264583648718) bank157639 +157640 POINT(40.04161578006332 73.66709095004445) bank157640 +157641 POINT(40.746987122534456 73.55580772384248) bank157641 +157642 POINT(40.361514294052235 73.55292133258956) bank157642 +157643 POINT(40.60668592619654 74.14855815540615) bank157643 +157644 POINT(41.37503826205878 73.50299812021134) bank157644 +157645 POINT(41.47773024031488 74.10287639695193) bank157645 +157646 POINT(40.296971153681504 73.33066238482577) bank157646 +157647 POINT(40.63582861906326 74.686313646377) bank157647 +157648 POINT(41.52256644871132 73.12380603556886) bank157648 +157649 POINT(41.565520908650534 73.90604224750976) bank157649 +157650 POINT(40.75505584100234 73.67866572633936) bank157650 +157651 POINT(41.01964179685656 74.43228344446433) bank157651 +157652 POINT(39.949508166160584 74.84916321261052) bank157652 +157653 POINT(40.8886071232085 73.84760142326148) bank157653 +157654 POINT(40.25337681138729 73.65978121445667) bank157654 +157655 POINT(40.59796045594955 73.14088662831894) bank157655 +157656 POINT(40.20030422951253 74.5084832247554) bank157656 +157657 POINT(41.277106799619546 73.34506482134063) bank157657 +157658 POINT(41.63538637731291 74.09936153471698) bank157658 +157659 POINT(41.473003275627725 73.29541862562009) bank157659 +157660 POINT(40.23183022681724 74.44907127820531) bank157660 +157661 POINT(40.53776566224125 74.41431620490526) bank157661 +157662 POINT(40.568625063965534 74.43617447872859) bank157662 +157663 POINT(39.96111563265221 73.84050058508193) bank157663 +157664 POINT(40.61483742773034 73.54023565449522) bank157664 +157665 POINT(40.49962775432539 73.96377967137856) bank157665 +157666 POINT(40.701308165169756 73.30926296782715) bank157666 +157667 POINT(39.76168262961768 74.5710384034104) bank157667 +157668 POINT(40.35044992458098 73.28404506457568) bank157668 +157669 POINT(40.64318995822509 74.69264022796217) bank157669 +157670 POINT(40.26353062144954 73.84333697057595) bank157670 +157671 POINT(41.65020956191712 73.18183133707804) bank157671 +157672 POINT(41.090364414648285 74.30488358270564) bank157672 +157673 POINT(40.15148347490298 74.69327374448925) bank157673 +157674 POINT(40.589369813103445 73.4854736590505) bank157674 +157675 POINT(39.73994702760184 74.63798030970142) bank157675 +157676 POINT(41.22961691283744 73.2092558129864) bank157676 +157677 POINT(41.22794597295079 74.8122184251993) bank157677 +157678 POINT(39.81976971910527 73.90630881781442) bank157678 +157679 POINT(39.727778083095515 73.2074639015103) bank157679 +157680 POINT(40.90803515025508 73.2669446161144) bank157680 +157681 POINT(41.20534557490651 74.98211426011332) bank157681 +157682 POINT(40.48135119631906 74.47459154572068) bank157682 +157683 POINT(41.02818911948166 73.88156248268982) bank157683 +157684 POINT(41.64277589007815 74.53344158169082) bank157684 +157685 POINT(40.915176459509325 74.4793948096753) bank157685 +157686 POINT(41.24156456944919 74.43167990305407) bank157686 +157687 POINT(40.88574249725561 74.97428825760194) bank157687 +157688 POINT(40.390844962434116 74.4958319007566) bank157688 +157689 POINT(41.69187830138579 74.49071493940082) bank157689 +157690 POINT(40.195534592194086 74.27224191081123) bank157690 +157691 POINT(40.119012963622026 73.69826004969396) bank157691 +157692 POINT(40.79723149982495 73.14143127898096) bank157692 +157693 POINT(41.52574014855165 73.39479971508167) bank157693 +157694 POINT(39.821100758768985 73.32946927414743) bank157694 +157695 POINT(41.05825679689212 74.4316901226453) bank157695 +157696 POINT(40.13332697800472 73.65582856743035) bank157696 +157697 POINT(39.79385525661361 73.96258188830367) bank157697 +157698 POINT(41.43262125634211 73.44572645403565) bank157698 +157699 POINT(40.55442751980308 74.75371482554974) bank157699 +157700 POINT(40.555847976359516 73.40315291926412) bank157700 +157701 POINT(41.623794735301324 74.95601831403121) bank157701 +157702 POINT(40.01613543600518 74.16780507621593) bank157702 +157703 POINT(41.477398744770184 74.84883508239429) bank157703 +157704 POINT(40.229058426257545 74.41093071060891) bank157704 +157705 POINT(40.60547466801886 74.2550620432119) bank157705 +157706 POINT(40.47453973295183 73.31292440244749) bank157706 +157707 POINT(40.021179602356746 74.93537123168883) bank157707 +157708 POINT(40.811088260489214 74.82700873678826) bank157708 +157709 POINT(40.718617663124704 73.35786194034195) bank157709 +157710 POINT(40.74310559549607 73.95171963335329) bank157710 +157711 POINT(41.669083914280506 74.36967501749754) bank157711 +157712 POINT(41.0909711318075 73.97725931676129) bank157712 +157713 POINT(40.87867045879773 74.5716544966537) bank157713 +157714 POINT(40.17069505795897 74.37876433381012) bank157714 +157715 POINT(41.690643989401515 74.33602730467861) bank157715 +157716 POINT(40.60828556508906 73.44694867056572) bank157716 +157717 POINT(39.94959747868031 74.51595334968967) bank157717 +157718 POINT(40.43463074825426 73.73258004866366) bank157718 +157719 POINT(41.49961834465163 73.7510537921585) bank157719 +157720 POINT(41.52898373552226 74.80423426665122) bank157720 +157721 POINT(41.398620494509814 74.91809736785756) bank157721 +157722 POINT(41.15944014649965 74.2648222260224) bank157722 +157723 POINT(41.35734963817747 74.0147877790532) bank157723 +157724 POINT(40.14532220120574 73.80686388531703) bank157724 +157725 POINT(40.67698402977412 73.43331700181093) bank157725 +157726 POINT(40.380207110422845 73.09235967301524) bank157726 +157727 POINT(40.91011546351667 74.54296205426704) bank157727 +157728 POINT(40.94379546194057 74.50552233649924) bank157728 +157729 POINT(39.968126604975765 74.85625423612704) bank157729 +157730 POINT(41.44552532717228 73.55394936244232) bank157730 +157731 POINT(40.05522913769201 73.15938176363858) bank157731 +157732 POINT(40.19111940673527 73.5149519357406) bank157732 +157733 POINT(41.22433229206471 73.5821206194488) bank157733 +157734 POINT(40.845816494011615 73.5344454182822) bank157734 +157735 POINT(41.525090471282105 73.3510188877177) bank157735 +157736 POINT(40.18269294929702 75.00332522824314) bank157736 +157737 POINT(41.509325642398096 73.16332402545113) bank157737 +157738 POINT(40.600596513314414 73.89176342805916) bank157738 +157739 POINT(39.91185821968898 73.76981571696133) bank157739 +157740 POINT(40.139997422544674 74.01767719507183) bank157740 +157741 POINT(41.028547695392916 73.53358797156262) bank157741 +157742 POINT(40.300005172771094 73.30868929349704) bank157742 +157743 POINT(40.29830418347055 74.83159127666669) bank157743 +157744 POINT(39.81571467848219 74.74840670263325) bank157744 +157745 POINT(39.80503506606886 74.2195095645991) bank157745 +157746 POINT(41.371649000295854 73.22923876594449) bank157746 +157747 POINT(39.845261999723945 73.98326031382092) bank157747 +157748 POINT(39.86841014880219 73.86417271024405) bank157748 +157749 POINT(40.33763703023586 73.31982554185028) bank157749 +157750 POINT(40.328806063005125 74.24502004246813) bank157750 +157751 POINT(40.14081478159724 73.78619313585696) bank157751 +157752 POINT(41.695638664532716 73.86686619354936) bank157752 +157753 POINT(41.29190619697893 73.09267032334661) bank157753 +157754 POINT(41.078191876110296 73.27050691935227) bank157754 +157755 POINT(41.21774883481458 73.48621013075845) bank157755 +157756 POINT(41.06708627557575 73.53331335158026) bank157756 +157757 POINT(40.1206194055998 74.48218696465662) bank157757 +157758 POINT(41.65312505021449 74.36820720282205) bank157758 +157759 POINT(39.92392294717296 73.3031919412276) bank157759 +157760 POINT(40.728144222278445 73.68106938061233) bank157760 +157761 POINT(40.02683604991525 74.8118774475545) bank157761 +157762 POINT(41.45726309913412 74.6350633395104) bank157762 +157763 POINT(41.52837737162123 73.19708700431346) bank157763 +157764 POINT(41.698684362083604 74.82998864340612) bank157764 +157765 POINT(40.162421816981386 73.7163509194287) bank157765 +157766 POINT(39.7184694007954 73.76404280226095) bank157766 +157767 POINT(40.57603084867097 74.98426842005003) bank157767 +157768 POINT(40.286394124840626 74.71763346158791) bank157768 +157769 POINT(40.730845738025934 74.65519588208839) bank157769 +157770 POINT(39.985180669536355 74.2816161310489) bank157770 +157771 POINT(40.70484806706816 73.45132975200724) bank157771 +157772 POINT(40.08594096187765 74.1168456916627) bank157772 +157773 POINT(40.28037407498088 73.03308761220198) bank157773 +157774 POINT(39.94593174488287 73.63585116388917) bank157774 +157775 POINT(41.45932142893252 74.69339854502536) bank157775 +157776 POINT(41.35620308456466 73.62175325661586) bank157776 +157777 POINT(40.974764363900306 74.91360262533657) bank157777 +157778 POINT(39.72046125376983 73.06446800246759) bank157778 +157779 POINT(40.185781131349884 74.13520002589661) bank157779 +157780 POINT(40.7686122972032 73.31965272424263) bank157780 +157781 POINT(40.63712087329663 73.13917734168943) bank157781 +157782 POINT(40.74462818722941 73.56328285535082) bank157782 +157783 POINT(41.15553611306144 74.79998311347653) bank157783 +157784 POINT(41.43703052415471 73.84802878300324) bank157784 +157785 POINT(40.92447383765875 74.76667769627267) bank157785 +157786 POINT(40.84178119574397 74.78025927875709) bank157786 +157787 POINT(41.401911539462354 74.13249930350791) bank157787 +157788 POINT(41.43340325415938 73.04079753949318) bank157788 +157789 POINT(39.961974000845544 73.40764104944009) bank157789 +157790 POINT(40.673613393385885 74.706881989703) bank157790 +157791 POINT(41.1733570802016 73.41193983173582) bank157791 +157792 POINT(40.060145557586274 74.37710024345665) bank157792 +157793 POINT(41.643549487349446 74.98501154620207) bank157793 +157794 POINT(41.494617541782326 73.50462416638187) bank157794 +157795 POINT(40.88371713819521 74.84616044043804) bank157795 +157796 POINT(41.595990423135284 73.27786186662975) bank157796 +157797 POINT(40.291572379968 73.27172039877135) bank157797 +157798 POINT(41.043972521877144 74.64755669061915) bank157798 +157799 POINT(40.406980605163135 73.9643105788384) bank157799 +157800 POINT(41.545663803195936 74.00129937855083) bank157800 +157801 POINT(40.59989716102801 73.85511454530847) bank157801 +157802 POINT(39.94454443354635 73.8311998443068) bank157802 +157803 POINT(39.73398838884595 73.50494671287991) bank157803 +157804 POINT(41.029075075166396 73.81144331749029) bank157804 +157805 POINT(40.58583896275882 74.30443252538466) bank157805 +157806 POINT(41.000240679248755 74.6264982479458) bank157806 +157807 POINT(40.886618503233805 74.36756549681785) bank157807 +157808 POINT(39.933921921571546 73.30758294718694) bank157808 +157809 POINT(40.87238762827538 74.35094324586801) bank157809 +157810 POINT(41.08341967265451 73.62154463112195) bank157810 +157811 POINT(40.14206737208382 73.1725728948953) bank157811 +157812 POINT(40.89536249710455 73.55225514706595) bank157812 +157813 POINT(41.219366505431104 74.97299208823108) bank157813 +157814 POINT(40.94064994214532 74.98297539100362) bank157814 +157815 POINT(40.38588983672225 74.20297005830801) bank157815 +157816 POINT(40.47866422162374 74.26099491641256) bank157816 +157817 POINT(40.31705928741479 73.48491393820308) bank157817 +157818 POINT(40.87400046286486 73.99326025999879) bank157818 +157819 POINT(39.939581815781494 73.54063878058004) bank157819 +157820 POINT(40.90102189478113 74.10603161338017) bank157820 +157821 POINT(40.04414671693076 74.44388496545032) bank157821 +157822 POINT(40.12613308804179 73.31907121611052) bank157822 +157823 POINT(40.70433442876113 74.08952142398246) bank157823 +157824 POINT(40.071104453033456 74.3249175221167) bank157824 +157825 POINT(41.211166963744276 73.29949870220824) bank157825 +157826 POINT(40.744078262174405 73.95422069163298) bank157826 +157827 POINT(40.08832298102037 74.52737242663574) bank157827 +157828 POINT(41.31600206645507 74.4997455998795) bank157828 +157829 POINT(41.06804090748092 73.44918008039248) bank157829 +157830 POINT(41.065086980539604 73.54031423494729) bank157830 +157831 POINT(40.5387526869967 74.07536332349967) bank157831 +157832 POINT(40.08294864829579 74.50835382116017) bank157832 +157833 POINT(39.79310411668741 74.3357237364492) bank157833 +157834 POINT(40.97887076060957 73.98952729346976) bank157834 +157835 POINT(41.26710996955163 74.80838626261081) bank157835 +157836 POINT(41.06929036719813 73.9232504000019) bank157836 +157837 POINT(41.675975643538614 73.09742254377589) bank157837 +157838 POINT(40.165007226371394 73.11664963109199) bank157838 +157839 POINT(39.968264131859314 74.17232970355631) bank157839 +157840 POINT(39.99584862181142 73.70262876822964) bank157840 +157841 POINT(39.85675209387854 74.47006579356457) bank157841 +157842 POINT(41.16394039083084 74.59165173887301) bank157842 +157843 POINT(41.589519979483754 73.78041087826266) bank157843 +157844 POINT(39.727976652195494 74.54805337628422) bank157844 +157845 POINT(40.43809353717313 73.72706725896401) bank157845 +157846 POINT(40.653236794237785 74.56245033075618) bank157846 +157847 POINT(40.84229775800018 74.40783404292584) bank157847 +157848 POINT(39.84164664234577 73.97447400125603) bank157848 +157849 POINT(41.05678034454353 74.57881361469745) bank157849 +157850 POINT(41.14236347826822 74.16512638173026) bank157850 +157851 POINT(41.10552231989201 74.69615716990306) bank157851 +157852 POINT(41.59021049373672 73.45744817710015) bank157852 +157853 POINT(41.35398438998773 74.62568586975671) bank157853 +157854 POINT(39.95985386042217 74.28572821231162) bank157854 +157855 POINT(41.14390979571737 74.42040550837605) bank157855 +157856 POINT(40.641109224171075 74.45065632830631) bank157856 +157857 POINT(40.32602445928757 73.0992538107254) bank157857 +157858 POINT(39.751250008345025 73.14845449297879) bank157858 +157859 POINT(40.81362753740207 73.02775522865099) bank157859 +157860 POINT(40.292835154812074 73.321956787863) bank157860 +157861 POINT(40.389738480051 73.01868786406017) bank157861 +157862 POINT(39.87095060632208 73.36796903828375) bank157862 +157863 POINT(41.71084319359512 74.50940370842281) bank157863 +157864 POINT(40.848548576237675 74.41199082620999) bank157864 +157865 POINT(40.660726144290834 74.93126235669074) bank157865 +157866 POINT(41.45847210843907 74.71864281102945) bank157866 +157867 POINT(40.30219472063788 73.46996067378478) bank157867 +157868 POINT(40.176611995782515 74.2645983830706) bank157868 +157869 POINT(41.53321066389811 73.8336553096903) bank157869 +157870 POINT(40.521981214496655 73.43712252428003) bank157870 +157871 POINT(40.82978080122442 73.8768111009914) bank157871 +157872 POINT(41.452781783474705 73.74815291861121) bank157872 +157873 POINT(40.60263930584674 73.97014879061965) bank157873 +157874 POINT(41.20703561802028 74.05832040209047) bank157874 +157875 POINT(39.82772282372984 74.48399778594955) bank157875 +157876 POINT(41.251874402287 74.11523549735065) bank157876 +157877 POINT(41.166178491784166 74.52306130671799) bank157877 +157878 POINT(41.566436405343886 73.99392269160899) bank157878 +157879 POINT(40.23921898884966 74.49488101135437) bank157879 +157880 POINT(40.263321487497166 73.43896952198975) bank157880 +157881 POINT(41.21761265480903 73.07285340821973) bank157881 +157882 POINT(39.943631787341225 74.14837934448835) bank157882 +157883 POINT(40.27539015915945 74.24144323036981) bank157883 +157884 POINT(41.017100657804455 73.02130313153759) bank157884 +157885 POINT(39.80438998095418 73.25723033505919) bank157885 +157886 POINT(41.557828544385956 73.71486652933305) bank157886 +157887 POINT(40.93774842841574 74.88220290973354) bank157887 +157888 POINT(41.513659283432894 73.63701659343154) bank157888 +157889 POINT(39.815818865093156 73.95090332431143) bank157889 +157890 POINT(41.31553762756338 73.01517812502749) bank157890 +157891 POINT(40.61450037815177 73.36343133205887) bank157891 +157892 POINT(40.72023359406605 73.22423231139071) bank157892 +157893 POINT(39.94577367047759 73.37545974050855) bank157893 +157894 POINT(40.1360850203196 73.78403819304971) bank157894 +157895 POINT(40.783299851981766 74.58838374659578) bank157895 +157896 POINT(41.496583851003706 74.73253163980583) bank157896 +157897 POINT(41.10064965812535 73.66435013122043) bank157897 +157898 POINT(40.54932363534268 74.76164255299733) bank157898 +157899 POINT(41.06856411844355 73.78006459546363) bank157899 +157900 POINT(41.04445299380272 73.38191657634952) bank157900 +157901 POINT(41.3468183001723 73.18681869184034) bank157901 +157902 POINT(40.697539899695755 74.64888876003754) bank157902 +157903 POINT(40.9748816461816 74.31279440876266) bank157903 +157904 POINT(40.234363266314986 74.98569237030362) bank157904 +157905 POINT(40.09476673446165 73.37965374442676) bank157905 +157906 POINT(41.262688601701065 73.71832931331727) bank157906 +157907 POINT(41.51971427081799 74.06992277374542) bank157907 +157908 POINT(40.966143420690344 74.29801592340908) bank157908 +157909 POINT(41.01496140354819 74.05005295846279) bank157909 +157910 POINT(41.44685817593937 73.98884274677008) bank157910 +157911 POINT(41.151422670914215 73.62729880975165) bank157911 +157912 POINT(40.09265868019418 74.59243278800321) bank157912 +157913 POINT(40.81515641254243 74.83618935586568) bank157913 +157914 POINT(40.015580410306185 73.83453576375885) bank157914 +157915 POINT(40.93768548949912 73.87536588410762) bank157915 +157916 POINT(41.135430234881625 74.15630034777875) bank157916 +157917 POINT(41.39390905655921 73.47004367355589) bank157917 +157918 POINT(40.703635311675235 74.63742278715216) bank157918 +157919 POINT(40.537867464851715 73.91342242645041) bank157919 +157920 POINT(41.37561276283118 73.98619505198906) bank157920 +157921 POINT(40.21480900661042 73.93431221384103) bank157921 +157922 POINT(39.84056957166682 73.67079757448062) bank157922 +157923 POINT(40.15885199506658 74.88594728254222) bank157923 +157924 POINT(40.575427698801946 74.16386226400762) bank157924 +157925 POINT(40.012636966540846 73.65510424882856) bank157925 +157926 POINT(41.06973940623912 74.14213032421893) bank157926 +157927 POINT(40.523775388475194 74.73571529320024) bank157927 +157928 POINT(40.17402386722992 74.488757875288) bank157928 +157929 POINT(40.02588000143492 74.29998266691001) bank157929 +157930 POINT(40.82781585841745 73.1767067364784) bank157930 +157931 POINT(40.4722633865761 73.53634846311368) bank157931 +157932 POINT(40.57302972952173 74.13495494872744) bank157932 +157933 POINT(41.6936961428794 74.50733108705458) bank157933 +157934 POINT(40.8235560014109 73.1576395764178) bank157934 +157935 POINT(39.86945123938122 74.53809454852804) bank157935 +157936 POINT(40.98262689231543 74.82007272434291) bank157936 +157937 POINT(40.143301032733795 73.91757950776066) bank157937 +157938 POINT(40.50880703240568 73.13530439384976) bank157938 +157939 POINT(40.264591975836055 74.01494894497968) bank157939 +157940 POINT(39.95529245881934 74.50980596386077) bank157940 +157941 POINT(41.43305289948989 74.21132938112142) bank157941 +157942 POINT(39.94854603090549 74.5552281807375) bank157942 +157943 POINT(40.63382462874803 73.95491013469395) bank157943 +157944 POINT(41.08382086594362 73.01357822750502) bank157944 +157945 POINT(39.81371460546823 74.51762673221317) bank157945 +157946 POINT(41.09117733142307 73.94980253581843) bank157946 +157947 POINT(40.40201749288222 74.86855735071681) bank157947 +157948 POINT(40.27092638171319 74.25387733279604) bank157948 +157949 POINT(41.56177557998641 73.41039104858383) bank157949 +157950 POINT(40.829876854034545 73.08154386820057) bank157950 +157951 POINT(40.80820907953351 74.51225271443408) bank157951 +157952 POINT(40.99589650919398 73.52798093804) bank157952 +157953 POINT(40.724993633697196 74.96705478753954) bank157953 +157954 POINT(41.32040426773894 73.6834260043106) bank157954 +157955 POINT(41.343601691416374 73.67339906135116) bank157955 +157956 POINT(40.214520970171144 74.68291348555368) bank157956 +157957 POINT(41.23020421488396 73.88293382465416) bank157957 +157958 POINT(39.976191254145505 74.47680956233468) bank157958 +157959 POINT(40.729086791259114 73.69933617484888) bank157959 +157960 POINT(39.8599756779515 73.29143672268195) bank157960 +157961 POINT(40.40008954715222 74.96249046565173) bank157961 +157962 POINT(40.87234325515129 74.47567754640613) bank157962 +157963 POINT(40.986195730958045 74.76645688700192) bank157963 +157964 POINT(40.35937004729126 74.46262017929169) bank157964 +157965 POINT(40.6423453156709 73.893028987411) bank157965 +157966 POINT(41.29097515937747 74.68699845391937) bank157966 +157967 POINT(39.93545177902756 73.69123512007751) bank157967 +157968 POINT(40.29479560543605 74.89281392788409) bank157968 +157969 POINT(40.27118348124973 73.74413900595815) bank157969 +157970 POINT(41.48293845359435 73.92275195930196) bank157970 +157971 POINT(41.39973470782534 74.9250426024684) bank157971 +157972 POINT(40.87515251299612 74.70782000149543) bank157972 +157973 POINT(41.08336612409954 74.89853566304645) bank157973 +157974 POINT(40.16439053723102 74.40476332631158) bank157974 +157975 POINT(40.19032896918312 73.29442119257581) bank157975 +157976 POINT(40.488289317329105 74.18561905943396) bank157976 +157977 POINT(40.299091771374535 73.46992823437824) bank157977 +157978 POINT(39.914696516389064 74.36030061846735) bank157978 +157979 POINT(41.27705895033103 73.23043308340567) bank157979 +157980 POINT(40.88650151086966 74.4603044740657) bank157980 +157981 POINT(39.98697627399162 74.65861993259097) bank157981 +157982 POINT(40.22188716873511 74.47865285800017) bank157982 +157983 POINT(40.57778248384498 73.19521162780498) bank157983 +157984 POINT(41.55485805836666 74.93439075717376) bank157984 +157985 POINT(40.06708980897708 73.60785310638603) bank157985 +157986 POINT(41.26452669869475 74.76404280057696) bank157986 +157987 POINT(41.321583898432074 73.28612627960028) bank157987 +157988 POINT(41.415000131786044 73.28245992183325) bank157988 +157989 POINT(40.24435992767752 74.59450064319357) bank157989 +157990 POINT(40.01745215330652 74.47496849837812) bank157990 +157991 POINT(40.30442444426383 74.01178226177585) bank157991 +157992 POINT(40.87095787545419 74.05970172363405) bank157992 +157993 POINT(41.06848757013703 74.00261568974422) bank157993 +157994 POINT(40.570525305478014 74.07922259658869) bank157994 +157995 POINT(40.06316168381924 73.83099516127709) bank157995 +157996 POINT(40.89712321218079 74.7976983090777) bank157996 +157997 POINT(41.02382855698719 74.82437409923448) bank157997 +157998 POINT(41.28697961002024 74.36998789649704) bank157998 +157999 POINT(40.549208311722026 73.08134165524308) bank157999 +158000 POINT(40.309542790110996 73.7380022278605) bank158000 +158001 POINT(39.72120766471899 74.88667688575566) bank158001 +158002 POINT(39.90070419091963 73.42912672975017) bank158002 +158003 POINT(41.28267007297471 74.06319602855399) bank158003 +158004 POINT(40.99134594801533 74.93363623518795) bank158004 +158005 POINT(40.76888363953444 73.63357213706112) bank158005 +158006 POINT(40.04440653266885 74.60492475751568) bank158006 +158007 POINT(40.11139663298978 73.69398110085874) bank158007 +158008 POINT(41.25053112978248 74.53721677150935) bank158008 +158009 POINT(41.606784408219774 74.45759318411974) bank158009 +158010 POINT(41.01551497099606 73.26826521610019) bank158010 +158011 POINT(40.7580336027644 73.78121070256107) bank158011 +158012 POINT(41.085436411032774 74.3467515548371) bank158012 +158013 POINT(39.92351000984847 74.33424366670572) bank158013 +158014 POINT(41.5192333896515 73.31524205747411) bank158014 +158015 POINT(40.11089441362304 74.32654689933526) bank158015 +158016 POINT(41.58240864775412 73.25159157194254) bank158016 +158017 POINT(41.66266803543605 74.54210467545286) bank158017 +158018 POINT(40.925295911992556 74.9855554563432) bank158018 +158019 POINT(40.117594287505575 73.86805893011329) bank158019 +158020 POINT(41.03970643181812 74.22347645563647) bank158020 +158021 POINT(39.96481634048721 73.83110251679501) bank158021 +158022 POINT(40.824372795099016 73.47003484273594) bank158022 +158023 POINT(40.936867576314256 74.7791464705979) bank158023 +158024 POINT(39.74829544710359 73.84504214055903) bank158024 +158025 POINT(41.530566422648484 74.80225938651431) bank158025 +158026 POINT(39.73717491714708 73.9225805387317) bank158026 +158027 POINT(39.85540437744492 74.61754602972562) bank158027 +158028 POINT(41.368554422137635 74.29759050071932) bank158028 +158029 POINT(40.755526897859085 74.98946094716491) bank158029 +158030 POINT(41.130558964354655 74.28987475105232) bank158030 +158031 POINT(40.1304905632982 74.58422884847006) bank158031 +158032 POINT(41.25415969742122 74.27496208942274) bank158032 +158033 POINT(40.12934578959569 73.01133665269026) bank158033 +158034 POINT(41.592596001427715 73.73443875602808) bank158034 +158035 POINT(40.87936898535517 73.14370182712199) bank158035 +158036 POINT(41.00696696139104 74.04562028168381) bank158036 +158037 POINT(40.448430345693076 74.71319924260062) bank158037 +158038 POINT(40.75177680290276 74.55697991522706) bank158038 +158039 POINT(40.70449162913355 74.35320434778782) bank158039 +158040 POINT(40.1421992721578 73.81429370836315) bank158040 +158041 POINT(40.15400601428921 73.82546178092868) bank158041 +158042 POINT(40.29514064039128 74.48845350731095) bank158042 +158043 POINT(39.86827946656624 73.56733580314179) bank158043 +158044 POINT(41.71214491447091 73.33044607624436) bank158044 +158045 POINT(40.52062182900837 73.92147245967824) bank158045 +158046 POINT(40.0359113626316 73.69796726990386) bank158046 +158047 POINT(40.8416061487944 74.9428988739782) bank158047 +158048 POINT(41.22321746032608 73.78492542280577) bank158048 +158049 POINT(40.60991793159268 73.05647068678813) bank158049 +158050 POINT(41.53157845642589 74.30264508759164) bank158050 +158051 POINT(41.531615191401535 73.55941455424495) bank158051 +158052 POINT(40.71898012250131 73.55750158899295) bank158052 +158053 POINT(41.180570072797714 74.6716927485789) bank158053 +158054 POINT(40.83660295347011 73.53265349085618) bank158054 +158055 POINT(40.73277663033387 73.46799735017136) bank158055 +158056 POINT(39.74799238962594 73.7590031132469) bank158056 +158057 POINT(40.944057480409455 73.98112608104887) bank158057 +158058 POINT(40.22800146240993 73.7219255905651) bank158058 +158059 POINT(39.92469755246245 74.07629416512107) bank158059 +158060 POINT(40.16580822519436 74.89410776739317) bank158060 +158061 POINT(41.12137142117818 73.03330320408323) bank158061 +158062 POINT(41.5147852406201 74.4969505670606) bank158062 +158063 POINT(41.244033413116384 73.44243953139845) bank158063 +158064 POINT(40.88776225913109 74.29905621476878) bank158064 +158065 POINT(40.01877072076815 73.00834682333817) bank158065 +158066 POINT(40.76360283765946 73.55395724630762) bank158066 +158067 POINT(40.61625160225432 74.3909865064815) bank158067 +158068 POINT(40.81091670318227 73.24907906309524) bank158068 +158069 POINT(40.16213169346718 73.71363464904009) bank158069 +158070 POINT(40.752560606293585 74.88932458456539) bank158070 +158071 POINT(40.38490700982271 73.5547429900515) bank158071 +158072 POINT(39.808965222771995 73.53761673364218) bank158072 +158073 POINT(41.14998671547605 74.89617280510996) bank158073 +158074 POINT(39.72866743583765 73.47860193863747) bank158074 +158075 POINT(40.47614642830712 73.54114357896111) bank158075 +158076 POINT(39.85125436516388 73.58291475245228) bank158076 +158077 POINT(41.52037359110812 73.91327023113337) bank158077 +158078 POINT(40.04115753819889 74.62175927626465) bank158078 +158079 POINT(39.864824527464485 73.0106816167006) bank158079 +158080 POINT(41.50396954277459 73.42022456752568) bank158080 +158081 POINT(39.95156842382836 73.47690196698623) bank158081 +158082 POINT(40.49877009109377 74.75612931647731) bank158082 +158083 POINT(40.316336413012586 74.55689525444272) bank158083 +158084 POINT(41.51339195361038 73.37505274301238) bank158084 +158085 POINT(40.14775648064746 74.382096352009) bank158085 +158086 POINT(40.729640167728355 74.68525012331509) bank158086 +158087 POINT(40.26355929292148 74.89441625351847) bank158087 +158088 POINT(41.594839723749715 73.0175868224155) bank158088 +158089 POINT(40.24845952111487 74.82524451569367) bank158089 +158090 POINT(40.968375008757775 74.71859345594365) bank158090 +158091 POINT(40.41923559401929 73.0387032492958) bank158091 +158092 POINT(40.59125446461619 73.9023167184782) bank158092 +158093 POINT(40.75520340673869 74.34819253872328) bank158093 +158094 POINT(41.64884750315595 73.08588812803339) bank158094 +158095 POINT(41.25648186266545 74.81281731807351) bank158095 +158096 POINT(41.36275604304219 73.63826767114038) bank158096 +158097 POINT(41.54118507900973 73.90200355928425) bank158097 +158098 POINT(41.392775582304516 73.45984072418474) bank158098 +158099 POINT(40.667132658643204 73.34997670339597) bank158099 +158100 POINT(40.14957360047949 74.67480157183637) bank158100 +158101 POINT(40.082781061959885 74.8986097086907) bank158101 +158102 POINT(41.11252993618337 74.52060803260122) bank158102 +158103 POINT(41.16721740415745 74.01386939592302) bank158103 +158104 POINT(41.009145281046656 74.44347711896161) bank158104 +158105 POINT(40.49701843746558 74.23630519189871) bank158105 +158106 POINT(40.86826682970532 73.06021860477252) bank158106 +158107 POINT(40.102304392536254 74.61912110544681) bank158107 +158108 POINT(39.77225971057323 73.02864989231887) bank158108 +158109 POINT(40.891176484341 74.65656683443402) bank158109 +158110 POINT(41.275367691578474 73.18910094520479) bank158110 +158111 POINT(40.939295565633984 74.87230805310514) bank158111 +158112 POINT(40.07924561713231 74.73048842925714) bank158112 +158113 POINT(41.0385829352749 73.36644996337024) bank158113 +158114 POINT(41.61496874483054 74.58749670600149) bank158114 +158115 POINT(40.23534436054077 74.64091542271721) bank158115 +158116 POINT(39.950279173593664 74.39442659570518) bank158116 +158117 POINT(40.471947824952785 74.97792418739002) bank158117 +158118 POINT(39.76147277990428 73.70769552067621) bank158118 +158119 POINT(40.11939000012671 73.88212056244052) bank158119 +158120 POINT(40.67618461102536 73.91273609895211) bank158120 +158121 POINT(41.68140964846748 73.60070850540859) bank158121 +158122 POINT(40.16417826532624 73.04889202534956) bank158122 +158123 POINT(40.87348566775874 74.26366320301028) bank158123 +158124 POINT(40.86604823565936 73.90260909357899) bank158124 +158125 POINT(41.38412971986799 74.29131908249236) bank158125 +158126 POINT(39.78551234761642 73.89471515636508) bank158126 +158127 POINT(39.993069572690295 74.12311973766077) bank158127 +158128 POINT(41.54935325064303 74.00778343363623) bank158128 +158129 POINT(40.129060531112586 73.22093404027208) bank158129 +158130 POINT(40.146408970340154 73.60042678579403) bank158130 +158131 POINT(41.26152162217474 73.61522435787583) bank158131 +158132 POINT(40.92489579996769 74.61915473444225) bank158132 +158133 POINT(40.20937490993612 73.53691335401167) bank158133 +158134 POINT(41.22036589737973 74.90704292449311) bank158134 +158135 POINT(40.498011745057894 73.19791258834564) bank158135 +158136 POINT(40.09275650689598 74.48336257985267) bank158136 +158137 POINT(39.874302377570956 73.87665030925761) bank158137 +158138 POINT(41.43726684256571 73.89799038297711) bank158138 +158139 POINT(40.66590608795962 73.34791201485055) bank158139 +158140 POINT(41.22648399205352 73.04278586980526) bank158140 +158141 POINT(41.08247443520421 74.38847514631131) bank158141 +158142 POINT(39.728360392735716 73.0753007079179) bank158142 +158143 POINT(41.054785926572876 74.11240661024655) bank158143 +158144 POINT(39.81745143854201 74.3965351679724) bank158144 +158145 POINT(40.977493492617015 73.54966534652789) bank158145 +158146 POINT(39.766067457201125 74.05962982289097) bank158146 +158147 POINT(41.47213853782284 74.53539600202456) bank158147 +158148 POINT(40.018285860240596 74.20769702890227) bank158148 +158149 POINT(41.16687538877624 74.85388509817544) bank158149 +158150 POINT(41.090529268617544 73.04712798394323) bank158150 +158151 POINT(40.60239474931449 74.35909119566537) bank158151 +158152 POINT(40.864574010332646 74.0232880904114) bank158152 +158153 POINT(40.856012973048706 73.9471009197592) bank158153 +158154 POINT(40.72048962630137 73.27508669940745) bank158154 +158155 POINT(40.71287908522263 73.0592826896248) bank158155 +158156 POINT(40.208227531446795 74.51896666742611) bank158156 +158157 POINT(40.833916641884095 74.41628798360983) bank158157 +158158 POINT(40.58034656081968 73.11143521328086) bank158158 +158159 POINT(40.01447115181728 73.25462291245971) bank158159 +158160 POINT(41.36001568642839 73.29421324052375) bank158160 +158161 POINT(39.78505989251569 73.28151470050508) bank158161 +158162 POINT(41.343322784750754 73.64709685636984) bank158162 +158163 POINT(40.51034959382487 73.75991121489622) bank158163 +158164 POINT(39.8802527019113 74.76644887401157) bank158164 +158165 POINT(40.60028015286802 73.59831492594509) bank158165 +158166 POINT(39.75621965049093 73.39465307212527) bank158166 +158167 POINT(41.695191296855896 74.9709962133121) bank158167 +158168 POINT(40.107026055777325 74.03857298574175) bank158168 +158169 POINT(41.701293239504416 74.23130855871405) bank158169 +158170 POINT(40.30749654059148 73.37387753172068) bank158170 +158171 POINT(40.150799566903416 74.9245965429771) bank158171 +158172 POINT(40.735583835968995 73.3930724197789) bank158172 +158173 POINT(40.289656669291695 74.21098145223048) bank158173 +158174 POINT(39.81945097634631 74.00665336858614) bank158174 +158175 POINT(40.590250874027205 74.10251854919674) bank158175 +158176 POINT(40.567587167104065 73.91458088522256) bank158176 +158177 POINT(39.96507751791131 74.98044572504399) bank158177 +158178 POINT(40.77264264561519 73.22757608630435) bank158178 +158179 POINT(41.34708521817634 74.51244396069403) bank158179 +158180 POINT(41.244038952343814 74.69939742986485) bank158180 +158181 POINT(40.98690271950977 73.96945195232787) bank158181 +158182 POINT(41.31257885728056 74.95764107046531) bank158182 +158183 POINT(40.55268866506666 73.06065495564813) bank158183 +158184 POINT(41.55789127871911 73.5283018375673) bank158184 +158185 POINT(40.26299955349124 74.79805685206101) bank158185 +158186 POINT(41.17454441192934 73.9828044241076) bank158186 +158187 POINT(39.88122102196374 74.99802738023389) bank158187 +158188 POINT(40.86334484779874 74.45797193368321) bank158188 +158189 POINT(40.381573155515454 73.60638565484817) bank158189 +158190 POINT(40.97052800684686 74.92752906489632) bank158190 +158191 POINT(40.83495123946438 74.5526661203008) bank158191 +158192 POINT(40.716319951715704 74.64343153538933) bank158192 +158193 POINT(41.312597035222005 74.30592612089033) bank158193 +158194 POINT(40.89449139534405 74.70363920968941) bank158194 +158195 POINT(39.87638285115365 74.03639705417847) bank158195 +158196 POINT(40.515850191944956 74.92348309731521) bank158196 +158197 POINT(40.02033530328984 74.35910181195133) bank158197 +158198 POINT(41.56424677452709 74.95679063457098) bank158198 +158199 POINT(41.468414006331656 74.92375379532331) bank158199 +158200 POINT(40.28704353669783 74.64752570759957) bank158200 +158201 POINT(40.26371472130631 74.72350103356442) bank158201 +158202 POINT(41.365819349398805 73.03274959731216) bank158202 +158203 POINT(40.99891906802979 73.15801447159872) bank158203 +158204 POINT(40.164649978771656 73.77050322030122) bank158204 +158205 POINT(39.89798403827593 73.71663154094026) bank158205 +158206 POINT(40.5731130915816 74.79326851256047) bank158206 +158207 POINT(39.73419240404077 74.00700576862666) bank158207 +158208 POINT(40.14648468706005 74.36705381215675) bank158208 +158209 POINT(41.306234690019416 74.63590650405044) bank158209 +158210 POINT(41.447622478002806 74.10331751527042) bank158210 +158211 POINT(41.33979637889729 74.56745493597974) bank158211 +158212 POINT(39.90252712394944 74.18149488623703) bank158212 +158213 POINT(40.75737888201686 73.89226126831365) bank158213 +158214 POINT(40.51649354827676 73.2075268531165) bank158214 +158215 POINT(41.3441513486385 74.00715624666593) bank158215 +158216 POINT(39.99219554463012 73.7178276971159) bank158216 +158217 POINT(41.58422029238724 73.40062073740613) bank158217 +158218 POINT(39.862016632647155 74.50984098751366) bank158218 +158219 POINT(40.99904110620047 74.38192700948454) bank158219 +158220 POINT(40.05925429182919 73.65197606006113) bank158220 +158221 POINT(40.064981170307384 73.8452660937308) bank158221 +158222 POINT(40.56633453196084 73.36896186777338) bank158222 +158223 POINT(40.912872811068645 74.61807984411523) bank158223 +158224 POINT(40.98054402517413 73.13127458753956) bank158224 +158225 POINT(41.62494265457979 74.68576141388996) bank158225 +158226 POINT(40.387258742131266 73.7051340131511) bank158226 +158227 POINT(40.99355517293741 74.00915031112007) bank158227 +158228 POINT(41.61318877745156 73.82006673127833) bank158228 +158229 POINT(40.88436556289663 74.69616685050956) bank158229 +158230 POINT(41.421224541493196 74.50146227763184) bank158230 +158231 POINT(41.47483933134011 73.27839906104921) bank158231 +158232 POINT(40.9456069643902 73.33684477049432) bank158232 +158233 POINT(41.295899145398735 73.40357561984433) bank158233 +158234 POINT(40.05939444335152 74.03712557155482) bank158234 +158235 POINT(40.31943270196913 74.26967397523585) bank158235 +158236 POINT(40.62857382893063 74.76784900178029) bank158236 +158237 POINT(40.70045937170631 73.4320269156862) bank158237 +158238 POINT(41.02605637270981 73.0450263152377) bank158238 +158239 POINT(40.81004695774437 73.84088409398412) bank158239 +158240 POINT(40.496709996034426 74.29240594836372) bank158240 +158241 POINT(41.035272443168914 74.26411264655277) bank158241 +158242 POINT(41.16402895828372 73.80634749704232) bank158242 +158243 POINT(40.57655264159111 74.58455300864368) bank158243 +158244 POINT(40.003986597470494 73.9337789894772) bank158244 +158245 POINT(40.63220178633282 74.58128626409814) bank158245 +158246 POINT(41.03067649882535 74.16097548843207) bank158246 +158247 POINT(41.271933767904315 75.00263689223625) bank158247 +158248 POINT(40.110321978090056 73.73971551745936) bank158248 +158249 POINT(40.45231709303326 74.7346746094448) bank158249 +158250 POINT(40.55131008713421 73.29527847399892) bank158250 +158251 POINT(41.45672930630196 74.40640302736233) bank158251 +158252 POINT(41.62916348161666 74.32220214854622) bank158252 +158253 POINT(40.32747893384633 74.26092236083923) bank158253 +158254 POINT(40.68298925066856 73.08340272321664) bank158254 +158255 POINT(40.20397700160487 73.79708584611149) bank158255 +158256 POINT(40.97672091317193 74.29670293814318) bank158256 +158257 POINT(41.4847289699396 74.32512952549489) bank158257 +158258 POINT(41.49915164023562 73.50083100582128) bank158258 +158259 POINT(41.712509155090885 73.96513887747186) bank158259 +158260 POINT(41.05970463539776 73.13393460579852) bank158260 +158261 POINT(40.766135466508004 73.71912569800054) bank158261 +158262 POINT(40.36533500838136 73.26598035398122) bank158262 +158263 POINT(40.278526595820054 73.7778032913866) bank158263 +158264 POINT(41.03860153543304 74.5745086082392) bank158264 +158265 POINT(40.85424048341201 74.05898504488546) bank158265 +158266 POINT(41.41143376624138 74.41778320829516) bank158266 +158267 POINT(41.167697114066634 73.5163257482901) bank158267 +158268 POINT(40.446684603852674 74.88131573749354) bank158268 +158269 POINT(41.47860270139906 73.32523737616224) bank158269 +158270 POINT(40.4745053687642 73.18746613543205) bank158270 +158271 POINT(40.41683061188632 73.28512738957522) bank158271 +158272 POINT(40.30965914165306 74.05349458618255) bank158272 +158273 POINT(40.942738358457014 74.81729660655172) bank158273 +158274 POINT(41.600092424115985 73.99161709208556) bank158274 +158275 POINT(41.385311617693795 74.1056733887657) bank158275 +158276 POINT(40.89816937647279 73.61118961546542) bank158276 +158277 POINT(41.06678263264678 74.91143591117866) bank158277 +158278 POINT(40.11299721882962 73.73358742759521) bank158278 +158279 POINT(40.28672268748489 73.15257274092777) bank158279 +158280 POINT(39.94485206622316 74.4735587969561) bank158280 +158281 POINT(40.31048940094606 74.52295920831592) bank158281 +158282 POINT(40.80287544896454 73.57930336908261) bank158282 +158283 POINT(40.430628729504356 74.40270853460993) bank158283 +158284 POINT(41.3338631360025 74.88008698494656) bank158284 +158285 POINT(41.58193266086215 74.77424850736851) bank158285 +158286 POINT(41.54709894253222 73.61849707511621) bank158286 +158287 POINT(41.07232743767387 74.67995933796969) bank158287 +158288 POINT(40.847126116522226 74.9486806775446) bank158288 +158289 POINT(40.510605828900275 74.11385348689585) bank158289 +158290 POINT(41.00457592141997 73.09954412780364) bank158290 +158291 POINT(40.78491031183218 74.08674241427877) bank158291 +158292 POINT(39.85227081034579 73.7729367180187) bank158292 +158293 POINT(40.496872970961086 73.87157758057538) bank158293 +158294 POINT(41.69969402492849 74.89297794046247) bank158294 +158295 POINT(39.72006136104671 73.47445606452439) bank158295 +158296 POINT(40.86559966466582 74.22253871558368) bank158296 +158297 POINT(41.39983302366648 74.13961577447333) bank158297 +158298 POINT(40.364296307113676 74.2843853056973) bank158298 +158299 POINT(41.157572711258716 73.48113319351472) bank158299 +158300 POINT(40.86066832332345 73.40551650064006) bank158300 +158301 POINT(41.13384227123358 73.06251955395584) bank158301 +158302 POINT(41.04894172008005 74.80160520410982) bank158302 +158303 POINT(40.044713131431806 74.45847944399293) bank158303 +158304 POINT(39.83506613958315 74.53237895482499) bank158304 +158305 POINT(41.37222325207907 74.90179610376204) bank158305 +158306 POINT(40.20530553875108 74.63295424184655) bank158306 +158307 POINT(40.49847572125216 74.97953406003425) bank158307 +158308 POINT(39.86618391907976 73.55025770364323) bank158308 +158309 POINT(40.24100981605633 74.82597870816412) bank158309 +158310 POINT(40.39879534713417 74.46082314966057) bank158310 +158311 POINT(41.53685180438546 74.18684459632945) bank158311 +158312 POINT(40.58471678085513 74.96606904889482) bank158312 +158313 POINT(40.59173168416991 73.78094936810648) bank158313 +158314 POINT(41.514575830064324 73.40779308025415) bank158314 +158315 POINT(40.48266471595585 74.92037310081903) bank158315 +158316 POINT(41.13537254470387 74.39444100310168) bank158316 +158317 POINT(40.95052714147704 74.25343691377257) bank158317 +158318 POINT(41.31368199720123 73.22259960301102) bank158318 +158319 POINT(40.22095773130527 74.68420816701187) bank158319 +158320 POINT(40.656905504594334 73.82133447465952) bank158320 +158321 POINT(40.2330070131102 73.74865047708002) bank158321 +158322 POINT(41.22509808146777 74.37510118999647) bank158322 +158323 POINT(40.8727735376544 74.05917930546278) bank158323 +158324 POINT(40.48304612234037 73.40274824050604) bank158324 +158325 POINT(39.973742853992185 74.95402255025006) bank158325 +158326 POINT(40.84562976976576 73.8289531310818) bank158326 +158327 POINT(41.1270692129552 74.8977808313183) bank158327 +158328 POINT(40.630633293360155 74.12384367807672) bank158328 +158329 POINT(41.35066064684116 74.19907133005647) bank158329 +158330 POINT(40.57230089742232 74.23218827829132) bank158330 +158331 POINT(41.37188754638158 73.04806097686976) bank158331 +158332 POINT(41.658594284233125 74.4985845342629) bank158332 +158333 POINT(39.83251691203157 74.4659202567678) bank158333 +158334 POINT(41.41105589534655 73.54255395233328) bank158334 +158335 POINT(41.148465448469175 74.78108422406864) bank158335 +158336 POINT(41.492725859657575 73.76449890671881) bank158336 +158337 POINT(41.2884328245864 73.27012884714452) bank158337 +158338 POINT(40.79496533797866 74.53849225199988) bank158338 +158339 POINT(40.355500909282 74.88480532144226) bank158339 +158340 POINT(41.21567495664081 73.89767459127454) bank158340 +158341 POINT(40.82144643960903 74.2243683797188) bank158341 +158342 POINT(40.20704859112352 74.15639505267306) bank158342 +158343 POINT(40.60653829353312 73.82368046582485) bank158343 +158344 POINT(41.622096896653574 73.69495514708522) bank158344 +158345 POINT(41.13559567655315 74.09074047502214) bank158345 +158346 POINT(41.24810705768232 74.65904431729189) bank158346 +158347 POINT(39.799668756784314 73.91786936464737) bank158347 +158348 POINT(40.42023732313559 74.61666607405618) bank158348 +158349 POINT(40.337739783023515 73.9180028804811) bank158349 +158350 POINT(40.430094899808246 74.12974885960848) bank158350 +158351 POINT(41.52938008934423 74.11193285676026) bank158351 +158352 POINT(40.220028207399714 73.36479465051869) bank158352 +158353 POINT(40.883523887104495 74.51282360768026) bank158353 +158354 POINT(40.02565371873653 73.82633908713295) bank158354 +158355 POINT(40.266537520021984 73.46778645998072) bank158355 +158356 POINT(39.8476058394939 73.96566420547514) bank158356 +158357 POINT(41.00008499661266 73.03023681963495) bank158357 +158358 POINT(41.118473033832096 74.63082412900201) bank158358 +158359 POINT(41.50288367414407 73.64087332559048) bank158359 +158360 POINT(41.45292975915614 74.133114332631) bank158360 +158361 POINT(40.3226633983076 73.52249201240355) bank158361 +158362 POINT(40.846605504748474 73.3757615373137) bank158362 +158363 POINT(40.861037267060134 74.11698093182437) bank158363 +158364 POINT(40.266622721752064 73.3854266789036) bank158364 +158365 POINT(41.314229143536174 74.51818608124483) bank158365 +158366 POINT(40.22577519695822 74.28700623330828) bank158366 +158367 POINT(40.20995448186009 74.60184298569322) bank158367 +158368 POINT(41.00382718384601 73.38715128520302) bank158368 +158369 POINT(40.99947757548733 74.50193230351769) bank158369 +158370 POINT(40.71858681349152 73.07984398494568) bank158370 +158371 POINT(41.09384764797265 74.09818389873985) bank158371 +158372 POINT(40.95640950304641 74.75134560196011) bank158372 +158373 POINT(41.04902755567349 74.26481363125579) bank158373 +158374 POINT(40.47830711880574 73.61085963161563) bank158374 +158375 POINT(40.04245584653551 73.75832768795486) bank158375 +158376 POINT(40.75730588270623 73.81370071718115) bank158376 +158377 POINT(40.564103691105984 74.47164239843472) bank158377 +158378 POINT(41.529311285072765 74.4854445171835) bank158378 +158379 POINT(40.57624350999001 74.32460549970995) bank158379 +158380 POINT(40.005468320423766 73.49858540903773) bank158380 +158381 POINT(40.9306588554482 74.76279458482226) bank158381 +158382 POINT(40.02396596792252 74.01479347635741) bank158382 +158383 POINT(39.9284400966658 73.21601741792024) bank158383 +158384 POINT(39.81939089306699 73.03191829861785) bank158384 +158385 POINT(40.14589481024169 74.61178616834367) bank158385 +158386 POINT(41.530993017318494 73.58500933956009) bank158386 +158387 POINT(41.20931831830699 74.47877065547634) bank158387 +158388 POINT(41.08023326394926 73.19252566710794) bank158388 +158389 POINT(39.74646530751398 73.64698896443174) bank158389 +158390 POINT(39.78694344508824 74.74474392203908) bank158390 +158391 POINT(39.726790107294285 73.48501736702667) bank158391 +158392 POINT(40.98268787466646 73.2481641377138) bank158392 +158393 POINT(41.32363696916904 74.71521004048385) bank158393 +158394 POINT(41.11993152349438 73.32666114768591) bank158394 +158395 POINT(41.55437668089898 74.89383818617783) bank158395 +158396 POINT(40.83081092760314 74.22063186962812) bank158396 +158397 POINT(41.19643470572127 73.382370902261) bank158397 +158398 POINT(39.934572958364775 73.41442048574395) bank158398 +158399 POINT(41.45400675546281 74.39570040714447) bank158399 +158400 POINT(40.2515159795478 73.20940813122301) bank158400 +158401 POINT(39.99166153425475 73.40192868899024) bank158401 +158402 POINT(41.382693083975184 74.99945135874476) bank158402 +158403 POINT(40.49944790393773 74.08171365947827) bank158403 +158404 POINT(40.17090668888675 74.68868289418337) bank158404 +158405 POINT(41.352982270374 73.5731655697358) bank158405 +158406 POINT(39.71904163038651 73.60694527412048) bank158406 +158407 POINT(39.856737002042074 73.09276211134011) bank158407 +158408 POINT(40.180423305526126 73.75237741463508) bank158408 +158409 POINT(40.12852721852846 74.18120065490903) bank158409 +158410 POINT(41.48477474356969 73.47727263450238) bank158410 +158411 POINT(41.03196156281161 73.68434237481856) bank158411 +158412 POINT(40.2956038271744 73.87891134719533) bank158412 +158413 POINT(39.77560371607342 74.12156270003877) bank158413 +158414 POINT(40.985651845536864 74.95758160778995) bank158414 +158415 POINT(40.68883591212678 73.00912674775827) bank158415 +158416 POINT(39.83760246538735 74.71304759191841) bank158416 +158417 POINT(41.53353872151345 73.2219094564199) bank158417 +158418 POINT(41.19500347067003 73.51552186811371) bank158418 +158419 POINT(40.62722049916305 73.84304092329435) bank158419 +158420 POINT(40.83060616511905 73.9465859746807) bank158420 +158421 POINT(41.43816277490878 74.73437063308066) bank158421 +158422 POINT(39.96397710088285 73.61355436043166) bank158422 +158423 POINT(40.94916116632474 73.76337953521512) bank158423 +158424 POINT(41.69868948366216 74.71610196521421) bank158424 +158425 POINT(41.62755646525629 73.32468515453753) bank158425 +158426 POINT(40.843487115205754 73.52775596988157) bank158426 +158427 POINT(39.90247200914828 73.28365262767642) bank158427 +158428 POINT(40.48688165181386 73.08070222168824) bank158428 +158429 POINT(41.02483233727141 74.37153676881722) bank158429 +158430 POINT(41.37248258762758 74.06007245197944) bank158430 +158431 POINT(40.29278729751204 74.58279721295885) bank158431 +158432 POINT(40.67597839807364 74.26672681705361) bank158432 +158433 POINT(39.99643649799051 74.42370891193625) bank158433 +158434 POINT(40.39748965889795 73.12333217624156) bank158434 +158435 POINT(41.09871347267797 73.3007681387026) bank158435 +158436 POINT(40.763135783101426 74.64616149750208) bank158436 +158437 POINT(40.040022488020206 73.42877110777073) bank158437 +158438 POINT(39.99205859355283 74.82181430743154) bank158438 +158439 POINT(40.29936495752263 73.37320919800938) bank158439 +158440 POINT(40.76571507149951 74.74721993183458) bank158440 +158441 POINT(39.82211002981991 73.18397464147218) bank158441 +158442 POINT(40.96805006428522 74.95510473696889) bank158442 +158443 POINT(39.96893060906093 74.73258188368457) bank158443 +158444 POINT(41.20967991748589 73.70288414190453) bank158444 +158445 POINT(41.634896829887936 74.11147423752347) bank158445 +158446 POINT(41.49236689248684 74.30263707624687) bank158446 +158447 POINT(40.32791732729032 74.75936443807826) bank158447 +158448 POINT(41.47044870159503 73.64803709346103) bank158448 +158449 POINT(41.336632425083806 73.19322481244792) bank158449 +158450 POINT(40.944837853660054 73.81090769566174) bank158450 +158451 POINT(40.88831868620497 73.99654869112376) bank158451 +158452 POINT(41.01113774048226 73.18754908114839) bank158452 +158453 POINT(41.146487715842746 74.62820256701374) bank158453 +158454 POINT(40.92730657610246 73.67705173035809) bank158454 +158455 POINT(40.72100271944215 73.55361782458012) bank158455 +158456 POINT(40.950451984456464 74.21921584197031) bank158456 +158457 POINT(39.72509840404383 73.84852031829473) bank158457 +158458 POINT(41.5143984805426 74.23838142822136) bank158458 +158459 POINT(41.289547327188096 74.31843814406535) bank158459 +158460 POINT(39.92616684159319 74.58915886824406) bank158460 +158461 POINT(40.14864378580896 74.37963196674534) bank158461 +158462 POINT(40.544131076638266 73.37224916446131) bank158462 +158463 POINT(41.624396246000636 73.40785065883222) bank158463 +158464 POINT(40.08752272111121 74.88188544890704) bank158464 +158465 POINT(41.20196033971487 74.01917060331544) bank158465 +158466 POINT(41.30316571162383 74.14547138994398) bank158466 +158467 POINT(41.3292851737717 74.73507138260887) bank158467 +158468 POINT(40.02424585443396 73.4756315908219) bank158468 +158469 POINT(40.402302234725646 74.9798331229765) bank158469 +158470 POINT(40.50582095114258 73.15320626857567) bank158470 +158471 POINT(41.27921179174301 73.57211112311614) bank158471 +158472 POINT(40.1451817765339 73.68000813465808) bank158472 +158473 POINT(41.044127586898725 74.10853032248393) bank158473 +158474 POINT(40.13974631115078 74.7636267198093) bank158474 +158475 POINT(41.393711522926765 74.73437132795176) bank158475 +158476 POINT(39.72435159961336 74.2060220584081) bank158476 +158477 POINT(41.482496193747075 73.64891471765853) bank158477 +158478 POINT(40.08995190549936 74.83550859541607) bank158478 +158479 POINT(40.17575252208021 74.8106262228366) bank158479 +158480 POINT(40.816585631806205 74.78178089544424) bank158480 +158481 POINT(41.16632038633756 74.30104848833764) bank158481 +158482 POINT(41.59448712929806 74.51202115649014) bank158482 +158483 POINT(40.380590275744304 73.32472862479551) bank158483 +158484 POINT(41.56729690540661 74.94442995595438) bank158484 +158485 POINT(41.696384705450065 73.52341954943394) bank158485 +158486 POINT(41.37651036019388 74.32201227438136) bank158486 +158487 POINT(40.06726283655415 73.36421769842359) bank158487 +158488 POINT(41.20652554272197 73.42200939341181) bank158488 +158489 POINT(41.521266518868295 74.48498017032075) bank158489 +158490 POINT(41.68310456408656 73.5010665555046) bank158490 +158491 POINT(41.14958202519027 73.38024173120601) bank158491 +158492 POINT(40.164479218199745 74.02194145131509) bank158492 +158493 POINT(40.65407319115711 74.32188737372468) bank158493 +158494 POINT(40.45548389359449 73.11624462364955) bank158494 +158495 POINT(40.00267836710839 74.15099231281201) bank158495 +158496 POINT(41.5099210861357 74.13686867419096) bank158496 +158497 POINT(40.67824747232264 73.54066068632524) bank158497 +158498 POINT(39.894455461905295 73.54537069551921) bank158498 +158499 POINT(40.282035942888854 74.11825342872832) bank158499 +158500 POINT(40.38043930293687 74.14161002090569) bank158500 +158501 POINT(40.242975595236175 74.65952667130018) bank158501 +158502 POINT(41.55180060192912 73.93384439434286) bank158502 +158503 POINT(41.594837514536415 74.36531895039231) bank158503 +158504 POINT(40.317729590209986 74.00081824346684) bank158504 +158505 POINT(41.188767126020245 74.5082082025505) bank158505 +158506 POINT(41.403124595535694 73.95956251835172) bank158506 +158507 POINT(41.23583040291207 74.3040157304448) bank158507 +158508 POINT(40.59199576432276 73.50530256163587) bank158508 +158509 POINT(40.25744897105139 73.66501990627607) bank158509 +158510 POINT(41.2205887885312 74.21858926279283) bank158510 +158511 POINT(40.29329835330291 74.9391896094627) bank158511 +158512 POINT(41.50985131504661 74.54040472142836) bank158512 +158513 POINT(40.01274314673992 74.26764433874153) bank158513 +158514 POINT(39.864751748233374 73.11723879499978) bank158514 +158515 POINT(40.60466127438118 74.00468739149088) bank158515 +158516 POINT(40.95877060442072 74.13642603332747) bank158516 +158517 POINT(41.03792843268652 74.54043110335785) bank158517 +158518 POINT(40.84826409204476 73.49085479538756) bank158518 +158519 POINT(39.84365969371632 74.81149132518195) bank158519 +158520 POINT(41.087904053383625 73.12300894956853) bank158520 +158521 POINT(40.80902472594549 73.31766776791925) bank158521 +158522 POINT(40.066656179177 74.97023019882077) bank158522 +158523 POINT(40.47792054074795 73.73688876061678) bank158523 +158524 POINT(40.441058576008345 74.40678617324306) bank158524 +158525 POINT(40.998095314638384 73.37521236331882) bank158525 +158526 POINT(39.94520323459132 74.3462036875611) bank158526 +158527 POINT(41.04898838051096 73.47573156771219) bank158527 +158528 POINT(41.15746125806332 73.6422183642514) bank158528 +158529 POINT(39.86901267529205 73.66032711052367) bank158529 +158530 POINT(39.901513515244 73.42212087479496) bank158530 +158531 POINT(40.094954482375655 73.6883661825439) bank158531 +158532 POINT(41.08951534909001 73.09533933974419) bank158532 +158533 POINT(40.87213408722435 73.92075695959856) bank158533 +158534 POINT(41.488300020445735 73.70310195034821) bank158534 +158535 POINT(40.80452355310385 73.88848728364712) bank158535 +158536 POINT(41.391495786656655 73.29844195283091) bank158536 +158537 POINT(39.8094804751095 74.42706217518453) bank158537 +158538 POINT(39.78912249889563 74.9073769454559) bank158538 +158539 POINT(40.61318822731282 74.45999897620254) bank158539 +158540 POINT(40.64323169407458 73.06421486539081) bank158540 +158541 POINT(40.04436046564595 74.24340670888749) bank158541 +158542 POINT(41.20181785706228 73.86618874359554) bank158542 +158543 POINT(40.644698669933625 74.66700624575462) bank158543 +158544 POINT(41.448618365266654 73.63600992172168) bank158544 +158545 POINT(40.7202547149671 73.7060272503451) bank158545 +158546 POINT(41.184632964245615 73.38077834458319) bank158546 +158547 POINT(39.99153265171639 73.50146526762738) bank158547 +158548 POINT(40.83472087566475 74.51372632616228) bank158548 +158549 POINT(40.53308630915845 74.57708354249604) bank158549 +158550 POINT(41.54817929520336 74.18927730570921) bank158550 +158551 POINT(41.08895795922666 74.95292595114479) bank158551 +158552 POINT(40.9295583265977 74.64452810870712) bank158552 +158553 POINT(40.622141835535444 73.10087695300145) bank158553 +158554 POINT(40.80641727497132 73.21234754976118) bank158554 +158555 POINT(41.530708019145635 74.04274022776862) bank158555 +158556 POINT(41.45923786853416 73.45252152325742) bank158556 +158557 POINT(41.22913604462917 74.39944066019682) bank158557 +158558 POINT(41.013635143645786 74.01718110080695) bank158558 +158559 POINT(40.609456216220934 73.04597590240408) bank158559 +158560 POINT(41.52483869673411 73.26895316864272) bank158560 +158561 POINT(41.396765996709256 74.48710124488194) bank158561 +158562 POINT(40.85812291448976 73.83384395360443) bank158562 +158563 POINT(41.49805841175935 73.5603478432128) bank158563 +158564 POINT(39.72025310904518 74.42384698785992) bank158564 +158565 POINT(41.21996205224428 73.55957511414883) bank158565 +158566 POINT(41.41053097557601 74.28901234143588) bank158566 +158567 POINT(40.69972189202134 73.9957396072086) bank158567 +158568 POINT(41.0322986001561 74.5077063207694) bank158568 +158569 POINT(41.14193700260541 73.25160613243914) bank158569 +158570 POINT(41.44292008468923 73.70381443081293) bank158570 +158571 POINT(40.960321241720436 73.30392459641675) bank158571 +158572 POINT(40.002061913788964 73.78836399791004) bank158572 +158573 POINT(40.266134078141896 73.3324942023389) bank158573 +158574 POINT(40.000980895890144 74.67243342864447) bank158574 +158575 POINT(40.399172021020725 73.85059383042555) bank158575 +158576 POINT(41.149105644484834 74.95285715565996) bank158576 +158577 POINT(39.96046105010933 73.90080715413661) bank158577 +158578 POINT(40.66384095675953 74.45442029982003) bank158578 +158579 POINT(40.89362490142477 74.72044603641633) bank158579 +158580 POINT(39.97769615670248 73.70886340099706) bank158580 +158581 POINT(40.28067042533614 73.28148273178186) bank158581 +158582 POINT(40.91190576341118 74.31042413024659) bank158582 +158583 POINT(41.63103148316631 73.92249675285044) bank158583 +158584 POINT(41.51832931390171 74.92587171248923) bank158584 +158585 POINT(41.0666306150526 74.61530759162551) bank158585 +158586 POINT(40.59613199951777 74.9275246912857) bank158586 +158587 POINT(41.12920093672909 74.86009849296657) bank158587 +158588 POINT(40.76132449772834 74.97885574998047) bank158588 +158589 POINT(40.988703352295786 73.58776389352413) bank158589 +158590 POINT(39.85372350088419 74.2974036022197) bank158590 +158591 POINT(41.521502536714564 74.56505370253731) bank158591 +158592 POINT(40.78210411434482 74.08226915197307) bank158592 +158593 POINT(40.59328444300244 73.78817333046486) bank158593 +158594 POINT(40.07003871243975 73.39071641971114) bank158594 +158595 POINT(41.19239575556145 74.17090051167352) bank158595 +158596 POINT(40.846371561345094 73.13097310717066) bank158596 +158597 POINT(39.96047275600843 74.88330917117763) bank158597 +158598 POINT(40.491122418667594 74.14990348561888) bank158598 +158599 POINT(40.802139382965706 74.36455309763586) bank158599 +158600 POINT(41.22227239376116 74.25038785148406) bank158600 +158601 POINT(40.592973794721345 74.0485523182975) bank158601 +158602 POINT(40.297983526763126 74.73281385800017) bank158602 +158603 POINT(41.60891875005348 73.50623942107906) bank158603 +158604 POINT(40.05564718743834 73.89954650595256) bank158604 +158605 POINT(40.926653441699344 73.42080112013396) bank158605 +158606 POINT(40.082382963271364 74.22531447794374) bank158606 +158607 POINT(40.545481608320166 74.68324054617467) bank158607 +158608 POINT(41.41546528868869 73.38383933106643) bank158608 +158609 POINT(40.88179444885083 74.48049796904233) bank158609 +158610 POINT(40.624434170924474 73.12837554880913) bank158610 +158611 POINT(40.41719954051757 73.67823091446122) bank158611 +158612 POINT(41.079162799402056 74.1647881778914) bank158612 +158613 POINT(40.59706002684824 74.8298660105323) bank158613 +158614 POINT(40.410610210221115 73.50372262432724) bank158614 +158615 POINT(41.68546285545459 73.06866467167104) bank158615 +158616 POINT(41.05425019228707 74.30390333599695) bank158616 +158617 POINT(41.05696601956438 73.80418820669117) bank158617 +158618 POINT(41.27582944621424 73.00643812012507) bank158618 +158619 POINT(40.089649474455776 73.22722885631497) bank158619 +158620 POINT(40.02305921597305 74.30677268935128) bank158620 +158621 POINT(39.92902776724748 74.89496038985355) bank158621 +158622 POINT(40.918464304058595 73.45713840656539) bank158622 +158623 POINT(40.98600891154655 74.26790963389062) bank158623 +158624 POINT(40.03836514862619 74.95225256116342) bank158624 +158625 POINT(41.335370870245626 73.4111513799482) bank158625 +158626 POINT(40.042346769450944 74.52718075942683) bank158626 +158627 POINT(39.869469541489416 74.00817893493455) bank158627 +158628 POINT(40.42413716726262 73.07605259402959) bank158628 +158629 POINT(41.45939187133673 74.85432990126856) bank158629 +158630 POINT(39.86205689356228 74.72819091666219) bank158630 +158631 POINT(40.168632619622656 74.5337900815986) bank158631 +158632 POINT(40.29855751073122 73.31902023996888) bank158632 +158633 POINT(40.3359990367838 73.71750623606589) bank158633 +158634 POINT(40.42029835269778 73.06784053855961) bank158634 +158635 POINT(41.283956013741715 74.74026872224663) bank158635 +158636 POINT(41.012844436225286 74.03805548733597) bank158636 +158637 POINT(40.50027299222575 74.24213264974593) bank158637 +158638 POINT(40.26122163124427 74.10666943457059) bank158638 +158639 POINT(39.894880990811956 73.36150276248969) bank158639 +158640 POINT(39.770258233441936 73.73686640265664) bank158640 +158641 POINT(40.2099185018813 74.29326843578379) bank158641 +158642 POINT(39.74817474050383 74.98775966149039) bank158642 +158643 POINT(40.254113098536486 74.03638353853934) bank158643 +158644 POINT(39.794119630311364 74.37782554852905) bank158644 +158645 POINT(40.61046457630767 73.44460078999141) bank158645 +158646 POINT(41.69246739949654 74.26446679618412) bank158646 +158647 POINT(39.82688782433773 73.47691884720241) bank158647 +158648 POINT(39.994597728023514 74.06932422325238) bank158648 +158649 POINT(40.963266275223745 73.29980696689101) bank158649 +158650 POINT(39.73637512090276 74.50213098593508) bank158650 +158651 POINT(40.77199483412585 74.252499441673) bank158651 +158652 POINT(41.591304806336716 74.9790243917881) bank158652 +158653 POINT(40.49250899167357 74.96272481621457) bank158653 +158654 POINT(41.56837498882499 73.60773076392105) bank158654 +158655 POINT(40.638736682725515 74.5031725700894) bank158655 +158656 POINT(40.75093168298176 74.6235628214279) bank158656 +158657 POINT(39.731098541490155 74.13581405184566) bank158657 +158658 POINT(41.15777451575713 73.2643938248451) bank158658 +158659 POINT(41.32936353550048 74.68056264980339) bank158659 +158660 POINT(40.46232523877677 74.24214990492653) bank158660 +158661 POINT(41.40477477870494 73.9427119461279) bank158661 +158662 POINT(40.14905801746088 73.9403217002425) bank158662 +158663 POINT(40.451749132060925 73.46405099384708) bank158663 +158664 POINT(40.1958156418751 74.2079717915161) bank158664 +158665 POINT(40.54465032440641 74.66391454558195) bank158665 +158666 POINT(41.37342693130421 73.03174762561376) bank158666 +158667 POINT(39.9303617296097 73.41033551527804) bank158667 +158668 POINT(39.925395478981024 73.60867348657371) bank158668 +158669 POINT(40.75738306246319 74.72746191897893) bank158669 +158670 POINT(40.712200395077936 74.42246084784492) bank158670 +158671 POINT(41.29203511286594 73.09444067265831) bank158671 +158672 POINT(40.81996643559013 73.90269031735268) bank158672 +158673 POINT(41.079105428476474 73.12262015228839) bank158673 +158674 POINT(40.10220822521143 74.00172756103854) bank158674 +158675 POINT(40.30724171644466 73.62127000562937) bank158675 +158676 POINT(40.351129702593475 73.7826820389183) bank158676 +158677 POINT(41.438506581185855 73.6945520423237) bank158677 +158678 POINT(40.49916674872085 74.35986778637906) bank158678 +158679 POINT(41.6186310723958 74.79032878726238) bank158679 +158680 POINT(41.53586862033077 73.97965029133942) bank158680 +158681 POINT(41.38198379200434 73.46030839760778) bank158681 +158682 POINT(40.01522754201949 74.44106499450548) bank158682 +158683 POINT(40.83483490428852 74.20972535261481) bank158683 +158684 POINT(41.502840128273256 73.56692343237069) bank158684 +158685 POINT(39.994914058713796 74.10892453013702) bank158685 +158686 POINT(40.372102380795894 74.00611662377507) bank158686 +158687 POINT(41.157888375554954 74.20191147380201) bank158687 +158688 POINT(39.965317249385606 74.74773883516225) bank158688 +158689 POINT(41.20835558515301 73.08178664589022) bank158689 +158690 POINT(39.818435216941666 74.77617140890045) bank158690 +158691 POINT(40.063647650096605 74.64754200500496) bank158691 +158692 POINT(40.035733776410474 73.77891265544307) bank158692 +158693 POINT(41.51031017602713 73.22207522778388) bank158693 +158694 POINT(40.281073932431745 74.1384333025602) bank158694 +158695 POINT(40.52152914806053 74.21752486033866) bank158695 +158696 POINT(41.50930190417325 73.92361909342549) bank158696 +158697 POINT(39.8420832595471 73.99795448977538) bank158697 +158698 POINT(40.76455629367477 74.2402262772551) bank158698 +158699 POINT(41.10012124306485 74.21963710612671) bank158699 +158700 POINT(41.16128843558782 73.80621019025084) bank158700 +158701 POINT(40.34978675357551 74.72006208228706) bank158701 +158702 POINT(40.40016221828565 74.81413963592126) bank158702 +158703 POINT(40.814500677839945 73.88843547017298) bank158703 +158704 POINT(39.920337909194856 73.03944782793548) bank158704 +158705 POINT(40.58915532361405 74.84491666312323) bank158705 +158706 POINT(41.37525094590648 73.4077130433395) bank158706 +158707 POINT(40.30553154306069 73.59468784870512) bank158707 +158708 POINT(40.73507253269599 73.11025995541254) bank158708 +158709 POINT(39.84001902420536 73.34816887876724) bank158709 +158710 POINT(40.073328284502345 74.17890458987792) bank158710 +158711 POINT(39.81650372847831 74.56483071294056) bank158711 +158712 POINT(41.364441211332554 73.65458037567336) bank158712 +158713 POINT(40.205789761673664 74.39756985626802) bank158713 +158714 POINT(40.68729274212792 73.07358103528375) bank158714 +158715 POINT(39.92713342725407 74.90741750204401) bank158715 +158716 POINT(41.059891280507976 74.80643864420126) bank158716 +158717 POINT(40.64380439890746 73.08706538371848) bank158717 +158718 POINT(41.44352017324458 73.51598953518348) bank158718 +158719 POINT(39.78420802599683 74.22160921733283) bank158719 +158720 POINT(40.458657238795034 74.57242491385801) bank158720 +158721 POINT(40.83181665419725 74.58925603336222) bank158721 +158722 POINT(39.94898789514476 73.09710917081) bank158722 +158723 POINT(39.94800821942141 74.38665495438548) bank158723 +158724 POINT(39.71384357979531 74.37132001478918) bank158724 +158725 POINT(39.92245060430411 74.90582098320424) bank158725 +158726 POINT(40.485314772116546 74.54751586820224) bank158726 +158727 POINT(41.081618337408415 74.29521091447009) bank158727 +158728 POINT(40.08178427442631 74.7285266273528) bank158728 +158729 POINT(40.67403527012231 73.59716341400049) bank158729 +158730 POINT(40.472741730622865 74.49291597890782) bank158730 +158731 POINT(41.35294870997529 74.9044712772478) bank158731 +158732 POINT(40.73910538520228 75.00037273849875) bank158732 +158733 POINT(41.66934580778681 74.50400327592777) bank158733 +158734 POINT(39.778293743425884 74.19378869256855) bank158734 +158735 POINT(41.18458982099706 74.615795186717) bank158735 +158736 POINT(40.20768318746056 74.56999485642844) bank158736 +158737 POINT(40.54218684233564 74.39730070225018) bank158737 +158738 POINT(39.899403493704526 74.72341452421507) bank158738 +158739 POINT(40.32014830580027 73.96860325618731) bank158739 +158740 POINT(40.98014696980968 73.48592240105525) bank158740 +158741 POINT(40.85986537022157 73.9546238856999) bank158741 +158742 POINT(41.09280313087782 74.98014494258761) bank158742 +158743 POINT(39.73541246980612 74.39978293444749) bank158743 +158744 POINT(40.0894930398326 73.21307661703621) bank158744 +158745 POINT(41.54199151188306 73.85223374344068) bank158745 +158746 POINT(39.759061466405306 73.61297227890861) bank158746 +158747 POINT(40.497766826466396 73.30354057574147) bank158747 +158748 POINT(39.95302802995056 73.34024400927429) bank158748 +158749 POINT(40.47708702166139 73.57971040523046) bank158749 +158750 POINT(41.41586649720809 73.06662638912344) bank158750 +158751 POINT(40.229022840585436 74.17674273639591) bank158751 +158752 POINT(39.83709373333245 74.27882586420797) bank158752 +158753 POINT(41.42221964823912 74.3542865027777) bank158753 +158754 POINT(40.806305060030944 74.42212252947178) bank158754 +158755 POINT(40.4479216551916 73.48962313640648) bank158755 +158756 POINT(40.62660200462246 73.60214972546065) bank158756 +158757 POINT(41.423694941231574 73.20017950841667) bank158757 +158758 POINT(41.13715249630101 74.62202301767277) bank158758 +158759 POINT(40.46990351186578 73.9418408322847) bank158759 +158760 POINT(41.3126731004538 74.66043371213465) bank158760 +158761 POINT(40.886506949205376 73.40701356466836) bank158761 +158762 POINT(39.945502851900365 74.05395430119594) bank158762 +158763 POINT(41.61360501733776 73.4444901509464) bank158763 +158764 POINT(40.22148280256275 74.06272929201336) bank158764 +158765 POINT(40.35110210599228 73.48592761227472) bank158765 +158766 POINT(40.444730449260895 74.48914683822115) bank158766 +158767 POINT(40.72572349084308 73.87912517215187) bank158767 +158768 POINT(39.92384294425541 73.63305911431128) bank158768 +158769 POINT(40.549336247785455 74.65211236499617) bank158769 +158770 POINT(40.85903655415892 73.11546298832623) bank158770 +158771 POINT(41.71169585384504 74.20771604778754) bank158771 +158772 POINT(40.580206646074636 74.7710422360162) bank158772 +158773 POINT(40.677269199576195 73.61074886541104) bank158773 +158774 POINT(39.98498667833491 73.28416953234249) bank158774 +158775 POINT(40.78131707107202 73.14386913090759) bank158775 +158776 POINT(41.08825484058326 73.08554646879183) bank158776 +158777 POINT(40.12554465965196 74.78182135581928) bank158777 +158778 POINT(40.5929536093276 74.43784974843555) bank158778 +158779 POINT(40.652280105323804 74.84533704644906) bank158779 +158780 POINT(40.679049162106935 74.83273070069919) bank158780 +158781 POINT(40.93568561193544 73.92574496793193) bank158781 +158782 POINT(41.063797046976525 73.559255097423) bank158782 +158783 POINT(41.25780697941394 74.83567306988806) bank158783 +158784 POINT(39.92079512537001 73.5511039527644) bank158784 +158785 POINT(40.76992156593119 73.94849113567119) bank158785 +158786 POINT(39.97902240077131 73.29279356962634) bank158786 +158787 POINT(41.118974019113956 74.15102555933211) bank158787 +158788 POINT(41.65766073216905 74.62444765209526) bank158788 +158789 POINT(40.774273759176815 73.41811524407062) bank158789 +158790 POINT(40.128557104253375 74.0276485017669) bank158790 +158791 POINT(40.04381061679293 74.31403639117755) bank158791 +158792 POINT(40.73725705044154 74.10819443735492) bank158792 +158793 POINT(40.72646342222772 74.68250005658689) bank158793 +158794 POINT(40.951146225875604 73.19098125429056) bank158794 +158795 POINT(40.40670911152697 74.55396915682832) bank158795 +158796 POINT(40.72578957706995 73.38042685270551) bank158796 +158797 POINT(40.393807311971315 74.22452167118873) bank158797 +158798 POINT(41.64723937334885 74.1470964173922) bank158798 +158799 POINT(40.79280706079059 73.74270227297077) bank158799 +158800 POINT(40.61782261613942 74.34028990802071) bank158800 +158801 POINT(40.51126996057668 74.38499603555064) bank158801 +158802 POINT(40.85708084054067 74.8110954016386) bank158802 +158803 POINT(40.11178658271359 74.13702594940264) bank158803 +158804 POINT(41.65334204319301 74.38923992495064) bank158804 +158805 POINT(41.600509015938705 74.95973634339846) bank158805 +158806 POINT(39.81899695653656 74.38986353236216) bank158806 +158807 POINT(41.447362742912404 74.83325640399556) bank158807 +158808 POINT(40.56268290482093 73.58356392396016) bank158808 +158809 POINT(40.34079698718652 74.39459840056735) bank158809 +158810 POINT(41.4523085517388 73.23056104296077) bank158810 +158811 POINT(41.33453228422785 74.06993841197796) bank158811 +158812 POINT(39.89062530108667 74.33340048520581) bank158812 +158813 POINT(39.89101232069083 73.39981990765305) bank158813 +158814 POINT(39.90829977168172 73.39933247782086) bank158814 +158815 POINT(40.981839843663046 73.32271661404612) bank158815 +158816 POINT(40.30163622389471 73.4417423558917) bank158816 +158817 POINT(41.58338527191531 74.14771777441506) bank158817 +158818 POINT(41.0400994075821 74.57805081426663) bank158818 +158819 POINT(41.580477590681554 73.42093442014858) bank158819 +158820 POINT(41.03419517482454 73.63105261201538) bank158820 +158821 POINT(40.81136734577201 74.28322346392625) bank158821 +158822 POINT(40.15322135501432 74.52206642274184) bank158822 +158823 POINT(40.52883319988328 73.14004733652794) bank158823 +158824 POINT(41.2433913621079 74.38068516609833) bank158824 +158825 POINT(41.107846374881774 73.4328299926482) bank158825 +158826 POINT(40.47972942695017 74.1472360817974) bank158826 +158827 POINT(40.59188763395358 74.88787336477246) bank158827 +158828 POINT(40.24010811205701 73.00833355875483) bank158828 +158829 POINT(39.87518599934329 73.70836952704687) bank158829 +158830 POINT(40.0649730657049 74.48820365539893) bank158830 +158831 POINT(41.00593393228148 73.20503137079835) bank158831 +158832 POINT(39.93038650065044 73.60283612775383) bank158832 +158833 POINT(41.12581433165721 73.71713663726452) bank158833 +158834 POINT(40.113368812958285 73.89066691027982) bank158834 +158835 POINT(39.7641662910907 74.43705950526592) bank158835 +158836 POINT(40.1409716474158 74.39035389054314) bank158836 +158837 POINT(39.85965580755864 74.55512057371213) bank158837 +158838 POINT(41.68722290026451 73.39039190632575) bank158838 +158839 POINT(41.3251361565005 74.60706091521295) bank158839 +158840 POINT(40.56633709148196 74.96358866493175) bank158840 +158841 POINT(40.13798006485291 74.46377831570966) bank158841 +158842 POINT(41.478160003983774 73.08675254888553) bank158842 +158843 POINT(40.51323291761734 74.78976554444989) bank158843 +158844 POINT(40.30131410392281 74.03015251591043) bank158844 +158845 POINT(39.87739940881052 74.07227256053852) bank158845 +158846 POINT(40.28554815784005 73.04427125940938) bank158846 +158847 POINT(41.167239089167765 74.23355025924913) bank158847 +158848 POINT(40.05032826735812 74.12161835382048) bank158848 +158849 POINT(41.310337032861845 73.65640219944412) bank158849 +158850 POINT(40.02554719385651 73.41854933830922) bank158850 +158851 POINT(41.10671402369521 74.94386568659483) bank158851 +158852 POINT(40.497009241948206 73.98670956714362) bank158852 +158853 POINT(41.59695349325647 74.39878135637755) bank158853 +158854 POINT(40.32175386868439 73.82900956004188) bank158854 +158855 POINT(41.70508209966601 74.94715302290805) bank158855 +158856 POINT(40.16258964065288 74.37969770965454) bank158856 +158857 POINT(40.899680484570155 73.99102745484903) bank158857 +158858 POINT(39.80031068207019 74.19895797382603) bank158858 +158859 POINT(40.12251284150604 73.40056792674935) bank158859 +158860 POINT(40.540719873733934 74.76970228878818) bank158860 +158861 POINT(41.589567517964596 73.39270267908995) bank158861 +158862 POINT(41.401699813442974 74.47373500698635) bank158862 +158863 POINT(39.820206557686774 73.2352102246168) bank158863 +158864 POINT(40.95258965295342 74.19438936701572) bank158864 +158865 POINT(39.800865500403745 73.97696742827294) bank158865 +158866 POINT(39.762923932498985 73.87019540008689) bank158866 +158867 POINT(40.33375538756983 73.69714020196784) bank158867 +158868 POINT(41.599110765904975 74.07620926259428) bank158868 +158869 POINT(41.66170541520277 73.71983119429775) bank158869 +158870 POINT(39.980110849483154 74.71643937400806) bank158870 +158871 POINT(40.994267002113986 73.28561460460026) bank158871 +158872 POINT(41.65942596078202 74.91352058261411) bank158872 +158873 POINT(39.79009503379295 73.89531766811464) bank158873 +158874 POINT(40.51226045503319 73.9766014137265) bank158874 +158875 POINT(40.94426560470191 73.65895134141677) bank158875 +158876 POINT(40.43191270603933 73.90369855512101) bank158876 +158877 POINT(41.06342361352647 73.4796889661189) bank158877 +158878 POINT(41.603115640562045 73.92244788330332) bank158878 +158879 POINT(41.16585710478229 74.23021599777202) bank158879 +158880 POINT(40.40698621140688 73.73499555874156) bank158880 +158881 POINT(40.24202077509349 74.17850811466187) bank158881 +158882 POINT(40.45361667965331 74.78529908896255) bank158882 +158883 POINT(40.84701742523751 73.61016839346722) bank158883 +158884 POINT(40.22098946390479 74.67015945899404) bank158884 +158885 POINT(41.481000918169876 73.03335648669918) bank158885 +158886 POINT(40.07870484803272 74.66242715070929) bank158886 +158887 POINT(41.48922006455311 74.33168701680205) bank158887 +158888 POINT(40.60848920587813 73.87145028634458) bank158888 +158889 POINT(41.337268476856224 73.58939387112417) bank158889 +158890 POINT(40.01860698264987 73.95542506285683) bank158890 +158891 POINT(40.00901227291011 73.4933358045886) bank158891 +158892 POINT(40.23644734924024 73.59206227718123) bank158892 +158893 POINT(39.85263409076258 73.43944948102445) bank158893 +158894 POINT(41.09446915242688 74.65252842776052) bank158894 +158895 POINT(40.732405386854644 73.2362916642447) bank158895 +158896 POINT(40.02679739078957 73.61354325785895) bank158896 +158897 POINT(41.29639797154373 74.47815683648213) bank158897 +158898 POINT(41.36097007221982 73.77937948444158) bank158898 +158899 POINT(40.14675275003436 74.67669959781516) bank158899 +158900 POINT(41.317159209035005 74.19604669301695) bank158900 +158901 POINT(40.78259982550041 73.51419934689514) bank158901 +158902 POINT(41.486478321733166 73.41952232907099) bank158902 +158903 POINT(40.787508750484676 74.07568293281177) bank158903 +158904 POINT(40.41172982318707 73.52635278893962) bank158904 +158905 POINT(40.26131611068134 74.55371769165183) bank158905 +158906 POINT(40.13853572878044 73.96591669236157) bank158906 +158907 POINT(40.73711490910456 74.23568626908876) bank158907 +158908 POINT(41.39647045156821 73.56395756335525) bank158908 +158909 POINT(40.347476636580204 74.20438335699949) bank158909 +158910 POINT(41.61666537425525 74.99805274733099) bank158910 +158911 POINT(41.64529125775889 73.75429468444267) bank158911 +158912 POINT(40.77281161665353 73.79490612318472) bank158912 +158913 POINT(40.57048991172826 74.54183987080982) bank158913 +158914 POINT(39.75272309321604 73.15091321993938) bank158914 +158915 POINT(40.620330207300576 73.71871318672521) bank158915 +158916 POINT(40.682809837354995 74.09022484147793) bank158916 +158917 POINT(40.52759114820277 74.37972280526445) bank158917 +158918 POINT(40.58362478256543 73.10894357473498) bank158918 +158919 POINT(40.49146541183407 74.35531486758912) bank158919 +158920 POINT(40.03916289513004 73.12736509866573) bank158920 +158921 POINT(40.628004257053696 73.64286089039001) bank158921 +158922 POINT(40.552851623822846 74.20508580770336) bank158922 +158923 POINT(40.79203077140551 73.94722142594576) bank158923 +158924 POINT(40.71778981215229 73.21122682078136) bank158924 +158925 POINT(40.39454947757097 74.7962262931291) bank158925 +158926 POINT(39.82308124703625 73.06244921107958) bank158926 +158927 POINT(40.91677360550249 74.7520858327326) bank158927 +158928 POINT(41.05822752118537 73.83515436146476) bank158928 +158929 POINT(41.54850240914166 73.13428062797351) bank158929 +158930 POINT(39.982787063625 74.78657544540846) bank158930 +158931 POINT(39.79156543002491 74.85442792520286) bank158931 +158932 POINT(41.52030311924242 74.83013009566511) bank158932 +158933 POINT(41.12707996162256 74.9885346739608) bank158933 +158934 POINT(41.54033653272996 73.47781799560629) bank158934 +158935 POINT(41.57488677961322 74.19386937819105) bank158935 +158936 POINT(41.06280194128786 73.95612072441513) bank158936 +158937 POINT(40.572530884209435 73.97913604271989) bank158937 +158938 POINT(40.3460178052547 73.72479513751752) bank158938 +158939 POINT(40.20172387749774 73.88207213440185) bank158939 +158940 POINT(41.140322702425365 73.24402147838731) bank158940 +158941 POINT(40.47136961719187 74.25760985016738) bank158941 +158942 POINT(39.801439007532764 73.98674305658638) bank158942 +158943 POINT(39.994852015784055 73.07096623867541) bank158943 +158944 POINT(40.34713016147728 74.94543126393171) bank158944 +158945 POINT(39.76596082151852 73.67040133273332) bank158945 +158946 POINT(41.27966446770931 74.65536228470837) bank158946 +158947 POINT(39.776450579601104 73.42653654757659) bank158947 +158948 POINT(39.852990613240884 73.31309170446389) bank158948 +158949 POINT(41.679529634992186 74.69175467474986) bank158949 +158950 POINT(41.62763250646388 74.73331201726313) bank158950 +158951 POINT(40.91302196788863 73.68500480728682) bank158951 +158952 POINT(40.49977266695713 73.19733165630561) bank158952 +158953 POINT(41.27985721189851 74.21726705077822) bank158953 +158954 POINT(40.31945644882621 73.54154302456026) bank158954 +158955 POINT(39.74046806365222 73.71708406309743) bank158955 +158956 POINT(40.36769609280268 74.54806461905511) bank158956 +158957 POINT(40.30652280863894 74.96529506501132) bank158957 +158958 POINT(41.25028904694315 73.06415203479735) bank158958 +158959 POINT(40.912469854105325 74.63126740443487) bank158959 +158960 POINT(41.454701581764525 73.45409672811277) bank158960 +158961 POINT(41.27366137772714 73.08295841154592) bank158961 +158962 POINT(41.10748589559732 74.65742586773274) bank158962 +158963 POINT(40.27699041480467 74.14011729486934) bank158963 +158964 POINT(40.543098177329526 73.74838876545921) bank158964 +158965 POINT(40.975002882579986 74.58736158249067) bank158965 +158966 POINT(41.4000149220212 73.03508627608527) bank158966 +158967 POINT(41.53639258768816 74.48337147538332) bank158967 +158968 POINT(40.192879315843584 74.20659957815435) bank158968 +158969 POINT(39.79372232533871 73.41810235269111) bank158969 +158970 POINT(40.81165696061854 74.32206523650902) bank158970 +158971 POINT(40.21606505697245 73.09573881313175) bank158971 +158972 POINT(40.0653379109071 73.59713562604928) bank158972 +158973 POINT(41.428066837620854 73.35287260184944) bank158973 +158974 POINT(41.690747189296566 74.17447409973386) bank158974 +158975 POINT(41.00993811346221 73.59845122472449) bank158975 +158976 POINT(40.18226225504644 74.64064482811108) bank158976 +158977 POINT(40.70683170082227 74.17312062757837) bank158977 +158978 POINT(39.96157623205712 73.96049754903798) bank158978 +158979 POINT(40.42915404877095 73.13914555315036) bank158979 +158980 POINT(40.54752420420715 74.78738501551474) bank158980 +158981 POINT(40.60254687538073 74.89212838917835) bank158981 +158982 POINT(39.972375069462466 73.48528840513276) bank158982 +158983 POINT(41.42950279913624 73.97812560840899) bank158983 +158984 POINT(40.22650785413703 74.48876081844374) bank158984 +158985 POINT(40.86123943373036 74.73827746898365) bank158985 +158986 POINT(40.02011250584563 73.335281762634) bank158986 +158987 POINT(41.13257707513047 74.19140166128074) bank158987 +158988 POINT(41.622483502961 73.327395089595) bank158988 +158989 POINT(41.085178400175316 74.34580271389488) bank158989 +158990 POINT(41.05268511344531 74.00489519985136) bank158990 +158991 POINT(41.17650190271248 74.71954678312594) bank158991 +158992 POINT(40.026622235733264 73.86977318301714) bank158992 +158993 POINT(41.709875604822294 73.64043575544018) bank158993 +158994 POINT(41.22945033250083 74.65105773603032) bank158994 +158995 POINT(39.799804669523354 74.75415767302648) bank158995 +158996 POINT(41.67529753945026 73.66319534861442) bank158996 +158997 POINT(40.56196779245407 73.27960735536604) bank158997 +158998 POINT(40.67555878630657 73.0451661581263) bank158998 +158999 POINT(41.214805247955425 73.33339251364859) bank158999 +159000 POINT(39.95366546886818 74.99753758128493) bank159000 +159001 POINT(40.913621898107664 73.91535410463035) bank159001 +159002 POINT(41.232545375307204 74.1782622383504) bank159002 +159003 POINT(39.83030682327864 73.7685600808273) bank159003 +159004 POINT(40.0740420220096 74.68929115495428) bank159004 +159005 POINT(41.46663727994691 74.67817524137631) bank159005 +159006 POINT(41.33716856792244 73.44592647486331) bank159006 +159007 POINT(41.32319720826622 74.90199294996978) bank159007 +159008 POINT(40.03030975711047 74.54354881561393) bank159008 +159009 POINT(41.09433954054747 74.55714817312179) bank159009 +159010 POINT(39.951421070548335 73.79073194014272) bank159010 +159011 POINT(40.49862003955667 74.65460967189196) bank159011 +159012 POINT(40.279077541976065 73.28672949975979) bank159012 +159013 POINT(41.328398310454766 74.79934205458098) bank159013 +159014 POINT(40.25825139962729 73.51459507510761) bank159014 +159015 POINT(41.691495134889294 73.39279434435745) bank159015 +159016 POINT(40.953726866333426 74.74863120303586) bank159016 +159017 POINT(39.78965392721923 74.6180645031443) bank159017 +159018 POINT(40.00700978308464 73.05147651773956) bank159018 +159019 POINT(40.29496532095801 74.36262888072258) bank159019 +159020 POINT(40.215105638688165 73.61181331272418) bank159020 +159021 POINT(39.792494461416744 73.64245283334186) bank159021 +159022 POINT(40.76990362447637 74.93163518987527) bank159022 +159023 POINT(39.80755623540456 73.74659657748703) bank159023 +159024 POINT(41.21621424488185 73.99294388296718) bank159024 +159025 POINT(40.07074710238732 73.45831167979719) bank159025 +159026 POINT(40.97545240162518 73.21221850346978) bank159026 +159027 POINT(40.61422005688918 74.71880795703647) bank159027 +159028 POINT(41.56650990852739 73.70663670449574) bank159028 +159029 POINT(40.671662141375386 73.01928575696694) bank159029 +159030 POINT(41.52852693442113 73.98673487161476) bank159030 +159031 POINT(41.519642950999625 73.41361948536144) bank159031 +159032 POINT(41.39304021081118 73.1682518840285) bank159032 +159033 POINT(40.15693532474209 74.14909609859254) bank159033 +159034 POINT(41.00265715443979 73.91953258555115) bank159034 +159035 POINT(40.757479800306264 74.88380606589965) bank159035 +159036 POINT(41.410500883386945 74.98985679561272) bank159036 +159037 POINT(41.66213513917198 74.50985268284727) bank159037 +159038 POINT(40.619920992461054 74.9827601673953) bank159038 +159039 POINT(40.97127778465622 73.66570719794608) bank159039 +159040 POINT(40.03517288243594 73.47123371839629) bank159040 +159041 POINT(39.79851118700211 74.05574465022305) bank159041 +159042 POINT(40.69289975937566 73.93696806605749) bank159042 +159043 POINT(39.7950140077857 73.90943513794079) bank159043 +159044 POINT(41.27666178148121 74.09324466529957) bank159044 +159045 POINT(41.69627410385007 73.72686154781938) bank159045 +159046 POINT(40.11438773044647 73.3886362122702) bank159046 +159047 POINT(39.87556180512877 73.29748219649704) bank159047 +159048 POINT(39.996910584183844 73.08256754658211) bank159048 +159049 POINT(41.70384322423014 74.77114170150595) bank159049 +159050 POINT(41.568076165680644 74.7309577303091) bank159050 +159051 POINT(40.11262980693209 74.71016819283311) bank159051 +159052 POINT(39.94710567794002 73.91532741373791) bank159052 +159053 POINT(39.877512122332156 74.67496634310525) bank159053 +159054 POINT(40.8174735938016 74.08516337589663) bank159054 +159055 POINT(39.959547542445875 74.50659335199327) bank159055 +159056 POINT(41.07834199090802 74.84267054586059) bank159056 +159057 POINT(40.188462284042636 73.27938547039554) bank159057 +159058 POINT(39.949851320843166 73.79665179871964) bank159058 +159059 POINT(40.73688654983764 73.52804473378465) bank159059 +159060 POINT(41.674315452986384 73.67039043697474) bank159060 +159061 POINT(39.89334925750176 73.90403343471506) bank159061 +159062 POINT(40.99310422654856 73.21549899729722) bank159062 +159063 POINT(40.79869887730611 74.17841064091989) bank159063 +159064 POINT(40.66899281580315 74.61664399763377) bank159064 +159065 POINT(40.19447572798045 74.88293203606551) bank159065 +159066 POINT(41.580295214981085 74.37778041960696) bank159066 +159067 POINT(40.6270828915413 73.36005713615819) bank159067 +159068 POINT(40.973631457141565 74.2548236817287) bank159068 +159069 POINT(41.042324593666436 74.80198488236982) bank159069 +159070 POINT(41.01967531492765 73.36072796644292) bank159070 +159071 POINT(41.37631369083415 73.64020068475874) bank159071 +159072 POINT(40.459369957272685 73.7666707632859) bank159072 +159073 POINT(39.913178236870195 73.26520341034018) bank159073 +159074 POINT(41.44094547470362 73.91293752090097) bank159074 +159075 POINT(40.70348786197127 73.20391003107008) bank159075 +159076 POINT(40.13081210811464 73.26869776658616) bank159076 +159077 POINT(39.83934857452364 73.91882818017037) bank159077 +159078 POINT(40.00231776618972 74.13395605624267) bank159078 +159079 POINT(41.64082191214727 74.91785450879503) bank159079 +159080 POINT(41.09633374345597 73.04913295512792) bank159080 +159081 POINT(40.11362640487376 74.56661396982601) bank159081 +159082 POINT(41.10978943109515 73.54681123786173) bank159082 +159083 POINT(40.344383378823615 74.66913414012319) bank159083 +159084 POINT(40.30657339993156 73.94224304196852) bank159084 +159085 POINT(41.04839942806746 74.03298874578387) bank159085 +159086 POINT(40.071727439483105 73.43643343848836) bank159086 +159087 POINT(40.35821399268301 73.66443262141067) bank159087 +159088 POINT(40.869644681437336 74.46423577958913) bank159088 +159089 POINT(40.024010124362555 74.03500784667357) bank159089 +159090 POINT(40.15602454919797 73.536153618351) bank159090 +159091 POINT(40.42493723860853 73.83349651200395) bank159091 +159092 POINT(40.936061072372176 73.57683346307779) bank159092 +159093 POINT(41.03256619414656 73.50428171798231) bank159093 +159094 POINT(40.36071146677862 74.79516573193112) bank159094 +159095 POINT(41.30194523546461 74.66497692309088) bank159095 +159096 POINT(39.72105192058217 74.29523822037055) bank159096 +159097 POINT(40.092061914404326 73.23170263995374) bank159097 +159098 POINT(41.338117492602464 74.66860246400583) bank159098 +159099 POINT(41.08984611214198 74.81365330026786) bank159099 +159100 POINT(40.78320522260952 73.53040278805958) bank159100 +159101 POINT(40.512897275769085 74.58591655422792) bank159101 +159102 POINT(40.55491593163701 74.43596676309707) bank159102 +159103 POINT(41.32741970379388 73.6224305891754) bank159103 +159104 POINT(41.05431542609238 74.39798006158088) bank159104 +159105 POINT(41.28268373481441 73.11256247989799) bank159105 +159106 POINT(40.35463315646446 74.64354070183009) bank159106 +159107 POINT(40.89613207425228 74.70082946368537) bank159107 +159108 POINT(40.16160561831661 73.56852183835338) bank159108 +159109 POINT(39.88843554347006 74.24853597627984) bank159109 +159110 POINT(40.098303318915086 73.60071751825222) bank159110 +159111 POINT(41.06509455286136 74.5553203199459) bank159111 +159112 POINT(41.36870979444765 73.14840429513822) bank159112 +159113 POINT(40.883718347680286 73.56174224725794) bank159113 +159114 POINT(39.94511044644128 74.88587442669511) bank159114 +159115 POINT(41.18438258712536 74.43911049409796) bank159115 +159116 POINT(41.65251026754398 73.585649730542) bank159116 +159117 POINT(40.324883638548485 73.66146010797989) bank159117 +159118 POINT(40.89368752410341 74.80955769591856) bank159118 +159119 POINT(40.15051650903928 74.75940253420784) bank159119 +159120 POINT(41.11190417002248 74.78895613600449) bank159120 +159121 POINT(41.69828948511538 73.4586176648909) bank159121 +159122 POINT(40.60351748774456 73.20143697670268) bank159122 +159123 POINT(40.968021898634625 73.2045570374626) bank159123 +159124 POINT(40.113490724133115 73.20928172080197) bank159124 +159125 POINT(41.23412665988286 74.06568304880322) bank159125 +159126 POINT(40.18063708287176 74.84288603108988) bank159126 +159127 POINT(40.409327530729264 73.2191769136189) bank159127 +159128 POINT(40.29337217886653 74.71091546893074) bank159128 +159129 POINT(41.42234761476586 74.697553039887) bank159129 +159130 POINT(41.00771915807713 74.78111231494951) bank159130 +159131 POINT(40.72053887560839 74.62334833106513) bank159131 +159132 POINT(39.80280588721938 73.93627657070385) bank159132 +159133 POINT(41.09374371967802 74.34299187469486) bank159133 +159134 POINT(39.98415642630728 74.78706620376458) bank159134 +159135 POINT(41.68742897323723 74.62174420636285) bank159135 +159136 POINT(40.151023296938504 73.74604735946633) bank159136 +159137 POINT(41.09294681388365 73.50239048749464) bank159137 +159138 POINT(41.33373438022515 74.68654922219815) bank159138 +159139 POINT(41.41972997998339 74.09992328341012) bank159139 +159140 POINT(40.90910200176312 74.46973926223187) bank159140 +159141 POINT(40.905749708536405 74.57999149708762) bank159141 +159142 POINT(40.05054864931318 74.63365628714325) bank159142 +159143 POINT(41.49290598229986 74.87438583871146) bank159143 +159144 POINT(41.64924190162182 74.21836647670638) bank159144 +159145 POINT(40.96875625310796 74.7040483392706) bank159145 +159146 POINT(41.18392204203453 74.41531835467148) bank159146 +159147 POINT(41.017846493326665 73.68223006072826) bank159147 +159148 POINT(40.29724585644682 74.61468950053747) bank159148 +159149 POINT(41.5743134768864 73.93012704282798) bank159149 +159150 POINT(41.35746566353509 74.53677136616035) bank159150 +159151 POINT(40.833909574275914 73.52781134831027) bank159151 +159152 POINT(41.244115803584556 73.63215608962695) bank159152 +159153 POINT(40.30846531023118 73.99315494777403) bank159153 +159154 POINT(40.63160400285919 73.32551106319777) bank159154 +159155 POINT(40.23180910319827 74.92996878182024) bank159155 +159156 POINT(40.210538271710234 73.91293245851054) bank159156 +159157 POINT(40.72860277992087 74.29394092461135) bank159157 +159158 POINT(41.30169814055547 73.88085784131098) bank159158 +159159 POINT(41.53634909525537 74.44140400778701) bank159159 +159160 POINT(41.11189987451484 73.76072149783916) bank159160 +159161 POINT(41.20271824874724 73.1811084368117) bank159161 +159162 POINT(40.70085232612651 74.53512041592603) bank159162 +159163 POINT(41.30946793957318 74.9692036675938) bank159163 +159164 POINT(40.781032486712434 73.66713227823821) bank159164 +159165 POINT(40.58454146015885 73.39386728230362) bank159165 +159166 POINT(41.36698899151204 74.17633950194234) bank159166 +159167 POINT(40.570226274035875 74.65074864320714) bank159167 +159168 POINT(40.22371755605648 74.03759278038548) bank159168 +159169 POINT(40.85171261429615 74.24145788794382) bank159169 +159170 POINT(41.16102353940357 74.13621472176825) bank159170 +159171 POINT(40.545581724937485 74.42165242480498) bank159171 +159172 POINT(41.704913632318004 73.94893468678254) bank159172 +159173 POINT(40.59295992078746 74.77574521676634) bank159173 +159174 POINT(41.17480335084989 73.09783215626067) bank159174 +159175 POINT(41.39028282384101 73.74223461742213) bank159175 +159176 POINT(40.88769603443494 73.86796892404728) bank159176 +159177 POINT(40.59306471321655 74.14644715708684) bank159177 +159178 POINT(40.10164036231991 74.10463680093685) bank159178 +159179 POINT(41.36875614233156 74.39371494996715) bank159179 +159180 POINT(39.97452163536565 74.1106827350252) bank159180 +159181 POINT(41.13687205176819 73.71515278659196) bank159181 +159182 POINT(40.40865358916924 73.93796098586039) bank159182 +159183 POINT(40.14790205572272 73.73183774415598) bank159183 +159184 POINT(41.303190336506184 73.70083194246986) bank159184 +159185 POINT(40.73711548543305 73.25009350264276) bank159185 +159186 POINT(41.07972467278934 73.63487885248836) bank159186 +159187 POINT(40.777894970285814 73.0626805587323) bank159187 +159188 POINT(40.26286932975266 74.20896862678714) bank159188 +159189 POINT(39.85787013257578 74.8773651974564) bank159189 +159190 POINT(41.246739481483765 74.8809137940995) bank159190 +159191 POINT(40.299149531938355 73.67834445819602) bank159191 +159192 POINT(41.56504886425013 73.92110518450842) bank159192 +159193 POINT(40.86487682168282 73.15400625830262) bank159193 +159194 POINT(39.970158755792646 73.23507387934656) bank159194 +159195 POINT(40.146730534475324 74.28004324926262) bank159195 +159196 POINT(41.57733785302309 74.66736941434061) bank159196 +159197 POINT(40.20554090126377 73.91942789924464) bank159197 +159198 POINT(40.48029952035197 74.71920477871295) bank159198 +159199 POINT(40.13492639734576 73.04489447242211) bank159199 +159200 POINT(40.739443868777556 74.76083548958464) bank159200 +159201 POINT(41.623346963245915 74.0326043990563) bank159201 +159202 POINT(41.54090216024197 73.12013210655412) bank159202 +159203 POINT(41.35867749614426 74.37291572534991) bank159203 +159204 POINT(40.82750582262012 74.07176428895197) bank159204 +159205 POINT(40.619497151856386 73.44128943413497) bank159205 +159206 POINT(40.836617380529994 73.15397042454217) bank159206 +159207 POINT(40.015350455990585 73.41585219169511) bank159207 +159208 POINT(40.590148355572126 74.92276407896952) bank159208 +159209 POINT(40.967147110101166 74.09073263043678) bank159209 +159210 POINT(41.19752687168719 74.77450019296248) bank159210 +159211 POINT(40.98296224762942 73.19300406026126) bank159211 +159212 POINT(40.81265866085962 73.2526956728856) bank159212 +159213 POINT(40.1142183692304 73.1818831223755) bank159213 +159214 POINT(40.12539690742716 74.18982670916519) bank159214 +159215 POINT(41.19589858088275 73.52927757866723) bank159215 +159216 POINT(40.092938685832806 74.19475046579387) bank159216 +159217 POINT(41.22034939384619 74.49136446545818) bank159217 +159218 POINT(41.66237054880518 73.35251946257605) bank159218 +159219 POINT(41.539373291991566 74.66337393696415) bank159219 +159220 POINT(40.50432921593653 74.2278909726595) bank159220 +159221 POINT(40.230590335221414 74.12906905931703) bank159221 +159222 POINT(41.0138616663656 74.84373646738639) bank159222 +159223 POINT(41.22139808706147 73.2615386175799) bank159223 +159224 POINT(39.88128865633098 74.62722579988693) bank159224 +159225 POINT(41.23300686028423 74.41475524294225) bank159225 +159226 POINT(40.70443617957891 73.9505188250099) bank159226 +159227 POINT(40.405055031932434 74.4423705875958) bank159227 +159228 POINT(40.48711374110504 74.10573267946309) bank159228 +159229 POINT(41.424505295542154 73.67911441237604) bank159229 +159230 POINT(40.61811589569544 73.31819739886343) bank159230 +159231 POINT(40.299915944744036 73.71907084991999) bank159231 +159232 POINT(40.405669902913075 74.40757171154335) bank159232 +159233 POINT(39.80237349364413 73.81758704989406) bank159233 +159234 POINT(41.60338960518561 74.43153282149407) bank159234 +159235 POINT(40.16878859834822 74.91334413598248) bank159235 +159236 POINT(39.959091412578566 73.29833621308222) bank159236 +159237 POINT(41.01945374760194 74.96665742458899) bank159237 +159238 POINT(40.122936821219135 73.6030307588195) bank159238 +159239 POINT(41.34276034557154 74.6508320698431) bank159239 +159240 POINT(40.915953384683185 74.82991751999594) bank159240 +159241 POINT(40.7930031822956 74.07835506120597) bank159241 +159242 POINT(41.10902106352784 74.23428161855136) bank159242 +159243 POINT(40.074981807863814 73.53966116588249) bank159243 +159244 POINT(40.64195610722567 73.981378248955) bank159244 +159245 POINT(41.289558993886175 74.4656416066832) bank159245 +159246 POINT(39.84493974017613 73.32286481281724) bank159246 +159247 POINT(41.477052891982325 73.54052115225456) bank159247 +159248 POINT(40.69808713384009 74.71668601596923) bank159248 +159249 POINT(41.39142946984304 73.13222694710545) bank159249 +159250 POINT(40.07969926531526 74.47892790622545) bank159250 +159251 POINT(39.998334717295876 74.9356583624282) bank159251 +159252 POINT(40.13055756668123 73.21898189758006) bank159252 +159253 POINT(40.86165168338321 73.11651632740563) bank159253 +159254 POINT(40.24465468691202 73.54660017159142) bank159254 +159255 POINT(40.82899262970213 73.22550971641971) bank159255 +159256 POINT(40.46604953865331 74.7395658468986) bank159256 +159257 POINT(40.60336294368719 74.40994176314815) bank159257 +159258 POINT(40.52027688609399 73.37609811954559) bank159258 +159259 POINT(39.959095583578375 74.65315775639317) bank159259 +159260 POINT(41.20133177835024 74.92182488630793) bank159260 +159261 POINT(40.93218982880295 74.63303357024118) bank159261 +159262 POINT(40.61501015572676 73.37135761281245) bank159262 +159263 POINT(41.404743566162125 74.89811344790719) bank159263 +159264 POINT(40.46461061361652 74.81242315948863) bank159264 +159265 POINT(40.93713128638497 74.61241774632529) bank159265 +159266 POINT(40.19675024621236 74.82129269449406) bank159266 +159267 POINT(40.20581145626707 74.41683003728679) bank159267 +159268 POINT(40.936754392421356 73.24504306972969) bank159268 +159269 POINT(41.66011221685583 73.1419843860695) bank159269 +159270 POINT(40.45038474878695 74.2410352269751) bank159270 +159271 POINT(41.13071596485015 74.11147279748167) bank159271 +159272 POINT(40.78350722579326 74.56235995657552) bank159272 +159273 POINT(40.0241179656249 73.40937695895701) bank159273 +159274 POINT(40.20644488839032 73.87875960084114) bank159274 +159275 POINT(40.14259515435857 73.72873696949465) bank159275 +159276 POINT(41.488139553615724 73.89053833782003) bank159276 +159277 POINT(41.31742519110427 74.52400461233441) bank159277 +159278 POINT(40.46743825018968 74.8303342111431) bank159278 +159279 POINT(41.66318987984399 73.91048209148008) bank159279 +159280 POINT(40.10842259786024 74.04403914252427) bank159280 +159281 POINT(41.43672152246891 73.56312538709841) bank159281 +159282 POINT(39.75078650139658 73.68980406160195) bank159282 +159283 POINT(40.14137200183877 74.67378137571447) bank159283 +159284 POINT(40.328999355904024 74.78823587984249) bank159284 +159285 POINT(40.65203781337138 74.35173697049814) bank159285 +159286 POINT(40.17430758111604 75.00224137061977) bank159286 +159287 POINT(41.354671232510825 74.84668937588539) bank159287 +159288 POINT(39.84357417606753 74.07223145232858) bank159288 +159289 POINT(39.81690916631256 73.47942619378918) bank159289 +159290 POINT(40.54512999298339 73.11417016969622) bank159290 +159291 POINT(40.69331218551063 74.45177688355103) bank159291 +159292 POINT(40.109377591410954 73.91933600980671) bank159292 +159293 POINT(40.104400349171385 74.52741682567714) bank159293 +159294 POINT(41.298628486868296 74.53564206156457) bank159294 +159295 POINT(41.00213757390717 73.39956150720106) bank159295 +159296 POINT(40.25267083574244 74.25515440297491) bank159296 +159297 POINT(40.19940509773139 73.62427436020712) bank159297 +159298 POINT(40.605101692380764 74.48726442600014) bank159298 +159299 POINT(39.89643180023346 74.7378279720507) bank159299 +159300 POINT(40.84282437993445 73.12073066588178) bank159300 +159301 POINT(41.52037300605989 73.44580225748977) bank159301 +159302 POINT(39.87374095289373 73.9981756564956) bank159302 +159303 POINT(41.58962641898145 73.3211197639006) bank159303 +159304 POINT(41.31359766584889 73.94921652129126) bank159304 +159305 POINT(40.10945968947479 73.59342365327066) bank159305 +159306 POINT(40.495442923534455 74.68822473850358) bank159306 +159307 POINT(40.66602208425645 74.95120657719822) bank159307 +159308 POINT(40.08354274912919 74.52336589151409) bank159308 +159309 POINT(41.46163958690779 73.37864008828613) bank159309 +159310 POINT(39.94607393614024 74.22660828664914) bank159310 +159311 POINT(41.18276204292551 73.63641748981132) bank159311 +159312 POINT(40.13081645699519 73.36517745517281) bank159312 +159313 POINT(41.49508640857357 73.37694783616003) bank159313 +159314 POINT(41.06382919999801 73.27593556213384) bank159314 +159315 POINT(39.758668763279545 73.29950971159043) bank159315 +159316 POINT(40.63093546103539 74.24046320508371) bank159316 +159317 POINT(39.95456215354418 73.56865584128737) bank159317 +159318 POINT(40.02715207154418 73.1508953371411) bank159318 +159319 POINT(40.95752954477718 73.28611529840956) bank159319 +159320 POINT(39.822548112574125 74.85634312008746) bank159320 +159321 POINT(41.05205426285624 74.0740182406253) bank159321 +159322 POINT(40.21774363773513 73.08538170284353) bank159322 +159323 POINT(41.26294656990663 74.15822840484184) bank159323 +159324 POINT(41.28674561187891 73.39849140704673) bank159324 +159325 POINT(41.02767345032275 73.387752071513) bank159325 +159326 POINT(40.33249911329282 74.55671760597474) bank159326 +159327 POINT(40.62759842663225 74.63519614944465) bank159327 +159328 POINT(41.144785253250404 73.84350956660803) bank159328 +159329 POINT(41.39027125117618 74.770538672803) bank159329 +159330 POINT(40.24459292705769 74.8133185089027) bank159330 +159331 POINT(40.03198343757664 73.7179451162979) bank159331 +159332 POINT(40.45487951519341 74.48258271576515) bank159332 +159333 POINT(39.97573673171412 73.94131086092912) bank159333 +159334 POINT(40.169785979492644 73.18974099183534) bank159334 +159335 POINT(40.02840155272329 74.69520308911115) bank159335 +159336 POINT(41.36315098019726 73.48724102677808) bank159336 +159337 POINT(41.18404037867175 73.55095201902921) bank159337 +159338 POINT(41.109935308247685 73.11536638957932) bank159338 +159339 POINT(39.74887908403222 73.46660011201782) bank159339 +159340 POINT(41.388309134492324 73.99729260074172) bank159340 +159341 POINT(39.83036242242046 74.66942552355337) bank159341 +159342 POINT(41.15768920140213 73.18129740401257) bank159342 +159343 POINT(41.00456510301434 73.27481309413756) bank159343 +159344 POINT(41.37781878478535 74.14121285660254) bank159344 +159345 POINT(40.659427736819154 73.98125425228523) bank159345 +159346 POINT(40.70703082897369 73.20484392423027) bank159346 +159347 POINT(40.78429570718358 75.00505588822675) bank159347 +159348 POINT(40.04062529352373 74.28976122511544) bank159348 +159349 POINT(40.26113857537936 74.76191204232563) bank159349 +159350 POINT(41.627624354979744 74.37221119162506) bank159350 +159351 POINT(41.172555398310294 73.79419252925875) bank159351 +159352 POINT(41.429401066546056 74.6140153007306) bank159352 +159353 POINT(39.96298195116594 74.78459750525741) bank159353 +159354 POINT(41.26249367209975 73.16873028480339) bank159354 +159355 POINT(39.81130733919201 73.5528085191191) bank159355 +159356 POINT(40.85935794428269 73.91703338243217) bank159356 +159357 POINT(40.695826565588064 74.74443204078682) bank159357 +159358 POINT(40.18006792532529 74.89609812079783) bank159358 +159359 POINT(40.27767790880721 74.39831714201586) bank159359 +159360 POINT(40.55671018430399 74.03066303440113) bank159360 +159361 POINT(40.50381300344061 73.166943583151) bank159361 +159362 POINT(40.50877891917967 73.2961843936949) bank159362 +159363 POINT(40.01460358273993 74.9301052139307) bank159363 +159364 POINT(40.900837682046266 73.48334057999118) bank159364 +159365 POINT(39.89819649755354 74.02827166010607) bank159365 +159366 POINT(41.26039462526097 74.54344708747087) bank159366 +159367 POINT(40.32037402631494 73.84351066280924) bank159367 +159368 POINT(40.24547569512044 73.23679317473028) bank159368 +159369 POINT(41.20997798338036 74.91054090228265) bank159369 +159370 POINT(41.28211855015667 74.68643610975968) bank159370 +159371 POINT(41.66121068035669 74.69791516505359) bank159371 +159372 POINT(40.64876721353251 74.35270252287859) bank159372 +159373 POINT(41.563155205759436 73.48314348165316) bank159373 +159374 POINT(40.24063781502207 73.22670143127976) bank159374 +159375 POINT(40.791543533168976 74.87316561328005) bank159375 +159376 POINT(40.45043777317774 74.83416862256334) bank159376 +159377 POINT(40.943966351949065 73.65945999599013) bank159377 +159378 POINT(41.31902215549182 74.56537780908205) bank159378 +159379 POINT(40.70633307624556 73.2230450419779) bank159379 +159380 POINT(40.64255586506577 73.50427765415044) bank159380 +159381 POINT(40.454850638813255 74.53164115900695) bank159381 +159382 POINT(40.72817299115542 73.01565684574625) bank159382 +159383 POINT(40.03106414688602 74.53389271851626) bank159383 +159384 POINT(40.052039751695204 74.56852048422881) bank159384 +159385 POINT(40.77699713899011 74.89031746557448) bank159385 +159386 POINT(40.860746723876574 74.65172207341482) bank159386 +159387 POINT(40.19224102973982 74.91242257193075) bank159387 +159388 POINT(40.20148648303145 74.21693133865361) bank159388 +159389 POINT(41.01092303292956 74.71463777050728) bank159389 +159390 POINT(41.662157996275496 73.09288843580009) bank159390 +159391 POINT(41.3884987706546 73.26195119630587) bank159391 +159392 POINT(40.441490950371545 74.90560454051835) bank159392 +159393 POINT(41.329793759662635 74.87727814978203) bank159393 +159394 POINT(41.03229580521393 73.08161925202005) bank159394 +159395 POINT(39.84858150411101 73.90397491299792) bank159395 +159396 POINT(40.22322353408762 73.25166785201009) bank159396 +159397 POINT(40.14343020387173 73.43144718851613) bank159397 +159398 POINT(39.739111878765726 73.16270796781068) bank159398 +159399 POINT(41.222226656530225 74.97243148417259) bank159399 +159400 POINT(41.39091952372272 73.370374634588) bank159400 +159401 POINT(40.37889810785935 74.63238507479097) bank159401 +159402 POINT(40.23852499175713 74.3979391205632) bank159402 +159403 POINT(40.27130892056188 73.44559670783126) bank159403 +159404 POINT(40.40883071720149 73.99772922691429) bank159404 +159405 POINT(41.49154140556505 73.32704977681912) bank159405 +159406 POINT(41.227136903784746 74.74952483079967) bank159406 +159407 POINT(40.18270104307983 73.92402458222983) bank159407 +159408 POINT(41.05365709924502 74.39140522448494) bank159408 +159409 POINT(41.680509945120605 73.51559504059293) bank159409 +159410 POINT(41.16832923654112 74.77104468142676) bank159410 +159411 POINT(39.8308441526106 73.57659274873821) bank159411 +159412 POINT(39.90667063667087 73.37841461529966) bank159412 +159413 POINT(41.66031650143002 74.30184909029687) bank159413 +159414 POINT(40.892571870511745 73.63745765383293) bank159414 +159415 POINT(40.411155409932284 74.74771303307841) bank159415 +159416 POINT(40.04194002774636 73.53117821229397) bank159416 +159417 POINT(40.39752142308649 73.8286847646491) bank159417 +159418 POINT(40.42531448093338 73.18294610226958) bank159418 +159419 POINT(40.97100954031777 73.54770718297819) bank159419 +159420 POINT(39.91718189955174 73.88668029703477) bank159420 +159421 POINT(41.38988647784807 74.61347812433115) bank159421 +159422 POINT(41.644331804859824 74.52054516537683) bank159422 +159423 POINT(41.71261179906551 73.38803574539793) bank159423 +159424 POINT(40.32803001439007 74.08623216291772) bank159424 +159425 POINT(40.62064951447446 73.46266708305086) bank159425 +159426 POINT(41.33205103882448 73.42259975291908) bank159426 +159427 POINT(39.84236166221977 73.62677910443061) bank159427 +159428 POINT(39.847596853114716 73.18153648590314) bank159428 +159429 POINT(40.98877216711159 73.97179735815028) bank159429 +159430 POINT(40.738244153061295 73.87632610198544) bank159430 +159431 POINT(40.33921298435401 73.46921995261157) bank159431 +159432 POINT(40.788137111268554 74.79200001097426) bank159432 +159433 POINT(41.34082658367541 73.94607776673996) bank159433 +159434 POINT(40.2967004349967 73.44256971203757) bank159434 +159435 POINT(41.606583778051515 73.57372750206163) bank159435 +159436 POINT(41.11293935310113 74.34280044386173) bank159436 +159437 POINT(40.527662994747544 73.84781922530777) bank159437 +159438 POINT(41.59887375376663 73.63653501831193) bank159438 +159439 POINT(41.370817781388475 73.76709353000004) bank159439 +159440 POINT(39.89626832761511 73.80663227128885) bank159440 +159441 POINT(40.893648432753686 74.45752978226274) bank159441 +159442 POINT(40.686537141030044 74.304752311015) bank159442 +159443 POINT(40.459200822585345 73.5410134917988) bank159443 +159444 POINT(41.62246502999639 74.13869956816238) bank159444 +159445 POINT(40.96966848241409 74.33708315116233) bank159445 +159446 POINT(41.51795464726687 73.56033157046181) bank159446 +159447 POINT(41.71026761225952 74.65615638938635) bank159447 +159448 POINT(41.59478332276561 73.90675725376092) bank159448 +159449 POINT(40.97136611793156 73.86483232509495) bank159449 +159450 POINT(40.16864345100856 74.31507206293986) bank159450 +159451 POINT(40.711412668547666 74.5273193022357) bank159451 +159452 POINT(41.241980640084066 73.91473876079257) bank159452 +159453 POINT(40.828031847578096 74.49010954856382) bank159453 +159454 POINT(40.8856455926792 74.97137192687573) bank159454 +159455 POINT(40.06926299051783 73.0349017405257) bank159455 +159456 POINT(40.65430588991175 74.06283361840512) bank159456 +159457 POINT(41.28405545419506 73.40672000192703) bank159457 +159458 POINT(40.20613692907391 74.90187555059865) bank159458 +159459 POINT(41.261954417785965 74.81993637468359) bank159459 +159460 POINT(40.62121102374469 74.87966730939806) bank159460 +159461 POINT(39.92446595182162 73.53629523920388) bank159461 +159462 POINT(39.83846948038432 74.59832168766873) bank159462 +159463 POINT(40.0217884805934 73.41988725957141) bank159463 +159464 POINT(40.311656404846815 74.20023490847088) bank159464 +159465 POINT(41.293198781351876 74.17844763147802) bank159465 +159466 POINT(41.00472463223088 73.3156348056665) bank159466 +159467 POINT(41.445564946853715 73.021963260383) bank159467 +159468 POINT(40.52719216795723 73.26523336459582) bank159468 +159469 POINT(41.70770018007439 74.31090093510136) bank159469 +159470 POINT(40.16091220310438 73.47996879205158) bank159470 +159471 POINT(40.202625407253315 74.1498967457001) bank159471 +159472 POINT(39.76138978411625 73.73231246576799) bank159472 +159473 POINT(40.78325132590679 73.80387201088325) bank159473 +159474 POINT(40.291820210790455 74.07576725706197) bank159474 +159475 POINT(41.63596195423335 74.3169887413068) bank159475 +159476 POINT(40.41668776683435 73.55661118089563) bank159476 +159477 POINT(41.112127827769726 73.27565714883889) bank159477 +159478 POINT(40.85275891560758 73.61495655969742) bank159478 +159479 POINT(40.56659632248299 73.54700992297148) bank159479 +159480 POINT(40.99828689015978 73.42191102494897) bank159480 +159481 POINT(41.07108878145494 74.28485562094234) bank159481 +159482 POINT(40.73035152142792 73.21187431726959) bank159482 +159483 POINT(40.23386712064373 73.79979169833321) bank159483 +159484 POINT(40.83076799236295 73.98145587609243) bank159484 +159485 POINT(41.09734080683126 74.83925276142494) bank159485 +159486 POINT(39.9017447540854 74.26599574127748) bank159486 +159487 POINT(41.23765735576569 74.20554661262179) bank159487 +159488 POINT(40.842520698781684 74.13369144890213) bank159488 +159489 POINT(40.07775112720469 74.34972518284873) bank159489 +159490 POINT(40.93552993225561 74.15057231067681) bank159490 +159491 POINT(40.10988467674437 73.12045838454273) bank159491 +159492 POINT(40.408426625440214 74.1094098421249) bank159492 +159493 POINT(40.67611132517013 73.72269345535928) bank159493 +159494 POINT(41.26173221888882 73.58613258574397) bank159494 +159495 POINT(41.40123538043428 74.65824673250978) bank159495 +159496 POINT(41.453434260163434 73.31402920640211) bank159496 +159497 POINT(41.56961117049445 74.35130875456959) bank159497 +159498 POINT(39.887490972592204 73.46555464626353) bank159498 +159499 POINT(40.52833317381348 74.96286693850182) bank159499 +159500 POINT(40.71167438641045 74.43905250069396) bank159500 +159501 POINT(41.412158815180675 73.21920715895834) bank159501 +159502 POINT(39.7988879159026 73.10640037535593) bank159502 +159503 POINT(41.10548401490165 73.58381867354936) bank159503 +159504 POINT(41.62941320472274 73.72867050947556) bank159504 +159505 POINT(40.24219404055031 73.10257087524073) bank159505 +159506 POINT(40.91060377810737 74.99346991121885) bank159506 +159507 POINT(40.289555296334 73.11707864699733) bank159507 +159508 POINT(40.499683776053274 73.54588893730931) bank159508 +159509 POINT(41.601690415572314 74.27096118711717) bank159509 +159510 POINT(41.434998262645365 74.12119643992618) bank159510 +159511 POINT(40.40468154870363 73.11658002082879) bank159511 +159512 POINT(40.1032519928029 73.66552730520733) bank159512 +159513 POINT(39.855480900680654 74.40807718818303) bank159513 +159514 POINT(40.84748370336446 73.07591447537439) bank159514 +159515 POINT(40.98678392146111 73.7827659587514) bank159515 +159516 POINT(41.49439222569525 74.35137909632121) bank159516 +159517 POINT(40.34699676923245 74.90383962458192) bank159517 +159518 POINT(41.16862323821581 74.24656668703756) bank159518 +159519 POINT(41.63346311697182 74.09288155663408) bank159519 +159520 POINT(41.53367552570103 74.76527132629741) bank159520 +159521 POINT(41.274522301231904 73.42235073365617) bank159521 +159522 POINT(41.51677713062412 74.4100399393005) bank159522 +159523 POINT(40.865891034273574 73.49553203295845) bank159523 +159524 POINT(41.314007494745326 73.30929729387864) bank159524 +159525 POINT(40.1043682377271 74.38364988289102) bank159525 +159526 POINT(40.31762010266754 74.59072083496102) bank159526 +159527 POINT(39.78028708255173 74.3773625108965) bank159527 +159528 POINT(41.67783786345617 73.04006443960635) bank159528 +159529 POINT(41.565543865739805 74.46195128710866) bank159529 +159530 POINT(41.203972871124776 74.51561419471619) bank159530 +159531 POINT(41.19667784456694 73.43535276983464) bank159531 +159532 POINT(40.43213079003513 73.29540912905784) bank159532 +159533 POINT(40.858177471229894 74.20989686134722) bank159533 +159534 POINT(41.481709821566504 74.18280765873975) bank159534 +159535 POINT(40.444211338303774 73.97287336704875) bank159535 +159536 POINT(39.775131099537965 73.22454374816778) bank159536 +159537 POINT(40.50940840082468 73.8667932520483) bank159537 +159538 POINT(41.40597536231528 74.76140877768326) bank159538 +159539 POINT(39.93559151643254 74.68088828628167) bank159539 +159540 POINT(40.99573699772781 73.715163704004) bank159540 +159541 POINT(39.91766824957489 74.23161303445305) bank159541 +159542 POINT(40.679388401111304 74.16581714489699) bank159542 +159543 POINT(41.710821912433495 73.91428786665358) bank159543 +159544 POINT(41.5865220127736 74.77181532432537) bank159544 +159545 POINT(41.20884553543357 74.4204244219372) bank159545 +159546 POINT(40.04272385860597 73.17463068273314) bank159546 +159547 POINT(40.22059282528248 74.77090157822037) bank159547 +159548 POINT(39.71657920889678 74.88290331449517) bank159548 +159549 POINT(39.932439451487404 73.12866369396384) bank159549 +159550 POINT(41.23850132134716 73.52936483068977) bank159550 +159551 POINT(40.41433815772295 74.63624728349282) bank159551 +159552 POINT(41.08911144545748 74.69745066463564) bank159552 +159553 POINT(41.002176826232194 74.63429066909015) bank159553 +159554 POINT(40.02489035903741 73.64166027911274) bank159554 +159555 POINT(41.63433053812734 74.58873065027136) bank159555 +159556 POINT(40.88098426849214 74.77710688550658) bank159556 +159557 POINT(40.19519114757124 73.70607834879127) bank159557 +159558 POINT(40.91551629946827 73.50225309955809) bank159558 +159559 POINT(40.65263050763619 74.33922559587242) bank159559 +159560 POINT(40.558561805520974 74.00325176480395) bank159560 +159561 POINT(40.789247149834736 73.8576554304069) bank159561 +159562 POINT(40.67026312218358 73.8012604424699) bank159562 +159563 POINT(40.83071532184003 73.43655788544567) bank159563 +159564 POINT(40.80925886411641 74.44887886591835) bank159564 +159565 POINT(40.94522963721316 74.37753866464482) bank159565 +159566 POINT(41.47132121995218 74.92616413208722) bank159566 +159567 POINT(40.83825771095127 74.15854386203897) bank159567 +159568 POINT(40.50037123965136 74.37610373922438) bank159568 +159569 POINT(40.67914224538091 74.80581159433329) bank159569 +159570 POINT(40.71473851290516 73.27381113952548) bank159570 +159571 POINT(40.93610309327023 73.51192652129164) bank159571 +159572 POINT(40.86965926577231 73.90680743084592) bank159572 +159573 POINT(40.5274225628464 74.37717338147336) bank159573 +159574 POINT(40.94340828190016 74.78823757071301) bank159574 +159575 POINT(41.21565952632847 74.20199329440521) bank159575 +159576 POINT(41.47322214766182 74.19540408236064) bank159576 +159577 POINT(40.64812064723312 73.46364248748415) bank159577 +159578 POINT(40.37876781782184 74.9299866511933) bank159578 +159579 POINT(41.696841195257754 74.64520418913897) bank159579 +159580 POINT(40.868433843717725 73.8620469676353) bank159580 +159581 POINT(41.491327961724245 74.76123800868352) bank159581 +159582 POINT(40.27593095444032 73.7868170602244) bank159582 +159583 POINT(41.01593706961723 73.54146512716085) bank159583 +159584 POINT(40.70445138976363 73.16590011969366) bank159584 +159585 POINT(40.41317052449788 74.39177915166643) bank159585 +159586 POINT(39.73023841382601 73.2071423170136) bank159586 +159587 POINT(40.602848292491736 73.51172369827258) bank159587 +159588 POINT(41.52602611673017 74.4716808672309) bank159588 +159589 POINT(39.721846175622844 74.25031998174092) bank159589 +159590 POINT(40.09881089392073 74.53254880178925) bank159590 +159591 POINT(41.03324504879665 74.72589615266176) bank159591 +159592 POINT(41.526771704862206 74.78295779850772) bank159592 +159593 POINT(40.13039345309677 74.73339892737683) bank159593 +159594 POINT(39.77109490291843 73.42661226601093) bank159594 +159595 POINT(40.76925540548142 73.87337749125057) bank159595 +159596 POINT(40.8882276907094 73.14885243236756) bank159596 +159597 POINT(40.08065735293599 73.80048081180125) bank159597 +159598 POINT(40.17089338159903 74.198468999719) bank159598 +159599 POINT(40.1548858599788 74.5055012352565) bank159599 +159600 POINT(40.84681409426564 74.6850655289749) bank159600 +159601 POINT(40.912954408367426 74.55792676928203) bank159601 +159602 POINT(40.65658480892025 73.36767863358946) bank159602 +159603 POINT(41.65031234726846 74.52493348578462) bank159603 +159604 POINT(40.179000975559084 73.23817831343351) bank159604 +159605 POINT(40.717615865317704 74.49927585804537) bank159605 +159606 POINT(40.338231441948 74.57901719984189) bank159606 +159607 POINT(39.732029498873914 73.40288896737749) bank159607 +159608 POINT(40.72729983597512 73.57858717273761) bank159608 +159609 POINT(41.18593432757625 73.8194104497673) bank159609 +159610 POINT(41.15756288588834 74.17196025335406) bank159610 +159611 POINT(39.86620558963008 73.42064916765598) bank159611 +159612 POINT(40.65689119130943 73.19415942748581) bank159612 +159613 POINT(39.82545404929113 74.21188737980474) bank159613 +159614 POINT(40.35884233380164 73.03670079117853) bank159614 +159615 POINT(40.60062930530107 74.01557159864858) bank159615 +159616 POINT(41.27389688849157 74.61442186942506) bank159616 +159617 POINT(41.397436652652814 74.49801653743422) bank159617 +159618 POINT(41.1845059551548 73.62335316836703) bank159618 +159619 POINT(40.282411621774116 73.27921958815271) bank159619 +159620 POINT(40.51443725454803 73.36812541703137) bank159620 +159621 POINT(39.939749056001105 74.50329944051218) bank159621 +159622 POINT(39.956166806068154 73.64349420181998) bank159622 +159623 POINT(40.900826772231305 73.99730240514413) bank159623 +159624 POINT(40.56093074296053 73.71771442140378) bank159624 +159625 POINT(40.34171990509671 74.52404682633103) bank159625 +159626 POINT(40.02520781660771 74.61328272438604) bank159626 +159627 POINT(39.80521667715848 74.931166896984) bank159627 +159628 POINT(40.1744719614359 73.74758797397317) bank159628 +159629 POINT(40.43017122246074 73.78028244069449) bank159629 +159630 POINT(41.67445468181049 73.75301820790308) bank159630 +159631 POINT(41.56642277321363 73.31657261745264) bank159631 +159632 POINT(40.450710826986914 74.96186975964261) bank159632 +159633 POINT(41.57950031675332 74.47760720701588) bank159633 +159634 POINT(41.56913192131736 73.0461224717194) bank159634 +159635 POINT(41.53591610603243 73.57713787719469) bank159635 +159636 POINT(40.2422176334474 74.80486785423257) bank159636 +159637 POINT(41.034990491602215 73.71173355498901) bank159637 +159638 POINT(40.5431987563767 74.43087418147223) bank159638 +159639 POINT(40.560651973598986 73.20295867914056) bank159639 +159640 POINT(40.2262482257982 74.802540277722) bank159640 +159641 POINT(41.48175196942171 74.68820409861026) bank159641 +159642 POINT(41.51198201312742 73.33574470929432) bank159642 +159643 POINT(41.20975098129731 73.92151681964607) bank159643 +159644 POINT(41.321770200140925 74.29833016856445) bank159644 +159645 POINT(41.00395715809091 73.53990283119444) bank159645 +159646 POINT(39.81594645339676 73.02464539568405) bank159646 +159647 POINT(40.06400535477924 74.95415910956892) bank159647 +159648 POINT(40.421032252606125 74.13941502577904) bank159648 +159649 POINT(40.47713303014962 74.95091838324304) bank159649 +159650 POINT(40.462846088363115 73.60079645982931) bank159650 +159651 POINT(40.96672828185476 74.16323954196075) bank159651 +159652 POINT(40.20563306156404 74.68626148176212) bank159652 +159653 POINT(40.15972402809765 74.80016933411893) bank159653 +159654 POINT(40.74065628613023 74.40920017552736) bank159654 +159655 POINT(41.3129788995282 74.75823026232194) bank159655 +159656 POINT(40.57325652804122 74.59422428667395) bank159656 +159657 POINT(40.76540413417749 73.4566248515082) bank159657 +159658 POINT(39.913708727506595 74.27290227125289) bank159658 +159659 POINT(39.8835287426751 74.44453101777036) bank159659 +159660 POINT(40.25685058727358 73.15413159759132) bank159660 +159661 POINT(40.02947652994106 73.4400715758756) bank159661 +159662 POINT(41.25309993767292 74.81316112372087) bank159662 +159663 POINT(40.75854925129928 73.64304486780132) bank159663 +159664 POINT(41.45701410770048 74.26695581352618) bank159664 +159665 POINT(41.45822884813943 75.00484624357527) bank159665 +159666 POINT(41.47028089341148 73.3021366446572) bank159666 +159667 POINT(39.75690900629891 73.2337176504071) bank159667 +159668 POINT(40.43733103142651 73.45388236747479) bank159668 +159669 POINT(40.1497174218588 73.05176861919594) bank159669 +159670 POINT(40.78195468657341 74.80680955857393) bank159670 +159671 POINT(41.63621095802594 74.37631626368578) bank159671 +159672 POINT(41.19949730357026 74.0597877188914) bank159672 +159673 POINT(39.87073299253411 74.70606454981487) bank159673 +159674 POINT(41.18612736941416 74.41429751851088) bank159674 +159675 POINT(41.21997013976925 73.0980962542685) bank159675 +159676 POINT(41.02942542295621 73.85318769614005) bank159676 +159677 POINT(41.09249561598391 73.47364458850103) bank159677 +159678 POINT(41.06991929743722 74.32764686092695) bank159678 +159679 POINT(41.192184482917945 73.36094142137492) bank159679 +159680 POINT(40.89953743563691 73.07580849750579) bank159680 +159681 POINT(41.41835782048364 73.10941522663603) bank159681 +159682 POINT(40.1466097290736 73.93746167945962) bank159682 +159683 POINT(41.48151848605709 74.92789709867978) bank159683 +159684 POINT(39.94049374899718 73.85175271078029) bank159684 +159685 POINT(41.2083854956763 73.183524371541) bank159685 +159686 POINT(40.08910670449228 73.11245561188957) bank159686 +159687 POINT(40.38878361331837 74.88286563826595) bank159687 +159688 POINT(39.88068895392923 74.1825851353712) bank159688 +159689 POINT(40.43345790580344 73.95958480117365) bank159689 +159690 POINT(41.265604105350576 73.66063494003983) bank159690 +159691 POINT(41.33749451122966 73.97255977872518) bank159691 +159692 POINT(40.090482142025856 73.04711659774375) bank159692 +159693 POINT(39.992890352092736 74.463922463789) bank159693 +159694 POINT(40.42763292161747 74.01950353334692) bank159694 +159695 POINT(40.62713379937851 73.68959086240342) bank159695 +159696 POINT(40.151443740916456 73.76898453604234) bank159696 +159697 POINT(41.61212096158091 74.14071716524914) bank159697 +159698 POINT(39.863185903820934 73.96473068193676) bank159698 +159699 POINT(40.12540347756953 74.91446555658524) bank159699 +159700 POINT(40.18829069317067 74.7523164383529) bank159700 +159701 POINT(40.439451697804614 74.4051448503476) bank159701 +159702 POINT(40.23447052373113 74.23613339245347) bank159702 +159703 POINT(41.5947340931847 74.31374734343193) bank159703 +159704 POINT(39.884271955247 73.21815168288725) bank159704 +159705 POINT(41.407870936915955 73.61847399439627) bank159705 +159706 POINT(41.47720376529693 73.46804271929098) bank159706 +159707 POINT(41.67389225065654 74.95933808721186) bank159707 +159708 POINT(41.61965798200704 74.48237498777029) bank159708 +159709 POINT(41.10949835874592 74.86545756430019) bank159709 +159710 POINT(40.41946266440235 74.01793768814896) bank159710 +159711 POINT(41.563161284521875 74.85485476302) bank159711 +159712 POINT(40.39751790010779 74.06078953178857) bank159712 +159713 POINT(40.769415285602044 74.31407475237923) bank159713 +159714 POINT(41.090401914248304 74.49290551274743) bank159714 +159715 POINT(40.546891335849665 73.3855123852669) bank159715 +159716 POINT(40.97553951478436 73.18963085073753) bank159716 +159717 POINT(41.51042098925528 74.57934829623068) bank159717 +159718 POINT(40.19225335154977 73.60036392103846) bank159718 +159719 POINT(41.36240807295906 73.66668100370178) bank159719 +159720 POINT(40.594512681822586 73.61888095001561) bank159720 +159721 POINT(40.625049538683 74.0119757006782) bank159721 +159722 POINT(39.964970798674685 74.2566164046333) bank159722 +159723 POINT(41.567854530191106 74.55337934483309) bank159723 +159724 POINT(40.10371772909726 73.35290929489645) bank159724 +159725 POINT(41.610696132431 73.07485104869905) bank159725 +159726 POINT(41.07759345779646 74.8183234972077) bank159726 +159727 POINT(41.39356419384956 74.22642031843219) bank159727 +159728 POINT(39.76651970088739 73.79124324951981) bank159728 +159729 POINT(41.22693925563977 74.97583954766503) bank159729 +159730 POINT(41.583047853561254 74.10575894715484) bank159730 +159731 POINT(41.053269728595744 73.84494741458802) bank159731 +159732 POINT(40.522420432550454 73.87335263243075) bank159732 +159733 POINT(40.553876280019956 74.83704482972266) bank159733 +159734 POINT(40.385125281612886 74.22138255168471) bank159734 +159735 POINT(39.96295479466075 74.20078263251769) bank159735 +159736 POINT(41.56151237140176 74.42378864189175) bank159736 +159737 POINT(40.23476390115732 73.21828446948595) bank159737 +159738 POINT(40.36086961799707 73.71456279100032) bank159738 +159739 POINT(40.17566269883577 73.67939011611286) bank159739 +159740 POINT(41.66543924668358 74.22387163528879) bank159740 +159741 POINT(39.76251828177782 73.01016395145705) bank159741 +159742 POINT(41.64671670401539 73.31593665513333) bank159742 +159743 POINT(40.043613861257654 74.40174397943028) bank159743 +159744 POINT(41.116105126283706 73.02323209912946) bank159744 +159745 POINT(41.631583742023594 73.09377145989758) bank159745 +159746 POINT(40.242833739964105 73.7947518105206) bank159746 +159747 POINT(41.5726638552818 74.71036823688543) bank159747 +159748 POINT(39.778401452146085 73.24573053843264) bank159748 +159749 POINT(41.65073018351754 74.40817420306479) bank159749 +159750 POINT(40.84700110265762 73.09966773448342) bank159750 +159751 POINT(40.09030461337863 73.8826476520715) bank159751 +159752 POINT(41.686763694385526 73.47514456318646) bank159752 +159753 POINT(40.17892902374592 74.36731430166753) bank159753 +159754 POINT(40.90580571996133 74.01481988931246) bank159754 +159755 POINT(40.7351968133284 73.4477294280504) bank159755 +159756 POINT(40.66085523977952 73.51668635124847) bank159756 +159757 POINT(40.43689947339052 74.46629389592765) bank159757 +159758 POINT(40.02693135183453 74.0097138034474) bank159758 +159759 POINT(40.83725886986583 74.80925133659238) bank159759 +159760 POINT(41.48412782917029 73.87226854120205) bank159760 +159761 POINT(39.85535414127954 73.1034125913477) bank159761 +159762 POINT(40.224707687458455 74.36391770667946) bank159762 +159763 POINT(40.062046553253154 73.95882968561378) bank159763 +159764 POINT(39.87404615336725 73.59065561255947) bank159764 +159765 POINT(40.17795883581733 74.10134272378355) bank159765 +159766 POINT(41.700829777604085 73.70621521615237) bank159766 +159767 POINT(40.33902721614379 73.62595359157989) bank159767 +159768 POINT(41.34071749567053 74.06931785380397) bank159768 +159769 POINT(40.28237424570877 74.3288589000661) bank159769 +159770 POINT(40.15892347013256 74.6988133497598) bank159770 +159771 POINT(41.14658109082869 73.59702141363574) bank159771 +159772 POINT(41.14538247740778 74.51755703333988) bank159772 +159773 POINT(40.465259347183 73.6537832816608) bank159773 +159774 POINT(41.56937707204035 73.32290122033069) bank159774 +159775 POINT(41.11213262849088 73.89566822630962) bank159775 +159776 POINT(40.72991090929696 74.00508636592198) bank159776 +159777 POINT(41.6932366326715 74.57387356556659) bank159777 +159778 POINT(40.33437165463586 73.47865873220688) bank159778 +159779 POINT(40.123171939660665 74.62017804705484) bank159779 +159780 POINT(39.781225021971665 74.54879614127752) bank159780 +159781 POINT(39.906051543699654 73.91984984577245) bank159781 +159782 POINT(41.397266306410515 73.15314874517317) bank159782 +159783 POINT(40.08496156107055 73.04451217314332) bank159783 +159784 POINT(41.447014989139866 73.3521399592567) bank159784 +159785 POINT(40.73565409675876 74.04474327146781) bank159785 +159786 POINT(39.72972695766723 73.89865339161219) bank159786 +159787 POINT(39.990740051186236 74.56000347152936) bank159787 +159788 POINT(40.76070860668952 73.8876693111092) bank159788 +159789 POINT(41.17832867020083 74.09332452179949) bank159789 +159790 POINT(40.777003411127396 73.97441393644242) bank159790 +159791 POINT(41.10468690404572 73.14743565585418) bank159791 +159792 POINT(40.04563226917753 74.44944807712763) bank159792 +159793 POINT(39.79631458244732 73.84877075149646) bank159793 +159794 POINT(41.224123010201 74.82945405046658) bank159794 +159795 POINT(41.39897960990092 74.34145989774213) bank159795 +159796 POINT(40.97592078803895 74.31811037433567) bank159796 +159797 POINT(41.18272942941272 73.38931643773093) bank159797 +159798 POINT(40.91074676222398 74.50878147680207) bank159798 +159799 POINT(40.12752477035906 74.43601075848788) bank159799 +159800 POINT(40.20446155518198 73.76055638090017) bank159800 +159801 POINT(41.3228837738666 74.10490780422995) bank159801 +159802 POINT(40.79680896651243 73.87748710418221) bank159802 +159803 POINT(41.13539418974842 74.01295205967362) bank159803 +159804 POINT(41.16770294439385 73.1153892180914) bank159804 +159805 POINT(39.73075265781114 73.49147001713402) bank159805 +159806 POINT(41.624309002538915 73.67056374707398) bank159806 +159807 POINT(40.2288646277814 73.327111912937) bank159807 +159808 POINT(39.77787518353916 73.39616889475246) bank159808 +159809 POINT(40.7879277337741 74.17484871696769) bank159809 +159810 POINT(40.302783502113996 73.25642003799761) bank159810 +159811 POINT(40.72555974195605 73.8631034863287) bank159811 +159812 POINT(41.4512042166591 74.64424365508368) bank159812 +159813 POINT(41.15773965524191 73.01775251465139) bank159813 +159814 POINT(41.650136906021075 74.16746200581376) bank159814 +159815 POINT(41.13048938478832 73.73485002899507) bank159815 +159816 POINT(40.36070176530995 74.74912269737489) bank159816 +159817 POINT(40.82305372452733 73.85392214059169) bank159817 +159818 POINT(41.636687596560456 73.62201240700159) bank159818 +159819 POINT(39.781418758612055 74.43100494711078) bank159819 +159820 POINT(41.34617985394419 74.38702856123648) bank159820 +159821 POINT(40.11166644246785 74.65996153687925) bank159821 +159822 POINT(40.413521987328814 73.34652414270525) bank159822 +159823 POINT(40.09949638817915 73.76626807194995) bank159823 +159824 POINT(39.780154521807574 74.65752505587466) bank159824 +159825 POINT(41.21314627137208 73.95180860585357) bank159825 +159826 POINT(40.7055163224088 74.9357876397184) bank159826 +159827 POINT(41.37185550981968 74.61887135446025) bank159827 +159828 POINT(41.25249317834987 74.74949941213683) bank159828 +159829 POINT(41.346618138322086 73.1114706806048) bank159829 +159830 POINT(40.93823288697029 73.4278777960178) bank159830 +159831 POINT(39.71533202558873 74.89439273822576) bank159831 +159832 POINT(40.729127785923076 73.83196002800736) bank159832 +159833 POINT(40.67428907175523 73.86603965957597) bank159833 +159834 POINT(39.717332605484955 73.80510618689723) bank159834 +159835 POINT(40.28808088138514 73.19798507455062) bank159835 +159836 POINT(40.579728633736465 73.51158965829822) bank159836 +159837 POINT(40.48446040776778 74.799192136694) bank159837 +159838 POINT(40.122491563035155 74.93512457484957) bank159838 +159839 POINT(41.10728742480286 74.0697824166667) bank159839 +159840 POINT(41.363731700747884 73.15336184436609) bank159840 +159841 POINT(41.143412965000906 74.39847482095898) bank159841 +159842 POINT(40.58981508466744 74.58647363715335) bank159842 +159843 POINT(41.143516597418476 73.9167787142451) bank159843 +159844 POINT(40.78288185315424 73.40656882264278) bank159844 +159845 POINT(40.096557594925216 74.04475508198323) bank159845 +159846 POINT(41.530296938068304 73.50943613295152) bank159846 +159847 POINT(40.06375631647199 73.7589852494622) bank159847 +159848 POINT(41.390591660385354 73.91510427201382) bank159848 +159849 POINT(41.30251206405551 74.53103828744267) bank159849 +159850 POINT(40.45550575171469 74.15445439856971) bank159850 +159851 POINT(39.92710994654476 73.37775384308334) bank159851 +159852 POINT(39.897456954646756 74.99032805333341) bank159852 +159853 POINT(40.182704636841265 74.35769378973441) bank159853 +159854 POINT(41.2153949749499 73.29815084941094) bank159854 +159855 POINT(40.157183027921384 74.74047605042313) bank159855 +159856 POINT(41.46468156208024 73.3442124548975) bank159856 +159857 POINT(40.71133275577181 73.71769633268111) bank159857 +159858 POINT(40.40420836457858 74.35685295217851) bank159858 +159859 POINT(41.6732334653509 73.70717374447406) bank159859 +159860 POINT(40.756894943467316 73.92035707699526) bank159860 +159861 POINT(40.12776549481127 73.48885506983851) bank159861 +159862 POINT(41.11094381138634 74.28893970740056) bank159862 +159863 POINT(41.3902830057624 73.95179496150077) bank159863 +159864 POINT(39.75523085547938 74.54432117503576) bank159864 +159865 POINT(40.914284294231216 74.2590948592164) bank159865 +159866 POINT(41.54907545015193 74.77375425818236) bank159866 +159867 POINT(41.50778274499191 74.0130238110438) bank159867 +159868 POINT(41.51807453563442 74.75395017673097) bank159868 +159869 POINT(41.29116629396842 74.85644371128917) bank159869 +159870 POINT(41.10248040078936 74.07485470175617) bank159870 +159871 POINT(40.72664725500023 73.9974130697877) bank159871 +159872 POINT(41.18904330990546 73.21403542032817) bank159872 +159873 POINT(41.32797821173677 73.82331264887158) bank159873 +159874 POINT(39.95066613077224 74.09214148263368) bank159874 +159875 POINT(40.75846309689636 73.67403167031897) bank159875 +159876 POINT(41.45112489787736 74.18774484178513) bank159876 +159877 POINT(40.30096268709739 74.26435673905632) bank159877 +159878 POINT(40.831952578018424 74.07478739842894) bank159878 +159879 POINT(41.55709204599036 74.17254231693411) bank159879 +159880 POINT(41.21793283579513 73.84301875056413) bank159880 +159881 POINT(39.959936638385884 74.0832447734265) bank159881 +159882 POINT(41.413199442899106 73.93562768749953) bank159882 +159883 POINT(40.54353791244641 73.40356727473083) bank159883 +159884 POINT(41.08821730411615 73.76467304520831) bank159884 +159885 POINT(40.24712093531161 73.26733153095644) bank159885 +159886 POINT(40.533974889490565 73.30251372484273) bank159886 +159887 POINT(40.745367364707235 74.77006682350293) bank159887 +159888 POINT(41.206651173802875 74.87494030363662) bank159888 +159889 POINT(40.58631029049144 74.90888251539815) bank159889 +159890 POINT(40.5818437154122 73.12259515660018) bank159890 +159891 POINT(41.03676927308626 73.26638408697434) bank159891 +159892 POINT(40.58349592762849 74.53484283756006) bank159892 +159893 POINT(41.37770321628377 74.21794880322247) bank159893 +159894 POINT(40.16678892061443 73.67565416067772) bank159894 +159895 POINT(40.630565038674504 73.29717457549927) bank159895 +159896 POINT(41.3155932612378 73.4643927819231) bank159896 +159897 POINT(40.92807106398922 73.32813333046938) bank159897 +159898 POINT(41.01224289489489 73.05442418641638) bank159898 +159899 POINT(41.04987384605702 73.21110984072834) bank159899 +159900 POINT(41.01702348122094 74.37156056423743) bank159900 +159901 POINT(41.31779273838126 74.35392804742857) bank159901 +159902 POINT(40.72702948236546 73.94143359239942) bank159902 +159903 POINT(40.07919803465741 74.65021974387705) bank159903 +159904 POINT(39.96835136860627 74.43716697949239) bank159904 +159905 POINT(39.78456362731634 73.40054306196481) bank159905 +159906 POINT(40.201804968001746 73.7432801614092) bank159906 +159907 POINT(41.29817354679828 74.68856724004416) bank159907 +159908 POINT(40.03416245521842 73.48021770720129) bank159908 +159909 POINT(41.208388922196704 74.51918511312803) bank159909 +159910 POINT(39.85810915743353 74.83282431602548) bank159910 +159911 POINT(40.84599447103203 73.54513023992305) bank159911 +159912 POINT(40.895953020161215 74.20441708490307) bank159912 +159913 POINT(40.089183405397 74.7536446172048) bank159913 +159914 POINT(40.469769303122476 73.1784420038953) bank159914 +159915 POINT(40.42391955115309 74.07413953645403) bank159915 +159916 POINT(39.73792008779108 74.67201295817107) bank159916 +159917 POINT(41.20233185944517 73.4717722480882) bank159917 +159918 POINT(41.543310485582424 73.61034175576297) bank159918 +159919 POINT(40.964084834300905 73.21554612161734) bank159919 +159920 POINT(41.593307017185886 74.31322645246806) bank159920 +159921 POINT(41.37852247392423 74.83250377837953) bank159921 +159922 POINT(40.20621010026911 74.33876051068947) bank159922 +159923 POINT(40.20095766242282 74.90076027884817) bank159923 +159924 POINT(39.78166099218618 74.00587752204983) bank159924 +159925 POINT(41.544573536290194 73.72776396476256) bank159925 +159926 POINT(40.150208804803036 74.66282367766382) bank159926 +159927 POINT(41.56822540158495 74.73956565028726) bank159927 +159928 POINT(39.782937711805424 74.2656617890222) bank159928 +159929 POINT(40.363508514513214 73.09725790901562) bank159929 +159930 POINT(40.8583304056057 73.86114626424808) bank159930 +159931 POINT(41.151776547090044 74.83059071623062) bank159931 +159932 POINT(41.28611871199931 74.48584081699623) bank159932 +159933 POINT(40.71546767550748 74.35213565726491) bank159933 +159934 POINT(41.430269788661306 74.99880522992696) bank159934 +159935 POINT(40.77908298151599 74.10465177629692) bank159935 +159936 POINT(40.68892858446904 73.70645840760068) bank159936 +159937 POINT(40.03251426842222 74.77032815491022) bank159937 +159938 POINT(40.34184467877758 73.2107789023288) bank159938 +159939 POINT(40.505391636477285 74.0853435189203) bank159939 +159940 POINT(41.6075008208381 73.47581018237213) bank159940 +159941 POINT(40.08487250840143 73.9979506968453) bank159941 +159942 POINT(40.876753399788676 73.3694789674903) bank159942 +159943 POINT(41.639811029093764 74.24103401829514) bank159943 +159944 POINT(40.31056970804266 73.65285626499613) bank159944 +159945 POINT(40.60779698162319 73.5709642299355) bank159945 +159946 POINT(40.091010920639775 74.2491891867564) bank159946 +159947 POINT(41.51656374898351 73.05660426058853) bank159947 +159948 POINT(41.70604500158131 74.07501723323831) bank159948 +159949 POINT(39.86259459997662 73.7395191527779) bank159949 +159950 POINT(41.056161616056414 73.49552787757236) bank159950 +159951 POINT(41.20661137000468 74.4973416787754) bank159951 +159952 POINT(41.20612800994453 73.52497377086787) bank159952 +159953 POINT(41.653703637738225 74.19698055106764) bank159953 +159954 POINT(40.237215180895184 73.01513217035377) bank159954 +159955 POINT(41.32172876836043 73.95318928290423) bank159955 +159956 POINT(41.12245874409839 73.06723886237916) bank159956 +159957 POINT(41.32133497084428 74.95602226761466) bank159957 +159958 POINT(40.09984636404458 73.09755771136366) bank159958 +159959 POINT(40.94809735252336 74.38675032537014) bank159959 +159960 POINT(41.025454571086755 73.8003051165684) bank159960 +159961 POINT(39.94250156045911 74.81965546935434) bank159961 +159962 POINT(40.036044362439945 74.86553522376299) bank159962 +159963 POINT(39.95347884403635 74.79876377367358) bank159963 +159964 POINT(40.50917296023944 74.2856636652959) bank159964 +159965 POINT(41.68393362258126 73.85358577073147) bank159965 +159966 POINT(41.585332527178146 73.85681454521227) bank159966 +159967 POINT(40.82859134427064 74.67224484892499) bank159967 +159968 POINT(40.12728372281223 73.32549602762677) bank159968 +159969 POINT(40.30226633404334 73.53625344178289) bank159969 +159970 POINT(40.569999057502706 74.91711999303952) bank159970 +159971 POINT(40.89007664096308 73.30746037531078) bank159971 +159972 POINT(41.3863315139216 74.74227709958186) bank159972 +159973 POINT(40.60390203045362 73.34444621579684) bank159973 +159974 POINT(40.99674459884397 74.68434397247194) bank159974 +159975 POINT(41.080015976210085 74.41678055488543) bank159975 +159976 POINT(40.17641230294972 74.16972030891112) bank159976 +159977 POINT(40.94413639652262 74.20230170015354) bank159977 +159978 POINT(41.59513404633446 74.07653909257373) bank159978 +159979 POINT(40.26204734346012 73.60179953685704) bank159979 +159980 POINT(39.99943440223936 73.37620049022887) bank159980 +159981 POINT(40.80538501730036 73.03687887910658) bank159981 +159982 POINT(41.33123087866826 73.22343149018796) bank159982 +159983 POINT(40.99858855202625 73.06242601553252) bank159983 +159984 POINT(40.024505261175605 74.38397332563565) bank159984 +159985 POINT(41.50539772660997 73.70146882795689) bank159985 +159986 POINT(41.145725541453515 74.19520430699562) bank159986 +159987 POINT(41.42755014924798 74.44327416245919) bank159987 +159988 POINT(41.04844187503073 73.60136942778246) bank159988 +159989 POINT(40.31188972484476 74.61989140123997) bank159989 +159990 POINT(40.98612461724113 74.66146738896434) bank159990 +159991 POINT(41.19581508500638 74.04273202439964) bank159991 +159992 POINT(41.31852410291382 73.65842941372891) bank159992 +159993 POINT(41.38561856900442 73.81715193759726) bank159993 +159994 POINT(40.007918379412466 74.56077952508933) bank159994 +159995 POINT(41.112368794674076 74.40019459341784) bank159995 +159996 POINT(40.438336815642586 74.59597338797404) bank159996 +159997 POINT(40.76668944731308 74.95661618128527) bank159997 +159998 POINT(40.41148923057172 74.33767943512835) bank159998 +159999 POINT(40.21164972554913 74.96167310852468) bank159999 +160000 POINT(39.71729108683428 73.72665399226221) bank160000 +160001 POINT(41.27037847661497 73.2273966760589) bank160001 +160002 POINT(39.89068911181718 74.54585215345043) bank160002 +160003 POINT(40.986526706679996 74.90345344830592) bank160003 +160004 POINT(40.683527851932446 73.6037933440802) bank160004 +160005 POINT(39.79551385484506 73.2480430488066) bank160005 +160006 POINT(40.29038083161498 74.34185747592434) bank160006 +160007 POINT(40.79686256815879 74.67186826721844) bank160007 +160008 POINT(40.15886415064687 73.56666661591319) bank160008 +160009 POINT(41.63575276797078 74.57468604600074) bank160009 +160010 POINT(41.64394777808118 74.88437385884687) bank160010 +160011 POINT(41.10505596479915 73.98041093910017) bank160011 +160012 POINT(40.18742675981922 73.23289594703306) bank160012 +160013 POINT(40.42930954827432 73.30175056444784) bank160013 +160014 POINT(41.25088473965586 74.97499156831081) bank160014 +160015 POINT(41.63773197008827 74.84704711228986) bank160015 +160016 POINT(41.660195351659596 73.71603587515301) bank160016 +160017 POINT(41.50642405766186 73.349284810336) bank160017 +160018 POINT(40.67745435543559 73.51008218839652) bank160018 +160019 POINT(40.856094325822994 73.01161034628582) bank160019 +160020 POINT(40.97015453703608 74.27294643591677) bank160020 +160021 POINT(40.11745644006961 73.06880737340204) bank160021 +160022 POINT(40.42250884796015 73.58761064080278) bank160022 +160023 POINT(40.510075811079055 74.46049026612852) bank160023 +160024 POINT(40.02607309102417 73.28748356708945) bank160024 +160025 POINT(40.11104729888303 73.16911446983812) bank160025 +160026 POINT(41.423403560713055 73.72409997156313) bank160026 +160027 POINT(41.65151872286076 73.12751672715488) bank160027 +160028 POINT(40.12577223653161 73.29149134175361) bank160028 +160029 POINT(40.30918186116654 74.97739411336565) bank160029 +160030 POINT(41.03020431580739 73.08130241084335) bank160030 +160031 POINT(41.381753611101146 74.73408151864726) bank160031 +160032 POINT(41.19309534521803 73.22603361119347) bank160032 +160033 POINT(40.088961154249375 73.1556450738105) bank160033 +160034 POINT(41.69891989382773 74.49966733214231) bank160034 +160035 POINT(40.50061536012963 73.68200735714058) bank160035 +160036 POINT(41.318509064601706 73.76644229522044) bank160036 +160037 POINT(40.55419667885547 73.35304836667119) bank160037 +160038 POINT(40.07542976123401 74.46968502196619) bank160038 +160039 POINT(41.45144125662988 74.427745176012) bank160039 +160040 POINT(40.92429087224531 74.61101914667907) bank160040 +160041 POINT(40.97065278044247 73.05843687635448) bank160041 +160042 POINT(40.030255136606016 73.44766713288197) bank160042 +160043 POINT(40.35659028998855 74.59986671412005) bank160043 +160044 POINT(40.46751943018514 73.92055096849697) bank160044 +160045 POINT(39.83193699328915 73.49475481716469) bank160045 +160046 POINT(41.64330516466555 73.51889747185322) bank160046 +160047 POINT(39.8686294438786 74.25865918874729) bank160047 +160048 POINT(40.44550742754186 73.42716369365246) bank160048 +160049 POINT(39.862921028081054 73.1565099026043) bank160049 +160050 POINT(40.97412766286128 73.6313171837527) bank160050 +160051 POINT(40.23915552075928 73.50432325528737) bank160051 +160052 POINT(41.631220373225 74.72476776694818) bank160052 +160053 POINT(40.99290192706989 73.44886365674252) bank160053 +160054 POINT(41.10476232603298 73.00893952022027) bank160054 +160055 POINT(41.58047573833575 74.55205923738698) bank160055 +160056 POINT(41.473449781584684 74.39356255172741) bank160056 +160057 POINT(40.638860277562635 74.29863488896413) bank160057 +160058 POINT(40.639199585286 74.9955988998978) bank160058 +160059 POINT(41.320937539476084 73.48157949834379) bank160059 +160060 POINT(41.11403867401228 74.67371666820655) bank160060 +160061 POINT(41.30548965574345 73.16395071716639) bank160061 +160062 POINT(40.45292727419279 74.87234345347177) bank160062 +160063 POINT(40.621217710488786 74.73947396068812) bank160063 +160064 POINT(41.061367807473964 73.92263652998092) bank160064 +160065 POINT(40.50521226166368 74.0144703750994) bank160065 +160066 POINT(40.882983377848284 73.05282531989292) bank160066 +160067 POINT(41.27197068262061 73.72217867453547) bank160067 +160068 POINT(39.74023355635398 74.65289446106698) bank160068 +160069 POINT(39.94666420103663 74.67286147018588) bank160069 +160070 POINT(41.4230184841983 74.73196570044959) bank160070 +160071 POINT(41.205215135949665 74.6432434092301) bank160071 +160072 POINT(40.83435060023187 73.77667931601223) bank160072 +160073 POINT(41.31181142671818 74.72297363329609) bank160073 +160074 POINT(41.07613965558423 73.86997670401662) bank160074 +160075 POINT(40.74833269962685 73.88581148023651) bank160075 +160076 POINT(41.11486033064144 73.45209986339158) bank160076 +160077 POINT(41.51256950411125 73.28197593618455) bank160077 +160078 POINT(40.18926487930644 73.99665091332606) bank160078 +160079 POINT(40.99407148289816 74.95334423701253) bank160079 +160080 POINT(39.79519961994631 74.26496448024734) bank160080 +160081 POINT(40.945385747286146 74.71695342359517) bank160081 +160082 POINT(41.14808941580919 73.60878876774368) bank160082 +160083 POINT(39.87482348542977 73.34413060031056) bank160083 +160084 POINT(41.63453416131513 74.13331871554172) bank160084 +160085 POINT(41.39789275418474 73.78245084033402) bank160085 +160086 POINT(40.7679771155614 74.4447221507474) bank160086 +160087 POINT(40.46644288332467 74.91219299371913) bank160087 +160088 POINT(40.84882977574307 73.40666163615876) bank160088 +160089 POINT(40.77518246636926 73.23712789925702) bank160089 +160090 POINT(41.60360965388931 74.75346590067103) bank160090 +160091 POINT(41.36011513398073 73.53041271803525) bank160091 +160092 POINT(41.51842481407225 73.53478514891351) bank160092 +160093 POINT(40.17059166475801 74.60390545525969) bank160093 +160094 POINT(41.19471735932675 74.64824776646638) bank160094 +160095 POINT(39.728957609287484 74.51936082394212) bank160095 +160096 POINT(40.533153889301495 73.2674493330887) bank160096 +160097 POINT(40.264102945844634 73.69800417652951) bank160097 +160098 POINT(40.240948459932994 74.98736098351391) bank160098 +160099 POINT(40.72123088643769 74.93499193039388) bank160099 +160100 POINT(40.877813366867834 74.90026235891692) bank160100 +160101 POINT(40.39910138600544 73.87012097254991) bank160101 +160102 POINT(40.521086226568585 73.6172195846476) bank160102 +160103 POINT(40.8227051419254 74.02176444207426) bank160103 +160104 POINT(40.58629129719746 74.74880865768189) bank160104 +160105 POINT(41.11854231624874 74.39713869007856) bank160105 +160106 POINT(40.94941061040264 73.16001239366447) bank160106 +160107 POINT(40.15188205099912 74.1993806674324) bank160107 +160108 POINT(39.87490680979278 73.80185602617516) bank160108 +160109 POINT(40.51598312474076 73.4339528961772) bank160109 +160110 POINT(40.98218526356722 73.87268435727167) bank160110 +160111 POINT(40.21913462704751 74.47651153081597) bank160111 +160112 POINT(40.43882188586673 73.89496387787511) bank160112 +160113 POINT(40.63025240349467 74.97701978468262) bank160113 +160114 POINT(41.26821160628231 73.16751053265969) bank160114 +160115 POINT(40.878067464758765 74.61248172765917) bank160115 +160116 POINT(40.79724773412719 75.00363444297791) bank160116 +160117 POINT(40.8094490451529 74.06180881984025) bank160117 +160118 POINT(40.23409004984872 74.16364429423177) bank160118 +160119 POINT(40.552027332687324 73.7693658709903) bank160119 +160120 POINT(40.21353568627961 74.46141786774358) bank160120 +160121 POINT(40.97350744399769 74.30288999788858) bank160121 +160122 POINT(41.160254982629304 73.95378183955422) bank160122 +160123 POINT(40.5496352987195 74.89467451162854) bank160123 +160124 POINT(40.22258174512447 74.68972483003854) bank160124 +160125 POINT(41.65587724194984 73.12671176369078) bank160125 +160126 POINT(40.52022730150131 74.38043093682357) bank160126 +160127 POINT(40.844548940222055 74.4535920085696) bank160127 +160128 POINT(41.327135751688985 73.84048570052532) bank160128 +160129 POINT(41.672335146089445 73.18697006969956) bank160129 +160130 POINT(41.51214726722807 73.31648764234473) bank160130 +160131 POINT(41.19824459649265 73.86016107050288) bank160131 +160132 POINT(41.55386713931265 74.9300626587852) bank160132 +160133 POINT(41.15723132456404 74.49446833256245) bank160133 +160134 POINT(41.132479520955386 73.12356442694976) bank160134 +160135 POINT(40.52803200830323 74.70119285116428) bank160135 +160136 POINT(39.72478447233913 74.9639248385509) bank160136 +160137 POINT(40.25688576557705 73.10217017201015) bank160137 +160138 POINT(41.698540911409786 73.43835007305717) bank160138 +160139 POINT(39.742197813778716 73.7226068044655) bank160139 +160140 POINT(41.603254293050796 74.02035897624846) bank160140 +160141 POINT(40.78312707927757 73.90124735518052) bank160141 +160142 POINT(40.22727726692785 73.14685629109955) bank160142 +160143 POINT(41.27324197384501 74.40825902668239) bank160143 +160144 POINT(41.53404037503786 74.18528327717169) bank160144 +160145 POINT(40.79893438127269 74.54528312015445) bank160145 +160146 POINT(40.41250779512262 74.76875929462145) bank160146 +160147 POINT(41.63077931739467 73.43525680440757) bank160147 +160148 POINT(40.61047624890063 73.16229686358807) bank160148 +160149 POINT(41.55758982178421 74.65189547784492) bank160149 +160150 POINT(40.8760429082248 73.87159321070187) bank160150 +160151 POINT(39.85586866522998 73.01849844172672) bank160151 +160152 POINT(40.09575133157355 74.27489859840792) bank160152 +160153 POINT(41.2857477203129 73.81854858938988) bank160153 +160154 POINT(41.61644503522453 74.92953605449605) bank160154 +160155 POINT(39.99905644320548 74.2092979086295) bank160155 +160156 POINT(41.388636315701824 74.81956572308671) bank160156 +160157 POINT(39.775118104819335 73.38441102866621) bank160157 +160158 POINT(40.9198955565798 73.77035597965708) bank160158 +160159 POINT(40.926454243140356 74.74353228027367) bank160159 +160160 POINT(40.68583418595717 73.2357678823746) bank160160 +160161 POINT(41.459945625692775 73.33549640002808) bank160161 +160162 POINT(40.57478758519903 74.71628702434262) bank160162 +160163 POINT(40.78903134067767 74.9151365368836) bank160163 +160164 POINT(40.95529474726075 73.63089282109281) bank160164 +160165 POINT(40.04551872399265 73.42532370204547) bank160165 +160166 POINT(40.477751276543124 73.12986020957663) bank160166 +160167 POINT(40.344264351022986 74.74822963401779) bank160167 +160168 POINT(41.26154745611386 73.22939611327962) bank160168 +160169 POINT(40.08050164825776 74.75138985357613) bank160169 +160170 POINT(41.20779822016444 73.29992955952595) bank160170 +160171 POINT(40.13836569768226 74.93172216038313) bank160171 +160172 POINT(41.31698250170642 73.88273836040283) bank160172 +160173 POINT(40.02321725727785 73.85805718193411) bank160173 +160174 POINT(41.38118622903606 74.83702042059784) bank160174 +160175 POINT(40.82783330424045 73.99679058399305) bank160175 +160176 POINT(40.53729661764804 73.53344598763746) bank160176 +160177 POINT(40.948828476939994 73.69226982843709) bank160177 +160178 POINT(40.50719475405683 73.07649311979492) bank160178 +160179 POINT(40.72640603974273 74.07969522467332) bank160179 +160180 POINT(40.416959287822664 73.5746998515104) bank160180 +160181 POINT(40.687108157632736 73.43228361359002) bank160181 +160182 POINT(40.9205832630522 73.46527373907723) bank160182 +160183 POINT(40.77077585876302 74.34735777998827) bank160183 +160184 POINT(41.62569704212189 74.14248110471868) bank160184 +160185 POINT(39.93402036413078 73.20734952134353) bank160185 +160186 POINT(41.60965208611562 73.47125761944669) bank160186 +160187 POINT(41.386194925139904 74.83653221278622) bank160187 +160188 POINT(39.924264804450964 74.98247016840122) bank160188 +160189 POINT(41.424446916809266 74.21728012336693) bank160189 +160190 POINT(41.621998861890674 73.84177046569967) bank160190 +160191 POINT(41.48618515823395 73.93320932909776) bank160191 +160192 POINT(40.52694015307697 74.36826645738951) bank160192 +160193 POINT(40.18281447098131 73.56652285960877) bank160193 +160194 POINT(40.95638260494072 74.27170127932318) bank160194 +160195 POINT(40.94516150383281 74.72927385213487) bank160195 +160196 POINT(39.747426015464065 73.29011255486314) bank160196 +160197 POINT(39.80035605770853 74.35658511966155) bank160197 +160198 POINT(40.74008917632997 74.92563227219374) bank160198 +160199 POINT(41.15367785016616 73.30112392604565) bank160199 +160200 POINT(41.5042218283145 74.92133321524784) bank160200 +160201 POINT(41.132991721382425 74.91316932628538) bank160201 +160202 POINT(41.048652441613065 74.73327994759181) bank160202 +160203 POINT(40.245906163198065 73.70647867529144) bank160203 +160204 POINT(40.17415913203276 73.79768635436749) bank160204 +160205 POINT(41.41230333463694 74.41057659262846) bank160205 +160206 POINT(41.30260895510843 74.19334815734346) bank160206 +160207 POINT(41.06800887493163 73.96799100216614) bank160207 +160208 POINT(41.00811267133754 74.34128186889453) bank160208 +160209 POINT(40.651957733402405 74.52272866314777) bank160209 +160210 POINT(39.778253102839805 73.96883017322176) bank160210 +160211 POINT(41.672742401685674 74.97344613013549) bank160211 +160212 POINT(40.48586048243084 74.5447351995646) bank160212 +160213 POINT(41.02830041726216 74.82242069996214) bank160213 +160214 POINT(41.1971348616774 74.18382772333395) bank160214 +160215 POINT(41.47382762519389 74.44281206042326) bank160215 +160216 POINT(41.08211726157393 73.67053761853668) bank160216 +160217 POINT(41.0211603148481 74.10141544425535) bank160217 +160218 POINT(39.95363371577053 74.9173154166936) bank160218 +160219 POINT(40.42790708811599 74.93982431637147) bank160219 +160220 POINT(41.673677339522506 73.84976572815648) bank160220 +160221 POINT(41.61284399949466 73.35388314053208) bank160221 +160222 POINT(41.08648653774677 74.10713280079477) bank160222 +160223 POINT(41.278018240784746 73.14839202813953) bank160223 +160224 POINT(40.46435337309392 73.74367357717105) bank160224 +160225 POINT(40.56589384610672 73.34071113600552) bank160225 +160226 POINT(40.930225046791485 74.1314228501979) bank160226 +160227 POINT(40.25762824125905 73.49654757613084) bank160227 +160228 POINT(41.22574510779894 74.37916462911457) bank160228 +160229 POINT(40.12688822571315 74.81302685687218) bank160229 +160230 POINT(41.615598291565526 73.80537812038374) bank160230 +160231 POINT(41.61156483139768 74.15838402131313) bank160231 +160232 POINT(40.58033284328941 74.7596661993968) bank160232 +160233 POINT(40.06388414552536 73.39197246675191) bank160233 +160234 POINT(39.873237126000525 73.20007893856021) bank160234 +160235 POINT(41.687776719139464 74.23854860313837) bank160235 +160236 POINT(40.27119662473237 74.9877723652052) bank160236 +160237 POINT(40.66275294915274 73.9547187433839) bank160237 +160238 POINT(40.44804797118396 73.33316454473824) bank160238 +160239 POINT(40.01203140871146 74.81815454245894) bank160239 +160240 POINT(41.44419036851366 73.15513356693205) bank160240 +160241 POINT(40.851326265537885 74.28814222658058) bank160241 +160242 POINT(40.938485364585645 73.94250034319222) bank160242 +160243 POINT(40.26526337586097 73.88039289136672) bank160243 +160244 POINT(40.90388403565424 74.62245677761904) bank160244 +160245 POINT(39.75757440868528 74.05943631830121) bank160245 +160246 POINT(40.86957264200598 73.98064716449191) bank160246 +160247 POINT(39.9829309900903 74.13214186781941) bank160247 +160248 POINT(40.558850421565005 74.06508662626534) bank160248 +160249 POINT(41.474434228622414 74.49884083569869) bank160249 +160250 POINT(40.33373034913061 74.74055944789465) bank160250 +160251 POINT(41.074253022710806 74.89613614605278) bank160251 +160252 POINT(39.89165103624308 74.77051722678496) bank160252 +160253 POINT(40.217633058065665 74.3540263399292) bank160253 +160254 POINT(40.33438763524781 73.55808878934387) bank160254 +160255 POINT(40.42480697811919 74.51567832559957) bank160255 +160256 POINT(41.343964318873944 74.17140615222172) bank160256 +160257 POINT(41.6235385469517 73.19654589814837) bank160257 +160258 POINT(40.83116141816186 74.35738217211774) bank160258 +160259 POINT(40.16565877511029 73.02896349582909) bank160259 +160260 POINT(39.75496141963181 74.204792157177) bank160260 +160261 POINT(41.10212731505242 74.94037095898341) bank160261 +160262 POINT(40.02640302648047 74.4454382029447) bank160262 +160263 POINT(41.32528725075311 73.8199277001473) bank160263 +160264 POINT(39.81733791245402 74.86105207560391) bank160264 +160265 POINT(41.219283588089645 73.23022069722428) bank160265 +160266 POINT(41.16547524234491 73.95715304252845) bank160266 +160267 POINT(41.64784794124772 74.54299084125805) bank160267 +160268 POINT(40.809940695339215 74.88816841640896) bank160268 +160269 POINT(40.38792557914707 73.96035473415732) bank160269 +160270 POINT(39.81472901716326 74.92676869255271) bank160270 +160271 POINT(41.673185099842904 74.53065306551335) bank160271 +160272 POINT(41.63697800675531 73.08660773221808) bank160272 +160273 POINT(40.151115054242894 74.26590348134766) bank160273 +160274 POINT(40.471775749690494 74.39934163689223) bank160274 +160275 POINT(41.41935132005196 74.77505435456656) bank160275 +160276 POINT(39.83268294776041 74.25005342352532) bank160276 +160277 POINT(40.27781982961109 74.7747917983334) bank160277 +160278 POINT(40.4987409246046 74.92338562743575) bank160278 +160279 POINT(39.917636537745025 73.55843208493876) bank160279 +160280 POINT(40.58594317877421 73.91801673205165) bank160280 +160281 POINT(39.743677331303715 73.22830743992304) bank160281 +160282 POINT(40.135568172183454 74.24483633275565) bank160282 +160283 POINT(40.12043174615401 74.8223542168678) bank160283 +160284 POINT(40.80481408218674 73.08477629183264) bank160284 +160285 POINT(41.10963808791589 73.22692849321395) bank160285 +160286 POINT(39.82444095516021 74.90389679115025) bank160286 +160287 POINT(40.28436334689238 73.87622113795828) bank160287 +160288 POINT(40.248365720518535 73.4136007403605) bank160288 +160289 POINT(40.15797931176508 73.66810958053335) bank160289 +160290 POINT(39.887811651876625 73.98407794853877) bank160290 +160291 POINT(40.10007257621771 74.61447546730267) bank160291 +160292 POINT(40.35279262330231 74.8442383877375) bank160292 +160293 POINT(41.163253462093316 74.77140188591821) bank160293 +160294 POINT(40.378793676811654 73.22792502524881) bank160294 +160295 POINT(40.95252883808983 74.08027344198737) bank160295 +160296 POINT(41.47271292017229 73.10589460983167) bank160296 +160297 POINT(40.388876669579524 73.24704959072953) bank160297 +160298 POINT(40.64125624653886 74.4181149547135) bank160298 +160299 POINT(41.227535977373236 74.8854972918178) bank160299 +160300 POINT(40.63563980553444 74.02522508864048) bank160300 +160301 POINT(41.415041296842226 74.83940237931273) bank160301 +160302 POINT(41.05427568638815 74.04690607452591) bank160302 +160303 POINT(41.66932281127178 73.19733268876588) bank160303 +160304 POINT(40.47403012455818 74.33233081309292) bank160304 +160305 POINT(40.20116965848224 74.90595793602273) bank160305 +160306 POINT(41.44882066182865 74.0191775381341) bank160306 +160307 POINT(40.45456127081461 74.21108320008429) bank160307 +160308 POINT(40.156719444704365 73.47842974552267) bank160308 +160309 POINT(40.33329102613038 74.91462247136768) bank160309 +160310 POINT(40.753867502723494 73.3902850682008) bank160310 +160311 POINT(40.225375275042964 74.64943241401689) bank160311 +160312 POINT(41.108102286078775 74.74146116062003) bank160312 +160313 POINT(41.54352945776839 74.05110609882078) bank160313 +160314 POINT(40.40889681526665 74.38447903801966) bank160314 +160315 POINT(40.997136299892105 73.32350117182382) bank160315 +160316 POINT(41.62498640822001 73.06416219413359) bank160316 +160317 POINT(39.740779872984504 73.79268446280163) bank160317 +160318 POINT(41.58890905667691 74.77512464237878) bank160318 +160319 POINT(40.88613053413011 73.03313348546472) bank160319 +160320 POINT(41.53507146020374 73.71240753317936) bank160320 +160321 POINT(41.20905404561248 73.90981121608098) bank160321 +160322 POINT(40.867337687769854 74.02215562615152) bank160322 +160323 POINT(41.292281392038376 73.52793288407962) bank160323 +160324 POINT(39.85831370495686 74.255688132481) bank160324 +160325 POINT(40.45153720407155 74.08927391466975) bank160325 +160326 POINT(40.419444957526814 73.23544276169375) bank160326 +160327 POINT(40.51858088881252 74.67255426834527) bank160327 +160328 POINT(41.406780802931564 73.14241741663399) bank160328 +160329 POINT(39.73623484658452 73.08086739419296) bank160329 +160330 POINT(41.155781385659076 73.78490495824173) bank160330 +160331 POINT(41.40222714676993 74.70341024119753) bank160331 +160332 POINT(41.677718110577025 74.33301039283441) bank160332 +160333 POINT(40.33472350608015 73.26885049833963) bank160333 +160334 POINT(41.209212513182635 74.85162590715764) bank160334 +160335 POINT(41.03090453538022 74.18034495343835) bank160335 +160336 POINT(41.123911444269964 73.98972682024022) bank160336 +160337 POINT(40.44842105367121 74.57764024063705) bank160337 +160338 POINT(41.40648831883063 73.07539979127722) bank160338 +160339 POINT(41.134003815053326 74.30669784026351) bank160339 +160340 POINT(39.724193822138254 74.63047568856483) bank160340 +160341 POINT(40.38311703590141 73.67657722904373) bank160341 +160342 POINT(41.56367403869714 74.73446076335505) bank160342 +160343 POINT(41.569033388742255 74.66187338597716) bank160343 +160344 POINT(41.02657980265737 74.0066469813785) bank160344 +160345 POINT(40.31201429840898 73.76135628838422) bank160345 +160346 POINT(40.6171616917819 73.01752353141951) bank160346 +160347 POINT(41.50131227736258 74.132982866163) bank160347 +160348 POINT(41.325271153166064 74.46682918981142) bank160348 +160349 POINT(41.230475391067436 73.7130134815931) bank160349 +160350 POINT(41.56716939891424 74.08686552146536) bank160350 +160351 POINT(40.66467165911913 74.6780343846779) bank160351 +160352 POINT(41.42112662474628 73.40536636137308) bank160352 +160353 POINT(41.362022957364466 74.13251469762393) bank160353 +160354 POINT(41.15305021917324 73.87027698023034) bank160354 +160355 POINT(40.403387736840614 74.07523365179803) bank160355 +160356 POINT(40.453832535769195 74.36082231198307) bank160356 +160357 POINT(41.595033471804314 74.46804439522815) bank160357 +160358 POINT(40.48995370673431 74.82446736880576) bank160358 +160359 POINT(40.11124145126163 73.01380779559962) bank160359 +160360 POINT(41.40783702189667 73.43757939896311) bank160360 +160361 POINT(40.56824444695331 73.96192942535217) bank160361 +160362 POINT(41.22127606827961 73.45049241270485) bank160362 +160363 POINT(40.83508995865353 73.55470168228781) bank160363 +160364 POINT(40.76649218828547 74.00088531668834) bank160364 +160365 POINT(39.771987568133746 74.3096124223982) bank160365 +160366 POINT(40.145051821360866 74.66658399877241) bank160366 +160367 POINT(40.51613721506263 73.50237120334583) bank160367 +160368 POINT(40.532492763202505 74.77657488665425) bank160368 +160369 POINT(39.94514030451319 73.41383301897086) bank160369 +160370 POINT(40.0712131788231 73.28290863039855) bank160370 +160371 POINT(40.32784334295532 73.92919965066508) bank160371 +160372 POINT(41.260016044426074 73.65064090129819) bank160372 +160373 POINT(41.28439182884088 73.97466619840212) bank160373 +160374 POINT(40.477233628602804 73.39396289146747) bank160374 +160375 POINT(41.59421587126299 74.48018670677625) bank160375 +160376 POINT(40.783373286482984 73.70012219569503) bank160376 +160377 POINT(40.14328740364454 74.07185432720725) bank160377 +160378 POINT(40.36438026285621 74.81980054122965) bank160378 +160379 POINT(39.73251326926962 74.79706731270241) bank160379 +160380 POINT(41.192249733993094 73.85156011886886) bank160380 +160381 POINT(40.791281397688834 74.34408140038967) bank160381 +160382 POINT(40.27974184628189 73.73467091200277) bank160382 +160383 POINT(40.057908846552536 74.82462941989056) bank160383 +160384 POINT(41.06982715760694 73.1601021835214) bank160384 +160385 POINT(39.94619147427471 74.81425225747353) bank160385 +160386 POINT(41.571336186423245 74.74789839719284) bank160386 +160387 POINT(40.185532639503464 73.65208650879973) bank160387 +160388 POINT(41.16096506975411 73.43272956155265) bank160388 +160389 POINT(40.6241296626872 73.62645879790664) bank160389 +160390 POINT(41.094315648186644 73.55075269523967) bank160390 +160391 POINT(41.33602072696601 73.76767743405766) bank160391 +160392 POINT(41.04211521454033 74.98507864101455) bank160392 +160393 POINT(41.34642348730345 74.0269015288709) bank160393 +160394 POINT(39.92884867877288 74.19590293182509) bank160394 +160395 POINT(41.02726904203195 73.42794702372555) bank160395 +160396 POINT(41.30513967623465 73.95091013033712) bank160396 +160397 POINT(39.94530932723929 74.75039522709777) bank160397 +160398 POINT(39.71454768668114 74.08225049598042) bank160398 +160399 POINT(41.302477193679536 74.81533982473857) bank160399 +160400 POINT(40.99483423183802 73.1686939499849) bank160400 +160401 POINT(41.634383374555924 73.52941984439201) bank160401 +160402 POINT(40.74804985125792 73.32434994790415) bank160402 +160403 POINT(39.86730139397673 73.68861514131557) bank160403 +160404 POINT(40.75014947332138 73.56070430850342) bank160404 +160405 POINT(39.719307696360055 74.52827545782337) bank160405 +160406 POINT(41.01435943881566 73.41282287713454) bank160406 +160407 POINT(41.21935461260121 74.21276116953781) bank160407 +160408 POINT(41.15496624912528 74.96726942375865) bank160408 +160409 POINT(40.94761391477573 74.88906530774196) bank160409 +160410 POINT(40.4922816664185 73.79995201598392) bank160410 +160411 POINT(41.03239051586874 73.76930472231895) bank160411 +160412 POINT(41.57420091769712 74.65934071055601) bank160412 +160413 POINT(41.05161789798056 73.10364532472349) bank160413 +160414 POINT(41.204457499099895 74.37756284083571) bank160414 +160415 POINT(41.13323316420926 74.24157309465542) bank160415 +160416 POINT(39.96837742179711 73.43222299798556) bank160416 +160417 POINT(40.508465940996636 73.17244846785172) bank160417 +160418 POINT(40.00071561433578 74.6093441496687) bank160418 +160419 POINT(40.344578306183536 74.1585975517092) bank160419 +160420 POINT(41.270048014603255 73.6125504875581) bank160420 +160421 POINT(41.23644980002719 73.73106162508164) bank160421 +160422 POINT(41.243037645523 73.25989322856466) bank160422 +160423 POINT(41.04521564899113 73.85738154746694) bank160423 +160424 POINT(40.32687986070867 74.69297278055609) bank160424 +160425 POINT(39.86930222646454 73.15015057948604) bank160425 +160426 POINT(41.059678647382455 73.31791850480099) bank160426 +160427 POINT(40.26695737096714 74.72751259533146) bank160427 +160428 POINT(41.47561796704753 74.77010758423687) bank160428 +160429 POINT(40.72445634917716 73.82585762659836) bank160429 +160430 POINT(40.51626657384147 74.56798314698413) bank160430 +160431 POINT(40.09181049982575 73.57322819185639) bank160431 +160432 POINT(40.338115980876815 74.403134002703) bank160432 +160433 POINT(40.87827345115311 73.83689693064889) bank160433 +160434 POINT(40.99537287534762 73.22049042297091) bank160434 +160435 POINT(40.56582976903245 73.37594053316593) bank160435 +160436 POINT(39.93156019595051 73.95219542947179) bank160436 +160437 POINT(40.33447260679875 74.7725153223744) bank160437 +160438 POINT(40.33042926169598 73.04279646967422) bank160438 +160439 POINT(40.422562531235734 74.15378965987352) bank160439 +160440 POINT(40.214743718262014 74.86940246117236) bank160440 +160441 POINT(40.42103110434721 74.7134758609998) bank160441 +160442 POINT(40.513679996002594 73.36592119053573) bank160442 +160443 POINT(40.14138385849531 74.1028044659047) bank160443 +160444 POINT(41.04675843420561 74.33542116761873) bank160444 +160445 POINT(39.80632210012867 73.73328892734486) bank160445 +160446 POINT(40.074671555549145 73.35973238571172) bank160446 +160447 POINT(39.894784604435294 73.87240014569127) bank160447 +160448 POINT(40.60304674044784 74.35418630853889) bank160448 +160449 POINT(41.60329197209688 74.05290989867684) bank160449 +160450 POINT(41.13788377505023 74.9778774532284) bank160450 +160451 POINT(40.32039211440845 73.51303812669704) bank160451 +160452 POINT(40.95602815580542 73.10530223453779) bank160452 +160453 POINT(39.80353300605092 74.37340647816147) bank160453 +160454 POINT(40.63519685998743 74.96463837112502) bank160454 +160455 POINT(39.806357802370194 74.90762454303837) bank160455 +160456 POINT(40.499204736779085 73.76812186330257) bank160456 +160457 POINT(41.21360515962168 73.14753493000518) bank160457 +160458 POINT(41.45661957784696 74.73077475543099) bank160458 +160459 POINT(41.13134877137544 73.96033839070418) bank160459 +160460 POINT(41.37076921338054 73.22680557462543) bank160460 +160461 POINT(41.35262618315473 74.1700525023478) bank160461 +160462 POINT(39.79305351233202 73.8271826723957) bank160462 +160463 POINT(40.82686522885908 73.52316586789945) bank160463 +160464 POINT(41.12680310710371 74.24007147831354) bank160464 +160465 POINT(39.96125877694251 73.90958696460781) bank160465 +160466 POINT(39.897728593469914 73.34438179597238) bank160466 +160467 POINT(41.29616047952522 73.10142363548248) bank160467 +160468 POINT(41.19226590014315 73.50229011343917) bank160468 +160469 POINT(41.248513751669755 73.8927989617265) bank160469 +160470 POINT(41.18525862691659 74.64685519895593) bank160470 +160471 POINT(40.885309659365085 74.6974048746755) bank160471 +160472 POINT(39.92883193294975 73.12974706096097) bank160472 +160473 POINT(39.894683825669404 74.56626677880772) bank160473 +160474 POINT(41.07430943075631 74.87002060648341) bank160474 +160475 POINT(40.31936775201937 73.13273320076193) bank160475 +160476 POINT(40.7232919523653 74.22958729932495) bank160476 +160477 POINT(40.83149667743462 74.58312103148087) bank160477 +160478 POINT(39.99236708107084 73.12231122068364) bank160478 +160479 POINT(40.18357720212247 74.82273282522979) bank160479 +160480 POINT(40.601590494494474 74.4156554266971) bank160480 +160481 POINT(41.22934175678727 73.87687825462767) bank160481 +160482 POINT(41.09067250151918 74.0991641524933) bank160482 +160483 POINT(40.4587743059281 73.33552372710581) bank160483 +160484 POINT(41.06892442686108 73.63186334632543) bank160484 +160485 POINT(41.49383735155932 73.1941544597133) bank160485 +160486 POINT(41.6374607313836 73.42576217612873) bank160486 +160487 POINT(41.680945667438415 73.19703704802293) bank160487 +160488 POINT(41.25109019272799 74.32156444160682) bank160488 +160489 POINT(40.13783243057232 73.96160597829963) bank160489 +160490 POINT(40.91664846364957 74.3994217805682) bank160490 +160491 POINT(41.56272992926051 73.23221343517886) bank160491 +160492 POINT(41.68565015246298 73.16501786059264) bank160492 +160493 POINT(40.19069286970867 73.4201618718616) bank160493 +160494 POINT(39.979898225252896 73.03115212723544) bank160494 +160495 POINT(41.662771818959634 74.72199737407614) bank160495 +160496 POINT(39.86015447989357 73.65239789869383) bank160496 +160497 POINT(41.154615876641685 73.19044528173988) bank160497 +160498 POINT(41.56521713515813 74.20576200166275) bank160498 +160499 POINT(41.52458142311164 74.5922531631054) bank160499 +160500 POINT(41.37780482924556 74.37425296538046) bank160500 +160501 POINT(41.18268906502634 74.25778814026049) bank160501 +160502 POINT(40.438359473839235 73.93540415740654) bank160502 +160503 POINT(40.82170901406376 73.63687581530671) bank160503 +160504 POINT(40.481630814262274 73.47357041404446) bank160504 +160505 POINT(41.36404549326587 74.40625227419864) bank160505 +160506 POINT(41.50938139085102 74.110783837634) bank160506 +160507 POINT(40.45376781373944 74.32565477288227) bank160507 +160508 POINT(41.03964577026501 74.89553626438726) bank160508 +160509 POINT(40.61317156263497 73.40357491788012) bank160509 +160510 POINT(41.50176499430606 73.01687756414402) bank160510 +160511 POINT(41.49224160120576 74.09268321955622) bank160511 +160512 POINT(41.56717676557447 73.23400615597053) bank160512 +160513 POINT(40.07953043171769 74.81388181469616) bank160513 +160514 POINT(40.95529233645322 74.5968686189218) bank160514 +160515 POINT(40.08954623283315 73.7209214149026) bank160515 +160516 POINT(40.91989982589424 73.50554101175663) bank160516 +160517 POINT(41.09468469933652 73.5121060868654) bank160517 +160518 POINT(41.251157369261115 74.1308785427957) bank160518 +160519 POINT(40.838182367116275 74.61127234386139) bank160519 +160520 POINT(40.92946810151053 74.52302392430425) bank160520 +160521 POINT(39.936342809435594 74.98327403699685) bank160521 +160522 POINT(40.47974802245337 73.4928734412278) bank160522 +160523 POINT(40.41520466150483 73.90630491424663) bank160523 +160524 POINT(40.21379565835038 73.4639980116949) bank160524 +160525 POINT(39.95187241966993 74.83502053742454) bank160525 +160526 POINT(40.02276316878121 74.21135060296733) bank160526 +160527 POINT(39.84633512478782 74.4429760060688) bank160527 +160528 POINT(41.48818087362029 73.43081661436581) bank160528 +160529 POINT(41.398023183874976 73.93237454909476) bank160529 +160530 POINT(40.4820498887007 73.92463599038132) bank160530 +160531 POINT(41.33600650948748 74.70898015337765) bank160531 +160532 POINT(39.79114430311961 73.06907897601383) bank160532 +160533 POINT(41.13518094508267 74.0549530732499) bank160533 +160534 POINT(41.50760460321411 73.46781810220837) bank160534 +160535 POINT(40.84515657588246 73.49894353036203) bank160535 +160536 POINT(39.93881533152041 73.78871199892181) bank160536 +160537 POINT(41.31656252239746 73.74309916368786) bank160537 +160538 POINT(40.57071465149014 73.50403710496467) bank160538 +160539 POINT(40.643876096235736 74.55634802193794) bank160539 +160540 POINT(41.41808374079801 74.68814166123464) bank160540 +160541 POINT(41.02004838755234 73.56455834389689) bank160541 +160542 POINT(40.535186447798246 74.52969118384459) bank160542 +160543 POINT(40.06616492063409 74.65751359838097) bank160543 +160544 POINT(40.372634052103315 73.7666182271553) bank160544 +160545 POINT(40.462220797336144 73.12024443373927) bank160545 +160546 POINT(39.91628441171178 73.26665981921627) bank160546 +160547 POINT(41.589979357864316 74.44167311595945) bank160547 +160548 POINT(39.81869701195684 74.86177219677438) bank160548 +160549 POINT(40.62381872964574 74.13014930724404) bank160549 +160550 POINT(41.082427990166565 75.00155396881364) bank160550 +160551 POINT(41.141658261368164 73.41510858108923) bank160551 +160552 POINT(40.29484261906545 73.97046215027456) bank160552 +160553 POINT(40.463659111042034 73.33715115755318) bank160553 +160554 POINT(41.54259288751931 74.08262667462282) bank160554 +160555 POINT(41.584589971998724 74.96087479137836) bank160555 +160556 POINT(40.15605887378935 73.31712179453315) bank160556 +160557 POINT(41.09941540199387 73.02493166880089) bank160557 +160558 POINT(40.64675356621309 74.52029864934742) bank160558 +160559 POINT(41.048278380971354 74.2034148995007) bank160559 +160560 POINT(40.29511382712173 73.65985453271432) bank160560 +160561 POINT(40.64578012333464 73.87814369501741) bank160561 +160562 POINT(41.070118592505146 73.10513647594823) bank160562 +160563 POINT(41.0189713544262 73.36197076243678) bank160563 +160564 POINT(40.97169598096089 74.80534591071195) bank160564 +160565 POINT(40.407569229020396 74.07859420961204) bank160565 +160566 POINT(40.42447058318586 74.52093247141218) bank160566 +160567 POINT(40.203213074891515 73.33707446829558) bank160567 +160568 POINT(41.25323528237487 73.70263138738152) bank160568 +160569 POINT(40.42820494901295 73.98928289494589) bank160569 +160570 POINT(40.585196384226215 74.27753752025846) bank160570 +160571 POINT(40.156564479948706 73.37114246339546) bank160571 +160572 POINT(39.976025431021846 73.19721382792012) bank160572 +160573 POINT(39.87314894635674 74.1900615634166) bank160573 +160574 POINT(40.363493417710224 74.82486220827757) bank160574 +160575 POINT(40.20754817743187 74.65245400529123) bank160575 +160576 POINT(39.72332412130171 74.04257959278046) bank160576 +160577 POINT(41.212115593249706 74.84656365752461) bank160577 +160578 POINT(39.8232089596996 74.43510611594589) bank160578 +160579 POINT(40.185188845062676 74.02587486138152) bank160579 +160580 POINT(41.32345065280315 73.78221751642855) bank160580 +160581 POINT(40.52190811373103 73.03257492580649) bank160581 +160582 POINT(40.96055411127452 73.2410468716841) bank160582 +160583 POINT(41.303467218253175 74.00042115408321) bank160583 +160584 POINT(40.458809512192595 74.30025796847782) bank160584 +160585 POINT(40.368955922511496 73.22667165955745) bank160585 +160586 POINT(40.64733706063685 73.69618944658781) bank160586 +160587 POINT(40.94894721265414 73.66986675311746) bank160587 +160588 POINT(41.46911501826811 74.97484304930629) bank160588 +160589 POINT(40.36630947564258 73.61593349967336) bank160589 +160590 POINT(40.32049199766486 73.68279912163169) bank160590 +160591 POINT(41.38613296254592 73.01247311512488) bank160591 +160592 POINT(41.385836899304294 73.23271055196355) bank160592 +160593 POINT(40.65250311710624 74.0584316286789) bank160593 +160594 POINT(40.02182033061847 74.93685624917417) bank160594 +160595 POINT(40.93463602133926 74.95167612960618) bank160595 +160596 POINT(40.64498973226852 74.8135055791945) bank160596 +160597 POINT(41.16243646429627 73.53954206535441) bank160597 +160598 POINT(40.40079754624188 74.1473537745166) bank160598 +160599 POINT(39.871670761386646 74.8665216160716) bank160599 +160600 POINT(40.67647112778574 74.43795563771879) bank160600 +160601 POINT(41.1968262498533 73.7883011701208) bank160601 +160602 POINT(40.485364214837674 74.84294040994503) bank160602 +160603 POINT(40.347807423962955 74.30616889737009) bank160603 +160604 POINT(39.73251562333913 74.88241480818657) bank160604 +160605 POINT(40.07360856970668 73.72894349866637) bank160605 +160606 POINT(41.63438080571828 74.73761073093789) bank160606 +160607 POINT(41.25402399577594 74.76591368296245) bank160607 +160608 POINT(39.92864143888497 74.89317211157207) bank160608 +160609 POINT(40.41244780357208 74.10811318883209) bank160609 +160610 POINT(41.42735491637335 73.0108485378336) bank160610 +160611 POINT(40.18334858047247 74.87916064521936) bank160611 +160612 POINT(41.56909851413577 73.21375747215687) bank160612 +160613 POINT(40.19780921254171 73.98612912924729) bank160613 +160614 POINT(41.10829827775633 73.64301295890373) bank160614 +160615 POINT(40.37650699298105 74.85212921249004) bank160615 +160616 POINT(41.3052847109822 73.3170552130877) bank160616 +160617 POINT(39.80689057233303 73.109399769814) bank160617 +160618 POINT(40.54559944771391 74.89040739840488) bank160618 +160619 POINT(41.6887718898649 74.91370465177444) bank160619 +160620 POINT(41.05161182466695 74.74688154277665) bank160620 +160621 POINT(40.25231749819009 74.6627218345967) bank160621 +160622 POINT(41.07214765893403 74.99166690364326) bank160622 +160623 POINT(40.33468037643549 73.33668542092983) bank160623 +160624 POINT(41.05243678330989 74.03621195611579) bank160624 +160625 POINT(40.03113174443216 73.05502483485189) bank160625 +160626 POINT(39.75224573274986 74.80381233813533) bank160626 +160627 POINT(39.924031704735285 73.76018773775718) bank160627 +160628 POINT(41.572563928891064 74.54008504060961) bank160628 +160629 POINT(40.90311156110245 74.15877816821276) bank160629 +160630 POINT(41.68091453764759 73.75907702676767) bank160630 +160631 POINT(41.65205964363042 73.79266236051527) bank160631 +160632 POINT(40.806457392389376 73.75116852682096) bank160632 +160633 POINT(40.02767725379721 74.79625221804496) bank160633 +160634 POINT(41.161899786193324 73.73155848681188) bank160634 +160635 POINT(39.78179628364681 74.0993741911662) bank160635 +160636 POINT(41.56309657958995 74.24734592674476) bank160636 +160637 POINT(39.906359567208725 74.64990583164297) bank160637 +160638 POINT(39.819143982342226 73.1354492990698) bank160638 +160639 POINT(40.574785674129416 73.72736077123402) bank160639 +160640 POINT(40.29815339593137 73.88579131067928) bank160640 +160641 POINT(40.77903910291673 74.26322081758364) bank160641 +160642 POINT(41.59163289094629 73.98886038215456) bank160642 +160643 POINT(40.80876113484693 74.31312656529715) bank160643 +160644 POINT(40.265810293021254 74.32073251751184) bank160644 +160645 POINT(40.15634222209308 74.6449874432805) bank160645 +160646 POINT(41.20433271029039 74.26707755835994) bank160646 +160647 POINT(41.07679731549887 74.4899298274797) bank160647 +160648 POINT(41.551822606641224 73.53301027111581) bank160648 +160649 POINT(39.999336085470674 73.6036629167258) bank160649 +160650 POINT(40.15476211542779 73.87016534757477) bank160650 +160651 POINT(40.96822411515083 73.97922761722796) bank160651 +160652 POINT(40.4177792245247 73.40647303920584) bank160652 +160653 POINT(41.67296105360818 74.71664679644002) bank160653 +160654 POINT(40.85301168300913 73.97346956052739) bank160654 +160655 POINT(40.76520152486739 73.41848379794861) bank160655 +160656 POINT(39.82689580071861 73.76811566095573) bank160656 +160657 POINT(39.86706661565693 73.21692107025068) bank160657 +160658 POINT(40.745521715868726 73.78962065905483) bank160658 +160659 POINT(40.41467030456356 73.2013686276107) bank160659 +160660 POINT(40.38492877459079 74.53672135197554) bank160660 +160661 POINT(41.573050514834115 73.78773763949623) bank160661 +160662 POINT(40.56250592308376 73.5187544915079) bank160662 +160663 POINT(40.366728892959024 74.96836145641258) bank160663 +160664 POINT(40.21581769017626 74.62372082513835) bank160664 +160665 POINT(41.26260269999346 73.23698194554649) bank160665 +160666 POINT(40.29133136936943 74.80958229385683) bank160666 +160667 POINT(40.49657805231813 74.92207593434763) bank160667 +160668 POINT(41.504146295349635 74.80760124424803) bank160668 +160669 POINT(41.067727502360334 73.33723100590291) bank160669 +160670 POINT(40.55947802095198 74.74226746732089) bank160670 +160671 POINT(40.24631259743604 73.96460684097211) bank160671 +160672 POINT(40.51948242587301 74.09878764693997) bank160672 +160673 POINT(40.670676725503625 73.69042262575127) bank160673 +160674 POINT(40.61529816649835 74.88328882437337) bank160674 +160675 POINT(39.95032650714578 74.95601401062476) bank160675 +160676 POINT(40.105130099184336 73.06608101107207) bank160676 +160677 POINT(41.62356504152301 73.08456565366923) bank160677 +160678 POINT(41.53433195235449 74.51201073960192) bank160678 +160679 POINT(39.755996565737064 73.30657982180134) bank160679 +160680 POINT(41.61510717981134 73.03507854566398) bank160680 +160681 POINT(40.34777349294829 73.5506635688321) bank160681 +160682 POINT(40.70367631457192 74.86718277826267) bank160682 +160683 POINT(40.4636534043294 73.59497054625176) bank160683 +160684 POINT(40.441314279823125 73.03488169146938) bank160684 +160685 POINT(41.11085852689237 74.35249102385968) bank160685 +160686 POINT(39.91135028105353 73.6036729231698) bank160686 +160687 POINT(41.17487520627551 73.70376500875751) bank160687 +160688 POINT(41.477665132030985 74.44136591658632) bank160688 +160689 POINT(40.11889800150088 73.12131604044187) bank160689 +160690 POINT(40.78745732567514 73.21625146513426) bank160690 +160691 POINT(39.77508018677662 74.61244342069219) bank160691 +160692 POINT(40.70215891323004 73.30222066173091) bank160692 +160693 POINT(41.6111393195535 73.57870873594204) bank160693 +160694 POINT(40.94932495104739 74.54258502550961) bank160694 +160695 POINT(41.702534466279076 73.76928550441139) bank160695 +160696 POINT(41.68805334862619 74.87994719706268) bank160696 +160697 POINT(41.34287339745268 73.49921041208508) bank160697 +160698 POINT(40.31395546701729 73.46575862889003) bank160698 +160699 POINT(40.16563584299048 73.95474427724878) bank160699 +160700 POINT(39.87073544332583 73.10801947293888) bank160700 +160701 POINT(40.55279103448741 73.85392851186558) bank160701 +160702 POINT(40.93773298982788 73.12586014623349) bank160702 +160703 POINT(41.19629352637968 73.87509054478221) bank160703 +160704 POINT(41.50337045849104 74.76003251511308) bank160704 +160705 POINT(41.50156757448662 73.59368892308824) bank160705 +160706 POINT(40.290436636427806 73.61018224883705) bank160706 +160707 POINT(40.111311559012584 74.74035013875522) bank160707 +160708 POINT(40.33592515016969 73.53776232741971) bank160708 +160709 POINT(40.402315993613726 73.94458133950097) bank160709 +160710 POINT(41.54192343663279 74.58620265681095) bank160710 +160711 POINT(40.962156139779026 74.24072128504032) bank160711 +160712 POINT(39.9620301726505 73.26489331245581) bank160712 +160713 POINT(41.54098889983268 74.1380849926948) bank160713 +160714 POINT(41.563619154401664 73.96358065716369) bank160714 +160715 POINT(41.66670919772131 73.75999915566335) bank160715 +160716 POINT(40.2121768411715 73.27788064984576) bank160716 +160717 POINT(40.13317666076649 74.19362916598888) bank160717 +160718 POINT(40.514384376325665 74.11771674549914) bank160718 +160719 POINT(40.27098093202379 74.36523677106445) bank160719 +160720 POINT(40.16644358707975 74.27672933548648) bank160720 +160721 POINT(41.1031593980422 73.43712640844325) bank160721 +160722 POINT(41.30862104051502 73.92904246102147) bank160722 +160723 POINT(39.857279703306794 74.06319364863845) bank160723 +160724 POINT(41.617781144805754 74.48487505569605) bank160724 +160725 POINT(39.90429711434613 74.82678608134046) bank160725 +160726 POINT(40.31153505711956 74.80857634052653) bank160726 +160727 POINT(40.19478702220425 73.58799045911505) bank160727 +160728 POINT(40.86763177321243 74.05006714915926) bank160728 +160729 POINT(40.93385351361795 73.225769202167) bank160729 +160730 POINT(39.99561221736555 74.08458498710526) bank160730 +160731 POINT(41.452730601467245 74.89972752205757) bank160731 +160732 POINT(40.69545634327534 73.98784107128658) bank160732 +160733 POINT(39.75987785712827 75.00173695599113) bank160733 +160734 POINT(41.168207128016476 73.3704901094037) bank160734 +160735 POINT(40.4665997399892 74.71464938061146) bank160735 +160736 POINT(40.802854890835334 73.48839185972578) bank160736 +160737 POINT(40.08792500866304 74.01956653962112) bank160737 +160738 POINT(40.50527912759227 73.49132085228202) bank160738 +160739 POINT(41.456663130816715 74.26349726209361) bank160739 +160740 POINT(41.584540551963315 74.3510385440371) bank160740 +160741 POINT(39.929877433438655 74.99701409391025) bank160741 +160742 POINT(41.350058858012915 73.85189452809215) bank160742 +160743 POINT(41.58681650374006 74.31639165276931) bank160743 +160744 POINT(41.30482986807017 74.89303035580446) bank160744 +160745 POINT(40.75783765434857 74.20233878425711) bank160745 +160746 POINT(41.422007357248575 73.7926122759156) bank160746 +160747 POINT(41.40331800278322 73.33116910507557) bank160747 +160748 POINT(40.65346480576899 74.7390029420718) bank160748 +160749 POINT(40.33927335119687 74.30003696528733) bank160749 +160750 POINT(41.4095643547209 73.72221162080368) bank160750 +160751 POINT(40.57309985441273 73.25585038054072) bank160751 +160752 POINT(41.680149258834085 74.921199273712) bank160752 +160753 POINT(40.21672795903986 73.21946295769318) bank160753 +160754 POINT(41.52892776816425 73.14447373829772) bank160754 +160755 POINT(39.81821415778994 74.9657497714024) bank160755 +160756 POINT(41.08559500847379 74.6884434467058) bank160756 +160757 POINT(41.590900808286456 73.30476613663588) bank160757 +160758 POINT(40.47355316225138 73.08619052086831) bank160758 +160759 POINT(39.970305913071165 73.24286387021415) bank160759 +160760 POINT(41.53526971464761 74.72686495049692) bank160760 +160761 POINT(41.037956096770706 74.72729981189212) bank160761 +160762 POINT(40.538661408247386 74.13082882202623) bank160762 +160763 POINT(40.541978716671466 74.29644577388268) bank160763 +160764 POINT(41.663732876309076 73.96987647544064) bank160764 +160765 POINT(41.43390072428043 73.21346673772784) bank160765 +160766 POINT(41.7014827441721 74.06350270962976) bank160766 +160767 POINT(40.29799850157412 73.11132250867048) bank160767 +160768 POINT(41.08593619872452 74.87545624839811) bank160768 +160769 POINT(41.50411550103038 73.42009221127537) bank160769 +160770 POINT(40.30519554415472 73.24520533961302) bank160770 +160771 POINT(40.58517773237586 73.55157387381571) bank160771 +160772 POINT(39.795662014763565 74.8890768847293) bank160772 +160773 POINT(41.409549065136744 73.59654359241728) bank160773 +160774 POINT(41.63715520404004 73.35721756225617) bank160774 +160775 POINT(40.2628917916281 73.43217566377457) bank160775 +160776 POINT(40.7993744464687 73.79287715225242) bank160776 +160777 POINT(40.012907542394586 74.60723218613275) bank160777 +160778 POINT(40.52091497146077 73.30547438595795) bank160778 +160779 POINT(41.027750176286276 73.09539404472841) bank160779 +160780 POINT(39.842734103629205 74.05163602275742) bank160780 +160781 POINT(40.452467344608124 74.04183990139968) bank160781 +160782 POINT(40.178860276201384 74.25792809202716) bank160782 +160783 POINT(41.013483674312695 74.3642666305629) bank160783 +160784 POINT(40.89155558091111 73.9909169860493) bank160784 +160785 POINT(40.097773881290266 73.51967116328066) bank160785 +160786 POINT(40.421403161069655 74.81074420668448) bank160786 +160787 POINT(40.328097690501124 73.97149937655885) bank160787 +160788 POINT(39.996075405815006 74.75453141791802) bank160788 +160789 POINT(41.33357379682278 74.57724355869783) bank160789 +160790 POINT(40.54611687123924 74.43395476918275) bank160790 +160791 POINT(41.60941328544596 73.44171194229699) bank160791 +160792 POINT(40.28634376937896 73.51230184302281) bank160792 +160793 POINT(41.251921760724485 74.33779643100145) bank160793 +160794 POINT(41.68173399132071 74.08899120192352) bank160794 +160795 POINT(40.2649465568014 73.57164424065246) bank160795 +160796 POINT(40.7421654142088 73.5889906790595) bank160796 +160797 POINT(39.798765882070306 73.36457816294693) bank160797 +160798 POINT(41.69515831930315 73.03666887352708) bank160798 +160799 POINT(39.901257220133175 74.55166340885893) bank160799 +160800 POINT(39.828867344992155 74.30878106976249) bank160800 +160801 POINT(40.738798536479 74.37496174880796) bank160801 +160802 POINT(40.51008838326824 73.30221746824738) bank160802 +160803 POINT(41.63462606659349 73.24787402193597) bank160803 +160804 POINT(39.83517169865404 73.34667433864571) bank160804 +160805 POINT(41.31452961765461 74.73046181828892) bank160805 +160806 POINT(40.422336575390034 73.76606397722618) bank160806 +160807 POINT(40.18263169888992 73.33750993983571) bank160807 +160808 POINT(40.513754653696004 73.54034387434815) bank160808 +160809 POINT(41.05272078381592 73.7948766989237) bank160809 +160810 POINT(39.75393157769665 73.19247965520418) bank160810 +160811 POINT(39.742921683312424 73.68401193242961) bank160811 +160812 POINT(41.466454840483436 74.43621320180807) bank160812 +160813 POINT(39.76060909800524 73.10387593246703) bank160813 +160814 POINT(41.55940804281901 73.52531750643618) bank160814 +160815 POINT(41.64229387031436 73.01122536123196) bank160815 +160816 POINT(40.71022932436545 73.76844188249108) bank160816 +160817 POINT(40.91317659213089 73.06589429945791) bank160817 +160818 POINT(41.482378327961584 74.44471391009938) bank160818 +160819 POINT(40.79429120269294 74.86230802421407) bank160819 +160820 POINT(40.47026992104819 73.50691415229194) bank160820 +160821 POINT(39.83045986434973 74.1850569703208) bank160821 +160822 POINT(40.79198424991925 73.252763304513) bank160822 +160823 POINT(40.11213682327388 73.92922747940905) bank160823 +160824 POINT(41.155221048707 74.8379692675468) bank160824 +160825 POINT(41.174682135926574 74.21432411569823) bank160825 +160826 POINT(41.3478300347249 74.96375958280952) bank160826 +160827 POINT(41.14380806516922 73.56548856009127) bank160827 +160828 POINT(41.211954816861905 74.87560621710891) bank160828 +160829 POINT(40.07674667572699 73.25016960157686) bank160829 +160830 POINT(41.309350191488925 73.77788367871773) bank160830 +160831 POINT(39.96446782051364 74.6070006137446) bank160831 +160832 POINT(41.53835631187525 73.66784085836552) bank160832 +160833 POINT(41.5381536950098 73.3423628608905) bank160833 +160834 POINT(40.59627045488496 74.86641307015032) bank160834 +160835 POINT(40.05372693037378 73.91096939642763) bank160835 +160836 POINT(40.802920899028294 74.27934625278871) bank160836 +160837 POINT(41.615189931375795 73.49102960578905) bank160837 +160838 POINT(40.30896172933816 74.11155687864847) bank160838 +160839 POINT(39.8005243175576 74.87298299016203) bank160839 +160840 POINT(41.46043853549572 74.84377547419312) bank160840 +160841 POINT(39.87938455018797 74.21941698377802) bank160841 +160842 POINT(40.70736530100322 75.00138342236507) bank160842 +160843 POINT(41.435839740737414 73.18550333403174) bank160843 +160844 POINT(41.608961426848055 74.27707861364358) bank160844 +160845 POINT(40.69308669028519 74.0255490530152) bank160845 +160846 POINT(39.912754336098246 74.11222683416364) bank160846 +160847 POINT(39.99566060810168 73.3828076685192) bank160847 +160848 POINT(41.31229662498548 74.21793569692616) bank160848 +160849 POINT(40.015554996219024 73.77493994813238) bank160849 +160850 POINT(40.02660281775594 74.54977778552967) bank160850 +160851 POINT(40.840299021599314 74.3477708279356) bank160851 +160852 POINT(40.852799666644984 74.92255832540296) bank160852 +160853 POINT(40.94919962529482 73.16613124047994) bank160853 +160854 POINT(39.844402275971206 74.99117821054867) bank160854 +160855 POINT(40.668651070780825 74.3373965899696) bank160855 +160856 POINT(40.04348749526589 74.10444501342612) bank160856 +160857 POINT(39.91432805510269 74.2258874407466) bank160857 +160858 POINT(40.40239352369208 74.08316423648796) bank160858 +160859 POINT(41.15607290700486 73.92393815118463) bank160859 +160860 POINT(40.42764598442523 73.10837342015489) bank160860 +160861 POINT(40.56532256013126 74.68908001726433) bank160861 +160862 POINT(40.82844679716692 73.17508078883314) bank160862 +160863 POINT(40.97407324078419 74.48302287722186) bank160863 +160864 POINT(40.54842041901328 73.48262631107255) bank160864 +160865 POINT(39.95218835409576 73.66790365486897) bank160865 +160866 POINT(40.077598950382736 73.55250370892165) bank160866 +160867 POINT(40.34727916266696 73.64907540973371) bank160867 +160868 POINT(40.655906796689926 73.41794468805496) bank160868 +160869 POINT(39.924461395176266 74.63196908865405) bank160869 +160870 POINT(39.84948162725575 73.08805486407717) bank160870 +160871 POINT(39.93173691048631 74.63337021840731) bank160871 +160872 POINT(39.97021591217583 74.38230990697319) bank160872 +160873 POINT(41.12656419922172 73.14806589499557) bank160873 +160874 POINT(41.48798348984476 73.99819288431311) bank160874 +160875 POINT(41.675028652910896 74.23689655652876) bank160875 +160876 POINT(41.54007206892218 73.60185661036043) bank160876 +160877 POINT(39.95655970440266 73.70587571062524) bank160877 +160878 POINT(39.75538270749924 73.0699965477333) bank160878 +160879 POINT(40.241270990880764 74.57256819682993) bank160879 +160880 POINT(40.28840307045029 74.96802802381889) bank160880 +160881 POINT(41.54521392216109 73.73029628075872) bank160881 +160882 POINT(40.9529405232785 74.11373645430004) bank160882 +160883 POINT(40.561487571561685 74.30767848071625) bank160883 +160884 POINT(39.93145662920207 73.34531228497292) bank160884 +160885 POINT(40.56165528815635 74.63494402516831) bank160885 +160886 POINT(40.630474170329386 73.53553363751773) bank160886 +160887 POINT(41.32509098012274 74.30256194098178) bank160887 +160888 POINT(40.00801378846989 74.16571867900367) bank160888 +160889 POINT(41.28811540852809 73.33784367691817) bank160889 +160890 POINT(39.8771163618559 73.46343532107211) bank160890 +160891 POINT(40.29366699849466 73.6667634424676) bank160891 +160892 POINT(41.434008017034145 74.94979367700505) bank160892 +160893 POINT(40.569058888938045 73.05892399689732) bank160893 +160894 POINT(40.51704641483167 73.50680336913061) bank160894 +160895 POINT(41.422520634341176 73.67776096526458) bank160895 +160896 POINT(41.42947866741619 74.81748395336042) bank160896 +160897 POINT(40.60649048029075 73.5398437343209) bank160897 +160898 POINT(41.09583883710188 73.90061550453416) bank160898 +160899 POINT(41.067680124609296 74.88673928252547) bank160899 +160900 POINT(40.21227927704731 73.13282644709308) bank160900 +160901 POINT(40.65117652329259 74.68479717591943) bank160901 +160902 POINT(40.08730555875994 74.6839984804997) bank160902 +160903 POINT(40.06922062343202 74.17175946607438) bank160903 +160904 POINT(40.81318804096097 74.83635183175525) bank160904 +160905 POINT(40.01496755654316 73.06753883921637) bank160905 +160906 POINT(40.72802286492673 73.59158248008698) bank160906 +160907 POINT(39.969318438087434 73.19599694116961) bank160907 +160908 POINT(40.432033579948566 73.32283646484724) bank160908 +160909 POINT(40.547138512284796 73.59567151667125) bank160909 +160910 POINT(40.2532693047395 74.83333786258855) bank160910 +160911 POINT(39.8083684092319 74.26965274613032) bank160911 +160912 POINT(41.05886424134552 74.89739916910051) bank160912 +160913 POINT(39.84963715511291 74.53401199779432) bank160913 +160914 POINT(40.07847889951028 73.62748265900113) bank160914 +160915 POINT(40.35813778048663 73.6782676520846) bank160915 +160916 POINT(40.19538828974521 73.3823270657207) bank160916 +160917 POINT(39.829940725959574 73.95514709865682) bank160917 +160918 POINT(41.47681720310732 74.43439180748243) bank160918 +160919 POINT(40.73792576430265 73.26326171156046) bank160919 +160920 POINT(40.28732251971389 73.92747668180479) bank160920 +160921 POINT(41.65839020297161 73.85320894458366) bank160921 +160922 POINT(40.74117209721195 73.5463226728789) bank160922 +160923 POINT(40.83775172150884 74.88182145518287) bank160923 +160924 POINT(41.48014503171805 74.23599654557738) bank160924 +160925 POINT(41.28347427420704 73.4548531643984) bank160925 +160926 POINT(41.155368839049686 74.04110789177074) bank160926 +160927 POINT(40.21380280839059 74.47339321033951) bank160927 +160928 POINT(41.263717653499334 73.40214225045317) bank160928 +160929 POINT(41.0921518952735 74.99304405652111) bank160929 +160930 POINT(40.3152460539306 73.27759034157855) bank160930 +160931 POINT(40.568495379461 74.14827576732435) bank160931 +160932 POINT(40.72443690420463 73.075682790751) bank160932 +160933 POINT(39.78201885868307 73.36418779659431) bank160933 +160934 POINT(41.36232311331655 74.41963401074997) bank160934 +160935 POINT(40.878758733780415 74.75995653328565) bank160935 +160936 POINT(40.68207992577696 73.62010914530988) bank160936 +160937 POINT(41.491583937195365 73.89441352049364) bank160937 +160938 POINT(40.54300552922469 74.74009618219345) bank160938 +160939 POINT(39.761000006283865 73.87713179790143) bank160939 +160940 POINT(41.67075152981008 74.62478246951089) bank160940 +160941 POINT(40.70226682911524 73.25735693078929) bank160941 +160942 POINT(41.03566637952913 74.9257703539208) bank160942 +160943 POINT(40.70266612630411 73.52860148081196) bank160943 +160944 POINT(39.73469500428854 74.91944417091344) bank160944 +160945 POINT(41.087760247751966 73.69071939947247) bank160945 +160946 POINT(41.048046299309625 73.44032287857694) bank160946 +160947 POINT(41.497627232064396 73.78550137246181) bank160947 +160948 POINT(39.859898724499885 73.1668667616193) bank160948 +160949 POINT(40.61984957691883 74.7741348275978) bank160949 +160950 POINT(41.602025921350425 73.1619787153526) bank160950 +160951 POINT(41.63097311506529 74.24936281843604) bank160951 +160952 POINT(40.48429554025007 73.84773273466568) bank160952 +160953 POINT(41.69346565971828 74.12141625733183) bank160953 +160954 POINT(41.44547329923263 74.16506861652377) bank160954 +160955 POINT(40.06339910973101 74.02528173439156) bank160955 +160956 POINT(40.75227417058199 73.98797389670499) bank160956 +160957 POINT(40.091613152219864 74.475270992366) bank160957 +160958 POINT(41.41259212957225 74.34522250625072) bank160958 +160959 POINT(41.169962007820416 73.29388873974248) bank160959 +160960 POINT(41.28165625753792 73.00645039756265) bank160960 +160961 POINT(41.472490655847196 74.94663067346922) bank160961 +160962 POINT(40.92267941118635 74.03999281007324) bank160962 +160963 POINT(40.77104461810198 74.76088068187813) bank160963 +160964 POINT(40.820114873713614 73.7275939701456) bank160964 +160965 POINT(39.948795432163415 74.48723278721957) bank160965 +160966 POINT(40.12831931936747 73.15013354077813) bank160966 +160967 POINT(40.40612650565181 74.49333128374309) bank160967 +160968 POINT(41.33969309545088 73.29428764847647) bank160968 +160969 POINT(39.930574182774215 74.9932458792975) bank160969 +160970 POINT(39.88044211463158 74.33054338300713) bank160970 +160971 POINT(40.48787275794795 73.19974883594772) bank160971 +160972 POINT(40.891439711388045 74.44337877144545) bank160972 +160973 POINT(41.24408649007768 74.1653004688667) bank160973 +160974 POINT(40.659773638669044 74.20475008971799) bank160974 +160975 POINT(40.500790732927754 73.48368753705671) bank160975 +160976 POINT(40.964198858705934 74.45856339941349) bank160976 +160977 POINT(40.17973692094576 74.98058480442404) bank160977 +160978 POINT(40.74707316878539 73.24223591448064) bank160978 +160979 POINT(41.261089458390266 73.7056021434172) bank160979 +160980 POINT(41.33088274409331 73.96077440666308) bank160980 +160981 POINT(40.26875244017781 74.32374013470695) bank160981 +160982 POINT(41.28340827357266 73.65971372840072) bank160982 +160983 POINT(40.80554195059139 74.12703004872766) bank160983 +160984 POINT(40.814368284627776 73.6073174907558) bank160984 +160985 POINT(41.67169145782635 74.66441588450061) bank160985 +160986 POINT(40.49865332292761 73.5972656226932) bank160986 +160987 POINT(41.30291363339394 74.51703343782557) bank160987 +160988 POINT(40.20720825889349 74.56565384723775) bank160988 +160989 POINT(40.397872234169746 74.00909607646713) bank160989 +160990 POINT(41.14334561848706 74.34169209343253) bank160990 +160991 POINT(41.52217787096052 73.97146106986267) bank160991 +160992 POINT(41.07762425685634 74.90871630180364) bank160992 +160993 POINT(41.502731127946255 74.78182997699831) bank160993 +160994 POINT(41.30937995957267 73.9106507426791) bank160994 +160995 POINT(39.95361154707146 73.5239130501089) bank160995 +160996 POINT(41.26828155674315 74.33567625874286) bank160996 +160997 POINT(41.14546646739818 74.34845570708397) bank160997 +160998 POINT(40.82357395631857 74.86109944633341) bank160998 +160999 POINT(39.90806246018605 74.33292416157472) bank160999 +161000 POINT(41.22549321146149 73.31618769530355) bank161000 +161001 POINT(40.023138500562574 74.48540023998751) bank161001 +161002 POINT(40.5131213187541 74.97308803887003) bank161002 +161003 POINT(40.82571796560836 74.28006182852438) bank161003 +161004 POINT(41.210663461748176 73.61560879850819) bank161004 +161005 POINT(40.79809555958227 73.2613614160487) bank161005 +161006 POINT(40.11867302009633 73.91802156484734) bank161006 +161007 POINT(41.45798178906715 74.5808592745975) bank161007 +161008 POINT(41.31812756384073 73.58071582533009) bank161008 +161009 POINT(40.91842120301502 74.84470718536718) bank161009 +161010 POINT(39.865604961487804 73.07385620836034) bank161010 +161011 POINT(40.63587261382185 74.87764901751268) bank161011 +161012 POINT(41.46418116370148 73.56396810090575) bank161012 +161013 POINT(41.260136831867385 74.36372702442) bank161013 +161014 POINT(40.36196908016188 74.77911802163206) bank161014 +161015 POINT(41.59847435911338 73.7020610061532) bank161015 +161016 POINT(41.085126388866506 74.81252932486447) bank161016 +161017 POINT(40.283431109897286 73.53539041742788) bank161017 +161018 POINT(41.03612128733941 73.90697452102485) bank161018 +161019 POINT(41.04971297428771 74.8847529957812) bank161019 +161020 POINT(41.20454505497059 73.42678638121039) bank161020 +161021 POINT(39.849186140284935 74.89469083459684) bank161021 +161022 POINT(40.27675779765787 73.41589382637324) bank161022 +161023 POINT(41.44393187970088 74.15555519084529) bank161023 +161024 POINT(41.140608291059166 73.47211243506318) bank161024 +161025 POINT(40.571219396426805 74.13887165818052) bank161025 +161026 POINT(40.95475893893206 74.12349747423198) bank161026 +161027 POINT(39.94908485518036 73.06287545199191) bank161027 +161028 POINT(41.01766687046793 74.8211844823202) bank161028 +161029 POINT(40.560153484176254 73.74494140660839) bank161029 +161030 POINT(39.87137287838367 74.03083684380134) bank161030 +161031 POINT(41.40035654077265 74.86584723459104) bank161031 +161032 POINT(41.22974734571981 73.1012362861614) bank161032 +161033 POINT(41.623897359343175 73.73751742112671) bank161033 +161034 POINT(40.92521860112717 73.94552627319001) bank161034 +161035 POINT(41.58388540465495 74.40108024337376) bank161035 +161036 POINT(40.86762421547926 73.32398245577993) bank161036 +161037 POINT(41.13504278728551 73.1872714454044) bank161037 +161038 POINT(41.221250233478656 74.87213229872577) bank161038 +161039 POINT(41.7025562904159 74.63049543543411) bank161039 +161040 POINT(41.3504354526951 73.64475890194412) bank161040 +161041 POINT(40.07299838258284 74.11963816704588) bank161041 +161042 POINT(40.098451216954764 73.6665747597609) bank161042 +161043 POINT(40.82914316521184 74.69883862097035) bank161043 +161044 POINT(41.62852453566175 73.20837777633848) bank161044 +161045 POINT(39.78558350625544 74.0289015360986) bank161045 +161046 POINT(40.64186154026703 74.47885343040159) bank161046 +161047 POINT(40.415834619772994 73.58289283242874) bank161047 +161048 POINT(39.77462363195021 74.42711002142067) bank161048 +161049 POINT(41.27376993824562 74.0447576776328) bank161049 +161050 POINT(39.884289615463956 73.80222385867891) bank161050 +161051 POINT(41.596681157059614 73.03247931651953) bank161051 +161052 POINT(40.88659884836622 74.46095659893331) bank161052 +161053 POINT(40.544703783924945 73.55221724217324) bank161053 +161054 POINT(41.58534469896921 73.41826896761397) bank161054 +161055 POINT(40.37645295334682 74.25478776005944) bank161055 +161056 POINT(41.16755175834915 73.57140195549874) bank161056 +161057 POINT(39.716857215369465 74.10040721948516) bank161057 +161058 POINT(41.443607868656855 73.18180890583025) bank161058 +161059 POINT(40.12546612654281 74.93863753928589) bank161059 +161060 POINT(40.796113509455324 73.9760022541266) bank161060 +161061 POINT(41.239848367957876 74.26553662250602) bank161061 +161062 POINT(41.441766337994295 74.87782014259572) bank161062 +161063 POINT(40.577465392458485 73.96496935955778) bank161063 +161064 POINT(41.49033100574477 73.50374897719534) bank161064 +161065 POINT(41.461171432424614 73.96424847810287) bank161065 +161066 POINT(41.34296338852655 74.26087330231954) bank161066 +161067 POINT(40.55641609094952 73.56202087684129) bank161067 +161068 POINT(40.41992791557405 73.31678466889852) bank161068 +161069 POINT(40.214929191508396 73.67545654620984) bank161069 +161070 POINT(41.6628595671484 73.40480645278497) bank161070 +161071 POINT(40.51851870982157 73.74563371959881) bank161071 +161072 POINT(40.12460479676953 74.4589610667447) bank161072 +161073 POINT(41.0666701445967 73.07092838057582) bank161073 +161074 POINT(40.96977710832774 74.65061929234676) bank161074 +161075 POINT(41.687403733278266 74.95539038741946) bank161075 +161076 POINT(39.770732790518665 74.6958752145981) bank161076 +161077 POINT(40.87945527338682 74.73282618376938) bank161077 +161078 POINT(40.02709730404198 74.30028903261787) bank161078 +161079 POINT(41.124406650518885 74.21010895412311) bank161079 +161080 POINT(41.46700519407232 74.6402902759508) bank161080 +161081 POINT(41.43819379961767 74.21804172682059) bank161081 +161082 POINT(41.63305677146363 74.20985478301883) bank161082 +161083 POINT(39.82572102403153 73.51047224283407) bank161083 +161084 POINT(41.5136683006776 74.19866422422777) bank161084 +161085 POINT(41.52932365746581 74.82594343881789) bank161085 +161086 POINT(41.26972859283562 74.97245906268361) bank161086 +161087 POINT(41.618683887560884 73.41635468925493) bank161087 +161088 POINT(41.70145160519418 73.29832233973065) bank161088 +161089 POINT(40.34328559755021 74.32955747138426) bank161089 +161090 POINT(41.61715032443461 74.72565709887694) bank161090 +161091 POINT(40.20193926320328 73.71188690846559) bank161091 +161092 POINT(40.990466292463665 74.85427697422351) bank161092 +161093 POINT(40.06867438553277 73.71252896753609) bank161093 +161094 POINT(41.593421883740675 74.06220354354667) bank161094 +161095 POINT(40.240196510803344 73.43155419589307) bank161095 +161096 POINT(40.573627944716556 74.35771850168801) bank161096 +161097 POINT(41.41544526807208 73.13564450272632) bank161097 +161098 POINT(40.70705799349811 73.52962997071685) bank161098 +161099 POINT(40.03344104546545 73.5994711770712) bank161099 +161100 POINT(39.927017402661434 74.29991370571956) bank161100 +161101 POINT(39.957630753787164 73.87576090058306) bank161101 +161102 POINT(40.768392999743625 73.09843736129314) bank161102 +161103 POINT(41.20402363487021 74.96736924758447) bank161103 +161104 POINT(40.50322936438587 74.98549412481012) bank161104 +161105 POINT(40.02569313761802 73.66485614126644) bank161105 +161106 POINT(39.76861717119672 74.52635067523268) bank161106 +161107 POINT(41.419766540831894 74.11881504626315) bank161107 +161108 POINT(41.002861090759794 74.22164157191327) bank161108 +161109 POINT(40.89137764690077 74.5284743129704) bank161109 +161110 POINT(40.970937444650666 73.59640599161133) bank161110 +161111 POINT(40.592095256490246 73.28784143446512) bank161111 +161112 POINT(40.06372137972521 74.68197265832583) bank161112 +161113 POINT(39.816653187715914 73.14866056906678) bank161113 +161114 POINT(41.49050417904088 73.40683795675758) bank161114 +161115 POINT(39.86141952948404 73.20760043023526) bank161115 +161116 POINT(41.20865964327369 74.88530566608378) bank161116 +161117 POINT(40.11476470324436 74.50907256629233) bank161117 +161118 POINT(40.1607385509762 73.11242354466594) bank161118 +161119 POINT(40.56270041012613 73.2982976409754) bank161119 +161120 POINT(40.2764318556598 73.44608017717727) bank161120 +161121 POINT(40.72150565096496 74.92796460518333) bank161121 +161122 POINT(39.785734490934075 74.21095683212728) bank161122 +161123 POINT(41.67533271224463 73.47290325837669) bank161123 +161124 POINT(41.26007835427222 74.49325562643158) bank161124 +161125 POINT(40.17758650159712 73.27055169315281) bank161125 +161126 POINT(41.06715180631133 73.17489003466005) bank161126 +161127 POINT(39.72703685856798 73.69253123666802) bank161127 +161128 POINT(40.432239238612105 74.7586512773299) bank161128 +161129 POINT(41.690104705411514 74.3307611050123) bank161129 +161130 POINT(40.39293783108771 74.06867806779809) bank161130 +161131 POINT(41.37861876716411 74.2486812165097) bank161131 +161132 POINT(40.84934751198993 74.12661872018398) bank161132 +161133 POINT(39.99101423492508 73.02585539022736) bank161133 +161134 POINT(41.197229198615005 74.40847874030409) bank161134 +161135 POINT(40.42018836341566 73.49991944658414) bank161135 +161136 POINT(40.736525100581446 73.15233003608645) bank161136 +161137 POINT(39.77474118521335 74.74522939326194) bank161137 +161138 POINT(39.89173932305362 74.71967050077535) bank161138 +161139 POINT(41.34597084551495 74.6764731912815) bank161139 +161140 POINT(40.438023054773375 74.35768514698977) bank161140 +161141 POINT(40.82310530841319 73.95109108768571) bank161141 +161142 POINT(41.191589782311844 74.21430573719161) bank161142 +161143 POINT(40.49033572654494 73.64251190411692) bank161143 +161144 POINT(40.355772327356036 73.76186361772099) bank161144 +161145 POINT(40.912987908266345 74.98494721161501) bank161145 +161146 POINT(41.149966575286314 74.03355991813007) bank161146 +161147 POINT(40.046441924283556 74.18410166104034) bank161147 +161148 POINT(40.80521699240464 73.59890014980826) bank161148 +161149 POINT(41.13709263577807 73.68056048616778) bank161149 +161150 POINT(40.0949220079464 73.20180293769339) bank161150 +161151 POINT(40.59765577403811 73.74961881662828) bank161151 +161152 POINT(41.20794436276602 73.17114966650212) bank161152 +161153 POINT(41.67405481519308 73.89093475865006) bank161153 +161154 POINT(40.92775177172561 74.24102833443496) bank161154 +161155 POINT(41.308727728145456 73.50472049921441) bank161155 +161156 POINT(40.56183082693551 74.8030974412202) bank161156 +161157 POINT(41.223065290487504 74.4101190356972) bank161157 +161158 POINT(40.76822924037778 74.88551580847904) bank161158 +161159 POINT(41.084091291594255 73.56592676894873) bank161159 +161160 POINT(40.28821571195722 73.44118596257898) bank161160 +161161 POINT(39.77133960662785 74.83453858525722) bank161161 +161162 POINT(40.957835452406265 74.44287397687268) bank161162 +161163 POINT(40.35467541464099 74.9858218782788) bank161163 +161164 POINT(40.17772301386657 74.7558647119912) bank161164 +161165 POINT(40.98538627533142 74.6338027053666) bank161165 +161166 POINT(40.93872172587607 74.84295635094715) bank161166 +161167 POINT(41.10210863874186 73.63998168447472) bank161167 +161168 POINT(40.27601737190542 74.98299762528713) bank161168 +161169 POINT(40.54357807109739 74.68022898824857) bank161169 +161170 POINT(40.12521994615932 73.88649205358362) bank161170 +161171 POINT(39.838730886075645 73.21341923419601) bank161171 +161172 POINT(41.1038274838953 73.50819370266373) bank161172 +161173 POINT(40.85681878880416 74.46535618870303) bank161173 +161174 POINT(40.165284797521316 74.60223723431714) bank161174 +161175 POINT(41.26661601680934 73.76128929463194) bank161175 +161176 POINT(40.69839232698257 73.1195106442227) bank161176 +161177 POINT(40.1163348922345 74.7742054815713) bank161177 +161178 POINT(41.211415707053646 73.41322576728501) bank161178 +161179 POINT(40.872592829542995 73.23787839132035) bank161179 +161180 POINT(41.062198789050214 73.60643115210524) bank161180 +161181 POINT(40.62337899809281 73.10406971708763) bank161181 +161182 POINT(40.80922159686577 74.34080018322285) bank161182 +161183 POINT(41.124742065491034 74.99648102371717) bank161183 +161184 POINT(40.86986164404237 74.44589474132374) bank161184 +161185 POINT(39.937844433518876 74.80820544251559) bank161185 +161186 POINT(41.54822556654175 73.08278182314889) bank161186 +161187 POINT(39.76624709836612 73.32581987635386) bank161187 +161188 POINT(39.96986018790145 73.55311356764605) bank161188 +161189 POINT(41.1797264700371 74.2874762263918) bank161189 +161190 POINT(40.494850289450525 73.33519828739409) bank161190 +161191 POINT(40.13524768843241 74.44619684251647) bank161191 +161192 POINT(41.17185293371971 74.27237833234491) bank161192 +161193 POINT(41.23616901600644 73.34343360354849) bank161193 +161194 POINT(40.06180248527195 74.2743978720139) bank161194 +161195 POINT(41.092818024489574 73.48522313996216) bank161195 +161196 POINT(39.73191130853394 73.0924014028063) bank161196 +161197 POINT(41.30773090366884 74.43056268646468) bank161197 +161198 POINT(41.39150558544882 73.66093264566432) bank161198 +161199 POINT(39.76034404215583 73.20925940626886) bank161199 +161200 POINT(39.97063450494684 73.8381466125107) bank161200 +161201 POINT(41.00654815081657 74.39747661630831) bank161201 +161202 POINT(41.37065784742967 73.07452680461795) bank161202 +161203 POINT(40.500881068276655 74.54215367507162) bank161203 +161204 POINT(39.826319381437536 73.68923433441441) bank161204 +161205 POINT(41.14680454779803 74.17378986247263) bank161205 +161206 POINT(40.8365417174692 74.69139104782732) bank161206 +161207 POINT(41.55465482260452 73.2336557541295) bank161207 +161208 POINT(41.00881342147388 73.42574373082802) bank161208 +161209 POINT(41.302881183579984 73.38992900917629) bank161209 +161210 POINT(41.09287144953475 74.76158128868609) bank161210 +161211 POINT(40.14590370288805 74.98335154037512) bank161211 +161212 POINT(40.465788213799826 73.65662419295268) bank161212 +161213 POINT(40.99637065709034 73.7449160410611) bank161213 +161214 POINT(39.872666106698624 73.96890028733371) bank161214 +161215 POINT(40.020546034907994 74.73289855433221) bank161215 +161216 POINT(40.42892035776289 74.64273077143655) bank161216 +161217 POINT(40.25645227159539 73.48217891667207) bank161217 +161218 POINT(41.19733114647862 73.03366563610652) bank161218 +161219 POINT(40.62487134571903 73.20669731566538) bank161219 +161220 POINT(40.296453818239755 74.65004562961714) bank161220 +161221 POINT(40.12797009292432 73.59856933611182) bank161221 +161222 POINT(41.03134743141733 74.16997318189772) bank161222 +161223 POINT(41.25752255511624 74.8646080357501) bank161223 +161224 POINT(41.4052232521492 74.3020416310233) bank161224 +161225 POINT(41.440621003937046 73.50107213373693) bank161225 +161226 POINT(40.518526912598105 74.99067040638471) bank161226 +161227 POINT(40.89061478440683 73.03400729583115) bank161227 +161228 POINT(40.25154192720017 73.63620126792114) bank161228 +161229 POINT(40.132995356345425 73.44218640924723) bank161229 +161230 POINT(40.98894430685907 74.31508891319663) bank161230 +161231 POINT(39.73215354568213 74.84377906511348) bank161231 +161232 POINT(40.432928328804 74.59057674990095) bank161232 +161233 POINT(41.443472408579304 74.30241858639653) bank161233 +161234 POINT(40.141772044074045 74.13738723244377) bank161234 +161235 POINT(40.30397077039152 73.68371910965818) bank161235 +161236 POINT(40.708658958237514 74.62979930859099) bank161236 +161237 POINT(40.102911025353706 73.88468780562766) bank161237 +161238 POINT(39.95788136470926 74.51524756979404) bank161238 +161239 POINT(40.78114576600807 73.343832809637) bank161239 +161240 POINT(40.814009731446134 73.61518402458188) bank161240 +161241 POINT(40.93353236536188 73.14369989031175) bank161241 +161242 POINT(41.33906367341073 74.65832465368649) bank161242 +161243 POINT(39.94969262786112 74.93290982180962) bank161243 +161244 POINT(40.04222712087297 73.38476095552392) bank161244 +161245 POINT(40.07911363700257 73.11742666486198) bank161245 +161246 POINT(41.467457034192776 73.31454244134845) bank161246 +161247 POINT(41.410141923891906 73.7262521490637) bank161247 +161248 POINT(41.187286139839735 74.63827153973658) bank161248 +161249 POINT(39.789115036032015 74.38465342594765) bank161249 +161250 POINT(40.62951841621888 73.11299455878601) bank161250 +161251 POINT(41.18838913293747 73.63292932692674) bank161251 +161252 POINT(40.79951629607782 73.05922690910103) bank161252 +161253 POINT(41.0837065121752 74.52797377157475) bank161253 +161254 POINT(40.20774824842873 74.20821973750097) bank161254 +161255 POINT(39.92177009889887 73.74276413099675) bank161255 +161256 POINT(40.830620576927345 74.40622322758931) bank161256 +161257 POINT(40.63069425562271 74.88045248440294) bank161257 +161258 POINT(39.897228800321344 73.52898601600474) bank161258 +161259 POINT(41.589061620379375 73.23864792328386) bank161259 +161260 POINT(40.66741809653234 74.64144974890829) bank161260 +161261 POINT(40.58005540195617 74.66581604122437) bank161261 +161262 POINT(41.23124180775998 74.54288160095349) bank161262 +161263 POINT(41.55684706454536 73.46119279002129) bank161263 +161264 POINT(40.083037971009624 73.83320070033724) bank161264 +161265 POINT(41.086018637421375 73.36755908081079) bank161265 +161266 POINT(40.08980230063786 73.55412222186689) bank161266 +161267 POINT(40.33284485126694 74.04169308360518) bank161267 +161268 POINT(40.73112285431739 73.34441727174314) bank161268 +161269 POINT(40.681926703268445 74.75018917319223) bank161269 +161270 POINT(40.0511664683003 73.52002799456281) bank161270 +161271 POINT(41.23938933120496 73.64238894943455) bank161271 +161272 POINT(41.70802455787459 74.63126049883292) bank161272 +161273 POINT(41.41129343532341 74.05744037858106) bank161273 +161274 POINT(39.735757448859644 74.5864802785462) bank161274 +161275 POINT(41.35000572521338 74.04633806881003) bank161275 +161276 POINT(39.80904755894357 74.9920705534032) bank161276 +161277 POINT(40.21138785413479 74.50365187229046) bank161277 +161278 POINT(40.378985290117555 73.10838950537003) bank161278 +161279 POINT(40.62707122166844 73.08548819730906) bank161279 +161280 POINT(41.213985566853246 73.53099020914658) bank161280 +161281 POINT(41.30007032747883 73.3960863504305) bank161281 +161282 POINT(39.90316286134038 73.33723212940473) bank161282 +161283 POINT(40.308035415129936 73.39090324310301) bank161283 +161284 POINT(41.63115119376974 74.08419947659804) bank161284 +161285 POINT(41.41208739804272 74.1998385589302) bank161285 +161286 POINT(40.62050988705011 74.72409646522657) bank161286 +161287 POINT(40.253835240576606 74.2106008416586) bank161287 +161288 POINT(40.98287191309388 74.0201760995308) bank161288 +161289 POINT(41.5063613249307 73.06614832457758) bank161289 +161290 POINT(41.69252340480514 73.14602393260463) bank161290 +161291 POINT(41.09639453838697 74.14309248852024) bank161291 +161292 POINT(40.223709359464316 74.69144228237265) bank161292 +161293 POINT(41.153784838908855 74.76332409626932) bank161293 +161294 POINT(41.12995025374035 74.68389684976293) bank161294 +161295 POINT(41.13323404983943 74.57011855030213) bank161295 +161296 POINT(39.88007388237032 74.57592935261103) bank161296 +161297 POINT(41.092201486561194 74.4134236739187) bank161297 +161298 POINT(40.67965542976394 73.52117118295685) bank161298 +161299 POINT(41.25188567042726 73.59874861308333) bank161299 +161300 POINT(40.78398961704906 74.23290271136433) bank161300 +161301 POINT(41.520941208726605 74.90787750091354) bank161301 +161302 POINT(40.67194274172768 74.50072931966338) bank161302 +161303 POINT(40.219574941543755 73.10162031480385) bank161303 +161304 POINT(41.50308482220434 73.08857560302718) bank161304 +161305 POINT(39.91233849797449 73.4297562291447) bank161305 +161306 POINT(40.714197415119465 73.17281238959762) bank161306 +161307 POINT(41.57096496832473 74.07847793494989) bank161307 +161308 POINT(41.498380488909376 73.07174915751753) bank161308 +161309 POINT(40.26411634736155 74.02243264604675) bank161309 +161310 POINT(40.47184287763796 74.34889405417248) bank161310 +161311 POINT(40.667603698304404 73.72425509768735) bank161311 +161312 POINT(41.4817809871267 74.51036585391437) bank161312 +161313 POINT(41.35578942970497 73.06546518372276) bank161313 +161314 POINT(40.1214834426192 74.23092436636269) bank161314 +161315 POINT(39.9519841684418 73.63816307711335) bank161315 +161316 POINT(39.90918085784365 73.18240683967471) bank161316 +161317 POINT(40.67865835025008 74.72961130839526) bank161317 +161318 POINT(40.978586332648526 74.85905618858746) bank161318 +161319 POINT(41.371822693596656 73.26091718873171) bank161319 +161320 POINT(41.35140033926339 73.94674132458847) bank161320 +161321 POINT(40.57104394348525 73.38990822334776) bank161321 +161322 POINT(39.91406611693261 74.09475624234013) bank161322 +161323 POINT(40.479358502631726 73.40432660671236) bank161323 +161324 POINT(41.16417596074306 73.76604587258362) bank161324 +161325 POINT(40.2794685059656 73.77576984066818) bank161325 +161326 POINT(39.89412292976173 74.40315402224151) bank161326 +161327 POINT(40.10462792981886 74.90953632352137) bank161327 +161328 POINT(40.23075328421308 73.19406317545509) bank161328 +161329 POINT(41.09430337587205 74.87312949104484) bank161329 +161330 POINT(41.07938377325631 74.20516985161622) bank161330 +161331 POINT(41.496762885773485 73.0954730529515) bank161331 +161332 POINT(41.478027452119804 73.02700684956042) bank161332 +161333 POINT(40.16125805441807 73.96353559805506) bank161333 +161334 POINT(41.5320473546831 73.97842400734861) bank161334 +161335 POINT(41.130786176581594 74.04036769005644) bank161335 +161336 POINT(41.46893833663285 74.26503480090133) bank161336 +161337 POINT(40.768079349020944 73.25345546181002) bank161337 +161338 POINT(40.42076359010899 73.00947707438598) bank161338 +161339 POINT(40.93934993240938 73.83454923998215) bank161339 +161340 POINT(39.83814934317533 73.40206598626565) bank161340 +161341 POINT(39.89524416134226 74.52123895366195) bank161341 +161342 POINT(40.20972884539886 73.10723694128521) bank161342 +161343 POINT(41.46892751410949 73.43043333276464) bank161343 +161344 POINT(41.43095698541072 74.76118561407993) bank161344 +161345 POINT(39.84003323837578 74.72091046119391) bank161345 +161346 POINT(41.49593414234756 74.80595994279345) bank161346 +161347 POINT(41.56014761552762 74.20885506980463) bank161347 +161348 POINT(41.023369208101734 74.64392443932927) bank161348 +161349 POINT(39.897248527513696 73.54934885488152) bank161349 +161350 POINT(40.89823711283647 74.69843816291554) bank161350 +161351 POINT(40.35491430256186 74.80120841043625) bank161351 +161352 POINT(40.68442162450512 73.05346603861463) bank161352 +161353 POINT(41.58077632255901 74.03002332072938) bank161353 +161354 POINT(40.74936752992818 73.6291620142226) bank161354 +161355 POINT(41.20581570186877 74.05080403657418) bank161355 +161356 POINT(40.70936686642121 74.47729025352177) bank161356 +161357 POINT(39.912942335083166 74.97932870067898) bank161357 +161358 POINT(41.42325909943831 73.75316541603831) bank161358 +161359 POINT(41.08408966484946 73.84530507569951) bank161359 +161360 POINT(39.722069563431766 73.59116835287584) bank161360 +161361 POINT(40.757106175142304 74.9561643673317) bank161361 +161362 POINT(39.784493959419585 74.70122959104891) bank161362 +161363 POINT(41.11003310653526 73.62170006185254) bank161363 +161364 POINT(41.17986639217089 74.72650967122632) bank161364 +161365 POINT(40.27959491107514 73.97588059301265) bank161365 +161366 POINT(40.84568202536442 74.7933829741733) bank161366 +161367 POINT(41.34636799623477 74.5469551918131) bank161367 +161368 POINT(41.09534919493004 74.49128630791797) bank161368 +161369 POINT(41.12272916097675 74.34352192503859) bank161369 +161370 POINT(40.532273285084315 73.57419460770454) bank161370 +161371 POINT(40.9607605943675 74.82547824208812) bank161371 +161372 POINT(41.176438726773846 74.98233659392002) bank161372 +161373 POINT(41.25104668503771 74.36515336934738) bank161373 +161374 POINT(41.70586625490203 73.07299948083305) bank161374 +161375 POINT(40.66852974796129 74.88886709725476) bank161375 +161376 POINT(41.54933574127769 73.19468029734523) bank161376 +161377 POINT(41.006905879822575 73.1117941372875) bank161377 +161378 POINT(40.99128468081474 73.2950996453301) bank161378 +161379 POINT(41.486154573002345 73.07231674939634) bank161379 +161380 POINT(40.388487820122904 73.3484556237012) bank161380 +161381 POINT(41.073218611562865 73.62465934538658) bank161381 +161382 POINT(40.24674538027612 74.1119742819581) bank161382 +161383 POINT(40.87553665266932 73.87882799346679) bank161383 +161384 POINT(41.30673137082127 73.80772239596743) bank161384 +161385 POINT(41.656522774525136 74.9724992525473) bank161385 +161386 POINT(40.24446978490137 73.95485870153632) bank161386 +161387 POINT(39.96314333275773 74.23980687213373) bank161387 +161388 POINT(40.13855054081017 73.94931823406017) bank161388 +161389 POINT(40.822890523295904 74.81841904110492) bank161389 +161390 POINT(39.770133279782016 73.88769273645389) bank161390 +161391 POINT(40.842052586304355 74.41989571174342) bank161391 +161392 POINT(40.70785852092769 74.06560440438604) bank161392 +161393 POINT(40.062622170060884 73.37443469140538) bank161393 +161394 POINT(40.19258499502418 74.29386269312373) bank161394 +161395 POINT(41.17371834931882 73.7042649734084) bank161395 +161396 POINT(40.590259589491595 73.49851021308211) bank161396 +161397 POINT(40.423828110973886 73.41159309185595) bank161397 +161398 POINT(39.970860521932174 73.54543949770594) bank161398 +161399 POINT(40.10388051419084 73.63149859950028) bank161399 +161400 POINT(40.51600231518656 74.01106383096739) bank161400 +161401 POINT(41.581125335514386 74.26699479810488) bank161401 +161402 POINT(40.22819851708683 73.94496592243596) bank161402 +161403 POINT(41.57748271132978 73.96104897102624) bank161403 +161404 POINT(41.43590323258172 74.86535020557125) bank161404 +161405 POINT(40.70855466275525 73.187424752123) bank161405 +161406 POINT(40.53824766562688 74.07218778273631) bank161406 +161407 POINT(39.72025976795647 74.0683248311739) bank161407 +161408 POINT(40.727098781767054 74.96044601479838) bank161408 +161409 POINT(40.886057131368716 73.98392145458068) bank161409 +161410 POINT(41.40082591099893 73.8851863486631) bank161410 +161411 POINT(41.05350000825709 73.25985210176518) bank161411 +161412 POINT(40.78205692331132 73.0849721443313) bank161412 +161413 POINT(40.36594397660316 74.50264414820253) bank161413 +161414 POINT(41.59655397420795 73.53843487171888) bank161414 +161415 POINT(41.440886170527875 73.20784539654397) bank161415 +161416 POINT(41.234095032345174 74.30711252981749) bank161416 +161417 POINT(41.17348216551131 73.7271983591532) bank161417 +161418 POINT(41.031930619869996 73.85181191982748) bank161418 +161419 POINT(40.68379666555796 74.96723940800099) bank161419 +161420 POINT(40.17766247519025 73.55787352514176) bank161420 +161421 POINT(41.392308853360795 74.22033414380307) bank161421 +161422 POINT(40.66637150268635 73.1429061363799) bank161422 +161423 POINT(40.27594139996081 74.6423543404924) bank161423 +161424 POINT(40.874540843906544 73.95775207407073) bank161424 +161425 POINT(41.144985964958984 73.53292309212483) bank161425 +161426 POINT(40.23887928291915 73.95682788223152) bank161426 +161427 POINT(40.32847125646078 74.07092116169224) bank161427 +161428 POINT(41.48165364836917 74.26737135153178) bank161428 +161429 POINT(40.05437838069466 73.6419883206144) bank161429 +161430 POINT(40.021133620360004 73.32194256817529) bank161430 +161431 POINT(40.86779389913313 74.02963772322754) bank161431 +161432 POINT(39.76118535377752 74.56859257444688) bank161432 +161433 POINT(40.530610747327586 73.50330443406759) bank161433 +161434 POINT(39.73413807981411 73.74173164251171) bank161434 +161435 POINT(40.85328091599028 73.08239429792553) bank161435 +161436 POINT(40.60688693031754 74.35443976767186) bank161436 +161437 POINT(41.079926500883445 74.82198473778071) bank161437 +161438 POINT(41.68436723215338 73.94112381443468) bank161438 +161439 POINT(41.68888048510162 74.7636788472328) bank161439 +161440 POINT(41.26666637952373 74.49475637589306) bank161440 +161441 POINT(40.15618362723108 73.03048923853613) bank161441 +161442 POINT(41.644492165757796 74.42763765414425) bank161442 +161443 POINT(40.081556730839274 73.31820385371631) bank161443 +161444 POINT(40.19213256648584 73.3051693824456) bank161444 +161445 POINT(41.35740183398175 73.14395949221374) bank161445 +161446 POINT(39.78751844006702 74.22381289445937) bank161446 +161447 POINT(40.86876943603326 73.5770516493643) bank161447 +161448 POINT(40.22988412498048 73.79718558261148) bank161448 +161449 POINT(39.95769722869934 73.03462737006552) bank161449 +161450 POINT(41.2588552666944 73.72782929199464) bank161450 +161451 POINT(41.35627556747715 74.21754483252255) bank161451 +161452 POINT(40.868442520184225 73.31274407816474) bank161452 +161453 POINT(41.55916342505558 73.62052613288606) bank161453 +161454 POINT(41.278613587994535 74.40654152886869) bank161454 +161455 POINT(40.11785768161749 73.19088885116471) bank161455 +161456 POINT(40.350208173685374 74.73894497622675) bank161456 +161457 POINT(40.84677166999845 73.54338808497366) bank161457 +161458 POINT(41.60878575938347 74.26736495565977) bank161458 +161459 POINT(41.372596135088635 73.92976141692917) bank161459 +161460 POINT(40.85928653883703 74.48105669300168) bank161460 +161461 POINT(39.744276746145125 74.91983137060716) bank161461 +161462 POINT(41.07390111935527 73.85903238083186) bank161462 +161463 POINT(41.52989087091811 73.79452581046293) bank161463 +161464 POINT(40.45795099371889 73.98902676194773) bank161464 +161465 POINT(40.35403218684818 73.4419706882358) bank161465 +161466 POINT(39.81251177501779 73.80111683885761) bank161466 +161467 POINT(41.49347666964017 74.4082502707333) bank161467 +161468 POINT(40.866156296557854 74.09053313645515) bank161468 +161469 POINT(41.6486976961539 74.74863721803638) bank161469 +161470 POINT(40.05665369818174 73.62529625206531) bank161470 +161471 POINT(41.5856168612714 73.33503015466002) bank161471 +161472 POINT(40.936424888404154 73.21600726965056) bank161472 +161473 POINT(40.27358936512509 73.42398271680034) bank161473 +161474 POINT(40.83772251674449 73.95715847820416) bank161474 +161475 POINT(40.85892624832329 73.48502599885785) bank161475 +161476 POINT(40.89758103913364 73.71593031691258) bank161476 +161477 POINT(40.39605566897479 73.78410342874122) bank161477 +161478 POINT(40.238167977446444 73.39700260253966) bank161478 +161479 POINT(41.12558574835368 73.85457553932743) bank161479 +161480 POINT(40.30084647145318 74.05236880813356) bank161480 +161481 POINT(39.78256170304452 73.49585231479477) bank161481 +161482 POINT(39.89825566320509 73.1194749012608) bank161482 +161483 POINT(40.14849808953003 73.11217414271546) bank161483 +161484 POINT(40.67372658213873 73.62889778247134) bank161484 +161485 POINT(41.497259019981406 74.39166137233626) bank161485 +161486 POINT(40.21262649362717 73.71887250343715) bank161486 +161487 POINT(40.94234490797425 73.67176734093546) bank161487 +161488 POINT(40.141007982519035 73.41860871833421) bank161488 +161489 POINT(39.77933742842565 74.38445550627134) bank161489 +161490 POINT(40.683229251852545 74.59640236516806) bank161490 +161491 POINT(41.68181100180305 73.74762495940213) bank161491 +161492 POINT(40.37723822229416 74.13823011450481) bank161492 +161493 POINT(40.46775605179222 74.11074307038523) bank161493 +161494 POINT(40.53347744654031 74.45786677009019) bank161494 +161495 POINT(40.4553731931053 73.59586356831464) bank161495 +161496 POINT(40.407441097455965 74.02918141443661) bank161496 +161497 POINT(41.68577280405827 73.52090838978215) bank161497 +161498 POINT(41.652264036729505 74.40637302527382) bank161498 +161499 POINT(40.46470627922082 74.51269633720065) bank161499 +161500 POINT(40.83257880054141 74.08909830469472) bank161500 +161501 POINT(40.908652292894736 73.93795101484743) bank161501 +161502 POINT(39.804099228223436 73.60500509563022) bank161502 +161503 POINT(39.87705190530603 74.3990373339993) bank161503 +161504 POINT(40.536774533016285 75.00083169210741) bank161504 +161505 POINT(40.325950703078405 74.7437827596029) bank161505 +161506 POINT(40.80007164937953 74.34050383023389) bank161506 +161507 POINT(41.054182727634654 74.83815719822462) bank161507 +161508 POINT(40.45903475946 74.57020056707843) bank161508 +161509 POINT(40.00006611291613 74.06339950983713) bank161509 +161510 POINT(39.98869240202661 74.11762227378117) bank161510 +161511 POINT(40.316971578427044 74.59525576034504) bank161511 +161512 POINT(41.374531617414 74.90713737506412) bank161512 +161513 POINT(40.22650886826555 73.68673881745612) bank161513 +161514 POINT(41.53361283132895 73.69727924542828) bank161514 +161515 POINT(41.1311014051307 73.77845231710414) bank161515 +161516 POINT(39.84959002968659 73.1268375693918) bank161516 +161517 POINT(41.34707353180012 74.24757744360353) bank161517 +161518 POINT(41.37995576877728 74.96110808590208) bank161518 +161519 POINT(40.695575691196346 73.64399791799191) bank161519 +161520 POINT(39.74761898139797 73.0623076100728) bank161520 +161521 POINT(41.01303078428503 74.9765302836829) bank161521 +161522 POINT(41.33882103478831 73.61944922670598) bank161522 +161523 POINT(39.790677137465195 74.3895228112212) bank161523 +161524 POINT(40.419366618176674 74.41386136453546) bank161524 +161525 POINT(40.40713719239019 73.16137467363703) bank161525 +161526 POINT(41.47311322754318 73.45780138387633) bank161526 +161527 POINT(40.638840155764456 74.27443249784645) bank161527 +161528 POINT(40.82198234198237 73.90253331253423) bank161528 +161529 POINT(41.01826291405533 74.9094814712644) bank161529 +161530 POINT(41.32694277714224 74.3171991487213) bank161530 +161531 POINT(40.939166040450075 73.33483917938841) bank161531 +161532 POINT(40.47548924961607 73.84759166846639) bank161532 +161533 POINT(41.482261463497316 73.09661499181179) bank161533 +161534 POINT(40.50609867494435 74.27603937059835) bank161534 +161535 POINT(40.15798632796131 74.10499192038026) bank161535 +161536 POINT(39.84064926302987 74.75500769250976) bank161536 +161537 POINT(39.912762310735836 73.21783865843487) bank161537 +161538 POINT(40.04722528119136 73.88975395758061) bank161538 +161539 POINT(40.64818683779478 74.1629871001097) bank161539 +161540 POINT(41.283869769982886 74.79902413083207) bank161540 +161541 POINT(39.99728651776222 73.19671421967719) bank161541 +161542 POINT(40.72592619334204 73.30721712748787) bank161542 +161543 POINT(40.66886490968422 73.26435190766462) bank161543 +161544 POINT(41.40576997383721 73.1385770591769) bank161544 +161545 POINT(40.10120310161566 74.25995906162053) bank161545 +161546 POINT(41.62246893175011 73.95960887390828) bank161546 +161547 POINT(40.26622525336634 73.58801328164053) bank161547 +161548 POINT(40.51627791409695 74.05131320848167) bank161548 +161549 POINT(40.11239581684687 74.5583732604424) bank161549 +161550 POINT(40.52063136597812 74.17933953037902) bank161550 +161551 POINT(40.76141108442238 74.06114702356393) bank161551 +161552 POINT(39.99209300351121 73.56105252688556) bank161552 +161553 POINT(41.58693335817923 74.82099077725978) bank161553 +161554 POINT(40.134269143164865 73.02834800978465) bank161554 +161555 POINT(41.604962749386274 73.10179152326722) bank161555 +161556 POINT(41.68238642243035 74.47899995691515) bank161556 +161557 POINT(40.230234213705735 74.28711399917222) bank161557 +161558 POINT(40.50942914309955 73.23232121049652) bank161558 +161559 POINT(41.151842157599965 74.3703670442717) bank161559 +161560 POINT(40.896960865885404 73.13484818493322) bank161560 +161561 POINT(40.831961502284024 74.88002933794002) bank161561 +161562 POINT(40.497054912342804 73.66679953834517) bank161562 +161563 POINT(41.2947540054762 74.15292128604491) bank161563 +161564 POINT(39.763577684220714 74.58701763614093) bank161564 +161565 POINT(40.10561725020348 74.52933960706552) bank161565 +161566 POINT(40.91552271333645 74.46369508567821) bank161566 +161567 POINT(40.981069776876836 74.69346870144477) bank161567 +161568 POINT(40.24374318927934 73.09070266017831) bank161568 +161569 POINT(41.650354214614396 74.60985627877662) bank161569 +161570 POINT(41.103904323208276 73.2307114914572) bank161570 +161571 POINT(41.62694961280733 73.69482592525613) bank161571 +161572 POINT(39.86794514254269 73.37795927589993) bank161572 +161573 POINT(41.63932474089798 73.36912747724816) bank161573 +161574 POINT(41.1324644268105 73.47897842931239) bank161574 +161575 POINT(40.64352462254397 74.38353161194443) bank161575 +161576 POINT(41.42320149058227 74.51373818633682) bank161576 +161577 POINT(40.97835475249079 73.27665969720277) bank161577 +161578 POINT(40.180550414601136 73.15716898371527) bank161578 +161579 POINT(40.07408286167496 74.04497050552304) bank161579 +161580 POINT(39.847419388216764 74.17937688279451) bank161580 +161581 POINT(39.79599522047276 73.83835100397577) bank161581 +161582 POINT(39.906677762160555 74.83210035588722) bank161582 +161583 POINT(40.80091052833042 73.72045633689697) bank161583 +161584 POINT(40.202238223084805 73.43384378011956) bank161584 +161585 POINT(40.757224639217306 73.50622310395008) bank161585 +161586 POINT(41.380759519807256 73.3110387608653) bank161586 +161587 POINT(39.82309314668426 73.18887656174975) bank161587 +161588 POINT(39.76514347151011 73.34437900076293) bank161588 +161589 POINT(40.69276163771528 73.54416133188255) bank161589 +161590 POINT(40.217847961747076 74.95223240838656) bank161590 +161591 POINT(40.03355568562294 73.15225440168862) bank161591 +161592 POINT(40.86521291841828 73.15415222997363) bank161592 +161593 POINT(40.78030227900526 73.28604290684684) bank161593 +161594 POINT(40.17519962348494 74.11876856360661) bank161594 +161595 POINT(41.33969858292037 74.53735203192471) bank161595 +161596 POINT(41.52652601906472 73.90000982088976) bank161596 +161597 POINT(39.92542317955784 73.81825035674417) bank161597 +161598 POINT(40.4709681962126 73.5974513166293) bank161598 +161599 POINT(41.23035797826546 73.93325130782924) bank161599 +161600 POINT(41.31326978758631 74.83510004980253) bank161600 +161601 POINT(40.02492613957746 74.7345783044682) bank161601 +161602 POINT(40.83960809384002 73.91064524779725) bank161602 +161603 POINT(41.054603131134115 74.99126487293468) bank161603 +161604 POINT(41.31503060391276 74.84271110803009) bank161604 +161605 POINT(40.18621032546037 74.94568169942022) bank161605 +161606 POINT(40.146723708204725 74.18557694693096) bank161606 +161607 POINT(41.06868534150541 73.71636187870189) bank161607 +161608 POINT(40.36141726241231 74.96367987854707) bank161608 +161609 POINT(41.480411911241355 73.32597134709148) bank161609 +161610 POINT(41.02197461318416 73.07759039777368) bank161610 +161611 POINT(41.02339732175804 73.89046381320559) bank161611 +161612 POINT(41.66508250548483 74.11121750290624) bank161612 +161613 POINT(40.72985987967403 74.75989527552055) bank161613 +161614 POINT(41.13887064589136 74.40451849327279) bank161614 +161615 POINT(40.640698608943374 73.75906556335624) bank161615 +161616 POINT(40.44835267532342 74.06759380215571) bank161616 +161617 POINT(41.18673775485511 73.97767238355561) bank161617 +161618 POINT(41.487615809678914 73.52641249141348) bank161618 +161619 POINT(40.10416235712864 73.1723114543201) bank161619 +161620 POINT(39.741468041927426 74.4048604503351) bank161620 +161621 POINT(40.34332687450159 74.86105549535968) bank161621 +161622 POINT(40.224248371792044 73.15317839968384) bank161622 +161623 POINT(40.577183649372685 73.58880953471453) bank161623 +161624 POINT(40.719342651827944 73.70775766760802) bank161624 +161625 POINT(41.593562878070585 74.5374851041268) bank161625 +161626 POINT(40.49270223601351 74.06499584701322) bank161626 +161627 POINT(39.88376760570113 73.63530356461354) bank161627 +161628 POINT(40.32770133802968 73.70431105691073) bank161628 +161629 POINT(41.12665109189002 74.24849199320941) bank161629 +161630 POINT(41.601258414594724 73.79346728684855) bank161630 +161631 POINT(41.182039767195825 73.47074774714544) bank161631 +161632 POINT(40.063035695190294 73.95320377391312) bank161632 +161633 POINT(40.6237499510439 74.58246311508644) bank161633 +161634 POINT(39.885607571126826 73.18820414844383) bank161634 +161635 POINT(40.22246578408981 74.34911014537383) bank161635 +161636 POINT(39.784898972927856 73.0247607359277) bank161636 +161637 POINT(40.800589442046295 73.73060568096588) bank161637 +161638 POINT(40.293466158650084 74.6750953407328) bank161638 +161639 POINT(41.10911465703978 73.47217194590793) bank161639 +161640 POINT(41.1862000476951 74.50505534272565) bank161640 +161641 POINT(40.62321713328281 73.00751899172104) bank161641 +161642 POINT(41.26112915104777 73.50710622259216) bank161642 +161643 POINT(40.99148618196211 74.08357585766983) bank161643 +161644 POINT(41.23623071188994 74.10565809619348) bank161644 +161645 POINT(39.890116654186535 73.3229810557302) bank161645 +161646 POINT(41.14719356811298 74.01590441438717) bank161646 +161647 POINT(41.40305111831414 73.04884768945483) bank161647 +161648 POINT(41.25592055129228 73.17728977484035) bank161648 +161649 POINT(39.92672153943564 73.9573363542231) bank161649 +161650 POINT(40.41175092204576 73.0779135061605) bank161650 +161651 POINT(40.87379554264824 73.05833432286022) bank161651 +161652 POINT(39.9797639335411 73.91100685156233) bank161652 +161653 POINT(40.98496816121826 74.48781176377524) bank161653 +161654 POINT(41.414644241213 74.26765996378855) bank161654 +161655 POINT(41.66851979942223 73.06486253829682) bank161655 +161656 POINT(39.96644497434568 74.87459043249254) bank161656 +161657 POINT(39.820655554491196 73.07437442330405) bank161657 +161658 POINT(40.94199387298761 73.4005858866572) bank161658 +161659 POINT(40.99630479361368 74.22828771146122) bank161659 +161660 POINT(41.54295663106918 73.6922472723884) bank161660 +161661 POINT(40.35637614634492 74.4751582495686) bank161661 +161662 POINT(40.38858481634895 73.90877179949258) bank161662 +161663 POINT(40.33649629965824 73.59499945199376) bank161663 +161664 POINT(41.08161741757913 73.73626676259519) bank161664 +161665 POINT(41.60976797660308 74.11764615653175) bank161665 +161666 POINT(41.41267179141092 74.42515907022694) bank161666 +161667 POINT(40.93043163909436 74.24306799341872) bank161667 +161668 POINT(40.23157155786174 74.46530387787531) bank161668 +161669 POINT(40.436013122972135 73.0856081506779) bank161669 +161670 POINT(40.50057704939499 73.33956635283015) bank161670 +161671 POINT(40.86264446420678 74.48368984202385) bank161671 +161672 POINT(40.39360727514882 73.72658284907175) bank161672 +161673 POINT(40.35455453296778 74.3181143015042) bank161673 +161674 POINT(40.347285855989 73.57227522258817) bank161674 +161675 POINT(40.01449701875346 73.05643146044186) bank161675 +161676 POINT(41.5833149012643 73.72759459168444) bank161676 +161677 POINT(40.43667666331394 73.93558047671456) bank161677 +161678 POINT(40.2527728992756 74.68179234532623) bank161678 +161679 POINT(40.83620844870665 73.29698297132227) bank161679 +161680 POINT(41.556557572902435 73.62999857000862) bank161680 +161681 POINT(41.31619687635286 73.4181564630481) bank161681 +161682 POINT(40.697120858734486 73.14527168450181) bank161682 +161683 POINT(41.18229093252218 74.93049454099537) bank161683 +161684 POINT(40.34387686826352 73.35029647557548) bank161684 +161685 POINT(39.71389140053799 73.01387785645272) bank161685 +161686 POINT(40.70655306871103 74.13311641572517) bank161686 +161687 POINT(40.20525700772958 74.37877058454731) bank161687 +161688 POINT(41.20116863896017 73.90833566165504) bank161688 +161689 POINT(39.723229367316385 74.27373010647037) bank161689 +161690 POINT(39.970723882678364 73.39387458795814) bank161690 +161691 POINT(40.74626583817525 73.3624330206606) bank161691 +161692 POINT(41.23948231748214 74.02918269848071) bank161692 +161693 POINT(40.957120463193704 73.25505357275854) bank161693 +161694 POINT(41.12750708611591 74.39856842691454) bank161694 +161695 POINT(41.02994515550489 74.50305457999013) bank161695 +161696 POINT(40.692770047740645 74.06305527066773) bank161696 +161697 POINT(40.872625677850806 73.71839814502988) bank161697 +161698 POINT(39.72834931687486 74.64739990683407) bank161698 +161699 POINT(39.806215870763744 73.79439677766796) bank161699 +161700 POINT(39.860997503505814 73.36201390844346) bank161700 +161701 POINT(39.97510335255578 74.71349621581379) bank161701 +161702 POINT(39.95836981337093 73.78527283244419) bank161702 +161703 POINT(39.97740335878289 73.8936570307953) bank161703 +161704 POINT(40.11320757824754 73.91426079303135) bank161704 +161705 POINT(40.874237137048006 74.09610142194826) bank161705 +161706 POINT(40.37917703058029 73.7539645113757) bank161706 +161707 POINT(40.15928358437565 74.62967936265758) bank161707 +161708 POINT(40.98629325670626 73.21720140998467) bank161708 +161709 POINT(40.02731994364276 74.18288515978809) bank161709 +161710 POINT(39.885681342649555 73.04754468208608) bank161710 +161711 POINT(41.013166222618175 73.43255818946017) bank161711 +161712 POINT(40.54286548516333 73.832068973543) bank161712 +161713 POINT(39.943112833286975 73.32796029743616) bank161713 +161714 POINT(40.26821302176688 74.29734091183677) bank161714 +161715 POINT(40.23149604098087 74.36961972564207) bank161715 +161716 POINT(41.010598773092795 74.86998639389743) bank161716 +161717 POINT(40.18126556890749 74.66567986476822) bank161717 +161718 POINT(40.900848284152445 74.44298320389414) bank161718 +161719 POINT(41.11289112625728 73.34441500889162) bank161719 +161720 POINT(41.59497613369297 73.14656499873894) bank161720 +161721 POINT(40.18499465339727 74.21085794642353) bank161721 +161722 POINT(39.82255369723118 74.25671889099725) bank161722 +161723 POINT(40.61375658920579 74.62921777785367) bank161723 +161724 POINT(40.08868149752774 73.75488974450987) bank161724 +161725 POINT(39.717699984998504 74.41336389622265) bank161725 +161726 POINT(41.56123305190898 73.31368616208094) bank161726 +161727 POINT(41.46524901003951 74.38899935711218) bank161727 +161728 POINT(39.79710359746682 73.50127108865622) bank161728 +161729 POINT(40.61754428549855 73.58096243497084) bank161729 +161730 POINT(41.24263690006399 74.37673845360955) bank161730 +161731 POINT(41.61312443787136 74.9200407121578) bank161731 +161732 POINT(40.69246160486031 74.58902368908083) bank161732 +161733 POINT(39.94502028483439 74.56245652660049) bank161733 +161734 POINT(41.429514958125324 74.54495992678596) bank161734 +161735 POINT(41.26884276096317 73.26749059566475) bank161735 +161736 POINT(39.94520461856243 73.1615148916655) bank161736 +161737 POINT(40.7828903903594 74.11871976081648) bank161737 +161738 POINT(41.2584130757067 73.37822107333959) bank161738 +161739 POINT(41.04435663147708 73.64678196710459) bank161739 +161740 POINT(40.7812919921398 74.80186309401581) bank161740 +161741 POINT(41.66648125526807 74.5718092607345) bank161741 +161742 POINT(39.78405822422348 73.19685897186471) bank161742 +161743 POINT(40.25952853041175 73.91108245447406) bank161743 +161744 POINT(40.16682476059687 73.09119732731699) bank161744 +161745 POINT(41.36992174117336 73.54339258391131) bank161745 +161746 POINT(39.808173169358476 73.74740363648522) bank161746 +161747 POINT(40.11985587838218 74.46257106814889) bank161747 +161748 POINT(40.52144907129379 74.6782137829551) bank161748 +161749 POINT(41.005898937482335 73.2976289287106) bank161749 +161750 POINT(39.96367789209904 73.43345250590137) bank161750 +161751 POINT(40.49081092977665 73.45127797320458) bank161751 +161752 POINT(39.92013429707078 74.71487939707085) bank161752 +161753 POINT(40.72122226577525 74.5108485047083) bank161753 +161754 POINT(39.828871590203036 73.81132420135246) bank161754 +161755 POINT(41.6444990729595 73.07343433192041) bank161755 +161756 POINT(40.62849448998385 73.63312270228035) bank161756 +161757 POINT(40.78022689455129 74.09532507795292) bank161757 +161758 POINT(39.79036572564897 73.09631359936509) bank161758 +161759 POINT(40.72856620424726 74.00418987852699) bank161759 +161760 POINT(40.614984115018295 73.41465189633523) bank161760 +161761 POINT(39.97835665039159 74.01979189784817) bank161761 +161762 POINT(41.26767359513022 74.15205677129852) bank161762 +161763 POINT(39.81255675615411 74.57815244972619) bank161763 +161764 POINT(40.3942105115903 73.1567819377336) bank161764 +161765 POINT(41.4620353240846 73.61493041786196) bank161765 +161766 POINT(40.81412937444691 74.42996069408439) bank161766 +161767 POINT(40.13435436505268 74.75372461899335) bank161767 +161768 POINT(40.93677632301903 74.60029656889796) bank161768 +161769 POINT(41.49338245101511 74.54554115644918) bank161769 +161770 POINT(40.13905623188638 74.35387849800448) bank161770 +161771 POINT(41.247991667760985 73.32419478397573) bank161771 +161772 POINT(41.67863037180341 73.38816478838763) bank161772 +161773 POINT(40.700275754627995 74.68024672952649) bank161773 +161774 POINT(39.998396643219124 73.66672611808899) bank161774 +161775 POINT(40.33547555683984 74.5791545514419) bank161775 +161776 POINT(40.28355378756679 74.92690007707967) bank161776 +161777 POINT(39.739342549843684 74.39956090569271) bank161777 +161778 POINT(40.98878081348557 73.60188250538867) bank161778 +161779 POINT(40.89172567449289 73.31178798443293) bank161779 +161780 POINT(39.726948174849326 73.59603886231437) bank161780 +161781 POINT(41.53590790415771 73.50779622209237) bank161781 +161782 POINT(41.38949477058188 74.96056816565431) bank161782 +161783 POINT(40.88924002022571 74.94123968790605) bank161783 +161784 POINT(40.45072703972789 73.18886434970432) bank161784 +161785 POINT(40.72089144103128 74.19097655012682) bank161785 +161786 POINT(41.693535800391366 74.6654155284161) bank161786 +161787 POINT(40.8895956578062 73.59717657444571) bank161787 +161788 POINT(40.63991894258336 74.60370251231228) bank161788 +161789 POINT(40.65681279238428 74.88379522671377) bank161789 +161790 POINT(41.68059218915598 73.281432857924) bank161790 +161791 POINT(39.78304183523414 73.77327362415278) bank161791 +161792 POINT(40.66003993515138 74.52725935822444) bank161792 +161793 POINT(40.08094869745373 73.59414421446549) bank161793 +161794 POINT(41.00234329795411 73.88645631591547) bank161794 +161795 POINT(40.474819390648804 74.49942198881766) bank161795 +161796 POINT(40.908380007254095 74.32959381910015) bank161796 +161797 POINT(41.05329440631993 74.9179649396906) bank161797 +161798 POINT(39.86644694719034 74.41826287873505) bank161798 +161799 POINT(41.48935559982132 74.41687437189927) bank161799 +161800 POINT(40.675979519223375 73.58136647975249) bank161800 +161801 POINT(40.425869863058246 73.7628554222461) bank161801 +161802 POINT(40.74660698473667 74.12783236035617) bank161802 +161803 POINT(41.624278666057016 74.27567684183705) bank161803 +161804 POINT(40.80642212917975 73.60303712827893) bank161804 +161805 POINT(39.75512828185652 74.19451431136099) bank161805 +161806 POINT(39.78095845951882 73.92752130283478) bank161806 +161807 POINT(40.45892672085577 73.58058005030539) bank161807 +161808 POINT(41.534349938293374 73.28038597189207) bank161808 +161809 POINT(41.37440078898024 74.82432433136542) bank161809 +161810 POINT(40.167252319143465 74.7975033198487) bank161810 +161811 POINT(41.69545012095882 74.6639517070589) bank161811 +161812 POINT(39.806920945347144 74.76111525195257) bank161812 +161813 POINT(39.838483410208106 74.81878809069309) bank161813 +161814 POINT(40.536760659922685 73.28230357698195) bank161814 +161815 POINT(40.893529528003945 74.16443209873285) bank161815 +161816 POINT(40.840108092392924 74.7248435057232) bank161816 +161817 POINT(40.65337732755863 74.67515262494419) bank161817 +161818 POINT(40.09711480616078 74.8038429994799) bank161818 +161819 POINT(40.45678306232171 74.83184637253832) bank161819 +161820 POINT(40.34388488833809 74.91533221725305) bank161820 +161821 POINT(40.084577759675724 74.02615806472598) bank161821 +161822 POINT(40.38550971972741 73.1874243233687) bank161822 +161823 POINT(40.569060752976775 73.50999351928705) bank161823 +161824 POINT(39.75879125422498 73.38969054066303) bank161824 +161825 POINT(41.62091393140509 73.81517387657031) bank161825 +161826 POINT(41.31884023580735 73.65397404554915) bank161826 +161827 POINT(39.83510742817844 74.67177092494806) bank161827 +161828 POINT(41.54938071957903 74.0551149186618) bank161828 +161829 POINT(41.52954345175838 74.38554874210357) bank161829 +161830 POINT(40.870009054343996 73.73467075740629) bank161830 +161831 POINT(40.99788001491903 73.96073304149006) bank161831 +161832 POINT(41.27622543815366 73.00617452717847) bank161832 +161833 POINT(40.27005722329008 73.63230041320199) bank161833 +161834 POINT(40.834894238747616 74.06784754440199) bank161834 +161835 POINT(39.93955900655491 74.13295211479493) bank161835 +161836 POINT(40.460597164231636 73.38458944732) bank161836 +161837 POINT(41.32914249617661 73.37474542615948) bank161837 +161838 POINT(41.28938732566381 73.66114469707529) bank161838 +161839 POINT(40.27482156537986 74.1871300675431) bank161839 +161840 POINT(41.35887575752839 73.74815267818745) bank161840 +161841 POINT(41.39928024784766 74.53431619918429) bank161841 +161842 POINT(40.643702951423904 73.15166835642592) bank161842 +161843 POINT(39.92400792459776 74.1016906970811) bank161843 +161844 POINT(41.263484679943225 74.84772021376577) bank161844 +161845 POINT(41.59526931272804 73.89862953203644) bank161845 +161846 POINT(41.53384258318396 74.05189724571568) bank161846 +161847 POINT(40.05492136299536 74.78768722917488) bank161847 +161848 POINT(41.32142585980567 73.97939991292317) bank161848 +161849 POINT(40.355139439770774 73.9131651841261) bank161849 +161850 POINT(41.2475660916278 73.82259324867796) bank161850 +161851 POINT(40.06549757374954 74.01333425805339) bank161851 +161852 POINT(41.596060036707776 73.8141055808352) bank161852 +161853 POINT(40.59865606868344 74.28313899821443) bank161853 +161854 POINT(41.500169558038095 73.98791202075317) bank161854 +161855 POINT(41.49898224774533 74.98220413051008) bank161855 +161856 POINT(41.59048319648642 74.59876959191104) bank161856 +161857 POINT(40.04565768729521 73.53326708259978) bank161857 +161858 POINT(41.66591552223934 73.34511929355149) bank161858 +161859 POINT(41.16675499568661 73.9435280456047) bank161859 +161860 POINT(41.63040482589528 73.5722546282635) bank161860 +161861 POINT(40.86814824049699 74.98854854903254) bank161861 +161862 POINT(41.12319192302939 74.96853882424706) bank161862 +161863 POINT(41.09239174379534 73.46571477022334) bank161863 +161864 POINT(41.07740486938745 73.0721599891979) bank161864 +161865 POINT(41.16221201162584 74.80602330169825) bank161865 +161866 POINT(40.20294782161046 74.87591687595881) bank161866 +161867 POINT(40.367775567052966 74.06216765631866) bank161867 +161868 POINT(40.267948184122965 73.03771854249894) bank161868 +161869 POINT(39.86526929143279 74.08509847221299) bank161869 +161870 POINT(40.20171825726452 74.05907272454647) bank161870 +161871 POINT(39.94584096439901 74.05547323780065) bank161871 +161872 POINT(39.81055182604336 74.59662262949969) bank161872 +161873 POINT(40.55645836020589 74.52209551208487) bank161873 +161874 POINT(39.98571708121306 74.4170172312868) bank161874 +161875 POINT(40.20272382487104 74.49229607569795) bank161875 +161876 POINT(41.16114979132644 73.60749131600085) bank161876 +161877 POINT(40.11255956709378 74.66987175510117) bank161877 +161878 POINT(40.49787929184628 73.77555139891868) bank161878 +161879 POINT(39.97764012805785 73.01017462742813) bank161879 +161880 POINT(41.66072646421585 73.18084501793625) bank161880 +161881 POINT(41.277059897494176 73.51605778093283) bank161881 +161882 POINT(41.32886279917699 74.89843379497202) bank161882 +161883 POINT(40.21971228524131 74.94201577620112) bank161883 +161884 POINT(40.22392356404875 74.30282606910794) bank161884 +161885 POINT(41.41601833047038 73.04820943899394) bank161885 +161886 POINT(40.736758554617644 74.89514646657338) bank161886 +161887 POINT(41.05852971507226 74.53824344958888) bank161887 +161888 POINT(40.01812403477372 74.51152588423132) bank161888 +161889 POINT(41.30970452365098 74.19431046754197) bank161889 +161890 POINT(40.636897962168376 74.98089825093052) bank161890 +161891 POINT(40.7898060201316 73.09697245292497) bank161891 +161892 POINT(39.95018602331117 74.32130945867614) bank161892 +161893 POINT(40.4221618788439 73.25383207615798) bank161893 +161894 POINT(40.85485700588676 74.69517931809153) bank161894 +161895 POINT(39.852261010088206 73.34655288884186) bank161895 +161896 POINT(40.68029372457653 73.67163687446609) bank161896 +161897 POINT(41.17110267669277 74.10040351533603) bank161897 +161898 POINT(40.92668574312585 73.25594829384728) bank161898 +161899 POINT(40.9217179780419 74.2636845215267) bank161899 +161900 POINT(39.833400866017975 74.08844069537251) bank161900 +161901 POINT(41.148967014397364 73.2736505131196) bank161901 +161902 POINT(41.15879494485399 73.29654385732464) bank161902 +161903 POINT(41.66044046362431 74.19681332958352) bank161903 +161904 POINT(40.39480247672739 73.58962740890495) bank161904 +161905 POINT(40.65603019004129 73.74067287207686) bank161905 +161906 POINT(40.45305202188461 73.70777116357425) bank161906 +161907 POINT(40.81837292814213 73.1543215716514) bank161907 +161908 POINT(41.453844202757956 74.25991362537238) bank161908 +161909 POINT(41.39393099667258 74.85930762472404) bank161909 +161910 POINT(39.71652350793191 74.70586913959293) bank161910 +161911 POINT(40.20361330793532 74.68970911245631) bank161911 +161912 POINT(41.19942986480289 73.7523037393729) bank161912 +161913 POINT(39.762228125283144 73.90886718607484) bank161913 +161914 POINT(39.95468291172403 73.28851705298813) bank161914 +161915 POINT(40.02761734690761 74.16373434234515) bank161915 +161916 POINT(41.07323460593809 74.42383752633295) bank161916 +161917 POINT(41.669872248582536 74.19581735609816) bank161917 +161918 POINT(41.192332006158566 74.86446192910883) bank161918 +161919 POINT(39.82423748274492 73.53448277921242) bank161919 +161920 POINT(40.46718514741003 73.86489774292286) bank161920 +161921 POINT(40.357740564480714 73.56687273547904) bank161921 +161922 POINT(40.8602485544976 74.69855711198183) bank161922 +161923 POINT(39.81463499761305 73.12605469304758) bank161923 +161924 POINT(40.896432973099024 74.20558335157416) bank161924 +161925 POINT(41.590115608477475 73.92022050761265) bank161925 +161926 POINT(40.952421668038376 74.89578891258249) bank161926 +161927 POINT(40.92040276942675 73.238143590934) bank161927 +161928 POINT(40.73907881692008 74.67868627455108) bank161928 +161929 POINT(40.64611943126408 73.51339492428971) bank161929 +161930 POINT(39.7643827007745 73.09651534618676) bank161930 +161931 POINT(41.45886825133436 74.29400558002156) bank161931 +161932 POINT(40.58290630272353 73.84383109659774) bank161932 +161933 POINT(41.681728727052175 74.08200671546857) bank161933 +161934 POINT(41.59269863018882 74.75651912788493) bank161934 +161935 POINT(41.60600214096195 74.13168608804939) bank161935 +161936 POINT(40.43255674189707 73.29880257694492) bank161936 +161937 POINT(41.37635175724144 73.96823147786561) bank161937 +161938 POINT(40.53830184356798 73.72511458706461) bank161938 +161939 POINT(40.59639338056524 74.84092254047434) bank161939 +161940 POINT(40.32725035769985 73.51102023424194) bank161940 +161941 POINT(41.22351258555085 73.09304955015307) bank161941 +161942 POINT(40.68785916007314 74.66834787276872) bank161942 +161943 POINT(41.20061472707348 74.52973015296212) bank161943 +161944 POINT(39.717908027982695 73.7547756871089) bank161944 +161945 POINT(40.295343147590984 74.88827578035895) bank161945 +161946 POINT(40.26293845968736 74.7427075527351) bank161946 +161947 POINT(41.01123623080711 73.26172479151083) bank161947 +161948 POINT(40.540619853306346 73.42382665541518) bank161948 +161949 POINT(40.975064087988514 73.30977203081646) bank161949 +161950 POINT(39.99002085520433 74.45694676667794) bank161950 +161951 POINT(40.62256359895648 74.08566670339702) bank161951 +161952 POINT(41.47066394025925 74.59933322087953) bank161952 +161953 POINT(40.254970761219155 73.09811242790977) bank161953 +161954 POINT(40.8949176318113 74.55387141738134) bank161954 +161955 POINT(41.32036946090788 73.12014792952854) bank161955 +161956 POINT(39.99796817014212 73.65493559791275) bank161956 +161957 POINT(41.513235849398896 74.62982578238075) bank161957 +161958 POINT(41.538888337272816 73.14928589817451) bank161958 +161959 POINT(40.63705202311838 74.73430406201983) bank161959 +161960 POINT(41.08297152900107 74.01760825018758) bank161960 +161961 POINT(39.955373964022094 74.21732292514389) bank161961 +161962 POINT(41.0243837629949 73.04561387439753) bank161962 +161963 POINT(40.58728446465911 74.07506225217836) bank161963 +161964 POINT(40.17649079242881 73.10447255390127) bank161964 +161965 POINT(41.09185464611974 73.31688597388656) bank161965 +161966 POINT(40.36525244174463 73.77034152903632) bank161966 +161967 POINT(39.86703866840841 73.30677090668469) bank161967 +161968 POINT(40.16064228769174 73.18793613069619) bank161968 +161969 POINT(41.48916766315328 73.87462964837665) bank161969 +161970 POINT(39.792995941305925 74.55345845409543) bank161970 +161971 POINT(41.58589264713848 73.80242202715137) bank161971 +161972 POINT(40.159459704317506 73.51607689688048) bank161972 +161973 POINT(39.83891164583445 73.5980993027726) bank161973 +161974 POINT(41.25752233131138 74.4931736759161) bank161974 +161975 POINT(40.812535935901685 73.35214706560396) bank161975 +161976 POINT(40.84968381643877 74.74207937129157) bank161976 +161977 POINT(40.29167918047463 73.66898584474964) bank161977 +161978 POINT(40.83721464661648 73.71825727459198) bank161978 +161979 POINT(40.067889400924315 73.50606020958651) bank161979 +161980 POINT(40.83730295338071 74.07268236497491) bank161980 +161981 POINT(40.93272287972418 74.362578778218) bank161981 +161982 POINT(41.206297367875244 73.26688184996685) bank161982 +161983 POINT(41.64818234823002 74.68208827800959) bank161983 +161984 POINT(40.604423451205705 73.37014611841502) bank161984 +161985 POINT(41.45592259704939 74.24927731600748) bank161985 +161986 POINT(41.04955156878802 73.43160560432528) bank161986 +161987 POINT(39.941137277241324 73.89818866107268) bank161987 +161988 POINT(41.16402495944919 74.29145630954726) bank161988 +161989 POINT(41.424787468367285 73.23164075429915) bank161989 +161990 POINT(40.46390245796726 73.8419049554249) bank161990 +161991 POINT(40.181249142346715 74.9389032546276) bank161991 +161992 POINT(40.27056866786402 74.74021468511532) bank161992 +161993 POINT(41.46334380918498 73.62171685621375) bank161993 +161994 POINT(40.75201848115609 74.22291624669403) bank161994 +161995 POINT(39.97604459300245 73.80621221883753) bank161995 +161996 POINT(39.95385411679653 73.72444866421999) bank161996 +161997 POINT(41.05742998722661 73.43841798271656) bank161997 +161998 POINT(41.38152394736225 73.84633512746422) bank161998 +161999 POINT(40.376732380051855 74.03946913515453) bank161999 +162000 POINT(41.71118999104715 74.84401687375869) bank162000 +162001 POINT(40.847411044700785 73.46168615701127) bank162001 +162002 POINT(40.12322658502626 73.45139119029844) bank162002 +162003 POINT(39.79040036955925 73.29516420142481) bank162003 +162004 POINT(40.032187374278486 73.98125951089854) bank162004 +162005 POINT(41.05084765828284 74.38711286729428) bank162005 +162006 POINT(40.80883624725988 73.23157897688658) bank162006 +162007 POINT(40.25864922564589 74.1503890592099) bank162007 +162008 POINT(39.7304871800681 74.44843617572674) bank162008 +162009 POINT(41.617553282251755 73.04624246876425) bank162009 +162010 POINT(40.332307463143025 74.01757130492128) bank162010 +162011 POINT(41.4189582160035 73.76458086385547) bank162011 +162012 POINT(41.068477708287446 74.4604914297651) bank162012 +162013 POINT(41.09639918459791 73.01442060082769) bank162013 +162014 POINT(41.56206228005375 73.16134470545572) bank162014 +162015 POINT(40.55502928955004 73.72385083996522) bank162015 +162016 POINT(40.35443653280717 73.29184486375362) bank162016 +162017 POINT(40.17901466526818 73.85733939192819) bank162017 +162018 POINT(40.656136594137614 73.9276342960385) bank162018 +162019 POINT(39.73501450732436 74.90066869473117) bank162019 +162020 POINT(40.47495474307988 73.20603010443872) bank162020 +162021 POINT(40.99779723406068 73.91933227481651) bank162021 +162022 POINT(41.4283445717025 74.69046460859623) bank162022 +162023 POINT(41.25253264387192 73.56288472846006) bank162023 +162024 POINT(40.388301560428395 73.8231494743511) bank162024 +162025 POINT(40.90914202764829 73.47318260755476) bank162025 +162026 POINT(40.790631529588055 74.0459755242231) bank162026 +162027 POINT(41.49804935854771 73.81424541375728) bank162027 +162028 POINT(40.75544516238474 73.2934791032711) bank162028 +162029 POINT(40.73803659878436 74.17756475289241) bank162029 +162030 POINT(41.32592653138053 73.71004654937757) bank162030 +162031 POINT(40.69204873781727 73.26717593164777) bank162031 +162032 POINT(41.67801974596252 73.10951326621847) bank162032 +162033 POINT(40.63410758578427 74.18495760601952) bank162033 +162034 POINT(41.536426560773975 73.21156114121068) bank162034 +162035 POINT(40.44193405148121 75.00580061520564) bank162035 +162036 POINT(40.12833388094147 74.68562364014575) bank162036 +162037 POINT(39.77015796078786 74.17097288562522) bank162037 +162038 POINT(41.572113813015775 73.96531568391974) bank162038 +162039 POINT(40.35798437520972 74.3866451903411) bank162039 +162040 POINT(40.964608481423944 73.82764433771483) bank162040 +162041 POINT(41.24329572700837 73.12169395043333) bank162041 +162042 POINT(41.27799622829752 73.80793937122267) bank162042 +162043 POINT(40.15589287944281 73.29255018755045) bank162043 +162044 POINT(40.22468654187029 73.93288756386121) bank162044 +162045 POINT(41.4130761842656 74.59969061372061) bank162045 +162046 POINT(40.02141430343465 74.55926223445034) bank162046 +162047 POINT(41.472534439382265 74.46747929599198) bank162047 +162048 POINT(40.38136565054766 73.14857576923797) bank162048 +162049 POINT(39.82639139600005 74.40834029635886) bank162049 +162050 POINT(41.008722107447014 73.09404106038231) bank162050 +162051 POINT(39.86999297068508 74.18854606739464) bank162051 +162052 POINT(40.26758941250335 74.29338796551885) bank162052 +162053 POINT(40.294779026404065 74.08822982894337) bank162053 +162054 POINT(40.69804613377712 74.14328480467987) bank162054 +162055 POINT(40.3624966441657 73.04650898523417) bank162055 +162056 POINT(39.97454422246072 74.72057041161904) bank162056 +162057 POINT(41.15684709269533 73.42440284329392) bank162057 +162058 POINT(40.5291648236096 74.36940830084491) bank162058 +162059 POINT(41.578386531744655 74.19472505473674) bank162059 +162060 POINT(41.119544240787285 74.14610373951584) bank162060 +162061 POINT(40.212971194312516 73.10673805789217) bank162061 +162062 POINT(40.17734464016056 74.79412222999942) bank162062 +162063 POINT(39.73602823679387 73.10857836546222) bank162063 +162064 POINT(40.87623612934409 74.92836870638368) bank162064 +162065 POINT(40.268686790343956 73.13627395656839) bank162065 +162066 POINT(40.32049242803858 73.05840352970645) bank162066 +162067 POINT(41.561519779796065 74.806687002708) bank162067 +162068 POINT(40.266853416148976 74.78384163283259) bank162068 +162069 POINT(40.44318886945557 73.64290990193489) bank162069 +162070 POINT(40.98968250564467 73.07556117038138) bank162070 +162071 POINT(41.51121516985904 73.16402337034212) bank162071 +162072 POINT(39.9559571356846 73.55363902950073) bank162072 +162073 POINT(40.23746787390948 73.85855172468078) bank162073 +162074 POINT(40.66929569840379 74.75407684787169) bank162074 +162075 POINT(39.77006909657725 74.09867134828691) bank162075 +162076 POINT(40.4427244649034 73.77238769801076) bank162076 +162077 POINT(40.482663422003704 73.19252992547668) bank162077 +162078 POINT(41.338404415245094 73.6813838722867) bank162078 +162079 POINT(40.89089221712027 73.39872635515668) bank162079 +162080 POINT(41.163069019186196 74.00109555471313) bank162080 +162081 POINT(41.24181258873042 73.33156202954996) bank162081 +162082 POINT(40.688627911899616 73.78247112096929) bank162082 +162083 POINT(40.61736320776587 73.68313231651041) bank162083 +162084 POINT(41.12829376170993 74.53837392066525) bank162084 +162085 POINT(39.866955200665686 74.12772314942357) bank162085 +162086 POINT(40.73370327489562 74.93510614674543) bank162086 +162087 POINT(40.425138621064754 73.62440315913271) bank162087 +162088 POINT(40.819623144215264 73.58259818873626) bank162088 +162089 POINT(40.95654107803387 74.58222310380265) bank162089 +162090 POINT(40.28460027751506 74.30006881213578) bank162090 +162091 POINT(40.51893620691485 74.3991726279775) bank162091 +162092 POINT(41.16108066338903 73.37137829381298) bank162092 +162093 POINT(39.72658119695705 73.38865015089111) bank162093 +162094 POINT(40.47004010611368 74.53114803491844) bank162094 +162095 POINT(41.712326561171025 74.33339224605855) bank162095 +162096 POINT(40.169664248156394 73.13476769694465) bank162096 +162097 POINT(40.16464274421955 73.31722799057295) bank162097 +162098 POINT(41.318904889502086 73.77964516475119) bank162098 +162099 POINT(40.806650091742114 74.02020618608103) bank162099 +162100 POINT(40.59358596871819 73.1050110314503) bank162100 +162101 POINT(39.80643617759408 74.15879748300557) bank162101 +162102 POINT(40.78649314556179 73.55859528633148) bank162102 +162103 POINT(41.38508675583914 74.31834204484075) bank162103 +162104 POINT(40.54373625744576 74.29300475808054) bank162104 +162105 POINT(40.07575130346425 74.80255965666247) bank162105 +162106 POINT(40.29686608346169 73.99104013734554) bank162106 +162107 POINT(41.46791659704732 73.93690076943133) bank162107 +162108 POINT(39.793845353577325 74.3357602229219) bank162108 +162109 POINT(39.76532292937357 74.1788690116887) bank162109 +162110 POINT(39.90458675918069 74.27606614984391) bank162110 +162111 POINT(40.31130592574597 73.71469021098784) bank162111 +162112 POINT(39.758892995739984 73.17586194045084) bank162112 +162113 POINT(39.92615317786063 73.4423624827094) bank162113 +162114 POINT(40.46850127095078 74.20302022800209) bank162114 +162115 POINT(39.828090973878595 74.04940629709137) bank162115 +162116 POINT(39.73487786158291 74.53913875274843) bank162116 +162117 POINT(41.57660716986171 74.1183973961348) bank162117 +162118 POINT(40.18457296566264 74.2869812980691) bank162118 +162119 POINT(40.83740082440688 74.56593078168166) bank162119 +162120 POINT(39.99954625343186 74.18261614778359) bank162120 +162121 POINT(39.87236153951407 74.2416181823038) bank162121 +162122 POINT(40.66367892100642 73.93624566874769) bank162122 +162123 POINT(40.169559495624256 73.25153201909067) bank162123 +162124 POINT(40.68129635880441 74.67487346702183) bank162124 +162125 POINT(39.7867194607479 74.53847518136202) bank162125 +162126 POINT(40.47767969614063 74.53697773109525) bank162126 +162127 POINT(40.448284861478356 74.90182287371509) bank162127 +162128 POINT(40.32958684887085 74.4281565026404) bank162128 +162129 POINT(39.873824259971535 74.97583654252162) bank162129 +162130 POINT(40.027974774204 73.5613056484344) bank162130 +162131 POINT(41.08487279349217 74.68016810634228) bank162131 +162132 POINT(40.49349302270899 73.01689107530771) bank162132 +162133 POINT(41.489379905089855 73.42696722714398) bank162133 +162134 POINT(40.29945414476336 73.38129933415772) bank162134 +162135 POINT(41.259097886255475 73.99098601537355) bank162135 +162136 POINT(40.5807763820306 73.1022374119913) bank162136 +162137 POINT(41.292995253045206 73.87815063421806) bank162137 +162138 POINT(40.20364173763344 73.68541263012895) bank162138 +162139 POINT(41.513579621827034 74.29665364384776) bank162139 +162140 POINT(40.63726260164714 73.94487080842268) bank162140 +162141 POINT(40.221934047647345 74.2750782216155) bank162141 +162142 POINT(40.710390529294216 73.80108148035484) bank162142 +162143 POINT(40.2114650156051 74.8947461232781) bank162143 +162144 POINT(40.59116485763572 74.99781893960491) bank162144 +162145 POINT(40.5360154192463 74.44131673618301) bank162145 +162146 POINT(41.05255949092537 74.95513009052907) bank162146 +162147 POINT(40.51661890967405 73.1357472026748) bank162147 +162148 POINT(41.52806400502548 74.353647388694) bank162148 +162149 POINT(40.68202633703676 73.04967118592911) bank162149 +162150 POINT(41.50824941781324 73.95545670931575) bank162150 +162151 POINT(41.017428896500626 74.16175645188969) bank162151 +162152 POINT(40.15619635947363 74.57930799236324) bank162152 +162153 POINT(41.253405878900296 73.57616956264313) bank162153 +162154 POINT(40.68948172349634 74.97924884783536) bank162154 +162155 POINT(40.362761985262 73.44706509276745) bank162155 +162156 POINT(40.12054183562441 74.04787331639886) bank162156 +162157 POINT(40.15339072648436 74.86185699705737) bank162157 +162158 POINT(41.288957036963076 73.93388370492968) bank162158 +162159 POINT(40.70044741855777 73.21787259308024) bank162159 +162160 POINT(40.964144136035095 73.232566112096) bank162160 +162161 POINT(41.38264942940038 74.17762809909091) bank162161 +162162 POINT(41.12419458849608 74.18681905821677) bank162162 +162163 POINT(41.289364977937744 73.92843497802598) bank162163 +162164 POINT(40.53696345325858 73.28940193956836) bank162164 +162165 POINT(39.934297993849725 74.09016120524225) bank162165 +162166 POINT(41.34931705033394 74.3451725657198) bank162166 +162167 POINT(41.4855058323623 73.80562018071201) bank162167 +162168 POINT(41.68936891094129 74.34430679080096) bank162168 +162169 POINT(40.94896063045225 74.6719874729899) bank162169 +162170 POINT(40.26724814805558 74.59508908071574) bank162170 +162171 POINT(40.299869818008325 74.00882602630591) bank162171 +162172 POINT(39.806635285113366 74.39554536813682) bank162172 +162173 POINT(41.653990309627986 73.5091103698219) bank162173 +162174 POINT(41.309706622384795 74.21839452445231) bank162174 +162175 POINT(40.38988682868618 74.85016233095672) bank162175 +162176 POINT(41.41485762643572 74.93711613574092) bank162176 +162177 POINT(40.77245812601088 74.68353503716513) bank162177 +162178 POINT(41.68919993714943 74.37508453195845) bank162178 +162179 POINT(41.226505300792994 74.7919483527073) bank162179 +162180 POINT(40.60514547430972 74.49627346389296) bank162180 +162181 POINT(39.7244709662003 74.16993171982936) bank162181 +162182 POINT(39.946662599213774 74.54707170355157) bank162182 +162183 POINT(40.213220825695615 74.77807072473261) bank162183 +162184 POINT(39.8934522704413 74.84462579265339) bank162184 +162185 POINT(41.50466128512633 74.32275255311673) bank162185 +162186 POINT(41.60845806251737 74.76235084990465) bank162186 +162187 POINT(40.511912721101346 74.258004613418) bank162187 +162188 POINT(39.71831494050876 74.45187476907199) bank162188 +162189 POINT(39.901227079348715 73.45984822328434) bank162189 +162190 POINT(40.98635610714084 73.66090410862998) bank162190 +162191 POINT(40.27254013153319 74.64784254191278) bank162191 +162192 POINT(41.51910518747618 73.56170805949583) bank162192 +162193 POINT(40.389061202447174 73.80836213278648) bank162193 +162194 POINT(40.01547489887946 73.24437260196375) bank162194 +162195 POINT(40.21703032128974 74.37053427332468) bank162195 +162196 POINT(40.23781526706966 74.5423267425545) bank162196 +162197 POINT(41.09878429015212 74.35447961946792) bank162197 +162198 POINT(40.17572326764818 74.96103348507268) bank162198 +162199 POINT(40.900860805011405 74.02508391731273) bank162199 +162200 POINT(40.07959382104151 74.33919413821074) bank162200 +162201 POINT(41.3610348309538 73.18508101916467) bank162201 +162202 POINT(40.58377088858231 73.64669425267202) bank162202 +162203 POINT(40.72994360694035 74.41662165779809) bank162203 +162204 POINT(41.08847198004567 73.36212393768716) bank162204 +162205 POINT(40.08706686198657 74.6621887394586) bank162205 +162206 POINT(40.59152698283193 73.86430548885278) bank162206 +162207 POINT(40.35389690946543 73.970328293597) bank162207 +162208 POINT(39.73979417349658 74.45913577647454) bank162208 +162209 POINT(40.152461501892525 73.12287957894769) bank162209 +162210 POINT(40.35132591512842 73.49131444237793) bank162210 +162211 POINT(39.72284416813993 74.2233425565252) bank162211 +162212 POINT(40.87051426312197 73.94603131105355) bank162212 +162213 POINT(40.980776849273035 74.58771073553872) bank162213 +162214 POINT(40.8506916097484 73.313567500759) bank162214 +162215 POINT(40.03471532342081 74.43734169649332) bank162215 +162216 POINT(39.81363401861439 74.39561863932873) bank162216 +162217 POINT(39.835163437826566 73.82291704580028) bank162217 +162218 POINT(40.81759541722792 73.88623259373065) bank162218 +162219 POINT(41.54094019367321 74.2132196181281) bank162219 +162220 POINT(40.19928438903014 73.79607985202861) bank162220 +162221 POINT(41.05480103012802 73.642909181596) bank162221 +162222 POINT(39.88507605509379 73.70977770201387) bank162222 +162223 POINT(40.48079119289021 74.5553059896607) bank162223 +162224 POINT(41.333114325748326 73.76401031754658) bank162224 +162225 POINT(40.555956092440994 74.58109722262076) bank162225 +162226 POINT(40.38393402804738 73.23927851037399) bank162226 +162227 POINT(41.358214984313 73.37394740961824) bank162227 +162228 POINT(40.32734007351997 73.33481975922888) bank162228 +162229 POINT(41.548673153865145 74.41913513811986) bank162229 +162230 POINT(39.758730255325744 74.55420279991311) bank162230 +162231 POINT(39.84155889779895 73.17521099952569) bank162231 +162232 POINT(40.064281379285 73.99596193454167) bank162232 +162233 POINT(41.455548842628 73.44643417218384) bank162233 +162234 POINT(40.75473071910089 74.66253312814479) bank162234 +162235 POINT(40.69261129639654 73.9707802440486) bank162235 +162236 POINT(40.09630472315255 73.33025652690024) bank162236 +162237 POINT(40.51199500481637 74.76793704332506) bank162237 +162238 POINT(40.96522750030096 73.51226447635806) bank162238 +162239 POINT(40.85949405273521 74.5918818824022) bank162239 +162240 POINT(41.64018009970255 73.0226961716904) bank162240 +162241 POINT(41.06628777663087 73.91816302590928) bank162241 +162242 POINT(41.14957926733946 74.1194424129535) bank162242 +162243 POINT(40.76780801827179 73.9855007676759) bank162243 +162244 POINT(39.959857296007954 74.72249162003806) bank162244 +162245 POINT(41.46812794295414 74.85484617745392) bank162245 +162246 POINT(41.02650682552654 73.34398472781227) bank162246 +162247 POINT(41.652166698488315 73.65458451486593) bank162247 +162248 POINT(40.86864647250974 74.29511704745694) bank162248 +162249 POINT(40.71925114118383 74.36683627168476) bank162249 +162250 POINT(40.47654705060801 74.65358006912612) bank162250 +162251 POINT(40.79565185518988 73.87876276961882) bank162251 +162252 POINT(40.09946692974086 74.26668754979288) bank162252 +162253 POINT(40.93579933589928 74.75594252757465) bank162253 +162254 POINT(41.4906971095426 74.78789896332258) bank162254 +162255 POINT(40.40051051479673 73.24107796022278) bank162255 +162256 POINT(41.23039277150682 73.45199062103465) bank162256 +162257 POINT(41.02824388251079 73.28541568108713) bank162257 +162258 POINT(39.83633742454562 74.11463226801139) bank162258 +162259 POINT(40.88500470193123 73.14359616184784) bank162259 +162260 POINT(40.05005686386657 73.21250462113642) bank162260 +162261 POINT(40.73661092752175 73.02258890144952) bank162261 +162262 POINT(41.27125898388523 74.29062762922923) bank162262 +162263 POINT(41.0411024215282 73.13091172657386) bank162263 +162264 POINT(41.2546007945051 73.03604636276631) bank162264 +162265 POINT(41.55626174446445 74.85572296189322) bank162265 +162266 POINT(39.77034578322452 74.84800988422641) bank162266 +162267 POINT(41.49643720482843 73.85656090877005) bank162267 +162268 POINT(40.43705437867624 74.26326945154874) bank162268 +162269 POINT(40.50494976704853 73.15368137493704) bank162269 +162270 POINT(40.783539977926964 73.69499485096286) bank162270 +162271 POINT(41.27512996081237 74.0429156912975) bank162271 +162272 POINT(40.290596114467554 74.82060379399634) bank162272 +162273 POINT(39.87224733586603 73.98965225769673) bank162273 +162274 POINT(40.17399902001407 73.53957047378444) bank162274 +162275 POINT(40.312830873587835 73.73693399939211) bank162275 +162276 POINT(40.66850377703022 73.68306060582727) bank162276 +162277 POINT(41.11359374544477 73.07305625854274) bank162277 +162278 POINT(39.982930262518764 74.72443399874405) bank162278 +162279 POINT(39.876367535249166 73.63234722646555) bank162279 +162280 POINT(41.68193745160606 74.28027231599242) bank162280 +162281 POINT(40.909962182317564 73.84925122125296) bank162281 +162282 POINT(40.924041210554016 74.72600125229962) bank162282 +162283 POINT(40.90888627924983 73.69734493372795) bank162283 +162284 POINT(41.23128003525457 74.51157477651313) bank162284 +162285 POINT(40.70725631838367 74.71109968793277) bank162285 +162286 POINT(40.63113635312594 73.39291832086766) bank162286 +162287 POINT(39.77419691979247 74.41793051734844) bank162287 +162288 POINT(40.78747927126593 74.86137222118138) bank162288 +162289 POINT(39.94542031276614 74.82211284311377) bank162289 +162290 POINT(41.18210911885263 73.24370053385626) bank162290 +162291 POINT(40.69257258125693 73.09764191884875) bank162291 +162292 POINT(40.260950871563836 74.97157228055173) bank162292 +162293 POINT(41.03783407191924 73.65562917342895) bank162293 +162294 POINT(40.2760495888062 74.9655556568897) bank162294 +162295 POINT(41.12292331369226 73.72037876058415) bank162295 +162296 POINT(41.33322481772914 74.99033703863941) bank162296 +162297 POINT(41.60725034620189 73.39313472097393) bank162297 +162298 POINT(40.76321002621078 73.85870741193115) bank162298 +162299 POINT(41.070713830917754 73.49073917983569) bank162299 +162300 POINT(41.570936020299264 74.30034847663735) bank162300 +162301 POINT(40.66486449893978 74.77300847436761) bank162301 +162302 POINT(41.036250625539516 74.51060119230559) bank162302 +162303 POINT(40.127744318150036 73.30203349142289) bank162303 +162304 POINT(41.12948521225801 73.66891689558146) bank162304 +162305 POINT(40.678481742573226 73.11441759713209) bank162305 +162306 POINT(40.57288069220834 73.2172076422104) bank162306 +162307 POINT(39.99019192114619 74.11076749672364) bank162307 +162308 POINT(40.207417498031944 74.16607409117772) bank162308 +162309 POINT(40.93537575566063 73.63671606188836) bank162309 +162310 POINT(40.91141024568029 73.46577489285539) bank162310 +162311 POINT(39.79378240530423 73.24229161422974) bank162311 +162312 POINT(41.55569601025159 73.20967192507747) bank162312 +162313 POINT(39.80820560121784 74.87939357253035) bank162313 +162314 POINT(40.577035062972556 73.10406143075382) bank162314 +162315 POINT(40.44946077010766 73.30650824794164) bank162315 +162316 POINT(39.75174178956587 74.97757364893779) bank162316 +162317 POINT(40.172256285885396 73.7597637916401) bank162317 +162318 POINT(40.14834979172165 74.72022087574992) bank162318 +162319 POINT(41.477172668174774 73.18842002339362) bank162319 +162320 POINT(41.43077914531373 74.48754175613976) bank162320 +162321 POINT(39.722310990953204 73.88847904849453) bank162321 +162322 POINT(39.748473859717286 74.83619770601972) bank162322 +162323 POINT(40.89495337628259 74.70324252521338) bank162323 +162324 POINT(40.028578726156056 74.38545190890231) bank162324 +162325 POINT(40.77170007550316 73.45858995273565) bank162325 +162326 POINT(41.08265835868063 74.28719125430015) bank162326 +162327 POINT(41.32004967061595 73.6962631879684) bank162327 +162328 POINT(41.57660367542157 74.8393538493474) bank162328 +162329 POINT(41.136804451307064 73.60247349860015) bank162329 +162330 POINT(39.85169220209525 74.54166859714508) bank162330 +162331 POINT(41.5529867933773 74.14569395017047) bank162331 +162332 POINT(39.85150339416412 74.14804594908955) bank162332 +162333 POINT(40.87313103175332 73.39069628321946) bank162333 +162334 POINT(40.70078073770957 73.54593618341222) bank162334 +162335 POINT(40.73059169296583 73.239294316688) bank162335 +162336 POINT(39.994656690305064 74.92117739189342) bank162336 +162337 POINT(40.2980639006889 74.86246458957963) bank162337 +162338 POINT(40.639849296059104 74.6991987283406) bank162338 +162339 POINT(41.09261810450069 73.91042310362721) bank162339 +162340 POINT(40.706150405988105 74.08832438850871) bank162340 +162341 POINT(39.92144431427215 74.01050663220703) bank162341 +162342 POINT(40.949994185545215 74.93756871349939) bank162342 +162343 POINT(39.73291718693896 74.55543504756723) bank162343 +162344 POINT(40.77090282893184 73.96975943391199) bank162344 +162345 POINT(41.504979036058096 74.66290774992265) bank162345 +162346 POINT(40.337532612484864 73.59938548124087) bank162346 +162347 POINT(41.333416795913 74.57686732580726) bank162347 +162348 POINT(40.81597842027067 74.33904512032161) bank162348 +162349 POINT(40.834661055572056 73.20023068814884) bank162349 +162350 POINT(40.16459329475919 74.54351203036762) bank162350 +162351 POINT(41.41906149404707 74.23592050532399) bank162351 +162352 POINT(41.09627190400275 74.21267589091181) bank162352 +162353 POINT(41.48544251930321 73.18371519516091) bank162353 +162354 POINT(41.032536471221704 74.92488548543915) bank162354 +162355 POINT(40.23872321859821 73.16551914615147) bank162355 +162356 POINT(40.166703159954814 73.34566397703479) bank162356 +162357 POINT(39.977289027971764 74.36547704068752) bank162357 +162358 POINT(41.56099564734 73.33812444598324) bank162358 +162359 POINT(41.382929207594515 74.25650742942544) bank162359 +162360 POINT(41.634281248294116 74.12199708967202) bank162360 +162361 POINT(40.15716307475452 73.64004402740038) bank162361 +162362 POINT(39.869135902485965 74.31650360387616) bank162362 +162363 POINT(40.99372377020258 74.74779888670044) bank162363 +162364 POINT(40.92099625506351 74.69737448639786) bank162364 +162365 POINT(40.44306521560387 73.48919061446962) bank162365 +162366 POINT(40.94269711520619 74.50222891693122) bank162366 +162367 POINT(41.31305267901611 74.60909811206449) bank162367 +162368 POINT(40.177001687406914 74.33784486851965) bank162368 +162369 POINT(41.52212275970215 73.45076024860667) bank162369 +162370 POINT(39.72891887515456 74.03378835075007) bank162370 +162371 POINT(40.642690202804964 74.58045998525573) bank162371 +162372 POINT(39.876541829652176 73.57809194223589) bank162372 +162373 POINT(40.84145679858483 73.39116219910109) bank162373 +162374 POINT(40.600881837410746 74.73117780102145) bank162374 +162375 POINT(41.39118249683618 73.83245121233695) bank162375 +162376 POINT(40.456995880281575 73.03324334499477) bank162376 +162377 POINT(39.91956143828189 74.08147158370026) bank162377 +162378 POINT(40.961134501128974 74.5350814538401) bank162378 +162379 POINT(40.76392712415742 74.76562991296095) bank162379 +162380 POINT(40.25203256578303 74.13619500549886) bank162380 +162381 POINT(41.51307269044985 73.62645793374152) bank162381 +162382 POINT(40.58455341043522 74.7568954582303) bank162382 +162383 POINT(39.71297464117968 74.0672789740251) bank162383 +162384 POINT(41.22844313380029 73.38150621559465) bank162384 +162385 POINT(41.0444319564801 74.31425808019087) bank162385 +162386 POINT(41.70512506205122 73.43617324783837) bank162386 +162387 POINT(41.46535739648146 74.34420975689751) bank162387 +162388 POINT(41.49500608912777 74.55725802296054) bank162388 +162389 POINT(40.99260363212936 74.77876772575561) bank162389 +162390 POINT(41.11228692202874 74.46360547787165) bank162390 +162391 POINT(40.18073017509589 73.53176912213515) bank162391 +162392 POINT(40.00087866692501 74.97263962263487) bank162392 +162393 POINT(41.584306935478544 74.56086301968594) bank162393 +162394 POINT(40.188293025879396 73.29164745973544) bank162394 +162395 POINT(41.62180425494331 73.31903787715466) bank162395 +162396 POINT(39.856297080790405 74.71972286589308) bank162396 +162397 POINT(41.48864565899864 74.99962720172019) bank162397 +162398 POINT(40.665540860494744 74.75891479010654) bank162398 +162399 POINT(39.77325989140635 73.69399171461289) bank162399 +162400 POINT(40.632200706320766 74.63468605928391) bank162400 +162401 POINT(41.05858979384193 74.90924132953369) bank162401 +162402 POINT(40.15617209847112 74.39298396928834) bank162402 +162403 POINT(41.56661228090427 74.93757505698788) bank162403 +162404 POINT(41.326002703683514 73.50035635978432) bank162404 +162405 POINT(40.887237129780175 73.02425901013471) bank162405 +162406 POINT(39.88208609761334 73.48950814675442) bank162406 +162407 POINT(40.9612016284373 73.75888015748521) bank162407 +162408 POINT(40.78503916259737 74.61144053648833) bank162408 +162409 POINT(40.139263401717336 73.92274989403478) bank162409 +162410 POINT(39.76702563674049 74.05346378784766) bank162410 +162411 POINT(40.170541861247045 73.78339752342666) bank162411 +162412 POINT(41.2710339019975 74.39989125919702) bank162412 +162413 POINT(40.03343615356539 73.01303183304202) bank162413 +162414 POINT(39.883884253377474 73.3786337179232) bank162414 +162415 POINT(40.10319221865609 73.70193637259598) bank162415 +162416 POINT(40.96165519842108 74.18781850807237) bank162416 +162417 POINT(41.55365468760088 74.86065308376506) bank162417 +162418 POINT(40.02465261937371 74.07085410778485) bank162418 +162419 POINT(40.954000208928804 74.81409923176706) bank162419 +162420 POINT(41.50030184494685 73.110156571042) bank162420 +162421 POINT(40.92084223433049 74.33543234785475) bank162421 +162422 POINT(41.51948580085914 73.7659195644051) bank162422 +162423 POINT(39.98677416073645 73.63195326962978) bank162423 +162424 POINT(40.38792989189036 74.61218592411075) bank162424 +162425 POINT(40.52208547973938 73.6041984494416) bank162425 +162426 POINT(41.200865712589824 74.41078057540572) bank162426 +162427 POINT(41.1237042890892 73.67303900170782) bank162427 +162428 POINT(40.974132069038376 73.56934550906554) bank162428 +162429 POINT(41.27663970021004 74.64679718421593) bank162429 +162430 POINT(41.67319599807456 74.53880416578157) bank162430 +162431 POINT(40.09360566658225 73.53806162001639) bank162431 +162432 POINT(41.35687709512003 73.47603849209433) bank162432 +162433 POINT(39.80109944562509 74.00807163300615) bank162433 +162434 POINT(40.91840325989514 74.85253020974412) bank162434 +162435 POINT(40.12247276744131 73.56088909930668) bank162435 +162436 POINT(40.41044373212226 74.66702216366177) bank162436 +162437 POINT(40.17761584179663 74.75904113816465) bank162437 +162438 POINT(40.28155648374258 73.57994151646852) bank162438 +162439 POINT(40.383102164388674 74.21462515400522) bank162439 +162440 POINT(41.01987800841478 73.74990317096781) bank162440 +162441 POINT(41.29400314050774 74.52624825382087) bank162441 +162442 POINT(40.87149221016016 74.30573167457979) bank162442 +162443 POINT(40.13282061041017 74.58943318885275) bank162443 +162444 POINT(41.078316557740045 73.28339698482957) bank162444 +162445 POINT(39.994874289267365 73.67225252330259) bank162445 +162446 POINT(40.90350247954602 73.24225256481148) bank162446 +162447 POINT(40.860680587293395 73.83235572404946) bank162447 +162448 POINT(41.32623934262212 73.32817972100827) bank162448 +162449 POINT(40.18846666458975 73.64014851253785) bank162449 +162450 POINT(40.86535556681934 73.37953639957466) bank162450 +162451 POINT(40.662828609944604 73.85177137010567) bank162451 +162452 POINT(41.16537078831359 74.72718906877725) bank162452 +162453 POINT(40.916140182819134 73.40976460759288) bank162453 +162454 POINT(40.311155995522 73.77838724765702) bank162454 +162455 POINT(39.80907569162035 73.63580084206649) bank162455 +162456 POINT(41.49620932160711 73.44733573192536) bank162456 +162457 POINT(40.09391346951586 73.21154562418884) bank162457 +162458 POINT(39.87838055590066 74.16801486890621) bank162458 +162459 POINT(40.02849262374679 73.39456091355865) bank162459 +162460 POINT(39.863126407875136 74.69124821518702) bank162460 +162461 POINT(41.61909359166674 74.1376877129774) bank162461 +162462 POINT(41.58468942751347 73.49739537094513) bank162462 +162463 POINT(40.448676692694335 73.67361845245016) bank162463 +162464 POINT(39.96782058349167 74.26844949240756) bank162464 +162465 POINT(40.15160309107845 74.62702162507182) bank162465 +162466 POINT(40.386345731139386 74.40643748544863) bank162466 +162467 POINT(40.90608687172169 74.45890920959415) bank162467 +162468 POINT(40.91306513735738 73.6887984908281) bank162468 +162469 POINT(40.859948631668 74.15198761310621) bank162469 +162470 POINT(40.54134236779427 73.7520577818433) bank162470 +162471 POINT(40.89594122198038 73.63029363410128) bank162471 +162472 POINT(40.016404987629414 74.56917587136464) bank162472 +162473 POINT(40.866711363569145 74.3659077379155) bank162473 +162474 POINT(40.89253949157056 73.54266258660782) bank162474 +162475 POINT(40.41440752039549 74.00889906257301) bank162475 +162476 POINT(41.704816055545884 74.30941588549176) bank162476 +162477 POINT(41.35683355094274 74.26158276683233) bank162477 +162478 POINT(41.054039022703684 73.91303802999472) bank162478 +162479 POINT(41.01817978957014 73.67143223717814) bank162479 +162480 POINT(41.55132564221577 74.65078161085559) bank162480 +162481 POINT(39.85652137029293 74.44238122402282) bank162481 +162482 POINT(40.28093070556529 73.17199630150476) bank162482 +162483 POINT(40.729963081386764 73.64234703626673) bank162483 +162484 POINT(40.30282246012048 74.51049114458759) bank162484 +162485 POINT(41.65323920047683 73.94977569582285) bank162485 +162486 POINT(40.785662196017654 73.2684220381923) bank162486 +162487 POINT(39.93757138018894 74.06689286271036) bank162487 +162488 POINT(40.306874829406645 74.94973516037065) bank162488 +162489 POINT(39.88150875011187 74.7245230854126) bank162489 +162490 POINT(41.5100665598068 74.32500054936799) bank162490 +162491 POINT(40.330515402923716 74.86305467397047) bank162491 +162492 POINT(40.09901000321035 74.4460055189717) bank162492 +162493 POINT(41.318580247925844 73.6889377924833) bank162493 +162494 POINT(41.56237483382076 73.70608419684942) bank162494 +162495 POINT(41.39461651214336 74.6379758064752) bank162495 +162496 POINT(40.040571735982745 74.62638138488984) bank162496 +162497 POINT(40.11054639056242 73.686567215175) bank162497 +162498 POINT(40.18036826872296 73.35764791229039) bank162498 +162499 POINT(41.602597345964654 73.8857785460513) bank162499 +162500 POINT(41.0282700376274 74.86174591188542) bank162500 +162501 POINT(41.06229654479563 73.20870103556227) bank162501 +162502 POINT(40.13969636735785 73.53228628431896) bank162502 +162503 POINT(41.216721244795565 73.11835330590019) bank162503 +162504 POINT(41.484338275735915 74.1007620028633) bank162504 +162505 POINT(41.096151994182726 73.57112721927118) bank162505 +162506 POINT(39.75191624550499 74.32376216236851) bank162506 +162507 POINT(41.12066138978831 74.7776497003103) bank162507 +162508 POINT(41.02408585190199 74.93090054709462) bank162508 +162509 POINT(40.16313370073165 74.29155949512773) bank162509 +162510 POINT(40.024264118476076 74.8129755481263) bank162510 +162511 POINT(41.45023246628463 74.39824380895193) bank162511 +162512 POINT(41.22974858736033 73.31550555784433) bank162512 +162513 POINT(40.1618534286422 73.52438077596243) bank162513 +162514 POINT(40.970261642790405 74.99237817028175) bank162514 +162515 POINT(40.16875844346856 74.74413652274445) bank162515 +162516 POINT(39.90928041163067 73.67601298505515) bank162516 +162517 POINT(40.171229142486524 74.78393597623955) bank162517 +162518 POINT(40.174390943076624 74.023560027891) bank162518 +162519 POINT(40.69606243592445 74.50085431423572) bank162519 +162520 POINT(41.45157421105921 74.8813787884331) bank162520 +162521 POINT(40.5164752246037 74.89778274251694) bank162521 +162522 POINT(40.421992652292516 74.15298441345566) bank162522 +162523 POINT(40.35414906138572 74.58134270468601) bank162523 +162524 POINT(40.182489703544476 74.76921108538278) bank162524 +162525 POINT(41.48653319601759 73.76450988672937) bank162525 +162526 POINT(40.49362881200187 73.74282912933128) bank162526 +162527 POINT(41.0393707669151 74.09866609026646) bank162527 +162528 POINT(40.7674494375704 74.70452597583959) bank162528 +162529 POINT(41.35690930488942 73.11309109006103) bank162529 +162530 POINT(39.75540725858336 74.43962275499013) bank162530 +162531 POINT(39.780463430619484 73.67758728877698) bank162531 +162532 POINT(40.68036291329001 73.97704286517698) bank162532 +162533 POINT(40.145766927691106 74.77585202386884) bank162533 +162534 POINT(41.6772142289232 73.52457597643381) bank162534 +162535 POINT(40.64406759237482 73.45668445415114) bank162535 +162536 POINT(40.564554352394 74.26231187300189) bank162536 +162537 POINT(41.072218110547915 73.5493143316187) bank162537 +162538 POINT(41.436382291172634 74.82642614170504) bank162538 +162539 POINT(40.95325624962367 73.77583218686198) bank162539 +162540 POINT(41.19463121885869 74.65851088479782) bank162540 +162541 POINT(41.32048912807024 74.0107416617663) bank162541 +162542 POINT(41.32389821614697 74.96030391003573) bank162542 +162543 POINT(41.29771671152603 73.80722880428138) bank162543 +162544 POINT(40.980435454122784 73.59144084011284) bank162544 +162545 POINT(40.9755635816682 74.23395043322041) bank162545 +162546 POINT(40.391241258837915 73.30128603843724) bank162546 +162547 POINT(40.90950915246153 73.06850681920356) bank162547 +162548 POINT(39.72321347962007 74.7412709019128) bank162548 +162549 POINT(39.96634234177289 73.95203766750953) bank162549 +162550 POINT(41.019936747480145 74.32193336214758) bank162550 +162551 POINT(40.30517722290808 74.47464143432549) bank162551 +162552 POINT(40.26504533419198 73.57744601099351) bank162552 +162553 POINT(40.25855145003622 74.84391623657164) bank162553 +162554 POINT(41.535014068586705 73.09709075758249) bank162554 +162555 POINT(40.73959303470107 74.46055923338983) bank162555 +162556 POINT(40.5984986344447 74.0747679569235) bank162556 +162557 POINT(39.915510722383054 74.15669000559436) bank162557 +162558 POINT(40.15343239788435 73.93970108030648) bank162558 +162559 POINT(41.35764740191083 73.57093875064419) bank162559 +162560 POINT(39.79711911764091 73.9005814229686) bank162560 +162561 POINT(40.045743622959314 73.70121384975559) bank162561 +162562 POINT(40.358484586755424 74.92881082750152) bank162562 +162563 POINT(39.73203134185025 74.79985816640169) bank162563 +162564 POINT(41.08138275721264 73.26910153417022) bank162564 +162565 POINT(40.71823400789916 74.31774732390653) bank162565 +162566 POINT(40.054415467226505 74.8008357072105) bank162566 +162567 POINT(39.73916752612614 74.14297948306235) bank162567 +162568 POINT(39.80164443404893 74.03991978476486) bank162568 +162569 POINT(40.62622863060399 74.12604325748222) bank162569 +162570 POINT(40.91744997652108 73.55105027282316) bank162570 +162571 POINT(39.98517344439163 73.86893601812949) bank162571 +162572 POINT(40.69255601102208 74.08067379822768) bank162572 +162573 POINT(41.11212263698133 74.87739571816572) bank162573 +162574 POINT(41.544361315341405 74.98064331378613) bank162574 +162575 POINT(40.10518807267573 74.16270580539593) bank162575 +162576 POINT(40.11869867640044 74.53814871682593) bank162576 +162577 POINT(41.47475906907124 74.18881907669571) bank162577 +162578 POINT(40.398648776781506 74.60145281392577) bank162578 +162579 POINT(39.97117735285333 74.94448618390221) bank162579 +162580 POINT(41.3728038417117 73.26162742112062) bank162580 +162581 POINT(40.233783077571246 73.81617771276423) bank162581 +162582 POINT(41.09892361100049 73.30970911890451) bank162582 +162583 POINT(40.800455359523255 74.35501464067967) bank162583 +162584 POINT(41.69922268330734 74.38742761294552) bank162584 +162585 POINT(41.534607191803914 73.65656445493663) bank162585 +162586 POINT(40.66358553563837 73.71642049346865) bank162586 +162587 POINT(40.82504282134023 74.47096111373989) bank162587 +162588 POINT(40.81270924166391 74.89889052031135) bank162588 +162589 POINT(41.24172145843451 73.89553243740038) bank162589 +162590 POINT(40.73718767386542 73.6876626553535) bank162590 +162591 POINT(40.95739782258491 73.88114268782964) bank162591 +162592 POINT(40.668697050580235 73.82187191653432) bank162592 +162593 POINT(39.861759862022865 74.89869002193494) bank162593 +162594 POINT(40.55358262839688 74.71165356214932) bank162594 +162595 POINT(40.24389849816328 74.32802156831717) bank162595 +162596 POINT(41.12086926460309 73.82607108355502) bank162596 +162597 POINT(41.18098683439121 74.39190900351717) bank162597 +162598 POINT(39.81607095109448 73.94879749965375) bank162598 +162599 POINT(40.295691547259636 73.6655039613332) bank162599 +162600 POINT(40.088927921780076 73.88391102011174) bank162600 +162601 POINT(40.70223016926236 74.89086637315981) bank162601 +162602 POINT(40.53243691270747 74.90675861240145) bank162602 +162603 POINT(40.61365614552115 73.82502583752917) bank162603 +162604 POINT(41.53520884703696 73.82650214722572) bank162604 +162605 POINT(41.04553208478892 73.42497451087752) bank162605 +162606 POINT(40.000406064960224 73.68791160039709) bank162606 +162607 POINT(40.15795614833219 73.31794279768962) bank162607 +162608 POINT(40.705083269712254 74.81038540968065) bank162608 +162609 POINT(40.308963758351844 74.85806495068708) bank162609 +162610 POINT(41.57407707907953 73.49954046677439) bank162610 +162611 POINT(39.728765912359975 73.77536421679316) bank162611 +162612 POINT(41.484572134362004 73.16212367554714) bank162612 +162613 POINT(41.09691398064175 73.65071572653979) bank162613 +162614 POINT(40.93283111171518 73.06877657376238) bank162614 +162615 POINT(40.124880011185724 73.5272695926477) bank162615 +162616 POINT(41.16996709182905 73.23195419073802) bank162616 +162617 POINT(40.97907679940857 73.96509809334475) bank162617 +162618 POINT(40.38849469631329 74.09084102188041) bank162618 +162619 POINT(40.26495832498612 74.64538264778653) bank162619 +162620 POINT(41.46558190265011 74.01638444953487) bank162620 +162621 POINT(41.43445439364589 74.9855507223749) bank162621 +162622 POINT(40.582107899753034 74.11577201799604) bank162622 +162623 POINT(40.374371844862885 74.66174380646979) bank162623 +162624 POINT(41.190049289722964 74.42556821638189) bank162624 +162625 POINT(40.06397125878976 74.70625194045633) bank162625 +162626 POINT(40.6372968156069 73.37875590731696) bank162626 +162627 POINT(41.211457088013894 74.5076962244809) bank162627 +162628 POINT(41.592762138894415 73.90607334294863) bank162628 +162629 POINT(41.42490053007239 74.08470960581096) bank162629 +162630 POINT(41.50678121653209 74.80115935466597) bank162630 +162631 POINT(40.71114304504357 73.38507543800793) bank162631 +162632 POINT(40.52170992859798 74.8978337524225) bank162632 +162633 POINT(39.80467617093801 74.3505822240817) bank162633 +162634 POINT(41.29902172000832 74.6423327901946) bank162634 +162635 POINT(39.93227505948403 73.26981952116772) bank162635 +162636 POINT(41.392924063455084 74.53033197608791) bank162636 +162637 POINT(41.489579363609174 73.20073123328922) bank162637 +162638 POINT(40.127062877153655 73.23731125445222) bank162638 +162639 POINT(40.0135729983443 74.60245130102263) bank162639 +162640 POINT(39.876411761975966 73.58477705532047) bank162640 +162641 POINT(40.51872551288025 73.75145989501239) bank162641 +162642 POINT(41.658430655151385 73.90958856625545) bank162642 +162643 POINT(39.781967638251864 73.02241592549863) bank162643 +162644 POINT(40.81582773525196 74.45825948345255) bank162644 +162645 POINT(40.05041506210692 73.73750836762846) bank162645 +162646 POINT(41.26232156289697 73.99512268472283) bank162646 +162647 POINT(40.31390505778463 74.24560914047656) bank162647 +162648 POINT(40.75657691740755 73.87944114725902) bank162648 +162649 POINT(41.54223126166398 73.03247308404153) bank162649 +162650 POINT(41.12102368861271 73.41942133094105) bank162650 +162651 POINT(39.95508743422172 74.84017031168571) bank162651 +162652 POINT(39.81267279991179 73.73597441027887) bank162652 +162653 POINT(41.46275798763288 73.97498236685098) bank162653 +162654 POINT(41.31665440194358 73.56234875037893) bank162654 +162655 POINT(40.26117332790231 74.15782626097278) bank162655 +162656 POINT(41.234457845454 73.26055956257365) bank162656 +162657 POINT(41.03851312661728 73.4999133433066) bank162657 +162658 POINT(41.295953543898406 74.89565423935053) bank162658 +162659 POINT(40.38910786569546 73.02319118814472) bank162659 +162660 POINT(40.45890017063111 73.78110669523512) bank162660 +162661 POINT(40.33613264998438 74.19723964689915) bank162661 +162662 POINT(40.22379982555755 74.6325952860981) bank162662 +162663 POINT(40.075710883992166 73.89679463657177) bank162663 +162664 POINT(41.405979555394104 74.77434955516637) bank162664 +162665 POINT(41.212676227319875 73.64849389614757) bank162665 +162666 POINT(41.319767822727734 73.92684303668855) bank162666 +162667 POINT(39.800811209398844 74.92353647774615) bank162667 +162668 POINT(39.99818564511698 73.4010573097023) bank162668 +162669 POINT(41.29172362064299 74.88425289213629) bank162669 +162670 POINT(41.523358751749846 74.05124878414935) bank162670 +162671 POINT(39.995525727633435 74.44502618447837) bank162671 +162672 POINT(41.51473513670448 74.40422993704264) bank162672 +162673 POINT(40.87937439435226 73.28383570163015) bank162673 +162674 POINT(41.325324349299024 74.2436104675866) bank162674 +162675 POINT(40.38961378208616 74.47176372221239) bank162675 +162676 POINT(41.23006051854442 73.70551327795076) bank162676 +162677 POINT(41.56812117966637 74.19885348831664) bank162677 +162678 POINT(39.96124594362688 74.9806660275116) bank162678 +162679 POINT(40.44766807634903 74.93921351124872) bank162679 +162680 POINT(39.87890366220812 73.84533939956691) bank162680 +162681 POINT(41.48478677689241 73.90687149704254) bank162681 +162682 POINT(40.075331842232465 75.00310018885895) bank162682 +162683 POINT(40.91869974886524 73.84934066770936) bank162683 +162684 POINT(40.17074513786341 73.68153119239153) bank162684 +162685 POINT(41.31083670213752 73.75301183140198) bank162685 +162686 POINT(40.50070945283807 73.11187205237655) bank162686 +162687 POINT(40.19591699767622 74.29425087019898) bank162687 +162688 POINT(39.89801214697357 74.21806650050152) bank162688 +162689 POINT(40.94813620640846 74.97773358219712) bank162689 +162690 POINT(41.115032662853544 73.27166008498476) bank162690 +162691 POINT(39.96211426517718 73.666341366791) bank162691 +162692 POINT(41.07801526840665 74.4126129747716) bank162692 +162693 POINT(40.72059992701028 74.10636252870738) bank162693 +162694 POINT(41.515803973389104 73.20982450360925) bank162694 +162695 POINT(40.757238807020165 74.73725735375587) bank162695 +162696 POINT(40.48345402542601 74.57855247729233) bank162696 +162697 POINT(40.39488867075402 73.38125249207428) bank162697 +162698 POINT(39.91271447186547 74.81229313785916) bank162698 +162699 POINT(40.280263048421816 74.40228583673074) bank162699 +162700 POINT(41.18957813265302 73.08064404366509) bank162700 +162701 POINT(41.38292347518579 73.52384499618876) bank162701 +162702 POINT(41.17405932721851 74.42206876050899) bank162702 +162703 POINT(40.09238211548104 73.51317068764476) bank162703 +162704 POINT(40.101419975014274 73.44916851098435) bank162704 +162705 POINT(41.14766557784044 73.02636933275372) bank162705 +162706 POINT(40.12636582098467 73.87775977721044) bank162706 +162707 POINT(40.09418534672237 74.46780409724103) bank162707 +162708 POINT(40.67427688676126 74.03769569395129) bank162708 +162709 POINT(39.972204258424036 74.57226965547021) bank162709 +162710 POINT(40.92289117913292 73.1273827560418) bank162710 +162711 POINT(40.28308300313443 74.250268036684) bank162711 +162712 POINT(39.72341931962725 74.80356781716523) bank162712 +162713 POINT(39.90337214016719 73.87448837310012) bank162713 +162714 POINT(39.803554051439434 74.5062498975082) bank162714 +162715 POINT(41.17223440388158 74.23203375765402) bank162715 +162716 POINT(40.330167852422086 74.98156573607616) bank162716 +162717 POINT(41.318959537677564 74.00891359779513) bank162717 +162718 POINT(40.74417657636838 73.9678522084603) bank162718 +162719 POINT(41.632426078353404 74.96823905625165) bank162719 +162720 POINT(39.851920820291426 73.21013960035454) bank162720 +162721 POINT(39.884345978831476 74.12136857064036) bank162721 +162722 POINT(40.03334718845293 73.35622070094962) bank162722 +162723 POINT(40.81166482584533 73.78454606962019) bank162723 +162724 POINT(39.93346506856542 74.36258734582997) bank162724 +162725 POINT(41.60018899947291 74.47850941346329) bank162725 +162726 POINT(40.28458542056239 73.18039310631247) bank162726 +162727 POINT(40.61817549383608 73.39554529235664) bank162727 +162728 POINT(41.53352701720382 73.1025743949973) bank162728 +162729 POINT(40.90406959594425 74.14980020755092) bank162729 +162730 POINT(41.100288067632626 73.81964798755874) bank162730 +162731 POINT(41.668564616558726 73.89901327272979) bank162731 +162732 POINT(41.00418330819346 73.5898862039632) bank162732 +162733 POINT(39.79829039048812 74.71454873765718) bank162733 +162734 POINT(41.208558290901 73.43448850111439) bank162734 +162735 POINT(40.68783925414461 73.7170447581428) bank162735 +162736 POINT(41.12077309939118 74.50862125600305) bank162736 +162737 POINT(40.140637996028396 73.43448296843363) bank162737 +162738 POINT(40.917605408536794 73.48480799861005) bank162738 +162739 POINT(40.857193438810896 74.36611671026189) bank162739 +162740 POINT(41.191962126927805 73.7372443906261) bank162740 +162741 POINT(41.02663349704565 74.53134583650856) bank162741 +162742 POINT(40.622534310018345 74.83872774228772) bank162742 +162743 POINT(39.94698218314731 73.91047272360801) bank162743 +162744 POINT(39.86486855084823 74.57755918270402) bank162744 +162745 POINT(41.104830489886346 73.32113560830129) bank162745 +162746 POINT(40.367833477650535 73.28160673905943) bank162746 +162747 POINT(41.45652003901422 73.18846361998871) bank162747 +162748 POINT(39.751396277961305 73.6599869943166) bank162748 +162749 POINT(40.85697394412863 74.9105505409652) bank162749 +162750 POINT(41.63463497706349 74.2410044224927) bank162750 +162751 POINT(40.277707183518885 74.28332527018927) bank162751 +162752 POINT(41.57692438930917 74.9200525421057) bank162752 +162753 POINT(40.38383558328599 73.70820719012269) bank162753 +162754 POINT(40.272217670972445 74.48258510045515) bank162754 +162755 POINT(41.007445897170214 74.52688915575528) bank162755 +162756 POINT(41.467608132627504 74.76823955329147) bank162756 +162757 POINT(39.87259918361221 74.93569341577202) bank162757 +162758 POINT(39.828198707100114 74.62782256195739) bank162758 +162759 POINT(40.1727089892709 74.72780104305717) bank162759 +162760 POINT(39.90859572864289 73.3917386149735) bank162760 +162761 POINT(40.83418086257133 73.01757180732) bank162761 +162762 POINT(40.19589050137982 74.3700558186682) bank162762 +162763 POINT(40.47340237511907 73.21895872599272) bank162763 +162764 POINT(40.80632545639208 75.00531252504747) bank162764 +162765 POINT(40.96794714516635 74.67758988073855) bank162765 +162766 POINT(40.21423143218216 74.23328557522015) bank162766 +162767 POINT(41.45343557970367 74.25197129844669) bank162767 +162768 POINT(40.111370667889815 73.53048420393085) bank162768 +162769 POINT(40.53657249274939 74.76508875030154) bank162769 +162770 POINT(41.02232082706353 74.022406505932) bank162770 +162771 POINT(39.79715378001367 73.77152104066678) bank162771 +162772 POINT(40.02367103092056 74.9483124324574) bank162772 +162773 POINT(41.63336146201866 73.22834759934618) bank162773 +162774 POINT(40.51460358201512 73.68422400997156) bank162774 +162775 POINT(40.06900304744985 74.59519967550901) bank162775 +162776 POINT(41.260123917955696 73.79580667441147) bank162776 +162777 POINT(40.34083429248603 73.35523880125159) bank162777 +162778 POINT(39.86937884647529 73.06376706119266) bank162778 +162779 POINT(40.85162302364639 74.2010921765428) bank162779 +162780 POINT(41.386626618959866 73.90521580674806) bank162780 +162781 POINT(40.09653890461645 74.08050360662116) bank162781 +162782 POINT(41.511528303389206 73.24111735417969) bank162782 +162783 POINT(40.47479490006724 73.6480966509362) bank162783 +162784 POINT(40.535578755350436 73.77045329798558) bank162784 +162785 POINT(40.171292709392176 74.16823833930464) bank162785 +162786 POINT(40.977033290969146 73.5378489953758) bank162786 +162787 POINT(40.52748595790214 74.42625561155742) bank162787 +162788 POINT(39.76481461030464 74.51393541915071) bank162788 +162789 POINT(40.56860838448134 74.7413506847171) bank162789 +162790 POINT(40.239259831939485 73.96119144096821) bank162790 +162791 POINT(41.27638712500808 73.54933273026752) bank162791 +162792 POINT(40.12295350951858 73.54606636804407) bank162792 +162793 POINT(40.56594766673649 73.20574570199445) bank162793 +162794 POINT(39.97641554130033 74.65013146265376) bank162794 +162795 POINT(40.99991012205546 73.75777867721672) bank162795 +162796 POINT(41.19150496604303 73.61836779642638) bank162796 +162797 POINT(41.02002133877218 73.726263161992) bank162797 +162798 POINT(40.80426779774139 73.63165762584629) bank162798 +162799 POINT(40.503214858256584 73.04890288060398) bank162799 +162800 POINT(41.29639903092458 74.20378227896258) bank162800 +162801 POINT(41.69216561797284 74.67399409810683) bank162801 +162802 POINT(41.51163387182822 74.44277466849339) bank162802 +162803 POINT(41.56130047227975 74.07585596862678) bank162803 +162804 POINT(40.671865109736764 74.73186622543844) bank162804 +162805 POINT(40.62798835442259 74.97590645402177) bank162805 +162806 POINT(41.63795310507315 73.96509153085414) bank162806 +162807 POINT(39.917540124593415 74.25251452232496) bank162807 +162808 POINT(40.864242766550355 73.11922148637913) bank162808 +162809 POINT(40.24431189109909 74.10102308112407) bank162809 +162810 POINT(40.27349742403497 74.82366419054904) bank162810 +162811 POINT(41.41702449233232 74.01652767625227) bank162811 +162812 POINT(40.02692951670062 73.23236297485126) bank162812 +162813 POINT(41.36802698953381 73.93308760041457) bank162813 +162814 POINT(41.55392901580051 74.82802526230459) bank162814 +162815 POINT(41.652914955589864 74.4752423551791) bank162815 +162816 POINT(40.78877821245882 74.11048431112067) bank162816 +162817 POINT(40.63185097780053 73.69398813880015) bank162817 +162818 POINT(40.69443428084559 73.68067882084866) bank162818 +162819 POINT(40.492515736882545 73.8314771598131) bank162819 +162820 POINT(40.1945294303989 74.18990460507123) bank162820 +162821 POINT(41.21641358576913 74.12911627729719) bank162821 +162822 POINT(41.68002809171791 74.28895453087308) bank162822 +162823 POINT(40.540267184774756 73.40509921437824) bank162823 +162824 POINT(40.127094164873895 74.6531591601871) bank162824 +162825 POINT(41.52320786562942 73.36903264543055) bank162825 +162826 POINT(41.21693592804106 74.18904966124303) bank162826 +162827 POINT(39.86596392124504 73.44923852506382) bank162827 +162828 POINT(39.820785547090274 73.80317784950972) bank162828 +162829 POINT(39.94099000592119 73.97164437288706) bank162829 +162830 POINT(40.1426684019855 73.70042689969051) bank162830 +162831 POINT(40.04464984718891 73.82504615625818) bank162831 +162832 POINT(40.0014329619603 73.97080897525531) bank162832 +162833 POINT(40.402770570719404 73.83302782001415) bank162833 +162834 POINT(40.46210385307791 74.56221053639113) bank162834 +162835 POINT(39.76637213418945 74.65445095693684) bank162835 +162836 POINT(40.66999000919071 73.68208210077087) bank162836 +162837 POINT(40.357317707174886 73.00874945300652) bank162837 +162838 POINT(39.92585893273609 73.48731169691874) bank162838 +162839 POINT(39.84776684781646 74.93999999947108) bank162839 +162840 POINT(39.99586787466061 74.75995584720651) bank162840 +162841 POINT(41.1679892034357 73.63025803808895) bank162841 +162842 POINT(41.540345453985864 73.61338974886993) bank162842 +162843 POINT(40.80606855128875 73.50908348072043) bank162843 +162844 POINT(40.53384558702661 74.71835394393491) bank162844 +162845 POINT(40.57450044416401 73.74829561748895) bank162845 +162846 POINT(40.90577039735433 74.34905032203702) bank162846 +162847 POINT(40.889319534182476 74.80256229134139) bank162847 +162848 POINT(40.545636125775324 73.57808629367845) bank162848 +162849 POINT(41.27220038622187 74.72127039507926) bank162849 +162850 POINT(39.74900074150886 74.04314204623789) bank162850 +162851 POINT(40.979471450969434 74.37223971131792) bank162851 +162852 POINT(40.55476553023226 74.04299781382623) bank162852 +162853 POINT(40.49797060355355 73.71692345658546) bank162853 +162854 POINT(41.08125228839231 74.9092293034122) bank162854 +162855 POINT(39.982507350007836 74.76020328943312) bank162855 +162856 POINT(39.717729037325086 73.85269270419931) bank162856 +162857 POINT(41.39347025220524 73.80923464794337) bank162857 +162858 POINT(40.12730083965901 74.70573652782201) bank162858 +162859 POINT(40.50329956207162 74.12530349090598) bank162859 +162860 POINT(41.00580331351391 74.18142037699373) bank162860 +162861 POINT(40.38104404703711 74.70411557751426) bank162861 +162862 POINT(40.385092731869705 73.18171819498492) bank162862 +162863 POINT(39.717437198959296 74.463514422282) bank162863 +162864 POINT(39.99802157903917 74.50146277402774) bank162864 +162865 POINT(41.2094074571387 74.58717552492958) bank162865 +162866 POINT(40.398031422891236 74.00266840749426) bank162866 +162867 POINT(40.99806884565048 73.67823538779842) bank162867 +162868 POINT(41.63825285485618 73.92571632271087) bank162868 +162869 POINT(40.77064121837149 73.68898532582722) bank162869 +162870 POINT(40.75588193398927 74.9466878064101) bank162870 +162871 POINT(40.42067660371729 73.16114634317941) bank162871 +162872 POINT(40.27727507448389 74.94572145519787) bank162872 +162873 POINT(41.17873783305056 74.90498848289425) bank162873 +162874 POINT(41.440085660910725 73.08028025978231) bank162874 +162875 POINT(40.86648918753649 74.50354980555004) bank162875 +162876 POINT(41.62381589983802 73.63543982377423) bank162876 +162877 POINT(41.06593776782639 73.53792147619478) bank162877 +162878 POINT(40.37005993638356 74.07883693349349) bank162878 +162879 POINT(41.00443446381561 74.34644672997818) bank162879 +162880 POINT(40.93040516046066 73.97849073735486) bank162880 +162881 POINT(40.74040575300949 74.42014733380061) bank162881 +162882 POINT(40.05540135020661 73.32060437698497) bank162882 +162883 POINT(41.702490714200216 73.04320554301448) bank162883 +162884 POINT(40.09446867763355 73.72640150797352) bank162884 +162885 POINT(40.568137760175034 74.90590268677077) bank162885 +162886 POINT(41.189794101519716 73.76199543308753) bank162886 +162887 POINT(41.25429626572239 73.36550919957003) bank162887 +162888 POINT(40.98973682501834 73.57767604772188) bank162888 +162889 POINT(40.60797172126729 74.83871328159945) bank162889 +162890 POINT(40.85305051353312 74.3410735297287) bank162890 +162891 POINT(40.4494033493662 74.31187475959511) bank162891 +162892 POINT(40.516342813264444 73.99407227993268) bank162892 +162893 POINT(41.379606055540584 73.6240500937805) bank162893 +162894 POINT(40.477676379964734 74.69679989565569) bank162894 +162895 POINT(41.48359456263246 73.94707989071719) bank162895 +162896 POINT(40.30300663442254 74.11388053960127) bank162896 +162897 POINT(40.808654093990505 74.12223914640495) bank162897 +162898 POINT(39.8863003791846 73.09326509678571) bank162898 +162899 POINT(40.20912043468852 74.17395670595184) bank162899 +162900 POINT(40.84467557156157 73.46132467109548) bank162900 +162901 POINT(40.92960335604496 73.753694300099) bank162901 +162902 POINT(41.238576190341774 73.54589534392518) bank162902 +162903 POINT(41.42298377071223 73.36171909645243) bank162903 +162904 POINT(41.55064836063933 74.46628967332417) bank162904 +162905 POINT(41.6708822569034 74.69813837672139) bank162905 +162906 POINT(40.87107668225068 74.05014375423322) bank162906 +162907 POINT(40.406345879250594 74.99520401539772) bank162907 +162908 POINT(41.677573756505716 73.60967140797688) bank162908 +162909 POINT(40.326035675471594 73.24361640621119) bank162909 +162910 POINT(39.76196153329853 74.98160302389287) bank162910 +162911 POINT(41.49822579244912 73.12244596736313) bank162911 +162912 POINT(40.6045086204863 74.04876485739247) bank162912 +162913 POINT(39.714854402238046 73.83414913641894) bank162913 +162914 POINT(40.527386387825224 73.17470012125794) bank162914 +162915 POINT(40.61766460440246 74.873641355842) bank162915 +162916 POINT(40.16700890024105 73.97511556994928) bank162916 +162917 POINT(41.49369078274485 73.34669711789957) bank162917 +162918 POINT(41.41398850989489 73.88735119129757) bank162918 +162919 POINT(39.76596842772219 74.46440373269826) bank162919 +162920 POINT(40.984633697542606 74.13889898406835) bank162920 +162921 POINT(40.47782073678346 74.11761800398868) bank162921 +162922 POINT(40.22014110768279 73.3376813829592) bank162922 +162923 POINT(40.550680394325916 74.85600990060435) bank162923 +162924 POINT(41.350230022889235 73.017104108006) bank162924 +162925 POINT(41.377630620117515 73.78059726308807) bank162925 +162926 POINT(39.95477907011339 75.00188000257344) bank162926 +162927 POINT(40.387633572109564 73.7579745640111) bank162927 +162928 POINT(41.14553429050297 73.45730410437267) bank162928 +162929 POINT(40.21449771912407 73.85913019708843) bank162929 +162930 POINT(41.66374071589571 74.91252614226353) bank162930 +162931 POINT(40.06672502569834 73.09774934355893) bank162931 +162932 POINT(40.06166628255525 73.15363294504974) bank162932 +162933 POINT(41.32481113615007 73.95592269908208) bank162933 +162934 POINT(40.3682594623414 73.05561528905622) bank162934 +162935 POINT(40.39414775838675 74.52977412140038) bank162935 +162936 POINT(40.176573303803735 73.9015750393834) bank162936 +162937 POINT(41.12851667629083 74.2115465477603) bank162937 +162938 POINT(40.77376817605093 73.90001927254106) bank162938 +162939 POINT(41.42927649669497 73.17959433372539) bank162939 +162940 POINT(41.07818916382785 74.60705987396173) bank162940 +162941 POINT(40.370180389282005 73.21529490206) bank162941 +162942 POINT(40.48906139919369 73.6119108071801) bank162942 +162943 POINT(40.2426321302641 73.78220444315693) bank162943 +162944 POINT(41.223960696537 73.52377625333668) bank162944 +162945 POINT(41.201436086478424 74.49746390697733) bank162945 +162946 POINT(41.007078231002296 74.3032388540702) bank162946 +162947 POINT(39.74561829254624 73.90403281146304) bank162947 +162948 POINT(40.81381022331553 74.92821472864219) bank162948 +162949 POINT(41.71165033635054 74.95994686506864) bank162949 +162950 POINT(41.141590960762834 74.89024817488728) bank162950 +162951 POINT(39.980268838865186 73.03206267915715) bank162951 +162952 POINT(40.44602556679917 74.8705333896872) bank162952 +162953 POINT(40.738390890172944 73.94260187728209) bank162953 +162954 POINT(40.85308296679293 73.48888192106337) bank162954 +162955 POINT(41.48899673380019 74.11057805498703) bank162955 +162956 POINT(41.61086054281826 74.0014215250423) bank162956 +162957 POINT(41.24520925506518 74.83079195353844) bank162957 +162958 POINT(41.00000869497747 74.3007178993812) bank162958 +162959 POINT(39.83700562750162 74.86919170105061) bank162959 +162960 POINT(40.579600705858034 74.5759418048783) bank162960 +162961 POINT(41.20142067735855 73.78211983032409) bank162961 +162962 POINT(41.3541747185488 74.69970937761823) bank162962 +162963 POINT(41.282690940723036 74.70016620495278) bank162963 +162964 POINT(40.352609320717534 73.75084610737804) bank162964 +162965 POINT(40.46772019176959 73.73648710828395) bank162965 +162966 POINT(40.11472919819664 74.73807387578151) bank162966 +162967 POINT(40.89279086723077 73.91488875761269) bank162967 +162968 POINT(40.0710967541787 73.68830965657779) bank162968 +162969 POINT(40.83422440407126 74.4734476408886) bank162969 +162970 POINT(41.500911305736835 74.08690939750974) bank162970 +162971 POINT(41.06087137878929 73.86196611612093) bank162971 +162972 POINT(41.424905349848764 73.81401473946298) bank162972 +162973 POINT(41.034530761383365 73.17813009441642) bank162973 +162974 POINT(41.228132072871006 73.07376127072904) bank162974 +162975 POINT(39.78609009533206 74.59862457034704) bank162975 +162976 POINT(40.76494698631245 73.268789333577) bank162976 +162977 POINT(40.5903460629294 74.5028678507818) bank162977 +162978 POINT(39.91063662205915 74.37001297966853) bank162978 +162979 POINT(40.99911006180477 74.59094352840681) bank162979 +162980 POINT(41.00431704353783 74.15647318564261) bank162980 +162981 POINT(40.91212085697354 73.10350696892183) bank162981 +162982 POINT(40.57050363003235 73.43442522018337) bank162982 +162983 POINT(41.616874096986976 73.48071329560258) bank162983 +162984 POINT(40.802412607487405 74.1352831050696) bank162984 +162985 POINT(40.52801861725953 73.03036584911146) bank162985 +162986 POINT(40.503500146430866 74.91590238108166) bank162986 +162987 POINT(40.07011377379855 73.63213562447814) bank162987 +162988 POINT(40.88545451162831 74.82756703806113) bank162988 +162989 POINT(40.31972887341943 74.3761516717186) bank162989 +162990 POINT(40.16098280700165 74.96862486449531) bank162990 +162991 POINT(41.44996076096821 74.66540354717198) bank162991 +162992 POINT(40.76140610138826 73.58688561004476) bank162992 +162993 POINT(40.66817603045587 73.23555338205301) bank162993 +162994 POINT(40.89327342137622 73.57720534288694) bank162994 +162995 POINT(39.7871735184052 74.123394737874) bank162995 +162996 POINT(41.00059500157029 73.70831799846958) bank162996 +162997 POINT(40.07198968249228 74.71662373405546) bank162997 +162998 POINT(41.352834543424684 73.85659733306208) bank162998 +162999 POINT(40.882229727157245 73.36147021119757) bank162999 +163000 POINT(41.53581227719559 73.44182474467401) bank163000 +163001 POINT(40.47988316738746 74.93316853392163) bank163001 +163002 POINT(41.0681471789258 74.23003691342335) bank163002 +163003 POINT(40.809238282543575 73.60448117913751) bank163003 +163004 POINT(39.83261054147219 73.80434491620761) bank163004 +163005 POINT(40.516638297109246 74.93593105473497) bank163005 +163006 POINT(40.6744277482942 74.41896205483526) bank163006 +163007 POINT(40.005927782907186 74.27316469958645) bank163007 +163008 POINT(41.52001204351319 74.83366031280384) bank163008 +163009 POINT(41.0440068170385 74.39544694390374) bank163009 +163010 POINT(40.125925551363416 74.8826646591224) bank163010 +163011 POINT(40.09106055305992 73.49704427348718) bank163011 +163012 POINT(39.80400838430319 74.19350422282767) bank163012 +163013 POINT(40.76550050309324 74.74963482395503) bank163013 +163014 POINT(40.26589938714345 74.23039267644502) bank163014 +163015 POINT(41.40869191781296 74.87456615831985) bank163015 +163016 POINT(40.25195864059006 74.95901548061249) bank163016 +163017 POINT(40.31868512396765 74.19981333981036) bank163017 +163018 POINT(41.54871849277993 74.61146715921534) bank163018 +163019 POINT(40.95529088885271 73.20453661514419) bank163019 +163020 POINT(41.35954948021742 74.00390269889562) bank163020 +163021 POINT(41.02808575939808 74.7527328425266) bank163021 +163022 POINT(40.76826336056058 73.11587269007596) bank163022 +163023 POINT(40.257301676425925 73.82140024778352) bank163023 +163024 POINT(41.283690072900946 73.9289586677691) bank163024 +163025 POINT(41.34097967923737 74.55014513620894) bank163025 +163026 POINT(40.041267109266485 74.75855878621225) bank163026 +163027 POINT(40.44060569480336 74.50957145557405) bank163027 +163028 POINT(40.3207522002829 73.21120463942462) bank163028 +163029 POINT(41.04516876564631 73.07655714169337) bank163029 +163030 POINT(41.60760455369061 73.34838766766946) bank163030 +163031 POINT(41.614721632228054 73.88260388261058) bank163031 +163032 POINT(40.373698156184666 74.13818831635216) bank163032 +163033 POINT(41.66891466511393 73.09136180122947) bank163033 +163034 POINT(40.3340908564107 73.68373637083162) bank163034 +163035 POINT(40.311062533855605 73.82650979851915) bank163035 +163036 POINT(39.74753596366073 74.88152327205496) bank163036 +163037 POINT(40.25977870008003 73.81542473738362) bank163037 +163038 POINT(39.81409291989307 73.80785999782387) bank163038 +163039 POINT(41.10978878120127 73.79574203230585) bank163039 +163040 POINT(40.956883282501934 74.83101170899953) bank163040 +163041 POINT(40.4882657658235 73.38255380280528) bank163041 +163042 POINT(40.35473508530368 73.38517188920702) bank163042 +163043 POINT(40.29900599661603 73.93537609901608) bank163043 +163044 POINT(40.86126581562696 73.32286094094223) bank163044 +163045 POINT(41.08160902230692 74.11041655717051) bank163045 +163046 POINT(40.78121091889172 74.27340924418122) bank163046 +163047 POINT(41.337502589043595 74.57639393717386) bank163047 +163048 POINT(40.20313316412279 73.88295980324224) bank163048 +163049 POINT(41.50656878557499 74.43611786477274) bank163049 +163050 POINT(40.48007370674243 73.33947392164757) bank163050 +163051 POINT(41.355502340799354 73.27254048519276) bank163051 +163052 POINT(40.63022444322762 74.49084520911545) bank163052 +163053 POINT(39.74008019697982 73.2100478860568) bank163053 +163054 POINT(40.83906497568575 74.21982232985022) bank163054 +163055 POINT(39.786897819121386 73.24151932510061) bank163055 +163056 POINT(39.85897384651519 73.80288381921817) bank163056 +163057 POINT(40.85874272231114 74.6548846543263) bank163057 +163058 POINT(40.735826602380584 73.61775140421051) bank163058 +163059 POINT(40.47110371420051 74.84548705754531) bank163059 +163060 POINT(41.37618663960595 74.42069126976996) bank163060 +163061 POINT(41.18610842668987 73.62394809859525) bank163061 +163062 POINT(41.4361833113025 74.9580700970955) bank163062 +163063 POINT(41.15924719998849 74.20958855710079) bank163063 +163064 POINT(39.78035420792423 73.21212311225595) bank163064 +163065 POINT(39.79201490556886 73.1675192225433) bank163065 +163066 POINT(40.44005952602038 74.64660865553822) bank163066 +163067 POINT(40.31784944344291 73.35347391772635) bank163067 +163068 POINT(41.50609625396396 74.42554598944751) bank163068 +163069 POINT(39.87235211749647 74.77704844819279) bank163069 +163070 POINT(41.353385127962895 73.72716222225023) bank163070 +163071 POINT(40.54079013963871 73.84928054053553) bank163071 +163072 POINT(40.363376565775575 73.22006937880323) bank163072 +163073 POINT(41.55361518283809 73.93718906773078) bank163073 +163074 POINT(40.829226233294925 73.73998216653534) bank163074 +163075 POINT(41.05492755468325 73.94130605630637) bank163075 +163076 POINT(41.63956459368326 73.144640842283) bank163076 +163077 POINT(41.060372896442665 73.68381636961053) bank163077 +163078 POINT(41.65887133685799 73.9242649790774) bank163078 +163079 POINT(41.62855207522146 73.19988992953908) bank163079 +163080 POINT(40.007117351353614 73.63291614600871) bank163080 +163081 POINT(41.507871903856895 74.77232053784003) bank163081 +163082 POINT(39.88075474829238 73.62928137761396) bank163082 +163083 POINT(41.576568074164214 73.65018259990704) bank163083 +163084 POINT(40.87553759630082 73.29539136370904) bank163084 +163085 POINT(40.11544755564152 74.11108346309581) bank163085 +163086 POINT(40.71795381890251 73.53338449812165) bank163086 +163087 POINT(41.47290240282474 74.55088440564282) bank163087 +163088 POINT(40.3011838707374 74.24567707242755) bank163088 +163089 POINT(40.19146079245872 73.84582322470955) bank163089 +163090 POINT(40.30535154965624 73.25673385517437) bank163090 +163091 POINT(41.10254521518753 73.32804807856745) bank163091 +163092 POINT(39.92704558686185 73.75788144601802) bank163092 +163093 POINT(40.97067925775716 74.96660479142515) bank163093 +163094 POINT(39.725702575783714 73.02778729359778) bank163094 +163095 POINT(40.69516885402799 73.85851715068381) bank163095 +163096 POINT(41.332915655545165 74.35462095637047) bank163096 +163097 POINT(40.0686346285415 73.18544508698267) bank163097 +163098 POINT(40.1229727327333 74.25891800485958) bank163098 +163099 POINT(40.825047666699476 74.89812082929936) bank163099 +163100 POINT(39.881740636426464 73.47906660319407) bank163100 +163101 POINT(40.74621418074374 73.85010555566045) bank163101 +163102 POINT(39.839908670756586 73.99364764234696) bank163102 +163103 POINT(40.710149898656304 74.61870250059295) bank163103 +163104 POINT(39.92325861351405 74.61443321223976) bank163104 +163105 POINT(41.471628412847934 74.39993366877553) bank163105 +163106 POINT(41.53094389124995 73.46170692973375) bank163106 +163107 POINT(39.883927579078495 73.54042226109085) bank163107 +163108 POINT(41.001944190368846 73.5608345712821) bank163108 +163109 POINT(40.377665961776806 74.60999480032342) bank163109 +163110 POINT(40.03236428491625 74.71804521868741) bank163110 +163111 POINT(40.72291152622262 74.39862237450356) bank163111 +163112 POINT(40.34321335632332 73.4673194266666) bank163112 +163113 POINT(40.22438022052432 74.30242917666155) bank163113 +163114 POINT(40.0934929391395 74.8569702161286) bank163114 +163115 POINT(40.5877040597725 73.30563147584122) bank163115 +163116 POINT(40.61773319618659 73.14157696939358) bank163116 +163117 POINT(40.479957320860855 73.89344159653209) bank163117 +163118 POINT(40.337871620460795 73.3996238082389) bank163118 +163119 POINT(41.379992804496084 73.54030282963198) bank163119 +163120 POINT(40.64727290218502 73.77945376478215) bank163120 +163121 POINT(40.5478041303693 74.84205274263815) bank163121 +163122 POINT(41.27298244494788 74.03884873961223) bank163122 +163123 POINT(40.1583457673122 73.32580331328765) bank163123 +163124 POINT(41.22903660758225 73.85333958082205) bank163124 +163125 POINT(41.21361119521304 74.84849999621733) bank163125 +163126 POINT(39.73376865656258 73.04740971036914) bank163126 +163127 POINT(40.4915813642712 74.40083457161658) bank163127 +163128 POINT(39.89945110429165 73.29755762838593) bank163128 +163129 POINT(39.954084112484075 73.13484292996255) bank163129 +163130 POINT(40.092849135308406 73.85812293743108) bank163130 +163131 POINT(39.78567151590348 73.64551200487877) bank163131 +163132 POINT(40.01111770397589 74.6520031155952) bank163132 +163133 POINT(40.25531061978761 74.67934421417314) bank163133 +163134 POINT(40.4920638635347 74.40714101092773) bank163134 +163135 POINT(40.728205546917145 73.75713383972004) bank163135 +163136 POINT(39.976733755953354 73.16235499452885) bank163136 +163137 POINT(40.11317240323241 73.12998200616562) bank163137 +163138 POINT(40.758063361853445 73.01593097826412) bank163138 +163139 POINT(40.985927968865724 74.03400756048273) bank163139 +163140 POINT(40.62981798382241 74.79693223665896) bank163140 +163141 POINT(40.16879803594848 74.89563859787985) bank163141 +163142 POINT(41.37256576083519 74.0669901214616) bank163142 +163143 POINT(41.539944126209285 74.18455397663313) bank163143 +163144 POINT(41.687570762800746 74.9414729148412) bank163144 +163145 POINT(40.334407218591416 73.69756026449161) bank163145 +163146 POINT(41.60188878010759 73.72192995085882) bank163146 +163147 POINT(40.74458757622947 73.41428647758741) bank163147 +163148 POINT(40.912751833076385 74.92619219934359) bank163148 +163149 POINT(40.9365094605151 73.38963179817527) bank163149 +163150 POINT(41.01394787393723 74.94718437916892) bank163150 +163151 POINT(41.54931058486658 74.32775937852514) bank163151 +163152 POINT(39.9532395105708 74.08484961120865) bank163152 +163153 POINT(40.20013965698074 74.69746297458389) bank163153 +163154 POINT(41.17343551985869 73.7846685715761) bank163154 +163155 POINT(41.61860973125906 73.06855065445485) bank163155 +163156 POINT(41.397533556263575 73.3806510615387) bank163156 +163157 POINT(40.5084982543972 74.86322334847297) bank163157 +163158 POINT(40.929478399762246 73.14379677873268) bank163158 +163159 POINT(41.15852369388743 73.3947572740141) bank163159 +163160 POINT(40.878555090549355 74.30528602966282) bank163160 +163161 POINT(40.457967407846326 74.27932587186041) bank163161 +163162 POINT(40.438973019680596 74.9600708158547) bank163162 +163163 POINT(40.988866685738465 73.17229484436001) bank163163 +163164 POINT(40.06147428676917 73.92296026021643) bank163164 +163165 POINT(39.864969962899465 74.25507276957977) bank163165 +163166 POINT(41.046645011582704 74.36231255840721) bank163166 +163167 POINT(40.71337105882685 73.43774507077956) bank163167 +163168 POINT(40.89240619075255 73.80910420779962) bank163168 +163169 POINT(41.07206972842959 73.79239373331619) bank163169 +163170 POINT(41.65810708155446 74.90903064997643) bank163170 +163171 POINT(40.95175231994469 73.68994942386358) bank163171 +163172 POINT(40.092296637652 73.21121808152664) bank163172 +163173 POINT(39.747971685322184 74.6547731126014) bank163173 +163174 POINT(40.5174365853891 73.78242333381425) bank163174 +163175 POINT(39.737185119556685 73.39141818050012) bank163175 +163176 POINT(40.123493113124546 73.98922713713065) bank163176 +163177 POINT(40.97226301206503 74.97206625680664) bank163177 +163178 POINT(40.56427152953338 73.21467327038783) bank163178 +163179 POINT(40.765989761471296 73.31138408675983) bank163179 +163180 POINT(40.064127215477576 73.32920465300026) bank163180 +163181 POINT(39.899484178908 73.04638199343003) bank163181 +163182 POINT(41.39351703631607 74.97554041145547) bank163182 +163183 POINT(41.68748636623083 74.51572872165853) bank163183 +163184 POINT(41.3091066049077 74.9345905019528) bank163184 +163185 POINT(40.45811758079522 73.06282009524112) bank163185 +163186 POINT(40.088309420285185 73.82525084250904) bank163186 +163187 POINT(41.505828456867704 74.79742194801281) bank163187 +163188 POINT(41.44307323518912 74.2778576182983) bank163188 +163189 POINT(39.98803090781224 73.91466274506267) bank163189 +163190 POINT(41.3163979756588 74.14334778702177) bank163190 +163191 POINT(40.14425254339335 73.39980437440187) bank163191 +163192 POINT(41.651388651369146 73.01051168655644) bank163192 +163193 POINT(40.53216263948078 74.52839367905068) bank163193 +163194 POINT(41.33571178080756 74.69653526596144) bank163194 +163195 POINT(40.660215378020446 74.21976932068905) bank163195 +163196 POINT(41.3853553260493 73.62014094049354) bank163196 +163197 POINT(40.884991785697814 74.78160641452902) bank163197 +163198 POINT(41.59448854395443 74.32100976852689) bank163198 +163199 POINT(41.6837039437349 74.49559534773731) bank163199 +163200 POINT(41.21681594299063 74.99829983783894) bank163200 +163201 POINT(41.57503125744784 74.15879165556576) bank163201 +163202 POINT(40.8731853497787 74.85272522506214) bank163202 +163203 POINT(40.69952460266137 73.351038699373) bank163203 +163204 POINT(41.657737494667714 73.81672060922654) bank163204 +163205 POINT(40.01104418035636 73.03870525187088) bank163205 +163206 POINT(39.91217639135931 73.173558126035) bank163206 +163207 POINT(41.592449634214034 74.76681410266957) bank163207 +163208 POINT(39.83555444555379 73.82622851829589) bank163208 +163209 POINT(40.6357552696067 74.0221100519119) bank163209 +163210 POINT(41.00513266555029 73.94091101169745) bank163210 +163211 POINT(41.6255429674763 74.86031351719167) bank163211 +163212 POINT(41.5522645989509 74.95029421241638) bank163212 +163213 POINT(41.133653480578154 74.3814986668295) bank163213 +163214 POINT(40.24851091693677 73.68221157970645) bank163214 +163215 POINT(41.26465997781972 74.76924646466128) bank163215 +163216 POINT(41.39085880849707 74.53479636500764) bank163216 +163217 POINT(41.44080263139922 73.90773420191319) bank163217 +163218 POINT(40.583327745195426 74.3058583964076) bank163218 +163219 POINT(40.60322845434499 73.15300998957058) bank163219 +163220 POINT(40.402924758897065 73.56465437033152) bank163220 +163221 POINT(41.16856503868295 74.40959567695126) bank163221 +163222 POINT(40.83928871687925 74.86006078080402) bank163222 +163223 POINT(41.694454655428636 74.31998461307957) bank163223 +163224 POINT(39.73907376253713 74.37002480517383) bank163224 +163225 POINT(40.05206200268648 73.72244334978839) bank163225 +163226 POINT(41.182748685190504 73.26801154302117) bank163226 +163227 POINT(40.694916273608776 74.52204722100514) bank163227 +163228 POINT(41.46167035269831 73.40986963046319) bank163228 +163229 POINT(39.80051470808497 73.37731379063645) bank163229 +163230 POINT(41.334851329877225 73.14746125461512) bank163230 +163231 POINT(39.991229767115506 74.0833816006875) bank163231 +163232 POINT(41.39765596176499 74.99671077870751) bank163232 +163233 POINT(40.375384014357 74.63873186566543) bank163233 +163234 POINT(40.79809865023964 73.83775273274348) bank163234 +163235 POINT(40.996197228455294 73.36246567740665) bank163235 +163236 POINT(41.690699746911406 73.88461317168189) bank163236 +163237 POINT(39.82568588142944 74.70943834451816) bank163237 +163238 POINT(40.22892700731766 73.12819363804738) bank163238 +163239 POINT(40.52637852731702 73.374428762534) bank163239 +163240 POINT(41.0240591090878 73.70804551497477) bank163240 +163241 POINT(41.143345960559444 74.6784635491131) bank163241 +163242 POINT(40.03497671446168 74.86333049816544) bank163242 +163243 POINT(39.77316801692204 74.59292151015028) bank163243 +163244 POINT(40.50230404742788 73.1715068384148) bank163244 +163245 POINT(41.1163998204274 74.73115674069145) bank163245 +163246 POINT(40.44916592773327 74.24754932372295) bank163246 +163247 POINT(40.86572021665356 73.97373689488691) bank163247 +163248 POINT(40.11907048379385 73.79731020006014) bank163248 +163249 POINT(40.78684584198076 74.63108519930674) bank163249 +163250 POINT(39.93977364192132 73.36973401338179) bank163250 +163251 POINT(40.697111958088406 73.84338796783514) bank163251 +163252 POINT(40.643914249575715 73.70467600629517) bank163252 +163253 POINT(41.01711615654487 73.06863513641298) bank163253 +163254 POINT(40.56568760313221 73.93995322345509) bank163254 +163255 POINT(41.13800284567418 73.83632776514531) bank163255 +163256 POINT(41.64156758535175 74.11171094728067) bank163256 +163257 POINT(40.74128958140321 73.85934076113027) bank163257 +163258 POINT(40.89189187604674 73.00933704856027) bank163258 +163259 POINT(41.03959087272644 73.75295419136783) bank163259 +163260 POINT(40.242381733318005 73.99122073989398) bank163260 +163261 POINT(41.57428606528084 73.62320502832726) bank163261 +163262 POINT(39.73298713547978 74.59090165800626) bank163262 +163263 POINT(40.85199783776538 73.73136121077243) bank163263 +163264 POINT(41.31675672395675 73.05394560586818) bank163264 +163265 POINT(41.5068249032702 74.69355080035483) bank163265 +163266 POINT(40.43655751421807 73.00926530902413) bank163266 +163267 POINT(41.279562530883574 73.23091693765612) bank163267 +163268 POINT(39.875705399726506 73.37534046143976) bank163268 +163269 POINT(41.34024089234668 74.19677370931552) bank163269 +163270 POINT(39.77556780076703 74.07467143002006) bank163270 +163271 POINT(40.890068979371264 73.08054373455595) bank163271 +163272 POINT(40.89351338835109 74.09758732731379) bank163272 +163273 POINT(39.80204536086421 74.61740572266892) bank163273 +163274 POINT(39.812161519809855 73.63685827122528) bank163274 +163275 POINT(40.833316726147665 74.18204237394963) bank163275 +163276 POINT(40.41755180143416 74.86315980097308) bank163276 +163277 POINT(40.384738652503295 74.27283449064785) bank163277 +163278 POINT(40.986740565292386 74.69229664460056) bank163278 +163279 POINT(41.459804607072 73.42255593847098) bank163279 +163280 POINT(41.698985671912745 73.31034039311405) bank163280 +163281 POINT(41.172197764632486 74.74697150201126) bank163281 +163282 POINT(40.39181771894259 74.86081239273176) bank163282 +163283 POINT(40.16405097853011 74.41716432767365) bank163283 +163284 POINT(41.38959076405983 74.72420623328135) bank163284 +163285 POINT(39.86460906908493 74.90528902434625) bank163285 +163286 POINT(39.99794105160642 73.81600214948487) bank163286 +163287 POINT(40.490182125172105 73.90316189568948) bank163287 +163288 POINT(41.567563748860195 73.82911389512455) bank163288 +163289 POINT(41.14172682263122 73.91378855611588) bank163289 +163290 POINT(40.03226599660914 73.13312871186818) bank163290 +163291 POINT(40.69272573273814 74.7598519767222) bank163291 +163292 POINT(40.02446531503998 73.44710164083102) bank163292 +163293 POINT(40.01045474227438 74.56606535360457) bank163293 +163294 POINT(41.51997562480722 73.17652457325461) bank163294 +163295 POINT(41.17583020436607 74.56312637223667) bank163295 +163296 POINT(41.23067982333744 73.23415378510532) bank163296 +163297 POINT(39.80769981352948 74.87133262378342) bank163297 +163298 POINT(39.9584931663716 74.73927556458379) bank163298 +163299 POINT(41.10458542711656 74.13955233632102) bank163299 +163300 POINT(40.803048660013246 74.77267103540103) bank163300 +163301 POINT(41.05017974203912 73.75145217699605) bank163301 +163302 POINT(40.71850609606357 74.6412817971822) bank163302 +163303 POINT(40.71906948666325 74.49081096459489) bank163303 +163304 POINT(41.18163654459006 74.67827456793364) bank163304 +163305 POINT(40.66279372986016 73.2567531113535) bank163305 +163306 POINT(41.11396044651434 74.21612585245069) bank163306 +163307 POINT(40.82053405002326 73.5371863086386) bank163307 +163308 POINT(40.15453134179362 73.95375623695506) bank163308 +163309 POINT(39.85779125534514 73.97503340014204) bank163309 +163310 POINT(40.22039830569639 73.97061455474363) bank163310 +163311 POINT(40.60826618775668 73.8397032917683) bank163311 +163312 POINT(40.77853338731391 74.74777244391397) bank163312 +163313 POINT(40.45575770511665 73.9149551103403) bank163313 +163314 POINT(40.757444905470805 74.01386628901543) bank163314 +163315 POINT(41.41451245741225 73.57305667921474) bank163315 +163316 POINT(40.9049151057069 73.95123604765217) bank163316 +163317 POINT(41.53405039778471 73.06068714073501) bank163317 +163318 POINT(39.76602502607784 74.58947935950226) bank163318 +163319 POINT(40.99579306360105 73.35136870488735) bank163319 +163320 POINT(41.48940720054897 74.91817732618918) bank163320 +163321 POINT(40.37868324560104 74.4238880057111) bank163321 +163322 POINT(39.90949636832009 73.04587428036784) bank163322 +163323 POINT(41.16531082158375 74.25208743182009) bank163323 +163324 POINT(40.20677921930781 74.10866021033654) bank163324 +163325 POINT(39.77900402087418 73.0711280949185) bank163325 +163326 POINT(40.35347606335397 74.44782458435873) bank163326 +163327 POINT(41.51562937538328 74.04481966027352) bank163327 +163328 POINT(40.736931097189846 74.07796043645901) bank163328 +163329 POINT(40.79529354696954 73.93440170975252) bank163329 +163330 POINT(39.89571082230583 74.97597855926908) bank163330 +163331 POINT(41.332438056046435 73.53319411268181) bank163331 +163332 POINT(41.672028732694415 73.28575740100196) bank163332 +163333 POINT(41.621926108486356 74.22937986444929) bank163333 +163334 POINT(41.064293886101545 73.03575206312283) bank163334 +163335 POINT(41.28528153132198 73.71729931728518) bank163335 +163336 POINT(40.017305056864046 74.36565333497153) bank163336 +163337 POINT(40.31626078368197 73.4892058046018) bank163337 +163338 POINT(39.75920359245049 73.59042785332016) bank163338 +163339 POINT(40.90964865060095 73.93424533987958) bank163339 +163340 POINT(40.36059842003807 73.53975018063723) bank163340 +163341 POINT(39.72495836186791 74.72903864255456) bank163341 +163342 POINT(41.02910672368567 74.7238497397244) bank163342 +163343 POINT(40.95002582338475 73.2127592556224) bank163343 +163344 POINT(40.243801534684685 73.69862619386278) bank163344 +163345 POINT(39.858889320391505 74.73423677898103) bank163345 +163346 POINT(41.272921028446504 73.93866228686818) bank163346 +163347 POINT(40.71409988890989 74.61571359472596) bank163347 +163348 POINT(39.79026913047778 73.34080613430297) bank163348 +163349 POINT(40.57185415543851 74.95530575564734) bank163349 +163350 POINT(40.12202253798137 74.6454669546908) bank163350 +163351 POINT(40.34891426698489 74.82554213457375) bank163351 +163352 POINT(40.872401537034214 73.84750248749737) bank163352 +163353 POINT(40.34309587516159 73.15992361101102) bank163353 +163354 POINT(40.129389420431195 73.52445513937954) bank163354 +163355 POINT(39.948017330054746 73.85460124624288) bank163355 +163356 POINT(40.935812963648964 74.61339946605399) bank163356 +163357 POINT(40.50478956710586 73.0897119152644) bank163357 +163358 POINT(40.27391344645485 74.68287101266114) bank163358 +163359 POINT(41.0918930939596 74.19717014621416) bank163359 +163360 POINT(40.54440823939677 74.20777378916885) bank163360 +163361 POINT(40.05491139810526 74.20472399051432) bank163361 +163362 POINT(40.39507311226661 74.97143360599904) bank163362 +163363 POINT(40.49583039064483 74.95123018137974) bank163363 +163364 POINT(40.653993779869104 73.73112181215852) bank163364 +163365 POINT(39.96931708019113 73.45911966954002) bank163365 +163366 POINT(40.347505248322285 73.10866713481018) bank163366 +163367 POINT(39.98816683628986 73.43748997560907) bank163367 +163368 POINT(39.84342359809219 73.13057743648264) bank163368 +163369 POINT(40.866706654157284 74.04722808662447) bank163369 +163370 POINT(41.39593807885225 73.91599473533917) bank163370 +163371 POINT(40.07147519883942 73.03965178928789) bank163371 +163372 POINT(40.94168278078383 73.17843858375223) bank163372 +163373 POINT(41.01178568822222 73.42993314744331) bank163373 +163374 POINT(40.39316401236838 73.17917279855286) bank163374 +163375 POINT(40.64512965937352 74.81550698801077) bank163375 +163376 POINT(40.879099372930895 73.30669720295514) bank163376 +163377 POINT(39.9814539928711 73.10372364131956) bank163377 +163378 POINT(39.99419746957381 74.27084740796165) bank163378 +163379 POINT(40.56002977773994 74.55104595534048) bank163379 +163380 POINT(40.611985593679506 74.91613871137162) bank163380 +163381 POINT(40.70564991756217 74.15996296561221) bank163381 +163382 POINT(40.60022139082392 73.82651710900466) bank163382 +163383 POINT(40.48959606628906 74.4722470995742) bank163383 +163384 POINT(40.46650498408279 73.09168981511039) bank163384 +163385 POINT(40.30424282986711 74.36819949814331) bank163385 +163386 POINT(41.18024940548549 73.48046685058172) bank163386 +163387 POINT(39.95297824079967 74.09294699389922) bank163387 +163388 POINT(40.59146452186505 74.32523021499324) bank163388 +163389 POINT(39.87806274097876 74.17274278535346) bank163389 +163390 POINT(39.78638933163379 73.41221206390492) bank163390 +163391 POINT(41.36826126760273 74.78146314987447) bank163391 +163392 POINT(40.11607673073164 74.90999896245198) bank163392 +163393 POINT(40.67820394986275 74.80788460994523) bank163393 +163394 POINT(40.05390561485087 74.79706689564026) bank163394 +163395 POINT(41.618336005292484 73.08695946419296) bank163395 +163396 POINT(41.11558780975128 74.36896127565514) bank163396 +163397 POINT(39.85282702178403 73.70400679225685) bank163397 +163398 POINT(40.88173685018385 73.40696753193285) bank163398 +163399 POINT(40.81534758830742 74.35290721159596) bank163399 +163400 POINT(41.645545254340796 74.48374113478555) bank163400 +163401 POINT(40.13468073536732 73.34022609709343) bank163401 +163402 POINT(40.41403526192716 74.50851340851587) bank163402 +163403 POINT(40.26275323672741 74.13269770123826) bank163403 +163404 POINT(40.27754170212802 73.61992028222119) bank163404 +163405 POINT(40.230707689475224 74.34842147885797) bank163405 +163406 POINT(40.647832353402656 74.9192147012991) bank163406 +163407 POINT(41.07384477086916 74.48106483590196) bank163407 +163408 POINT(39.94789555604249 74.37786396703379) bank163408 +163409 POINT(40.223658923143326 74.9601348007658) bank163409 +163410 POINT(41.38932535326425 74.26653218888328) bank163410 +163411 POINT(41.44976705285731 73.54146133280281) bank163411 +163412 POINT(41.27660344028438 74.26905352575622) bank163412 +163413 POINT(41.48245482742104 73.63968910820097) bank163413 +163414 POINT(41.19863393077387 74.40284344090931) bank163414 +163415 POINT(40.80983356828943 73.20360278658492) bank163415 +163416 POINT(39.78120693664251 73.31556394408875) bank163416 +163417 POINT(39.912865458605175 73.71880774618448) bank163417 +163418 POINT(39.98231876141898 74.00836156911359) bank163418 +163419 POINT(41.10221002217673 73.64177838487007) bank163419 +163420 POINT(40.41803030045404 74.89955347818591) bank163420 +163421 POINT(39.75337347078217 74.57292635162104) bank163421 +163422 POINT(41.44711891410701 73.31405492543256) bank163422 +163423 POINT(40.71516899471737 74.12561549627492) bank163423 +163424 POINT(39.875657983669974 73.1206518186456) bank163424 +163425 POINT(41.5683746682496 74.25610940754272) bank163425 +163426 POINT(40.705275556663885 73.19648408284236) bank163426 +163427 POINT(40.910798657202974 74.31206333422561) bank163427 +163428 POINT(39.96673207564727 73.64368128716629) bank163428 +163429 POINT(41.12252406768257 74.24503616496142) bank163429 +163430 POINT(40.53442318613117 74.49020767832214) bank163430 +163431 POINT(41.661173051853574 73.40085900919092) bank163431 +163432 POINT(41.33993201499592 74.09654779933929) bank163432 +163433 POINT(39.772339495464216 74.66251792634878) bank163433 +163434 POINT(41.611379881458475 73.69931537115284) bank163434 +163435 POINT(40.836467989111064 74.39260160968828) bank163435 +163436 POINT(41.60734006113801 74.93642725538562) bank163436 +163437 POINT(40.10537604494504 74.4987236378161) bank163437 +163438 POINT(40.19080210546426 74.89622003819989) bank163438 +163439 POINT(41.3863861589879 74.50174340467166) bank163439 +163440 POINT(40.61255521622254 73.4577950244259) bank163440 +163441 POINT(40.06679029217306 74.09270024589856) bank163441 +163442 POINT(40.120911721123754 73.9722759715921) bank163442 +163443 POINT(41.34328022451272 74.20545156549304) bank163443 +163444 POINT(40.719606519700505 74.70208042949655) bank163444 +163445 POINT(41.47627922361148 73.56492241951689) bank163445 +163446 POINT(41.0963560124856 74.83422452309233) bank163446 +163447 POINT(40.850364740723535 73.27127022018107) bank163447 +163448 POINT(39.99925938209101 74.0375720878693) bank163448 +163449 POINT(40.12753366309613 73.73073254675228) bank163449 +163450 POINT(41.06076356268904 74.15690866261022) bank163450 +163451 POINT(41.36356299246474 73.87014938563607) bank163451 +163452 POINT(40.71703905337365 73.56141702685228) bank163452 +163453 POINT(40.37924383929917 73.95212470773768) bank163453 +163454 POINT(41.0952579833182 74.65442955565622) bank163454 +163455 POINT(40.452320508083126 73.98085674757446) bank163455 +163456 POINT(40.965063613961405 73.97737981419367) bank163456 +163457 POINT(39.92535861749693 73.05977821049152) bank163457 +163458 POINT(40.75646993829671 74.18763452561824) bank163458 +163459 POINT(41.31056852417519 73.8893353944155) bank163459 +163460 POINT(39.79999721053517 73.99210837021499) bank163460 +163461 POINT(39.73343213842463 73.24024045221759) bank163461 +163462 POINT(39.73817118450132 73.95143537675904) bank163462 +163463 POINT(41.264652826780825 73.67430793165846) bank163463 +163464 POINT(41.568504962046205 74.74458169248459) bank163464 +163465 POINT(40.265443847564605 74.76672714401724) bank163465 +163466 POINT(40.93079122127356 74.4438916297362) bank163466 +163467 POINT(40.86863848088084 74.2328599780585) bank163467 +163468 POINT(40.612322627287156 74.97095704274086) bank163468 +163469 POINT(41.479937834899054 75.00254867058116) bank163469 +163470 POINT(40.14943440491246 74.53953140714576) bank163470 +163471 POINT(40.381726693593095 74.8420052213222) bank163471 +163472 POINT(41.42385157978999 74.5768732752847) bank163472 +163473 POINT(40.21627899285213 73.96966656226259) bank163473 +163474 POINT(40.69428834265827 74.12298766281417) bank163474 +163475 POINT(40.95735443170075 74.97266051116523) bank163475 +163476 POINT(41.071602059918966 74.47673351407087) bank163476 +163477 POINT(41.312932046254716 73.20937828209517) bank163477 +163478 POINT(39.75831390087015 73.8568060272149) bank163478 +163479 POINT(40.96209689615329 73.13762282596961) bank163479 +163480 POINT(41.599796187486135 74.00282193072464) bank163480 +163481 POINT(41.60867467005346 73.68683900317053) bank163481 +163482 POINT(39.761521922852076 74.29590198306254) bank163482 +163483 POINT(41.13183080436019 74.49384739837164) bank163483 +163484 POINT(41.02098639722969 74.6050467302412) bank163484 +163485 POINT(41.514045270617046 73.63700285129917) bank163485 +163486 POINT(40.39378729618245 73.70604549476236) bank163486 +163487 POINT(40.04789743523856 74.14094870747836) bank163487 +163488 POINT(40.55757337163264 74.85329763046661) bank163488 +163489 POINT(39.79838145760269 73.65318520570645) bank163489 +163490 POINT(40.103413482056446 74.90463953589233) bank163490 +163491 POINT(39.74730182231427 74.79787646223869) bank163491 +163492 POINT(40.95022577102917 73.1535532310689) bank163492 +163493 POINT(40.19084204206396 74.42100183947262) bank163493 +163494 POINT(40.647711632675005 73.26869044125682) bank163494 +163495 POINT(40.628656474372384 73.90417691078682) bank163495 +163496 POINT(41.081713030988716 74.03928098245211) bank163496 +163497 POINT(39.74073611128389 73.42627412436725) bank163497 +163498 POINT(40.05298645652317 73.59881334261647) bank163498 +163499 POINT(41.083048593122896 73.76293266677126) bank163499 +163500 POINT(39.81177662117833 74.33049116350871) bank163500 +163501 POINT(39.896621288592584 74.70139002356456) bank163501 +163502 POINT(39.762705497832265 74.08334917022529) bank163502 +163503 POINT(39.841812115226375 74.97662567943549) bank163503 +163504 POINT(41.26257508981109 73.19391948437672) bank163504 +163505 POINT(41.608316941938995 74.44214982951671) bank163505 +163506 POINT(40.052974625468885 74.4485523215258) bank163506 +163507 POINT(40.02254464330645 74.15885491347942) bank163507 +163508 POINT(41.16680946748633 73.78939815047586) bank163508 +163509 POINT(40.642440365674275 74.1784032206135) bank163509 +163510 POINT(40.498891071223376 73.34133025223127) bank163510 +163511 POINT(41.61958397921851 73.74847609063062) bank163511 +163512 POINT(39.97001336645168 74.38881949522424) bank163512 +163513 POINT(41.09623185017142 74.04928267990121) bank163513 +163514 POINT(40.18617288135103 74.39151812964413) bank163514 +163515 POINT(40.079543166031776 73.18298453408815) bank163515 +163516 POINT(41.52275816396142 74.46890738057989) bank163516 +163517 POINT(40.13520013195931 74.99347393443259) bank163517 +163518 POINT(39.88393066661595 74.13283774001468) bank163518 +163519 POINT(39.83026580316435 74.61300229412058) bank163519 +163520 POINT(40.310627828634495 73.08258631653499) bank163520 +163521 POINT(39.85433236290616 74.58723621482883) bank163521 +163522 POINT(40.490123831616046 74.63488680896538) bank163522 +163523 POINT(41.5911804800273 74.1440029813562) bank163523 +163524 POINT(41.116561095236925 73.72303926536289) bank163524 +163525 POINT(39.91713705111136 73.11460792740182) bank163525 +163526 POINT(39.75712821456283 74.58662346755465) bank163526 +163527 POINT(41.63016039813156 73.70445175848089) bank163527 +163528 POINT(40.25810682793528 74.20156635808655) bank163528 +163529 POINT(41.065956928695016 73.77567009499691) bank163529 +163530 POINT(41.43204149876024 74.55091342479045) bank163530 +163531 POINT(40.16188235540906 74.05242256715748) bank163531 +163532 POINT(40.220717603674004 73.15259554199258) bank163532 +163533 POINT(41.48289957992548 73.35975504252292) bank163533 +163534 POINT(40.61438607104233 74.67026868797139) bank163534 +163535 POINT(40.81932093373633 74.63059756918267) bank163535 +163536 POINT(40.56764618821714 74.5312545157542) bank163536 +163537 POINT(40.9929821563695 73.79142499663162) bank163537 +163538 POINT(40.015927159805216 74.8022723120215) bank163538 +163539 POINT(40.30036552312629 74.98684786900566) bank163539 +163540 POINT(40.85179986184082 74.09148758157832) bank163540 +163541 POINT(40.65069552521688 73.12082438293687) bank163541 +163542 POINT(41.34619697489992 73.1675782980104) bank163542 +163543 POINT(39.89099825991472 73.34724655693343) bank163543 +163544 POINT(39.821390852911485 73.43758841607043) bank163544 +163545 POINT(39.96292144809144 74.84655673633772) bank163545 +163546 POINT(40.21102489817506 73.6083021025389) bank163546 +163547 POINT(41.23198619972106 73.16486518785531) bank163547 +163548 POINT(40.6660424381789 74.14013354002576) bank163548 +163549 POINT(40.15230188746003 74.77764766142195) bank163549 +163550 POINT(40.29495390119663 74.20587652652628) bank163550 +163551 POINT(40.932532457711396 74.46575713580035) bank163551 +163552 POINT(40.50455417929252 73.39304505384708) bank163552 +163553 POINT(39.74706888274998 74.29472127431833) bank163553 +163554 POINT(40.841274152501065 74.71108198187822) bank163554 +163555 POINT(40.55343162780862 73.28846648161259) bank163555 +163556 POINT(40.13614561386484 74.36336252634187) bank163556 +163557 POINT(40.02753488465769 73.61076357357969) bank163557 +163558 POINT(41.398644156153324 74.41196147503072) bank163558 +163559 POINT(39.8281832348409 74.05104331409585) bank163559 +163560 POINT(40.379206521559254 74.0526194508918) bank163560 +163561 POINT(40.03913156099454 73.54518367615051) bank163561 +163562 POINT(40.10995827972321 73.72072432560526) bank163562 +163563 POINT(41.13392285881046 73.80497638672078) bank163563 +163564 POINT(40.511547407723555 73.76554859249788) bank163564 +163565 POINT(39.86921593335317 73.40127279520786) bank163565 +163566 POINT(40.813033988919884 73.3887849395614) bank163566 +163567 POINT(39.94154786841814 74.90068451469655) bank163567 +163568 POINT(40.434492895054895 73.88866638095463) bank163568 +163569 POINT(40.70193125806548 74.52472883439319) bank163569 +163570 POINT(41.508068923420865 73.20132233582366) bank163570 +163571 POINT(40.655699329105055 73.23078724462223) bank163571 +163572 POINT(40.379845771993324 73.63795057988848) bank163572 +163573 POINT(41.49866427877989 73.11180753902704) bank163573 +163574 POINT(40.81913578065841 73.066969618456) bank163574 +163575 POINT(40.657674224946696 74.57288677007796) bank163575 +163576 POINT(40.85983908048367 74.6000590422463) bank163576 +163577 POINT(40.81514842041682 73.24441880352174) bank163577 +163578 POINT(41.30867666213034 73.30929068520297) bank163578 +163579 POINT(41.12850659367397 74.35674458723017) bank163579 +163580 POINT(40.56033686333679 73.55836904960924) bank163580 +163581 POINT(40.12860964218005 73.7177071404952) bank163581 +163582 POINT(40.433910633759 74.01151501628637) bank163582 +163583 POINT(41.40370412188902 74.30720584218741) bank163583 +163584 POINT(41.192872651790985 73.35960528338832) bank163584 +163585 POINT(40.044889860922844 74.95758410952044) bank163585 +163586 POINT(39.79063176338591 73.13674224625673) bank163586 +163587 POINT(40.52141696733278 73.10765821075758) bank163587 +163588 POINT(41.32922327799903 73.84218751228079) bank163588 +163589 POINT(41.275579553704794 73.36711619393813) bank163589 +163590 POINT(41.40821192588888 74.83003027558613) bank163590 +163591 POINT(40.41278014717761 73.16543057446314) bank163591 +163592 POINT(40.1406049657578 73.35353176084118) bank163592 +163593 POINT(41.34034578154487 74.08960610035017) bank163593 +163594 POINT(41.27104141366923 74.0745162999722) bank163594 +163595 POINT(39.972546644185776 73.49824115716825) bank163595 +163596 POINT(40.58220554463237 73.81083002193581) bank163596 +163597 POINT(40.05976530231991 74.18394851024783) bank163597 +163598 POINT(41.2986851784274 74.50245787645831) bank163598 +163599 POINT(41.36961614297038 74.75145450628672) bank163599 +163600 POINT(41.223481678459365 73.89373876850333) bank163600 +163601 POINT(41.70809975844161 74.41923012766166) bank163601 +163602 POINT(39.93833868673866 74.50539720209366) bank163602 +163603 POINT(40.821865393657674 74.07613272603918) bank163603 +163604 POINT(41.45924737686418 74.61953379007566) bank163604 +163605 POINT(40.070498821669304 74.77490679786142) bank163605 +163606 POINT(39.825820989861214 73.67199595755919) bank163606 +163607 POINT(41.60633818762134 73.24311442138215) bank163607 +163608 POINT(40.512330331782664 74.20753227008822) bank163608 +163609 POINT(41.03383729353646 73.82423271303595) bank163609 +163610 POINT(40.43342913501843 73.5357454182352) bank163610 +163611 POINT(40.384914334081046 73.47812189418629) bank163611 +163612 POINT(40.39979157487521 73.36491497850787) bank163612 +163613 POINT(39.8566855260489 74.87617256864299) bank163613 +163614 POINT(41.43074973720341 74.91546470441564) bank163614 +163615 POINT(40.1981109229552 73.62835882700341) bank163615 +163616 POINT(40.4083391623569 74.83139685619845) bank163616 +163617 POINT(39.92697201855987 74.14121052523552) bank163617 +163618 POINT(40.74972983346714 73.41955052908838) bank163618 +163619 POINT(41.272131804000885 74.88332793363432) bank163619 +163620 POINT(40.08820979041878 73.67912402429512) bank163620 +163621 POINT(39.868092060318894 74.16721897154694) bank163621 +163622 POINT(41.390252550325215 73.15763586624027) bank163622 +163623 POINT(40.1937759184295 74.9455112742615) bank163623 +163624 POINT(40.21920421576556 74.42158804178466) bank163624 +163625 POINT(39.788584234733 74.54135794622451) bank163625 +163626 POINT(40.088070858945954 73.76366824926433) bank163626 +163627 POINT(40.94786264079481 73.65767734657142) bank163627 +163628 POINT(40.24150183614829 74.75510633308411) bank163628 +163629 POINT(40.307077153136724 73.30879018657073) bank163629 +163630 POINT(40.03164431781995 74.17709892098014) bank163630 +163631 POINT(41.13093916974029 74.08051947888895) bank163631 +163632 POINT(40.428422856276164 74.99178186798383) bank163632 +163633 POINT(41.05759720646259 74.81118255948664) bank163633 +163634 POINT(40.29015219469546 73.1500224271135) bank163634 +163635 POINT(40.181808748151376 73.22626513346263) bank163635 +163636 POINT(40.89323161503524 73.21488485903986) bank163636 +163637 POINT(40.773655386114605 73.74108153295026) bank163637 +163638 POINT(40.25389296845299 74.63060549448947) bank163638 +163639 POINT(39.8049571000599 73.99111309883851) bank163639 +163640 POINT(40.19025541010098 74.37393894603396) bank163640 +163641 POINT(40.63762733246037 74.66853748427684) bank163641 +163642 POINT(41.083698303054604 74.72168050383368) bank163642 +163643 POINT(41.01978207276886 73.59821492158245) bank163643 +163644 POINT(40.39459489692898 73.69139503643011) bank163644 +163645 POINT(40.29901741796393 74.88446842229364) bank163645 +163646 POINT(40.13570010443673 74.3616908199877) bank163646 +163647 POINT(40.318395518227774 74.7704130634174) bank163647 +163648 POINT(40.34257525790834 74.34790992327255) bank163648 +163649 POINT(41.17122688892904 73.89238138776176) bank163649 +163650 POINT(41.09837257660908 73.98374951709663) bank163650 +163651 POINT(39.83322685871815 73.86409379238603) bank163651 +163652 POINT(41.60268373625118 73.45390077505729) bank163652 +163653 POINT(41.2427019941713 74.36655780277344) bank163653 +163654 POINT(39.91294330410853 74.18147079760062) bank163654 +163655 POINT(40.3015231826289 73.72326027349777) bank163655 +163656 POINT(40.7188066810149 73.16059156008771) bank163656 +163657 POINT(40.10982739060816 73.41515995859555) bank163657 +163658 POINT(40.954574751222786 74.88513915597025) bank163658 +163659 POINT(39.92056355291109 73.17088350871428) bank163659 +163660 POINT(40.333322714170244 74.7631695223624) bank163660 +163661 POINT(41.31027165621218 73.99288810284827) bank163661 +163662 POINT(41.01381213556074 74.17256886182163) bank163662 +163663 POINT(40.60513586566173 73.1344905816746) bank163663 +163664 POINT(41.44625349312895 73.23957923619606) bank163664 +163665 POINT(40.40900807257349 73.83087764643514) bank163665 +163666 POINT(40.91838009699634 73.37080124323401) bank163666 +163667 POINT(40.01366093497049 73.52835473722811) bank163667 +163668 POINT(41.100401375334805 74.06134382474166) bank163668 +163669 POINT(41.47986651510698 73.72486638968597) bank163669 +163670 POINT(41.41075713049497 73.8010180971552) bank163670 +163671 POINT(41.329295559847125 74.66200437282231) bank163671 +163672 POINT(40.62581868897973 74.84248770155517) bank163672 +163673 POINT(41.691820359934034 74.77000368055138) bank163673 +163674 POINT(41.20282498205149 73.664683246766) bank163674 +163675 POINT(41.12174354624482 73.29816331721773) bank163675 +163676 POINT(41.678271695759264 74.72619755957851) bank163676 +163677 POINT(41.13929841676465 73.18446200023584) bank163677 +163678 POINT(40.88537916693793 74.85309271167934) bank163678 +163679 POINT(41.44892205275891 73.05025277748922) bank163679 +163680 POINT(41.18064205970069 73.14422436658899) bank163680 +163681 POINT(41.20874524451471 74.20651897149061) bank163681 +163682 POINT(40.56992614893267 73.75140234527886) bank163682 +163683 POINT(41.0614267659741 73.4442845590647) bank163683 +163684 POINT(40.216556147248305 74.69465574305498) bank163684 +163685 POINT(40.63997353382934 73.65386800233603) bank163685 +163686 POINT(41.583566051907596 73.77091226354788) bank163686 +163687 POINT(40.23431962402871 73.36796312391226) bank163687 +163688 POINT(41.54228699079017 73.25137641787045) bank163688 +163689 POINT(40.95036096835916 74.34453404696329) bank163689 +163690 POINT(41.51846577702266 74.4040595100118) bank163690 +163691 POINT(40.33971744001419 73.47439042416117) bank163691 +163692 POINT(40.52612378949391 74.69000984928827) bank163692 +163693 POINT(39.91891069213269 74.56818646068011) bank163693 +163694 POINT(41.565144143797774 74.41436026212992) bank163694 +163695 POINT(40.237802270801346 74.89877436534981) bank163695 +163696 POINT(39.93005167503837 74.8449840852393) bank163696 +163697 POINT(40.357213818365295 73.20242345144712) bank163697 +163698 POINT(39.96996073224236 73.6795836468972) bank163698 +163699 POINT(41.13831785698285 73.22205332250496) bank163699 +163700 POINT(41.66680902230972 73.8209414325955) bank163700 +163701 POINT(41.224174719579736 74.79082817903341) bank163701 +163702 POINT(40.18989924638274 73.38377935533964) bank163702 +163703 POINT(41.28646069938559 74.00561233611079) bank163703 +163704 POINT(40.52288631294668 73.18336805626237) bank163704 +163705 POINT(40.40266310112262 73.5885146982173) bank163705 +163706 POINT(40.43493938849116 73.78940815803804) bank163706 +163707 POINT(40.34579590434841 74.11128897645507) bank163707 +163708 POINT(41.65295616454449 74.78994540217036) bank163708 +163709 POINT(40.52653862240968 73.5737442934279) bank163709 +163710 POINT(40.889554318728614 73.98667784718701) bank163710 +163711 POINT(41.52829345885208 73.72914358658306) bank163711 +163712 POINT(41.41858941093565 75.00390536631892) bank163712 +163713 POINT(40.639009684590874 74.10286586722503) bank163713 +163714 POINT(41.14953301689584 74.88050691597618) bank163714 +163715 POINT(40.06130923642714 74.66940783625796) bank163715 +163716 POINT(39.95516922164443 75.00009928142259) bank163716 +163717 POINT(40.857866101119406 73.33371049374259) bank163717 +163718 POINT(39.96501013482193 73.98808556995512) bank163718 +163719 POINT(40.74304836833842 73.98907982849298) bank163719 +163720 POINT(41.666725354421644 73.62562606859954) bank163720 +163721 POINT(40.99764038362374 74.3028491845148) bank163721 +163722 POINT(40.67818208415473 73.54037299757798) bank163722 +163723 POINT(41.51868011808418 74.99097045791173) bank163723 +163724 POINT(40.68760943187152 73.769427866742) bank163724 +163725 POINT(40.40172700964885 73.12460456459061) bank163725 +163726 POINT(40.7515876384027 73.17710365083927) bank163726 +163727 POINT(40.22228719245535 73.69665929055563) bank163727 +163728 POINT(41.32651356927103 73.90754429436451) bank163728 +163729 POINT(40.96369432711945 74.93101016875723) bank163729 +163730 POINT(41.50849353000024 73.82307452658915) bank163730 +163731 POINT(40.837848705746296 73.0950053041874) bank163731 +163732 POINT(40.22932710400999 74.15311939661453) bank163732 +163733 POINT(39.73867660626519 73.87783205886673) bank163733 +163734 POINT(40.60024023445915 74.56406182983321) bank163734 +163735 POINT(40.58819180125087 74.11657233527674) bank163735 +163736 POINT(41.187977350968815 73.87807319808547) bank163736 +163737 POINT(40.74218854832634 73.98212000633949) bank163737 +163738 POINT(41.294290832764 73.67226202114486) bank163738 +163739 POINT(40.228301571548066 73.91773602385508) bank163739 +163740 POINT(40.36782038899409 73.87235639272761) bank163740 +163741 POINT(40.261935527627465 73.7986925411802) bank163741 +163742 POINT(40.849915686822 73.30466808042077) bank163742 +163743 POINT(39.81769620051643 73.68566678654064) bank163743 +163744 POINT(40.87134257199102 74.46272337454577) bank163744 +163745 POINT(40.20449236416163 74.74277843643004) bank163745 +163746 POINT(41.01042000941016 74.93443150676917) bank163746 +163747 POINT(40.133882987401435 73.37175784920647) bank163747 +163748 POINT(41.14318825765008 73.95692072282775) bank163748 +163749 POINT(40.61377868199116 74.40152106261394) bank163749 +163750 POINT(40.70915548694961 74.66041247391144) bank163750 +163751 POINT(41.15257768000598 74.85791912048447) bank163751 +163752 POINT(41.62382726729329 73.91243063698789) bank163752 +163753 POINT(40.93610336234855 73.64911259521915) bank163753 +163754 POINT(41.235000517025746 74.1370401637661) bank163754 +163755 POINT(40.72361053442824 74.47038916339042) bank163755 +163756 POINT(41.28885431451843 73.02147207997663) bank163756 +163757 POINT(41.14702342148294 73.68665260621198) bank163757 +163758 POINT(41.134948503491586 74.47328161719415) bank163758 +163759 POINT(40.67165676036564 74.78799330114354) bank163759 +163760 POINT(41.08683523156699 73.31180471324159) bank163760 +163761 POINT(41.080278301862464 73.69717379915691) bank163761 +163762 POINT(40.45727485413827 73.4315524170337) bank163762 +163763 POINT(40.22090226611959 73.41095932950194) bank163763 +163764 POINT(40.71367582847808 74.56512253589463) bank163764 +163765 POINT(41.131510169512936 74.80958524324184) bank163765 +163766 POINT(40.724841329985 73.12812604180333) bank163766 +163767 POINT(40.56518503837606 74.92446293353426) bank163767 +163768 POINT(40.36752211741679 74.89568601705764) bank163768 +163769 POINT(41.46441759304729 74.59459821876581) bank163769 +163770 POINT(41.34094958436544 73.07831562882238) bank163770 +163771 POINT(41.04339447769524 74.54826639151443) bank163771 +163772 POINT(40.63623555828828 73.41550447088963) bank163772 +163773 POINT(40.9852375254998 74.10971964433368) bank163773 +163774 POINT(41.390317639274635 74.72354273958467) bank163774 +163775 POINT(40.70390704196758 73.56256501188828) bank163775 +163776 POINT(41.244912575565436 74.9643401682418) bank163776 +163777 POINT(41.08975823096418 74.7851064247501) bank163777 +163778 POINT(40.61757106648809 73.06683379649465) bank163778 +163779 POINT(40.46246413152789 73.68073573665164) bank163779 +163780 POINT(40.93198797164052 73.3529071886702) bank163780 +163781 POINT(40.004224882899564 73.71476118375014) bank163781 +163782 POINT(40.01349875726931 73.82124950627862) bank163782 +163783 POINT(41.05242151775406 73.04352738284932) bank163783 +163784 POINT(39.879936106015045 74.444589686154) bank163784 +163785 POINT(41.00203025965048 73.44908703058796) bank163785 +163786 POINT(41.354860025902035 73.34671521581936) bank163786 +163787 POINT(40.379681540010964 74.45935107916046) bank163787 +163788 POINT(39.962025146934465 73.08142914915486) bank163788 +163789 POINT(40.933196056836 74.4171969034054) bank163789 +163790 POINT(40.60076278470178 74.19962329670642) bank163790 +163791 POINT(40.56374650259803 74.12673091663464) bank163791 +163792 POINT(40.644800622366525 74.13157007581668) bank163792 +163793 POINT(41.70295173508992 73.49859758395334) bank163793 +163794 POINT(39.954689861824285 73.10115326206456) bank163794 +163795 POINT(40.23077027363596 74.87148773475512) bank163795 +163796 POINT(40.97929043126177 73.23384069589856) bank163796 +163797 POINT(41.44393713578545 74.69049738247332) bank163797 +163798 POINT(40.38838112950434 73.22894639569124) bank163798 +163799 POINT(40.64765774288361 74.77731485640027) bank163799 +163800 POINT(40.91204949051376 73.98833641122062) bank163800 +163801 POINT(39.82505735843653 74.027668755051) bank163801 +163802 POINT(41.35841169816602 73.08132303463593) bank163802 +163803 POINT(41.689820488038166 73.98911983407149) bank163803 +163804 POINT(40.73676514638958 73.31283445822946) bank163804 +163805 POINT(40.61499216419434 74.10685488819092) bank163805 +163806 POINT(41.36204909301432 73.65472438727743) bank163806 +163807 POINT(41.066169941349145 74.76343768209921) bank163807 +163808 POINT(40.11351781765806 73.17973747705702) bank163808 +163809 POINT(41.37078378419117 73.71490688488645) bank163809 +163810 POINT(40.2014543031775 74.40464570999484) bank163810 +163811 POINT(41.525790899142 73.02376110708583) bank163811 +163812 POINT(40.36528906680279 73.93430468567824) bank163812 +163813 POINT(41.18577662624346 73.32138756958446) bank163813 +163814 POINT(40.25677316747286 74.33551852547183) bank163814 +163815 POINT(41.10752087637057 73.03047029060944) bank163815 +163816 POINT(40.0722325204037 73.53853402026063) bank163816 +163817 POINT(40.493787653670864 73.56679428871244) bank163817 +163818 POINT(41.70464039556101 74.8541815055765) bank163818 +163819 POINT(40.94064802208618 73.31015989760594) bank163819 +163820 POINT(40.990923599316176 73.27829359166809) bank163820 +163821 POINT(40.901156557752046 74.34140154905955) bank163821 +163822 POINT(40.25414945568278 73.40162352992982) bank163822 +163823 POINT(40.12286588856458 74.80984577186508) bank163823 +163824 POINT(41.184286681006604 74.32140816294472) bank163824 +163825 POINT(39.8213422507071 73.41871201838906) bank163825 +163826 POINT(40.60692044969339 73.17870908135595) bank163826 +163827 POINT(40.74197722310649 73.37060921080784) bank163827 +163828 POINT(39.94712189645693 73.92659037048038) bank163828 +163829 POINT(40.06175538401443 74.81724005260023) bank163829 +163830 POINT(40.74240035086458 74.04210316331081) bank163830 +163831 POINT(40.742526400839644 74.40780847644858) bank163831 +163832 POINT(39.95257728574573 74.27046673413903) bank163832 +163833 POINT(40.616158209828185 73.32251361131209) bank163833 +163834 POINT(40.51646407304779 73.06279780655923) bank163834 +163835 POINT(40.15362150293012 73.10096006406411) bank163835 +163836 POINT(40.24258319323052 73.11533715242466) bank163836 +163837 POINT(41.59681691606932 73.83236738964435) bank163837 +163838 POINT(41.527302057340975 73.8497282610258) bank163838 +163839 POINT(40.71089427093065 73.28696688866206) bank163839 +163840 POINT(41.346630196018566 74.35334703313386) bank163840 +163841 POINT(40.95856143452367 73.20175794023835) bank163841 +163842 POINT(40.96988323333793 73.43796487350201) bank163842 +163843 POINT(41.07957936572863 74.6894737533717) bank163843 +163844 POINT(41.497329595425484 73.32163804088083) bank163844 +163845 POINT(41.03770539867546 73.71271132824064) bank163845 +163846 POINT(40.96760380293715 74.29823734353862) bank163846 +163847 POINT(40.624154522641405 73.25562966575247) bank163847 +163848 POINT(41.61253435287531 73.41382186265767) bank163848 +163849 POINT(41.52343230104553 73.4345312450227) bank163849 +163850 POINT(39.949837992797775 74.82503934189842) bank163850 +163851 POINT(40.65076866551138 74.40988311050694) bank163851 +163852 POINT(41.163695456600024 73.92897664510004) bank163852 +163853 POINT(40.17505445581594 74.10171420383358) bank163853 +163854 POINT(41.623889397110446 74.65950003583724) bank163854 +163855 POINT(40.2016488514551 74.00494469940256) bank163855 +163856 POINT(41.1311717187608 73.48703483916648) bank163856 +163857 POINT(40.91869080810633 73.74037791180932) bank163857 +163858 POINT(39.99442615497619 74.2170244263182) bank163858 +163859 POINT(41.13909921363327 74.92009487643391) bank163859 +163860 POINT(40.573478009223436 74.32474098263064) bank163860 +163861 POINT(39.72738669566618 74.83419106204852) bank163861 +163862 POINT(41.47258958200849 73.12826305341741) bank163862 +163863 POINT(40.798841162130806 74.15949359769171) bank163863 +163864 POINT(40.37270563736292 73.1764304768896) bank163864 +163865 POINT(40.15691950066656 74.4682191498651) bank163865 +163866 POINT(40.76405645318419 74.97975102418829) bank163866 +163867 POINT(41.03460087499147 74.33873846741525) bank163867 +163868 POINT(40.41308093493603 73.29395895537864) bank163868 +163869 POINT(40.36165081033695 73.37481734551487) bank163869 +163870 POINT(41.278219337238525 73.6534293362849) bank163870 +163871 POINT(40.160446755198215 73.9979405254399) bank163871 +163872 POINT(40.25769131457702 74.38755168547569) bank163872 +163873 POINT(40.33523739978184 74.76887724406525) bank163873 +163874 POINT(40.03561743869182 74.73331131641824) bank163874 +163875 POINT(40.90560631469334 74.67301069187147) bank163875 +163876 POINT(41.28669384590213 74.10131680063988) bank163876 +163877 POINT(41.36814309866103 74.39450911620226) bank163877 +163878 POINT(39.767905628851985 74.21622030032492) bank163878 +163879 POINT(39.96079570176221 74.70808241077077) bank163879 +163880 POINT(39.7894603534801 74.65606190045465) bank163880 +163881 POINT(39.90913240357092 74.653013524732) bank163881 +163882 POINT(40.469852926014966 73.89303473569194) bank163882 +163883 POINT(41.12700352714288 73.19415440288078) bank163883 +163884 POINT(41.066678836858074 74.21230387326966) bank163884 +163885 POINT(39.75386963980908 73.28713531497439) bank163885 +163886 POINT(41.45846327733057 73.5589847230987) bank163886 +163887 POINT(41.067598526975246 74.39258098353253) bank163887 +163888 POINT(41.48528169142558 74.89703704877995) bank163888 +163889 POINT(40.67088111047878 73.03496358832935) bank163889 +163890 POINT(41.402817175904765 74.95348801155086) bank163890 +163891 POINT(40.781811560935566 73.74172551752095) bank163891 +163892 POINT(40.532966766911585 73.80144742975078) bank163892 +163893 POINT(40.38138593743714 73.83331572020782) bank163893 +163894 POINT(40.47829786552347 74.07799487987789) bank163894 +163895 POINT(41.36319036974715 73.48041276510449) bank163895 +163896 POINT(41.04502344346715 73.67181449240493) bank163896 +163897 POINT(41.06442743014026 73.07334211206431) bank163897 +163898 POINT(41.17507232332312 73.9414384624529) bank163898 +163899 POINT(40.97985915757827 73.79737715249729) bank163899 +163900 POINT(39.89206366545874 74.41189049803407) bank163900 +163901 POINT(39.87524755174631 74.68460853772447) bank163901 +163902 POINT(40.204692082450975 73.20711597698676) bank163902 +163903 POINT(41.49736402608763 74.48879538320946) bank163903 +163904 POINT(40.00787200274669 73.61639147219547) bank163904 +163905 POINT(40.30046965891757 74.91617575822467) bank163905 +163906 POINT(41.125419109820655 73.47714197542787) bank163906 +163907 POINT(41.16476924325507 74.5469695434945) bank163907 +163908 POINT(40.20577614593577 73.82944143993309) bank163908 +163909 POINT(39.86975811522152 74.65242345379544) bank163909 +163910 POINT(40.40087483595281 73.7157293434861) bank163910 +163911 POINT(41.38833789699861 74.82287295701292) bank163911 +163912 POINT(40.2640656215791 73.3761328795469) bank163912 +163913 POINT(40.32376634459306 73.60015261308405) bank163913 +163914 POINT(39.99166132997648 73.01407225384712) bank163914 +163915 POINT(40.90606971480613 73.78574584346327) bank163915 +163916 POINT(40.530304791658 74.12024565840268) bank163916 +163917 POINT(41.11290221395567 74.9926574605738) bank163917 +163918 POINT(40.58565794499464 73.89387395463) bank163918 +163919 POINT(41.59266722513026 74.88142237957291) bank163919 +163920 POINT(41.14483453239042 73.4207841086758) bank163920 +163921 POINT(40.36048672511984 74.89997930729815) bank163921 +163922 POINT(40.63366429862685 74.87552913520621) bank163922 +163923 POINT(40.52008658852288 74.7936138827736) bank163923 +163924 POINT(40.8095541574328 74.10734794127335) bank163924 +163925 POINT(40.052140552757216 73.11011556063856) bank163925 +163926 POINT(41.20985034988752 74.9087957967904) bank163926 +163927 POINT(40.36541081036394 74.31471844882735) bank163927 +163928 POINT(41.69495658826716 73.63083902869487) bank163928 +163929 POINT(40.714999744520156 74.89248539743924) bank163929 +163930 POINT(40.35234206506928 74.89738231069474) bank163930 +163931 POINT(41.321310169531515 74.8723325457937) bank163931 +163932 POINT(41.290485545196745 74.45425463168249) bank163932 +163933 POINT(40.46595249255383 74.63855090433563) bank163933 +163934 POINT(41.5885037278596 74.2105433549234) bank163934 +163935 POINT(40.87137901323138 73.53941923037497) bank163935 +163936 POINT(41.06368382902564 74.99657440980654) bank163936 +163937 POINT(41.452013670360394 73.7712581218571) bank163937 +163938 POINT(40.6167814232411 74.79168730161194) bank163938 +163939 POINT(40.94355873189212 74.87242683860713) bank163939 +163940 POINT(40.99972178272929 73.5331513624083) bank163940 +163941 POINT(41.53095480907317 74.23302953138509) bank163941 +163942 POINT(40.59513325058366 74.67810578375077) bank163942 +163943 POINT(41.56367036591659 74.45793003927558) bank163943 +163944 POINT(41.674533009590064 74.33825372259176) bank163944 +163945 POINT(40.68552568852 73.88624734437829) bank163945 +163946 POINT(41.439530468131146 74.02124447482075) bank163946 +163947 POINT(40.5516874133139 73.72690750121075) bank163947 +163948 POINT(40.7589394028457 74.25218982297923) bank163948 +163949 POINT(41.032571016349976 74.6113997478032) bank163949 +163950 POINT(40.79264621414483 73.09363537337019) bank163950 +163951 POINT(40.54769055312971 74.87007851136374) bank163951 +163952 POINT(41.556726523715746 74.71468161102817) bank163952 +163953 POINT(40.342613806134025 74.73205783745901) bank163953 +163954 POINT(40.79562715261873 73.95536135370834) bank163954 +163955 POINT(40.90553750140041 74.28853549967596) bank163955 +163956 POINT(41.38490196871277 74.6714637796209) bank163956 +163957 POINT(40.74635320268449 73.6462021858321) bank163957 +163958 POINT(40.10158960862755 74.42336953152873) bank163958 +163959 POINT(41.207426865771396 74.29000876147516) bank163959 +163960 POINT(39.80180517728329 74.48386276632148) bank163960 +163961 POINT(40.73525499362192 74.47422252577874) bank163961 +163962 POINT(40.50289235728088 74.32518619892737) bank163962 +163963 POINT(41.014015814032504 74.91208513295645) bank163963 +163964 POINT(40.40352014514879 74.8804845960171) bank163964 +163965 POINT(39.80542682578618 73.38232027065867) bank163965 +163966 POINT(39.86647699557609 74.41106863452228) bank163966 +163967 POINT(40.96927211272789 74.51856948445602) bank163967 +163968 POINT(41.07879243056308 73.54679299958514) bank163968 +163969 POINT(41.70637034297034 73.68570525217984) bank163969 +163970 POINT(40.65339678505971 73.2372909703993) bank163970 +163971 POINT(40.3094949015953 74.45997862337144) bank163971 +163972 POINT(40.84187040081914 74.68459019137741) bank163972 +163973 POINT(41.36696241670495 73.93471117430481) bank163973 +163974 POINT(40.97045466930536 74.39389458494985) bank163974 +163975 POINT(41.59647251608407 73.5692029106755) bank163975 +163976 POINT(40.15428728416721 73.68235674553725) bank163976 +163977 POINT(40.00637805777447 74.95826930634168) bank163977 +163978 POINT(39.821184125170205 74.93469305446595) bank163978 +163979 POINT(40.19535566969322 74.14549255024833) bank163979 +163980 POINT(41.58527224649569 73.77512595386298) bank163980 +163981 POINT(40.932170353947804 74.70159683183063) bank163981 +163982 POINT(40.94991022702797 73.28054395821506) bank163982 +163983 POINT(40.172357877977205 74.43758275115172) bank163983 +163984 POINT(41.36045460735319 74.69092849227891) bank163984 +163985 POINT(40.25530435848015 73.34468130023036) bank163985 +163986 POINT(40.82320841127127 73.15180152501068) bank163986 +163987 POINT(41.02125042000706 74.25670557770738) bank163987 +163988 POINT(40.349799371198785 74.17715833408785) bank163988 +163989 POINT(40.208736166581815 74.50497207762794) bank163989 +163990 POINT(40.6439606235582 74.16234344467058) bank163990 +163991 POINT(41.16816727938159 74.24536820272483) bank163991 +163992 POINT(40.89049580678946 73.4787809109754) bank163992 +163993 POINT(41.59699340637486 74.46805702809714) bank163993 +163994 POINT(41.546180166805655 74.92603487665686) bank163994 +163995 POINT(40.725743683974464 74.93292962330133) bank163995 +163996 POINT(40.237441593393804 73.21028761875637) bank163996 +163997 POINT(41.57562541335931 73.41838492596798) bank163997 +163998 POINT(40.429420261420944 74.98139297071073) bank163998 +163999 POINT(40.65863278209378 73.64842360198233) bank163999 +164000 POINT(40.41472590972855 73.52313986953185) bank164000 +164001 POINT(39.812207073787164 73.53298736486998) bank164001 +164002 POINT(41.309796601928994 73.91317478214079) bank164002 +164003 POINT(41.41349047275014 74.63956814931754) bank164003 +164004 POINT(41.087264004186515 73.70521325387992) bank164004 +164005 POINT(40.205276240482945 74.5573825746346) bank164005 +164006 POINT(40.59080309525618 74.42455147796292) bank164006 +164007 POINT(40.80138809579441 73.43729639017059) bank164007 +164008 POINT(39.82550527117607 74.18227519445774) bank164008 +164009 POINT(40.58985427679618 74.47919592896584) bank164009 +164010 POINT(40.1741125838546 74.94294892242328) bank164010 +164011 POINT(41.05135169051327 73.22924950093713) bank164011 +164012 POINT(40.73721503144909 74.07508588923164) bank164012 +164013 POINT(39.99891826175609 73.07416615229998) bank164013 +164014 POINT(39.94593382599559 73.32097839220991) bank164014 +164015 POINT(41.34999137602654 74.8926765987075) bank164015 +164016 POINT(41.28700251210517 73.16719183692817) bank164016 +164017 POINT(40.10784115829428 74.59770662118383) bank164017 +164018 POINT(39.908063821001846 73.17815865295897) bank164018 +164019 POINT(40.84309282587892 73.5611402255017) bank164019 +164020 POINT(39.79046386114198 74.61069273852547) bank164020 +164021 POINT(40.038022210399156 74.35628402129447) bank164021 +164022 POINT(40.024389073444006 74.58694890033442) bank164022 +164023 POINT(41.342453161367075 73.26677524384424) bank164023 +164024 POINT(39.911414760135656 74.5130125642963) bank164024 +164025 POINT(41.12271547248686 74.79405876269554) bank164025 +164026 POINT(41.27182254961373 73.22596614680435) bank164026 +164027 POINT(41.22744050109883 74.1049203962511) bank164027 +164028 POINT(40.96847888079468 73.76302049794096) bank164028 +164029 POINT(40.207954395870054 74.62507967542048) bank164029 +164030 POINT(40.352150987386395 73.31239343436427) bank164030 +164031 POINT(40.86939741242281 73.48683677561056) bank164031 +164032 POINT(40.186340729090105 74.65153996123287) bank164032 +164033 POINT(40.65272244692624 74.47295301531786) bank164033 +164034 POINT(41.20997239388258 73.4356483396094) bank164034 +164035 POINT(41.39694493839812 74.82592139795722) bank164035 +164036 POINT(39.832842937717565 73.85474815829063) bank164036 +164037 POINT(39.8272928194888 74.65792612304077) bank164037 +164038 POINT(41.21419984993432 73.57657949519209) bank164038 +164039 POINT(39.98167577730406 74.4328360342188) bank164039 +164040 POINT(40.9062330322501 73.02333720143362) bank164040 +164041 POINT(41.66403951891109 73.79653161575177) bank164041 +164042 POINT(40.07823607938838 74.48807753992229) bank164042 +164043 POINT(40.337115853309506 73.16610221193554) bank164043 +164044 POINT(39.91427841542438 73.88264495436566) bank164044 +164045 POINT(39.806095005110585 74.93936404341439) bank164045 +164046 POINT(41.13269152182175 74.21223272116902) bank164046 +164047 POINT(41.113188062306854 74.08383949564524) bank164047 +164048 POINT(40.139741840964355 74.16528939052635) bank164048 +164049 POINT(41.004374785732324 73.60573626911473) bank164049 +164050 POINT(40.927117071024895 74.86233782000933) bank164050 +164051 POINT(40.081857343843005 74.12146059171674) bank164051 +164052 POINT(41.13272378888867 74.89341322703454) bank164052 +164053 POINT(41.09008850549102 74.94028248953853) bank164053 +164054 POINT(41.62321424530579 74.70689789621703) bank164054 +164055 POINT(40.54778784582203 74.53227527331822) bank164055 +164056 POINT(41.197286222949465 74.43508287009173) bank164056 +164057 POINT(39.995316556326955 74.41763004392426) bank164057 +164058 POINT(40.69418781006761 73.22823721858641) bank164058 +164059 POINT(41.56469107138618 74.49714880297422) bank164059 +164060 POINT(40.31040579395808 74.27950914170871) bank164060 +164061 POINT(40.90986072961834 74.76361648371689) bank164061 +164062 POINT(40.03427637157587 74.89968308582388) bank164062 +164063 POINT(41.04595575834555 73.40509846278778) bank164063 +164064 POINT(39.98397323204093 74.39208824206338) bank164064 +164065 POINT(40.26691869693715 74.29877374887828) bank164065 +164066 POINT(40.49538095437224 74.40927611307453) bank164066 +164067 POINT(41.51009858429092 74.18207382593967) bank164067 +164068 POINT(40.438426674728376 73.91340415797863) bank164068 +164069 POINT(40.4865901089632 74.07952012634564) bank164069 +164070 POINT(39.95425235324866 73.049783304891) bank164070 +164071 POINT(40.131018044971285 73.91106294906116) bank164071 +164072 POINT(40.56350222684182 74.32722356195967) bank164072 +164073 POINT(40.770375885579796 74.99435137811392) bank164073 +164074 POINT(41.613203104576954 73.95936000871123) bank164074 +164075 POINT(40.70584915083741 73.71969335834098) bank164075 +164076 POINT(40.578980175054774 73.02337529438869) bank164076 +164077 POINT(41.16653356519695 74.2472963095902) bank164077 +164078 POINT(39.954445989191385 74.14560180637261) bank164078 +164079 POINT(41.56934401414029 73.43978529914574) bank164079 +164080 POINT(40.99239702144332 74.895976704286) bank164080 +164081 POINT(40.36481735728044 73.50002431166703) bank164081 +164082 POINT(41.34915794655821 73.40153939688552) bank164082 +164083 POINT(41.12788597134601 74.6537698066004) bank164083 +164084 POINT(41.43762902927416 73.16643772371769) bank164084 +164085 POINT(41.45486686119465 73.95313530688706) bank164085 +164086 POINT(40.92614291088566 74.26800627694544) bank164086 +164087 POINT(41.59779510825421 74.05242744127932) bank164087 +164088 POINT(40.52021173967184 74.98330791316303) bank164088 +164089 POINT(39.71954256627896 74.95121184840725) bank164089 +164090 POINT(40.713737499906266 74.40619693988315) bank164090 +164091 POINT(40.52760214734104 73.52371673247058) bank164091 +164092 POINT(40.19970262826657 73.19503184812717) bank164092 +164093 POINT(40.35105642445276 73.38229093431312) bank164093 +164094 POINT(39.96804108927347 74.77942978884796) bank164094 +164095 POINT(40.03421922355187 73.52977263097293) bank164095 +164096 POINT(41.634447359924984 73.67880942997206) bank164096 +164097 POINT(39.9626072415825 73.18767858919138) bank164097 +164098 POINT(40.40011373100354 74.81985492623691) bank164098 +164099 POINT(41.55694889430711 73.4835419200657) bank164099 +164100 POINT(39.813986204120376 73.23656320315017) bank164100 +164101 POINT(40.29838633457218 74.63295304831712) bank164101 +164102 POINT(39.90780944193554 73.34309506239259) bank164102 +164103 POINT(40.07598904561354 73.60414959713098) bank164103 +164104 POINT(41.48543355967919 74.73517765947217) bank164104 +164105 POINT(41.557191650520146 74.02831459761009) bank164105 +164106 POINT(41.556019559534235 73.61106372633473) bank164106 +164107 POINT(40.07392536697141 74.14388357766612) bank164107 +164108 POINT(41.14868544806602 74.36534543327274) bank164108 +164109 POINT(39.7600375575216 74.84341868678862) bank164109 +164110 POINT(41.57928788014689 73.345886316922) bank164110 +164111 POINT(40.59530070934172 73.17303663175129) bank164111 +164112 POINT(41.57636219185191 74.9733537267435) bank164112 +164113 POINT(40.85142159221179 74.49905983625867) bank164113 +164114 POINT(40.42888662392844 74.99555416717799) bank164114 +164115 POINT(40.58167109372853 73.78514535894843) bank164115 +164116 POINT(41.3335499293474 74.4508140088797) bank164116 +164117 POINT(41.09714334566284 74.74243009612024) bank164117 +164118 POINT(40.03979791957491 73.21870676530553) bank164118 +164119 POINT(41.067272125453556 73.7653705352635) bank164119 +164120 POINT(40.02648779889194 73.02758516075991) bank164120 +164121 POINT(41.26333783375401 73.02578978364565) bank164121 +164122 POINT(41.33977861974167 74.33615828333929) bank164122 +164123 POINT(40.494355094036415 74.66081633187815) bank164123 +164124 POINT(41.488145934711916 73.0350145949886) bank164124 +164125 POINT(41.16895966598006 74.72040074766443) bank164125 +164126 POINT(39.7390697136392 73.53144655803263) bank164126 +164127 POINT(41.03179811628061 73.44050638603797) bank164127 +164128 POINT(41.01881031199967 73.63934070019003) bank164128 +164129 POINT(41.63330549476037 74.01366020124823) bank164129 +164130 POINT(39.893326004493765 73.95568146768444) bank164130 +164131 POINT(40.88728943967591 74.697709005552) bank164131 +164132 POINT(40.691732434909724 74.71166241349935) bank164132 +164133 POINT(40.27316654776042 74.58478094884745) bank164133 +164134 POINT(41.00005081573345 73.22937263920458) bank164134 +164135 POINT(41.49593062139637 74.58909903257447) bank164135 +164136 POINT(40.41596945317104 73.92781208102376) bank164136 +164137 POINT(40.760584077669826 74.84478870801233) bank164137 +164138 POINT(41.04050001005873 73.89168773136295) bank164138 +164139 POINT(40.05536135993578 73.20224287760365) bank164139 +164140 POINT(40.76948930475205 73.56484188926034) bank164140 +164141 POINT(40.34799159361315 73.73180031055563) bank164141 +164142 POINT(41.39462951426273 74.74950140942003) bank164142 +164143 POINT(40.74660052919538 73.25219347781437) bank164143 +164144 POINT(41.36540686737009 74.43562538811611) bank164144 +164145 POINT(41.7049197439117 73.33555148099612) bank164145 +164146 POINT(41.33340584717511 74.73535985423048) bank164146 +164147 POINT(40.050122019271164 74.35077006492776) bank164147 +164148 POINT(41.69268518015988 74.50916885059446) bank164148 +164149 POINT(41.54324268616592 74.64722031240294) bank164149 +164150 POINT(40.49160456227363 73.31461866504436) bank164150 +164151 POINT(41.14445318257472 74.50173661613343) bank164151 +164152 POINT(40.311150841654516 73.67932717653501) bank164152 +164153 POINT(41.276991008842444 74.18473985052628) bank164153 +164154 POINT(40.11522528851944 73.2224425466011) bank164154 +164155 POINT(40.039390339934116 74.7235191682379) bank164155 +164156 POINT(41.5634653317362 74.03170491888157) bank164156 +164157 POINT(40.3029221880552 73.69157947171419) bank164157 +164158 POINT(39.81531448053397 74.17253176332137) bank164158 +164159 POINT(41.39137721126112 73.86125987234882) bank164159 +164160 POINT(39.89161344505652 73.40837951498732) bank164160 +164161 POINT(41.352188315153825 73.17205611320588) bank164161 +164162 POINT(39.86106058512423 74.39591670914282) bank164162 +164163 POINT(41.59965692575461 73.48040966502323) bank164163 +164164 POINT(40.49200035129928 74.48565748077475) bank164164 +164165 POINT(39.98620069129905 73.19456878012524) bank164165 +164166 POINT(41.127762358023254 74.64628174656332) bank164166 +164167 POINT(39.71878575849983 74.45277355740708) bank164167 +164168 POINT(40.53066456206448 74.92392013711911) bank164168 +164169 POINT(39.778307515010376 74.93392159671664) bank164169 +164170 POINT(41.009789493113395 74.46823678482328) bank164170 +164171 POINT(39.72195362133675 73.64366316329637) bank164171 +164172 POINT(41.07747785705757 74.52452263310443) bank164172 +164173 POINT(40.8873429908283 73.21218884254841) bank164173 +164174 POINT(41.45480044371123 73.12381760833742) bank164174 +164175 POINT(41.55093159700581 74.09119416887013) bank164175 +164176 POINT(40.13473225760848 73.62784475878642) bank164176 +164177 POINT(40.35152580266557 74.96738278178333) bank164177 +164178 POINT(41.66343537401617 73.17066960829865) bank164178 +164179 POINT(40.73277707705586 74.33118944986231) bank164179 +164180 POINT(41.14187523827801 74.00267773770283) bank164180 +164181 POINT(41.16437814107489 74.24062859464702) bank164181 +164182 POINT(40.825816030121686 73.1254543366056) bank164182 +164183 POINT(40.454647761418684 74.71516626909126) bank164183 +164184 POINT(40.22756437355332 74.96288221409512) bank164184 +164185 POINT(41.35096987455403 73.71561094934751) bank164185 +164186 POINT(40.999090958095266 73.40813638733314) bank164186 +164187 POINT(40.30961772692512 74.730302057195) bank164187 +164188 POINT(40.19959025688168 74.00810027280988) bank164188 +164189 POINT(41.68948792264741 73.79841273640153) bank164189 +164190 POINT(40.778647316272114 73.25647558997692) bank164190 +164191 POINT(41.48324520698558 74.22004695380134) bank164191 +164192 POINT(40.63453280540466 74.43254199203197) bank164192 +164193 POINT(39.992319816543585 74.3048070639874) bank164193 +164194 POINT(41.33118759100882 74.83480652229048) bank164194 +164195 POINT(41.02183093792288 74.00920792765993) bank164195 +164196 POINT(39.84376948819848 74.07412858492349) bank164196 +164197 POINT(39.79897144582281 74.51174822927588) bank164197 +164198 POINT(41.16659038375577 73.08751708143316) bank164198 +164199 POINT(40.74987492738376 74.201263032261) bank164199 +164200 POINT(41.038240386140224 74.22128151671001) bank164200 +164201 POINT(39.7572915022777 74.48988531566583) bank164201 +164202 POINT(41.31746581826672 74.5942351542132) bank164202 +164203 POINT(41.2094495839239 73.03651079733478) bank164203 +164204 POINT(40.60682233564667 74.37321955789736) bank164204 +164205 POINT(41.13455820356474 74.34122992446277) bank164205 +164206 POINT(41.5960257738397 74.28594404663933) bank164206 +164207 POINT(40.97738073574455 74.3218201369013) bank164207 +164208 POINT(39.89798175762266 73.04613130035764) bank164208 +164209 POINT(41.57402801636979 73.08399589287451) bank164209 +164210 POINT(41.058056953736866 73.02734308325536) bank164210 +164211 POINT(39.882615093186395 73.30932039891033) bank164211 +164212 POINT(40.855637440228904 73.1617689900016) bank164212 +164213 POINT(41.55477288719806 73.95263146990315) bank164213 +164214 POINT(40.6906694902817 74.96308373971708) bank164214 +164215 POINT(41.154028678074965 74.56834969613861) bank164215 +164216 POINT(41.564540168211785 74.74034117321123) bank164216 +164217 POINT(39.944713757828 73.40027056270368) bank164217 +164218 POINT(39.723276507907975 74.84513642280002) bank164218 +164219 POINT(40.580988933748074 74.6792710429738) bank164219 +164220 POINT(41.69841599838899 74.94411717116411) bank164220 +164221 POINT(40.81430713267875 73.17899968622213) bank164221 +164222 POINT(40.76171554706768 73.42818579411964) bank164222 +164223 POINT(41.29409341652686 73.56883649459405) bank164223 +164224 POINT(40.43418535526501 73.81954537245547) bank164224 +164225 POINT(41.236376234753266 74.68965492875674) bank164225 +164226 POINT(41.18700524897601 74.70816697460481) bank164226 +164227 POINT(40.40828931361309 73.09640080554333) bank164227 +164228 POINT(41.014208458415375 74.86847874495106) bank164228 +164229 POINT(41.13083777396368 73.79946989237654) bank164229 +164230 POINT(39.80806729320214 74.12469617044097) bank164230 +164231 POINT(40.16164822153469 73.24451411188949) bank164231 +164232 POINT(40.463686594569495 73.8630140534735) bank164232 +164233 POINT(40.306501139380146 73.4408463860115) bank164233 +164234 POINT(41.60624110880331 73.71061973381228) bank164234 +164235 POINT(41.20909781739157 74.77882125097486) bank164235 +164236 POINT(41.64201773186977 73.8531081333224) bank164236 +164237 POINT(41.05897304270537 73.33892087768429) bank164237 +164238 POINT(39.790717713181486 74.48380738197675) bank164238 +164239 POINT(41.20419181466066 73.92915093894514) bank164239 +164240 POINT(41.677310285913634 74.43416744729981) bank164240 +164241 POINT(41.25199263387276 74.05018811631533) bank164241 +164242 POINT(40.726848280857524 74.63482192557356) bank164242 +164243 POINT(40.57980664295621 74.15799114513835) bank164243 +164244 POINT(40.496267304137405 73.83409634363555) bank164244 +164245 POINT(41.57231715867508 74.41053601321491) bank164245 +164246 POINT(40.80248838383757 73.91281577424031) bank164246 +164247 POINT(40.578998203338216 73.4693377847155) bank164247 +164248 POINT(41.68937902733571 73.91416684204638) bank164248 +164249 POINT(41.483692599923735 73.16424353457447) bank164249 +164250 POINT(41.23245452028232 73.98284564398202) bank164250 +164251 POINT(40.715561414412 73.10194178386085) bank164251 +164252 POINT(40.10916390594205 74.51731431221428) bank164252 +164253 POINT(40.394453791441315 73.52604761608154) bank164253 +164254 POINT(41.436063433554814 74.76045066913449) bank164254 +164255 POINT(39.75109741860979 74.57555466146819) bank164255 +164256 POINT(41.30916065660304 74.76511161796309) bank164256 +164257 POINT(41.42911800703203 73.84050525440031) bank164257 +164258 POINT(40.24492388750035 74.85986437625618) bank164258 +164259 POINT(40.79108248949224 73.22820070581425) bank164259 +164260 POINT(39.874339866012626 73.51427117602393) bank164260 +164261 POINT(40.95717236221804 74.06146995533601) bank164261 +164262 POINT(41.378255058929426 73.87318676774372) bank164262 +164263 POINT(39.746268701692145 73.38166171412158) bank164263 +164264 POINT(41.58808656738789 73.64514135605819) bank164264 +164265 POINT(40.314003013603056 74.05294939149658) bank164265 +164266 POINT(40.28330287690441 73.28757520301727) bank164266 +164267 POINT(40.11107967142023 74.06562602789388) bank164267 +164268 POINT(40.90125274593591 73.2085441931245) bank164268 +164269 POINT(40.429331145781035 74.09958902532648) bank164269 +164270 POINT(39.90939344182114 74.82307073162315) bank164270 +164271 POINT(41.47545584383898 74.30886038615502) bank164271 +164272 POINT(40.32996206347003 74.38189216652705) bank164272 +164273 POINT(41.60902168800104 73.99838312951945) bank164273 +164274 POINT(41.70698743401701 74.96281408421729) bank164274 +164275 POINT(40.77234837258311 74.16577265085596) bank164275 +164276 POINT(41.586825196913075 74.54801886716463) bank164276 +164277 POINT(41.66919081526817 74.33425695464888) bank164277 +164278 POINT(40.94865151787971 74.66744948682613) bank164278 +164279 POINT(40.845392069739944 73.48615243242617) bank164279 +164280 POINT(40.665209300767295 74.78608341828465) bank164280 +164281 POINT(40.52740441817814 73.77566995099653) bank164281 +164282 POINT(41.551914864031154 74.30159901256052) bank164282 +164283 POINT(41.34450838987719 74.75287338635064) bank164283 +164284 POINT(39.91344351141588 73.76035053791688) bank164284 +164285 POINT(41.30620721728225 74.68688297710507) bank164285 +164286 POINT(40.39081639717066 73.11124239502782) bank164286 +164287 POINT(40.04694253966199 74.6485059771801) bank164287 +164288 POINT(39.93921979750841 73.21362490965176) bank164288 +164289 POINT(40.471763409482215 73.33088487099037) bank164289 +164290 POINT(40.13550719334266 74.61463396407545) bank164290 +164291 POINT(40.87876185227604 73.61202714117383) bank164291 +164292 POINT(41.42358930045208 73.37667287561507) bank164292 +164293 POINT(41.578698703110845 74.22360911413647) bank164293 +164294 POINT(41.49064913216299 73.05973689104779) bank164294 +164295 POINT(40.44803664778757 74.70808258454034) bank164295 +164296 POINT(40.126480947292556 73.2896367972174) bank164296 +164297 POINT(39.87634127406592 73.9481903688051) bank164297 +164298 POINT(41.596755886756995 73.68928662495138) bank164298 +164299 POINT(41.52024604917141 74.47234561109684) bank164299 +164300 POINT(41.04502607668432 74.97253220323964) bank164300 +164301 POINT(40.25823575873898 74.15195296667756) bank164301 +164302 POINT(40.058486184061664 74.38809022305735) bank164302 +164303 POINT(41.209568600655174 74.69012101738713) bank164303 +164304 POINT(40.069429027874705 73.94797854923512) bank164304 +164305 POINT(39.91557342608204 73.394695819721) bank164305 +164306 POINT(41.081576396528575 74.97956980978087) bank164306 +164307 POINT(41.0379235496041 73.04896732315915) bank164307 +164308 POINT(40.557326549957665 73.84800021090008) bank164308 +164309 POINT(39.802932104954934 73.48910115759553) bank164309 +164310 POINT(40.263319902643026 74.9038447382592) bank164310 +164311 POINT(40.56260333034731 74.90563303194324) bank164311 +164312 POINT(40.90837810077068 73.36170171429151) bank164312 +164313 POINT(41.05042351875845 74.9497355657638) bank164313 +164314 POINT(40.11073152755959 73.95546682076011) bank164314 +164315 POINT(41.28052646039303 74.74897788086905) bank164315 +164316 POINT(40.41911461842715 74.11059133931624) bank164316 +164317 POINT(41.28080226378285 73.54271420175857) bank164317 +164318 POINT(41.469882382857804 73.0737946141126) bank164318 +164319 POINT(40.14558787116465 74.0308342036259) bank164319 +164320 POINT(41.30090089413822 74.55692015972863) bank164320 +164321 POINT(40.676608300816646 73.12874305903979) bank164321 +164322 POINT(40.22897955403254 74.48670569443942) bank164322 +164323 POINT(41.67935013941762 74.25927790843525) bank164323 +164324 POINT(40.346436443765775 73.40741030634881) bank164324 +164325 POINT(40.645023225858125 73.54408704868311) bank164325 +164326 POINT(40.643676022172784 73.52542144616712) bank164326 +164327 POINT(40.70186085123178 73.56359449954583) bank164327 +164328 POINT(40.45310803595836 74.11551486780054) bank164328 +164329 POINT(41.69632895617714 73.33131404464426) bank164329 +164330 POINT(40.29339765741477 73.06713739345139) bank164330 +164331 POINT(41.10395893515484 73.42198987236628) bank164331 +164332 POINT(40.044903700086884 73.53259476584196) bank164332 +164333 POINT(41.40030467138175 74.26802339800116) bank164333 +164334 POINT(39.82993117425502 73.85355915732522) bank164334 +164335 POINT(40.16247183023879 74.98251077086017) bank164335 +164336 POINT(40.78890056440974 73.70030462066244) bank164336 +164337 POINT(39.950984970814524 73.09472372344437) bank164337 +164338 POINT(40.77653608383741 74.43862884472072) bank164338 +164339 POINT(40.04173767038254 74.96532087924781) bank164339 +164340 POINT(40.260130619095506 73.73331979067054) bank164340 +164341 POINT(40.84687860381446 74.49057561910729) bank164341 +164342 POINT(40.14765042636566 74.50453587186333) bank164342 +164343 POINT(40.72583919876526 73.35370223818077) bank164343 +164344 POINT(40.202922610091925 73.80677460800771) bank164344 +164345 POINT(41.44096079437137 74.80714495190745) bank164345 +164346 POINT(40.945333357544214 74.35427617335907) bank164346 +164347 POINT(41.654447903700635 74.81418651100446) bank164347 +164348 POINT(40.35879906296261 74.09585442537015) bank164348 +164349 POINT(40.4019395001334 74.01939563292197) bank164349 +164350 POINT(40.78128300635167 74.75289744383649) bank164350 +164351 POINT(39.71471651845527 74.91241296199276) bank164351 +164352 POINT(40.00752328279612 74.68972283775072) bank164352 +164353 POINT(41.506722168361335 74.49742294689057) bank164353 +164354 POINT(40.025780632090814 74.62203468016774) bank164354 +164355 POINT(41.527083009299204 74.89004923230519) bank164355 +164356 POINT(40.0349878853422 73.3806296336937) bank164356 +164357 POINT(39.86352792819292 74.50130174294941) bank164357 +164358 POINT(39.84217589932635 73.1259474167624) bank164358 +164359 POINT(40.37732349257366 73.57848089180112) bank164359 +164360 POINT(41.271954151608156 73.6697658356218) bank164360 +164361 POINT(39.75179370548319 74.44212373215721) bank164361 +164362 POINT(41.280677274628594 73.20569604916628) bank164362 +164363 POINT(40.13417212958485 74.81144438912727) bank164363 +164364 POINT(40.71898940049064 73.53862247036113) bank164364 +164365 POINT(40.035645093553825 74.8434681950711) bank164365 +164366 POINT(41.19039118831941 74.66779413501499) bank164366 +164367 POINT(41.30680119315269 73.39261681299465) bank164367 +164368 POINT(41.046763824338484 73.0478344470639) bank164368 +164369 POINT(39.78476232490741 74.08226377141726) bank164369 +164370 POINT(41.29344647585055 74.46128366850536) bank164370 +164371 POINT(39.97711195611203 74.29479682004022) bank164371 +164372 POINT(39.72386480738988 74.26640440229284) bank164372 +164373 POINT(40.28034707432536 74.09887655222866) bank164373 +164374 POINT(41.10903825171959 74.437081497451) bank164374 +164375 POINT(41.26771902231147 74.38164455708741) bank164375 +164376 POINT(39.941732974461395 73.8080303742721) bank164376 +164377 POINT(40.898802172716685 74.54785676091068) bank164377 +164378 POINT(40.2379569119487 73.07852110788292) bank164378 +164379 POINT(41.461482425148084 74.37637826468725) bank164379 +164380 POINT(41.67989064609267 74.83694389941316) bank164380 +164381 POINT(41.131755042364965 73.50193957406076) bank164381 +164382 POINT(39.76942908722679 74.23563670409845) bank164382 +164383 POINT(40.67890465889618 73.78548857483626) bank164383 +164384 POINT(40.910431434779355 73.04876697457077) bank164384 +164385 POINT(40.87549521343854 74.55172533194603) bank164385 +164386 POINT(41.19604612845053 74.43505895945626) bank164386 +164387 POINT(40.77671398082502 74.87506314743473) bank164387 +164388 POINT(41.18832172421916 74.30897795510494) bank164388 +164389 POINT(41.689393283703296 74.8164958518442) bank164389 +164390 POINT(41.39584700507717 74.70577802536448) bank164390 +164391 POINT(41.01166628441026 73.84602326384812) bank164391 +164392 POINT(40.51348669100547 74.03212573989039) bank164392 +164393 POINT(40.64211200211829 74.32647789409198) bank164393 +164394 POINT(41.00320088139677 74.13812216717224) bank164394 +164395 POINT(41.57569622597686 73.28236629352722) bank164395 +164396 POINT(41.0963669997605 73.76741088682788) bank164396 +164397 POINT(39.96936382393859 74.59337639146739) bank164397 +164398 POINT(41.43058568359742 74.21511851569521) bank164398 +164399 POINT(40.962638369610005 73.0961995406133) bank164399 +164400 POINT(39.861788750299795 73.67967846762375) bank164400 +164401 POINT(41.49531174597354 73.09256835312993) bank164401 +164402 POINT(41.41026064049406 74.13539751382108) bank164402 +164403 POINT(41.22369747898973 74.83692593474612) bank164403 +164404 POINT(41.035210019258095 73.74833484223278) bank164404 +164405 POINT(41.29103790946183 73.44560242519245) bank164405 +164406 POINT(41.00952812459459 73.31807117036541) bank164406 +164407 POINT(39.761124834997894 74.50837792183195) bank164407 +164408 POINT(40.63417900149683 73.26682674854337) bank164408 +164409 POINT(39.78684979233455 74.70259188483797) bank164409 +164410 POINT(40.76605537111888 74.76447818698284) bank164410 +164411 POINT(40.92998016799638 73.23642731661495) bank164411 +164412 POINT(41.541362184886026 73.28488659208304) bank164412 +164413 POINT(39.88476292495369 74.64073413435321) bank164413 +164414 POINT(41.39986289400204 74.97341624697378) bank164414 +164415 POINT(41.6549707817001 74.9188029533829) bank164415 +164416 POINT(39.82268000171364 74.03991383311866) bank164416 +164417 POINT(40.97235274892237 74.68883216001458) bank164417 +164418 POINT(40.466926459377405 73.48356149924196) bank164418 +164419 POINT(40.84778330254547 74.30645405177377) bank164419 +164420 POINT(41.199193406579205 73.66940259188829) bank164420 +164421 POINT(41.60151454666438 73.30751850635073) bank164421 +164422 POINT(40.29081769169257 73.37415973062883) bank164422 +164423 POINT(41.14363792343903 74.94544848672787) bank164423 +164424 POINT(39.795866578911124 74.25444023346026) bank164424 +164425 POINT(39.842658855378914 74.96712136927319) bank164425 +164426 POINT(40.79949116152421 73.97298063335613) bank164426 +164427 POINT(40.06880921391225 74.22152217017866) bank164427 +164428 POINT(39.755107197857065 75.00069149412903) bank164428 +164429 POINT(40.39255837205565 74.04540239196456) bank164429 +164430 POINT(41.14419213435979 74.29565868528361) bank164430 +164431 POINT(40.015040463165754 73.72696459990433) bank164431 +164432 POINT(40.356910442019895 74.88518612523833) bank164432 +164433 POINT(40.81094642561423 73.21416574638928) bank164433 +164434 POINT(40.938450967498895 74.43354672664319) bank164434 +164435 POINT(41.623131110641204 73.9232450783275) bank164435 +164436 POINT(41.02569593423221 73.20132836070579) bank164436 +164437 POINT(41.28001421359946 74.20142448272475) bank164437 +164438 POINT(40.65402663258682 73.43281758660015) bank164438 +164439 POINT(40.052992902671505 74.63735650094326) bank164439 +164440 POINT(40.6944939165476 73.09142846485865) bank164440 +164441 POINT(39.91216748751789 73.23469872792089) bank164441 +164442 POINT(40.43710958402811 74.8037489981091) bank164442 +164443 POINT(39.76408828669406 74.81313139466833) bank164443 +164444 POINT(40.24174837890417 74.78097844996736) bank164444 +164445 POINT(40.61697204919739 74.24408419188123) bank164445 +164446 POINT(40.41859399609973 73.76605333414753) bank164446 +164447 POINT(40.84604166501502 73.03473073037472) bank164447 +164448 POINT(40.58400570916613 73.133712897398) bank164448 +164449 POINT(40.97939415645351 74.9207713142726) bank164449 +164450 POINT(41.70322555815091 73.08629986967937) bank164450 +164451 POINT(40.492540456975306 74.69025621140527) bank164451 +164452 POINT(40.89184620172953 73.10847121604195) bank164452 +164453 POINT(39.94764671320564 74.09371862943922) bank164453 +164454 POINT(40.575732676722005 73.13440243651338) bank164454 +164455 POINT(40.992899018755494 73.75391290567927) bank164455 +164456 POINT(40.88560473467363 73.83654515740949) bank164456 +164457 POINT(40.16686834827107 73.12122165362067) bank164457 +164458 POINT(40.47923905439328 74.02058085084875) bank164458 +164459 POINT(41.13423132125343 73.95771051697315) bank164459 +164460 POINT(41.709861101957664 74.17930200599449) bank164460 +164461 POINT(41.58227680000631 74.95786772906229) bank164461 +164462 POINT(41.34300088332861 74.87938355538849) bank164462 +164463 POINT(39.76778096008688 74.03910784414781) bank164463 +164464 POINT(41.16957717433589 73.25525359585704) bank164464 +164465 POINT(41.63700067750632 73.2028260950889) bank164465 +164466 POINT(41.399582279546124 73.78261653658787) bank164466 +164467 POINT(40.00536395023627 73.08690558127928) bank164467 +164468 POINT(40.056328983346404 73.05219162210967) bank164468 +164469 POINT(41.63027347068328 74.51287765856674) bank164469 +164470 POINT(40.26377889207609 73.92610941700507) bank164470 +164471 POINT(40.19215366211776 74.58098210096318) bank164471 +164472 POINT(40.42487929393951 74.05499302519412) bank164472 +164473 POINT(40.115709313974115 73.44304623168937) bank164473 +164474 POINT(41.08744327001354 74.5740507470297) bank164474 +164475 POINT(40.89482723278445 73.49234311692264) bank164475 +164476 POINT(40.94324933875738 73.12613957229144) bank164476 +164477 POINT(41.47929032566987 73.64823984926458) bank164477 +164478 POINT(41.11367842543035 73.6210812472609) bank164478 +164479 POINT(40.57519493832745 73.3292098520194) bank164479 +164480 POINT(41.11268706298915 73.04501362449062) bank164480 +164481 POINT(40.17065144925935 73.36974117748478) bank164481 +164482 POINT(39.980688145796805 73.77876316997936) bank164482 +164483 POINT(41.697464478611344 73.01649308453466) bank164483 +164484 POINT(40.63718356120569 74.4605388366764) bank164484 +164485 POINT(41.030468463892575 74.83086154951611) bank164485 +164486 POINT(41.11973708220844 73.3309589137205) bank164486 +164487 POINT(40.7814449126217 74.158580824593) bank164487 +164488 POINT(41.493218566813 74.49718035451085) bank164488 +164489 POINT(40.692011010757696 73.37692363569494) bank164489 +164490 POINT(40.889709972101315 74.45381807897229) bank164490 +164491 POINT(40.97003902663171 73.70083059278016) bank164491 +164492 POINT(40.050219450658574 73.90015467678482) bank164492 +164493 POINT(40.72516013062365 73.21919112022374) bank164493 +164494 POINT(40.47107555020388 73.92006348108286) bank164494 +164495 POINT(41.6288444534804 74.43536793112621) bank164495 +164496 POINT(40.19701818632384 74.97967202036975) bank164496 +164497 POINT(40.288519863866085 74.45002096355582) bank164497 +164498 POINT(40.94033503332238 74.93351559014658) bank164498 +164499 POINT(41.25762741178397 73.39461526100139) bank164499 +164500 POINT(41.048496791381524 73.9893056544959) bank164500 +164501 POINT(40.68773555265985 73.82095361231806) bank164501 +164502 POINT(40.16310837150945 73.48937041351644) bank164502 +164503 POINT(40.39707300813023 74.84332966259439) bank164503 +164504 POINT(40.81379453536756 74.2672723012073) bank164504 +164505 POINT(41.06815573776664 73.20819337378389) bank164505 +164506 POINT(41.411890805153455 73.55523661002684) bank164506 +164507 POINT(41.24682381281371 73.89114064996016) bank164507 +164508 POINT(40.483684839777375 73.88101318056081) bank164508 +164509 POINT(40.519318425596346 74.27563273478104) bank164509 +164510 POINT(41.58877123711015 73.19387887813818) bank164510 +164511 POINT(41.44466961347617 74.23921217817367) bank164511 +164512 POINT(40.45998056863859 73.13203566223389) bank164512 +164513 POINT(41.43320303293278 74.21500820756924) bank164513 +164514 POINT(41.25240816166693 73.35962108411437) bank164514 +164515 POINT(41.44963805736776 74.57802022410412) bank164515 +164516 POINT(40.35153169374158 73.97489353320218) bank164516 +164517 POINT(41.47525502275253 74.18484999085602) bank164517 +164518 POINT(40.2772240273724 73.35903986489443) bank164518 +164519 POINT(40.676349500422404 74.74859000616577) bank164519 +164520 POINT(40.30367222046254 73.1039678886431) bank164520 +164521 POINT(40.49647469080661 73.73961161238492) bank164521 +164522 POINT(40.31710973589002 74.49102015264228) bank164522 +164523 POINT(40.62695492120943 74.10867068370847) bank164523 +164524 POINT(41.494891375236456 73.55126594887179) bank164524 +164525 POINT(40.40522692648454 73.93412538275895) bank164525 +164526 POINT(40.094001011891386 73.10295180176881) bank164526 +164527 POINT(41.530385629993 74.35166602373505) bank164527 +164528 POINT(40.034967089477625 73.03564451005516) bank164528 +164529 POINT(40.57505924634872 74.36812572825565) bank164529 +164530 POINT(39.73258522376517 73.28826011661086) bank164530 +164531 POINT(40.73656251684369 74.86265560200322) bank164531 +164532 POINT(41.351533840375666 74.79803772556731) bank164532 +164533 POINT(39.91499645097754 74.41680575957103) bank164533 +164534 POINT(41.63059335551919 73.71842421844529) bank164534 +164535 POINT(41.491937920798584 73.35169278225715) bank164535 +164536 POINT(40.4247145406443 73.48404863643715) bank164536 +164537 POINT(41.374024953916546 74.64476220796784) bank164537 +164538 POINT(40.663351117586835 73.2027178133195) bank164538 +164539 POINT(41.37363747457835 74.69221105598747) bank164539 +164540 POINT(39.99607004171583 74.3906829194299) bank164540 +164541 POINT(41.01744893995019 73.45867994645391) bank164541 +164542 POINT(39.958852405467596 74.21055353668199) bank164542 +164543 POINT(40.04458636109537 73.80516061254919) bank164543 +164544 POINT(40.99405465515311 74.19716781247321) bank164544 +164545 POINT(41.70366548111699 74.28081390494955) bank164545 +164546 POINT(41.30349173570923 74.30863214559773) bank164546 +164547 POINT(39.85288390816062 74.20308308866672) bank164547 +164548 POINT(39.794403556392055 73.24251995144724) bank164548 +164549 POINT(41.11273746338142 74.54671592524683) bank164549 +164550 POINT(41.42853140399142 73.53217405574912) bank164550 +164551 POINT(41.47879283689874 74.17847684267365) bank164551 +164552 POINT(41.1129335227154 73.33053091920412) bank164552 +164553 POINT(40.05673627879517 73.70764212692244) bank164553 +164554 POINT(41.65662278688678 74.9075475215695) bank164554 +164555 POINT(40.47316453336132 73.42957127609267) bank164555 +164556 POINT(41.176603563426646 74.52281887265245) bank164556 +164557 POINT(41.130255942377694 73.5390106822962) bank164557 +164558 POINT(40.73736046948145 73.38791400633788) bank164558 +164559 POINT(41.07709990358206 73.07450695665675) bank164559 +164560 POINT(40.932775850390286 74.4758535596058) bank164560 +164561 POINT(41.60847129652462 73.28598141611147) bank164561 +164562 POINT(41.0941121232176 74.18556058925061) bank164562 +164563 POINT(40.968280337180246 73.05918175077629) bank164563 +164564 POINT(40.62735506128876 73.30686115658773) bank164564 +164565 POINT(40.82316136063822 74.95676077834148) bank164565 +164566 POINT(41.45216692588185 74.05665763645442) bank164566 +164567 POINT(40.28576591354078 73.38623815014674) bank164567 +164568 POINT(40.94762895070914 74.30955232497644) bank164568 +164569 POINT(40.440463496853006 74.1134378310748) bank164569 +164570 POINT(39.88192919653562 74.96646712481616) bank164570 +164571 POINT(40.70979660609564 73.92489017670623) bank164571 +164572 POINT(41.14921914239182 73.34040361627785) bank164572 +164573 POINT(40.80139653607123 74.43799530299307) bank164573 +164574 POINT(41.163282072909304 73.04884258777949) bank164574 +164575 POINT(40.595584097249436 73.64527463354766) bank164575 +164576 POINT(39.771331650441276 74.05094864086011) bank164576 +164577 POINT(41.412221346829156 74.21168496000803) bank164577 +164578 POINT(40.57173082112915 73.05940415843662) bank164578 +164579 POINT(40.62828113868151 73.32793944458908) bank164579 +164580 POINT(40.379166995235686 74.2579389573323) bank164580 +164581 POINT(40.071884798486806 73.43378668931919) bank164581 +164582 POINT(39.730879181203434 73.31837912861089) bank164582 +164583 POINT(41.52201214106532 74.75077481921073) bank164583 +164584 POINT(39.95823697889839 74.96707713494838) bank164584 +164585 POINT(41.249827501474506 74.60553713982769) bank164585 +164586 POINT(40.07320908809716 73.22183811292523) bank164586 +164587 POINT(40.27966274062515 74.31066722099033) bank164587 +164588 POINT(40.85664000870766 74.59402486196487) bank164588 +164589 POINT(41.30338485313341 73.19692332460475) bank164589 +164590 POINT(40.45407149228009 74.76050065273476) bank164590 +164591 POINT(40.00522808517357 73.54077103331646) bank164591 +164592 POINT(40.30472790407389 74.2315157772628) bank164592 +164593 POINT(40.21463351076206 74.09686484956866) bank164593 +164594 POINT(40.01099249678254 74.75939407035156) bank164594 +164595 POINT(40.19139832239103 73.70008004569897) bank164595 +164596 POINT(39.89504677415175 74.28114143620212) bank164596 +164597 POINT(41.05688503561356 74.5115879838613) bank164597 +164598 POINT(39.73596624745734 74.79708547388256) bank164598 +164599 POINT(40.08818293640304 73.64254894003487) bank164599 +164600 POINT(41.308027098596355 73.90851942320243) bank164600 +164601 POINT(41.01152091704683 74.69778068753189) bank164601 +164602 POINT(41.60742603586637 74.48330072188257) bank164602 +164603 POINT(40.91957451034093 73.5080981923684) bank164603 +164604 POINT(41.2990691446077 74.96149898700021) bank164604 +164605 POINT(40.71584956413356 73.07775205338288) bank164605 +164606 POINT(41.67696921421549 74.32247063836478) bank164606 +164607 POINT(39.96198758791121 74.07170519782984) bank164607 +164608 POINT(39.73505679987508 74.89372683523919) bank164608 +164609 POINT(40.8537726767712 73.12912501253395) bank164609 +164610 POINT(41.3619009640607 74.48164206503333) bank164610 +164611 POINT(41.45793979078037 74.42467205003693) bank164611 +164612 POINT(41.3723548364627 74.0585596928644) bank164612 +164613 POINT(40.19555131887791 74.6244010286705) bank164613 +164614 POINT(40.51341001154106 73.68682833574115) bank164614 +164615 POINT(40.76389444367744 73.70280422542064) bank164615 +164616 POINT(40.27959222371168 73.482594354137) bank164616 +164617 POINT(40.65126487397443 74.64082594792562) bank164617 +164618 POINT(41.404090460440436 73.29689483628832) bank164618 +164619 POINT(40.88642683586406 74.37818071658472) bank164619 +164620 POINT(40.752396282766504 73.57641315822362) bank164620 +164621 POINT(41.04519663041867 73.30519460843252) bank164621 +164622 POINT(40.291043087765246 73.47258986145674) bank164622 +164623 POINT(40.47835452293518 73.98171091371974) bank164623 +164624 POINT(39.98638522050588 73.11314996107326) bank164624 +164625 POINT(41.265334551322184 73.72448640117538) bank164625 +164626 POINT(40.605015868247904 74.17891691251593) bank164626 +164627 POINT(40.73112480414003 73.80926641485775) bank164627 +164628 POINT(41.120530298732696 73.99099631545978) bank164628 +164629 POINT(40.82668343368548 73.5115580755104) bank164629 +164630 POINT(39.88165829134183 73.12399465453352) bank164630 +164631 POINT(41.306503621889284 73.43386790809289) bank164631 +164632 POINT(40.12529088617081 73.23205924537557) bank164632 +164633 POINT(41.02309300486796 73.10177731977632) bank164633 +164634 POINT(41.07501774287823 74.39247191225591) bank164634 +164635 POINT(41.61546541312562 73.57141267985774) bank164635 +164636 POINT(40.824569628043285 73.09604994429488) bank164636 +164637 POINT(40.756572593594285 74.92616502302779) bank164637 +164638 POINT(40.72596529204652 74.97438886041259) bank164638 +164639 POINT(41.38575007238749 74.49487908604945) bank164639 +164640 POINT(40.781255888971394 74.07298376950956) bank164640 +164641 POINT(39.72837436900475 74.06398045193139) bank164641 +164642 POINT(41.443979082598766 73.79170713918411) bank164642 +164643 POINT(40.64923409753568 74.35579029409801) bank164643 +164644 POINT(41.18022730064515 74.54043741541581) bank164644 +164645 POINT(40.156980431918775 74.6035124377022) bank164645 +164646 POINT(40.38716764179843 74.54336952264994) bank164646 +164647 POINT(40.007063072309656 74.61169541322873) bank164647 +164648 POINT(40.03207095090828 73.9869357669874) bank164648 +164649 POINT(41.21880451370846 73.02637595564515) bank164649 +164650 POINT(39.98966897872783 74.17080212366703) bank164650 +164651 POINT(40.821270949888174 73.7415064432767) bank164651 +164652 POINT(40.95424938249067 73.53306790378161) bank164652 +164653 POINT(40.16478985689819 74.02925813577848) bank164653 +164654 POINT(40.74230325969318 74.01556668583613) bank164654 +164655 POINT(41.169080922183184 73.80258457557744) bank164655 +164656 POINT(40.92854294231001 74.00713288749604) bank164656 +164657 POINT(40.051774419817534 74.34068718764892) bank164657 +164658 POINT(40.092431260557426 73.62424422755934) bank164658 +164659 POINT(40.24683813655315 73.90358148842526) bank164659 +164660 POINT(41.17585660959416 74.51005127510547) bank164660 +164661 POINT(40.06205314653411 73.46851463522184) bank164661 +164662 POINT(41.17302732145847 73.1894536687491) bank164662 +164663 POINT(39.90515243088372 73.63193672797846) bank164663 +164664 POINT(41.310612608339824 74.79657559279278) bank164664 +164665 POINT(40.108472966147666 74.6430847729343) bank164665 +164666 POINT(39.80855373935793 74.97758755232324) bank164666 +164667 POINT(41.554433732023426 74.18808285691934) bank164667 +164668 POINT(40.41638084724428 74.20658509029633) bank164668 +164669 POINT(41.03073618107517 73.58076087719239) bank164669 +164670 POINT(41.59170706314619 73.29823039165169) bank164670 +164671 POINT(40.1817223617913 74.9212309658041) bank164671 +164672 POINT(41.672384566811424 74.20562346947078) bank164672 +164673 POINT(40.47884150858359 74.30046865044625) bank164673 +164674 POINT(40.76233177021659 74.51965302938768) bank164674 +164675 POINT(40.25500022986529 74.02592522087997) bank164675 +164676 POINT(40.36676453074547 74.808175058317) bank164676 +164677 POINT(40.63731291905452 74.59963293497614) bank164677 +164678 POINT(40.68608727233088 73.0329593006159) bank164678 +164679 POINT(41.260292437891025 73.35543357078932) bank164679 +164680 POINT(40.47121518845321 73.92202037000384) bank164680 +164681 POINT(41.4031107572027 73.30938247825958) bank164681 +164682 POINT(40.47860199051213 73.41443897928794) bank164682 +164683 POINT(40.67164626890959 73.11531828086392) bank164683 +164684 POINT(39.81474405078571 74.51405640556627) bank164684 +164685 POINT(41.10299349694695 74.49109695764609) bank164685 +164686 POINT(41.36654416257933 74.4290079385642) bank164686 +164687 POINT(40.29429656683495 74.72120376871327) bank164687 +164688 POINT(40.16960176719738 73.17286956223172) bank164688 +164689 POINT(41.615121387982406 74.35093527302288) bank164689 +164690 POINT(40.71252645416544 73.184349255683) bank164690 +164691 POINT(40.09806843973045 74.93858393219702) bank164691 +164692 POINT(41.56680474601799 73.29184934909114) bank164692 +164693 POINT(40.53051006533744 73.62474028407543) bank164693 +164694 POINT(41.577253142839666 73.71924774953968) bank164694 +164695 POINT(41.4570096869163 74.62645964345752) bank164695 +164696 POINT(41.54015382672787 74.89758813759033) bank164696 +164697 POINT(41.51275092402156 74.46277963186421) bank164697 +164698 POINT(41.44385218751699 74.4297375988898) bank164698 +164699 POINT(40.81834432207875 73.33582862542868) bank164699 +164700 POINT(40.64284239431132 73.44407044767262) bank164700 +164701 POINT(39.88679467873115 74.13910587862078) bank164701 +164702 POINT(41.642294248452274 74.43558320594435) bank164702 +164703 POINT(39.77746714183518 74.40067339558283) bank164703 +164704 POINT(40.084889220945264 74.12618812470198) bank164704 +164705 POINT(40.22366899954783 74.18018194552793) bank164705 +164706 POINT(41.342841247065536 74.32594567844706) bank164706 +164707 POINT(40.26877579491706 74.10090179134806) bank164707 +164708 POINT(40.067732652396444 73.74578912834953) bank164708 +164709 POINT(40.40472754173985 73.70351785306586) bank164709 +164710 POINT(41.2103078364034 74.5642667388826) bank164710 +164711 POINT(39.90633619968673 73.3208130148773) bank164711 +164712 POINT(41.493411145815834 73.56339945628363) bank164712 +164713 POINT(41.54328968063533 73.62403643895499) bank164713 +164714 POINT(40.634074974620106 74.43659389074449) bank164714 +164715 POINT(41.60966268640207 73.36743550731423) bank164715 +164716 POINT(40.007093405819404 74.51388045074303) bank164716 +164717 POINT(41.38953859812539 74.67134470436736) bank164717 +164718 POINT(40.606469357742164 74.15759188101899) bank164718 +164719 POINT(40.36976162539473 73.64410549348453) bank164719 +164720 POINT(41.146734110950476 74.25514143820396) bank164720 +164721 POINT(41.37955383132465 74.29483098148565) bank164721 +164722 POINT(41.679105859530196 74.54799239790819) bank164722 +164723 POINT(40.445465601292675 74.39418351883184) bank164723 +164724 POINT(39.97292497291275 74.63159649525498) bank164724 +164725 POINT(41.12382464739342 73.81683987429346) bank164725 +164726 POINT(41.111475609519246 74.72372614493652) bank164726 +164727 POINT(39.82003322868217 73.38389186293361) bank164727 +164728 POINT(40.587844085709825 73.32550038638995) bank164728 +164729 POINT(41.397698712320484 73.09665073613526) bank164729 +164730 POINT(39.788742702845205 74.36588286664542) bank164730 +164731 POINT(39.75347275429985 75.00137652616704) bank164731 +164732 POINT(41.525160796736785 73.33797879788118) bank164732 +164733 POINT(40.199799464377634 74.91976129765263) bank164733 +164734 POINT(39.87128444950471 74.42241394029075) bank164734 +164735 POINT(41.21098025594015 73.33024925067862) bank164735 +164736 POINT(39.84969610603759 73.19846456224953) bank164736 +164737 POINT(39.80457500522121 74.46562038851812) bank164737 +164738 POINT(40.61177437584676 73.20294504314448) bank164738 +164739 POINT(41.067213462015225 73.13942968637126) bank164739 +164740 POINT(39.79238371475566 74.20459192198983) bank164740 +164741 POINT(40.76723830758839 74.83671621730068) bank164741 +164742 POINT(40.253655072222585 74.57265254427091) bank164742 +164743 POINT(40.843251606980516 73.05444492724135) bank164743 +164744 POINT(40.203274892478035 74.33018535654128) bank164744 +164745 POINT(40.71780104380124 73.78340644504921) bank164745 +164746 POINT(40.14855065492716 73.95029542972202) bank164746 +164747 POINT(39.729431037710114 73.24350807449778) bank164747 +164748 POINT(41.526370054117656 73.14576832315821) bank164748 +164749 POINT(40.326976096051396 73.39200336755717) bank164749 +164750 POINT(39.833501490094 74.29533977039014) bank164750 +164751 POINT(40.462449363759504 74.59211982319123) bank164751 +164752 POINT(41.05320822937581 74.57897708851333) bank164752 +164753 POINT(40.13706367187651 74.18661594381986) bank164753 +164754 POINT(39.98674494917711 74.34514196578291) bank164754 +164755 POINT(41.66903013848596 73.91828526096779) bank164755 +164756 POINT(40.313984153293745 73.82239714632226) bank164756 +164757 POINT(40.96741661683447 73.79427784084643) bank164757 +164758 POINT(41.25161617706523 73.55422464333064) bank164758 +164759 POINT(40.945168872301764 74.25893428931275) bank164759 +164760 POINT(41.303319683809754 74.38545463659936) bank164760 +164761 POINT(39.9616407166548 74.13833858617183) bank164761 +164762 POINT(39.97154406546329 73.90927053443407) bank164762 +164763 POINT(40.48117200934027 74.91088921695778) bank164763 +164764 POINT(40.73404075221826 74.22700602410404) bank164764 +164765 POINT(41.19652599264759 73.71954991416827) bank164765 +164766 POINT(41.14125343483625 73.61867976765221) bank164766 +164767 POINT(40.37577522481568 73.65654960551804) bank164767 +164768 POINT(40.19602444156103 73.49521789392205) bank164768 +164769 POINT(40.77838672696659 74.584425617069) bank164769 +164770 POINT(41.42253047197707 74.69159994851883) bank164770 +164771 POINT(39.71763981703075 73.06783555117094) bank164771 +164772 POINT(41.57736012744768 74.58975300994592) bank164772 +164773 POINT(40.81440813254199 74.795748705508) bank164773 +164774 POINT(40.95603601961539 73.2031321372288) bank164774 +164775 POINT(41.099937890658786 74.83789675790851) bank164775 +164776 POINT(41.6150898954963 74.00017287141107) bank164776 +164777 POINT(41.06064526931899 74.15892637649506) bank164777 +164778 POINT(40.247992743355475 74.14035985517957) bank164778 +164779 POINT(41.71267778066169 74.02872658607424) bank164779 +164780 POINT(41.33579704143889 74.34585273687232) bank164780 +164781 POINT(39.74229083980385 73.97384944887438) bank164781 +164782 POINT(40.18711124928353 74.04975316854026) bank164782 +164783 POINT(41.51199940665995 74.13998262916478) bank164783 +164784 POINT(41.46511740861254 73.46952004148379) bank164784 +164785 POINT(40.0184235921739 74.89564815465074) bank164785 +164786 POINT(40.664111898377016 74.58101135288122) bank164786 +164787 POINT(40.76850794586815 73.5920970671036) bank164787 +164788 POINT(40.45417606161432 73.26553745485049) bank164788 +164789 POINT(40.290596869532905 73.35153149912877) bank164789 +164790 POINT(40.95592690205406 74.2844789846936) bank164790 +164791 POINT(41.64325813639156 74.04871269172828) bank164791 +164792 POINT(40.067672635125774 74.45575478950548) bank164792 +164793 POINT(41.600429661642295 73.49720976956097) bank164793 +164794 POINT(39.97898234216469 74.9631762291433) bank164794 +164795 POINT(41.48781477846332 74.4571778101211) bank164795 +164796 POINT(41.39825626230922 74.0024573633819) bank164796 +164797 POINT(40.42585748158641 73.72138515463365) bank164797 +164798 POINT(40.34141763126168 73.2683503544497) bank164798 +164799 POINT(40.499327044149815 74.04682825707478) bank164799 +164800 POINT(41.45152229468125 73.59831087731082) bank164800 +164801 POINT(40.30060524752992 73.0307654691436) bank164801 +164802 POINT(41.621347897667135 74.39686359591776) bank164802 +164803 POINT(40.10310389022739 73.93663857848894) bank164803 +164804 POINT(40.31130681320654 74.77737360653617) bank164804 +164805 POINT(40.856867835911885 74.1461620190486) bank164805 +164806 POINT(39.80731717338745 73.71878615157969) bank164806 +164807 POINT(40.16735970406934 73.07001021278342) bank164807 +164808 POINT(40.35670468985867 74.76154784107847) bank164808 +164809 POINT(41.355808834597866 74.68915463938879) bank164809 +164810 POINT(40.40200198212605 74.78568348076266) bank164810 +164811 POINT(41.60319168201231 73.68945326010378) bank164811 +164812 POINT(40.843116401614424 73.8758322199589) bank164812 +164813 POINT(41.258690029233456 74.20353888120437) bank164813 +164814 POINT(39.880371645418435 73.08181315439504) bank164814 +164815 POINT(39.887286049424475 74.05056019530602) bank164815 +164816 POINT(41.484605998825195 73.75161347963078) bank164816 +164817 POINT(40.467172056248614 73.6348511630311) bank164817 +164818 POINT(40.914453212954015 73.49410061017039) bank164818 +164819 POINT(40.963472983667145 74.69491077697582) bank164819 +164820 POINT(40.185856166512096 74.72911234059974) bank164820 +164821 POINT(41.64639896318948 74.1840816564021) bank164821 +164822 POINT(39.90515926673378 73.08673322421517) bank164822 +164823 POINT(40.693740974424706 74.53512584031249) bank164823 +164824 POINT(40.08431536364717 74.92178559087063) bank164824 +164825 POINT(41.27544875772723 73.54709361955993) bank164825 +164826 POINT(41.43500579993534 73.20753671250151) bank164826 +164827 POINT(41.428536935202274 73.59576528997376) bank164827 +164828 POINT(40.365523085527684 73.05756936850776) bank164828 +164829 POINT(40.507285262682146 74.93019672084691) bank164829 +164830 POINT(41.039017135615865 73.4223763752865) bank164830 +164831 POINT(40.427525807563576 73.49016538536159) bank164831 +164832 POINT(40.62014580279306 73.83168424537891) bank164832 +164833 POINT(41.706898981289136 74.13635283104904) bank164833 +164834 POINT(40.39491922603953 74.18605099546187) bank164834 +164835 POINT(40.61782177242491 73.52884634099907) bank164835 +164836 POINT(40.263997245202894 74.7142251986771) bank164836 +164837 POINT(39.992910132413634 73.94347065808236) bank164837 +164838 POINT(40.900159022833165 74.9677695452664) bank164838 +164839 POINT(41.65640486477311 74.04306791029325) bank164839 +164840 POINT(40.376942316848194 73.48304242487772) bank164840 +164841 POINT(40.995099866366445 73.08388005017056) bank164841 +164842 POINT(40.168395562744834 74.92214990133878) bank164842 +164843 POINT(40.94721967166237 73.95598033887993) bank164843 +164844 POINT(40.12149967951225 74.20738707800066) bank164844 +164845 POINT(40.1243157212165 73.14783413321652) bank164845 +164846 POINT(39.77712862664308 73.28988563162814) bank164846 +164847 POINT(41.03896511364411 73.13925947042829) bank164847 +164848 POINT(40.38337809570169 74.729766107874) bank164848 +164849 POINT(40.8657757314695 73.69844379831288) bank164849 +164850 POINT(40.784382360923104 73.13127649348264) bank164850 +164851 POINT(41.4953821101274 73.8583569619197) bank164851 +164852 POINT(41.558886058257826 73.46445636806533) bank164852 +164853 POINT(41.27617216024452 74.4907817321478) bank164853 +164854 POINT(41.60502903671942 73.77291631493289) bank164854 +164855 POINT(39.77192552506288 73.11429048656143) bank164855 +164856 POINT(41.519404024197456 74.7022829409345) bank164856 +164857 POINT(41.27444821839214 73.10972496703036) bank164857 +164858 POINT(40.00640683392561 74.33994728659016) bank164858 +164859 POINT(41.32762881698173 74.55985371952677) bank164859 +164860 POINT(40.71273595296178 74.65823112360889) bank164860 +164861 POINT(39.97325167824939 74.2948273469818) bank164861 +164862 POINT(40.848632669041784 73.4751408725857) bank164862 +164863 POINT(40.78392992592382 74.400300767481) bank164863 +164864 POINT(40.64219544346972 73.90610110472399) bank164864 +164865 POINT(40.85537802488268 73.56841846555724) bank164865 +164866 POINT(41.640076211114 74.3430850906162) bank164866 +164867 POINT(40.20863329507215 73.25464280397263) bank164867 +164868 POINT(41.470241505607916 73.91616640851208) bank164868 +164869 POINT(40.50219480014453 73.11620614571136) bank164869 +164870 POINT(39.715715511394095 74.60148199109857) bank164870 +164871 POINT(39.877304031139474 74.28124452226258) bank164871 +164872 POINT(41.3998814572398 73.60286148498587) bank164872 +164873 POINT(41.467377082801576 73.82285340112158) bank164873 +164874 POINT(41.14109203866527 73.12325585976059) bank164874 +164875 POINT(41.02562128548393 74.60345788022013) bank164875 +164876 POINT(41.190296901697835 74.3086800974086) bank164876 +164877 POINT(39.9226082055208 74.03117621815872) bank164877 +164878 POINT(40.335035857053434 74.67450284153304) bank164878 +164879 POINT(41.25660825346679 74.55925850210451) bank164879 +164880 POINT(41.502946032795556 73.091077532851) bank164880 +164881 POINT(41.62302267681128 74.13494915439317) bank164881 +164882 POINT(40.27008513226544 74.63137517508869) bank164882 +164883 POINT(41.17827722952951 73.83518249044371) bank164883 +164884 POINT(40.086980082440824 73.47970153288183) bank164884 +164885 POINT(40.71126024316082 73.68671294935571) bank164885 +164886 POINT(39.90966699841128 74.14789056948887) bank164886 +164887 POINT(40.47060946967213 74.5816197292906) bank164887 +164888 POINT(40.058056462276355 74.29821545748628) bank164888 +164889 POINT(40.56009146282466 74.25467717250275) bank164889 +164890 POINT(41.2071100166618 73.0404315637269) bank164890 +164891 POINT(40.941103085012806 74.79939290240426) bank164891 +164892 POINT(40.22101375098406 74.27816199936017) bank164892 +164893 POINT(41.358133967512465 73.77180740081505) bank164893 +164894 POINT(41.41749218056102 73.75542087608885) bank164894 +164895 POINT(40.10322264066138 73.98794035750585) bank164895 +164896 POINT(40.31509122303566 74.05233171999475) bank164896 +164897 POINT(40.58186476004737 73.12813797529991) bank164897 +164898 POINT(41.41767379081738 74.71906586783203) bank164898 +164899 POINT(40.49051058424313 74.90789825679914) bank164899 +164900 POINT(40.001354219962415 74.48308767022753) bank164900 +164901 POINT(40.131676041646955 74.76444241124898) bank164901 +164902 POINT(41.08650086011514 73.39154980745177) bank164902 +164903 POINT(39.792465153878865 73.85943864785561) bank164903 +164904 POINT(40.826570083636824 73.69136056905862) bank164904 +164905 POINT(40.9355585313819 73.95995653912122) bank164905 +164906 POINT(41.09346231352166 73.91757853777315) bank164906 +164907 POINT(39.85570702618257 73.9374018283979) bank164907 +164908 POINT(40.63023558362797 73.83092003508298) bank164908 +164909 POINT(40.38144177458233 73.1550397888804) bank164909 +164910 POINT(40.740427363007115 74.48724639628514) bank164910 +164911 POINT(39.90500453372429 74.03406831801993) bank164911 +164912 POINT(40.642750361264895 74.78807495726095) bank164912 +164913 POINT(41.63834833319739 74.14313542462372) bank164913 +164914 POINT(41.41818861472146 73.71462582649713) bank164914 +164915 POINT(41.403919123316186 74.74396526022932) bank164915 +164916 POINT(39.94177688746827 74.34510309115294) bank164916 +164917 POINT(41.06370022487491 74.55054100998119) bank164917 +164918 POINT(40.97871755653606 73.23452082800873) bank164918 +164919 POINT(41.05264494552139 74.23408114213147) bank164919 +164920 POINT(41.55675561782068 74.46516593281959) bank164920 +164921 POINT(40.82690477725384 73.50815222866248) bank164921 +164922 POINT(40.20263296585993 74.97622042148772) bank164922 +164923 POINT(39.97487573422667 74.73504918312675) bank164923 +164924 POINT(39.719011640214134 74.5383383823036) bank164924 +164925 POINT(40.54914962654496 73.70234408432013) bank164925 +164926 POINT(41.24316555411908 73.9545336342472) bank164926 +164927 POINT(41.61413272742301 74.04954783105583) bank164927 +164928 POINT(40.51513738012825 74.73599031785584) bank164928 +164929 POINT(39.86510520366378 74.43581328440027) bank164929 +164930 POINT(40.56088516700184 74.38887091405057) bank164930 +164931 POINT(39.76671154134502 74.89940410233365) bank164931 +164932 POINT(40.065255836403466 73.60532217765441) bank164932 +164933 POINT(41.4588696686942 74.47388923314986) bank164933 +164934 POINT(40.989394892205496 74.32758838052399) bank164934 +164935 POINT(40.757561398935195 74.45354768913943) bank164935 +164936 POINT(40.46476821626165 74.04796950263214) bank164936 +164937 POINT(40.869795491158925 74.9379958813057) bank164937 +164938 POINT(41.110145025144035 74.40297424335083) bank164938 +164939 POINT(41.63232327729924 73.37855615288098) bank164939 +164940 POINT(39.96784922840317 74.14890438035208) bank164940 +164941 POINT(40.46182173875623 73.62023255214527) bank164941 +164942 POINT(41.57047283374402 73.39940600677565) bank164942 +164943 POINT(40.92295354552692 73.98423989874344) bank164943 +164944 POINT(41.32586160945333 74.5765076349137) bank164944 +164945 POINT(41.69228046734264 73.47100270732098) bank164945 +164946 POINT(41.05463850941065 74.62654014133521) bank164946 +164947 POINT(40.40870264983599 73.77580242551169) bank164947 +164948 POINT(41.70662202973883 74.9738927811898) bank164948 +164949 POINT(40.97254272166182 74.8604632964886) bank164949 +164950 POINT(40.93023893008002 74.64146932966797) bank164950 +164951 POINT(40.89385672095444 74.33440675381344) bank164951 +164952 POINT(41.39174654141367 73.90493332127764) bank164952 +164953 POINT(40.871577683441146 73.58211917387132) bank164953 +164954 POINT(39.80811080112252 74.85581171110618) bank164954 +164955 POINT(41.441420902940294 73.1078093105433) bank164955 +164956 POINT(40.92361175743883 73.78380729660263) bank164956 +164957 POINT(40.891018414608986 74.74838699524639) bank164957 +164958 POINT(41.05377715659999 73.03807721144369) bank164958 +164959 POINT(40.80152634872191 74.26178948490659) bank164959 +164960 POINT(40.729629914878515 74.0799500627557) bank164960 +164961 POINT(39.910882035198625 74.11589233525012) bank164961 +164962 POINT(40.293981429743745 73.23272333788786) bank164962 +164963 POINT(41.66032535221509 73.29362418196277) bank164963 +164964 POINT(39.7768712199065 73.45275865480714) bank164964 +164965 POINT(40.093995844304686 74.7760618881621) bank164965 +164966 POINT(41.36593171532362 74.51736887273367) bank164966 +164967 POINT(40.592918842545494 73.48775026418207) bank164967 +164968 POINT(39.78374704556422 74.2138092731278) bank164968 +164969 POINT(41.53750285711028 74.13437142522733) bank164969 +164970 POINT(40.156688373144405 74.76801149326084) bank164970 +164971 POINT(39.713849940164636 74.36714880593472) bank164971 +164972 POINT(41.06491590944838 73.99460333365431) bank164972 +164973 POINT(40.543075478112065 74.48597032634383) bank164973 +164974 POINT(39.781219853692114 74.75240773854051) bank164974 +164975 POINT(39.923762146595166 73.5197642170387) bank164975 +164976 POINT(40.826944589631616 73.0735566399458) bank164976 +164977 POINT(40.65859284476919 73.36741544215462) bank164977 +164978 POINT(40.775879982066805 74.6851545693665) bank164978 +164979 POINT(41.09815883983975 74.9612180358683) bank164979 +164980 POINT(41.62151219440443 74.26073364458951) bank164980 +164981 POINT(40.56036385404842 73.78772222006695) bank164981 +164982 POINT(40.882585180910404 74.15611486083365) bank164982 +164983 POINT(40.24873121172314 73.14288401735207) bank164983 +164984 POINT(41.579889488008575 74.32659367527017) bank164984 +164985 POINT(40.6484031410125 74.68624604394905) bank164985 +164986 POINT(41.06634034237928 74.11072974436807) bank164986 +164987 POINT(40.77854784858716 73.52905479660612) bank164987 +164988 POINT(41.63287936005039 73.81559887799423) bank164988 +164989 POINT(40.54737530235111 74.01235974784142) bank164989 +164990 POINT(40.295108631648624 73.22448114399208) bank164990 +164991 POINT(40.95175103275927 73.5616662095666) bank164991 +164992 POINT(41.31298260762737 73.30632339010404) bank164992 +164993 POINT(41.43135781123961 74.18068259908787) bank164993 +164994 POINT(40.59203990556927 74.50420430577769) bank164994 +164995 POINT(41.15194781978906 73.10282233226394) bank164995 +164996 POINT(40.598971669957265 73.05377598612984) bank164996 +164997 POINT(39.80059506299516 73.84421951654994) bank164997 +164998 POINT(41.58253337569598 74.99546372919404) bank164998 +164999 POINT(40.0570803294084 74.3183425089913) bank164999 +165000 POINT(40.28484160778585 73.79816652820655) bank165000 +165001 POINT(40.5768059126589 74.71716374666116) bank165001 +165002 POINT(41.499299639895916 73.60007934952333) bank165002 +165003 POINT(40.12564866888941 73.17701762571423) bank165003 +165004 POINT(41.45811099654957 73.0688522670707) bank165004 +165005 POINT(41.70165453122481 73.25244174320973) bank165005 +165006 POINT(40.93783133556458 74.24356553774628) bank165006 +165007 POINT(40.535972934110646 73.5389136898651) bank165007 +165008 POINT(41.6494939559201 73.28932994903374) bank165008 +165009 POINT(41.30112208466524 73.98231549480998) bank165009 +165010 POINT(41.211035658730594 73.5859810769963) bank165010 +165011 POINT(39.831696700800606 73.46335465480092) bank165011 +165012 POINT(41.568934509350846 74.05678798851682) bank165012 +165013 POINT(40.40011316483577 74.95268664610452) bank165013 +165014 POINT(41.02847518591621 73.39161045004886) bank165014 +165015 POINT(40.95884985805111 73.09514731712527) bank165015 +165016 POINT(40.36605792161254 74.17594606988857) bank165016 +165017 POINT(40.528766287948365 74.79157811692012) bank165017 +165018 POINT(40.95465578130864 73.01375691856849) bank165018 +165019 POINT(39.94887916788987 74.42622783284914) bank165019 +165020 POINT(40.694810831362936 73.46066092761038) bank165020 +165021 POINT(40.59165710401916 74.85016053442196) bank165021 +165022 POINT(41.69290422611778 73.63566731047312) bank165022 +165023 POINT(39.89546596254006 73.52432431490793) bank165023 +165024 POINT(39.86531551442822 73.80843388163821) bank165024 +165025 POINT(41.54587144579255 73.61672830636188) bank165025 +165026 POINT(39.88663693024038 74.48924073504242) bank165026 +165027 POINT(40.812770644013625 74.60548423154935) bank165027 +165028 POINT(39.849797181612544 73.17205133040179) bank165028 +165029 POINT(41.69000879865663 74.90569982125851) bank165029 +165030 POINT(40.76709700189285 74.91016948785396) bank165030 +165031 POINT(41.53418325492378 73.68651633503066) bank165031 +165032 POINT(39.856989611634525 73.15599986086471) bank165032 +165033 POINT(41.51257580274712 74.3938717596443) bank165033 +165034 POINT(41.54585942912886 73.82602990321749) bank165034 +165035 POINT(39.98701255815748 73.59907643081263) bank165035 +165036 POINT(40.946140247533606 73.44965584398253) bank165036 +165037 POINT(41.527645051886545 73.78874388901451) bank165037 +165038 POINT(40.351609897633544 73.78193108355264) bank165038 +165039 POINT(40.40777408162481 73.2017137515136) bank165039 +165040 POINT(41.255141719563206 74.4039619262041) bank165040 +165041 POINT(41.114052340599386 73.64137995279498) bank165041 +165042 POINT(39.99276355561006 73.601210544672) bank165042 +165043 POINT(40.56339740014961 73.36220887511531) bank165043 +165044 POINT(40.21133031815961 74.07655203542721) bank165044 +165045 POINT(40.62589994190587 74.65867852747954) bank165045 +165046 POINT(40.10697455332319 73.3181340127706) bank165046 +165047 POINT(40.317842773055254 73.29252711254631) bank165047 +165048 POINT(40.63653107603156 73.41581304841043) bank165048 +165049 POINT(39.9689993688228 74.26844286875631) bank165049 +165050 POINT(40.786343143763965 73.31268983368028) bank165050 +165051 POINT(41.67666408721184 73.2711796047959) bank165051 +165052 POINT(39.75366858991104 73.97221489468276) bank165052 +165053 POINT(40.54634346850083 74.45699386809349) bank165053 +165054 POINT(40.59377676560046 74.39708231855292) bank165054 +165055 POINT(40.7894783988681 74.3789792737103) bank165055 +165056 POINT(39.92813241078789 73.48460133004382) bank165056 +165057 POINT(39.75337089395564 73.7248823777748) bank165057 +165058 POINT(40.01863662413289 74.92032126826261) bank165058 +165059 POINT(40.94149638697101 74.33889625663502) bank165059 +165060 POINT(39.72608642254046 74.93786615167492) bank165060 +165061 POINT(40.50168110850818 73.0484335940772) bank165061 +165062 POINT(41.46972526474837 74.92980072370861) bank165062 +165063 POINT(40.145678621722425 73.78883463051794) bank165063 +165064 POINT(40.0712491612651 73.07745680119393) bank165064 +165065 POINT(40.83209084676919 74.14933521823072) bank165065 +165066 POINT(41.13978042925021 74.31070228865039) bank165066 +165067 POINT(40.268158799691655 73.21679415964562) bank165067 +165068 POINT(40.30516009234234 74.13521675762082) bank165068 +165069 POINT(40.45093662166586 73.18982942544753) bank165069 +165070 POINT(41.64750065819613 74.02489840956865) bank165070 +165071 POINT(41.55823725222136 74.7811137390655) bank165071 +165072 POINT(39.87522972087066 73.25008026261533) bank165072 +165073 POINT(40.276492581634294 73.34819321782474) bank165073 +165074 POINT(41.30387732668118 74.74234878320925) bank165074 +165075 POINT(41.38813826927408 73.83834242252051) bank165075 +165076 POINT(41.455423025258284 74.19262251715172) bank165076 +165077 POINT(40.2813193977058 74.6347600637214) bank165077 +165078 POINT(41.70233596014965 73.47241812829489) bank165078 +165079 POINT(41.581351681287146 74.4035678179782) bank165079 +165080 POINT(40.207590848619226 74.37082400288946) bank165080 +165081 POINT(40.07807573149232 73.4989612703733) bank165081 +165082 POINT(40.076032910153245 74.51052103620336) bank165082 +165083 POINT(41.542595672665705 73.29583064099894) bank165083 +165084 POINT(40.654805696452144 73.50007949951173) bank165084 +165085 POINT(41.25380961054172 74.39300629634569) bank165085 +165086 POINT(41.542359384939026 74.05985404407208) bank165086 +165087 POINT(40.56432337125793 73.65956342535264) bank165087 +165088 POINT(41.267857276711574 73.39693967789502) bank165088 +165089 POINT(40.7936071413939 73.04335922379211) bank165089 +165090 POINT(39.92871388187698 73.28545187658197) bank165090 +165091 POINT(41.30073959513536 74.40475326221922) bank165091 +165092 POINT(40.10236287352647 73.13005452752041) bank165092 +165093 POINT(39.78602532045619 74.18961556532219) bank165093 +165094 POINT(41.38712386970465 73.68438727521063) bank165094 +165095 POINT(39.980590982514144 73.10935822165142) bank165095 +165096 POINT(41.53666026593224 74.61588484569873) bank165096 +165097 POINT(40.709946802503666 74.87950438357292) bank165097 +165098 POINT(41.549785717014935 73.11828534909478) bank165098 +165099 POINT(40.964443368614255 73.89559333260796) bank165099 +165100 POINT(41.549437754671644 73.86855881187185) bank165100 +165101 POINT(41.47192608703627 74.44821036735105) bank165101 +165102 POINT(40.15754604357991 73.96339188298812) bank165102 +165103 POINT(40.384314861014126 73.68028132796361) bank165103 +165104 POINT(40.72643919528859 74.33533278482574) bank165104 +165105 POINT(41.677021194147436 73.55269754485917) bank165105 +165106 POINT(41.37703026453283 74.90623904810745) bank165106 +165107 POINT(40.163782230512325 73.60144788298886) bank165107 +165108 POINT(41.66282349670291 73.90545499123715) bank165108 +165109 POINT(40.04040868563735 74.17899825148407) bank165109 +165110 POINT(40.80816864337844 73.65245153181608) bank165110 +165111 POINT(41.116730810732086 73.64887168853754) bank165111 +165112 POINT(40.39514248253714 73.11295591753287) bank165112 +165113 POINT(40.65128701860984 73.68943036646259) bank165113 +165114 POINT(40.97738382396228 74.73868288877782) bank165114 +165115 POINT(41.18439539807991 73.62781297044025) bank165115 +165116 POINT(41.41284959826228 73.1698641368386) bank165116 +165117 POINT(39.95574570238221 74.57865497110886) bank165117 +165118 POINT(39.912330349403355 74.10486337986985) bank165118 +165119 POINT(40.31545271460756 73.18069496842982) bank165119 +165120 POINT(41.27699053614945 74.11020423803905) bank165120 +165121 POINT(41.05669052409375 74.8157580282173) bank165121 +165122 POINT(41.309689711686126 73.76891863292639) bank165122 +165123 POINT(40.552198377218645 74.24039414766345) bank165123 +165124 POINT(41.077333245458696 74.8369506186059) bank165124 +165125 POINT(40.67609253039095 73.8770190324715) bank165125 +165126 POINT(40.71152806195885 74.22196522798247) bank165126 +165127 POINT(40.71747157851073 74.35745260931313) bank165127 +165128 POINT(40.74855404601643 74.27715780778533) bank165128 +165129 POINT(40.39107281175791 73.00781146080725) bank165129 +165130 POINT(41.666955390893676 74.08325274433638) bank165130 +165131 POINT(41.63785045887542 74.31612437848226) bank165131 +165132 POINT(41.256477469393076 74.65505933953699) bank165132 +165133 POINT(39.75601201502217 73.5722010514941) bank165133 +165134 POINT(41.098549414755254 73.04535753670982) bank165134 +165135 POINT(40.398988992138634 73.53605473561318) bank165135 +165136 POINT(39.75716156564176 73.9338694842755) bank165136 +165137 POINT(40.4712783964658 73.12416401916155) bank165137 +165138 POINT(40.53050615917008 73.87754230583143) bank165138 +165139 POINT(40.732847838486634 74.26587102995757) bank165139 +165140 POINT(40.63510456211349 73.32320780622197) bank165140 +165141 POINT(39.95015323729191 74.74725600396468) bank165141 +165142 POINT(41.35213353815312 74.55221784322904) bank165142 +165143 POINT(41.5556749597727 73.25637934199187) bank165143 +165144 POINT(40.10082150190248 74.63889597635182) bank165144 +165145 POINT(40.82352037972209 74.38188584206824) bank165145 +165146 POINT(40.829087181235145 73.13431496648792) bank165146 +165147 POINT(40.61154278421909 73.99039432171136) bank165147 +165148 POINT(39.970755934952614 74.85072361080877) bank165148 +165149 POINT(41.36425409054485 74.08473571741239) bank165149 +165150 POINT(40.878945431952694 73.04055881771883) bank165150 +165151 POINT(41.3076313135844 73.72635936033805) bank165151 +165152 POINT(40.38312805474871 74.8508578368789) bank165152 +165153 POINT(41.478253815855176 74.87635624346244) bank165153 +165154 POINT(40.105862517868346 74.00222508238932) bank165154 +165155 POINT(41.50527175218634 73.50594684185087) bank165155 +165156 POINT(40.161439381561344 73.41832463709831) bank165156 +165157 POINT(40.163518189671 74.65600330052204) bank165157 +165158 POINT(41.54828121063002 73.18658797102903) bank165158 +165159 POINT(40.63972509937912 74.63582381201249) bank165159 +165160 POINT(41.598281466437015 74.56936988823925) bank165160 +165161 POINT(41.6263103613864 74.76237375442815) bank165161 +165162 POINT(41.00754312286793 74.34806263955913) bank165162 +165163 POINT(40.28366545162293 74.25095588719189) bank165163 +165164 POINT(41.11211040838961 73.46269269214154) bank165164 +165165 POINT(41.12806249589645 73.99298802157402) bank165165 +165166 POINT(40.33676742083545 73.68025721059584) bank165166 +165167 POINT(40.79657637006075 73.84881565897929) bank165167 +165168 POINT(40.50037426441528 73.1458444383541) bank165168 +165169 POINT(40.22749795661948 73.4088672254674) bank165169 +165170 POINT(41.37218749535726 74.39625808919074) bank165170 +165171 POINT(40.92113093506996 73.2395616241289) bank165171 +165172 POINT(41.390548598198244 74.09361468403243) bank165172 +165173 POINT(40.40858227074458 74.11479714978105) bank165173 +165174 POINT(40.825245065714256 75.0044440917135) bank165174 +165175 POINT(40.491729815405286 73.0527564672667) bank165175 +165176 POINT(41.227674927972046 73.53373181411476) bank165176 +165177 POINT(40.148291622247775 74.7728888807757) bank165177 +165178 POINT(41.289916892708185 74.03537064369692) bank165178 +165179 POINT(39.89430607188839 73.45552599982928) bank165179 +165180 POINT(41.078307662712696 73.52753796680122) bank165180 +165181 POINT(40.26774843280252 74.29742527292196) bank165181 +165182 POINT(39.81488699927265 73.64872386196288) bank165182 +165183 POINT(41.6277282244764 73.16352354331931) bank165183 +165184 POINT(40.36049749126323 74.77433283366818) bank165184 +165185 POINT(40.598005149380626 74.25539340691283) bank165185 +165186 POINT(40.595873253056176 73.05503185295657) bank165186 +165187 POINT(41.50147439254681 74.70008692895621) bank165187 +165188 POINT(40.58884813253319 73.27037112179009) bank165188 +165189 POINT(41.46528710288666 74.90814619817013) bank165189 +165190 POINT(41.29942629585851 74.89224306500056) bank165190 +165191 POINT(40.5724075107221 74.83293134872574) bank165191 +165192 POINT(40.19288385524666 73.315741906202) bank165192 +165193 POINT(40.214767624328736 74.5924457944638) bank165193 +165194 POINT(40.874247691866564 73.1635235362635) bank165194 +165195 POINT(41.30496648921633 73.99927766443815) bank165195 +165196 POINT(40.10000308204744 73.45082196542752) bank165196 +165197 POINT(40.7438266611141 74.66143321126751) bank165197 +165198 POINT(41.60644643086159 73.04915360556758) bank165198 +165199 POINT(40.486441481880114 74.98608319514707) bank165199 +165200 POINT(40.71719837012661 74.09389140657119) bank165200 +165201 POINT(41.03654268406557 74.01638069988849) bank165201 +165202 POINT(40.48542985388618 73.17738224110845) bank165202 +165203 POINT(39.78529495315344 74.82148190228801) bank165203 +165204 POINT(41.16022350050595 74.23965888649624) bank165204 +165205 POINT(40.88639663145572 74.93814668659743) bank165205 +165206 POINT(40.9535046211039 74.964251963594) bank165206 +165207 POINT(41.43179336079849 74.86866607391228) bank165207 +165208 POINT(41.13759266791303 74.87343023830469) bank165208 +165209 POINT(40.44996918167307 73.33119433706324) bank165209 +165210 POINT(40.2816684660268 74.83596770706225) bank165210 +165211 POINT(41.65096249704948 73.83895966450804) bank165211 +165212 POINT(41.489979739168035 74.39963612033625) bank165212 +165213 POINT(40.47460348353855 74.28011407862878) bank165213 +165214 POINT(39.91361270896531 73.62504733652084) bank165214 +165215 POINT(41.08971043021358 74.5544943463723) bank165215 +165216 POINT(40.0007860040367 74.16396761595762) bank165216 +165217 POINT(41.52560269398189 73.27573621414732) bank165217 +165218 POINT(41.13506304341766 74.53970499156983) bank165218 +165219 POINT(40.343818061337004 74.08804214447753) bank165219 +165220 POINT(39.993229429128675 74.36413859763661) bank165220 +165221 POINT(40.52589707744721 74.84368709126211) bank165221 +165222 POINT(41.33598320924055 74.80949671640333) bank165222 +165223 POINT(40.195919018310825 74.61376943463745) bank165223 +165224 POINT(40.6578121790625 74.49012329170968) bank165224 +165225 POINT(39.77479362211032 74.43038006700027) bank165225 +165226 POINT(39.74997607638395 73.76568579066314) bank165226 +165227 POINT(41.281508716422806 73.34407743961151) bank165227 +165228 POINT(40.12239234391302 74.51470300530119) bank165228 +165229 POINT(40.26892002802027 74.93231761638675) bank165229 +165230 POINT(40.313585647635705 73.15788670419698) bank165230 +165231 POINT(39.740040725625676 74.12564932421829) bank165231 +165232 POINT(41.5688245559726 73.08526297848731) bank165232 +165233 POINT(40.75546088504242 73.16611488209426) bank165233 +165234 POINT(40.80765871705933 73.93785674710449) bank165234 +165235 POINT(41.363327428753394 73.12754062875096) bank165235 +165236 POINT(40.58768203333905 73.81706754259393) bank165236 +165237 POINT(40.50435477275047 73.55675663103254) bank165237 +165238 POINT(40.47040841557599 74.53440237727959) bank165238 +165239 POINT(39.91666338745545 74.2608070201902) bank165239 +165240 POINT(40.70403458122756 74.26759643826801) bank165240 +165241 POINT(40.641154578474044 74.79749995640184) bank165241 +165242 POINT(41.70961572313461 73.84206900676064) bank165242 +165243 POINT(40.277843434573704 73.04040478126402) bank165243 +165244 POINT(40.74569148278543 73.37405657962442) bank165244 +165245 POINT(40.76558798772562 74.57304606997917) bank165245 +165246 POINT(40.449699090025845 73.91733398249043) bank165246 +165247 POINT(40.33524721967242 74.92873656469095) bank165247 +165248 POINT(40.19024599538899 73.72601676371907) bank165248 +165249 POINT(41.46608329893293 73.78701308486222) bank165249 +165250 POINT(41.691446036118684 74.83959058819234) bank165250 +165251 POINT(40.715482382455335 73.35719799302264) bank165251 +165252 POINT(40.78128065629431 74.24453502021923) bank165252 +165253 POINT(39.99115240272829 73.61735318972013) bank165253 +165254 POINT(40.29705244535181 74.87714702590783) bank165254 +165255 POINT(41.6095152703712 74.78522296304561) bank165255 +165256 POINT(40.950858944076636 74.2862947598425) bank165256 +165257 POINT(39.992958514381904 73.71220496526449) bank165257 +165258 POINT(41.216967514042246 74.17935820392485) bank165258 +165259 POINT(40.64853072163931 73.42399640781039) bank165259 +165260 POINT(40.96929481499633 73.21346709349842) bank165260 +165261 POINT(41.65308492503974 73.07106066521688) bank165261 +165262 POINT(40.02170942483805 73.98634216068727) bank165262 +165263 POINT(40.13076711695307 73.39208666366666) bank165263 +165264 POINT(41.01843537085959 73.21740277963586) bank165264 +165265 POINT(39.9235674458209 74.40573279776186) bank165265 +165266 POINT(40.2188224346556 73.21812975458073) bank165266 +165267 POINT(40.18753597249965 73.77482118843896) bank165267 +165268 POINT(40.03009948689439 73.59385792448953) bank165268 +165269 POINT(40.843899546895976 73.40553944820276) bank165269 +165270 POINT(41.02681007193102 74.8359342904872) bank165270 +165271 POINT(39.74506561138512 74.96965417016237) bank165271 +165272 POINT(40.72518784190825 74.44915876883766) bank165272 +165273 POINT(40.47251420560149 74.09984635684145) bank165273 +165274 POINT(40.85282448746194 73.22185066390487) bank165274 +165275 POINT(40.21159750994894 73.54973970274445) bank165275 +165276 POINT(39.85064328731601 73.67161915752624) bank165276 +165277 POINT(40.564668190634826 73.88576933037265) bank165277 +165278 POINT(41.50135596013763 74.9278658666022) bank165278 +165279 POINT(39.96432472316113 74.57220684782504) bank165279 +165280 POINT(40.02060408788487 73.73286690952902) bank165280 +165281 POINT(41.171876307459094 74.14715478821394) bank165281 +165282 POINT(41.29586867396416 74.5548414906432) bank165282 +165283 POINT(40.9264080315877 74.574734336005) bank165283 +165284 POINT(41.08351261770451 74.6989785662639) bank165284 +165285 POINT(41.38581302971826 73.82895181992625) bank165285 +165286 POINT(40.39068811210289 74.36677708620262) bank165286 +165287 POINT(40.605278670560416 73.05850212783815) bank165287 +165288 POINT(40.23232479975527 74.48137185303088) bank165288 +165289 POINT(40.838230285767374 74.66427901025156) bank165289 +165290 POINT(41.109451840763306 74.83566214007647) bank165290 +165291 POINT(40.589706955403884 73.51950876980665) bank165291 +165292 POINT(41.08721650234124 73.87491679800209) bank165292 +165293 POINT(40.21390125860144 73.57383281111761) bank165293 +165294 POINT(40.55538748371558 73.55521297408565) bank165294 +165295 POINT(40.3841508608848 74.06045549228989) bank165295 +165296 POINT(40.58702484709235 74.75820977516777) bank165296 +165297 POINT(41.32785338895863 73.99953496032327) bank165297 +165298 POINT(39.96230263625601 73.85034180453435) bank165298 +165299 POINT(40.209114490583765 73.12421346927808) bank165299 +165300 POINT(40.96835974737651 74.0570888619274) bank165300 +165301 POINT(41.610370265394074 73.1820970576825) bank165301 +165302 POINT(40.40356939773504 74.37062591773362) bank165302 +165303 POINT(40.926736664949736 74.87095940799581) bank165303 +165304 POINT(40.84010417302968 73.81621873884777) bank165304 +165305 POINT(40.39480302669905 74.08165047339492) bank165305 +165306 POINT(41.006729707467315 73.13747750851628) bank165306 +165307 POINT(41.63571535626875 73.89856264135005) bank165307 +165308 POINT(40.46313682229469 73.4364223738223) bank165308 +165309 POINT(39.93213913154606 73.7733285421487) bank165309 +165310 POINT(40.086382108576274 74.83398675636845) bank165310 +165311 POINT(41.697190902567165 74.74215695843674) bank165311 +165312 POINT(40.83414000620772 73.92577949818786) bank165312 +165313 POINT(41.24716509280848 74.1827879436867) bank165313 +165314 POINT(40.05195885754731 74.40398610259803) bank165314 +165315 POINT(40.169881383803606 74.03538553523961) bank165315 +165316 POINT(39.80903539087743 74.48058878843518) bank165316 +165317 POINT(41.64929866450275 74.54752205321502) bank165317 +165318 POINT(40.17407435217162 74.48339495732036) bank165318 +165319 POINT(40.003602835990044 73.074913630253) bank165319 +165320 POINT(41.026148975740014 73.24434472490219) bank165320 +165321 POINT(40.5603684331538 73.78828528241401) bank165321 +165322 POINT(40.95039027329979 73.74585530754264) bank165322 +165323 POINT(41.24181956841523 73.1895201787529) bank165323 +165324 POINT(39.844675899956705 73.91059289429717) bank165324 +165325 POINT(39.954093305138876 74.1701375861381) bank165325 +165326 POINT(40.063858578479255 74.77473800058189) bank165326 +165327 POINT(41.15426403412439 73.33770485433433) bank165327 +165328 POINT(39.92229638012781 74.71986519938409) bank165328 +165329 POINT(39.90320050894264 73.87227627894045) bank165329 +165330 POINT(40.6098835594689 74.02330572467461) bank165330 +165331 POINT(40.12060528753857 74.44676594946465) bank165331 +165332 POINT(40.219003459364615 74.25746550070674) bank165332 +165333 POINT(41.238597812404464 74.19471147677484) bank165333 +165334 POINT(40.95841415695042 73.80590046869996) bank165334 +165335 POINT(39.88761965641054 73.66697822514142) bank165335 +165336 POINT(41.69768580730876 74.3113347189471) bank165336 +165337 POINT(41.186866781384445 73.14499149963633) bank165337 +165338 POINT(40.74358437979842 74.4211271879076) bank165338 +165339 POINT(41.56940182739402 73.44165109255299) bank165339 +165340 POINT(41.245540754955684 73.03476478150658) bank165340 +165341 POINT(41.078603271154726 73.89902282860542) bank165341 +165342 POINT(40.73732140499957 73.72995038598599) bank165342 +165343 POINT(40.57438517487017 74.96527579593135) bank165343 +165344 POINT(40.150674121799575 74.37188832960591) bank165344 +165345 POINT(40.69240321062642 73.69137551716204) bank165345 +165346 POINT(40.056295442084945 74.72439591845036) bank165346 +165347 POINT(41.47338020503937 74.53689998062069) bank165347 +165348 POINT(40.91540924126353 74.11152402285732) bank165348 +165349 POINT(40.78369193414702 74.4244490532298) bank165349 +165350 POINT(41.12027210847742 74.8647245399526) bank165350 +165351 POINT(41.150494540826436 73.69206581080323) bank165351 +165352 POINT(41.373929846536576 73.2359324561086) bank165352 +165353 POINT(41.32921154009388 74.31306161703027) bank165353 +165354 POINT(39.80571835519647 73.21472031318999) bank165354 +165355 POINT(41.7011231597084 74.09327419345952) bank165355 +165356 POINT(40.14334801783402 74.82857799890769) bank165356 +165357 POINT(41.68336457882414 74.10254407791908) bank165357 +165358 POINT(41.158593993947555 74.3043261056869) bank165358 +165359 POINT(41.52737869020137 73.59005564244416) bank165359 +165360 POINT(40.41214491126469 73.01287684908625) bank165360 +165361 POINT(39.9107796979211 73.13335844013221) bank165361 +165362 POINT(41.443231354357714 74.63739216721164) bank165362 +165363 POINT(41.646693932557916 74.06120851379536) bank165363 +165364 POINT(41.4514824999239 73.50299792306019) bank165364 +165365 POINT(40.44442946175525 74.09657222015596) bank165365 +165366 POINT(40.348583272063806 73.22160527788185) bank165366 +165367 POINT(40.85956742905035 73.45167910564274) bank165367 +165368 POINT(41.42255664837611 74.33495892521121) bank165368 +165369 POINT(40.722253032918836 73.97491634059455) bank165369 +165370 POINT(41.52014839673946 73.82253164677674) bank165370 +165371 POINT(40.09770266482497 74.54127227941777) bank165371 +165372 POINT(39.99315655592105 74.22410046872317) bank165372 +165373 POINT(39.97006135221484 73.756284932905) bank165373 +165374 POINT(40.488588769291304 73.75235638044656) bank165374 +165375 POINT(40.426837923301235 74.68042627139519) bank165375 +165376 POINT(40.29262000373753 73.32951017451128) bank165376 +165377 POINT(40.87048589808807 74.33370096487957) bank165377 +165378 POINT(41.37919988109406 73.14194984898425) bank165378 +165379 POINT(41.46145630856654 73.57517819070512) bank165379 +165380 POINT(41.27105778423266 73.38781120743063) bank165380 +165381 POINT(40.106754533882615 74.1795652398442) bank165381 +165382 POINT(41.009144130859994 74.214330985146) bank165382 +165383 POINT(39.97743058445246 73.3477293451912) bank165383 +165384 POINT(41.63298932877679 74.40744921501606) bank165384 +165385 POINT(40.48230951235181 74.15824166384944) bank165385 +165386 POINT(39.74806809193963 73.80355734653976) bank165386 +165387 POINT(40.619105417745324 74.41847698808171) bank165387 +165388 POINT(41.59537879419355 73.9181326286128) bank165388 +165389 POINT(39.90644193139222 74.14836273500933) bank165389 +165390 POINT(40.69494509218854 73.30293480357243) bank165390 +165391 POINT(40.08557153476493 73.48257311617509) bank165391 +165392 POINT(40.0156101321205 74.55424083805735) bank165392 +165393 POINT(40.0781969948685 74.21236561089631) bank165393 +165394 POINT(40.32244895558231 74.61525119497202) bank165394 +165395 POINT(40.29439505077451 74.5179664048127) bank165395 +165396 POINT(40.037437701035266 73.41660805539188) bank165396 +165397 POINT(40.875178129287434 73.98551127813508) bank165397 +165398 POINT(39.77817033381738 73.09043705593088) bank165398 +165399 POINT(41.173345341605554 74.47917306191363) bank165399 +165400 POINT(39.86253697811027 74.17845245059972) bank165400 +165401 POINT(40.02932707220279 74.08825104653386) bank165401 +165402 POINT(39.99990244972649 73.75834268699903) bank165402 +165403 POINT(41.694380752508145 74.21732450361351) bank165403 +165404 POINT(40.920451846155615 73.14614078709967) bank165404 +165405 POINT(41.0646136129801 74.55247490517735) bank165405 +165406 POINT(40.13005090820226 73.01086688142188) bank165406 +165407 POINT(41.133658066677704 74.9081616480072) bank165407 +165408 POINT(41.386918431780124 74.12851577509917) bank165408 +165409 POINT(41.32465646808244 73.81937007247092) bank165409 +165410 POINT(39.89491533584202 73.68148739364004) bank165410 +165411 POINT(40.951664298914416 73.49232188075322) bank165411 +165412 POINT(39.7136461741792 74.69267848966545) bank165412 +165413 POINT(40.52058046242006 73.82582700321139) bank165413 +165414 POINT(40.95346700006862 74.00203019419202) bank165414 +165415 POINT(41.42257332506088 73.55967442235698) bank165415 +165416 POINT(41.20374880707925 74.15553522604384) bank165416 +165417 POINT(39.961613021620956 74.79747612116797) bank165417 +165418 POINT(41.62927611227152 74.09490233073869) bank165418 +165419 POINT(40.05575567202532 73.75377409464163) bank165419 +165420 POINT(40.63027487298057 73.61474906376091) bank165420 +165421 POINT(41.106018038896565 73.84835629412753) bank165421 +165422 POINT(40.810521047212006 74.01165101507065) bank165422 +165423 POINT(39.87185666855402 74.74716830462494) bank165423 +165424 POINT(40.31210884593627 73.22535919762055) bank165424 +165425 POINT(41.01383608826266 74.37181089083914) bank165425 +165426 POINT(40.40562507802603 74.23964268026135) bank165426 +165427 POINT(40.08267065579163 74.3285395124926) bank165427 +165428 POINT(40.63911379019033 74.39840113831319) bank165428 +165429 POINT(40.64961848329684 73.80586566300516) bank165429 +165430 POINT(41.60176169489955 73.37433905107531) bank165430 +165431 POINT(40.848115278113895 73.54187589181971) bank165431 +165432 POINT(41.23341097561126 73.48925298557194) bank165432 +165433 POINT(40.84745324042686 73.86619403313667) bank165433 +165434 POINT(41.66183286908293 74.15814259680928) bank165434 +165435 POINT(41.524726411369116 73.4539742180827) bank165435 +165436 POINT(39.94049547788329 74.60833293019569) bank165436 +165437 POINT(40.17755632807247 73.43652959828397) bank165437 +165438 POINT(40.244221035396485 73.85999554398249) bank165438 +165439 POINT(40.62974982133752 73.86390503335058) bank165439 +165440 POINT(40.93672356424363 73.43447270512564) bank165440 +165441 POINT(40.80236587332172 74.29571357123282) bank165441 +165442 POINT(40.57215308682534 74.6552348361036) bank165442 +165443 POINT(40.5447522260346 73.8461320504071) bank165443 +165444 POINT(41.6865806699559 74.90538131182292) bank165444 +165445 POINT(41.31278486839038 73.40893988871603) bank165445 +165446 POINT(40.06744622251198 74.38106347102092) bank165446 +165447 POINT(41.416048460127286 74.57891346169048) bank165447 +165448 POINT(41.600191924323425 73.75777222157441) bank165448 +165449 POINT(40.21216290007682 73.5003508797249) bank165449 +165450 POINT(39.785629208267174 73.45070970218265) bank165450 +165451 POINT(40.60341620798318 74.18082671315979) bank165451 +165452 POINT(40.85331740709941 74.24919502279818) bank165452 +165453 POINT(39.75305171585913 74.46016645676615) bank165453 +165454 POINT(40.32607486486077 73.81538753225784) bank165454 +165455 POINT(41.699613660915624 73.38454862617766) bank165455 +165456 POINT(39.937632070015674 74.60117140347822) bank165456 +165457 POINT(40.978512326331106 73.05575891593607) bank165457 +165458 POINT(40.37926520961423 73.24090741706607) bank165458 +165459 POINT(40.21590728519987 73.48379265510742) bank165459 +165460 POINT(40.80965657973521 73.44128581365241) bank165460 +165461 POINT(41.226067937248786 73.40111089627075) bank165461 +165462 POINT(39.843657902088 74.45577387310168) bank165462 +165463 POINT(40.16949190512717 73.98604711552323) bank165463 +165464 POINT(40.60656318432358 73.49878866982311) bank165464 +165465 POINT(40.23726482995568 74.95212437790953) bank165465 +165466 POINT(40.30737310118501 74.61209392476162) bank165466 +165467 POINT(41.12661284066778 74.58889375315373) bank165467 +165468 POINT(40.41301394219923 73.03148703836273) bank165468 +165469 POINT(40.691794226424946 74.20970763429048) bank165469 +165470 POINT(41.672020925777126 73.77896039159268) bank165470 +165471 POINT(41.477794009106475 73.27560482313557) bank165471 +165472 POINT(39.89155157756755 73.07983161316733) bank165472 +165473 POINT(41.072948526285735 73.74088257316441) bank165473 +165474 POINT(41.67195493104517 73.82496904344814) bank165474 +165475 POINT(40.240439992944744 74.91778798059588) bank165475 +165476 POINT(40.30626754952837 74.88248827773492) bank165476 +165477 POINT(39.7374373052896 73.83745490368716) bank165477 +165478 POINT(40.79176556394125 74.92083707510612) bank165478 +165479 POINT(40.72613934457154 73.52230703765542) bank165479 +165480 POINT(40.65109253691211 73.53748991707572) bank165480 +165481 POINT(40.25032438475998 73.57907451176538) bank165481 +165482 POINT(40.39337439291045 73.23883752025833) bank165482 +165483 POINT(40.7099510712564 73.22775717970731) bank165483 +165484 POINT(41.503443613041846 74.87014482499492) bank165484 +165485 POINT(39.76278613427879 73.70100070756675) bank165485 +165486 POINT(40.75782833403161 75.00424210630854) bank165486 +165487 POINT(41.14957971463858 73.5781897930044) bank165487 +165488 POINT(41.06665444524472 74.5964384772608) bank165488 +165489 POINT(41.43503983524913 74.31690098192047) bank165489 +165490 POINT(40.13034976804087 73.37116914844532) bank165490 +165491 POINT(39.98211725632269 73.77217578104377) bank165491 +165492 POINT(41.010891270110555 74.62098576309424) bank165492 +165493 POINT(39.90656609300787 73.86773591312023) bank165493 +165494 POINT(40.984259121760516 73.72590998451011) bank165494 +165495 POINT(41.44526130750419 73.39197233966739) bank165495 +165496 POINT(41.01976758241394 73.72924870636119) bank165496 +165497 POINT(41.07381286443286 74.40409057618074) bank165497 +165498 POINT(41.456554689231595 73.41715888107629) bank165498 +165499 POINT(41.67886793339042 73.45649090780834) bank165499 +165500 POINT(39.98270221895564 73.35303381148462) bank165500 +165501 POINT(40.11803127231094 74.98690816730283) bank165501 +165502 POINT(40.91828880752921 73.09075929691737) bank165502 +165503 POINT(40.83996911827829 74.9898122022816) bank165503 +165504 POINT(40.4889154703828 74.64600154688975) bank165504 +165505 POINT(39.82133391804673 73.11104075966966) bank165505 +165506 POINT(40.53173375156021 74.46833898844615) bank165506 +165507 POINT(41.701716313549994 73.37666042878845) bank165507 +165508 POINT(40.715464292073996 73.9301272434718) bank165508 +165509 POINT(41.504855078646344 74.82882318128085) bank165509 +165510 POINT(40.978913163102405 74.46013927097381) bank165510 +165511 POINT(41.007998164453014 73.28847276714492) bank165511 +165512 POINT(39.94717655580609 74.00672376557982) bank165512 +165513 POINT(40.8713130727572 74.98275418805028) bank165513 +165514 POINT(40.094368544261386 73.14639090164943) bank165514 +165515 POINT(41.071827280267335 73.98644939894503) bank165515 +165516 POINT(39.85935966169525 73.48789131499696) bank165516 +165517 POINT(41.625142022901606 74.58973324918809) bank165517 +165518 POINT(40.28764231565966 74.33947430141143) bank165518 +165519 POINT(40.901028117468535 73.89170615069746) bank165519 +165520 POINT(41.38684913963732 73.16381822159181) bank165520 +165521 POINT(41.41086406131542 73.77778498422423) bank165521 +165522 POINT(40.980806246059466 74.35273785140956) bank165522 +165523 POINT(40.80881579686725 73.48371387827527) bank165523 +165524 POINT(41.244456123549206 74.38540731696489) bank165524 +165525 POINT(40.244876458289234 73.2920867232826) bank165525 +165526 POINT(40.25036766134584 73.29310160900789) bank165526 +165527 POINT(40.763916748630265 74.41620399055707) bank165527 +165528 POINT(40.892758100730035 74.74949613293012) bank165528 +165529 POINT(39.77975229078215 74.59683677958765) bank165529 +165530 POINT(41.22279153151047 73.52380304082541) bank165530 +165531 POINT(40.846691234384224 73.25430591884536) bank165531 +165532 POINT(41.57389014875028 74.37002813394379) bank165532 +165533 POINT(40.766057125281975 75.00007105964177) bank165533 +165534 POINT(40.0444979718541 74.28178810136106) bank165534 +165535 POINT(41.14253719956287 73.79734208979518) bank165535 +165536 POINT(39.98476230114074 73.42372973747612) bank165536 +165537 POINT(41.21350296315421 74.56270485733764) bank165537 +165538 POINT(40.68863454868706 74.680960285247) bank165538 +165539 POINT(41.54877561638069 74.7621936331108) bank165539 +165540 POINT(40.869715933785244 74.46106953157378) bank165540 +165541 POINT(39.94785389135188 74.92838567887891) bank165541 +165542 POINT(40.83727203014226 73.23128327016649) bank165542 +165543 POINT(40.11263476912686 73.33347096141412) bank165543 +165544 POINT(40.206158769085064 73.59518813363682) bank165544 +165545 POINT(41.18459672286907 74.5529335120039) bank165545 +165546 POINT(41.269243298869746 74.44361583737674) bank165546 +165547 POINT(40.41179490578294 73.24265090418925) bank165547 +165548 POINT(40.875998087282476 73.70556741097505) bank165548 +165549 POINT(39.828839756874416 73.75627117402641) bank165549 +165550 POINT(41.38287460373029 73.0758839513188) bank165550 +165551 POINT(40.26783028510308 74.66984211919838) bank165551 +165552 POINT(40.18815890551065 74.03082319551002) bank165552 +165553 POINT(41.54551330470136 74.94430035975351) bank165553 +165554 POINT(40.83992343499141 73.16226861442685) bank165554 +165555 POINT(41.414385992462606 73.60430261346843) bank165555 +165556 POINT(40.69188327853282 73.52170151893333) bank165556 +165557 POINT(41.076155685252 73.84495126243063) bank165557 +165558 POINT(39.986683839092734 74.88853870029351) bank165558 +165559 POINT(40.71163426596438 74.88516143844478) bank165559 +165560 POINT(41.434006238074836 73.21745769466699) bank165560 +165561 POINT(39.84752939435842 73.86732613168255) bank165561 +165562 POINT(40.318880332462705 74.3422816923214) bank165562 +165563 POINT(39.97491383110288 74.65547414277022) bank165563 +165564 POINT(40.70011817695554 74.34855465701163) bank165564 +165565 POINT(41.21510767819959 73.1758725781474) bank165565 +165566 POINT(40.054261212519094 73.87653907341401) bank165566 +165567 POINT(40.82873100510163 74.46077489651057) bank165567 +165568 POINT(40.22969711238147 74.80512464145825) bank165568 +165569 POINT(40.179854322837116 74.36639016177666) bank165569 +165570 POINT(40.15042471053058 74.00763606527421) bank165570 +165571 POINT(41.257832131551176 73.82845725227085) bank165571 +165572 POINT(41.68048468309216 73.94032014320129) bank165572 +165573 POINT(39.92876346991447 74.32524596637111) bank165573 +165574 POINT(40.989484442802784 74.49032791415318) bank165574 +165575 POINT(41.62019657019362 74.85136106627566) bank165575 +165576 POINT(40.434077473170696 73.21299912532818) bank165576 +165577 POINT(39.724427157321465 73.05621688759194) bank165577 +165578 POINT(40.46701557409629 74.6985236545998) bank165578 +165579 POINT(41.13851914043265 74.29884895378474) bank165579 +165580 POINT(39.982551414864254 74.38031332185598) bank165580 +165581 POINT(41.55891545013666 73.01229491445906) bank165581 +165582 POINT(41.24274587626673 74.42070605680954) bank165582 +165583 POINT(40.02864855223504 73.63026206434166) bank165583 +165584 POINT(40.827018523226286 74.07961025486235) bank165584 +165585 POINT(40.883203249843 74.4754202787184) bank165585 +165586 POINT(40.34385564819619 73.79014450982369) bank165586 +165587 POINT(41.19095183010566 73.13036000813857) bank165587 +165588 POINT(40.331908050268105 74.1135089859336) bank165588 +165589 POINT(40.952855291825784 73.90833582848619) bank165589 +165590 POINT(40.22854901838446 73.28613448765746) bank165590 +165591 POINT(41.68269416583953 73.09105122512618) bank165591 +165592 POINT(41.504389934859006 73.38533096909437) bank165592 +165593 POINT(41.396094939690826 73.88915944983435) bank165593 +165594 POINT(40.222880985838174 74.33504127849135) bank165594 +165595 POINT(40.03985912476904 73.15552639966012) bank165595 +165596 POINT(41.074503874218436 74.57562609767743) bank165596 +165597 POINT(41.32018049318619 74.68052336059843) bank165597 +165598 POINT(40.791075230966186 74.27647434416897) bank165598 +165599 POINT(41.57607821215527 74.29810840526841) bank165599 +165600 POINT(41.29346144124055 73.70514597326518) bank165600 +165601 POINT(40.14065204829251 73.23814673347141) bank165601 +165602 POINT(40.47437412466157 74.1454843012924) bank165602 +165603 POINT(40.368432879070646 73.71457118402607) bank165603 +165604 POINT(39.917939128100976 73.9354635443662) bank165604 +165605 POINT(39.76704249888847 73.46515515701157) bank165605 +165606 POINT(40.42756048533359 74.96408562233876) bank165606 +165607 POINT(40.098298393265125 73.88205713573483) bank165607 +165608 POINT(39.928301064168245 74.78580843882503) bank165608 +165609 POINT(41.689796545085606 73.83204836592391) bank165609 +165610 POINT(40.82713566961506 73.87057825813113) bank165610 +165611 POINT(41.234790324485225 74.15623107512197) bank165611 +165612 POINT(41.15972093935754 74.11210957504288) bank165612 +165613 POINT(41.25460975790754 73.29054440258955) bank165613 +165614 POINT(40.52608080304209 73.97514190099024) bank165614 +165615 POINT(40.39685002390339 74.87142055106652) bank165615 +165616 POINT(40.790370564358966 74.25161712328374) bank165616 +165617 POINT(40.5035817812239 73.95582346975294) bank165617 +165618 POINT(40.72776162386482 73.23540506940266) bank165618 +165619 POINT(40.227338552605154 74.044961970614) bank165619 +165620 POINT(40.24918611571553 74.11309636976003) bank165620 +165621 POINT(41.037060480735946 74.01098090982872) bank165621 +165622 POINT(41.11706518366117 73.80943915401814) bank165622 +165623 POINT(40.25427720875979 73.38592348683163) bank165623 +165624 POINT(40.4145610386143 74.5665711652504) bank165624 +165625 POINT(40.33166318556966 74.15569923590238) bank165625 +165626 POINT(40.86927801006531 73.01848760261105) bank165626 +165627 POINT(40.222558207586346 73.74855928228338) bank165627 +165628 POINT(41.13957195107925 74.3228120704375) bank165628 +165629 POINT(40.99343518759069 74.69249487177945) bank165629 +165630 POINT(41.079657307182295 74.27314552890348) bank165630 +165631 POINT(40.68244818902896 74.8247859539171) bank165631 +165632 POINT(41.496850652664186 73.68648531927859) bank165632 +165633 POINT(40.24614447040041 74.47959852835855) bank165633 +165634 POINT(41.2358552859159 74.99310887191747) bank165634 +165635 POINT(39.86241012185289 74.38116362793552) bank165635 +165636 POINT(41.45003672549947 73.39857512029506) bank165636 +165637 POINT(41.343356591056036 74.63574840809079) bank165637 +165638 POINT(41.132085694380955 73.23447744309505) bank165638 +165639 POINT(40.8808378698243 73.87106487020405) bank165639 +165640 POINT(41.299825302002816 73.26778009726054) bank165640 +165641 POINT(41.04530750842328 73.1581873341256) bank165641 +165642 POINT(41.153235179124685 73.91411372872362) bank165642 +165643 POINT(41.46306813593619 73.91165711479647) bank165643 +165644 POINT(40.05114076708497 73.27571858981219) bank165644 +165645 POINT(40.71623286696842 73.14883232423284) bank165645 +165646 POINT(41.03844965812167 74.92055739997645) bank165646 +165647 POINT(40.27787624366766 73.8332068443218) bank165647 +165648 POINT(39.75064004659936 74.27218816945572) bank165648 +165649 POINT(39.775847947996716 73.99196356704509) bank165649 +165650 POINT(41.070973890736276 73.35248830924209) bank165650 +165651 POINT(40.10106475247078 73.50968593227832) bank165651 +165652 POINT(39.920450907762394 73.3262630341454) bank165652 +165653 POINT(40.869027375189134 73.36260066706772) bank165653 +165654 POINT(40.33978596611698 73.09321005927886) bank165654 +165655 POINT(40.21276694851246 73.51421057524452) bank165655 +165656 POINT(41.06745825606697 74.12733386826584) bank165656 +165657 POINT(40.41787925162315 73.76878807141712) bank165657 +165658 POINT(41.145911031805866 74.41262787769621) bank165658 +165659 POINT(40.639890480923384 75.00140006308739) bank165659 +165660 POINT(40.10428903089234 74.7549911052243) bank165660 +165661 POINT(40.359601902757824 74.14325254225915) bank165661 +165662 POINT(40.65810748603068 74.77838640766386) bank165662 +165663 POINT(41.1618086107318 73.53542874225917) bank165663 +165664 POINT(40.69519594898803 74.93393025205135) bank165664 +165665 POINT(40.246158787094686 73.69632831770058) bank165665 +165666 POINT(41.64058695623641 73.50318388383594) bank165666 +165667 POINT(41.38405313409958 73.38283535590926) bank165667 +165668 POINT(40.15148578951999 74.40261647407658) bank165668 +165669 POINT(40.66683667526121 73.39842311794129) bank165669 +165670 POINT(40.13037130440211 74.06918390256429) bank165670 +165671 POINT(41.10002226565511 74.95973022097252) bank165671 +165672 POINT(39.98402256976116 74.68997985431443) bank165672 +165673 POINT(40.15186361447831 74.2973064366779) bank165673 +165674 POINT(40.168522343160205 73.61818744419782) bank165674 +165675 POINT(40.93406839107114 74.16624563035883) bank165675 +165676 POINT(40.31034946574437 73.71683674512329) bank165676 +165677 POINT(39.790533295123616 74.59332088099414) bank165677 +165678 POINT(40.23560633230337 73.87800106883812) bank165678 +165679 POINT(39.844664518866814 73.12202185331226) bank165679 +165680 POINT(40.42763628722191 73.21464904702277) bank165680 +165681 POINT(41.50266699897242 74.5654969254746) bank165681 +165682 POINT(40.338421147990424 73.28467005762833) bank165682 +165683 POINT(40.44760549417664 73.2178956830158) bank165683 +165684 POINT(41.13744225469363 73.61168249190547) bank165684 +165685 POINT(41.63207182470732 74.28915877417026) bank165685 +165686 POINT(41.35611944746128 73.53225477559404) bank165686 +165687 POINT(40.23649754363822 73.48709252290067) bank165687 +165688 POINT(41.54999075033014 74.84485330886018) bank165688 +165689 POINT(41.381418658808414 74.36909174282877) bank165689 +165690 POINT(40.462991932520865 73.26354139195509) bank165690 +165691 POINT(40.53190687955466 74.83663333965177) bank165691 +165692 POINT(40.985331148359705 75.00480637489704) bank165692 +165693 POINT(40.85992797344435 74.99964348331737) bank165693 +165694 POINT(39.77954116152905 73.21680006011039) bank165694 +165695 POINT(40.44450102124291 74.7601731509693) bank165695 +165696 POINT(40.11711266486485 73.62217823519735) bank165696 +165697 POINT(41.45055800220864 73.7528167563588) bank165697 +165698 POINT(41.64457345692266 74.80161369196193) bank165698 +165699 POINT(39.96033602310626 74.91298347663923) bank165699 +165700 POINT(40.29746080087376 73.45607048399683) bank165700 +165701 POINT(41.20606559531129 74.55549681793096) bank165701 +165702 POINT(41.38396362027153 74.3439832825709) bank165702 +165703 POINT(40.1921510796038 74.29004187653719) bank165703 +165704 POINT(40.83933148252943 74.89994991133972) bank165704 +165705 POINT(41.034153213053294 73.7485040746983) bank165705 +165706 POINT(40.04067393534945 74.51636978380054) bank165706 +165707 POINT(40.50909622765311 74.6347635219505) bank165707 +165708 POINT(40.60892894589763 74.36723721368112) bank165708 +165709 POINT(41.49754873578654 74.48472179998522) bank165709 +165710 POINT(40.97228283494635 73.84227354864156) bank165710 +165711 POINT(39.92138861855021 73.45916757656191) bank165711 +165712 POINT(39.80794581348933 74.73269502822241) bank165712 +165713 POINT(40.686399732425514 74.60374550247555) bank165713 +165714 POINT(41.257304722381406 74.65702062644087) bank165714 +165715 POINT(40.51594991982565 74.24360873168668) bank165715 +165716 POINT(40.37751615206585 73.61462341000622) bank165716 +165717 POINT(40.36434453642481 73.89585228042971) bank165717 +165718 POINT(40.702328858188665 73.75375898055457) bank165718 +165719 POINT(40.673387646416614 73.50167173648141) bank165719 +165720 POINT(41.53816025097014 73.13251188301648) bank165720 +165721 POINT(40.090242847589 74.21475021781454) bank165721 +165722 POINT(40.645126690190104 73.31423036580453) bank165722 +165723 POINT(41.29405777101912 74.27254789257101) bank165723 +165724 POINT(40.526506199861636 74.43290588195151) bank165724 +165725 POINT(41.26488786631492 73.11603431894686) bank165725 +165726 POINT(39.904218471825054 74.09532134132745) bank165726 +165727 POINT(40.950002876837516 73.33446112759229) bank165727 +165728 POINT(41.45500358609549 73.04446613137401) bank165728 +165729 POINT(40.734423022199124 73.97437485067765) bank165729 +165730 POINT(40.29268299881583 73.03455533236051) bank165730 +165731 POINT(40.984534888232886 73.27158732743189) bank165731 +165732 POINT(40.56105796017539 74.45518758792133) bank165732 +165733 POINT(40.99545270602408 74.51783973378011) bank165733 +165734 POINT(40.7161877376415 74.05561153545473) bank165734 +165735 POINT(41.24543265512344 73.13855405047576) bank165735 +165736 POINT(40.266241793682696 74.44874180707646) bank165736 +165737 POINT(39.8659947467134 73.56464181867875) bank165737 +165738 POINT(41.50332166507642 74.79188654589478) bank165738 +165739 POINT(41.48694376009181 73.64745869913403) bank165739 +165740 POINT(40.5602191764264 74.53164268740211) bank165740 +165741 POINT(40.537829998540836 74.31245666062439) bank165741 +165742 POINT(40.1076855427938 74.75383616743484) bank165742 +165743 POINT(39.764139243019585 74.6821137792899) bank165743 +165744 POINT(39.72673059348044 74.89281389604974) bank165744 +165745 POINT(41.59898653478994 74.82389995594552) bank165745 +165746 POINT(41.65723875395027 73.28118253284008) bank165746 +165747 POINT(41.523031835596626 74.59684010576274) bank165747 +165748 POINT(41.06791961888026 74.08231937472011) bank165748 +165749 POINT(41.05975248047052 74.53439574114357) bank165749 +165750 POINT(40.01654215184325 74.87576558489606) bank165750 +165751 POINT(39.76274255007426 74.89705301038943) bank165751 +165752 POINT(40.63436660755053 74.34670410805512) bank165752 +165753 POINT(40.02566615736557 73.0444920478906) bank165753 +165754 POINT(40.89069951219122 74.45100157395834) bank165754 +165755 POINT(40.333482670992616 73.43689914973176) bank165755 +165756 POINT(41.18834018798892 73.77469062705217) bank165756 +165757 POINT(41.18387263984881 74.52394241587827) bank165757 +165758 POINT(40.54702084912696 74.53015852244754) bank165758 +165759 POINT(40.32273488210391 73.61802493083191) bank165759 +165760 POINT(40.54844853843977 74.28991307532627) bank165760 +165761 POINT(41.424405738410535 73.67573047063814) bank165761 +165762 POINT(41.699002572485576 74.46543303497693) bank165762 +165763 POINT(40.1386187701985 73.77767543856778) bank165763 +165764 POINT(41.588839957875884 74.86291455029776) bank165764 +165765 POINT(41.65972963080468 74.13882954283586) bank165765 +165766 POINT(40.36901092167676 73.5007685553662) bank165766 +165767 POINT(40.58051565563275 74.2659468805252) bank165767 +165768 POINT(40.50049971851189 73.81709158957783) bank165768 +165769 POINT(40.68427094761211 74.46227725956787) bank165769 +165770 POINT(40.114088763518424 74.58732992303675) bank165770 +165771 POINT(41.338608108126586 73.01747877455949) bank165771 +165772 POINT(40.85826732334245 73.03477251687778) bank165772 +165773 POINT(41.478864698961985 74.90716334046603) bank165773 +165774 POINT(40.877790584078895 74.54562590312673) bank165774 +165775 POINT(40.923302463949895 73.19672721578505) bank165775 +165776 POINT(40.595683203941306 74.89771490170499) bank165776 +165777 POINT(41.695344053841595 74.21646163674413) bank165777 +165778 POINT(40.48570415846133 74.0941067536194) bank165778 +165779 POINT(41.47570944295188 73.50289942038796) bank165779 +165780 POINT(40.38836608382587 73.92162801221873) bank165780 +165781 POINT(40.64037960784745 74.06679300755006) bank165781 +165782 POINT(40.42005751705835 73.90523924007053) bank165782 +165783 POINT(39.890007324883676 73.70565989962208) bank165783 +165784 POINT(40.428354026049035 74.83214705640722) bank165784 +165785 POINT(41.4627134356612 74.67132114098901) bank165785 +165786 POINT(40.16497619137408 73.84064303387296) bank165786 +165787 POINT(41.230044957395336 73.55510096953948) bank165787 +165788 POINT(40.56259628678129 73.783683462798) bank165788 +165789 POINT(40.97348008134103 74.01510497339083) bank165789 +165790 POINT(40.36498103718663 74.19006234134112) bank165790 +165791 POINT(40.420013033946155 73.87437688735605) bank165791 +165792 POINT(39.865375174290946 73.6848418351237) bank165792 +165793 POINT(39.89275328739899 74.68587675505687) bank165793 +165794 POINT(41.38492690996781 73.1015311113269) bank165794 +165795 POINT(40.11895278449894 73.2194528196217) bank165795 +165796 POINT(40.60410851058425 74.01262304799909) bank165796 +165797 POINT(41.40471621376573 73.8895868702291) bank165797 +165798 POINT(41.57406691441978 74.49734048151255) bank165798 +165799 POINT(41.66042077900971 74.50160737683697) bank165799 +165800 POINT(39.993852025622246 73.20016861068851) bank165800 +165801 POINT(40.42792739567316 73.82035057626362) bank165801 +165802 POINT(40.03062852809035 73.31188002442326) bank165802 +165803 POINT(41.4329636869869 73.82829490372755) bank165803 +165804 POINT(40.60635150445987 74.1701342695163) bank165804 +165805 POINT(39.81287037570606 74.67259686629288) bank165805 +165806 POINT(39.87236468103739 73.01931870423442) bank165806 +165807 POINT(41.191866944877624 73.35046100429666) bank165807 +165808 POINT(41.03566081005572 73.85420703661133) bank165808 +165809 POINT(41.03330627562202 73.93290546105993) bank165809 +165810 POINT(41.50246233943335 73.85526591565217) bank165810 +165811 POINT(40.387248695072856 73.01568560082264) bank165811 +165812 POINT(41.098880348479945 73.92203049269867) bank165812 +165813 POINT(39.791798947367354 73.2011430983262) bank165813 +165814 POINT(40.959383669383655 73.84225633889132) bank165814 +165815 POINT(40.39649077405068 73.28983406884349) bank165815 +165816 POINT(41.27735783918884 73.32325842774152) bank165816 +165817 POINT(40.298048179122176 73.7526652003492) bank165817 +165818 POINT(40.30760913816802 73.7816962764738) bank165818 +165819 POINT(41.523472264606575 74.26389452754135) bank165819 +165820 POINT(40.14275586688844 73.73096257142817) bank165820 +165821 POINT(40.05047655254668 73.0751678185119) bank165821 +165822 POINT(40.02930165979019 74.27396563872897) bank165822 +165823 POINT(41.594835737383896 73.39539128812983) bank165823 +165824 POINT(41.02751513788665 73.31472394417585) bank165824 +165825 POINT(40.27590719660494 74.35429954806777) bank165825 +165826 POINT(40.738086205829084 74.1946058482014) bank165826 +165827 POINT(40.076030226227694 74.14828963401219) bank165827 +165828 POINT(40.690276123085994 73.29670008450844) bank165828 +165829 POINT(40.890353261312555 73.1985855277845) bank165829 +165830 POINT(41.087476673751326 73.54215980034537) bank165830 +165831 POINT(40.129726722803234 74.4730728361926) bank165831 +165832 POINT(41.48074282570404 73.24773142228045) bank165832 +165833 POINT(39.74789090808255 73.16923205995244) bank165833 +165834 POINT(41.63116806408296 73.9846936694019) bank165834 +165835 POINT(40.347223859160536 74.71508531215434) bank165835 +165836 POINT(40.069118954606814 74.51737210408666) bank165836 +165837 POINT(40.65568128337328 73.67294025924359) bank165837 +165838 POINT(40.220400965536754 74.53838506833335) bank165838 +165839 POINT(41.147976231040836 73.1494188224278) bank165839 +165840 POINT(39.81053780353939 74.90099787864673) bank165840 +165841 POINT(41.33511477981126 73.5726162559075) bank165841 +165842 POINT(41.695023793057665 73.94062979158521) bank165842 +165843 POINT(40.81124485542229 74.57277891653754) bank165843 +165844 POINT(40.02576460310671 74.12902238186555) bank165844 +165845 POINT(40.37110690809333 73.45958755777242) bank165845 +165846 POINT(41.586904648647 73.82763714639056) bank165846 +165847 POINT(41.48388009596332 73.70446421787113) bank165847 +165848 POINT(41.50489554617968 74.00086308487822) bank165848 +165849 POINT(40.73882357629815 73.90176668116993) bank165849 +165850 POINT(41.117036583260585 74.80608138243409) bank165850 +165851 POINT(41.58409353503268 74.28444228320862) bank165851 +165852 POINT(41.36225342840243 73.82773528065921) bank165852 +165853 POINT(40.97966126005827 74.62780990499468) bank165853 +165854 POINT(41.45788978154384 74.10777961589781) bank165854 +165855 POINT(40.66198556214807 74.6222784345243) bank165855 +165856 POINT(40.73042833680048 73.74031021224837) bank165856 +165857 POINT(40.265005720809214 74.35538668319286) bank165857 +165858 POINT(40.786483475501036 74.0608458567545) bank165858 +165859 POINT(40.2906095919744 73.90539255349846) bank165859 +165860 POINT(41.10691959905624 73.23259368254308) bank165860 +165861 POINT(41.602563174887166 73.5152157599817) bank165861 +165862 POINT(40.050197248356675 73.87053316322437) bank165862 +165863 POINT(40.772632882887855 73.12358214656973) bank165863 +165864 POINT(41.163877613016396 73.89809755286946) bank165864 +165865 POINT(41.067476028664835 73.49952539412675) bank165865 +165866 POINT(41.292513601413816 73.9277250646595) bank165866 +165867 POINT(39.82782991039614 73.71509809110776) bank165867 +165868 POINT(41.05841817979905 74.42978931279835) bank165868 +165869 POINT(40.06672825227026 74.33161368033731) bank165869 +165870 POINT(40.837176228160374 74.2558906248836) bank165870 +165871 POINT(41.195600312695596 74.81317740471563) bank165871 +165872 POINT(41.39498699031851 74.05524915782156) bank165872 +165873 POINT(41.591403100702294 74.99309697915686) bank165873 +165874 POINT(40.038878998398395 74.90231108443503) bank165874 +165875 POINT(40.46001115300214 74.29366490907357) bank165875 +165876 POINT(40.258786730445394 73.99599824702283) bank165876 +165877 POINT(39.927217687473515 73.96131456905611) bank165877 +165878 POINT(40.01857792568524 74.84833328672605) bank165878 +165879 POINT(41.133401898792236 74.89432593798166) bank165879 +165880 POINT(40.64691934420094 74.0747692099244) bank165880 +165881 POINT(41.08877305013194 74.42615791990981) bank165881 +165882 POINT(40.84739144895339 74.31199608007242) bank165882 +165883 POINT(40.773530512181146 74.13206382133357) bank165883 +165884 POINT(39.766696633386466 74.82581814787964) bank165884 +165885 POINT(41.464968808176245 73.33673531820138) bank165885 +165886 POINT(40.38617924084496 73.96694963962919) bank165886 +165887 POINT(40.01763252005089 74.3764864852627) bank165887 +165888 POINT(41.0121779815714 73.2255703815274) bank165888 +165889 POINT(40.21332010610893 74.91295633859157) bank165889 +165890 POINT(40.85184930275223 73.76682213193712) bank165890 +165891 POINT(41.2340677586777 73.24062859638522) bank165891 +165892 POINT(40.51387678162909 74.52025714340579) bank165892 +165893 POINT(40.45001080244021 74.37459978782154) bank165893 +165894 POINT(41.2706436935859 73.77587573869194) bank165894 +165895 POINT(39.992230193709645 73.72774852066809) bank165895 +165896 POINT(41.46768091732163 74.41439360403713) bank165896 +165897 POINT(40.188351073520316 73.60723576975771) bank165897 +165898 POINT(40.30891171647703 73.97894946345053) bank165898 +165899 POINT(41.52048401826102 74.0792460394293) bank165899 +165900 POINT(41.159085072177845 74.18452063344895) bank165900 +165901 POINT(40.296839661072966 73.63572473453195) bank165901 +165902 POINT(41.654255238292386 73.57882472106118) bank165902 +165903 POINT(40.885754705934474 73.44514172299185) bank165903 +165904 POINT(40.35795114499417 74.89755682689336) bank165904 +165905 POINT(40.613272495765074 74.9676211472417) bank165905 +165906 POINT(41.14912169343055 73.63387485339958) bank165906 +165907 POINT(41.04840995321541 73.6095654407495) bank165907 +165908 POINT(41.52805488648089 73.00783630629769) bank165908 +165909 POINT(40.55939842242616 74.6552282183738) bank165909 +165910 POINT(40.64834819533236 73.15413216253933) bank165910 +165911 POINT(41.483439238734555 74.59285003055057) bank165911 +165912 POINT(40.09633551315253 74.88979839304646) bank165912 +165913 POINT(41.357558856777494 73.2323872333706) bank165913 +165914 POINT(40.51467760101112 74.04189711883551) bank165914 +165915 POINT(40.19409149810926 73.55446180557068) bank165915 +165916 POINT(39.99286480426853 74.87946999951389) bank165916 +165917 POINT(41.5787653420124 73.796999765922) bank165917 +165918 POINT(41.050748066367554 74.48644518535019) bank165918 +165919 POINT(41.42460464461581 73.54529628720019) bank165919 +165920 POINT(41.0864687816915 74.23067189745389) bank165920 +165921 POINT(40.57044271173248 73.52472452752068) bank165921 +165922 POINT(40.32263659145374 73.51447698931509) bank165922 +165923 POINT(40.37637404142789 73.33866899509763) bank165923 +165924 POINT(40.95300326116518 74.02040197154551) bank165924 +165925 POINT(40.29240526741983 74.80157737145247) bank165925 +165926 POINT(40.11666270135245 73.02087112527305) bank165926 +165927 POINT(40.22728742560721 73.47050187878877) bank165927 +165928 POINT(40.433807782825255 74.03623582819993) bank165928 +165929 POINT(41.36941275647887 74.75310058547362) bank165929 +165930 POINT(40.75305439744881 74.97304271381306) bank165930 +165931 POINT(40.80930911913421 73.33440607514027) bank165931 +165932 POINT(40.116435093199634 73.31239581506777) bank165932 +165933 POINT(41.234466793443566 74.58841432656668) bank165933 +165934 POINT(40.98918785343708 74.7786062817896) bank165934 +165935 POINT(41.204493741073755 73.24759888547295) bank165935 +165936 POINT(40.20980005021958 74.00262071223098) bank165936 +165937 POINT(41.51841706468594 74.34238264119362) bank165937 +165938 POINT(41.13877624548328 74.3969358191664) bank165938 +165939 POINT(40.598602417098185 74.36588061711953) bank165939 +165940 POINT(41.24046146645197 74.34591503676764) bank165940 +165941 POINT(40.47117699426844 74.67163707021047) bank165941 +165942 POINT(41.514577493003145 73.07998966623019) bank165942 +165943 POINT(40.344011735192986 74.74243702856977) bank165943 +165944 POINT(40.4905527494597 74.89465244599593) bank165944 +165945 POINT(40.485484882541215 73.05989552887227) bank165945 +165946 POINT(40.11702716399734 74.01662708849926) bank165946 +165947 POINT(40.04487870818587 74.20671995619182) bank165947 +165948 POINT(40.911904718838215 74.90927986834274) bank165948 +165949 POINT(40.71838219296116 74.63656090092366) bank165949 +165950 POINT(40.189505427977615 74.29058035663263) bank165950 +165951 POINT(40.16666034608885 74.26376208334783) bank165951 +165952 POINT(40.88131477660046 74.5128799997434) bank165952 +165953 POINT(41.19347601298498 73.14388282473737) bank165953 +165954 POINT(39.99074381438142 73.07195629874367) bank165954 +165955 POINT(40.68922685888238 73.53500294484411) bank165955 +165956 POINT(41.420832651250215 73.6399561609484) bank165956 +165957 POINT(41.02942691641284 74.35827920485283) bank165957 +165958 POINT(40.48294243299307 73.31379102495153) bank165958 +165959 POINT(41.37008560886789 74.96526259285355) bank165959 +165960 POINT(41.582536865161316 73.46426946565607) bank165960 +165961 POINT(39.927453714362585 73.092142163955) bank165961 +165962 POINT(39.86563492190105 74.05102271149774) bank165962 +165963 POINT(40.21670660014951 74.37453921087857) bank165963 +165964 POINT(40.518332493908176 74.94359413979664) bank165964 +165965 POINT(40.26635987631079 75.00264594895246) bank165965 +165966 POINT(40.902352507292605 73.40301891167576) bank165966 +165967 POINT(40.80246739214393 73.52429221864978) bank165967 +165968 POINT(40.3549762595893 74.5481159111355) bank165968 +165969 POINT(40.239337281896645 74.87922082940015) bank165969 +165970 POINT(40.73286649940188 74.92587349456456) bank165970 +165971 POINT(40.34433418863189 73.87015872629178) bank165971 +165972 POINT(40.62124440682416 73.41947248598801) bank165972 +165973 POINT(41.6622523442828 74.75009301113406) bank165973 +165974 POINT(39.88371078540058 73.50256492839519) bank165974 +165975 POINT(40.920336151881976 74.28833974748741) bank165975 +165976 POINT(40.73077214907109 73.57708264431531) bank165976 +165977 POINT(40.221348530116146 73.49443826370273) bank165977 +165978 POINT(41.60658403890757 74.3836228641394) bank165978 +165979 POINT(40.54145237671372 74.79692924436694) bank165979 +165980 POINT(40.78958870207511 74.65818472352119) bank165980 +165981 POINT(41.54838576643031 74.15147338558737) bank165981 +165982 POINT(40.6700071062575 74.62857980427896) bank165982 +165983 POINT(41.22004215914797 74.04530915608228) bank165983 +165984 POINT(40.43053212531157 73.44815591251044) bank165984 +165985 POINT(40.99417589706707 73.25733622352872) bank165985 +165986 POINT(40.1874014917187 73.01536175273147) bank165986 +165987 POINT(41.144624706959185 74.38075562835745) bank165987 +165988 POINT(41.10240229213081 74.9543160516128) bank165988 +165989 POINT(40.55684206327428 74.86880270060402) bank165989 +165990 POINT(41.330097959731 73.66935196703417) bank165990 +165991 POINT(41.561643432763866 74.40668663518935) bank165991 +165992 POINT(40.15796580322788 74.67822035638495) bank165992 +165993 POINT(40.24772361389722 74.08320338390931) bank165993 +165994 POINT(39.71617418315396 74.8460792208488) bank165994 +165995 POINT(41.2208864348376 73.92611600197861) bank165995 +165996 POINT(40.99139239046715 74.55828973154249) bank165996 +165997 POINT(40.76156577632115 74.7332919460218) bank165997 +165998 POINT(40.79410979659023 74.1959627291462) bank165998 +165999 POINT(40.55183104565653 74.8483729411455) bank165999 +166000 POINT(40.325657607066574 73.6783279030717) bank166000 +166001 POINT(39.981454327102426 74.50903697801628) bank166001 +166002 POINT(41.63818811854756 73.14685516547769) bank166002 +166003 POINT(40.444796477205365 73.49665047365296) bank166003 +166004 POINT(40.732858300915154 74.45355356936155) bank166004 +166005 POINT(41.630191561572055 73.84634024263995) bank166005 +166006 POINT(39.714593102618295 74.53631966131766) bank166006 +166007 POINT(40.496990645783924 73.38488304390829) bank166007 +166008 POINT(40.757722236588386 73.9782961143817) bank166008 +166009 POINT(40.95344157512927 74.14780410580758) bank166009 +166010 POINT(40.743831312608634 73.5049112442962) bank166010 +166011 POINT(40.83492936648419 73.32663842029623) bank166011 +166012 POINT(41.40866281263059 74.60781119052308) bank166012 +166013 POINT(40.7978848537477 74.73807045531774) bank166013 +166014 POINT(41.3632378631329 74.01704479393915) bank166014 +166015 POINT(41.67465200829963 74.09593137777628) bank166015 +166016 POINT(40.58846247692627 73.06648942587735) bank166016 +166017 POINT(41.09676251056593 73.13982392508507) bank166017 +166018 POINT(40.16018372700639 73.42102245442624) bank166018 +166019 POINT(40.61519165281839 74.17524996938342) bank166019 +166020 POINT(40.10275836333068 73.79857412296349) bank166020 +166021 POINT(41.28271477040406 74.36663968004581) bank166021 +166022 POINT(40.19884552742697 73.39389668873339) bank166022 +166023 POINT(41.19812866418053 74.980201194008) bank166023 +166024 POINT(39.8631052168581 73.77618080779521) bank166024 +166025 POINT(40.20972479278984 73.33908296722645) bank166025 +166026 POINT(40.749280108902475 74.78011175346079) bank166026 +166027 POINT(41.10635803260556 74.09450446873049) bank166027 +166028 POINT(40.294197612581854 74.75075664948503) bank166028 +166029 POINT(41.54634107085557 74.05392538477973) bank166029 +166030 POINT(41.29204414193432 73.5584030862086) bank166030 +166031 POINT(40.6791431924937 74.03341051012035) bank166031 +166032 POINT(41.160822449730006 73.94958848742368) bank166032 +166033 POINT(41.70560735340593 73.37452537820576) bank166033 +166034 POINT(41.675264794461896 73.8277254284559) bank166034 +166035 POINT(40.1839241048884 73.71263719263342) bank166035 +166036 POINT(40.1230981704859 73.66258916639786) bank166036 +166037 POINT(41.34020079755498 74.01697104455602) bank166037 +166038 POINT(40.795243774607926 73.4347795822812) bank166038 +166039 POINT(41.1978669151155 73.0213427074405) bank166039 +166040 POINT(41.43633058163488 74.14534022419302) bank166040 +166041 POINT(41.14203335434276 74.75139846703948) bank166041 +166042 POINT(39.74967653128382 73.73395448564818) bank166042 +166043 POINT(40.9236774400738 74.17295340573907) bank166043 +166044 POINT(40.16261763324934 73.07282550609658) bank166044 +166045 POINT(40.10627103588558 73.75083514351489) bank166045 +166046 POINT(41.405632803969155 74.89308464354022) bank166046 +166047 POINT(39.74176232409703 73.0926050536793) bank166047 +166048 POINT(41.26940244048598 73.72796426100027) bank166048 +166049 POINT(41.0193405676158 73.27525173549911) bank166049 +166050 POINT(40.84347751008044 73.9863873893787) bank166050 +166051 POINT(40.279289894771516 74.69937156941126) bank166051 +166052 POINT(41.012884845662825 73.69843856260718) bank166052 +166053 POINT(40.95703935845557 74.3174496727629) bank166053 +166054 POINT(39.78233854406816 74.22580654424004) bank166054 +166055 POINT(40.06965844295914 73.26129570586433) bank166055 +166056 POINT(41.672917028229875 73.55282546140376) bank166056 +166057 POINT(40.40996121904574 73.02131977255392) bank166057 +166058 POINT(40.28073836005242 74.88042082868587) bank166058 +166059 POINT(40.69219938894891 74.53272209168674) bank166059 +166060 POINT(40.96940236256317 73.9292550540498) bank166060 +166061 POINT(40.013866844419105 73.85311411431627) bank166061 +166062 POINT(40.11637396215599 74.29476786332478) bank166062 +166063 POINT(39.7188303634308 74.87017068274498) bank166063 +166064 POINT(40.42903400271185 74.7841301681223) bank166064 +166065 POINT(40.00762868349903 73.71137005101949) bank166065 +166066 POINT(40.239766991480664 73.8693897337058) bank166066 +166067 POINT(40.29493242941717 73.93699374264818) bank166067 +166068 POINT(41.29726975745038 73.39930416023817) bank166068 +166069 POINT(40.006462617354096 74.2757203651791) bank166069 +166070 POINT(40.709705092064524 73.19322766198634) bank166070 +166071 POINT(41.06495169429968 74.75709370535236) bank166071 +166072 POINT(40.20995047918473 73.04841714570043) bank166072 +166073 POINT(40.99393414918108 74.57049538070251) bank166073 +166074 POINT(39.86751475203417 74.81852689437927) bank166074 +166075 POINT(41.306254657699185 73.68200551967871) bank166075 +166076 POINT(41.666058062378994 74.80546411592776) bank166076 +166077 POINT(40.13752432294422 74.91575911659761) bank166077 +166078 POINT(39.95572351575089 73.48116826302238) bank166078 +166079 POINT(40.80103153864218 74.09933278686475) bank166079 +166080 POINT(40.56902823196979 74.70241777424476) bank166080 +166081 POINT(39.98646449064663 73.48636352444123) bank166081 +166082 POINT(41.50607727311805 74.98897000149219) bank166082 +166083 POINT(40.605021743134465 73.429799834829) bank166083 +166084 POINT(41.395405191576096 74.04433453445081) bank166084 +166085 POINT(41.58948573653482 74.6888930567539) bank166085 +166086 POINT(40.39878598995271 73.87337637544077) bank166086 +166087 POINT(39.76682373842002 74.20229849479817) bank166087 +166088 POINT(40.06015114169949 73.76827502848974) bank166088 +166089 POINT(40.301416656828835 73.75678521245418) bank166089 +166090 POINT(41.21314386119272 73.08404436080038) bank166090 +166091 POINT(41.47762439399195 73.83669834128887) bank166091 +166092 POINT(40.96062435218073 73.83525184699758) bank166092 +166093 POINT(41.05851618089685 74.0285660109726) bank166093 +166094 POINT(39.82857591107943 74.52081736207307) bank166094 +166095 POINT(41.26636969017786 74.08239273555695) bank166095 +166096 POINT(40.47136141108653 73.82221170927468) bank166096 +166097 POINT(40.72764060421069 74.62912941151086) bank166097 +166098 POINT(41.30168553196794 73.34377049959893) bank166098 +166099 POINT(41.51964923635006 74.32953461454925) bank166099 +166100 POINT(41.480443681078796 73.36667463908493) bank166100 +166101 POINT(41.29752849596617 73.69966931326516) bank166101 +166102 POINT(39.77190844057206 74.255330239671) bank166102 +166103 POINT(41.53065327474562 74.65021827873083) bank166103 +166104 POINT(40.75038412968876 73.9768467877144) bank166104 +166105 POINT(40.23634592522318 74.76908765961954) bank166105 +166106 POINT(41.140705743854525 74.8730927217886) bank166106 +166107 POINT(39.813727844864395 74.14633187976497) bank166107 +166108 POINT(39.876591053903134 74.14688007228429) bank166108 +166109 POINT(40.64128377956035 74.25540002199936) bank166109 +166110 POINT(40.41759143537736 73.84349373984139) bank166110 +166111 POINT(40.930952899082214 73.58797049762524) bank166111 +166112 POINT(41.597896068151464 74.6716950407153) bank166112 +166113 POINT(41.6994201554201 74.9313494345281) bank166113 +166114 POINT(40.92320733167469 74.33010830266828) bank166114 +166115 POINT(40.49884169714235 73.57324811191451) bank166115 +166116 POINT(40.98440307883081 73.4557870348585) bank166116 +166117 POINT(39.78129618278137 73.40025155595964) bank166117 +166118 POINT(41.4541014328757 73.86175065818773) bank166118 +166119 POINT(39.85994949118182 73.95478944296805) bank166119 +166120 POINT(40.228346897975534 73.6550870604774) bank166120 +166121 POINT(40.517795415221016 73.80552293900621) bank166121 +166122 POINT(40.664284929590515 74.50334534541547) bank166122 +166123 POINT(39.90100495693948 74.36904596045625) bank166123 +166124 POINT(40.57126810943506 74.27710561027413) bank166124 +166125 POINT(40.23119142970809 73.07301461642663) bank166125 +166126 POINT(41.067069185804634 74.64153930930233) bank166126 +166127 POINT(41.027630465364545 74.3190529373804) bank166127 +166128 POINT(39.72501216748083 73.79916995673932) bank166128 +166129 POINT(40.95915069308929 74.63877100420417) bank166129 +166130 POINT(41.48834726076817 73.37258623182217) bank166130 +166131 POINT(39.890269841432456 73.18709633718593) bank166131 +166132 POINT(40.31956916354579 73.56058781646173) bank166132 +166133 POINT(40.58178989448069 73.1253916871302) bank166133 +166134 POINT(40.79448359977889 73.05951214245157) bank166134 +166135 POINT(40.25404336755421 73.38004160051366) bank166135 +166136 POINT(40.874890589274465 73.7748575886453) bank166136 +166137 POINT(39.79803202346328 73.60773246615341) bank166137 +166138 POINT(41.557845714072975 74.2113685453159) bank166138 +166139 POINT(40.38646972225546 73.95043984632551) bank166139 +166140 POINT(40.41265252613816 73.89395872937979) bank166140 +166141 POINT(40.3959005268503 74.46159042276771) bank166141 +166142 POINT(41.20077832132652 74.76847688618996) bank166142 +166143 POINT(41.30086606729817 74.56936435968997) bank166143 +166144 POINT(40.52499366029563 73.08301995213863) bank166144 +166145 POINT(41.441967243289575 74.52900511755526) bank166145 +166146 POINT(41.02585048614723 74.43900622850788) bank166146 +166147 POINT(41.20908844873705 73.8164176479312) bank166147 +166148 POINT(41.44345281379261 73.2197534766448) bank166148 +166149 POINT(41.377876523175736 74.56395586972839) bank166149 +166150 POINT(41.636037316264044 74.13391539662078) bank166150 +166151 POINT(39.89848330814374 73.95252692872405) bank166151 +166152 POINT(40.536092250948506 74.68126876028496) bank166152 +166153 POINT(40.08590235343723 74.4168057973462) bank166153 +166154 POINT(41.19678083379861 74.18366301049255) bank166154 +166155 POINT(40.93291537170563 73.53855741438518) bank166155 +166156 POINT(40.944251448283886 74.04939964466041) bank166156 +166157 POINT(40.14193363025162 73.23286742026818) bank166157 +166158 POINT(41.37432489767141 73.1466699312047) bank166158 +166159 POINT(41.64079982580194 73.92899049338006) bank166159 +166160 POINT(41.56449465207482 73.9392807555471) bank166160 +166161 POINT(41.39410229208409 73.67206337807357) bank166161 +166162 POINT(41.47376900879509 74.52952464043801) bank166162 +166163 POINT(41.29058453783908 73.57346614570555) bank166163 +166164 POINT(40.98795727231407 74.37677730297864) bank166164 +166165 POINT(41.02190466333916 73.98228849878687) bank166165 +166166 POINT(40.198277382511876 73.31112881835611) bank166166 +166167 POINT(39.87155156263458 74.32834780459551) bank166167 +166168 POINT(40.239022288652045 73.5913485922634) bank166168 +166169 POINT(41.67315297921821 74.40691106073902) bank166169 +166170 POINT(41.44448959386796 73.96080021470013) bank166170 +166171 POINT(39.843716661915224 74.49544513113669) bank166171 +166172 POINT(40.712111271167245 73.65403498720534) bank166172 +166173 POINT(41.14939305190048 74.5286343991184) bank166173 +166174 POINT(40.65741043995926 73.65464827729558) bank166174 +166175 POINT(41.553014566607764 73.39034361475844) bank166175 +166176 POINT(40.92743807262843 74.41198270947669) bank166176 +166177 POINT(40.68199779085502 73.19851333432231) bank166177 +166178 POINT(40.6008120197084 73.27059165528782) bank166178 +166179 POINT(40.60240769513446 74.6341521311323) bank166179 +166180 POINT(41.064874023199984 74.0269149244049) bank166180 +166181 POINT(39.724295380780056 73.82469973436481) bank166181 +166182 POINT(39.8943348504004 74.6392860854371) bank166182 +166183 POINT(41.62359130395363 73.55720473222473) bank166183 +166184 POINT(40.29201305822362 74.56976878027987) bank166184 +166185 POINT(41.1002742212325 73.54729269617354) bank166185 +166186 POINT(39.98423541974567 73.78486677832463) bank166186 +166187 POINT(40.82597245973655 73.57391201801887) bank166187 +166188 POINT(41.38468926055945 74.66043691384245) bank166188 +166189 POINT(39.784401522797644 73.88528361829121) bank166189 +166190 POINT(41.187149945283 74.41649660589162) bank166190 +166191 POINT(39.87830240504532 73.70718004606542) bank166191 +166192 POINT(40.67362520493434 73.70877188677404) bank166192 +166193 POINT(41.67902366553703 73.2865694419392) bank166193 +166194 POINT(40.07179288996839 73.98049913271281) bank166194 +166195 POINT(40.33083369517968 74.98714083979868) bank166195 +166196 POINT(40.96105413634502 74.06919941793146) bank166196 +166197 POINT(40.49131828184883 73.9876968178558) bank166197 +166198 POINT(39.793969365066786 74.48924941496726) bank166198 +166199 POINT(40.102109518008774 74.795600329721) bank166199 +166200 POINT(40.02635543610802 73.6999447835429) bank166200 +166201 POINT(40.989620706464414 74.84748804972185) bank166201 +166202 POINT(41.067885395905336 73.31781151375408) bank166202 +166203 POINT(41.08960273475689 74.62720803200449) bank166203 +166204 POINT(40.92451491897384 73.67046947492659) bank166204 +166205 POINT(40.887273013027034 73.2439916296006) bank166205 +166206 POINT(40.28394117255159 73.67615427997073) bank166206 +166207 POINT(40.93522360494934 73.34306038212641) bank166207 +166208 POINT(40.10978749475831 74.89087762142395) bank166208 +166209 POINT(41.246505808179585 73.40613897028263) bank166209 +166210 POINT(40.21474899298767 73.01556603456275) bank166210 +166211 POINT(41.499807092887536 73.74425399930604) bank166211 +166212 POINT(40.222921551129865 74.65174356050083) bank166212 +166213 POINT(40.90465336358452 74.42275891460564) bank166213 +166214 POINT(39.862703699521084 73.05792603904779) bank166214 +166215 POINT(40.01780389054348 74.16005998377845) bank166215 +166216 POINT(41.30263251603439 73.31929152601322) bank166216 +166217 POINT(40.54368283631429 74.92856208172746) bank166217 +166218 POINT(39.98995123622241 74.74070143238349) bank166218 +166219 POINT(41.69282603462295 74.5398461915508) bank166219 +166220 POINT(41.02988997632416 74.8622554017875) bank166220 +166221 POINT(40.545211558972234 73.80784331963116) bank166221 +166222 POINT(40.73358015007771 74.08214081758689) bank166222 +166223 POINT(39.947510741928 74.84130581643973) bank166223 +166224 POINT(40.14901608709138 74.78307582906086) bank166224 +166225 POINT(40.007150210945035 74.23357610636197) bank166225 +166226 POINT(40.95243028549554 73.79512405043207) bank166226 +166227 POINT(40.900650308118884 73.30681397618321) bank166227 +166228 POINT(40.14814822346132 73.59176830266783) bank166228 +166229 POINT(39.84531772546681 74.25131768516299) bank166229 +166230 POINT(40.78421645011953 74.67658096782245) bank166230 +166231 POINT(41.392040230943955 74.24294396212451) bank166231 +166232 POINT(41.51443822499423 74.58213234167006) bank166232 +166233 POINT(40.832715768041936 74.64458214072974) bank166233 +166234 POINT(41.59817742430963 74.98743817095897) bank166234 +166235 POINT(39.750106256057094 73.89723962307838) bank166235 +166236 POINT(41.33115400049154 73.74519313795152) bank166236 +166237 POINT(40.132604208760796 73.40772439324448) bank166237 +166238 POINT(40.930925570411205 74.10702549422496) bank166238 +166239 POINT(40.62863577034416 74.42592985320117) bank166239 +166240 POINT(40.506455833094094 74.6823145836051) bank166240 +166241 POINT(41.17431553450832 73.37144029955803) bank166241 +166242 POINT(40.58837424309154 73.86455233008799) bank166242 +166243 POINT(40.342945227430064 73.97385175067882) bank166243 +166244 POINT(40.136673111066415 74.78032261258801) bank166244 +166245 POINT(40.90604925494137 73.26719985999475) bank166245 +166246 POINT(41.631867204639015 74.89648918266448) bank166246 +166247 POINT(39.901598357600534 74.67878681402681) bank166247 +166248 POINT(40.39255320354999 73.78903928315167) bank166248 +166249 POINT(41.15086568726008 74.18029819676485) bank166249 +166250 POINT(40.276663065731086 74.47233018807262) bank166250 +166251 POINT(39.98171042358854 74.58928994724134) bank166251 +166252 POINT(41.32564665293493 74.73699410690583) bank166252 +166253 POINT(39.82725085248403 74.18658388297173) bank166253 +166254 POINT(41.442798175275776 74.18002685858463) bank166254 +166255 POINT(41.59675536532501 73.82570745652477) bank166255 +166256 POINT(40.50587978611169 73.83683432723166) bank166256 +166257 POINT(40.35469834790901 73.14286423000777) bank166257 +166258 POINT(40.44410201618279 74.71036479685962) bank166258 +166259 POINT(41.69440761367707 73.8682337152677) bank166259 +166260 POINT(40.07038175072354 74.09121377053124) bank166260 +166261 POINT(40.97028191138934 73.01972060297761) bank166261 +166262 POINT(40.17200808978468 74.05842122290471) bank166262 +166263 POINT(41.2079952965462 74.51330954069368) bank166263 +166264 POINT(40.45032590115339 73.46507952855586) bank166264 +166265 POINT(41.253516831826886 74.85049739910575) bank166265 +166266 POINT(40.4470168961401 74.14483460602473) bank166266 +166267 POINT(39.92864173795272 74.56584242851508) bank166267 +166268 POINT(41.42312347006375 74.90165976541905) bank166268 +166269 POINT(40.90630997206909 73.90965881442519) bank166269 +166270 POINT(39.8079094499369 73.61502359038624) bank166270 +166271 POINT(41.549243001145115 73.62837167171153) bank166271 +166272 POINT(39.993936272508826 73.25920363367129) bank166272 +166273 POINT(41.20749330712462 73.58247597898335) bank166273 +166274 POINT(41.26236256196384 73.2503350936665) bank166274 +166275 POINT(40.60013082709689 74.68227155013274) bank166275 +166276 POINT(40.958964609001036 74.57909040773752) bank166276 +166277 POINT(41.40288793341577 73.83615421990251) bank166277 +166278 POINT(41.254654995245666 74.13289127062068) bank166278 +166279 POINT(41.09205701796022 73.19494672347062) bank166279 +166280 POINT(41.02806357996464 74.98086828899933) bank166280 +166281 POINT(40.55805854633028 74.61854488882746) bank166281 +166282 POINT(41.30138618193274 73.4495017383079) bank166282 +166283 POINT(39.72005867167401 73.19261791812978) bank166283 +166284 POINT(40.74289556973543 74.08222327651721) bank166284 +166285 POINT(41.25951105801148 73.88849116644518) bank166285 +166286 POINT(40.18641851473259 73.62766718027676) bank166286 +166287 POINT(40.75464873284111 73.48466433950534) bank166287 +166288 POINT(39.76771439114857 74.78202225018346) bank166288 +166289 POINT(39.76684433334275 74.34499311811629) bank166289 +166290 POINT(40.714161416765315 74.61586362737887) bank166290 +166291 POINT(40.76375386193509 73.32811058275774) bank166291 +166292 POINT(41.418965317739065 74.32012784867696) bank166292 +166293 POINT(39.82898344588118 73.14068359315469) bank166293 +166294 POINT(40.037005506017806 74.86674134414825) bank166294 +166295 POINT(41.142222643875236 74.15221820351869) bank166295 +166296 POINT(39.839472901303566 74.93203329404962) bank166296 +166297 POINT(40.07529766164079 73.37873642293448) bank166297 +166298 POINT(39.97625246623091 73.43714468554919) bank166298 +166299 POINT(41.19142309284828 74.23241839938503) bank166299 +166300 POINT(39.81613992488296 74.58753656862498) bank166300 +166301 POINT(40.946563884669125 74.75814604987292) bank166301 +166302 POINT(41.40296843448534 73.92111573014037) bank166302 +166303 POINT(41.5249587062077 73.56432672390922) bank166303 +166304 POINT(40.81610777237442 74.9369484872663) bank166304 +166305 POINT(40.16993494681125 74.89121126581719) bank166305 +166306 POINT(41.17281966997789 74.80832541644321) bank166306 +166307 POINT(41.355068899049265 74.16201840201) bank166307 +166308 POINT(40.27249653045938 73.07037756915835) bank166308 +166309 POINT(40.23562076169921 74.94475397048076) bank166309 +166310 POINT(40.73548984143267 73.42745928753367) bank166310 +166311 POINT(41.49628340355543 73.37032078873341) bank166311 +166312 POINT(39.82696875750466 73.95966509192452) bank166312 +166313 POINT(41.312413444361454 74.06365432636464) bank166313 +166314 POINT(41.113311748506845 73.16350813676726) bank166314 +166315 POINT(40.044158887167626 73.47640601716553) bank166315 +166316 POINT(41.04893768551596 73.37081192827284) bank166316 +166317 POINT(41.655852709995045 74.89179794695679) bank166317 +166318 POINT(40.73765201159382 74.11907297051545) bank166318 +166319 POINT(41.06674204337178 74.34092608943256) bank166319 +166320 POINT(41.491057941208595 73.95893634322604) bank166320 +166321 POINT(41.186593122660355 74.57632750566124) bank166321 +166322 POINT(40.326425468654634 74.61568142622922) bank166322 +166323 POINT(41.30725356494926 74.67159477961141) bank166323 +166324 POINT(39.76814975369914 73.28304970049528) bank166324 +166325 POINT(40.336726786989594 74.75596808257542) bank166325 +166326 POINT(40.09584574904541 73.60842125194335) bank166326 +166327 POINT(41.13465710666369 74.90780366433297) bank166327 +166328 POINT(41.36457606593508 73.3992838483801) bank166328 +166329 POINT(41.611838070831325 73.8874840445644) bank166329 +166330 POINT(40.2172856980838 74.78687966902395) bank166330 +166331 POINT(39.96450303480676 73.38912373705762) bank166331 +166332 POINT(40.71176486475763 73.06085490366969) bank166332 +166333 POINT(40.450017072932276 73.52804594983229) bank166333 +166334 POINT(40.547160616328206 73.23990136521013) bank166334 +166335 POINT(40.30677801707267 73.88201065333035) bank166335 +166336 POINT(39.94555077027355 74.92111478311514) bank166336 +166337 POINT(40.74227606800593 74.7789970580157) bank166337 +166338 POINT(40.57690099187494 74.32875227967472) bank166338 +166339 POINT(40.31662991615146 73.96638944163496) bank166339 +166340 POINT(40.938148974351684 74.64901160529224) bank166340 +166341 POINT(41.542463374895135 74.52708575484026) bank166341 +166342 POINT(41.15418700478434 73.39315493079972) bank166342 +166343 POINT(40.15237274505056 74.13301532211543) bank166343 +166344 POINT(40.69900878920033 74.61604948458987) bank166344 +166345 POINT(40.58305184933125 73.83494557193586) bank166345 +166346 POINT(40.22834388865464 74.3036780714992) bank166346 +166347 POINT(40.425728011872486 73.49394085253356) bank166347 +166348 POINT(40.139927920292436 73.22572021380057) bank166348 +166349 POINT(40.23130796345413 74.60294898754601) bank166349 +166350 POINT(40.305822528301015 74.84459703243984) bank166350 +166351 POINT(39.9951367595318 74.97294736349454) bank166351 +166352 POINT(41.32623749054468 73.29735381356012) bank166352 +166353 POINT(40.13307047526508 74.5939589286021) bank166353 +166354 POINT(40.80863457590695 73.17249203150934) bank166354 +166355 POINT(41.389196513845626 73.96849030367704) bank166355 +166356 POINT(41.48584131854858 74.22199993518896) bank166356 +166357 POINT(40.62486107449997 73.90847398838986) bank166357 +166358 POINT(41.5838083413545 75.00198364942766) bank166358 +166359 POINT(40.94991884799377 73.90169830587928) bank166359 +166360 POINT(41.62284376953292 74.82429411298138) bank166360 +166361 POINT(40.661782213670726 74.76593643404343) bank166361 +166362 POINT(40.79665541036465 73.90085371256106) bank166362 +166363 POINT(40.933249627752836 73.63899432567098) bank166363 +166364 POINT(40.58888865832812 73.42041963807955) bank166364 +166365 POINT(40.46691628907197 74.27165512062334) bank166365 +166366 POINT(40.11533367490416 74.18875463480106) bank166366 +166367 POINT(39.98958127349277 74.28300367029496) bank166367 +166368 POINT(41.13698685008479 74.0908851138786) bank166368 +166369 POINT(40.24586828622552 74.26318232932157) bank166369 +166370 POINT(41.14135567001816 74.21050437508106) bank166370 +166371 POINT(41.45779604701963 73.73980346197143) bank166371 +166372 POINT(41.61531860707327 74.01901121327751) bank166372 +166373 POINT(40.306983692915416 74.00013929561034) bank166373 +166374 POINT(39.85743771349389 74.32046345786783) bank166374 +166375 POINT(39.92312966869918 73.70949108616261) bank166375 +166376 POINT(40.73081473355285 74.04424593752374) bank166376 +166377 POINT(40.29944769331464 74.54377852406026) bank166377 +166378 POINT(39.914656061874354 74.73401856291395) bank166378 +166379 POINT(39.71740631038731 74.14960200011785) bank166379 +166380 POINT(40.3737911317977 73.15174538546243) bank166380 +166381 POINT(40.8085918962546 74.77106338431598) bank166381 +166382 POINT(41.20262945887145 74.68109975657683) bank166382 +166383 POINT(39.77547232059415 73.72062116725043) bank166383 +166384 POINT(40.56685224712017 73.42656191406047) bank166384 +166385 POINT(41.08947643880204 73.66202640193366) bank166385 +166386 POINT(41.56678601459636 73.79939589660867) bank166386 +166387 POINT(40.50957461769855 73.6656131593643) bank166387 +166388 POINT(41.541811538486236 74.11428159063111) bank166388 +166389 POINT(41.28498628354972 74.52325809949473) bank166389 +166390 POINT(40.36215706098758 73.95715381783542) bank166390 +166391 POINT(40.4957500692987 74.47339861007735) bank166391 +166392 POINT(41.67608373435061 74.87716158035593) bank166392 +166393 POINT(39.89193290269446 73.37429895088758) bank166393 +166394 POINT(41.41394146643331 73.76930280555823) bank166394 +166395 POINT(41.05040322194268 73.3249697770909) bank166395 +166396 POINT(40.74567591350966 74.57061302797483) bank166396 +166397 POINT(40.637012329002225 73.41612763030419) bank166397 +166398 POINT(40.588938677322446 74.98781862416519) bank166398 +166399 POINT(40.93202451072164 73.24461858925453) bank166399 +166400 POINT(40.49978850151624 74.00518560906329) bank166400 +166401 POINT(40.919369287765925 74.26907853718096) bank166401 +166402 POINT(41.19058795788829 74.94047735699596) bank166402 +166403 POINT(39.931187399722624 73.5792359453493) bank166403 +166404 POINT(40.95623558137451 73.6520357648487) bank166404 +166405 POINT(41.388706021313574 73.56415706281055) bank166405 +166406 POINT(39.96394476195812 74.07628885421533) bank166406 +166407 POINT(41.662155953216946 73.05633083539867) bank166407 +166408 POINT(40.35074831009083 74.39326409493437) bank166408 +166409 POINT(41.05105364190208 73.34201263949289) bank166409 +166410 POINT(40.81965213729093 74.59067234728522) bank166410 +166411 POINT(40.39927574003316 74.01798211035265) bank166411 +166412 POINT(40.123592392373666 74.2317571011049) bank166412 +166413 POINT(39.799247298153276 73.17222453039282) bank166413 +166414 POINT(40.57515515350376 73.46867762334406) bank166414 +166415 POINT(40.0561914363495 73.61410542322852) bank166415 +166416 POINT(41.36411711894542 73.35316258696791) bank166416 +166417 POINT(40.58355648186801 74.1050100188357) bank166417 +166418 POINT(41.39710970315319 73.84318599104549) bank166418 +166419 POINT(41.219713280733785 73.98681526065884) bank166419 +166420 POINT(41.10917891139742 74.030489876428) bank166420 +166421 POINT(41.17120308857253 74.5395835266132) bank166421 +166422 POINT(41.4996352002938 74.2223154093998) bank166422 +166423 POINT(41.362148194890395 74.21311492015741) bank166423 +166424 POINT(40.16620455016947 74.9857315236119) bank166424 +166425 POINT(40.309440093374064 74.04358419469406) bank166425 +166426 POINT(40.91209768696646 73.9233439123564) bank166426 +166427 POINT(41.144479160398774 74.44683778088418) bank166427 +166428 POINT(40.11482352743471 73.3274468108258) bank166428 +166429 POINT(41.16632429114729 73.81822081355902) bank166429 +166430 POINT(39.73123237310392 74.16856119184534) bank166430 +166431 POINT(40.655513646131546 73.74908722840662) bank166431 +166432 POINT(40.763038308197906 74.00908808335522) bank166432 +166433 POINT(41.24075340271562 73.26975705030233) bank166433 +166434 POINT(41.295921243611765 74.2606188739089) bank166434 +166435 POINT(41.12239880926031 73.23799063362797) bank166435 +166436 POINT(41.424491838495086 74.36709543163828) bank166436 +166437 POINT(41.71103203205068 73.15080799952351) bank166437 +166438 POINT(40.440796894708676 74.56556737695448) bank166438 +166439 POINT(40.64011678713387 74.31913685350351) bank166439 +166440 POINT(40.931875860746494 73.38562518133111) bank166440 +166441 POINT(40.30036479924649 73.35407059610293) bank166441 +166442 POINT(39.939903608280524 73.61946760549843) bank166442 +166443 POINT(40.36278428001913 73.37922293693862) bank166443 +166444 POINT(41.262159655095815 74.4511206885622) bank166444 +166445 POINT(41.27180894246589 73.58295721177542) bank166445 +166446 POINT(40.20844294497901 74.42788273541726) bank166446 +166447 POINT(40.910888270822724 73.17880057147474) bank166447 +166448 POINT(40.91037542909213 74.83541914786522) bank166448 +166449 POINT(39.75487392384552 74.3098218392976) bank166449 +166450 POINT(41.4925420246404 73.61321597414403) bank166450 +166451 POINT(41.18410128733074 73.22278062344073) bank166451 +166452 POINT(41.28660258712364 74.39539687236238) bank166452 +166453 POINT(41.1126673860901 73.90702446200801) bank166453 +166454 POINT(41.63157047115282 74.8486671525534) bank166454 +166455 POINT(40.71839008111775 73.15732315055438) bank166455 +166456 POINT(41.65504652522277 73.1291280037673) bank166456 +166457 POINT(41.34503042255311 74.4171320820301) bank166457 +166458 POINT(40.20798130485738 73.97865315271457) bank166458 +166459 POINT(39.8727369745125 74.39946270253779) bank166459 +166460 POINT(40.692565994986445 74.25543752285112) bank166460 +166461 POINT(41.09672613366501 74.71110577473527) bank166461 +166462 POINT(40.65113794214198 73.06231051915397) bank166462 +166463 POINT(41.14389065663711 73.5916115626155) bank166463 +166464 POINT(40.73447508371441 73.62033248107065) bank166464 +166465 POINT(40.6578559593592 73.49128088075638) bank166465 +166466 POINT(40.554291666564296 74.5847034135758) bank166466 +166467 POINT(41.594325857516395 74.25657402227449) bank166467 +166468 POINT(40.2716474825914 74.94174777464816) bank166468 +166469 POINT(40.958631713444206 74.86285562415013) bank166469 +166470 POINT(40.24957423607159 74.5847249972277) bank166470 +166471 POINT(40.78740132776917 73.10378804327395) bank166471 +166472 POINT(40.228230587077206 74.81069001877802) bank166472 +166473 POINT(40.907828336618294 73.35334083589376) bank166473 +166474 POINT(40.89662924340097 74.02179951818513) bank166474 +166475 POINT(41.233471909132945 73.57972001725994) bank166475 +166476 POINT(40.13429246695267 73.36467946673746) bank166476 +166477 POINT(40.44070277771246 74.32769476590785) bank166477 +166478 POINT(40.94841329906603 74.78345800701092) bank166478 +166479 POINT(41.47610681025871 74.94159463016982) bank166479 +166480 POINT(41.68904783975107 74.38361878933196) bank166480 +166481 POINT(39.88315379427175 74.64304843816356) bank166481 +166482 POINT(39.76844453543945 73.7453262622402) bank166482 +166483 POINT(41.14219064596359 74.29618223323178) bank166483 +166484 POINT(41.69234928974788 73.89520515144852) bank166484 +166485 POINT(41.48682802719805 73.58167363443484) bank166485 +166486 POINT(40.10557716618433 74.80904746320776) bank166486 +166487 POINT(40.391392697990284 74.5179131207181) bank166487 +166488 POINT(41.07401586285066 74.51420327405874) bank166488 +166489 POINT(40.950373114705776 73.31840795382742) bank166489 +166490 POINT(40.04099456127056 73.56622624633751) bank166490 +166491 POINT(40.9561064004687 74.9448552225879) bank166491 +166492 POINT(39.79415423810636 73.75346979152474) bank166492 +166493 POINT(40.469567038213825 73.18071926068946) bank166493 +166494 POINT(39.72672250108598 74.58284443364715) bank166494 +166495 POINT(41.03564411672886 74.7529788632779) bank166495 +166496 POINT(41.451586437243996 74.93513740950559) bank166496 +166497 POINT(41.637730439395874 73.39035618658875) bank166497 +166498 POINT(41.24335194315373 73.16485331249565) bank166498 +166499 POINT(41.41421742142999 73.28005139792155) bank166499 +166500 POINT(40.98161220107726 74.64731413478064) bank166500 +166501 POINT(40.25627929479114 74.05279295224793) bank166501 +166502 POINT(40.840498788833145 74.39384619188415) bank166502 +166503 POINT(39.977293492103236 74.7158554942987) bank166503 +166504 POINT(40.85767493418375 74.7459673707894) bank166504 +166505 POINT(40.01836959994666 74.06997119787994) bank166505 +166506 POINT(41.23547234228527 74.59560135844173) bank166506 +166507 POINT(40.57766596770233 73.55795969267777) bank166507 +166508 POINT(39.97699954630198 74.7740927958178) bank166508 +166509 POINT(41.125645667174965 74.37441560473597) bank166509 +166510 POINT(39.80249267902044 74.02254392158761) bank166510 +166511 POINT(41.030688239974886 73.60298993586255) bank166511 +166512 POINT(40.43283542208281 74.99691311730126) bank166512 +166513 POINT(41.257114709943245 74.64411320094709) bank166513 +166514 POINT(40.205479133609856 73.86761735053318) bank166514 +166515 POINT(41.2415532345177 74.46214320495717) bank166515 +166516 POINT(40.0412925936812 73.34652326269158) bank166516 +166517 POINT(41.14775216365081 73.75033912697116) bank166517 +166518 POINT(41.663035410248796 74.63989824556468) bank166518 +166519 POINT(41.44776836521454 73.25686350279702) bank166519 +166520 POINT(41.68873704157959 74.28795086754886) bank166520 +166521 POINT(40.51497304514808 74.4551374840622) bank166521 +166522 POINT(40.57733817836164 74.48430266107724) bank166522 +166523 POINT(40.918964362833265 73.20858711571708) bank166523 +166524 POINT(40.30420909934865 74.20268546855196) bank166524 +166525 POINT(41.01512007225966 73.09522079561876) bank166525 +166526 POINT(40.30505488556596 73.24951211585103) bank166526 +166527 POINT(40.23856673415363 73.92963794205666) bank166527 +166528 POINT(41.69942840400041 74.16095011636789) bank166528 +166529 POINT(40.40915046440531 74.06682782865641) bank166529 +166530 POINT(40.785136149921584 74.87920235731339) bank166530 +166531 POINT(40.187751662116675 73.82192652525401) bank166531 +166532 POINT(41.0910707605052 74.6984392532466) bank166532 +166533 POINT(40.11620941348364 73.32381225323276) bank166533 +166534 POINT(40.14269269427835 73.74332345656) bank166534 +166535 POINT(40.170894279794844 74.45887301645783) bank166535 +166536 POINT(40.818298975141616 74.1467792052513) bank166536 +166537 POINT(41.46529911530117 73.05482578292) bank166537 +166538 POINT(39.88282605690207 74.52793826824764) bank166538 +166539 POINT(40.02641017279671 73.77593440551829) bank166539 +166540 POINT(40.69022806374661 73.70305080500891) bank166540 +166541 POINT(40.4819506657222 74.23455951989624) bank166541 +166542 POINT(40.83138967789784 73.14844693456295) bank166542 +166543 POINT(41.282425736019576 74.19497375684689) bank166543 +166544 POINT(40.21081863375122 73.32766507647888) bank166544 +166545 POINT(41.117242099116595 74.8394034959804) bank166545 +166546 POINT(39.809905810411095 73.39845175707823) bank166546 +166547 POINT(41.54492822151651 73.16874553446839) bank166547 +166548 POINT(40.736991950739885 74.11184108195012) bank166548 +166549 POINT(41.503765029628525 73.75361693684154) bank166549 +166550 POINT(40.87788176612617 74.26309898970673) bank166550 +166551 POINT(40.464298274781186 73.09047871077044) bank166551 +166552 POINT(40.95246010293087 74.5905357641712) bank166552 +166553 POINT(40.989111090223865 74.73339425090823) bank166553 +166554 POINT(40.59670535254747 74.84846078764332) bank166554 +166555 POINT(40.74807340779212 73.58916513059744) bank166555 +166556 POINT(40.59375118206321 74.02610448883134) bank166556 +166557 POINT(39.80444238473772 74.19604083856991) bank166557 +166558 POINT(40.432601749087915 73.29535550674223) bank166558 +166559 POINT(40.47197184997921 73.80485009744346) bank166559 +166560 POINT(41.62363495441327 74.63271057609259) bank166560 +166561 POINT(40.368922874843946 74.29008264129278) bank166561 +166562 POINT(41.003206518814515 73.05869753515877) bank166562 +166563 POINT(40.215755536344524 73.1819364632094) bank166563 +166564 POINT(40.39673824392098 74.48116656717355) bank166564 +166565 POINT(41.62903887044234 74.81012142624574) bank166565 +166566 POINT(41.69913380465709 73.76735753978676) bank166566 +166567 POINT(41.24041555937125 73.65738396705392) bank166567 +166568 POINT(41.02941766369146 74.9167973981985) bank166568 +166569 POINT(40.143278842158246 73.63052677961001) bank166569 +166570 POINT(40.995712754367474 74.54362007426695) bank166570 +166571 POINT(41.21169417641276 74.93573827175356) bank166571 +166572 POINT(40.65321166202973 73.17248481840451) bank166572 +166573 POINT(41.52846635771306 74.8458297845104) bank166573 +166574 POINT(40.87941642433802 73.59333849379271) bank166574 +166575 POINT(40.13243954306705 74.02856817306814) bank166575 +166576 POINT(41.406694239755986 73.44292933562005) bank166576 +166577 POINT(40.2243613826476 74.08394486337764) bank166577 +166578 POINT(40.18290780577402 74.12379362942517) bank166578 +166579 POINT(40.698863388732455 73.15575285166604) bank166579 +166580 POINT(40.14234954221237 74.67675295512382) bank166580 +166581 POINT(40.68246959160178 74.78143022731655) bank166581 +166582 POINT(40.54274665932801 74.4937863048497) bank166582 +166583 POINT(40.00660778030266 74.74009735512429) bank166583 +166584 POINT(39.97630043775961 73.87686228410412) bank166584 +166585 POINT(40.65464785558275 73.89359446871352) bank166585 +166586 POINT(41.46582435686721 73.40718756533693) bank166586 +166587 POINT(40.47263387539833 73.03445783996143) bank166587 +166588 POINT(41.27783746898576 73.25304492294022) bank166588 +166589 POINT(40.53780324903265 74.01567318844488) bank166589 +166590 POINT(40.80174094305996 73.71746244699108) bank166590 +166591 POINT(41.0373811036236 74.9874277658615) bank166591 +166592 POINT(40.535495634575945 73.29744276761762) bank166592 +166593 POINT(41.56974021725907 73.59677228691193) bank166593 +166594 POINT(40.811177870028345 74.78532648830472) bank166594 +166595 POINT(41.19473584000987 73.90821589141535) bank166595 +166596 POINT(41.67393245032184 73.134655794156) bank166596 +166597 POINT(40.74259961505753 73.49694717482795) bank166597 +166598 POINT(39.90730540287477 73.92663573210386) bank166598 +166599 POINT(40.89768858482537 73.49478561304385) bank166599 +166600 POINT(41.06664943010459 74.6377956047006) bank166600 +166601 POINT(39.87570269968163 74.74227870163386) bank166601 +166602 POINT(40.00016531017845 73.76753572139702) bank166602 +166603 POINT(40.0788359209056 73.57750958702965) bank166603 +166604 POINT(39.715856472236936 73.6729178645904) bank166604 +166605 POINT(40.24093132785163 73.07811984198169) bank166605 +166606 POINT(41.549707626672 74.33574451040276) bank166606 +166607 POINT(41.41439954392437 74.53170096183312) bank166607 +166608 POINT(40.73022665327963 74.04682473937928) bank166608 +166609 POINT(41.32066881984173 74.7094273631043) bank166609 +166610 POINT(41.37088322110257 74.15989651332372) bank166610 +166611 POINT(40.112713262755584 73.65274513641766) bank166611 +166612 POINT(40.38810461760977 74.96546366709771) bank166612 +166613 POINT(40.139659929081695 73.4713555009206) bank166613 +166614 POINT(40.47102589466866 74.24302811617494) bank166614 +166615 POINT(40.907175259373616 74.71914779471854) bank166615 +166616 POINT(40.14339127735668 73.64280724000722) bank166616 +166617 POINT(40.19725450758753 74.08624237553553) bank166617 +166618 POINT(40.11469728449786 73.65514854197136) bank166618 +166619 POINT(39.843081974243425 73.4429618747911) bank166619 +166620 POINT(40.17537905933601 74.9962265076225) bank166620 +166621 POINT(40.39310905680405 73.26062735520867) bank166621 +166622 POINT(40.53795405840213 74.2687831222746) bank166622 +166623 POINT(41.64566820558028 74.88890794903567) bank166623 +166624 POINT(39.79282713538321 73.93714439550135) bank166624 +166625 POINT(40.671912736915424 74.01029880433855) bank166625 +166626 POINT(40.34216586661996 73.22868261815823) bank166626 +166627 POINT(40.13337042236496 74.0362686551933) bank166627 +166628 POINT(41.041428646980236 73.81134228925991) bank166628 +166629 POINT(41.40684714909632 74.2940549237698) bank166629 +166630 POINT(40.85758786630909 74.77138011641813) bank166630 +166631 POINT(40.330909740997846 73.53155323653206) bank166631 +166632 POINT(40.3718122536779 74.70053916875607) bank166632 +166633 POINT(41.347296973384495 73.30051648553543) bank166633 +166634 POINT(41.34542013974572 73.54450026091685) bank166634 +166635 POINT(39.93731524481959 73.38181906878975) bank166635 +166636 POINT(41.18883976736977 73.64645987039928) bank166636 +166637 POINT(40.44137355120325 74.79671116714842) bank166637 +166638 POINT(41.22710484563076 74.48404955339839) bank166638 +166639 POINT(40.664788560860934 73.55440264989036) bank166639 +166640 POINT(39.8487548427331 74.5622699865041) bank166640 +166641 POINT(40.803474117387054 74.74470581269435) bank166641 +166642 POINT(40.18736245789722 73.23820625346343) bank166642 +166643 POINT(39.76837564357039 74.31097951309114) bank166643 +166644 POINT(41.52959006350331 74.0735607877128) bank166644 +166645 POINT(41.153836574603766 73.79776672624335) bank166645 +166646 POINT(40.72896264053426 73.52947221622239) bank166646 +166647 POINT(39.77913318236567 74.05029618816742) bank166647 +166648 POINT(40.2365677587506 73.83744085115462) bank166648 +166649 POINT(39.808508769851755 74.93118987569471) bank166649 +166650 POINT(41.45311497548872 74.59201770884302) bank166650 +166651 POINT(41.49434193429579 74.23819572850564) bank166651 +166652 POINT(40.06102671546887 74.09654030902435) bank166652 +166653 POINT(40.15376605347768 74.61664843976557) bank166653 +166654 POINT(41.53720297172445 74.00273365959734) bank166654 +166655 POINT(40.10410772975382 74.55817690640066) bank166655 +166656 POINT(40.683531357678255 73.79932023805657) bank166656 +166657 POINT(40.366716582281896 73.68608584150256) bank166657 +166658 POINT(40.14666958794924 73.557772374168) bank166658 +166659 POINT(41.38114549348865 73.71163971175133) bank166659 +166660 POINT(39.925796845104905 73.52804230069886) bank166660 +166661 POINT(41.24793657306281 73.03497974305739) bank166661 +166662 POINT(41.38487194900034 74.93220786637369) bank166662 +166663 POINT(41.430861550184645 74.27568772768389) bank166663 +166664 POINT(39.942367475553716 74.03966792742318) bank166664 +166665 POINT(41.57380839580789 74.18936427699185) bank166665 +166666 POINT(40.1881452395646 74.18141387746653) bank166666 +166667 POINT(40.50931843574162 74.82191625341822) bank166667 +166668 POINT(40.854832820583084 73.85673072044791) bank166668 +166669 POINT(40.43932346538076 73.50254762688554) bank166669 +166670 POINT(41.381855818962116 74.80204227576286) bank166670 +166671 POINT(40.973451472510824 73.25095663612674) bank166671 +166672 POINT(40.08596030593677 74.65822579205074) bank166672 +166673 POINT(41.34638905604292 73.03188460376617) bank166673 +166674 POINT(40.08792384908627 73.59240923814818) bank166674 +166675 POINT(40.1278257013325 74.11255075669521) bank166675 +166676 POINT(40.39777190224639 73.2406492652973) bank166676 +166677 POINT(41.578060605864216 74.40307169853752) bank166677 +166678 POINT(40.8276892330521 74.59524746268431) bank166678 +166679 POINT(41.52923832624434 74.95283839402936) bank166679 +166680 POINT(40.233044505765534 74.19609765733654) bank166680 +166681 POINT(40.969436984137765 74.1447289251922) bank166681 +166682 POINT(39.822311574309246 73.3857292416594) bank166682 +166683 POINT(40.636999110624096 74.5337418879562) bank166683 +166684 POINT(40.6040995594128 74.47625431441107) bank166684 +166685 POINT(41.549723462669064 74.42303116421057) bank166685 +166686 POINT(41.259490974603466 74.92370431235909) bank166686 +166687 POINT(40.066910767219795 74.30830607991349) bank166687 +166688 POINT(40.51744125265648 73.33516210406157) bank166688 +166689 POINT(40.48274360652466 74.67330549288275) bank166689 +166690 POINT(41.21139633286884 73.66867320699566) bank166690 +166691 POINT(41.349660601436995 74.04391202367542) bank166691 +166692 POINT(41.130057034469466 73.27164415104687) bank166692 +166693 POINT(39.8844810717724 73.65855069272044) bank166693 +166694 POINT(39.883560006944414 73.68491236877057) bank166694 +166695 POINT(40.83981976551886 73.23107766472077) bank166695 +166696 POINT(40.7384631593913 73.68221803588341) bank166696 +166697 POINT(39.88002766140428 74.27069740782781) bank166697 +166698 POINT(40.608126187971536 73.44599645888522) bank166698 +166699 POINT(40.43781861424889 73.56736799257132) bank166699 +166700 POINT(40.905362175164186 73.61920353900213) bank166700 +166701 POINT(40.90394036654122 73.12106190516378) bank166701 +166702 POINT(41.41822203680022 74.44842858825503) bank166702 +166703 POINT(41.08242890829273 74.31589405585818) bank166703 +166704 POINT(41.48223303235373 74.54763207778763) bank166704 +166705 POINT(39.89911903009295 73.01248543646234) bank166705 +166706 POINT(41.430707463360974 73.66084269483248) bank166706 +166707 POINT(41.00762273940692 73.61997835552964) bank166707 +166708 POINT(40.809354192069144 73.15137221726135) bank166708 +166709 POINT(40.93605006078141 74.75399042818998) bank166709 +166710 POINT(41.11731477506423 73.75666880052884) bank166710 +166711 POINT(40.06924103124267 73.52990378569227) bank166711 +166712 POINT(41.59173226917728 73.59380341927394) bank166712 +166713 POINT(40.45413198186746 74.41825828704044) bank166713 +166714 POINT(41.28292454659117 73.74544845526259) bank166714 +166715 POINT(40.546348913671295 73.33216015734727) bank166715 +166716 POINT(40.26617580777022 73.79509465474858) bank166716 +166717 POINT(40.612843018201154 74.3490062908528) bank166717 +166718 POINT(41.371603674990965 73.7906627560296) bank166718 +166719 POINT(41.41369121213437 74.41657643813295) bank166719 +166720 POINT(40.27639356156842 74.07891987296564) bank166720 +166721 POINT(40.17012454686559 73.44330800606426) bank166721 +166722 POINT(41.452395451374514 74.56156902538723) bank166722 +166723 POINT(41.345132571848175 74.88991046668754) bank166723 +166724 POINT(41.623530189948305 74.38152932074965) bank166724 +166725 POINT(41.49505788921832 74.05528373505796) bank166725 +166726 POINT(40.13405835003736 74.61586187389004) bank166726 +166727 POINT(41.44011678400389 74.66645986569127) bank166727 +166728 POINT(41.09628608842606 74.68883628266707) bank166728 +166729 POINT(40.16541582759579 74.83805202825421) bank166729 +166730 POINT(40.99282915045498 73.59526912419557) bank166730 +166731 POINT(39.93705344266347 73.7573390504686) bank166731 +166732 POINT(40.97921808027142 74.88005012186105) bank166732 +166733 POINT(40.34117992857843 73.54009133794965) bank166733 +166734 POINT(41.29083767885783 74.60570877783937) bank166734 +166735 POINT(41.64512536484023 74.04695775782157) bank166735 +166736 POINT(41.30006060221657 73.92811629200096) bank166736 +166737 POINT(41.48138550279974 73.16047729321721) bank166737 +166738 POINT(41.24281933854814 73.07619479416265) bank166738 +166739 POINT(41.636407579122505 74.6563248109053) bank166739 +166740 POINT(41.438246842004105 74.55692445938772) bank166740 +166741 POINT(40.06022226412707 74.51419854819608) bank166741 +166742 POINT(40.42604193515218 73.6845231351037) bank166742 +166743 POINT(40.69568135429983 74.32419695452725) bank166743 +166744 POINT(40.92378972448394 74.1340565217981) bank166744 +166745 POINT(41.29981403604832 74.65237036366489) bank166745 +166746 POINT(41.42169828577775 73.85559035796321) bank166746 +166747 POINT(39.84456876007805 74.50412680706377) bank166747 +166748 POINT(40.38048226647997 74.40984900389505) bank166748 +166749 POINT(40.332966191972616 74.69044369524497) bank166749 +166750 POINT(40.60001916520022 74.75798614896726) bank166750 +166751 POINT(41.68434695347652 73.5154892613939) bank166751 +166752 POINT(40.478332582301555 73.33272476832894) bank166752 +166753 POINT(41.17527910979015 74.04742759394586) bank166753 +166754 POINT(41.13756105636558 73.33695510794556) bank166754 +166755 POINT(40.53861215051661 73.89684348663694) bank166755 +166756 POINT(40.55604657827921 73.69351287251273) bank166756 +166757 POINT(39.910564607821996 74.6160694092845) bank166757 +166758 POINT(40.32666600964115 74.17828130608932) bank166758 +166759 POINT(39.751832558141466 73.79558582101602) bank166759 +166760 POINT(40.383269535375504 74.78435461285261) bank166760 +166761 POINT(41.22912216388381 74.46906219153111) bank166761 +166762 POINT(40.135427286664374 74.08572588998064) bank166762 +166763 POINT(41.10479193836734 73.66894972709557) bank166763 +166764 POINT(40.884173029057486 74.72786228574822) bank166764 +166765 POINT(41.409119103946665 74.28017998932837) bank166765 +166766 POINT(41.56178974469202 74.48444403862877) bank166766 +166767 POINT(41.63128919752983 74.306983592354) bank166767 +166768 POINT(40.20764579028135 74.5476949936068) bank166768 +166769 POINT(39.921996817092946 74.95991225655717) bank166769 +166770 POINT(40.07943501400343 73.92180007149311) bank166770 +166771 POINT(41.53824896739812 73.33052158047765) bank166771 +166772 POINT(41.02888368642745 73.7483136392017) bank166772 +166773 POINT(40.201936475465835 74.19092437047941) bank166773 +166774 POINT(40.07516851062202 73.92992125247349) bank166774 +166775 POINT(40.66813266634856 74.99980914898599) bank166775 +166776 POINT(39.86846529715341 73.1605830437487) bank166776 +166777 POINT(41.29023157731077 74.6289772447588) bank166777 +166778 POINT(40.85859041538944 73.8706328522168) bank166778 +166779 POINT(40.68400595557101 74.7148541672457) bank166779 +166780 POINT(41.043681158411744 74.82754793075163) bank166780 +166781 POINT(40.80509696100129 73.85070670322895) bank166781 +166782 POINT(41.08526703521683 73.36695616658581) bank166782 +166783 POINT(40.966292003730636 74.99578847573109) bank166783 +166784 POINT(40.10208443045094 73.32767824599927) bank166784 +166785 POINT(41.209267402175854 74.77419234934473) bank166785 +166786 POINT(40.78330677518164 73.37730139407972) bank166786 +166787 POINT(39.78915059870758 73.47890849938445) bank166787 +166788 POINT(40.933223542244036 74.67854851450308) bank166788 +166789 POINT(40.56813925428257 74.75923202978346) bank166789 +166790 POINT(41.55128993673745 73.99011361068635) bank166790 +166791 POINT(40.54078648806262 73.33546402025233) bank166791 +166792 POINT(41.0039089966423 74.64135536782081) bank166792 +166793 POINT(40.620503286387574 74.31422714173969) bank166793 +166794 POINT(40.76765750094413 74.90953259686752) bank166794 +166795 POINT(41.03447581488909 73.40190763270303) bank166795 +166796 POINT(40.26659332661947 73.05463366934762) bank166796 +166797 POINT(40.349808784238476 73.85460073746836) bank166797 +166798 POINT(39.74788673453691 73.52353957825405) bank166798 +166799 POINT(41.601488731963656 73.41934566403069) bank166799 +166800 POINT(40.574227699636474 73.64967632753589) bank166800 +166801 POINT(40.24614632458154 74.77577863814483) bank166801 +166802 POINT(41.57716826693952 73.93810321937673) bank166802 +166803 POINT(40.59453560443993 73.01007416427593) bank166803 +166804 POINT(40.00277634476156 74.3132606284746) bank166804 +166805 POINT(39.908575304154056 73.43500256827502) bank166805 +166806 POINT(41.363635064151666 74.32878887620784) bank166806 +166807 POINT(41.26880513029547 74.99935343154554) bank166807 +166808 POINT(41.236140059874714 73.3876857448202) bank166808 +166809 POINT(40.81012238251911 74.03065663333369) bank166809 +166810 POINT(41.03295687771034 74.53079714234707) bank166810 +166811 POINT(41.452419553683356 75.00196056889703) bank166811 +166812 POINT(40.45483177736165 73.65138616511189) bank166812 +166813 POINT(41.04820407947158 73.69753858692768) bank166813 +166814 POINT(40.22255309661098 73.79003043178406) bank166814 +166815 POINT(40.382941000179116 73.10196684172828) bank166815 +166816 POINT(39.963533138341745 73.84823518502128) bank166816 +166817 POINT(40.808422348629016 74.45567343205065) bank166817 +166818 POINT(41.487846483320894 74.33770395867661) bank166818 +166819 POINT(40.98493370608974 74.68484468052922) bank166819 +166820 POINT(41.5377642292783 73.98349275488802) bank166820 +166821 POINT(40.41224190307793 74.8020348937908) bank166821 +166822 POINT(39.73134917675567 73.32378439753812) bank166822 +166823 POINT(40.266663991306295 74.81188959755563) bank166823 +166824 POINT(41.00482588983062 74.09966427362716) bank166824 +166825 POINT(39.80640374277575 74.24082641503587) bank166825 +166826 POINT(41.507579507161864 74.84011470481957) bank166826 +166827 POINT(41.1611985644077 73.94122286694036) bank166827 +166828 POINT(39.83540843799051 74.913222910269) bank166828 +166829 POINT(40.619077054594676 74.66072938102702) bank166829 +166830 POINT(41.511859258052766 74.23881484674452) bank166830 +166831 POINT(40.99339364630732 73.23707593880638) bank166831 +166832 POINT(40.47631831161223 74.83054632382833) bank166832 +166833 POINT(40.57656593008499 74.06701063131688) bank166833 +166834 POINT(41.36485315790379 74.43833098554848) bank166834 +166835 POINT(41.382656506247365 73.02929753710288) bank166835 +166836 POINT(41.00947825808865 74.00607020091192) bank166836 +166837 POINT(39.769650884696645 74.93836640583842) bank166837 +166838 POINT(41.638163869557545 74.50301960054436) bank166838 +166839 POINT(41.49196901416806 74.77427590013149) bank166839 +166840 POINT(40.50342046068084 75.00305151620333) bank166840 +166841 POINT(40.760886807651595 74.44470129091405) bank166841 +166842 POINT(41.612996374179744 74.86785311994772) bank166842 +166843 POINT(41.4086688980576 74.15954673956507) bank166843 +166844 POINT(40.10514179458688 74.32355112319131) bank166844 +166845 POINT(40.42766414413919 73.94174961578746) bank166845 +166846 POINT(40.115838593325854 73.28608093992543) bank166846 +166847 POINT(39.956655443480145 73.52578946467695) bank166847 +166848 POINT(40.90646832764821 73.9144315336374) bank166848 +166849 POINT(40.88031703182822 73.7663422327615) bank166849 +166850 POINT(40.246997302952735 74.02120577030438) bank166850 +166851 POINT(41.0540567477121 73.27677543066484) bank166851 +166852 POINT(41.331632021724815 74.85432731382183) bank166852 +166853 POINT(41.04015821087102 73.93292436719754) bank166853 +166854 POINT(41.07098544092456 73.572149106041) bank166854 +166855 POINT(40.146587065529815 73.92043931651962) bank166855 +166856 POINT(40.27271428823873 74.66363445047227) bank166856 +166857 POINT(40.33522558088457 73.56043388832765) bank166857 +166858 POINT(41.44850259414051 74.48440192351867) bank166858 +166859 POINT(39.821519238354234 74.32035672405682) bank166859 +166860 POINT(39.724028982474984 73.28182098872662) bank166860 +166861 POINT(40.76144389749869 74.51857180964602) bank166861 +166862 POINT(40.42533557672965 74.42151165247769) bank166862 +166863 POINT(41.3472299410943 74.35183632360955) bank166863 +166864 POINT(40.79847938491545 74.53289101328298) bank166864 +166865 POINT(39.838424711604695 74.13077536642612) bank166865 +166866 POINT(41.42473792119907 74.7846162901929) bank166866 +166867 POINT(39.89382540050236 74.16171664208863) bank166867 +166868 POINT(40.06429469944276 73.9968400082381) bank166868 +166869 POINT(39.97400607990229 74.96913998714282) bank166869 +166870 POINT(40.63486715144392 73.71511829618171) bank166870 +166871 POINT(39.810527592265814 74.9604547947557) bank166871 +166872 POINT(40.32984984894576 73.08434519076333) bank166872 +166873 POINT(40.257905522076456 73.10417377715798) bank166873 +166874 POINT(40.372216282245205 73.55940696241964) bank166874 +166875 POINT(40.2784569535608 74.8984591988018) bank166875 +166876 POINT(40.59458615119251 74.99726455316546) bank166876 +166877 POINT(41.237456725473166 74.47989483838822) bank166877 +166878 POINT(41.35775723395261 73.7532427026039) bank166878 +166879 POINT(41.62869873262322 73.18515314319428) bank166879 +166880 POINT(40.24071073035039 73.16152786916761) bank166880 +166881 POINT(40.786629226131815 73.79691197836716) bank166881 +166882 POINT(40.29607801374765 73.74897671308543) bank166882 +166883 POINT(41.604102843888796 74.82516393214311) bank166883 +166884 POINT(39.80275647779488 73.99199528548701) bank166884 +166885 POINT(39.890816802200234 74.80236612718168) bank166885 +166886 POINT(41.049872945722 74.27624105038687) bank166886 +166887 POINT(41.63194068409074 74.50407810305231) bank166887 +166888 POINT(41.39208212670369 74.19122774929001) bank166888 +166889 POINT(40.352678606584554 74.40823477764361) bank166889 +166890 POINT(41.13990673521136 73.80610824773629) bank166890 +166891 POINT(40.17441131952244 74.6194299176443) bank166891 +166892 POINT(41.64722944687059 74.14641316777863) bank166892 +166893 POINT(39.9927334308038 73.74058178486473) bank166893 +166894 POINT(40.483525324651936 74.25352587219705) bank166894 +166895 POINT(40.53186690925621 73.93493284884428) bank166895 +166896 POINT(40.44254442465618 73.62997152105793) bank166896 +166897 POINT(40.91905657778573 73.99981591385968) bank166897 +166898 POINT(40.535958443829514 74.23049303830255) bank166898 +166899 POINT(39.89808444227102 74.1842542325006) bank166899 +166900 POINT(39.887323414020486 74.30413647251083) bank166900 +166901 POINT(40.946468077860835 74.43009789399574) bank166901 +166902 POINT(40.64606307397259 73.21147904884869) bank166902 +166903 POINT(41.352065249789625 73.81867109252885) bank166903 +166904 POINT(41.06955778569229 74.68150925623509) bank166904 +166905 POINT(41.56148938884223 73.505011900485) bank166905 +166906 POINT(39.863438477161885 74.84293359940891) bank166906 +166907 POINT(40.485312376028894 74.79499759114509) bank166907 +166908 POINT(39.91356722930227 74.67675070382661) bank166908 +166909 POINT(40.336199179473994 73.92500717660124) bank166909 +166910 POINT(40.0825319704499 74.95981804059977) bank166910 +166911 POINT(40.328486338930304 74.47925032312499) bank166911 +166912 POINT(41.22306558159647 74.70412667702722) bank166912 +166913 POINT(41.64546147405346 74.20703179684514) bank166913 +166914 POINT(40.1177659380716 74.42461802400211) bank166914 +166915 POINT(40.38704508791034 73.51138462370713) bank166915 +166916 POINT(40.97399704891635 74.0572921091237) bank166916 +166917 POINT(39.87529984316449 73.01716250504973) bank166917 +166918 POINT(40.13976781975844 74.59711920152331) bank166918 +166919 POINT(41.06213884768713 74.1565850636981) bank166919 +166920 POINT(39.947156050794426 73.69813655941734) bank166920 +166921 POINT(40.085106181238345 74.01061838772661) bank166921 +166922 POINT(39.98614511894686 73.69121781265818) bank166922 +166923 POINT(39.911702944710626 74.1120102551847) bank166923 +166924 POINT(41.58021359091825 73.35919503813511) bank166924 +166925 POINT(40.144066150477194 74.1294568339448) bank166925 +166926 POINT(40.79202702195526 74.07059321702442) bank166926 +166927 POINT(40.424213855312985 74.40360110676912) bank166927 +166928 POINT(41.3262187335415 74.25054141267371) bank166928 +166929 POINT(40.10489408983895 73.80396627023457) bank166929 +166930 POINT(40.901290395420496 74.44793405804836) bank166930 +166931 POINT(39.81395413230768 74.43205963390601) bank166931 +166932 POINT(39.87252223140636 74.87930088679563) bank166932 +166933 POINT(39.78328791091481 74.56872995980689) bank166933 +166934 POINT(41.442326719762995 74.81922129777607) bank166934 +166935 POINT(41.08412946266547 74.09891814936617) bank166935 +166936 POINT(40.816275582381 73.21124717105303) bank166936 +166937 POINT(40.880786527301794 74.77293750099153) bank166937 +166938 POINT(41.36754010123954 73.60596695460593) bank166938 +166939 POINT(40.420855752753184 74.05719816544156) bank166939 +166940 POINT(40.97184333884209 74.63252305725483) bank166940 +166941 POINT(41.484418801733646 74.71169685757349) bank166941 +166942 POINT(39.72663623693892 74.56473220825414) bank166942 +166943 POINT(39.98176706526596 74.92792518818352) bank166943 +166944 POINT(40.849368249509354 73.61806289037808) bank166944 +166945 POINT(40.52633032407818 74.74829825620017) bank166945 +166946 POINT(41.193682204648255 73.54585655662906) bank166946 +166947 POINT(41.094393133065296 73.92018242374006) bank166947 +166948 POINT(40.75437004104646 74.93071109385238) bank166948 +166949 POINT(41.03975160334005 73.23656189099373) bank166949 +166950 POINT(41.606140437700006 74.7966332096217) bank166950 +166951 POINT(39.878669259362525 74.35659214793533) bank166951 +166952 POINT(40.78958638524313 74.45234456047277) bank166952 +166953 POINT(40.513436945064846 73.31920498893041) bank166953 +166954 POINT(41.172518813410555 73.44522031202543) bank166954 +166955 POINT(41.17426089742564 73.38053204792888) bank166955 +166956 POINT(40.46998557962528 74.79304881800837) bank166956 +166957 POINT(41.2763609913161 74.91029137317287) bank166957 +166958 POINT(40.6796032485784 73.85987679710136) bank166958 +166959 POINT(39.913770542387695 74.47388198429942) bank166959 +166960 POINT(39.90905963030325 73.91661193248248) bank166960 +166961 POINT(41.29502194562346 73.26866482889028) bank166961 +166962 POINT(41.540473004617354 73.7560300841165) bank166962 +166963 POINT(39.94441474227547 74.85487857685182) bank166963 +166964 POINT(40.241817773738276 74.88102123494215) bank166964 +166965 POINT(39.73270129021447 73.84529666219906) bank166965 +166966 POINT(40.578291087914764 74.96574310903016) bank166966 +166967 POINT(41.01695732321878 73.35972437273787) bank166967 +166968 POINT(40.208939184520624 74.8731440885456) bank166968 +166969 POINT(41.47291288767204 73.21741155040557) bank166969 +166970 POINT(41.60383704112703 74.01754656957013) bank166970 +166971 POINT(40.72812104157617 73.10501181227824) bank166971 +166972 POINT(41.44988486492236 74.92260802172534) bank166972 +166973 POINT(41.25089821640557 74.43489448469579) bank166973 +166974 POINT(41.04487897494876 74.83129435129852) bank166974 +166975 POINT(39.911020251875655 73.4444308525076) bank166975 +166976 POINT(39.77147780998058 73.06374934824542) bank166976 +166977 POINT(40.576179533389535 73.55828413994492) bank166977 +166978 POINT(41.17208349793708 73.09475809764852) bank166978 +166979 POINT(40.04116039056824 73.12267345341732) bank166979 +166980 POINT(39.82133221054894 73.93005667988503) bank166980 +166981 POINT(39.85208441130617 73.79039774033998) bank166981 +166982 POINT(39.817381336685436 73.8607134287758) bank166982 +166983 POINT(40.80586368135702 73.87332386163679) bank166983 +166984 POINT(39.89613365867358 74.04564746107765) bank166984 +166985 POINT(40.66731979677025 74.33539982131629) bank166985 +166986 POINT(41.49852338727351 73.6934278323345) bank166986 +166987 POINT(41.19688038434848 74.29454849701217) bank166987 +166988 POINT(40.952434235290134 73.2087525520871) bank166988 +166989 POINT(41.009142027075235 73.77028983068388) bank166989 +166990 POINT(41.101223108581486 73.20554799560928) bank166990 +166991 POINT(40.50192986392666 73.82812953659055) bank166991 +166992 POINT(40.455137364208746 74.5753101581244) bank166992 +166993 POINT(40.570803931507015 74.54951520863774) bank166993 +166994 POINT(41.46211524764622 74.75522694545374) bank166994 +166995 POINT(41.0411597222131 73.08883323197769) bank166995 +166996 POINT(41.1381386462743 73.5730401565868) bank166996 +166997 POINT(40.08006802505906 73.03784042387431) bank166997 +166998 POINT(40.201265608023974 73.5814615856927) bank166998 +166999 POINT(41.270979655138404 74.88601958979021) bank166999 +167000 POINT(39.77888737870009 73.67559200353809) bank167000 +167001 POINT(40.14334598561747 73.88306030370049) bank167001 +167002 POINT(40.339744640079914 73.45694926224527) bank167002 +167003 POINT(41.28227147364168 73.67743658271367) bank167003 +167004 POINT(40.2538089246688 73.37393315550769) bank167004 +167005 POINT(40.632311268142175 73.61740778670888) bank167005 +167006 POINT(40.78457566366992 74.29103986858382) bank167006 +167007 POINT(41.05102270247377 74.65270274041194) bank167007 +167008 POINT(41.01319819691348 73.41332856974634) bank167008 +167009 POINT(41.24673662644238 74.25714324450692) bank167009 +167010 POINT(41.65248274478921 74.8186331269793) bank167010 +167011 POINT(39.90489698756479 73.52976492089878) bank167011 +167012 POINT(40.23084564577492 74.95551002005658) bank167012 +167013 POINT(41.5547601620959 74.6914460341903) bank167013 +167014 POINT(40.75072667541316 74.30111477235124) bank167014 +167015 POINT(40.447468091824184 73.70415890969973) bank167015 +167016 POINT(41.70623386439237 73.23494369925608) bank167016 +167017 POINT(39.95623083130169 73.14252934665748) bank167017 +167018 POINT(40.561586034596004 74.10360088101929) bank167018 +167019 POINT(41.59406581855951 74.34178496101661) bank167019 +167020 POINT(41.108948849700404 74.3594199316894) bank167020 +167021 POINT(41.01914683233934 73.03498267908843) bank167021 +167022 POINT(39.92550449315539 73.57904974166925) bank167022 +167023 POINT(41.157081894093395 74.93733343077574) bank167023 +167024 POINT(41.164861573620854 74.77316850079276) bank167024 +167025 POINT(40.4822822005881 74.32161288642763) bank167025 +167026 POINT(40.315006066658654 74.85017846850232) bank167026 +167027 POINT(41.38832379124191 74.6108760754637) bank167027 +167028 POINT(39.97598031983381 74.8854136185773) bank167028 +167029 POINT(40.71674331464073 73.52763604431979) bank167029 +167030 POINT(41.56008551943756 74.95319173440402) bank167030 +167031 POINT(40.574599247830534 73.79845848304403) bank167031 +167032 POINT(40.40714984394106 73.13196566002625) bank167032 +167033 POINT(40.43653523251929 73.94987493904507) bank167033 +167034 POINT(40.54351302831682 74.01631644289442) bank167034 +167035 POINT(40.11542687692787 74.51034572859379) bank167035 +167036 POINT(41.52180799744859 73.81126651959188) bank167036 +167037 POINT(40.102767996822536 73.14370772369352) bank167037 +167038 POINT(40.537524419853185 73.44032621509851) bank167038 +167039 POINT(41.523991439150045 74.05042825635554) bank167039 +167040 POINT(39.80214458185178 74.55094986370172) bank167040 +167041 POINT(41.20512959414018 74.1612864583461) bank167041 +167042 POINT(39.847141131227374 74.09985965433742) bank167042 +167043 POINT(40.914210851357836 74.98710368244377) bank167043 +167044 POINT(39.842624418707054 73.32624794017086) bank167044 +167045 POINT(41.08990179615636 74.32833977353788) bank167045 +167046 POINT(40.29039049326439 74.5249475897823) bank167046 +167047 POINT(40.06163612713352 74.38735416845992) bank167047 +167048 POINT(40.274902502984936 74.14701832854982) bank167048 +167049 POINT(40.50340180362444 73.50093065755001) bank167049 +167050 POINT(40.62465918691269 73.04952860033863) bank167050 +167051 POINT(40.845665527500586 73.99420926002342) bank167051 +167052 POINT(41.575239324718765 74.5070743510883) bank167052 +167053 POINT(41.4369391891419 74.40159060853587) bank167053 +167054 POINT(39.759519071791786 73.66280177847732) bank167054 +167055 POINT(40.30603420049726 73.5036206245139) bank167055 +167056 POINT(39.85575652212968 74.8768767422215) bank167056 +167057 POINT(39.74744847921918 73.76748839801775) bank167057 +167058 POINT(41.03834267475312 74.06703728670536) bank167058 +167059 POINT(40.03298009850142 74.59703983880564) bank167059 +167060 POINT(40.67558547271465 74.44242999441035) bank167060 +167061 POINT(41.4826425280342 73.90089869153314) bank167061 +167062 POINT(40.32000131007186 74.59417738073579) bank167062 +167063 POINT(39.742956531270224 73.91148583626534) bank167063 +167064 POINT(41.58273168432805 74.48226292301499) bank167064 +167065 POINT(40.86088316117162 73.53971176428637) bank167065 +167066 POINT(41.320777963857005 73.17503259228006) bank167066 +167067 POINT(41.30097704150011 74.15149745695669) bank167067 +167068 POINT(40.02688047070408 74.09778881642083) bank167068 +167069 POINT(41.695837483541226 74.39889862376093) bank167069 +167070 POINT(40.44357962673292 74.585517137847) bank167070 +167071 POINT(41.47174171301003 74.59241602408187) bank167071 +167072 POINT(40.79467710872292 74.85714184001432) bank167072 +167073 POINT(41.05438941630191 74.60392651050228) bank167073 +167074 POINT(40.531957189130395 74.1818518374586) bank167074 +167075 POINT(40.42475639750854 74.71860853099895) bank167075 +167076 POINT(40.26497051032223 74.71056089038734) bank167076 +167077 POINT(40.082595992214195 74.32118592004531) bank167077 +167078 POINT(41.36217082047454 73.96813518642571) bank167078 +167079 POINT(41.32670995370627 74.97863701951509) bank167079 +167080 POINT(40.690117027798124 73.76679447149353) bank167080 +167081 POINT(40.59798479855324 74.83673133287523) bank167081 +167082 POINT(40.36772410715914 74.8243719441111) bank167082 +167083 POINT(40.340789776291736 73.11266584386358) bank167083 +167084 POINT(39.872841718496 74.21459096817055) bank167084 +167085 POINT(41.381347144017106 73.84706207042629) bank167085 +167086 POINT(40.77599937944927 74.16492662986353) bank167086 +167087 POINT(39.773946947633725 73.21798500102683) bank167087 +167088 POINT(41.54336361565037 74.2867299114133) bank167088 +167089 POINT(40.310415035069596 74.85234257085574) bank167089 +167090 POINT(40.59916379863721 73.13895584034292) bank167090 +167091 POINT(39.93684555521235 73.441911649835) bank167091 +167092 POINT(39.82167862307671 73.58414024900608) bank167092 +167093 POINT(41.06020655209984 74.56010733218564) bank167093 +167094 POINT(40.091661736656825 73.20774559438682) bank167094 +167095 POINT(40.1384498540949 74.0510319313255) bank167095 +167096 POINT(40.00107768492103 74.89803797048663) bank167096 +167097 POINT(40.82986614529251 74.57050081091478) bank167097 +167098 POINT(40.79281235758733 74.52055494880825) bank167098 +167099 POINT(41.46145042628163 74.11474026213764) bank167099 +167100 POINT(40.21439244548698 73.14231056367335) bank167100 +167101 POINT(40.672020857866336 74.94642680800575) bank167101 +167102 POINT(39.76640767992067 73.53813318325811) bank167102 +167103 POINT(41.49721456034487 74.17203406598227) bank167103 +167104 POINT(40.11889017962269 74.6893933983291) bank167104 +167105 POINT(40.59159706321707 74.1056443420006) bank167105 +167106 POINT(40.62443582268041 74.93742859487236) bank167106 +167107 POINT(41.280059410069434 74.12149974720936) bank167107 +167108 POINT(41.44052238047813 73.33030675647171) bank167108 +167109 POINT(39.79312212654097 74.63971484249758) bank167109 +167110 POINT(40.045371892380714 73.89489479151898) bank167110 +167111 POINT(40.01298568844139 73.5394350998914) bank167111 +167112 POINT(41.06570185146678 73.58611591175064) bank167112 +167113 POINT(40.2004733285227 74.0556984507702) bank167113 +167114 POINT(39.79443429139102 73.21685641290908) bank167114 +167115 POINT(41.686341394741206 74.22638966349865) bank167115 +167116 POINT(39.87156649857003 73.59294481447444) bank167116 +167117 POINT(41.07183626478255 74.96445309315737) bank167117 +167118 POINT(41.256326082195685 74.10009937281913) bank167118 +167119 POINT(41.00295033525988 74.32539048344127) bank167119 +167120 POINT(41.5710369468551 74.03115824195717) bank167120 +167121 POINT(41.511774001169705 73.37206246672423) bank167121 +167122 POINT(39.955793031388104 73.69744759939883) bank167122 +167123 POINT(41.33106705865324 73.76264609178469) bank167123 +167124 POINT(41.20424237397101 74.77219614739498) bank167124 +167125 POINT(39.95863932806909 74.32101997760023) bank167125 +167126 POINT(40.49062328805321 73.10481526324858) bank167126 +167127 POINT(41.51344715220615 73.34740726114508) bank167127 +167128 POINT(39.74013351983694 73.34010965891471) bank167128 +167129 POINT(40.06745217261003 74.68518240787554) bank167129 +167130 POINT(41.07368257786188 74.60532145372005) bank167130 +167131 POINT(40.410487042649514 74.99005346524646) bank167131 +167132 POINT(41.194504227016445 73.92027337781656) bank167132 +167133 POINT(40.82035399792365 74.06707983786063) bank167133 +167134 POINT(41.24573154584668 73.87262788762791) bank167134 +167135 POINT(41.50895282510545 74.72608704639546) bank167135 +167136 POINT(41.34190669434968 73.03249878870132) bank167136 +167137 POINT(40.59849533525812 74.79384936569876) bank167137 +167138 POINT(39.905020148636545 74.02792261856487) bank167138 +167139 POINT(40.70667957906775 74.33461466945474) bank167139 +167140 POINT(39.99343387464424 74.02791085342152) bank167140 +167141 POINT(39.88562721752434 73.16691486962394) bank167141 +167142 POINT(40.32918327223349 73.98467679383837) bank167142 +167143 POINT(40.22865435554255 74.59318612320702) bank167143 +167144 POINT(39.963396725645254 73.62410538062161) bank167144 +167145 POINT(41.20370979587017 74.8657435078175) bank167145 +167146 POINT(40.591783296517605 73.20360100936024) bank167146 +167147 POINT(40.53008244393808 73.99529398806216) bank167147 +167148 POINT(40.89226968170055 73.6710440232576) bank167148 +167149 POINT(41.211948959944515 73.96128838603964) bank167149 +167150 POINT(40.85160308136035 74.93537260245085) bank167150 +167151 POINT(39.805834653592356 74.85805022995997) bank167151 +167152 POINT(41.23748587090349 73.02057368830963) bank167152 +167153 POINT(40.429069540034625 74.03828688675098) bank167153 +167154 POINT(40.793037230577134 74.7694552063818) bank167154 +167155 POINT(40.21415871758298 74.62317704583212) bank167155 +167156 POINT(39.9708367870901 73.15204232058696) bank167156 +167157 POINT(40.43468779929818 73.76700709245854) bank167157 +167158 POINT(40.782621444187136 73.30102323643838) bank167158 +167159 POINT(41.710120936213634 74.34336560101707) bank167159 +167160 POINT(41.01222384714181 73.2586590551955) bank167160 +167161 POINT(40.96984958511723 73.48376994381097) bank167161 +167162 POINT(40.23536049370389 74.3177477811251) bank167162 +167163 POINT(41.56853306898182 73.05442883314817) bank167163 +167164 POINT(40.52066895255478 74.62447876206164) bank167164 +167165 POINT(40.92607586608041 73.75629932950793) bank167165 +167166 POINT(41.25402437705097 74.71630604690891) bank167166 +167167 POINT(41.282356099535704 73.90215612577664) bank167167 +167168 POINT(40.80980709709025 74.7199624796419) bank167168 +167169 POINT(40.63181879212532 73.27072078138384) bank167169 +167170 POINT(40.14905030752039 73.1207581118879) bank167170 +167171 POINT(40.35737218965105 74.10174301961482) bank167171 +167172 POINT(40.304550939255705 73.17411744616649) bank167172 +167173 POINT(39.718412404153085 74.2580521381638) bank167173 +167174 POINT(40.07486091540123 74.42724675741945) bank167174 +167175 POINT(40.57237426393434 74.74793909851748) bank167175 +167176 POINT(40.18299544037365 73.50782321722771) bank167176 +167177 POINT(40.92267496741243 74.25498102843898) bank167177 +167178 POINT(40.21739045440574 74.92650499514434) bank167178 +167179 POINT(39.987956727586976 74.17955556943501) bank167179 +167180 POINT(41.23900103268081 73.51967515738879) bank167180 +167181 POINT(40.6310147091509 74.33717434038132) bank167181 +167182 POINT(41.43309769010643 73.01599655831495) bank167182 +167183 POINT(41.46826723877329 74.20429925460596) bank167183 +167184 POINT(41.57608410086669 73.91646050722665) bank167184 +167185 POINT(41.052783141194254 74.50184326603309) bank167185 +167186 POINT(40.00714476919513 73.87602753674018) bank167186 +167187 POINT(39.92851782401519 73.11570178752068) bank167187 +167188 POINT(41.47604971136283 73.61659065944299) bank167188 +167189 POINT(40.94752441897061 73.59269909971908) bank167189 +167190 POINT(40.29421468432237 74.16477757137467) bank167190 +167191 POINT(39.93349397255101 74.32278598406849) bank167191 +167192 POINT(40.09948683640969 74.52654183744264) bank167192 +167193 POINT(41.33491349813009 74.08591665920697) bank167193 +167194 POINT(39.9099313349151 73.56739222259942) bank167194 +167195 POINT(41.176038744903344 74.50052494938903) bank167195 +167196 POINT(40.87337379134376 74.75186102356706) bank167196 +167197 POINT(40.56311544737433 74.79046293341621) bank167197 +167198 POINT(41.57133954023877 74.50189361759796) bank167198 +167199 POINT(41.15860959612737 73.99807184372231) bank167199 +167200 POINT(41.44636925147159 74.40919481729863) bank167200 +167201 POINT(40.037554693862404 74.80994666046645) bank167201 +167202 POINT(39.9569934411784 74.90200694218056) bank167202 +167203 POINT(40.141785063236924 74.45285995646154) bank167203 +167204 POINT(40.17660375926444 73.84852043112423) bank167204 +167205 POINT(41.46704703841168 73.7138525364802) bank167205 +167206 POINT(40.85617037081791 74.95235993325926) bank167206 +167207 POINT(40.986738597879345 74.27942128968748) bank167207 +167208 POINT(41.38874122498589 74.23558172067537) bank167208 +167209 POINT(41.67189135184569 74.19041747518112) bank167209 +167210 POINT(40.31532897266932 73.68889627899432) bank167210 +167211 POINT(39.766471262287816 73.29630895730185) bank167211 +167212 POINT(40.6409026973742 74.52494097263272) bank167212 +167213 POINT(40.389778108907606 74.8948400133937) bank167213 +167214 POINT(40.43454294292034 73.60777536181786) bank167214 +167215 POINT(40.26423466094743 73.52165646229257) bank167215 +167216 POINT(41.40239140856802 73.52939856908152) bank167216 +167217 POINT(40.74201160342251 74.77222629153474) bank167217 +167218 POINT(40.821832434654326 74.45016663320726) bank167218 +167219 POINT(41.53769227150966 74.07144247482422) bank167219 +167220 POINT(40.47336450664838 73.7537245172835) bank167220 +167221 POINT(39.75383127784392 74.18881922319446) bank167221 +167222 POINT(41.077827205173605 73.94356288538373) bank167222 +167223 POINT(41.53665475718027 73.68493696862565) bank167223 +167224 POINT(41.18508216906525 73.59976934978918) bank167224 +167225 POINT(40.73155953196753 74.37205076115384) bank167225 +167226 POINT(41.60377685181521 74.6539525016893) bank167226 +167227 POINT(39.8340665670999 73.53606323213594) bank167227 +167228 POINT(40.126945832312344 73.6230435516229) bank167228 +167229 POINT(41.493348536243566 73.03359631701267) bank167229 +167230 POINT(41.226068098292075 73.3520013139543) bank167230 +167231 POINT(40.88052817708362 73.06136621795808) bank167231 +167232 POINT(41.18686292398842 74.71443685552497) bank167232 +167233 POINT(40.272933552954974 73.57461771953706) bank167233 +167234 POINT(41.677925509713276 73.8580816281757) bank167234 +167235 POINT(40.99423045604814 74.89525269006363) bank167235 +167236 POINT(40.28753037199977 74.00464406329388) bank167236 +167237 POINT(39.806010210263565 73.0941185773048) bank167237 +167238 POINT(40.36476612768672 73.14693106941102) bank167238 +167239 POINT(40.90358553682743 74.31665818589086) bank167239 +167240 POINT(40.91899538846427 74.23806589782001) bank167240 +167241 POINT(41.70499175809645 74.46027783764916) bank167241 +167242 POINT(40.09804768288401 74.8664275997734) bank167242 +167243 POINT(41.16152114505904 74.3768833540797) bank167243 +167244 POINT(40.2004008459208 74.96130065092046) bank167244 +167245 POINT(40.34118412247761 73.37753921468085) bank167245 +167246 POINT(40.07949343625283 74.33687077225274) bank167246 +167247 POINT(40.185829706556184 74.70955759310267) bank167247 +167248 POINT(40.407399136902505 74.49782541864731) bank167248 +167249 POINT(41.640553418016765 73.52072789527979) bank167249 +167250 POINT(40.871006323118245 73.17568575739332) bank167250 +167251 POINT(41.135174686931904 74.2655321461265) bank167251 +167252 POINT(40.70288180698877 74.89229379079951) bank167252 +167253 POINT(40.495168729249514 74.93901809313108) bank167253 +167254 POINT(40.74899402346216 74.2208028137417) bank167254 +167255 POINT(41.30620279686766 73.81622510074223) bank167255 +167256 POINT(40.72827660562227 73.50681403288418) bank167256 +167257 POINT(39.71547748988167 73.01712287401834) bank167257 +167258 POINT(41.017685266986625 73.03396378490623) bank167258 +167259 POINT(40.8346776683307 73.97629315495642) bank167259 +167260 POINT(40.096850938850764 74.22026937292131) bank167260 +167261 POINT(40.53227068059027 74.8822346255615) bank167261 +167262 POINT(40.918855882863305 73.32831710087426) bank167262 +167263 POINT(41.489321665573335 73.21479269514614) bank167263 +167264 POINT(41.59858264532202 73.58502861858697) bank167264 +167265 POINT(39.97874092900895 73.93135591600428) bank167265 +167266 POINT(40.188135614820304 73.68897203079008) bank167266 +167267 POINT(41.09456018401624 74.7125498792029) bank167267 +167268 POINT(40.99642905356505 74.72960462723657) bank167268 +167269 POINT(41.0091890052769 74.78993431691977) bank167269 +167270 POINT(40.16061442473947 73.78220126056365) bank167270 +167271 POINT(40.18274853508243 73.4830160780821) bank167271 +167272 POINT(40.7053975124566 73.57938245846233) bank167272 +167273 POINT(41.09407395805929 73.20014486275677) bank167273 +167274 POINT(40.35652369969545 73.30821570817668) bank167274 +167275 POINT(40.44783942318351 73.1750748437158) bank167275 +167276 POINT(39.84539145733183 73.3656532584796) bank167276 +167277 POINT(41.37437112704727 73.36071094808429) bank167277 +167278 POINT(40.008767153464575 73.12401479872759) bank167278 +167279 POINT(40.29007802021364 74.90420475579634) bank167279 +167280 POINT(39.80551451708324 74.18180348745938) bank167280 +167281 POINT(40.21364130647891 74.91054436533507) bank167281 +167282 POINT(40.28509293463766 73.41468065815643) bank167282 +167283 POINT(41.13697526397323 73.21036494481429) bank167283 +167284 POINT(41.15623166740984 73.2206246036212) bank167284 +167285 POINT(41.163530367957996 74.83238171519469) bank167285 +167286 POINT(41.56926602674986 74.14358179131504) bank167286 +167287 POINT(41.088588824226186 73.55531496027709) bank167287 +167288 POINT(40.640819651445575 74.48839084078533) bank167288 +167289 POINT(40.074933536706354 74.05706489669261) bank167289 +167290 POINT(41.56386836844432 73.21438029812494) bank167290 +167291 POINT(41.463022549238026 73.1378265968966) bank167291 +167292 POINT(41.13589654409307 74.94665825162528) bank167292 +167293 POINT(41.43240000927544 74.16878466770413) bank167293 +167294 POINT(39.95818494073806 74.5373221657281) bank167294 +167295 POINT(39.881802472032476 74.87297095817019) bank167295 +167296 POINT(40.40804582491842 74.45772276485711) bank167296 +167297 POINT(40.858495404888586 73.4164940329419) bank167297 +167298 POINT(41.24741263804399 73.77297230353116) bank167298 +167299 POINT(40.759226124432175 74.190669042321) bank167299 +167300 POINT(40.952038031641905 73.49738702849797) bank167300 +167301 POINT(41.55327919703828 74.79155309614552) bank167301 +167302 POINT(41.37875489468325 74.06483782482843) bank167302 +167303 POINT(40.76336399148483 74.46988595423332) bank167303 +167304 POINT(41.452448858546646 73.74537788949479) bank167304 +167305 POINT(41.07237675249238 73.92061251058134) bank167305 +167306 POINT(40.82773395384177 73.34351587185407) bank167306 +167307 POINT(41.087317402699604 73.32145413496723) bank167307 +167308 POINT(40.346401000318394 74.58089356106052) bank167308 +167309 POINT(41.24585131972786 74.97412210152577) bank167309 +167310 POINT(40.574840350918635 73.66325485475468) bank167310 +167311 POINT(40.72928876580522 74.00509361346438) bank167311 +167312 POINT(40.70871922359882 73.21221332961575) bank167312 +167313 POINT(40.09616365506252 73.18746058062796) bank167313 +167314 POINT(40.031896166079946 73.66253401749358) bank167314 +167315 POINT(41.144580542071076 74.59625445417444) bank167315 +167316 POINT(41.06521990822797 73.9702183632302) bank167316 +167317 POINT(41.09701944454006 74.87871577526994) bank167317 +167318 POINT(40.74672970056029 74.98287727680285) bank167318 +167319 POINT(40.44493137611069 74.24878891892872) bank167319 +167320 POINT(40.6911171325061 74.98862609839175) bank167320 +167321 POINT(41.550351955806384 74.64268263344358) bank167321 +167322 POINT(40.80517431167768 73.96998294208177) bank167322 +167323 POINT(40.86797703848853 73.80033928193875) bank167323 +167324 POINT(41.01234175809027 74.41909432162974) bank167324 +167325 POINT(39.85187568673658 73.39760780308973) bank167325 +167326 POINT(40.74087594225022 73.74779862063765) bank167326 +167327 POINT(40.15994313153243 74.26511744737606) bank167327 +167328 POINT(41.17776945795215 74.66512261097323) bank167328 +167329 POINT(41.3665283417119 73.68285590531676) bank167329 +167330 POINT(40.51935503399709 74.86225077953478) bank167330 +167331 POINT(41.456930726790794 73.66275891809605) bank167331 +167332 POINT(40.168927214494666 74.1716299643556) bank167332 +167333 POINT(40.66243713709943 74.1461569412572) bank167333 +167334 POINT(40.8799944658577 73.4581723167557) bank167334 +167335 POINT(40.07850936348341 73.6365854817639) bank167335 +167336 POINT(40.2640527229725 73.2534575196895) bank167336 +167337 POINT(41.51446662268106 73.16331295323593) bank167337 +167338 POINT(40.93704476667191 74.59747942343756) bank167338 +167339 POINT(40.824644909444366 73.82642076787177) bank167339 +167340 POINT(40.284053931664594 73.60202874646004) bank167340 +167341 POINT(39.885457971539736 73.74565704978613) bank167341 +167342 POINT(41.493861441591534 74.90224886247753) bank167342 +167343 POINT(40.71128436918522 73.69469647236059) bank167343 +167344 POINT(41.02961834108445 73.17976494820724) bank167344 +167345 POINT(40.42554816604557 73.86041917395335) bank167345 +167346 POINT(41.233096044918426 74.78548929621678) bank167346 +167347 POINT(40.21217930654626 74.82327137128586) bank167347 +167348 POINT(39.82935376892153 73.23099872076543) bank167348 +167349 POINT(39.71947435837918 74.11389682058379) bank167349 +167350 POINT(39.77295044070496 73.77978970232611) bank167350 +167351 POINT(40.081218506815226 74.58322361132316) bank167351 +167352 POINT(40.32109771278625 74.2342217935556) bank167352 +167353 POINT(40.24256417009696 73.82744159649044) bank167353 +167354 POINT(40.2392231719519 74.9767448328771) bank167354 +167355 POINT(39.80711754846266 73.23119811230065) bank167355 +167356 POINT(41.37143857324543 74.28631241932172) bank167356 +167357 POINT(39.89006743550265 73.61541786897959) bank167357 +167358 POINT(40.015405427315116 74.4005216980457) bank167358 +167359 POINT(40.32388605651909 73.83634349317053) bank167359 +167360 POINT(41.65307911875466 74.96472749408855) bank167360 +167361 POINT(40.99056533264256 73.80982233543858) bank167361 +167362 POINT(39.98444034083461 74.38420403866736) bank167362 +167363 POINT(39.93359407891942 73.09195760991047) bank167363 +167364 POINT(40.974456728726274 74.91574252752277) bank167364 +167365 POINT(41.64881968041788 73.53578175987761) bank167365 +167366 POINT(40.72405346975069 74.82728505243897) bank167366 +167367 POINT(40.02763001291306 74.54416328439194) bank167367 +167368 POINT(40.43061691775614 73.26399514245092) bank167368 +167369 POINT(41.6429490041212 73.48995004893918) bank167369 +167370 POINT(40.45839479264861 74.13069866489083) bank167370 +167371 POINT(40.03780931833316 74.79487469701608) bank167371 +167372 POINT(41.155107670265416 73.03781001017276) bank167372 +167373 POINT(41.16025339593347 74.9019232775639) bank167373 +167374 POINT(41.18364560553065 73.59126337809882) bank167374 +167375 POINT(40.12577607974689 74.01896869378145) bank167375 +167376 POINT(39.755120181674286 74.36228751944209) bank167376 +167377 POINT(40.314128923503695 74.21181137472574) bank167377 +167378 POINT(40.97650308320196 73.80628657282129) bank167378 +167379 POINT(39.821517712217904 74.31707478218952) bank167379 +167380 POINT(40.22258255424935 74.97176010329322) bank167380 +167381 POINT(41.07443607103334 74.61806765449545) bank167381 +167382 POINT(40.61520594593873 74.34647064226871) bank167382 +167383 POINT(41.35200789400184 73.24130781496781) bank167383 +167384 POINT(41.012108244278785 73.13071044481029) bank167384 +167385 POINT(41.12965620384015 73.93488279246931) bank167385 +167386 POINT(41.24585526387661 74.00744305244179) bank167386 +167387 POINT(40.822095681651554 73.81460810614236) bank167387 +167388 POINT(39.8899419795527 74.96512693273094) bank167388 +167389 POINT(41.17406678031964 73.33546764047836) bank167389 +167390 POINT(39.803324775401336 73.22430095434903) bank167390 +167391 POINT(40.71792231754661 73.35857444032315) bank167391 +167392 POINT(40.69145693944773 73.83080993263762) bank167392 +167393 POINT(41.13766881751092 74.24191960963383) bank167393 +167394 POINT(41.17581106515065 73.15043716234463) bank167394 +167395 POINT(39.81396441904131 74.31314631088824) bank167395 +167396 POINT(41.5729628159326 73.57233521615134) bank167396 +167397 POINT(40.55813696326058 74.5477843123721) bank167397 +167398 POINT(40.92937122986681 73.58275370759111) bank167398 +167399 POINT(41.03846646343747 74.3398187262818) bank167399 +167400 POINT(40.06620952991448 74.57755348276712) bank167400 +167401 POINT(41.4002586807435 74.50658425705461) bank167401 +167402 POINT(40.281828935801485 73.06169768820389) bank167402 +167403 POINT(40.991658833311334 73.83165473393136) bank167403 +167404 POINT(41.684832513522224 73.08946099305852) bank167404 +167405 POINT(39.7202011937658 73.16026166261676) bank167405 +167406 POINT(40.26448785261131 74.92083922697947) bank167406 +167407 POINT(41.51510299473869 73.99876991729955) bank167407 +167408 POINT(40.44366991861595 73.642047988973) bank167408 +167409 POINT(41.42245843181419 74.04613856109432) bank167409 +167410 POINT(40.200951769819895 74.76522498045068) bank167410 +167411 POINT(41.59486997207441 73.57218013283736) bank167411 +167412 POINT(41.462575076692715 73.03381705935527) bank167412 +167413 POINT(41.07439120427597 73.42132419711555) bank167413 +167414 POINT(41.496851036751885 74.00036454642361) bank167414 +167415 POINT(40.99550674425349 74.16834382219784) bank167415 +167416 POINT(39.80124821039899 74.6449983400404) bank167416 +167417 POINT(40.00752261369155 74.03335266659032) bank167417 +167418 POINT(41.14265695404496 73.83043372642597) bank167418 +167419 POINT(40.65622691222159 73.62090013365436) bank167419 +167420 POINT(40.4603456966268 73.34858709923756) bank167420 +167421 POINT(41.42578257498078 74.72452301138568) bank167421 +167422 POINT(40.548811522962154 73.0531186888096) bank167422 +167423 POINT(40.47799487753865 73.83731264887722) bank167423 +167424 POINT(40.629028517381826 73.30821419584163) bank167424 +167425 POINT(40.83833766042247 74.59832473547573) bank167425 +167426 POINT(39.89146200974559 74.20672749202934) bank167426 +167427 POINT(40.2776236879742 74.34853982974938) bank167427 +167428 POINT(41.0662005326375 73.70562430723508) bank167428 +167429 POINT(40.036583654027645 73.12873970516245) bank167429 +167430 POINT(41.252507419259416 74.90533010197986) bank167430 +167431 POINT(41.549731683632956 74.3407063365894) bank167431 +167432 POINT(39.96535966400553 73.45439092287751) bank167432 +167433 POINT(40.7214429429047 74.87668434413916) bank167433 +167434 POINT(40.062439901441074 73.94405241281794) bank167434 +167435 POINT(40.63870477620789 74.2457399725653) bank167435 +167436 POINT(41.10856972934664 74.12557170534335) bank167436 +167437 POINT(40.003535054995375 73.75009262361931) bank167437 +167438 POINT(40.85265406102147 73.26439429551688) bank167438 +167439 POINT(41.68058317661768 73.3355794360014) bank167439 +167440 POINT(40.898140073018155 74.13266500814898) bank167440 +167441 POINT(40.904084868888035 73.37278891502585) bank167441 +167442 POINT(40.02386189496431 73.86431818505568) bank167442 +167443 POINT(41.305851200565996 74.91364165867537) bank167443 +167444 POINT(40.51318962062351 73.17707399846915) bank167444 +167445 POINT(40.99883497167225 73.76632978326808) bank167445 +167446 POINT(41.243459650742125 74.33758291177259) bank167446 +167447 POINT(39.973249990836614 74.5992630789334) bank167447 +167448 POINT(40.005342148519055 74.91845990093175) bank167448 +167449 POINT(40.110375919475494 73.82139995635809) bank167449 +167450 POINT(40.12957687595019 74.69906485497484) bank167450 +167451 POINT(40.38525783844366 73.86292906675405) bank167451 +167452 POINT(40.31310672260635 73.5723623919837) bank167452 +167453 POINT(39.73028291045216 73.57200095659996) bank167453 +167454 POINT(41.147607111680784 74.81845177102194) bank167454 +167455 POINT(40.37550367825611 73.21362668495019) bank167455 +167456 POINT(40.43680530640777 74.12569585251657) bank167456 +167457 POINT(39.837941396236985 73.13396895126282) bank167457 +167458 POINT(41.184804357729654 73.55765048240596) bank167458 +167459 POINT(40.04714373085245 73.73187735336707) bank167459 +167460 POINT(41.17268741925902 74.28251460635495) bank167460 +167461 POINT(41.45019382766372 73.04673113267818) bank167461 +167462 POINT(41.135927932888926 74.32660002843862) bank167462 +167463 POINT(41.66380157370447 73.13180850155909) bank167463 +167464 POINT(40.21620967319113 73.14353347430567) bank167464 +167465 POINT(40.27860747038794 73.45107552765839) bank167465 +167466 POINT(41.43263234208097 74.4598514718134) bank167466 +167467 POINT(40.27565287360221 74.79069829571516) bank167467 +167468 POINT(39.80306648714388 73.67144182073106) bank167468 +167469 POINT(41.016361896190716 74.9386902971107) bank167469 +167470 POINT(41.43166771155086 73.46953811138243) bank167470 +167471 POINT(40.9971976362054 73.98017080931446) bank167471 +167472 POINT(39.77600415209422 74.03794099120594) bank167472 +167473 POINT(40.1151769840313 73.93975115563886) bank167473 +167474 POINT(40.597527446123884 74.78395165469357) bank167474 +167475 POINT(40.81411797235232 74.89721673519094) bank167475 +167476 POINT(40.401835297385716 74.5686045687071) bank167476 +167477 POINT(40.5940195416865 74.85692502077738) bank167477 +167478 POINT(41.42091901413436 73.58574611452201) bank167478 +167479 POINT(40.001668910010736 73.02435709384793) bank167479 +167480 POINT(40.55142179795892 73.882978024384) bank167480 +167481 POINT(40.51020530897148 74.0550817963185) bank167481 +167482 POINT(41.424662097170014 74.41761454979729) bank167482 +167483 POINT(39.79828806310481 73.21520350890128) bank167483 +167484 POINT(39.83920356758115 73.39472715313649) bank167484 +167485 POINT(39.820137550454895 73.89551026766688) bank167485 +167486 POINT(40.94393058992368 73.08403006772868) bank167486 +167487 POINT(41.14672845120356 73.18578819290502) bank167487 +167488 POINT(40.36195699826986 73.0747611648178) bank167488 +167489 POINT(41.549242796373136 73.30036456912639) bank167489 +167490 POINT(41.41531648565409 74.57062032466736) bank167490 +167491 POINT(40.236678237364096 73.78072724190459) bank167491 +167492 POINT(41.53594739550639 74.21845120670186) bank167492 +167493 POINT(39.997377270812805 74.94908485726538) bank167493 +167494 POINT(40.6095593675764 74.15465989649043) bank167494 +167495 POINT(39.93792553829988 73.03403230557979) bank167495 +167496 POINT(41.01252554776805 74.27598535757127) bank167496 +167497 POINT(41.005647889303674 74.22619632244069) bank167497 +167498 POINT(40.50860067717057 74.48292025230978) bank167498 +167499 POINT(40.88666575573978 74.80984675334749) bank167499 +167500 POINT(40.94202261531884 73.20085025712957) bank167500 +167501 POINT(41.231368435195655 73.61108585086834) bank167501 +167502 POINT(41.51427918338964 73.34302865055244) bank167502 +167503 POINT(39.75095873557461 73.2259003617229) bank167503 +167504 POINT(41.65707273624459 74.05024318557955) bank167504 +167505 POINT(40.279300470441704 73.31643585739685) bank167505 +167506 POINT(39.95165862457088 74.48966754889948) bank167506 +167507 POINT(41.61389691966139 74.03033078554475) bank167507 +167508 POINT(41.092601701230215 73.61748798917615) bank167508 +167509 POINT(40.44217472326012 73.63844118456682) bank167509 +167510 POINT(40.924867103260375 73.94776071960757) bank167510 +167511 POINT(39.71610483704909 73.20496824226157) bank167511 +167512 POINT(41.47797825923481 73.56012736839084) bank167512 +167513 POINT(40.14914197178502 73.49720878155301) bank167513 +167514 POINT(40.46261956644754 74.07986028746836) bank167514 +167515 POINT(41.36542717747808 74.89840013546358) bank167515 +167516 POINT(41.157557656859154 74.31683598142747) bank167516 +167517 POINT(40.70719932700614 73.39839612624749) bank167517 +167518 POINT(39.98455965842242 74.79906738359855) bank167518 +167519 POINT(39.78377099072348 73.73487869297549) bank167519 +167520 POINT(39.791921547092954 74.6604085178108) bank167520 +167521 POINT(40.40296105034671 73.02624284510759) bank167521 +167522 POINT(41.16871432469 73.20106946989802) bank167522 +167523 POINT(41.11692018479565 73.00856658526905) bank167523 +167524 POINT(41.47464741615794 74.5943904447202) bank167524 +167525 POINT(40.906619561920245 74.81171375985018) bank167525 +167526 POINT(41.55844542556043 74.74377227250855) bank167526 +167527 POINT(40.59967452954257 73.88571772974821) bank167527 +167528 POINT(41.01238691787638 74.57207714782824) bank167528 +167529 POINT(39.79228138177902 74.43700082832628) bank167529 +167530 POINT(40.241961072200304 74.42414604461587) bank167530 +167531 POINT(40.26104050037065 73.94857174546819) bank167531 +167532 POINT(41.6109017079631 74.00030114852682) bank167532 +167533 POINT(41.40223128485975 74.01400195636178) bank167533 +167534 POINT(40.557906394706656 74.73794938460505) bank167534 +167535 POINT(41.12107549042097 73.43507311247473) bank167535 +167536 POINT(40.08802962613856 74.92552972196192) bank167536 +167537 POINT(40.13512572636023 73.40404187725922) bank167537 +167538 POINT(41.44288004793563 73.89951070205305) bank167538 +167539 POINT(40.25414901883152 74.66424699101714) bank167539 +167540 POINT(40.027140984936906 73.80725956349276) bank167540 +167541 POINT(39.75504278272157 73.90451953085179) bank167541 +167542 POINT(40.5747972088812 73.6408535664889) bank167542 +167543 POINT(40.959406225776824 74.92408977939722) bank167543 +167544 POINT(40.11405519132504 73.30452062759188) bank167544 +167545 POINT(41.58172494618264 73.25264180385581) bank167545 +167546 POINT(40.91350492731807 74.41525729801968) bank167546 +167547 POINT(41.02968924683624 74.70921233745239) bank167547 +167548 POINT(39.85994700843263 74.30791515979502) bank167548 +167549 POINT(40.9936251371871 73.87666659807556) bank167549 +167550 POINT(40.558856777692476 73.38015863123803) bank167550 +167551 POINT(41.21747350152792 73.74035071014609) bank167551 +167552 POINT(39.90354281635731 74.49813631435602) bank167552 +167553 POINT(39.88318337932849 73.2207039079571) bank167553 +167554 POINT(41.04268204028512 73.89846661808589) bank167554 +167555 POINT(41.510180026111755 74.17992620331522) bank167555 +167556 POINT(40.17975122068945 74.06340902854912) bank167556 +167557 POINT(40.82257870368449 73.18675787252918) bank167557 +167558 POINT(39.91233721815964 74.77274805611248) bank167558 +167559 POINT(40.6042105827907 73.75493748838903) bank167559 +167560 POINT(40.07542301472729 74.37874761695859) bank167560 +167561 POINT(41.1686399346475 73.42260460509912) bank167561 +167562 POINT(40.663906422819736 74.30445183993785) bank167562 +167563 POINT(39.821567148963915 74.46581124558978) bank167563 +167564 POINT(39.7261413607512 73.20788642884126) bank167564 +167565 POINT(41.09189457226833 73.31303827074373) bank167565 +167566 POINT(40.54833116626671 74.28415171092053) bank167566 +167567 POINT(41.030559147760336 74.19502823164741) bank167567 +167568 POINT(40.813541934873726 74.37268121031349) bank167568 +167569 POINT(40.414857344988356 74.24587941886126) bank167569 +167570 POINT(40.4577346303069 74.13395208639034) bank167570 +167571 POINT(41.43001041433146 74.32697950430784) bank167571 +167572 POINT(40.76717540522 74.58304846467364) bank167572 +167573 POINT(41.61051669685487 73.06464384801073) bank167573 +167574 POINT(40.366710247243184 74.01984715456564) bank167574 +167575 POINT(40.55893068460842 73.85788065318889) bank167575 +167576 POINT(39.97251737548139 73.4748135824389) bank167576 +167577 POINT(41.421572395233056 73.37321099843412) bank167577 +167578 POINT(40.89693278576036 73.52789081451543) bank167578 +167579 POINT(41.082903399521385 73.36822685404017) bank167579 +167580 POINT(39.82089322900874 73.4707448415765) bank167580 +167581 POINT(41.3753697759916 73.80884859075138) bank167581 +167582 POINT(40.25532646183317 73.47563239841743) bank167582 +167583 POINT(39.84127728178294 74.67345954178589) bank167583 +167584 POINT(40.76585256693781 73.45653001706339) bank167584 +167585 POINT(41.244221777133326 73.91840437688407) bank167585 +167586 POINT(40.8526385747836 73.93495341088767) bank167586 +167587 POINT(40.949029443278896 73.95438244668733) bank167587 +167588 POINT(41.37128968343132 73.84462540202944) bank167588 +167589 POINT(40.12337593233517 74.77528396111853) bank167589 +167590 POINT(40.78026592005455 73.63794317538112) bank167590 +167591 POINT(40.723772225387364 74.40985358170994) bank167591 +167592 POINT(41.506100567005106 73.1933634544421) bank167592 +167593 POINT(41.096725174392766 73.59146725277724) bank167593 +167594 POINT(40.77543340485306 74.79802647167065) bank167594 +167595 POINT(40.364221097306086 74.1047251749403) bank167595 +167596 POINT(40.93729281150687 74.70738824730766) bank167596 +167597 POINT(39.861682429852905 73.45857932958023) bank167597 +167598 POINT(39.96799140118731 74.05767304775554) bank167598 +167599 POINT(40.516130555368726 73.34233485210821) bank167599 +167600 POINT(40.863820921910566 74.58733542985179) bank167600 +167601 POINT(41.089263324265595 74.02580853850458) bank167601 +167602 POINT(41.22134814116879 73.25030739781957) bank167602 +167603 POINT(41.11807890232575 74.78272161400506) bank167603 +167604 POINT(40.379929837479054 74.86047591868265) bank167604 +167605 POINT(41.42755911762614 74.63798263390346) bank167605 +167606 POINT(40.333594988683714 74.76190830555007) bank167606 +167607 POINT(40.641297663103096 74.30311239030742) bank167607 +167608 POINT(39.81149862464134 74.95323642589395) bank167608 +167609 POINT(40.039507348644314 74.60100821825722) bank167609 +167610 POINT(41.6498462188914 73.48848691143868) bank167610 +167611 POINT(41.601132421983635 73.33222115256683) bank167611 +167612 POINT(40.13694574243341 73.09207435263835) bank167612 +167613 POINT(41.610944239382455 74.61007983398129) bank167613 +167614 POINT(41.30253854670404 73.19429145772763) bank167614 +167615 POINT(41.68951147561519 74.06965669962294) bank167615 +167616 POINT(40.59168059452719 73.52216248911434) bank167616 +167617 POINT(41.37500992050931 74.90768696536507) bank167617 +167618 POINT(40.091396369275266 74.73328489400251) bank167618 +167619 POINT(40.201965942518804 73.27870514902082) bank167619 +167620 POINT(41.082422606433816 73.95246648604446) bank167620 +167621 POINT(41.465983817619566 73.63718741674738) bank167621 +167622 POINT(40.79682074572796 74.13109918290526) bank167622 +167623 POINT(41.40789750962749 73.88325491637721) bank167623 +167624 POINT(41.14736192732285 73.5108083411158) bank167624 +167625 POINT(40.75300666742426 74.98608061387917) bank167625 +167626 POINT(40.29482214081157 73.94523159244328) bank167626 +167627 POINT(41.678536864385535 73.81320801310396) bank167627 +167628 POINT(41.2145969828607 73.30259892044819) bank167628 +167629 POINT(40.619152434800114 73.01959753665642) bank167629 +167630 POINT(39.96632503600038 73.29497508858785) bank167630 +167631 POINT(40.577389839927605 74.6997701358587) bank167631 +167632 POINT(41.310498115221286 73.81743791141015) bank167632 +167633 POINT(40.48990953709356 73.07567176475264) bank167633 +167634 POINT(40.112197078901616 73.14589491052504) bank167634 +167635 POINT(41.40789364230871 73.46601004765621) bank167635 +167636 POINT(40.523888778918746 73.75521712636498) bank167636 +167637 POINT(40.75582875291973 74.37981963812409) bank167637 +167638 POINT(39.99215950240581 73.97466682665306) bank167638 +167639 POINT(41.364917647598666 74.9384972848484) bank167639 +167640 POINT(40.9762822906582 74.91346244880764) bank167640 +167641 POINT(40.450127816397185 74.54639285517258) bank167641 +167642 POINT(40.19732692858575 73.0400454292919) bank167642 +167643 POINT(41.574692357203034 73.32592513088959) bank167643 +167644 POINT(40.8299424563072 73.04592540192279) bank167644 +167645 POINT(40.79404327699161 73.82170320353681) bank167645 +167646 POINT(41.01014932377378 73.05336644849568) bank167646 +167647 POINT(40.19481971196177 73.22399127932654) bank167647 +167648 POINT(40.98055163909018 73.33348357762414) bank167648 +167649 POINT(39.95688635876772 73.25727537294043) bank167649 +167650 POINT(40.16338146776781 74.48710977685138) bank167650 +167651 POINT(40.935299802980005 74.89862153299227) bank167651 +167652 POINT(41.379663539176335 73.39507774541183) bank167652 +167653 POINT(40.75029621715886 74.1883154509855) bank167653 +167654 POINT(40.244778609844325 73.0631869795996) bank167654 +167655 POINT(41.53887641348024 74.85101639035405) bank167655 +167656 POINT(39.746342079242964 73.37774019163811) bank167656 +167657 POINT(39.94345837543092 74.799944243595) bank167657 +167658 POINT(39.880857866471025 73.25989738394269) bank167658 +167659 POINT(41.41047872957743 74.73215730442438) bank167659 +167660 POINT(39.874009706385785 73.17547263509245) bank167660 +167661 POINT(40.22629922658688 73.28402927292706) bank167661 +167662 POINT(41.01706355471074 73.89500602421593) bank167662 +167663 POINT(40.28187976737273 74.4255415863064) bank167663 +167664 POINT(40.626563363614274 74.77915309289986) bank167664 +167665 POINT(40.79693633018276 73.78814884224946) bank167665 +167666 POINT(41.199360508500426 73.44520754009714) bank167666 +167667 POINT(40.91649266227859 74.41556204833071) bank167667 +167668 POINT(41.2018041827124 74.14740157730236) bank167668 +167669 POINT(40.47063714416378 73.41035621929326) bank167669 +167670 POINT(40.67548447241284 74.07155808606132) bank167670 +167671 POINT(40.2322467217975 73.52113505581137) bank167671 +167672 POINT(40.91011707093426 73.35141616179727) bank167672 +167673 POINT(39.776650568266206 74.3321447980043) bank167673 +167674 POINT(40.87176945850613 74.2854803566334) bank167674 +167675 POINT(40.40351814634534 74.90018946075037) bank167675 +167676 POINT(40.83386018616214 73.1691730265853) bank167676 +167677 POINT(40.39979990855039 73.82963464724702) bank167677 +167678 POINT(39.796787275295124 74.670639568933) bank167678 +167679 POINT(41.154168549927036 74.75629880504786) bank167679 +167680 POINT(39.72202314312792 74.06213256865333) bank167680 +167681 POINT(41.28904075059557 73.03892137543409) bank167681 +167682 POINT(41.32512593490548 74.85783873930393) bank167682 +167683 POINT(40.58422349260459 74.62942282336435) bank167683 +167684 POINT(39.75307901585147 73.93570001277519) bank167684 +167685 POINT(41.029600213034364 73.91935609649323) bank167685 +167686 POINT(41.13200635891491 74.88469931126542) bank167686 +167687 POINT(40.61425361111018 74.39264494223991) bank167687 +167688 POINT(40.944393776025485 74.693797481191) bank167688 +167689 POINT(40.92415093611351 73.86332029551104) bank167689 +167690 POINT(39.75708744639651 74.88629768092235) bank167690 +167691 POINT(39.918827708718815 74.79682088537778) bank167691 +167692 POINT(40.82318737580643 74.7947125571713) bank167692 +167693 POINT(41.060208189229 74.13953321475636) bank167693 +167694 POINT(40.400281957555826 73.13138472989283) bank167694 +167695 POINT(40.6762586468419 74.61815436047696) bank167695 +167696 POINT(40.4320393397166 74.3089926544588) bank167696 +167697 POINT(40.727370420723894 73.05610603338577) bank167697 +167698 POINT(39.81459808173019 74.29253462552032) bank167698 +167699 POINT(40.007263920258154 74.71198617216014) bank167699 +167700 POINT(40.603114780458725 74.43441274740084) bank167700 +167701 POINT(41.16471215713825 73.47172151309775) bank167701 +167702 POINT(40.34381976476407 73.71964349336471) bank167702 +167703 POINT(41.601668498656686 73.63147851615273) bank167703 +167704 POINT(41.27648781983598 74.04665019584623) bank167704 +167705 POINT(39.88367373621129 73.96155356206516) bank167705 +167706 POINT(40.56143672409544 73.04162134649135) bank167706 +167707 POINT(40.166144812570366 74.37498430026032) bank167707 +167708 POINT(40.81308569782303 73.75866579592137) bank167708 +167709 POINT(40.51424976138514 74.5462780929582) bank167709 +167710 POINT(40.76817668214426 73.35005698253728) bank167710 +167711 POINT(40.260025128927445 74.3539879858938) bank167711 +167712 POINT(40.751504816502134 73.08726452743363) bank167712 +167713 POINT(41.527556714072425 73.44354999629901) bank167713 +167714 POINT(40.789350550814675 74.70566459714566) bank167714 +167715 POINT(41.58211684167709 74.36630449747959) bank167715 +167716 POINT(39.81378638781336 74.80092198252298) bank167716 +167717 POINT(41.43129557704732 73.6623920345503) bank167717 +167718 POINT(41.468344903398844 74.10486443262472) bank167718 +167719 POINT(39.81254576802558 73.57240536932406) bank167719 +167720 POINT(40.776849385285914 73.35374981160086) bank167720 +167721 POINT(40.50900789247997 73.23505348822749) bank167721 +167722 POINT(40.65016671975417 74.43138818580181) bank167722 +167723 POINT(41.598537250620296 74.9810465572901) bank167723 +167724 POINT(40.88139174611986 74.3653011603826) bank167724 +167725 POINT(40.031710882162514 74.68174735449837) bank167725 +167726 POINT(41.673397957160915 73.25717175227788) bank167726 +167727 POINT(41.15197124848313 74.25259843755488) bank167727 +167728 POINT(40.78758144143051 74.35643776582708) bank167728 +167729 POINT(40.514331118673496 73.25941201590575) bank167729 +167730 POINT(40.6940943563293 74.96040052032782) bank167730 +167731 POINT(40.63186972939303 73.45848218334244) bank167731 +167732 POINT(41.48914557194866 73.73047019790705) bank167732 +167733 POINT(40.04242571906266 74.37995510254223) bank167733 +167734 POINT(40.584180452758325 74.37525479858391) bank167734 +167735 POINT(40.02741216674015 74.52866715708676) bank167735 +167736 POINT(41.50966687643319 74.39386505283969) bank167736 +167737 POINT(40.845727810590176 73.93439399484672) bank167737 +167738 POINT(40.59744504256494 73.65739076843116) bank167738 +167739 POINT(40.218274099245534 73.78989478146484) bank167739 +167740 POINT(40.41661114933531 73.41408230435698) bank167740 +167741 POINT(40.54122813632709 73.77029308004167) bank167741 +167742 POINT(40.82587099161871 74.67187235882702) bank167742 +167743 POINT(40.89101068654955 73.14791216218129) bank167743 +167744 POINT(40.07503858522743 74.9391984835095) bank167744 +167745 POINT(41.518734461898426 73.92133359922333) bank167745 +167746 POINT(41.18923516401063 73.68025567804376) bank167746 +167747 POINT(41.00342236383006 74.73716112550946) bank167747 +167748 POINT(41.43627897037803 73.19044323834326) bank167748 +167749 POINT(40.90531949973675 74.47616220049166) bank167749 +167750 POINT(40.344075078331564 74.90690002665427) bank167750 +167751 POINT(40.09818414415681 73.47958091647469) bank167751 +167752 POINT(40.40378645573122 74.95452741838768) bank167752 +167753 POINT(41.13381142133729 73.04424042138413) bank167753 +167754 POINT(41.06541203463297 74.06240901623505) bank167754 +167755 POINT(39.72897459899517 74.47723205956022) bank167755 +167756 POINT(40.34922095633804 73.30172864875435) bank167756 +167757 POINT(40.31378588360114 74.05292747438254) bank167757 +167758 POINT(40.44498727331321 74.25837708932582) bank167758 +167759 POINT(41.41534783009139 73.05146738365852) bank167759 +167760 POINT(41.56216225417573 74.94579773534905) bank167760 +167761 POINT(41.696734455427176 74.23882986411964) bank167761 +167762 POINT(40.944674191686595 74.29031867157816) bank167762 +167763 POINT(41.58626746276421 74.86148816385663) bank167763 +167764 POINT(40.33344574568131 73.17811967368631) bank167764 +167765 POINT(40.42436870263697 73.25095877482865) bank167765 +167766 POINT(41.21133149277057 73.0963727772463) bank167766 +167767 POINT(40.81445953691913 74.01461522691) bank167767 +167768 POINT(41.3329261666511 73.56702951366326) bank167768 +167769 POINT(40.38368612344848 73.74361619414816) bank167769 +167770 POINT(40.57011215467286 73.9339437583782) bank167770 +167771 POINT(40.486837676857924 73.90609406094319) bank167771 +167772 POINT(41.59316329334967 74.16117985626353) bank167772 +167773 POINT(40.22970571301783 74.79406055961527) bank167773 +167774 POINT(40.94323449321168 73.94827880198078) bank167774 +167775 POINT(41.506664444572536 74.36659329234588) bank167775 +167776 POINT(40.28085497565276 74.05709949694895) bank167776 +167777 POINT(40.417830465435536 73.95344736686361) bank167777 +167778 POINT(41.29074409909181 74.85315301120309) bank167778 +167779 POINT(39.85480143807227 74.38375157577988) bank167779 +167780 POINT(41.091182948121975 74.32467533048583) bank167780 +167781 POINT(40.821659904985964 74.95706822533708) bank167781 +167782 POINT(40.400913686695716 73.14168353235225) bank167782 +167783 POINT(40.299073592513196 73.35832488868304) bank167783 +167784 POINT(41.3293808676213 74.72059497406883) bank167784 +167785 POINT(40.09443400563259 74.59191660663785) bank167785 +167786 POINT(41.32706704146903 73.19368890938523) bank167786 +167787 POINT(40.189450664460495 73.42191266863821) bank167787 +167788 POINT(40.719789452640285 73.207670271889) bank167788 +167789 POINT(40.75230035079166 74.05783682578786) bank167789 +167790 POINT(40.95362737848873 74.05267995406521) bank167790 +167791 POINT(40.91724816652454 74.70986787664575) bank167791 +167792 POINT(40.41259786218363 74.86170587893096) bank167792 +167793 POINT(39.890556665227756 73.73510788438851) bank167793 +167794 POINT(40.350246424100476 73.24260019494758) bank167794 +167795 POINT(41.61589628415964 74.81449504528877) bank167795 +167796 POINT(40.05429471065116 74.48474400790244) bank167796 +167797 POINT(40.75527319413955 73.68085027647155) bank167797 +167798 POINT(40.25972444819341 73.75904964452339) bank167798 +167799 POINT(41.678266427408744 74.52681905267134) bank167799 +167800 POINT(40.793959178159696 73.71524095188322) bank167800 +167801 POINT(41.504004284511545 74.70361068647533) bank167801 +167802 POINT(40.0520052229447 74.3276492817906) bank167802 +167803 POINT(40.287459955484806 73.6495197550389) bank167803 +167804 POINT(41.37828117397458 73.76470673687943) bank167804 +167805 POINT(40.242247253508104 73.92502962973234) bank167805 +167806 POINT(40.04985028034429 74.68213762257288) bank167806 +167807 POINT(39.929795554252884 74.58075347259035) bank167807 +167808 POINT(40.67684235448386 74.94027964011595) bank167808 +167809 POINT(41.590732355696716 74.77647075470155) bank167809 +167810 POINT(41.48199811462315 73.36080871661305) bank167810 +167811 POINT(40.62773282330178 73.06510283911872) bank167811 +167812 POINT(39.78086749037547 74.5526630895447) bank167812 +167813 POINT(39.894606080798795 73.04532939460391) bank167813 +167814 POINT(39.84091185388642 74.55431057485647) bank167814 +167815 POINT(40.970813112802226 74.27424256535323) bank167815 +167816 POINT(40.18523103787725 73.68714484314685) bank167816 +167817 POINT(41.4259224388608 73.49922966933558) bank167817 +167818 POINT(41.43602012889237 73.5240579379022) bank167818 +167819 POINT(40.253742201593994 73.91642857239374) bank167819 +167820 POINT(41.14953275221471 73.51165223686381) bank167820 +167821 POINT(41.49528903717286 73.50629078452864) bank167821 +167822 POINT(40.44216181552415 73.52516939126134) bank167822 +167823 POINT(41.045530602371784 74.29512411741209) bank167823 +167824 POINT(41.27309766021793 74.34697553476654) bank167824 +167825 POINT(40.28365026726879 74.12539341317631) bank167825 +167826 POINT(41.337137496718285 73.70561099847794) bank167826 +167827 POINT(40.2363818910341 73.40254950092881) bank167827 +167828 POINT(40.95490612644136 74.99989792886431) bank167828 +167829 POINT(40.69370687916063 73.50464291060698) bank167829 +167830 POINT(40.029586191214705 73.35815720888777) bank167830 +167831 POINT(40.36264441532787 74.77913268306317) bank167831 +167832 POINT(41.55004191910321 74.69963758101453) bank167832 +167833 POINT(40.69130597321462 73.85663073683696) bank167833 +167834 POINT(40.459339306626156 73.64451139194459) bank167834 +167835 POINT(39.88777259244227 73.62787836645995) bank167835 +167836 POINT(40.55310611936277 73.88751324638885) bank167836 +167837 POINT(39.99702842464705 73.96273625491371) bank167837 +167838 POINT(40.59486290436184 73.41180305448565) bank167838 +167839 POINT(41.43704351346787 73.05052851530014) bank167839 +167840 POINT(40.56211064273419 74.4026450714792) bank167840 +167841 POINT(41.19622978664158 73.21191567506284) bank167841 +167842 POINT(40.96535938465758 74.4108257052132) bank167842 +167843 POINT(41.27023390005679 74.81980745279594) bank167843 +167844 POINT(39.763700961683334 73.6498307147157) bank167844 +167845 POINT(40.142423446836126 73.40644919179317) bank167845 +167846 POINT(40.68812075555258 74.0880391983424) bank167846 +167847 POINT(41.28838356395774 74.4067158670751) bank167847 +167848 POINT(41.370176044975175 74.71159577289077) bank167848 +167849 POINT(40.275304069680075 73.44474469561895) bank167849 +167850 POINT(41.34223697725986 73.97343919284442) bank167850 +167851 POINT(40.77507773856997 74.56917785578061) bank167851 +167852 POINT(40.807849356475 73.47902066813953) bank167852 +167853 POINT(40.00098616929054 74.11833256929533) bank167853 +167854 POINT(39.721183792641426 74.96628643237773) bank167854 +167855 POINT(40.72844837225428 74.84988094056227) bank167855 +167856 POINT(39.72388040023782 74.17569243336305) bank167856 +167857 POINT(40.21683723743704 73.48899336627878) bank167857 +167858 POINT(39.73047747820364 73.61856503027667) bank167858 +167859 POINT(40.53682207014595 73.35329183096846) bank167859 +167860 POINT(40.84908344775848 73.44999204759827) bank167860 +167861 POINT(40.412081347155585 73.52800933591752) bank167861 +167862 POINT(41.65079454994275 73.96210882846553) bank167862 +167863 POINT(39.96474386772416 73.4037526762834) bank167863 +167864 POINT(40.646517408099704 73.10762533281589) bank167864 +167865 POINT(41.086670596679966 74.1956429234265) bank167865 +167866 POINT(40.65199356230346 73.4467101264852) bank167866 +167867 POINT(39.74242569292829 74.54026850310765) bank167867 +167868 POINT(40.55784796061906 73.42771669616087) bank167868 +167869 POINT(41.61337639745889 73.04669981642982) bank167869 +167870 POINT(41.11360682694567 74.1908916322969) bank167870 +167871 POINT(40.65784619279469 74.94880012977814) bank167871 +167872 POINT(41.482297521650324 74.81907124597541) bank167872 +167873 POINT(40.226837039524355 73.94587797463667) bank167873 +167874 POINT(41.10083630231798 74.72432634448266) bank167874 +167875 POINT(40.00428076871246 73.24455276397752) bank167875 +167876 POINT(40.76889994956326 74.63704444295418) bank167876 +167877 POINT(40.28107652508762 74.72912447441607) bank167877 +167878 POINT(39.94650468837229 73.18538105829984) bank167878 +167879 POINT(40.488519248008224 73.70144385519535) bank167879 +167880 POINT(40.8962168759652 73.53527560282525) bank167880 +167881 POINT(40.31967962878651 74.19689019151882) bank167881 +167882 POINT(41.01980201057064 73.92969801365258) bank167882 +167883 POINT(41.361640171604115 74.95695598094251) bank167883 +167884 POINT(40.4649264881447 73.5121840130764) bank167884 +167885 POINT(40.076110589320415 74.68782333770642) bank167885 +167886 POINT(41.47426237227315 73.83164962720345) bank167886 +167887 POINT(39.77258561205446 73.02719508564817) bank167887 +167888 POINT(40.367809634456925 73.36776040369858) bank167888 +167889 POINT(39.73351060077666 73.14197578125332) bank167889 +167890 POINT(40.64091882738569 73.85971691108665) bank167890 +167891 POINT(39.85280543107324 74.60162457551107) bank167891 +167892 POINT(40.29172090958396 74.9164533777109) bank167892 +167893 POINT(40.26069482011558 73.82624463824715) bank167893 +167894 POINT(41.25394158143978 74.595247084434) bank167894 +167895 POINT(39.84579978147083 73.18977191231352) bank167895 +167896 POINT(40.74021339506872 73.32846133740564) bank167896 +167897 POINT(41.17706074081025 73.04663854228026) bank167897 +167898 POINT(40.6402006484635 74.52739867551566) bank167898 +167899 POINT(40.477341630310995 74.49944211001308) bank167899 +167900 POINT(41.40584690854534 74.33088811211286) bank167900 +167901 POINT(40.27353217723435 74.90473207406023) bank167901 +167902 POINT(41.193180191466496 73.04249235634954) bank167902 +167903 POINT(41.46223105069828 73.02217933391226) bank167903 +167904 POINT(41.22723451399041 74.34289924469898) bank167904 +167905 POINT(40.95576242454407 73.62760429866631) bank167905 +167906 POINT(39.8377735775066 74.5827125499552) bank167906 +167907 POINT(40.56525167477895 74.14344834220448) bank167907 +167908 POINT(40.497810898195965 74.34782305752357) bank167908 +167909 POINT(40.26574025269413 73.75325634555192) bank167909 +167910 POINT(39.92399868715166 74.24363669810401) bank167910 +167911 POINT(40.725345074077936 74.54940422896333) bank167911 +167912 POINT(40.488646974329974 74.14819096221406) bank167912 +167913 POINT(41.499051486926646 74.8557923325215) bank167913 +167914 POINT(41.50920235903403 74.29657003629966) bank167914 +167915 POINT(41.21907559548205 73.28800171078942) bank167915 +167916 POINT(40.5425009426429 74.47987031952763) bank167916 +167917 POINT(40.48501495120709 73.14797749667144) bank167917 +167918 POINT(40.3398651343492 73.67649288046594) bank167918 +167919 POINT(40.46447516583767 73.85303607150166) bank167919 +167920 POINT(41.22133568677897 73.814918484745) bank167920 +167921 POINT(40.23357447042244 73.87075125875607) bank167921 +167922 POINT(40.63708193124121 73.09587769240903) bank167922 +167923 POINT(40.79632310717041 74.05195476050196) bank167923 +167924 POINT(40.26426046070884 74.19642194375695) bank167924 +167925 POINT(40.188672969836254 74.95098878701185) bank167925 +167926 POINT(40.74986818033615 74.86949385628466) bank167926 +167927 POINT(40.013665703846804 73.12817427511692) bank167927 +167928 POINT(40.716781695449285 73.40957678533215) bank167928 +167929 POINT(40.727740678268184 73.06244213332309) bank167929 +167930 POINT(41.71049100679151 74.25746288826853) bank167930 +167931 POINT(40.34206983708692 73.98647338324159) bank167931 +167932 POINT(41.535469305666695 73.93481783700624) bank167932 +167933 POINT(40.35255002578624 74.3902248164329) bank167933 +167934 POINT(39.757459372401186 73.41754546743282) bank167934 +167935 POINT(40.22910930073717 74.40050792299576) bank167935 +167936 POINT(40.167155345088595 73.55500697204003) bank167936 +167937 POINT(40.25759932148473 73.79499414642873) bank167937 +167938 POINT(41.498253233297035 74.59124213681223) bank167938 +167939 POINT(40.6483854321193 73.18350145101064) bank167939 +167940 POINT(41.30054129143272 73.24724870903184) bank167940 +167941 POINT(41.09302602216762 74.15600844822312) bank167941 +167942 POINT(40.82625862576154 74.61817699155941) bank167942 +167943 POINT(40.3984079786837 73.42172998690411) bank167943 +167944 POINT(40.74564561775093 73.73393049718852) bank167944 +167945 POINT(41.39719781608729 74.88325796252009) bank167945 +167946 POINT(39.774185361178574 74.96665435173401) bank167946 +167947 POINT(39.97556581376922 74.92998282781674) bank167947 +167948 POINT(39.84535060560875 74.10149774520929) bank167948 +167949 POINT(40.73013695350356 73.78652485666119) bank167949 +167950 POINT(40.012647030343444 73.9311247002322) bank167950 +167951 POINT(40.048074347017796 73.45267188771936) bank167951 +167952 POINT(40.90900886960235 74.91451354201388) bank167952 +167953 POINT(41.035504070059986 74.34725927783757) bank167953 +167954 POINT(40.704637426376 73.00728080184284) bank167954 +167955 POINT(40.696227966780306 73.59036109642136) bank167955 +167956 POINT(40.06371807681533 73.49567892703847) bank167956 +167957 POINT(40.3516717084081 73.41587488877678) bank167957 +167958 POINT(40.925361045252444 74.42207638197678) bank167958 +167959 POINT(40.82878239262557 73.27900901567651) bank167959 +167960 POINT(40.02452491569624 73.20544001531547) bank167960 +167961 POINT(40.40069452689225 74.5969132230882) bank167961 +167962 POINT(41.66229726037169 73.8056599584248) bank167962 +167963 POINT(39.74932329189092 74.69586680702066) bank167963 +167964 POINT(41.261297394251635 73.66285142860816) bank167964 +167965 POINT(41.35646121267655 74.29386983105839) bank167965 +167966 POINT(41.08390773867475 73.36448067455608) bank167966 +167967 POINT(41.29380887718973 73.15139989376644) bank167967 +167968 POINT(39.858323539043624 74.9008520629752) bank167968 +167969 POINT(41.30938522769597 73.07809674054542) bank167969 +167970 POINT(39.747083533787816 73.49322787669593) bank167970 +167971 POINT(41.19392437697594 73.69721952421173) bank167971 +167972 POINT(39.878770057122104 73.60715072781299) bank167972 +167973 POINT(40.78838326244234 73.27278524175793) bank167973 +167974 POINT(40.51429115793379 73.89322188230186) bank167974 +167975 POINT(39.72295607094004 74.52242463263867) bank167975 +167976 POINT(40.000121589115174 73.37224504769866) bank167976 +167977 POINT(40.68180526547255 74.35957052707282) bank167977 +167978 POINT(39.819098605567156 73.62886890747747) bank167978 +167979 POINT(40.14002219828158 73.90005313708794) bank167979 +167980 POINT(41.56159651671517 73.53707382280864) bank167980 +167981 POINT(39.82451550706291 73.64407162606445) bank167981 +167982 POINT(41.33000017335664 73.19069830347266) bank167982 +167983 POINT(40.76842441371189 74.92238381002517) bank167983 +167984 POINT(40.814620268165065 73.41547072319216) bank167984 +167985 POINT(41.11038776903375 74.05559842818263) bank167985 +167986 POINT(39.97045500594736 74.52869999597469) bank167986 +167987 POINT(40.97194562103563 74.37386806151817) bank167987 +167988 POINT(41.62213125927431 73.111311242075) bank167988 +167989 POINT(39.93499232207139 74.88067494659946) bank167989 +167990 POINT(40.580477709168065 74.50227230546888) bank167990 +167991 POINT(40.18210071180545 74.10599480190186) bank167991 +167992 POINT(41.62467338332088 74.9254227643043) bank167992 +167993 POINT(41.399034209965876 74.05245672860651) bank167993 +167994 POINT(39.73949891421341 73.59251213734747) bank167994 +167995 POINT(39.78141831514923 74.62283301524343) bank167995 +167996 POINT(41.46168749524732 73.38377355710071) bank167996 +167997 POINT(40.24358148055965 74.2406561347832) bank167997 +167998 POINT(41.56381666495988 73.87192353727464) bank167998 +167999 POINT(40.615012140602566 73.28221715505953) bank167999 +168000 POINT(40.950364031944794 73.02572040448024) bank168000 +168001 POINT(41.61339139176637 73.07288088377678) bank168001 +168002 POINT(39.77589115783843 74.05022251563412) bank168002 +168003 POINT(41.54988796868696 74.93417469292977) bank168003 +168004 POINT(40.60148387024075 74.25430560345396) bank168004 +168005 POINT(40.02275875859329 73.88795889054022) bank168005 +168006 POINT(41.29160696406634 73.98373218146997) bank168006 +168007 POINT(40.21044421756423 73.89990933175986) bank168007 +168008 POINT(41.62421494764594 74.57326260057111) bank168008 +168009 POINT(41.0522179497715 74.22238289678619) bank168009 +168010 POINT(41.53395969658522 74.756330871939) bank168010 +168011 POINT(41.60130833434761 73.20310959449249) bank168011 +168012 POINT(39.75038617010992 73.35038464010718) bank168012 +168013 POINT(41.5795365884598 73.79511540168413) bank168013 +168014 POINT(39.87342806470778 74.41197222794601) bank168014 +168015 POINT(39.86456983459605 74.267791827655) bank168015 +168016 POINT(40.42917152113022 73.2309344529253) bank168016 +168017 POINT(41.13600833321546 73.58075373441294) bank168017 +168018 POINT(40.8188889309449 74.96726266099623) bank168018 +168019 POINT(40.4046565567757 73.58302339934077) bank168019 +168020 POINT(40.21122875339195 73.85305561655362) bank168020 +168021 POINT(41.544315657741045 74.2240383634845) bank168021 +168022 POINT(39.93143474070612 73.20983189616682) bank168022 +168023 POINT(39.88976044162816 73.07467354963573) bank168023 +168024 POINT(40.6490616852781 73.8748768208889) bank168024 +168025 POINT(41.348488546161605 73.59648843804332) bank168025 +168026 POINT(41.67893102577567 73.76265894558304) bank168026 +168027 POINT(40.136126530742686 74.30045677279448) bank168027 +168028 POINT(40.730650415252455 73.6536122786096) bank168028 +168029 POINT(40.970199406128295 73.13470404262164) bank168029 +168030 POINT(41.23231222200228 73.07865349824087) bank168030 +168031 POINT(40.889659253926254 73.13797367907213) bank168031 +168032 POINT(41.60779700808125 73.68490667109853) bank168032 +168033 POINT(40.014529012694446 74.60065164925187) bank168033 +168034 POINT(41.59842193972707 74.82178683918808) bank168034 +168035 POINT(41.56537058626198 73.63525810961625) bank168035 +168036 POINT(40.14469698714846 73.30262331943936) bank168036 +168037 POINT(39.83048587394741 73.45713856137739) bank168037 +168038 POINT(40.426881157367625 73.37214375571997) bank168038 +168039 POINT(41.265567639451014 74.13165503829984) bank168039 +168040 POINT(40.95750251939056 74.490721054076) bank168040 +168041 POINT(41.70773081542914 74.9917002835573) bank168041 +168042 POINT(41.062764998657684 73.75676027810174) bank168042 +168043 POINT(41.51874381705897 74.85114108375801) bank168043 +168044 POINT(41.627156590644304 73.20665130344206) bank168044 +168045 POINT(40.11259093328989 73.67126847482815) bank168045 +168046 POINT(39.721405121723905 73.17838734085963) bank168046 +168047 POINT(41.16475397796431 75.00007874943316) bank168047 +168048 POINT(40.24567240674675 74.80860150716246) bank168048 +168049 POINT(39.898437546226255 73.78845585179181) bank168049 +168050 POINT(39.955495412033855 73.3076859232235) bank168050 +168051 POINT(41.28958429802206 74.7167843079797) bank168051 +168052 POINT(40.04451864379026 74.69658509098127) bank168052 +168053 POINT(41.223670940256056 73.5080593109985) bank168053 +168054 POINT(41.433801655575465 73.19445003807945) bank168054 +168055 POINT(39.799198054811825 74.33268271068393) bank168055 +168056 POINT(41.00048968868334 74.53429814843133) bank168056 +168057 POINT(40.74636844551023 74.6547405635353) bank168057 +168058 POINT(39.85229671755678 73.22759573782845) bank168058 +168059 POINT(40.84273217791295 74.40029918191755) bank168059 +168060 POINT(41.47293313251019 75.00486129783914) bank168060 +168061 POINT(41.35418953282729 74.47671504513872) bank168061 +168062 POINT(41.534308612869 74.07392561326444) bank168062 +168063 POINT(40.40164926855634 74.93219392816911) bank168063 +168064 POINT(39.817441057648146 74.69910859448312) bank168064 +168065 POINT(40.35716416024375 74.9516209147617) bank168065 +168066 POINT(41.20473390428398 74.55923082378668) bank168066 +168067 POINT(41.57661310840583 74.31373043127651) bank168067 +168068 POINT(40.20839682824854 73.8351979739369) bank168068 +168069 POINT(40.243939953321686 73.31427764492447) bank168069 +168070 POINT(40.34683832217486 74.71491541133418) bank168070 +168071 POINT(41.09356973014889 73.53623250056222) bank168071 +168072 POINT(40.30485721367673 74.34524501983867) bank168072 +168073 POINT(40.56966895471298 73.56269018713887) bank168073 +168074 POINT(40.67464239437071 74.5771280913911) bank168074 +168075 POINT(41.1376627719575 74.51345388260471) bank168075 +168076 POINT(40.49496739691816 73.7246628747529) bank168076 +168077 POINT(40.141047672453944 74.43378892083945) bank168077 +168078 POINT(40.8314067434287 73.24979276192423) bank168078 +168079 POINT(40.17128366912779 73.20839592181896) bank168079 +168080 POINT(40.0063334004897 73.10252598939232) bank168080 +168081 POINT(41.21185702654087 73.92974029340162) bank168081 +168082 POINT(39.954264917481495 73.98590990083078) bank168082 +168083 POINT(39.75456875958408 73.41773679026801) bank168083 +168084 POINT(40.62305570506524 73.79232146466532) bank168084 +168085 POINT(40.000861706887456 73.88171442321543) bank168085 +168086 POINT(40.422216823925254 74.83763353117948) bank168086 +168087 POINT(40.65755479511283 74.67465986126277) bank168087 +168088 POINT(41.50945490430439 73.03181871907412) bank168088 +168089 POINT(41.312036953537486 73.35321365476288) bank168089 +168090 POINT(41.33274988090616 74.14228038496064) bank168090 +168091 POINT(41.02900346345272 73.95657013148904) bank168091 +168092 POINT(41.01051733581989 73.92984793835421) bank168092 +168093 POINT(39.764564687310774 73.75668617674592) bank168093 +168094 POINT(41.37759817422039 74.7899753970737) bank168094 +168095 POINT(41.2755391373938 74.65601499858981) bank168095 +168096 POINT(41.518852005569556 73.42539884997636) bank168096 +168097 POINT(41.51380566510849 74.06831466430177) bank168097 +168098 POINT(40.75382028924652 73.54309069621775) bank168098 +168099 POINT(40.06592609919206 74.74846114521223) bank168099 +168100 POINT(41.11555951189137 74.20577526148638) bank168100 +168101 POINT(41.22363996181373 74.1399231423405) bank168101 +168102 POINT(40.0837472112616 74.89102203506425) bank168102 +168103 POINT(40.296630189481796 73.97846176925371) bank168103 +168104 POINT(41.463964598271026 73.7681291909447) bank168104 +168105 POINT(41.214347697261005 74.20233427818219) bank168105 +168106 POINT(41.13688162484757 73.1906091227101) bank168106 +168107 POINT(40.25612200601574 73.72579080912438) bank168107 +168108 POINT(39.96025979428913 74.5689339069989) bank168108 +168109 POINT(40.38286906956457 73.37405229548742) bank168109 +168110 POINT(40.3529220098902 74.66174153218142) bank168110 +168111 POINT(39.97403217739295 74.80650918508091) bank168111 +168112 POINT(39.8109534459684 73.4658838594012) bank168112 +168113 POINT(41.374580073946326 73.16149622835445) bank168113 +168114 POINT(39.75071175902917 73.46856094584669) bank168114 +168115 POINT(40.43841354350555 73.71192016488295) bank168115 +168116 POINT(40.08228233644426 74.13842401360546) bank168116 +168117 POINT(41.59733848812756 74.76669019094582) bank168117 +168118 POINT(40.6927169636226 74.14959966058525) bank168118 +168119 POINT(40.40801253910741 74.840218524244) bank168119 +168120 POINT(40.31983293103929 74.36976896832508) bank168120 +168121 POINT(41.35013882364984 73.51962799885503) bank168121 +168122 POINT(41.55478980635697 73.55961990986034) bank168122 +168123 POINT(41.067995407977214 74.82522469379091) bank168123 +168124 POINT(39.96879371874649 74.31744683003441) bank168124 +168125 POINT(40.79171040414038 73.70014311340663) bank168125 +168126 POINT(41.71073197535182 74.2229667863406) bank168126 +168127 POINT(40.90367807523015 73.3132530978319) bank168127 +168128 POINT(40.50939031820305 74.81593396487423) bank168128 +168129 POINT(41.06870779461989 74.38496489520911) bank168129 +168130 POINT(41.08032123027674 74.21365960737698) bank168130 +168131 POINT(41.218346580946665 74.57934862735159) bank168131 +168132 POINT(41.19623363457914 73.24002046437036) bank168132 +168133 POINT(40.942862380760104 74.19669829992775) bank168133 +168134 POINT(41.11187665957522 73.94715482459459) bank168134 +168135 POINT(40.371148513327576 73.26722433464482) bank168135 +168136 POINT(41.38922992765239 74.47744777679952) bank168136 +168137 POINT(41.3204179098877 73.6473249071498) bank168137 +168138 POINT(41.19693137383906 73.60107568489833) bank168138 +168139 POINT(41.559665205481174 73.92757054758151) bank168139 +168140 POINT(41.19871977293723 74.7396315498946) bank168140 +168141 POINT(40.69984089217281 73.85786905524361) bank168141 +168142 POINT(39.98309222830013 74.47499937673872) bank168142 +168143 POINT(39.82572547318938 74.22059966452176) bank168143 +168144 POINT(40.41626647395757 73.04755150102355) bank168144 +168145 POINT(39.72640422136312 73.99680814790966) bank168145 +168146 POINT(40.235537835725225 74.05575953299103) bank168146 +168147 POINT(40.979431014289496 73.77096623313393) bank168147 +168148 POINT(40.962363139660866 73.74491597364874) bank168148 +168149 POINT(41.18839920695007 73.8373349666061) bank168149 +168150 POINT(40.809287115881666 74.18706510283133) bank168150 +168151 POINT(39.71500355394545 73.80996364937863) bank168151 +168152 POINT(39.854137544403066 74.35981691560437) bank168152 +168153 POINT(41.104186863006625 74.36641535525176) bank168153 +168154 POINT(40.70262111805964 74.14806701518576) bank168154 +168155 POINT(41.02782577231655 73.86862448665343) bank168155 +168156 POINT(40.495719852528964 74.36757501805658) bank168156 +168157 POINT(41.50917680678876 74.34003653341827) bank168157 +168158 POINT(41.57831188522986 74.58179393870103) bank168158 +168159 POINT(40.14609341653899 74.11255844777213) bank168159 +168160 POINT(40.618759524501435 74.87787114969743) bank168160 +168161 POINT(40.541397327298625 74.28696095131798) bank168161 +168162 POINT(40.12622926248703 73.55588444884829) bank168162 +168163 POINT(41.34891379185148 74.2901949020929) bank168163 +168164 POINT(40.073794145470494 74.72398178615447) bank168164 +168165 POINT(41.21487007082525 73.81800166542862) bank168165 +168166 POINT(40.47942504602392 74.27848569725047) bank168166 +168167 POINT(41.59228658193804 73.79657987210928) bank168167 +168168 POINT(41.57668608757567 73.30936956745738) bank168168 +168169 POINT(40.73700207590708 74.34584386689816) bank168169 +168170 POINT(40.649933872207086 73.03791809708201) bank168170 +168171 POINT(41.36193718514308 73.871383782813) bank168171 +168172 POINT(40.52830102013929 74.73734662432938) bank168172 +168173 POINT(40.315820745835495 73.38754592314939) bank168173 +168174 POINT(41.628788377435576 73.58020434678325) bank168174 +168175 POINT(40.97001027436562 73.61101938716922) bank168175 +168176 POINT(40.84002812529261 74.27652151674384) bank168176 +168177 POINT(41.15396320654318 74.0666707561999) bank168177 +168178 POINT(40.75373530658776 73.10403629095855) bank168178 +168179 POINT(40.87091725567152 73.29487667754063) bank168179 +168180 POINT(40.379870382507505 74.57260410996994) bank168180 +168181 POINT(40.38886125636487 74.57096895272772) bank168181 +168182 POINT(40.41421136481657 73.55492462289548) bank168182 +168183 POINT(40.82402856341477 73.84294600368128) bank168183 +168184 POINT(41.0432141923304 74.98282876724699) bank168184 +168185 POINT(41.18505008405257 74.66073579519725) bank168185 +168186 POINT(41.08239594215906 74.2933190245453) bank168186 +168187 POINT(41.703988606863625 74.97847411960116) bank168187 +168188 POINT(40.42535496746892 73.78532068238465) bank168188 +168189 POINT(41.297979230162454 74.29078746289464) bank168189 +168190 POINT(41.19205512896854 73.65184053028555) bank168190 +168191 POINT(40.4856192970075 74.28319885732033) bank168191 +168192 POINT(40.8172820583078 73.1976837035396) bank168192 +168193 POINT(40.25051415123581 73.8727059830579) bank168193 +168194 POINT(40.389253869833425 74.749599013755) bank168194 +168195 POINT(40.93908374350327 73.9638750404514) bank168195 +168196 POINT(40.32653503286198 74.8241119289884) bank168196 +168197 POINT(40.78512074703259 73.9761276530872) bank168197 +168198 POINT(40.333977911078826 73.90104769729982) bank168198 +168199 POINT(40.802145491589044 74.83228802962458) bank168199 +168200 POINT(41.04701250248513 74.97193944880708) bank168200 +168201 POINT(41.622853162990324 74.73413385051228) bank168201 +168202 POINT(40.90561006872045 73.68782542738418) bank168202 +168203 POINT(40.480673443051 74.50629200529652) bank168203 +168204 POINT(40.66473432217076 74.43515606422675) bank168204 +168205 POINT(40.71984233870756 74.97968235407602) bank168205 +168206 POINT(41.30748383109777 73.82199895546451) bank168206 +168207 POINT(40.54504558036783 74.3306392687757) bank168207 +168208 POINT(40.9688743152241 74.95698101082424) bank168208 +168209 POINT(41.233088551640435 73.63166684592743) bank168209 +168210 POINT(41.072715752306024 73.62553264182867) bank168210 +168211 POINT(40.579890049723474 74.53562097547338) bank168211 +168212 POINT(40.18718310027771 73.73033101655312) bank168212 +168213 POINT(41.10072506937379 74.083401498267) bank168213 +168214 POINT(40.031132956932616 73.47659693798107) bank168214 +168215 POINT(39.96586190426565 73.50379708596995) bank168215 +168216 POINT(41.68964652963535 73.46939893073544) bank168216 +168217 POINT(40.26598591867506 74.79437534972617) bank168217 +168218 POINT(40.00400964671893 74.80109458536624) bank168218 +168219 POINT(41.66913920316158 73.70243853698335) bank168219 +168220 POINT(40.46196823402291 73.39498456177333) bank168220 +168221 POINT(40.59127201192281 74.85568651339824) bank168221 +168222 POINT(40.427086132397974 73.2242709651091) bank168222 +168223 POINT(40.32871479702879 73.81509712610055) bank168223 +168224 POINT(41.19323939592848 74.08668462454732) bank168224 +168225 POINT(40.64488559412605 74.21229918511034) bank168225 +168226 POINT(40.50669921551059 73.42353242678406) bank168226 +168227 POINT(40.50634483174863 73.28187070660961) bank168227 +168228 POINT(40.451372061545165 74.6459890828176) bank168228 +168229 POINT(39.79469675836711 74.50981206385572) bank168229 +168230 POINT(40.44332336181114 74.46098291790547) bank168230 +168231 POINT(40.12730082433044 73.5835082666518) bank168231 +168232 POINT(40.81078026503874 73.4104914094624) bank168232 +168233 POINT(40.85282747898745 73.20051859450402) bank168233 +168234 POINT(40.23129598143175 73.3449982534911) bank168234 +168235 POINT(40.90252096807323 74.94118331624432) bank168235 +168236 POINT(41.07917138056223 73.90798952960917) bank168236 +168237 POINT(41.57035336444577 74.90870089344844) bank168237 +168238 POINT(40.35154544213566 74.71929048743804) bank168238 +168239 POINT(40.06817692903101 74.27272835947481) bank168239 +168240 POINT(41.49829231758023 74.99271155555412) bank168240 +168241 POINT(41.12945383798575 74.97647710141214) bank168241 +168242 POINT(40.661153968735405 74.47417925977724) bank168242 +168243 POINT(41.44141238862479 74.95478318703081) bank168243 +168244 POINT(39.9296589580336 74.69219852138541) bank168244 +168245 POINT(40.822957199392825 74.28512090557082) bank168245 +168246 POINT(40.931602863693996 74.73220274905108) bank168246 +168247 POINT(40.88677449974288 73.94794304894562) bank168247 +168248 POINT(40.817134379958425 73.67341812204839) bank168248 +168249 POINT(41.04402269351476 73.1229372558363) bank168249 +168250 POINT(40.89853562402725 74.24970965470169) bank168250 +168251 POINT(41.48211325027452 73.9126590687701) bank168251 +168252 POINT(41.69365096569356 74.26806976945652) bank168252 +168253 POINT(40.50077182073321 74.78556386173791) bank168253 +168254 POINT(40.46916816844843 74.45019021449589) bank168254 +168255 POINT(41.41765164611763 73.80252377146729) bank168255 +168256 POINT(39.77761571635644 73.94830014939485) bank168256 +168257 POINT(40.38233550287919 74.46543452298836) bank168257 +168258 POINT(41.30024910306571 73.5740858956495) bank168258 +168259 POINT(40.342248316931176 73.681822291093) bank168259 +168260 POINT(40.18939301375364 74.21982832487163) bank168260 +168261 POINT(40.23191902746237 74.42856160883821) bank168261 +168262 POINT(40.289333945940605 74.14680607260637) bank168262 +168263 POINT(40.44793307068041 73.47663006728453) bank168263 +168264 POINT(40.12674378555645 74.54469786869652) bank168264 +168265 POINT(39.80282230641343 74.11721694151205) bank168265 +168266 POINT(40.94373290995945 73.42792487526228) bank168266 +168267 POINT(41.139392809292644 73.90106572903125) bank168267 +168268 POINT(40.604145867031015 73.33671240027607) bank168268 +168269 POINT(40.12124791857635 73.28486835408349) bank168269 +168270 POINT(41.14949561579054 74.52748460410753) bank168270 +168271 POINT(40.52156815898982 74.15567222732444) bank168271 +168272 POINT(39.95563996791971 74.06986976959037) bank168272 +168273 POINT(40.388179548967635 73.77358456675904) bank168273 +168274 POINT(40.44354639449904 74.0359885498008) bank168274 +168275 POINT(39.85626533511563 74.44808364633434) bank168275 +168276 POINT(40.214087371312864 74.31494271161223) bank168276 +168277 POINT(40.909220414768484 74.28390629672697) bank168277 +168278 POINT(40.2976938581515 74.93686585951349) bank168278 +168279 POINT(41.58255112961606 74.0704315364201) bank168279 +168280 POINT(40.9820223133402 74.64437820880254) bank168280 +168281 POINT(40.828401581441284 73.70285180940434) bank168281 +168282 POINT(39.94706327836195 74.43020458680739) bank168282 +168283 POINT(41.215338762205555 74.8188698562185) bank168283 +168284 POINT(41.5413356489455 73.09690937059545) bank168284 +168285 POINT(40.98925287058507 74.89598257732698) bank168285 +168286 POINT(39.92192816690995 74.22311284447106) bank168286 +168287 POINT(41.603938865827466 74.49619283460581) bank168287 +168288 POINT(41.261520517803746 73.79668572597153) bank168288 +168289 POINT(40.90271175499435 74.24982782421975) bank168289 +168290 POINT(40.07889229460077 74.25859724609742) bank168290 +168291 POINT(40.56357341089063 74.84943326201488) bank168291 +168292 POINT(40.338628282771026 74.13638879017923) bank168292 +168293 POINT(41.14516667604321 73.72363136675739) bank168293 +168294 POINT(40.518313467313035 74.35147962811162) bank168294 +168295 POINT(41.21835198076119 74.13776577905827) bank168295 +168296 POINT(41.13813759248391 74.13183792883613) bank168296 +168297 POINT(41.25078378165433 74.74683388405732) bank168297 +168298 POINT(40.40929231422899 74.67772342155313) bank168298 +168299 POINT(40.59833226328357 73.12496255421046) bank168299 +168300 POINT(41.50907937689642 73.33193058481754) bank168300 +168301 POINT(41.5962318068248 74.29872313719802) bank168301 +168302 POINT(41.35923417396647 73.3738216473417) bank168302 +168303 POINT(41.10122048392228 73.48875213532561) bank168303 +168304 POINT(40.59325881729849 73.03153918059152) bank168304 +168305 POINT(39.76454711280753 74.6323371850127) bank168305 +168306 POINT(40.0853096772257 73.4975333027509) bank168306 +168307 POINT(40.496175011995064 74.16307440677608) bank168307 +168308 POINT(41.030866341047464 74.4376522854727) bank168308 +168309 POINT(40.70951058478417 74.36335892669946) bank168309 +168310 POINT(39.878743316200165 74.78756835291489) bank168310 +168311 POINT(40.23727307277947 73.54476661433259) bank168311 +168312 POINT(40.23133813257107 74.48151013681374) bank168312 +168313 POINT(41.11179110627087 74.03689604977095) bank168313 +168314 POINT(40.32452101154904 74.24285152679533) bank168314 +168315 POINT(41.43871221293465 74.14495014551267) bank168315 +168316 POINT(41.04379977865389 74.75315318155033) bank168316 +168317 POINT(40.79981443821644 73.68087777755485) bank168317 +168318 POINT(41.24599852677759 73.1375629425514) bank168318 +168319 POINT(40.48239394036881 74.78133803450278) bank168319 +168320 POINT(41.59296689223364 73.20792811972736) bank168320 +168321 POINT(39.99749625636267 74.51083571218308) bank168321 +168322 POINT(41.30684127735787 74.06114258037728) bank168322 +168323 POINT(40.8333827179131 74.09563956158426) bank168323 +168324 POINT(40.96502872742704 73.78423897920015) bank168324 +168325 POINT(40.94240552518243 74.9727182347165) bank168325 +168326 POINT(40.55489510567907 73.68511203957246) bank168326 +168327 POINT(41.21599776892058 73.95511032738432) bank168327 +168328 POINT(40.54155479333248 74.10594537270157) bank168328 +168329 POINT(39.82208974654296 73.73368796068732) bank168329 +168330 POINT(41.21015127738849 73.21751712173948) bank168330 +168331 POINT(41.3613171355558 74.12259460283423) bank168331 +168332 POINT(39.93062279700167 74.68030171960027) bank168332 +168333 POINT(39.74849875494656 73.7446374372414) bank168333 +168334 POINT(40.209790458850684 73.91388162254862) bank168334 +168335 POINT(41.262142980126676 73.99717220169354) bank168335 +168336 POINT(41.54704768568258 73.84757748406462) bank168336 +168337 POINT(41.32363265900366 73.04085644301468) bank168337 +168338 POINT(40.16340387255196 74.6492985804034) bank168338 +168339 POINT(41.474892380061746 73.0922276110627) bank168339 +168340 POINT(40.987027469420966 73.53823135130048) bank168340 +168341 POINT(39.833776865191425 73.39723595666285) bank168341 +168342 POINT(39.80915594689784 73.42408530370884) bank168342 +168343 POINT(39.98194653750302 74.83309123889198) bank168343 +168344 POINT(41.60554397679531 73.43661153919311) bank168344 +168345 POINT(40.96630271371356 74.08565305546236) bank168345 +168346 POINT(41.04695478653975 73.46525384149945) bank168346 +168347 POINT(41.60324111165175 73.15393770333355) bank168347 +168348 POINT(41.57402965733823 74.06898229759521) bank168348 +168349 POINT(41.28501752621223 73.66287494877085) bank168349 +168350 POINT(40.05022829457971 73.32251961509141) bank168350 +168351 POINT(41.2425516517256 74.20305430293703) bank168351 +168352 POINT(40.00385290436622 74.80858725583158) bank168352 +168353 POINT(40.54521549992807 74.81533009548838) bank168353 +168354 POINT(40.788894006946066 73.9814675643104) bank168354 +168355 POINT(40.81940322880454 73.41917817540971) bank168355 +168356 POINT(39.78037727745156 73.81945479595784) bank168356 +168357 POINT(41.6058303108876 74.79480442725509) bank168357 +168358 POINT(41.24873976262951 74.28330781189285) bank168358 +168359 POINT(39.96671845331374 73.786039117358) bank168359 +168360 POINT(41.13144869308929 73.41643976955805) bank168360 +168361 POINT(40.830422511161004 74.93394604264965) bank168361 +168362 POINT(40.30107531956081 73.56857590198851) bank168362 +168363 POINT(41.61167268906365 73.43554815235734) bank168363 +168364 POINT(40.47977561124726 74.24469689827595) bank168364 +168365 POINT(40.002059266896595 74.2732413269676) bank168365 +168366 POINT(41.57300063636676 74.96506465104176) bank168366 +168367 POINT(41.06476264684871 74.97860560096301) bank168367 +168368 POINT(40.032961405251285 74.00078374738551) bank168368 +168369 POINT(41.3920041420755 74.3164412240706) bank168369 +168370 POINT(40.988388860798 74.3825032517782) bank168370 +168371 POINT(40.16763090029447 74.90536295872701) bank168371 +168372 POINT(39.76045035877318 73.71024917322944) bank168372 +168373 POINT(40.98230752304361 73.71958162511991) bank168373 +168374 POINT(41.54857303928136 73.41878689045218) bank168374 +168375 POINT(40.5927037535985 73.17674550728054) bank168375 +168376 POINT(41.155856992577554 73.4578480801144) bank168376 +168377 POINT(40.306789639736415 73.99683687691082) bank168377 +168378 POINT(40.698486840066316 74.31310285315271) bank168378 +168379 POINT(40.72226654040731 73.38600936996268) bank168379 +168380 POINT(41.54022631616752 73.70540709165739) bank168380 +168381 POINT(41.515659608213916 73.38551151723938) bank168381 +168382 POINT(41.02083800493786 73.36836824086856) bank168382 +168383 POINT(40.1507081299318 74.70222078652965) bank168383 +168384 POINT(41.69630905594354 73.1648921968253) bank168384 +168385 POINT(41.462872443212774 74.07244881362816) bank168385 +168386 POINT(41.346438960423725 73.62085209360202) bank168386 +168387 POINT(39.80426514502144 74.16921984659237) bank168387 +168388 POINT(39.888131313628726 73.8830569724525) bank168388 +168389 POINT(40.41512913135549 73.76872916308551) bank168389 +168390 POINT(39.88891381016328 73.44628720112603) bank168390 +168391 POINT(40.68580773817397 74.30764366718725) bank168391 +168392 POINT(40.1182959155004 74.23584440801852) bank168392 +168393 POINT(40.47213798089695 73.18147790313944) bank168393 +168394 POINT(40.65770686230626 74.48866405224106) bank168394 +168395 POINT(41.326482461600776 75.00531980815558) bank168395 +168396 POINT(41.48775822205089 74.58601702460575) bank168396 +168397 POINT(40.26574530604003 74.11667749454247) bank168397 +168398 POINT(40.498332570881416 73.69332721616559) bank168398 +168399 POINT(39.74136274439869 73.04645259196516) bank168399 +168400 POINT(39.88450496306906 74.13970182307916) bank168400 +168401 POINT(41.448570732355165 73.25503048345365) bank168401 +168402 POINT(41.64418651787641 73.3759332683229) bank168402 +168403 POINT(41.30373621174341 73.85680650959249) bank168403 +168404 POINT(40.18736558834987 74.97971152219705) bank168404 +168405 POINT(41.583255073090996 73.44363396891687) bank168405 +168406 POINT(40.06217868770772 74.84613845898289) bank168406 +168407 POINT(40.66859235068865 73.43577876848913) bank168407 +168408 POINT(41.27481444012663 74.55725086976396) bank168408 +168409 POINT(40.026532369404336 74.29597024860978) bank168409 +168410 POINT(40.03525106414618 73.48734858864982) bank168410 +168411 POINT(41.269548287301845 74.51978274629055) bank168411 +168412 POINT(39.72450769011188 74.85298432283889) bank168412 +168413 POINT(39.76790353770863 73.6292321111192) bank168413 +168414 POINT(40.55204919783674 74.5497135605716) bank168414 +168415 POINT(40.26756897769042 74.29127347812336) bank168415 +168416 POINT(40.109879566630866 74.28419518822557) bank168416 +168417 POINT(41.04592928680165 74.74787875485634) bank168417 +168418 POINT(40.14665327436455 74.91199467395055) bank168418 +168419 POINT(40.42358991984488 74.54010128667933) bank168419 +168420 POINT(41.07819092642708 74.49978720182247) bank168420 +168421 POINT(40.94975669484655 73.0627899428368) bank168421 +168422 POINT(41.25788037794875 74.73576776284725) bank168422 +168423 POINT(41.22432615085812 73.69467877315273) bank168423 +168424 POINT(40.39712046359183 73.68981174889878) bank168424 +168425 POINT(41.18259925447148 74.01185336874376) bank168425 +168426 POINT(41.65458805960361 73.46701978839572) bank168426 +168427 POINT(40.72614613168609 73.14611761543233) bank168427 +168428 POINT(39.88303548598533 73.21390722118876) bank168428 +168429 POINT(40.217582842453126 73.4421617159388) bank168429 +168430 POINT(41.57810958360188 73.73679215114919) bank168430 +168431 POINT(40.39912690006071 73.29663292507993) bank168431 +168432 POINT(40.139068037185204 73.81573367996955) bank168432 +168433 POINT(41.28230102387094 74.8014338837561) bank168433 +168434 POINT(40.34675771651465 73.53737200347075) bank168434 +168435 POINT(40.03753913405343 73.91424593656428) bank168435 +168436 POINT(41.518980330693196 74.06878801771357) bank168436 +168437 POINT(39.78956852950791 74.09489787442591) bank168437 +168438 POINT(40.04597534347162 73.86086254465347) bank168438 +168439 POINT(40.69050809773834 73.11823566413038) bank168439 +168440 POINT(41.679924246091815 73.73491050045925) bank168440 +168441 POINT(40.86904875537422 73.92574784622745) bank168441 +168442 POINT(40.37762098790549 74.79313953954856) bank168442 +168443 POINT(39.82743125683575 73.93000455994512) bank168443 +168444 POINT(40.32582790901676 74.61824285264626) bank168444 +168445 POINT(40.47822767293199 73.34159937864572) bank168445 +168446 POINT(41.45914306521784 73.75860654908894) bank168446 +168447 POINT(39.7645669742972 73.27499598341112) bank168447 +168448 POINT(40.386703891242604 73.06449325576357) bank168448 +168449 POINT(41.67565090119547 73.89317943813595) bank168449 +168450 POINT(41.179244622915895 73.32977343820147) bank168450 +168451 POINT(40.356750973322136 73.2768288853783) bank168451 +168452 POINT(40.60723507010277 74.47711912934149) bank168452 +168453 POINT(40.940933665008465 74.80890559762315) bank168453 +168454 POINT(40.44629690628644 73.03068852510981) bank168454 +168455 POINT(40.28714893936964 74.36038669971931) bank168455 +168456 POINT(41.306452682598746 73.46796829374274) bank168456 +168457 POINT(40.870783290189365 74.76972495849283) bank168457 +168458 POINT(40.950479543541746 74.21939137260252) bank168458 +168459 POINT(40.02573239572642 73.88683834420401) bank168459 +168460 POINT(41.0267278473744 74.69904449453189) bank168460 +168461 POINT(41.268392651204096 73.33481305257483) bank168461 +168462 POINT(39.84741912084096 74.64987335828255) bank168462 +168463 POINT(41.0874041472129 73.34928778788877) bank168463 +168464 POINT(41.21489038232898 74.94856643542293) bank168464 +168465 POINT(39.929929600470004 74.96235229971042) bank168465 +168466 POINT(40.485194747192054 74.56599044607364) bank168466 +168467 POINT(41.20956132605736 73.1204858173217) bank168467 +168468 POINT(41.35719015247648 74.01398916060872) bank168468 +168469 POINT(41.444713408938185 74.8795567754252) bank168469 +168470 POINT(39.96604065665794 73.03897886351706) bank168470 +168471 POINT(40.51448413636787 73.79436273393516) bank168471 +168472 POINT(41.185389555142194 74.07829944767637) bank168472 +168473 POINT(39.94299407080095 74.33572791593492) bank168473 +168474 POINT(40.57990945471352 74.9527256840455) bank168474 +168475 POINT(41.50446931514201 74.21373865439601) bank168475 +168476 POINT(40.66499495557821 74.92008325371289) bank168476 +168477 POINT(39.97855819883312 74.36660388036472) bank168477 +168478 POINT(40.34508133663127 73.36947929995742) bank168478 +168479 POINT(40.60268244640682 74.57747050581791) bank168479 +168480 POINT(39.71287385640505 74.71347472779148) bank168480 +168481 POINT(39.93194491545941 74.74256245907405) bank168481 +168482 POINT(41.02050328525042 74.94304532889454) bank168482 +168483 POINT(39.740175277692664 73.67351977514025) bank168483 +168484 POINT(40.57808107931128 74.53944589799366) bank168484 +168485 POINT(40.823076064163324 74.55803327580641) bank168485 +168486 POINT(40.98410274828674 74.43785861506666) bank168486 +168487 POINT(40.80728437747271 73.61990647901422) bank168487 +168488 POINT(40.54051129125486 73.21519520213609) bank168488 +168489 POINT(40.24571736454784 73.84054607140797) bank168489 +168490 POINT(41.43043029275614 73.23381664393489) bank168490 +168491 POINT(40.5232111930266 73.26083742329325) bank168491 +168492 POINT(40.522432740617724 73.37514844996457) bank168492 +168493 POINT(41.60965679862999 73.22099797834454) bank168493 +168494 POINT(39.899154743341846 74.47635188867487) bank168494 +168495 POINT(40.6667570365918 73.44593346333524) bank168495 +168496 POINT(40.059325169493626 73.593545511304) bank168496 +168497 POINT(40.47562141277479 74.7039572169112) bank168497 +168498 POINT(41.387637908241345 74.64436825668007) bank168498 +168499 POINT(40.8575718100485 73.42877217942598) bank168499 +168500 POINT(41.2922963622982 74.48999556136357) bank168500 +168501 POINT(40.13740273051575 73.15551393306546) bank168501 +168502 POINT(40.660991956806434 73.24594504697151) bank168502 +168503 POINT(40.445278585541196 73.8843275244996) bank168503 +168504 POINT(40.750052824427044 73.02225549020524) bank168504 +168505 POINT(41.389902600992365 73.18812259462332) bank168505 +168506 POINT(40.965349815242874 74.01358010696131) bank168506 +168507 POINT(41.68338280129073 74.018137049649) bank168507 +168508 POINT(41.49539522677167 74.87450758254262) bank168508 +168509 POINT(40.9854893249965 74.5992433063192) bank168509 +168510 POINT(39.940420653120626 73.92148205334517) bank168510 +168511 POINT(41.193971259596374 74.8077111340362) bank168511 +168512 POINT(41.32477862592583 74.04864780209743) bank168512 +168513 POINT(40.300523055268286 73.31429433717771) bank168513 +168514 POINT(39.799960491052204 73.98539198245204) bank168514 +168515 POINT(41.233815894092146 73.01192499576648) bank168515 +168516 POINT(39.85103114078778 74.82856137452005) bank168516 +168517 POINT(40.711611645242336 74.12986648300122) bank168517 +168518 POINT(41.60459852601827 74.16650833424244) bank168518 +168519 POINT(41.620461427785294 74.18855633234494) bank168519 +168520 POINT(39.922576698803276 74.90062863208088) bank168520 +168521 POINT(41.071967162247894 74.27874064468148) bank168521 +168522 POINT(39.87334534404334 74.68202211974392) bank168522 +168523 POINT(39.93176763838886 73.82995639876367) bank168523 +168524 POINT(40.806291953042425 74.99171414815648) bank168524 +168525 POINT(39.88788875098095 73.77471962771195) bank168525 +168526 POINT(41.580592674573545 73.10796610835438) bank168526 +168527 POINT(41.64840253694025 73.93408761890032) bank168527 +168528 POINT(41.50913387844153 73.71971461774726) bank168528 +168529 POINT(40.53713945610576 74.2558825653057) bank168529 +168530 POINT(41.33635838166367 74.31407267400849) bank168530 +168531 POINT(41.16688090887671 74.5659930118833) bank168531 +168532 POINT(41.39469230388974 73.34980640357566) bank168532 +168533 POINT(41.368470849213466 74.8264650935744) bank168533 +168534 POINT(39.78187987183804 74.08078412227484) bank168534 +168535 POINT(40.52195679732537 74.07634895477665) bank168535 +168536 POINT(41.584213452455394 73.87368136468864) bank168536 +168537 POINT(40.8852779863389 73.97957488088836) bank168537 +168538 POINT(41.06484116306379 73.49226676043905) bank168538 +168539 POINT(40.64741751776582 74.91378856814136) bank168539 +168540 POINT(39.85143122267609 73.91183417100264) bank168540 +168541 POINT(41.68412942756144 73.3604446339495) bank168541 +168542 POINT(40.97707168136189 74.09953776741885) bank168542 +168543 POINT(39.78902863640088 73.77433891533455) bank168543 +168544 POINT(40.7630688195062 73.86457003942265) bank168544 +168545 POINT(41.33611883054729 74.95593907457985) bank168545 +168546 POINT(40.44162899039702 74.21805223067427) bank168546 +168547 POINT(40.39687087875302 74.53305333518288) bank168547 +168548 POINT(40.366837394248044 74.59533016901779) bank168548 +168549 POINT(39.76808527994914 74.2998510966255) bank168549 +168550 POINT(40.977074464623634 74.87144384895187) bank168550 +168551 POINT(39.77880723302356 74.73244331122399) bank168551 +168552 POINT(39.88994860731869 74.55318205890995) bank168552 +168553 POINT(40.30599679453267 73.03864461457901) bank168553 +168554 POINT(41.41470020063518 74.29585550420484) bank168554 +168555 POINT(40.6194141156642 74.84126477268717) bank168555 +168556 POINT(40.894574611912 73.51469316442788) bank168556 +168557 POINT(39.9416843535456 74.72659377310535) bank168557 +168558 POINT(39.78895712765405 73.7374528392309) bank168558 +168559 POINT(40.17896551871251 74.66200867512828) bank168559 +168560 POINT(41.57831354729682 73.07366946413688) bank168560 +168561 POINT(41.421498742062084 74.56778635177298) bank168561 +168562 POINT(40.18374052681082 74.4680159505304) bank168562 +168563 POINT(41.24767900699862 74.09425228167454) bank168563 +168564 POINT(40.14342690531751 73.93760147496762) bank168564 +168565 POINT(40.68872622019195 74.26288462146466) bank168565 +168566 POINT(41.12412061907737 74.39588783966232) bank168566 +168567 POINT(40.02857942598762 73.79272349360461) bank168567 +168568 POINT(39.93263276156341 73.09985028471216) bank168568 +168569 POINT(40.16247176322996 74.47858608224462) bank168569 +168570 POINT(40.240114167792186 73.0774825322904) bank168570 +168571 POINT(41.52085131801517 73.10829606897258) bank168571 +168572 POINT(40.386808229022954 74.78993827973837) bank168572 +168573 POINT(40.465182361138616 73.02940030709675) bank168573 +168574 POINT(40.0377706478759 73.88011988658937) bank168574 +168575 POINT(40.118688760766304 73.42892007259304) bank168575 +168576 POINT(41.161965004378295 73.1973308522703) bank168576 +168577 POINT(41.694913636014874 74.50897408147262) bank168577 +168578 POINT(41.65876744777882 74.64315766942374) bank168578 +168579 POINT(41.047582488611724 74.61788643657675) bank168579 +168580 POINT(40.53248789020647 74.25060330397518) bank168580 +168581 POINT(41.43811728171326 73.37184462209181) bank168581 +168582 POINT(40.981431430094034 73.14770391955669) bank168582 +168583 POINT(41.50039542110748 74.1022144142542) bank168583 +168584 POINT(40.85614614361157 73.09025718693563) bank168584 +168585 POINT(41.23489466462753 73.53847778788352) bank168585 +168586 POINT(41.12162788960354 74.04623944793133) bank168586 +168587 POINT(41.53416134488053 74.38146876597396) bank168587 +168588 POINT(40.64683470621048 74.76260689058998) bank168588 +168589 POINT(41.67408562631899 73.70350140487935) bank168589 +168590 POINT(41.14016088268519 73.04780327774482) bank168590 +168591 POINT(41.66437957443215 73.89162637046435) bank168591 +168592 POINT(40.41606160107612 74.82389622936365) bank168592 +168593 POINT(41.08622819221271 73.74490674666902) bank168593 +168594 POINT(40.757082609293434 73.62945872089381) bank168594 +168595 POINT(41.067131300255916 74.81650445906675) bank168595 +168596 POINT(41.64136374679565 74.58250257345426) bank168596 +168597 POINT(41.30913236433499 73.61051139105761) bank168597 +168598 POINT(41.21584410093527 74.3008814859164) bank168598 +168599 POINT(40.35131317418985 73.47702763706098) bank168599 +168600 POINT(40.0925090220262 73.18319726271847) bank168600 +168601 POINT(40.94395212231517 73.83224610050698) bank168601 +168602 POINT(40.88693544772413 73.76092708539764) bank168602 +168603 POINT(40.4025983094534 73.61298398489255) bank168603 +168604 POINT(40.42118924485483 74.25268939547473) bank168604 +168605 POINT(41.166287673236475 73.8304801231128) bank168605 +168606 POINT(41.2858228308906 74.84970653872335) bank168606 +168607 POINT(40.19845556106089 73.89806634437396) bank168607 +168608 POINT(40.8789429476285 74.00926091995898) bank168608 +168609 POINT(40.28612199328591 74.94497926176376) bank168609 +168610 POINT(41.59115033537456 74.78427744407337) bank168610 +168611 POINT(41.44767642558764 74.96989252988114) bank168611 +168612 POINT(41.32800718994556 74.48677919537123) bank168612 +168613 POINT(41.64671184835027 73.86077536181399) bank168613 +168614 POINT(39.95144822610801 73.85722078949412) bank168614 +168615 POINT(40.430694547096856 74.21623658327918) bank168615 +168616 POINT(40.134039721514625 73.81536124664697) bank168616 +168617 POINT(40.249390128222835 73.0409920438095) bank168617 +168618 POINT(40.65098887465586 74.00131756906738) bank168618 +168619 POINT(39.732599640066994 74.4922008809604) bank168619 +168620 POINT(40.19880869919845 74.83023692256155) bank168620 +168621 POINT(41.002749398311266 73.90872825196489) bank168621 +168622 POINT(39.83920072657861 73.46268436545665) bank168622 +168623 POINT(40.56075705914187 73.96120664205564) bank168623 +168624 POINT(41.67693455814493 73.98558023342795) bank168624 +168625 POINT(40.655573736877166 73.5038354256641) bank168625 +168626 POINT(40.43781617227278 73.43803582953784) bank168626 +168627 POINT(39.7487106890244 74.35266204110626) bank168627 +168628 POINT(41.65501444333924 74.61085506846014) bank168628 +168629 POINT(40.88081811283778 74.3575716305584) bank168629 +168630 POINT(40.07349505659474 73.56734856089848) bank168630 +168631 POINT(41.30598845809896 74.64338147288129) bank168631 +168632 POINT(41.181160585992274 74.80963758249031) bank168632 +168633 POINT(41.151508459008284 73.066671164883) bank168633 +168634 POINT(40.57782064519103 74.12694059866465) bank168634 +168635 POINT(40.32098532373689 73.3228252895842) bank168635 +168636 POINT(41.10596216141968 73.54502023314404) bank168636 +168637 POINT(41.52749072970681 74.07639596527278) bank168637 +168638 POINT(40.93872101708652 73.93118340368952) bank168638 +168639 POINT(40.0696334742545 74.4738096517173) bank168639 +168640 POINT(41.62764073723933 74.60358814837399) bank168640 +168641 POINT(39.871223069069345 74.91969754814517) bank168641 +168642 POINT(41.261799109932575 73.75846033563258) bank168642 +168643 POINT(41.14349730953016 74.8533882573853) bank168643 +168644 POINT(40.502927676858405 74.04804329275392) bank168644 +168645 POINT(41.07863329812352 73.15895585034605) bank168645 +168646 POINT(40.603533695235285 73.93476764780138) bank168646 +168647 POINT(40.051578399594554 73.7466521136924) bank168647 +168648 POINT(40.175760610960694 73.71420869697796) bank168648 +168649 POINT(39.91531660086252 74.95983946312316) bank168649 +168650 POINT(39.872216332503854 73.71577500742492) bank168650 +168651 POINT(41.42818735036208 74.6613500045084) bank168651 +168652 POINT(40.92521805774068 73.15489562915452) bank168652 +168653 POINT(40.27360709672918 73.33319409631416) bank168653 +168654 POINT(41.61448903022419 74.79297355138748) bank168654 +168655 POINT(41.13537294558528 73.91148842703109) bank168655 +168656 POINT(41.52546309980616 73.608230520361) bank168656 +168657 POINT(40.073248149232136 74.34506929237644) bank168657 +168658 POINT(39.98734798236241 74.07207948011947) bank168658 +168659 POINT(41.38974483533062 74.16835464029154) bank168659 +168660 POINT(39.87349100746315 73.3737495578958) bank168660 +168661 POINT(40.9279223822815 73.9708237769199) bank168661 +168662 POINT(40.52685427307421 74.21438240222196) bank168662 +168663 POINT(40.57357411463734 74.55606884105535) bank168663 +168664 POINT(40.007155654169864 74.34880076752114) bank168664 +168665 POINT(41.66441002928789 74.92136588348649) bank168665 +168666 POINT(39.913985763980804 73.05090909084932) bank168666 +168667 POINT(40.486168608099646 73.26001721725878) bank168667 +168668 POINT(39.85963750785019 74.69128504328529) bank168668 +168669 POINT(41.4021720442075 73.56491396176548) bank168669 +168670 POINT(39.88575575576397 74.99414775529141) bank168670 +168671 POINT(39.81946701574862 73.50668972992511) bank168671 +168672 POINT(41.29577341816277 74.84137939491737) bank168672 +168673 POINT(41.40998718535495 73.23868527504843) bank168673 +168674 POINT(40.8184768824811 73.03490885820018) bank168674 +168675 POINT(40.90743575074693 74.41446775261984) bank168675 +168676 POINT(40.113025200320486 74.13695505524703) bank168676 +168677 POINT(41.58781378639993 73.64481920695955) bank168677 +168678 POINT(41.41378065689602 73.14099406751711) bank168678 +168679 POINT(40.291930119592564 74.16220342998398) bank168679 +168680 POINT(40.09150734826655 73.41516855028061) bank168680 +168681 POINT(40.087349025845036 73.70403296393692) bank168681 +168682 POINT(41.43405757606646 74.08173879184835) bank168682 +168683 POINT(40.00338841903631 74.28721039292405) bank168683 +168684 POINT(41.23672729804245 74.47304083771887) bank168684 +168685 POINT(40.47669289724612 73.88345028536315) bank168685 +168686 POINT(40.527197352041306 73.72321337491911) bank168686 +168687 POINT(40.55657082068691 73.54300800143601) bank168687 +168688 POINT(41.681385549470775 74.75511387065964) bank168688 +168689 POINT(41.17314791224585 73.70159224206041) bank168689 +168690 POINT(40.631306617425594 73.13329056366359) bank168690 +168691 POINT(40.95348894224617 73.34889691095186) bank168691 +168692 POINT(41.63120216947072 73.29084390062012) bank168692 +168693 POINT(39.87681036281405 74.49320677529701) bank168693 +168694 POINT(39.8327300198656 73.8372095980993) bank168694 +168695 POINT(41.46147202259434 74.48064639674169) bank168695 +168696 POINT(41.512979993942345 74.21019292596873) bank168696 +168697 POINT(41.3525420075678 73.85082157481685) bank168697 +168698 POINT(40.23620351429368 74.39452645892331) bank168698 +168699 POINT(40.75978325004286 73.74333715326935) bank168699 +168700 POINT(40.210242943440726 74.66796833660652) bank168700 +168701 POINT(40.955204083454504 74.83306857880774) bank168701 +168702 POINT(40.52992577589698 73.376794048521) bank168702 +168703 POINT(41.17366145509834 74.6388360116247) bank168703 +168704 POINT(40.48312433902525 73.05120697017965) bank168704 +168705 POINT(40.80787051080874 74.00553467323918) bank168705 +168706 POINT(40.27504931898669 74.855089048355) bank168706 +168707 POINT(40.83721972965631 73.92356533783655) bank168707 +168708 POINT(41.52644200579644 74.31196371803225) bank168708 +168709 POINT(40.39036265568448 73.22628471312501) bank168709 +168710 POINT(40.30680032052488 73.44636959454368) bank168710 +168711 POINT(41.171270674361416 74.85258724443469) bank168711 +168712 POINT(41.65051136519697 73.86933385898469) bank168712 +168713 POINT(41.12488949693117 74.06758822304081) bank168713 +168714 POINT(40.54899758091807 73.19612213214081) bank168714 +168715 POINT(41.25777555013882 73.03587879879983) bank168715 +168716 POINT(41.09132417499528 74.57504502462598) bank168716 +168717 POINT(40.426071315930784 74.99196327800576) bank168717 +168718 POINT(41.13855524071778 73.89993125242738) bank168718 +168719 POINT(40.088234278742405 74.66164292551066) bank168719 +168720 POINT(40.24334606867995 73.32411351947967) bank168720 +168721 POINT(40.092670106649784 73.62928388277221) bank168721 +168722 POINT(40.57512014156091 74.61258385980341) bank168722 +168723 POINT(39.92175978238474 73.25606410912067) bank168723 +168724 POINT(41.00227867899928 74.13531375320225) bank168724 +168725 POINT(41.11998924860546 73.07079120911362) bank168725 +168726 POINT(40.14243736897957 73.04060546891469) bank168726 +168727 POINT(39.963248508290356 74.56201842361966) bank168727 +168728 POINT(41.122883324886224 74.15074185901481) bank168728 +168729 POINT(39.751796241885344 74.14065871440907) bank168729 +168730 POINT(40.98362043407521 74.01526974627637) bank168730 +168731 POINT(41.62368818838075 73.98843774092764) bank168731 +168732 POINT(40.91879751766026 73.8707991085895) bank168732 +168733 POINT(40.35480737730342 74.78757631565455) bank168733 +168734 POINT(41.0094939227171 74.42765358102488) bank168734 +168735 POINT(40.123372914454265 74.74612745763349) bank168735 +168736 POINT(40.83680093571252 74.34829196778335) bank168736 +168737 POINT(41.003662168764386 74.43801192888387) bank168737 +168738 POINT(39.83780292035395 74.75663717156242) bank168738 +168739 POINT(40.255000907919204 74.8096342035813) bank168739 +168740 POINT(41.4301458698505 74.064140872362) bank168740 +168741 POINT(40.19840409434368 74.89323752602769) bank168741 +168742 POINT(41.650491590896216 74.77169812733167) bank168742 +168743 POINT(41.59189034893363 73.34052274555346) bank168743 +168744 POINT(40.566179393344 73.18536503169999) bank168744 +168745 POINT(41.018680362519376 74.82328689242046) bank168745 +168746 POINT(41.48602006786476 73.83497094329817) bank168746 +168747 POINT(40.557832177322325 74.62933568958447) bank168747 +168748 POINT(41.31626681464322 74.88797455254823) bank168748 +168749 POINT(41.00477011907509 73.33877383235618) bank168749 +168750 POINT(41.3338681232417 74.84554386510665) bank168750 +168751 POINT(40.42002810430428 73.23539083448867) bank168751 +168752 POINT(40.392725566947476 73.40009834084864) bank168752 +168753 POINT(40.703737144795056 73.91416589982398) bank168753 +168754 POINT(40.655190728108046 73.40289460586774) bank168754 +168755 POINT(41.2826380443028 74.97465518305941) bank168755 +168756 POINT(39.722304915099706 74.60023729999179) bank168756 +168757 POINT(39.99179672524512 74.1353269727508) bank168757 +168758 POINT(41.18175420656266 74.11912493401388) bank168758 +168759 POINT(41.659905292357564 74.34835789141574) bank168759 +168760 POINT(39.98440508915524 73.30840400971181) bank168760 +168761 POINT(40.700749586071225 73.21961791456467) bank168761 +168762 POINT(41.467033279908044 73.44655557553209) bank168762 +168763 POINT(40.82712824149771 73.42188481115502) bank168763 +168764 POINT(40.46094690520489 73.41037781100276) bank168764 +168765 POINT(41.1808991126111 74.46172907038458) bank168765 +168766 POINT(40.87246271926031 73.99196314401365) bank168766 +168767 POINT(41.52365719494685 73.04453216609524) bank168767 +168768 POINT(40.71719296493059 74.85695703758823) bank168768 +168769 POINT(39.77860286668801 74.07827602597389) bank168769 +168770 POINT(41.2521470875048 74.67031793437282) bank168770 +168771 POINT(41.26772460467539 73.97128921203574) bank168771 +168772 POINT(40.042849536696394 74.29902248233819) bank168772 +168773 POINT(39.934763296420066 74.4778759923905) bank168773 +168774 POINT(40.046669091434545 73.0588273637458) bank168774 +168775 POINT(40.41697008056328 74.75470386956134) bank168775 +168776 POINT(40.04564577493367 73.20038901827415) bank168776 +168777 POINT(40.42474287917621 73.90896321587546) bank168777 +168778 POINT(40.16042169348259 73.3191110479489) bank168778 +168779 POINT(40.11026895797013 74.09202486294389) bank168779 +168780 POINT(40.35799368604757 73.4544606367995) bank168780 +168781 POINT(39.740086280857966 74.00654713655574) bank168781 +168782 POINT(40.91542266301067 73.03712054158795) bank168782 +168783 POINT(40.8805329585599 73.75159110744846) bank168783 +168784 POINT(40.04071430427139 74.558462974741) bank168784 +168785 POINT(40.62322326213019 73.027291391709) bank168785 +168786 POINT(40.237674615436894 73.70432863046587) bank168786 +168787 POINT(40.72442306528106 74.5138947016777) bank168787 +168788 POINT(41.698778410468734 74.82040015290775) bank168788 +168789 POINT(39.840679100662584 74.66611060155391) bank168789 +168790 POINT(40.23614696718867 74.84104649944459) bank168790 +168791 POINT(41.04054468459363 74.0434284203295) bank168791 +168792 POINT(39.82293209640376 74.51059908557653) bank168792 +168793 POINT(40.187873458286845 73.48418353501994) bank168793 +168794 POINT(41.1096820694555 74.58179304046381) bank168794 +168795 POINT(40.34388027269363 73.64337431035194) bank168795 +168796 POINT(40.02059296892793 73.47120628084146) bank168796 +168797 POINT(41.35723917666802 74.83702433766253) bank168797 +168798 POINT(40.081681489231045 74.83578122221776) bank168798 +168799 POINT(40.41700331680177 74.1707818624618) bank168799 +168800 POINT(41.602415535819354 73.07444590441867) bank168800 +168801 POINT(41.62491772232114 73.59764320983258) bank168801 +168802 POINT(40.72440118235745 74.79648500786183) bank168802 +168803 POINT(41.524529331948294 73.77192481793355) bank168803 +168804 POINT(40.39689130078346 74.83180347834129) bank168804 +168805 POINT(40.36174906625395 74.47342201352193) bank168805 +168806 POINT(41.47104734808072 73.48716460961593) bank168806 +168807 POINT(39.762218934466816 74.16215381346936) bank168807 +168808 POINT(40.82545611150596 74.84331550384707) bank168808 +168809 POINT(40.72117003112248 73.34570534793346) bank168809 +168810 POINT(41.570403922857764 73.24990786992977) bank168810 +168811 POINT(40.70645260180035 74.33548710994306) bank168811 +168812 POINT(40.6005145919919 73.0378504897156) bank168812 +168813 POINT(40.12392031708529 73.97758748453136) bank168813 +168814 POINT(41.69931183380722 73.56956141356063) bank168814 +168815 POINT(40.85592539883282 74.03754260413288) bank168815 +168816 POINT(41.26967538939288 74.47543802612917) bank168816 +168817 POINT(40.70389087592992 73.24380896623008) bank168817 +168818 POINT(40.11543598168916 74.15252556323173) bank168818 +168819 POINT(40.06810524303037 74.8078021387676) bank168819 +168820 POINT(40.887107017432136 74.41945703348843) bank168820 +168821 POINT(40.594438189300426 73.67571240007288) bank168821 +168822 POINT(40.4209318491701 74.12784842067995) bank168822 +168823 POINT(40.8531557543333 73.46600588151036) bank168823 +168824 POINT(40.67638018440921 74.57677379615843) bank168824 +168825 POINT(39.86075820521378 74.47814282777559) bank168825 +168826 POINT(41.0514424941691 74.36645499548592) bank168826 +168827 POINT(41.22574837096787 74.76573848241941) bank168827 +168828 POINT(41.39723771796263 74.86076965602325) bank168828 +168829 POINT(40.44156640277937 73.25712317260891) bank168829 +168830 POINT(40.06718215254436 73.58589173805267) bank168830 +168831 POINT(41.62198777383687 74.18554466593264) bank168831 +168832 POINT(41.458237341497544 73.70136587311954) bank168832 +168833 POINT(40.13430772363029 73.13609179789115) bank168833 +168834 POINT(39.89383881607523 74.55086727571577) bank168834 +168835 POINT(40.58261419567503 73.05086830239074) bank168835 +168836 POINT(39.91169916851717 74.20473330408853) bank168836 +168837 POINT(40.703218190135345 74.27415943065385) bank168837 +168838 POINT(39.92037836398191 73.16950830067907) bank168838 +168839 POINT(39.79666906400416 74.3770903310813) bank168839 +168840 POINT(40.61913142800047 73.35579624013128) bank168840 +168841 POINT(40.79156866376794 74.53305867654542) bank168841 +168842 POINT(39.89707065658783 74.1596445946342) bank168842 +168843 POINT(40.92002322382238 74.60519550337403) bank168843 +168844 POINT(41.710013568835755 73.18638189242905) bank168844 +168845 POINT(40.903107175262384 74.42031456462699) bank168845 +168846 POINT(39.77467679078534 74.8471318528293) bank168846 +168847 POINT(40.41654608363054 73.141199897312) bank168847 +168848 POINT(40.437037601588464 73.04521589698332) bank168848 +168849 POINT(40.90620137464094 74.36658796529527) bank168849 +168850 POINT(40.71916337891973 73.9854828876239) bank168850 +168851 POINT(40.26317969291092 74.72260639845264) bank168851 +168852 POINT(40.03988980379101 73.5736720399596) bank168852 +168853 POINT(40.15384337260471 74.22028250461182) bank168853 +168854 POINT(40.33396438543495 74.84534247395585) bank168854 +168855 POINT(40.40282239167283 73.69885299548315) bank168855 +168856 POINT(40.51044402532457 74.97625820100792) bank168856 +168857 POINT(39.75053723634289 73.4156381741597) bank168857 +168858 POINT(40.023034598604035 74.38051580661623) bank168858 +168859 POINT(41.23531458904985 73.23774666622668) bank168859 +168860 POINT(39.924552548787766 74.38386482694126) bank168860 +168861 POINT(40.53173812082077 73.95162955303806) bank168861 +168862 POINT(40.350207122013465 73.12822615010322) bank168862 +168863 POINT(41.11631502706395 74.89956490000175) bank168863 +168864 POINT(40.927528930109936 73.70454770661809) bank168864 +168865 POINT(40.63396570359134 73.44076663954732) bank168865 +168866 POINT(41.16259465164742 73.70544454395535) bank168866 +168867 POINT(40.76945042771293 73.16666194248701) bank168867 +168868 POINT(40.11635941364702 74.98731460345076) bank168868 +168869 POINT(40.60763853512788 73.47382868606545) bank168869 +168870 POINT(40.805076196767956 74.61243779568102) bank168870 +168871 POINT(41.0514093704219 73.79746070652008) bank168871 +168872 POINT(41.371330048052194 73.73307123819323) bank168872 +168873 POINT(40.926297017534274 73.16587494211747) bank168873 +168874 POINT(40.35680126659022 74.49969559470081) bank168874 +168875 POINT(40.586598766425595 73.31158564493509) bank168875 +168876 POINT(41.06430835133348 74.562649165344) bank168876 +168877 POINT(40.824791376732804 73.80701993557788) bank168877 +168878 POINT(40.78028134027955 73.51621030956878) bank168878 +168879 POINT(40.14441622657429 73.62871826264625) bank168879 +168880 POINT(40.99126944546786 73.08363352284921) bank168880 +168881 POINT(40.29219589553089 74.37120820011555) bank168881 +168882 POINT(41.26127617615136 73.97659300932938) bank168882 +168883 POINT(40.971256899377465 73.72958057784277) bank168883 +168884 POINT(41.18102522571904 74.75229232984466) bank168884 +168885 POINT(41.01895349641913 73.15119270631652) bank168885 +168886 POINT(41.05491904532417 73.99384207017374) bank168886 +168887 POINT(40.4775110047511 74.49239204264131) bank168887 +168888 POINT(40.19498311820596 73.18699238300482) bank168888 +168889 POINT(41.651110751967714 73.18067373170366) bank168889 +168890 POINT(39.97437063058172 73.90077197471422) bank168890 +168891 POINT(39.99093785988969 73.4124897671702) bank168891 +168892 POINT(40.5938081372316 74.67534852875185) bank168892 +168893 POINT(39.742341228216134 74.40955366081894) bank168893 +168894 POINT(40.51523393588791 74.17792319464152) bank168894 +168895 POINT(40.93663987705285 74.35564985536014) bank168895 +168896 POINT(41.13727728856015 74.82563029168875) bank168896 +168897 POINT(41.61955859475373 74.15601601439609) bank168897 +168898 POINT(39.88057985021309 73.33902027126044) bank168898 +168899 POINT(39.87246701947065 74.05648814964388) bank168899 +168900 POINT(41.36733043010199 73.10028367499127) bank168900 +168901 POINT(41.038775054315174 74.46835422744769) bank168901 +168902 POINT(40.6475170677387 74.6085293006261) bank168902 +168903 POINT(40.518727972251746 74.01084120202283) bank168903 +168904 POINT(41.450978537673706 74.4235583914259) bank168904 +168905 POINT(40.85871940413432 74.26229343523889) bank168905 +168906 POINT(39.74978145446159 73.2296026121636) bank168906 +168907 POINT(40.41738026238726 73.23184502579274) bank168907 +168908 POINT(40.56028182222349 73.35130000991558) bank168908 +168909 POINT(40.4287899242236 74.86316327877452) bank168909 +168910 POINT(40.87431188606371 73.61154998505249) bank168910 +168911 POINT(40.78060070209876 74.14033515160096) bank168911 +168912 POINT(39.91455705552977 74.328355756545) bank168912 +168913 POINT(39.722913686344505 73.46012532472304) bank168913 +168914 POINT(40.33461299657077 73.93711926238927) bank168914 +168915 POINT(39.76468192946045 73.68944250113277) bank168915 +168916 POINT(41.34577200428383 74.33939904169475) bank168916 +168917 POINT(41.59541652381922 73.20266916080335) bank168917 +168918 POINT(39.92203491968839 74.6761467692048) bank168918 +168919 POINT(40.23308096619515 73.78477931690519) bank168919 +168920 POINT(40.14864883867474 73.738908717617) bank168920 +168921 POINT(40.154906004154185 73.0259314254919) bank168921 +168922 POINT(39.9884854040833 73.18741559514423) bank168922 +168923 POINT(41.535298820245714 73.60857898551151) bank168923 +168924 POINT(40.85236143417279 74.3716077218836) bank168924 +168925 POINT(40.981826836217536 73.51988209016902) bank168925 +168926 POINT(40.944112557298844 73.76733495461544) bank168926 +168927 POINT(40.599576472496786 74.6497536429144) bank168927 +168928 POINT(40.93153915843364 73.92877320254111) bank168928 +168929 POINT(40.68535241929179 74.78302363160257) bank168929 +168930 POINT(41.59718968282963 73.20798592541526) bank168930 +168931 POINT(40.55669242713399 74.81947773505944) bank168931 +168932 POINT(40.8895498978331 74.89228642684895) bank168932 +168933 POINT(40.51455523106245 73.45285985696047) bank168933 +168934 POINT(41.071412864362266 74.87096847004503) bank168934 +168935 POINT(41.0424839200877 74.27572554842769) bank168935 +168936 POINT(40.3225632751965 74.50588015630956) bank168936 +168937 POINT(39.727921002996816 73.02097139534159) bank168937 +168938 POINT(40.373989618624094 74.44251602155384) bank168938 +168939 POINT(39.82863790868404 74.99068460495236) bank168939 +168940 POINT(40.12190089257522 73.59156501688956) bank168940 +168941 POINT(40.88902679323637 74.59938060468168) bank168941 +168942 POINT(40.550709722106944 74.03797164839608) bank168942 +168943 POINT(41.66563007448431 73.79196231205783) bank168943 +168944 POINT(40.138210935450765 73.02048415963127) bank168944 +168945 POINT(40.723309175912945 74.08582745093683) bank168945 +168946 POINT(40.35596155083536 73.61351902448071) bank168946 +168947 POINT(40.80584908137263 73.91197641686121) bank168947 +168948 POINT(40.957169908208904 74.55443276658663) bank168948 +168949 POINT(41.36387587509288 73.66325605701088) bank168949 +168950 POINT(41.2633545016544 74.46272876461258) bank168950 +168951 POINT(40.75217139058176 74.01341574025037) bank168951 +168952 POINT(40.450812195959294 73.69613112376557) bank168952 +168953 POINT(40.8725481222506 74.0055363523255) bank168953 +168954 POINT(40.72988380843254 73.15438843719707) bank168954 +168955 POINT(40.25224735904432 74.24089219118514) bank168955 +168956 POINT(41.14770696691658 73.02166569118387) bank168956 +168957 POINT(40.503032592084715 73.03255534127256) bank168957 +168958 POINT(40.012708376559935 74.5506767429258) bank168958 +168959 POINT(40.35742419025156 74.0348210904765) bank168959 +168960 POINT(41.171016516554324 74.74391252546236) bank168960 +168961 POINT(41.121926926765184 74.51526588000146) bank168961 +168962 POINT(40.300252692124026 74.49721437301322) bank168962 +168963 POINT(40.42583331273907 74.2091900354852) bank168963 +168964 POINT(40.71178967849149 74.67580770044354) bank168964 +168965 POINT(41.022235381574376 73.31423303921775) bank168965 +168966 POINT(40.55254543937932 73.5688104681686) bank168966 +168967 POINT(40.40974161464937 74.17142050206964) bank168967 +168968 POINT(41.51437238862235 74.41278445959203) bank168968 +168969 POINT(40.89251666272985 74.14768046847149) bank168969 +168970 POINT(40.154132989031744 74.89456341756885) bank168970 +168971 POINT(40.858595132319266 73.10595167560281) bank168971 +168972 POINT(41.06651172295425 73.4350373724033) bank168972 +168973 POINT(39.748300888059866 74.83037538868889) bank168973 +168974 POINT(41.54193404658751 74.57946772326018) bank168974 +168975 POINT(41.347305123022565 73.03671050061756) bank168975 +168976 POINT(40.0918890698959 74.4003766574838) bank168976 +168977 POINT(40.11679252701607 74.18008626098269) bank168977 +168978 POINT(40.525260287211964 73.66666534262762) bank168978 +168979 POINT(41.03048677061486 74.5670256669893) bank168979 +168980 POINT(40.75730323267392 73.14309193632997) bank168980 +168981 POINT(40.81180536988483 74.84474937324627) bank168981 +168982 POINT(40.17214925227348 74.15868488658568) bank168982 +168983 POINT(40.952234691415704 73.55868552412872) bank168983 +168984 POINT(39.89612059077509 73.74419401178787) bank168984 +168985 POINT(41.51873474281211 73.79059747776721) bank168985 +168986 POINT(40.52624534788392 74.22445899002764) bank168986 +168987 POINT(41.03963923727879 74.54655438481107) bank168987 +168988 POINT(41.3268267655707 74.6375709246902) bank168988 +168989 POINT(39.74298474528248 73.74500442500981) bank168989 +168990 POINT(40.845757653099255 73.23019357658586) bank168990 +168991 POINT(40.45589126832626 74.2109602027811) bank168991 +168992 POINT(40.82415223871906 74.10140914110568) bank168992 +168993 POINT(40.959618995708404 73.03761638896879) bank168993 +168994 POINT(40.40840377609529 74.61697835910228) bank168994 +168995 POINT(41.231097686297545 74.84822369258227) bank168995 +168996 POINT(40.66923054586802 74.26425741534631) bank168996 +168997 POINT(40.57908832951275 73.20286861867106) bank168997 +168998 POINT(40.12362921379068 73.41544217726806) bank168998 +168999 POINT(40.64604466294601 73.65639084176948) bank168999 +169000 POINT(41.62985421178874 74.11961171997241) bank169000 +169001 POINT(41.4208873825812 73.42083319041295) bank169001 +169002 POINT(40.528956495848334 74.51719168844777) bank169002 +169003 POINT(40.13195585936091 73.01594644651678) bank169003 +169004 POINT(41.6039895007088 73.26758295370398) bank169004 +169005 POINT(41.338578422147265 73.68879907542035) bank169005 +169006 POINT(41.216321812819366 74.93802736185182) bank169006 +169007 POINT(40.24069418898277 74.36989300670594) bank169007 +169008 POINT(39.88807949774386 74.06722691437244) bank169008 +169009 POINT(41.61332380289369 74.86059672600884) bank169009 +169010 POINT(40.277948058081826 74.1964908177376) bank169010 +169011 POINT(41.29904659640942 73.91544922104013) bank169011 +169012 POINT(41.60110790130043 73.77946645463425) bank169012 +169013 POINT(41.22880593949605 74.35576447259763) bank169013 +169014 POINT(40.6384560175833 74.77114303508644) bank169014 +169015 POINT(41.58848146688202 74.90627658741766) bank169015 +169016 POINT(40.68815977625014 74.99290203742392) bank169016 +169017 POINT(39.99618600314288 73.82521793361308) bank169017 +169018 POINT(40.8828783905549 73.24969004831603) bank169018 +169019 POINT(40.55310883641715 73.77676759224214) bank169019 +169020 POINT(40.46434515789697 73.1947807038858) bank169020 +169021 POINT(41.14808096076483 73.40482836926783) bank169021 +169022 POINT(39.842302071409954 73.74480006145843) bank169022 +169023 POINT(41.34839656802092 73.92087694563767) bank169023 +169024 POINT(41.17244017424527 74.82142899420462) bank169024 +169025 POINT(41.50662947610457 74.95468174568526) bank169025 +169026 POINT(41.351199326851905 74.90492294989953) bank169026 +169027 POINT(40.869939978375875 74.00433742494198) bank169027 +169028 POINT(41.43225552436805 74.24314358261273) bank169028 +169029 POINT(41.659797998379716 73.15208871004927) bank169029 +169030 POINT(39.931349251864034 73.5866040367787) bank169030 +169031 POINT(39.868833672806055 73.52238862938498) bank169031 +169032 POINT(40.70142518542152 73.70483568075716) bank169032 +169033 POINT(39.725861264427415 74.16569689345329) bank169033 +169034 POINT(41.225287030070405 74.55327388031415) bank169034 +169035 POINT(40.1130790276415 73.4924832075971) bank169035 +169036 POINT(40.6885597759817 73.34732163450182) bank169036 +169037 POINT(41.264257725751584 73.37963515434322) bank169037 +169038 POINT(41.2213544717741 73.53777285726545) bank169038 +169039 POINT(40.04950103098261 74.49533735018247) bank169039 +169040 POINT(40.2137030943968 73.64370473397155) bank169040 +169041 POINT(39.75031070998584 73.4244759052574) bank169041 +169042 POINT(41.493727751469166 74.08846803141448) bank169042 +169043 POINT(39.82591566721408 73.17758016853796) bank169043 +169044 POINT(41.620712982756174 74.64069894749471) bank169044 +169045 POINT(40.72377698650381 73.96876446314796) bank169045 +169046 POINT(40.60180366015536 74.1299388742673) bank169046 +169047 POINT(40.848575426666955 73.98288119632666) bank169047 +169048 POINT(41.185771628078115 74.14911899960481) bank169048 +169049 POINT(40.02928063889289 74.64321639079908) bank169049 +169050 POINT(41.411201954243836 74.83685280603373) bank169050 +169051 POINT(41.42801589575675 74.12569172452082) bank169051 +169052 POINT(40.32129472997273 73.81828967882966) bank169052 +169053 POINT(40.93805934490178 74.74213929187825) bank169053 +169054 POINT(40.04196616412418 73.74670735776961) bank169054 +169055 POINT(41.01913805929359 73.1960813551349) bank169055 +169056 POINT(41.586189757538406 73.91210311173054) bank169056 +169057 POINT(40.10538779458103 73.68574525937025) bank169057 +169058 POINT(40.92565337215023 74.9046136824877) bank169058 +169059 POINT(41.621385173711104 74.30084985329553) bank169059 +169060 POINT(41.34185881323549 74.39490920326254) bank169060 +169061 POINT(39.810169111917716 73.6318691028372) bank169061 +169062 POINT(39.95976772285115 73.68623983411666) bank169062 +169063 POINT(41.17618411044774 73.34920132510362) bank169063 +169064 POINT(40.7250582169254 74.82565965386425) bank169064 +169065 POINT(41.231594387481884 73.12268783737616) bank169065 +169066 POINT(40.632557999892015 73.46442157324901) bank169066 +169067 POINT(41.35016807675353 74.13234521776853) bank169067 +169068 POINT(41.165227680167284 74.90406306453488) bank169068 +169069 POINT(41.342905855501144 73.33056445618566) bank169069 +169070 POINT(41.04676310739086 74.27223733558813) bank169070 +169071 POINT(41.44852037346581 74.98572813344833) bank169071 +169072 POINT(41.119308309260354 74.81428586363675) bank169072 +169073 POINT(40.21626792015485 73.07820637521681) bank169073 +169074 POINT(40.152367704643694 74.69231064485972) bank169074 +169075 POINT(39.98298178274861 73.08017701370974) bank169075 +169076 POINT(41.67468798053585 73.65669136190155) bank169076 +169077 POINT(40.4559436303433 73.59273268585002) bank169077 +169078 POINT(40.82757246492701 74.39413810777226) bank169078 +169079 POINT(40.08957787799629 74.5647768708312) bank169079 +169080 POINT(40.12310806274663 74.18509585421994) bank169080 +169081 POINT(40.101582342597155 73.6503131225229) bank169081 +169082 POINT(39.97991779080873 73.55619062344803) bank169082 +169083 POINT(41.52757879273082 74.2329350893751) bank169083 +169084 POINT(40.25128985558545 74.30723548902823) bank169084 +169085 POINT(41.562922518462344 74.14108507578922) bank169085 +169086 POINT(41.56096178867725 73.1005080876394) bank169086 +169087 POINT(41.191631644223364 74.45585919285952) bank169087 +169088 POINT(40.78324332341167 74.72597866534232) bank169088 +169089 POINT(40.450260940821344 73.87263092755606) bank169089 +169090 POINT(41.1656542439852 73.22440264042751) bank169090 +169091 POINT(39.717055225594684 74.83460448147034) bank169091 +169092 POINT(40.109517956827105 74.17980201790485) bank169092 +169093 POINT(41.135821545167396 74.61676961399667) bank169093 +169094 POINT(39.855569914050086 74.69126873240397) bank169094 +169095 POINT(41.00795265809682 73.31307407038345) bank169095 +169096 POINT(40.434161754654305 73.39478211925066) bank169096 +169097 POINT(41.018033404206975 74.21501369751246) bank169097 +169098 POINT(41.449650800013444 74.6528882281752) bank169098 +169099 POINT(40.65277477052873 74.72793067990484) bank169099 +169100 POINT(40.208324569872126 74.72372951115025) bank169100 +169101 POINT(40.63138449580549 74.9695678392451) bank169101 +169102 POINT(41.570853817427654 74.00893834528468) bank169102 +169103 POINT(40.61073298220957 74.66263853547181) bank169103 +169104 POINT(40.568094699919364 73.66111414063833) bank169104 +169105 POINT(41.43970519959565 74.10262899175649) bank169105 +169106 POINT(41.1569262608819 74.73926355739556) bank169106 +169107 POINT(39.89178931927555 73.44793671254327) bank169107 +169108 POINT(40.760436660582556 73.5602319637275) bank169108 +169109 POINT(40.69669002223239 74.51798792440432) bank169109 +169110 POINT(40.75259039976068 73.32668870458853) bank169110 +169111 POINT(40.4165078719269 74.45402343766884) bank169111 +169112 POINT(41.65796879022642 73.7315352763509) bank169112 +169113 POINT(40.76580407850387 74.69621846525176) bank169113 +169114 POINT(39.92335918175582 74.47850902040837) bank169114 +169115 POINT(40.998210584931016 73.92132291170441) bank169115 +169116 POINT(41.251842098037045 74.63914038958696) bank169116 +169117 POINT(41.02219058463502 73.33107405719011) bank169117 +169118 POINT(41.28478686571583 73.98591769663848) bank169118 +169119 POINT(39.95762965481565 74.71800002850736) bank169119 +169120 POINT(40.44426344795068 74.85580698459567) bank169120 +169121 POINT(41.60104705610342 74.54397382483032) bank169121 +169122 POINT(40.741009297828136 74.18070881627922) bank169122 +169123 POINT(40.752968952637566 73.08358716733476) bank169123 +169124 POINT(40.599079602321055 73.5207092533346) bank169124 +169125 POINT(41.028275551370285 74.3483328429148) bank169125 +169126 POINT(41.010109090288395 74.1990350824846) bank169126 +169127 POINT(40.50959337817027 74.18059848940156) bank169127 +169128 POINT(41.34381296470779 73.25738778457836) bank169128 +169129 POINT(41.03722139658182 74.90218776927047) bank169129 +169130 POINT(40.60484252340326 73.2946685047805) bank169130 +169131 POINT(41.423083603303866 74.04249146379125) bank169131 +169132 POINT(40.398016432147465 73.07400103042724) bank169132 +169133 POINT(40.7696049849837 73.09305522616272) bank169133 +169134 POINT(40.86087580207915 74.05464620489701) bank169134 +169135 POINT(41.150534030671864 73.80430832059933) bank169135 +169136 POINT(41.33551501650639 74.19126862243658) bank169136 +169137 POINT(40.530980384275836 73.35681287210564) bank169137 +169138 POINT(40.51050866850616 73.60395706386167) bank169138 +169139 POINT(40.23571422647274 73.7349993261069) bank169139 +169140 POINT(41.06442918409399 73.05663996855465) bank169140 +169141 POINT(40.66773965937715 73.54991422530958) bank169141 +169142 POINT(39.81115579045291 73.02051892793412) bank169142 +169143 POINT(41.53923745007637 73.72138925879749) bank169143 +169144 POINT(39.82673996698133 73.17342961885582) bank169144 +169145 POINT(39.89990291802652 74.54961512463635) bank169145 +169146 POINT(41.56717116967921 74.3324339569241) bank169146 +169147 POINT(39.74791990062119 74.10379714108726) bank169147 +169148 POINT(40.28382801768551 73.91554116578935) bank169148 +169149 POINT(39.87423315071441 74.14920079237605) bank169149 +169150 POINT(41.23242975149468 74.83130886638793) bank169150 +169151 POINT(40.86488743965649 74.32138865813758) bank169151 +169152 POINT(40.17558297834088 74.07522805937596) bank169152 +169153 POINT(40.301974995015286 74.6461735208128) bank169153 +169154 POINT(39.7306504816098 73.96658310203458) bank169154 +169155 POINT(39.716251731824805 73.0336179536132) bank169155 +169156 POINT(41.36803200157527 73.60259203317383) bank169156 +169157 POINT(40.812148382563265 73.29203140819132) bank169157 +169158 POINT(39.74118610374185 73.70899013003695) bank169158 +169159 POINT(41.00900942673282 74.03729038787166) bank169159 +169160 POINT(39.77807898468932 74.67518981913386) bank169160 +169161 POINT(40.590798639641164 73.63445740646) bank169161 +169162 POINT(40.92439472844317 74.59034225726013) bank169162 +169163 POINT(40.7756328216739 73.65479768571375) bank169163 +169164 POINT(39.78579835836962 74.73448132905175) bank169164 +169165 POINT(41.67857940164613 73.006722160074) bank169165 +169166 POINT(40.7235900915932 73.26038200525784) bank169166 +169167 POINT(40.18414983632946 73.73230514947882) bank169167 +169168 POINT(41.59511938597107 74.63040448900226) bank169168 +169169 POINT(40.36786178610571 74.02967730307269) bank169169 +169170 POINT(39.99413272515083 73.70891052642976) bank169170 +169171 POINT(39.73354567499392 73.64800637555875) bank169171 +169172 POINT(40.015267692996005 74.32381505762635) bank169172 +169173 POINT(41.683248047311366 74.2077293728588) bank169173 +169174 POINT(39.883049784146756 74.32285504347053) bank169174 +169175 POINT(40.12821988821087 74.32131122583382) bank169175 +169176 POINT(40.91702753461948 74.52699655132086) bank169176 +169177 POINT(40.23836185239453 73.20378323939504) bank169177 +169178 POINT(41.629633689447985 74.07693070061634) bank169178 +169179 POINT(40.90755970712143 74.18486751419599) bank169179 +169180 POINT(41.345009781944384 74.11402692395575) bank169180 +169181 POINT(41.54716516486277 73.91211842963686) bank169181 +169182 POINT(41.24875305439412 73.2248277357476) bank169182 +169183 POINT(40.16942599994551 73.41139013090384) bank169183 +169184 POINT(41.629364716949134 73.02806765512376) bank169184 +169185 POINT(40.61501217439948 74.63622179613651) bank169185 +169186 POINT(41.177473527827345 74.20215868879495) bank169186 +169187 POINT(41.41807591073183 73.5039192224137) bank169187 +169188 POINT(40.54230376339236 74.54658392358374) bank169188 +169189 POINT(40.89431021080933 74.49967518641404) bank169189 +169190 POINT(40.968898012311826 74.73562101554177) bank169190 +169191 POINT(41.26282584518567 73.2936293502109) bank169191 +169192 POINT(39.82579307843829 74.34431217561243) bank169192 +169193 POINT(39.93357092307494 73.77653542948353) bank169193 +169194 POINT(40.75076586603396 73.27828036220416) bank169194 +169195 POINT(41.2214971553154 73.83579843999206) bank169195 +169196 POINT(41.33982732464083 73.31280289454884) bank169196 +169197 POINT(39.930723141891164 73.48395921861422) bank169197 +169198 POINT(41.4404089161654 73.99834769431399) bank169198 +169199 POINT(40.3766373623317 74.12034443107287) bank169199 +169200 POINT(41.25909002976367 74.25571544577923) bank169200 +169201 POINT(41.370030305298314 73.47147503021247) bank169201 +169202 POINT(40.94326012234209 74.23118786319884) bank169202 +169203 POINT(41.61840247313996 73.84116536171987) bank169203 +169204 POINT(41.54641762591313 74.85629090157141) bank169204 +169205 POINT(40.46248662647301 73.28416391164919) bank169205 +169206 POINT(40.197486911006024 73.83170229895653) bank169206 +169207 POINT(40.79830770688564 74.17090695243988) bank169207 +169208 POINT(40.37029350187024 73.99658594797754) bank169208 +169209 POINT(39.71384528346158 74.72745962288494) bank169209 +169210 POINT(40.33523972042288 74.08623350122969) bank169210 +169211 POINT(40.26639704232419 73.24907810420012) bank169211 +169212 POINT(41.123022522532395 74.80341832278029) bank169212 +169213 POINT(41.50782582807217 74.13911207984803) bank169213 +169214 POINT(40.806432603076026 74.06391298442979) bank169214 +169215 POINT(41.59857496036203 73.27265398619069) bank169215 +169216 POINT(41.5037218747631 74.06085622476311) bank169216 +169217 POINT(41.577133763109494 74.9017680626861) bank169217 +169218 POINT(40.95136883231238 73.64144118051574) bank169218 +169219 POINT(39.77686396272929 73.146293090369) bank169219 +169220 POINT(39.870639730943104 73.77085154919423) bank169220 +169221 POINT(39.987092567961604 73.74173529037498) bank169221 +169222 POINT(41.58832102171652 73.20905393806962) bank169222 +169223 POINT(40.45451496994664 74.14349178054506) bank169223 +169224 POINT(40.96486187391158 73.66855455992041) bank169224 +169225 POINT(40.16624121745501 73.01313633778473) bank169225 +169226 POINT(41.56439677034858 73.25721851175719) bank169226 +169227 POINT(41.55553607970641 74.11988358767854) bank169227 +169228 POINT(40.487905334330534 73.6474798686276) bank169228 +169229 POINT(40.526948023698885 73.09102706155834) bank169229 +169230 POINT(41.680127163507116 74.93136638817762) bank169230 +169231 POINT(40.04007366712826 73.47132687233827) bank169231 +169232 POINT(40.576434645474144 74.17278929400199) bank169232 +169233 POINT(40.095573713639375 73.58374899439029) bank169233 +169234 POINT(40.656441125631595 74.07721190402071) bank169234 +169235 POINT(40.07815602227174 73.98300914230437) bank169235 +169236 POINT(40.219816225409694 74.49549680148999) bank169236 +169237 POINT(41.457006141153364 74.7888904388473) bank169237 +169238 POINT(41.307658040367826 73.87471637251916) bank169238 +169239 POINT(40.435343519917424 73.07527486015853) bank169239 +169240 POINT(41.540354044090904 74.52077359246202) bank169240 +169241 POINT(41.050470426166 74.25676250409762) bank169241 +169242 POINT(40.96616203637238 73.76000357869493) bank169242 +169243 POINT(40.54868799192099 73.63306845454755) bank169243 +169244 POINT(41.28002571244398 74.05777974615657) bank169244 +169245 POINT(41.69328158550616 73.19928848280706) bank169245 +169246 POINT(40.110088970899156 74.82351933823426) bank169246 +169247 POINT(41.15294348115776 74.44782958483006) bank169247 +169248 POINT(40.48866068608862 74.52030241278233) bank169248 +169249 POINT(41.46785519435627 73.94514642763906) bank169249 +169250 POINT(40.58572332574933 73.664290726652) bank169250 +169251 POINT(39.91855706913057 73.30121951838304) bank169251 +169252 POINT(41.36474279275741 73.36795903622213) bank169252 +169253 POINT(40.15560135978141 74.73994206198829) bank169253 +169254 POINT(41.439622980899394 73.28495281918804) bank169254 +169255 POINT(40.68441552605139 74.74549998812918) bank169255 +169256 POINT(40.85432541523157 74.32268701831336) bank169256 +169257 POINT(41.10833766993474 73.28381140444488) bank169257 +169258 POINT(40.517709309699384 73.71119011110251) bank169258 +169259 POINT(40.7054943929823 74.97352161637114) bank169259 +169260 POINT(39.88716767991843 73.57157005204378) bank169260 +169261 POINT(41.32065545650715 74.28338603630078) bank169261 +169262 POINT(40.29603832478581 74.14222960137457) bank169262 +169263 POINT(41.42868282923532 73.6035648911937) bank169263 +169264 POINT(41.5094435216509 73.73689697809533) bank169264 +169265 POINT(39.80666368387541 73.4327657877678) bank169265 +169266 POINT(41.20687587319693 74.24437292792999) bank169266 +169267 POINT(41.291940969434634 74.44765858641718) bank169267 +169268 POINT(39.91285416923637 73.55722657142944) bank169268 +169269 POINT(41.51748935246799 74.60647008296363) bank169269 +169270 POINT(40.60570197772882 73.27982766329197) bank169270 +169271 POINT(40.36107393549121 74.43548079267559) bank169271 +169272 POINT(40.18030520419874 73.73363867983929) bank169272 +169273 POINT(41.22846278161668 74.80200854454982) bank169273 +169274 POINT(40.30700443741907 74.79421152824038) bank169274 +169275 POINT(41.51322246559355 74.2198954556922) bank169275 +169276 POINT(39.91114277344923 73.99130385493318) bank169276 +169277 POINT(41.39908432407438 73.64130672997086) bank169277 +169278 POINT(41.077757815661464 74.66633000948615) bank169278 +169279 POINT(40.39793944126848 74.87256650135056) bank169279 +169280 POINT(40.71836219426478 74.19909024880577) bank169280 +169281 POINT(41.48259919915185 74.09102868851802) bank169281 +169282 POINT(41.46134272784887 73.89422260354556) bank169282 +169283 POINT(40.01782818905647 74.51646952708349) bank169283 +169284 POINT(40.17408847397551 73.85131654003541) bank169284 +169285 POINT(40.24476725032658 73.84832476695848) bank169285 +169286 POINT(41.05082505892274 74.52881614818392) bank169286 +169287 POINT(40.1486586170402 73.74820619265041) bank169287 +169288 POINT(40.98856030165813 73.12360612873685) bank169288 +169289 POINT(40.86538989477521 74.03462838250239) bank169289 +169290 POINT(40.11890566682867 73.87739655269344) bank169290 +169291 POINT(40.48063802214324 74.94241182103329) bank169291 +169292 POINT(40.79467400736779 74.91970246078071) bank169292 +169293 POINT(41.36311212866447 74.8540373709561) bank169293 +169294 POINT(40.307487739292355 74.912053033309) bank169294 +169295 POINT(40.889369977878935 74.00789983159008) bank169295 +169296 POINT(40.77210927968873 74.28585085677028) bank169296 +169297 POINT(40.087120714881 74.7000782526983) bank169297 +169298 POINT(41.4170998056387 73.65652375130476) bank169298 +169299 POINT(40.59887164532526 74.2926041390127) bank169299 +169300 POINT(41.001370714893675 74.4130507932303) bank169300 +169301 POINT(40.30902010912849 74.98693104920322) bank169301 +169302 POINT(41.502764757267634 73.01245174000648) bank169302 +169303 POINT(39.996333908441315 73.98343433647038) bank169303 +169304 POINT(41.57261444543457 74.02306173586142) bank169304 +169305 POINT(41.4474534923045 74.47747710077593) bank169305 +169306 POINT(40.83694318229435 74.22380997348121) bank169306 +169307 POINT(41.15428162882903 74.6062049539447) bank169307 +169308 POINT(41.66657507773203 73.54811235020077) bank169308 +169309 POINT(41.15471870380631 74.33745748398779) bank169309 +169310 POINT(39.9474108278267 74.80894617032396) bank169310 +169311 POINT(40.332307817797 73.31475175346876) bank169311 +169312 POINT(40.72817017793335 74.29196551392947) bank169312 +169313 POINT(40.449628655028526 73.18646659747724) bank169313 +169314 POINT(39.92240104127212 74.57743816470423) bank169314 +169315 POINT(40.8751669678776 74.49044899291427) bank169315 +169316 POINT(39.91534165058202 74.57917961551357) bank169316 +169317 POINT(41.696584161793865 74.79031465171828) bank169317 +169318 POINT(40.0913252978342 74.21743363640174) bank169318 +169319 POINT(41.21423488705601 73.93252255912205) bank169319 +169320 POINT(39.99238857155064 74.19473548358462) bank169320 +169321 POINT(41.21866677464011 73.75914689597253) bank169321 +169322 POINT(40.93937673760697 73.40758977398062) bank169322 +169323 POINT(41.41018706386445 73.90072298824076) bank169323 +169324 POINT(41.27047492439285 73.9448443841547) bank169324 +169325 POINT(41.14920471917256 73.74766636399386) bank169325 +169326 POINT(40.89250627212839 74.81856556671522) bank169326 +169327 POINT(40.646232393680485 74.05569392707753) bank169327 +169328 POINT(41.507465588023116 73.87657099968484) bank169328 +169329 POINT(40.90716214872334 74.28986648628796) bank169329 +169330 POINT(40.66860196249532 73.56459491685521) bank169330 +169331 POINT(41.53339963248821 74.15860491640512) bank169331 +169332 POINT(41.06387223498602 73.4983614857568) bank169332 +169333 POINT(41.64197652517481 73.93273389574765) bank169333 +169334 POINT(40.24123905615012 73.27337419143475) bank169334 +169335 POINT(41.679371376763164 74.48216069336111) bank169335 +169336 POINT(41.51724854856121 73.51363736157053) bank169336 +169337 POINT(40.148391065735716 73.54635830600252) bank169337 +169338 POINT(40.29889521954194 74.09808671557195) bank169338 +169339 POINT(41.678858573261685 73.69047648072043) bank169339 +169340 POINT(40.276210201907034 73.78799517048608) bank169340 +169341 POINT(40.68716090556236 73.1513013415897) bank169341 +169342 POINT(40.95437375237889 74.79646819294246) bank169342 +169343 POINT(41.31385508433288 73.02415377383012) bank169343 +169344 POINT(40.73685948004754 73.66593122482968) bank169344 +169345 POINT(39.73888895096498 73.75652094137716) bank169345 +169346 POINT(39.74365242243614 73.14576032014212) bank169346 +169347 POINT(40.464316964889235 74.4645459990541) bank169347 +169348 POINT(40.14634826335612 73.27617007673511) bank169348 +169349 POINT(41.261691507223006 74.15974070479557) bank169349 +169350 POINT(39.90136652985009 73.64896336713352) bank169350 +169351 POINT(41.514861856130956 73.68639157527889) bank169351 +169352 POINT(40.472619268664864 74.73414920365737) bank169352 +169353 POINT(41.537490746873196 73.87210834988011) bank169353 +169354 POINT(41.182467718971296 74.05905192871133) bank169354 +169355 POINT(40.23619301755632 74.37622018680405) bank169355 +169356 POINT(40.08795471687912 73.91531203160336) bank169356 +169357 POINT(40.361053749327475 74.3294758907045) bank169357 +169358 POINT(41.07567311801739 73.98983416157758) bank169358 +169359 POINT(41.266231837252704 73.97142692928402) bank169359 +169360 POINT(40.63250610964456 74.77442675343447) bank169360 +169361 POINT(41.14792728921667 74.80704951490226) bank169361 +169362 POINT(39.90179669297597 74.90965983460808) bank169362 +169363 POINT(40.74408867948366 73.57988162819184) bank169363 +169364 POINT(39.8134021342592 73.6507655355208) bank169364 +169365 POINT(40.49411762508214 74.61202176185732) bank169365 +169366 POINT(40.167633737748254 73.18092503503378) bank169366 +169367 POINT(40.73659492670914 74.273747872269) bank169367 +169368 POINT(40.304280688075046 73.71899622385946) bank169368 +169369 POINT(40.830143710699524 73.85283461943334) bank169369 +169370 POINT(40.32509334663759 73.12554430009419) bank169370 +169371 POINT(41.176133274764034 73.86927168951043) bank169371 +169372 POINT(40.05713271527893 73.80808297306962) bank169372 +169373 POINT(41.49841805884717 73.96729663082036) bank169373 +169374 POINT(40.112110621956056 73.47530932352218) bank169374 +169375 POINT(40.857083603891944 73.7367177876255) bank169375 +169376 POINT(40.95027456725172 73.68358362310126) bank169376 +169377 POINT(39.86516647618115 74.71583252127161) bank169377 +169378 POINT(41.11034595562874 73.56221864541412) bank169378 +169379 POINT(41.22494560824675 74.69305531774499) bank169379 +169380 POINT(39.80025048771775 73.765451060932) bank169380 +169381 POINT(39.96581001628867 73.66877125904725) bank169381 +169382 POINT(40.78544429101691 73.6209255018747) bank169382 +169383 POINT(40.073584366842766 73.5604667697447) bank169383 +169384 POINT(41.1400935257448 73.93852670005867) bank169384 +169385 POINT(39.97797614037539 73.98273304074682) bank169385 +169386 POINT(40.906210205566886 73.21532211164278) bank169386 +169387 POINT(40.72291412258546 74.35334371133251) bank169387 +169388 POINT(41.06998996111191 74.22592961295692) bank169388 +169389 POINT(39.87951049670387 73.39426436168883) bank169389 +169390 POINT(41.56289531673644 73.99086682200853) bank169390 +169391 POINT(40.277594554034344 73.56817748529063) bank169391 +169392 POINT(41.13667595138356 74.00005023198258) bank169392 +169393 POINT(41.14596902087397 73.32234591838433) bank169393 +169394 POINT(40.4796738043614 74.78014479178887) bank169394 +169395 POINT(41.50942025022733 73.2808881467461) bank169395 +169396 POINT(40.75324471826835 74.41991698660698) bank169396 +169397 POINT(40.83725101630235 75.00585518461884) bank169397 +169398 POINT(41.15118677003978 74.25356361771323) bank169398 +169399 POINT(40.15539575579412 73.5271125009821) bank169399 +169400 POINT(40.56922495698296 73.0355924051846) bank169400 +169401 POINT(40.25610235306889 74.79400490271999) bank169401 +169402 POINT(39.8814356691124 73.36310726829308) bank169402 +169403 POINT(41.478294733281274 74.63220074524156) bank169403 +169404 POINT(41.52389154873242 73.22085670637628) bank169404 +169405 POINT(40.17677535278419 73.73716893143673) bank169405 +169406 POINT(39.867971286071224 73.25404545264496) bank169406 +169407 POINT(41.07488122904534 73.77553962904639) bank169407 +169408 POINT(40.70683545032759 73.26794381471677) bank169408 +169409 POINT(39.92967425877592 74.670724979184) bank169409 +169410 POINT(40.69719125468635 74.81760901582548) bank169410 +169411 POINT(41.01093486378145 73.33799617580887) bank169411 +169412 POINT(40.70379677918746 73.54108083715893) bank169412 +169413 POINT(39.805712535192434 74.38460476448051) bank169413 +169414 POINT(40.04140807860647 73.03326537990105) bank169414 +169415 POINT(39.944463514292224 74.98517945108195) bank169415 +169416 POINT(41.29527165550754 73.41462369461685) bank169416 +169417 POINT(40.88508643461485 74.12383291100073) bank169417 +169418 POINT(39.90297172517035 74.11669971064579) bank169418 +169419 POINT(40.15826484700767 73.39068036234468) bank169419 +169420 POINT(40.95926434625391 73.42721435813588) bank169420 +169421 POINT(40.00612018331366 73.44621622786963) bank169421 +169422 POINT(41.315746048875624 73.42122045403954) bank169422 +169423 POINT(39.85484054266226 73.54036237035677) bank169423 +169424 POINT(40.8368199883277 73.7880866551175) bank169424 +169425 POINT(40.25116576219871 74.80071891231363) bank169425 +169426 POINT(40.1960194026445 73.64168015884529) bank169426 +169427 POINT(41.12691454533521 73.58270218712285) bank169427 +169428 POINT(39.71553632554169 74.11075043977515) bank169428 +169429 POINT(41.297613413487475 73.24065155038505) bank169429 +169430 POINT(40.33713364198797 74.70658686892114) bank169430 +169431 POINT(40.24908352359861 74.40803946594531) bank169431 +169432 POINT(40.38255045113081 73.41477244764938) bank169432 +169433 POINT(39.88238748105262 73.62284010005357) bank169433 +169434 POINT(40.983134364460675 74.93280789286923) bank169434 +169435 POINT(41.46512540849119 74.27070340798738) bank169435 +169436 POINT(40.58288648873276 73.06203500836031) bank169436 +169437 POINT(40.92997449176745 74.81415640326257) bank169437 +169438 POINT(40.120823026962455 73.16339424786038) bank169438 +169439 POINT(40.85512824545901 74.49888468675258) bank169439 +169440 POINT(39.76166884043702 73.49695260873989) bank169440 +169441 POINT(41.492174970900415 73.5492330316132) bank169441 +169442 POINT(40.93009891108562 73.72707979272558) bank169442 +169443 POINT(40.73477310957101 73.74806244621743) bank169443 +169444 POINT(40.590005341115564 74.59892595193158) bank169444 +169445 POINT(41.61421152242828 74.94382705120242) bank169445 +169446 POINT(40.905565401497356 74.94961051360632) bank169446 +169447 POINT(40.249912131811485 74.49380899401815) bank169447 +169448 POINT(41.709260373972995 73.9943404993228) bank169448 +169449 POINT(40.736169053780216 74.63215010200851) bank169449 +169450 POINT(39.81368987795105 74.98708651177026) bank169450 +169451 POINT(40.02454563043142 73.51593091223324) bank169451 +169452 POINT(40.62186041684004 74.16255817430036) bank169452 +169453 POINT(41.21213553770575 74.2057334733301) bank169453 +169454 POINT(41.65016237140833 73.85683505098831) bank169454 +169455 POINT(39.877627555418385 73.29854873701045) bank169455 +169456 POINT(39.85487296740304 74.14326465753263) bank169456 +169457 POINT(39.80311176458543 74.0850318177968) bank169457 +169458 POINT(39.93130662588997 73.98853808912065) bank169458 +169459 POINT(39.83606019730378 74.35097899961232) bank169459 +169460 POINT(40.17891108922551 73.51351620903343) bank169460 +169461 POINT(41.28108953311323 73.46694835839244) bank169461 +169462 POINT(40.54256734382726 74.4309702071103) bank169462 +169463 POINT(40.45604968443524 73.83876343349564) bank169463 +169464 POINT(41.204710123841664 74.14603484547217) bank169464 +169465 POINT(41.54797311642577 73.4501812649522) bank169465 +169466 POINT(41.199210994114 73.0717662496258) bank169466 +169467 POINT(40.443071206632005 74.37382553877343) bank169467 +169468 POINT(40.02242689259784 74.59108006164432) bank169468 +169469 POINT(39.93142539621897 73.09672883558557) bank169469 +169470 POINT(40.45490088225655 73.24601096018975) bank169470 +169471 POINT(40.3264095364721 74.51303214499401) bank169471 +169472 POINT(40.69345286962643 74.11535407483957) bank169472 +169473 POINT(41.029889630377205 73.4829990453848) bank169473 +169474 POINT(39.80376064075462 73.28203166741874) bank169474 +169475 POINT(41.702740993734096 74.5219130206369) bank169475 +169476 POINT(41.241502023368305 73.27436453508645) bank169476 +169477 POINT(40.47867860765128 73.18860619066709) bank169477 +169478 POINT(40.56652379629034 74.29752122863786) bank169478 +169479 POINT(40.62993853946542 74.71536561341387) bank169479 +169480 POINT(40.00322620122193 73.89986817514932) bank169480 +169481 POINT(40.01763475694162 73.74927977983866) bank169481 +169482 POINT(41.09437584033199 73.85274872198387) bank169482 +169483 POINT(41.38938678728473 73.99755144199672) bank169483 +169484 POINT(41.35952406215722 73.20291476644775) bank169484 +169485 POINT(40.96696011756743 73.20607259418314) bank169485 +169486 POINT(40.2270302219385 74.89366106126936) bank169486 +169487 POINT(39.92037517383449 73.59686018237261) bank169487 +169488 POINT(41.29385131174119 74.35184371188882) bank169488 +169489 POINT(40.45033682704003 73.10526838516418) bank169489 +169490 POINT(40.32326710917784 74.88611699928079) bank169490 +169491 POINT(41.58787904832482 74.50279819875226) bank169491 +169492 POINT(40.901526847595015 74.67787461874866) bank169492 +169493 POINT(41.02584961720429 74.30319525262627) bank169493 +169494 POINT(41.481124266038165 73.92371247012865) bank169494 +169495 POINT(41.54929426653749 74.51899336101863) bank169495 +169496 POINT(41.37798439231065 73.74892166883305) bank169496 +169497 POINT(40.20911736186018 74.00068862907685) bank169497 +169498 POINT(40.68388108449714 73.03028641942652) bank169498 +169499 POINT(39.76307620974992 74.90653550516393) bank169499 +169500 POINT(39.95935136799178 74.63105502070728) bank169500 +169501 POINT(40.04196462376966 73.91768848830442) bank169501 +169502 POINT(41.250676840182756 73.29895462715093) bank169502 +169503 POINT(41.12111092903049 73.75158922296087) bank169503 +169504 POINT(40.74792915573332 74.21049176817512) bank169504 +169505 POINT(41.59770883429264 74.88591596771225) bank169505 +169506 POINT(39.79606790427589 73.7474954388274) bank169506 +169507 POINT(41.60112421855382 73.46864379645123) bank169507 +169508 POINT(41.37299384446194 73.52161394260192) bank169508 +169509 POINT(40.45928805157142 74.75269946743016) bank169509 +169510 POINT(40.131802487251 74.13739052635981) bank169510 +169511 POINT(40.025897037472944 73.97526172376718) bank169511 +169512 POINT(40.65588905314225 73.10582755492872) bank169512 +169513 POINT(40.69804269569729 73.20301184418106) bank169513 +169514 POINT(41.606446154724196 74.44978552859028) bank169514 +169515 POINT(41.27914298500339 74.62272345164064) bank169515 +169516 POINT(40.900049480245194 73.6689149494759) bank169516 +169517 POINT(40.781186168622845 74.38441047566221) bank169517 +169518 POINT(40.10265681021069 73.57260964852951) bank169518 +169519 POINT(39.91292816283108 74.82374127364052) bank169519 +169520 POINT(41.46665826379521 74.17638402027558) bank169520 +169521 POINT(41.15304946225248 74.6477143193984) bank169521 +169522 POINT(40.64943403785033 74.74340013453373) bank169522 +169523 POINT(41.669949570167375 74.11728900343084) bank169523 +169524 POINT(40.356241422447994 74.19241108455566) bank169524 +169525 POINT(39.84678696103729 75.00369597451875) bank169525 +169526 POINT(39.79991195544271 73.36026636181904) bank169526 +169527 POINT(40.66552843567288 74.07502443945256) bank169527 +169528 POINT(41.700528189214076 73.24114450242334) bank169528 +169529 POINT(40.4348417248674 73.52115571622195) bank169529 +169530 POINT(40.187020861806246 74.19443506643239) bank169530 +169531 POINT(40.786026035384815 74.18512888502829) bank169531 +169532 POINT(41.01500181542522 74.07127779564681) bank169532 +169533 POINT(40.740132763611626 74.42894149097587) bank169533 +169534 POINT(40.716081451848325 73.78864347478631) bank169534 +169535 POINT(40.41071642940024 74.058120738712) bank169535 +169536 POINT(40.0165644443186 74.44569019233889) bank169536 +169537 POINT(40.09768278299044 73.09304715764888) bank169537 +169538 POINT(40.210474395039306 73.16283751876618) bank169538 +169539 POINT(41.602081340575864 74.15123760677085) bank169539 +169540 POINT(40.46849633344225 73.01344695046627) bank169540 +169541 POINT(41.570940831938465 73.42888964300258) bank169541 +169542 POINT(39.96914075270733 73.5882911098161) bank169542 +169543 POINT(41.59211857716884 73.93768850459335) bank169543 +169544 POINT(41.681872190002004 74.13573746356822) bank169544 +169545 POINT(41.55957706664743 74.80708718234378) bank169545 +169546 POINT(39.739797920678264 73.59553803689617) bank169546 +169547 POINT(40.51825587389846 74.21253461453449) bank169547 +169548 POINT(41.285220716882485 73.76604067460282) bank169548 +169549 POINT(40.56438716410833 74.55034307114254) bank169549 +169550 POINT(40.52748778574669 74.3235756305722) bank169550 +169551 POINT(40.606954859956296 73.65933937899663) bank169551 +169552 POINT(40.46257699642561 73.63830367847731) bank169552 +169553 POINT(40.704486288955025 74.38677402422864) bank169553 +169554 POINT(41.566540860416914 73.28305785571882) bank169554 +169555 POINT(40.164826725926964 74.16551132263031) bank169555 +169556 POINT(40.662465380171206 74.26322840797587) bank169556 +169557 POINT(41.341888855807085 73.47384578580387) bank169557 +169558 POINT(40.88393402303798 73.65011688674988) bank169558 +169559 POINT(39.98944935981966 74.86686646718756) bank169559 +169560 POINT(41.1061657853867 73.5311922491076) bank169560 +169561 POINT(41.230949833439496 73.94630152802286) bank169561 +169562 POINT(40.33857139009581 73.93135155025529) bank169562 +169563 POINT(41.5685457864091 74.69860977538944) bank169563 +169564 POINT(40.629606965356 73.28429704488472) bank169564 +169565 POINT(40.61147596826379 74.39818361134573) bank169565 +169566 POINT(40.10244203914052 74.40836730694231) bank169566 +169567 POINT(39.88190629987489 74.20049007280194) bank169567 +169568 POINT(41.23256064604834 74.91741870478948) bank169568 +169569 POINT(40.41004893670468 74.5065530136037) bank169569 +169570 POINT(39.73403096357315 73.21064913440512) bank169570 +169571 POINT(40.79489412483789 73.52560640975949) bank169571 +169572 POINT(39.9143143066119 73.49856878827129) bank169572 +169573 POINT(41.63209665708227 73.28131188470498) bank169573 +169574 POINT(41.32240524714394 73.77464002631899) bank169574 +169575 POINT(39.80399613020036 74.98358124326894) bank169575 +169576 POINT(40.21250587553071 73.96329025684777) bank169576 +169577 POINT(41.15170652887831 73.10209149501605) bank169577 +169578 POINT(40.07667156977175 73.28325413047313) bank169578 +169579 POINT(41.58208881186222 73.62231134733969) bank169579 +169580 POINT(41.524164342220125 74.54860807683293) bank169580 +169581 POINT(40.81217069926512 73.73376608869233) bank169581 +169582 POINT(39.85641971443988 73.14197940547172) bank169582 +169583 POINT(40.255336365064494 73.50982012356464) bank169583 +169584 POINT(41.70450105033513 74.218027256838) bank169584 +169585 POINT(40.77105828602438 74.38478649822633) bank169585 +169586 POINT(41.003809145453 74.48589514713696) bank169586 +169587 POINT(39.74564142434494 73.03431339224774) bank169587 +169588 POINT(41.45789828991993 74.44255039544734) bank169588 +169589 POINT(40.15597598725424 74.68954651667002) bank169589 +169590 POINT(40.42294035876845 73.86326851601348) bank169590 +169591 POINT(41.624008687944055 73.33544494747554) bank169591 +169592 POINT(41.586314715782414 74.27347403960859) bank169592 +169593 POINT(40.03819454531413 74.10127957557752) bank169593 +169594 POINT(41.05855328471431 74.92654372655171) bank169594 +169595 POINT(40.64790486373577 74.95286479955016) bank169595 +169596 POINT(41.081495963961345 73.0774776808026) bank169596 +169597 POINT(40.82078068875242 74.7900992857425) bank169597 +169598 POINT(41.34119658640661 74.30370102597774) bank169598 +169599 POINT(41.580727320276964 73.03349188262378) bank169599 +169600 POINT(40.743762257102276 74.37408507698419) bank169600 +169601 POINT(40.03151984300986 73.5204740079863) bank169601 +169602 POINT(41.099768182143876 74.08266089695991) bank169602 +169603 POINT(41.041180904175675 73.93823229377689) bank169603 +169604 POINT(41.25254046046649 74.60222516236261) bank169604 +169605 POINT(41.30273033234168 74.41204014023731) bank169605 +169606 POINT(40.64745059487283 74.94085760698468) bank169606 +169607 POINT(41.61851565329332 74.43268883750441) bank169607 +169608 POINT(41.66876705993951 74.81958478480115) bank169608 +169609 POINT(41.232434898076335 74.4805594347476) bank169609 +169610 POINT(41.51082420058536 74.48040537354282) bank169610 +169611 POINT(40.55277022468132 73.01614218163353) bank169611 +169612 POINT(40.47552096584252 74.68786185211204) bank169612 +169613 POINT(40.820259581372326 73.57320751332088) bank169613 +169614 POINT(39.957444139836106 74.45222322617757) bank169614 +169615 POINT(40.657756772670666 73.80916171090328) bank169615 +169616 POINT(39.86886745116971 74.29907909953917) bank169616 +169617 POINT(41.01112709868793 74.40834309188453) bank169617 +169618 POINT(40.86293425999856 74.08750387057702) bank169618 +169619 POINT(40.90982833835181 74.67034114656187) bank169619 +169620 POINT(39.752470782469175 74.1329981613946) bank169620 +169621 POINT(40.025296876777794 74.01096097519603) bank169621 +169622 POINT(40.96274488357725 74.78025548450209) bank169622 +169623 POINT(40.10146868608352 73.51622906574002) bank169623 +169624 POINT(40.68619269120752 74.50814159438102) bank169624 +169625 POINT(40.593977015944596 74.80139639211113) bank169625 +169626 POINT(39.985764528979786 73.76493996982059) bank169626 +169627 POINT(41.40037187531319 73.09449888719153) bank169627 +169628 POINT(40.4190361923307 74.18828502271427) bank169628 +169629 POINT(40.37149711173986 73.03740444373517) bank169629 +169630 POINT(39.953259731634084 73.42386976455936) bank169630 +169631 POINT(41.595530789413836 73.12239151394728) bank169631 +169632 POINT(41.28164202535647 74.08870698247522) bank169632 +169633 POINT(40.16458330451665 74.28392946387235) bank169633 +169634 POINT(40.784094281227006 73.7186181044666) bank169634 +169635 POINT(40.058863376894614 74.9792137893055) bank169635 +169636 POINT(40.18515169007312 73.38216796108192) bank169636 +169637 POINT(41.11809086924261 73.83541284301918) bank169637 +169638 POINT(40.81034306956952 74.13838468103117) bank169638 +169639 POINT(41.17644889361895 74.25178073829736) bank169639 +169640 POINT(39.7527909448692 74.6744065685036) bank169640 +169641 POINT(40.190577299844186 74.1288357119922) bank169641 +169642 POINT(41.70201427852441 73.87003725760688) bank169642 +169643 POINT(40.75652477386582 74.04657489670171) bank169643 +169644 POINT(41.30392837900793 74.83039395028055) bank169644 +169645 POINT(41.083012726901735 73.04908602451984) bank169645 +169646 POINT(40.959277648609635 73.60430271507119) bank169646 +169647 POINT(41.284558097841504 74.06547918158793) bank169647 +169648 POINT(39.94615894999152 73.94747076253893) bank169648 +169649 POINT(40.48722503124026 74.66172032238943) bank169649 +169650 POINT(40.91610091488505 74.1175424700498) bank169650 +169651 POINT(40.57065298404205 73.3699798941331) bank169651 +169652 POINT(41.03881625107929 74.24491521444556) bank169652 +169653 POINT(40.0461075027314 73.91808782138874) bank169653 +169654 POINT(40.802590311061046 73.06776629689844) bank169654 +169655 POINT(41.61238744137675 73.82587966006209) bank169655 +169656 POINT(40.8519603538528 73.50286666363881) bank169656 +169657 POINT(41.33329679337853 73.07287928713329) bank169657 +169658 POINT(39.87371039548101 74.4338895835945) bank169658 +169659 POINT(41.29987306357489 73.15391116535265) bank169659 +169660 POINT(40.50840962526046 73.4943789127266) bank169660 +169661 POINT(39.97531603311479 73.11374379427103) bank169661 +169662 POINT(39.980774823787435 74.71693042578818) bank169662 +169663 POINT(40.331594661301736 73.14175025397425) bank169663 +169664 POINT(39.729280350408054 73.42810349091269) bank169664 +169665 POINT(40.330070085941195 73.27418469420058) bank169665 +169666 POINT(40.54086640898247 73.96797460945913) bank169666 +169667 POINT(41.32997156157701 73.81477899884982) bank169667 +169668 POINT(41.18503963608587 74.06595366604469) bank169668 +169669 POINT(40.43773292718324 74.14092222821512) bank169669 +169670 POINT(40.74011639096906 73.74595117197612) bank169670 +169671 POINT(40.462487957749616 73.96701826882499) bank169671 +169672 POINT(40.33763517729693 74.46729761692154) bank169672 +169673 POINT(41.62657871984201 73.0632704567029) bank169673 +169674 POINT(40.22872664198316 74.68052688010091) bank169674 +169675 POINT(40.83373438033195 74.1905110507307) bank169675 +169676 POINT(41.17221045853391 73.89303123508549) bank169676 +169677 POINT(40.70446432280304 73.69542302036109) bank169677 +169678 POINT(39.72139151609087 73.67415016733676) bank169678 +169679 POINT(40.819023468793205 73.36660997162991) bank169679 +169680 POINT(40.759691010566435 73.39204241382744) bank169680 +169681 POINT(40.91057248081073 73.7167339361229) bank169681 +169682 POINT(39.98265921677235 73.57151612072857) bank169682 +169683 POINT(39.97068339921956 74.25584951347955) bank169683 +169684 POINT(41.014525840309844 73.6710908285675) bank169684 +169685 POINT(41.28826628237512 73.38731732268904) bank169685 +169686 POINT(41.465831146424705 73.37509836912669) bank169686 +169687 POINT(40.002378354363174 74.98383410609064) bank169687 +169688 POINT(40.0651567219206 74.64043816042975) bank169688 +169689 POINT(40.94946483581655 73.20169038222515) bank169689 +169690 POINT(40.30368645387579 74.46225891917825) bank169690 +169691 POINT(40.48591510281137 74.59459358490857) bank169691 +169692 POINT(40.77263045345972 73.31766105643501) bank169692 +169693 POINT(39.956436590200724 74.1029576636936) bank169693 +169694 POINT(40.248687098302106 74.93743125509066) bank169694 +169695 POINT(39.996616309781075 74.53494321908674) bank169695 +169696 POINT(41.061874796176184 73.2972466245333) bank169696 +169697 POINT(40.64288755707524 73.8737986549417) bank169697 +169698 POINT(41.6874450815073 74.54891819516365) bank169698 +169699 POINT(39.82591542532845 73.00728789215938) bank169699 +169700 POINT(41.0814088042773 75.00540229342265) bank169700 +169701 POINT(40.54182939875087 73.73580381265838) bank169701 +169702 POINT(40.840612241795824 73.97721255201905) bank169702 +169703 POINT(40.60420510885355 73.73804842376197) bank169703 +169704 POINT(40.791285689707124 73.55247201004505) bank169704 +169705 POINT(41.03408022099909 73.1811427908401) bank169705 +169706 POINT(40.46173611608215 74.24205492673349) bank169706 +169707 POINT(39.82821128131803 73.61965720943394) bank169707 +169708 POINT(40.48957472973574 73.20162675451358) bank169708 +169709 POINT(40.30469236864487 73.17475816676846) bank169709 +169710 POINT(41.21206265941067 74.70463734258033) bank169710 +169711 POINT(40.21432683860767 74.30863781316006) bank169711 +169712 POINT(40.861061055039194 73.09375749160068) bank169712 +169713 POINT(40.80471695752987 74.81846797407019) bank169713 +169714 POINT(40.213690951154724 73.62510126881222) bank169714 +169715 POINT(40.26136172604475 74.39070792540699) bank169715 +169716 POINT(39.72485278123756 73.78730166227328) bank169716 +169717 POINT(41.59916554629923 74.40719210298293) bank169717 +169718 POINT(40.6320592636508 73.3581223209358) bank169718 +169719 POINT(40.530829454024506 74.49250148631629) bank169719 +169720 POINT(41.392378337230895 73.47172504197664) bank169720 +169721 POINT(41.09446894554505 73.90529722361863) bank169721 +169722 POINT(41.157737829683334 74.99957361961737) bank169722 +169723 POINT(40.99278274285042 74.89938196024995) bank169723 +169724 POINT(41.1347194332266 73.99178128956234) bank169724 +169725 POINT(41.33222791861591 74.35085202318147) bank169725 +169726 POINT(40.262851603248144 74.16685049964357) bank169726 +169727 POINT(39.879346918118586 74.59147351245721) bank169727 +169728 POINT(39.72715301397559 73.91427196422113) bank169728 +169729 POINT(40.16742196331215 73.64154809241862) bank169729 +169730 POINT(40.922364400807844 74.46430183301591) bank169730 +169731 POINT(40.29482428470658 73.42366205095416) bank169731 +169732 POINT(40.089838587598 73.24182342576378) bank169732 +169733 POINT(40.17667728237789 73.45818543640871) bank169733 +169734 POINT(40.199597686894464 74.13267178142513) bank169734 +169735 POINT(40.523545324447866 73.97243777044619) bank169735 +169736 POINT(40.36351623001586 73.27916226236476) bank169736 +169737 POINT(40.619894778911004 73.70545188564311) bank169737 +169738 POINT(40.81109169365161 74.13167779737292) bank169738 +169739 POINT(41.67151263736281 73.05152366057885) bank169739 +169740 POINT(41.256252609127515 74.05744108804174) bank169740 +169741 POINT(40.78393911182884 73.3985001157579) bank169741 +169742 POINT(39.807961655829295 73.84794340128155) bank169742 +169743 POINT(41.69848663542395 74.67688562102109) bank169743 +169744 POINT(41.598818326421146 73.7881239328981) bank169744 +169745 POINT(40.561520461291245 73.33671017693071) bank169745 +169746 POINT(41.633212221037596 74.97225406379086) bank169746 +169747 POINT(40.13700096347538 73.84538273498242) bank169747 +169748 POINT(41.20686843028336 74.5271088911623) bank169748 +169749 POINT(41.64628344927027 74.11193043116428) bank169749 +169750 POINT(40.31589650858755 73.13438008998814) bank169750 +169751 POINT(40.07855673280276 73.46094039539022) bank169751 +169752 POINT(41.252841736863395 74.6922847838039) bank169752 +169753 POINT(40.7811365471341 74.29184520166726) bank169753 +169754 POINT(41.66273851919903 74.15889175217538) bank169754 +169755 POINT(40.16156991752839 74.42838043927206) bank169755 +169756 POINT(40.807162047665685 74.50357625903312) bank169756 +169757 POINT(40.63506627560728 74.70522429978615) bank169757 +169758 POINT(40.729787488174665 73.52453507885745) bank169758 +169759 POINT(41.040164625848746 73.48152245275249) bank169759 +169760 POINT(40.41826947987744 74.68738046707293) bank169760 +169761 POINT(41.701484802264744 74.7557828299427) bank169761 +169762 POINT(41.246992152352895 74.65735113242361) bank169762 +169763 POINT(40.51002836839007 74.4291268961675) bank169763 +169764 POINT(41.32335790927527 74.98907302724318) bank169764 +169765 POINT(40.029742168766155 73.31812880381426) bank169765 +169766 POINT(39.803696098345014 74.260499968067) bank169766 +169767 POINT(41.12502763359206 73.27474001781405) bank169767 +169768 POINT(41.5525430346705 73.12959819376998) bank169768 +169769 POINT(41.26362832002766 73.26744190801713) bank169769 +169770 POINT(40.80465091621125 73.82284567737642) bank169770 +169771 POINT(41.127202197715434 74.23426771537493) bank169771 +169772 POINT(41.573700892961526 74.04981703260927) bank169772 +169773 POINT(39.7191800836941 73.6547726978217) bank169773 +169774 POINT(41.1517892882105 73.74436725796969) bank169774 +169775 POINT(41.40597947709891 74.4237931441797) bank169775 +169776 POINT(40.913640172072796 74.57455412636571) bank169776 +169777 POINT(39.910072491139054 74.28576573051897) bank169777 +169778 POINT(40.81628615413386 74.30267337134148) bank169778 +169779 POINT(41.184234783139715 73.12333814662959) bank169779 +169780 POINT(40.483957195418355 73.81075437032884) bank169780 +169781 POINT(40.354753448726896 73.77394239382089) bank169781 +169782 POINT(40.01082188956482 74.37072525552492) bank169782 +169783 POINT(40.00682960537853 73.66944967769957) bank169783 +169784 POINT(40.038657757219575 73.13756321923277) bank169784 +169785 POINT(40.74421257684663 74.42692897380367) bank169785 +169786 POINT(41.47589225072255 73.93600906693183) bank169786 +169787 POINT(40.24489462217159 73.5178028741465) bank169787 +169788 POINT(41.67037487299007 74.08434822073022) bank169788 +169789 POINT(41.14111082242395 74.61201049362026) bank169789 +169790 POINT(40.4386173965226 73.69188989158434) bank169790 +169791 POINT(40.22299291641425 74.87509810213284) bank169791 +169792 POINT(40.055624611568774 73.0234755783196) bank169792 +169793 POINT(41.44648790642797 74.88451998156658) bank169793 +169794 POINT(40.23672663860606 74.9029284280798) bank169794 +169795 POINT(41.49871788230562 73.75263841006948) bank169795 +169796 POINT(40.79120080106534 73.04946525546168) bank169796 +169797 POINT(41.31012594899863 73.66304639873842) bank169797 +169798 POINT(40.76344892970263 74.85020092803956) bank169798 +169799 POINT(41.49511597566753 73.02983627671453) bank169799 +169800 POINT(40.762477660649 74.30645003667841) bank169800 +169801 POINT(41.33133361309106 73.9815603281472) bank169801 +169802 POINT(40.975662708194115 73.98802679048262) bank169802 +169803 POINT(39.89676696091424 74.86549155054986) bank169803 +169804 POINT(40.94079150953977 73.23721523523095) bank169804 +169805 POINT(40.25333233239262 73.10792755573486) bank169805 +169806 POINT(40.58042991360393 73.3425687169194) bank169806 +169807 POINT(41.150336481842515 74.72215279789316) bank169807 +169808 POINT(41.28215592539329 73.21370421743971) bank169808 +169809 POINT(41.06789091197301 74.03550086732906) bank169809 +169810 POINT(40.3753880781597 73.35126119646368) bank169810 +169811 POINT(41.28933073989024 73.12537440455404) bank169811 +169812 POINT(41.26715159008338 74.24356111630102) bank169812 +169813 POINT(41.101436741676125 73.90536630556913) bank169813 +169814 POINT(41.41849873722745 74.62530496601795) bank169814 +169815 POINT(40.27191762311193 73.48490741830484) bank169815 +169816 POINT(40.786026134302425 73.0286038017181) bank169816 +169817 POINT(41.08846093151501 73.80337450645237) bank169817 +169818 POINT(39.71834750844531 74.16773147887818) bank169818 +169819 POINT(40.06175981650353 74.51600406337921) bank169819 +169820 POINT(41.05618035687598 74.77699902557139) bank169820 +169821 POINT(40.06944346080111 73.95729848168527) bank169821 +169822 POINT(40.966474132943695 73.68695417042393) bank169822 +169823 POINT(41.4700123581798 73.85123722380683) bank169823 +169824 POINT(41.10689550774181 73.86439452288757) bank169824 +169825 POINT(41.15521098441177 74.641821375361) bank169825 +169826 POINT(40.50471106550478 74.60201189686498) bank169826 +169827 POINT(40.13856992186937 74.35282157872606) bank169827 +169828 POINT(40.14955914954274 74.2567644177605) bank169828 +169829 POINT(40.87618858664028 74.05559720203118) bank169829 +169830 POINT(40.81204889380783 74.8115677030938) bank169830 +169831 POINT(41.224694218802895 74.85789623499261) bank169831 +169832 POINT(40.10664602967729 74.19129753030671) bank169832 +169833 POINT(40.33935544370148 74.95218351664444) bank169833 +169834 POINT(41.33916638457858 74.16423912214717) bank169834 +169835 POINT(39.99367628732591 74.33189394510984) bank169835 +169836 POINT(39.931023580496486 74.31028288545669) bank169836 +169837 POINT(39.81896532434499 74.27864066783745) bank169837 +169838 POINT(39.93315202182206 74.79418363294874) bank169838 +169839 POINT(41.618848639723836 74.61650675534452) bank169839 +169840 POINT(39.910501623436474 73.70623438023652) bank169840 +169841 POINT(39.755944239760275 73.51589649240695) bank169841 +169842 POINT(40.76967181764108 73.22347787027577) bank169842 +169843 POINT(40.266452681029605 74.28012875071107) bank169843 +169844 POINT(40.37556902130908 74.76489738199402) bank169844 +169845 POINT(40.593119168046286 73.99597923173718) bank169845 +169846 POINT(40.86622353937215 74.07946309371141) bank169846 +169847 POINT(40.16890170160222 74.58800630244797) bank169847 +169848 POINT(41.324035046936025 73.76570874685754) bank169848 +169849 POINT(40.053351931348466 73.60802914646354) bank169849 +169850 POINT(39.80284755686014 73.64467969223615) bank169850 +169851 POINT(40.6439724075494 74.60113796029114) bank169851 +169852 POINT(41.3863073221346 74.8384177980115) bank169852 +169853 POINT(40.95752398139434 73.26831579799189) bank169853 +169854 POINT(39.74562954790035 74.97484619442963) bank169854 +169855 POINT(40.777114187561565 74.29217967871925) bank169855 +169856 POINT(39.834375146949746 73.1439550618083) bank169856 +169857 POINT(41.69798433834299 74.1219022855565) bank169857 +169858 POINT(40.092379118067015 74.1554813518642) bank169858 +169859 POINT(40.21139653776004 74.51462894483467) bank169859 +169860 POINT(41.45174152134211 73.4954361156738) bank169860 +169861 POINT(40.144883111886266 73.48159961884112) bank169861 +169862 POINT(40.78084636868261 74.34443390125975) bank169862 +169863 POINT(41.53429538510012 74.74014069387589) bank169863 +169864 POINT(41.226796685877396 74.23661598751241) bank169864 +169865 POINT(41.50053759811576 73.3541273492791) bank169865 +169866 POINT(41.291864924720606 73.19312452860342) bank169866 +169867 POINT(41.60115547388701 73.87453388700554) bank169867 +169868 POINT(41.06973574302926 74.08871057957735) bank169868 +169869 POINT(41.25398702231853 73.17014152233295) bank169869 +169870 POINT(39.80831677267158 74.61663878671604) bank169870 +169871 POINT(41.4068258467928 74.10179549247758) bank169871 +169872 POINT(40.71019844028822 74.399072565661) bank169872 +169873 POINT(40.83438162324091 74.46230385764265) bank169873 +169874 POINT(39.76557683526557 73.03056675896053) bank169874 +169875 POINT(41.710501855655814 74.8511700470566) bank169875 +169876 POINT(40.85858005654799 74.1935863663579) bank169876 +169877 POINT(41.40173543540604 74.60556204493393) bank169877 +169878 POINT(40.47349773185131 74.67793394533915) bank169878 +169879 POINT(40.09616966685889 74.68134301378424) bank169879 +169880 POINT(40.653069511395415 74.26186370870815) bank169880 +169881 POINT(40.96812927914303 74.02112969756112) bank169881 +169882 POINT(40.72851040958243 73.68494472793166) bank169882 +169883 POINT(39.899084761339225 74.52441925816257) bank169883 +169884 POINT(39.974584160502346 74.10958455649694) bank169884 +169885 POINT(41.589385567101374 74.97257261836576) bank169885 +169886 POINT(40.292545620292024 73.82845080220574) bank169886 +169887 POINT(39.95528324346695 73.45544385793748) bank169887 +169888 POINT(39.84212619069213 74.75153182158192) bank169888 +169889 POINT(41.2671610466585 73.97411218459499) bank169889 +169890 POINT(40.187403639428005 74.32199062523654) bank169890 +169891 POINT(41.09387577659518 74.10000582132206) bank169891 +169892 POINT(41.35018728728229 73.04843712328858) bank169892 +169893 POINT(40.86219966780028 73.68442889912704) bank169893 +169894 POINT(41.513791523008734 73.79632080296295) bank169894 +169895 POINT(41.458863250083915 73.38480944423905) bank169895 +169896 POINT(40.86120503653937 74.63111087812983) bank169896 +169897 POINT(41.433818310399964 74.75988169378492) bank169897 +169898 POINT(39.7423355624217 73.34662007052974) bank169898 +169899 POINT(39.94405626609748 74.59776831520533) bank169899 +169900 POINT(40.84261689196773 73.63329039773463) bank169900 +169901 POINT(40.95680287082584 73.81388766704801) bank169901 +169902 POINT(40.71701573513825 74.116031003892) bank169902 +169903 POINT(41.24279991582478 74.13816254373761) bank169903 +169904 POINT(41.55684732645904 73.79770207078745) bank169904 +169905 POINT(41.07307884211893 73.22064037222123) bank169905 +169906 POINT(41.0889093619287 74.71840999188696) bank169906 +169907 POINT(41.42925868067824 74.37579573178859) bank169907 +169908 POINT(40.370078857378964 74.63474224191815) bank169908 +169909 POINT(40.97020027433472 74.2514167434634) bank169909 +169910 POINT(40.991131794189776 74.40998383215832) bank169910 +169911 POINT(40.802080908745815 74.49729546634094) bank169911 +169912 POINT(41.0406741021865 73.99536856705744) bank169912 +169913 POINT(40.94908117348534 73.2108932763868) bank169913 +169914 POINT(40.74658872881236 74.01110022053271) bank169914 +169915 POINT(41.15462695880153 74.6890257266323) bank169915 +169916 POINT(39.95176924704929 73.4399913307754) bank169916 +169917 POINT(40.77874154376198 73.90749036253662) bank169917 +169918 POINT(41.28816580250337 74.17693819764847) bank169918 +169919 POINT(40.80050478874032 74.51405278847128) bank169919 +169920 POINT(41.513184464837266 73.75232468592924) bank169920 +169921 POINT(41.270856406076355 73.55704624726056) bank169921 +169922 POINT(40.503719309484005 74.97967216958273) bank169922 +169923 POINT(41.68321081004198 73.78599207994745) bank169923 +169924 POINT(41.32462136307588 73.948390159954) bank169924 +169925 POINT(40.494895110779765 74.8734225321686) bank169925 +169926 POINT(41.19797712229213 73.18997320357543) bank169926 +169927 POINT(41.40574909306608 73.29454103287274) bank169927 +169928 POINT(41.57496828965055 73.65587213642456) bank169928 +169929 POINT(41.26890998137535 73.44923608556599) bank169929 +169930 POINT(41.09886715736146 73.04776631524194) bank169930 +169931 POINT(40.1129100585844 73.13645292821596) bank169931 +169932 POINT(41.20797333544987 73.66524888150342) bank169932 +169933 POINT(40.16469725188566 73.9428070790954) bank169933 +169934 POINT(40.26588837612198 73.11497085939327) bank169934 +169935 POINT(41.66423244444209 73.76507933094578) bank169935 +169936 POINT(40.9021657562575 74.32661314564783) bank169936 +169937 POINT(41.004781294520306 74.9492394335031) bank169937 +169938 POINT(40.68872581276619 74.73274948634617) bank169938 +169939 POINT(41.166426490803865 73.67396769004324) bank169939 +169940 POINT(39.9374145744818 73.49343369284526) bank169940 +169941 POINT(41.44842514218507 73.58657916872033) bank169941 +169942 POINT(40.70952812333799 73.01923728049493) bank169942 +169943 POINT(40.193756801713974 74.79605498942766) bank169943 +169944 POINT(40.20435071551335 73.51464018789919) bank169944 +169945 POINT(40.33373358854322 73.82882607137306) bank169945 +169946 POINT(40.53810966935986 73.60232896172278) bank169946 +169947 POINT(40.32023645986831 74.34600669847104) bank169947 +169948 POINT(40.69948772457873 73.87873852432506) bank169948 +169949 POINT(41.08034409617728 74.1442054472178) bank169949 +169950 POINT(40.16531346375372 73.78286224856993) bank169950 +169951 POINT(40.76397299787807 73.16637568146885) bank169951 +169952 POINT(40.97845454395549 74.81213766818766) bank169952 +169953 POINT(40.83206380620409 73.12100291552467) bank169953 +169954 POINT(39.94866039576618 73.12121098366484) bank169954 +169955 POINT(41.298336016775544 73.60435635365681) bank169955 +169956 POINT(40.73426927730327 73.43109075614983) bank169956 +169957 POINT(41.07740215292715 73.12237844008378) bank169957 +169958 POINT(41.27176343597004 74.91468034452848) bank169958 +169959 POINT(40.02579273132542 73.30802106277275) bank169959 +169960 POINT(40.810331655125424 73.08978879457531) bank169960 +169961 POINT(41.36446735818818 74.65278201982673) bank169961 +169962 POINT(39.779625431099596 74.4845190622906) bank169962 +169963 POINT(41.59695583042631 73.34974505808793) bank169963 +169964 POINT(41.64270948409105 74.68685411168549) bank169964 +169965 POINT(40.739319455297675 73.14158545702023) bank169965 +169966 POINT(40.54527239949872 73.52029023514709) bank169966 +169967 POINT(41.42338775550674 74.48886195985695) bank169967 +169968 POINT(39.92075799950341 74.421264335262) bank169968 +169969 POINT(41.685331647833024 74.23688217997872) bank169969 +169970 POINT(40.747533963363374 73.74006884863125) bank169970 +169971 POINT(41.48359128305788 73.20118137580573) bank169971 +169972 POINT(41.207982872813545 74.99058745777144) bank169972 +169973 POINT(41.00569121390054 74.21599326414182) bank169973 +169974 POINT(40.69637159757443 73.11114276965179) bank169974 +169975 POINT(41.07121712289575 73.93601585621748) bank169975 +169976 POINT(39.815391276201794 74.9924482583054) bank169976 +169977 POINT(41.44080132695862 73.9344292936151) bank169977 +169978 POINT(41.26387111297242 74.39237003544085) bank169978 +169979 POINT(41.68046908702258 73.0939556269046) bank169979 +169980 POINT(40.278275893894524 74.7444815821109) bank169980 +169981 POINT(41.22711180072192 74.39176771583053) bank169981 +169982 POINT(41.18818608768973 73.25398695255872) bank169982 +169983 POINT(41.12567611761742 74.23215738677845) bank169983 +169984 POINT(39.801574673052166 73.14156810248544) bank169984 +169985 POINT(39.92862249016146 73.56856576429566) bank169985 +169986 POINT(40.7634215223046 73.971041602263) bank169986 +169987 POINT(40.90846164138405 73.44008312015401) bank169987 +169988 POINT(41.71208171650273 73.71408090952715) bank169988 +169989 POINT(40.35923758559576 73.61490653419403) bank169989 +169990 POINT(40.951225675003734 73.70080389628285) bank169990 +169991 POINT(40.24075681998429 74.11172077806616) bank169991 +169992 POINT(40.15142809518075 74.91601484624923) bank169992 +169993 POINT(40.390269643214786 73.32445016785584) bank169993 +169994 POINT(41.711528950166816 74.26368179081219) bank169994 +169995 POINT(41.68736633295713 74.3264852070564) bank169995 +169996 POINT(41.661980397095434 74.39646971338296) bank169996 +169997 POINT(40.630020025030795 74.79794811967908) bank169997 +169998 POINT(40.62230789746225 73.36598956249564) bank169998 +169999 POINT(40.077385293035945 74.3981809283161) bank169999 +170000 POINT(40.11995172514312 74.54113588176635) bank170000 +170001 POINT(41.445360758026254 74.1588080225521) bank170001 +170002 POINT(41.33055596717266 73.34256278665916) bank170002 +170003 POINT(39.94238411636776 74.29921583637243) bank170003 +170004 POINT(41.344407808122384 73.3651378231562) bank170004 +170005 POINT(41.623589409251025 73.76334434595177) bank170005 +170006 POINT(40.212159931927296 74.80720952058863) bank170006 +170007 POINT(40.64697494643252 74.59131697954938) bank170007 +170008 POINT(41.520048556822665 74.47989468590896) bank170008 +170009 POINT(40.90999386795266 73.49020730574743) bank170009 +170010 POINT(40.52382268330899 74.39639378060693) bank170010 +170011 POINT(40.23281990493134 73.42840104689822) bank170011 +170012 POINT(41.30076825159727 74.08920075863726) bank170012 +170013 POINT(39.74722992441132 74.19144932470186) bank170013 +170014 POINT(41.382412767778405 73.73758147316298) bank170014 +170015 POINT(41.38700534960431 74.53344426447285) bank170015 +170016 POINT(40.84666722784223 73.81689871591259) bank170016 +170017 POINT(41.66849259088279 73.40733875826903) bank170017 +170018 POINT(40.15327489373345 73.03757882458414) bank170018 +170019 POINT(39.943963829868494 74.49953121741122) bank170019 +170020 POINT(41.26205087157114 74.65562486581636) bank170020 +170021 POINT(40.263763327497045 73.69480527091403) bank170021 +170022 POINT(41.112208589188 74.81857325638029) bank170022 +170023 POINT(40.58780252562518 73.68001389131466) bank170023 +170024 POINT(41.1670033915271 74.06453108616125) bank170024 +170025 POINT(41.66782962461085 73.97566420256636) bank170025 +170026 POINT(40.00953136712381 74.54814252534966) bank170026 +170027 POINT(40.03521658871566 73.9615510230028) bank170027 +170028 POINT(40.529581754815766 73.9996885500944) bank170028 +170029 POINT(41.52841061939554 74.28913919352952) bank170029 +170030 POINT(40.08613085565263 74.49645733381377) bank170030 +170031 POINT(40.165298616173956 73.26574798465025) bank170031 +170032 POINT(41.24169051905356 73.02218744093263) bank170032 +170033 POINT(41.33524626946047 73.9115754941064) bank170033 +170034 POINT(41.25593569580577 74.65435013468883) bank170034 +170035 POINT(40.10917713877794 73.05318330429776) bank170035 +170036 POINT(40.27688434726084 73.94399724799752) bank170036 +170037 POINT(39.90785557833209 74.80106570802765) bank170037 +170038 POINT(40.56169430042401 73.82386056843576) bank170038 +170039 POINT(40.51325191891242 73.06385334181061) bank170039 +170040 POINT(41.23519385632065 74.08033320499308) bank170040 +170041 POINT(40.11710766083567 74.31209505965107) bank170041 +170042 POINT(40.62750920608245 73.97459817377697) bank170042 +170043 POINT(41.372666332748324 74.65337705414119) bank170043 +170044 POINT(41.245849791728126 74.84474922509361) bank170044 +170045 POINT(40.933406574543056 74.1516245738644) bank170045 +170046 POINT(41.42046043755071 74.94658359339698) bank170046 +170047 POINT(40.08202621600768 74.19292792126735) bank170047 +170048 POINT(40.363104153374124 73.38071143142685) bank170048 +170049 POINT(41.334481821669804 74.86536775385089) bank170049 +170050 POINT(39.78152502602722 74.04810637225202) bank170050 +170051 POINT(41.65614221255061 74.66883249755935) bank170051 +170052 POINT(40.388572324372994 73.5047516886432) bank170052 +170053 POINT(41.12428452324173 74.82471989536884) bank170053 +170054 POINT(40.29646474202012 73.49205932711698) bank170054 +170055 POINT(40.225686120814174 74.87259669319303) bank170055 +170056 POINT(40.0939012680874 73.48058971524193) bank170056 +170057 POINT(39.903392618013044 73.22945397083024) bank170057 +170058 POINT(41.084626099800914 73.58821700953398) bank170058 +170059 POINT(40.61884900289084 74.68387163035302) bank170059 +170060 POINT(41.31683710201429 73.59758885831074) bank170060 +170061 POINT(41.67898896368017 73.54983458491581) bank170061 +170062 POINT(41.64945158287952 73.51031124849777) bank170062 +170063 POINT(41.01698879722681 73.03595765801633) bank170063 +170064 POINT(40.38141443880405 73.5412604940806) bank170064 +170065 POINT(41.425773265250854 73.56067666250486) bank170065 +170066 POINT(41.418501962558956 73.30594775416657) bank170066 +170067 POINT(40.833945025744406 73.63721461832283) bank170067 +170068 POINT(40.976087986358614 73.70943945400371) bank170068 +170069 POINT(40.77068507464877 74.59796296334953) bank170069 +170070 POINT(40.188243603388834 73.96392079805882) bank170070 +170071 POINT(41.08737805374193 73.14861045669964) bank170071 +170072 POINT(41.47948436446654 73.08635079912621) bank170072 +170073 POINT(41.433995708567586 74.43795033997108) bank170073 +170074 POINT(41.14164806414661 73.73401992158774) bank170074 +170075 POINT(40.726457975812224 73.03908361154753) bank170075 +170076 POINT(41.398684799337175 74.01362597353975) bank170076 +170077 POINT(40.121747785522714 73.87887758975555) bank170077 +170078 POINT(40.3568765879765 74.18818339585066) bank170078 +170079 POINT(40.59582645066012 74.90750291076452) bank170079 +170080 POINT(41.05777516727139 74.01284319143186) bank170080 +170081 POINT(40.64371218213063 73.58386370235455) bank170081 +170082 POINT(40.45631094238086 74.6948040499246) bank170082 +170083 POINT(41.43386703663675 74.7145417874043) bank170083 +170084 POINT(41.56704612272273 73.2736526879331) bank170084 +170085 POINT(41.53684045025364 73.61200163650602) bank170085 +170086 POINT(40.919336528039885 73.65690458555024) bank170086 +170087 POINT(39.965752535618286 73.44580398687054) bank170087 +170088 POINT(39.9246789909744 74.46230792081863) bank170088 +170089 POINT(40.71429412186992 74.66492260909583) bank170089 +170090 POINT(40.58975370051788 74.24610610819084) bank170090 +170091 POINT(39.748929209517414 74.17268406717722) bank170091 +170092 POINT(41.16485980887718 73.3485833502166) bank170092 +170093 POINT(40.66871019895776 74.25578700986142) bank170093 +170094 POINT(41.38792554989298 73.40659573320887) bank170094 +170095 POINT(40.28098661439855 73.45144962973802) bank170095 +170096 POINT(40.20780379998901 73.45058418585194) bank170096 +170097 POINT(41.27104763603466 73.00962170010177) bank170097 +170098 POINT(41.3248314863538 74.94491493084934) bank170098 +170099 POINT(41.50378170844184 73.60037580666726) bank170099 +170100 POINT(40.01240629499955 73.28749039416347) bank170100 +170101 POINT(41.63746322338159 74.65043302875009) bank170101 +170102 POINT(40.45638156658685 73.64521319428113) bank170102 +170103 POINT(40.40026565943745 73.76516856591556) bank170103 +170104 POINT(40.45811262625478 73.98870610968623) bank170104 +170105 POINT(40.16449468247389 73.85568417986705) bank170105 +170106 POINT(40.42056534421593 73.18889906673319) bank170106 +170107 POINT(41.3875963980025 74.32408046484811) bank170107 +170108 POINT(40.045132561735244 73.006417391156) bank170108 +170109 POINT(40.69256027633801 73.71170905190122) bank170109 +170110 POINT(40.93421361492851 73.89854936502125) bank170110 +170111 POINT(41.43678754544797 73.58371014112024) bank170111 +170112 POINT(41.52650006417936 74.47030492915125) bank170112 +170113 POINT(39.898652052876706 73.14762030562026) bank170113 +170114 POINT(40.00221156996707 74.45164363948379) bank170114 +170115 POINT(41.2632852655756 73.45484442592476) bank170115 +170116 POINT(40.57174479466191 73.55446523029168) bank170116 +170117 POINT(41.19025377746337 74.72527000607371) bank170117 +170118 POINT(41.0597133980413 73.2710102845973) bank170118 +170119 POINT(40.838659936702754 73.47786663343801) bank170119 +170120 POINT(40.86950044114563 73.82884437306222) bank170120 +170121 POINT(40.197294866766846 73.15440323411399) bank170121 +170122 POINT(40.01398158004693 73.73716773706921) bank170122 +170123 POINT(40.050590219414865 74.99709167562789) bank170123 +170124 POINT(40.14123691971163 73.98713103738007) bank170124 +170125 POINT(40.099907162778145 74.21022715326912) bank170125 +170126 POINT(41.46203025093367 73.65666255274392) bank170126 +170127 POINT(39.93207166237533 73.57287037256262) bank170127 +170128 POINT(40.73087357047919 74.36446334801335) bank170128 +170129 POINT(41.169314457901464 74.09272248567413) bank170129 +170130 POINT(39.81204821123074 73.12820449008086) bank170130 +170131 POINT(40.19846161842085 74.25200770900955) bank170131 +170132 POINT(40.58046224798295 73.14932883569166) bank170132 +170133 POINT(39.943161061025926 73.15805234996758) bank170133 +170134 POINT(40.040871173542634 73.75420428339815) bank170134 +170135 POINT(40.01976858503767 73.79806129225449) bank170135 +170136 POINT(40.94596334109732 74.76754609063666) bank170136 +170137 POINT(40.96795132610613 73.4329456896135) bank170137 +170138 POINT(41.571319719249516 74.23677420476972) bank170138 +170139 POINT(39.95807161315583 73.68118407525726) bank170139 +170140 POINT(41.09642662979803 74.78325666472749) bank170140 +170141 POINT(40.44344928526151 74.68930125372559) bank170141 +170142 POINT(41.57334671528244 74.75407221684674) bank170142 +170143 POINT(41.1638040928546 74.62978965011432) bank170143 +170144 POINT(39.958949871741396 74.00879644939667) bank170144 +170145 POINT(39.95965695644472 73.81035859711962) bank170145 +170146 POINT(40.48810530552132 74.12020876412217) bank170146 +170147 POINT(40.78630591444414 73.20186795415098) bank170147 +170148 POINT(40.88309842740219 74.68813463356247) bank170148 +170149 POINT(40.1430213680034 74.62217314469014) bank170149 +170150 POINT(40.57458903750562 73.75549080708946) bank170150 +170151 POINT(39.780749428254296 74.58315280153113) bank170151 +170152 POINT(40.86116366840416 74.57828023517973) bank170152 +170153 POINT(40.261734108851215 74.828720356646) bank170153 +170154 POINT(41.32741867404058 73.89001906206933) bank170154 +170155 POINT(41.15527355465947 73.55954679888578) bank170155 +170156 POINT(40.95012100201159 73.53150797725141) bank170156 +170157 POINT(41.55625172530506 73.46826806942684) bank170157 +170158 POINT(40.10256428912489 73.5585770491231) bank170158 +170159 POINT(41.33059034200096 74.43586172338159) bank170159 +170160 POINT(41.0789051699619 73.9408389794625) bank170160 +170161 POINT(40.73474592069375 73.54775903090163) bank170161 +170162 POINT(39.7509625489417 73.69755375352314) bank170162 +170163 POINT(40.17355403886675 73.1434123345664) bank170163 +170164 POINT(40.46637604359742 74.12705356271466) bank170164 +170165 POINT(41.42652499269275 73.59303123114631) bank170165 +170166 POINT(41.4053885823092 74.70698338567027) bank170166 +170167 POINT(40.453988471057016 73.13882407262942) bank170167 +170168 POINT(40.608578098331996 73.71821146077839) bank170168 +170169 POINT(40.88943747243131 73.4150920212159) bank170169 +170170 POINT(41.295576789374046 73.30328943871254) bank170170 +170171 POINT(40.27982787034287 73.097482177534) bank170171 +170172 POINT(40.17947298229992 74.69166209440793) bank170172 +170173 POINT(39.76520198404204 74.3258913194005) bank170173 +170174 POINT(40.26343147239953 73.28156812006293) bank170174 +170175 POINT(40.51726661177188 74.03371888301541) bank170175 +170176 POINT(40.43631655494786 73.77994577549939) bank170176 +170177 POINT(40.08671320618759 73.08641649533106) bank170177 +170178 POINT(40.60833178629152 73.78276106609995) bank170178 +170179 POINT(39.72579825890042 73.93755072236777) bank170179 +170180 POINT(41.13897941594933 74.39641482525896) bank170180 +170181 POINT(40.18423636081341 73.1795433069953) bank170181 +170182 POINT(41.457991506337585 73.60422886402104) bank170182 +170183 POINT(40.19448545327931 74.68261576369902) bank170183 +170184 POINT(41.071552134625016 74.31488543945305) bank170184 +170185 POINT(41.431824252071884 74.72656376802655) bank170185 +170186 POINT(40.88036054900853 73.21603344846528) bank170186 +170187 POINT(39.86508147719609 73.70202891976932) bank170187 +170188 POINT(40.876583078232436 73.28505035120982) bank170188 +170189 POINT(40.16442422626575 74.79589817895997) bank170189 +170190 POINT(41.01618509924799 73.72559332304618) bank170190 +170191 POINT(41.71234948328121 74.1044905646473) bank170191 +170192 POINT(40.652939575831844 74.61343891600549) bank170192 +170193 POINT(41.33572815378753 74.59407481119007) bank170193 +170194 POINT(41.3498734309984 73.90468893704116) bank170194 +170195 POINT(40.06134984456341 73.8456685377562) bank170195 +170196 POINT(40.015390158074425 73.53831582827547) bank170196 +170197 POINT(41.51745873937992 73.02480662729467) bank170197 +170198 POINT(40.00475901506661 73.87586527423996) bank170198 +170199 POINT(41.558955492256146 73.15118004851632) bank170199 +170200 POINT(40.427687071449654 74.4221159346483) bank170200 +170201 POINT(40.84072987860165 75.00121769311372) bank170201 +170202 POINT(41.09490495384527 73.493014844458) bank170202 +170203 POINT(40.960062212203894 73.05890785656715) bank170203 +170204 POINT(40.11539578374174 73.91767753994503) bank170204 +170205 POINT(39.85689395152278 74.00159901528303) bank170205 +170206 POINT(40.14317606104397 74.05015087791008) bank170206 +170207 POINT(41.6346504275408 73.67811507008203) bank170207 +170208 POINT(40.149502393822125 73.7639814243702) bank170208 +170209 POINT(39.7917923442407 73.5481884522046) bank170209 +170210 POINT(40.39448362863852 74.65510482439446) bank170210 +170211 POINT(40.060874840831076 73.77563321059422) bank170211 +170212 POINT(40.83683317866095 73.53224733444691) bank170212 +170213 POINT(40.4015043508859 73.43453952964268) bank170213 +170214 POINT(40.84800380076248 73.36248822639499) bank170214 +170215 POINT(39.76895553205616 73.72465385259407) bank170215 +170216 POINT(40.61064024118474 74.66370328135035) bank170216 +170217 POINT(40.69288697528914 73.33009515652383) bank170217 +170218 POINT(41.21226425696736 73.91843821568239) bank170218 +170219 POINT(40.69798940429296 74.22535274807882) bank170219 +170220 POINT(40.73946198455577 73.49600556995725) bank170220 +170221 POINT(40.90687310129455 74.76426081652944) bank170221 +170222 POINT(39.738162472229966 74.52314598182204) bank170222 +170223 POINT(41.07443778325237 74.67981924953855) bank170223 +170224 POINT(41.25030127169971 74.27031532347272) bank170224 +170225 POINT(40.99607114073376 73.34999129589502) bank170225 +170226 POINT(40.14610158553145 74.15524102680266) bank170226 +170227 POINT(40.98637170038484 74.32330386214326) bank170227 +170228 POINT(41.30830081896385 74.72598594913921) bank170228 +170229 POINT(40.633589441039064 74.55222465153297) bank170229 +170230 POINT(39.77870319707157 73.59165378694432) bank170230 +170231 POINT(41.503140940416145 73.39302246955992) bank170231 +170232 POINT(41.209680492942624 73.64344561704841) bank170232 +170233 POINT(41.23426604201708 74.59043447583974) bank170233 +170234 POINT(40.635169678895366 73.17338889989635) bank170234 +170235 POINT(39.84316617912036 74.26330033819406) bank170235 +170236 POINT(39.875479116907194 73.29584428836836) bank170236 +170237 POINT(41.041190014795234 73.13069718234561) bank170237 +170238 POINT(40.969906700612135 73.74590172481422) bank170238 +170239 POINT(39.89761803973165 73.29022099539759) bank170239 +170240 POINT(41.41554668261274 74.10044590937649) bank170240 +170241 POINT(41.51658585750225 74.41032190239032) bank170241 +170242 POINT(40.35416906718124 74.16030899948743) bank170242 +170243 POINT(40.28961968159281 73.43859747826212) bank170243 +170244 POINT(40.73318797103234 73.43201783491862) bank170244 +170245 POINT(41.141378397428035 74.44337811153056) bank170245 +170246 POINT(40.89917281243221 73.82275799742617) bank170246 +170247 POINT(40.28965361327337 74.45219009120736) bank170247 +170248 POINT(40.182284242504416 73.32440116402158) bank170248 +170249 POINT(40.336016439637895 74.46338375971234) bank170249 +170250 POINT(41.078415918873716 73.90136719730849) bank170250 +170251 POINT(40.40378113151673 74.74735296320083) bank170251 +170252 POINT(39.907631279776425 74.11267423142293) bank170252 +170253 POINT(40.10473548242903 74.43389703882475) bank170253 +170254 POINT(40.47120528382972 74.20785573791393) bank170254 +170255 POINT(40.89258080930293 74.96188636738181) bank170255 +170256 POINT(40.15849967451982 73.82293281896989) bank170256 +170257 POINT(40.82848601470984 74.14502352045574) bank170257 +170258 POINT(41.2214086634259 73.24513722653691) bank170258 +170259 POINT(39.91170525801494 74.92431205543434) bank170259 +170260 POINT(39.72737759157061 74.04447642301197) bank170260 +170261 POINT(41.547437681060515 73.08233537316144) bank170261 +170262 POINT(40.46412169963699 73.64651313622605) bank170262 +170263 POINT(40.655974898305004 74.73905689812729) bank170263 +170264 POINT(40.65515166076088 73.42227043447032) bank170264 +170265 POINT(40.9472177375955 73.50255390796686) bank170265 +170266 POINT(39.98515723667905 73.65783326803151) bank170266 +170267 POINT(40.2011116441657 74.34912971828061) bank170267 +170268 POINT(40.832084284424454 74.93402254980597) bank170268 +170269 POINT(40.83588973078912 74.36054602952251) bank170269 +170270 POINT(40.15721083088334 74.40435352177161) bank170270 +170271 POINT(40.549530601324726 73.37895249389535) bank170271 +170272 POINT(40.67219482690228 73.63532036682179) bank170272 +170273 POINT(40.376366489832485 74.32628189222048) bank170273 +170274 POINT(40.8571239061413 74.82568004980195) bank170274 +170275 POINT(40.91750648897579 73.21226231828662) bank170275 +170276 POINT(40.132570826346274 73.74231414052588) bank170276 +170277 POINT(41.47706672558045 74.88112265890668) bank170277 +170278 POINT(40.801658437670945 74.91717543662112) bank170278 +170279 POINT(40.58662051164349 74.55060784768246) bank170279 +170280 POINT(40.27516004443196 73.48564127202644) bank170280 +170281 POINT(39.76091266030414 74.14569057991017) bank170281 +170282 POINT(40.623612563669845 73.87792419714884) bank170282 +170283 POINT(41.02040940776133 74.51860183552506) bank170283 +170284 POINT(40.7960247237708 74.30554253920226) bank170284 +170285 POINT(40.64279852593393 73.58926882996272) bank170285 +170286 POINT(40.71805136776144 74.22623248624349) bank170286 +170287 POINT(40.73877375862556 73.00952970667372) bank170287 +170288 POINT(40.46962615998969 73.73192387334302) bank170288 +170289 POINT(39.760302543406304 73.27677322030245) bank170289 +170290 POINT(39.94298055858902 74.54467009504125) bank170290 +170291 POINT(40.48311054709548 73.21174900851035) bank170291 +170292 POINT(40.9547789976503 74.04533467098882) bank170292 +170293 POINT(39.73844138958017 74.37125333447601) bank170293 +170294 POINT(41.28357066725145 73.45666161195967) bank170294 +170295 POINT(40.56482881844099 74.07561160424117) bank170295 +170296 POINT(41.41844815490784 73.72802451449198) bank170296 +170297 POINT(40.772356829428745 73.09110201541466) bank170297 +170298 POINT(40.013229403680434 73.08039648218086) bank170298 +170299 POINT(41.129689300286365 74.19411063418563) bank170299 +170300 POINT(41.22143095248126 74.47408089769888) bank170300 +170301 POINT(40.337848511277784 74.32950132772696) bank170301 +170302 POINT(41.578235834617665 73.21735741440608) bank170302 +170303 POINT(40.979330477008006 73.15185370964046) bank170303 +170304 POINT(40.773837577325246 73.09347174133327) bank170304 +170305 POINT(41.11216527333376 73.7398770056123) bank170305 +170306 POINT(39.789745025455744 74.62334539358933) bank170306 +170307 POINT(40.735693512948394 73.16381651141806) bank170307 +170308 POINT(40.46471728169568 74.23065196402607) bank170308 +170309 POINT(40.23902836997424 74.5315604515483) bank170309 +170310 POINT(40.461852555327816 74.71985879380921) bank170310 +170311 POINT(40.32258447692169 73.87579977884468) bank170311 +170312 POINT(40.882897775213046 74.60756940995851) bank170312 +170313 POINT(40.51121648632354 73.63779802302004) bank170313 +170314 POINT(40.4959768971415 74.5471378947917) bank170314 +170315 POINT(40.41651149349516 74.8427130685304) bank170315 +170316 POINT(40.92418450031245 73.96593093032929) bank170316 +170317 POINT(40.241654894545576 74.6196536219689) bank170317 +170318 POINT(40.855832548758656 74.9220860999164) bank170318 +170319 POINT(40.46854233904084 74.08964480853705) bank170319 +170320 POINT(39.85587971563857 74.24540331335034) bank170320 +170321 POINT(41.03312919710983 73.11652523228226) bank170321 +170322 POINT(41.056668961589594 74.38094777605616) bank170322 +170323 POINT(40.753724725311464 73.75472699492295) bank170323 +170324 POINT(41.167185393241034 74.49350518743283) bank170324 +170325 POINT(39.95391902014118 74.56334916526959) bank170325 +170326 POINT(41.12369512956849 74.7023524835172) bank170326 +170327 POINT(40.58493630413317 73.22755115530263) bank170327 +170328 POINT(41.04148720390039 74.4046635767094) bank170328 +170329 POINT(41.535200484778784 74.29106518676313) bank170329 +170330 POINT(41.34476435226699 74.99288893300175) bank170330 +170331 POINT(41.48965648867452 73.88266091285756) bank170331 +170332 POINT(40.0098716126788 73.70671628712613) bank170332 +170333 POINT(39.83252950904948 73.12953136782386) bank170333 +170334 POINT(39.89316883073783 73.62832241461787) bank170334 +170335 POINT(40.2149363118397 74.25603507050582) bank170335 +170336 POINT(40.953415208660566 74.91920642776465) bank170336 +170337 POINT(40.49228699782679 73.69022354418009) bank170337 +170338 POINT(41.11569120610765 73.82968377396759) bank170338 +170339 POINT(40.74833291609313 73.01902960737698) bank170339 +170340 POINT(40.12405939442975 74.22785565949626) bank170340 +170341 POINT(40.67668815501504 73.80655234508119) bank170341 +170342 POINT(40.10846134898526 74.24772312514392) bank170342 +170343 POINT(40.83517193953873 74.5576597646124) bank170343 +170344 POINT(40.199056462550146 73.97534697252591) bank170344 +170345 POINT(40.37666696964837 74.4182040870524) bank170345 +170346 POINT(41.261535069707904 73.14090447540866) bank170346 +170347 POINT(41.30697622041835 73.32830612795401) bank170347 +170348 POINT(41.51629746072621 73.3751228471934) bank170348 +170349 POINT(41.53533633054083 74.2400469247266) bank170349 +170350 POINT(40.01682976845946 73.3740202713918) bank170350 +170351 POINT(41.46493854119022 74.97624473359957) bank170351 +170352 POINT(41.15590291057376 74.67387607096093) bank170352 +170353 POINT(40.49567555109893 74.53977091484236) bank170353 +170354 POINT(39.753341078441935 73.6741160841127) bank170354 +170355 POINT(40.32968960978443 74.72803585576463) bank170355 +170356 POINT(41.09747104306333 73.33213711244366) bank170356 +170357 POINT(39.83207624321682 73.54205718272863) bank170357 +170358 POINT(41.574362557731405 74.5472985535989) bank170358 +170359 POINT(40.16410035056541 73.00984428440822) bank170359 +170360 POINT(41.21495428435364 73.7631899734627) bank170360 +170361 POINT(40.3895522661765 74.9539770373599) bank170361 +170362 POINT(41.403520408954726 73.20201024969798) bank170362 +170363 POINT(41.28710087422181 73.35581763332851) bank170363 +170364 POINT(41.449931290942885 74.00776511830058) bank170364 +170365 POINT(41.56219782827158 73.32127850124382) bank170365 +170366 POINT(41.28377969450908 73.51804475710136) bank170366 +170367 POINT(41.50265591254249 73.67640267299973) bank170367 +170368 POINT(40.673242653689904 73.92510973181882) bank170368 +170369 POINT(41.0165454139463 73.22627722192547) bank170369 +170370 POINT(41.41298422153376 74.81406326208345) bank170370 +170371 POINT(40.133972770042156 73.4410944683768) bank170371 +170372 POINT(40.16576067051473 73.99446189940811) bank170372 +170373 POINT(39.790849321531965 74.63529858494847) bank170373 +170374 POINT(40.3485709613478 73.01842809956051) bank170374 +170375 POINT(41.71193202938936 74.1024943274973) bank170375 +170376 POINT(40.974043522755004 74.83471057458297) bank170376 +170377 POINT(41.38219345345544 73.23645160188407) bank170377 +170378 POINT(39.993825503037144 74.78796736554017) bank170378 +170379 POINT(39.978346098469856 73.26890226143955) bank170379 +170380 POINT(41.04482790601905 74.87614519108837) bank170380 +170381 POINT(40.66534707583081 74.41242929688354) bank170381 +170382 POINT(41.36275360333182 73.94544812431339) bank170382 +170383 POINT(41.65930523870022 74.42496720349155) bank170383 +170384 POINT(40.87585688056381 74.56595389841367) bank170384 +170385 POINT(39.778395498105844 73.07193952717991) bank170385 +170386 POINT(40.22066360289584 74.06199163726411) bank170386 +170387 POINT(39.85278415251948 74.09716016811915) bank170387 +170388 POINT(41.56849399484778 73.99044841842338) bank170388 +170389 POINT(41.35461510814255 74.94982230959548) bank170389 +170390 POINT(40.386846153531955 74.62394249960039) bank170390 +170391 POINT(41.45022109176109 73.28902850778039) bank170391 +170392 POINT(41.17064441549784 73.74916424106937) bank170392 +170393 POINT(40.53652369018029 73.13346986613477) bank170393 +170394 POINT(40.09815580036777 73.16436806619784) bank170394 +170395 POINT(40.06022061138685 74.5333691372039) bank170395 +170396 POINT(40.558935065724334 73.58168517000928) bank170396 +170397 POINT(40.0840390166911 74.79946100345516) bank170397 +170398 POINT(40.08685921857039 73.46994538169368) bank170398 +170399 POINT(40.984311851610286 74.72455879553941) bank170399 +170400 POINT(40.06156097990434 74.10472223029443) bank170400 +170401 POINT(41.128407467841576 74.03652099739884) bank170401 +170402 POINT(41.66696427877102 73.82152605388885) bank170402 +170403 POINT(39.913516717504436 74.96414293468791) bank170403 +170404 POINT(41.01599527410138 74.9832472177423) bank170404 +170405 POINT(39.845677941385496 74.43380585562632) bank170405 +170406 POINT(40.59911865947436 73.8768645718666) bank170406 +170407 POINT(40.24716677991365 73.56669930526215) bank170407 +170408 POINT(41.577855394491365 74.91016004953332) bank170408 +170409 POINT(41.402455574096884 74.24489747023573) bank170409 +170410 POINT(41.39070125816977 73.36414104126791) bank170410 +170411 POINT(39.772290379283945 74.5027756169469) bank170411 +170412 POINT(40.04078710978483 74.33969435531085) bank170412 +170413 POINT(41.20767204161672 73.45934625626484) bank170413 +170414 POINT(41.11678209830933 73.37497214591511) bank170414 +170415 POINT(40.35360149049688 74.22796229034381) bank170415 +170416 POINT(41.35678857409067 73.63611283295332) bank170416 +170417 POINT(40.164967103179976 73.17156198765873) bank170417 +170418 POINT(39.864412935080246 73.2192330119108) bank170418 +170419 POINT(41.04675019596696 73.94210854841747) bank170419 +170420 POINT(40.84833668994729 73.77216551954241) bank170420 +170421 POINT(40.68856528269569 73.53091236837759) bank170421 +170422 POINT(41.16063486613159 74.77832796861291) bank170422 +170423 POINT(41.610271823411125 73.60820859328852) bank170423 +170424 POINT(41.099126606432776 73.39047484684076) bank170424 +170425 POINT(39.98264306486163 73.15637379037179) bank170425 +170426 POINT(39.882085689473115 73.34399770403357) bank170426 +170427 POINT(41.43531408735866 73.66367131793632) bank170427 +170428 POINT(40.63703805736878 73.89032198071118) bank170428 +170429 POINT(40.338009343406775 73.88790930439237) bank170429 +170430 POINT(40.83023438277442 73.55470723399722) bank170430 +170431 POINT(40.443826343033145 73.21290848191238) bank170431 +170432 POINT(39.76905978107756 73.80138805976166) bank170432 +170433 POINT(40.19997114330206 74.43704579008501) bank170433 +170434 POINT(40.02006909097867 74.7707383010387) bank170434 +170435 POINT(41.05143288740747 74.82920832381451) bank170435 +170436 POINT(40.16896993235584 74.94066863206251) bank170436 +170437 POINT(41.53755888541313 73.34904054557128) bank170437 +170438 POINT(40.59629273096398 74.36598775800867) bank170438 +170439 POINT(41.576301850527464 73.35077551006033) bank170439 +170440 POINT(39.898423136189734 74.39706836555034) bank170440 +170441 POINT(40.39547571574113 73.3445169684405) bank170441 +170442 POINT(40.507074130220595 74.07019176930815) bank170442 +170443 POINT(41.345234067110546 74.94785125803492) bank170443 +170444 POINT(41.26084568453345 74.44109205995419) bank170444 +170445 POINT(40.584128912459164 74.32063085212117) bank170445 +170446 POINT(41.0828297153919 74.92874010305454) bank170446 +170447 POINT(40.309295045265074 73.19740709436695) bank170447 +170448 POINT(40.97075408002863 73.3628079315401) bank170448 +170449 POINT(41.46566060236188 73.94197841407973) bank170449 +170450 POINT(40.289516559438916 74.25062991622383) bank170450 +170451 POINT(40.75149203447199 73.9841611597135) bank170451 +170452 POINT(40.412834944735486 73.80704202407225) bank170452 +170453 POINT(39.91353449774985 73.52703197287096) bank170453 +170454 POINT(41.64991545119119 74.22463230588768) bank170454 +170455 POINT(40.72013296269806 74.25098135755634) bank170455 +170456 POINT(39.96177362847244 74.48402356407213) bank170456 +170457 POINT(40.08444964664802 74.15730829714579) bank170457 +170458 POINT(39.74124386128644 74.84820153045486) bank170458 +170459 POINT(39.948904039948516 74.94773988028676) bank170459 +170460 POINT(39.86451220584368 73.2921821705023) bank170460 +170461 POINT(41.005382332340076 73.11334979334184) bank170461 +170462 POINT(40.87728482832283 73.736746058909) bank170462 +170463 POINT(40.46416704806042 73.3572608101805) bank170463 +170464 POINT(40.403843593434495 74.1161794813342) bank170464 +170465 POINT(41.39107985065499 73.69924681449244) bank170465 +170466 POINT(40.05751777906961 73.25258846449401) bank170466 +170467 POINT(40.86413529365532 73.38485585201448) bank170467 +170468 POINT(41.324960756407 74.78876861894562) bank170468 +170469 POINT(40.537794679312434 74.07426827487258) bank170469 +170470 POINT(40.19579856295316 73.45664816246789) bank170470 +170471 POINT(39.83277947707631 74.29427993125832) bank170471 +170472 POINT(40.516116605637116 73.17656771955662) bank170472 +170473 POINT(40.32716868903806 74.09736699727523) bank170473 +170474 POINT(41.65999984623135 74.89093489005262) bank170474 +170475 POINT(41.66761522597928 74.1703846065286) bank170475 +170476 POINT(40.30419211087099 74.07638791877521) bank170476 +170477 POINT(41.043854966978415 73.57500956997694) bank170477 +170478 POINT(40.93982640146886 74.30016168398349) bank170478 +170479 POINT(40.26008753323739 73.32743567501906) bank170479 +170480 POINT(40.436946432711586 73.28098985614103) bank170480 +170481 POINT(41.59255209884621 73.40228089820772) bank170481 +170482 POINT(40.65365016090853 73.92036977880908) bank170482 +170483 POINT(41.08072821413573 73.16999250271556) bank170483 +170484 POINT(41.0890655584621 74.06265737615418) bank170484 +170485 POINT(40.82682209311433 74.0581673145491) bank170485 +170486 POINT(40.34179333171418 74.6136381084222) bank170486 +170487 POINT(40.97755892516442 73.67811780385499) bank170487 +170488 POINT(40.464837818829956 73.45109621716976) bank170488 +170489 POINT(39.88379286517784 74.76204479314046) bank170489 +170490 POINT(39.98484110247435 73.71286260215012) bank170490 +170491 POINT(41.69044619132489 73.9854060513168) bank170491 +170492 POINT(40.42038476311979 74.80291467299429) bank170492 +170493 POINT(40.489423786144286 74.06096813443294) bank170493 +170494 POINT(41.2134257756365 74.02583959238868) bank170494 +170495 POINT(41.63409508333081 74.91550145029231) bank170495 +170496 POINT(40.247732377907795 74.52445280576129) bank170496 +170497 POINT(41.234465766717975 73.06120715780702) bank170497 +170498 POINT(39.8798576339872 74.72898171661639) bank170498 +170499 POINT(41.200801690180214 74.05032442918778) bank170499 +170500 POINT(40.41728719551805 74.99265044250298) bank170500 +170501 POINT(40.457850389692055 74.67605147757442) bank170501 +170502 POINT(40.344249495932964 73.5223298484501) bank170502 +170503 POINT(40.336878719585734 73.9810255523437) bank170503 +170504 POINT(40.0707303765734 73.52250694435484) bank170504 +170505 POINT(40.24206014175788 75.00470362684095) bank170505 +170506 POINT(39.72667916208941 74.21454658562062) bank170506 +170507 POINT(40.144887682615284 74.35930832397023) bank170507 +170508 POINT(40.52605102124865 74.24180133349202) bank170508 +170509 POINT(40.530708415711764 73.6479996052117) bank170509 +170510 POINT(40.010634656036295 73.38602765405162) bank170510 +170511 POINT(40.39238150457733 74.1348079557344) bank170511 +170512 POINT(41.21277486389185 74.86416803721477) bank170512 +170513 POINT(40.7041564853578 74.04537213019746) bank170513 +170514 POINT(39.78762690700262 73.81648833611739) bank170514 +170515 POINT(41.522795313888615 74.92338330515487) bank170515 +170516 POINT(40.72692767738826 73.32398995953898) bank170516 +170517 POINT(39.76457036441039 73.251583426461) bank170517 +170518 POINT(40.636719458240414 74.75251614420979) bank170518 +170519 POINT(40.85538471452677 73.47756100882648) bank170519 +170520 POINT(41.06357648367975 73.14293004200138) bank170520 +170521 POINT(41.33899231586239 74.98227807425191) bank170521 +170522 POINT(40.91597598532185 73.15375526787088) bank170522 +170523 POINT(41.63450515664227 74.65547490941357) bank170523 +170524 POINT(41.649491069606015 74.69881919885613) bank170524 +170525 POINT(40.342574918823345 73.06786325377197) bank170525 +170526 POINT(39.945567110352314 73.3554440923542) bank170526 +170527 POINT(41.5034582982382 73.56567891996184) bank170527 +170528 POINT(40.84757060383266 74.19846552653875) bank170528 +170529 POINT(41.26473317653785 74.74964279265652) bank170529 +170530 POINT(41.51831507291594 73.12486352256906) bank170530 +170531 POINT(41.55273273893939 74.33468221282223) bank170531 +170532 POINT(41.26083327561145 74.80513735965211) bank170532 +170533 POINT(40.80927132655314 73.98752808614775) bank170533 +170534 POINT(39.97112435337465 73.53952002147442) bank170534 +170535 POINT(40.92024588722381 74.97329287112362) bank170535 +170536 POINT(40.111491315580245 74.1358314821642) bank170536 +170537 POINT(40.43841978246705 74.29741876824387) bank170537 +170538 POINT(40.75632622326757 73.0420529261607) bank170538 +170539 POINT(40.06808947347696 73.6960624682143) bank170539 +170540 POINT(40.83310680064521 74.89908199882586) bank170540 +170541 POINT(40.18223512939489 73.10043761765182) bank170541 +170542 POINT(40.95032338788767 73.36448769878282) bank170542 +170543 POINT(40.19835847193832 73.0561126869005) bank170543 +170544 POINT(40.421873405407666 74.68615126351995) bank170544 +170545 POINT(41.54259395285674 74.97722681933203) bank170545 +170546 POINT(39.82978716754167 74.41138250248453) bank170546 +170547 POINT(41.6712822968412 73.43859334008289) bank170547 +170548 POINT(41.380518697625135 74.93177216373172) bank170548 +170549 POINT(41.59970115828604 74.8834114043156) bank170549 +170550 POINT(40.310425947123655 75.00069470832021) bank170550 +170551 POINT(40.36624336981053 73.15509148122003) bank170551 +170552 POINT(41.53973996659852 74.36504569274008) bank170552 +170553 POINT(40.23139796715409 74.67020884125088) bank170553 +170554 POINT(39.714080704040256 74.47039694050521) bank170554 +170555 POINT(40.94332935703859 73.10542984726176) bank170555 +170556 POINT(40.10225610392589 74.59380373896377) bank170556 +170557 POINT(41.590060485110676 73.75479875058265) bank170557 +170558 POINT(40.17737028308227 73.67672755196601) bank170558 +170559 POINT(40.39743917711019 73.20047366510373) bank170559 +170560 POINT(40.539579462511405 73.66341790118032) bank170560 +170561 POINT(41.326844495912574 73.60316018262112) bank170561 +170562 POINT(40.55041385149426 73.50248343858445) bank170562 +170563 POINT(39.9547070286504 74.50589013597991) bank170563 +170564 POINT(40.643816552828866 73.96587820052298) bank170564 +170565 POINT(41.439578965415706 74.33710384322015) bank170565 +170566 POINT(40.82206681911483 73.90574750307353) bank170566 +170567 POINT(40.924632282084836 74.39394018594679) bank170567 +170568 POINT(40.654423305864924 73.53915233705025) bank170568 +170569 POINT(40.63371501584085 73.96651041882023) bank170569 +170570 POINT(41.323591584115874 73.07720096767581) bank170570 +170571 POINT(40.55997215856135 74.39027201992057) bank170571 +170572 POINT(39.88160141707467 73.4296058677449) bank170572 +170573 POINT(40.701142368161065 73.27060597780464) bank170573 +170574 POINT(39.785585796024115 74.20685560840747) bank170574 +170575 POINT(40.52879490875605 74.2960078836621) bank170575 +170576 POINT(40.52283595030234 74.43502902802952) bank170576 +170577 POINT(40.125437055145795 73.08069120919545) bank170577 +170578 POINT(41.6511622238066 74.57037682116594) bank170578 +170579 POINT(41.44848150870415 74.96665491449808) bank170579 +170580 POINT(41.35997348291129 73.15496607855785) bank170580 +170581 POINT(41.0486954556259 74.9448002803182) bank170581 +170582 POINT(40.15389302578982 73.28715215462343) bank170582 +170583 POINT(40.27234031607888 73.31715347157818) bank170583 +170584 POINT(39.841344716527416 73.60445208620399) bank170584 +170585 POINT(40.08694105938094 74.18940230596424) bank170585 +170586 POINT(39.91975299952843 73.31975255264194) bank170586 +170587 POINT(41.70760656316793 74.24932740218071) bank170587 +170588 POINT(39.93152254670956 74.04004001126087) bank170588 +170589 POINT(40.246871729491914 74.15454234879485) bank170589 +170590 POINT(41.56335433621219 73.38910916798515) bank170590 +170591 POINT(40.538687072698764 73.90207420933535) bank170591 +170592 POINT(39.825421122756175 74.89585810166075) bank170592 +170593 POINT(40.639179688728305 74.13471072807845) bank170593 +170594 POINT(39.887681092387865 74.68962740544069) bank170594 +170595 POINT(41.658373193896544 73.50657435635361) bank170595 +170596 POINT(40.18356497959276 73.50149680519942) bank170596 +170597 POINT(40.29004227887188 74.23815448845718) bank170597 +170598 POINT(41.43019836281251 74.39122404833603) bank170598 +170599 POINT(41.11553328833801 73.26437396091457) bank170599 +170600 POINT(40.43097626047412 73.12895672761645) bank170600 +170601 POINT(39.86881457029419 74.4298546068929) bank170601 +170602 POINT(40.937059649252014 73.1475328661966) bank170602 +170603 POINT(40.89544750175306 73.12652799649001) bank170603 +170604 POINT(40.59953562079337 74.72417915891798) bank170604 +170605 POINT(40.56558546937962 74.20516597023814) bank170605 +170606 POINT(41.228251950399745 73.88174713039572) bank170606 +170607 POINT(40.44114300803551 74.53293750832843) bank170607 +170608 POINT(41.21414351309521 73.2256191101667) bank170608 +170609 POINT(41.695958298751634 73.51226767427812) bank170609 +170610 POINT(40.40013867590461 74.07288255672896) bank170610 +170611 POINT(40.41322670542599 74.14399869397882) bank170611 +170612 POINT(41.05843312519629 73.61012942175199) bank170612 +170613 POINT(40.4957989527987 74.11122929750206) bank170613 +170614 POINT(40.71568313906663 74.17568585262568) bank170614 +170615 POINT(41.297505608492656 73.43522502406256) bank170615 +170616 POINT(41.55591581148993 73.24490015581765) bank170616 +170617 POINT(39.96445714128929 74.10144646884214) bank170617 +170618 POINT(40.75054476114983 73.75051635782) bank170618 +170619 POINT(39.99620608642161 74.49047573390976) bank170619 +170620 POINT(41.50892456488075 74.70889191067869) bank170620 +170621 POINT(40.45378867954372 74.37708015747624) bank170621 +170622 POINT(40.73614117996437 74.37574517122803) bank170622 +170623 POINT(40.66926613029371 74.2267245017568) bank170623 +170624 POINT(41.05125562703759 74.8611586980583) bank170624 +170625 POINT(40.86680528857156 74.17843918023243) bank170625 +170626 POINT(41.695823812597084 73.25948082370313) bank170626 +170627 POINT(40.03716688416854 74.64174781661491) bank170627 +170628 POINT(40.05144886704503 73.5639057103469) bank170628 +170629 POINT(41.10643408065514 74.19155729580882) bank170629 +170630 POINT(40.01135123473071 74.68262877759958) bank170630 +170631 POINT(40.32548414628474 74.93419324556805) bank170631 +170632 POINT(40.788296775199555 74.42868413613608) bank170632 +170633 POINT(40.96901816619671 73.51695512477968) bank170633 +170634 POINT(40.672223364941104 74.07126476885209) bank170634 +170635 POINT(39.93722190012604 73.83762240784073) bank170635 +170636 POINT(40.168276669696226 73.57362693041765) bank170636 +170637 POINT(40.31234856588652 74.2014261913184) bank170637 +170638 POINT(40.07545475448857 73.96485323745924) bank170638 +170639 POINT(41.353719380462785 73.94598276776198) bank170639 +170640 POINT(40.39630481858082 74.97766621563825) bank170640 +170641 POINT(41.05464846722282 74.35739417856465) bank170641 +170642 POINT(41.32884699849957 74.1697771989521) bank170642 +170643 POINT(40.5368392759807 73.28565392578598) bank170643 +170644 POINT(40.10267547863904 74.53309773443638) bank170644 +170645 POINT(40.23433199554579 73.57215463861169) bank170645 +170646 POINT(41.37412750040839 74.95885602438311) bank170646 +170647 POINT(40.58176489061129 73.88021734248886) bank170647 +170648 POINT(40.11697780791942 73.74523388904143) bank170648 +170649 POINT(41.25527672175168 73.62417807752193) bank170649 +170650 POINT(41.19612461909046 74.49348591644068) bank170650 +170651 POINT(40.448750500385955 74.33801038621074) bank170651 +170652 POINT(39.87087575283933 74.2404097759638) bank170652 +170653 POINT(40.793938657625375 74.55498515084871) bank170653 +170654 POINT(41.13349204995735 74.48523848566217) bank170654 +170655 POINT(41.45202269800602 73.10627834966452) bank170655 +170656 POINT(39.889757561753804 73.36365018274579) bank170656 +170657 POINT(40.54694488686478 73.86671311023946) bank170657 +170658 POINT(40.15974212951105 74.6103406716468) bank170658 +170659 POINT(39.90917413035517 73.96404347465203) bank170659 +170660 POINT(40.88239624628339 74.67344202522281) bank170660 +170661 POINT(40.0251453876637 74.92513157710736) bank170661 +170662 POINT(41.17140811621419 73.92075021868806) bank170662 +170663 POINT(41.222503798735 74.3296187757764) bank170663 +170664 POINT(40.06605260233946 74.06638622755729) bank170664 +170665 POINT(40.1689873970641 73.39375041335566) bank170665 +170666 POINT(41.59367693307841 73.06199702160923) bank170666 +170667 POINT(39.77797295014637 74.9120280355131) bank170667 +170668 POINT(41.64026446814937 73.35376614584294) bank170668 +170669 POINT(40.64580306106362 74.47550158856426) bank170669 +170670 POINT(39.818951210244116 74.00960262796295) bank170670 +170671 POINT(40.924641645451075 74.198910003805) bank170671 +170672 POINT(40.491253724416815 74.68090441613066) bank170672 +170673 POINT(41.58066416838214 73.58786520067478) bank170673 +170674 POINT(40.12622771921487 73.83589864269632) bank170674 +170675 POINT(41.38796855917003 74.46651067181584) bank170675 +170676 POINT(41.67997905127348 74.5763960578734) bank170676 +170677 POINT(40.9345933710039 74.79634021544271) bank170677 +170678 POINT(40.72857766298286 74.29928810049672) bank170678 +170679 POINT(40.32070586847025 74.29964868596795) bank170679 +170680 POINT(39.95443542322419 74.02312959633113) bank170680 +170681 POINT(41.347027530875486 74.80251705110851) bank170681 +170682 POINT(40.97081435995461 73.08925642096933) bank170682 +170683 POINT(41.04404382613261 74.40058563880461) bank170683 +170684 POINT(40.20711402096459 73.86591932276373) bank170684 +170685 POINT(41.082913694503596 74.06973858147725) bank170685 +170686 POINT(41.71119077729118 74.53478079072467) bank170686 +170687 POINT(40.19447196720642 74.11386334839486) bank170687 +170688 POINT(39.896044524538226 74.56088161706357) bank170688 +170689 POINT(41.2427198009139 74.04485019361175) bank170689 +170690 POINT(39.83027662023197 73.9795284382492) bank170690 +170691 POINT(41.6997909466901 74.4830317999263) bank170691 +170692 POINT(40.50745263960768 73.7333091390431) bank170692 +170693 POINT(41.2881429475106 73.99428274335743) bank170693 +170694 POINT(39.95581849869945 74.49729544660289) bank170694 +170695 POINT(41.658882758372364 74.65847509377606) bank170695 +170696 POINT(41.616084577076705 73.76922446620901) bank170696 +170697 POINT(39.7852587268741 74.78397361723685) bank170697 +170698 POINT(40.2294465528624 73.82046896903317) bank170698 +170699 POINT(40.46749543437696 73.64548668511505) bank170699 +170700 POINT(39.7768118533135 74.53263978680428) bank170700 +170701 POINT(39.854796813386116 74.41167741783181) bank170701 +170702 POINT(40.366874613842825 73.3321948380792) bank170702 +170703 POINT(40.48875024201341 74.44494492720996) bank170703 +170704 POINT(40.8365736389445 73.2134616750633) bank170704 +170705 POINT(39.724096530238114 74.51316061263842) bank170705 +170706 POINT(41.37815443217654 73.18501764041436) bank170706 +170707 POINT(40.93001582927425 74.49039532036913) bank170707 +170708 POINT(41.229455214837614 74.9469547379603) bank170708 +170709 POINT(40.04641978024489 74.85447122451282) bank170709 +170710 POINT(40.90580487325905 73.75796562394551) bank170710 +170711 POINT(41.39482225669594 74.29586346347664) bank170711 +170712 POINT(40.412914675878945 73.8124730950599) bank170712 +170713 POINT(40.94162428380871 74.4378172489083) bank170713 +170714 POINT(39.98323466574913 74.24932310342136) bank170714 +170715 POINT(40.464696817447205 74.92478867062113) bank170715 +170716 POINT(40.443937882838256 74.21575303083571) bank170716 +170717 POINT(40.21523164058016 74.61943458818568) bank170717 +170718 POINT(40.54870692488653 73.08641518379987) bank170718 +170719 POINT(40.549578236180814 73.19889844695653) bank170719 +170720 POINT(40.76285455385428 74.82401711564593) bank170720 +170721 POINT(40.70417915022273 73.1478749393049) bank170721 +170722 POINT(41.469253721357916 73.5306903892022) bank170722 +170723 POINT(40.40446229875877 74.83102187510427) bank170723 +170724 POINT(39.891622250636296 73.58114620311565) bank170724 +170725 POINT(40.14425217400869 74.24865620612225) bank170725 +170726 POINT(41.54110499977293 73.71418261228388) bank170726 +170727 POINT(39.916802639281194 74.93759341354986) bank170727 +170728 POINT(41.15149624218905 74.26148887603193) bank170728 +170729 POINT(40.61316268003075 74.46823349895848) bank170729 +170730 POINT(41.48928561563739 74.8780325974539) bank170730 +170731 POINT(40.30209069049307 73.11640230024554) bank170731 +170732 POINT(41.484521439766354 74.20273565666258) bank170732 +170733 POINT(41.00946151936102 73.02031127860513) bank170733 +170734 POINT(41.34520607966757 73.78411077201893) bank170734 +170735 POINT(40.03462095444509 73.99812360769761) bank170735 +170736 POINT(39.890204729584866 74.17914321805563) bank170736 +170737 POINT(40.71131392916379 75.00263891430319) bank170737 +170738 POINT(40.31962114507874 73.5980598254096) bank170738 +170739 POINT(40.3747485168482 73.45856975812968) bank170739 +170740 POINT(40.226597137224644 74.11160625791919) bank170740 +170741 POINT(41.1700264560082 74.0128481831836) bank170741 +170742 POINT(40.339103516604524 74.15761477358078) bank170742 +170743 POINT(40.76676832652788 73.53707746616782) bank170743 +170744 POINT(41.554333312221935 73.516979214049) bank170744 +170745 POINT(40.063821222030946 74.04645321349498) bank170745 +170746 POINT(40.46583003997869 73.54504700818242) bank170746 +170747 POINT(41.48419783989712 74.61010007440741) bank170747 +170748 POINT(41.07645127349316 74.18151595100584) bank170748 +170749 POINT(40.02762618750102 73.04930711051554) bank170749 +170750 POINT(40.185005023249396 73.19230485248936) bank170750 +170751 POINT(40.47759117490004 73.40317521207473) bank170751 +170752 POINT(41.197286606833075 73.84153878105772) bank170752 +170753 POINT(39.75751555953735 73.01487701807886) bank170753 +170754 POINT(40.99802618653762 74.74758674714576) bank170754 +170755 POINT(39.95825676901018 73.22719910366678) bank170755 +170756 POINT(40.91855455476271 73.28396616007709) bank170756 +170757 POINT(41.48621212933352 74.61172326943067) bank170757 +170758 POINT(41.088935417527736 73.89289303494664) bank170758 +170759 POINT(39.83542200049485 74.09104030321417) bank170759 +170760 POINT(40.17875473457539 73.36485102353474) bank170760 +170761 POINT(40.11561212659835 74.24143099050626) bank170761 +170762 POINT(40.8449287521396 73.99227957728098) bank170762 +170763 POINT(40.54641761270823 74.46979707297334) bank170763 +170764 POINT(39.76964609496573 74.63230142349578) bank170764 +170765 POINT(40.064003729256726 74.66424533556226) bank170765 +170766 POINT(41.412209185882574 74.24727320776735) bank170766 +170767 POINT(40.07479079919959 74.58715298344816) bank170767 +170768 POINT(40.81286010672457 73.16995421261355) bank170768 +170769 POINT(41.11060522959002 74.63418009524847) bank170769 +170770 POINT(41.01192636123586 73.55848660612673) bank170770 +170771 POINT(40.18215181657773 73.95766455399556) bank170771 +170772 POINT(41.60027319742338 73.92514679883242) bank170772 +170773 POINT(40.007302342682806 74.56163957868539) bank170773 +170774 POINT(39.94403463443886 74.90776274899936) bank170774 +170775 POINT(41.49588911410557 73.71460394570687) bank170775 +170776 POINT(40.31280625837412 74.29189905866707) bank170776 +170777 POINT(41.6275586469749 74.2463619942936) bank170777 +170778 POINT(41.60686445819142 74.35757228159382) bank170778 +170779 POINT(41.44235094769633 74.78343102212989) bank170779 +170780 POINT(40.55781894753278 74.65103195603403) bank170780 +170781 POINT(40.99481113733661 74.46577930187289) bank170781 +170782 POINT(40.32857399884956 74.87975721619871) bank170782 +170783 POINT(40.56175459254765 74.33230064011427) bank170783 +170784 POINT(41.53543395309143 73.62983919306174) bank170784 +170785 POINT(40.32190357466192 74.13555620672135) bank170785 +170786 POINT(40.1228424900165 74.60980548014197) bank170786 +170787 POINT(40.380156459921984 73.7526379671769) bank170787 +170788 POINT(41.672291787454434 74.28526412639323) bank170788 +170789 POINT(41.31395400302673 73.56616465039376) bank170789 +170790 POINT(40.30361323745803 73.32775263372837) bank170790 +170791 POINT(40.01132349476855 73.34049093518114) bank170791 +170792 POINT(40.145063351695015 73.81084540267526) bank170792 +170793 POINT(39.762726182575065 73.93487316619391) bank170793 +170794 POINT(41.31231909854031 73.2330102817351) bank170794 +170795 POINT(40.06836190733319 74.89742835759165) bank170795 +170796 POINT(39.74174838699138 74.8703485233349) bank170796 +170797 POINT(40.07477658974329 74.4245775647415) bank170797 +170798 POINT(41.53951348485761 73.55765749121906) bank170798 +170799 POINT(40.159470645589025 73.4616046978194) bank170799 +170800 POINT(41.47240383611127 73.58428934705252) bank170800 +170801 POINT(40.68802780988757 73.10149329525277) bank170801 +170802 POINT(40.7706792737875 74.11717100142073) bank170802 +170803 POINT(40.048688343224455 73.60964362933903) bank170803 +170804 POINT(40.33317757334419 73.63252624465329) bank170804 +170805 POINT(39.72130010077937 74.93558810738618) bank170805 +170806 POINT(40.60363822664627 73.44706321256044) bank170806 +170807 POINT(40.97188211288183 73.79161616046675) bank170807 +170808 POINT(40.56534830486478 73.63927786822292) bank170808 +170809 POINT(40.762621545074396 74.00690285099103) bank170809 +170810 POINT(40.021024091033745 74.15649809636712) bank170810 +170811 POINT(40.689652432816835 73.21547454747522) bank170811 +170812 POINT(40.552329127768125 74.42956988856051) bank170812 +170813 POINT(40.187850332619014 74.52985567315709) bank170813 +170814 POINT(40.04927964182059 74.86142991893887) bank170814 +170815 POINT(40.254493289312116 73.76126791038122) bank170815 +170816 POINT(41.481218998595075 73.79466514209037) bank170816 +170817 POINT(41.45085912737459 73.83505598806677) bank170817 +170818 POINT(40.04817959191312 73.86330744146504) bank170818 +170819 POINT(41.52532062784075 74.31023732138605) bank170819 +170820 POINT(40.296706617778554 73.80693648560201) bank170820 +170821 POINT(39.89212491119777 73.59882431560811) bank170821 +170822 POINT(41.21954609591764 73.71482060337183) bank170822 +170823 POINT(41.15176627545693 73.45946900635008) bank170823 +170824 POINT(41.22125471051681 74.38158547093587) bank170824 +170825 POINT(40.996872831880104 74.761029222872) bank170825 +170826 POINT(39.76002845797923 73.4075518672493) bank170826 +170827 POINT(39.88126714759245 74.33233557361099) bank170827 +170828 POINT(40.01164739204883 73.05950332660414) bank170828 +170829 POINT(40.8579507591772 73.54559751308393) bank170829 +170830 POINT(41.46618203704069 73.09560525746777) bank170830 +170831 POINT(41.00191838564761 73.14835538408379) bank170831 +170832 POINT(39.87675230857074 74.48318478397198) bank170832 +170833 POINT(40.73901215460975 74.25838062824486) bank170833 +170834 POINT(40.23243416748264 74.20419442528413) bank170834 +170835 POINT(39.94982310364216 73.02698162186478) bank170835 +170836 POINT(40.366566144412936 74.35111793871677) bank170836 +170837 POINT(40.72784527272309 74.44407512704797) bank170837 +170838 POINT(40.03230711918088 74.33779153796989) bank170838 +170839 POINT(40.163189187699246 73.21527456364639) bank170839 +170840 POINT(41.42693156004809 74.77846908977953) bank170840 +170841 POINT(39.8290994093322 73.29359237972717) bank170841 +170842 POINT(41.435765713272026 73.7883179410607) bank170842 +170843 POINT(40.96610306146693 73.21869002513596) bank170843 +170844 POINT(39.763436188240235 74.07731261125552) bank170844 +170845 POINT(40.66880336763991 73.50307302825667) bank170845 +170846 POINT(39.84043558946738 73.41471510539081) bank170846 +170847 POINT(41.66969447100749 74.54117887177567) bank170847 +170848 POINT(40.222479677361044 73.38817901369943) bank170848 +170849 POINT(41.56455689331746 74.12817258818939) bank170849 +170850 POINT(41.13688956532964 74.8824554886143) bank170850 +170851 POINT(39.845674925200655 73.6600690126141) bank170851 +170852 POINT(40.784058898863066 73.45185109957396) bank170852 +170853 POINT(41.15651950311171 73.54038060964614) bank170853 +170854 POINT(39.77790309560525 73.9678820615823) bank170854 +170855 POINT(40.28883649221817 74.09573379735981) bank170855 +170856 POINT(41.49600460897933 73.70826061817576) bank170856 +170857 POINT(41.0582335292429 73.51995822366851) bank170857 +170858 POINT(40.6748189707947 73.75818789069831) bank170858 +170859 POINT(40.243614030243506 74.50361985299072) bank170859 +170860 POINT(39.76584122003692 74.7574360487766) bank170860 +170861 POINT(40.16041042572573 73.70943740424059) bank170861 +170862 POINT(41.48842106408277 73.75042828806946) bank170862 +170863 POINT(40.77995299714069 73.86073599981253) bank170863 +170864 POINT(40.79463077390766 74.54846839690204) bank170864 +170865 POINT(40.69753508931488 74.21046075662916) bank170865 +170866 POINT(40.6663808357076 74.86000091072697) bank170866 +170867 POINT(40.00489681915718 73.9257119358533) bank170867 +170868 POINT(40.81775274486262 74.38912686888789) bank170868 +170869 POINT(41.03197897376082 74.17463227717441) bank170869 +170870 POINT(39.85209645050587 73.97903949859557) bank170870 +170871 POINT(41.36682700892511 74.85077605300708) bank170871 +170872 POINT(40.75132159764314 74.51996125008539) bank170872 +170873 POINT(40.15820484315685 74.0519251812509) bank170873 +170874 POINT(41.11156866702155 74.94267138269201) bank170874 +170875 POINT(40.040511307783234 73.3324783055497) bank170875 +170876 POINT(41.054888016286775 74.4145016206101) bank170876 +170877 POINT(41.208292246805286 74.56310178335401) bank170877 +170878 POINT(40.750849251606795 73.77955191776036) bank170878 +170879 POINT(40.3853012825184 73.29042526818023) bank170879 +170880 POINT(40.224226645553586 73.74534321965676) bank170880 +170881 POINT(40.32072745821901 74.68236950548099) bank170881 +170882 POINT(40.720263628288 73.02629251969775) bank170882 +170883 POINT(41.23415021050302 73.82622548212217) bank170883 +170884 POINT(41.50974086507824 74.022222368111) bank170884 +170885 POINT(39.772343555154166 73.74076484183448) bank170885 +170886 POINT(41.41591929440918 73.84674791559874) bank170886 +170887 POINT(39.88571716127507 73.85374146016936) bank170887 +170888 POINT(41.44931001010239 74.88218552270826) bank170888 +170889 POINT(40.438929363360224 73.77981162250384) bank170889 +170890 POINT(41.567522614950185 74.64926398053697) bank170890 +170891 POINT(40.20165815782023 73.86279249075501) bank170891 +170892 POINT(39.91436564738225 73.99994945173181) bank170892 +170893 POINT(41.29883524148749 74.53140654604871) bank170893 +170894 POINT(39.85831976435792 74.74862853685165) bank170894 +170895 POINT(40.88619842312045 73.88594883441877) bank170895 +170896 POINT(41.41254574729286 74.04157043600699) bank170896 +170897 POINT(41.544832938382925 73.91315596102844) bank170897 +170898 POINT(40.587165288253274 74.37246435460808) bank170898 +170899 POINT(40.601056973735965 74.20639210783007) bank170899 +170900 POINT(40.88659417683274 73.5080875171387) bank170900 +170901 POINT(39.83281274631475 74.43958142218852) bank170901 +170902 POINT(40.365524784669354 73.69473747380934) bank170902 +170903 POINT(40.998945060919816 74.23744492274047) bank170903 +170904 POINT(40.05151039926693 73.052286342667) bank170904 +170905 POINT(40.062048633830706 73.79496404691258) bank170905 +170906 POINT(40.467810944415675 73.78979338191157) bank170906 +170907 POINT(41.05132692274477 74.8599836183357) bank170907 +170908 POINT(41.406718166100426 73.6048468043835) bank170908 +170909 POINT(41.70313500004241 73.89132217226532) bank170909 +170910 POINT(41.28088274207883 73.57257524718176) bank170910 +170911 POINT(40.688022311555 73.01584211194472) bank170911 +170912 POINT(41.14802081530383 74.96237898230056) bank170912 +170913 POINT(41.563565006077475 73.1155638221085) bank170913 +170914 POINT(40.312761985002545 74.80917270800528) bank170914 +170915 POINT(40.92621701735618 73.27747205449839) bank170915 +170916 POINT(40.65697513059237 73.6194632831786) bank170916 +170917 POINT(40.05013808782771 73.2596107883256) bank170917 +170918 POINT(40.6674520239404 74.4523021561101) bank170918 +170919 POINT(41.33532891111962 73.69642014162528) bank170919 +170920 POINT(41.46692465827501 73.36469267308519) bank170920 +170921 POINT(39.992220881795134 73.27902438696454) bank170921 +170922 POINT(40.04926666308038 74.74225066278676) bank170922 +170923 POINT(41.386619629502434 74.03348001267449) bank170923 +170924 POINT(40.78013085439967 73.56564947423406) bank170924 +170925 POINT(40.06418856415945 73.31759892298643) bank170925 +170926 POINT(41.32725914680224 73.90717104118467) bank170926 +170927 POINT(41.187633659160234 73.76945920908793) bank170927 +170928 POINT(39.93892952089215 74.79706909034182) bank170928 +170929 POINT(41.29836634119403 73.06526096750456) bank170929 +170930 POINT(41.25548843090047 73.90443399132924) bank170930 +170931 POINT(40.60498239998789 74.115032609545) bank170931 +170932 POINT(40.85940490094957 73.11928733200759) bank170932 +170933 POINT(39.85780333302622 74.83292413864949) bank170933 +170934 POINT(40.42973331522931 73.03469666942536) bank170934 +170935 POINT(41.597333199354615 73.55028938303994) bank170935 +170936 POINT(40.99599264889018 73.74132009531904) bank170936 +170937 POINT(41.64884817229632 73.34541268794975) bank170937 +170938 POINT(39.76573605409143 73.81003829705138) bank170938 +170939 POINT(40.105696067942276 73.93273463177628) bank170939 +170940 POINT(40.32071068803894 73.12474143566519) bank170940 +170941 POINT(41.48211848844631 74.41391020966253) bank170941 +170942 POINT(40.85118808245636 74.5105552605504) bank170942 +170943 POINT(40.39380157090555 73.13825691554695) bank170943 +170944 POINT(40.870602822054 73.06825054209688) bank170944 +170945 POINT(40.698344454390345 74.93226252188641) bank170945 +170946 POINT(40.38958819824799 73.55825252937387) bank170946 +170947 POINT(40.562663188933904 74.2731101736417) bank170947 +170948 POINT(41.02689623514228 73.39763307887226) bank170948 +170949 POINT(40.54046648616881 74.01468706671008) bank170949 +170950 POINT(40.428348022037355 73.28069925232727) bank170950 +170951 POINT(40.674727313991745 73.68849561492223) bank170951 +170952 POINT(41.32048463427919 74.47433388397614) bank170952 +170953 POINT(40.201977304809716 73.68621256349232) bank170953 +170954 POINT(40.051632190279314 73.24510460099758) bank170954 +170955 POINT(40.93762439481072 74.0460906046561) bank170955 +170956 POINT(41.52461973822481 73.49517104624775) bank170956 +170957 POINT(40.15898568005804 74.73459295092766) bank170957 +170958 POINT(41.6194501431248 73.31717433412418) bank170958 +170959 POINT(40.775908750281 74.6929492066379) bank170959 +170960 POINT(41.66033987596875 73.58857366157373) bank170960 +170961 POINT(40.55839654249632 74.02953370261048) bank170961 +170962 POINT(41.22414897721673 73.08460946267144) bank170962 +170963 POINT(41.58978540373905 73.64700862397616) bank170963 +170964 POINT(40.11138261468056 73.31207298948847) bank170964 +170965 POINT(40.23708574187237 74.7859080500498) bank170965 +170966 POINT(40.44859546742447 74.30043943695755) bank170966 +170967 POINT(41.17565306302256 73.93569474083067) bank170967 +170968 POINT(40.785304183652805 74.52634739921841) bank170968 +170969 POINT(40.132568426976356 74.66204554943326) bank170969 +170970 POINT(40.26535703578854 74.8127638085402) bank170970 +170971 POINT(41.05861323856789 73.6997605980669) bank170971 +170972 POINT(40.958638293312845 74.11144289094139) bank170972 +170973 POINT(39.883261324578804 73.76558644658209) bank170973 +170974 POINT(39.84159957393602 74.38040532188813) bank170974 +170975 POINT(40.53103519661299 73.5653457645093) bank170975 +170976 POINT(40.69765845181417 74.32540275447307) bank170976 +170977 POINT(40.70095526702505 74.83838284915011) bank170977 +170978 POINT(40.43524356324588 73.98257475958303) bank170978 +170979 POINT(40.17540861378422 73.67503630842937) bank170979 +170980 POINT(40.872972629593384 73.63937301141848) bank170980 +170981 POINT(40.63372595521828 73.28814873024201) bank170981 +170982 POINT(40.34673903396716 73.23039084375858) bank170982 +170983 POINT(40.97035125636186 73.97347319339377) bank170983 +170984 POINT(41.38149280124432 74.6121169206048) bank170984 +170985 POINT(40.02095174165688 73.38193828730098) bank170985 +170986 POINT(39.87582636014285 73.32451060653327) bank170986 +170987 POINT(39.7213167796508 74.62639653741152) bank170987 +170988 POINT(40.77069039903824 73.51609694627878) bank170988 +170989 POINT(40.58670762581622 74.1647603262686) bank170989 +170990 POINT(40.851993586913984 73.92676972640896) bank170990 +170991 POINT(39.87448811503241 74.047884712981) bank170991 +170992 POINT(40.019558993107495 73.22876604041623) bank170992 +170993 POINT(41.27216180726019 74.69004962071239) bank170993 +170994 POINT(41.54972915578475 74.63672954196872) bank170994 +170995 POINT(39.77972093441038 74.48511785694726) bank170995 +170996 POINT(40.02970588839864 74.48626697996377) bank170996 +170997 POINT(39.767627109491734 73.61759739747109) bank170997 +170998 POINT(41.66040358819545 74.06791736591353) bank170998 +170999 POINT(40.420284806620096 74.00998086951152) bank170999 +171000 POINT(40.98540692361639 73.51224042610717) bank171000 +171001 POINT(41.27942738685208 74.58550510186141) bank171001 +171002 POINT(40.99072769171062 73.71545363949907) bank171002 +171003 POINT(40.055925947119135 74.2009322063621) bank171003 +171004 POINT(41.190551622280886 73.82717668549606) bank171004 +171005 POINT(40.287587507312004 74.106657202294) bank171005 +171006 POINT(41.397130367226964 74.3040255755337) bank171006 +171007 POINT(40.39860613313089 73.24148326452239) bank171007 +171008 POINT(40.48646201126396 74.21800416262771) bank171008 +171009 POINT(41.70150711782301 74.87775827424949) bank171009 +171010 POINT(41.37548798555915 74.12477165286063) bank171010 +171011 POINT(40.929802733737354 73.47323228881858) bank171011 +171012 POINT(40.91501338252235 73.62929108788967) bank171012 +171013 POINT(41.363546563550116 74.63779821523572) bank171013 +171014 POINT(40.128506494387025 73.88248837725392) bank171014 +171015 POINT(40.38080650499309 74.37102062273814) bank171015 +171016 POINT(39.79110406504047 73.8477241441602) bank171016 +171017 POINT(41.237283135510985 74.69105492410694) bank171017 +171018 POINT(40.06217250249605 74.37509963074028) bank171018 +171019 POINT(40.65293097169676 74.10349373316656) bank171019 +171020 POINT(40.344921303307295 74.31530653561913) bank171020 +171021 POINT(40.05846875585778 73.9427566064564) bank171021 +171022 POINT(40.08061421828492 73.50526109808301) bank171022 +171023 POINT(40.73531488824778 73.37015311862008) bank171023 +171024 POINT(41.037563728572984 73.18391029435736) bank171024 +171025 POINT(40.07433410908579 73.83463768427785) bank171025 +171026 POINT(41.42964667956707 74.13496282887132) bank171026 +171027 POINT(40.48936851892174 74.02339674302057) bank171027 +171028 POINT(40.59773981415742 73.85851238037354) bank171028 +171029 POINT(40.40928966471914 73.98166552527172) bank171029 +171030 POINT(39.76354059646205 73.99022542874754) bank171030 +171031 POINT(39.8706031413497 74.12075886117883) bank171031 +171032 POINT(40.864452130323016 74.67328648284986) bank171032 +171033 POINT(41.55259512928026 74.22395327067412) bank171033 +171034 POINT(40.81554642280524 74.44702303602709) bank171034 +171035 POINT(40.4217675009002 73.78571445632186) bank171035 +171036 POINT(40.71053705136566 73.9619768746165) bank171036 +171037 POINT(39.75625171521829 74.65345096529893) bank171037 +171038 POINT(41.2645888245319 73.59652022198964) bank171038 +171039 POINT(39.987908086905975 74.6919302341456) bank171039 +171040 POINT(40.44185398117229 73.15878480717349) bank171040 +171041 POINT(39.960598111438465 73.74339591697321) bank171041 +171042 POINT(39.992943797230396 73.29738411690059) bank171042 +171043 POINT(40.13440703326732 74.62864936573571) bank171043 +171044 POINT(39.96220664380407 74.22474014802103) bank171044 +171045 POINT(40.24411195979523 74.7668578794158) bank171045 +171046 POINT(40.27693910138097 73.78619708696627) bank171046 +171047 POINT(40.21536063079268 74.81249712332341) bank171047 +171048 POINT(41.51389005453458 73.32126329511006) bank171048 +171049 POINT(40.58702598200432 74.79981230033754) bank171049 +171050 POINT(40.35064837184831 73.47895237099422) bank171050 +171051 POINT(40.64205070450593 73.9814193153004) bank171051 +171052 POINT(40.60441219273113 74.0234285733923) bank171052 +171053 POINT(41.26034934359254 73.55227885893257) bank171053 +171054 POINT(41.444848707532664 73.10956833602064) bank171054 +171055 POINT(40.24964299890571 73.50459862651063) bank171055 +171056 POINT(41.61767506659 73.24763863883526) bank171056 +171057 POINT(40.371168629592816 74.45086193250718) bank171057 +171058 POINT(41.15356532447944 74.77487188576086) bank171058 +171059 POINT(41.31692930365716 73.1010849730389) bank171059 +171060 POINT(40.689735396879996 74.54647023549153) bank171060 +171061 POINT(41.65850705334618 73.3936493992569) bank171061 +171062 POINT(40.174292811321976 73.06956235060187) bank171062 +171063 POINT(40.43131119689924 74.48173317032627) bank171063 +171064 POINT(41.38070936905821 73.35109156129292) bank171064 +171065 POINT(41.25928752287545 74.07726889601773) bank171065 +171066 POINT(39.718978226325476 74.25492905631437) bank171066 +171067 POINT(41.15259110052236 73.1324602239895) bank171067 +171068 POINT(40.525160930590076 73.7040878860169) bank171068 +171069 POINT(41.65612822127403 74.7074513571995) bank171069 +171070 POINT(40.19630690259323 73.57308878263221) bank171070 +171071 POINT(40.937756548104964 73.96720907110642) bank171071 +171072 POINT(40.47968071378932 74.1924146779965) bank171072 +171073 POINT(40.16487648854304 73.37530684173288) bank171073 +171074 POINT(41.51709695431747 73.40735874059348) bank171074 +171075 POINT(39.93887138153963 74.45163393226859) bank171075 +171076 POINT(40.95607432465722 74.1204812772788) bank171076 +171077 POINT(40.42721989760147 73.71562698520351) bank171077 +171078 POINT(41.22089648967705 73.52770999563171) bank171078 +171079 POINT(40.5223142717137 74.34781525399234) bank171079 +171080 POINT(39.8676313080929 74.5242906728027) bank171080 +171081 POINT(40.98076863330354 74.7090863899651) bank171081 +171082 POINT(41.37057161288721 73.22571985289855) bank171082 +171083 POINT(40.20255788681593 73.887290058914) bank171083 +171084 POINT(40.54618394195851 73.37413081060161) bank171084 +171085 POINT(41.176995759810794 73.65749095570708) bank171085 +171086 POINT(40.45120128707055 73.8087934231102) bank171086 +171087 POINT(39.88042104847484 74.71826397279328) bank171087 +171088 POINT(40.55415845772478 74.09240897169498) bank171088 +171089 POINT(41.30954420629293 74.8679923811233) bank171089 +171090 POINT(40.02884770128678 73.65863152653597) bank171090 +171091 POINT(40.718977237853 74.83444763144612) bank171091 +171092 POINT(41.64132125839599 73.444527719961) bank171092 +171093 POINT(40.35704866896616 73.52780829100084) bank171093 +171094 POINT(41.1980664350004 73.62943005252016) bank171094 +171095 POINT(40.56601660758184 74.29211885086892) bank171095 +171096 POINT(40.94398006231902 74.60855157567843) bank171096 +171097 POINT(40.60573363870475 73.21200427734055) bank171097 +171098 POINT(41.35372839750547 73.44669491601624) bank171098 +171099 POINT(40.60633295001259 73.41032511330211) bank171099 +171100 POINT(41.252271740036754 74.06611374276545) bank171100 +171101 POINT(40.64993439727266 73.33302260655033) bank171101 +171102 POINT(40.511341803232845 74.34022209721388) bank171102 +171103 POINT(40.27807910069753 74.43110448105757) bank171103 +171104 POINT(41.16184111913706 73.25442395845691) bank171104 +171105 POINT(41.55066462754689 73.5060604109033) bank171105 +171106 POINT(39.99772491785093 73.0355882494541) bank171106 +171107 POINT(41.12668513474312 74.32368234229554) bank171107 +171108 POINT(39.72383758967833 73.42863990409052) bank171108 +171109 POINT(40.64433822858584 74.86464418372323) bank171109 +171110 POINT(39.860692154857276 74.59214633831728) bank171110 +171111 POINT(40.606799918346724 73.2204948193683) bank171111 +171112 POINT(40.62111117512353 74.14778401593325) bank171112 +171113 POINT(40.29091791346738 74.64415941797714) bank171113 +171114 POINT(40.582844396120784 74.35359207467093) bank171114 +171115 POINT(39.772097378713006 73.63279199613802) bank171115 +171116 POINT(40.181577282946215 74.53583386944204) bank171116 +171117 POINT(41.64050978767817 74.38818098410808) bank171117 +171118 POINT(40.268242526353816 74.42187658232551) bank171118 +171119 POINT(40.293486792217806 74.51810607793296) bank171119 +171120 POINT(40.79285360489909 73.69734088683165) bank171120 +171121 POINT(41.334092996934494 73.03223645096037) bank171121 +171122 POINT(40.7892258602475 73.76722053367527) bank171122 +171123 POINT(40.69292316989972 73.41233377573994) bank171123 +171124 POINT(40.07331620436424 73.36239471285556) bank171124 +171125 POINT(41.33117777103676 74.63397579815619) bank171125 +171126 POINT(40.19982673397859 74.66657509772496) bank171126 +171127 POINT(41.38979334448859 74.5619327877503) bank171127 +171128 POINT(41.418436599955285 74.33005499739407) bank171128 +171129 POINT(40.27948720414488 74.98248012269103) bank171129 +171130 POINT(40.5353790529077 73.9445475742799) bank171130 +171131 POINT(41.502463781418605 74.70780069539815) bank171131 +171132 POINT(41.70444274446032 74.01224377209611) bank171132 +171133 POINT(40.52800738777067 73.04906180441596) bank171133 +171134 POINT(40.50143737685371 74.26367386508691) bank171134 +171135 POINT(40.385246123437156 73.68086840018694) bank171135 +171136 POINT(40.22571930455324 73.9430260488905) bank171136 +171137 POINT(39.866658370545125 74.1683660418394) bank171137 +171138 POINT(40.08206537007085 74.0031069419668) bank171138 +171139 POINT(40.13557667725388 73.88883433776374) bank171139 +171140 POINT(40.86056872653411 74.84581850986679) bank171140 +171141 POINT(40.82245401489335 73.7802194500494) bank171141 +171142 POINT(40.68633229452653 74.94923788316319) bank171142 +171143 POINT(41.50928955898476 74.8925231621698) bank171143 +171144 POINT(40.4198796772022 74.40615857431423) bank171144 +171145 POINT(39.79581551143225 74.09570206752144) bank171145 +171146 POINT(41.31788455411026 73.8319714163969) bank171146 +171147 POINT(41.21737349885255 73.6807227643857) bank171147 +171148 POINT(40.901743999068884 73.14567880916768) bank171148 +171149 POINT(40.02648435501524 73.79301405257571) bank171149 +171150 POINT(39.74083673503613 73.28894674306788) bank171150 +171151 POINT(41.29122625041494 73.53257473117492) bank171151 +171152 POINT(41.43172164638489 73.64033396828378) bank171152 +171153 POINT(40.44942880033487 74.07032343246169) bank171153 +171154 POINT(39.93197026803438 73.28892914728097) bank171154 +171155 POINT(40.41531274751222 74.79955295642746) bank171155 +171156 POINT(41.374100881312415 73.28653699466402) bank171156 +171157 POINT(41.507362174609106 74.73069182222324) bank171157 +171158 POINT(40.35295851819447 74.53804554862673) bank171158 +171159 POINT(39.94716125037403 74.13427592645328) bank171159 +171160 POINT(41.590145270630174 73.53569853191875) bank171160 +171161 POINT(39.83310027740515 74.01860350831421) bank171161 +171162 POINT(41.26454275209268 73.57772081087153) bank171162 +171163 POINT(40.565543792508585 74.4588932811234) bank171163 +171164 POINT(40.962964243291275 73.73600701647314) bank171164 +171165 POINT(41.29549699557187 73.21552052159328) bank171165 +171166 POINT(41.4215991216439 74.89817907866477) bank171166 +171167 POINT(41.19186028255029 74.70242867509621) bank171167 +171168 POINT(40.15263723556792 73.97655912493852) bank171168 +171169 POINT(40.286876879318726 75.00202495960214) bank171169 +171170 POINT(40.32581383254337 74.38690236420393) bank171170 +171171 POINT(40.22332206756616 73.52625001450174) bank171171 +171172 POINT(39.99973632790422 74.92268806288506) bank171172 +171173 POINT(41.708200903706825 73.54121263701515) bank171173 +171174 POINT(40.28107742418349 74.56886393705447) bank171174 +171175 POINT(40.613627058812064 73.79636374182424) bank171175 +171176 POINT(41.436839471196386 73.71035479071972) bank171176 +171177 POINT(41.13847938556935 74.38426233415622) bank171177 +171178 POINT(39.88692088676308 74.86512388224565) bank171178 +171179 POINT(39.758606377914234 73.3659299160226) bank171179 +171180 POINT(40.27426619051571 73.57098186189724) bank171180 +171181 POINT(40.26106182779762 74.66116055856997) bank171181 +171182 POINT(41.20926559866584 73.10548227719647) bank171182 +171183 POINT(39.77885914607197 73.31541358005822) bank171183 +171184 POINT(41.62326064838705 73.51204545513058) bank171184 +171185 POINT(40.26822319881287 74.9081682110524) bank171185 +171186 POINT(41.37626204894017 74.63565428517948) bank171186 +171187 POINT(40.07296054472378 73.65880058146324) bank171187 +171188 POINT(41.37137600653516 74.80876642271197) bank171188 +171189 POINT(39.73019135427269 74.50025128050956) bank171189 +171190 POINT(40.641938690736644 74.80619407401068) bank171190 +171191 POINT(40.880674495769746 74.30784316443493) bank171191 +171192 POINT(40.13480770448591 73.49860779373145) bank171192 +171193 POINT(41.6581212666189 73.8617873785331) bank171193 +171194 POINT(39.831394273260294 73.33567566798907) bank171194 +171195 POINT(41.186620724068305 74.52857756640277) bank171195 +171196 POINT(40.13761150098778 74.3610417038607) bank171196 +171197 POINT(41.58905491492935 73.40986949431381) bank171197 +171198 POINT(40.11500933147054 74.47577255739162) bank171198 +171199 POINT(41.293330362125154 73.4208747201483) bank171199 +171200 POINT(40.6633269548699 73.62166718846082) bank171200 +171201 POINT(40.84574981720614 73.63002545804773) bank171201 +171202 POINT(41.30876761127538 73.69294453546338) bank171202 +171203 POINT(41.54111052286602 73.57122150704366) bank171203 +171204 POINT(41.264516161108084 74.27691151373271) bank171204 +171205 POINT(41.3628075241116 73.16315479405938) bank171205 +171206 POINT(40.87286474442945 73.44129830034078) bank171206 +171207 POINT(41.36482890127725 73.356184504896) bank171207 +171208 POINT(40.721249521792274 73.34407161270352) bank171208 +171209 POINT(41.695862465398726 74.54001265744807) bank171209 +171210 POINT(40.475513182905495 74.4654001561067) bank171210 +171211 POINT(40.92737602969836 74.93313238641355) bank171211 +171212 POINT(41.576227876870696 74.91981880107463) bank171212 +171213 POINT(40.8293354639697 73.83185452785406) bank171213 +171214 POINT(40.888573029733905 73.82027817030487) bank171214 +171215 POINT(40.13106751846348 73.35441783388329) bank171215 +171216 POINT(41.63335478692515 74.31548426301465) bank171216 +171217 POINT(41.04492050539219 73.55864546053508) bank171217 +171218 POINT(41.535689300138486 74.86761366547077) bank171218 +171219 POINT(41.44918499680383 73.19760876880041) bank171219 +171220 POINT(39.93130606661877 73.41428259418957) bank171220 +171221 POINT(41.10582909112354 74.24595707804424) bank171221 +171222 POINT(41.48308074632983 73.25804158212681) bank171222 +171223 POINT(41.070488557970364 74.45544261285713) bank171223 +171224 POINT(40.72161124273343 73.9148721500909) bank171224 +171225 POINT(40.18520675945593 73.77593314786897) bank171225 +171226 POINT(41.2011626232154 73.09279580416747) bank171226 +171227 POINT(40.978214287280494 74.62931738125573) bank171227 +171228 POINT(40.786568693699174 74.19719455814902) bank171228 +171229 POINT(40.51587419820934 73.13246234683439) bank171229 +171230 POINT(40.620481297827524 73.22714155947911) bank171230 +171231 POINT(39.71512181477977 73.58757742495813) bank171231 +171232 POINT(40.0148599243545 73.0748375155422) bank171232 +171233 POINT(41.39412532242675 73.95250545396125) bank171233 +171234 POINT(41.550437240475155 73.5261818953957) bank171234 +171235 POINT(40.37873880165864 73.48498064336268) bank171235 +171236 POINT(40.564119987577236 73.87631343512722) bank171236 +171237 POINT(40.313477348762206 74.07766756954962) bank171237 +171238 POINT(39.82616970866513 73.70408617703514) bank171238 +171239 POINT(40.49777309253049 73.9716455688262) bank171239 +171240 POINT(41.21572229208858 73.20096634520387) bank171240 +171241 POINT(41.039550920842174 74.57111448541767) bank171241 +171242 POINT(40.214690272971595 74.24784481355495) bank171242 +171243 POINT(40.27347839838187 74.98498257122705) bank171243 +171244 POINT(40.26819911572545 73.44492684816048) bank171244 +171245 POINT(40.56668279073014 74.8844455940245) bank171245 +171246 POINT(40.53172285977124 73.93738845218734) bank171246 +171247 POINT(40.602548379149155 74.39613106460071) bank171247 +171248 POINT(40.40164027650838 73.20681915132225) bank171248 +171249 POINT(40.56145497829017 73.42890878901915) bank171249 +171250 POINT(41.4186146584421 73.33964640410633) bank171250 +171251 POINT(40.96466582728779 73.07930579454786) bank171251 +171252 POINT(41.184805590541686 74.23238600778346) bank171252 +171253 POINT(40.91229548788337 74.21557942676354) bank171253 +171254 POINT(40.31022290627159 73.43653682425338) bank171254 +171255 POINT(40.19916248452436 74.8019812908334) bank171255 +171256 POINT(41.255896985576946 73.0268335883047) bank171256 +171257 POINT(39.71304580817694 73.24549134596334) bank171257 +171258 POINT(40.84853168980905 74.15568025114445) bank171258 +171259 POINT(41.55305363505823 73.05242523569201) bank171259 +171260 POINT(41.14532713071796 74.61426550528041) bank171260 +171261 POINT(40.46002267288167 74.91202934373835) bank171261 +171262 POINT(41.547817515052245 73.67635227286414) bank171262 +171263 POINT(40.693542347800246 74.67966493472439) bank171263 +171264 POINT(39.86647484144648 74.2843453079502) bank171264 +171265 POINT(40.00325483810072 74.2999584675693) bank171265 +171266 POINT(41.496951314271705 73.57745085895765) bank171266 +171267 POINT(40.60869091578182 73.6003433587746) bank171267 +171268 POINT(41.678343719801134 74.83383419168592) bank171268 +171269 POINT(40.147023667276706 74.17738979604778) bank171269 +171270 POINT(39.954270409039644 73.77902322385421) bank171270 +171271 POINT(40.67862501757622 73.99890717454343) bank171271 +171272 POINT(40.01227773165513 74.7867926936573) bank171272 +171273 POINT(41.108361855748214 74.24513364046585) bank171273 +171274 POINT(41.45298763617198 73.11428096302267) bank171274 +171275 POINT(40.570539264109485 74.77891045000024) bank171275 +171276 POINT(40.658956617147794 73.50316463105814) bank171276 +171277 POINT(40.46852578812371 74.27372372987851) bank171277 +171278 POINT(41.26930779711661 74.15139302600927) bank171278 +171279 POINT(39.817461035409686 73.58994257293462) bank171279 +171280 POINT(40.86019266978956 73.87942287576429) bank171280 +171281 POINT(40.14593499766913 74.06706395033179) bank171281 +171282 POINT(40.717526505044844 73.3326213741587) bank171282 +171283 POINT(40.64450980645067 74.50623386560632) bank171283 +171284 POINT(40.27178594408619 74.09435828705175) bank171284 +171285 POINT(40.33344536423325 73.9127597943916) bank171285 +171286 POINT(40.466783757697435 73.98406575799493) bank171286 +171287 POINT(39.872084584640966 74.84006176898926) bank171287 +171288 POINT(39.78993562623155 74.00116263479923) bank171288 +171289 POINT(40.940598338103975 74.68130385670482) bank171289 +171290 POINT(40.489344754308135 74.70515279966234) bank171290 +171291 POINT(40.67254638659934 73.90369443937081) bank171291 +171292 POINT(41.01113593159024 74.60711033135182) bank171292 +171293 POINT(41.46919642991003 73.05942699591083) bank171293 +171294 POINT(41.29563435280035 74.9457528001366) bank171294 +171295 POINT(41.055470437176346 74.01665073334522) bank171295 +171296 POINT(41.316259253170344 73.80743779371407) bank171296 +171297 POINT(40.342918327053454 73.50269002526505) bank171297 +171298 POINT(40.29237897902577 74.95246954500875) bank171298 +171299 POINT(40.99585713642619 73.36100643754692) bank171299 +171300 POINT(40.60992777010302 74.25942887011887) bank171300 +171301 POINT(41.02911829961655 73.84442720127146) bank171301 +171302 POINT(39.994792407512094 74.49631223704422) bank171302 +171303 POINT(40.19752155721254 73.35115368528888) bank171303 +171304 POINT(41.48920262761416 74.52214325463123) bank171304 +171305 POINT(41.54243318556935 74.11083506155319) bank171305 +171306 POINT(39.975117857273766 74.8235164224716) bank171306 +171307 POINT(40.83763921192096 74.45891775282456) bank171307 +171308 POINT(40.58420805058097 73.52145555356097) bank171308 +171309 POINT(41.260702658280735 74.92353396071636) bank171309 +171310 POINT(41.12436644522354 73.31303289236006) bank171310 +171311 POINT(40.72540514212115 73.15084908400209) bank171311 +171312 POINT(40.83938024688405 74.23557672125897) bank171312 +171313 POINT(39.927126479520446 73.63415100095361) bank171313 +171314 POINT(41.14404424447918 73.38459738153432) bank171314 +171315 POINT(39.715691493542636 73.39956908853719) bank171315 +171316 POINT(40.20331747903233 73.38739675949748) bank171316 +171317 POINT(41.17658577241955 74.86169633101161) bank171317 +171318 POINT(40.67199535047239 74.92162536840719) bank171318 +171319 POINT(39.98782404911189 73.09090688369173) bank171319 +171320 POINT(41.59735294435592 74.62594319510262) bank171320 +171321 POINT(40.10908496623358 73.17838849458795) bank171321 +171322 POINT(41.297040521409805 73.68013837826197) bank171322 +171323 POINT(40.89363233351825 74.90054489958874) bank171323 +171324 POINT(40.99762906192033 73.70185053751412) bank171324 +171325 POINT(40.322811266192964 74.40555553653465) bank171325 +171326 POINT(41.08766180868162 74.60954395202681) bank171326 +171327 POINT(40.38425927655842 73.35294605438341) bank171327 +171328 POINT(39.9610202211836 74.55874639375533) bank171328 +171329 POINT(40.84471847742292 74.88337857428253) bank171329 +171330 POINT(41.24335059243862 73.74915176371414) bank171330 +171331 POINT(40.94565191640608 73.45088667827794) bank171331 +171332 POINT(40.7728147513673 73.84249508037936) bank171332 +171333 POINT(41.35345285272351 74.38022361663761) bank171333 +171334 POINT(41.2861824761267 73.76968301476848) bank171334 +171335 POINT(40.14299545234521 73.17912804245907) bank171335 +171336 POINT(41.605445418489715 74.71776332358064) bank171336 +171337 POINT(41.331503882504634 73.1724867070122) bank171337 +171338 POINT(40.441524937107474 74.30443914376998) bank171338 +171339 POINT(41.36823392241669 73.83699232727538) bank171339 +171340 POINT(39.77411949782169 74.08753350359865) bank171340 +171341 POINT(39.80352142940183 73.05605091290866) bank171341 +171342 POINT(39.931646722598344 73.65166823497569) bank171342 +171343 POINT(40.28269918526934 73.13731368595343) bank171343 +171344 POINT(41.03040577603941 74.31430087629826) bank171344 +171345 POINT(39.92168731308876 73.41360372517974) bank171345 +171346 POINT(41.27262988214843 73.99176680664871) bank171346 +171347 POINT(41.11723487653727 73.49453103633974) bank171347 +171348 POINT(40.09101489497023 74.34034253534203) bank171348 +171349 POINT(41.08013999974109 73.91250997344731) bank171349 +171350 POINT(40.79299570022234 74.56280698140463) bank171350 +171351 POINT(39.85233179951569 73.93331657233269) bank171351 +171352 POINT(40.76877752941066 74.6920934889434) bank171352 +171353 POINT(39.993382928621635 74.14179286715304) bank171353 +171354 POINT(40.74100657252382 74.1394367583433) bank171354 +171355 POINT(40.131765117875446 74.54794632196558) bank171355 +171356 POINT(41.04672171491089 74.46300615126374) bank171356 +171357 POINT(41.51948577089755 74.03818438591782) bank171357 +171358 POINT(40.088190994768205 74.80845494839066) bank171358 +171359 POINT(39.77722495495222 74.52702983412048) bank171359 +171360 POINT(39.98756598501849 73.88372327583014) bank171360 +171361 POINT(40.458918357862075 73.25871425897212) bank171361 +171362 POINT(41.15942397944501 74.25598194904839) bank171362 +171363 POINT(41.115845271191624 73.51149337526685) bank171363 +171364 POINT(41.08057645016652 73.80674340548346) bank171364 +171365 POINT(40.72415133986973 73.99665202097162) bank171365 +171366 POINT(41.36364290712131 74.99181759336146) bank171366 +171367 POINT(41.36210475808447 74.43636899166653) bank171367 +171368 POINT(40.77651251290684 73.66207427059852) bank171368 +171369 POINT(40.33028761533808 74.67683911105058) bank171369 +171370 POINT(41.3598440742104 74.38890163724747) bank171370 +171371 POINT(41.0526485972709 73.76676560703541) bank171371 +171372 POINT(40.951882200511214 74.48346561861727) bank171372 +171373 POINT(41.10567393359321 73.65841939193658) bank171373 +171374 POINT(40.439122832810455 74.40009830880872) bank171374 +171375 POINT(40.339808912456974 73.56067723183945) bank171375 +171376 POINT(40.159406837949 74.29369040591662) bank171376 +171377 POINT(41.210866363922804 74.53050350673159) bank171377 +171378 POINT(40.561306585624735 74.82668420549876) bank171378 +171379 POINT(39.951062242066186 74.12106016358996) bank171379 +171380 POINT(40.02114612813755 74.44350325949547) bank171380 +171381 POINT(41.01697114722981 74.63649656100111) bank171381 +171382 POINT(39.98276114588038 73.55358897224774) bank171382 +171383 POINT(40.13974237474291 73.15354651835392) bank171383 +171384 POINT(41.021340668732975 74.00635066063825) bank171384 +171385 POINT(41.21198996450762 74.76696758660961) bank171385 +171386 POINT(39.859631082025075 73.68321653065817) bank171386 +171387 POINT(39.750699168837734 73.484267905714) bank171387 +171388 POINT(40.29158720391296 73.48672647230204) bank171388 +171389 POINT(40.29245360959321 73.42905598320858) bank171389 +171390 POINT(41.03737388136275 74.83953939420248) bank171390 +171391 POINT(41.06539742048917 74.62061755992048) bank171391 +171392 POINT(40.805983467743204 74.86036362254409) bank171392 +171393 POINT(40.52694126775657 74.7473985336976) bank171393 +171394 POINT(40.776469054757555 74.01136602058601) bank171394 +171395 POINT(39.82690620560134 73.10718450594293) bank171395 +171396 POINT(39.7636533414253 74.43426377676523) bank171396 +171397 POINT(40.304581880462315 74.30117433183283) bank171397 +171398 POINT(40.646685997051065 73.09836031893443) bank171398 +171399 POINT(39.99787193661338 74.83926841754045) bank171399 +171400 POINT(40.259090102850436 74.34966733723978) bank171400 +171401 POINT(41.05834776817581 74.66010579023141) bank171401 +171402 POINT(41.53232859565198 73.21917597874732) bank171402 +171403 POINT(40.37718430118735 74.71920601031636) bank171403 +171404 POINT(40.77965316530741 74.21769729431742) bank171404 +171405 POINT(41.30724137878469 73.09626943952375) bank171405 +171406 POINT(41.352451742686796 74.66252051099079) bank171406 +171407 POINT(41.29672330023431 74.30972089479229) bank171407 +171408 POINT(40.525671543609086 73.45833162315768) bank171408 +171409 POINT(40.876650836609734 73.14591182225716) bank171409 +171410 POINT(40.90574866477462 73.40635154609987) bank171410 +171411 POINT(41.25664658852431 74.0224598817231) bank171411 +171412 POINT(41.55362954715972 73.8824127296746) bank171412 +171413 POINT(40.617858913873754 74.53738764114945) bank171413 +171414 POINT(40.87230715828311 74.73213851730384) bank171414 +171415 POINT(39.8462728236399 73.4549238724941) bank171415 +171416 POINT(41.55694163565441 73.40095467655061) bank171416 +171417 POINT(41.249593552655995 74.51926683766754) bank171417 +171418 POINT(41.15056173911176 74.83234602590443) bank171418 +171419 POINT(40.87687120438159 74.3171007416216) bank171419 +171420 POINT(39.725519241977295 73.65216287747242) bank171420 +171421 POINT(40.181054518227356 73.08827916507002) bank171421 +171422 POINT(40.3115098232505 74.06525818082734) bank171422 +171423 POINT(40.28050424854407 74.96129333360277) bank171423 +171424 POINT(41.534876318876194 74.62242103834005) bank171424 +171425 POINT(40.52151354882404 73.02502052454766) bank171425 +171426 POINT(40.001236917598575 74.64152078470725) bank171426 +171427 POINT(40.61853505034766 74.09051487330936) bank171427 +171428 POINT(39.84520330170789 74.42732905706343) bank171428 +171429 POINT(40.273878051021924 73.27330029275049) bank171429 +171430 POINT(40.31298495751596 74.2312791342213) bank171430 +171431 POINT(40.3095760068125 74.82933748132962) bank171431 +171432 POINT(39.94766140708496 74.98751368421374) bank171432 +171433 POINT(41.1474912689018 73.31153988195508) bank171433 +171434 POINT(41.479132835168286 73.33015106318022) bank171434 +171435 POINT(39.89256842445642 73.85717597348761) bank171435 +171436 POINT(41.23967370067979 74.83872143111172) bank171436 +171437 POINT(41.13476926534475 73.85806781083014) bank171437 +171438 POINT(39.95869875362455 73.96098445512847) bank171438 +171439 POINT(39.846916554214175 74.82515200855579) bank171439 +171440 POINT(39.90182218860873 73.72286404079662) bank171440 +171441 POINT(41.05333256967849 74.53433112275653) bank171441 +171442 POINT(41.45530784687945 73.90011792093921) bank171442 +171443 POINT(40.47108987036018 74.20852313764054) bank171443 +171444 POINT(41.47568750497936 73.33015963171725) bank171444 +171445 POINT(40.84561678808913 74.44047976303005) bank171445 +171446 POINT(39.86875548628183 74.71480377574397) bank171446 +171447 POINT(41.62289768118254 74.46500286782961) bank171447 +171448 POINT(40.427988209243296 74.24643008318263) bank171448 +171449 POINT(39.836391795952466 73.42847160667782) bank171449 +171450 POINT(39.901457360811314 74.91503831486878) bank171450 +171451 POINT(40.321498177192936 74.3443025393293) bank171451 +171452 POINT(40.49894716779805 74.29839991815686) bank171452 +171453 POINT(41.010474586501296 74.8041142302487) bank171453 +171454 POINT(40.58666373068334 73.37554079469503) bank171454 +171455 POINT(40.55383292664126 74.20164023960106) bank171455 +171456 POINT(41.67793161591836 73.2934729638878) bank171456 +171457 POINT(39.952986963962545 74.87544156822958) bank171457 +171458 POINT(41.088865455682885 73.27774961491777) bank171458 +171459 POINT(40.455667549387506 73.61618184934817) bank171459 +171460 POINT(40.95221864345727 74.94478615567621) bank171460 +171461 POINT(40.55539226361865 73.34055398919187) bank171461 +171462 POINT(39.79862760688976 73.6462323639117) bank171462 +171463 POINT(40.94906282831397 74.21142096018487) bank171463 +171464 POINT(40.328933411645345 74.0634790822674) bank171464 +171465 POINT(41.18675428332822 74.13922288713725) bank171465 +171466 POINT(40.554469457857216 73.20589142091147) bank171466 +171467 POINT(41.690529175942444 74.72388468100677) bank171467 +171468 POINT(40.09963125664847 74.28384295085421) bank171468 +171469 POINT(41.28508314217105 74.20475488797373) bank171469 +171470 POINT(41.4265432886818 74.15378949284113) bank171470 +171471 POINT(41.39244182453899 74.8625906231032) bank171471 +171472 POINT(41.35600618960579 74.15974226852778) bank171472 +171473 POINT(40.941754476976804 73.58987495002528) bank171473 +171474 POINT(41.16630205004696 73.83672637342713) bank171474 +171475 POINT(41.29239934409207 73.18112781418469) bank171475 +171476 POINT(40.829767126596195 74.05838578924418) bank171476 +171477 POINT(41.51539718052027 73.84473979546819) bank171477 +171478 POINT(39.86166157049695 74.82491759849249) bank171478 +171479 POINT(40.141414812821914 74.74331909132087) bank171479 +171480 POINT(41.616305679404675 74.85864537825388) bank171480 +171481 POINT(41.165672336976655 73.96428562369549) bank171481 +171482 POINT(41.63292892677276 73.23281981723511) bank171482 +171483 POINT(40.17963626158248 73.70799998264586) bank171483 +171484 POINT(40.95920557137715 74.78048867604655) bank171484 +171485 POINT(41.33446827676083 74.4000045532911) bank171485 +171486 POINT(40.566822243453444 74.219118257067) bank171486 +171487 POINT(40.276646381911775 74.00555609858236) bank171487 +171488 POINT(39.859062519455314 73.88663685631776) bank171488 +171489 POINT(41.06013446064992 74.49528469198921) bank171489 +171490 POINT(41.074994114144005 74.98789728209431) bank171490 +171491 POINT(41.42301183218195 74.45016490021115) bank171491 +171492 POINT(40.70428810085224 74.0915186456126) bank171492 +171493 POINT(40.65766717555464 74.13995948408161) bank171493 +171494 POINT(40.8803715885532 74.99328022939082) bank171494 +171495 POINT(40.30704877221593 73.47281611309194) bank171495 +171496 POINT(40.443594821100476 74.43794239969664) bank171496 +171497 POINT(40.943693834101346 73.41886977181312) bank171497 +171498 POINT(41.166757477463214 74.6953631230552) bank171498 +171499 POINT(40.69304442282501 74.4861554324967) bank171499 +171500 POINT(40.769048756773955 74.39269537856467) bank171500 +171501 POINT(40.78633121002145 73.5796304497913) bank171501 +171502 POINT(40.178605080983 74.91451804197756) bank171502 +171503 POINT(40.968235994598146 73.40289703122883) bank171503 +171504 POINT(41.37171505971223 73.13110839771764) bank171504 +171505 POINT(41.51113391469525 74.23499422587264) bank171505 +171506 POINT(40.917883314116324 73.48170089557134) bank171506 +171507 POINT(41.327228523183436 73.01913422842466) bank171507 +171508 POINT(41.64720433352649 73.29292226927926) bank171508 +171509 POINT(39.76828007672591 73.28693428336713) bank171509 +171510 POINT(40.451692419428035 74.41445575318733) bank171510 +171511 POINT(40.41932853746949 74.68723995431122) bank171511 +171512 POINT(41.623193602202136 73.70073047698182) bank171512 +171513 POINT(41.62488058164853 74.15750793156596) bank171513 +171514 POINT(41.58254906842793 74.96930027665523) bank171514 +171515 POINT(40.35989583650514 73.87360864294898) bank171515 +171516 POINT(41.59293176402571 74.59449020781952) bank171516 +171517 POINT(40.92396282773924 73.89906834307737) bank171517 +171518 POINT(39.958173494412144 74.70089482396102) bank171518 +171519 POINT(41.60968781928515 73.01004432577828) bank171519 +171520 POINT(41.07815304343964 74.41092454208426) bank171520 +171521 POINT(40.58641435987517 74.8553869529213) bank171521 +171522 POINT(40.09890499602349 73.86646270904414) bank171522 +171523 POINT(39.966369111292934 74.98887906771392) bank171523 +171524 POINT(39.93931344628453 74.10596470737329) bank171524 +171525 POINT(39.93991418877252 74.23581792915607) bank171525 +171526 POINT(40.362087481960124 74.59848309565493) bank171526 +171527 POINT(40.098084444006716 74.31716756848637) bank171527 +171528 POINT(41.2202561993797 74.46745259371902) bank171528 +171529 POINT(40.43443721062836 73.75570192839298) bank171529 +171530 POINT(41.61600528750356 73.80344608212371) bank171530 +171531 POINT(41.329477676825235 73.87839553095944) bank171531 +171532 POINT(40.4538642241508 73.51167974794042) bank171532 +171533 POINT(40.914142178123974 74.98090938782232) bank171533 +171534 POINT(40.407818084070584 74.30613439751819) bank171534 +171535 POINT(41.6458951484032 74.20826573496501) bank171535 +171536 POINT(40.62116383919734 73.23504719198841) bank171536 +171537 POINT(40.58802691842801 74.10663642302801) bank171537 +171538 POINT(41.70519866536384 74.16581374829259) bank171538 +171539 POINT(41.58900143928582 73.78861762181707) bank171539 +171540 POINT(40.0200513451029 74.23360327726039) bank171540 +171541 POINT(40.79761143383965 73.5029699439104) bank171541 +171542 POINT(41.615403210357826 74.19937503881894) bank171542 +171543 POINT(39.73878514702466 73.23850331460466) bank171543 +171544 POINT(40.206709177812776 73.28214910572615) bank171544 +171545 POINT(41.26777160865763 74.50899652530522) bank171545 +171546 POINT(41.104261041932396 73.32403358406306) bank171546 +171547 POINT(41.234568397699555 74.8796247762297) bank171547 +171548 POINT(40.9033505150759 74.66630649994774) bank171548 +171549 POINT(41.21270603791109 74.7760391256379) bank171549 +171550 POINT(41.289777675740176 73.4477216833269) bank171550 +171551 POINT(41.30783789368391 74.1822634222025) bank171551 +171552 POINT(39.89674825957688 73.29961274300398) bank171552 +171553 POINT(40.65804795559753 73.65543885917836) bank171553 +171554 POINT(41.16731971811187 74.39120199785567) bank171554 +171555 POINT(40.09727956815972 73.79497878137838) bank171555 +171556 POINT(40.478862631141304 74.95727202935468) bank171556 +171557 POINT(41.27145173256862 73.61248798516314) bank171557 +171558 POINT(41.28538019352428 74.83735655012647) bank171558 +171559 POINT(39.74645479009041 73.27148511875386) bank171559 +171560 POINT(40.00384539722708 73.45302183247586) bank171560 +171561 POINT(40.727521155907965 73.38565406737607) bank171561 +171562 POINT(40.53370405868964 73.48478509069885) bank171562 +171563 POINT(40.00854100831861 74.9559655940482) bank171563 +171564 POINT(41.56083852943774 73.08826923944144) bank171564 +171565 POINT(39.73296742048618 74.54537569752966) bank171565 +171566 POINT(40.075334747833054 74.73158260038308) bank171566 +171567 POINT(41.52541220068733 74.05525995688431) bank171567 +171568 POINT(40.84066372153326 73.33344335862927) bank171568 +171569 POINT(40.54025062911391 73.56022007391068) bank171569 +171570 POINT(40.5564756706418 73.70434002507388) bank171570 +171571 POINT(40.3351881707225 73.05773011436939) bank171571 +171572 POINT(41.27153746720174 74.14474094402003) bank171572 +171573 POINT(40.97827791923785 73.65746161619202) bank171573 +171574 POINT(41.67011842514807 73.14250413647116) bank171574 +171575 POINT(41.472771169475415 73.22940506727703) bank171575 +171576 POINT(41.63567868655318 73.9170665489717) bank171576 +171577 POINT(40.6466474620908 73.38588232190524) bank171577 +171578 POINT(40.4581943491304 73.66273244564448) bank171578 +171579 POINT(40.72273988614671 73.56740002228344) bank171579 +171580 POINT(40.63337331440134 73.59382936729014) bank171580 +171581 POINT(41.35408078246507 73.67697779388479) bank171581 +171582 POINT(40.21047934902252 73.02070603997824) bank171582 +171583 POINT(39.76838379901593 74.6465220183563) bank171583 +171584 POINT(41.45742318375044 73.7352802361385) bank171584 +171585 POINT(41.625770525200075 73.5805143880655) bank171585 +171586 POINT(39.810186397609264 73.86607724063927) bank171586 +171587 POINT(39.97830661296627 73.49094145045585) bank171587 +171588 POINT(40.49394177365903 74.55243909303638) bank171588 +171589 POINT(41.667177900933076 74.32629546058683) bank171589 +171590 POINT(41.56475405210059 73.9099176587756) bank171590 +171591 POINT(41.40883804527411 74.96647789146341) bank171591 +171592 POINT(39.85994497325239 73.02089855646476) bank171592 +171593 POINT(40.382240241783975 74.94900031146557) bank171593 +171594 POINT(39.728453401490185 74.5841906162106) bank171594 +171595 POINT(41.15323277908248 73.79513941649145) bank171595 +171596 POINT(41.07729501503299 74.43834205388934) bank171596 +171597 POINT(41.48902946972884 73.55918435886784) bank171597 +171598 POINT(39.80072964304971 74.12999317604802) bank171598 +171599 POINT(39.767194737682985 73.155579813579) bank171599 +171600 POINT(41.70302294305895 74.36183925665316) bank171600 +171601 POINT(41.14358753331651 74.30410018102666) bank171601 +171602 POINT(40.5761382965241 74.39066078580062) bank171602 +171603 POINT(40.844988936756295 73.72095740005352) bank171603 +171604 POINT(40.56456648014085 74.03568035622754) bank171604 +171605 POINT(40.121178203845744 73.25417166085383) bank171605 +171606 POINT(41.37961487430387 74.31973418541781) bank171606 +171607 POINT(40.31239844708084 73.47325040383951) bank171607 +171608 POINT(41.13614853991608 74.25713056087672) bank171608 +171609 POINT(41.318045929097586 73.24454170374582) bank171609 +171610 POINT(40.50371596532705 73.83181343524254) bank171610 +171611 POINT(40.537828577482145 74.4702965846513) bank171611 +171612 POINT(40.824296498909945 73.02259805792613) bank171612 +171613 POINT(39.72739294609233 73.04617221537852) bank171613 +171614 POINT(40.216865923954636 73.64108751438643) bank171614 +171615 POINT(40.12321830441684 73.16177967116595) bank171615 +171616 POINT(40.84783129064131 73.06935542196963) bank171616 +171617 POINT(40.2816513318765 74.80613117351275) bank171617 +171618 POINT(40.109248410653606 74.06176048755057) bank171618 +171619 POINT(40.656801644349045 73.71977889526445) bank171619 +171620 POINT(39.750696002455406 74.96218251765372) bank171620 +171621 POINT(39.91794685360749 74.52265567046787) bank171621 +171622 POINT(39.92455207712766 73.43351804534558) bank171622 +171623 POINT(39.978339189666805 74.85049118813839) bank171623 +171624 POINT(41.10254600805197 73.88860925231735) bank171624 +171625 POINT(41.18969879268088 73.84697908504944) bank171625 +171626 POINT(40.42641694585557 74.93040415861951) bank171626 +171627 POINT(39.86101290180994 73.83328934314036) bank171627 +171628 POINT(40.70748673049396 73.62311220132649) bank171628 +171629 POINT(41.69978228572705 73.81971782589875) bank171629 +171630 POINT(39.95012371138572 74.01093190620041) bank171630 +171631 POINT(40.82241842052006 73.95601715887001) bank171631 +171632 POINT(39.732352031221545 74.42294409555453) bank171632 +171633 POINT(41.35762930351655 73.55044302009208) bank171633 +171634 POINT(41.22527556366892 73.28938415140152) bank171634 +171635 POINT(40.45742498848918 73.57048971014758) bank171635 +171636 POINT(40.74884122335595 73.5685307024299) bank171636 +171637 POINT(40.72657183667764 74.95561645195127) bank171637 +171638 POINT(40.80558240459731 73.66328966073661) bank171638 +171639 POINT(40.99425157060959 74.78890899434495) bank171639 +171640 POINT(40.61417064193003 73.70735961528774) bank171640 +171641 POINT(41.074124873291325 74.36481832594977) bank171641 +171642 POINT(41.62044956047664 73.97533938401324) bank171642 +171643 POINT(41.146004865700874 74.36653013701695) bank171643 +171644 POINT(40.152931476351036 73.29388994616194) bank171644 +171645 POINT(40.70656043413302 73.69418229320334) bank171645 +171646 POINT(40.35184807795667 73.87411841415756) bank171646 +171647 POINT(41.048998627353534 74.00969816122324) bank171647 +171648 POINT(39.73820399775213 74.86594996259406) bank171648 +171649 POINT(41.5152023923618 74.33308990911169) bank171649 +171650 POINT(40.524817921393634 73.08871703993059) bank171650 +171651 POINT(40.31728246834375 74.09985330425462) bank171651 +171652 POINT(41.3018788239179 74.63317908409238) bank171652 +171653 POINT(41.44813490921185 74.75884960343251) bank171653 +171654 POINT(39.861808885347564 74.64078375944318) bank171654 +171655 POINT(40.1830150243797 74.15996261396825) bank171655 +171656 POINT(40.814358138900246 73.18452896841188) bank171656 +171657 POINT(41.441828415065025 73.69530282872726) bank171657 +171658 POINT(40.223927475385544 74.3207174782507) bank171658 +171659 POINT(39.90012697675047 74.33376280379096) bank171659 +171660 POINT(40.23977542728583 73.64058331654238) bank171660 +171661 POINT(40.972407000852165 73.13560791515985) bank171661 +171662 POINT(40.19813748335073 73.2086376768669) bank171662 +171663 POINT(41.69773104402173 73.61303341373693) bank171663 +171664 POINT(40.57635490375315 73.62871402003898) bank171664 +171665 POINT(40.475887962227645 73.34457344758513) bank171665 +171666 POINT(41.3832986645194 73.54101181183506) bank171666 +171667 POINT(39.821987029172824 74.76313437121689) bank171667 +171668 POINT(41.6060738217434 73.62966112964028) bank171668 +171669 POINT(41.493900073801896 73.56643024780776) bank171669 +171670 POINT(41.44204675307327 74.49697423623687) bank171670 +171671 POINT(40.320511112795835 74.80880049928585) bank171671 +171672 POINT(39.9059162355556 74.13887480814273) bank171672 +171673 POINT(40.97187539290277 74.86539471858994) bank171673 +171674 POINT(41.3488838346217 73.91601569873363) bank171674 +171675 POINT(40.876614776184255 74.22586748193282) bank171675 +171676 POINT(41.619404337095155 74.56420494757408) bank171676 +171677 POINT(39.77293317135828 74.15722876808215) bank171677 +171678 POINT(40.66199754926392 73.07366663520565) bank171678 +171679 POINT(41.46363458378151 73.18041321671556) bank171679 +171680 POINT(40.68635116818815 73.91288957373048) bank171680 +171681 POINT(41.68826255715482 73.81043312820128) bank171681 +171682 POINT(40.14346633703411 73.17909215882932) bank171682 +171683 POINT(39.982509725494005 74.8799526068678) bank171683 +171684 POINT(40.31426416895628 74.72747825600366) bank171684 +171685 POINT(40.265469181825125 73.44439386470472) bank171685 +171686 POINT(40.07720475586773 73.92585831653439) bank171686 +171687 POINT(41.147525917669164 74.90285042738542) bank171687 +171688 POINT(40.89101117853521 74.84425771794106) bank171688 +171689 POINT(39.829871893570534 73.37225375158695) bank171689 +171690 POINT(39.87691949209027 73.08882756440096) bank171690 +171691 POINT(40.54590622248963 73.78376901449924) bank171691 +171692 POINT(41.088638141931966 74.29099445799007) bank171692 +171693 POINT(40.756481115634394 73.04136529436342) bank171693 +171694 POINT(40.04563031331502 73.31941035075731) bank171694 +171695 POINT(40.31604642653381 74.86344782144563) bank171695 +171696 POINT(41.435326578689164 73.54836935790895) bank171696 +171697 POINT(40.812187428844936 73.16601577806598) bank171697 +171698 POINT(41.70173500376791 73.7702439529286) bank171698 +171699 POINT(39.72880767882065 74.65569552726396) bank171699 +171700 POINT(39.98500926355988 73.167065159902) bank171700 +171701 POINT(41.06833088435439 74.2825554043327) bank171701 +171702 POINT(40.073871400797614 73.4308507169386) bank171702 +171703 POINT(41.20756093968224 74.06132903352106) bank171703 +171704 POINT(40.7265639729267 74.08358305743461) bank171704 +171705 POINT(41.58732673881543 74.48478535825144) bank171705 +171706 POINT(40.87611314128755 74.98756809681767) bank171706 +171707 POINT(41.5455588408129 73.55818358724513) bank171707 +171708 POINT(40.02785885790157 74.74943721824928) bank171708 +171709 POINT(39.94658027514424 74.90856999638017) bank171709 +171710 POINT(41.595449251483956 73.73604980006866) bank171710 +171711 POINT(41.59826426169437 74.01826356879403) bank171711 +171712 POINT(41.0083710715074 73.20602120035666) bank171712 +171713 POINT(41.41071407104359 74.95133448833978) bank171713 +171714 POINT(41.040093195230185 73.41030735424602) bank171714 +171715 POINT(40.392520693362215 73.96598934928998) bank171715 +171716 POINT(40.12663682351917 73.02368088410697) bank171716 +171717 POINT(39.77218041858035 73.02452908807575) bank171717 +171718 POINT(41.52062807123724 74.58779152138703) bank171718 +171719 POINT(40.59189305660314 74.4451224156139) bank171719 +171720 POINT(41.11550131430506 73.27298784194797) bank171720 +171721 POINT(40.983995481625925 74.21913221133754) bank171721 +171722 POINT(40.03518127101188 73.6558320174698) bank171722 +171723 POINT(40.43077298172286 74.85422222746857) bank171723 +171724 POINT(40.51018924108249 74.57931228430759) bank171724 +171725 POINT(41.686620177896856 74.35493487122073) bank171725 +171726 POINT(39.87563830367031 74.48482809745403) bank171726 +171727 POINT(41.47136520240391 74.63757493092223) bank171727 +171728 POINT(40.11405048128029 73.9687267706861) bank171728 +171729 POINT(39.91709144737603 74.692243366636) bank171729 +171730 POINT(39.782369694662144 74.90083608645638) bank171730 +171731 POINT(41.39310379076112 74.28873121621297) bank171731 +171732 POINT(40.69210331248791 74.64166173352028) bank171732 +171733 POINT(39.98233363815375 74.72993351638925) bank171733 +171734 POINT(41.2348042403201 74.68461480794404) bank171734 +171735 POINT(40.03608614880116 74.87319327650476) bank171735 +171736 POINT(40.34258725926955 73.42793184497576) bank171736 +171737 POINT(39.82996988942709 74.24557329795766) bank171737 +171738 POINT(40.56927256268861 73.66246211137253) bank171738 +171739 POINT(41.18071349180459 73.01207273364498) bank171739 +171740 POINT(39.77704289168861 74.98586671321131) bank171740 +171741 POINT(40.05514486542207 73.07979971351752) bank171741 +171742 POINT(41.278758274472786 73.39764055489138) bank171742 +171743 POINT(41.30041650053736 74.17252205830245) bank171743 +171744 POINT(39.93357693436841 73.32897257629081) bank171744 +171745 POINT(39.956416370187036 73.10601930047268) bank171745 +171746 POINT(41.474826048149296 73.32789252384545) bank171746 +171747 POINT(39.899540667462475 74.34909826433712) bank171747 +171748 POINT(41.11894094286303 73.5343829200151) bank171748 +171749 POINT(40.74235363541286 73.01740234692527) bank171749 +171750 POINT(41.31698316756391 74.99916338713575) bank171750 +171751 POINT(40.99220032371561 73.58545270785427) bank171751 +171752 POINT(41.51271027651004 74.24448628830895) bank171752 +171753 POINT(39.72149707576327 74.1032499881894) bank171753 +171754 POINT(39.850682520381575 73.20040968706383) bank171754 +171755 POINT(41.15623768992396 74.72603394433709) bank171755 +171756 POINT(40.18589967035504 74.41872255055402) bank171756 +171757 POINT(40.562050290503976 73.96450041981281) bank171757 +171758 POINT(40.19154824230787 74.63839905761797) bank171758 +171759 POINT(40.94439133059329 73.98709480448852) bank171759 +171760 POINT(41.29243354655894 74.83935706674242) bank171760 +171761 POINT(40.87513879785401 74.23804567786341) bank171761 +171762 POINT(40.783496598202035 73.85982993796502) bank171762 +171763 POINT(40.07150909643183 73.04054231516471) bank171763 +171764 POINT(40.702474022032376 74.08723037496898) bank171764 +171765 POINT(41.034893434855604 74.27752575994484) bank171765 +171766 POINT(40.683728054030794 73.8923065409872) bank171766 +171767 POINT(39.91341301441081 73.27955819318571) bank171767 +171768 POINT(39.723498223887816 73.9233144716241) bank171768 +171769 POINT(39.86140086122239 73.82256876023206) bank171769 +171770 POINT(40.78277171357424 73.52972591403159) bank171770 +171771 POINT(41.61935945426349 74.76699489359883) bank171771 +171772 POINT(41.68240353816585 73.7481529945835) bank171772 +171773 POINT(40.22337287373019 74.90508152564308) bank171773 +171774 POINT(41.506292631743506 74.08023810292494) bank171774 +171775 POINT(40.04814013085573 73.17993033160631) bank171775 +171776 POINT(39.74615727806099 73.83235371383631) bank171776 +171777 POINT(39.96296888168468 73.8476757233473) bank171777 +171778 POINT(41.60539249680624 73.52739443272466) bank171778 +171779 POINT(40.35482714979835 73.25583698002208) bank171779 +171780 POINT(41.051445160734744 74.12963509449808) bank171780 +171781 POINT(41.342826097198724 73.74933406354704) bank171781 +171782 POINT(41.62593565342602 74.85505170387745) bank171782 +171783 POINT(40.66878032204964 73.18277166238276) bank171783 +171784 POINT(40.83991400009929 74.27829502305828) bank171784 +171785 POINT(41.37033703446153 74.08140589218758) bank171785 +171786 POINT(40.79792237504248 73.50001350063529) bank171786 +171787 POINT(41.40131795806483 73.41736652205037) bank171787 +171788 POINT(40.3677684553575 74.69910898227624) bank171788 +171789 POINT(39.928246734045295 74.20787346618617) bank171789 +171790 POINT(40.85080899990189 74.01107327629794) bank171790 +171791 POINT(40.94244926010184 73.87527286403993) bank171791 +171792 POINT(40.214142807307795 73.90462249380226) bank171792 +171793 POINT(41.04848759920324 73.34431582191944) bank171793 +171794 POINT(41.131640851370825 74.47515820662592) bank171794 +171795 POINT(41.48227870053925 73.241933517528) bank171795 +171796 POINT(40.96751613323911 74.5006860771617) bank171796 +171797 POINT(40.594494912641714 73.29003180697714) bank171797 +171798 POINT(40.65871673101004 74.09310409147103) bank171798 +171799 POINT(40.00767895140817 73.40729640302685) bank171799 +171800 POINT(40.92156997633529 74.6295212430763) bank171800 +171801 POINT(40.58329844446923 73.0880446835516) bank171801 +171802 POINT(40.87818702871635 73.99686724856521) bank171802 +171803 POINT(40.11921019944623 74.54207900795234) bank171803 +171804 POINT(39.96586131442761 74.86801831902172) bank171804 +171805 POINT(40.54827529015062 74.51743748927935) bank171805 +171806 POINT(40.00589179393584 73.54466451736128) bank171806 +171807 POINT(41.204095339562585 74.7351302786842) bank171807 +171808 POINT(40.84335553361765 73.5314296268757) bank171808 +171809 POINT(40.31036525346662 74.18622227626416) bank171809 +171810 POINT(41.70097986569995 74.20653206979536) bank171810 +171811 POINT(40.63488269257657 74.65178771586967) bank171811 +171812 POINT(40.22780685143404 74.10832981286983) bank171812 +171813 POINT(40.04439584889787 74.48042728086014) bank171813 +171814 POINT(40.889687088382786 74.83209728521193) bank171814 +171815 POINT(41.38533386739431 74.54714306147991) bank171815 +171816 POINT(41.03210676195056 73.87520467932003) bank171816 +171817 POINT(41.1800546010626 73.26000937745447) bank171817 +171818 POINT(40.46483325653381 73.50827320107982) bank171818 +171819 POINT(39.884499052214515 74.89432958770458) bank171819 +171820 POINT(41.68990886801301 73.07230831074621) bank171820 +171821 POINT(40.32235215657609 74.53253256960836) bank171821 +171822 POINT(41.46899251780118 73.07964459017603) bank171822 +171823 POINT(40.70301591053908 73.32024486667304) bank171823 +171824 POINT(39.75690459905632 73.89485581448294) bank171824 +171825 POINT(40.44136944330247 74.29903482112363) bank171825 +171826 POINT(41.26549373798183 73.13800440592591) bank171826 +171827 POINT(41.65808202296165 73.30136728261245) bank171827 +171828 POINT(40.370046168852355 73.02584757855165) bank171828 +171829 POINT(41.16840656570212 73.02541167875073) bank171829 +171830 POINT(40.124249346666545 74.89096968461182) bank171830 +171831 POINT(41.23409291097919 74.07950964419442) bank171831 +171832 POINT(40.83138515129091 74.30636994925563) bank171832 +171833 POINT(41.07456308054895 74.68558237697872) bank171833 +171834 POINT(39.804642690194505 74.9057627197082) bank171834 +171835 POINT(41.14679406068575 73.17875350550602) bank171835 +171836 POINT(40.9233919161295 73.40521713344107) bank171836 +171837 POINT(40.59873616816636 73.15150719877447) bank171837 +171838 POINT(40.111182061110384 73.46455056828917) bank171838 +171839 POINT(41.550611940245794 74.41801397939648) bank171839 +171840 POINT(39.92147645613985 74.96249748741367) bank171840 +171841 POINT(40.985877281513 74.000824814748) bank171841 +171842 POINT(40.24485922778134 74.47993958588145) bank171842 +171843 POINT(40.596971981698736 74.57653481845118) bank171843 +171844 POINT(40.46678573903019 74.62259498139244) bank171844 +171845 POINT(40.87824178260581 73.8919748380174) bank171845 +171846 POINT(41.42133941067406 74.89154481723745) bank171846 +171847 POINT(41.5753090586842 74.20789239541982) bank171847 +171848 POINT(40.49958802527326 73.18679278906782) bank171848 +171849 POINT(41.52733138677367 74.94180671039369) bank171849 +171850 POINT(40.04763666593629 74.65832389134691) bank171850 +171851 POINT(40.183373680236166 74.56775416921334) bank171851 +171852 POINT(41.59603508458853 73.14849916415191) bank171852 +171853 POINT(41.59030786647907 74.89359794594884) bank171853 +171854 POINT(40.66247669029744 74.52654202428815) bank171854 +171855 POINT(40.023618610787956 74.47494328291418) bank171855 +171856 POINT(39.92324704710588 74.5778791229287) bank171856 +171857 POINT(39.963294859305826 74.84256400446053) bank171857 +171858 POINT(40.41989780178709 73.3609447513984) bank171858 +171859 POINT(40.69707662281684 74.2020351869415) bank171859 +171860 POINT(41.06606615555148 73.64952393408322) bank171860 +171861 POINT(40.69689827093781 74.56237031002863) bank171861 +171862 POINT(40.66044956244583 74.3117077481648) bank171862 +171863 POINT(39.85317374406168 73.07878584657743) bank171863 +171864 POINT(40.82619360013433 74.8018937959305) bank171864 +171865 POINT(41.513172422621814 73.1560311797312) bank171865 +171866 POINT(41.53273100607648 74.01317849833038) bank171866 +171867 POINT(41.021238870738884 74.8773335169367) bank171867 +171868 POINT(39.9577851284424 74.87423709302841) bank171868 +171869 POINT(41.64434214028403 74.11642557562085) bank171869 +171870 POINT(40.9204019529947 73.18901712024568) bank171870 +171871 POINT(40.9182376937899 73.97459697774798) bank171871 +171872 POINT(40.0495539075037 74.29786528403169) bank171872 +171873 POINT(39.73887708492971 73.35800109470809) bank171873 +171874 POINT(40.10568024830127 73.08219784413036) bank171874 +171875 POINT(41.22930965359006 74.42419394716839) bank171875 +171876 POINT(41.48393427682533 74.84494201526171) bank171876 +171877 POINT(40.087704964651294 73.08203536266839) bank171877 +171878 POINT(39.73813134072803 74.06858315208102) bank171878 +171879 POINT(41.56798234861236 73.61831790603917) bank171879 +171880 POINT(41.129858271201144 74.22856931896939) bank171880 +171881 POINT(41.03662630065372 74.70391730587293) bank171881 +171882 POINT(40.48560713391996 74.15182559469847) bank171882 +171883 POINT(40.930625251363594 73.45204017924671) bank171883 +171884 POINT(41.05449870072255 74.56823159077594) bank171884 +171885 POINT(40.62312071864453 74.58031713143305) bank171885 +171886 POINT(40.23159618119839 74.50303889192705) bank171886 +171887 POINT(41.248809868531715 73.90070453367439) bank171887 +171888 POINT(41.401919472711086 73.04905943924255) bank171888 +171889 POINT(40.22736971823066 73.85560197467647) bank171889 +171890 POINT(40.86380714888236 73.25816457590017) bank171890 +171891 POINT(40.70073675804857 73.19343966136748) bank171891 +171892 POINT(39.88260528532501 73.19413827157926) bank171892 +171893 POINT(39.87986934745638 74.66191756107219) bank171893 +171894 POINT(41.28949480633433 73.44540859006524) bank171894 +171895 POINT(41.42575958082505 73.7118552075007) bank171895 +171896 POINT(40.170956155376366 73.89030982082582) bank171896 +171897 POINT(40.69616833849232 73.15865594702292) bank171897 +171898 POINT(39.840858128371906 73.67285865107225) bank171898 +171899 POINT(41.11919695609121 74.79066264832663) bank171899 +171900 POINT(40.66944038478256 73.33696060557287) bank171900 +171901 POINT(40.30292498990102 74.26814946983201) bank171901 +171902 POINT(39.98501088823632 73.76804457079564) bank171902 +171903 POINT(40.35614722905456 74.45585774463251) bank171903 +171904 POINT(41.19693534062254 74.42871134615315) bank171904 +171905 POINT(40.71284500534951 74.03479683405614) bank171905 +171906 POINT(41.28479819142007 73.34493354754403) bank171906 +171907 POINT(40.18825162703381 74.77112760209599) bank171907 +171908 POINT(40.28751905629841 73.84253676765555) bank171908 +171909 POINT(41.28290197092141 73.83046260813025) bank171909 +171910 POINT(40.957711806533354 73.60303756645548) bank171910 +171911 POINT(40.06946628655268 74.44843182361845) bank171911 +171912 POINT(40.97367777551207 74.31939703169172) bank171912 +171913 POINT(40.181090919734594 73.84883363343457) bank171913 +171914 POINT(40.242267009025944 73.81850819941347) bank171914 +171915 POINT(40.72309615413913 73.49137840401201) bank171915 +171916 POINT(41.641951987107 73.28304190181983) bank171916 +171917 POINT(40.35307607900669 73.81381856556573) bank171917 +171918 POINT(41.07222348532136 74.13829446679398) bank171918 +171919 POINT(40.54754978851666 74.76179707734546) bank171919 +171920 POINT(41.305186470628406 73.365856050165) bank171920 +171921 POINT(40.833711803729095 74.1920117482255) bank171921 +171922 POINT(41.46740606206886 73.96524012205461) bank171922 +171923 POINT(40.694159923006936 74.06399286214162) bank171923 +171924 POINT(41.583190428355074 74.25082537679363) bank171924 +171925 POINT(40.54355162521808 73.09572672857085) bank171925 +171926 POINT(39.995055549264535 73.92849508980136) bank171926 +171927 POINT(40.28094924477124 73.2341854105261) bank171927 +171928 POINT(39.82410433851097 74.66118359488425) bank171928 +171929 POINT(39.84140817117063 73.13655511632399) bank171929 +171930 POINT(41.680745787120266 73.56105718818701) bank171930 +171931 POINT(40.92148686769229 74.19185073086403) bank171931 +171932 POINT(39.99481903659556 73.54887502384784) bank171932 +171933 POINT(40.71446839794425 74.61813026322953) bank171933 +171934 POINT(41.30363107411917 73.23997232747767) bank171934 +171935 POINT(40.2841199401065 73.09685373173849) bank171935 +171936 POINT(39.72178585416888 74.42097377867772) bank171936 +171937 POINT(40.105163685600786 74.43973005496093) bank171937 +171938 POINT(40.8271969848089 74.45854877065696) bank171938 +171939 POINT(40.89924313425627 73.34166144742653) bank171939 +171940 POINT(41.58368222626737 73.79485384771007) bank171940 +171941 POINT(40.60801610616764 74.84385016945382) bank171941 +171942 POINT(39.74934941613533 74.40388653877557) bank171942 +171943 POINT(41.315175674753306 73.2071229295105) bank171943 +171944 POINT(40.04921278434514 74.02036181930654) bank171944 +171945 POINT(40.07538365857765 73.92368123895533) bank171945 +171946 POINT(41.37208542060133 74.18465599060917) bank171946 +171947 POINT(41.6757852994904 74.71818354498829) bank171947 +171948 POINT(41.164731244722724 74.44700318356767) bank171948 +171949 POINT(40.695169042603936 74.27503310240235) bank171949 +171950 POINT(40.91012496437654 73.39166279387116) bank171950 +171951 POINT(41.57891403493843 74.48441972088241) bank171951 +171952 POINT(40.739507190576035 74.98955244751691) bank171952 +171953 POINT(40.9157461046371 74.48966323043868) bank171953 +171954 POINT(41.60843186924695 73.21075875853629) bank171954 +171955 POINT(41.174788657198995 74.25171802745692) bank171955 +171956 POINT(41.61143214937325 73.68939439474983) bank171956 +171957 POINT(40.03061398062727 74.66835044034741) bank171957 +171958 POINT(40.09997657191254 74.23790465074649) bank171958 +171959 POINT(40.677484411329225 74.51397248240087) bank171959 +171960 POINT(40.549706758619664 73.3192344180575) bank171960 +171961 POINT(40.76544612623295 74.2467559599249) bank171961 +171962 POINT(40.18402824013031 73.58609350410879) bank171962 +171963 POINT(41.13576779894065 74.73038505504763) bank171963 +171964 POINT(40.952358920318055 73.09251967421419) bank171964 +171965 POINT(40.874522966561486 73.24402367379304) bank171965 +171966 POINT(41.46192866985747 74.17527904555853) bank171966 +171967 POINT(40.63130868895893 73.08809132333879) bank171967 +171968 POINT(41.26399463055223 73.31586812285465) bank171968 +171969 POINT(40.2428136457815 73.93913903447975) bank171969 +171970 POINT(39.980982510006925 74.62366357285747) bank171970 +171971 POINT(41.188671830197904 74.47517554149184) bank171971 +171972 POINT(40.56946150913723 74.86442395583619) bank171972 +171973 POINT(40.77070120064876 74.14335198635301) bank171973 +171974 POINT(39.83689710000594 74.77941511580558) bank171974 +171975 POINT(39.94056236288458 74.45805882185657) bank171975 +171976 POINT(40.942901433944456 74.12314989039022) bank171976 +171977 POINT(39.908143293542 74.54493516094) bank171977 +171978 POINT(39.82716264664035 73.86563708056842) bank171978 +171979 POINT(39.8081845124381 74.51264108465529) bank171979 +171980 POINT(40.77163695947221 74.66169093777862) bank171980 +171981 POINT(41.55665077427322 73.19274060106203) bank171981 +171982 POINT(41.486274831815386 74.83347685449854) bank171982 +171983 POINT(40.197880597007114 74.92203536918616) bank171983 +171984 POINT(40.72282422223687 74.64219320222952) bank171984 +171985 POINT(40.664239098277214 74.8496175684596) bank171985 +171986 POINT(41.61937915473613 73.50037213033113) bank171986 +171987 POINT(41.62191990853486 74.63166768464606) bank171987 +171988 POINT(41.3463569931613 74.84315034108324) bank171988 +171989 POINT(40.94795755273163 73.93802846886) bank171989 +171990 POINT(41.29693397287044 74.80657599210765) bank171990 +171991 POINT(40.99724524253305 73.36617025105411) bank171991 +171992 POINT(40.51873511007626 73.97618571704876) bank171992 +171993 POINT(39.721064635875734 74.66766562989294) bank171993 +171994 POINT(40.369293167662235 73.05449850331917) bank171994 +171995 POINT(40.2871514466532 74.4676578026997) bank171995 +171996 POINT(41.44991256050219 73.75430605022271) bank171996 +171997 POINT(41.30384548350954 73.62423486559766) bank171997 +171998 POINT(41.25232146251221 74.04297305247536) bank171998 +171999 POINT(40.94572897868821 73.08953702877167) bank171999 +172000 POINT(40.05578794179269 74.00844604402421) bank172000 +172001 POINT(41.066559938134354 74.054161284509) bank172001 +172002 POINT(40.666610234783164 74.5061006843282) bank172002 +172003 POINT(41.03348053918648 74.66333988989409) bank172003 +172004 POINT(41.34725699799652 73.96002207573085) bank172004 +172005 POINT(40.009276478234234 73.59555088593763) bank172005 +172006 POINT(41.527389537180476 73.33118825620065) bank172006 +172007 POINT(40.141001549002674 74.89495647024381) bank172007 +172008 POINT(40.78268428462657 73.66302289882631) bank172008 +172009 POINT(39.98270486754788 73.12482353942956) bank172009 +172010 POINT(40.32483623379011 73.99680546538501) bank172010 +172011 POINT(41.22365671649942 73.32572850004425) bank172011 +172012 POINT(40.155270860965466 74.17554868233705) bank172012 +172013 POINT(40.455275952737225 73.97423384883892) bank172013 +172014 POINT(41.235137102914244 73.13491146134052) bank172014 +172015 POINT(41.18417056152065 74.35399773183798) bank172015 +172016 POINT(40.41490084030685 73.56938239635124) bank172016 +172017 POINT(40.8114889678709 74.35930099923097) bank172017 +172018 POINT(39.82490738753943 74.47000016093932) bank172018 +172019 POINT(41.39942879520122 74.07456570686786) bank172019 +172020 POINT(40.75748393977533 74.18908491148562) bank172020 +172021 POINT(39.73322500393102 74.07277216828024) bank172021 +172022 POINT(41.20981709330171 73.73361890558337) bank172022 +172023 POINT(40.34336979571254 74.25447523278709) bank172023 +172024 POINT(41.3962976637389 74.10407734359926) bank172024 +172025 POINT(40.03773799009295 74.45529589640367) bank172025 +172026 POINT(40.80994934669451 73.3593372617624) bank172026 +172027 POINT(41.3052751187822 73.30454240929174) bank172027 +172028 POINT(40.38712598608549 73.86956831041982) bank172028 +172029 POINT(40.397924310196174 73.5011769850805) bank172029 +172030 POINT(40.68785565528385 74.18608196031755) bank172030 +172031 POINT(41.39307273706507 74.46660054253884) bank172031 +172032 POINT(41.668083315647124 74.51075485091437) bank172032 +172033 POINT(41.25058387768629 74.20038827883371) bank172033 +172034 POINT(39.992041043326864 74.51292743220927) bank172034 +172035 POINT(40.94065214066245 73.14116120073422) bank172035 +172036 POINT(41.25641952658642 73.99375356329037) bank172036 +172037 POINT(40.968436720885386 74.2832542729408) bank172037 +172038 POINT(40.73034742712194 74.46248344516141) bank172038 +172039 POINT(41.39750306415743 73.73393412727208) bank172039 +172040 POINT(41.186962188098654 73.89027099649157) bank172040 +172041 POINT(40.82214142897647 73.58280818820144) bank172041 +172042 POINT(40.88982249563573 73.70526898602161) bank172042 +172043 POINT(41.05528645872231 73.61322207713187) bank172043 +172044 POINT(41.564155477881954 73.64547712291103) bank172044 +172045 POINT(40.55238147570647 74.3934070666877) bank172045 +172046 POINT(39.95932128160991 74.38084409093959) bank172046 +172047 POINT(40.058828375910316 73.0233211104146) bank172047 +172048 POINT(40.80487361175397 73.9402500154827) bank172048 +172049 POINT(41.20045529722579 74.55470568718538) bank172049 +172050 POINT(40.10316156944666 73.41351056905926) bank172050 +172051 POINT(41.682093305407406 74.42158940863233) bank172051 +172052 POINT(41.020799138115756 74.85560765630791) bank172052 +172053 POINT(40.091719468879376 73.15159698169222) bank172053 +172054 POINT(41.306482265319474 73.16665697380822) bank172054 +172055 POINT(40.957712224951415 74.83576592994343) bank172055 +172056 POINT(40.32121492046732 73.77852147703504) bank172056 +172057 POINT(40.930172494829364 74.01039133664808) bank172057 +172058 POINT(41.02074387056133 73.13569149037446) bank172058 +172059 POINT(39.85738231311665 74.0421021981864) bank172059 +172060 POINT(41.077061946200715 73.610535277196) bank172060 +172061 POINT(41.16424611603469 73.73645005482473) bank172061 +172062 POINT(41.143294875419386 74.95149455382018) bank172062 +172063 POINT(40.58884480911396 74.85617853396023) bank172063 +172064 POINT(41.115776337288196 73.23716527951106) bank172064 +172065 POINT(41.58276427178449 74.07434815096465) bank172065 +172066 POINT(41.05569892292371 73.15442102764796) bank172066 +172067 POINT(39.89498396164069 73.66774326003478) bank172067 +172068 POINT(40.754643608853854 73.00926859475916) bank172068 +172069 POINT(41.709203535109 74.7696264873433) bank172069 +172070 POINT(40.72121642898919 73.58058672641693) bank172070 +172071 POINT(40.461081281825095 74.25717316153545) bank172071 +172072 POINT(40.48273979462138 74.38886733579993) bank172072 +172073 POINT(39.84775483972352 73.70691197723049) bank172073 +172074 POINT(39.76903800614975 74.4874411234825) bank172074 +172075 POINT(39.83455213452997 74.59487466938349) bank172075 +172076 POINT(40.222142418358345 73.44949356487739) bank172076 +172077 POINT(39.840555347277366 74.12997817757463) bank172077 +172078 POINT(41.23005659847475 73.1264300957545) bank172078 +172079 POINT(40.3814540657355 74.86581871773272) bank172079 +172080 POINT(40.445485433017545 73.27260115013344) bank172080 +172081 POINT(39.84570329606912 73.79319455601409) bank172081 +172082 POINT(39.840760364060635 73.58169003477312) bank172082 +172083 POINT(40.93136376522184 73.52888495740389) bank172083 +172084 POINT(41.66248226196716 74.5134181862319) bank172084 +172085 POINT(39.80035766696369 74.03768468715185) bank172085 +172086 POINT(40.58410243688218 73.13458304811614) bank172086 +172087 POINT(40.58726595734461 73.47803571674059) bank172087 +172088 POINT(40.25229988229542 74.08296021272689) bank172088 +172089 POINT(40.67713764288174 73.81947963353045) bank172089 +172090 POINT(40.93511657093686 74.61584808849273) bank172090 +172091 POINT(40.83325271936462 74.34757467755824) bank172091 +172092 POINT(41.599075951296896 73.49817891396792) bank172092 +172093 POINT(40.228167617498954 74.93622156803472) bank172093 +172094 POINT(41.20813876895211 74.25804146708302) bank172094 +172095 POINT(40.50299296515964 74.07525737427713) bank172095 +172096 POINT(40.988304870406935 73.78751721626303) bank172096 +172097 POINT(41.28978334097783 73.32400158398497) bank172097 +172098 POINT(40.53162854134775 74.71903409253822) bank172098 +172099 POINT(40.53638342272596 73.87762502813642) bank172099 +172100 POINT(41.066346197166965 74.45066225917468) bank172100 +172101 POINT(40.39520757413206 74.3714696013792) bank172101 +172102 POINT(40.01426990389182 74.02481282946067) bank172102 +172103 POINT(40.39024100561475 74.21835421341298) bank172103 +172104 POINT(39.84088846751067 73.8933135722236) bank172104 +172105 POINT(41.43362740055089 74.27125562968256) bank172105 +172106 POINT(40.0223721843524 73.12711155917246) bank172106 +172107 POINT(41.57459076927098 73.77234430958592) bank172107 +172108 POINT(39.85872935149099 73.7399800168134) bank172108 +172109 POINT(40.408377931806264 74.46148249056752) bank172109 +172110 POINT(40.828495382252356 74.58391473238528) bank172110 +172111 POINT(41.329860505463785 74.94887601802273) bank172111 +172112 POINT(40.81085047155811 74.68738363299786) bank172112 +172113 POINT(39.907099516481054 74.08251336817358) bank172113 +172114 POINT(41.42060359644015 73.67888678018774) bank172114 +172115 POINT(41.18726929311421 73.25322690537959) bank172115 +172116 POINT(41.45140798749279 73.47754246820226) bank172116 +172117 POINT(41.60567101130795 73.49037063762911) bank172117 +172118 POINT(40.754971938447056 73.29705442982532) bank172118 +172119 POINT(41.259705621941315 73.91414509109303) bank172119 +172120 POINT(41.288802344829776 74.68961512088441) bank172120 +172121 POINT(40.213029323612275 73.36384565841479) bank172121 +172122 POINT(41.58940177486414 73.36148468481804) bank172122 +172123 POINT(41.67755961260398 74.21693588333676) bank172123 +172124 POINT(40.99145147092568 73.2522109271231) bank172124 +172125 POINT(40.678515111832056 74.89042499148583) bank172125 +172126 POINT(40.02906941234699 74.41026443195109) bank172126 +172127 POINT(41.49110278602981 73.2853153233008) bank172127 +172128 POINT(39.80561790333228 73.89542140511142) bank172128 +172129 POINT(41.201057774083175 73.23779894130837) bank172129 +172130 POINT(40.02059711609286 73.06458549381053) bank172130 +172131 POINT(40.00846167741578 73.31320297984414) bank172131 +172132 POINT(40.985236837010504 74.10514629815862) bank172132 +172133 POINT(40.855084299537886 74.27391740307407) bank172133 +172134 POINT(40.970652147933194 73.31784463646005) bank172134 +172135 POINT(40.047510185904855 73.04123024902614) bank172135 +172136 POINT(39.87230999725867 74.97213370637961) bank172136 +172137 POINT(40.60850368149128 74.68859145169053) bank172137 +172138 POINT(40.79138672062356 74.67971936002108) bank172138 +172139 POINT(41.533790141879905 74.53592358786368) bank172139 +172140 POINT(41.616473562770814 74.37646041330841) bank172140 +172141 POINT(41.55985609661376 74.13500495261684) bank172141 +172142 POINT(41.13684581295734 73.38655492940913) bank172142 +172143 POINT(40.69444433895147 73.36254409143895) bank172143 +172144 POINT(41.67595141524006 74.99688864511148) bank172144 +172145 POINT(41.40121848532304 74.19549132921446) bank172145 +172146 POINT(39.73823329286867 74.2611895337975) bank172146 +172147 POINT(40.76319996940045 74.40430567215462) bank172147 +172148 POINT(40.38616702344207 73.23734453544732) bank172148 +172149 POINT(41.06612995011827 73.08100888646824) bank172149 +172150 POINT(39.89970830745743 73.61429185884428) bank172150 +172151 POINT(40.47173482379892 74.80371471918944) bank172151 +172152 POINT(41.60069978751701 73.57054406808398) bank172152 +172153 POINT(40.20956418941888 73.37084560070915) bank172153 +172154 POINT(41.09448042601952 74.2939433544233) bank172154 +172155 POINT(41.14847290840571 74.44685072951297) bank172155 +172156 POINT(40.90564556285252 73.99422603545747) bank172156 +172157 POINT(39.872804430282 73.12281606480765) bank172157 +172158 POINT(40.778815116633005 73.09206087647014) bank172158 +172159 POINT(39.74012094185053 73.25063135021911) bank172159 +172160 POINT(41.599524677708956 73.28279000815488) bank172160 +172161 POINT(41.313943806168695 73.05100669428109) bank172161 +172162 POINT(41.34614511150475 73.07224478350686) bank172162 +172163 POINT(41.702343834494336 74.1683778815823) bank172163 +172164 POINT(40.771497008261115 73.32184652735911) bank172164 +172165 POINT(41.459594471545074 73.77518758583504) bank172165 +172166 POINT(40.10969650360455 73.73377724280515) bank172166 +172167 POINT(40.93784555838007 74.12270337464112) bank172167 +172168 POINT(40.41247281351838 74.75058632710473) bank172168 +172169 POINT(41.28023423474246 73.19659354894726) bank172169 +172170 POINT(41.454367198446214 73.08256385685524) bank172170 +172171 POINT(41.07462405479876 74.26861480558271) bank172171 +172172 POINT(41.142066067164365 74.9643876056951) bank172172 +172173 POINT(40.624864670108046 74.54846065853161) bank172173 +172174 POINT(41.193040779093835 74.61384219834666) bank172174 +172175 POINT(40.1087328935412 73.4402031565672) bank172175 +172176 POINT(41.07278854830072 73.98358808902935) bank172176 +172177 POINT(41.34783079407611 73.87088708030298) bank172177 +172178 POINT(40.99257641188987 73.11072008202528) bank172178 +172179 POINT(40.97645743098944 74.08914504257315) bank172179 +172180 POINT(40.4105749825963 74.05392347440356) bank172180 +172181 POINT(41.511487489934765 73.81739089004483) bank172181 +172182 POINT(40.6599438491503 73.31970291200076) bank172182 +172183 POINT(39.94618389058733 73.30680498452793) bank172183 +172184 POINT(40.46797073666367 73.23778024260406) bank172184 +172185 POINT(41.0478264491372 74.79406168942894) bank172185 +172186 POINT(41.65990954811429 73.59745628842074) bank172186 +172187 POINT(40.959293211151255 74.26411197672142) bank172187 +172188 POINT(41.00079513262477 74.51834584678367) bank172188 +172189 POINT(40.940670717044526 73.2813901235362) bank172189 +172190 POINT(40.276529539989234 73.92704740186463) bank172190 +172191 POINT(41.70352406247028 74.69180476797402) bank172191 +172192 POINT(39.922761510755244 74.3260400020447) bank172192 +172193 POINT(40.00729325818556 73.91440848525073) bank172193 +172194 POINT(41.262340505903026 73.54517274175137) bank172194 +172195 POINT(39.90244301358676 74.08733170919423) bank172195 +172196 POINT(40.21584323454307 74.00409560793511) bank172196 +172197 POINT(40.043417126053555 74.2908971875736) bank172197 +172198 POINT(39.80416271570754 74.49786009106697) bank172198 +172199 POINT(41.028308866905796 74.3724995185011) bank172199 +172200 POINT(39.898339496636716 73.56354108148871) bank172200 +172201 POINT(40.034916074805714 73.2148552732715) bank172201 +172202 POINT(41.4472349740264 74.48120894752878) bank172202 +172203 POINT(40.117625094518374 73.27468830722417) bank172203 +172204 POINT(40.996726651937344 73.93754550901915) bank172204 +172205 POINT(41.26267039775868 74.37723519941851) bank172205 +172206 POINT(40.26259134977205 73.56884334735877) bank172206 +172207 POINT(41.38160381271385 74.31502531507807) bank172207 +172208 POINT(39.99573841830038 74.93737344824603) bank172208 +172209 POINT(40.68708503723352 73.96773452155368) bank172209 +172210 POINT(41.49351640529198 73.1845300060826) bank172210 +172211 POINT(40.7813069751362 74.97074445817015) bank172211 +172212 POINT(40.338987211787845 74.36352133921567) bank172212 +172213 POINT(41.24270900035401 74.31365289247482) bank172213 +172214 POINT(39.83154849957652 73.48774038399479) bank172214 +172215 POINT(41.481539787738114 73.75070769911956) bank172215 +172216 POINT(40.28623457195192 74.13648752105618) bank172216 +172217 POINT(40.19179626850738 74.90739854157773) bank172217 +172218 POINT(41.39484851778242 74.23532142515644) bank172218 +172219 POINT(40.88707410153104 73.09066665543091) bank172219 +172220 POINT(39.93016897157666 74.80047073331588) bank172220 +172221 POINT(41.41753689105387 74.68866679555815) bank172221 +172222 POINT(40.015668354207385 74.3369254352827) bank172222 +172223 POINT(40.1687784911293 73.01287345740701) bank172223 +172224 POINT(41.116772219437564 73.86539026164839) bank172224 +172225 POINT(39.85237658080624 73.32750666619705) bank172225 +172226 POINT(41.45984899366012 74.36417671337534) bank172226 +172227 POINT(39.75581444934921 74.74157643129551) bank172227 +172228 POINT(41.22606343311166 73.0713047814275) bank172228 +172229 POINT(40.61423878716428 73.84344755851616) bank172229 +172230 POINT(40.96161647968101 73.15539500985675) bank172230 +172231 POINT(40.89494460105448 74.55682674193922) bank172231 +172232 POINT(40.89210165780459 73.16838123807142) bank172232 +172233 POINT(39.78747175889552 73.65530389351824) bank172233 +172234 POINT(40.38648357329129 74.01531942611216) bank172234 +172235 POINT(40.89109929632322 73.12667895593799) bank172235 +172236 POINT(40.02196932988169 74.51258850079716) bank172236 +172237 POINT(40.47464942607014 73.49536463265967) bank172237 +172238 POINT(40.710490602933625 73.88530336451478) bank172238 +172239 POINT(40.912294004801105 73.595539465745) bank172239 +172240 POINT(40.7939836035963 74.92671526287988) bank172240 +172241 POINT(40.55735239310015 74.60266827139294) bank172241 +172242 POINT(39.77859721003558 74.78659174704205) bank172242 +172243 POINT(40.704391256162495 73.85300405065773) bank172243 +172244 POINT(40.17419588562315 73.92101709088598) bank172244 +172245 POINT(41.5527004237076 73.0997611586296) bank172245 +172246 POINT(41.33286345863178 73.09489524854894) bank172246 +172247 POINT(41.68723661844556 74.17990579763882) bank172247 +172248 POINT(40.814745197536766 74.85962624761208) bank172248 +172249 POINT(40.87761109897126 73.2700419998258) bank172249 +172250 POINT(41.30730149119002 74.6169644699789) bank172250 +172251 POINT(40.72195805080622 73.46870071768684) bank172251 +172252 POINT(40.73566169334107 73.48746226500043) bank172252 +172253 POINT(40.63310578219859 74.38219482618686) bank172253 +172254 POINT(40.988383670709794 73.82538387628782) bank172254 +172255 POINT(40.29925659717956 74.82365980612697) bank172255 +172256 POINT(41.44073635792897 74.44709098888372) bank172256 +172257 POINT(41.70356581650291 73.29303691161675) bank172257 +172258 POINT(41.28168229974054 73.55729046378288) bank172258 +172259 POINT(39.920291908986734 73.19545690089193) bank172259 +172260 POINT(40.98405942710575 74.08175247421799) bank172260 +172261 POINT(39.75195181019568 74.62327139871688) bank172261 +172262 POINT(40.94638980519739 74.1697353704081) bank172262 +172263 POINT(40.57514430467167 73.40761292066334) bank172263 +172264 POINT(40.68191882462832 73.07114815425618) bank172264 +172265 POINT(40.20353363553485 73.19570489565668) bank172265 +172266 POINT(41.01039768477667 74.71406144819544) bank172266 +172267 POINT(40.59965561015209 73.94171744289106) bank172267 +172268 POINT(41.17302222029071 73.71794528927927) bank172268 +172269 POINT(40.85108058736336 74.76190290532917) bank172269 +172270 POINT(39.89179979057907 73.07309183399404) bank172270 +172271 POINT(39.72362168973376 74.46448392859668) bank172271 +172272 POINT(40.335786516739965 74.3755959471076) bank172272 +172273 POINT(41.053581155341526 74.68136856104205) bank172273 +172274 POINT(40.57602449106653 73.2821479031182) bank172274 +172275 POINT(41.198990998463735 73.33230599941423) bank172275 +172276 POINT(41.51105962643809 74.84505917098151) bank172276 +172277 POINT(39.92255536017965 74.19622148320047) bank172277 +172278 POINT(41.45726720859584 74.79364580284947) bank172278 +172279 POINT(41.08422625859606 73.06392178458029) bank172279 +172280 POINT(39.78954930981504 73.9371581612036) bank172280 +172281 POINT(41.523345580932315 73.22454707726797) bank172281 +172282 POINT(40.738851759461504 74.63199129966155) bank172282 +172283 POINT(41.551524845133386 74.70413418834096) bank172283 +172284 POINT(40.592726759188636 73.01795195873085) bank172284 +172285 POINT(41.47726510267413 73.85088763673411) bank172285 +172286 POINT(40.590144796109755 74.82174879170852) bank172286 +172287 POINT(39.846321383343444 73.31375487760491) bank172287 +172288 POINT(40.82419592287715 73.66753485981478) bank172288 +172289 POINT(39.755283181464456 73.41835032843531) bank172289 +172290 POINT(40.37812095707445 74.50907739138825) bank172290 +172291 POINT(40.44976667073043 74.214879420294) bank172291 +172292 POINT(39.770298832328656 73.71073450365208) bank172292 +172293 POINT(41.61768430852424 74.53518573225533) bank172293 +172294 POINT(39.77218935685812 74.51871792209751) bank172294 +172295 POINT(40.32568794452531 73.49323836844339) bank172295 +172296 POINT(41.376532750850025 74.6000877214841) bank172296 +172297 POINT(40.07852184023084 74.18656144439299) bank172297 +172298 POINT(39.81904949027706 74.25372366640894) bank172298 +172299 POINT(41.57325863482735 73.7603878325633) bank172299 +172300 POINT(40.14139648668416 74.29883035879642) bank172300 +172301 POINT(40.61283969555097 73.80052758251473) bank172301 +172302 POINT(41.573017132355616 74.24895706037513) bank172302 +172303 POINT(39.91254469904269 74.54818557804914) bank172303 +172304 POINT(40.151420978211355 73.45575570494015) bank172304 +172305 POINT(41.27158270039086 73.60566272668915) bank172305 +172306 POINT(40.67231659948004 73.18137421443879) bank172306 +172307 POINT(39.97633464247346 73.82957220124713) bank172307 +172308 POINT(40.02778056952872 73.2095653549391) bank172308 +172309 POINT(40.191642528316656 73.46081093929728) bank172309 +172310 POINT(41.10493806851746 74.39397620388631) bank172310 +172311 POINT(41.527276669116176 74.7101548047687) bank172311 +172312 POINT(39.774811340333756 73.11603505675896) bank172312 +172313 POINT(41.281181768580645 73.72834678731301) bank172313 +172314 POINT(39.953089935836545 74.43566998351889) bank172314 +172315 POINT(40.27448968292379 74.32656009940798) bank172315 +172316 POINT(39.90446033428878 74.57794008887275) bank172316 +172317 POINT(40.437934660597676 73.85050574144938) bank172317 +172318 POINT(39.81196974090075 74.73372950836432) bank172318 +172319 POINT(39.99735507035724 73.39954344109977) bank172319 +172320 POINT(40.565447392912276 74.00088730749403) bank172320 +172321 POINT(40.32269304905505 74.00309608427519) bank172321 +172322 POINT(41.6435050226166 73.55821944078583) bank172322 +172323 POINT(40.55996678179562 73.2222946808137) bank172323 +172324 POINT(41.0841627102148 73.2647328967797) bank172324 +172325 POINT(40.82933783547276 73.42829652357543) bank172325 +172326 POINT(40.005507831633125 73.67406809579138) bank172326 +172327 POINT(40.47833417223613 73.83699799827912) bank172327 +172328 POINT(40.689388957024704 73.26582721220522) bank172328 +172329 POINT(39.842572835760905 74.2222873699665) bank172329 +172330 POINT(40.6223603699017 73.8875075343499) bank172330 +172331 POINT(41.11907061398358 74.64216212902453) bank172331 +172332 POINT(40.84401420275767 73.9499293236192) bank172332 +172333 POINT(39.80242961253233 73.55369635671109) bank172333 +172334 POINT(40.55491509356274 74.50565333075774) bank172334 +172335 POINT(40.47287904452356 73.60263394922691) bank172335 +172336 POINT(41.23563351048326 73.97503923023729) bank172336 +172337 POINT(41.16706934729274 73.89554904157349) bank172337 +172338 POINT(41.57755966172994 74.66178794160123) bank172338 +172339 POINT(40.25382686755549 74.69820033546105) bank172339 +172340 POINT(40.947364541522056 73.17607366454841) bank172340 +172341 POINT(40.14258857203328 73.55035193771711) bank172341 +172342 POINT(40.89262304660969 73.07329892015734) bank172342 +172343 POINT(39.89242085572307 73.28794083359085) bank172343 +172344 POINT(40.57161339831868 74.15034881813693) bank172344 +172345 POINT(41.23342376625084 74.02171216152955) bank172345 +172346 POINT(40.87611894114494 74.96014695122513) bank172346 +172347 POINT(41.13063659552381 74.62362049121646) bank172347 +172348 POINT(40.3026048888369 74.48158481918773) bank172348 +172349 POINT(40.32745074069665 74.94752440781053) bank172349 +172350 POINT(41.01565506568183 73.01123499452497) bank172350 +172351 POINT(39.809965961576715 73.29360672915135) bank172351 +172352 POINT(41.37539841928184 74.75123678997086) bank172352 +172353 POINT(40.87398110703238 74.67979177139217) bank172353 +172354 POINT(40.14227786815719 73.68359665821136) bank172354 +172355 POINT(41.49745007422172 73.36331511479547) bank172355 +172356 POINT(41.67815131516251 74.51420272398644) bank172356 +172357 POINT(39.90997724683777 74.86947643481864) bank172357 +172358 POINT(40.31387848772029 74.4312607456634) bank172358 +172359 POINT(41.02860409851294 74.65061437518594) bank172359 +172360 POINT(40.6851911940522 73.11158837465342) bank172360 +172361 POINT(40.17178011362731 73.75583817698157) bank172361 +172362 POINT(39.83814910424807 73.13806044555797) bank172362 +172363 POINT(40.359861892249675 73.21445309382321) bank172363 +172364 POINT(41.50341046558332 74.88486910075581) bank172364 +172365 POINT(40.3632945495406 74.88901349357693) bank172365 +172366 POINT(41.50173176570512 73.69069353353156) bank172366 +172367 POINT(40.246880123385544 73.15454615908712) bank172367 +172368 POINT(40.39066980985677 73.02967291960645) bank172368 +172369 POINT(40.906616993543814 73.3920171202444) bank172369 +172370 POINT(41.57887553798578 74.58190526051241) bank172370 +172371 POINT(40.24080158156095 73.05550157427331) bank172371 +172372 POINT(41.515337838625065 73.13890271462583) bank172372 +172373 POINT(40.0004157137604 73.48530342045913) bank172373 +172374 POINT(40.891219079344594 74.27531040071139) bank172374 +172375 POINT(40.632003838113526 73.5723864791008) bank172375 +172376 POINT(40.894974550176585 74.69710955696743) bank172376 +172377 POINT(40.84902385623986 73.39305500050472) bank172377 +172378 POINT(40.48261655195129 73.49673416653405) bank172378 +172379 POINT(40.48133487176216 74.72292628181488) bank172379 +172380 POINT(40.007667413189964 74.42181020775277) bank172380 +172381 POINT(41.22898218652315 74.69306151627745) bank172381 +172382 POINT(40.3708079549369 73.9346569617805) bank172382 +172383 POINT(39.9044995742343 73.4751609213883) bank172383 +172384 POINT(40.06570115906036 73.53654179930638) bank172384 +172385 POINT(40.40074815612818 73.60616530801245) bank172385 +172386 POINT(41.50786819183695 73.21973417726132) bank172386 +172387 POINT(41.53915274134292 73.46969702125998) bank172387 +172388 POINT(39.8888731242587 73.3202298042158) bank172388 +172389 POINT(40.521304202739444 73.39316047529792) bank172389 +172390 POINT(41.05473002450453 73.96264377381436) bank172390 +172391 POINT(40.326353914893616 73.06321266684634) bank172391 +172392 POINT(40.46616785040204 74.53602463379876) bank172392 +172393 POINT(40.29778736744607 73.13491947504647) bank172393 +172394 POINT(40.29012331492447 74.77145057487888) bank172394 +172395 POINT(40.24347635475591 73.65157846768703) bank172395 +172396 POINT(40.19994648404711 73.3701503191433) bank172396 +172397 POINT(40.813977874579805 73.49757945398403) bank172397 +172398 POINT(40.990496373871856 73.64308920579235) bank172398 +172399 POINT(41.15514728606044 74.25836350987875) bank172399 +172400 POINT(41.603175799088554 74.67347331534613) bank172400 +172401 POINT(40.96741211694889 74.84008415903497) bank172401 +172402 POINT(40.16073972354864 73.8858113643184) bank172402 +172403 POINT(40.83002191528811 73.49266928521095) bank172403 +172404 POINT(40.42579210632881 74.82208497083356) bank172404 +172405 POINT(40.78652713978681 74.67322610218828) bank172405 +172406 POINT(40.417072269705855 73.36990182315206) bank172406 +172407 POINT(41.399043641645605 74.99216168956534) bank172407 +172408 POINT(39.719182240661794 73.05321809703005) bank172408 +172409 POINT(41.59034804466671 74.36998294215678) bank172409 +172410 POINT(40.17409124915026 73.06266171551107) bank172410 +172411 POINT(40.58260762560377 74.54882034986647) bank172411 +172412 POINT(40.78837044998685 74.62950693291691) bank172412 +172413 POINT(41.55013695367946 74.15094035347289) bank172413 +172414 POINT(40.32648736969503 73.06633504179516) bank172414 +172415 POINT(40.828107660388135 73.72072586997896) bank172415 +172416 POINT(40.8871662465976 74.40157174713259) bank172416 +172417 POINT(39.857951612666795 73.29510024483395) bank172417 +172418 POINT(41.069941446780845 74.26961735305265) bank172418 +172419 POINT(40.37619283328446 73.08308144884484) bank172419 +172420 POINT(40.945667286667394 73.34513165253604) bank172420 +172421 POINT(40.65763186319668 74.7964731441251) bank172421 +172422 POINT(41.058857853218356 73.41787444612623) bank172422 +172423 POINT(41.23398733684747 73.6784567995463) bank172423 +172424 POINT(40.098875308798505 74.49161743787496) bank172424 +172425 POINT(41.026311670764116 74.52753890672149) bank172425 +172426 POINT(40.559987486475215 74.69168042571881) bank172426 +172427 POINT(41.630904463112095 73.31614062208129) bank172427 +172428 POINT(40.429565107753355 73.27452426455584) bank172428 +172429 POINT(39.82175869204446 73.18329068900424) bank172429 +172430 POINT(39.96344044614861 73.48141441804528) bank172430 +172431 POINT(40.30953369045879 74.42634880357211) bank172431 +172432 POINT(40.725084415024696 73.9805177123763) bank172432 +172433 POINT(41.06526728848672 73.71866379779073) bank172433 +172434 POINT(40.49296253166043 73.0773118435253) bank172434 +172435 POINT(41.50477766764182 74.13556210028675) bank172435 +172436 POINT(40.074916851394235 74.71856736099325) bank172436 +172437 POINT(41.515633703905834 74.79825782359417) bank172437 +172438 POINT(41.001243440499344 73.71087241370046) bank172438 +172439 POINT(40.31057741768332 74.2812103707437) bank172439 +172440 POINT(41.24958303156238 73.8192554600975) bank172440 +172441 POINT(41.192007543554055 73.22148280965357) bank172441 +172442 POINT(40.82682148543129 73.6701604526049) bank172442 +172443 POINT(41.35257114822166 73.5726624696111) bank172443 +172444 POINT(40.80206136173097 73.14590940948295) bank172444 +172445 POINT(40.44158039911515 74.62239336098413) bank172445 +172446 POINT(40.45839911308859 74.42082258859227) bank172446 +172447 POINT(40.077208677099726 73.42100874621984) bank172447 +172448 POINT(40.79415982455076 74.62713655174282) bank172448 +172449 POINT(41.55056736525791 74.45713090124585) bank172449 +172450 POINT(41.09836034780244 74.05433075618754) bank172450 +172451 POINT(41.56843304289286 73.37851861576361) bank172451 +172452 POINT(39.91572550352318 74.04093405194486) bank172452 +172453 POINT(40.334440313248045 73.98408104671192) bank172453 +172454 POINT(40.57522366258849 74.29185306073572) bank172454 +172455 POINT(40.260556093817804 74.50140748369527) bank172455 +172456 POINT(41.12375427091228 73.41863941558837) bank172456 +172457 POINT(41.34622383890563 74.33619256435061) bank172457 +172458 POINT(41.015572582901775 74.02584069193303) bank172458 +172459 POINT(41.064719561691845 73.92828656698552) bank172459 +172460 POINT(40.689369848100576 73.97826446710678) bank172460 +172461 POINT(41.304872673322656 74.93968252381872) bank172461 +172462 POINT(40.31714610412583 73.34473987369212) bank172462 +172463 POINT(40.87636346154186 74.84813338546813) bank172463 +172464 POINT(40.00996888037847 73.48728874409781) bank172464 +172465 POINT(41.38581803942625 73.48999370703339) bank172465 +172466 POINT(40.716804835462256 73.91026293303442) bank172466 +172467 POINT(41.40794331714384 74.05833801181483) bank172467 +172468 POINT(39.78082158980788 73.75929187522844) bank172468 +172469 POINT(41.412805951820566 73.75999917642304) bank172469 +172470 POINT(41.66544541166213 73.61121162007228) bank172470 +172471 POINT(39.82033129846003 73.9898846884593) bank172471 +172472 POINT(40.73504829506006 73.69134595573902) bank172472 +172473 POINT(41.61808878163119 74.43984282747685) bank172473 +172474 POINT(39.821412843797376 73.78605582495386) bank172474 +172475 POINT(39.734338526021574 74.534472902064) bank172475 +172476 POINT(40.36970511889353 73.04496424699133) bank172476 +172477 POINT(39.9194635601321 74.73152757450735) bank172477 +172478 POINT(41.24384255666083 74.3756849115984) bank172478 +172479 POINT(41.66381351376656 73.29374053168137) bank172479 +172480 POINT(41.53484706381713 73.06139902426516) bank172480 +172481 POINT(41.10257632333183 73.80334294167662) bank172481 +172482 POINT(40.892538144963375 74.97070363799376) bank172482 +172483 POINT(40.40462726211158 74.86180714743169) bank172483 +172484 POINT(41.11188885219358 73.76186364582234) bank172484 +172485 POINT(40.78430230817411 74.54046897816421) bank172485 +172486 POINT(41.656652220810095 74.0298505900554) bank172486 +172487 POINT(40.33860801532804 74.4172696908023) bank172487 +172488 POINT(40.78974740278365 74.03051914405485) bank172488 +172489 POINT(39.92112748277436 73.57307677821869) bank172489 +172490 POINT(40.91657452573645 74.65928733231084) bank172490 +172491 POINT(41.227765454667924 73.72948537782614) bank172491 +172492 POINT(39.86069389679375 73.00715357871668) bank172492 +172493 POINT(40.799303303895854 73.9479780403895) bank172493 +172494 POINT(41.28205450472948 73.7271250647153) bank172494 +172495 POINT(40.88584559534956 74.42804043553056) bank172495 +172496 POINT(41.40257999496145 74.79610578159868) bank172496 +172497 POINT(40.40206179684527 73.85658223103337) bank172497 +172498 POINT(40.06992972249928 74.25767207903021) bank172498 +172499 POINT(41.60022794762896 73.19385707620654) bank172499 +172500 POINT(40.819998673890034 74.86846724728406) bank172500 +172501 POINT(40.33964062512687 74.76121494010951) bank172501 +172502 POINT(41.32808007171735 73.95216855817326) bank172502 +172503 POINT(40.623311170731405 73.59526203182469) bank172503 +172504 POINT(39.87443032309299 73.47861233279525) bank172504 +172505 POINT(41.46484885088554 74.76904432777498) bank172505 +172506 POINT(41.20626142698022 74.5833602719755) bank172506 +172507 POINT(41.593158469600674 73.44114286816841) bank172507 +172508 POINT(40.27185958029422 73.88045271243891) bank172508 +172509 POINT(39.984251970454594 73.64790319065726) bank172509 +172510 POINT(41.5705658373777 74.29991632979637) bank172510 +172511 POINT(40.74113262335961 73.80707141135795) bank172511 +172512 POINT(39.96387759778766 74.46251592136608) bank172512 +172513 POINT(40.23931760271505 74.26333152112387) bank172513 +172514 POINT(40.93462311423307 73.62488983655321) bank172514 +172515 POINT(40.624991129104906 74.11963274784542) bank172515 +172516 POINT(41.43608791263311 74.25875368510006) bank172516 +172517 POINT(41.25027404571873 73.5793391614696) bank172517 +172518 POINT(40.92816106751115 73.13949433371664) bank172518 +172519 POINT(40.34443207670782 73.89020323863829) bank172519 +172520 POINT(41.47540085894715 73.21489836361668) bank172520 +172521 POINT(40.63526724260018 73.0197576462706) bank172521 +172522 POINT(40.42618581124778 73.01355820457886) bank172522 +172523 POINT(40.39904053820772 74.33911615089768) bank172523 +172524 POINT(40.503623000792906 73.79449272220707) bank172524 +172525 POINT(41.47295967378687 74.73767677237949) bank172525 +172526 POINT(40.004879247593905 73.99888710267244) bank172526 +172527 POINT(41.543397259141905 74.18660707039551) bank172527 +172528 POINT(40.31738012322339 74.41243674220233) bank172528 +172529 POINT(40.860806153090365 74.09808378178701) bank172529 +172530 POINT(40.385886437187715 73.48470282584684) bank172530 +172531 POINT(40.202908918279405 74.02186599906891) bank172531 +172532 POINT(40.487420857446125 74.2592676876421) bank172532 +172533 POINT(40.76002176476337 74.43613093595911) bank172533 +172534 POINT(41.37416855536962 74.82288006423637) bank172534 +172535 POINT(40.595651965650326 74.14990051271582) bank172535 +172536 POINT(39.90418896709058 73.23580511520431) bank172536 +172537 POINT(40.138087137787224 73.06688148612896) bank172537 +172538 POINT(40.335821931771626 74.9498314183559) bank172538 +172539 POINT(40.61814828242131 74.81771145097419) bank172539 +172540 POINT(41.585658616705594 74.9700709686454) bank172540 +172541 POINT(41.23358462817376 73.3159801135339) bank172541 +172542 POINT(41.07024117253253 74.68390561001189) bank172542 +172543 POINT(40.035200016924435 73.90810531861953) bank172543 +172544 POINT(41.06415337307227 74.3722819684925) bank172544 +172545 POINT(41.57083151703958 74.99354922337076) bank172545 +172546 POINT(40.54556975771396 73.15349838778928) bank172546 +172547 POINT(41.32580545698717 74.88924145247957) bank172547 +172548 POINT(41.56458359302521 74.36025113415998) bank172548 +172549 POINT(40.68800205799443 74.13671281652954) bank172549 +172550 POINT(39.90145405534022 73.7883000352774) bank172550 +172551 POINT(40.00000132101762 74.9115403675565) bank172551 +172552 POINT(39.89411899798503 74.11012652361761) bank172552 +172553 POINT(40.717407134366965 73.04079583364613) bank172553 +172554 POINT(41.224152269936994 74.81546002039047) bank172554 +172555 POINT(41.24777803055017 73.30367573650895) bank172555 +172556 POINT(40.01639354115928 74.32651747265665) bank172556 +172557 POINT(40.98222665727976 74.97014829076035) bank172557 +172558 POINT(40.03948068281263 73.11313119252176) bank172558 +172559 POINT(40.308762404369986 73.06533018661821) bank172559 +172560 POINT(41.49637993223302 73.84496985071868) bank172560 +172561 POINT(40.03977537792754 74.70061339485093) bank172561 +172562 POINT(41.288106119246464 74.71477617242064) bank172562 +172563 POINT(40.702382679646355 73.56818374011635) bank172563 +172564 POINT(41.49102189707032 73.36419845742662) bank172564 +172565 POINT(41.59102097309642 74.46233115195588) bank172565 +172566 POINT(40.149535855981355 73.2536341690864) bank172566 +172567 POINT(40.38061039318986 74.04388122220803) bank172567 +172568 POINT(40.676196615951945 74.71482218294284) bank172568 +172569 POINT(41.71140170183752 73.16308295811262) bank172569 +172570 POINT(40.654615905221966 74.86928912094308) bank172570 +172571 POINT(39.9918389689879 73.41997460352118) bank172571 +172572 POINT(40.55551279348112 73.0291760786867) bank172572 +172573 POINT(40.33900688954419 74.19574515910797) bank172573 +172574 POINT(40.87802596395449 73.293541232184) bank172574 +172575 POINT(40.84322906856721 74.97305180583346) bank172575 +172576 POINT(40.61549303931585 74.23666468717092) bank172576 +172577 POINT(40.97403393588342 73.80177827920193) bank172577 +172578 POINT(40.25131059422869 73.76939911339004) bank172578 +172579 POINT(41.01222525437688 74.3185549483536) bank172579 +172580 POINT(40.54112890438492 73.1112872374028) bank172580 +172581 POINT(40.78568394069997 73.69728858095509) bank172581 +172582 POINT(41.458603181514235 74.86220500046268) bank172582 +172583 POINT(40.99493059370093 74.1972170417179) bank172583 +172584 POINT(41.00084264067651 73.14116875141812) bank172584 +172585 POINT(41.66796548572803 73.57597611984941) bank172585 +172586 POINT(40.17663025699394 73.375380741802) bank172586 +172587 POINT(40.37153066890508 74.05429220647248) bank172587 +172588 POINT(41.01204459606571 74.52503236053153) bank172588 +172589 POINT(41.468789740759796 74.75889689960117) bank172589 +172590 POINT(40.718697979100924 74.95046984804613) bank172590 +172591 POINT(40.06913235008039 73.6048383426236) bank172591 +172592 POINT(40.67124263240545 73.59507533125107) bank172592 +172593 POINT(40.39829952482765 73.2797768323465) bank172593 +172594 POINT(40.88446742091231 73.66868260623257) bank172594 +172595 POINT(41.21987631168343 73.15417145658701) bank172595 +172596 POINT(39.962730574370454 74.91257858860054) bank172596 +172597 POINT(41.646549777501846 74.13403593110284) bank172597 +172598 POINT(41.49970681455351 74.44570983447431) bank172598 +172599 POINT(39.9007413018373 74.73800185064438) bank172599 +172600 POINT(39.75404941711212 73.22672260218516) bank172600 +172601 POINT(41.333323157620214 73.7527270111416) bank172601 +172602 POINT(40.71899878799379 73.95738678095363) bank172602 +172603 POINT(40.730620877306286 73.06463144357652) bank172603 +172604 POINT(40.04700242526325 74.10952001806993) bank172604 +172605 POINT(39.825874858432954 73.47727476854091) bank172605 +172606 POINT(40.38706962594665 73.72665553706122) bank172606 +172607 POINT(41.10392742351775 73.12864225641056) bank172607 +172608 POINT(39.87182786001869 73.19363331782972) bank172608 +172609 POINT(41.11755075304138 73.5684994071376) bank172609 +172610 POINT(41.42230030881037 74.34720674231627) bank172610 +172611 POINT(40.06872992598838 74.04191520020098) bank172611 +172612 POINT(39.739531123345365 74.3551902161365) bank172612 +172613 POINT(40.44685314575052 73.77912968830262) bank172613 +172614 POINT(39.82752944423043 73.7287436304749) bank172614 +172615 POINT(40.058024625317195 73.49495688172556) bank172615 +172616 POINT(41.30408673099492 74.72861922380919) bank172616 +172617 POINT(40.409252825162454 73.5886661486802) bank172617 +172618 POINT(40.94708639300096 74.78017039558654) bank172618 +172619 POINT(41.4896970586086 74.64673843833295) bank172619 +172620 POINT(40.706402563475955 73.86270692103054) bank172620 +172621 POINT(40.722991810959776 74.05557037684451) bank172621 +172622 POINT(40.534898235380766 74.02784148896498) bank172622 +172623 POINT(40.54451875750308 74.34903245655643) bank172623 +172624 POINT(40.22447378211071 73.24780461177112) bank172624 +172625 POINT(39.93109870494989 73.60348642305341) bank172625 +172626 POINT(40.954829464458406 74.30116158632745) bank172626 +172627 POINT(39.97369841033965 73.96404963887831) bank172627 +172628 POINT(40.876166356101514 74.1613964838398) bank172628 +172629 POINT(39.827936310987376 74.92787691153288) bank172629 +172630 POINT(39.92208437817456 73.02525713988085) bank172630 +172631 POINT(41.62415395820143 73.111665419792) bank172631 +172632 POINT(40.757171051333195 74.87231897192727) bank172632 +172633 POINT(41.06081023112806 74.98037230805325) bank172633 +172634 POINT(40.99239603036557 74.29510536834995) bank172634 +172635 POINT(40.454902647070405 73.59988016066855) bank172635 +172636 POINT(40.91170832031388 73.04985472881069) bank172636 +172637 POINT(39.8200112441237 74.97626127350728) bank172637 +172638 POINT(40.254930486560006 73.5684808532445) bank172638 +172639 POINT(41.42566962310141 74.78587690602251) bank172639 +172640 POINT(40.57150832507234 73.8554763282831) bank172640 +172641 POINT(40.88333632469102 73.55584069463879) bank172641 +172642 POINT(40.182942451051886 73.18772808809707) bank172642 +172643 POINT(39.790959533674034 73.18598802548723) bank172643 +172644 POINT(40.94679333469833 74.89663058920576) bank172644 +172645 POINT(39.76486825790344 74.94475867572548) bank172645 +172646 POINT(40.700540670364745 73.05666437258093) bank172646 +172647 POINT(41.33270820445863 74.54939294254282) bank172647 +172648 POINT(40.65589860069113 74.70760307526832) bank172648 +172649 POINT(40.671878213952695 73.87235394219613) bank172649 +172650 POINT(40.11117494112978 74.7431946390006) bank172650 +172651 POINT(40.89098790403848 74.08295020721101) bank172651 +172652 POINT(40.79700130980138 74.00311543904252) bank172652 +172653 POINT(41.43599733055922 74.28362699181356) bank172653 +172654 POINT(40.627528635253846 73.30998761558021) bank172654 +172655 POINT(39.7368097815627 73.48717572079559) bank172655 +172656 POINT(40.53867347344483 73.02254886776161) bank172656 +172657 POINT(40.67091565262285 73.73922963794276) bank172657 +172658 POINT(39.87706654610282 74.01730960524873) bank172658 +172659 POINT(41.053974532495715 74.91840451213453) bank172659 +172660 POINT(40.2931252847478 73.26123222507964) bank172660 +172661 POINT(41.505529298040116 74.38087972748421) bank172661 +172662 POINT(41.13790820629327 74.55387413929635) bank172662 +172663 POINT(40.36345143909518 74.026046286987) bank172663 +172664 POINT(41.410634648724084 74.2555665596676) bank172664 +172665 POINT(40.01563263708732 73.3326804933513) bank172665 +172666 POINT(40.75094039256034 74.19529748564408) bank172666 +172667 POINT(41.51169362797028 73.32728817012902) bank172667 +172668 POINT(41.318516197771295 74.40713500455601) bank172668 +172669 POINT(40.99445638063314 73.77222401579199) bank172669 +172670 POINT(39.74249319984126 73.40857691535841) bank172670 +172671 POINT(40.893337073953795 74.53912256949668) bank172671 +172672 POINT(40.62211804051834 74.18785185768411) bank172672 +172673 POINT(40.373911038181184 73.71097109875457) bank172673 +172674 POINT(40.41238723057978 73.64704264894243) bank172674 +172675 POINT(40.7680061849141 74.52488304185954) bank172675 +172676 POINT(40.769000083995024 74.32863990212786) bank172676 +172677 POINT(41.54761516042223 74.1713712477236) bank172677 +172678 POINT(40.31900779510085 74.06503587078195) bank172678 +172679 POINT(40.287350465941316 73.4558483651743) bank172679 +172680 POINT(41.35093537033625 73.39743017362781) bank172680 +172681 POINT(40.42362818287833 74.68925308989189) bank172681 +172682 POINT(40.88217058004616 73.25637341191374) bank172682 +172683 POINT(39.89325219507058 73.33598830581101) bank172683 +172684 POINT(41.56777097847181 74.006175825432) bank172684 +172685 POINT(41.52183817944983 73.83583469093611) bank172685 +172686 POINT(40.53654059531898 74.76365161770394) bank172686 +172687 POINT(41.00656487515659 74.88222130134439) bank172687 +172688 POINT(40.60637687809313 74.2628303711837) bank172688 +172689 POINT(40.59497805326558 73.7348022512661) bank172689 +172690 POINT(40.98834816616932 73.72739700646649) bank172690 +172691 POINT(40.74892491754433 73.47283433648033) bank172691 +172692 POINT(41.13678747476919 73.208391831672) bank172692 +172693 POINT(40.85786214761426 73.38257937417451) bank172693 +172694 POINT(41.48481069848831 74.27591814840834) bank172694 +172695 POINT(40.89984437488933 73.25408221579934) bank172695 +172696 POINT(41.55921512844159 74.81874916121502) bank172696 +172697 POINT(40.664665741458 74.17215258156993) bank172697 +172698 POINT(40.50690554928761 73.30782019237375) bank172698 +172699 POINT(40.1179421606667 73.0639113272607) bank172699 +172700 POINT(41.46679249838097 73.6486021457707) bank172700 +172701 POINT(41.59551782251083 74.89638655997784) bank172701 +172702 POINT(41.01805021265121 73.25363541471445) bank172702 +172703 POINT(40.05275533424259 74.685582949914) bank172703 +172704 POINT(41.016078693834224 74.4883788579488) bank172704 +172705 POINT(40.04498901569228 73.32964683205672) bank172705 +172706 POINT(41.455674073410535 73.13286590110222) bank172706 +172707 POINT(40.66365304547495 73.37103304581863) bank172707 +172708 POINT(40.046556179811915 73.6624348013332) bank172708 +172709 POINT(40.368403834698526 73.04096560067218) bank172709 +172710 POINT(41.644604554553574 74.02340712665367) bank172710 +172711 POINT(39.98115887633386 74.8805498157116) bank172711 +172712 POINT(41.68679984960178 74.90775047725812) bank172712 +172713 POINT(41.236829367139975 73.83223923603322) bank172713 +172714 POINT(40.09966588977919 73.53317508373145) bank172714 +172715 POINT(40.90566254772768 73.77572101012761) bank172715 +172716 POINT(41.46510107829692 73.73804625227521) bank172716 +172717 POINT(41.666799232852846 73.81215539100417) bank172717 +172718 POINT(41.32306563081187 74.21196876866685) bank172718 +172719 POINT(40.627687080174425 73.15838870551048) bank172719 +172720 POINT(40.46305689592486 73.69183130557522) bank172720 +172721 POINT(39.93100241687145 74.66171715243601) bank172721 +172722 POINT(40.58143644422563 73.76727788347542) bank172722 +172723 POINT(41.15855536413408 73.79633866195857) bank172723 +172724 POINT(41.560888346836244 73.50305901927435) bank172724 +172725 POINT(40.474494574557866 74.12024406334817) bank172725 +172726 POINT(41.39769452118184 73.64796473905535) bank172726 +172727 POINT(41.655344768916976 73.15219516538127) bank172727 +172728 POINT(40.13677393990371 73.17602178878658) bank172728 +172729 POINT(40.54195892819876 74.92231811519399) bank172729 +172730 POINT(41.38269925285338 74.03996486921197) bank172730 +172731 POINT(40.77366917498981 74.98120097299896) bank172731 +172732 POINT(41.41277851556466 73.81307292404695) bank172732 +172733 POINT(40.6198063030031 73.22035997529323) bank172733 +172734 POINT(40.812110787967384 74.375811212091) bank172734 +172735 POINT(41.28844843550053 73.22806559661065) bank172735 +172736 POINT(40.245649128354096 73.18075258245426) bank172736 +172737 POINT(40.330475261606274 74.6555239788683) bank172737 +172738 POINT(40.083731272390295 74.05434792166005) bank172738 +172739 POINT(41.42787229278606 73.24207763123826) bank172739 +172740 POINT(40.933788418638684 74.33481914561673) bank172740 +172741 POINT(40.793891127199345 74.1234184550141) bank172741 +172742 POINT(40.48009477165382 73.72146147318536) bank172742 +172743 POINT(40.780494320567904 73.83029887901915) bank172743 +172744 POINT(41.17267149262682 74.05054341185694) bank172744 +172745 POINT(39.79393952424668 73.66101587793666) bank172745 +172746 POINT(39.78250279740407 73.51795831972824) bank172746 +172747 POINT(40.114465215656864 73.92388527234213) bank172747 +172748 POINT(41.49030159473802 74.40143245893142) bank172748 +172749 POINT(40.4787650516829 73.71030868548795) bank172749 +172750 POINT(41.19561621458813 74.37284643937404) bank172750 +172751 POINT(40.90932907268278 73.40874923722103) bank172751 +172752 POINT(40.17240101723218 74.3005935762013) bank172752 +172753 POINT(40.22903721065147 73.45649867164784) bank172753 +172754 POINT(40.102506307828385 74.60216034789835) bank172754 +172755 POINT(40.833515125667915 74.23581044119442) bank172755 +172756 POINT(41.129174588280165 73.9713098691289) bank172756 +172757 POINT(40.08189341319398 74.06772330593695) bank172757 +172758 POINT(39.773139874727185 74.66204153640506) bank172758 +172759 POINT(39.72692469789113 74.87946147864044) bank172759 +172760 POINT(40.64613333804508 73.66795361509035) bank172760 +172761 POINT(40.06285645735848 73.32656131297647) bank172761 +172762 POINT(40.568541785492876 74.04936968543711) bank172762 +172763 POINT(39.80235529906001 74.06731286629724) bank172763 +172764 POINT(40.07392699657438 74.32799778322536) bank172764 +172765 POINT(39.882433233623274 74.3358458071743) bank172765 +172766 POINT(41.07384551360864 73.07388478210514) bank172766 +172767 POINT(41.27499259529923 73.22436285941197) bank172767 +172768 POINT(40.449518829605516 74.88025139082708) bank172768 +172769 POINT(41.421187015748266 74.32266037588417) bank172769 +172770 POINT(41.18488096956667 73.49401245648512) bank172770 +172771 POINT(41.48417030408799 73.02236158924934) bank172771 +172772 POINT(40.79828976425676 74.59826523858987) bank172772 +172773 POINT(40.73098340660851 73.84506575554364) bank172773 +172774 POINT(40.44229983550177 74.04094634426184) bank172774 +172775 POINT(40.82042218638918 73.44937246057341) bank172775 +172776 POINT(40.60994798861026 74.85348754190629) bank172776 +172777 POINT(40.02462712113054 73.75819150464609) bank172777 +172778 POINT(40.41320171009552 73.52699442334321) bank172778 +172779 POINT(41.06403632758057 74.28836044394251) bank172779 +172780 POINT(40.1591773211 74.3467616290809) bank172780 +172781 POINT(41.24357932624585 74.99426952817109) bank172781 +172782 POINT(40.70243809899576 74.7825957499535) bank172782 +172783 POINT(41.61048790390425 74.41461813815843) bank172783 +172784 POINT(41.474978701941524 74.03081814892107) bank172784 +172785 POINT(40.54344670597014 73.43495550475193) bank172785 +172786 POINT(40.21359218891263 73.87723506259425) bank172786 +172787 POINT(39.78668135123773 74.81321269643466) bank172787 +172788 POINT(39.847698164713414 73.89345635836429) bank172788 +172789 POINT(41.710488817711116 74.73675178055966) bank172789 +172790 POINT(40.693163152593186 73.87940237262282) bank172790 +172791 POINT(40.08937051532825 74.402468755066) bank172791 +172792 POINT(39.876315639447625 74.83860333470777) bank172792 +172793 POINT(41.61800652872406 73.62586162754059) bank172793 +172794 POINT(41.65684707259079 73.09480217394598) bank172794 +172795 POINT(40.508955377979945 74.3794330957583) bank172795 +172796 POINT(41.69665228175542 74.66093362256962) bank172796 +172797 POINT(40.32788255294788 74.39769099417188) bank172797 +172798 POINT(40.81399467293063 74.6238690186909) bank172798 +172799 POINT(40.188815479825124 73.68786544462559) bank172799 +172800 POINT(40.9899807329563 73.45348627381082) bank172800 +172801 POINT(41.39355181677814 73.07032956576423) bank172801 +172802 POINT(40.63420549128447 73.15677107576457) bank172802 +172803 POINT(40.26723968009613 74.58882413135689) bank172803 +172804 POINT(40.922622016142924 74.3125968304454) bank172804 +172805 POINT(41.38388123978347 74.85221857094506) bank172805 +172806 POINT(40.07162688161006 74.06855081290036) bank172806 +172807 POINT(40.265865614943 74.38775424424436) bank172807 +172808 POINT(40.060597132636 73.45558835933664) bank172808 +172809 POINT(41.28397242818879 74.17061308696744) bank172809 +172810 POINT(40.64652230470797 73.81138603562165) bank172810 +172811 POINT(41.27886403144155 74.42791346590805) bank172811 +172812 POINT(40.833228888417345 74.76666856750931) bank172812 +172813 POINT(39.962061286821815 74.72834347175386) bank172813 +172814 POINT(41.702138802744585 74.60430834084605) bank172814 +172815 POINT(40.61507410789747 73.88489379703883) bank172815 +172816 POINT(41.243899475793 73.11827854473992) bank172816 +172817 POINT(40.93538294399808 73.21358939951547) bank172817 +172818 POINT(40.845756742223 73.67885229942695) bank172818 +172819 POINT(40.56806941268896 73.03761722316268) bank172819 +172820 POINT(40.46259937044251 73.76261876971765) bank172820 +172821 POINT(40.49136674038376 74.14842210436284) bank172821 +172822 POINT(40.56943178893838 74.01718882511072) bank172822 +172823 POINT(40.91786468932912 73.48653468671054) bank172823 +172824 POINT(41.48160314302556 74.76436425899641) bank172824 +172825 POINT(39.80687339820574 74.56056343863348) bank172825 +172826 POINT(40.87400005177478 74.7690261442897) bank172826 +172827 POINT(40.00757984336947 74.20904141235044) bank172827 +172828 POINT(41.16384546181607 73.22088638996065) bank172828 +172829 POINT(40.807359038566865 74.89570328895668) bank172829 +172830 POINT(40.766779023371086 74.18390643023292) bank172830 +172831 POINT(40.58996409132016 74.86385903109736) bank172831 +172832 POINT(41.49019632667092 73.86310358052337) bank172832 +172833 POINT(40.651132490081075 74.9035164603937) bank172833 +172834 POINT(40.04676669724705 73.3867733724756) bank172834 +172835 POINT(39.76940281433418 73.3467992578677) bank172835 +172836 POINT(40.08918323789193 73.22039500275454) bank172836 +172837 POINT(40.205543130927 74.5819531277175) bank172837 +172838 POINT(40.99066639609205 74.09085693293903) bank172838 +172839 POINT(39.78292068193356 73.59170290647639) bank172839 +172840 POINT(41.40829904109353 74.4752065283366) bank172840 +172841 POINT(39.89705310578807 74.58762823267739) bank172841 +172842 POINT(39.90729167846627 73.3039672026155) bank172842 +172843 POINT(39.8407825794784 74.70604061192242) bank172843 +172844 POINT(41.05146580476793 74.57303451333273) bank172844 +172845 POINT(40.24573584941752 74.74224553768634) bank172845 +172846 POINT(39.88440811068072 73.02655509529136) bank172846 +172847 POINT(40.63678473185909 74.34963395133042) bank172847 +172848 POINT(40.96328579717166 73.6342069688507) bank172848 +172849 POINT(39.75532237432328 74.5978157937303) bank172849 +172850 POINT(41.560163260047695 74.79335695131692) bank172850 +172851 POINT(41.4740719710794 74.816485723326) bank172851 +172852 POINT(41.48637210428806 73.08146396948365) bank172852 +172853 POINT(39.76037873954112 74.83118510495174) bank172853 +172854 POINT(41.29825095475462 74.99009054107835) bank172854 +172855 POINT(41.111561522356766 74.80137510671884) bank172855 +172856 POINT(40.149825012004 73.4360308057319) bank172856 +172857 POINT(41.02778850716217 73.37151664209748) bank172857 +172858 POINT(40.62180243726714 74.00126825595234) bank172858 +172859 POINT(40.346987676828206 74.4271806233828) bank172859 +172860 POINT(40.64924128607399 74.76435063701068) bank172860 +172861 POINT(40.8716041679538 73.30971839137065) bank172861 +172862 POINT(40.66778402272787 73.68150478577762) bank172862 +172863 POINT(41.14670001983447 74.04681530514831) bank172863 +172864 POINT(40.968082290732745 74.11764743181745) bank172864 +172865 POINT(40.977834141962674 73.58011213366603) bank172865 +172866 POINT(39.95844413572681 73.28875517069748) bank172866 +172867 POINT(40.6704709037921 73.347070498957) bank172867 +172868 POINT(39.93414083716978 74.94472542316886) bank172868 +172869 POINT(40.16395793696977 73.6195896343782) bank172869 +172870 POINT(40.51611495104262 73.93315291639964) bank172870 +172871 POINT(41.32370414432851 73.39371604578528) bank172871 +172872 POINT(40.40094129217951 74.88769558196677) bank172872 +172873 POINT(41.12754389202225 74.30941153116116) bank172873 +172874 POINT(41.53716558962936 73.9747075745965) bank172874 +172875 POINT(41.45519981919641 74.53861398719282) bank172875 +172876 POINT(40.462086393513474 73.24536174137845) bank172876 +172877 POINT(39.766907572136546 73.86552065654695) bank172877 +172878 POINT(41.15800512172897 74.07537589635027) bank172878 +172879 POINT(40.85669711339723 74.16767512385236) bank172879 +172880 POINT(39.955647308375504 74.29614299599636) bank172880 +172881 POINT(40.69493213968243 74.42052517960732) bank172881 +172882 POINT(41.44418237464395 74.63415276000985) bank172882 +172883 POINT(41.438172992388225 73.75618498183977) bank172883 +172884 POINT(40.490309193871866 74.20542255470586) bank172884 +172885 POINT(41.67200234010572 74.78280990205143) bank172885 +172886 POINT(40.68899479025929 73.01451526385951) bank172886 +172887 POINT(41.6116470754024 73.77861379266562) bank172887 +172888 POINT(41.5027017450463 73.79927900934406) bank172888 +172889 POINT(39.84984509098009 74.51172905610905) bank172889 +172890 POINT(41.17518554194934 74.95549841140094) bank172890 +172891 POINT(41.605166727781736 73.25558789253888) bank172891 +172892 POINT(40.12601488175235 73.5837183359911) bank172892 +172893 POINT(40.076584974366654 73.37124661889312) bank172893 +172894 POINT(41.27930851487735 74.96073260280909) bank172894 +172895 POINT(39.79375100454557 74.51125531600717) bank172895 +172896 POINT(41.559006362103155 74.42376331027694) bank172896 +172897 POINT(40.354754264022404 74.22895369762782) bank172897 +172898 POINT(40.56654182743009 74.34060333581016) bank172898 +172899 POINT(41.04281461519471 74.87208725476313) bank172899 +172900 POINT(40.00841654994723 74.36770300761468) bank172900 +172901 POINT(41.38702696992893 74.58665681871454) bank172901 +172902 POINT(40.74051000069315 73.94317070053769) bank172902 +172903 POINT(40.715620955569136 73.06130598230699) bank172903 +172904 POINT(41.03344530144532 73.52629142683509) bank172904 +172905 POINT(41.458450502706654 74.91147769980076) bank172905 +172906 POINT(40.11530154964617 74.59001379574416) bank172906 +172907 POINT(40.935055377371775 74.88538384573872) bank172907 +172908 POINT(41.648889174437954 74.27241161895982) bank172908 +172909 POINT(40.3540978708509 74.13227621679567) bank172909 +172910 POINT(41.22488708842839 73.6458476015452) bank172910 +172911 POINT(41.050360000479614 73.37190275036664) bank172911 +172912 POINT(40.37311959521518 74.27704014866414) bank172912 +172913 POINT(39.71620097054849 73.62838367149443) bank172913 +172914 POINT(41.05833778729266 74.48152033416045) bank172914 +172915 POINT(41.00423031559279 74.20980202583434) bank172915 +172916 POINT(41.69798375992801 74.26056546749598) bank172916 +172917 POINT(40.67406501364414 74.98938777760895) bank172917 +172918 POINT(40.63058401491401 74.0485543410967) bank172918 +172919 POINT(40.61536953250114 74.82339615745298) bank172919 +172920 POINT(40.0840338100687 74.7199085521573) bank172920 +172921 POINT(40.1945667004675 73.5211720219437) bank172921 +172922 POINT(40.66379268168851 73.09320717844348) bank172922 +172923 POINT(41.32246152281204 73.17335096202554) bank172923 +172924 POINT(40.604698734784904 73.37395761918216) bank172924 +172925 POINT(41.55564209412128 73.50310188158825) bank172925 +172926 POINT(41.276542967371675 73.81502115614649) bank172926 +172927 POINT(39.74861228035409 74.27330283976222) bank172927 +172928 POINT(40.17953034325702 73.47686110444486) bank172928 +172929 POINT(41.32360487442252 73.1966858615207) bank172929 +172930 POINT(40.42671014012246 74.84401638438919) bank172930 +172931 POINT(41.579729893658126 74.6125035867125) bank172931 +172932 POINT(40.626807176454506 74.50069699608963) bank172932 +172933 POINT(41.30752371088106 73.6282027929923) bank172933 +172934 POINT(39.93364811701995 74.7771524517727) bank172934 +172935 POINT(41.20552662482609 74.12756971065967) bank172935 +172936 POINT(40.251331182297974 73.17026637681876) bank172936 +172937 POINT(41.69749777781648 74.19283674818362) bank172937 +172938 POINT(41.153203833073555 73.86306401196646) bank172938 +172939 POINT(41.389761673393785 73.44445669782824) bank172939 +172940 POINT(41.6312293927556 74.1796921433636) bank172940 +172941 POINT(40.9356252752069 74.58199334514664) bank172941 +172942 POINT(41.097452878300835 74.71634540186841) bank172942 +172943 POINT(41.5844684837011 73.55986523597079) bank172943 +172944 POINT(40.65839252869118 73.04987057506483) bank172944 +172945 POINT(41.45569131168011 73.71825528318661) bank172945 +172946 POINT(41.61675859750087 74.68487703251944) bank172946 +172947 POINT(40.037580806623176 74.12796783523439) bank172947 +172948 POINT(39.807837141676124 73.14739854216293) bank172948 +172949 POINT(41.66440231813775 73.65575362034693) bank172949 +172950 POINT(41.26399738166523 74.47250915674165) bank172950 +172951 POINT(40.19844116992607 74.6819108981671) bank172951 +172952 POINT(41.70512170843449 74.48412071021433) bank172952 +172953 POINT(41.54656179507383 74.74302894855504) bank172953 +172954 POINT(40.46748333020972 73.80076446013372) bank172954 +172955 POINT(39.837800510706934 74.92680586559277) bank172955 +172956 POINT(40.422984301877314 74.02197914296575) bank172956 +172957 POINT(39.88366229109869 73.41582200163433) bank172957 +172958 POINT(40.432227075547935 73.65216149637726) bank172958 +172959 POINT(39.75599349600672 74.48247711720259) bank172959 +172960 POINT(39.902170981778994 73.57546340242648) bank172960 +172961 POINT(41.66150958613531 74.96902329685345) bank172961 +172962 POINT(39.94500161420402 73.08825818673103) bank172962 +172963 POINT(41.070554029949626 74.72528229047404) bank172963 +172964 POINT(40.96446355361137 74.43359396622736) bank172964 +172965 POINT(40.31908521655724 74.29951257299356) bank172965 +172966 POINT(40.89547485957843 74.63775330721703) bank172966 +172967 POINT(39.89145197076106 74.89549949405317) bank172967 +172968 POINT(41.05541799989713 73.16184161862408) bank172968 +172969 POINT(39.73371866528646 74.87132338539111) bank172969 +172970 POINT(41.6301930521753 74.46502182663231) bank172970 +172971 POINT(40.39068772836015 73.81979336594628) bank172971 +172972 POINT(40.86267366800372 73.48956678102766) bank172972 +172973 POINT(41.53527581975924 74.78731981935108) bank172973 +172974 POINT(41.57120153101255 74.68337041234558) bank172974 +172975 POINT(41.11616290258966 74.22623707570901) bank172975 +172976 POINT(41.48880554183099 73.49351535110294) bank172976 +172977 POINT(41.28783004313401 74.34829992076553) bank172977 +172978 POINT(41.423762516172566 74.70662941728965) bank172978 +172979 POINT(40.69354163241418 74.34668918938881) bank172979 +172980 POINT(41.57994017281833 74.42667101310992) bank172980 +172981 POINT(40.931503752994125 74.8831812062749) bank172981 +172982 POINT(40.49809874978158 73.71777538787113) bank172982 +172983 POINT(40.31672765769835 74.54940435228986) bank172983 +172984 POINT(41.628234529707946 73.39883714796923) bank172984 +172985 POINT(41.060160934833476 73.09874544665347) bank172985 +172986 POINT(41.27765823773739 74.13025055919758) bank172986 +172987 POINT(39.7680643108224 73.20461033336132) bank172987 +172988 POINT(40.725090103909665 74.72552005939855) bank172988 +172989 POINT(41.606782188754146 73.39947250912849) bank172989 +172990 POINT(39.717209038323496 74.80878399706755) bank172990 +172991 POINT(40.719669438061636 74.35027466016615) bank172991 +172992 POINT(40.621030564216944 73.79328658618319) bank172992 +172993 POINT(40.9800781611697 73.49484261063151) bank172993 +172994 POINT(40.776471530427116 73.87881095204344) bank172994 +172995 POINT(40.906616254115185 73.3363430322186) bank172995 +172996 POINT(41.38793987617601 73.23363138102046) bank172996 +172997 POINT(41.61678345956455 74.76340326872324) bank172997 +172998 POINT(41.083565883531094 74.17225802719565) bank172998 +172999 POINT(41.04205489031522 73.88003217280675) bank172999 +173000 POINT(40.55940121621333 73.88291009857502) bank173000 +173001 POINT(40.816258901484375 74.1170711264868) bank173001 +173002 POINT(41.07956488998679 74.49782566747832) bank173002 +173003 POINT(39.96401635429052 74.16716068452132) bank173003 +173004 POINT(41.59559749778967 73.69658045635265) bank173004 +173005 POINT(40.11000353630171 73.63849753180696) bank173005 +173006 POINT(39.80291292092315 74.84954989688927) bank173006 +173007 POINT(39.81937937862499 73.49700230058635) bank173007 +173008 POINT(40.71090105551277 73.10297429609926) bank173008 +173009 POINT(39.87334792046583 74.37921370604393) bank173009 +173010 POINT(40.175268283248045 74.14461319535381) bank173010 +173011 POINT(40.613059996899715 73.32411056710836) bank173011 +173012 POINT(40.89158549766226 74.49853363960838) bank173012 +173013 POINT(40.37580488283542 74.86036732385962) bank173013 +173014 POINT(40.029469418147 73.34886614540632) bank173014 +173015 POINT(41.25199569272584 73.69463027762092) bank173015 +173016 POINT(39.921451673180584 73.02558069409153) bank173016 +173017 POINT(41.7055489325816 73.88748545699546) bank173017 +173018 POINT(39.74977792895051 74.07149159161588) bank173018 +173019 POINT(39.943287744961275 73.84525720431105) bank173019 +173020 POINT(41.33586698025896 73.38135466048632) bank173020 +173021 POINT(40.80808440614454 73.6626306272314) bank173021 +173022 POINT(39.80396885507096 73.77970264577668) bank173022 +173023 POINT(41.04724424304291 73.89813318779126) bank173023 +173024 POINT(40.537473336432974 74.45092729591063) bank173024 +173025 POINT(41.18783780408602 74.44843890515001) bank173025 +173026 POINT(41.45911163932491 73.1950569367404) bank173026 +173027 POINT(40.03561820695038 73.34122601736607) bank173027 +173028 POINT(41.31403347157533 73.06543554241296) bank173028 +173029 POINT(39.756464064689865 74.52464761059228) bank173029 +173030 POINT(40.046934626195686 74.22070106691267) bank173030 +173031 POINT(40.593697567675115 73.54128505499453) bank173031 +173032 POINT(39.88830791060446 74.32195050204385) bank173032 +173033 POINT(39.73783074237153 73.00923064576556) bank173033 +173034 POINT(40.71165108278166 73.91032727883112) bank173034 +173035 POINT(41.36614904492306 74.8893781098346) bank173035 +173036 POINT(40.600555572818585 74.40182007305235) bank173036 +173037 POINT(41.217958280491935 74.52375281550384) bank173037 +173038 POINT(41.549474854518685 74.60910880105435) bank173038 +173039 POINT(40.10787737979576 74.1321555401192) bank173039 +173040 POINT(40.59780530951353 73.40572209554607) bank173040 +173041 POINT(39.98267989269392 74.85248808951638) bank173041 +173042 POINT(41.54703407578647 74.53717890621554) bank173042 +173043 POINT(39.95858857398476 73.75944897168758) bank173043 +173044 POINT(41.184366589258225 73.37133690945808) bank173044 +173045 POINT(40.342385796681825 74.29575291218985) bank173045 +173046 POINT(40.18692052037348 74.08956804826076) bank173046 +173047 POINT(40.57619499463428 73.21143764275094) bank173047 +173048 POINT(40.98684121220337 73.48336107007668) bank173048 +173049 POINT(41.02000913211975 74.02936425541418) bank173049 +173050 POINT(40.06825995605268 74.04853728024267) bank173050 +173051 POINT(40.868301039205605 73.36327032132374) bank173051 +173052 POINT(40.8793502713166 74.60166262710908) bank173052 +173053 POINT(39.77475761741173 74.06649135461173) bank173053 +173054 POINT(40.766315954626506 74.51558511776112) bank173054 +173055 POINT(40.538344031348636 74.83757654835495) bank173055 +173056 POINT(39.73771236207587 73.74098627077335) bank173056 +173057 POINT(40.46302520186549 74.94238781938834) bank173057 +173058 POINT(41.61792269558122 74.81232556096694) bank173058 +173059 POINT(40.20286416014515 73.92821330997607) bank173059 +173060 POINT(41.05776650617994 74.83908439108717) bank173060 +173061 POINT(41.099572748072696 74.19600043288443) bank173061 +173062 POINT(40.43540232305941 73.7436660190065) bank173062 +173063 POINT(40.56341470260697 73.65007461707621) bank173063 +173064 POINT(40.09069015039444 74.20425858340141) bank173064 +173065 POINT(41.09817916780481 73.91923592933708) bank173065 +173066 POINT(41.52693986253806 73.6074174090904) bank173066 +173067 POINT(39.98049414070774 73.92121566992455) bank173067 +173068 POINT(41.43623843374324 74.5574671348548) bank173068 +173069 POINT(41.06855976035236 73.43133318037455) bank173069 +173070 POINT(40.35180591703464 74.09519806179897) bank173070 +173071 POINT(39.73643426078696 73.17556938355557) bank173071 +173072 POINT(39.74558638045202 74.82469805037809) bank173072 +173073 POINT(40.26031671576565 73.18856169680402) bank173073 +173074 POINT(41.51321708660945 74.64853793290708) bank173074 +173075 POINT(40.47332706943336 74.50526559363654) bank173075 +173076 POINT(40.88963669127303 73.57172584067246) bank173076 +173077 POINT(41.611069254771785 74.1731270343423) bank173077 +173078 POINT(40.550829661327406 73.03751513649031) bank173078 +173079 POINT(41.09452937868154 74.91453373212943) bank173079 +173080 POINT(40.085759719321835 74.14126127984491) bank173080 +173081 POINT(40.06993129470978 74.2839157197255) bank173081 +173082 POINT(40.700793600275134 73.16393385346782) bank173082 +173083 POINT(40.38911834198977 74.97376404600706) bank173083 +173084 POINT(41.380459825387554 73.67139385428922) bank173084 +173085 POINT(41.430097753291754 73.73420887050204) bank173085 +173086 POINT(41.28421018044034 73.69023122043393) bank173086 +173087 POINT(40.088377928188464 74.71089485620224) bank173087 +173088 POINT(39.753850388322185 74.01860683719434) bank173088 +173089 POINT(41.22532772222891 73.47123099577301) bank173089 +173090 POINT(40.9550547623857 73.14453853509887) bank173090 +173091 POINT(40.48720933373841 74.74225632895067) bank173091 +173092 POINT(41.59188642879546 73.90012711542362) bank173092 +173093 POINT(41.09750605475556 74.44681105423129) bank173093 +173094 POINT(41.662890330725496 73.38003454945004) bank173094 +173095 POINT(39.71447589521981 73.82838407333149) bank173095 +173096 POINT(41.3489882417352 73.03046466174003) bank173096 +173097 POINT(41.37670238213098 74.53370535394724) bank173097 +173098 POINT(40.25477972017175 73.65746994817898) bank173098 +173099 POINT(40.652671068181576 73.4563533543314) bank173099 +173100 POINT(41.52072850385617 74.90262232817378) bank173100 +173101 POINT(40.38253326951464 73.59868578382515) bank173101 +173102 POINT(41.52828484893577 74.97522673102084) bank173102 +173103 POINT(41.56883896523785 74.76054805520786) bank173103 +173104 POINT(41.205210299209135 74.93364963487915) bank173104 +173105 POINT(40.98466682020791 73.65208280498736) bank173105 +173106 POINT(41.693780515709406 74.31386699728151) bank173106 +173107 POINT(41.27314625940923 73.76107916177881) bank173107 +173108 POINT(41.05833673912375 74.47405194503219) bank173108 +173109 POINT(41.256095267064985 73.1370168947413) bank173109 +173110 POINT(40.15339159613505 74.04049926137326) bank173110 +173111 POINT(40.73285896380471 74.5553914320897) bank173111 +173112 POINT(41.180915678612735 73.82477394003696) bank173112 +173113 POINT(40.15531900992549 74.4995752552375) bank173113 +173114 POINT(40.49946199365019 73.498125144452) bank173114 +173115 POINT(39.91460773063631 73.90949669395192) bank173115 +173116 POINT(40.87169005531716 73.81183715729284) bank173116 +173117 POINT(41.5242488773894 74.61503469906744) bank173117 +173118 POINT(40.07934185283237 73.49844325494058) bank173118 +173119 POINT(41.046457833176945 73.08119330962607) bank173119 +173120 POINT(40.48120219037244 73.76763871640956) bank173120 +173121 POINT(40.94789270701523 73.84472362590338) bank173121 +173122 POINT(40.521450351259475 74.32019437130542) bank173122 +173123 POINT(40.1164199273051 73.99704803661876) bank173123 +173124 POINT(41.53597962198077 74.8895508171737) bank173124 +173125 POINT(41.34790937125026 73.2445294473537) bank173125 +173126 POINT(41.62880895036732 73.55394880259459) bank173126 +173127 POINT(40.784573393739564 74.04406119848203) bank173127 +173128 POINT(41.61786512729444 73.56465060392091) bank173128 +173129 POINT(40.717512632621016 74.6054651768187) bank173129 +173130 POINT(40.84804478448592 74.77447460342134) bank173130 +173131 POINT(40.17106784830513 73.94965543621028) bank173131 +173132 POINT(41.451712276517036 73.15873421585911) bank173132 +173133 POINT(40.965722974383404 74.91356217534468) bank173133 +173134 POINT(41.220742437939165 73.54047328875635) bank173134 +173135 POINT(39.84166692908722 74.43124739849503) bank173135 +173136 POINT(40.68768402034497 74.50789004852813) bank173136 +173137 POINT(39.75645065300615 74.19247871011373) bank173137 +173138 POINT(40.08489043056697 73.51491013325078) bank173138 +173139 POINT(41.35403077332204 73.92603554896702) bank173139 +173140 POINT(41.44992692934902 73.213556172972) bank173140 +173141 POINT(41.47691995991414 74.67401681699599) bank173141 +173142 POINT(41.23996423612821 73.05318729497588) bank173142 +173143 POINT(41.477344558141766 73.12218398280062) bank173143 +173144 POINT(40.38231242977227 74.0584947673469) bank173144 +173145 POINT(40.47710711007481 73.26303661398299) bank173145 +173146 POINT(40.00369897321819 74.71405337503792) bank173146 +173147 POINT(41.4269024583189 74.70362244374053) bank173147 +173148 POINT(40.547407249028296 74.67460690310484) bank173148 +173149 POINT(40.102196706368204 73.30952648451398) bank173149 +173150 POINT(40.33678380798932 73.21841883022067) bank173150 +173151 POINT(40.19205860510895 73.17312255580737) bank173151 +173152 POINT(40.95247075348337 73.97496591893793) bank173152 +173153 POINT(39.994463785772574 74.839683215218) bank173153 +173154 POINT(41.318355429429594 74.20228081789344) bank173154 +173155 POINT(40.19097905329902 74.33783831449655) bank173155 +173156 POINT(41.39927856847479 73.47335128672836) bank173156 +173157 POINT(41.41541150923439 73.80709975481436) bank173157 +173158 POINT(41.59444877336689 73.04646988655645) bank173158 +173159 POINT(40.33987496084183 73.36589425769628) bank173159 +173160 POINT(41.56696669157881 74.85981573804051) bank173160 +173161 POINT(41.67408392977865 74.69611835618568) bank173161 +173162 POINT(41.49472398443871 74.91639179870687) bank173162 +173163 POINT(39.87635306900016 74.35490292535607) bank173163 +173164 POINT(40.49754443483726 73.20483150413332) bank173164 +173165 POINT(40.55602850433599 73.73554475000547) bank173165 +173166 POINT(41.042385900646266 73.32076817404852) bank173166 +173167 POINT(41.675532741196434 73.64402912202966) bank173167 +173168 POINT(40.64671068734418 74.39707481466547) bank173168 +173169 POINT(39.73979989648978 73.092901018156) bank173169 +173170 POINT(41.081064733802435 73.57309825023161) bank173170 +173171 POINT(39.765044115429866 74.54144027560409) bank173171 +173172 POINT(40.56313157877429 73.18901582042919) bank173172 +173173 POINT(41.1482002096399 73.04923011103529) bank173173 +173174 POINT(40.54600112131097 74.7910218473524) bank173174 +173175 POINT(41.111563602137196 74.87724824082692) bank173175 +173176 POINT(40.998856383874525 73.6169407479803) bank173176 +173177 POINT(40.90729086583854 73.21894346267047) bank173177 +173178 POINT(40.94130134912368 73.53081272360541) bank173178 +173179 POINT(41.29606951300712 73.2086741467596) bank173179 +173180 POINT(41.59938445993865 73.50238961184583) bank173180 +173181 POINT(40.65667125807654 73.72972649648105) bank173181 +173182 POINT(40.76298730436755 74.66306985921398) bank173182 +173183 POINT(41.69571081091934 73.78468537790704) bank173183 +173184 POINT(41.22746592663861 73.24935166624887) bank173184 +173185 POINT(40.429387755762015 74.3120779085633) bank173185 +173186 POINT(41.58473953696567 74.0773382415533) bank173186 +173187 POINT(40.92125666353148 74.2830441091525) bank173187 +173188 POINT(40.2555649071275 74.70924500760034) bank173188 +173189 POINT(40.32302675777572 74.90823249648068) bank173189 +173190 POINT(40.994036544312436 74.10397495215265) bank173190 +173191 POINT(40.85737822482674 73.59515650758372) bank173191 +173192 POINT(40.22734664870652 73.24113799956376) bank173192 +173193 POINT(40.76062954158867 74.66456916861917) bank173193 +173194 POINT(39.944484803306274 73.61115378890779) bank173194 +173195 POINT(40.414880827094116 73.47572892342936) bank173195 +173196 POINT(41.62264767730715 74.27884794794255) bank173196 +173197 POINT(40.34810640276037 74.74779221711341) bank173197 +173198 POINT(40.47959017821699 74.40243061158064) bank173198 +173199 POINT(40.298246352383 74.83179911885782) bank173199 +173200 POINT(40.6532050248165 74.43619614151362) bank173200 +173201 POINT(40.474760055415686 73.07652263077324) bank173201 +173202 POINT(40.278126804338406 75.00109047853425) bank173202 +173203 POINT(41.39910483370383 74.98989326783634) bank173203 +173204 POINT(41.001586944345725 73.91953360254094) bank173204 +173205 POINT(40.26726035137326 73.34769117125893) bank173205 +173206 POINT(41.37216280161246 73.04853462755939) bank173206 +173207 POINT(40.29127105276786 73.3576207761663) bank173207 +173208 POINT(40.844694199251904 74.25226128730277) bank173208 +173209 POINT(40.09430842025896 73.11760929832158) bank173209 +173210 POINT(40.9691722613009 74.0994581791664) bank173210 +173211 POINT(39.87391210250383 74.486027881237) bank173211 +173212 POINT(41.19745453096731 73.20923739365864) bank173212 +173213 POINT(41.26900468526656 73.54356775284936) bank173213 +173214 POINT(41.098301834905506 74.55873402086355) bank173214 +173215 POINT(40.794345568187204 74.58475696744476) bank173215 +173216 POINT(41.59613665143559 74.87113249217239) bank173216 +173217 POINT(40.820188153533415 73.18374063938795) bank173217 +173218 POINT(41.109180132465575 73.25298947872669) bank173218 +173219 POINT(40.87148532373033 74.98486436976346) bank173219 +173220 POINT(40.65063198332652 74.73954871654995) bank173220 +173221 POINT(39.78701244745918 74.88619778412759) bank173221 +173222 POINT(41.37582603254539 73.17883730529027) bank173222 +173223 POINT(41.68065707438006 74.58242754369998) bank173223 +173224 POINT(41.10108805622439 73.70935025907195) bank173224 +173225 POINT(39.87664314670113 73.9508544768225) bank173225 +173226 POINT(40.957894361672174 74.84940594347908) bank173226 +173227 POINT(40.65717471505661 74.874983220888) bank173227 +173228 POINT(40.06052827033578 74.34686813989936) bank173228 +173229 POINT(40.95904929462432 74.52272267230836) bank173229 +173230 POINT(41.501240635130216 73.45324775055884) bank173230 +173231 POINT(40.752398779142496 74.61783252617619) bank173231 +173232 POINT(40.953510916408526 74.82448865595717) bank173232 +173233 POINT(40.480857926579255 74.52260626628278) bank173233 +173234 POINT(40.893379434288235 74.32131617625777) bank173234 +173235 POINT(40.97551432603025 73.58998859151485) bank173235 +173236 POINT(40.65367978508939 74.25218431711632) bank173236 +173237 POINT(39.84818104755641 74.02667008206706) bank173237 +173238 POINT(40.87885241070656 74.20115262453896) bank173238 +173239 POINT(41.67537405358197 74.1479268423421) bank173239 +173240 POINT(40.721347640396154 74.47405426892293) bank173240 +173241 POINT(40.65226869820629 73.95944912774729) bank173241 +173242 POINT(41.37470861144812 73.98210146048507) bank173242 +173243 POINT(39.7637185873008 73.99768458068401) bank173243 +173244 POINT(41.07591873937938 74.59514360558902) bank173244 +173245 POINT(40.570280442519845 74.88501458471302) bank173245 +173246 POINT(40.031596953696194 73.86522387493696) bank173246 +173247 POINT(41.66544045325058 74.80106423226277) bank173247 +173248 POINT(39.97472241721779 73.75607815187779) bank173248 +173249 POINT(40.143646279918876 74.6013151312548) bank173249 +173250 POINT(39.73357846389848 74.5747118292623) bank173250 +173251 POINT(40.33657680026338 73.4299090627757) bank173251 +173252 POINT(40.17361865438554 73.6974314751198) bank173252 +173253 POINT(40.05341890953329 73.89252853449764) bank173253 +173254 POINT(40.485323169548415 73.89128949732691) bank173254 +173255 POINT(40.17062896639343 74.35133824600499) bank173255 +173256 POINT(40.725735839510634 74.43645266483814) bank173256 +173257 POINT(41.34205679631897 74.20185510664774) bank173257 +173258 POINT(40.98393309154173 73.83765324831336) bank173258 +173259 POINT(40.939122021444504 73.20166801852807) bank173259 +173260 POINT(40.93753067658152 74.12860115897867) bank173260 +173261 POINT(40.209108340716206 74.89118985995958) bank173261 +173262 POINT(40.5467765683964 74.65963016760223) bank173262 +173263 POINT(40.18675799595356 73.91961847160385) bank173263 +173264 POINT(41.31366765808827 73.01075116071304) bank173264 +173265 POINT(40.31040993114611 74.73801005283799) bank173265 +173266 POINT(40.8307969203835 74.50590552603215) bank173266 +173267 POINT(40.76153735389069 73.07166260562384) bank173267 +173268 POINT(40.27682245030679 74.14209129837643) bank173268 +173269 POINT(41.02036797607408 74.84683144554074) bank173269 +173270 POINT(39.96604078690242 74.83159844542858) bank173270 +173271 POINT(39.716507881887864 73.05775779104543) bank173271 +173272 POINT(40.74619363336323 73.1233068960355) bank173272 +173273 POINT(39.74082363740808 74.27757984545703) bank173273 +173274 POINT(41.08830629701177 74.11406929741848) bank173274 +173275 POINT(41.53503719831069 73.40375300424198) bank173275 +173276 POINT(41.11393606087445 74.04544009349843) bank173276 +173277 POINT(41.58742510544086 73.31146441723854) bank173277 +173278 POINT(41.428161643320706 74.47600126814275) bank173278 +173279 POINT(41.281666213284545 73.03178057169603) bank173279 +173280 POINT(40.80495634181347 73.20575223528863) bank173280 +173281 POINT(40.27536290392303 74.77054432464911) bank173281 +173282 POINT(41.36311226228 74.18543618196652) bank173282 +173283 POINT(40.89728350856776 74.6569477218856) bank173283 +173284 POINT(39.99865371001742 74.84911460020028) bank173284 +173285 POINT(41.5833545590434 74.69762754945127) bank173285 +173286 POINT(40.9757596723851 74.43324530360609) bank173286 +173287 POINT(40.669670750631894 74.8495636788085) bank173287 +173288 POINT(41.16979954178946 73.74614757665144) bank173288 +173289 POINT(41.29952555624351 74.43847510617837) bank173289 +173290 POINT(40.576936840615474 74.22820321788866) bank173290 +173291 POINT(41.460807352044654 73.32476931429925) bank173291 +173292 POINT(41.58421012571151 73.9721010910443) bank173292 +173293 POINT(40.43941225596976 74.86124477554641) bank173293 +173294 POINT(41.07946274428243 74.28539126277728) bank173294 +173295 POINT(40.52832148060519 73.41877617849752) bank173295 +173296 POINT(40.829639698053654 73.49378517469496) bank173296 +173297 POINT(40.023579995257506 73.71386039005118) bank173297 +173298 POINT(40.11308158113131 74.74026523992708) bank173298 +173299 POINT(41.31023725658785 73.01587859685684) bank173299 +173300 POINT(39.84159564674744 73.55817130525942) bank173300 +173301 POINT(40.703012226343155 73.62817901596883) bank173301 +173302 POINT(41.065987614686264 73.522804248098) bank173302 +173303 POINT(40.50215049438759 74.65471773418929) bank173303 +173304 POINT(40.4637914198974 73.10570731329409) bank173304 +173305 POINT(39.759033708319954 73.50138860483148) bank173305 +173306 POINT(40.02105829259656 73.99570362022388) bank173306 +173307 POINT(39.84715845279727 74.64562587688513) bank173307 +173308 POINT(39.99543517117535 73.31042574874084) bank173308 +173309 POINT(41.6978866005596 73.87208242782977) bank173309 +173310 POINT(41.71271104883791 73.83100277307622) bank173310 +173311 POINT(41.595270342145874 74.9691907674656) bank173311 +173312 POINT(41.398593055352464 73.02140295987488) bank173312 +173313 POINT(41.47269498816829 73.31318049605734) bank173313 +173314 POINT(41.1076032215905 74.25879040616634) bank173314 +173315 POINT(40.674020092205666 74.99849656521043) bank173315 +173316 POINT(41.519889339577354 74.34645356366352) bank173316 +173317 POINT(40.50891053114839 74.68825235644672) bank173317 +173318 POINT(41.52753906606671 74.65524605433627) bank173318 +173319 POINT(40.925162144546064 74.55248587002684) bank173319 +173320 POINT(40.70022394562386 73.4806458342026) bank173320 +173321 POINT(41.03860011709977 74.72300726885152) bank173321 +173322 POINT(40.04126957550681 74.57601577287939) bank173322 +173323 POINT(40.29767915997663 74.36837510973152) bank173323 +173324 POINT(40.4511592378268 74.37522349800697) bank173324 +173325 POINT(39.956361698504494 74.82308061870988) bank173325 +173326 POINT(41.01005809920549 74.07770605518235) bank173326 +173327 POINT(40.74020300286763 73.97972165689947) bank173327 +173328 POINT(40.9791147877409 73.68445721446848) bank173328 +173329 POINT(41.199520536973964 73.40883814414782) bank173329 +173330 POINT(40.37140171221256 73.66851005605972) bank173330 +173331 POINT(41.46742037646924 74.47588522900439) bank173331 +173332 POINT(41.3561960574133 74.81752409931026) bank173332 +173333 POINT(41.127655928342385 74.29685857058071) bank173333 +173334 POINT(40.940142536694786 73.1657304559114) bank173334 +173335 POINT(39.97428770191962 73.50720916296628) bank173335 +173336 POINT(40.79544133827427 74.01388611455621) bank173336 +173337 POINT(40.12890586150859 74.67857421574924) bank173337 +173338 POINT(41.06241893592726 73.72158548432029) bank173338 +173339 POINT(40.601464451726976 74.24398658276529) bank173339 +173340 POINT(40.67447652515636 74.47587358256726) bank173340 +173341 POINT(39.84795976074862 73.44828124946443) bank173341 +173342 POINT(39.92987064969866 74.00745800615256) bank173342 +173343 POINT(41.38583402540842 73.48891777140105) bank173343 +173344 POINT(39.87052105138291 73.99238867708931) bank173344 +173345 POINT(41.41645679069278 74.55299240275329) bank173345 +173346 POINT(39.95666488000271 74.62880191888569) bank173346 +173347 POINT(40.24971612095793 73.89897888302549) bank173347 +173348 POINT(40.78734018316949 74.40002786889444) bank173348 +173349 POINT(39.747494568884676 73.95228813167854) bank173349 +173350 POINT(41.42970398457673 74.06592762266257) bank173350 +173351 POINT(39.76281252346305 73.27347878981523) bank173351 +173352 POINT(39.72351551708257 74.5622352465908) bank173352 +173353 POINT(41.55105872480331 73.05604706901455) bank173353 +173354 POINT(40.594174939391195 74.99262039588903) bank173354 +173355 POINT(41.354741992818425 74.41450639394743) bank173355 +173356 POINT(40.806934691980736 74.25608075610243) bank173356 +173357 POINT(41.11718346960488 73.77146134575646) bank173357 +173358 POINT(40.852167523476766 73.4181811198757) bank173358 +173359 POINT(39.957528723424 73.22049664758154) bank173359 +173360 POINT(41.24390134999079 74.99280840896498) bank173360 +173361 POINT(41.21482209399689 74.01840268106737) bank173361 +173362 POINT(41.51586464279298 73.40003431306148) bank173362 +173363 POINT(41.30896227111724 74.51525539884187) bank173363 +173364 POINT(41.23523500062857 73.41265238253084) bank173364 +173365 POINT(41.26272193578835 73.18867663666143) bank173365 +173366 POINT(41.04026705696918 73.36765484472507) bank173366 +173367 POINT(40.371144165290076 74.68802184920575) bank173367 +173368 POINT(41.49652320964671 74.11155343190772) bank173368 +173369 POINT(40.48224142629552 73.5446374718283) bank173369 +173370 POINT(41.08291709474615 73.88643481530207) bank173370 +173371 POINT(39.978368934487314 73.04156664403956) bank173371 +173372 POINT(41.1868103843266 74.31513994417118) bank173372 +173373 POINT(40.8709626586057 74.66204171390012) bank173373 +173374 POINT(40.03776639763679 74.36121188110442) bank173374 +173375 POINT(41.110716180253576 74.81050150512712) bank173375 +173376 POINT(40.82875315592946 73.15069816257119) bank173376 +173377 POINT(40.827611672295646 74.51907704806794) bank173377 +173378 POINT(40.132253811881036 74.50210059810259) bank173378 +173379 POINT(41.296819410437436 74.05226230878742) bank173379 +173380 POINT(40.159701231580485 73.43318761471811) bank173380 +173381 POINT(40.16296275919614 73.63803483147937) bank173381 +173382 POINT(40.75606970743414 74.70836749879642) bank173382 +173383 POINT(40.268349212898116 74.18320993068944) bank173383 +173384 POINT(41.20498598601504 73.96827985598956) bank173384 +173385 POINT(40.51507592877009 74.5145201282652) bank173385 +173386 POINT(41.14235574807126 74.12699599647954) bank173386 +173387 POINT(40.89808952792722 73.1726043580816) bank173387 +173388 POINT(40.68837599341166 74.11851099863216) bank173388 +173389 POINT(41.466934476611314 73.02945864436299) bank173389 +173390 POINT(41.41947847955369 74.85269573000573) bank173390 +173391 POINT(39.910861773040146 74.6067427005439) bank173391 +173392 POINT(40.29997730391612 73.21337136919794) bank173392 +173393 POINT(41.45533380401348 74.20289742213042) bank173393 +173394 POINT(41.6727552295052 74.94018161600349) bank173394 +173395 POINT(39.81415044294252 73.74357705586289) bank173395 +173396 POINT(41.27521545157065 74.89337450176231) bank173396 +173397 POINT(41.37189356312766 74.0300510931291) bank173397 +173398 POINT(41.303676452259765 74.39565575874524) bank173398 +173399 POINT(39.76606191732205 73.52085478151226) bank173399 +173400 POINT(41.01393691073652 74.40978353242453) bank173400 +173401 POINT(41.40026294190558 73.75456184567248) bank173401 +173402 POINT(41.0329726596985 73.82074503278871) bank173402 +173403 POINT(39.81827227381315 74.45336537182169) bank173403 +173404 POINT(40.29614309535171 74.42462478263984) bank173404 +173405 POINT(39.96232571499858 74.05812214540582) bank173405 +173406 POINT(39.720665226434406 73.92291656174693) bank173406 +173407 POINT(41.60820229245192 74.75508874566141) bank173407 +173408 POINT(41.71272894255627 74.34623562828055) bank173408 +173409 POINT(40.98316416488052 74.97241290359732) bank173409 +173410 POINT(40.37443578288924 74.98669748284128) bank173410 +173411 POINT(41.489956202243434 74.94521760064602) bank173411 +173412 POINT(41.380716676965626 73.61118168094444) bank173412 +173413 POINT(41.57077932668923 73.51675826223874) bank173413 +173414 POINT(40.89983811902997 74.27800582568197) bank173414 +173415 POINT(40.435215697434664 73.44301888263912) bank173415 +173416 POINT(41.430071653269835 73.6057569910883) bank173416 +173417 POINT(40.99038293456204 73.0341390806623) bank173417 +173418 POINT(41.36645022266219 74.62959655016995) bank173418 +173419 POINT(39.914780143599884 74.8728902516916) bank173419 +173420 POINT(41.25712557883997 73.26257720671688) bank173420 +173421 POINT(39.73692244785057 73.73374151417404) bank173421 +173422 POINT(39.81734517402141 74.07886324721515) bank173422 +173423 POINT(40.0215348669894 74.81954000858325) bank173423 +173424 POINT(40.16124275346757 74.27202200912267) bank173424 +173425 POINT(40.15895675236369 73.28208977030705) bank173425 +173426 POINT(41.37618850647298 74.38928145273204) bank173426 +173427 POINT(41.125212321171404 73.31696366886739) bank173427 +173428 POINT(40.82308648282253 73.55488685233144) bank173428 +173429 POINT(40.99087137989824 73.22302428379051) bank173429 +173430 POINT(41.64118701899291 73.53198364515545) bank173430 +173431 POINT(39.99269447809714 73.96184547264473) bank173431 +173432 POINT(40.53890500199139 74.21448923861314) bank173432 +173433 POINT(40.495240387258875 74.57896374236432) bank173433 +173434 POINT(41.57598612501162 74.455261080494) bank173434 +173435 POINT(40.41087128962884 73.01988233445452) bank173435 +173436 POINT(41.155238628692516 74.34568130256142) bank173436 +173437 POINT(40.009259794294465 74.5140058099125) bank173437 +173438 POINT(41.44921158097315 74.54015065720361) bank173438 +173439 POINT(40.401238399993524 74.28893822254982) bank173439 +173440 POINT(40.57043188589822 74.37568678456167) bank173440 +173441 POINT(40.03639239132836 74.64129076493566) bank173441 +173442 POINT(41.06204388544871 73.17906133292632) bank173442 +173443 POINT(40.4146388810437 74.35711487803056) bank173443 +173444 POINT(39.856386982362224 73.33984586781393) bank173444 +173445 POINT(41.042628457596216 74.47098318771147) bank173445 +173446 POINT(39.8967448817773 73.83833351259712) bank173446 +173447 POINT(41.55418351557409 74.5310743408992) bank173447 +173448 POINT(41.001484266881334 73.53608195875817) bank173448 +173449 POINT(40.97087197453415 73.9256837054163) bank173449 +173450 POINT(40.01662432861821 73.03380177223768) bank173450 +173451 POINT(40.0553901085255 73.22022378072835) bank173451 +173452 POINT(40.02871915351796 74.8582036341012) bank173452 +173453 POINT(40.82660569403295 74.65560794693378) bank173453 +173454 POINT(41.18866414483686 73.0987741325775) bank173454 +173455 POINT(41.0011098521912 73.9100018098111) bank173455 +173456 POINT(40.34499130078744 73.1788492716633) bank173456 +173457 POINT(41.28516159723875 73.26698299416066) bank173457 +173458 POINT(41.08869588893853 73.71493498385273) bank173458 +173459 POINT(41.4028556978694 73.55460679740617) bank173459 +173460 POINT(41.184382877847106 74.30504100085925) bank173460 +173461 POINT(39.995021052729626 74.12633744731231) bank173461 +173462 POINT(40.07386925117803 73.55682097301236) bank173462 +173463 POINT(40.764973071387004 74.99952145667304) bank173463 +173464 POINT(40.77431201928082 73.79978930510329) bank173464 +173465 POINT(41.50195488980667 73.82770356773081) bank173465 +173466 POINT(41.47058475126608 73.58851984944388) bank173466 +173467 POINT(40.238880520349525 73.09011552092839) bank173467 +173468 POINT(41.38189863783383 74.88239351388657) bank173468 +173469 POINT(40.65854809662049 74.02759023861296) bank173469 +173470 POINT(41.0739668667248 74.74714757701827) bank173470 +173471 POINT(41.63509659926687 73.64679051091241) bank173471 +173472 POINT(40.924722731592645 73.57754188031842) bank173472 +173473 POINT(41.61634111130981 73.39187446423587) bank173473 +173474 POINT(41.05066998628424 73.7545879935429) bank173474 +173475 POINT(41.68906098414263 74.03872129459519) bank173475 +173476 POINT(40.19114594614552 73.98220663927866) bank173476 +173477 POINT(40.148805990435456 74.44716279493296) bank173477 +173478 POINT(40.97724379145361 74.26555912144171) bank173478 +173479 POINT(40.49022284962007 73.33478903271003) bank173479 +173480 POINT(40.09072995278295 74.7099655040862) bank173480 +173481 POINT(40.93399208393506 73.8603820441241) bank173481 +173482 POINT(41.577062512671446 73.16403527053446) bank173482 +173483 POINT(41.23913593453465 74.91506621463607) bank173483 +173484 POINT(41.266264741507115 73.04142081516149) bank173484 +173485 POINT(41.594855310866855 73.7837444972957) bank173485 +173486 POINT(41.44393455487892 73.56680424444757) bank173486 +173487 POINT(41.243544859829015 73.6334481527029) bank173487 +173488 POINT(39.974525751388 73.868979416368) bank173488 +173489 POINT(39.758550920682666 73.42006605538718) bank173489 +173490 POINT(40.131058600716926 74.54519308649476) bank173490 +173491 POINT(41.69884733864359 73.9880106789398) bank173491 +173492 POINT(40.83278899519056 73.94887707817146) bank173492 +173493 POINT(41.62199172081316 74.51869830190022) bank173493 +173494 POINT(40.83552898265739 74.2752360659319) bank173494 +173495 POINT(41.49495160899605 73.37473004740777) bank173495 +173496 POINT(39.74498459073147 73.42841559412437) bank173496 +173497 POINT(40.073618482555844 74.3208426840905) bank173497 +173498 POINT(40.21851536098712 74.48267547267467) bank173498 +173499 POINT(41.55545840094278 74.4865265885956) bank173499 +173500 POINT(40.06497526653666 74.04365846216568) bank173500 +173501 POINT(39.739397775409046 73.70168676269179) bank173501 +173502 POINT(40.26933881520986 74.11339333965583) bank173502 +173503 POINT(41.51854882249704 74.04316379567227) bank173503 +173504 POINT(40.0133183448434 74.10718844196991) bank173504 +173505 POINT(40.77398184979422 73.62249634456657) bank173505 +173506 POINT(40.001857185651026 73.39424768721311) bank173506 +173507 POINT(41.1689806597013 73.62151726894287) bank173507 +173508 POINT(40.96416964270403 73.20125703641085) bank173508 +173509 POINT(39.852974441864525 73.15445606109287) bank173509 +173510 POINT(40.61906790294065 73.46403586461764) bank173510 +173511 POINT(41.301306237571296 73.16342464121144) bank173511 +173512 POINT(40.47124560118337 74.78729871436117) bank173512 +173513 POINT(40.171651593287166 74.02579860725665) bank173513 +173514 POINT(40.76025203452579 74.3761393168792) bank173514 +173515 POINT(40.735174509286196 73.27159168205665) bank173515 +173516 POINT(40.18832334528296 73.31255422437685) bank173516 +173517 POINT(39.812921864994124 74.05774266767467) bank173517 +173518 POINT(41.6689774724031 74.35729004684606) bank173518 +173519 POINT(41.19860379710652 73.46970930729466) bank173519 +173520 POINT(41.30392254356299 74.9319049309641) bank173520 +173521 POINT(40.99686256524851 73.725921162643) bank173521 +173522 POINT(40.666999500726355 73.02116627715267) bank173522 +173523 POINT(40.013220285880095 74.58566224008227) bank173523 +173524 POINT(40.79904465450492 74.42855344307644) bank173524 +173525 POINT(41.3248929113053 73.66639859411116) bank173525 +173526 POINT(40.14762881178526 74.33965050478959) bank173526 +173527 POINT(41.665606584050956 73.21777470722074) bank173527 +173528 POINT(41.57113525051898 74.28342829015945) bank173528 +173529 POINT(40.470244492649435 74.46089397619181) bank173529 +173530 POINT(40.18963305751488 73.02946123368922) bank173530 +173531 POINT(40.34366200309721 74.63968710654055) bank173531 +173532 POINT(40.81495454883981 73.08496778818555) bank173532 +173533 POINT(41.68456563934976 73.16442006332657) bank173533 +173534 POINT(40.13056061719094 74.50560179236534) bank173534 +173535 POINT(40.67405024182865 73.36418205158783) bank173535 +173536 POINT(40.378912267671254 74.50193689865998) bank173536 +173537 POINT(41.25880961985629 74.05188310750084) bank173537 +173538 POINT(40.94334445660262 74.83942112836395) bank173538 +173539 POINT(40.304347252835036 73.26747744447808) bank173539 +173540 POINT(40.51415691885298 73.11214782297188) bank173540 +173541 POINT(40.3685193161448 74.21015991272023) bank173541 +173542 POINT(41.35231478764864 73.85473402878544) bank173542 +173543 POINT(39.920277543975395 74.28593158352133) bank173543 +173544 POINT(40.256248254452885 73.4774419071369) bank173544 +173545 POINT(39.85751087370363 73.93799402371128) bank173545 +173546 POINT(41.1658518550141 73.56545325346502) bank173546 +173547 POINT(41.45367282327673 74.34005471886668) bank173547 +173548 POINT(40.001767731617434 74.51396123445535) bank173548 +173549 POINT(40.13666570935294 73.10282445708897) bank173549 +173550 POINT(39.87404594659808 73.25494255225104) bank173550 +173551 POINT(41.38111058905434 73.34137295023696) bank173551 +173552 POINT(40.9030790875908 73.8577842019269) bank173552 +173553 POINT(41.38186689514435 73.06037339725916) bank173553 +173554 POINT(41.414256271282184 74.88579010281128) bank173554 +173555 POINT(40.8358430349944 73.58141733801527) bank173555 +173556 POINT(39.812014535298914 73.62326783080479) bank173556 +173557 POINT(41.65742347305755 74.60998628140453) bank173557 +173558 POINT(40.716602272225266 73.24151044773255) bank173558 +173559 POINT(39.748981345124385 73.4692486566301) bank173559 +173560 POINT(41.509185742524025 73.78222805073847) bank173560 +173561 POINT(40.89530030142236 73.41016048018433) bank173561 +173562 POINT(39.82325976773545 73.09370306958867) bank173562 +173563 POINT(41.54764391209528 74.63563143247933) bank173563 +173564 POINT(40.105749142246516 73.15701374883588) bank173564 +173565 POINT(41.014042992955595 74.00431569019604) bank173565 +173566 POINT(40.64503391148587 74.48682729784588) bank173566 +173567 POINT(41.45243357009918 74.00874321219786) bank173567 +173568 POINT(40.29377359593737 73.35835557240588) bank173568 +173569 POINT(40.004417117438166 73.43167944428127) bank173569 +173570 POINT(40.62519951382265 74.18464727077071) bank173570 +173571 POINT(41.06789139928811 74.82270923537403) bank173571 +173572 POINT(40.51097927374077 74.49243448049944) bank173572 +173573 POINT(41.632109102474125 73.26252668686301) bank173573 +173574 POINT(41.223592994654446 74.44602764963676) bank173574 +173575 POINT(41.609581895866675 74.75477054461263) bank173575 +173576 POINT(39.764645855222774 73.45969584557021) bank173576 +173577 POINT(40.830087349784854 74.57449851303753) bank173577 +173578 POINT(41.14554541842124 74.1867642704404) bank173578 +173579 POINT(40.05306211515839 74.84692711170557) bank173579 +173580 POINT(39.835809301113635 73.3337598260565) bank173580 +173581 POINT(40.009882259049945 73.4109193452446) bank173581 +173582 POINT(41.15071724143257 74.88445469096844) bank173582 +173583 POINT(40.20100960622188 73.10824296056458) bank173583 +173584 POINT(41.28877567010136 74.3069263388366) bank173584 +173585 POINT(39.768264771513486 74.17231768217717) bank173585 +173586 POINT(41.058600982934756 74.57379269109408) bank173586 +173587 POINT(39.98795838025594 74.08837797198846) bank173587 +173588 POINT(40.95193059221299 73.74921759615604) bank173588 +173589 POINT(41.48625870664289 74.79031506066498) bank173589 +173590 POINT(40.152520462411495 73.11331149228312) bank173590 +173591 POINT(39.76285158984497 74.58158410624624) bank173591 +173592 POINT(40.42471690260864 73.40283666094179) bank173592 +173593 POINT(40.79582553014127 74.30321641830461) bank173593 +173594 POINT(40.579516579778684 74.37995593859253) bank173594 +173595 POINT(40.79545248645879 73.49000586719949) bank173595 +173596 POINT(41.184289589962674 74.76326829875522) bank173596 +173597 POINT(40.11758986281586 74.82526934327358) bank173597 +173598 POINT(39.759614769716904 73.52041092214694) bank173598 +173599 POINT(40.5853534035361 73.46124698575832) bank173599 +173600 POINT(40.191237877909394 73.45707180494009) bank173600 +173601 POINT(41.629952434147256 73.97027903795303) bank173601 +173602 POINT(40.347764901513074 74.60098237513131) bank173602 +173603 POINT(41.293113822670165 74.24448973699869) bank173603 +173604 POINT(40.361746919042645 73.23421808767488) bank173604 +173605 POINT(40.51609625163206 74.1374299356888) bank173605 +173606 POINT(40.68463669486595 74.31546982155945) bank173606 +173607 POINT(41.37380751242815 74.74486008310677) bank173607 +173608 POINT(40.49612769137131 73.47150775350845) bank173608 +173609 POINT(41.0700742425873 74.36037434614491) bank173609 +173610 POINT(40.421219177060436 73.75338121503637) bank173610 +173611 POINT(41.35520218864888 73.58532631880901) bank173611 +173612 POINT(40.93691689086764 74.01627587549808) bank173612 +173613 POINT(41.30471472685794 73.58917135022448) bank173613 +173614 POINT(40.78631061514171 74.35100760680218) bank173614 +173615 POINT(41.38726618430479 73.90156104023835) bank173615 +173616 POINT(41.448709575016004 74.25456528624677) bank173616 +173617 POINT(40.404543344299356 73.7982564190447) bank173617 +173618 POINT(40.81184690356689 74.83456482486491) bank173618 +173619 POINT(40.859250612093895 73.76891672496279) bank173619 +173620 POINT(40.67819513365077 73.09441970576567) bank173620 +173621 POINT(41.50502378203073 73.04595829711377) bank173621 +173622 POINT(40.5380289608067 74.0423829254624) bank173622 +173623 POINT(41.180848914080904 74.5975113363441) bank173623 +173624 POINT(40.701313183432234 74.17060314502812) bank173624 +173625 POINT(41.14434836092543 73.37690499088619) bank173625 +173626 POINT(41.21917306817316 73.04002693865907) bank173626 +173627 POINT(40.02934231597833 74.71204777389563) bank173627 +173628 POINT(41.67457982988591 73.93778436546103) bank173628 +173629 POINT(40.00863461393648 74.31123967028344) bank173629 +173630 POINT(41.04390971074762 73.75797294979444) bank173630 +173631 POINT(40.25742938559451 73.21364347832028) bank173631 +173632 POINT(40.28124085270704 73.02473323176994) bank173632 +173633 POINT(41.42454094050439 73.46943603304912) bank173633 +173634 POINT(41.6454958489374 74.51729844160936) bank173634 +173635 POINT(40.056328409394744 73.17740294075662) bank173635 +173636 POINT(40.25577058578829 74.76362252803088) bank173636 +173637 POINT(40.369010291726305 73.56938072112203) bank173637 +173638 POINT(40.77504383458371 73.15560663776547) bank173638 +173639 POINT(39.87932126425889 74.77146101958525) bank173639 +173640 POINT(41.08363302514751 74.64150286967167) bank173640 +173641 POINT(41.400599836806606 74.70801846921819) bank173641 +173642 POINT(41.54749703955023 73.56204334511725) bank173642 +173643 POINT(40.287410284162036 74.42159845391534) bank173643 +173644 POINT(41.523322457907376 73.27246100692508) bank173644 +173645 POINT(40.88861525932696 74.20612137129926) bank173645 +173646 POINT(41.084732325578955 74.20771431054223) bank173646 +173647 POINT(40.494344227833885 74.04888258963595) bank173647 +173648 POINT(40.65873108859442 73.8258414533643) bank173648 +173649 POINT(41.69599968265472 73.31894243906319) bank173649 +173650 POINT(40.29055046780496 73.80926761064485) bank173650 +173651 POINT(40.353191196757386 74.94732734721819) bank173651 +173652 POINT(41.58794221339494 74.6497545877184) bank173652 +173653 POINT(41.13508088860178 74.56877667032535) bank173653 +173654 POINT(41.57692222789793 73.30404835677501) bank173654 +173655 POINT(41.26992724831873 73.48562938819371) bank173655 +173656 POINT(40.59766421267274 73.84211474219798) bank173656 +173657 POINT(41.568541854306616 74.4666056935591) bank173657 +173658 POINT(41.17725211684304 74.53069409802123) bank173658 +173659 POINT(40.85982995178803 73.55192160196096) bank173659 +173660 POINT(40.795600035632916 73.5513408812333) bank173660 +173661 POINT(40.18902756365867 73.7436735846966) bank173661 +173662 POINT(40.891888934783964 73.22739991532434) bank173662 +173663 POINT(41.276794376591575 73.24502889010246) bank173663 +173664 POINT(41.545458881411605 73.14848050437648) bank173664 +173665 POINT(40.97610021916757 74.98484279399501) bank173665 +173666 POINT(41.60251253601267 73.0946160142626) bank173666 +173667 POINT(39.93208530779359 73.86867216900859) bank173667 +173668 POINT(40.76791549874222 74.03497579412975) bank173668 +173669 POINT(40.10174596052111 73.96588276639744) bank173669 +173670 POINT(40.57192836323561 74.98782359231662) bank173670 +173671 POINT(40.02519160644272 74.14722274497124) bank173671 +173672 POINT(39.86644111133981 74.39374549430828) bank173672 +173673 POINT(39.912472780724144 73.35664677281468) bank173673 +173674 POINT(40.58480512001266 74.71695204058608) bank173674 +173675 POINT(40.21915639760373 73.81885277476023) bank173675 +173676 POINT(41.19564635559927 73.94399505288284) bank173676 +173677 POINT(40.02032838524201 73.58317551981506) bank173677 +173678 POINT(39.843034244913554 73.87859393293841) bank173678 +173679 POINT(40.99403297512269 74.70082601063731) bank173679 +173680 POINT(40.57524712421391 73.93708943627455) bank173680 +173681 POINT(40.86392013218501 73.9063731834156) bank173681 +173682 POINT(40.04500354379091 73.5947496775037) bank173682 +173683 POINT(40.74882776183043 73.17084941614537) bank173683 +173684 POINT(40.44618044940265 74.53365181884632) bank173684 +173685 POINT(41.1936246647779 73.97514426305791) bank173685 +173686 POINT(41.33069967956479 74.58836725372672) bank173686 +173687 POINT(41.57197157422725 74.60239289663217) bank173687 +173688 POINT(40.505276867072844 73.54136275739357) bank173688 +173689 POINT(40.13114795881921 74.53808067203823) bank173689 +173690 POINT(40.31937273466397 73.8867217561264) bank173690 +173691 POINT(41.582275496310956 73.16902636473836) bank173691 +173692 POINT(40.432913862556774 74.12474839235281) bank173692 +173693 POINT(39.80952316305171 74.67996697419319) bank173693 +173694 POINT(41.258766267574075 73.75707043135935) bank173694 +173695 POINT(40.15236927009274 74.62589063549984) bank173695 +173696 POINT(41.28866067794959 74.4759778888921) bank173696 +173697 POINT(41.473592166725304 74.0516837113104) bank173697 +173698 POINT(40.72275334265732 74.44236123404589) bank173698 +173699 POINT(40.49347224918211 73.49405746775943) bank173699 +173700 POINT(40.98309606355488 73.30598588573882) bank173700 +173701 POINT(40.91325129766351 74.80313766003327) bank173701 +173702 POINT(40.70885295860259 73.81769273675387) bank173702 +173703 POINT(40.793470821399055 74.1620095700866) bank173703 +173704 POINT(40.73936754110289 74.73533141286356) bank173704 +173705 POINT(40.63736607683051 73.27466735140578) bank173705 +173706 POINT(40.88422934067493 74.58234859512159) bank173706 +173707 POINT(40.7610046150677 73.36783417453938) bank173707 +173708 POINT(40.112860941238914 73.31202686654999) bank173708 +173709 POINT(41.230418709842375 73.17301801007926) bank173709 +173710 POINT(39.96854094935557 73.96728667566009) bank173710 +173711 POINT(40.743406443625865 73.048462268984) bank173711 +173712 POINT(41.067957480997414 74.23756895181266) bank173712 +173713 POINT(40.01912744501051 74.49073508960981) bank173713 +173714 POINT(41.63524468737948 74.87779591516907) bank173714 +173715 POINT(40.34309342057887 73.94934795781354) bank173715 +173716 POINT(40.78763392983558 73.49156084183541) bank173716 +173717 POINT(41.51728122352413 73.27836477635411) bank173717 +173718 POINT(39.89434135363143 74.56016793489476) bank173718 +173719 POINT(40.978361822954945 73.41036152232748) bank173719 +173720 POINT(40.12409164918742 74.42434871649255) bank173720 +173721 POINT(41.082947314721395 73.27865524167521) bank173721 +173722 POINT(40.78824669542623 73.3489689861395) bank173722 +173723 POINT(41.551755271770666 74.79981792148264) bank173723 +173724 POINT(39.947200937928166 74.53059426347306) bank173724 +173725 POINT(39.88347357846781 74.04793694868557) bank173725 +173726 POINT(39.95467611373908 74.95548170833239) bank173726 +173727 POINT(41.4264228504949 73.60391127657287) bank173727 +173728 POINT(40.992817834451586 74.90836243436317) bank173728 +173729 POINT(41.0427750218162 73.33017638070845) bank173729 +173730 POINT(39.949861639402506 74.29088676721614) bank173730 +173731 POINT(40.304854720478964 74.78874539491292) bank173731 +173732 POINT(40.58661425476311 73.99534353633823) bank173732 +173733 POINT(40.30848547313964 73.92337078355095) bank173733 +173734 POINT(39.98086496901506 74.36453513557915) bank173734 +173735 POINT(40.754921442964765 74.49622037271524) bank173735 +173736 POINT(40.91692285253151 74.69612837570779) bank173736 +173737 POINT(40.23788293396805 74.66025170497926) bank173737 +173738 POINT(40.456782794025905 74.69067916162383) bank173738 +173739 POINT(40.3107752942422 73.94334872519589) bank173739 +173740 POINT(40.40939596012791 73.83283042814904) bank173740 +173741 POINT(41.08572674398693 74.90436407432625) bank173741 +173742 POINT(39.77775468315977 73.49905923485673) bank173742 +173743 POINT(40.03968800498988 74.69596239695436) bank173743 +173744 POINT(41.59873155389193 73.09776449044435) bank173744 +173745 POINT(40.02819570741573 73.13222940558911) bank173745 +173746 POINT(40.814382887790856 73.06276944089186) bank173746 +173747 POINT(40.93229313501488 73.70146880425028) bank173747 +173748 POINT(40.03210203970993 74.06148409093556) bank173748 +173749 POINT(40.15922671812035 73.90854151688693) bank173749 +173750 POINT(41.33324570526827 74.08545110477483) bank173750 +173751 POINT(41.12280121160645 74.94607817828218) bank173751 +173752 POINT(40.01934658827223 73.09669202483634) bank173752 +173753 POINT(41.55170195437964 73.80645483647854) bank173753 +173754 POINT(41.25827427947142 74.17193756998797) bank173754 +173755 POINT(40.117777667816966 73.22641143901025) bank173755 +173756 POINT(40.30967507236255 73.77008023765225) bank173756 +173757 POINT(40.6194550346953 74.55845536006221) bank173757 +173758 POINT(41.139646854979134 73.33159852704205) bank173758 +173759 POINT(40.46815943962256 74.90331095178956) bank173759 +173760 POINT(40.1097454860197 73.34293913195354) bank173760 +173761 POINT(40.968755122805014 73.8482613601653) bank173761 +173762 POINT(41.29666776719638 73.12052339123369) bank173762 +173763 POINT(41.124520226951894 74.59100165311239) bank173763 +173764 POINT(39.80611392124901 73.9000974730814) bank173764 +173765 POINT(40.96974746700361 74.88931417405406) bank173765 +173766 POINT(40.65594285823877 74.92883163067482) bank173766 +173767 POINT(41.697162832719776 73.70771363446492) bank173767 +173768 POINT(41.67131633439753 74.43387895285457) bank173768 +173769 POINT(41.64379914881098 74.67055406952178) bank173769 +173770 POINT(41.67016568790026 73.5728223407219) bank173770 +173771 POINT(40.83342288676138 74.52714233796492) bank173771 +173772 POINT(41.14694685343634 73.74379903894659) bank173772 +173773 POINT(40.0969960144937 74.4273888231695) bank173773 +173774 POINT(41.641550706648424 74.89310063684631) bank173774 +173775 POINT(40.77775995848378 74.21687827775125) bank173775 +173776 POINT(40.4325625547002 74.12348474935902) bank173776 +173777 POINT(40.25639538105369 74.52921689016875) bank173777 +173778 POINT(40.3562049639622 73.93958193643974) bank173778 +173779 POINT(41.04391538349994 73.51018771923002) bank173779 +173780 POINT(41.45930054321461 73.02896485484294) bank173780 +173781 POINT(41.423941005471654 73.85698529830212) bank173781 +173782 POINT(40.5170275644048 74.0746549059432) bank173782 +173783 POINT(40.40771429710854 74.16422617288758) bank173783 +173784 POINT(40.69262108105593 73.70907831492033) bank173784 +173785 POINT(41.34894736735227 74.999232706046) bank173785 +173786 POINT(41.679529418037596 73.04455016868518) bank173786 +173787 POINT(40.67183540324834 73.71346879058774) bank173787 +173788 POINT(41.163971584293535 73.6973935025392) bank173788 +173789 POINT(40.586504300903066 74.80028702334711) bank173789 +173790 POINT(39.83600363059411 73.17121321868032) bank173790 +173791 POINT(39.77357096063853 73.89063936978638) bank173791 +173792 POINT(40.59485886310365 73.97714597525432) bank173792 +173793 POINT(40.01240539663388 73.32588233717951) bank173793 +173794 POINT(40.151235189544444 74.4342783511741) bank173794 +173795 POINT(40.851885230023555 75.00325944584709) bank173795 +173796 POINT(41.3799673325084 73.76974704714192) bank173796 +173797 POINT(41.062147500945024 73.93179818605635) bank173797 +173798 POINT(41.4670831969118 74.59893843073212) bank173798 +173799 POINT(40.041106895167744 73.03986401425631) bank173799 +173800 POINT(41.26976413120722 73.3408672996868) bank173800 +173801 POINT(40.81462361992915 74.20234250344258) bank173801 +173802 POINT(40.05342460322057 74.93957743824576) bank173802 +173803 POINT(40.21888783443715 73.99532823015343) bank173803 +173804 POINT(40.98950440205696 73.27269646548407) bank173804 +173805 POINT(41.67310999859947 74.42386648778142) bank173805 +173806 POINT(39.77023553718188 74.95217953481766) bank173806 +173807 POINT(39.82834252660812 73.29922537910515) bank173807 +173808 POINT(40.87069315430789 73.52196856199535) bank173808 +173809 POINT(39.95337235977409 74.61536978388276) bank173809 +173810 POINT(40.852650199331734 73.05739414071034) bank173810 +173811 POINT(39.883994027348784 74.23890976168401) bank173811 +173812 POINT(41.39403149505376 74.1574233481966) bank173812 +173813 POINT(39.79539295428958 73.01478255745238) bank173813 +173814 POINT(40.302529113348804 74.55315043672583) bank173814 +173815 POINT(40.117533269255304 73.86172581859364) bank173815 +173816 POINT(40.72400332980545 74.16341890057663) bank173816 +173817 POINT(40.96546285155033 73.46502309228774) bank173817 +173818 POINT(41.17963199925005 74.08599019995506) bank173818 +173819 POINT(41.50420635803388 74.55385317887072) bank173819 +173820 POINT(40.02030171298685 73.44424503963931) bank173820 +173821 POINT(40.556157268686526 73.83492689834351) bank173821 +173822 POINT(39.82351863074608 74.82816163530013) bank173822 +173823 POINT(41.51769990036227 73.48503429388427) bank173823 +173824 POINT(40.87497707873361 73.9437959068923) bank173824 +173825 POINT(40.67828702350109 74.64190038948682) bank173825 +173826 POINT(41.09216954152521 74.27427448684494) bank173826 +173827 POINT(40.797553649457214 74.59578592992536) bank173827 +173828 POINT(39.866704304443836 74.05433953056583) bank173828 +173829 POINT(41.36906395368002 73.91409815439538) bank173829 +173830 POINT(39.72461214774003 74.72826126533336) bank173830 +173831 POINT(40.40223464115272 74.81547523734037) bank173831 +173832 POINT(41.33962339121944 73.66933092765981) bank173832 +173833 POINT(40.794372607658154 74.50353943169624) bank173833 +173834 POINT(39.943884923466356 74.12025518956635) bank173834 +173835 POINT(40.0970561152564 73.84020780403982) bank173835 +173836 POINT(41.06406652691465 74.55182025097689) bank173836 +173837 POINT(40.55690294468233 74.2932386235463) bank173837 +173838 POINT(40.44049235784332 74.58195502711342) bank173838 +173839 POINT(41.4417121094742 74.02538181417349) bank173839 +173840 POINT(40.32207050665238 73.76941082568477) bank173840 +173841 POINT(40.109533279371846 74.38424910251032) bank173841 +173842 POINT(40.43111589095393 73.3032102931657) bank173842 +173843 POINT(40.74311349324174 73.83560112887062) bank173843 +173844 POINT(40.668060220608474 74.68200946798686) bank173844 +173845 POINT(40.449234823218475 74.95365608702186) bank173845 +173846 POINT(41.47272724225484 74.64071346579385) bank173846 +173847 POINT(41.611868953675376 73.52777131703236) bank173847 +173848 POINT(39.96987981751033 74.04629117810713) bank173848 +173849 POINT(40.49442973939076 73.31895909565397) bank173849 +173850 POINT(39.92975411899124 74.2434857315934) bank173850 +173851 POINT(41.03071016818975 74.60902297315387) bank173851 +173852 POINT(40.85973561644419 74.05545867742615) bank173852 +173853 POINT(41.431993668884026 73.89240530548118) bank173853 +173854 POINT(40.7632042892267 73.23902488468886) bank173854 +173855 POINT(41.48064340805346 74.61393104266007) bank173855 +173856 POINT(41.22236513634992 73.10154630144338) bank173856 +173857 POINT(41.31936152746324 74.62252387053073) bank173857 +173858 POINT(41.5737184659446 74.92434966057844) bank173858 +173859 POINT(41.09550444268162 74.5948092939743) bank173859 +173860 POINT(39.99705192421919 74.09816643492947) bank173860 +173861 POINT(40.52857019597682 73.6877938212737) bank173861 +173862 POINT(40.38819611665042 73.14452048207745) bank173862 +173863 POINT(40.9433110628342 74.16779920065942) bank173863 +173864 POINT(39.93989749278795 74.33139733717233) bank173864 +173865 POINT(41.453983974673676 73.87715355377227) bank173865 +173866 POINT(41.07617658139314 74.38962176534557) bank173866 +173867 POINT(41.2487850757483 73.94449723814171) bank173867 +173868 POINT(41.28809793400404 73.37585876906306) bank173868 +173869 POINT(40.886822632701346 73.59127607244886) bank173869 +173870 POINT(40.062948592137175 74.5955693853071) bank173870 +173871 POINT(40.135983261528025 74.1358484084242) bank173871 +173872 POINT(40.45284119484178 74.5255445924943) bank173872 +173873 POINT(40.532007366821446 73.54824060054418) bank173873 +173874 POINT(41.67023775954516 73.44024371780384) bank173874 +173875 POINT(40.557573205516476 74.73486449611111) bank173875 +173876 POINT(40.01322834877595 74.72153551571857) bank173876 +173877 POINT(40.03486261509709 73.15911123434343) bank173877 +173878 POINT(39.87498355514985 74.70987210713959) bank173878 +173879 POINT(41.52568346287744 73.08662456949196) bank173879 +173880 POINT(41.14151664617194 74.05359705220805) bank173880 +173881 POINT(41.08693203072059 74.45538129229264) bank173881 +173882 POINT(40.90670193275079 73.51551808409873) bank173882 +173883 POINT(40.53168766762084 74.51596479539306) bank173883 +173884 POINT(40.36808590159885 74.4721838637637) bank173884 +173885 POINT(40.43830489583485 73.99961543925116) bank173885 +173886 POINT(41.21739280962927 74.08730137720785) bank173886 +173887 POINT(40.48224848577703 73.9873264744991) bank173887 +173888 POINT(41.068352175917546 73.16641309439012) bank173888 +173889 POINT(40.79672908350658 74.05618754651984) bank173889 +173890 POINT(40.71005354964656 74.80011628709482) bank173890 +173891 POINT(39.936368279248036 73.98079869463632) bank173891 +173892 POINT(40.838850243253226 73.47542129619254) bank173892 +173893 POINT(41.159129416823966 74.2528539935954) bank173893 +173894 POINT(40.30634745808333 73.53477486927657) bank173894 +173895 POINT(39.98311710330545 74.15721870796764) bank173895 +173896 POINT(40.61445705459341 73.0297392133624) bank173896 +173897 POINT(41.214760719840626 73.56681644679865) bank173897 +173898 POINT(41.00392588965397 74.08440082612626) bank173898 +173899 POINT(41.60418575784971 73.1379809735861) bank173899 +173900 POINT(41.35178132554845 73.48815591205788) bank173900 +173901 POINT(41.18612528114844 74.45373890200148) bank173901 +173902 POINT(39.94297454973469 73.88226709690889) bank173902 +173903 POINT(40.74347875379465 73.6085755422674) bank173903 +173904 POINT(41.108282383973524 74.21271981373046) bank173904 +173905 POINT(41.25107570895049 74.71128607530858) bank173905 +173906 POINT(40.94103653475654 74.34814105539226) bank173906 +173907 POINT(40.13740878614192 74.83242207249481) bank173907 +173908 POINT(39.928327426105 74.24337639331199) bank173908 +173909 POINT(40.715088925949665 74.89956864536342) bank173909 +173910 POINT(39.866562130414906 74.12929305543115) bank173910 +173911 POINT(40.45421512894516 74.0183449161598) bank173911 +173912 POINT(40.090835292800975 73.63991039903925) bank173912 +173913 POINT(40.39872655535818 74.18290018311536) bank173913 +173914 POINT(40.10167749115681 74.47384887311753) bank173914 +173915 POINT(41.32265393795629 73.61809606631833) bank173915 +173916 POINT(40.16431544908306 73.6302106596211) bank173916 +173917 POINT(40.797097452840575 74.6213197751659) bank173917 +173918 POINT(39.74596217181954 73.88020424629971) bank173918 +173919 POINT(41.209344225505234 73.84838606912753) bank173919 +173920 POINT(39.98994920627809 75.00461577040774) bank173920 +173921 POINT(41.17498919357566 74.34981275549887) bank173921 +173922 POINT(40.91262113883319 74.24529566311784) bank173922 +173923 POINT(40.26671383186873 74.34189809656719) bank173923 +173924 POINT(40.183693500763916 73.41846257322328) bank173924 +173925 POINT(39.884502576314055 74.08522236630121) bank173925 +173926 POINT(39.877539982577545 73.56399382996169) bank173926 +173927 POINT(40.93934309896677 74.12987306472532) bank173927 +173928 POINT(41.257118818238645 74.65375536042121) bank173928 +173929 POINT(40.83648667649987 74.86692890570204) bank173929 +173930 POINT(40.22622432092469 73.30965871902396) bank173930 +173931 POINT(40.193154790825005 74.63514763260345) bank173931 +173932 POINT(39.84383409587182 74.20430735797211) bank173932 +173933 POINT(41.59199761167185 73.40271008690397) bank173933 +173934 POINT(41.514667028283284 73.48262828910327) bank173934 +173935 POINT(41.19986228924958 73.1653212176202) bank173935 +173936 POINT(41.221060664806366 74.72301058947502) bank173936 +173937 POINT(40.65129202616107 74.75588646223022) bank173937 +173938 POINT(40.166920562225386 73.1358011550184) bank173938 +173939 POINT(40.871461059495985 74.60277140941348) bank173939 +173940 POINT(41.68613460083619 74.39627447769995) bank173940 +173941 POINT(40.67304101206812 74.59758233738627) bank173941 +173942 POINT(39.76310665868319 73.4853860843141) bank173942 +173943 POINT(40.08026470374642 73.91461261152557) bank173943 +173944 POINT(41.371434448182384 73.03454595815552) bank173944 +173945 POINT(39.849755119380305 74.62373293018835) bank173945 +173946 POINT(41.60157527837709 73.35759898578588) bank173946 +173947 POINT(40.449365703841636 73.53638677000875) bank173947 +173948 POINT(40.99106064321064 73.89315746983681) bank173948 +173949 POINT(40.38239783158279 73.42329946358491) bank173949 +173950 POINT(40.9813304338045 74.30172518521964) bank173950 +173951 POINT(41.365726943216785 73.57294967039243) bank173951 +173952 POINT(39.868190434872744 74.66566909673783) bank173952 +173953 POINT(39.9103183705522 74.99570614105131) bank173953 +173954 POINT(40.86995905261764 73.01024231953565) bank173954 +173955 POINT(39.77410289984505 74.23359144819239) bank173955 +173956 POINT(40.81644812934959 73.99307491165143) bank173956 +173957 POINT(41.10347911098035 73.21261077682006) bank173957 +173958 POINT(40.92746970224223 74.23115767843403) bank173958 +173959 POINT(40.814365707715524 74.40509414859973) bank173959 +173960 POINT(41.608572755662095 73.11998391204911) bank173960 +173961 POINT(41.152355667654334 74.52288315087169) bank173961 +173962 POINT(39.83846220922522 74.69718816240024) bank173962 +173963 POINT(39.91572871942891 73.71202624901599) bank173963 +173964 POINT(39.891449336767344 73.84511838889071) bank173964 +173965 POINT(41.1151234268384 74.49751374092601) bank173965 +173966 POINT(39.95267814681059 74.93142995681667) bank173966 +173967 POINT(40.54167650634544 73.0647337653308) bank173967 +173968 POINT(39.85629928909697 73.55257579565372) bank173968 +173969 POINT(40.77127782435138 73.74719932989527) bank173969 +173970 POINT(40.212777311056975 74.20453957955924) bank173970 +173971 POINT(40.407197707408095 74.27991136200164) bank173971 +173972 POINT(40.83375933274028 73.14305057927936) bank173972 +173973 POINT(40.508796131075655 74.101364227115) bank173973 +173974 POINT(40.14135930179407 73.5990365749587) bank173974 +173975 POINT(41.6959769576049 74.41913919596746) bank173975 +173976 POINT(40.49677217501566 74.83089264929922) bank173976 +173977 POINT(40.76010800416097 73.04780248958299) bank173977 +173978 POINT(40.7901590007191 73.45996478172756) bank173978 +173979 POINT(41.27878187997747 73.64830099723196) bank173979 +173980 POINT(41.5202806573307 73.84383533652431) bank173980 +173981 POINT(40.37226923785654 74.02884699314576) bank173981 +173982 POINT(40.24562980568447 73.87361216892232) bank173982 +173983 POINT(41.053720661042156 74.13615715220108) bank173983 +173984 POINT(40.82549223726661 74.45589098267682) bank173984 +173985 POINT(40.903953460666344 73.58904940257239) bank173985 +173986 POINT(41.1801667932736 73.60550587020022) bank173986 +173987 POINT(41.20918759258126 73.28761031984196) bank173987 +173988 POINT(41.442114117629615 74.2816559912405) bank173988 +173989 POINT(41.649219642690994 74.8182402204856) bank173989 +173990 POINT(40.076478420787346 73.5635053413665) bank173990 +173991 POINT(40.62359425199557 74.55553543678472) bank173991 +173992 POINT(41.09649938946846 73.5339246155596) bank173992 +173993 POINT(39.97624807066117 73.26244309473118) bank173993 +173994 POINT(39.86083609980936 74.09136435629394) bank173994 +173995 POINT(41.08936302248367 74.38987422432486) bank173995 +173996 POINT(39.81931060675206 74.41610565214371) bank173996 +173997 POINT(41.08235670646397 74.25714744193476) bank173997 +173998 POINT(39.93202582952764 73.64031073490227) bank173998 +173999 POINT(41.2292965967464 73.15766103050176) bank173999 +174000 POINT(41.54140992920981 73.01595984109419) bank174000 +174001 POINT(40.97591134038464 74.42839997785023) bank174001 +174002 POINT(40.94062992165638 75.0015684771889) bank174002 +174003 POINT(41.03691982788789 74.50033323010261) bank174003 +174004 POINT(40.47550680231691 74.84545802266481) bank174004 +174005 POINT(39.930690612755214 74.99803166027323) bank174005 +174006 POINT(40.07624674058921 74.14087207251659) bank174006 +174007 POINT(41.0367216190078 74.11881689372431) bank174007 +174008 POINT(41.01640564611068 73.40741967184597) bank174008 +174009 POINT(40.47334871584715 73.8968736370075) bank174009 +174010 POINT(40.06663833338965 73.69395813686756) bank174010 +174011 POINT(40.16026160839175 73.64357008702889) bank174011 +174012 POINT(41.68431152592057 74.41286576781809) bank174012 +174013 POINT(39.83964369826811 74.640318060322) bank174013 +174014 POINT(41.580974395693374 74.54419187306794) bank174014 +174015 POINT(41.49961421580172 74.26915678288418) bank174015 +174016 POINT(41.68819137225922 73.57274089381797) bank174016 +174017 POINT(40.7000853765706 74.3025439000743) bank174017 +174018 POINT(39.969086989979914 73.14011609071052) bank174018 +174019 POINT(40.661960066541624 73.21315982437554) bank174019 +174020 POINT(39.94785510579133 74.28718645130475) bank174020 +174021 POINT(40.63555034336557 74.36051676052982) bank174021 +174022 POINT(40.24129999147027 73.59866162953494) bank174022 +174023 POINT(40.71703899923257 74.04396143082714) bank174023 +174024 POINT(41.023118932298715 74.36627313155552) bank174024 +174025 POINT(40.32305423555418 73.42276380816583) bank174025 +174026 POINT(40.7574578665035 73.65083400522735) bank174026 +174027 POINT(41.05188194523352 73.40187889787391) bank174027 +174028 POINT(39.92548316317877 74.10631533338454) bank174028 +174029 POINT(40.87862316915656 73.77430987498681) bank174029 +174030 POINT(40.136592741293754 73.11518544675062) bank174030 +174031 POINT(41.3613772671875 74.15647735202269) bank174031 +174032 POINT(40.61636157541628 73.97628443303721) bank174032 +174033 POINT(41.298109200801356 73.23974758536835) bank174033 +174034 POINT(40.450455966832855 73.20390717740864) bank174034 +174035 POINT(40.78984651209424 73.92731794694414) bank174035 +174036 POINT(41.35237465435168 73.54123417634169) bank174036 +174037 POINT(40.01207891388528 73.13869413289667) bank174037 +174038 POINT(41.6461591703212 74.41482897371053) bank174038 +174039 POINT(41.06268950559465 74.30708379207475) bank174039 +174040 POINT(40.506459620818 73.52333519880932) bank174040 +174041 POINT(41.28426192556505 73.01624869411621) bank174041 +174042 POINT(40.30484380308253 74.29339919039988) bank174042 +174043 POINT(41.47249393939555 73.90168965112402) bank174043 +174044 POINT(40.54676073399819 73.74775433121245) bank174044 +174045 POINT(41.05870655555485 74.379481740285) bank174045 +174046 POINT(40.10480163321451 73.11536595418421) bank174046 +174047 POINT(41.077108379981865 74.41630853493676) bank174047 +174048 POINT(40.24979957597787 74.30493430445566) bank174048 +174049 POINT(40.9940633433395 73.69426690280226) bank174049 +174050 POINT(41.48030164626602 73.13641175507504) bank174050 +174051 POINT(39.82723748309065 74.65819751255422) bank174051 +174052 POINT(39.75582117335304 73.67474694401547) bank174052 +174053 POINT(41.06507185892817 73.71774696709109) bank174053 +174054 POINT(40.402208279967766 74.51994780080379) bank174054 +174055 POINT(39.902054083630524 73.72474674457695) bank174055 +174056 POINT(40.62325960904937 74.18879558761844) bank174056 +174057 POINT(40.71058531059754 74.28340823100841) bank174057 +174058 POINT(39.92070646855301 73.97572597682425) bank174058 +174059 POINT(41.3847833612728 73.9051142316932) bank174059 +174060 POINT(41.39127774197683 74.65945074283853) bank174060 +174061 POINT(40.10324180398103 74.1510632063197) bank174061 +174062 POINT(40.50782823398409 73.5308516496652) bank174062 +174063 POINT(41.042998461147015 73.58472405977355) bank174063 +174064 POINT(39.845263277307545 73.84058369801524) bank174064 +174065 POINT(39.72035678499541 74.37002564966886) bank174065 +174066 POINT(41.365948234538074 74.09846514791774) bank174066 +174067 POINT(40.961928938407084 74.86957226123856) bank174067 +174068 POINT(40.987958266436785 73.14906049390714) bank174068 +174069 POINT(41.43580076099575 74.89015188010596) bank174069 +174070 POINT(41.45029759100639 73.14427538787434) bank174070 +174071 POINT(41.612034723896834 73.56241863068851) bank174071 +174072 POINT(40.844570130729494 74.35143828940187) bank174072 +174073 POINT(40.48921159076256 74.69424557364731) bank174073 +174074 POINT(40.557142377670104 73.38134105000145) bank174074 +174075 POINT(41.18504862362851 74.0734887647072) bank174075 +174076 POINT(40.650537870215146 74.26237422991493) bank174076 +174077 POINT(40.00566269601943 73.6055671908254) bank174077 +174078 POINT(41.25692220200937 74.87889906616209) bank174078 +174079 POINT(39.80004877921583 73.15524416712039) bank174079 +174080 POINT(40.488317079408326 74.69218481696224) bank174080 +174081 POINT(41.02362576509453 73.16457281204531) bank174081 +174082 POINT(39.816338600549585 74.69972896542335) bank174082 +174083 POINT(41.61353453081824 73.72116459830357) bank174083 +174084 POINT(41.383626787165596 74.93277874846065) bank174084 +174085 POINT(40.7321581651182 73.89623371731237) bank174085 +174086 POINT(41.2751789319766 73.49777606004228) bank174086 +174087 POINT(41.56419362640505 74.15121595903416) bank174087 +174088 POINT(40.91041065516194 74.82627634639465) bank174088 +174089 POINT(41.636804888882644 74.58335603661494) bank174089 +174090 POINT(41.46368922457337 73.17975721810667) bank174090 +174091 POINT(41.58219118002569 73.18059409242436) bank174091 +174092 POINT(40.862526956737334 74.17993991507458) bank174092 +174093 POINT(40.923366128325966 74.89188679201101) bank174093 +174094 POINT(40.83464894196277 74.39596123032184) bank174094 +174095 POINT(41.22084939449764 73.96287560035742) bank174095 +174096 POINT(39.98915898299903 73.4992223033755) bank174096 +174097 POINT(41.305022614499954 73.75476042339697) bank174097 +174098 POINT(41.072747631072545 74.22883452700898) bank174098 +174099 POINT(40.958175406981795 74.12738520156594) bank174099 +174100 POINT(41.574635417536115 73.46744085359352) bank174100 +174101 POINT(41.406522906551345 73.65566617243765) bank174101 +174102 POINT(41.07599789838261 74.16589990478207) bank174102 +174103 POINT(41.42916370907037 74.88826855150724) bank174103 +174104 POINT(41.481379424971195 73.34445286553543) bank174104 +174105 POINT(39.75505736942947 74.0102164347646) bank174105 +174106 POINT(40.39456702412906 73.315465946659) bank174106 +174107 POINT(40.006834741105536 73.49848322171474) bank174107 +174108 POINT(40.91200086349889 73.9036746160018) bank174108 +174109 POINT(39.85220778388982 73.79635112869128) bank174109 +174110 POINT(40.32163821888145 73.29610485685927) bank174110 +174111 POINT(40.568049176774835 73.12754927798787) bank174111 +174112 POINT(41.37412682641175 73.21147837536434) bank174112 +174113 POINT(40.41976313748044 73.38060117599792) bank174113 +174114 POINT(40.96662073247807 73.69166262542478) bank174114 +174115 POINT(40.36416936704333 73.74024997302668) bank174115 +174116 POINT(40.17260832067765 73.0602517117415) bank174116 +174117 POINT(39.982499880461546 73.85519511275587) bank174117 +174118 POINT(39.90438579590881 74.91600126155473) bank174118 +174119 POINT(39.959954794311955 73.09428988975124) bank174119 +174120 POINT(41.39175314281279 73.80026231104696) bank174120 +174121 POINT(41.69498169787424 73.70631873708524) bank174121 +174122 POINT(40.039543352695475 74.95956571692147) bank174122 +174123 POINT(39.92747941814625 73.92765782251028) bank174123 +174124 POINT(41.01722976198601 73.98752895291253) bank174124 +174125 POINT(40.286053315513165 74.4407204221515) bank174125 +174126 POINT(40.547150749143874 73.72377326070952) bank174126 +174127 POINT(41.10161289124668 73.95334674118116) bank174127 +174128 POINT(41.44963674851571 73.55098785930956) bank174128 +174129 POINT(40.028480697886806 73.3412490089462) bank174129 +174130 POINT(41.70929373987049 74.35111855125457) bank174130 +174131 POINT(41.59648693030033 73.34495393674148) bank174131 +174132 POINT(40.469611375217 74.31972688755268) bank174132 +174133 POINT(39.76303184944898 73.16740954029993) bank174133 +174134 POINT(39.867499041147674 73.4705870066067) bank174134 +174135 POINT(41.25892573398795 74.68583268135225) bank174135 +174136 POINT(39.968534193573355 74.30280520255441) bank174136 +174137 POINT(41.11603529609133 73.05226413812512) bank174137 +174138 POINT(41.15026953135172 73.66634713963771) bank174138 +174139 POINT(40.53608056657409 74.18859593178817) bank174139 +174140 POINT(41.09503115953401 74.50405527402056) bank174140 +174141 POINT(39.76049322556922 73.79057261326075) bank174141 +174142 POINT(40.14946255874252 73.324756267647) bank174142 +174143 POINT(41.17644399490315 73.90833454132225) bank174143 +174144 POINT(41.5003279267585 73.14182217559114) bank174144 +174145 POINT(41.706219419990205 73.77010306768197) bank174145 +174146 POINT(41.27270026385026 74.5081122791576) bank174146 +174147 POINT(40.6143342588955 73.61611851665785) bank174147 +174148 POINT(40.74036234229644 74.71047614297085) bank174148 +174149 POINT(39.823325351747386 73.1263428783014) bank174149 +174150 POINT(40.16546406696167 74.04423918639445) bank174150 +174151 POINT(41.3649371296147 74.12242345170544) bank174151 +174152 POINT(41.710589796077976 74.90388773380656) bank174152 +174153 POINT(41.39425374664581 74.3677005323793) bank174153 +174154 POINT(40.124985259198645 73.56428868558633) bank174154 +174155 POINT(40.943669583001295 74.19443410036293) bank174155 +174156 POINT(41.43421756654292 73.15419640013252) bank174156 +174157 POINT(40.7143693057649 74.2365401568924) bank174157 +174158 POINT(39.866458504436686 73.74522496297742) bank174158 +174159 POINT(39.7241179639935 74.36636297014861) bank174159 +174160 POINT(40.37047317799025 73.43112025924067) bank174160 +174161 POINT(41.128707365056805 74.03677001508007) bank174161 +174162 POINT(41.310159745141696 74.05253929650674) bank174162 +174163 POINT(40.50810632432225 73.90143065775028) bank174163 +174164 POINT(40.18845976429299 74.79085741286569) bank174164 +174165 POINT(41.191204837194014 73.26218972449426) bank174165 +174166 POINT(40.56541579366648 74.89055328597286) bank174166 +174167 POINT(40.90587004018458 74.6712217746491) bank174167 +174168 POINT(39.72078316503962 74.18474864906192) bank174168 +174169 POINT(41.65592382001476 74.4838119677011) bank174169 +174170 POINT(40.754322473386736 73.66980921248613) bank174170 +174171 POINT(40.0711870178466 74.57475158977758) bank174171 +174172 POINT(40.08754471322027 74.29264732064493) bank174172 +174173 POINT(41.19354588919564 73.29267371696137) bank174173 +174174 POINT(41.46779860104458 73.46896498638291) bank174174 +174175 POINT(40.494758338789765 73.3028903007004) bank174175 +174176 POINT(41.428813086542235 74.70649793696784) bank174176 +174177 POINT(41.336104349687325 73.61586517426326) bank174177 +174178 POINT(40.15339052876714 73.24950067696258) bank174178 +174179 POINT(40.33331863477895 74.94479974214771) bank174179 +174180 POINT(41.40755904468002 74.82947493876868) bank174180 +174181 POINT(39.837210575103114 74.31106327257604) bank174181 +174182 POINT(40.191166708224195 73.23818683948926) bank174182 +174183 POINT(39.739629003999895 73.73859168643932) bank174183 +174184 POINT(41.474050718743115 74.85348780423284) bank174184 +174185 POINT(41.0334941353226 74.57395767014853) bank174185 +174186 POINT(39.804724171938524 74.1511790860161) bank174186 +174187 POINT(41.678153246845966 74.9189639855485) bank174187 +174188 POINT(41.4726311065281 74.76529702807433) bank174188 +174189 POINT(40.611842782909875 74.17488187627329) bank174189 +174190 POINT(39.85790126354863 74.80490536765167) bank174190 +174191 POINT(41.253887980350925 73.14160442090947) bank174191 +174192 POINT(41.30250028686338 73.77373281894599) bank174192 +174193 POINT(41.37888737570661 73.99518928177393) bank174193 +174194 POINT(41.20542485119296 74.50057274494795) bank174194 +174195 POINT(40.08375473545024 74.99333380307998) bank174195 +174196 POINT(41.15487680369339 73.86821809897776) bank174196 +174197 POINT(40.03289648902921 73.02519105254943) bank174197 +174198 POINT(40.46422265019654 73.5483034076163) bank174198 +174199 POINT(40.40795219041916 74.9987278773404) bank174199 +174200 POINT(40.56511101690872 73.11614926492196) bank174200 +174201 POINT(39.844489376190104 73.00906641775141) bank174201 +174202 POINT(40.95448901413564 73.09470948619409) bank174202 +174203 POINT(39.925360661265195 74.00251340468014) bank174203 +174204 POINT(41.341987568613064 73.57124192319083) bank174204 +174205 POINT(40.170082333048036 74.49131372931257) bank174205 +174206 POINT(41.431377890842136 74.73103469459348) bank174206 +174207 POINT(40.81426096424018 73.37534086984436) bank174207 +174208 POINT(41.198839531856144 73.29533335973612) bank174208 +174209 POINT(41.047520768654884 73.64412309134755) bank174209 +174210 POINT(39.88833704287609 74.12083013902624) bank174210 +174211 POINT(41.2198382492497 73.95961942029655) bank174211 +174212 POINT(40.65900216932213 74.7808949566501) bank174212 +174213 POINT(41.19207709239389 74.68376838041335) bank174213 +174214 POINT(41.1895434371453 74.83351906892958) bank174214 +174215 POINT(41.33806554891356 74.3963809348999) bank174215 +174216 POINT(39.880730789003714 74.22359389579215) bank174216 +174217 POINT(41.60578412656425 73.37387457958248) bank174217 +174218 POINT(41.00628913226335 73.1994008807004) bank174218 +174219 POINT(40.80500707535686 73.46490005778236) bank174219 +174220 POINT(40.637407650885294 74.26392767933045) bank174220 +174221 POINT(40.281920472345206 74.29844632603914) bank174221 +174222 POINT(40.20886539662879 74.7553398044021) bank174222 +174223 POINT(40.37338492436261 74.33677241305169) bank174223 +174224 POINT(39.76173824461424 74.89568495014183) bank174224 +174225 POINT(40.10885251734307 73.20756942927297) bank174225 +174226 POINT(41.05801922425245 74.06789332348438) bank174226 +174227 POINT(40.28487379512489 73.44090688365885) bank174227 +174228 POINT(40.86819467953868 73.54317966342603) bank174228 +174229 POINT(40.70651556662638 73.81583241026765) bank174229 +174230 POINT(40.17151021494194 74.03719959113813) bank174230 +174231 POINT(41.35895668066545 73.81499204473154) bank174231 +174232 POINT(40.51976550190241 73.24290797800202) bank174232 +174233 POINT(41.09276529984369 74.22546130152097) bank174233 +174234 POINT(40.855614841319024 73.9893591021469) bank174234 +174235 POINT(39.74903113344086 74.93076966440336) bank174235 +174236 POINT(40.51499113041177 73.07220305464784) bank174236 +174237 POINT(39.973238836196614 74.54470713057572) bank174237 +174238 POINT(39.74967134922925 74.89248030250911) bank174238 +174239 POINT(39.95965838863304 74.88047776577469) bank174239 +174240 POINT(41.036635624480574 74.11793581339545) bank174240 +174241 POINT(40.234346225310546 74.73910235307827) bank174241 +174242 POINT(40.88521914376408 73.03643139427018) bank174242 +174243 POINT(40.635553366008075 74.91676580783603) bank174243 +174244 POINT(40.542097127666835 74.41035203533087) bank174244 +174245 POINT(40.40999289359508 73.75495022297508) bank174245 +174246 POINT(41.12812645708296 73.90412299683271) bank174246 +174247 POINT(40.8593481460043 74.44407007141332) bank174247 +174248 POINT(41.34379608969528 73.35219051166474) bank174248 +174249 POINT(41.708797058550346 74.45096669168345) bank174249 +174250 POINT(40.944516178579654 74.01921092128384) bank174250 +174251 POINT(40.97790829616048 74.63503087445243) bank174251 +174252 POINT(41.630449799361315 73.40100737063426) bank174252 +174253 POINT(41.546056051306245 74.59470522105016) bank174253 +174254 POINT(41.40695777347128 74.46387534594815) bank174254 +174255 POINT(40.541227362070344 73.71755771854929) bank174255 +174256 POINT(40.74366807335316 74.60454265214452) bank174256 +174257 POINT(41.50238834766662 74.38379011370485) bank174257 +174258 POINT(41.095335666057814 74.22390012362149) bank174258 +174259 POINT(40.94594555352825 73.52054904688062) bank174259 +174260 POINT(41.36687508895573 73.81689589310616) bank174260 +174261 POINT(41.01356549494885 74.97414417506188) bank174261 +174262 POINT(41.54568888226708 73.97292383655278) bank174262 +174263 POINT(40.38628555999018 74.57798201202813) bank174263 +174264 POINT(40.4607548811882 73.45537268316197) bank174264 +174265 POINT(39.93109669130857 73.01408202040204) bank174265 +174266 POINT(40.539560823273376 74.20819719867758) bank174266 +174267 POINT(40.43198305457497 73.67916100534491) bank174267 +174268 POINT(41.269668802959075 74.62182512146627) bank174268 +174269 POINT(40.83244497644476 73.74201622989001) bank174269 +174270 POINT(39.78357501125505 74.26837475985747) bank174270 +174271 POINT(40.04966939654511 74.34528771056691) bank174271 +174272 POINT(41.56604354330383 74.28651580819029) bank174272 +174273 POINT(41.571229248502384 73.90746690012996) bank174273 +174274 POINT(41.28561069525376 73.93537949568749) bank174274 +174275 POINT(40.23908869946521 73.88601421367736) bank174275 +174276 POINT(41.30142736658943 73.83228071263693) bank174276 +174277 POINT(40.64684459691653 74.64812160508959) bank174277 +174278 POINT(41.19968999588475 73.4516875147901) bank174278 +174279 POINT(41.099841531265525 74.74048641726577) bank174279 +174280 POINT(41.296168033360715 74.07104201128838) bank174280 +174281 POINT(41.01755105910736 73.11831368312876) bank174281 +174282 POINT(40.70183514144222 74.36046334972275) bank174282 +174283 POINT(39.9814589008657 74.52043936546214) bank174283 +174284 POINT(40.52644578851939 74.96155998250701) bank174284 +174285 POINT(40.767158451111996 73.95247876339825) bank174285 +174286 POINT(40.48577399869928 74.22539060669939) bank174286 +174287 POINT(39.986307259914966 74.16019063686531) bank174287 +174288 POINT(41.1655292281522 74.70459694769127) bank174288 +174289 POINT(39.97489103547565 74.84791803698248) bank174289 +174290 POINT(40.245299864268276 74.8575768828117) bank174290 +174291 POINT(40.0995326076095 73.34086051429408) bank174291 +174292 POINT(40.550288307225074 74.16415344865892) bank174292 +174293 POINT(40.18291177383404 73.66293732816814) bank174293 +174294 POINT(40.20450419883247 74.96977160047786) bank174294 +174295 POINT(40.86618750509561 73.13687416558845) bank174295 +174296 POINT(40.57724241495116 74.11891943418318) bank174296 +174297 POINT(39.74566292659167 73.30111932904339) bank174297 +174298 POINT(40.33275366666572 73.14668848927874) bank174298 +174299 POINT(39.8863009498063 74.94364840934097) bank174299 +174300 POINT(40.83092132622491 74.50128118661165) bank174300 +174301 POINT(40.69014813451332 73.2694306982088) bank174301 +174302 POINT(40.99748133582883 74.76392938910138) bank174302 +174303 POINT(39.91140383020233 74.46975566683673) bank174303 +174304 POINT(41.222879743236184 74.70133502035269) bank174304 +174305 POINT(39.83662327432616 74.55111168543742) bank174305 +174306 POINT(41.341046720239774 73.55457255041735) bank174306 +174307 POINT(40.94375202675353 74.26341889721576) bank174307 +174308 POINT(40.09218934600608 74.05089840569998) bank174308 +174309 POINT(40.635818738985876 74.52695812998172) bank174309 +174310 POINT(40.9440942426192 73.14681700431889) bank174310 +174311 POINT(40.430886078822056 73.51833477880115) bank174311 +174312 POINT(41.433383970087135 73.94218020579146) bank174312 +174313 POINT(41.295203459807496 73.90966120898523) bank174313 +174314 POINT(40.779587853521974 74.51910433097065) bank174314 +174315 POINT(39.77025647230017 74.60143377044028) bank174315 +174316 POINT(41.52637994351554 74.39381529972187) bank174316 +174317 POINT(40.46803349673023 73.90308284301604) bank174317 +174318 POINT(40.77974484227287 73.88688982886579) bank174318 +174319 POINT(41.3109802168889 73.07442343429481) bank174319 +174320 POINT(41.4607960259528 73.24237346750631) bank174320 +174321 POINT(40.90572378496065 74.65304807150557) bank174321 +174322 POINT(40.94706911838426 74.93753763878314) bank174322 +174323 POINT(40.20082102877951 74.52657194512109) bank174323 +174324 POINT(40.25667403974871 73.55400725951061) bank174324 +174325 POINT(39.833801467189936 73.0753650654994) bank174325 +174326 POINT(40.88991137784974 73.97683434973439) bank174326 +174327 POINT(40.75767824721192 74.50392621550405) bank174327 +174328 POINT(41.62948223196919 73.31818952256452) bank174328 +174329 POINT(40.46890773727134 74.89084615339543) bank174329 +174330 POINT(40.97478412830239 74.77126226753171) bank174330 +174331 POINT(40.90692917860179 74.19209680283204) bank174331 +174332 POINT(40.92078648391709 74.84752442466785) bank174332 +174333 POINT(40.77020356116816 73.95930573917654) bank174333 +174334 POINT(40.670767570687126 74.47725213294703) bank174334 +174335 POINT(41.66587391747686 73.67574555315736) bank174335 +174336 POINT(41.38273721981216 73.1398639764991) bank174336 +174337 POINT(39.88660082615952 74.42708747416931) bank174337 +174338 POINT(40.37429117125155 74.26486432431811) bank174338 +174339 POINT(41.46687927537215 74.30585787785077) bank174339 +174340 POINT(41.324084737009635 74.1084766613595) bank174340 +174341 POINT(40.243597772961415 74.26827879189959) bank174341 +174342 POINT(40.950593820871646 73.10552365567891) bank174342 +174343 POINT(41.07291817276141 73.1823516457656) bank174343 +174344 POINT(41.539509043893894 73.97090201873515) bank174344 +174345 POINT(39.952831578616774 74.81073989236289) bank174345 +174346 POINT(41.07882567572949 74.89388926432524) bank174346 +174347 POINT(40.77630620223226 74.56469128427736) bank174347 +174348 POINT(41.246389308294624 73.77235336678936) bank174348 +174349 POINT(40.264180826943054 74.60185584120508) bank174349 +174350 POINT(41.61220970431307 73.75204161298329) bank174350 +174351 POINT(40.64876013935164 73.4998826265348) bank174351 +174352 POINT(40.30481408881722 73.55712287116188) bank174352 +174353 POINT(40.2892746420383 74.29720599904954) bank174353 +174354 POINT(41.49877854248367 74.24072914622396) bank174354 +174355 POINT(39.74305975361033 73.91252946889486) bank174355 +174356 POINT(40.694765340405496 74.8378969527638) bank174356 +174357 POINT(39.992579604527975 74.41924886101303) bank174357 +174358 POINT(39.85180045461768 73.6185526612486) bank174358 +174359 POINT(40.61507197484922 73.33855865371073) bank174359 +174360 POINT(40.02459636345195 73.85187664559656) bank174360 +174361 POINT(40.76662930511456 73.69556816112755) bank174361 +174362 POINT(40.542130164050576 73.88394663518177) bank174362 +174363 POINT(40.47202530187585 73.6687395323363) bank174363 +174364 POINT(40.324573092493466 74.61671329884166) bank174364 +174365 POINT(40.18611374002534 73.10178947816372) bank174365 +174366 POINT(40.3672357756894 74.39544981547498) bank174366 +174367 POINT(41.0572217904735 73.80773326290694) bank174367 +174368 POINT(41.640985410014956 74.31404054589188) bank174368 +174369 POINT(40.433978116150286 74.6135603714771) bank174369 +174370 POINT(41.175140601187344 74.60469759668106) bank174370 +174371 POINT(41.3811633425256 73.25987926157529) bank174371 +174372 POINT(41.386176389000305 73.42166338445551) bank174372 +174373 POINT(41.20642308276462 73.8168689672333) bank174373 +174374 POINT(40.8180257417987 74.36513252509467) bank174374 +174375 POINT(39.981857905014415 74.44197794749968) bank174375 +174376 POINT(41.53364446469548 73.13945095697555) bank174376 +174377 POINT(40.308880609606405 73.70293732912863) bank174377 +174378 POINT(40.25862184500697 74.07887008922268) bank174378 +174379 POINT(40.39527019157799 74.02537103130093) bank174379 +174380 POINT(41.367651472652064 74.82964080781467) bank174380 +174381 POINT(40.519245413151445 74.96868405147416) bank174381 +174382 POINT(41.441891404942126 74.25871977214757) bank174382 +174383 POINT(40.761973951899556 74.04776208058715) bank174383 +174384 POINT(40.26182051065783 73.40517904912576) bank174384 +174385 POINT(41.00546434508344 74.87924130065933) bank174385 +174386 POINT(39.90268043093462 74.25642567254773) bank174386 +174387 POINT(41.03798728155258 73.56949265372808) bank174387 +174388 POINT(40.15958077116469 74.28759884011704) bank174388 +174389 POINT(41.58670988699187 73.86238098938799) bank174389 +174390 POINT(40.65218640182276 73.83687734819847) bank174390 +174391 POINT(40.623890936101546 74.17654688685717) bank174391 +174392 POINT(40.828500878554934 74.35766407430211) bank174392 +174393 POINT(41.117599793158874 74.41005069427723) bank174393 +174394 POINT(41.49114612750471 74.60884482068407) bank174394 +174395 POINT(41.50378795636693 73.06688294459508) bank174395 +174396 POINT(40.58705380010646 74.74886732395004) bank174396 +174397 POINT(40.99523689322986 74.99278524329573) bank174397 +174398 POINT(41.12419852255733 73.80427193862748) bank174398 +174399 POINT(40.17899285401608 74.65929549050345) bank174399 +174400 POINT(40.203328517057635 74.94042581686281) bank174400 +174401 POINT(40.191431071734485 74.17972967079325) bank174401 +174402 POINT(39.90953292093371 73.8609782976577) bank174402 +174403 POINT(40.785522913087505 74.21276604275307) bank174403 +174404 POINT(40.055862919434325 73.31005470973581) bank174404 +174405 POINT(41.428976465623194 74.73416720146231) bank174405 +174406 POINT(41.47977760749817 73.60673835766843) bank174406 +174407 POINT(39.938238310515494 74.89531038315262) bank174407 +174408 POINT(40.822272260785226 73.16266582999553) bank174408 +174409 POINT(41.69031173715937 74.17815196591214) bank174409 +174410 POINT(41.38999574574762 73.10017283673388) bank174410 +174411 POINT(40.64774464639353 73.30936273261426) bank174411 +174412 POINT(40.563526450965874 74.891400227953) bank174412 +174413 POINT(41.31377855839342 73.2140381968112) bank174413 +174414 POINT(41.36102093234746 73.72238541747136) bank174414 +174415 POINT(41.33805105564774 73.67605010617768) bank174415 +174416 POINT(39.78927599142959 74.28211697448253) bank174416 +174417 POINT(41.54177440546136 74.28737290301878) bank174417 +174418 POINT(40.23547130448931 74.61164231602703) bank174418 +174419 POINT(41.5733123724244 74.5491345790968) bank174419 +174420 POINT(41.299557482355446 74.81867862132735) bank174420 +174421 POINT(40.371337610810585 74.26383192051861) bank174421 +174422 POINT(40.93047337313909 73.69766039260045) bank174422 +174423 POINT(39.855625158648216 74.84256360494452) bank174423 +174424 POINT(40.35359394263756 73.18623219037217) bank174424 +174425 POINT(40.022401063361876 73.82550370806895) bank174425 +174426 POINT(40.4170155729643 73.25220124179184) bank174426 +174427 POINT(40.55215769246808 73.30606734589581) bank174427 +174428 POINT(41.07209450462997 74.83766432219447) bank174428 +174429 POINT(40.26627795241505 73.67257960311251) bank174429 +174430 POINT(40.974481598681166 73.41060217674425) bank174430 +174431 POINT(40.17318140419093 74.36589563849186) bank174431 +174432 POINT(41.19562757445981 73.77040792866805) bank174432 +174433 POINT(40.45820913598246 73.48972141328616) bank174433 +174434 POINT(41.44204872652529 73.9461441765675) bank174434 +174435 POINT(41.29259008596054 74.7437240658347) bank174435 +174436 POINT(40.75599276967007 74.67731172963857) bank174436 +174437 POINT(40.185647073754154 74.01021272032861) bank174437 +174438 POINT(39.91862387761051 74.7840360569763) bank174438 +174439 POINT(40.079867391308944 74.62092900579856) bank174439 +174440 POINT(39.743867895001685 73.7990358288681) bank174440 +174441 POINT(40.31471157255758 74.10642056781349) bank174441 +174442 POINT(40.89782198023135 74.01169651798122) bank174442 +174443 POINT(40.533249629708564 73.99272739948711) bank174443 +174444 POINT(41.06041127757196 73.69003546429478) bank174444 +174445 POINT(40.15334882197186 73.15764651185302) bank174445 +174446 POINT(41.35512268466109 74.40722148685346) bank174446 +174447 POINT(39.917064149192164 73.06985907274817) bank174447 +174448 POINT(40.38329173692349 74.7742734671819) bank174448 +174449 POINT(41.555931988033734 73.96212717050236) bank174449 +174450 POINT(41.553339039583186 73.23610409634676) bank174450 +174451 POINT(41.20560303398341 74.7094424365501) bank174451 +174452 POINT(41.57455991215995 73.82336536112172) bank174452 +174453 POINT(39.78214038016815 73.70861726665561) bank174453 +174454 POINT(39.79691567351326 73.77405929892198) bank174454 +174455 POINT(41.23760826718703 74.89244161818148) bank174455 +174456 POINT(40.404712970776565 74.19230236858367) bank174456 +174457 POINT(41.190998405874936 74.31381940814008) bank174457 +174458 POINT(40.85983576832704 74.68250335155693) bank174458 +174459 POINT(40.967819267345924 73.6269437814797) bank174459 +174460 POINT(41.08149256027449 73.9709630596276) bank174460 +174461 POINT(41.4769364146562 73.38675931949996) bank174461 +174462 POINT(41.535390149670036 73.53292999164474) bank174462 +174463 POINT(41.05665624117648 74.23125786259666) bank174463 +174464 POINT(39.945333414215035 74.15778028748124) bank174464 +174465 POINT(39.98107311577713 73.02945506511618) bank174465 +174466 POINT(39.80803456965895 74.33920836084744) bank174466 +174467 POINT(40.81391557125873 74.52620649978232) bank174467 +174468 POINT(40.08598893481567 74.21240363410381) bank174468 +174469 POINT(40.91188468191329 73.18909792339616) bank174469 +174470 POINT(40.91353498822673 74.96301682653147) bank174470 +174471 POINT(41.71183397356579 74.51794075380616) bank174471 +174472 POINT(41.239230711252226 74.088710349985) bank174472 +174473 POINT(41.19978809102057 73.73990182210845) bank174473 +174474 POINT(40.791099794798065 73.37034656865882) bank174474 +174475 POINT(40.10352495276284 73.31202196515557) bank174475 +174476 POINT(40.737165451061074 74.6631572386577) bank174476 +174477 POINT(40.24995765081804 73.3321322554541) bank174477 +174478 POINT(40.240231936586994 74.92733566527859) bank174478 +174479 POINT(41.62736894410952 74.96461356447601) bank174479 +174480 POINT(41.134285133686376 74.02124621788417) bank174480 +174481 POINT(40.238483595438595 73.96858627148036) bank174481 +174482 POINT(41.51400589720376 74.1782802262437) bank174482 +174483 POINT(41.63931830542725 74.52875370346753) bank174483 +174484 POINT(41.533773377874596 74.5806965954229) bank174484 +174485 POINT(41.584530175252674 73.26980597246516) bank174485 +174486 POINT(40.1556827363138 73.0771419763945) bank174486 +174487 POINT(40.15640981509475 74.58186167543312) bank174487 +174488 POINT(40.030177569115146 74.35995915424287) bank174488 +174489 POINT(41.1550666714691 73.46413209628366) bank174489 +174490 POINT(40.285354034189474 73.16193771566444) bank174490 +174491 POINT(40.981252728819484 73.89047187118065) bank174491 +174492 POINT(39.79122482556213 73.44946711247093) bank174492 +174493 POINT(41.362272833608735 73.52769183651385) bank174493 +174494 POINT(41.31436440185467 74.04372418615492) bank174494 +174495 POINT(41.596787856545454 74.23078192243626) bank174495 +174496 POINT(39.885818058523114 73.31993676869212) bank174496 +174497 POINT(40.120786031437376 73.08372250527395) bank174497 +174498 POINT(41.064711265107775 74.40186521922833) bank174498 +174499 POINT(39.84119396160205 74.55324095333025) bank174499 +174500 POINT(39.753538432055656 74.49769371863052) bank174500 +174501 POINT(41.015269000968466 73.3346181256794) bank174501 +174502 POINT(41.66797743600322 74.84660745805117) bank174502 +174503 POINT(41.228525528296835 73.55210769857047) bank174503 +174504 POINT(40.84710814110848 75.00020309766502) bank174504 +174505 POINT(41.70356876635892 74.3669478213609) bank174505 +174506 POINT(40.666534502474846 74.32613605402618) bank174506 +174507 POINT(40.77775127412204 74.74143464893389) bank174507 +174508 POINT(40.40509664220138 73.19703465535966) bank174508 +174509 POINT(41.551349954833945 74.02381610479102) bank174509 +174510 POINT(40.004151228273216 73.8979505977779) bank174510 +174511 POINT(40.42959511699788 74.6956612885169) bank174511 +174512 POINT(40.301342362446015 74.40122807659095) bank174512 +174513 POINT(39.820628439176026 73.44107662335236) bank174513 +174514 POINT(40.53061847524081 73.92000403765272) bank174514 +174515 POINT(40.085620146666095 74.49535471582828) bank174515 +174516 POINT(40.027399525970885 73.75172057999697) bank174516 +174517 POINT(41.0326155232189 73.3936253488441) bank174517 +174518 POINT(41.3043254184333 74.69781122763398) bank174518 +174519 POINT(40.882658982098114 73.60171438619453) bank174519 +174520 POINT(40.7265090326786 74.13026007093076) bank174520 +174521 POINT(41.08312065165879 73.81325570543954) bank174521 +174522 POINT(41.44018483046248 73.13234637734612) bank174522 +174523 POINT(41.05758043554297 74.29674435658664) bank174523 +174524 POINT(39.89360760852336 74.88455620149597) bank174524 +174525 POINT(41.43591267948272 74.00821327779431) bank174525 +174526 POINT(40.40588951579344 74.13183665627349) bank174526 +174527 POINT(40.74903338845238 73.01966074701893) bank174527 +174528 POINT(40.557153779559634 74.86873639682308) bank174528 +174529 POINT(40.31164822485083 73.16046406829292) bank174529 +174530 POINT(40.26043657241377 74.40848397869141) bank174530 +174531 POINT(41.27091645650381 73.68954153146292) bank174531 +174532 POINT(41.55255323667427 74.25976538693925) bank174532 +174533 POINT(41.01816114465849 73.44026954607936) bank174533 +174534 POINT(40.74191686928012 73.27269235766983) bank174534 +174535 POINT(40.18652112345956 73.27891694813873) bank174535 +174536 POINT(40.2698832289805 73.7927642222799) bank174536 +174537 POINT(40.240295789851984 74.44959575291969) bank174537 +174538 POINT(40.666361394446334 73.63979595072335) bank174538 +174539 POINT(40.18825459206056 73.38869377659162) bank174539 +174540 POINT(39.71627138575777 74.34704087558401) bank174540 +174541 POINT(40.84905338421161 73.19652709450011) bank174541 +174542 POINT(40.85465868588137 74.53094709868635) bank174542 +174543 POINT(41.08273672623394 74.18587494115029) bank174543 +174544 POINT(40.733367471344344 74.89486613391539) bank174544 +174545 POINT(40.53759688335524 74.87429126824956) bank174545 +174546 POINT(40.20300175707917 74.85081790340793) bank174546 +174547 POINT(40.48836527516014 73.79307038154246) bank174547 +174548 POINT(41.21598983308934 74.80839219350209) bank174548 +174549 POINT(41.607246565998196 74.60548321378344) bank174549 +174550 POINT(40.69956013759967 73.58846825455757) bank174550 +174551 POINT(40.7033120810603 73.04458700250571) bank174551 +174552 POINT(41.41597200478828 74.80720548986328) bank174552 +174553 POINT(40.612459962300555 74.75543786239508) bank174553 +174554 POINT(39.997175612675434 73.51792590932757) bank174554 +174555 POINT(39.85720450043055 73.4091512544339) bank174555 +174556 POINT(39.96999361522686 74.36635572721255) bank174556 +174557 POINT(41.18278700678018 74.97520308173013) bank174557 +174558 POINT(39.84635569938356 74.01337334324235) bank174558 +174559 POINT(39.846559760302604 73.07259142020085) bank174559 +174560 POINT(41.30636183411811 73.09955027105276) bank174560 +174561 POINT(41.52070761617787 73.95534091767404) bank174561 +174562 POINT(41.05078190120968 74.80799926381913) bank174562 +174563 POINT(40.092445159029076 74.5665431090107) bank174563 +174564 POINT(41.50860955996329 74.05842793440969) bank174564 +174565 POINT(39.80297677233224 73.11387372818064) bank174565 +174566 POINT(40.20715611937293 73.88703706467894) bank174566 +174567 POINT(41.53515624456625 74.19397288778707) bank174567 +174568 POINT(39.87801764856933 73.48944603100783) bank174568 +174569 POINT(41.409604647749184 73.72494058063927) bank174569 +174570 POINT(40.86691638005098 74.65181035445633) bank174570 +174571 POINT(41.02449217623394 73.4170825105147) bank174571 +174572 POINT(41.38612257557666 74.78218372431687) bank174572 +174573 POINT(41.015282493324854 73.16873139453477) bank174573 +174574 POINT(40.755120650260395 74.80121984687922) bank174574 +174575 POINT(41.27110234834935 74.71438472802107) bank174575 +174576 POINT(40.81395353493834 74.60801851667888) bank174576 +174577 POINT(41.50480914555726 73.72737849045546) bank174577 +174578 POINT(40.31240313325604 74.83029193918279) bank174578 +174579 POINT(41.61850750166073 74.89958895185804) bank174579 +174580 POINT(41.480243703646885 74.7526642889426) bank174580 +174581 POINT(40.93371820990421 73.3383814773782) bank174581 +174582 POINT(40.8430223947668 73.05671482720274) bank174582 +174583 POINT(41.55211179784103 74.9401403629338) bank174583 +174584 POINT(41.44539531481794 74.83208895694627) bank174584 +174585 POINT(39.73513673220173 74.3727810918104) bank174585 +174586 POINT(40.9572292507293 74.88394354426944) bank174586 +174587 POINT(40.04092888126799 74.21225724013802) bank174587 +174588 POINT(40.57783771272134 73.08844773447903) bank174588 +174589 POINT(41.237366701172284 73.62979558671225) bank174589 +174590 POINT(41.192246231838126 73.09585883748805) bank174590 +174591 POINT(39.99897119532025 74.2187530923589) bank174591 +174592 POINT(40.68117593788622 74.88310085510547) bank174592 +174593 POINT(41.43502909996677 74.63818059141796) bank174593 +174594 POINT(40.633771021466906 74.84619313746029) bank174594 +174595 POINT(41.09963046609964 73.36774501117955) bank174595 +174596 POINT(40.99006370025068 73.10975566519636) bank174596 +174597 POINT(41.39604546977854 73.66496345819284) bank174597 +174598 POINT(41.09618517183272 74.46528256598941) bank174598 +174599 POINT(41.65277824852563 74.18264504928212) bank174599 +174600 POINT(40.46409147309917 74.60745657916421) bank174600 +174601 POINT(40.56060152679255 74.37496060464156) bank174601 +174602 POINT(41.091059104172864 73.55241947579148) bank174602 +174603 POINT(41.55439207384286 74.23345280424927) bank174603 +174604 POINT(40.801794448261006 74.06692269281427) bank174604 +174605 POINT(40.69727820891698 74.49306596982281) bank174605 +174606 POINT(40.79953921451555 73.08403466235961) bank174606 +174607 POINT(40.0009202212944 73.32041496439578) bank174607 +174608 POINT(41.51118304410837 74.93776924161598) bank174608 +174609 POINT(41.66446425202052 74.45900283601404) bank174609 +174610 POINT(39.814790216111405 74.22204922896205) bank174610 +174611 POINT(40.87756018107572 74.81750912395256) bank174611 +174612 POINT(39.80597884204899 74.68659951397092) bank174612 +174613 POINT(39.84065596181405 74.68791652859626) bank174613 +174614 POINT(41.70953374828541 73.29264716346141) bank174614 +174615 POINT(41.69249989166264 74.14852541529352) bank174615 +174616 POINT(40.96491968533217 74.13316075369903) bank174616 +174617 POINT(40.41569184842403 74.69851621177838) bank174617 +174618 POINT(40.75535343719583 74.57122105762758) bank174618 +174619 POINT(41.24272446146746 74.01833379917682) bank174619 +174620 POINT(40.99997779900888 74.68937444365343) bank174620 +174621 POINT(39.96642949738196 73.63809245429391) bank174621 +174622 POINT(39.766848850404884 74.00835948662316) bank174622 +174623 POINT(40.99217658053386 73.99997898577634) bank174623 +174624 POINT(40.384532110569445 73.09943455522264) bank174624 +174625 POINT(40.20593839735641 74.72725484464539) bank174625 +174626 POINT(40.019018001263305 74.9362445328445) bank174626 +174627 POINT(39.78258742289227 74.19631147141034) bank174627 +174628 POINT(40.5506207358052 74.26573945581087) bank174628 +174629 POINT(39.95069032914636 74.35103433461563) bank174629 +174630 POINT(41.21740291278281 74.320748269899) bank174630 +174631 POINT(41.44728314087719 74.53661992349792) bank174631 +174632 POINT(40.38349792206691 74.591223052683) bank174632 +174633 POINT(40.65651197137231 73.79516888657037) bank174633 +174634 POINT(40.76289956223381 74.06309820810284) bank174634 +174635 POINT(39.892217907566575 74.24775420234214) bank174635 +174636 POINT(40.18821953933382 73.12073125555432) bank174636 +174637 POINT(40.57718106910406 74.16134913965475) bank174637 +174638 POINT(40.464012233893506 73.88861371891382) bank174638 +174639 POINT(41.48507311316469 74.82575380946807) bank174639 +174640 POINT(41.032986300751624 73.91405724795602) bank174640 +174641 POINT(41.68735237998777 73.7975334656649) bank174641 +174642 POINT(40.90771395110889 74.35501155499429) bank174642 +174643 POINT(39.71671943778323 74.144559085734) bank174643 +174644 POINT(41.41789458979919 73.45265803804261) bank174644 +174645 POINT(40.33355805547882 73.52134313012982) bank174645 +174646 POINT(40.370329847183314 73.94087231077233) bank174646 +174647 POINT(40.539200044267055 74.94845123101929) bank174647 +174648 POINT(41.68817596428219 73.65828668583771) bank174648 +174649 POINT(40.20909796761671 74.30679013995778) bank174649 +174650 POINT(40.944164610958666 74.88289639590973) bank174650 +174651 POINT(40.57508858661176 73.45249135678053) bank174651 +174652 POINT(41.66453609190014 73.72338903383977) bank174652 +174653 POINT(41.2889309049347 74.5155893732368) bank174653 +174654 POINT(39.98110548226168 74.51994013628772) bank174654 +174655 POINT(39.793865628497706 74.4476888934074) bank174655 +174656 POINT(41.513613204773335 74.38721929588256) bank174656 +174657 POINT(40.06670185823495 74.27228377161211) bank174657 +174658 POINT(40.831225242692106 73.1046108899824) bank174658 +174659 POINT(41.39612660803242 73.26996961039445) bank174659 +174660 POINT(41.36053568902489 73.34451067308122) bank174660 +174661 POINT(40.720177958606044 74.93832448164568) bank174661 +174662 POINT(40.075256232267876 73.32716698693805) bank174662 +174663 POINT(39.82923250742489 74.0303469460831) bank174663 +174664 POINT(40.32261308296358 73.22583318116227) bank174664 +174665 POINT(40.760911015610226 73.08288383519863) bank174665 +174666 POINT(41.46875225687704 73.77108914222592) bank174666 +174667 POINT(41.05723776287043 74.48460056608853) bank174667 +174668 POINT(40.91311165880812 74.3719364996648) bank174668 +174669 POINT(40.75009383533036 73.65867978034882) bank174669 +174670 POINT(40.27865088844414 73.38104685408632) bank174670 +174671 POINT(41.14843229101255 73.69619113952125) bank174671 +174672 POINT(41.122329034975174 73.820536664669) bank174672 +174673 POINT(41.29138437795896 73.5437794909111) bank174673 +174674 POINT(41.042236690488274 73.6281888589833) bank174674 +174675 POINT(40.06456638161271 74.63023934281203) bank174675 +174676 POINT(40.73537886662183 74.81380579138172) bank174676 +174677 POINT(40.31869468549621 74.7515276333376) bank174677 +174678 POINT(40.52979688526928 73.2041716541743) bank174678 +174679 POINT(41.711392715876926 74.88807513312643) bank174679 +174680 POINT(41.27726394794519 74.55166260177886) bank174680 +174681 POINT(41.041958635628305 74.98086151994065) bank174681 +174682 POINT(40.67518129120826 73.22640970901723) bank174682 +174683 POINT(40.41489287657312 73.92363109585477) bank174683 +174684 POINT(39.933827212479635 73.53023770668878) bank174684 +174685 POINT(41.502307223921555 73.25740553686761) bank174685 +174686 POINT(40.68131544702928 74.5658207330248) bank174686 +174687 POINT(41.52247938526999 73.22993548128522) bank174687 +174688 POINT(41.042593037440334 74.96313745667582) bank174688 +174689 POINT(41.34932414121274 74.98412026485268) bank174689 +174690 POINT(41.43946449535959 73.37576347087935) bank174690 +174691 POINT(39.80723261243908 73.84476113249453) bank174691 +174692 POINT(40.412783696116534 74.51933885318705) bank174692 +174693 POINT(41.53149757778522 73.94939118123523) bank174693 +174694 POINT(41.36698070522808 74.77254821607461) bank174694 +174695 POINT(39.81844101901321 73.0781838455074) bank174695 +174696 POINT(39.88799611302621 73.94728888284477) bank174696 +174697 POINT(41.2074427676687 73.53883019601986) bank174697 +174698 POINT(40.7615341452963 74.53555554319412) bank174698 +174699 POINT(40.97562620561279 74.05092596231296) bank174699 +174700 POINT(39.85925506220269 74.48535757340775) bank174700 +174701 POINT(41.31804804278114 73.2006389568905) bank174701 +174702 POINT(39.7819445954926 73.76937489080234) bank174702 +174703 POINT(40.354564224446214 74.87191148519302) bank174703 +174704 POINT(41.376887987275396 74.43254250492342) bank174704 +174705 POINT(41.122621286603994 74.80371844179261) bank174705 +174706 POINT(40.72195752740297 73.66894837052877) bank174706 +174707 POINT(40.97396466512487 73.54960560805088) bank174707 +174708 POINT(41.547195170706736 73.70752229399825) bank174708 +174709 POINT(40.9844814076264 74.86321623160514) bank174709 +174710 POINT(39.946135059323105 73.38904827617414) bank174710 +174711 POINT(40.7400629395221 73.2440404405343) bank174711 +174712 POINT(41.352726057136294 74.5003720331412) bank174712 +174713 POINT(40.19055604732261 73.52202105138602) bank174713 +174714 POINT(41.16445870756606 73.10798903739979) bank174714 +174715 POINT(41.49731611285434 74.17180306268054) bank174715 +174716 POINT(41.11944361606316 74.5703003290367) bank174716 +174717 POINT(39.901113204846375 74.46082617579951) bank174717 +174718 POINT(40.46153490556442 74.08547690351102) bank174718 +174719 POINT(40.89624180011528 73.64974988933642) bank174719 +174720 POINT(40.61514975582782 74.99764462451184) bank174720 +174721 POINT(40.21504059782144 74.23452263630904) bank174721 +174722 POINT(40.50105058275258 73.16009853452168) bank174722 +174723 POINT(41.558788457994304 73.73645253926244) bank174723 +174724 POINT(41.534805484314546 73.28522172927602) bank174724 +174725 POINT(40.246698710937004 73.83261396124406) bank174725 +174726 POINT(40.38130795942954 74.74137161073907) bank174726 +174727 POINT(41.64390010929099 74.78850887074604) bank174727 +174728 POINT(41.54157465614408 73.86334005208306) bank174728 +174729 POINT(40.652153366079396 74.98057494214689) bank174729 +174730 POINT(40.57298647065356 74.68536338093276) bank174730 +174731 POINT(41.18783787163815 74.7662728568643) bank174731 +174732 POINT(41.40854287161948 73.48559441999686) bank174732 +174733 POINT(41.34569081075985 73.36427289637288) bank174733 +174734 POINT(40.63561041205704 73.51270344356813) bank174734 +174735 POINT(39.87580983739265 73.76089726108454) bank174735 +174736 POINT(41.177335737854904 73.24340863860698) bank174736 +174737 POINT(40.28426372416502 73.21643239822608) bank174737 +174738 POINT(40.10936204840134 74.91689035550235) bank174738 +174739 POINT(40.25852570147405 74.37488156937191) bank174739 +174740 POINT(39.93802317091004 73.0372044956364) bank174740 +174741 POINT(40.00131365787473 73.19583585297295) bank174741 +174742 POINT(40.355861334051035 73.01263057572896) bank174742 +174743 POINT(41.36729040880797 74.57531608414449) bank174743 +174744 POINT(40.3025541796461 74.48169849598308) bank174744 +174745 POINT(40.21565696075599 74.04886380594387) bank174745 +174746 POINT(40.686397568821775 74.07572882109204) bank174746 +174747 POINT(41.06239814897838 74.65277380450243) bank174747 +174748 POINT(40.15682156742799 74.86900097540435) bank174748 +174749 POINT(39.73830935409612 73.54152444758309) bank174749 +174750 POINT(40.025794734894944 73.02180998520163) bank174750 +174751 POINT(39.71968555479503 73.89984922306812) bank174751 +174752 POINT(39.958511425875805 73.72327689997627) bank174752 +174753 POINT(40.91517743826288 74.8030956607074) bank174753 +174754 POINT(41.5839288061185 74.94559680638162) bank174754 +174755 POINT(41.0087617436809 74.86935946918533) bank174755 +174756 POINT(41.69828616413958 73.98786542483542) bank174756 +174757 POINT(41.505657175374886 73.4664219169958) bank174757 +174758 POINT(40.602900983867386 74.5285028473496) bank174758 +174759 POINT(41.248774499314656 74.0083237456674) bank174759 +174760 POINT(40.09559099839299 74.36596071623246) bank174760 +174761 POINT(39.98854340839189 73.01669721025476) bank174761 +174762 POINT(39.74686471386052 73.51121508728927) bank174762 +174763 POINT(39.79580409538873 74.34986035430425) bank174763 +174764 POINT(40.212434343315735 74.49402286716311) bank174764 +174765 POINT(41.66887765498086 74.2225431260149) bank174765 +174766 POINT(40.52767501014878 73.679063189264) bank174766 +174767 POINT(40.317230654726984 74.01154791505095) bank174767 +174768 POINT(40.58883895049314 74.16848699295677) bank174768 +174769 POINT(40.626315176990516 73.53747623658089) bank174769 +174770 POINT(41.07897762645144 74.28433618093382) bank174770 +174771 POINT(40.46089852181707 74.349437520052) bank174771 +174772 POINT(41.63939878920043 73.78855530549515) bank174772 +174773 POINT(40.01826449333847 74.66971623868513) bank174773 +174774 POINT(39.80744575419922 73.5897249223291) bank174774 +174775 POINT(41.66536468548429 73.35376371396724) bank174775 +174776 POINT(40.564358454179335 74.88329159518892) bank174776 +174777 POINT(40.794119028669535 74.96324837399885) bank174777 +174778 POINT(41.46040382309694 74.28052262672769) bank174778 +174779 POINT(39.91087742685649 73.45523722730576) bank174779 +174780 POINT(40.888726884410744 74.29829731231992) bank174780 +174781 POINT(41.60334692468241 73.1485707225782) bank174781 +174782 POINT(40.866630779484424 74.7130701381928) bank174782 +174783 POINT(40.533883624388565 73.09200165485524) bank174783 +174784 POINT(40.21322216026509 73.7514393675713) bank174784 +174785 POINT(39.88873197392656 74.63722103318374) bank174785 +174786 POINT(39.888897117192904 73.88815325588037) bank174786 +174787 POINT(40.47588536605872 73.02139631104204) bank174787 +174788 POINT(41.27013265106887 74.10856639044026) bank174788 +174789 POINT(40.500247933745236 74.48449180174367) bank174789 +174790 POINT(40.79486748769658 73.86001474461979) bank174790 +174791 POINT(39.92655212917097 73.89725372062075) bank174791 +174792 POINT(41.67065212722504 73.98343086535408) bank174792 +174793 POINT(40.820468534907114 74.90375740977856) bank174793 +174794 POINT(41.56382348400594 74.16715064643383) bank174794 +174795 POINT(39.83311561116159 74.39375824061513) bank174795 +174796 POINT(41.129262156391256 74.86930952534512) bank174796 +174797 POINT(41.28362695415327 73.05510558817831) bank174797 +174798 POINT(41.41239564353062 74.46323166474771) bank174798 +174799 POINT(40.89970738225631 74.82068363537985) bank174799 +174800 POINT(40.531623762521285 74.78098879820513) bank174800 +174801 POINT(40.6657030448801 73.74217096713772) bank174801 +174802 POINT(41.189815636142455 73.55696453863288) bank174802 +174803 POINT(41.21204704892913 74.36480881189443) bank174803 +174804 POINT(40.0952556456534 73.5848016958557) bank174804 +174805 POINT(40.95458483355711 73.95318472470761) bank174805 +174806 POINT(41.670815483034595 73.06181167364676) bank174806 +174807 POINT(41.5333370679374 74.7346721491768) bank174807 +174808 POINT(40.608794985908446 73.84363937307705) bank174808 +174809 POINT(40.734576051772976 73.92766164249325) bank174809 +174810 POINT(40.19447641551582 74.97426659103151) bank174810 +174811 POINT(40.20439689813088 73.93923905521724) bank174811 +174812 POINT(40.24584271953179 73.47397830080504) bank174812 +174813 POINT(40.5263007018391 74.72622705847314) bank174813 +174814 POINT(39.72913088575389 74.01424185971543) bank174814 +174815 POINT(39.72506343461141 73.37311059920839) bank174815 +174816 POINT(40.59795021486738 74.65993858211006) bank174816 +174817 POINT(39.88712373022884 74.05664961962852) bank174817 +174818 POINT(40.92752635142002 73.61750087636723) bank174818 +174819 POINT(40.33937715916026 74.12999421874221) bank174819 +174820 POINT(39.971308274877316 73.51822323333083) bank174820 +174821 POINT(41.0801804325129 73.71257317050035) bank174821 +174822 POINT(40.086424752622065 73.13117311026193) bank174822 +174823 POINT(40.66180651676587 74.52584769718715) bank174823 +174824 POINT(40.48498834347701 73.51761637720432) bank174824 +174825 POINT(41.482271734937505 73.10277005529731) bank174825 +174826 POINT(40.823077999156766 74.18104188693879) bank174826 +174827 POINT(40.89928557457611 73.92727191968108) bank174827 +174828 POINT(40.453067729550256 74.40734318896413) bank174828 +174829 POINT(40.093110233326506 73.3831915195445) bank174829 +174830 POINT(39.72268084623265 73.86590413743765) bank174830 +174831 POINT(41.087881741626475 73.26488878555409) bank174831 +174832 POINT(40.80948620219099 74.09157699105857) bank174832 +174833 POINT(41.473408546280844 73.45393307799405) bank174833 +174834 POINT(40.25348863530738 74.91017440988763) bank174834 +174835 POINT(41.012776412603046 73.15586718336891) bank174835 +174836 POINT(40.570089395675126 74.22696670168074) bank174836 +174837 POINT(40.25689296556579 73.93029711168361) bank174837 +174838 POINT(41.20619124501092 74.7742933222963) bank174838 +174839 POINT(40.6572998772783 74.21060135127344) bank174839 +174840 POINT(40.431446349914076 73.34657513961176) bank174840 +174841 POINT(41.27043246459344 74.53533395658351) bank174841 +174842 POINT(41.056994280367505 73.71489891261645) bank174842 +174843 POINT(40.653112212026116 73.54951411302167) bank174843 +174844 POINT(40.07669467326177 73.9510419946449) bank174844 +174845 POINT(41.55418654541683 73.98090530405442) bank174845 +174846 POINT(40.374947246684165 74.97706040257616) bank174846 +174847 POINT(41.634581828712236 73.34901255484674) bank174847 +174848 POINT(40.68933227461119 74.94671503820119) bank174848 +174849 POINT(41.04898797661873 73.00677159797262) bank174849 +174850 POINT(40.02860012271465 73.4805430711202) bank174850 +174851 POINT(41.558070596955865 73.3252945682963) bank174851 +174852 POINT(40.98182085182443 73.51744819141175) bank174852 +174853 POINT(40.62147257178165 73.97722465226235) bank174853 +174854 POINT(41.30797316860468 74.45559770722328) bank174854 +174855 POINT(40.956032525301495 73.03927569300173) bank174855 +174856 POINT(40.52172404718205 74.83318451283088) bank174856 +174857 POINT(40.951548064431016 73.74849162154514) bank174857 +174858 POINT(41.521793300104065 74.03469544946869) bank174858 +174859 POINT(40.15314638350479 74.66455011908499) bank174859 +174860 POINT(41.06025467954913 74.13277715608228) bank174860 +174861 POINT(40.08999532649817 73.4254549469717) bank174861 +174862 POINT(40.90306249287901 73.46799451262032) bank174862 +174863 POINT(41.22597587011955 74.06794125907949) bank174863 +174864 POINT(40.04878073974162 73.30346634375346) bank174864 +174865 POINT(40.30047849904077 73.07932463772494) bank174865 +174866 POINT(40.27536478148598 74.02612469745611) bank174866 +174867 POINT(41.05660232446872 73.66457548036527) bank174867 +174868 POINT(40.15874715672867 74.98648336713083) bank174868 +174869 POINT(40.019710404915216 73.6956761577581) bank174869 +174870 POINT(41.11383153008971 73.05441527022057) bank174870 +174871 POINT(40.69147835395371 73.23617682397665) bank174871 +174872 POINT(40.70686233936639 74.20590431862408) bank174872 +174873 POINT(41.52516954134753 73.67797893759757) bank174873 +174874 POINT(39.72837579461526 74.05873435778452) bank174874 +174875 POINT(39.80282442165118 74.15584387981767) bank174875 +174876 POINT(41.58381718523044 73.19417502444354) bank174876 +174877 POINT(41.61785887459287 73.48868974845385) bank174877 +174878 POINT(40.40562816815398 73.70292550182337) bank174878 +174879 POINT(39.828006194658066 73.61168005830893) bank174879 +174880 POINT(39.842469670073356 74.8137909685333) bank174880 +174881 POINT(40.79881412883576 73.69058710897588) bank174881 +174882 POINT(40.341721636506215 73.70150818444134) bank174882 +174883 POINT(39.826430017102915 74.1604773228979) bank174883 +174884 POINT(40.99485010614198 73.45396169619059) bank174884 +174885 POINT(40.01088670036376 74.08822004641439) bank174885 +174886 POINT(41.68552583906789 74.72590243583913) bank174886 +174887 POINT(40.65849686703627 73.9658740990229) bank174887 +174888 POINT(39.80932762315792 74.19299959253269) bank174888 +174889 POINT(41.29890205895396 73.62982285689215) bank174889 +174890 POINT(39.93285345416143 74.10552521029017) bank174890 +174891 POINT(41.31332650429074 73.98167384870888) bank174891 +174892 POINT(39.88003885546616 74.05798431261557) bank174892 +174893 POINT(41.61543897042241 74.6742131540278) bank174893 +174894 POINT(40.79654029974471 74.1980345845094) bank174894 +174895 POINT(40.71784521438934 73.59786506455652) bank174895 +174896 POINT(40.39175894349676 74.92017854609045) bank174896 +174897 POINT(40.47757604519219 73.28651293382447) bank174897 +174898 POINT(40.577983324866665 74.77841738474118) bank174898 +174899 POINT(41.30942462543951 74.43346257383178) bank174899 +174900 POINT(39.73926007496823 74.30427814752767) bank174900 +174901 POINT(41.02669774427419 74.16037205982911) bank174901 +174902 POINT(40.49747735994 74.62266669885973) bank174902 +174903 POINT(41.05548188454582 74.89801357310955) bank174903 +174904 POINT(40.18267801428832 73.25004765924203) bank174904 +174905 POINT(41.10296371831875 73.94679868794373) bank174905 +174906 POINT(41.06455407560317 73.33564454298215) bank174906 +174907 POINT(40.36297648602518 74.03460325508343) bank174907 +174908 POINT(40.77053141108887 74.77746511308263) bank174908 +174909 POINT(41.334730798900544 74.82375975954805) bank174909 +174910 POINT(40.08815781795188 74.82866441414356) bank174910 +174911 POINT(41.52793857530574 74.2787360037973) bank174911 +174912 POINT(40.993667852604354 74.118470816311) bank174912 +174913 POINT(40.245360455396074 73.06912734875318) bank174913 +174914 POINT(40.38135567061572 73.97883142824185) bank174914 +174915 POINT(41.327583818963035 73.7081548835791) bank174915 +174916 POINT(40.07882841908172 73.71656290879137) bank174916 +174917 POINT(40.66748371502949 74.35854419982272) bank174917 +174918 POINT(40.03361223467542 74.44483769654539) bank174918 +174919 POINT(40.73390343858038 73.80520150539688) bank174919 +174920 POINT(41.01547776699585 74.07118927771026) bank174920 +174921 POINT(40.35747526769113 73.98800863063775) bank174921 +174922 POINT(41.456927935204206 73.86206607730546) bank174922 +174923 POINT(40.153944301609684 73.23940957612301) bank174923 +174924 POINT(41.13806557266387 74.73865491283672) bank174924 +174925 POINT(40.108282050469406 74.58018608560877) bank174925 +174926 POINT(40.61516280120976 74.04742296476512) bank174926 +174927 POINT(40.62706887265367 74.57922152454154) bank174927 +174928 POINT(39.751115960636966 74.35697157529954) bank174928 +174929 POINT(40.68606442735172 73.16625908411808) bank174929 +174930 POINT(40.56262739698019 73.82545168858515) bank174930 +174931 POINT(40.505527084613306 74.233003795915) bank174931 +174932 POINT(40.9004524782861 73.30519669812318) bank174932 +174933 POINT(39.953749670622756 74.67968709075875) bank174933 +174934 POINT(40.66899993528547 73.584784556027) bank174934 +174935 POINT(40.8403313374498 73.72058972395641) bank174935 +174936 POINT(40.78372981827419 73.50345244564608) bank174936 +174937 POINT(41.34282070291302 74.81181469311484) bank174937 +174938 POINT(40.63005848301369 74.20624530829211) bank174938 +174939 POINT(40.97026507907719 73.36250186746916) bank174939 +174940 POINT(40.742915704885256 73.92186216472383) bank174940 +174941 POINT(40.14621456506087 74.40526471531672) bank174941 +174942 POINT(39.93325598917035 73.50571643214873) bank174942 +174943 POINT(41.0185630107683 74.20321043383065) bank174943 +174944 POINT(40.32984683075084 74.04211098286181) bank174944 +174945 POINT(41.115638704888994 74.59144284687804) bank174945 +174946 POINT(41.12500102777036 74.77918832352618) bank174946 +174947 POINT(41.69579677812353 73.39179450999501) bank174947 +174948 POINT(39.84534230057161 73.11375172309488) bank174948 +174949 POINT(40.88677181135145 73.33230368117931) bank174949 +174950 POINT(40.359705127628 74.04653625936398) bank174950 +174951 POINT(40.84156194342261 73.36890595152327) bank174951 +174952 POINT(39.8208432163475 73.56776314011846) bank174952 +174953 POINT(40.101439563283854 73.96002196549085) bank174953 +174954 POINT(40.086301934138135 73.30098940125549) bank174954 +174955 POINT(40.81596158130622 74.92363224319224) bank174955 +174956 POINT(40.974284300827804 74.8416464789766) bank174956 +174957 POINT(39.91139141315564 74.74704672820828) bank174957 +174958 POINT(41.14034271575664 74.94335582331124) bank174958 +174959 POINT(41.580791119945545 73.85964191620764) bank174959 +174960 POINT(40.459032432820244 73.32434101739692) bank174960 +174961 POINT(41.1557546012541 73.93674165571333) bank174961 +174962 POINT(40.408645635471 74.39592749477292) bank174962 +174963 POINT(41.538913687944714 73.75612610023376) bank174963 +174964 POINT(41.067228514080966 73.05777345031888) bank174964 +174965 POINT(40.054166334194505 73.21891564884109) bank174965 +174966 POINT(41.18358343064999 74.88509232537459) bank174966 +174967 POINT(39.852189275973714 73.53432809582435) bank174967 +174968 POINT(40.39001540947939 74.25899388610124) bank174968 +174969 POINT(41.366630641423384 74.49285481842016) bank174969 +174970 POINT(40.892119172994825 74.40815567373927) bank174970 +174971 POINT(40.7726585908716 73.59584093625156) bank174971 +174972 POINT(39.97701760356017 73.5128644567837) bank174972 +174973 POINT(41.60438165070208 73.15024020623633) bank174973 +174974 POINT(41.12519445309037 73.04258732710603) bank174974 +174975 POINT(41.441381223754014 74.76785666217661) bank174975 +174976 POINT(41.111941450298836 73.43024760574558) bank174976 +174977 POINT(40.210626966322664 73.96389825199103) bank174977 +174978 POINT(41.62633934983083 74.60474924576265) bank174978 +174979 POINT(41.403991145413706 73.10298109132057) bank174979 +174980 POINT(41.64062290586217 73.64245209847984) bank174980 +174981 POINT(40.622220818604646 74.77400644164555) bank174981 +174982 POINT(40.08987277775307 73.35475870889837) bank174982 +174983 POINT(41.39298977464864 73.17203569656961) bank174983 +174984 POINT(40.85763555352333 74.79168014577742) bank174984 +174985 POINT(40.90309937382178 74.78305199069004) bank174985 +174986 POINT(40.28298038154781 74.65151737200556) bank174986 +174987 POINT(40.87201086137648 73.1397766980704) bank174987 +174988 POINT(40.83639541102936 73.31063656962066) bank174988 +174989 POINT(41.449070359600995 74.63582712505682) bank174989 +174990 POINT(41.16878103787591 73.67926810359111) bank174990 +174991 POINT(41.494089206150036 74.86667597070462) bank174991 +174992 POINT(40.948775254568716 73.63749167270309) bank174992 +174993 POINT(40.130127270663394 74.36427353838155) bank174993 +174994 POINT(41.257820032196385 74.51849837365215) bank174994 +174995 POINT(40.73517804217371 73.6132144944724) bank174995 +174996 POINT(41.30306421112653 73.29548988394285) bank174996 +174997 POINT(40.956130452237865 74.81901684190089) bank174997 +174998 POINT(40.89571227302124 73.13452067402852) bank174998 +174999 POINT(40.591665416815665 74.46390524441026) bank174999 +175000 POINT(40.61230496488756 74.84510558617315) bank175000 +175001 POINT(40.19055055919783 74.57709459225458) bank175001 +175002 POINT(41.65239964266858 73.30058236573605) bank175002 +175003 POINT(41.26923291667855 74.14655153738462) bank175003 +175004 POINT(41.09109376292878 74.79191933139307) bank175004 +175005 POINT(40.31884566385979 74.83757071178496) bank175005 +175006 POINT(41.08845306238671 74.66455562924732) bank175006 +175007 POINT(39.856558005365905 74.07196039120564) bank175007 +175008 POINT(41.15465749690833 73.48076679991098) bank175008 +175009 POINT(39.95839162667107 75.00586813251127) bank175009 +175010 POINT(41.61625991237843 73.48988578851443) bank175010 +175011 POINT(40.30530166852364 73.17437554016472) bank175011 +175012 POINT(41.29967491410666 74.94890944940964) bank175012 +175013 POINT(40.46195610267968 74.53563421365526) bank175013 +175014 POINT(39.97623041322333 74.9439129586747) bank175014 +175015 POINT(39.848313604728304 73.66088335091274) bank175015 +175016 POINT(40.99721160717431 73.3122010441886) bank175016 +175017 POINT(41.58962304210351 74.38779791358645) bank175017 +175018 POINT(40.50113494558098 73.44598397401879) bank175018 +175019 POINT(40.99486969022843 74.48347103400972) bank175019 +175020 POINT(40.22887893349231 74.69802709947862) bank175020 +175021 POINT(40.070615602520405 73.26200342309546) bank175021 +175022 POINT(39.90875376557763 73.36046385537759) bank175022 +175023 POINT(40.680708448210204 73.12388690217017) bank175023 +175024 POINT(41.64846557256394 73.1846133774489) bank175024 +175025 POINT(41.13002612730278 74.98218877888934) bank175025 +175026 POINT(40.53222622005728 74.29306408457985) bank175026 +175027 POINT(41.28469085691562 74.51480321548907) bank175027 +175028 POINT(40.48904106821936 74.4902920222652) bank175028 +175029 POINT(40.109677229662026 74.6829217153415) bank175029 +175030 POINT(39.98084943669694 74.27175398798356) bank175030 +175031 POINT(40.53815166194195 73.58039164648274) bank175031 +175032 POINT(40.262617528296616 73.83842719138539) bank175032 +175033 POINT(40.300260095462775 74.48510121926657) bank175033 +175034 POINT(40.24039727345413 74.43067773053762) bank175034 +175035 POINT(39.84088856684594 74.25328636561242) bank175035 +175036 POINT(40.31651295470303 73.38330504564472) bank175036 +175037 POINT(41.66838907985591 73.29150099163998) bank175037 +175038 POINT(40.261327213419776 73.17007481320756) bank175038 +175039 POINT(40.753430639382614 73.9740206492136) bank175039 +175040 POINT(40.74554753538597 74.60902024264828) bank175040 +175041 POINT(40.08294072369552 74.22107285716926) bank175041 +175042 POINT(40.849493590848496 73.79602280212336) bank175042 +175043 POINT(40.472333523249844 75.00213297836142) bank175043 +175044 POINT(41.230575415584354 73.0734254473066) bank175044 +175045 POINT(40.00585975359832 73.98327634714445) bank175045 +175046 POINT(40.094093463897316 74.81675995881031) bank175046 +175047 POINT(40.55313358966578 74.45321383316997) bank175047 +175048 POINT(40.73540525804506 74.94380789595606) bank175048 +175049 POINT(41.68008772107278 74.19587616924031) bank175049 +175050 POINT(40.745389760207715 74.55702670702142) bank175050 +175051 POINT(41.62093689836645 74.59541732272065) bank175051 +175052 POINT(41.71199921424904 73.48012536480417) bank175052 +175053 POINT(41.69743717924931 74.1134278846035) bank175053 +175054 POINT(40.49557993962311 73.31246479914863) bank175054 +175055 POINT(41.09937631404633 74.52066499077932) bank175055 +175056 POINT(40.65021786855893 74.12894031904085) bank175056 +175057 POINT(41.57287291330404 74.45134722422881) bank175057 +175058 POINT(41.53635145359108 74.11525546367344) bank175058 +175059 POINT(40.11388933321449 74.02309544389536) bank175059 +175060 POINT(41.56388590153536 74.62287994098868) bank175060 +175061 POINT(41.56028862898697 73.77861002305573) bank175061 +175062 POINT(41.18475093929445 74.24184826534587) bank175062 +175063 POINT(40.27996891757277 73.42103931969484) bank175063 +175064 POINT(40.00460208218871 74.07270074192641) bank175064 +175065 POINT(40.50653049472774 73.30082341549785) bank175065 +175066 POINT(40.58111648968526 73.46751489431081) bank175066 +175067 POINT(41.664977952230146 73.30686277368656) bank175067 +175068 POINT(40.352999156970114 73.41302109125225) bank175068 +175069 POINT(40.69509342065651 73.1741842166433) bank175069 +175070 POINT(41.068769903542965 73.78594770586457) bank175070 +175071 POINT(41.66757465524947 74.02590324589107) bank175071 +175072 POINT(41.6379815822505 74.80565853091827) bank175072 +175073 POINT(39.960873514531976 73.02939542959582) bank175073 +175074 POINT(40.31797892051766 74.23808026519428) bank175074 +175075 POINT(40.20997663077494 73.73089118014164) bank175075 +175076 POINT(39.87387137211442 73.17557118056969) bank175076 +175077 POINT(40.27047407242443 74.00782715414708) bank175077 +175078 POINT(40.62612129117493 73.38113898541702) bank175078 +175079 POINT(41.61895897484194 73.37406669525042) bank175079 +175080 POINT(39.75220246285593 73.90675842752823) bank175080 +175081 POINT(40.677576470861155 73.8036562405546) bank175081 +175082 POINT(41.332074694432166 74.01928358351049) bank175082 +175083 POINT(40.22034667550429 73.37985382954459) bank175083 +175084 POINT(40.03979695970616 74.89723967444519) bank175084 +175085 POINT(39.870087926667466 73.88931230524909) bank175085 +175086 POINT(40.575728680964765 73.72729224662032) bank175086 +175087 POINT(40.98959218858199 74.88876541880984) bank175087 +175088 POINT(41.217248737725285 74.05047257105029) bank175088 +175089 POINT(39.81053182863164 73.2685256639211) bank175089 +175090 POINT(41.10132320883145 73.51493397671486) bank175090 +175091 POINT(41.65429645011555 74.09465842333604) bank175091 +175092 POINT(41.05155009120732 73.91317692651947) bank175092 +175093 POINT(41.62547045370067 74.22547508730085) bank175093 +175094 POINT(39.9495656486306 74.51668475435481) bank175094 +175095 POINT(40.277199313657086 74.29931928781015) bank175095 +175096 POINT(40.14406981141629 73.1262055156786) bank175096 +175097 POINT(40.84201627778585 73.5228029855073) bank175097 +175098 POINT(40.737576921241924 73.974189406866) bank175098 +175099 POINT(40.37557788186651 74.12778489532704) bank175099 +175100 POINT(40.953972449221425 73.24263153479451) bank175100 +175101 POINT(41.64385071955742 74.67923997052247) bank175101 +175102 POINT(40.58758056415123 74.14964700022153) bank175102 +175103 POINT(40.891839492342825 73.1293089851442) bank175103 +175104 POINT(40.496526460346466 73.76624360199973) bank175104 +175105 POINT(40.70490833253839 74.68262526341299) bank175105 +175106 POINT(40.99084247241663 73.35940339951452) bank175106 +175107 POINT(40.05657335754655 73.47991365636157) bank175107 +175108 POINT(40.31328122759295 74.04185802635592) bank175108 +175109 POINT(41.7066224291744 73.34693969974124) bank175109 +175110 POINT(40.39923026609168 74.23572207348803) bank175110 +175111 POINT(39.80996688222703 73.8605464644037) bank175111 +175112 POINT(40.90695481313005 73.4988672119375) bank175112 +175113 POINT(40.45081264319417 73.82449969929395) bank175113 +175114 POINT(40.80265352310542 73.12315672300674) bank175114 +175115 POINT(41.32246288112678 73.41236862894793) bank175115 +175116 POINT(40.33108978428925 73.18522206451046) bank175116 +175117 POINT(40.435134669175525 74.10466178240709) bank175117 +175118 POINT(40.39616634323242 73.61562631495309) bank175118 +175119 POINT(40.5988876574911 73.1022814062639) bank175119 +175120 POINT(41.10134620417889 73.00996098214193) bank175120 +175121 POINT(41.50843643265386 73.12955353649576) bank175121 +175122 POINT(41.01596248252779 73.56193037240452) bank175122 +175123 POINT(40.68959331117243 74.7496596591201) bank175123 +175124 POINT(41.45062981142052 74.99855613620784) bank175124 +175125 POINT(40.84272495184892 74.62090454532698) bank175125 +175126 POINT(41.20193673590486 73.93644876975077) bank175126 +175127 POINT(40.76896790609126 74.71686494153278) bank175127 +175128 POINT(39.81739133546236 74.07458809816183) bank175128 +175129 POINT(41.11566668194494 73.3730947775807) bank175129 +175130 POINT(41.455120931135895 74.40011487523645) bank175130 +175131 POINT(39.76126771887316 74.86165042995574) bank175131 +175132 POINT(40.618089758656275 73.82698361530294) bank175132 +175133 POINT(40.142947356856354 74.30480869034908) bank175133 +175134 POINT(40.86850719592467 74.77071060235541) bank175134 +175135 POINT(39.97205845781538 73.56542718782941) bank175135 +175136 POINT(40.0591269349402 74.42900469727441) bank175136 +175137 POINT(41.17041771238131 74.40057305720579) bank175137 +175138 POINT(40.62881479250471 74.44702544520858) bank175138 +175139 POINT(41.67702932739067 73.72695392103763) bank175139 +175140 POINT(39.98555798610788 73.04403478291643) bank175140 +175141 POINT(41.569246799673245 74.33219685095065) bank175141 +175142 POINT(40.38155318916516 74.85465142141533) bank175142 +175143 POINT(40.61282422543572 73.58882857327576) bank175143 +175144 POINT(40.049503308515575 74.27498552109085) bank175144 +175145 POINT(40.56389098328636 73.63691984205684) bank175145 +175146 POINT(41.37109167804913 74.16415409081007) bank175146 +175147 POINT(41.323115973632845 73.02435380358442) bank175147 +175148 POINT(41.60409465038907 74.05624610383495) bank175148 +175149 POINT(41.13695895294747 74.56281141871351) bank175149 +175150 POINT(40.814664568834274 74.65245208690332) bank175150 +175151 POINT(41.45268824800558 74.5133448338313) bank175151 +175152 POINT(40.748899381924424 73.91882493727219) bank175152 +175153 POINT(41.338376565257704 73.81597030295703) bank175153 +175154 POINT(40.79974650593052 74.3534194974362) bank175154 +175155 POINT(40.7980778070595 74.65163620128033) bank175155 +175156 POINT(41.386013458405365 74.82328942019853) bank175156 +175157 POINT(39.979577092911995 73.55317635592775) bank175157 +175158 POINT(41.08069510055017 74.95536108565311) bank175158 +175159 POINT(40.3794437901314 73.71506581298684) bank175159 +175160 POINT(41.117833570559206 74.56121157747958) bank175160 +175161 POINT(41.64270257341419 73.8962235027037) bank175161 +175162 POINT(41.23541755470326 74.80250096729344) bank175162 +175163 POINT(41.121480624148106 74.67934427307313) bank175163 +175164 POINT(41.25985942484722 74.38996011418361) bank175164 +175165 POINT(41.51325507611567 74.64851341743496) bank175165 +175166 POINT(40.30282460282894 74.97417065036763) bank175166 +175167 POINT(41.65429572227533 73.16052582003653) bank175167 +175168 POINT(41.64002642473039 73.30745098766644) bank175168 +175169 POINT(40.34597878743614 73.96480972903792) bank175169 +175170 POINT(39.951851791324906 73.01755759328681) bank175170 +175171 POINT(41.19268550261763 74.39696680564586) bank175171 +175172 POINT(41.03994271254265 74.21567729484123) bank175172 +175173 POINT(40.52724931565406 74.29917029501452) bank175173 +175174 POINT(41.29417730144376 74.07810165597046) bank175174 +175175 POINT(41.47568350379785 74.4300561710636) bank175175 +175176 POINT(40.97110219296558 73.52489573561672) bank175176 +175177 POINT(41.288700792790976 74.69687018739526) bank175177 +175178 POINT(40.39315210166943 73.53255052242216) bank175178 +175179 POINT(40.88168337184516 74.91721381817074) bank175179 +175180 POINT(41.61872270450208 73.22065098428473) bank175180 +175181 POINT(41.3515972580148 74.2115874164736) bank175181 +175182 POINT(41.23595970561789 74.11838823355052) bank175182 +175183 POINT(39.769456664483684 74.9830422106282) bank175183 +175184 POINT(40.499873137149514 73.40514902757482) bank175184 +175185 POINT(40.8984061866173 74.19702799803254) bank175185 +175186 POINT(39.92294248685064 73.06865200925925) bank175186 +175187 POINT(39.820428801635344 73.55743065889432) bank175187 +175188 POINT(40.581494491862756 73.03864997313846) bank175188 +175189 POINT(41.251136947645385 73.29472267466952) bank175189 +175190 POINT(41.6952469999329 73.86010657332444) bank175190 +175191 POINT(41.34834303834575 73.80822162681469) bank175191 +175192 POINT(40.80079143217282 74.7202164753825) bank175192 +175193 POINT(41.090000587468374 74.50478662641615) bank175193 +175194 POINT(39.93313828346082 74.93786159602625) bank175194 +175195 POINT(41.673060142459484 73.85235303784899) bank175195 +175196 POINT(40.49798080357268 73.24969512299293) bank175196 +175197 POINT(39.86653061033894 73.83448730855024) bank175197 +175198 POINT(41.626514884279146 74.55155817813738) bank175198 +175199 POINT(41.69983810293687 73.18976794312603) bank175199 +175200 POINT(41.28286133200499 74.47729016140742) bank175200 +175201 POINT(40.377811669126864 73.97800233086392) bank175201 +175202 POINT(41.01430652753343 73.96640316098777) bank175202 +175203 POINT(40.568812814727124 73.81143136014957) bank175203 +175204 POINT(40.395780521344335 74.10915204765327) bank175204 +175205 POINT(41.20354285250928 74.01778431608749) bank175205 +175206 POINT(40.90039361615219 73.22094527937863) bank175206 +175207 POINT(41.69990747324366 73.8374548245805) bank175207 +175208 POINT(39.81582962642716 74.18270677112018) bank175208 +175209 POINT(41.02522718139719 74.96079975206091) bank175209 +175210 POINT(40.94013067117861 74.03061927835545) bank175210 +175211 POINT(41.52514480791358 73.86495714434193) bank175211 +175212 POINT(40.22779324443053 74.54245987292816) bank175212 +175213 POINT(39.7789705385305 74.09592386980938) bank175213 +175214 POINT(39.7882780486393 73.23767992397113) bank175214 +175215 POINT(40.1373352826548 73.74765160191116) bank175215 +175216 POINT(41.253201927554954 73.6164751977335) bank175216 +175217 POINT(40.14357272209476 73.59356055092266) bank175217 +175218 POINT(41.66113357586879 73.1801355032214) bank175218 +175219 POINT(40.473338476168905 73.45446101270278) bank175219 +175220 POINT(40.13434234422696 74.60388303784727) bank175220 +175221 POINT(40.82071229607015 73.18247531025285) bank175221 +175222 POINT(41.021828498932805 73.07344986684772) bank175222 +175223 POINT(41.659893321840904 73.9370955286393) bank175223 +175224 POINT(41.02676067803173 74.11548853919668) bank175224 +175225 POINT(39.82634661082616 74.01382872727723) bank175225 +175226 POINT(40.28830551208344 73.05427093023887) bank175226 +175227 POINT(40.69167742477052 73.0885943208956) bank175227 +175228 POINT(41.437193619247914 73.84731230504943) bank175228 +175229 POINT(41.44757502524738 74.95121315953698) bank175229 +175230 POINT(39.76260661974342 73.85546525385975) bank175230 +175231 POINT(40.680537542041264 73.26703307689017) bank175231 +175232 POINT(40.39614855786535 74.3532314686217) bank175232 +175233 POINT(41.50574038119332 74.16789870958085) bank175233 +175234 POINT(39.71547644578191 73.82291880292955) bank175234 +175235 POINT(41.37799082506086 74.26840137872506) bank175235 +175236 POINT(41.000051162237455 73.82587880337057) bank175236 +175237 POINT(40.38387065898557 74.100744214403) bank175237 +175238 POINT(41.30796367410655 74.73664760663344) bank175238 +175239 POINT(39.95389233455953 74.76399285825389) bank175239 +175240 POINT(41.28580545451102 73.57778065688994) bank175240 +175241 POINT(41.146823253110064 74.82817490195998) bank175241 +175242 POINT(41.06522708275655 74.95597128012606) bank175242 +175243 POINT(40.94999788221535 73.23680460562005) bank175243 +175244 POINT(40.10542097954488 74.98482923720306) bank175244 +175245 POINT(39.84417189830171 73.28624761837304) bank175245 +175246 POINT(40.559304878890806 74.19179402967856) bank175246 +175247 POINT(40.821501531265554 73.15187627015884) bank175247 +175248 POINT(41.590500323681624 74.77789376927682) bank175248 +175249 POINT(41.33249313984918 73.53995101496663) bank175249 +175250 POINT(39.97787889026627 73.96330574592557) bank175250 +175251 POINT(39.89322935005728 73.0074905913079) bank175251 +175252 POINT(41.29801971105131 73.4328847070177) bank175252 +175253 POINT(40.8508242089275 74.67309651200249) bank175253 +175254 POINT(40.8198364978797 73.23823002690598) bank175254 +175255 POINT(40.4909805126254 74.5235476177815) bank175255 +175256 POINT(40.731030292722274 73.28259533206486) bank175256 +175257 POINT(40.083780334968864 74.56621636332628) bank175257 +175258 POINT(40.45538581596517 74.54175220230691) bank175258 +175259 POINT(40.366731047765015 74.76804458069563) bank175259 +175260 POINT(41.660416725645234 74.96878820680473) bank175260 +175261 POINT(40.06783149147179 74.01054987143765) bank175261 +175262 POINT(40.19216284125591 73.9026421149543) bank175262 +175263 POINT(41.14123473569839 74.96536520200846) bank175263 +175264 POINT(41.672014567913436 74.03227843066136) bank175264 +175265 POINT(40.04284062517167 74.63996812539567) bank175265 +175266 POINT(41.073693890795134 73.26852191030096) bank175266 +175267 POINT(41.1790488989373 73.73047601165253) bank175267 +175268 POINT(40.14922023738033 74.9789761688239) bank175268 +175269 POINT(41.67148949265891 73.55175663350535) bank175269 +175270 POINT(41.17395534067392 73.65433903060358) bank175270 +175271 POINT(40.57247418008743 73.9156144620333) bank175271 +175272 POINT(40.72066375948314 73.33561183105846) bank175272 +175273 POINT(41.23148948548565 73.27254897231893) bank175273 +175274 POINT(40.70567418039042 74.31316936219318) bank175274 +175275 POINT(40.553339378589214 73.80153758193201) bank175275 +175276 POINT(41.14006214666657 73.87044533504978) bank175276 +175277 POINT(39.98713715239032 73.5592264611159) bank175277 +175278 POINT(39.79129639626443 74.17492829602793) bank175278 +175279 POINT(41.632292106084414 73.6728021689681) bank175279 +175280 POINT(39.86853725043906 74.88190051583132) bank175280 +175281 POINT(40.58958702210449 73.03761901016784) bank175281 +175282 POINT(40.48856664666047 74.9834672588023) bank175282 +175283 POINT(40.424559817618494 74.25872213315898) bank175283 +175284 POINT(41.111837225376576 73.72932245058638) bank175284 +175285 POINT(40.15980145358298 74.05366270538217) bank175285 +175286 POINT(41.432184560690295 73.95215797797307) bank175286 +175287 POINT(41.54510326112883 74.23407454797193) bank175287 +175288 POINT(40.13738905724853 74.72130180426873) bank175288 +175289 POINT(39.75164477089633 73.91247493619261) bank175289 +175290 POINT(41.52608542675244 73.80780463515796) bank175290 +175291 POINT(40.061844527363164 74.57824569012615) bank175291 +175292 POINT(40.88783250114409 74.32997691363296) bank175292 +175293 POINT(39.75768623013761 74.28928381521274) bank175293 +175294 POINT(40.87935306165124 74.587987975613) bank175294 +175295 POINT(41.53991160602461 73.9554308010977) bank175295 +175296 POINT(40.991181338766935 74.65071605293684) bank175296 +175297 POINT(41.33789036301575 73.04988094033796) bank175297 +175298 POINT(40.423052974911734 74.93612182256537) bank175298 +175299 POINT(40.097672651933266 73.21012520962144) bank175299 +175300 POINT(40.32271656042737 73.892376383156) bank175300 +175301 POINT(41.11399773144169 74.31955867504277) bank175301 +175302 POINT(41.19482103323969 74.01472567742957) bank175302 +175303 POINT(40.77071700002433 73.80375837218774) bank175303 +175304 POINT(40.86057503575113 74.72755357205472) bank175304 +175305 POINT(40.6383405472952 73.2347368544667) bank175305 +175306 POINT(40.82516646342819 74.91332148344361) bank175306 +175307 POINT(40.87219844143388 73.33400159716908) bank175307 +175308 POINT(39.92123842285833 73.70467442008128) bank175308 +175309 POINT(40.69534777613958 74.59066475041328) bank175309 +175310 POINT(40.64778991753622 74.29786821130484) bank175310 +175311 POINT(41.07094994989844 73.81462524218705) bank175311 +175312 POINT(41.114514413554886 74.86945497292079) bank175312 +175313 POINT(41.507042070597116 74.30142409614052) bank175313 +175314 POINT(41.421031367897996 74.97716052134369) bank175314 +175315 POINT(40.12972285636708 74.35264852137627) bank175315 +175316 POINT(41.59962657778314 73.13100492167047) bank175316 +175317 POINT(39.946019343256616 74.72478310230984) bank175317 +175318 POINT(41.180211391603656 73.0704221027012) bank175318 +175319 POINT(40.331751003139935 73.70504605352434) bank175319 +175320 POINT(40.09681269211983 74.76111730885434) bank175320 +175321 POINT(41.19964214488424 74.57835921063564) bank175321 +175322 POINT(40.95486476044392 73.52234944724486) bank175322 +175323 POINT(40.670574131704726 74.35646732749092) bank175323 +175324 POINT(40.14987507317481 73.28723851129101) bank175324 +175325 POINT(41.54183172267075 73.76181504356022) bank175325 +175326 POINT(40.075355553293775 73.54315427773058) bank175326 +175327 POINT(41.6297519627344 73.45122028385124) bank175327 +175328 POINT(40.814061769610504 73.62216882061924) bank175328 +175329 POINT(40.353707821025296 73.69742344029568) bank175329 +175330 POINT(39.746469583325265 73.7782457510182) bank175330 +175331 POINT(41.28555048504931 73.06859753586974) bank175331 +175332 POINT(41.149792971845365 74.21352996483323) bank175332 +175333 POINT(40.72947485198703 74.74430955490296) bank175333 +175334 POINT(40.48636988431656 74.88566495799886) bank175334 +175335 POINT(40.87465498598958 73.33317567428799) bank175335 +175336 POINT(41.687872164353635 73.05479962386133) bank175336 +175337 POINT(40.909323811619494 73.34469526480896) bank175337 +175338 POINT(40.13377153516576 74.12921986853807) bank175338 +175339 POINT(40.41001869283037 74.31362408901684) bank175339 +175340 POINT(40.92517190212252 74.18895575377707) bank175340 +175341 POINT(40.96791412055749 73.32470526131647) bank175341 +175342 POINT(40.11977525054334 73.16053803971346) bank175342 +175343 POINT(41.37799315990651 74.24489008477741) bank175343 +175344 POINT(40.797077941665904 74.19261755215753) bank175344 +175345 POINT(39.79424586493098 73.35704203223004) bank175345 +175346 POINT(40.142032519829364 73.46503760799489) bank175346 +175347 POINT(41.00087249727164 73.79014605105814) bank175347 +175348 POINT(40.41132134596569 74.19646133258645) bank175348 +175349 POINT(40.04953038645821 74.90175656164197) bank175349 +175350 POINT(41.530784029676205 74.87892015228286) bank175350 +175351 POINT(41.20957310741118 73.51469569319242) bank175351 +175352 POINT(39.83586488382506 73.79072660961465) bank175352 +175353 POINT(41.55721530614647 73.1724512913945) bank175353 +175354 POINT(41.47226497471266 73.4663768651189) bank175354 +175355 POINT(40.832978452375706 73.13408044236434) bank175355 +175356 POINT(40.28663824131561 74.3975076697912) bank175356 +175357 POINT(40.54756521577113 73.3624876904775) bank175357 +175358 POINT(41.256047651301024 73.7579629889864) bank175358 +175359 POINT(40.776203581909826 73.96468527739005) bank175359 +175360 POINT(39.96724753880031 73.98629619092135) bank175360 +175361 POINT(40.4729303633776 73.95339251416448) bank175361 +175362 POINT(40.62600304458236 73.87927954644233) bank175362 +175363 POINT(40.68383929281314 73.5624012246164) bank175363 +175364 POINT(41.126993504410386 73.09863471939124) bank175364 +175365 POINT(41.38325335818658 73.26705003281455) bank175365 +175366 POINT(40.80077544582156 73.03583546225676) bank175366 +175367 POINT(40.18122796905739 73.80630649843651) bank175367 +175368 POINT(40.678613868918646 74.01819607307498) bank175368 +175369 POINT(41.48654323155145 74.36923145039306) bank175369 +175370 POINT(39.976750775633256 74.74254482375186) bank175370 +175371 POINT(40.24478479491209 73.62865848475282) bank175371 +175372 POINT(40.45126733395299 73.52099169907729) bank175372 +175373 POINT(40.795053404188565 74.09968757872637) bank175373 +175374 POINT(40.542626355161154 74.29635066261167) bank175374 +175375 POINT(40.01949994571967 74.16686090745267) bank175375 +175376 POINT(41.576614518086565 74.79249504508884) bank175376 +175377 POINT(39.91876766749471 73.99086674624259) bank175377 +175378 POINT(41.091765052912656 73.54178046161418) bank175378 +175379 POINT(39.76238606300204 74.20518220001024) bank175379 +175380 POINT(40.998016099510856 73.18392001062259) bank175380 +175381 POINT(41.285877948751015 74.02761344940365) bank175381 +175382 POINT(41.589218830789704 74.39141498221561) bank175382 +175383 POINT(40.62191982064247 74.11787625419721) bank175383 +175384 POINT(41.208754516534924 73.65304836029878) bank175384 +175385 POINT(41.65186196864232 74.86824738593043) bank175385 +175386 POINT(41.246314745741024 74.45356911326043) bank175386 +175387 POINT(40.456440790246276 73.96656867459147) bank175387 +175388 POINT(41.331350189717035 73.67036677141256) bank175388 +175389 POINT(41.43968012802786 73.32626725006996) bank175389 +175390 POINT(41.58276151788458 74.15375362354344) bank175390 +175391 POINT(40.53565134091398 74.45561589718784) bank175391 +175392 POINT(40.6850992531239 73.25631102143893) bank175392 +175393 POINT(40.918573954556415 74.283575982869) bank175393 +175394 POINT(40.04336415598547 74.49857471020596) bank175394 +175395 POINT(40.189381502986066 74.98986005544224) bank175395 +175396 POINT(40.5962330858246 73.30980793557521) bank175396 +175397 POINT(39.81434085219609 74.74306805943425) bank175397 +175398 POINT(40.8683983344317 74.341047273818) bank175398 +175399 POINT(41.58548179059977 73.05320306431253) bank175399 +175400 POINT(40.39946519055919 73.33540865798997) bank175400 +175401 POINT(41.541109311105814 73.54161230105358) bank175401 +175402 POINT(41.16421561076961 73.10450996071043) bank175402 +175403 POINT(40.1820943955516 74.5615945764109) bank175403 +175404 POINT(40.80499152878183 74.42988416784836) bank175404 +175405 POINT(40.03697626609491 73.74686845644791) bank175405 +175406 POINT(41.28798153517288 73.05923983109663) bank175406 +175407 POINT(40.723424070126995 74.20279225074347) bank175407 +175408 POINT(40.95489496270751 73.92731605686227) bank175408 +175409 POINT(41.069417532748574 74.52208426505408) bank175409 +175410 POINT(41.435692130295784 73.56697963497635) bank175410 +175411 POINT(41.17793802899106 74.76852898899052) bank175411 +175412 POINT(41.178516640411765 73.48771194718614) bank175412 +175413 POINT(40.23388481859491 73.35086438204111) bank175413 +175414 POINT(40.54406936872139 74.70275804419774) bank175414 +175415 POINT(39.81054625765196 74.24761082463182) bank175415 +175416 POINT(41.44403432286152 73.05534568684982) bank175416 +175417 POINT(41.3884265659609 74.78775328382291) bank175417 +175418 POINT(41.10510414749725 73.74292839636908) bank175418 +175419 POINT(41.32397148158535 73.0479788482588) bank175419 +175420 POINT(41.576766920494265 74.85590758900501) bank175420 +175421 POINT(39.9472212389383 74.97790456847868) bank175421 +175422 POINT(40.616831232315064 74.24645247038559) bank175422 +175423 POINT(40.7902513099551 73.97531170279382) bank175423 +175424 POINT(40.13798745566842 74.9627959907815) bank175424 +175425 POINT(40.71763168268414 73.6359751543975) bank175425 +175426 POINT(41.16508747525318 74.22831724102929) bank175426 +175427 POINT(40.77369881075033 74.3094849683035) bank175427 +175428 POINT(40.56674987159342 73.23860250363931) bank175428 +175429 POINT(40.29363114092897 73.8138777116672) bank175429 +175430 POINT(39.745291788684504 73.65718945361903) bank175430 +175431 POINT(40.270020760936276 73.32828635524648) bank175431 +175432 POINT(40.91290164781363 74.9771515177214) bank175432 +175433 POINT(41.11430032786442 74.69244613297698) bank175433 +175434 POINT(40.700667931210205 73.2181602062283) bank175434 +175435 POINT(40.79133963257318 74.92605029834273) bank175435 +175436 POINT(40.95453242323762 74.29064175305709) bank175436 +175437 POINT(41.2238281876423 74.68974543381115) bank175437 +175438 POINT(40.53309620330083 74.17813623800349) bank175438 +175439 POINT(41.22112512332208 74.36078383164009) bank175439 +175440 POINT(40.39935976428529 73.11698590113608) bank175440 +175441 POINT(40.72626331489768 73.431942797803) bank175441 +175442 POINT(40.3243740440086 74.2970744218068) bank175442 +175443 POINT(40.55715658411876 73.59512387666506) bank175443 +175444 POINT(40.54078843915869 73.24393851737078) bank175444 +175445 POINT(40.6379606613823 73.38132253902705) bank175445 +175446 POINT(41.09655930067882 73.43548348392231) bank175446 +175447 POINT(41.0852033939991 73.25132572623943) bank175447 +175448 POINT(40.607617500314674 73.24028594834475) bank175448 +175449 POINT(40.579598216790586 74.8211045491294) bank175449 +175450 POINT(39.87731853265197 74.84267917516433) bank175450 +175451 POINT(41.48352656971129 73.99751105613444) bank175451 +175452 POINT(41.56144029606203 74.95538244620488) bank175452 +175453 POINT(41.69062346116476 74.06648980510785) bank175453 +175454 POINT(39.7352256745416 73.70246038615858) bank175454 +175455 POINT(41.46933694297337 73.66625623056528) bank175455 +175456 POINT(41.17392066571901 74.22803768633912) bank175456 +175457 POINT(39.864568802005195 73.96122458203064) bank175457 +175458 POINT(40.897676761379614 74.94679627595738) bank175458 +175459 POINT(41.2700456735888 74.96487021809017) bank175459 +175460 POINT(41.014163795705116 74.45290507185791) bank175460 +175461 POINT(39.7387860874966 73.77724815438397) bank175461 +175462 POINT(40.10402897005377 73.47646155204114) bank175462 +175463 POINT(39.726654906645834 74.22933642684404) bank175463 +175464 POINT(40.83842916201661 74.38109801102588) bank175464 +175465 POINT(40.17068514063905 73.21700082033611) bank175465 +175466 POINT(41.03179353822922 74.79223327407001) bank175466 +175467 POINT(40.28767699701622 74.71424780594174) bank175467 +175468 POINT(40.08981467746537 74.97329838050206) bank175468 +175469 POINT(40.332490140939655 73.51047219296555) bank175469 +175470 POINT(40.576712104499755 73.74193779194707) bank175470 +175471 POINT(41.57513506359161 73.78619056291923) bank175471 +175472 POINT(40.578944706465485 74.6854452283102) bank175472 +175473 POINT(40.601037104328846 73.52919205024585) bank175473 +175474 POINT(41.41833020733009 73.7319788315501) bank175474 +175475 POINT(39.73200824982035 74.07006089972145) bank175475 +175476 POINT(40.426275189065095 74.59151620974617) bank175476 +175477 POINT(39.986096083887816 74.99414068237927) bank175477 +175478 POINT(41.257479822804676 73.47819697051126) bank175478 +175479 POINT(40.11853593503971 73.84377580302198) bank175479 +175480 POINT(39.87596601899376 74.03272108511335) bank175480 +175481 POINT(39.876365445111276 74.09088399252131) bank175481 +175482 POINT(40.20754772719036 73.86862815351517) bank175482 +175483 POINT(39.74479764299834 73.98099352529903) bank175483 +175484 POINT(40.30665173481663 74.89695465378803) bank175484 +175485 POINT(39.92482839827437 73.45863527050054) bank175485 +175486 POINT(40.65970141692925 74.87283202920064) bank175486 +175487 POINT(40.24855496904222 73.701595507392) bank175487 +175488 POINT(40.19709930757943 74.0148891681276) bank175488 +175489 POINT(39.72112875279643 73.9706221121765) bank175489 +175490 POINT(41.14785103343547 73.68773404908677) bank175490 +175491 POINT(41.332973443060055 74.96721635676319) bank175491 +175492 POINT(41.65521145416592 73.6095949405651) bank175492 +175493 POINT(41.28264695668815 74.11618488329282) bank175493 +175494 POINT(41.162522522528015 74.29511571738384) bank175494 +175495 POINT(40.93156140043072 74.50693268555284) bank175495 +175496 POINT(40.72262863420734 73.70275247711763) bank175496 +175497 POINT(40.212676788448825 73.88342283468515) bank175497 +175498 POINT(39.84931544917436 73.28934794467972) bank175498 +175499 POINT(40.32111095853432 73.77237454634172) bank175499 +175500 POINT(40.677909863225935 74.25512165224681) bank175500 +175501 POINT(40.77131516321544 74.75084537314763) bank175501 +175502 POINT(41.704143305132796 73.8245654923598) bank175502 +175503 POINT(41.327537703378525 74.67299739425692) bank175503 +175504 POINT(39.973822703915175 73.39038925121297) bank175504 +175505 POINT(40.97557186570091 73.0343590398232) bank175505 +175506 POINT(40.552001765554635 74.47970055977233) bank175506 +175507 POINT(40.71472517381265 73.21290418286979) bank175507 +175508 POINT(41.30117777231356 74.44460249247435) bank175508 +175509 POINT(40.332837604694106 73.16272786554569) bank175509 +175510 POINT(41.55142895079246 73.5244193310845) bank175510 +175511 POINT(40.5689300902729 74.0473828395844) bank175511 +175512 POINT(41.46583139204975 73.65669332659962) bank175512 +175513 POINT(40.93784033417202 73.05268618862509) bank175513 +175514 POINT(41.403869303288076 73.24407131714118) bank175514 +175515 POINT(41.20639214634385 73.4235225528665) bank175515 +175516 POINT(40.772454550295535 73.62436225336684) bank175516 +175517 POINT(41.31259879712273 73.4797574297685) bank175517 +175518 POINT(39.88410013900929 74.43757568545567) bank175518 +175519 POINT(41.03222680340849 74.15475907370575) bank175519 +175520 POINT(41.08857008415726 74.91845332193685) bank175520 +175521 POINT(40.59718245345398 73.66070216633462) bank175521 +175522 POINT(40.16289938429945 73.78260011953925) bank175522 +175523 POINT(41.25735811131258 74.59727520802805) bank175523 +175524 POINT(41.251160043373126 73.71999179130644) bank175524 +175525 POINT(40.081964252872176 73.59376514953132) bank175525 +175526 POINT(40.80015346828801 74.53468905554246) bank175526 +175527 POINT(40.03145922595868 74.5593013691573) bank175527 +175528 POINT(41.66249536327056 73.58960544486229) bank175528 +175529 POINT(40.14626374866333 74.07281503714366) bank175529 +175530 POINT(41.353955205499254 74.84571854374094) bank175530 +175531 POINT(40.172341568803965 73.76687812739004) bank175531 +175532 POINT(41.05823639680426 73.05223975446485) bank175532 +175533 POINT(41.22821142007664 73.73880472691178) bank175533 +175534 POINT(40.451546265483344 73.37205356337417) bank175534 +175535 POINT(39.8655070647542 73.48536214396199) bank175535 +175536 POINT(41.60339452904144 73.90416274016124) bank175536 +175537 POINT(40.891540972988736 73.64098294715319) bank175537 +175538 POINT(41.325759197860776 74.0470678768262) bank175538 +175539 POINT(40.48532996503696 74.20537658093778) bank175539 +175540 POINT(40.61297818935118 74.67143838602861) bank175540 +175541 POINT(40.41758134225941 74.73855267451894) bank175541 +175542 POINT(40.08750997384379 73.97280550934686) bank175542 +175543 POINT(40.812829575843864 74.98047895591927) bank175543 +175544 POINT(40.328720721985505 74.46264150060843) bank175544 +175545 POINT(40.230354313450825 73.21666971786154) bank175545 +175546 POINT(40.76855041620275 73.11298211193149) bank175546 +175547 POINT(40.641765098921006 73.99246042163276) bank175547 +175548 POINT(40.56203449948093 74.83807615749123) bank175548 +175549 POINT(39.86154219100207 73.34153841889578) bank175549 +175550 POINT(40.15314381836708 74.83498023197703) bank175550 +175551 POINT(40.06088316592411 74.6400743568535) bank175551 +175552 POINT(41.66878790353368 74.61226255093057) bank175552 +175553 POINT(40.27139931106811 74.69880255819399) bank175553 +175554 POINT(41.21033118314466 74.77143441420853) bank175554 +175555 POINT(39.92663148454278 73.68886999495228) bank175555 +175556 POINT(40.34622308113343 73.85719236124622) bank175556 +175557 POINT(39.974708560071186 73.66295266394262) bank175557 +175558 POINT(41.33360090047632 73.7913354028994) bank175558 +175559 POINT(40.172504757688806 73.21562233109181) bank175559 +175560 POINT(40.47010954911556 73.56415908097661) bank175560 +175561 POINT(41.66143259714183 74.13046127770036) bank175561 +175562 POINT(40.23576089150207 74.47362335811684) bank175562 +175563 POINT(40.235505898342176 73.04538053991566) bank175563 +175564 POINT(41.336199731497324 73.34778854311374) bank175564 +175565 POINT(41.23302350328881 73.45175647526996) bank175565 +175566 POINT(40.17865364853695 74.89187986262678) bank175566 +175567 POINT(39.738114563220726 73.07638480182224) bank175567 +175568 POINT(40.39933376677529 73.12403691084762) bank175568 +175569 POINT(40.70480633685774 74.96500294476462) bank175569 +175570 POINT(41.542360399486505 74.35125322134309) bank175570 +175571 POINT(39.782928672749435 73.01063431635099) bank175571 +175572 POINT(41.09595355787649 73.48437203467722) bank175572 +175573 POINT(40.761697485546705 74.92207335963845) bank175573 +175574 POINT(40.22326342675114 74.60030307293168) bank175574 +175575 POINT(39.7502922496945 74.7659812152002) bank175575 +175576 POINT(40.12336345075585 73.2715264802299) bank175576 +175577 POINT(40.76168834306547 73.96865530133513) bank175577 +175578 POINT(39.92188497941584 74.95531798612764) bank175578 +175579 POINT(41.13361478415861 74.60704236915977) bank175579 +175580 POINT(40.74373286597046 74.33358176293248) bank175580 +175581 POINT(41.21200978195459 74.47608236619908) bank175581 +175582 POINT(39.87715637158472 74.01672280669376) bank175582 +175583 POINT(41.57592567748635 74.47944689052865) bank175583 +175584 POINT(41.54222070899467 73.32085239122617) bank175584 +175585 POINT(40.613359794439944 74.54791838212948) bank175585 +175586 POINT(41.19006581248802 74.09384846890975) bank175586 +175587 POINT(41.68614434211641 73.01091617103197) bank175587 +175588 POINT(39.91746161191351 74.11202844848572) bank175588 +175589 POINT(39.91387456513724 73.29625117465481) bank175589 +175590 POINT(40.979429400245266 74.24524459856221) bank175590 +175591 POINT(40.70094087020373 73.44677697993772) bank175591 +175592 POINT(41.13823502686804 74.41870365639366) bank175592 +175593 POINT(40.586362179067216 74.3615824492666) bank175593 +175594 POINT(40.442346309410674 73.80881441995298) bank175594 +175595 POINT(40.5083355654885 74.74932943945873) bank175595 +175596 POINT(40.88346217057428 74.91554734314859) bank175596 +175597 POINT(41.374865573741445 73.40103892914881) bank175597 +175598 POINT(40.972310684455195 73.83793216368475) bank175598 +175599 POINT(40.45902621187923 74.03696256270219) bank175599 +175600 POINT(40.21602684471901 74.95471750706628) bank175600 +175601 POINT(41.15097694004586 74.4770245106359) bank175601 +175602 POINT(41.103943130595496 74.61405741652048) bank175602 +175603 POINT(40.91253892621151 73.39775183551737) bank175603 +175604 POINT(41.01423443236345 74.09329242493958) bank175604 +175605 POINT(40.35767300423503 74.60389616969495) bank175605 +175606 POINT(41.4694406231571 74.19337069511076) bank175606 +175607 POINT(39.9052159911623 74.72199506764039) bank175607 +175608 POINT(41.14679738005165 73.73778167618623) bank175608 +175609 POINT(41.55860828443968 74.16929602039781) bank175609 +175610 POINT(41.3906107308525 73.61184308435008) bank175610 +175611 POINT(41.55059623988111 73.25425117773612) bank175611 +175612 POINT(41.24381071795199 73.81824536550366) bank175612 +175613 POINT(40.79377926784633 74.18794260396166) bank175613 +175614 POINT(40.3963396824998 74.85868455485887) bank175614 +175615 POINT(41.11295810505088 74.28306924088157) bank175615 +175616 POINT(39.884484005967444 74.14985620222907) bank175616 +175617 POINT(39.85483806971116 74.60061478411325) bank175617 +175618 POINT(41.01992233504285 73.82685997881333) bank175618 +175619 POINT(40.37073537736975 73.59296610114995) bank175619 +175620 POINT(39.95334324856074 74.68391999889558) bank175620 +175621 POINT(41.302239274747095 73.65855674825411) bank175621 +175622 POINT(41.508906828282456 74.78418112320884) bank175622 +175623 POINT(40.24879203702311 74.6282281009458) bank175623 +175624 POINT(40.32391446914687 73.88659487074985) bank175624 +175625 POINT(40.801713850813876 75.00265946800602) bank175625 +175626 POINT(41.6169105188128 73.01324343656972) bank175626 +175627 POINT(39.82214419029787 73.32549297721724) bank175627 +175628 POINT(41.5482931406882 73.63868205951815) bank175628 +175629 POINT(40.13076565648241 73.35080724091672) bank175629 +175630 POINT(41.230148338117864 73.96917699284516) bank175630 +175631 POINT(40.596393245694046 74.21429913610748) bank175631 +175632 POINT(39.71587758910107 73.95287465208551) bank175632 +175633 POINT(40.19619603543649 73.7947873170837) bank175633 +175634 POINT(40.09583488264432 73.43023207596576) bank175634 +175635 POINT(40.229144878448444 74.80734436922637) bank175635 +175636 POINT(40.0812496337046 74.25585286211978) bank175636 +175637 POINT(41.34280106550168 74.29085488937909) bank175637 +175638 POINT(41.55020149434111 74.88783834228502) bank175638 +175639 POINT(41.08141402192755 73.1817155869501) bank175639 +175640 POINT(40.043697076232036 74.37535915525304) bank175640 +175641 POINT(40.075260449942014 73.15794612567927) bank175641 +175642 POINT(40.485983018685815 73.07537478553377) bank175642 +175643 POINT(40.54203324398376 73.68414523338515) bank175643 +175644 POINT(41.577583586660005 74.79438831354734) bank175644 +175645 POINT(40.72374680555234 73.38575118391013) bank175645 +175646 POINT(40.289271378154 73.61377555846859) bank175646 +175647 POINT(41.5038941415409 74.32221635136528) bank175647 +175648 POINT(40.19585777394302 74.87895352056545) bank175648 +175649 POINT(41.13534694105241 73.22372654643) bank175649 +175650 POINT(40.622845446925766 73.22622487021216) bank175650 +175651 POINT(40.54107004508372 73.6615274785233) bank175651 +175652 POINT(39.8063312785081 74.11973316425278) bank175652 +175653 POINT(41.06782295173281 73.50911081080257) bank175653 +175654 POINT(40.24737875900074 74.32300064902827) bank175654 +175655 POINT(40.08830441208328 74.54064233126033) bank175655 +175656 POINT(41.34236951353844 73.87823193602487) bank175656 +175657 POINT(41.14821865866251 74.5950454096135) bank175657 +175658 POINT(40.487036493980504 74.74489068039415) bank175658 +175659 POINT(40.078515525058236 73.22261125133144) bank175659 +175660 POINT(40.388010133471376 73.44012432749533) bank175660 +175661 POINT(41.02683204198541 73.24625320814839) bank175661 +175662 POINT(41.26288045593068 74.26251075867845) bank175662 +175663 POINT(40.973005796965644 73.78861695667035) bank175663 +175664 POINT(39.99656994739859 74.55414015093166) bank175664 +175665 POINT(40.955606913983864 74.95530488829418) bank175665 +175666 POINT(39.8881194127357 73.60142050291354) bank175666 +175667 POINT(40.27639399550542 74.68634836602307) bank175667 +175668 POINT(40.538781866639525 73.283248540373) bank175668 +175669 POINT(41.55891958355143 73.53317762386608) bank175669 +175670 POINT(41.302671866320395 74.18222921101109) bank175670 +175671 POINT(39.761630677641655 73.64945200745913) bank175671 +175672 POINT(39.74656933990072 73.20822933426726) bank175672 +175673 POINT(41.54016362930994 73.2455217700514) bank175673 +175674 POINT(39.825035826651096 74.72820721762905) bank175674 +175675 POINT(41.24292952939961 73.0712539512278) bank175675 +175676 POINT(39.841009799767754 74.47843360474862) bank175676 +175677 POINT(41.2535035000324 74.53973524337427) bank175677 +175678 POINT(40.884145261439066 73.10354349600478) bank175678 +175679 POINT(41.19107836932233 74.40481900212299) bank175679 +175680 POINT(41.10159908596135 74.51314593551061) bank175680 +175681 POINT(40.08461160268303 74.96695160725356) bank175681 +175682 POINT(41.59564248378068 74.76780824706772) bank175682 +175683 POINT(40.71532612287831 73.70347148771882) bank175683 +175684 POINT(41.09411857364972 74.1691254219734) bank175684 +175685 POINT(40.929753197444946 74.88746401401835) bank175685 +175686 POINT(41.16237634113503 73.35280248733655) bank175686 +175687 POINT(41.4876478094364 73.04065325850458) bank175687 +175688 POINT(41.14331017168137 74.74491376355063) bank175688 +175689 POINT(41.1196150947922 73.35288957355729) bank175689 +175690 POINT(39.784539398113836 74.71169101403792) bank175690 +175691 POINT(40.03095068383569 73.45418025305976) bank175691 +175692 POINT(41.29030836576003 73.48063334994474) bank175692 +175693 POINT(41.55398521660975 74.30190879450488) bank175693 +175694 POINT(40.95470996863226 73.01675586168021) bank175694 +175695 POINT(41.23419523313189 73.5695215560322) bank175695 +175696 POINT(41.362140061079 74.50881491590269) bank175696 +175697 POINT(39.75546628969113 74.95139312003825) bank175697 +175698 POINT(41.23334607509972 73.94613061271536) bank175698 +175699 POINT(40.19082914540573 73.40461468371176) bank175699 +175700 POINT(39.91872030339551 73.46755391853958) bank175700 +175701 POINT(40.721347679946795 73.45850418655056) bank175701 +175702 POINT(40.689286768409666 74.97505980626012) bank175702 +175703 POINT(40.73704069451778 74.94246819621277) bank175703 +175704 POINT(39.73037733613642 73.50734654661649) bank175704 +175705 POINT(39.940129230695796 73.94418533823782) bank175705 +175706 POINT(41.07187147499564 73.59233718667362) bank175706 +175707 POINT(41.20554830898129 74.03728676409219) bank175707 +175708 POINT(40.56179453513234 73.17840503105303) bank175708 +175709 POINT(41.634261788873204 73.74930487847477) bank175709 +175710 POINT(40.17866939146086 73.34375882441259) bank175710 +175711 POINT(40.75149344404367 73.33708986411818) bank175711 +175712 POINT(41.30789836941653 73.6113321804687) bank175712 +175713 POINT(41.47305810569451 73.8696965046537) bank175713 +175714 POINT(40.658122779441634 74.69863005791447) bank175714 +175715 POINT(41.64630864905208 73.27938326480685) bank175715 +175716 POINT(39.9994915606503 73.12063503592336) bank175716 +175717 POINT(41.65977272592654 73.57385571442911) bank175717 +175718 POINT(40.016332535300855 73.10326381629298) bank175718 +175719 POINT(39.90544214977504 73.93548525991558) bank175719 +175720 POINT(40.98472633034108 74.58591358715124) bank175720 +175721 POINT(41.55497784376345 74.0256412007544) bank175721 +175722 POINT(40.60997258424059 73.77960346990682) bank175722 +175723 POINT(41.261559833275136 73.40128802357754) bank175723 +175724 POINT(39.99439043293465 74.88829502765931) bank175724 +175725 POINT(40.373394126741864 74.37186385401928) bank175725 +175726 POINT(40.58508993047302 74.0807188851457) bank175726 +175727 POINT(40.80318959346848 73.55761944867247) bank175727 +175728 POINT(40.27558568681526 73.21908648174393) bank175728 +175729 POINT(39.84082845948201 74.86977187162587) bank175729 +175730 POINT(40.8935050495882 73.35069497941588) bank175730 +175731 POINT(40.4902840356544 74.67154451981146) bank175731 +175732 POINT(41.53230807022762 74.97683731368704) bank175732 +175733 POINT(40.04084766988327 74.51298046189831) bank175733 +175734 POINT(40.86642648478732 74.65840551691761) bank175734 +175735 POINT(40.18943594683414 73.25545139053635) bank175735 +175736 POINT(41.394795438546815 74.52252139175097) bank175736 +175737 POINT(41.18576099972904 74.1835108814294) bank175737 +175738 POINT(41.52196836971136 73.76046361804121) bank175738 +175739 POINT(39.965434290165724 73.23351860448209) bank175739 +175740 POINT(40.12616751270398 74.503823210677) bank175740 +175741 POINT(40.49421135190189 73.19639056227115) bank175741 +175742 POINT(39.85617625473065 74.74269445262277) bank175742 +175743 POINT(40.90305283554811 74.30768371162041) bank175743 +175744 POINT(40.40083518131433 73.67428158754042) bank175744 +175745 POINT(40.29804992828755 74.0494917221561) bank175745 +175746 POINT(41.1763655287072 74.55897504038846) bank175746 +175747 POINT(41.67056466083502 73.53242998296696) bank175747 +175748 POINT(40.68589043418402 74.83910698163295) bank175748 +175749 POINT(40.879442199999204 73.2224331923849) bank175749 +175750 POINT(40.90607778215663 74.65969058294398) bank175750 +175751 POINT(41.4440578697351 74.57955399939651) bank175751 +175752 POINT(40.06686591050812 73.55475376053708) bank175752 +175753 POINT(41.262848797078654 74.03829451246928) bank175753 +175754 POINT(40.95569128225311 74.68112038506304) bank175754 +175755 POINT(40.04824627349118 73.5740027669171) bank175755 +175756 POINT(41.69834315355984 74.18591196841392) bank175756 +175757 POINT(40.385393237392115 73.51456313210016) bank175757 +175758 POINT(41.41044386390648 73.15245014667116) bank175758 +175759 POINT(41.4329213649326 73.73415826389527) bank175759 +175760 POINT(40.89405771338774 74.31457420102836) bank175760 +175761 POINT(40.30192383081806 73.10905758584047) bank175761 +175762 POINT(39.90425293796618 74.20516879486945) bank175762 +175763 POINT(40.85395739258108 74.27788690593673) bank175763 +175764 POINT(39.913117767332146 73.5649176379642) bank175764 +175765 POINT(40.14351051255135 73.5742116910988) bank175765 +175766 POINT(40.23036520580692 74.67951463456973) bank175766 +175767 POINT(40.519849527496234 74.37251261241406) bank175767 +175768 POINT(41.395890657080805 73.77332797822172) bank175768 +175769 POINT(41.66399818889941 73.41963000658896) bank175769 +175770 POINT(40.783798696798065 73.7025661830875) bank175770 +175771 POINT(41.377193769181076 74.32784792989753) bank175771 +175772 POINT(39.987194708292755 73.07843146166344) bank175772 +175773 POINT(40.43298319151403 74.72291293227154) bank175773 +175774 POINT(40.61367775498535 74.89107368258375) bank175774 +175775 POINT(40.26413194202813 74.07214463285257) bank175775 +175776 POINT(41.07077954823176 74.580662600773) bank175776 +175777 POINT(40.95864716975302 73.7324384553427) bank175777 +175778 POINT(41.38851161254547 74.89996542731588) bank175778 +175779 POINT(41.64216428331013 74.61502386202486) bank175779 +175780 POINT(40.899813682241685 74.08937346018412) bank175780 +175781 POINT(41.13331231580428 74.14041656580548) bank175781 +175782 POINT(40.36564632659947 73.47358948253579) bank175782 +175783 POINT(39.99806597340091 74.20124688786548) bank175783 +175784 POINT(40.651200644221944 74.0920470836358) bank175784 +175785 POINT(41.0517419096799 74.7251952763308) bank175785 +175786 POINT(40.838704522417245 74.62554094189329) bank175786 +175787 POINT(39.98599824293762 74.80617291292664) bank175787 +175788 POINT(41.209416208850755 74.50177602879474) bank175788 +175789 POINT(39.888547160971484 74.49969333346377) bank175789 +175790 POINT(40.53143077205127 73.0158494811277) bank175790 +175791 POINT(40.74327088183185 74.16365691601156) bank175791 +175792 POINT(40.61949240120442 74.29516373327134) bank175792 +175793 POINT(41.55637759204728 74.91308169168761) bank175793 +175794 POINT(40.09648144170425 73.04446089771174) bank175794 +175795 POINT(41.18800505625258 74.77946673750444) bank175795 +175796 POINT(40.841227136690556 74.86540388463683) bank175796 +175797 POINT(41.165548873348385 73.51171184652952) bank175797 +175798 POINT(41.04041728912319 74.20348387919591) bank175798 +175799 POINT(41.155343310704396 73.202059942961) bank175799 +175800 POINT(41.34069833711185 74.20864668672853) bank175800 +175801 POINT(41.704492397846295 74.13492270323337) bank175801 +175802 POINT(40.26099051581866 73.51551641186633) bank175802 +175803 POINT(40.12884411637117 74.60260589737474) bank175803 +175804 POINT(40.58887652304402 74.02211205141528) bank175804 +175805 POINT(41.535733891877285 73.55518834769639) bank175805 +175806 POINT(39.8701405512457 74.01751115066415) bank175806 +175807 POINT(39.716091953540925 73.9960078607247) bank175807 +175808 POINT(39.98765517424085 74.77003712680309) bank175808 +175809 POINT(41.66348572454236 73.91868785870591) bank175809 +175810 POINT(41.60776074621268 74.2386888504549) bank175810 +175811 POINT(40.16478776718339 73.88734823783017) bank175811 +175812 POINT(41.65301610045274 74.70799333184765) bank175812 +175813 POINT(40.60076240430156 73.26350250696424) bank175813 +175814 POINT(41.26903515333401 73.3275886466198) bank175814 +175815 POINT(41.14041518835529 74.61380303768874) bank175815 +175816 POINT(41.29562019552975 73.39382536528495) bank175816 +175817 POINT(39.9871020868641 73.01033671749397) bank175817 +175818 POINT(41.29320275307225 73.77772260303097) bank175818 +175819 POINT(41.5652549865525 73.09474003115929) bank175819 +175820 POINT(40.668465752657774 74.46254727454541) bank175820 +175821 POINT(41.56685794572173 74.57687563258551) bank175821 +175822 POINT(41.52527951630772 73.97175042479135) bank175822 +175823 POINT(39.901728049835484 74.01328517099641) bank175823 +175824 POINT(40.960373446049076 74.24640635496462) bank175824 +175825 POINT(40.838761288927465 73.0990825945204) bank175825 +175826 POINT(41.57289648528268 73.02891205516954) bank175826 +175827 POINT(41.39878599835093 74.61385696835265) bank175827 +175828 POINT(40.95611012958079 74.00845522179209) bank175828 +175829 POINT(40.5139008425593 74.37125452505211) bank175829 +175830 POINT(40.43596639827507 74.49449489680686) bank175830 +175831 POINT(40.07130025077554 73.0954059264403) bank175831 +175832 POINT(39.82044725638061 73.01719103565709) bank175832 +175833 POINT(40.32531406756287 74.26366029815662) bank175833 +175834 POINT(40.1809563228406 73.3380797139626) bank175834 +175835 POINT(40.464091093253714 74.16991185421145) bank175835 +175836 POINT(40.00215932052302 74.57400557320842) bank175836 +175837 POINT(40.57728666773727 74.88416802983036) bank175837 +175838 POINT(41.05999339185864 73.53294434727712) bank175838 +175839 POINT(41.3182239474883 74.13820687067589) bank175839 +175840 POINT(40.494061688176906 74.16296153605646) bank175840 +175841 POINT(40.17016552739083 73.16302869323994) bank175841 +175842 POINT(40.87617409973197 74.82558108905279) bank175842 +175843 POINT(41.49487770469667 74.5253358542591) bank175843 +175844 POINT(41.55345534572376 73.60358065266529) bank175844 +175845 POINT(40.774128964852345 74.79567263505655) bank175845 +175846 POINT(41.482433599246114 73.95011763697921) bank175846 +175847 POINT(40.84820715837832 73.67782049240499) bank175847 +175848 POINT(39.931704395495515 74.16569715128105) bank175848 +175849 POINT(41.53220993345057 73.61427648020765) bank175849 +175850 POINT(40.34533220012357 74.21953019347072) bank175850 +175851 POINT(40.76828065024423 73.85043497063329) bank175851 +175852 POINT(41.264357243389306 74.4809490240479) bank175852 +175853 POINT(40.140911295213165 73.85891010423448) bank175853 +175854 POINT(41.34346094326963 74.93809236087534) bank175854 +175855 POINT(41.36768266527933 73.86245187485504) bank175855 +175856 POINT(41.25196330549771 73.05498847325815) bank175856 +175857 POINT(41.044194029124945 74.14582466015216) bank175857 +175858 POINT(40.37897279991326 73.4105768968925) bank175858 +175859 POINT(40.07084138358043 74.19384072513944) bank175859 +175860 POINT(41.0354904479059 74.76298061400158) bank175860 +175861 POINT(40.881320760187606 73.71981914186959) bank175861 +175862 POINT(40.6725334878264 74.22047536599726) bank175862 +175863 POINT(40.47559504464515 74.90416667792947) bank175863 +175864 POINT(41.5969993410368 74.15451588724801) bank175864 +175865 POINT(40.57641319039255 73.33720032097341) bank175865 +175866 POINT(40.17171545310112 74.22038972179074) bank175866 +175867 POINT(40.889844684028404 74.34968359522772) bank175867 +175868 POINT(40.931922467712745 73.83193394016804) bank175868 +175869 POINT(41.54756723435702 73.59312748555442) bank175869 +175870 POINT(41.39205194977971 74.37119077901318) bank175870 +175871 POINT(40.73881688826044 74.75268395834944) bank175871 +175872 POINT(41.160271389648 73.18197873071958) bank175872 +175873 POINT(40.61358802679345 74.9001766624386) bank175873 +175874 POINT(41.54937846222557 73.19765610821004) bank175874 +175875 POINT(41.50428945947797 73.52931219624179) bank175875 +175876 POINT(40.160454739821915 74.19447843882149) bank175876 +175877 POINT(40.9861266027747 74.92829421881558) bank175877 +175878 POINT(40.12874354518681 73.33503052941893) bank175878 +175879 POINT(40.44111489416441 73.96737497758089) bank175879 +175880 POINT(40.82999042426707 74.72395859776861) bank175880 +175881 POINT(40.60675626475142 73.66690252475635) bank175881 +175882 POINT(39.728510499237125 73.32053094630966) bank175882 +175883 POINT(40.664074877173206 74.87065967889923) bank175883 +175884 POINT(41.12710624275821 74.73613628473397) bank175884 +175885 POINT(41.40266174509532 73.91860931187112) bank175885 +175886 POINT(41.122477816418275 73.27351768479853) bank175886 +175887 POINT(40.66561925853985 73.73178139554275) bank175887 +175888 POINT(39.877374401650854 73.82553064053515) bank175888 +175889 POINT(40.96539252243174 73.98130931765706) bank175889 +175890 POINT(40.98746852427938 73.47983122656582) bank175890 +175891 POINT(41.17653744987856 73.90621500473632) bank175891 +175892 POINT(41.56268275032565 73.15567553726208) bank175892 +175893 POINT(40.305885786980014 74.05449693996282) bank175893 +175894 POINT(40.884549325295936 74.79538677918885) bank175894 +175895 POINT(39.81591161700066 74.3959113636206) bank175895 +175896 POINT(40.14330878801081 74.53412422545682) bank175896 +175897 POINT(41.29541103930515 73.06454055082334) bank175897 +175898 POINT(40.36879341612691 73.97731042180256) bank175898 +175899 POINT(40.79879957388544 73.5596143162113) bank175899 +175900 POINT(40.25470612732075 73.74643611552615) bank175900 +175901 POINT(41.21021282658825 73.11253199522582) bank175901 +175902 POINT(41.14854937910911 73.24794293183483) bank175902 +175903 POINT(41.35829564628031 74.96786479964659) bank175903 +175904 POINT(41.61892175393649 74.3035185680732) bank175904 +175905 POINT(41.31166169250597 74.59069104710261) bank175905 +175906 POINT(40.16439297069151 73.31550597267315) bank175906 +175907 POINT(40.382452419912006 73.27118755297838) bank175907 +175908 POINT(40.00049864113039 73.6574757327728) bank175908 +175909 POINT(39.807007754259615 73.71595401891396) bank175909 +175910 POINT(40.380783583213365 73.65962126683563) bank175910 +175911 POINT(40.84124836101453 73.14819169513197) bank175911 +175912 POINT(41.0824823214639 73.74009952221111) bank175912 +175913 POINT(40.901390268886864 74.4792823344923) bank175913 +175914 POINT(41.2219599628516 73.08156914842922) bank175914 +175915 POINT(39.816539665797215 74.87720617606413) bank175915 +175916 POINT(41.64017419897957 74.89216363291935) bank175916 +175917 POINT(40.260805869696426 73.71931479873889) bank175917 +175918 POINT(40.918804776836254 74.54821049619903) bank175918 +175919 POINT(41.4679202267101 74.10887838392534) bank175919 +175920 POINT(41.54126701967536 73.06130794528197) bank175920 +175921 POINT(40.15416486346734 74.59855133307276) bank175921 +175922 POINT(41.03178930335132 73.55868452668143) bank175922 +175923 POINT(41.22033442467038 73.94610729506714) bank175923 +175924 POINT(41.099977975068114 73.0627339219505) bank175924 +175925 POINT(41.494722599975546 73.51413786389115) bank175925 +175926 POINT(41.44455839364322 74.56040198510439) bank175926 +175927 POINT(40.67033711592517 74.54884217070456) bank175927 +175928 POINT(40.0645753709394 74.77814411471087) bank175928 +175929 POINT(39.8922899386655 73.13873170868116) bank175929 +175930 POINT(41.580303484917195 73.70722860462541) bank175930 +175931 POINT(40.765757042478945 73.7724949534503) bank175931 +175932 POINT(40.66178473781453 74.31502133530071) bank175932 +175933 POINT(41.1998101766953 73.71604209546491) bank175933 +175934 POINT(41.184814277194 74.84442782188874) bank175934 +175935 POINT(39.84177014134567 73.5320045453754) bank175935 +175936 POINT(40.21555277965331 73.24251068406207) bank175936 +175937 POINT(39.90856115063024 73.04960066947638) bank175937 +175938 POINT(40.511569335939015 74.37259534238594) bank175938 +175939 POINT(39.81296363656405 73.95472636563103) bank175939 +175940 POINT(41.47159685466663 74.91375410663008) bank175940 +175941 POINT(39.82898079580296 73.296922759089) bank175941 +175942 POINT(41.66844778819076 73.23469962403396) bank175942 +175943 POINT(39.912093790750404 73.32691129218867) bank175943 +175944 POINT(41.44382149700387 74.28746618796356) bank175944 +175945 POINT(41.33007054910421 74.13110199023662) bank175945 +175946 POINT(41.09942223106581 73.43650252933732) bank175946 +175947 POINT(41.380234861981094 74.92136611503413) bank175947 +175948 POINT(40.86585272541888 74.1054366887445) bank175948 +175949 POINT(40.019516907857515 74.34240887749667) bank175949 +175950 POINT(41.61568479559967 74.51105587722462) bank175950 +175951 POINT(40.910873302339134 73.18074666476438) bank175951 +175952 POINT(41.109757518030904 73.2404475154655) bank175952 +175953 POINT(40.521828578606964 73.83437166619085) bank175953 +175954 POINT(40.56241551494345 74.25706356455471) bank175954 +175955 POINT(41.10725901821322 73.5825999887992) bank175955 +175956 POINT(41.17160593355394 74.2790073128667) bank175956 +175957 POINT(40.34489680301659 73.8950550208633) bank175957 +175958 POINT(41.35185603249473 74.25990794402003) bank175958 +175959 POINT(41.202946292761254 73.89749616757068) bank175959 +175960 POINT(40.04824734068851 73.89318321970411) bank175960 +175961 POINT(40.887339125597265 73.65278461980976) bank175961 +175962 POINT(41.22154200799907 74.53630631869521) bank175962 +175963 POINT(40.91463242217052 74.7220435723647) bank175963 +175964 POINT(40.618263674705474 73.57211072471266) bank175964 +175965 POINT(40.08172145625293 74.35684174639273) bank175965 +175966 POINT(40.92447510923658 74.22587349688905) bank175966 +175967 POINT(40.398345826440604 73.29854715843815) bank175967 +175968 POINT(40.48100867604964 73.50383013949065) bank175968 +175969 POINT(41.45504794618916 74.15506556420132) bank175969 +175970 POINT(41.62984886672253 73.34064374187263) bank175970 +175971 POINT(41.60291817297911 74.79236516035824) bank175971 +175972 POINT(40.26153635298042 74.22798936347284) bank175972 +175973 POINT(41.358868110098754 73.45663060928446) bank175973 +175974 POINT(40.02726446555079 74.33577375491114) bank175974 +175975 POINT(41.126403606811785 74.42355964024628) bank175975 +175976 POINT(41.20160531774105 73.06287923478325) bank175976 +175977 POINT(40.00705681742757 74.42721134672185) bank175977 +175978 POINT(40.310051939465154 74.77519530149628) bank175978 +175979 POINT(40.04035947316239 73.44670804914448) bank175979 +175980 POINT(39.8020426444646 73.06137995796088) bank175980 +175981 POINT(39.77679148816139 74.96514026975758) bank175981 +175982 POINT(40.060360716090564 73.89195692068583) bank175982 +175983 POINT(40.67022989329883 74.24952626518844) bank175983 +175984 POINT(39.78781431192039 73.92720983344547) bank175984 +175985 POINT(40.68033705204943 73.30571739686862) bank175985 +175986 POINT(40.34697478368849 74.56462078326354) bank175986 +175987 POINT(41.01808940210304 73.2953273056278) bank175987 +175988 POINT(40.914763511491735 73.09844856068774) bank175988 +175989 POINT(40.83719890875228 74.29559719344097) bank175989 +175990 POINT(39.75148497827252 73.0656982700897) bank175990 +175991 POINT(40.51342620203229 74.3492941748776) bank175991 +175992 POINT(40.48051187335843 73.38460366922371) bank175992 +175993 POINT(41.03575064655033 73.96472773891078) bank175993 +175994 POINT(41.13176661781872 74.00914873916089) bank175994 +175995 POINT(40.86371476996325 74.90703856512663) bank175995 +175996 POINT(41.32556641625231 73.27041822868561) bank175996 +175997 POINT(40.93545025393105 73.23664380420813) bank175997 +175998 POINT(40.58615872641941 74.24580668260494) bank175998 +175999 POINT(40.38058755818674 73.7785776582097) bank175999 +176000 POINT(41.64540420829748 74.66184950731466) bank176000 +176001 POINT(40.78067583502308 73.49754652086824) bank176001 +176002 POINT(41.55787957553019 74.64998809299361) bank176002 +176003 POINT(40.109012332624395 73.8504091209963) bank176003 +176004 POINT(40.701901801045175 74.98777694088534) bank176004 +176005 POINT(40.99936788591639 73.01821163422156) bank176005 +176006 POINT(41.57009684157826 74.72103038761709) bank176006 +176007 POINT(40.173817600319914 74.1746177899026) bank176007 +176008 POINT(39.791436536082 74.51616643839209) bank176008 +176009 POINT(40.467652412019575 73.92970604664414) bank176009 +176010 POINT(40.26164003393128 74.48274453549206) bank176010 +176011 POINT(40.06815325952014 73.8855756747784) bank176011 +176012 POINT(41.595469875312034 73.27999097703704) bank176012 +176013 POINT(41.2117801480795 74.86240623384563) bank176013 +176014 POINT(41.48560085974664 73.07091977421776) bank176014 +176015 POINT(41.22597408412372 74.19724757617091) bank176015 +176016 POINT(39.809974609006375 73.55154648428315) bank176016 +176017 POINT(40.363139380881194 74.64094425158821) bank176017 +176018 POINT(40.04093973339383 73.23144420549161) bank176018 +176019 POINT(40.45198220045548 74.08294622453312) bank176019 +176020 POINT(41.238705343866364 74.77073559449545) bank176020 +176021 POINT(41.552845996067084 74.2517207078129) bank176021 +176022 POINT(40.406638492515754 73.8226000846527) bank176022 +176023 POINT(40.72823840065578 73.21844570317286) bank176023 +176024 POINT(40.34116221119504 73.66036694119052) bank176024 +176025 POINT(40.96140674276186 74.12147243749244) bank176025 +176026 POINT(41.50492386593903 74.58017226389835) bank176026 +176027 POINT(40.647034414300634 73.76081556670097) bank176027 +176028 POINT(40.69035854110711 73.80723292287162) bank176028 +176029 POINT(40.9782188200562 73.65195396324512) bank176029 +176030 POINT(40.60451435680209 73.25709976621371) bank176030 +176031 POINT(41.07402218841779 74.07059336561437) bank176031 +176032 POINT(39.849906902828955 73.90013301614094) bank176032 +176033 POINT(40.21554196803555 73.5424421745778) bank176033 +176034 POINT(40.81107851890678 73.89902180605134) bank176034 +176035 POINT(40.61708776906486 74.8257628935157) bank176035 +176036 POINT(40.86641006631645 73.29183663201785) bank176036 +176037 POINT(41.473180985884746 74.20522669339215) bank176037 +176038 POINT(40.15382080396019 73.80160379588128) bank176038 +176039 POINT(40.58186832698752 74.06437626301772) bank176039 +176040 POINT(41.286476837818824 73.71781669663065) bank176040 +176041 POINT(41.137887018359315 73.40233066392389) bank176041 +176042 POINT(40.4716669136382 73.42025991392721) bank176042 +176043 POINT(41.273259028192534 73.25584997744566) bank176043 +176044 POINT(39.99257202909328 73.95327777274443) bank176044 +176045 POINT(41.46210749203323 73.50282165753424) bank176045 +176046 POINT(41.62311886516301 73.79068944315065) bank176046 +176047 POINT(41.21534211508974 73.3971589135754) bank176047 +176048 POINT(41.45883273981117 73.59337811848424) bank176048 +176049 POINT(41.64500857558012 74.3062600438855) bank176049 +176050 POINT(40.10460506144531 73.04844798891196) bank176050 +176051 POINT(41.0470628105052 73.95393853137611) bank176051 +176052 POINT(40.905952727113444 74.55853143808271) bank176052 +176053 POINT(41.69040118809752 74.65072285237382) bank176053 +176054 POINT(40.95015104871274 74.27874957428016) bank176054 +176055 POINT(40.89564371664424 73.64390622313354) bank176055 +176056 POINT(39.894937364581644 73.0127289985705) bank176056 +176057 POINT(41.00677418672157 74.6106471660941) bank176057 +176058 POINT(40.78991645968433 73.88798239931607) bank176058 +176059 POINT(40.14677328252518 73.09565152707428) bank176059 +176060 POINT(40.11809261394357 73.51325601496484) bank176060 +176061 POINT(39.97967070870683 74.97306841486787) bank176061 +176062 POINT(40.462453898285816 73.99693100170063) bank176062 +176063 POINT(40.110218466411844 73.73422648564285) bank176063 +176064 POINT(39.960541372980565 74.4204159675681) bank176064 +176065 POINT(40.73425308633308 73.85543914114105) bank176065 +176066 POINT(40.22647100744507 73.4877833018692) bank176066 +176067 POINT(41.4916206042832 74.47332634237749) bank176067 +176068 POINT(41.303401122013696 73.7784427320034) bank176068 +176069 POINT(39.98895341646054 73.64987854960069) bank176069 +176070 POINT(40.70816593879658 73.76090190283962) bank176070 +176071 POINT(40.55511000971673 73.65671016027132) bank176071 +176072 POINT(40.21121947531337 74.29927005390184) bank176072 +176073 POINT(40.79323770103508 73.372366977491) bank176073 +176074 POINT(40.63119311434355 73.03717882152469) bank176074 +176075 POINT(40.52195439050877 73.31239127088251) bank176075 +176076 POINT(41.23624266038766 73.5116133850845) bank176076 +176077 POINT(40.619778906200644 73.3009855690128) bank176077 +176078 POINT(41.276249567019256 73.69858965054597) bank176078 +176079 POINT(39.9284676586181 73.68007771072992) bank176079 +176080 POINT(41.27375611491006 73.58177727069504) bank176080 +176081 POINT(39.87811404643259 74.56453418913324) bank176081 +176082 POINT(41.213997858469625 73.4532797996785) bank176082 +176083 POINT(41.24901879887829 74.89616285727998) bank176083 +176084 POINT(40.63373806474786 73.02961721198692) bank176084 +176085 POINT(39.798722314140214 73.59731194454102) bank176085 +176086 POINT(40.35775322174234 73.41302451096645) bank176086 +176087 POINT(39.7259872078416 73.07136274432224) bank176087 +176088 POINT(41.3071881264152 74.11062405740813) bank176088 +176089 POINT(40.81371129611153 73.38913181952887) bank176089 +176090 POINT(41.40031946125694 74.82079731213493) bank176090 +176091 POINT(41.062114430898625 74.61720397311632) bank176091 +176092 POINT(41.021220705759134 73.28460214288414) bank176092 +176093 POINT(41.28928835179127 74.30669462211843) bank176093 +176094 POINT(40.91369427074724 74.88737899473705) bank176094 +176095 POINT(40.08304140504539 73.34506345954988) bank176095 +176096 POINT(40.928415200356376 73.27581417532025) bank176096 +176097 POINT(40.69660444356237 74.98217371558955) bank176097 +176098 POINT(41.14365644510916 74.35862257368481) bank176098 +176099 POINT(40.1579284958414 74.451589887867) bank176099 +176100 POINT(41.654620870025695 74.28912696753544) bank176100 +176101 POINT(40.639364464651024 74.52817890647603) bank176101 +176102 POINT(41.01751879547397 73.78994211438916) bank176102 +176103 POINT(39.80486956956499 73.02671169748763) bank176103 +176104 POINT(40.54204425058297 73.18702800130828) bank176104 +176105 POINT(41.61703767362034 74.31528842383999) bank176105 +176106 POINT(41.6384813269998 73.84337927150041) bank176106 +176107 POINT(40.803651169480574 73.57335520951986) bank176107 +176108 POINT(41.39625105401642 73.50647862050707) bank176108 +176109 POINT(40.102970763729175 73.57293602346962) bank176109 +176110 POINT(39.76921130177884 73.70816187425098) bank176110 +176111 POINT(41.43261918148797 74.1647868447111) bank176111 +176112 POINT(41.46997675300471 74.68472136656733) bank176112 +176113 POINT(39.98743365551447 73.61750063638972) bank176113 +176114 POINT(39.97440198562893 74.02533414769238) bank176114 +176115 POINT(41.320564976538144 74.55981796921628) bank176115 +176116 POINT(40.14463581900442 74.13339635544007) bank176116 +176117 POINT(40.20779395761088 74.72461239087586) bank176117 +176118 POINT(40.95206858038975 74.88235753538943) bank176118 +176119 POINT(41.153898661037466 74.40147362971419) bank176119 +176120 POINT(41.3631968849671 74.39098339506229) bank176120 +176121 POINT(40.9186711313601 74.02234818131619) bank176121 +176122 POINT(41.43304051393932 74.7420866878844) bank176122 +176123 POINT(40.68805719252207 74.1543259506232) bank176123 +176124 POINT(41.308832086666854 74.03119680283716) bank176124 +176125 POINT(41.57965941254568 74.78106058558623) bank176125 +176126 POINT(40.114338563650065 73.81639060297317) bank176126 +176127 POINT(41.23730979331424 73.01603815775361) bank176127 +176128 POINT(40.24022067844054 73.39341582780474) bank176128 +176129 POINT(39.90694364792339 74.21396853259857) bank176129 +176130 POINT(40.270708899755874 74.35734813724467) bank176130 +176131 POINT(41.04768931017576 74.20473978146559) bank176131 +176132 POINT(40.34612473968143 74.2105068843158) bank176132 +176133 POINT(40.75401638677262 74.06385378827139) bank176133 +176134 POINT(41.06572758433909 74.90323518267547) bank176134 +176135 POINT(40.06793325732748 74.05673904171572) bank176135 +176136 POINT(41.31090219733614 73.00778569328922) bank176136 +176137 POINT(40.25986508350543 74.04465009602146) bank176137 +176138 POINT(39.81823754613152 74.40447157785604) bank176138 +176139 POINT(40.896020220690346 75.00290131085109) bank176139 +176140 POINT(40.9717256127709 73.91434650312584) bank176140 +176141 POINT(40.120496746648364 74.68752408414892) bank176141 +176142 POINT(40.17398520347435 74.73407986823435) bank176142 +176143 POINT(39.81616188097339 74.62303634137531) bank176143 +176144 POINT(39.817091640450656 73.73110650474324) bank176144 +176145 POINT(40.09596958513306 73.52534317611047) bank176145 +176146 POINT(40.90085577707706 73.77785263883146) bank176146 +176147 POINT(41.59076948553891 74.7397069066882) bank176147 +176148 POINT(40.854589368872794 74.02279262226676) bank176148 +176149 POINT(41.23563856139859 74.27829555195945) bank176149 +176150 POINT(40.704137150227545 73.02237858560214) bank176150 +176151 POINT(40.40430353348438 73.08412816288144) bank176151 +176152 POINT(41.00666559678817 73.57848970429629) bank176152 +176153 POINT(40.93061726523183 74.87541304196407) bank176153 +176154 POINT(40.18356435158052 73.93455629129959) bank176154 +176155 POINT(39.73103483668255 74.67642644709571) bank176155 +176156 POINT(40.29183479791761 73.27835645052639) bank176156 +176157 POINT(41.20598256063587 74.13275977912986) bank176157 +176158 POINT(39.9728206753676 74.11079201254523) bank176158 +176159 POINT(41.112572583489076 73.11311624777548) bank176159 +176160 POINT(39.90148455988803 74.59170418831768) bank176160 +176161 POINT(40.18155571095983 73.43867137756793) bank176161 +176162 POINT(41.37252631541294 73.35958515309765) bank176162 +176163 POINT(40.06579013916465 73.42620846028507) bank176163 +176164 POINT(41.39213964690943 73.25741125562485) bank176164 +176165 POINT(39.95821043224797 74.3660812938278) bank176165 +176166 POINT(40.54143109139712 73.70369144806205) bank176166 +176167 POINT(40.77295937843766 73.56543831205624) bank176167 +176168 POINT(41.160644479992726 73.37356445488822) bank176168 +176169 POINT(41.489233046283424 74.97213072573041) bank176169 +176170 POINT(40.83921277736813 74.5707850566465) bank176170 +176171 POINT(40.84739177513602 73.97912650080151) bank176171 +176172 POINT(40.937579025283924 74.02584720919599) bank176172 +176173 POINT(39.89167597070384 74.37522875295852) bank176173 +176174 POINT(39.86234992117554 73.95172826545058) bank176174 +176175 POINT(39.84492618589395 73.46155025031563) bank176175 +176176 POINT(40.00911798944659 74.98042770864191) bank176176 +176177 POINT(40.74751536774767 73.70155908354337) bank176177 +176178 POINT(40.950013948625916 73.63641154688199) bank176178 +176179 POINT(40.64795067688173 74.19344113265342) bank176179 +176180 POINT(41.1282017099368 73.58563397510918) bank176180 +176181 POINT(41.553586347588535 74.64973405897288) bank176181 +176182 POINT(41.06617725701179 74.54879127520509) bank176182 +176183 POINT(39.71965193731202 73.93127277069227) bank176183 +176184 POINT(40.17662572068073 74.12927363341633) bank176184 +176185 POINT(41.34640111598552 73.5248262906364) bank176185 +176186 POINT(41.59699488432597 74.93907889589826) bank176186 +176187 POINT(40.57448628590402 74.69940244457341) bank176187 +176188 POINT(41.023004037908635 73.05000679994762) bank176188 +176189 POINT(39.75090604751827 74.70079454603933) bank176189 +176190 POINT(41.37785900387578 73.41415636194088) bank176190 +176191 POINT(40.30181584603446 73.47052124815015) bank176191 +176192 POINT(41.59028447134199 73.63471444052068) bank176192 +176193 POINT(40.72755057823757 74.34838723957184) bank176193 +176194 POINT(41.61276716987389 74.43468602629986) bank176194 +176195 POINT(40.74419499633324 74.75676005509162) bank176195 +176196 POINT(40.474363535929236 74.70222967102069) bank176196 +176197 POINT(40.673424619851566 74.33629619484077) bank176197 +176198 POINT(40.81348042171392 73.59897018778656) bank176198 +176199 POINT(40.61519243262232 74.8350638565051) bank176199 +176200 POINT(39.98343402451484 73.59455380772467) bank176200 +176201 POINT(40.338723110845145 73.54562207895582) bank176201 +176202 POINT(40.35763248382116 74.10625285800035) bank176202 +176203 POINT(41.0841561992537 74.61919248022706) bank176203 +176204 POINT(41.49454517959645 73.66468237261452) bank176204 +176205 POINT(40.52911813794683 73.25765612455528) bank176205 +176206 POINT(39.948206626508906 73.25959606249984) bank176206 +176207 POINT(41.18145096975092 74.92662518767071) bank176207 +176208 POINT(41.25264603967447 74.09232859357942) bank176208 +176209 POINT(40.47836193099191 74.17209167075671) bank176209 +176210 POINT(41.558007947917936 73.01256397659894) bank176210 +176211 POINT(41.12433514857458 73.73931099275626) bank176211 +176212 POINT(41.097361311354284 73.09127974960025) bank176212 +176213 POINT(40.42003770249987 74.35833696265178) bank176213 +176214 POINT(40.3755504643268 74.66806847632238) bank176214 +176215 POINT(40.69774650425884 74.03515931919121) bank176215 +176216 POINT(41.24543454773821 73.80283648353097) bank176216 +176217 POINT(40.75872185218061 74.40347609394911) bank176217 +176218 POINT(40.007386286573954 74.28130714490692) bank176218 +176219 POINT(41.70383306951936 73.9726097510194) bank176219 +176220 POINT(40.915037347766344 73.63683781891343) bank176220 +176221 POINT(40.24940376117095 73.78203128946697) bank176221 +176222 POINT(40.578767176349324 74.69672033533361) bank176222 +176223 POINT(39.73645895391334 74.0986282859544) bank176223 +176224 POINT(40.497585814220805 73.56858482309013) bank176224 +176225 POINT(40.79938623467322 74.04644110233833) bank176225 +176226 POINT(41.48641630619623 74.68391165086051) bank176226 +176227 POINT(41.195755787002064 74.05137318101795) bank176227 +176228 POINT(40.02707868926251 74.22010036033024) bank176228 +176229 POINT(40.4597802996719 74.53658702349554) bank176229 +176230 POINT(40.277846121212846 74.49852897228806) bank176230 +176231 POINT(40.31605103685687 73.05317088155007) bank176231 +176232 POINT(41.522429767561626 74.21123885551033) bank176232 +176233 POINT(41.57533606672775 73.57956712366195) bank176233 +176234 POINT(40.057285540039885 74.84772370678084) bank176234 +176235 POINT(41.29447717014126 73.2486033026859) bank176235 +176236 POINT(40.01091577521013 73.93358060974867) bank176236 +176237 POINT(40.26238367683497 73.04499841201918) bank176237 +176238 POINT(41.63520365492672 74.19865990509987) bank176238 +176239 POINT(40.935756006801284 73.50773160609036) bank176239 +176240 POINT(40.40999779873253 74.2315196672405) bank176240 +176241 POINT(40.75799818642468 73.35375901206754) bank176241 +176242 POINT(40.64796412390076 73.2944747542079) bank176242 +176243 POINT(40.25650082571317 74.49562032259502) bank176243 +176244 POINT(40.0015900063472 73.53520819051444) bank176244 +176245 POINT(41.16555547955312 73.21857569296304) bank176245 +176246 POINT(39.85454209972357 73.35164606835815) bank176246 +176247 POINT(39.77805079335623 74.87431600429458) bank176247 +176248 POINT(40.924925196850324 73.6990540808397) bank176248 +176249 POINT(41.41422358974818 74.53569421041996) bank176249 +176250 POINT(41.52973579382249 73.25290231400821) bank176250 +176251 POINT(40.25212793028371 74.50134670045135) bank176251 +176252 POINT(40.56938126182942 73.5661258883181) bank176252 +176253 POINT(41.40240858714741 74.22383526577897) bank176253 +176254 POINT(41.7008766717599 74.43108941650038) bank176254 +176255 POINT(40.71901226849598 73.30444779134044) bank176255 +176256 POINT(41.19451870123514 73.54902416228929) bank176256 +176257 POINT(40.89294091932831 74.00516355109669) bank176257 +176258 POINT(40.35628556945118 74.77304872449595) bank176258 +176259 POINT(40.912668982443215 74.07012894838746) bank176259 +176260 POINT(40.02888698154954 73.34693317966705) bank176260 +176261 POINT(41.20782053778297 74.82170737897671) bank176261 +176262 POINT(40.54467194085692 74.72446274767935) bank176262 +176263 POINT(40.708953085734166 73.03563516095953) bank176263 +176264 POINT(40.65370148806932 73.33775509854475) bank176264 +176265 POINT(41.3152840396718 74.89486116385683) bank176265 +176266 POINT(39.82000775010594 73.30233277583619) bank176266 +176267 POINT(39.82909852163693 74.78048209297616) bank176267 +176268 POINT(40.61626662925828 73.80066775144539) bank176268 +176269 POINT(40.727546790467684 74.79630165512947) bank176269 +176270 POINT(40.68126946034963 73.34056934151586) bank176270 +176271 POINT(40.29153444085801 74.42770878892657) bank176271 +176272 POINT(40.17760986724301 73.00645218895569) bank176272 +176273 POINT(40.83358767450715 74.76072007798665) bank176273 +176274 POINT(39.8180909324221 73.50770326650608) bank176274 +176275 POINT(40.085489889260174 73.00610839416902) bank176275 +176276 POINT(41.48614343451244 73.20715642967511) bank176276 +176277 POINT(40.87178351797583 73.77234880507531) bank176277 +176278 POINT(41.5741028510755 73.21211470356474) bank176278 +176279 POINT(39.89829532467516 73.34206431590351) bank176279 +176280 POINT(41.12050589093158 74.93537261544729) bank176280 +176281 POINT(40.83400531940036 73.77808568050239) bank176281 +176282 POINT(41.27623185952249 73.31455744347967) bank176282 +176283 POINT(40.94218034576452 74.84775013863558) bank176283 +176284 POINT(41.11993233925914 74.2975668248924) bank176284 +176285 POINT(40.25303685380093 73.92142713417611) bank176285 +176286 POINT(41.29516722202059 73.86064493909848) bank176286 +176287 POINT(40.86854912414831 74.44111980925972) bank176287 +176288 POINT(41.46960020097572 73.33358645816463) bank176288 +176289 POINT(41.02611848030574 74.77855914272175) bank176289 +176290 POINT(40.33112550878872 74.93668122408513) bank176290 +176291 POINT(40.087940674452085 74.97254060320901) bank176291 +176292 POINT(41.27634793245247 74.25528484580279) bank176292 +176293 POINT(40.22279246873731 73.28699778865474) bank176293 +176294 POINT(40.29101447460172 74.94011861511707) bank176294 +176295 POINT(40.01111559586507 74.46881078856747) bank176295 +176296 POINT(41.0502529639463 73.81834352655568) bank176296 +176297 POINT(40.88325270654142 74.70121788129654) bank176297 +176298 POINT(41.04723673710445 74.7789491620186) bank176298 +176299 POINT(41.43066977629239 74.91454304177962) bank176299 +176300 POINT(40.377916638837235 73.36647752328265) bank176300 +176301 POINT(41.489588425006566 73.60859503200183) bank176301 +176302 POINT(40.98158877267747 73.63557186112061) bank176302 +176303 POINT(40.74608030142707 74.84337875478352) bank176303 +176304 POINT(40.875002432164074 74.41237890869503) bank176304 +176305 POINT(40.75044980411383 73.74501203469443) bank176305 +176306 POINT(40.916284854307406 73.36980602358945) bank176306 +176307 POINT(40.96017074777759 73.74270642099846) bank176307 +176308 POINT(39.898601485183825 73.3777990783204) bank176308 +176309 POINT(40.53507789484715 73.53540085562368) bank176309 +176310 POINT(40.22774968729983 73.64587507132117) bank176310 +176311 POINT(40.446953292652864 73.69415808592736) bank176311 +176312 POINT(40.729306599297004 74.22460444111842) bank176312 +176313 POINT(41.468254917185085 73.08240109224495) bank176313 +176314 POINT(40.190092519235456 74.04742025078296) bank176314 +176315 POINT(40.8131070052286 74.02923422789523) bank176315 +176316 POINT(41.22640827185778 73.29423723420452) bank176316 +176317 POINT(40.79740553875947 73.30389566017783) bank176317 +176318 POINT(40.48917216740529 73.70080051513015) bank176318 +176319 POINT(41.63039158080427 73.11187109590924) bank176319 +176320 POINT(41.65237988268021 73.28598124960652) bank176320 +176321 POINT(41.46707944104852 73.05760770594378) bank176321 +176322 POINT(40.836553683804176 73.7961882971668) bank176322 +176323 POINT(41.10905689582785 74.89889408395823) bank176323 +176324 POINT(40.084598137209156 73.04094834150095) bank176324 +176325 POINT(40.09711588517503 73.8288149218754) bank176325 +176326 POINT(40.148106631603646 73.08831255922887) bank176326 +176327 POINT(40.52478550342514 74.6284183678319) bank176327 +176328 POINT(40.063668324885946 75.00391769670031) bank176328 +176329 POINT(40.37175547749655 73.02123551795808) bank176329 +176330 POINT(39.9385601902275 73.1665334773248) bank176330 +176331 POINT(40.3547754669599 74.79474271930238) bank176331 +176332 POINT(40.94313530402495 73.75852380711339) bank176332 +176333 POINT(40.85752254304279 74.58154926488449) bank176333 +176334 POINT(40.67942260672489 73.27492931547405) bank176334 +176335 POINT(41.06964527141992 74.33366574164383) bank176335 +176336 POINT(39.96431622265784 73.86913222451406) bank176336 +176337 POINT(41.1417176742121 74.93419084896925) bank176337 +176338 POINT(41.373488845201905 73.60702284509476) bank176338 +176339 POINT(40.52032190965582 73.14828880753188) bank176339 +176340 POINT(40.716221829427624 74.67748362613162) bank176340 +176341 POINT(41.59271966775177 73.18038433540019) bank176341 +176342 POINT(40.541668612658924 73.92758172510636) bank176342 +176343 POINT(41.51949185243213 73.35051312021399) bank176343 +176344 POINT(41.468052635252704 74.15566031089331) bank176344 +176345 POINT(39.7364785524065 73.46088132180947) bank176345 +176346 POINT(40.3265864397213 74.00718211907436) bank176346 +176347 POINT(41.02090432248581 73.69903994069406) bank176347 +176348 POINT(41.046984488882025 74.5233939752672) bank176348 +176349 POINT(40.22408179441339 73.88487242554966) bank176349 +176350 POINT(40.674558806960725 74.38422799723581) bank176350 +176351 POINT(41.69751027866415 73.16112079882978) bank176351 +176352 POINT(40.507493897187494 74.9932597674328) bank176352 +176353 POINT(40.291489688365374 74.74835815543456) bank176353 +176354 POINT(40.92696066734769 73.59159197180442) bank176354 +176355 POINT(41.38333675398531 74.02495781984379) bank176355 +176356 POINT(40.83140887641857 73.70925439073326) bank176356 +176357 POINT(39.86932195415135 73.27950960766742) bank176357 +176358 POINT(41.12996243639179 73.37181002234043) bank176358 +176359 POINT(40.041242778907794 73.68095307589337) bank176359 +176360 POINT(39.726890295472124 74.90421524055458) bank176360 +176361 POINT(41.20005208098432 74.99876289951413) bank176361 +176362 POINT(40.85963455908068 74.76424563174318) bank176362 +176363 POINT(41.49459539199058 73.13497146805985) bank176363 +176364 POINT(41.414668777029405 73.25232009819895) bank176364 +176365 POINT(41.23213358320161 74.28386896176964) bank176365 +176366 POINT(40.5950060302541 73.43831640211812) bank176366 +176367 POINT(40.96099577143821 74.97150917748938) bank176367 +176368 POINT(39.75307521536393 73.81237596420371) bank176368 +176369 POINT(40.027645106002595 74.93347564624828) bank176369 +176370 POINT(41.65452864927158 73.87743408842503) bank176370 +176371 POINT(40.283310272722204 73.89686246340821) bank176371 +176372 POINT(41.48732951152296 74.89249401568618) bank176372 +176373 POINT(41.54431432940161 73.46840896233793) bank176373 +176374 POINT(41.47458408376163 73.07472167801062) bank176374 +176375 POINT(41.62361155259623 74.03161517492578) bank176375 +176376 POINT(39.73499559374883 74.40908780295598) bank176376 +176377 POINT(39.909282698548594 74.28426354790959) bank176377 +176378 POINT(41.30405059152307 74.73982439902575) bank176378 +176379 POINT(40.985183270414524 73.75204080910017) bank176379 +176380 POINT(41.25954723323101 74.48241928370358) bank176380 +176381 POINT(40.381856098711275 74.1216062184918) bank176381 +176382 POINT(40.90805611192083 74.53310230844652) bank176382 +176383 POINT(40.07366705958579 73.86473788272257) bank176383 +176384 POINT(40.83109843282632 73.22734585113768) bank176384 +176385 POINT(40.38580908784249 74.86637924228839) bank176385 +176386 POINT(41.590952276012075 73.87269225680167) bank176386 +176387 POINT(41.43599099419183 74.80133583129916) bank176387 +176388 POINT(41.606060628112736 73.02520087779082) bank176388 +176389 POINT(40.217357808392165 73.14257506035888) bank176389 +176390 POINT(41.225645096510014 74.85696501493537) bank176390 +176391 POINT(41.700562985889505 74.90168725545263) bank176391 +176392 POINT(41.020092341905844 74.64364003005744) bank176392 +176393 POINT(40.36022987887698 74.49388165980115) bank176393 +176394 POINT(40.48049556830535 74.52119325062922) bank176394 +176395 POINT(41.202032854724244 73.81059994870706) bank176395 +176396 POINT(40.48221370890954 73.63719555214972) bank176396 +176397 POINT(41.48435173266535 73.1766366505729) bank176397 +176398 POINT(40.83899947714766 73.33014746511209) bank176398 +176399 POINT(40.81028481852921 74.9431256787698) bank176399 +176400 POINT(41.032361095076254 74.7720443774473) bank176400 +176401 POINT(41.31528721844298 73.44136456643176) bank176401 +176402 POINT(41.12206230428051 74.20358174540058) bank176402 +176403 POINT(39.78361506464182 74.92012264482526) bank176403 +176404 POINT(40.984103843140886 73.57988485869359) bank176404 +176405 POINT(39.84836321005356 73.0293994843954) bank176405 +176406 POINT(40.84678325384844 74.01342489719936) bank176406 +176407 POINT(40.820033034568986 73.6957963054122) bank176407 +176408 POINT(40.06329746436048 73.32555343742901) bank176408 +176409 POINT(41.021298592755414 74.94847246618984) bank176409 +176410 POINT(41.00501544362448 73.08487004758057) bank176410 +176411 POINT(41.463302003810355 73.34932553811927) bank176411 +176412 POINT(40.684629507047525 74.19416627432538) bank176412 +176413 POINT(40.26083303420784 73.40525648323873) bank176413 +176414 POINT(41.16269276456468 74.05887554484792) bank176414 +176415 POINT(40.703397312466564 74.1882287974869) bank176415 +176416 POINT(40.30580228644999 74.18655874056861) bank176416 +176417 POINT(40.86442851218098 73.35011880640921) bank176417 +176418 POINT(41.65049643976558 73.60790121446196) bank176418 +176419 POINT(40.95859170538005 74.09414266424882) bank176419 +176420 POINT(40.69496619261964 74.66956913516134) bank176420 +176421 POINT(40.97340859086483 74.65679380130402) bank176421 +176422 POINT(41.355712498325644 74.55353909004967) bank176422 +176423 POINT(41.63276082598166 74.60005922907443) bank176423 +176424 POINT(41.08816023716565 74.23792147515425) bank176424 +176425 POINT(40.17064737263913 74.85820951038751) bank176425 +176426 POINT(40.200858871700035 73.54661432001154) bank176426 +176427 POINT(39.842390231089276 73.16569505916398) bank176427 +176428 POINT(40.729711130076126 74.02059123468105) bank176428 +176429 POINT(40.214286925172 73.8677028174821) bank176429 +176430 POINT(40.74570034366842 74.56163302784319) bank176430 +176431 POINT(40.122735841278576 74.74108792078316) bank176431 +176432 POINT(40.82953005929998 73.72408489397675) bank176432 +176433 POINT(40.1186080938662 73.44383346723424) bank176433 +176434 POINT(39.9862191323349 73.63579175361234) bank176434 +176435 POINT(39.83126541792043 73.65082009783865) bank176435 +176436 POINT(39.745257889243675 73.07708802308103) bank176436 +176437 POINT(40.084008026573976 74.64343144109873) bank176437 +176438 POINT(40.78500435826131 74.05761738113206) bank176438 +176439 POINT(40.042515143826144 73.71113978650922) bank176439 +176440 POINT(40.76662342709089 74.69528355679812) bank176440 +176441 POINT(40.73519465464634 74.3586648192657) bank176441 +176442 POINT(41.3117744333102 73.65965074128219) bank176442 +176443 POINT(41.03358030032657 74.65697566091644) bank176443 +176444 POINT(40.70842445230358 73.74709432512283) bank176444 +176445 POINT(40.89328625506591 74.28710602490308) bank176445 +176446 POINT(39.997196978474 74.71820652601923) bank176446 +176447 POINT(40.564646754710004 74.84865259040762) bank176447 +176448 POINT(40.213926173908455 73.40220145200793) bank176448 +176449 POINT(41.291783788362594 74.10988134655585) bank176449 +176450 POINT(41.31832868400916 73.02499774899611) bank176450 +176451 POINT(39.731273675854354 73.57304248688307) bank176451 +176452 POINT(39.83069697385727 73.71001654249555) bank176452 +176453 POINT(40.99096721880474 74.96008573743028) bank176453 +176454 POINT(39.89733026537078 74.63993698575733) bank176454 +176455 POINT(41.428974780700834 73.16628109661067) bank176455 +176456 POINT(41.527254458155774 74.41895647252696) bank176456 +176457 POINT(41.047746662799085 74.29350069345323) bank176457 +176458 POINT(40.55116999528187 74.86646921621644) bank176458 +176459 POINT(40.65964780097715 74.11337354196924) bank176459 +176460 POINT(41.50117560661696 73.99326668240239) bank176460 +176461 POINT(41.6720608098413 74.53455958811094) bank176461 +176462 POINT(39.72118955355293 74.27215190903046) bank176462 +176463 POINT(39.89599615357693 73.28978067930178) bank176463 +176464 POINT(41.136820679539134 73.88449573044714) bank176464 +176465 POINT(41.26405166602297 73.4214137945283) bank176465 +176466 POINT(41.018028330910944 73.88069470129776) bank176466 +176467 POINT(40.69526996948556 73.889441519575) bank176467 +176468 POINT(40.053059242310965 73.9830799400499) bank176468 +176469 POINT(40.84218924352805 74.43906165691936) bank176469 +176470 POINT(41.46350048363518 73.1413015625948) bank176470 +176471 POINT(40.81682763283971 73.89277549433196) bank176471 +176472 POINT(39.76861838691938 74.94093202690131) bank176472 +176473 POINT(40.81958251584209 74.84184502153349) bank176473 +176474 POINT(40.1201741658926 73.32562715423236) bank176474 +176475 POINT(41.4608719618369 74.64708432988436) bank176475 +176476 POINT(41.03339274520196 73.99040602601421) bank176476 +176477 POINT(41.569581888309855 73.57166089523263) bank176477 +176478 POINT(40.33432237345351 74.05573619179172) bank176478 +176479 POINT(41.21212229033353 73.52431997970854) bank176479 +176480 POINT(41.50795455982582 73.99000320653154) bank176480 +176481 POINT(41.25471647592396 73.39474568809796) bank176481 +176482 POINT(40.10881579987132 74.15201683348423) bank176482 +176483 POINT(40.92813482189729 73.40721573984453) bank176483 +176484 POINT(40.0284204108176 74.93072112833353) bank176484 +176485 POINT(41.14109353080208 74.67888234098474) bank176485 +176486 POINT(39.979522234345936 73.58945881853593) bank176486 +176487 POINT(41.61991533093465 73.25850921161782) bank176487 +176488 POINT(40.29574166858072 73.56717714388093) bank176488 +176489 POINT(41.34607213154402 73.54462300932033) bank176489 +176490 POINT(40.25841853907402 74.38241059675167) bank176490 +176491 POINT(40.42118510089814 74.45880841826506) bank176491 +176492 POINT(39.95019842314757 73.03469994413312) bank176492 +176493 POINT(41.06301549272183 73.61295634415313) bank176493 +176494 POINT(41.02250189027325 73.02586735933365) bank176494 +176495 POINT(41.350774590004995 75.00257001041369) bank176495 +176496 POINT(39.96554613027319 73.27182551971198) bank176496 +176497 POINT(41.1169824412615 74.15837391453607) bank176497 +176498 POINT(41.69854534893422 74.18233047172964) bank176498 +176499 POINT(41.073254843703424 74.0567410485633) bank176499 +176500 POINT(39.82070366641737 73.16148154751754) bank176500 +176501 POINT(39.94770057434641 74.92879753868449) bank176501 +176502 POINT(40.50089078530184 74.14741173285417) bank176502 +176503 POINT(40.32756264904279 74.6966638391588) bank176503 +176504 POINT(40.33948238686898 74.42062792974787) bank176504 +176505 POINT(39.804927968353184 73.15780711491331) bank176505 +176506 POINT(40.58463719748554 73.60680181398351) bank176506 +176507 POINT(40.53148418049982 74.9534821490337) bank176507 +176508 POINT(40.28720075679631 74.18734482753261) bank176508 +176509 POINT(41.19130307378298 74.76471419662359) bank176509 +176510 POINT(40.561245859192354 73.32356330254132) bank176510 +176511 POINT(41.61558250885623 73.08222947206025) bank176511 +176512 POINT(40.903723572341825 74.52371457425805) bank176512 +176513 POINT(41.62072680579787 74.95074118581617) bank176513 +176514 POINT(40.36435098000875 74.30923442782381) bank176514 +176515 POINT(41.18469723584684 73.97604905463972) bank176515 +176516 POINT(40.703297484593 73.14074851378643) bank176516 +176517 POINT(41.28154698411505 73.8945587630612) bank176517 +176518 POINT(39.90531960016786 73.07702566463486) bank176518 +176519 POINT(40.10742460927184 74.63516272094132) bank176519 +176520 POINT(41.02841832300716 74.85271752316382) bank176520 +176521 POINT(40.65731834630458 73.90610566352171) bank176521 +176522 POINT(41.45039001960595 73.24502415184334) bank176522 +176523 POINT(39.71329595292676 74.16666210315393) bank176523 +176524 POINT(40.24016379317071 74.80435117260727) bank176524 +176525 POINT(40.96224813866749 74.76370158193917) bank176525 +176526 POINT(40.88503213745267 74.6573213296922) bank176526 +176527 POINT(41.529433870053026 74.27871350540644) bank176527 +176528 POINT(40.61861435769756 74.69873328752028) bank176528 +176529 POINT(41.002168727209124 74.68806950934658) bank176529 +176530 POINT(41.058472912061795 74.93749839121776) bank176530 +176531 POINT(39.90775214963455 73.33624318461379) bank176531 +176532 POINT(41.70839429222549 74.5268065653042) bank176532 +176533 POINT(40.704538452727014 73.88769424324718) bank176533 +176534 POINT(40.202701809935974 74.84110808690758) bank176534 +176535 POINT(39.73958272626781 73.820602978673) bank176535 +176536 POINT(39.771258520000224 74.75709872573957) bank176536 +176537 POINT(40.37931238886543 73.35200295855954) bank176537 +176538 POINT(40.36692139899452 73.90698964767218) bank176538 +176539 POINT(41.35586017838654 74.26838434253854) bank176539 +176540 POINT(40.79995944012999 74.72297438620497) bank176540 +176541 POINT(39.76254164657262 73.72231901172273) bank176541 +176542 POINT(41.105429047456056 74.24364148201029) bank176542 +176543 POINT(40.51901164878757 73.8554118523418) bank176543 +176544 POINT(41.244926557177756 73.95307236897288) bank176544 +176545 POINT(40.39670243633307 74.94032148314254) bank176545 +176546 POINT(39.871277882660635 73.26625391923353) bank176546 +176547 POINT(39.89408485767926 73.7867013073917) bank176547 +176548 POINT(40.03980227075754 74.20049128611248) bank176548 +176549 POINT(41.310950571251404 74.88250742272658) bank176549 +176550 POINT(39.94997150366935 73.347966303161) bank176550 +176551 POINT(41.294089877196576 74.19835479598078) bank176551 +176552 POINT(40.18071724490941 73.72353176366302) bank176552 +176553 POINT(40.601693381262706 74.0096151618577) bank176553 +176554 POINT(41.58027804046946 73.63007184594416) bank176554 +176555 POINT(40.10483096212166 73.34770811465513) bank176555 +176556 POINT(40.27253835520034 74.82918766457506) bank176556 +176557 POINT(40.332937156650196 74.64474419291405) bank176557 +176558 POINT(41.61088174240201 73.66409661446012) bank176558 +176559 POINT(40.934440531250445 74.12970364287649) bank176559 +176560 POINT(40.46226103949891 74.77039341693985) bank176560 +176561 POINT(41.66951097523325 74.65308985473835) bank176561 +176562 POINT(40.031599593057734 74.61112080973726) bank176562 +176563 POINT(41.53083468091711 73.5787647848596) bank176563 +176564 POINT(39.85172564745644 74.47480789252748) bank176564 +176565 POINT(41.063237711620786 74.22285370294782) bank176565 +176566 POINT(41.18953082464462 73.09017507533675) bank176566 +176567 POINT(40.20132224125414 73.45100128931819) bank176567 +176568 POINT(40.69501129177237 73.1688453152495) bank176568 +176569 POINT(41.54118644695789 74.38195316670986) bank176569 +176570 POINT(40.199899794923915 74.93183530814967) bank176570 +176571 POINT(40.06307238924269 73.28344375207297) bank176571 +176572 POINT(41.20474963581824 74.25406304343885) bank176572 +176573 POINT(40.08339161665107 73.89357097140362) bank176573 +176574 POINT(40.54865884177112 74.49507316097397) bank176574 +176575 POINT(40.054183046141496 74.88801148457048) bank176575 +176576 POINT(40.10272415763402 74.11190470941347) bank176576 +176577 POINT(41.197032378014555 74.74096063969688) bank176577 +176578 POINT(41.0081630185239 73.56495715836898) bank176578 +176579 POINT(39.81640922438032 73.69410719624481) bank176579 +176580 POINT(40.95637236089148 74.59534567489229) bank176580 +176581 POINT(39.972587988414446 73.45987160015248) bank176581 +176582 POINT(39.87783122358176 73.4500645957002) bank176582 +176583 POINT(41.189453201217056 73.25091096480584) bank176583 +176584 POINT(41.29518266780139 74.58176693637326) bank176584 +176585 POINT(40.932529032190864 74.99780268198516) bank176585 +176586 POINT(41.12202170625018 73.28454063063448) bank176586 +176587 POINT(40.83656711519674 73.52213593241636) bank176587 +176588 POINT(40.517976942603696 73.88970027099815) bank176588 +176589 POINT(40.36405487107867 74.67047246901602) bank176589 +176590 POINT(40.908303586485964 73.71615558653903) bank176590 +176591 POINT(40.78870186857036 74.3484938833764) bank176591 +176592 POINT(41.46787333422909 73.45348896790087) bank176592 +176593 POINT(41.52456028884548 74.13100651232712) bank176593 +176594 POINT(40.480942548236634 73.99437709106803) bank176594 +176595 POINT(41.561534595464764 73.99314564757216) bank176595 +176596 POINT(40.97848304508405 74.6239766642853) bank176596 +176597 POINT(41.0088824998126 74.72994115429084) bank176597 +176598 POINT(39.71418783823754 73.93461357027203) bank176598 +176599 POINT(40.05795157390092 73.59306981200304) bank176599 +176600 POINT(40.23862625425515 74.67704141825257) bank176600 +176601 POINT(41.08576469536767 73.49788433447962) bank176601 +176602 POINT(40.59402022323876 74.60116801002151) bank176602 +176603 POINT(40.30647809299279 73.75848183748488) bank176603 +176604 POINT(40.07411877768308 74.14555401413685) bank176604 +176605 POINT(40.638412996700936 73.27552256717885) bank176605 +176606 POINT(40.44458079511534 74.33617027352113) bank176606 +176607 POINT(41.038340088112946 74.71502861406626) bank176607 +176608 POINT(40.18190556494967 73.16149058696121) bank176608 +176609 POINT(40.65844846144229 73.72629214301794) bank176609 +176610 POINT(39.73711314212411 74.848796895319) bank176610 +176611 POINT(40.40042266574057 73.53587466383495) bank176611 +176612 POINT(40.86515042749738 74.21191101528281) bank176612 +176613 POINT(39.80657286936272 74.1584538640833) bank176613 +176614 POINT(40.74029150896422 73.30621722905556) bank176614 +176615 POINT(41.461754683209826 74.75102101833197) bank176615 +176616 POINT(39.95690152657419 74.71167374759781) bank176616 +176617 POINT(41.54413214138068 73.56742762275609) bank176617 +176618 POINT(40.594856265210794 73.59824606682288) bank176618 +176619 POINT(40.41660879429775 73.36131021871144) bank176619 +176620 POINT(39.92922476218791 73.58400250997404) bank176620 +176621 POINT(41.24258857676932 73.26845960789345) bank176621 +176622 POINT(39.8293519173563 74.58676240981023) bank176622 +176623 POINT(40.178042425564904 74.78255965231074) bank176623 +176624 POINT(39.8727390671302 73.48387979049544) bank176624 +176625 POINT(40.825515744962736 74.14741115855593) bank176625 +176626 POINT(40.00900611874144 74.3618811466701) bank176626 +176627 POINT(41.66318769897592 74.03822711713619) bank176627 +176628 POINT(41.65198959358754 74.72222950533221) bank176628 +176629 POINT(40.54947373277144 73.99705974284463) bank176629 +176630 POINT(40.300871486722386 73.5159071224732) bank176630 +176631 POINT(40.16521598290887 73.4736733004805) bank176631 +176632 POINT(41.00847360818834 74.9413176212714) bank176632 +176633 POINT(41.318160160854184 73.45152808507628) bank176633 +176634 POINT(40.395828381983826 73.27989893173522) bank176634 +176635 POINT(40.593122278584026 73.3601936829683) bank176635 +176636 POINT(39.98656933120099 74.71548457833491) bank176636 +176637 POINT(40.105716548301366 74.3227219861935) bank176637 +176638 POINT(40.72420037338775 74.31778107282749) bank176638 +176639 POINT(40.189452441569095 73.80564669329873) bank176639 +176640 POINT(39.77818342296279 73.97670832957664) bank176640 +176641 POINT(41.30415504266524 73.67156925240663) bank176641 +176642 POINT(40.681919842250906 74.92135390662122) bank176642 +176643 POINT(40.75855979802438 74.64788894578906) bank176643 +176644 POINT(41.462776477114346 74.43681824447513) bank176644 +176645 POINT(40.88616955346739 73.49900458092132) bank176645 +176646 POINT(41.189577309024884 73.01672719046115) bank176646 +176647 POINT(40.424521493568 73.93651035450418) bank176647 +176648 POINT(39.74638437504227 73.22388444494625) bank176648 +176649 POINT(40.324955439535955 74.40020998354781) bank176649 +176650 POINT(40.84076830510412 74.02375603260467) bank176650 +176651 POINT(41.18650947055222 74.47896525850372) bank176651 +176652 POINT(41.20315182925666 74.6174148494521) bank176652 +176653 POINT(39.884427515234876 73.78180650277343) bank176653 +176654 POINT(40.325928511957976 73.22622579263225) bank176654 +176655 POINT(40.95536610195286 74.2465301747419) bank176655 +176656 POINT(41.574276117816474 73.29869659008823) bank176656 +176657 POINT(40.5381371123828 74.92081139062581) bank176657 +176658 POINT(40.89653045789661 74.04677358140373) bank176658 +176659 POINT(41.41322337028705 74.11282223685195) bank176659 +176660 POINT(40.47617962252223 73.16764805526398) bank176660 +176661 POINT(41.47013494057678 74.59890001704399) bank176661 +176662 POINT(41.64971181645781 74.43661129177076) bank176662 +176663 POINT(40.58428759935841 73.65608573613649) bank176663 +176664 POINT(41.01099797541793 74.9168221965027) bank176664 +176665 POINT(40.70466020942177 74.83729057215801) bank176665 +176666 POINT(40.719750763117254 73.148009732829) bank176666 +176667 POINT(41.266501332882974 74.63749724745004) bank176667 +176668 POINT(40.19765837657975 74.5366649529227) bank176668 +176669 POINT(41.16406085722505 74.27506865969882) bank176669 +176670 POINT(40.912350317498316 74.64191767046368) bank176670 +176671 POINT(41.637747896377455 74.6955185947756) bank176671 +176672 POINT(40.10944843005164 73.85802900850018) bank176672 +176673 POINT(40.8890889045544 73.07338601365858) bank176673 +176674 POINT(41.52980888881907 73.67267402754075) bank176674 +176675 POINT(40.53248663825414 74.38283723633937) bank176675 +176676 POINT(39.845654759565974 74.35229601706888) bank176676 +176677 POINT(40.01453826711097 74.77257649325777) bank176677 +176678 POINT(41.3277732241032 74.88701348028935) bank176678 +176679 POINT(41.33480029524523 74.36683628284324) bank176679 +176680 POINT(41.35280258385552 74.35127365167548) bank176680 +176681 POINT(41.182024687167605 73.56151880516651) bank176681 +176682 POINT(40.824850962578154 73.14938598635615) bank176682 +176683 POINT(40.039632912331726 73.97304096668121) bank176683 +176684 POINT(40.238720015214014 74.8825504448145) bank176684 +176685 POINT(40.218210080311714 73.3207663436519) bank176685 +176686 POINT(39.99657590736068 74.47036041620755) bank176686 +176687 POINT(40.35814541415491 73.95022577959364) bank176687 +176688 POINT(40.33624496067441 73.15976250525993) bank176688 +176689 POINT(40.74863764490617 74.8927414682637) bank176689 +176690 POINT(41.06437475478641 73.95454377364278) bank176690 +176691 POINT(39.98375812224679 74.62013764452452) bank176691 +176692 POINT(41.13356148032762 73.78525598948065) bank176692 +176693 POINT(39.82311077479186 73.41979610330517) bank176693 +176694 POINT(40.09753081152244 74.79980543101792) bank176694 +176695 POINT(40.52093372969619 75.00325512202238) bank176695 +176696 POINT(41.65299164147601 74.5450140475748) bank176696 +176697 POINT(41.06355070419036 74.41240508088359) bank176697 +176698 POINT(40.25456712643885 74.7543945265033) bank176698 +176699 POINT(40.39635501773643 74.29780658804742) bank176699 +176700 POINT(39.856757239181476 74.6249211362525) bank176700 +176701 POINT(40.923067894192236 73.64725549862521) bank176701 +176702 POINT(41.712674550033576 73.2354024946784) bank176702 +176703 POINT(40.295248972156934 74.16938550515306) bank176703 +176704 POINT(40.66142984908041 74.6711572797934) bank176704 +176705 POINT(40.45133595823757 74.53510826775779) bank176705 +176706 POINT(41.4829834330662 74.13049673315234) bank176706 +176707 POINT(41.55957614415307 74.25908995068555) bank176707 +176708 POINT(41.06057153356006 73.88333941079698) bank176708 +176709 POINT(40.14544171400805 73.25564251822593) bank176709 +176710 POINT(40.747665294723184 74.50184308996057) bank176710 +176711 POINT(41.30518125395911 73.71217653129406) bank176711 +176712 POINT(41.29242328415341 74.23659920771738) bank176712 +176713 POINT(40.99453011320906 74.0832891565452) bank176713 +176714 POINT(41.02338272226428 73.11333687309904) bank176714 +176715 POINT(41.33518667613087 73.04974962552421) bank176715 +176716 POINT(41.47142514917517 74.90560742026052) bank176716 +176717 POINT(40.63631508849688 74.94695021298419) bank176717 +176718 POINT(40.493065866850756 74.00043389969707) bank176718 +176719 POINT(41.32444312353527 73.53751371037782) bank176719 +176720 POINT(40.84571963645268 73.96863224121668) bank176720 +176721 POINT(40.00701203560266 73.47457328710261) bank176721 +176722 POINT(41.18837781877098 74.15387783871044) bank176722 +176723 POINT(40.87760728778516 73.81478772625893) bank176723 +176724 POINT(41.10141021922361 74.25667153285525) bank176724 +176725 POINT(40.38176119574338 73.13858370472171) bank176725 +176726 POINT(40.542688867008366 74.88979419893016) bank176726 +176727 POINT(40.27433510243968 74.80695242170961) bank176727 +176728 POINT(40.03272228690446 74.33976568867391) bank176728 +176729 POINT(39.765286909660816 73.27215014701427) bank176729 +176730 POINT(41.61264426921933 73.29188078406771) bank176730 +176731 POINT(40.25928926916401 73.57548330935211) bank176731 +176732 POINT(40.561486745098335 73.94492153888372) bank176732 +176733 POINT(40.65698644306128 74.36318614304146) bank176733 +176734 POINT(39.93130099310486 74.97137300088951) bank176734 +176735 POINT(39.98016491109854 73.88727209162985) bank176735 +176736 POINT(39.94084147394724 73.54554868986959) bank176736 +176737 POINT(41.60397537584052 73.89115713173527) bank176737 +176738 POINT(40.07070475753537 74.46347029724922) bank176738 +176739 POINT(41.610160331878575 74.29025484384593) bank176739 +176740 POINT(41.108918943196414 74.91374277312084) bank176740 +176741 POINT(39.75762741290497 74.67408487763652) bank176741 +176742 POINT(41.2605009727445 74.31056763868712) bank176742 +176743 POINT(40.60851481951304 73.70158516992049) bank176743 +176744 POINT(41.341767481318925 73.23721575286098) bank176744 +176745 POINT(40.37154649026496 73.52654600763556) bank176745 +176746 POINT(39.80566733147553 73.96965078910124) bank176746 +176747 POINT(41.49588832112741 73.51879627809664) bank176747 +176748 POINT(40.9253458595403 74.34505469153866) bank176748 +176749 POINT(40.32869713735674 74.57803750377454) bank176749 +176750 POINT(40.86258539620207 73.73068621989869) bank176750 +176751 POINT(40.83423128602971 73.58162290394574) bank176751 +176752 POINT(41.629974443459574 74.06835124685814) bank176752 +176753 POINT(41.0776141112119 73.20420921437187) bank176753 +176754 POINT(40.405435839959985 73.96018465112576) bank176754 +176755 POINT(40.71466852812752 74.38202514550923) bank176755 +176756 POINT(40.6863181236039 73.53812101691655) bank176756 +176757 POINT(40.365849158502115 73.03707575700817) bank176757 +176758 POINT(40.53956075936284 73.88810504522091) bank176758 +176759 POINT(41.14311129530053 74.90005652011526) bank176759 +176760 POINT(40.593068755326826 74.47951391855635) bank176760 +176761 POINT(41.710859767158894 74.41737553388842) bank176761 +176762 POINT(40.43775619183531 74.32894190162015) bank176762 +176763 POINT(41.313592062741535 74.95599202650511) bank176763 +176764 POINT(41.13888744526372 73.31284792066033) bank176764 +176765 POINT(41.19670721619315 74.04273938961038) bank176765 +176766 POINT(39.979967872136754 73.49463229603003) bank176766 +176767 POINT(40.811175721525224 74.0816027609993) bank176767 +176768 POINT(40.03522203600536 73.11425344701208) bank176768 +176769 POINT(41.16153940502086 73.39583518681779) bank176769 +176770 POINT(40.678608967983344 73.51403314936023) bank176770 +176771 POINT(39.764434062521694 74.25234312526214) bank176771 +176772 POINT(40.88800804571571 73.50289727561884) bank176772 +176773 POINT(40.97795052715569 74.747431654757) bank176773 +176774 POINT(40.393950699376 74.38747424193443) bank176774 +176775 POINT(40.951946910845585 73.75546692945886) bank176775 +176776 POINT(41.13314540062142 74.69141291208695) bank176776 +176777 POINT(41.18012141568429 73.35989038211528) bank176777 +176778 POINT(40.087656070273866 73.0794245321453) bank176778 +176779 POINT(40.13799305853478 74.61168654474659) bank176779 +176780 POINT(39.91506043250812 74.79869564931384) bank176780 +176781 POINT(40.91479622204631 73.12874636122281) bank176781 +176782 POINT(41.67256917367164 73.79366298637939) bank176782 +176783 POINT(40.20646794224477 73.69323720991068) bank176783 +176784 POINT(40.79296136565829 73.70700517160572) bank176784 +176785 POINT(41.71076415873001 73.04376798662162) bank176785 +176786 POINT(39.98276535757681 74.14361463129109) bank176786 +176787 POINT(41.68103873394795 73.5990124011566) bank176787 +176788 POINT(40.85365068216939 73.7048959199437) bank176788 +176789 POINT(40.73295016440192 73.82711988127085) bank176789 +176790 POINT(39.94906640408176 74.67410625304309) bank176790 +176791 POINT(39.999552207427584 74.36442375912878) bank176791 +176792 POINT(41.44833395734247 74.22730730181351) bank176792 +176793 POINT(40.306206876372656 73.06352144444449) bank176793 +176794 POINT(40.41974586292943 74.59511357906727) bank176794 +176795 POINT(41.093036881397744 73.61758363533056) bank176795 +176796 POINT(39.86821690109005 73.88673494565028) bank176796 +176797 POINT(41.02632366638277 74.90067815213364) bank176797 +176798 POINT(40.7498924696977 74.72821271764913) bank176798 +176799 POINT(41.192922274220166 74.35971560823823) bank176799 +176800 POINT(41.61518956956833 73.54985194738856) bank176800 +176801 POINT(40.693912121404836 73.04016943437732) bank176801 +176802 POINT(40.786104602961906 73.53626322876461) bank176802 +176803 POINT(39.8414010324402 74.01139843429698) bank176803 +176804 POINT(40.648238613103274 73.08249637170526) bank176804 +176805 POINT(41.43616462154493 73.43864537758925) bank176805 +176806 POINT(41.091235453288924 73.03251856495032) bank176806 +176807 POINT(40.767490534618396 73.52576695241346) bank176807 +176808 POINT(41.30152459783911 74.34676134674847) bank176808 +176809 POINT(40.97896532189379 73.99254646999604) bank176809 +176810 POINT(41.09619421366596 73.97337972317882) bank176810 +176811 POINT(41.16937249162431 73.4855078167564) bank176811 +176812 POINT(41.43762379145811 74.31246656814885) bank176812 +176813 POINT(40.87820355793874 73.67180881796001) bank176813 +176814 POINT(41.35683987793944 73.01371633492117) bank176814 +176815 POINT(39.741884760350274 74.00478250281395) bank176815 +176816 POINT(41.65547472989089 73.8036867847014) bank176816 +176817 POINT(40.88183021656539 74.40662311048814) bank176817 +176818 POINT(40.769807944257714 74.24119271915735) bank176818 +176819 POINT(41.369152830243166 74.28587313211852) bank176819 +176820 POINT(40.4867703399409 74.44161902214695) bank176820 +176821 POINT(41.56662657849163 74.5265423881655) bank176821 +176822 POINT(41.475217233703454 74.25157976355517) bank176822 +176823 POINT(41.46583892848884 73.05764348608969) bank176823 +176824 POINT(40.41462603173249 73.14139047623877) bank176824 +176825 POINT(41.63607199568416 73.05323154433837) bank176825 +176826 POINT(40.67555221709152 73.26463811323691) bank176826 +176827 POINT(40.774620788322736 73.53021500555883) bank176827 +176828 POINT(40.58459414935721 74.11081787816507) bank176828 +176829 POINT(39.92623291368713 73.02807236763913) bank176829 +176830 POINT(40.63604777430511 73.35522480350339) bank176830 +176831 POINT(40.57089768478732 73.51205911850514) bank176831 +176832 POINT(41.03198153007538 73.31201102506404) bank176832 +176833 POINT(41.70806652364725 73.38995385710405) bank176833 +176834 POINT(40.5514339567278 74.19721042115827) bank176834 +176835 POINT(40.27459148306193 73.9627470786347) bank176835 +176836 POINT(41.178831983954936 73.83823567790508) bank176836 +176837 POINT(40.76442576023044 74.03919691204953) bank176837 +176838 POINT(40.440620862202884 74.39339723480973) bank176838 +176839 POINT(41.44796807717406 73.5706568072427) bank176839 +176840 POINT(40.12884962267248 73.36167919354381) bank176840 +176841 POINT(40.49733001606998 74.28417768472724) bank176841 +176842 POINT(41.45697489912391 74.8676040967429) bank176842 +176843 POINT(40.46320884482066 74.85283910209638) bank176843 +176844 POINT(41.19138894148216 73.61294991533558) bank176844 +176845 POINT(41.428510047512674 73.90947414372042) bank176845 +176846 POINT(40.41164973526999 74.36701610440113) bank176846 +176847 POINT(41.00184625712963 74.54511670191145) bank176847 +176848 POINT(40.87658949471886 73.31724427765153) bank176848 +176849 POINT(40.407324318908934 73.68084803037652) bank176849 +176850 POINT(40.91035494425315 73.64451359510127) bank176850 +176851 POINT(40.88039499209441 73.69675136672522) bank176851 +176852 POINT(40.73098378310568 74.7424515307296) bank176852 +176853 POINT(40.019777329688864 73.99301638737006) bank176853 +176854 POINT(40.77664853378676 74.23814592512726) bank176854 +176855 POINT(39.809755258251556 74.47130065908839) bank176855 +176856 POINT(41.2745365882043 73.7439105415602) bank176856 +176857 POINT(40.28962391897995 73.46864537120348) bank176857 +176858 POINT(40.46956552922891 73.40831651732232) bank176858 +176859 POINT(39.71823248653026 74.70591263724187) bank176859 +176860 POINT(41.62418507934509 74.27643576285313) bank176860 +176861 POINT(40.63127516089473 74.191791298864) bank176861 +176862 POINT(40.33984381493845 73.04531798675875) bank176862 +176863 POINT(41.09506598605987 74.28332902107067) bank176863 +176864 POINT(41.58306959758674 74.09178520844661) bank176864 +176865 POINT(41.52419052668479 74.85672667052091) bank176865 +176866 POINT(41.66714042076482 74.87726059983044) bank176866 +176867 POINT(39.97925426158161 73.4013393227101) bank176867 +176868 POINT(41.19296696888948 73.59529881441217) bank176868 +176869 POINT(39.834556285249604 74.84421116050646) bank176869 +176870 POINT(40.255401978976316 73.28421056275458) bank176870 +176871 POINT(40.761544512466 74.65357317858529) bank176871 +176872 POINT(40.62664544946386 73.16346989872373) bank176872 +176873 POINT(40.796245894647605 74.42744047012472) bank176873 +176874 POINT(40.58265988329908 73.82876979705934) bank176874 +176875 POINT(41.52184826635173 73.85185148929966) bank176875 +176876 POINT(39.718236489848486 73.55214177700665) bank176876 +176877 POINT(41.063375709292814 73.02343338243817) bank176877 +176878 POINT(40.14757773927512 73.60100043677326) bank176878 +176879 POINT(40.602706661170195 73.38914132032578) bank176879 +176880 POINT(40.2903255078779 74.78787347620471) bank176880 +176881 POINT(41.181147637082574 73.71164674888188) bank176881 +176882 POINT(41.024010186561526 73.371686320977) bank176882 +176883 POINT(40.386246966844276 74.46829719578443) bank176883 +176884 POINT(41.5474979591235 74.86723628944013) bank176884 +176885 POINT(41.647561093395446 73.25253858582492) bank176885 +176886 POINT(41.47806955540286 74.45392692046231) bank176886 +176887 POINT(40.24020383600218 74.88482382948973) bank176887 +176888 POINT(40.914169353776764 74.06725700434747) bank176888 +176889 POINT(40.95320615821079 74.72702128637577) bank176889 +176890 POINT(40.675951387689935 74.36800683955362) bank176890 +176891 POINT(41.04123172092939 74.46867551652416) bank176891 +176892 POINT(40.89886950560188 73.99451850593202) bank176892 +176893 POINT(41.2490363179775 74.75355740408324) bank176893 +176894 POINT(40.06711605228128 74.74763521596702) bank176894 +176895 POINT(40.49931127911839 73.27890252141098) bank176895 +176896 POINT(40.25763293316514 74.5148704047543) bank176896 +176897 POINT(40.278798940824416 74.21746169512274) bank176897 +176898 POINT(41.615720189381314 73.15177853241266) bank176898 +176899 POINT(39.83042232302276 74.81436504029229) bank176899 +176900 POINT(41.46513684678953 73.12126540242583) bank176900 +176901 POINT(40.469204483740896 74.55588536007282) bank176901 +176902 POINT(40.38478830931314 74.05738758705596) bank176902 +176903 POINT(41.13629759032496 74.81954661734174) bank176903 +176904 POINT(41.078302640618844 74.75501944658832) bank176904 +176905 POINT(40.505574648139934 74.57694533991152) bank176905 +176906 POINT(40.31472513981594 74.14943655101973) bank176906 +176907 POINT(41.512834748828865 74.05941233506798) bank176907 +176908 POINT(41.543884284182774 74.39067635274112) bank176908 +176909 POINT(40.30270480028 74.33115228113039) bank176909 +176910 POINT(41.05435160578003 73.17911941850598) bank176910 +176911 POINT(40.70575026439257 73.3752248926677) bank176911 +176912 POINT(40.922336435771946 73.42656619146439) bank176912 +176913 POINT(40.526138529914704 74.10698196137332) bank176913 +176914 POINT(41.05993259241788 73.28933058149089) bank176914 +176915 POINT(41.020746233739196 74.92521637564927) bank176915 +176916 POINT(41.08218343154834 74.43588003699637) bank176916 +176917 POINT(41.62999074986033 74.59310310879857) bank176917 +176918 POINT(40.83808996927552 73.70734267569408) bank176918 +176919 POINT(40.17646636429955 74.93419067104807) bank176919 +176920 POINT(40.5891849483916 73.72711359912577) bank176920 +176921 POINT(40.642969512393265 73.006094362121) bank176921 +176922 POINT(41.6536407486258 74.81067768793551) bank176922 +176923 POINT(40.217386991167714 74.94045835783231) bank176923 +176924 POINT(40.569827362264675 73.55001092603214) bank176924 +176925 POINT(39.893348088356774 73.25214297561706) bank176925 +176926 POINT(40.40181851129934 73.24622355543316) bank176926 +176927 POINT(40.58214694153101 73.46255550665586) bank176927 +176928 POINT(39.91795145230569 73.82803261459411) bank176928 +176929 POINT(39.98263400327104 73.74032764401835) bank176929 +176930 POINT(40.61963548907364 74.65559936566324) bank176930 +176931 POINT(39.77630968555111 73.913364982235) bank176931 +176932 POINT(40.47955687667793 73.50201511556544) bank176932 +176933 POINT(40.82864360850818 73.50947802553353) bank176933 +176934 POINT(40.94612469625826 73.5061768882756) bank176934 +176935 POINT(40.1482784436614 74.3923403646083) bank176935 +176936 POINT(40.00287120295925 74.40566862198291) bank176936 +176937 POINT(41.54228565923328 73.82381437485763) bank176937 +176938 POINT(39.931353053692426 74.99754823065115) bank176938 +176939 POINT(41.681072421912916 74.09995175650367) bank176939 +176940 POINT(40.68683752682468 74.34393498627762) bank176940 +176941 POINT(40.76582602028172 73.98040560458354) bank176941 +176942 POINT(41.31846811118278 74.31529806150044) bank176942 +176943 POINT(39.85752760388488 73.48289956301457) bank176943 +176944 POINT(40.89012527339482 73.3819834790165) bank176944 +176945 POINT(40.370792884163905 73.9833186519633) bank176945 +176946 POINT(41.125486549208006 73.25304017303345) bank176946 +176947 POINT(40.74173287076317 73.48777143786566) bank176947 +176948 POINT(40.51751532576018 73.06711515488152) bank176948 +176949 POINT(41.39518279015958 74.59124439591783) bank176949 +176950 POINT(40.14068423397274 74.93684264182222) bank176950 +176951 POINT(40.537875874277844 73.85187620551082) bank176951 +176952 POINT(39.83757147180501 74.49424406456295) bank176952 +176953 POINT(40.15012844266263 73.74258352736484) bank176953 +176954 POINT(41.5738460834978 73.46081806464764) bank176954 +176955 POINT(41.03119089817944 73.28718538205197) bank176955 +176956 POINT(40.37722505738184 73.25079821832203) bank176956 +176957 POINT(41.0575065606547 74.50998987175292) bank176957 +176958 POINT(41.06233683884176 74.99591403518674) bank176958 +176959 POINT(41.02944363669916 74.9675947556685) bank176959 +176960 POINT(40.912724483674786 73.30278597059907) bank176960 +176961 POINT(40.69929574771907 74.76644696849087) bank176961 +176962 POINT(39.890424741718036 73.93188984702569) bank176962 +176963 POINT(39.7631742909228 73.32605771128922) bank176963 +176964 POINT(39.88835148053509 73.614508158364) bank176964 +176965 POINT(41.158718942590305 74.75988863203624) bank176965 +176966 POINT(40.784127720653636 74.50831559702343) bank176966 +176967 POINT(40.19609840338526 74.81309218592146) bank176967 +176968 POINT(40.01246787800372 73.64602709631288) bank176968 +176969 POINT(40.25502604400083 73.70703015878102) bank176969 +176970 POINT(40.80221546636266 73.04430090729328) bank176970 +176971 POINT(40.43034162511084 73.0888455793489) bank176971 +176972 POINT(41.457057785394824 74.00577961317788) bank176972 +176973 POINT(41.24339852232183 74.30172872064789) bank176973 +176974 POINT(40.45782784175486 74.88430568379324) bank176974 +176975 POINT(39.85372837143825 73.31528667351539) bank176975 +176976 POINT(41.54110891342265 74.61164036380877) bank176976 +176977 POINT(40.72759973528808 73.61194887354364) bank176977 +176978 POINT(41.465847286146946 74.88268975993955) bank176978 +176979 POINT(41.00430927278877 74.42699644585059) bank176979 +176980 POINT(40.75166449648666 73.2639887486666) bank176980 +176981 POINT(39.84392128376554 74.14178007277276) bank176981 +176982 POINT(40.23676721345676 73.31431995977573) bank176982 +176983 POINT(40.752656717154835 73.06205000493539) bank176983 +176984 POINT(41.47861755255137 73.09293949227293) bank176984 +176985 POINT(41.53972425626483 74.48819060185664) bank176985 +176986 POINT(41.25111127575611 73.64315265224026) bank176986 +176987 POINT(40.65474413622902 73.37277114952852) bank176987 +176988 POINT(41.46538438838511 74.97404449778027) bank176988 +176989 POINT(39.74239444019759 74.39729151652412) bank176989 +176990 POINT(39.78161689285879 74.5818537924318) bank176990 +176991 POINT(40.68135086888407 73.77851253168471) bank176991 +176992 POINT(40.58484233229654 73.7378527944436) bank176992 +176993 POINT(40.620172645451156 73.13175322245672) bank176993 +176994 POINT(41.373930116003145 73.66895883121305) bank176994 +176995 POINT(41.40032692533614 74.23258457602797) bank176995 +176996 POINT(41.460618038169244 73.97035017880609) bank176996 +176997 POINT(40.03015138545817 73.05172876553925) bank176997 +176998 POINT(40.95440692069182 73.13087856023256) bank176998 +176999 POINT(40.236469590005875 74.05374051171205) bank176999 +177000 POINT(40.086694314152496 74.77860882535687) bank177000 +177001 POINT(40.92858617960684 73.66730220070565) bank177001 +177002 POINT(40.66360917974236 73.37993143114372) bank177002 +177003 POINT(41.48583245109522 73.56694272015216) bank177003 +177004 POINT(41.6176252156803 73.22379107875665) bank177004 +177005 POINT(40.396235058555064 73.62495639734874) bank177005 +177006 POINT(40.53778346576201 73.63749868250174) bank177006 +177007 POINT(40.0621432443328 73.89733851751402) bank177007 +177008 POINT(39.91543208376455 73.27731651318093) bank177008 +177009 POINT(41.06561379050306 74.72411746029864) bank177009 +177010 POINT(41.35856381751358 73.09465412831018) bank177010 +177011 POINT(40.35819754841916 73.50756641685317) bank177011 +177012 POINT(39.73362815816304 73.84311931418316) bank177012 +177013 POINT(39.94079782332175 73.1434999930031) bank177013 +177014 POINT(40.82833925807757 73.4731595031276) bank177014 +177015 POINT(40.38690276137024 73.32444648474986) bank177015 +177016 POINT(41.09246620259436 73.71141539728329) bank177016 +177017 POINT(41.37639536336191 73.50007101426621) bank177017 +177018 POINT(40.77878318386197 74.90588245597826) bank177018 +177019 POINT(40.31173093377119 73.55483146129656) bank177019 +177020 POINT(39.889762395604485 73.32847240204686) bank177020 +177021 POINT(40.31869157531027 74.05247721001986) bank177021 +177022 POINT(41.062194337215644 74.0919667291374) bank177022 +177023 POINT(40.79244038132099 73.22350082292061) bank177023 +177024 POINT(39.84268615153196 74.72478807071982) bank177024 +177025 POINT(41.228221845216446 74.47105819502119) bank177025 +177026 POINT(40.49357154751299 73.14082515589244) bank177026 +177027 POINT(39.960255933973855 74.64308888099472) bank177027 +177028 POINT(40.597305285527824 74.59802436655055) bank177028 +177029 POINT(41.24963404782156 73.37786321148498) bank177029 +177030 POINT(40.89404883411535 74.43903150104225) bank177030 +177031 POINT(39.92023297101868 74.57430583444096) bank177031 +177032 POINT(39.84569531984391 73.40420923362949) bank177032 +177033 POINT(40.89114906713955 74.19861143309335) bank177033 +177034 POINT(40.77636478440239 74.0295501137809) bank177034 +177035 POINT(41.707114707852014 74.49302310298937) bank177035 +177036 POINT(40.913953260378925 73.11928773346004) bank177036 +177037 POINT(41.33844550326279 73.8549529512704) bank177037 +177038 POINT(41.64732017124454 73.00978413187921) bank177038 +177039 POINT(41.163882640487856 73.50415682916257) bank177039 +177040 POINT(41.6525791568799 74.42762505215352) bank177040 +177041 POINT(40.103925780417434 73.02581069656439) bank177041 +177042 POINT(40.79593867342831 73.2778037873941) bank177042 +177043 POINT(41.49050764072702 73.78726752998027) bank177043 +177044 POINT(41.07813752043472 74.34460524640008) bank177044 +177045 POINT(40.05272876233091 74.33008511124828) bank177045 +177046 POINT(41.52034865965145 73.52824948074039) bank177046 +177047 POINT(41.6337288605497 74.518016998833) bank177047 +177048 POINT(41.54152115780907 73.35059898700088) bank177048 +177049 POINT(40.88098167456496 74.46330176366159) bank177049 +177050 POINT(40.36215028680648 74.91352927252805) bank177050 +177051 POINT(41.49141364795767 73.47027213288332) bank177051 +177052 POINT(41.1715753080205 74.39379938629718) bank177052 +177053 POINT(40.319369759203646 73.74335046517551) bank177053 +177054 POINT(40.290599978211226 73.2675972482288) bank177054 +177055 POINT(40.123029995164266 74.86675632366583) bank177055 +177056 POINT(40.192708396869946 73.40264550797554) bank177056 +177057 POINT(41.34266445849836 73.80455131212042) bank177057 +177058 POINT(40.09906920856242 74.93963058168683) bank177058 +177059 POINT(40.63378358963008 74.41963691560937) bank177059 +177060 POINT(40.98282629357513 73.55276406404913) bank177060 +177061 POINT(41.40525838225242 74.30991710208201) bank177061 +177062 POINT(41.25071941916305 74.11247868294112) bank177062 +177063 POINT(40.67803950153099 73.8704969091709) bank177063 +177064 POINT(40.88104349816911 74.04683856346848) bank177064 +177065 POINT(39.92002217216427 73.75697905622218) bank177065 +177066 POINT(40.30569678422064 74.3715574829078) bank177066 +177067 POINT(40.21050953484544 74.62034464808761) bank177067 +177068 POINT(40.95370993031235 73.18724866223504) bank177068 +177069 POINT(40.62774451151168 73.19028529075376) bank177069 +177070 POINT(41.24693004234033 73.61390952028655) bank177070 +177071 POINT(40.23987824514636 73.40708782958568) bank177071 +177072 POINT(41.17113188385473 74.60840538339059) bank177072 +177073 POINT(39.77524697624686 74.68104808144429) bank177073 +177074 POINT(41.304433302251766 74.49762743377292) bank177074 +177075 POINT(41.508112116924345 73.9723461793589) bank177075 +177076 POINT(41.559855309664115 74.08499977132671) bank177076 +177077 POINT(41.46392657099926 73.19181663515593) bank177077 +177078 POINT(40.89470946230331 73.06591578203901) bank177078 +177079 POINT(40.80302274533419 73.3360778680776) bank177079 +177080 POINT(41.09188751779686 73.59210261899635) bank177080 +177081 POINT(40.96032241319189 74.24113089480402) bank177081 +177082 POINT(41.39266561987116 74.78547708972614) bank177082 +177083 POINT(40.54155311069675 74.53647872462096) bank177083 +177084 POINT(40.95792279294471 73.34136940635648) bank177084 +177085 POINT(40.67823563431507 73.28107911261468) bank177085 +177086 POINT(41.134216228398635 74.6780653535021) bank177086 +177087 POINT(41.18860305576113 73.77251660871048) bank177087 +177088 POINT(41.005821355926216 73.86670068126789) bank177088 +177089 POINT(41.06020799872293 74.59188744679587) bank177089 +177090 POINT(40.59127908713864 74.17458766615637) bank177090 +177091 POINT(40.28341582730377 73.24585555815851) bank177091 +177092 POINT(40.442992824406794 74.93154174676258) bank177092 +177093 POINT(40.536982958199346 73.36204954783315) bank177093 +177094 POINT(40.56386931361383 73.73941038630338) bank177094 +177095 POINT(41.17942625811863 74.83945323693649) bank177095 +177096 POINT(40.87271523581644 74.87016772929992) bank177096 +177097 POINT(40.33357682576068 73.1404205291031) bank177097 +177098 POINT(41.316589710932256 73.66109239446149) bank177098 +177099 POINT(40.27245873990523 74.53263107695321) bank177099 +177100 POINT(41.576640233030886 74.97356291523487) bank177100 +177101 POINT(40.66887244681677 74.68659641317349) bank177101 +177102 POINT(41.04428961859685 74.91905407283657) bank177102 +177103 POINT(40.226703274318766 74.10595946957187) bank177103 +177104 POINT(40.169762686884155 73.07234614216097) bank177104 +177105 POINT(40.11354249981195 74.55301094088539) bank177105 +177106 POINT(40.510522721095974 74.9493935930297) bank177106 +177107 POINT(39.90043449757943 75.00136156004811) bank177107 +177108 POINT(41.12028947336772 73.35319765483727) bank177108 +177109 POINT(40.28505680866301 73.94717421544614) bank177109 +177110 POINT(40.43493839404939 73.42648996743823) bank177110 +177111 POINT(40.997353283896096 73.14506810032435) bank177111 +177112 POINT(40.17043370789107 74.35392893874565) bank177112 +177113 POINT(39.87563512856134 73.52890386894364) bank177113 +177114 POINT(39.85090846255198 74.03069894559778) bank177114 +177115 POINT(39.76193924353817 74.83332920272257) bank177115 +177116 POINT(41.17542793630294 73.19121487486419) bank177116 +177117 POINT(40.083042208528944 73.93602462061503) bank177117 +177118 POINT(41.14732507347968 73.24143759935532) bank177118 +177119 POINT(40.764670980538476 73.46592530677987) bank177119 +177120 POINT(41.6308248230533 74.93833732132572) bank177120 +177121 POINT(41.461427855706596 74.41562766842884) bank177121 +177122 POINT(40.71421466678522 74.85457614677892) bank177122 +177123 POINT(39.866318988463284 74.25821828785183) bank177123 +177124 POINT(40.47251560683779 73.3924626582324) bank177124 +177125 POINT(41.60623854015945 73.77596075004817) bank177125 +177126 POINT(40.80471837824846 74.62046351424476) bank177126 +177127 POINT(41.46984912920336 73.80420640354433) bank177127 +177128 POINT(40.876905450542964 73.27597691836854) bank177128 +177129 POINT(40.32657371820753 74.5903245064774) bank177129 +177130 POINT(40.846202055496825 74.73465880532768) bank177130 +177131 POINT(39.821466297913624 73.77242157604915) bank177131 +177132 POINT(40.551121689045694 74.3789370903003) bank177132 +177133 POINT(40.592134278436255 73.19383875049755) bank177133 +177134 POINT(41.70329194344352 73.61144404905359) bank177134 +177135 POINT(41.49290989195096 74.37282598560749) bank177135 +177136 POINT(41.68788779634447 73.06850656693445) bank177136 +177137 POINT(41.464892411776106 73.65507843249425) bank177137 +177138 POINT(40.341768293925654 73.64337000130836) bank177138 +177139 POINT(40.92374821611554 74.18488043738725) bank177139 +177140 POINT(39.71293273395062 74.54399943788499) bank177140 +177141 POINT(40.4826705277734 73.02846203325251) bank177141 +177142 POINT(41.00306732892515 73.72064139623328) bank177142 +177143 POINT(40.55021302272342 74.76480622786468) bank177143 +177144 POINT(39.874763808968495 73.11003478472502) bank177144 +177145 POINT(41.21196722822789 73.86748733488726) bank177145 +177146 POINT(40.46765048601076 74.74274798880643) bank177146 +177147 POINT(41.08270596636557 73.61219863771636) bank177147 +177148 POINT(41.49170287802037 73.29177327762181) bank177148 +177149 POINT(41.47190670697447 73.23800302347482) bank177149 +177150 POINT(41.14642565492859 73.51674045762557) bank177150 +177151 POINT(40.32817973362151 73.15959167484999) bank177151 +177152 POINT(41.12708341362506 73.67350188819553) bank177152 +177153 POINT(40.82475842004203 74.45340964803967) bank177153 +177154 POINT(39.841009063300284 73.57701430510983) bank177154 +177155 POINT(39.82914037946156 74.93709339552545) bank177155 +177156 POINT(40.3519326260925 73.54326283250792) bank177156 +177157 POINT(41.426734878634456 74.0367874583796) bank177157 +177158 POINT(41.15546803651317 74.63427211839725) bank177158 +177159 POINT(41.54557122235809 73.33709926696125) bank177159 +177160 POINT(39.98438087565034 73.25381262800204) bank177160 +177161 POINT(40.94121826302928 73.81936673459353) bank177161 +177162 POINT(41.258890830403246 74.15164077503097) bank177162 +177163 POINT(40.076364906882375 73.91122988923523) bank177163 +177164 POINT(39.769692437869146 74.5052228546936) bank177164 +177165 POINT(39.96320174034252 73.23233495820357) bank177165 +177166 POINT(41.55245369106003 74.17739188389928) bank177166 +177167 POINT(40.823671914574504 74.1497082850367) bank177167 +177168 POINT(40.074577403330686 73.53749385852755) bank177168 +177169 POINT(40.314357778287444 74.51502505865446) bank177169 +177170 POINT(40.72423520378991 74.59600344533429) bank177170 +177171 POINT(40.00842483730848 73.40260155293308) bank177171 +177172 POINT(39.81735043780664 73.54367905817722) bank177172 +177173 POINT(40.586312066809676 74.73663781339799) bank177173 +177174 POINT(40.00850114882835 73.88927116525711) bank177174 +177175 POINT(41.02528911253606 73.83280864322045) bank177175 +177176 POINT(41.24259288942483 74.79220717380694) bank177176 +177177 POINT(40.92338930429251 74.88269912396355) bank177177 +177178 POINT(39.79249704476848 73.53436723498123) bank177178 +177179 POINT(39.95538681311381 74.29630458814773) bank177179 +177180 POINT(40.82435793035071 74.76515506855597) bank177180 +177181 POINT(40.071094116414564 74.38223033948101) bank177181 +177182 POINT(40.12731792026243 74.06647361850392) bank177182 +177183 POINT(40.71824712628576 73.1007071942143) bank177183 +177184 POINT(40.52018088364501 73.59316921973799) bank177184 +177185 POINT(40.290688366960296 74.506090228944) bank177185 +177186 POINT(40.99305608906317 73.89400032023659) bank177186 +177187 POINT(40.11859958808083 74.14008335786646) bank177187 +177188 POINT(40.24268354306919 74.12437435057525) bank177188 +177189 POINT(40.28794945701638 73.48039677062485) bank177189 +177190 POINT(41.35745544254475 73.25792558958423) bank177190 +177191 POINT(39.83933865406955 74.85631134704816) bank177191 +177192 POINT(41.040153423758646 74.16720197320836) bank177192 +177193 POINT(40.28170449606751 74.87649889701294) bank177193 +177194 POINT(41.25897232706794 74.35293829559703) bank177194 +177195 POINT(41.59160595282466 74.48884433659505) bank177195 +177196 POINT(40.373923137644084 73.00757541089459) bank177196 +177197 POINT(41.000032374766 74.7814360385792) bank177197 +177198 POINT(41.578787276540304 73.28569458008545) bank177198 +177199 POINT(41.68310579916056 74.47676286654372) bank177199 +177200 POINT(40.26749673170819 73.96030938786812) bank177200 +177201 POINT(39.94932470383923 74.29173071041923) bank177201 +177202 POINT(40.24327923086653 73.97683814330323) bank177202 +177203 POINT(39.983256363902065 74.6832522576762) bank177203 +177204 POINT(41.07191285368433 73.22904255646897) bank177204 +177205 POINT(40.827906179045776 74.60589503521803) bank177205 +177206 POINT(40.25674244816569 73.31647163890416) bank177206 +177207 POINT(40.29287840235664 73.28131494922951) bank177207 +177208 POINT(40.681796070800765 74.95573009814382) bank177208 +177209 POINT(40.279602805062275 74.79926607798281) bank177209 +177210 POINT(39.843996255438626 74.07882936787313) bank177210 +177211 POINT(41.20879357721507 73.97042242663557) bank177211 +177212 POINT(40.795057558130296 73.31730235034547) bank177212 +177213 POINT(41.495134042970044 73.78854949536732) bank177213 +177214 POINT(40.2056011094848 73.23736289259044) bank177214 +177215 POINT(41.55054752480848 73.6247539135326) bank177215 +177216 POINT(40.4367351266352 73.37554438096737) bank177216 +177217 POINT(40.92987259648909 74.57697167747148) bank177217 +177218 POINT(40.41087770534729 74.39745124695018) bank177218 +177219 POINT(41.573119788704275 73.07325074955521) bank177219 +177220 POINT(41.68829506929615 74.32462796089949) bank177220 +177221 POINT(40.241585793883644 73.72865740322173) bank177221 +177222 POINT(40.14296365674439 74.16569488908964) bank177222 +177223 POINT(41.44915832354994 73.31628022602601) bank177223 +177224 POINT(39.7549374453703 74.70661115505894) bank177224 +177225 POINT(40.07299938634771 74.10414229676978) bank177225 +177226 POINT(40.88708585391416 74.19428685065593) bank177226 +177227 POINT(41.356181545017165 73.65133877688027) bank177227 +177228 POINT(39.93189023657583 74.88014508069809) bank177228 +177229 POINT(41.30403955414215 74.89202667544798) bank177229 +177230 POINT(41.70794222374173 73.90477491487889) bank177230 +177231 POINT(40.658134125801574 73.8564328035131) bank177231 +177232 POINT(40.007260147256495 74.94770475852863) bank177232 +177233 POINT(40.685305320745385 74.28216245678017) bank177233 +177234 POINT(41.67946691248981 73.72823637399043) bank177234 +177235 POINT(40.574406103980415 73.1452970198826) bank177235 +177236 POINT(40.6538198347338 73.95577752724881) bank177236 +177237 POINT(41.64191541644387 73.54561568180493) bank177237 +177238 POINT(39.911529769864075 74.02006131377728) bank177238 +177239 POINT(41.44952460628177 73.02827968314553) bank177239 +177240 POINT(41.692824121225165 73.73569628519613) bank177240 +177241 POINT(40.53762034415073 73.59228184808627) bank177241 +177242 POINT(39.92641154306612 74.33891000232522) bank177242 +177243 POINT(40.00973509212397 73.18407725283319) bank177243 +177244 POINT(39.92944711978504 74.38161380561368) bank177244 +177245 POINT(41.59450699847548 74.2298995873641) bank177245 +177246 POINT(41.53111627716042 73.7463408563537) bank177246 +177247 POINT(40.28787990931491 73.22158585104775) bank177247 +177248 POINT(40.18459898524435 73.51037604883395) bank177248 +177249 POINT(39.75731296201509 73.50749169558239) bank177249 +177250 POINT(40.26765265542528 74.64069303490291) bank177250 +177251 POINT(41.13328789493534 73.52896752439183) bank177251 +177252 POINT(39.72092068208667 73.89047951462318) bank177252 +177253 POINT(40.40138760866091 73.06362285438487) bank177253 +177254 POINT(39.73066988344247 73.59669987489437) bank177254 +177255 POINT(41.03246321941647 74.54415002734315) bank177255 +177256 POINT(40.368746750575916 73.89898441807313) bank177256 +177257 POINT(39.78651543201211 73.76577372977663) bank177257 +177258 POINT(41.631020302637516 73.90008950279088) bank177258 +177259 POINT(41.071044855398554 74.22073690061673) bank177259 +177260 POINT(40.90927088997373 74.92343485326627) bank177260 +177261 POINT(41.53476152816873 73.88908300828703) bank177261 +177262 POINT(40.34870305212495 73.92976732214139) bank177262 +177263 POINT(39.858901031367246 74.34248478741756) bank177263 +177264 POINT(41.029707313166234 73.43485691898029) bank177264 +177265 POINT(41.59669842560617 73.24657438349918) bank177265 +177266 POINT(39.80149700187597 74.72709168675159) bank177266 +177267 POINT(40.33827191274047 74.6421602546431) bank177267 +177268 POINT(40.03972811334113 73.35916427208387) bank177268 +177269 POINT(41.36428114102 74.16776946250974) bank177269 +177270 POINT(40.637167369787996 73.97253462704285) bank177270 +177271 POINT(39.78406869158213 73.18135111382337) bank177271 +177272 POINT(41.408415799645645 74.26765534522798) bank177272 +177273 POINT(41.66664400585372 74.68503154153916) bank177273 +177274 POINT(40.85926311946533 73.5093147749226) bank177274 +177275 POINT(40.46943930350473 73.5064090716365) bank177275 +177276 POINT(41.22855408966951 74.4520532182692) bank177276 +177277 POINT(40.249277133408576 73.7580403711383) bank177277 +177278 POINT(40.92511616892299 73.78399251767175) bank177278 +177279 POINT(40.941530941597016 73.46015591805154) bank177279 +177280 POINT(40.94062691298037 74.62566554742122) bank177280 +177281 POINT(40.915782833912246 74.83830549942017) bank177281 +177282 POINT(41.01265906666303 74.72947173940838) bank177282 +177283 POINT(40.4400852768357 73.54964629717674) bank177283 +177284 POINT(41.53500690662503 73.75509962529797) bank177284 +177285 POINT(40.838400232346885 74.49735035124081) bank177285 +177286 POINT(40.6331795311028 74.80143843916711) bank177286 +177287 POINT(41.234494612742786 74.50999497067302) bank177287 +177288 POINT(40.870121028913694 73.49061692336743) bank177288 +177289 POINT(41.38569741331408 73.0570429010327) bank177289 +177290 POINT(41.107976270152946 74.9839528115801) bank177290 +177291 POINT(40.07662430011255 74.00798586335044) bank177291 +177292 POINT(41.70639295503228 74.56225510809249) bank177292 +177293 POINT(40.098658559883205 73.10773684470357) bank177293 +177294 POINT(40.34933937228132 74.0444105679201) bank177294 +177295 POINT(39.96797653867973 73.16567497818963) bank177295 +177296 POINT(41.61094751782911 73.00912753220217) bank177296 +177297 POINT(40.99304053558804 73.50658950272533) bank177297 +177298 POINT(41.31568637410162 73.64653623275439) bank177298 +177299 POINT(41.274168552781276 74.16340268118) bank177299 +177300 POINT(40.31373133075776 74.92975170342213) bank177300 +177301 POINT(40.56085807313806 74.39660637703194) bank177301 +177302 POINT(40.52004218950833 73.0302988519152) bank177302 +177303 POINT(40.86314856117679 73.94037888028589) bank177303 +177304 POINT(40.11589328421929 74.80880077797968) bank177304 +177305 POINT(40.49727777752059 74.47207281842283) bank177305 +177306 POINT(40.98732003525052 74.94256979497679) bank177306 +177307 POINT(41.015355388668965 74.93224140695781) bank177307 +177308 POINT(40.67623673507048 74.52290043558402) bank177308 +177309 POINT(41.677094529557706 74.25242029737409) bank177309 +177310 POINT(41.08028894403173 74.28731825588677) bank177310 +177311 POINT(40.2334718065056 73.79565657905961) bank177311 +177312 POINT(41.28237596881343 73.60945893038071) bank177312 +177313 POINT(41.376512797371866 73.50515447982818) bank177313 +177314 POINT(40.18630842046425 73.04341631644635) bank177314 +177315 POINT(40.06085413293852 73.935351721376) bank177315 +177316 POINT(40.74395627713756 74.05677663115492) bank177316 +177317 POINT(40.18777545855192 74.19136272397415) bank177317 +177318 POINT(41.55275713447723 73.50312991282905) bank177318 +177319 POINT(41.578897765798416 74.718919158786) bank177319 +177320 POINT(40.861518561268 73.03082884696451) bank177320 +177321 POINT(39.769093050509944 74.86467241255892) bank177321 +177322 POINT(41.34758316738902 73.6108562182093) bank177322 +177323 POINT(41.61737767113431 73.95799766832928) bank177323 +177324 POINT(40.53054138293434 73.90781935421887) bank177324 +177325 POINT(39.9681132765738 74.53914465484473) bank177325 +177326 POINT(39.805359484656975 74.69088021567538) bank177326 +177327 POINT(41.34057252391464 73.95738175021518) bank177327 +177328 POINT(39.71791211409255 73.887241951332) bank177328 +177329 POINT(40.103262895275854 73.96607962447935) bank177329 +177330 POINT(41.34972572959501 73.17093275775612) bank177330 +177331 POINT(41.32951727441996 74.74585625365171) bank177331 +177332 POINT(40.55909340061689 74.04936610248129) bank177332 +177333 POINT(41.01484582686862 74.99373151602471) bank177333 +177334 POINT(40.49304492145951 73.78444726468328) bank177334 +177335 POINT(40.73319232906595 73.2031516165424) bank177335 +177336 POINT(40.43463590557166 73.40308173398769) bank177336 +177337 POINT(40.16392795891651 74.1426874685949) bank177337 +177338 POINT(39.87158154031907 74.77478639260244) bank177338 +177339 POINT(41.708257163122624 73.90502692459404) bank177339 +177340 POINT(40.67560613375281 74.13950507590347) bank177340 +177341 POINT(41.46411268346518 74.85175172648081) bank177341 +177342 POINT(39.92040914657935 74.64721711394616) bank177342 +177343 POINT(41.48552357860303 74.77535527778183) bank177343 +177344 POINT(40.31538429385566 73.04505963833238) bank177344 +177345 POINT(40.410895134442335 73.86350397876227) bank177345 +177346 POINT(41.64326956933213 73.3161570416767) bank177346 +177347 POINT(40.892510162537874 74.18579319503702) bank177347 +177348 POINT(40.8536569584456 74.62065861250012) bank177348 +177349 POINT(40.034988972204616 73.99280275764619) bank177349 +177350 POINT(41.472932310166364 73.0105137044366) bank177350 +177351 POINT(41.469909660966174 74.35333260054777) bank177351 +177352 POINT(40.15546053808595 74.13701092945621) bank177352 +177353 POINT(40.421916299420225 73.51990343475246) bank177353 +177354 POINT(41.293794190708695 74.83636675459478) bank177354 +177355 POINT(40.696173732183254 73.96729639674969) bank177355 +177356 POINT(40.295273211276694 73.84007779958645) bank177356 +177357 POINT(40.82832985736868 74.11854133666478) bank177357 +177358 POINT(40.78359378948852 74.28920994559627) bank177358 +177359 POINT(39.84066515704833 73.49725717632148) bank177359 +177360 POINT(40.29170563803159 73.22559929474835) bank177360 +177361 POINT(40.307599502036446 73.97709357219343) bank177361 +177362 POINT(39.78342099411752 73.03552550274236) bank177362 +177363 POINT(40.68421003005735 73.40821197150288) bank177363 +177364 POINT(40.74319817934947 74.76079374713073) bank177364 +177365 POINT(40.80585899401204 74.8860035139516) bank177365 +177366 POINT(39.91770315130081 73.98626883863021) bank177366 +177367 POINT(41.329401745550825 74.37124984696116) bank177367 +177368 POINT(40.37459846814613 74.12165371831972) bank177368 +177369 POINT(41.32320164943213 73.45289944594968) bank177369 +177370 POINT(40.121653211978966 73.94297319180248) bank177370 +177371 POINT(39.758625532647564 74.38673684636107) bank177371 +177372 POINT(40.01535305198312 73.6111121826653) bank177372 +177373 POINT(40.64321964740485 74.9475005416669) bank177373 +177374 POINT(40.5532447068774 74.774109733923) bank177374 +177375 POINT(39.881268705733646 74.20496386781217) bank177375 +177376 POINT(41.140335483602485 73.59293333593321) bank177376 +177377 POINT(40.20810995682776 74.31083878169547) bank177377 +177378 POINT(40.03900153196709 74.04068969659272) bank177378 +177379 POINT(41.127685063677156 73.19829861277312) bank177379 +177380 POINT(41.4599603279341 73.0312643318308) bank177380 +177381 POINT(41.169780964911745 73.01526505156929) bank177381 +177382 POINT(40.38393760841269 74.65857174592752) bank177382 +177383 POINT(40.27458646472522 73.52299444959176) bank177383 +177384 POINT(41.43544859496423 74.76907413210802) bank177384 +177385 POINT(40.472093269433806 74.23495861388508) bank177385 +177386 POINT(39.934309615719165 74.52734685589391) bank177386 +177387 POINT(40.74196046775386 73.53440913751825) bank177387 +177388 POINT(40.94294929854399 73.82602660806029) bank177388 +177389 POINT(41.16461811844997 74.93594835625588) bank177389 +177390 POINT(39.79464537780354 74.09047854908452) bank177390 +177391 POINT(40.136292315113465 74.51914090054679) bank177391 +177392 POINT(40.7823873152381 73.69392247042585) bank177392 +177393 POINT(40.72547309113403 74.74926535002763) bank177393 +177394 POINT(40.832079426498154 74.59301183715411) bank177394 +177395 POINT(40.470403855018006 73.26023243752908) bank177395 +177396 POINT(40.26740698018225 73.94241553611997) bank177396 +177397 POINT(41.46411941099608 73.69354171146458) bank177397 +177398 POINT(39.916984267698915 73.24017762863721) bank177398 +177399 POINT(41.40183389241822 73.44360793232346) bank177399 +177400 POINT(40.5475956402913 73.74769101106224) bank177400 +177401 POINT(39.95762867610402 74.66409955188814) bank177401 +177402 POINT(40.17776487952122 74.41548407097223) bank177402 +177403 POINT(39.750091608431966 74.72208280444339) bank177403 +177404 POINT(40.176216411948126 74.18161603708668) bank177404 +177405 POINT(39.82055492986793 74.33573174355612) bank177405 +177406 POINT(41.488102615623546 74.10002250191874) bank177406 +177407 POINT(40.00036284101333 74.16966355781642) bank177407 +177408 POINT(40.79679894293312 73.47366470854037) bank177408 +177409 POINT(40.78534488386298 73.56042692386809) bank177409 +177410 POINT(41.52583551841574 74.10491610389376) bank177410 +177411 POINT(40.73303427184614 74.86497543441939) bank177411 +177412 POINT(40.907950273710526 73.72081759007584) bank177412 +177413 POINT(41.43734051908369 74.6962431793585) bank177413 +177414 POINT(41.44689358422917 74.462357389438) bank177414 +177415 POINT(41.11789695537243 73.91168553046381) bank177415 +177416 POINT(40.5653914235777 74.15154591280807) bank177416 +177417 POINT(41.459255629977385 74.92410839278892) bank177417 +177418 POINT(40.35033740579998 74.81003882305444) bank177418 +177419 POINT(40.1095484684098 73.76352082679848) bank177419 +177420 POINT(39.72935170623675 73.72788492630349) bank177420 +177421 POINT(41.0096225850962 74.89688618119408) bank177421 +177422 POINT(39.88619581273873 73.64862276508221) bank177422 +177423 POINT(41.632313176063434 73.57821611469022) bank177423 +177424 POINT(41.629741208730344 73.98421758635942) bank177424 +177425 POINT(41.35724552233128 73.12927878807001) bank177425 +177426 POINT(40.23558668457429 73.30533040127804) bank177426 +177427 POINT(40.354789085472106 73.27453037608916) bank177427 +177428 POINT(40.93400303930846 73.76294383573062) bank177428 +177429 POINT(40.80627883421715 73.8148741491961) bank177429 +177430 POINT(40.095178030397726 74.62349116525915) bank177430 +177431 POINT(40.100502601623376 74.7371038708752) bank177431 +177432 POINT(41.45189433332079 73.81108891734182) bank177432 +177433 POINT(39.7423021868229 73.80592151570362) bank177433 +177434 POINT(41.029395898223655 74.02531074879549) bank177434 +177435 POINT(41.156878786605866 74.13737115263821) bank177435 +177436 POINT(41.6851723377123 74.41709409201381) bank177436 +177437 POINT(41.126564680213406 73.71527478638079) bank177437 +177438 POINT(40.45662676419881 74.59911276703872) bank177438 +177439 POINT(40.446183698353366 73.2951982439282) bank177439 +177440 POINT(41.67683715035855 73.5603869095709) bank177440 +177441 POINT(39.87254144239413 73.75742141622041) bank177441 +177442 POINT(41.67630230033685 73.45657166529939) bank177442 +177443 POINT(40.06820638927135 73.09519259665674) bank177443 +177444 POINT(41.17428520176912 73.11867008858066) bank177444 +177445 POINT(41.44972599668477 73.51358373493034) bank177445 +177446 POINT(39.7275770783856 73.28134412462477) bank177446 +177447 POINT(41.4391632189741 74.87955055317403) bank177447 +177448 POINT(39.825079120488795 74.06582301180006) bank177448 +177449 POINT(40.79943596861933 74.703280181771) bank177449 +177450 POINT(41.463713583422326 74.64371520741926) bank177450 +177451 POINT(40.18582641863612 74.21812290761025) bank177451 +177452 POINT(40.6951663659663 74.04137595958603) bank177452 +177453 POINT(40.81478127626867 73.09699397695816) bank177453 +177454 POINT(40.888221382849096 73.82149568373777) bank177454 +177455 POINT(41.51973712436567 73.58774552685612) bank177455 +177456 POINT(39.96375607366587 73.53299295589707) bank177456 +177457 POINT(40.84464257294884 74.46277654102956) bank177457 +177458 POINT(40.86683728126921 73.49630065868378) bank177458 +177459 POINT(41.47509739526124 74.23143182143568) bank177459 +177460 POINT(41.31610345980453 74.69680999905066) bank177460 +177461 POINT(39.777919436945304 73.92754362706071) bank177461 +177462 POINT(41.619707579607386 74.01398673316386) bank177462 +177463 POINT(41.111996154696904 74.93634528306261) bank177463 +177464 POINT(40.91110761994862 74.11919219185347) bank177464 +177465 POINT(40.66888391393461 74.69118027894051) bank177465 +177466 POINT(39.948929144443795 74.96428657623579) bank177466 +177467 POINT(40.65447543784324 74.10710511838509) bank177467 +177468 POINT(40.941642863530035 73.07239046256251) bank177468 +177469 POINT(41.41999926335363 74.73541589481496) bank177469 +177470 POINT(41.42784916711033 74.53748377525659) bank177470 +177471 POINT(40.16084882496017 73.4407904683568) bank177471 +177472 POINT(40.95887637302045 74.71931763660719) bank177472 +177473 POINT(41.575072279840704 74.59374317903507) bank177473 +177474 POINT(41.349937770140336 74.54541690984121) bank177474 +177475 POINT(39.82677333861494 73.44961473340639) bank177475 +177476 POINT(39.8434467215429 73.83738855115124) bank177476 +177477 POINT(41.21925115301984 74.61440409594843) bank177477 +177478 POINT(40.20523073890847 73.45229524706376) bank177478 +177479 POINT(41.128081908265 73.20039550202077) bank177479 +177480 POINT(41.07601847193864 73.28610651810598) bank177480 +177481 POINT(41.32043331530327 73.7899188484218) bank177481 +177482 POINT(40.43869234398704 73.26996544396029) bank177482 +177483 POINT(40.765520963468084 73.93624913815295) bank177483 +177484 POINT(40.12896350224827 74.76294955382046) bank177484 +177485 POINT(39.86111162170896 74.76296913489497) bank177485 +177486 POINT(41.6450038958739 73.24094055758746) bank177486 +177487 POINT(41.1262576149241 73.01612974486919) bank177487 +177488 POINT(40.18538367992384 74.26741278917991) bank177488 +177489 POINT(39.90450943131 74.29681838786854) bank177489 +177490 POINT(40.374712875000064 73.06313805664739) bank177490 +177491 POINT(40.22836831828898 74.92943593265889) bank177491 +177492 POINT(39.81524459780749 73.58381864153601) bank177492 +177493 POINT(39.7662116361642 74.92984340743278) bank177493 +177494 POINT(41.564486953383884 73.91854879832813) bank177494 +177495 POINT(40.78887170127466 74.80744362033356) bank177495 +177496 POINT(41.30718881757057 74.28637139470997) bank177496 +177497 POINT(40.304404195512895 73.02326828734387) bank177497 +177498 POINT(39.832181712069655 73.19410282791377) bank177498 +177499 POINT(40.498856607338666 74.1685681679387) bank177499 +177500 POINT(41.025318013880124 74.93649159593318) bank177500 +177501 POINT(40.411397060051264 74.1061242188361) bank177501 +177502 POINT(40.865906758292816 74.88122626618626) bank177502 +177503 POINT(41.33835773806014 74.1659550388805) bank177503 +177504 POINT(40.34971087826229 74.78034182264201) bank177504 +177505 POINT(39.757593138876146 74.28025588535525) bank177505 +177506 POINT(39.726792368349535 74.33793101242124) bank177506 +177507 POINT(40.87396289771344 74.29371025826899) bank177507 +177508 POINT(41.467652760258915 73.22550512653619) bank177508 +177509 POINT(41.379381128613666 73.57805852915293) bank177509 +177510 POINT(40.87607675731566 73.63779427038676) bank177510 +177511 POINT(39.79664666296682 73.47593209569683) bank177511 +177512 POINT(41.59255502832148 73.7480574636697) bank177512 +177513 POINT(39.79477413241135 74.55034996599352) bank177513 +177514 POINT(41.04862559232408 73.43162877040963) bank177514 +177515 POINT(40.06046231434009 73.82292334115223) bank177515 +177516 POINT(40.54736418176671 73.87999542981349) bank177516 +177517 POINT(41.675953512602945 73.4626052965471) bank177517 +177518 POINT(39.86430780051935 73.74560541610973) bank177518 +177519 POINT(39.79040426460333 74.86230403488568) bank177519 +177520 POINT(41.4234946994688 73.54736540558213) bank177520 +177521 POINT(41.24020636167737 74.89134272887091) bank177521 +177522 POINT(40.98371568574913 74.77964508150379) bank177522 +177523 POINT(39.73237487689938 73.07532665183948) bank177523 +177524 POINT(41.439865359938246 73.19719544266614) bank177524 +177525 POINT(41.229297367524495 73.72703891602615) bank177525 +177526 POINT(40.355498792352975 73.33848355851066) bank177526 +177527 POINT(40.82376688080179 74.0236988133903) bank177527 +177528 POINT(41.61262226634936 73.65008253592029) bank177528 +177529 POINT(40.79640555858158 74.79297071084945) bank177529 +177530 POINT(40.178065832546984 73.0190101504454) bank177530 +177531 POINT(40.62518719694386 73.98214473089146) bank177531 +177532 POINT(41.57646378176619 73.60434213800782) bank177532 +177533 POINT(41.63072667733839 73.70130119446837) bank177533 +177534 POINT(40.555924914019464 73.2831331610358) bank177534 +177535 POINT(41.24875208815249 74.32368087042444) bank177535 +177536 POINT(40.9745621456197 74.62099356342456) bank177536 +177537 POINT(39.840731726067276 74.60090827816502) bank177537 +177538 POINT(40.71424635265223 73.97877099820195) bank177538 +177539 POINT(40.751515381611696 74.09350962304802) bank177539 +177540 POINT(41.53764229829874 74.30234649335645) bank177540 +177541 POINT(39.85446683883957 73.31634710460838) bank177541 +177542 POINT(40.374439243952764 74.12621566312681) bank177542 +177543 POINT(41.3872075661705 74.23636296617357) bank177543 +177544 POINT(40.08248643843742 74.56860000720965) bank177544 +177545 POINT(39.916062006294425 74.68655317011316) bank177545 +177546 POINT(40.89464898132517 74.88543630693093) bank177546 +177547 POINT(40.43615120453893 74.12868011318093) bank177547 +177548 POINT(40.918368055821816 73.41382036397603) bank177548 +177549 POINT(40.563649158404644 74.05605002894858) bank177549 +177550 POINT(41.099419801863846 73.75438795489946) bank177550 +177551 POINT(41.57643979240991 74.26434886277576) bank177551 +177552 POINT(39.79327704094093 74.83060050650123) bank177552 +177553 POINT(41.61453537057357 74.99833797725317) bank177553 +177554 POINT(41.419563390869286 74.33784585502352) bank177554 +177555 POINT(40.50872959464649 73.01023410496812) bank177555 +177556 POINT(40.20757392114778 73.71988529555632) bank177556 +177557 POINT(40.22341624940325 74.20368861689646) bank177557 +177558 POINT(40.8136454582795 73.8099904745466) bank177558 +177559 POINT(40.10434863640679 73.64327062401449) bank177559 +177560 POINT(40.80177577686539 74.92907449069604) bank177560 +177561 POINT(41.66485320904416 73.8546290713724) bank177561 +177562 POINT(40.369103775895915 74.70150678818864) bank177562 +177563 POINT(41.535203081706456 74.39348639788383) bank177563 +177564 POINT(40.69145204045731 74.45046956973268) bank177564 +177565 POINT(41.137446366329534 74.44790669625604) bank177565 +177566 POINT(39.871226377525744 74.99388774230945) bank177566 +177567 POINT(41.073503873521815 74.12539681779285) bank177567 +177568 POINT(41.38370012891373 74.25626751249749) bank177568 +177569 POINT(40.658453846687685 73.31384901495767) bank177569 +177570 POINT(41.28857857250494 73.4898240326555) bank177570 +177571 POINT(40.19095302965594 73.42278570388099) bank177571 +177572 POINT(41.67529898923685 74.43294011950488) bank177572 +177573 POINT(41.50738226831879 74.05218209377787) bank177573 +177574 POINT(41.153961885985105 74.07888353397225) bank177574 +177575 POINT(41.557204376508 74.27281570788065) bank177575 +177576 POINT(40.54618016573674 74.12002059543032) bank177576 +177577 POINT(39.798812434123185 73.23372592111957) bank177577 +177578 POINT(40.5086998785374 73.55752554139025) bank177578 +177579 POINT(39.82522392673144 73.52517630550204) bank177579 +177580 POINT(40.11933554112357 73.96403806425555) bank177580 +177581 POINT(40.82180608651696 74.80008607589993) bank177581 +177582 POINT(40.059965828111686 74.66382494087438) bank177582 +177583 POINT(41.47965982140824 74.52326097134791) bank177583 +177584 POINT(41.27660646709461 73.3209264998345) bank177584 +177585 POINT(40.76111484079539 74.28344855402759) bank177585 +177586 POINT(40.06345257218289 74.38849527675013) bank177586 +177587 POINT(40.15984796907192 74.99317951008622) bank177587 +177588 POINT(41.07625143342712 73.96460729562305) bank177588 +177589 POINT(41.168603255664046 73.61548237740641) bank177589 +177590 POINT(40.27712909975474 73.24571584122174) bank177590 +177591 POINT(40.7336909362388 73.47290326927683) bank177591 +177592 POINT(40.31666440959893 74.0637822062356) bank177592 +177593 POINT(40.18227089723905 73.80558746860187) bank177593 +177594 POINT(39.79984242132734 73.40109413251427) bank177594 +177595 POINT(40.072547190693165 73.91467746652752) bank177595 +177596 POINT(40.23056952731279 73.97010425033405) bank177596 +177597 POINT(40.741513660456484 74.27059874493291) bank177597 +177598 POINT(41.317438561180516 74.26729703213883) bank177598 +177599 POINT(41.050643775864685 74.53495898836692) bank177599 +177600 POINT(41.060563993112375 74.71165278844971) bank177600 +177601 POINT(41.554352680729835 73.19951484527192) bank177601 +177602 POINT(40.639607615953516 73.7090057621396) bank177602 +177603 POINT(39.75919306280136 73.88999100694899) bank177603 +177604 POINT(40.813559038204104 74.67013016691715) bank177604 +177605 POINT(40.432973331621184 73.2079172868891) bank177605 +177606 POINT(40.469021409593374 73.58408888848446) bank177606 +177607 POINT(41.66676464442816 74.85342138164306) bank177607 +177608 POINT(40.45667803875156 74.0469412609948) bank177608 +177609 POINT(41.21225042873135 74.51420786832972) bank177609 +177610 POINT(40.911641469813866 74.21744524741025) bank177610 +177611 POINT(40.150031691605115 73.26822209825404) bank177611 +177612 POINT(40.03245870747406 74.39892568446233) bank177612 +177613 POINT(41.20733106646795 73.93962559427129) bank177613 +177614 POINT(39.72630914035708 73.41191591438346) bank177614 +177615 POINT(40.102991684193576 74.67803991192932) bank177615 +177616 POINT(41.48698987776858 74.07609090339852) bank177616 +177617 POINT(41.47974386867837 73.31339778747231) bank177617 +177618 POINT(40.665810159246014 74.39195409604555) bank177618 +177619 POINT(41.6464115802515 74.78297129336919) bank177619 +177620 POINT(41.068097003043 74.56639858259777) bank177620 +177621 POINT(41.37757256433179 74.42622526696593) bank177621 +177622 POINT(41.60925669042965 73.5517432082706) bank177622 +177623 POINT(39.89336403666982 73.109434026453) bank177623 +177624 POINT(40.54302875037827 73.04838342924111) bank177624 +177625 POINT(39.739441130801346 74.7432720156043) bank177625 +177626 POINT(40.457792047759185 74.31151988957603) bank177626 +177627 POINT(41.430868071270226 73.12562901110881) bank177627 +177628 POINT(39.72816307479141 74.19069624579465) bank177628 +177629 POINT(40.739186990025026 74.94322304098927) bank177629 +177630 POINT(40.92768958047818 73.8347473216206) bank177630 +177631 POINT(40.36235619824201 73.31190293004022) bank177631 +177632 POINT(40.03618113561367 73.98649961331967) bank177632 +177633 POINT(40.084064946046226 73.55881136903022) bank177633 +177634 POINT(41.66292379394153 74.47414179195621) bank177634 +177635 POINT(40.92145259764154 73.04271316804005) bank177635 +177636 POINT(41.25138585844644 74.66220794058185) bank177636 +177637 POINT(40.0671442733352 74.08088449881527) bank177637 +177638 POINT(40.93714733848028 73.6187274001148) bank177638 +177639 POINT(40.978649237902175 74.1359954741779) bank177639 +177640 POINT(40.831895437502695 74.92943392072185) bank177640 +177641 POINT(41.46225187574172 73.27198848980055) bank177641 +177642 POINT(40.053802626686455 73.79029548044487) bank177642 +177643 POINT(41.27758393846933 73.44421093720686) bank177643 +177644 POINT(40.3306061062973 73.91469416020882) bank177644 +177645 POINT(39.73030590429709 73.93002559526636) bank177645 +177646 POINT(40.726426808982026 73.51573159202526) bank177646 +177647 POINT(40.69807295277877 74.76964596542867) bank177647 +177648 POINT(41.193177430189806 74.63399624789328) bank177648 +177649 POINT(40.13446468779229 73.39024337759501) bank177649 +177650 POINT(40.25043177548028 74.48271223757652) bank177650 +177651 POINT(41.4481252139294 74.29584811373995) bank177651 +177652 POINT(41.39811216541089 73.01608861156355) bank177652 +177653 POINT(40.96548961645565 73.45768707505474) bank177653 +177654 POINT(40.07273235750439 74.76085161757034) bank177654 +177655 POINT(40.21696350753103 73.0594006234117) bank177655 +177656 POINT(41.57031365123834 74.09629376935435) bank177656 +177657 POINT(40.85905938712064 73.28414680483169) bank177657 +177658 POINT(40.57076258300623 73.93329979883588) bank177658 +177659 POINT(39.913131345789225 74.91069044798364) bank177659 +177660 POINT(40.37568729198084 73.80860502871971) bank177660 +177661 POINT(40.895129135032896 74.35013946869606) bank177661 +177662 POINT(41.594518707280855 73.05414983664093) bank177662 +177663 POINT(40.155349784062665 74.82106398290718) bank177663 +177664 POINT(41.27196593231233 73.21910000906234) bank177664 +177665 POINT(41.51704279376525 73.60094816227824) bank177665 +177666 POINT(39.99344678989547 74.86734673338556) bank177666 +177667 POINT(40.87236887365163 74.54326924093509) bank177667 +177668 POINT(40.0144928170476 73.25908167148651) bank177668 +177669 POINT(40.36540441345332 74.06754955369365) bank177669 +177670 POINT(41.24414078271884 73.82543453250622) bank177670 +177671 POINT(41.49440731241061 74.46830733222568) bank177671 +177672 POINT(41.691206847544244 74.68450380474818) bank177672 +177673 POINT(40.77678328303914 74.63344478274344) bank177673 +177674 POINT(40.42334502389427 73.29868178003808) bank177674 +177675 POINT(41.70027784155834 74.00646199507024) bank177675 +177676 POINT(40.20848827637067 74.3945227625308) bank177676 +177677 POINT(40.47593692428699 73.5911109791803) bank177677 +177678 POINT(39.80913896404262 74.84242419383841) bank177678 +177679 POINT(40.71534216225885 74.42053104516197) bank177679 +177680 POINT(40.08182208422237 74.58431492750239) bank177680 +177681 POINT(40.98255046077834 73.10190878649034) bank177681 +177682 POINT(40.998793500791244 73.02356559825272) bank177682 +177683 POINT(41.530437957455796 73.70622370449965) bank177683 +177684 POINT(41.491672829714545 74.43530190384134) bank177684 +177685 POINT(41.56995620515569 74.99624238025655) bank177685 +177686 POINT(41.47901878841008 74.54113370104167) bank177686 +177687 POINT(40.95371753506955 73.55379993339919) bank177687 +177688 POINT(40.726780399235174 73.31035367680269) bank177688 +177689 POINT(40.279489307452984 73.75992313531157) bank177689 +177690 POINT(41.67924865427825 73.46843579999883) bank177690 +177691 POINT(40.32849430666689 74.2619005087049) bank177691 +177692 POINT(40.804746767552786 74.19543854423377) bank177692 +177693 POINT(40.728029079294586 73.80896916368765) bank177693 +177694 POINT(40.72815123091087 73.86523594514233) bank177694 +177695 POINT(41.096791697734815 73.90128810022998) bank177695 +177696 POINT(40.22473814215586 73.89226647090882) bank177696 +177697 POINT(40.35630143070878 73.22300067613192) bank177697 +177698 POINT(39.931787584758055 74.0946406356986) bank177698 +177699 POINT(41.30234842469367 74.54392248678535) bank177699 +177700 POINT(41.158789204144966 74.95726151099885) bank177700 +177701 POINT(41.62348321511946 73.43050224590381) bank177701 +177702 POINT(40.25109896291996 74.68847798183603) bank177702 +177703 POINT(41.68972350312175 73.01621338980218) bank177703 +177704 POINT(40.44844803751162 74.15580679092285) bank177704 +177705 POINT(40.02667722905371 74.61753389257859) bank177705 +177706 POINT(41.65177781313841 74.48579417351347) bank177706 +177707 POINT(40.29760407259428 74.66386845726066) bank177707 +177708 POINT(40.41783219000988 73.10542421721999) bank177708 +177709 POINT(40.91776316826741 73.21758427483333) bank177709 +177710 POINT(41.66136362612339 73.31304770162609) bank177710 +177711 POINT(41.301213869142266 74.28823244589744) bank177711 +177712 POINT(41.30897835490326 73.92921422986822) bank177712 +177713 POINT(41.427767690938985 73.8698319777788) bank177713 +177714 POINT(41.094489747589314 74.51590631730872) bank177714 +177715 POINT(40.227065392293845 74.85308497629259) bank177715 +177716 POINT(40.20478844457759 74.03491420554232) bank177716 +177717 POINT(40.175870291821006 74.06665336884954) bank177717 +177718 POINT(40.348638820199874 74.62081444108817) bank177718 +177719 POINT(41.60957403897754 73.93194429718227) bank177719 +177720 POINT(41.39915969707063 74.2843436088694) bank177720 +177721 POINT(41.68122289975372 74.42883446814662) bank177721 +177722 POINT(41.15055389329948 74.66487252031806) bank177722 +177723 POINT(40.423170306800394 73.25607497822581) bank177723 +177724 POINT(39.73562090203753 74.82263377654967) bank177724 +177725 POINT(40.9074027883767 73.89525861354298) bank177725 +177726 POINT(41.66424678296573 74.0305454459702) bank177726 +177727 POINT(39.741880618564075 74.87374366963478) bank177727 +177728 POINT(39.89517337213418 73.97476249673777) bank177728 +177729 POINT(39.833913115817836 73.96831046055216) bank177729 +177730 POINT(39.777463780472 74.01727077935773) bank177730 +177731 POINT(41.37631173964013 74.12660734041239) bank177731 +177732 POINT(40.035567678138335 74.41865454034931) bank177732 +177733 POINT(40.55623018397016 73.28746699303832) bank177733 +177734 POINT(41.61095273444148 73.23806431817214) bank177734 +177735 POINT(39.88674474077017 74.64113998722551) bank177735 +177736 POINT(41.122163322172014 73.10698995917153) bank177736 +177737 POINT(40.1288983079685 73.56986153236107) bank177737 +177738 POINT(40.63064736166541 74.19735690709925) bank177738 +177739 POINT(41.34158041259338 73.5128431568858) bank177739 +177740 POINT(41.18337181266634 73.99158365378113) bank177740 +177741 POINT(39.96211487365423 73.8340275482373) bank177741 +177742 POINT(41.68986709625611 74.84137139156779) bank177742 +177743 POINT(41.09318049205367 75.00412301208576) bank177743 +177744 POINT(40.2022634369379 74.65097049672805) bank177744 +177745 POINT(39.97341052269403 74.51026954756134) bank177745 +177746 POINT(40.84809445793864 73.05406181644797) bank177746 +177747 POINT(39.94092480830035 73.18076608668936) bank177747 +177748 POINT(41.05439850807933 73.02023670576467) bank177748 +177749 POINT(40.820291972400966 74.60390142536413) bank177749 +177750 POINT(39.76635642825042 74.25099902384181) bank177750 +177751 POINT(40.1846210958021 74.63858414431928) bank177751 +177752 POINT(40.192774001811834 73.97622149684928) bank177752 +177753 POINT(41.473429251184065 74.48949101960613) bank177753 +177754 POINT(41.43221572584537 73.90938478081402) bank177754 +177755 POINT(39.898781872269474 74.50470806485144) bank177755 +177756 POINT(40.037711461958494 73.07362850605939) bank177756 +177757 POINT(40.302573324966154 73.81473918080425) bank177757 +177758 POINT(41.58794958928471 74.0458641129877) bank177758 +177759 POINT(41.62950158656391 73.3284534126106) bank177759 +177760 POINT(40.629356008665205 73.32510764280359) bank177760 +177761 POINT(41.467636365636 73.3803612204712) bank177761 +177762 POINT(40.8654352617142 73.13222326677524) bank177762 +177763 POINT(40.106378206971556 73.30589675675196) bank177763 +177764 POINT(41.30866916781052 73.83599879504081) bank177764 +177765 POINT(40.93947049051334 74.3373980166842) bank177765 +177766 POINT(40.624835429955745 73.18402089951618) bank177766 +177767 POINT(40.141521568364496 74.47393822981292) bank177767 +177768 POINT(41.14540384529068 74.4714162495221) bank177768 +177769 POINT(41.08167443690138 73.40735584058827) bank177769 +177770 POINT(40.24898967024481 73.4399499618812) bank177770 +177771 POINT(40.08136527447415 73.16136215168058) bank177771 +177772 POINT(40.80764637726306 74.59924265923958) bank177772 +177773 POINT(39.99877353223854 73.6361383372047) bank177773 +177774 POINT(40.076330682131925 73.04627746729085) bank177774 +177775 POINT(39.97848869358298 74.92956697357837) bank177775 +177776 POINT(39.75484922813747 74.69693225496334) bank177776 +177777 POINT(41.35188263010603 73.65691032390444) bank177777 +177778 POINT(40.72875047435941 73.95398210858144) bank177778 +177779 POINT(40.10338931766898 73.95187136020108) bank177779 +177780 POINT(41.50765490487174 74.97334482683002) bank177780 +177781 POINT(39.73744837639812 74.39916710885032) bank177781 +177782 POINT(41.2541481619378 74.99834882264989) bank177782 +177783 POINT(39.83425100818272 74.87010417985454) bank177783 +177784 POINT(40.544154188289816 73.73398642661392) bank177784 +177785 POINT(40.36682587256211 74.99227789487465) bank177785 +177786 POINT(41.57156557685082 74.07625658303121) bank177786 +177787 POINT(40.27054493885276 74.86520135401925) bank177787 +177788 POINT(40.19405952875189 74.84593367833799) bank177788 +177789 POINT(39.82704602495647 73.98943203374667) bank177789 +177790 POINT(40.408882726467304 73.53487077717878) bank177790 +177791 POINT(40.40358965570394 73.66399597089452) bank177791 +177792 POINT(40.93792228243879 74.66093661594661) bank177792 +177793 POINT(40.88346268202235 74.12173006656495) bank177793 +177794 POINT(40.66591724599204 73.68084185155169) bank177794 +177795 POINT(41.47165252370813 73.29615294085656) bank177795 +177796 POINT(40.728473460412786 74.34398006607054) bank177796 +177797 POINT(40.88682770747118 74.45889198861656) bank177797 +177798 POINT(40.45435024303193 74.13767048190843) bank177798 +177799 POINT(40.683574238890884 73.4054185200285) bank177799 +177800 POINT(41.26380808159743 73.36098706027273) bank177800 +177801 POINT(40.02352068618529 74.78750064716725) bank177801 +177802 POINT(41.262366568968815 74.84223426891035) bank177802 +177803 POINT(40.43939497066654 73.28559468267626) bank177803 +177804 POINT(40.94588618581648 74.48950189575031) bank177804 +177805 POINT(41.404326717255245 74.44628265930699) bank177805 +177806 POINT(40.28251505931553 73.28483521804048) bank177806 +177807 POINT(40.955464461749536 73.28608446720861) bank177807 +177808 POINT(40.36865968845101 74.42628206914296) bank177808 +177809 POINT(41.449337510154045 74.31077745833846) bank177809 +177810 POINT(40.806599925400775 74.13080991884335) bank177810 +177811 POINT(40.45827444450686 74.85030979445662) bank177811 +177812 POINT(41.68960141026273 74.82323653721197) bank177812 +177813 POINT(40.08722553010734 74.63631583202937) bank177813 +177814 POINT(41.4670466980805 73.38169384563751) bank177814 +177815 POINT(39.88641476912938 74.56280888516103) bank177815 +177816 POINT(41.12224065336846 73.08750815338449) bank177816 +177817 POINT(40.19439898637047 73.19032692281534) bank177817 +177818 POINT(41.00157964926534 73.20844908342535) bank177818 +177819 POINT(41.328952632748944 74.13563644278996) bank177819 +177820 POINT(40.01618203530557 74.93897044532618) bank177820 +177821 POINT(40.364775614503564 74.71449668932554) bank177821 +177822 POINT(40.52711087932121 74.63999143489582) bank177822 +177823 POINT(40.096537093108026 74.56078346863559) bank177823 +177824 POINT(41.56617070032641 74.47921013245758) bank177824 +177825 POINT(41.62995623199509 73.29044475997729) bank177825 +177826 POINT(41.218195725614486 74.06399394115864) bank177826 +177827 POINT(40.80181435494679 73.13117224790895) bank177827 +177828 POINT(40.309888358262675 74.53936963301314) bank177828 +177829 POINT(41.3112372017564 73.40459349331297) bank177829 +177830 POINT(40.17494491399643 73.44027444077942) bank177830 +177831 POINT(40.636394641128405 73.9742916634693) bank177831 +177832 POINT(40.41569864703531 73.3681451524969) bank177832 +177833 POINT(40.46640566292989 73.85920350821027) bank177833 +177834 POINT(40.5685311262091 74.28847965239271) bank177834 +177835 POINT(41.46543564831596 74.06809088747846) bank177835 +177836 POINT(41.24555190549247 74.35887158586756) bank177836 +177837 POINT(40.18740086837818 73.80876247406387) bank177837 +177838 POINT(39.84016334620437 73.69460030351972) bank177838 +177839 POINT(41.70224763099987 74.5830494832587) bank177839 +177840 POINT(40.387741519465266 73.37418883187551) bank177840 +177841 POINT(41.44364979154974 74.44918439072818) bank177841 +177842 POINT(40.76902265618297 73.58014691467639) bank177842 +177843 POINT(41.25611420848014 73.27678564959884) bank177843 +177844 POINT(40.39823096537583 73.5831761277884) bank177844 +177845 POINT(40.01197264610569 73.08142630631355) bank177845 +177846 POINT(40.09115573903847 74.75741043772146) bank177846 +177847 POINT(39.84631403177427 73.59653226453281) bank177847 +177848 POINT(41.66582980612509 74.88898428706611) bank177848 +177849 POINT(41.6619558338386 74.28364985209492) bank177849 +177850 POINT(41.13206306724047 74.91617732879746) bank177850 +177851 POINT(40.64527083276646 73.66264993522825) bank177851 +177852 POINT(40.249801590595716 74.75318163695155) bank177852 +177853 POINT(40.730104490015954 74.71273495828864) bank177853 +177854 POINT(41.06562628669673 73.71799421230261) bank177854 +177855 POINT(41.10966678148694 73.31042166295993) bank177855 +177856 POINT(41.09431462384913 74.17355185012167) bank177856 +177857 POINT(40.06639409140627 74.94561328175752) bank177857 +177858 POINT(41.52524647150789 74.80345189087078) bank177858 +177859 POINT(39.81326025003593 73.32055736865382) bank177859 +177860 POINT(41.02826948693487 74.47130677567992) bank177860 +177861 POINT(41.51243877055289 74.00428028566475) bank177861 +177862 POINT(39.77420305575221 74.8506771059873) bank177862 +177863 POINT(40.24335934537044 73.22214787479246) bank177863 +177864 POINT(41.2204859073332 74.92988430202811) bank177864 +177865 POINT(41.18908470663246 74.52650714550082) bank177865 +177866 POINT(40.866087385937604 73.7516488043605) bank177866 +177867 POINT(41.274955249192104 74.13149335641147) bank177867 +177868 POINT(40.28090482910369 74.2448237421586) bank177868 +177869 POINT(41.19889207440241 74.06498024042293) bank177869 +177870 POINT(40.44686373938801 74.61094274106361) bank177870 +177871 POINT(41.07563103643399 74.67220756398564) bank177871 +177872 POINT(40.71259896177127 73.09573689208932) bank177872 +177873 POINT(41.28684753370641 73.61652251374318) bank177873 +177874 POINT(41.155745274492155 74.40346096648003) bank177874 +177875 POINT(40.25287924287822 74.1464674094463) bank177875 +177876 POINT(40.841406272090595 73.79792172039159) bank177876 +177877 POINT(40.99571266211601 74.27648984285608) bank177877 +177878 POINT(41.3579191452163 73.84868278066884) bank177878 +177879 POINT(40.535331901680514 73.61362153685799) bank177879 +177880 POINT(40.33832981018568 74.77604789344548) bank177880 +177881 POINT(40.20454908045346 74.72006095286052) bank177881 +177882 POINT(40.36196343795616 73.60978044883058) bank177882 +177883 POINT(41.664888011445385 73.52526681734437) bank177883 +177884 POINT(41.39827770812822 74.17530871025589) bank177884 +177885 POINT(40.12402998867957 74.2060083050288) bank177885 +177886 POINT(40.64992825004774 73.9488940345582) bank177886 +177887 POINT(40.80374001125131 73.12292459262191) bank177887 +177888 POINT(40.1477388395622 74.96568362291836) bank177888 +177889 POINT(41.34852399924305 74.87629932080436) bank177889 +177890 POINT(40.4497701572917 74.4116183371088) bank177890 +177891 POINT(40.36486439398768 73.84311308551706) bank177891 +177892 POINT(40.315652227167405 74.19461548038682) bank177892 +177893 POINT(39.93584168182104 73.9812383836566) bank177893 +177894 POINT(41.08270411477446 74.06872344603092) bank177894 +177895 POINT(40.459964108729714 73.57220597417428) bank177895 +177896 POINT(40.46620444074608 73.61466272652946) bank177896 +177897 POINT(41.090614859594666 74.26703733259001) bank177897 +177898 POINT(40.36445030931341 73.4123870704596) bank177898 +177899 POINT(41.258552359385426 73.52788431494841) bank177899 +177900 POINT(40.084945989761316 73.90522627652612) bank177900 +177901 POINT(41.71162373899065 74.72071069509607) bank177901 +177902 POINT(40.71721564375293 74.74220170831386) bank177902 +177903 POINT(40.44499587626521 74.14115844882095) bank177903 +177904 POINT(40.02640880395 74.4998540401953) bank177904 +177905 POINT(40.05843458063259 73.5207393714821) bank177905 +177906 POINT(41.01954368674166 73.45493518212174) bank177906 +177907 POINT(41.329477473430686 74.67510843217538) bank177907 +177908 POINT(41.0468885936864 74.39399102765861) bank177908 +177909 POINT(41.441318153069304 74.29436011159927) bank177909 +177910 POINT(39.82092465696001 73.05870307501775) bank177910 +177911 POINT(40.464999633270075 74.02953842190452) bank177911 +177912 POINT(40.6923775528012 73.8880084253384) bank177912 +177913 POINT(39.863325021632846 73.54666997931085) bank177913 +177914 POINT(40.2342960910297 74.31393860556666) bank177914 +177915 POINT(41.3979907229999 74.80969853017959) bank177915 +177916 POINT(40.53045241090766 74.19238763214273) bank177916 +177917 POINT(39.81807113519207 73.01952827146452) bank177917 +177918 POINT(40.79432122440396 73.85342463233104) bank177918 +177919 POINT(41.42186793209801 73.41689400799766) bank177919 +177920 POINT(41.47035355583942 74.23339675142994) bank177920 +177921 POINT(39.837019547582116 73.25773558026725) bank177921 +177922 POINT(40.40787080735418 74.79251512984744) bank177922 +177923 POINT(40.488146099352434 74.94292787779557) bank177923 +177924 POINT(40.97809793795491 74.55039275516943) bank177924 +177925 POINT(41.61901471993629 74.66206547588988) bank177925 +177926 POINT(41.27399222437 74.1990287600381) bank177926 +177927 POINT(40.14059663099245 74.68341544213006) bank177927 +177928 POINT(39.82139084640698 74.32977544256408) bank177928 +177929 POINT(40.851402813307956 74.7277796232182) bank177929 +177930 POINT(40.409584009137525 73.21847542705139) bank177930 +177931 POINT(39.719793017775885 73.01914725573661) bank177931 +177932 POINT(40.18355682532945 74.074031415793) bank177932 +177933 POINT(40.12997289695961 74.05603528711194) bank177933 +177934 POINT(40.96610749660447 74.14624112478255) bank177934 +177935 POINT(40.316139378423266 73.46413451447721) bank177935 +177936 POINT(41.452988956599825 74.0077247320498) bank177936 +177937 POINT(41.3327591793699 74.28269440614883) bank177937 +177938 POINT(41.69253255682539 73.25280314228966) bank177938 +177939 POINT(41.009891562039655 73.65508338583862) bank177939 +177940 POINT(40.59006312752005 74.99533939558538) bank177940 +177941 POINT(40.565385228678956 74.8125915622121) bank177941 +177942 POINT(40.12579715892958 73.22335700583942) bank177942 +177943 POINT(39.848143100586874 74.89010498475102) bank177943 +177944 POINT(40.704615504122295 73.96477648374558) bank177944 +177945 POINT(40.4853144066686 73.46448062410585) bank177945 +177946 POINT(40.604074472618834 74.37319831496693) bank177946 +177947 POINT(41.076002765160986 74.66314052150857) bank177947 +177948 POINT(41.43619468737075 74.79115786060893) bank177948 +177949 POINT(41.13345114921755 74.72694637772132) bank177949 +177950 POINT(39.9318302539957 74.44330371184098) bank177950 +177951 POINT(39.86545755822446 74.14216105409346) bank177951 +177952 POINT(40.840805113980196 73.90523405630094) bank177952 +177953 POINT(40.03139624028565 74.04265667475731) bank177953 +177954 POINT(40.87396631852928 73.23771631469198) bank177954 +177955 POINT(41.13615552475632 74.21350510534643) bank177955 +177956 POINT(40.947428193850584 74.01232175605432) bank177956 +177957 POINT(41.02388517514976 74.64940079482035) bank177957 +177958 POINT(39.763398641840915 74.92059552883033) bank177958 +177959 POINT(39.86949631073614 74.7291071382898) bank177959 +177960 POINT(40.26101166541986 73.33474197840157) bank177960 +177961 POINT(39.741864933005935 74.94600907152602) bank177961 +177962 POINT(39.970343795432214 73.48733463108718) bank177962 +177963 POINT(41.57127215076358 74.21875798764206) bank177963 +177964 POINT(40.40598374425382 74.87970367782968) bank177964 +177965 POINT(40.78519776631531 73.54541301429731) bank177965 +177966 POINT(40.219885003051274 74.45043000692023) bank177966 +177967 POINT(41.4649422388595 73.2047567815175) bank177967 +177968 POINT(41.66828653337621 74.14398297815173) bank177968 +177969 POINT(40.18708150231909 73.93804391891567) bank177969 +177970 POINT(39.96981162404471 74.62265115462998) bank177970 +177971 POINT(40.18504818455895 74.02025632703167) bank177971 +177972 POINT(40.1484050892148 74.10913114913645) bank177972 +177973 POINT(41.64253799516012 74.70481762587191) bank177973 +177974 POINT(41.394299123718085 74.8682851352323) bank177974 +177975 POINT(41.58484152133605 73.59229431961063) bank177975 +177976 POINT(40.20359550024911 73.71641398330752) bank177976 +177977 POINT(41.54327096860665 74.72436941780161) bank177977 +177978 POINT(41.03311843719229 73.95481313106046) bank177978 +177979 POINT(39.79725088769346 73.6092690565732) bank177979 +177980 POINT(41.51036578028649 73.40796124917347) bank177980 +177981 POINT(40.62393173776856 73.75604702279315) bank177981 +177982 POINT(40.56056829417245 73.10483913776487) bank177982 +177983 POINT(41.43776187666208 74.64818691699757) bank177983 +177984 POINT(41.24986509315714 73.18195381464531) bank177984 +177985 POINT(40.794619057594026 73.69893638358732) bank177985 +177986 POINT(40.69729170314844 73.49531058604644) bank177986 +177987 POINT(39.92110713321866 73.19573281134913) bank177987 +177988 POINT(40.269249840658276 73.75341402045447) bank177988 +177989 POINT(39.84237370631503 74.01240878885638) bank177989 +177990 POINT(40.32299359073118 73.51665336588565) bank177990 +177991 POINT(40.0060740495916 74.53755778625047) bank177991 +177992 POINT(39.92535649596205 73.34748085444215) bank177992 +177993 POINT(40.14147227256487 74.11718716872986) bank177993 +177994 POINT(39.805654191458096 74.62637005245489) bank177994 +177995 POINT(40.533396188741804 74.22961484088057) bank177995 +177996 POINT(40.60777355274118 74.61679698297712) bank177996 +177997 POINT(40.610529091404096 73.33990054032377) bank177997 +177998 POINT(41.115293331287795 74.22764340907585) bank177998 +177999 POINT(41.4775325910687 74.67649053646153) bank177999 +178000 POINT(40.05206644009989 73.43149147632762) bank178000 +178001 POINT(40.83704917114105 73.6452809853011) bank178001 +178002 POINT(41.137433842181224 74.2357863169279) bank178002 +178003 POINT(40.82607841295648 74.5752633010734) bank178003 +178004 POINT(40.210809555488474 74.79582227169601) bank178004 +178005 POINT(41.226618113326246 74.48457316028785) bank178005 +178006 POINT(39.8005656063874 74.56753516037855) bank178006 +178007 POINT(40.26841037914859 74.59180197453092) bank178007 +178008 POINT(39.947720811073296 73.7723763254322) bank178008 +178009 POINT(39.78955490034962 73.39960647414321) bank178009 +178010 POINT(40.95136818009143 73.9061115224028) bank178010 +178011 POINT(41.24142800304735 73.9041796613805) bank178011 +178012 POINT(41.49748897288648 73.58199993364771) bank178012 +178013 POINT(40.099183153058455 74.83741141220197) bank178013 +178014 POINT(40.84917299369127 74.97408632302242) bank178014 +178015 POINT(40.52166068800327 73.67583712166751) bank178015 +178016 POINT(41.6893465052854 74.02520774374001) bank178016 +178017 POINT(41.44608027435198 73.31580938540257) bank178017 +178018 POINT(41.288367008671685 74.3117058816047) bank178018 +178019 POINT(39.87245147166428 74.57236618476063) bank178019 +178020 POINT(39.96152026202626 74.53784033319563) bank178020 +178021 POINT(39.97791878857007 73.40076654603763) bank178021 +178022 POINT(41.66713447775597 74.77514424737376) bank178022 +178023 POINT(41.11580626845302 74.49515910141723) bank178023 +178024 POINT(40.28861329941182 74.18670920867454) bank178024 +178025 POINT(40.133314191280824 73.85919564336305) bank178025 +178026 POINT(41.65415842453069 73.34942926697816) bank178026 +178027 POINT(41.00702822566039 73.91705606427551) bank178027 +178028 POINT(40.00677254250774 73.98913491159098) bank178028 +178029 POINT(40.355879730213026 73.44369254211162) bank178029 +178030 POINT(41.41911743174069 73.77220038876716) bank178030 +178031 POINT(40.39551947838617 74.85294564339878) bank178031 +178032 POINT(39.99299137051209 73.43573616043118) bank178032 +178033 POINT(40.34763317010742 73.32038094773783) bank178033 +178034 POINT(40.63581493640556 74.6622443117436) bank178034 +178035 POINT(39.72364261003932 74.79153871419193) bank178035 +178036 POINT(40.24363031624622 73.52138802500792) bank178036 +178037 POINT(41.64711444321828 73.20695791064921) bank178037 +178038 POINT(41.02986086964913 73.4670402341598) bank178038 +178039 POINT(40.165523655420174 73.8065171238723) bank178039 +178040 POINT(41.67195768545927 73.18395529472026) bank178040 +178041 POINT(41.50917554499715 74.45747111660869) bank178041 +178042 POINT(41.47962054270594 73.76957114985082) bank178042 +178043 POINT(40.042916072091934 74.51478330202319) bank178043 +178044 POINT(41.4368345294572 74.6540858777703) bank178044 +178045 POINT(41.195450595565184 73.35550807514281) bank178045 +178046 POINT(39.921635820522795 73.83603575487814) bank178046 +178047 POINT(40.95568366522252 74.84723103835967) bank178047 +178048 POINT(39.87163752057035 74.3490396895336) bank178048 +178049 POINT(41.28536254612126 74.49346615412193) bank178049 +178050 POINT(40.53826761798696 73.99198834828536) bank178050 +178051 POINT(41.196998665358336 73.13519289475583) bank178051 +178052 POINT(41.47435804567746 73.31200159033718) bank178052 +178053 POINT(41.57772768310061 74.7946852681167) bank178053 +178054 POINT(39.953639392821174 74.21863880276005) bank178054 +178055 POINT(41.348945902678835 73.33991270694219) bank178055 +178056 POINT(40.67361804186851 73.76794151431876) bank178056 +178057 POINT(40.5129792108257 74.88436171804717) bank178057 +178058 POINT(41.08898785951782 73.34959819904597) bank178058 +178059 POINT(41.51155669184291 73.01109901127232) bank178059 +178060 POINT(40.92478693134102 73.71667383488187) bank178060 +178061 POINT(39.841343658455884 74.40772791852046) bank178061 +178062 POINT(41.52250310995215 73.10717206891698) bank178062 +178063 POINT(40.91041019696074 73.03538847276248) bank178063 +178064 POINT(41.15187332120227 73.06950057022922) bank178064 +178065 POINT(40.42590435137318 73.43283406023923) bank178065 +178066 POINT(40.66786580636404 74.20862128490276) bank178066 +178067 POINT(41.18235957420678 73.86845024439944) bank178067 +178068 POINT(40.46821444433057 73.2871141078401) bank178068 +178069 POINT(39.98959040838628 74.01402384274715) bank178069 +178070 POINT(41.06488035868902 73.5376717949654) bank178070 +178071 POINT(40.167643526815546 74.71882444983567) bank178071 +178072 POINT(39.934996163390494 74.27152453949651) bank178072 +178073 POINT(40.05645532610749 74.51908546503192) bank178073 +178074 POINT(41.39181014318883 74.62955494881483) bank178074 +178075 POINT(40.96465299616779 73.20854457352645) bank178075 +178076 POINT(41.21532200370068 74.91223827560029) bank178076 +178077 POINT(40.054681474404006 73.84516675616014) bank178077 +178078 POINT(40.142713546099 73.41454535417589) bank178078 +178079 POINT(40.96216689820979 74.8854446613997) bank178079 +178080 POINT(40.6578615517834 73.54698523981864) bank178080 +178081 POINT(40.353200888229914 73.06875577387125) bank178081 +178082 POINT(40.66615149484708 73.5724749595242) bank178082 +178083 POINT(40.569085950858565 73.42907055832048) bank178083 +178084 POINT(41.63552136336744 74.9984129909714) bank178084 +178085 POINT(41.34809603997458 73.09520684050463) bank178085 +178086 POINT(40.087860340892966 73.40238645642336) bank178086 +178087 POINT(41.08512379845981 73.76547011790122) bank178087 +178088 POINT(39.82981807241709 74.22256836385492) bank178088 +178089 POINT(41.46741926174284 73.48240391933273) bank178089 +178090 POINT(41.12323646156177 74.26905300028238) bank178090 +178091 POINT(41.49387772343721 73.95017440837245) bank178091 +178092 POINT(40.27768244846446 73.39632392175906) bank178092 +178093 POINT(40.577107485372515 74.06529743989346) bank178093 +178094 POINT(41.25608411071789 74.72523810740455) bank178094 +178095 POINT(40.8232456895825 74.48915092425314) bank178095 +178096 POINT(41.35688878446091 73.46117634902481) bank178096 +178097 POINT(41.45464988808554 74.91328011171521) bank178097 +178098 POINT(40.73044969886946 73.39891278175014) bank178098 +178099 POINT(40.98226433547424 73.26291059896644) bank178099 +178100 POINT(41.06780237135955 74.57941121765327) bank178100 +178101 POINT(40.27590771045259 73.20456151555614) bank178101 +178102 POINT(41.558577801392424 74.31534995487078) bank178102 +178103 POINT(39.73120731297435 73.04713555161536) bank178103 +178104 POINT(39.97528024737461 73.16523608566959) bank178104 +178105 POINT(39.91765445036779 74.98194589689072) bank178105 +178106 POINT(40.63860364774683 73.13518630443505) bank178106 +178107 POINT(40.06833558380427 73.01929886768987) bank178107 +178108 POINT(40.552491675399935 73.07090714742047) bank178108 +178109 POINT(41.51157947957246 74.03925967614309) bank178109 +178110 POINT(40.97621884262207 74.0582967867124) bank178110 +178111 POINT(40.2091851631682 73.81495602012768) bank178111 +178112 POINT(41.150537898020936 73.03392369269928) bank178112 +178113 POINT(40.04440531531863 73.46515293647977) bank178113 +178114 POINT(40.20280448949717 73.18050125742283) bank178114 +178115 POINT(41.23765443967751 73.98067447040391) bank178115 +178116 POINT(39.983540348711884 74.39511581663832) bank178116 +178117 POINT(40.4977336531645 73.08974353650629) bank178117 +178118 POINT(41.53081942197962 73.7581303738213) bank178118 +178119 POINT(40.97768038080758 73.3658920303736) bank178119 +178120 POINT(40.79230688313579 74.91690229461082) bank178120 +178121 POINT(40.6916971492464 73.88621131309353) bank178121 +178122 POINT(41.27114468956745 74.07789531120008) bank178122 +178123 POINT(40.519142609867224 74.26111269036466) bank178123 +178124 POINT(40.64083986905757 73.51563301242409) bank178124 +178125 POINT(40.03158839268706 73.62990260475819) bank178125 +178126 POINT(41.530421198361516 74.88880930231126) bank178126 +178127 POINT(39.79995776773835 73.46175392024283) bank178127 +178128 POINT(40.05396535272003 73.5623650218566) bank178128 +178129 POINT(40.49985988185298 73.46936144963023) bank178129 +178130 POINT(41.39215387490746 73.50842028212251) bank178130 +178131 POINT(39.8364564241751 74.84153915983894) bank178131 +178132 POINT(40.30182985498038 74.41746745168898) bank178132 +178133 POINT(41.145583387011804 74.37158395560029) bank178133 +178134 POINT(40.16335404190201 74.89063890483067) bank178134 +178135 POINT(41.29550503073348 73.85226259570943) bank178135 +178136 POINT(40.87729781616808 73.60516884609187) bank178136 +178137 POINT(41.02512330942745 74.44299955641834) bank178137 +178138 POINT(40.52351029411705 73.34231362093071) bank178138 +178139 POINT(41.04651980815307 73.64795581433158) bank178139 +178140 POINT(41.37996824763284 74.08481680174862) bank178140 +178141 POINT(41.23445523070885 74.00021947209314) bank178141 +178142 POINT(40.96665180477036 73.5148466196871) bank178142 +178143 POINT(40.09397843724339 74.15137692514104) bank178143 +178144 POINT(41.564103699925674 74.71991711311728) bank178144 +178145 POINT(41.646678297446044 74.25204211787792) bank178145 +178146 POINT(39.92512800790393 74.32438596350755) bank178146 +178147 POINT(40.75803971055522 73.30491970257843) bank178147 +178148 POINT(41.06613244321141 74.38768464147466) bank178148 +178149 POINT(40.87853817834167 74.26473066816881) bank178149 +178150 POINT(40.530370474905276 74.55114549023557) bank178150 +178151 POINT(41.243420936348755 73.92495382039732) bank178151 +178152 POINT(40.45874060960928 73.61895595858262) bank178152 +178153 POINT(41.450843446360366 73.95326825639957) bank178153 +178154 POINT(40.33456567830156 73.64668927153943) bank178154 +178155 POINT(41.22175086814763 74.5009845576148) bank178155 +178156 POINT(40.671153106373765 74.15645951523713) bank178156 +178157 POINT(41.44760885604047 74.64509328747181) bank178157 +178158 POINT(39.81826123265128 74.02445698221015) bank178158 +178159 POINT(40.285692658993874 74.76337724585224) bank178159 +178160 POINT(41.14044446715062 73.46232446664278) bank178160 +178161 POINT(41.51193785757381 73.26123645368139) bank178161 +178162 POINT(40.829004573495126 74.30796984983142) bank178162 +178163 POINT(40.10983947081364 74.31429199217138) bank178163 +178164 POINT(41.396725491295726 73.04518558834982) bank178164 +178165 POINT(40.3161162037775 73.36729015075211) bank178165 +178166 POINT(41.359332064372566 74.51352346134776) bank178166 +178167 POINT(40.32184844258657 73.72110331746354) bank178167 +178168 POINT(40.321772079104335 73.8598939483307) bank178168 +178169 POINT(40.86403061052163 74.3294826182486) bank178169 +178170 POINT(40.1816329959687 73.94602402036648) bank178170 +178171 POINT(40.38748566088729 74.38382956514229) bank178171 +178172 POINT(40.73603854076218 74.7795317396414) bank178172 +178173 POINT(41.13564553272965 73.06606073654696) bank178173 +178174 POINT(41.68647731314102 73.87738721002992) bank178174 +178175 POINT(40.22659680906775 73.81392499171235) bank178175 +178176 POINT(41.4219651698927 73.1363285966688) bank178176 +178177 POINT(41.02243412211943 73.14380238609978) bank178177 +178178 POINT(41.54875991031418 74.20071843486973) bank178178 +178179 POINT(41.01163820160836 73.03089975323054) bank178179 +178180 POINT(40.81459923729233 73.63181300885446) bank178180 +178181 POINT(40.14207406576618 74.22321553359197) bank178181 +178182 POINT(40.66984411120192 74.36581670132412) bank178182 +178183 POINT(40.05729024422362 73.6864137696665) bank178183 +178184 POINT(39.9866220992898 73.75579493553629) bank178184 +178185 POINT(40.66672063745439 73.96418826150456) bank178185 +178186 POINT(40.40573532292025 74.75302582091872) bank178186 +178187 POINT(41.347966773964465 74.25698553391577) bank178187 +178188 POINT(40.05111431019538 74.88337929964796) bank178188 +178189 POINT(40.25028074572145 73.0767861281734) bank178189 +178190 POINT(40.404821308493005 73.67598803633295) bank178190 +178191 POINT(40.89442154189896 74.51301200952034) bank178191 +178192 POINT(41.17776229074086 74.48624526376193) bank178192 +178193 POINT(41.31569458991826 73.87378827705555) bank178193 +178194 POINT(41.013164298439136 74.49907465019675) bank178194 +178195 POINT(40.51151221118709 73.94318799694808) bank178195 +178196 POINT(40.769145662766256 74.13682613524975) bank178196 +178197 POINT(40.01557518051153 73.2518264942311) bank178197 +178198 POINT(40.408804883570895 73.29210156567748) bank178198 +178199 POINT(41.01614689816838 73.24627176247269) bank178199 +178200 POINT(40.645697092315295 74.00151343849264) bank178200 +178201 POINT(40.65679835703857 73.5809342052655) bank178201 +178202 POINT(39.88542197782178 73.11682605348517) bank178202 +178203 POINT(40.73516995514245 73.70001834359405) bank178203 +178204 POINT(40.444897742316705 74.75881365296225) bank178204 +178205 POINT(41.05592215993327 73.06676395531098) bank178205 +178206 POINT(40.63531356636979 73.8048257830517) bank178206 +178207 POINT(40.27286532892777 74.64926425766998) bank178207 +178208 POINT(40.986850485829116 73.26394620984692) bank178208 +178209 POINT(41.33299579477863 74.11861310360607) bank178209 +178210 POINT(40.181228058500764 73.15054023306013) bank178210 +178211 POINT(41.49220485628728 73.18332793829504) bank178211 +178212 POINT(40.90235088381696 73.16030260746795) bank178212 +178213 POINT(41.44613581123286 74.74092681464509) bank178213 +178214 POINT(40.64332793867484 74.83184316023174) bank178214 +178215 POINT(40.348324090912875 73.65857461547891) bank178215 +178216 POINT(41.456671627608074 73.76900295973579) bank178216 +178217 POINT(40.3730661825032 74.38337343900088) bank178217 +178218 POINT(41.29606111752342 73.3663351928868) bank178218 +178219 POINT(41.60588214785487 74.58263810471429) bank178219 +178220 POINT(39.80265458509592 73.36368678919696) bank178220 +178221 POINT(41.617206610763496 74.57694221998715) bank178221 +178222 POINT(40.979315601396905 73.41441744877864) bank178222 +178223 POINT(40.75015639753445 74.40953761130848) bank178223 +178224 POINT(41.42055716040962 73.89162897956759) bank178224 +178225 POINT(40.23941704504955 73.94395774089415) bank178225 +178226 POINT(41.0998862169339 73.39306354952281) bank178226 +178227 POINT(40.64000651196999 73.73065576832158) bank178227 +178228 POINT(39.85606231421875 73.16834471605105) bank178228 +178229 POINT(40.713677768665875 73.56379400619873) bank178229 +178230 POINT(39.96115747407995 73.04503240219053) bank178230 +178231 POINT(40.277729202114266 74.19493839733697) bank178231 +178232 POINT(41.55448969960012 74.44102395085463) bank178232 +178233 POINT(40.20511137908224 74.62634632842261) bank178233 +178234 POINT(40.81519735745828 74.51578454270053) bank178234 +178235 POINT(39.714268074127055 73.6192424511983) bank178235 +178236 POINT(41.487334067674134 74.52588149516238) bank178236 +178237 POINT(41.61336506926363 74.70097571864795) bank178237 +178238 POINT(40.51832300598979 73.25996134830545) bank178238 +178239 POINT(40.61222901712381 73.27734471597657) bank178239 +178240 POINT(40.21687457803195 74.29895944533953) bank178240 +178241 POINT(40.54240164919239 73.77317550664917) bank178241 +178242 POINT(40.428818459805015 74.46624557365165) bank178242 +178243 POINT(40.075585457577986 73.17235253300652) bank178243 +178244 POINT(40.688121816264434 73.35857327727435) bank178244 +178245 POINT(41.329004552539644 73.46100961946752) bank178245 +178246 POINT(41.64587290622724 73.39392590960927) bank178246 +178247 POINT(39.80325715161058 74.50028126910128) bank178247 +178248 POINT(40.90519005467487 73.61426887782183) bank178248 +178249 POINT(40.240066065453355 73.31259523325275) bank178249 +178250 POINT(41.06876938596103 73.47450938983854) bank178250 +178251 POINT(40.77796596629194 73.32925849014813) bank178251 +178252 POINT(40.14278214242791 73.54864279267157) bank178252 +178253 POINT(40.59904916892449 73.4777155089673) bank178253 +178254 POINT(40.39746295367646 74.28187652558816) bank178254 +178255 POINT(40.77466414946457 73.81116030305456) bank178255 +178256 POINT(39.951460943821886 73.75025695546518) bank178256 +178257 POINT(39.74631779843638 73.50150618437266) bank178257 +178258 POINT(39.750918439841094 74.33130254256501) bank178258 +178259 POINT(40.01785867062218 74.11973668929828) bank178259 +178260 POINT(41.285846048628954 74.3673762703386) bank178260 +178261 POINT(39.839920550324216 73.85324631026823) bank178261 +178262 POINT(40.954925095296396 74.60990242295028) bank178262 +178263 POINT(39.97628228182037 73.91210906415614) bank178263 +178264 POINT(41.291086484448 73.36450268906684) bank178264 +178265 POINT(39.77721764202464 74.66460309456409) bank178265 +178266 POINT(40.03086727301701 73.7131895853631) bank178266 +178267 POINT(41.22429326821589 73.15843147584987) bank178267 +178268 POINT(40.51302601646417 73.97478194919861) bank178268 +178269 POINT(41.13505978440859 74.8609706652711) bank178269 +178270 POINT(40.81839251739945 74.73100683915213) bank178270 +178271 POINT(41.27790548550144 74.73793379886907) bank178271 +178272 POINT(40.962701587134625 74.11023849871222) bank178272 +178273 POINT(39.885036259934544 73.47849531291477) bank178273 +178274 POINT(41.097991621485725 74.86963686650319) bank178274 +178275 POINT(41.308176729273335 74.8030278563823) bank178275 +178276 POINT(40.818378825448264 74.77606692440784) bank178276 +178277 POINT(41.2605683887487 74.59245137453823) bank178277 +178278 POINT(41.275199857999894 74.78603150819185) bank178278 +178279 POINT(39.973889613631876 73.66966559557572) bank178279 +178280 POINT(40.71730959632565 73.77409220039598) bank178280 +178281 POINT(41.23071242827681 73.67679833377954) bank178281 +178282 POINT(39.856337109326795 74.53927057102295) bank178282 +178283 POINT(41.706443377129204 74.21065082443864) bank178283 +178284 POINT(41.20669935478923 73.0416335716158) bank178284 +178285 POINT(41.531351740356364 73.02649523329164) bank178285 +178286 POINT(41.14897252669257 74.22958763929496) bank178286 +178287 POINT(40.69168926302322 73.26903884564275) bank178287 +178288 POINT(41.49198534119461 73.5032228861556) bank178288 +178289 POINT(40.26752415154392 73.15287609481885) bank178289 +178290 POINT(40.41625194988096 73.04998225171857) bank178290 +178291 POINT(40.52249023782937 73.83495491212591) bank178291 +178292 POINT(41.18889830111579 74.2608128389237) bank178292 +178293 POINT(40.57235328539639 73.29416354805295) bank178293 +178294 POINT(40.84151310697586 74.64648731421669) bank178294 +178295 POINT(41.17104983853745 74.59804823371577) bank178295 +178296 POINT(41.37937571298934 73.9068201728236) bank178296 +178297 POINT(39.82834960725292 73.03614269833872) bank178297 +178298 POINT(41.17180803897306 73.45308757845244) bank178298 +178299 POINT(41.54139274962669 73.0374083211686) bank178299 +178300 POINT(41.35262469309236 74.56789243089938) bank178300 +178301 POINT(40.06627554039974 74.01632142596131) bank178301 +178302 POINT(41.27357213189542 73.48491041778112) bank178302 +178303 POINT(41.37970743324186 73.76093470278022) bank178303 +178304 POINT(41.17195099447966 74.34182122997699) bank178304 +178305 POINT(40.470029642797705 73.10685255988757) bank178305 +178306 POINT(41.12496232860274 74.67314864295575) bank178306 +178307 POINT(40.217015558067246 74.20187209832093) bank178307 +178308 POINT(40.42740215264001 73.14589418174643) bank178308 +178309 POINT(39.83094354551265 74.37022532774847) bank178309 +178310 POINT(41.67199110424204 74.37228959263548) bank178310 +178311 POINT(40.84565489408826 74.19366986712383) bank178311 +178312 POINT(40.71414995909633 74.81716563190463) bank178312 +178313 POINT(41.015350738071525 73.52139334340191) bank178313 +178314 POINT(40.99892294677211 73.23502244732745) bank178314 +178315 POINT(40.7697529953369 73.14215479675775) bank178315 +178316 POINT(39.71397431771217 73.15012301591706) bank178316 +178317 POINT(41.11317616814075 73.38077308180382) bank178317 +178318 POINT(40.10197758772884 73.27931750503681) bank178318 +178319 POINT(40.409705336249964 73.68622498435127) bank178319 +178320 POINT(41.617399092189046 73.40344483671116) bank178320 +178321 POINT(40.07566665964619 74.79465333742624) bank178321 +178322 POINT(41.57256040028668 74.36269216794294) bank178322 +178323 POINT(39.96875342026036 74.93159610993804) bank178323 +178324 POINT(41.349747418976634 73.83227695985507) bank178324 +178325 POINT(40.44220408394377 74.80245919444228) bank178325 +178326 POINT(40.083381264369706 74.1787605233712) bank178326 +178327 POINT(40.40572887905989 73.75099745706663) bank178327 +178328 POINT(40.085458060348245 74.76738786628945) bank178328 +178329 POINT(41.10516524046524 74.7275321692259) bank178329 +178330 POINT(40.153599608730616 74.08495650404062) bank178330 +178331 POINT(40.20268586207676 73.47878523972716) bank178331 +178332 POINT(41.61332220803408 73.04719661089084) bank178332 +178333 POINT(41.08673146190941 73.87674013158906) bank178333 +178334 POINT(39.793602909487454 74.91178981074485) bank178334 +178335 POINT(41.50470166182838 74.19984673744013) bank178335 +178336 POINT(41.477776345676 73.67745172288129) bank178336 +178337 POINT(40.35810953181709 74.06438215278808) bank178337 +178338 POINT(39.74032001123106 73.99822183594146) bank178338 +178339 POINT(40.933298186189546 74.59977855093935) bank178339 +178340 POINT(40.59433225490562 73.2402049091826) bank178340 +178341 POINT(40.49223438247413 73.13479486573426) bank178341 +178342 POINT(40.91621170533477 74.54665240325247) bank178342 +178343 POINT(41.684742786146685 73.95578306033448) bank178343 +178344 POINT(40.22789744672329 73.95270947513522) bank178344 +178345 POINT(39.790893367340814 73.91400840359714) bank178345 +178346 POINT(41.46904600704118 74.90697707455175) bank178346 +178347 POINT(39.908870239419905 74.27104018193438) bank178347 +178348 POINT(41.29345947354007 73.46170079975036) bank178348 +178349 POINT(41.50620744289189 73.14252918784749) bank178349 +178350 POINT(41.49100418495547 74.55470083293608) bank178350 +178351 POINT(41.38635811981358 74.99703327733724) bank178351 +178352 POINT(41.3025740491919 73.85393198880068) bank178352 +178353 POINT(41.467125514752325 74.061427482813) bank178353 +178354 POINT(41.58482661011796 74.49758746420486) bank178354 +178355 POINT(39.980932804608635 73.17395840786807) bank178355 +178356 POINT(40.74173840964423 74.54618301682625) bank178356 +178357 POINT(40.77824088178481 73.69409525609485) bank178357 +178358 POINT(40.20940393594663 73.47153183841702) bank178358 +178359 POINT(39.81177695125019 75.00299609453089) bank178359 +178360 POINT(41.587032775135015 74.65197649315677) bank178360 +178361 POINT(41.49435772546495 74.30614303167148) bank178361 +178362 POINT(40.7556114251795 73.38215123151957) bank178362 +178363 POINT(41.67787893347637 73.35285331185432) bank178363 +178364 POINT(41.00540875380665 74.30471899578424) bank178364 +178365 POINT(40.3995355053966 73.10349803190509) bank178365 +178366 POINT(40.42879246884475 74.01155347343395) bank178366 +178367 POINT(40.59703431610918 74.78408358079302) bank178367 +178368 POINT(39.90522399449816 73.31860245018589) bank178368 +178369 POINT(39.89334665149728 73.9881651369829) bank178369 +178370 POINT(40.04643467340311 73.22120757986652) bank178370 +178371 POINT(41.02645397183694 73.79633923170307) bank178371 +178372 POINT(40.106224082765785 74.68299789348194) bank178372 +178373 POINT(41.13771497883973 74.72391365743698) bank178373 +178374 POINT(40.128691248485126 74.26813903009167) bank178374 +178375 POINT(41.1521202200806 73.24635328937903) bank178375 +178376 POINT(40.2990853600867 74.46404407286661) bank178376 +178377 POINT(41.62018121888354 74.31311873310568) bank178377 +178378 POINT(40.499717746878716 74.58716468278607) bank178378 +178379 POINT(40.46729654870051 73.75148224114129) bank178379 +178380 POINT(41.351904944281586 74.45885591909216) bank178380 +178381 POINT(41.10172556639995 73.41480492451697) bank178381 +178382 POINT(41.28648568235867 73.15610832506542) bank178382 +178383 POINT(40.39047956730994 73.83129639975982) bank178383 +178384 POINT(41.335526259012205 74.95335204265805) bank178384 +178385 POINT(41.70200071973881 74.95865086814308) bank178385 +178386 POINT(40.32184328730938 73.94252311353308) bank178386 +178387 POINT(40.01769679014214 74.72176822690159) bank178387 +178388 POINT(40.39087552867711 73.71051672447248) bank178388 +178389 POINT(41.22607457627794 73.5541869361885) bank178389 +178390 POINT(39.95299331948194 73.09500835227492) bank178390 +178391 POINT(40.73792157610291 74.00000125367971) bank178391 +178392 POINT(40.8271753002619 74.29805726481561) bank178392 +178393 POINT(41.335300819753364 74.17615962418152) bank178393 +178394 POINT(41.67612993009441 74.76919489350969) bank178394 +178395 POINT(40.88095501919506 74.56076245622404) bank178395 +178396 POINT(40.6699026102628 73.89560585099352) bank178396 +178397 POINT(39.77677472650088 74.2249019963247) bank178397 +178398 POINT(41.581236509500826 74.6342984373706) bank178398 +178399 POINT(40.02794244090772 74.76147854718855) bank178399 +178400 POINT(40.9709902944087 74.27209934245748) bank178400 +178401 POINT(39.819462944684396 74.52362602960862) bank178401 +178402 POINT(39.91742244885987 73.54950885446861) bank178402 +178403 POINT(40.29221154101049 74.13378055727514) bank178403 +178404 POINT(41.14523645454182 73.82676279149794) bank178404 +178405 POINT(40.85570746370506 74.52891144013756) bank178405 +178406 POINT(40.74557597504089 73.90209664495269) bank178406 +178407 POINT(41.28626373286176 73.40309261358234) bank178407 +178408 POINT(40.909713379931055 73.5309557490383) bank178408 +178409 POINT(41.26103324478228 73.83653550611162) bank178409 +178410 POINT(39.97517352854449 74.19603504525072) bank178410 +178411 POINT(39.85770394188639 73.0269832623007) bank178411 +178412 POINT(41.331665158314614 74.66199745607558) bank178412 +178413 POINT(40.48435931119641 74.19229542267755) bank178413 +178414 POINT(40.61287952927319 74.6567036860775) bank178414 +178415 POINT(40.06537690839165 73.14234324566131) bank178415 +178416 POINT(41.67907023063036 73.88295218797982) bank178416 +178417 POINT(39.93005577048917 73.06110456031112) bank178417 +178418 POINT(40.37230893704052 74.78291766578721) bank178418 +178419 POINT(40.490954029280445 74.93925770301523) bank178419 +178420 POINT(40.163764764353886 73.4661956941807) bank178420 +178421 POINT(39.79352949335428 74.75506534052829) bank178421 +178422 POINT(40.25871907555675 74.6216557611757) bank178422 +178423 POINT(39.937447123124635 73.95492946620895) bank178423 +178424 POINT(41.18950899355381 74.68256680224222) bank178424 +178425 POINT(39.85261700919598 73.03019566291823) bank178425 +178426 POINT(39.90148204094302 74.50117540232542) bank178426 +178427 POINT(40.068052692561366 73.40260554589322) bank178427 +178428 POINT(40.16519968032179 73.45330798251183) bank178428 +178429 POINT(40.09386738326348 73.70152889202151) bank178429 +178430 POINT(41.704889413204796 74.92149573347078) bank178430 +178431 POINT(41.169275695986485 73.96897501118136) bank178431 +178432 POINT(40.65378939036109 73.87908797662207) bank178432 +178433 POINT(41.040554075978115 73.9369750090857) bank178433 +178434 POINT(40.25568725600817 74.43062375345853) bank178434 +178435 POINT(40.05901683713415 74.79045186692377) bank178435 +178436 POINT(40.81944073836358 74.55259209273343) bank178436 +178437 POINT(40.15939593072374 74.903180616113) bank178437 +178438 POINT(41.336677478938384 74.8061878994141) bank178438 +178439 POINT(40.47482162481943 74.23699715629525) bank178439 +178440 POINT(39.77743663665747 73.4236241084192) bank178440 +178441 POINT(41.28514762389338 74.4357069278678) bank178441 +178442 POINT(40.837034307665114 73.4012259189986) bank178442 +178443 POINT(41.46849435169195 74.7376198739947) bank178443 +178444 POINT(40.52111751831259 74.52746908400059) bank178444 +178445 POINT(39.81909046712103 73.92957769451876) bank178445 +178446 POINT(41.23400660072903 73.56773121608826) bank178446 +178447 POINT(40.449347430626915 73.4783783089508) bank178447 +178448 POINT(39.74787925269148 74.18491162354726) bank178448 +178449 POINT(39.904876498200714 74.63676719896996) bank178449 +178450 POINT(41.26367481572356 73.13201497520868) bank178450 +178451 POINT(40.558679467036754 74.679997709045) bank178451 +178452 POINT(40.64408221696244 73.06780714480026) bank178452 +178453 POINT(41.27283834529129 73.27197681411252) bank178453 +178454 POINT(40.36939342398384 73.66083220050439) bank178454 +178455 POINT(41.42294590223154 74.37370723497499) bank178455 +178456 POINT(40.34533292896765 73.49301896686494) bank178456 +178457 POINT(39.996076692890796 74.5129882625399) bank178457 +178458 POINT(41.38096287532488 74.50199820876719) bank178458 +178459 POINT(40.07101765393445 73.03721791127069) bank178459 +178460 POINT(41.103897159540374 74.19166494967659) bank178460 +178461 POINT(39.841580685551854 74.54095026798021) bank178461 +178462 POINT(39.97746244610494 73.83474591244446) bank178462 +178463 POINT(40.48255521975818 73.198633692081) bank178463 +178464 POINT(39.80996714228228 74.69455899289514) bank178464 +178465 POINT(41.27954601672598 74.00170805799837) bank178465 +178466 POINT(40.67775076032153 74.04235819591784) bank178466 +178467 POINT(40.70356781048622 74.76550916663012) bank178467 +178468 POINT(40.401665156106645 74.97587172672087) bank178468 +178469 POINT(40.6581598126416 74.95526083410374) bank178469 +178470 POINT(39.86045876058326 73.55356503825507) bank178470 +178471 POINT(40.69549632978445 74.71346084053003) bank178471 +178472 POINT(40.589437780007394 74.4948223165773) bank178472 +178473 POINT(41.20814585347505 73.4355512270509) bank178473 +178474 POINT(40.58955708576581 73.25537898354654) bank178474 +178475 POINT(40.835977224727294 73.71440466573641) bank178475 +178476 POINT(41.03270906087558 74.14253948768562) bank178476 +178477 POINT(40.69403353344708 73.13578883212337) bank178477 +178478 POINT(41.290149848363846 74.05910266246197) bank178478 +178479 POINT(41.441295364577044 74.66124032457562) bank178479 +178480 POINT(40.21585951054511 74.20591514880144) bank178480 +178481 POINT(41.18912902853653 74.65271375192697) bank178481 +178482 POINT(39.98766099696898 73.10957302822482) bank178482 +178483 POINT(40.62948675403824 73.32419975144892) bank178483 +178484 POINT(40.15124232464592 74.72341351526032) bank178484 +178485 POINT(40.892615008569585 73.7041411070259) bank178485 +178486 POINT(39.88932928706795 73.46573171149467) bank178486 +178487 POINT(40.815319094940214 73.91305167378556) bank178487 +178488 POINT(39.760964976781835 74.89929331528765) bank178488 +178489 POINT(39.954106042588045 73.5955560998175) bank178489 +178490 POINT(40.43647046925974 73.25862355920215) bank178490 +178491 POINT(40.96271205020548 73.24335099789079) bank178491 +178492 POINT(39.866076248634755 73.61529373871718) bank178492 +178493 POINT(40.471577216111534 74.59131016658938) bank178493 +178494 POINT(41.4571990896363 73.33719688770262) bank178494 +178495 POINT(41.550766080955036 73.307249584348) bank178495 +178496 POINT(40.00931453852316 73.20778726872453) bank178496 +178497 POINT(41.26662575532572 73.5267602732527) bank178497 +178498 POINT(40.663079294458235 74.57981893060675) bank178498 +178499 POINT(41.209944988252936 73.69229800183881) bank178499 +178500 POINT(40.942235018231074 73.63721846765154) bank178500 +178501 POINT(41.156027027115165 73.03634771621195) bank178501 +178502 POINT(41.38536023100047 74.48205017363176) bank178502 +178503 POINT(41.64878065053113 74.52604804604132) bank178503 +178504 POINT(40.24700341500251 73.5969480071104) bank178504 +178505 POINT(41.10928169033258 73.84949675150627) bank178505 +178506 POINT(40.65018811660993 74.66453295070374) bank178506 +178507 POINT(41.31961890336095 73.22678715526173) bank178507 +178508 POINT(41.39816760112515 74.31552636746837) bank178508 +178509 POINT(39.89770093088292 74.54555932439975) bank178509 +178510 POINT(40.86332851763262 73.95234498793755) bank178510 +178511 POINT(40.43383354999916 73.53475616402189) bank178511 +178512 POINT(40.387439600288516 73.3074415264301) bank178512 +178513 POINT(41.65937148072951 74.71970631842) bank178513 +178514 POINT(40.638613149612915 74.31979981385007) bank178514 +178515 POINT(40.23267568824011 74.81270774117945) bank178515 +178516 POINT(41.17940119052901 74.97083939524303) bank178516 +178517 POINT(40.21660572933139 74.83379622467734) bank178517 +178518 POINT(40.97067654570219 74.08016901930226) bank178518 +178519 POINT(40.64936766473049 73.77180597761333) bank178519 +178520 POINT(40.24147967294259 73.88484727409688) bank178520 +178521 POINT(40.34142815926808 74.66491196526758) bank178521 +178522 POINT(41.29716079961421 74.2318976224931) bank178522 +178523 POINT(40.494030191351705 74.0902744141292) bank178523 +178524 POINT(39.77598288725183 73.43905389175758) bank178524 +178525 POINT(39.76948234240386 73.70719888053617) bank178525 +178526 POINT(40.310388679959594 74.0702821171713) bank178526 +178527 POINT(41.66417507920394 73.09900393426204) bank178527 +178528 POINT(40.78928791558678 74.8501294153791) bank178528 +178529 POINT(41.654890151725546 74.05567983079564) bank178529 +178530 POINT(40.111679409140606 74.86820697014215) bank178530 +178531 POINT(40.513046464125964 74.95893369257391) bank178531 +178532 POINT(40.55234102671456 73.40972192721168) bank178532 +178533 POINT(41.14218357257833 74.60886040284025) bank178533 +178534 POINT(41.39740295960587 73.70887492220844) bank178534 +178535 POINT(39.97850030417896 73.14962437558347) bank178535 +178536 POINT(39.91490036181986 73.84887952849006) bank178536 +178537 POINT(41.576511827059335 74.94101313014141) bank178537 +178538 POINT(41.619437235641655 73.08354343293273) bank178538 +178539 POINT(40.01029087041548 74.84495951054912) bank178539 +178540 POINT(41.6867333135523 74.88072477049802) bank178540 +178541 POINT(41.06826843345786 73.89245704965992) bank178541 +178542 POINT(41.54622168503275 74.55622708475981) bank178542 +178543 POINT(41.24462207485652 74.43041685059585) bank178543 +178544 POINT(41.076494244850096 73.69791464923559) bank178544 +178545 POINT(40.77635423599717 74.81208789390622) bank178545 +178546 POINT(39.82257652365384 75.00079797257166) bank178546 +178547 POINT(40.29145385867331 73.40155973143548) bank178547 +178548 POINT(39.99663461806305 74.14438234048822) bank178548 +178549 POINT(40.37425133492725 73.30814154507875) bank178549 +178550 POINT(41.64182586689176 73.06078056377528) bank178550 +178551 POINT(40.719805359946996 74.74322337561784) bank178551 +178552 POINT(40.43267506934938 73.24671163441543) bank178552 +178553 POINT(41.52965436500983 74.5370260882182) bank178553 +178554 POINT(41.55815448475817 73.84480502674755) bank178554 +178555 POINT(41.336670385379016 73.5481759064734) bank178555 +178556 POINT(39.936130266387856 73.7919358759262) bank178556 +178557 POINT(41.40205580710324 74.17808807800048) bank178557 +178558 POINT(40.38317075496953 73.08093351590446) bank178558 +178559 POINT(40.75589883629572 74.45527211549293) bank178559 +178560 POINT(40.668829650272485 73.35378450812127) bank178560 +178561 POINT(41.541097803909885 73.87188725813101) bank178561 +178562 POINT(41.61114779878349 74.36316612854051) bank178562 +178563 POINT(39.83279407971709 74.79395579114593) bank178563 +178564 POINT(39.772601006160656 73.77204901708441) bank178564 +178565 POINT(40.50066203524649 73.5507803402809) bank178565 +178566 POINT(39.7244032304521 74.20002580632148) bank178566 +178567 POINT(41.23892014712736 73.36927371803425) bank178567 +178568 POINT(39.737778502449146 73.72399424557882) bank178568 +178569 POINT(41.58119167957945 73.0881707807118) bank178569 +178570 POINT(40.290386134637714 74.90325527774513) bank178570 +178571 POINT(41.558162105037 73.8909875535277) bank178571 +178572 POINT(41.391312409658866 74.79930142449147) bank178572 +178573 POINT(41.27074935816229 73.59670343961413) bank178573 +178574 POINT(40.18014203647644 74.27561509774495) bank178574 +178575 POINT(41.14067217856856 73.57490567370553) bank178575 +178576 POINT(41.17323819749039 73.5700151475089) bank178576 +178577 POINT(40.20328386358991 74.90488197348847) bank178577 +178578 POINT(40.32142849380729 73.67668387686079) bank178578 +178579 POINT(40.88194421561475 73.94399983473888) bank178579 +178580 POINT(40.34061117715315 73.30344761659077) bank178580 +178581 POINT(40.54860102798334 73.71017620921499) bank178581 +178582 POINT(39.90421813460639 74.21922409136354) bank178582 +178583 POINT(40.88434569877955 73.78556479799717) bank178583 +178584 POINT(40.031316999532805 74.42646303013605) bank178584 +178585 POINT(40.018749409523565 73.27864750605806) bank178585 +178586 POINT(39.89438650562937 74.44661062131755) bank178586 +178587 POINT(41.637717711876995 74.432546908785) bank178587 +178588 POINT(41.57895380901305 74.21143021015422) bank178588 +178589 POINT(41.496662375320625 74.91067665257368) bank178589 +178590 POINT(39.81499107605624 74.45233933958525) bank178590 +178591 POINT(39.78840284684214 74.2633915836871) bank178591 +178592 POINT(40.821609621078366 74.78267975143578) bank178592 +178593 POINT(40.73588949107058 73.66616512628818) bank178593 +178594 POINT(39.78088249664924 73.47689626278294) bank178594 +178595 POINT(41.391928119289716 74.19032710896776) bank178595 +178596 POINT(41.01516713732897 73.16537709441737) bank178596 +178597 POINT(41.517542010807574 73.27442666476996) bank178597 +178598 POINT(41.709856786187274 73.3754921114739) bank178598 +178599 POINT(40.18203457172245 73.14179095680952) bank178599 +178600 POINT(40.631563400885 73.61814979011427) bank178600 +178601 POINT(41.61672672103605 73.46669984606619) bank178601 +178602 POINT(41.04582099051846 73.25461542134764) bank178602 +178603 POINT(40.65929508239619 73.82600131370945) bank178603 +178604 POINT(41.704824641382835 73.12241762006329) bank178604 +178605 POINT(39.95877135641037 73.63193822275443) bank178605 +178606 POINT(40.39085489933648 74.78264434481842) bank178606 +178607 POINT(39.92193927155228 73.58752234538865) bank178607 +178608 POINT(40.187291505370695 73.35198896076436) bank178608 +178609 POINT(41.04347655146354 74.52398078528023) bank178609 +178610 POINT(39.82829196450337 73.5823740872819) bank178610 +178611 POINT(41.56880851397666 73.06984702382857) bank178611 +178612 POINT(40.13676607116203 74.17431569782201) bank178612 +178613 POINT(40.52292259026318 74.39569858299016) bank178613 +178614 POINT(40.34619726378469 73.73804398259365) bank178614 +178615 POINT(41.62361028470969 73.17039926237763) bank178615 +178616 POINT(39.81684571084513 74.92156891680462) bank178616 +178617 POINT(41.045298149727614 74.59917160453129) bank178617 +178618 POINT(41.52134062822036 73.04561554723918) bank178618 +178619 POINT(40.82704411486002 73.91003132096179) bank178619 +178620 POINT(41.26574387986276 74.6592660893961) bank178620 +178621 POINT(39.77085616540483 74.36026337566858) bank178621 +178622 POINT(41.43041140744623 74.4164199351327) bank178622 +178623 POINT(40.244958605565685 73.65469541871853) bank178623 +178624 POINT(39.806415097658515 74.82691896106238) bank178624 +178625 POINT(40.745056728425205 74.95184249464474) bank178625 +178626 POINT(40.538387691046324 73.6125274157671) bank178626 +178627 POINT(40.814194840084596 74.76111899128784) bank178627 +178628 POINT(41.08505095206735 74.70330743847445) bank178628 +178629 POINT(41.69749001469845 73.06814373511708) bank178629 +178630 POINT(41.04899758432102 73.39937036596314) bank178630 +178631 POINT(41.214878961564736 73.68655326332261) bank178631 +178632 POINT(40.919106962295004 74.3722803098516) bank178632 +178633 POINT(40.68593223154639 73.28592811407262) bank178633 +178634 POINT(40.78070057621296 73.40196292611793) bank178634 +178635 POINT(40.01181710726894 73.8887633883982) bank178635 +178636 POINT(40.15590865692442 73.03398758010042) bank178636 +178637 POINT(41.450977634812034 73.25054253229455) bank178637 +178638 POINT(40.55729876788254 73.7710437730857) bank178638 +178639 POINT(40.698499627510806 74.93263999706211) bank178639 +178640 POINT(40.71115582608464 74.00612062231686) bank178640 +178641 POINT(40.130748029093425 73.01098715762107) bank178641 +178642 POINT(41.12194138368172 74.3646887742392) bank178642 +178643 POINT(39.866411661182454 73.40428938900853) bank178643 +178644 POINT(41.65577750384819 73.77141444292145) bank178644 +178645 POINT(40.36000175298213 74.23042822354469) bank178645 +178646 POINT(40.824616155305776 73.20285136707324) bank178646 +178647 POINT(39.73524105337015 73.87938619438836) bank178647 +178648 POINT(39.86441268641643 74.41247008072969) bank178648 +178649 POINT(39.896593070075525 73.20317464352648) bank178649 +178650 POINT(41.57107031856825 73.10207504694509) bank178650 +178651 POINT(40.012683427468396 74.72129069822665) bank178651 +178652 POINT(41.330172166059775 74.66265076159368) bank178652 +178653 POINT(41.1896605983777 73.94366459138502) bank178653 +178654 POINT(41.49034181677543 73.84037303425445) bank178654 +178655 POINT(40.21788058887758 74.46149623638802) bank178655 +178656 POINT(40.45317385644109 74.54946769195703) bank178656 +178657 POINT(40.420327894475015 73.91438044464286) bank178657 +178658 POINT(40.3669883962493 73.19484600172285) bank178658 +178659 POINT(40.269354449538824 73.83229988185124) bank178659 +178660 POINT(40.93278976528123 74.82965886046041) bank178660 +178661 POINT(40.61939447445054 73.2409162930325) bank178661 +178662 POINT(40.834195971066556 74.84245684072174) bank178662 +178663 POINT(41.02091118471321 73.98832331563406) bank178663 +178664 POINT(40.558558784253094 73.6060022287318) bank178664 +178665 POINT(40.84281563326966 74.91164931111425) bank178665 +178666 POINT(41.65321519274255 74.66625501077276) bank178666 +178667 POINT(41.446629376824454 74.01382864578034) bank178667 +178668 POINT(39.768794968588644 73.72658918334837) bank178668 +178669 POINT(40.60150872008579 73.86578448607432) bank178669 +178670 POINT(41.451990274466326 74.65934363817169) bank178670 +178671 POINT(40.770106951301265 73.25339895313326) bank178671 +178672 POINT(41.41201161478838 73.94941937354832) bank178672 +178673 POINT(40.25812009168814 73.09650790476746) bank178673 +178674 POINT(41.21772053712853 73.62692363381771) bank178674 +178675 POINT(41.35588767030937 74.26306572766362) bank178675 +178676 POINT(41.06164681613899 74.95288387234532) bank178676 +178677 POINT(41.24760572934337 73.53280668294583) bank178677 +178678 POINT(41.65490588190365 73.28562749129264) bank178678 +178679 POINT(40.52359347632176 73.73007446922716) bank178679 +178680 POINT(39.902686969985844 73.44118418734581) bank178680 +178681 POINT(40.43644622978541 74.2325384997381) bank178681 +178682 POINT(41.077169984678605 74.14511693331366) bank178682 +178683 POINT(41.30930509350427 73.57502093190165) bank178683 +178684 POINT(40.48593306530978 74.64129028567906) bank178684 +178685 POINT(40.40573047475253 73.84051839939157) bank178685 +178686 POINT(40.045890826654734 73.41571113541845) bank178686 +178687 POINT(40.4587136872704 73.51410328997407) bank178687 +178688 POINT(40.0433528835508 73.33435259893324) bank178688 +178689 POINT(39.78604408217229 73.27661871659424) bank178689 +178690 POINT(41.693954057959125 74.33185440798711) bank178690 +178691 POINT(40.44789489723093 73.1572997183204) bank178691 +178692 POINT(40.13966263229036 73.7161182745362) bank178692 +178693 POINT(39.72427822465194 73.01160520873441) bank178693 +178694 POINT(40.72598435797971 74.63401788463455) bank178694 +178695 POINT(40.39668215231026 73.96010897781676) bank178695 +178696 POINT(41.0920379146993 73.11700307717193) bank178696 +178697 POINT(41.36979660272218 74.97208348784542) bank178697 +178698 POINT(40.07772891699058 74.49371029226498) bank178698 +178699 POINT(39.72088013438919 74.21208024209905) bank178699 +178700 POINT(40.701371078090055 73.9252396479469) bank178700 +178701 POINT(39.71767032403976 74.3458239286693) bank178701 +178702 POINT(39.893375039297 73.76380361675098) bank178702 +178703 POINT(40.72046727803289 74.62091578465896) bank178703 +178704 POINT(40.35089572878993 73.31334552456691) bank178704 +178705 POINT(41.435832445250085 74.03087825847801) bank178705 +178706 POINT(39.7553038494807 73.91388242098749) bank178706 +178707 POINT(40.07782301108895 73.30729120896072) bank178707 +178708 POINT(41.238246123654434 73.01588828535836) bank178708 +178709 POINT(40.67276930164054 73.71572292823136) bank178709 +178710 POINT(40.77447101431362 73.10199338075994) bank178710 +178711 POINT(40.39779509377096 74.06160613657349) bank178711 +178712 POINT(41.127346894637796 74.03601482592678) bank178712 +178713 POINT(40.318802758817085 73.46636751449336) bank178713 +178714 POINT(41.49698504270302 73.40233837714081) bank178714 +178715 POINT(41.49601353015199 74.56916785824313) bank178715 +178716 POINT(40.222428427518636 74.2969791027228) bank178716 +178717 POINT(40.30931584510569 73.74594518730106) bank178717 +178718 POINT(40.981422234510255 73.9958518930377) bank178718 +178719 POINT(40.96056255515516 73.8130147858057) bank178719 +178720 POINT(40.60277979628678 73.2437440580136) bank178720 +178721 POINT(41.35388297592563 73.38834464298456) bank178721 +178722 POINT(40.69450835128342 73.61016795994847) bank178722 +178723 POINT(40.54346557094855 73.31615215843063) bank178723 +178724 POINT(40.77411461461159 73.18715594907871) bank178724 +178725 POINT(40.29226030776492 73.84016499073047) bank178725 +178726 POINT(39.963897376442894 73.67815114477665) bank178726 +178727 POINT(40.29865883401888 74.4752760250575) bank178727 +178728 POINT(39.85396572593889 73.9327146517529) bank178728 +178729 POINT(41.600002397155876 74.33404421874681) bank178729 +178730 POINT(40.27789123177194 74.8641003105456) bank178730 +178731 POINT(41.20005663051997 73.07408100998956) bank178731 +178732 POINT(41.420445022263245 73.37900692146947) bank178732 +178733 POINT(40.6526285637149 74.78965186401324) bank178733 +178734 POINT(41.5152786034502 74.75356098534344) bank178734 +178735 POINT(40.01626038365064 74.94486589820917) bank178735 +178736 POINT(41.41073115941165 73.60116890065913) bank178736 +178737 POINT(40.85095200204315 74.71400318166519) bank178737 +178738 POINT(39.76551298603086 73.31636288159707) bank178738 +178739 POINT(40.59320463951335 73.53260719354321) bank178739 +178740 POINT(41.66679055799713 74.99654697627192) bank178740 +178741 POINT(39.94027190081774 73.60731486680402) bank178741 +178742 POINT(40.899519040568315 74.80327086349614) bank178742 +178743 POINT(40.08801884146019 73.12477428116426) bank178743 +178744 POINT(41.45681988527231 73.78914446483753) bank178744 +178745 POINT(41.365622386951095 73.0972747941295) bank178745 +178746 POINT(39.79630154301453 73.68190946617652) bank178746 +178747 POINT(39.9145215833366 74.25911696336986) bank178747 +178748 POINT(40.32574596537284 73.10965981142745) bank178748 +178749 POINT(40.63122759469894 74.6257959924348) bank178749 +178750 POINT(41.320863727577006 73.58942789870999) bank178750 +178751 POINT(40.28728357590425 73.82910577919147) bank178751 +178752 POINT(39.84447786811986 74.18716633280243) bank178752 +178753 POINT(40.916832973012 73.5087735947633) bank178753 +178754 POINT(41.06079568286982 73.65065271054189) bank178754 +178755 POINT(41.24226382296708 73.53247312491933) bank178755 +178756 POINT(40.917184597728046 73.65761101365251) bank178756 +178757 POINT(39.832689633059815 74.26286773319161) bank178757 +178758 POINT(41.64502859999717 74.68332925024292) bank178758 +178759 POINT(41.408394175052315 74.42089195663583) bank178759 +178760 POINT(41.187948223043115 74.97706358740986) bank178760 +178761 POINT(41.59430237415828 74.20607989986979) bank178761 +178762 POINT(41.6017975560802 73.1338503145979) bank178762 +178763 POINT(41.701863075105166 74.24848562383308) bank178763 +178764 POINT(39.85329194509128 74.79930944817468) bank178764 +178765 POINT(41.168644497481196 74.0585916796184) bank178765 +178766 POINT(41.294811351566395 73.23694065270105) bank178766 +178767 POINT(41.5464216704177 73.42325844980228) bank178767 +178768 POINT(40.54060016121114 74.4602783257256) bank178768 +178769 POINT(40.453361504972996 73.63846207255189) bank178769 +178770 POINT(40.744352873248076 74.41963309876512) bank178770 +178771 POINT(40.21780997218522 74.00380903233142) bank178771 +178772 POINT(41.5673532549876 74.67459257875417) bank178772 +178773 POINT(41.44634520639477 73.84757300722225) bank178773 +178774 POINT(40.381870357712 73.23890282078752) bank178774 +178775 POINT(39.97663664468402 74.16887310913437) bank178775 +178776 POINT(41.49345331962357 74.96995595795038) bank178776 +178777 POINT(39.990931100146405 73.19733725876434) bank178777 +178778 POINT(40.17181843858403 73.72307592806723) bank178778 +178779 POINT(41.54701051615758 73.80702758602814) bank178779 +178780 POINT(40.38554099476699 74.46574370580426) bank178780 +178781 POINT(41.502709850531154 73.80989624720058) bank178781 +178782 POINT(40.510076835706016 74.68452456985598) bank178782 +178783 POINT(39.73418387451026 74.0556747885448) bank178783 +178784 POINT(40.30112583880791 74.28432881500189) bank178784 +178785 POINT(41.13665429365221 73.86752861259828) bank178785 +178786 POINT(40.76138047648463 73.35166067512044) bank178786 +178787 POINT(39.823702987630995 73.81135511409144) bank178787 +178788 POINT(40.41784746355092 73.24386528976606) bank178788 +178789 POINT(40.707873472630006 73.88252725360509) bank178789 +178790 POINT(40.356510118464826 74.54978551158574) bank178790 +178791 POINT(41.37333661468751 73.87601275562604) bank178791 +178792 POINT(39.92468063205035 73.93834538661132) bank178792 +178793 POINT(41.39012656803488 74.56009237966605) bank178793 +178794 POINT(40.4723112677576 73.10552774541625) bank178794 +178795 POINT(41.489703682951564 74.79456105111575) bank178795 +178796 POINT(40.90393417241794 73.35924547510884) bank178796 +178797 POINT(40.22551410719115 73.39150315210438) bank178797 +178798 POINT(41.691162811992825 73.93899702926821) bank178798 +178799 POINT(40.53746707375555 73.22795275556614) bank178799 +178800 POINT(41.70377699917748 74.21379847274693) bank178800 +178801 POINT(40.883104753752676 73.01349465969385) bank178801 +178802 POINT(40.735358972799055 74.92982657505391) bank178802 +178803 POINT(40.45851342066927 74.605221870814) bank178803 +178804 POINT(41.09392828279462 74.81343672683666) bank178804 +178805 POINT(40.563859800679836 73.36998862232478) bank178805 +178806 POINT(40.06233545855215 73.360409390467) bank178806 +178807 POINT(40.1455697592901 74.45311510831023) bank178807 +178808 POINT(41.03672660310053 73.72832825555453) bank178808 +178809 POINT(41.54623688205494 73.87816524683784) bank178809 +178810 POINT(41.15204208661623 74.86297584547471) bank178810 +178811 POINT(41.629605928684754 73.69291204037117) bank178811 +178812 POINT(41.52494657423977 73.0939797682782) bank178812 +178813 POINT(40.22632467208497 74.86861337097375) bank178813 +178814 POINT(41.068490368024726 73.9204068507733) bank178814 +178815 POINT(40.7528968612664 74.79482037475853) bank178815 +178816 POINT(39.92506964961113 73.15979936973635) bank178816 +178817 POINT(41.04139754754457 73.26334284120288) bank178817 +178818 POINT(41.10423698020992 73.18241367442442) bank178818 +178819 POINT(40.70358920418925 73.97033615224105) bank178819 +178820 POINT(40.53301672725985 73.73911417955473) bank178820 +178821 POINT(40.488181765157 73.4912836032853) bank178821 +178822 POINT(41.457421466722344 73.2791618107871) bank178822 +178823 POINT(40.28005850994699 73.03847329200357) bank178823 +178824 POINT(40.310408498131835 73.02798581547998) bank178824 +178825 POINT(40.840383928425 74.02322283974378) bank178825 +178826 POINT(40.702569089318146 74.64206439168579) bank178826 +178827 POINT(39.77449490436654 74.72418107236862) bank178827 +178828 POINT(39.797496875676046 74.19415483510912) bank178828 +178829 POINT(40.60981387218062 74.56372034156999) bank178829 +178830 POINT(41.67019124501893 74.99947236805318) bank178830 +178831 POINT(40.20239295307563 73.85556799108525) bank178831 +178832 POINT(40.663344215912026 74.71780241479439) bank178832 +178833 POINT(40.78484070580201 73.91522038506079) bank178833 +178834 POINT(41.69659804905916 73.72422804280548) bank178834 +178835 POINT(40.23998407779715 74.22587122054112) bank178835 +178836 POINT(39.75251500022835 73.52965807325802) bank178836 +178837 POINT(39.82936667042327 73.67480486359189) bank178837 +178838 POINT(41.26735414345331 73.42437215434593) bank178838 +178839 POINT(40.549816934889705 74.0889803081089) bank178839 +178840 POINT(41.086920953384954 74.66572360936088) bank178840 +178841 POINT(40.10961809407249 73.03260148284356) bank178841 +178842 POINT(41.10148968048896 73.96503666255025) bank178842 +178843 POINT(40.6491127881863 73.94489613387893) bank178843 +178844 POINT(39.8359193483518 74.37171399847995) bank178844 +178845 POINT(41.40708138994849 74.79628604006427) bank178845 +178846 POINT(40.57077450648825 74.96865852554885) bank178846 +178847 POINT(39.89462525173484 74.93439061759496) bank178847 +178848 POINT(41.55761379853913 73.94164164352776) bank178848 +178849 POINT(41.37042782141057 74.80155421866273) bank178849 +178850 POINT(40.07030061781989 74.44574697611772) bank178850 +178851 POINT(41.49767701229492 74.04119724101604) bank178851 +178852 POINT(40.495357229387324 74.46752632097126) bank178852 +178853 POINT(41.1895759895187 73.6639011687878) bank178853 +178854 POINT(39.949597814781995 74.278552680547) bank178854 +178855 POINT(41.45717790495426 74.0001916975208) bank178855 +178856 POINT(40.75496308322833 74.49609359402292) bank178856 +178857 POINT(40.087238727968234 74.29116470243193) bank178857 +178858 POINT(39.98561874039268 73.58509431571828) bank178858 +178859 POINT(41.39781714263361 73.20460151586767) bank178859 +178860 POINT(40.99935231612518 73.1410179928556) bank178860 +178861 POINT(40.907388683895014 74.41665794966936) bank178861 +178862 POINT(41.18273428312688 74.40294548661413) bank178862 +178863 POINT(40.309473656467325 73.19418219302318) bank178863 +178864 POINT(41.25049917053678 73.8166388226148) bank178864 +178865 POINT(40.93213888755304 74.27664837694152) bank178865 +178866 POINT(40.89781563759302 74.0079877758109) bank178866 +178867 POINT(40.479263452647274 74.90316659164286) bank178867 +178868 POINT(41.711875975500874 74.64475947855219) bank178868 +178869 POINT(40.72277746256433 73.53037880064814) bank178869 +178870 POINT(39.72563253316231 73.74890805087034) bank178870 +178871 POINT(41.52630521661498 74.58187559220643) bank178871 +178872 POINT(40.46252062853327 74.30678172252227) bank178872 +178873 POINT(39.97195767341046 74.89254731337434) bank178873 +178874 POINT(40.559429772357824 74.30253842937488) bank178874 +178875 POINT(40.210927567694426 74.47653588152158) bank178875 +178876 POINT(40.19696334026015 73.74265867933278) bank178876 +178877 POINT(41.5023606637966 73.85599832321329) bank178877 +178878 POINT(40.89054574104824 74.03925363054819) bank178878 +178879 POINT(40.519608653185706 73.91128852258986) bank178879 +178880 POINT(41.00853909494555 73.02993734427189) bank178880 +178881 POINT(41.20086447993839 73.72261460076162) bank178881 +178882 POINT(41.01967291287442 74.35326527293779) bank178882 +178883 POINT(40.74113374256752 74.51193958671496) bank178883 +178884 POINT(41.300295879379256 73.13926385584678) bank178884 +178885 POINT(40.580556768663875 74.63220177103503) bank178885 +178886 POINT(40.95871206135542 73.0650063898809) bank178886 +178887 POINT(39.71423749237244 73.73088960108424) bank178887 +178888 POINT(41.6673470280037 74.52216932568409) bank178888 +178889 POINT(40.910605034760344 74.67636467097526) bank178889 +178890 POINT(41.57436024688265 73.49956764389187) bank178890 +178891 POINT(39.86466195092791 74.72581517657738) bank178891 +178892 POINT(41.39685558674314 74.98446578876901) bank178892 +178893 POINT(39.946741961518136 73.23484106290908) bank178893 +178894 POINT(40.914467957763904 73.60383161551749) bank178894 +178895 POINT(40.48516895428528 73.37363465031885) bank178895 +178896 POINT(40.013983140169465 74.1673631218433) bank178896 +178897 POINT(39.88401318297098 74.86736352127514) bank178897 +178898 POINT(40.64905059375011 73.13750098716945) bank178898 +178899 POINT(40.947935756722714 74.32194759551624) bank178899 +178900 POINT(40.93616787826788 74.28138374637356) bank178900 +178901 POINT(40.30815877179841 73.32694009641868) bank178901 +178902 POINT(41.41424170493154 73.97334083565714) bank178902 +178903 POINT(41.608626032436476 73.41139699759404) bank178903 +178904 POINT(40.703984505837056 74.47049294838395) bank178904 +178905 POINT(40.66884721232545 74.63106782969446) bank178905 +178906 POINT(41.499387331443614 73.63149336282684) bank178906 +178907 POINT(40.37809207483973 74.44703990739647) bank178907 +178908 POINT(41.53498836331898 74.23927273937227) bank178908 +178909 POINT(40.16805641327411 73.37123462589084) bank178909 +178910 POINT(41.48508309835865 73.27895668294119) bank178910 +178911 POINT(40.91140820466563 73.73786548922295) bank178911 +178912 POINT(40.100711288715544 74.90351294304912) bank178912 +178913 POINT(41.6990833787772 73.76554073549458) bank178913 +178914 POINT(39.998274211643086 73.79274503605721) bank178914 +178915 POINT(40.028036457112066 73.91091276982283) bank178915 +178916 POINT(40.025261219234835 73.59539848430792) bank178916 +178917 POINT(41.30145319442112 74.77389574289799) bank178917 +178918 POINT(40.566810008943094 73.59918749664284) bank178918 +178919 POINT(41.62144784650301 73.6327353458526) bank178919 +178920 POINT(40.49277085424156 73.58066143668925) bank178920 +178921 POINT(41.13286823538554 73.95392309002429) bank178921 +178922 POINT(40.688002715960614 73.46390929078655) bank178922 +178923 POINT(41.343548761957564 73.97224220149025) bank178923 +178924 POINT(40.529965118013294 73.3349359844072) bank178924 +178925 POINT(40.97436439846199 74.04759214123655) bank178925 +178926 POINT(41.29583746593495 73.29815792940602) bank178926 +178927 POINT(40.05801399146292 74.55474446032714) bank178927 +178928 POINT(40.6131766289919 74.74467612052624) bank178928 +178929 POINT(39.73500895874044 74.38628085957953) bank178929 +178930 POINT(41.3327050606197 74.3154202090103) bank178930 +178931 POINT(40.491199940793884 74.45755895483491) bank178931 +178932 POINT(41.03127630353337 73.9155194602583) bank178932 +178933 POINT(40.865236973650376 74.4193704782522) bank178933 +178934 POINT(40.415064764999244 73.19134315872411) bank178934 +178935 POINT(40.6695427830452 73.75324934009672) bank178935 +178936 POINT(41.67550697272811 73.81663150089004) bank178936 +178937 POINT(39.7251775993295 74.37520119944192) bank178937 +178938 POINT(40.64502982705272 73.8364503133097) bank178938 +178939 POINT(40.619403236150795 73.73088053325212) bank178939 +178940 POINT(41.23768822475376 74.60375435375687) bank178940 +178941 POINT(40.86017044256131 74.47289817825633) bank178941 +178942 POINT(40.20730322048235 74.82251228454919) bank178942 +178943 POINT(41.45842015958141 74.72801574195476) bank178943 +178944 POINT(41.61169020552381 73.56596815473469) bank178944 +178945 POINT(40.149837572900786 74.61081925592319) bank178945 +178946 POINT(40.40450672542444 75.00135821596255) bank178946 +178947 POINT(40.56739932169389 73.946957526344) bank178947 +178948 POINT(41.64531861166469 73.28978716580407) bank178948 +178949 POINT(40.49496161330586 74.93029131409287) bank178949 +178950 POINT(40.57035996724924 74.6332129512014) bank178950 +178951 POINT(40.49738336077154 73.47349250626228) bank178951 +178952 POINT(41.01381755473312 73.53468894519673) bank178952 +178953 POINT(41.58363742786371 73.05039254201279) bank178953 +178954 POINT(40.7187169541379 74.74788787676682) bank178954 +178955 POINT(40.509275333708224 73.6413145827406) bank178955 +178956 POINT(41.25788243934585 73.03393646757938) bank178956 +178957 POINT(40.49204751802369 73.67567748891085) bank178957 +178958 POINT(40.40439748643259 74.25567037252414) bank178958 +178959 POINT(40.95153236862761 74.78337319667486) bank178959 +178960 POINT(40.771241033871156 74.87801923655879) bank178960 +178961 POINT(40.21332276904154 73.53575476596224) bank178961 +178962 POINT(40.09182070937969 74.87845966585634) bank178962 +178963 POINT(40.343483837434846 74.6570742397988) bank178963 +178964 POINT(40.71010691126907 74.78670491829067) bank178964 +178965 POINT(40.64120340162619 74.0322512884005) bank178965 +178966 POINT(40.22641423609251 73.36345254146744) bank178966 +178967 POINT(40.47206664602057 73.56446782483948) bank178967 +178968 POINT(40.70383486478267 73.5825221486174) bank178968 +178969 POINT(39.99717355515379 73.12022358476325) bank178969 +178970 POINT(41.136641838641154 74.62045239555934) bank178970 +178971 POINT(40.457221228119664 73.70669235700852) bank178971 +178972 POINT(40.01700539526288 73.27532012746963) bank178972 +178973 POINT(40.44148768924492 74.11909131755071) bank178973 +178974 POINT(41.64124353318581 73.72339596520267) bank178974 +178975 POINT(40.13593215412121 74.47650218934557) bank178975 +178976 POINT(40.53016782753082 73.73621180105451) bank178976 +178977 POINT(40.46364292201209 74.24621903613287) bank178977 +178978 POINT(41.531912709213046 73.64112882453365) bank178978 +178979 POINT(40.4362145600476 73.22854426462034) bank178979 +178980 POINT(40.75904820779799 73.83283555766972) bank178980 +178981 POINT(41.22207093351652 73.73190572907984) bank178981 +178982 POINT(41.54633081918211 74.96085161850804) bank178982 +178983 POINT(40.5420749120818 73.82319540591362) bank178983 +178984 POINT(40.693321008337676 73.371396731857) bank178984 +178985 POINT(41.322699216410584 74.76420833023607) bank178985 +178986 POINT(41.127269403046434 74.30702726696886) bank178986 +178987 POINT(40.648712945925524 73.3197829166249) bank178987 +178988 POINT(39.975816872148265 73.05466154500607) bank178988 +178989 POINT(40.46143240453203 73.15402940336126) bank178989 +178990 POINT(40.271688409532665 73.08757049192174) bank178990 +178991 POINT(41.03135036078834 73.18507422394208) bank178991 +178992 POINT(41.25669170044225 74.05777839724858) bank178992 +178993 POINT(41.3090677300382 74.71595730259041) bank178993 +178994 POINT(41.205411361056825 74.94517076767683) bank178994 +178995 POINT(40.75883042280158 74.18523750277865) bank178995 +178996 POINT(41.52170576864467 73.85545347048316) bank178996 +178997 POINT(39.84450116988287 74.16022529422105) bank178997 +178998 POINT(41.519218126377524 73.36431858306437) bank178998 +178999 POINT(40.000340567128205 74.33124856321248) bank178999 +179000 POINT(40.086297706125784 73.15264035252353) bank179000 +179001 POINT(40.97965216385567 73.14218143265967) bank179001 +179002 POINT(41.441870095771726 74.327083506347) bank179002 +179003 POINT(40.52436497001538 74.51916902269377) bank179003 +179004 POINT(41.71225171996805 73.7335813219626) bank179004 +179005 POINT(40.010245679229016 74.99585191661383) bank179005 +179006 POINT(41.06141748879719 73.6362724740665) bank179006 +179007 POINT(40.293723943591154 74.13753509996404) bank179007 +179008 POINT(39.96184348025353 73.82733284751504) bank179008 +179009 POINT(40.25299319511282 74.33425373270167) bank179009 +179010 POINT(40.343513077907204 73.61798052258364) bank179010 +179011 POINT(40.524925351761915 74.16334744239325) bank179011 +179012 POINT(40.88794123852886 74.90969466260141) bank179012 +179013 POINT(39.91117322008884 74.68408124682533) bank179013 +179014 POINT(40.22587839032495 74.11828186281306) bank179014 +179015 POINT(40.043668964427454 73.33520547998154) bank179015 +179016 POINT(40.34629989803936 73.48324229806727) bank179016 +179017 POINT(40.98367955800988 74.63339972521214) bank179017 +179018 POINT(41.069253685896534 73.10191915990201) bank179018 +179019 POINT(41.17405290691343 73.8080526702588) bank179019 +179020 POINT(41.624866752343436 73.96809897173927) bank179020 +179021 POINT(40.06219798284628 74.75733709278578) bank179021 +179022 POINT(40.41838752106592 74.53722897203475) bank179022 +179023 POINT(41.30591305597247 74.49880154852977) bank179023 +179024 POINT(41.650130870962016 73.09809624540263) bank179024 +179025 POINT(39.83083803028506 74.73650496996275) bank179025 +179026 POINT(40.79598045121414 74.8393994246761) bank179026 +179027 POINT(41.255323105984786 73.57486295221373) bank179027 +179028 POINT(41.39348258142337 74.60834056740342) bank179028 +179029 POINT(40.7563905520686 74.44798088923775) bank179029 +179030 POINT(40.901317028116885 74.99589226410617) bank179030 +179031 POINT(39.906290941977375 73.46507598539537) bank179031 +179032 POINT(41.219207242267515 74.61794659115837) bank179032 +179033 POINT(40.52461266182138 73.5167926761422) bank179033 +179034 POINT(41.47208277695552 74.88930311664767) bank179034 +179035 POINT(41.6372144359836 74.71982179126125) bank179035 +179036 POINT(40.22783331968921 73.12744396977328) bank179036 +179037 POINT(41.37712246977927 73.51361240515375) bank179037 +179038 POINT(40.738198042495775 74.75242140138707) bank179038 +179039 POINT(40.085527007751075 73.90233742053326) bank179039 +179040 POINT(40.255359969980304 73.29843755947965) bank179040 +179041 POINT(41.28587652320256 74.33065532213409) bank179041 +179042 POINT(40.75601039750589 74.07785654482831) bank179042 +179043 POINT(40.98047378502765 74.87587578432093) bank179043 +179044 POINT(41.46263202300585 74.25791177919776) bank179044 +179045 POINT(41.0359681162086 73.84149673655375) bank179045 +179046 POINT(41.35636471870606 74.64931473401656) bank179046 +179047 POINT(41.693731408824135 73.03324980203023) bank179047 +179048 POINT(41.44355019425819 73.35611963118954) bank179048 +179049 POINT(40.55690054252748 74.73236321052866) bank179049 +179050 POINT(40.010472792799426 73.74803518949336) bank179050 +179051 POINT(40.47633279860914 74.33801545509087) bank179051 +179052 POINT(39.802640713057926 74.5237414560152) bank179052 +179053 POINT(41.154251554886486 73.91845968792984) bank179053 +179054 POINT(40.800225592588454 74.5220393936728) bank179054 +179055 POINT(40.96955930213216 73.4761655895911) bank179055 +179056 POINT(40.24631684725193 74.85726861247242) bank179056 +179057 POINT(41.140304772353744 73.90779904001275) bank179057 +179058 POINT(40.444971161689416 73.99164254802426) bank179058 +179059 POINT(40.006785142756 74.35510997012828) bank179059 +179060 POINT(40.10732128536419 73.30042102794351) bank179060 +179061 POINT(41.50420460763898 73.11545657236326) bank179061 +179062 POINT(40.77519293572559 73.87810571342536) bank179062 +179063 POINT(40.573531729734 73.05670998332997) bank179063 +179064 POINT(40.44783769900369 73.17139064605419) bank179064 +179065 POINT(41.18501529264949 74.23192617507145) bank179065 +179066 POINT(41.53068854860706 73.62080350949246) bank179066 +179067 POINT(39.76470895584379 73.88214673043377) bank179067 +179068 POINT(41.57800693284208 73.39457499548546) bank179068 +179069 POINT(40.811249732136574 73.38556866404814) bank179069 +179070 POINT(40.853056014373024 74.98980619541788) bank179070 +179071 POINT(40.78786902707458 73.40449706065952) bank179071 +179072 POINT(40.3444295939473 73.26684709095034) bank179072 +179073 POINT(40.379183224195145 74.68188968984072) bank179073 +179074 POINT(40.70168789391704 73.97163082527457) bank179074 +179075 POINT(40.504058144177826 74.31081229809126) bank179075 +179076 POINT(40.12402110935531 74.1751084275428) bank179076 +179077 POINT(40.34338027253701 74.22304812471823) bank179077 +179078 POINT(40.32179538618441 74.56087961887201) bank179078 +179079 POINT(41.42341900730614 73.64471098447954) bank179079 +179080 POINT(41.13693391023225 74.80597029951244) bank179080 +179081 POINT(41.23517101413044 74.15361844569252) bank179081 +179082 POINT(39.98855919206475 74.9093641290385) bank179082 +179083 POINT(40.28910713119651 73.16874448047955) bank179083 +179084 POINT(40.31093162994678 73.54183740622078) bank179084 +179085 POINT(39.8196087069613 74.96253587852972) bank179085 +179086 POINT(41.40140636293726 74.9047816802747) bank179086 +179087 POINT(40.80941254880055 74.82140351202358) bank179087 +179088 POINT(40.159328110565134 74.56118175072992) bank179088 +179089 POINT(40.8279920907788 73.87754741994189) bank179089 +179090 POINT(40.034770030811146 73.47075598786961) bank179090 +179091 POINT(41.12054170973945 74.23915857971255) bank179091 +179092 POINT(40.457040240399266 74.23100611358113) bank179092 +179093 POINT(41.67593132151503 74.14921741448194) bank179093 +179094 POINT(41.692212268772955 73.70780766259632) bank179094 +179095 POINT(41.42495781478836 73.83689766758256) bank179095 +179096 POINT(41.6749595543203 73.96368886234762) bank179096 +179097 POINT(41.12381374601323 73.75889412470366) bank179097 +179098 POINT(41.24814236361528 73.88713594477501) bank179098 +179099 POINT(40.03238698078735 74.92604386841491) bank179099 +179100 POINT(40.716976972752995 74.29693628604515) bank179100 +179101 POINT(40.13106963886408 73.4398379617007) bank179101 +179102 POINT(41.091485055852594 73.20221204892249) bank179102 +179103 POINT(41.03003436435504 73.7282308080062) bank179103 +179104 POINT(41.524217974493574 73.32667995538732) bank179104 +179105 POINT(40.92057379741372 74.88715884257498) bank179105 +179106 POINT(40.36137733461021 74.06236532427023) bank179106 +179107 POINT(40.153305701339384 74.59626185340169) bank179107 +179108 POINT(41.4871194491344 73.10054451729656) bank179108 +179109 POINT(40.90416932236645 73.71221959719826) bank179109 +179110 POINT(40.67874127874394 73.98521493334827) bank179110 +179111 POINT(41.09328695106151 74.9540285418755) bank179111 +179112 POINT(40.893741764208215 73.80234413436496) bank179112 +179113 POINT(39.9542071268103 73.47124863607955) bank179113 +179114 POINT(40.85819632252328 74.66746893118093) bank179114 +179115 POINT(40.84307506469379 74.24953204633387) bank179115 +179116 POINT(40.7634976465334 73.58547985386778) bank179116 +179117 POINT(40.40335679204659 73.96772588219142) bank179117 +179118 POINT(41.194849958900775 74.4350676671055) bank179118 +179119 POINT(41.445305682274864 73.30961194415988) bank179119 +179120 POINT(41.297386181445475 74.52222929582643) bank179120 +179121 POINT(39.74140265017763 73.6410166663882) bank179121 +179122 POINT(40.79535249503564 73.12249140960496) bank179122 +179123 POINT(41.032152450196456 73.76313765556233) bank179123 +179124 POINT(40.94876140204357 74.33618713562555) bank179124 +179125 POINT(39.8359461205325 74.19247216621501) bank179125 +179126 POINT(40.8066418400447 74.41470772824852) bank179126 +179127 POINT(40.03316321978788 73.3202376547853) bank179127 +179128 POINT(40.127162497290186 73.647355169518) bank179128 +179129 POINT(41.5281593680891 74.6583688906739) bank179129 +179130 POINT(41.61557121272067 73.04405602195492) bank179130 +179131 POINT(40.532305823651775 73.8397871746451) bank179131 +179132 POINT(40.30382612963113 74.50104601035139) bank179132 +179133 POINT(41.67015928913064 74.99372414238944) bank179133 +179134 POINT(40.447776665941575 74.6900369449406) bank179134 +179135 POINT(41.035717793346244 74.53048481634326) bank179135 +179136 POINT(39.83405234108585 74.68912937028348) bank179136 +179137 POINT(40.85445090721126 73.49710130140551) bank179137 +179138 POINT(41.59066202790739 73.79255906850325) bank179138 +179139 POINT(40.38335821514762 74.82074823680489) bank179139 +179140 POINT(40.45248276696623 74.151743720828) bank179140 +179141 POINT(39.73058523047051 73.77915579426873) bank179141 +179142 POINT(40.324198661752845 73.6981890792523) bank179142 +179143 POINT(40.44657550932925 74.27084946895876) bank179143 +179144 POINT(39.88581768138469 74.69685291093604) bank179144 +179145 POINT(39.83658012892811 73.10555208231153) bank179145 +179146 POINT(40.92959330127211 74.86209308190816) bank179146 +179147 POINT(39.81588127625062 73.16696087973787) bank179147 +179148 POINT(39.898177687418126 74.7588385898091) bank179148 +179149 POINT(41.54935612797891 73.85026861292238) bank179149 +179150 POINT(39.737456861855875 73.87223301723091) bank179150 +179151 POINT(41.36704607372584 74.35184562489482) bank179151 +179152 POINT(40.36471374529289 74.43738432570903) bank179152 +179153 POINT(41.0399758017747 73.84456043167101) bank179153 +179154 POINT(41.532535425185365 73.78796584728015) bank179154 +179155 POINT(40.24516622330241 74.76675595238704) bank179155 +179156 POINT(39.991687729438446 74.86601307128052) bank179156 +179157 POINT(40.74556944194675 74.93362533609672) bank179157 +179158 POINT(41.4608358633572 74.39489468510183) bank179158 +179159 POINT(40.86202474325042 74.14532467455682) bank179159 +179160 POINT(39.76586387986462 73.30341840699145) bank179160 +179161 POINT(41.47000816816199 74.19739132772578) bank179161 +179162 POINT(41.295906489534325 74.56800297003578) bank179162 +179163 POINT(41.244262819711565 74.45095371827118) bank179163 +179164 POINT(40.58529998357189 74.05336303118811) bank179164 +179165 POINT(39.938884544499665 74.02865266113989) bank179165 +179166 POINT(40.23400022719017 73.63665206718221) bank179166 +179167 POINT(41.34834137528752 73.96262791722785) bank179167 +179168 POINT(41.144970304770396 73.55823016157254) bank179168 +179169 POINT(40.01694304817345 73.56514744147103) bank179169 +179170 POINT(40.57885154961155 73.96016028614167) bank179170 +179171 POINT(41.2858276178931 73.42382241662591) bank179171 +179172 POINT(40.2350958015084 73.04620895890774) bank179172 +179173 POINT(40.0455884154389 74.06174667095715) bank179173 +179174 POINT(40.487153802939496 73.40767753205982) bank179174 +179175 POINT(41.41148484002396 74.51072036792455) bank179175 +179176 POINT(40.48372506143832 74.52909796321664) bank179176 +179177 POINT(41.28408091704609 74.2582328566046) bank179177 +179178 POINT(39.88201580471584 73.26748660861952) bank179178 +179179 POINT(41.67723147199462 74.66973277033027) bank179179 +179180 POINT(40.748352730148824 74.63393716535806) bank179180 +179181 POINT(40.50200490399133 73.42604387286042) bank179181 +179182 POINT(40.063865628959604 74.23793280096812) bank179182 +179183 POINT(41.26417950465977 74.84228204643988) bank179183 +179184 POINT(41.22681068765377 73.7172725345993) bank179184 +179185 POINT(39.90522632540356 74.26029757932459) bank179185 +179186 POINT(41.69730346285428 73.02999348021518) bank179186 +179187 POINT(40.330277470214156 73.96235664544713) bank179187 +179188 POINT(40.20459060061491 73.98454245383894) bank179188 +179189 POINT(39.775516044862776 73.46567661674992) bank179189 +179190 POINT(40.25395471916702 73.67522440429728) bank179190 +179191 POINT(39.9641002919135 74.16665479476949) bank179191 +179192 POINT(40.498283533978366 74.8429806616983) bank179192 +179193 POINT(41.54849975062141 74.32736955671862) bank179193 +179194 POINT(40.533548067978295 73.35767477314769) bank179194 +179195 POINT(41.0139932829738 74.4117235097798) bank179195 +179196 POINT(41.4096464629207 74.68314440670913) bank179196 +179197 POINT(41.13147645940252 73.22295236704042) bank179197 +179198 POINT(40.57035106155454 74.80247984488403) bank179198 +179199 POINT(39.887555995248235 73.32106655995347) bank179199 +179200 POINT(41.26079326270916 74.20612496737839) bank179200 +179201 POINT(41.25487171106399 74.8318141015988) bank179201 +179202 POINT(39.92268528808321 74.18020510445584) bank179202 +179203 POINT(40.3920882080544 74.9133097905371) bank179203 +179204 POINT(39.7997762122324 73.2819774953887) bank179204 +179205 POINT(41.05138571814083 74.36379424670498) bank179205 +179206 POINT(40.35016869529087 73.75194573990903) bank179206 +179207 POINT(40.401274839283914 74.58248923493582) bank179207 +179208 POINT(40.34490742430847 74.82673722176676) bank179208 +179209 POINT(41.561444767609935 74.63883916118372) bank179209 +179210 POINT(40.962707734432286 74.47092732117166) bank179210 +179211 POINT(40.93523592366723 74.2844978146039) bank179211 +179212 POINT(40.9042326129304 74.18168683783982) bank179212 +179213 POINT(40.99759281091878 73.63505202374684) bank179213 +179214 POINT(40.53881162999385 73.0488807431277) bank179214 +179215 POINT(40.07750864848679 74.89771041830423) bank179215 +179216 POINT(41.55299125644913 73.37329846016206) bank179216 +179217 POINT(41.49532054510968 74.69416835983372) bank179217 +179218 POINT(41.310842957450525 74.21479377141452) bank179218 +179219 POINT(41.14816040050688 73.94011425409431) bank179219 +179220 POINT(40.65069595068712 74.54547378804301) bank179220 +179221 POINT(41.20591763047787 74.3338944293759) bank179221 +179222 POINT(41.208212171311814 73.06461960217999) bank179222 +179223 POINT(40.842297796069545 74.99313913340272) bank179223 +179224 POINT(40.223314044248106 73.85819779815063) bank179224 +179225 POINT(41.26519455034977 74.5903762725731) bank179225 +179226 POINT(41.40541902854805 73.00941311198005) bank179226 +179227 POINT(40.904027221268116 74.26410919501194) bank179227 +179228 POINT(40.52774738686228 73.31583202321599) bank179228 +179229 POINT(39.73061933923979 73.19275778218167) bank179229 +179230 POINT(41.138244387086075 73.26008516061788) bank179230 +179231 POINT(41.030046585457185 73.13936059699707) bank179231 +179232 POINT(39.73074303517258 74.62489902557273) bank179232 +179233 POINT(40.112549203938556 74.87746504037818) bank179233 +179234 POINT(39.75020240761842 74.49897130820973) bank179234 +179235 POINT(40.81364292252034 73.16476289964935) bank179235 +179236 POINT(40.04253556177959 73.12181674806892) bank179236 +179237 POINT(41.09094059780633 73.94168863697398) bank179237 +179238 POINT(39.7491520077679 73.44002530044652) bank179238 +179239 POINT(40.93583949342102 73.05795680967073) bank179239 +179240 POINT(40.71229860980094 73.58833213450563) bank179240 +179241 POINT(40.94924262658625 74.94912882133852) bank179241 +179242 POINT(40.98138630045216 74.4654595649956) bank179242 +179243 POINT(41.46458201999944 74.95838936507508) bank179243 +179244 POINT(39.92820371848356 73.09138322657033) bank179244 +179245 POINT(40.765502763188714 74.32945303237211) bank179245 +179246 POINT(39.92466079730697 74.08174572438783) bank179246 +179247 POINT(41.558871524164736 73.66694405912868) bank179247 +179248 POINT(40.9317843223415 73.20323066840037) bank179248 +179249 POINT(41.56466638516491 74.65175546289724) bank179249 +179250 POINT(41.06045235526281 74.7463314650434) bank179250 +179251 POINT(40.91363298039482 74.53873627390286) bank179251 +179252 POINT(40.70587073154423 74.1155702582091) bank179252 +179253 POINT(40.09096895427283 73.36321055269931) bank179253 +179254 POINT(40.56560025685791 73.16143460690503) bank179254 +179255 POINT(39.941554898122796 73.91074337239604) bank179255 +179256 POINT(39.79662356085578 73.16336869130974) bank179256 +179257 POINT(41.510892791257945 74.8259992869593) bank179257 +179258 POINT(41.50708603558734 74.1677850868292) bank179258 +179259 POINT(41.47026739160865 73.62258431083211) bank179259 +179260 POINT(41.12737073593699 74.2004460955242) bank179260 +179261 POINT(40.45674597746189 74.84177639223061) bank179261 +179262 POINT(40.824493402283004 73.1255591099422) bank179262 +179263 POINT(41.29713376326701 74.15596143329329) bank179263 +179264 POINT(39.904646962215104 73.58076957776439) bank179264 +179265 POINT(40.668965459857596 73.89651390822617) bank179265 +179266 POINT(39.79998684373955 73.99811001551852) bank179266 +179267 POINT(40.86346471261861 73.3170404362495) bank179267 +179268 POINT(41.000475339004375 74.01059877290501) bank179268 +179269 POINT(39.96974041015875 73.22388796817232) bank179269 +179270 POINT(41.23174068173452 73.21914417505708) bank179270 +179271 POINT(40.353659397461186 74.17110356221083) bank179271 +179272 POINT(41.22895981069587 74.09805875757938) bank179272 +179273 POINT(41.17413127896038 73.34810091167692) bank179273 +179274 POINT(41.199024240325635 73.91033591610353) bank179274 +179275 POINT(40.742281332191226 74.50277325982019) bank179275 +179276 POINT(39.86699057751469 74.34317726171919) bank179276 +179277 POINT(39.763025835440445 74.40699204020888) bank179277 +179278 POINT(40.84321305709993 73.39062550794196) bank179278 +179279 POINT(41.56882201324353 74.07513818364667) bank179279 +179280 POINT(41.673970057531776 73.72079931607604) bank179280 +179281 POINT(41.08551179906083 73.50007053174032) bank179281 +179282 POINT(41.02374674442898 74.28674034146313) bank179282 +179283 POINT(39.71742207424959 73.77938745332634) bank179283 +179284 POINT(39.94002450073615 73.95571364820161) bank179284 +179285 POINT(41.6620766830003 74.59279118502792) bank179285 +179286 POINT(40.88687409121305 74.80481878139723) bank179286 +179287 POINT(41.499700531309 74.51795875269943) bank179287 +179288 POINT(41.274345077409244 73.80035766626007) bank179288 +179289 POINT(40.51841146877526 73.73096930152812) bank179289 +179290 POINT(39.993561837168386 74.70382540638927) bank179290 +179291 POINT(40.62319485001737 73.53536686862417) bank179291 +179292 POINT(40.95953356637332 74.29719883519148) bank179292 +179293 POINT(40.90352499669678 74.83434406860069) bank179293 +179294 POINT(40.78834918212289 73.56410714521776) bank179294 +179295 POINT(39.79882286650904 74.72737343863048) bank179295 +179296 POINT(41.10173631129906 73.5164047432297) bank179296 +179297 POINT(41.44501533664165 73.10867498358532) bank179297 +179298 POINT(40.16386470997908 73.94810230976937) bank179298 +179299 POINT(40.112945244571875 73.27377300536695) bank179299 +179300 POINT(39.92492056536132 73.4748577587899) bank179300 +179301 POINT(40.344194222374895 73.37697447331774) bank179301 +179302 POINT(40.21554138575671 73.04645431050967) bank179302 +179303 POINT(41.21773315242629 74.83381702504477) bank179303 +179304 POINT(39.99995548324478 74.02789739075558) bank179304 +179305 POINT(41.4023648338705 73.72321235702853) bank179305 +179306 POINT(40.11612413582918 74.6005653762448) bank179306 +179307 POINT(39.961826020338634 74.18805890404151) bank179307 +179308 POINT(41.456761476772 74.2321174383587) bank179308 +179309 POINT(40.458707906187 73.43819014602508) bank179309 +179310 POINT(40.28220754896438 73.25606187165677) bank179310 +179311 POINT(39.769820083146016 73.8839939550097) bank179311 +179312 POINT(40.327194027257185 73.85928488850882) bank179312 +179313 POINT(39.89181645078534 73.04021084589172) bank179313 +179314 POINT(41.14115116513614 73.73783340690086) bank179314 +179315 POINT(40.41685008904919 74.40918197867839) bank179315 +179316 POINT(41.131908289392314 73.97463113117631) bank179316 +179317 POINT(41.236730712459924 73.64979240474722) bank179317 +179318 POINT(40.702358632819376 74.01871541098897) bank179318 +179319 POINT(41.28428741636128 73.45525436055544) bank179319 +179320 POINT(40.84810939726203 73.41449876238441) bank179320 +179321 POINT(41.5623940401092 74.01410522150778) bank179321 +179322 POINT(40.751754331656684 74.3573654691701) bank179322 +179323 POINT(40.07967755994563 74.21489796431794) bank179323 +179324 POINT(41.44982640309802 73.53631846516978) bank179324 +179325 POINT(40.082869742237285 73.80952992948488) bank179325 +179326 POINT(41.09030910235918 73.48057320132253) bank179326 +179327 POINT(40.824468836223474 73.34943097902277) bank179327 +179328 POINT(40.36909142673195 74.76773031137645) bank179328 +179329 POINT(40.92080667356397 73.26326549658408) bank179329 +179330 POINT(40.79696307195226 74.04960041918363) bank179330 +179331 POINT(39.82147764084574 73.48145342767852) bank179331 +179332 POINT(40.51147458329647 74.07978827865821) bank179332 +179333 POINT(39.71485695979573 73.65883464915962) bank179333 +179334 POINT(39.949991424653994 74.37302599410096) bank179334 +179335 POINT(40.066369890158185 73.87475177767513) bank179335 +179336 POINT(40.08387729721207 73.87614474011411) bank179336 +179337 POINT(40.596099690583834 73.39064212457235) bank179337 +179338 POINT(40.1198257750719 73.83928020338293) bank179338 +179339 POINT(41.56079096522961 73.12636794477694) bank179339 +179340 POINT(41.13696074861554 73.24390954287765) bank179340 +179341 POINT(39.78453687071583 73.01479605123873) bank179341 +179342 POINT(40.93182624330335 74.99213067354034) bank179342 +179343 POINT(40.52640479600582 74.41968468211317) bank179343 +179344 POINT(39.71905848424312 73.62754824724148) bank179344 +179345 POINT(40.17125368632492 74.50018836990608) bank179345 +179346 POINT(41.566722634798076 73.30447449082628) bank179346 +179347 POINT(39.96019776665272 73.67706431362635) bank179347 +179348 POINT(40.05811565412651 73.59727756731475) bank179348 +179349 POINT(41.324139641468506 73.14506868593878) bank179349 +179350 POINT(40.81713474772796 73.33049111070238) bank179350 +179351 POINT(41.463420823621135 73.93211156457399) bank179351 +179352 POINT(41.13828763943842 73.83817799932538) bank179352 +179353 POINT(40.490931876640246 73.50058054206589) bank179353 +179354 POINT(40.42893850708058 74.57989075312328) bank179354 +179355 POINT(40.2621625816629 74.72151857092751) bank179355 +179356 POINT(40.80246570509092 74.02487889818968) bank179356 +179357 POINT(40.290762540436546 74.63308733164851) bank179357 +179358 POINT(40.13721665120268 74.35129053240499) bank179358 +179359 POINT(40.44622546988456 73.79639374791321) bank179359 +179360 POINT(40.2742654779874 74.0858580421019) bank179360 +179361 POINT(40.32150421886271 73.11816265836488) bank179361 +179362 POINT(40.81593802772901 74.64866148870217) bank179362 +179363 POINT(41.267389877049624 73.62469241436855) bank179363 +179364 POINT(39.8794118597412 73.80638611489898) bank179364 +179365 POINT(40.16395823423643 74.10223660024157) bank179365 +179366 POINT(40.44338219009958 74.39926893033761) bank179366 +179367 POINT(41.509528807310474 74.03528046458223) bank179367 +179368 POINT(41.29401227394458 73.45382325196836) bank179368 +179369 POINT(41.24067815863286 74.19507632522722) bank179369 +179370 POINT(40.08701458134178 73.63566306834397) bank179370 +179371 POINT(41.244859027224436 74.26148145541676) bank179371 +179372 POINT(40.71379654559333 74.46299516102016) bank179372 +179373 POINT(41.24478138006858 73.16478542143207) bank179373 +179374 POINT(41.65216246564735 73.81857838423288) bank179374 +179375 POINT(39.8976669457112 73.47790617395214) bank179375 +179376 POINT(40.447658872357806 74.95813348887894) bank179376 +179377 POINT(40.48700110500046 74.79813413271549) bank179377 +179378 POINT(41.68732613031246 74.2160446654884) bank179378 +179379 POINT(40.6408683607549 73.8087718562618) bank179379 +179380 POINT(41.4172980075501 74.60123507172878) bank179380 +179381 POINT(39.72496267299824 73.81217743785027) bank179381 +179382 POINT(39.92663185241173 74.32917107301648) bank179382 +179383 POINT(41.141788389958656 73.23112353758152) bank179383 +179384 POINT(41.16068119305943 73.23838262009703) bank179384 +179385 POINT(39.99405854881857 73.34448132654227) bank179385 +179386 POINT(40.95671903709283 73.84452143511042) bank179386 +179387 POINT(41.031978326463786 73.09514371280466) bank179387 +179388 POINT(40.322083592416526 73.0599188335945) bank179388 +179389 POINT(39.80535160488637 74.06652450785064) bank179389 +179390 POINT(41.208207977335384 73.82582928953096) bank179390 +179391 POINT(40.95616824628615 73.7078891044612) bank179391 +179392 POINT(41.56131203705576 73.07900125332644) bank179392 +179393 POINT(41.27907923135386 73.67414400541827) bank179393 +179394 POINT(39.733080359532394 74.9377275663644) bank179394 +179395 POINT(40.76599185847171 74.44804682875636) bank179395 +179396 POINT(41.35733372118561 74.32272426805254) bank179396 +179397 POINT(41.44876954415655 74.95696891807874) bank179397 +179398 POINT(40.98810126003099 74.28109380782386) bank179398 +179399 POINT(41.449223513691095 73.72900076526936) bank179399 +179400 POINT(40.78624654703105 74.44119973054502) bank179400 +179401 POINT(40.9685337019958 73.16958968076274) bank179401 +179402 POINT(41.33006764393343 73.31479814814604) bank179402 +179403 POINT(41.14426202947011 73.20849957962041) bank179403 +179404 POINT(41.27816387801886 74.63875948499049) bank179404 +179405 POINT(40.021643946973725 74.22972296308852) bank179405 +179406 POINT(40.10759096706937 73.03798787600267) bank179406 +179407 POINT(41.3720443647738 74.842526574747) bank179407 +179408 POINT(40.3889995827881 73.6341461425078) bank179408 +179409 POINT(40.22919508451682 73.3350875015042) bank179409 +179410 POINT(41.670458289768554 74.33251221913254) bank179410 +179411 POINT(40.229768136935824 73.42535259083066) bank179411 +179412 POINT(41.0211403364133 73.13209154098054) bank179412 +179413 POINT(40.347106181792164 74.11410115321675) bank179413 +179414 POINT(40.69752938817416 73.51692077475927) bank179414 +179415 POINT(40.61619723451667 74.79220238954872) bank179415 +179416 POINT(40.32752447914759 74.42042557744924) bank179416 +179417 POINT(40.15039884063603 74.45297665035295) bank179417 +179418 POINT(41.49919659656846 74.304948725909) bank179418 +179419 POINT(41.57549453020877 74.33744087266015) bank179419 +179420 POINT(41.33549217211101 73.56159924961925) bank179420 +179421 POINT(39.80024849528283 74.50334776020317) bank179421 +179422 POINT(40.143339161844914 74.94344545320658) bank179422 +179423 POINT(41.05032713815347 73.6057569043979) bank179423 +179424 POINT(41.27947695433712 73.05513252001933) bank179424 +179425 POINT(40.6946378231302 73.53304303194162) bank179425 +179426 POINT(41.17922674385166 73.14698326743816) bank179426 +179427 POINT(40.6447709293529 74.07854422164024) bank179427 +179428 POINT(40.886491212807165 74.01689312544681) bank179428 +179429 POINT(40.90982175708151 74.15926583683073) bank179429 +179430 POINT(39.93576727079833 73.94786370330672) bank179430 +179431 POINT(40.79234414087909 74.13185638415231) bank179431 +179432 POINT(40.225692086093275 74.86578420885128) bank179432 +179433 POINT(39.93676596832254 74.75342159591477) bank179433 +179434 POINT(41.50480005534376 74.48258017283045) bank179434 +179435 POINT(40.165226904491846 73.87623792383384) bank179435 +179436 POINT(39.88923101762159 73.2127807152786) bank179436 +179437 POINT(40.461688256420345 73.86321944629437) bank179437 +179438 POINT(39.86694562901531 73.76269981767159) bank179438 +179439 POINT(40.36823289557139 73.08667241744743) bank179439 +179440 POINT(40.651107077764586 74.83770521489762) bank179440 +179441 POINT(39.99606129667041 73.35377197380066) bank179441 +179442 POINT(41.00594118786685 73.28127398698578) bank179442 +179443 POINT(41.64507625347022 73.45040813871178) bank179443 +179444 POINT(41.16870305381008 74.2088415454821) bank179444 +179445 POINT(40.845595048949455 74.42262579726118) bank179445 +179446 POINT(40.40781177727739 73.60270748600189) bank179446 +179447 POINT(39.77849064083616 73.33967524812499) bank179447 +179448 POINT(41.54932678302798 73.40928153845904) bank179448 +179449 POINT(41.183911735202344 73.90042636065992) bank179449 +179450 POINT(41.42742864231487 74.78930851396277) bank179450 +179451 POINT(41.07865012680221 73.85976161632674) bank179451 +179452 POINT(41.37483135537758 74.18427886718632) bank179452 +179453 POINT(39.765857273940526 73.43480614049277) bank179453 +179454 POINT(39.90587435357387 73.98288915627363) bank179454 +179455 POINT(41.698163657676176 73.81051878655282) bank179455 +179456 POINT(41.12983661712302 74.45959144160956) bank179456 +179457 POINT(41.47017277936025 73.32624195103399) bank179457 +179458 POINT(40.26302290604178 74.00431832026811) bank179458 +179459 POINT(41.22620032789116 73.67197497008259) bank179459 +179460 POINT(39.78083678767379 74.29533334074517) bank179460 +179461 POINT(40.81754333337766 74.9898904229477) bank179461 +179462 POINT(41.1221691281247 73.05931786494556) bank179462 +179463 POINT(40.94369859236087 74.73642787422696) bank179463 +179464 POINT(39.79833521446302 74.55038447333283) bank179464 +179465 POINT(40.82160296411075 74.74314877357106) bank179465 +179466 POINT(40.85838099766258 74.36665202846945) bank179466 +179467 POINT(40.42947869941611 74.99594312632698) bank179467 +179468 POINT(40.79132767393301 74.99949527668348) bank179468 +179469 POINT(40.99199788603633 74.01286953699787) bank179469 +179470 POINT(41.48902575611168 73.11700830275738) bank179470 +179471 POINT(40.632067897999136 74.7073192253475) bank179471 +179472 POINT(40.33943691058871 73.19999106703777) bank179472 +179473 POINT(41.34509392187602 74.08612858713055) bank179473 +179474 POINT(40.914810220365496 74.93049442846261) bank179474 +179475 POINT(40.71210618081556 74.7636118269145) bank179475 +179476 POINT(41.23061686480506 73.98753352521994) bank179476 +179477 POINT(41.19877487001561 74.83828369462452) bank179477 +179478 POINT(41.27548204458662 73.15446899861853) bank179478 +179479 POINT(40.11782059889423 73.35169193006443) bank179479 +179480 POINT(41.07928525615643 73.01874201769101) bank179480 +179481 POINT(40.37481208385878 74.66240541213536) bank179481 +179482 POINT(40.13540362622115 74.62365125534866) bank179482 +179483 POINT(41.30009525200036 73.67549135627775) bank179483 +179484 POINT(40.452110908983045 74.19896230845096) bank179484 +179485 POINT(41.18058712700248 73.51859993165385) bank179485 +179486 POINT(40.9878096526645 74.03781543824824) bank179486 +179487 POINT(40.64022772672995 74.35273290941663) bank179487 +179488 POINT(41.51046188185914 73.62502773891264) bank179488 +179489 POINT(40.09197059085402 73.54575372172836) bank179489 +179490 POINT(39.784124494380364 73.06574726365542) bank179490 +179491 POINT(41.00016524513588 74.63709535143651) bank179491 +179492 POINT(40.82439147619388 73.03224807337031) bank179492 +179493 POINT(40.66413463920747 73.08868454585912) bank179493 +179494 POINT(40.09674035336907 74.55898710884779) bank179494 +179495 POINT(39.83651837434976 74.35125291617848) bank179495 +179496 POINT(40.947009177013015 74.84449497817552) bank179496 +179497 POINT(41.49199175824158 73.78222567358031) bank179497 +179498 POINT(41.350549308423986 74.039679502719) bank179498 +179499 POINT(41.34743636734179 73.88655979283081) bank179499 +179500 POINT(41.032333852299104 73.12472800470488) bank179500 +179501 POINT(40.08151221965636 74.05974450993341) bank179501 +179502 POINT(39.98926775533887 74.62247565676572) bank179502 +179503 POINT(40.4531988028117 74.7797191115773) bank179503 +179504 POINT(40.36216002300256 73.41341140900195) bank179504 +179505 POINT(40.363125927086 74.41393207721525) bank179505 +179506 POINT(40.99559262566643 73.7062260344369) bank179506 +179507 POINT(41.278562328095916 74.5891952932392) bank179507 +179508 POINT(40.03481477846389 73.26071598302644) bank179508 +179509 POINT(41.1253724179358 73.05361972726395) bank179509 +179510 POINT(40.83634843276133 74.34257186165549) bank179510 +179511 POINT(41.508419321526524 74.15889987969808) bank179511 +179512 POINT(39.95716341030589 74.44029778342816) bank179512 +179513 POINT(41.48555645257056 74.72875613123924) bank179513 +179514 POINT(41.40811414751348 74.81155989713083) bank179514 +179515 POINT(40.36528560825398 74.64330583250236) bank179515 +179516 POINT(41.6926756897948 73.20865593461762) bank179516 +179517 POINT(39.74142105540208 74.13237642294195) bank179517 +179518 POINT(40.12390764458921 74.46958266154184) bank179518 +179519 POINT(41.11588866721165 73.99515646038334) bank179519 +179520 POINT(40.18907290540414 74.1195488873805) bank179520 +179521 POINT(39.88803315510293 73.73709697129271) bank179521 +179522 POINT(39.98413746594053 74.11258987500592) bank179522 +179523 POINT(40.16207675894074 73.69166228427748) bank179523 +179524 POINT(41.35209805333764 74.99858211804663) bank179524 +179525 POINT(40.13690985855403 74.523985580457) bank179525 +179526 POINT(40.358736249560536 74.01988170966624) bank179526 +179527 POINT(40.90539534647454 74.43962322862676) bank179527 +179528 POINT(40.05019972045463 74.1452770643737) bank179528 +179529 POINT(41.13870181839668 73.89638020665616) bank179529 +179530 POINT(40.86049790991373 74.65741064914445) bank179530 +179531 POINT(41.37093695236453 74.01565248302404) bank179531 +179532 POINT(39.77244874705348 74.51491354079732) bank179532 +179533 POINT(40.93791868751188 74.43156140247366) bank179533 +179534 POINT(40.60492842916964 74.80899846074709) bank179534 +179535 POINT(41.03211175687737 74.80353683987028) bank179535 +179536 POINT(41.179240895964774 74.13307164337785) bank179536 +179537 POINT(39.880508043669906 74.44741169922362) bank179537 +179538 POINT(41.34091905966768 74.91515358921113) bank179538 +179539 POINT(39.78690030445987 74.12323338368714) bank179539 +179540 POINT(40.52565236912051 73.60444059805744) bank179540 +179541 POINT(39.9609452122722 73.7265998810346) bank179541 +179542 POINT(40.675192560878706 74.76056142061847) bank179542 +179543 POINT(40.16434974572811 74.72973637135674) bank179543 +179544 POINT(41.00738945831854 74.88627745067305) bank179544 +179545 POINT(41.332493433785025 74.44995949535634) bank179545 +179546 POINT(41.008997656083345 73.37715340824333) bank179546 +179547 POINT(40.93951080160145 73.67764194726092) bank179547 +179548 POINT(41.308535150142546 73.55807047338506) bank179548 +179549 POINT(40.11253724541376 74.71152548521466) bank179549 +179550 POINT(39.92788330751048 73.28762856347483) bank179550 +179551 POINT(40.451993614692476 73.30609948213818) bank179551 +179552 POINT(40.96834717933295 73.950955295249) bank179552 +179553 POINT(40.45138624807734 74.99570294239462) bank179553 +179554 POINT(40.17115477021648 74.1473075125301) bank179554 +179555 POINT(41.600605593929515 73.07495747755955) bank179555 +179556 POINT(41.42206395208872 74.49936973478583) bank179556 +179557 POINT(40.73666775259724 73.02768824891712) bank179557 +179558 POINT(39.98954952071451 74.82907970502765) bank179558 +179559 POINT(40.7004630294323 73.31183963255843) bank179559 +179560 POINT(41.33965311857659 73.05989879022881) bank179560 +179561 POINT(41.06725342505635 73.9728517792176) bank179561 +179562 POINT(39.71626183925373 74.65441512371892) bank179562 +179563 POINT(41.23899377369389 73.02037200590891) bank179563 +179564 POINT(41.23751787806422 73.41302538805823) bank179564 +179565 POINT(41.3268351348785 73.51781748975863) bank179565 +179566 POINT(39.97111282276825 74.19534681086851) bank179566 +179567 POINT(41.24676618337814 74.95682541753929) bank179567 +179568 POINT(41.24367326718928 73.66147566337058) bank179568 +179569 POINT(41.16873916000427 74.85600313754614) bank179569 +179570 POINT(40.92021380080502 73.72550801556778) bank179570 +179571 POINT(41.71048247990495 73.3114279241527) bank179571 +179572 POINT(39.796944659448144 73.32839532928394) bank179572 +179573 POINT(40.71420069035062 74.42705498487335) bank179573 +179574 POINT(40.58250556531446 73.48954561590398) bank179574 +179575 POINT(41.25131948860564 74.16935058705972) bank179575 +179576 POINT(41.38358808997989 74.69508369515995) bank179576 +179577 POINT(41.20245395532594 73.34580415227406) bank179577 +179578 POINT(40.51448963971705 74.15113851712262) bank179578 +179579 POINT(39.83032467615611 73.06633010635507) bank179579 +179580 POINT(40.487957115899405 74.1739419767087) bank179580 +179581 POINT(40.33182610503934 73.65705344799224) bank179581 +179582 POINT(39.79113047981423 73.38658536805326) bank179582 +179583 POINT(39.92525244931035 73.6512965623889) bank179583 +179584 POINT(41.172549456399246 73.18925437362118) bank179584 +179585 POINT(40.06414420396883 74.4541570614028) bank179585 +179586 POINT(40.0285169777562 73.38504117798291) bank179586 +179587 POINT(41.33578454104621 74.43968670801831) bank179587 +179588 POINT(41.40985018413846 74.42847332777775) bank179588 +179589 POINT(40.128882382208 74.84671713774756) bank179589 +179590 POINT(41.704819880404116 73.60015445409726) bank179590 +179591 POINT(40.81259918933895 73.97922841972914) bank179591 +179592 POINT(41.19679549795308 73.53047454238448) bank179592 +179593 POINT(41.536369854889564 73.78172663379341) bank179593 +179594 POINT(40.88357293765689 74.62140002312476) bank179594 +179595 POINT(40.426839594024585 73.70739631812025) bank179595 +179596 POINT(41.461739164276736 73.43572947176561) bank179596 +179597 POINT(40.56731034776911 73.24234291762475) bank179597 +179598 POINT(40.89899129323242 74.7469967885232) bank179598 +179599 POINT(40.20903758778335 73.0458134914778) bank179599 +179600 POINT(41.657865070795005 73.52277405297028) bank179600 +179601 POINT(40.31361172298934 73.66437843216708) bank179601 +179602 POINT(41.33661605694083 73.79784891931006) bank179602 +179603 POINT(40.58957468897299 75.00240545980672) bank179603 +179604 POINT(41.50153292069195 74.39594722827117) bank179604 +179605 POINT(41.21942792832538 74.07739844880687) bank179605 +179606 POINT(41.503583598523306 73.02414961839817) bank179606 +179607 POINT(40.17666208969457 74.64656257845263) bank179607 +179608 POINT(40.73155163398829 74.9214343519854) bank179608 +179609 POINT(40.17371055378712 73.88292121401251) bank179609 +179610 POINT(41.247846125041086 74.32797223676855) bank179610 +179611 POINT(40.228563894342436 73.16325174675033) bank179611 +179612 POINT(39.811846159116314 74.23456826930828) bank179612 +179613 POINT(39.938123706019134 73.05372659744191) bank179613 +179614 POINT(40.669406930601504 74.722459266531) bank179614 +179615 POINT(40.71227563976853 73.75142779250588) bank179615 +179616 POINT(40.23611217024456 74.75228288720396) bank179616 +179617 POINT(40.50897990666997 73.81107330389271) bank179617 +179618 POINT(41.41389056610862 74.6559876256461) bank179618 +179619 POINT(39.94069209543726 73.44593999434461) bank179619 +179620 POINT(41.15040437689867 73.33325433557552) bank179620 +179621 POINT(39.86039909485014 73.89175920445824) bank179621 +179622 POINT(41.23137455002294 74.33472464061089) bank179622 +179623 POINT(40.51779712562093 73.77656906492277) bank179623 +179624 POINT(40.1595050219224 73.6796061701541) bank179624 +179625 POINT(40.70508213787103 73.62274736550576) bank179625 +179626 POINT(40.89214868492484 73.25936785186057) bank179626 +179627 POINT(40.78791245880353 74.26123637158211) bank179627 +179628 POINT(40.90905140626174 74.27210761137351) bank179628 +179629 POINT(40.80018824717766 74.12204752120715) bank179629 +179630 POINT(41.08663207829269 74.10933858179753) bank179630 +179631 POINT(41.4870867857861 73.71520682457796) bank179631 +179632 POINT(40.691505825006764 74.05205815204943) bank179632 +179633 POINT(40.64646244679748 74.46361652195309) bank179633 +179634 POINT(39.84698655066954 73.88746795879227) bank179634 +179635 POINT(39.802303220133886 73.16669329446819) bank179635 +179636 POINT(41.07344548727161 73.50091011339228) bank179636 +179637 POINT(40.0556728973562 73.42136503311258) bank179637 +179638 POINT(40.622001495274944 73.49681199500235) bank179638 +179639 POINT(40.78238764415834 73.59555441637916) bank179639 +179640 POINT(40.00946508265067 74.37850045509381) bank179640 +179641 POINT(40.608025855373405 74.48780911062737) bank179641 +179642 POINT(41.34441329984626 74.27850401390859) bank179642 +179643 POINT(40.46053686546745 73.31410562841513) bank179643 +179644 POINT(41.16556010940031 74.15705668249137) bank179644 +179645 POINT(41.02945221953711 74.3397373906575) bank179645 +179646 POINT(41.14337172656967 73.87630774743604) bank179646 +179647 POINT(40.52695526969813 74.08393227635541) bank179647 +179648 POINT(40.81176374758649 73.40592502125405) bank179648 +179649 POINT(40.40472966769599 74.19463254029233) bank179649 +179650 POINT(40.6195661913929 74.31276754416567) bank179650 +179651 POINT(40.64421301474358 73.73064460105714) bank179651 +179652 POINT(41.557788548011075 75.00162934795121) bank179652 +179653 POINT(41.270077243463035 73.59329667905656) bank179653 +179654 POINT(41.70252015681056 73.06590992297954) bank179654 +179655 POINT(40.0209805786742 74.99342909117945) bank179655 +179656 POINT(41.48421767274747 74.26784652733708) bank179656 +179657 POINT(41.00315201864118 73.72824940294646) bank179657 +179658 POINT(40.21194606252364 73.80781381166504) bank179658 +179659 POINT(40.87991891826432 74.72386685134563) bank179659 +179660 POINT(40.611895232248 74.85600146739127) bank179660 +179661 POINT(40.91904453881327 73.37427908404943) bank179661 +179662 POINT(41.00937415523814 74.14230648646515) bank179662 +179663 POINT(41.537980521018945 73.42146189782129) bank179663 +179664 POINT(40.68357387815499 73.56410639114237) bank179664 +179665 POINT(40.51332746474808 74.22032006610239) bank179665 +179666 POINT(41.13773153815324 74.05942515576069) bank179666 +179667 POINT(39.72724866653949 74.46618188327284) bank179667 +179668 POINT(40.724408246537116 73.52307571175362) bank179668 +179669 POINT(40.14957769523879 74.32196366381113) bank179669 +179670 POINT(40.58279881989622 73.25813772060395) bank179670 +179671 POINT(39.8030276282155 74.93608892870907) bank179671 +179672 POINT(40.069890851635584 73.08513717763044) bank179672 +179673 POINT(41.45451166396998 74.08543261131275) bank179673 +179674 POINT(41.06980611499458 73.74796543037458) bank179674 +179675 POINT(40.06482403843013 74.35788818526886) bank179675 +179676 POINT(40.80945928489 74.66062272718177) bank179676 +179677 POINT(41.63718011717697 74.20305296541758) bank179677 +179678 POINT(40.48463894744175 74.46521281887752) bank179678 +179679 POINT(40.19969129536276 74.38639496165483) bank179679 +179680 POINT(41.170091972006475 74.35898831923772) bank179680 +179681 POINT(39.913567266247505 74.98614472083213) bank179681 +179682 POINT(41.03600140312364 73.29739377811075) bank179682 +179683 POINT(40.67727523653593 73.82560297956333) bank179683 +179684 POINT(40.57361512669911 74.73027936964115) bank179684 +179685 POINT(41.58037314903385 74.05405625768492) bank179685 +179686 POINT(39.97292962798995 73.28180156757062) bank179686 +179687 POINT(39.750542621862415 74.76200552733097) bank179687 +179688 POINT(41.707841090826896 74.34295726552538) bank179688 +179689 POINT(40.31647545335603 74.41629795250881) bank179689 +179690 POINT(39.721082176886206 74.86822341354257) bank179690 +179691 POINT(40.025595478237555 74.67677646305134) bank179691 +179692 POINT(41.00557824769484 73.35285252444598) bank179692 +179693 POINT(41.68220344833991 73.14158632872525) bank179693 +179694 POINT(39.830623136668244 74.0803884979953) bank179694 +179695 POINT(40.7040650371676 74.57195970304132) bank179695 +179696 POINT(40.57040971587878 74.22101314898005) bank179696 +179697 POINT(40.59095888814877 74.70194525686321) bank179697 +179698 POINT(40.415728932646296 74.58906251402558) bank179698 +179699 POINT(40.48835860823682 74.51258579202067) bank179699 +179700 POINT(41.437146191059895 74.25613914458077) bank179700 +179701 POINT(39.99221048301115 73.55380531081558) bank179701 +179702 POINT(39.921623828780525 74.65278072466495) bank179702 +179703 POINT(40.22584962560271 73.09624538629252) bank179703 +179704 POINT(41.60526044000878 74.00061973932534) bank179704 +179705 POINT(41.2698439315766 74.88799501958897) bank179705 +179706 POINT(39.83323981583061 74.54182773905104) bank179706 +179707 POINT(40.04037286356573 73.86948288732161) bank179707 +179708 POINT(41.10637548322535 73.39856203557325) bank179708 +179709 POINT(41.201873432731134 74.2300639316432) bank179709 +179710 POINT(40.54106181683277 73.41491354751506) bank179710 +179711 POINT(41.06968860411356 74.36287813002495) bank179711 +179712 POINT(41.304623431674685 74.39897212395758) bank179712 +179713 POINT(39.91340475043475 73.31486726269581) bank179713 +179714 POINT(41.41734181099468 74.05797401990314) bank179714 +179715 POINT(40.43532764937673 73.01779252783342) bank179715 +179716 POINT(40.748734080327935 73.55879301887973) bank179716 +179717 POINT(40.31777145902085 74.40881134726509) bank179717 +179718 POINT(41.21903519105862 74.99829129447339) bank179718 +179719 POINT(39.75014409363743 74.22800998764272) bank179719 +179720 POINT(40.017324781339475 74.27974035596054) bank179720 +179721 POINT(39.929281485733426 73.1235194751236) bank179721 +179722 POINT(41.28444599209939 73.41142622966969) bank179722 +179723 POINT(41.23351419344348 74.42635902847988) bank179723 +179724 POINT(40.831600031387445 74.45204700466729) bank179724 +179725 POINT(41.02245263482203 73.89995786494181) bank179725 +179726 POINT(41.350152099520884 74.94984650518856) bank179726 +179727 POINT(41.49310401127191 73.60416757680579) bank179727 +179728 POINT(41.33214994403967 74.0386347874788) bank179728 +179729 POINT(40.953635740295425 74.48523096805556) bank179729 +179730 POINT(40.083580249375494 73.01063992179652) bank179730 +179731 POINT(40.672839871041475 73.90284002693146) bank179731 +179732 POINT(41.03249187838201 73.89151937511147) bank179732 +179733 POINT(40.12985201150482 73.9596935481484) bank179733 +179734 POINT(39.783063811716566 74.71327194199488) bank179734 +179735 POINT(40.90992954857842 74.45509199180054) bank179735 +179736 POINT(40.92270941880345 73.2565955997134) bank179736 +179737 POINT(41.449668828756856 73.90094630624601) bank179737 +179738 POINT(39.75826536218405 73.67410405669109) bank179738 +179739 POINT(41.09219326270395 73.85891045035952) bank179739 +179740 POINT(39.983581845646626 74.68101360903357) bank179740 +179741 POINT(40.29830006746209 73.87781651130017) bank179741 +179742 POINT(40.53943265287422 74.81845974134096) bank179742 +179743 POINT(41.60155339116248 74.47737167188833) bank179743 +179744 POINT(40.41329350107816 74.03325131940495) bank179744 +179745 POINT(41.27512706233325 74.03640477420123) bank179745 +179746 POINT(41.103122640956926 74.4768685878681) bank179746 +179747 POINT(41.037249959624596 73.112982833455) bank179747 +179748 POINT(41.22126738886676 74.83878827944639) bank179748 +179749 POINT(40.34370843707484 73.21929027007339) bank179749 +179750 POINT(40.614032372352625 74.36077891311096) bank179750 +179751 POINT(41.17144928088055 73.65103967224564) bank179751 +179752 POINT(40.81469211686217 74.64476564915994) bank179752 +179753 POINT(41.6936151805828 74.63499800199669) bank179753 +179754 POINT(41.3714625545024 74.91167178863003) bank179754 +179755 POINT(41.315894859397375 74.23219435702141) bank179755 +179756 POINT(40.844028169159586 74.05816943234281) bank179756 +179757 POINT(41.004267537733256 73.47351915837298) bank179757 +179758 POINT(41.41518944668075 74.7249022497316) bank179758 +179759 POINT(39.736998121645215 74.35157109220329) bank179759 +179760 POINT(40.494174307732685 74.70721280254195) bank179760 +179761 POINT(41.269177409993254 73.84317716851747) bank179761 +179762 POINT(40.148658562196346 74.50759492080678) bank179762 +179763 POINT(40.434041934868155 73.96735627824668) bank179763 +179764 POINT(41.36421386438816 74.89554185454678) bank179764 +179765 POINT(41.238306245397496 74.30655911425067) bank179765 +179766 POINT(40.25258898798829 73.13531812136056) bank179766 +179767 POINT(39.73331065634941 74.73419298737325) bank179767 +179768 POINT(40.136169732730664 73.24611578549103) bank179768 +179769 POINT(40.20859040428102 73.39738304895063) bank179769 +179770 POINT(40.00730691445935 73.98114951656963) bank179770 +179771 POINT(40.01684670506596 74.10390489360763) bank179771 +179772 POINT(40.568326808985844 73.53063711253809) bank179772 +179773 POINT(39.995353293295885 74.61306833072102) bank179773 +179774 POINT(41.44134986480517 73.46051588643245) bank179774 +179775 POINT(41.022858886992815 73.36845394502419) bank179775 +179776 POINT(41.19235250341801 74.07213346113268) bank179776 +179777 POINT(41.46076619129082 74.03014433541367) bank179777 +179778 POINT(39.962768894572065 74.45651132708991) bank179778 +179779 POINT(39.99143047772046 74.53739217123851) bank179779 +179780 POINT(40.87324684323454 74.00908629183262) bank179780 +179781 POINT(41.00140392785897 73.55900358500901) bank179781 +179782 POINT(40.96074844671539 74.25440628185201) bank179782 +179783 POINT(39.82191179108678 74.48310494671523) bank179783 +179784 POINT(40.981646148179415 73.01958981156534) bank179784 +179785 POINT(40.82965465316172 73.80591783196193) bank179785 +179786 POINT(41.244488726953 73.98173339295792) bank179786 +179787 POINT(39.72554655008563 74.74988238809244) bank179787 +179788 POINT(40.39051286797189 73.13902930354368) bank179788 +179789 POINT(41.368545208777576 74.79131395620652) bank179789 +179790 POINT(40.46615179512445 73.58389865536472) bank179790 +179791 POINT(41.10320891207608 74.82849651777113) bank179791 +179792 POINT(39.98420755055639 73.01232762314442) bank179792 +179793 POINT(40.639287856727975 74.2322310084555) bank179793 +179794 POINT(41.43176078448824 73.26294069297705) bank179794 +179795 POINT(40.890802176690336 74.43594019394283) bank179795 +179796 POINT(40.29575803852178 74.35210145568274) bank179796 +179797 POINT(40.981205151287696 74.78666151943605) bank179797 +179798 POINT(40.58824673726607 73.83680061175882) bank179798 +179799 POINT(40.21441408718379 73.54661206998794) bank179799 +179800 POINT(40.505948930547774 74.2399044537904) bank179800 +179801 POINT(40.36453461392876 74.90184429775368) bank179801 +179802 POINT(40.894091200177364 73.89676113148653) bank179802 +179803 POINT(40.08352644060115 74.36425326785235) bank179803 +179804 POINT(40.24824551171509 73.50093055877953) bank179804 +179805 POINT(41.36879339603034 74.6112089045399) bank179805 +179806 POINT(41.22939423540525 73.6721761052607) bank179806 +179807 POINT(40.915543614838995 73.30107876675015) bank179807 +179808 POINT(39.990229314833705 74.65553764397005) bank179808 +179809 POINT(40.505407659569684 74.17778302671081) bank179809 +179810 POINT(41.30888875228813 74.87682227741838) bank179810 +179811 POINT(39.74875144942572 73.4197594890008) bank179811 +179812 POINT(39.715621013119666 74.72589023863696) bank179812 +179813 POINT(41.40146327479054 74.40034806584573) bank179813 +179814 POINT(40.83891132012889 74.25378814701423) bank179814 +179815 POINT(40.114270516123966 73.82323628509478) bank179815 +179816 POINT(39.910280590271164 73.29321508366135) bank179816 +179817 POINT(40.37048942945556 74.0552638073054) bank179817 +179818 POINT(40.973087349993385 73.33019209140693) bank179818 +179819 POINT(40.360604699922064 74.11988727963269) bank179819 +179820 POINT(40.01186471435503 73.1916749029861) bank179820 +179821 POINT(40.2262146095025 73.81100902251843) bank179821 +179822 POINT(40.89490358678505 74.33617346963987) bank179822 +179823 POINT(40.64600845252412 73.16673474770033) bank179823 +179824 POINT(40.296572200755236 74.44619288992118) bank179824 +179825 POINT(40.65556657528849 74.47012961139322) bank179825 +179826 POINT(40.42420147964069 73.77992840009841) bank179826 +179827 POINT(41.39862487590177 73.19407003343778) bank179827 +179828 POINT(40.5030321672606 73.95519464357544) bank179828 +179829 POINT(41.178277746983994 73.69931620648084) bank179829 +179830 POINT(41.12445575479908 73.3336960653985) bank179830 +179831 POINT(40.8031141761841 73.53112737246981) bank179831 +179832 POINT(40.84807278768312 74.87301571056244) bank179832 +179833 POINT(40.21940019702407 73.10442411945449) bank179833 +179834 POINT(40.43545403745968 73.02537882598774) bank179834 +179835 POINT(41.37677729513284 73.21834742909334) bank179835 +179836 POINT(40.46464545048103 74.17547458531823) bank179836 +179837 POINT(41.24128342545151 73.24678919245426) bank179837 +179838 POINT(40.79827149904135 74.89975655499562) bank179838 +179839 POINT(41.36772734579597 73.24132984678997) bank179839 +179840 POINT(40.509480770521265 73.81706711843906) bank179840 +179841 POINT(39.7904107583893 74.94233912817805) bank179841 +179842 POINT(41.01856899615346 74.24678685629075) bank179842 +179843 POINT(41.532698836265666 74.64010749521523) bank179843 +179844 POINT(39.84255399505329 74.77342827259288) bank179844 +179845 POINT(41.001822635324295 74.47842537493473) bank179845 +179846 POINT(39.78016217321892 73.8489614503218) bank179846 +179847 POINT(41.291234742514476 73.24724641750355) bank179847 +179848 POINT(40.73014991033381 73.86014098763435) bank179848 +179849 POINT(40.949426921345506 74.11898585144868) bank179849 +179850 POINT(40.868391921949446 74.07824017479231) bank179850 +179851 POINT(41.57871352944578 73.571993147424) bank179851 +179852 POINT(41.391584652983965 73.55138098773091) bank179852 +179853 POINT(41.25070229511383 74.73106538442883) bank179853 +179854 POINT(39.91800546165713 74.99732634445526) bank179854 +179855 POINT(41.04455254541176 73.93590320428837) bank179855 +179856 POINT(41.003819338829246 74.47152725003666) bank179856 +179857 POINT(40.437804223311964 74.32526737562074) bank179857 +179858 POINT(39.87822722108686 73.26925344705347) bank179858 +179859 POINT(41.10581378643503 74.06863287158707) bank179859 +179860 POINT(41.36195197221677 74.32698920505699) bank179860 +179861 POINT(39.90125182126975 73.58032339016609) bank179861 +179862 POINT(40.6323632356923 74.66064241658717) bank179862 +179863 POINT(41.45173754219284 73.07141960630265) bank179863 +179864 POINT(41.04854377478721 73.44514842659521) bank179864 +179865 POINT(40.29141554545419 73.25207530740597) bank179865 +179866 POINT(41.29346009692619 73.0993718221662) bank179866 +179867 POINT(39.78954538740236 73.73055553594166) bank179867 +179868 POINT(40.25345020943175 74.86978175857973) bank179868 +179869 POINT(40.69223093804534 74.23103444047874) bank179869 +179870 POINT(40.5470050983725 74.8112115480187) bank179870 +179871 POINT(40.314822587166056 74.38860878052132) bank179871 +179872 POINT(40.92669580058399 73.84386975592683) bank179872 +179873 POINT(40.97803755773995 74.59267742363042) bank179873 +179874 POINT(40.67207056338622 73.55073101357411) bank179874 +179875 POINT(41.57525971074295 73.40381258549418) bank179875 +179876 POINT(40.72681597675479 74.09161661141104) bank179876 +179877 POINT(39.72821542500403 74.65019021208636) bank179877 +179878 POINT(39.97614436276972 74.46954076787546) bank179878 +179879 POINT(41.51012033318961 74.42554084045473) bank179879 +179880 POINT(40.75606411870858 74.45037146625496) bank179880 +179881 POINT(39.83944464205148 74.9356480658938) bank179881 +179882 POINT(40.18525541869498 74.51850967213525) bank179882 +179883 POINT(41.626127952738756 74.29300264903193) bank179883 +179884 POINT(40.960941678055704 73.16941126567318) bank179884 +179885 POINT(40.270534572802475 73.24998256909588) bank179885 +179886 POINT(40.26229814581338 73.14602153594758) bank179886 +179887 POINT(40.131850696442925 73.7267182431396) bank179887 +179888 POINT(41.060886352669556 73.28932037233925) bank179888 +179889 POINT(40.12923093377308 73.76687082593786) bank179889 +179890 POINT(39.97984653693294 73.03849910114236) bank179890 +179891 POINT(40.49880125063139 73.28202960536328) bank179891 +179892 POINT(41.11377305573495 74.21873216871019) bank179892 +179893 POINT(40.01075003768773 74.8554301443833) bank179893 +179894 POINT(41.08615720474109 73.53509552166899) bank179894 +179895 POINT(41.32305192658248 73.0505065038064) bank179895 +179896 POINT(41.5343553407255 74.89132842522127) bank179896 +179897 POINT(40.42803815009247 73.06740755786853) bank179897 +179898 POINT(40.17039857238955 74.17928290272343) bank179898 +179899 POINT(40.9172296882006 74.15862071838266) bank179899 +179900 POINT(40.34286555822701 73.13755827540407) bank179900 +179901 POINT(41.15077854000021 73.78663793300865) bank179901 +179902 POINT(39.99698906064133 73.30521754721974) bank179902 +179903 POINT(40.34463453088325 73.45830741076779) bank179903 +179904 POINT(40.34882942718436 74.21372122295472) bank179904 +179905 POINT(40.760807340890324 73.32355955566393) bank179905 +179906 POINT(41.29570694768148 74.34047085244178) bank179906 +179907 POINT(40.699068231516925 74.31027498351222) bank179907 +179908 POINT(41.13061053872094 74.51472560770848) bank179908 +179909 POINT(40.45750299742285 73.09341158750915) bank179909 +179910 POINT(40.80092656870212 73.5532061280522) bank179910 +179911 POINT(40.482559659746094 74.22100309847177) bank179911 +179912 POINT(39.876978907045455 74.54951934702615) bank179912 +179913 POINT(40.12931674318055 73.12009578547759) bank179913 +179914 POINT(40.46442257134512 73.48085687187964) bank179914 +179915 POINT(40.644956590688224 73.41367366070713) bank179915 +179916 POINT(40.31970464773127 74.34360422718814) bank179916 +179917 POINT(39.99115579373684 73.81854408451524) bank179917 +179918 POINT(41.607542221602664 73.71591053221243) bank179918 +179919 POINT(41.06228006460779 73.97681535308551) bank179919 +179920 POINT(40.05183140016999 73.48497118115384) bank179920 +179921 POINT(40.29791711805457 73.5800357737928) bank179921 +179922 POINT(40.063865414928536 74.80949674640804) bank179922 +179923 POINT(39.71548415092749 74.1671253904984) bank179923 +179924 POINT(41.512801498156435 73.93428444612518) bank179924 +179925 POINT(40.69638028883963 73.66728504609853) bank179925 +179926 POINT(41.43654479056594 73.3121383783465) bank179926 +179927 POINT(40.06381562355343 73.11712806839103) bank179927 +179928 POINT(41.11059983826172 74.11259242019277) bank179928 +179929 POINT(41.66308602302116 74.55059066858202) bank179929 +179930 POINT(39.994073475932204 73.27185589224982) bank179930 +179931 POINT(40.298123118669814 74.82983993486789) bank179931 +179932 POINT(40.19733175468275 74.96006451180979) bank179932 +179933 POINT(41.49201290315967 73.62411784811115) bank179933 +179934 POINT(40.273065515253556 74.1710440603496) bank179934 +179935 POINT(41.16716510576263 73.73082127236786) bank179935 +179936 POINT(40.54981529989583 74.65568036236331) bank179936 +179937 POINT(40.86505131216148 74.20982289653757) bank179937 +179938 POINT(40.999815826832915 74.70942799857016) bank179938 +179939 POINT(41.593734184355434 73.41508064002743) bank179939 +179940 POINT(39.780881787936224 73.08794137800493) bank179940 +179941 POINT(40.23564390616744 73.88578970831011) bank179941 +179942 POINT(40.251218548272625 73.45262831370479) bank179942 +179943 POINT(40.925345406846176 73.91363241136756) bank179943 +179944 POINT(41.48156270321328 73.99906517190125) bank179944 +179945 POINT(40.8287646572435 73.64438765616913) bank179945 +179946 POINT(41.287070065211786 73.29189473479687) bank179946 +179947 POINT(41.30535214051855 73.69839824316753) bank179947 +179948 POINT(40.775845864901875 73.88405489300115) bank179948 +179949 POINT(40.138941973605235 73.70287718182303) bank179949 +179950 POINT(41.65592059402994 73.15648690781553) bank179950 +179951 POINT(39.79074961682444 73.98268009405568) bank179951 +179952 POINT(41.37773995899824 73.82040586942054) bank179952 +179953 POINT(41.47925869241821 74.47119340378858) bank179953 +179954 POINT(40.87025435183997 73.09505802282006) bank179954 +179955 POINT(40.661845127427064 74.0222024339968) bank179955 +179956 POINT(40.65604567484118 74.30844960315557) bank179956 +179957 POINT(40.62667806603293 74.02267178626768) bank179957 +179958 POINT(39.74508722411759 74.43976669495319) bank179958 +179959 POINT(41.66652889646297 73.50259371198298) bank179959 +179960 POINT(40.735364975507224 74.69850136145571) bank179960 +179961 POINT(41.41517124897437 74.52766739105155) bank179961 +179962 POINT(40.77429395823177 74.24793499216621) bank179962 +179963 POINT(40.753823865817594 74.09309971820934) bank179963 +179964 POINT(40.05194010276168 73.55882961833123) bank179964 +179965 POINT(41.70457982753444 73.55054995822114) bank179965 +179966 POINT(41.213028367196834 73.25293425025612) bank179966 +179967 POINT(40.32966755083436 73.4348578870125) bank179967 +179968 POINT(39.958510694296436 74.72772735996901) bank179968 +179969 POINT(39.92539292374667 74.70815002246724) bank179969 +179970 POINT(40.73029433215633 73.6292795858711) bank179970 +179971 POINT(40.56201775577765 73.98495179400621) bank179971 +179972 POINT(40.777743133064526 73.20215659388548) bank179972 +179973 POINT(40.01369578319691 73.64110235714708) bank179973 +179974 POINT(41.00484339389751 73.98783494027467) bank179974 +179975 POINT(41.35861910900248 74.37889721688339) bank179975 +179976 POINT(40.14357487978696 74.83825814207889) bank179976 +179977 POINT(40.088084862882496 74.9295281257014) bank179977 +179978 POINT(41.40414660325812 74.1212098711798) bank179978 +179979 POINT(40.6010031226105 73.12800146225099) bank179979 +179980 POINT(41.39837192656192 74.65908349534517) bank179980 +179981 POINT(41.5693148674086 74.23712994103924) bank179981 +179982 POINT(40.02407451363011 74.43502891082494) bank179982 +179983 POINT(41.61047167555585 74.10335196990961) bank179983 +179984 POINT(40.999618130214316 73.31509828245802) bank179984 +179985 POINT(40.75555310259189 74.32787560055353) bank179985 +179986 POINT(40.45476431905818 73.06785850244552) bank179986 +179987 POINT(41.005281258547406 73.09115865213927) bank179987 +179988 POINT(40.71766602651682 74.5432340743713) bank179988 +179989 POINT(41.29083335366963 74.80677186517659) bank179989 +179990 POINT(40.33121007517676 73.03535515740468) bank179990 +179991 POINT(41.43991704869911 73.64881974040536) bank179991 +179992 POINT(40.230869604261535 74.80130130721622) bank179992 +179993 POINT(40.86416945329964 73.68242093149483) bank179993 +179994 POINT(41.57088993526414 74.34548430212256) bank179994 +179995 POINT(40.994052634409044 73.08252680048908) bank179995 +179996 POINT(41.146363188652174 73.57444062512764) bank179996 +179997 POINT(40.91525885796154 73.99123250630062) bank179997 +179998 POINT(39.81650390476425 74.05418115317231) bank179998 +179999 POINT(40.04608165757061 73.48637153719842) bank179999 +180000 POINT(41.11288838862921 73.39969191306955) bank180000 +180001 POINT(40.39267945045277 74.59200634356219) bank180001 +180002 POINT(41.39506305416941 74.94754539022641) bank180002 +180003 POINT(41.39485406938373 74.72588808895603) bank180003 +180004 POINT(41.249557498231674 73.86811893256004) bank180004 +180005 POINT(39.88452592569714 74.5379806772412) bank180005 +180006 POINT(40.758199073926086 73.38189499449682) bank180006 +180007 POINT(41.328565483241555 73.99618910003177) bank180007 +180008 POINT(41.155576687917645 73.23269612961504) bank180008 +180009 POINT(40.93965716009918 74.4700581362686) bank180009 +180010 POINT(39.989850424815614 73.95872996878151) bank180010 +180011 POINT(40.6708695202528 73.893004565857) bank180011 +180012 POINT(40.67217177382159 74.31016036027275) bank180012 +180013 POINT(41.0224487917797 73.57023428092009) bank180013 +180014 POINT(39.73390569471802 73.9062971919731) bank180014 +180015 POINT(40.85868280059489 73.26727989539124) bank180015 +180016 POINT(40.78129214731379 73.009666359228) bank180016 +180017 POINT(40.25980928792654 74.90542928488645) bank180017 +180018 POINT(40.50400365637472 73.21115548045317) bank180018 +180019 POINT(39.958779136740525 74.68347671458811) bank180019 +180020 POINT(40.269623806177606 74.46140105948221) bank180020 +180021 POINT(41.206281808785356 74.25372214047117) bank180021 +180022 POINT(40.18416830130338 73.0709055400982) bank180022 +180023 POINT(40.81552018110032 74.33710417914732) bank180023 +180024 POINT(41.080064778244235 74.06702849944499) bank180024 +180025 POINT(40.82965499964194 73.71969336062665) bank180025 +180026 POINT(40.62155003008366 73.7268060868706) bank180026 +180027 POINT(40.11740192405871 73.6356890977832) bank180027 +180028 POINT(39.75344048662715 74.658969317322) bank180028 +180029 POINT(40.46645985959239 73.42884734782596) bank180029 +180030 POINT(41.25333729107671 74.82292162969549) bank180030 +180031 POINT(40.33586565628152 74.27648825461961) bank180031 +180032 POINT(40.7401089783844 73.99509955286346) bank180032 +180033 POINT(40.74552973405517 73.9376362244858) bank180033 +180034 POINT(39.987503617737744 74.20063386184671) bank180034 +180035 POINT(40.52509730661906 73.07749836666287) bank180035 +180036 POINT(40.24798889096154 73.05628125203827) bank180036 +180037 POINT(41.05819092824788 73.8072307404627) bank180037 +180038 POINT(41.25700510256713 73.88617978130696) bank180038 +180039 POINT(40.494157606634495 73.06990215757193) bank180039 +180040 POINT(41.147553148258936 73.37847429521702) bank180040 +180041 POINT(41.64206349879906 74.04691443441249) bank180041 +180042 POINT(41.2133593926839 74.44542320647476) bank180042 +180043 POINT(40.53200280141604 73.50679924777943) bank180043 +180044 POINT(39.85611472227501 73.33310798304089) bank180044 +180045 POINT(40.6563615353553 73.75316868476081) bank180045 +180046 POINT(41.59757759989035 73.77043105017013) bank180046 +180047 POINT(41.143982857977946 73.91808721652174) bank180047 +180048 POINT(40.84814922519453 73.60672908673783) bank180048 +180049 POINT(40.87627167442515 74.30285426558561) bank180049 +180050 POINT(41.634680492101886 74.87410516908051) bank180050 +180051 POINT(40.473234714472234 73.82210788505891) bank180051 +180052 POINT(39.738807706602074 74.32989260927233) bank180052 +180053 POINT(41.40790142394263 74.47395088133942) bank180053 +180054 POINT(41.00078240870998 73.12947816503477) bank180054 +180055 POINT(41.57666714820538 74.80615212698984) bank180055 +180056 POINT(41.23784972187183 74.78166914479074) bank180056 +180057 POINT(41.51466166812995 74.57675682214649) bank180057 +180058 POINT(40.86773761415399 73.23967770470881) bank180058 +180059 POINT(40.34832315167096 73.56545166859706) bank180059 +180060 POINT(41.053304002429854 73.09949589570735) bank180060 +180061 POINT(39.9742163161247 74.10210236653403) bank180061 +180062 POINT(40.8853828967778 73.68931926677493) bank180062 +180063 POINT(41.12408138990726 74.13831982172745) bank180063 +180064 POINT(40.30813576587074 74.91104897997678) bank180064 +180065 POINT(40.746596632317235 74.05181455048925) bank180065 +180066 POINT(40.44460119009636 73.48456646942383) bank180066 +180067 POINT(40.728059515661556 73.8425110123192) bank180067 +180068 POINT(40.077257024041465 73.59520924681608) bank180068 +180069 POINT(39.94387758170377 74.49869869515963) bank180069 +180070 POINT(40.498893091724234 73.23681532060141) bank180070 +180071 POINT(40.58114998165112 73.75148023574859) bank180071 +180072 POINT(40.86125932403176 73.19080650008732) bank180072 +180073 POINT(40.78320271771622 73.95074216595685) bank180073 +180074 POINT(40.96369905140334 74.23888561974115) bank180074 +180075 POINT(39.864928766326145 74.49545436160234) bank180075 +180076 POINT(41.621554944407336 73.53279519895547) bank180076 +180077 POINT(40.894902357656996 74.72883598666878) bank180077 +180078 POINT(41.44305341338502 74.75187768702371) bank180078 +180079 POINT(39.94249179158733 74.1422313617725) bank180079 +180080 POINT(40.158945314484946 74.05738170525751) bank180080 +180081 POINT(40.89309046010623 73.49264725473368) bank180081 +180082 POINT(40.53882031046576 73.00663888934854) bank180082 +180083 POINT(40.44328347524493 74.44911375192378) bank180083 +180084 POINT(41.02408083313465 74.1037935381798) bank180084 +180085 POINT(41.037428179290856 73.91147889674464) bank180085 +180086 POINT(41.31277326389615 74.99361600681459) bank180086 +180087 POINT(40.67794536423132 73.00659190090352) bank180087 +180088 POINT(40.615883947124175 74.77015731207268) bank180088 +180089 POINT(39.80060378926743 74.40856472603394) bank180089 +180090 POINT(40.02086999037262 74.20273696843145) bank180090 +180091 POINT(41.6030359518269 73.13635804430558) bank180091 +180092 POINT(40.22078982341149 74.14904901294334) bank180092 +180093 POINT(41.325879932441886 73.6382793048527) bank180093 +180094 POINT(40.28247577356768 73.22186388638846) bank180094 +180095 POINT(39.7960128832813 74.43965502633384) bank180095 +180096 POINT(39.77270164071806 73.92309509525221) bank180096 +180097 POINT(39.757465242393344 73.25980270544933) bank180097 +180098 POINT(40.77717798118106 73.80443750332658) bank180098 +180099 POINT(40.531098129121794 73.30980835902903) bank180099 +180100 POINT(40.57370059788441 74.48550840922644) bank180100 +180101 POINT(40.93012641390493 73.43389843147209) bank180101 +180102 POINT(41.10778117236293 74.79188987792647) bank180102 +180103 POINT(41.069873301851594 73.04155340705685) bank180103 +180104 POINT(41.66380686056422 73.54307297274215) bank180104 +180105 POINT(41.599811322368986 73.88956687441991) bank180105 +180106 POINT(41.26813345594075 74.4502420736514) bank180106 +180107 POINT(41.67953955581455 74.78938414919749) bank180107 +180108 POINT(41.30673033669369 74.19345581353782) bank180108 +180109 POINT(40.472721157460334 73.23500945286501) bank180109 +180110 POINT(39.88982567576424 73.55744154843602) bank180110 +180111 POINT(41.69999824037862 73.59868405247734) bank180111 +180112 POINT(40.27890941704824 73.71928652697285) bank180112 +180113 POINT(40.55468915508173 73.63281803128767) bank180113 +180114 POINT(41.25441259953613 73.91210213022742) bank180114 +180115 POINT(41.602924930471076 73.9205842357519) bank180115 +180116 POINT(40.09576580969425 73.62002287547782) bank180116 +180117 POINT(40.593260863907304 74.55961236879915) bank180117 +180118 POINT(40.19016319358756 74.6318582411661) bank180118 +180119 POINT(41.204635108624714 73.70234207907372) bank180119 +180120 POINT(39.97324781566001 74.72191123331876) bank180120 +180121 POINT(39.750305050507166 74.92091821797958) bank180121 +180122 POINT(40.22952981482377 73.35078830696304) bank180122 +180123 POINT(41.015772547843795 74.1472082398479) bank180123 +180124 POINT(40.59468496585165 74.44150617760785) bank180124 +180125 POINT(40.41934205427097 73.76938711440415) bank180125 +180126 POINT(41.65960991356367 73.9788670514549) bank180126 +180127 POINT(40.05891116240827 74.52899313634373) bank180127 +180128 POINT(39.925261252664086 73.7661154070967) bank180128 +180129 POINT(41.523315555647756 73.20882459160852) bank180129 +180130 POINT(40.748056515626416 74.46296419903354) bank180130 +180131 POINT(39.86259488135514 74.29179467107689) bank180131 +180132 POINT(40.012545568925226 74.16234895057862) bank180132 +180133 POINT(41.02887478343748 74.88536059626203) bank180133 +180134 POINT(40.53966417767903 74.33272863297742) bank180134 +180135 POINT(40.67916305908841 74.61707590803353) bank180135 +180136 POINT(39.84672109509901 73.456689836336) bank180136 +180137 POINT(41.43014534456827 73.88862809826166) bank180137 +180138 POINT(40.89007950562276 73.99289826611052) bank180138 +180139 POINT(41.1405351566182 74.74095551399357) bank180139 +180140 POINT(40.8761529120881 73.82843728402626) bank180140 +180141 POINT(40.665317310933695 74.95869098974563) bank180141 +180142 POINT(40.33280257641937 74.95914041113889) bank180142 +180143 POINT(41.29392296900091 73.47305181812851) bank180143 +180144 POINT(40.26526343645132 73.72104220356663) bank180144 +180145 POINT(40.28740782879135 74.59703896304329) bank180145 +180146 POINT(41.385107782260356 73.3817207784258) bank180146 +180147 POINT(40.05658857821557 73.89059591630759) bank180147 +180148 POINT(41.48190425457252 74.62260489905701) bank180148 +180149 POINT(40.78707721720129 74.67887531012931) bank180149 +180150 POINT(40.25665226059987 73.85631204210469) bank180150 +180151 POINT(40.94113294652559 73.59802719252562) bank180151 +180152 POINT(39.87663097349329 74.71576652714205) bank180152 +180153 POINT(40.68126351641755 73.20157379762986) bank180153 +180154 POINT(41.48679008444684 73.13321977072748) bank180154 +180155 POINT(40.50465507068745 74.43538148690885) bank180155 +180156 POINT(39.93984133375365 73.97476699345414) bank180156 +180157 POINT(40.67735049416954 73.59236132399751) bank180157 +180158 POINT(40.36214526604509 73.23556230869154) bank180158 +180159 POINT(40.008506624728575 73.40261099010408) bank180159 +180160 POINT(39.98655771387887 74.86874810349487) bank180160 +180161 POINT(41.21335567520326 74.52350458125605) bank180161 +180162 POINT(40.49877176034207 74.98853322397781) bank180162 +180163 POINT(41.29118038675658 74.02488251765074) bank180163 +180164 POINT(41.48237953756177 73.04756702138636) bank180164 +180165 POINT(39.88485298142376 74.74354253544294) bank180165 +180166 POINT(40.492082167653585 73.99665559062814) bank180166 +180167 POINT(41.29577144858411 74.54608752818247) bank180167 +180168 POINT(39.76946938057355 74.7204483081637) bank180168 +180169 POINT(40.82920205000808 73.30927645944371) bank180169 +180170 POINT(40.97322565212331 74.45246406614903) bank180170 +180171 POINT(39.93856653326475 74.53392844532547) bank180171 +180172 POINT(39.73950929770799 73.83629535853656) bank180172 +180173 POINT(41.689391725431996 73.16085519748614) bank180173 +180174 POINT(39.98647093891794 73.48387458789495) bank180174 +180175 POINT(39.94763462528187 73.35521102451565) bank180175 +180176 POINT(40.702570249575245 73.5073088542308) bank180176 +180177 POINT(39.799666509952964 74.51892798329979) bank180177 +180178 POINT(40.80183091904949 73.79469552776955) bank180178 +180179 POINT(40.08953939272949 74.17858065496848) bank180179 +180180 POINT(40.962559050009155 74.38592693829318) bank180180 +180181 POINT(40.53992505030468 73.80394512028774) bank180181 +180182 POINT(41.105439516149715 73.84953409429382) bank180182 +180183 POINT(40.73620028075026 74.43167945741743) bank180183 +180184 POINT(41.21503984692735 74.05212499893902) bank180184 +180185 POINT(40.79884569293654 74.62041617233412) bank180185 +180186 POINT(39.984476985056055 73.7844002307472) bank180186 +180187 POINT(41.069970397482344 73.88833191956742) bank180187 +180188 POINT(40.07120814882772 74.89060894968979) bank180188 +180189 POINT(41.55989219465773 74.28338394381376) bank180189 +180190 POINT(41.641073222241076 73.96817860902239) bank180190 +180191 POINT(41.247010287590726 74.17327478444784) bank180191 +180192 POINT(40.104367042034326 74.6348851147161) bank180192 +180193 POINT(40.100800825556 74.23886831304122) bank180193 +180194 POINT(40.439528400317464 74.15967784612323) bank180194 +180195 POINT(40.892560145209075 74.26497644781136) bank180195 +180196 POINT(41.066474767265866 73.2493804220301) bank180196 +180197 POINT(40.84565215400559 74.69490234348484) bank180197 +180198 POINT(39.81756806838052 73.27084166840227) bank180198 +180199 POINT(40.884118486167 74.21392174241666) bank180199 +180200 POINT(40.76101036277238 74.56299757959792) bank180200 +180201 POINT(40.70267245749691 74.37890308304283) bank180201 +180202 POINT(41.67543147960294 74.29309396931765) bank180202 +180203 POINT(39.8164124304684 73.9159750405179) bank180203 +180204 POINT(40.11018298528333 73.01986767457586) bank180204 +180205 POINT(40.47696229830806 74.21792869553522) bank180205 +180206 POINT(40.098931474107765 74.69194391856259) bank180206 +180207 POINT(39.842682836538565 73.21668754918501) bank180207 +180208 POINT(41.2392745227691 73.62926061432343) bank180208 +180209 POINT(40.32095941128352 74.40916114804634) bank180209 +180210 POINT(41.53978006936692 74.10765417235343) bank180210 +180211 POINT(41.02559631694945 74.09755705117841) bank180211 +180212 POINT(41.1777023674543 74.89928688994036) bank180212 +180213 POINT(41.154266532921 73.84997822665912) bank180213 +180214 POINT(39.942888239697815 73.5071537205832) bank180214 +180215 POINT(41.48414892086612 73.82822283622113) bank180215 +180216 POINT(41.37094787152022 74.72926201994703) bank180216 +180217 POINT(40.48804384539624 73.22934534231) bank180217 +180218 POINT(40.905030666955575 73.32270052626251) bank180218 +180219 POINT(40.771845186705995 74.44073417846826) bank180219 +180220 POINT(39.94216682263554 74.80304655066719) bank180220 +180221 POINT(41.133155184699916 74.38350301956245) bank180221 +180222 POINT(41.39823010754198 74.54772250329235) bank180222 +180223 POINT(41.30226280409834 73.60357069798532) bank180223 +180224 POINT(40.272885509446326 73.84822157063509) bank180224 +180225 POINT(39.926336354072006 73.72791148678814) bank180225 +180226 POINT(39.73475725907848 73.9184341586761) bank180226 +180227 POINT(41.33253710670185 73.48173024539044) bank180227 +180228 POINT(40.47275191296325 73.19864863052865) bank180228 +180229 POINT(40.819846236856876 74.28585426752545) bank180229 +180230 POINT(41.25116643506167 74.80797821812611) bank180230 +180231 POINT(41.42556040575876 74.17458197012525) bank180231 +180232 POINT(40.24345061500526 73.53165002861242) bank180232 +180233 POINT(41.38235610095374 73.67820901717815) bank180233 +180234 POINT(40.42938760458698 73.96398639902121) bank180234 +180235 POINT(40.2005806145715 74.45052860510054) bank180235 +180236 POINT(41.31204093659524 74.58837388637055) bank180236 +180237 POINT(39.98810037917183 73.72820993070725) bank180237 +180238 POINT(39.95852401002228 74.81473446761207) bank180238 +180239 POINT(39.75946758223348 73.93856229671748) bank180239 +180240 POINT(40.298051133139 73.4931199145998) bank180240 +180241 POINT(39.81058943229679 74.77149309342016) bank180241 +180242 POINT(41.261397166130514 73.58625153585514) bank180242 +180243 POINT(39.9939679744335 74.997410035471) bank180243 +180244 POINT(40.62275208838861 74.4332801472231) bank180244 +180245 POINT(39.88524192220031 73.63030329926664) bank180245 +180246 POINT(40.41389827002641 74.82172783778894) bank180246 +180247 POINT(41.183262626238125 73.39621767044427) bank180247 +180248 POINT(40.738476037604656 74.04631376165257) bank180248 +180249 POINT(40.56232527721095 74.05165625875908) bank180249 +180250 POINT(40.79378477285756 74.19665729242176) bank180250 +180251 POINT(40.48243346060551 73.23878955866891) bank180251 +180252 POINT(41.24707392067358 73.0896123845947) bank180252 +180253 POINT(40.029575730995965 74.02483895590662) bank180253 +180254 POINT(39.94995269247185 73.54675586638731) bank180254 +180255 POINT(39.902434759385436 73.37148984043682) bank180255 +180256 POINT(39.80541555676469 74.74945613798901) bank180256 +180257 POINT(41.30587932282391 73.09407062444808) bank180257 +180258 POINT(41.013054541737155 73.50442313978296) bank180258 +180259 POINT(41.31571472373338 73.40063497000092) bank180259 +180260 POINT(40.71482034486072 73.71185613666513) bank180260 +180261 POINT(41.441325921797386 74.22536957405813) bank180261 +180262 POINT(40.78059157648562 73.73551686055247) bank180262 +180263 POINT(40.07057477042059 74.51068861541214) bank180263 +180264 POINT(41.36479717333719 73.9324975598019) bank180264 +180265 POINT(39.89980702693635 74.90245609295712) bank180265 +180266 POINT(39.839761135275396 73.40891913777725) bank180266 +180267 POINT(40.47017633348659 73.11808126982521) bank180267 +180268 POINT(41.28453911959582 74.61887895262882) bank180268 +180269 POINT(41.32427280047859 73.38163378249658) bank180269 +180270 POINT(41.50622852733332 73.78203923767833) bank180270 +180271 POINT(40.30740325905889 73.34005883187588) bank180271 +180272 POINT(40.30391565640125 74.5114024631406) bank180272 +180273 POINT(40.641786282857815 74.68959456867998) bank180273 +180274 POINT(40.48891108741532 74.82670615801463) bank180274 +180275 POINT(39.98496990792916 73.92410762789332) bank180275 +180276 POINT(41.202481281095906 74.72962272571371) bank180276 +180277 POINT(41.22766283079676 74.34068888776376) bank180277 +180278 POINT(41.58000348875523 73.40352909561409) bank180278 +180279 POINT(40.577639822316215 73.25443932261793) bank180279 +180280 POINT(40.99264925375697 74.89610627222156) bank180280 +180281 POINT(41.666605480689206 73.48803254531015) bank180281 +180282 POINT(39.76191090939202 74.4909090974625) bank180282 +180283 POINT(40.029780172738846 74.60466833730608) bank180283 +180284 POINT(41.21722633462695 74.05116598700444) bank180284 +180285 POINT(41.36312707880411 74.29082636703023) bank180285 +180286 POINT(40.222798607043146 74.75534981165215) bank180286 +180287 POINT(39.751879363026156 74.97109181454647) bank180287 +180288 POINT(39.831780355605346 73.16554863168658) bank180288 +180289 POINT(40.66913319358865 74.49425609839106) bank180289 +180290 POINT(39.85976952968034 74.16214941807125) bank180290 +180291 POINT(40.49132138287931 74.34310283618939) bank180291 +180292 POINT(40.96667174514648 73.50914947402147) bank180292 +180293 POINT(40.26725231724531 73.97800270210058) bank180293 +180294 POINT(40.52552407063232 74.61072082006854) bank180294 +180295 POINT(41.68823928728711 73.39550973084282) bank180295 +180296 POINT(40.48001509314288 74.9573305743525) bank180296 +180297 POINT(41.6676535744876 73.70495702007608) bank180297 +180298 POINT(39.72942157977498 74.63407733168557) bank180298 +180299 POINT(40.29201634047956 73.96773901601942) bank180299 +180300 POINT(41.46048001052305 73.29050969512788) bank180300 +180301 POINT(39.92159778730832 73.71032420842707) bank180301 +180302 POINT(40.486994674713074 73.83510785028544) bank180302 +180303 POINT(40.09301435649416 73.86366022667737) bank180303 +180304 POINT(39.93543184978153 73.62879894075807) bank180304 +180305 POINT(40.49796527048026 74.83087723841263) bank180305 +180306 POINT(41.37147851610236 74.07059454718477) bank180306 +180307 POINT(41.493869696505364 73.38903652418895) bank180307 +180308 POINT(41.545625684321855 74.58963291754444) bank180308 +180309 POINT(40.86966702659351 74.81643106538192) bank180309 +180310 POINT(40.45109625985086 73.47574440767147) bank180310 +180311 POINT(40.29644657405085 73.44609454392729) bank180311 +180312 POINT(40.26732338390813 74.72529240811649) bank180312 +180313 POINT(39.97967756439466 73.91819169910525) bank180313 +180314 POINT(40.785082505681046 73.63499963035484) bank180314 +180315 POINT(41.46265085544351 73.57419887689645) bank180315 +180316 POINT(40.41847078357837 74.43815257216087) bank180316 +180317 POINT(41.53738661160554 74.74145686695532) bank180317 +180318 POINT(41.61899191757947 73.51335715893406) bank180318 +180319 POINT(40.24708634846145 74.25117087484767) bank180319 +180320 POINT(40.53282886138176 73.64469144999754) bank180320 +180321 POINT(40.53921736048197 73.97605983170078) bank180321 +180322 POINT(40.874230069278326 74.32658732393011) bank180322 +180323 POINT(40.905148214044466 73.33851615304336) bank180323 +180324 POINT(39.88218304935519 74.72630980746497) bank180324 +180325 POINT(40.301239624495494 74.24668565763749) bank180325 +180326 POINT(41.33273726202628 74.22790692430891) bank180326 +180327 POINT(40.78585644830988 74.04523765868107) bank180327 +180328 POINT(41.282957094615526 73.34693390536351) bank180328 +180329 POINT(41.006264058894104 73.71162180434101) bank180329 +180330 POINT(40.23632522584285 74.49374164046569) bank180330 +180331 POINT(40.70787084913597 73.09168430207251) bank180331 +180332 POINT(39.75819752195268 74.72132764808912) bank180332 +180333 POINT(41.49319833880414 73.49573188222774) bank180333 +180334 POINT(40.63995765165754 73.69404812486121) bank180334 +180335 POINT(39.92525881669719 74.62121320259172) bank180335 +180336 POINT(40.12373329988157 73.52267602689162) bank180336 +180337 POINT(39.92332511546786 73.40163902895287) bank180337 +180338 POINT(41.237080086458796 74.2515040979616) bank180338 +180339 POINT(40.173156455048286 73.84644696279722) bank180339 +180340 POINT(41.26944353745806 74.98669708562468) bank180340 +180341 POINT(41.113311374900796 74.1140758890936) bank180341 +180342 POINT(40.87998719546985 74.85069214215592) bank180342 +180343 POINT(40.30721367098331 73.07238820371278) bank180343 +180344 POINT(40.015510534541754 74.9726493506179) bank180344 +180345 POINT(41.64945910653364 73.4200421986736) bank180345 +180346 POINT(41.40275719487017 73.6585163971405) bank180346 +180347 POINT(41.27461200460658 73.07598872848432) bank180347 +180348 POINT(41.442882383401475 74.4588825939671) bank180348 +180349 POINT(40.83382142227288 73.53146589184708) bank180349 +180350 POINT(40.696722052645065 73.35655953090941) bank180350 +180351 POINT(41.351591105259374 73.14838216041572) bank180351 +180352 POINT(41.60977557526585 73.59859571253381) bank180352 +180353 POINT(40.75050993347289 74.13346300461224) bank180353 +180354 POINT(41.44371403620255 73.19256587078674) bank180354 +180355 POINT(40.973495673014206 73.92482438768675) bank180355 +180356 POINT(40.6832702831459 74.6866955205729) bank180356 +180357 POINT(40.85119237289684 74.76030735518971) bank180357 +180358 POINT(39.94873963461722 74.08396013829534) bank180358 +180359 POINT(39.792759956792935 74.59073674149593) bank180359 +180360 POINT(41.29621673615996 74.94605348814032) bank180360 +180361 POINT(40.14381914693532 73.77510873478654) bank180361 +180362 POINT(40.47911161107421 73.19360059199815) bank180362 +180363 POINT(40.19786080346855 74.10062679114158) bank180363 +180364 POINT(40.01870195781987 73.26287662362758) bank180364 +180365 POINT(40.25129542226844 73.13961689651077) bank180365 +180366 POINT(40.334863061190475 73.62396729268582) bank180366 +180367 POINT(40.19346380083711 74.64566629750294) bank180367 +180368 POINT(40.48645566854886 74.9851140929667) bank180368 +180369 POINT(41.158518609140195 73.75084225232675) bank180369 +180370 POINT(40.43750040201477 74.32745475102784) bank180370 +180371 POINT(39.94168566090724 73.13478124378099) bank180371 +180372 POINT(40.24947254499912 74.2659038904216) bank180372 +180373 POINT(39.742224004873094 74.74106578639608) bank180373 +180374 POINT(40.63994383352525 73.24495442937604) bank180374 +180375 POINT(41.197685158495425 74.72775736174695) bank180375 +180376 POINT(41.039875015137575 73.29756705852073) bank180376 +180377 POINT(39.83136180121129 73.7210392154102) bank180377 +180378 POINT(39.933032231028456 74.85945356946522) bank180378 +180379 POINT(40.45165596521333 73.68823298486404) bank180379 +180380 POINT(40.37778696233908 74.29310116457906) bank180380 +180381 POINT(41.043861861910116 73.87110435955472) bank180381 +180382 POINT(41.6558171090958 74.86977345357707) bank180382 +180383 POINT(40.78509935255725 73.0771010477701) bank180383 +180384 POINT(41.629681095330994 74.15760134087832) bank180384 +180385 POINT(41.30556046003398 73.55509409316727) bank180385 +180386 POINT(40.622761931495496 74.10709787428124) bank180386 +180387 POINT(41.47996825623518 73.32783854990288) bank180387 +180388 POINT(40.73427566309035 74.47505133123656) bank180388 +180389 POINT(39.79454118961202 73.79375511770725) bank180389 +180390 POINT(41.37420802884794 74.63327214393372) bank180390 +180391 POINT(39.83673901801057 73.48999244395864) bank180391 +180392 POINT(41.708577882578695 73.02473015779827) bank180392 +180393 POINT(41.0957092085126 74.02209305534107) bank180393 +180394 POINT(40.05801514221903 73.37962463312147) bank180394 +180395 POINT(40.059497785516506 73.50542191952387) bank180395 +180396 POINT(40.727764286974065 74.49475378568759) bank180396 +180397 POINT(40.10970219202109 73.56361005556828) bank180397 +180398 POINT(39.869042799717654 74.71166095210012) bank180398 +180399 POINT(41.2883049960665 73.4421086743966) bank180399 +180400 POINT(41.59008125016541 74.92210636081157) bank180400 +180401 POINT(41.35446031695057 73.44459208351785) bank180401 +180402 POINT(40.35643816877858 74.81380199204563) bank180402 +180403 POINT(41.36229247987997 73.7816363573978) bank180403 +180404 POINT(41.11277758944187 73.85349006602229) bank180404 +180405 POINT(40.735894426814745 73.81125120674588) bank180405 +180406 POINT(40.47981271717282 73.1592411112571) bank180406 +180407 POINT(40.48567480101146 74.26962841233893) bank180407 +180408 POINT(40.23744309864763 73.01780259205975) bank180408 +180409 POINT(39.78301434271487 73.34703789117476) bank180409 +180410 POINT(40.59210574445032 74.98387604214018) bank180410 +180411 POINT(40.006589691091015 74.88183799954717) bank180411 +180412 POINT(40.64602937448298 73.96603936826858) bank180412 +180413 POINT(39.88914135124841 73.51553533405834) bank180413 +180414 POINT(41.202163816395846 73.51625573690313) bank180414 +180415 POINT(41.2751680706522 74.20615122932853) bank180415 +180416 POINT(39.91800982192848 74.68069056784766) bank180416 +180417 POINT(40.66561016274378 73.04237382754825) bank180417 +180418 POINT(39.94037061166796 74.92001661957863) bank180418 +180419 POINT(39.88262304093869 73.58568442938548) bank180419 +180420 POINT(41.04515492201025 74.72173945873676) bank180420 +180421 POINT(41.7084885296492 74.29042822124724) bank180421 +180422 POINT(41.29088864725957 74.48708131415272) bank180422 +180423 POINT(40.73090336692624 73.04086393393126) bank180423 +180424 POINT(41.348267818418186 74.21433684367443) bank180424 +180425 POINT(39.89182943269863 74.24960015542828) bank180425 +180426 POINT(40.29600395755566 74.69479130053898) bank180426 +180427 POINT(40.77851989205931 73.96847577078876) bank180427 +180428 POINT(40.4048471357385 74.875625950613) bank180428 +180429 POINT(40.41949936240595 74.1377171423913) bank180429 +180430 POINT(39.82449978290607 74.72924554352218) bank180430 +180431 POINT(40.57420843718474 73.77576485992897) bank180431 +180432 POINT(40.27866180253895 74.40862361352305) bank180432 +180433 POINT(41.324938688664076 74.39983785123711) bank180433 +180434 POINT(41.38523283156905 74.43971804160857) bank180434 +180435 POINT(40.17382801174925 73.42766708416366) bank180435 +180436 POINT(40.78074421254285 74.0394144745069) bank180436 +180437 POINT(40.427380251088955 74.48181430689866) bank180437 +180438 POINT(40.81766237309917 73.36752328847687) bank180438 +180439 POINT(39.732992606992795 73.93298060303395) bank180439 +180440 POINT(40.60955989956113 74.52152171723792) bank180440 +180441 POINT(40.67648776814431 74.73920566579393) bank180441 +180442 POINT(40.095796943054204 73.29775632883677) bank180442 +180443 POINT(41.35298294496959 74.94374219044231) bank180443 +180444 POINT(40.585983623493895 74.34807310141302) bank180444 +180445 POINT(41.68975738240274 73.54960716246856) bank180445 +180446 POINT(41.37731467711719 73.99075248496278) bank180446 +180447 POINT(40.906359194628564 73.92500641851846) bank180447 +180448 POINT(41.46016190306325 74.47747530034965) bank180448 +180449 POINT(41.286520819888864 74.96719154031685) bank180449 +180450 POINT(40.30958098616159 74.9631100048755) bank180450 +180451 POINT(41.441032189703115 73.52922719767076) bank180451 +180452 POINT(40.50803977545749 74.69816338353974) bank180452 +180453 POINT(39.80393721393105 73.67708556904606) bank180453 +180454 POINT(40.08257314153901 74.65849574653646) bank180454 +180455 POINT(39.997568027737586 74.74620224964387) bank180455 +180456 POINT(40.166789923776214 73.28353601113524) bank180456 +180457 POINT(40.272721181447665 74.34221568811103) bank180457 +180458 POINT(41.70208071905608 73.16766700837593) bank180458 +180459 POINT(40.10251251739189 74.69376317387622) bank180459 +180460 POINT(41.190880621049146 73.51581902360543) bank180460 +180461 POINT(41.707906803158906 73.83560061979476) bank180461 +180462 POINT(41.136422935734146 73.02779956281135) bank180462 +180463 POINT(40.63014381400404 73.17108195240763) bank180463 +180464 POINT(40.35443356604054 73.26838015789149) bank180464 +180465 POINT(40.91212075747236 74.84935346981179) bank180465 +180466 POINT(41.502114014382066 73.92619582437106) bank180466 +180467 POINT(41.259928560454156 74.60189144510016) bank180467 +180468 POINT(39.97034018722704 74.25604530171537) bank180468 +180469 POINT(41.294190610854564 74.76797114662493) bank180469 +180470 POINT(40.10834986752922 74.38999484647942) bank180470 +180471 POINT(40.73090396275488 73.28792455352138) bank180471 +180472 POINT(39.799051913003645 73.32033685994901) bank180472 +180473 POINT(40.5532142363572 74.69062718071979) bank180473 +180474 POINT(41.6799347548027 74.68214918936607) bank180474 +180475 POINT(40.85436018918158 73.23635005583512) bank180475 +180476 POINT(39.88933839202339 73.78152837008146) bank180476 +180477 POINT(40.88218283942549 74.48090139155624) bank180477 +180478 POINT(41.50757762996708 74.12702237871684) bank180478 +180479 POINT(40.11573668831561 73.54239809362458) bank180479 +180480 POINT(41.11742929112633 74.17684795661062) bank180480 +180481 POINT(40.248469150296 73.98395148982732) bank180481 +180482 POINT(41.018973127906605 74.63488287743063) bank180482 +180483 POINT(41.01208047059284 73.26650323789973) bank180483 +180484 POINT(41.06787599859889 74.15181793923384) bank180484 +180485 POINT(40.74851655974137 74.87779490874969) bank180485 +180486 POINT(40.77503120321125 74.00730566075998) bank180486 +180487 POINT(40.16418363636046 73.0389086282788) bank180487 +180488 POINT(40.58167130017704 73.23810405278097) bank180488 +180489 POINT(40.80181145640749 73.46300752386716) bank180489 +180490 POINT(40.72665015962898 74.21560313645764) bank180490 +180491 POINT(39.822196218391525 73.12193885818917) bank180491 +180492 POINT(40.58830692184501 73.45261949516005) bank180492 +180493 POINT(40.72002731559816 73.34373261075348) bank180493 +180494 POINT(40.85155684779284 73.41900912494812) bank180494 +180495 POINT(41.222220218443276 73.17520912205052) bank180495 +180496 POINT(41.3259964417286 73.37432449466303) bank180496 +180497 POINT(40.85359761649526 73.11993420254268) bank180497 +180498 POINT(40.98310990973771 74.68239062311899) bank180498 +180499 POINT(41.36881566067636 73.52481042089607) bank180499 +180500 POINT(39.72433037202236 74.0860307159252) bank180500 +180501 POINT(40.89230432562897 74.93085060895488) bank180501 +180502 POINT(40.55550057174103 74.11806918438005) bank180502 +180503 POINT(40.62372575031354 74.98483484047578) bank180503 +180504 POINT(40.72856707253863 74.46806907773752) bank180504 +180505 POINT(41.0877601260558 73.55014511757494) bank180505 +180506 POINT(41.45872479175209 74.03350788365381) bank180506 +180507 POINT(40.61669011212977 73.65811034015731) bank180507 +180508 POINT(40.1599956081374 73.96401792763324) bank180508 +180509 POINT(39.752653280789836 73.96946278725927) bank180509 +180510 POINT(39.934312886671094 74.32036555524554) bank180510 +180511 POINT(40.49386116418503 73.3054984379834) bank180511 +180512 POINT(40.332813907894796 74.7728673540442) bank180512 +180513 POINT(41.087823547851016 74.3845169950588) bank180513 +180514 POINT(40.1059203954764 74.36297865174487) bank180514 +180515 POINT(41.26682196377976 74.90814118945057) bank180515 +180516 POINT(41.670352041311915 73.69011232915928) bank180516 +180517 POINT(41.45980559234592 73.80016625147901) bank180517 +180518 POINT(39.795664753210175 73.67876658646873) bank180518 +180519 POINT(40.609232578737 73.78468075647287) bank180519 +180520 POINT(41.39635608856683 74.8260899658424) bank180520 +180521 POINT(39.83447369517465 74.79566393645456) bank180521 +180522 POINT(39.98664634897458 73.63163202239319) bank180522 +180523 POINT(40.28721490583986 73.84218107631396) bank180523 +180524 POINT(40.99452533892604 73.60839481892779) bank180524 +180525 POINT(41.10344018734366 73.46525606377304) bank180525 +180526 POINT(40.22806345053108 73.95414200026192) bank180526 +180527 POINT(40.98361710677748 74.1185939858657) bank180527 +180528 POINT(41.407320274480256 74.94499384186338) bank180528 +180529 POINT(40.756064376384785 74.81874898238705) bank180529 +180530 POINT(41.12201500998119 73.33206543031456) bank180530 +180531 POINT(40.5567981000409 73.86676269170907) bank180531 +180532 POINT(41.34791273579136 73.23418041755703) bank180532 +180533 POINT(40.708108935355476 73.34601639830076) bank180533 +180534 POINT(39.77702928581005 73.50720404279025) bank180534 +180535 POINT(40.89750633483247 73.97060776778848) bank180535 +180536 POINT(41.04292119041051 73.87840659652406) bank180536 +180537 POINT(41.44932332953101 74.18255496292502) bank180537 +180538 POINT(40.45949823249563 73.26785908276074) bank180538 +180539 POINT(41.46412662699185 74.35240037288283) bank180539 +180540 POINT(41.405546347175196 74.87433035710298) bank180540 +180541 POINT(40.93254277458918 74.17821038380482) bank180541 +180542 POINT(40.637841218160645 73.8701481968425) bank180542 +180543 POINT(41.03819651114565 74.68042421901619) bank180543 +180544 POINT(40.014804987950896 73.26146224308303) bank180544 +180545 POINT(40.98828775517322 74.91007906626143) bank180545 +180546 POINT(40.11879918239354 73.74155004420147) bank180546 +180547 POINT(40.25943847406242 73.7830848708003) bank180547 +180548 POINT(40.93594754128488 74.8799413509805) bank180548 +180549 POINT(41.01237896427852 73.43304450920034) bank180549 +180550 POINT(40.71822060841415 74.17637585658234) bank180550 +180551 POINT(40.07287873435931 74.25023518117332) bank180551 +180552 POINT(40.488618849781396 73.61654135522544) bank180552 +180553 POINT(41.32414564553598 73.6676089573858) bank180553 +180554 POINT(41.05562502584841 74.47506709984748) bank180554 +180555 POINT(39.81581912837614 73.98949331409666) bank180555 +180556 POINT(41.50232576283606 74.98190546222982) bank180556 +180557 POINT(40.06939328267866 74.69318492583382) bank180557 +180558 POINT(41.52268840992513 74.66047162972993) bank180558 +180559 POINT(40.362286156096154 74.66341120674066) bank180559 +180560 POINT(40.79501160215197 74.40199152441834) bank180560 +180561 POINT(40.19996053744732 73.06972932432708) bank180561 +180562 POINT(40.6967982386439 73.95419858902862) bank180562 +180563 POINT(41.549968965118616 74.50547948205956) bank180563 +180564 POINT(41.3500112632247 74.61282337981893) bank180564 +180565 POINT(40.32416685111265 73.78554061170811) bank180565 +180566 POINT(39.88870781690723 73.06875237993867) bank180566 +180567 POINT(40.31381028447999 74.83181921819651) bank180567 +180568 POINT(41.52818733265193 73.55090715532731) bank180568 +180569 POINT(39.836592846738235 74.88866939562922) bank180569 +180570 POINT(39.857911443263816 73.5987711641211) bank180570 +180571 POINT(40.45945761139721 74.5981034289231) bank180571 +180572 POINT(39.95634056550217 73.26023644933898) bank180572 +180573 POINT(41.47891683766915 73.81011846696846) bank180573 +180574 POINT(40.11635182602616 73.08143760700324) bank180574 +180575 POINT(40.834558580832386 74.56427826817117) bank180575 +180576 POINT(39.7653013137887 74.61134742273714) bank180576 +180577 POINT(39.995544335733726 73.67539770355039) bank180577 +180578 POINT(39.82866867205463 73.46173601726798) bank180578 +180579 POINT(41.55932404147113 74.42802206347137) bank180579 +180580 POINT(41.293800453377244 74.10346227255563) bank180580 +180581 POINT(40.466118680373214 74.16625433974214) bank180581 +180582 POINT(39.810556938022444 74.20743816139765) bank180582 +180583 POINT(41.39325230927508 74.08487140375637) bank180583 +180584 POINT(40.43413242176476 74.30899260530586) bank180584 +180585 POINT(41.60239922281006 74.55509629921204) bank180585 +180586 POINT(40.408305662544834 74.70744966394021) bank180586 +180587 POINT(40.115642939587296 73.5743227116008) bank180587 +180588 POINT(40.914526766123494 74.96138178936474) bank180588 +180589 POINT(39.984850157928804 74.97233317807988) bank180589 +180590 POINT(41.48122281465797 73.01261278854992) bank180590 +180591 POINT(41.053947703565846 73.45202813089571) bank180591 +180592 POINT(41.48030935263416 74.52771495300227) bank180592 +180593 POINT(40.79034034356786 74.27796319842946) bank180593 +180594 POINT(41.62438913284078 74.0781301919499) bank180594 +180595 POINT(40.84610561138341 73.19321313328678) bank180595 +180596 POINT(40.633301445604744 74.99593389019238) bank180596 +180597 POINT(40.20690838479591 74.97453885288304) bank180597 +180598 POINT(40.38597087214675 74.37263747287925) bank180598 +180599 POINT(40.459886443749376 74.57549908978446) bank180599 +180600 POINT(39.83094830274995 73.22669940648387) bank180600 +180601 POINT(41.64784813855424 73.57890746074986) bank180601 +180602 POINT(39.95664029251188 73.40184114926228) bank180602 +180603 POINT(39.97114613446222 73.5127153257524) bank180603 +180604 POINT(41.20496826972203 73.400317533924) bank180604 +180605 POINT(40.45736668956444 74.43312095589647) bank180605 +180606 POINT(40.3843419182042 74.15537888537285) bank180606 +180607 POINT(40.409548849876984 74.91509665726177) bank180607 +180608 POINT(41.260835301263285 74.99480867040975) bank180608 +180609 POINT(40.356519318272156 73.6461359027423) bank180609 +180610 POINT(39.83515073255676 73.59053079032502) bank180610 +180611 POINT(39.98984565074973 73.02728168790291) bank180611 +180612 POINT(41.112323969723306 74.40593132602541) bank180612 +180613 POINT(40.756822284980814 74.54259354903837) bank180613 +180614 POINT(40.123484539413205 73.13338490610404) bank180614 +180615 POINT(40.24565534297753 73.52860157101797) bank180615 +180616 POINT(41.40402963237403 74.33591069143957) bank180616 +180617 POINT(41.486438445244886 74.98805623562001) bank180617 +180618 POINT(40.20797836964852 74.30652738929469) bank180618 +180619 POINT(41.19161270547602 75.00229808047217) bank180619 +180620 POINT(41.60579545603349 74.72581116725529) bank180620 +180621 POINT(41.560386593608975 74.24783251453603) bank180621 +180622 POINT(41.14856963248964 74.72913631361747) bank180622 +180623 POINT(40.530417249792315 74.62779642132676) bank180623 +180624 POINT(41.40413033509865 73.33096847993424) bank180624 +180625 POINT(40.33065977422412 74.40262260943528) bank180625 +180626 POINT(40.64672431936904 73.82034563475268) bank180626 +180627 POINT(40.511329210221604 74.46519723785417) bank180627 +180628 POINT(41.326259933280404 74.8867813331861) bank180628 +180629 POINT(41.55851207025624 74.23174627666319) bank180629 +180630 POINT(41.44359270899234 74.51900654841315) bank180630 +180631 POINT(39.764031879049675 73.64754560439702) bank180631 +180632 POINT(41.54202221592852 74.52096591053474) bank180632 +180633 POINT(40.91917819030462 73.76364018864756) bank180633 +180634 POINT(40.10393828290152 74.07992349571614) bank180634 +180635 POINT(40.67425439672679 73.80686930496651) bank180635 +180636 POINT(40.93178819566209 73.7800869862819) bank180636 +180637 POINT(40.246867613506225 73.40364914633959) bank180637 +180638 POINT(40.63793364648762 74.32129611256413) bank180638 +180639 POINT(40.0412262244029 73.19486151052) bank180639 +180640 POINT(41.63909945045905 75.00450423107796) bank180640 +180641 POINT(40.516249793790124 73.86096701281906) bank180641 +180642 POINT(41.1071249313364 73.78230915439427) bank180642 +180643 POINT(41.356060827447834 73.10122157008418) bank180643 +180644 POINT(41.013744415004126 73.78181005668151) bank180644 +180645 POINT(40.1321217909416 73.21222939338251) bank180645 +180646 POINT(40.91582652547866 73.99726409180215) bank180646 +180647 POINT(41.608933470271914 73.19372588869825) bank180647 +180648 POINT(39.83781155944749 74.38826851904768) bank180648 +180649 POINT(40.48797065148931 73.2587456490844) bank180649 +180650 POINT(41.30589675378907 73.1490661648127) bank180650 +180651 POINT(40.24317290623677 74.05096723786835) bank180651 +180652 POINT(40.34436893375726 74.58840704331568) bank180652 +180653 POINT(41.2098402529872 73.19685062956006) bank180653 +180654 POINT(40.513059798229065 74.78233372050981) bank180654 +180655 POINT(40.36731195926095 74.08805855380925) bank180655 +180656 POINT(41.10138273599467 73.63408160642373) bank180656 +180657 POINT(40.861286542460256 74.1624891237923) bank180657 +180658 POINT(41.441589213754334 74.22179483265995) bank180658 +180659 POINT(41.54719169259461 74.89944682119318) bank180659 +180660 POINT(41.274627235225914 74.84497346176266) bank180660 +180661 POINT(40.19274311521275 73.85791918733186) bank180661 +180662 POINT(40.37384583959976 74.79299319031395) bank180662 +180663 POINT(41.36463902334483 74.88330765583663) bank180663 +180664 POINT(40.46073452260419 73.43711128314357) bank180664 +180665 POINT(39.82724264331718 73.16345519173028) bank180665 +180666 POINT(41.39927360895781 73.33580869113649) bank180666 +180667 POINT(41.06278982442233 74.66620715453318) bank180667 +180668 POINT(41.06641338309885 74.79582357272787) bank180668 +180669 POINT(40.65214997076426 74.5777914972323) bank180669 +180670 POINT(41.70694360268923 74.65711969684007) bank180670 +180671 POINT(41.617043660636995 73.40054683209694) bank180671 +180672 POINT(41.642116286837 73.74144185565832) bank180672 +180673 POINT(40.028243119610536 73.51911503543081) bank180673 +180674 POINT(41.5709627314467 73.80850730661834) bank180674 +180675 POINT(41.5154583801613 74.79976064465022) bank180675 +180676 POINT(40.66017587472574 74.23986552048208) bank180676 +180677 POINT(40.95448797409858 73.4013316332621) bank180677 +180678 POINT(41.6672988572299 73.71960197420495) bank180678 +180679 POINT(39.73942135249531 74.06259632712636) bank180679 +180680 POINT(41.22953328627029 74.89493207775368) bank180680 +180681 POINT(40.464819953720855 74.74205825930925) bank180681 +180682 POINT(40.070179327203604 73.88991553033104) bank180682 +180683 POINT(41.697561725739256 74.44010702260776) bank180683 +180684 POINT(41.522369109302666 74.92116285602775) bank180684 +180685 POINT(39.720384727198876 74.03858648087908) bank180685 +180686 POINT(40.94012227879382 73.51687566311378) bank180686 +180687 POINT(40.679113187187774 74.31855565315064) bank180687 +180688 POINT(40.515929746740944 73.89573029143892) bank180688 +180689 POINT(40.323487346090474 74.9906608063979) bank180689 +180690 POINT(41.49551943506525 73.23971319184575) bank180690 +180691 POINT(41.02268832417286 73.45996001512266) bank180691 +180692 POINT(40.05075861581526 74.44937112586395) bank180692 +180693 POINT(41.68196174834595 73.51770921546719) bank180693 +180694 POINT(40.089419123562955 74.27664817048141) bank180694 +180695 POINT(41.14882807017904 73.59465713840841) bank180695 +180696 POINT(41.5674837437137 74.61184794233762) bank180696 +180697 POINT(40.82693953447494 74.33351771918112) bank180697 +180698 POINT(40.18021963572714 73.41756570045678) bank180698 +180699 POINT(39.74777120638934 74.72118151976542) bank180699 +180700 POINT(41.576689972648076 73.54750551331692) bank180700 +180701 POINT(40.63548522490503 74.29621248154054) bank180701 +180702 POINT(40.19045698506929 74.47012605487514) bank180702 +180703 POINT(40.780350807456124 73.26104930670617) bank180703 +180704 POINT(40.36859290634214 74.85127553822984) bank180704 +180705 POINT(40.80618644688087 74.36206333942941) bank180705 +180706 POINT(40.81668004714723 74.32513629203365) bank180706 +180707 POINT(39.75347530702893 74.88298225304149) bank180707 +180708 POINT(39.86371404123089 74.13897349948165) bank180708 +180709 POINT(40.97339276224584 73.84617412398023) bank180709 +180710 POINT(41.25319623871419 73.30208010710471) bank180710 +180711 POINT(41.18644395653842 74.79166475675345) bank180711 +180712 POINT(40.30148915765055 74.49042972885587) bank180712 +180713 POINT(41.16487912401828 73.39467308952602) bank180713 +180714 POINT(40.54122730290134 74.06176383122028) bank180714 +180715 POINT(39.930910743600975 74.34989705277462) bank180715 +180716 POINT(40.96369689601468 74.30461904022002) bank180716 +180717 POINT(40.79535722059192 74.29556026004472) bank180717 +180718 POINT(40.842615976980625 74.04941611913517) bank180718 +180719 POINT(40.2880321937597 74.41010964524243) bank180719 +180720 POINT(40.26542287682908 73.27829068308007) bank180720 +180721 POINT(41.28496901698205 73.55255191748613) bank180721 +180722 POINT(40.98211342277154 73.80059596561523) bank180722 +180723 POINT(40.14816982365403 73.34562006028487) bank180723 +180724 POINT(40.065652617250194 73.58349033832368) bank180724 +180725 POINT(39.88428654694211 73.21251095345116) bank180725 +180726 POINT(40.9860618019016 74.64875915623436) bank180726 +180727 POINT(41.21051542816378 74.49756576203612) bank180727 +180728 POINT(40.9696819069132 73.47667746349684) bank180728 +180729 POINT(41.63287480633701 73.06656531408866) bank180729 +180730 POINT(40.03547606636177 73.52295271100306) bank180730 +180731 POINT(40.12227608056932 74.65569672780238) bank180731 +180732 POINT(41.571622933761006 73.6829851888671) bank180732 +180733 POINT(41.2640054946531 73.72451767676556) bank180733 +180734 POINT(39.7640603514237 73.14501843277243) bank180734 +180735 POINT(39.772750673513904 74.39880116151413) bank180735 +180736 POINT(40.398030420237745 74.37756111657549) bank180736 +180737 POINT(41.54181310891126 74.45086435836875) bank180737 +180738 POINT(41.42634544209769 74.42418346858055) bank180738 +180739 POINT(39.759580044117754 74.42808527812508) bank180739 +180740 POINT(41.685898543577785 74.33746725275684) bank180740 +180741 POINT(40.383593486613435 73.04209095885955) bank180741 +180742 POINT(40.87750895400623 73.16815681625286) bank180742 +180743 POINT(41.37244950596848 73.38779756044653) bank180743 +180744 POINT(40.78900621341501 73.39370037816646) bank180744 +180745 POINT(40.80629416996315 73.12035030439854) bank180745 +180746 POINT(40.53342831342709 73.97920157110002) bank180746 +180747 POINT(41.701072502476016 74.39744521749358) bank180747 +180748 POINT(40.559360617253 73.90557333903021) bank180748 +180749 POINT(41.27528691681286 73.68655681420692) bank180749 +180750 POINT(41.610298387898624 73.60489021729902) bank180750 +180751 POINT(41.04879388143666 73.0843728746991) bank180751 +180752 POINT(40.30050265182531 74.01518515219183) bank180752 +180753 POINT(40.69363913825034 73.84550415412808) bank180753 +180754 POINT(40.40686203850428 73.24638877000775) bank180754 +180755 POINT(39.96682529611224 74.45865940057256) bank180755 +180756 POINT(41.540465626438085 73.75925817382206) bank180756 +180757 POINT(41.503835741768256 74.02598646710558) bank180757 +180758 POINT(40.30886900355104 74.01320577369623) bank180758 +180759 POINT(40.29641551615424 73.15171590688735) bank180759 +180760 POINT(41.500194232822075 73.93356872713646) bank180760 +180761 POINT(40.2470941045614 73.78559304158583) bank180761 +180762 POINT(40.3423217573641 74.36172982288244) bank180762 +180763 POINT(41.424928548774496 73.83396407166738) bank180763 +180764 POINT(41.36265370756582 74.27695564868246) bank180764 +180765 POINT(41.50915377748078 73.01522788901542) bank180765 +180766 POINT(40.48402704158216 74.78016347703044) bank180766 +180767 POINT(40.48576288941236 74.72318820240329) bank180767 +180768 POINT(40.93147089084197 73.21471322392746) bank180768 +180769 POINT(40.07886080551695 73.43421726771493) bank180769 +180770 POINT(40.60754595873388 73.97282420332238) bank180770 +180771 POINT(41.33781181036129 73.02488764012568) bank180771 +180772 POINT(40.54844449021748 74.54236838789235) bank180772 +180773 POINT(41.48157546078225 74.95665713889332) bank180773 +180774 POINT(40.23874343032732 73.37665645592645) bank180774 +180775 POINT(40.67082976211864 73.09999982298424) bank180775 +180776 POINT(41.62335510299103 74.70608587410955) bank180776 +180777 POINT(39.97545709577071 74.37349153161468) bank180777 +180778 POINT(40.85410960543352 73.1846133082571) bank180778 +180779 POINT(39.91059839340925 73.7283884111701) bank180779 +180780 POINT(40.97895486179263 74.69867011608495) bank180780 +180781 POINT(40.36002463387511 73.39273497545639) bank180781 +180782 POINT(40.259855520519054 73.09560815473881) bank180782 +180783 POINT(39.74458661594937 73.50154995564503) bank180783 +180784 POINT(40.808074578354855 74.46553797338112) bank180784 +180785 POINT(40.54169326664905 73.78553229540876) bank180785 +180786 POINT(40.73708675147222 73.25130392219151) bank180786 +180787 POINT(41.69188521999407 74.81467531183739) bank180787 +180788 POINT(41.21422070881699 74.45735547025437) bank180788 +180789 POINT(40.348900632021376 73.72429432881596) bank180789 +180790 POINT(41.52958592760252 73.83974123320732) bank180790 +180791 POINT(41.23416755526052 73.74406851665762) bank180791 +180792 POINT(41.65082555245638 73.68504326828676) bank180792 +180793 POINT(41.210409606443875 73.72385293422647) bank180793 +180794 POINT(41.31726537499274 74.00969897968828) bank180794 +180795 POINT(41.22013181312759 74.93366223501573) bank180795 +180796 POINT(39.795745702528166 73.53383313913801) bank180796 +180797 POINT(41.57270697859546 73.58421070984127) bank180797 +180798 POINT(40.778105801459354 74.69709492045568) bank180798 +180799 POINT(41.093807988487235 73.61751995164967) bank180799 +180800 POINT(41.27894987635938 73.61298843501876) bank180800 +180801 POINT(40.898800708896395 73.15767511227189) bank180801 +180802 POINT(40.363661071257184 74.30775326770727) bank180802 +180803 POINT(40.43663869382144 74.6180784067608) bank180803 +180804 POINT(40.17933645314426 73.24412796527444) bank180804 +180805 POINT(40.54885931831313 74.13178965191115) bank180805 +180806 POINT(40.984390679462166 73.11190189669125) bank180806 +180807 POINT(40.69263804420744 73.03092779476795) bank180807 +180808 POINT(40.313627527566474 74.62974513033899) bank180808 +180809 POINT(41.03979329639702 74.57618792257131) bank180809 +180810 POINT(40.173118735641744 74.57530722586517) bank180810 +180811 POINT(41.06354976068157 74.2030573182781) bank180811 +180812 POINT(39.74228665716144 73.94489825972619) bank180812 +180813 POINT(40.89002847946903 73.95955877289097) bank180813 +180814 POINT(40.323220244002314 74.72651897660334) bank180814 +180815 POINT(41.4645310635002 73.30480388594835) bank180815 +180816 POINT(41.15483088707235 73.56927313884202) bank180816 +180817 POINT(40.99511644483755 74.99163091154945) bank180817 +180818 POINT(41.51590464794746 73.76282396644027) bank180818 +180819 POINT(41.46772735152479 73.69822091612095) bank180819 +180820 POINT(39.94163666749568 74.5195150728627) bank180820 +180821 POINT(40.965457042211916 73.15518884605765) bank180821 +180822 POINT(39.7843708314505 74.23229431642449) bank180822 +180823 POINT(40.82197091062041 73.14895755789945) bank180823 +180824 POINT(40.486094428256756 73.06205404397149) bank180824 +180825 POINT(41.04850081304565 73.7023077920469) bank180825 +180826 POINT(41.30129368628913 74.05698651374865) bank180826 +180827 POINT(41.26029157122535 74.45393127363144) bank180827 +180828 POINT(41.64380611611686 73.98180583223258) bank180828 +180829 POINT(40.450941483068036 74.03980978220855) bank180829 +180830 POINT(40.65051673546766 74.5923855234483) bank180830 +180831 POINT(41.10389624190245 73.81842560634959) bank180831 +180832 POINT(41.4965543987503 73.43528778269625) bank180832 +180833 POINT(39.91085431140151 73.77082740283927) bank180833 +180834 POINT(39.902682982178725 73.76038481636304) bank180834 +180835 POINT(39.7897975321959 74.30096463136928) bank180835 +180836 POINT(40.491830992102976 74.70782142555842) bank180836 +180837 POINT(40.84941936277782 73.083540378154) bank180837 +180838 POINT(41.54144021627246 74.40184739451067) bank180838 +180839 POINT(40.449196152981365 74.10896250303293) bank180839 +180840 POINT(40.84842806826386 73.8585230967737) bank180840 +180841 POINT(41.56141488808546 73.09619943013385) bank180841 +180842 POINT(40.74768598748116 73.3343608704858) bank180842 +180843 POINT(41.39539811684542 74.35972267458489) bank180843 +180844 POINT(40.88960255178972 74.50971744167715) bank180844 +180845 POINT(40.38074719330722 73.73678812047922) bank180845 +180846 POINT(41.51179686284069 73.49719830717981) bank180846 +180847 POINT(40.27304785941682 74.48008536281066) bank180847 +180848 POINT(41.07457574714585 74.88724931680332) bank180848 +180849 POINT(41.19414745889731 74.93746192514703) bank180849 +180850 POINT(40.5570787451053 74.25331726107885) bank180850 +180851 POINT(40.3313032363619 73.92245631102249) bank180851 +180852 POINT(40.92784447666348 73.74532238351284) bank180852 +180853 POINT(40.09592927277379 74.28859331129463) bank180853 +180854 POINT(41.00791528966257 73.45424011352833) bank180854 +180855 POINT(41.027732209152425 73.49586296866691) bank180855 +180856 POINT(40.64907022616686 74.62818258060359) bank180856 +180857 POINT(40.298290418898986 73.51429857478055) bank180857 +180858 POINT(41.27477081653789 74.11325548135594) bank180858 +180859 POINT(40.725696772612444 73.94869036187582) bank180859 +180860 POINT(40.096110346527986 73.61564598829659) bank180860 +180861 POINT(40.50303296077828 74.75290257549342) bank180861 +180862 POINT(41.38302493721089 73.69831055266282) bank180862 +180863 POINT(40.291244484801965 74.84305072270006) bank180863 +180864 POINT(41.69544654673599 74.6764546123205) bank180864 +180865 POINT(41.54117288639638 74.35205629901778) bank180865 +180866 POINT(40.17184632614674 73.62064714553829) bank180866 +180867 POINT(39.954128033480046 73.39130266097548) bank180867 +180868 POINT(40.35342818198951 73.14528460070699) bank180868 +180869 POINT(40.03697913507854 73.77328267474552) bank180869 +180870 POINT(40.1376649923559 73.64017233943606) bank180870 +180871 POINT(40.82889602327789 74.16779515544111) bank180871 +180872 POINT(41.446516808236574 73.96926997036157) bank180872 +180873 POINT(40.07494454292523 73.87347887138391) bank180873 +180874 POINT(40.80933513086869 74.40423465159094) bank180874 +180875 POINT(40.95742739620182 74.62633317675805) bank180875 +180876 POINT(40.62762194239439 74.88543291440219) bank180876 +180877 POINT(41.235166452581836 74.0278415778969) bank180877 +180878 POINT(41.09706629237263 74.9398695782291) bank180878 +180879 POINT(41.37174667031243 73.09285210448445) bank180879 +180880 POINT(41.61097017883749 74.97025077074765) bank180880 +180881 POINT(39.928405236945764 74.16934350776678) bank180881 +180882 POINT(40.72890911317391 74.8242965422653) bank180882 +180883 POINT(40.36825026494969 73.57011194790476) bank180883 +180884 POINT(41.62017893222176 73.74151283636401) bank180884 +180885 POINT(41.28350523153494 73.63872422296002) bank180885 +180886 POINT(41.59076364024288 73.14393682544708) bank180886 +180887 POINT(40.05861437338291 73.20924817247409) bank180887 +180888 POINT(40.444105741550864 73.99196404886746) bank180888 +180889 POINT(40.467340239905575 74.07387571654202) bank180889 +180890 POINT(40.394074201929925 74.08868540624916) bank180890 +180891 POINT(41.64093481753642 74.861620894466) bank180891 +180892 POINT(40.611014120201254 73.82833728174273) bank180892 +180893 POINT(41.34863462490893 74.60400127922595) bank180893 +180894 POINT(40.328475156041854 73.76562113052253) bank180894 +180895 POINT(41.61056543838149 74.57720970385412) bank180895 +180896 POINT(40.6848107286842 74.57808233646546) bank180896 +180897 POINT(40.89791873328377 73.75193583736747) bank180897 +180898 POINT(41.306526535307135 74.46927523993443) bank180898 +180899 POINT(39.81028242328827 73.39717831302521) bank180899 +180900 POINT(40.21385976271674 74.4120851359938) bank180900 +180901 POINT(40.02848784317036 73.6484423700621) bank180901 +180902 POINT(40.29015447774671 74.68456516960093) bank180902 +180903 POINT(40.23007432637242 73.49618054620043) bank180903 +180904 POINT(39.96416197985983 74.62263128590745) bank180904 +180905 POINT(39.715568806612204 74.83225098927322) bank180905 +180906 POINT(40.50007837695016 74.37876369864011) bank180906 +180907 POINT(40.619994869917306 74.06983158648796) bank180907 +180908 POINT(40.70687961212686 73.20118912772031) bank180908 +180909 POINT(40.390712781122325 73.09364436852309) bank180909 +180910 POINT(41.51990167169361 74.61721628416292) bank180910 +180911 POINT(40.86603511966933 73.4067736013466) bank180911 +180912 POINT(40.097025841879514 74.91897723525564) bank180912 +180913 POINT(41.38922800284715 73.53344724775707) bank180913 +180914 POINT(39.812665828886985 73.7037283256165) bank180914 +180915 POINT(39.734521259434224 74.06988349359467) bank180915 +180916 POINT(40.31701802143601 73.67781592307348) bank180916 +180917 POINT(40.4773036346899 73.03463110085083) bank180917 +180918 POINT(40.1397518208147 74.6728331890683) bank180918 +180919 POINT(41.26360246180707 74.05201144675401) bank180919 +180920 POINT(40.27357392759359 74.80671934444666) bank180920 +180921 POINT(39.793272757828106 73.69452006094143) bank180921 +180922 POINT(41.27277837545513 74.57750770013234) bank180922 +180923 POINT(41.57395794450655 74.85525710428304) bank180923 +180924 POINT(41.07918263970774 73.25438443383732) bank180924 +180925 POINT(41.22587416934414 73.38566589049567) bank180925 +180926 POINT(40.796835157097874 74.41045182204405) bank180926 +180927 POINT(41.11772682632187 74.10395334740714) bank180927 +180928 POINT(39.94082635387713 74.52580003081769) bank180928 +180929 POINT(41.61740490873669 74.90008078652143) bank180929 +180930 POINT(40.90745062151779 74.56062059246271) bank180930 +180931 POINT(41.15202694161341 73.03041079451508) bank180931 +180932 POINT(40.54302006222963 74.10374966242522) bank180932 +180933 POINT(39.98457887739462 73.6989793420027) bank180933 +180934 POINT(40.60596698002155 74.8261183396455) bank180934 +180935 POINT(41.087120838786326 74.66790751777523) bank180935 +180936 POINT(41.20922466689747 73.66939083773019) bank180936 +180937 POINT(40.90231272337452 74.6090458822255) bank180937 +180938 POINT(40.75060186198582 73.01850054634369) bank180938 +180939 POINT(40.85071829516134 73.35042444489441) bank180939 +180940 POINT(40.771271399177195 74.86788367075586) bank180940 +180941 POINT(40.26280883289193 73.67985529442005) bank180941 +180942 POINT(40.054098859764615 74.06596474549195) bank180942 +180943 POINT(39.7803506614573 73.79446678491722) bank180943 +180944 POINT(41.28629284467184 73.02683286307935) bank180944 +180945 POINT(41.13592308666394 73.92110249512771) bank180945 +180946 POINT(40.293317379524254 73.45646969657781) bank180946 +180947 POINT(40.354919767704104 74.24058242634281) bank180947 +180948 POINT(41.61805872262348 74.46669856119323) bank180948 +180949 POINT(41.62090315925748 73.95800241512802) bank180949 +180950 POINT(40.48401030960649 73.37371081307028) bank180950 +180951 POINT(40.880558300048904 73.51278926852399) bank180951 +180952 POINT(40.09224533821021 74.89479529383807) bank180952 +180953 POINT(40.60249285703835 73.11803409442287) bank180953 +180954 POINT(41.404521976601195 74.39763939345266) bank180954 +180955 POINT(40.959033881382204 74.6113831667459) bank180955 +180956 POINT(40.66163162411444 74.01709086123138) bank180956 +180957 POINT(39.94153286285629 74.20699189702592) bank180957 +180958 POINT(40.03785981745009 73.73139331636169) bank180958 +180959 POINT(40.31948173596983 73.63991067378957) bank180959 +180960 POINT(41.18645987011863 73.18181533536486) bank180960 +180961 POINT(41.628119987621886 73.52930577836109) bank180961 +180962 POINT(40.934201503135746 74.73015508143195) bank180962 +180963 POINT(40.81931058776116 73.55625732648134) bank180963 +180964 POINT(40.58526699285107 74.68886741582178) bank180964 +180965 POINT(40.494174206408125 74.96367101174722) bank180965 +180966 POINT(39.72675400005691 74.67429281496624) bank180966 +180967 POINT(40.919456912084996 74.0175163275229) bank180967 +180968 POINT(41.375235232805395 74.26859199361571) bank180968 +180969 POINT(41.511662136290724 74.024392079639) bank180969 +180970 POINT(40.86581909593988 74.65505271101371) bank180970 +180971 POINT(40.12945436523994 73.56483723821344) bank180971 +180972 POINT(40.40225773780086 73.62444668788004) bank180972 +180973 POINT(41.23723205992097 73.23565260678434) bank180973 +180974 POINT(40.67724630889766 74.37596244858089) bank180974 +180975 POINT(41.25257614336323 74.336899915213) bank180975 +180976 POINT(40.22802325194352 74.25849516473838) bank180976 +180977 POINT(39.90735196909407 74.11743809971075) bank180977 +180978 POINT(41.64674105773941 73.31507340171053) bank180978 +180979 POINT(41.654492735931825 74.90683017992161) bank180979 +180980 POINT(40.63320982970271 74.49593281144082) bank180980 +180981 POINT(41.14522599617978 74.62420618334916) bank180981 +180982 POINT(41.6307404564719 74.07665681205954) bank180982 +180983 POINT(40.7994712901151 74.20238634301208) bank180983 +180984 POINT(40.20977778520054 74.49438743902462) bank180984 +180985 POINT(41.56580032066973 73.7562448888559) bank180985 +180986 POINT(40.81424689716354 74.76143591973923) bank180986 +180987 POINT(40.35369918293505 73.48195532220176) bank180987 +180988 POINT(40.265288651057936 74.88152740676071) bank180988 +180989 POINT(41.43417756442603 74.1645572771055) bank180989 +180990 POINT(41.04686310839777 73.19972004380926) bank180990 +180991 POINT(40.856470734101386 74.79317684513973) bank180991 +180992 POINT(40.54213082244722 74.36021884598902) bank180992 +180993 POINT(39.74353576258978 74.74390841425334) bank180993 +180994 POINT(41.33771842633991 73.65532379014272) bank180994 +180995 POINT(40.225272096946526 73.70659979275803) bank180995 +180996 POINT(40.49597330679006 73.57529637438073) bank180996 +180997 POINT(41.19305700528332 74.04346894475982) bank180997 +180998 POINT(40.16796992428066 73.49282821938627) bank180998 +180999 POINT(41.15789349124748 74.0901587380128) bank180999 +181000 POINT(40.057719317282256 74.84016203144641) bank181000 +181001 POINT(40.24929951504478 73.26933279174382) bank181001 +181002 POINT(41.305257520793106 73.06398666306498) bank181002 +181003 POINT(41.49663367986357 73.22764846956251) bank181003 +181004 POINT(41.621612894576856 73.2436063840734) bank181004 +181005 POINT(41.01717525537803 74.20413816080223) bank181005 +181006 POINT(41.009684759330874 74.29973574445239) bank181006 +181007 POINT(39.9300755056387 73.99691878957206) bank181007 +181008 POINT(40.447404125122056 73.55525088327576) bank181008 +181009 POINT(40.90343393934181 73.96680939544846) bank181009 +181010 POINT(41.33208954384276 74.7190669991856) bank181010 +181011 POINT(41.25603532087428 73.42768334500627) bank181011 +181012 POINT(41.680596775109244 74.15051085015517) bank181012 +181013 POINT(39.8673421159851 74.00537752208483) bank181013 +181014 POINT(40.42151647031619 73.74540565311052) bank181014 +181015 POINT(40.19209457056947 73.57077573092965) bank181015 +181016 POINT(40.960537281373085 73.56874622501553) bank181016 +181017 POINT(40.87658267578809 73.196980628333) bank181017 +181018 POINT(40.84790337659472 74.6905618182965) bank181018 +181019 POINT(40.63803345590577 73.59853121207459) bank181019 +181020 POINT(41.411658400887944 73.49290627249094) bank181020 +181021 POINT(40.84857285538791 74.38309416505261) bank181021 +181022 POINT(41.05483085649376 73.11291294390941) bank181022 +181023 POINT(40.937781147210366 74.7935446334032) bank181023 +181024 POINT(39.91214382567095 73.74127909295778) bank181024 +181025 POINT(41.44643923827292 73.1627601335157) bank181025 +181026 POINT(40.919289235204516 73.55760845153127) bank181026 +181027 POINT(40.59160119282243 73.9911579683501) bank181027 +181028 POINT(39.82987766984385 74.03746188984698) bank181028 +181029 POINT(41.3440433113155 73.01304896032471) bank181029 +181030 POINT(40.03702527907766 74.93624325182249) bank181030 +181031 POINT(41.634650929545074 73.54359185803179) bank181031 +181032 POINT(41.11655741135723 73.2516356637187) bank181032 +181033 POINT(40.19323305521055 73.87740880354936) bank181033 +181034 POINT(40.267886891854225 74.25162172376075) bank181034 +181035 POINT(41.68824643185469 74.45064998405086) bank181035 +181036 POINT(41.18263059843898 74.38508759676705) bank181036 +181037 POINT(41.55220359539012 73.33884207223099) bank181037 +181038 POINT(40.05683923246883 73.06258136071573) bank181038 +181039 POINT(40.07368588468679 74.65754538646877) bank181039 +181040 POINT(40.26215913588177 74.63434621954208) bank181040 +181041 POINT(41.12963442258916 73.36977714469138) bank181041 +181042 POINT(40.38693517868353 73.15985757389568) bank181042 +181043 POINT(41.46429359391199 74.32337153866891) bank181043 +181044 POINT(41.59340163271049 74.52476681943203) bank181044 +181045 POINT(41.64635133268902 74.24917051592803) bank181045 +181046 POINT(41.18101640093587 73.81264573036874) bank181046 +181047 POINT(39.902253674931245 73.34996961103091) bank181047 +181048 POINT(41.52330570339032 74.55897739103952) bank181048 +181049 POINT(40.56199204970859 73.04585570508094) bank181049 +181050 POINT(40.8881486810078 74.6145676485835) bank181050 +181051 POINT(39.882618407072556 73.54326281182809) bank181051 +181052 POINT(40.35469569531838 74.31282834925628) bank181052 +181053 POINT(40.29086826311185 74.73273632123879) bank181053 +181054 POINT(41.29740574124146 73.21886520501845) bank181054 +181055 POINT(40.835203915391666 73.17067971085267) bank181055 +181056 POINT(40.426104880245404 74.61930544967066) bank181056 +181057 POINT(40.778857995689734 74.9784241070631) bank181057 +181058 POINT(39.87073964946181 74.53295032126736) bank181058 +181059 POINT(40.413414013902965 73.5464484629233) bank181059 +181060 POINT(41.31045077010626 74.66600088479434) bank181060 +181061 POINT(41.54401064014536 73.76774032181777) bank181061 +181062 POINT(41.27179806110697 74.14372047457648) bank181062 +181063 POINT(39.72979596800113 74.50503129966219) bank181063 +181064 POINT(41.367170935853 73.54723591744204) bank181064 +181065 POINT(39.99240582905968 74.77621203523898) bank181065 +181066 POINT(40.27442239290661 73.34885902670804) bank181066 +181067 POINT(41.60203540905432 73.15585075213981) bank181067 +181068 POINT(39.88447436449723 73.78169080825303) bank181068 +181069 POINT(40.89654661808891 73.68481682281772) bank181069 +181070 POINT(39.98639386661772 74.72784227912437) bank181070 +181071 POINT(41.31292446638103 74.54390845498693) bank181071 +181072 POINT(40.21652641438368 74.40396908137437) bank181072 +181073 POINT(40.782460581554325 74.83933498263296) bank181073 +181074 POINT(40.61215977507505 73.49595306736484) bank181074 +181075 POINT(41.13767749716033 73.9734867362369) bank181075 +181076 POINT(40.297068184075776 74.41844810368669) bank181076 +181077 POINT(41.02902258698493 73.85813819452306) bank181077 +181078 POINT(39.79025277666532 74.53121916565664) bank181078 +181079 POINT(41.419742075155696 74.87202803154905) bank181079 +181080 POINT(40.013746567695534 73.00883977563208) bank181080 +181081 POINT(41.62725782143157 73.0068245536252) bank181081 +181082 POINT(40.0977205828425 74.97926692588273) bank181082 +181083 POINT(40.693684765864695 73.24910359741334) bank181083 +181084 POINT(40.28042612264447 73.3928236511812) bank181084 +181085 POINT(40.0625282842443 74.54861277885593) bank181085 +181086 POINT(40.49909846801904 74.64360326193133) bank181086 +181087 POINT(41.03533742818329 73.25785246932742) bank181087 +181088 POINT(41.64761313602872 74.25500341620597) bank181088 +181089 POINT(39.78436119034837 73.55539620427332) bank181089 +181090 POINT(40.660135501906815 73.97410988884008) bank181090 +181091 POINT(41.54037921633953 74.78501985845259) bank181091 +181092 POINT(40.00421629434968 74.02312622192366) bank181092 +181093 POINT(40.675842929581535 74.3271429826151) bank181093 +181094 POINT(41.15989030370848 74.05185448052946) bank181094 +181095 POINT(40.375338509967186 74.96032775695879) bank181095 +181096 POINT(41.28769337716115 74.61064504346545) bank181096 +181097 POINT(39.85441469008421 73.71073551393727) bank181097 +181098 POINT(40.320293462733794 74.02948207180198) bank181098 +181099 POINT(41.66909251119041 73.63492701780747) bank181099 +181100 POINT(39.88690778093808 73.93208730495137) bank181100 +181101 POINT(40.746947898587074 73.97444110120567) bank181101 +181102 POINT(41.05524797183416 74.83108424256086) bank181102 +181103 POINT(41.25124557692695 73.49687252008253) bank181103 +181104 POINT(41.46200757842067 73.49458220382297) bank181104 +181105 POINT(40.470794662120795 73.82340641867147) bank181105 +181106 POINT(39.866211044173596 74.62580750392266) bank181106 +181107 POINT(41.37747812956866 74.91644413997554) bank181107 +181108 POINT(40.78341711296788 73.57707483987895) bank181108 +181109 POINT(41.22622075238656 74.66708278441789) bank181109 +181110 POINT(40.22274327264117 73.37134127928935) bank181110 +181111 POINT(40.886928580493546 73.02216368721655) bank181111 +181112 POINT(40.91945740585998 73.95418451454293) bank181112 +181113 POINT(40.18064556660076 73.5663134664753) bank181113 +181114 POINT(41.2208610113596 73.9280392185865) bank181114 +181115 POINT(41.11770343071315 73.87222069125438) bank181115 +181116 POINT(39.98268760375757 74.70973132733855) bank181116 +181117 POINT(41.292771140486046 73.58875769587839) bank181117 +181118 POINT(40.89241155211582 74.63711229776132) bank181118 +181119 POINT(41.38988443260046 74.21751510460336) bank181119 +181120 POINT(41.09230267418121 73.10645117133686) bank181120 +181121 POINT(41.044410524367926 74.96281235038697) bank181121 +181122 POINT(41.448798520782326 73.74230598610619) bank181122 +181123 POINT(41.11560146280567 74.74279181596438) bank181123 +181124 POINT(40.726499037373635 73.32887772208878) bank181124 +181125 POINT(41.2974127777059 73.62358189802222) bank181125 +181126 POINT(41.033211484527186 74.93922051892865) bank181126 +181127 POINT(41.41981313828086 73.74141819935983) bank181127 +181128 POINT(39.888187719142195 74.60073680054467) bank181128 +181129 POINT(39.91875101110986 74.06522710341541) bank181129 +181130 POINT(40.76770417363192 73.51203328427019) bank181130 +181131 POINT(41.120295427103684 73.34946074838581) bank181131 +181132 POINT(40.06384361551351 74.80459677797568) bank181132 +181133 POINT(39.765731145724345 73.19434693487393) bank181133 +181134 POINT(41.56418204716369 73.75463141448691) bank181134 +181135 POINT(40.02251334551101 73.48308932567635) bank181135 +181136 POINT(40.802866470286126 74.40166676188932) bank181136 +181137 POINT(41.60840369611389 73.62834948323015) bank181137 +181138 POINT(39.753956346187174 74.19116289506607) bank181138 +181139 POINT(41.184171315408015 73.4214844579402) bank181139 +181140 POINT(39.909952036868596 73.43032682300354) bank181140 +181141 POINT(40.49937884225629 73.13466888814717) bank181141 +181142 POINT(39.841726180390644 73.71732557481955) bank181142 +181143 POINT(41.610031229936894 73.16742744865905) bank181143 +181144 POINT(41.50222975370836 74.61769458688049) bank181144 +181145 POINT(40.22837891908991 74.01295053344141) bank181145 +181146 POINT(41.37657131988589 73.5473608885217) bank181146 +181147 POINT(40.41001554262743 73.5222751498222) bank181147 +181148 POINT(40.19775153528629 73.03632484019337) bank181148 +181149 POINT(39.87942919322596 74.25388203648912) bank181149 +181150 POINT(40.5381057644776 73.2906588966741) bank181150 +181151 POINT(40.39560632665109 73.4198080220394) bank181151 +181152 POINT(39.88211217547701 74.41802251025942) bank181152 +181153 POINT(41.42620665488755 74.17585454857965) bank181153 +181154 POINT(40.094587119705494 73.82145188654312) bank181154 +181155 POINT(40.45320469289129 73.13133921737129) bank181155 +181156 POINT(40.524076867342814 73.99357493010491) bank181156 +181157 POINT(40.13216616979957 73.72928310206864) bank181157 +181158 POINT(39.75755895648234 73.61833406528554) bank181158 +181159 POINT(40.70859317554922 74.28148835014368) bank181159 +181160 POINT(40.5210902183961 74.64087507232155) bank181160 +181161 POINT(39.94419068176507 73.54604891378284) bank181161 +181162 POINT(40.05441621816525 74.86115563532306) bank181162 +181163 POINT(41.569185035765656 73.79360092282424) bank181163 +181164 POINT(40.042272768060876 74.58278036426958) bank181164 +181165 POINT(40.294063606429596 74.35870966554995) bank181165 +181166 POINT(41.49786756727206 73.68641284842158) bank181166 +181167 POINT(41.06975968514619 73.73981172123818) bank181167 +181168 POINT(40.61255741042371 73.428601800831) bank181168 +181169 POINT(41.467254685373604 73.85682116168024) bank181169 +181170 POINT(40.47305751887269 74.63120492727425) bank181170 +181171 POINT(40.32252816878987 74.02003440165072) bank181171 +181172 POINT(40.78143928061651 74.53179800921052) bank181172 +181173 POINT(39.96586317664541 73.21649880201478) bank181173 +181174 POINT(39.76525177167725 74.11662751466646) bank181174 +181175 POINT(40.05897254909781 74.73196379369722) bank181175 +181176 POINT(40.23565556883947 74.25958075301921) bank181176 +181177 POINT(40.33800749027629 74.59209741788719) bank181177 +181178 POINT(41.03591904227078 73.08965498036106) bank181178 +181179 POINT(40.434216062124136 73.10724875848767) bank181179 +181180 POINT(39.86879278053845 74.45680697907315) bank181180 +181181 POINT(40.24006052491825 73.78736607865184) bank181181 +181182 POINT(40.14552089560707 74.90193554383494) bank181182 +181183 POINT(40.96167017963842 73.8495618028468) bank181183 +181184 POINT(41.4076350328229 74.68162900191267) bank181184 +181185 POINT(40.81686646281013 73.60299578320729) bank181185 +181186 POINT(41.10556773121769 74.7907827411006) bank181186 +181187 POINT(41.59741870435793 74.75283032920821) bank181187 +181188 POINT(40.354107018585836 74.3690605274693) bank181188 +181189 POINT(41.477847764272134 74.1941350044674) bank181189 +181190 POINT(41.65815148517371 73.61516298268207) bank181190 +181191 POINT(40.598783217152175 73.04825042314249) bank181191 +181192 POINT(41.34765907492505 74.14096243120586) bank181192 +181193 POINT(40.33568347258113 73.12210777815689) bank181193 +181194 POINT(41.13200575979196 73.16843366374923) bank181194 +181195 POINT(39.98808889396504 73.574455400532) bank181195 +181196 POINT(40.00225821229202 74.29497513890128) bank181196 +181197 POINT(41.2036869250979 74.63242858826491) bank181197 +181198 POINT(40.136086704714224 73.02096338238839) bank181198 +181199 POINT(39.964734273021975 73.75196887829858) bank181199 +181200 POINT(39.854524240912376 74.33373718643443) bank181200 +181201 POINT(40.27211044006104 74.93087647586117) bank181201 +181202 POINT(41.61716657429779 74.97449925648182) bank181202 +181203 POINT(40.31212575728968 73.55692461425066) bank181203 +181204 POINT(41.270797350091634 73.21929134125013) bank181204 +181205 POINT(41.0872411200106 73.42422648338852) bank181205 +181206 POINT(41.70491665441596 74.80628363628763) bank181206 +181207 POINT(41.49128487815053 74.70643635788687) bank181207 +181208 POINT(40.21465562135178 73.13323231522322) bank181208 +181209 POINT(40.49749568565775 74.45304500351762) bank181209 +181210 POINT(40.14454854907364 73.43000923847174) bank181210 +181211 POINT(40.17324483330221 73.91281914416156) bank181211 +181212 POINT(40.48303966943768 74.76382203923076) bank181212 +181213 POINT(41.620062440270544 73.64762731481095) bank181213 +181214 POINT(41.30418962340839 73.94555232371027) bank181214 +181215 POINT(39.75468492618893 73.44380777158841) bank181215 +181216 POINT(40.10064031607243 73.96585604243414) bank181216 +181217 POINT(41.088429517925945 73.45771672568732) bank181217 +181218 POINT(41.52253324881268 74.0417197565201) bank181218 +181219 POINT(41.2555708513941 73.60377789279423) bank181219 +181220 POINT(40.105431802026075 74.80645397177025) bank181220 +181221 POINT(40.780809824638936 74.44519356629978) bank181221 +181222 POINT(40.945872797284736 73.83934937285858) bank181222 +181223 POINT(41.00870545892336 73.62029099134651) bank181223 +181224 POINT(41.69699172775554 73.27852910289386) bank181224 +181225 POINT(40.294310676339364 74.71554533825697) bank181225 +181226 POINT(41.53085337461992 74.89068394798046) bank181226 +181227 POINT(40.161182550758504 73.40944469506525) bank181227 +181228 POINT(41.386087318063005 74.74890106908573) bank181228 +181229 POINT(41.413136542473204 74.689113781586) bank181229 +181230 POINT(41.00518870554821 73.75553868668469) bank181230 +181231 POINT(41.08892453693836 74.87431249661626) bank181231 +181232 POINT(40.26452339725552 73.24898728990593) bank181232 +181233 POINT(41.21600901701433 74.44354013193458) bank181233 +181234 POINT(41.41777607191133 73.34515280086427) bank181234 +181235 POINT(41.30180595226824 74.99046066665966) bank181235 +181236 POINT(39.961788347227255 73.68031003695553) bank181236 +181237 POINT(39.874298086663586 73.71371564185254) bank181237 +181238 POINT(41.63921365683729 73.5096725761148) bank181238 +181239 POINT(40.02397620487324 74.68438352571617) bank181239 +181240 POINT(40.766408512798705 73.37026924251522) bank181240 +181241 POINT(40.58369413894496 74.39435650241292) bank181241 +181242 POINT(41.5170902198596 74.3666873424725) bank181242 +181243 POINT(41.33942291603801 74.50014609010371) bank181243 +181244 POINT(40.507978045986874 73.77912530397008) bank181244 +181245 POINT(40.20480110475643 74.3495526479186) bank181245 +181246 POINT(41.514751157237995 74.39533214808891) bank181246 +181247 POINT(40.94110585326108 73.48455476354123) bank181247 +181248 POINT(40.83193913568862 73.55021259909786) bank181248 +181249 POINT(40.84897250134072 73.46827491003243) bank181249 +181250 POINT(40.0738573422406 74.1762242741536) bank181250 +181251 POINT(40.30627179557707 74.74410919207719) bank181251 +181252 POINT(40.708429413221225 74.93016588024777) bank181252 +181253 POINT(41.01468241162025 74.24645038747245) bank181253 +181254 POINT(40.94439068285582 73.19811381997431) bank181254 +181255 POINT(39.887901266903505 74.52299953149772) bank181255 +181256 POINT(41.001701243535784 74.62288601414073) bank181256 +181257 POINT(39.98225195910079 73.99381311869324) bank181257 +181258 POINT(41.65399007685764 73.25682501179304) bank181258 +181259 POINT(40.329882938806186 73.95867957174289) bank181259 +181260 POINT(39.95012869457944 74.36385022573593) bank181260 +181261 POINT(40.066951230043344 74.58556921282958) bank181261 +181262 POINT(41.04410158678656 74.9438655189028) bank181262 +181263 POINT(40.761988860550574 74.23317365493125) bank181263 +181264 POINT(41.37955871914242 74.03972428767506) bank181264 +181265 POINT(40.49676402248623 73.97136317749924) bank181265 +181266 POINT(41.469718797995625 74.47955710768716) bank181266 +181267 POINT(41.402611818804395 73.93499133579039) bank181267 +181268 POINT(40.819452569965655 74.22321553018337) bank181268 +181269 POINT(39.95664393643275 74.21498336611687) bank181269 +181270 POINT(40.56730677258805 73.97732869888543) bank181270 +181271 POINT(40.944157075092676 73.93595790967457) bank181271 +181272 POINT(40.71361792622536 73.49053300120147) bank181272 +181273 POINT(40.17057010018174 73.34703265475169) bank181273 +181274 POINT(40.548470334522875 73.02689530193733) bank181274 +181275 POINT(40.20750371114663 73.84413238061342) bank181275 +181276 POINT(40.442782831536114 73.55748255760808) bank181276 +181277 POINT(39.93830768473265 73.04854102647475) bank181277 +181278 POINT(40.88864527309157 74.80963621182444) bank181278 +181279 POINT(40.16912739121008 73.2345232941237) bank181279 +181280 POINT(41.06434889263301 74.38544290028915) bank181280 +181281 POINT(40.81161055120374 73.28168362836142) bank181281 +181282 POINT(40.467225320307 74.45210149519333) bank181282 +181283 POINT(40.39104817835862 74.58265789217988) bank181283 +181284 POINT(39.86545126663262 74.1303763852701) bank181284 +181285 POINT(40.241448642977794 73.79153572113559) bank181285 +181286 POINT(39.81080903807709 74.5276876776116) bank181286 +181287 POINT(41.50156079192145 74.00756171418926) bank181287 +181288 POINT(41.62416051809027 73.0999399255123) bank181288 +181289 POINT(40.14213540153986 73.45662860851508) bank181289 +181290 POINT(40.27998013227915 74.34720631606854) bank181290 +181291 POINT(40.93326388833408 73.97877768342092) bank181291 +181292 POINT(40.34910607564978 74.42965559559863) bank181292 +181293 POINT(40.811138592917956 74.06308911810808) bank181293 +181294 POINT(40.50714718705451 74.20923120546608) bank181294 +181295 POINT(40.01665130942236 74.04987297014547) bank181295 +181296 POINT(39.9004270683146 73.52025548493144) bank181296 +181297 POINT(40.36894663577838 73.88713952481135) bank181297 +181298 POINT(41.17935358517851 73.26801985162251) bank181298 +181299 POINT(40.26403216297854 73.67148806899353) bank181299 +181300 POINT(40.973633866143544 74.54866107330591) bank181300 +181301 POINT(41.223908893605454 74.02564748815291) bank181301 +181302 POINT(41.193801668789696 73.01389051379667) bank181302 +181303 POINT(41.59519537292035 74.34562788226151) bank181303 +181304 POINT(41.1355274604755 74.96592293067583) bank181304 +181305 POINT(40.12198024436693 73.19790666057418) bank181305 +181306 POINT(40.993461812856815 74.77107200363898) bank181306 +181307 POINT(40.195988210494725 74.81807283550933) bank181307 +181308 POINT(41.06989830657198 73.62590305524877) bank181308 +181309 POINT(40.12069079561247 74.88444352446517) bank181309 +181310 POINT(40.642678384704524 74.57959555751971) bank181310 +181311 POINT(39.75309190346183 73.46224069640364) bank181311 +181312 POINT(41.42297816396422 73.08845475892062) bank181312 +181313 POINT(40.363046291436326 74.51411809060288) bank181313 +181314 POINT(40.393888172978755 74.46766041560683) bank181314 +181315 POINT(41.22061976818553 73.88047455784846) bank181315 +181316 POINT(41.54444617831916 73.7093930051866) bank181316 +181317 POINT(40.827711655104714 74.92479750478829) bank181317 +181318 POINT(41.08902181827608 74.03610890689339) bank181318 +181319 POINT(41.51706969650971 74.01030461472445) bank181319 +181320 POINT(41.638047721319914 74.32005754681964) bank181320 +181321 POINT(41.293839468376284 74.0311554821138) bank181321 +181322 POINT(41.12128996584316 74.96605189650924) bank181322 +181323 POINT(39.79800078968404 73.13748473329478) bank181323 +181324 POINT(41.492471850472214 73.7684893453023) bank181324 +181325 POINT(39.720668639561225 73.56816222798882) bank181325 +181326 POINT(41.62401424496439 73.13654520907663) bank181326 +181327 POINT(39.989554634955525 73.96983558909051) bank181327 +181328 POINT(41.21219561729001 73.57450510341101) bank181328 +181329 POINT(40.64921024533089 73.08365996570349) bank181329 +181330 POINT(41.193215646663695 73.33781254828732) bank181330 +181331 POINT(41.02355873238943 73.09278973903969) bank181331 +181332 POINT(39.851858693925514 74.98359301263241) bank181332 +181333 POINT(40.11231613422495 74.83637581106387) bank181333 +181334 POINT(40.30693078792147 73.17520405944191) bank181334 +181335 POINT(40.23301907082352 73.46859723545592) bank181335 +181336 POINT(41.64205941748235 74.15565836811612) bank181336 +181337 POINT(41.18365149928254 73.91230488691605) bank181337 +181338 POINT(40.110708471694224 74.75589027002489) bank181338 +181339 POINT(40.85126041972062 74.96595073601533) bank181339 +181340 POINT(41.246634735623246 73.65620446466417) bank181340 +181341 POINT(40.31296304237836 73.29379481415765) bank181341 +181342 POINT(40.75163133640218 73.83575272381385) bank181342 +181343 POINT(41.084068133677405 74.22486977788276) bank181343 +181344 POINT(41.0798160712545 73.20537193130052) bank181344 +181345 POINT(41.04342965012789 73.91970732342816) bank181345 +181346 POINT(39.95742938764592 73.07940262531598) bank181346 +181347 POINT(40.58991979152436 73.69036739068082) bank181347 +181348 POINT(40.78000199252618 73.47156135139738) bank181348 +181349 POINT(41.51467125173559 74.97104719805907) bank181349 +181350 POINT(41.17836085390792 74.63672685828213) bank181350 +181351 POINT(40.10875207445755 74.3952045181572) bank181351 +181352 POINT(40.52947236584505 74.82318581105507) bank181352 +181353 POINT(40.3317477486794 73.7511722596195) bank181353 +181354 POINT(39.81066701213249 73.84600629606044) bank181354 +181355 POINT(41.61836592960109 74.51104389764538) bank181355 +181356 POINT(39.947209017819866 74.63798206758834) bank181356 +181357 POINT(40.96369914745552 73.35600738456438) bank181357 +181358 POINT(41.444982830198576 73.20814734849615) bank181358 +181359 POINT(41.05403424868009 73.08324867706627) bank181359 +181360 POINT(41.688744683693166 73.33627028953553) bank181360 +181361 POINT(40.65251010782697 74.48029216511523) bank181361 +181362 POINT(41.46988975974897 74.55878622859991) bank181362 +181363 POINT(41.64803867329575 74.35911828118199) bank181363 +181364 POINT(39.965466446541186 74.31300943397885) bank181364 +181365 POINT(40.60753251526302 73.54615556984814) bank181365 +181366 POINT(41.39661617575301 74.5327843507865) bank181366 +181367 POINT(40.12366431318232 74.58514613913292) bank181367 +181368 POINT(41.43527967664589 73.94934651690474) bank181368 +181369 POINT(39.8834995984204 73.09793547611434) bank181369 +181370 POINT(41.67137386664704 73.59041494350375) bank181370 +181371 POINT(40.72768081216895 74.27097716171352) bank181371 +181372 POINT(41.366032081401556 73.61542258193498) bank181372 +181373 POINT(40.10783583491956 73.7212255064526) bank181373 +181374 POINT(40.74259795068697 74.20520697639694) bank181374 +181375 POINT(40.694211526618076 74.29951199993306) bank181375 +181376 POINT(40.06461033795398 74.81066723857796) bank181376 +181377 POINT(40.35988554815171 73.0390246826455) bank181377 +181378 POINT(40.89772091732816 73.65086511786681) bank181378 +181379 POINT(40.02275960697099 73.70186449555173) bank181379 +181380 POINT(41.32626142654407 74.91588172594747) bank181380 +181381 POINT(40.1940980755223 74.16923209595484) bank181381 +181382 POINT(41.02748903336071 74.00169438256356) bank181382 +181383 POINT(41.552327234818705 74.98173309298728) bank181383 +181384 POINT(40.95422903120089 74.52685014877763) bank181384 +181385 POINT(41.2775976195462 73.96786752327831) bank181385 +181386 POINT(40.086483075042906 74.87518766702719) bank181386 +181387 POINT(41.49440689567438 73.51202663597974) bank181387 +181388 POINT(40.787081701307834 74.2132121609332) bank181388 +181389 POINT(40.24125889805975 73.29554300967334) bank181389 +181390 POINT(40.501790883533864 74.08479914220874) bank181390 +181391 POINT(40.213821975031195 74.01164039824741) bank181391 +181392 POINT(40.34622672633921 74.03533341979444) bank181392 +181393 POINT(41.28606126338889 73.1357603645663) bank181393 +181394 POINT(39.954621144757155 74.9910000392821) bank181394 +181395 POINT(41.118210653196385 74.5546545701804) bank181395 +181396 POINT(40.73089728451796 74.59863212558245) bank181396 +181397 POINT(40.97224144743221 74.60983395663955) bank181397 +181398 POINT(39.9676088189577 73.64155704180887) bank181398 +181399 POINT(41.27133012002255 74.49485715191615) bank181399 +181400 POINT(40.98438981047562 73.26257221334811) bank181400 +181401 POINT(41.1952845080516 73.2721913030481) bank181401 +181402 POINT(40.914514995205636 74.6090352668315) bank181402 +181403 POINT(40.62117132126048 73.56851658775241) bank181403 +181404 POINT(40.87968397214972 73.16870919375677) bank181404 +181405 POINT(40.491209485575375 73.94283302189454) bank181405 +181406 POINT(40.40461960892369 74.60845010602739) bank181406 +181407 POINT(41.17606419592291 74.89593124171967) bank181407 +181408 POINT(41.628776113731675 73.09264937126181) bank181408 +181409 POINT(39.88065261724765 73.12451968014226) bank181409 +181410 POINT(41.295281322132226 74.82596300926018) bank181410 +181411 POINT(41.21897091530351 74.40078485326914) bank181411 +181412 POINT(41.17607808994468 74.56166779104521) bank181412 +181413 POINT(39.72353513747832 74.7568561059442) bank181413 +181414 POINT(40.30930645513792 74.99408958376702) bank181414 +181415 POINT(40.3522594841095 73.44427207049185) bank181415 +181416 POINT(41.2469729447592 73.06700552141166) bank181416 +181417 POINT(40.63360458048576 73.08556389668698) bank181417 +181418 POINT(40.12889535489999 74.85917147364597) bank181418 +181419 POINT(41.28738841388504 74.26038172074244) bank181419 +181420 POINT(40.36452993495814 74.01410272824023) bank181420 +181421 POINT(40.0725760489192 74.29420950256237) bank181421 +181422 POINT(40.6544214536599 73.1387046815085) bank181422 +181423 POINT(41.491925316996564 73.99788216157545) bank181423 +181424 POINT(40.23943066683275 73.63576156760733) bank181424 +181425 POINT(41.67188224925252 74.52309318588229) bank181425 +181426 POINT(41.277855935339765 73.20083647261333) bank181426 +181427 POINT(39.73101827634333 73.80943658551429) bank181427 +181428 POINT(40.28300960476339 74.12785385549958) bank181428 +181429 POINT(39.844619631327895 74.62402150420913) bank181429 +181430 POINT(41.07720117376945 74.74199274660992) bank181430 +181431 POINT(41.23106769861237 73.41894348105113) bank181431 +181432 POINT(40.0549886442952 74.0620858695011) bank181432 +181433 POINT(41.619796019277196 74.4183234186362) bank181433 +181434 POINT(40.49629926685358 73.67877747906284) bank181434 +181435 POINT(40.270056296171234 73.03857407168124) bank181435 +181436 POINT(39.815829318112165 73.88848427484506) bank181436 +181437 POINT(40.84806724554718 73.40402343028748) bank181437 +181438 POINT(39.872001511020265 74.12091261136698) bank181438 +181439 POINT(40.67126213840597 73.6360070912526) bank181439 +181440 POINT(41.63042557303751 74.90257528024252) bank181440 +181441 POINT(40.617580729364576 73.50942730778955) bank181441 +181442 POINT(41.18161329186713 74.89271936510946) bank181442 +181443 POINT(41.48598730952854 74.24564494975023) bank181443 +181444 POINT(41.16496811744251 73.91441835523625) bank181444 +181445 POINT(40.1104538164075 73.51658776751012) bank181445 +181446 POINT(40.15356454379229 74.46317857156643) bank181446 +181447 POINT(41.19582595918367 74.60790093402514) bank181447 +181448 POINT(41.59141995170494 73.8033488373236) bank181448 +181449 POINT(40.40677525732238 74.79290584769102) bank181449 +181450 POINT(40.07998565199923 74.35443264314843) bank181450 +181451 POINT(39.98403019772706 73.69509220316435) bank181451 +181452 POINT(40.59052407092794 75.00272685569557) bank181452 +181453 POINT(41.695107364241075 74.42073445577635) bank181453 +181454 POINT(40.14625207001222 74.1067549208974) bank181454 +181455 POINT(40.644105530571316 74.2972189868689) bank181455 +181456 POINT(41.635092051260735 73.17061207108875) bank181456 +181457 POINT(40.70539614132897 73.76401887268949) bank181457 +181458 POINT(39.79817647722686 74.4801688995461) bank181458 +181459 POINT(41.529248258004955 73.51639600098672) bank181459 +181460 POINT(40.90346864310222 74.19901947692313) bank181460 +181461 POINT(39.92316960243809 74.96705740599909) bank181461 +181462 POINT(40.659338304017034 74.66524802853068) bank181462 +181463 POINT(41.52080749817979 74.17219809932074) bank181463 +181464 POINT(41.099556801305226 73.01908854825982) bank181464 +181465 POINT(41.14494573412112 73.22353379943134) bank181465 +181466 POINT(41.30203929630819 73.4952052161729) bank181466 +181467 POINT(40.90462454817664 75.00596765502702) bank181467 +181468 POINT(40.163558634702575 74.09321550563581) bank181468 +181469 POINT(40.42236531083509 73.0576738741392) bank181469 +181470 POINT(40.76890616017679 74.78958305265346) bank181470 +181471 POINT(40.01363032345588 73.62627850464507) bank181471 +181472 POINT(40.83916554496339 73.65527315310592) bank181472 +181473 POINT(41.50253772101899 74.24817447705671) bank181473 +181474 POINT(41.55159897319553 73.7169187710354) bank181474 +181475 POINT(39.7577057136872 74.3006861003872) bank181475 +181476 POINT(41.43042168696674 74.8082469385508) bank181476 +181477 POINT(39.8861440024657 73.02673738256814) bank181477 +181478 POINT(41.0930945178231 74.3163647912458) bank181478 +181479 POINT(40.77348488476039 73.27257012787928) bank181479 +181480 POINT(41.31670536019637 73.9450521165752) bank181480 +181481 POINT(40.15585822364281 74.67064170025975) bank181481 +181482 POINT(40.10383544403977 73.06353647720452) bank181482 +181483 POINT(40.861901751428384 73.24886557656923) bank181483 +181484 POINT(41.37853833249702 74.23659269755167) bank181484 +181485 POINT(40.01990866097325 74.03010690143411) bank181485 +181486 POINT(41.52035546894042 74.30806692786874) bank181486 +181487 POINT(40.69349829900884 74.40986506938101) bank181487 +181488 POINT(41.231219192336106 74.86909729136275) bank181488 +181489 POINT(41.04230864310911 73.87868614600548) bank181489 +181490 POINT(39.895302286842366 74.89846597636961) bank181490 +181491 POINT(40.16217225300316 74.37871165361913) bank181491 +181492 POINT(39.751599510283256 73.07320040232703) bank181492 +181493 POINT(40.654433679453454 73.09064320710887) bank181493 +181494 POINT(41.38306584873717 74.8609867032852) bank181494 +181495 POINT(40.266109448108274 74.10092217548318) bank181495 +181496 POINT(40.57465228732228 74.21393798746362) bank181496 +181497 POINT(41.37659883562995 74.76506881231717) bank181497 +181498 POINT(41.23477349036713 74.17333811342789) bank181498 +181499 POINT(40.337621928578656 74.51104916867544) bank181499 +181500 POINT(40.19596317459244 73.28027730249806) bank181500 +181501 POINT(40.48180541791648 74.70320731061399) bank181501 +181502 POINT(40.615878109783296 73.79688308607349) bank181502 +181503 POINT(41.49138803050262 74.50860449323824) bank181503 +181504 POINT(41.57665661764881 74.27390603245072) bank181504 +181505 POINT(41.682615420620586 73.04326550188817) bank181505 +181506 POINT(40.258878443399965 73.3976308016798) bank181506 +181507 POINT(40.687467515877884 73.95359607797369) bank181507 +181508 POINT(41.31171317488325 74.7823103821408) bank181508 +181509 POINT(40.50267317769992 73.06924731516905) bank181509 +181510 POINT(39.83078796820702 74.86102597280036) bank181510 +181511 POINT(41.551860610139165 74.56437210794866) bank181511 +181512 POINT(40.28288105111426 74.53097849042628) bank181512 +181513 POINT(41.27664433221211 74.45646911283364) bank181513 +181514 POINT(39.84979833613217 74.88963510382725) bank181514 +181515 POINT(40.014249480174165 74.0819739776876) bank181515 +181516 POINT(39.99921982708084 73.33052785738464) bank181516 +181517 POINT(41.70941021994097 73.24203043689263) bank181517 +181518 POINT(40.82969620663454 73.52549134860917) bank181518 +181519 POINT(41.21687110688087 74.04272107762557) bank181519 +181520 POINT(41.46978340504782 73.75293183711732) bank181520 +181521 POINT(41.51221321074332 73.86209264279513) bank181521 +181522 POINT(41.61482596186384 73.34450219387827) bank181522 +181523 POINT(40.089761482302585 73.78770388410473) bank181523 +181524 POINT(40.37853262973317 74.03778823863112) bank181524 +181525 POINT(41.66670110222286 74.74854787805273) bank181525 +181526 POINT(41.08120431179402 74.75780716778958) bank181526 +181527 POINT(40.14079831189843 74.96823764660398) bank181527 +181528 POINT(39.99195848404304 73.28154135395968) bank181528 +181529 POINT(41.326144355463605 74.19613420256769) bank181529 +181530 POINT(40.841940729194455 74.57678191356305) bank181530 +181531 POINT(40.032191230383475 73.08338118442389) bank181531 +181532 POINT(40.58683160075845 73.8837422847442) bank181532 +181533 POINT(40.917931857562515 73.96769926102732) bank181533 +181534 POINT(41.11414765300104 73.97667973022511) bank181534 +181535 POINT(41.167414840970416 73.41361340520761) bank181535 +181536 POINT(41.5780136024413 73.87840502112675) bank181536 +181537 POINT(41.2306105473876 74.62753444232678) bank181537 +181538 POINT(41.19191557021388 74.95380860342857) bank181538 +181539 POINT(40.434172710152644 73.66720539142698) bank181539 +181540 POINT(39.88891402471175 74.5603432270742) bank181540 +181541 POINT(40.521906306514715 73.18617858878163) bank181541 +181542 POINT(41.57371100054942 74.77196764383723) bank181542 +181543 POINT(40.86200591450117 74.42155381908317) bank181543 +181544 POINT(40.03268213504488 74.17003249923198) bank181544 +181545 POINT(40.35934672153105 73.99926027187465) bank181545 +181546 POINT(40.092474916453604 73.92885109384929) bank181546 +181547 POINT(41.13063665198304 73.333057965357) bank181547 +181548 POINT(40.132704734183704 73.08255511023563) bank181548 +181549 POINT(40.48669272666713 74.26166380335889) bank181549 +181550 POINT(41.31970492711176 74.25187378203368) bank181550 +181551 POINT(41.099185237760494 73.17736410877221) bank181551 +181552 POINT(39.844285623589954 73.66460430198846) bank181552 +181553 POINT(40.720485097119294 73.34552675144452) bank181553 +181554 POINT(41.06168424454106 74.70963528057) bank181554 +181555 POINT(39.98664363471128 74.74790087230167) bank181555 +181556 POINT(41.201179883064796 73.74486437700578) bank181556 +181557 POINT(41.06099878874166 73.12280214362974) bank181557 +181558 POINT(39.91793790837063 73.52670024091442) bank181558 +181559 POINT(39.82095109490236 73.44209181804972) bank181559 +181560 POINT(39.74882967010702 73.10929163968677) bank181560 +181561 POINT(40.590236818917155 74.57399608647918) bank181561 +181562 POINT(41.06427290270255 73.85451164915682) bank181562 +181563 POINT(41.61877619864444 74.0514933716996) bank181563 +181564 POINT(39.852076061068765 73.15352795098636) bank181564 +181565 POINT(40.956819713763934 73.93949409077553) bank181565 +181566 POINT(39.817102008962344 74.25093755881198) bank181566 +181567 POINT(40.28646805990753 73.8376315120323) bank181567 +181568 POINT(40.07733212406618 74.16324141727615) bank181568 +181569 POINT(41.2811859767057 73.76083508659262) bank181569 +181570 POINT(40.02781042694715 73.40617823244672) bank181570 +181571 POINT(40.3746141071618 73.70359028770612) bank181571 +181572 POINT(40.826396814765744 73.62774748737304) bank181572 +181573 POINT(40.27666314700391 74.56201010095666) bank181573 +181574 POINT(39.85614274280128 73.76273016956729) bank181574 +181575 POINT(41.19635525073713 74.84902025489562) bank181575 +181576 POINT(40.817983856008496 73.05563252382572) bank181576 +181577 POINT(40.42702892962322 73.84115616188154) bank181577 +181578 POINT(40.851249618392565 73.59499257805811) bank181578 +181579 POINT(40.73495768020022 73.08512243018922) bank181579 +181580 POINT(39.75579053300245 73.12649252076841) bank181580 +181581 POINT(41.09706528856224 73.76661015904801) bank181581 +181582 POINT(41.532683964873065 74.5344797692257) bank181582 +181583 POINT(40.028956672595704 74.51327997209039) bank181583 +181584 POINT(40.76831095054432 73.67386267255468) bank181584 +181585 POINT(41.52466779762806 74.12854041434083) bank181585 +181586 POINT(41.70993591675629 74.24608707433225) bank181586 +181587 POINT(40.13835437514091 73.9998884030754) bank181587 +181588 POINT(40.51283373503381 74.34529808217655) bank181588 +181589 POINT(41.10684745246097 73.05489376326462) bank181589 +181590 POINT(41.443074770916496 73.41357118002489) bank181590 +181591 POINT(40.23044146063842 74.69881149609363) bank181591 +181592 POINT(40.303484138000364 73.77656372029979) bank181592 +181593 POINT(40.26266165484775 74.5630604119876) bank181593 +181594 POINT(40.77943086697574 74.79351094910025) bank181594 +181595 POINT(41.21331208603938 74.9371176969108) bank181595 +181596 POINT(40.774964352502984 73.63162739198711) bank181596 +181597 POINT(40.536513094992 74.10980784712662) bank181597 +181598 POINT(40.98869250801505 74.96041389580243) bank181598 +181599 POINT(41.6355445934174 73.26035898512791) bank181599 +181600 POINT(40.961592228612375 74.18896744875859) bank181600 +181601 POINT(40.21414858692 73.52123336170033) bank181601 +181602 POINT(40.89616620446755 73.63162995261963) bank181602 +181603 POINT(39.862748721153324 73.5726565015965) bank181603 +181604 POINT(41.53855432906026 73.01652254369114) bank181604 +181605 POINT(40.5515475173657 73.08114094595774) bank181605 +181606 POINT(40.85191369255842 74.78437043239963) bank181606 +181607 POINT(41.539644982305795 74.8387515733411) bank181607 +181608 POINT(40.72072609082772 74.2287044453467) bank181608 +181609 POINT(40.30913797842245 73.14075701693244) bank181609 +181610 POINT(40.585897130665565 74.92644751670944) bank181610 +181611 POINT(41.01668248651731 73.35237860029939) bank181611 +181612 POINT(41.46678772174843 73.18309962833389) bank181612 +181613 POINT(41.694775200192645 74.12546084464157) bank181613 +181614 POINT(40.80681261651405 74.32260230520484) bank181614 +181615 POINT(40.80399392499877 73.45581205682683) bank181615 +181616 POINT(40.11246859609049 74.22211049867512) bank181616 +181617 POINT(40.896076474551585 74.02059341370814) bank181617 +181618 POINT(40.51823928637165 73.2285511381716) bank181618 +181619 POINT(39.77537707852238 74.16591248232487) bank181619 +181620 POINT(40.717864695888494 73.38107745317625) bank181620 +181621 POINT(40.2125835066941 73.53241478343554) bank181621 +181622 POINT(41.64736309547631 73.6978360831956) bank181622 +181623 POINT(40.015142799147995 73.35913041256097) bank181623 +181624 POINT(39.800316509888894 74.25023447469103) bank181624 +181625 POINT(40.77310185254004 74.8268997356625) bank181625 +181626 POINT(40.63684832955288 74.88778548612572) bank181626 +181627 POINT(41.31290885899499 74.67098741120887) bank181627 +181628 POINT(40.46017374200037 73.22982160385817) bank181628 +181629 POINT(40.78749379243231 74.44971150675488) bank181629 +181630 POINT(41.02558134248515 73.12853124501444) bank181630 +181631 POINT(41.031032126507895 73.57181545332917) bank181631 +181632 POINT(40.28755923045766 73.26509998124823) bank181632 +181633 POINT(39.72817174624556 73.34015159769565) bank181633 +181634 POINT(39.85307551784168 73.76148981471572) bank181634 +181635 POINT(40.12534995546168 74.81577052324063) bank181635 +181636 POINT(40.42700450299208 74.40185953446952) bank181636 +181637 POINT(41.347850854816144 73.5742297009502) bank181637 +181638 POINT(40.57536173994678 73.94362841632471) bank181638 +181639 POINT(40.95591496309141 73.24623057534365) bank181639 +181640 POINT(40.724162722601335 74.36440772974484) bank181640 +181641 POINT(39.9868485452377 73.54026280456206) bank181641 +181642 POINT(40.589463713863 74.98237134964769) bank181642 +181643 POINT(41.54247314108635 73.83991359015313) bank181643 +181644 POINT(39.758756002631614 74.1528163748639) bank181644 +181645 POINT(40.38867515453945 74.66111028259004) bank181645 +181646 POINT(40.488070148002556 73.02940882606434) bank181646 +181647 POINT(41.063388710243764 73.08855942800734) bank181647 +181648 POINT(40.46266909852499 74.11738034735528) bank181648 +181649 POINT(40.53150516840375 74.70424915539596) bank181649 +181650 POINT(40.4075749663078 73.38814734782918) bank181650 +181651 POINT(40.27688637852116 74.47744384651139) bank181651 +181652 POINT(41.070197934242465 74.29447049200449) bank181652 +181653 POINT(41.095377095115275 73.59353767407744) bank181653 +181654 POINT(39.81487552266826 73.97734255466564) bank181654 +181655 POINT(40.908212269103586 73.24265541101609) bank181655 +181656 POINT(41.68836262161899 73.43215610943166) bank181656 +181657 POINT(40.79576504826746 74.7716633643414) bank181657 +181658 POINT(41.181964137775736 73.06353686440644) bank181658 +181659 POINT(41.15538186265573 74.36545304303057) bank181659 +181660 POINT(40.841879264199726 73.9361059664056) bank181660 +181661 POINT(39.806761116280406 73.47919474175377) bank181661 +181662 POINT(39.917274431116745 74.5952373542269) bank181662 +181663 POINT(39.89861288802486 74.18982091195818) bank181663 +181664 POINT(40.615917738304894 74.93041182484399) bank181664 +181665 POINT(41.20092198078089 74.81624500160139) bank181665 +181666 POINT(40.871614876959484 74.00280537958426) bank181666 +181667 POINT(41.19994141060264 74.03137270644274) bank181667 +181668 POINT(39.722707646483926 73.213057194275) bank181668 +181669 POINT(40.59351962003066 73.95534386942686) bank181669 +181670 POINT(41.67458600235999 74.63954119317235) bank181670 +181671 POINT(40.685267457298025 74.93795795543245) bank181671 +181672 POINT(41.10204808741477 73.97599972793553) bank181672 +181673 POINT(41.4951426878129 73.3820625693333) bank181673 +181674 POINT(39.92659413081623 74.51627103464865) bank181674 +181675 POINT(41.3115842188825 73.67969773339456) bank181675 +181676 POINT(41.390455291977744 73.23730141300413) bank181676 +181677 POINT(41.22212600015554 74.91794760012236) bank181677 +181678 POINT(39.90197107762324 74.61372043342283) bank181678 +181679 POINT(40.346443560635095 73.85255061492889) bank181679 +181680 POINT(40.51648840200083 74.80932193513506) bank181680 +181681 POINT(41.21679570608059 73.72272320544624) bank181681 +181682 POINT(41.58901910373782 73.12241951297773) bank181682 +181683 POINT(39.93618246804722 73.54518038284759) bank181683 +181684 POINT(41.18754844597441 74.54036609231197) bank181684 +181685 POINT(40.411425937791705 73.3521052606625) bank181685 +181686 POINT(40.16475925148288 74.90651928561061) bank181686 +181687 POINT(40.895347123325536 73.70126713598901) bank181687 +181688 POINT(41.50807268635867 73.29792268022315) bank181688 +181689 POINT(40.60504776818708 73.13138952471756) bank181689 +181690 POINT(40.613904603381755 74.65167793815687) bank181690 +181691 POINT(39.91718295920067 73.38153113101485) bank181691 +181692 POINT(40.48930658741691 74.44913050040579) bank181692 +181693 POINT(41.386288838886664 73.76436267106915) bank181693 +181694 POINT(40.54661371095929 73.6777487233279) bank181694 +181695 POINT(41.68596420540475 74.82338223055025) bank181695 +181696 POINT(40.889686541405986 73.66264269513285) bank181696 +181697 POINT(41.035410544821254 74.07429083850825) bank181697 +181698 POINT(40.0225215792545 74.7065665958367) bank181698 +181699 POINT(41.14888704304295 74.68233709371607) bank181699 +181700 POINT(40.29076307044177 74.98093942010179) bank181700 +181701 POINT(41.47040308633572 73.39074979981208) bank181701 +181702 POINT(41.367624957674494 73.63483897171538) bank181702 +181703 POINT(40.59211906529733 74.02274605488078) bank181703 +181704 POINT(41.558464192819656 73.5731830551171) bank181704 +181705 POINT(41.55882208072488 74.32920413154297) bank181705 +181706 POINT(39.78802707095856 74.12827552157765) bank181706 +181707 POINT(41.071010598852084 74.24047626495228) bank181707 +181708 POINT(41.354483999807314 74.03380528655521) bank181708 +181709 POINT(40.520512742994356 73.70516652696851) bank181709 +181710 POINT(40.39823846010607 73.29666810848855) bank181710 +181711 POINT(40.071064534794104 73.29997692759383) bank181711 +181712 POINT(41.090050001493594 74.71478344472887) bank181712 +181713 POINT(39.77942501159127 74.3503161690218) bank181713 +181714 POINT(41.030942544271646 74.72997461200598) bank181714 +181715 POINT(41.60688814070326 73.31897881196946) bank181715 +181716 POINT(41.396163532132526 73.0591005206963) bank181716 +181717 POINT(39.7656785951127 73.03069877761703) bank181717 +181718 POINT(40.99014305186515 73.27637968449964) bank181718 +181719 POINT(40.5331335745861 73.78511839087679) bank181719 +181720 POINT(40.38885682196211 73.41349272675573) bank181720 +181721 POINT(41.38305430002007 74.8951548884632) bank181721 +181722 POINT(39.8908575464784 74.59263123137492) bank181722 +181723 POINT(40.54437045850205 74.09584257841709) bank181723 +181724 POINT(41.06062094766005 73.63245856608657) bank181724 +181725 POINT(40.80085431418032 74.01205687514593) bank181725 +181726 POINT(40.7999340936834 74.33860793594627) bank181726 +181727 POINT(40.0446564854596 73.03040229141371) bank181727 +181728 POINT(41.413015521504164 73.70968893104447) bank181728 +181729 POINT(40.36234789913732 74.74573285532762) bank181729 +181730 POINT(40.52712479194142 74.56219152882258) bank181730 +181731 POINT(41.21884427997596 73.09534853663826) bank181731 +181732 POINT(40.20394170097294 73.80130362732284) bank181732 +181733 POINT(41.41270562650266 74.47020733548473) bank181733 +181734 POINT(41.07110371665425 74.92272556955946) bank181734 +181735 POINT(41.19171616750166 73.14830507013279) bank181735 +181736 POINT(40.05961240876457 74.95493714956162) bank181736 +181737 POINT(40.03862841148875 74.83975974612376) bank181737 +181738 POINT(40.041039197767205 73.1029696032505) bank181738 +181739 POINT(39.95621658995225 73.27189480432097) bank181739 +181740 POINT(41.38751954862756 73.20106457939646) bank181740 +181741 POINT(40.730051758130976 73.78741785158458) bank181741 +181742 POINT(40.479277918333345 74.28683523188552) bank181742 +181743 POINT(41.614275470738036 73.94909037687583) bank181743 +181744 POINT(40.22899493010018 74.95650415317901) bank181744 +181745 POINT(39.883941857157836 73.77527480497841) bank181745 +181746 POINT(40.8967417947731 73.83841738008151) bank181746 +181747 POINT(40.35323912384259 73.66196289423806) bank181747 +181748 POINT(40.08052566887622 73.97568504129639) bank181748 +181749 POINT(40.59140607347661 74.9864694792382) bank181749 +181750 POINT(40.56213980615805 74.84987967937128) bank181750 +181751 POINT(41.465040491751644 74.33209220711113) bank181751 +181752 POINT(41.63884573213946 73.16797118258162) bank181752 +181753 POINT(41.670503016178785 74.85715217937678) bank181753 +181754 POINT(40.10292582042888 73.14895893046064) bank181754 +181755 POINT(41.66071805937076 73.56699531859556) bank181755 +181756 POINT(40.05861178975874 74.37161263688817) bank181756 +181757 POINT(40.89361643282938 74.05254443740053) bank181757 +181758 POINT(39.81547284721244 73.90525801235967) bank181758 +181759 POINT(41.01315399386305 74.47081415603898) bank181759 +181760 POINT(41.42943281503067 74.80045085049966) bank181760 +181761 POINT(40.688654196307844 73.22685756328272) bank181761 +181762 POINT(40.82573446932241 74.0838145065911) bank181762 +181763 POINT(40.390389475354006 74.00093863433914) bank181763 +181764 POINT(40.48713936720811 74.22030240355248) bank181764 +181765 POINT(41.127549172070374 74.83811354500466) bank181765 +181766 POINT(41.00995454654093 73.22666256490383) bank181766 +181767 POINT(40.39726575523484 74.73676968358906) bank181767 +181768 POINT(41.54647402434594 74.197369870606) bank181768 +181769 POINT(41.01776145583536 73.98240787480485) bank181769 +181770 POINT(40.75759294678623 73.26967175119795) bank181770 +181771 POINT(39.80131932781774 73.64343154336729) bank181771 +181772 POINT(41.60308267587527 74.84521188584263) bank181772 +181773 POINT(40.084528456197894 73.31221545449404) bank181773 +181774 POINT(40.870095350741074 74.09850574487477) bank181774 +181775 POINT(39.74341112678364 73.61217657814922) bank181775 +181776 POINT(41.68288847763685 73.18636761853857) bank181776 +181777 POINT(41.16671655022759 73.38026353476215) bank181777 +181778 POINT(40.75875606168058 73.89018801326634) bank181778 +181779 POINT(39.76260659847963 74.99014873384706) bank181779 +181780 POINT(41.57101882204086 73.66954553677937) bank181780 +181781 POINT(40.878644029350035 73.66716015582631) bank181781 +181782 POINT(41.087624487824186 74.7313111487492) bank181782 +181783 POINT(41.07917065767687 74.72457858374833) bank181783 +181784 POINT(40.8617310725564 73.67871789887413) bank181784 +181785 POINT(41.58798379313037 74.27098449214932) bank181785 +181786 POINT(40.088674221931015 74.27977337518102) bank181786 +181787 POINT(40.24471662217112 73.09287729912609) bank181787 +181788 POINT(40.94085026727427 73.49663876842911) bank181788 +181789 POINT(40.68589178929863 74.57710362891554) bank181789 +181790 POINT(41.39619274586453 73.17982148880891) bank181790 +181791 POINT(40.764644058937584 73.07007713657158) bank181791 +181792 POINT(41.17171317553267 74.3699173719696) bank181792 +181793 POINT(39.78563430619877 73.1536470071288) bank181793 +181794 POINT(40.184723283818336 73.54136938570069) bank181794 +181795 POINT(40.761862564031134 74.49999834042539) bank181795 +181796 POINT(41.33321133019968 74.51753404046056) bank181796 +181797 POINT(40.346731008745664 73.7276663398492) bank181797 +181798 POINT(40.124848146587894 74.88831786236527) bank181798 +181799 POINT(39.897595021439265 73.62793294191749) bank181799 +181800 POINT(40.268034156541205 74.98625332519792) bank181800 +181801 POINT(40.84640554642579 74.5762646472988) bank181801 +181802 POINT(40.68242687825073 74.66436420387323) bank181802 +181803 POINT(41.56537509313192 74.01354086516035) bank181803 +181804 POINT(40.951869946883775 73.06945074194563) bank181804 +181805 POINT(41.482254273885296 73.92609809665187) bank181805 +181806 POINT(40.4130375543218 74.87749244318024) bank181806 +181807 POINT(40.18105414774277 74.091999026621) bank181807 +181808 POINT(39.94146669628051 73.31390051354848) bank181808 +181809 POINT(40.76328196632056 73.31283139425368) bank181809 +181810 POINT(41.30041424187381 73.06553964898688) bank181810 +181811 POINT(40.02349711055369 73.05855672336543) bank181811 +181812 POINT(40.18544853621122 74.92416997738178) bank181812 +181813 POINT(40.58860134102801 73.8847826880774) bank181813 +181814 POINT(41.66293926506586 74.15955964925575) bank181814 +181815 POINT(40.124502965255004 73.05038194277563) bank181815 +181816 POINT(40.456787388933485 73.71927120111921) bank181816 +181817 POINT(41.6398597891304 73.77495732570728) bank181817 +181818 POINT(39.88924705117085 74.5692651774963) bank181818 +181819 POINT(41.35031902242947 74.47809042231677) bank181819 +181820 POINT(40.753838161949695 74.68072029857555) bank181820 +181821 POINT(39.88751084101784 74.03269351225548) bank181821 +181822 POINT(41.509240333684374 74.6766485662803) bank181822 +181823 POINT(40.017745727746046 73.24475388765094) bank181823 +181824 POINT(41.63559451144896 74.87201928971419) bank181824 +181825 POINT(41.48421842101252 74.59538521706851) bank181825 +181826 POINT(40.41203821663177 74.77984696660565) bank181826 +181827 POINT(41.29653760599578 74.98864911633456) bank181827 +181828 POINT(40.09952119769828 73.93588486920156) bank181828 +181829 POINT(39.979171116556316 73.9343161826749) bank181829 +181830 POINT(39.793796720168544 73.62859361149525) bank181830 +181831 POINT(41.14291365231939 74.22856766476339) bank181831 +181832 POINT(41.4289915503798 74.19623474127036) bank181832 +181833 POINT(40.441976113927375 74.47873437838551) bank181833 +181834 POINT(40.27297138641561 74.89564069969299) bank181834 +181835 POINT(40.48987772610031 74.78935898519909) bank181835 +181836 POINT(39.903914326226094 73.04590912704978) bank181836 +181837 POINT(41.253007067821464 73.04629545082716) bank181837 +181838 POINT(40.03260941693414 74.18617433103431) bank181838 +181839 POINT(41.159289350670235 73.91073942286141) bank181839 +181840 POINT(41.03035346121684 73.01785697343912) bank181840 +181841 POINT(41.2108839814486 74.91395174943375) bank181841 +181842 POINT(39.832487709732575 73.18263072249934) bank181842 +181843 POINT(40.34521145848923 73.661779153406) bank181843 +181844 POINT(40.413495023706105 74.71550799242758) bank181844 +181845 POINT(41.34367541725493 74.83941304608993) bank181845 +181846 POINT(40.54192038448609 73.39644250293132) bank181846 +181847 POINT(41.27466120562531 73.2586431951182) bank181847 +181848 POINT(40.834580054694264 74.47394802006184) bank181848 +181849 POINT(41.67499475867441 74.21399584306921) bank181849 +181850 POINT(41.30299101054781 74.98047805136325) bank181850 +181851 POINT(41.316917814086466 73.83620206193704) bank181851 +181852 POINT(40.34141990338244 74.50183677631803) bank181852 +181853 POINT(40.21134290091556 73.53734968149686) bank181853 +181854 POINT(40.06168327383641 73.62359777102928) bank181854 +181855 POINT(40.299812179814445 74.54628970276829) bank181855 +181856 POINT(40.395250619374636 74.22959847343843) bank181856 +181857 POINT(40.0741957153834 73.69901578144452) bank181857 +181858 POINT(40.87426575391006 74.8963788047638) bank181858 +181859 POINT(39.91591836116957 73.73258736368417) bank181859 +181860 POINT(39.85499849628722 74.55688794620048) bank181860 +181861 POINT(41.058384552438724 74.96765634178028) bank181861 +181862 POINT(41.54156442604373 74.9417839285617) bank181862 +181863 POINT(40.80366181368696 74.32141341112981) bank181863 +181864 POINT(40.60319869468865 74.72159170299989) bank181864 +181865 POINT(39.915046054551574 74.15503197294724) bank181865 +181866 POINT(40.32342163147137 73.93757814018544) bank181866 +181867 POINT(40.300523081832 73.53199799089245) bank181867 +181868 POINT(40.14978620581187 73.64072508897588) bank181868 +181869 POINT(40.13568631040261 74.389377487418) bank181869 +181870 POINT(40.187710108744014 74.24126335559534) bank181870 +181871 POINT(40.795327836389575 74.68925629495669) bank181871 +181872 POINT(40.183009839730694 73.51118446126074) bank181872 +181873 POINT(41.356691884315744 73.51648671089617) bank181873 +181874 POINT(40.21264538314964 73.22843704649746) bank181874 +181875 POINT(41.43473417611244 73.74889454849226) bank181875 +181876 POINT(40.40118985405121 73.35106262957747) bank181876 +181877 POINT(41.69034547277221 73.77922056326511) bank181877 +181878 POINT(40.95418036276905 73.95940174976413) bank181878 +181879 POINT(41.55526993138645 73.5095607956308) bank181879 +181880 POINT(41.70170142071313 73.30852771833484) bank181880 +181881 POINT(41.3468119843822 73.30760288507658) bank181881 +181882 POINT(41.41731438898843 73.76592605278924) bank181882 +181883 POINT(40.94585291191219 73.2105866570377) bank181883 +181884 POINT(41.675968368224694 74.71594054472358) bank181884 +181885 POINT(40.549996790001885 74.35495553327733) bank181885 +181886 POINT(40.84698304741252 74.56438727074698) bank181886 +181887 POINT(40.86818725518485 74.05988372564492) bank181887 +181888 POINT(39.91875894580583 73.1820847201165) bank181888 +181889 POINT(41.25076714349408 73.53308660138906) bank181889 +181890 POINT(40.41114446828323 73.58176022535314) bank181890 +181891 POINT(40.92199983858456 74.80693716087953) bank181891 +181892 POINT(40.369183207147884 73.16184972236685) bank181892 +181893 POINT(41.45704569816449 74.41083720299561) bank181893 +181894 POINT(40.4567499488046 73.28726439481524) bank181894 +181895 POINT(41.21414178010971 73.97853393068152) bank181895 +181896 POINT(40.940156783586694 74.01090220082251) bank181896 +181897 POINT(41.526006457823314 73.47173906216626) bank181897 +181898 POINT(40.496723904370285 73.69054965609722) bank181898 +181899 POINT(40.07431191136805 74.77400459702153) bank181899 +181900 POINT(39.75840530222135 73.94161714198417) bank181900 +181901 POINT(40.79089311981515 73.68026633488896) bank181901 +181902 POINT(40.09401846843539 74.98675103602895) bank181902 +181903 POINT(41.324905472056116 74.85464978390229) bank181903 +181904 POINT(41.332584351760886 74.09454330964549) bank181904 +181905 POINT(40.89176960838334 73.60262824802943) bank181905 +181906 POINT(41.32070361393824 74.83379536496697) bank181906 +181907 POINT(40.10813067147075 74.04004358509546) bank181907 +181908 POINT(41.04734594344702 74.62743508235934) bank181908 +181909 POINT(40.49720347772988 74.28949184759355) bank181909 +181910 POINT(40.28078627046047 74.51627740864579) bank181910 +181911 POINT(41.11783070080696 73.5930560424998) bank181911 +181912 POINT(41.67638706032309 74.58789779981636) bank181912 +181913 POINT(41.570115115218975 74.17586623639302) bank181913 +181914 POINT(40.700094196826896 73.21719682062653) bank181914 +181915 POINT(39.92731773343277 74.00659106363138) bank181915 +181916 POINT(41.49450656153973 73.3790250969321) bank181916 +181917 POINT(40.61572063818535 73.73253715046073) bank181917 +181918 POINT(40.6926288110357 73.77760147855196) bank181918 +181919 POINT(41.63918704795118 74.24742386811761) bank181919 +181920 POINT(41.40239541132185 74.04212777149885) bank181920 +181921 POINT(41.501501285186144 74.63819096332895) bank181921 +181922 POINT(40.00841089401868 74.91956923092164) bank181922 +181923 POINT(40.106905823745 74.78387872236839) bank181923 +181924 POINT(41.25075132998409 74.88963912276458) bank181924 +181925 POINT(40.829036461633315 73.85341337887007) bank181925 +181926 POINT(41.40233852481141 73.84896440758268) bank181926 +181927 POINT(40.15414989734972 74.65988118492) bank181927 +181928 POINT(40.523592826089455 74.0068816280574) bank181928 +181929 POINT(40.200840711439916 74.67868794076112) bank181929 +181930 POINT(39.78908643177283 73.67892706443382) bank181930 +181931 POINT(40.895526048971185 74.36682592819741) bank181931 +181932 POINT(41.11608645486399 73.37180472064749) bank181932 +181933 POINT(39.81839523826566 73.89741827096731) bank181933 +181934 POINT(40.77660301749533 74.7226398388758) bank181934 +181935 POINT(40.39593938803915 73.79479707274174) bank181935 +181936 POINT(40.895151320210985 73.37217465539595) bank181936 +181937 POINT(41.261095376462286 74.17321802434805) bank181937 +181938 POINT(41.47132874236781 74.03009605190061) bank181938 +181939 POINT(40.66678324255679 74.95424951185171) bank181939 +181940 POINT(40.02251605060594 73.9969713223436) bank181940 +181941 POINT(41.01848842621468 74.54566160539073) bank181941 +181942 POINT(40.324859006845706 74.2740641783235) bank181942 +181943 POINT(41.41519415024817 73.03239581784054) bank181943 +181944 POINT(41.18882860862127 74.47362341823995) bank181944 +181945 POINT(40.57803839675079 74.14546851109935) bank181945 +181946 POINT(40.20695782253388 74.01361536703112) bank181946 +181947 POINT(41.39090712776459 73.39731617934368) bank181947 +181948 POINT(40.876361610583395 74.61444281386021) bank181948 +181949 POINT(39.843669373258344 74.61147112346775) bank181949 +181950 POINT(41.10290007566574 74.19421047129778) bank181950 +181951 POINT(41.523688474194074 74.08237413381912) bank181951 +181952 POINT(41.39102915349171 73.53438756717517) bank181952 +181953 POINT(40.948770733407855 74.45402037230564) bank181953 +181954 POINT(41.27923067477365 73.3976606810391) bank181954 +181955 POINT(41.66656984567656 73.50472629183801) bank181955 +181956 POINT(41.52288172675046 73.6216944450553) bank181956 +181957 POINT(40.3745951988448 73.55074082025791) bank181957 +181958 POINT(40.9350740792308 73.48755671863628) bank181958 +181959 POINT(39.931059231097535 73.44309971894465) bank181959 +181960 POINT(41.28705129095994 74.81861679722314) bank181960 +181961 POINT(39.72832883413434 73.32212526470529) bank181961 +181962 POINT(41.32576668253645 74.31326695071965) bank181962 +181963 POINT(41.639278386221555 74.85431186823854) bank181963 +181964 POINT(41.375774156070825 73.1205030057581) bank181964 +181965 POINT(39.728953582666605 74.56867774515678) bank181965 +181966 POINT(40.535098685376056 74.5895861549342) bank181966 +181967 POINT(39.86649123260276 74.06780032337484) bank181967 +181968 POINT(39.77243625137787 74.10208158343306) bank181968 +181969 POINT(41.27555002796547 73.3108991582816) bank181969 +181970 POINT(41.35180925349261 74.27685729820978) bank181970 +181971 POINT(41.63913014099492 74.72330978779866) bank181971 +181972 POINT(41.50816008710588 73.92922361659149) bank181972 +181973 POINT(40.07994676552321 74.22104104502677) bank181973 +181974 POINT(40.67841744170695 74.57976675715081) bank181974 +181975 POINT(40.58797315978145 73.75436264317003) bank181975 +181976 POINT(41.016175585711785 73.00619947151813) bank181976 +181977 POINT(40.48706409558278 74.43430539412377) bank181977 +181978 POINT(41.537500614268424 73.64217246842182) bank181978 +181979 POINT(39.99947813972771 74.73285276836525) bank181979 +181980 POINT(40.977392200541296 73.72707226537756) bank181980 +181981 POINT(41.13385129361606 73.58526277158771) bank181981 +181982 POINT(41.41693683807048 74.95055579641921) bank181982 +181983 POINT(41.672366574302366 74.27597547949149) bank181983 +181984 POINT(40.63029622314654 73.43269933106278) bank181984 +181985 POINT(40.00016662560419 73.73866446160291) bank181985 +181986 POINT(40.398790627491756 73.37681418331437) bank181986 +181987 POINT(39.99676690681904 73.16155525380579) bank181987 +181988 POINT(41.59488426672066 73.01769635701204) bank181988 +181989 POINT(41.02327731576559 73.32414924795265) bank181989 +181990 POINT(41.43383400577946 73.76839270833133) bank181990 +181991 POINT(39.72983087198561 74.57917519867898) bank181991 +181992 POINT(40.31513627213329 73.24706969286476) bank181992 +181993 POINT(41.26514479610104 74.54101058328203) bank181993 +181994 POINT(40.03255061900848 74.22840238994483) bank181994 +181995 POINT(40.199398260423514 74.72136679092024) bank181995 +181996 POINT(39.82754945363465 73.55799339653531) bank181996 +181997 POINT(40.69709654421722 73.89863260730493) bank181997 +181998 POINT(40.104986650404776 74.10004072766351) bank181998 +181999 POINT(40.000742588892905 74.58068940927755) bank181999 +182000 POINT(39.81485826325833 73.6933563955803) bank182000 +182001 POINT(39.84802840794059 74.62463629467305) bank182001 +182002 POINT(40.71981625958878 73.81820233533558) bank182002 +182003 POINT(40.19710141683003 73.01621346270382) bank182003 +182004 POINT(41.17680753828394 73.02248069395313) bank182004 +182005 POINT(39.8569636555885 73.36664269994459) bank182005 +182006 POINT(40.076174167844016 74.24012516147165) bank182006 +182007 POINT(39.72690363090141 73.58853497468697) bank182007 +182008 POINT(41.045736178889946 74.10780504131911) bank182008 +182009 POINT(41.527829165648676 74.49933186447292) bank182009 +182010 POINT(40.803266185488816 73.95632981250273) bank182010 +182011 POINT(40.03670711356781 73.76963909736868) bank182011 +182012 POINT(40.98259070580859 73.73838712578944) bank182012 +182013 POINT(41.44266415973104 74.22029025318564) bank182013 +182014 POINT(40.67229359715772 74.77538241968885) bank182014 +182015 POINT(41.55068410540422 73.2073834241451) bank182015 +182016 POINT(40.69243315513849 74.89048296330736) bank182016 +182017 POINT(40.3693859579305 73.03370086813437) bank182017 +182018 POINT(40.21278459517688 73.7329786290712) bank182018 +182019 POINT(39.76796146670023 74.45238704682761) bank182019 +182020 POINT(41.10747581213716 74.44802614832444) bank182020 +182021 POINT(39.96831482541172 73.31002595623531) bank182021 +182022 POINT(40.36612738417781 74.9457532570432) bank182022 +182023 POINT(40.47909092720231 74.03386339337884) bank182023 +182024 POINT(41.28802701609563 74.61935198672907) bank182024 +182025 POINT(40.818166863760915 73.78709348349572) bank182025 +182026 POINT(40.85541532688175 73.95914672097979) bank182026 +182027 POINT(41.184338406933456 73.16664345281838) bank182027 +182028 POINT(40.41691018768001 73.40870126334347) bank182028 +182029 POINT(40.36192240421657 74.99629138107439) bank182029 +182030 POINT(40.687857582217646 74.11729713966757) bank182030 +182031 POINT(40.11782883798324 73.06455685497136) bank182031 +182032 POINT(40.49386633994453 74.08069632559405) bank182032 +182033 POINT(40.13976978995335 73.99758230641542) bank182033 +182034 POINT(40.27190606030708 74.5499717379673) bank182034 +182035 POINT(41.45756655710817 73.10253848068282) bank182035 +182036 POINT(41.199624020917895 73.76173780587281) bank182036 +182037 POINT(40.562584607757344 73.33123808832016) bank182037 +182038 POINT(41.31795428043383 74.50841114385578) bank182038 +182039 POINT(40.65321112200141 74.687578107625) bank182039 +182040 POINT(41.46032697431089 74.86223412593228) bank182040 +182041 POINT(40.58320442491535 73.54780605199417) bank182041 +182042 POINT(40.664195982351586 74.77976525424884) bank182042 +182043 POINT(40.152646994662504 74.54444527695212) bank182043 +182044 POINT(41.558892053156306 73.27075572954493) bank182044 +182045 POINT(39.8267731624982 73.82395345261197) bank182045 +182046 POINT(41.160411184932535 73.02983236290231) bank182046 +182047 POINT(41.1453821450233 73.9124129150895) bank182047 +182048 POINT(39.97900781693288 74.1005073410883) bank182048 +182049 POINT(40.53471973728771 74.81769770238681) bank182049 +182050 POINT(40.31325861017771 74.45583735802151) bank182050 +182051 POINT(41.10130419546414 74.74926191914874) bank182051 +182052 POINT(41.082006298945814 74.42317387334951) bank182052 +182053 POINT(40.16764918551048 74.40617945978161) bank182053 +182054 POINT(41.61860008701748 74.64264070605978) bank182054 +182055 POINT(40.41461489571719 73.57603072870097) bank182055 +182056 POINT(40.92789238178426 74.30439999506036) bank182056 +182057 POINT(40.47245313319753 74.57561249159194) bank182057 +182058 POINT(40.901510426519756 74.55842838766256) bank182058 +182059 POINT(41.21334841427828 74.24324597345019) bank182059 +182060 POINT(40.712710220913884 73.53398667576315) bank182060 +182061 POINT(41.27679693035707 73.26255238776606) bank182061 +182062 POINT(40.00312423187071 73.4110522050065) bank182062 +182063 POINT(41.06218381361911 73.44454656624835) bank182063 +182064 POINT(41.13671311763958 74.85273923531858) bank182064 +182065 POINT(40.82969287453769 73.9744653376739) bank182065 +182066 POINT(40.010107137029145 74.71793092190723) bank182066 +182067 POINT(40.72766578842615 74.61122388648299) bank182067 +182068 POINT(39.788175949104236 74.90064739681752) bank182068 +182069 POINT(41.17368183160375 73.11874488570889) bank182069 +182070 POINT(40.468839425966564 74.81208999428098) bank182070 +182071 POINT(40.62507125011195 74.48830291428364) bank182071 +182072 POINT(41.17763400556491 73.19817768437497) bank182072 +182073 POINT(41.346264171743485 73.0176196449011) bank182073 +182074 POINT(40.34235013406456 74.25035158202132) bank182074 +182075 POINT(40.24892107635889 74.80861350104016) bank182075 +182076 POINT(41.27926046102084 74.56298246931084) bank182076 +182077 POINT(40.96724102289125 73.84674271415396) bank182077 +182078 POINT(39.886304688582676 73.28985082036577) bank182078 +182079 POINT(40.126588023227946 74.66456965812775) bank182079 +182080 POINT(39.71487415592889 74.07939410797161) bank182080 +182081 POINT(41.330198048125524 74.12505626781909) bank182081 +182082 POINT(40.32405021355649 73.0447082734066) bank182082 +182083 POINT(41.6331434101399 73.19573730866962) bank182083 +182084 POINT(41.2157047287724 73.55484762172267) bank182084 +182085 POINT(41.4839804506553 74.51423315985204) bank182085 +182086 POINT(41.154139707916094 73.43583863797588) bank182086 +182087 POINT(41.31971518801677 74.48779062240521) bank182087 +182088 POINT(41.18534696564669 73.26865961516741) bank182088 +182089 POINT(41.170345337038185 74.37454277011017) bank182089 +182090 POINT(41.08933814633686 73.65215224414753) bank182090 +182091 POINT(41.12396455953162 73.86432897827976) bank182091 +182092 POINT(41.233351478613 75.00120515248793) bank182092 +182093 POINT(40.13856795530019 73.91267746290958) bank182093 +182094 POINT(41.234821551248444 73.73650534399657) bank182094 +182095 POINT(41.343459855285715 74.1166307049851) bank182095 +182096 POINT(41.084977414628845 73.97006541558832) bank182096 +182097 POINT(40.91224833832371 73.07888802667006) bank182097 +182098 POINT(40.870376826472146 73.93320232681194) bank182098 +182099 POINT(41.195347804244854 74.85439832356842) bank182099 +182100 POINT(40.00650185117453 73.98240428313034) bank182100 +182101 POINT(41.39646177629663 73.12666102114984) bank182101 +182102 POINT(41.2971844863954 74.08534073971323) bank182102 +182103 POINT(39.9185255018195 73.873814678545) bank182103 +182104 POINT(41.12990754539038 73.77106695619135) bank182104 +182105 POINT(40.295499590257606 74.47216665489542) bank182105 +182106 POINT(40.52356455813775 73.8413481538919) bank182106 +182107 POINT(40.1299706591819 74.14455939115732) bank182107 +182108 POINT(40.959705940254246 74.68822985105646) bank182108 +182109 POINT(41.66077868095108 74.39237063844018) bank182109 +182110 POINT(41.20444528602485 73.02870996515603) bank182110 +182111 POINT(40.61051387512894 74.38029723593279) bank182111 +182112 POINT(40.63018263668566 73.05269638625579) bank182112 +182113 POINT(40.35429704653591 74.32965858859643) bank182113 +182114 POINT(40.12345542968193 73.24754525772927) bank182114 +182115 POINT(41.50078663126412 74.73642606544738) bank182115 +182116 POINT(39.784339956572296 74.70721506394779) bank182116 +182117 POINT(40.45714691236377 73.72987680314361) bank182117 +182118 POINT(40.0825976160323 74.73629985208983) bank182118 +182119 POINT(40.46747035185521 73.17364455471188) bank182119 +182120 POINT(41.367044014376695 74.47915549764322) bank182120 +182121 POINT(40.64335026549483 73.76420668698596) bank182121 +182122 POINT(40.45430470323944 73.84016553902923) bank182122 +182123 POINT(40.609958098831726 73.5013202319887) bank182123 +182124 POINT(41.585456200048725 74.85007432400023) bank182124 +182125 POINT(41.53495520061238 74.83263741351661) bank182125 +182126 POINT(40.31796329962285 74.11704730598882) bank182126 +182127 POINT(40.35706390512077 73.09484638380836) bank182127 +182128 POINT(40.18254022125235 74.74836781952497) bank182128 +182129 POINT(39.901833433816975 73.65025293904131) bank182129 +182130 POINT(39.896348071207896 74.03074776727286) bank182130 +182131 POINT(40.92190059791379 73.98128131143088) bank182131 +182132 POINT(39.97411265165531 73.43569503068092) bank182132 +182133 POINT(40.41243570293681 73.93329904119204) bank182133 +182134 POINT(41.127881196082605 74.39401083936094) bank182134 +182135 POINT(41.41863785017477 74.50155409758044) bank182135 +182136 POINT(41.59795279648884 73.02691934722262) bank182136 +182137 POINT(41.45706975245742 73.87443252866514) bank182137 +182138 POINT(41.35862871916575 74.91140475042229) bank182138 +182139 POINT(40.79956543445972 74.19649560967682) bank182139 +182140 POINT(41.41580410893073 73.57247218428365) bank182140 +182141 POINT(40.73415695766762 74.48366039407614) bank182141 +182142 POINT(39.81327531219837 73.55235020383755) bank182142 +182143 POINT(40.05568757660177 74.39924981685057) bank182143 +182144 POINT(40.50484271064878 74.69981693913145) bank182144 +182145 POINT(40.029920598266926 74.32666765110281) bank182145 +182146 POINT(41.340100052341754 74.85894475106912) bank182146 +182147 POINT(41.045619569224044 74.99964434463836) bank182147 +182148 POINT(40.242212272387626 74.9978870750635) bank182148 +182149 POINT(40.382174680717974 74.22221747140475) bank182149 +182150 POINT(41.38358756946096 73.10239348706622) bank182150 +182151 POINT(41.37298647523268 74.97415586272643) bank182151 +182152 POINT(40.76110931608322 74.5586225860085) bank182152 +182153 POINT(40.433257938066184 73.92103735040605) bank182153 +182154 POINT(40.09759627301289 73.07246268168056) bank182154 +182155 POINT(41.400281350566935 74.51735103440775) bank182155 +182156 POINT(40.72961428156004 73.32956297224523) bank182156 +182157 POINT(40.15014731312983 74.90569201381066) bank182157 +182158 POINT(40.98975178640448 74.88495356085154) bank182158 +182159 POINT(40.035509229426104 73.22102332145992) bank182159 +182160 POINT(40.9208607524496 74.85541265242176) bank182160 +182161 POINT(39.75887101862894 74.27833510873005) bank182161 +182162 POINT(41.47412693783125 74.10248719338828) bank182162 +182163 POINT(40.93079125049748 73.7922453451359) bank182163 +182164 POINT(40.8814367506709 73.4155444445415) bank182164 +182165 POINT(41.47715477022138 73.16630495451807) bank182165 +182166 POINT(41.185139910556856 74.33483051551869) bank182166 +182167 POINT(41.41615568454374 73.58357023810356) bank182167 +182168 POINT(41.151568020639566 73.1679676787383) bank182168 +182169 POINT(39.963140968732375 73.44558800380747) bank182169 +182170 POINT(40.26969190863964 74.14908363454344) bank182170 +182171 POINT(40.289438478558466 73.90903541652669) bank182171 +182172 POINT(41.088478147272255 74.57210633777268) bank182172 +182173 POINT(40.45824670040765 73.03209677733561) bank182173 +182174 POINT(39.72946253017413 74.61023627731218) bank182174 +182175 POINT(40.793713283563505 74.09007728373717) bank182175 +182176 POINT(40.133625978169974 73.74102189635344) bank182176 +182177 POINT(41.46010667383071 74.37273592940268) bank182177 +182178 POINT(40.51497568036264 73.58958913937546) bank182178 +182179 POINT(41.05415165645488 73.39841790515261) bank182179 +182180 POINT(40.96354747371208 74.78969301925119) bank182180 +182181 POINT(41.49384417916537 73.64799257815017) bank182181 +182182 POINT(41.58826043011271 73.50579670143304) bank182182 +182183 POINT(41.038039708815695 73.1466023209874) bank182183 +182184 POINT(40.36528841946758 73.93088373042264) bank182184 +182185 POINT(40.70460187499945 74.39458066308868) bank182185 +182186 POINT(40.71599076647789 74.68830732756332) bank182186 +182187 POINT(40.1954053036049 73.3333251814661) bank182187 +182188 POINT(41.57757311394882 74.10628784984968) bank182188 +182189 POINT(40.98182504685905 74.67287005175601) bank182189 +182190 POINT(40.36864097668243 74.69000864788795) bank182190 +182191 POINT(39.929155684813786 73.5628524804353) bank182191 +182192 POINT(40.519577664005375 73.29301817599215) bank182192 +182193 POINT(39.99089214739475 73.96783095093905) bank182193 +182194 POINT(41.1755825009724 74.8774321004577) bank182194 +182195 POINT(40.47193977953439 74.07057744272305) bank182195 +182196 POINT(39.81745196866133 73.7492543062634) bank182196 +182197 POINT(40.29067715468507 74.80765880242639) bank182197 +182198 POINT(40.912562247118714 73.09004721462811) bank182198 +182199 POINT(39.964885160665595 73.76191380900423) bank182199 +182200 POINT(40.035603259349266 74.73574844489168) bank182200 +182201 POINT(40.64693585427683 75.0028398462829) bank182201 +182202 POINT(40.90230703509432 73.05491020064737) bank182202 +182203 POINT(40.9392243332628 74.72096232675965) bank182203 +182204 POINT(40.85142780257844 74.20515355020075) bank182204 +182205 POINT(39.830011336178785 74.79707284269618) bank182205 +182206 POINT(40.04547329300992 73.38153950979495) bank182206 +182207 POINT(39.94630930970865 73.97200296649164) bank182207 +182208 POINT(41.54411132364455 74.3503736196138) bank182208 +182209 POINT(41.42456101415558 73.48408124919997) bank182209 +182210 POINT(40.82142844064107 73.50941767474097) bank182210 +182211 POINT(41.270925781018704 74.14491496062548) bank182211 +182212 POINT(40.47323955777111 73.53031187243197) bank182212 +182213 POINT(40.056160682468224 74.99210113531699) bank182213 +182214 POINT(41.44934588188198 73.71574089370266) bank182214 +182215 POINT(40.995547924421615 73.82962102542116) bank182215 +182216 POINT(39.75634599929571 73.20214957004669) bank182216 +182217 POINT(39.73768765794154 74.09642123573484) bank182217 +182218 POINT(40.61604375205962 74.06136904776442) bank182218 +182219 POINT(39.86994271512821 74.18716916938334) bank182219 +182220 POINT(41.50839469635052 74.50503560464608) bank182220 +182221 POINT(40.43222771911638 74.87676339991373) bank182221 +182222 POINT(40.609118148748884 73.26650463281139) bank182222 +182223 POINT(41.123671228360564 73.56938317852006) bank182223 +182224 POINT(39.86924427107936 74.71253701606692) bank182224 +182225 POINT(41.16320567545838 74.6291114478351) bank182225 +182226 POINT(40.41694637817776 73.95145307231866) bank182226 +182227 POINT(40.547869718349354 74.86919928724916) bank182227 +182228 POINT(40.85246588702312 74.22840682481481) bank182228 +182229 POINT(41.349978310588845 74.64885378316507) bank182229 +182230 POINT(39.950674001005495 73.79326071798793) bank182230 +182231 POINT(41.54225708593906 73.70299829304743) bank182231 +182232 POINT(40.60971842424152 74.4172222990472) bank182232 +182233 POINT(39.78643924699782 74.06293441075988) bank182233 +182234 POINT(40.51621628076247 73.85655073330119) bank182234 +182235 POINT(40.82733970064271 73.05666884615964) bank182235 +182236 POINT(40.57970334680249 73.29194880790077) bank182236 +182237 POINT(41.54083631672709 73.99220599287467) bank182237 +182238 POINT(41.006034149471276 74.73594222981566) bank182238 +182239 POINT(39.82424137469447 74.51359919672491) bank182239 +182240 POINT(39.99526495595814 74.17024915119993) bank182240 +182241 POINT(41.167697427955424 73.25130135364786) bank182241 +182242 POINT(39.77524259028394 74.1899644770664) bank182242 +182243 POINT(41.428498169074544 74.14776240446889) bank182243 +182244 POINT(40.17576564187797 74.40009184740401) bank182244 +182245 POINT(41.420305864141184 73.0469174123403) bank182245 +182246 POINT(41.24317943554633 74.18636202595304) bank182246 +182247 POINT(40.55148062078988 74.0405757783232) bank182247 +182248 POINT(41.10130368579789 73.9272708836117) bank182248 +182249 POINT(40.19245135805248 74.63003696351805) bank182249 +182250 POINT(41.66548120126105 73.01915600354862) bank182250 +182251 POINT(39.85866986351638 73.91287825981283) bank182251 +182252 POINT(39.85493976218687 74.42682503447577) bank182252 +182253 POINT(40.89241378021768 74.28278310413674) bank182253 +182254 POINT(40.17756114599184 74.69759159007464) bank182254 +182255 POINT(41.61678431787153 74.12222346471773) bank182255 +182256 POINT(39.759554328906866 74.3651166386047) bank182256 +182257 POINT(40.21189128425275 74.11670657027167) bank182257 +182258 POINT(41.03217472511314 74.14115555048018) bank182258 +182259 POINT(41.24533511817073 73.68975773984424) bank182259 +182260 POINT(41.63818520900212 74.62988160896892) bank182260 +182261 POINT(39.825502029296096 74.14627606119657) bank182261 +182262 POINT(40.013812078355244 73.87223481322299) bank182262 +182263 POINT(39.77317254777685 73.59905957733748) bank182263 +182264 POINT(40.95089527990622 73.59562396376218) bank182264 +182265 POINT(40.920524479820266 74.55818850717915) bank182265 +182266 POINT(41.305511386723076 74.53126859753498) bank182266 +182267 POINT(40.8661676750248 73.07855252080454) bank182267 +182268 POINT(40.32347428933411 74.64456715652582) bank182268 +182269 POINT(41.160118452032314 73.4734178969683) bank182269 +182270 POINT(40.750110447689735 73.24160351914603) bank182270 +182271 POINT(40.81775720407395 73.16176434814147) bank182271 +182272 POINT(40.482240986541434 73.07675156828455) bank182272 +182273 POINT(40.943085568178304 74.24639825261582) bank182273 +182274 POINT(40.27299068977277 73.48985712125778) bank182274 +182275 POINT(40.51954744207642 73.24669487772559) bank182275 +182276 POINT(40.34378367626739 73.42097373813691) bank182276 +182277 POINT(40.79089384249849 74.18479429938115) bank182277 +182278 POINT(41.689977989653066 74.51962520857043) bank182278 +182279 POINT(40.93211199797191 73.01035702360677) bank182279 +182280 POINT(40.4450754682118 73.06624742076664) bank182280 +182281 POINT(40.09581637709762 74.33711795717694) bank182281 +182282 POINT(40.011034530832134 73.95158033623306) bank182282 +182283 POINT(41.676925026732135 74.67284821421877) bank182283 +182284 POINT(41.26179760732025 73.38326546827824) bank182284 +182285 POINT(41.237751920002644 73.12302865857001) bank182285 +182286 POINT(39.84898481823372 74.76114999649833) bank182286 +182287 POINT(39.93939932426373 73.2406846945553) bank182287 +182288 POINT(40.534068069229704 74.96608480093529) bank182288 +182289 POINT(39.832698884961786 74.37288627714344) bank182289 +182290 POINT(39.898414754707446 74.15487416743788) bank182290 +182291 POINT(39.79355631551702 73.59333231312866) bank182291 +182292 POINT(40.35635563420711 74.91495465465715) bank182292 +182293 POINT(41.53613437356695 74.31613108977643) bank182293 +182294 POINT(41.41063282872521 74.82508030951587) bank182294 +182295 POINT(40.61872697112683 74.20833144182295) bank182295 +182296 POINT(39.88660802049216 74.67597340630962) bank182296 +182297 POINT(40.14996179117023 74.90081962348954) bank182297 +182298 POINT(40.89231245821393 73.94883353524924) bank182298 +182299 POINT(39.91930296535678 74.59405329546982) bank182299 +182300 POINT(40.41020150580489 74.09164880103721) bank182300 +182301 POINT(40.998114344350626 74.85907639064379) bank182301 +182302 POINT(40.72709522264298 73.46553509255298) bank182302 +182303 POINT(40.513573019954116 74.46218269350464) bank182303 +182304 POINT(41.52344622285299 73.4890125564048) bank182304 +182305 POINT(39.89253920557351 74.9343420660349) bank182305 +182306 POINT(40.53625424377973 74.07205072051643) bank182306 +182307 POINT(40.055247098391575 74.66047080058863) bank182307 +182308 POINT(40.940582108105474 73.65345745239307) bank182308 +182309 POINT(41.089141030806886 74.63322257215444) bank182309 +182310 POINT(40.588434951737504 74.44343280145401) bank182310 +182311 POINT(40.226171089451206 73.50194863465829) bank182311 +182312 POINT(41.570607205848766 74.1848674490402) bank182312 +182313 POINT(39.969837791681435 74.1574681636095) bank182313 +182314 POINT(41.56784705608345 74.1733672352177) bank182314 +182315 POINT(41.627199894832486 73.43767409531593) bank182315 +182316 POINT(40.6093524836016 73.84866592259931) bank182316 +182317 POINT(40.19088463112948 74.81574016375258) bank182317 +182318 POINT(40.6697346491716 74.66570514015051) bank182318 +182319 POINT(41.6782437036788 73.52053386955458) bank182319 +182320 POINT(40.72428130379592 74.8120836130542) bank182320 +182321 POINT(40.093895400288 74.81708494855603) bank182321 +182322 POINT(40.682620331639875 73.5998719937704) bank182322 +182323 POINT(40.63726714090666 73.32200403263286) bank182323 +182324 POINT(40.61422344138077 74.65102664873291) bank182324 +182325 POINT(40.602536762886736 74.50474435027112) bank182325 +182326 POINT(41.204823450384275 74.25985931384312) bank182326 +182327 POINT(41.51261974454906 74.96079935745513) bank182327 +182328 POINT(39.90056650165336 73.51249877585653) bank182328 +182329 POINT(41.518096417237665 73.22172102646421) bank182329 +182330 POINT(41.01067331978409 73.32497686961845) bank182330 +182331 POINT(40.184872316108546 75.0039090459103) bank182331 +182332 POINT(40.24741218028992 74.56497046859248) bank182332 +182333 POINT(40.38693943276406 74.31807608686125) bank182333 +182334 POINT(40.24510680990324 73.9741029765341) bank182334 +182335 POINT(40.96575019567435 74.19136254296524) bank182335 +182336 POINT(40.62073415170996 73.61189425396594) bank182336 +182337 POINT(40.485766640808656 73.38900433451404) bank182337 +182338 POINT(40.271776752655775 74.76593206511723) bank182338 +182339 POINT(41.06971638694493 73.45954328214864) bank182339 +182340 POINT(39.886700675970566 74.76057505041075) bank182340 +182341 POINT(40.18104848586009 74.37442098240842) bank182341 +182342 POINT(41.576811608435996 74.49337943717451) bank182342 +182343 POINT(41.69311430961374 73.08804057653937) bank182343 +182344 POINT(39.93928705503842 73.75493725483945) bank182344 +182345 POINT(40.818660135065365 73.51815686208403) bank182345 +182346 POINT(40.57192251812344 73.3726510336212) bank182346 +182347 POINT(40.352916395510675 74.78243259640898) bank182347 +182348 POINT(40.23810186772094 74.59060374033658) bank182348 +182349 POINT(41.122464945360434 74.1802082664407) bank182349 +182350 POINT(40.41434385031548 74.72593460160431) bank182350 +182351 POINT(40.51082017036184 73.53353360479514) bank182351 +182352 POINT(40.258882512209816 74.19916256528042) bank182352 +182353 POINT(39.89149771435155 73.74656269282718) bank182353 +182354 POINT(41.460251075412565 74.18939353847341) bank182354 +182355 POINT(41.0390195568536 73.56167571040514) bank182355 +182356 POINT(41.6072174462876 73.81583368305503) bank182356 +182357 POINT(40.98198730583518 73.87470325581138) bank182357 +182358 POINT(40.84048197564163 74.33482345766566) bank182358 +182359 POINT(40.916426702854565 73.67182022433315) bank182359 +182360 POINT(41.235868836682656 73.23987524578558) bank182360 +182361 POINT(40.30363534510859 73.24279095409518) bank182361 +182362 POINT(40.68570696816905 73.2863834553128) bank182362 +182363 POINT(41.144409372838396 73.84587641064901) bank182363 +182364 POINT(41.00141326163217 74.1356449342443) bank182364 +182365 POINT(41.458739466588746 74.9597299562051) bank182365 +182366 POINT(40.20728139170425 73.76747438159) bank182366 +182367 POINT(41.42054176086419 73.94508977548277) bank182367 +182368 POINT(41.3182132008202 74.40174696903921) bank182368 +182369 POINT(40.94435894988298 74.8792923417768) bank182369 +182370 POINT(41.285902493878396 73.8116116685815) bank182370 +182371 POINT(41.379539152510496 74.4460261784285) bank182371 +182372 POINT(40.08897417668937 74.26691873350177) bank182372 +182373 POINT(40.95598103363273 73.3940220839876) bank182373 +182374 POINT(39.97344434464745 73.6260935180118) bank182374 +182375 POINT(40.54826885406774 74.73972682892132) bank182375 +182376 POINT(40.93043077960879 74.1766311049322) bank182376 +182377 POINT(40.28896435205509 73.9712728241771) bank182377 +182378 POINT(40.29657614653633 74.31940704282559) bank182378 +182379 POINT(41.23751826815342 74.70012446742011) bank182379 +182380 POINT(41.109619518740615 74.47393444598539) bank182380 +182381 POINT(40.89609929401695 73.01591641831314) bank182381 +182382 POINT(41.38475683845741 73.38130676445235) bank182382 +182383 POINT(40.78725312502196 74.27319419983696) bank182383 +182384 POINT(41.178108623884135 73.34986746846454) bank182384 +182385 POINT(41.45247781557425 74.12716688778764) bank182385 +182386 POINT(40.55761875883144 73.73033391677247) bank182386 +182387 POINT(40.81112262520602 74.55632708775241) bank182387 +182388 POINT(40.72814877692704 73.41017313937596) bank182388 +182389 POINT(40.62321887365239 74.81462125124537) bank182389 +182390 POINT(40.56232772253409 73.16011725411369) bank182390 +182391 POINT(39.78084260369656 74.34966033155415) bank182391 +182392 POINT(41.49475302488176 74.23243391650179) bank182392 +182393 POINT(41.43820172487815 73.92228962110816) bank182393 +182394 POINT(39.9725117651083 73.84238148633787) bank182394 +182395 POINT(41.65369752109762 73.37484388414347) bank182395 +182396 POINT(40.87280484153882 73.02377928077473) bank182396 +182397 POINT(40.90654845370096 73.63282658210971) bank182397 +182398 POINT(39.817470049239006 73.57853112448227) bank182398 +182399 POINT(39.88644845993949 73.42746611395053) bank182399 +182400 POINT(40.741981180938275 73.45505107076531) bank182400 +182401 POINT(39.971005668819714 73.12232900787097) bank182401 +182402 POINT(39.857300109584045 74.48186546347237) bank182402 +182403 POINT(41.178835461068864 74.49708976439565) bank182403 +182404 POINT(41.13028568586393 74.30276586332842) bank182404 +182405 POINT(40.92919306827394 73.74719994558325) bank182405 +182406 POINT(41.345550447062394 74.88970874971344) bank182406 +182407 POINT(40.67904280549163 73.87848465947678) bank182407 +182408 POINT(40.96920116482478 73.65625405334947) bank182408 +182409 POINT(41.22139664967892 73.20261989154851) bank182409 +182410 POINT(40.40740018082082 73.64190855752152) bank182410 +182411 POINT(40.48201356866622 74.04150028944079) bank182411 +182412 POINT(40.62792933463768 74.3411246890646) bank182412 +182413 POINT(40.88271338389738 74.79102960011647) bank182413 +182414 POINT(41.301467046575254 74.90465022042689) bank182414 +182415 POINT(39.81446504828193 73.05508691807219) bank182415 +182416 POINT(41.48300529977146 74.79596774159707) bank182416 +182417 POINT(40.41484436202988 74.79070171058001) bank182417 +182418 POINT(40.181961932484484 74.52630672334404) bank182418 +182419 POINT(40.74696261268762 74.25231399599504) bank182419 +182420 POINT(40.15898696116309 74.05968039193567) bank182420 +182421 POINT(40.40167302557858 74.88417050981387) bank182421 +182422 POINT(40.5782097351489 73.50249911311336) bank182422 +182423 POINT(40.16035740920585 74.14579664547009) bank182423 +182424 POINT(40.21531693449737 73.01440776909295) bank182424 +182425 POINT(40.99362836052835 73.60852610873808) bank182425 +182426 POINT(41.4992613019662 73.59291875537959) bank182426 +182427 POINT(40.78469038572451 74.55979222323154) bank182427 +182428 POINT(40.72170498318804 73.69696435978476) bank182428 +182429 POINT(41.52140424438221 73.09010222565246) bank182429 +182430 POINT(40.30849350088398 73.34024022058638) bank182430 +182431 POINT(39.90672963499627 73.06566488635028) bank182431 +182432 POINT(40.00682332041985 74.41188790163605) bank182432 +182433 POINT(41.19981224977719 74.20750061668956) bank182433 +182434 POINT(39.99556539113652 73.77534687432313) bank182434 +182435 POINT(41.65388075372873 74.50612170206713) bank182435 +182436 POINT(39.74737236015609 74.8252688984817) bank182436 +182437 POINT(39.75769264212849 73.35709604292818) bank182437 +182438 POINT(41.17386766922581 74.85843274004607) bank182438 +182439 POINT(40.32859721329188 74.32287249554905) bank182439 +182440 POINT(40.53341760720135 74.57043669573507) bank182440 +182441 POINT(40.86727005871117 73.15536076725459) bank182441 +182442 POINT(41.03644561702402 73.57959250821447) bank182442 +182443 POINT(40.490834919188664 74.960330714791) bank182443 +182444 POINT(40.290130588013085 74.40804744510311) bank182444 +182445 POINT(41.18749356841885 73.60206161399401) bank182445 +182446 POINT(40.366963780806444 73.45589252047978) bank182446 +182447 POINT(41.03557250594162 74.97521299209177) bank182447 +182448 POINT(41.651479435607115 73.43289617821503) bank182448 +182449 POINT(41.71062811667266 73.41993498182987) bank182449 +182450 POINT(40.65503443553995 73.96024900626894) bank182450 +182451 POINT(41.03687011158698 74.83176430997916) bank182451 +182452 POINT(41.10569618573885 74.82964842504471) bank182452 +182453 POINT(40.53590633180341 73.96064237198429) bank182453 +182454 POINT(41.63239668152475 73.51811197272833) bank182454 +182455 POINT(41.56710633743133 73.02343234690787) bank182455 +182456 POINT(39.919118571713156 73.99182820678533) bank182456 +182457 POINT(41.50425296623772 74.21009887707334) bank182457 +182458 POINT(40.206078312328515 74.2971728220609) bank182458 +182459 POINT(40.1982657482555 74.48999270188826) bank182459 +182460 POINT(40.80616428104822 74.53247383902948) bank182460 +182461 POINT(41.70090246390002 73.2475950959893) bank182461 +182462 POINT(40.55719306875632 74.15645274166718) bank182462 +182463 POINT(41.570271033188924 74.60105406137929) bank182463 +182464 POINT(39.98523573593534 73.9880923918847) bank182464 +182465 POINT(40.958035893495165 73.89342442975584) bank182465 +182466 POINT(41.62830231328194 74.55435498853056) bank182466 +182467 POINT(39.9635834881859 73.72600056646071) bank182467 +182468 POINT(40.77917088509927 73.69715011621923) bank182468 +182469 POINT(40.12061831615558 74.95987364104523) bank182469 +182470 POINT(40.77558898647603 73.500679836403) bank182470 +182471 POINT(40.20976488857425 73.29858434949568) bank182471 +182472 POINT(41.213532695713724 74.2184558255329) bank182472 +182473 POINT(40.70584109813235 73.46256638576912) bank182473 +182474 POINT(41.706110604329176 73.58036358570169) bank182474 +182475 POINT(41.12393708413295 74.15241271726185) bank182475 +182476 POINT(40.998713575037016 73.70493807879497) bank182476 +182477 POINT(41.623743632399474 73.49899033105741) bank182477 +182478 POINT(41.33790494908318 74.97263699886828) bank182478 +182479 POINT(40.6762500268399 73.57909797569289) bank182479 +182480 POINT(40.0643917231674 73.68242926262944) bank182480 +182481 POINT(40.07263526621309 74.09542823008678) bank182481 +182482 POINT(41.359732554836796 74.31968129221656) bank182482 +182483 POINT(39.90961110684319 74.34112629211565) bank182483 +182484 POINT(40.710085967868125 73.27545489444253) bank182484 +182485 POINT(40.93629006530673 73.45783943700508) bank182485 +182486 POINT(39.89773655324421 74.02612791870419) bank182486 +182487 POINT(40.807199247275356 74.64900602637383) bank182487 +182488 POINT(41.32508388527561 74.43489396552064) bank182488 +182489 POINT(40.78591370975248 73.86948785929636) bank182489 +182490 POINT(40.05402011756489 74.94120202676744) bank182490 +182491 POINT(40.30348716009527 75.00051007499168) bank182491 +182492 POINT(40.14031414370739 74.77616349806843) bank182492 +182493 POINT(40.375001049857104 73.6325750154011) bank182493 +182494 POINT(40.70388512823591 73.17805447525674) bank182494 +182495 POINT(41.161892372460066 73.32104609172335) bank182495 +182496 POINT(40.907429452843054 74.9716435787108) bank182496 +182497 POINT(40.0911955935718 74.97711017873839) bank182497 +182498 POINT(39.86198291029601 74.08869822281295) bank182498 +182499 POINT(40.4227598316864 74.46777796605332) bank182499 +182500 POINT(39.950894495066265 73.80649738057068) bank182500 +182501 POINT(41.400029665780984 73.04577640342903) bank182501 +182502 POINT(40.99501616343634 74.89898490429962) bank182502 +182503 POINT(39.769856992031855 73.28011130413462) bank182503 +182504 POINT(40.13014464517448 73.11660779425631) bank182504 +182505 POINT(40.88938492671101 73.32823394567218) bank182505 +182506 POINT(41.327227138054084 73.5183848499278) bank182506 +182507 POINT(39.77699832064284 74.25021021249405) bank182507 +182508 POINT(40.419257636399095 73.94049470016287) bank182508 +182509 POINT(39.77500213405406 74.49839147211055) bank182509 +182510 POINT(40.71155223841889 73.50646633881249) bank182510 +182511 POINT(40.57090040230305 74.70326156276411) bank182511 +182512 POINT(41.171159205424956 73.68901182339997) bank182512 +182513 POINT(40.06824744662147 74.02998212737224) bank182513 +182514 POINT(40.351684206481906 73.01321699972716) bank182514 +182515 POINT(39.937789798270245 74.88350731928348) bank182515 +182516 POINT(40.586134557342874 74.35016603412129) bank182516 +182517 POINT(40.10529014979134 73.52378430786702) bank182517 +182518 POINT(41.6898925655406 73.29626324524473) bank182518 +182519 POINT(40.289379753022594 73.53635021052689) bank182519 +182520 POINT(41.48637835931077 73.68076584854063) bank182520 +182521 POINT(41.40412537175753 73.88174115096365) bank182521 +182522 POINT(40.518156242216 74.35289631547401) bank182522 +182523 POINT(40.19917345473452 74.00072439322064) bank182523 +182524 POINT(41.10777342425966 74.54119416614226) bank182524 +182525 POINT(40.8115609427474 74.21260530227408) bank182525 +182526 POINT(40.17898129824769 74.11393777244278) bank182526 +182527 POINT(41.371082891777064 73.73833658364758) bank182527 +182528 POINT(39.80023348851823 73.20524566953382) bank182528 +182529 POINT(40.372110413399234 74.4558522391846) bank182529 +182530 POINT(39.8241833759583 73.80903292111687) bank182530 +182531 POINT(40.75822218027801 73.6388809183995) bank182531 +182532 POINT(41.36137236239706 73.84144319841005) bank182532 +182533 POINT(40.25186471891204 73.65540933692328) bank182533 +182534 POINT(39.8509711803455 73.63324689744238) bank182534 +182535 POINT(40.65208512478433 73.48076719920559) bank182535 +182536 POINT(41.69233877488728 74.70997544153838) bank182536 +182537 POINT(40.168647996562065 73.23578878980055) bank182537 +182538 POINT(41.00818030568639 74.64407869558345) bank182538 +182539 POINT(40.710955998533684 73.69180110877363) bank182539 +182540 POINT(41.42020498435443 74.8058484246889) bank182540 +182541 POINT(41.14646163974218 74.2844875201085) bank182541 +182542 POINT(40.46757445133689 74.97021578664557) bank182542 +182543 POINT(40.078076262719925 73.1737612527524) bank182543 +182544 POINT(40.20068608171575 73.4638169354616) bank182544 +182545 POINT(41.23858971916251 73.04698515326326) bank182545 +182546 POINT(39.75684179540156 73.02599063748684) bank182546 +182547 POINT(40.926836340220596 74.07428831836565) bank182547 +182548 POINT(40.932509939705085 74.52843412168636) bank182548 +182549 POINT(40.75341372668875 74.02927131848683) bank182549 +182550 POINT(41.41457359947393 74.80824971480386) bank182550 +182551 POINT(40.79463515530875 73.3387730737447) bank182551 +182552 POINT(40.18036467592415 73.58817618632068) bank182552 +182553 POINT(40.06667897573784 73.13575914198567) bank182553 +182554 POINT(41.43543073792271 73.50618761595635) bank182554 +182555 POINT(41.599470673823774 74.32686034383556) bank182555 +182556 POINT(40.10448775703348 74.4033905765068) bank182556 +182557 POINT(40.49356490939476 74.28602291770683) bank182557 +182558 POINT(40.78333470903215 74.7295408494173) bank182558 +182559 POINT(40.312487936842544 73.20899975057682) bank182559 +182560 POINT(39.890423736330966 74.12520932049) bank182560 +182561 POINT(40.67909650832523 74.10167878161077) bank182561 +182562 POINT(41.30135953520958 73.27459451863146) bank182562 +182563 POINT(40.119314149784 73.89310094928922) bank182563 +182564 POINT(41.53566422515306 74.27459460307752) bank182564 +182565 POINT(41.493723012874554 74.63025921342226) bank182565 +182566 POINT(41.26527782744581 73.71702124340058) bank182566 +182567 POINT(40.77814071681934 73.77722957328659) bank182567 +182568 POINT(40.16445440294773 74.1068884165583) bank182568 +182569 POINT(40.8950033524729 73.20705414531837) bank182569 +182570 POINT(41.54887009379435 74.71204295189268) bank182570 +182571 POINT(39.714099364831114 73.82390117574465) bank182571 +182572 POINT(40.623332156243364 73.06606617440853) bank182572 +182573 POINT(40.666517093995026 73.29177479558648) bank182573 +182574 POINT(41.56536228180695 74.1062093142643) bank182574 +182575 POINT(41.4812622459133 73.64016816083797) bank182575 +182576 POINT(40.384684153750165 74.37526996226806) bank182576 +182577 POINT(40.51604042689281 74.48968614394252) bank182577 +182578 POINT(41.18437382654484 73.98961165690005) bank182578 +182579 POINT(39.88924554457525 74.9910487801652) bank182579 +182580 POINT(41.56265401684853 73.97644606207194) bank182580 +182581 POINT(41.272786090341874 73.9844250115594) bank182581 +182582 POINT(40.926320396662575 74.45656367641475) bank182582 +182583 POINT(39.80708142664695 74.72058951059239) bank182583 +182584 POINT(39.81433919802985 74.27851755629761) bank182584 +182585 POINT(40.59033215229059 73.52058444763753) bank182585 +182586 POINT(39.95801517308403 74.0252176464879) bank182586 +182587 POINT(40.13386940886409 74.51949065375152) bank182587 +182588 POINT(41.191093541184216 74.89183736714988) bank182588 +182589 POINT(39.814681356373235 73.30726609252889) bank182589 +182590 POINT(41.05844751138905 73.50691284754618) bank182590 +182591 POINT(40.08469422161083 74.4899547590004) bank182591 +182592 POINT(40.03976520449159 73.5567784567311) bank182592 +182593 POINT(41.37960836807194 73.71786154522609) bank182593 +182594 POINT(41.09682353691811 74.44702414975644) bank182594 +182595 POINT(41.00896423730833 73.49614956052775) bank182595 +182596 POINT(40.606686039398866 73.66344648865145) bank182596 +182597 POINT(40.400428261424466 73.82450098435136) bank182597 +182598 POINT(41.34777761768755 74.80412172024263) bank182598 +182599 POINT(41.50385258605803 74.10966049116351) bank182599 +182600 POINT(40.66177895797615 73.9695469654823) bank182600 +182601 POINT(41.576464992231706 73.71167723316852) bank182601 +182602 POINT(41.30049619601767 73.30483977773494) bank182602 +182603 POINT(40.138556307633635 74.22904244878717) bank182603 +182604 POINT(41.70080212977525 74.10470228364247) bank182604 +182605 POINT(40.162979443393475 74.54657443348005) bank182605 +182606 POINT(40.845614010215066 73.51662197675574) bank182606 +182607 POINT(40.05093365168183 74.6744272511337) bank182607 +182608 POINT(41.501081807415 74.16959675848842) bank182608 +182609 POINT(39.99069887749288 74.23261049805242) bank182609 +182610 POINT(40.888430559202426 74.52180211369522) bank182610 +182611 POINT(40.51605130726326 74.09400864977712) bank182611 +182612 POINT(39.82405395185392 73.57443524357821) bank182612 +182613 POINT(40.360032172507125 74.42533772300477) bank182613 +182614 POINT(40.18506525279032 74.00742208962428) bank182614 +182615 POINT(41.557422579382624 73.14958780790916) bank182615 +182616 POINT(41.42689743735117 73.85239358071598) bank182616 +182617 POINT(40.13083441483567 73.71596646015298) bank182617 +182618 POINT(40.200430663468154 74.63712644132349) bank182618 +182619 POINT(40.339770700046934 73.9739032823604) bank182619 +182620 POINT(40.25444831963326 74.7910469296853) bank182620 +182621 POINT(40.820578364810565 74.84679388812177) bank182621 +182622 POINT(41.608423420718594 74.44341056450368) bank182622 +182623 POINT(41.624023890309715 74.44490606344527) bank182623 +182624 POINT(40.11626696899612 74.22993715609911) bank182624 +182625 POINT(39.783819639504465 74.07622927341151) bank182625 +182626 POINT(41.08643535928533 74.28917687654162) bank182626 +182627 POINT(41.388803534724495 73.82460179665993) bank182627 +182628 POINT(41.3167912543677 74.30719944023183) bank182628 +182629 POINT(40.52148278819692 73.54916253639017) bank182629 +182630 POINT(41.20445045660786 73.14410410210928) bank182630 +182631 POINT(40.47397715675475 73.82027601837616) bank182631 +182632 POINT(40.916329782289175 73.13529510648189) bank182632 +182633 POINT(40.291395873496256 73.50019399853196) bank182633 +182634 POINT(40.55558431373647 74.80350613550452) bank182634 +182635 POINT(40.15389128594742 74.93289189038167) bank182635 +182636 POINT(39.752449733750296 73.5343544299375) bank182636 +182637 POINT(40.3049806480404 73.36449423541512) bank182637 +182638 POINT(40.736915972066 74.74676472924808) bank182638 +182639 POINT(41.16027706713523 73.12253824052813) bank182639 +182640 POINT(40.6009814216248 73.25027098133498) bank182640 +182641 POINT(41.47177095418063 74.47698853754493) bank182641 +182642 POINT(40.350850850933845 74.09721069563591) bank182642 +182643 POINT(41.35316796334742 73.55543203139668) bank182643 +182644 POINT(39.95256406867064 74.67469288625603) bank182644 +182645 POINT(41.6913638827387 74.10618991697106) bank182645 +182646 POINT(40.576880141794206 74.87720966218929) bank182646 +182647 POINT(41.303735457413225 73.23075579390951) bank182647 +182648 POINT(41.1521974976124 73.98863681929792) bank182648 +182649 POINT(39.966771622521 74.73271358332592) bank182649 +182650 POINT(41.37874225240547 74.67844796922638) bank182650 +182651 POINT(41.264050734853086 74.78574747800894) bank182651 +182652 POINT(40.375696438546214 73.9899607123757) bank182652 +182653 POINT(41.04974316168723 73.81922209983846) bank182653 +182654 POINT(41.39726254767685 73.31525626767835) bank182654 +182655 POINT(40.40675689478073 73.36359740741901) bank182655 +182656 POINT(41.19816522657998 74.43145533243928) bank182656 +182657 POINT(40.93082037801029 73.04387952663316) bank182657 +182658 POINT(40.30857566830852 74.08081737060222) bank182658 +182659 POINT(40.32699988208187 73.62117254676953) bank182659 +182660 POINT(41.19898759816856 74.07709680664959) bank182660 +182661 POINT(39.834482528135524 74.66130941518131) bank182661 +182662 POINT(39.968248087186275 73.84374180422729) bank182662 +182663 POINT(40.27417723037498 73.49523068409356) bank182663 +182664 POINT(41.6107997954138 74.35346512665754) bank182664 +182665 POINT(40.538649975181215 74.76125923312114) bank182665 +182666 POINT(40.626854865902615 73.60057668427419) bank182666 +182667 POINT(40.177672483226935 74.1571277433448) bank182667 +182668 POINT(40.84031450240583 73.5323716598731) bank182668 +182669 POINT(41.109209636693365 74.35920648011687) bank182669 +182670 POINT(41.64055087525254 74.6946739621313) bank182670 +182671 POINT(40.743834295066186 73.59805909801581) bank182671 +182672 POINT(40.92530387800429 73.0652880425747) bank182672 +182673 POINT(40.87333794058605 73.4883921684435) bank182673 +182674 POINT(41.051359733544615 74.9646747417599) bank182674 +182675 POINT(40.813408420057655 74.67289935293955) bank182675 +182676 POINT(41.239289015830416 73.70589931694394) bank182676 +182677 POINT(39.944448350796854 74.51946035474569) bank182677 +182678 POINT(40.6740073211603 74.41492181328017) bank182678 +182679 POINT(39.89842597843983 73.41379525461329) bank182679 +182680 POINT(39.95952912077089 74.87958605788026) bank182680 +182681 POINT(41.320725470734025 73.03990664092468) bank182681 +182682 POINT(40.88178249573504 73.95338590563792) bank182682 +182683 POINT(40.73692960269539 73.46192242647952) bank182683 +182684 POINT(40.15430752142563 73.59359160830593) bank182684 +182685 POINT(40.133450320851075 74.19055538128755) bank182685 +182686 POINT(41.641305934159426 73.42351263007929) bank182686 +182687 POINT(41.196398951574785 74.57775618165952) bank182687 +182688 POINT(40.29079750028099 73.29656385914306) bank182688 +182689 POINT(41.433580089311405 74.36986546928387) bank182689 +182690 POINT(40.929378047292275 74.40592648187301) bank182690 +182691 POINT(39.79917302706737 74.7947265672639) bank182691 +182692 POINT(40.83285662805343 74.0672309622401) bank182692 +182693 POINT(41.602773327299175 74.47771651727972) bank182693 +182694 POINT(41.42512479834461 73.21199425240636) bank182694 +182695 POINT(40.36986872776873 73.63575485474443) bank182695 +182696 POINT(40.73658597585525 74.42932084991563) bank182696 +182697 POINT(40.820220521323776 74.18532205904111) bank182697 +182698 POINT(41.06512448341067 74.06153974756441) bank182698 +182699 POINT(41.241030597847214 74.03225108048824) bank182699 +182700 POINT(41.49427979607321 74.40331004090714) bank182700 +182701 POINT(40.816901273624175 74.86765456788288) bank182701 +182702 POINT(40.45680169162258 74.7684873931948) bank182702 +182703 POINT(41.6913789535539 73.86173629457753) bank182703 +182704 POINT(40.80687461184492 74.61700623193445) bank182704 +182705 POINT(40.89431176063499 73.23769559016091) bank182705 +182706 POINT(40.91676310260947 74.27891088547396) bank182706 +182707 POINT(41.191548692086585 74.35639423065606) bank182707 +182708 POINT(40.696937287755354 73.0789127249718) bank182708 +182709 POINT(41.50978662456548 74.31187953024894) bank182709 +182710 POINT(40.82883296184015 74.87835551387415) bank182710 +182711 POINT(41.47692010400186 74.96000986592345) bank182711 +182712 POINT(40.23588750845525 73.23381890070594) bank182712 +182713 POINT(41.422871269021606 74.70798802351035) bank182713 +182714 POINT(40.124797179797916 74.66444216557598) bank182714 +182715 POINT(41.075488736061686 73.75054620323579) bank182715 +182716 POINT(40.25952057619507 73.32162391354339) bank182716 +182717 POINT(40.449362310435546 74.67015819260233) bank182717 +182718 POINT(40.78754628698821 74.39427966645069) bank182718 +182719 POINT(40.44987856796991 73.79112625712128) bank182719 +182720 POINT(40.839311580059636 73.11842105146394) bank182720 +182721 POINT(40.61048525422154 73.68026028552532) bank182721 +182722 POINT(41.073349817413316 74.47734286352564) bank182722 +182723 POINT(40.183914515442154 74.47462026820222) bank182723 +182724 POINT(41.499752243714944 73.78105528989194) bank182724 +182725 POINT(40.68046145116771 74.8702312057649) bank182725 +182726 POINT(40.60559456050924 74.75453134208244) bank182726 +182727 POINT(41.384958446122695 74.04653824585344) bank182727 +182728 POINT(40.520954746338326 73.45607612467313) bank182728 +182729 POINT(41.199185372420104 73.1979906854961) bank182729 +182730 POINT(41.611617608168046 73.98248399169553) bank182730 +182731 POINT(39.88740523875481 74.74797827797134) bank182731 +182732 POINT(41.4698548722096 73.1190063616882) bank182732 +182733 POINT(40.464058768209476 73.50089782033375) bank182733 +182734 POINT(40.90293827374398 73.77493415290341) bank182734 +182735 POINT(40.07088471140077 74.12296637974114) bank182735 +182736 POINT(41.57432380178023 74.74751497628122) bank182736 +182737 POINT(40.23694231783703 73.56054874504535) bank182737 +182738 POINT(41.36563639152018 74.69544456526054) bank182738 +182739 POINT(41.37525172620726 74.74194436148703) bank182739 +182740 POINT(40.162155435706296 73.06741349545476) bank182740 +182741 POINT(39.979106475326404 74.57154361586989) bank182741 +182742 POINT(40.40366667132232 74.39701300507149) bank182742 +182743 POINT(40.98079608136153 74.53076039797033) bank182743 +182744 POINT(40.60314106324928 73.88551157051734) bank182744 +182745 POINT(40.00899772324571 74.59654314671708) bank182745 +182746 POINT(41.57422648224353 74.69526598285687) bank182746 +182747 POINT(41.138235633077656 74.01827107599978) bank182747 +182748 POINT(41.32987599226696 73.31914818114409) bank182748 +182749 POINT(40.06075133826996 73.47175010921436) bank182749 +182750 POINT(40.67025080998713 73.63575331800821) bank182750 +182751 POINT(40.65607952584917 73.2060624580257) bank182751 +182752 POINT(41.09121022441832 74.66975911466639) bank182752 +182753 POINT(40.6108983325622 73.4010876760326) bank182753 +182754 POINT(41.39830789287743 73.84335575752397) bank182754 +182755 POINT(41.25610885231873 74.60739367168742) bank182755 +182756 POINT(41.19875526871887 73.19544736515213) bank182756 +182757 POINT(41.06762313628137 74.98428199604132) bank182757 +182758 POINT(41.05458722671447 74.32485327316235) bank182758 +182759 POINT(40.96019887205048 73.02937696876907) bank182759 +182760 POINT(41.60472815433018 74.7739725456656) bank182760 +182761 POINT(40.05557147298295 74.4642116646636) bank182761 +182762 POINT(40.81032778613862 73.39654026879774) bank182762 +182763 POINT(41.332520978044506 73.31488290423849) bank182763 +182764 POINT(40.01288441905619 73.98570560447062) bank182764 +182765 POINT(41.38247614312778 74.64482443711736) bank182765 +182766 POINT(41.00322039421702 74.86007626308303) bank182766 +182767 POINT(40.63890558476636 73.95469939367243) bank182767 +182768 POINT(40.32394528175217 73.89647304463224) bank182768 +182769 POINT(41.254594982311204 74.81077954089717) bank182769 +182770 POINT(41.384075675136394 74.9599937691691) bank182770 +182771 POINT(40.1290793511 73.27911434704984) bank182771 +182772 POINT(40.345868313981825 73.48267093793474) bank182772 +182773 POINT(39.7986110781825 73.40692823490853) bank182773 +182774 POINT(40.915710933312376 73.66009514091343) bank182774 +182775 POINT(40.347846661756975 74.0720918369736) bank182775 +182776 POINT(41.363899381207716 73.23006237751261) bank182776 +182777 POINT(40.868764101554525 73.26524839495679) bank182777 +182778 POINT(40.107646367384234 74.1475452160614) bank182778 +182779 POINT(40.72010052885432 74.9582926831137) bank182779 +182780 POINT(39.83065738701467 74.70510634501429) bank182780 +182781 POINT(39.816289155527556 74.33353498945146) bank182781 +182782 POINT(39.72727418890718 73.79520146291853) bank182782 +182783 POINT(41.371867050568056 74.26003063395002) bank182783 +182784 POINT(40.775593182202584 74.78761659230032) bank182784 +182785 POINT(41.28365631182262 73.45845427185347) bank182785 +182786 POINT(39.910300759727996 73.21828657412674) bank182786 +182787 POINT(40.70682511038093 73.23472889602323) bank182787 +182788 POINT(41.57985595412941 74.86332418602285) bank182788 +182789 POINT(39.86045940223169 74.60647057521035) bank182789 +182790 POINT(40.20329824166675 74.58175245792256) bank182790 +182791 POINT(41.26446853849006 74.74291818787499) bank182791 +182792 POINT(41.61018712517203 74.82084789520611) bank182792 +182793 POINT(40.83216413074561 73.43118312552005) bank182793 +182794 POINT(40.67812887564382 74.94825410089483) bank182794 +182795 POINT(41.37701975973412 74.8312711300476) bank182795 +182796 POINT(40.23239633654736 73.59001801303761) bank182796 +182797 POINT(40.48778753422498 74.39430307847103) bank182797 +182798 POINT(40.87212343773968 74.51446788849118) bank182798 +182799 POINT(41.02849950482312 74.27255937371487) bank182799 +182800 POINT(40.51684590010949 73.26825460134813) bank182800 +182801 POINT(41.66679332349334 74.31647701622467) bank182801 +182802 POINT(41.27683043390508 73.34903312468828) bank182802 +182803 POINT(40.08614739089015 74.0426787615721) bank182803 +182804 POINT(40.73768526848503 73.6988053145315) bank182804 +182805 POINT(41.4899154578594 73.62061022582147) bank182805 +182806 POINT(40.80977784567889 74.0660683642823) bank182806 +182807 POINT(41.66350342811481 74.14237650044444) bank182807 +182808 POINT(41.07636101792083 73.3839326590422) bank182808 +182809 POINT(39.722644537392924 74.76816622132739) bank182809 +182810 POINT(41.140581032321066 73.67620824718509) bank182810 +182811 POINT(40.409655908128514 73.22916692876272) bank182811 +182812 POINT(39.88307194176272 74.41287741886076) bank182812 +182813 POINT(40.65925096068868 74.52844345560858) bank182813 +182814 POINT(41.29183526038897 74.2677283536856) bank182814 +182815 POINT(41.570181634923955 74.6087641556929) bank182815 +182816 POINT(40.59131329268969 73.83933334665807) bank182816 +182817 POINT(40.075627908115656 73.44876326902693) bank182817 +182818 POINT(39.96597779757991 73.83608043135067) bank182818 +182819 POINT(40.306678117816496 73.19161015317233) bank182819 +182820 POINT(40.93279621293414 74.62825183028795) bank182820 +182821 POINT(41.20804819327658 73.14555552016347) bank182821 +182822 POINT(40.47167708030523 73.84003320058382) bank182822 +182823 POINT(41.452810369339176 73.36768358995286) bank182823 +182824 POINT(41.70420860773572 73.01250789461669) bank182824 +182825 POINT(40.81134180029303 73.33621385791002) bank182825 +182826 POINT(41.39012334741694 73.76735153169935) bank182826 +182827 POINT(40.743644235461396 73.04543044740332) bank182827 +182828 POINT(40.022920872244796 73.15392485475685) bank182828 +182829 POINT(39.914235382240726 73.74975834084299) bank182829 +182830 POINT(41.25997982833716 73.40889811452593) bank182830 +182831 POINT(41.453760477938204 74.91676345251538) bank182831 +182832 POINT(41.19239168065835 73.49093443992234) bank182832 +182833 POINT(41.36939062985441 73.30215270292251) bank182833 +182834 POINT(40.06504393910096 74.11477937260872) bank182834 +182835 POINT(40.799845776938845 73.62108410222186) bank182835 +182836 POINT(40.140972263794325 74.60490890241009) bank182836 +182837 POINT(40.55912286945656 73.0334672928796) bank182837 +182838 POINT(41.1836196787886 74.1113025372492) bank182838 +182839 POINT(39.808351489189654 74.92343386786847) bank182839 +182840 POINT(40.950128430598745 74.65939261562217) bank182840 +182841 POINT(40.816998738270605 74.36579572302924) bank182841 +182842 POINT(41.34838360712646 73.70567004193214) bank182842 +182843 POINT(39.84864679926538 73.49914984531682) bank182843 +182844 POINT(40.615993509411744 74.04676464779884) bank182844 +182845 POINT(41.33288738381809 73.03084109902531) bank182845 +182846 POINT(41.6656541991204 74.39524526635336) bank182846 +182847 POINT(41.506698017920314 73.48097980141652) bank182847 +182848 POINT(41.374213804492875 74.49364130825535) bank182848 +182849 POINT(40.47591743405563 74.1004419742747) bank182849 +182850 POINT(40.07816066377882 74.48020088376921) bank182850 +182851 POINT(41.17986437557167 73.10178932054406) bank182851 +182852 POINT(40.69654231942368 74.82277712244125) bank182852 +182853 POINT(40.650761252611424 73.77971881222422) bank182853 +182854 POINT(41.485661361879885 73.56467919039619) bank182854 +182855 POINT(39.771897881501815 73.3842140984683) bank182855 +182856 POINT(41.42608777766169 74.12021797097003) bank182856 +182857 POINT(40.32758986186344 73.52124234339725) bank182857 +182858 POINT(40.58592698140763 74.22469120933688) bank182858 +182859 POINT(40.06519382351654 73.19334877678304) bank182859 +182860 POINT(40.83823710707921 73.81160893518336) bank182860 +182861 POINT(40.42377621470036 73.66563969338374) bank182861 +182862 POINT(40.969430053164814 73.68620816980426) bank182862 +182863 POINT(39.90919574201218 73.1257406419627) bank182863 +182864 POINT(40.82024605666045 74.79850823144763) bank182864 +182865 POINT(41.396452349026795 73.26421266361271) bank182865 +182866 POINT(39.90930029737006 74.85831921677908) bank182866 +182867 POINT(40.110354881170004 73.8203672237932) bank182867 +182868 POINT(41.518576824727916 73.5235637753766) bank182868 +182869 POINT(40.65920973276362 73.81274588223971) bank182869 +182870 POINT(39.83978799555766 74.3188258031376) bank182870 +182871 POINT(40.210827429732134 73.69931714138488) bank182871 +182872 POINT(41.670669730343434 74.72270168625673) bank182872 +182873 POINT(39.80567592736799 73.52891605862779) bank182873 +182874 POINT(41.041030829008726 73.99723497377724) bank182874 +182875 POINT(40.185437844940914 74.78593664812703) bank182875 +182876 POINT(40.12897223043983 74.11585647509138) bank182876 +182877 POINT(40.444874658335934 74.36688446788435) bank182877 +182878 POINT(40.52740185741212 73.21149036610038) bank182878 +182879 POINT(40.881488312916915 73.76834182956658) bank182879 +182880 POINT(41.40888964917314 73.31930795650558) bank182880 +182881 POINT(41.649899089382316 74.23082964751342) bank182881 +182882 POINT(40.636428836720995 74.89814216325306) bank182882 +182883 POINT(40.93083709095773 73.4439183232885) bank182883 +182884 POINT(40.53016840067928 74.19305064482054) bank182884 +182885 POINT(41.246433377580054 74.73197238051877) bank182885 +182886 POINT(39.78926089959759 74.85577842219152) bank182886 +182887 POINT(40.4860559793206 74.94618392023796) bank182887 +182888 POINT(39.72106846892591 73.37142710530615) bank182888 +182889 POINT(41.70826813451666 73.3463604359849) bank182889 +182890 POINT(40.82217065766253 74.26949731543674) bank182890 +182891 POINT(41.494728013025174 74.76043007571005) bank182891 +182892 POINT(41.60367769221927 74.6653558561565) bank182892 +182893 POINT(40.10757857682455 74.42815717491301) bank182893 +182894 POINT(40.76053854709692 73.7873760058985) bank182894 +182895 POINT(40.730951356338366 73.51659906819383) bank182895 +182896 POINT(39.948491230134856 73.02473824535367) bank182896 +182897 POINT(41.62808853651065 73.98864780371602) bank182897 +182898 POINT(40.15095697745491 74.48772499197017) bank182898 +182899 POINT(40.40831453489046 73.29161015060266) bank182899 +182900 POINT(41.34683361793163 74.9626276649956) bank182900 +182901 POINT(40.984604346899125 73.2252310391562) bank182901 +182902 POINT(40.60875308063854 73.51437205500129) bank182902 +182903 POINT(40.99500797071154 74.85943578220316) bank182903 +182904 POINT(39.89101704674924 74.73752921020137) bank182904 +182905 POINT(39.808133133923896 74.56560024093254) bank182905 +182906 POINT(41.169533044691434 73.72830022350126) bank182906 +182907 POINT(41.67994308440026 73.66750008786956) bank182907 +182908 POINT(41.34185216241511 73.2600165930652) bank182908 +182909 POINT(40.7326387294241 73.09573227497559) bank182909 +182910 POINT(41.20577407350257 73.73026841485373) bank182910 +182911 POINT(40.07261588099576 74.75424990664278) bank182911 +182912 POINT(40.346062039857486 74.50589717923016) bank182912 +182913 POINT(41.179007310489425 73.9910619972813) bank182913 +182914 POINT(40.97397503416158 73.8224399077815) bank182914 +182915 POINT(40.61865129939218 73.94484773121413) bank182915 +182916 POINT(39.71763038087161 74.71373251554373) bank182916 +182917 POINT(41.483188475606866 73.88985989475044) bank182917 +182918 POINT(41.477974642453106 73.38498409976346) bank182918 +182919 POINT(40.25919623823586 73.832962702646) bank182919 +182920 POINT(41.60594268338888 73.5303843676291) bank182920 +182921 POINT(41.20429878329241 73.76466823053394) bank182921 +182922 POINT(41.43107830637728 73.09631474905218) bank182922 +182923 POINT(41.11821115064142 73.98719708733142) bank182923 +182924 POINT(40.499310916835135 74.02991072396345) bank182924 +182925 POINT(41.415069461416174 73.26163476569702) bank182925 +182926 POINT(41.045905528788815 74.31789797357686) bank182926 +182927 POINT(40.497820485454795 74.52487346410037) bank182927 +182928 POINT(40.062892278732136 74.40441926817986) bank182928 +182929 POINT(41.58518303894477 74.46393240518381) bank182929 +182930 POINT(41.52940117391632 74.81939660110086) bank182930 +182931 POINT(40.43241843631606 74.36611356918435) bank182931 +182932 POINT(41.49423309577722 73.64740483399619) bank182932 +182933 POINT(39.806939617436434 74.58957054097363) bank182933 +182934 POINT(40.261660827698456 73.1539207079537) bank182934 +182935 POINT(41.31696720442419 74.76843436105334) bank182935 +182936 POINT(41.39605970596245 73.48534209180951) bank182936 +182937 POINT(41.582210484490176 73.47346283841168) bank182937 +182938 POINT(41.490621018609865 74.98246390533458) bank182938 +182939 POINT(40.44900618879413 74.64225246672603) bank182939 +182940 POINT(40.58147880795541 73.16661802754253) bank182940 +182941 POINT(41.31609206800836 73.99512846408189) bank182941 +182942 POINT(40.28305728223026 73.20989220667222) bank182942 +182943 POINT(41.152261381879974 73.61590370610088) bank182943 +182944 POINT(41.08727296929036 73.737276376773) bank182944 +182945 POINT(39.980802488542565 73.80619083424276) bank182945 +182946 POINT(40.64524742099899 74.4846026223054) bank182946 +182947 POINT(40.73998323385424 73.9686338451253) bank182947 +182948 POINT(40.41985454588415 73.43230200973589) bank182948 +182949 POINT(41.09627605800808 74.1006703888425) bank182949 +182950 POINT(39.882308869189075 73.03193972758469) bank182950 +182951 POINT(40.782708261942375 74.19704443505753) bank182951 +182952 POINT(39.76957477065598 73.80862701280495) bank182952 +182953 POINT(41.189377032954994 73.63585474565708) bank182953 +182954 POINT(40.12041644037624 74.33667627711415) bank182954 +182955 POINT(41.21352455829212 74.04558892555056) bank182955 +182956 POINT(40.21758763713466 74.34292690980666) bank182956 +182957 POINT(41.08258614969645 74.97432905374217) bank182957 +182958 POINT(39.91577477315454 73.68388909027381) bank182958 +182959 POINT(40.523556120282414 74.51835754334375) bank182959 +182960 POINT(40.066266683475 73.19620435975615) bank182960 +182961 POINT(40.69717523132454 73.16649597376896) bank182961 +182962 POINT(40.71428562144771 74.23319537422371) bank182962 +182963 POINT(41.143084016547085 73.50225748385925) bank182963 +182964 POINT(41.506502870558926 73.5789762212273) bank182964 +182965 POINT(39.798499426579035 74.70543050824207) bank182965 +182966 POINT(40.4389328367087 74.7594716320013) bank182966 +182967 POINT(41.07946526305992 74.47574230812936) bank182967 +182968 POINT(39.90535404490996 74.24640144734565) bank182968 +182969 POINT(40.89967245070038 74.56336064426925) bank182969 +182970 POINT(40.897526047195136 73.1480660548392) bank182970 +182971 POINT(40.019051923891716 73.62771548719856) bank182971 +182972 POINT(40.92462804268605 73.88686616841771) bank182972 +182973 POINT(41.29695907497298 74.05949605563477) bank182973 +182974 POINT(41.59170666099948 74.66973346416313) bank182974 +182975 POINT(40.60227670964985 73.71670141631932) bank182975 +182976 POINT(40.92148706039382 74.65681050997404) bank182976 +182977 POINT(41.179981923071544 74.06937622082285) bank182977 +182978 POINT(41.061592413180136 73.27144012729242) bank182978 +182979 POINT(41.144810819146336 74.98483878643616) bank182979 +182980 POINT(40.46966356457111 74.03793878953071) bank182980 +182981 POINT(39.815477760816805 74.6194509697229) bank182981 +182982 POINT(40.14532145323529 73.08134294042908) bank182982 +182983 POINT(40.10454751338333 74.96175214584912) bank182983 +182984 POINT(39.89402604927415 74.57825881129114) bank182984 +182985 POINT(41.40955153287289 74.36831579705256) bank182985 +182986 POINT(40.980900473663425 74.52602759867717) bank182986 +182987 POINT(41.53659747668885 73.35494283912406) bank182987 +182988 POINT(41.13224071719588 74.47461895213767) bank182988 +182989 POINT(40.436694511311316 73.03384057235328) bank182989 +182990 POINT(40.18038540975786 73.92540821855819) bank182990 +182991 POINT(41.53718054812347 74.91193249058894) bank182991 +182992 POINT(40.7215916615079 73.9848439215186) bank182992 +182993 POINT(40.19438748352627 74.24902535617376) bank182993 +182994 POINT(39.93686320231032 73.88369656245634) bank182994 +182995 POINT(40.3159466229636 73.09450561027396) bank182995 +182996 POINT(41.110954170479864 74.26576884074582) bank182996 +182997 POINT(40.582049582006675 74.40341398654797) bank182997 +182998 POINT(40.024732232601295 74.6000745105761) bank182998 +182999 POINT(40.844412988681746 73.36836761486232) bank182999 +183000 POINT(41.27988856191083 73.21304465467497) bank183000 +183001 POINT(39.8320322837475 73.60255070239106) bank183001 +183002 POINT(41.4865059704683 73.96638828287365) bank183002 +183003 POINT(40.38148915831793 73.96553022426102) bank183003 +183004 POINT(41.33594082964532 74.01016683891592) bank183004 +183005 POINT(40.42247870163778 74.42401506827038) bank183005 +183006 POINT(41.49826251098453 74.30692912379074) bank183006 +183007 POINT(40.31583267809342 73.71120580136505) bank183007 +183008 POINT(40.768460239764295 73.79982422432752) bank183008 +183009 POINT(40.98678634711455 74.64502771684107) bank183009 +183010 POINT(40.02055322330007 74.83690680947062) bank183010 +183011 POINT(41.005535121262845 73.7016073174318) bank183011 +183012 POINT(40.183124107195304 73.43038002775765) bank183012 +183013 POINT(40.72444256473496 73.13886410716239) bank183013 +183014 POINT(41.313793019824956 74.37950845870117) bank183014 +183015 POINT(40.02605334526389 74.21161260579915) bank183015 +183016 POINT(40.84056678361339 74.72320441533645) bank183016 +183017 POINT(41.67857949678446 74.61519635410218) bank183017 +183018 POINT(41.58451217730012 73.29560768097066) bank183018 +183019 POINT(41.1001002592001 73.91203188145373) bank183019 +183020 POINT(39.80152548475968 74.4155423345113) bank183020 +183021 POINT(39.796115908181044 74.7687114457382) bank183021 +183022 POINT(40.8588698601658 73.43247619175189) bank183022 +183023 POINT(39.99171954841784 73.41040547316756) bank183023 +183024 POINT(40.416485086590875 74.01362785568494) bank183024 +183025 POINT(40.73578169200307 73.1524124742051) bank183025 +183026 POINT(40.56210038908834 74.1536400831828) bank183026 +183027 POINT(40.991568460463824 74.95582316606331) bank183027 +183028 POINT(40.67172466611221 74.25512809264721) bank183028 +183029 POINT(40.32906323747959 74.46349494618408) bank183029 +183030 POINT(40.73615292480881 74.05958657522027) bank183030 +183031 POINT(40.88516965738296 73.85113007585092) bank183031 +183032 POINT(40.11647530536672 74.13246294327512) bank183032 +183033 POINT(40.188102060365 73.84052923626017) bank183033 +183034 POINT(41.37703353511361 74.74981243782437) bank183034 +183035 POINT(41.339135306038294 74.65916479495002) bank183035 +183036 POINT(41.040439781345846 73.43620446177815) bank183036 +183037 POINT(40.47050152760384 73.24476035370641) bank183037 +183038 POINT(40.71038495798816 74.9294292995436) bank183038 +183039 POINT(39.77093474933287 74.04129668802283) bank183039 +183040 POINT(40.70130107445562 73.63797012511012) bank183040 +183041 POINT(41.710593966528755 74.14402210161393) bank183041 +183042 POINT(40.78973397850357 74.1728289677681) bank183042 +183043 POINT(40.73459232542689 73.79420381890642) bank183043 +183044 POINT(39.858550834118304 73.09814777039651) bank183044 +183045 POINT(40.2547287962977 74.52685762898908) bank183045 +183046 POINT(40.00029089364598 73.77265943652411) bank183046 +183047 POINT(41.60285403937456 73.5684555474644) bank183047 +183048 POINT(40.55239809759456 74.43379145977123) bank183048 +183049 POINT(40.55769782218171 74.39618364510721) bank183049 +183050 POINT(41.27058856374984 74.92486984280188) bank183050 +183051 POINT(40.253047481818456 73.62748571880388) bank183051 +183052 POINT(41.662508197759195 74.41996099764124) bank183052 +183053 POINT(41.48504123313461 73.86391178556956) bank183053 +183054 POINT(41.02674028661027 73.38011488064272) bank183054 +183055 POINT(40.115236347321094 73.03888327435763) bank183055 +183056 POINT(40.16401807589374 73.99210155952736) bank183056 +183057 POINT(41.66784107320118 74.66649642132089) bank183057 +183058 POINT(40.156070646583096 73.0446595833288) bank183058 +183059 POINT(41.7008937344115 74.66387714426524) bank183059 +183060 POINT(40.172520928371895 73.33178239918159) bank183060 +183061 POINT(40.05280190451323 73.39584524758945) bank183061 +183062 POINT(40.493840930420106 73.4861439931781) bank183062 +183063 POINT(41.54375789010315 74.84844936713813) bank183063 +183064 POINT(41.32042330897053 74.76930307428536) bank183064 +183065 POINT(40.321283566274126 73.53844950002846) bank183065 +183066 POINT(39.975706163321725 74.46325080880429) bank183066 +183067 POINT(40.80500110553259 74.12838382667681) bank183067 +183068 POINT(41.24211818026736 73.98122393288959) bank183068 +183069 POINT(41.56908972399787 74.38116464921586) bank183069 +183070 POINT(41.04547784251162 73.63980367395357) bank183070 +183071 POINT(40.283051931696406 74.22099704939869) bank183071 +183072 POINT(39.78624776534548 73.07579164319695) bank183072 +183073 POINT(41.23139431004085 73.01489088703711) bank183073 +183074 POINT(40.46946834321559 74.46717357305036) bank183074 +183075 POINT(41.32035283894527 74.41642903246068) bank183075 +183076 POINT(40.84737701475804 74.02420313264791) bank183076 +183077 POINT(41.41704032551414 74.62995846468822) bank183077 +183078 POINT(41.67282676755604 73.73875051775467) bank183078 +183079 POINT(40.61944444456431 74.66849263029172) bank183079 +183080 POINT(41.31110022842922 74.40106107018792) bank183080 +183081 POINT(40.65965305803342 73.49954870531417) bank183081 +183082 POINT(40.641825768467534 74.0746748593719) bank183082 +183083 POINT(40.63890698746062 74.71968659582296) bank183083 +183084 POINT(40.465463423605335 74.82758780987339) bank183084 +183085 POINT(41.635224767949694 74.33172186828038) bank183085 +183086 POINT(40.0107794185531 73.16679936249373) bank183086 +183087 POINT(41.11602969210792 74.57784622538927) bank183087 +183088 POINT(40.411160110130055 73.59037965730975) bank183088 +183089 POINT(40.44143932123461 74.50654326333351) bank183089 +183090 POINT(40.53382832954584 74.35374232170277) bank183090 +183091 POINT(40.20324442314118 74.11538551485702) bank183091 +183092 POINT(39.86601160729677 74.4931199598214) bank183092 +183093 POINT(40.618327631259504 74.47042717283226) bank183093 +183094 POINT(40.24808688739796 74.10800434521563) bank183094 +183095 POINT(41.02266031986061 73.41762595006463) bank183095 +183096 POINT(40.332788438988715 74.41857831281135) bank183096 +183097 POINT(40.1344535081362 73.18455284192463) bank183097 +183098 POINT(40.08666825345415 74.13515033018601) bank183098 +183099 POINT(39.80069150698571 73.28554181484556) bank183099 +183100 POINT(41.487295942246284 73.49971840969081) bank183100 +183101 POINT(41.621911811894506 74.9940326284888) bank183101 +183102 POINT(40.49153592118047 73.65522355532931) bank183102 +183103 POINT(40.00837323844499 74.56896566178088) bank183103 +183104 POINT(39.84749075564528 73.8225097428909) bank183104 +183105 POINT(41.27250257960066 74.36966687669499) bank183105 +183106 POINT(41.66778111677068 74.60996041485235) bank183106 +183107 POINT(40.04993873048477 74.68753985819096) bank183107 +183108 POINT(41.026349626263894 73.70551113157278) bank183108 +183109 POINT(40.70325913607231 74.0232839515021) bank183109 +183110 POINT(40.57407097145329 73.69092767867362) bank183110 +183111 POINT(39.964017693431416 73.68557610263358) bank183111 +183112 POINT(41.56299008279455 73.51197649245238) bank183112 +183113 POINT(41.70856687191499 74.45017603815919) bank183113 +183114 POINT(39.87316334102353 73.94527487959824) bank183114 +183115 POINT(39.731835001325734 74.7262377996678) bank183115 +183116 POINT(40.280457470067226 74.06998925979994) bank183116 +183117 POINT(40.07590653467451 74.0921342159729) bank183117 +183118 POINT(40.5406039664409 74.05950596757872) bank183118 +183119 POINT(40.00781608587518 73.88375085896153) bank183119 +183120 POINT(39.847244976154066 74.3261649764623) bank183120 +183121 POINT(39.7480337005626 74.14125100430974) bank183121 +183122 POINT(40.275019034481254 73.91093603273525) bank183122 +183123 POINT(40.2610090147204 73.47663842130088) bank183123 +183124 POINT(41.382411956595895 74.74212322991038) bank183124 +183125 POINT(41.300037819971294 74.57195048455236) bank183125 +183126 POINT(40.26886339394166 74.11044610205775) bank183126 +183127 POINT(40.36154918893804 74.3130170941531) bank183127 +183128 POINT(40.30067171744121 74.47291291431654) bank183128 +183129 POINT(39.99278028206762 74.71664433647233) bank183129 +183130 POINT(40.30532743982551 74.09267605033867) bank183130 +183131 POINT(40.50481610703395 73.16565569706418) bank183131 +183132 POINT(40.770413287189974 73.78074615430663) bank183132 +183133 POINT(40.343589455577366 74.1965535058362) bank183133 +183134 POINT(41.606981698230456 73.09025584653175) bank183134 +183135 POINT(41.38540374563212 74.10149690720063) bank183135 +183136 POINT(41.280669651626326 74.8824976853844) bank183136 +183137 POINT(39.7919569966624 73.3749161499048) bank183137 +183138 POINT(40.92350270109676 73.96124206244797) bank183138 +183139 POINT(40.82611203190169 73.348802056562) bank183139 +183140 POINT(41.20657483805273 74.85242297304744) bank183140 +183141 POINT(41.223865686542176 74.18197198987818) bank183141 +183142 POINT(40.86729359202035 73.48685027275899) bank183142 +183143 POINT(39.81906723663485 74.27191961469417) bank183143 +183144 POINT(40.92367624942482 74.69689407072684) bank183144 +183145 POINT(40.21718670147998 73.59225546024) bank183145 +183146 POINT(39.98933730986876 74.76600393778705) bank183146 +183147 POINT(40.518563248369894 73.07552583380836) bank183147 +183148 POINT(39.896754762729195 73.81346241984942) bank183148 +183149 POINT(40.33490110456541 74.10866513241454) bank183149 +183150 POINT(41.26229657312463 74.65335875293718) bank183150 +183151 POINT(40.88182321079095 73.85068749069912) bank183151 +183152 POINT(40.8568645467171 74.80179281905147) bank183152 +183153 POINT(41.02951608961605 74.44045486525526) bank183153 +183154 POINT(40.44781696555376 74.42399773946178) bank183154 +183155 POINT(40.486122707195804 73.71411938154753) bank183155 +183156 POINT(41.00835564984756 73.76526855685667) bank183156 +183157 POINT(39.7677676124865 73.90703003355202) bank183157 +183158 POINT(41.581717861750526 73.88343357186686) bank183158 +183159 POINT(40.57553489652489 74.24982424896142) bank183159 +183160 POINT(41.24652262786987 73.06519847862037) bank183160 +183161 POINT(40.29993353399781 73.67755995187419) bank183161 +183162 POINT(41.42669486446213 73.71881161408743) bank183162 +183163 POINT(39.82248907465758 74.45061971221887) bank183163 +183164 POINT(40.79635015827536 74.39561472445052) bank183164 +183165 POINT(41.518567069818516 73.72021012657936) bank183165 +183166 POINT(39.83390826028869 74.89350940043465) bank183166 +183167 POINT(41.075376076606545 74.72809797353813) bank183167 +183168 POINT(41.33020513377702 73.99406349043863) bank183168 +183169 POINT(40.384495521788246 74.5876991764683) bank183169 +183170 POINT(41.26280789069752 73.04181766385535) bank183170 +183171 POINT(41.48142150341396 74.84557404343124) bank183171 +183172 POINT(41.20878417365662 73.44005671640448) bank183172 +183173 POINT(41.59259929531005 74.30278400927347) bank183173 +183174 POINT(41.67654664406093 74.51459295997047) bank183174 +183175 POINT(40.21799906862791 73.12233532632955) bank183175 +183176 POINT(40.760445688746714 74.74131886842501) bank183176 +183177 POINT(40.54490361295579 74.2770871074436) bank183177 +183178 POINT(39.756903307787795 73.89222321727534) bank183178 +183179 POINT(40.889427553797766 74.57363936036455) bank183179 +183180 POINT(41.63646918762278 74.71670174025947) bank183180 +183181 POINT(40.3285369191363 73.69731602370618) bank183181 +183182 POINT(41.11796406713819 73.05167942209405) bank183182 +183183 POINT(40.52451378332568 73.68152554051228) bank183183 +183184 POINT(41.43933748714465 74.47632607981147) bank183184 +183185 POINT(41.612922686513414 73.10038754901291) bank183185 +183186 POINT(41.171637191933705 74.92470287628174) bank183186 +183187 POINT(40.64148737810297 73.93024037933544) bank183187 +183188 POINT(41.24070229102731 74.27520771671418) bank183188 +183189 POINT(41.708221453023704 74.62485630038981) bank183189 +183190 POINT(40.29689552883818 74.82548735664442) bank183190 +183191 POINT(39.86089077265657 73.50116416872791) bank183191 +183192 POINT(40.89963610643298 74.94205664938954) bank183192 +183193 POINT(41.65681068809379 74.62734582953226) bank183193 +183194 POINT(40.944037392519846 73.44721138171704) bank183194 +183195 POINT(40.044790173784016 73.75436729414211) bank183195 +183196 POINT(41.68945005171228 74.88884525710024) bank183196 +183197 POINT(40.53780516250009 74.57162035291104) bank183197 +183198 POINT(41.6269987752195 73.61324492090706) bank183198 +183199 POINT(40.06067956552306 73.79034852676423) bank183199 +183200 POINT(40.8083712383558 73.0471188098461) bank183200 +183201 POINT(40.39128318134233 73.70034002306745) bank183201 +183202 POINT(41.61043017950899 73.66453405604615) bank183202 +183203 POINT(41.54649778032294 74.22323497011683) bank183203 +183204 POINT(40.61695354162662 73.72719744591062) bank183204 +183205 POINT(41.0827279033288 74.34807390637293) bank183205 +183206 POINT(41.26702123211467 74.90962871945825) bank183206 +183207 POINT(40.79570247027564 74.42693363504932) bank183207 +183208 POINT(40.37545381979987 73.30477426319439) bank183208 +183209 POINT(40.813220856398544 73.5244217395954) bank183209 +183210 POINT(40.20332878900753 73.96142418603523) bank183210 +183211 POINT(39.81784816793326 74.12772963370388) bank183211 +183212 POINT(39.77517532601135 74.65855709278422) bank183212 +183213 POINT(41.049480782932974 73.19476333774138) bank183213 +183214 POINT(40.12501553722089 74.4172104483805) bank183214 +183215 POINT(40.033576506764504 73.51713655545026) bank183215 +183216 POINT(41.563616390408 74.46510112072885) bank183216 +183217 POINT(40.58021134156634 74.98892304595711) bank183217 +183218 POINT(40.37000280892172 73.07083126730575) bank183218 +183219 POINT(41.64825272221801 73.13413115314523) bank183219 +183220 POINT(41.65079220980403 73.30453250090373) bank183220 +183221 POINT(40.70028176760526 73.14994847361471) bank183221 +183222 POINT(40.4666884679086 73.61332446390597) bank183222 +183223 POINT(41.301748095949904 74.29999002097414) bank183223 +183224 POINT(40.451449968569776 73.2731926750788) bank183224 +183225 POINT(41.63122692990025 73.01931787819991) bank183225 +183226 POINT(40.12252993729838 74.42392756381116) bank183226 +183227 POINT(39.72730142058405 74.02496663592797) bank183227 +183228 POINT(40.364506655049915 74.5783504130992) bank183228 +183229 POINT(39.957411320218 74.84547249246334) bank183229 +183230 POINT(40.299783925430106 74.07911919075586) bank183230 +183231 POINT(40.51376238285123 73.66609270978493) bank183231 +183232 POINT(41.04422309234787 74.20719949486134) bank183232 +183233 POINT(41.37619144385929 73.58433971177989) bank183233 +183234 POINT(40.65896165756428 73.70057026415101) bank183234 +183235 POINT(40.461942895992685 73.72571202852859) bank183235 +183236 POINT(40.39024569743037 73.32780869574941) bank183236 +183237 POINT(40.3406950285936 73.0471680028187) bank183237 +183238 POINT(40.08963572793712 74.65869274964918) bank183238 +183239 POINT(41.51792299474607 74.56607542674168) bank183239 +183240 POINT(41.271370406494135 73.91419543034829) bank183240 +183241 POINT(41.367782635455185 74.38865359331596) bank183241 +183242 POINT(41.53108318204429 73.9196349676498) bank183242 +183243 POINT(41.6481838790254 73.6236559967206) bank183243 +183244 POINT(41.65879254107864 74.72856788083774) bank183244 +183245 POINT(40.348992567608256 73.48950508480567) bank183245 +183246 POINT(41.004139149370154 74.28805320507993) bank183246 +183247 POINT(40.46506397088039 73.32473142876759) bank183247 +183248 POINT(40.365256974645675 74.62482220218166) bank183248 +183249 POINT(41.52653800078494 74.49316317243165) bank183249 +183250 POINT(40.910856958922025 74.39916181256353) bank183250 +183251 POINT(41.31420181190449 73.11455284417504) bank183251 +183252 POINT(40.162425442199975 73.90808002067163) bank183252 +183253 POINT(41.43349442721991 74.25553029391918) bank183253 +183254 POINT(40.51698217156118 73.1921257376495) bank183254 +183255 POINT(40.292638417215535 73.97298702656389) bank183255 +183256 POINT(39.91737807867916 73.11063047694358) bank183256 +183257 POINT(40.793244563977375 74.6491807456698) bank183257 +183258 POINT(41.48748930909947 73.52851726448648) bank183258 +183259 POINT(40.139757780879386 74.15064636637379) bank183259 +183260 POINT(40.40054160313519 74.70042960619806) bank183260 +183261 POINT(41.08407775477074 74.74217360196982) bank183261 +183262 POINT(40.59347124189353 74.60607738627539) bank183262 +183263 POINT(39.900065734613925 73.56451065835657) bank183263 +183264 POINT(41.29650769749358 73.94328285363375) bank183264 +183265 POINT(39.73569525198486 73.85044977981005) bank183265 +183266 POINT(40.17921897181065 73.38554511022335) bank183266 +183267 POINT(41.260235863869546 74.17424394735562) bank183267 +183268 POINT(41.54394512096589 74.63068880844509) bank183268 +183269 POINT(40.703817995981446 74.46031729346096) bank183269 +183270 POINT(40.63574788294324 73.01476837016857) bank183270 +183271 POINT(39.78900268254925 73.73652344971075) bank183271 +183272 POINT(39.84093574820423 73.62502647868146) bank183272 +183273 POINT(40.742408600379584 74.89725582932755) bank183273 +183274 POINT(40.87840432423436 74.66263889485359) bank183274 +183275 POINT(40.52473425999983 73.53727621749475) bank183275 +183276 POINT(40.55533531917076 74.5717160681181) bank183276 +183277 POINT(39.84064607057026 73.51581961893702) bank183277 +183278 POINT(40.86645829776919 74.65724610533735) bank183278 +183279 POINT(40.05456689844959 73.4012186170684) bank183279 +183280 POINT(39.803821980759494 73.43068390859288) bank183280 +183281 POINT(41.30995556750388 74.77323909992741) bank183281 +183282 POINT(41.03577337037984 74.60951246405894) bank183282 +183283 POINT(40.07795612047084 73.3492705521668) bank183283 +183284 POINT(40.80803085731892 74.4600121512002) bank183284 +183285 POINT(41.643657324345924 74.23844131958325) bank183285 +183286 POINT(40.322903635034336 73.73225602047525) bank183286 +183287 POINT(40.40133529302737 74.20969890352413) bank183287 +183288 POINT(39.9288203144188 74.64517587145137) bank183288 +183289 POINT(39.972082449075955 73.21551616158075) bank183289 +183290 POINT(40.80983152257998 74.23724558362537) bank183290 +183291 POINT(40.90688008109339 74.15613532830503) bank183291 +183292 POINT(40.29935481822402 73.49761493453651) bank183292 +183293 POINT(41.228213304303175 74.63425555289488) bank183293 +183294 POINT(41.23274717179678 73.74429694380834) bank183294 +183295 POINT(40.16697176820952 73.18914528811864) bank183295 +183296 POINT(41.561794204882716 74.71848656522948) bank183296 +183297 POINT(40.41185871775955 74.34727015599182) bank183297 +183298 POINT(40.05434869692037 74.8118699712662) bank183298 +183299 POINT(41.345355646890205 74.82955176166094) bank183299 +183300 POINT(40.32670130217805 74.26930358380083) bank183300 +183301 POINT(40.55647693751627 73.02871485198308) bank183301 +183302 POINT(41.34085572253593 74.49504832852433) bank183302 +183303 POINT(41.405274381940316 74.68269199826823) bank183303 +183304 POINT(39.90970424761493 74.79231463260939) bank183304 +183305 POINT(40.68561691186425 74.37383517389465) bank183305 +183306 POINT(40.20752845641386 73.7581300742227) bank183306 +183307 POINT(40.23230681676657 73.06440186932262) bank183307 +183308 POINT(40.549021057032576 73.35668888501381) bank183308 +183309 POINT(40.25697978062049 74.04724742229921) bank183309 +183310 POINT(41.52083501753664 74.51318739636974) bank183310 +183311 POINT(41.377143545557274 73.81917235090687) bank183311 +183312 POINT(39.820936568503086 74.11387734150475) bank183312 +183313 POINT(41.67815673847862 73.60966752242031) bank183313 +183314 POINT(39.915061615954414 73.01376345165089) bank183314 +183315 POINT(39.865449047810735 74.99612182155116) bank183315 +183316 POINT(41.30707123291099 74.75818874655188) bank183316 +183317 POINT(41.099570775348866 73.04332397559645) bank183317 +183318 POINT(41.539189742537275 74.53649802077103) bank183318 +183319 POINT(40.777537151040676 73.33750335879309) bank183319 +183320 POINT(41.02955915327841 73.19577405256192) bank183320 +183321 POINT(40.264762301077 73.23419561629878) bank183321 +183322 POINT(41.410029861756755 74.72291981387121) bank183322 +183323 POINT(40.08042311393921 74.7107539624516) bank183323 +183324 POINT(40.01087393921906 74.96186014073922) bank183324 +183325 POINT(40.30733645844101 74.5845395402805) bank183325 +183326 POINT(40.2721210328701 73.17166581737358) bank183326 +183327 POINT(40.5944393882411 74.29516778732547) bank183327 +183328 POINT(41.50323035920989 73.9745831944476) bank183328 +183329 POINT(41.534656536227246 73.24974799604695) bank183329 +183330 POINT(41.178472920651416 73.83962803988062) bank183330 +183331 POINT(41.236978975846455 73.27028813391811) bank183331 +183332 POINT(41.10739779450513 74.07770616446099) bank183332 +183333 POINT(41.19451586258912 74.36912578083724) bank183333 +183334 POINT(40.27895106630572 73.66540622577617) bank183334 +183335 POINT(41.317464741086255 74.77348803149485) bank183335 +183336 POINT(40.137772302414376 73.81710526778514) bank183336 +183337 POINT(41.240185505581564 73.61495484083486) bank183337 +183338 POINT(41.334911486758315 74.5524923840906) bank183338 +183339 POINT(40.98570785157777 74.04915056931239) bank183339 +183340 POINT(41.331358119187485 74.13164638076631) bank183340 +183341 POINT(41.071607623074605 74.40572549184164) bank183341 +183342 POINT(41.53647459003622 73.11659960606572) bank183342 +183343 POINT(40.85624199775048 74.86326991939039) bank183343 +183344 POINT(41.52279038942185 73.78902169846252) bank183344 +183345 POINT(39.726588758688486 74.11467752845164) bank183345 +183346 POINT(40.7650821666727 74.76798304611974) bank183346 +183347 POINT(41.707269629075434 74.66591011459344) bank183347 +183348 POINT(40.13610392513137 73.61035951206017) bank183348 +183349 POINT(40.642903397837294 73.06775880715008) bank183349 +183350 POINT(40.197425685054014 73.38552209573761) bank183350 +183351 POINT(40.6302450670661 74.78660161344585) bank183351 +183352 POINT(41.01389937931504 73.31352724580387) bank183352 +183353 POINT(40.116068538811874 73.48083481064953) bank183353 +183354 POINT(39.78420508504231 73.93135493691425) bank183354 +183355 POINT(40.00972952845286 73.64802960580488) bank183355 +183356 POINT(40.17286507810224 73.60476170637816) bank183356 +183357 POINT(39.98795510269312 73.42856322599083) bank183357 +183358 POINT(39.77547644028246 74.61744160866915) bank183358 +183359 POINT(41.09156020722947 73.77928083851263) bank183359 +183360 POINT(41.097807935344086 74.14723354455332) bank183360 +183361 POINT(41.35640520545004 73.7971182505233) bank183361 +183362 POINT(40.52946076016913 74.44359803203304) bank183362 +183363 POINT(39.839898860959295 74.85761578179277) bank183363 +183364 POINT(41.14458810297311 74.74831813765552) bank183364 +183365 POINT(39.9224361497803 73.12214545413852) bank183365 +183366 POINT(40.74147378342216 73.01952732502747) bank183366 +183367 POINT(41.082626337523706 73.2830171560211) bank183367 +183368 POINT(40.856710918047156 73.47285106647219) bank183368 +183369 POINT(40.615967614199654 74.71772826951533) bank183369 +183370 POINT(40.393264514935126 73.48418759789445) bank183370 +183371 POINT(40.668008266137754 74.61089760367251) bank183371 +183372 POINT(40.1078749425146 74.9304835099709) bank183372 +183373 POINT(40.57513187780817 73.3262043301832) bank183373 +183374 POINT(39.83893444493764 73.27563381056271) bank183374 +183375 POINT(40.09451660336821 73.27421623982399) bank183375 +183376 POINT(39.85471604933031 74.81053641024276) bank183376 +183377 POINT(40.488064033624426 73.63032641349615) bank183377 +183378 POINT(40.72844200812627 73.77850493463094) bank183378 +183379 POINT(39.962339769115545 74.73128312964518) bank183379 +183380 POINT(40.326704969647814 74.47649793465548) bank183380 +183381 POINT(40.52862490953283 74.03924594893068) bank183381 +183382 POINT(40.287007633907066 74.88499384998937) bank183382 +183383 POINT(41.15332900796636 73.75932807930718) bank183383 +183384 POINT(40.01802075169085 74.44849561334343) bank183384 +183385 POINT(40.16402903703187 73.31551416450431) bank183385 +183386 POINT(40.770196133803594 73.3670095148938) bank183386 +183387 POINT(40.54861927849879 74.7598513590872) bank183387 +183388 POINT(40.95087938267234 74.83115380141795) bank183388 +183389 POINT(41.11553485483255 73.55260769563674) bank183389 +183390 POINT(40.05369461974747 74.74702067835949) bank183390 +183391 POINT(39.97083949116425 74.53047791552565) bank183391 +183392 POINT(41.466126697323844 73.77219108694771) bank183392 +183393 POINT(41.69406320132012 74.7469788824177) bank183393 +183394 POINT(40.388538772621814 73.83177738502195) bank183394 +183395 POINT(41.402656431474306 73.21123267333567) bank183395 +183396 POINT(40.234794784094014 74.63542569805414) bank183396 +183397 POINT(41.662436480024226 73.4274553340097) bank183397 +183398 POINT(40.30348736935202 74.7017880595496) bank183398 +183399 POINT(40.10033664495242 73.84166561983989) bank183399 +183400 POINT(41.53581134373962 74.39632805665859) bank183400 +183401 POINT(40.35734153256675 73.13995979812653) bank183401 +183402 POINT(40.600666279372454 74.54215530060223) bank183402 +183403 POINT(40.50680405711972 74.36159895962852) bank183403 +183404 POINT(40.33535022890499 74.84224611325234) bank183404 +183405 POINT(40.33262841735774 74.48807153095257) bank183405 +183406 POINT(41.359158254301924 74.9512491747847) bank183406 +183407 POINT(41.56844890330001 73.3805935557683) bank183407 +183408 POINT(41.01267280887757 74.83480432584732) bank183408 +183409 POINT(39.93690966395147 74.41529363127783) bank183409 +183410 POINT(41.09581022708998 74.5888521315601) bank183410 +183411 POINT(39.97364342496627 73.78505758704847) bank183411 +183412 POINT(39.85701314840679 74.67402684534191) bank183412 +183413 POINT(41.29467104408321 73.21118182814524) bank183413 +183414 POINT(40.81265709828278 73.69242690089922) bank183414 +183415 POINT(40.09568903957872 74.00008477033832) bank183415 +183416 POINT(41.08228944846318 73.062225223705) bank183416 +183417 POINT(41.356773728186994 74.53598072634747) bank183417 +183418 POINT(41.13377056027178 74.32901864436525) bank183418 +183419 POINT(39.739217318437056 73.98688085784201) bank183419 +183420 POINT(41.42756290611727 74.14893971519952) bank183420 +183421 POINT(39.78121103995901 74.02974704018483) bank183421 +183422 POINT(40.2739612583844 74.09797839509282) bank183422 +183423 POINT(40.549474947573884 74.73011032242776) bank183423 +183424 POINT(41.43714078887257 74.9965604447166) bank183424 +183425 POINT(41.496912195515016 73.60741089492267) bank183425 +183426 POINT(39.76971800583956 74.53731791940042) bank183426 +183427 POINT(40.84523694543506 73.10185642638243) bank183427 +183428 POINT(41.518415463035396 74.67176262740412) bank183428 +183429 POINT(40.59584566069435 73.29854696112108) bank183429 +183430 POINT(40.63406702416962 73.16689667642773) bank183430 +183431 POINT(40.88458224078386 73.34138244930831) bank183431 +183432 POINT(39.82058496486577 73.54085461613629) bank183432 +183433 POINT(40.3370705743707 73.76847693029613) bank183433 +183434 POINT(40.418837393562384 73.65481970179066) bank183434 +183435 POINT(40.71915890175026 74.24424698928048) bank183435 +183436 POINT(41.488962750839214 73.44564997066121) bank183436 +183437 POINT(41.082908519225185 74.86300882637404) bank183437 +183438 POINT(40.06622595784015 74.03163969044328) bank183438 +183439 POINT(40.276052054304294 73.16494443706709) bank183439 +183440 POINT(40.05338366124938 74.48124019116291) bank183440 +183441 POINT(41.53591777282912 73.17694695281813) bank183441 +183442 POINT(40.73081879698342 73.91280830720835) bank183442 +183443 POINT(40.68233700674631 74.43788007869136) bank183443 +183444 POINT(40.03633827587445 74.04686901509588) bank183444 +183445 POINT(40.43786553161115 73.74964267867442) bank183445 +183446 POINT(39.74467547065385 73.33120330041166) bank183446 +183447 POINT(41.218848869620615 73.77384056574667) bank183447 +183448 POINT(40.96654019037144 74.20363305524468) bank183448 +183449 POINT(40.58646540973758 74.5541492067382) bank183449 +183450 POINT(40.64270258046367 74.83310340883689) bank183450 +183451 POINT(40.01117093694167 73.53440260274292) bank183451 +183452 POINT(41.26361372174879 73.32097112163545) bank183452 +183453 POINT(41.36634054179006 75.00329655157793) bank183453 +183454 POINT(41.08652626858368 73.65007451579544) bank183454 +183455 POINT(41.61793973171566 73.41190664167596) bank183455 +183456 POINT(41.14902591040732 73.47787841239494) bank183456 +183457 POINT(39.97875053299498 74.38451413350339) bank183457 +183458 POINT(41.05313376448352 73.75979224755737) bank183458 +183459 POINT(41.210350833670724 73.28789518323393) bank183459 +183460 POINT(40.988824644187766 73.54185679744973) bank183460 +183461 POINT(40.722456769813654 74.59765453845705) bank183461 +183462 POINT(40.029712334700484 74.93882805614487) bank183462 +183463 POINT(39.952485201162794 73.72521763921971) bank183463 +183464 POINT(41.54049693681238 74.7545015538513) bank183464 +183465 POINT(40.51023808389142 73.5503580805064) bank183465 +183466 POINT(40.096792418916806 73.43308719121157) bank183466 +183467 POINT(40.00060434269488 74.51283798508999) bank183467 +183468 POINT(40.92256522787714 74.70319420039782) bank183468 +183469 POINT(40.469405361885 74.24781004581608) bank183469 +183470 POINT(41.536801565742145 74.26013230434958) bank183470 +183471 POINT(41.50454965999051 73.71348273574428) bank183471 +183472 POINT(41.43945019631999 74.22016376969098) bank183472 +183473 POINT(40.99303696264943 73.18616620796004) bank183473 +183474 POINT(41.71108985998198 74.04731716217375) bank183474 +183475 POINT(40.650058746070094 73.2386021951074) bank183475 +183476 POINT(40.57659058803755 74.13176149663872) bank183476 +183477 POINT(41.637533213736496 73.46884482181555) bank183477 +183478 POINT(40.22748821580661 74.27918786455461) bank183478 +183479 POINT(40.0108879461853 74.88241389602474) bank183479 +183480 POINT(41.616146071401175 73.89499401421256) bank183480 +183481 POINT(40.85903081192535 73.42517972346178) bank183481 +183482 POINT(41.36767962735628 73.8852851633378) bank183482 +183483 POINT(39.74659213893132 74.45698996748865) bank183483 +183484 POINT(41.477216358273914 74.76681759707742) bank183484 +183485 POINT(41.41392705402714 73.56800742696926) bank183485 +183486 POINT(39.764279563664374 74.06242970406501) bank183486 +183487 POINT(39.748030276864185 73.55121051653552) bank183487 +183488 POINT(40.19408419769249 74.72745566603592) bank183488 +183489 POINT(40.62208858308991 74.95383621433805) bank183489 +183490 POINT(40.81881592689368 74.32642955339841) bank183490 +183491 POINT(40.39521600337155 73.69683487040376) bank183491 +183492 POINT(40.98751238505486 73.86505080586717) bank183492 +183493 POINT(40.803264472783525 74.69658277870687) bank183493 +183494 POINT(41.08827287449899 74.05521904025206) bank183494 +183495 POINT(40.67946381676728 73.16235536527425) bank183495 +183496 POINT(40.282176037995626 74.83471504670285) bank183496 +183497 POINT(40.6793906735 73.78559264717208) bank183497 +183498 POINT(41.483526494793765 73.94431773060029) bank183498 +183499 POINT(41.464819234763056 73.03496719161801) bank183499 +183500 POINT(39.94821597436189 74.59784139486146) bank183500 +183501 POINT(39.87879099865464 74.85238350426395) bank183501 +183502 POINT(40.94759747738847 74.01872362379) bank183502 +183503 POINT(41.37128411933491 73.22791431873779) bank183503 +183504 POINT(40.800717396164885 74.566143298924) bank183504 +183505 POINT(41.546481803875224 73.70378346001276) bank183505 +183506 POINT(39.76194680774801 73.88247276746027) bank183506 +183507 POINT(40.29461906394525 73.57974548207106) bank183507 +183508 POINT(41.43282205572038 74.56935503784442) bank183508 +183509 POINT(40.299815300924685 74.04356105704048) bank183509 +183510 POINT(39.94369312980724 74.55975487037428) bank183510 +183511 POINT(40.69772497241589 73.17822700678015) bank183511 +183512 POINT(40.270024920163756 73.568262126928) bank183512 +183513 POINT(39.75000115498691 73.18827975494949) bank183513 +183514 POINT(40.821423814355214 74.29360037220702) bank183514 +183515 POINT(41.064992513667335 74.89451582240466) bank183515 +183516 POINT(41.45298058859879 74.67945971923294) bank183516 +183517 POINT(40.79359053856346 74.04763204490722) bank183517 +183518 POINT(41.007869085107764 73.19446814990383) bank183518 +183519 POINT(40.55956204912338 74.65049999984079) bank183519 +183520 POINT(40.325130435358055 73.04291676847821) bank183520 +183521 POINT(41.20955584964865 74.26862018586907) bank183521 +183522 POINT(40.51055990457991 73.44657929663767) bank183522 +183523 POINT(40.81043474085494 74.20696160248224) bank183523 +183524 POINT(39.87706727283098 73.41668388488527) bank183524 +183525 POINT(41.40052177261368 73.30014547331689) bank183525 +183526 POINT(40.741945837639975 73.49333510504903) bank183526 +183527 POINT(39.88653659438514 73.51134238894734) bank183527 +183528 POINT(40.3182057912346 74.16771032313328) bank183528 +183529 POINT(41.07905039323657 74.0031351538314) bank183529 +183530 POINT(41.368992132196844 73.33525888429145) bank183530 +183531 POINT(41.41990888770133 74.00126945495998) bank183531 +183532 POINT(40.708399287311025 74.6080317870901) bank183532 +183533 POINT(39.978985949663425 73.47007138579008) bank183533 +183534 POINT(40.0049006759112 74.55826887649063) bank183534 +183535 POINT(40.700450594958546 73.10198283710257) bank183535 +183536 POINT(40.22205701893396 74.52934871734934) bank183536 +183537 POINT(40.7910213873988 73.93522507234258) bank183537 +183538 POINT(40.33654643319138 74.6523574412478) bank183538 +183539 POINT(40.85179069023442 73.33955769256829) bank183539 +183540 POINT(40.245101010684195 73.2384376777526) bank183540 +183541 POINT(41.68698544617021 74.5231043517475) bank183541 +183542 POINT(40.85980383635953 74.05257656408676) bank183542 +183543 POINT(40.10798919471805 73.04862799284848) bank183543 +183544 POINT(40.186866350947696 74.97384088738673) bank183544 +183545 POINT(39.749429140597805 73.37314386863065) bank183545 +183546 POINT(39.73100027219 73.11447999986923) bank183546 +183547 POINT(40.326299148696876 73.36611545286893) bank183547 +183548 POINT(40.116795065689786 73.25114876960384) bank183548 +183549 POINT(40.397491181491546 74.2759080750908) bank183549 +183550 POINT(40.64530953071686 73.30171111265966) bank183550 +183551 POINT(41.55738304204292 73.5908785247195) bank183551 +183552 POINT(40.57164016197915 73.55918233752458) bank183552 +183553 POINT(39.75021402866387 74.2415165284312) bank183553 +183554 POINT(40.912409579176874 73.23201246044286) bank183554 +183555 POINT(40.46232186099907 74.6078572585616) bank183555 +183556 POINT(40.097147051371245 73.89575977519978) bank183556 +183557 POINT(40.238043702498146 74.34971662744002) bank183557 +183558 POINT(39.942109741295845 73.37197995847144) bank183558 +183559 POINT(41.439057783588 74.04477147638578) bank183559 +183560 POINT(41.431910416008634 73.7220563053932) bank183560 +183561 POINT(40.610162155511176 74.19665608912975) bank183561 +183562 POINT(39.81688409712043 74.23021529278847) bank183562 +183563 POINT(41.266826208170066 74.88753462594806) bank183563 +183564 POINT(40.1011797138527 74.27248042742926) bank183564 +183565 POINT(40.091723984256795 73.10884191945235) bank183565 +183566 POINT(40.654138380805875 74.26127679356509) bank183566 +183567 POINT(40.99523878949426 73.79018886312863) bank183567 +183568 POINT(41.69622434967124 73.40725789123171) bank183568 +183569 POINT(41.42670378715885 73.08496372260564) bank183569 +183570 POINT(40.967234574860164 73.94859764442354) bank183570 +183571 POINT(40.16827213415149 74.76594060530924) bank183571 +183572 POINT(40.31448734979226 73.25825283714907) bank183572 +183573 POINT(39.97235337654247 74.00609631127851) bank183573 +183574 POINT(39.73094264298595 74.64622747341483) bank183574 +183575 POINT(40.25328558802732 74.46742775666011) bank183575 +183576 POINT(41.69060116065668 74.24653086369409) bank183576 +183577 POINT(39.75205706010761 73.66303399088771) bank183577 +183578 POINT(41.20647647496351 73.72384162531228) bank183578 +183579 POINT(41.04054902847858 73.4422567085513) bank183579 +183580 POINT(41.382760125132194 74.6962108611493) bank183580 +183581 POINT(40.25260067148957 74.98852268980953) bank183581 +183582 POINT(40.74741892785604 73.21622685123069) bank183582 +183583 POINT(40.326636833100466 74.2785933421187) bank183583 +183584 POINT(40.52708672457587 73.78234873323619) bank183584 +183585 POINT(41.43144623875835 74.45752209581072) bank183585 +183586 POINT(41.133272370297334 74.71381175124472) bank183586 +183587 POINT(41.67837297836907 73.80109238164957) bank183587 +183588 POINT(40.249316481935175 73.3804375792862) bank183588 +183589 POINT(40.241833464990314 74.23036007427315) bank183589 +183590 POINT(41.160222255683244 74.42338004764906) bank183590 +183591 POINT(41.408277126656145 74.83084594324423) bank183591 +183592 POINT(41.33485863311745 73.04018023622606) bank183592 +183593 POINT(41.32121305293002 74.21019672080385) bank183593 +183594 POINT(41.047745752049025 74.42019763568238) bank183594 +183595 POINT(41.56561582822373 74.57772022441465) bank183595 +183596 POINT(40.10600010607513 74.09874680591726) bank183596 +183597 POINT(40.02624915272712 74.19659351504866) bank183597 +183598 POINT(40.432098208169066 74.076310281242) bank183598 +183599 POINT(40.999041330525245 74.1167383624132) bank183599 +183600 POINT(39.75181203974537 73.53519559300422) bank183600 +183601 POINT(39.98076345760856 74.81583422464188) bank183601 +183602 POINT(40.63392248055692 74.2234095031665) bank183602 +183603 POINT(40.44089406277916 74.58497609685566) bank183603 +183604 POINT(40.340789067855304 74.78034839576344) bank183604 +183605 POINT(41.117640125864995 73.83781886247257) bank183605 +183606 POINT(40.750914983024316 74.02137677698765) bank183606 +183607 POINT(41.61958637262331 73.06124923697348) bank183607 +183608 POINT(39.936196397709914 73.64375076668247) bank183608 +183609 POINT(40.3744835284979 73.1091634810399) bank183609 +183610 POINT(39.82416796921164 73.88503105225882) bank183610 +183611 POINT(41.404346716417415 74.59637610449765) bank183611 +183612 POINT(41.45627019945639 74.906103741985) bank183612 +183613 POINT(40.08833851777917 74.60411316297805) bank183613 +183614 POINT(40.98081940539472 73.07436569531276) bank183614 +183615 POINT(41.24639016655357 73.82940025968263) bank183615 +183616 POINT(40.15157234008244 74.35605705592629) bank183616 +183617 POINT(41.44669018733779 74.9775159902346) bank183617 +183618 POINT(41.23463353487353 74.07701798304447) bank183618 +183619 POINT(41.49420641594792 74.81716708905) bank183619 +183620 POINT(40.5796084130034 74.52780514130157) bank183620 +183621 POINT(40.501923941142564 73.37800598556126) bank183621 +183622 POINT(41.33487020182828 73.10882316742942) bank183622 +183623 POINT(41.27205928402615 73.99270008904374) bank183623 +183624 POINT(41.55117946275473 74.74242167797959) bank183624 +183625 POINT(41.06586886328144 74.94654082205832) bank183625 +183626 POINT(41.5841173787135 73.35352885134722) bank183626 +183627 POINT(40.70573299034842 73.32931102584645) bank183627 +183628 POINT(40.062937381074754 74.71433036254896) bank183628 +183629 POINT(39.93941884009655 73.55177998204684) bank183629 +183630 POINT(40.595279130990995 73.07724999027695) bank183630 +183631 POINT(40.182503243550755 74.33622269020799) bank183631 +183632 POINT(41.406041773073454 73.63376452032924) bank183632 +183633 POINT(41.573206405907264 74.82966287513393) bank183633 +183634 POINT(41.60601425727916 74.18950947726918) bank183634 +183635 POINT(41.37659571211887 74.44908649109438) bank183635 +183636 POINT(41.301184041622925 73.17993670097141) bank183636 +183637 POINT(40.19091229779551 74.60838600671286) bank183637 +183638 POINT(40.37581434774928 73.75258531690467) bank183638 +183639 POINT(40.85787677037766 74.01879365376934) bank183639 +183640 POINT(41.61951506810659 74.54318763619416) bank183640 +183641 POINT(40.740463495398814 73.54467998298871) bank183641 +183642 POINT(41.26947067470943 74.83331998216937) bank183642 +183643 POINT(39.791878952868274 73.20130180767224) bank183643 +183644 POINT(41.29987265386594 74.9580462585775) bank183644 +183645 POINT(41.08171093865661 74.97968906772155) bank183645 +183646 POINT(41.459280593194684 73.56879062752489) bank183646 +183647 POINT(39.78077538337974 73.05541066198728) bank183647 +183648 POINT(39.9449507515847 74.11186517178417) bank183648 +183649 POINT(40.91716331183183 73.8438151225097) bank183649 +183650 POINT(40.19452388556795 73.17325099124572) bank183650 +183651 POINT(40.05980537065257 74.6898202031833) bank183651 +183652 POINT(41.33404687789292 73.8626145659677) bank183652 +183653 POINT(41.30090821028471 74.2037488990604) bank183653 +183654 POINT(41.05114006616684 74.8327529707966) bank183654 +183655 POINT(41.58387053671564 74.62199530745339) bank183655 +183656 POINT(40.38193685769478 73.79770139552431) bank183656 +183657 POINT(40.96029584071551 73.70958806887492) bank183657 +183658 POINT(41.55915484105578 74.9586624090258) bank183658 +183659 POINT(40.93149751193238 73.5945962968301) bank183659 +183660 POINT(40.24489143657108 73.6549119342519) bank183660 +183661 POINT(40.449872992427096 74.44167324211905) bank183661 +183662 POINT(40.16745168050833 74.08125333572826) bank183662 +183663 POINT(39.898905793516484 74.95985120963985) bank183663 +183664 POINT(41.22863833624198 74.98782197793918) bank183664 +183665 POINT(40.481066061515946 73.89532038731198) bank183665 +183666 POINT(41.3748508549969 74.59041812870008) bank183666 +183667 POINT(39.83211964885997 73.21061149425503) bank183667 +183668 POINT(40.9665864319017 74.47709678255043) bank183668 +183669 POINT(40.70863420748739 73.80852708558861) bank183669 +183670 POINT(40.21509989766634 74.57224795169756) bank183670 +183671 POINT(40.14694151586291 74.99180504534931) bank183671 +183672 POINT(40.739248042471885 73.99227041128593) bank183672 +183673 POINT(41.11039780413219 73.26516163188418) bank183673 +183674 POINT(41.124978558641935 73.80570452301139) bank183674 +183675 POINT(41.04260405979361 73.59591542879728) bank183675 +183676 POINT(39.83353312293301 74.8117422982054) bank183676 +183677 POINT(40.11591174599806 73.90726092211624) bank183677 +183678 POINT(41.226152980653914 73.5695790888838) bank183678 +183679 POINT(39.90969799867954 74.86502715336677) bank183679 +183680 POINT(41.18851667188258 73.19367345088772) bank183680 +183681 POINT(40.486571803707555 73.60394250644849) bank183681 +183682 POINT(40.29264152837098 74.06210654022766) bank183682 +183683 POINT(39.865218886671705 73.89277546771797) bank183683 +183684 POINT(40.30150219057798 73.01916357198188) bank183684 +183685 POINT(40.41317978663308 73.11093629253729) bank183685 +183686 POINT(40.12347529489172 73.04904712872137) bank183686 +183687 POINT(41.06950795912593 73.13968209038906) bank183687 +183688 POINT(40.44685000379367 74.69743295742029) bank183688 +183689 POINT(41.672697422364514 73.35382631709396) bank183689 +183690 POINT(41.68806471831555 74.74105035506932) bank183690 +183691 POINT(40.83842792518661 73.77456672179397) bank183691 +183692 POINT(41.624340266620955 73.51657376414398) bank183692 +183693 POINT(39.72602279911975 73.87209049302933) bank183693 +183694 POINT(39.854913932386424 73.65373426411543) bank183694 +183695 POINT(40.87425435293421 74.20149304844949) bank183695 +183696 POINT(39.82584496795179 73.88851679173553) bank183696 +183697 POINT(41.313409590739845 73.42925979547975) bank183697 +183698 POINT(40.61710781345043 73.10498905358828) bank183698 +183699 POINT(40.86713220283563 74.68076132359255) bank183699 +183700 POINT(39.75473372754023 74.65015960674768) bank183700 +183701 POINT(41.40183334747411 73.97870317881265) bank183701 +183702 POINT(41.52856663342729 74.13709242587458) bank183702 +183703 POINT(41.68804314106944 74.15963127658262) bank183703 +183704 POINT(40.24569741630711 73.30117616236258) bank183704 +183705 POINT(40.06700073509149 73.99784767826412) bank183705 +183706 POINT(40.70316652248926 73.6996512246316) bank183706 +183707 POINT(41.189395077044495 73.46866951896939) bank183707 +183708 POINT(41.265706984871514 74.39692231439831) bank183708 +183709 POINT(39.89468785792926 73.19659109708414) bank183709 +183710 POINT(41.264949370133714 74.40995622178504) bank183710 +183711 POINT(39.99425445069836 73.58534404514482) bank183711 +183712 POINT(39.880418645787245 74.93971931704391) bank183712 +183713 POINT(40.16887749104508 74.53662042400275) bank183713 +183714 POINT(40.07353583335435 73.47990912146294) bank183714 +183715 POINT(41.314528231596704 73.73632017636382) bank183715 +183716 POINT(40.20623087922776 73.13432968706302) bank183716 +183717 POINT(41.49357724578282 73.08987351080515) bank183717 +183718 POINT(39.9546604057361 74.14797525831895) bank183718 +183719 POINT(40.025330187177374 74.85497400397246) bank183719 +183720 POINT(40.43711847405218 73.0673210869986) bank183720 +183721 POINT(41.17073081990153 73.94486812723864) bank183721 +183722 POINT(40.85572938523386 73.0493829390563) bank183722 +183723 POINT(41.2709379306588 73.04184782798826) bank183723 +183724 POINT(40.2132860542379 73.95824242120133) bank183724 +183725 POINT(40.85168536353212 73.16774944441) bank183725 +183726 POINT(41.073606888195904 73.64974015325605) bank183726 +183727 POINT(40.269784359139365 74.73363487975044) bank183727 +183728 POINT(40.16541922977713 73.13504266398084) bank183728 +183729 POINT(39.939566627009064 74.42805343591125) bank183729 +183730 POINT(40.878315707204095 73.55096994252898) bank183730 +183731 POINT(39.932898838518526 73.88096502918295) bank183731 +183732 POINT(39.89951553536599 73.03742468975472) bank183732 +183733 POINT(40.40342659235642 73.54842096847818) bank183733 +183734 POINT(39.88888912430468 74.92472922652661) bank183734 +183735 POINT(40.59475979554861 74.90171965063462) bank183735 +183736 POINT(40.80936120469772 73.39905915332348) bank183736 +183737 POINT(40.93182483994226 73.11295546727638) bank183737 +183738 POINT(40.40534276312132 73.26779687278355) bank183738 +183739 POINT(40.41894132125335 74.61098203171166) bank183739 +183740 POINT(40.43696328549144 73.09020135687638) bank183740 +183741 POINT(40.98046054078629 74.05152425755215) bank183741 +183742 POINT(40.89213996282977 73.54110078590519) bank183742 +183743 POINT(41.279877982794645 73.9082420889129) bank183743 +183744 POINT(39.92038043093721 74.04059259508372) bank183744 +183745 POINT(40.334541693510715 73.72254000858662) bank183745 +183746 POINT(39.77389444019827 73.89430359375763) bank183746 +183747 POINT(39.97722827313076 74.73703484853615) bank183747 +183748 POINT(41.16676415107369 74.90643552714019) bank183748 +183749 POINT(41.30380491279646 73.9150891302607) bank183749 +183750 POINT(41.240475892935585 74.87386224615082) bank183750 +183751 POINT(40.671555867095556 73.83194754771684) bank183751 +183752 POINT(39.86080761989081 74.3253081114441) bank183752 +183753 POINT(39.84679158784135 74.28090069047447) bank183753 +183754 POINT(39.92784054379465 73.77227148645594) bank183754 +183755 POINT(41.49799351272614 73.33352050998135) bank183755 +183756 POINT(39.9741218698589 74.43517949198309) bank183756 +183757 POINT(41.09693883532287 73.33515787591438) bank183757 +183758 POINT(40.885427956788334 74.49224401266261) bank183758 +183759 POINT(39.8958905597148 73.54052212523577) bank183759 +183760 POINT(40.406231809311606 74.98588896715754) bank183760 +183761 POINT(40.78225118830616 73.45339665080193) bank183761 +183762 POINT(40.153043456401626 73.80024848446763) bank183762 +183763 POINT(40.1426849287813 73.2946035499386) bank183763 +183764 POINT(41.62949463253251 74.64519278629612) bank183764 +183765 POINT(41.150632564161 73.6908978457899) bank183765 +183766 POINT(41.06873146090822 74.42140704567562) bank183766 +183767 POINT(40.58187533046179 74.04329755465952) bank183767 +183768 POINT(41.47547978604255 73.44220231500289) bank183768 +183769 POINT(40.754575405451476 74.85893486048384) bank183769 +183770 POINT(41.04749477332663 74.96006639299617) bank183770 +183771 POINT(40.46044752936852 73.75440731550198) bank183771 +183772 POINT(40.31428141785671 73.9214413605328) bank183772 +183773 POINT(40.67729414277188 73.69699180082165) bank183773 +183774 POINT(41.51997581447385 73.49290789633396) bank183774 +183775 POINT(41.401969697673 73.21914192173801) bank183775 +183776 POINT(41.632709781176466 74.50271833324435) bank183776 +183777 POINT(41.36837543589104 74.02349477278386) bank183777 +183778 POINT(41.515762021637 73.58381817335689) bank183778 +183779 POINT(40.13923165409621 73.34220563314544) bank183779 +183780 POINT(39.76708721170354 74.63500393350313) bank183780 +183781 POINT(40.14277833860793 73.28967951096229) bank183781 +183782 POINT(41.46584329082337 73.49443181160893) bank183782 +183783 POINT(40.52563858257241 74.62293187707812) bank183783 +183784 POINT(40.01553701088158 73.6981170078499) bank183784 +183785 POINT(40.629464264800575 74.35877620845369) bank183785 +183786 POINT(39.735419015781076 74.85209807756618) bank183786 +183787 POINT(40.310291301148936 73.45335882234784) bank183787 +183788 POINT(39.78839525424799 74.8895811064134) bank183788 +183789 POINT(40.54058463608627 74.49914374791199) bank183789 +183790 POINT(40.37268446377659 74.51001857259315) bank183790 +183791 POINT(40.93811723985632 73.92016463606112) bank183791 +183792 POINT(40.18695128709114 73.21566458725863) bank183792 +183793 POINT(39.82433787100354 74.25853404103958) bank183793 +183794 POINT(40.77357909342451 73.3542261177558) bank183794 +183795 POINT(40.13198803456973 74.48158840477348) bank183795 +183796 POINT(41.4471943675539 74.05396595246246) bank183796 +183797 POINT(39.89878646826135 73.65882133937346) bank183797 +183798 POINT(40.14564158432589 73.38179568303883) bank183798 +183799 POINT(40.22046065501146 74.18512397901128) bank183799 +183800 POINT(41.092922647554566 74.08088098303817) bank183800 +183801 POINT(41.3693531419905 73.4418565841704) bank183801 +183802 POINT(41.225673437092325 74.48700717752446) bank183802 +183803 POINT(41.49579354046228 73.75057236302177) bank183803 +183804 POINT(40.04235177074228 74.44387446063234) bank183804 +183805 POINT(41.69008128652022 74.38386451485911) bank183805 +183806 POINT(41.32527825622264 73.57429631945925) bank183806 +183807 POINT(40.52476051616975 74.9507366109515) bank183807 +183808 POINT(41.00246872082204 74.13955588336327) bank183808 +183809 POINT(41.38076996250758 73.37920981482254) bank183809 +183810 POINT(40.454765786870986 74.17107241012049) bank183810 +183811 POINT(40.77795828254653 73.07201759915897) bank183811 +183812 POINT(39.72244180119094 73.89591925167623) bank183812 +183813 POINT(39.982703926860935 73.39726545632608) bank183813 +183814 POINT(40.52399808362661 73.79525535411298) bank183814 +183815 POINT(40.36857582499053 73.6485273066854) bank183815 +183816 POINT(40.05714203358805 73.21418264365515) bank183816 +183817 POINT(40.19419086284997 74.96330680717085) bank183817 +183818 POINT(39.929751041688064 73.48137746451886) bank183818 +183819 POINT(41.21771768379911 73.74960537736686) bank183819 +183820 POINT(40.25575349669722 73.58246667421878) bank183820 +183821 POINT(41.55034115719837 74.1890300224315) bank183821 +183822 POINT(40.79647566540876 73.38752348490979) bank183822 +183823 POINT(39.938398617083045 73.58657174107834) bank183823 +183824 POINT(39.85424073440226 74.84340394863088) bank183824 +183825 POINT(40.163058715017065 74.90497691087495) bank183825 +183826 POINT(40.91112848028308 73.25499539783402) bank183826 +183827 POINT(39.798784991359994 73.49666287078851) bank183827 +183828 POINT(41.008363811920354 74.65623446417143) bank183828 +183829 POINT(39.79245580667396 74.80601720391763) bank183829 +183830 POINT(39.83192214784861 73.30302141057987) bank183830 +183831 POINT(40.073058128330445 74.92538815497429) bank183831 +183832 POINT(39.96066346912475 73.48215402282506) bank183832 +183833 POINT(40.96393223650624 73.01272431552837) bank183833 +183834 POINT(39.997053972677996 74.99296658784536) bank183834 +183835 POINT(41.607122972101706 73.41544010494013) bank183835 +183836 POINT(39.82825093641111 73.16681942659781) bank183836 +183837 POINT(39.77585135238644 73.16752659648832) bank183837 +183838 POINT(40.41466410037411 73.62117531976743) bank183838 +183839 POINT(40.25564642848444 73.58819703296277) bank183839 +183840 POINT(40.38680922007306 73.54679729571947) bank183840 +183841 POINT(41.385742374646945 74.67864929117107) bank183841 +183842 POINT(41.09540932370129 73.69225039719797) bank183842 +183843 POINT(40.191728046340515 73.1677086681776) bank183843 +183844 POINT(40.84269797457505 74.19504937023888) bank183844 +183845 POINT(40.65753885248914 74.90981069275335) bank183845 +183846 POINT(40.508846130865145 73.08669501771142) bank183846 +183847 POINT(40.45177845600131 73.9949485706048) bank183847 +183848 POINT(41.400647894382445 74.44198921208726) bank183848 +183849 POINT(40.62078614893842 73.9371394097214) bank183849 +183850 POINT(41.233661096084795 74.79760519642375) bank183850 +183851 POINT(40.83493430800111 73.41600037248135) bank183851 +183852 POINT(41.557398009155186 74.04352127827205) bank183852 +183853 POINT(40.63277970467297 73.32401511294404) bank183853 +183854 POINT(41.63659520512893 74.08511834451939) bank183854 +183855 POINT(40.78634041490999 73.33994372891786) bank183855 +183856 POINT(41.288906221507546 73.68007024360836) bank183856 +183857 POINT(40.894798217017865 74.37889496391907) bank183857 +183858 POINT(41.710776732248874 73.40774578731106) bank183858 +183859 POINT(40.708353515590595 73.59452909565776) bank183859 +183860 POINT(41.63422048145984 73.91706861081039) bank183860 +183861 POINT(40.51604280458371 73.27418260328676) bank183861 +183862 POINT(41.11511461005442 73.9803853567174) bank183862 +183863 POINT(41.516142722625574 73.05064829509212) bank183863 +183864 POINT(41.604446076819414 74.50093647771105) bank183864 +183865 POINT(40.80443146247483 74.69939386678823) bank183865 +183866 POINT(41.296468988290755 74.88409977281772) bank183866 +183867 POINT(40.19095529250689 73.45185925146146) bank183867 +183868 POINT(41.12914310532109 73.80781302864065) bank183868 +183869 POINT(39.75908209512323 74.511669520661) bank183869 +183870 POINT(40.00767448873563 74.88316429375686) bank183870 +183871 POINT(41.10508031717563 74.38458128525122) bank183871 +183872 POINT(40.73636112530412 73.05867036827925) bank183872 +183873 POINT(40.1664644872904 73.45546943555809) bank183873 +183874 POINT(41.181789893970496 74.83635379725216) bank183874 +183875 POINT(40.50211695829783 73.83246926676033) bank183875 +183876 POINT(40.46448349521494 73.1807376456485) bank183876 +183877 POINT(39.851138670854645 74.05509833122959) bank183877 +183878 POINT(41.37408194247792 73.42226456373774) bank183878 +183879 POINT(41.46001803195537 73.99101540265312) bank183879 +183880 POINT(39.84345169222855 73.07397108366393) bank183880 +183881 POINT(41.30448740165739 74.23906924511044) bank183881 +183882 POINT(40.23349638890656 74.84549362612785) bank183882 +183883 POINT(40.622467564215256 74.70096930803793) bank183883 +183884 POINT(39.84047331532428 74.3245592699567) bank183884 +183885 POINT(40.98586876726169 74.18087043022075) bank183885 +183886 POINT(39.71716568794139 74.92933269807574) bank183886 +183887 POINT(40.87372850239395 74.84180650836444) bank183887 +183888 POINT(39.94116438507403 73.32523878585135) bank183888 +183889 POINT(40.970586213938496 73.92599826522887) bank183889 +183890 POINT(39.73749712605848 74.93844427554063) bank183890 +183891 POINT(40.5061329287339 73.36431732969984) bank183891 +183892 POINT(39.997363420214256 74.8933881943385) bank183892 +183893 POINT(39.97282758595204 74.10662007471906) bank183893 +183894 POINT(39.93145637062291 73.7010025664589) bank183894 +183895 POINT(41.62187853056185 74.91189317949123) bank183895 +183896 POINT(41.58642900463676 74.50526108737031) bank183896 +183897 POINT(41.42508809818855 73.72682082381439) bank183897 +183898 POINT(40.072715154363024 74.84281068155342) bank183898 +183899 POINT(39.8545089445279 73.27483768074383) bank183899 +183900 POINT(41.034739857666885 74.77398923919364) bank183900 +183901 POINT(40.965487276190146 74.68601441135444) bank183901 +183902 POINT(40.77814096920024 73.18776495126095) bank183902 +183903 POINT(41.110054013996454 73.20838420973674) bank183903 +183904 POINT(41.33269976750912 74.34940528064162) bank183904 +183905 POINT(41.20876087688095 74.73386494162197) bank183905 +183906 POINT(39.976998363456495 74.80419411852388) bank183906 +183907 POINT(41.507708816648524 73.36865522827432) bank183907 +183908 POINT(40.83375019114946 73.03193719947812) bank183908 +183909 POINT(40.5275481470725 74.53230965965118) bank183909 +183910 POINT(41.470887614657435 73.6294821760947) bank183910 +183911 POINT(40.05251569794044 73.66260251754744) bank183911 +183912 POINT(41.59429269884518 74.05291505872651) bank183912 +183913 POINT(40.132631708731644 73.91409575246095) bank183913 +183914 POINT(40.386639558716716 74.43207679545154) bank183914 +183915 POINT(40.48936158413437 74.15254315104329) bank183915 +183916 POINT(41.08527474582573 74.7096923184967) bank183916 +183917 POINT(40.195112850652365 74.59020741867415) bank183917 +183918 POINT(40.08847400904343 73.40241055758756) bank183918 +183919 POINT(40.099326375728936 74.52817576430415) bank183919 +183920 POINT(40.61531240453837 74.06999427692682) bank183920 +183921 POINT(39.858027505417695 73.35022219162566) bank183921 +183922 POINT(40.29955030653776 74.30974143420866) bank183922 +183923 POINT(40.41736998538633 74.88438789326247) bank183923 +183924 POINT(41.19536307955835 73.53098914396806) bank183924 +183925 POINT(39.7679733066247 73.17608629237256) bank183925 +183926 POINT(40.180111083007105 73.92803116660686) bank183926 +183927 POINT(40.9678759366341 73.35164035091185) bank183927 +183928 POINT(40.19560064002271 73.54399978420444) bank183928 +183929 POINT(40.34396896848632 73.78922284697093) bank183929 +183930 POINT(40.4231267209558 74.09042030330285) bank183930 +183931 POINT(40.35770890704092 73.89374051266071) bank183931 +183932 POINT(41.457907761608034 73.48867419995835) bank183932 +183933 POINT(40.58169209218988 73.95831604730708) bank183933 +183934 POINT(41.094591116764185 74.17915295585937) bank183934 +183935 POINT(41.261157487166194 73.01826062505774) bank183935 +183936 POINT(41.7072373221569 73.97439834394814) bank183936 +183937 POINT(40.53813283653494 74.27210131275501) bank183937 +183938 POINT(39.96533652471372 74.02087377962228) bank183938 +183939 POINT(41.25685052093433 74.5808687576501) bank183939 +183940 POINT(39.938947533494115 73.99949726595007) bank183940 +183941 POINT(41.255766792039275 74.68874194628754) bank183941 +183942 POINT(41.23782545143596 73.08055180744662) bank183942 +183943 POINT(41.04300168851201 73.21465341300438) bank183943 +183944 POINT(40.06022638966798 73.5573307600487) bank183944 +183945 POINT(41.17102793030551 73.34126209372421) bank183945 +183946 POINT(39.96396965814951 74.74105705631858) bank183946 +183947 POINT(40.29281727857656 73.75088460510628) bank183947 +183948 POINT(40.36825602972603 74.15588286377451) bank183948 +183949 POINT(40.21395939083544 74.4157061720546) bank183949 +183950 POINT(40.5423500170079 74.77305960632891) bank183950 +183951 POINT(39.86131928997913 74.24476411244314) bank183951 +183952 POINT(39.83450999566213 74.0303243668656) bank183952 +183953 POINT(39.71434106983425 73.94697144958442) bank183953 +183954 POINT(41.65708370406859 73.21872280986706) bank183954 +183955 POINT(40.00468126420404 74.8479116089301) bank183955 +183956 POINT(41.07394775711305 73.7726717159736) bank183956 +183957 POINT(39.76782232756607 74.87769734894995) bank183957 +183958 POINT(40.73250657399919 73.75148238367237) bank183958 +183959 POINT(40.613087867812446 73.23419678909939) bank183959 +183960 POINT(40.28569987056901 74.19400708093436) bank183960 +183961 POINT(40.31653612619838 73.50840376218551) bank183961 +183962 POINT(40.99444489580878 73.10817286804138) bank183962 +183963 POINT(41.61056293687919 73.42443826601928) bank183963 +183964 POINT(40.53192273621224 74.90414999683195) bank183964 +183965 POINT(41.11088561736728 74.69346859690724) bank183965 +183966 POINT(40.6136896108507 73.6543678977604) bank183966 +183967 POINT(41.151022066243826 74.1319662014701) bank183967 +183968 POINT(40.21614905286978 73.83835372205239) bank183968 +183969 POINT(41.23137209237347 73.52028901535431) bank183969 +183970 POINT(41.00542233933635 74.50260943617792) bank183970 +183971 POINT(39.77075822567933 73.10650849282577) bank183971 +183972 POINT(40.5121030036844 74.71024299748073) bank183972 +183973 POINT(40.0210704997524 73.47830260666387) bank183973 +183974 POINT(40.973190077700295 73.68440279680323) bank183974 +183975 POINT(40.16860923186691 73.4573414004728) bank183975 +183976 POINT(41.45305255172422 73.72248148270313) bank183976 +183977 POINT(40.189976267805164 74.89016724477878) bank183977 +183978 POINT(41.61944257734775 74.918162446984) bank183978 +183979 POINT(40.74069566194523 75.00219747227347) bank183979 +183980 POINT(41.06582667305753 74.27352038805785) bank183980 +183981 POINT(41.699473540950294 73.76975650167732) bank183981 +183982 POINT(40.575660290854714 74.01261011645619) bank183982 +183983 POINT(40.81989767810893 74.1367904140689) bank183983 +183984 POINT(41.275373365049376 74.13418693457714) bank183984 +183985 POINT(40.28467498253915 74.23570954562682) bank183985 +183986 POINT(40.17813386274527 73.16179776235273) bank183986 +183987 POINT(41.44161430329507 74.02162026260119) bank183987 +183988 POINT(40.171611379162634 73.61054864790825) bank183988 +183989 POINT(40.846083953476366 73.74358822731911) bank183989 +183990 POINT(40.224352088639776 74.25984888298797) bank183990 +183991 POINT(40.19234005506631 73.51767431401366) bank183991 +183992 POINT(41.60654259527089 73.49323788745933) bank183992 +183993 POINT(39.767492579163616 74.94392731996086) bank183993 +183994 POINT(40.86046320426696 73.60552097607201) bank183994 +183995 POINT(41.02090888147339 73.5383487081535) bank183995 +183996 POINT(40.9735600803611 73.2050269409173) bank183996 +183997 POINT(41.51669965153961 73.04842844670077) bank183997 +183998 POINT(40.89891652559853 73.40297129765905) bank183998 +183999 POINT(39.74311301355028 73.79218232781278) bank183999 +184000 POINT(40.224781895840344 74.03339527368352) bank184000 +184001 POINT(40.686537367971205 74.56642422320797) bank184001 +184002 POINT(41.29383114348222 74.024410567122) bank184002 +184003 POINT(40.667978551587616 74.97587773113811) bank184003 +184004 POINT(40.902552503266875 74.76385931318784) bank184004 +184005 POINT(40.788238523464386 73.79418192010303) bank184005 +184006 POINT(40.28728212673831 74.79736902441203) bank184006 +184007 POINT(41.033500092105 73.83197994177866) bank184007 +184008 POINT(41.57546712346346 73.04370368246755) bank184008 +184009 POINT(40.122252451854514 73.25970011703244) bank184009 +184010 POINT(41.02859581229465 73.92756949952914) bank184010 +184011 POINT(40.38456497290384 73.85042574171003) bank184011 +184012 POINT(40.117359945729014 74.13426373038905) bank184012 +184013 POINT(40.627714664573915 74.07085765723737) bank184013 +184014 POINT(40.743175921719185 73.71065103569562) bank184014 +184015 POINT(41.51886822045228 73.73836242830814) bank184015 +184016 POINT(40.230349603613696 74.43600487159172) bank184016 +184017 POINT(40.997246694470796 73.19881372572897) bank184017 +184018 POINT(40.132933083277536 74.83553477263435) bank184018 +184019 POINT(40.74153282463912 74.81332761104372) bank184019 +184020 POINT(40.06578554537074 73.57466155882229) bank184020 +184021 POINT(41.70705668471837 74.34828898145273) bank184021 +184022 POINT(41.705846500910596 73.88051279387857) bank184022 +184023 POINT(40.110865480026774 74.00815713696004) bank184023 +184024 POINT(40.37316245415272 74.29985399900764) bank184024 +184025 POINT(41.686317961924914 74.31763531668223) bank184025 +184026 POINT(40.828408920364865 73.14719886468149) bank184026 +184027 POINT(41.67866151299056 73.66800078579149) bank184027 +184028 POINT(40.083774097012885 74.28019537051973) bank184028 +184029 POINT(40.83979783791878 74.62462379385633) bank184029 +184030 POINT(40.91762176418729 73.57534385349223) bank184030 +184031 POINT(39.80027665746498 73.03010276602667) bank184031 +184032 POINT(40.992629278054736 73.54547103016002) bank184032 +184033 POINT(39.71971940202853 73.3414561593071) bank184033 +184034 POINT(40.98852782537454 73.87037338001443) bank184034 +184035 POINT(39.83851950427565 73.01897033542845) bank184035 +184036 POINT(39.95829918683902 73.80247619579706) bank184036 +184037 POINT(41.506962720772364 74.02657737112067) bank184037 +184038 POINT(41.29694383515457 73.40068898883423) bank184038 +184039 POINT(41.61944134914127 73.95351095434141) bank184039 +184040 POINT(41.653938136986646 73.53449874164268) bank184040 +184041 POINT(40.00283414265237 73.44581263494939) bank184041 +184042 POINT(41.485175374534116 74.6288579946392) bank184042 +184043 POINT(41.45892994822685 73.34339212662005) bank184043 +184044 POINT(39.714724879378466 73.38395649189678) bank184044 +184045 POINT(39.824889975376195 74.50893725647525) bank184045 +184046 POINT(40.41465562024212 73.5301876370351) bank184046 +184047 POINT(41.68815598001043 73.60015085643504) bank184047 +184048 POINT(39.95405501256756 73.86268750588502) bank184048 +184049 POINT(39.96107249047831 73.04176924701528) bank184049 +184050 POINT(40.487804977929954 73.73748081386124) bank184050 +184051 POINT(39.71676489311562 73.42864028070774) bank184051 +184052 POINT(39.78289324634968 73.65695207064198) bank184052 +184053 POINT(39.90119390195613 73.56598593813204) bank184053 +184054 POINT(40.49189630183509 74.60091659577942) bank184054 +184055 POINT(41.56861849073944 73.97682802751979) bank184055 +184056 POINT(39.907594179393755 73.48685678957305) bank184056 +184057 POINT(39.89081730924191 73.62480710528787) bank184057 +184058 POINT(40.34670757866744 73.33288820478839) bank184058 +184059 POINT(40.19265458907558 74.92353218843257) bank184059 +184060 POINT(41.34059327971436 73.65551407165472) bank184060 +184061 POINT(39.93464521964053 73.34551674365282) bank184061 +184062 POINT(40.61075553407738 74.25764459028136) bank184062 +184063 POINT(39.899599185377944 73.62916388127107) bank184063 +184064 POINT(41.253505067957235 73.63891970115479) bank184064 +184065 POINT(41.37825493578226 73.48116566472024) bank184065 +184066 POINT(41.70358505569628 73.92142001282339) bank184066 +184067 POINT(40.45147405418443 74.80734439046654) bank184067 +184068 POINT(41.22823099402106 73.89983038558134) bank184068 +184069 POINT(40.9099180815988 73.00741212478884) bank184069 +184070 POINT(41.39038644746404 74.13430162073682) bank184070 +184071 POINT(41.58639798122164 74.58477106597628) bank184071 +184072 POINT(41.18512060206967 74.08419115403892) bank184072 +184073 POINT(41.339535786535976 74.50115566185305) bank184073 +184074 POINT(41.67316410394571 74.31297834486458) bank184074 +184075 POINT(40.65002329722163 74.76048494730891) bank184075 +184076 POINT(40.71686321780609 73.42569271688795) bank184076 +184077 POINT(40.746071375786556 73.7948202180503) bank184077 +184078 POINT(41.01848138454811 74.8078913158692) bank184078 +184079 POINT(41.612921740960886 73.04244828894876) bank184079 +184080 POINT(40.23243693136733 73.67708725514915) bank184080 +184081 POINT(41.5892295080024 73.66017263415227) bank184081 +184082 POINT(41.486650558954224 73.3708629561688) bank184082 +184083 POINT(41.70901992659449 74.50947690386242) bank184083 +184084 POINT(40.4514190198647 73.25431575899144) bank184084 +184085 POINT(40.92820109060068 74.19145670684547) bank184085 +184086 POINT(40.52360352599797 74.0361353367978) bank184086 +184087 POINT(40.20270336061472 73.53562469918727) bank184087 +184088 POINT(39.915710373034365 74.47436507140425) bank184088 +184089 POINT(40.21567711218672 74.54829737091939) bank184089 +184090 POINT(40.40206308366409 74.80459569279213) bank184090 +184091 POINT(41.64309262908165 74.27465893788948) bank184091 +184092 POINT(40.920797665854685 73.75910847420366) bank184092 +184093 POINT(41.215943323809725 74.96450385528749) bank184093 +184094 POINT(40.17049737278857 73.75572321809904) bank184094 +184095 POINT(40.19108317156672 73.98587782126937) bank184095 +184096 POINT(40.22027385644365 74.54371798666936) bank184096 +184097 POINT(40.85072561963998 74.9573743593907) bank184097 +184098 POINT(40.78139110164852 73.17012411123271) bank184098 +184099 POINT(41.65307822347817 73.75098045967064) bank184099 +184100 POINT(40.37514213559953 74.29806560168437) bank184100 +184101 POINT(41.38592985986432 73.12022216974606) bank184101 +184102 POINT(41.65066724356687 74.81270615120103) bank184102 +184103 POINT(41.2472671122291 74.15489045895649) bank184103 +184104 POINT(41.32724424334747 73.67585832796445) bank184104 +184105 POINT(41.53005784998339 73.21382937197744) bank184105 +184106 POINT(40.73231822117476 73.53780809352098) bank184106 +184107 POINT(40.649718905475886 73.67930876574218) bank184107 +184108 POINT(41.69482626290872 73.46134923683803) bank184108 +184109 POINT(40.69574753242213 73.44559014760023) bank184109 +184110 POINT(39.75061803475837 73.1093195582846) bank184110 +184111 POINT(40.97858096901854 73.48604634818157) bank184111 +184112 POINT(40.195980586401475 74.25164661351907) bank184112 +184113 POINT(40.615749910554 73.41232540847322) bank184113 +184114 POINT(41.28521998639123 74.08470740486403) bank184114 +184115 POINT(41.24092864792584 73.5873794494164) bank184115 +184116 POINT(39.91193401094861 74.71065242438492) bank184116 +184117 POINT(39.72175326502634 73.85182851333204) bank184117 +184118 POINT(40.25707718343276 74.12796802665555) bank184118 +184119 POINT(40.776469836546056 73.44082230557947) bank184119 +184120 POINT(41.477932792523504 74.23370433582969) bank184120 +184121 POINT(40.80405530453608 73.86744832812785) bank184121 +184122 POINT(40.63829110055098 74.5743289056729) bank184122 +184123 POINT(40.324363762713375 73.13748899526033) bank184123 +184124 POINT(40.23236366573279 74.50825995203667) bank184124 +184125 POINT(41.63529287119986 74.76262624902405) bank184125 +184126 POINT(40.755353584341215 74.36182916561047) bank184126 +184127 POINT(40.093359355881844 73.9740324500682) bank184127 +184128 POINT(40.01894165964852 73.27152764825072) bank184128 +184129 POINT(40.90059421486919 74.76160415190124) bank184129 +184130 POINT(41.40075406374566 73.61314765495275) bank184130 +184131 POINT(41.0286537353352 74.87588001695663) bank184131 +184132 POINT(41.52015973167381 73.2269550872302) bank184132 +184133 POINT(40.64821100775029 74.25875907809252) bank184133 +184134 POINT(40.94393900131597 73.88006878263072) bank184134 +184135 POINT(40.780859188142216 74.18087701288235) bank184135 +184136 POINT(39.82999858241762 74.12475725847335) bank184136 +184137 POINT(39.84927371422004 74.35138552184387) bank184137 +184138 POINT(40.638478998485425 74.56002567003381) bank184138 +184139 POINT(40.811711301681136 74.24809974146646) bank184139 +184140 POINT(40.23600501295443 74.27230485604069) bank184140 +184141 POINT(40.25684048299816 73.30625759391376) bank184141 +184142 POINT(40.188311345075874 73.41864285146379) bank184142 +184143 POINT(40.58802473616073 74.57782768693795) bank184143 +184144 POINT(40.109732374970584 73.76945596696876) bank184144 +184145 POINT(41.50372323857874 73.81205319173726) bank184145 +184146 POINT(41.258353916536215 74.36462323720586) bank184146 +184147 POINT(41.23988363498834 73.57416680706723) bank184147 +184148 POINT(39.80967011270888 74.04009334640509) bank184148 +184149 POINT(40.2275381271443 74.39267445853288) bank184149 +184150 POINT(40.12097954680475 74.53277817851946) bank184150 +184151 POINT(41.450051523026836 73.12942628060543) bank184151 +184152 POINT(40.39959288048833 73.34088464653757) bank184152 +184153 POINT(41.52199446239297 73.83325558954813) bank184153 +184154 POINT(40.36140790114437 74.7688960447838) bank184154 +184155 POINT(39.90045061451815 73.81801532843286) bank184155 +184156 POINT(41.38110891703119 74.01892269240618) bank184156 +184157 POINT(41.30455268627534 73.35163180816787) bank184157 +184158 POINT(40.72954893044928 74.38692765771019) bank184158 +184159 POINT(40.229047031669275 74.88109433419923) bank184159 +184160 POINT(40.153116669032464 74.47011901472919) bank184160 +184161 POINT(41.67263542858595 74.49443197711966) bank184161 +184162 POINT(41.13801995592583 74.235517109026) bank184162 +184163 POINT(41.02265315607495 73.95083686241449) bank184163 +184164 POINT(40.2155658913508 74.63021175178228) bank184164 +184165 POINT(41.16653606210443 74.73789608361449) bank184165 +184166 POINT(41.70709133980237 74.76292007060202) bank184166 +184167 POINT(41.55950382031022 74.81041821237612) bank184167 +184168 POINT(40.640317608458766 74.42357264753018) bank184168 +184169 POINT(40.258936360388944 73.25922474412494) bank184169 +184170 POINT(41.63591971042019 74.52548391584666) bank184170 +184171 POINT(41.14087198843135 74.18141055193868) bank184171 +184172 POINT(41.135617443354484 74.0568769486093) bank184172 +184173 POINT(41.5117739991667 74.78439963835432) bank184173 +184174 POINT(39.81478059248136 73.20574778045156) bank184174 +184175 POINT(41.43213542902808 74.82713837699873) bank184175 +184176 POINT(40.90395374253412 74.46358211412752) bank184176 +184177 POINT(40.79998729030155 74.05534438984479) bank184177 +184178 POINT(41.27022655448829 74.47982215311724) bank184178 +184179 POINT(40.44958867231978 73.64444401666437) bank184179 +184180 POINT(40.997703941751844 73.97799223139108) bank184180 +184181 POINT(41.41985564811951 74.12175895268305) bank184181 +184182 POINT(40.1856769946807 74.52742753394975) bank184182 +184183 POINT(40.94960578729677 74.86936939068461) bank184183 +184184 POINT(41.20374700027049 74.97703756582071) bank184184 +184185 POINT(41.183758939036586 74.90823495893683) bank184185 +184186 POINT(40.94377138901623 73.8871216457672) bank184186 +184187 POINT(41.42271473898632 73.7480308641862) bank184187 +184188 POINT(39.91578186463539 74.17861077843774) bank184188 +184189 POINT(40.142385747394314 73.58175165551701) bank184189 +184190 POINT(41.21963788376674 73.2669930727541) bank184190 +184191 POINT(40.090828117000974 73.10838534800652) bank184191 +184192 POINT(39.961729434812305 73.73010144502462) bank184192 +184193 POINT(41.42423158822065 73.43122395427457) bank184193 +184194 POINT(40.600474473697034 74.07540302515925) bank184194 +184195 POINT(40.71948089667873 73.90772651406866) bank184195 +184196 POINT(41.50368090085274 73.27305672939643) bank184196 +184197 POINT(41.22620278198114 74.37171297363355) bank184197 +184198 POINT(40.84291535148484 73.275957827361) bank184198 +184199 POINT(39.95651572181159 73.95716353546237) bank184199 +184200 POINT(39.81516283404057 74.0244440528547) bank184200 +184201 POINT(41.05367317473324 74.28066645048015) bank184201 +184202 POINT(39.89887681703128 74.9920461876694) bank184202 +184203 POINT(40.48117770563853 74.42090462435623) bank184203 +184204 POINT(40.498161212110176 74.93483110646152) bank184204 +184205 POINT(40.14458806308865 74.33395088573091) bank184205 +184206 POINT(41.021594874644926 73.69448668300764) bank184206 +184207 POINT(41.6856046898793 73.82839072025803) bank184207 +184208 POINT(40.73700606502803 74.50069356077465) bank184208 +184209 POINT(40.53122826089973 73.92605981894648) bank184209 +184210 POINT(41.38769072943316 74.75166529542408) bank184210 +184211 POINT(40.31205154321074 73.366913827691) bank184211 +184212 POINT(40.24748367107044 73.32966884826033) bank184212 +184213 POINT(40.940220305432504 74.5047184377018) bank184213 +184214 POINT(40.65282777168332 73.07956827373603) bank184214 +184215 POINT(39.76203110533626 73.57072903512912) bank184215 +184216 POINT(41.09640156779559 73.4494495763318) bank184216 +184217 POINT(41.23158080278949 73.39513511600754) bank184217 +184218 POINT(41.03002505740338 73.31622626610974) bank184218 +184219 POINT(41.10702982850649 74.36542085585708) bank184219 +184220 POINT(40.9533201039839 73.44869005679419) bank184220 +184221 POINT(41.32695129327034 74.2851753377242) bank184221 +184222 POINT(40.84346468977511 73.77937660883572) bank184222 +184223 POINT(41.44335970579461 73.96684883769886) bank184223 +184224 POINT(41.30126093049842 73.58737992776678) bank184224 +184225 POINT(39.990721239343074 74.50303142770574) bank184225 +184226 POINT(40.19847400910736 73.56167474277315) bank184226 +184227 POINT(40.46476922986006 73.5363331295724) bank184227 +184228 POINT(41.61049410534839 74.00380437078869) bank184228 +184229 POINT(41.27390551626698 73.5987795416573) bank184229 +184230 POINT(40.82908480907771 74.85977904595622) bank184230 +184231 POINT(41.438740651449066 74.19472215924745) bank184231 +184232 POINT(40.64460654274915 73.58836896972743) bank184232 +184233 POINT(39.983905476589385 74.57884939279666) bank184233 +184234 POINT(40.738558637177846 74.27051127098366) bank184234 +184235 POINT(41.19377564988087 73.93474272699405) bank184235 +184236 POINT(40.48923904029759 73.61457661351623) bank184236 +184237 POINT(41.60612865802082 74.75664932705891) bank184237 +184238 POINT(40.95117434785757 73.14175168040954) bank184238 +184239 POINT(40.23340334852695 74.64807279197018) bank184239 +184240 POINT(40.66513094187951 73.68732515096971) bank184240 +184241 POINT(40.97677452212712 74.39602147904104) bank184241 +184242 POINT(40.25637451755066 74.88876329402055) bank184242 +184243 POINT(41.436050129320435 73.96830760367365) bank184243 +184244 POINT(39.74168499928452 73.90390869697605) bank184244 +184245 POINT(41.53554291333699 73.54032699576415) bank184245 +184246 POINT(41.12771265400154 73.3541048174389) bank184246 +184247 POINT(41.227970064468856 74.67739566246438) bank184247 +184248 POINT(39.90222156421773 74.87135828022042) bank184248 +184249 POINT(40.50831709865795 73.86807099226719) bank184249 +184250 POINT(40.06306630890534 73.82170996885606) bank184250 +184251 POINT(40.745370174406126 73.17082387833871) bank184251 +184252 POINT(39.7146346959794 73.3684923056464) bank184252 +184253 POINT(40.885714366211545 74.6417060391487) bank184253 +184254 POINT(40.26935039973107 74.52310368150646) bank184254 +184255 POINT(41.01577078542821 73.39142403807826) bank184255 +184256 POINT(40.86300262113794 73.55852258539844) bank184256 +184257 POINT(40.76128523787357 74.81349026771869) bank184257 +184258 POINT(41.05464401569982 74.04074418375001) bank184258 +184259 POINT(39.82785485295634 73.92836868624259) bank184259 +184260 POINT(41.32152688060117 73.71323702327932) bank184260 +184261 POINT(40.955339275569365 74.10439296246675) bank184261 +184262 POINT(40.84861505729954 73.27393069045718) bank184262 +184263 POINT(40.15638146990813 73.89812533034848) bank184263 +184264 POINT(41.133087148965274 73.32298519326476) bank184264 +184265 POINT(39.76406197679634 74.88956548878267) bank184265 +184266 POINT(40.80744602860324 73.35809032792923) bank184266 +184267 POINT(40.14936662640281 73.63859299952905) bank184267 +184268 POINT(41.07357922298842 73.81635168966983) bank184268 +184269 POINT(40.77820954714419 74.63318978986102) bank184269 +184270 POINT(41.48363721874077 74.68238148370526) bank184270 +184271 POINT(41.229889773901746 74.62989886722417) bank184271 +184272 POINT(41.19502582946026 73.38860517211731) bank184272 +184273 POINT(41.26182597539192 73.22968522913592) bank184273 +184274 POINT(40.573378083019094 73.68931098747659) bank184274 +184275 POINT(40.42647767183455 73.13273114040251) bank184275 +184276 POINT(41.661993780461756 74.31313724355302) bank184276 +184277 POINT(40.82178415333135 74.27961518576866) bank184277 +184278 POINT(40.274077173702935 73.82353111176565) bank184278 +184279 POINT(40.4492306562686 74.33849823858777) bank184279 +184280 POINT(41.48196164939522 74.68272370517958) bank184280 +184281 POINT(39.92316787246878 73.88094059054758) bank184281 +184282 POINT(41.44110858407071 73.98519719340047) bank184282 +184283 POINT(41.52885907617416 73.89925890985856) bank184283 +184284 POINT(40.61572978958836 73.65731823347511) bank184284 +184285 POINT(41.0629286236189 73.84857012016403) bank184285 +184286 POINT(40.852738765035696 73.40433212601184) bank184286 +184287 POINT(40.46360150023023 74.38865156754807) bank184287 +184288 POINT(39.952804264061335 74.90812127159472) bank184288 +184289 POINT(40.800807573462855 73.87145407172618) bank184289 +184290 POINT(41.32417998075198 73.6088225231504) bank184290 +184291 POINT(39.88300438948911 74.39767455495524) bank184291 +184292 POINT(40.97870718513783 74.57007670093913) bank184292 +184293 POINT(40.23045318307484 74.29232108741513) bank184293 +184294 POINT(40.83586090425436 73.21053445215202) bank184294 +184295 POINT(41.587665543697746 73.79387868112576) bank184295 +184296 POINT(40.06760547973003 74.00235575351608) bank184296 +184297 POINT(41.214482968946456 74.46400786832606) bank184297 +184298 POINT(40.101022107942434 73.83049049550723) bank184298 +184299 POINT(41.159524404872506 73.96990571747244) bank184299 +184300 POINT(40.78982175182421 74.67819377820565) bank184300 +184301 POINT(41.3305330816848 73.38142583743722) bank184301 +184302 POINT(41.0234702150366 74.90665501189342) bank184302 +184303 POINT(41.05941698609594 74.9771613069542) bank184303 +184304 POINT(40.12568158408087 73.25824928123578) bank184304 +184305 POINT(40.42885543549243 74.85222873850466) bank184305 +184306 POINT(41.38937251475149 73.41104556544148) bank184306 +184307 POINT(39.9585555340576 73.4990181818098) bank184307 +184308 POINT(39.73388597013196 74.15777136544617) bank184308 +184309 POINT(41.30922683895568 73.29790165103827) bank184309 +184310 POINT(40.73819055191206 73.68260700520015) bank184310 +184311 POINT(39.89556905365657 74.87676448533593) bank184311 +184312 POINT(40.34270742536583 74.59336187154115) bank184312 +184313 POINT(40.15597026843141 74.70577847202779) bank184313 +184314 POINT(40.33408104594076 73.09946899755924) bank184314 +184315 POINT(40.491777563731894 74.4807333403205) bank184315 +184316 POINT(41.17387455430817 73.29598830161156) bank184316 +184317 POINT(40.34382521469417 73.45210162648466) bank184317 +184318 POINT(40.57447617472773 73.0290074627754) bank184318 +184319 POINT(40.86375466239776 73.22608023789128) bank184319 +184320 POINT(41.66496271961152 73.49016354972018) bank184320 +184321 POINT(40.74310961219323 74.00376195670644) bank184321 +184322 POINT(40.81083529000659 73.77241911799662) bank184322 +184323 POINT(40.130573479950534 74.34997872050539) bank184323 +184324 POINT(41.52146510306365 74.93047071380073) bank184324 +184325 POINT(40.58556306839731 73.43758903053558) bank184325 +184326 POINT(40.4266804014808 74.19491949692171) bank184326 +184327 POINT(39.96691005215961 74.13495629559529) bank184327 +184328 POINT(41.2705622188917 74.78911620314662) bank184328 +184329 POINT(41.53720690336442 74.9443670199564) bank184329 +184330 POINT(40.57562288190371 73.5944925495242) bank184330 +184331 POINT(41.1816622671 74.90052200731772) bank184331 +184332 POINT(41.62857792483242 73.1723068601034) bank184332 +184333 POINT(39.89947391295019 74.30494522980311) bank184333 +184334 POINT(41.37913398691539 74.92681910893073) bank184334 +184335 POINT(41.13444417546438 73.89751051153257) bank184335 +184336 POINT(41.27792098663568 74.65978453102755) bank184336 +184337 POINT(40.081428076904814 73.38727310035749) bank184337 +184338 POINT(39.815775983510925 73.23712139975284) bank184338 +184339 POINT(40.40221022848981 74.47753102889524) bank184339 +184340 POINT(40.759646334052185 74.3042463767002) bank184340 +184341 POINT(41.22355349604412 73.21880531713805) bank184341 +184342 POINT(40.573023136988915 73.28901953319065) bank184342 +184343 POINT(39.91124978169288 74.63134181720991) bank184343 +184344 POINT(40.93066288554854 73.88846182137588) bank184344 +184345 POINT(40.173383358468385 73.85564556212952) bank184345 +184346 POINT(41.16447848110049 73.40711477673224) bank184346 +184347 POINT(40.88503267533438 73.04623139938182) bank184347 +184348 POINT(39.80536984983134 74.95167961200912) bank184348 +184349 POINT(40.29064413314072 73.3128118356042) bank184349 +184350 POINT(40.39868408725238 74.55791447801879) bank184350 +184351 POINT(41.33634495360563 73.553858458067) bank184351 +184352 POINT(40.22679808760415 73.76736391515527) bank184352 +184353 POINT(41.65633685267973 73.85145651955519) bank184353 +184354 POINT(41.41017192721846 74.45408156333885) bank184354 +184355 POINT(40.98904009785127 73.31896747424412) bank184355 +184356 POINT(41.570027018253036 73.5849759816988) bank184356 +184357 POINT(40.148314142299306 74.35087535906518) bank184357 +184358 POINT(40.58772716331123 73.13690692546679) bank184358 +184359 POINT(40.345072436902846 73.38365338422572) bank184359 +184360 POINT(40.923492589079025 73.39169321022318) bank184360 +184361 POINT(41.56762307377766 73.8056207083315) bank184361 +184362 POINT(40.257644132570015 74.31328242607017) bank184362 +184363 POINT(41.46549048196307 73.39799454851173) bank184363 +184364 POINT(40.233108721039486 73.93308223986377) bank184364 +184365 POINT(40.631450023035384 74.92903162981993) bank184365 +184366 POINT(40.05888139755668 73.16571374280085) bank184366 +184367 POINT(40.084662225006106 73.27825071129342) bank184367 +184368 POINT(39.86258455211542 74.5781351469585) bank184368 +184369 POINT(40.904663191861275 73.44142797437848) bank184369 +184370 POINT(40.69110438098784 73.25453736263097) bank184370 +184371 POINT(40.31803567472401 74.72697699409919) bank184371 +184372 POINT(41.57732448042033 74.23424492415425) bank184372 +184373 POINT(40.10539197804664 73.12764982719924) bank184373 +184374 POINT(39.911928021986135 74.59701236856162) bank184374 +184375 POINT(41.13688357566561 74.86830058939998) bank184375 +184376 POINT(40.280126514963015 74.75664365677206) bank184376 +184377 POINT(41.39080416882323 73.06814318250568) bank184377 +184378 POINT(41.65067648295282 73.70410446024493) bank184378 +184379 POINT(40.926491519460725 74.55038467549474) bank184379 +184380 POINT(41.44434290702859 74.08182991899523) bank184380 +184381 POINT(40.31189071511188 74.52115532438356) bank184381 +184382 POINT(40.62166950052812 73.50214647202075) bank184382 +184383 POINT(41.631113547718385 74.6817524890761) bank184383 +184384 POINT(41.0793689953992 73.58400812397608) bank184384 +184385 POINT(41.682675986046725 74.57474116743919) bank184385 +184386 POINT(39.74430706339283 73.7890873736844) bank184386 +184387 POINT(41.26966783712516 73.91108086898963) bank184387 +184388 POINT(39.97567549536077 73.13199324101024) bank184388 +184389 POINT(41.476894033021814 73.64929230706188) bank184389 +184390 POINT(40.84487314837557 73.47109158511763) bank184390 +184391 POINT(41.16349305985688 73.52903841055289) bank184391 +184392 POINT(40.11178123859521 73.84351412510605) bank184392 +184393 POINT(41.06172887047351 74.37269986288919) bank184393 +184394 POINT(41.27244890496487 74.1006403238101) bank184394 +184395 POINT(40.251051307827545 74.83179587058322) bank184395 +184396 POINT(40.63805156851871 73.71377605103554) bank184396 +184397 POINT(41.4583180553251 73.03566970085403) bank184397 +184398 POINT(40.22990892473545 73.02246281326991) bank184398 +184399 POINT(40.65809621966034 73.05125802875929) bank184399 +184400 POINT(40.23659902304652 73.1078557354702) bank184400 +184401 POINT(40.902657397376444 74.95501846178362) bank184401 +184402 POINT(41.55554902490751 73.2274674572643) bank184402 +184403 POINT(39.80121335183972 74.84009416478467) bank184403 +184404 POINT(40.30518290838916 73.68491064877486) bank184404 +184405 POINT(40.40335419722116 73.96405340326127) bank184405 +184406 POINT(40.31880049925565 73.54895293426551) bank184406 +184407 POINT(41.46651118052768 73.80243000490603) bank184407 +184408 POINT(40.5074473997336 74.82102978382946) bank184408 +184409 POINT(40.46288772202019 73.5517961559517) bank184409 +184410 POINT(39.98016496689254 73.7429082128827) bank184410 +184411 POINT(39.924983648289995 74.37530447887926) bank184411 +184412 POINT(41.70277834455301 73.95444748019088) bank184412 +184413 POINT(39.862762241128415 73.75654096365096) bank184413 +184414 POINT(40.516422620575646 74.1929853792489) bank184414 +184415 POINT(40.33677296713101 73.79314473255828) bank184415 +184416 POINT(41.659778416440005 73.61291047460217) bank184416 +184417 POINT(40.585899501140894 73.62636866674686) bank184417 +184418 POINT(41.667826522381844 73.70779291022957) bank184418 +184419 POINT(41.003160459040856 74.26693118022722) bank184419 +184420 POINT(40.11820313094864 74.58140911860274) bank184420 +184421 POINT(40.86503264192928 74.97189045976604) bank184421 +184422 POINT(40.732464657715035 73.1099907291623) bank184422 +184423 POINT(41.18602080901771 73.47152270145847) bank184423 +184424 POINT(39.81092277544864 74.43430444518955) bank184424 +184425 POINT(41.47229534416945 73.2828487141905) bank184425 +184426 POINT(41.22451760344891 73.23829045344705) bank184426 +184427 POINT(40.72186006788867 74.34978090663078) bank184427 +184428 POINT(41.25022899008542 73.58060286774572) bank184428 +184429 POINT(41.27144919421764 73.22890557510216) bank184429 +184430 POINT(40.44737026086746 73.56679637898411) bank184430 +184431 POINT(39.963573199740146 73.3484556140556) bank184431 +184432 POINT(41.601091258838366 73.79886568012016) bank184432 +184433 POINT(40.26071183171305 74.90686168818753) bank184433 +184434 POINT(40.26277527819596 74.70027629368612) bank184434 +184435 POINT(39.854471742291835 73.55645444224939) bank184435 +184436 POINT(40.43947820327115 73.27774440486832) bank184436 +184437 POINT(40.49631681825643 74.3515674154217) bank184437 +184438 POINT(39.742344752458216 74.71231416783519) bank184438 +184439 POINT(40.60276505305873 74.28943086174462) bank184439 +184440 POINT(40.82785780728265 74.354453949767) bank184440 +184441 POINT(40.17123311887469 74.39710277512637) bank184441 +184442 POINT(41.13941964791048 74.6311145295714) bank184442 +184443 POINT(40.373531495791816 73.88657960290514) bank184443 +184444 POINT(40.2760677310235 74.47455264282391) bank184444 +184445 POINT(40.02069200862694 73.54560184423217) bank184445 +184446 POINT(40.36960100106307 74.72059562786511) bank184446 +184447 POINT(41.199908750821386 74.23507709847472) bank184447 +184448 POINT(41.527740119704525 73.17576799479548) bank184448 +184449 POINT(39.939313848533466 73.14853153074479) bank184449 +184450 POINT(40.51468775671346 73.42872647744035) bank184450 +184451 POINT(41.30929630504727 74.10660065642797) bank184451 +184452 POINT(40.382439398417056 74.57590494306035) bank184452 +184453 POINT(41.67299093436084 74.66927314854155) bank184453 +184454 POINT(41.626714432170516 73.18356417318306) bank184454 +184455 POINT(39.922282442434444 73.23462819708212) bank184455 +184456 POINT(40.357517487679736 74.17986716398944) bank184456 +184457 POINT(41.459055308980226 73.39834656798075) bank184457 +184458 POINT(41.54343930952003 73.42301395664435) bank184458 +184459 POINT(40.880353390353925 74.82528517976993) bank184459 +184460 POINT(40.54219842237109 74.59737203531301) bank184460 +184461 POINT(41.61377821568586 74.59615651240857) bank184461 +184462 POINT(40.62380967457621 74.15190225879932) bank184462 +184463 POINT(41.072090843747965 74.82728204530945) bank184463 +184464 POINT(40.57318715518297 74.7741933129678) bank184464 +184465 POINT(41.01511317406536 73.19507518311372) bank184465 +184466 POINT(41.46968444475404 74.52147846016007) bank184466 +184467 POINT(41.63841521754678 73.42759632461575) bank184467 +184468 POINT(40.01567185120804 73.59329849723518) bank184468 +184469 POINT(40.48348281791953 74.39581514591914) bank184469 +184470 POINT(40.13844317373468 74.09209131566631) bank184470 +184471 POINT(40.461727194069184 73.85803857468218) bank184471 +184472 POINT(40.200235230039695 73.70234669301934) bank184472 +184473 POINT(40.76672309269212 74.64041602022749) bank184473 +184474 POINT(40.620591655139606 73.28763905045376) bank184474 +184475 POINT(41.273425938032695 73.40030529008024) bank184475 +184476 POINT(40.86410642845768 73.72644533614711) bank184476 +184477 POINT(40.75330923113219 74.81675096017474) bank184477 +184478 POINT(39.99945541755368 73.80591668316582) bank184478 +184479 POINT(40.23903015634829 74.68965432651468) bank184479 +184480 POINT(39.81210388098405 74.54628855033128) bank184480 +184481 POINT(40.41064512951299 73.85379508022325) bank184481 +184482 POINT(41.10553618023142 73.92496963964656) bank184482 +184483 POINT(40.82732082059762 74.7063058975934) bank184483 +184484 POINT(40.70646860658524 73.91179160992921) bank184484 +184485 POINT(40.01862064355716 73.77917768231163) bank184485 +184486 POINT(40.586483458916575 74.29736703386064) bank184486 +184487 POINT(40.54116717672601 73.94732941298108) bank184487 +184488 POINT(39.78308934760199 73.68028662584858) bank184488 +184489 POINT(41.250366926435625 74.81930557647641) bank184489 +184490 POINT(41.3318571330143 74.01028299211269) bank184490 +184491 POINT(39.7299782979397 73.67599309483265) bank184491 +184492 POINT(41.68661805689773 74.40659737425645) bank184492 +184493 POINT(41.4478678405481 73.66647864429396) bank184493 +184494 POINT(40.27779294625346 74.80939254301369) bank184494 +184495 POINT(40.86127523853717 74.0199986531419) bank184495 +184496 POINT(40.74040143304111 73.04300805778159) bank184496 +184497 POINT(40.65967405251498 74.38650052512551) bank184497 +184498 POINT(41.160224842295584 73.70245744345856) bank184498 +184499 POINT(39.893336663858626 73.57692174602089) bank184499 +184500 POINT(40.22723938399741 73.7319644520303) bank184500 +184501 POINT(40.50657588278388 73.32094691756167) bank184501 +184502 POINT(39.72924310786089 74.52935731849743) bank184502 +184503 POINT(40.228059993037675 73.8831673417859) bank184503 +184504 POINT(40.65174265731142 74.9398781687493) bank184504 +184505 POINT(41.46873564189337 73.35236286955137) bank184505 +184506 POINT(41.23239427453598 74.42981369626223) bank184506 +184507 POINT(39.93378672891045 74.01867766139077) bank184507 +184508 POINT(39.90519532032611 74.11825974188903) bank184508 +184509 POINT(40.00648316851167 74.26293260640848) bank184509 +184510 POINT(40.20428113692862 73.80242859412213) bank184510 +184511 POINT(40.75929190982954 74.91988943940625) bank184511 +184512 POINT(40.24092816423791 74.5841630949069) bank184512 +184513 POINT(41.594323039948584 74.84284396197525) bank184513 +184514 POINT(41.11906742771506 73.66030793118549) bank184514 +184515 POINT(40.11416477864761 73.35473164842068) bank184515 +184516 POINT(40.545149157508796 73.66506691236978) bank184516 +184517 POINT(41.042528181150445 73.62813441545988) bank184517 +184518 POINT(40.060143970231934 73.05626634795202) bank184518 +184519 POINT(41.01168498651288 74.57390167803224) bank184519 +184520 POINT(41.40373426271103 73.99480601233589) bank184520 +184521 POINT(41.10164664189402 73.11381770602759) bank184521 +184522 POINT(39.75016747386368 74.09843455469907) bank184522 +184523 POINT(41.01279773751085 74.04889419450974) bank184523 +184524 POINT(41.43456445599265 73.1581866641077) bank184524 +184525 POINT(40.81239527470073 74.25880641108249) bank184525 +184526 POINT(41.35977940655428 74.23677669172011) bank184526 +184527 POINT(41.57026981071553 74.11433337154558) bank184527 +184528 POINT(41.19257637997971 73.1790531345078) bank184528 +184529 POINT(40.53936007593226 74.81255325660526) bank184529 +184530 POINT(40.66088872694787 74.87581489436975) bank184530 +184531 POINT(40.255938271256014 74.14508517891217) bank184531 +184532 POINT(39.78545080324731 73.60032301521757) bank184532 +184533 POINT(39.864702921737056 73.22525359815381) bank184533 +184534 POINT(40.39468401472256 73.69451021944903) bank184534 +184535 POINT(41.636118467368156 74.70314817647235) bank184535 +184536 POINT(40.58715895674531 74.54469027517145) bank184536 +184537 POINT(41.55689744984883 74.06793701533886) bank184537 +184538 POINT(40.48794082275139 74.14472126804787) bank184538 +184539 POINT(40.4215900767562 74.04912903729067) bank184539 +184540 POINT(41.29219308296586 73.58016102443239) bank184540 +184541 POINT(40.6661996153365 74.96488992510646) bank184541 +184542 POINT(41.5185939835679 73.14280417164743) bank184542 +184543 POINT(39.97021998404641 73.36595477455398) bank184543 +184544 POINT(41.00218540864019 73.52090775361172) bank184544 +184545 POINT(41.41856380601743 73.4946751691923) bank184545 +184546 POINT(39.81616717512553 74.9973549994962) bank184546 +184547 POINT(41.41205490738731 73.900235093641) bank184547 +184548 POINT(41.67933581433905 73.715262851751) bank184548 +184549 POINT(39.88412891520173 74.97304697796913) bank184549 +184550 POINT(40.143489083191646 74.23245438239579) bank184550 +184551 POINT(39.939184332303455 73.20229574664945) bank184551 +184552 POINT(41.40932456611586 73.53483048117805) bank184552 +184553 POINT(40.00044559425329 74.12275485604232) bank184553 +184554 POINT(40.25634386015816 73.61348094137983) bank184554 +184555 POINT(41.66729429182557 74.8839087764681) bank184555 +184556 POINT(41.53044634132675 73.86890474892074) bank184556 +184557 POINT(40.33971289323191 74.96240506614937) bank184557 +184558 POINT(41.314618254077736 73.62391311388684) bank184558 +184559 POINT(40.384343208891465 73.13939911614656) bank184559 +184560 POINT(41.57545020715602 73.18636797967571) bank184560 +184561 POINT(40.35705011070348 73.83672583583555) bank184561 +184562 POINT(39.80905061620227 74.43185304205666) bank184562 +184563 POINT(40.99367283324728 74.35636265402614) bank184563 +184564 POINT(40.32678760778053 74.81395773878103) bank184564 +184565 POINT(41.350239073734755 73.62153181497531) bank184565 +184566 POINT(40.81711418819548 74.08204663179562) bank184566 +184567 POINT(41.38845203960629 74.02725122704683) bank184567 +184568 POINT(41.19043841766418 73.40871633900382) bank184568 +184569 POINT(40.89588000176405 73.43660741532868) bank184569 +184570 POINT(40.406811719407344 74.92977847785083) bank184570 +184571 POINT(39.72593513465526 73.12576377896495) bank184571 +184572 POINT(40.87527970138984 73.6844327949839) bank184572 +184573 POINT(41.57792016706716 74.6068929014133) bank184573 +184574 POINT(41.601347731722775 73.88620904964813) bank184574 +184575 POINT(41.20647931691073 74.77113376498377) bank184575 +184576 POINT(40.11475881424282 73.03085400414228) bank184576 +184577 POINT(40.343613815339296 73.97188719436404) bank184577 +184578 POINT(41.48630247873023 74.66324091333601) bank184578 +184579 POINT(41.055116265041455 74.45458156523993) bank184579 +184580 POINT(39.8268175721113 73.36744990907758) bank184580 +184581 POINT(40.56928217511192 73.53509069040776) bank184581 +184582 POINT(40.839349799083934 73.0503117002121) bank184582 +184583 POINT(40.506049099927004 74.0168698688254) bank184583 +184584 POINT(40.34001831952208 74.53042822794417) bank184584 +184585 POINT(40.733210468523936 73.48799012625871) bank184585 +184586 POINT(41.544251077018 73.06298857061302) bank184586 +184587 POINT(39.93094134774461 74.85131398640937) bank184587 +184588 POINT(40.655304429245696 74.01427282600095) bank184588 +184589 POINT(41.33368057180341 73.88637937060822) bank184589 +184590 POINT(40.65270722460554 73.55685229562651) bank184590 +184591 POINT(40.972112579862866 73.35529637898537) bank184591 +184592 POINT(41.30145742535863 74.32856947213793) bank184592 +184593 POINT(40.39930856909013 74.31494516097332) bank184593 +184594 POINT(40.34830574794842 73.24928060351733) bank184594 +184595 POINT(40.214054861577694 73.08938978973349) bank184595 +184596 POINT(40.88587130657616 74.68281125819266) bank184596 +184597 POINT(39.79955260492884 73.78454137668966) bank184597 +184598 POINT(40.405863089225925 73.3335047459849) bank184598 +184599 POINT(41.467862168083634 74.49221999528422) bank184599 +184600 POINT(41.39733656149926 73.57662836671597) bank184600 +184601 POINT(41.05179273405308 74.10451672247468) bank184601 +184602 POINT(40.10756098749127 73.86823064677793) bank184602 +184603 POINT(40.39342770959877 73.2141738887717) bank184603 +184604 POINT(41.41960208015871 74.42568120063748) bank184604 +184605 POINT(40.02725530798953 73.70629086225841) bank184605 +184606 POINT(39.86328702952657 74.85010393609546) bank184606 +184607 POINT(40.814674383944414 74.25220993603415) bank184607 +184608 POINT(40.40185234790441 73.24011592818228) bank184608 +184609 POINT(41.06551858293645 73.86158320842438) bank184609 +184610 POINT(41.24140674158879 73.12912084994457) bank184610 +184611 POINT(39.97488498737094 73.79081245103873) bank184611 +184612 POINT(41.378477605742276 73.2267088553278) bank184612 +184613 POINT(40.88250046183115 73.54550280357134) bank184613 +184614 POINT(41.39100614585533 74.7963287466967) bank184614 +184615 POINT(41.40009016497194 74.4549185733048) bank184615 +184616 POINT(39.814883774483434 74.0212673480599) bank184616 +184617 POINT(41.17796348647282 74.84492322614547) bank184617 +184618 POINT(41.23498473706481 73.94194906626045) bank184618 +184619 POINT(40.65947412057816 74.34662599427864) bank184619 +184620 POINT(41.65017170152205 73.11474065773503) bank184620 +184621 POINT(39.72758625251312 73.35000554910434) bank184621 +184622 POINT(41.183016696925485 74.4474884711436) bank184622 +184623 POINT(41.297897964250495 73.33615920203263) bank184623 +184624 POINT(40.795937536484445 73.09163028754557) bank184624 +184625 POINT(40.70214874550754 73.16125469150846) bank184625 +184626 POINT(40.93770950206329 74.85355802307456) bank184626 +184627 POINT(41.0508399142021 73.0979097395021) bank184627 +184628 POINT(40.89794014180641 73.86744179755152) bank184628 +184629 POINT(40.27678206075872 74.73720127613004) bank184629 +184630 POINT(41.2204178725952 73.30611560657394) bank184630 +184631 POINT(40.72734421091486 74.35196963576949) bank184631 +184632 POINT(40.94144788623201 74.06602298046309) bank184632 +184633 POINT(40.794927908272285 73.23998296720076) bank184633 +184634 POINT(40.29405067792027 73.86764665932212) bank184634 +184635 POINT(39.953825204501285 73.48462632164487) bank184635 +184636 POINT(40.500666952409176 74.07752533876364) bank184636 +184637 POINT(39.77088991805155 74.06344422921931) bank184637 +184638 POINT(40.52100125672378 74.4886192170262) bank184638 +184639 POINT(41.16305841670021 74.0960357131798) bank184639 +184640 POINT(41.006219912455755 73.98036952820496) bank184640 +184641 POINT(39.92191046258609 74.43291718841952) bank184641 +184642 POINT(40.65498897983 73.96764782131763) bank184642 +184643 POINT(41.603808676579234 73.43834850439988) bank184643 +184644 POINT(41.56684833986123 74.08782652671442) bank184644 +184645 POINT(41.15049057702949 74.43731829796462) bank184645 +184646 POINT(40.15023324121019 74.8651456143655) bank184646 +184647 POINT(39.8424473691426 73.02068819306203) bank184647 +184648 POINT(41.0980913777523 73.22216129677494) bank184648 +184649 POINT(41.079134540081434 74.66083218949557) bank184649 +184650 POINT(40.80860301264564 73.4364086777413) bank184650 +184651 POINT(40.955210948249935 73.67377284085396) bank184651 +184652 POINT(39.79030166372044 74.88209176723069) bank184652 +184653 POINT(41.53561672930356 74.83172621810378) bank184653 +184654 POINT(41.508633013359486 74.88048874971132) bank184654 +184655 POINT(41.70590276233716 73.05101478921554) bank184655 +184656 POINT(41.66238836810943 74.68909237829469) bank184656 +184657 POINT(39.95326824098005 74.52837601144641) bank184657 +184658 POINT(40.35824930909011 74.33387345639221) bank184658 +184659 POINT(40.69721796527756 73.23138479061907) bank184659 +184660 POINT(39.94590241137667 74.0272097475851) bank184660 +184661 POINT(40.21082370920556 73.0326134292199) bank184661 +184662 POINT(40.93468606645918 74.21074110402557) bank184662 +184663 POINT(40.08022580984122 74.36624064929819) bank184663 +184664 POINT(40.46440494950845 74.23469063317556) bank184664 +184665 POINT(39.82768502882685 74.99635548181311) bank184665 +184666 POINT(40.96589713031893 74.53249108527514) bank184666 +184667 POINT(40.82831020922612 73.73800750467477) bank184667 +184668 POINT(40.64406792239051 74.94608326349574) bank184668 +184669 POINT(40.21504327371614 74.58782882676408) bank184669 +184670 POINT(41.05316642912391 73.21804332469051) bank184670 +184671 POINT(40.02615185865446 73.12717523236816) bank184671 +184672 POINT(40.02227233274695 74.05021161930247) bank184672 +184673 POINT(40.270564033554 73.51842753907239) bank184673 +184674 POINT(41.550551341271124 73.1359216343381) bank184674 +184675 POINT(41.357284114583194 73.54809707464062) bank184675 +184676 POINT(39.980913600956555 74.91931125882697) bank184676 +184677 POINT(41.37568817107625 74.15822230105753) bank184677 +184678 POINT(40.96070656090172 73.66812968149584) bank184678 +184679 POINT(39.82290722641318 73.78339967888918) bank184679 +184680 POINT(41.23375068555398 73.26064898968573) bank184680 +184681 POINT(40.393867561644505 73.82719623154316) bank184681 +184682 POINT(40.97085688494291 74.4288563365033) bank184682 +184683 POINT(40.406481434648256 73.90832309437637) bank184683 +184684 POINT(40.146194373856 74.1057561145728) bank184684 +184685 POINT(40.045424070619035 73.6208529596596) bank184685 +184686 POINT(40.36670628294322 73.49750703402364) bank184686 +184687 POINT(41.36816452264827 73.94119093061822) bank184687 +184688 POINT(40.72149865884724 74.83884181557073) bank184688 +184689 POINT(41.62500059781544 73.07583818649769) bank184689 +184690 POINT(41.11752768229521 74.62615419752659) bank184690 +184691 POINT(41.307771354088494 73.77200045996162) bank184691 +184692 POINT(39.97106699670266 74.40268493046905) bank184692 +184693 POINT(39.73894920924535 74.05246878206827) bank184693 +184694 POINT(41.49128719564502 73.13720153037839) bank184694 +184695 POINT(40.80701092436008 73.82886432881199) bank184695 +184696 POINT(40.63400589389269 73.17462717826649) bank184696 +184697 POINT(41.1483487234015 73.54441183054462) bank184697 +184698 POINT(39.901346557455234 74.74735723891492) bank184698 +184699 POINT(41.07815063298154 73.64882712543273) bank184699 +184700 POINT(40.87015527469802 74.27095624587966) bank184700 +184701 POINT(41.35684157841259 73.6924273295068) bank184701 +184702 POINT(41.499638332107835 74.56121395452817) bank184702 +184703 POINT(41.183659178255716 73.80004053257306) bank184703 +184704 POINT(40.223264729345374 73.95775851732353) bank184704 +184705 POINT(41.62059400124603 73.93666846739946) bank184705 +184706 POINT(41.343611871548475 73.50972560952191) bank184706 +184707 POINT(40.020130951229795 74.01884324430566) bank184707 +184708 POINT(41.54518738648703 73.67340559307279) bank184708 +184709 POINT(40.87171690343831 74.73421218543662) bank184709 +184710 POINT(41.400967449329414 74.74573390285852) bank184710 +184711 POINT(40.63597899990223 74.05875279075968) bank184711 +184712 POINT(41.389159778912884 73.01110533065302) bank184712 +184713 POINT(40.40027257247616 73.395017543528) bank184713 +184714 POINT(41.614151411994875 74.72704252667143) bank184714 +184715 POINT(41.22306531322044 74.65687759464305) bank184715 +184716 POINT(41.22287296600319 73.4373690933694) bank184716 +184717 POINT(41.687045267638986 74.55328405546152) bank184717 +184718 POINT(40.60032374737588 73.82224307476882) bank184718 +184719 POINT(40.92334243836423 73.36773123673537) bank184719 +184720 POINT(40.76662170283394 74.44694167423653) bank184720 +184721 POINT(40.0942860015487 74.95247644697369) bank184721 +184722 POINT(41.15395241703601 74.77248783377324) bank184722 +184723 POINT(39.92823166331636 73.8066204135639) bank184723 +184724 POINT(41.39850422194488 74.30016000666262) bank184724 +184725 POINT(40.80912537230504 73.46399434088421) bank184725 +184726 POINT(40.80956545757757 74.9082335497724) bank184726 +184727 POINT(40.73887344442822 74.61286003386545) bank184727 +184728 POINT(40.31113649135779 74.8641127187522) bank184728 +184729 POINT(40.161906838493444 74.32147960936862) bank184729 +184730 POINT(40.29103662643811 73.11042177303263) bank184730 +184731 POINT(40.47071627633792 73.20780112093526) bank184731 +184732 POINT(41.31335017537398 74.95354654468912) bank184732 +184733 POINT(40.78227978616606 74.22213481080088) bank184733 +184734 POINT(40.64626667291973 73.42262194455108) bank184734 +184735 POINT(41.025878399908876 73.86796737508642) bank184735 +184736 POINT(41.16063084445524 74.73609937560101) bank184736 +184737 POINT(41.48951662186733 74.46904067323202) bank184737 +184738 POINT(39.990626425912545 73.4621008733104) bank184738 +184739 POINT(40.23079587962804 74.17770785754503) bank184739 +184740 POINT(41.21517921593123 73.15038376303347) bank184740 +184741 POINT(39.93681382636879 73.503384763304) bank184741 +184742 POINT(40.56735257291946 73.27410804206816) bank184742 +184743 POINT(40.5382331370451 73.89302531847483) bank184743 +184744 POINT(40.504543347622345 74.77159312571696) bank184744 +184745 POINT(40.86124415609259 74.23987037356409) bank184745 +184746 POINT(40.032949362953936 74.3775651876926) bank184746 +184747 POINT(39.89878674648489 74.87052163135874) bank184747 +184748 POINT(41.136109029978854 74.42265671689947) bank184748 +184749 POINT(41.32091423013827 73.99428972064663) bank184749 +184750 POINT(41.68192389867718 74.49365578607137) bank184750 +184751 POINT(40.817980718635965 74.72690966449457) bank184751 +184752 POINT(40.53308199510317 73.6998560073197) bank184752 +184753 POINT(41.18327935016739 74.91789918899636) bank184753 +184754 POINT(40.87930014170122 73.29375105115949) bank184754 +184755 POINT(40.269398860360425 73.65772446236538) bank184755 +184756 POINT(40.52837911114526 74.30241928704964) bank184756 +184757 POINT(41.21931950404707 74.08954695478816) bank184757 +184758 POINT(40.94290968260399 74.42234586444359) bank184758 +184759 POINT(41.53443636530106 73.50069597213522) bank184759 +184760 POINT(40.29766724601041 74.01056036574887) bank184760 +184761 POINT(41.25024363961871 73.76523948987293) bank184761 +184762 POINT(40.07990531212476 74.66409433588215) bank184762 +184763 POINT(40.02564268612839 73.41684556430924) bank184763 +184764 POINT(40.250536879443565 73.17440421030481) bank184764 +184765 POINT(40.39700020259853 74.31657421619249) bank184765 +184766 POINT(39.97377834509158 74.02866972861412) bank184766 +184767 POINT(41.02375126002871 73.3237538838851) bank184767 +184768 POINT(40.34262529094438 74.87194996056054) bank184768 +184769 POINT(40.21940871494634 73.55627620911241) bank184769 +184770 POINT(39.75863347230807 73.73750461866545) bank184770 +184771 POINT(41.48341898789601 74.42694516464371) bank184771 +184772 POINT(40.947879272525725 74.89310293418214) bank184772 +184773 POINT(40.35097478446271 73.02256920708025) bank184773 +184774 POINT(40.944450734784766 74.04490557206795) bank184774 +184775 POINT(40.96960639792853 74.11228588392223) bank184775 +184776 POINT(40.64401682833911 74.79046626460409) bank184776 +184777 POINT(41.27164194624395 73.30506854843449) bank184777 +184778 POINT(41.47624098042955 73.46928367887757) bank184778 +184779 POINT(41.35751797273377 73.4373421072996) bank184779 +184780 POINT(40.727776182777355 74.28738711150014) bank184780 +184781 POINT(41.61488487679917 73.62097013106107) bank184781 +184782 POINT(40.88776362469943 74.40099197838092) bank184782 +184783 POINT(40.526652089116304 74.54895956636393) bank184783 +184784 POINT(40.63847587083423 73.45989834259778) bank184784 +184785 POINT(40.98720306957492 73.73047808339527) bank184785 +184786 POINT(39.87259588271811 73.08014953654748) bank184786 +184787 POINT(40.8652097309389 73.07807103678262) bank184787 +184788 POINT(40.01085376999801 73.22574220112377) bank184788 +184789 POINT(40.129413272168584 74.16616779788619) bank184789 +184790 POINT(39.942518833172194 73.67871719483688) bank184790 +184791 POINT(41.49753353688302 74.71667906024277) bank184791 +184792 POINT(41.43727362355589 73.19165657702817) bank184792 +184793 POINT(39.71916007313615 73.85778841307537) bank184793 +184794 POINT(40.6142886228947 74.93883568816167) bank184794 +184795 POINT(41.27262084520057 73.63857928999255) bank184795 +184796 POINT(40.004775785261856 73.70031654827928) bank184796 +184797 POINT(40.26210976967008 74.76914992581806) bank184797 +184798 POINT(41.053410520617675 74.84687937160064) bank184798 +184799 POINT(40.08319213452214 74.06876286214329) bank184799 +184800 POINT(41.46435148996612 74.99127102266762) bank184800 +184801 POINT(41.562406600606 74.39450731689696) bank184801 +184802 POINT(40.96262630811961 73.59083626437814) bank184802 +184803 POINT(41.46164087670412 73.41309486786812) bank184803 +184804 POINT(39.858650371308144 73.15306758658721) bank184804 +184805 POINT(41.66571980435281 74.57294562170632) bank184805 +184806 POINT(41.141424885313356 73.28206362637007) bank184806 +184807 POINT(41.27421758329024 74.23906091783138) bank184807 +184808 POINT(41.48947927046216 73.04723587636767) bank184808 +184809 POINT(40.844558116181446 73.89425991659463) bank184809 +184810 POINT(41.11010141668072 74.3574935821286) bank184810 +184811 POINT(41.64184541351832 74.90042397558412) bank184811 +184812 POINT(40.575240796729965 74.28364926058447) bank184812 +184813 POINT(40.28059239753202 74.47648512622126) bank184813 +184814 POINT(40.14948944563886 74.53463217931326) bank184814 +184815 POINT(40.075707730034495 73.78085398440193) bank184815 +184816 POINT(41.33744176592296 73.03942091028921) bank184816 +184817 POINT(41.25730437033395 73.50345695639794) bank184817 +184818 POINT(41.28216492081226 74.7180849010763) bank184818 +184819 POINT(40.993128715184234 73.3847954478844) bank184819 +184820 POINT(41.35531992454797 74.05117231730986) bank184820 +184821 POINT(39.93470812031351 74.05512036164197) bank184821 +184822 POINT(41.639280578962946 74.96794990632924) bank184822 +184823 POINT(40.8832721917766 73.56659480047462) bank184823 +184824 POINT(41.43187087773732 74.36663024954153) bank184824 +184825 POINT(41.420134288643204 74.2972839107028) bank184825 +184826 POINT(40.39280355248059 74.38598519746186) bank184826 +184827 POINT(40.34910777758011 73.99016186326233) bank184827 +184828 POINT(39.832959628069545 74.6218412101269) bank184828 +184829 POINT(40.09469872614577 73.75856462956206) bank184829 +184830 POINT(40.55584135395763 74.8092575666783) bank184830 +184831 POINT(40.09397254177511 73.36939746345955) bank184831 +184832 POINT(39.99087365493164 73.29190516391881) bank184832 +184833 POINT(39.847355496954386 73.21262712330594) bank184833 +184834 POINT(41.067236002717024 73.810888683894) bank184834 +184835 POINT(41.467660257616345 73.90860227480671) bank184835 +184836 POINT(40.34654682372276 74.8572984741488) bank184836 +184837 POINT(40.14641943482465 73.94994945883855) bank184837 +184838 POINT(41.60678310920075 73.89541888722295) bank184838 +184839 POINT(41.66189906408669 73.69527548741675) bank184839 +184840 POINT(41.118990815455305 74.11628236589637) bank184840 +184841 POINT(40.453544305426014 73.80154697419722) bank184841 +184842 POINT(39.72413190193955 74.31017176926004) bank184842 +184843 POINT(40.36995671524396 74.99927709297017) bank184843 +184844 POINT(40.58986024315336 74.32130676221242) bank184844 +184845 POINT(40.73646919018681 73.28951770477569) bank184845 +184846 POINT(41.333053830088225 73.54725216987279) bank184846 +184847 POINT(41.26818246526202 73.38301874311951) bank184847 +184848 POINT(40.524017129411256 73.86664336315064) bank184848 +184849 POINT(41.550625881028324 73.57541573656748) bank184849 +184850 POINT(39.83761427979675 74.99905744040265) bank184850 +184851 POINT(41.563773407385064 73.2012947429438) bank184851 +184852 POINT(39.746792162242926 73.48853774766013) bank184852 +184853 POINT(41.40473197370103 74.70456154521807) bank184853 +184854 POINT(41.16888272185375 74.4636148296315) bank184854 +184855 POINT(40.02062949039718 74.35936281639206) bank184855 +184856 POINT(40.99868705422233 74.50452701516917) bank184856 +184857 POINT(41.22919921475813 73.20753963215918) bank184857 +184858 POINT(41.332604979042024 74.54524535413061) bank184858 +184859 POINT(40.71999962721247 74.44048232352314) bank184859 +184860 POINT(40.27637158753343 73.59546922819138) bank184860 +184861 POINT(41.465169612215185 73.8716510417836) bank184861 +184862 POINT(39.81548323978253 74.91853518824193) bank184862 +184863 POINT(41.2420788063967 73.95355831313431) bank184863 +184864 POINT(40.235248821946726 74.50854607975587) bank184864 +184865 POINT(39.97248249295769 74.16254799720926) bank184865 +184866 POINT(41.410555365433666 74.26640209613716) bank184866 +184867 POINT(40.552177757909156 74.52531208946742) bank184867 +184868 POINT(41.425301746443665 73.94048063258693) bank184868 +184869 POINT(41.01921319569764 74.51183798282817) bank184869 +184870 POINT(39.805728971285966 73.82356790187185) bank184870 +184871 POINT(41.21038617108044 74.07939333194275) bank184871 +184872 POINT(40.58993593246659 73.65327720877916) bank184872 +184873 POINT(41.02861321112442 74.13809294970595) bank184873 +184874 POINT(40.916768313329925 74.8412132210471) bank184874 +184875 POINT(39.738700373791026 73.97855730471994) bank184875 +184876 POINT(41.37885228941578 73.07843527146363) bank184876 +184877 POINT(39.7741461404958 73.08922244177104) bank184877 +184878 POINT(40.167425319750855 74.7123462443046) bank184878 +184879 POINT(40.18613602993963 73.91403508947428) bank184879 +184880 POINT(39.875510198345964 73.1890353883439) bank184880 +184881 POINT(40.5633750856018 74.48662820579901) bank184881 +184882 POINT(41.5531165741867 73.22102960825252) bank184882 +184883 POINT(40.906802306755004 73.29262292952872) bank184883 +184884 POINT(41.38912181804096 73.3945577250773) bank184884 +184885 POINT(41.47883002714905 74.69229865764295) bank184885 +184886 POINT(41.17607636389537 74.61758642731829) bank184886 +184887 POINT(41.66002554444533 74.24802665336055) bank184887 +184888 POINT(40.146118564989926 74.34616962886345) bank184888 +184889 POINT(41.2201338398876 74.36242915841012) bank184889 +184890 POINT(40.99625901272785 73.08126667284193) bank184890 +184891 POINT(41.028262043698 73.97314208687384) bank184891 +184892 POINT(39.821948400316906 74.95140001315696) bank184892 +184893 POINT(41.516313470888036 73.22444112040186) bank184893 +184894 POINT(41.58438588258336 73.39710952337022) bank184894 +184895 POINT(40.64468785684289 73.60052526905717) bank184895 +184896 POINT(41.154138349571355 74.39818167571576) bank184896 +184897 POINT(41.18233891602371 73.85234333293889) bank184897 +184898 POINT(40.934081106721884 74.08421334476972) bank184898 +184899 POINT(40.61880465773731 74.87104247186396) bank184899 +184900 POINT(41.48278928442596 74.39004795354498) bank184900 +184901 POINT(41.27647299221231 73.37853035575499) bank184901 +184902 POINT(39.73989495342031 73.27467778279582) bank184902 +184903 POINT(40.78342747869587 73.85276921448786) bank184903 +184904 POINT(39.88470395064529 73.40309256183221) bank184904 +184905 POINT(40.386163157738935 73.09365970316004) bank184905 +184906 POINT(40.974964464502975 73.70255596870906) bank184906 +184907 POINT(39.93469492826703 73.00939310910631) bank184907 +184908 POINT(39.960784453312336 74.4785505358833) bank184908 +184909 POINT(41.591034152636176 74.18633603892097) bank184909 +184910 POINT(40.820499392124546 73.85586514761559) bank184910 +184911 POINT(39.736859004252096 73.19270534429204) bank184911 +184912 POINT(41.507612185006465 74.13607303112741) bank184912 +184913 POINT(40.64732310561686 74.34265295238087) bank184913 +184914 POINT(41.30562428947995 73.98722296552243) bank184914 +184915 POINT(40.00939841698077 74.65836832201852) bank184915 +184916 POINT(40.55396294273391 74.95242877528845) bank184916 +184917 POINT(40.580031323051514 73.21575322401488) bank184917 +184918 POINT(40.02371165709169 73.9272699904462) bank184918 +184919 POINT(40.92816575966232 74.86356261002662) bank184919 +184920 POINT(40.53346940645445 73.59146610441779) bank184920 +184921 POINT(40.89027551586402 73.97729108408875) bank184921 +184922 POINT(41.341197652498565 74.68995797888067) bank184922 +184923 POINT(39.766006028354234 73.51575970824264) bank184923 +184924 POINT(41.39784229043989 73.40394021589123) bank184924 +184925 POINT(41.129380125982955 73.728813462107) bank184925 +184926 POINT(40.59736454799568 73.9866862550186) bank184926 +184927 POINT(40.2022016501419 73.87971737825616) bank184927 +184928 POINT(39.752978467778625 73.70042984681196) bank184928 +184929 POINT(40.06050973075742 74.10926035824303) bank184929 +184930 POINT(41.17979016842669 74.92431018524775) bank184930 +184931 POINT(40.94293838834354 74.96097134137459) bank184931 +184932 POINT(40.19449601822139 74.84859755243976) bank184932 +184933 POINT(39.77891869832637 73.9380887631664) bank184933 +184934 POINT(40.140218248735 73.34306297347247) bank184934 +184935 POINT(40.23772571936054 73.187499105851) bank184935 +184936 POINT(40.80862577739934 74.75010919693295) bank184936 +184937 POINT(39.934976380690344 74.26958567816263) bank184937 +184938 POINT(40.92520349642186 73.96276329289807) bank184938 +184939 POINT(40.37732216177511 74.06913508759175) bank184939 +184940 POINT(41.538015418873954 73.37261155662655) bank184940 +184941 POINT(40.2560467481091 74.04175685562976) bank184941 +184942 POINT(40.28918568928969 73.73641343340876) bank184942 +184943 POINT(39.71409629353005 73.49560433796887) bank184943 +184944 POINT(41.09436161180968 73.9902792672262) bank184944 +184945 POINT(40.77969371989338 73.17085510451298) bank184945 +184946 POINT(41.35162080722438 74.06444355906942) bank184946 +184947 POINT(41.129642487979304 74.79176716831842) bank184947 +184948 POINT(40.37145797729223 73.69339764058036) bank184948 +184949 POINT(40.30341394422573 73.7846595247468) bank184949 +184950 POINT(40.91808620608616 73.44762289796869) bank184950 +184951 POINT(39.98767507226926 74.52424645249648) bank184951 +184952 POINT(40.78362482477851 74.8209670530432) bank184952 +184953 POINT(41.125506581155584 74.11607547690936) bank184953 +184954 POINT(40.897901721790184 73.40048891107286) bank184954 +184955 POINT(40.97178871042431 74.57587598891335) bank184955 +184956 POINT(41.62901410923963 73.65180291186651) bank184956 +184957 POINT(40.53304463469851 73.43163551412876) bank184957 +184958 POINT(40.968198972590045 73.1962534570864) bank184958 +184959 POINT(41.32171399861564 74.40295641160102) bank184959 +184960 POINT(40.418467282258284 74.75736914105896) bank184960 +184961 POINT(41.61920040904719 74.21823536741381) bank184961 +184962 POINT(41.33637626308295 73.38225246680668) bank184962 +184963 POINT(40.2121935588581 73.38319757249653) bank184963 +184964 POINT(40.03721020387081 74.83968429121471) bank184964 +184965 POINT(41.28621984581313 74.15573810739866) bank184965 +184966 POINT(40.143725880163956 73.88273601151722) bank184966 +184967 POINT(39.99476179863078 73.05505998425687) bank184967 +184968 POINT(40.094875619635545 73.65752993254212) bank184968 +184969 POINT(40.28876724685611 74.79568668050642) bank184969 +184970 POINT(40.986317435939426 74.54703131587227) bank184970 +184971 POINT(40.44538566812464 74.63626007906785) bank184971 +184972 POINT(40.70862716134973 73.06030800914523) bank184972 +184973 POINT(41.32708126734723 74.35014131452567) bank184973 +184974 POINT(41.19813360251674 74.86387515076534) bank184974 +184975 POINT(41.49239734990098 73.32526783043618) bank184975 +184976 POINT(40.421590470501485 74.08716154440435) bank184976 +184977 POINT(40.052275092290806 74.6410523268853) bank184977 +184978 POINT(40.669162967293644 74.80239951339325) bank184978 +184979 POINT(40.921465853815896 74.66092634406999) bank184979 +184980 POINT(39.73741255884404 74.50206078877319) bank184980 +184981 POINT(39.801015212308656 73.44876210910077) bank184981 +184982 POINT(40.44238594548604 73.31755656023819) bank184982 +184983 POINT(40.404605836203956 73.531641173792) bank184983 +184984 POINT(41.17779897304321 73.93606052287417) bank184984 +184985 POINT(39.88912643948068 74.22574313865394) bank184985 +184986 POINT(40.20911239069812 73.8578904227215) bank184986 +184987 POINT(39.95683703378509 73.95861024181787) bank184987 +184988 POINT(40.703528914743536 73.47518009837924) bank184988 +184989 POINT(40.88314598883997 74.62988359681994) bank184989 +184990 POINT(40.96623552659498 73.83929919684087) bank184990 +184991 POINT(41.375939354408274 73.44081495497035) bank184991 +184992 POINT(39.86132886025875 73.05879026276897) bank184992 +184993 POINT(41.03799363395959 74.98825727561427) bank184993 +184994 POINT(40.0105437999725 74.92140274472567) bank184994 +184995 POINT(41.13218749101785 73.43869367527012) bank184995 +184996 POINT(41.36196338695433 74.49378687561229) bank184996 +184997 POINT(41.14791246446947 74.60298413563443) bank184997 +184998 POINT(41.031063817038266 73.61521600157705) bank184998 +184999 POINT(41.42170800080011 74.74738620832258) bank184999 +185000 POINT(41.2599144326442 74.83232742330122) bank185000 +185001 POINT(41.60456146924533 74.61142634033673) bank185001 +185002 POINT(39.78218514993052 74.58059485408312) bank185002 +185003 POINT(40.952171146081774 73.6386823658313) bank185003 +185004 POINT(39.74645235976031 73.39365068058204) bank185004 +185005 POINT(40.74589018483116 74.12447471119663) bank185005 +185006 POINT(40.50165448948511 73.80473363578102) bank185006 +185007 POINT(40.22041705846073 74.17688924129179) bank185007 +185008 POINT(40.08878784949486 74.22686037484439) bank185008 +185009 POINT(41.4137024706378 73.84476464743877) bank185009 +185010 POINT(41.22732275184921 74.82944735320393) bank185010 +185011 POINT(39.88197237842275 73.09829330632773) bank185011 +185012 POINT(40.20778319923544 73.64816117862175) bank185012 +185013 POINT(40.0877458698829 74.69593757868019) bank185013 +185014 POINT(40.20390005319952 73.39207987428827) bank185014 +185015 POINT(39.75813553547519 73.22010234563263) bank185015 +185016 POINT(40.07129142285302 73.8997613598664) bank185016 +185017 POINT(41.24922470507658 74.3978160348941) bank185017 +185018 POINT(40.21035480143756 73.0905078353382) bank185018 +185019 POINT(40.93837478326314 73.70722672710703) bank185019 +185020 POINT(40.654673562913906 73.249492747976) bank185020 +185021 POINT(40.56138622206413 74.85325905281202) bank185021 +185022 POINT(40.09953549293217 74.43142419737691) bank185022 +185023 POINT(41.01635488121175 73.6058796379193) bank185023 +185024 POINT(40.62644823706928 73.67937123417997) bank185024 +185025 POINT(41.17476958763832 73.31268250566075) bank185025 +185026 POINT(40.37004350180071 74.08916054597509) bank185026 +185027 POINT(41.2685741626832 74.15150586994483) bank185027 +185028 POINT(41.29693311881064 73.24685985391177) bank185028 +185029 POINT(40.043594823253 74.15151028708985) bank185029 +185030 POINT(41.03868771813676 74.72590570785013) bank185030 +185031 POINT(39.94172453573416 74.4440464065804) bank185031 +185032 POINT(40.615734406888635 74.63027456185331) bank185032 +185033 POINT(41.33343816694786 73.59555539538465) bank185033 +185034 POINT(40.49669280006099 74.11160659842562) bank185034 +185035 POINT(39.88357579217975 73.80681021888508) bank185035 +185036 POINT(40.1118865869678 74.53258236271371) bank185036 +185037 POINT(40.14941209550134 74.94155589782588) bank185037 +185038 POINT(41.54095573083716 73.37247084565287) bank185038 +185039 POINT(41.61112760023906 73.7568967830728) bank185039 +185040 POINT(40.353486269040765 74.84368653791273) bank185040 +185041 POINT(39.79715205346816 74.58662924104664) bank185041 +185042 POINT(41.45540852350906 74.30693193355995) bank185042 +185043 POINT(41.20558597643313 74.35688786873683) bank185043 +185044 POINT(40.38169467800704 73.22242123916834) bank185044 +185045 POINT(40.68054818978592 74.45811533770532) bank185045 +185046 POINT(41.10518960653636 73.07166576990524) bank185046 +185047 POINT(40.32331491220141 74.57485972198127) bank185047 +185048 POINT(41.19229975877882 74.59089792760797) bank185048 +185049 POINT(40.671307638617954 73.80100706608435) bank185049 +185050 POINT(41.54129417105769 74.48743825985142) bank185050 +185051 POINT(41.69532934589654 74.60144781718442) bank185051 +185052 POINT(40.121045220840145 74.9329448675939) bank185052 +185053 POINT(40.28238691914909 73.10434719350125) bank185053 +185054 POINT(40.79181209469079 73.41302756152282) bank185054 +185055 POINT(40.63018287280773 74.61009636768163) bank185055 +185056 POINT(40.272950457052765 73.58789782954278) bank185056 +185057 POINT(41.23023306055975 73.92803682603869) bank185057 +185058 POINT(39.854060325360074 73.61858766218037) bank185058 +185059 POINT(41.68441598594421 74.07062134651399) bank185059 +185060 POINT(40.89358095587906 74.24449407594503) bank185060 +185061 POINT(40.183074274619095 73.26274960821881) bank185061 +185062 POINT(39.790879641005276 73.21587743380414) bank185062 +185063 POINT(41.085743631257806 73.46687360216075) bank185063 +185064 POINT(40.33715996057007 73.69362028129876) bank185064 +185065 POINT(41.1327499151409 73.2213562517445) bank185065 +185066 POINT(39.90074720595369 74.57441920338604) bank185066 +185067 POINT(40.99621719040724 73.26910376238453) bank185067 +185068 POINT(40.205944518838834 74.77847292010668) bank185068 +185069 POINT(40.777009021306746 73.6439992863617) bank185069 +185070 POINT(40.88827599782526 74.52707939190455) bank185070 +185071 POINT(40.761109435947034 74.14248240477414) bank185071 +185072 POINT(40.960596135016544 73.79313681106213) bank185072 +185073 POINT(39.95738806613125 74.96245765024912) bank185073 +185074 POINT(40.78089317156882 73.0998818638227) bank185074 +185075 POINT(39.964877901526535 73.0650594832204) bank185075 +185076 POINT(40.90190815257272 74.7715228248746) bank185076 +185077 POINT(40.02521228469776 73.60462330534214) bank185077 +185078 POINT(41.17432793946382 73.29333676476517) bank185078 +185079 POINT(40.720377061292844 73.71275220312809) bank185079 +185080 POINT(41.46269587541428 73.56327188538606) bank185080 +185081 POINT(41.066254571701805 73.46744761837081) bank185081 +185082 POINT(41.70093864348436 73.87520648652007) bank185082 +185083 POINT(41.062190523953234 73.98022602440153) bank185083 +185084 POINT(40.129435146878855 74.69126011369447) bank185084 +185085 POINT(41.41568726533451 74.06207855390144) bank185085 +185086 POINT(40.704262651608396 74.55976752389604) bank185086 +185087 POINT(41.60041875016773 73.39235394693249) bank185087 +185088 POINT(41.350831220566995 73.21578963919183) bank185088 +185089 POINT(40.614143856069745 74.01921463414828) bank185089 +185090 POINT(40.970641393594704 74.52279244934985) bank185090 +185091 POINT(41.45819029521551 73.69882920741861) bank185091 +185092 POINT(41.490179193523126 73.66486087785772) bank185092 +185093 POINT(40.97619380116145 73.80519785119621) bank185093 +185094 POINT(41.27216830466207 73.37477290787909) bank185094 +185095 POINT(41.09837861344504 73.48924495334968) bank185095 +185096 POINT(41.28834405259033 73.89989496496419) bank185096 +185097 POINT(41.10083761136745 73.84011349707421) bank185097 +185098 POINT(39.75481584949944 74.75452554131208) bank185098 +185099 POINT(41.09227076973824 74.04741632693747) bank185099 +185100 POINT(39.955308092811734 73.08262148984372) bank185100 +185101 POINT(40.55402114992192 74.62869855369684) bank185101 +185102 POINT(40.10024779458539 74.33042128488374) bank185102 +185103 POINT(40.42568223476246 73.36665008155263) bank185103 +185104 POINT(40.95936192700466 73.38923834033245) bank185104 +185105 POINT(40.88931833494979 73.81867828692074) bank185105 +185106 POINT(41.44361041971659 74.69174165900581) bank185106 +185107 POINT(41.07310659430134 74.14217941955779) bank185107 +185108 POINT(40.036021089953074 74.96872455699553) bank185108 +185109 POINT(40.38755764358801 74.30531137966685) bank185109 +185110 POINT(40.18709666962808 73.07010006776008) bank185110 +185111 POINT(41.55113668681162 73.34432930225785) bank185111 +185112 POINT(40.308855419362665 73.29676301464129) bank185112 +185113 POINT(40.88677466204167 74.42515365286381) bank185113 +185114 POINT(40.27528496198059 74.66626207092476) bank185114 +185115 POINT(40.287316634607244 73.6068255970381) bank185115 +185116 POINT(40.50646048052302 73.07715218176986) bank185116 +185117 POINT(40.282582008596165 74.58947430518319) bank185117 +185118 POINT(41.40975741598859 74.69359861833014) bank185118 +185119 POINT(39.9483266022536 73.45972540734968) bank185119 +185120 POINT(41.439776446336275 74.13766584635971) bank185120 +185121 POINT(41.627083909487894 73.0270314152946) bank185121 +185122 POINT(41.343692106625724 73.35313681245523) bank185122 +185123 POINT(41.333164003452445 73.11338465090817) bank185123 +185124 POINT(40.84999719965565 73.67445452673729) bank185124 +185125 POINT(39.807990540982686 74.24919176143499) bank185125 +185126 POINT(41.42317098083579 74.66805218327679) bank185126 +185127 POINT(41.63605644757137 73.55703644660441) bank185127 +185128 POINT(41.22263423208272 74.41932793557744) bank185128 +185129 POINT(41.024597740245795 74.80473056658558) bank185129 +185130 POINT(40.035840278768106 73.08847562452463) bank185130 +185131 POINT(40.44171087879491 74.237494031504) bank185131 +185132 POINT(41.32303143637528 73.2352491825674) bank185132 +185133 POINT(41.57122306016033 73.71669235146231) bank185133 +185134 POINT(40.670649327645194 74.48698974429155) bank185134 +185135 POINT(40.1123179727474 73.4713589690219) bank185135 +185136 POINT(40.94617609410595 73.20472676203663) bank185136 +185137 POINT(41.17623454302223 73.60588159016025) bank185137 +185138 POINT(40.25995049640092 73.69644637913537) bank185138 +185139 POINT(41.149015336490805 74.77891016135254) bank185139 +185140 POINT(41.43657566376696 73.30068334516339) bank185140 +185141 POINT(40.73743447068676 74.28743158488705) bank185141 +185142 POINT(40.81635432031351 74.62714210564384) bank185142 +185143 POINT(39.89886776138522 73.70331062219721) bank185143 +185144 POINT(40.02045543588677 73.34292789436057) bank185144 +185145 POINT(40.592923650736864 73.12301266710766) bank185145 +185146 POINT(40.15991146993432 73.83224024687418) bank185146 +185147 POINT(41.51730877350899 74.26669709828484) bank185147 +185148 POINT(40.781083606853755 73.16531104746743) bank185148 +185149 POINT(39.922928640038926 73.65680372055141) bank185149 +185150 POINT(40.999851148876 73.98851178538231) bank185150 +185151 POINT(40.449928005461125 74.3202549356185) bank185151 +185152 POINT(41.478283520621176 73.64534173748551) bank185152 +185153 POINT(40.62102422220684 73.59446623132185) bank185153 +185154 POINT(39.9162043463213 74.77611213855353) bank185154 +185155 POINT(39.737047514111715 73.93093486915012) bank185155 +185156 POINT(41.677972786562194 73.017209802264) bank185156 +185157 POINT(40.792777283243936 74.35793501199437) bank185157 +185158 POINT(40.23731535158697 74.8107066051009) bank185158 +185159 POINT(39.7437834193534 74.89182245877446) bank185159 +185160 POINT(39.9116622741298 73.74012794680756) bank185160 +185161 POINT(40.93935850343292 74.11070044967605) bank185161 +185162 POINT(39.98605822199435 73.69783562653551) bank185162 +185163 POINT(41.52970710410154 74.07863073416777) bank185163 +185164 POINT(41.601035264186045 74.56225970222631) bank185164 +185165 POINT(40.0011335418125 73.46138462381224) bank185165 +185166 POINT(40.3068808718453 74.8253260636801) bank185166 +185167 POINT(40.72114405260586 74.92464402819121) bank185167 +185168 POINT(41.06629145824912 74.45673417615765) bank185168 +185169 POINT(39.74810811498308 73.06836146787317) bank185169 +185170 POINT(41.65968256660821 73.23671157176811) bank185170 +185171 POINT(41.35395133571718 73.62849891974416) bank185171 +185172 POINT(41.08142341782577 74.00855036322302) bank185172 +185173 POINT(40.289613698876444 73.26315909088878) bank185173 +185174 POINT(41.65413353031004 74.612127659376) bank185174 +185175 POINT(40.36521519393002 74.96095986283183) bank185175 +185176 POINT(39.966190748334874 73.11356383001593) bank185176 +185177 POINT(40.73949249438748 73.86149324628187) bank185177 +185178 POINT(40.699764743383234 74.0882481651026) bank185178 +185179 POINT(41.247346696373974 74.81814270750667) bank185179 +185180 POINT(39.947038699094406 73.53724463879524) bank185180 +185181 POINT(40.09474076981871 73.29321218946161) bank185181 +185182 POINT(39.96110808261072 73.06973700108419) bank185182 +185183 POINT(41.14086938422118 74.39289425661838) bank185183 +185184 POINT(41.691592833687565 73.54286599123662) bank185184 +185185 POINT(41.07835716805587 73.70029959811062) bank185185 +185186 POINT(40.59525891527358 73.65698597069856) bank185186 +185187 POINT(41.366441779514815 74.45310778885191) bank185187 +185188 POINT(41.260186642537334 74.3699673480995) bank185188 +185189 POINT(40.19394203406567 74.72391807892085) bank185189 +185190 POINT(41.41896207954703 73.16748124345887) bank185190 +185191 POINT(41.48603167792162 73.53771008826213) bank185191 +185192 POINT(40.95943978856162 73.14800748476078) bank185192 +185193 POINT(40.28638102657807 73.23491795358937) bank185193 +185194 POINT(40.02744242118613 73.01576863016265) bank185194 +185195 POINT(39.919936424739504 73.80324910544596) bank185195 +185196 POINT(40.58810992295902 73.00785017562387) bank185196 +185197 POINT(40.997222603477134 74.43812412722826) bank185197 +185198 POINT(40.017486999852316 73.69043754183244) bank185198 +185199 POINT(40.49911961155971 73.94083986501855) bank185199 +185200 POINT(39.762529207498005 74.1069956131747) bank185200 +185201 POINT(40.481813202759014 74.1827147326373) bank185201 +185202 POINT(40.430525021555155 74.21271856327728) bank185202 +185203 POINT(39.776006213333474 74.93407950755906) bank185203 +185204 POINT(41.536048092356594 74.90361379570216) bank185204 +185205 POINT(40.09336208021672 74.65410885617875) bank185205 +185206 POINT(40.727799626061305 74.87018880695828) bank185206 +185207 POINT(41.35968772261354 74.68569602795655) bank185207 +185208 POINT(41.526332341654665 73.79413673992562) bank185208 +185209 POINT(41.139548193420026 74.74925033284056) bank185209 +185210 POINT(41.66080453275954 74.41183293713904) bank185210 +185211 POINT(40.07477121446085 74.71045731593864) bank185211 +185212 POINT(41.62114795687365 74.26146178084282) bank185212 +185213 POINT(41.3949576216512 73.54663043857705) bank185213 +185214 POINT(41.058290601043076 73.61093359100984) bank185214 +185215 POINT(39.81453049715085 73.88445779557387) bank185215 +185216 POINT(41.247988589894724 74.22720345991765) bank185216 +185217 POINT(39.80479543420344 73.79440866749447) bank185217 +185218 POINT(40.471910397138366 73.91406176880355) bank185218 +185219 POINT(41.54421914577749 73.75139089227513) bank185219 +185220 POINT(39.76158675015587 74.87745970179564) bank185220 +185221 POINT(41.6321046001439 73.51914051146782) bank185221 +185222 POINT(40.57713333326691 74.96141449135922) bank185222 +185223 POINT(40.04770858425265 74.7637876941444) bank185223 +185224 POINT(41.259113498384096 73.29130648460604) bank185224 +185225 POINT(39.89467797459333 74.35200067407246) bank185225 +185226 POINT(39.95478260404709 74.48861722022367) bank185226 +185227 POINT(41.638845726530796 73.55215120356523) bank185227 +185228 POINT(41.02333186691044 73.15644626353796) bank185228 +185229 POINT(39.96953837129108 74.56462283641123) bank185229 +185230 POINT(40.86991700411137 73.78481041438233) bank185230 +185231 POINT(40.965828090680894 74.86838142265759) bank185231 +185232 POINT(41.58857337335247 74.16203577579556) bank185232 +185233 POINT(40.50088530786181 73.23065461096382) bank185233 +185234 POINT(39.78996812178689 74.7364560217647) bank185234 +185235 POINT(40.757474633689704 73.4528424001032) bank185235 +185236 POINT(40.968116682980266 73.35122176853892) bank185236 +185237 POINT(41.636947130085225 74.46116885853746) bank185237 +185238 POINT(41.441499878284 74.64797335819797) bank185238 +185239 POINT(41.24698862011807 73.41376530128082) bank185239 +185240 POINT(41.23005723441832 73.53902022816449) bank185240 +185241 POINT(40.33799622340251 74.67647694238984) bank185241 +185242 POINT(41.64793300751973 73.1919216783959) bank185242 +185243 POINT(41.272390378038395 73.15493260196952) bank185243 +185244 POINT(40.29171939667471 73.56451043107613) bank185244 +185245 POINT(41.074927947055066 74.98647435640099) bank185245 +185246 POINT(41.67580723579856 73.45156464512158) bank185246 +185247 POINT(41.36443704937431 74.18379397381547) bank185247 +185248 POINT(40.99804006036784 74.22528504489965) bank185248 +185249 POINT(40.80655930332717 74.32532364534175) bank185249 +185250 POINT(41.4638697691817 73.3031165839053) bank185250 +185251 POINT(40.66797943362942 73.63035039596075) bank185251 +185252 POINT(39.97284956067913 73.06516831565315) bank185252 +185253 POINT(41.597897296641975 73.09912494873278) bank185253 +185254 POINT(40.07552319265887 73.48023922796011) bank185254 +185255 POINT(40.83692029523081 73.40080255916313) bank185255 +185256 POINT(40.943131348968826 73.29861526944057) bank185256 +185257 POINT(40.06832364430386 73.38533335283843) bank185257 +185258 POINT(39.98036076759311 73.4556983238771) bank185258 +185259 POINT(41.683661015010664 74.5650361683213) bank185259 +185260 POINT(40.92617017823779 74.4750461387972) bank185260 +185261 POINT(40.44033564122232 74.05587000980752) bank185261 +185262 POINT(39.95147172739804 73.5946447066335) bank185262 +185263 POINT(39.84207911269371 73.11268527081207) bank185263 +185264 POINT(40.82112747288556 74.42582182999617) bank185264 +185265 POINT(40.925239178793014 74.43820524856903) bank185265 +185266 POINT(39.985377413020316 74.28877464893543) bank185266 +185267 POINT(41.11246147732659 73.50376767688213) bank185267 +185268 POINT(40.103150899962884 74.00391646178302) bank185268 +185269 POINT(41.190823916477385 73.61415690631553) bank185269 +185270 POINT(40.382883208507764 73.14217580441539) bank185270 +185271 POINT(40.64781292201901 73.07309325751845) bank185271 +185272 POINT(39.74481845326062 74.27973566053389) bank185272 +185273 POINT(41.198746669093964 73.51897075428185) bank185273 +185274 POINT(41.09653138167471 73.05068997769548) bank185274 +185275 POINT(40.50411929002171 74.5314321145075) bank185275 +185276 POINT(40.0978045754588 73.28649105989047) bank185276 +185277 POINT(40.3222740593923 74.13789215115237) bank185277 +185278 POINT(40.92860131089271 73.91189501169644) bank185278 +185279 POINT(41.35530344707701 73.01968550768798) bank185279 +185280 POINT(41.60558980520257 73.37065456584081) bank185280 +185281 POINT(40.49629627856281 73.34114206649448) bank185281 +185282 POINT(39.9864412048452 73.4580931692979) bank185282 +185283 POINT(40.05306012079562 74.6843974278096) bank185283 +185284 POINT(41.019379131426334 74.36830699980091) bank185284 +185285 POINT(41.36244636791693 74.36884688839406) bank185285 +185286 POINT(40.51556718213117 73.03016119826144) bank185286 +185287 POINT(40.894246062006154 74.87775399736071) bank185287 +185288 POINT(40.50410843560521 74.46890076961547) bank185288 +185289 POINT(41.53495416921152 73.04111127911827) bank185289 +185290 POINT(40.541465176876635 74.36623196753777) bank185290 +185291 POINT(41.42684336739778 74.19188610776257) bank185291 +185292 POINT(41.17818256613965 73.25258247111033) bank185292 +185293 POINT(40.21780413893639 73.4005879674796) bank185293 +185294 POINT(39.811098467891426 74.86591972817257) bank185294 +185295 POINT(40.36069959883196 74.60250325070844) bank185295 +185296 POINT(41.0187920316822 73.03633991160781) bank185296 +185297 POINT(40.44400190234268 73.79216119317495) bank185297 +185298 POINT(41.10371703240114 74.85079282223224) bank185298 +185299 POINT(40.48012055732202 75.00356117467824) bank185299 +185300 POINT(40.30412535721755 74.35044188880504) bank185300 +185301 POINT(41.66740819258945 74.49502124943824) bank185301 +185302 POINT(40.23809646332943 74.36979573536293) bank185302 +185303 POINT(40.74606097467735 73.233641687849) bank185303 +185304 POINT(39.81229534370526 73.32592442799198) bank185304 +185305 POINT(40.4006631537377 74.31962786489888) bank185305 +185306 POINT(41.40229958977773 74.92025745760306) bank185306 +185307 POINT(41.56622921802143 73.31892269265363) bank185307 +185308 POINT(41.18295405037016 73.66911653714402) bank185308 +185309 POINT(39.95475787553905 73.27615721136718) bank185309 +185310 POINT(40.893090043305655 74.19638888927574) bank185310 +185311 POINT(40.055485349762506 74.41111276123253) bank185311 +185312 POINT(41.260949644273346 73.46227943360341) bank185312 +185313 POINT(40.547302413913705 73.75981101246082) bank185313 +185314 POINT(40.93766554016902 74.41245845225922) bank185314 +185315 POINT(39.785442381337944 74.65940895279186) bank185315 +185316 POINT(40.55949068071449 73.08297912989784) bank185316 +185317 POINT(41.13381342270804 73.9271364127291) bank185317 +185318 POINT(41.59725246483267 74.19450319585832) bank185318 +185319 POINT(40.679730483944795 73.99845146613282) bank185319 +185320 POINT(40.47189042072144 73.43488855098573) bank185320 +185321 POINT(41.23206988665287 73.87739957055742) bank185321 +185322 POINT(41.141403920199764 73.20940012907639) bank185322 +185323 POINT(40.546119267230594 73.87822202655596) bank185323 +185324 POINT(41.310814945354 74.21819471787921) bank185324 +185325 POINT(40.811509896849365 74.71236637089014) bank185325 +185326 POINT(41.35568977786954 74.42387313378578) bank185326 +185327 POINT(39.74463120278319 74.09327560189983) bank185327 +185328 POINT(40.77349641290487 73.76369656894875) bank185328 +185329 POINT(39.72093322528676 73.86053922879907) bank185329 +185330 POINT(40.51555336148288 74.93341008093785) bank185330 +185331 POINT(40.75604555584341 73.1436119265649) bank185331 +185332 POINT(39.84635201358955 73.06093384748807) bank185332 +185333 POINT(41.30064379462721 73.39096010425483) bank185333 +185334 POINT(39.733339719565265 73.06047518693632) bank185334 +185335 POINT(41.6996252788017 74.61689641290933) bank185335 +185336 POINT(41.427743507699944 74.75848784516543) bank185336 +185337 POINT(41.06587131950917 74.39272258502179) bank185337 +185338 POINT(41.049702216260904 74.6538992440349) bank185338 +185339 POINT(41.4531554394532 74.8385200038173) bank185339 +185340 POINT(39.7987454886399 74.03899711453845) bank185340 +185341 POINT(41.32992360918954 73.87865198892925) bank185341 +185342 POINT(41.53103908246861 73.9662995729896) bank185342 +185343 POINT(40.93646628702747 74.03863686797023) bank185343 +185344 POINT(39.904466968727185 74.71103912740304) bank185344 +185345 POINT(40.50251313395381 73.57779503419285) bank185345 +185346 POINT(40.0076528128508 73.66380730039556) bank185346 +185347 POINT(41.60877094233608 74.28661741352543) bank185347 +185348 POINT(40.228797320339645 73.46097859561462) bank185348 +185349 POINT(41.52108154132591 73.08612509655319) bank185349 +185350 POINT(41.53847559838331 74.52928443191996) bank185350 +185351 POINT(41.36041840151679 74.78605395254081) bank185351 +185352 POINT(41.707097266597934 74.43677272016747) bank185352 +185353 POINT(40.55202402423743 74.55072674505264) bank185353 +185354 POINT(41.09583591682903 74.33834146231344) bank185354 +185355 POINT(40.303185394051795 73.6462305350591) bank185355 +185356 POINT(41.567049880356464 73.67381854909642) bank185356 +185357 POINT(39.780803862599505 73.87224589174176) bank185357 +185358 POINT(41.65246762301328 74.97330496282945) bank185358 +185359 POINT(40.671854640819554 74.17247524982304) bank185359 +185360 POINT(40.191366842357354 74.26068478738287) bank185360 +185361 POINT(39.91911395483714 73.50533762012446) bank185361 +185362 POINT(40.037393425388025 74.60451551049918) bank185362 +185363 POINT(39.734658274200385 74.00864130206355) bank185363 +185364 POINT(41.65730525005312 73.71435491667985) bank185364 +185365 POINT(40.1672757706927 74.47747814105084) bank185365 +185366 POINT(41.044454783298335 74.87365237978774) bank185366 +185367 POINT(40.42787639338175 74.208431470238) bank185367 +185368 POINT(40.84978125480003 74.05279643071289) bank185368 +185369 POINT(41.69542889993467 73.16158437942246) bank185369 +185370 POINT(40.21559849005976 74.96585431370687) bank185370 +185371 POINT(41.69996403107043 73.98962940669746) bank185371 +185372 POINT(39.856082232866136 74.73531996868245) bank185372 +185373 POINT(40.287356110499786 74.59112401588976) bank185373 +185374 POINT(40.833344292084774 74.14044504532204) bank185374 +185375 POINT(40.31440782214712 73.74469519511678) bank185375 +185376 POINT(41.4660011254215 73.38935690033253) bank185376 +185377 POINT(40.30010822006701 73.14403001766571) bank185377 +185378 POINT(40.10694048309191 74.91018225762511) bank185378 +185379 POINT(40.041802335321364 74.83037843486488) bank185379 +185380 POINT(40.726659964097074 73.40684579010272) bank185380 +185381 POINT(40.315947687462 74.28113281130176) bank185381 +185382 POINT(41.3850532023683 73.07636169310756) bank185382 +185383 POINT(40.86261773068365 73.46784236772287) bank185383 +185384 POINT(41.29808626997894 74.00081664423) bank185384 +185385 POINT(39.89427330619369 73.98004309188636) bank185385 +185386 POINT(41.55131778411666 74.11275385288079) bank185386 +185387 POINT(41.1781777575961 73.96699470671626) bank185387 +185388 POINT(41.423689479601464 73.1627432600247) bank185388 +185389 POINT(41.0214461047463 74.56118256488685) bank185389 +185390 POINT(40.85908830628061 74.4535864096994) bank185390 +185391 POINT(40.68136886284254 74.10024893025239) bank185391 +185392 POINT(40.803676684517356 73.53631971950779) bank185392 +185393 POINT(40.66827644142007 74.4857423105919) bank185393 +185394 POINT(40.22575659209581 73.06811670075501) bank185394 +185395 POINT(40.60474398836133 74.81147358817383) bank185395 +185396 POINT(41.14865145963271 74.21441451008316) bank185396 +185397 POINT(40.82323586084352 73.74805112269641) bank185397 +185398 POINT(41.691648481720144 73.49470477119382) bank185398 +185399 POINT(40.62054285028409 73.71939557448967) bank185399 +185400 POINT(41.652581923267256 74.85548955885433) bank185400 +185401 POINT(39.76974639215351 74.73676651316171) bank185401 +185402 POINT(40.38641272760874 73.43278145487649) bank185402 +185403 POINT(40.178871901056915 74.45925801738693) bank185403 +185404 POINT(40.038890120409604 73.01998770893992) bank185404 +185405 POINT(39.87654788900739 74.41751153098413) bank185405 +185406 POINT(41.10216144283635 73.58538390247628) bank185406 +185407 POINT(39.90262647090044 73.96657906455147) bank185407 +185408 POINT(40.56156679186269 73.2221045963602) bank185408 +185409 POINT(41.64300466960377 74.33485903511287) bank185409 +185410 POINT(40.545551917839255 74.79796663152548) bank185410 +185411 POINT(39.786002647729006 73.898893717142) bank185411 +185412 POINT(40.675819788493435 73.24088079434458) bank185412 +185413 POINT(40.52791170335103 73.9839474791467) bank185413 +185414 POINT(40.36992670143463 73.86596435538662) bank185414 +185415 POINT(41.446246037263435 73.72482355353303) bank185415 +185416 POINT(40.025476529194876 73.49510721468462) bank185416 +185417 POINT(40.13214025284642 74.54730996354978) bank185417 +185418 POINT(40.97296232985113 74.38514810449892) bank185418 +185419 POINT(40.692521815326785 74.82004893718214) bank185419 +185420 POINT(40.93966248147618 73.01914144514372) bank185420 +185421 POINT(40.157719693022116 74.40980078782734) bank185421 +185422 POINT(39.83904498689698 74.19235240117354) bank185422 +185423 POINT(40.379229726487274 73.77310336232597) bank185423 +185424 POINT(40.508597315526636 73.74789407143886) bank185424 +185425 POINT(41.614268353470905 73.27767785055318) bank185425 +185426 POINT(39.85906370578669 74.12777690327833) bank185426 +185427 POINT(40.074456522174515 73.54151200228941) bank185427 +185428 POINT(40.76599752987275 73.67350732010783) bank185428 +185429 POINT(40.14544171711939 73.00977404586625) bank185429 +185430 POINT(40.04647584145165 73.51605086381267) bank185430 +185431 POINT(41.32140658644647 73.94982407770134) bank185431 +185432 POINT(41.642070695470544 73.45520253917775) bank185432 +185433 POINT(41.173236179982595 73.04904786943234) bank185433 +185434 POINT(41.23400579125545 74.20957222934045) bank185434 +185435 POINT(40.684926703742754 74.11835274296418) bank185435 +185436 POINT(40.89702790157708 73.52505489910105) bank185436 +185437 POINT(40.672944349401455 73.1580198983395) bank185437 +185438 POINT(41.60497293818942 73.8612285066527) bank185438 +185439 POINT(40.37076103732467 74.86132141984218) bank185439 +185440 POINT(39.714982050216925 74.85154377406641) bank185440 +185441 POINT(40.69990870940668 74.80472596492284) bank185441 +185442 POINT(40.821678620494666 73.23451417332949) bank185442 +185443 POINT(40.89912040078782 74.63427068232745) bank185443 +185444 POINT(41.150240431333984 73.8391772670915) bank185444 +185445 POINT(40.5189739315131 74.16165437241953) bank185445 +185446 POINT(40.12263646673255 73.94037518824835) bank185446 +185447 POINT(40.28134163295949 74.56389886457244) bank185447 +185448 POINT(41.58719043356214 74.13550554555356) bank185448 +185449 POINT(41.01761463885125 73.81355491865276) bank185449 +185450 POINT(41.10229968181943 73.72776116333976) bank185450 +185451 POINT(40.38546731475699 73.57321534204542) bank185451 +185452 POINT(39.86578287303711 73.26573798241596) bank185452 +185453 POINT(41.280577107005584 74.78288401609215) bank185453 +185454 POINT(40.11884349545962 74.12789315335849) bank185454 +185455 POINT(40.93016576444617 73.78321143347218) bank185455 +185456 POINT(40.94201433421104 74.46188348283192) bank185456 +185457 POINT(40.080738972134704 73.44074393900088) bank185457 +185458 POINT(39.94835236520549 73.4055163347129) bank185458 +185459 POINT(40.18285247108567 74.74158002929396) bank185459 +185460 POINT(41.37172581293116 73.32214752236094) bank185460 +185461 POINT(39.89380155348621 74.8838242861751) bank185461 +185462 POINT(41.52124561238691 73.46540054667973) bank185462 +185463 POINT(41.188121779970444 74.51401789664648) bank185463 +185464 POINT(40.86485680123148 73.4235327813026) bank185464 +185465 POINT(40.342195342977384 74.71707363754946) bank185465 +185466 POINT(40.74189399283477 73.20760180963549) bank185466 +185467 POINT(41.13968335914262 74.53492149161676) bank185467 +185468 POINT(40.33958020698475 73.58177797584419) bank185468 +185469 POINT(41.26970618957202 74.07713733718009) bank185469 +185470 POINT(40.498329858749116 73.20495967343003) bank185470 +185471 POINT(40.95939683511493 74.57556857521844) bank185471 +185472 POINT(41.34180597493312 74.23590793715269) bank185472 +185473 POINT(40.850922095071745 73.47940873099674) bank185473 +185474 POINT(40.226356100600285 73.55980965138846) bank185474 +185475 POINT(41.02742386686215 74.80295287998361) bank185475 +185476 POINT(39.71704753507694 74.07856831445167) bank185476 +185477 POINT(40.96302997103912 74.1058363252712) bank185477 +185478 POINT(41.428501105605314 74.58513592220149) bank185478 +185479 POINT(41.37487010169331 73.55838529024547) bank185479 +185480 POINT(39.76196792770726 73.93735565695721) bank185480 +185481 POINT(39.768071995768764 74.46696217885352) bank185481 +185482 POINT(40.508400710085596 73.39721194397296) bank185482 +185483 POINT(40.433327206645004 74.56559642472791) bank185483 +185484 POINT(41.15765148411789 74.19181342302369) bank185484 +185485 POINT(39.870368772242564 73.918413411313) bank185485 +185486 POINT(40.78204131324736 73.37715692779119) bank185486 +185487 POINT(41.43738188197658 74.4308264381435) bank185487 +185488 POINT(40.90522102776647 73.63263312652602) bank185488 +185489 POINT(40.42166072485041 73.0612588164682) bank185489 +185490 POINT(40.89028871816577 73.17772225874411) bank185490 +185491 POINT(41.68669284480573 74.2375662769966) bank185491 +185492 POINT(40.130681729869316 74.59002342553792) bank185492 +185493 POINT(41.321137009815985 74.8049973937394) bank185493 +185494 POINT(40.11444244889828 73.89429914679663) bank185494 +185495 POINT(39.881830975168086 74.51060642563128) bank185495 +185496 POINT(39.96344175896581 74.6223347798806) bank185496 +185497 POINT(39.86032965480291 73.13100310915797) bank185497 +185498 POINT(40.685319009568474 73.31037432277621) bank185498 +185499 POINT(40.010073299649456 74.40654907991146) bank185499 +185500 POINT(41.69701656045151 74.33940404141046) bank185500 +185501 POINT(41.15337741306256 74.1228089025846) bank185501 +185502 POINT(40.981142993127094 73.72974972664862) bank185502 +185503 POINT(40.7596291731573 74.03100130506185) bank185503 +185504 POINT(40.96750219731458 74.51486213482792) bank185504 +185505 POINT(39.95472588873675 73.37793058779582) bank185505 +185506 POINT(40.20299705248617 74.00236930210849) bank185506 +185507 POINT(40.761681447287145 74.46122028591158) bank185507 +185508 POINT(39.87068642034917 74.09788652869085) bank185508 +185509 POINT(40.5709258687234 73.88156341634686) bank185509 +185510 POINT(41.08420362177635 73.74807184140992) bank185510 +185511 POINT(39.87754765392639 74.10080584355667) bank185511 +185512 POINT(40.23383356215943 73.93585475036737) bank185512 +185513 POINT(40.643552113040876 74.51066439696544) bank185513 +185514 POINT(40.380769476360946 73.47563646797735) bank185514 +185515 POINT(41.09270938899203 73.2050961821488) bank185515 +185516 POINT(41.00345650477186 73.51803004081474) bank185516 +185517 POINT(40.14800130999975 74.23113831444064) bank185517 +185518 POINT(41.346453503810736 74.43717801111632) bank185518 +185519 POINT(40.45179083946294 73.78632107913678) bank185519 +185520 POINT(41.62426250641456 73.79216475811765) bank185520 +185521 POINT(40.985355516947706 73.77894042311144) bank185521 +185522 POINT(41.62126048077517 73.4785277795258) bank185522 +185523 POINT(39.92728721150012 73.85016039310341) bank185523 +185524 POINT(41.23375311216378 74.08014846262539) bank185524 +185525 POINT(40.90516595848565 73.90168905772917) bank185525 +185526 POINT(40.94104612957558 74.76116527414467) bank185526 +185527 POINT(41.120142371693085 73.13863679129909) bank185527 +185528 POINT(41.19087766964695 74.80847907697034) bank185528 +185529 POINT(41.60682654984611 74.55908057068935) bank185529 +185530 POINT(39.825166058609945 73.84629547971839) bank185530 +185531 POINT(39.77680213269429 73.48537763097735) bank185531 +185532 POINT(41.12462933045063 74.32016477732105) bank185532 +185533 POINT(41.624131063006864 74.35713010879219) bank185533 +185534 POINT(39.76621478763705 73.16114919704351) bank185534 +185535 POINT(41.64399511017057 73.97055182418771) bank185535 +185536 POINT(40.34277819913261 74.47041051568031) bank185536 +185537 POINT(39.95105871396638 74.14430898904477) bank185537 +185538 POINT(41.339297322408285 74.96431192216396) bank185538 +185539 POINT(40.17078548434255 73.50828031574983) bank185539 +185540 POINT(41.105378564251914 73.39272534715059) bank185540 +185541 POINT(40.076769428797924 73.70310644322763) bank185541 +185542 POINT(40.76307237287407 73.73362387857189) bank185542 +185543 POINT(41.44402382386458 73.35302224282721) bank185543 +185544 POINT(40.62322109704572 74.95413352930728) bank185544 +185545 POINT(39.991880657894434 74.17970451009936) bank185545 +185546 POINT(41.099677252527975 74.37144526687291) bank185546 +185547 POINT(41.3262606747799 73.02386614352697) bank185547 +185548 POINT(40.738563080019915 74.09347944751671) bank185548 +185549 POINT(40.45755931131423 73.29871678023333) bank185549 +185550 POINT(40.308737453237725 74.51861602141692) bank185550 +185551 POINT(39.867268367135786 73.40705594932103) bank185551 +185552 POINT(40.28089908951139 73.99838714823707) bank185552 +185553 POINT(41.35776544720461 74.57284009714095) bank185553 +185554 POINT(40.02607817207162 73.68672138937502) bank185554 +185555 POINT(40.74384383121256 73.18822185344037) bank185555 +185556 POINT(39.79673035508374 73.52003016648209) bank185556 +185557 POINT(41.70132929768211 73.273853317821) bank185557 +185558 POINT(41.3249638268922 74.79476907220575) bank185558 +185559 POINT(39.92915999205849 74.02966941847934) bank185559 +185560 POINT(40.285360330594244 74.04990303356723) bank185560 +185561 POINT(41.09234774056858 73.12210129520979) bank185561 +185562 POINT(39.7431253906286 74.99531512117095) bank185562 +185563 POINT(40.287189152755126 73.583936959159) bank185563 +185564 POINT(40.16633763828464 73.69278526644172) bank185564 +185565 POINT(41.23827483830304 73.16663294397225) bank185565 +185566 POINT(40.37893617821762 74.69689173831557) bank185566 +185567 POINT(41.419361310268755 74.95958644231627) bank185567 +185568 POINT(40.844367844221644 74.26532602157121) bank185568 +185569 POINT(41.65668567624917 74.46922417213261) bank185569 +185570 POINT(40.51055031777047 73.93646133419985) bank185570 +185571 POINT(41.4515800858619 74.72218063847971) bank185571 +185572 POINT(41.483906509602804 74.6995018030424) bank185572 +185573 POINT(41.36086330238052 73.93660843895758) bank185573 +185574 POINT(41.12700842659165 74.26324714067675) bank185574 +185575 POINT(40.12748059222186 73.6491250372204) bank185575 +185576 POINT(40.27608128796533 73.19919017094524) bank185576 +185577 POINT(39.82549800814993 73.42130748364762) bank185577 +185578 POINT(41.15899616180204 74.04702011908061) bank185578 +185579 POINT(39.86811774478145 74.23512686567135) bank185579 +185580 POINT(40.98766505868478 73.15699551179809) bank185580 +185581 POINT(40.9452585267496 73.94363967163747) bank185581 +185582 POINT(39.828785431400014 74.13864616437235) bank185582 +185583 POINT(40.41487432845927 74.42898454713693) bank185583 +185584 POINT(39.743711822200694 74.27052816825851) bank185584 +185585 POINT(41.19846662339824 73.21015150108506) bank185585 +185586 POINT(41.60706011659705 73.34326963111104) bank185586 +185587 POINT(40.568021419811345 74.58975354068158) bank185587 +185588 POINT(41.23306633373648 73.62438366412374) bank185588 +185589 POINT(39.898624283048804 74.67044492730028) bank185589 +185590 POINT(40.404694587209114 74.18875265386347) bank185590 +185591 POINT(40.330065345467354 74.14451680471524) bank185591 +185592 POINT(39.819834086813124 74.61596339846155) bank185592 +185593 POINT(41.26973828768177 74.97404616478896) bank185593 +185594 POINT(39.74343630294528 74.41060083983817) bank185594 +185595 POINT(40.51739021848557 73.0265090246462) bank185595 +185596 POINT(40.48666339973436 74.28145152742195) bank185596 +185597 POINT(41.295219795147275 73.88255902550841) bank185597 +185598 POINT(41.215038309185445 74.72662356194495) bank185598 +185599 POINT(40.99094212914988 74.98765164622087) bank185599 +185600 POINT(40.32681675802776 74.13017158956094) bank185600 +185601 POINT(41.12517117895734 74.57711803619317) bank185601 +185602 POINT(40.11535748020583 74.4929531319541) bank185602 +185603 POINT(40.200314510603704 73.88532562532606) bank185603 +185604 POINT(40.72233689742301 74.39138924262971) bank185604 +185605 POINT(40.70163593081756 74.50134809414583) bank185605 +185606 POINT(40.43441355465669 74.37960044965297) bank185606 +185607 POINT(40.88962793388336 74.97575016879779) bank185607 +185608 POINT(40.52103755155289 74.21670066914724) bank185608 +185609 POINT(41.28674671483699 74.08180872640396) bank185609 +185610 POINT(40.65968870092933 74.04953650012557) bank185610 +185611 POINT(40.26335456413879 73.61295465271073) bank185611 +185612 POINT(40.55849848500148 74.04291388104001) bank185612 +185613 POINT(41.345306562368876 73.42923310756866) bank185613 +185614 POINT(40.434989660230976 74.51062283208847) bank185614 +185615 POINT(41.62008227444501 74.40410265554002) bank185615 +185616 POINT(40.49431929072577 73.70368166396028) bank185616 +185617 POINT(40.38503105040794 73.43521498743917) bank185617 +185618 POINT(40.232853762924584 74.22880498294288) bank185618 +185619 POINT(40.30891568903789 73.28784607047773) bank185619 +185620 POINT(39.85340400420712 73.00745984878616) bank185620 +185621 POINT(39.88121055371937 74.47392058271141) bank185621 +185622 POINT(40.79623714973983 73.46701526803686) bank185622 +185623 POINT(40.92658927274542 74.32768055773784) bank185623 +185624 POINT(40.00470493542231 73.9164025316197) bank185624 +185625 POINT(40.22362940045679 74.9730116209931) bank185625 +185626 POINT(39.8582550255239 74.23711228972108) bank185626 +185627 POINT(40.00647367542862 74.31977384087755) bank185627 +185628 POINT(39.980607437722234 73.68935936905426) bank185628 +185629 POINT(41.15649969298224 73.01697565790512) bank185629 +185630 POINT(41.00993225772277 74.36806084954709) bank185630 +185631 POINT(41.084052000151765 74.00604307004018) bank185631 +185632 POINT(41.05929397079477 74.13052476891959) bank185632 +185633 POINT(40.42901806217602 73.2595927452887) bank185633 +185634 POINT(41.570836815993395 74.15246367142268) bank185634 +185635 POINT(40.393229229695685 74.51289035345462) bank185635 +185636 POINT(40.360476134680816 74.7419043950133) bank185636 +185637 POINT(40.05766938372928 74.61211277958091) bank185637 +185638 POINT(40.53552265428299 74.74484515495297) bank185638 +185639 POINT(41.18049473391169 74.57626097673041) bank185639 +185640 POINT(40.52726517513134 74.25097779395009) bank185640 +185641 POINT(40.62395059287485 74.27055720546981) bank185641 +185642 POINT(40.857610854848694 73.83309612157429) bank185642 +185643 POINT(40.68895805842323 73.45761484066091) bank185643 +185644 POINT(41.20793991039682 73.04628130248157) bank185644 +185645 POINT(41.356030394845654 74.58538409123861) bank185645 +185646 POINT(40.402883102420745 74.20653504950114) bank185646 +185647 POINT(40.519869991709584 73.96714707953244) bank185647 +185648 POINT(40.28670748175792 73.45499248202366) bank185648 +185649 POINT(41.68688335516447 73.60958533706079) bank185649 +185650 POINT(41.53367971222093 73.29914874688541) bank185650 +185651 POINT(40.05817421638289 74.59013773979824) bank185651 +185652 POINT(41.37970757996191 73.46358541384546) bank185652 +185653 POINT(41.213520814968426 73.18060613064442) bank185653 +185654 POINT(40.60895595224278 73.0376413976477) bank185654 +185655 POINT(41.596830991261015 73.12102923268723) bank185655 +185656 POINT(39.73491722497349 73.59597565153277) bank185656 +185657 POINT(39.88940072294969 73.23949177340037) bank185657 +185658 POINT(40.93326579403565 73.10058014988971) bank185658 +185659 POINT(41.198064225076855 73.80519470853001) bank185659 +185660 POINT(40.543608299312254 74.51322104868147) bank185660 +185661 POINT(40.12408155205066 74.66399283066868) bank185661 +185662 POINT(40.95788638031335 74.57202974509266) bank185662 +185663 POINT(40.857573019857966 73.9442382284394) bank185663 +185664 POINT(39.804418956634734 74.89754367698865) bank185664 +185665 POINT(40.198286190209984 73.25120067178153) bank185665 +185666 POINT(39.862306162070624 73.61400080999107) bank185666 +185667 POINT(41.12382572217592 73.78229714373943) bank185667 +185668 POINT(40.231513186898255 74.09899975433125) bank185668 +185669 POINT(41.44300701115765 73.07363315449425) bank185669 +185670 POINT(39.98309584288028 74.97157261765568) bank185670 +185671 POINT(40.65671141766045 74.8442279966608) bank185671 +185672 POINT(40.77460798244814 73.14405230434024) bank185672 +185673 POINT(41.04130820021937 74.53071162217819) bank185673 +185674 POINT(39.85722931688514 74.24233804171575) bank185674 +185675 POINT(40.26210927722316 73.63969978175932) bank185675 +185676 POINT(41.60896362702073 74.54909576924068) bank185676 +185677 POINT(41.03621467981149 73.74895041823711) bank185677 +185678 POINT(40.28613470202532 74.99502522044945) bank185678 +185679 POINT(40.30550706568659 73.91639447188345) bank185679 +185680 POINT(39.966614838398975 74.14621204312387) bank185680 +185681 POINT(39.84253461740285 74.39427803442345) bank185681 +185682 POINT(40.67643992273818 73.67251193254539) bank185682 +185683 POINT(39.75292991795864 74.21521631637287) bank185683 +185684 POINT(41.626547323510266 74.21557993705925) bank185684 +185685 POINT(41.26153716906507 74.43066589840103) bank185685 +185686 POINT(41.35983836021217 74.5989781057005) bank185686 +185687 POINT(40.01060822406629 73.0544423946657) bank185687 +185688 POINT(40.28833029787943 74.03917594572376) bank185688 +185689 POINT(40.09132503257997 74.4796403324541) bank185689 +185690 POINT(40.854274563257576 74.70685767060212) bank185690 +185691 POINT(40.518444916349864 74.69424537754924) bank185691 +185692 POINT(40.931614051840455 74.29915005292737) bank185692 +185693 POINT(39.7516574441075 74.56792988517883) bank185693 +185694 POINT(41.61799728621042 73.08921456097526) bank185694 +185695 POINT(40.42265373573758 74.6792713190397) bank185695 +185696 POINT(41.100909501125685 74.17999122687567) bank185696 +185697 POINT(41.04810621208868 74.48450005810005) bank185697 +185698 POINT(41.40813653005501 74.43186572232426) bank185698 +185699 POINT(41.50690354725219 73.20765231669334) bank185699 +185700 POINT(40.219611102582036 73.8335758859925) bank185700 +185701 POINT(40.28287117644733 73.94369227666178) bank185701 +185702 POINT(40.8439402364834 73.15590492546569) bank185702 +185703 POINT(40.80226403154794 73.97232919277731) bank185703 +185704 POINT(40.980546315336895 74.80236318011404) bank185704 +185705 POINT(39.77061782307754 73.21087209514592) bank185705 +185706 POINT(39.9642232608805 73.89839974362785) bank185706 +185707 POINT(40.030881483327306 74.36954515660861) bank185707 +185708 POINT(40.37920579208022 73.02341702673668) bank185708 +185709 POINT(41.19906747640453 74.34823912943965) bank185709 +185710 POINT(41.51709078971684 73.98250635510576) bank185710 +185711 POINT(40.867733928446896 74.02638478178386) bank185711 +185712 POINT(40.05694781681399 74.00125035641454) bank185712 +185713 POINT(41.68785028035198 73.79623596927117) bank185713 +185714 POINT(41.17660601285653 74.48379227579065) bank185714 +185715 POINT(41.06088146127185 74.62915103732458) bank185715 +185716 POINT(40.56431155048486 73.3984486622429) bank185716 +185717 POINT(40.30086502896146 73.59270412424122) bank185717 +185718 POINT(40.095202169040775 74.6042159283855) bank185718 +185719 POINT(40.65868401617068 73.50449071637945) bank185719 +185720 POINT(40.521815446123995 73.26634762799661) bank185720 +185721 POINT(40.714854437936836 73.81276879568895) bank185721 +185722 POINT(41.24952661855912 74.59166810101702) bank185722 +185723 POINT(41.66695020057944 74.01083792735997) bank185723 +185724 POINT(39.72112427666472 74.61820025346535) bank185724 +185725 POINT(41.48791685418099 74.25640718520621) bank185725 +185726 POINT(40.491493232166135 74.64049891206149) bank185726 +185727 POINT(40.96977251436491 74.41420157403074) bank185727 +185728 POINT(39.810114588005604 74.75036705109042) bank185728 +185729 POINT(39.75140476163426 73.35390585950816) bank185729 +185730 POINT(40.91555796166111 74.01531173834168) bank185730 +185731 POINT(40.34181967279868 73.79206896292139) bank185731 +185732 POINT(39.7410154480344 73.11619271843702) bank185732 +185733 POINT(39.819931613975 73.23159793330815) bank185733 +185734 POINT(39.89974616245943 73.6363647026264) bank185734 +185735 POINT(41.15469596068062 73.11789393691915) bank185735 +185736 POINT(41.403371474182215 74.59400955458734) bank185736 +185737 POINT(40.15206203790362 74.27801630957708) bank185737 +185738 POINT(40.79765906885774 74.48452343666034) bank185738 +185739 POINT(40.16913201407837 73.77832660648947) bank185739 +185740 POINT(40.39784357716101 74.11724587289764) bank185740 +185741 POINT(39.977017673518404 74.96995261967423) bank185741 +185742 POINT(40.5153004134957 73.33236817347249) bank185742 +185743 POINT(40.906557021434274 73.49954642743005) bank185743 +185744 POINT(40.88261584312464 74.32899062693969) bank185744 +185745 POINT(40.36539416682373 74.05810144688257) bank185745 +185746 POINT(41.537532504535335 74.87660886387265) bank185746 +185747 POINT(41.564513186932 74.38220900163492) bank185747 +185748 POINT(40.921967708645305 73.3448474670809) bank185748 +185749 POINT(41.538515605438576 73.46248143179419) bank185749 +185750 POINT(40.78418204570015 73.04487015144602) bank185750 +185751 POINT(40.807918494268236 73.12167097923815) bank185751 +185752 POINT(40.35809852241072 73.95063984569046) bank185752 +185753 POINT(41.58129612096743 74.07840189544042) bank185753 +185754 POINT(41.152866323301446 74.54440362284595) bank185754 +185755 POINT(40.40653716555675 74.70231698057901) bank185755 +185756 POINT(39.832220591045704 73.96439201090435) bank185756 +185757 POINT(41.09448971198191 74.43304993228416) bank185757 +185758 POINT(40.069300386664224 73.78985265223342) bank185758 +185759 POINT(41.51924352190969 73.91284409580997) bank185759 +185760 POINT(40.79394220188082 74.45931555635745) bank185760 +185761 POINT(41.44282810365536 73.0183693096636) bank185761 +185762 POINT(41.309091796324786 73.97578924806554) bank185762 +185763 POINT(41.57043617306992 73.4340279519796) bank185763 +185764 POINT(41.62444445596677 73.81404042338747) bank185764 +185765 POINT(40.54540630154567 73.32699965189218) bank185765 +185766 POINT(40.79446529020589 74.7607359401553) bank185766 +185767 POINT(41.18437675912192 74.89321859920452) bank185767 +185768 POINT(39.87155587735802 73.61272441808326) bank185768 +185769 POINT(41.698321275130525 73.35303276169189) bank185769 +185770 POINT(39.88598100937005 74.01222810683261) bank185770 +185771 POINT(41.201292489355104 74.53577955264484) bank185771 +185772 POINT(40.72839182540182 73.07130033995155) bank185772 +185773 POINT(41.44177669673258 73.91485823746758) bank185773 +185774 POINT(41.469244041172765 73.63050612714815) bank185774 +185775 POINT(40.687845647819614 74.75104373196956) bank185775 +185776 POINT(41.68474244575199 73.97823880309632) bank185776 +185777 POINT(40.907129837668705 74.08477669576986) bank185777 +185778 POINT(41.5056268414126 73.17106612904554) bank185778 +185779 POINT(41.60100635846072 73.8810398293502) bank185779 +185780 POINT(41.589792767946044 74.98676800453264) bank185780 +185781 POINT(39.88746382879084 73.46139064009984) bank185781 +185782 POINT(40.970370032483885 74.62954870962314) bank185782 +185783 POINT(41.2990174854629 74.88449163256084) bank185783 +185784 POINT(41.07076385034582 73.98739616276318) bank185784 +185785 POINT(40.06039087512898 73.83280135898508) bank185785 +185786 POINT(40.953006569384264 73.27787496581436) bank185786 +185787 POINT(40.7330604158837 73.09694693719864) bank185787 +185788 POINT(41.32557686144357 74.39423306727018) bank185788 +185789 POINT(40.491000105258344 74.76365215913201) bank185789 +185790 POINT(41.455855627799274 74.30570885908062) bank185790 +185791 POINT(40.56587596908415 74.5675211533754) bank185791 +185792 POINT(41.04687016389485 73.75368935733906) bank185792 +185793 POINT(39.77741771750744 73.57810622268634) bank185793 +185794 POINT(41.11924800090781 74.33540194476069) bank185794 +185795 POINT(40.89605479576288 73.68928715637145) bank185795 +185796 POINT(41.241908904704424 73.95285888040253) bank185796 +185797 POINT(41.429122327452696 74.30863569480411) bank185797 +185798 POINT(40.50668073642788 73.72892787654374) bank185798 +185799 POINT(39.951955406621806 74.82860743150461) bank185799 +185800 POINT(41.072259732558706 74.94210704448754) bank185800 +185801 POINT(41.42505856605126 74.9869012442698) bank185801 +185802 POINT(41.02931014077002 73.39193691524673) bank185802 +185803 POINT(41.66312781514654 74.29146366305973) bank185803 +185804 POINT(41.41164163942278 74.47484439960937) bank185804 +185805 POINT(41.694528721193855 74.46263047414695) bank185805 +185806 POINT(40.34716672610596 73.98776329308582) bank185806 +185807 POINT(41.46303495008987 74.07414668206945) bank185807 +185808 POINT(41.08502922752323 74.34087812177968) bank185808 +185809 POINT(41.615443818245666 73.42961462737487) bank185809 +185810 POINT(40.467456220537294 73.88512565892964) bank185810 +185811 POINT(41.175712455290274 74.87012588101094) bank185811 +185812 POINT(41.3089053927094 73.53693354233945) bank185812 +185813 POINT(40.319276662213824 73.16735810712053) bank185813 +185814 POINT(39.98993692338301 73.21961801517914) bank185814 +185815 POINT(40.361207207108976 74.98659653212039) bank185815 +185816 POINT(40.873724891105866 73.4628778733459) bank185816 +185817 POINT(40.94472144186251 74.54654710832934) bank185817 +185818 POINT(41.26723447816199 74.18870234045453) bank185818 +185819 POINT(41.14012636782448 73.6519889292036) bank185819 +185820 POINT(40.87269522415809 74.22408993195647) bank185820 +185821 POINT(40.587938498543544 74.97586255928654) bank185821 +185822 POINT(40.1332685185854 74.7826682704694) bank185822 +185823 POINT(40.268357148920394 73.67965109370424) bank185823 +185824 POINT(40.87196647180576 73.63687801066192) bank185824 +185825 POINT(39.88517834629484 74.31261424981) bank185825 +185826 POINT(40.68225387338331 74.19622559894871) bank185826 +185827 POINT(41.00719517429744 74.0921930946698) bank185827 +185828 POINT(41.05760316736838 73.91645361167033) bank185828 +185829 POINT(41.395910416873406 74.53153782647539) bank185829 +185830 POINT(41.2065602705674 74.68442441351826) bank185830 +185831 POINT(41.345606266929906 74.54313405032899) bank185831 +185832 POINT(41.06383818583124 73.11738594274749) bank185832 +185833 POINT(41.491282374552405 74.34124975785541) bank185833 +185834 POINT(40.920749753129925 73.3407285990219) bank185834 +185835 POINT(39.964413772376176 74.64828976241552) bank185835 +185836 POINT(40.86754276929035 74.80441733972212) bank185836 +185837 POINT(40.59768795381991 74.76350724123024) bank185837 +185838 POINT(41.14528081441147 73.49587023435096) bank185838 +185839 POINT(40.96163413360639 74.39243572453391) bank185839 +185840 POINT(40.03284830185457 73.03515352108123) bank185840 +185841 POINT(41.22339591299461 73.10223275522097) bank185841 +185842 POINT(40.16282448746985 73.89825383554842) bank185842 +185843 POINT(41.0604060784574 74.09919310180865) bank185843 +185844 POINT(41.35387653691504 74.9504065054691) bank185844 +185845 POINT(40.90769034762397 73.45164338092074) bank185845 +185846 POINT(40.373429119244406 74.02821881098555) bank185846 +185847 POINT(40.822900771413316 73.1558440750819) bank185847 +185848 POINT(41.602962686879984 73.30032356784596) bank185848 +185849 POINT(39.72262718935211 74.67591061850712) bank185849 +185850 POINT(41.608752217120404 74.50767282409949) bank185850 +185851 POINT(39.763137530012465 74.51829083187742) bank185851 +185852 POINT(40.82296344129917 74.78477066994473) bank185852 +185853 POINT(41.547037642029636 74.0801165293435) bank185853 +185854 POINT(41.563339252400944 73.49846417841991) bank185854 +185855 POINT(40.08055628516832 74.18504265468667) bank185855 +185856 POINT(39.851867069270064 73.74293363236445) bank185856 +185857 POINT(41.506585895898176 73.87996754720949) bank185857 +185858 POINT(39.907564292076955 73.61109065581284) bank185858 +185859 POINT(39.946143023521635 74.65666652041567) bank185859 +185860 POINT(41.20951084435297 74.62825002896248) bank185860 +185861 POINT(40.314439138360456 74.7375291948772) bank185861 +185862 POINT(39.95774437407376 73.25829689123411) bank185862 +185863 POINT(41.052513184327864 74.28819626442262) bank185863 +185864 POINT(39.97565779117335 74.9908363040255) bank185864 +185865 POINT(40.94550142257256 73.19825308436438) bank185865 +185866 POINT(40.893074774485534 73.91968900569962) bank185866 +185867 POINT(41.16407763911848 73.6083557569444) bank185867 +185868 POINT(41.18030762626706 73.30317963192259) bank185868 +185869 POINT(40.044613264435746 73.09687015411568) bank185869 +185870 POINT(40.711542688339705 74.35990514338644) bank185870 +185871 POINT(40.00325280486109 74.22970919106143) bank185871 +185872 POINT(40.65465425889546 74.11513879560748) bank185872 +185873 POINT(41.58939724585781 74.07475300876082) bank185873 +185874 POINT(40.84389112503528 73.21104049541316) bank185874 +185875 POINT(41.29978634015206 73.39355059236371) bank185875 +185876 POINT(39.94523475498778 74.35888821704683) bank185876 +185877 POINT(41.58644480173767 74.52315010014844) bank185877 +185878 POINT(41.21883199978333 73.68767598034846) bank185878 +185879 POINT(40.53933254653632 74.25491909994396) bank185879 +185880 POINT(41.46286286282778 74.1244277443064) bank185880 +185881 POINT(41.353875475995515 73.9221040518619) bank185881 +185882 POINT(40.21886952186319 74.62815711694655) bank185882 +185883 POINT(40.50951238256669 73.77028654057992) bank185883 +185884 POINT(40.887038116120955 73.73267473401918) bank185884 +185885 POINT(40.29874995802067 74.40855858062218) bank185885 +185886 POINT(41.36315128811623 74.66939488280099) bank185886 +185887 POINT(41.62676773510131 74.4896510305002) bank185887 +185888 POINT(40.06566256542291 74.1214875867199) bank185888 +185889 POINT(41.33726506738327 73.15265575561372) bank185889 +185890 POINT(41.522847237235794 74.87758995181328) bank185890 +185891 POINT(39.73039182999027 73.01978194907463) bank185891 +185892 POINT(40.988365410364146 73.89039329550312) bank185892 +185893 POINT(40.15932287884565 74.74594248586742) bank185893 +185894 POINT(39.929066372583094 74.11851026949408) bank185894 +185895 POINT(40.78206284863023 73.40830014828596) bank185895 +185896 POINT(41.605156117841545 73.09471618353699) bank185896 +185897 POINT(40.1964650004543 73.41469834630149) bank185897 +185898 POINT(39.986963116647814 74.85496265898846) bank185898 +185899 POINT(40.94203400106451 74.38134577525466) bank185899 +185900 POINT(40.11724443577633 73.07255380166286) bank185900 +185901 POINT(41.45826511477878 74.42777356466647) bank185901 +185902 POINT(40.92910268726163 74.4355828099947) bank185902 +185903 POINT(41.644030058169 74.57296049922232) bank185903 +185904 POINT(41.14707616641735 73.66759510001609) bank185904 +185905 POINT(40.99937850078826 73.0707833797807) bank185905 +185906 POINT(41.57259662646104 73.80136532177754) bank185906 +185907 POINT(39.84369943444662 73.27823777064208) bank185907 +185908 POINT(41.45358361101238 74.27314668607212) bank185908 +185909 POINT(41.03100911985722 74.5465578715315) bank185909 +185910 POINT(39.82479753261303 74.56137340890346) bank185910 +185911 POINT(41.2116461204031 73.9623201985135) bank185911 +185912 POINT(39.979097040612636 73.17171102737677) bank185912 +185913 POINT(40.8943773574684 74.1082904449405) bank185913 +185914 POINT(41.15118692333178 74.3958300335023) bank185914 +185915 POINT(39.93444618224514 73.61970652541326) bank185915 +185916 POINT(41.51314250095451 74.24888337771645) bank185916 +185917 POINT(41.23209467548006 74.66526536188482) bank185917 +185918 POINT(40.86237564656351 74.39464171643608) bank185918 +185919 POINT(40.966576221876515 74.70876775690783) bank185919 +185920 POINT(40.4387186487298 73.80581399573498) bank185920 +185921 POINT(40.003944436586195 74.24008819985474) bank185921 +185922 POINT(40.549515998766665 74.99681349620225) bank185922 +185923 POINT(40.955288178451944 73.15677826983519) bank185923 +185924 POINT(40.97864441415828 74.5544253756664) bank185924 +185925 POINT(41.18951786366704 73.11253476476827) bank185925 +185926 POINT(41.62488716528878 73.25714428621922) bank185926 +185927 POINT(40.75639723652448 74.07102407189977) bank185927 +185928 POINT(39.895499997287146 73.98773339484514) bank185928 +185929 POINT(41.269494163986536 74.91573285545968) bank185929 +185930 POINT(40.09401878415082 73.59211495018924) bank185930 +185931 POINT(41.041742243952 73.50341630917691) bank185931 +185932 POINT(41.155019191636455 74.18152824699591) bank185932 +185933 POINT(41.26034037742873 74.11184157405609) bank185933 +185934 POINT(40.885733025085656 74.83475716074227) bank185934 +185935 POINT(41.263752568435756 74.06004946861694) bank185935 +185936 POINT(39.735013926502695 73.43623016475556) bank185936 +185937 POINT(41.13583604378369 74.51060052703515) bank185937 +185938 POINT(40.57580002212554 74.15938417256761) bank185938 +185939 POINT(41.1762187215555 73.81248733078422) bank185939 +185940 POINT(41.27345477982001 73.51737015342704) bank185940 +185941 POINT(39.788002506576746 73.2040360771767) bank185941 +185942 POINT(39.74456734816344 73.24410762555637) bank185942 +185943 POINT(40.853526923629445 74.0050856756988) bank185943 +185944 POINT(40.15852372375771 74.06872407533398) bank185944 +185945 POINT(41.27841901121654 73.95572926346757) bank185945 +185946 POINT(41.07416696852251 74.82331728050444) bank185946 +185947 POINT(40.89605861277281 74.61881722826301) bank185947 +185948 POINT(40.52625931215486 74.1662843368748) bank185948 +185949 POINT(41.37567440690038 73.68130854499313) bank185949 +185950 POINT(41.47147703554125 73.59395371986196) bank185950 +185951 POINT(39.84141980613346 74.92609349226103) bank185951 +185952 POINT(41.51294696626557 74.22486090620369) bank185952 +185953 POINT(40.12335640256693 74.4606658964127) bank185953 +185954 POINT(41.14648307066249 74.9215144400549) bank185954 +185955 POINT(39.78063153748283 74.05280576879618) bank185955 +185956 POINT(40.32137312217144 74.31008517789732) bank185956 +185957 POINT(41.34945235563589 74.82303144602524) bank185957 +185958 POINT(41.13753943149324 74.4422544974332) bank185958 +185959 POINT(41.22687787675448 73.86820682525128) bank185959 +185960 POINT(40.87828591893701 73.42490730609069) bank185960 +185961 POINT(40.37903144093696 74.1982592499319) bank185961 +185962 POINT(40.46757870799995 73.62438778436318) bank185962 +185963 POINT(41.4211106610049 74.55498257040674) bank185963 +185964 POINT(41.68832753640465 73.64849502486602) bank185964 +185965 POINT(40.31359383575466 74.49181283140904) bank185965 +185966 POINT(40.52591359453244 74.82198531457375) bank185966 +185967 POINT(40.073816637272664 74.5238228748682) bank185967 +185968 POINT(40.089501380506846 73.8907732776294) bank185968 +185969 POINT(40.343695545785124 73.84535983670862) bank185969 +185970 POINT(40.56877553266204 73.4911759584096) bank185970 +185971 POINT(41.05661975014092 74.79536196403436) bank185971 +185972 POINT(41.412004811837456 74.23792521467601) bank185972 +185973 POINT(40.635841923891924 73.2865008616915) bank185973 +185974 POINT(40.00442448459941 74.7900377739417) bank185974 +185975 POINT(40.89199738067342 73.21465709368273) bank185975 +185976 POINT(41.139273559393864 74.86380851019858) bank185976 +185977 POINT(40.15821225288842 73.25172190777032) bank185977 +185978 POINT(40.85812462952094 73.59376232284177) bank185978 +185979 POINT(40.86176940354255 73.051784820504) bank185979 +185980 POINT(41.431594745905414 74.66896323449306) bank185980 +185981 POINT(40.67834264088576 74.08643010072855) bank185981 +185982 POINT(41.567610722766126 74.97590602861635) bank185982 +185983 POINT(39.824905381680416 74.40664918512203) bank185983 +185984 POINT(41.2466886221709 73.01919824012752) bank185984 +185985 POINT(40.29597634352215 74.5300680877096) bank185985 +185986 POINT(40.04931516237956 73.2453663093618) bank185986 +185987 POINT(41.12446368358156 74.52506930435176) bank185987 +185988 POINT(41.391561912949705 74.88424730376967) bank185988 +185989 POINT(40.09062236241606 73.45461494841688) bank185989 +185990 POINT(41.049337366518444 73.62775267504486) bank185990 +185991 POINT(40.84665258644627 74.39736408680722) bank185991 +185992 POINT(39.878005620721744 73.4342775241727) bank185992 +185993 POINT(40.9956919788464 73.365652125179) bank185993 +185994 POINT(41.54893978299855 74.94162859400333) bank185994 +185995 POINT(41.33785887318028 73.30766685413093) bank185995 +185996 POINT(41.636281568942444 73.9622903628444) bank185996 +185997 POINT(40.40586667525521 73.80108462779143) bank185997 +185998 POINT(39.91046442358766 73.8602972371749) bank185998 +185999 POINT(40.31097928487827 74.03789419497369) bank185999 +186000 POINT(41.272700789535136 73.73405433028145) bank186000 +186001 POINT(41.2708757333256 73.46228287179015) bank186001 +186002 POINT(39.91696467502346 74.31273663283284) bank186002 +186003 POINT(40.87614236228008 73.54651264129821) bank186003 +186004 POINT(40.974150750423675 74.14198994521713) bank186004 +186005 POINT(41.070151621268245 74.10298839930371) bank186005 +186006 POINT(40.948391493414846 73.39338844700178) bank186006 +186007 POINT(40.06013216106967 73.10907724002918) bank186007 +186008 POINT(39.99792931657188 74.40234158247146) bank186008 +186009 POINT(40.22464241211303 74.19234133160997) bank186009 +186010 POINT(40.649030905246555 74.41336711018634) bank186010 +186011 POINT(41.45702990295751 74.66484962523171) bank186011 +186012 POINT(39.81133335011968 74.61221242387666) bank186012 +186013 POINT(39.76157333092451 73.09277277866956) bank186013 +186014 POINT(39.95726292601537 74.23365561083837) bank186014 +186015 POINT(41.02915632121937 74.12494893695919) bank186015 +186016 POINT(41.07777651168592 74.69492500531398) bank186016 +186017 POINT(40.48628657363215 73.77065889318465) bank186017 +186018 POINT(40.75467121631094 73.22128155114075) bank186018 +186019 POINT(40.23753093134818 73.99150434350082) bank186019 +186020 POINT(40.38507639678669 73.61871703857089) bank186020 +186021 POINT(40.20396996798878 74.50051886774321) bank186021 +186022 POINT(39.942198116894915 74.18593085317266) bank186022 +186023 POINT(41.540046121239236 73.24258423298586) bank186023 +186024 POINT(41.68715740884018 73.96715943257178) bank186024 +186025 POINT(40.62828137914548 73.43087970790604) bank186025 +186026 POINT(40.88277282448582 74.26305065242416) bank186026 +186027 POINT(41.020474013646194 74.01543997695316) bank186027 +186028 POINT(40.17090341364074 73.62381601107386) bank186028 +186029 POINT(41.25147668473214 74.72540334372084) bank186029 +186030 POINT(40.43314185179722 73.86887658378706) bank186030 +186031 POINT(40.17297645470821 73.05762119116463) bank186031 +186032 POINT(40.10578706365107 74.87724588514064) bank186032 +186033 POINT(41.499636611272074 73.25854515846616) bank186033 +186034 POINT(41.695729337542346 73.73653773649355) bank186034 +186035 POINT(41.1027970908928 74.15854992237142) bank186035 +186036 POINT(41.059084955748396 73.23046233248004) bank186036 +186037 POINT(40.48888300829485 74.0310839919168) bank186037 +186038 POINT(41.23406661268252 74.88100219655472) bank186038 +186039 POINT(41.12633267629328 74.24258394953254) bank186039 +186040 POINT(40.647550295109845 74.17059118492465) bank186040 +186041 POINT(40.24885667818127 73.86688620127343) bank186041 +186042 POINT(40.28022696238695 74.2827849912258) bank186042 +186043 POINT(41.49512092421289 74.21381144120728) bank186043 +186044 POINT(41.49752207870196 73.99493639587958) bank186044 +186045 POINT(39.76829791373166 74.60436047282309) bank186045 +186046 POINT(40.658347850570976 73.77884111712643) bank186046 +186047 POINT(40.52533689892016 73.65650991184268) bank186047 +186048 POINT(39.880147684143836 74.84002953447359) bank186048 +186049 POINT(40.68688985091279 74.68559319492225) bank186049 +186050 POINT(41.518857946341086 73.28486669584663) bank186050 +186051 POINT(41.632771900219886 74.05799584683264) bank186051 +186052 POINT(40.33385488771795 73.47984790872378) bank186052 +186053 POINT(40.501031404781116 73.21643301547515) bank186053 +186054 POINT(40.21868581801542 73.17645300982213) bank186054 +186055 POINT(41.4378127940038 74.04373265618685) bank186055 +186056 POINT(41.02630094201071 73.94201183224087) bank186056 +186057 POINT(40.00856337526315 74.7295097994963) bank186057 +186058 POINT(41.24830990740425 73.90844783701799) bank186058 +186059 POINT(41.43448415890121 73.0195188218105) bank186059 +186060 POINT(40.86580348058346 73.1073251088532) bank186060 +186061 POINT(40.06860704932514 73.33369724856733) bank186061 +186062 POINT(41.58051687842231 73.6416154283311) bank186062 +186063 POINT(41.01057949416916 73.37027117278974) bank186063 +186064 POINT(41.6104691617187 74.49687858142279) bank186064 +186065 POINT(40.34859448399319 73.743771830301) bank186065 +186066 POINT(39.786111877964885 73.08772109490103) bank186066 +186067 POINT(41.15102712210234 73.41152629092112) bank186067 +186068 POINT(41.05476165343126 74.14607135522942) bank186068 +186069 POINT(41.54737497595786 74.84063201570916) bank186069 +186070 POINT(40.35711729978788 74.86588340728402) bank186070 +186071 POINT(40.72724840081376 74.2890581441655) bank186071 +186072 POINT(40.73851709140851 73.4997681201507) bank186072 +186073 POINT(40.03759339142642 74.05997551862973) bank186073 +186074 POINT(41.15113012729975 73.46598749283137) bank186074 +186075 POINT(40.604382803052005 74.32999918350632) bank186075 +186076 POINT(40.03597952443249 73.6926301729793) bank186076 +186077 POINT(39.97509125950806 74.95864502577986) bank186077 +186078 POINT(41.50890301093826 73.76581596504256) bank186078 +186079 POINT(40.94888664275366 73.2428730528687) bank186079 +186080 POINT(40.0465110090422 74.79100549375417) bank186080 +186081 POINT(39.762295342927615 73.72128303583118) bank186081 +186082 POINT(40.731537556531464 73.81811218146055) bank186082 +186083 POINT(40.8547718058583 74.50648318657669) bank186083 +186084 POINT(41.32986398971902 74.77027815937561) bank186084 +186085 POINT(39.77655570526146 73.33024921151195) bank186085 +186086 POINT(40.838304487900906 73.48291287400497) bank186086 +186087 POINT(41.34879882180359 74.6002969725338) bank186087 +186088 POINT(40.21876537610392 74.10317868960783) bank186088 +186089 POINT(40.254154299685794 74.93496865623796) bank186089 +186090 POINT(40.65653221092569 73.63330998914384) bank186090 +186091 POINT(41.65859811103989 73.62763733454618) bank186091 +186092 POINT(41.70152563460357 73.79991618419325) bank186092 +186093 POINT(41.60938129426273 73.596645059856) bank186093 +186094 POINT(41.65714762608887 74.60845770124881) bank186094 +186095 POINT(40.255782669956474 73.03160896918418) bank186095 +186096 POINT(40.1938885474091 73.07747522988971) bank186096 +186097 POINT(40.90256821852208 74.76142273740624) bank186097 +186098 POINT(41.244522141582465 74.93149322073845) bank186098 +186099 POINT(40.90246446388407 73.25485993678484) bank186099 +186100 POINT(40.68043395593111 73.63413734732455) bank186100 +186101 POINT(41.65798032582617 74.2421740144045) bank186101 +186102 POINT(39.959311318728865 73.33496633052943) bank186102 +186103 POINT(41.28227140327994 73.70261584230585) bank186103 +186104 POINT(40.42272335974021 74.40003132298094) bank186104 +186105 POINT(40.75332741913042 74.54979537928186) bank186105 +186106 POINT(41.687943519514306 74.41322451553894) bank186106 +186107 POINT(39.76628785428349 74.76544073084106) bank186107 +186108 POINT(41.436497217417674 74.60749030904597) bank186108 +186109 POINT(40.126149365854765 73.1949717988071) bank186109 +186110 POINT(39.83301800489666 74.93084253957569) bank186110 +186111 POINT(40.937851765289594 74.36502611700728) bank186111 +186112 POINT(40.69336205396035 74.88222438218281) bank186112 +186113 POINT(41.25790045429836 74.40147875757012) bank186113 +186114 POINT(41.61703180037093 74.13970713416057) bank186114 +186115 POINT(40.698842960150365 74.96908321994383) bank186115 +186116 POINT(40.24572318460411 74.28804251911104) bank186116 +186117 POINT(40.65257130914589 73.54696151206599) bank186117 +186118 POINT(40.71766975471589 74.9100920855074) bank186118 +186119 POINT(40.172201613651445 74.93976313698313) bank186119 +186120 POINT(41.65221799346973 74.9098890963858) bank186120 +186121 POINT(41.5810103039136 73.13635256823852) bank186121 +186122 POINT(40.899432438079295 74.62130519868538) bank186122 +186123 POINT(40.521869037139666 73.94392530061255) bank186123 +186124 POINT(40.39058946490568 73.99437439278752) bank186124 +186125 POINT(41.05785319284318 74.86357812953356) bank186125 +186126 POINT(40.50682450571533 74.94947555358445) bank186126 +186127 POINT(39.8998587666749 74.04668569634111) bank186127 +186128 POINT(40.58657222720448 73.75917571907888) bank186128 +186129 POINT(40.71424562473379 73.74035402239559) bank186129 +186130 POINT(41.012185404819306 74.0526922772037) bank186130 +186131 POINT(39.968848346108935 73.49085398552663) bank186131 +186132 POINT(40.176967949606855 73.9375514052252) bank186132 +186133 POINT(41.02270333483809 73.05455413729605) bank186133 +186134 POINT(40.671356091748265 73.71940927646821) bank186134 +186135 POINT(39.90544278699208 73.4821994940735) bank186135 +186136 POINT(40.836618103510176 73.0728717172415) bank186136 +186137 POINT(41.50483803134219 73.1964305944478) bank186137 +186138 POINT(40.62266104651431 74.59701799524564) bank186138 +186139 POINT(40.2364117909861 73.083276261161) bank186139 +186140 POINT(40.38823253727237 74.52848650875215) bank186140 +186141 POINT(40.539347509243896 74.36603676595736) bank186141 +186142 POINT(40.86237565943825 74.6947340533169) bank186142 +186143 POINT(40.27449092825555 74.1543076264243) bank186143 +186144 POINT(40.43232192484628 73.84347452692664) bank186144 +186145 POINT(41.073095833883684 74.55975158359061) bank186145 +186146 POINT(40.76888052045399 74.16252429591525) bank186146 +186147 POINT(39.7193781529916 73.81197639968701) bank186147 +186148 POINT(41.05653676293522 73.27026970479984) bank186148 +186149 POINT(41.60237045116766 73.53017033181663) bank186149 +186150 POINT(41.03818183902238 73.68172864888166) bank186150 +186151 POINT(40.037613491798645 73.87701693926826) bank186151 +186152 POINT(41.200778407417836 74.8090536415855) bank186152 +186153 POINT(40.30576727172172 74.74353629089008) bank186153 +186154 POINT(39.925544887571746 74.24483791972975) bank186154 +186155 POINT(39.78874009705969 73.45823369662021) bank186155 +186156 POINT(40.23812206907734 73.0914839855432) bank186156 +186157 POINT(40.69449291879308 74.48750845308892) bank186157 +186158 POINT(40.24739802732625 74.59133840380225) bank186158 +186159 POINT(39.88282811975845 74.1878147518946) bank186159 +186160 POINT(39.954222027921716 74.71924327296172) bank186160 +186161 POINT(40.1164058225027 74.70859785339555) bank186161 +186162 POINT(40.719211590297924 73.35085619651692) bank186162 +186163 POINT(39.776418202214984 73.43683899294908) bank186163 +186164 POINT(41.63091753389262 74.60472139770403) bank186164 +186165 POINT(41.56213127557274 74.74262811542002) bank186165 +186166 POINT(40.771691027257695 74.33746805324789) bank186166 +186167 POINT(40.739728023103325 73.30307107341396) bank186167 +186168 POINT(40.63462306665681 74.21967919769537) bank186168 +186169 POINT(40.368500778903275 74.73859313643332) bank186169 +186170 POINT(40.85070625347456 74.81807272711738) bank186170 +186171 POINT(39.89348924423163 73.15210773251023) bank186171 +186172 POINT(40.96719177707368 74.23198641187362) bank186172 +186173 POINT(41.422262271135935 74.0602237882841) bank186173 +186174 POINT(40.13367553329246 74.73310664044887) bank186174 +186175 POINT(40.33109979414731 74.21996002248507) bank186175 +186176 POINT(41.29373049481306 74.80950966021273) bank186176 +186177 POINT(39.85729424000097 74.90353760229048) bank186177 +186178 POINT(40.098514413739466 74.327291374588) bank186178 +186179 POINT(41.60122146772417 73.61607105486284) bank186179 +186180 POINT(41.4551743005668 73.92468774351373) bank186180 +186181 POINT(41.51981752452452 73.58505326087702) bank186181 +186182 POINT(41.34738714977084 73.6729084047737) bank186182 +186183 POINT(41.12881656936231 73.07411895096222) bank186183 +186184 POINT(40.13022085797252 73.68794961429224) bank186184 +186185 POINT(41.296030687960375 74.37780610263825) bank186185 +186186 POINT(41.019479418781465 73.43011931566276) bank186186 +186187 POINT(41.70530617564309 74.65470304147908) bank186187 +186188 POINT(40.639902731153235 74.02368382482184) bank186188 +186189 POINT(40.86424966011838 74.3751397051894) bank186189 +186190 POINT(40.45554944544853 73.676943390338) bank186190 +186191 POINT(41.42156942557695 73.85493803188866) bank186191 +186192 POINT(41.39845106091225 74.25924070808372) bank186192 +186193 POINT(40.10709354274256 73.75702714654068) bank186193 +186194 POINT(40.54484052039744 73.95422594049269) bank186194 +186195 POINT(41.69220380432695 73.44813348036183) bank186195 +186196 POINT(41.470447993921134 73.25574183321308) bank186196 +186197 POINT(40.10123482137838 74.99301701157599) bank186197 +186198 POINT(40.949532964770384 73.51011497403088) bank186198 +186199 POINT(40.19521226209446 73.44995042813532) bank186199 +186200 POINT(39.74775433200096 74.37493922628985) bank186200 +186201 POINT(40.80544617092978 73.6551038962349) bank186201 +186202 POINT(41.195704701620066 73.57513051700886) bank186202 +186203 POINT(40.61926014725601 74.03741663926955) bank186203 +186204 POINT(39.862025065691185 74.89976369739766) bank186204 +186205 POINT(39.99332542189101 73.90331973174384) bank186205 +186206 POINT(40.847131598675944 74.89706776789485) bank186206 +186207 POINT(41.54499449056376 73.70696273231364) bank186207 +186208 POINT(39.93718730112812 73.41078606715396) bank186208 +186209 POINT(40.59240604627828 73.2734857889132) bank186209 +186210 POINT(39.826912590952155 74.33919024717304) bank186210 +186211 POINT(39.96491693772339 73.41870291703378) bank186211 +186212 POINT(41.64128095185582 73.85175879249273) bank186212 +186213 POINT(39.74096925310516 74.67729189706844) bank186213 +186214 POINT(41.31968877337722 74.58157292901983) bank186214 +186215 POINT(41.586654094815906 73.61704338549842) bank186215 +186216 POINT(40.48135710159457 74.45781573387949) bank186216 +186217 POINT(40.76799667586172 74.74343636229197) bank186217 +186218 POINT(41.07429559052784 74.98379221156267) bank186218 +186219 POINT(40.03577716917598 74.36045866747682) bank186219 +186220 POINT(40.13278244844534 73.69438190892325) bank186220 +186221 POINT(40.51646324597091 74.58135491464495) bank186221 +186222 POINT(41.207717560848586 74.84891131490471) bank186222 +186223 POINT(39.76329303460646 74.31297459016936) bank186223 +186224 POINT(39.790737477901956 74.44880474889001) bank186224 +186225 POINT(40.55745880250591 73.172092383969) bank186225 +186226 POINT(39.948468074624415 73.49994364094333) bank186226 +186227 POINT(40.6168774952183 73.34245913307358) bank186227 +186228 POINT(41.60392561974419 73.73060048450363) bank186228 +186229 POINT(40.13075593620086 74.52259075611376) bank186229 +186230 POINT(40.88309394856452 74.29802087874576) bank186230 +186231 POINT(40.182617291073576 73.03504022624574) bank186231 +186232 POINT(40.25889643434457 74.02847005061454) bank186232 +186233 POINT(40.076907833466905 73.62449549266574) bank186233 +186234 POINT(40.925653675324476 73.46517563228666) bank186234 +186235 POINT(40.65505531752975 74.03997449961076) bank186235 +186236 POINT(41.70914411322692 73.32932760006649) bank186236 +186237 POINT(40.30401715664824 74.16403461224446) bank186237 +186238 POINT(41.25125716900115 73.7988003812059) bank186238 +186239 POINT(40.674898579570275 74.32876982273152) bank186239 +186240 POINT(41.54281171023859 74.90303546144821) bank186240 +186241 POINT(41.44116095259796 73.19490062636238) bank186241 +186242 POINT(41.68934222548858 73.90134918441782) bank186242 +186243 POINT(40.50927909724054 74.26328376162817) bank186243 +186244 POINT(40.17196864628302 74.53176245452241) bank186244 +186245 POINT(40.13730649239902 74.27724394817774) bank186245 +186246 POINT(40.46997580429172 73.21776031520264) bank186246 +186247 POINT(40.6731144019785 73.72333304269522) bank186247 +186248 POINT(39.774279445154725 73.59583014578365) bank186248 +186249 POINT(41.62501626389289 73.25524736179379) bank186249 +186250 POINT(39.860218925186494 73.13031985656436) bank186250 +186251 POINT(40.86916576274991 73.02118640697799) bank186251 +186252 POINT(39.94642623159294 73.37486489378622) bank186252 +186253 POINT(39.850041576732146 74.68249852770361) bank186253 +186254 POINT(41.05238803616789 73.57076594805802) bank186254 +186255 POINT(41.04288533557286 73.70949640280583) bank186255 +186256 POINT(41.664344320821876 74.94513463796959) bank186256 +186257 POINT(40.33160423665453 73.47028014063999) bank186257 +186258 POINT(40.862191451583534 73.17705338081839) bank186258 +186259 POINT(40.2976716760912 73.33457613006505) bank186259 +186260 POINT(39.83284228893065 74.06554425590684) bank186260 +186261 POINT(41.524390950670295 73.62147816220563) bank186261 +186262 POINT(40.18458735997585 73.54401599792081) bank186262 +186263 POINT(41.22758647022393 73.14019089248166) bank186263 +186264 POINT(40.305747695461505 73.18265804187773) bank186264 +186265 POINT(40.64828400319334 74.54166497199053) bank186265 +186266 POINT(39.905334006293444 74.44108994177245) bank186266 +186267 POINT(40.05399706389775 74.2971195404669) bank186267 +186268 POINT(40.657236305763334 74.57028450282257) bank186268 +186269 POINT(40.76756764393839 73.80111057974722) bank186269 +186270 POINT(41.695398072301835 74.39818893337262) bank186270 +186271 POINT(40.59214280427043 73.8240395223956) bank186271 +186272 POINT(40.97083263959937 74.19801964188201) bank186272 +186273 POINT(40.158005375486105 73.65255472749821) bank186273 +186274 POINT(41.152284374266436 73.8611247236138) bank186274 +186275 POINT(40.44931931173028 74.58992512974264) bank186275 +186276 POINT(40.996861042566955 73.99542807972638) bank186276 +186277 POINT(40.59628072556243 73.78300241694308) bank186277 +186278 POINT(39.85061188155836 73.19853407918657) bank186278 +186279 POINT(39.96349175691033 74.19855201310445) bank186279 +186280 POINT(39.931167099726835 74.00042499123535) bank186280 +186281 POINT(40.64997729079023 74.82504733574702) bank186281 +186282 POINT(41.02302149240821 74.69847372336075) bank186282 +186283 POINT(39.973702779359115 74.82823819675768) bank186283 +186284 POINT(40.6375723289501 73.47398844317597) bank186284 +186285 POINT(39.78093773306323 74.90796270839422) bank186285 +186286 POINT(41.021509335758864 73.72695342735034) bank186286 +186287 POINT(40.040143299720874 73.51872833860658) bank186287 +186288 POINT(40.372834977536485 74.15188473283419) bank186288 +186289 POINT(40.64526065106085 74.13312365688745) bank186289 +186290 POINT(41.037128112282666 73.29685029987102) bank186290 +186291 POINT(39.827705801340684 73.4476515752217) bank186291 +186292 POINT(40.848159876084274 73.65370735702467) bank186292 +186293 POINT(40.843691088406025 74.23438958515239) bank186293 +186294 POINT(40.92740727179019 73.9117380602274) bank186294 +186295 POINT(40.79679389642578 74.24059302358395) bank186295 +186296 POINT(40.09933789046252 74.1516882685233) bank186296 +186297 POINT(40.496276617047386 73.2065120350473) bank186297 +186298 POINT(40.52878512626591 74.9165925492158) bank186298 +186299 POINT(39.9632364649813 74.04644019913947) bank186299 +186300 POINT(41.0221939918547 74.50904807439021) bank186300 +186301 POINT(40.72936337051693 74.0368926593375) bank186301 +186302 POINT(41.4768413192123 73.49773256017447) bank186302 +186303 POINT(40.13418693696357 73.01624412424829) bank186303 +186304 POINT(40.883299592465946 73.1056544739615) bank186304 +186305 POINT(41.396651437646845 73.23081520403939) bank186305 +186306 POINT(40.13254371942954 73.63858533834855) bank186306 +186307 POINT(40.796817224337495 73.83792418252105) bank186307 +186308 POINT(41.502456341084944 74.095532492801) bank186308 +186309 POINT(40.985183652100254 73.50543305755548) bank186309 +186310 POINT(40.986215195044295 73.73584877543722) bank186310 +186311 POINT(40.60268477741636 73.07767292554728) bank186311 +186312 POINT(40.37018458720035 74.13795143959364) bank186312 +186313 POINT(41.534283745595 73.97107956077271) bank186313 +186314 POINT(40.69339933934712 74.82204879869724) bank186314 +186315 POINT(40.59430582651441 74.73929755508713) bank186315 +186316 POINT(40.7444534492988 74.97114651905122) bank186316 +186317 POINT(39.905909981099484 73.37245346931255) bank186317 +186318 POINT(40.50564956078231 74.69667111163577) bank186318 +186319 POINT(40.207151066707695 74.95486954492867) bank186319 +186320 POINT(41.22506275644272 74.79568174641177) bank186320 +186321 POINT(39.71292841179434 73.47673785236279) bank186321 +186322 POINT(41.64561082336557 73.67763140074126) bank186322 +186323 POINT(40.43393448268681 74.0307325905568) bank186323 +186324 POINT(41.701216658955715 73.05112305163614) bank186324 +186325 POINT(41.095597930351715 73.64814158231708) bank186325 +186326 POINT(40.10802959019082 74.56916153660048) bank186326 +186327 POINT(40.23147529855963 73.63071196192799) bank186327 +186328 POINT(40.10416547877666 74.43556151170318) bank186328 +186329 POINT(41.64146686511522 73.10227127913218) bank186329 +186330 POINT(41.08985270433006 73.046024531496) bank186330 +186331 POINT(41.47291203590366 73.93338394712723) bank186331 +186332 POINT(41.20309532484098 73.13742291548506) bank186332 +186333 POINT(39.917377536428006 73.34833276669355) bank186333 +186334 POINT(40.50798180184648 73.89497558054589) bank186334 +186335 POINT(40.20533434644244 73.85270134820384) bank186335 +186336 POINT(39.97185999731967 73.13342677807515) bank186336 +186337 POINT(40.050021897266014 74.55123679534275) bank186337 +186338 POINT(41.19093423398115 74.36482035700745) bank186338 +186339 POINT(40.858643032493 73.58857551097798) bank186339 +186340 POINT(40.90124103304558 74.83623068432722) bank186340 +186341 POINT(40.18252104197911 73.05736371396223) bank186341 +186342 POINT(39.878741080042566 74.40348940513029) bank186342 +186343 POINT(40.674779700175336 74.80828045412699) bank186343 +186344 POINT(40.18186038037686 73.04350468887911) bank186344 +186345 POINT(41.41651027482426 73.58031839810141) bank186345 +186346 POINT(40.76488760458621 74.36162746076359) bank186346 +186347 POINT(41.358834252273965 74.55669966388699) bank186347 +186348 POINT(40.35515200170931 74.32479725639924) bank186348 +186349 POINT(40.40834352979157 73.36930941870172) bank186349 +186350 POINT(40.12436813248725 73.80693862449041) bank186350 +186351 POINT(40.431963697838725 73.97887380171593) bank186351 +186352 POINT(41.200549169650586 73.86484587493803) bank186352 +186353 POINT(40.3285016509189 74.33109624336993) bank186353 +186354 POINT(41.65475255711291 74.58036026293799) bank186354 +186355 POINT(41.68956602631234 73.10569338468811) bank186355 +186356 POINT(40.97375367094667 74.67197064578487) bank186356 +186357 POINT(40.20853781556592 74.4271657304584) bank186357 +186358 POINT(39.96644711110817 73.0879611433117) bank186358 +186359 POINT(40.10198539832312 74.68869196339402) bank186359 +186360 POINT(41.68663932080733 73.11140505693406) bank186360 +186361 POINT(40.621420479723675 74.61030235685836) bank186361 +186362 POINT(40.484976053658094 73.32152253149877) bank186362 +186363 POINT(40.123763360877824 74.7143089173344) bank186363 +186364 POINT(40.46301894333332 73.17491064558264) bank186364 +186365 POINT(40.64504277307941 73.97498734747266) bank186365 +186366 POINT(41.620090527648856 74.5922520257865) bank186366 +186367 POINT(41.694359839985395 74.47712196441572) bank186367 +186368 POINT(40.54652767673205 74.57110930029468) bank186368 +186369 POINT(40.91712787425349 74.34812571893605) bank186369 +186370 POINT(39.858235809928104 74.53442079265236) bank186370 +186371 POINT(40.43756742577901 74.52715524611281) bank186371 +186372 POINT(40.34978862045642 73.78495585550036) bank186372 +186373 POINT(41.63977931825927 74.72597584977932) bank186373 +186374 POINT(40.922967823927145 73.11956304736782) bank186374 +186375 POINT(40.14960395218349 74.0817153070455) bank186375 +186376 POINT(41.58353887438663 73.72230934535345) bank186376 +186377 POINT(41.21687475100339 74.1797450838758) bank186377 +186378 POINT(40.4465889278773 74.18008529463123) bank186378 +186379 POINT(40.038944638495316 73.69522843502806) bank186379 +186380 POINT(41.660523371376264 74.77950374627332) bank186380 +186381 POINT(39.971513617076 74.13264842960969) bank186381 +186382 POINT(41.383236240500665 74.96163946969739) bank186382 +186383 POINT(40.01149778412178 73.9738515946827) bank186383 +186384 POINT(40.18402132193443 73.7741562338644) bank186384 +186385 POINT(41.61517688744393 73.92749689589208) bank186385 +186386 POINT(40.20148906334646 74.982001675585) bank186386 +186387 POINT(41.23208089942157 73.85333738348658) bank186387 +186388 POINT(40.77586469588715 73.98800924521991) bank186388 +186389 POINT(40.31842891698221 73.2499227895929) bank186389 +186390 POINT(39.99617004156842 74.93556120862358) bank186390 +186391 POINT(40.86793915239722 74.94042973879945) bank186391 +186392 POINT(39.870185453696195 73.39722053190603) bank186392 +186393 POINT(40.50444110642573 74.59770255043759) bank186393 +186394 POINT(40.861319518253 73.23490131007486) bank186394 +186395 POINT(39.731538316402286 74.71709421972437) bank186395 +186396 POINT(41.399464098956905 74.4940808756593) bank186396 +186397 POINT(41.66597789415565 73.48003859094344) bank186397 +186398 POINT(41.6194205595898 74.44435821761837) bank186398 +186399 POINT(40.26545332584158 73.2186462443456) bank186399 +186400 POINT(40.73804082874577 74.3816189212846) bank186400 +186401 POINT(40.6681256789355 74.83463554090136) bank186401 +186402 POINT(41.517524580685794 73.21654063596777) bank186402 +186403 POINT(41.212331730781564 74.18414426332109) bank186403 +186404 POINT(40.43995762121588 73.32673598710593) bank186404 +186405 POINT(41.566320662731336 74.32815758186348) bank186405 +186406 POINT(39.732548272025994 74.68517791543175) bank186406 +186407 POINT(39.75595670464298 73.70646787823061) bank186407 +186408 POINT(39.89263854062682 73.64322988148506) bank186408 +186409 POINT(40.29671272088985 73.52721079378725) bank186409 +186410 POINT(40.57346199183933 74.67898721663502) bank186410 +186411 POINT(40.63853638181015 74.98480673152001) bank186411 +186412 POINT(40.95657251346033 73.96702732195801) bank186412 +186413 POINT(41.242556527502394 74.61537564828811) bank186413 +186414 POINT(40.5274959808022 74.43916508996364) bank186414 +186415 POINT(40.30091256411236 73.45870070186842) bank186415 +186416 POINT(41.68271040160702 73.91800498006896) bank186416 +186417 POINT(40.399695676612666 74.03351301987716) bank186417 +186418 POINT(41.08471418219526 74.82330116043526) bank186418 +186419 POINT(40.347778767557514 74.9770194478153) bank186419 +186420 POINT(41.481234756259404 73.02876903881224) bank186420 +186421 POINT(41.55231767564201 74.58805273124396) bank186421 +186422 POINT(39.96865695797352 74.98288849049423) bank186422 +186423 POINT(41.473801062003275 73.61782937366247) bank186423 +186424 POINT(41.60458017674229 74.28350857019902) bank186424 +186425 POINT(39.949677811448495 73.44592645644111) bank186425 +186426 POINT(39.91757099693059 73.75491436475659) bank186426 +186427 POINT(40.90479423593305 74.4345742294431) bank186427 +186428 POINT(41.480878380163986 73.83307929488964) bank186428 +186429 POINT(39.756939267784865 74.48144282757782) bank186429 +186430 POINT(41.5147637525526 73.57477562735689) bank186430 +186431 POINT(40.933388437029535 73.31207568181276) bank186431 +186432 POINT(40.72483432392427 74.42608042832164) bank186432 +186433 POINT(40.92143640513321 74.14462342640829) bank186433 +186434 POINT(40.63760364283023 74.32436392566868) bank186434 +186435 POINT(41.37433451762008 74.44201949834674) bank186435 +186436 POINT(40.037661712036645 74.62276718431292) bank186436 +186437 POINT(41.1887652939886 73.85327926035215) bank186437 +186438 POINT(40.72127242002156 74.92682619914098) bank186438 +186439 POINT(40.2111331742218 73.94184780088992) bank186439 +186440 POINT(40.926847698727805 73.17515787556088) bank186440 +186441 POINT(40.89263758925641 74.25282613000682) bank186441 +186442 POINT(41.02944492051129 73.65772430107609) bank186442 +186443 POINT(41.599183381481836 74.95157698727259) bank186443 +186444 POINT(40.416536626857095 73.65143254023455) bank186444 +186445 POINT(40.59821467112593 74.75384451274894) bank186445 +186446 POINT(40.410236954467884 73.86872153522003) bank186446 +186447 POINT(40.18366652856399 73.77575361716181) bank186447 +186448 POINT(40.339920502546384 74.27763954804024) bank186448 +186449 POINT(40.95320322230029 74.28811587737462) bank186449 +186450 POINT(40.17135305376383 74.2849158998771) bank186450 +186451 POINT(40.46279946741664 74.02683485005666) bank186451 +186452 POINT(40.84527674571123 73.79868926451881) bank186452 +186453 POINT(39.75340211421682 74.32892954221974) bank186453 +186454 POINT(40.472231060425266 73.80483711963232) bank186454 +186455 POINT(41.39323740535425 73.9947416594574) bank186455 +186456 POINT(41.67710858079876 74.73119242187305) bank186456 +186457 POINT(39.74593617915109 73.10964130120145) bank186457 +186458 POINT(41.566884128887544 73.1133675301082) bank186458 +186459 POINT(40.14252881258618 73.27415224656465) bank186459 +186460 POINT(41.01263674682142 74.07118613286636) bank186460 +186461 POINT(39.89204296545353 74.71354025671064) bank186461 +186462 POINT(41.162080390729855 73.63225307174923) bank186462 +186463 POINT(40.7493154330554 74.58512889514031) bank186463 +186464 POINT(41.70911079150434 74.2083051358992) bank186464 +186465 POINT(41.234952427427366 74.77335799550465) bank186465 +186466 POINT(41.01730648531567 73.48368461052374) bank186466 +186467 POINT(40.2080942081331 74.21090389665565) bank186467 +186468 POINT(40.25667101298939 73.14693897357279) bank186468 +186469 POINT(40.25094069315673 73.82099355010074) bank186469 +186470 POINT(40.309661539361784 74.81755354434205) bank186470 +186471 POINT(41.26875167451514 74.49535280179113) bank186471 +186472 POINT(41.28971752996568 74.88987946257248) bank186472 +186473 POINT(40.83092723374165 74.98951353987816) bank186473 +186474 POINT(41.483830221834985 73.52366312974665) bank186474 +186475 POINT(41.24371430173341 74.92131993499447) bank186475 +186476 POINT(41.45704005194875 74.15346802588743) bank186476 +186477 POINT(40.995697063400186 73.78864812271803) bank186477 +186478 POINT(40.36335524256829 74.77839514903668) bank186478 +186479 POINT(40.00622102706992 73.95338414119287) bank186479 +186480 POINT(39.83301968177194 74.33525033110034) bank186480 +186481 POINT(39.97729492849279 74.1485937703665) bank186481 +186482 POINT(40.45713386613833 73.31047993620412) bank186482 +186483 POINT(40.95601521724073 73.8553654646773) bank186483 +186484 POINT(39.98935549036916 74.7249786621831) bank186484 +186485 POINT(40.82431319565156 73.8834497883629) bank186485 +186486 POINT(41.45921028695513 74.92124546698942) bank186486 +186487 POINT(41.57064840646387 73.7506216402382) bank186487 +186488 POINT(40.79066816060252 74.2855622607601) bank186488 +186489 POINT(39.919878877719654 73.7024173747541) bank186489 +186490 POINT(41.16924364340342 73.09318983133474) bank186490 +186491 POINT(40.765395133863535 73.99475522932046) bank186491 +186492 POINT(41.355898659514644 74.44764266709188) bank186492 +186493 POINT(40.14449119968615 74.53317352419381) bank186493 +186494 POINT(40.2629560598903 74.4114994240287) bank186494 +186495 POINT(41.09747516602491 73.77944149469702) bank186495 +186496 POINT(40.01668766655441 73.14042068558201) bank186496 +186497 POINT(40.93432387556323 74.66312751666467) bank186497 +186498 POINT(40.00982787404074 73.76560459686961) bank186498 +186499 POINT(40.50846108169101 73.47231067156642) bank186499 +186500 POINT(39.80496693291374 73.87854699185748) bank186500 +186501 POINT(40.8630403532003 73.73938840272083) bank186501 +186502 POINT(39.82764254098608 74.98082222339836) bank186502 +186503 POINT(40.175922945268425 74.89989278895932) bank186503 +186504 POINT(40.16066052626477 73.28854607021097) bank186504 +186505 POINT(40.95669886741752 74.34531135007772) bank186505 +186506 POINT(41.24604701757753 73.69089714337952) bank186506 +186507 POINT(41.48477450887108 74.01741710584693) bank186507 +186508 POINT(41.0380161954665 73.90017289613802) bank186508 +186509 POINT(40.37992918141214 73.46258638012407) bank186509 +186510 POINT(40.68320220527223 73.40197264102994) bank186510 +186511 POINT(39.89972819510156 73.3910738547356) bank186511 +186512 POINT(41.00660933737384 73.5543977331546) bank186512 +186513 POINT(40.79147412490874 74.63911751231527) bank186513 +186514 POINT(41.33400377250714 73.59972499126391) bank186514 +186515 POINT(40.237329215830165 73.08992817864308) bank186515 +186516 POINT(40.76038841238636 74.22003464321064) bank186516 +186517 POINT(40.35407375594173 74.22336243463258) bank186517 +186518 POINT(40.88793067277379 73.29813974926907) bank186518 +186519 POINT(40.58256965587376 74.22845071357135) bank186519 +186520 POINT(41.33334820962917 73.5076729873457) bank186520 +186521 POINT(40.705593503335805 74.30611084323102) bank186521 +186522 POINT(40.35766032268092 74.58829295895889) bank186522 +186523 POINT(39.81222990580699 73.22936503202769) bank186523 +186524 POINT(40.862756553206886 73.79212826007246) bank186524 +186525 POINT(40.237039107957315 73.06110609846134) bank186525 +186526 POINT(41.46094109751623 73.14630432850531) bank186526 +186527 POINT(39.910166218363216 73.91634519724609) bank186527 +186528 POINT(41.390630061726846 73.42095453003917) bank186528 +186529 POINT(39.86359617285707 74.66361740458716) bank186529 +186530 POINT(41.27349901946134 74.99175075761835) bank186530 +186531 POINT(41.09049995596676 73.5089040895833) bank186531 +186532 POINT(41.29644713875105 74.43727785024282) bank186532 +186533 POINT(40.11708049682066 73.8445846491063) bank186533 +186534 POINT(40.07649816525162 74.6041972889832) bank186534 +186535 POINT(40.16749353874209 73.80276997176419) bank186535 +186536 POINT(40.019516757087686 74.21792522766707) bank186536 +186537 POINT(41.21069926894232 74.7266884615682) bank186537 +186538 POINT(39.75829033041771 73.37948722520795) bank186538 +186539 POINT(40.45575180504513 73.62562073203394) bank186539 +186540 POINT(40.82765767679896 74.5758234704437) bank186540 +186541 POINT(39.89596741146673 74.2503208457591) bank186541 +186542 POINT(39.91932818888454 73.5447690721864) bank186542 +186543 POINT(40.18229862955893 74.44261314658692) bank186543 +186544 POINT(40.69140508025776 74.71049025523207) bank186544 +186545 POINT(40.432208133445464 73.73930999368919) bank186545 +186546 POINT(40.371662049877465 74.52256780055396) bank186546 +186547 POINT(40.74364456506647 73.55190236730328) bank186547 +186548 POINT(40.681449923987245 73.42801379880606) bank186548 +186549 POINT(41.42063205312546 73.17203415974583) bank186549 +186550 POINT(39.84157920024792 74.39684347801068) bank186550 +186551 POINT(40.66715408775825 74.83094045089273) bank186551 +186552 POINT(40.032877746005276 74.77897598148647) bank186552 +186553 POINT(41.32593682793771 73.93091291924753) bank186553 +186554 POINT(41.39421793934421 73.69113628678602) bank186554 +186555 POINT(41.64637154459546 74.08779504628951) bank186555 +186556 POINT(40.03267441352191 73.31679780529856) bank186556 +186557 POINT(40.92937252533082 74.08520651517688) bank186557 +186558 POINT(41.62474361702086 73.9714766617404) bank186558 +186559 POINT(40.5396593878076 73.44584078960311) bank186559 +186560 POINT(40.77531455813067 73.58677716112365) bank186560 +186561 POINT(41.1697317419268 73.84773010824668) bank186561 +186562 POINT(41.23763236285613 74.09222531892256) bank186562 +186563 POINT(41.437470024665316 74.30820522626843) bank186563 +186564 POINT(41.07607003542919 73.92130998504263) bank186564 +186565 POINT(39.91742650291249 74.2449531746076) bank186565 +186566 POINT(40.25952602853418 73.12765617667193) bank186566 +186567 POINT(41.0434451106373 74.02404938469395) bank186567 +186568 POINT(40.303731979810145 74.38191522956438) bank186568 +186569 POINT(41.68135945599013 73.67362119278435) bank186569 +186570 POINT(41.39957711919773 73.63010968238812) bank186570 +186571 POINT(40.1786168683507 73.3426544235124) bank186571 +186572 POINT(41.307260123792304 73.33811114046486) bank186572 +186573 POINT(41.374483787496956 74.16385905538982) bank186573 +186574 POINT(41.35657218594552 73.10696184495193) bank186574 +186575 POINT(40.06726417347266 74.35187551538355) bank186575 +186576 POINT(41.46681619381082 73.888362529609) bank186576 +186577 POINT(40.91113551438762 73.77840097146448) bank186577 +186578 POINT(41.63861777058535 74.67601910361279) bank186578 +186579 POINT(40.099541078602684 73.27793274650149) bank186579 +186580 POINT(40.23804615570168 74.82347685150957) bank186580 +186581 POINT(40.519249033287686 74.55521230994928) bank186581 +186582 POINT(40.25145067684601 74.18133587707194) bank186582 +186583 POINT(39.763368208074176 73.15169591420141) bank186583 +186584 POINT(40.59960593916447 74.53331148078531) bank186584 +186585 POINT(41.211328600442876 74.38156338169048) bank186585 +186586 POINT(40.218244834801105 74.17208758522011) bank186586 +186587 POINT(40.46960439294566 73.52767289294056) bank186587 +186588 POINT(40.99972821993715 74.97752957296676) bank186588 +186589 POINT(40.81177017589729 74.72814911233542) bank186589 +186590 POINT(41.43908947444592 74.92832099816621) bank186590 +186591 POINT(41.60960094274457 74.97166896248524) bank186591 +186592 POINT(39.98764698034187 74.82830200754954) bank186592 +186593 POINT(41.408918148005625 73.43898396052896) bank186593 +186594 POINT(41.29520910425939 74.17241314485787) bank186594 +186595 POINT(41.031611425641856 74.65059998957724) bank186595 +186596 POINT(40.42747248090672 74.32748012098264) bank186596 +186597 POINT(40.250669541325756 74.64045952756233) bank186597 +186598 POINT(41.62475532330434 73.33960934411688) bank186598 +186599 POINT(40.25837999056015 74.28225394350906) bank186599 +186600 POINT(39.825774824504315 73.26692332800015) bank186600 +186601 POINT(39.8785382751933 74.57653374927459) bank186601 +186602 POINT(40.91316319805229 73.67643390469176) bank186602 +186603 POINT(41.46579996359692 74.72711324806022) bank186603 +186604 POINT(41.416356799622505 73.47288282387093) bank186604 +186605 POINT(39.86883505195607 74.65614810886085) bank186605 +186606 POINT(40.567917940361056 74.70577673099382) bank186606 +186607 POINT(41.5489940802195 73.68143271771196) bank186607 +186608 POINT(40.492222980192714 74.76794451046187) bank186608 +186609 POINT(41.693803689036606 74.27436024417896) bank186609 +186610 POINT(41.63121385745769 74.74158634003582) bank186610 +186611 POINT(40.399154374734394 73.18156055244262) bank186611 +186612 POINT(40.760292388472244 73.19965105673361) bank186612 +186613 POINT(41.35675117923627 74.26214407343225) bank186613 +186614 POINT(40.70084054282077 73.86207680808955) bank186614 +186615 POINT(41.156937074148814 74.67624097541812) bank186615 +186616 POINT(40.31154314419394 73.50233301956972) bank186616 +186617 POINT(41.70686211125683 73.17672902223484) bank186617 +186618 POINT(40.45673792871476 74.59060578113903) bank186618 +186619 POINT(40.620413430582474 74.58226878086238) bank186619 +186620 POINT(41.548370767961785 73.01401795167253) bank186620 +186621 POINT(40.07858162176901 73.72253862418364) bank186621 +186622 POINT(41.011439940086525 73.43913901121604) bank186622 +186623 POINT(41.13706766463368 74.76676734865342) bank186623 +186624 POINT(41.13698595475007 73.73951656454535) bank186624 +186625 POINT(41.65225570238211 74.88895826861193) bank186625 +186626 POINT(40.40479890191758 74.80079073047689) bank186626 +186627 POINT(40.255180336104324 73.29505246120996) bank186627 +186628 POINT(40.25747775269218 74.9666709523994) bank186628 +186629 POINT(40.44237168914864 73.49695920016583) bank186629 +186630 POINT(39.950165302033945 74.85166663852749) bank186630 +186631 POINT(40.311894180109114 73.66626782300378) bank186631 +186632 POINT(41.01258678287085 74.53730968041661) bank186632 +186633 POINT(39.95731161902402 73.4774488198971) bank186633 +186634 POINT(40.605075925113184 73.5983411082066) bank186634 +186635 POINT(40.84366272286201 73.2841463480012) bank186635 +186636 POINT(41.37483775657998 73.35889599898945) bank186636 +186637 POINT(40.387104398830125 73.17330550025993) bank186637 +186638 POINT(41.12973226032306 74.21038578241527) bank186638 +186639 POINT(41.45663991612507 74.74590852536137) bank186639 +186640 POINT(40.00602065834344 73.1497145708079) bank186640 +186641 POINT(40.44378824602818 74.03713459585755) bank186641 +186642 POINT(39.90020955621818 73.29676594284399) bank186642 +186643 POINT(40.85688044685191 73.82145952502921) bank186643 +186644 POINT(40.60314756835797 74.77447305217272) bank186644 +186645 POINT(40.688015776301405 73.38581096966145) bank186645 +186646 POINT(40.67762250720366 73.778849414188) bank186646 +186647 POINT(41.47617982670614 73.50727462704445) bank186647 +186648 POINT(40.50507189016765 73.53499680177973) bank186648 +186649 POINT(40.17289975782595 74.5340207944942) bank186649 +186650 POINT(39.770503368424166 73.1110489924149) bank186650 +186651 POINT(41.48070427462184 74.43942276072154) bank186651 +186652 POINT(40.49236427355755 74.87616521122806) bank186652 +186653 POINT(39.995946551029625 74.05016754563198) bank186653 +186654 POINT(41.032892420661945 73.60361386106615) bank186654 +186655 POINT(40.02686562422257 73.45121411958239) bank186655 +186656 POINT(39.77388887504821 73.24011019562803) bank186656 +186657 POINT(40.13981089795722 74.7484595735266) bank186657 +186658 POINT(41.4942463361093 73.94092471524097) bank186658 +186659 POINT(40.188068176017225 74.64138651613658) bank186659 +186660 POINT(40.97563956012385 73.30067484127194) bank186660 +186661 POINT(41.53945976405025 74.66994203636834) bank186661 +186662 POINT(41.195273310521465 74.40097658058957) bank186662 +186663 POINT(40.016251624017265 73.61505110940536) bank186663 +186664 POINT(41.448335774458386 73.66486688671253) bank186664 +186665 POINT(41.139482112582066 74.30551759597786) bank186665 +186666 POINT(41.471302840073 74.27144136278056) bank186666 +186667 POINT(41.214061302733874 73.23353873919798) bank186667 +186668 POINT(40.065770487092564 73.61924756130334) bank186668 +186669 POINT(39.92093350705597 73.01238125255304) bank186669 +186670 POINT(39.869150877023934 73.79205530153851) bank186670 +186671 POINT(40.971349643760476 74.72470097091606) bank186671 +186672 POINT(40.58678832392587 73.11482382270208) bank186672 +186673 POINT(41.018170878859834 74.60649558765162) bank186673 +186674 POINT(41.680790515126574 73.30772099075043) bank186674 +186675 POINT(40.53363006443419 73.6491388485411) bank186675 +186676 POINT(40.115890596149306 74.57989883250606) bank186676 +186677 POINT(41.05860596872704 73.66321898845285) bank186677 +186678 POINT(39.87906490218853 74.30774112445539) bank186678 +186679 POINT(40.61005752371411 73.78444639395396) bank186679 +186680 POINT(40.690965703039375 73.5007868371518) bank186680 +186681 POINT(39.91816430939143 74.56714854640981) bank186681 +186682 POINT(40.327326878615935 74.95801352836841) bank186682 +186683 POINT(40.80077206842577 74.01284014874363) bank186683 +186684 POINT(41.116465047565775 73.63471387835479) bank186684 +186685 POINT(40.91456951624593 73.49717280830077) bank186685 +186686 POINT(40.01449729763817 74.6838198429327) bank186686 +186687 POINT(40.00251148623306 73.88475443647134) bank186687 +186688 POINT(39.71426671572878 73.18698099059874) bank186688 +186689 POINT(40.15859593766348 73.61000517481347) bank186689 +186690 POINT(40.58329877505614 73.9513020110181) bank186690 +186691 POINT(39.86195884805665 73.15625092470086) bank186691 +186692 POINT(41.013554779996824 74.29658660509595) bank186692 +186693 POINT(41.202716284421236 74.96607596031843) bank186693 +186694 POINT(40.987298812013634 74.08373371120163) bank186694 +186695 POINT(40.420197679857544 74.16043009274675) bank186695 +186696 POINT(39.955633212663024 74.04621846815256) bank186696 +186697 POINT(40.301959427526555 73.5391142556522) bank186697 +186698 POINT(39.83258601122467 74.26335220800183) bank186698 +186699 POINT(41.34017506895498 73.25436013797626) bank186699 +186700 POINT(40.509541962783366 74.88760873351323) bank186700 +186701 POINT(40.661995998833994 73.16532119729499) bank186701 +186702 POINT(40.72307668326582 73.40513329007551) bank186702 +186703 POINT(40.40182112681239 74.00165843415617) bank186703 +186704 POINT(40.93957320068306 73.30926806504962) bank186704 +186705 POINT(41.630897002647 74.26802865754011) bank186705 +186706 POINT(41.399257244350245 73.19602774306067) bank186706 +186707 POINT(39.9254090795946 73.58351461330003) bank186707 +186708 POINT(39.98622785936894 74.57616441294921) bank186708 +186709 POINT(40.49989125426658 74.82669014695402) bank186709 +186710 POINT(40.33163484725235 74.47418590289973) bank186710 +186711 POINT(41.543988109180106 74.6668880722856) bank186711 +186712 POINT(40.19556271025081 74.90269786286034) bank186712 +186713 POINT(41.07369940263564 73.47015372287066) bank186713 +186714 POINT(41.27710639080826 73.63091825767317) bank186714 +186715 POINT(41.10411920479362 73.11463525892462) bank186715 +186716 POINT(39.87805947422594 73.03694218430115) bank186716 +186717 POINT(41.61329408922542 73.1374706733804) bank186717 +186718 POINT(39.75764747993896 73.54478010342102) bank186718 +186719 POINT(40.46933415862002 74.88829701163127) bank186719 +186720 POINT(39.77720266830799 73.97309841114235) bank186720 +186721 POINT(41.52934328154866 74.54878655876567) bank186721 +186722 POINT(41.55646521254455 74.5144677376724) bank186722 +186723 POINT(40.569817486163586 73.19792817984947) bank186723 +186724 POINT(40.616148730734174 74.3403390852607) bank186724 +186725 POINT(41.279637941699974 74.16452408076071) bank186725 +186726 POINT(40.07020581215274 73.78048474511127) bank186726 +186727 POINT(41.22990848818888 73.4215532999384) bank186727 +186728 POINT(40.6877105780479 73.51197562920102) bank186728 +186729 POINT(40.75598462924368 73.9900052890763) bank186729 +186730 POINT(39.77073027602481 74.89018872724317) bank186730 +186731 POINT(40.13274334762405 73.30374772561927) bank186731 +186732 POINT(40.28578919594763 74.79583503869742) bank186732 +186733 POINT(41.52761296749834 73.06221225259996) bank186733 +186734 POINT(41.5352614518159 74.2076558662544) bank186734 +186735 POINT(40.89167853276392 74.28173188209388) bank186735 +186736 POINT(41.325622492083866 73.08268657803448) bank186736 +186737 POINT(41.221128958184906 73.60456906357257) bank186737 +186738 POINT(41.02133095984774 74.10207887349893) bank186738 +186739 POINT(40.67485384538122 73.78090732061901) bank186739 +186740 POINT(39.816861206137446 73.42318023640203) bank186740 +186741 POINT(40.93831800878771 74.6964228658024) bank186741 +186742 POINT(40.44288290835656 73.38568063038919) bank186742 +186743 POINT(39.937821691472585 73.20917762613307) bank186743 +186744 POINT(39.8204645013513 74.91381783207511) bank186744 +186745 POINT(41.67263493908577 73.82991758075993) bank186745 +186746 POINT(40.11598496355449 74.19142001043059) bank186746 +186747 POINT(41.40258415574092 73.82514686260416) bank186747 +186748 POINT(40.42367021230036 73.39732401861711) bank186748 +186749 POINT(41.33787859200093 73.78801013319972) bank186749 +186750 POINT(40.63297609751138 74.20953371271348) bank186750 +186751 POINT(40.99180958399783 74.56643707517316) bank186751 +186752 POINT(40.45338133703956 74.70811325928675) bank186752 +186753 POINT(40.96701782234296 74.09718811496342) bank186753 +186754 POINT(41.37822652317457 74.4865755425569) bank186754 +186755 POINT(40.462317643763534 74.27538260950963) bank186755 +186756 POINT(41.35592954971453 73.1830250602594) bank186756 +186757 POINT(40.83036284687059 74.83751812838554) bank186757 +186758 POINT(41.63273644371674 73.06735814184432) bank186758 +186759 POINT(39.96713132488705 74.47755575307987) bank186759 +186760 POINT(40.48783755132122 73.41194239310933) bank186760 +186761 POINT(39.77029724696924 74.52543132874014) bank186761 +186762 POINT(41.225583099819005 73.19461120767018) bank186762 +186763 POINT(40.20899158511468 74.9890859053526) bank186763 +186764 POINT(40.98586838917156 73.04805553469629) bank186764 +186765 POINT(40.067054290735705 74.154253030909) bank186765 +186766 POINT(40.320996413279374 74.90395336632734) bank186766 +186767 POINT(41.06357054661506 74.97120493901731) bank186767 +186768 POINT(41.27057650952038 74.59873895216339) bank186768 +186769 POINT(40.769476755525886 73.9608943053131) bank186769 +186770 POINT(40.481548162558475 73.88265857863549) bank186770 +186771 POINT(40.72167525557326 73.58526970754318) bank186771 +186772 POINT(40.11840964669185 73.8331204690269) bank186772 +186773 POINT(41.592681489307246 73.66881153984123) bank186773 +186774 POINT(41.306596010749686 74.26768151968433) bank186774 +186775 POINT(41.084066065498305 74.47555613499546) bank186775 +186776 POINT(40.36030325853449 74.59505809664972) bank186776 +186777 POINT(39.97598465131965 73.93020888698332) bank186777 +186778 POINT(41.39465566576571 74.34094237232023) bank186778 +186779 POINT(40.83626708534901 74.84799987166528) bank186779 +186780 POINT(40.8845525940879 74.05696402562899) bank186780 +186781 POINT(41.53543702384172 73.03389521896516) bank186781 +186782 POINT(39.84220362037637 74.72214746367558) bank186782 +186783 POINT(40.823752943025404 74.93703577885884) bank186783 +186784 POINT(39.891719604657396 73.70718349879073) bank186784 +186785 POINT(40.956051363132524 73.40144117103704) bank186785 +186786 POINT(40.729239855823735 73.16018910602887) bank186786 +186787 POINT(41.17615611550008 74.97711817924746) bank186787 +186788 POINT(41.19235008433227 73.13748628011342) bank186788 +186789 POINT(39.84961513037439 73.1280983099245) bank186789 +186790 POINT(40.31660004975459 73.9006096482917) bank186790 +186791 POINT(41.14764550976887 73.01287171543913) bank186791 +186792 POINT(40.89954686658157 73.64176918207917) bank186792 +186793 POINT(40.69671763725741 74.33572488553237) bank186793 +186794 POINT(40.902720783681666 74.96318261034995) bank186794 +186795 POINT(39.734485728454636 74.25208496253052) bank186795 +186796 POINT(40.722819224510154 73.94687529845763) bank186796 +186797 POINT(40.141114399193704 74.03541258758267) bank186797 +186798 POINT(41.24057548102538 75.00334479780709) bank186798 +186799 POINT(40.184157163303794 74.24960713811988) bank186799 +186800 POINT(40.72119806104076 74.83339269823183) bank186800 +186801 POINT(40.24078757785329 73.60238834959333) bank186801 +186802 POINT(40.06779380847424 74.37646000155297) bank186802 +186803 POINT(40.59947955727506 74.59626182569009) bank186803 +186804 POINT(40.98891466475766 73.00930378745193) bank186804 +186805 POINT(40.468320576547804 74.87326642714793) bank186805 +186806 POINT(40.96519962771951 73.2134811907711) bank186806 +186807 POINT(40.35574823194577 74.32643884080835) bank186807 +186808 POINT(41.41963571768649 74.52059817641043) bank186808 +186809 POINT(40.33755109938736 74.83947498629841) bank186809 +186810 POINT(40.461850947546225 73.89079785262638) bank186810 +186811 POINT(41.21239490364795 73.03077205110921) bank186811 +186812 POINT(41.07950909330906 73.17516283515111) bank186812 +186813 POINT(39.84846496058051 74.06816448270406) bank186813 +186814 POINT(41.59697560290451 73.88771979165077) bank186814 +186815 POINT(41.64299181639133 73.05132180700029) bank186815 +186816 POINT(40.051241506991765 73.31005006857939) bank186816 +186817 POINT(40.91519899978654 74.33615997253897) bank186817 +186818 POINT(40.78922980960409 74.98471117083595) bank186818 +186819 POINT(40.7837588974059 74.32325790320388) bank186819 +186820 POINT(40.00549736583828 73.45103803363804) bank186820 +186821 POINT(40.17156184157675 73.65154572881531) bank186821 +186822 POINT(41.35729729431695 73.9131323554524) bank186822 +186823 POINT(41.34562934783477 74.50959174579677) bank186823 +186824 POINT(40.40648754027201 74.33148420367307) bank186824 +186825 POINT(40.18496646909416 74.71251157540915) bank186825 +186826 POINT(41.56996158961107 73.2430817742549) bank186826 +186827 POINT(40.55094956134361 73.21985351754853) bank186827 +186828 POINT(40.50810517237394 73.19041916058765) bank186828 +186829 POINT(41.638355280698015 73.3177782002483) bank186829 +186830 POINT(41.70278661314629 73.73068238137783) bank186830 +186831 POINT(40.7929991099453 74.05046264844306) bank186831 +186832 POINT(41.69630424670921 74.14682697445298) bank186832 +186833 POINT(40.637593759426196 74.66550015898014) bank186833 +186834 POINT(40.37282191186623 73.95921000071858) bank186834 +186835 POINT(41.24201664363223 73.7719898371391) bank186835 +186836 POINT(40.64193790102385 74.68476522381681) bank186836 +186837 POINT(40.49899143627206 73.51991333659069) bank186837 +186838 POINT(41.61986154489431 73.24857671469162) bank186838 +186839 POINT(40.1513421781062 74.59053562717371) bank186839 +186840 POINT(40.305954949030074 74.00951083048683) bank186840 +186841 POINT(40.64593507686103 73.28518895770932) bank186841 +186842 POINT(40.59033673647448 74.62853720567047) bank186842 +186843 POINT(41.61043799021901 73.12969273296831) bank186843 +186844 POINT(41.12166890514574 73.61757875643616) bank186844 +186845 POINT(40.10964843845795 74.58556378027254) bank186845 +186846 POINT(40.532374424337775 73.62378818156863) bank186846 +186847 POINT(39.88767376563049 73.2756771806863) bank186847 +186848 POINT(39.84265780380045 73.62322800486324) bank186848 +186849 POINT(40.970118270417466 74.77193343216622) bank186849 +186850 POINT(40.201159577702 73.52323086278253) bank186850 +186851 POINT(41.632080068160484 73.24342355658544) bank186851 +186852 POINT(41.09714550179017 73.92383389054379) bank186852 +186853 POINT(41.5134272735571 74.24200481180766) bank186853 +186854 POINT(41.2404054111863 74.32081673298562) bank186854 +186855 POINT(40.390476885477234 73.33944674487489) bank186855 +186856 POINT(40.55890726639126 74.46220776980493) bank186856 +186857 POINT(41.024837028367365 73.23371460186462) bank186857 +186858 POINT(41.28613981187093 74.09965673834867) bank186858 +186859 POINT(39.90080073098186 74.77163532545742) bank186859 +186860 POINT(40.56134461215797 73.81967287864182) bank186860 +186861 POINT(40.57367481555889 74.72242505114087) bank186861 +186862 POINT(40.95666007107684 74.80774360239492) bank186862 +186863 POINT(40.27436268034379 74.36264983224261) bank186863 +186864 POINT(40.104880048362425 74.52225184064483) bank186864 +186865 POINT(41.233471231580566 74.30457046997002) bank186865 +186866 POINT(40.09964985114863 74.33351425093528) bank186866 +186867 POINT(41.663174213006236 73.51441481935801) bank186867 +186868 POINT(41.021466526376756 74.10541625093407) bank186868 +186869 POINT(40.10723553507156 73.4684429368576) bank186869 +186870 POINT(41.39442101413422 73.84536443649502) bank186870 +186871 POINT(40.17825838863709 73.57396416647383) bank186871 +186872 POINT(40.46191487239622 73.08349181296985) bank186872 +186873 POINT(40.869777635093506 73.05550206809231) bank186873 +186874 POINT(40.01312184818239 73.2201096949041) bank186874 +186875 POINT(40.908993023047614 73.62153054594783) bank186875 +186876 POINT(40.64583684243612 73.88566124792241) bank186876 +186877 POINT(41.18702052397404 73.1987739552816) bank186877 +186878 POINT(40.59868131185898 74.17154798253682) bank186878 +186879 POINT(40.41225105853605 73.50390331145393) bank186879 +186880 POINT(40.79357289435465 73.10063828186668) bank186880 +186881 POINT(40.501551634650575 74.6487226729805) bank186881 +186882 POINT(40.98458862702526 74.64851734495251) bank186882 +186883 POINT(41.266291727089246 74.84764145768729) bank186883 +186884 POINT(40.63789906138278 74.36894736134876) bank186884 +186885 POINT(40.87570388429241 73.66261798044566) bank186885 +186886 POINT(41.39781809533999 73.37587400556899) bank186886 +186887 POINT(40.6156216243684 73.13304830610791) bank186887 +186888 POINT(40.64658880732669 74.11096984286996) bank186888 +186889 POINT(40.5083121419316 74.69750795585887) bank186889 +186890 POINT(40.29434006840488 73.94810535903969) bank186890 +186891 POINT(40.59920153460087 73.1469493925248) bank186891 +186892 POINT(40.11575790720716 74.17230268855342) bank186892 +186893 POINT(40.89792192262576 74.67475067923775) bank186893 +186894 POINT(41.30708845984388 74.65413969290026) bank186894 +186895 POINT(40.68691443750516 74.73414421051761) bank186895 +186896 POINT(41.2773693324007 73.96215092428508) bank186896 +186897 POINT(41.221389487658165 74.05327098170055) bank186897 +186898 POINT(39.94077273739762 73.03381060232597) bank186898 +186899 POINT(41.695427298024896 73.07625975354968) bank186899 +186900 POINT(39.92551467295979 74.81652539443533) bank186900 +186901 POINT(41.37053967812795 73.9838243468754) bank186901 +186902 POINT(40.62530524127307 74.72402693878234) bank186902 +186903 POINT(40.59860849424467 73.58994044835134) bank186903 +186904 POINT(40.8368582023824 73.94497869580339) bank186904 +186905 POINT(40.87473600424992 74.46471288863313) bank186905 +186906 POINT(40.21383554523103 73.79174154568221) bank186906 +186907 POINT(40.547356816930034 73.45132739355604) bank186907 +186908 POINT(41.19498221830568 73.04033181470894) bank186908 +186909 POINT(40.74137385283796 73.18440597540048) bank186909 +186910 POINT(41.69292059918509 73.87154627821207) bank186910 +186911 POINT(41.11486381095507 73.99878030196874) bank186911 +186912 POINT(41.02703257272654 74.37401737734812) bank186912 +186913 POINT(40.35214097480002 74.19547715627053) bank186913 +186914 POINT(41.14134719405972 73.13908305277262) bank186914 +186915 POINT(41.44092034906712 74.33012333899737) bank186915 +186916 POINT(40.36728466141722 75.00048890596409) bank186916 +186917 POINT(39.89591668610583 73.78653161649434) bank186917 +186918 POINT(39.85734798932395 74.605229721276) bank186918 +186919 POINT(39.76139464564905 74.08649583508661) bank186919 +186920 POINT(40.826755236711186 74.41795603604452) bank186920 +186921 POINT(41.460766342140296 74.83545754492793) bank186921 +186922 POINT(41.35002054685074 74.50209226058402) bank186922 +186923 POINT(40.42670145390038 73.17547769379271) bank186923 +186924 POINT(40.310240315208596 74.79745246130385) bank186924 +186925 POINT(40.38834322660628 74.38461943302163) bank186925 +186926 POINT(40.090434651434656 73.31668949928536) bank186926 +186927 POINT(41.30772763792522 74.8435794567547) bank186927 +186928 POINT(41.03399436641253 74.27193393695069) bank186928 +186929 POINT(40.516018831546646 73.78932535716298) bank186929 +186930 POINT(41.34571704139421 73.88889561330139) bank186930 +186931 POINT(40.56484634858777 73.86418887446925) bank186931 +186932 POINT(39.739912657863016 73.56067039418525) bank186932 +186933 POINT(41.30805573175501 74.32215585758483) bank186933 +186934 POINT(40.31738616049403 73.82674721673463) bank186934 +186935 POINT(39.99265698668313 74.53334403900459) bank186935 +186936 POINT(40.2892875485901 73.793739326683) bank186936 +186937 POINT(40.18053521719935 73.76282168863933) bank186937 +186938 POINT(40.97777669024748 73.04390522381074) bank186938 +186939 POINT(41.58906177318694 74.71745387453001) bank186939 +186940 POINT(40.394177892248855 74.23844383292047) bank186940 +186941 POINT(41.26532194164933 73.95683524236978) bank186941 +186942 POINT(41.35942942853544 74.00148969067638) bank186942 +186943 POINT(40.423437429519666 74.45649199382912) bank186943 +186944 POINT(40.21752452224547 73.77936326667027) bank186944 +186945 POINT(40.22188966789381 74.47757780938863) bank186945 +186946 POINT(40.563084293292036 73.83517563167148) bank186946 +186947 POINT(39.83726552306543 73.02999272069617) bank186947 +186948 POINT(41.30358577960193 73.72577508642192) bank186948 +186949 POINT(40.999192567886126 74.1778230552265) bank186949 +186950 POINT(39.98065892766059 74.27668443497335) bank186950 +186951 POINT(40.556105241118644 73.32505846570461) bank186951 +186952 POINT(40.83403615974104 74.8847864372501) bank186952 +186953 POINT(39.972860924888614 73.69667474501644) bank186953 +186954 POINT(39.77693474706116 74.64249159830176) bank186954 +186955 POINT(39.977709507253536 73.39077292716878) bank186955 +186956 POINT(41.508778284834506 74.74641608385902) bank186956 +186957 POINT(40.29387060824596 73.425995123895) bank186957 +186958 POINT(40.25186471716145 74.20622018623897) bank186958 +186959 POINT(41.32558636009497 74.15184978829694) bank186959 +186960 POINT(40.39288170936432 73.83911288257936) bank186960 +186961 POINT(39.941082440011364 74.31230915811852) bank186961 +186962 POINT(39.9722625111455 74.34103412919983) bank186962 +186963 POINT(39.930633846555686 74.40576112840564) bank186963 +186964 POINT(40.63018801728744 73.2122771299005) bank186964 +186965 POINT(41.499857815579695 74.57797156174325) bank186965 +186966 POINT(40.036468922904184 73.70985971947366) bank186966 +186967 POINT(40.89497725871415 73.55417475709532) bank186967 +186968 POINT(41.0393809941462 73.48036048694642) bank186968 +186969 POINT(40.0792201439866 74.19078397370323) bank186969 +186970 POINT(41.53115166280742 74.32135013496938) bank186970 +186971 POINT(40.4396531466564 74.21569650372952) bank186971 +186972 POINT(40.28359702203074 74.20859752140862) bank186972 +186973 POINT(40.032134142279766 74.6203740986708) bank186973 +186974 POINT(40.622331370155 74.53305012161888) bank186974 +186975 POINT(41.062059056850195 74.28766475533176) bank186975 +186976 POINT(39.73491987658543 73.30751918737741) bank186976 +186977 POINT(41.1752208880756 73.12612831463964) bank186977 +186978 POINT(40.682100379756974 74.21078215702603) bank186978 +186979 POINT(40.360041372342366 74.9728027707679) bank186979 +186980 POINT(39.89627846631665 74.9837019309956) bank186980 +186981 POINT(39.902577745984615 73.82124922253814) bank186981 +186982 POINT(40.843327689108996 74.79511318135775) bank186982 +186983 POINT(40.145695179339306 74.8347856104132) bank186983 +186984 POINT(40.883165749089976 73.21238044230851) bank186984 +186985 POINT(41.5848625663896 74.47857391446013) bank186985 +186986 POINT(39.794775200237034 74.75469350560796) bank186986 +186987 POINT(40.11504810275204 74.77282505674025) bank186987 +186988 POINT(41.56343702170837 73.4545054819595) bank186988 +186989 POINT(39.73012976594884 73.2518595974843) bank186989 +186990 POINT(40.85574545401214 73.33425349722258) bank186990 +186991 POINT(41.255935026485105 74.97306762658157) bank186991 +186992 POINT(40.66941297447579 73.85279009435247) bank186992 +186993 POINT(39.78088170965362 73.3484092887731) bank186993 +186994 POINT(39.901426905636214 74.28129432669009) bank186994 +186995 POINT(39.92459343857553 73.62839382318829) bank186995 +186996 POINT(41.44570770649566 73.76261775589273) bank186996 +186997 POINT(40.536314630963815 74.08633270241282) bank186997 +186998 POINT(40.23077684376015 74.668181605932) bank186998 +186999 POINT(41.69153467438643 74.82125079833469) bank186999 +187000 POINT(41.42451451691449 74.19940969594998) bank187000 +187001 POINT(40.08637656624282 74.21140480908068) bank187001 +187002 POINT(41.488262795798796 74.61648310259089) bank187002 +187003 POINT(40.81277064611367 74.64900787400612) bank187003 +187004 POINT(40.7370987319965 73.73318500469752) bank187004 +187005 POINT(41.257712522222334 74.86713082486368) bank187005 +187006 POINT(40.13093221188801 74.20954771843358) bank187006 +187007 POINT(41.16411243099489 73.98913944771026) bank187007 +187008 POINT(41.70708917854189 74.58833989012054) bank187008 +187009 POINT(40.103446877084366 74.71112549646922) bank187009 +187010 POINT(40.82497482704754 74.12344832040729) bank187010 +187011 POINT(40.27463775049029 73.49225322910738) bank187011 +187012 POINT(40.13030862643596 74.05801693192235) bank187012 +187013 POINT(41.0987256911106 74.5847803951498) bank187013 +187014 POINT(41.68224128902767 74.92653706034537) bank187014 +187015 POINT(40.4509186901121 74.27789627459329) bank187015 +187016 POINT(40.44599070256575 74.42270568201049) bank187016 +187017 POINT(41.54817726308018 74.80256547859753) bank187017 +187018 POINT(40.2228961954812 73.9431583625738) bank187018 +187019 POINT(41.41077446009256 74.46797166482291) bank187019 +187020 POINT(41.55585926388833 74.8275072680169) bank187020 +187021 POINT(41.44887235593067 73.3592936407591) bank187021 +187022 POINT(40.834341917674834 73.57278467365559) bank187022 +187023 POINT(40.910158251709625 74.58749188946027) bank187023 +187024 POINT(40.46596992260371 74.89289081386639) bank187024 +187025 POINT(40.806848537644 73.32695319709711) bank187025 +187026 POINT(41.256080223296806 74.57799030571273) bank187026 +187027 POINT(39.87196698936662 73.17095713413582) bank187027 +187028 POINT(40.05356507956345 74.55733277223311) bank187028 +187029 POINT(39.80036920808757 74.285302437598) bank187029 +187030 POINT(40.96196236996637 73.1759255198621) bank187030 +187031 POINT(41.451563984032674 74.7885064638061) bank187031 +187032 POINT(39.98623221396822 73.67230645288456) bank187032 +187033 POINT(39.93657810820417 74.39449880931872) bank187033 +187034 POINT(41.34052845177238 73.59899608272069) bank187034 +187035 POINT(40.72250859629045 73.78739439663177) bank187035 +187036 POINT(40.814364810502006 73.18925399685855) bank187036 +187037 POINT(39.98579043567591 74.41400620130192) bank187037 +187038 POINT(40.69429081835463 74.80840251010956) bank187038 +187039 POINT(41.27479502825944 73.25064314879785) bank187039 +187040 POINT(40.80457215707909 74.25423329931265) bank187040 +187041 POINT(40.242886017482114 73.53372527258558) bank187041 +187042 POINT(40.05658725997067 74.65691298462946) bank187042 +187043 POINT(40.62193958157483 73.26718097654232) bank187043 +187044 POINT(40.897068232129875 73.06092087085332) bank187044 +187045 POINT(41.05523552115813 74.44542517860715) bank187045 +187046 POINT(40.54018588483537 74.39108693969985) bank187046 +187047 POINT(40.64414157943286 73.27026380917974) bank187047 +187048 POINT(41.15951305429056 74.9944506035756) bank187048 +187049 POINT(40.779785405232026 73.07228920265653) bank187049 +187050 POINT(40.00153429101979 74.87541488935881) bank187050 +187051 POINT(41.344658513391245 75.0055413598317) bank187051 +187052 POINT(39.9262494324397 73.35921980145503) bank187052 +187053 POINT(40.41652533875149 73.16987695068894) bank187053 +187054 POINT(40.503910787347664 74.37053211525043) bank187054 +187055 POINT(39.80804879782565 73.4772529495991) bank187055 +187056 POINT(40.02902700008108 73.82871671650133) bank187056 +187057 POINT(40.61905835213244 73.87308501930012) bank187057 +187058 POINT(39.75270721619028 73.02389583934662) bank187058 +187059 POINT(40.88072545002219 74.7622581327897) bank187059 +187060 POINT(41.71153025394277 73.72286562146323) bank187060 +187061 POINT(41.02267402709552 74.93161670979931) bank187061 +187062 POINT(41.22254063495755 74.52715392810643) bank187062 +187063 POINT(40.73121804310948 73.3058647166768) bank187063 +187064 POINT(39.85558783397884 74.59502016818254) bank187064 +187065 POINT(40.51600561102169 73.01251229300223) bank187065 +187066 POINT(40.432057644732176 73.1808957753656) bank187066 +187067 POINT(41.15595070586589 74.72918188046876) bank187067 +187068 POINT(41.338700129098925 74.1130995853227) bank187068 +187069 POINT(40.86896719135543 74.39226409312197) bank187069 +187070 POINT(39.98005673770776 74.78966124716548) bank187070 +187071 POINT(41.58608928596568 74.43541763449518) bank187071 +187072 POINT(41.338889659922856 73.78901682454857) bank187072 +187073 POINT(41.31617385664735 73.5676335755885) bank187073 +187074 POINT(40.94053142206077 74.31973463493631) bank187074 +187075 POINT(40.02342306166973 74.32005047121191) bank187075 +187076 POINT(40.494006501263655 74.37384688514817) bank187076 +187077 POINT(41.69293232328024 74.52557007824423) bank187077 +187078 POINT(41.517705712325636 73.63080691484787) bank187078 +187079 POINT(40.44034839490105 73.82052004625614) bank187079 +187080 POINT(40.66281783836434 74.10784074613126) bank187080 +187081 POINT(41.1631286286651 73.0355576708708) bank187081 +187082 POINT(41.06661248714902 73.42389798248401) bank187082 +187083 POINT(41.55053695231402 73.09013807178262) bank187083 +187084 POINT(40.771370256957205 74.84665538761682) bank187084 +187085 POINT(41.706791379988495 74.45178545356397) bank187085 +187086 POINT(41.361714255211794 74.16893085791182) bank187086 +187087 POINT(40.612241911760975 74.04616843997226) bank187087 +187088 POINT(39.976200715969526 73.24766680267487) bank187088 +187089 POINT(41.38833007995364 73.29771289221542) bank187089 +187090 POINT(39.960821287369114 74.03928474695181) bank187090 +187091 POINT(40.7421594889971 74.18096868095193) bank187091 +187092 POINT(40.647736688966305 73.80625698650317) bank187092 +187093 POINT(40.25960961000438 74.2455103243103) bank187093 +187094 POINT(41.19567033621776 74.80941138392595) bank187094 +187095 POINT(41.39724640262575 73.96732302577082) bank187095 +187096 POINT(40.39158758429248 74.30317494276395) bank187096 +187097 POINT(40.24959284918569 74.14761448073507) bank187097 +187098 POINT(40.01312089023393 74.89851585936196) bank187098 +187099 POINT(40.07033128413078 73.67255880930927) bank187099 +187100 POINT(41.695087801952255 74.23099703824235) bank187100 +187101 POINT(41.05541980659014 74.01449748939888) bank187101 +187102 POINT(41.0747509437475 73.66467092215551) bank187102 +187103 POINT(41.69579180214329 74.93999288714537) bank187103 +187104 POINT(39.74008578745912 74.00344663539782) bank187104 +187105 POINT(41.610347511682896 73.49686899902369) bank187105 +187106 POINT(40.7668721918727 73.24029188692124) bank187106 +187107 POINT(41.62509530076393 73.2140377156161) bank187107 +187108 POINT(41.49702293411455 73.10399322893491) bank187108 +187109 POINT(40.73404250873727 74.92837693392535) bank187109 +187110 POINT(40.584562670477375 73.53536377481312) bank187110 +187111 POINT(40.062830568374814 74.25831050408058) bank187111 +187112 POINT(40.23408403732474 74.51348244255148) bank187112 +187113 POINT(41.50525686018646 74.82750795582155) bank187113 +187114 POINT(40.71289673983712 74.30841460669409) bank187114 +187115 POINT(40.555695269909585 74.24045714591536) bank187115 +187116 POINT(40.1914811537624 74.09937471740771) bank187116 +187117 POINT(40.249828340892954 74.87472122328586) bank187117 +187118 POINT(39.86164191820461 74.48476683709679) bank187118 +187119 POINT(39.940802262032115 74.68004531445975) bank187119 +187120 POINT(40.092391342282816 73.95363972533245) bank187120 +187121 POINT(41.02538360895724 73.17782197428902) bank187121 +187122 POINT(41.47186420999582 74.40219255453302) bank187122 +187123 POINT(40.86681198884988 73.63035260580466) bank187123 +187124 POINT(39.76714058177129 73.68967813780864) bank187124 +187125 POINT(41.61171771563627 73.62013881601945) bank187125 +187126 POINT(40.23708543859641 73.14054570841367) bank187126 +187127 POINT(40.843125195066875 74.24170761727542) bank187127 +187128 POINT(40.04768735229027 74.45382385120143) bank187128 +187129 POINT(40.793964572659675 74.24411521371908) bank187129 +187130 POINT(41.14078057338825 74.11649350032728) bank187130 +187131 POINT(40.31404245348979 73.45471829543533) bank187131 +187132 POINT(41.69867671342005 73.18448794320946) bank187132 +187133 POINT(40.108080856890275 73.81055845954398) bank187133 +187134 POINT(41.55420310629664 74.09962789390168) bank187134 +187135 POINT(41.454503790800516 73.56650067287926) bank187135 +187136 POINT(40.687017458138776 74.67630855575487) bank187136 +187137 POINT(40.17980525024083 73.64460646590736) bank187137 +187138 POINT(40.07288316256116 74.40000562419358) bank187138 +187139 POINT(40.063897932378495 73.63545340957788) bank187139 +187140 POINT(40.24152484691748 73.86586998930385) bank187140 +187141 POINT(40.14180374081457 73.07833737866123) bank187141 +187142 POINT(41.4937537245399 73.14372934832907) bank187142 +187143 POINT(40.558240957011826 73.08474185030406) bank187143 +187144 POINT(40.30718757424451 74.72065156549105) bank187144 +187145 POINT(40.98345582696263 73.57566232549017) bank187145 +187146 POINT(40.43200760082201 74.02038721036762) bank187146 +187147 POINT(40.58743617776725 74.28947799595926) bank187147 +187148 POINT(39.95716115215526 73.5847546938476) bank187148 +187149 POINT(40.469804405280065 73.16370173385724) bank187149 +187150 POINT(41.08830413164032 73.39874922506417) bank187150 +187151 POINT(40.07543831672626 73.62655354179918) bank187151 +187152 POINT(40.669849589087896 74.02603652663248) bank187152 +187153 POINT(40.46658275951778 73.74626614516445) bank187153 +187154 POINT(40.812465666810844 74.13086525122509) bank187154 +187155 POINT(40.74419296910798 74.30685449726882) bank187155 +187156 POINT(41.05181571546632 74.54435245976866) bank187156 +187157 POINT(40.8357189204307 74.10458509732993) bank187157 +187158 POINT(40.61236820546051 74.1322435328501) bank187158 +187159 POINT(41.01251973102995 74.99068617808281) bank187159 +187160 POINT(41.40278737769882 73.93484942177149) bank187160 +187161 POINT(41.699045746365265 73.55094982493395) bank187161 +187162 POINT(41.51819694056458 74.47116651013897) bank187162 +187163 POINT(41.66757161250765 74.35748102079411) bank187163 +187164 POINT(40.82773802078497 73.1365752609527) bank187164 +187165 POINT(40.4305279889109 74.92720467941194) bank187165 +187166 POINT(41.67108454232123 74.95782407856888) bank187166 +187167 POINT(41.15405650997488 73.94670028400873) bank187167 +187168 POINT(41.20590752067487 73.64759188217464) bank187168 +187169 POINT(40.89843442863363 73.97437404975445) bank187169 +187170 POINT(40.07225063066754 73.12295284311735) bank187170 +187171 POINT(40.614156135245445 74.77691223600218) bank187171 +187172 POINT(41.29687060218142 73.49869565226692) bank187172 +187173 POINT(41.542753158916724 74.28921141452678) bank187173 +187174 POINT(39.806204735296035 73.98053381337546) bank187174 +187175 POINT(39.83372761378671 74.65977317791535) bank187175 +187176 POINT(40.1158438006458 74.05939410654143) bank187176 +187177 POINT(40.81144698639014 74.15432186927376) bank187177 +187178 POINT(40.948936188781865 74.88767333811111) bank187178 +187179 POINT(40.5780587500273 74.28588758318979) bank187179 +187180 POINT(39.720973749786566 73.57634203835536) bank187180 +187181 POINT(40.59843147516726 74.84784101759028) bank187181 +187182 POINT(41.50277018586586 73.4600771940526) bank187182 +187183 POINT(39.76533304234088 73.14546711679347) bank187183 +187184 POINT(41.219699607407584 74.14258347831083) bank187184 +187185 POINT(41.09740313066829 73.51661957042111) bank187185 +187186 POINT(39.781162165167395 74.44997381833255) bank187186 +187187 POINT(41.67355651987217 73.2584201269955) bank187187 +187188 POINT(41.484645696689675 74.14053275106421) bank187188 +187189 POINT(40.00195682951689 73.60553161137553) bank187189 +187190 POINT(40.91081985235898 73.30333161896242) bank187190 +187191 POINT(39.929791894085206 73.4282155336884) bank187191 +187192 POINT(40.82788988540706 74.96952667034238) bank187192 +187193 POINT(41.703427588759034 74.41378203699357) bank187193 +187194 POINT(40.308967148198256 73.87134342099263) bank187194 +187195 POINT(40.28939544210038 73.50194587690883) bank187195 +187196 POINT(40.1902520918279 74.33401604302745) bank187196 +187197 POINT(41.23904953014983 74.84397804410379) bank187197 +187198 POINT(40.14107509860981 73.06638846718909) bank187198 +187199 POINT(39.87469785133768 73.70345503499365) bank187199 +187200 POINT(40.786240740990266 74.48190743987253) bank187200 +187201 POINT(40.399378027603085 74.55260181842579) bank187201 +187202 POINT(40.544420267339675 74.30000575873542) bank187202 +187203 POINT(40.062437256719534 74.67656367575037) bank187203 +187204 POINT(41.488677296118446 74.03056173656618) bank187204 +187205 POINT(40.424311110022174 73.37696995231298) bank187205 +187206 POINT(41.43231748492656 74.42217653373751) bank187206 +187207 POINT(41.369946794929014 74.96816944183549) bank187207 +187208 POINT(41.1184619094572 74.70144642129573) bank187208 +187209 POINT(40.04431539735754 74.05976198682957) bank187209 +187210 POINT(41.171398352096396 73.40846917626952) bank187210 +187211 POINT(40.36367392679269 74.88955633626442) bank187211 +187212 POINT(41.23629591050191 73.2898175763108) bank187212 +187213 POINT(41.629234598261895 74.92569706245551) bank187213 +187214 POINT(40.02827791199747 73.04865931886079) bank187214 +187215 POINT(41.009324705454375 73.89382580112009) bank187215 +187216 POINT(41.02860832825737 74.2814436162735) bank187216 +187217 POINT(40.34922848561885 74.61549221050872) bank187217 +187218 POINT(40.692040861688504 73.85760929485726) bank187218 +187219 POINT(41.23010784253059 73.57468565822128) bank187219 +187220 POINT(40.30354334204896 73.22335302198131) bank187220 +187221 POINT(41.44960454517935 74.04483694375632) bank187221 +187222 POINT(41.038686718668444 73.2206154875847) bank187222 +187223 POINT(39.97359071873485 74.34121699708209) bank187223 +187224 POINT(40.59355805146542 73.2588883636457) bank187224 +187225 POINT(40.186238920653494 73.69608848507208) bank187225 +187226 POINT(41.29026895893572 74.71550941796481) bank187226 +187227 POINT(41.1986487951259 73.92727249068457) bank187227 +187228 POINT(40.57465316075683 73.9838736455182) bank187228 +187229 POINT(41.55438860471816 74.13452179432343) bank187229 +187230 POINT(41.4232654942201 73.6154998598115) bank187230 +187231 POINT(39.98354737134516 73.73989541967671) bank187231 +187232 POINT(40.46323351028099 74.21423229334982) bank187232 +187233 POINT(41.081091964988815 74.28120341865977) bank187233 +187234 POINT(40.14195183508046 73.42644236839251) bank187234 +187235 POINT(41.04428012602418 73.6623823861676) bank187235 +187236 POINT(40.8995211796356 74.9621440539287) bank187236 +187237 POINT(39.94476988234329 74.08696336693002) bank187237 +187238 POINT(41.30462150349256 73.65739523894592) bank187238 +187239 POINT(40.784374495354726 73.73044657547966) bank187239 +187240 POINT(40.53753287156557 74.01167725582098) bank187240 +187241 POINT(41.58670618811686 74.10907290226824) bank187241 +187242 POINT(40.473070618413786 74.27691848507668) bank187242 +187243 POINT(41.12288048091776 74.04847342478925) bank187243 +187244 POINT(40.88434335335009 74.03934653116872) bank187244 +187245 POINT(40.871028054042625 73.87671283211874) bank187245 +187246 POINT(40.00400535524947 74.30004858104587) bank187246 +187247 POINT(40.59238833737724 73.20995295625582) bank187247 +187248 POINT(40.31861907542274 73.97523626724976) bank187248 +187249 POINT(40.58977785897619 73.54270073158823) bank187249 +187250 POINT(41.4213710543852 74.41752184683239) bank187250 +187251 POINT(40.970232300531286 73.25171716565902) bank187251 +187252 POINT(40.28075082634028 74.6389338784418) bank187252 +187253 POINT(40.28972836733092 73.54339677261392) bank187253 +187254 POINT(40.31776464534586 74.49384408876729) bank187254 +187255 POINT(41.08484991019309 74.37635048594608) bank187255 +187256 POINT(39.92111773962537 73.38314629865347) bank187256 +187257 POINT(39.9732763083913 73.4976665612551) bank187257 +187258 POINT(41.43352372513369 73.76295680533053) bank187258 +187259 POINT(39.726269873816584 73.74079576215155) bank187259 +187260 POINT(41.231060063999045 74.29892619622122) bank187260 +187261 POINT(40.60058334654548 74.3741595597376) bank187261 +187262 POINT(40.06257890661166 74.97904520992411) bank187262 +187263 POINT(40.83324571811493 74.05462043779681) bank187263 +187264 POINT(40.856948510159235 73.83699544723932) bank187264 +187265 POINT(39.85810279369637 73.50226712391891) bank187265 +187266 POINT(41.179756153364245 74.72274224988914) bank187266 +187267 POINT(40.05057147619719 73.12602151927942) bank187267 +187268 POINT(39.856417598873236 73.51912799446207) bank187268 +187269 POINT(39.99208974626744 74.48071779625502) bank187269 +187270 POINT(41.340899436630764 73.66661001335594) bank187270 +187271 POINT(41.48850796241057 74.53242871865726) bank187271 +187272 POINT(40.658235569019986 73.39350657455226) bank187272 +187273 POINT(40.51579679628541 73.8092971243479) bank187273 +187274 POINT(39.823260284660165 74.66527796639139) bank187274 +187275 POINT(40.82531208657087 74.33005895912194) bank187275 +187276 POINT(40.96554713354643 74.58635124012905) bank187276 +187277 POINT(40.6146008725764 74.04268081127462) bank187277 +187278 POINT(40.524254227021764 73.75200147058571) bank187278 +187279 POINT(40.009928689566344 74.42596239425112) bank187279 +187280 POINT(41.47225853706048 74.86877640424382) bank187280 +187281 POINT(40.549104328436535 74.10496460498982) bank187281 +187282 POINT(40.679496291226734 73.24686144506063) bank187282 +187283 POINT(40.8534262847874 73.43621573136215) bank187283 +187284 POINT(40.57372836758427 74.47046871967245) bank187284 +187285 POINT(40.52266161555194 74.04219116634874) bank187285 +187286 POINT(41.70646927432136 73.81122719216134) bank187286 +187287 POINT(40.90619750655641 74.50370290016173) bank187287 +187288 POINT(40.454194567204866 73.77259337185689) bank187288 +187289 POINT(39.80340596933289 73.82787513747924) bank187289 +187290 POINT(41.44849714705707 73.52053070660568) bank187290 +187291 POINT(41.310384054117684 73.50494029564098) bank187291 +187292 POINT(39.968928814968834 73.75060315072625) bank187292 +187293 POINT(40.05824244599654 74.36988851356197) bank187293 +187294 POINT(41.16771735468809 74.08467153503507) bank187294 +187295 POINT(40.809223392730324 74.86381255438805) bank187295 +187296 POINT(41.174589852435936 74.32416039138141) bank187296 +187297 POINT(41.376287315841786 73.81660436969732) bank187297 +187298 POINT(40.50962044299887 73.1808938031229) bank187298 +187299 POINT(39.911979510465784 73.30310816771156) bank187299 +187300 POINT(39.771118721125546 73.85444237192694) bank187300 +187301 POINT(41.0689853321102 74.82497970155364) bank187301 +187302 POINT(41.33810022560958 74.9551637457157) bank187302 +187303 POINT(40.989821675715774 73.0890937906694) bank187303 +187304 POINT(40.29880140459651 74.60775749733) bank187304 +187305 POINT(41.67297512571228 74.11205571451684) bank187305 +187306 POINT(40.13817192946498 73.56965966982632) bank187306 +187307 POINT(41.01346863589539 73.65526198816707) bank187307 +187308 POINT(39.998061185084396 73.2571694311073) bank187308 +187309 POINT(40.45497760611738 73.21196707063906) bank187309 +187310 POINT(41.10546678593739 73.27992079680179) bank187310 +187311 POINT(40.5858005283917 74.86683433870438) bank187311 +187312 POINT(40.82120768721097 73.18388073143205) bank187312 +187313 POINT(40.621912483855475 73.49513994606993) bank187313 +187314 POINT(41.4869119828811 74.28965438022865) bank187314 +187315 POINT(41.51735349169332 73.07878589857813) bank187315 +187316 POINT(40.99114133121642 74.61057576018652) bank187316 +187317 POINT(40.28982816172368 74.72301560319372) bank187317 +187318 POINT(41.01266359686263 74.10297063709433) bank187318 +187319 POINT(39.96091558699151 74.73456361988477) bank187319 +187320 POINT(41.37492464021502 74.02979124439271) bank187320 +187321 POINT(41.06067733879458 73.79347995727366) bank187321 +187322 POINT(41.58298180116943 73.35290394977613) bank187322 +187323 POINT(39.72705858747331 73.20461049184173) bank187323 +187324 POINT(40.51358965579282 73.19980207159678) bank187324 +187325 POINT(41.04291778148948 73.82554005237125) bank187325 +187326 POINT(40.16663102048902 73.02012804795993) bank187326 +187327 POINT(41.01174130296346 74.25407695972582) bank187327 +187328 POINT(40.87027380232305 73.01523531149975) bank187328 +187329 POINT(39.794049449593096 73.4226560629593) bank187329 +187330 POINT(40.320594020686954 74.91100687945274) bank187330 +187331 POINT(41.36335707637012 73.632587661079) bank187331 +187332 POINT(39.89676113464093 73.29993704553983) bank187332 +187333 POINT(41.43609646366663 74.93306721339168) bank187333 +187334 POINT(39.98722158897935 73.22229055702725) bank187334 +187335 POINT(39.88243688294322 74.53687911836302) bank187335 +187336 POINT(39.78533099266987 73.23543656883275) bank187336 +187337 POINT(40.70007263458981 73.57081118584212) bank187337 +187338 POINT(39.950305599897746 74.52773304112995) bank187338 +187339 POINT(40.43722759824399 73.17233524106484) bank187339 +187340 POINT(40.280086871131296 73.1418312124449) bank187340 +187341 POINT(41.42994003471035 73.15180054736074) bank187341 +187342 POINT(40.694259055549246 74.59741661681068) bank187342 +187343 POINT(39.96909584336676 74.43468614820301) bank187343 +187344 POINT(39.83872837534694 73.22669984383064) bank187344 +187345 POINT(40.68551070202713 74.28048408319484) bank187345 +187346 POINT(41.67474775597492 73.66789057412154) bank187346 +187347 POINT(41.59598419919733 73.62494866246077) bank187347 +187348 POINT(41.13456325773621 74.58081363829587) bank187348 +187349 POINT(40.004432515531356 74.37427793542254) bank187349 +187350 POINT(40.02619601158736 74.88301445443695) bank187350 +187351 POINT(40.092735853614016 74.21530067828283) bank187351 +187352 POINT(39.88974767509033 73.69366160634111) bank187352 +187353 POINT(39.749360130775685 73.50416937148876) bank187353 +187354 POINT(41.12092247866929 74.97210430641442) bank187354 +187355 POINT(41.42390489907686 73.10766437174861) bank187355 +187356 POINT(40.315832531297794 74.08364350128039) bank187356 +187357 POINT(41.10129713352895 73.04218591747137) bank187357 +187358 POINT(40.678755795791524 74.52229020763464) bank187358 +187359 POINT(41.19827794849869 74.64945206674246) bank187359 +187360 POINT(41.6334940893078 73.17474473643136) bank187360 +187361 POINT(41.06521138415663 74.19732511624206) bank187361 +187362 POINT(39.80235409455292 74.064617542748) bank187362 +187363 POINT(40.96402341879141 73.4803752016798) bank187363 +187364 POINT(40.80853018606454 73.57637693099889) bank187364 +187365 POINT(40.04186576239252 73.10199576203449) bank187365 +187366 POINT(40.14318712658672 74.42228863680971) bank187366 +187367 POINT(39.83230569518278 74.32016447128098) bank187367 +187368 POINT(41.41346864097578 74.708149616284) bank187368 +187369 POINT(41.15378214424688 73.7663172079482) bank187369 +187370 POINT(39.96180941037739 74.44745383237228) bank187370 +187371 POINT(40.588837047692984 73.93774809565443) bank187371 +187372 POINT(41.083505660651035 73.4029678418809) bank187372 +187373 POINT(40.40403785789894 73.09505734393508) bank187373 +187374 POINT(41.08839794729433 74.96685105826059) bank187374 +187375 POINT(41.49948630610457 73.32450828533875) bank187375 +187376 POINT(40.92488007269915 73.02504246180582) bank187376 +187377 POINT(41.06732195258355 73.95256936371779) bank187377 +187378 POINT(41.29342786608569 74.93869963151838) bank187378 +187379 POINT(41.668946753018126 73.71989100045408) bank187379 +187380 POINT(41.20786320834525 73.07116361466917) bank187380 +187381 POINT(39.86000453615233 73.12909295009801) bank187381 +187382 POINT(40.297042529495435 74.77652541724402) bank187382 +187383 POINT(41.276842040423375 73.55486986594654) bank187383 +187384 POINT(40.45435828683369 74.84584060783172) bank187384 +187385 POINT(41.70757999766512 73.36475051057889) bank187385 +187386 POINT(40.56107500291883 74.95945974449478) bank187386 +187387 POINT(41.21590285154202 74.25462971126218) bank187387 +187388 POINT(40.67120267145993 73.19795762487956) bank187388 +187389 POINT(40.694983721235324 73.46747737513675) bank187389 +187390 POINT(40.30100775635104 73.39013753710503) bank187390 +187391 POINT(40.43606999414482 73.21460907037238) bank187391 +187392 POINT(40.358380372126696 73.70858533824024) bank187392 +187393 POINT(39.85642613008004 73.73407917557711) bank187393 +187394 POINT(41.64772302888796 73.23914666416199) bank187394 +187395 POINT(41.332369717943244 73.25573258136963) bank187395 +187396 POINT(41.406643095229484 74.41206923615573) bank187396 +187397 POINT(40.79065559321772 74.15784595715944) bank187397 +187398 POINT(41.55518515961285 73.04209388569004) bank187398 +187399 POINT(41.22363359920061 73.66516866934326) bank187399 +187400 POINT(41.5471703241501 74.72218106081539) bank187400 +187401 POINT(40.531514461212865 74.02605410808505) bank187401 +187402 POINT(40.48698535983854 74.09634200663865) bank187402 +187403 POINT(40.9739239017033 74.72350887018695) bank187403 +187404 POINT(40.23180507230598 74.862203569069) bank187404 +187405 POINT(41.576950981475804 74.3668344855238) bank187405 +187406 POINT(40.37225835078925 73.89003699091525) bank187406 +187407 POINT(39.80971528621365 74.68478050647529) bank187407 +187408 POINT(39.7971036049411 73.69800300456332) bank187408 +187409 POINT(41.66913949050502 73.39267195537883) bank187409 +187410 POINT(41.06089936749064 74.74071366066823) bank187410 +187411 POINT(41.08985623204801 74.23991381734719) bank187411 +187412 POINT(40.05145614775174 73.79667607044996) bank187412 +187413 POINT(40.24406333898718 74.1470608632349) bank187413 +187414 POINT(39.97770313218029 74.80284710857018) bank187414 +187415 POINT(40.736209373836374 73.89593620974912) bank187415 +187416 POINT(41.43572764355739 73.54749298981923) bank187416 +187417 POINT(41.54189705348967 73.04772431748987) bank187417 +187418 POINT(40.81732672290324 74.6526377046397) bank187418 +187419 POINT(40.63288857739495 75.00396888360217) bank187419 +187420 POINT(40.01431686719413 73.15323624693109) bank187420 +187421 POINT(41.01566112916916 73.14242417986131) bank187421 +187422 POINT(41.3770377451135 74.98966409428235) bank187422 +187423 POINT(41.53330094191693 73.327780871) bank187423 +187424 POINT(39.77447001298152 73.16204398050037) bank187424 +187425 POINT(40.621207549582564 73.86672168436924) bank187425 +187426 POINT(40.687461954400696 74.34068661131685) bank187426 +187427 POINT(41.45290280335151 73.73722951079309) bank187427 +187428 POINT(41.442877774380946 73.37067602747162) bank187428 +187429 POINT(41.0431584407098 74.77497178826856) bank187429 +187430 POINT(39.86905046386119 73.53045270044402) bank187430 +187431 POINT(39.730068022949084 74.61568014926495) bank187431 +187432 POINT(40.99642563244465 73.91346262842632) bank187432 +187433 POINT(39.989914714404755 73.07083088198435) bank187433 +187434 POINT(40.801692819543554 74.53507630806989) bank187434 +187435 POINT(41.388715286540375 74.99855818021916) bank187435 +187436 POINT(41.682954095145625 73.4704169018671) bank187436 +187437 POINT(41.23786008184837 74.95615194629919) bank187437 +187438 POINT(40.679044773730624 73.10194702760529) bank187438 +187439 POINT(41.59355661739797 73.25150891248585) bank187439 +187440 POINT(40.11183097464059 74.82037122599833) bank187440 +187441 POINT(41.38237977653513 73.61094142721609) bank187441 +187442 POINT(40.93812809896177 73.82732070579148) bank187442 +187443 POINT(41.24839716712317 74.19415991616546) bank187443 +187444 POINT(40.87570853001267 73.87562711774964) bank187444 +187445 POINT(40.93334541702203 73.71320112869324) bank187445 +187446 POINT(40.76734855373874 74.85631722586444) bank187446 +187447 POINT(40.6135211774921 73.21344294722914) bank187447 +187448 POINT(40.87260914363512 74.7312620282522) bank187448 +187449 POINT(39.717373098349476 74.73437283143433) bank187449 +187450 POINT(41.116622158191014 73.28708405263191) bank187450 +187451 POINT(39.7670554166352 74.94780831905763) bank187451 +187452 POINT(40.18954934556295 74.04683274451358) bank187452 +187453 POINT(40.43421100470673 74.62906935343405) bank187453 +187454 POINT(41.412587086352 74.2658166006349) bank187454 +187455 POINT(40.8899157583815 73.68929748024708) bank187455 +187456 POINT(40.2850617784186 73.72480127715649) bank187456 +187457 POINT(41.07235947906414 73.36746616827969) bank187457 +187458 POINT(40.088126975687175 74.24872628252945) bank187458 +187459 POINT(41.119898880754405 74.67493345704476) bank187459 +187460 POINT(41.40452708337454 73.91999135263767) bank187460 +187461 POINT(41.303366109570014 73.72923719974555) bank187461 +187462 POINT(40.21131839903343 73.06092757372558) bank187462 +187463 POINT(41.35715567754951 73.0242788419269) bank187463 +187464 POINT(39.86854112057813 73.85974543198698) bank187464 +187465 POINT(40.718319818573406 74.07692673303798) bank187465 +187466 POINT(40.12871328497278 73.8796634790934) bank187466 +187467 POINT(41.677627843371546 73.33305207387367) bank187467 +187468 POINT(41.2920685088037 74.17452133369534) bank187468 +187469 POINT(39.852405554347534 73.86159059905106) bank187469 +187470 POINT(39.924666364203425 73.7246853737889) bank187470 +187471 POINT(41.23578991324473 73.37746206231985) bank187471 +187472 POINT(40.070729167043744 73.65844607286188) bank187472 +187473 POINT(40.67966441852329 73.30626158388759) bank187473 +187474 POINT(40.373170470252774 73.04497215736932) bank187474 +187475 POINT(39.87262633598939 73.28062974758788) bank187475 +187476 POINT(40.60373320822258 74.80307231721477) bank187476 +187477 POINT(40.25252152155598 73.25581094578259) bank187477 +187478 POINT(41.48552720968407 74.60095281732312) bank187478 +187479 POINT(39.85653418101873 73.96358786659472) bank187479 +187480 POINT(41.392765596691106 73.97202038850173) bank187480 +187481 POINT(41.123512283391754 73.3902596390227) bank187481 +187482 POINT(40.96758430825807 73.2386072919899) bank187482 +187483 POINT(41.53377413125328 74.77168163581136) bank187483 +187484 POINT(41.34678732978062 74.90103112103887) bank187484 +187485 POINT(41.16696880173064 73.32410242697682) bank187485 +187486 POINT(41.19915196754608 73.2088066580987) bank187486 +187487 POINT(40.4997138516522 74.57337464208757) bank187487 +187488 POINT(40.86748717852708 73.33434996757609) bank187488 +187489 POINT(39.75550848517224 74.417798774805) bank187489 +187490 POINT(40.83942158067407 73.44404616190042) bank187490 +187491 POINT(40.86816698941696 73.24123269263752) bank187491 +187492 POINT(39.79065578186667 73.05920724999895) bank187492 +187493 POINT(39.793818342671464 73.55265357592455) bank187493 +187494 POINT(40.07327392163432 74.14747077263077) bank187494 +187495 POINT(40.43197130061612 73.03215982508486) bank187495 +187496 POINT(39.77606635506715 74.01669573343325) bank187496 +187497 POINT(39.73179628342878 73.17185413458988) bank187497 +187498 POINT(39.929745118976896 73.89060060755874) bank187498 +187499 POINT(40.9382924241077 74.0244652905089) bank187499 +187500 POINT(40.791342584290014 73.90135722487295) bank187500 +187501 POINT(41.31032956500746 74.67745275092608) bank187501 +187502 POINT(39.772359582500336 74.93028808945655) bank187502 +187503 POINT(41.15939704649079 73.04196448339647) bank187503 +187504 POINT(41.21015252447495 74.03891542206695) bank187504 +187505 POINT(40.48036255430735 74.92509797060492) bank187505 +187506 POINT(40.00127902552305 74.94445573059768) bank187506 +187507 POINT(40.48320521983369 74.09796477209882) bank187507 +187508 POINT(40.13624516196456 74.18910658572399) bank187508 +187509 POINT(40.50367093078508 74.00261635353381) bank187509 +187510 POINT(41.01233589646472 73.57914582553154) bank187510 +187511 POINT(39.75111081755642 73.93283780361) bank187511 +187512 POINT(41.2472827350015 74.8078594235868) bank187512 +187513 POINT(39.80460115372434 73.19837244113648) bank187513 +187514 POINT(39.78896477452546 74.7766673793245) bank187514 +187515 POINT(40.727087765017224 73.83692255768534) bank187515 +187516 POINT(41.004842739927895 74.34465006831485) bank187516 +187517 POINT(41.59443524697501 73.52267074454286) bank187517 +187518 POINT(40.45796938687508 73.51406437458564) bank187518 +187519 POINT(39.73355057674643 73.37335697368333) bank187519 +187520 POINT(40.807073043478574 73.37986966776182) bank187520 +187521 POINT(40.44294971206385 74.5335116842713) bank187521 +187522 POINT(41.358060803533775 74.24263301814491) bank187522 +187523 POINT(41.11160552287143 73.06033145495306) bank187523 +187524 POINT(40.604452987558126 73.42862921465216) bank187524 +187525 POINT(40.115996925296535 74.17099381859904) bank187525 +187526 POINT(41.57458062303968 73.84911843936851) bank187526 +187527 POINT(40.565650850682886 73.52705175201125) bank187527 +187528 POINT(39.83253163756903 73.08038451240886) bank187528 +187529 POINT(41.670402116619 74.62888702923144) bank187529 +187530 POINT(40.86571476964345 74.9442175669761) bank187530 +187531 POINT(40.58874926759809 73.76355344370545) bank187531 +187532 POINT(41.527208638203426 74.12482708618795) bank187532 +187533 POINT(41.47878518762792 74.61330134923669) bank187533 +187534 POINT(40.566577364209564 74.446682849073) bank187534 +187535 POINT(40.057405859482365 73.1434949478911) bank187535 +187536 POINT(41.001321636055195 74.18980148700945) bank187536 +187537 POINT(41.68520483035922 73.88717205732773) bank187537 +187538 POINT(41.03745713711405 74.66897935704895) bank187538 +187539 POINT(40.41573315549994 74.38882032694588) bank187539 +187540 POINT(41.66055253944924 74.1795658499742) bank187540 +187541 POINT(41.25871552865634 73.78961350365792) bank187541 +187542 POINT(40.626660400232794 74.1082143901903) bank187542 +187543 POINT(41.144637289389664 73.74826526398937) bank187543 +187544 POINT(41.39966530228838 73.70599011168066) bank187544 +187545 POINT(41.70180163161774 73.47090339540158) bank187545 +187546 POINT(41.306657402066264 73.33698187392928) bank187546 +187547 POINT(41.25664180319365 74.73167003803196) bank187547 +187548 POINT(40.374163390769304 73.86115046875881) bank187548 +187549 POINT(40.68780934937031 73.31423725264837) bank187549 +187550 POINT(40.620297618310694 74.04206127929537) bank187550 +187551 POINT(39.73942794549163 74.14387641658557) bank187551 +187552 POINT(41.67804274465027 73.60433755828042) bank187552 +187553 POINT(40.79426336277297 73.58496034013946) bank187553 +187554 POINT(39.83505061017852 74.65261312587427) bank187554 +187555 POINT(41.39105071995626 73.6168897051361) bank187555 +187556 POINT(41.14692941223278 74.78658001688223) bank187556 +187557 POINT(41.036337411617176 73.06697613148498) bank187557 +187558 POINT(40.23161138825758 74.67193173510344) bank187558 +187559 POINT(40.764464212668706 73.53210161602689) bank187559 +187560 POINT(40.01820941009536 74.27814987708116) bank187560 +187561 POINT(41.6612785592753 74.16393324593528) bank187561 +187562 POINT(41.078418385030844 74.58475096675609) bank187562 +187563 POINT(41.58758293333987 73.66798398508779) bank187563 +187564 POINT(40.01825000907162 74.45441302881507) bank187564 +187565 POINT(40.52840768336196 74.78123622675153) bank187565 +187566 POINT(40.21972182479638 74.62926105153208) bank187566 +187567 POINT(40.12616451057903 74.0056155043065) bank187567 +187568 POINT(41.47723250548117 73.60888610853698) bank187568 +187569 POINT(40.39156897878992 73.49416650403411) bank187569 +187570 POINT(40.62211313647236 74.93458235952585) bank187570 +187571 POINT(40.36693587319667 74.59647892401853) bank187571 +187572 POINT(40.31356608960203 74.66034172173045) bank187572 +187573 POINT(40.626932472424215 74.87285725268693) bank187573 +187574 POINT(41.136868579156996 74.67619166362745) bank187574 +187575 POINT(40.5059155662868 73.93424197901872) bank187575 +187576 POINT(39.82332248342949 74.57749241390829) bank187576 +187577 POINT(40.30150703276349 73.61474939136583) bank187577 +187578 POINT(40.2368890727188 74.61508643229006) bank187578 +187579 POINT(41.0186798355913 73.68618848105652) bank187579 +187580 POINT(40.649533820389316 74.03722757261771) bank187580 +187581 POINT(40.69663016689743 73.00952292746854) bank187581 +187582 POINT(40.555828248328254 74.1515728520883) bank187582 +187583 POINT(39.909431416686864 74.71156026646476) bank187583 +187584 POINT(39.933781354415096 73.2572514933199) bank187584 +187585 POINT(40.44395020096942 73.19425942774048) bank187585 +187586 POINT(40.43126781492478 73.9595017903251) bank187586 +187587 POINT(40.55563582146993 74.7092295137216) bank187587 +187588 POINT(40.60996038470331 74.89700836630708) bank187588 +187589 POINT(40.682204681046706 73.42849196895568) bank187589 +187590 POINT(41.056253173878765 74.62020075343271) bank187590 +187591 POINT(40.185466355509035 73.39624289412907) bank187591 +187592 POINT(40.11517493558582 74.61550518788226) bank187592 +187593 POINT(40.43738762935882 73.80630360559547) bank187593 +187594 POINT(41.54712961791247 74.88183576021322) bank187594 +187595 POINT(40.290373092780456 74.2814506623949) bank187595 +187596 POINT(41.207774169588 74.63481043447007) bank187596 +187597 POINT(40.04755781321965 74.74471670533178) bank187597 +187598 POINT(39.82691252192022 73.56747317873707) bank187598 +187599 POINT(40.57238671140861 74.40798554542704) bank187599 +187600 POINT(40.875743434325386 73.17258574756057) bank187600 +187601 POINT(40.874608941402606 74.63067322161307) bank187601 +187602 POINT(40.45935174461248 74.53312953376381) bank187602 +187603 POINT(40.86951659443555 74.6160719092104) bank187603 +187604 POINT(40.94389594429121 74.71908496941653) bank187604 +187605 POINT(40.54584484125741 74.10683621088924) bank187605 +187606 POINT(40.85442174610614 73.02673594321699) bank187606 +187607 POINT(41.6407537103423 73.25934899269512) bank187607 +187608 POINT(41.196706707846985 73.92773292608459) bank187608 +187609 POINT(41.07682809496657 73.53813717890306) bank187609 +187610 POINT(40.4054248589243 74.92919217478637) bank187610 +187611 POINT(41.28989250489451 74.84722653099108) bank187611 +187612 POINT(41.519923434454796 73.75422605161837) bank187612 +187613 POINT(39.91744224487222 74.11736039504919) bank187613 +187614 POINT(40.83067982683106 73.78717369445043) bank187614 +187615 POINT(41.38442423149742 74.53716163930058) bank187615 +187616 POINT(40.31619944286458 74.72427718494185) bank187616 +187617 POINT(40.01994802467431 74.37667630841716) bank187617 +187618 POINT(40.3912737634798 74.68315069458454) bank187618 +187619 POINT(41.551209804896125 74.68560502014789) bank187619 +187620 POINT(40.52649439304964 74.85288207928001) bank187620 +187621 POINT(41.41702436515553 74.40951045724894) bank187621 +187622 POINT(40.772557593960876 73.78755082386125) bank187622 +187623 POINT(40.3282829919367 74.14053381309878) bank187623 +187624 POINT(39.92368844869071 73.40310954967252) bank187624 +187625 POINT(40.15049286146953 73.33751306387626) bank187625 +187626 POINT(39.78643926222994 74.44400826904736) bank187626 +187627 POINT(41.08180542573198 74.60590883873982) bank187627 +187628 POINT(40.071691820316225 73.9456310281308) bank187628 +187629 POINT(40.95467805393632 73.5793835599687) bank187629 +187630 POINT(41.141946239725705 73.8207370749157) bank187630 +187631 POINT(39.80451033350297 74.43725540590803) bank187631 +187632 POINT(41.57101605476431 73.796822985994) bank187632 +187633 POINT(41.360387452200875 74.03212899682478) bank187633 +187634 POINT(41.3638642825813 73.76025312889524) bank187634 +187635 POINT(40.296568996749336 74.83869466247158) bank187635 +187636 POINT(40.85986542749195 74.66608905315022) bank187636 +187637 POINT(40.55023789561921 74.00155179844215) bank187637 +187638 POINT(40.77833479130041 74.86606896097396) bank187638 +187639 POINT(41.16025432157476 73.95530795143354) bank187639 +187640 POINT(40.328389458576474 74.32568134161023) bank187640 +187641 POINT(41.42651814418835 74.33172171458523) bank187641 +187642 POINT(39.99427764918451 73.64002846752186) bank187642 +187643 POINT(41.15772629494675 74.72502219900932) bank187643 +187644 POINT(40.8403170135599 74.73394663144218) bank187644 +187645 POINT(41.19885316706925 74.23757521211462) bank187645 +187646 POINT(41.24238640493277 74.30228643001227) bank187646 +187647 POINT(41.5407088335861 74.44012878147991) bank187647 +187648 POINT(40.12564094129849 73.00861597665663) bank187648 +187649 POINT(41.43366731275717 73.40260990557472) bank187649 +187650 POINT(41.37908445122258 73.77377506572556) bank187650 +187651 POINT(40.69576006164401 74.82468712015137) bank187651 +187652 POINT(39.95043327095257 74.93698819893913) bank187652 +187653 POINT(41.42955629267988 73.08307560363716) bank187653 +187654 POINT(41.30006853682523 73.25244913283629) bank187654 +187655 POINT(39.83200831662568 74.83432693513663) bank187655 +187656 POINT(39.81232878968848 74.14531549639386) bank187656 +187657 POINT(41.05436532623622 73.10279697640676) bank187657 +187658 POINT(40.8142116098521 73.06730626347301) bank187658 +187659 POINT(39.72539206195119 74.78063628556804) bank187659 +187660 POINT(40.576679948243445 73.74484336953) bank187660 +187661 POINT(41.09216892239131 73.03099399292931) bank187661 +187662 POINT(40.773215559179526 74.81689874096249) bank187662 +187663 POINT(40.69858818820951 74.62893482319875) bank187663 +187664 POINT(40.07917161475583 73.84264833063942) bank187664 +187665 POINT(40.18476325956434 74.91566181867911) bank187665 +187666 POINT(41.58859969483778 73.12425099644487) bank187666 +187667 POINT(40.55239387082645 74.57642233950466) bank187667 +187668 POINT(40.75974215964179 74.32780561282249) bank187668 +187669 POINT(40.621325038631575 74.0626815729002) bank187669 +187670 POINT(41.11837628476257 73.21397759866318) bank187670 +187671 POINT(40.332792555231464 74.65125220937266) bank187671 +187672 POINT(40.982026992234154 73.77975510702147) bank187672 +187673 POINT(40.253765600874345 74.3198936410602) bank187673 +187674 POINT(40.48417806954865 73.94805615415258) bank187674 +187675 POINT(40.71421548826969 73.3864351384864) bank187675 +187676 POINT(41.24338024548535 74.03195690960088) bank187676 +187677 POINT(41.31055266373042 73.9289895055215) bank187677 +187678 POINT(40.739371255480854 73.09718470440568) bank187678 +187679 POINT(40.059354346096086 74.24100276767857) bank187679 +187680 POINT(41.49011476466708 74.63780698610671) bank187680 +187681 POINT(41.68171112751892 74.11851072287861) bank187681 +187682 POINT(40.89964685560251 74.34797183246629) bank187682 +187683 POINT(40.96446922433701 74.4298243499579) bank187683 +187684 POINT(40.2715779573913 73.22289417376513) bank187684 +187685 POINT(41.187264504442815 73.1664865036912) bank187685 +187686 POINT(41.309684419316895 74.74084031724583) bank187686 +187687 POINT(40.76232261723444 74.13615714035387) bank187687 +187688 POINT(40.48463001919556 74.5118250279129) bank187688 +187689 POINT(41.15968504163714 73.97573486850114) bank187689 +187690 POINT(40.109687918921196 74.37127167320975) bank187690 +187691 POINT(40.31255092632739 73.16252804712445) bank187691 +187692 POINT(39.96893029049253 74.66649714990126) bank187692 +187693 POINT(39.860773985842826 74.83289696184468) bank187693 +187694 POINT(41.08347056322841 74.79307758173157) bank187694 +187695 POINT(40.67893801162997 73.4604682632083) bank187695 +187696 POINT(41.48705598999536 73.33504871261098) bank187696 +187697 POINT(41.48658736261556 73.50552875254208) bank187697 +187698 POINT(41.66335626450579 73.95878864801743) bank187698 +187699 POINT(39.81995313683544 73.85466862388272) bank187699 +187700 POINT(39.91605535176258 74.43588370284412) bank187700 +187701 POINT(40.97945068840851 73.03990202417829) bank187701 +187702 POINT(40.79191133137988 73.56920931841674) bank187702 +187703 POINT(40.9858407571191 73.87575600750127) bank187703 +187704 POINT(39.88682772265176 73.15894712199226) bank187704 +187705 POINT(41.64834547090609 74.21627398303501) bank187705 +187706 POINT(40.798161712920546 74.11792058696064) bank187706 +187707 POINT(40.040956457381505 74.91095706407421) bank187707 +187708 POINT(40.813904700551454 73.12811371537514) bank187708 +187709 POINT(40.3614728561898 74.50848048771098) bank187709 +187710 POINT(41.06456767244881 73.37309443892029) bank187710 +187711 POINT(41.66518885202387 74.94313882242692) bank187711 +187712 POINT(39.73299184262577 74.09861892109261) bank187712 +187713 POINT(41.605233719748476 74.14107795719416) bank187713 +187714 POINT(40.59046749224925 73.7509323155211) bank187714 +187715 POINT(40.19321469571769 73.72316181368659) bank187715 +187716 POINT(40.175310701118285 74.56027097627859) bank187716 +187717 POINT(40.906415711650524 73.18185554487084) bank187717 +187718 POINT(40.00498164602176 74.24645310308388) bank187718 +187719 POINT(40.4562795714876 74.06484531547564) bank187719 +187720 POINT(39.847719757329024 74.45739002079286) bank187720 +187721 POINT(40.528534415645204 73.69328414304339) bank187721 +187722 POINT(40.060495471404884 73.44855377800455) bank187722 +187723 POINT(39.895103699229075 73.1518719259309) bank187723 +187724 POINT(40.35146236122679 74.63993897641066) bank187724 +187725 POINT(39.7430341497279 73.24340320682957) bank187725 +187726 POINT(40.60840840756974 73.10557111407859) bank187726 +187727 POINT(40.332852378714634 73.0179623968221) bank187727 +187728 POINT(41.506470238941574 73.8738220023456) bank187728 +187729 POINT(41.37469781190363 74.89293352724664) bank187729 +187730 POINT(41.46521236751895 74.9008315400148) bank187730 +187731 POINT(40.66114557804264 73.03165760849517) bank187731 +187732 POINT(40.70652224687086 73.19243620846943) bank187732 +187733 POINT(40.5791064668131 73.97783306711875) bank187733 +187734 POINT(40.38403320380006 73.93191525074734) bank187734 +187735 POINT(41.44265577470864 73.09360559525851) bank187735 +187736 POINT(41.63304210560352 74.33021528017694) bank187736 +187737 POINT(40.47163806395731 74.32079589459016) bank187737 +187738 POINT(41.196061438142934 73.36659222900455) bank187738 +187739 POINT(41.643045974535035 74.03887891144124) bank187739 +187740 POINT(41.63438691699991 74.64343863812518) bank187740 +187741 POINT(40.53113227133627 73.55117109193874) bank187741 +187742 POINT(40.46458602380448 73.11142566872932) bank187742 +187743 POINT(41.340489576098776 73.83129025654121) bank187743 +187744 POINT(40.30162823299971 74.9364510413422) bank187744 +187745 POINT(41.32219525177919 74.67699382333949) bank187745 +187746 POINT(40.159472323758706 73.62572996265884) bank187746 +187747 POINT(40.30456003390819 73.98313996821295) bank187747 +187748 POINT(41.48298577440994 74.79549827275655) bank187748 +187749 POINT(40.606925058641785 74.33111072982031) bank187749 +187750 POINT(40.759558914699866 73.37627180528025) bank187750 +187751 POINT(41.47918741987142 74.67418037106425) bank187751 +187752 POINT(40.727080626443296 74.96915045450085) bank187752 +187753 POINT(40.44638461917206 74.45849849228712) bank187753 +187754 POINT(40.661849780940365 73.77394853225094) bank187754 +187755 POINT(40.92939963307152 73.44203265649288) bank187755 +187756 POINT(40.27498399884667 73.10529034537034) bank187756 +187757 POINT(40.05680869137123 74.15044509196215) bank187757 +187758 POINT(41.330523872649046 74.01373102789498) bank187758 +187759 POINT(39.748923081694045 74.83115870207985) bank187759 +187760 POINT(39.79355302283461 73.656747711494) bank187760 +187761 POINT(41.06632332575685 73.49300488320766) bank187761 +187762 POINT(40.31613303218037 74.51171015712096) bank187762 +187763 POINT(39.851274772528924 74.04977083894855) bank187763 +187764 POINT(41.28660855686673 74.37888866747608) bank187764 +187765 POINT(40.51719469907499 74.89826021003233) bank187765 +187766 POINT(39.92292968361221 74.39376689311187) bank187766 +187767 POINT(39.90297830217169 74.85483693606041) bank187767 +187768 POINT(41.039098340777414 73.02080575810363) bank187768 +187769 POINT(41.65187549291788 74.7802242624552) bank187769 +187770 POINT(40.18064339033073 73.11051465758341) bank187770 +187771 POINT(40.98612459970339 74.37606202472007) bank187771 +187772 POINT(40.63098441069428 75.00155400762569) bank187772 +187773 POINT(40.43341014694621 73.13373473444686) bank187773 +187774 POINT(41.376363920239996 74.07095384193163) bank187774 +187775 POINT(41.29207995242127 74.03590431286986) bank187775 +187776 POINT(39.89126123885641 73.02119556934136) bank187776 +187777 POINT(40.66394675692282 73.48434540621744) bank187777 +187778 POINT(40.3660532121376 74.64898881689079) bank187778 +187779 POINT(41.56189009945977 73.92351300286948) bank187779 +187780 POINT(40.99065999512203 74.30202095291676) bank187780 +187781 POINT(41.68814582165733 73.40086288688595) bank187781 +187782 POINT(40.722835969825084 73.77362488601125) bank187782 +187783 POINT(41.52509015462918 74.85491736573616) bank187783 +187784 POINT(41.63604683359016 74.85507656066584) bank187784 +187785 POINT(40.6361027159778 74.05632452237293) bank187785 +187786 POINT(41.05740456625831 73.04881894934898) bank187786 +187787 POINT(41.52356629569503 74.98711604541438) bank187787 +187788 POINT(41.38734213757793 74.14895540343666) bank187788 +187789 POINT(40.430847766432734 73.19484437701458) bank187789 +187790 POINT(39.99365798923694 73.52027292199125) bank187790 +187791 POINT(39.733209713779345 74.95032720714583) bank187791 +187792 POINT(40.18892805700874 74.30437328841408) bank187792 +187793 POINT(40.159384417635266 74.59289692364543) bank187793 +187794 POINT(40.35239190259909 73.61813728576132) bank187794 +187795 POINT(39.71516028858207 74.68308592389016) bank187795 +187796 POINT(41.443861965711065 73.78358407089853) bank187796 +187797 POINT(40.361661580349306 74.06991181120588) bank187797 +187798 POINT(40.04292430717417 74.38452376673575) bank187798 +187799 POINT(40.55435960709126 73.90300186804978) bank187799 +187800 POINT(40.38872959227299 73.62835659138717) bank187800 +187801 POINT(40.60320268219669 74.19101208847414) bank187801 +187802 POINT(40.46410608183784 73.9942331912854) bank187802 +187803 POINT(41.31759621423994 74.41174844410578) bank187803 +187804 POINT(40.65949419802772 74.3247162798834) bank187804 +187805 POINT(40.321806555958986 74.56980636450993) bank187805 +187806 POINT(40.013647463223094 74.75345544583668) bank187806 +187807 POINT(41.01953279060197 73.78166798266575) bank187807 +187808 POINT(41.62953006489113 73.01536865057066) bank187808 +187809 POINT(40.03236286754852 74.45435387386009) bank187809 +187810 POINT(41.43651231956248 73.45713053360808) bank187810 +187811 POINT(40.157833226999124 74.67679717548891) bank187811 +187812 POINT(40.28432953668055 73.5491627322364) bank187812 +187813 POINT(40.37485345365202 74.28484096531155) bank187813 +187814 POINT(41.22606610660724 74.63185627289225) bank187814 +187815 POINT(40.508345165330894 74.77780485017583) bank187815 +187816 POINT(41.26108494040409 74.26773361443809) bank187816 +187817 POINT(40.252377522334875 73.32550100701107) bank187817 +187818 POINT(40.19987090860102 73.42127300337937) bank187818 +187819 POINT(39.73913706082059 74.8317444245883) bank187819 +187820 POINT(40.78665204940801 73.51524135505413) bank187820 +187821 POINT(40.21583691326616 73.48199354257807) bank187821 +187822 POINT(40.20698072109441 74.74285778019585) bank187822 +187823 POINT(39.82272744913387 73.41801596661637) bank187823 +187824 POINT(41.472430959350355 73.33131851764934) bank187824 +187825 POINT(40.99973426603751 73.73971276204452) bank187825 +187826 POINT(40.193477553575704 74.6539648988741) bank187826 +187827 POINT(40.44837820539709 74.59345332884703) bank187827 +187828 POINT(41.129545772069186 74.26355673817565) bank187828 +187829 POINT(41.02022257067999 74.33772756472743) bank187829 +187830 POINT(40.72115640349526 73.59000660756207) bank187830 +187831 POINT(40.07132831168567 74.18008917629554) bank187831 +187832 POINT(40.15609673103962 73.96212730156543) bank187832 +187833 POINT(41.114503684168234 74.10255212701956) bank187833 +187834 POINT(41.689737989417516 73.81293440533118) bank187834 +187835 POINT(40.719416191953606 74.22951942617577) bank187835 +187836 POINT(39.89961046920283 74.28150461613109) bank187836 +187837 POINT(41.33271549417668 74.3204264228262) bank187837 +187838 POINT(40.2110489107423 73.80510105249581) bank187838 +187839 POINT(40.646984914983214 73.96175289960858) bank187839 +187840 POINT(41.6038297471562 73.18147126870323) bank187840 +187841 POINT(40.77347380462594 73.87744115658468) bank187841 +187842 POINT(39.82070729587085 73.79058904970141) bank187842 +187843 POINT(40.04332382627256 74.0504912735598) bank187843 +187844 POINT(40.74021875098059 73.42811952962364) bank187844 +187845 POINT(40.4496210715207 73.16782560351218) bank187845 +187846 POINT(40.36720769927843 74.22667387877298) bank187846 +187847 POINT(40.92410842928062 74.03519771980021) bank187847 +187848 POINT(40.23274526343491 74.00485374537372) bank187848 +187849 POINT(40.19631686300296 74.4903886454294) bank187849 +187850 POINT(41.47166754138133 73.92513498496923) bank187850 +187851 POINT(41.15498406468181 74.86290880999925) bank187851 +187852 POINT(39.853956440085966 73.66314335778303) bank187852 +187853 POINT(40.11262640808805 73.6298399411712) bank187853 +187854 POINT(40.92969771854839 73.10004865679899) bank187854 +187855 POINT(40.719640760306376 74.65996110627638) bank187855 +187856 POINT(40.717125289281306 73.31309001610352) bank187856 +187857 POINT(41.05434740357384 74.61892937383087) bank187857 +187858 POINT(40.92142964501225 74.63739703664638) bank187858 +187859 POINT(41.32892264687578 73.7849137202494) bank187859 +187860 POINT(41.70249570708646 74.08143102383947) bank187860 +187861 POINT(39.86761229091977 74.9440374294767) bank187861 +187862 POINT(40.62801615529715 74.65532167709523) bank187862 +187863 POINT(41.65414901739323 74.86498396820622) bank187863 +187864 POINT(40.082131728518796 74.211527376461) bank187864 +187865 POINT(40.01980167162055 74.47002035351252) bank187865 +187866 POINT(40.75737228635424 74.63877453569951) bank187866 +187867 POINT(41.66798605301866 73.75751986330371) bank187867 +187868 POINT(40.286268633754695 74.2113985547221) bank187868 +187869 POINT(40.42049371361136 74.70824195273217) bank187869 +187870 POINT(39.80423344164886 74.2667192844412) bank187870 +187871 POINT(41.69750927434245 73.6164371327907) bank187871 +187872 POINT(41.1968609020126 74.17502881139282) bank187872 +187873 POINT(40.222808210500794 73.9691312086808) bank187873 +187874 POINT(40.527249212034405 73.13315923901756) bank187874 +187875 POINT(40.26477431907184 73.12911312104255) bank187875 +187876 POINT(39.9794559270556 74.81101892705392) bank187876 +187877 POINT(40.348649346133755 74.5945062104175) bank187877 +187878 POINT(41.6008024417923 74.87212088388412) bank187878 +187879 POINT(40.522376970964054 73.50569343986031) bank187879 +187880 POINT(40.76073910969645 74.83573719277761) bank187880 +187881 POINT(39.9716443238328 73.72961864348922) bank187881 +187882 POINT(40.97335111635691 73.68979157242462) bank187882 +187883 POINT(40.71564303017246 74.25409473044056) bank187883 +187884 POINT(41.13824065743581 73.4501044201116) bank187884 +187885 POINT(41.08311342946799 74.90927609699325) bank187885 +187886 POINT(40.49367104072361 73.8186009116802) bank187886 +187887 POINT(40.176483187990755 74.78456757946027) bank187887 +187888 POINT(40.38879890458894 74.78700259908399) bank187888 +187889 POINT(41.03834768566851 74.11675313758488) bank187889 +187890 POINT(41.43376130816437 73.10465569325568) bank187890 +187891 POINT(40.97196604794706 73.35229738153181) bank187891 +187892 POINT(40.15494804626188 74.22650860233702) bank187892 +187893 POINT(41.17113276492717 73.89300294156753) bank187893 +187894 POINT(41.00685258681972 73.73837733518882) bank187894 +187895 POINT(40.88392083941286 73.19926573359014) bank187895 +187896 POINT(41.377805759122516 73.15821993143317) bank187896 +187897 POINT(40.0869241195358 74.8455571563525) bank187897 +187898 POINT(41.537578105969835 74.17975584245484) bank187898 +187899 POINT(40.37977428842549 73.0900070744556) bank187899 +187900 POINT(40.16252855843987 74.82256335160031) bank187900 +187901 POINT(41.64701874762243 73.92323645217479) bank187901 +187902 POINT(40.94780841896906 73.53080253053268) bank187902 +187903 POINT(39.94495567064702 74.86789515189002) bank187903 +187904 POINT(40.633620199670574 74.13841454029681) bank187904 +187905 POINT(40.69346936543138 73.32851213482377) bank187905 +187906 POINT(39.82959573191622 74.03414935315574) bank187906 +187907 POINT(41.054071176585396 73.41146179759114) bank187907 +187908 POINT(40.42037029044947 73.69492835870876) bank187908 +187909 POINT(40.215828280387804 73.49972969004219) bank187909 +187910 POINT(40.13305324449732 74.73292023728683) bank187910 +187911 POINT(40.644902695232744 74.6068498242172) bank187911 +187912 POINT(40.81263131605254 74.34645022483659) bank187912 +187913 POINT(41.32740964038687 74.09678168001675) bank187913 +187914 POINT(40.76834840247445 73.50978054724135) bank187914 +187915 POINT(40.29869734758694 74.78841303748622) bank187915 +187916 POINT(40.03376788476426 73.19919514326074) bank187916 +187917 POINT(40.52979779861067 73.84096648680358) bank187917 +187918 POINT(39.900188313750085 73.91217193452837) bank187918 +187919 POINT(39.89183150497901 74.38758076353557) bank187919 +187920 POINT(41.531846266481594 73.38640847554696) bank187920 +187921 POINT(41.47779701099469 73.731615018748) bank187921 +187922 POINT(40.052635751912675 73.41010243089646) bank187922 +187923 POINT(39.98920675836732 73.99887477271112) bank187923 +187924 POINT(40.63093787417549 73.45814647481191) bank187924 +187925 POINT(40.82521662008391 74.78717358283872) bank187925 +187926 POINT(40.635402413481415 74.55174271514271) bank187926 +187927 POINT(41.569637227074296 74.9200608227946) bank187927 +187928 POINT(41.15914670848053 74.37893903055968) bank187928 +187929 POINT(41.506511105175235 74.34669612866601) bank187929 +187930 POINT(39.8004153639397 74.5721267754702) bank187930 +187931 POINT(41.42409727696879 73.34519742809799) bank187931 +187932 POINT(41.50448599444725 74.7053095866324) bank187932 +187933 POINT(41.53028113852578 74.34356988370008) bank187933 +187934 POINT(40.312504851476135 73.67626137840212) bank187934 +187935 POINT(40.334231969675585 73.73376093964431) bank187935 +187936 POINT(41.247628268962465 73.72512395301644) bank187936 +187937 POINT(39.84378027867251 74.73664566180254) bank187937 +187938 POINT(41.3219169401578 74.95688603009665) bank187938 +187939 POINT(41.56070948441289 73.87313488946651) bank187939 +187940 POINT(40.09404197781024 74.33922775387776) bank187940 +187941 POINT(39.73074854152405 73.96291131441416) bank187941 +187942 POINT(40.35091128011659 73.22199241250766) bank187942 +187943 POINT(40.37667585918351 74.8751036414208) bank187943 +187944 POINT(40.60156703143428 74.2830904220904) bank187944 +187945 POINT(41.06623985594144 73.51886836622569) bank187945 +187946 POINT(40.348998669241695 73.04825238034456) bank187946 +187947 POINT(39.855370849408295 73.17256984124104) bank187947 +187948 POINT(41.164915359455165 74.84255044552887) bank187948 +187949 POINT(40.3232105831348 73.22252656435487) bank187949 +187950 POINT(40.23676854348625 74.96030456600421) bank187950 +187951 POINT(40.39883902203821 73.09912163608809) bank187951 +187952 POINT(39.713324508143984 74.51836499232103) bank187952 +187953 POINT(40.71918330611164 74.18855197376175) bank187953 +187954 POINT(39.820617316126885 73.1806618821649) bank187954 +187955 POINT(40.64539477720397 74.55549360239436) bank187955 +187956 POINT(40.543844530869144 73.50476919078848) bank187956 +187957 POINT(40.245752156737296 73.41720212252649) bank187957 +187958 POINT(41.10010814775422 74.48168499109376) bank187958 +187959 POINT(39.98260832645232 74.32745327053334) bank187959 +187960 POINT(40.91258935019597 74.29208382342382) bank187960 +187961 POINT(41.077014632714366 73.33606079107386) bank187961 +187962 POINT(41.68283996207241 73.91824245872527) bank187962 +187963 POINT(39.91481796160493 73.98418126752364) bank187963 +187964 POINT(40.052059920448976 73.27664343626302) bank187964 +187965 POINT(40.62955388481372 74.2551051053175) bank187965 +187966 POINT(40.20938066020253 74.27287613092192) bank187966 +187967 POINT(40.07456330806603 73.74725425000345) bank187967 +187968 POINT(39.98286039684911 73.67972433562883) bank187968 +187969 POINT(40.172844583017906 73.32121613724792) bank187969 +187970 POINT(40.80036693388105 73.1238762431135) bank187970 +187971 POINT(40.47615910646974 74.24769479000884) bank187971 +187972 POINT(40.43586737327583 74.69949886022346) bank187972 +187973 POINT(40.46565185988015 74.51166587797307) bank187973 +187974 POINT(41.18899182455429 73.30984633744697) bank187974 +187975 POINT(40.20668799686095 73.30660700529636) bank187975 +187976 POINT(40.87235108491312 74.5224446541371) bank187976 +187977 POINT(41.16016818502305 74.12487797029104) bank187977 +187978 POINT(41.00171304986463 73.40340450942422) bank187978 +187979 POINT(40.11466601347875 73.07857737076759) bank187979 +187980 POINT(40.16549987103001 74.64003174265811) bank187980 +187981 POINT(41.42885531249004 73.29825941960817) bank187981 +187982 POINT(39.82946894616438 73.23054314553231) bank187982 +187983 POINT(40.00927172153331 73.70137451444484) bank187983 +187984 POINT(40.59568389415929 73.31162932929543) bank187984 +187985 POINT(39.773428043797665 73.58729741439315) bank187985 +187986 POINT(39.71821970029516 74.45222866988786) bank187986 +187987 POINT(41.291706709952095 74.66529394677053) bank187987 +187988 POINT(40.98855747401975 73.20214508112967) bank187988 +187989 POINT(40.85171569443101 74.64277232919869) bank187989 +187990 POINT(41.42259232280657 73.47476474344542) bank187990 +187991 POINT(39.74118022285904 74.56709831915539) bank187991 +187992 POINT(41.37913116089172 74.13177308510427) bank187992 +187993 POINT(39.932160877454365 73.81191033422891) bank187993 +187994 POINT(39.73803096278028 74.25749907941814) bank187994 +187995 POINT(40.74554583367772 74.60093652660822) bank187995 +187996 POINT(39.75679806493383 74.03861397785668) bank187996 +187997 POINT(41.58626585750079 74.56621099394197) bank187997 +187998 POINT(40.242315964585636 73.84237733243565) bank187998 +187999 POINT(40.775032927907034 74.25736478629689) bank187999 +188000 POINT(41.154529082376925 73.35669579390122) bank188000 +188001 POINT(40.12557700951782 73.74536267855791) bank188001 +188002 POINT(40.17937094669634 74.18653191457162) bank188002 +188003 POINT(40.95799855911122 74.33581181149509) bank188003 +188004 POINT(39.949149389721065 74.54727468795107) bank188004 +188005 POINT(40.94653226651 74.54791330893181) bank188005 +188006 POINT(40.52396257823192 73.1858358982853) bank188006 +188007 POINT(41.1410531157909 74.46110822407628) bank188007 +188008 POINT(39.83687209232231 74.934621957478) bank188008 +188009 POINT(40.31164931110752 73.07994988309052) bank188009 +188010 POINT(41.43222686379477 73.8283074962264) bank188010 +188011 POINT(40.83012785619111 73.15859965042812) bank188011 +188012 POINT(40.42451119003007 73.71982281444878) bank188012 +188013 POINT(40.037296096733776 73.59244641949421) bank188013 +188014 POINT(40.2614087802939 74.51202744760387) bank188014 +188015 POINT(40.66235590421142 73.74995215743466) bank188015 +188016 POINT(40.9401402666908 73.29293274477344) bank188016 +188017 POINT(40.693083076190135 74.51060965741432) bank188017 +188018 POINT(40.90335161524726 73.72951248371042) bank188018 +188019 POINT(41.570773247317696 74.8377715511833) bank188019 +188020 POINT(40.86188476748435 73.53039364697459) bank188020 +188021 POINT(41.57682239050111 73.46821466759533) bank188021 +188022 POINT(40.8376335361228 73.25684984495865) bank188022 +188023 POINT(40.74945588101551 74.62683581790074) bank188023 +188024 POINT(40.25575986512878 73.3217709606373) bank188024 +188025 POINT(39.75255046221328 74.65371455455485) bank188025 +188026 POINT(40.83476219162436 73.6808569970063) bank188026 +188027 POINT(39.89779580682711 74.46947076260398) bank188027 +188028 POINT(41.336907562649216 74.0948471886234) bank188028 +188029 POINT(39.945831333144874 74.14751148307178) bank188029 +188030 POINT(41.05802243840882 74.16234950111395) bank188030 +188031 POINT(39.903577250100994 73.90421142446924) bank188031 +188032 POINT(41.406887401841125 74.36554330094795) bank188032 +188033 POINT(39.76611923319144 73.90791420463239) bank188033 +188034 POINT(40.50778366639325 74.20089393925898) bank188034 +188035 POINT(40.831076114327345 73.4853712769559) bank188035 +188036 POINT(40.092301476206174 73.46948440671765) bank188036 +188037 POINT(40.52781183272465 73.93171375571262) bank188037 +188038 POINT(39.73626045126663 73.54669346915728) bank188038 +188039 POINT(41.45621868376479 73.9259721009906) bank188039 +188040 POINT(40.69032810715078 73.6747086838274) bank188040 +188041 POINT(41.42357665180769 73.15369110263755) bank188041 +188042 POINT(41.33870076015282 73.98157783291026) bank188042 +188043 POINT(40.48434267350266 74.667768654149) bank188043 +188044 POINT(40.37994336470718 73.59947334450978) bank188044 +188045 POINT(41.000151511744754 74.92837432832285) bank188045 +188046 POINT(41.534720443708125 73.63977242068862) bank188046 +188047 POINT(41.255921567112935 73.05393312960058) bank188047 +188048 POINT(40.60691012699738 74.90476887415457) bank188048 +188049 POINT(40.29378976032527 74.7863705421456) bank188049 +188050 POINT(41.61127759420809 74.22195879075161) bank188050 +188051 POINT(40.049835566177585 73.97298857446312) bank188051 +188052 POINT(41.19690224184514 74.65362303648593) bank188052 +188053 POINT(41.246382254526985 73.54467324734264) bank188053 +188054 POINT(40.57514230244351 73.74881298084415) bank188054 +188055 POINT(41.29616366591743 73.51094637268424) bank188055 +188056 POINT(40.06122316170846 74.57761615124505) bank188056 +188057 POINT(40.81692626389566 73.01065937930464) bank188057 +188058 POINT(40.96466654196427 73.05913287791033) bank188058 +188059 POINT(40.5181142481723 74.78815078725287) bank188059 +188060 POINT(41.2969034028443 74.09374864164894) bank188060 +188061 POINT(40.06980176973547 73.64973925680836) bank188061 +188062 POINT(41.698426616488824 74.13237145011385) bank188062 +188063 POINT(41.585593677701574 74.52028838092431) bank188063 +188064 POINT(41.10762304784674 73.365670759184) bank188064 +188065 POINT(39.74226315390843 73.23700872044037) bank188065 +188066 POINT(41.565931403458194 74.95758439093579) bank188066 +188067 POINT(40.463144966257275 74.51661218087963) bank188067 +188068 POINT(40.92500525478271 73.57527674157704) bank188068 +188069 POINT(40.121573623852626 73.45605365190671) bank188069 +188070 POINT(40.641890001847045 74.18265421663013) bank188070 +188071 POINT(41.46095612414906 74.77036939054088) bank188071 +188072 POINT(40.81550230622694 74.06809416301043) bank188072 +188073 POINT(41.45178240128549 74.30317876862132) bank188073 +188074 POINT(40.01243869661605 73.49186640739846) bank188074 +188075 POINT(40.82208679646583 73.54688188708454) bank188075 +188076 POINT(41.39909062836442 73.01442085750755) bank188076 +188077 POINT(41.62071562204799 73.12945546269177) bank188077 +188078 POINT(40.169326075725785 74.31840006168093) bank188078 +188079 POINT(40.60253122787444 73.29347307145285) bank188079 +188080 POINT(41.49340107466145 74.47927578099221) bank188080 +188081 POINT(39.99441856736537 73.1427361531562) bank188081 +188082 POINT(39.98321590119396 73.07787505472643) bank188082 +188083 POINT(40.99657987149078 74.23875527203518) bank188083 +188084 POINT(40.24592225544666 74.1744731857338) bank188084 +188085 POINT(40.78148664754651 74.86390103708193) bank188085 +188086 POINT(40.79628387796418 73.39624492713997) bank188086 +188087 POINT(39.957121438131 73.4703800605828) bank188087 +188088 POINT(39.795557450450936 73.27993187509172) bank188088 +188089 POINT(40.95179838162049 73.49403516074287) bank188089 +188090 POINT(39.74068469721949 74.90482624322094) bank188090 +188091 POINT(41.673873568612336 73.5847755819732) bank188091 +188092 POINT(40.58642321433635 74.46494304749356) bank188092 +188093 POINT(41.08883662340466 74.95223925354982) bank188093 +188094 POINT(41.010174378518606 73.48037312093773) bank188094 +188095 POINT(40.05480321384287 74.70213764228977) bank188095 +188096 POINT(41.52864385905455 74.94930069996117) bank188096 +188097 POINT(41.411291691155085 74.96224286087788) bank188097 +188098 POINT(40.613801397248594 74.16216759052502) bank188098 +188099 POINT(41.35149058604156 74.79776886725958) bank188099 +188100 POINT(39.8837796848233 73.08169271970743) bank188100 +188101 POINT(39.909806257044984 73.7936554343713) bank188101 +188102 POINT(40.95377006929184 74.13728950082172) bank188102 +188103 POINT(39.812503568250015 73.3627346144802) bank188103 +188104 POINT(40.762650901848055 73.74361348584252) bank188104 +188105 POINT(40.465337624892854 73.98014379548205) bank188105 +188106 POINT(39.98771967876618 73.10426907072085) bank188106 +188107 POINT(40.095794619477175 73.65806464532031) bank188107 +188108 POINT(40.85684093400475 74.80850397752957) bank188108 +188109 POINT(40.951284934844395 74.42177538469615) bank188109 +188110 POINT(40.31161595188677 73.82707634206365) bank188110 +188111 POINT(40.308072634767946 73.16363184550167) bank188111 +188112 POINT(41.677835366087734 73.25801110296378) bank188112 +188113 POINT(41.35283494066052 73.66654845197239) bank188113 +188114 POINT(40.8916069486548 74.12351533561974) bank188114 +188115 POINT(41.25490473063652 74.86257915025261) bank188115 +188116 POINT(41.38769000624406 74.30929175868164) bank188116 +188117 POINT(41.46287467339724 74.6690971535927) bank188117 +188118 POINT(41.51752908576701 73.09645523705694) bank188118 +188119 POINT(40.04268039936697 74.39084462093179) bank188119 +188120 POINT(39.904073028645236 73.98114505851838) bank188120 +188121 POINT(40.41776958987224 74.04398011370465) bank188121 +188122 POINT(40.54478554690641 73.52167492776805) bank188122 +188123 POINT(41.29181755679836 74.61997034548597) bank188123 +188124 POINT(40.64594699404341 73.87922163191146) bank188124 +188125 POINT(40.686484748262835 73.60933236928665) bank188125 +188126 POINT(39.92443192302474 74.06945258145926) bank188126 +188127 POINT(41.53237979496296 74.74057864569389) bank188127 +188128 POINT(41.59612394641606 74.88289294914148) bank188128 +188129 POINT(41.46435334938306 74.69041666251952) bank188129 +188130 POINT(40.63491982328106 74.87415915012996) bank188130 +188131 POINT(39.73352149984487 74.77506612843862) bank188131 +188132 POINT(41.65111899516811 73.88203097820755) bank188132 +188133 POINT(41.69272487240576 73.11003524843409) bank188133 +188134 POINT(39.85621774144206 73.43399801326859) bank188134 +188135 POINT(40.44789050262643 73.79297218659795) bank188135 +188136 POINT(40.441935612289036 74.96192949537719) bank188136 +188137 POINT(40.16080529687127 74.30796040151687) bank188137 +188138 POINT(41.409528734191746 74.00442271537833) bank188138 +188139 POINT(40.65513882697427 73.46614090787159) bank188139 +188140 POINT(40.6653093516127 73.42975481895341) bank188140 +188141 POINT(41.49463122076315 74.38365906354326) bank188141 +188142 POINT(40.37340996200098 73.64465462138868) bank188142 +188143 POINT(40.294267935956476 73.41223285822325) bank188143 +188144 POINT(39.965523331228134 73.70877817761311) bank188144 +188145 POINT(41.23780529269361 73.12452771923647) bank188145 +188146 POINT(40.028080354352305 73.70959164944723) bank188146 +188147 POINT(40.67581061360116 74.73330097644943) bank188147 +188148 POINT(40.9161912057974 74.37613877642166) bank188148 +188149 POINT(40.83724149018589 73.72654121764255) bank188149 +188150 POINT(40.891217742049875 74.84516166878205) bank188150 +188151 POINT(40.83797505107573 73.47687954155278) bank188151 +188152 POINT(41.7070522942155 74.100889192083) bank188152 +188153 POINT(40.944469232419756 74.53654647507342) bank188153 +188154 POINT(40.560415759730915 74.11356129868317) bank188154 +188155 POINT(41.683091488841 73.48182098471933) bank188155 +188156 POINT(41.414272271728464 73.33116130750405) bank188156 +188157 POINT(41.506352431927276 73.94634016531255) bank188157 +188158 POINT(41.31618815288629 73.3243318574615) bank188158 +188159 POINT(40.489734473991874 73.73532121000441) bank188159 +188160 POINT(40.67641380468158 73.19968284700576) bank188160 +188161 POINT(41.374721044454844 74.52011384468604) bank188161 +188162 POINT(41.21218217923605 74.24846937793079) bank188162 +188163 POINT(40.69231711647296 74.45768311343645) bank188163 +188164 POINT(40.23673028828903 74.87502341522656) bank188164 +188165 POINT(40.55691569122973 73.16065542187998) bank188165 +188166 POINT(40.00415912792529 73.70603747331754) bank188166 +188167 POINT(41.32806370101178 74.00952995588527) bank188167 +188168 POINT(40.872549568216584 73.49981087629853) bank188168 +188169 POINT(40.453929777662495 73.91366607346649) bank188169 +188170 POINT(40.609061303231236 73.93867731605341) bank188170 +188171 POINT(41.67647870508479 73.72626082532273) bank188171 +188172 POINT(40.80622539450932 74.73947557741803) bank188172 +188173 POINT(41.10376699052061 73.87882505331278) bank188173 +188174 POINT(40.82869773072707 74.49549727759023) bank188174 +188175 POINT(41.213368677089804 74.09089273556788) bank188175 +188176 POINT(41.12787066497257 73.58843403367226) bank188176 +188177 POINT(40.261704778158695 74.16834484250364) bank188177 +188178 POINT(41.6414478811209 73.2072358596574) bank188178 +188179 POINT(41.38436164260607 74.37677023140432) bank188179 +188180 POINT(40.98580357292413 73.07239838252244) bank188180 +188181 POINT(40.04789313024119 73.65895840490407) bank188181 +188182 POINT(40.32147361463749 74.98507227818271) bank188182 +188183 POINT(41.45972803992057 74.36246626639878) bank188183 +188184 POINT(39.98985493143683 74.46615066197575) bank188184 +188185 POINT(41.32881127811001 74.88512530612108) bank188185 +188186 POINT(41.338738459557575 73.55805902858094) bank188186 +188187 POINT(40.91085636807045 73.40084827557688) bank188187 +188188 POINT(41.103365148569296 74.86236540268392) bank188188 +188189 POINT(40.567743600117886 73.30133066254469) bank188189 +188190 POINT(41.08572854212117 73.48503211651796) bank188190 +188191 POINT(40.138765683337695 73.40540058599385) bank188191 +188192 POINT(40.237343189306614 74.39270236677503) bank188192 +188193 POINT(41.57493406853672 74.21912089434188) bank188193 +188194 POINT(40.305326435417065 74.76032466952209) bank188194 +188195 POINT(40.55900564594292 73.46883913262099) bank188195 +188196 POINT(41.49499732765587 73.51129004292565) bank188196 +188197 POINT(39.75584357437225 73.87567314773827) bank188197 +188198 POINT(41.487566699237775 74.02273341742765) bank188198 +188199 POINT(40.05837454997891 73.62721547377235) bank188199 +188200 POINT(40.93828978623656 74.60041132016194) bank188200 +188201 POINT(41.113232603851 73.0285170769186) bank188201 +188202 POINT(40.897947084033206 74.11282397198875) bank188202 +188203 POINT(40.27416964846553 73.32878250327164) bank188203 +188204 POINT(40.136641191071924 74.53750501850114) bank188204 +188205 POINT(41.46189637459836 74.62921115279477) bank188205 +188206 POINT(40.204709171898365 73.29219514991328) bank188206 +188207 POINT(41.01079336869482 73.47513423040002) bank188207 +188208 POINT(41.22664908621098 73.60825886043709) bank188208 +188209 POINT(39.89331514019236 74.24400205130945) bank188209 +188210 POINT(40.824789668157194 74.39236475098309) bank188210 +188211 POINT(41.018220588097876 73.68794556098938) bank188211 +188212 POINT(40.202826424229556 73.86658747969186) bank188212 +188213 POINT(39.80146934609292 73.00743687759696) bank188213 +188214 POINT(40.14845812310908 74.89484462464632) bank188214 +188215 POINT(39.79671615009291 73.36637128190816) bank188215 +188216 POINT(41.19222166426445 73.67500426979836) bank188216 +188217 POINT(40.77607089254743 73.79742730292239) bank188217 +188218 POINT(39.979397345967996 73.03042261624051) bank188218 +188219 POINT(41.007488558910936 74.23557964234469) bank188219 +188220 POINT(40.81036746585523 73.47568796042832) bank188220 +188221 POINT(39.96320279417275 73.75682987915881) bank188221 +188222 POINT(40.73001558254472 73.57991674426609) bank188222 +188223 POINT(39.98125179206168 73.04575761695757) bank188223 +188224 POINT(41.66224848721598 73.57396630865254) bank188224 +188225 POINT(41.218821182141724 73.70827625789414) bank188225 +188226 POINT(41.56115771985477 74.16722713389879) bank188226 +188227 POINT(41.69267273190731 73.20172697437553) bank188227 +188228 POINT(40.097417314175765 73.83337779707959) bank188228 +188229 POINT(39.827639268057716 74.51517840299881) bank188229 +188230 POINT(40.405706200265115 74.99097323659294) bank188230 +188231 POINT(40.19621465675942 73.32970245871752) bank188231 +188232 POINT(41.63558923791114 74.92358560152331) bank188232 +188233 POINT(40.00986044229068 74.90594758808784) bank188233 +188234 POINT(40.561598522319336 74.4186233261953) bank188234 +188235 POINT(40.44894786561749 73.1436475003119) bank188235 +188236 POINT(41.44754556082131 74.22451407469016) bank188236 +188237 POINT(41.559901730250296 73.95388920750376) bank188237 +188238 POINT(41.243621027343565 73.89574073546359) bank188238 +188239 POINT(40.287453864745686 74.4808717136042) bank188239 +188240 POINT(40.27504898316898 73.05983389237807) bank188240 +188241 POINT(40.49738396349809 74.29595996508415) bank188241 +188242 POINT(41.17095663806082 73.41567950513239) bank188242 +188243 POINT(39.868080656563805 73.17493649304652) bank188243 +188244 POINT(39.717106529734345 74.46424364464868) bank188244 +188245 POINT(41.275981109932054 74.139241307387) bank188245 +188246 POINT(40.098865805625394 74.85483834500728) bank188246 +188247 POINT(40.96254702241392 73.59389412072542) bank188247 +188248 POINT(40.479613771151364 74.57214383988189) bank188248 +188249 POINT(40.01532701947332 73.76134550925059) bank188249 +188250 POINT(40.4358882428608 73.9748840045877) bank188250 +188251 POINT(40.61586397959574 73.99110540364204) bank188251 +188252 POINT(40.663783970520164 73.35247501557173) bank188252 +188253 POINT(40.71413578210276 74.78408523925502) bank188253 +188254 POINT(40.953341247371554 73.89656473947294) bank188254 +188255 POINT(40.01076485384974 74.58721365902765) bank188255 +188256 POINT(41.23139938712441 74.53652545987201) bank188256 +188257 POINT(41.20512446745032 74.77535962921185) bank188257 +188258 POINT(40.69189658649748 73.59914748122665) bank188258 +188259 POINT(39.92013361254718 73.55171293076565) bank188259 +188260 POINT(40.56688046741543 74.66596497132718) bank188260 +188261 POINT(40.46267309946945 73.82947432520338) bank188261 +188262 POINT(41.24709542856657 74.52337889295313) bank188262 +188263 POINT(41.1560304203023 73.33239340881117) bank188263 +188264 POINT(40.992673646996444 73.04807010689315) bank188264 +188265 POINT(41.36709252870384 74.86734681657435) bank188265 +188266 POINT(39.927811351904005 74.15642404133783) bank188266 +188267 POINT(39.80634125770101 74.19347533050019) bank188267 +188268 POINT(41.214202143380106 74.97546628676471) bank188268 +188269 POINT(40.246185836374124 73.70681773857694) bank188269 +188270 POINT(41.238064665716315 74.91638133813959) bank188270 +188271 POINT(39.980983409074014 74.51624492591235) bank188271 +188272 POINT(39.89106934088194 74.58974150579733) bank188272 +188273 POINT(40.57611937185868 73.8125411750887) bank188273 +188274 POINT(41.55741869579238 74.15425915112206) bank188274 +188275 POINT(39.9335814696183 73.54333259088662) bank188275 +188276 POINT(40.53579925586975 74.3978972408832) bank188276 +188277 POINT(40.18009568846503 73.13505383851435) bank188277 +188278 POINT(39.8359276090687 73.29819614493329) bank188278 +188279 POINT(40.60065968759296 74.28980217137136) bank188279 +188280 POINT(41.315217426911694 74.52583030680489) bank188280 +188281 POINT(40.86750818743855 73.08640044241821) bank188281 +188282 POINT(41.5946648939926 74.97393458726634) bank188282 +188283 POINT(41.053511395763906 74.42961699248724) bank188283 +188284 POINT(40.06503481734444 74.27932506135382) bank188284 +188285 POINT(41.26787214304571 74.42850610437736) bank188285 +188286 POINT(41.14194687907711 74.26660588486904) bank188286 +188287 POINT(39.83238761660067 74.67324841136899) bank188287 +188288 POINT(40.56481511615282 74.53639517661631) bank188288 +188289 POINT(40.95380734762844 73.36386300971601) bank188289 +188290 POINT(40.936015711227085 74.34986225160357) bank188290 +188291 POINT(40.12999882967168 73.2594048521603) bank188291 +188292 POINT(40.67024800548939 73.79395254900336) bank188292 +188293 POINT(39.81439149489326 73.46502142821907) bank188293 +188294 POINT(39.73723761188692 74.92835620467517) bank188294 +188295 POINT(40.94742004180607 73.31196459599815) bank188295 +188296 POINT(40.56081878473573 73.05599773319871) bank188296 +188297 POINT(40.825143470690804 73.48651193380158) bank188297 +188298 POINT(41.01211585044181 73.33184113950631) bank188298 +188299 POINT(41.28184641543021 74.36336351416368) bank188299 +188300 POINT(40.234545257310394 73.75721726737989) bank188300 +188301 POINT(39.85566218124443 73.36483285659054) bank188301 +188302 POINT(41.34055880577403 74.17955712058671) bank188302 +188303 POINT(39.800481382255136 73.6211321174783) bank188303 +188304 POINT(40.796632724686 74.78644875819896) bank188304 +188305 POINT(41.586723536995066 74.12611595320395) bank188305 +188306 POINT(41.04971073457115 73.46379747342388) bank188306 +188307 POINT(40.16137174114389 74.60512831655177) bank188307 +188308 POINT(40.23361953379459 73.88447987987198) bank188308 +188309 POINT(41.541698811356596 73.72876907032374) bank188309 +188310 POINT(41.29094483555083 74.26516511687892) bank188310 +188311 POINT(40.72737905222444 73.81882047499599) bank188311 +188312 POINT(41.35993812454436 74.81890690720076) bank188312 +188313 POINT(41.618008048208246 73.14029613078512) bank188313 +188314 POINT(40.5329970210588 74.19598925859646) bank188314 +188315 POINT(40.243638888671406 73.8755949623588) bank188315 +188316 POINT(40.13465490870084 73.81039827980852) bank188316 +188317 POINT(41.26387924642699 73.46830878830013) bank188317 +188318 POINT(40.75502064390764 73.94699131291557) bank188318 +188319 POINT(41.348703454576196 74.11332124831526) bank188319 +188320 POINT(41.68873327229517 74.34967854288168) bank188320 +188321 POINT(40.237078137491146 73.27364363249154) bank188321 +188322 POINT(39.80599417695733 74.39920517777566) bank188322 +188323 POINT(41.128114226111066 74.85434043897243) bank188323 +188324 POINT(41.283512834288636 74.87759035755951) bank188324 +188325 POINT(40.30437052066753 74.50297622938756) bank188325 +188326 POINT(41.63436208532179 73.2231880744093) bank188326 +188327 POINT(40.85355752812612 73.55815338111833) bank188327 +188328 POINT(40.49665435076781 74.96010942279268) bank188328 +188329 POINT(41.00571659708146 74.3873185186465) bank188329 +188330 POINT(40.782719548132235 74.07264499657589) bank188330 +188331 POINT(40.77753026366362 74.33841827768981) bank188331 +188332 POINT(41.57869764288801 73.04020161004203) bank188332 +188333 POINT(39.866388769111076 73.2227612700138) bank188333 +188334 POINT(40.39615073193973 73.49312062538031) bank188334 +188335 POINT(41.34353464024198 73.91220506549776) bank188335 +188336 POINT(39.94446776038291 73.87447265703452) bank188336 +188337 POINT(40.3846468851859 74.26843191598964) bank188337 +188338 POINT(41.30037817706544 73.64670924856057) bank188338 +188339 POINT(41.51664378542382 74.36044225255871) bank188339 +188340 POINT(41.346703870492455 73.07751427296587) bank188340 +188341 POINT(41.104300352669604 73.59073069195745) bank188341 +188342 POINT(40.58504699423727 74.73237562727189) bank188342 +188343 POINT(40.19164599702665 73.87673878916384) bank188343 +188344 POINT(40.00365632880971 73.99662574122054) bank188344 +188345 POINT(40.053358243048315 73.72830052067074) bank188345 +188346 POINT(39.7153125966866 73.08880814174063) bank188346 +188347 POINT(39.74830092584523 73.90544652292155) bank188347 +188348 POINT(40.81889493217766 74.02211854618616) bank188348 +188349 POINT(40.057453265863096 74.83914520878851) bank188349 +188350 POINT(40.41759203678171 74.04093512857499) bank188350 +188351 POINT(40.39120937296479 73.24210414419821) bank188351 +188352 POINT(41.35752629637505 74.63477720458958) bank188352 +188353 POINT(39.844386218658606 73.9937675425645) bank188353 +188354 POINT(40.980221297084704 74.90800525243607) bank188354 +188355 POINT(41.202281866941775 74.70242787814063) bank188355 +188356 POINT(41.62061194529967 73.65991912701831) bank188356 +188357 POINT(39.92639899004096 73.11188815024232) bank188357 +188358 POINT(41.02736470320467 73.2671539364202) bank188358 +188359 POINT(41.19496048619798 73.26378514343865) bank188359 +188360 POINT(41.632950842778975 73.34767988439802) bank188360 +188361 POINT(41.5889587929843 74.5998894550329) bank188361 +188362 POINT(41.2551003824629 74.80221066107929) bank188362 +188363 POINT(41.589519171508826 74.75638954688787) bank188363 +188364 POINT(40.669203493941616 74.790690661819) bank188364 +188365 POINT(41.03586012870651 73.91805029005994) bank188365 +188366 POINT(40.88882351710403 74.66344712031241) bank188366 +188367 POINT(40.79026437976031 74.80483118471595) bank188367 +188368 POINT(40.58858393190426 74.9598053509399) bank188368 +188369 POINT(41.295961782522106 74.869630420339) bank188369 +188370 POINT(41.58553597951159 74.83768807090904) bank188370 +188371 POINT(40.265646027878574 73.39085647475791) bank188371 +188372 POINT(41.340460708770344 74.670332754709) bank188372 +188373 POINT(40.121787479508136 74.50335208626935) bank188373 +188374 POINT(40.05974486541633 73.97408329559329) bank188374 +188375 POINT(39.92194811845187 74.03699016181999) bank188375 +188376 POINT(41.29474286420356 74.26787969391259) bank188376 +188377 POINT(40.928980423349984 74.02198309731948) bank188377 +188378 POINT(40.140097212889046 73.43687660590702) bank188378 +188379 POINT(41.57264562102591 74.16200223735095) bank188379 +188380 POINT(41.09502806759452 73.11669725159655) bank188380 +188381 POINT(41.312279319015325 73.99631332136832) bank188381 +188382 POINT(40.87537037254665 74.45580319390317) bank188382 +188383 POINT(40.67509791987773 73.75435797100523) bank188383 +188384 POINT(40.7568766353113 73.15130767719405) bank188384 +188385 POINT(41.22838210751914 73.21747740320521) bank188385 +188386 POINT(41.261274077455525 73.98390929206488) bank188386 +188387 POINT(41.57556724366985 74.39109255572279) bank188387 +188388 POINT(41.593392894203596 73.73307064038161) bank188388 +188389 POINT(40.06887234733098 73.48560917831716) bank188389 +188390 POINT(39.869889106587216 74.92100257754043) bank188390 +188391 POINT(41.10695181401682 74.14974177881822) bank188391 +188392 POINT(39.76633305556045 73.87921238128814) bank188392 +188393 POINT(40.43282836764594 73.45036902020979) bank188393 +188394 POINT(41.46157233855226 74.63178602457843) bank188394 +188395 POINT(41.516039999658865 73.67423169661008) bank188395 +188396 POINT(40.34721075936672 73.67419655975624) bank188396 +188397 POINT(40.61840374645899 74.76598194687489) bank188397 +188398 POINT(40.74222456481786 73.40667410920715) bank188398 +188399 POINT(40.5890172689934 74.05520848451678) bank188399 +188400 POINT(40.57189665630237 74.79403601382278) bank188400 +188401 POINT(41.47246771445036 73.51342811710774) bank188401 +188402 POINT(41.38525031240199 73.55765818907412) bank188402 +188403 POINT(40.426047028026225 74.55634475068211) bank188403 +188404 POINT(41.01976425220199 73.12852527014482) bank188404 +188405 POINT(39.91329339665451 73.05604623785258) bank188405 +188406 POINT(40.77685873130571 73.63298661617947) bank188406 +188407 POINT(41.53643930646854 74.00028233862508) bank188407 +188408 POINT(40.51802651525824 73.9313528845838) bank188408 +188409 POINT(41.29336377447947 74.14383442137733) bank188409 +188410 POINT(40.41269840058181 74.76530726549663) bank188410 +188411 POINT(41.151771949978176 74.44705548509646) bank188411 +188412 POINT(41.10275431980326 73.31677845207625) bank188412 +188413 POINT(41.494046356956076 73.8337088338312) bank188413 +188414 POINT(41.47967635817145 73.78045264720889) bank188414 +188415 POINT(41.590096717757305 74.33340215140537) bank188415 +188416 POINT(41.3775735992985 74.57249795621057) bank188416 +188417 POINT(39.7303664858952 73.93427748709698) bank188417 +188418 POINT(40.601786639769884 73.4264909619777) bank188418 +188419 POINT(39.75517323313112 73.93475337314234) bank188419 +188420 POINT(39.913033568899486 74.59405952736844) bank188420 +188421 POINT(41.25364599534128 73.41753498589667) bank188421 +188422 POINT(41.376333215117796 74.89887220609542) bank188422 +188423 POINT(40.588775018987064 73.69605937087275) bank188423 +188424 POINT(40.586887113636685 73.80951736163333) bank188424 +188425 POINT(40.86689109835192 74.22879082645916) bank188425 +188426 POINT(40.2429591362967 73.63585319686584) bank188426 +188427 POINT(40.15778691332615 73.04342326268493) bank188427 +188428 POINT(39.876633440981266 73.79611643112435) bank188428 +188429 POINT(39.856283025382126 73.30995885662382) bank188429 +188430 POINT(40.77406089688124 73.27341037229368) bank188430 +188431 POINT(41.19506801218901 73.44889790918873) bank188431 +188432 POINT(40.41691266788067 74.42696923653817) bank188432 +188433 POINT(40.89164438832588 73.06115173649937) bank188433 +188434 POINT(40.29452317218161 73.3875233707574) bank188434 +188435 POINT(41.65682845251233 74.01622724985712) bank188435 +188436 POINT(39.96518837935195 74.23774124445097) bank188436 +188437 POINT(41.34098216587188 74.76829047905305) bank188437 +188438 POINT(40.76946377262409 74.54891912131589) bank188438 +188439 POINT(41.203934759132565 74.0138526768115) bank188439 +188440 POINT(40.093798418544324 73.78579412654975) bank188440 +188441 POINT(40.01691724393291 73.48922375129172) bank188441 +188442 POINT(41.512772355955406 73.64150368074613) bank188442 +188443 POINT(40.21544327486156 74.7134046801679) bank188443 +188444 POINT(39.75309938084841 74.35050711679378) bank188444 +188445 POINT(41.25858023430214 73.83275084822584) bank188445 +188446 POINT(41.00160877400735 74.79147334237938) bank188446 +188447 POINT(39.92514561038108 73.10785531777239) bank188447 +188448 POINT(40.32903439535694 74.72936575488824) bank188448 +188449 POINT(40.63511426186483 74.58554922465687) bank188449 +188450 POINT(41.011931597974716 73.03439490198465) bank188450 +188451 POINT(40.27795709746515 74.96048904171539) bank188451 +188452 POINT(40.22718076001647 73.3722313817556) bank188452 +188453 POINT(40.395123552092045 73.92410632133455) bank188453 +188454 POINT(39.92597235421106 73.30942487003492) bank188454 +188455 POINT(40.57923004860074 73.7517513308915) bank188455 +188456 POINT(40.63593903514652 73.792836783329) bank188456 +188457 POINT(40.03976321989081 74.0909502854087) bank188457 +188458 POINT(41.643240163312285 73.65926013311586) bank188458 +188459 POINT(40.33243727101343 74.8441446447225) bank188459 +188460 POINT(41.35109762227774 73.96291520358758) bank188460 +188461 POINT(40.504287191272645 74.89215466298782) bank188461 +188462 POINT(40.28393304377275 74.56097101801379) bank188462 +188463 POINT(40.35190249657949 74.10476088110948) bank188463 +188464 POINT(40.061459713239614 73.37617931794335) bank188464 +188465 POINT(41.55547888237622 74.03392410488557) bank188465 +188466 POINT(41.6138735857779 74.0500172069905) bank188466 +188467 POINT(40.6260695332538 74.97101602246387) bank188467 +188468 POINT(41.60672206101788 74.00284022789083) bank188468 +188469 POINT(39.94679347563412 73.35174830750526) bank188469 +188470 POINT(40.17663677319049 74.19784009567951) bank188470 +188471 POINT(41.15921368906674 73.46004493322258) bank188471 +188472 POINT(41.46993336102401 74.22449123214986) bank188472 +188473 POINT(41.60237689403079 73.4926344689577) bank188473 +188474 POINT(40.686329985710145 74.94233486323239) bank188474 +188475 POINT(40.45777234993261 74.73644950271496) bank188475 +188476 POINT(40.06260924394943 74.15838816067699) bank188476 +188477 POINT(40.99117129114325 73.8612918737587) bank188477 +188478 POINT(40.8108088618722 74.37767287343972) bank188478 +188479 POINT(40.928964985691074 74.92126936878276) bank188479 +188480 POINT(40.52504309701748 74.32242471700548) bank188480 +188481 POINT(40.73481136484752 74.5974525733516) bank188481 +188482 POINT(39.98024444278449 74.88340068460862) bank188482 +188483 POINT(40.44636390232761 73.41546848738129) bank188483 +188484 POINT(41.37610629162982 74.86974240050938) bank188484 +188485 POINT(40.67026772484324 74.15633124739749) bank188485 +188486 POINT(40.096224920901506 74.08060582966473) bank188486 +188487 POINT(41.1582524808284 74.03830389466079) bank188487 +188488 POINT(40.65520338380549 74.24552518971562) bank188488 +188489 POINT(40.35210177510601 74.9239421629371) bank188489 +188490 POINT(40.776867447505744 73.52051135778498) bank188490 +188491 POINT(40.46211407140097 74.55006560640297) bank188491 +188492 POINT(40.3346193567396 74.4891451691654) bank188492 +188493 POINT(41.16213772705412 74.42606796876525) bank188493 +188494 POINT(40.366441352598294 74.23824375382786) bank188494 +188495 POINT(41.682520506535226 73.01256352424635) bank188495 +188496 POINT(41.25919284966436 74.01734257549285) bank188496 +188497 POINT(41.59475138712662 73.23841458426577) bank188497 +188498 POINT(39.86676923190926 73.1552943468588) bank188498 +188499 POINT(39.71460466960829 74.40785218451482) bank188499 +188500 POINT(40.442243170622 73.38906945741024) bank188500 +188501 POINT(41.17990115902829 74.76100003770857) bank188501 +188502 POINT(40.88226975518002 74.52925157097853) bank188502 +188503 POINT(39.94296183911678 74.48387061503455) bank188503 +188504 POINT(40.41839447483912 73.35911683823582) bank188504 +188505 POINT(41.52153403485603 73.88463533177732) bank188505 +188506 POINT(40.633999552801896 73.65457375058689) bank188506 +188507 POINT(41.44215874267748 74.06819684863389) bank188507 +188508 POINT(41.29526617096029 73.26764737974267) bank188508 +188509 POINT(40.81753489256481 74.67622939003479) bank188509 +188510 POINT(41.157008593934236 74.41969986763091) bank188510 +188511 POINT(41.70170435293947 74.58951170487613) bank188511 +188512 POINT(41.20650731704258 73.70178473648203) bank188512 +188513 POINT(40.04663011787405 73.9386807806332) bank188513 +188514 POINT(40.236076998015434 74.71407322080398) bank188514 +188515 POINT(41.03797702902639 73.4425963353325) bank188515 +188516 POINT(39.86115836373471 74.87065474341792) bank188516 +188517 POINT(40.370746617064775 74.42782774988612) bank188517 +188518 POINT(41.59183915926527 73.78853088854022) bank188518 +188519 POINT(40.65536791607869 73.80424874978301) bank188519 +188520 POINT(40.384055331451904 73.460328588323) bank188520 +188521 POINT(41.160355254848874 73.15616413040702) bank188521 +188522 POINT(40.550307855923606 73.29827468410673) bank188522 +188523 POINT(40.228786005520845 73.76860380064791) bank188523 +188524 POINT(39.728575068619364 73.32897484910687) bank188524 +188525 POINT(40.53886358898918 73.27326904937343) bank188525 +188526 POINT(40.1875537056636 74.35268127677689) bank188526 +188527 POINT(40.16792323449763 73.0321010485503) bank188527 +188528 POINT(41.54374395290998 73.73250723225016) bank188528 +188529 POINT(40.693227577489985 73.92151272709663) bank188529 +188530 POINT(40.69426805790596 73.54607188873278) bank188530 +188531 POINT(40.46915372299173 73.53053693518441) bank188531 +188532 POINT(41.1958304102488 74.22395201288305) bank188532 +188533 POINT(41.548145610885925 73.01763068220563) bank188533 +188534 POINT(41.50078943438007 73.17895609269517) bank188534 +188535 POINT(41.510031950420704 73.20929914078809) bank188535 +188536 POINT(41.473791400741455 74.5716512192117) bank188536 +188537 POINT(40.35866156645336 73.70290669198259) bank188537 +188538 POINT(40.88614775184302 73.12702432131297) bank188538 +188539 POINT(40.902581405642536 74.58585379343617) bank188539 +188540 POINT(40.77363200403791 74.61510549606875) bank188540 +188541 POINT(40.25223334613329 74.33913099338008) bank188541 +188542 POINT(41.08617844832516 74.01222946561296) bank188542 +188543 POINT(40.379544034620956 74.03858794122552) bank188543 +188544 POINT(41.44377005341798 74.09907589051915) bank188544 +188545 POINT(40.31438202063099 73.28968734240861) bank188545 +188546 POINT(40.61109343537977 74.55495980276565) bank188546 +188547 POINT(41.65768716671709 74.75587930360666) bank188547 +188548 POINT(41.033536768825314 74.14248242255039) bank188548 +188549 POINT(40.76648553863298 73.84461485746637) bank188549 +188550 POINT(39.76870141997427 74.61046177976318) bank188550 +188551 POINT(40.46695490766623 73.53945106422957) bank188551 +188552 POINT(39.82352925146513 73.87937932020947) bank188552 +188553 POINT(41.67394704088313 74.60770140275466) bank188553 +188554 POINT(41.69387673435736 73.79823647079333) bank188554 +188555 POINT(40.12700446280868 73.16750782200279) bank188555 +188556 POINT(40.28378687898563 73.82369735264608) bank188556 +188557 POINT(40.075205163241975 73.28606448991187) bank188557 +188558 POINT(40.66208155183085 73.07536446508584) bank188558 +188559 POINT(41.329753115056896 73.46554891734688) bank188559 +188560 POINT(41.30235126901518 74.63125471840335) bank188560 +188561 POINT(41.552604871072504 73.79069379845491) bank188561 +188562 POINT(40.56972133495389 73.27309815876437) bank188562 +188563 POINT(40.34174913420557 73.97475676773091) bank188563 +188564 POINT(41.40369841579119 73.90871588412638) bank188564 +188565 POINT(40.01359794391058 74.19775192808068) bank188565 +188566 POINT(41.539898632458 74.92790246546973) bank188566 +188567 POINT(39.920521143859986 73.28304309976272) bank188567 +188568 POINT(41.69070864262603 74.92627629138222) bank188568 +188569 POINT(41.168248903527605 73.31027171831978) bank188569 +188570 POINT(41.21365843725377 74.05288215301029) bank188570 +188571 POINT(40.22883667411904 73.47633418203343) bank188571 +188572 POINT(41.49377903074411 74.50616791693322) bank188572 +188573 POINT(39.98066311883956 74.25770507894963) bank188573 +188574 POINT(40.03072995919138 73.08245652992804) bank188574 +188575 POINT(40.412146625889676 74.48876902281452) bank188575 +188576 POINT(40.78673518680469 74.39860155442058) bank188576 +188577 POINT(41.0009201689074 73.63455314688034) bank188577 +188578 POINT(41.351104245308406 73.07018933069564) bank188578 +188579 POINT(40.049970217437476 74.67957192414563) bank188579 +188580 POINT(40.91723254368352 74.54148533041615) bank188580 +188581 POINT(40.079212088929424 74.4429341200916) bank188581 +188582 POINT(40.19069448473991 73.65829361099775) bank188582 +188583 POINT(41.35412406798721 74.89193390523074) bank188583 +188584 POINT(40.68859210278509 74.94169422823286) bank188584 +188585 POINT(40.2939957627486 74.48957325871504) bank188585 +188586 POINT(41.343744925331485 73.63673440925469) bank188586 +188587 POINT(41.24362750207651 74.10506013216332) bank188587 +188588 POINT(40.90203098973258 73.28613829343192) bank188588 +188589 POINT(41.188056139196505 73.84002042989948) bank188589 +188590 POINT(41.4666324393643 74.31616410063553) bank188590 +188591 POINT(40.93077003555438 74.63970795415203) bank188591 +188592 POINT(41.21522717275978 74.16583693634766) bank188592 +188593 POINT(40.755271914669954 73.96671576116093) bank188593 +188594 POINT(40.425636543808444 74.35631350356871) bank188594 +188595 POINT(40.579278644015226 73.48444864981505) bank188595 +188596 POINT(40.48029634353907 73.32553471040507) bank188596 +188597 POINT(40.77840420369618 74.4248806536967) bank188597 +188598 POINT(41.34897618792378 73.71722657186244) bank188598 +188599 POINT(41.35218998317831 74.20506221989652) bank188599 +188600 POINT(41.5599535735626 74.97504793801585) bank188600 +188601 POINT(40.582335969295244 74.1822807086296) bank188601 +188602 POINT(40.283832473563024 74.51290990075499) bank188602 +188603 POINT(41.68421637874186 74.22682930869733) bank188603 +188604 POINT(40.884960265300194 74.90209588096444) bank188604 +188605 POINT(40.80883536060413 73.67206100905078) bank188605 +188606 POINT(41.22728263031943 74.04768599677155) bank188606 +188607 POINT(40.72029037306249 73.82937070645042) bank188607 +188608 POINT(40.688099590568356 74.53928180636271) bank188608 +188609 POINT(41.07603966840794 74.05419044242414) bank188609 +188610 POINT(40.75212638874593 74.77277583724546) bank188610 +188611 POINT(39.82710706428603 74.53693818635834) bank188611 +188612 POINT(40.08551819373817 73.10003063494388) bank188612 +188613 POINT(40.303834043913696 74.38609709258292) bank188613 +188614 POINT(39.77368004958276 75.00018568014643) bank188614 +188615 POINT(40.03859913820272 74.83611432383444) bank188615 +188616 POINT(39.85539005500464 74.58048278890776) bank188616 +188617 POINT(40.20325899852755 74.38483651260957) bank188617 +188618 POINT(39.950285135707674 74.88767377795624) bank188618 +188619 POINT(41.53248611549613 73.63700792759221) bank188619 +188620 POINT(40.83844410413413 73.9468125108716) bank188620 +188621 POINT(40.228312933207526 73.50244653717972) bank188621 +188622 POINT(40.67820615396779 75.00520867222912) bank188622 +188623 POINT(41.45442056242415 73.04712207260773) bank188623 +188624 POINT(40.49999287032099 73.37752597495232) bank188624 +188625 POINT(41.49645113480916 74.91798775614053) bank188625 +188626 POINT(41.296021309261974 73.15779589330282) bank188626 +188627 POINT(41.23018208314715 74.61730135497035) bank188627 +188628 POINT(41.60350509178087 74.23351868879267) bank188628 +188629 POINT(41.052069805972565 73.17761308406325) bank188629 +188630 POINT(41.08784924727669 73.54096105441869) bank188630 +188631 POINT(40.23791268981054 73.05249325209672) bank188631 +188632 POINT(40.700441779056554 73.86670866562972) bank188632 +188633 POINT(39.86101772287184 74.44887763457281) bank188633 +188634 POINT(41.4236292752767 74.9745489901758) bank188634 +188635 POINT(41.456625061969284 73.20021789153287) bank188635 +188636 POINT(41.150191911662176 74.16917912675967) bank188636 +188637 POINT(41.187336724475195 73.39425586466297) bank188637 +188638 POINT(40.061329773588284 74.73517965721247) bank188638 +188639 POINT(39.873448909161745 73.4058778201278) bank188639 +188640 POINT(40.44083121744267 73.51781467755451) bank188640 +188641 POINT(41.08178487933208 73.63201978621315) bank188641 +188642 POINT(40.30741269266415 74.52067998271899) bank188642 +188643 POINT(41.20341066418635 73.57922026564464) bank188643 +188644 POINT(40.59995089768863 74.86554940095316) bank188644 +188645 POINT(41.41655239457723 74.6247344408926) bank188645 +188646 POINT(41.324942867080175 73.75148510308182) bank188646 +188647 POINT(40.57857556511145 73.73955091447975) bank188647 +188648 POINT(41.017115408579386 73.94711915701825) bank188648 +188649 POINT(41.00111841996367 74.00631445796418) bank188649 +188650 POINT(40.03531230809569 73.92092594413447) bank188650 +188651 POINT(40.48011040880515 74.81648213108465) bank188651 +188652 POINT(41.48562112573034 74.7342407933309) bank188652 +188653 POINT(41.49314760770329 74.0465923387687) bank188653 +188654 POINT(40.170356636558935 74.65044517630005) bank188654 +188655 POINT(39.914548094442885 74.31099979130987) bank188655 +188656 POINT(39.75832439023983 74.56276061652285) bank188656 +188657 POINT(40.29363287246598 74.4892284171576) bank188657 +188658 POINT(39.7290683267826 74.80480787734935) bank188658 +188659 POINT(41.37874333229523 73.16473641700718) bank188659 +188660 POINT(40.53115577126461 74.21737156761831) bank188660 +188661 POINT(41.0018359356894 74.97403518576557) bank188661 +188662 POINT(41.349236457139895 73.12450831307994) bank188662 +188663 POINT(41.104813359724766 74.14025375375988) bank188663 +188664 POINT(40.612627335360564 73.97431754860646) bank188664 +188665 POINT(41.18579896041561 73.22754249246702) bank188665 +188666 POINT(40.216854212931715 73.75196076073453) bank188666 +188667 POINT(40.59739790133444 74.55170587227558) bank188667 +188668 POINT(41.04455025031865 73.38131106990699) bank188668 +188669 POINT(41.270578668582154 74.77331244718913) bank188669 +188670 POINT(40.81991499463606 73.58373857013251) bank188670 +188671 POINT(41.51960941941085 73.33592937014012) bank188671 +188672 POINT(39.8820727555232 74.62836729505166) bank188672 +188673 POINT(40.825071949141226 74.85133667158387) bank188673 +188674 POINT(41.631677797564215 74.0032164671804) bank188674 +188675 POINT(40.045674379256454 74.863304752777) bank188675 +188676 POINT(39.83778137420159 74.03601564793932) bank188676 +188677 POINT(41.23267732320824 73.24919695543373) bank188677 +188678 POINT(40.28578036641652 73.70205531265701) bank188678 +188679 POINT(40.85578428519129 74.95347134234329) bank188679 +188680 POINT(41.55796631922256 73.65965795706472) bank188680 +188681 POINT(39.995353627114646 73.80678363555201) bank188681 +188682 POINT(41.070547677181594 73.65416859001678) bank188682 +188683 POINT(39.80740016544698 74.45551426156243) bank188683 +188684 POINT(40.41558892017753 73.30381392458236) bank188684 +188685 POINT(40.69156559004302 74.50271762177579) bank188685 +188686 POINT(41.70501827028006 73.90678290204352) bank188686 +188687 POINT(41.363071407033765 74.41758581764023) bank188687 +188688 POINT(41.0806193782823 74.97317496068952) bank188688 +188689 POINT(41.154443315081934 73.8187422337361) bank188689 +188690 POINT(40.16726340748068 74.7958577463436) bank188690 +188691 POINT(40.57654893121599 73.72231618431746) bank188691 +188692 POINT(41.05360217028684 74.2092912968575) bank188692 +188693 POINT(41.232711078645124 73.7368705474757) bank188693 +188694 POINT(41.55579646511035 74.93212639927772) bank188694 +188695 POINT(40.412637232840225 74.03814455248835) bank188695 +188696 POINT(41.68257891820154 74.68911562361886) bank188696 +188697 POINT(41.44319816844962 74.18743362695179) bank188697 +188698 POINT(40.4701899789248 73.99906038730768) bank188698 +188699 POINT(39.86539480748572 73.58334612458768) bank188699 +188700 POINT(40.88847310530267 74.92608483331338) bank188700 +188701 POINT(40.751874087349165 73.97958153599579) bank188701 +188702 POINT(40.24520006052164 74.35578669627967) bank188702 +188703 POINT(39.973510055392616 74.49181159475516) bank188703 +188704 POINT(40.483707064783225 73.4216414586103) bank188704 +188705 POINT(39.717734595712216 74.94544178640474) bank188705 +188706 POINT(40.322399530208095 73.51927282702222) bank188706 +188707 POINT(40.66087581823664 74.45806655543542) bank188707 +188708 POINT(40.139951105886645 73.54029234949591) bank188708 +188709 POINT(39.96651852144813 74.78722523596292) bank188709 +188710 POINT(40.20288510148566 74.34163493422479) bank188710 +188711 POINT(40.782242057475536 74.78792274518652) bank188711 +188712 POINT(41.65392504778127 73.07713934352714) bank188712 +188713 POINT(41.55260635855032 74.68159535633917) bank188713 +188714 POINT(41.069716645184656 74.13643195171825) bank188714 +188715 POINT(40.919287846827714 74.8557252224601) bank188715 +188716 POINT(39.99600696516114 73.08272948899452) bank188716 +188717 POINT(41.15744238142224 73.59541557035924) bank188717 +188718 POINT(41.652468884019726 74.67516517657279) bank188718 +188719 POINT(40.4396866264656 74.86711342031782) bank188719 +188720 POINT(41.642825362335834 73.22281154051643) bank188720 +188721 POINT(40.46850727410649 74.60644541905128) bank188721 +188722 POINT(40.45930871468876 74.0015685099762) bank188722 +188723 POINT(41.58685225720729 74.44339125273129) bank188723 +188724 POINT(41.108898824727326 73.04337365818049) bank188724 +188725 POINT(39.87028245045264 74.37522613880095) bank188725 +188726 POINT(39.749914021735115 74.75196817729432) bank188726 +188727 POINT(40.665342170277754 74.12377785475216) bank188727 +188728 POINT(41.11207430923533 74.18883374261618) bank188728 +188729 POINT(40.68412069438866 73.11277810396658) bank188729 +188730 POINT(40.98364615268947 74.52942964024565) bank188730 +188731 POINT(41.07762923570067 73.01564331427933) bank188731 +188732 POINT(40.74657742284976 73.7567482917051) bank188732 +188733 POINT(41.227317115056785 74.19938193979104) bank188733 +188734 POINT(40.40183304129822 74.79546848924068) bank188734 +188735 POINT(41.331195956711106 74.46801529930792) bank188735 +188736 POINT(40.13016087893095 73.19765110517365) bank188736 +188737 POINT(40.29165704084046 74.6490442029972) bank188737 +188738 POINT(41.293984759751204 74.31726392669441) bank188738 +188739 POINT(41.30686625597421 74.68452505700134) bank188739 +188740 POINT(41.47723792822569 73.44591903323463) bank188740 +188741 POINT(40.70996751783704 73.35343924940834) bank188741 +188742 POINT(39.99285510918364 73.73150004462474) bank188742 +188743 POINT(40.88026254345222 74.7107067166415) bank188743 +188744 POINT(40.259982476600975 74.69222937150663) bank188744 +188745 POINT(41.51411580730741 73.52416189364745) bank188745 +188746 POINT(39.8072139203894 73.62011930572427) bank188746 +188747 POINT(41.33354727836739 74.42378415471516) bank188747 +188748 POINT(40.78501352641642 73.71160740196473) bank188748 +188749 POINT(40.76708020556058 73.43543444037006) bank188749 +188750 POINT(40.905785942326006 73.6030786454057) bank188750 +188751 POINT(39.747561005395454 73.43950013448814) bank188751 +188752 POINT(40.41513083047797 74.4764698151486) bank188752 +188753 POINT(40.19937924325213 73.77928993389607) bank188753 +188754 POINT(41.37701554457057 73.59765041330617) bank188754 +188755 POINT(40.384679845217285 73.16736666754682) bank188755 +188756 POINT(41.40843834966033 74.01258005265277) bank188756 +188757 POINT(40.59435794987887 73.46121871141953) bank188757 +188758 POINT(39.83341596539561 74.4270746729382) bank188758 +188759 POINT(39.77919571159729 74.76513482758186) bank188759 +188760 POINT(40.289201622156284 73.31121941720531) bank188760 +188761 POINT(40.02313696735898 73.68297506508392) bank188761 +188762 POINT(41.4451876059348 74.14667457897762) bank188762 +188763 POINT(40.43766276408355 73.09396970254612) bank188763 +188764 POINT(40.002888699886775 73.90834178465273) bank188764 +188765 POINT(41.343751794557484 74.12508902204361) bank188765 +188766 POINT(40.41239192369977 74.95148510122401) bank188766 +188767 POINT(40.14729285491466 74.17000409783469) bank188767 +188768 POINT(40.509897839885 74.27429924965143) bank188768 +188769 POINT(40.12016971677572 73.39303168558503) bank188769 +188770 POINT(41.23664162587175 74.62190824184535) bank188770 +188771 POINT(40.8368505675705 74.28133929368042) bank188771 +188772 POINT(41.368426571184564 73.15552147768301) bank188772 +188773 POINT(41.38027734464393 74.69902849085464) bank188773 +188774 POINT(41.535521010892374 74.90004989015634) bank188774 +188775 POINT(40.767223924855124 74.85720471587759) bank188775 +188776 POINT(41.697975094232305 74.73188721472734) bank188776 +188777 POINT(40.07886683075735 73.73878310705375) bank188777 +188778 POINT(40.075016773206066 74.93189478183429) bank188778 +188779 POINT(41.70616577978036 74.74379007546254) bank188779 +188780 POINT(40.281615159551 74.01531942928212) bank188780 +188781 POINT(39.90191155797157 74.67777882175744) bank188781 +188782 POINT(39.78584158696838 73.03041038442232) bank188782 +188783 POINT(39.84492312125544 74.3375065783683) bank188783 +188784 POINT(40.77175523632424 74.6484401099949) bank188784 +188785 POINT(41.31756222796597 74.40736475434505) bank188785 +188786 POINT(40.707443562462736 73.04344003291929) bank188786 +188787 POINT(41.64951275181782 74.58764970936782) bank188787 +188788 POINT(39.84655189515254 74.35368455577105) bank188788 +188789 POINT(41.368916759708924 74.51515442363674) bank188789 +188790 POINT(41.28979538749752 74.60165923632667) bank188790 +188791 POINT(39.753958515180976 73.36949591952636) bank188791 +188792 POINT(41.20836001996048 73.64891231956949) bank188792 +188793 POINT(41.48298121021777 74.94070195023761) bank188793 +188794 POINT(40.56835171346841 73.51577758322891) bank188794 +188795 POINT(40.680682669049446 73.62535716544919) bank188795 +188796 POINT(40.72281894626635 74.01937857289347) bank188796 +188797 POINT(41.57934500089798 73.73684055199166) bank188797 +188798 POINT(40.554936745322586 73.80810147309901) bank188798 +188799 POINT(40.86542744138671 74.3969285395457) bank188799 +188800 POINT(40.74636543147256 74.91388862343275) bank188800 +188801 POINT(40.501381499533245 74.97649447606295) bank188801 +188802 POINT(40.420888037240324 74.50153313208344) bank188802 +188803 POINT(40.489878544712354 74.21149836099481) bank188803 +188804 POINT(40.20769698067392 73.29724271693352) bank188804 +188805 POINT(41.64827705537506 73.452881620014) bank188805 +188806 POINT(41.01648479470363 73.34040201413933) bank188806 +188807 POINT(40.370572406139715 74.70927275211672) bank188807 +188808 POINT(41.02814232274272 73.60175646821445) bank188808 +188809 POINT(40.241093150851235 73.71918199614937) bank188809 +188810 POINT(40.71106605529594 74.11120921750498) bank188810 +188811 POINT(41.30021882702107 74.51448605386629) bank188811 +188812 POINT(41.62168771980644 74.34218805037403) bank188812 +188813 POINT(40.50406114685401 73.46432945082377) bank188813 +188814 POINT(39.9399745854802 74.56071844928405) bank188814 +188815 POINT(40.15248714987372 74.92128018011874) bank188815 +188816 POINT(40.788964173650996 73.15665788400307) bank188816 +188817 POINT(39.942074856613175 74.00512671129432) bank188817 +188818 POINT(40.00158446324592 74.17695790509968) bank188818 +188819 POINT(41.38784549511183 73.01727900388079) bank188819 +188820 POINT(41.26305172448255 74.92697618842702) bank188820 +188821 POINT(39.94692176481526 73.0949980602319) bank188821 +188822 POINT(40.43357804005316 73.20202523823666) bank188822 +188823 POINT(40.644966273978746 73.34333366622123) bank188823 +188824 POINT(40.561376341983674 74.84567714898823) bank188824 +188825 POINT(40.23150518225739 73.18593203881984) bank188825 +188826 POINT(41.0564875505558 74.01633048798996) bank188826 +188827 POINT(39.97981603087662 73.51926805378812) bank188827 +188828 POINT(40.75293455779607 74.07675690027436) bank188828 +188829 POINT(41.03950600612693 74.03984499890606) bank188829 +188830 POINT(40.69360096856555 73.9997632316405) bank188830 +188831 POINT(40.14217376900366 74.83065335266949) bank188831 +188832 POINT(41.31053956818369 74.9883587693611) bank188832 +188833 POINT(40.87529341754877 74.91182480677504) bank188833 +188834 POINT(40.11639891518083 73.43272608644698) bank188834 +188835 POINT(40.69035455452479 73.3704605703462) bank188835 +188836 POINT(40.07659369542544 74.26304744065442) bank188836 +188837 POINT(40.373107895131604 73.60543547348176) bank188837 +188838 POINT(40.69153430624606 73.72091030678556) bank188838 +188839 POINT(40.512382679455676 74.65139415945428) bank188839 +188840 POINT(40.44315022128265 73.4659165926342) bank188840 +188841 POINT(41.1640020750748 74.58592747019236) bank188841 +188842 POINT(40.51986477875957 73.31994855665212) bank188842 +188843 POINT(41.18464899617584 73.30154992487032) bank188843 +188844 POINT(39.985220471463684 73.02312703364846) bank188844 +188845 POINT(39.94763737944756 73.4831290254719) bank188845 +188846 POINT(39.86998093808023 74.42222348860217) bank188846 +188847 POINT(39.88147018115457 74.55048279900284) bank188847 +188848 POINT(41.675273645375334 74.62926738046926) bank188848 +188849 POINT(40.719231870785976 74.347642552632) bank188849 +188850 POINT(40.69840005514101 73.57389452440592) bank188850 +188851 POINT(40.20393029575222 73.31861927594086) bank188851 +188852 POINT(40.38325067178498 74.99453529889142) bank188852 +188853 POINT(40.577599853409076 74.33853260260804) bank188853 +188854 POINT(40.650284842784004 74.71922560733643) bank188854 +188855 POINT(40.98560272102706 73.94878868973153) bank188855 +188856 POINT(40.09962656788205 74.96973819337778) bank188856 +188857 POINT(39.903492704453946 74.98491062887216) bank188857 +188858 POINT(41.41907667261303 73.52755297576726) bank188858 +188859 POINT(40.63059215246996 73.55456031561584) bank188859 +188860 POINT(40.19006170653094 73.46559645469911) bank188860 +188861 POINT(40.81696569888488 74.12878620350726) bank188861 +188862 POINT(40.17170354152202 73.35813616812247) bank188862 +188863 POINT(40.2718957639773 73.22634323551256) bank188863 +188864 POINT(41.044643374241275 73.21289916073827) bank188864 +188865 POINT(41.304656088391305 74.17311893038945) bank188865 +188866 POINT(41.56544689073435 73.70979861077151) bank188866 +188867 POINT(40.217290833801265 73.13330166098835) bank188867 +188868 POINT(40.55895094533508 73.74939939073117) bank188868 +188869 POINT(39.97737124419144 74.76506219414702) bank188869 +188870 POINT(41.17384189801343 74.52244510238621) bank188870 +188871 POINT(40.197999836979335 74.97231469844718) bank188871 +188872 POINT(39.89587883988844 74.80443717239932) bank188872 +188873 POINT(41.49659643176666 74.15285261270488) bank188873 +188874 POINT(40.44695039037004 74.29188741986687) bank188874 +188875 POINT(39.838540656387266 74.74202201943075) bank188875 +188876 POINT(40.62162781562256 74.1616953820355) bank188876 +188877 POINT(40.70319004997866 74.51928572002471) bank188877 +188878 POINT(40.51210463009695 74.82727909112269) bank188878 +188879 POINT(40.98613076631082 73.64864399013206) bank188879 +188880 POINT(40.6425401936786 74.23123855119366) bank188880 +188881 POINT(39.74513316012911 73.46369724105426) bank188881 +188882 POINT(39.724499178669404 74.2797809414883) bank188882 +188883 POINT(40.56477322914329 73.04172134265932) bank188883 +188884 POINT(39.935169335347595 74.86977517602278) bank188884 +188885 POINT(41.06192637708656 74.45431057825633) bank188885 +188886 POINT(40.62633062359326 74.04334390717446) bank188886 +188887 POINT(40.25078041984126 73.97225424434404) bank188887 +188888 POINT(39.93517954647904 74.18298947694458) bank188888 +188889 POINT(39.834828612166085 74.85553639209824) bank188889 +188890 POINT(41.284518130612824 74.00760013940558) bank188890 +188891 POINT(41.58696977733116 74.64590081111704) bank188891 +188892 POINT(41.7038285642427 73.14292504771164) bank188892 +188893 POINT(40.515992647305 73.86099881518382) bank188893 +188894 POINT(40.01745710141085 74.90964091812089) bank188894 +188895 POINT(41.166537432738146 73.04398929990018) bank188895 +188896 POINT(40.83460260898688 74.94815072340974) bank188896 +188897 POINT(40.6249207526894 74.31196815639197) bank188897 +188898 POINT(40.69549439189342 74.6902198143141) bank188898 +188899 POINT(41.480992682454016 73.22858115605031) bank188899 +188900 POINT(40.86963784870193 73.28403388746156) bank188900 +188901 POINT(39.9703999418273 74.29254016206494) bank188901 +188902 POINT(40.244106049898036 73.84004057709181) bank188902 +188903 POINT(40.602665088372085 73.93477716026217) bank188903 +188904 POINT(41.10866278428493 73.61528345186811) bank188904 +188905 POINT(40.60088357562632 73.8142655111622) bank188905 +188906 POINT(41.24650180229652 74.42510350733723) bank188906 +188907 POINT(39.997796236874336 74.98305339019484) bank188907 +188908 POINT(41.02530501318764 74.72975809840173) bank188908 +188909 POINT(41.2876379083836 73.8436075121549) bank188909 +188910 POINT(40.31248903573046 74.40245435731796) bank188910 +188911 POINT(41.32044943744308 74.07622724922558) bank188911 +188912 POINT(41.289180510489984 74.11095635115447) bank188912 +188913 POINT(40.99610840471204 73.19612754550936) bank188913 +188914 POINT(40.76705153965451 74.8931994606327) bank188914 +188915 POINT(41.05157528210435 73.61779749851897) bank188915 +188916 POINT(40.2869531989513 74.04014999399061) bank188916 +188917 POINT(40.71432855980305 74.12213793210393) bank188917 +188918 POINT(40.75710319959691 74.66297978047972) bank188918 +188919 POINT(40.59544901137882 73.24189816307957) bank188919 +188920 POINT(39.89531247705023 74.1025018315053) bank188920 +188921 POINT(41.12550689847072 73.87403908754405) bank188921 +188922 POINT(41.300528890079995 73.64853835642228) bank188922 +188923 POINT(40.138357134177255 74.85748087921752) bank188923 +188924 POINT(39.72008661763056 73.32821118368369) bank188924 +188925 POINT(41.50481714679534 74.07834315044073) bank188925 +188926 POINT(41.163632702864426 74.47672873215689) bank188926 +188927 POINT(41.16280332254255 73.23794304274419) bank188927 +188928 POINT(40.98055172421927 74.11125605700012) bank188928 +188929 POINT(40.81852276606226 73.9281719525285) bank188929 +188930 POINT(40.411185030965235 73.28116168768278) bank188930 +188931 POINT(39.77134298281186 73.90729511180604) bank188931 +188932 POINT(41.26998255846158 73.5045885762444) bank188932 +188933 POINT(40.273384695480516 74.79964749201314) bank188933 +188934 POINT(40.82026942479431 73.67333464851052) bank188934 +188935 POINT(39.72031509855185 74.2807935370795) bank188935 +188936 POINT(40.66502807036776 74.97757904839375) bank188936 +188937 POINT(41.56624243501328 74.96977317417394) bank188937 +188938 POINT(41.29461800536399 74.23352138382008) bank188938 +188939 POINT(40.33481756969993 74.39942792311827) bank188939 +188940 POINT(40.459089225886366 73.77631894690138) bank188940 +188941 POINT(41.23995745008633 74.08244161175186) bank188941 +188942 POINT(39.961161060443 73.87199914217072) bank188942 +188943 POINT(40.81766897656295 74.3483853766549) bank188943 +188944 POINT(41.541030866132914 73.17461747875238) bank188944 +188945 POINT(41.16382495240109 74.93953115246323) bank188945 +188946 POINT(40.05989416488335 73.29054655644924) bank188946 +188947 POINT(40.28152481811809 73.57577665451859) bank188947 +188948 POINT(40.47918530512253 73.61332213859698) bank188948 +188949 POINT(41.62898414976795 74.13444363657253) bank188949 +188950 POINT(40.76262157414347 73.46869839778095) bank188950 +188951 POINT(40.279687203117064 73.0680859062268) bank188951 +188952 POINT(41.56364546221758 74.0036756273379) bank188952 +188953 POINT(40.878407397599105 74.88504763403763) bank188953 +188954 POINT(41.00229578976445 73.26717064106943) bank188954 +188955 POINT(40.81327939747655 74.26183127017394) bank188955 +188956 POINT(40.103860885214694 74.45272777104809) bank188956 +188957 POINT(40.09129256693924 73.96718966401437) bank188957 +188958 POINT(40.20309264312829 74.09365256418407) bank188958 +188959 POINT(39.93799477635238 74.80704444868012) bank188959 +188960 POINT(40.3668376040484 73.5091821166147) bank188960 +188961 POINT(40.98122607453533 74.0695250631761) bank188961 +188962 POINT(40.12714687855135 73.58291075711726) bank188962 +188963 POINT(40.45404798558337 73.44100713227353) bank188963 +188964 POINT(40.63409140647677 74.6883100496662) bank188964 +188965 POINT(40.07240138603722 73.6731177815516) bank188965 +188966 POINT(41.291193487550004 74.74295563827934) bank188966 +188967 POINT(40.48384973146675 74.61354437312602) bank188967 +188968 POINT(40.4014636685309 73.64969162114502) bank188968 +188969 POINT(41.50122554976054 74.18408513459617) bank188969 +188970 POINT(39.981969423864115 73.90522733377374) bank188970 +188971 POINT(41.700583225286515 73.27090887201966) bank188971 +188972 POINT(41.48450897122583 74.56270974044601) bank188972 +188973 POINT(40.72751349977346 73.58504977791314) bank188973 +188974 POINT(39.9796093447955 74.57376556514184) bank188974 +188975 POINT(41.051613162153075 73.44214150892623) bank188975 +188976 POINT(40.8342525934449 74.0464441151455) bank188976 +188977 POINT(39.91685619083485 74.62186432577771) bank188977 +188978 POINT(39.84998515461261 73.42327941887189) bank188978 +188979 POINT(40.80730022036894 73.98207693606896) bank188979 +188980 POINT(39.76431584630013 74.07165295592014) bank188980 +188981 POINT(41.10530268118322 73.30141914667333) bank188981 +188982 POINT(40.89030052857966 74.26200999367337) bank188982 +188983 POINT(40.41801546421772 73.28710418768691) bank188983 +188984 POINT(40.27205090086666 73.34549771096638) bank188984 +188985 POINT(40.41639562297241 74.05084489330555) bank188985 +188986 POINT(40.0048900513138 73.6828334646502) bank188986 +188987 POINT(39.78841196129812 74.99681035143908) bank188987 +188988 POINT(40.74464837700044 74.60987245963732) bank188988 +188989 POINT(40.657984630881224 74.34511494067613) bank188989 +188990 POINT(39.86418187126703 74.14621950798791) bank188990 +188991 POINT(41.25017441241035 73.74663221506565) bank188991 +188992 POINT(39.918557956411426 74.821069579825) bank188992 +188993 POINT(40.18843932171389 73.3876168966767) bank188993 +188994 POINT(40.9749017463198 73.20413719976385) bank188994 +188995 POINT(41.02430099399627 74.13021589608206) bank188995 +188996 POINT(41.705251915002286 73.98619182114822) bank188996 +188997 POINT(41.574628660702174 73.49451075067282) bank188997 +188998 POINT(41.093476974941446 73.12597395320435) bank188998 +188999 POINT(40.463103198543415 73.45624849858488) bank188999 +189000 POINT(40.81636665344632 74.05702550305509) bank189000 +189001 POINT(40.04188649133827 74.01351220792074) bank189001 +189002 POINT(41.03624295796579 73.35407512280568) bank189002 +189003 POINT(39.99213274919262 74.71265245076745) bank189003 +189004 POINT(39.74306879902219 73.47898342527172) bank189004 +189005 POINT(40.07663697036499 73.47890546607111) bank189005 +189006 POINT(41.623739014544185 73.5257753961582) bank189006 +189007 POINT(41.43115076999133 74.68086005575016) bank189007 +189008 POINT(40.00130334970469 73.29037885417085) bank189008 +189009 POINT(40.19545302369813 73.09592777196593) bank189009 +189010 POINT(41.147866558579544 74.06933805724645) bank189010 +189011 POINT(41.61251786589181 73.2731257953179) bank189011 +189012 POINT(41.04766551873883 74.20363872566341) bank189012 +189013 POINT(40.65076570826562 74.64258878574985) bank189013 +189014 POINT(39.7480761414401 73.6061397278514) bank189014 +189015 POINT(40.24101146660896 73.63744036943767) bank189015 +189016 POINT(40.29554804120274 74.17536647241599) bank189016 +189017 POINT(40.817718986575045 74.90348919754469) bank189017 +189018 POINT(40.3129102019856 73.8346080648658) bank189018 +189019 POINT(41.15477868487409 73.4735482462432) bank189019 +189020 POINT(40.98735282890656 74.7738486836173) bank189020 +189021 POINT(39.7814336533226 73.87578287484698) bank189021 +189022 POINT(41.65982510198971 74.90572037667185) bank189022 +189023 POINT(39.84652621573361 73.9569103898745) bank189023 +189024 POINT(41.040033770387375 74.04903275411219) bank189024 +189025 POINT(39.895551225427766 73.79441856775036) bank189025 +189026 POINT(39.90651520645987 73.4745157773957) bank189026 +189027 POINT(40.486035459473975 74.11296268067471) bank189027 +189028 POINT(40.05434255074388 74.12049170958028) bank189028 +189029 POINT(39.79402395200577 74.20633844764188) bank189029 +189030 POINT(41.60299331724366 73.01446100401911) bank189030 +189031 POINT(39.89592245969442 73.29875083262372) bank189031 +189032 POINT(41.32151880731661 74.50598694026533) bank189032 +189033 POINT(40.61541831705445 74.63863745707323) bank189033 +189034 POINT(40.87948828941771 73.48717925785958) bank189034 +189035 POINT(41.15814591936386 73.2673236118966) bank189035 +189036 POINT(40.740661189627296 74.14326074238727) bank189036 +189037 POINT(41.36686468113665 73.2905925425633) bank189037 +189038 POINT(40.628017181034046 74.78890945659339) bank189038 +189039 POINT(39.79405984591131 73.02647814400115) bank189039 +189040 POINT(40.20114793561351 74.92552283264298) bank189040 +189041 POINT(40.36553758945041 73.43061728098589) bank189041 +189042 POINT(39.97571981142929 73.93499677666942) bank189042 +189043 POINT(40.934319336516985 74.89430364203099) bank189043 +189044 POINT(41.23060593178589 73.058425592634) bank189044 +189045 POINT(40.394992913029284 74.89717525842144) bank189045 +189046 POINT(39.83966963595118 74.10513303184821) bank189046 +189047 POINT(41.36245703863866 74.177121195907) bank189047 +189048 POINT(39.75018056682518 74.24356607222104) bank189048 +189049 POINT(40.62822291082084 73.51945097511506) bank189049 +189050 POINT(41.18843860631481 74.45007675473414) bank189050 +189051 POINT(40.59973683836391 74.57160647721322) bank189051 +189052 POINT(41.25270586513402 73.86067850711902) bank189052 +189053 POINT(40.620472466391085 73.98229835144465) bank189053 +189054 POINT(41.333045219989984 74.53222082693945) bank189054 +189055 POINT(40.19823042781355 73.28000280847183) bank189055 +189056 POINT(41.57011370176168 73.13394829643214) bank189056 +189057 POINT(40.30813393320461 73.75537880024189) bank189057 +189058 POINT(41.112996946300484 74.5898398580821) bank189058 +189059 POINT(40.819240647558495 73.4712182738743) bank189059 +189060 POINT(40.612438116308695 73.87359281681098) bank189060 +189061 POINT(40.155092880306825 74.11163347473858) bank189061 +189062 POINT(39.80447202681997 73.37011870575009) bank189062 +189063 POINT(41.03773466672549 74.02494807356894) bank189063 +189064 POINT(41.08337696341379 73.01968918751683) bank189064 +189065 POINT(40.38805873851225 74.38950235560404) bank189065 +189066 POINT(39.740510429490435 74.86869410907505) bank189066 +189067 POINT(40.40988199028515 73.32209740948979) bank189067 +189068 POINT(41.255861376895005 73.254489771136) bank189068 +189069 POINT(40.056291717178375 74.03140328353797) bank189069 +189070 POINT(41.616517860061634 74.08932483194822) bank189070 +189071 POINT(41.013793654926985 73.87555539016994) bank189071 +189072 POINT(40.98023194522124 73.41782514723991) bank189072 +189073 POINT(40.00696569231667 73.95928216804785) bank189073 +189074 POINT(41.20866533122966 74.74039968001506) bank189074 +189075 POINT(41.325432150722726 74.52629648526865) bank189075 +189076 POINT(41.17586495422228 73.53517179000563) bank189076 +189077 POINT(40.50100168220723 73.6830239231462) bank189077 +189078 POINT(39.91964685086636 74.18028080807) bank189078 +189079 POINT(40.74409300133302 73.47363570590007) bank189079 +189080 POINT(39.8009137508663 73.52977032221773) bank189080 +189081 POINT(41.42060986013659 74.00308227149587) bank189081 +189082 POINT(40.8437750532857 74.92054325082036) bank189082 +189083 POINT(39.750660436796885 74.40625069179335) bank189083 +189084 POINT(41.706967782613035 74.95914329463874) bank189084 +189085 POINT(41.17834231756652 73.49528298802593) bank189085 +189086 POINT(40.905443910685335 73.64200219871539) bank189086 +189087 POINT(40.390139459565255 74.99639100064482) bank189087 +189088 POINT(39.81836936645537 73.5685615544218) bank189088 +189089 POINT(40.236215246270945 74.21950866043501) bank189089 +189090 POINT(40.47343901460347 73.10879648296395) bank189090 +189091 POINT(40.920404862033756 74.50390257032498) bank189091 +189092 POINT(40.77140296907885 74.38120318714827) bank189092 +189093 POINT(40.74131245033891 74.22018623466148) bank189093 +189094 POINT(40.91016127743366 73.47205358698862) bank189094 +189095 POINT(40.061193365041994 74.87621440600745) bank189095 +189096 POINT(40.22770256530299 74.34621230638307) bank189096 +189097 POINT(40.74940249554915 74.56766683413817) bank189097 +189098 POINT(41.54553857021676 74.55894951661949) bank189098 +189099 POINT(40.20525576413568 74.40453614812687) bank189099 +189100 POINT(40.9055608583959 73.52113742810887) bank189100 +189101 POINT(40.28298521609764 73.55865974757921) bank189101 +189102 POINT(41.19928344530485 73.98343665362965) bank189102 +189103 POINT(41.168383513033874 74.11778998748707) bank189103 +189104 POINT(39.74982355898149 73.48491550867213) bank189104 +189105 POINT(40.87921862249357 74.10863158514502) bank189105 +189106 POINT(40.59117431020957 73.55787258666828) bank189106 +189107 POINT(40.396989856160815 74.19411073298828) bank189107 +189108 POINT(40.307140874424704 73.39761218140313) bank189108 +189109 POINT(40.79690797510368 73.54918466206765) bank189109 +189110 POINT(40.67904615463105 73.33584318991497) bank189110 +189111 POINT(41.132839994880534 73.0784054638178) bank189111 +189112 POINT(40.38716868857755 74.80839164427576) bank189112 +189113 POINT(39.8605657452212 73.96516457922104) bank189113 +189114 POINT(39.772105159511455 74.99242590138331) bank189114 +189115 POINT(41.15752638126676 73.86120339558127) bank189115 +189116 POINT(41.54155078307435 73.51549343510752) bank189116 +189117 POINT(41.21835483284354 73.0500093081833) bank189117 +189118 POINT(39.90960963452091 74.33684846799356) bank189118 +189119 POINT(40.18959485869211 74.5463655399479) bank189119 +189120 POINT(39.81319940690153 73.38792963517683) bank189120 +189121 POINT(39.79851009115969 73.35085392238585) bank189121 +189122 POINT(41.084420119512714 74.1318111326309) bank189122 +189123 POINT(40.94134387473193 74.2729944497339) bank189123 +189124 POINT(40.81168131981025 74.33938638252872) bank189124 +189125 POINT(40.42953171455437 74.86509563166268) bank189125 +189126 POINT(41.57060949942358 74.26292956714275) bank189126 +189127 POINT(41.20888694897371 73.367947652421) bank189127 +189128 POINT(40.22169026032394 73.44359982690844) bank189128 +189129 POINT(41.37228886610505 74.26547541878126) bank189129 +189130 POINT(39.92359531629577 73.02932301098824) bank189130 +189131 POINT(41.02273795085031 73.5558533447937) bank189131 +189132 POINT(40.376973577633734 73.81688439052441) bank189132 +189133 POINT(40.25704508579612 73.66175866058715) bank189133 +189134 POINT(40.796473670646705 74.32311047656917) bank189134 +189135 POINT(41.46357128429932 74.49626636704463) bank189135 +189136 POINT(40.744941563919646 73.36659737853132) bank189136 +189137 POINT(41.04735172817779 74.76227003538578) bank189137 +189138 POINT(40.17245929285117 74.84878555233014) bank189138 +189139 POINT(41.21591171613576 74.45630221289991) bank189139 +189140 POINT(40.54971324314655 73.62493508002956) bank189140 +189141 POINT(41.3080602173981 74.42958110487139) bank189141 +189142 POINT(40.1665414995569 74.12022180050084) bank189142 +189143 POINT(40.371065655535425 73.30239767435836) bank189143 +189144 POINT(41.68148487900742 74.65729300774971) bank189144 +189145 POINT(40.918676079534194 73.9398626679962) bank189145 +189146 POINT(40.8756694047001 74.64367839759268) bank189146 +189147 POINT(40.80548644591112 73.58389112452865) bank189147 +189148 POINT(41.44651905440994 73.80438864728832) bank189148 +189149 POINT(39.959351755873506 73.25809999878395) bank189149 +189150 POINT(41.175839369854785 73.86692271200263) bank189150 +189151 POINT(40.156054638689675 73.00738210400932) bank189151 +189152 POINT(41.10660183316033 73.97738313776439) bank189152 +189153 POINT(40.254641736490996 73.72740100712086) bank189153 +189154 POINT(41.33802745558671 74.13357096195976) bank189154 +189155 POINT(40.25815309068072 73.45732842680671) bank189155 +189156 POINT(40.6978806867631 73.33462379913311) bank189156 +189157 POINT(41.69688701187212 73.37398773987732) bank189157 +189158 POINT(39.72164410705125 73.79556318943293) bank189158 +189159 POINT(41.659262434814835 74.71868415464876) bank189159 +189160 POINT(40.41567402979881 73.19925718086526) bank189160 +189161 POINT(41.19726646191122 73.87709923388803) bank189161 +189162 POINT(40.762143485052114 74.74751521349437) bank189162 +189163 POINT(40.853533423066565 73.99173635451605) bank189163 +189164 POINT(40.246679550736886 74.8095653001035) bank189164 +189165 POINT(41.68036287830497 73.78650783888362) bank189165 +189166 POINT(41.55365944571788 74.4555824798325) bank189166 +189167 POINT(41.506688746640584 74.02377977954903) bank189167 +189168 POINT(41.645596379582614 73.54573276304804) bank189168 +189169 POINT(41.14567267416768 73.49403038718549) bank189169 +189170 POINT(40.45192929269255 74.80970396379058) bank189170 +189171 POINT(39.75853765816421 73.92620057953056) bank189171 +189172 POINT(41.31161001768457 74.548078311867) bank189172 +189173 POINT(40.27038823399306 73.29851428082625) bank189173 +189174 POINT(40.40610995082666 73.11411862515243) bank189174 +189175 POINT(41.52638062622955 73.09130671901079) bank189175 +189176 POINT(41.41372500204289 74.68895327724891) bank189176 +189177 POINT(40.827565579147276 73.78665017278334) bank189177 +189178 POINT(41.58562266176466 74.91136628790349) bank189178 +189179 POINT(40.49871815108393 74.73495853520865) bank189179 +189180 POINT(40.08324715711733 73.66875528474597) bank189180 +189181 POINT(40.089533359410005 73.26944082250265) bank189181 +189182 POINT(41.28351170897585 73.13020185440273) bank189182 +189183 POINT(41.36676556071336 74.42603067324089) bank189183 +189184 POINT(41.55750025988795 74.59304112816186) bank189184 +189185 POINT(41.07246222987036 73.35366149602916) bank189185 +189186 POINT(39.72377115526094 73.3192625643592) bank189186 +189187 POINT(40.18454961479678 74.98387688310545) bank189187 +189188 POINT(41.19350210281637 73.82133147307923) bank189188 +189189 POINT(41.3204060204572 73.852410519148) bank189189 +189190 POINT(40.33057509086729 73.73383291894154) bank189190 +189191 POINT(40.832714401846424 73.8733278581807) bank189191 +189192 POINT(41.037994432828754 74.63734400765946) bank189192 +189193 POINT(41.61869540624615 74.48572646103446) bank189193 +189194 POINT(39.88142939951264 74.21636893983649) bank189194 +189195 POINT(41.285361556257534 74.57599589949778) bank189195 +189196 POINT(41.62761387864828 74.10237411335511) bank189196 +189197 POINT(40.07077982482369 74.20998250366003) bank189197 +189198 POINT(40.649112735081474 73.03456866415802) bank189198 +189199 POINT(41.40389283071576 74.27725418539357) bank189199 +189200 POINT(40.271443765621974 73.87073589549621) bank189200 +189201 POINT(41.326646760914144 74.16796292359363) bank189201 +189202 POINT(40.479168567725864 74.41815609062529) bank189202 +189203 POINT(40.440239137806564 74.95814711459722) bank189203 +189204 POINT(40.96024305784217 74.28400298769851) bank189204 +189205 POINT(40.82150349826245 73.28817294501604) bank189205 +189206 POINT(40.29979694870478 74.94042061054978) bank189206 +189207 POINT(40.372503658979674 74.61945630707937) bank189207 +189208 POINT(41.26669587704284 73.67878764439035) bank189208 +189209 POINT(40.51892867135993 74.80543449331579) bank189209 +189210 POINT(40.981954551730595 73.8793783892185) bank189210 +189211 POINT(41.29371855274816 74.69275039849755) bank189211 +189212 POINT(39.871693259290254 73.50299009251195) bank189212 +189213 POINT(41.30371608608265 74.93369233026567) bank189213 +189214 POINT(39.80026311370482 73.313612600572) bank189214 +189215 POINT(40.762701448530365 73.06559629489108) bank189215 +189216 POINT(41.69989264796705 73.16898696310517) bank189216 +189217 POINT(40.06038302726801 74.040119489279) bank189217 +189218 POINT(40.99730458181789 73.42289215555134) bank189218 +189219 POINT(40.65692698887641 74.25422818148678) bank189219 +189220 POINT(40.68734133590496 73.14453578332059) bank189220 +189221 POINT(41.058762309807534 73.52790398637975) bank189221 +189222 POINT(41.66791408938394 74.5292345407264) bank189222 +189223 POINT(40.054551158981965 73.74236856317275) bank189223 +189224 POINT(40.28080971984691 74.38194818359013) bank189224 +189225 POINT(41.57354034340005 74.82748715521406) bank189225 +189226 POINT(40.210543555395475 74.04112205279448) bank189226 +189227 POINT(41.14324351289808 74.4156915832673) bank189227 +189228 POINT(41.56579311257328 73.51738876852129) bank189228 +189229 POINT(40.21476689619549 74.47158894225774) bank189229 +189230 POINT(41.62002209228222 73.71392553574854) bank189230 +189231 POINT(41.68816803971023 74.80685919749733) bank189231 +189232 POINT(39.88961335249193 73.88894028597088) bank189232 +189233 POINT(41.640406210774714 73.62199338079733) bank189233 +189234 POINT(40.96477050013456 74.28222237118213) bank189234 +189235 POINT(40.03495774667741 73.84633660677416) bank189235 +189236 POINT(40.137009490604775 74.77757167053717) bank189236 +189237 POINT(40.40667288722841 73.93001080608418) bank189237 +189238 POINT(40.724864357184295 73.95567356605304) bank189238 +189239 POINT(40.7345215806361 73.12874079644303) bank189239 +189240 POINT(39.72270882485429 73.16812183241844) bank189240 +189241 POINT(41.580103608608525 73.13890875219768) bank189241 +189242 POINT(39.9049802866421 74.52391226533804) bank189242 +189243 POINT(40.74795872034708 74.10445220690038) bank189243 +189244 POINT(41.474911882221896 74.27391672263457) bank189244 +189245 POINT(40.75049427561753 73.70812616117777) bank189245 +189246 POINT(39.91350125091322 74.42734602315227) bank189246 +189247 POINT(40.7732813440064 73.95831893277054) bank189247 +189248 POINT(40.4686395451451 74.45111064388014) bank189248 +189249 POINT(39.76976256171334 73.24736564023816) bank189249 +189250 POINT(40.93573553745984 73.12701503489423) bank189250 +189251 POINT(41.36499964010021 73.06365852131981) bank189251 +189252 POINT(41.22354255664622 74.9056749263283) bank189252 +189253 POINT(40.92799340028048 74.09871592419611) bank189253 +189254 POINT(40.908440321500436 74.55020241406753) bank189254 +189255 POINT(40.51292636349014 73.85501317306324) bank189255 +189256 POINT(40.62889943470185 74.63586626734192) bank189256 +189257 POINT(39.86265366160102 74.83633591668857) bank189257 +189258 POINT(40.46601053079873 74.52399833754347) bank189258 +189259 POINT(41.117583537717266 73.13405731644052) bank189259 +189260 POINT(40.50529454418551 74.70830991378683) bank189260 +189261 POINT(39.941984460120324 73.72316375466212) bank189261 +189262 POINT(41.15690087515997 74.7424827586025) bank189262 +189263 POINT(40.738377797212166 73.40176797823885) bank189263 +189264 POINT(39.85513703845226 74.02469420844268) bank189264 +189265 POINT(40.40531641253374 73.70339357898375) bank189265 +189266 POINT(40.01700146272801 74.779120518706) bank189266 +189267 POINT(40.16310290396743 74.34205725146614) bank189267 +189268 POINT(40.952956897800135 73.49833918813759) bank189268 +189269 POINT(40.42557411174693 73.61151303361346) bank189269 +189270 POINT(40.11063791569055 74.80605074597372) bank189270 +189271 POINT(40.3571957272204 74.87795054434272) bank189271 +189272 POINT(41.50653235609511 74.07680158243768) bank189272 +189273 POINT(41.5069291834132 74.20548239163708) bank189273 +189274 POINT(41.37327246652403 74.97213912582718) bank189274 +189275 POINT(41.54380414768303 73.81793504819223) bank189275 +189276 POINT(41.1234702110498 74.39657510062716) bank189276 +189277 POINT(39.8386790289671 73.9206822611078) bank189277 +189278 POINT(40.84847709088399 73.13938621690964) bank189278 +189279 POINT(40.45970189064821 74.80247442841265) bank189279 +189280 POINT(41.223212108535726 74.48902255612569) bank189280 +189281 POINT(39.913666686953334 74.6166201681425) bank189281 +189282 POINT(41.19789076150782 74.60699478886391) bank189282 +189283 POINT(40.81451855995382 74.35412081481316) bank189283 +189284 POINT(40.965420560579545 73.26665672901524) bank189284 +189285 POINT(39.87340501376995 74.6606001166493) bank189285 +189286 POINT(40.74735776700321 73.30610362241728) bank189286 +189287 POINT(40.048603737610776 74.2785598072048) bank189287 +189288 POINT(41.248986161222284 73.97298702341627) bank189288 +189289 POINT(40.38356611560967 74.85620979070845) bank189289 +189290 POINT(41.569576509087504 74.79613077849366) bank189290 +189291 POINT(40.4211611843867 73.0404268133685) bank189291 +189292 POINT(40.600584849104635 73.05888746943683) bank189292 +189293 POINT(41.57692142906118 74.21033558969881) bank189293 +189294 POINT(41.4625852992198 74.07084356921085) bank189294 +189295 POINT(40.8546627569658 74.8403853416217) bank189295 +189296 POINT(40.50969505320618 73.61749063309055) bank189296 +189297 POINT(41.357446449827286 74.86950202329498) bank189297 +189298 POINT(39.72312423996282 74.81720644000858) bank189298 +189299 POINT(40.73837972779985 73.04061871516296) bank189299 +189300 POINT(40.9856802983194 73.2380533193241) bank189300 +189301 POINT(41.56213648562231 74.85788194214349) bank189301 +189302 POINT(41.51762729630767 73.6187503649136) bank189302 +189303 POINT(40.829965913838215 74.27860416870546) bank189303 +189304 POINT(41.664530179221764 73.79675903312138) bank189304 +189305 POINT(41.462970587555 73.07338994199385) bank189305 +189306 POINT(40.76576650473027 73.78731958955323) bank189306 +189307 POINT(40.70610972312856 73.27847768457147) bank189307 +189308 POINT(41.514023685296934 73.68604753596286) bank189308 +189309 POINT(40.8412488356745 74.9115876862471) bank189309 +189310 POINT(40.91194083871522 73.66613158697491) bank189310 +189311 POINT(41.12478422980998 74.53615136891307) bank189311 +189312 POINT(39.731443785011884 74.86932989438046) bank189312 +189313 POINT(40.83930721034586 74.87843409096715) bank189313 +189314 POINT(39.86330757559066 73.24660009970616) bank189314 +189315 POINT(39.83041234767021 73.38034935908527) bank189315 +189316 POINT(40.68199432629887 75.00156084003052) bank189316 +189317 POINT(40.95018820336073 73.36818751287231) bank189317 +189318 POINT(41.38236601218596 74.82826666643263) bank189318 +189319 POINT(41.08132753793901 73.44439444720183) bank189319 +189320 POINT(41.16763894175786 73.32122714178074) bank189320 +189321 POINT(39.77761606874478 74.65032620090017) bank189321 +189322 POINT(40.80867213414202 74.98357354494415) bank189322 +189323 POINT(41.2728528994229 74.35738733562043) bank189323 +189324 POINT(40.99358417958012 73.28615635771922) bank189324 +189325 POINT(41.70087723588989 73.73985391141983) bank189325 +189326 POINT(40.83879913086849 74.32558756340575) bank189326 +189327 POINT(40.218473928848745 74.12097558457978) bank189327 +189328 POINT(39.999740378722414 74.86048067554314) bank189328 +189329 POINT(41.3261886637607 73.15095685652251) bank189329 +189330 POINT(41.53678548485827 73.80357656651083) bank189330 +189331 POINT(40.472855627997646 73.61931883112962) bank189331 +189332 POINT(41.1802128923887 74.22439551904137) bank189332 +189333 POINT(40.58780032515647 75.0045163729795) bank189333 +189334 POINT(40.91552039521433 74.11148470118276) bank189334 +189335 POINT(40.9858715438682 73.75818813040054) bank189335 +189336 POINT(40.80271581532386 73.21659566351772) bank189336 +189337 POINT(40.50308888793047 73.66533073896288) bank189337 +189338 POINT(40.98343937333276 73.8899910088589) bank189338 +189339 POINT(41.224460624877835 73.07062549369113) bank189339 +189340 POINT(40.46927500023622 74.56730026242418) bank189340 +189341 POINT(40.49869665730207 73.61044604829162) bank189341 +189342 POINT(40.94054569718461 74.0886254125695) bank189342 +189343 POINT(39.95606994649965 74.28897564884578) bank189343 +189344 POINT(41.18755690906445 73.20243477749968) bank189344 +189345 POINT(39.7927912856308 74.74513554872728) bank189345 +189346 POINT(40.52023267753417 73.03557659945784) bank189346 +189347 POINT(39.90459676836564 74.48893831941254) bank189347 +189348 POINT(40.60602980088252 74.7039840100331) bank189348 +189349 POINT(40.47120875522772 73.33916232651818) bank189349 +189350 POINT(41.45631693519678 73.98218016336831) bank189350 +189351 POINT(39.91200795387659 74.94118134477307) bank189351 +189352 POINT(40.197762927250146 74.05605287278212) bank189352 +189353 POINT(41.16301755534709 74.90305176390709) bank189353 +189354 POINT(41.05077491833466 73.88036777920163) bank189354 +189355 POINT(39.747696933585466 73.26503330175083) bank189355 +189356 POINT(40.747997418458866 74.39118799444206) bank189356 +189357 POINT(41.51345162621814 73.77088083901091) bank189357 +189358 POINT(41.696711871292706 73.61156633127771) bank189358 +189359 POINT(41.15379879438377 74.61113625473754) bank189359 +189360 POINT(40.96385470617889 73.70316179822498) bank189360 +189361 POINT(40.76800201330141 73.33596912695543) bank189361 +189362 POINT(40.46471207218522 73.51571307998293) bank189362 +189363 POINT(39.92293577994449 74.45550474248505) bank189363 +189364 POINT(40.9947215154215 73.72861155472233) bank189364 +189365 POINT(41.60921367207553 74.31579168748746) bank189365 +189366 POINT(41.0078621566805 74.01621088588975) bank189366 +189367 POINT(41.19621887459387 74.18577855498276) bank189367 +189368 POINT(41.479846457835116 74.49308664508509) bank189368 +189369 POINT(41.409286800537096 74.65236036995553) bank189369 +189370 POINT(41.20039156914819 73.88722043714763) bank189370 +189371 POINT(39.97076719701283 73.29480884062171) bank189371 +189372 POINT(40.00253113793522 73.40999392476924) bank189372 +189373 POINT(40.12607301193687 73.7103099653936) bank189373 +189374 POINT(40.537801118525095 74.19677660239867) bank189374 +189375 POINT(40.10887765553287 74.58929066620121) bank189375 +189376 POINT(40.938642640312366 74.21691759275083) bank189376 +189377 POINT(40.47141046497131 74.5686262839185) bank189377 +189378 POINT(40.13463971425301 73.46809959941534) bank189378 +189379 POINT(39.94417994503043 74.03484550733928) bank189379 +189380 POINT(41.46555414454875 73.24125926774374) bank189380 +189381 POINT(41.57559173171951 74.40619820362174) bank189381 +189382 POINT(40.8700115399871 73.35410067243532) bank189382 +189383 POINT(41.22890307201238 74.8312764926388) bank189383 +189384 POINT(41.19855986212707 73.66167348150721) bank189384 +189385 POINT(41.43677836492315 73.1331996363123) bank189385 +189386 POINT(40.488335717550136 73.46746601450153) bank189386 +189387 POINT(41.16052134836118 74.14440443732231) bank189387 +189388 POINT(40.39248606251415 74.47770820386725) bank189388 +189389 POINT(41.081655423035535 74.37042202726312) bank189389 +189390 POINT(40.2135787890705 73.32666559447709) bank189390 +189391 POINT(40.4578501382981 74.39336186035251) bank189391 +189392 POINT(40.25336232834019 74.60438058913815) bank189392 +189393 POINT(41.367099543330866 74.4393120073702) bank189393 +189394 POINT(40.63836571829829 73.80808675948325) bank189394 +189395 POINT(41.39653257518259 73.70136541492107) bank189395 +189396 POINT(39.918464854893514 73.1488993724189) bank189396 +189397 POINT(41.049358105385785 74.7024412838198) bank189397 +189398 POINT(40.90353428800475 74.99606872480337) bank189398 +189399 POINT(40.13168651963269 74.77178281877471) bank189399 +189400 POINT(41.627270810932224 74.55883817825038) bank189400 +189401 POINT(40.9088723081891 74.62131131030766) bank189401 +189402 POINT(39.729307511155184 74.49040525216631) bank189402 +189403 POINT(40.40625380166993 74.88471167375961) bank189403 +189404 POINT(41.52110247255637 74.96328283298962) bank189404 +189405 POINT(41.48928824440254 74.0494197756964) bank189405 +189406 POINT(40.55886029723561 73.43339019536873) bank189406 +189407 POINT(41.64881623076105 73.13988835706222) bank189407 +189408 POINT(39.9079499591315 74.84060562451698) bank189408 +189409 POINT(41.44235358097072 74.32650522461908) bank189409 +189410 POINT(41.617803890172745 74.59315500369095) bank189410 +189411 POINT(40.36438207643781 74.56251748672994) bank189411 +189412 POINT(41.04232932239072 73.46588256509027) bank189412 +189413 POINT(41.63484709352333 74.86761604467931) bank189413 +189414 POINT(41.647299554388 74.8585844933053) bank189414 +189415 POINT(39.72240568625781 74.51034992185139) bank189415 +189416 POINT(40.904180559681585 74.77064856943385) bank189416 +189417 POINT(41.55439057305168 73.49053799128744) bank189417 +189418 POINT(40.199276994230594 74.9864500963304) bank189418 +189419 POINT(40.24391074492222 74.33587512730531) bank189419 +189420 POINT(40.5895498212639 73.01247968797398) bank189420 +189421 POINT(40.48535617917599 74.66470016640112) bank189421 +189422 POINT(41.15922486835931 74.07405837013381) bank189422 +189423 POINT(40.68419847349883 74.2039012587721) bank189423 +189424 POINT(40.20877426650012 73.53491381762412) bank189424 +189425 POINT(40.477916668891936 73.66751994315409) bank189425 +189426 POINT(41.50522728702264 74.988226154917) bank189426 +189427 POINT(40.0855432474837 73.95339317495018) bank189427 +189428 POINT(39.99957230994062 74.22798791806987) bank189428 +189429 POINT(40.400314431859684 74.13643872345943) bank189429 +189430 POINT(40.0291900166153 74.2973540377968) bank189430 +189431 POINT(41.353743173047704 73.81813619963619) bank189431 +189432 POINT(40.49641408785367 73.41405468343837) bank189432 +189433 POINT(41.66311402596391 74.31220322863427) bank189433 +189434 POINT(40.50508737281174 73.91471510499579) bank189434 +189435 POINT(41.46301790812625 74.4439829469508) bank189435 +189436 POINT(40.528179377325735 73.67417701223803) bank189436 +189437 POINT(40.81368162955287 73.03378246119985) bank189437 +189438 POINT(40.184317432678185 73.2653546272842) bank189438 +189439 POINT(41.27691762117959 73.3590304172985) bank189439 +189440 POINT(39.89197676943983 73.844102229617) bank189440 +189441 POINT(41.265246344924066 73.49646157374605) bank189441 +189442 POINT(41.16383551105988 74.35734914802953) bank189442 +189443 POINT(41.492323507160236 74.5063124355314) bank189443 +189444 POINT(40.87044744518213 73.15675342413829) bank189444 +189445 POINT(40.416001624251145 74.6449929407663) bank189445 +189446 POINT(40.05885678813506 73.11737716016978) bank189446 +189447 POINT(41.32485809981001 74.23417019260835) bank189447 +189448 POINT(41.449922738008105 73.74491575918329) bank189448 +189449 POINT(40.53234130443662 73.14190088701014) bank189449 +189450 POINT(39.892382343564066 73.21082042774921) bank189450 +189451 POINT(40.82134263421928 74.1843564784008) bank189451 +189452 POINT(40.47298679274531 74.65335459471166) bank189452 +189453 POINT(40.28193101559807 74.05976619268844) bank189453 +189454 POINT(40.98029542109389 74.21890491352198) bank189454 +189455 POINT(40.36412762173945 73.27286439116436) bank189455 +189456 POINT(40.08982178848514 74.48561768573383) bank189456 +189457 POINT(41.655929018265546 73.2995427323032) bank189457 +189458 POINT(40.672237118674644 73.71791122896535) bank189458 +189459 POINT(39.77787378290349 74.92459912798864) bank189459 +189460 POINT(40.21340311924393 74.105620964373) bank189460 +189461 POINT(41.617588724171696 73.92716226528086) bank189461 +189462 POINT(39.76040026177448 74.72650793796281) bank189462 +189463 POINT(41.10810152343304 74.4518715537163) bank189463 +189464 POINT(40.260329782149256 73.38588706551413) bank189464 +189465 POINT(40.37159181382539 73.14417016504288) bank189465 +189466 POINT(41.002013707666215 74.11769982171451) bank189466 +189467 POINT(40.37226828308895 74.15572090897271) bank189467 +189468 POINT(40.29256811469109 74.79446171675168) bank189468 +189469 POINT(40.01617067969851 74.82989587544755) bank189469 +189470 POINT(40.4544441083245 73.35443719517805) bank189470 +189471 POINT(40.264713315462046 74.68408956219791) bank189471 +189472 POINT(41.252636934507045 73.38560048556303) bank189472 +189473 POINT(40.74461568924306 73.67921034246484) bank189473 +189474 POINT(41.167841267842796 73.7280628506066) bank189474 +189475 POINT(40.7204184284482 74.83614795221466) bank189475 +189476 POINT(40.959510108170896 73.09745900856596) bank189476 +189477 POINT(41.43032922540677 73.70189872131722) bank189477 +189478 POINT(41.51321683577952 74.02558889115097) bank189478 +189479 POINT(41.59791445930333 74.6591433438771) bank189479 +189480 POINT(40.42913179912906 73.3486395458709) bank189480 +189481 POINT(40.26027454576555 74.06930772534518) bank189481 +189482 POINT(41.612877125999674 74.9393196158737) bank189482 +189483 POINT(41.69782173804029 73.6212632837996) bank189483 +189484 POINT(41.39280141665719 74.42813961300055) bank189484 +189485 POINT(41.337772289878124 73.87916736653068) bank189485 +189486 POINT(40.93155765780904 74.02332643224098) bank189486 +189487 POINT(39.756281550047426 73.0790159251543) bank189487 +189488 POINT(40.390158586130596 74.46726345089321) bank189488 +189489 POINT(40.74025124736548 74.35421334755229) bank189489 +189490 POINT(40.91996243590799 73.4646879612021) bank189490 +189491 POINT(41.163365163924404 73.28557801842423) bank189491 +189492 POINT(40.45519598844169 74.28092104046829) bank189492 +189493 POINT(40.92945175555008 73.55045753050824) bank189493 +189494 POINT(40.69912053736855 73.56867557684158) bank189494 +189495 POINT(41.33355356774133 73.69254280355398) bank189495 +189496 POINT(40.093225052540504 74.74375226909555) bank189496 +189497 POINT(40.98259568515224 73.13652821777073) bank189497 +189498 POINT(40.15743095487529 74.9859889575242) bank189498 +189499 POINT(39.980052777713695 74.0155444899487) bank189499 +189500 POINT(39.725373102674354 73.36576480737598) bank189500 +189501 POINT(40.04746561981398 74.13026876499282) bank189501 +189502 POINT(40.93073598827231 73.64675830998141) bank189502 +189503 POINT(41.359393153653045 74.26897983595273) bank189503 +189504 POINT(39.92226966587388 74.28910380722371) bank189504 +189505 POINT(40.407887010344346 74.1748604200015) bank189505 +189506 POINT(40.89576611956568 73.31080289307867) bank189506 +189507 POINT(40.32736647740822 73.20297912349648) bank189507 +189508 POINT(40.90888352024543 74.15821686452406) bank189508 +189509 POINT(40.450253759783116 74.10479673923734) bank189509 +189510 POINT(39.97655826664182 73.87699505957235) bank189510 +189511 POINT(41.22159582986125 73.93427267191633) bank189511 +189512 POINT(40.233255168369205 74.99851590779828) bank189512 +189513 POINT(40.87535769276369 74.43567635726892) bank189513 +189514 POINT(39.844229615529215 73.42007478313305) bank189514 +189515 POINT(40.2771113676638 73.26571977197389) bank189515 +189516 POINT(41.51181338668276 73.66018101277238) bank189516 +189517 POINT(40.2925165953416 74.0291412871126) bank189517 +189518 POINT(40.77605344503513 73.28061330851972) bank189518 +189519 POINT(40.54653621767722 73.67839329080829) bank189519 +189520 POINT(41.077429369273425 74.55661784690376) bank189520 +189521 POINT(41.01429235435914 73.19679602391354) bank189521 +189522 POINT(40.39112408663198 73.5534277186473) bank189522 +189523 POINT(41.207964542207144 74.07167512460694) bank189523 +189524 POINT(40.91867130142283 74.68499706105817) bank189524 +189525 POINT(41.59495013512374 74.87425299107045) bank189525 +189526 POINT(41.31842893450052 74.49697929821436) bank189526 +189527 POINT(39.84712926281893 73.63440646473127) bank189527 +189528 POINT(40.34741072776469 73.9765696560944) bank189528 +189529 POINT(40.10083692698134 73.56852726788817) bank189529 +189530 POINT(39.82242182516405 74.47306093885138) bank189530 +189531 POINT(40.50575223410851 74.81757599061282) bank189531 +189532 POINT(40.96894473781288 74.37601310019318) bank189532 +189533 POINT(40.520809557030425 73.93304245948772) bank189533 +189534 POINT(41.28640623878458 73.37808435525484) bank189534 +189535 POINT(40.5114440499274 74.24276425556167) bank189535 +189536 POINT(40.41891098092706 74.49559397640826) bank189536 +189537 POINT(40.702088237902665 74.3184361387937) bank189537 +189538 POINT(41.43459846497181 73.79387614687087) bank189538 +189539 POINT(39.86199343727034 74.36080243869355) bank189539 +189540 POINT(41.253037733364174 73.66635802056642) bank189540 +189541 POINT(40.90604863600789 74.35559339341775) bank189541 +189542 POINT(40.31739539720822 74.45874749893889) bank189542 +189543 POINT(39.807053475361606 74.14663678542497) bank189543 +189544 POINT(40.16929474975672 74.19647555757331) bank189544 +189545 POINT(41.61220644059012 73.36455964917627) bank189545 +189546 POINT(41.11933468604661 74.7186684932213) bank189546 +189547 POINT(40.49033696628773 74.68532226977099) bank189547 +189548 POINT(40.558364956089605 73.96547664938713) bank189548 +189549 POINT(40.20390756278954 73.65767589656669) bank189549 +189550 POINT(40.9958161492403 74.88967705857881) bank189550 +189551 POINT(39.92849086410895 74.42217524937446) bank189551 +189552 POINT(41.177894939062 73.47906340355819) bank189552 +189553 POINT(40.72861446015912 73.07215811299777) bank189553 +189554 POINT(41.15841748781187 73.63287497333282) bank189554 +189555 POINT(40.85981333071241 74.98957194245706) bank189555 +189556 POINT(40.454338822145914 73.01490593006898) bank189556 +189557 POINT(40.010690395549226 74.38312562863327) bank189557 +189558 POINT(41.60121112261404 73.2880808236872) bank189558 +189559 POINT(40.1887087086081 74.72376984180293) bank189559 +189560 POINT(40.80731252928091 73.27974024850026) bank189560 +189561 POINT(39.904847968043846 73.6249110177927) bank189561 +189562 POINT(40.86652882884569 74.36938985059047) bank189562 +189563 POINT(41.389026849517506 73.28986985202548) bank189563 +189564 POINT(39.7280419709761 73.53206570564613) bank189564 +189565 POINT(40.680628370374286 74.14003364941028) bank189565 +189566 POINT(40.383442975873045 73.594192158655) bank189566 +189567 POINT(41.58316464767632 74.02791739511994) bank189567 +189568 POINT(41.286513661721024 73.19966284875319) bank189568 +189569 POINT(41.33521792116 74.23104031870969) bank189569 +189570 POINT(40.11979776110656 73.92294599801642) bank189570 +189571 POINT(40.70403730339405 74.97920418998721) bank189571 +189572 POINT(39.73278020477367 74.44176331295272) bank189572 +189573 POINT(39.850275881876 74.1331121754765) bank189573 +189574 POINT(41.46923886084253 74.61348381211266) bank189574 +189575 POINT(40.60301662080211 74.66284996286574) bank189575 +189576 POINT(39.78178056503639 73.9752091229834) bank189576 +189577 POINT(40.63788430658508 74.96670323491789) bank189577 +189578 POINT(41.651754934643805 74.81906685609684) bank189578 +189579 POINT(40.88152531237192 73.90254067562226) bank189579 +189580 POINT(40.533728905210715 73.46745433370047) bank189580 +189581 POINT(40.84957364212369 74.40608238156628) bank189581 +189582 POINT(40.808785672818054 74.67267193553091) bank189582 +189583 POINT(41.030462241479235 73.69823461001874) bank189583 +189584 POINT(40.69999405149963 74.471200547056) bank189584 +189585 POINT(40.142805787007795 74.70826001130776) bank189585 +189586 POINT(41.02614008941831 74.52882116161533) bank189586 +189587 POINT(41.440521048333956 74.83711380886291) bank189587 +189588 POINT(40.47661154648007 74.6284706694666) bank189588 +189589 POINT(40.31883096520234 73.02948307312957) bank189589 +189590 POINT(40.544413644481494 74.8192978866102) bank189590 +189591 POINT(41.69634938644069 73.66198141521159) bank189591 +189592 POINT(40.68100040901179 73.86429730500325) bank189592 +189593 POINT(39.93853564609656 73.38167176294992) bank189593 +189594 POINT(41.56722302677568 74.49642061751815) bank189594 +189595 POINT(39.81677413155782 73.93213049050232) bank189595 +189596 POINT(40.9686753030035 74.45575588623765) bank189596 +189597 POINT(40.760641992723784 74.90275349029146) bank189597 +189598 POINT(41.401317543548736 74.46233563759823) bank189598 +189599 POINT(40.68887853637885 73.253095094456) bank189599 +189600 POINT(40.44228895188941 73.6282366041197) bank189600 +189601 POINT(41.71216234491072 73.97524681568603) bank189601 +189602 POINT(39.93832551188247 74.48353654725953) bank189602 +189603 POINT(40.50906235822055 73.86127847583116) bank189603 +189604 POINT(41.255795744360285 74.30037895978212) bank189604 +189605 POINT(39.80902537778317 74.43933486207192) bank189605 +189606 POINT(39.781565566082215 74.43519227591717) bank189606 +189607 POINT(41.396505700340875 74.34136564949405) bank189607 +189608 POINT(39.96572258090321 73.17907513990629) bank189608 +189609 POINT(40.985941204424655 75.00062105168718) bank189609 +189610 POINT(39.94381652627968 73.77693581005315) bank189610 +189611 POINT(40.74881383099367 73.85348879578613) bank189611 +189612 POINT(40.86256999464126 73.90541668548862) bank189612 +189613 POINT(41.39056095197704 73.80796680923102) bank189613 +189614 POINT(41.124436719932085 74.30526109331875) bank189614 +189615 POINT(41.701803323936794 74.1390719977609) bank189615 +189616 POINT(40.36961930782556 73.43623974504503) bank189616 +189617 POINT(40.621282465142095 73.05148202828457) bank189617 +189618 POINT(40.870443257156474 73.43051223602455) bank189618 +189619 POINT(41.479458463959354 74.93764018632261) bank189619 +189620 POINT(40.92726279852434 74.18551768276129) bank189620 +189621 POINT(40.38032758755873 73.62108801742475) bank189621 +189622 POINT(40.113926974125576 74.54213806459397) bank189622 +189623 POINT(40.801668270491064 73.74479232841924) bank189623 +189624 POINT(40.639344065480955 74.60536489685177) bank189624 +189625 POINT(40.643497734400604 73.73084216461008) bank189625 +189626 POINT(39.80408753146661 73.97961736853661) bank189626 +189627 POINT(39.85377371776063 73.51207417024663) bank189627 +189628 POINT(39.86174106402606 74.81181844234183) bank189628 +189629 POINT(40.03575789630097 74.48240307791761) bank189629 +189630 POINT(41.612558084888136 73.3096683382011) bank189630 +189631 POINT(41.495273506457906 74.17060180607605) bank189631 +189632 POINT(41.09751171353841 73.05515351283468) bank189632 +189633 POINT(40.49013285564476 74.3174643498513) bank189633 +189634 POINT(40.157808228535494 73.087370860878) bank189634 +189635 POINT(41.082916592135014 73.05892313758326) bank189635 +189636 POINT(40.954550638036146 73.00727177134667) bank189636 +189637 POINT(40.075497562226325 74.74596113557092) bank189637 +189638 POINT(40.36670155642918 73.02776878391542) bank189638 +189639 POINT(40.9396088445895 73.81269521601686) bank189639 +189640 POINT(40.17496983314607 73.92288652030823) bank189640 +189641 POINT(40.267851049309726 73.92210691310004) bank189641 +189642 POINT(40.22229138370985 74.03697422974386) bank189642 +189643 POINT(40.44452455096751 74.37479402033924) bank189643 +189644 POINT(40.827140813767606 74.67419836521766) bank189644 +189645 POINT(40.48422361857886 74.6191944840439) bank189645 +189646 POINT(41.42724560479772 74.2063240084035) bank189646 +189647 POINT(40.40825703333861 74.2936891433455) bank189647 +189648 POINT(40.584049004371906 73.89875796352777) bank189648 +189649 POINT(40.31851510246124 73.57328468765225) bank189649 +189650 POINT(40.61516042654079 73.54675536456133) bank189650 +189651 POINT(41.4543172751867 73.21355314193067) bank189651 +189652 POINT(40.50334493560869 74.96266175882961) bank189652 +189653 POINT(41.55673705062528 73.89677926659792) bank189653 +189654 POINT(40.81788436282844 73.55529882530769) bank189654 +189655 POINT(40.42583418368338 73.72761284384308) bank189655 +189656 POINT(40.12314669187514 73.73022740894604) bank189656 +189657 POINT(41.21791388225079 74.21582259257785) bank189657 +189658 POINT(40.86476607861401 74.82067339126021) bank189658 +189659 POINT(40.51750712319252 73.48485626067935) bank189659 +189660 POINT(40.360686179529424 74.17775541494004) bank189660 +189661 POINT(39.71888490025041 74.35933873495044) bank189661 +189662 POINT(41.44357736136286 74.94163895970382) bank189662 +189663 POINT(41.266087828669384 74.6612921266119) bank189663 +189664 POINT(40.49101810878029 73.70765041250655) bank189664 +189665 POINT(39.89115178540307 74.8378332793715) bank189665 +189666 POINT(40.619396918192095 74.54995628481807) bank189666 +189667 POINT(41.36369858690841 74.36293356296684) bank189667 +189668 POINT(41.13001986876304 73.46483369143542) bank189668 +189669 POINT(39.838957721576975 74.40192698154839) bank189669 +189670 POINT(40.19979424690356 73.14737113685361) bank189670 +189671 POINT(40.57866924702016 73.86692997492871) bank189671 +189672 POINT(40.33943358688375 73.57161258471106) bank189672 +189673 POINT(41.234085691229936 73.21812634081499) bank189673 +189674 POINT(40.930492954673866 73.67894722163335) bank189674 +189675 POINT(41.4162328763254 73.77341814460307) bank189675 +189676 POINT(40.01212519565672 74.16123474695719) bank189676 +189677 POINT(40.87506731345952 73.23019283676308) bank189677 +189678 POINT(41.581058486714376 73.80392477062188) bank189678 +189679 POINT(39.94975699164687 73.83770440982178) bank189679 +189680 POINT(39.84273987664442 73.39625095419622) bank189680 +189681 POINT(39.74714458275082 73.01581495963549) bank189681 +189682 POINT(40.94971455546459 73.51897216056082) bank189682 +189683 POINT(41.043592210022965 74.35834962785758) bank189683 +189684 POINT(41.38001421290585 73.14105083350029) bank189684 +189685 POINT(39.78970226226707 74.90702855433933) bank189685 +189686 POINT(40.769548538663486 73.93595919364688) bank189686 +189687 POINT(41.64675808550559 73.75784941266664) bank189687 +189688 POINT(39.890213195435834 74.17995542316386) bank189688 +189689 POINT(40.3109147492782 74.51959753007088) bank189689 +189690 POINT(41.24811547365305 73.96868819805229) bank189690 +189691 POINT(40.48669236426512 74.24097635251175) bank189691 +189692 POINT(41.4903948256336 73.14888343975002) bank189692 +189693 POINT(41.664196821102 74.40348909970365) bank189693 +189694 POINT(40.204057251520645 73.02189986678368) bank189694 +189695 POINT(41.42052743222463 73.09850374932029) bank189695 +189696 POINT(40.04023805638728 74.19240118914215) bank189696 +189697 POINT(40.315285490175675 74.81145296485514) bank189697 +189698 POINT(40.23833090604884 73.4403272026715) bank189698 +189699 POINT(41.51043662581873 74.90654813644917) bank189699 +189700 POINT(40.380306346061545 73.74398754126722) bank189700 +189701 POINT(41.088057624004854 74.62099397473852) bank189701 +189702 POINT(41.55939322481224 74.74578939626225) bank189702 +189703 POINT(41.33315554424539 74.52400936906791) bank189703 +189704 POINT(40.06996136191956 73.1978793514847) bank189704 +189705 POINT(41.05625151673157 73.7039315851806) bank189705 +189706 POINT(40.27420471365486 73.05373718189122) bank189706 +189707 POINT(41.58228746854632 73.42427417943786) bank189707 +189708 POINT(40.40848124619116 74.6496882448227) bank189708 +189709 POINT(40.68636434107999 74.8739240024913) bank189709 +189710 POINT(40.694294127918596 73.51421990168731) bank189710 +189711 POINT(41.45218828807227 74.01321122197693) bank189711 +189712 POINT(41.69274952217841 73.8727638188819) bank189712 +189713 POINT(40.63516857916219 74.5799267461238) bank189713 +189714 POINT(40.29957778689095 74.3351429624537) bank189714 +189715 POINT(40.48500652769195 73.24049864202243) bank189715 +189716 POINT(39.9203072319539 74.55471769959009) bank189716 +189717 POINT(40.031988381203966 73.60584584672334) bank189717 +189718 POINT(41.50464810597236 73.51355346505123) bank189718 +189719 POINT(39.92415691385422 74.40039750300002) bank189719 +189720 POINT(40.08097095245391 73.17795929042796) bank189720 +189721 POINT(41.559260966406946 74.45011322231466) bank189721 +189722 POINT(40.64356054702944 74.2189747285886) bank189722 +189723 POINT(39.88034794542755 74.51352276006928) bank189723 +189724 POINT(41.644986221937096 73.57532349093681) bank189724 +189725 POINT(40.57028319059186 73.89228951048095) bank189725 +189726 POINT(40.360978510187515 74.21001478807145) bank189726 +189727 POINT(40.34300116735335 74.27319018002464) bank189727 +189728 POINT(41.27851678278085 73.62659098612131) bank189728 +189729 POINT(41.30953559388431 74.89983589199709) bank189729 +189730 POINT(41.35071292499695 73.36731084386373) bank189730 +189731 POINT(41.5425179190684 73.0633829450948) bank189731 +189732 POINT(39.87644838440398 74.03612457839077) bank189732 +189733 POINT(39.817540121529724 74.26332505614606) bank189733 +189734 POINT(41.55094086005797 74.34287769960191) bank189734 +189735 POINT(40.5718267226108 73.57789826728026) bank189735 +189736 POINT(40.40676579620901 74.15875167714712) bank189736 +189737 POINT(41.32412700988275 73.08562477849142) bank189737 +189738 POINT(40.424215282083736 74.71704937467582) bank189738 +189739 POINT(40.53677342489419 74.66828844795563) bank189739 +189740 POINT(39.95360413278975 74.8949642444825) bank189740 +189741 POINT(40.48341799755918 73.63531357243484) bank189741 +189742 POINT(39.964704164112966 74.74893616440448) bank189742 +189743 POINT(39.77211710331434 73.20566742299854) bank189743 +189744 POINT(40.34511170201346 74.27314046340162) bank189744 +189745 POINT(41.45591414900502 74.01426281259131) bank189745 +189746 POINT(41.02319075683924 74.02300117267404) bank189746 +189747 POINT(40.03759246038383 74.42572921790592) bank189747 +189748 POINT(39.7283682237924 73.58673177175523) bank189748 +189749 POINT(39.75385282102487 74.82156178558078) bank189749 +189750 POINT(40.16890844770148 73.19682769418748) bank189750 +189751 POINT(41.4674710275377 73.38575480578989) bank189751 +189752 POINT(41.49196761762016 73.3341642039266) bank189752 +189753 POINT(39.8697304127068 73.6111866817718) bank189753 +189754 POINT(41.58064872952042 73.79361239742715) bank189754 +189755 POINT(41.14677845194942 74.40691843227503) bank189755 +189756 POINT(39.90466867157992 74.2193764130767) bank189756 +189757 POINT(41.23469751894177 74.21552863151602) bank189757 +189758 POINT(39.913636707568756 73.52334635048359) bank189758 +189759 POINT(39.8448938397897 73.19443171212033) bank189759 +189760 POINT(41.23384990982731 74.3061453314351) bank189760 +189761 POINT(39.9398164463093 74.74273506830795) bank189761 +189762 POINT(41.13192473027266 74.41307480475426) bank189762 +189763 POINT(39.99034245958639 73.06340006316624) bank189763 +189764 POINT(40.30104329925379 73.75771348991513) bank189764 +189765 POINT(40.11327932458851 74.56768299725141) bank189765 +189766 POINT(41.50195272265998 73.87630639015248) bank189766 +189767 POINT(40.07624101901936 74.34295023223417) bank189767 +189768 POINT(41.18370210620132 73.58865307843907) bank189768 +189769 POINT(40.62455773563434 74.60486906458829) bank189769 +189770 POINT(40.04445220158072 74.18590447389333) bank189770 +189771 POINT(40.77843835987956 73.2882071377807) bank189771 +189772 POINT(40.61475487299967 74.37353983844662) bank189772 +189773 POINT(41.06486517513917 73.47150186967234) bank189773 +189774 POINT(41.61370663691189 73.62028703086065) bank189774 +189775 POINT(39.9908796459544 74.49429225998941) bank189775 +189776 POINT(39.7776087543488 73.6493266412323) bank189776 +189777 POINT(40.910046555513205 74.86688415105768) bank189777 +189778 POINT(41.293177597045975 73.82960651855953) bank189778 +189779 POINT(41.22866475356436 74.31926597736964) bank189779 +189780 POINT(39.87756925152815 74.36880383560619) bank189780 +189781 POINT(40.174340572222206 73.9951527563868) bank189781 +189782 POINT(40.54002190058346 73.0821092266224) bank189782 +189783 POINT(41.596867197670626 73.45422004824032) bank189783 +189784 POINT(40.97477626107045 73.53171031200857) bank189784 +189785 POINT(39.834530016037725 74.4370016750501) bank189785 +189786 POINT(40.313457419531474 73.35097532656309) bank189786 +189787 POINT(40.76480966379545 73.57810954248164) bank189787 +189788 POINT(41.44010134284139 73.96323211853517) bank189788 +189789 POINT(39.79233872500172 74.69916487069378) bank189789 +189790 POINT(40.48697544304552 73.9741193660165) bank189790 +189791 POINT(39.76387955378939 73.965321313656) bank189791 +189792 POINT(40.936864982763346 74.87578227707748) bank189792 +189793 POINT(40.031045313654374 74.682607087441) bank189793 +189794 POINT(41.211336272111275 73.41410994454776) bank189794 +189795 POINT(41.378170718693724 74.23334808137085) bank189795 +189796 POINT(41.681947374734676 74.20016688621126) bank189796 +189797 POINT(41.58685038883461 74.48443291768972) bank189797 +189798 POINT(41.158632015514286 74.19827458006381) bank189798 +189799 POINT(41.53827966398631 74.62095119304749) bank189799 +189800 POINT(40.4189465842226 74.2392247605681) bank189800 +189801 POINT(41.43731671429017 74.15026610723102) bank189801 +189802 POINT(40.95120239320781 74.7188883414722) bank189802 +189803 POINT(40.40589932142673 74.33111337707597) bank189803 +189804 POINT(40.75649623625105 73.16162873193097) bank189804 +189805 POINT(40.88319502621993 74.57161406541366) bank189805 +189806 POINT(40.74211586486139 73.07345457568289) bank189806 +189807 POINT(41.63566908294686 73.31596402637484) bank189807 +189808 POINT(41.691949101942306 73.46120192870089) bank189808 +189809 POINT(41.51301916327792 74.95211894279869) bank189809 +189810 POINT(40.27445496605034 74.30554638401586) bank189810 +189811 POINT(41.44063594165285 74.90656981215669) bank189811 +189812 POINT(40.89644003860484 73.60826519762043) bank189812 +189813 POINT(40.31623353319653 74.2608045298356) bank189813 +189814 POINT(41.49210524873204 73.91133916213029) bank189814 +189815 POINT(40.47055123350084 73.81304347015379) bank189815 +189816 POINT(40.9120224604096 74.43544065016904) bank189816 +189817 POINT(40.73272445636339 74.90062360065716) bank189817 +189818 POINT(40.25383824177822 74.04124604996056) bank189818 +189819 POINT(41.69769515466071 73.99853207174293) bank189819 +189820 POINT(40.1502007624917 74.09395064639897) bank189820 +189821 POINT(40.51498697394875 73.16922162368013) bank189821 +189822 POINT(41.33696186078773 74.01709767135858) bank189822 +189823 POINT(41.02103721794377 74.85000926003872) bank189823 +189824 POINT(40.75563913069031 74.33621975827296) bank189824 +189825 POINT(40.07155035445506 73.87654278909129) bank189825 +189826 POINT(41.23616672964699 74.422603216697) bank189826 +189827 POINT(41.14916096554836 73.9496573263946) bank189827 +189828 POINT(40.96552570203367 73.61856976824447) bank189828 +189829 POINT(40.9045789310914 74.41593063998103) bank189829 +189830 POINT(41.65482729874187 74.81043241326933) bank189830 +189831 POINT(41.586997497781155 73.91547512981049) bank189831 +189832 POINT(41.490137891393545 74.17809832255465) bank189832 +189833 POINT(40.618603801966955 73.12912201924458) bank189833 +189834 POINT(41.09406207454826 74.60506433813154) bank189834 +189835 POINT(41.47663897276529 74.30015886530937) bank189835 +189836 POINT(41.33136877676246 73.60948485562479) bank189836 +189837 POINT(39.848682112494075 73.45554148338105) bank189837 +189838 POINT(41.362029621953724 73.7326834962347) bank189838 +189839 POINT(40.857251499912 74.30770604879709) bank189839 +189840 POINT(40.154563884374475 73.34716181688695) bank189840 +189841 POINT(41.14333424059928 74.46830806118959) bank189841 +189842 POINT(41.055171043451814 74.71784661353753) bank189842 +189843 POINT(40.387383023776486 73.20647269567768) bank189843 +189844 POINT(41.120946889710886 74.55835431198437) bank189844 +189845 POINT(40.62206563129721 74.06542999609624) bank189845 +189846 POINT(41.15700897723292 73.45814832358461) bank189846 +189847 POINT(41.1719476631046 74.65590565798526) bank189847 +189848 POINT(40.08928723410492 74.26747685237136) bank189848 +189849 POINT(40.61796789804215 73.4753190526671) bank189849 +189850 POINT(40.17307155889168 74.04722438486706) bank189850 +189851 POINT(41.60033062516235 73.9819320755036) bank189851 +189852 POINT(40.058987781251965 74.19282891611928) bank189852 +189853 POINT(40.47223744566282 73.94088238166395) bank189853 +189854 POINT(41.426725442977336 73.30498172928239) bank189854 +189855 POINT(40.84171296998724 73.32338022625676) bank189855 +189856 POINT(40.80905803504041 73.72328793107748) bank189856 +189857 POINT(40.72686989145865 74.65413991201888) bank189857 +189858 POINT(39.94529571435873 74.07560461075796) bank189858 +189859 POINT(41.635096873411 74.8119589367348) bank189859 +189860 POINT(40.746891349129676 73.56742098294256) bank189860 +189861 POINT(41.40036733067323 73.90678754563119) bank189861 +189862 POINT(40.05594440546958 74.07003930426495) bank189862 +189863 POINT(40.528159562560866 73.03546547054604) bank189863 +189864 POINT(41.18792759768844 73.98848192626396) bank189864 +189865 POINT(40.06766571183372 74.447131061183) bank189865 +189866 POINT(41.68168703551517 74.90140001240242) bank189866 +189867 POINT(40.208981910202155 73.92901682010032) bank189867 +189868 POINT(40.88360415947585 73.04680805343389) bank189868 +189869 POINT(41.57016684760155 73.50063651953728) bank189869 +189870 POINT(40.82484335598216 73.07185300485773) bank189870 +189871 POINT(40.36556443692547 74.16947611142854) bank189871 +189872 POINT(39.85283345504399 73.63695511762263) bank189872 +189873 POINT(40.315034980299274 73.69331554971276) bank189873 +189874 POINT(40.49656517672079 74.40439308594668) bank189874 +189875 POINT(39.82770957557685 73.9558828889516) bank189875 +189876 POINT(41.32650418918479 73.12612302204197) bank189876 +189877 POINT(39.753647741644045 74.37840850163482) bank189877 +189878 POINT(41.14825327359963 74.7900906278032) bank189878 +189879 POINT(41.11738325055823 74.62690205419645) bank189879 +189880 POINT(41.27522852564604 73.1523749071462) bank189880 +189881 POINT(39.88070261454758 73.24212366494672) bank189881 +189882 POINT(39.96429173204075 74.1169142173251) bank189882 +189883 POINT(39.749326555432894 74.69604250106745) bank189883 +189884 POINT(39.9416407440077 74.71724023486527) bank189884 +189885 POINT(40.06361798685518 73.38060170577675) bank189885 +189886 POINT(41.321224798605016 73.49827586528112) bank189886 +189887 POINT(40.21067060182833 73.44886670234736) bank189887 +189888 POINT(39.794669089809126 73.94005530424977) bank189888 +189889 POINT(40.21716261636449 74.87078365694028) bank189889 +189890 POINT(40.877093669323614 73.70187378483836) bank189890 +189891 POINT(40.31342390505023 73.78833290349156) bank189891 +189892 POINT(39.92335926265712 74.27957426495476) bank189892 +189893 POINT(40.7101101905805 74.33434994897962) bank189893 +189894 POINT(41.039794776513794 74.74471377656718) bank189894 +189895 POINT(41.61045845775384 73.72741037308737) bank189895 +189896 POINT(40.20332754235148 73.28485148567054) bank189896 +189897 POINT(40.216857268789774 73.64737851716079) bank189897 +189898 POINT(40.82668948827424 73.88519508435851) bank189898 +189899 POINT(40.47239229247114 74.26118971021464) bank189899 +189900 POINT(40.868626033082315 74.58172469351383) bank189900 +189901 POINT(41.4746741092478 74.40120012934965) bank189901 +189902 POINT(41.358360370844814 74.43021733416622) bank189902 +189903 POINT(41.29071839482116 74.99869069904632) bank189903 +189904 POINT(40.48547163691748 74.74803088535144) bank189904 +189905 POINT(39.757106317612575 73.169427436571) bank189905 +189906 POINT(40.496656414827136 74.93087989976779) bank189906 +189907 POINT(41.503040038867304 74.3126021123605) bank189907 +189908 POINT(40.2148983932448 73.94635382048874) bank189908 +189909 POINT(41.484042955747825 74.90046347903039) bank189909 +189910 POINT(41.20697511914612 73.21274331754807) bank189910 +189911 POINT(40.45389661430294 73.08385177552519) bank189911 +189912 POINT(40.79189240014381 73.80001275421364) bank189912 +189913 POINT(41.46112770614636 74.95525688514839) bank189913 +189914 POINT(40.159334835541195 74.1433930617969) bank189914 +189915 POINT(41.17158329840679 73.53898817145888) bank189915 +189916 POINT(40.57651702181651 73.07899637362749) bank189916 +189917 POINT(41.513373447878834 74.67786039847704) bank189917 +189918 POINT(40.03650836488467 73.1023746051629) bank189918 +189919 POINT(39.744269254620654 73.66904806200641) bank189919 +189920 POINT(40.571459502082725 74.07082945057958) bank189920 +189921 POINT(41.47994753348283 74.5581009710151) bank189921 +189922 POINT(39.922588412736744 73.23130071308672) bank189922 +189923 POINT(40.68386690811995 73.98153019241792) bank189923 +189924 POINT(40.78769278646958 73.13640015605627) bank189924 +189925 POINT(41.23516873291126 74.30740473783968) bank189925 +189926 POINT(39.803850028939635 73.97693942810922) bank189926 +189927 POINT(40.23395479193127 74.18104826635903) bank189927 +189928 POINT(41.28932468640414 73.37695450482342) bank189928 +189929 POINT(40.50211128985819 73.93922983720698) bank189929 +189930 POINT(40.70592860487988 73.8312314857498) bank189930 +189931 POINT(40.67502985815555 73.9252153373083) bank189931 +189932 POINT(41.06027310302305 74.39012464576197) bank189932 +189933 POINT(39.761570540841454 74.14565941938537) bank189933 +189934 POINT(40.92208828579346 73.63317716728828) bank189934 +189935 POINT(39.85843263312852 73.55207586883944) bank189935 +189936 POINT(41.57465502280675 74.77793740463332) bank189936 +189937 POINT(39.82836068734116 73.92671482038794) bank189937 +189938 POINT(40.309206262155236 73.42767919477676) bank189938 +189939 POINT(40.41989920543709 74.19846206366371) bank189939 +189940 POINT(41.66956542304201 73.5709629961442) bank189940 +189941 POINT(40.47753697590225 73.82886834520338) bank189941 +189942 POINT(41.15392275448114 74.65174825102393) bank189942 +189943 POINT(41.03721555109439 73.8716349404375) bank189943 +189944 POINT(40.75437582601758 73.42925245717669) bank189944 +189945 POINT(41.20998701581604 74.91447860174284) bank189945 +189946 POINT(40.338873791812546 74.14604936091666) bank189946 +189947 POINT(39.71551460169666 73.37306460717713) bank189947 +189948 POINT(40.684814931512705 73.40519212045218) bank189948 +189949 POINT(40.17754476173692 74.81514633588597) bank189949 +189950 POINT(40.49452693047769 73.03055994794624) bank189950 +189951 POINT(40.988261003060224 73.65278965453463) bank189951 +189952 POINT(40.899620468625116 74.68460580558398) bank189952 +189953 POINT(41.341207333538826 73.49169938321349) bank189953 +189954 POINT(41.70309244048992 75.00340502573216) bank189954 +189955 POINT(40.18121252353868 74.34847229344962) bank189955 +189956 POINT(40.95473460611306 74.62707219329961) bank189956 +189957 POINT(40.493869977461976 74.42483889278931) bank189957 +189958 POINT(39.891395318758825 73.92899589973662) bank189958 +189959 POINT(41.679401612259504 73.09333290168503) bank189959 +189960 POINT(40.77330823620024 73.47230720424464) bank189960 +189961 POINT(39.85224347122259 74.0668668823559) bank189961 +189962 POINT(41.3728954125678 74.03035190191301) bank189962 +189963 POINT(41.015771990247735 74.97942363039726) bank189963 +189964 POINT(40.55852614963889 73.62185854769906) bank189964 +189965 POINT(40.06784492715208 74.07073570590397) bank189965 +189966 POINT(40.21674875685853 73.45275694712525) bank189966 +189967 POINT(41.47182998325792 74.62508656105926) bank189967 +189968 POINT(39.872057977233005 73.96787560986505) bank189968 +189969 POINT(40.64553921000784 74.56406829422836) bank189969 +189970 POINT(40.6246809835969 73.55938584277793) bank189970 +189971 POINT(40.10253510396962 73.978956561797) bank189971 +189972 POINT(41.096352942192354 73.93481526634055) bank189972 +189973 POINT(40.42889825970004 73.99278356820058) bank189973 +189974 POINT(41.43119319096877 73.22589627591661) bank189974 +189975 POINT(41.57890689268247 73.66858484209818) bank189975 +189976 POINT(40.855378346521434 74.4717994643805) bank189976 +189977 POINT(40.759286502029624 74.63171571390488) bank189977 +189978 POINT(40.894420715268815 74.26920547079774) bank189978 +189979 POINT(41.17409126134723 73.92511422775425) bank189979 +189980 POINT(40.16684387749698 73.10377878766668) bank189980 +189981 POINT(40.52540744233834 74.50403208690328) bank189981 +189982 POINT(41.4159969839666 74.53616222069594) bank189982 +189983 POINT(40.04325502850195 74.3750200747168) bank189983 +189984 POINT(39.93772843384627 73.10056916226206) bank189984 +189985 POINT(40.65961080947016 73.57118732689149) bank189985 +189986 POINT(39.99015607887749 73.47261440113674) bank189986 +189987 POINT(40.86429670320641 74.55601934966805) bank189987 +189988 POINT(41.210098326001166 74.88332527185035) bank189988 +189989 POINT(41.46867214211708 73.60070959097044) bank189989 +189990 POINT(41.051545726975135 73.17274118759917) bank189990 +189991 POINT(41.058405819248684 73.5054881669793) bank189991 +189992 POINT(40.736707134818204 74.01501413742109) bank189992 +189993 POINT(41.334856046366625 73.20628177586012) bank189993 +189994 POINT(40.109779436759396 73.81464891501624) bank189994 +189995 POINT(39.92997609532908 73.35870669377744) bank189995 +189996 POINT(41.59595408980448 73.28825955702443) bank189996 +189997 POINT(40.43297890617109 73.61267746414075) bank189997 +189998 POINT(41.24833850554834 74.43778120490859) bank189998 +189999 POINT(39.93389573646265 73.47388746315427) bank189999 +190000 POINT(41.111894444433084 74.42899115289312) bank190000 +190001 POINT(39.83943796954118 74.200031245328) bank190001 +190002 POINT(40.16598202997539 73.28213411349303) bank190002 +190003 POINT(41.576008531324234 74.91377693825909) bank190003 +190004 POINT(40.54089175637684 74.81478726207958) bank190004 +190005 POINT(41.36279192663641 74.32347622615295) bank190005 +190006 POINT(40.78796169034837 73.50947089069233) bank190006 +190007 POINT(41.151399114315495 74.88009144747593) bank190007 +190008 POINT(40.452315041432385 74.74194313823482) bank190008 +190009 POINT(41.32830432998212 73.4031094237155) bank190009 +190010 POINT(40.85745407874815 73.01359208176596) bank190010 +190011 POINT(39.72779736299505 73.46664756315639) bank190011 +190012 POINT(41.05542291167636 74.06655955855048) bank190012 +190013 POINT(41.38690452773384 73.76024750351823) bank190013 +190014 POINT(41.19974987267475 73.06503761666521) bank190014 +190015 POINT(40.96765688889544 74.32864473855223) bank190015 +190016 POINT(40.48234663358528 74.41409569432885) bank190016 +190017 POINT(41.11245428004735 74.1665686468613) bank190017 +190018 POINT(41.03948527574439 74.94223953985579) bank190018 +190019 POINT(40.26531902743071 73.72683800309531) bank190019 +190020 POINT(41.23554882555771 74.09034929850868) bank190020 +190021 POINT(40.18515307309333 74.0034885197332) bank190021 +190022 POINT(40.37995257644137 74.17083482409559) bank190022 +190023 POINT(40.63056873382644 74.35220817926691) bank190023 +190024 POINT(39.94733748888365 74.27542894644976) bank190024 +190025 POINT(40.83759958281065 74.09953159709266) bank190025 +190026 POINT(39.89802506803207 73.47738936006166) bank190026 +190027 POINT(40.859108812782715 74.28882476729417) bank190027 +190028 POINT(41.18783883896479 73.67650424414984) bank190028 +190029 POINT(39.88524155311798 74.45606418443148) bank190029 +190030 POINT(39.9154753949999 74.6965758022071) bank190030 +190031 POINT(39.80553907312517 74.45388749310494) bank190031 +190032 POINT(41.295627412794616 74.79586894078803) bank190032 +190033 POINT(41.39390793483031 74.73799345783208) bank190033 +190034 POINT(40.402469337100065 74.20275923343381) bank190034 +190035 POINT(39.85670404050907 74.58132544061408) bank190035 +190036 POINT(40.23923664170911 74.91340972029813) bank190036 +190037 POINT(41.62377722204359 73.39596226234744) bank190037 +190038 POINT(41.654688271330194 74.08516916172044) bank190038 +190039 POINT(41.665741348542966 74.422732691393) bank190039 +190040 POINT(41.26931630875067 74.39046703663297) bank190040 +190041 POINT(40.692637133962265 73.47680010019765) bank190041 +190042 POINT(40.87448469716544 73.86567835187232) bank190042 +190043 POINT(40.34054168563813 73.84049893207259) bank190043 +190044 POINT(40.07613304633002 73.48202259455383) bank190044 +190045 POINT(41.5305304146053 74.70275996793005) bank190045 +190046 POINT(41.565653786521466 73.52492623282943) bank190046 +190047 POINT(40.57859281240533 73.6914312980309) bank190047 +190048 POINT(40.85152179088445 74.23934610933804) bank190048 +190049 POINT(41.514829393195114 73.47730133302785) bank190049 +190050 POINT(41.27784584548352 73.34260434529358) bank190050 +190051 POINT(40.52484457707825 74.16128905071498) bank190051 +190052 POINT(39.78184362147627 73.12866022587268) bank190052 +190053 POINT(41.338761423517155 73.25320461217872) bank190053 +190054 POINT(40.58361223407922 74.56093072524834) bank190054 +190055 POINT(41.10649197420381 73.08491629847076) bank190055 +190056 POINT(41.031536956868905 74.09144683999473) bank190056 +190057 POINT(41.29618754934685 73.5780467782856) bank190057 +190058 POINT(40.17139410447954 73.75813668309554) bank190058 +190059 POINT(41.02771555159451 74.57420917975514) bank190059 +190060 POINT(40.62050030202542 73.81330252912976) bank190060 +190061 POINT(41.65429957237997 73.86720837127234) bank190061 +190062 POINT(39.83424405739956 73.00667146354256) bank190062 +190063 POINT(40.86366026408302 74.16582716108684) bank190063 +190064 POINT(40.39805122940204 73.82919419706944) bank190064 +190065 POINT(40.52606074058987 73.59306412960655) bank190065 +190066 POINT(39.991211834345016 73.39639361000428) bank190066 +190067 POINT(39.779935838983306 74.53096236477641) bank190067 +190068 POINT(40.942318441053274 74.06999490073876) bank190068 +190069 POINT(40.14687093277698 73.88688226636965) bank190069 +190070 POINT(41.28555657376087 74.38970466263986) bank190070 +190071 POINT(40.741646118879245 74.06222016624531) bank190071 +190072 POINT(41.31440082100903 74.46194606432117) bank190072 +190073 POINT(40.846452248906076 73.67690347082842) bank190073 +190074 POINT(41.36730419830969 73.77761317463005) bank190074 +190075 POINT(40.06793069158116 74.3199254109166) bank190075 +190076 POINT(40.67823256104669 74.96046760127768) bank190076 +190077 POINT(41.53487905216096 73.03439623415068) bank190077 +190078 POINT(41.119538908886206 74.62249898729267) bank190078 +190079 POINT(40.47681318051691 73.35868710853289) bank190079 +190080 POINT(40.54903619414641 74.49155504908109) bank190080 +190081 POINT(40.874965637098995 73.82157284350313) bank190081 +190082 POINT(41.51371338946035 74.43112187003821) bank190082 +190083 POINT(40.146972435472584 74.35333404334489) bank190083 +190084 POINT(39.735519230215225 73.78809205060922) bank190084 +190085 POINT(40.597869126979525 74.18011765361528) bank190085 +190086 POINT(40.28595413057326 74.80812864354353) bank190086 +190087 POINT(39.76864365861627 73.63911359348225) bank190087 +190088 POINT(41.34832275459858 73.86922493292994) bank190088 +190089 POINT(40.11387188280614 73.17320911649506) bank190089 +190090 POINT(41.52322966832069 73.73964364573841) bank190090 +190091 POINT(40.30720136888294 74.82448391288509) bank190091 +190092 POINT(41.04054099608992 74.73426105011745) bank190092 +190093 POINT(40.16013099415584 73.28108743584482) bank190093 +190094 POINT(40.36270693346737 73.99903801299243) bank190094 +190095 POINT(40.003662531513186 74.77379486814682) bank190095 +190096 POINT(41.074479301629374 73.80449994388445) bank190096 +190097 POINT(41.67709702380861 73.76728952629173) bank190097 +190098 POINT(39.713694588050174 74.48526845802387) bank190098 +190099 POINT(41.49412175566956 74.50176878701555) bank190099 +190100 POINT(41.6887470694914 74.15370196309767) bank190100 +190101 POINT(39.81426460505634 73.35539038034814) bank190101 +190102 POINT(40.65330076738026 73.7037593254725) bank190102 +190103 POINT(41.27092985140665 74.06400961663915) bank190103 +190104 POINT(41.59467141534301 74.54085200386982) bank190104 +190105 POINT(41.239979435795135 74.55268972456362) bank190105 +190106 POINT(39.803017384711495 73.50368980195692) bank190106 +190107 POINT(41.00547955019238 74.94168545641782) bank190107 +190108 POINT(41.47863770397989 74.8738493563922) bank190108 +190109 POINT(40.79675717357824 74.27316540018299) bank190109 +190110 POINT(40.075716871009035 73.98413602202744) bank190110 +190111 POINT(39.99695611163206 74.46255394278131) bank190111 +190112 POINT(40.299581207915615 74.17101037895317) bank190112 +190113 POINT(40.75354214978695 73.93958891208227) bank190113 +190114 POINT(41.14369280514985 73.03014658570439) bank190114 +190115 POINT(41.30874137048436 74.54961016400088) bank190115 +190116 POINT(40.40260377727172 74.02958893469226) bank190116 +190117 POINT(41.655230565701245 73.45203861343396) bank190117 +190118 POINT(41.104671727162405 74.16662178533629) bank190118 +190119 POINT(40.05054576657871 74.9966105975742) bank190119 +190120 POINT(41.16556880184751 74.15765529151041) bank190120 +190121 POINT(39.72678754353863 73.74337736395009) bank190121 +190122 POINT(41.50558138535267 74.78024309648289) bank190122 +190123 POINT(40.5818508738428 74.61818678357207) bank190123 +190124 POINT(39.78731155043668 73.91661837815437) bank190124 +190125 POINT(41.383617350681526 74.730643422852) bank190125 +190126 POINT(40.25827898514917 73.74830993210902) bank190126 +190127 POINT(40.98115955520463 74.0571639809234) bank190127 +190128 POINT(40.311827676853156 74.18948094148853) bank190128 +190129 POINT(40.96500153220759 74.28183639406598) bank190129 +190130 POINT(40.88264158082822 73.45156900983602) bank190130 +190131 POINT(41.52651385720316 73.3761588798884) bank190131 +190132 POINT(41.5516840792619 74.60540411564645) bank190132 +190133 POINT(40.37663152480351 73.12701729017223) bank190133 +190134 POINT(40.14130140354888 74.82831033511299) bank190134 +190135 POINT(41.68235925562297 73.08278612933819) bank190135 +190136 POINT(41.32603942328549 74.70477508566114) bank190136 +190137 POINT(40.27730947612902 74.43736027503594) bank190137 +190138 POINT(40.66939724616914 74.76261709653721) bank190138 +190139 POINT(40.36496374718873 73.97957781414455) bank190139 +190140 POINT(39.78169450042947 73.46925759799808) bank190140 +190141 POINT(41.60353871875593 73.85450303406083) bank190141 +190142 POINT(40.25884736687524 73.53283869371025) bank190142 +190143 POINT(40.61019208257123 74.26754642029339) bank190143 +190144 POINT(40.326200377307345 73.36744328459324) bank190144 +190145 POINT(41.20569696679753 74.77052799152187) bank190145 +190146 POINT(41.04999556409948 74.39639017975811) bank190146 +190147 POINT(40.84919953867188 74.25541656427117) bank190147 +190148 POINT(40.1343235183682 74.57312974284442) bank190148 +190149 POINT(40.14761009637468 73.50097184839814) bank190149 +190150 POINT(40.38307626698962 74.3046367109482) bank190150 +190151 POINT(40.12263933055302 73.36224002287683) bank190151 +190152 POINT(40.61938174892583 74.8577786395261) bank190152 +190153 POINT(40.55745673201088 73.6744230953051) bank190153 +190154 POINT(41.23416552629589 74.72693519603484) bank190154 +190155 POINT(40.15527113110518 74.13999805183516) bank190155 +190156 POINT(40.65053180619525 74.65936800276465) bank190156 +190157 POINT(40.820212982852325 73.56674001051485) bank190157 +190158 POINT(40.11060385409949 73.39095996069912) bank190158 +190159 POINT(39.99757717679387 73.84561553243962) bank190159 +190160 POINT(40.025578640205104 74.30341174704192) bank190160 +190161 POINT(39.76976323406569 74.40434502501674) bank190161 +190162 POINT(41.68031349415189 73.86742645597369) bank190162 +190163 POINT(40.64992640234023 74.58029056027574) bank190163 +190164 POINT(41.44709514902355 73.39873853503022) bank190164 +190165 POINT(41.43091714744332 73.56294059899975) bank190165 +190166 POINT(40.74874145155677 73.24738268909547) bank190166 +190167 POINT(41.116164840561645 73.25914657572582) bank190167 +190168 POINT(40.80899875551583 74.73918866794517) bank190168 +190169 POINT(40.291601441587474 73.18544072321397) bank190169 +190170 POINT(40.15950012020777 73.6508734148328) bank190170 +190171 POINT(40.97519873258989 73.10043468157295) bank190171 +190172 POINT(40.80340911550127 74.39735373984982) bank190172 +190173 POINT(40.057882572757066 73.19336549122607) bank190173 +190174 POINT(40.84942969507188 73.26909312187497) bank190174 +190175 POINT(40.82396931966911 73.64470520902081) bank190175 +190176 POINT(40.92640354798595 73.37054937187293) bank190176 +190177 POINT(40.18617831723461 73.51775704323353) bank190177 +190178 POINT(40.60803260579261 73.80828698967382) bank190178 +190179 POINT(40.87420993471582 73.15546152120103) bank190179 +190180 POINT(40.37281630002846 73.36564067515386) bank190180 +190181 POINT(40.73352658959171 73.84392566210957) bank190181 +190182 POINT(41.48566730005418 74.81373069760174) bank190182 +190183 POINT(40.27172125304454 73.69990383500237) bank190183 +190184 POINT(40.39327403146672 73.4366196621924) bank190184 +190185 POINT(40.898974746953535 74.77946966693327) bank190185 +190186 POINT(40.41618301783531 74.57057580667636) bank190186 +190187 POINT(40.18583069712335 73.76037507734114) bank190187 +190188 POINT(39.73275492399848 74.98036746033308) bank190188 +190189 POINT(40.76953027196116 74.22328000240829) bank190189 +190190 POINT(40.30547678630792 73.14130249974684) bank190190 +190191 POINT(41.6223666009318 74.85476673179164) bank190191 +190192 POINT(40.529659529782336 74.76204903140592) bank190192 +190193 POINT(40.51435424971045 73.15463209410161) bank190193 +190194 POINT(41.21029680985554 73.6517157417129) bank190194 +190195 POINT(40.29417278188821 74.53445676203512) bank190195 +190196 POINT(41.42243352584464 73.10461817346581) bank190196 +190197 POINT(39.7418204034805 73.5227412258424) bank190197 +190198 POINT(40.896708699036694 73.55169751113351) bank190198 +190199 POINT(39.893102408728716 74.76411062619363) bank190199 +190200 POINT(40.79238214686001 74.82920265361042) bank190200 +190201 POINT(41.30014771800428 73.7554075563617) bank190201 +190202 POINT(41.46894406374458 74.85109066489689) bank190202 +190203 POINT(40.94051870571958 74.4287020604188) bank190203 +190204 POINT(40.7875804817561 74.173962523578) bank190204 +190205 POINT(41.00219971543222 74.75224114749159) bank190205 +190206 POINT(40.48347583238425 74.7855961202189) bank190206 +190207 POINT(39.7408471476692 73.24002620490486) bank190207 +190208 POINT(41.5480115588683 74.95839789866257) bank190208 +190209 POINT(40.004060002823614 73.31976699846949) bank190209 +190210 POINT(41.01945758131652 73.06461445518025) bank190210 +190211 POINT(40.26560212415618 73.6216096728247) bank190211 +190212 POINT(41.37322053370942 73.06321727838298) bank190212 +190213 POINT(40.579149426317834 74.1107563806403) bank190213 +190214 POINT(40.06841800588333 73.83106684107742) bank190214 +190215 POINT(41.156464269499956 73.87181084089131) bank190215 +190216 POINT(39.94672078569055 74.68753835084541) bank190216 +190217 POINT(40.690125237688875 73.64747276523329) bank190217 +190218 POINT(41.45184489564843 73.28074361842827) bank190218 +190219 POINT(40.301557660291394 74.12632926878054) bank190219 +190220 POINT(40.94541438945402 73.39089507843849) bank190220 +190221 POINT(41.107275308071635 74.07957407776127) bank190221 +190222 POINT(41.10572167508865 74.10032221368148) bank190222 +190223 POINT(41.449265045726655 74.23631050522114) bank190223 +190224 POINT(40.76236604230837 73.58646383779777) bank190224 +190225 POINT(41.11466519695288 74.53483365455745) bank190225 +190226 POINT(40.57247700148439 74.9694422027241) bank190226 +190227 POINT(41.56869737017388 73.27917059609078) bank190227 +190228 POINT(41.23996554548024 73.37396439377896) bank190228 +190229 POINT(39.781550349115875 74.94200074854446) bank190229 +190230 POINT(41.13788649300126 73.70849684746685) bank190230 +190231 POINT(40.816204573138236 73.68819983446869) bank190231 +190232 POINT(39.8917138165436 74.67088855208732) bank190232 +190233 POINT(40.3394229072179 74.29408903102033) bank190233 +190234 POINT(40.923592811154656 73.62234719313093) bank190234 +190235 POINT(40.62136650753884 74.34403063801895) bank190235 +190236 POINT(40.48330580631873 73.61699186851637) bank190236 +190237 POINT(39.85723333606981 73.35759275560014) bank190237 +190238 POINT(40.15562727562694 73.12249833012288) bank190238 +190239 POINT(41.31932725082989 74.77709598299839) bank190239 +190240 POINT(39.747341327712945 74.76805647226938) bank190240 +190241 POINT(41.18185585197409 73.17701439481422) bank190241 +190242 POINT(39.887801043593576 74.70253260832301) bank190242 +190243 POINT(40.67110119043322 74.15252238025757) bank190243 +190244 POINT(39.83910025368674 73.04205420348914) bank190244 +190245 POINT(40.02750730542139 74.48757372199569) bank190245 +190246 POINT(40.93015532343804 74.30640767981495) bank190246 +190247 POINT(41.24146829070823 73.81658321565496) bank190247 +190248 POINT(40.82403677539622 74.49595144536774) bank190248 +190249 POINT(40.88748260247752 73.06105524858262) bank190249 +190250 POINT(39.96373502452029 73.23473024122544) bank190250 +190251 POINT(39.76413113884617 73.80831529526047) bank190251 +190252 POINT(40.51220998447248 73.33883149211665) bank190252 +190253 POINT(41.170881216959295 74.04886364297937) bank190253 +190254 POINT(40.79119538930804 74.29896780523067) bank190254 +190255 POINT(40.115125450268685 73.64403632083453) bank190255 +190256 POINT(41.46551187761141 73.93003598799291) bank190256 +190257 POINT(40.86827362211679 73.20256978210126) bank190257 +190258 POINT(39.97917616698397 74.44795850387275) bank190258 +190259 POINT(40.28004861429515 74.97951329409257) bank190259 +190260 POINT(39.94217835111685 74.91463463843009) bank190260 +190261 POINT(40.98353414416073 73.29251638695513) bank190261 +190262 POINT(40.53594198502201 73.0378452940532) bank190262 +190263 POINT(40.56713602194006 73.31578885674509) bank190263 +190264 POINT(40.47610250242156 73.67652872923637) bank190264 +190265 POINT(39.94454334949901 73.66612363889921) bank190265 +190266 POINT(40.28264259116747 74.13765174373458) bank190266 +190267 POINT(40.83498041391236 73.75320012045297) bank190267 +190268 POINT(40.39932101692482 74.91838728108486) bank190268 +190269 POINT(40.50940048214898 74.71421082607642) bank190269 +190270 POINT(40.38460141946873 73.20553557167594) bank190270 +190271 POINT(39.78680330185005 74.17328711426235) bank190271 +190272 POINT(41.059885160233144 73.67868399150571) bank190272 +190273 POINT(41.62538915055418 73.25538887189494) bank190273 +190274 POINT(40.08720003780889 74.67669351468274) bank190274 +190275 POINT(40.24523494975715 74.48577525623381) bank190275 +190276 POINT(40.523190777598906 73.11922800558638) bank190276 +190277 POINT(40.51904877936497 74.13538591370124) bank190277 +190278 POINT(41.35014305710401 74.51506349923436) bank190278 +190279 POINT(40.327716366591766 74.20937950684016) bank190279 +190280 POINT(40.8014973680322 73.10784346067278) bank190280 +190281 POINT(40.291799624794244 74.94371560224383) bank190281 +190282 POINT(39.76397369889109 74.05050628794136) bank190282 +190283 POINT(40.39853070571661 74.69405095939842) bank190283 +190284 POINT(41.6700223361369 73.19332165568204) bank190284 +190285 POINT(40.74499603267751 73.9772438809583) bank190285 +190286 POINT(41.57145281187137 73.11906088366635) bank190286 +190287 POINT(40.28808493951187 74.11261356591875) bank190287 +190288 POINT(40.97740170055148 73.1742563813233) bank190288 +190289 POINT(39.99530378431135 74.68606983892634) bank190289 +190290 POINT(40.979519382580285 73.43150231662707) bank190290 +190291 POINT(39.97309664355025 74.39710753789593) bank190291 +190292 POINT(40.27331097805697 73.09116177799535) bank190292 +190293 POINT(41.71187255211611 74.81877598937399) bank190293 +190294 POINT(40.6585770263595 74.16495776668695) bank190294 +190295 POINT(40.278142948732935 74.18106484304944) bank190295 +190296 POINT(41.66778782350894 74.69143610458694) bank190296 +190297 POINT(39.89453466753249 74.39806666264214) bank190297 +190298 POINT(40.039867677803834 73.90048438327743) bank190298 +190299 POINT(41.576844293826476 73.14217545177029) bank190299 +190300 POINT(40.467185538963435 74.93878283071774) bank190300 +190301 POINT(40.23556927503521 73.0408353653673) bank190301 +190302 POINT(41.363623759225455 74.15717812238412) bank190302 +190303 POINT(40.373313246995274 74.47469291816103) bank190303 +190304 POINT(40.54780839208988 74.4551969187477) bank190304 +190305 POINT(40.426152272303774 73.2536741806594) bank190305 +190306 POINT(41.320574813883105 73.24840190709578) bank190306 +190307 POINT(41.61113187966071 73.89806060632158) bank190307 +190308 POINT(41.19065877332651 74.58428275458715) bank190308 +190309 POINT(41.57785958289093 74.89802026474484) bank190309 +190310 POINT(41.543245225778016 74.95664482206901) bank190310 +190311 POINT(41.17477011912685 74.58226962840656) bank190311 +190312 POINT(40.05807194982754 73.75524605528463) bank190312 +190313 POINT(40.68903157603231 73.45388034378374) bank190313 +190314 POINT(40.32416766588743 74.38056113849618) bank190314 +190315 POINT(39.73047269830726 74.99848414488436) bank190315 +190316 POINT(41.64294221643687 74.44948277468214) bank190316 +190317 POINT(40.30255906987172 74.38805104587034) bank190317 +190318 POINT(41.19350236451865 74.09135281459308) bank190318 +190319 POINT(39.89761166632334 74.49523487105122) bank190319 +190320 POINT(40.34676361271293 74.40707110107194) bank190320 +190321 POINT(41.28517139675761 73.16007875688051) bank190321 +190322 POINT(40.45922045698089 74.3767644204773) bank190322 +190323 POINT(40.14448848340862 73.89643820191984) bank190323 +190324 POINT(41.02064756360518 74.62965982868594) bank190324 +190325 POINT(40.2976389419139 74.86935528888993) bank190325 +190326 POINT(41.55309091737811 73.19623316091256) bank190326 +190327 POINT(41.30425845975841 73.57043796565048) bank190327 +190328 POINT(40.74691584111357 73.97950138265008) bank190328 +190329 POINT(40.37235472755611 73.58740476729118) bank190329 +190330 POINT(40.26016102168585 73.38441036954204) bank190330 +190331 POINT(41.64244925877535 74.60068871461182) bank190331 +190332 POINT(41.34900342475003 74.32835263608644) bank190332 +190333 POINT(40.06909120579022 73.79556594288526) bank190333 +190334 POINT(40.045323459669845 74.34770178132888) bank190334 +190335 POINT(40.42262078577415 74.73444865243391) bank190335 +190336 POINT(39.79291937315717 73.32368085807259) bank190336 +190337 POINT(40.47278808827261 74.43644883761985) bank190337 +190338 POINT(40.60967866572866 74.7743960981448) bank190338 +190339 POINT(40.307397556045984 73.01792295454956) bank190339 +190340 POINT(41.293550803692966 73.41712757786019) bank190340 +190341 POINT(40.128142933787565 74.00755630071211) bank190341 +190342 POINT(40.232350228485295 73.84895447362125) bank190342 +190343 POINT(41.42421135372471 73.44062483568665) bank190343 +190344 POINT(41.1696061635946 73.79911587848989) bank190344 +190345 POINT(41.55972671659458 74.3908445419585) bank190345 +190346 POINT(39.8214008221734 74.70271736475797) bank190346 +190347 POINT(40.47651710599873 73.47741119912774) bank190347 +190348 POINT(41.18994427062879 74.82538065047594) bank190348 +190349 POINT(40.1466903754884 73.10046920041397) bank190349 +190350 POINT(40.041701573916846 74.64694683814709) bank190350 +190351 POINT(40.57601183192208 74.57278915489894) bank190351 +190352 POINT(40.59389441950002 73.4019462585964) bank190352 +190353 POINT(39.75309612362738 74.29730288917006) bank190353 +190354 POINT(40.96846758814806 74.00213561232356) bank190354 +190355 POINT(39.784551113193935 73.54491923912107) bank190355 +190356 POINT(41.14093590834963 73.86519445215058) bank190356 +190357 POINT(41.420616053838884 74.52120458951212) bank190357 +190358 POINT(41.485996049768985 74.05704019877204) bank190358 +190359 POINT(39.98775101516627 74.50277707712127) bank190359 +190360 POINT(39.749863592353016 74.77709071589338) bank190360 +190361 POINT(41.41427471478716 73.17916174229558) bank190361 +190362 POINT(40.35732236574036 74.38433955127665) bank190362 +190363 POINT(41.62830595442911 74.46134647198866) bank190363 +190364 POINT(40.98505493618739 73.09494020069661) bank190364 +190365 POINT(41.66632798462565 73.05108052252653) bank190365 +190366 POINT(41.572105360314744 73.3920891299881) bank190366 +190367 POINT(41.25827075673785 73.35853650461917) bank190367 +190368 POINT(40.65724021330478 73.32443323350759) bank190368 +190369 POINT(40.05340836782656 74.3821708789571) bank190369 +190370 POINT(40.16813796546395 74.76820814325666) bank190370 +190371 POINT(40.68459032595227 74.01191328085756) bank190371 +190372 POINT(39.90043119863259 73.59711944875593) bank190372 +190373 POINT(39.78828365558446 74.45992352125768) bank190373 +190374 POINT(40.99415341740707 74.46873791905358) bank190374 +190375 POINT(41.34899671895109 74.09769332864836) bank190375 +190376 POINT(39.72277243767954 74.23866515290186) bank190376 +190377 POINT(40.708888628165546 73.02773793232704) bank190377 +190378 POINT(40.129874964485914 74.2066110796465) bank190378 +190379 POINT(41.59521229482964 73.27401268998159) bank190379 +190380 POINT(40.70795390674851 74.8326676311048) bank190380 +190381 POINT(41.37985629339201 73.89047281704154) bank190381 +190382 POINT(41.17617391012243 73.60757777679873) bank190382 +190383 POINT(40.85790777251408 73.28316221638103) bank190383 +190384 POINT(40.69716641548025 73.505612540875) bank190384 +190385 POINT(39.86720498875662 73.8608743563895) bank190385 +190386 POINT(40.471659739459284 73.59713250648524) bank190386 +190387 POINT(41.37797507666237 74.99596507243668) bank190387 +190388 POINT(41.21275082112566 74.04601813056104) bank190388 +190389 POINT(40.3969115096334 74.57495002629774) bank190389 +190390 POINT(41.68596540632963 73.44180438501287) bank190390 +190391 POINT(41.16421949005336 73.36807535080102) bank190391 +190392 POINT(41.62076264605446 74.92030790312255) bank190392 +190393 POINT(40.13092378981362 74.8006205356207) bank190393 +190394 POINT(40.01947526765115 73.72482063775445) bank190394 +190395 POINT(41.486284868478364 75.00306712191687) bank190395 +190396 POINT(39.907789281261465 74.58732182727603) bank190396 +190397 POINT(40.66947238183987 73.32672098356161) bank190397 +190398 POINT(40.382032341024484 73.71830273039623) bank190398 +190399 POINT(40.250245723997594 73.07676352090516) bank190399 +190400 POINT(39.74956182987869 73.63114325717378) bank190400 +190401 POINT(40.52665485734922 73.7421170670697) bank190401 +190402 POINT(41.41287494596394 73.71777926276904) bank190402 +190403 POINT(40.27136282147898 74.7503095361593) bank190403 +190404 POINT(41.48603317524291 74.9581273825838) bank190404 +190405 POINT(40.27491030531089 74.88967201541075) bank190405 +190406 POINT(39.7856428332799 73.19097620478387) bank190406 +190407 POINT(41.352587665517156 73.63421354180561) bank190407 +190408 POINT(41.52474103723179 73.58510575984424) bank190408 +190409 POINT(41.677055172234 73.22634433396082) bank190409 +190410 POINT(40.016519000621386 74.73942713927495) bank190410 +190411 POINT(41.597547199445216 73.71324449948814) bank190411 +190412 POINT(41.57413965071167 73.49290099538061) bank190412 +190413 POINT(41.221151780737095 73.0735107944184) bank190413 +190414 POINT(40.755110990761324 74.70792742411923) bank190414 +190415 POINT(41.44349536692209 74.87619288680163) bank190415 +190416 POINT(39.971472375879145 73.33480777506165) bank190416 +190417 POINT(40.89062794916897 74.84012795979824) bank190417 +190418 POINT(41.38322839109064 74.98030844967354) bank190418 +190419 POINT(41.673237320264654 73.64792304018687) bank190419 +190420 POINT(40.59384007548184 74.3204614014887) bank190420 +190421 POINT(40.036128364606995 73.67204137473547) bank190421 +190422 POINT(41.349954246296235 74.78000287851067) bank190422 +190423 POINT(41.67474144586573 73.5378475355464) bank190423 +190424 POINT(40.25271032830877 74.64627744267489) bank190424 +190425 POINT(40.24309334635894 73.59296071592213) bank190425 +190426 POINT(40.585980037631074 74.12213274856042) bank190426 +190427 POINT(40.19015303017896 74.25249647869775) bank190427 +190428 POINT(41.317192790658915 74.62650034117144) bank190428 +190429 POINT(41.14076616044008 74.23093258490606) bank190429 +190430 POINT(41.13621220862047 73.73456090508418) bank190430 +190431 POINT(39.88649521531148 74.66650347059216) bank190431 +190432 POINT(40.68875064843398 73.12568156519671) bank190432 +190433 POINT(39.724017596186805 74.77706245484212) bank190433 +190434 POINT(39.90177087861801 73.56298767774328) bank190434 +190435 POINT(40.99679366522844 74.81411125452638) bank190435 +190436 POINT(41.47705169553384 73.88925814332744) bank190436 +190437 POINT(39.81266797804176 74.17833285008955) bank190437 +190438 POINT(39.87812870078497 74.97222641017127) bank190438 +190439 POINT(40.91887007038482 73.35287696408065) bank190439 +190440 POINT(39.858545754916285 73.18354452023382) bank190440 +190441 POINT(40.99591347246499 73.55941684485289) bank190441 +190442 POINT(40.38044714889403 74.83363157160538) bank190442 +190443 POINT(41.31318932438007 73.62830167962751) bank190443 +190444 POINT(41.22304262441729 73.19135780198526) bank190444 +190445 POINT(40.378178971992156 73.52122032573695) bank190445 +190446 POINT(41.34896481256766 73.80631897100322) bank190446 +190447 POINT(39.829040763830996 73.09676083695508) bank190447 +190448 POINT(40.364589252201725 74.30906724932309) bank190448 +190449 POINT(41.497840331512876 73.67836260503334) bank190449 +190450 POINT(40.951899821919554 74.40157345112061) bank190450 +190451 POINT(39.903001566201176 74.43869761958571) bank190451 +190452 POINT(40.148436105484166 73.16288938445474) bank190452 +190453 POINT(39.93178263995495 74.65748743697442) bank190453 +190454 POINT(39.8223759919045 74.63195873666334) bank190454 +190455 POINT(40.26877117981587 73.30902959552951) bank190455 +190456 POINT(40.20362233012989 73.71746299378903) bank190456 +190457 POINT(40.60420651408037 74.67938052909894) bank190457 +190458 POINT(39.80763429089863 73.85007900722562) bank190458 +190459 POINT(39.94896951445003 73.10430189042759) bank190459 +190460 POINT(41.18262330223981 74.00317807253478) bank190460 +190461 POINT(41.32131604998685 73.56271732532292) bank190461 +190462 POINT(40.32290381093754 74.51648220662666) bank190462 +190463 POINT(40.489718994987626 74.00431571516137) bank190463 +190464 POINT(40.885243266192994 73.82146081159952) bank190464 +190465 POINT(40.19533883175247 73.64305137997154) bank190465 +190466 POINT(41.00393143962866 73.73104710417624) bank190466 +190467 POINT(40.70789976241726 74.97743040522373) bank190467 +190468 POINT(40.911454415348366 73.90621689861503) bank190468 +190469 POINT(41.05306050720882 73.05553101422457) bank190469 +190470 POINT(41.11096669116737 74.52633238739399) bank190470 +190471 POINT(40.59995977481357 74.69806371881604) bank190471 +190472 POINT(40.64714843738824 74.951758518305) bank190472 +190473 POINT(41.13032859792582 74.03162250436137) bank190473 +190474 POINT(40.928277408663796 73.68674728051853) bank190474 +190475 POINT(40.59602926479743 73.83644958958963) bank190475 +190476 POINT(40.972290271704075 74.70574690724638) bank190476 +190477 POINT(40.331082305285705 74.56360347284627) bank190477 +190478 POINT(41.55280825115587 74.43180901921168) bank190478 +190479 POINT(40.8835861720827 73.60647933997053) bank190479 +190480 POINT(40.55928286164922 74.30935973514161) bank190480 +190481 POINT(40.15767302893193 73.90091149100323) bank190481 +190482 POINT(41.20910946704857 74.84516323102797) bank190482 +190483 POINT(40.98916977749382 74.73948907991263) bank190483 +190484 POINT(39.762683848186235 74.10253422988544) bank190484 +190485 POINT(41.34208436548173 74.05074455259388) bank190485 +190486 POINT(41.40787292706189 73.19577609574334) bank190486 +190487 POINT(41.07505776275856 74.26506363797856) bank190487 +190488 POINT(41.293169748314654 73.73745834059855) bank190488 +190489 POINT(40.453552273473676 73.03197028489723) bank190489 +190490 POINT(40.746193069543175 74.35163827739808) bank190490 +190491 POINT(40.177510263300505 73.71653925447892) bank190491 +190492 POINT(40.04078991898131 73.30556839773158) bank190492 +190493 POINT(39.99878630685301 74.24681583487224) bank190493 +190494 POINT(40.877473327878974 73.16063501607846) bank190494 +190495 POINT(40.26436868142672 73.9444941857137) bank190495 +190496 POINT(41.61812943610992 74.93532489711257) bank190496 +190497 POINT(41.59997572247242 73.3032834490733) bank190497 +190498 POINT(40.090892897098044 73.16004677855629) bank190498 +190499 POINT(40.85565713714481 73.0772331182778) bank190499 +190500 POINT(40.17232465360494 73.1633381526935) bank190500 +190501 POINT(40.841104945550896 74.24895848363735) bank190501 +190502 POINT(40.99011761619171 74.503732489619) bank190502 +190503 POINT(39.94570876713567 74.03510646849749) bank190503 +190504 POINT(41.177667761297215 73.36998163887127) bank190504 +190505 POINT(41.23334688523027 74.50579327155499) bank190505 +190506 POINT(41.27585481277945 73.98447232144368) bank190506 +190507 POINT(40.258610105576615 74.07005674243977) bank190507 +190508 POINT(41.18374096122038 73.5027818178455) bank190508 +190509 POINT(39.83885003452911 73.36514995251615) bank190509 +190510 POINT(40.664161162126135 74.00076258372066) bank190510 +190511 POINT(39.81717935097307 74.54936264054) bank190511 +190512 POINT(40.92957956367483 73.94698279903572) bank190512 +190513 POINT(39.82512734760614 73.26481361386081) bank190513 +190514 POINT(39.87384919395507 73.53427526911203) bank190514 +190515 POINT(41.29331048028091 73.57235230964724) bank190515 +190516 POINT(41.56519708186459 73.10842948194856) bank190516 +190517 POINT(39.791018483346605 74.3051708215301) bank190517 +190518 POINT(41.21201224803101 73.37561168560016) bank190518 +190519 POINT(40.86482621911649 73.23676275072684) bank190519 +190520 POINT(40.267169410155724 73.95534037018669) bank190520 +190521 POINT(40.7987558921502 73.3704614771469) bank190521 +190522 POINT(41.57923921263713 73.1638619986288) bank190522 +190523 POINT(41.54175072052564 74.09056659539662) bank190523 +190524 POINT(40.88227325050265 74.52953075408672) bank190524 +190525 POINT(40.323473472837016 74.38703675091597) bank190525 +190526 POINT(40.44564889974858 73.51297804900862) bank190526 +190527 POINT(41.69591469612879 74.8495812186291) bank190527 +190528 POINT(40.49240426962862 73.751869002958) bank190528 +190529 POINT(40.41126001274193 75.00333276133135) bank190529 +190530 POINT(40.283146959779344 73.60423468137901) bank190530 +190531 POINT(41.02212691816301 73.9373870389849) bank190531 +190532 POINT(41.1688760136054 73.2825652690064) bank190532 +190533 POINT(40.319219245619145 74.96544301514513) bank190533 +190534 POINT(40.36388668419253 73.77940067248817) bank190534 +190535 POINT(40.601577508957995 74.70196997737598) bank190535 +190536 POINT(40.65754199116084 74.83005551569892) bank190536 +190537 POINT(40.46931268257303 73.55602480662489) bank190537 +190538 POINT(39.744468878751675 74.7714219078464) bank190538 +190539 POINT(40.67139037599668 74.40140512742454) bank190539 +190540 POINT(40.3009584952787 74.84739171841768) bank190540 +190541 POINT(40.52187227841983 73.60795937943065) bank190541 +190542 POINT(41.52771127142749 74.5618122122078) bank190542 +190543 POINT(39.89887290150915 74.70919166634596) bank190543 +190544 POINT(40.556023145347524 73.98165279043144) bank190544 +190545 POINT(40.78127767365797 74.90214445880295) bank190545 +190546 POINT(40.82313890520395 73.3705238625726) bank190546 +190547 POINT(40.0857600496846 74.72228242013779) bank190547 +190548 POINT(41.322984456140006 73.51960737970444) bank190548 +190549 POINT(40.87722994061069 73.42509707296647) bank190549 +190550 POINT(40.669770663242666 74.79400724674312) bank190550 +190551 POINT(39.858268162751685 74.47448088737683) bank190551 +190552 POINT(39.739323841165046 73.94420678016427) bank190552 +190553 POINT(41.410423489851276 73.40097426797917) bank190553 +190554 POINT(39.96526064858312 74.52766636000626) bank190554 +190555 POINT(40.199471959591605 73.33522382434637) bank190555 +190556 POINT(41.51736037966933 74.37444957583547) bank190556 +190557 POINT(41.04544153858134 74.52276706921361) bank190557 +190558 POINT(40.687338442625276 73.3591780593732) bank190558 +190559 POINT(41.04636491953596 74.63527458007238) bank190559 +190560 POINT(40.45953949805511 74.66984517546233) bank190560 +190561 POINT(40.003146575499116 74.13737216518625) bank190561 +190562 POINT(39.91006141957675 74.52616838697776) bank190562 +190563 POINT(41.28238214485195 73.98867091857203) bank190563 +190564 POINT(39.845164048493565 73.82931165865281) bank190564 +190565 POINT(40.23155993404787 74.10155507425367) bank190565 +190566 POINT(41.39676640276552 74.826226245289) bank190566 +190567 POINT(40.19786522200142 74.26776545248877) bank190567 +190568 POINT(39.83259630809589 73.90085695909461) bank190568 +190569 POINT(41.6034966261587 74.3066461960799) bank190569 +190570 POINT(41.63043430760654 73.02432270413117) bank190570 +190571 POINT(41.469708041500596 74.21133717074584) bank190571 +190572 POINT(41.07690993581565 74.44650993896319) bank190572 +190573 POINT(41.057836161073034 73.42791196171255) bank190573 +190574 POINT(41.00546494800272 73.75703169017662) bank190574 +190575 POINT(40.02843903673302 74.50039777202907) bank190575 +190576 POINT(39.98479254801962 74.83730788457193) bank190576 +190577 POINT(39.96559984655731 74.91977842668253) bank190577 +190578 POINT(40.877694276914745 73.97319856674272) bank190578 +190579 POINT(41.190144384772466 74.9638747606707) bank190579 +190580 POINT(40.980674557787886 74.95361757315916) bank190580 +190581 POINT(41.316313987014304 73.55268752334912) bank190581 +190582 POINT(39.937503583635774 74.74923479312464) bank190582 +190583 POINT(40.348242026294564 74.13112055566741) bank190583 +190584 POINT(39.90916003196102 74.95050958665271) bank190584 +190585 POINT(40.716766089725034 74.68407115550517) bank190585 +190586 POINT(40.59828084265335 74.79598434321461) bank190586 +190587 POINT(40.908441517634785 74.86555671813667) bank190587 +190588 POINT(40.10454495852521 74.47413359291393) bank190588 +190589 POINT(41.06550491368794 74.43153492039819) bank190589 +190590 POINT(39.94195804453638 74.02701380632004) bank190590 +190591 POINT(41.49693714707257 74.46019010832174) bank190591 +190592 POINT(40.53750929401376 73.34022137771193) bank190592 +190593 POINT(40.424807115679315 74.09953828327922) bank190593 +190594 POINT(41.01414993686301 73.32912014165942) bank190594 +190595 POINT(41.21772900941754 74.6260903796207) bank190595 +190596 POINT(41.376837457701036 73.5298749466049) bank190596 +190597 POINT(40.035911330829556 73.60464949562932) bank190597 +190598 POINT(41.43993847658744 73.74392742038054) bank190598 +190599 POINT(40.65153437494637 73.93436043973819) bank190599 +190600 POINT(41.04102944636901 74.08829477381657) bank190600 +190601 POINT(41.147782320146405 73.29242614575881) bank190601 +190602 POINT(40.55932723069647 74.34513538667511) bank190602 +190603 POINT(40.10181885334709 74.80903420197265) bank190603 +190604 POINT(40.66362084523456 74.88442212255065) bank190604 +190605 POINT(39.84066739751488 73.00670267017624) bank190605 +190606 POINT(40.106100599033816 74.31350512972995) bank190606 +190607 POINT(41.25196147006555 73.24882466370491) bank190607 +190608 POINT(39.713022519035626 73.99533002519739) bank190608 +190609 POINT(39.96379189016482 74.62148106478914) bank190609 +190610 POINT(41.46386066066764 73.85010202783462) bank190610 +190611 POINT(40.627302798752744 74.51769733801014) bank190611 +190612 POINT(40.9794072300405 73.24903050261261) bank190612 +190613 POINT(40.14359014487902 73.24494173300005) bank190613 +190614 POINT(41.19608956597398 73.2853535666936) bank190614 +190615 POINT(41.60897157002249 73.81363694449279) bank190615 +190616 POINT(40.06634936023486 74.78013553825924) bank190616 +190617 POINT(41.191748742792065 74.48941775474955) bank190617 +190618 POINT(40.40180935904209 73.01670423034275) bank190618 +190619 POINT(41.441458756065316 73.58814856876795) bank190619 +190620 POINT(41.033949274515564 73.36425400460082) bank190620 +190621 POINT(40.38830956634637 74.36298967079372) bank190621 +190622 POINT(41.36666267007646 74.98593895180875) bank190622 +190623 POINT(39.9882968277315 74.46844444506728) bank190623 +190624 POINT(40.771717776072485 74.06827749190964) bank190624 +190625 POINT(40.028283460351226 73.79617218421241) bank190625 +190626 POINT(40.31567942391268 73.2909244563037) bank190626 +190627 POINT(40.14516392356039 74.24907574008725) bank190627 +190628 POINT(40.960761736440105 73.44840012313387) bank190628 +190629 POINT(41.269044636275055 74.86659234530671) bank190629 +190630 POINT(41.472946605551904 73.79106178204248) bank190630 +190631 POINT(40.476052587785226 74.0004369592289) bank190631 +190632 POINT(39.96887183689948 73.95729911114913) bank190632 +190633 POINT(41.53436331515083 73.46402154238444) bank190633 +190634 POINT(40.08277622862721 74.57198577234789) bank190634 +190635 POINT(40.78760536675979 73.34973901606438) bank190635 +190636 POINT(41.52400151068791 73.55979119144529) bank190636 +190637 POINT(41.56266148357047 74.92903290352885) bank190637 +190638 POINT(41.59866962867718 73.69859398421612) bank190638 +190639 POINT(40.903045808357824 73.19139491007026) bank190639 +190640 POINT(40.66518661182644 73.34363967686534) bank190640 +190641 POINT(40.349848382403195 73.16335293853092) bank190641 +190642 POINT(40.32255169558375 73.97839758606955) bank190642 +190643 POINT(41.517044313546904 73.52151648189643) bank190643 +190644 POINT(40.41440699407798 73.08687299293122) bank190644 +190645 POINT(41.148479428540284 73.97556974723679) bank190645 +190646 POINT(39.77003662555137 74.07693190395544) bank190646 +190647 POINT(41.62937824220707 74.17780363398172) bank190647 +190648 POINT(39.85375021595841 73.71821892083385) bank190648 +190649 POINT(41.03523134427161 73.70498148191328) bank190649 +190650 POINT(41.70057827836043 74.6901885477824) bank190650 +190651 POINT(40.291423944171555 74.22737652070575) bank190651 +190652 POINT(40.965139727120594 74.23979094376627) bank190652 +190653 POINT(40.84322043470604 73.31819172950213) bank190653 +190654 POINT(40.014091662703926 73.71787598257816) bank190654 +190655 POINT(40.31142329757259 73.14796832339952) bank190655 +190656 POINT(40.84805988279416 73.34440822154963) bank190656 +190657 POINT(40.82975246316753 73.57482811643686) bank190657 +190658 POINT(40.0134564142151 74.47937871121015) bank190658 +190659 POINT(41.471874687173134 73.07767701819422) bank190659 +190660 POINT(39.738403382862884 74.52159447764832) bank190660 +190661 POINT(39.85209416251342 73.38456898532688) bank190661 +190662 POINT(40.80471642304743 73.59511159349329) bank190662 +190663 POINT(41.15985881124009 74.15617051375003) bank190663 +190664 POINT(41.31233849777128 74.55730086250425) bank190664 +190665 POINT(40.88756520908249 73.53109599632641) bank190665 +190666 POINT(39.956176965420845 74.69947078535725) bank190666 +190667 POINT(40.898399240569795 74.8503081091284) bank190667 +190668 POINT(40.68496200180973 73.07939460704515) bank190668 +190669 POINT(40.544834992627706 74.46774992388727) bank190669 +190670 POINT(41.15369852837167 74.78385583982437) bank190670 +190671 POINT(40.69261312594999 74.02377075739093) bank190671 +190672 POINT(40.6392808786395 73.61622977927733) bank190672 +190673 POINT(41.62633717437091 73.58380811854904) bank190673 +190674 POINT(39.849610662737625 74.50535890938178) bank190674 +190675 POINT(40.930136283790624 74.79580100430236) bank190675 +190676 POINT(40.707988942455394 74.51122964535297) bank190676 +190677 POINT(40.93252998899821 74.60805873151563) bank190677 +190678 POINT(41.03760279906735 73.65489847006917) bank190678 +190679 POINT(40.782475023767105 73.95059857357727) bank190679 +190680 POINT(41.01921614315548 74.38251634801372) bank190680 +190681 POINT(40.11286105849787 73.44026856217873) bank190681 +190682 POINT(40.57159474715054 73.74660195614658) bank190682 +190683 POINT(40.58586243848788 74.76273904055672) bank190683 +190684 POINT(41.56091978809404 74.00399025740253) bank190684 +190685 POINT(41.1714389062222 74.63294523106529) bank190685 +190686 POINT(40.15382259779816 74.3283585752253) bank190686 +190687 POINT(40.34672057362025 73.09489841440757) bank190687 +190688 POINT(41.280819519732894 73.2410297449183) bank190688 +190689 POINT(40.02417864435891 74.53569855774116) bank190689 +190690 POINT(39.8776588515444 73.9996633828995) bank190690 +190691 POINT(41.06026402563422 74.78765468284972) bank190691 +190692 POINT(39.81923680146536 73.65651939092402) bank190692 +190693 POINT(40.18886046497358 74.55477530813116) bank190693 +190694 POINT(41.60105153271871 74.98932526489237) bank190694 +190695 POINT(41.20468973427921 74.50379046694746) bank190695 +190696 POINT(39.82465996367704 73.56647840776712) bank190696 +190697 POINT(39.930045961917486 74.17925459566975) bank190697 +190698 POINT(40.22725813179271 73.61239372042684) bank190698 +190699 POINT(41.374584412239756 73.67893931700128) bank190699 +190700 POINT(41.0303801355787 73.81433482257943) bank190700 +190701 POINT(40.848325679868225 73.88005697554064) bank190701 +190702 POINT(40.0592577333356 73.92428454554093) bank190702 +190703 POINT(39.83085333122535 73.26070044960741) bank190703 +190704 POINT(40.73158401261624 73.22432179601891) bank190704 +190705 POINT(39.99411448343638 74.31243893306778) bank190705 +190706 POINT(40.954474247444 73.17049723737138) bank190706 +190707 POINT(41.708394787190166 74.6895097454511) bank190707 +190708 POINT(40.652855798154754 73.49080612137814) bank190708 +190709 POINT(40.280622439158066 74.69971528996186) bank190709 +190710 POINT(40.02596840632392 74.78943203292808) bank190710 +190711 POINT(41.55396467198246 74.95518040477671) bank190711 +190712 POINT(40.40882502526246 73.19166770945094) bank190712 +190713 POINT(41.49295122564858 74.91590087747494) bank190713 +190714 POINT(39.85785450254056 73.42902209756195) bank190714 +190715 POINT(40.75790780259394 73.52052165983083) bank190715 +190716 POINT(41.47764817418206 74.16066305295602) bank190716 +190717 POINT(41.18553256311271 74.45897770091918) bank190717 +190718 POINT(40.121842678454506 73.87341091479112) bank190718 +190719 POINT(40.110239213183085 74.66750653814029) bank190719 +190720 POINT(40.89694664342349 74.53329099503685) bank190720 +190721 POINT(41.32977720963386 73.59405159898247) bank190721 +190722 POINT(40.522321669045844 74.6196233352956) bank190722 +190723 POINT(40.41317186456401 74.73563958485735) bank190723 +190724 POINT(40.40218974985619 73.66923250287468) bank190724 +190725 POINT(41.30410133207259 74.24489057218521) bank190725 +190726 POINT(40.023620134644254 73.12368228044036) bank190726 +190727 POINT(41.4742005264478 73.27648063167264) bank190727 +190728 POINT(41.70430783839862 73.50242935216728) bank190728 +190729 POINT(39.93262600437513 74.1346110228923) bank190729 +190730 POINT(39.73645058145763 73.02207383799734) bank190730 +190731 POINT(41.433610715438384 73.0980804948028) bank190731 +190732 POINT(41.06013287164553 74.47736791523765) bank190732 +190733 POINT(41.11003380781492 74.05822922499235) bank190733 +190734 POINT(39.770131252736135 74.89777842366037) bank190734 +190735 POINT(41.50819998259808 73.26827815126954) bank190735 +190736 POINT(41.486148404515376 74.77175139232543) bank190736 +190737 POINT(40.99517160930843 73.14845589789314) bank190737 +190738 POINT(39.93463358181746 73.42717769000515) bank190738 +190739 POINT(41.04640908055004 73.6660146228886) bank190739 +190740 POINT(41.3973231331911 73.4184778960758) bank190740 +190741 POINT(41.044630760303264 73.30753030968826) bank190741 +190742 POINT(40.38047475287114 74.02650531135538) bank190742 +190743 POINT(39.89430532928273 74.15595667048994) bank190743 +190744 POINT(40.99282700054589 74.45655486917786) bank190744 +190745 POINT(40.37988318103345 73.18598909526771) bank190745 +190746 POINT(39.85713174880096 74.42895849652338) bank190746 +190747 POINT(40.662892397195264 73.79785520936417) bank190747 +190748 POINT(41.47352797700055 73.86138578643721) bank190748 +190749 POINT(39.72263235111829 73.59504331827932) bank190749 +190750 POINT(40.78020756761417 73.37031439482377) bank190750 +190751 POINT(41.05764520336925 74.81767015953625) bank190751 +190752 POINT(40.85891559557169 74.75954675546201) bank190752 +190753 POINT(39.78512022684031 74.69210826424468) bank190753 +190754 POINT(40.624677818555874 74.42718375087017) bank190754 +190755 POINT(40.60551155254892 74.85291888428694) bank190755 +190756 POINT(41.22168229024743 74.93654802757433) bank190756 +190757 POINT(41.59437573300893 74.71348327228611) bank190757 +190758 POINT(40.881707769969616 74.49211906280595) bank190758 +190759 POINT(40.371486218649686 74.37615113182541) bank190759 +190760 POINT(40.555948789311266 73.09554730881219) bank190760 +190761 POINT(41.39806714917918 73.30498397098096) bank190761 +190762 POINT(40.76565771183942 74.67059859673085) bank190762 +190763 POINT(39.90454314119667 74.88727715127712) bank190763 +190764 POINT(40.359840016130896 73.38616926825024) bank190764 +190765 POINT(40.5949061605336 74.4125711244094) bank190765 +190766 POINT(40.622811271795946 73.23653740633195) bank190766 +190767 POINT(39.71515972996921 73.42698938862681) bank190767 +190768 POINT(41.65766112479657 73.44028184821586) bank190768 +190769 POINT(39.776856486481286 74.08639237582501) bank190769 +190770 POINT(41.53800015629303 74.08311645758859) bank190770 +190771 POINT(39.857105112433615 74.49194668810708) bank190771 +190772 POINT(41.47935196358548 74.4834895808242) bank190772 +190773 POINT(40.027673023896554 74.4449876999777) bank190773 +190774 POINT(41.62282548146026 73.9499137273388) bank190774 +190775 POINT(39.878304836173044 74.33320371954302) bank190775 +190776 POINT(40.360591484579956 74.30255837675882) bank190776 +190777 POINT(41.240971564440926 73.64558580680317) bank190777 +190778 POINT(41.17596588661105 73.20413623298165) bank190778 +190779 POINT(41.462732113582156 73.46052381433101) bank190779 +190780 POINT(40.74022478773998 74.58916207941935) bank190780 +190781 POINT(41.17154094282102 74.6972291920418) bank190781 +190782 POINT(40.762345562437524 75.0038859468335) bank190782 +190783 POINT(40.701801735019735 73.15081404541081) bank190783 +190784 POINT(40.16266726932776 74.70386321242836) bank190784 +190785 POINT(40.689242766448174 74.2082820613868) bank190785 +190786 POINT(40.03674088086355 74.2724915090538) bank190786 +190787 POINT(41.39593500883266 74.8382996301613) bank190787 +190788 POINT(41.57137018876192 73.70806326385807) bank190788 +190789 POINT(41.47413964879534 74.24321702237782) bank190789 +190790 POINT(40.231324539875104 73.5753553899716) bank190790 +190791 POINT(40.432140755120955 74.16901845772934) bank190791 +190792 POINT(40.197528823866364 73.4481946460581) bank190792 +190793 POINT(40.929220980511246 73.22846606483866) bank190793 +190794 POINT(40.595785663283706 74.47736791969238) bank190794 +190795 POINT(40.9888273344519 74.94332287593686) bank190795 +190796 POINT(41.18073224773379 74.60868252719571) bank190796 +190797 POINT(40.05079417177613 74.507594363342) bank190797 +190798 POINT(40.077869647598575 74.98263105454619) bank190798 +190799 POINT(40.19598295441514 74.0061594953845) bank190799 +190800 POINT(39.872725232383694 74.34953971798008) bank190800 +190801 POINT(41.3503223455968 74.74305484416816) bank190801 +190802 POINT(40.53826520219524 73.2408784571635) bank190802 +190803 POINT(41.015787420012124 73.30912799032441) bank190803 +190804 POINT(40.76236774243447 73.57362163329041) bank190804 +190805 POINT(40.93435250768559 73.2030742635932) bank190805 +190806 POINT(41.054546807050976 73.03405861094738) bank190806 +190807 POINT(40.08544483414409 73.13923158092611) bank190807 +190808 POINT(40.16123821445993 74.82869526895593) bank190808 +190809 POINT(40.12258113929178 73.29598238321958) bank190809 +190810 POINT(40.64303522379733 74.1883899741357) bank190810 +190811 POINT(41.172644261349255 74.35233879961562) bank190811 +190812 POINT(40.528067102135495 74.54585674966688) bank190812 +190813 POINT(40.72757066541736 74.08682544765968) bank190813 +190814 POINT(41.46778305371084 73.58740348349004) bank190814 +190815 POINT(39.786346468385396 73.49467123852358) bank190815 +190816 POINT(41.62648942718299 74.50202315362533) bank190816 +190817 POINT(40.37022557122255 73.6100183129765) bank190817 +190818 POINT(40.047921228492406 74.91662129730271) bank190818 +190819 POINT(41.57646242064915 73.85066267696732) bank190819 +190820 POINT(40.02332460089456 74.79110067077993) bank190820 +190821 POINT(41.44186787567346 73.31251007440558) bank190821 +190822 POINT(41.407031139581846 74.74847327044121) bank190822 +190823 POINT(40.65068777544754 73.27292901321445) bank190823 +190824 POINT(40.06359806068222 74.31984835468533) bank190824 +190825 POINT(40.993059715515386 73.14803954112224) bank190825 +190826 POINT(41.058044982156204 73.2017786632543) bank190826 +190827 POINT(40.57737845093687 74.40199016825734) bank190827 +190828 POINT(39.797417650009734 73.62782539201237) bank190828 +190829 POINT(40.324122645019486 74.94053374044016) bank190829 +190830 POINT(40.84528149080223 73.59015346333634) bank190830 +190831 POINT(41.130220468789474 73.84824581093181) bank190831 +190832 POINT(40.621039178692605 73.07551733495241) bank190832 +190833 POINT(40.73884995839878 74.03856351445745) bank190833 +190834 POINT(40.41995374912983 73.06865867987267) bank190834 +190835 POINT(40.25857207836413 74.32482757423415) bank190835 +190836 POINT(39.863024609228184 73.03323354285479) bank190836 +190837 POINT(40.22101061059248 73.86221811671585) bank190837 +190838 POINT(40.56593317891065 73.60619940801871) bank190838 +190839 POINT(40.38544701195665 73.8145068410297) bank190839 +190840 POINT(41.35645167935616 73.6433671733394) bank190840 +190841 POINT(39.94408955759534 73.28590133161853) bank190841 +190842 POINT(40.216266320367446 74.01329282904254) bank190842 +190843 POINT(40.08254514019533 74.86838595013906) bank190843 +190844 POINT(41.10816639685574 73.98575094338312) bank190844 +190845 POINT(41.59209933934215 74.80324449363899) bank190845 +190846 POINT(39.86188082226582 74.90663563835633) bank190846 +190847 POINT(39.810179076121486 74.2127677479942) bank190847 +190848 POINT(39.747842438691116 73.44749297481027) bank190848 +190849 POINT(40.70420241261161 73.79069342089734) bank190849 +190850 POINT(40.20118401850247 74.6037021908918) bank190850 +190851 POINT(41.66255732930794 74.59322884666454) bank190851 +190852 POINT(40.04794676550821 74.11735354020327) bank190852 +190853 POINT(40.65353819875247 73.36615549608781) bank190853 +190854 POINT(41.49927978593499 73.99909544667942) bank190854 +190855 POINT(41.289290033057604 73.62183525419832) bank190855 +190856 POINT(39.943066561853065 74.72716955475025) bank190856 +190857 POINT(39.809025193000345 74.4392435167042) bank190857 +190858 POINT(40.031764727124646 73.3207344454633) bank190858 +190859 POINT(40.47872057662729 73.38036837187946) bank190859 +190860 POINT(40.64585534130865 74.90346449254554) bank190860 +190861 POINT(41.140954890286345 73.01313991949442) bank190861 +190862 POINT(39.989847778307166 74.47436722329319) bank190862 +190863 POINT(39.928209359721656 74.44216098191811) bank190863 +190864 POINT(41.453657509801126 73.01274906736846) bank190864 +190865 POINT(41.16249727684855 73.39596734247225) bank190865 +190866 POINT(41.514230443918336 73.4415646867253) bank190866 +190867 POINT(40.64457819969874 73.02472411103047) bank190867 +190868 POINT(40.865694971766715 73.27998355096177) bank190868 +190869 POINT(40.54125180399703 74.46219029508292) bank190869 +190870 POINT(40.72596149979495 74.89381658414274) bank190870 +190871 POINT(40.49105674483562 73.74045767743515) bank190871 +190872 POINT(40.94707900877722 74.84391844199118) bank190872 +190873 POINT(40.982923846077746 74.43146364553076) bank190873 +190874 POINT(40.4914611180922 73.07171692030653) bank190874 +190875 POINT(41.16803815775393 73.48798474840181) bank190875 +190876 POINT(41.056290774819324 74.22292061698063) bank190876 +190877 POINT(41.709432373847996 74.84024673912351) bank190877 +190878 POINT(40.322417769396424 73.38736834202369) bank190878 +190879 POINT(39.974382547919184 73.51552677392245) bank190879 +190880 POINT(40.105564035433225 73.69770736414827) bank190880 +190881 POINT(40.055841568062704 74.24532312713588) bank190881 +190882 POINT(41.28131772166147 73.24534374154673) bank190882 +190883 POINT(41.24305902214342 73.62676049095158) bank190883 +190884 POINT(40.37472453082145 74.2789015170405) bank190884 +190885 POINT(39.84914302679055 73.60623497579009) bank190885 +190886 POINT(39.82753641634001 74.7174879088487) bank190886 +190887 POINT(41.14104453028773 73.86332142724928) bank190887 +190888 POINT(39.72049008634484 74.37235145081378) bank190888 +190889 POINT(40.453004887773425 74.62302410563497) bank190889 +190890 POINT(41.56429559380366 73.31368388860095) bank190890 +190891 POINT(41.26654616130918 74.66566582433103) bank190891 +190892 POINT(41.498730086376156 73.59090876212574) bank190892 +190893 POINT(39.96926126024606 74.90465697715845) bank190893 +190894 POINT(39.79896380635264 74.51908710982741) bank190894 +190895 POINT(40.65978108737477 74.1367625348938) bank190895 +190896 POINT(39.93083913037611 74.98045992482146) bank190896 +190897 POINT(40.63480665576012 74.61460208201123) bank190897 +190898 POINT(41.41615562514481 74.68712515007155) bank190898 +190899 POINT(41.3468436194306 73.11670491441112) bank190899 +190900 POINT(40.51666273368133 73.81569291826996) bank190900 +190901 POINT(40.40314720720614 73.75015894835452) bank190901 +190902 POINT(40.72667429434178 73.74023040656789) bank190902 +190903 POINT(40.22133425046909 74.48042386727285) bank190903 +190904 POINT(40.43993020464395 73.09376235369369) bank190904 +190905 POINT(41.23616149169158 73.90581522313602) bank190905 +190906 POINT(41.24068538461745 73.40646419926273) bank190906 +190907 POINT(40.38510348046441 73.82129910535335) bank190907 +190908 POINT(40.4795688374787 74.18573789739929) bank190908 +190909 POINT(40.86872987021976 74.86264532013632) bank190909 +190910 POINT(41.59030921026756 74.94917073960885) bank190910 +190911 POINT(41.03758616092488 73.33337167662937) bank190911 +190912 POINT(40.95317506239343 73.41199248975006) bank190912 +190913 POINT(41.58422321856923 73.0511714965389) bank190913 +190914 POINT(40.06458438833771 73.07196148110559) bank190914 +190915 POINT(41.49261255141804 74.0850626838835) bank190915 +190916 POINT(40.0015781428249 74.0807679496953) bank190916 +190917 POINT(41.22271145256387 74.4154269210283) bank190917 +190918 POINT(39.88206763080305 73.28577518997383) bank190918 +190919 POINT(39.91085150512973 73.52787932116911) bank190919 +190920 POINT(40.612510459016534 75.00142288506358) bank190920 +190921 POINT(40.53882623400765 73.03948151558039) bank190921 +190922 POINT(40.66160710378436 74.81930938301679) bank190922 +190923 POINT(40.74945216531761 73.66619297895072) bank190923 +190924 POINT(40.49390753172505 74.61261974969291) bank190924 +190925 POINT(40.693858247301876 74.75866602680465) bank190925 +190926 POINT(41.37613629978417 74.65150595487655) bank190926 +190927 POINT(40.12472481087009 74.21024856760963) bank190927 +190928 POINT(41.04656712201697 74.12015180602513) bank190928 +190929 POINT(40.884884007053664 74.61740019753915) bank190929 +190930 POINT(40.43829871841345 74.4852187842487) bank190930 +190931 POINT(39.78969846802423 73.65641594399911) bank190931 +190932 POINT(40.29230856722108 73.76437300098421) bank190932 +190933 POINT(41.632704710966635 74.00350946826535) bank190933 +190934 POINT(41.587724806768534 73.53022894932361) bank190934 +190935 POINT(40.89774498967207 73.25184239362292) bank190935 +190936 POINT(40.2417699397612 73.99321399228097) bank190936 +190937 POINT(40.15641810848978 74.50127767980761) bank190937 +190938 POINT(40.482700934346674 74.20221921730439) bank190938 +190939 POINT(40.467752201988255 73.98901958627803) bank190939 +190940 POINT(41.44321881108949 73.47211934634875) bank190940 +190941 POINT(39.72918818900333 74.44885302837483) bank190941 +190942 POINT(40.18069031485 73.32853640334233) bank190942 +190943 POINT(40.54674881774692 74.66324387199926) bank190943 +190944 POINT(41.1646577773202 74.66009488386929) bank190944 +190945 POINT(41.61450302145128 74.61852751799003) bank190945 +190946 POINT(40.26634084184903 74.46405465027605) bank190946 +190947 POINT(40.54609283101765 73.9204548039145) bank190947 +190948 POINT(40.65423972047649 73.55032843383809) bank190948 +190949 POINT(39.85346381085583 74.11422984659569) bank190949 +190950 POINT(39.71634363880722 73.21884954113898) bank190950 +190951 POINT(40.606562594988716 74.19756373405099) bank190951 +190952 POINT(40.12288669459637 73.43831585185954) bank190952 +190953 POINT(41.28815690540477 73.25079716913596) bank190953 +190954 POINT(40.7027787046837 74.95622914659474) bank190954 +190955 POINT(41.583150987873324 73.86643011104947) bank190955 +190956 POINT(41.49016537724147 73.28915044219002) bank190956 +190957 POINT(40.248128458428845 74.68553328330202) bank190957 +190958 POINT(39.97350308424158 74.62150812392304) bank190958 +190959 POINT(40.32003241698837 73.62462760141008) bank190959 +190960 POINT(40.61544246469324 73.67280095872185) bank190960 +190961 POINT(40.22767520222143 74.89904505153062) bank190961 +190962 POINT(39.95699550015009 73.90928461058738) bank190962 +190963 POINT(39.83525345544122 73.9978497689057) bank190963 +190964 POINT(39.749528170321014 73.94906153562259) bank190964 +190965 POINT(40.848882540582906 73.21749098303228) bank190965 +190966 POINT(40.12648145889536 74.78039138239599) bank190966 +190967 POINT(41.07496572739998 73.1321047499156) bank190967 +190968 POINT(39.9288766761101 74.19039388126058) bank190968 +190969 POINT(40.66656532517678 73.29167938727284) bank190969 +190970 POINT(40.70203662669682 74.18453359323739) bank190970 +190971 POINT(39.962207349503586 74.1251118356697) bank190971 +190972 POINT(41.297963626533026 74.47733934861769) bank190972 +190973 POINT(41.08812223867554 73.06368976485194) bank190973 +190974 POINT(41.678241365545226 73.88474597072994) bank190974 +190975 POINT(40.73626569171281 73.15036040433942) bank190975 +190976 POINT(41.64671377806324 73.38426993331993) bank190976 +190977 POINT(39.897916913086284 74.3525130511492) bank190977 +190978 POINT(39.85884547123229 74.94943898687775) bank190978 +190979 POINT(40.00950436715969 74.68792882706069) bank190979 +190980 POINT(40.95107535814578 74.00545222834698) bank190980 +190981 POINT(40.65761496159955 73.7330574831553) bank190981 +190982 POINT(40.28602704424111 73.27471775294748) bank190982 +190983 POINT(41.485214743411646 73.3763254859346) bank190983 +190984 POINT(41.491409168361045 73.1604819038219) bank190984 +190985 POINT(41.704308421141825 73.27669798322306) bank190985 +190986 POINT(40.79951026845 73.4278715228843) bank190986 +190987 POINT(41.28404555575702 74.16666340397403) bank190987 +190988 POINT(39.99170663726192 74.55255247664178) bank190988 +190989 POINT(41.08553720086903 73.77804744498822) bank190989 +190990 POINT(41.375674327428804 74.27102699178899) bank190990 +190991 POINT(41.13595316702364 73.76391216008035) bank190991 +190992 POINT(40.59788085876259 74.44188958875975) bank190992 +190993 POINT(41.343090055687775 74.37341989457829) bank190993 +190994 POINT(41.212193723115426 74.93597262900073) bank190994 +190995 POINT(39.728911684200376 74.68711279945096) bank190995 +190996 POINT(40.179148640941904 73.7518068384878) bank190996 +190997 POINT(40.04939012462161 74.08100518516407) bank190997 +190998 POINT(40.76040384679159 74.37334355186105) bank190998 +190999 POINT(39.82093821759223 74.86564081497015) bank190999 +191000 POINT(41.68792916680426 74.33019529959967) bank191000 +191001 POINT(39.80155260252936 74.4325315464687) bank191001 +191002 POINT(40.02055283084406 73.9533097367865) bank191002 +191003 POINT(41.129644581794366 74.6182665196366) bank191003 +191004 POINT(40.77777977623099 74.78598668004935) bank191004 +191005 POINT(40.861279888261784 74.36396715826797) bank191005 +191006 POINT(40.29911843824366 73.60448458600241) bank191006 +191007 POINT(40.30683682988801 73.58828209616759) bank191007 +191008 POINT(40.8535832507185 74.33953325133774) bank191008 +191009 POINT(41.45348582046423 73.98267223705811) bank191009 +191010 POINT(39.7674997597755 73.96593115430719) bank191010 +191011 POINT(41.60016308912902 74.3950380902546) bank191011 +191012 POINT(40.090554893166804 73.3590374876514) bank191012 +191013 POINT(39.75777714524389 73.72480493207317) bank191013 +191014 POINT(40.59834472708738 73.12116583494469) bank191014 +191015 POINT(40.45481359490501 74.10364317408913) bank191015 +191016 POINT(40.43027346600895 73.87961598564048) bank191016 +191017 POINT(41.551147031117594 74.39990215377985) bank191017 +191018 POINT(40.14251897384347 73.81096225331201) bank191018 +191019 POINT(39.75201427594702 74.49610698809995) bank191019 +191020 POINT(41.36600080089459 73.60248341812691) bank191020 +191021 POINT(40.845038886789226 74.5833435130769) bank191021 +191022 POINT(41.18373463655518 74.03284955459458) bank191022 +191023 POINT(39.777072828467404 73.58705566635253) bank191023 +191024 POINT(39.84983336301907 73.72093862746827) bank191024 +191025 POINT(41.234446187783874 74.13036002086871) bank191025 +191026 POINT(41.676415509621506 74.50120317329059) bank191026 +191027 POINT(41.362227716427604 73.4665678035012) bank191027 +191028 POINT(40.625904682234214 73.81477608022075) bank191028 +191029 POINT(41.44724132534561 74.13830016166088) bank191029 +191030 POINT(41.55265030457718 74.70237363302377) bank191030 +191031 POINT(40.09376540291973 73.47145606508144) bank191031 +191032 POINT(41.2509788962967 73.62113735321157) bank191032 +191033 POINT(40.79729872476664 73.03619375837958) bank191033 +191034 POINT(40.24094166052183 73.86880328330514) bank191034 +191035 POINT(41.002251891049276 74.4530851053534) bank191035 +191036 POINT(40.323032065419284 73.37714920532717) bank191036 +191037 POINT(39.742402113717844 73.71947026137546) bank191037 +191038 POINT(41.23115414276712 74.36469166939844) bank191038 +191039 POINT(41.34791971954234 73.82969896584474) bank191039 +191040 POINT(40.882806447870415 73.20562743340223) bank191040 +191041 POINT(39.971585256669606 74.02666966687678) bank191041 +191042 POINT(41.268524616793094 73.7548958927129) bank191042 +191043 POINT(41.30765046128095 74.27409006765352) bank191043 +191044 POINT(41.555746097113726 73.62253774724711) bank191044 +191045 POINT(41.706647935233384 73.95253174912843) bank191045 +191046 POINT(41.26200561387225 73.27485003553264) bank191046 +191047 POINT(39.867401526498966 74.43435556474086) bank191047 +191048 POINT(39.867561808289686 74.43953075331748) bank191048 +191049 POINT(41.539475460582935 74.66699431165137) bank191049 +191050 POINT(40.310870123163085 74.51889764951) bank191050 +191051 POINT(40.53441706962635 74.70289873709368) bank191051 +191052 POINT(41.308914762633854 74.90907291725794) bank191052 +191053 POINT(39.81970286695451 73.41633100651914) bank191053 +191054 POINT(40.17081357663541 73.6968629658791) bank191054 +191055 POINT(41.50048291536094 73.73743435398649) bank191055 +191056 POINT(39.76691432039868 74.72937487100633) bank191056 +191057 POINT(40.006754560226085 73.28848075270874) bank191057 +191058 POINT(39.90318152798934 73.56323614705826) bank191058 +191059 POINT(41.580801715427086 74.15264382814318) bank191059 +191060 POINT(39.74544295281591 73.74376350056859) bank191060 +191061 POINT(41.156801456239826 73.48166213298343) bank191061 +191062 POINT(39.75572752216003 73.92966707163362) bank191062 +191063 POINT(41.02551521656413 74.59395232597215) bank191063 +191064 POINT(39.866988158245604 73.3593303766317) bank191064 +191065 POINT(40.961838446353944 73.85834540969185) bank191065 +191066 POINT(39.985379272720344 74.97337702833055) bank191066 +191067 POINT(40.66714342607579 73.63403170188028) bank191067 +191068 POINT(40.039482498361735 74.4813932785636) bank191068 +191069 POINT(40.6403587702554 73.46867462530646) bank191069 +191070 POINT(41.54374708677729 73.94895503785625) bank191070 +191071 POINT(41.216755954541924 74.23907811325445) bank191071 +191072 POINT(41.22811959756241 74.11335465710357) bank191072 +191073 POINT(40.510938414355316 73.0431260077249) bank191073 +191074 POINT(40.888371682460196 73.65189451211258) bank191074 +191075 POINT(41.600441001107576 73.679670886245) bank191075 +191076 POINT(40.161852936215986 73.32425184544289) bank191076 +191077 POINT(41.090309354538654 74.75198367646779) bank191077 +191078 POINT(40.15498325201376 74.09931435736074) bank191078 +191079 POINT(40.40921877302682 73.35924453590587) bank191079 +191080 POINT(40.2927950557478 73.8155936373591) bank191080 +191081 POINT(39.812408858502145 74.80894168584236) bank191081 +191082 POINT(40.02430622569519 74.98341372737801) bank191082 +191083 POINT(40.500263733703804 73.12757503594645) bank191083 +191084 POINT(41.485535202383225 73.68974253830876) bank191084 +191085 POINT(40.2977763813011 74.07593264313383) bank191085 +191086 POINT(41.44440736163772 73.38625636961527) bank191086 +191087 POINT(40.741782652075116 74.44433857161576) bank191087 +191088 POINT(40.49820341794211 73.10057500325331) bank191088 +191089 POINT(40.97248762995925 73.8531388182632) bank191089 +191090 POINT(40.75970546912485 73.46313414186294) bank191090 +191091 POINT(40.46844139049684 74.52319043985875) bank191091 +191092 POINT(40.28344614776695 74.58000816585567) bank191092 +191093 POINT(40.729515790306756 74.70613979439199) bank191093 +191094 POINT(40.17336340857258 75.00069747998464) bank191094 +191095 POINT(41.6755625034804 73.4198210937041) bank191095 +191096 POINT(40.48449590880858 74.14470027244633) bank191096 +191097 POINT(39.77409808569793 73.80932751333293) bank191097 +191098 POINT(41.483098972226635 73.84337932885427) bank191098 +191099 POINT(41.70816588006015 73.27022176682665) bank191099 +191100 POINT(41.00670885236014 74.10011607238678) bank191100 +191101 POINT(40.53626260628603 74.2173286550381) bank191101 +191102 POINT(40.32710774100109 74.60131749137754) bank191102 +191103 POINT(39.917186829906 73.32786983231341) bank191103 +191104 POINT(40.05305283920737 73.10416396733191) bank191104 +191105 POINT(40.22366130177491 73.1246508683553) bank191105 +191106 POINT(41.44759752449506 73.24475710542212) bank191106 +191107 POINT(41.55425355326304 73.22162208810245) bank191107 +191108 POINT(40.34587527664885 74.26445659834143) bank191108 +191109 POINT(41.49523938084894 74.62341033437947) bank191109 +191110 POINT(41.17613881914044 73.71417874794369) bank191110 +191111 POINT(40.0877970752985 73.28614721138041) bank191111 +191112 POINT(40.32210319097311 74.9938091118823) bank191112 +191113 POINT(40.22726384642048 74.74945124728337) bank191113 +191114 POINT(41.50101445619504 73.87736042374506) bank191114 +191115 POINT(41.617559337726405 73.92125182347124) bank191115 +191116 POINT(41.50752161910354 73.61277055425364) bank191116 +191117 POINT(40.30751105937047 75.00017389177994) bank191117 +191118 POINT(39.88034600554467 74.69313457540193) bank191118 +191119 POINT(41.168393513916406 73.18104834820274) bank191119 +191120 POINT(40.95700463226568 74.01790089605264) bank191120 +191121 POINT(40.937181045712805 74.69218218220065) bank191121 +191122 POINT(40.881059942079126 73.6828856986564) bank191122 +191123 POINT(39.98165662107261 74.31357563770293) bank191123 +191124 POINT(41.59770888667746 73.72698506549439) bank191124 +191125 POINT(41.6879062235038 74.86727134223698) bank191125 +191126 POINT(39.90861739026073 73.7590516457183) bank191126 +191127 POINT(40.7558075046386 73.10329604297351) bank191127 +191128 POINT(39.97833803010731 73.50500539529901) bank191128 +191129 POINT(40.28586597521584 73.67861267942038) bank191129 +191130 POINT(40.69350392301051 73.40840056944965) bank191130 +191131 POINT(40.33534719618562 73.21035678148678) bank191131 +191132 POINT(40.26048681257005 73.77012022162701) bank191132 +191133 POINT(41.34233166689211 74.80940316999765) bank191133 +191134 POINT(41.11626393750437 73.57262429668451) bank191134 +191135 POINT(41.62125063834216 73.4697928047539) bank191135 +191136 POINT(39.97811681968904 73.14458587844229) bank191136 +191137 POINT(40.27136702963875 74.61646889920908) bank191137 +191138 POINT(41.69529766026361 74.48956498465643) bank191138 +191139 POINT(40.580685362208335 73.25189639869927) bank191139 +191140 POINT(40.01245465235589 73.63496534329228) bank191140 +191141 POINT(41.282173200792734 74.99106145181207) bank191141 +191142 POINT(40.1300079590842 73.45889727822986) bank191142 +191143 POINT(41.38740889147015 73.66829266219871) bank191143 +191144 POINT(40.079327400486044 74.41364266421677) bank191144 +191145 POINT(39.77180092360746 73.35319478815589) bank191145 +191146 POINT(40.19490150037686 73.79938215337286) bank191146 +191147 POINT(41.634737843117314 73.60580376219366) bank191147 +191148 POINT(40.30324854226878 73.64993506575122) bank191148 +191149 POINT(41.04781831840836 73.51272955522829) bank191149 +191150 POINT(40.29616640518193 73.6607419674474) bank191150 +191151 POINT(40.91210304770582 73.17095627037895) bank191151 +191152 POINT(39.918981290152644 74.58470979011946) bank191152 +191153 POINT(40.85253002292791 74.23129412205061) bank191153 +191154 POINT(39.941698486497636 73.03255206828348) bank191154 +191155 POINT(41.498632679401155 74.4090512353409) bank191155 +191156 POINT(40.35230577334598 74.29156762308205) bank191156 +191157 POINT(40.00122554727192 74.65012470984412) bank191157 +191158 POINT(40.23059849652563 74.71477877733192) bank191158 +191159 POINT(41.674907748459134 73.14967547357287) bank191159 +191160 POINT(40.803063250321784 73.15214499586565) bank191160 +191161 POINT(41.480642710229134 73.92472157655112) bank191161 +191162 POINT(41.66609640174237 73.35576340481629) bank191162 +191163 POINT(40.02136437708815 73.26772984408305) bank191163 +191164 POINT(41.36059963954659 73.97653386464958) bank191164 +191165 POINT(41.064652709447614 73.77485521618472) bank191165 +191166 POINT(40.48756271323579 73.13575987194264) bank191166 +191167 POINT(39.942147543153496 74.2180007204459) bank191167 +191168 POINT(39.9532525529141 74.4001135194053) bank191168 +191169 POINT(41.06170457999377 74.5536579863142) bank191169 +191170 POINT(41.16036475294009 73.39330592483671) bank191170 +191171 POINT(41.56871054915899 74.11163192238185) bank191171 +191172 POINT(40.14274199849047 74.64646723613407) bank191172 +191173 POINT(40.76679345435192 74.25128617289425) bank191173 +191174 POINT(40.28881408925798 74.3109509485931) bank191174 +191175 POINT(40.542801589750916 75.00430172111854) bank191175 +191176 POINT(40.23380189087421 73.737354642421) bank191176 +191177 POINT(39.852582006311856 73.28164770377646) bank191177 +191178 POINT(39.99961417244503 74.81585628969079) bank191178 +191179 POINT(40.78393609529664 73.5008314042352) bank191179 +191180 POINT(41.24552872515726 73.15065444713248) bank191180 +191181 POINT(41.23790810028107 74.63114871792554) bank191181 +191182 POINT(40.82524290076582 73.1468489670486) bank191182 +191183 POINT(41.43608624121479 74.27092599124478) bank191183 +191184 POINT(41.15160574723512 73.80675377541796) bank191184 +191185 POINT(39.88333257852651 73.35751037953868) bank191185 +191186 POINT(40.820911787659284 73.34978647467732) bank191186 +191187 POINT(40.36588791806163 73.895930573451) bank191187 +191188 POINT(41.709736535848734 74.17520519931823) bank191188 +191189 POINT(39.92339977983138 74.80815825197163) bank191189 +191190 POINT(40.85923848239092 74.67805811072326) bank191190 +191191 POINT(40.11063570005248 74.19113675288929) bank191191 +191192 POINT(39.750914951694455 73.55292624510838) bank191192 +191193 POINT(40.71675629024168 73.75105773594456) bank191193 +191194 POINT(40.794551982639106 73.39174488746472) bank191194 +191195 POINT(40.7659637558196 74.14060181892665) bank191195 +191196 POINT(41.410408637689386 74.20994406623201) bank191196 +191197 POINT(41.08830319967549 73.8799969469079) bank191197 +191198 POINT(40.67548140409569 74.49522125895706) bank191198 +191199 POINT(40.83115995731236 74.56336442864563) bank191199 +191200 POINT(40.525018324218635 74.32178691067622) bank191200 +191201 POINT(39.86266042106257 73.42942089806239) bank191201 +191202 POINT(41.367020037160195 73.29099082948431) bank191202 +191203 POINT(41.54723672800151 73.19221327003628) bank191203 +191204 POINT(41.47733987326283 74.92416300732114) bank191204 +191205 POINT(40.759881168185146 74.72771103341591) bank191205 +191206 POINT(40.07886768902303 73.74351786046975) bank191206 +191207 POINT(41.65996920946494 73.87019880408737) bank191207 +191208 POINT(40.76540956710537 73.29480740816382) bank191208 +191209 POINT(40.225209433960295 74.51990675270596) bank191209 +191210 POINT(41.49407684861002 73.94850641219014) bank191210 +191211 POINT(39.857257929711665 74.01013586264568) bank191211 +191212 POINT(39.957871210527706 74.94953939527029) bank191212 +191213 POINT(41.30308229557296 74.90338331174085) bank191213 +191214 POINT(41.44916355726322 74.92081493118182) bank191214 +191215 POINT(41.501372708486215 74.21322087757785) bank191215 +191216 POINT(41.391533527696566 73.42969580929375) bank191216 +191217 POINT(41.387390215380016 74.57664499830076) bank191217 +191218 POINT(40.83860163159406 73.65168063299573) bank191218 +191219 POINT(40.40666595897949 74.76113488153679) bank191219 +191220 POINT(40.67796219897107 73.2214723614094) bank191220 +191221 POINT(41.61371229085725 74.39354845498183) bank191221 +191222 POINT(39.95380513305452 74.66406129660827) bank191222 +191223 POINT(41.629092824043475 74.85188118210546) bank191223 +191224 POINT(39.85575190512275 73.91377066379596) bank191224 +191225 POINT(40.64127760116023 74.98256408515071) bank191225 +191226 POINT(40.26370836714566 73.95198753762153) bank191226 +191227 POINT(39.96847621248103 73.78896382365978) bank191227 +191228 POINT(41.213675357377014 73.59552971009299) bank191228 +191229 POINT(40.488945934088704 73.3225476109951) bank191229 +191230 POINT(40.70680633684412 74.17691289074983) bank191230 +191231 POINT(41.2446974224784 74.84681703496634) bank191231 +191232 POINT(41.690247618361816 73.36260553264583) bank191232 +191233 POINT(40.22761307106731 73.60950882756971) bank191233 +191234 POINT(41.05467686598033 74.10237053175418) bank191234 +191235 POINT(41.16403446506136 74.15239314215852) bank191235 +191236 POINT(40.951666364053565 74.34001674690946) bank191236 +191237 POINT(40.299280710753166 73.44068648827455) bank191237 +191238 POINT(39.773960195855196 74.51849353748155) bank191238 +191239 POINT(41.25199502471959 73.24762957922135) bank191239 +191240 POINT(40.34305193163551 73.78052230563453) bank191240 +191241 POINT(40.26193998705325 74.70657860593182) bank191241 +191242 POINT(40.687421448099 73.13327440409405) bank191242 +191243 POINT(40.307811501550674 74.52516601752448) bank191243 +191244 POINT(41.12095533338696 74.16452415159536) bank191244 +191245 POINT(40.07755389033311 74.78744961658623) bank191245 +191246 POINT(41.216032131005946 74.82855379304677) bank191246 +191247 POINT(40.896992642778955 74.18596213730252) bank191247 +191248 POINT(40.60174528949648 73.19971686373275) bank191248 +191249 POINT(40.68086585140401 74.0908164345241) bank191249 +191250 POINT(41.43090788185557 73.37376190522994) bank191250 +191251 POINT(40.13689886073152 73.52023790133626) bank191251 +191252 POINT(39.924689979643446 74.71561718978712) bank191252 +191253 POINT(40.92213413725184 73.3979652093289) bank191253 +191254 POINT(39.86270343994862 74.79523366480086) bank191254 +191255 POINT(41.41856815419784 73.95728012558138) bank191255 +191256 POINT(40.79373857883522 73.22171871478614) bank191256 +191257 POINT(39.94219323961223 73.94339646199197) bank191257 +191258 POINT(40.4501181014072 74.18753917835454) bank191258 +191259 POINT(40.268200814445116 73.99851668597645) bank191259 +191260 POINT(41.48347868239607 74.81779055761021) bank191260 +191261 POINT(39.91229396859087 73.01607907145036) bank191261 +191262 POINT(40.467004565718334 74.30912460856226) bank191262 +191263 POINT(40.73885942400546 74.26598106850635) bank191263 +191264 POINT(40.609010188386044 74.70354051989585) bank191264 +191265 POINT(40.72043097320055 74.09742713774405) bank191265 +191266 POINT(39.77022980529112 74.77018346263742) bank191266 +191267 POINT(39.76646303288848 73.20104384760889) bank191267 +191268 POINT(39.91443258508292 74.16204573793736) bank191268 +191269 POINT(41.055025848171795 74.74129122117884) bank191269 +191270 POINT(41.66298887791025 74.82412267028768) bank191270 +191271 POINT(40.23992493594197 74.8884297399533) bank191271 +191272 POINT(40.766966833662515 74.0787107939619) bank191272 +191273 POINT(40.263570386325874 74.57539003138245) bank191273 +191274 POINT(40.776168501791574 74.98238185036399) bank191274 +191275 POINT(41.121106980178276 73.57758873841397) bank191275 +191276 POINT(39.9836829455223 74.29759979572123) bank191276 +191277 POINT(39.71967486919831 74.53402126412496) bank191277 +191278 POINT(41.285242233929594 74.67236004970142) bank191278 +191279 POINT(41.07190531507809 73.64477774289904) bank191279 +191280 POINT(41.260447750349044 74.85107807388441) bank191280 +191281 POINT(40.379829116593086 74.59244471165589) bank191281 +191282 POINT(39.79206033999336 73.7465668971986) bank191282 +191283 POINT(40.92768811526535 74.13469905203216) bank191283 +191284 POINT(41.46880296094882 74.3023410398413) bank191284 +191285 POINT(40.88133671343601 74.10959550446533) bank191285 +191286 POINT(41.24315368265077 73.4703745103767) bank191286 +191287 POINT(41.04234539370128 73.63012488353726) bank191287 +191288 POINT(40.732182501768634 73.24298679812176) bank191288 +191289 POINT(41.00521756520265 74.67853859515769) bank191289 +191290 POINT(40.60588631415675 73.0136952000372) bank191290 +191291 POINT(40.25004405697635 73.85609398225128) bank191291 +191292 POINT(39.79529447405214 73.54666196934886) bank191292 +191293 POINT(40.99839149910082 73.80252052240836) bank191293 +191294 POINT(40.90814039669304 73.85261959634428) bank191294 +191295 POINT(41.04908729123397 74.75487883434957) bank191295 +191296 POINT(39.77450899619632 73.0371102414243) bank191296 +191297 POINT(40.75922064034783 74.38165117135479) bank191297 +191298 POINT(39.736410985620644 74.25759511433267) bank191298 +191299 POINT(41.20489627483302 73.96240407912286) bank191299 +191300 POINT(41.614871232642294 73.48430429183816) bank191300 +191301 POINT(41.32450390559198 74.18384158944822) bank191301 +191302 POINT(41.159367282532095 74.26859604614624) bank191302 +191303 POINT(40.467568106727434 74.52468346086351) bank191303 +191304 POINT(41.36475532873413 74.07911212438218) bank191304 +191305 POINT(39.72661739414972 73.67474419318002) bank191305 +191306 POINT(40.74598984223198 73.16325641845101) bank191306 +191307 POINT(41.65517630760911 73.27514164486722) bank191307 +191308 POINT(40.82670964299941 73.13938871670193) bank191308 +191309 POINT(40.423223023130774 74.65437915310133) bank191309 +191310 POINT(40.77108969031401 73.85389557674903) bank191310 +191311 POINT(41.18087788706924 74.91601216541773) bank191311 +191312 POINT(40.09334361588555 73.80958957499267) bank191312 +191313 POINT(41.2771114261212 74.05289723449641) bank191313 +191314 POINT(41.2227758614427 74.30932485603174) bank191314 +191315 POINT(39.918223967753384 74.69963960964618) bank191315 +191316 POINT(39.899542277860064 74.50967790301895) bank191316 +191317 POINT(41.41100671347005 73.52899510654949) bank191317 +191318 POINT(39.885773388453664 73.76596087751591) bank191318 +191319 POINT(41.36862519693075 74.70997013428968) bank191319 +191320 POINT(41.061053695139584 74.9086776559189) bank191320 +191321 POINT(39.90059545840209 73.10884038464296) bank191321 +191322 POINT(40.27975961118293 74.76620205342867) bank191322 +191323 POINT(40.04811448688755 73.55240344703655) bank191323 +191324 POINT(39.875924369153715 74.85897405350808) bank191324 +191325 POINT(41.25128056478638 73.25677311168035) bank191325 +191326 POINT(40.29780535335017 74.59043651408092) bank191326 +191327 POINT(41.421136997955884 73.52537409007122) bank191327 +191328 POINT(40.76292045117 74.03015502054592) bank191328 +191329 POINT(40.99804472306149 74.06648316565852) bank191329 +191330 POINT(40.47394427090659 73.73200304755055) bank191330 +191331 POINT(39.896492450682636 73.6727684077169) bank191331 +191332 POINT(41.277073973776304 74.76092601268743) bank191332 +191333 POINT(40.619184393048755 73.81401495437449) bank191333 +191334 POINT(41.05032615453495 73.13552174534651) bank191334 +191335 POINT(41.5193866199044 74.99593524591532) bank191335 +191336 POINT(41.024937367654736 73.04720431795973) bank191336 +191337 POINT(40.19732509055885 74.90052289900656) bank191337 +191338 POINT(41.588151460941816 73.52223606931702) bank191338 +191339 POINT(40.19606949894142 74.85586680304435) bank191339 +191340 POINT(41.13553714328614 74.39170514617187) bank191340 +191341 POINT(40.98120402252718 74.41845228492151) bank191341 +191342 POINT(40.010079891527106 73.06775931925564) bank191342 +191343 POINT(41.2343244847477 74.26609166839062) bank191343 +191344 POINT(41.51764322626504 73.82260391308758) bank191344 +191345 POINT(40.71871235682915 74.44405834593202) bank191345 +191346 POINT(39.86750260781758 73.19243524432063) bank191346 +191347 POINT(39.772364587229596 74.7695653619564) bank191347 +191348 POINT(40.62294393104366 73.30189688114365) bank191348 +191349 POINT(41.05784357994237 74.17419701988005) bank191349 +191350 POINT(40.7572090036721 74.77372580246235) bank191350 +191351 POINT(40.38852441205557 74.12038781316534) bank191351 +191352 POINT(40.80990383825019 73.21320681287405) bank191352 +191353 POINT(40.71570290917286 74.6368605516803) bank191353 +191354 POINT(40.00974904865982 74.41561651522791) bank191354 +191355 POINT(40.74034908971464 73.05899740776931) bank191355 +191356 POINT(41.402774085609344 73.01109011890303) bank191356 +191357 POINT(40.37102017784067 73.03488735422778) bank191357 +191358 POINT(41.09127093459678 73.86239871691758) bank191358 +191359 POINT(41.48990896619596 74.63138859924105) bank191359 +191360 POINT(41.66885063075184 74.80629083301787) bank191360 +191361 POINT(40.21911740318733 73.91266276462618) bank191361 +191362 POINT(40.48128905907387 74.93224005215689) bank191362 +191363 POINT(41.13219275416808 73.72390428442273) bank191363 +191364 POINT(40.87098217684326 73.75911829388859) bank191364 +191365 POINT(40.96148785744271 74.65727620876743) bank191365 +191366 POINT(40.558083531411704 73.85166355218247) bank191366 +191367 POINT(39.89643366870679 73.67257585369688) bank191367 +191368 POINT(40.455821503696264 73.42479965296414) bank191368 +191369 POINT(40.751088616786426 74.77304922914745) bank191369 +191370 POINT(40.299304534883035 74.40833140107235) bank191370 +191371 POINT(40.949216370775154 74.15909743794843) bank191371 +191372 POINT(41.515233360752376 74.92898520253098) bank191372 +191373 POINT(40.11222361266351 74.83072067893198) bank191373 +191374 POINT(40.40951987764651 73.7273529665337) bank191374 +191375 POINT(39.876977072827756 74.49707983296689) bank191375 +191376 POINT(40.57345199875328 74.9552485103867) bank191376 +191377 POINT(40.7196341213106 73.57907344930811) bank191377 +191378 POINT(39.73545829118207 73.88128537104427) bank191378 +191379 POINT(40.04435368960806 74.6780512173581) bank191379 +191380 POINT(40.77901945616119 73.97544359011854) bank191380 +191381 POINT(41.02570425174954 74.67306793623469) bank191381 +191382 POINT(39.79558244691532 74.18634795014913) bank191382 +191383 POINT(40.59977186892082 73.27956126309768) bank191383 +191384 POINT(40.98824428647948 73.73634993876277) bank191384 +191385 POINT(41.34519341389303 73.92621438359753) bank191385 +191386 POINT(40.2755149306138 74.18999129828325) bank191386 +191387 POINT(40.930755910559505 74.40606384503526) bank191387 +191388 POINT(40.27308566110863 74.89061748832539) bank191388 +191389 POINT(40.67397007561652 73.52924938365189) bank191389 +191390 POINT(40.16116099964431 73.18996915482629) bank191390 +191391 POINT(40.65276673931604 73.5467456210098) bank191391 +191392 POINT(41.36093735972081 74.29901623750298) bank191392 +191393 POINT(41.607976040252325 74.39395659038753) bank191393 +191394 POINT(40.90607312550361 74.5634310397316) bank191394 +191395 POINT(40.494294801766685 73.00699334697644) bank191395 +191396 POINT(40.14145004002476 73.6206444194651) bank191396 +191397 POINT(40.41900359935519 73.48350955272471) bank191397 +191398 POINT(41.008388179658354 74.3025233317822) bank191398 +191399 POINT(41.258991235203034 73.92072193652822) bank191399 +191400 POINT(40.76093219173058 73.28570714896168) bank191400 +191401 POINT(40.47889145669781 73.61097105483974) bank191401 +191402 POINT(39.764307323028255 74.89516316015629) bank191402 +191403 POINT(41.5964033948606 73.81157379558057) bank191403 +191404 POINT(40.366665845206306 73.47565141017549) bank191404 +191405 POINT(40.64138649461392 73.97349206901484) bank191405 +191406 POINT(40.7296778800325 74.34698656778352) bank191406 +191407 POINT(41.33559655592343 73.99424992324553) bank191407 +191408 POINT(41.006875497441534 74.97096712651756) bank191408 +191409 POINT(40.15078527107149 74.53782931562795) bank191409 +191410 POINT(40.688072923911946 73.34661268630506) bank191410 +191411 POINT(40.62901919243823 74.55461191620111) bank191411 +191412 POINT(39.84646156554263 73.70835270992123) bank191412 +191413 POINT(40.08513231615651 74.92385132606992) bank191413 +191414 POINT(41.606666402458764 74.6872237384563) bank191414 +191415 POINT(41.01537675166786 73.68613717079242) bank191415 +191416 POINT(41.500706628614005 74.19574292556197) bank191416 +191417 POINT(40.83741409326657 74.07361655098715) bank191417 +191418 POINT(40.577919079668135 73.66893973770486) bank191418 +191419 POINT(40.5315290073831 73.50673557179334) bank191419 +191420 POINT(41.07651622804709 74.889353480315) bank191420 +191421 POINT(40.161316177257234 73.10802933287049) bank191421 +191422 POINT(40.81053000972334 73.58285364302186) bank191422 +191423 POINT(41.60454345927741 74.65335905178576) bank191423 +191424 POINT(40.6805096702622 73.77334851680644) bank191424 +191425 POINT(41.29992862601175 73.97050463557824) bank191425 +191426 POINT(40.04730912542968 73.62928841495828) bank191426 +191427 POINT(40.78457213088024 74.00953120425964) bank191427 +191428 POINT(39.91707967171627 73.20159524135495) bank191428 +191429 POINT(40.47873228584512 73.61794528130567) bank191429 +191430 POINT(40.42388133334543 74.92289192165283) bank191430 +191431 POINT(39.96334760097514 73.14686585294193) bank191431 +191432 POINT(41.15782702928634 74.4276522503256) bank191432 +191433 POINT(40.561994421020856 74.59047817440411) bank191433 +191434 POINT(39.7393922114284 73.93799160137354) bank191434 +191435 POINT(41.64376931822742 73.16383173717554) bank191435 +191436 POINT(41.24450816043271 73.52299962958621) bank191436 +191437 POINT(41.01080115049217 74.8919790219931) bank191437 +191438 POINT(40.76849360510113 73.87592850201932) bank191438 +191439 POINT(41.237846076003805 73.70106019876486) bank191439 +191440 POINT(40.84380638790574 74.99271859895393) bank191440 +191441 POINT(40.03452654081618 75.00524859933216) bank191441 +191442 POINT(41.369323277357495 73.14467701816072) bank191442 +191443 POINT(40.49589700368799 74.20681791141337) bank191443 +191444 POINT(41.49115765389379 73.91360644062442) bank191444 +191445 POINT(40.257877036951314 74.49852480546744) bank191445 +191446 POINT(40.64756411159236 74.84118598896444) bank191446 +191447 POINT(40.449216156000276 74.75954454486087) bank191447 +191448 POINT(40.55006950772166 74.02528480512086) bank191448 +191449 POINT(40.41665539187807 74.64904436222082) bank191449 +191450 POINT(40.30779067386112 74.17343157482863) bank191450 +191451 POINT(40.53900063828384 74.3008687617258) bank191451 +191452 POINT(40.80073848224884 74.46382937345375) bank191452 +191453 POINT(41.0309435021616 74.06592356089398) bank191453 +191454 POINT(40.81105573350262 73.4297149101055) bank191454 +191455 POINT(39.80232367525877 74.51959380804034) bank191455 +191456 POINT(40.936788993316895 73.56931455968795) bank191456 +191457 POINT(41.557679938882664 73.39844997490921) bank191457 +191458 POINT(39.91273625578573 74.66990981926763) bank191458 +191459 POINT(40.37307695185103 73.2557245409131) bank191459 +191460 POINT(41.60213281079334 74.30409606873266) bank191460 +191461 POINT(40.44256255821158 74.37041356381727) bank191461 +191462 POINT(39.887376785452915 73.2210169768646) bank191462 +191463 POINT(39.89373232376997 73.73525593871096) bank191463 +191464 POINT(40.770808628657534 74.51164289537917) bank191464 +191465 POINT(39.9006962881103 74.9904820939515) bank191465 +191466 POINT(40.89688626034892 73.24591858549083) bank191466 +191467 POINT(41.62081404524145 73.52587951602084) bank191467 +191468 POINT(41.62388984969242 73.83827006317247) bank191468 +191469 POINT(39.73072771369909 73.96471411179627) bank191469 +191470 POINT(41.41099285259122 74.8888108506211) bank191470 +191471 POINT(40.90441993943042 73.38973329915626) bank191471 +191472 POINT(39.86261150854323 73.92811303854573) bank191472 +191473 POINT(40.55107607153322 74.9996441096814) bank191473 +191474 POINT(41.180922963196934 74.28300744855882) bank191474 +191475 POINT(41.655309679223905 73.92967818735357) bank191475 +191476 POINT(40.96351829284369 74.74339158583082) bank191476 +191477 POINT(40.900979802397956 74.35049795740044) bank191477 +191478 POINT(41.08659361914258 73.59852625926952) bank191478 +191479 POINT(40.47220091759697 73.6287752555186) bank191479 +191480 POINT(40.43082458862002 74.425871260257) bank191480 +191481 POINT(40.13252092623442 74.56434284569309) bank191481 +191482 POINT(40.14978773701694 73.2483870988889) bank191482 +191483 POINT(40.89250999755514 73.58125119091056) bank191483 +191484 POINT(41.49496452061853 73.5234276458722) bank191484 +191485 POINT(40.88633973866267 74.8527899953981) bank191485 +191486 POINT(40.84135695547008 74.50347979445037) bank191486 +191487 POINT(40.550761285605084 74.00313963800045) bank191487 +191488 POINT(41.483603265232475 74.14695192157937) bank191488 +191489 POINT(40.44734537276797 73.70120115080476) bank191489 +191490 POINT(41.177587318124154 73.55475115928004) bank191490 +191491 POINT(41.068343279222766 73.97249802487275) bank191491 +191492 POINT(40.4402879212618 74.75394214703921) bank191492 +191493 POINT(41.126212239179814 73.21486786109462) bank191493 +191494 POINT(40.93242142754952 73.51534508471835) bank191494 +191495 POINT(40.472446690391095 74.76745521481925) bank191495 +191496 POINT(40.2178832692497 73.40349952488711) bank191496 +191497 POINT(41.54704915712838 74.92734651664276) bank191497 +191498 POINT(40.422153103618946 73.42972054017476) bank191498 +191499 POINT(41.66528888395512 73.1190062874337) bank191499 +191500 POINT(41.68333442231315 73.35102062965022) bank191500 +191501 POINT(39.78976943353765 73.4069769613447) bank191501 +191502 POINT(40.16441725802086 74.04807601172956) bank191502 +191503 POINT(39.95124135990276 73.12517921906147) bank191503 +191504 POINT(40.39755927872571 74.74887630477842) bank191504 +191505 POINT(40.82512830097063 73.35473674845889) bank191505 +191506 POINT(41.069255215826324 74.1395136527426) bank191506 +191507 POINT(41.20470408478236 73.07043403197515) bank191507 +191508 POINT(39.785385671894716 73.54000663278748) bank191508 +191509 POINT(40.005396102581855 73.38802229216655) bank191509 +191510 POINT(39.84692593717048 74.66576948302041) bank191510 +191511 POINT(40.386069512470286 74.5038227349939) bank191511 +191512 POINT(41.25684592432671 73.0669604283914) bank191512 +191513 POINT(41.34004464761591 73.73537251747548) bank191513 +191514 POINT(39.974246631332335 73.13253153078314) bank191514 +191515 POINT(41.17300911234817 74.63734544997766) bank191515 +191516 POINT(40.08462535828074 73.11477113381338) bank191516 +191517 POINT(41.17378803082701 74.49116205442485) bank191517 +191518 POINT(41.55446035041104 73.66250395604736) bank191518 +191519 POINT(39.72154569096396 73.43483356897211) bank191519 +191520 POINT(41.68917438449434 74.47166593558924) bank191520 +191521 POINT(40.76578608077631 73.15306641241344) bank191521 +191522 POINT(41.15084983629677 74.72800559210984) bank191522 +191523 POINT(41.225146304002344 73.14763755161485) bank191523 +191524 POINT(41.68776648124423 74.16048963599084) bank191524 +191525 POINT(40.46794434795291 73.97785849300494) bank191525 +191526 POINT(39.94968865724146 73.79355856812984) bank191526 +191527 POINT(40.23208971972542 73.4125001081048) bank191527 +191528 POINT(40.3015859753934 73.19476874057531) bank191528 +191529 POINT(41.424680602905006 73.39641102390415) bank191529 +191530 POINT(40.944667195987265 74.42029080558585) bank191530 +191531 POINT(41.1058642514644 74.22507828865513) bank191531 +191532 POINT(40.06458543402571 73.86037339190587) bank191532 +191533 POINT(41.50490610555059 73.3274412959017) bank191533 +191534 POINT(41.567617558349795 73.10229788725151) bank191534 +191535 POINT(41.51770383802925 74.52994772358313) bank191535 +191536 POINT(39.85691679003844 74.01176508530799) bank191536 +191537 POINT(41.0699362328993 74.56785367280716) bank191537 +191538 POINT(40.36141088617008 74.05972330551852) bank191538 +191539 POINT(39.90621933121398 74.78460856297424) bank191539 +191540 POINT(39.89409113030296 73.0481164267859) bank191540 +191541 POINT(41.28559239306038 74.89970242612603) bank191541 +191542 POINT(40.919868520623474 74.07227477964405) bank191542 +191543 POINT(40.532259259680124 73.26130755604524) bank191543 +191544 POINT(40.72560003609316 74.03019419748534) bank191544 +191545 POINT(41.11932262994827 73.02844548778289) bank191545 +191546 POINT(41.081448273524835 73.08635832885152) bank191546 +191547 POINT(40.23443130603095 73.41843802793052) bank191547 +191548 POINT(40.598437291014655 73.41032813149828) bank191548 +191549 POINT(41.36143596065826 74.63924226378838) bank191549 +191550 POINT(40.5980254404827 74.51896090301972) bank191550 +191551 POINT(40.96401111242236 73.53662022141032) bank191551 +191552 POINT(39.82426433859311 74.7870492654306) bank191552 +191553 POINT(40.96484173029499 73.61364186377229) bank191553 +191554 POINT(41.36025241856132 74.6245377865678) bank191554 +191555 POINT(41.0374731183893 73.21123327302882) bank191555 +191556 POINT(40.805357229393785 74.92499618633782) bank191556 +191557 POINT(39.810561063233635 73.78960367216948) bank191557 +191558 POINT(39.78240034388809 74.55688363911399) bank191558 +191559 POINT(41.549032802737585 73.7552496819794) bank191559 +191560 POINT(41.11197505905299 74.58119233331634) bank191560 +191561 POINT(40.03540286358388 74.50784590981603) bank191561 +191562 POINT(41.26894701485022 73.80436458607905) bank191562 +191563 POINT(40.06601400255944 74.03328459471386) bank191563 +191564 POINT(39.749076929443966 73.04297784472033) bank191564 +191565 POINT(40.625934271311976 74.62628769886244) bank191565 +191566 POINT(40.45531342159225 73.8253187103819) bank191566 +191567 POINT(39.8303719855636 74.83210743775777) bank191567 +191568 POINT(39.99066250345972 74.50281155134867) bank191568 +191569 POINT(40.213913587602256 74.15334490671192) bank191569 +191570 POINT(41.117361738752514 73.29308368710333) bank191570 +191571 POINT(40.54865275254991 73.98586456055992) bank191571 +191572 POINT(41.630189999942566 74.67414447571169) bank191572 +191573 POINT(40.65586164065398 73.11029785635436) bank191573 +191574 POINT(39.72922051132635 74.39351244352368) bank191574 +191575 POINT(40.6714380121371 74.10155534090572) bank191575 +191576 POINT(40.692624270849805 73.7077224380635) bank191576 +191577 POINT(39.83665402241986 74.51402417699907) bank191577 +191578 POINT(40.5748879224897 74.27151871948176) bank191578 +191579 POINT(41.519404729910654 74.00854465586862) bank191579 +191580 POINT(40.207558070645796 73.88299705421042) bank191580 +191581 POINT(41.31324302290173 74.15283238811323) bank191581 +191582 POINT(40.40861033393266 73.549375764042) bank191582 +191583 POINT(41.058861960687416 74.180882066334) bank191583 +191584 POINT(40.08621917889582 74.97689035640462) bank191584 +191585 POINT(40.86806219749192 74.66362110000416) bank191585 +191586 POINT(41.30087866136198 74.3415334647311) bank191586 +191587 POINT(40.95147378381456 73.47542367136006) bank191587 +191588 POINT(40.32670093474364 73.82271957654808) bank191588 +191589 POINT(39.80800833586294 74.32636253497228) bank191589 +191590 POINT(41.48993983916639 74.76562850023117) bank191590 +191591 POINT(39.970657328504494 73.02092520020437) bank191591 +191592 POINT(41.488005352531275 74.84501710840304) bank191592 +191593 POINT(40.619124767513064 74.35774023388304) bank191593 +191594 POINT(39.903475405196055 74.92377356130197) bank191594 +191595 POINT(40.3493250017311 73.982564671816) bank191595 +191596 POINT(40.408689456964744 73.79122291624108) bank191596 +191597 POINT(41.68390794263739 74.53989298768514) bank191597 +191598 POINT(40.40485179387369 74.12608542122418) bank191598 +191599 POINT(40.31843481163135 74.90224175236213) bank191599 +191600 POINT(41.305010811290785 73.8810291402653) bank191600 +191601 POINT(41.317613793883226 73.0126338453941) bank191601 +191602 POINT(41.10934455439463 73.21904364585558) bank191602 +191603 POINT(41.34753705438711 74.95482645566742) bank191603 +191604 POINT(40.5762682739528 73.84963702007825) bank191604 +191605 POINT(41.56265635957971 73.38885620377151) bank191605 +191606 POINT(41.43946458720663 74.89734879899662) bank191606 +191607 POINT(39.77145355711877 74.62103795484029) bank191607 +191608 POINT(40.102255190631496 73.47993976420548) bank191608 +191609 POINT(41.34398276702372 74.93794866724713) bank191609 +191610 POINT(40.59134876700305 74.66725471905794) bank191610 +191611 POINT(40.02292268129571 74.4499668434679) bank191611 +191612 POINT(40.24624589213798 73.79850616619198) bank191612 +191613 POINT(40.620503050327365 74.84620748247156) bank191613 +191614 POINT(40.891996446627566 74.04383776150952) bank191614 +191615 POINT(40.1650623654737 73.8586645862508) bank191615 +191616 POINT(40.82693730203721 73.97032737922125) bank191616 +191617 POINT(40.38344104181488 73.9569607478794) bank191617 +191618 POINT(40.16750675795009 73.9461648130859) bank191618 +191619 POINT(40.468856381292696 73.01052665854583) bank191619 +191620 POINT(40.748499256552726 74.31365272154876) bank191620 +191621 POINT(41.69494406473016 74.59233484055724) bank191621 +191622 POINT(40.062079863402644 74.657805246497) bank191622 +191623 POINT(40.98518122980305 73.17739991266981) bank191623 +191624 POINT(40.86853461817977 74.17832533039177) bank191624 +191625 POINT(41.19911579759623 73.17023156273974) bank191625 +191626 POINT(39.99271667515426 74.98518766943032) bank191626 +191627 POINT(39.72465039810364 73.20161886825528) bank191627 +191628 POINT(41.458251394445725 73.11382572016046) bank191628 +191629 POINT(39.799332316431794 74.04525730660413) bank191629 +191630 POINT(41.70970270272667 74.3076861829983) bank191630 +191631 POINT(40.13417991626552 74.37935613665728) bank191631 +191632 POINT(41.36956119908947 73.14851622936403) bank191632 +191633 POINT(41.31811111161409 74.45975109314664) bank191633 +191634 POINT(39.7176767125118 73.38294418829868) bank191634 +191635 POINT(40.99162634343836 73.62576312197247) bank191635 +191636 POINT(40.70864985854961 74.07965219479057) bank191636 +191637 POINT(40.3863458178865 73.31983736154667) bank191637 +191638 POINT(40.35658529691734 73.66432313529351) bank191638 +191639 POINT(41.16034978000272 73.70289998238796) bank191639 +191640 POINT(40.06525673615841 73.76987434472423) bank191640 +191641 POINT(40.28277115309329 74.401431425621) bank191641 +191642 POINT(40.4268336820971 73.86599354670483) bank191642 +191643 POINT(40.48124110253525 73.48755666189972) bank191643 +191644 POINT(41.185774629336414 73.88941042469833) bank191644 +191645 POINT(41.159945953530276 74.21555618784251) bank191645 +191646 POINT(40.87693817084532 73.9217430891206) bank191646 +191647 POINT(40.847780524504266 74.77674903011041) bank191647 +191648 POINT(41.26475844855902 74.08818013760931) bank191648 +191649 POINT(40.298553032407234 74.2631074166766) bank191649 +191650 POINT(39.96830292396042 74.53522886729016) bank191650 +191651 POINT(40.47599757354383 73.26336736846201) bank191651 +191652 POINT(40.2544542593391 74.67903601581811) bank191652 +191653 POINT(39.82543534800286 74.53077756192681) bank191653 +191654 POINT(39.77075894943675 74.85729645793064) bank191654 +191655 POINT(40.36187418776178 74.72073619921903) bank191655 +191656 POINT(39.946358987382716 74.26826091998853) bank191656 +191657 POINT(40.876588617101454 73.83019183938708) bank191657 +191658 POINT(41.679966455653755 73.43510458751192) bank191658 +191659 POINT(40.30416447495997 73.6380498299062) bank191659 +191660 POINT(39.80249931142682 73.94037524398833) bank191660 +191661 POINT(41.02071123221687 73.65533142168363) bank191661 +191662 POINT(40.8099028606181 74.54030371046707) bank191662 +191663 POINT(41.1126507353059 73.24810016544812) bank191663 +191664 POINT(41.68705394897363 73.75712963388071) bank191664 +191665 POINT(40.36752597835621 74.42120093654442) bank191665 +191666 POINT(41.27910578237615 74.96284078168051) bank191666 +191667 POINT(41.68454538010143 74.33535125300955) bank191667 +191668 POINT(40.46008879631996 73.25954763315562) bank191668 +191669 POINT(39.94231348816029 74.19377659630594) bank191669 +191670 POINT(40.41509370531286 74.32202742073679) bank191670 +191671 POINT(39.90126955163813 74.80517583880557) bank191671 +191672 POINT(41.69853242254668 73.81423747510505) bank191672 +191673 POINT(41.323119577285404 73.4418956867626) bank191673 +191674 POINT(41.54203018837057 73.51018655269353) bank191674 +191675 POINT(40.91828689459293 74.38230698400208) bank191675 +191676 POINT(40.13872034060837 73.24702170448799) bank191676 +191677 POINT(39.94545507965569 74.94226481829668) bank191677 +191678 POINT(41.24257738023002 74.84758859010896) bank191678 +191679 POINT(40.4426091952058 74.11373041241796) bank191679 +191680 POINT(40.94079045316569 73.15832526734685) bank191680 +191681 POINT(41.43873119436351 74.02421761184763) bank191681 +191682 POINT(40.71433798777833 73.23644088837837) bank191682 +191683 POINT(39.73609712803781 74.40435071459855) bank191683 +191684 POINT(40.12523379249495 73.59046490527373) bank191684 +191685 POINT(41.59374353223826 73.9684464199995) bank191685 +191686 POINT(40.88566118827797 74.86508364587999) bank191686 +191687 POINT(40.621537367501546 73.33594525278451) bank191687 +191688 POINT(40.060085491266136 73.05681185945734) bank191688 +191689 POINT(40.985502709618025 73.6718391122609) bank191689 +191690 POINT(40.684694327007726 74.46442844042865) bank191690 +191691 POINT(40.743425854190676 73.52599276314632) bank191691 +191692 POINT(40.478509951064595 74.31657107794743) bank191692 +191693 POINT(39.90725198565347 74.27293366643615) bank191693 +191694 POINT(40.20372882661262 73.1386191580891) bank191694 +191695 POINT(41.3640132497129 74.2228530888902) bank191695 +191696 POINT(41.41559426691454 74.23202170448884) bank191696 +191697 POINT(41.55073419836356 73.64129805128276) bank191697 +191698 POINT(41.68937618669248 74.44677391664462) bank191698 +191699 POINT(39.95222177700248 73.94952772510322) bank191699 +191700 POINT(39.97403489900981 73.14591881094115) bank191700 +191701 POINT(40.06490902817246 74.09827109997913) bank191701 +191702 POINT(41.634535786292666 73.516015902023) bank191702 +191703 POINT(40.49455556555265 74.80163418480703) bank191703 +191704 POINT(40.16453995135166 74.87619591973622) bank191704 +191705 POINT(40.872792102243785 74.6011903915157) bank191705 +191706 POINT(40.742906775983336 73.54965461317529) bank191706 +191707 POINT(39.85297560866779 74.23216202847418) bank191707 +191708 POINT(39.83724518808822 74.97977772993144) bank191708 +191709 POINT(40.17547866991222 73.46915029575864) bank191709 +191710 POINT(40.002546280396636 73.37714167202846) bank191710 +191711 POINT(40.2737745225607 74.54706957798024) bank191711 +191712 POINT(41.32235444100109 73.86716738990921) bank191712 +191713 POINT(39.77313573050691 74.19391002104365) bank191713 +191714 POINT(40.6826090072759 74.38401891620694) bank191714 +191715 POINT(41.100732380125734 73.15169489520288) bank191715 +191716 POINT(40.13418658006104 73.53963373694472) bank191716 +191717 POINT(39.93359288855714 73.29590733930038) bank191717 +191718 POINT(40.33291803252793 73.75112679821424) bank191718 +191719 POINT(40.19530080306723 74.37315165885848) bank191719 +191720 POINT(40.43243619629255 74.58819758999724) bank191720 +191721 POINT(39.755137538108166 74.59641446498877) bank191721 +191722 POINT(40.808188045871844 73.5221827228235) bank191722 +191723 POINT(40.31311435295481 74.44461426624466) bank191723 +191724 POINT(41.35831053430429 74.90280840912918) bank191724 +191725 POINT(41.309622194916464 73.165842745138) bank191725 +191726 POINT(41.261620418201375 73.30050938847134) bank191726 +191727 POINT(40.058968176555624 74.52995262991188) bank191727 +191728 POINT(39.71553085739009 74.5845004199559) bank191728 +191729 POINT(40.15463035289786 74.8287320273708) bank191729 +191730 POINT(40.23577298100889 74.288531056867) bank191730 +191731 POINT(40.04856972151061 73.38860065305239) bank191731 +191732 POINT(40.86600058226847 73.40723347338088) bank191732 +191733 POINT(40.61280958466361 73.87142915339469) bank191733 +191734 POINT(40.72269525347811 74.98451881477972) bank191734 +191735 POINT(40.82058663203213 74.49762748185914) bank191735 +191736 POINT(39.768262210356774 74.00136406172032) bank191736 +191737 POINT(41.399481503720956 73.57872154564038) bank191737 +191738 POINT(39.83470486007828 74.57397414994361) bank191738 +191739 POINT(40.63697783824486 73.47808389540667) bank191739 +191740 POINT(41.23352166028386 74.33023030233532) bank191740 +191741 POINT(41.52552862674697 74.48591967655442) bank191741 +191742 POINT(39.822092888633215 73.44056575005523) bank191742 +191743 POINT(40.01891132761498 73.34393151755508) bank191743 +191744 POINT(40.97359563793506 73.03319437383368) bank191744 +191745 POINT(41.54401373257594 74.4071642309431) bank191745 +191746 POINT(40.57694663736343 74.4708839420842) bank191746 +191747 POINT(40.591874322798006 73.66884352883235) bank191747 +191748 POINT(40.02486134860195 74.38217282926094) bank191748 +191749 POINT(40.94138850804947 73.9164300645744) bank191749 +191750 POINT(41.21603002432027 73.80176306672169) bank191750 +191751 POINT(41.14885916208863 74.41918892054733) bank191751 +191752 POINT(40.865013396926955 74.38614381800495) bank191752 +191753 POINT(39.82646354124166 74.44438520449947) bank191753 +191754 POINT(40.78660047643478 73.41994793178576) bank191754 +191755 POINT(41.322355340778024 73.1209742932245) bank191755 +191756 POINT(40.04887003585616 73.98977387273705) bank191756 +191757 POINT(41.0307482602212 74.38396771516788) bank191757 +191758 POINT(41.40413611400109 73.64577394735524) bank191758 +191759 POINT(40.37154678604998 73.11940163563837) bank191759 +191760 POINT(39.83615162244664 73.58995626476357) bank191760 +191761 POINT(41.185978923617064 74.68891273338848) bank191761 +191762 POINT(41.55187896040878 73.85786356027553) bank191762 +191763 POINT(41.12000988322852 73.23479002651041) bank191763 +191764 POINT(40.77466961043692 74.45109706796391) bank191764 +191765 POINT(41.359129721608255 73.0251737430752) bank191765 +191766 POINT(39.92970340422288 74.1669228326723) bank191766 +191767 POINT(41.54362259162602 74.90475450747914) bank191767 +191768 POINT(41.15868361809326 74.11795952516522) bank191768 +191769 POINT(39.995756697563536 74.61561456838089) bank191769 +191770 POINT(41.508413289714994 74.53687312726937) bank191770 +191771 POINT(40.287664853777045 74.10251655461134) bank191771 +191772 POINT(40.11392163263196 73.87961065304856) bank191772 +191773 POINT(40.45725509651092 73.35363217814545) bank191773 +191774 POINT(40.88631143266865 74.87894301214389) bank191774 +191775 POINT(40.496670689888845 73.14438268651432) bank191775 +191776 POINT(39.856455067992066 73.24325383867318) bank191776 +191777 POINT(39.808757214342286 74.7091133396388) bank191777 +191778 POINT(41.37984866486262 73.3470792184978) bank191778 +191779 POINT(40.64272285985863 74.50027823489161) bank191779 +191780 POINT(40.481853791945504 74.84812989548509) bank191780 +191781 POINT(41.23843102649267 74.53190966692542) bank191781 +191782 POINT(40.60543933013482 73.94664952684273) bank191782 +191783 POINT(41.60860847505742 74.88845994171976) bank191783 +191784 POINT(41.57356791951647 74.28386622499538) bank191784 +191785 POINT(41.15716000969179 73.21477087568795) bank191785 +191786 POINT(41.62342459980334 74.54784522734917) bank191786 +191787 POINT(40.37573968734034 74.79510145149223) bank191787 +191788 POINT(40.67441696191324 73.40029650279098) bank191788 +191789 POINT(40.02305132816443 74.20955834049303) bank191789 +191790 POINT(40.736690120569364 74.87591463769922) bank191790 +191791 POINT(39.72642504191075 74.93100864195391) bank191791 +191792 POINT(40.599222885744354 73.36438150483077) bank191792 +191793 POINT(40.59057942324283 73.98012562295106) bank191793 +191794 POINT(39.94366621389491 74.22979261415806) bank191794 +191795 POINT(40.407423424922015 74.7301763539989) bank191795 +191796 POINT(40.672679425667255 73.21144008975106) bank191796 +191797 POINT(40.62307043256087 73.3641546471671) bank191797 +191798 POINT(40.42244494338258 74.98856771949849) bank191798 +191799 POINT(41.37662087836916 74.31539133200988) bank191799 +191800 POINT(40.8496962132149 73.74065553290608) bank191800 +191801 POINT(40.95257214669656 74.07676743952568) bank191801 +191802 POINT(40.85354436807779 73.22716352759605) bank191802 +191803 POINT(40.202796084861376 74.8944013286363) bank191803 +191804 POINT(40.91426137165611 73.19743222215251) bank191804 +191805 POINT(40.390855831137806 73.28546213116027) bank191805 +191806 POINT(41.100796366432704 74.73333140291376) bank191806 +191807 POINT(41.47833099273937 73.32435666083718) bank191807 +191808 POINT(40.88283198032179 74.95109130516597) bank191808 +191809 POINT(40.35838707250328 74.6295718907941) bank191809 +191810 POINT(41.30848310400458 73.38100081665755) bank191810 +191811 POINT(40.202274743393076 73.2079578185902) bank191811 +191812 POINT(41.448730253290606 74.94866930657948) bank191812 +191813 POINT(41.70005733677745 73.52596176920768) bank191813 +191814 POINT(40.89064402027697 74.82680852144928) bank191814 +191815 POINT(41.00931101467638 74.30667065700308) bank191815 +191816 POINT(41.6598967316249 73.95140814470786) bank191816 +191817 POINT(41.02691975927756 73.2818826081822) bank191817 +191818 POINT(41.20228596803595 75.0046703607006) bank191818 +191819 POINT(41.60320875206516 73.74628676379089) bank191819 +191820 POINT(40.825283495062976 74.40081446260723) bank191820 +191821 POINT(40.910332507189736 74.67970318209443) bank191821 +191822 POINT(39.87639180771239 73.31404064440244) bank191822 +191823 POINT(41.64155136425209 73.40027769830893) bank191823 +191824 POINT(41.33001906502504 74.29228853197466) bank191824 +191825 POINT(39.88978356509848 73.53367398815365) bank191825 +191826 POINT(41.15895251565843 73.516758249359) bank191826 +191827 POINT(40.597180525415176 74.95869566451826) bank191827 +191828 POINT(40.716217453313604 73.99043781960887) bank191828 +191829 POINT(39.89616789659892 73.60800763314845) bank191829 +191830 POINT(41.42342434733402 73.42192600465772) bank191830 +191831 POINT(41.47257698596405 73.15655671465117) bank191831 +191832 POINT(40.53109516247283 74.18944657296234) bank191832 +191833 POINT(39.932042525366754 73.42380124581547) bank191833 +191834 POINT(40.60633712781349 74.85542030482236) bank191834 +191835 POINT(41.217696847502566 73.82968885205331) bank191835 +191836 POINT(40.95822807449798 73.61096152387367) bank191836 +191837 POINT(41.11814689228699 73.8078864781562) bank191837 +191838 POINT(39.80663758142912 74.56748278182802) bank191838 +191839 POINT(40.1075118692795 73.2996631178117) bank191839 +191840 POINT(40.20711913718271 73.58709351766062) bank191840 +191841 POINT(40.33074782566839 74.1948018357209) bank191841 +191842 POINT(40.65495331986012 74.27391086470908) bank191842 +191843 POINT(39.887217277397504 73.20091608300545) bank191843 +191844 POINT(41.08310565650148 73.2345757733572) bank191844 +191845 POINT(41.700710310499396 74.5967658752818) bank191845 +191846 POINT(40.68062886205054 74.49965906472603) bank191846 +191847 POINT(40.46559241363336 74.16155191964377) bank191847 +191848 POINT(40.58865803434284 74.42916559427057) bank191848 +191849 POINT(41.20838623549193 73.13972862405267) bank191849 +191850 POINT(40.194553395356714 73.6230542012988) bank191850 +191851 POINT(41.644588563228254 73.7546931032589) bank191851 +191852 POINT(41.372983134053115 74.27485564853583) bank191852 +191853 POINT(40.717001333978374 73.59536147867234) bank191853 +191854 POINT(40.470389054125626 73.55503550973819) bank191854 +191855 POINT(41.17369812269381 74.77179784284799) bank191855 +191856 POINT(41.545915024469124 73.06805056090164) bank191856 +191857 POINT(39.979038881444026 73.50695568298167) bank191857 +191858 POINT(40.606159533173994 74.93356950300081) bank191858 +191859 POINT(40.05427928461458 73.44346586625095) bank191859 +191860 POINT(40.90079746905639 73.58681699624545) bank191860 +191861 POINT(40.87957623946312 73.0561223195344) bank191861 +191862 POINT(40.91480706500388 74.19888934102671) bank191862 +191863 POINT(39.71603374366662 73.51417860462982) bank191863 +191864 POINT(40.846816752018036 74.93285424667812) bank191864 +191865 POINT(40.46619366689598 73.82550824644025) bank191865 +191866 POINT(40.622053267798734 74.0259818375507) bank191866 +191867 POINT(40.84119553712836 73.85158424840839) bank191867 +191868 POINT(40.80358606019705 74.55940273324072) bank191868 +191869 POINT(40.59461735080781 74.99734444594282) bank191869 +191870 POINT(41.528517127828856 74.76088234739697) bank191870 +191871 POINT(40.507157342044806 73.59007864302019) bank191871 +191872 POINT(40.87027858667577 73.99879023723337) bank191872 +191873 POINT(41.55009040862249 74.02995582226472) bank191873 +191874 POINT(40.97032546815411 73.98001604952333) bank191874 +191875 POINT(39.80840501207123 74.94238658116373) bank191875 +191876 POINT(41.1069963812518 74.80041388298224) bank191876 +191877 POINT(40.37463606537621 74.98146079926606) bank191877 +191878 POINT(39.89111641266452 74.89912931855629) bank191878 +191879 POINT(41.083605737796645 73.26721796482464) bank191879 +191880 POINT(39.79175730408314 73.39575903466658) bank191880 +191881 POINT(41.52776369369927 73.9298257135984) bank191881 +191882 POINT(40.64914154114262 74.53647398772473) bank191882 +191883 POINT(40.578711216357206 73.8737455296888) bank191883 +191884 POINT(40.512587901638256 74.79477385012419) bank191884 +191885 POINT(39.98236678586014 73.4293596892031) bank191885 +191886 POINT(39.78434694733091 73.63052410321106) bank191886 +191887 POINT(41.68606208457398 73.30981340996476) bank191887 +191888 POINT(41.23864332316124 74.10013845440359) bank191888 +191889 POINT(39.85633636064557 73.82437900823909) bank191889 +191890 POINT(41.15522368406691 74.92931934887542) bank191890 +191891 POINT(40.11167109134185 73.4196405616438) bank191891 +191892 POINT(40.80923917339305 73.01606761039747) bank191892 +191893 POINT(41.70433910215295 73.69012362834589) bank191893 +191894 POINT(40.16819342837112 74.9965091264226) bank191894 +191895 POINT(40.10808148691869 73.40515700932538) bank191895 +191896 POINT(41.02889451454985 73.85645978586932) bank191896 +191897 POINT(39.94075853947714 74.8569515528561) bank191897 +191898 POINT(40.482424503043234 73.41294757325105) bank191898 +191899 POINT(40.28059538169785 73.15184263482917) bank191899 +191900 POINT(41.578992900564245 74.22506223290686) bank191900 +191901 POINT(40.161183137205406 73.12048039072323) bank191901 +191902 POINT(41.47088903500568 74.62978582388612) bank191902 +191903 POINT(40.75241935215705 74.0660868666461) bank191903 +191904 POINT(40.16336506122201 73.78545657030709) bank191904 +191905 POINT(40.05599910144893 73.66493027922627) bank191905 +191906 POINT(41.14789355836941 74.43652169689615) bank191906 +191907 POINT(41.5643653241416 74.26141963987553) bank191907 +191908 POINT(40.58050161405069 73.27698676586438) bank191908 +191909 POINT(40.05616105347664 73.33896462037353) bank191909 +191910 POINT(39.99205854497775 74.94093989129635) bank191910 +191911 POINT(40.23118097189991 74.43790391959462) bank191911 +191912 POINT(41.0901476087223 74.35087757640034) bank191912 +191913 POINT(39.875598461660886 73.87149532665133) bank191913 +191914 POINT(40.249510652497115 74.07311077943056) bank191914 +191915 POINT(40.61819892929009 73.42867154996853) bank191915 +191916 POINT(39.85276922230611 74.61958074031219) bank191916 +191917 POINT(40.042825018623255 73.51339625183321) bank191917 +191918 POINT(40.43425823536911 73.89708260657547) bank191918 +191919 POINT(40.086765657522875 73.5526648964565) bank191919 +191920 POINT(41.168975126377674 74.63868565883365) bank191920 +191921 POINT(41.467631470267534 74.56861719851062) bank191921 +191922 POINT(40.32989791639298 74.85775168583804) bank191922 +191923 POINT(40.88836989420958 73.09542472036664) bank191923 +191924 POINT(40.71183087537082 73.60163178918809) bank191924 +191925 POINT(39.916403692409816 74.3031143983169) bank191925 +191926 POINT(41.51139184495142 73.75871247615925) bank191926 +191927 POINT(40.668148255353714 74.6391024268214) bank191927 +191928 POINT(39.88557997535754 73.55859165519199) bank191928 +191929 POINT(40.40301807725267 73.74846183118669) bank191929 +191930 POINT(41.35560176674374 74.50695589005552) bank191930 +191931 POINT(40.376803953475054 73.22803552626561) bank191931 +191932 POINT(40.44036313718093 74.29421799875371) bank191932 +191933 POINT(40.339352684552146 74.46204742486177) bank191933 +191934 POINT(40.34590085716506 73.43282717996658) bank191934 +191935 POINT(40.82753801761057 73.19109794275757) bank191935 +191936 POINT(40.396702595287785 73.2714989152723) bank191936 +191937 POINT(40.28806699325464 74.96456044482844) bank191937 +191938 POINT(40.41541139325374 74.44887392507951) bank191938 +191939 POINT(41.225530634578156 73.36135766559507) bank191939 +191940 POINT(41.367446728789915 73.5195176563161) bank191940 +191941 POINT(39.78958230646967 73.02608472393123) bank191941 +191942 POINT(41.66472869066506 74.41435084766674) bank191942 +191943 POINT(41.69580722058422 74.82877079274871) bank191943 +191944 POINT(41.437692046833135 74.46387493326488) bank191944 +191945 POINT(40.650668881092955 73.5308679600873) bank191945 +191946 POINT(40.991123578713854 74.5812925343354) bank191946 +191947 POINT(41.614746273363735 73.29741839631325) bank191947 +191948 POINT(40.27981590859632 73.94892143078812) bank191948 +191949 POINT(41.70970061681917 73.04947176673102) bank191949 +191950 POINT(40.44647606347748 74.1592199206882) bank191950 +191951 POINT(40.95841485430419 74.99607854037455) bank191951 +191952 POINT(41.56425865550354 74.531672581088) bank191952 +191953 POINT(40.02378322028137 74.0095759224761) bank191953 +191954 POINT(41.66100926169548 73.54191595864552) bank191954 +191955 POINT(40.91089996619152 73.71119651102661) bank191955 +191956 POINT(39.87475496075816 73.223308154136) bank191956 +191957 POINT(40.58564746819464 73.94137424671695) bank191957 +191958 POINT(40.12911295234386 74.04053836777388) bank191958 +191959 POINT(41.119563180600494 74.19680508076225) bank191959 +191960 POINT(41.47856339300454 73.77717787512046) bank191960 +191961 POINT(40.93643362215501 74.68797251536286) bank191961 +191962 POINT(41.328644422344524 73.46788084565387) bank191962 +191963 POINT(40.417765309864016 74.90238765729046) bank191963 +191964 POINT(41.13575292910398 74.60261514013501) bank191964 +191965 POINT(41.52277994662852 74.15445761726006) bank191965 +191966 POINT(41.635356792745654 74.9762696572713) bank191966 +191967 POINT(40.40708862138143 73.47800456588271) bank191967 +191968 POINT(41.10634249919795 74.28170133480042) bank191968 +191969 POINT(41.11296157842856 73.26480084630677) bank191969 +191970 POINT(40.75404643766492 74.43235455280461) bank191970 +191971 POINT(39.90027879351315 73.8188739924701) bank191971 +191972 POINT(40.31115972667691 73.62928992446459) bank191972 +191973 POINT(40.4986240351112 73.89417187867134) bank191973 +191974 POINT(41.01857912552481 74.04120716096148) bank191974 +191975 POINT(41.646562179801755 74.05082597733202) bank191975 +191976 POINT(40.167068702508004 74.23441884117535) bank191976 +191977 POINT(41.495890395771255 73.39780604617266) bank191977 +191978 POINT(41.49855207392935 73.90629036010432) bank191978 +191979 POINT(40.761865353360086 73.52763427478148) bank191979 +191980 POINT(40.233839303621195 73.67912681073335) bank191980 +191981 POINT(40.33078448289644 73.48902099964813) bank191981 +191982 POINT(41.28171722324841 73.91601286683687) bank191982 +191983 POINT(41.67077198060703 73.52210040211648) bank191983 +191984 POINT(40.672411304150884 74.26105829441796) bank191984 +191985 POINT(40.27367751972215 74.41800607913999) bank191985 +191986 POINT(40.3042845415828 73.05648870729979) bank191986 +191987 POINT(41.26765667750107 73.72642915338614) bank191987 +191988 POINT(41.41693519914659 73.65361130565735) bank191988 +191989 POINT(40.57754969449447 74.36869038334164) bank191989 +191990 POINT(39.863798964184696 74.3785481901859) bank191990 +191991 POINT(40.88485622327259 74.4573505760741) bank191991 +191992 POINT(41.6584881960514 73.17640964088504) bank191992 +191993 POINT(40.534627579120745 73.11797949470997) bank191993 +191994 POINT(41.028946471723636 74.95983153739037) bank191994 +191995 POINT(40.04872839620388 74.93893023422396) bank191995 +191996 POINT(41.51975318929959 74.69860246193326) bank191996 +191997 POINT(41.157438324739736 73.407188372789) bank191997 +191998 POINT(39.776817450360305 73.94338699684239) bank191998 +191999 POINT(41.35575725505722 73.89187977208829) bank191999 +192000 POINT(41.54664969217496 73.87325158877363) bank192000 +192001 POINT(41.05149135267605 74.68550743799157) bank192001 +192002 POINT(40.85268637444701 73.69205400675185) bank192002 +192003 POINT(41.07916394333042 73.86603434654337) bank192003 +192004 POINT(39.9634120668771 73.35912658466434) bank192004 +192005 POINT(41.521591806947065 74.22287763961518) bank192005 +192006 POINT(41.34685503635786 73.6696766722891) bank192006 +192007 POINT(40.27663961396831 73.36937456927228) bank192007 +192008 POINT(41.67389260038904 73.63000157337851) bank192008 +192009 POINT(41.0311887255 74.62231308866515) bank192009 +192010 POINT(40.36206283653073 73.77511046990632) bank192010 +192011 POINT(40.71693644520213 73.92285034514326) bank192011 +192012 POINT(39.992719477309144 73.1576875681547) bank192012 +192013 POINT(40.297677994837755 74.27933052518351) bank192013 +192014 POINT(39.834354573386825 74.14595653402004) bank192014 +192015 POINT(40.8597030509715 73.34812645507533) bank192015 +192016 POINT(40.59226820694099 73.50679210538196) bank192016 +192017 POINT(41.570570462722 73.48941794821853) bank192017 +192018 POINT(41.00402290812284 74.17056859822956) bank192018 +192019 POINT(40.456442500205505 74.03017986003718) bank192019 +192020 POINT(40.430420907040094 73.38245661675747) bank192020 +192021 POINT(41.17951890763139 74.0773006693319) bank192021 +192022 POINT(41.53730073421741 73.87805027105044) bank192022 +192023 POINT(41.66848465379045 74.89917832115285) bank192023 +192024 POINT(40.45726937064409 74.51446714143086) bank192024 +192025 POINT(41.317863146152796 74.83066187165245) bank192025 +192026 POINT(40.595046595159516 74.84664577421802) bank192026 +192027 POINT(40.269983769265615 73.48439606723252) bank192027 +192028 POINT(40.92080379991213 73.16460673247528) bank192028 +192029 POINT(40.53484147769025 73.6973607058578) bank192029 +192030 POINT(40.71737870504596 74.23595102837442) bank192030 +192031 POINT(39.81239930322672 74.78737351579784) bank192031 +192032 POINT(41.11430570741863 73.73535072550334) bank192032 +192033 POINT(40.754786958103566 74.16104492414091) bank192033 +192034 POINT(40.774898108818675 74.26655177419654) bank192034 +192035 POINT(40.93347988462485 73.88600810494808) bank192035 +192036 POINT(40.04533929409846 74.20160802733822) bank192036 +192037 POINT(41.20184548180936 74.0942175735634) bank192037 +192038 POINT(41.20833872037767 74.97859827570208) bank192038 +192039 POINT(40.73336706780043 74.42872552363319) bank192039 +192040 POINT(41.04680982425623 74.33111315183272) bank192040 +192041 POINT(39.94442779659648 74.61778419753865) bank192041 +192042 POINT(41.38119019800281 73.0604111201167) bank192042 +192043 POINT(40.46248543746398 74.59290747998541) bank192043 +192044 POINT(40.66759890576872 74.91540278367826) bank192044 +192045 POINT(40.50522380442367 74.22005627293288) bank192045 +192046 POINT(41.58931837988629 74.4012472884342) bank192046 +192047 POINT(40.70234846166544 73.55508549619238) bank192047 +192048 POINT(39.88244241313932 74.52718423789383) bank192048 +192049 POINT(41.291384109086515 73.62594161803332) bank192049 +192050 POINT(40.46347143179959 73.44709427484516) bank192050 +192051 POINT(40.20138371299443 73.8267068395287) bank192051 +192052 POINT(40.1205809880554 73.28593307358852) bank192052 +192053 POINT(39.74971309793709 74.38338518900102) bank192053 +192054 POINT(40.12008169189524 74.1759017090976) bank192054 +192055 POINT(41.65083537413008 74.58343772446004) bank192055 +192056 POINT(41.66758858842135 73.20859110824209) bank192056 +192057 POINT(41.106019600009304 73.05306312474288) bank192057 +192058 POINT(40.442368642040805 74.95008535417415) bank192058 +192059 POINT(39.79285897152385 73.80727968699237) bank192059 +192060 POINT(41.67253161502626 74.72586810028146) bank192060 +192061 POINT(40.44831472419552 74.13121230292099) bank192061 +192062 POINT(40.67290963041345 74.62928479723226) bank192062 +192063 POINT(40.98065607540337 74.5261220142544) bank192063 +192064 POINT(41.5988408910499 74.0693224188966) bank192064 +192065 POINT(39.80278223383501 73.13308241296454) bank192065 +192066 POINT(40.86198370261513 74.87465318567537) bank192066 +192067 POINT(40.56463061680987 74.80929520223603) bank192067 +192068 POINT(40.15596781027176 73.2897544392274) bank192068 +192069 POINT(39.80807258723777 74.2949249100358) bank192069 +192070 POINT(39.75674736616474 74.56780505947187) bank192070 +192071 POINT(40.32832162882509 74.23252433663784) bank192071 +192072 POINT(40.38600639172144 74.09367803460104) bank192072 +192073 POINT(39.73965568954373 73.35291824096413) bank192073 +192074 POINT(41.29048897864204 73.22832529802223) bank192074 +192075 POINT(41.46982043957677 74.35555487927265) bank192075 +192076 POINT(41.60524634429222 73.43517415322712) bank192076 +192077 POINT(40.99971240288619 73.02559871022603) bank192077 +192078 POINT(39.7423266352751 73.66766995340916) bank192078 +192079 POINT(40.31481313957407 73.55820098680609) bank192079 +192080 POINT(41.552213455793996 74.29592848856267) bank192080 +192081 POINT(41.20213084300707 73.84834221046137) bank192081 +192082 POINT(39.98665695166519 74.1317989945325) bank192082 +192083 POINT(40.620456513180784 74.2969073213448) bank192083 +192084 POINT(41.17631756332731 74.40222933797648) bank192084 +192085 POINT(39.93419180876767 73.15466420454857) bank192085 +192086 POINT(40.759817623614644 73.63306229835014) bank192086 +192087 POINT(41.56824091539472 74.77883048813982) bank192087 +192088 POINT(40.2084317938641 74.05914931844853) bank192088 +192089 POINT(40.18367159379744 74.74349969014519) bank192089 +192090 POINT(40.571324347292 73.37377817475117) bank192090 +192091 POINT(40.24101543107872 74.10321388802434) bank192091 +192092 POINT(40.32302231142742 74.37372568871832) bank192092 +192093 POINT(41.12882372043799 74.39386690013127) bank192093 +192094 POINT(41.7102034525878 73.21721788915013) bank192094 +192095 POINT(41.39592954183815 73.4314765558307) bank192095 +192096 POINT(41.29172512206914 73.75671874056177) bank192096 +192097 POINT(41.29478371420903 73.67457386459068) bank192097 +192098 POINT(40.424453767608476 74.42896378417541) bank192098 +192099 POINT(40.58679980891763 74.30346793568852) bank192099 +192100 POINT(39.71503923957913 74.82243947262816) bank192100 +192101 POINT(40.1530252225904 74.12427676314661) bank192101 +192102 POINT(40.030342709151455 74.58313150411364) bank192102 +192103 POINT(39.9950546475417 73.78951264460308) bank192103 +192104 POINT(40.02884034811237 73.3462495095262) bank192104 +192105 POINT(40.48744918443461 73.54773842081718) bank192105 +192106 POINT(40.45498553837177 73.25379501868568) bank192106 +192107 POINT(41.059017267582014 74.79267418908658) bank192107 +192108 POINT(40.609475100946185 73.47406716287149) bank192108 +192109 POINT(41.675807991486174 74.23027675035517) bank192109 +192110 POINT(41.04258951247469 73.860624903029) bank192110 +192111 POINT(40.74122744060492 74.5523242699842) bank192111 +192112 POINT(40.558816332805456 73.28339516295087) bank192112 +192113 POINT(41.266918928119864 73.55823266556408) bank192113 +192114 POINT(40.304004912369074 73.46218702291736) bank192114 +192115 POINT(41.69713195583916 74.62185443793437) bank192115 +192116 POINT(40.610053841331016 74.45463002340843) bank192116 +192117 POINT(41.10498682848438 74.06597616961314) bank192117 +192118 POINT(40.78262914587224 74.26729924335638) bank192118 +192119 POINT(40.433166953694474 74.17304698501782) bank192119 +192120 POINT(40.74453277276732 74.98261305415551) bank192120 +192121 POINT(41.42603914497873 74.9956877175428) bank192121 +192122 POINT(41.599081138870304 74.10616410659247) bank192122 +192123 POINT(40.84928724337071 73.37929100043864) bank192123 +192124 POINT(41.270593562199906 73.79863701702878) bank192124 +192125 POINT(40.060930784689624 73.09788883051176) bank192125 +192126 POINT(39.98977529208267 73.28971790884881) bank192126 +192127 POINT(40.2297099934962 73.45126867460034) bank192127 +192128 POINT(40.58392905716638 74.5583522703114) bank192128 +192129 POINT(41.597814318767306 74.16534100113462) bank192129 +192130 POINT(41.15764009702021 74.94836877729446) bank192130 +192131 POINT(40.04523830308046 73.0465134564088) bank192131 +192132 POINT(41.48236532996638 74.36451177882931) bank192132 +192133 POINT(41.50835197995179 74.48035331743372) bank192133 +192134 POINT(40.484875119570056 74.84977885978762) bank192134 +192135 POINT(41.40683337664114 73.16667808992823) bank192135 +192136 POINT(39.74273344477493 74.63396829670945) bank192136 +192137 POINT(40.83044143098692 74.11925380412247) bank192137 +192138 POINT(41.70532485241898 74.3804033875999) bank192138 +192139 POINT(40.730537952698015 73.62780355820333) bank192139 +192140 POINT(39.75567650331699 74.89057064595947) bank192140 +192141 POINT(39.80413567555019 74.96003259338949) bank192141 +192142 POINT(40.04496695762093 73.35793435522335) bank192142 +192143 POINT(40.03193298221392 73.32209680320673) bank192143 +192144 POINT(41.675868198474454 73.99387420604043) bank192144 +192145 POINT(40.799216003975694 74.74730263521191) bank192145 +192146 POINT(40.398047058346684 74.31352870085227) bank192146 +192147 POINT(40.99321607764913 73.59987177803) bank192147 +192148 POINT(41.449130990679876 74.21626230495558) bank192148 +192149 POINT(40.05404123535247 74.26875448542401) bank192149 +192150 POINT(41.5381739406055 74.67742468934996) bank192150 +192151 POINT(40.01208539218975 73.51708562872126) bank192151 +192152 POINT(41.35777735034401 73.25174874305858) bank192152 +192153 POINT(41.037140104880784 73.91909162339036) bank192153 +192154 POINT(41.7053199980675 73.08421515093734) bank192154 +192155 POINT(41.256314889880244 74.52648726777542) bank192155 +192156 POINT(41.45049941011363 73.98843125548206) bank192156 +192157 POINT(40.35954924474053 74.32828138013963) bank192157 +192158 POINT(40.179754831601834 73.31535133478886) bank192158 +192159 POINT(40.63044424105008 74.93183488656416) bank192159 +192160 POINT(40.44517631083477 73.61499633738353) bank192160 +192161 POINT(41.303330795720335 74.99778799766024) bank192161 +192162 POINT(41.67668346166077 74.74208257332451) bank192162 +192163 POINT(41.28983779428008 73.3174442847955) bank192163 +192164 POINT(41.282376042457244 73.16683679884436) bank192164 +192165 POINT(40.772150508806 74.7206909778351) bank192165 +192166 POINT(41.10236040732825 74.88298599326681) bank192166 +192167 POINT(40.89425887581692 73.330224608462) bank192167 +192168 POINT(40.34423088946675 73.06244829381225) bank192168 +192169 POINT(39.77637053217644 73.55364852352531) bank192169 +192170 POINT(40.64476586240005 73.13611533409107) bank192170 +192171 POINT(39.74793556873963 73.32172307859742) bank192171 +192172 POINT(40.0581803266317 74.21005188337872) bank192172 +192173 POINT(41.30976049096755 73.55896484171777) bank192173 +192174 POINT(40.53904758388917 73.34535207873239) bank192174 +192175 POINT(40.69814021589222 73.31453645812756) bank192175 +192176 POINT(40.28184265873749 74.3303519004386) bank192176 +192177 POINT(41.422738045083435 74.26839328165312) bank192177 +192178 POINT(41.261652553846986 73.01048666377363) bank192178 +192179 POINT(41.12856642279894 73.57631929917662) bank192179 +192180 POINT(39.99614821929183 74.84149261563334) bank192180 +192181 POINT(40.70360850239327 74.5137446955422) bank192181 +192182 POINT(41.199708831082575 74.24340113929729) bank192182 +192183 POINT(40.08326386011188 73.41350459266926) bank192183 +192184 POINT(39.86498290308033 74.32492622525794) bank192184 +192185 POINT(41.16485107288319 74.39636364284378) bank192185 +192186 POINT(40.6383380586606 74.3773258638437) bank192186 +192187 POINT(41.50023779806679 74.59523024636387) bank192187 +192188 POINT(40.030462353517336 74.28535852093115) bank192188 +192189 POINT(41.35980428420465 74.66810091868122) bank192189 +192190 POINT(40.591322060857806 73.62576122729966) bank192190 +192191 POINT(39.93970612363443 73.10428972917876) bank192191 +192192 POINT(41.1379185776021 74.04824583859289) bank192192 +192193 POINT(41.609978932542326 74.33432328805817) bank192193 +192194 POINT(41.634533490697336 74.2893091968804) bank192194 +192195 POINT(40.47156223212026 74.55539481796117) bank192195 +192196 POINT(39.72266320028507 74.80161252673163) bank192196 +192197 POINT(39.93243150238908 74.86037336531074) bank192197 +192198 POINT(39.962213754197805 74.66673616678105) bank192198 +192199 POINT(40.012206680099084 74.24620614663966) bank192199 +192200 POINT(40.70680311580681 73.16338665553101) bank192200 +192201 POINT(40.66305045069324 73.18822111167873) bank192201 +192202 POINT(41.162158956387124 73.44005988272532) bank192202 +192203 POINT(40.55347658106067 73.38563773469643) bank192203 +192204 POINT(40.65998880881189 73.54742526257529) bank192204 +192205 POINT(41.42621031814068 74.35380668969069) bank192205 +192206 POINT(40.011547355077454 74.5271147890804) bank192206 +192207 POINT(40.20710881752795 74.28532800426781) bank192207 +192208 POINT(40.02342478450781 74.5544151389866) bank192208 +192209 POINT(41.24443379267886 73.9229706026106) bank192209 +192210 POINT(41.107622963851746 74.88790782912024) bank192210 +192211 POINT(40.32224012628524 74.61768111490265) bank192211 +192212 POINT(40.32661088238516 73.45719576414618) bank192212 +192213 POINT(41.229300651921235 74.79890190609979) bank192213 +192214 POINT(40.82818232283373 74.25163543748695) bank192214 +192215 POINT(40.70208685973779 73.58770702586165) bank192215 +192216 POINT(41.0211027551894 74.23008103910347) bank192216 +192217 POINT(40.01264371050631 74.49172996173957) bank192217 +192218 POINT(41.41758795204851 73.88284579659808) bank192218 +192219 POINT(40.22755010395798 74.56534811036536) bank192219 +192220 POINT(40.43280292944727 73.26342830150531) bank192220 +192221 POINT(41.264993562344095 73.89529849190987) bank192221 +192222 POINT(40.83778398912845 74.79287514534255) bank192222 +192223 POINT(40.26214240315575 74.34817762423476) bank192223 +192224 POINT(39.8519495886672 73.48026713784733) bank192224 +192225 POINT(41.708337395730936 73.2141492469702) bank192225 +192226 POINT(40.3877007588961 74.4585469630726) bank192226 +192227 POINT(40.38839340353705 74.41795055106412) bank192227 +192228 POINT(41.32645577100675 74.41293061588173) bank192228 +192229 POINT(41.50419164157723 73.34616645736988) bank192229 +192230 POINT(41.33921389949198 74.6260399315012) bank192230 +192231 POINT(40.460608437447306 73.3267741342143) bank192231 +192232 POINT(39.93979014667082 73.56737820323154) bank192232 +192233 POINT(41.40600569648791 73.53057304312937) bank192233 +192234 POINT(40.43820557636835 74.05643460660004) bank192234 +192235 POINT(40.35516221274306 74.08792428830621) bank192235 +192236 POINT(40.755847941102594 74.85606182553967) bank192236 +192237 POINT(39.869075888162016 74.56552716001478) bank192237 +192238 POINT(39.750147823694924 73.70768203909942) bank192238 +192239 POINT(41.29399547579394 74.80547301606323) bank192239 +192240 POINT(40.39325170360784 74.58398654523123) bank192240 +192241 POINT(40.111655427825234 74.20944050255105) bank192241 +192242 POINT(39.93281031870608 73.95287029886227) bank192242 +192243 POINT(40.35544782758422 73.08313548308338) bank192243 +192244 POINT(41.50735037065158 73.74936915280809) bank192244 +192245 POINT(40.40004634511275 73.51701635272329) bank192245 +192246 POINT(40.29240707990748 74.9965886955494) bank192246 +192247 POINT(39.85976209639548 73.19214258798822) bank192247 +192248 POINT(40.542458501528834 74.42045292593441) bank192248 +192249 POINT(41.02785583532817 73.39352770003222) bank192249 +192250 POINT(39.876419722570496 74.58636931360766) bank192250 +192251 POINT(41.71101678709536 74.66222134742883) bank192251 +192252 POINT(41.65623901312068 74.33101130809388) bank192252 +192253 POINT(40.52057987022412 74.74028826824049) bank192253 +192254 POINT(40.209759668935966 73.99447127386252) bank192254 +192255 POINT(40.76941175953168 74.30180074664969) bank192255 +192256 POINT(40.02380494852905 74.94056060040353) bank192256 +192257 POINT(39.9834719603279 73.30195654763257) bank192257 +192258 POINT(40.921665601268195 74.31307872889) bank192258 +192259 POINT(40.76108201166975 73.20842512988636) bank192259 +192260 POINT(41.69774384308537 73.11275855076751) bank192260 +192261 POINT(40.61879605792083 73.90208475211526) bank192261 +192262 POINT(40.570630435272896 73.89816221111377) bank192262 +192263 POINT(40.53707004721992 74.00721044279943) bank192263 +192264 POINT(41.04192307922704 73.14557822515039) bank192264 +192265 POINT(41.495247422771186 74.9147045774925) bank192265 +192266 POINT(40.12323455937446 73.16222319232907) bank192266 +192267 POINT(40.2328127475544 74.7928583606144) bank192267 +192268 POINT(40.436692735157465 74.79919637455647) bank192268 +192269 POINT(41.394769876955905 73.74604757327789) bank192269 +192270 POINT(41.48202197914742 73.84881259968681) bank192270 +192271 POINT(39.81966904313847 74.89888918793577) bank192271 +192272 POINT(41.51755763085574 74.96312073686686) bank192272 +192273 POINT(40.481585913879044 74.4299965878121) bank192273 +192274 POINT(41.31831634735918 73.12316174243387) bank192274 +192275 POINT(41.21329176568252 73.3350553021617) bank192275 +192276 POINT(41.3486823504995 74.05827041764736) bank192276 +192277 POINT(41.389174959299666 73.79997517814691) bank192277 +192278 POINT(39.844725919483444 73.76177199538924) bank192278 +192279 POINT(39.812675662183814 73.56997260188605) bank192279 +192280 POINT(41.43879803968979 73.80494795347444) bank192280 +192281 POINT(40.063245846090226 74.98362751403747) bank192281 +192282 POINT(40.707574201427036 74.89262580485138) bank192282 +192283 POINT(40.81715952262095 74.07912085357874) bank192283 +192284 POINT(40.72179673442042 73.32216043942049) bank192284 +192285 POINT(39.81499456317649 73.9555394713507) bank192285 +192286 POINT(40.46963758655999 73.22175121309043) bank192286 +192287 POINT(40.38787559901186 73.86609934109441) bank192287 +192288 POINT(41.3523176222264 73.89036875271913) bank192288 +192289 POINT(40.272391469349124 74.73496264066226) bank192289 +192290 POINT(41.02157551778436 73.04126005793869) bank192290 +192291 POINT(40.70429672827074 74.8591573258599) bank192291 +192292 POINT(39.91925257588459 73.74959082995947) bank192292 +192293 POINT(40.1755382231107 74.66852535437717) bank192293 +192294 POINT(39.91013497518303 73.21244900065523) bank192294 +192295 POINT(41.05367114240577 74.07087976878597) bank192295 +192296 POINT(41.05835959209208 73.94336657817867) bank192296 +192297 POINT(40.1760858676318 73.330678906917) bank192297 +192298 POINT(39.98722767798578 73.87195449800387) bank192298 +192299 POINT(40.783231490714826 73.4375817497175) bank192299 +192300 POINT(39.80569796581715 73.16274548561235) bank192300 +192301 POINT(40.850528113981426 73.67289334204239) bank192301 +192302 POINT(41.19640076922639 74.04587094751146) bank192302 +192303 POINT(39.76499760441258 74.57023461191406) bank192303 +192304 POINT(40.58066274703065 73.96102415164742) bank192304 +192305 POINT(39.99324361939784 73.51340593694817) bank192305 +192306 POINT(39.939596974182145 73.81290224843832) bank192306 +192307 POINT(41.650978364281805 73.46876761638407) bank192307 +192308 POINT(41.291676890064444 74.16940216162413) bank192308 +192309 POINT(40.12107158304866 73.96686264655645) bank192309 +192310 POINT(39.94783244798515 73.60816948479722) bank192310 +192311 POINT(40.78902201183972 73.34935102094848) bank192311 +192312 POINT(40.04475164967749 74.02773158912112) bank192312 +192313 POINT(41.10194714803632 73.36979465993254) bank192313 +192314 POINT(40.868210502508035 73.67501693188478) bank192314 +192315 POINT(41.09965163945178 73.82139852126943) bank192315 +192316 POINT(41.29414311010094 74.17676615796289) bank192316 +192317 POINT(40.30492553612958 74.65951836988366) bank192317 +192318 POINT(39.79550039010526 74.6509717834087) bank192318 +192319 POINT(41.26951496070606 74.97455755503992) bank192319 +192320 POINT(40.199602140631875 73.06695020193213) bank192320 +192321 POINT(41.514782392918086 73.07310084341306) bank192321 +192322 POINT(39.99323594799776 74.41331496539524) bank192322 +192323 POINT(40.322002830535716 74.26434032228416) bank192323 +192324 POINT(41.035967998273115 73.67103588617712) bank192324 +192325 POINT(40.266761142651276 73.81312156780348) bank192325 +192326 POINT(40.35692055194867 74.0339775773044) bank192326 +192327 POINT(39.983478143446995 74.29312747770848) bank192327 +192328 POINT(39.85263071823295 74.82430121539136) bank192328 +192329 POINT(41.00236424953645 74.63323631359492) bank192329 +192330 POINT(39.83618243804905 74.57304906989668) bank192330 +192331 POINT(41.00647042537953 73.00898639637838) bank192331 +192332 POINT(39.753245405517234 74.1485772157408) bank192332 +192333 POINT(41.57841358565658 74.55516871454344) bank192333 +192334 POINT(41.059472393208196 74.36456469285153) bank192334 +192335 POINT(41.142556848282155 74.741224360576) bank192335 +192336 POINT(40.80075200024339 73.98767345405602) bank192336 +192337 POINT(41.50521531823019 74.06622898175142) bank192337 +192338 POINT(40.47611354978568 74.3519853690747) bank192338 +192339 POINT(39.78795497096183 74.28853771120352) bank192339 +192340 POINT(40.695388173973285 73.77577484679952) bank192340 +192341 POINT(41.220167446547265 73.64364805072452) bank192341 +192342 POINT(40.92769213059399 74.18766192430236) bank192342 +192343 POINT(40.662965432966075 74.89623380080936) bank192343 +192344 POINT(41.114563020291165 73.67007904974031) bank192344 +192345 POINT(40.97307665821817 73.98081370834346) bank192345 +192346 POINT(41.23641042015708 73.52380719715956) bank192346 +192347 POINT(39.91691302467921 73.22194663675872) bank192347 +192348 POINT(41.20305022155236 73.76404091683646) bank192348 +192349 POINT(41.08814496385334 73.42651277665517) bank192349 +192350 POINT(39.84802013483721 73.44070618857414) bank192350 +192351 POINT(40.39740850282207 74.29655123292713) bank192351 +192352 POINT(40.28642204705208 74.89066517692207) bank192352 +192353 POINT(40.529601150753585 74.22478273842704) bank192353 +192354 POINT(41.40079144362465 74.7757272389409) bank192354 +192355 POINT(40.248576403085686 73.30118723312707) bank192355 +192356 POINT(40.02856095603555 73.51360126832142) bank192356 +192357 POINT(40.87822014619554 74.970520723095) bank192357 +192358 POINT(41.00707533391297 74.1270460560165) bank192358 +192359 POINT(40.893551681788175 74.29637548032754) bank192359 +192360 POINT(40.80576619874745 74.94617262839068) bank192360 +192361 POINT(41.18449986529745 73.66993177728025) bank192361 +192362 POINT(41.392981042428346 74.13859981140574) bank192362 +192363 POINT(40.901058328738685 74.28339259507831) bank192363 +192364 POINT(41.56690089338167 73.38461995656381) bank192364 +192365 POINT(40.03210539579078 74.16744324721253) bank192365 +192366 POINT(41.40218225771234 73.0498363373364) bank192366 +192367 POINT(39.758088319154496 74.12428953513833) bank192367 +192368 POINT(40.75535673575965 73.5268626640013) bank192368 +192369 POINT(40.769222304454175 73.82502944030234) bank192369 +192370 POINT(41.00004767926602 74.78773204564447) bank192370 +192371 POINT(41.23900887520564 73.83162176499933) bank192371 +192372 POINT(40.99920438624107 74.03955639824525) bank192372 +192373 POINT(40.698194920086706 73.66567202015243) bank192373 +192374 POINT(40.534163788433936 74.9825116799562) bank192374 +192375 POINT(39.958402808112055 74.08863990268607) bank192375 +192376 POINT(41.37265379387108 73.47096882117465) bank192376 +192377 POINT(40.43035594887453 74.3183909616843) bank192377 +192378 POINT(40.99904841400731 73.64425799241607) bank192378 +192379 POINT(40.24153374143459 74.16185481406905) bank192379 +192380 POINT(40.8928974843047 73.73040991203347) bank192380 +192381 POINT(41.4596724766812 73.65897221753569) bank192381 +192382 POINT(39.87325608794503 73.23133864828345) bank192382 +192383 POINT(39.715074277494146 74.06950674325616) bank192383 +192384 POINT(40.096543384073634 73.14757570477217) bank192384 +192385 POINT(40.24983799860707 74.0383021631606) bank192385 +192386 POINT(40.0025984861563 74.02154691167306) bank192386 +192387 POINT(41.54032528529402 74.70202626064943) bank192387 +192388 POINT(41.4192849072923 74.1954126183397) bank192388 +192389 POINT(41.342037694178565 74.99793409943754) bank192389 +192390 POINT(40.5105935762992 73.75450255116563) bank192390 +192391 POINT(40.335544986163455 74.55849748159197) bank192391 +192392 POINT(40.218718410608794 73.53063163311215) bank192392 +192393 POINT(41.16301990421345 73.515241323009) bank192393 +192394 POINT(41.45174749670594 73.69221747829009) bank192394 +192395 POINT(40.283991997174894 73.44213819452125) bank192395 +192396 POINT(41.18053112419588 74.84748665830118) bank192396 +192397 POINT(41.016826629040885 74.18007202650331) bank192397 +192398 POINT(41.52172722526399 74.24949480582016) bank192398 +192399 POINT(41.32300388983001 74.50505424105582) bank192399 +192400 POINT(40.98060387584474 73.50432309816281) bank192400 +192401 POINT(39.87692243557723 74.80399429357307) bank192401 +192402 POINT(40.308476990545834 74.77573526304796) bank192402 +192403 POINT(40.997729659636725 73.05292667665765) bank192403 +192404 POINT(41.61092370668109 73.69003118478936) bank192404 +192405 POINT(39.826272119865095 73.17517408423866) bank192405 +192406 POINT(41.32425083698569 74.37087004020705) bank192406 +192407 POINT(40.79987294856791 73.20821003082251) bank192407 +192408 POINT(41.50404962695706 74.67589630711268) bank192408 +192409 POINT(41.32550441623006 73.13199653550232) bank192409 +192410 POINT(39.909877391357966 73.01332330104061) bank192410 +192411 POINT(41.69512737952674 73.63599635295871) bank192411 +192412 POINT(41.02650479158958 74.96450339418267) bank192412 +192413 POINT(39.80860914101385 74.50562542860342) bank192413 +192414 POINT(40.75821232446731 73.7614120905794) bank192414 +192415 POINT(40.34883383716426 73.09672152684219) bank192415 +192416 POINT(39.803241649008655 73.98077838843061) bank192416 +192417 POINT(40.65898241513019 74.903602174465) bank192417 +192418 POINT(41.52809988282115 73.6344941485033) bank192418 +192419 POINT(39.950649264316915 74.93405607860315) bank192419 +192420 POINT(41.09821741486298 74.50441416900311) bank192420 +192421 POINT(40.23429787400412 73.18923979894782) bank192421 +192422 POINT(40.19061219381273 73.8871807618117) bank192422 +192423 POINT(40.868145786430645 73.32069947835897) bank192423 +192424 POINT(41.27211535259186 73.98296228877881) bank192424 +192425 POINT(41.35379658851247 73.63297147412963) bank192425 +192426 POINT(40.53162337362426 74.53324200859828) bank192426 +192427 POINT(41.63095760417544 73.58926224836499) bank192427 +192428 POINT(41.27743029979734 74.83419539154059) bank192428 +192429 POINT(40.539894116314194 74.48164616931015) bank192429 +192430 POINT(41.426213588925435 74.06748710660518) bank192430 +192431 POINT(41.493411185816214 73.07584805503498) bank192431 +192432 POINT(40.30994410649346 74.65876071289762) bank192432 +192433 POINT(41.39413242531867 74.60950158402578) bank192433 +192434 POINT(40.66853275121427 74.89601735462976) bank192434 +192435 POINT(41.381649647321176 74.91421255835321) bank192435 +192436 POINT(40.52492707629975 73.79075148302171) bank192436 +192437 POINT(40.73597061078105 74.48378717826024) bank192437 +192438 POINT(39.78387640900769 74.34465466775664) bank192438 +192439 POINT(40.02022408641759 74.51702377367717) bank192439 +192440 POINT(40.130508603025575 74.36470835454249) bank192440 +192441 POINT(41.05940580640353 74.75695242046382) bank192441 +192442 POINT(41.55269648280649 74.49489579449589) bank192442 +192443 POINT(41.53079793698585 73.64687617819803) bank192443 +192444 POINT(40.86214458680317 73.47164928222338) bank192444 +192445 POINT(40.65158777988766 73.32961696000606) bank192445 +192446 POINT(40.38597064330874 74.0054843568426) bank192446 +192447 POINT(40.01982235040743 74.65845556381977) bank192447 +192448 POINT(40.11301869259592 74.27066047330408) bank192448 +192449 POINT(40.534955344677584 73.09382112608097) bank192449 +192450 POINT(40.26591203916089 74.18674922789064) bank192450 +192451 POINT(41.24596836493765 73.8765260285627) bank192451 +192452 POINT(40.75970610333798 74.87800081371876) bank192452 +192453 POINT(40.04598771735716 73.83660301151264) bank192453 +192454 POINT(41.28049599565301 73.54550834665666) bank192454 +192455 POINT(40.25426908194769 73.57793033650283) bank192455 +192456 POINT(41.50252111403697 74.2571939227458) bank192456 +192457 POINT(41.14514749734304 74.78870372924925) bank192457 +192458 POINT(41.31161457954237 74.49720611674353) bank192458 +192459 POINT(39.85392196155745 73.21689976915708) bank192459 +192460 POINT(41.49632650966284 74.86906673146801) bank192460 +192461 POINT(40.56804338253718 73.52000770639069) bank192461 +192462 POINT(40.644871758269865 73.21154503617565) bank192462 +192463 POINT(40.9771418468786 74.15746586801919) bank192463 +192464 POINT(41.58077184623258 74.76166131547204) bank192464 +192465 POINT(40.391570626537884 74.8661748784829) bank192465 +192466 POINT(41.32750091671625 73.25665622615536) bank192466 +192467 POINT(41.57749540957907 73.71549723529279) bank192467 +192468 POINT(40.48779634624404 74.43562795133651) bank192468 +192469 POINT(40.64403127050735 73.48932992036993) bank192469 +192470 POINT(41.43933331181433 74.42157066782319) bank192470 +192471 POINT(40.68268318926722 74.73545891239023) bank192471 +192472 POINT(40.21046182064816 73.29914641911822) bank192472 +192473 POINT(40.503754697219804 73.85043276640918) bank192473 +192474 POINT(40.428463888463284 74.08130933581516) bank192474 +192475 POINT(41.20962101740768 73.32950748160496) bank192475 +192476 POINT(40.20430039593702 73.78160018512844) bank192476 +192477 POINT(40.15608355420123 74.53311992104183) bank192477 +192478 POINT(39.77250944012415 73.06784094027569) bank192478 +192479 POINT(40.22602197529718 73.78480694566099) bank192479 +192480 POINT(41.2633419015671 74.49027560667888) bank192480 +192481 POINT(41.5713687016304 73.23708560409113) bank192481 +192482 POINT(39.83340814815045 74.92271885638088) bank192482 +192483 POINT(40.55192791578681 73.02916950408901) bank192483 +192484 POINT(41.40280268434882 74.37514388131969) bank192484 +192485 POINT(41.023231789447316 73.3332184506962) bank192485 +192486 POINT(40.15061511642245 74.19400965404847) bank192486 +192487 POINT(41.21180906225768 73.44766184649065) bank192487 +192488 POINT(41.58596309980355 74.95249111595548) bank192488 +192489 POINT(41.04491430609136 74.82057967747967) bank192489 +192490 POINT(39.949342469223055 73.17853092975245) bank192490 +192491 POINT(39.99669791277564 74.66573335802349) bank192491 +192492 POINT(41.515870888063034 74.83706490405797) bank192492 +192493 POINT(40.66501929251631 73.4945280755776) bank192493 +192494 POINT(40.926202347097686 73.25886655136888) bank192494 +192495 POINT(40.58180404695749 74.16457038100579) bank192495 +192496 POINT(39.81852213546976 73.67164761122432) bank192496 +192497 POINT(41.15206767480055 73.55974937218977) bank192497 +192498 POINT(40.59476664068122 74.36621354511368) bank192498 +192499 POINT(40.456878879416 74.60808071928392) bank192499 +192500 POINT(41.109756670525925 74.24340581428247) bank192500 +192501 POINT(41.573008220435064 74.31370260215463) bank192501 +192502 POINT(40.225771386397355 73.63130450764476) bank192502 +192503 POINT(41.04481839135774 73.39279516397214) bank192503 +192504 POINT(41.1975579605484 74.37649313329104) bank192504 +192505 POINT(40.0457019329323 74.79099713476482) bank192505 +192506 POINT(41.13224957576483 74.37392891905716) bank192506 +192507 POINT(41.62394054261018 74.4607806356591) bank192507 +192508 POINT(40.78490300189184 73.97237576069053) bank192508 +192509 POINT(39.91283387139522 73.18606362928281) bank192509 +192510 POINT(41.62537085796273 73.52602914417177) bank192510 +192511 POINT(40.33192136675845 73.06216269950072) bank192511 +192512 POINT(40.94064594834218 74.94567636228675) bank192512 +192513 POINT(41.23126426409282 73.4101131860652) bank192513 +192514 POINT(41.464788137070244 74.37944648277154) bank192514 +192515 POINT(39.78063043295332 74.08349365867625) bank192515 +192516 POINT(40.58900855814579 73.16123494924221) bank192516 +192517 POINT(40.9694971304441 73.99228498276234) bank192517 +192518 POINT(40.35274994285414 74.65486489575501) bank192518 +192519 POINT(41.40426013870496 74.14371533737697) bank192519 +192520 POINT(40.511639992735006 74.13448839835928) bank192520 +192521 POINT(40.13479735788626 74.62169301101109) bank192521 +192522 POINT(40.7549415961444 74.49658902720424) bank192522 +192523 POINT(40.60760085655445 74.32315905654016) bank192523 +192524 POINT(41.160513949406266 73.66622420800007) bank192524 +192525 POINT(40.2882640115053 73.8432321188004) bank192525 +192526 POINT(41.337678693168684 73.65660585090696) bank192526 +192527 POINT(41.3295569086022 73.18219250883996) bank192527 +192528 POINT(39.84746324370874 73.05541911810421) bank192528 +192529 POINT(40.271296993896776 74.42009769885716) bank192529 +192530 POINT(40.75689695348673 74.3058704957768) bank192530 +192531 POINT(40.22870359621738 73.34567786349679) bank192531 +192532 POINT(40.863261874204674 74.08364980208248) bank192532 +192533 POINT(40.998502084852845 74.35675613706346) bank192533 +192534 POINT(41.55445393233487 73.16854577019782) bank192534 +192535 POINT(40.043380231781995 73.22227413451418) bank192535 +192536 POINT(40.65836960569888 74.01067635788584) bank192536 +192537 POINT(39.91033185377896 73.7967514540899) bank192537 +192538 POINT(40.46784326377659 73.04432818754917) bank192538 +192539 POINT(40.89107738457254 74.14546093396184) bank192539 +192540 POINT(40.89866459068211 73.09621317260158) bank192540 +192541 POINT(40.93518226047882 74.62074168529642) bank192541 +192542 POINT(40.78064268837916 74.65860581808955) bank192542 +192543 POINT(41.642811991834904 73.23502112484178) bank192543 +192544 POINT(40.618583609306796 73.4987548294885) bank192544 +192545 POINT(41.538505284768924 74.49280462100596) bank192545 +192546 POINT(41.59969738862653 73.64141815006788) bank192546 +192547 POINT(40.698037052219206 73.88689096443134) bank192547 +192548 POINT(40.696282375991316 74.01754772370809) bank192548 +192549 POINT(41.210638711840666 75.00116477627111) bank192549 +192550 POINT(40.68061973080833 73.41502797326547) bank192550 +192551 POINT(40.54157583030721 73.8741589689673) bank192551 +192552 POINT(40.83906947561459 74.34930674860853) bank192552 +192553 POINT(41.653923848075685 74.98981211152498) bank192553 +192554 POINT(41.27785218002288 74.38822096451868) bank192554 +192555 POINT(40.66444271604282 74.20102328979459) bank192555 +192556 POINT(41.00915799447816 73.87870264966719) bank192556 +192557 POINT(40.69616085627331 74.8326548254413) bank192557 +192558 POINT(41.599304777773135 73.90211308723383) bank192558 +192559 POINT(40.59737187617677 73.54852525648509) bank192559 +192560 POINT(41.69008893311925 74.38580912638533) bank192560 +192561 POINT(41.471376299205936 73.78802244181317) bank192561 +192562 POINT(39.79958820750426 74.33139165832917) bank192562 +192563 POINT(41.0476705707382 73.33734541067402) bank192563 +192564 POINT(40.203081862154896 73.26219290247761) bank192564 +192565 POINT(41.358920785993384 74.04571441971213) bank192565 +192566 POINT(40.158136328092866 73.3312704272871) bank192566 +192567 POINT(41.602385207184284 73.1929424190122) bank192567 +192568 POINT(39.83935554640579 73.95349640139787) bank192568 +192569 POINT(40.118865118062836 74.99825716948611) bank192569 +192570 POINT(40.86563996511658 74.03716326808228) bank192570 +192571 POINT(40.34881754658865 74.71767320231648) bank192571 +192572 POINT(41.250001259655264 73.67318941678786) bank192572 +192573 POINT(40.102163956580654 74.20728781558078) bank192573 +192574 POINT(39.88831733956909 74.58032052746375) bank192574 +192575 POINT(40.289616710546596 73.76966749691242) bank192575 +192576 POINT(41.469446062287574 73.51808679733398) bank192576 +192577 POINT(41.09251958605403 73.33635657180773) bank192577 +192578 POINT(40.76911332681031 74.79448664877636) bank192578 +192579 POINT(40.45411600049658 74.43622003327629) bank192579 +192580 POINT(40.81138653622836 73.94765835708489) bank192580 +192581 POINT(40.49837431595322 73.59084364937168) bank192581 +192582 POINT(40.101888132473725 73.68654008701591) bank192582 +192583 POINT(40.04287908284384 74.06263455790975) bank192583 +192584 POINT(41.62785479629095 74.18869141773665) bank192584 +192585 POINT(40.46830624236469 73.35932601524799) bank192585 +192586 POINT(40.41485775841834 74.31231670482356) bank192586 +192587 POINT(40.0943171512586 73.08474220839736) bank192587 +192588 POINT(40.36672143177593 74.2547337624994) bank192588 +192589 POINT(41.28780211471465 73.30280738708792) bank192589 +192590 POINT(41.66303211600927 74.29620005813638) bank192590 +192591 POINT(40.76644762056834 73.22370329873073) bank192591 +192592 POINT(41.58292430280783 74.60516347604789) bank192592 +192593 POINT(40.449411294011206 74.35072115611811) bank192593 +192594 POINT(40.92785313528987 74.5862814798422) bank192594 +192595 POINT(39.831527627755435 74.99880644760037) bank192595 +192596 POINT(39.94287466671762 74.12257808366297) bank192596 +192597 POINT(41.40900957118583 74.04762813743213) bank192597 +192598 POINT(39.73916336116761 73.70949795408853) bank192598 +192599 POINT(40.73737027521263 74.92055350806447) bank192599 +192600 POINT(40.25171568367207 73.27961530687392) bank192600 +192601 POINT(40.256358088703955 74.14699110690268) bank192601 +192602 POINT(39.843587556729666 73.04938107684016) bank192602 +192603 POINT(40.24430310947514 73.70912518078474) bank192603 +192604 POINT(40.54719896909182 73.95406909218187) bank192604 +192605 POINT(40.776446882190335 73.86904108981622) bank192605 +192606 POINT(41.49557464376667 73.04643498146842) bank192606 +192607 POINT(39.89806417436072 73.74172325423372) bank192607 +192608 POINT(40.98635112138869 73.42844626307819) bank192608 +192609 POINT(40.34248957867938 73.10168889270093) bank192609 +192610 POINT(40.80326768829973 74.09964454775073) bank192610 +192611 POINT(41.123520123936316 74.61884586702004) bank192611 +192612 POINT(41.48378657443557 73.51774607730937) bank192612 +192613 POINT(41.175425665735034 74.13474053352343) bank192613 +192614 POINT(39.88239053312029 74.07918913029935) bank192614 +192615 POINT(39.882166419113 73.04603407682777) bank192615 +192616 POINT(41.32812129276377 73.04463009970965) bank192616 +192617 POINT(39.889005343560555 73.42393417426774) bank192617 +192618 POINT(40.209842334445014 74.00044779283789) bank192618 +192619 POINT(40.548458376548425 73.44042815671092) bank192619 +192620 POINT(40.58364184971978 73.04084855249276) bank192620 +192621 POINT(40.16141833581379 73.63996122576577) bank192621 +192622 POINT(39.859653112501434 73.54748369673004) bank192622 +192623 POINT(40.87351658340503 74.98402029470624) bank192623 +192624 POINT(39.87577508879056 73.95547634393998) bank192624 +192625 POINT(39.97269242400568 74.92381653071115) bank192625 +192626 POINT(40.721226382253654 74.82301121093302) bank192626 +192627 POINT(41.22187644851028 73.22861886192162) bank192627 +192628 POINT(40.126392964002555 73.09028278609843) bank192628 +192629 POINT(41.10326699929136 74.10716660839267) bank192629 +192630 POINT(40.33533236988544 73.56641924586133) bank192630 +192631 POINT(40.2156818904576 74.03788993153302) bank192631 +192632 POINT(41.40109672694524 73.62780852911054) bank192632 +192633 POINT(40.90184296525363 73.89559493762351) bank192633 +192634 POINT(40.697885602332256 73.90746650755665) bank192634 +192635 POINT(41.270671276043664 73.35698682510946) bank192635 +192636 POINT(41.688643621888474 73.37182296309648) bank192636 +192637 POINT(41.592592351255874 74.85800775413594) bank192637 +192638 POINT(40.784917106938416 73.29151968071508) bank192638 +192639 POINT(40.43693670063268 73.40871832565331) bank192639 +192640 POINT(39.92182410298819 73.03487861379557) bank192640 +192641 POINT(40.53780225533092 73.67155358884106) bank192641 +192642 POINT(40.650738541774395 73.19403972079894) bank192642 +192643 POINT(40.05653411002127 74.30581826522841) bank192643 +192644 POINT(40.1948749884124 73.68065963922248) bank192644 +192645 POINT(40.21392154425638 74.96636306550596) bank192645 +192646 POINT(40.485222951579125 74.65640413693079) bank192646 +192647 POINT(40.2398696718174 73.74692806156548) bank192647 +192648 POINT(40.810880329945995 74.12068899119106) bank192648 +192649 POINT(41.09360885196704 74.93305852784633) bank192649 +192650 POINT(40.08800929801825 74.22032490830496) bank192650 +192651 POINT(41.66168088312538 73.45156427364923) bank192651 +192652 POINT(41.16910393887903 73.89137304678654) bank192652 +192653 POINT(41.550968798724654 74.49204128257504) bank192653 +192654 POINT(40.155652796005846 74.46821223409624) bank192654 +192655 POINT(39.7887101488033 73.87023701841277) bank192655 +192656 POINT(40.54148280560211 74.36186739812524) bank192656 +192657 POINT(40.17017311858629 73.85177950728965) bank192657 +192658 POINT(39.90768395115485 74.78370116612561) bank192658 +192659 POINT(41.400652491169986 74.81750708800955) bank192659 +192660 POINT(41.63506773799662 73.67470051743119) bank192660 +192661 POINT(40.97610525064397 74.19342642260816) bank192661 +192662 POINT(40.4806555762698 73.48309065348506) bank192662 +192663 POINT(41.339139612506344 74.79985021133916) bank192663 +192664 POINT(39.882916873853816 73.31444643479479) bank192664 +192665 POINT(39.883096393883136 73.40161270830178) bank192665 +192666 POINT(40.29101164649147 74.91334799156378) bank192666 +192667 POINT(41.098036740227634 74.32358138399685) bank192667 +192668 POINT(41.71178016318186 74.75889219385935) bank192668 +192669 POINT(40.955018904154535 74.02338522712515) bank192669 +192670 POINT(41.43812023698576 73.00727371436619) bank192670 +192671 POINT(41.05326536361514 73.83730999013451) bank192671 +192672 POINT(40.34980789117136 74.73333731315357) bank192672 +192673 POINT(39.92682103148123 74.1022328255467) bank192673 +192674 POINT(41.071367716311535 75.00155718985536) bank192674 +192675 POINT(40.152221400812095 73.81971038676619) bank192675 +192676 POINT(41.707757777575445 73.36073203209676) bank192676 +192677 POINT(41.37078728687007 73.47507774012516) bank192677 +192678 POINT(40.712620857851675 73.73650335211954) bank192678 +192679 POINT(41.60435845025519 73.7173355277972) bank192679 +192680 POINT(40.77206478662839 73.14836200721027) bank192680 +192681 POINT(40.57518436332122 73.51416693795956) bank192681 +192682 POINT(40.15706677776033 73.90135639692936) bank192682 +192683 POINT(40.08284961357762 73.63307767736663) bank192683 +192684 POINT(40.17375776364413 74.91784187450845) bank192684 +192685 POINT(40.35387903198561 74.66017563595267) bank192685 +192686 POINT(39.90729320812183 73.66396221929553) bank192686 +192687 POINT(40.39514329944502 73.63454296473822) bank192687 +192688 POINT(41.32578492266121 74.71649419974551) bank192688 +192689 POINT(41.68123601877837 73.04900003815412) bank192689 +192690 POINT(40.93798714977365 73.65150544616368) bank192690 +192691 POINT(41.09632013649074 74.88682941680445) bank192691 +192692 POINT(39.90808554641569 74.3264865634854) bank192692 +192693 POINT(39.79814849207692 73.11509936850061) bank192693 +192694 POINT(39.98191572773407 74.2325382465289) bank192694 +192695 POINT(40.63965755051669 74.06260122684972) bank192695 +192696 POINT(40.31963913838089 73.31044092884231) bank192696 +192697 POINT(39.814673302068336 73.07537942417505) bank192697 +192698 POINT(40.45081022949725 73.85615055565567) bank192698 +192699 POINT(41.02917567054879 73.05916448768612) bank192699 +192700 POINT(40.31472769195373 74.99352035172826) bank192700 +192701 POINT(40.19039274408917 73.82405387887128) bank192701 +192702 POINT(41.01791905392114 74.40190107941832) bank192702 +192703 POINT(40.063725055891645 74.38976144415658) bank192703 +192704 POINT(41.56388503291184 73.19006665831967) bank192704 +192705 POINT(41.239387101923484 74.14521482801732) bank192705 +192706 POINT(41.674517203989005 73.80380202823186) bank192706 +192707 POINT(41.48484914157684 74.95313279005902) bank192707 +192708 POINT(40.65788872197019 73.55456022164535) bank192708 +192709 POINT(41.4390690958206 73.07963854389824) bank192709 +192710 POINT(41.423759754466666 73.24127894849408) bank192710 +192711 POINT(40.60091327259555 73.93397325122663) bank192711 +192712 POINT(40.926479933558426 73.64315654750571) bank192712 +192713 POINT(40.045741100837716 74.1099761329346) bank192713 +192714 POINT(40.987932964921825 73.43191070097068) bank192714 +192715 POINT(41.338106743186685 73.32492729040199) bank192715 +192716 POINT(40.60836435929349 73.7912129988207) bank192716 +192717 POINT(40.82813845385784 73.48379325142113) bank192717 +192718 POINT(39.99868346733588 73.86842490244058) bank192718 +192719 POINT(40.183271919486366 74.95204680482588) bank192719 +192720 POINT(40.07801028747438 73.0397661415751) bank192720 +192721 POINT(41.07160176695836 74.55002175790307) bank192721 +192722 POINT(39.92405975684539 73.39974766064582) bank192722 +192723 POINT(41.483347553051416 74.15223140563134) bank192723 +192724 POINT(40.008558810656204 74.77051066259712) bank192724 +192725 POINT(40.29530867135474 73.64786950050227) bank192725 +192726 POINT(40.64984931432939 74.51615436783806) bank192726 +192727 POINT(41.64427076592052 74.7230075278438) bank192727 +192728 POINT(41.52865532554094 74.15010459006868) bank192728 +192729 POINT(40.88318879506534 73.25276682940805) bank192729 +192730 POINT(40.134195998586534 74.15458855392063) bank192730 +192731 POINT(40.62715180220488 73.29402638707121) bank192731 +192732 POINT(40.25616696734843 73.49965491439274) bank192732 +192733 POINT(41.14143889040454 73.63454934442186) bank192733 +192734 POINT(39.88475189271043 73.46732183812196) bank192734 +192735 POINT(40.952533073331395 73.35709394775611) bank192735 +192736 POINT(40.88929028726172 74.48872068063793) bank192736 +192737 POINT(40.534351713655475 74.6837388098131) bank192737 +192738 POINT(41.60605603009405 74.47966659614104) bank192738 +192739 POINT(39.724049973759485 74.34047107604786) bank192739 +192740 POINT(41.47316404160039 74.2532907735598) bank192740 +192741 POINT(41.16789357807808 74.66217935037542) bank192741 +192742 POINT(39.81970260621253 74.43202709557639) bank192742 +192743 POINT(41.15579261990305 74.44938427834369) bank192743 +192744 POINT(40.41398390423995 74.93721896390424) bank192744 +192745 POINT(41.62642198040652 74.37058778918727) bank192745 +192746 POINT(40.73965794167291 74.94529707324529) bank192746 +192747 POINT(40.16742774239597 74.32054586071636) bank192747 +192748 POINT(41.10628748802571 74.21083622068161) bank192748 +192749 POINT(41.68227812049168 73.81230055752022) bank192749 +192750 POINT(40.639323991872 73.75737471861615) bank192750 +192751 POINT(40.372581668583585 73.31538240495753) bank192751 +192752 POINT(39.714500143249396 74.03605497365325) bank192752 +192753 POINT(41.48587875210151 73.60555978526443) bank192753 +192754 POINT(41.03278230771326 74.51047387280715) bank192754 +192755 POINT(41.41444358104716 73.0834128842518) bank192755 +192756 POINT(39.94467825943212 74.95142288636995) bank192756 +192757 POINT(40.9318680701546 74.53078914287023) bank192757 +192758 POINT(40.39454582873949 74.0687236244354) bank192758 +192759 POINT(41.20436235555152 74.30528323461415) bank192759 +192760 POINT(39.83488037862878 74.57619606528928) bank192760 +192761 POINT(40.10975166277499 73.81240584180877) bank192761 +192762 POINT(40.26415847241933 74.23686504502734) bank192762 +192763 POINT(39.96103499771094 75.00098610869722) bank192763 +192764 POINT(41.34481445569496 74.14269594729666) bank192764 +192765 POINT(41.073659044562035 74.11056768166634) bank192765 +192766 POINT(41.504646616342995 74.13747405084348) bank192766 +192767 POINT(40.46601176910128 74.16526951819952) bank192767 +192768 POINT(41.48766024965494 74.5942072349711) bank192768 +192769 POINT(40.46936541771111 73.97238964022824) bank192769 +192770 POINT(41.35869658989525 73.27443863936776) bank192770 +192771 POINT(41.15153167337404 74.72290295736941) bank192771 +192772 POINT(40.99857751793959 74.78554755881201) bank192772 +192773 POINT(41.47059404591329 74.60671673945461) bank192773 +192774 POINT(41.25545122900093 74.2001235484559) bank192774 +192775 POINT(40.301732773818756 74.17182081162693) bank192775 +192776 POINT(40.83849020605369 73.53472328967038) bank192776 +192777 POINT(40.77065258729353 73.66952978742893) bank192777 +192778 POINT(40.59379205454903 74.63684066133997) bank192778 +192779 POINT(41.434310658462636 74.12606881139615) bank192779 +192780 POINT(39.75054277271222 74.47601707429305) bank192780 +192781 POINT(40.853484374426415 73.85357085105227) bank192781 +192782 POINT(40.54871334001703 74.82011994858362) bank192782 +192783 POINT(41.35696001173147 74.7633706210655) bank192783 +192784 POINT(41.262587232939346 74.57758705369794) bank192784 +192785 POINT(39.89841685513636 73.63883671800998) bank192785 +192786 POINT(39.9197223109056 74.80312798835021) bank192786 +192787 POINT(40.130587282169444 74.03450361925672) bank192787 +192788 POINT(40.323096869749605 74.57282344020719) bank192788 +192789 POINT(41.08703658835374 74.85931615500076) bank192789 +192790 POINT(41.01245088021352 73.11544553353136) bank192790 +192791 POINT(40.52981884465962 73.48329371435267) bank192791 +192792 POINT(39.89195647320959 73.6180640147897) bank192792 +192793 POINT(41.142041788207194 73.82092020046997) bank192793 +192794 POINT(41.21798903742017 74.42623620526219) bank192794 +192795 POINT(41.627211703709456 74.04125088814097) bank192795 +192796 POINT(40.26168518074276 73.79863373832981) bank192796 +192797 POINT(40.039194739960735 73.22441096826945) bank192797 +192798 POINT(40.57825319281186 73.01441655397826) bank192798 +192799 POINT(40.105391762160494 74.04161143913817) bank192799 +192800 POINT(40.27594774425131 74.2080655338206) bank192800 +192801 POINT(40.97499039982104 74.48936976762326) bank192801 +192802 POINT(39.747264128994026 73.56120594419058) bank192802 +192803 POINT(41.26084848356378 73.07947155918546) bank192803 +192804 POINT(40.87198807297537 74.08116880651106) bank192804 +192805 POINT(41.706696624237864 74.5451552411322) bank192805 +192806 POINT(40.457555956850676 74.71264623763346) bank192806 +192807 POINT(41.234470712503516 74.05651701049283) bank192807 +192808 POINT(39.73943936399209 73.16416880178646) bank192808 +192809 POINT(41.64987734900355 74.57010326493085) bank192809 +192810 POINT(41.53177769776168 73.3429468779214) bank192810 +192811 POINT(40.07067373586888 74.97378789617566) bank192811 +192812 POINT(40.45088876819283 74.87434713244733) bank192812 +192813 POINT(41.26840408042983 74.84589199974778) bank192813 +192814 POINT(39.749706314180735 73.72333476470799) bank192814 +192815 POINT(41.084225683068375 74.8975659374219) bank192815 +192816 POINT(41.38135591403406 74.84571695727942) bank192816 +192817 POINT(40.06126763290888 73.35189564682406) bank192817 +192818 POINT(40.13927341706689 73.60000247665886) bank192818 +192819 POINT(41.080774617586904 73.85031511255507) bank192819 +192820 POINT(40.31416940012272 73.24860153268709) bank192820 +192821 POINT(40.13108450927177 73.95469650040506) bank192821 +192822 POINT(41.70543973779307 74.13112889417997) bank192822 +192823 POINT(41.47439259630823 73.90310500320714) bank192823 +192824 POINT(40.92315590441892 73.18799112189316) bank192824 +192825 POINT(39.718200003454086 74.18715242186025) bank192825 +192826 POINT(41.172957363208155 74.79037149234253) bank192826 +192827 POINT(39.871625576997175 73.62704198609784) bank192827 +192828 POINT(41.42620162291015 74.76060378740647) bank192828 +192829 POINT(40.51510648963729 73.49677464995004) bank192829 +192830 POINT(41.64469170839886 74.05963509438172) bank192830 +192831 POINT(40.97796946689877 74.3937553177851) bank192831 +192832 POINT(40.86114600923053 74.9118934504799) bank192832 +192833 POINT(40.17140050092113 73.0729635307396) bank192833 +192834 POINT(40.556431374528216 73.05012628942097) bank192834 +192835 POINT(40.00059161755964 73.50554765937954) bank192835 +192836 POINT(40.01778572926777 73.42690087242607) bank192836 +192837 POINT(39.96689271991995 73.89127571904449) bank192837 +192838 POINT(39.95794867768247 73.40691117249405) bank192838 +192839 POINT(41.26283339117004 74.14431940496287) bank192839 +192840 POINT(39.84678852984494 73.46607926477795) bank192840 +192841 POINT(40.07204701629844 74.69471438236587) bank192841 +192842 POINT(41.09096650781653 73.23704519272633) bank192842 +192843 POINT(41.403644397095064 74.15399390840778) bank192843 +192844 POINT(39.81624826172009 73.20164399093515) bank192844 +192845 POINT(40.62904871153939 73.69012101534318) bank192845 +192846 POINT(41.208791441840354 73.30681190998628) bank192846 +192847 POINT(40.02566470828741 74.49762282351203) bank192847 +192848 POINT(40.14541055790321 74.3800995861743) bank192848 +192849 POINT(39.71572714814316 73.22641634934264) bank192849 +192850 POINT(40.70401313130565 73.621453828663) bank192850 +192851 POINT(40.36041921428703 73.20248864762142) bank192851 +192852 POINT(40.6277543554429 73.96808884529825) bank192852 +192853 POINT(41.07557081879902 73.13473044317195) bank192853 +192854 POINT(41.444552135925456 74.58494522228682) bank192854 +192855 POINT(40.46022727868337 73.13886928893501) bank192855 +192856 POINT(41.01132194425361 74.03683038527498) bank192856 +192857 POINT(41.696701059515384 74.89805849491938) bank192857 +192858 POINT(41.57455470089814 74.46168901371242) bank192858 +192859 POINT(40.65673696729693 74.29644067788183) bank192859 +192860 POINT(39.93418581755505 73.23570222120357) bank192860 +192861 POINT(39.99146448953364 74.6070871751546) bank192861 +192862 POINT(39.98618230359198 73.87841017175667) bank192862 +192863 POINT(40.6852467566671 73.90727511308934) bank192863 +192864 POINT(40.7508599750916 73.79223358701374) bank192864 +192865 POINT(41.69911584327783 73.77435682515565) bank192865 +192866 POINT(40.00285791728807 73.55895934765202) bank192866 +192867 POINT(40.683084199208885 74.06529205070325) bank192867 +192868 POINT(40.18131385233792 74.92534430562735) bank192868 +192869 POINT(41.329933386091035 73.38337317111915) bank192869 +192870 POINT(40.17531274190185 73.7118461639485) bank192870 +192871 POINT(41.04436633573666 73.654370554803) bank192871 +192872 POINT(40.49520427467351 73.87020316595616) bank192872 +192873 POINT(40.83201644182699 73.4041162829418) bank192873 +192874 POINT(40.69236135597956 74.40108054415454) bank192874 +192875 POINT(40.80427976041396 73.23344050664795) bank192875 +192876 POINT(39.94991235677572 74.92956391936637) bank192876 +192877 POINT(40.225758343575734 74.85003057344724) bank192877 +192878 POINT(41.052476948834226 74.46234054265408) bank192878 +192879 POINT(40.5972360456852 73.49931410595775) bank192879 +192880 POINT(40.823651201723386 74.46506730347066) bank192880 +192881 POINT(40.031245823308296 74.38133767830979) bank192881 +192882 POINT(41.51700984036505 73.29007942559247) bank192882 +192883 POINT(41.70672942991412 74.02961737136238) bank192883 +192884 POINT(41.619769121604925 74.9776082799742) bank192884 +192885 POINT(40.15721609553963 74.8293130771339) bank192885 +192886 POINT(40.665636232146085 73.53706863607518) bank192886 +192887 POINT(41.4322491126827 73.23113659075277) bank192887 +192888 POINT(40.38905933314416 73.3301965492621) bank192888 +192889 POINT(41.24117526992328 74.92075010062113) bank192889 +192890 POINT(41.2289145696536 74.96594542556639) bank192890 +192891 POINT(40.894195273217605 73.57194895415833) bank192891 +192892 POINT(39.787768835840595 73.47775299813945) bank192892 +192893 POINT(41.60068405650026 74.31593891435672) bank192893 +192894 POINT(41.19525815072942 74.31720795730594) bank192894 +192895 POINT(41.10934506594658 74.1744847683537) bank192895 +192896 POINT(40.82761689941733 74.7905693835803) bank192896 +192897 POINT(41.51300080815994 74.54098043664304) bank192897 +192898 POINT(39.817909107593415 74.86086087316697) bank192898 +192899 POINT(41.20882259226465 74.3240593351084) bank192899 +192900 POINT(40.0407301196972 74.88934211882065) bank192900 +192901 POINT(40.486920169217385 73.07649231010942) bank192901 +192902 POINT(40.4340703695409 74.11867217584363) bank192902 +192903 POINT(40.09483327492326 73.3672472398338) bank192903 +192904 POINT(41.433693080808645 73.75801550549542) bank192904 +192905 POINT(39.94564918233735 73.27488744828884) bank192905 +192906 POINT(41.052638719724555 73.18436127541499) bank192906 +192907 POINT(40.790708955263206 73.95615658961991) bank192907 +192908 POINT(41.582826653517124 74.42299747268031) bank192908 +192909 POINT(40.198936516216264 73.93315941467287) bank192909 +192910 POINT(41.0360622548339 73.46639792675003) bank192910 +192911 POINT(40.19163267826036 73.64653871538243) bank192911 +192912 POINT(41.56029177570059 74.97951329978603) bank192912 +192913 POINT(41.318834976766226 73.83088526080084) bank192913 +192914 POINT(41.17137868841815 74.5916463329021) bank192914 +192915 POINT(41.474703910584665 73.20169037642206) bank192915 +192916 POINT(41.3268774378918 74.23696604538766) bank192916 +192917 POINT(40.0467488738292 74.60772169782972) bank192917 +192918 POINT(40.934975863447455 73.35041415169309) bank192918 +192919 POINT(40.03115766430794 74.99920951373933) bank192919 +192920 POINT(40.26127413416646 74.04830189692126) bank192920 +192921 POINT(41.63133021254146 74.20192547165973) bank192921 +192922 POINT(39.97014800042997 73.3176093950239) bank192922 +192923 POINT(41.453888008403965 73.0270293417656) bank192923 +192924 POINT(40.84626255271919 74.56800880564437) bank192924 +192925 POINT(39.77041699218741 73.17700372360208) bank192925 +192926 POINT(40.802678845635405 74.08049813166626) bank192926 +192927 POINT(40.6400627446856 73.40633605607707) bank192927 +192928 POINT(41.14972684595982 74.3448980171151) bank192928 +192929 POINT(40.14394826844048 75.00368826611972) bank192929 +192930 POINT(39.813705844031595 74.53480990336) bank192930 +192931 POINT(40.359399518548585 73.73244271770952) bank192931 +192932 POINT(41.54602290189296 74.95631847624958) bank192932 +192933 POINT(40.641640093760785 73.19281070416268) bank192933 +192934 POINT(40.11289498921061 73.36201316941688) bank192934 +192935 POINT(40.62999158354498 73.1196911252237) bank192935 +192936 POINT(41.0645191756633 74.98444799994633) bank192936 +192937 POINT(40.450959865248926 73.50672597324427) bank192937 +192938 POINT(40.93192986315505 74.70260289321544) bank192938 +192939 POINT(40.976195784556005 73.97657734639714) bank192939 +192940 POINT(41.398475546906255 74.01622920197337) bank192940 +192941 POINT(40.89967269359169 74.25071053748485) bank192941 +192942 POINT(40.725064999638796 73.90628173955066) bank192942 +192943 POINT(41.40404518526344 74.6022505400498) bank192943 +192944 POINT(41.669312577134846 74.75449546726693) bank192944 +192945 POINT(40.683602578481334 73.512162566844) bank192945 +192946 POINT(40.170386145252586 73.62901435431591) bank192946 +192947 POINT(41.438513237005864 73.51462202156317) bank192947 +192948 POINT(39.99358082823482 74.2139963722861) bank192948 +192949 POINT(40.07511511153182 73.27306014580968) bank192949 +192950 POINT(40.47503697912982 73.65189433492975) bank192950 +192951 POINT(41.005007759553536 73.31219339933457) bank192951 +192952 POINT(40.184526099671615 73.07091625853103) bank192952 +192953 POINT(40.51159427338892 74.52634656765191) bank192953 +192954 POINT(39.86773338960149 73.78607666013787) bank192954 +192955 POINT(39.94494501387333 73.85773123142218) bank192955 +192956 POINT(39.881078410356736 73.30648477755621) bank192956 +192957 POINT(41.111677640061124 74.45462424241242) bank192957 +192958 POINT(40.94167278589828 74.16534866062035) bank192958 +192959 POINT(41.02009816961295 73.05498733832349) bank192959 +192960 POINT(40.7237464926546 73.15174072461373) bank192960 +192961 POINT(40.38589273637953 74.82792592953788) bank192961 +192962 POINT(40.22835393526945 74.94832090032703) bank192962 +192963 POINT(39.79019589337346 74.305214471034) bank192963 +192964 POINT(40.73634932138301 73.23573534225471) bank192964 +192965 POINT(40.75104624752661 74.77490913927505) bank192965 +192966 POINT(39.94670500896681 74.54264618194338) bank192966 +192967 POINT(40.662840359791495 73.23148412437945) bank192967 +192968 POINT(40.41618511445675 73.63805385730004) bank192968 +192969 POINT(40.57748175863477 74.95918178968854) bank192969 +192970 POINT(40.03395227754548 73.68389437541579) bank192970 +192971 POINT(40.42682121915561 73.32626354906571) bank192971 +192972 POINT(41.126183460979654 74.00417228037496) bank192972 +192973 POINT(40.7016595297887 74.38193735731164) bank192973 +192974 POINT(40.21574196061352 73.35063477422943) bank192974 +192975 POINT(41.5151093764962 74.36388330483378) bank192975 +192976 POINT(40.092896121685634 74.602217695452) bank192976 +192977 POINT(41.21601493522687 74.81257435022272) bank192977 +192978 POINT(39.83037408877191 73.99047989896532) bank192978 +192979 POINT(40.84004521843393 73.5015870152569) bank192979 +192980 POINT(39.79614817768654 73.41054462006615) bank192980 +192981 POINT(39.724190073718674 74.18199279031417) bank192981 +192982 POINT(39.88865985238543 73.76823319961194) bank192982 +192983 POINT(40.56250197210689 73.87983476688112) bank192983 +192984 POINT(41.26887134168051 74.9600794270166) bank192984 +192985 POINT(40.003441917354266 73.58319383218493) bank192985 +192986 POINT(40.08199076635509 74.13124572228496) bank192986 +192987 POINT(41.53166230655339 73.40561713506696) bank192987 +192988 POINT(41.357349897681495 73.60252766939783) bank192988 +192989 POINT(39.73591598063466 73.15039357572864) bank192989 +192990 POINT(39.995567017929034 73.98491120803713) bank192990 +192991 POINT(39.95706469190015 74.21365663296638) bank192991 +192992 POINT(41.099076008587446 74.41748928895159) bank192992 +192993 POINT(39.79914532530083 73.8568384265097) bank192993 +192994 POINT(40.20779846668535 73.89099190830966) bank192994 +192995 POINT(40.50408740503162 74.58296438632226) bank192995 +192996 POINT(39.99816892037481 74.74502069918934) bank192996 +192997 POINT(40.18113254492165 73.97302208482259) bank192997 +192998 POINT(41.35181394493887 73.54017064362948) bank192998 +192999 POINT(39.74460378583894 74.48047186304775) bank192999 +193000 POINT(41.60427796548053 73.49661279884398) bank193000 +193001 POINT(41.40614272255723 73.45490014010025) bank193001 +193002 POINT(41.640567727305 74.43372209424437) bank193002 +193003 POINT(41.03016064351148 73.28011455961072) bank193003 +193004 POINT(40.50051638927813 73.04766863892411) bank193004 +193005 POINT(40.63469491574946 73.56425862628763) bank193005 +193006 POINT(40.44633227144183 73.60402525194752) bank193006 +193007 POINT(41.5901571486556 74.48092224460477) bank193007 +193008 POINT(41.18128590988643 73.09947033243927) bank193008 +193009 POINT(40.6946502257821 73.9796292000366) bank193009 +193010 POINT(40.908544950884554 74.48402242993355) bank193010 +193011 POINT(40.460932713601196 73.42595519199169) bank193011 +193012 POINT(41.64151099364568 73.8922965098198) bank193012 +193013 POINT(41.42903249023097 74.52915236617275) bank193013 +193014 POINT(40.66075462119862 73.61407921449364) bank193014 +193015 POINT(40.68903103028093 74.50981313142294) bank193015 +193016 POINT(40.53728963926945 74.83003134898424) bank193016 +193017 POINT(41.53654014667087 73.91629048807829) bank193017 +193018 POINT(41.02287824002654 73.74286172687405) bank193018 +193019 POINT(40.64901994747782 73.82667443439816) bank193019 +193020 POINT(40.34356923850354 73.70904976215839) bank193020 +193021 POINT(41.04373109865416 73.10525181769913) bank193021 +193022 POINT(41.625276004432855 74.44374403516812) bank193022 +193023 POINT(41.18300716482287 74.65714648425191) bank193023 +193024 POINT(40.52656966663597 74.77891648618525) bank193024 +193025 POINT(41.25252857059094 74.10763053983588) bank193025 +193026 POINT(39.99777091680322 73.85437915214165) bank193026 +193027 POINT(40.391196569413545 73.3916477998317) bank193027 +193028 POINT(41.49369419137594 73.94285875891816) bank193028 +193029 POINT(41.488658222353465 74.59042486343522) bank193029 +193030 POINT(39.905323541562154 73.0436151972891) bank193030 +193031 POINT(39.89780051186916 73.70929124602328) bank193031 +193032 POINT(40.476914410230755 73.76463365350376) bank193032 +193033 POINT(41.381502683247675 73.10311632049103) bank193033 +193034 POINT(41.45416176478535 74.00591028948227) bank193034 +193035 POINT(40.2992980118725 73.37076211115802) bank193035 +193036 POINT(41.24653361138085 74.68994473218336) bank193036 +193037 POINT(41.317509430901175 73.53958897291216) bank193037 +193038 POINT(40.961734132595666 74.41844580129015) bank193038 +193039 POINT(41.36425765909397 74.52112228275503) bank193039 +193040 POINT(41.18845227031957 73.86385715735875) bank193040 +193041 POINT(40.478263666587566 73.85104008361955) bank193041 +193042 POINT(40.24088539645447 74.9326487008157) bank193042 +193043 POINT(39.775157774602334 74.72809383099397) bank193043 +193044 POINT(41.588198305384466 74.64056336142991) bank193044 +193045 POINT(41.510161831164325 74.58229871904754) bank193045 +193046 POINT(41.04758196322502 74.26598352122078) bank193046 +193047 POINT(40.646370948417555 73.5194288097842) bank193047 +193048 POINT(41.63910012010726 73.84105890043446) bank193048 +193049 POINT(41.13357734226769 73.97055544408865) bank193049 +193050 POINT(41.6533995219858 73.69400123302104) bank193050 +193051 POINT(40.764063614342064 74.23840143149795) bank193051 +193052 POINT(41.69566552637377 73.98412301628208) bank193052 +193053 POINT(40.17035286269691 73.01231681590211) bank193053 +193054 POINT(41.66702867846219 73.14563905715585) bank193054 +193055 POINT(40.83616300968536 73.45080167430207) bank193055 +193056 POINT(40.193901506291176 74.08061006748912) bank193056 +193057 POINT(41.701203355772684 73.17264661515972) bank193057 +193058 POINT(41.166477933242724 74.71871251094903) bank193058 +193059 POINT(39.925711886591145 73.99158278223135) bank193059 +193060 POINT(41.63214512655568 74.52168629224688) bank193060 +193061 POINT(40.406926664120284 74.02185666768456) bank193061 +193062 POINT(41.460203892891506 73.18424232722448) bank193062 +193063 POINT(40.10610242092111 73.64642948843961) bank193063 +193064 POINT(40.7074388592727 74.03495373167186) bank193064 +193065 POINT(40.639162040361455 73.66070325985639) bank193065 +193066 POINT(41.261412262700844 73.24279055517252) bank193066 +193067 POINT(40.144279364974274 74.65027810331352) bank193067 +193068 POINT(40.873012785620666 74.92516786829748) bank193068 +193069 POINT(41.04985577625816 73.70879571329776) bank193069 +193070 POINT(40.52501228122125 73.36837107834562) bank193070 +193071 POINT(41.288946474815454 74.6922546655292) bank193071 +193072 POINT(41.35565495062998 73.55622816020197) bank193072 +193073 POINT(40.251669966344345 73.88689738473468) bank193073 +193074 POINT(41.00789392374293 73.14115917035238) bank193074 +193075 POINT(41.59739702374216 73.01829865583257) bank193075 +193076 POINT(39.920327254718714 73.55882284432039) bank193076 +193077 POINT(39.81994105412349 74.40826476543899) bank193077 +193078 POINT(41.48303451052611 73.38781739375348) bank193078 +193079 POINT(41.14350187872799 73.39347825427998) bank193079 +193080 POINT(40.584210477996 73.1922937374438) bank193080 +193081 POINT(40.04311826329949 74.36479014927114) bank193081 +193082 POINT(41.53419047520103 74.51705692836026) bank193082 +193083 POINT(39.97353881387069 74.0427782391178) bank193083 +193084 POINT(40.63916650500495 73.66038225219835) bank193084 +193085 POINT(40.15881951623597 74.42215598646064) bank193085 +193086 POINT(39.98621037386389 73.68712112779181) bank193086 +193087 POINT(41.593624837772644 74.38092104320575) bank193087 +193088 POINT(39.71877335766744 73.09896576840387) bank193088 +193089 POINT(41.589415433471096 73.98464485576429) bank193089 +193090 POINT(41.66981656508984 74.35937951463765) bank193090 +193091 POINT(41.56347858342195 73.51835434381296) bank193091 +193092 POINT(40.728750712737124 74.57987382605265) bank193092 +193093 POINT(40.240233758633565 74.17797559246334) bank193093 +193094 POINT(39.949223215447674 74.29586825155216) bank193094 +193095 POINT(40.80384357418674 74.86400580345475) bank193095 +193096 POINT(41.46751708548114 74.02826046544554) bank193096 +193097 POINT(41.22181321815228 74.23629343704059) bank193097 +193098 POINT(41.25747023936319 73.16654556308833) bank193098 +193099 POINT(41.35708760790129 73.65115524831285) bank193099 +193100 POINT(41.170601024915165 74.61095137999236) bank193100 +193101 POINT(40.36685835459506 73.24840259801513) bank193101 +193102 POINT(40.295398249243455 73.20912471407105) bank193102 +193103 POINT(40.68633545090521 74.24260033059699) bank193103 +193104 POINT(40.32064099025268 73.50305086629983) bank193104 +193105 POINT(40.038217279623865 73.80342665880646) bank193105 +193106 POINT(41.66125251373767 73.43123973306166) bank193106 +193107 POINT(40.83221484255602 73.27177490968798) bank193107 +193108 POINT(39.79434153600399 74.6274223332273) bank193108 +193109 POINT(41.63299837196749 74.19783497497211) bank193109 +193110 POINT(41.4430267822232 73.93059929786732) bank193110 +193111 POINT(39.8473224284243 74.52691845966957) bank193111 +193112 POINT(41.121959145867386 73.81591795169544) bank193112 +193113 POINT(40.39694319489888 74.89345718499209) bank193113 +193114 POINT(41.34071883482867 73.18577459800346) bank193114 +193115 POINT(39.99056829856485 73.63530789555858) bank193115 +193116 POINT(40.9763672664947 74.05348777187642) bank193116 +193117 POINT(40.70116303258728 73.56540127862634) bank193117 +193118 POINT(40.83417221157021 73.30710859837092) bank193118 +193119 POINT(40.478365287179635 74.47884354744139) bank193119 +193120 POINT(40.08215210685154 73.52939321698139) bank193120 +193121 POINT(41.3552904528448 74.40536894417322) bank193121 +193122 POINT(40.23479737693506 74.37796666254334) bank193122 +193123 POINT(41.0133942393169 73.28021975799818) bank193123 +193124 POINT(40.56126591797238 74.955277542362) bank193124 +193125 POINT(41.045047742378685 73.05403904352396) bank193125 +193126 POINT(40.720215084093226 73.35633683881534) bank193126 +193127 POINT(41.37427794797429 74.74971002092566) bank193127 +193128 POINT(39.71668001273136 73.72096585601581) bank193128 +193129 POINT(41.451703891284495 74.71107020213114) bank193129 +193130 POINT(41.566303367353584 74.79268714765902) bank193130 +193131 POINT(40.72141028334884 74.17387066160022) bank193131 +193132 POINT(39.956501533919244 73.86391917410619) bank193132 +193133 POINT(41.1665673541676 75.00370671694021) bank193133 +193134 POINT(41.070013428065266 73.37740608908133) bank193134 +193135 POINT(41.23089426433854 74.70128031256574) bank193135 +193136 POINT(40.107248915572185 73.94132314361705) bank193136 +193137 POINT(41.22612991330028 74.6380477281282) bank193137 +193138 POINT(41.13265813969178 73.70056870913392) bank193138 +193139 POINT(40.08937484558315 73.93367521282144) bank193139 +193140 POINT(41.40949093186455 74.72545802469527) bank193140 +193141 POINT(41.27610495429636 73.3427355486333) bank193141 +193142 POINT(40.68310909883774 74.27717182251659) bank193142 +193143 POINT(41.61073681050133 74.23493468589868) bank193143 +193144 POINT(40.42534781806111 74.47013271163992) bank193144 +193145 POINT(40.982082888178304 74.74724067288038) bank193145 +193146 POINT(40.540943714744586 73.97630343939103) bank193146 +193147 POINT(39.870065080262414 73.0550558334778) bank193147 +193148 POINT(40.2264357477685 73.15803271924875) bank193148 +193149 POINT(40.42588889777625 73.42949333400905) bank193149 +193150 POINT(40.130332952993015 73.47019231167086) bank193150 +193151 POINT(40.68895668989797 74.25199273324327) bank193151 +193152 POINT(40.36496440240337 74.8141761955158) bank193152 +193153 POINT(41.34891429612719 73.00998932959715) bank193153 +193154 POINT(40.2271636054959 73.56646471024456) bank193154 +193155 POINT(40.8904251823349 74.63392380957276) bank193155 +193156 POINT(40.28163115279313 74.03553653464346) bank193156 +193157 POINT(40.28810113840297 74.70932174412609) bank193157 +193158 POINT(40.722918929843146 73.40690736415691) bank193158 +193159 POINT(39.81876781119003 74.24836428221191) bank193159 +193160 POINT(41.44067180160447 73.76370248234609) bank193160 +193161 POINT(41.22820633394016 74.18220236708653) bank193161 +193162 POINT(40.29887114035839 73.15124480270214) bank193162 +193163 POINT(39.86264524136411 73.956170113587) bank193163 +193164 POINT(41.004630592051114 74.48485464715768) bank193164 +193165 POINT(40.18718777944242 74.57556841062977) bank193165 +193166 POINT(41.68217329817512 73.66424874946613) bank193166 +193167 POINT(41.04363883319187 74.00490622787113) bank193167 +193168 POINT(40.48314561473856 74.74599876876265) bank193168 +193169 POINT(39.78490208089006 74.81722566398386) bank193169 +193170 POINT(41.07184047362134 73.51198232793779) bank193170 +193171 POINT(41.711480188101774 74.84660724771764) bank193171 +193172 POINT(41.56231360321078 74.75367500850972) bank193172 +193173 POINT(41.29688972222667 73.22492498351538) bank193173 +193174 POINT(40.2672427832241 74.38251965205879) bank193174 +193175 POINT(41.30389241442585 74.11851727930271) bank193175 +193176 POINT(41.67247209468536 73.09086535789972) bank193176 +193177 POINT(41.47367395333524 73.1145583367058) bank193177 +193178 POINT(40.18861964920027 74.20397326334881) bank193178 +193179 POINT(41.090195656272215 74.34740481035227) bank193179 +193180 POINT(41.58797080326045 73.20835930488988) bank193180 +193181 POINT(40.67942698613124 73.2486628061287) bank193181 +193182 POINT(40.6182385292035 73.65556287823595) bank193182 +193183 POINT(40.54552042795345 74.42931957291341) bank193183 +193184 POINT(39.91485675361995 74.30883355444439) bank193184 +193185 POINT(41.23666911267614 74.84886877929597) bank193185 +193186 POINT(41.23967017319382 74.86744643998456) bank193186 +193187 POINT(41.70964832119404 73.43029338953426) bank193187 +193188 POINT(41.62650616837427 74.5133943916516) bank193188 +193189 POINT(41.529059028175695 73.87837357677935) bank193189 +193190 POINT(41.54125374534044 73.77423503523754) bank193190 +193191 POINT(40.694012805785576 73.62847941974574) bank193191 +193192 POINT(41.1589734362689 74.54364495488136) bank193192 +193193 POINT(41.33575624648811 73.49095116034225) bank193193 +193194 POINT(39.76795626151734 73.84963903638896) bank193194 +193195 POINT(41.654410663518405 74.69285024334235) bank193195 +193196 POINT(40.99285646886705 73.19722506942024) bank193196 +193197 POINT(40.74531377331791 73.74576461161209) bank193197 +193198 POINT(40.1861441710447 73.71470407841241) bank193198 +193199 POINT(40.30447122763447 74.39529772554151) bank193199 +193200 POINT(41.048236533442946 74.69384846282966) bank193200 +193201 POINT(41.59711663459249 74.07016668595146) bank193201 +193202 POINT(40.70662321779127 74.81454870797488) bank193202 +193203 POINT(41.40529865703581 73.46688524914889) bank193203 +193204 POINT(40.288038198083235 74.87694430493212) bank193204 +193205 POINT(41.206533993381306 74.54201466370941) bank193205 +193206 POINT(40.055608728475356 74.04396293351961) bank193206 +193207 POINT(40.782781319204425 73.11939173437116) bank193207 +193208 POINT(40.226643784900396 73.5045902582935) bank193208 +193209 POINT(39.81204561515359 74.82765603052019) bank193209 +193210 POINT(40.615191948583245 74.06219507748104) bank193210 +193211 POINT(41.293908399052434 74.175415603274) bank193211 +193212 POINT(41.15939095509525 73.10903133680846) bank193212 +193213 POINT(40.424847437542745 73.45510962469423) bank193213 +193214 POINT(41.071429954222154 74.81946339755599) bank193214 +193215 POINT(41.172292740086995 73.64214009560291) bank193215 +193216 POINT(40.993786117048785 74.74105081092713) bank193216 +193217 POINT(41.06978820522367 73.68472766467457) bank193217 +193218 POINT(40.498083184273675 73.72692247216342) bank193218 +193219 POINT(41.05636919841659 74.7379912428065) bank193219 +193220 POINT(40.163416653526546 74.55696482728332) bank193220 +193221 POINT(40.71341069298268 74.28365556971639) bank193221 +193222 POINT(41.32215352064376 74.79658441914476) bank193222 +193223 POINT(40.60802468004281 74.82839398838972) bank193223 +193224 POINT(41.49722351986841 74.43927501720316) bank193224 +193225 POINT(40.79519241939379 74.78377664560139) bank193225 +193226 POINT(39.97167110476978 73.36822339260584) bank193226 +193227 POINT(41.57710912712959 73.4701833289676) bank193227 +193228 POINT(41.125226706471814 74.3135302293345) bank193228 +193229 POINT(40.19319699143025 73.23480258390808) bank193229 +193230 POINT(40.259042218756655 74.67624870325135) bank193230 +193231 POINT(40.576725972477696 74.90337152469496) bank193231 +193232 POINT(39.97569911090955 73.20698265725564) bank193232 +193233 POINT(39.75934737038121 73.06377291793757) bank193233 +193234 POINT(41.0979874124153 73.01176501305254) bank193234 +193235 POINT(40.45218618690315 73.13445149998634) bank193235 +193236 POINT(40.866379105327205 74.01960047248649) bank193236 +193237 POINT(40.17587322144518 74.35420219065342) bank193237 +193238 POINT(41.03847360273795 74.13319452124132) bank193238 +193239 POINT(41.473276232374864 74.32589092717251) bank193239 +193240 POINT(41.09223170965196 74.15615116483741) bank193240 +193241 POINT(40.02935245056562 73.32650751699938) bank193241 +193242 POINT(40.76813460351357 74.69184584189233) bank193242 +193243 POINT(41.1407599926068 74.4129610105709) bank193243 +193244 POINT(39.9558043072853 74.07409802022157) bank193244 +193245 POINT(40.85692005942812 73.28679425211304) bank193245 +193246 POINT(41.689370391902976 73.73766724662056) bank193246 +193247 POINT(41.11943374337794 74.8624890976861) bank193247 +193248 POINT(41.24829758523987 73.04673528159829) bank193248 +193249 POINT(39.740400588224745 74.32252040898182) bank193249 +193250 POINT(41.612778752673826 73.60071326888959) bank193250 +193251 POINT(40.697447085326374 73.59687669087279) bank193251 +193252 POINT(41.69656728284076 74.43819408193828) bank193252 +193253 POINT(40.746850883554075 73.51741081060847) bank193253 +193254 POINT(40.931593222545814 73.65346591666092) bank193254 +193255 POINT(39.81329568033762 73.43948797166658) bank193255 +193256 POINT(40.29682423739727 74.12039632159042) bank193256 +193257 POINT(40.85244455365053 74.55960139986173) bank193257 +193258 POINT(39.93976525900806 73.79201565810101) bank193258 +193259 POINT(40.04102958429689 73.03866672858554) bank193259 +193260 POINT(40.29276731141777 73.76563164150639) bank193260 +193261 POINT(40.0419930610242 74.5856291609001) bank193261 +193262 POINT(39.822248086053584 73.41918203289717) bank193262 +193263 POINT(40.69865016918533 74.94407194874813) bank193263 +193264 POINT(40.967295703127434 73.8846087054898) bank193264 +193265 POINT(40.54869105741771 73.26058329656074) bank193265 +193266 POINT(40.53210038663186 73.53073840166607) bank193266 +193267 POINT(40.302714888416396 73.19664972636724) bank193267 +193268 POINT(41.071805853974055 73.40770567038925) bank193268 +193269 POINT(40.080768789353456 74.2840508656041) bank193269 +193270 POINT(41.496434674698826 74.54838744931266) bank193270 +193271 POINT(41.306823322149434 73.24787558288263) bank193271 +193272 POINT(41.480261857191685 74.03759681551722) bank193272 +193273 POINT(41.38762691772655 74.49278055262448) bank193273 +193274 POINT(41.57573440313715 74.94490510232484) bank193274 +193275 POINT(41.53439892465755 73.68262418330228) bank193275 +193276 POINT(40.92684012917939 74.42900510889422) bank193276 +193277 POINT(40.67276996718058 73.44303843121439) bank193277 +193278 POINT(40.757298865805524 74.25653115905126) bank193278 +193279 POINT(40.61103397091166 74.923087240176) bank193279 +193280 POINT(41.160623981634565 74.60778610198193) bank193280 +193281 POINT(40.4918835035933 74.86336004570282) bank193281 +193282 POINT(41.5807276945967 74.21783098589178) bank193282 +193283 POINT(39.94694884004033 74.27057091373699) bank193283 +193284 POINT(41.65726553268418 73.42937230799323) bank193284 +193285 POINT(41.474179477987796 73.98546925141294) bank193285 +193286 POINT(40.638562106655655 73.1753675892128) bank193286 +193287 POINT(39.818817859739966 73.2489518543052) bank193287 +193288 POINT(40.92183562571467 73.75220312941347) bank193288 +193289 POINT(41.01192670217734 73.95110662454879) bank193289 +193290 POINT(40.84930189183236 73.80694269352077) bank193290 +193291 POINT(39.761055554023194 73.40434530632534) bank193291 +193292 POINT(40.00352125297709 74.85349006556021) bank193292 +193293 POINT(41.17265525495782 73.55705965519988) bank193293 +193294 POINT(40.455775387005986 74.79465291672668) bank193294 +193295 POINT(41.66586677501368 73.93306905096391) bank193295 +193296 POINT(40.43368372648963 74.6677825768644) bank193296 +193297 POINT(40.79775496172257 73.96402559136993) bank193297 +193298 POINT(40.63958088247333 74.61747070005462) bank193298 +193299 POINT(41.32582657346891 74.21412568298695) bank193299 +193300 POINT(40.835057851814824 74.95996272481712) bank193300 +193301 POINT(41.22296598554658 74.80393210076247) bank193301 +193302 POINT(40.63692093300829 73.4045403940756) bank193302 +193303 POINT(41.63733357055744 73.35824855193545) bank193303 +193304 POINT(41.674267006996466 74.30430494706845) bank193304 +193305 POINT(40.76288980429427 73.25016613379755) bank193305 +193306 POINT(40.585730286754654 74.31420863240162) bank193306 +193307 POINT(40.192140221550176 74.74437560147248) bank193307 +193308 POINT(40.75682000610871 74.25370870653617) bank193308 +193309 POINT(39.86609157708282 73.46511765628087) bank193309 +193310 POINT(41.41472023843266 74.98445200007966) bank193310 +193311 POINT(39.80695590570166 74.0958694480047) bank193311 +193312 POINT(40.09534016533035 74.12270710818233) bank193312 +193313 POINT(40.823651659211826 73.64296065868915) bank193313 +193314 POINT(40.907937048539395 73.9371417839057) bank193314 +193315 POINT(40.808877042515014 74.47513419810223) bank193315 +193316 POINT(39.71650777628467 74.05300437330969) bank193316 +193317 POINT(41.0747812042782 73.75959028832953) bank193317 +193318 POINT(41.280840188324554 74.26168344131817) bank193318 +193319 POINT(41.37804360557706 74.22166789761401) bank193319 +193320 POINT(41.27031936743358 74.76966101505894) bank193320 +193321 POINT(40.71143400385554 74.75373489628662) bank193321 +193322 POINT(41.23807641569283 73.61904375936534) bank193322 +193323 POINT(41.511675637315676 74.83937087881353) bank193323 +193324 POINT(41.144561187888186 73.81898146673694) bank193324 +193325 POINT(41.65575058905724 73.97354107930565) bank193325 +193326 POINT(39.921328317868216 74.51499899384828) bank193326 +193327 POINT(40.42531733648788 74.08194063616303) bank193327 +193328 POINT(40.342622585622514 74.2598110338275) bank193328 +193329 POINT(40.53540165553277 74.70408917626862) bank193329 +193330 POINT(40.46487604126158 73.40025386373682) bank193330 +193331 POINT(41.01344551734935 74.76773423565913) bank193331 +193332 POINT(40.52569990202099 73.81819756648466) bank193332 +193333 POINT(41.59092746949852 74.37201834792573) bank193333 +193334 POINT(39.778370445632525 74.85320367932293) bank193334 +193335 POINT(40.06100310344945 74.1384371230724) bank193335 +193336 POINT(40.56224294163155 74.62262211824282) bank193336 +193337 POINT(41.36188313524428 73.35695894069097) bank193337 +193338 POINT(40.703792524454585 74.00346946718702) bank193338 +193339 POINT(40.34442905477065 73.51260659071716) bank193339 +193340 POINT(40.35115860686273 74.18497632598053) bank193340 +193341 POINT(41.56745147372866 74.96082849642384) bank193341 +193342 POINT(41.286205704687625 73.81076300852732) bank193342 +193343 POINT(40.57253265422625 74.42374000571569) bank193343 +193344 POINT(40.002751415389795 74.39602229872301) bank193344 +193345 POINT(40.24584052432521 73.8202475808088) bank193345 +193346 POINT(40.04113218008289 73.47283628739902) bank193346 +193347 POINT(39.787459945402325 74.54833793404975) bank193347 +193348 POINT(40.28440360076923 74.34300842473942) bank193348 +193349 POINT(39.726339374236446 73.49873284339586) bank193349 +193350 POINT(41.204884684933084 74.36162835686285) bank193350 +193351 POINT(40.687919843575386 73.92737306994847) bank193351 +193352 POINT(40.31064023343177 73.78836646569637) bank193352 +193353 POINT(40.267032677224016 73.09899714318166) bank193353 +193354 POINT(41.13881680001827 73.23004548773316) bank193354 +193355 POINT(41.27734927141221 74.61882176327333) bank193355 +193356 POINT(40.32786567491754 73.04097085099092) bank193356 +193357 POINT(40.75176564486548 73.41372586411121) bank193357 +193358 POINT(41.34710693695378 73.12765011434155) bank193358 +193359 POINT(40.843421954242736 73.95867418007892) bank193359 +193360 POINT(40.59975718800986 74.44248095421182) bank193360 +193361 POINT(41.268709696851275 74.84913357719861) bank193361 +193362 POINT(40.4878629482078 73.50730722320952) bank193362 +193363 POINT(39.95245008618256 74.08174954163515) bank193363 +193364 POINT(39.80771215613856 74.90476812990772) bank193364 +193365 POINT(41.112054007263886 73.17606367467049) bank193365 +193366 POINT(41.13557527790689 74.54681303927734) bank193366 +193367 POINT(40.97781735552492 73.02181377914106) bank193367 +193368 POINT(40.26485121660354 74.26174120596211) bank193368 +193369 POINT(40.537199929356504 74.12689262634504) bank193369 +193370 POINT(39.79717133771544 73.98290166847632) bank193370 +193371 POINT(41.393535845554545 73.97937757448871) bank193371 +193372 POINT(40.19053456980545 73.03009096964902) bank193372 +193373 POINT(40.89469763235498 74.28825142389285) bank193373 +193374 POINT(39.76510569915164 73.08179333635721) bank193374 +193375 POINT(40.27589625487238 74.65556206984067) bank193375 +193376 POINT(40.68248101298743 73.10009478556995) bank193376 +193377 POINT(41.63785580014567 74.1728241839692) bank193377 +193378 POINT(40.56831012301486 73.53410121737205) bank193378 +193379 POINT(39.988910228337275 73.94719037115122) bank193379 +193380 POINT(40.116806475540045 73.63714006892701) bank193380 +193381 POINT(40.64046309031449 73.98862777290758) bank193381 +193382 POINT(40.996620425947064 74.56182381292386) bank193382 +193383 POINT(40.70147899530878 73.87573595390027) bank193383 +193384 POINT(40.79568021591252 73.87163668065456) bank193384 +193385 POINT(39.854350569536884 73.28376271335128) bank193385 +193386 POINT(40.31911722978006 73.62171791997947) bank193386 +193387 POINT(41.202282823546994 73.51997874993147) bank193387 +193388 POINT(41.69364825432252 74.19855945747581) bank193388 +193389 POINT(40.14068734274863 74.68223558752355) bank193389 +193390 POINT(40.97334526108345 74.04708096628535) bank193390 +193391 POINT(40.81887680463067 73.21275976139472) bank193391 +193392 POINT(40.80658780819516 74.15553880858802) bank193392 +193393 POINT(41.666394332874766 73.59197430286385) bank193393 +193394 POINT(39.851308375636975 74.58173155201547) bank193394 +193395 POINT(40.829513705933444 74.16014018101906) bank193395 +193396 POINT(40.879972416938244 74.72708229447622) bank193396 +193397 POINT(40.85080231616481 73.72655141427336) bank193397 +193398 POINT(40.631354196552536 73.78852311115362) bank193398 +193399 POINT(41.07030879283825 73.96538630352315) bank193399 +193400 POINT(41.06812377758054 74.07253265601686) bank193400 +193401 POINT(40.904266584109514 73.79839402276247) bank193401 +193402 POINT(40.30730298235398 74.01734729735797) bank193402 +193403 POINT(41.03946921102082 74.85972163943836) bank193403 +193404 POINT(40.010953232216785 73.34560108647618) bank193404 +193405 POINT(41.28334275703096 74.91682716294336) bank193405 +193406 POINT(40.511559323497636 73.43695315810491) bank193406 +193407 POINT(40.63746161193437 73.73465244304779) bank193407 +193408 POINT(40.392080537079465 73.32326788995186) bank193408 +193409 POINT(40.475685441627164 73.3783014381433) bank193409 +193410 POINT(39.768085720684624 74.95675319916958) bank193410 +193411 POINT(41.33212272809704 73.65985720202593) bank193411 +193412 POINT(41.438428570556326 73.0937398837283) bank193412 +193413 POINT(41.207340543056745 73.32058223949154) bank193413 +193414 POINT(41.341090575897944 74.47901266483805) bank193414 +193415 POINT(41.549473651228375 74.22433869093253) bank193415 +193416 POINT(40.25275811209048 74.13496910077751) bank193416 +193417 POINT(41.14304780047118 74.97711279448015) bank193417 +193418 POINT(39.83862913354682 74.2581947535065) bank193418 +193419 POINT(39.89383584630651 73.46191109215654) bank193419 +193420 POINT(40.13828915457744 73.48683623623614) bank193420 +193421 POINT(41.568369217689835 73.1838730673047) bank193421 +193422 POINT(40.50381089345611 73.90417813025896) bank193422 +193423 POINT(40.95607823425023 73.1524282170146) bank193423 +193424 POINT(40.029206294258024 74.0652559363025) bank193424 +193425 POINT(41.44908498434779 74.14741451322357) bank193425 +193426 POINT(41.13279606715653 73.04961533766875) bank193426 +193427 POINT(39.74432246521292 73.11276625900047) bank193427 +193428 POINT(40.53395729623884 74.1259089471521) bank193428 +193429 POINT(39.914168577671916 74.4413062839206) bank193429 +193430 POINT(40.63231679891457 74.19652013881324) bank193430 +193431 POINT(41.22557187783409 73.69745969847487) bank193431 +193432 POINT(40.96373261071633 74.30290278489784) bank193432 +193433 POINT(41.081313403017155 73.26725692095162) bank193433 +193434 POINT(39.911735104734504 73.26465328059713) bank193434 +193435 POINT(40.3561746127269 73.89788615190872) bank193435 +193436 POINT(40.752356956312646 73.38540116374051) bank193436 +193437 POINT(41.71005167161524 74.06664671263937) bank193437 +193438 POINT(40.989406452736006 74.03239195404326) bank193438 +193439 POINT(41.43037948648564 73.32111912204624) bank193439 +193440 POINT(40.87048424126087 74.10358495120154) bank193440 +193441 POINT(40.51588479688063 74.6953481580196) bank193441 +193442 POINT(40.977912010539754 74.63953652811142) bank193442 +193443 POINT(40.94102116323128 74.46478897155814) bank193443 +193444 POINT(39.99712359388623 74.56599831154121) bank193444 +193445 POINT(40.528710711470325 73.38203275204187) bank193445 +193446 POINT(40.97768975723441 73.89207449419752) bank193446 +193447 POINT(40.081856134417094 74.8222882224702) bank193447 +193448 POINT(40.50747510550369 73.62195311302223) bank193448 +193449 POINT(39.97611968744158 74.7336844877985) bank193449 +193450 POINT(40.43372926457708 74.74497738090632) bank193450 +193451 POINT(41.241139320240755 74.20095211786443) bank193451 +193452 POINT(40.63444138193967 73.29086369627231) bank193452 +193453 POINT(40.77686746544492 74.09276222718321) bank193453 +193454 POINT(41.54228350303064 74.86425572553516) bank193454 +193455 POINT(40.671504981885235 73.74060441543696) bank193455 +193456 POINT(41.53260894241954 74.20495884629601) bank193456 +193457 POINT(40.695377186536604 74.84740529426497) bank193457 +193458 POINT(41.44260191812238 74.51536187526548) bank193458 +193459 POINT(41.19320514427207 74.58823883183615) bank193459 +193460 POINT(40.69934638605114 73.62624676151366) bank193460 +193461 POINT(40.30082681587862 74.37987854168371) bank193461 +193462 POINT(39.749018394372 74.19501792000497) bank193462 +193463 POINT(41.534748233308385 74.08986645649216) bank193463 +193464 POINT(40.114475765975634 73.34549969516338) bank193464 +193465 POINT(40.946702031459566 74.20929340793958) bank193465 +193466 POINT(40.81889964880975 74.57576823246475) bank193466 +193467 POINT(40.359084551028516 74.78859225804982) bank193467 +193468 POINT(40.1613454772741 74.64929916461692) bank193468 +193469 POINT(40.54459817549905 74.87077616394225) bank193469 +193470 POINT(41.648749723457755 74.76044117055338) bank193470 +193471 POINT(40.36414884277846 73.41193618513779) bank193471 +193472 POINT(40.44437991112446 74.31963228569118) bank193472 +193473 POINT(40.64361813219678 73.50699453424346) bank193473 +193474 POINT(41.30594830104187 73.03656199979861) bank193474 +193475 POINT(41.63317338663999 73.25084028555645) bank193475 +193476 POINT(40.08196389865647 74.67986712044664) bank193476 +193477 POINT(39.89780204494224 74.68592050641959) bank193477 +193478 POINT(41.31819427924466 74.7885148339554) bank193478 +193479 POINT(41.417064402887 74.63823811498749) bank193479 +193480 POINT(41.50738849615028 74.3455129214404) bank193480 +193481 POINT(40.488144219561114 73.78882285054476) bank193481 +193482 POINT(41.24268268134958 73.15583381707194) bank193482 +193483 POINT(39.94165315261697 73.0967315729076) bank193483 +193484 POINT(40.156439316255344 74.40848193016242) bank193484 +193485 POINT(39.76957174598043 74.84418790526897) bank193485 +193486 POINT(40.41840650620772 73.22344387446195) bank193486 +193487 POINT(39.86042235319336 73.38795052168405) bank193487 +193488 POINT(40.36071771934398 73.6523775744573) bank193488 +193489 POINT(40.35867059469558 73.02745251091092) bank193489 +193490 POINT(41.475484854486574 74.20759197417085) bank193490 +193491 POINT(40.30336948702091 74.15066886209524) bank193491 +193492 POINT(40.97501571831756 73.59572399371095) bank193492 +193493 POINT(41.27175709201848 73.82012200934967) bank193493 +193494 POINT(41.194160431289504 73.0809541078188) bank193494 +193495 POINT(40.802408549029415 73.74373748124918) bank193495 +193496 POINT(39.78429093250289 74.91975393257654) bank193496 +193497 POINT(39.7387665359226 74.35152894238921) bank193497 +193498 POINT(40.08448467602405 73.83757833195118) bank193498 +193499 POINT(40.561720004182625 74.83332972957034) bank193499 +193500 POINT(39.78332759994063 74.29120410235919) bank193500 +193501 POINT(41.16733124697683 74.03659564563247) bank193501 +193502 POINT(40.67245859512519 74.6813434827102) bank193502 +193503 POINT(40.30152382686676 73.39997829410407) bank193503 +193504 POINT(39.76779077052561 73.65457486179632) bank193504 +193505 POINT(40.59245251024776 74.92560691559511) bank193505 +193506 POINT(39.79325137660864 73.43786937859643) bank193506 +193507 POINT(40.00634865955156 73.99019486970751) bank193507 +193508 POINT(40.17164674057007 74.62116194192642) bank193508 +193509 POINT(40.04813574909984 74.96817481218869) bank193509 +193510 POINT(40.14114211743575 73.6740759831761) bank193510 +193511 POINT(41.25835105486407 74.98134704814497) bank193511 +193512 POINT(41.43837859736393 74.63927958356696) bank193512 +193513 POINT(40.504564862615815 73.8550133020088) bank193513 +193514 POINT(39.89584973093045 73.39851504679213) bank193514 +193515 POINT(41.44727745427928 73.40548045625992) bank193515 +193516 POINT(40.38683092611418 74.99245631942883) bank193516 +193517 POINT(39.896888919795785 74.05830216205892) bank193517 +193518 POINT(41.549381753252554 73.07568606282977) bank193518 +193519 POINT(41.01721438837288 73.9943132640564) bank193519 +193520 POINT(40.299190197062835 74.80171481758624) bank193520 +193521 POINT(41.16171358747734 74.20405007392576) bank193521 +193522 POINT(40.60454358006693 74.6410858971274) bank193522 +193523 POINT(39.95215863154399 74.48803363365526) bank193523 +193524 POINT(41.24148935260839 73.47889601133906) bank193524 +193525 POINT(41.6765372042035 73.14193162378672) bank193525 +193526 POINT(40.9695141003234 74.43202191157984) bank193526 +193527 POINT(40.11508903929552 73.6719386079473) bank193527 +193528 POINT(41.256309256020955 74.18620293617752) bank193528 +193529 POINT(41.20385673155027 74.87982237010127) bank193529 +193530 POINT(41.470096244144 73.299308561404) bank193530 +193531 POINT(40.45015028737145 74.51209519941311) bank193531 +193532 POINT(40.00097049565871 74.06297818145615) bank193532 +193533 POINT(40.64798116151202 74.31864826262388) bank193533 +193534 POINT(40.098961662877436 73.15475098085496) bank193534 +193535 POINT(41.530447784435715 74.04720104402887) bank193535 +193536 POINT(41.43891239986038 74.35588577088403) bank193536 +193537 POINT(40.447019756235534 73.13069198109804) bank193537 +193538 POINT(40.994514584932304 73.48565470479194) bank193538 +193539 POINT(40.50678971141969 74.83131468351623) bank193539 +193540 POINT(40.39642375771693 73.0447531772838) bank193540 +193541 POINT(40.4809816314078 74.58371407319605) bank193541 +193542 POINT(40.45950837908248 74.15154732516199) bank193542 +193543 POINT(40.95549210653208 74.49404342950903) bank193543 +193544 POINT(39.793234492292775 74.91168880787359) bank193544 +193545 POINT(40.64036695971999 74.91435524233648) bank193545 +193546 POINT(40.46997587054813 74.05753623503156) bank193546 +193547 POINT(40.50737458860825 74.07893775018691) bank193547 +193548 POINT(39.81065979206035 73.13340201141953) bank193548 +193549 POINT(41.035137404839006 73.14309666914596) bank193549 +193550 POINT(41.66708124825429 74.31752909280921) bank193550 +193551 POINT(41.5735946307977 73.03978226487376) bank193551 +193552 POINT(41.65870957445089 73.86967700357714) bank193552 +193553 POINT(39.956010724257254 74.89218350049835) bank193553 +193554 POINT(39.87328950716271 74.30553029414646) bank193554 +193555 POINT(41.68786148246154 73.92341068263445) bank193555 +193556 POINT(40.655503665841046 73.31594796481673) bank193556 +193557 POINT(41.069522843178774 73.12034843352527) bank193557 +193558 POINT(41.03465487849039 74.6288199054063) bank193558 +193559 POINT(40.71527596197903 73.75701835076252) bank193559 +193560 POINT(41.6114727426177 74.76488426110726) bank193560 +193561 POINT(40.41819578588684 74.09937572239104) bank193561 +193562 POINT(40.995502660965734 74.15838660544233) bank193562 +193563 POINT(41.634405960282784 74.41817992665048) bank193563 +193564 POINT(40.36002845552629 73.0856308804435) bank193564 +193565 POINT(41.00596997162072 74.12127779430588) bank193565 +193566 POINT(39.740971637375694 74.18865782150206) bank193566 +193567 POINT(40.85892448446699 73.32483209984603) bank193567 +193568 POINT(40.556064934257336 74.01785158290421) bank193568 +193569 POINT(41.19138266895948 74.3332770185577) bank193569 +193570 POINT(41.385365943468884 73.89222303524073) bank193570 +193571 POINT(40.419427487247496 73.02047628611064) bank193571 +193572 POINT(40.12912238648589 73.9011843803343) bank193572 +193573 POINT(41.0449774892554 74.34722557933799) bank193573 +193574 POINT(41.131928901722794 74.70696861965897) bank193574 +193575 POINT(41.42800669620954 73.32668899186561) bank193575 +193576 POINT(40.375415491291385 74.01150842539651) bank193576 +193577 POINT(40.75363775153922 74.75118064047594) bank193577 +193578 POINT(39.806187359703124 73.5149162918328) bank193578 +193579 POINT(40.31070002380157 73.1148834553081) bank193579 +193580 POINT(40.62018722480202 74.59872576022724) bank193580 +193581 POINT(40.54586897963754 74.9569922940915) bank193581 +193582 POINT(40.35812316527462 74.89513094741696) bank193582 +193583 POINT(40.341711624618036 73.15808504484119) bank193583 +193584 POINT(40.949426760843124 74.81420223763192) bank193584 +193585 POINT(40.85566582877918 73.992047050059) bank193585 +193586 POINT(40.41972965750004 73.4750737543204) bank193586 +193587 POINT(40.208410558644324 74.88383520200227) bank193587 +193588 POINT(39.81534384472958 73.2732863927646) bank193588 +193589 POINT(40.71635261021151 73.62620277580163) bank193589 +193590 POINT(41.46754838064387 73.66793824338012) bank193590 +193591 POINT(39.91150942456949 74.88463953433758) bank193591 +193592 POINT(39.950594810933424 74.80425722648219) bank193592 +193593 POINT(39.77748794548877 73.84864765315073) bank193593 +193594 POINT(39.777117585707884 73.81954622346234) bank193594 +193595 POINT(41.08991286978976 73.86896363094063) bank193595 +193596 POINT(41.2064363717784 73.73916640285553) bank193596 +193597 POINT(41.64109814105475 73.53164190310898) bank193597 +193598 POINT(41.69103227774046 73.61308445081674) bank193598 +193599 POINT(40.33138618167011 73.4950792434776) bank193599 +193600 POINT(41.39079257648872 73.69108556280226) bank193600 +193601 POINT(40.901375449126974 74.96567488917438) bank193601 +193602 POINT(40.915525715712164 73.45589705307323) bank193602 +193603 POINT(41.70273618641044 74.60417436504602) bank193603 +193604 POINT(41.08778232233037 73.09976259153602) bank193604 +193605 POINT(39.96274593791984 74.31980405708144) bank193605 +193606 POINT(41.38759932134575 74.29593610129304) bank193606 +193607 POINT(41.085199350360156 74.64399622227813) bank193607 +193608 POINT(41.30715208847802 73.94083255040456) bank193608 +193609 POINT(40.522471911738506 74.84776986605014) bank193609 +193610 POINT(40.18965652221518 73.4217982934295) bank193610 +193611 POINT(39.84872678007955 74.16845013780105) bank193611 +193612 POINT(40.88157122283865 74.63872338582635) bank193612 +193613 POINT(39.92824318763116 73.67713899441382) bank193613 +193614 POINT(40.58696003867411 73.40943329036321) bank193614 +193615 POINT(40.57844450387907 74.18359487089592) bank193615 +193616 POINT(41.2347352057648 74.95180684515117) bank193616 +193617 POINT(40.61166582164473 74.37450984280201) bank193617 +193618 POINT(40.76236574168226 73.835838691061) bank193618 +193619 POINT(41.2030400953189 74.94569787596554) bank193619 +193620 POINT(41.175459573330414 74.93466394512234) bank193620 +193621 POINT(40.01712901076358 73.76878889197842) bank193621 +193622 POINT(40.13857839361792 73.91866150226058) bank193622 +193623 POINT(41.06174966860414 74.08773653430941) bank193623 +193624 POINT(40.33677886948693 74.6854135249095) bank193624 +193625 POINT(40.65666800852963 73.0422908621061) bank193625 +193626 POINT(40.55488699256731 74.36477043863452) bank193626 +193627 POINT(40.331911253092706 74.40280722959197) bank193627 +193628 POINT(39.97208406916609 73.56475185479677) bank193628 +193629 POINT(41.13912821625745 73.19407372733914) bank193629 +193630 POINT(41.221794532200434 74.50765953269163) bank193630 +193631 POINT(40.3233714406886 74.43600443393105) bank193631 +193632 POINT(39.96415819088749 73.59276611818655) bank193632 +193633 POINT(41.416070226488934 73.65893915240478) bank193633 +193634 POINT(40.93495905645547 74.07267236704935) bank193634 +193635 POINT(39.789350748793524 73.34125433109338) bank193635 +193636 POINT(40.31473288982888 73.87692996134979) bank193636 +193637 POINT(41.02517246700883 74.4637657025656) bank193637 +193638 POINT(40.36118799982445 73.26479416902642) bank193638 +193639 POINT(40.67240748402516 73.87345217982127) bank193639 +193640 POINT(40.844185893003505 73.68599247630107) bank193640 +193641 POINT(39.98001964995857 74.0431544790846) bank193641 +193642 POINT(39.98555485740028 75.00517963900707) bank193642 +193643 POINT(41.37280453416383 74.1930750605833) bank193643 +193644 POINT(40.1257230745924 74.00607018385244) bank193644 +193645 POINT(39.72936768825277 74.60919708717667) bank193645 +193646 POINT(40.000853508696025 73.24520436492608) bank193646 +193647 POINT(40.11779907641787 74.29970842918421) bank193647 +193648 POINT(40.32387790187789 73.67364299088523) bank193648 +193649 POINT(41.33295041820401 74.66212758662446) bank193649 +193650 POINT(40.21904012665822 74.75204833848534) bank193650 +193651 POINT(39.756018821162144 73.15965268791564) bank193651 +193652 POINT(40.86873853593519 73.94390428348358) bank193652 +193653 POINT(40.303199583085615 73.92513421954929) bank193653 +193654 POINT(40.32287977518163 74.62981952884266) bank193654 +193655 POINT(40.21833393247928 74.6127150612483) bank193655 +193656 POINT(39.732275532007854 73.18805381354466) bank193656 +193657 POINT(40.46935654760346 73.50394084746245) bank193657 +193658 POINT(40.921113025133195 74.08433647286564) bank193658 +193659 POINT(40.90299275665043 74.03839736168605) bank193659 +193660 POINT(41.233580525523976 73.17785935305032) bank193660 +193661 POINT(41.41597696950621 74.9409712723484) bank193661 +193662 POINT(41.28628106593541 74.8980335844686) bank193662 +193663 POINT(39.85489109162715 73.73870590856667) bank193663 +193664 POINT(41.68552766461227 74.88467303258226) bank193664 +193665 POINT(41.427814745303614 74.83616677560344) bank193665 +193666 POINT(39.99977563564861 73.82279113473501) bank193666 +193667 POINT(41.43337497088409 74.29747685157282) bank193667 +193668 POINT(40.744620772711336 73.54266930289295) bank193668 +193669 POINT(40.052840968399046 74.96330708954218) bank193669 +193670 POINT(41.45764800719193 74.2435956846954) bank193670 +193671 POINT(40.09985269852644 73.62454797921498) bank193671 +193672 POINT(40.31660923918107 73.21633076923551) bank193672 +193673 POINT(40.371521348713536 73.13042219480964) bank193673 +193674 POINT(41.01495271210378 74.35878258655768) bank193674 +193675 POINT(40.84952514920333 73.49967941574586) bank193675 +193676 POINT(40.34635638224537 74.40959843458563) bank193676 +193677 POINT(40.3376566445148 74.3424296653718) bank193677 +193678 POINT(40.974753527994345 73.73960901066246) bank193678 +193679 POINT(40.921791292851466 74.75966861505583) bank193679 +193680 POINT(40.220131103640476 74.91337262292825) bank193680 +193681 POINT(39.832137431131564 73.90942658690207) bank193681 +193682 POINT(40.2031694222745 73.83456951514694) bank193682 +193683 POINT(40.640564236160166 73.83187661242509) bank193683 +193684 POINT(40.97863741472561 74.61334258228405) bank193684 +193685 POINT(41.24407434670876 74.37119132863782) bank193685 +193686 POINT(41.175191946924706 74.8247151621993) bank193686 +193687 POINT(40.708193763688016 73.6374853628083) bank193687 +193688 POINT(41.59172184973429 74.61226773737067) bank193688 +193689 POINT(41.45250747085786 74.49297616978602) bank193689 +193690 POINT(40.67884401458238 74.79521528867699) bank193690 +193691 POINT(40.04743558692613 73.71130268928127) bank193691 +193692 POINT(41.42448636772193 73.09638069168322) bank193692 +193693 POINT(39.9518909658059 73.20624307163807) bank193693 +193694 POINT(40.44691087896616 73.12196805054782) bank193694 +193695 POINT(39.77138744829549 73.20158938599927) bank193695 +193696 POINT(40.409347972853595 74.54941256797682) bank193696 +193697 POINT(39.93131867060255 73.81750077641193) bank193697 +193698 POINT(41.259340551846485 74.93534891922447) bank193698 +193699 POINT(40.11839435439305 73.81521625279476) bank193699 +193700 POINT(41.15639750133682 73.20079233115106) bank193700 +193701 POINT(40.61691070909074 74.96073947543384) bank193701 +193702 POINT(39.97823400698053 74.97994937054519) bank193702 +193703 POINT(40.72925596387268 74.69469859048064) bank193703 +193704 POINT(41.4413558320386 74.75306088574594) bank193704 +193705 POINT(39.90542653882665 73.69896813897054) bank193705 +193706 POINT(41.11614946665002 73.63177576415231) bank193706 +193707 POINT(41.23764810677596 74.02662458283517) bank193707 +193708 POINT(40.07912035754163 74.37686470495913) bank193708 +193709 POINT(40.17947840414016 74.42617923738545) bank193709 +193710 POINT(40.43278778813096 74.08690450168966) bank193710 +193711 POINT(41.34147181127533 73.92676337398302) bank193711 +193712 POINT(41.456659838105004 73.65609886338859) bank193712 +193713 POINT(40.64228570082684 74.05986495962343) bank193713 +193714 POINT(39.81655316940995 74.64202324131077) bank193714 +193715 POINT(40.02560336200975 74.13952317479476) bank193715 +193716 POINT(41.34359395695672 74.2215840561957) bank193716 +193717 POINT(40.903673200656456 73.37429996624837) bank193717 +193718 POINT(40.02700883333176 74.68857268986955) bank193718 +193719 POINT(40.46440270159154 73.42070395994101) bank193719 +193720 POINT(40.96813959125393 73.63623960525497) bank193720 +193721 POINT(39.79593993729107 73.73915959160598) bank193721 +193722 POINT(39.989121925905714 73.18710546624048) bank193722 +193723 POINT(41.616195548829246 73.29872453106363) bank193723 +193724 POINT(40.8174386507667 74.22670870415992) bank193724 +193725 POINT(40.18286530529581 74.42866630826877) bank193725 +193726 POINT(40.399608251274294 74.2501966384391) bank193726 +193727 POINT(40.55745368739412 74.7597166398676) bank193727 +193728 POINT(40.861353097629745 74.27991276517025) bank193728 +193729 POINT(41.155928664219836 74.42221598710991) bank193729 +193730 POINT(39.92209575608301 73.73812791371704) bank193730 +193731 POINT(39.895498594357406 74.78739211976608) bank193731 +193732 POINT(40.57066426376097 74.85071103941867) bank193732 +193733 POINT(41.4044186170939 73.03810206319959) bank193733 +193734 POINT(40.14175297485455 74.58610996356056) bank193734 +193735 POINT(41.64947309487219 73.40962230903376) bank193735 +193736 POINT(40.23226984800384 73.6406873190397) bank193736 +193737 POINT(40.43927158845923 73.36929680753234) bank193737 +193738 POINT(40.52640276772335 73.76915475625398) bank193738 +193739 POINT(39.7760402463264 73.38241681201039) bank193739 +193740 POINT(39.893030603413266 74.67245648195636) bank193740 +193741 POINT(40.77538550043077 73.07606354048572) bank193741 +193742 POINT(40.62102095682684 74.27936952192375) bank193742 +193743 POINT(40.54949753364475 74.11754980430017) bank193743 +193744 POINT(40.10168413714045 74.69422065212026) bank193744 +193745 POINT(39.913353594949335 74.89382818104993) bank193745 +193746 POINT(40.40350448495464 73.22670792214082) bank193746 +193747 POINT(39.891849845611716 73.07462187886432) bank193747 +193748 POINT(40.05355503220372 73.25262388514317) bank193748 +193749 POINT(40.99457358866525 74.74205997844335) bank193749 +193750 POINT(41.65220575346109 73.53326246655291) bank193750 +193751 POINT(40.6819727412023 73.23782128443199) bank193751 +193752 POINT(41.078987928464606 74.75648867288173) bank193752 +193753 POINT(41.47701893985643 74.57919875338455) bank193753 +193754 POINT(41.10457465262557 74.88018904044091) bank193754 +193755 POINT(41.14214701466681 73.57452281355745) bank193755 +193756 POINT(40.47923146701705 73.31171935300551) bank193756 +193757 POINT(41.212815119749045 74.14217447615948) bank193757 +193758 POINT(40.98189442445113 73.44498655470883) bank193758 +193759 POINT(41.245806744666055 73.60811524536898) bank193759 +193760 POINT(41.11906965343918 74.13413367732994) bank193760 +193761 POINT(41.133510048473354 74.44193999980102) bank193761 +193762 POINT(40.25043306758151 73.03326926687544) bank193762 +193763 POINT(40.736083864132866 74.18430265939084) bank193763 +193764 POINT(41.34428499324104 74.60540700219077) bank193764 +193765 POINT(40.95781967635559 73.40970197648092) bank193765 +193766 POINT(41.42551058190166 74.26164321472923) bank193766 +193767 POINT(41.57897785112098 73.1364664409003) bank193767 +193768 POINT(40.707876544086204 74.24494908724358) bank193768 +193769 POINT(39.73918014938713 74.66004701297084) bank193769 +193770 POINT(40.21867372812301 74.24289291193132) bank193770 +193771 POINT(41.34104256372975 74.44520953689872) bank193771 +193772 POINT(39.72598410057405 73.11727758879583) bank193772 +193773 POINT(40.96632689983439 74.64021880523892) bank193773 +193774 POINT(39.80486584973749 73.08757240851504) bank193774 +193775 POINT(41.415662312521114 73.57992775949624) bank193775 +193776 POINT(40.63810026321153 73.5971165385797) bank193776 +193777 POINT(41.13128954473575 73.60579568553176) bank193777 +193778 POINT(41.4562250881393 74.93861015435074) bank193778 +193779 POINT(39.82464573136077 74.69063212666408) bank193779 +193780 POINT(40.15156576297487 74.1106841255224) bank193780 +193781 POINT(40.429704305627745 73.86035619895763) bank193781 +193782 POINT(39.830892832012196 74.53021157103574) bank193782 +193783 POINT(41.28965675071146 74.78283771467179) bank193783 +193784 POINT(41.33249077598837 73.66296483657315) bank193784 +193785 POINT(41.65719121145434 74.23257693402749) bank193785 +193786 POINT(41.0141063354758 74.3202694910907) bank193786 +193787 POINT(40.259253271090046 74.70436012399476) bank193787 +193788 POINT(40.011575692238694 74.01229058055979) bank193788 +193789 POINT(41.09406096076698 73.7723057374466) bank193789 +193790 POINT(40.97139138670407 74.38430940353823) bank193790 +193791 POINT(41.45485692846694 73.88478455701497) bank193791 +193792 POINT(40.24333829305893 73.25800798945014) bank193792 +193793 POINT(40.970858491512885 73.8183077982911) bank193793 +193794 POINT(41.260976014793016 74.71135375687918) bank193794 +193795 POINT(39.78489552343696 74.43030341368113) bank193795 +193796 POINT(40.69874379364858 74.03964588144248) bank193796 +193797 POINT(40.46016766483615 73.58168464240889) bank193797 +193798 POINT(40.816151867232676 73.61824083652817) bank193798 +193799 POINT(41.45156115098916 74.84984291186453) bank193799 +193800 POINT(40.36609665322075 73.91011272230222) bank193800 +193801 POINT(40.89428968967544 74.92567488250654) bank193801 +193802 POINT(40.483480281477405 73.50253459360086) bank193802 +193803 POINT(40.78769297541422 74.33294912262956) bank193803 +193804 POINT(41.66669659126267 74.71955809323536) bank193804 +193805 POINT(40.51694138448938 74.41379418194472) bank193805 +193806 POINT(41.065086773869396 74.1831899102206) bank193806 +193807 POINT(40.787024193140304 74.46368026702997) bank193807 +193808 POINT(40.830202102692105 73.75026265128524) bank193808 +193809 POINT(39.892846909645975 73.6859740051901) bank193809 +193810 POINT(40.34322091277403 74.12168823112081) bank193810 +193811 POINT(40.23444988804213 74.20526579162912) bank193811 +193812 POINT(41.70196819416129 74.12830503404041) bank193812 +193813 POINT(39.875729086498296 74.68609977960637) bank193813 +193814 POINT(40.150040983100816 73.10607105458165) bank193814 +193815 POINT(41.317582938066785 74.4362191298954) bank193815 +193816 POINT(40.57749408554357 74.85031110265) bank193816 +193817 POINT(39.87693453599984 73.16974911473682) bank193817 +193818 POINT(41.29886471298748 74.22433896523385) bank193818 +193819 POINT(41.46434910476173 73.6127842602774) bank193819 +193820 POINT(40.15733708941158 74.31519323299953) bank193820 +193821 POINT(40.38432617302404 73.63440752409343) bank193821 +193822 POINT(41.367451506361924 74.07788455646482) bank193822 +193823 POINT(39.947452237865846 74.42000370809977) bank193823 +193824 POINT(39.725640175741916 74.58082895077364) bank193824 +193825 POINT(41.35559092508269 74.37880287193741) bank193825 +193826 POINT(41.60344682537172 74.07549202551328) bank193826 +193827 POINT(40.83524479231197 73.69139400511743) bank193827 +193828 POINT(41.342666999090156 74.02035375782134) bank193828 +193829 POINT(39.797022253511344 73.47473174632758) bank193829 +193830 POINT(40.04834541908522 74.966747059385) bank193830 +193831 POINT(41.5936754969599 73.33007398671182) bank193831 +193832 POINT(41.36345542014993 74.51718918495112) bank193832 +193833 POINT(41.027416772286266 74.33443967560022) bank193833 +193834 POINT(40.22452613265749 73.27834679366676) bank193834 +193835 POINT(40.717591386790396 74.18729639279628) bank193835 +193836 POINT(40.73393960760714 74.73102357750227) bank193836 +193837 POINT(41.26619390607274 73.30719216294449) bank193837 +193838 POINT(41.197758076074635 73.53273564136525) bank193838 +193839 POINT(41.03339202637263 73.6369725216259) bank193839 +193840 POINT(41.104233043001244 74.66878168654938) bank193840 +193841 POINT(41.139897110045126 74.30863669274225) bank193841 +193842 POINT(40.76637474935282 74.400728026529) bank193842 +193843 POINT(40.278010229740815 73.29229670553356) bank193843 +193844 POINT(41.12146660517221 73.29870590450193) bank193844 +193845 POINT(41.654515218132964 74.6211024391507) bank193845 +193846 POINT(41.69479223700251 74.16737971347327) bank193846 +193847 POINT(40.06442894722126 74.385387432872) bank193847 +193848 POINT(41.711663642346124 74.97640826403611) bank193848 +193849 POINT(40.740454715541716 74.62472129416606) bank193849 +193850 POINT(41.02791903812435 73.54890017277876) bank193850 +193851 POINT(41.46480237643351 74.81530884491131) bank193851 +193852 POINT(41.37610719496064 73.36753119339697) bank193852 +193853 POINT(41.455885634835056 73.82836766067663) bank193853 +193854 POINT(40.69186816007153 73.85978353095574) bank193854 +193855 POINT(39.95325110310953 74.31944176057065) bank193855 +193856 POINT(41.45683960966114 73.09691356546789) bank193856 +193857 POINT(41.50493683586043 73.07742513373954) bank193857 +193858 POINT(41.146789560520816 74.26217974769418) bank193858 +193859 POINT(40.76489639471513 74.46614517305352) bank193859 +193860 POINT(40.52225065470313 74.6236450510251) bank193860 +193861 POINT(39.76631419015587 74.7838411340024) bank193861 +193862 POINT(40.76337439632004 74.50814579769478) bank193862 +193863 POINT(40.57468341200241 73.56650018223208) bank193863 +193864 POINT(41.4432096553068 73.31061676131745) bank193864 +193865 POINT(40.2885625654693 74.45305462033029) bank193865 +193866 POINT(39.920577643583115 74.34741153648692) bank193866 +193867 POINT(41.33602039428115 74.13841639578968) bank193867 +193868 POINT(40.78496436257536 73.72529582802599) bank193868 +193869 POINT(40.05662374389623 74.00816411680742) bank193869 +193870 POINT(40.82660251027247 74.60225743487861) bank193870 +193871 POINT(40.180866921016545 74.39992958033454) bank193871 +193872 POINT(39.863090824888474 74.76661355409748) bank193872 +193873 POINT(40.53072949717191 74.25147080231478) bank193873 +193874 POINT(41.52367353899641 73.07424371311237) bank193874 +193875 POINT(41.62457344270055 74.03912036865984) bank193875 +193876 POINT(41.04397403027153 73.81691711384995) bank193876 +193877 POINT(40.5507285491524 74.27249721299287) bank193877 +193878 POINT(40.08560841689384 74.1158891872509) bank193878 +193879 POINT(41.691626391446896 73.20918350432359) bank193879 +193880 POINT(40.13353289948135 74.05465650040207) bank193880 +193881 POINT(40.57566541744925 73.25100581792852) bank193881 +193882 POINT(40.40917037202191 73.67513914587695) bank193882 +193883 POINT(40.58114677531338 74.00676429859107) bank193883 +193884 POINT(39.88335505977056 73.996702054486) bank193884 +193885 POINT(41.05984092841696 74.41038618795113) bank193885 +193886 POINT(40.024416121787226 73.69322851780599) bank193886 +193887 POINT(41.00462664915147 74.20232241842291) bank193887 +193888 POINT(40.60577732092516 74.17895696225295) bank193888 +193889 POINT(41.05534511891787 73.5052879140473) bank193889 +193890 POINT(41.09140994941898 73.24368623443478) bank193890 +193891 POINT(41.70702730635065 74.08669793710227) bank193891 +193892 POINT(40.783517659352434 73.70038927330107) bank193892 +193893 POINT(40.65077053633705 73.69018428930192) bank193893 +193894 POINT(40.496053459060654 73.87752937246188) bank193894 +193895 POINT(41.69585072103778 73.02490658646269) bank193895 +193896 POINT(41.5776995134085 73.9820843411187) bank193896 +193897 POINT(41.03118524161993 74.40758728212018) bank193897 +193898 POINT(40.73831229917745 73.07522075933596) bank193898 +193899 POINT(41.581759814696206 73.17460334539602) bank193899 +193900 POINT(40.010372963907024 74.75775045553823) bank193900 +193901 POINT(40.36699894294509 73.4530336384047) bank193901 +193902 POINT(40.28872805511969 73.87274746048922) bank193902 +193903 POINT(41.262008339530716 74.74760262599595) bank193903 +193904 POINT(41.43267027749916 73.69623670543488) bank193904 +193905 POINT(41.42365397828883 73.48611068258842) bank193905 +193906 POINT(40.21722002425147 73.49449139207944) bank193906 +193907 POINT(40.19027233571749 74.89527851160258) bank193907 +193908 POINT(40.25169136476457 74.23764636593198) bank193908 +193909 POINT(40.203656437838774 74.82542643872091) bank193909 +193910 POINT(40.87916360023839 74.44050869435546) bank193910 +193911 POINT(40.5372038159464 74.77095964455225) bank193911 +193912 POINT(39.71798390649583 74.00958393359778) bank193912 +193913 POINT(40.04477118611519 73.81963907054195) bank193913 +193914 POINT(40.08260200706129 73.38095812649205) bank193914 +193915 POINT(41.603244926217286 73.65424604047838) bank193915 +193916 POINT(41.37839705852382 73.60729369657872) bank193916 +193917 POINT(40.940339566453424 74.24601023564551) bank193917 +193918 POINT(41.55405210415594 73.86843324265709) bank193918 +193919 POINT(40.92114317807207 74.63224731850875) bank193919 +193920 POINT(40.292081684993605 74.53063462378678) bank193920 +193921 POINT(40.6323300402124 73.74111544239274) bank193921 +193922 POINT(40.762632207383504 73.51527316497798) bank193922 +193923 POINT(40.31336815065504 74.61674914488928) bank193923 +193924 POINT(40.66904240107476 74.67605043534175) bank193924 +193925 POINT(40.03246426132363 74.27346486422752) bank193925 +193926 POINT(40.642857669252265 73.89282785252149) bank193926 +193927 POINT(41.254283857332425 73.78982553517817) bank193927 +193928 POINT(40.98744372023501 73.57322650150329) bank193928 +193929 POINT(41.60352231098135 74.72408447289112) bank193929 +193930 POINT(41.64529964488308 73.97577466254494) bank193930 +193931 POINT(40.853824262596575 74.39873827154578) bank193931 +193932 POINT(40.19540491990584 74.06085744764584) bank193932 +193933 POINT(39.870258013464856 73.16069312702095) bank193933 +193934 POINT(40.112620829837056 74.97817244756659) bank193934 +193935 POINT(40.36135606827938 74.90570915360847) bank193935 +193936 POINT(40.27148232961201 73.81016579603521) bank193936 +193937 POINT(39.81951135165657 73.02417643741015) bank193937 +193938 POINT(40.66401028028156 73.97767523676785) bank193938 +193939 POINT(41.23392373482501 74.95235415379871) bank193939 +193940 POINT(40.33714195197922 73.70117804544829) bank193940 +193941 POINT(40.3359480599982 74.90829869776367) bank193941 +193942 POINT(40.15725661861627 73.26061711404721) bank193942 +193943 POINT(41.487827381968195 73.47997762651708) bank193943 +193944 POINT(41.08670437599047 74.4724976883986) bank193944 +193945 POINT(40.6703628089566 73.68120868068169) bank193945 +193946 POINT(40.4117565597628 73.83043143899832) bank193946 +193947 POINT(41.29862761777917 74.1082974862915) bank193947 +193948 POINT(39.71964212752692 74.77088338298353) bank193948 +193949 POINT(40.53851748639622 73.654089205991) bank193949 +193950 POINT(40.90778667932068 73.22068893917877) bank193950 +193951 POINT(41.073553923983894 74.08310420090056) bank193951 +193952 POINT(39.99923483298841 73.45303894143102) bank193952 +193953 POINT(41.665636238610716 74.06475476157186) bank193953 +193954 POINT(40.590032673016744 73.5733045020492) bank193954 +193955 POINT(40.13042495459042 74.68669095766418) bank193955 +193956 POINT(41.1977905526524 74.92208692525833) bank193956 +193957 POINT(41.61130059751638 73.54893803655855) bank193957 +193958 POINT(40.1254941338957 73.3110739360082) bank193958 +193959 POINT(40.11011131947156 73.68184924227248) bank193959 +193960 POINT(40.37200164050992 74.20035940940548) bank193960 +193961 POINT(41.278847234330165 73.67139763893468) bank193961 +193962 POINT(41.40585236144099 73.51211899765103) bank193962 +193963 POINT(40.240291720286336 74.49779606826121) bank193963 +193964 POINT(40.86920106235607 73.71275467705588) bank193964 +193965 POINT(40.354601882379484 73.02768964046312) bank193965 +193966 POINT(40.101417240510116 74.4971319858565) bank193966 +193967 POINT(40.5143876163731 74.11036168781237) bank193967 +193968 POINT(39.85689539869431 73.40071967758163) bank193968 +193969 POINT(39.88051053247495 73.21765052395438) bank193969 +193970 POINT(40.643096753400336 74.65292599540223) bank193970 +193971 POINT(41.69914091128336 74.15679576776766) bank193971 +193972 POINT(40.01192880754844 74.38626308767324) bank193972 +193973 POINT(40.965188948597664 74.62029709583814) bank193973 +193974 POINT(40.50776834934877 73.11273024681101) bank193974 +193975 POINT(39.87349770155547 74.68569929410808) bank193975 +193976 POINT(40.87285813845925 73.67107115354973) bank193976 +193977 POINT(41.57678149642511 73.45057249676408) bank193977 +193978 POINT(41.1224588349144 73.37787177776045) bank193978 +193979 POINT(40.017112192264975 74.21452640167674) bank193979 +193980 POINT(40.38888139832014 74.81379824637993) bank193980 +193981 POINT(41.33270702774637 73.9205226569513) bank193981 +193982 POINT(40.09214464711196 74.16436285419347) bank193982 +193983 POINT(40.56922227965238 74.71568403168382) bank193983 +193984 POINT(40.966781751968156 73.37065351816857) bank193984 +193985 POINT(41.4954036193852 74.62501005785133) bank193985 +193986 POINT(40.7638831207225 73.96849366837542) bank193986 +193987 POINT(39.79770202338257 74.13221861545834) bank193987 +193988 POINT(41.60513525514596 74.62923477499052) bank193988 +193989 POINT(41.56132701933828 74.1545523253718) bank193989 +193990 POINT(39.7771957279588 74.98455429499202) bank193990 +193991 POINT(40.41058966136524 74.52782974677709) bank193991 +193992 POINT(41.4877527438777 74.48683271662665) bank193992 +193993 POINT(41.11938045209015 74.82128193421009) bank193993 +193994 POINT(39.88588659911085 74.70619911934872) bank193994 +193995 POINT(40.76896622914674 74.08687144886966) bank193995 +193996 POINT(40.36258833376949 73.76073277888426) bank193996 +193997 POINT(40.44280474213599 73.13982094967876) bank193997 +193998 POINT(39.97361785217604 74.49009339411455) bank193998 +193999 POINT(41.23432285937233 73.91122620407309) bank193999 +194000 POINT(40.48897766590356 73.86780144321196) bank194000 +194001 POINT(41.71158752545625 74.4777690164902) bank194001 +194002 POINT(41.26338167366654 74.76045058570752) bank194002 +194003 POINT(40.124649031654826 74.26535304855447) bank194003 +194004 POINT(41.57033887675863 74.05964736467322) bank194004 +194005 POINT(39.99545673904453 73.84117369941681) bank194005 +194006 POINT(40.30493893546303 74.62449849041887) bank194006 +194007 POINT(41.1956338074376 74.88734252765123) bank194007 +194008 POINT(39.71449065759108 73.53279088854592) bank194008 +194009 POINT(40.008301925103204 73.380930141554) bank194009 +194010 POINT(40.62089438658884 74.00897365548936) bank194010 +194011 POINT(41.18942853859565 73.52729022825109) bank194011 +194012 POINT(40.33758907777711 74.7017007254747) bank194012 +194013 POINT(39.758735657502285 74.23311259560973) bank194013 +194014 POINT(41.052385415645084 73.17564025719768) bank194014 +194015 POINT(41.65708620308649 73.60168296394299) bank194015 +194016 POINT(40.29550257133861 74.15260021288313) bank194016 +194017 POINT(40.96924798392589 73.37600754177576) bank194017 +194018 POINT(40.356626670059235 74.89782505030466) bank194018 +194019 POINT(40.61533442464428 74.53378859322562) bank194019 +194020 POINT(41.09212142110353 74.19114761008358) bank194020 +194021 POINT(40.01195053409783 73.80472249761408) bank194021 +194022 POINT(40.4901028664107 74.87481547841674) bank194022 +194023 POINT(40.950723863576336 73.50288681992619) bank194023 +194024 POINT(40.70543052926422 73.53457586124584) bank194024 +194025 POINT(41.250921261486425 74.73254131821594) bank194025 +194026 POINT(39.719163955158706 74.31021747423458) bank194026 +194027 POINT(40.38946235474337 74.07302099738415) bank194027 +194028 POINT(40.280951340158296 73.89542329895541) bank194028 +194029 POINT(41.2010682199563 74.30013428106125) bank194029 +194030 POINT(40.39651881173993 74.44000399513847) bank194030 +194031 POINT(40.86481420776814 74.04609618529474) bank194031 +194032 POINT(40.86465954012311 73.01545543674938) bank194032 +194033 POINT(40.30725729911873 73.17622058768025) bank194033 +194034 POINT(40.788559388311015 73.25737097663533) bank194034 +194035 POINT(41.606342583550386 73.97339854528116) bank194035 +194036 POINT(41.104607039085124 74.92461874832486) bank194036 +194037 POINT(40.10287009447182 73.79167489459202) bank194037 +194038 POINT(41.12138068910569 74.48408928363743) bank194038 +194039 POINT(40.29142877908975 73.57716705720398) bank194039 +194040 POINT(41.14425280753486 73.59821450367792) bank194040 +194041 POINT(39.8584223274013 74.09383110528883) bank194041 +194042 POINT(41.19388863648704 73.0474937555651) bank194042 +194043 POINT(41.6568243139857 74.6312800024228) bank194043 +194044 POINT(40.56899444752274 73.62575151372951) bank194044 +194045 POINT(39.899742325963906 74.99089319955877) bank194045 +194046 POINT(41.46577127175461 74.8227455360416) bank194046 +194047 POINT(40.02131118184269 73.69442207039019) bank194047 +194048 POINT(41.17932717292404 73.97036976432905) bank194048 +194049 POINT(40.13654958080552 73.92272189183275) bank194049 +194050 POINT(40.792285908045955 74.66601270303741) bank194050 +194051 POINT(40.45061827625577 73.09218819093604) bank194051 +194052 POINT(40.82788567921199 73.21321329121562) bank194052 +194053 POINT(40.03555050101694 74.98053181576161) bank194053 +194054 POINT(41.292248477128545 74.36297965641424) bank194054 +194055 POINT(41.52490571731814 73.11893929576402) bank194055 +194056 POINT(40.34129870128804 73.34591449790771) bank194056 +194057 POINT(39.77470732661295 74.49095673016474) bank194057 +194058 POINT(41.219696838880424 74.6564096310378) bank194058 +194059 POINT(39.73458509163936 73.98487573677393) bank194059 +194060 POINT(39.882360887485675 74.3382279332697) bank194060 +194061 POINT(40.27789083513774 74.86540836628905) bank194061 +194062 POINT(41.07001552030096 73.1601273052381) bank194062 +194063 POINT(40.07062194355038 73.81037930719627) bank194063 +194064 POINT(40.0460788192827 73.6615456909511) bank194064 +194065 POINT(39.74349524358605 73.82924471104619) bank194065 +194066 POINT(40.781560648539354 74.71916499262565) bank194066 +194067 POINT(40.90693863126573 74.82366583887513) bank194067 +194068 POINT(39.85003183021689 74.4098957710996) bank194068 +194069 POINT(41.69767705217115 73.52738434791993) bank194069 +194070 POINT(40.81090319570644 74.08087162821575) bank194070 +194071 POINT(40.389074783236 73.58495971132801) bank194071 +194072 POINT(41.547307328057556 74.73714355093624) bank194072 +194073 POINT(40.90027616048252 73.49704957214094) bank194073 +194074 POINT(40.676975920528086 73.99989354955864) bank194074 +194075 POINT(39.784068220523636 74.45502173080045) bank194075 +194076 POINT(40.052341088800134 74.45212337710468) bank194076 +194077 POINT(41.68202518636075 73.8908451814759) bank194077 +194078 POINT(39.938564047175866 74.49961780603849) bank194078 +194079 POINT(40.32001383120292 74.94835064086689) bank194079 +194080 POINT(40.76848963806797 74.84411452356422) bank194080 +194081 POINT(41.138781121044474 74.77117101618016) bank194081 +194082 POINT(40.846852325770854 73.99934264944557) bank194082 +194083 POINT(41.451238309723394 73.97227446253814) bank194083 +194084 POINT(41.302350982173095 74.16830376305145) bank194084 +194085 POINT(40.302174699576895 73.46280960399355) bank194085 +194086 POINT(41.61223103961126 73.35525564885634) bank194086 +194087 POINT(39.91640532770367 73.53973167685457) bank194087 +194088 POINT(40.92676947640594 74.32697084125574) bank194088 +194089 POINT(39.92055681926496 74.40423698030676) bank194089 +194090 POINT(40.94829189141185 73.40743215704421) bank194090 +194091 POINT(40.531411434644596 74.69942721347127) bank194091 +194092 POINT(40.80254775267781 73.51345654308203) bank194092 +194093 POINT(41.70439536238311 74.75259311034048) bank194093 +194094 POINT(40.13569241164452 73.32725475381338) bank194094 +194095 POINT(41.27422330991797 74.72473938094093) bank194095 +194096 POINT(41.38588602913092 73.69545327458637) bank194096 +194097 POINT(40.938123796865725 74.34295182896459) bank194097 +194098 POINT(40.304431946526385 73.99122667588315) bank194098 +194099 POINT(40.793438487076074 73.59704372291861) bank194099 +194100 POINT(40.612267044708446 73.6530116301005) bank194100 +194101 POINT(40.242595267976846 73.63995822325404) bank194101 +194102 POINT(39.96140605455807 73.75467381043205) bank194102 +194103 POINT(40.87765174015858 74.01019737753687) bank194103 +194104 POINT(39.77802604921523 74.18175927535009) bank194104 +194105 POINT(41.07438643522251 74.28619061410512) bank194105 +194106 POINT(41.328382074736226 73.74401054440793) bank194106 +194107 POINT(40.88330190805895 73.16296648477068) bank194107 +194108 POINT(40.101268605504025 73.60505113754786) bank194108 +194109 POINT(39.939104138681884 74.4006488770293) bank194109 +194110 POINT(40.58410703197714 73.4232179081587) bank194110 +194111 POINT(40.12486390923478 74.3310894103664) bank194111 +194112 POINT(40.254542949674104 73.51129234584447) bank194112 +194113 POINT(40.97942155075052 73.04648736104325) bank194113 +194114 POINT(40.535679532933315 74.407627023126) bank194114 +194115 POINT(41.60743505218459 74.0123378502439) bank194115 +194116 POINT(41.544255631500526 74.0414325930402) bank194116 +194117 POINT(41.55834494755922 74.97934825031203) bank194117 +194118 POINT(41.18652171818165 74.89609777578731) bank194118 +194119 POINT(40.705917907648484 74.16608969834131) bank194119 +194120 POINT(41.47522146160458 74.20663359060137) bank194120 +194121 POINT(40.86440439721653 73.84025339057052) bank194121 +194122 POINT(41.70078352560073 74.7940379652342) bank194122 +194123 POINT(41.20951832547896 75.00358283635771) bank194123 +194124 POINT(41.05932808676388 73.56158433930314) bank194124 +194125 POINT(41.70615883389676 74.05086902398612) bank194125 +194126 POINT(41.45632882539353 74.15567202895977) bank194126 +194127 POINT(40.78148361268181 74.49551959957633) bank194127 +194128 POINT(41.29245571840232 73.93589061713223) bank194128 +194129 POINT(40.27152002830634 73.72451087175986) bank194129 +194130 POINT(40.07956265291147 74.48505433708107) bank194130 +194131 POINT(41.06592095871984 74.76171852606612) bank194131 +194132 POINT(41.61889697098012 74.01989992612211) bank194132 +194133 POINT(39.84889664277936 73.8241570696817) bank194133 +194134 POINT(40.0273565400242 74.25000890352129) bank194134 +194135 POINT(41.30718984418937 74.86290881275741) bank194135 +194136 POINT(41.3537606697874 74.16246917142512) bank194136 +194137 POINT(41.633302960514484 73.42425791774093) bank194137 +194138 POINT(41.63884583331389 74.75614289608487) bank194138 +194139 POINT(39.82875963912664 73.1098006102814) bank194139 +194140 POINT(41.23231820136772 74.99153693898973) bank194140 +194141 POINT(40.26644513553667 73.98263727552003) bank194141 +194142 POINT(40.565252200211 73.04072980036536) bank194142 +194143 POINT(41.18879503063418 73.45214904854744) bank194143 +194144 POINT(41.40672399406111 74.69729648320897) bank194144 +194145 POINT(41.68072603289969 73.84718919525622) bank194145 +194146 POINT(40.00112263355768 74.42111691183416) bank194146 +194147 POINT(41.199963259395574 73.22158801267646) bank194147 +194148 POINT(40.90579761400638 74.03219576323625) bank194148 +194149 POINT(41.57163641334069 73.40202056507181) bank194149 +194150 POINT(40.06172662537139 73.31868319383138) bank194150 +194151 POINT(40.062420474868325 73.43628752989355) bank194151 +194152 POINT(41.66785560976321 73.66609116757823) bank194152 +194153 POINT(39.820626077207436 73.80770579049616) bank194153 +194154 POINT(41.039705844355616 74.38914920054674) bank194154 +194155 POINT(39.851309913122726 73.42381407454882) bank194155 +194156 POINT(39.901677505382125 73.17348670699266) bank194156 +194157 POINT(39.86850712859338 73.95752920601916) bank194157 +194158 POINT(40.74423969505761 74.7259253517798) bank194158 +194159 POINT(40.062832945123986 73.09428881989918) bank194159 +194160 POINT(40.534650435696435 73.65597226643914) bank194160 +194161 POINT(39.82411005582256 74.58849780764746) bank194161 +194162 POINT(40.06210698582867 73.8078209068419) bank194162 +194163 POINT(40.80442293421312 74.5718524592778) bank194163 +194164 POINT(39.80453097727252 74.07136783869974) bank194164 +194165 POINT(40.50089793269891 74.70951963720545) bank194165 +194166 POINT(41.19412822165425 73.62738060706792) bank194166 +194167 POINT(41.4725442403912 73.47402833946241) bank194167 +194168 POINT(40.557251742466434 74.55413346941252) bank194168 +194169 POINT(40.02372971911249 74.72449523376552) bank194169 +194170 POINT(41.12886026507431 74.38439440584362) bank194170 +194171 POINT(40.085074156893945 73.35772407995805) bank194171 +194172 POINT(40.86059379808829 73.90956174885116) bank194172 +194173 POINT(41.42553723399226 74.1416510099408) bank194173 +194174 POINT(41.52532815472589 75.00549149789117) bank194174 +194175 POINT(41.07943271799637 73.72433885838498) bank194175 +194176 POINT(39.99511958471797 74.73466367918753) bank194176 +194177 POINT(40.06691694431057 74.70056724096402) bank194177 +194178 POINT(40.921162352664375 73.23608548162474) bank194178 +194179 POINT(40.51896844060571 73.83743439563167) bank194179 +194180 POINT(40.24011287691305 73.91681669249323) bank194180 +194181 POINT(41.39130299994114 73.14162848801914) bank194181 +194182 POINT(40.93364362965373 73.39831696409848) bank194182 +194183 POINT(41.23254022448264 73.49253867854839) bank194183 +194184 POINT(40.2721245967583 73.89806229765195) bank194184 +194185 POINT(40.7797408763239 73.45358237979545) bank194185 +194186 POINT(40.052342426190194 73.2658007492384) bank194186 +194187 POINT(41.592869481959994 73.08993562864221) bank194187 +194188 POINT(39.95739677556254 74.19232202872725) bank194188 +194189 POINT(40.192301741283295 73.161961264484) bank194189 +194190 POINT(40.71869405439606 73.18480057653943) bank194190 +194191 POINT(39.74121911050745 73.76219420703966) bank194191 +194192 POINT(39.95636562192336 73.05670415702502) bank194192 +194193 POINT(39.84370855206924 73.25701690599337) bank194193 +194194 POINT(40.27214011177471 74.35373074118331) bank194194 +194195 POINT(39.80695825617332 74.93881705710012) bank194195 +194196 POINT(40.88118555043291 74.20405267254549) bank194196 +194197 POINT(41.52371094741148 73.93527638949978) bank194197 +194198 POINT(41.62724498790564 74.02331607906974) bank194198 +194199 POINT(40.91763550005919 73.19741016533301) bank194199 +194200 POINT(40.82596705314153 73.9911483212871) bank194200 +194201 POINT(41.53980437498342 74.94431643301984) bank194201 +194202 POINT(40.22957806787066 74.37831031238997) bank194202 +194203 POINT(40.48199881947342 73.6316470240334) bank194203 +194204 POINT(40.434031765212445 74.0558729153836) bank194204 +194205 POINT(40.459880223961676 73.5379685037064) bank194205 +194206 POINT(40.608462185647404 73.67013287059473) bank194206 +194207 POINT(39.81391323156183 74.66700969484849) bank194207 +194208 POINT(40.62644256765636 74.85912069571981) bank194208 +194209 POINT(40.81667820671287 73.09479632275449) bank194209 +194210 POINT(39.731990376360926 74.5717555882572) bank194210 +194211 POINT(39.816358170224305 74.05057913686849) bank194211 +194212 POINT(40.73613106076239 73.87282206001073) bank194212 +194213 POINT(40.21886875815704 74.21395453463735) bank194213 +194214 POINT(40.02973138222415 74.4011636251648) bank194214 +194215 POINT(40.79770054208982 73.37890784017637) bank194215 +194216 POINT(41.51668835957627 74.99765504357859) bank194216 +194217 POINT(41.05287641493686 73.23727563414171) bank194217 +194218 POINT(39.7807885703028 73.7865760651568) bank194218 +194219 POINT(41.232718460262205 73.34710415378439) bank194219 +194220 POINT(40.44774755017695 74.97644673707616) bank194220 +194221 POINT(40.612223010989865 73.70818750768137) bank194221 +194222 POINT(39.88312175300642 74.66023227562701) bank194222 +194223 POINT(39.99483404168628 73.82169253943194) bank194223 +194224 POINT(41.4738027595328 73.27847845767802) bank194224 +194225 POINT(40.411458632289474 74.10156845467426) bank194225 +194226 POINT(41.05075056752638 73.37461751078577) bank194226 +194227 POINT(39.87040244177469 73.98534057150538) bank194227 +194228 POINT(40.92075746595553 74.3009640869758) bank194228 +194229 POINT(41.680211036809425 74.64805633984865) bank194229 +194230 POINT(39.87256609072964 73.07520090939589) bank194230 +194231 POINT(41.52787253055656 74.01052596953062) bank194231 +194232 POINT(39.790478341260034 74.66488536644857) bank194232 +194233 POINT(39.890349642575735 74.6058197349266) bank194233 +194234 POINT(40.048212619075755 74.0588831367573) bank194234 +194235 POINT(41.34644155266708 74.89837675317288) bank194235 +194236 POINT(41.24165928319099 73.81708079532675) bank194236 +194237 POINT(40.50532391090693 73.2367485400624) bank194237 +194238 POINT(40.83747049762747 74.4197771211558) bank194238 +194239 POINT(41.03080898123934 74.95510018914493) bank194239 +194240 POINT(41.303234118369865 74.77496121648156) bank194240 +194241 POINT(41.35811737249529 73.06636491812915) bank194241 +194242 POINT(41.483863424279185 74.26490673818856) bank194242 +194243 POINT(39.99271964080359 73.8275476115394) bank194243 +194244 POINT(39.78752966683748 73.67012463801524) bank194244 +194245 POINT(40.17874669132392 73.35638888197867) bank194245 +194246 POINT(39.934884062923565 74.59162010868495) bank194246 +194247 POINT(39.73577078741124 74.37836543041453) bank194247 +194248 POINT(41.60341793334096 74.48235090599351) bank194248 +194249 POINT(40.04209673179457 73.34417300219759) bank194249 +194250 POINT(40.817612668434506 73.85660843880399) bank194250 +194251 POINT(40.87350563832398 74.78573911103051) bank194251 +194252 POINT(40.16356388637885 73.16236616348547) bank194252 +194253 POINT(41.69702355864379 74.75815879334685) bank194253 +194254 POINT(40.57406450328041 74.70576672883776) bank194254 +194255 POINT(40.42079096800832 73.92466946600977) bank194255 +194256 POINT(39.84195857515494 74.64906510850027) bank194256 +194257 POINT(41.580559939472124 74.86507179169456) bank194257 +194258 POINT(40.15176224561933 73.04880556609919) bank194258 +194259 POINT(40.51251316621663 73.46162819995077) bank194259 +194260 POINT(41.137127953435176 74.90926838394121) bank194260 +194261 POINT(41.53007289227565 73.86205226631937) bank194261 +194262 POINT(41.51834520514102 73.63896919336321) bank194262 +194263 POINT(41.28515901193358 73.89886836117508) bank194263 +194264 POINT(41.33990875608289 73.04150659712649) bank194264 +194265 POINT(40.61674253928661 73.29191029745218) bank194265 +194266 POINT(40.42676031440026 73.58495993186392) bank194266 +194267 POINT(41.19545422397426 73.21038916793808) bank194267 +194268 POINT(40.127370861649744 73.09441428248377) bank194268 +194269 POINT(39.87563518248612 73.38984872190873) bank194269 +194270 POINT(41.52874492173615 74.54304141534917) bank194270 +194271 POINT(39.96366689120328 74.95184209598904) bank194271 +194272 POINT(41.41806822322532 73.38283836080275) bank194272 +194273 POINT(40.435700331304886 73.42243840047048) bank194273 +194274 POINT(40.31256668516688 74.27952661919745) bank194274 +194275 POINT(40.267592825676665 73.98742045120582) bank194275 +194276 POINT(40.47682206505611 74.53069633163366) bank194276 +194277 POINT(39.80111911610762 73.39931360497528) bank194277 +194278 POINT(40.493817296113875 74.8247127442022) bank194278 +194279 POINT(41.49382201416839 74.68379123082448) bank194279 +194280 POINT(40.3898986348333 73.93047785573185) bank194280 +194281 POINT(41.43751570201766 74.68427331502042) bank194281 +194282 POINT(41.090798209890124 73.73564045462955) bank194282 +194283 POINT(41.604613251520746 73.20569698330586) bank194283 +194284 POINT(40.377520378228716 73.35341119890198) bank194284 +194285 POINT(41.10002666401632 74.67178049656671) bank194285 +194286 POINT(39.879944709435925 74.83097692912618) bank194286 +194287 POINT(40.12483303033009 74.88149960209796) bank194287 +194288 POINT(41.380382084491984 73.38148165827413) bank194288 +194289 POINT(40.43392080357054 73.13033405962182) bank194289 +194290 POINT(41.1242569223193 73.84627207887957) bank194290 +194291 POINT(39.848254421604 74.58915957341806) bank194291 +194292 POINT(39.78614936500538 73.69145859695594) bank194292 +194293 POINT(41.26090646899132 73.19634387171243) bank194293 +194294 POINT(40.336937062828724 74.55992086494159) bank194294 +194295 POINT(41.29323116271292 73.09735884312596) bank194295 +194296 POINT(41.58878089723153 73.94590729911542) bank194296 +194297 POINT(41.55275650373306 74.9904956574546) bank194297 +194298 POINT(41.286365659982856 73.55036989259497) bank194298 +194299 POINT(41.568444032429845 73.36577578333609) bank194299 +194300 POINT(40.10459950064095 74.66280346716816) bank194300 +194301 POINT(40.27441672179001 73.9323129453777) bank194301 +194302 POINT(40.80146605619076 73.30365584256124) bank194302 +194303 POINT(40.2351244179811 73.63454447796049) bank194303 +194304 POINT(40.737190044987564 74.38566086928228) bank194304 +194305 POINT(41.02038703421359 74.3914210226507) bank194305 +194306 POINT(41.68758739652221 74.1179143641571) bank194306 +194307 POINT(41.2345296616541 73.98751337375357) bank194307 +194308 POINT(41.14353921182347 73.93039644557895) bank194308 +194309 POINT(41.482783396754954 74.45722525678815) bank194309 +194310 POINT(41.322401129518745 74.84583329239848) bank194310 +194311 POINT(39.758736978104864 73.24563222730059) bank194311 +194312 POINT(40.1635257230722 73.85301170945533) bank194312 +194313 POINT(40.805114324569686 73.4152225415429) bank194313 +194314 POINT(41.17200667987404 74.52815692589996) bank194314 +194315 POINT(39.94144946720027 73.09112762755032) bank194315 +194316 POINT(40.34742971983363 73.57895246319957) bank194316 +194317 POINT(39.90593972970792 73.67317701714272) bank194317 +194318 POINT(41.03676781445646 74.5493876884088) bank194318 +194319 POINT(40.90594179512861 73.37663772078832) bank194319 +194320 POINT(40.83760355680486 73.83133363043586) bank194320 +194321 POINT(41.58150143380706 73.31592758278842) bank194321 +194322 POINT(41.60965959093337 73.8699175791064) bank194322 +194323 POINT(40.204784654481564 74.74054689061909) bank194323 +194324 POINT(40.43955760557003 74.55046739671579) bank194324 +194325 POINT(40.9335503103339 73.15291337070808) bank194325 +194326 POINT(40.49713014391692 73.85018006894875) bank194326 +194327 POINT(40.14587160256752 74.99414197977372) bank194327 +194328 POINT(41.08960335097345 73.0448258386999) bank194328 +194329 POINT(40.55974530752341 74.50645620986194) bank194329 +194330 POINT(41.66359344962204 73.54811822706075) bank194330 +194331 POINT(41.1609031093549 74.69015220479626) bank194331 +194332 POINT(40.334789831877345 73.2033778045672) bank194332 +194333 POINT(41.426609337273185 74.54272124660875) bank194333 +194334 POINT(39.93761948099495 74.45392050424275) bank194334 +194335 POINT(40.74924048975549 73.53605073515503) bank194335 +194336 POINT(40.99183520679582 74.06161280866384) bank194336 +194337 POINT(41.377669066863625 74.7750411549633) bank194337 +194338 POINT(40.35618548997663 74.5348518366804) bank194338 +194339 POINT(41.595697683015985 74.77513266360192) bank194339 +194340 POINT(40.80710029489615 73.3372805441582) bank194340 +194341 POINT(39.96854784081268 74.57526733751845) bank194341 +194342 POINT(40.297214917711926 73.47257258416752) bank194342 +194343 POINT(39.980680357450275 73.92947516476585) bank194343 +194344 POINT(40.192891803581105 73.06106731044903) bank194344 +194345 POINT(40.67280820468902 74.46753776624577) bank194345 +194346 POINT(40.49908050932468 73.56671769842737) bank194346 +194347 POINT(39.928329163202136 73.73741141943471) bank194347 +194348 POINT(40.85321976823742 74.01702564469595) bank194348 +194349 POINT(40.37727246231465 74.5640205562274) bank194349 +194350 POINT(40.51795191521941 73.07955718852152) bank194350 +194351 POINT(40.19930171150213 74.90541002204085) bank194351 +194352 POINT(40.31842518580553 74.43292272758073) bank194352 +194353 POINT(41.404080070182836 75.00345360949888) bank194353 +194354 POINT(40.44100322255464 73.22075398250223) bank194354 +194355 POINT(41.21543705621404 74.15165634228985) bank194355 +194356 POINT(40.39073171958409 74.38660563138839) bank194356 +194357 POINT(41.29462309386609 74.61737719529965) bank194357 +194358 POINT(41.167968904001185 73.42508810020406) bank194358 +194359 POINT(41.13402642787506 73.41856091204302) bank194359 +194360 POINT(40.622739043248814 74.24219130467664) bank194360 +194361 POINT(40.97061972456849 74.83938735719838) bank194361 +194362 POINT(41.2938337354258 73.78978499606251) bank194362 +194363 POINT(41.366913576536476 74.1146000623579) bank194363 +194364 POINT(41.39517071920671 73.8572126529248) bank194364 +194365 POINT(39.80491242537117 73.31202703401256) bank194365 +194366 POINT(40.778489212657064 73.16785618634341) bank194366 +194367 POINT(41.128986701968536 74.14402755155722) bank194367 +194368 POINT(40.22428338599611 74.8739421828519) bank194368 +194369 POINT(41.023637280504914 74.2187947018896) bank194369 +194370 POINT(40.90297812842667 73.49036144585659) bank194370 +194371 POINT(41.60413398230831 74.02515989236407) bank194371 +194372 POINT(41.30123548875349 74.08129652298305) bank194372 +194373 POINT(41.167038200700915 73.61006760672358) bank194373 +194374 POINT(40.59030861214526 74.83645130070042) bank194374 +194375 POINT(41.081057946866025 73.11686520989235) bank194375 +194376 POINT(40.26098999645364 74.61345178389111) bank194376 +194377 POINT(41.099036630713854 74.42221680673266) bank194377 +194378 POINT(40.18152007467814 73.85951600994007) bank194378 +194379 POINT(40.3518954641221 74.31728147400696) bank194379 +194380 POINT(40.69467066372042 74.96393279333472) bank194380 +194381 POINT(40.068430075242354 73.07153875716494) bank194381 +194382 POINT(41.27909252680787 73.8482804029472) bank194382 +194383 POINT(41.28177284708973 74.94026281198788) bank194383 +194384 POINT(40.26212799205052 73.6956972502319) bank194384 +194385 POINT(39.995861425383126 74.70497384538609) bank194385 +194386 POINT(40.08542833849659 74.17982304742863) bank194386 +194387 POINT(41.32448236361301 74.33890354776328) bank194387 +194388 POINT(41.00731866355781 74.05948180051733) bank194388 +194389 POINT(39.788356225283046 73.4746309694677) bank194389 +194390 POINT(41.70224108528234 74.18239860021619) bank194390 +194391 POINT(41.54336704557658 74.27746276313108) bank194391 +194392 POINT(39.89078328557922 73.71573508031118) bank194392 +194393 POINT(39.785385121308494 73.9580128110282) bank194393 +194394 POINT(40.21812311128087 74.78890544273406) bank194394 +194395 POINT(40.15871318828111 74.76212326901675) bank194395 +194396 POINT(40.48298814624156 73.20900940646915) bank194396 +194397 POINT(41.39357006758117 73.39569836194667) bank194397 +194398 POINT(40.85986440754223 73.71126727348901) bank194398 +194399 POINT(41.15456740200003 73.10817895557368) bank194399 +194400 POINT(41.69652496891117 73.12655482432253) bank194400 +194401 POINT(41.64374019259729 73.6318647576973) bank194401 +194402 POINT(40.11887709993602 74.50390218544747) bank194402 +194403 POINT(40.07243161142435 73.11327853636483) bank194403 +194404 POINT(40.22602391027504 74.07980145306821) bank194404 +194405 POINT(40.75150859507128 74.13509439469634) bank194405 +194406 POINT(39.832442161941565 74.5037553176663) bank194406 +194407 POINT(41.46170751513911 74.50854651288644) bank194407 +194408 POINT(40.88782567167496 73.90508369694768) bank194408 +194409 POINT(41.24752234704295 73.44300320574831) bank194409 +194410 POINT(40.070848593112764 73.31725977748378) bank194410 +194411 POINT(40.29225318997851 73.19076239160498) bank194411 +194412 POINT(40.04765365197896 73.98577818891587) bank194412 +194413 POINT(40.978261990882594 73.22270191790753) bank194413 +194414 POINT(41.46915756776664 73.36496782776626) bank194414 +194415 POINT(40.106959251834624 74.96869910420914) bank194415 +194416 POINT(40.068408667901416 73.13091889045268) bank194416 +194417 POINT(41.24443302835726 73.98748712852213) bank194417 +194418 POINT(41.42616757676207 73.89031983541172) bank194418 +194419 POINT(41.02312482965861 74.3669485153393) bank194419 +194420 POINT(40.44193085919581 74.90098872515406) bank194420 +194421 POINT(41.6279732139299 74.76935925918065) bank194421 +194422 POINT(40.61713082365117 73.89257559820825) bank194422 +194423 POINT(40.38707731687646 73.53987839363872) bank194423 +194424 POINT(41.45366675745064 74.60098704614957) bank194424 +194425 POINT(41.566521755479314 74.92763971942861) bank194425 +194426 POINT(41.03484835906981 74.02258277690893) bank194426 +194427 POINT(41.21910093367704 73.17024267293608) bank194427 +194428 POINT(39.940770197646195 73.58477655084938) bank194428 +194429 POINT(40.088015850037856 73.17999355492667) bank194429 +194430 POINT(41.643990846425346 73.67746661512714) bank194430 +194431 POINT(40.4138573014276 73.672808386487) bank194431 +194432 POINT(40.41666050591923 73.08691158810731) bank194432 +194433 POINT(40.43172211024519 74.36562201112407) bank194433 +194434 POINT(41.01503743744328 73.74959836060738) bank194434 +194435 POINT(40.63926376589721 74.47884015062397) bank194435 +194436 POINT(40.83784209009608 73.01827126206338) bank194436 +194437 POINT(40.089641663199934 73.5506860534029) bank194437 +194438 POINT(40.79502326701799 73.34791256682004) bank194438 +194439 POINT(40.749752859274736 74.0997200844945) bank194439 +194440 POINT(40.75712995946459 73.72369599215119) bank194440 +194441 POINT(41.70557925437351 73.56560010156328) bank194441 +194442 POINT(40.79549231797055 74.01185969928197) bank194442 +194443 POINT(39.845394698431846 73.94882620116118) bank194443 +194444 POINT(41.53679055624735 73.09542295243241) bank194444 +194445 POINT(39.82090081900346 73.17458865892478) bank194445 +194446 POINT(39.922112713915865 74.29453521763755) bank194446 +194447 POINT(40.99817042311015 74.90620718308082) bank194447 +194448 POINT(41.207852418079504 74.61166319025854) bank194448 +194449 POINT(40.63678484373684 73.93091600043998) bank194449 +194450 POINT(41.524650485131644 74.06023029537518) bank194450 +194451 POINT(40.336663496035186 74.28202182257995) bank194451 +194452 POINT(40.59196058933311 74.36300913091802) bank194452 +194453 POINT(41.68925377014025 73.91986344368162) bank194453 +194454 POINT(39.91144492597059 73.469732283372) bank194454 +194455 POINT(40.873492018163915 73.94121062470565) bank194455 +194456 POINT(41.14268787047096 73.18149626321821) bank194456 +194457 POINT(40.047537704663824 73.59915073778808) bank194457 +194458 POINT(40.99779594651088 74.53840014293615) bank194458 +194459 POINT(41.358466542393906 74.47043827257036) bank194459 +194460 POINT(41.69390185300794 74.43833700234686) bank194460 +194461 POINT(40.351452382635316 74.20687064049805) bank194461 +194462 POINT(41.22921088700589 74.23552638832743) bank194462 +194463 POINT(39.87182464500889 73.03423466226543) bank194463 +194464 POINT(41.63615884727933 73.9466486630631) bank194464 +194465 POINT(39.99211800410042 74.1229299355533) bank194465 +194466 POINT(40.474055868058805 74.14506624272565) bank194466 +194467 POINT(40.9322040244697 73.13905910811768) bank194467 +194468 POINT(41.24826767412237 74.37836501463397) bank194468 +194469 POINT(41.23428231346263 74.4396408006836) bank194469 +194470 POINT(40.008535672878814 74.16215324906614) bank194470 +194471 POINT(40.35601753369066 73.8650932834619) bank194471 +194472 POINT(39.76469391132686 73.18982305874395) bank194472 +194473 POINT(41.24048541976118 74.43594717434522) bank194473 +194474 POINT(40.7270466796301 73.66743326342515) bank194474 +194475 POINT(40.97143697601433 73.40354812054548) bank194475 +194476 POINT(41.3666167878205 74.62477120144662) bank194476 +194477 POINT(41.6515405336772 74.19633666127098) bank194477 +194478 POINT(39.89390885785526 73.20420671287603) bank194478 +194479 POINT(40.16009511906134 73.69599652350476) bank194479 +194480 POINT(40.82708912411759 74.87266787290247) bank194480 +194481 POINT(40.35560608415919 73.13601373358426) bank194481 +194482 POINT(40.12842245175317 74.11080942560791) bank194482 +194483 POINT(40.12739307784131 74.19513123606164) bank194483 +194484 POINT(41.01932144183602 73.87161468368501) bank194484 +194485 POINT(41.18775392934118 73.0516730727793) bank194485 +194486 POINT(40.812227056929146 73.01679397347428) bank194486 +194487 POINT(40.32177587159481 73.6908592692672) bank194487 +194488 POINT(39.785392875531755 74.83876247422172) bank194488 +194489 POINT(41.296550727296186 73.5712355559451) bank194489 +194490 POINT(40.730354342227876 74.07307285621751) bank194490 +194491 POINT(41.19733776945621 74.18204108592013) bank194491 +194492 POINT(41.15276687073367 73.71367641796155) bank194492 +194493 POINT(40.36078548051385 73.46405636553936) bank194493 +194494 POINT(41.39983652878661 73.51637312209476) bank194494 +194495 POINT(41.685161944813565 73.62102893905927) bank194495 +194496 POINT(40.60870793576545 73.74808398981924) bank194496 +194497 POINT(41.225746384476835 74.370869157374) bank194497 +194498 POINT(40.57704129265824 73.37116629627313) bank194498 +194499 POINT(40.87732539663489 73.90354292347284) bank194499 +194500 POINT(41.60040715564928 73.5558952935856) bank194500 +194501 POINT(40.54342963722941 74.41678456599236) bank194501 +194502 POINT(40.34914192606443 73.31350591165945) bank194502 +194503 POINT(41.65254386596367 74.86479698948528) bank194503 +194504 POINT(40.10028427778636 74.61882963339576) bank194504 +194505 POINT(40.36922415818775 73.25625164630513) bank194505 +194506 POINT(39.77411895550689 73.10273836718339) bank194506 +194507 POINT(40.52550593753557 74.41745811272625) bank194507 +194508 POINT(39.936642994288796 74.63949864075347) bank194508 +194509 POINT(40.67414139519831 74.6092587918781) bank194509 +194510 POINT(40.337592175836065 73.18110232557008) bank194510 +194511 POINT(39.79351597701557 74.97869591923933) bank194511 +194512 POINT(39.80635031741492 73.49679594274859) bank194512 +194513 POINT(41.387457354608 74.52710341427631) bank194513 +194514 POINT(40.03767402178213 74.98369346335345) bank194514 +194515 POINT(39.97424552251295 74.40897689290416) bank194515 +194516 POINT(41.09079971963413 74.05069531085927) bank194516 +194517 POINT(41.54922787104089 73.65761970384227) bank194517 +194518 POINT(39.896659519317964 74.73789930132097) bank194518 +194519 POINT(40.90762700129349 73.65720692113955) bank194519 +194520 POINT(39.739146567005776 74.83377555612735) bank194520 +194521 POINT(40.11273002503309 74.36434856915025) bank194521 +194522 POINT(41.678863979025465 74.3208047041263) bank194522 +194523 POINT(40.61851671933253 73.5099345880691) bank194523 +194524 POINT(41.2529969185114 73.82339647599913) bank194524 +194525 POINT(40.2589525030079 74.72967408831278) bank194525 +194526 POINT(40.609543280830785 73.25664406015383) bank194526 +194527 POINT(40.98706245199606 73.86140891744634) bank194527 +194528 POINT(39.85401380269307 73.82413588887106) bank194528 +194529 POINT(40.89165575831947 73.9770824394145) bank194529 +194530 POINT(41.2155798757725 73.9834429248974) bank194530 +194531 POINT(41.209648894836334 74.16374002575559) bank194531 +194532 POINT(39.94101964896069 73.2262927973936) bank194532 +194533 POINT(40.23726546678341 74.84773081373255) bank194533 +194534 POINT(41.06604909988959 73.16382912693611) bank194534 +194535 POINT(40.87434231278675 73.14356248300804) bank194535 +194536 POINT(41.09179349527703 74.19181245775505) bank194536 +194537 POINT(39.96223824656698 73.06996417708005) bank194537 +194538 POINT(39.8847457283333 74.31994537863272) bank194538 +194539 POINT(40.583748591384555 74.64875155883563) bank194539 +194540 POINT(41.16173385886301 73.3439932430822) bank194540 +194541 POINT(40.05325505889977 74.86382710260031) bank194541 +194542 POINT(41.568002004800846 73.45856992503508) bank194542 +194543 POINT(39.786689465942416 74.76171040226889) bank194543 +194544 POINT(41.0733797646664 74.98913109725154) bank194544 +194545 POINT(41.53453720250157 74.23293674843171) bank194545 +194546 POINT(40.43205103818994 74.9584262310585) bank194546 +194547 POINT(41.555369392179585 74.07427572512297) bank194547 +194548 POINT(40.60937567918304 73.61620199603175) bank194548 +194549 POINT(40.55494691024813 74.60234986507709) bank194549 +194550 POINT(40.058251343168585 74.17123001470708) bank194550 +194551 POINT(40.72687577346118 73.56238799941033) bank194551 +194552 POINT(40.03887330558572 73.50336968680287) bank194552 +194553 POINT(40.444106272880305 74.59629520742985) bank194553 +194554 POINT(40.809737412166484 73.05270016686464) bank194554 +194555 POINT(39.84742707870284 73.91165908761859) bank194555 +194556 POINT(40.33365663640743 73.2399434145211) bank194556 +194557 POINT(41.45887878134449 74.14887795670622) bank194557 +194558 POINT(41.43305770728357 73.20001854321045) bank194558 +194559 POINT(41.42347370708165 73.04741910729298) bank194559 +194560 POINT(41.13436065277624 73.9411244834356) bank194560 +194561 POINT(41.58090278452387 74.3475558491587) bank194561 +194562 POINT(40.434450104038355 73.81391881228888) bank194562 +194563 POINT(40.49399514716012 74.92954153777416) bank194563 +194564 POINT(40.69149160075269 73.73769004056462) bank194564 +194565 POINT(39.78590208649616 73.298530700643) bank194565 +194566 POINT(41.29279510800974 73.98140452379428) bank194566 +194567 POINT(41.5069189761809 73.99533280673367) bank194567 +194568 POINT(40.60276065337916 73.69405715682264) bank194568 +194569 POINT(41.07970614220213 73.07159039231153) bank194569 +194570 POINT(41.037322947206874 74.79383722918297) bank194570 +194571 POINT(40.993077733253195 74.10068667092261) bank194571 +194572 POINT(41.44257885649969 74.77211906497166) bank194572 +194573 POINT(39.87150765047194 73.36440098423341) bank194573 +194574 POINT(39.731275859876156 73.49564059103145) bank194574 +194575 POINT(40.99277175601875 74.73093288868185) bank194575 +194576 POINT(40.04332749360846 74.58982736791265) bank194576 +194577 POINT(40.61152885100459 74.1293301480551) bank194577 +194578 POINT(41.43977604112211 74.96250190087598) bank194578 +194579 POINT(41.28956781176223 74.62993437907454) bank194579 +194580 POINT(39.93626867749831 74.33864284135394) bank194580 +194581 POINT(40.28426412869432 73.01854858617101) bank194581 +194582 POINT(41.35203089334727 74.90379459962837) bank194582 +194583 POINT(40.02093485827208 73.75760876515646) bank194583 +194584 POINT(40.03176841176605 74.36094036005889) bank194584 +194585 POINT(40.06579325485788 73.97460356003127) bank194585 +194586 POINT(41.422054606240245 73.15753080646836) bank194586 +194587 POINT(41.44758174895159 73.03144040646174) bank194587 +194588 POINT(40.30324110324209 74.45340550793213) bank194588 +194589 POINT(40.96452082419795 74.72431537360472) bank194589 +194590 POINT(41.48807830475102 74.00796397514439) bank194590 +194591 POINT(41.3712091418936 73.24481616176989) bank194591 +194592 POINT(40.27854574874972 73.4721405181191) bank194592 +194593 POINT(41.02893949003401 73.19794531873849) bank194593 +194594 POINT(40.64497861213041 73.63582198521759) bank194594 +194595 POINT(39.82788168215626 74.16665595527307) bank194595 +194596 POINT(40.14253241384652 74.44002010682554) bank194596 +194597 POINT(40.99335233369808 74.2977758998827) bank194597 +194598 POINT(41.134716989991915 74.39000569548622) bank194598 +194599 POINT(41.61827577038594 74.98836357522798) bank194599 +194600 POINT(40.234811379129056 74.1369326731158) bank194600 +194601 POINT(40.56609926823042 73.562650187749) bank194601 +194602 POINT(41.15817827569988 74.1676108572433) bank194602 +194603 POINT(41.44091062885229 74.10022330349068) bank194603 +194604 POINT(41.51481210040566 74.8618529569207) bank194604 +194605 POINT(41.69998416114914 74.61910485939316) bank194605 +194606 POINT(40.60858254606704 73.74365751020014) bank194606 +194607 POINT(40.078361793396155 74.92032404729872) bank194607 +194608 POINT(41.00089284241686 73.44227276959367) bank194608 +194609 POINT(41.517336226197365 74.39869493071858) bank194609 +194610 POINT(41.48726888651657 73.66402605305151) bank194610 +194611 POINT(40.68967327935883 74.3761327800107) bank194611 +194612 POINT(39.96797576515946 73.11391220120245) bank194612 +194613 POINT(40.697202488785805 73.95455269898831) bank194613 +194614 POINT(40.172916216357926 74.6198394634551) bank194614 +194615 POINT(39.92292703851828 74.86453597783594) bank194615 +194616 POINT(41.56408961662789 74.89967841986797) bank194616 +194617 POINT(40.790569207616166 74.06660035970613) bank194617 +194618 POINT(41.52054362457576 74.15372875361207) bank194618 +194619 POINT(40.3134434074038 73.0732556267902) bank194619 +194620 POINT(41.68406747203467 74.3434369375333) bank194620 +194621 POINT(40.29225010976027 73.49052457166275) bank194621 +194622 POINT(40.01687582003093 74.27719470030863) bank194622 +194623 POINT(40.300504638729336 74.56231466817653) bank194623 +194624 POINT(41.32825645174645 73.27283331281487) bank194624 +194625 POINT(39.93538537495236 74.80969860088372) bank194625 +194626 POINT(41.42453187906103 74.26049968365318) bank194626 +194627 POINT(41.54818129040969 73.60131506908428) bank194627 +194628 POINT(40.814575150105505 73.39326450347372) bank194628 +194629 POINT(39.74768937358037 74.17572282799487) bank194629 +194630 POINT(40.96098054610304 73.77190993498692) bank194630 +194631 POINT(40.26240949901319 74.78436317399327) bank194631 +194632 POINT(40.689214092715574 73.0942053318878) bank194632 +194633 POINT(41.10479640660701 74.89807940331042) bank194633 +194634 POINT(40.04612829504908 73.04510536224852) bank194634 +194635 POINT(41.09877598869629 74.67470610212337) bank194635 +194636 POINT(40.24191160411492 73.72042904528864) bank194636 +194637 POINT(41.694787402443076 73.34408851962836) bank194637 +194638 POINT(40.46270802333265 73.84500323300519) bank194638 +194639 POINT(40.6075356742019 74.00212810868952) bank194639 +194640 POINT(40.484013466500265 73.62704209641781) bank194640 +194641 POINT(41.42559946895243 73.5729699693441) bank194641 +194642 POINT(40.59592527726517 73.98353961543764) bank194642 +194643 POINT(39.79597446134952 74.91587524131324) bank194643 +194644 POINT(39.786026224071485 74.02575176331518) bank194644 +194645 POINT(39.9920578272035 73.20398217321336) bank194645 +194646 POINT(41.71088279034262 73.67410337781993) bank194646 +194647 POINT(41.41768441021852 74.82868627677645) bank194647 +194648 POINT(41.29205596733984 74.37877075764088) bank194648 +194649 POINT(40.8295191506631 73.12994514607408) bank194649 +194650 POINT(40.827010658067486 73.0461756573923) bank194650 +194651 POINT(40.44098880934751 74.95895585437422) bank194651 +194652 POINT(41.54960205147716 73.09785483839484) bank194652 +194653 POINT(41.692118616313934 74.0659412721211) bank194653 +194654 POINT(39.774529171148835 74.45428353901427) bank194654 +194655 POINT(41.24645396991515 74.41593207327125) bank194655 +194656 POINT(41.22610467891902 74.96902667233236) bank194656 +194657 POINT(41.46135783887948 73.1288676877708) bank194657 +194658 POINT(40.0463189520108 74.15204892246413) bank194658 +194659 POINT(41.21637546691042 73.30824487938145) bank194659 +194660 POINT(39.77945510638484 73.7570596923482) bank194660 +194661 POINT(40.58755474818636 74.84425802689081) bank194661 +194662 POINT(39.93022431608151 73.50938801738035) bank194662 +194663 POINT(39.80000342833487 73.28733099902561) bank194663 +194664 POINT(40.97537390867562 74.91448833770781) bank194664 +194665 POINT(40.550501961340785 73.5841235662762) bank194665 +194666 POINT(40.615765462756706 74.07324896392132) bank194666 +194667 POINT(39.99847814164942 73.89944806904946) bank194667 +194668 POINT(39.955929981206026 73.3103949243105) bank194668 +194669 POINT(40.6921024849219 74.58346784981062) bank194669 +194670 POINT(40.98504063908141 74.291779654212) bank194670 +194671 POINT(39.7592804333093 74.18070762313015) bank194671 +194672 POINT(41.553157882923145 74.7505914259398) bank194672 +194673 POINT(39.717384650812804 74.30072248775625) bank194673 +194674 POINT(41.21914091553459 73.14290768408185) bank194674 +194675 POINT(39.74701104407766 73.83990520831247) bank194675 +194676 POINT(39.86412445101296 74.4610818637713) bank194676 +194677 POINT(39.92459236227108 74.90736224436955) bank194677 +194678 POINT(39.835968723454776 74.95359784593529) bank194678 +194679 POINT(40.62062200038592 74.57204106347183) bank194679 +194680 POINT(39.96820857026057 74.95741214367742) bank194680 +194681 POINT(40.89336322318433 73.33565890299671) bank194681 +194682 POINT(41.32053040101049 73.85303716421599) bank194682 +194683 POINT(41.28944860684533 73.51731343469365) bank194683 +194684 POINT(41.68770870798574 74.76338940788528) bank194684 +194685 POINT(40.669773543760115 73.85614584047974) bank194685 +194686 POINT(40.482690403806856 74.21872715307157) bank194686 +194687 POINT(40.556998331916866 73.28188298644983) bank194687 +194688 POINT(41.01716050440291 73.69497717308971) bank194688 +194689 POINT(41.49920396555145 74.469717969699) bank194689 +194690 POINT(40.2348777391887 74.46392697310075) bank194690 +194691 POINT(40.22011438424806 74.85363895770959) bank194691 +194692 POINT(41.691158857860344 74.82768783343286) bank194692 +194693 POINT(40.93618101954854 73.71293578192841) bank194693 +194694 POINT(40.592394483321414 73.29476390348552) bank194694 +194695 POINT(40.979422258066606 73.72807303434223) bank194695 +194696 POINT(40.17919832358883 73.82588252938665) bank194696 +194697 POINT(40.29616966567918 73.28003010013748) bank194697 +194698 POINT(41.10416758550786 73.70056779067768) bank194698 +194699 POINT(40.80472570384601 73.05615596436108) bank194699 +194700 POINT(41.310308026126975 73.75050105580864) bank194700 +194701 POINT(41.38426638147031 73.68564355379459) bank194701 +194702 POINT(40.21196565004876 74.6034940373034) bank194702 +194703 POINT(40.74576845390474 73.8182474816611) bank194703 +194704 POINT(41.45966353390265 74.96451300435024) bank194704 +194705 POINT(40.85230617394682 73.03589486670035) bank194705 +194706 POINT(41.4807554179307 74.55747227391717) bank194706 +194707 POINT(41.34924861380797 74.21164024060455) bank194707 +194708 POINT(40.36898455577176 74.11300994320702) bank194708 +194709 POINT(39.733707913089326 73.22936203415796) bank194709 +194710 POINT(40.14643248934411 74.14268931837663) bank194710 +194711 POINT(40.70237441160011 74.70919648603362) bank194711 +194712 POINT(41.52588255112268 73.15725453468221) bank194712 +194713 POINT(40.30058129217738 74.4929453861602) bank194713 +194714 POINT(40.323505894902915 74.32948040482256) bank194714 +194715 POINT(41.20004338464551 74.71836300507321) bank194715 +194716 POINT(40.64450475930459 74.98425390979168) bank194716 +194717 POINT(40.40640877358495 74.94702858291527) bank194717 +194718 POINT(41.03864504309574 74.85747466040034) bank194718 +194719 POINT(41.35767880843075 73.0394793721827) bank194719 +194720 POINT(39.94967063195123 74.6699902138565) bank194720 +194721 POINT(41.17674297150903 74.71018649466367) bank194721 +194722 POINT(41.4824959488855 74.12069668240855) bank194722 +194723 POINT(41.12802465657438 74.76254613097058) bank194723 +194724 POINT(39.91376556422523 74.68377241302822) bank194724 +194725 POINT(40.89724591594854 73.57272146113596) bank194725 +194726 POINT(40.653648881396215 73.08759732430313) bank194726 +194727 POINT(41.06691463409875 73.02207230513335) bank194727 +194728 POINT(41.61613653800461 74.29783924782231) bank194728 +194729 POINT(40.969775086839256 73.23023885075355) bank194729 +194730 POINT(41.637582580466095 73.8771330781858) bank194730 +194731 POINT(41.46740015604801 74.22847984539423) bank194731 +194732 POINT(39.95839303891216 74.59733931923736) bank194732 +194733 POINT(39.792341534467084 74.20447841637468) bank194733 +194734 POINT(40.03322880179751 73.9673053627471) bank194734 +194735 POINT(40.51511996076808 73.6305284491611) bank194735 +194736 POINT(40.00673251936399 74.43333614655953) bank194736 +194737 POINT(40.87813979939645 74.19828125475408) bank194737 +194738 POINT(40.642702380021085 73.62082707032998) bank194738 +194739 POINT(40.11565641716341 73.71204368756095) bank194739 +194740 POINT(39.722110165688946 73.64301546709964) bank194740 +194741 POINT(41.46072411302178 74.22719821867967) bank194741 +194742 POINT(39.97152273988903 73.15813027041887) bank194742 +194743 POINT(40.10136492308831 74.7464909389226) bank194743 +194744 POINT(40.46284775867643 74.56960128068944) bank194744 +194745 POINT(41.045890493485 74.057703059289) bank194745 +194746 POINT(40.34423375032168 73.6444202277301) bank194746 +194747 POINT(39.804736224903834 74.7169898970252) bank194747 +194748 POINT(40.71772371166782 73.69134147626326) bank194748 +194749 POINT(40.92222572264839 74.5903051580375) bank194749 +194750 POINT(40.830044780228135 74.29251130982486) bank194750 +194751 POINT(39.830304620175156 74.28161085284094) bank194751 +194752 POINT(41.60048079522322 73.59726028181437) bank194752 +194753 POINT(41.04119091651097 74.46228577013945) bank194753 +194754 POINT(39.74645357000475 73.27571011103284) bank194754 +194755 POINT(41.265485370641464 73.40790372078484) bank194755 +194756 POINT(41.657392828521374 73.6001844087691) bank194756 +194757 POINT(39.77853375675575 73.12309699369425) bank194757 +194758 POINT(40.956989713417165 74.45902892375537) bank194758 +194759 POINT(40.34267693527676 73.76023957061155) bank194759 +194760 POINT(41.10851519431379 74.00124695294747) bank194760 +194761 POINT(41.53960479132524 73.32018927640169) bank194761 +194762 POINT(40.954925794513116 74.63107002742922) bank194762 +194763 POINT(41.23802852052804 74.51336141889973) bank194763 +194764 POINT(40.884585796014946 74.1387502666217) bank194764 +194765 POINT(41.57634556244172 73.51986775366025) bank194765 +194766 POINT(39.807406257314945 73.14603102571616) bank194766 +194767 POINT(41.2979060895875 74.41304846749307) bank194767 +194768 POINT(40.827118385383876 73.48063449365662) bank194768 +194769 POINT(40.516033364733694 74.43668845583503) bank194769 +194770 POINT(41.34823053887079 74.8748797549111) bank194770 +194771 POINT(41.497640871918875 73.1316980719811) bank194771 +194772 POINT(39.97759839260195 74.28105895713412) bank194772 +194773 POINT(40.013838483185054 73.47980111622526) bank194773 +194774 POINT(41.393680886029685 73.62694334690099) bank194774 +194775 POINT(40.3146739842133 73.2750549112533) bank194775 +194776 POINT(39.759304009000346 74.10715463218446) bank194776 +194777 POINT(41.199974715240295 73.93898368981978) bank194777 +194778 POINT(41.68411172434465 74.72347467254612) bank194778 +194779 POINT(40.200623170022965 74.5222109725372) bank194779 +194780 POINT(40.94850184198973 74.93073159758855) bank194780 +194781 POINT(41.33744226996719 73.75673494972192) bank194781 +194782 POINT(39.74972695946684 74.02460208329401) bank194782 +194783 POINT(41.20268452153771 74.33035932893874) bank194783 +194784 POINT(39.92708472262993 73.85666364749896) bank194784 +194785 POINT(40.644871046832804 73.05328812734095) bank194785 +194786 POINT(41.42962513380468 74.84463280111703) bank194786 +194787 POINT(41.15961616649118 74.7834211648309) bank194787 +194788 POINT(40.16086152944727 73.83537559150534) bank194788 +194789 POINT(40.087581613290006 73.88265275128644) bank194789 +194790 POINT(41.43488397661366 74.94883375119623) bank194790 +194791 POINT(41.209173585199274 73.94246715920498) bank194791 +194792 POINT(41.4448495855372 73.25058556256566) bank194792 +194793 POINT(41.15765852785733 73.48230964629977) bank194793 +194794 POINT(41.6121167300712 73.5385018048892) bank194794 +194795 POINT(41.10930452106516 74.05009278512668) bank194795 +194796 POINT(40.78471490619064 74.16555404077548) bank194796 +194797 POINT(39.989749367742355 74.16501548955837) bank194797 +194798 POINT(40.158272130578474 74.41551847936162) bank194798 +194799 POINT(41.1086193818403 73.32886478666497) bank194799 +194800 POINT(41.14600519415026 73.80278847159786) bank194800 +194801 POINT(40.27837786640511 73.33036156403816) bank194801 +194802 POINT(40.494144081454145 73.9576542189982) bank194802 +194803 POINT(41.20259815147112 73.08576872572391) bank194803 +194804 POINT(40.77992491217989 73.70422470465529) bank194804 +194805 POINT(40.21566755746603 73.26689546509043) bank194805 +194806 POINT(41.01546262561608 74.57734000074346) bank194806 +194807 POINT(40.61922812026253 74.66739942479653) bank194807 +194808 POINT(39.94441347671072 74.53371619218954) bank194808 +194809 POINT(41.155711185859275 73.51824458870134) bank194809 +194810 POINT(39.88329462701516 73.38466317800506) bank194810 +194811 POINT(40.76576856744635 74.5552501288218) bank194811 +194812 POINT(41.283296414331964 73.60230168153814) bank194812 +194813 POINT(40.58495845863544 74.37666021493884) bank194813 +194814 POINT(40.03087023351969 74.29539761884762) bank194814 +194815 POINT(40.09757364923282 73.73822921157078) bank194815 +194816 POINT(41.621182709325424 74.06608750352378) bank194816 +194817 POINT(40.654088535198916 73.67619873461489) bank194817 +194818 POINT(40.129124284566544 73.78645694450297) bank194818 +194819 POINT(41.116160221750526 74.76593310091238) bank194819 +194820 POINT(40.11693240670524 74.94816744593734) bank194820 +194821 POINT(41.13495944137011 73.59515115434225) bank194821 +194822 POINT(40.218944824230846 74.51807294316502) bank194822 +194823 POINT(41.587060010496636 73.37384737020113) bank194823 +194824 POINT(41.12395119185271 74.232660803661) bank194824 +194825 POINT(40.25372499254293 73.7346421087913) bank194825 +194826 POINT(41.03855026921337 73.11368072061177) bank194826 +194827 POINT(41.56952212231659 74.51680058812092) bank194827 +194828 POINT(40.12757436182995 73.18133447145816) bank194828 +194829 POINT(40.87377245190295 73.7498121973974) bank194829 +194830 POINT(40.44769097989234 73.13639155646435) bank194830 +194831 POINT(40.186015140950495 74.04228571335976) bank194831 +194832 POINT(41.63458142274133 73.50848696747401) bank194832 +194833 POINT(41.49268882286794 73.82576915451648) bank194833 +194834 POINT(40.31219479974115 74.12880389997555) bank194834 +194835 POINT(40.55786401744218 73.05599554295104) bank194835 +194836 POINT(40.28280207848597 74.27593810857846) bank194836 +194837 POINT(41.364001207693676 74.40903835087114) bank194837 +194838 POINT(41.625471420908156 74.37507935407304) bank194838 +194839 POINT(41.28671943338904 73.19683056532354) bank194839 +194840 POINT(41.48873370994659 74.84741828275547) bank194840 +194841 POINT(39.90563014631399 74.56206384058389) bank194841 +194842 POINT(40.45588446813528 73.27820977249654) bank194842 +194843 POINT(40.37109407021157 73.320863908325) bank194843 +194844 POINT(39.889144517694184 73.80923502913393) bank194844 +194845 POINT(41.129602715915446 73.98696781642808) bank194845 +194846 POINT(41.22095476668826 73.71435665728754) bank194846 +194847 POINT(40.51150437900536 74.23934103320782) bank194847 +194848 POINT(41.638437500671394 74.33165274882064) bank194848 +194849 POINT(40.65789558776364 73.90693259038237) bank194849 +194850 POINT(40.80426668338569 73.52336297048844) bank194850 +194851 POINT(40.132161443015235 73.97597533763198) bank194851 +194852 POINT(40.81442354849732 73.48419852796283) bank194852 +194853 POINT(40.27305591641775 74.496926550982) bank194853 +194854 POINT(40.375833455007125 74.62796089179749) bank194854 +194855 POINT(41.316703137114104 74.60513457609336) bank194855 +194856 POINT(40.36228505974041 73.16237275004147) bank194856 +194857 POINT(39.97929777199867 74.16657416673897) bank194857 +194858 POINT(41.59080661575394 74.30603883019229) bank194858 +194859 POINT(41.55755643733481 73.13107562691661) bank194859 +194860 POINT(40.6119810394975 73.21268074237418) bank194860 +194861 POINT(39.761582398877096 73.33648342987428) bank194861 +194862 POINT(40.180035224585026 73.06718250508426) bank194862 +194863 POINT(41.661476783721575 74.57128721903476) bank194863 +194864 POINT(40.152642183512945 73.45267048756791) bank194864 +194865 POINT(41.109325758460216 73.40805298500854) bank194865 +194866 POINT(39.802895255212164 74.84220225316943) bank194866 +194867 POINT(40.02585952975013 73.92891295746534) bank194867 +194868 POINT(41.23546796628536 73.07063499421463) bank194868 +194869 POINT(40.567427124247445 73.9926876645198) bank194869 +194870 POINT(40.49074250031892 73.19548088935244) bank194870 +194871 POINT(40.750250256677056 73.80721891776584) bank194871 +194872 POINT(41.18461731412014 73.81095445065935) bank194872 +194873 POINT(41.63325969301339 73.78086076278576) bank194873 +194874 POINT(40.73179719239106 73.2514857813345) bank194874 +194875 POINT(39.98700535291018 73.89908847223056) bank194875 +194876 POINT(41.58765796893481 74.73467505658515) bank194876 +194877 POINT(40.6065346251208 74.84728620010158) bank194877 +194878 POINT(40.249398985993444 73.39480742183508) bank194878 +194879 POINT(41.187859787941925 74.72883768725903) bank194879 +194880 POINT(40.65278787533586 73.88243000298294) bank194880 +194881 POINT(41.21832223361523 73.02665729597207) bank194881 +194882 POINT(40.515900361668294 74.5382768932725) bank194882 +194883 POINT(41.658767225816014 74.40201486902112) bank194883 +194884 POINT(39.8254516514445 73.17367036441154) bank194884 +194885 POINT(40.15671214228646 74.594769175106) bank194885 +194886 POINT(40.688345726536575 73.18368481071572) bank194886 +194887 POINT(41.10216224853476 74.63164718221904) bank194887 +194888 POINT(40.67634346325077 73.66141112988394) bank194888 +194889 POINT(41.22541210627286 74.30441726117888) bank194889 +194890 POINT(39.88922633306574 73.08187939307042) bank194890 +194891 POINT(41.32864162262556 74.03376384752059) bank194891 +194892 POINT(40.32416668782179 74.37590033713897) bank194892 +194893 POINT(40.948433420567895 73.99525951136911) bank194893 +194894 POINT(41.43703371636532 73.69234149145008) bank194894 +194895 POINT(40.17847378973954 73.64975242474101) bank194895 +194896 POINT(40.91249779741191 74.59212005572935) bank194896 +194897 POINT(40.27377878207729 73.74745643635033) bank194897 +194898 POINT(40.96955009727415 73.56266651629016) bank194898 +194899 POINT(41.08152309091414 73.88665830177858) bank194899 +194900 POINT(41.148224491942834 74.97998480551591) bank194900 +194901 POINT(40.64660932129047 74.60998857866478) bank194901 +194902 POINT(41.026554255518434 73.15166659062155) bank194902 +194903 POINT(41.49258911480777 74.93444850405366) bank194903 +194904 POINT(40.795358538448454 74.21853124433576) bank194904 +194905 POINT(40.79043657922382 74.5883744834782) bank194905 +194906 POINT(40.475314066577546 74.86690789108395) bank194906 +194907 POINT(39.98768606350021 73.09368344247613) bank194907 +194908 POINT(39.73010903895781 73.07345361684997) bank194908 +194909 POINT(40.949303018169154 73.73561861931829) bank194909 +194910 POINT(39.807570325394224 73.13437021136151) bank194910 +194911 POINT(41.46706612451148 73.36365066988019) bank194911 +194912 POINT(40.206861004001176 73.5134047938732) bank194912 +194913 POINT(41.30742973734431 73.51979326434669) bank194913 +194914 POINT(41.585088328499715 74.12906556634834) bank194914 +194915 POINT(39.87875553531559 73.03777795501077) bank194915 +194916 POINT(40.75789878204545 73.75753355619463) bank194916 +194917 POINT(39.80442997024324 74.5478733142328) bank194917 +194918 POINT(40.061253012239995 74.6109868394371) bank194918 +194919 POINT(40.90442765590152 74.80220354670067) bank194919 +194920 POINT(41.447061638973494 74.71836915653681) bank194920 +194921 POINT(39.84725856411181 73.95967146335414) bank194921 +194922 POINT(40.309343159838406 74.97782777485361) bank194922 +194923 POINT(40.19476710410727 74.18767268566549) bank194923 +194924 POINT(41.31946417917469 73.7297366959619) bank194924 +194925 POINT(40.762478438993185 74.29717757377796) bank194925 +194926 POINT(40.97982004553624 73.40231358261154) bank194926 +194927 POINT(41.34852599012741 74.87239284223111) bank194927 +194928 POINT(40.94846232141649 74.03664705426749) bank194928 +194929 POINT(41.70674805163378 73.42275730333836) bank194929 +194930 POINT(39.79798072775965 73.08375212260091) bank194930 +194931 POINT(41.25916427659213 73.89362744952219) bank194931 +194932 POINT(40.696626923038224 74.33681355865066) bank194932 +194933 POINT(41.37052610911888 74.95583755025754) bank194933 +194934 POINT(40.17031042946838 74.7835869598625) bank194934 +194935 POINT(40.02124921012779 74.21267879051962) bank194935 +194936 POINT(41.37967438001518 73.6885188664831) bank194936 +194937 POINT(40.389257045975576 74.38622800725935) bank194937 +194938 POINT(40.767561003352576 74.2700090079718) bank194938 +194939 POINT(39.82330924305465 73.69188996153574) bank194939 +194940 POINT(40.51201258672243 73.48578121095301) bank194940 +194941 POINT(39.912846467407704 73.97681339118606) bank194941 +194942 POINT(41.192099784924146 74.88045378684069) bank194942 +194943 POINT(40.98206596247229 74.80056390617074) bank194943 +194944 POINT(40.29895637227094 73.82859178430344) bank194944 +194945 POINT(41.21654404227467 74.10459522130456) bank194945 +194946 POINT(40.783388609852416 74.72632994980566) bank194946 +194947 POINT(40.59553581353641 74.37604822912489) bank194947 +194948 POINT(41.55145536872991 73.5935983827292) bank194948 +194949 POINT(40.80974662208099 73.90170438587225) bank194949 +194950 POINT(40.86194856660971 73.84467870126716) bank194950 +194951 POINT(40.466839924602766 74.53446977455371) bank194951 +194952 POINT(41.15978743489268 73.29906394685376) bank194952 +194953 POINT(41.34817694756153 74.78452657457949) bank194953 +194954 POINT(39.7900931090256 74.99733539363253) bank194954 +194955 POINT(41.652647432269546 73.90279479119405) bank194955 +194956 POINT(40.16525586644605 73.93148745214143) bank194956 +194957 POINT(40.85881338526925 74.02402095772669) bank194957 +194958 POINT(40.76320024197698 73.07065179738194) bank194958 +194959 POINT(41.385555121956976 74.16464495147275) bank194959 +194960 POINT(40.103478439765055 74.73785583948967) bank194960 +194961 POINT(39.80901060775841 74.04945876226819) bank194961 +194962 POINT(41.46565837740428 74.81114817213374) bank194962 +194963 POINT(39.93401776330344 73.03999203957771) bank194963 +194964 POINT(40.21719684105318 74.92850091900665) bank194964 +194965 POINT(41.14341076536667 73.28894424573909) bank194965 +194966 POINT(41.56060573176253 73.31245958308878) bank194966 +194967 POINT(41.38754383185822 73.5420336281976) bank194967 +194968 POINT(39.97548334455597 74.90893229024476) bank194968 +194969 POINT(40.03483682897814 73.11866071051446) bank194969 +194970 POINT(41.58442106021183 73.36474249142069) bank194970 +194971 POINT(41.1877841128949 74.75077721402275) bank194971 +194972 POINT(41.47133775477903 74.32418522600724) bank194972 +194973 POINT(40.568677690020046 74.23357054399914) bank194973 +194974 POINT(40.08979416757289 73.31229742786452) bank194974 +194975 POINT(40.813128764468615 74.7068732680447) bank194975 +194976 POINT(40.624203207213625 74.93783140343686) bank194976 +194977 POINT(41.56602281895142 74.08674986750816) bank194977 +194978 POINT(40.566735879054136 73.49831447047328) bank194978 +194979 POINT(41.47775966696024 73.77744239538876) bank194979 +194980 POINT(39.729465918693464 74.3716490639322) bank194980 +194981 POINT(41.38671871827444 74.89714104675839) bank194981 +194982 POINT(40.81111370152769 74.8085919450964) bank194982 +194983 POINT(41.221195670294236 74.01586708365846) bank194983 +194984 POINT(40.28145956526184 74.26695521959175) bank194984 +194985 POINT(40.16038107121245 73.92383405524376) bank194985 +194986 POINT(41.1802870635559 73.45729287003775) bank194986 +194987 POINT(40.19928548558503 74.10157295677767) bank194987 +194988 POINT(39.91785718670923 74.21717004592767) bank194988 +194989 POINT(41.09094927857108 74.22621901542229) bank194989 +194990 POINT(40.64101211690066 74.46231107767635) bank194990 +194991 POINT(40.46622548925731 73.04569823553365) bank194991 +194992 POINT(40.45147910563104 73.08034156603132) bank194992 +194993 POINT(41.034661153013865 73.53655098149815) bank194993 +194994 POINT(40.87971639953457 73.12027149378501) bank194994 +194995 POINT(40.552623828196914 74.36279443408442) bank194995 +194996 POINT(40.373545843580985 74.73394310670196) bank194996 +194997 POINT(40.36035600546303 73.93617353691859) bank194997 +194998 POINT(41.17489415605383 73.88853621201935) bank194998 +194999 POINT(40.70118715554763 73.04766855480055) bank194999 +195000 POINT(40.32150804533026 74.62304968837908) bank195000 +195001 POINT(40.011835234412935 73.33294070330689) bank195001 +195002 POINT(40.4665571953077 74.72381868618295) bank195002 +195003 POINT(40.621061186274716 73.48606489191502) bank195003 +195004 POINT(41.23588086394591 73.27175422009488) bank195004 +195005 POINT(41.0326523356563 74.71977041292728) bank195005 +195006 POINT(40.4881541692444 74.16174712744912) bank195006 +195007 POINT(40.652273161558114 73.58543429085032) bank195007 +195008 POINT(41.53203163388047 73.84875341305215) bank195008 +195009 POINT(40.1698529261673 73.16846809934748) bank195009 +195010 POINT(40.020164424139516 74.28467291097037) bank195010 +195011 POINT(40.78515192060398 74.76530437313505) bank195011 +195012 POINT(39.8857181085484 74.04748607077809) bank195012 +195013 POINT(41.690762640403875 73.21466900913398) bank195013 +195014 POINT(40.83956901666513 73.87706362487215) bank195014 +195015 POINT(41.20963499198325 74.2133471502616) bank195015 +195016 POINT(40.41926822934386 74.8344808068756) bank195016 +195017 POINT(40.304702798147346 73.25948581867482) bank195017 +195018 POINT(40.26461111724215 74.66603352471037) bank195018 +195019 POINT(41.42584174392054 74.80544687209512) bank195019 +195020 POINT(39.97074747496396 74.53837518888382) bank195020 +195021 POINT(40.43624200869822 73.59472789304886) bank195021 +195022 POINT(40.28037724009344 74.41232572960303) bank195022 +195023 POINT(41.67796674693521 73.98349262271698) bank195023 +195024 POINT(40.19813423288468 74.34158022164003) bank195024 +195025 POINT(39.77891374048136 73.2530245747312) bank195025 +195026 POINT(40.565042340456735 73.20461068072146) bank195026 +195027 POINT(40.85095722376132 73.97334931800741) bank195027 +195028 POINT(40.08747907302846 74.49072233668917) bank195028 +195029 POINT(41.69209679739117 74.23158707887353) bank195029 +195030 POINT(40.47661134592835 74.78923358944418) bank195030 +195031 POINT(39.869011276410006 73.29703585506672) bank195031 +195032 POINT(39.996305385311494 73.13088400556602) bank195032 +195033 POINT(40.303542542973254 73.70949618926919) bank195033 +195034 POINT(41.03912919861624 74.22329417751115) bank195034 +195035 POINT(39.964617768628734 73.65357092585515) bank195035 +195036 POINT(39.97935980616907 74.05823434094664) bank195036 +195037 POINT(40.58076366611844 73.84050038557216) bank195037 +195038 POINT(40.82520279381573 74.5895561465862) bank195038 +195039 POINT(40.27043443491777 74.0756750078773) bank195039 +195040 POINT(41.25888723322529 74.22043353429326) bank195040 +195041 POINT(41.66113594093319 73.48350855103016) bank195041 +195042 POINT(41.28631614810319 74.40240192881132) bank195042 +195043 POINT(41.14793459631767 74.12327555307753) bank195043 +195044 POINT(41.237564534832146 74.22834651339609) bank195044 +195045 POINT(40.861108204783704 74.20965429914312) bank195045 +195046 POINT(40.752657450749865 74.7148654584224) bank195046 +195047 POINT(41.024589523808004 74.64780680879247) bank195047 +195048 POINT(41.42607378908241 73.1383667553018) bank195048 +195049 POINT(40.97467232565673 74.1963901516002) bank195049 +195050 POINT(40.8284241439608 73.55688381834796) bank195050 +195051 POINT(41.066895273915755 73.42238955520429) bank195051 +195052 POINT(41.00102758176194 73.70632520909477) bank195052 +195053 POINT(39.9716857079822 74.99432280289007) bank195053 +195054 POINT(40.59455643174199 74.17163290922744) bank195054 +195055 POINT(40.795820836199916 73.30647996556425) bank195055 +195056 POINT(41.44466869152735 73.06114701111377) bank195056 +195057 POINT(40.243033768222425 73.91277757797516) bank195057 +195058 POINT(40.847243164151884 74.63659262906276) bank195058 +195059 POINT(40.173609912726434 74.60968647596447) bank195059 +195060 POINT(41.58163614917075 73.56448678808873) bank195060 +195061 POINT(41.29548824574743 73.93407160171598) bank195061 +195062 POINT(40.61295912817253 74.68721063144807) bank195062 +195063 POINT(41.54403982293747 73.40002839091989) bank195063 +195064 POINT(40.11142327851155 73.0495343367827) bank195064 +195065 POINT(39.73889046697466 74.52793749842104) bank195065 +195066 POINT(41.20924760380818 74.52825799359542) bank195066 +195067 POINT(41.55309234548601 74.77380906770655) bank195067 +195068 POINT(40.16320033970827 74.12810946106558) bank195068 +195069 POINT(40.24521571893847 73.56225076109779) bank195069 +195070 POINT(40.27821366714885 74.68940702035037) bank195070 +195071 POINT(40.42576909842041 74.46746197457142) bank195071 +195072 POINT(39.87005768036454 73.58115045583402) bank195072 +195073 POINT(40.121708793851866 73.56694975824257) bank195073 +195074 POINT(41.27306927142171 74.30317761636) bank195074 +195075 POINT(41.08127293235273 74.32182569998179) bank195075 +195076 POINT(40.85256105216113 73.58753332769206) bank195076 +195077 POINT(41.6855468521144 73.02550358051703) bank195077 +195078 POINT(40.297259295759766 73.02747281011095) bank195078 +195079 POINT(41.476796329269895 74.58347818454864) bank195079 +195080 POINT(40.15895105524011 74.16660194565381) bank195080 +195081 POINT(41.31308070135409 74.10413997523419) bank195081 +195082 POINT(41.29751843426581 73.20918427994269) bank195082 +195083 POINT(41.70444352048977 74.06866614507483) bank195083 +195084 POINT(39.88859082318344 73.01699991020507) bank195084 +195085 POINT(40.767278391570095 73.74851374986291) bank195085 +195086 POINT(40.64858910941957 74.98785324571973) bank195086 +195087 POINT(41.527840680291504 74.86331768808164) bank195087 +195088 POINT(40.81836093708531 73.02834971741557) bank195088 +195089 POINT(41.12373951486229 74.58904019610883) bank195089 +195090 POINT(39.83085721937593 73.08546681687747) bank195090 +195091 POINT(41.41785079136776 73.10954965478096) bank195091 +195092 POINT(41.105567781509514 73.96166145454954) bank195092 +195093 POINT(39.89673989589875 74.52474223995146) bank195093 +195094 POINT(39.77222019707627 74.11164483131117) bank195094 +195095 POINT(40.39582564334386 74.11120067747413) bank195095 +195096 POINT(40.267817010932035 74.46647409743053) bank195096 +195097 POINT(40.29566455719259 73.39052733909946) bank195097 +195098 POINT(39.95924019191749 74.6005877443663) bank195098 +195099 POINT(40.82794507494111 73.68142202424438) bank195099 +195100 POINT(41.29081799981358 73.45891335352216) bank195100 +195101 POINT(40.56374791225467 73.25922044020352) bank195101 +195102 POINT(40.06702764339129 74.5670951387533) bank195102 +195103 POINT(40.46541737677946 74.53659168855394) bank195103 +195104 POINT(41.651399645702796 73.60825607503503) bank195104 +195105 POINT(40.662682425332704 73.25372486000542) bank195105 +195106 POINT(41.1670812692119 74.65205894769502) bank195106 +195107 POINT(41.682522690639075 74.0224306926109) bank195107 +195108 POINT(40.7892578820337 73.81716682436654) bank195108 +195109 POINT(41.57448093771056 73.59762909130575) bank195109 +195110 POINT(40.585681205247674 74.01362114648987) bank195110 +195111 POINT(41.15130358035906 74.81927639624811) bank195111 +195112 POINT(40.513885616950084 73.78804588332738) bank195112 +195113 POINT(40.09697309024037 73.42445632209645) bank195113 +195114 POINT(40.18662966127085 74.51438029099448) bank195114 +195115 POINT(40.408707977164276 73.85309036481127) bank195115 +195116 POINT(41.63756521527941 74.86305370737705) bank195116 +195117 POINT(41.707092550606774 73.87804905099519) bank195117 +195118 POINT(39.87401780129813 73.1225207511017) bank195118 +195119 POINT(41.04044869465653 73.22594513061617) bank195119 +195120 POINT(39.925537968604345 74.49800565982063) bank195120 +195121 POINT(40.43768134004968 73.0063366584916) bank195121 +195122 POINT(39.727303914652374 74.27323321468816) bank195122 +195123 POINT(41.25383233599769 73.53221364252995) bank195123 +195124 POINT(39.80352188327209 73.47042441343493) bank195124 +195125 POINT(41.27341618111404 73.4364894508701) bank195125 +195126 POINT(40.23425047118281 73.35620513370104) bank195126 +195127 POINT(40.93350149629284 74.96098806974706) bank195127 +195128 POINT(41.18813734619274 73.73196552024352) bank195128 +195129 POINT(40.238838019006664 74.29331442025774) bank195129 +195130 POINT(40.61726591946753 74.61034983932859) bank195130 +195131 POINT(41.27140980811569 74.25695017066742) bank195131 +195132 POINT(39.90614355107171 73.17607561258642) bank195132 +195133 POINT(40.29331954377619 73.51593809915032) bank195133 +195134 POINT(40.96775026931887 74.24766705068583) bank195134 +195135 POINT(41.254774021401595 74.83564477989373) bank195135 +195136 POINT(41.077809403301764 74.96150871122869) bank195136 +195137 POINT(40.760943602382845 73.58329299350103) bank195137 +195138 POINT(40.87408547937537 74.8266115040018) bank195138 +195139 POINT(39.7956000645806 74.92153128732927) bank195139 +195140 POINT(40.82819379810606 74.91818005138089) bank195140 +195141 POINT(39.96058212237102 73.82993194635507) bank195141 +195142 POINT(40.4675374942345 73.64256613241554) bank195142 +195143 POINT(40.380293341319415 74.08056407585343) bank195143 +195144 POINT(40.41945878222252 74.05099161212524) bank195144 +195145 POINT(41.153353988733016 74.54051522326753) bank195145 +195146 POINT(41.66853486987494 73.9314824076336) bank195146 +195147 POINT(40.72336986425715 73.40396916509232) bank195147 +195148 POINT(40.19274050303574 73.2585457212298) bank195148 +195149 POINT(40.02195583543796 74.2343556843518) bank195149 +195150 POINT(40.933819867306156 73.47567188718085) bank195150 +195151 POINT(41.059198449851706 73.22764038118098) bank195151 +195152 POINT(39.97819454830245 74.61038843037144) bank195152 +195153 POINT(40.18262199739051 73.6524951330366) bank195153 +195154 POINT(40.205094530644025 73.9324868573197) bank195154 +195155 POINT(41.51524301576731 73.82455321271475) bank195155 +195156 POINT(41.33753299702713 73.52382102503734) bank195156 +195157 POINT(40.867156931529905 74.31286891046277) bank195157 +195158 POINT(40.60948726315827 74.28449502298534) bank195158 +195159 POINT(40.357541949296355 74.63072058565251) bank195159 +195160 POINT(39.97210241413426 73.35195337030508) bank195160 +195161 POINT(40.93168311888006 74.95177226820523) bank195161 +195162 POINT(40.08862402513244 73.63974391748076) bank195162 +195163 POINT(41.46456077171316 73.55861720292738) bank195163 +195164 POINT(40.53997596229141 73.5356410964342) bank195164 +195165 POINT(41.317483446528165 74.02857952741785) bank195165 +195166 POINT(39.750858318722024 73.14962715070395) bank195166 +195167 POINT(40.47343244413826 74.08908781030418) bank195167 +195168 POINT(40.1916441962675 74.3563572382012) bank195168 +195169 POINT(40.582278008213926 73.9036694956737) bank195169 +195170 POINT(41.076203990898975 74.67520148547577) bank195170 +195171 POINT(41.046049710738124 74.68687650209219) bank195171 +195172 POINT(41.01316008557598 74.91852154006723) bank195172 +195173 POINT(40.22464046626812 73.22113003829975) bank195173 +195174 POINT(41.13072920537159 74.51451816418634) bank195174 +195175 POINT(40.40670337474596 74.70614736635093) bank195175 +195176 POINT(40.17359269992924 74.23505808026731) bank195176 +195177 POINT(39.824299201799704 73.61623922178556) bank195177 +195178 POINT(40.04565966169638 73.40662482615843) bank195178 +195179 POINT(39.890120645870184 73.02708121093549) bank195179 +195180 POINT(40.42886608722828 74.88085142863937) bank195180 +195181 POINT(40.241776593150846 74.05816153452923) bank195181 +195182 POINT(41.15365658321088 73.17697916269621) bank195182 +195183 POINT(40.91268390996907 73.91775206008512) bank195183 +195184 POINT(39.816145132881864 74.01989494783763) bank195184 +195185 POINT(41.6312989052427 74.46754122313325) bank195185 +195186 POINT(40.7317792696256 74.78251892085538) bank195186 +195187 POINT(41.26616618951108 73.40468934154116) bank195187 +195188 POINT(40.44191239937554 74.17901724900075) bank195188 +195189 POINT(41.34564952421208 73.86508528334487) bank195189 +195190 POINT(40.09380844149881 74.84169682578238) bank195190 +195191 POINT(39.7251795585241 74.11197403701908) bank195191 +195192 POINT(41.53067398048555 74.17601630595347) bank195192 +195193 POINT(41.43755156541817 74.89337228872385) bank195193 +195194 POINT(40.25888375096243 73.33634050382749) bank195194 +195195 POINT(39.94868669298586 74.16853834436179) bank195195 +195196 POINT(41.529514340800354 73.55777685519328) bank195196 +195197 POINT(40.05365677187824 73.71349440720557) bank195197 +195198 POINT(40.33280679316837 73.92636988384574) bank195198 +195199 POINT(40.42396593612831 73.63742893817813) bank195199 +195200 POINT(40.66116725851282 74.8659964173783) bank195200 +195201 POINT(41.42387964972289 73.43624068614466) bank195201 +195202 POINT(41.391165461138115 74.32159036271626) bank195202 +195203 POINT(40.68492401056208 73.23436828480426) bank195203 +195204 POINT(41.591909949930084 74.99503214748887) bank195204 +195205 POINT(41.00604785186625 74.01811826226063) bank195205 +195206 POINT(40.29444227105344 73.61662112174238) bank195206 +195207 POINT(41.70171165381167 74.64812921721786) bank195207 +195208 POINT(41.10529078034462 74.90859610256317) bank195208 +195209 POINT(40.337444677315595 73.93639849645534) bank195209 +195210 POINT(40.996115009330666 74.87533822292416) bank195210 +195211 POINT(41.544488085362545 73.47959179393415) bank195211 +195212 POINT(41.125712234714605 74.00035971999257) bank195212 +195213 POINT(41.1422852168952 73.61369249693023) bank195213 +195214 POINT(41.43817697688728 74.07027661791011) bank195214 +195215 POINT(40.56051212067579 73.30030575946907) bank195215 +195216 POINT(41.16815903720074 73.19647935359247) bank195216 +195217 POINT(40.23448581398909 74.13615780822855) bank195217 +195218 POINT(40.427030617288715 73.64747901653102) bank195218 +195219 POINT(39.816177178285464 74.02826862077224) bank195219 +195220 POINT(39.94443345614761 73.63604373256914) bank195220 +195221 POINT(39.99501417017323 73.88743059738674) bank195221 +195222 POINT(40.46455058395482 74.26871610988124) bank195222 +195223 POINT(40.49208184684451 74.46147153031465) bank195223 +195224 POINT(40.738078785957136 73.4217707590608) bank195224 +195225 POINT(40.87440744743341 73.27508799584335) bank195225 +195226 POINT(40.92811729874415 74.77078464550485) bank195226 +195227 POINT(40.33283956156772 73.24139139893327) bank195227 +195228 POINT(41.16962642171917 74.6900769650542) bank195228 +195229 POINT(40.35003444568909 74.20891880003276) bank195229 +195230 POINT(40.08146877953844 73.06631904459245) bank195230 +195231 POINT(40.22504920794815 74.28366978580438) bank195231 +195232 POINT(40.24304542575469 73.79146099427344) bank195232 +195233 POINT(40.16088562309438 74.63709259569501) bank195233 +195234 POINT(40.77660135557136 74.71487028360112) bank195234 +195235 POINT(40.49033626460939 73.33510438748762) bank195235 +195236 POINT(41.17386403317752 74.84931930488185) bank195236 +195237 POINT(40.67720074725184 73.7887811555389) bank195237 +195238 POINT(41.44248675676255 74.45387037750967) bank195238 +195239 POINT(40.20195516146015 73.88476016648882) bank195239 +195240 POINT(41.49149111748534 75.00521001027293) bank195240 +195241 POINT(41.58690885879566 74.68870279825828) bank195241 +195242 POINT(41.126495416833144 73.30656506916219) bank195242 +195243 POINT(41.06752897361728 73.72403413761155) bank195243 +195244 POINT(39.825562272783856 74.68678794279928) bank195244 +195245 POINT(39.91853802873704 74.5535887659004) bank195245 +195246 POINT(40.467657057799634 74.92995853361937) bank195246 +195247 POINT(40.376273702683115 73.18770228619253) bank195247 +195248 POINT(40.991802735773135 73.41195409272228) bank195248 +195249 POINT(41.04721986325494 73.4608125683956) bank195249 +195250 POINT(40.799349212858104 73.12183590636765) bank195250 +195251 POINT(40.76037903825775 74.77711558137045) bank195251 +195252 POINT(41.36039920913122 73.41819201676142) bank195252 +195253 POINT(41.3799867000196 74.17308616979574) bank195253 +195254 POINT(41.33302586568435 74.30562760817116) bank195254 +195255 POINT(40.52334262669929 74.79477341749666) bank195255 +195256 POINT(39.89097288270638 74.8866204627759) bank195256 +195257 POINT(40.335856265569205 74.03343925113857) bank195257 +195258 POINT(41.28262888940367 73.10439381424594) bank195258 +195259 POINT(40.12272710553157 73.11390680697833) bank195259 +195260 POINT(39.87881241196848 74.46911783251534) bank195260 +195261 POINT(41.176510315502256 74.75448392117367) bank195261 +195262 POINT(40.00515403074759 74.72348476089162) bank195262 +195263 POINT(40.76207054220592 74.26110475565771) bank195263 +195264 POINT(40.24398063938861 73.56911543101839) bank195264 +195265 POINT(41.108191328333035 74.93442710856309) bank195265 +195266 POINT(41.388201371009295 73.52372433479276) bank195266 +195267 POINT(39.987910299518475 73.00932011027177) bank195267 +195268 POINT(40.06202074680937 73.03419554958553) bank195268 +195269 POINT(40.630377519080476 73.61724484610025) bank195269 +195270 POINT(40.29852710239029 74.7571086445276) bank195270 +195271 POINT(41.41653292346547 73.19549150940306) bank195271 +195272 POINT(40.609660285046836 74.92422652459977) bank195272 +195273 POINT(40.212827760295355 73.05113401393194) bank195273 +195274 POINT(40.3977260459884 74.45961208610346) bank195274 +195275 POINT(41.59625089734372 74.9800294279992) bank195275 +195276 POINT(40.58899682245462 73.2648679560989) bank195276 +195277 POINT(40.36728792517351 74.5059083298797) bank195277 +195278 POINT(39.749887524364986 74.10406307121303) bank195278 +195279 POINT(40.18813093648453 73.82419771024452) bank195279 +195280 POINT(41.69833336337862 74.67728565539194) bank195280 +195281 POINT(40.947213133341606 73.20716538110673) bank195281 +195282 POINT(41.59228730670003 73.59277133281317) bank195282 +195283 POINT(41.15810717403994 74.02492053809962) bank195283 +195284 POINT(40.16515534554059 74.08863159982926) bank195284 +195285 POINT(39.973717961034005 73.69539449494931) bank195285 +195286 POINT(40.66515784133326 73.89220356577198) bank195286 +195287 POINT(39.88360570003845 74.80501472483729) bank195287 +195288 POINT(41.642487948593 74.35383116560381) bank195288 +195289 POINT(39.83639680536267 74.70358787120838) bank195289 +195290 POINT(39.72408845043386 73.23415836140308) bank195290 +195291 POINT(39.75537110174955 74.03927249915408) bank195291 +195292 POINT(40.170985909560805 73.96116219368142) bank195292 +195293 POINT(40.64575147203286 73.25691432834132) bank195293 +195294 POINT(40.31086949261228 73.93938249129278) bank195294 +195295 POINT(40.17261032109235 74.78419825632105) bank195295 +195296 POINT(40.86998682518552 73.51242368337444) bank195296 +195297 POINT(41.441196334899224 73.19861197083763) bank195297 +195298 POINT(40.12552858121538 73.1350200537749) bank195298 +195299 POINT(41.43037371850587 74.13123435599965) bank195299 +195300 POINT(41.12813228536654 74.55697702354875) bank195300 +195301 POINT(40.542827523991 73.15024131739355) bank195301 +195302 POINT(41.04898142888294 73.36459167677336) bank195302 +195303 POINT(40.97973342176297 74.49793149910373) bank195303 +195304 POINT(39.9332124267521 74.08314007344099) bank195304 +195305 POINT(40.13770932706859 74.45616974371734) bank195305 +195306 POINT(40.78954648382125 74.86228305849737) bank195306 +195307 POINT(40.32010051912848 74.86726428562356) bank195307 +195308 POINT(41.40953616819537 73.33976674032786) bank195308 +195309 POINT(39.84920821283352 73.93388946941879) bank195309 +195310 POINT(41.58117778569511 73.69999140604732) bank195310 +195311 POINT(40.32431119190526 73.97272703380386) bank195311 +195312 POINT(41.68791666519495 74.19720600442851) bank195312 +195313 POINT(40.84697296439643 73.18098798632246) bank195313 +195314 POINT(40.845995221538566 74.26721881200422) bank195314 +195315 POINT(40.3094646652948 74.16590131780609) bank195315 +195316 POINT(41.08285157430412 74.00481569381168) bank195316 +195317 POINT(40.44130104946493 74.93640021384066) bank195317 +195318 POINT(41.187370189945895 73.3076789123797) bank195318 +195319 POINT(40.9225461320781 73.84721342274925) bank195319 +195320 POINT(41.27467891114738 74.74944649475941) bank195320 +195321 POINT(40.220315970152576 74.54163159640137) bank195321 +195322 POINT(40.12384351618517 74.39996176165789) bank195322 +195323 POINT(40.1194438934396 74.80789973285565) bank195323 +195324 POINT(40.80875201340646 73.04891784461668) bank195324 +195325 POINT(40.65105887687008 73.2190530785439) bank195325 +195326 POINT(40.94296406957135 73.83852557717753) bank195326 +195327 POINT(40.85337854400978 74.41831711115827) bank195327 +195328 POINT(41.18481756449234 74.55477439331536) bank195328 +195329 POINT(41.36484318924918 74.27356323369634) bank195329 +195330 POINT(39.79202625072951 74.55807406483129) bank195330 +195331 POINT(41.206362605967186 73.93318431586805) bank195331 +195332 POINT(40.5723527649378 73.02389318844685) bank195332 +195333 POINT(41.29980230973665 74.5804810366867) bank195333 +195334 POINT(41.269337725694825 73.53248758798675) bank195334 +195335 POINT(40.482801116433784 73.48695823113249) bank195335 +195336 POINT(41.34326083570404 73.17387183415676) bank195336 +195337 POINT(40.01158903125984 74.8127047276921) bank195337 +195338 POINT(40.34715840952964 74.65491284467618) bank195338 +195339 POINT(40.592915704041644 74.1257776154608) bank195339 +195340 POINT(40.596134071716 73.60373357071248) bank195340 +195341 POINT(40.035150552118765 73.6862686487587) bank195341 +195342 POINT(40.72239004331809 74.5621664480527) bank195342 +195343 POINT(40.5055525409407 73.90704380978039) bank195343 +195344 POINT(41.3038173327178 73.12841355358232) bank195344 +195345 POINT(40.46512818082393 74.6851807846195) bank195345 +195346 POINT(39.89263789026195 74.09898195968297) bank195346 +195347 POINT(39.88961061960609 73.52562236714488) bank195347 +195348 POINT(40.03871298572712 73.11539893528261) bank195348 +195349 POINT(41.22164620220231 73.2936214980137) bank195349 +195350 POINT(40.29464456583951 74.27488909327157) bank195350 +195351 POINT(40.454153566835316 73.28030897525318) bank195351 +195352 POINT(41.67345955613415 74.94141120981475) bank195352 +195353 POINT(41.52797895960869 73.67897919164793) bank195353 +195354 POINT(41.113552500732254 74.39604776730074) bank195354 +195355 POINT(40.837583488393925 73.36931180708586) bank195355 +195356 POINT(40.14495385586268 74.99953694837171) bank195356 +195357 POINT(41.137167448487205 73.84510765267206) bank195357 +195358 POINT(41.203915149981256 73.95832947635049) bank195358 +195359 POINT(41.35810516461983 74.23094168619815) bank195359 +195360 POINT(39.927602874097005 73.08355012375993) bank195360 +195361 POINT(40.80139335470091 73.4039585081667) bank195361 +195362 POINT(40.87757598404763 73.88692914996943) bank195362 +195363 POINT(41.01318879428384 73.46342861678426) bank195363 +195364 POINT(40.59489218966836 74.98917826262345) bank195364 +195365 POINT(41.51277971563854 73.49248736051189) bank195365 +195366 POINT(39.974584791083714 73.29216791706209) bank195366 +195367 POINT(40.16414392485779 74.42388208691506) bank195367 +195368 POINT(39.885449199015554 74.51269952479169) bank195368 +195369 POINT(41.63605740948597 73.50085226610547) bank195369 +195370 POINT(41.1138921087594 75.00370593650514) bank195370 +195371 POINT(40.85349407070238 73.92183347118949) bank195371 +195372 POINT(40.153136501799175 73.02524342607207) bank195372 +195373 POINT(41.19382775746844 73.7009307591273) bank195373 +195374 POINT(39.7899696557855 74.87360130712823) bank195374 +195375 POINT(41.203810092724396 73.03970087258061) bank195375 +195376 POINT(40.616587327073084 73.9832788351429) bank195376 +195377 POINT(41.5088488608982 73.13427620613552) bank195377 +195378 POINT(40.80942024429391 74.34168647273647) bank195378 +195379 POINT(39.792580548656375 74.49292950306842) bank195379 +195380 POINT(40.60576174860868 74.88674884747331) bank195380 +195381 POINT(40.143481735219595 73.8338744692432) bank195381 +195382 POINT(40.61075939569042 73.44140568523855) bank195382 +195383 POINT(40.85417779965922 74.59662902511565) bank195383 +195384 POINT(41.0710304747822 74.8955641792446) bank195384 +195385 POINT(41.38329338440339 73.57794259045706) bank195385 +195386 POINT(39.74098778168001 73.01376337791878) bank195386 +195387 POINT(41.628409122342894 73.43015804113085) bank195387 +195388 POINT(40.3134926083418 74.29043935770555) bank195388 +195389 POINT(40.188649111454936 73.71853214920547) bank195389 +195390 POINT(39.90876683888966 74.24062159002693) bank195390 +195391 POINT(41.66967483698734 73.83269985010287) bank195391 +195392 POINT(41.379751678008375 74.49423472857727) bank195392 +195393 POINT(40.74028661094833 73.69494268000581) bank195393 +195394 POINT(40.6665427219979 74.68708055483037) bank195394 +195395 POINT(40.991454551096865 73.5546980834038) bank195395 +195396 POINT(40.96577954959817 74.95597580541826) bank195396 +195397 POINT(41.37319306562364 73.63689461988368) bank195397 +195398 POINT(40.89444055174771 73.45665331799461) bank195398 +195399 POINT(40.04406706588588 73.14431631158017) bank195399 +195400 POINT(41.63585496404523 74.05776390039038) bank195400 +195401 POINT(40.6310106371007 74.60039176096623) bank195401 +195402 POINT(41.60302443934407 74.21073974028668) bank195402 +195403 POINT(40.819559413697796 73.78829459130297) bank195403 +195404 POINT(41.590353277604066 73.11835753459252) bank195404 +195405 POINT(41.2492323464104 73.81020794684704) bank195405 +195406 POINT(41.511424054376995 73.40587133127289) bank195406 +195407 POINT(40.89809611298753 73.16299578065721) bank195407 +195408 POINT(40.8885991483225 74.32212188341322) bank195408 +195409 POINT(41.02345607067047 73.99726022125776) bank195409 +195410 POINT(41.183839795084076 73.9775788788616) bank195410 +195411 POINT(39.895143772027595 73.98093965534831) bank195411 +195412 POINT(39.89509431410117 74.31714456134135) bank195412 +195413 POINT(41.0459008030105 73.60881949679319) bank195413 +195414 POINT(41.05623864212809 73.44153846268557) bank195414 +195415 POINT(40.87271548638022 73.36912453524137) bank195415 +195416 POINT(40.26137361865055 73.9189132543933) bank195416 +195417 POINT(41.50880853204727 73.79971975538719) bank195417 +195418 POINT(40.35834156597726 73.10602281222988) bank195418 +195419 POINT(39.96691331662136 74.48821805315008) bank195419 +195420 POINT(40.72724827775142 73.88660511508283) bank195420 +195421 POINT(41.463914364797226 73.24031298824235) bank195421 +195422 POINT(41.35957564403964 74.5542206313383) bank195422 +195423 POINT(40.989027008093615 74.80194185382051) bank195423 +195424 POINT(40.60982319958823 73.45484708714365) bank195424 +195425 POINT(41.48619797001151 73.55665701989072) bank195425 +195426 POINT(39.893722902080754 74.46085468453111) bank195426 +195427 POINT(39.93068286740682 74.77702519666985) bank195427 +195428 POINT(41.100036330523906 73.42957409942484) bank195428 +195429 POINT(40.9157480363622 73.90450996611385) bank195429 +195430 POINT(40.83751898293646 73.68837315532032) bank195430 +195431 POINT(40.122769553227954 74.66628049244494) bank195431 +195432 POINT(40.69601924891585 73.24616557145264) bank195432 +195433 POINT(41.54610059753647 74.545457887098) bank195433 +195434 POINT(41.252964947953224 73.48557998231787) bank195434 +195435 POINT(40.72170132528617 74.73300528715694) bank195435 +195436 POINT(40.45028777190451 74.91438238911674) bank195436 +195437 POINT(39.72519265741111 73.38480617459749) bank195437 +195438 POINT(40.857647632845634 73.05610136152646) bank195438 +195439 POINT(40.01863385762075 74.25541150895013) bank195439 +195440 POINT(40.043562403347906 73.58822271908684) bank195440 +195441 POINT(41.219566067921704 75.00001670647079) bank195441 +195442 POINT(40.39416136811967 73.18037397940842) bank195442 +195443 POINT(40.17912115077987 74.394295520383) bank195443 +195444 POINT(40.89066599948252 73.55056814797287) bank195444 +195445 POINT(40.85475126335412 74.09656642337022) bank195445 +195446 POINT(40.09310062919455 74.05536109805459) bank195446 +195447 POINT(41.21906340428698 73.8003492848882) bank195447 +195448 POINT(39.774030419284195 74.5787976466425) bank195448 +195449 POINT(41.09774279325641 74.06537099576776) bank195449 +195450 POINT(41.2236821535607 73.11749594607929) bank195450 +195451 POINT(41.47915116656456 73.95663625802432) bank195451 +195452 POINT(41.62678779932457 74.90876633218915) bank195452 +195453 POINT(40.12757270409247 73.65275398691345) bank195453 +195454 POINT(41.419863436861704 74.26537586567724) bank195454 +195455 POINT(40.618891955109 73.02801547665528) bank195455 +195456 POINT(40.740713084200266 73.89295298821854) bank195456 +195457 POINT(41.23123656471766 73.82707536239317) bank195457 +195458 POINT(40.04746154958112 73.09731104966923) bank195458 +195459 POINT(41.537494931824654 74.28025012872739) bank195459 +195460 POINT(39.73629513022734 74.25223363719235) bank195460 +195461 POINT(41.69021394958794 73.77595455857391) bank195461 +195462 POINT(39.94126402742829 73.43567496565286) bank195462 +195463 POINT(40.10897065099529 73.34607075982716) bank195463 +195464 POINT(40.65202417930295 74.83074751318455) bank195464 +195465 POINT(40.76824044711789 73.24366916287617) bank195465 +195466 POINT(40.84491317408817 73.97605943846877) bank195466 +195467 POINT(41.12646183126785 74.28185143267096) bank195467 +195468 POINT(41.06433686345362 73.61242133756011) bank195468 +195469 POINT(40.46575360341116 74.36125763699738) bank195469 +195470 POINT(40.43930012811364 73.46044241157608) bank195470 +195471 POINT(41.32637633542379 73.01366332115558) bank195471 +195472 POINT(41.667847765380515 74.08258552063721) bank195472 +195473 POINT(41.63384006956183 73.1728598247807) bank195473 +195474 POINT(40.44948231648699 73.21607817541603) bank195474 +195475 POINT(41.158314817457615 74.09191780685146) bank195475 +195476 POINT(40.032801081989376 74.60861502568657) bank195476 +195477 POINT(41.14620460825556 74.5129199014014) bank195477 +195478 POINT(39.95747217215476 73.96339954995214) bank195478 +195479 POINT(40.67777721478292 73.23981834982244) bank195479 +195480 POINT(40.899032143486046 73.64977272311309) bank195480 +195481 POINT(41.55451589511027 74.07705766050931) bank195481 +195482 POINT(39.73389300350438 73.30161749257877) bank195482 +195483 POINT(40.9098100048099 73.23421947267757) bank195483 +195484 POINT(40.842260973771126 73.07006531871286) bank195484 +195485 POINT(41.317345196529445 73.25955580882197) bank195485 +195486 POINT(39.818314919841 73.83800103417049) bank195486 +195487 POINT(40.96304577625873 74.47262928952651) bank195487 +195488 POINT(40.164279656179126 74.44156259795712) bank195488 +195489 POINT(40.370654222155046 74.65024774307545) bank195489 +195490 POINT(41.36576067232421 74.24227376247372) bank195490 +195491 POINT(39.966093155833605 73.00724790734455) bank195491 +195492 POINT(40.30099974019229 74.7480009738987) bank195492 +195493 POINT(40.47391803695426 73.38859680085874) bank195493 +195494 POINT(41.41120553936702 73.38896444434762) bank195494 +195495 POINT(40.95243576350638 74.46798993941597) bank195495 +195496 POINT(40.72411122115324 74.05430848777033) bank195496 +195497 POINT(40.92892832671894 73.53836924757218) bank195497 +195498 POINT(41.13116893889436 73.18737170443852) bank195498 +195499 POINT(41.70445376402408 73.7161060749688) bank195499 +195500 POINT(41.4485265901849 74.18051096869642) bank195500 +195501 POINT(40.554047495885044 74.08404676969205) bank195501 +195502 POINT(41.26718319262666 74.08457070339396) bank195502 +195503 POINT(41.511662108996035 73.92292288862556) bank195503 +195504 POINT(41.286342872185294 74.23348471992487) bank195504 +195505 POINT(40.904349714284734 73.75884595478175) bank195505 +195506 POINT(40.887348624271496 74.81389368648165) bank195506 +195507 POINT(40.375484100988785 74.3754552887019) bank195507 +195508 POINT(40.72884679163495 73.33064965996715) bank195508 +195509 POINT(39.90911181607853 74.52960280910854) bank195509 +195510 POINT(41.19546427026007 74.38796806822783) bank195510 +195511 POINT(41.56598483111243 73.88597790436553) bank195511 +195512 POINT(40.574627194580074 73.76460144533502) bank195512 +195513 POINT(41.655068036838365 73.93387864569752) bank195513 +195514 POINT(40.93356310601147 74.1572894102196) bank195514 +195515 POINT(40.75986684232889 73.7213784152896) bank195515 +195516 POINT(40.1040869615164 73.3796475691377) bank195516 +195517 POINT(41.430476146754884 74.92061907062663) bank195517 +195518 POINT(40.175044372008564 74.7696639126581) bank195518 +195519 POINT(41.1534500919018 73.09481068692179) bank195519 +195520 POINT(41.00262034047818 73.35290774365052) bank195520 +195521 POINT(41.39069733213207 74.2414638863155) bank195521 +195522 POINT(41.17776236828874 74.95959439694596) bank195522 +195523 POINT(40.24631386289061 73.19192014661887) bank195523 +195524 POINT(41.62851163286871 74.82506619607454) bank195524 +195525 POINT(41.27848937947402 73.24073905134901) bank195525 +195526 POINT(41.61781740646954 74.60063946073188) bank195526 +195527 POINT(41.563086634130734 74.51264751862385) bank195527 +195528 POINT(40.99966329792135 73.78333093840496) bank195528 +195529 POINT(39.775805560545095 74.66019726641342) bank195529 +195530 POINT(40.20934833580107 74.23319600683212) bank195530 +195531 POINT(41.32754028706039 74.37995889728481) bank195531 +195532 POINT(40.12375920475476 74.17252934305841) bank195532 +195533 POINT(40.885564876489475 73.57808330324333) bank195533 +195534 POINT(40.26388109559003 73.84123165523974) bank195534 +195535 POINT(40.019175016882976 73.50918749667623) bank195535 +195536 POINT(40.5936171253545 73.62167596426714) bank195536 +195537 POINT(40.36105803991354 74.02839681909713) bank195537 +195538 POINT(41.62364881422694 73.33423997407385) bank195538 +195539 POINT(41.1925821462568 74.95621216563245) bank195539 +195540 POINT(39.8014498970384 73.67094403043845) bank195540 +195541 POINT(39.999469716248356 73.56936727129981) bank195541 +195542 POINT(41.430951132424525 74.77663498468384) bank195542 +195543 POINT(41.178759104998164 74.9899572380185) bank195543 +195544 POINT(40.19968437230469 74.32032187409867) bank195544 +195545 POINT(40.61253657868094 73.03179115971682) bank195545 +195546 POINT(41.46299171548527 73.55700721443954) bank195546 +195547 POINT(41.45119366051828 74.4346036634091) bank195547 +195548 POINT(40.33650668212343 74.50337348600083) bank195548 +195549 POINT(40.032128182882985 73.37448401256044) bank195549 +195550 POINT(40.30089203085711 73.81562102099655) bank195550 +195551 POINT(40.83185742132742 74.25974619165002) bank195551 +195552 POINT(41.139383584016244 74.47381501514006) bank195552 +195553 POINT(41.38445632938741 74.61943943075619) bank195553 +195554 POINT(40.52782315290954 73.53176998337739) bank195554 +195555 POINT(40.244331651526295 73.44967954908795) bank195555 +195556 POINT(41.57456507099711 74.39890508181573) bank195556 +195557 POINT(40.269275643568356 74.46155405585718) bank195557 +195558 POINT(40.82885941036677 73.93218537569193) bank195558 +195559 POINT(41.374320644699566 74.64125435374) bank195559 +195560 POINT(40.249064411475366 73.63564148851015) bank195560 +195561 POINT(40.966424793302565 74.72382225732584) bank195561 +195562 POINT(41.27067936592004 74.82588525475228) bank195562 +195563 POINT(40.79438117043662 73.60036839281055) bank195563 +195564 POINT(40.552461672633434 74.86276937614473) bank195564 +195565 POINT(41.67992703401199 74.4660576203301) bank195565 +195566 POINT(40.665999601763055 74.10224743599899) bank195566 +195567 POINT(39.92191881000611 74.51646877884242) bank195567 +195568 POINT(40.394741505340775 74.17877354515187) bank195568 +195569 POINT(40.57651286324355 73.50417819511621) bank195569 +195570 POINT(39.72582251525759 74.61238425301124) bank195570 +195571 POINT(40.59209777646236 74.77861887488014) bank195571 +195572 POINT(40.51446895436751 73.92500943922423) bank195572 +195573 POINT(41.32456201404915 74.10883304362218) bank195573 +195574 POINT(41.08214465883506 74.06679018638795) bank195574 +195575 POINT(41.239905211550976 73.88527833702554) bank195575 +195576 POINT(40.12489557068318 74.90444119387837) bank195576 +195577 POINT(40.82399639530679 73.76899603060302) bank195577 +195578 POINT(40.61268877964644 74.63428314387518) bank195578 +195579 POINT(40.86692069288057 74.5143478904306) bank195579 +195580 POINT(39.73026624244244 74.10426893656481) bank195580 +195581 POINT(41.42531882179676 73.23216939562916) bank195581 +195582 POINT(40.609214467756864 74.1250437530529) bank195582 +195583 POINT(41.42057938164263 74.74946366417711) bank195583 +195584 POINT(39.733360011574895 74.2831326008223) bank195584 +195585 POINT(40.20226001851735 73.60125455773786) bank195585 +195586 POINT(40.909873304036346 74.92646422462796) bank195586 +195587 POINT(41.09406329876003 73.5406482787228) bank195587 +195588 POINT(39.91769226320784 74.92461706411936) bank195588 +195589 POINT(40.99287956405392 73.97860314080759) bank195589 +195590 POINT(40.2673304788523 74.11909624460554) bank195590 +195591 POINT(41.70319174721138 74.27512335761598) bank195591 +195592 POINT(40.534225511402504 73.53676182894874) bank195592 +195593 POINT(39.92514854850175 74.44828936861717) bank195593 +195594 POINT(39.91492383933617 74.00144836456015) bank195594 +195595 POINT(41.59765277597725 73.73332013782178) bank195595 +195596 POINT(41.03448794265395 73.82368936634256) bank195596 +195597 POINT(40.8169248895932 74.88819662806135) bank195597 +195598 POINT(40.481317097463 74.18529359250589) bank195598 +195599 POINT(41.261044917021074 74.66637698914508) bank195599 +195600 POINT(40.4665963536629 73.17497219379099) bank195600 +195601 POINT(40.00690769130768 74.30734015113514) bank195601 +195602 POINT(40.27895873563945 73.508662210807) bank195602 +195603 POINT(41.28331559230529 73.9396438713148) bank195603 +195604 POINT(39.94950070215395 74.28408834352491) bank195604 +195605 POINT(40.442493685951824 74.21804819005429) bank195605 +195606 POINT(40.77725973252238 74.6292206118132) bank195606 +195607 POINT(40.36126733688971 73.93038748659161) bank195607 +195608 POINT(40.663613230390894 73.66808823167494) bank195608 +195609 POINT(40.15721967522531 74.30887620266687) bank195609 +195610 POINT(40.79838146842242 73.32760181245054) bank195610 +195611 POINT(41.39943754328907 74.52141705273122) bank195611 +195612 POINT(40.095283890688094 74.89699347321275) bank195612 +195613 POINT(40.725286770383384 74.89791382459074) bank195613 +195614 POINT(41.67461885435807 73.27973702345763) bank195614 +195615 POINT(39.986226710087614 74.31638674701927) bank195615 +195616 POINT(40.50086437597778 74.48360089779626) bank195616 +195617 POINT(41.33824368934389 74.29384937480671) bank195617 +195618 POINT(39.98014152414247 73.84300171563929) bank195618 +195619 POINT(41.38219568199847 73.47787193501786) bank195619 +195620 POINT(39.77309076664701 73.27034085043248) bank195620 +195621 POINT(39.73552023955163 74.15604969231131) bank195621 +195622 POINT(40.862910773061714 74.57339761406571) bank195622 +195623 POINT(40.28509510939992 74.57913902452309) bank195623 +195624 POINT(40.762780280889224 74.56364339183186) bank195624 +195625 POINT(39.73747587103638 73.93581595256204) bank195625 +195626 POINT(40.881845893444876 74.70222912617078) bank195626 +195627 POINT(41.63906829160383 73.18873878131795) bank195627 +195628 POINT(41.41878355011532 74.0909137510404) bank195628 +195629 POINT(41.323173039149445 74.54706599646347) bank195629 +195630 POINT(40.89312354159275 74.55096586169931) bank195630 +195631 POINT(40.45171196370223 73.72674540888032) bank195631 +195632 POINT(41.252404158015025 74.99727668220913) bank195632 +195633 POINT(40.714116280210945 73.0666499562567) bank195633 +195634 POINT(41.289501752969365 74.00115590384291) bank195634 +195635 POINT(39.88494024788364 73.0563694849725) bank195635 +195636 POINT(41.64020256702214 74.84016668124507) bank195636 +195637 POINT(40.76584013105092 73.3294689031523) bank195637 +195638 POINT(41.707388073865175 74.90713367822684) bank195638 +195639 POINT(40.240197569926934 74.49406917621071) bank195639 +195640 POINT(41.63577960914367 74.94157551454111) bank195640 +195641 POINT(41.22974117733163 73.68947324776151) bank195641 +195642 POINT(39.86114065100945 74.34526215675386) bank195642 +195643 POINT(40.58665376562391 73.06573222771755) bank195643 +195644 POINT(39.797483090005194 74.7937398238651) bank195644 +195645 POINT(40.187904229153155 73.53320522956068) bank195645 +195646 POINT(40.80952373918321 74.87096467556917) bank195646 +195647 POINT(40.93001624460025 73.20474462465401) bank195647 +195648 POINT(41.58128061275743 74.28931832645296) bank195648 +195649 POINT(40.55582188833144 74.66187907648447) bank195649 +195650 POINT(41.112868899924614 73.93217713667154) bank195650 +195651 POINT(40.39908471018317 73.75929910926214) bank195651 +195652 POINT(40.20412266077004 74.39603451120212) bank195652 +195653 POINT(39.98523009201676 74.73045109246868) bank195653 +195654 POINT(41.671184624877625 73.60244012306039) bank195654 +195655 POINT(40.01621891037435 74.70523213147266) bank195655 +195656 POINT(39.90762713815648 74.14312691896696) bank195656 +195657 POINT(41.02373640809649 73.37764359871) bank195657 +195658 POINT(40.72466501757953 73.20893947967969) bank195658 +195659 POINT(41.681255670321804 73.82574237215843) bank195659 +195660 POINT(40.17916575351727 73.39039564796177) bank195660 +195661 POINT(39.904931018224076 73.40975207391715) bank195661 +195662 POINT(40.811288328653326 74.59258322850616) bank195662 +195663 POINT(40.93818534750569 74.18596225737824) bank195663 +195664 POINT(41.441679954825645 74.98792340406202) bank195664 +195665 POINT(41.42027462533355 73.85915882962836) bank195665 +195666 POINT(40.35674214748804 73.22768913426476) bank195666 +195667 POINT(40.89566682842064 74.43136066633416) bank195667 +195668 POINT(41.158580036244096 74.58450579639414) bank195668 +195669 POINT(41.229840780000345 74.9015594703482) bank195669 +195670 POINT(40.03710832718164 74.57046781712063) bank195670 +195671 POINT(40.44449156073807 74.41371410221606) bank195671 +195672 POINT(41.38124270293262 74.98406332794644) bank195672 +195673 POINT(41.529940049667495 73.02262702628181) bank195673 +195674 POINT(41.16689169078114 74.72859782568825) bank195674 +195675 POINT(41.41288408513918 73.54003166058722) bank195675 +195676 POINT(40.76409055893231 74.64740866646348) bank195676 +195677 POINT(40.54329370918085 74.87066630866376) bank195677 +195678 POINT(41.177607348746896 73.40759961426785) bank195678 +195679 POINT(40.446954958650224 74.51663644789674) bank195679 +195680 POINT(40.579880957491284 73.25267307672866) bank195680 +195681 POINT(41.6723068228259 73.32177047991308) bank195681 +195682 POINT(40.29048834191839 73.57567563972387) bank195682 +195683 POINT(41.384938704255404 74.33821846915234) bank195683 +195684 POINT(40.196012855001726 74.90410909679322) bank195684 +195685 POINT(41.14054618737874 73.390428662795) bank195685 +195686 POINT(41.397145171101386 74.93284786678119) bank195686 +195687 POINT(40.44631513955932 73.65676017760397) bank195687 +195688 POINT(40.21442556178111 73.551049337541) bank195688 +195689 POINT(40.298477634959944 74.69969976348777) bank195689 +195690 POINT(40.22739087100033 73.24804593593252) bank195690 +195691 POINT(39.75965667171663 73.26177342653259) bank195691 +195692 POINT(41.56660115977935 74.40195185670055) bank195692 +195693 POINT(41.08805574820285 73.5296465867978) bank195693 +195694 POINT(40.39215697073117 74.1929113281327) bank195694 +195695 POINT(41.5931443745074 74.44504454707152) bank195695 +195696 POINT(41.019374404438366 74.11588264557844) bank195696 +195697 POINT(40.130500779698494 74.15949349750252) bank195697 +195698 POINT(40.248495915264556 74.54804974801604) bank195698 +195699 POINT(40.90336840211535 73.89281848131982) bank195699 +195700 POINT(40.62012742905124 74.99931027619296) bank195700 +195701 POINT(41.3034566618488 73.83185610205007) bank195701 +195702 POINT(41.413978446492834 73.61398393109056) bank195702 +195703 POINT(40.59529468800373 73.45355202213153) bank195703 +195704 POINT(40.29114796918894 73.65724454735435) bank195704 +195705 POINT(40.83689063171282 73.7974107261556) bank195705 +195706 POINT(40.93747880667845 73.69533269478265) bank195706 +195707 POINT(40.612638674554354 73.86385960516286) bank195707 +195708 POINT(40.37022150268343 73.9499349767536) bank195708 +195709 POINT(41.35680994068697 73.66119237862821) bank195709 +195710 POINT(41.03179132772718 74.42591654215177) bank195710 +195711 POINT(41.285630939875844 74.32501124466134) bank195711 +195712 POINT(41.07937809210057 73.26357423066425) bank195712 +195713 POINT(41.47458386369185 73.41730429772166) bank195713 +195714 POINT(40.01970127356458 74.17531749650131) bank195714 +195715 POINT(39.73651705498847 74.87131622803943) bank195715 +195716 POINT(40.148241951532114 73.96632408645793) bank195716 +195717 POINT(40.2390435625719 74.28663367622775) bank195717 +195718 POINT(40.37106150621341 74.95472364214345) bank195718 +195719 POINT(40.6149012701574 73.86756338032569) bank195719 +195720 POINT(40.814967811911394 73.57245279002852) bank195720 +195721 POINT(41.28202451371261 74.87162473076671) bank195721 +195722 POINT(40.71789777628713 73.62242875097641) bank195722 +195723 POINT(40.14809500979708 73.75367961726778) bank195723 +195724 POINT(40.09689445573834 74.50448582431744) bank195724 +195725 POINT(40.15431550709793 73.5346189743447) bank195725 +195726 POINT(41.133612369753834 73.77041422906274) bank195726 +195727 POINT(40.58870418572171 74.83760000220187) bank195727 +195728 POINT(40.72712614963233 74.2410683492954) bank195728 +195729 POINT(40.049518120950005 73.06321660520234) bank195729 +195730 POINT(39.769145006412174 74.42870522249795) bank195730 +195731 POINT(40.10931620039774 74.89368830005937) bank195731 +195732 POINT(40.00584888797084 74.51441274755523) bank195732 +195733 POINT(40.16411412035548 73.31525718629823) bank195733 +195734 POINT(39.87975182352326 73.3315995650736) bank195734 +195735 POINT(41.651790377491544 73.6380592905321) bank195735 +195736 POINT(41.15378089444529 74.82879024415492) bank195736 +195737 POINT(41.26577799661097 74.87590711559781) bank195737 +195738 POINT(39.880738599329206 73.53142752378653) bank195738 +195739 POINT(40.21925644057199 74.45756804386193) bank195739 +195740 POINT(40.299714857876204 74.82900952247847) bank195740 +195741 POINT(41.50733634854479 73.47758810725354) bank195741 +195742 POINT(40.956532506150815 73.17623382593032) bank195742 +195743 POINT(40.59997669128846 74.6159565068055) bank195743 +195744 POINT(40.37244447625632 73.35776428279371) bank195744 +195745 POINT(39.92491509541474 73.74233013264032) bank195745 +195746 POINT(39.94219750303166 73.10935968979346) bank195746 +195747 POINT(40.10185664203051 74.7827083972761) bank195747 +195748 POINT(41.60762474604096 73.87476363078855) bank195748 +195749 POINT(40.41545865733565 73.8442829814417) bank195749 +195750 POINT(40.44526569199454 74.73198965781589) bank195750 +195751 POINT(39.88231863405071 73.15469055703012) bank195751 +195752 POINT(39.72898945195671 74.53948078081478) bank195752 +195753 POINT(41.33125043365297 73.03433273294674) bank195753 +195754 POINT(40.43801615045962 73.97887310358414) bank195754 +195755 POINT(41.586428378168975 74.68147580749906) bank195755 +195756 POINT(40.1283515261741 74.96648301315665) bank195756 +195757 POINT(40.0917117775899 73.59222281353314) bank195757 +195758 POINT(41.21835359667328 74.54608591514433) bank195758 +195759 POINT(40.14446736216741 73.15495801948539) bank195759 +195760 POINT(40.24728017143353 74.88791340367882) bank195760 +195761 POINT(41.31493501428083 73.73260348103783) bank195761 +195762 POINT(40.329854201777636 74.3708222608047) bank195762 +195763 POINT(40.5827561561624 74.51221380138902) bank195763 +195764 POINT(41.633285304928336 74.13336778849904) bank195764 +195765 POINT(40.14271469069087 74.13670840979269) bank195765 +195766 POINT(40.751515466153286 74.05431549971513) bank195766 +195767 POINT(41.24723937409061 73.93371332940752) bank195767 +195768 POINT(39.980183155442404 74.01460119153177) bank195768 +195769 POINT(41.222063774366106 74.18840014472684) bank195769 +195770 POINT(41.45065017426587 74.86562469624) bank195770 +195771 POINT(41.34071009921558 74.61205242015475) bank195771 +195772 POINT(40.56910195565479 73.48476211530378) bank195772 +195773 POINT(41.08778550979786 73.2471458282514) bank195773 +195774 POINT(41.27369699935611 74.96926163088298) bank195774 +195775 POINT(39.72512708143525 74.4955481425976) bank195775 +195776 POINT(40.75752211385409 74.1041984987342) bank195776 +195777 POINT(40.93128775431982 73.94777435659023) bank195777 +195778 POINT(39.7326870397492 73.49090402769899) bank195778 +195779 POINT(40.261151181133904 74.78894572502972) bank195779 +195780 POINT(41.1993561285238 73.74039640647702) bank195780 +195781 POINT(41.29057155153338 73.28419075123693) bank195781 +195782 POINT(40.68440715305487 74.70708087954557) bank195782 +195783 POINT(41.49720700672152 73.08217000049167) bank195783 +195784 POINT(39.92127766088169 73.31656676924626) bank195784 +195785 POINT(41.52271732570009 74.86427306350258) bank195785 +195786 POINT(39.835465020233606 73.30371459786858) bank195786 +195787 POINT(40.00583907795722 74.44769061593495) bank195787 +195788 POINT(40.83358315928395 74.1938813712366) bank195788 +195789 POINT(40.74192864679622 74.25026047172577) bank195789 +195790 POINT(39.89949328806352 73.41854733129453) bank195790 +195791 POINT(40.82436805531308 74.72841999025304) bank195791 +195792 POINT(40.365389818319215 73.39147710988689) bank195792 +195793 POINT(40.52670134201711 73.53178896670873) bank195793 +195794 POINT(40.9840383398529 73.1907935180346) bank195794 +195795 POINT(40.724444809238065 73.3458412212673) bank195795 +195796 POINT(40.50665314465498 75.00188012877108) bank195796 +195797 POINT(41.151124469157736 74.49118882606815) bank195797 +195798 POINT(41.02452045210932 73.77326132318568) bank195798 +195799 POINT(41.33771372627575 73.49000819007207) bank195799 +195800 POINT(39.80789013007784 73.29888417585693) bank195800 +195801 POINT(40.31680810323195 74.00861597767486) bank195801 +195802 POINT(39.725888573953014 74.6471937564896) bank195802 +195803 POINT(40.0286715378174 73.69673508359125) bank195803 +195804 POINT(40.12006348482634 74.52261908283577) bank195804 +195805 POINT(40.099809339676234 74.41017749295042) bank195805 +195806 POINT(40.99404895134347 73.27080412100666) bank195806 +195807 POINT(41.0618730949793 73.98390837634619) bank195807 +195808 POINT(39.895439162725495 73.67829889819248) bank195808 +195809 POINT(39.95954407898423 73.72037520696273) bank195809 +195810 POINT(41.382145865801654 74.11021655316114) bank195810 +195811 POINT(39.73064358510167 73.91183488315589) bank195811 +195812 POINT(41.28208688719085 74.47430390975123) bank195812 +195813 POINT(41.04542425834489 73.68473584090569) bank195813 +195814 POINT(40.64104665906055 74.1145182197591) bank195814 +195815 POINT(40.53257402960399 74.43845867216677) bank195815 +195816 POINT(41.490411871332455 74.23304378535826) bank195816 +195817 POINT(40.14751327781486 74.11643640157135) bank195817 +195818 POINT(40.79167554937752 74.83550096651716) bank195818 +195819 POINT(41.48037269442866 73.02034415102132) bank195819 +195820 POINT(39.876827153999756 74.57981931222977) bank195820 +195821 POINT(40.65718256283504 74.91088886716112) bank195821 +195822 POINT(40.618564447529465 74.59587153127242) bank195822 +195823 POINT(40.902768600975826 75.00125129128863) bank195823 +195824 POINT(41.0009898977279 73.58902790580576) bank195824 +195825 POINT(41.41629876252919 73.09886011639567) bank195825 +195826 POINT(40.71129434327545 74.35239494748396) bank195826 +195827 POINT(41.02141018195078 74.06020249512088) bank195827 +195828 POINT(40.95252572542798 74.09579577381749) bank195828 +195829 POINT(41.31824288838875 73.61128257532522) bank195829 +195830 POINT(39.80644567374546 73.65174103598301) bank195830 +195831 POINT(41.161853553193886 74.37745726413837) bank195831 +195832 POINT(39.71482752308379 73.17779909275808) bank195832 +195833 POINT(40.710202060038405 74.75788956064382) bank195833 +195834 POINT(39.93919561610743 73.89705877236747) bank195834 +195835 POINT(40.21223356113789 73.4082040120911) bank195835 +195836 POINT(40.47465539764038 73.73367743242704) bank195836 +195837 POINT(40.53296284081536 74.64046100592998) bank195837 +195838 POINT(40.5813475127045 74.30879518146418) bank195838 +195839 POINT(41.03071885860093 73.66152495072923) bank195839 +195840 POINT(40.16092535375127 74.93564666594038) bank195840 +195841 POINT(40.551630396342816 74.30609499064064) bank195841 +195842 POINT(41.68047435786371 73.87595472896157) bank195842 +195843 POINT(41.02888813244758 73.34268874850606) bank195843 +195844 POINT(40.56485060320219 73.72441040704719) bank195844 +195845 POINT(41.560739190336676 73.70698369382707) bank195845 +195846 POINT(40.39063024627873 73.36534209821413) bank195846 +195847 POINT(39.78711723482192 73.93412619359248) bank195847 +195848 POINT(40.42396288395163 73.9361438860158) bank195848 +195849 POINT(41.283099552997854 73.13646364145526) bank195849 +195850 POINT(40.45935045395996 73.04876947556382) bank195850 +195851 POINT(41.173850065988724 73.62074257921604) bank195851 +195852 POINT(40.23662752926407 73.8078233076289) bank195852 +195853 POINT(40.66017408221995 74.35001438147546) bank195853 +195854 POINT(39.97505461344689 74.80513991733845) bank195854 +195855 POINT(39.82635994506832 74.03215846358499) bank195855 +195856 POINT(41.47058599513143 74.01274492587288) bank195856 +195857 POINT(41.39056679668881 74.15609325221209) bank195857 +195858 POINT(40.64008737149603 74.98555552254834) bank195858 +195859 POINT(39.80057970065055 73.42145294245836) bank195859 +195860 POINT(41.33865166088398 74.0352920521185) bank195860 +195861 POINT(41.3973074836139 73.18028688930049) bank195861 +195862 POINT(41.2280345277071 74.25172070854532) bank195862 +195863 POINT(40.56304954419082 74.71482597671344) bank195863 +195864 POINT(40.08739872198441 73.97983578549083) bank195864 +195865 POINT(39.72101923690646 74.46692941587821) bank195865 +195866 POINT(41.541614560649954 73.81333792649505) bank195866 +195867 POINT(40.2862757720778 73.28482379313739) bank195867 +195868 POINT(40.06909783070269 74.59561332677035) bank195868 +195869 POINT(40.11426346527918 73.56627449870489) bank195869 +195870 POINT(39.88442000463219 73.96381175779962) bank195870 +195871 POINT(41.36471349832343 74.56888677138737) bank195871 +195872 POINT(40.16937968720183 74.32575257091224) bank195872 +195873 POINT(40.01935645349647 74.75699504032262) bank195873 +195874 POINT(40.30004680013578 73.86501474974364) bank195874 +195875 POINT(40.05864421123466 73.50116237756373) bank195875 +195876 POINT(41.22637418043884 74.35305077344739) bank195876 +195877 POINT(40.29919972403509 74.75708322251626) bank195877 +195878 POINT(41.48957850277227 73.45411000657134) bank195878 +195879 POINT(40.37445918253019 74.41011592970477) bank195879 +195880 POINT(40.71805625626228 74.26938204627646) bank195880 +195881 POINT(39.96169574282545 73.89525765840146) bank195881 +195882 POINT(40.164415108285176 74.53604956123745) bank195882 +195883 POINT(39.89839340069882 73.75143583204762) bank195883 +195884 POINT(40.36917380262615 74.17668096108834) bank195884 +195885 POINT(41.578928057178445 73.49858361924593) bank195885 +195886 POINT(40.98895612649498 73.87706008752197) bank195886 +195887 POINT(39.90075041559791 73.81619295295442) bank195887 +195888 POINT(41.190267113784685 74.00255207133613) bank195888 +195889 POINT(40.59855260140042 74.71877748692285) bank195889 +195890 POINT(41.32476626981244 74.29162587114311) bank195890 +195891 POINT(41.07341245103176 74.5288148179113) bank195891 +195892 POINT(40.44102825248223 73.72104908084731) bank195892 +195893 POINT(41.14341328732347 73.42620939619869) bank195893 +195894 POINT(40.1847879065672 74.35637301194673) bank195894 +195895 POINT(41.64600034246492 74.07408661340182) bank195895 +195896 POINT(41.41636862090267 74.0756893300748) bank195896 +195897 POINT(41.40683310864204 73.06331250840718) bank195897 +195898 POINT(39.90249989938435 74.36178988283301) bank195898 +195899 POINT(41.02812751107687 73.50644283875683) bank195899 +195900 POINT(40.51205875323904 73.43538726711726) bank195900 +195901 POINT(39.9079270260335 73.72354014032993) bank195901 +195902 POINT(41.403466331704614 74.05698500687409) bank195902 +195903 POINT(39.720831049946476 73.25782421760177) bank195903 +195904 POINT(40.453579842972644 74.4410991230067) bank195904 +195905 POINT(40.8764199139271 74.61328240316713) bank195905 +195906 POINT(41.622163409151156 73.55420498484892) bank195906 +195907 POINT(41.60177794279781 74.07790490041116) bank195907 +195908 POINT(41.47601908146536 73.79578777260176) bank195908 +195909 POINT(40.26650449052858 73.69638640705051) bank195909 +195910 POINT(41.45491317421806 73.28465168740514) bank195910 +195911 POINT(40.562266711589 73.14547671353793) bank195911 +195912 POINT(41.0210828390818 73.06805980872367) bank195912 +195913 POINT(41.201359309202424 74.53318927736515) bank195913 +195914 POINT(41.703156969913024 73.68184542793537) bank195914 +195915 POINT(40.42758147572435 73.95084392705449) bank195915 +195916 POINT(40.47369089858747 73.10105073476151) bank195916 +195917 POINT(40.90148798125383 73.6389955882754) bank195917 +195918 POINT(40.87624140829722 74.31495409034784) bank195918 +195919 POINT(41.26560456365867 73.22251666327043) bank195919 +195920 POINT(40.95774596549993 74.92241238804657) bank195920 +195921 POINT(40.762827289387204 74.93870453685052) bank195921 +195922 POINT(40.466390007001536 73.39484563332765) bank195922 +195923 POINT(40.80806971600999 73.43824577251802) bank195923 +195924 POINT(41.34840991797508 74.78543793593845) bank195924 +195925 POINT(40.02917538322323 74.80743980851445) bank195925 +195926 POINT(39.732918058758116 74.66596103734585) bank195926 +195927 POINT(39.80770181948779 73.14508435485236) bank195927 +195928 POINT(40.93125161347475 74.84406635475905) bank195928 +195929 POINT(40.53970125981696 74.3717180310219) bank195929 +195930 POINT(41.58546262719338 74.89212144399698) bank195930 +195931 POINT(40.29439372735558 74.1606073573712) bank195931 +195932 POINT(40.57870133844453 74.9877614679324) bank195932 +195933 POINT(40.44837941512409 74.59235798857584) bank195933 +195934 POINT(40.83901726325184 74.03135562174252) bank195934 +195935 POINT(40.780362066257965 73.29177325602606) bank195935 +195936 POINT(41.413387460223305 74.29815487399931) bank195936 +195937 POINT(41.2635880955981 73.45773802000768) bank195937 +195938 POINT(41.37746043819001 74.55807270840778) bank195938 +195939 POINT(40.88359166136397 73.5981557994981) bank195939 +195940 POINT(41.192802586286575 73.0288829472912) bank195940 +195941 POINT(40.860035900210576 74.3302717669958) bank195941 +195942 POINT(40.495287347727036 74.85656611951676) bank195942 +195943 POINT(40.8701363693572 73.29030608245736) bank195943 +195944 POINT(40.12580117070087 73.21981228290615) bank195944 +195945 POINT(41.27483938445814 73.96765884745366) bank195945 +195946 POINT(40.27898008584985 74.5241357092403) bank195946 +195947 POINT(39.978415327616204 74.02815337584732) bank195947 +195948 POINT(39.743593721052044 74.60980673609843) bank195948 +195949 POINT(41.3612536679569 74.88262387815485) bank195949 +195950 POINT(40.564242599606956 73.80113311913175) bank195950 +195951 POINT(41.00287176379859 74.88527637360822) bank195951 +195952 POINT(40.36966508363746 74.7864804995146) bank195952 +195953 POINT(40.916551109208214 73.95106095531362) bank195953 +195954 POINT(41.48918498555752 74.49269883782641) bank195954 +195955 POINT(40.87420984291015 73.59258088636058) bank195955 +195956 POINT(39.72310168771369 73.02831986340257) bank195956 +195957 POINT(40.517780733853236 73.66417835656041) bank195957 +195958 POINT(40.30703300244626 74.19222299388194) bank195958 +195959 POINT(40.93530308469094 73.21518195341928) bank195959 +195960 POINT(41.61284549204778 73.22824886308193) bank195960 +195961 POINT(40.92924115942576 74.83082438915011) bank195961 +195962 POINT(40.67575323630135 73.63882576550593) bank195962 +195963 POINT(40.0338788095726 73.8710243754964) bank195963 +195964 POINT(41.145523498918664 73.76914190889168) bank195964 +195965 POINT(40.87286407116033 74.10014569250814) bank195965 +195966 POINT(39.87395511658176 73.61040998065478) bank195966 +195967 POINT(40.85206621922531 73.55640813397441) bank195967 +195968 POINT(41.07899991551342 74.63220480577885) bank195968 +195969 POINT(40.18820489216989 73.93232040581853) bank195969 +195970 POINT(39.842255811711624 74.66199304708799) bank195970 +195971 POINT(39.920491974408364 73.45675837822039) bank195971 +195972 POINT(41.2421243886277 74.43088951437979) bank195972 +195973 POINT(41.18352566776747 73.91220664111118) bank195973 +195974 POINT(40.39208299221432 73.13743253153122) bank195974 +195975 POINT(39.77167666814615 73.01929467404135) bank195975 +195976 POINT(39.93449053004453 74.19986269652973) bank195976 +195977 POINT(41.499531007115095 74.38911318774383) bank195977 +195978 POINT(39.867988404281746 73.16424278913765) bank195978 +195979 POINT(39.771157429960574 73.03522670915967) bank195979 +195980 POINT(39.985759431237895 74.17255322873888) bank195980 +195981 POINT(40.19378160506872 73.58026056289947) bank195981 +195982 POINT(40.494218872386234 74.69790570927199) bank195982 +195983 POINT(41.644495117477014 73.57666411983469) bank195983 +195984 POINT(41.11999352993938 73.45391764957007) bank195984 +195985 POINT(40.36606503071073 74.4972249934631) bank195985 +195986 POINT(39.76821339386194 74.64639896574988) bank195986 +195987 POINT(41.133196687781414 73.91872988942038) bank195987 +195988 POINT(40.16665293307613 73.48623006167156) bank195988 +195989 POINT(41.46585968496157 74.16953249731552) bank195989 +195990 POINT(41.38585843513644 73.20050500656257) bank195990 +195991 POINT(40.49059896247488 74.39071601375112) bank195991 +195992 POINT(41.65303090674364 74.63428333064103) bank195992 +195993 POINT(41.51190564587737 73.40841036000504) bank195993 +195994 POINT(41.31249344909541 73.73244628426305) bank195994 +195995 POINT(41.12505577264257 73.77854157656195) bank195995 +195996 POINT(40.478594563639625 74.12738093998229) bank195996 +195997 POINT(40.34893376964846 74.11600924004793) bank195997 +195998 POINT(41.18046389706994 74.42899565636401) bank195998 +195999 POINT(40.87099128039275 74.11943449391669) bank195999 +196000 POINT(40.9323436994816 73.73171005569269) bank196000 +196001 POINT(39.77196394626193 74.23208587158177) bank196001 +196002 POINT(41.16809113659741 73.79524154814239) bank196002 +196003 POINT(41.08725289908227 73.78023379413993) bank196003 +196004 POINT(40.80764879059095 74.1518369355086) bank196004 +196005 POINT(40.039483937633065 74.76721933035417) bank196005 +196006 POINT(40.562596181127866 74.48832378596094) bank196006 +196007 POINT(40.426814087156934 73.18509792355889) bank196007 +196008 POINT(40.970848915605764 74.29438387551527) bank196008 +196009 POINT(40.610704171428 73.07642148540428) bank196009 +196010 POINT(41.49404022686806 73.24788345478157) bank196010 +196011 POINT(40.23140338582057 74.83670850753505) bank196011 +196012 POINT(40.05945172583564 73.50823465273687) bank196012 +196013 POINT(41.55858316467923 73.9861179347645) bank196013 +196014 POINT(40.9463598090604 74.4421466939289) bank196014 +196015 POINT(40.164261879626096 74.70369301089764) bank196015 +196016 POINT(40.10087320460408 74.4988846609987) bank196016 +196017 POINT(40.601492028906385 74.3959504977521) bank196017 +196018 POINT(40.249715582069555 73.43183156214099) bank196018 +196019 POINT(40.76830917394211 73.21284688914106) bank196019 +196020 POINT(41.03228318205494 73.2865033798273) bank196020 +196021 POINT(40.8004994808092 73.337860993767) bank196021 +196022 POINT(41.629946703876485 73.1204999309852) bank196022 +196023 POINT(41.27552665286659 73.57850194996564) bank196023 +196024 POINT(41.4533287383227 74.99742420269494) bank196024 +196025 POINT(41.25297801484336 73.59766907326852) bank196025 +196026 POINT(41.272026677578864 73.80636081039181) bank196026 +196027 POINT(40.96805828426358 74.63146402891458) bank196027 +196028 POINT(41.56568446910281 73.129643430335) bank196028 +196029 POINT(40.29010229330288 74.54540718954269) bank196029 +196030 POINT(41.01558146179899 74.29717416563058) bank196030 +196031 POINT(40.41818500504662 73.43421964293941) bank196031 +196032 POINT(41.542640451126275 74.96276445613475) bank196032 +196033 POINT(41.01160025713446 74.50392103494863) bank196033 +196034 POINT(41.30805474696884 73.74733392264675) bank196034 +196035 POINT(41.57302116409264 73.84089890531142) bank196035 +196036 POINT(40.70408459032929 73.33917593186457) bank196036 +196037 POINT(40.46455103180536 73.83314201852481) bank196037 +196038 POINT(40.57512380720554 74.13442389620825) bank196038 +196039 POINT(41.38403303033709 73.98970869710756) bank196039 +196040 POINT(41.444280383777325 73.98012448261184) bank196040 +196041 POINT(39.8506551930642 74.54619556302936) bank196041 +196042 POINT(41.445734462978805 73.70263843632787) bank196042 +196043 POINT(40.92662579050546 74.35974795699299) bank196043 +196044 POINT(40.94032727386354 73.45049204396773) bank196044 +196045 POINT(41.37959599068513 73.87005642454704) bank196045 +196046 POINT(40.06011051185483 73.58404487188396) bank196046 +196047 POINT(40.7144501710035 74.57092582799946) bank196047 +196048 POINT(40.7444103997991 73.38644741353743) bank196048 +196049 POINT(41.08454418086327 73.82257792968531) bank196049 +196050 POINT(40.35563543984024 74.17219284917525) bank196050 +196051 POINT(41.04331678441211 73.94394808295915) bank196051 +196052 POINT(40.36285128531668 73.8543413885763) bank196052 +196053 POINT(40.7837182232736 74.09640113118455) bank196053 +196054 POINT(40.66448400946014 73.62984173122635) bank196054 +196055 POINT(39.952456758429875 74.01899954893489) bank196055 +196056 POINT(41.10027868803492 73.39570971338112) bank196056 +196057 POINT(40.05134382021631 74.70942025118657) bank196057 +196058 POINT(40.1656519256949 74.96066633028103) bank196058 +196059 POINT(40.26423571721279 74.20275553929005) bank196059 +196060 POINT(40.29344486600198 73.80450317958596) bank196060 +196061 POINT(40.154928867431224 74.50503620834195) bank196061 +196062 POINT(41.452486565202854 74.62670152442455) bank196062 +196063 POINT(40.55369148726999 73.91368813605968) bank196063 +196064 POINT(39.73669349231175 74.86613174361814) bank196064 +196065 POINT(41.28395737713168 74.70463814695697) bank196065 +196066 POINT(40.77849886447204 73.2319672516041) bank196066 +196067 POINT(39.86330233084196 73.2105392192223) bank196067 +196068 POINT(39.729813863847426 73.60695080514293) bank196068 +196069 POINT(41.00507885254379 74.25957122768767) bank196069 +196070 POINT(41.434881931862456 73.58428366856664) bank196070 +196071 POINT(39.972241535936114 74.9458287434361) bank196071 +196072 POINT(40.42737304944703 73.79902120380372) bank196072 +196073 POINT(39.917982053271686 74.89560495888757) bank196073 +196074 POINT(40.12670942925348 73.57828989886156) bank196074 +196075 POINT(39.9212991540763 74.67927037057477) bank196075 +196076 POINT(39.78579186483552 73.21896708330526) bank196076 +196077 POINT(41.67633605053202 74.75359946733951) bank196077 +196078 POINT(40.20668109268873 73.60590351243262) bank196078 +196079 POINT(40.34112192508167 74.7267327401278) bank196079 +196080 POINT(40.2542581013596 74.13757815629174) bank196080 +196081 POINT(40.34340570094954 74.45673808425224) bank196081 +196082 POINT(41.67393445724073 74.0431304499856) bank196082 +196083 POINT(40.43787990234113 73.9863185647364) bank196083 +196084 POINT(39.94300591160365 74.70452940814498) bank196084 +196085 POINT(40.08565126703309 74.70595559041439) bank196085 +196086 POINT(39.862961290684964 74.1225174646635) bank196086 +196087 POINT(40.12929497350792 73.8875346214109) bank196087 +196088 POINT(40.11177850900896 73.68022493163674) bank196088 +196089 POINT(40.604820270834615 73.27930532459578) bank196089 +196090 POINT(40.345501099625004 73.74851539116858) bank196090 +196091 POINT(39.91508822336322 73.99202935129352) bank196091 +196092 POINT(40.890756472478365 74.76451103678662) bank196092 +196093 POINT(40.056215201582894 74.12273799766669) bank196093 +196094 POINT(39.90329568735411 74.64421365871908) bank196094 +196095 POINT(41.056850425191385 74.41819066453147) bank196095 +196096 POINT(40.56745272633045 73.66605483791298) bank196096 +196097 POINT(41.610353670481516 74.09426330724415) bank196097 +196098 POINT(41.127574259650245 73.30492408696418) bank196098 +196099 POINT(39.8381815515152 73.63172357258168) bank196099 +196100 POINT(41.69632135434515 73.45482003954041) bank196100 +196101 POINT(41.65846455407687 73.98807789669576) bank196101 +196102 POINT(41.25543655069126 74.8796695692391) bank196102 +196103 POINT(41.43208194695877 73.0519133536164) bank196103 +196104 POINT(40.18146015556205 74.80858847694043) bank196104 +196105 POINT(39.99358943027912 74.99162097289778) bank196105 +196106 POINT(40.93409294648957 73.76012230057003) bank196106 +196107 POINT(40.9829006414995 74.12104969521171) bank196107 +196108 POINT(40.32745228000316 73.30562714014627) bank196108 +196109 POINT(40.11992819878238 73.55447085241042) bank196109 +196110 POINT(41.14384778323106 73.24101843391198) bank196110 +196111 POINT(40.28162597750478 73.49418678199812) bank196111 +196112 POINT(39.818391113957105 74.90885563115168) bank196112 +196113 POINT(40.62557370907142 74.42151813797685) bank196113 +196114 POINT(41.15952418101976 73.93606068187233) bank196114 +196115 POINT(41.711411320154795 73.2669031757002) bank196115 +196116 POINT(40.19798239708427 74.49289932403423) bank196116 +196117 POINT(41.66071116653063 73.10728344700391) bank196117 +196118 POINT(41.46230457862744 73.68525372292699) bank196118 +196119 POINT(40.07510584998311 73.18035853484922) bank196119 +196120 POINT(40.51180623562895 74.06704070343308) bank196120 +196121 POINT(40.076807674733644 73.87369181418325) bank196121 +196122 POINT(39.88219093322803 73.6722117017254) bank196122 +196123 POINT(39.97376474812079 73.95348787477367) bank196123 +196124 POINT(40.5038017711759 73.21017386296471) bank196124 +196125 POINT(41.69202633703533 74.65130994506723) bank196125 +196126 POINT(39.893780639325705 74.52954041317021) bank196126 +196127 POINT(41.3563340693471 73.5646795806) bank196127 +196128 POINT(39.95248901613394 73.45686422065754) bank196128 +196129 POINT(41.54509694665301 74.43180032664922) bank196129 +196130 POINT(40.61858322499361 73.43370402188778) bank196130 +196131 POINT(41.495427823827235 73.3488975244873) bank196131 +196132 POINT(41.32700333582563 74.51696670970804) bank196132 +196133 POINT(40.61218904910777 73.6308229309088) bank196133 +196134 POINT(40.277878640643976 73.32129753939124) bank196134 +196135 POINT(40.35887771920281 74.3663929303387) bank196135 +196136 POINT(40.7743144643603 74.31727728319225) bank196136 +196137 POINT(41.2545782397847 74.60165904982351) bank196137 +196138 POINT(41.32470154824668 73.68739071664616) bank196138 +196139 POINT(41.485380104298635 74.97517522802677) bank196139 +196140 POINT(40.721520913550165 73.06076238101925) bank196140 +196141 POINT(40.02911996765117 73.54851934064028) bank196141 +196142 POINT(41.31985140838395 73.03155019579624) bank196142 +196143 POINT(39.98086678648171 73.34059350985405) bank196143 +196144 POINT(40.589881080128656 73.38534574461414) bank196144 +196145 POINT(40.082409213230925 73.57098562366508) bank196145 +196146 POINT(40.27327513053195 74.68710193630926) bank196146 +196147 POINT(40.6058975767387 73.25428977049857) bank196147 +196148 POINT(41.24770783839338 73.98788221385234) bank196148 +196149 POINT(41.62528658206768 73.22169146276882) bank196149 +196150 POINT(41.557200439474556 74.46763836602837) bank196150 +196151 POINT(41.32876667250039 73.1848387262675) bank196151 +196152 POINT(41.3052456712194 74.82436879012239) bank196152 +196153 POINT(41.67763398912745 73.90835608562247) bank196153 +196154 POINT(40.884523671997094 74.926713856967) bank196154 +196155 POINT(40.068505157959 74.84196755114672) bank196155 +196156 POINT(40.01434621474451 73.91405352362787) bank196156 +196157 POINT(40.64343630426421 73.42892581615178) bank196157 +196158 POINT(41.336344241644724 73.20439126772757) bank196158 +196159 POINT(40.630749564391884 73.49198991560189) bank196159 +196160 POINT(40.520210272294136 74.82640817740996) bank196160 +196161 POINT(41.47614315753778 74.01024122774257) bank196161 +196162 POINT(41.62466441907494 74.56720694630343) bank196162 +196163 POINT(41.550756112559185 73.9784738093767) bank196163 +196164 POINT(41.04777168304035 73.88899746753648) bank196164 +196165 POINT(40.826811881215725 73.85829672580186) bank196165 +196166 POINT(40.511566844418745 74.22060913866498) bank196166 +196167 POINT(41.515558927016684 74.45640631673393) bank196167 +196168 POINT(41.66493116630908 73.32852077860895) bank196168 +196169 POINT(41.45086540911081 74.78068510240315) bank196169 +196170 POINT(40.938796744683145 73.61336053572406) bank196170 +196171 POINT(40.63318479581191 74.96590093335716) bank196171 +196172 POINT(40.35387549360675 73.7713517267623) bank196172 +196173 POINT(41.11792269484862 73.25838913665216) bank196173 +196174 POINT(41.33958057210938 73.48666960872096) bank196174 +196175 POINT(41.61874570202243 73.54062424029904) bank196175 +196176 POINT(40.55988827489969 74.59987668333531) bank196176 +196177 POINT(41.021965981461676 73.69866519489814) bank196177 +196178 POINT(40.45325637430452 73.89700444246391) bank196178 +196179 POINT(41.63963955775292 73.32875988137496) bank196179 +196180 POINT(40.69637918953541 73.06415100915424) bank196180 +196181 POINT(41.128784750395376 73.87570208304754) bank196181 +196182 POINT(40.89966479834273 73.87902344922593) bank196182 +196183 POINT(40.73969848065369 74.88233360203544) bank196183 +196184 POINT(40.47198591554608 74.87118936689969) bank196184 +196185 POINT(41.41648613319976 73.20999802836525) bank196185 +196186 POINT(40.58031266756622 73.76887135725973) bank196186 +196187 POINT(41.42528708279322 73.55481295196788) bank196187 +196188 POINT(39.784624687547854 74.8790009179399) bank196188 +196189 POINT(41.66515225885355 74.86663218517984) bank196189 +196190 POINT(41.21167850080656 74.89101502486683) bank196190 +196191 POINT(39.777266307944316 74.56311458387975) bank196191 +196192 POINT(39.84893240565689 74.31903628558358) bank196192 +196193 POINT(40.93533315312812 73.77739558744601) bank196193 +196194 POINT(39.96113048708112 73.7506068871877) bank196194 +196195 POINT(39.81624533339765 73.6503129237143) bank196195 +196196 POINT(40.98953419427097 73.57217555258345) bank196196 +196197 POINT(41.56938297574226 74.52804899344396) bank196197 +196198 POINT(41.66494141750008 73.4900739421719) bank196198 +196199 POINT(40.724052742551116 74.52094564349854) bank196199 +196200 POINT(40.373001094059624 73.86641728141285) bank196200 +196201 POINT(40.206227656329155 73.94056649458197) bank196201 +196202 POINT(39.84557593080858 73.09212378912505) bank196202 +196203 POINT(40.16623385042264 74.63474153086156) bank196203 +196204 POINT(41.67889221046075 73.41538718314757) bank196204 +196205 POINT(39.767996236852575 73.44976888396744) bank196205 +196206 POINT(40.774651435974114 74.7047277705607) bank196206 +196207 POINT(40.44600539913299 74.53495110035435) bank196207 +196208 POINT(41.219539892934996 74.23005310683529) bank196208 +196209 POINT(40.74344397554203 74.05953243734857) bank196209 +196210 POINT(40.563282891453966 74.6715946110188) bank196210 +196211 POINT(41.52395828701469 74.95214721529211) bank196211 +196212 POINT(40.41502309099242 74.77485356333676) bank196212 +196213 POINT(41.16527252810918 73.56355764820493) bank196213 +196214 POINT(41.07271632352297 73.15038510726441) bank196214 +196215 POINT(40.33528569150482 73.98692207172928) bank196215 +196216 POINT(40.58507901690454 73.72756698204827) bank196216 +196217 POINT(41.12711491182107 73.63037049844965) bank196217 +196218 POINT(41.07047448976785 73.15174920629313) bank196218 +196219 POINT(39.91912669880545 73.38379057819469) bank196219 +196220 POINT(41.249581994866745 74.80440097712192) bank196220 +196221 POINT(41.62714502803422 73.19365342602465) bank196221 +196222 POINT(40.97659942379383 73.05714924991162) bank196222 +196223 POINT(40.059884845929936 74.49925273345993) bank196223 +196224 POINT(40.49336762584936 73.47858157768025) bank196224 +196225 POINT(40.94009179140021 74.17904081523892) bank196225 +196226 POINT(40.09219876356017 73.60795736134158) bank196226 +196227 POINT(40.59249108470067 74.57782958390378) bank196227 +196228 POINT(41.19996680573 73.89596690909924) bank196228 +196229 POINT(41.44012857767321 74.01032350454035) bank196229 +196230 POINT(40.9016142302042 74.24993266095984) bank196230 +196231 POINT(39.791516617458505 74.58028337993925) bank196231 +196232 POINT(41.28576526665292 74.34526901419956) bank196232 +196233 POINT(41.23643450266585 73.12265134141188) bank196233 +196234 POINT(40.715892696043355 74.14382740403393) bank196234 +196235 POINT(40.7477982039821 73.74290561634147) bank196235 +196236 POINT(39.935916553459094 74.78898540186805) bank196236 +196237 POINT(39.74933431032265 73.15419653331256) bank196237 +196238 POINT(39.71842869420863 74.5089797970102) bank196238 +196239 POINT(40.960249462317286 74.3663426733619) bank196239 +196240 POINT(41.55960670535081 73.61620792382902) bank196240 +196241 POINT(40.89016953744313 74.31658820063032) bank196241 +196242 POINT(40.49381781955327 74.57449764136207) bank196242 +196243 POINT(39.87315533405665 74.93215179151427) bank196243 +196244 POINT(39.78404501635677 74.54239908701902) bank196244 +196245 POINT(40.80090521016581 73.39114500681134) bank196245 +196246 POINT(40.91641825997358 73.15601262396706) bank196246 +196247 POINT(40.195233751185434 73.20006195545369) bank196247 +196248 POINT(41.55362893931406 73.26267253551886) bank196248 +196249 POINT(41.70100304169383 73.90930298971948) bank196249 +196250 POINT(41.240882472507415 74.99164186248332) bank196250 +196251 POINT(39.779125929638674 74.81948151675677) bank196251 +196252 POINT(40.359837519979166 74.49128361077524) bank196252 +196253 POINT(40.42426027290742 73.46486262633061) bank196253 +196254 POINT(39.879524955660266 73.52679286584157) bank196254 +196255 POINT(40.80326670228315 74.0340915062766) bank196255 +196256 POINT(41.03499031163353 73.7200626578998) bank196256 +196257 POINT(40.953407102952674 73.17706101690118) bank196257 +196258 POINT(40.069914299545225 74.2517374256453) bank196258 +196259 POINT(40.033969727285466 74.12855125351967) bank196259 +196260 POINT(41.36717994421831 74.45351237425099) bank196260 +196261 POINT(39.75205815762264 73.62573395026047) bank196261 +196262 POINT(40.69393134899741 74.68444534365005) bank196262 +196263 POINT(40.42867562759109 74.65678199742362) bank196263 +196264 POINT(40.37053486993767 73.94264205025488) bank196264 +196265 POINT(39.929973747569946 73.60468236538323) bank196265 +196266 POINT(40.62921227086584 73.02171704027809) bank196266 +196267 POINT(40.204810012588815 74.2326456894469) bank196267 +196268 POINT(41.378805868388646 74.55014066448769) bank196268 +196269 POINT(41.28360075451441 73.43228996398497) bank196269 +196270 POINT(41.07416391376013 73.84310443900712) bank196270 +196271 POINT(41.04444256410175 74.01282828030466) bank196271 +196272 POINT(40.27508873803083 74.43456288585108) bank196272 +196273 POINT(41.01533474209826 73.60620990029199) bank196273 +196274 POINT(40.53506630252302 73.07779616549608) bank196274 +196275 POINT(41.55199406224632 73.17606180988881) bank196275 +196276 POINT(40.719865796276046 73.42855119599879) bank196276 +196277 POINT(39.87690533492645 74.17196876576143) bank196277 +196278 POINT(40.484636452672774 74.611467358943) bank196278 +196279 POINT(41.67647219104319 73.72918634461942) bank196279 +196280 POINT(40.73765698663549 73.67958843942245) bank196280 +196281 POINT(41.31109901075764 74.62600226525153) bank196281 +196282 POINT(41.025620835084965 73.21177773257293) bank196282 +196283 POINT(40.381344757769874 73.27134153738787) bank196283 +196284 POINT(39.937448891400216 74.4445433344805) bank196284 +196285 POINT(39.9169176355823 73.97854855825537) bank196285 +196286 POINT(40.530402086575506 73.67201280544522) bank196286 +196287 POINT(40.007049469532326 73.52546837431602) bank196287 +196288 POINT(41.42996199826366 74.2088254101605) bank196288 +196289 POINT(40.23998770470537 73.60774287418838) bank196289 +196290 POINT(40.23313996845393 73.66118883402136) bank196290 +196291 POINT(39.99390218097681 73.02783763644472) bank196291 +196292 POINT(40.958891828810714 73.91673453042677) bank196292 +196293 POINT(40.52171269977069 73.93809889361556) bank196293 +196294 POINT(39.95307923285192 74.95084836359125) bank196294 +196295 POINT(41.092970484106345 74.0988524467509) bank196295 +196296 POINT(40.85525080881783 74.85009248978781) bank196296 +196297 POINT(40.66127241794552 73.71963051538545) bank196297 +196298 POINT(40.357408977625006 74.00242610442083) bank196298 +196299 POINT(40.404382091433874 73.04155390804404) bank196299 +196300 POINT(40.39421443834948 74.37938611461666) bank196300 +196301 POINT(40.255302141783446 73.25111654861871) bank196301 +196302 POINT(41.50023033997246 74.7661727774645) bank196302 +196303 POINT(40.94484528387457 74.80636234563205) bank196303 +196304 POINT(40.8363913705397 74.99230785226327) bank196304 +196305 POINT(40.747606352385695 73.59852552721554) bank196305 +196306 POINT(40.92429758909216 73.35203033142216) bank196306 +196307 POINT(41.697857717006244 73.29413827832188) bank196307 +196308 POINT(41.000845295129736 74.65961410943702) bank196308 +196309 POINT(41.71129046139488 73.5891126263983) bank196309 +196310 POINT(41.176877682890094 73.23959762853627) bank196310 +196311 POINT(40.62259000142879 74.11499583304158) bank196311 +196312 POINT(41.38669975159492 73.87139966167854) bank196312 +196313 POINT(41.34216887221484 73.28299413452243) bank196313 +196314 POINT(40.634029862453225 73.7018444286887) bank196314 +196315 POINT(40.022488244631106 73.40917018413387) bank196315 +196316 POINT(41.38161891516722 73.9223789824076) bank196316 +196317 POINT(39.91166269660606 73.9569833725411) bank196317 +196318 POINT(39.936765401289605 73.74269757805241) bank196318 +196319 POINT(40.316593686737946 73.3411197180441) bank196319 +196320 POINT(41.41376027578587 74.5133450121861) bank196320 +196321 POINT(40.08948338743335 73.2450910945927) bank196321 +196322 POINT(40.808960897448756 73.349267984546) bank196322 +196323 POINT(41.54352962245943 74.91008081803568) bank196323 +196324 POINT(40.05594977447133 73.78871030076436) bank196324 +196325 POINT(40.15757333927839 74.66607609109354) bank196325 +196326 POINT(39.95318231805875 73.94194639068753) bank196326 +196327 POINT(40.13727884442484 73.41957585261936) bank196327 +196328 POINT(40.38858421754901 73.13812401743883) bank196328 +196329 POINT(41.21586756698854 73.83055359814307) bank196329 +196330 POINT(41.27641418412478 73.61296773996975) bank196330 +196331 POINT(40.055832330272715 74.79417421746845) bank196331 +196332 POINT(40.56366084166937 74.63486624465575) bank196332 +196333 POINT(40.69810465236044 73.96386845278086) bank196333 +196334 POINT(40.53739280594149 73.15951731910438) bank196334 +196335 POINT(40.144974213745826 73.58375090391941) bank196335 +196336 POINT(40.21645323304533 74.28999090444465) bank196336 +196337 POINT(40.00443019468273 74.47928831794941) bank196337 +196338 POINT(40.15250611487324 74.98258979035079) bank196338 +196339 POINT(40.13272333499332 73.42222790766046) bank196339 +196340 POINT(40.902466511474664 74.68740442574597) bank196340 +196341 POINT(39.9921890715088 74.84226916734309) bank196341 +196342 POINT(41.22896861356472 73.8689124522005) bank196342 +196343 POINT(40.8425769756226 73.60167733558795) bank196343 +196344 POINT(40.99577915787578 73.23391864367376) bank196344 +196345 POINT(39.93718354641034 73.71101270620566) bank196345 +196346 POINT(40.71365128980554 74.43742928004237) bank196346 +196347 POINT(40.88116985094683 73.22991157550484) bank196347 +196348 POINT(40.56868140876312 74.15360797259306) bank196348 +196349 POINT(41.12179547490498 73.7012969832978) bank196349 +196350 POINT(40.86374079755602 73.75706081418859) bank196350 +196351 POINT(40.38582149637443 73.61270213860308) bank196351 +196352 POINT(40.40149024912359 73.45942357271717) bank196352 +196353 POINT(41.23561770166778 74.27331998697372) bank196353 +196354 POINT(41.09889120459419 73.0870866366066) bank196354 +196355 POINT(39.776315243737734 73.43239537211153) bank196355 +196356 POINT(40.688115538985556 74.39384160610517) bank196356 +196357 POINT(40.834882289206 74.8765387886122) bank196357 +196358 POINT(40.84901966066536 74.55078643902992) bank196358 +196359 POINT(40.205689186970815 74.88983828550833) bank196359 +196360 POINT(40.94411901720253 74.47752004336692) bank196360 +196361 POINT(40.69366978411322 73.68454729635317) bank196361 +196362 POINT(41.00919137191914 73.43958020228169) bank196362 +196363 POINT(39.848893478346696 74.97910215796776) bank196363 +196364 POINT(41.649392539503566 74.86524847635434) bank196364 +196365 POINT(41.11773495960732 73.44888962614951) bank196365 +196366 POINT(40.8091442180635 74.06771482516753) bank196366 +196367 POINT(41.20289961935782 73.89860279244901) bank196367 +196368 POINT(40.81083622937964 73.65435720220596) bank196368 +196369 POINT(41.18096221330023 74.65011467438197) bank196369 +196370 POINT(41.51467951914534 74.76963952675602) bank196370 +196371 POINT(40.23249220068911 74.42821481228441) bank196371 +196372 POINT(41.01311123478072 73.44390073803869) bank196372 +196373 POINT(40.54268449599643 73.7540425395496) bank196373 +196374 POINT(41.45219870843695 74.10565688805893) bank196374 +196375 POINT(39.72117489471879 73.34379286437009) bank196375 +196376 POINT(40.15222597168679 73.39365340429354) bank196376 +196377 POINT(40.958791649845416 73.86745169689131) bank196377 +196378 POINT(40.29561689187777 73.74822982919375) bank196378 +196379 POINT(39.868046639231494 74.04913589814524) bank196379 +196380 POINT(39.973716380887936 73.03318187252809) bank196380 +196381 POINT(40.44507860683843 73.49571027731564) bank196381 +196382 POINT(40.549450434373185 73.6798739586934) bank196382 +196383 POINT(40.53891028331918 73.92751172513404) bank196383 +196384 POINT(39.910694867313474 73.59285849493854) bank196384 +196385 POINT(39.991422512985864 74.69196985821657) bank196385 +196386 POINT(41.21429690304109 74.56537006152521) bank196386 +196387 POINT(41.17925810956714 74.84711067867735) bank196387 +196388 POINT(40.875345816596905 73.90347020492743) bank196388 +196389 POINT(41.17268975323937 73.56634670983429) bank196389 +196390 POINT(40.390284849050985 74.80820492191809) bank196390 +196391 POINT(40.32620938088654 73.64865034401983) bank196391 +196392 POINT(39.92333138321347 73.10651103417794) bank196392 +196393 POINT(40.748926226587855 73.87330654755594) bank196393 +196394 POINT(40.26538250247253 74.2362332104509) bank196394 +196395 POINT(40.82742993090872 73.25868217137108) bank196395 +196396 POINT(40.87031662215378 73.64000676296538) bank196396 +196397 POINT(41.37643688366854 73.3538255303234) bank196397 +196398 POINT(40.12453824180394 73.4122715893081) bank196398 +196399 POINT(39.74466771537242 73.5200647744731) bank196399 +196400 POINT(41.24161989233569 74.51468693151254) bank196400 +196401 POINT(41.013999565701994 73.36168525271994) bank196401 +196402 POINT(40.63189612830034 74.54664918766673) bank196402 +196403 POINT(41.13398147953738 73.16201589794917) bank196403 +196404 POINT(40.74418394637224 74.98814153021056) bank196404 +196405 POINT(41.610716342888374 74.35633938533756) bank196405 +196406 POINT(40.737953897715954 73.32474424331056) bank196406 +196407 POINT(41.474262064335406 73.12079635490026) bank196407 +196408 POINT(40.55831512805102 73.74928090915631) bank196408 +196409 POINT(40.13735900664409 74.03826608113907) bank196409 +196410 POINT(40.270762896160626 73.71327302959908) bank196410 +196411 POINT(41.167431943881475 74.38976261826934) bank196411 +196412 POINT(40.96484884999731 73.05515529155764) bank196412 +196413 POINT(39.96456208719479 73.89949371668861) bank196413 +196414 POINT(40.79359839188274 73.766427649093) bank196414 +196415 POINT(41.69476533819826 74.44954466001748) bank196415 +196416 POINT(41.025728426684275 74.93922225656266) bank196416 +196417 POINT(41.106804407388054 73.13101929425304) bank196417 +196418 POINT(40.89625816058277 74.0666358201753) bank196418 +196419 POINT(40.208635670332406 73.26987221108442) bank196419 +196420 POINT(41.17757006984356 74.2300970464013) bank196420 +196421 POINT(39.93513203588827 73.0722286022042) bank196421 +196422 POINT(40.03488503921513 73.1679947011261) bank196422 +196423 POINT(41.22170010774221 74.73297923696958) bank196423 +196424 POINT(40.61638377624499 73.74126780364337) bank196424 +196425 POINT(40.025621043406126 73.02638235279824) bank196425 +196426 POINT(39.82481866440469 73.75653675082266) bank196426 +196427 POINT(41.68738288890436 74.35390677490693) bank196427 +196428 POINT(41.61377689577716 74.73181333315014) bank196428 +196429 POINT(39.747665620454185 73.10020354741363) bank196429 +196430 POINT(41.40308866548272 74.99624840284889) bank196430 +196431 POINT(40.2216818336053 73.790547103642) bank196431 +196432 POINT(40.5471281191525 74.2590840102508) bank196432 +196433 POINT(40.964042843979065 74.41145603678225) bank196433 +196434 POINT(41.17599527419808 73.03153218264976) bank196434 +196435 POINT(41.36174500148662 73.58978102938359) bank196435 +196436 POINT(41.606828015696344 74.43655081995384) bank196436 +196437 POINT(41.61191009437339 73.62531618560818) bank196437 +196438 POINT(40.14877715308528 74.96341708185935) bank196438 +196439 POINT(40.920279211244875 74.72027031482888) bank196439 +196440 POINT(40.12275520479597 73.36410616472236) bank196440 +196441 POINT(40.42463690110146 74.18343202241905) bank196441 +196442 POINT(40.26409968354313 73.15204575649874) bank196442 +196443 POINT(41.702027467334844 73.03819455229947) bank196443 +196444 POINT(41.167463997239174 74.70108566713692) bank196444 +196445 POINT(39.99318126221584 73.26184645131906) bank196445 +196446 POINT(40.80259327038085 74.32318304721144) bank196446 +196447 POINT(41.273309794353274 74.91219674291838) bank196447 +196448 POINT(41.684883908303945 74.95563070409506) bank196448 +196449 POINT(40.157855866750296 74.88040272761458) bank196449 +196450 POINT(41.4025878018498 73.50893909514973) bank196450 +196451 POINT(40.05852321574912 73.63190464277545) bank196451 +196452 POINT(40.00066732134646 73.92263521326842) bank196452 +196453 POINT(41.224794752075276 73.30727392605513) bank196453 +196454 POINT(40.21535567902969 74.81321986529726) bank196454 +196455 POINT(39.924432395943995 73.55523674108525) bank196455 +196456 POINT(41.18089760124426 73.02793401277263) bank196456 +196457 POINT(40.68804102908548 74.37847083874725) bank196457 +196458 POINT(40.25732505629157 74.1069195567982) bank196458 +196459 POINT(40.0532224398457 73.29687765812452) bank196459 +196460 POINT(39.955701496206686 73.30068667650956) bank196460 +196461 POINT(41.707306419089754 74.46263144000304) bank196461 +196462 POINT(41.136230440680905 74.97912308055147) bank196462 +196463 POINT(40.4737673683679 73.64840092181312) bank196463 +196464 POINT(40.085940220635734 73.79584619793657) bank196464 +196465 POINT(41.48664408825919 73.25427143496556) bank196465 +196466 POINT(40.92843031415607 73.82442572011135) bank196466 +196467 POINT(40.17758390726021 74.2414653706575) bank196467 +196468 POINT(40.33149261304531 73.72454542300474) bank196468 +196469 POINT(40.043384954055824 74.44080708021563) bank196469 +196470 POINT(40.9895629181964 73.20129065685234) bank196470 +196471 POINT(40.01462674978482 74.5514625552068) bank196471 +196472 POINT(40.31400059322923 73.0642119481093) bank196472 +196473 POINT(40.61548786969769 73.56256456587171) bank196473 +196474 POINT(40.82698703543419 74.57717503725628) bank196474 +196475 POINT(41.38081658469059 73.2198086006621) bank196475 +196476 POINT(40.31516362139991 74.29115952418925) bank196476 +196477 POINT(41.42956227913041 73.55850101309535) bank196477 +196478 POINT(39.842977191591146 73.69989006646615) bank196478 +196479 POINT(41.55657408162074 74.69651367524905) bank196479 +196480 POINT(41.70825618004105 74.00398039664326) bank196480 +196481 POINT(41.05597170564351 73.8163978478011) bank196481 +196482 POINT(40.411055822285455 74.53847765171741) bank196482 +196483 POINT(39.812950086327284 73.5141241805546) bank196483 +196484 POINT(40.9642317819947 73.3647672764082) bank196484 +196485 POINT(41.34637218214928 74.53719041546935) bank196485 +196486 POINT(40.845893986079936 73.37514713953662) bank196486 +196487 POINT(41.13770397775886 74.32909329895766) bank196487 +196488 POINT(39.7632048452805 74.66168455625254) bank196488 +196489 POINT(41.470939275639374 74.89020755711854) bank196489 +196490 POINT(41.561151565619895 74.73171932261259) bank196490 +196491 POINT(40.13085304877875 74.35206575076414) bank196491 +196492 POINT(40.06345387941091 74.90553203486485) bank196492 +196493 POINT(40.79424900659257 73.06169952630208) bank196493 +196494 POINT(40.160705787414805 73.40669669596832) bank196494 +196495 POINT(40.74025878529059 74.48482023334937) bank196495 +196496 POINT(40.61898129841035 73.96296704865134) bank196496 +196497 POINT(39.988080950404346 74.61093598447869) bank196497 +196498 POINT(40.17775149159735 74.77965719565586) bank196498 +196499 POINT(41.20296364028211 73.74959495844313) bank196499 +196500 POINT(39.88476142716407 74.34336653855617) bank196500 +196501 POINT(41.03568489015715 74.02435212744862) bank196501 +196502 POINT(40.16204412055727 73.95484824343092) bank196502 +196503 POINT(41.59438867925828 74.7929110824908) bank196503 +196504 POINT(40.27530648275033 74.61089503412725) bank196504 +196505 POINT(40.91712966028163 73.94291780916646) bank196505 +196506 POINT(41.47659111916938 73.4702648628048) bank196506 +196507 POINT(41.45330707943338 73.20601468319738) bank196507 +196508 POINT(40.065297732711464 73.86029380252481) bank196508 +196509 POINT(40.611103413147994 74.33454893175572) bank196509 +196510 POINT(41.269237818772694 73.02715996694693) bank196510 +196511 POINT(40.004722554482484 73.74267909339513) bank196511 +196512 POINT(40.1964947720986 73.68595491440709) bank196512 +196513 POINT(41.669071207471674 73.81232983169063) bank196513 +196514 POINT(41.49007331489705 73.8625245941368) bank196514 +196515 POINT(41.01142351447738 73.6204123826881) bank196515 +196516 POINT(40.321845808059194 73.81087865976525) bank196516 +196517 POINT(41.5969148794024 73.75268115294227) bank196517 +196518 POINT(40.72981301328277 73.10460743994634) bank196518 +196519 POINT(40.23247522101979 74.20045331163578) bank196519 +196520 POINT(40.907472530011006 74.34393832718524) bank196520 +196521 POINT(41.712572715360984 73.12631966976522) bank196521 +196522 POINT(41.41036221437103 73.29840077668348) bank196522 +196523 POINT(41.23491266533648 73.57233248731123) bank196523 +196524 POINT(40.94283181184526 73.58821869860847) bank196524 +196525 POINT(41.50070254354338 73.43347272002113) bank196525 +196526 POINT(40.21165429888551 74.14544705533996) bank196526 +196527 POINT(40.548622274552656 73.99068055171706) bank196527 +196528 POINT(41.42003787102441 73.12069184373404) bank196528 +196529 POINT(40.676055122813594 74.7460145887752) bank196529 +196530 POINT(40.39931120835078 74.95284783498634) bank196530 +196531 POINT(41.15451928720551 74.94841445950117) bank196531 +196532 POINT(41.31643545340858 73.85687839680546) bank196532 +196533 POINT(40.11166730305274 74.16010228179462) bank196533 +196534 POINT(41.17609978341349 74.73750898565258) bank196534 +196535 POINT(40.81869445598282 74.82580326823224) bank196535 +196536 POINT(40.03378589655956 73.52280735258245) bank196536 +196537 POINT(40.49094813633758 73.87552618836615) bank196537 +196538 POINT(40.02999193443488 74.4155340530491) bank196538 +196539 POINT(40.992827521171925 73.81361611556635) bank196539 +196540 POINT(41.44001566583292 74.45003463090988) bank196540 +196541 POINT(41.039111825955715 73.73655399415702) bank196541 +196542 POINT(41.63333161861445 73.27126459724585) bank196542 +196543 POINT(39.84827125884666 73.26322240568263) bank196543 +196544 POINT(40.90876764360501 74.14293127444577) bank196544 +196545 POINT(40.76676794449992 74.19108850842201) bank196545 +196546 POINT(40.59899196933139 74.12375381219182) bank196546 +196547 POINT(39.98537461621029 74.94117716484531) bank196547 +196548 POINT(40.6761276661418 73.15106577037872) bank196548 +196549 POINT(41.07597122033565 73.15249819036002) bank196549 +196550 POINT(41.671090015583005 74.12825442039222) bank196550 +196551 POINT(40.667707810139646 74.35662060845505) bank196551 +196552 POINT(40.65931103296693 73.68636944294848) bank196552 +196553 POINT(40.245042073886154 74.54227862240958) bank196553 +196554 POINT(39.86924364644668 73.62895621733854) bank196554 +196555 POINT(41.65790234838483 75.00019244047807) bank196555 +196556 POINT(41.04909391943641 73.95415657553087) bank196556 +196557 POINT(40.71387038133982 74.63605766299469) bank196557 +196558 POINT(41.28967348653598 73.53604803088808) bank196558 +196559 POINT(41.47612771501009 74.94228113409389) bank196559 +196560 POINT(41.20132860773528 73.6032885654468) bank196560 +196561 POINT(41.51694005395323 73.76760693897748) bank196561 +196562 POINT(40.172386204746616 74.62587812265417) bank196562 +196563 POINT(40.56388404320617 73.78912683884528) bank196563 +196564 POINT(39.938697873131645 74.04270772673685) bank196564 +196565 POINT(40.07185395616578 74.46596529716024) bank196565 +196566 POINT(40.53334941424782 73.46753108752415) bank196566 +196567 POINT(40.00753294184918 73.28466670096819) bank196567 +196568 POINT(40.56959696458866 74.83474016372533) bank196568 +196569 POINT(40.03587356981033 74.47066498394237) bank196569 +196570 POINT(40.21608449663052 73.77897968712274) bank196570 +196571 POINT(40.575245173975794 73.23868261540895) bank196571 +196572 POINT(39.78298432820956 74.08592898714825) bank196572 +196573 POINT(40.09553604384707 73.07259775516705) bank196573 +196574 POINT(39.81536397211315 74.21934091307898) bank196574 +196575 POINT(40.87600347666276 74.48694956011275) bank196575 +196576 POINT(40.940977510419295 74.52789684339805) bank196576 +196577 POINT(40.288066521984035 73.3830567292804) bank196577 +196578 POINT(40.77538567301184 74.49559516331226) bank196578 +196579 POINT(40.24870140323679 74.43008740021868) bank196579 +196580 POINT(41.59363702796071 74.54234709745704) bank196580 +196581 POINT(41.51208141539774 74.61661539287219) bank196581 +196582 POINT(39.95718844370382 73.14815363349626) bank196582 +196583 POINT(40.496196008390264 74.18104681109011) bank196583 +196584 POINT(41.3865254220712 73.39142286025164) bank196584 +196585 POINT(40.09330801795384 74.78318579324916) bank196585 +196586 POINT(41.37561812441872 74.45146021111275) bank196586 +196587 POINT(40.481131117466234 74.30617360410224) bank196587 +196588 POINT(40.472228909102014 73.67701047309212) bank196588 +196589 POINT(41.24703971445389 74.26538012857289) bank196589 +196590 POINT(41.0117117466787 74.44306401938461) bank196590 +196591 POINT(41.699277512446876 73.83669698277536) bank196591 +196592 POINT(41.433289678282605 74.22031987354183) bank196592 +196593 POINT(41.50077417562693 73.66232928149817) bank196593 +196594 POINT(40.946710213105 74.38446761503518) bank196594 +196595 POINT(41.448771776928744 73.22994244202643) bank196595 +196596 POINT(41.63340841515181 74.0206553571754) bank196596 +196597 POINT(40.06968678165208 74.75023911074004) bank196597 +196598 POINT(41.46351985435787 74.77573755878147) bank196598 +196599 POINT(40.60879881992385 74.67018614702594) bank196599 +196600 POINT(41.67220722224767 74.70408927876588) bank196600 +196601 POINT(39.84021880770942 73.80466348595816) bank196601 +196602 POINT(41.130774757917514 73.7988791705107) bank196602 +196603 POINT(39.96906040273776 73.37206654097805) bank196603 +196604 POINT(40.92741367016582 74.72718308142463) bank196604 +196605 POINT(39.71803654871639 73.89518736086282) bank196605 +196606 POINT(40.87000166525155 74.06758091900967) bank196606 +196607 POINT(40.06880458445924 74.47459724880353) bank196607 +196608 POINT(40.5334417885833 73.61542908672962) bank196608 +196609 POINT(40.44930725328726 74.63569661770386) bank196609 +196610 POINT(40.458401595381766 74.46923024418086) bank196610 +196611 POINT(41.63069332985802 74.8332343383921) bank196611 +196612 POINT(40.6256220832521 74.9450360373879) bank196612 +196613 POINT(40.39527902955128 73.58354149218759) bank196613 +196614 POINT(39.87204667006797 74.41750466993676) bank196614 +196615 POINT(40.799441889790295 74.86581849410794) bank196615 +196616 POINT(40.45137461651484 73.17874247606343) bank196616 +196617 POINT(41.54385991363831 74.26050366525084) bank196617 +196618 POINT(41.484531051146774 73.38233856254205) bank196618 +196619 POINT(40.90017718370142 73.27996804921935) bank196619 +196620 POINT(40.29916930159018 73.26491087920958) bank196620 +196621 POINT(40.910657625391785 74.62950355316644) bank196621 +196622 POINT(40.45788108952367 73.48243038991598) bank196622 +196623 POINT(41.023981364041866 74.13294081179319) bank196623 +196624 POINT(40.82699565499923 74.07474378346772) bank196624 +196625 POINT(40.62396076057658 73.7383569815249) bank196625 +196626 POINT(40.90788287039786 74.46970385251593) bank196626 +196627 POINT(40.27159443576855 73.06555472290502) bank196627 +196628 POINT(40.9437574663088 73.25986661191838) bank196628 +196629 POINT(40.00418086991178 74.27462874202972) bank196629 +196630 POINT(40.12327148666048 73.86234700450649) bank196630 +196631 POINT(40.01931036969971 73.83569771659928) bank196631 +196632 POINT(39.786693846306434 74.27918710023499) bank196632 +196633 POINT(41.57256992496221 73.70762805689711) bank196633 +196634 POINT(41.21708137692604 73.93227490314474) bank196634 +196635 POINT(40.45691273327621 74.9152097777298) bank196635 +196636 POINT(40.44749089597933 74.96107074169726) bank196636 +196637 POINT(41.22382071677212 73.9674910778585) bank196637 +196638 POINT(40.2414174378644 73.43478771765257) bank196638 +196639 POINT(41.53292193066882 74.0880010833131) bank196639 +196640 POINT(40.5096558410106 73.87291537794239) bank196640 +196641 POINT(39.780973291890035 74.97617576776221) bank196641 +196642 POINT(41.70998832996443 74.72538408035135) bank196642 +196643 POINT(39.78723277336669 73.55198542263143) bank196643 +196644 POINT(41.03777296437473 73.98015656973318) bank196644 +196645 POINT(40.10265873822224 74.46784233388315) bank196645 +196646 POINT(41.4819239566768 73.60574181094229) bank196646 +196647 POINT(41.31575668631454 73.57961274095045) bank196647 +196648 POINT(40.499941287332405 73.14489418468085) bank196648 +196649 POINT(40.30445319087365 73.62534953165888) bank196649 +196650 POINT(41.119697450923894 74.8169274951282) bank196650 +196651 POINT(40.15497746847392 73.44086271702501) bank196651 +196652 POINT(39.81140910337199 73.38097142493045) bank196652 +196653 POINT(41.70769561043992 74.10142419153827) bank196653 +196654 POINT(40.431051266565966 74.0911044786202) bank196654 +196655 POINT(40.71943477718182 74.34452556735917) bank196655 +196656 POINT(41.290165028742 74.06042908715008) bank196656 +196657 POINT(40.27168530775823 73.04421362504341) bank196657 +196658 POINT(40.05684320525985 74.61587026561372) bank196658 +196659 POINT(40.3797082665748 73.27437441284714) bank196659 +196660 POINT(39.92618939113461 73.66183125122102) bank196660 +196661 POINT(41.23100255334277 73.42975550117178) bank196661 +196662 POINT(40.61347841631506 74.62304348366888) bank196662 +196663 POINT(40.45847251447589 73.97008321277272) bank196663 +196664 POINT(40.331651215604595 74.0542012508952) bank196664 +196665 POINT(40.02880652775003 74.00818051586712) bank196665 +196666 POINT(41.599112905424946 74.06301843609971) bank196666 +196667 POINT(40.95232673807533 73.2084475093523) bank196667 +196668 POINT(40.86106333812138 73.10829446432503) bank196668 +196669 POINT(40.63793736685392 74.7822641505503) bank196669 +196670 POINT(39.991458159855576 74.54494491641756) bank196670 +196671 POINT(40.60323060401949 74.37084705341395) bank196671 +196672 POINT(39.8761728067846 73.46358829920449) bank196672 +196673 POINT(41.50456302854552 73.85123567923185) bank196673 +196674 POINT(41.69555598630739 74.18700460146779) bank196674 +196675 POINT(41.18932017520183 73.29880718130684) bank196675 +196676 POINT(40.01538674637149 74.84215883437945) bank196676 +196677 POINT(41.261803639440785 74.78503012792675) bank196677 +196678 POINT(41.29103304925921 74.75231956607806) bank196678 +196679 POINT(41.41959569350318 73.01669676505774) bank196679 +196680 POINT(40.26654731337741 73.03830927258547) bank196680 +196681 POINT(40.82216566668576 74.7582993015291) bank196681 +196682 POINT(40.436546158835746 73.05012829046323) bank196682 +196683 POINT(41.23492908381863 73.49148328444724) bank196683 +196684 POINT(41.60154952667195 74.10747404312801) bank196684 +196685 POINT(40.86313200120672 73.2499949856569) bank196685 +196686 POINT(40.15773887728571 73.67370769950742) bank196686 +196687 POINT(40.678319475838194 73.46746556658995) bank196687 +196688 POINT(40.10078544556754 74.86299159451517) bank196688 +196689 POINT(40.46323467496663 74.96189730692466) bank196689 +196690 POINT(41.22961701954627 74.37424654957734) bank196690 +196691 POINT(41.083504699553366 73.82102079077512) bank196691 +196692 POINT(41.27553331870775 73.66319249980322) bank196692 +196693 POINT(40.43875803670082 73.82998709629481) bank196693 +196694 POINT(40.87419125388539 74.12058504312462) bank196694 +196695 POINT(39.79655574781312 74.56129961147893) bank196695 +196696 POINT(39.817645243510185 73.8380894001992) bank196696 +196697 POINT(41.60386933556318 74.93818842170981) bank196697 +196698 POINT(41.61955030530626 74.62381013767246) bank196698 +196699 POINT(40.36831343303925 73.35486544704072) bank196699 +196700 POINT(41.027894489092716 74.55903627450454) bank196700 +196701 POINT(40.80449865032963 73.4525978407759) bank196701 +196702 POINT(41.24096476842771 73.15841479214343) bank196702 +196703 POINT(41.3990799645023 73.9036130945344) bank196703 +196704 POINT(40.63929807512095 73.97411430303382) bank196704 +196705 POINT(40.55185660396996 73.23635340439186) bank196705 +196706 POINT(39.9345372512617 74.65915340738664) bank196706 +196707 POINT(41.67000784522373 74.15583764139518) bank196707 +196708 POINT(39.97239997399404 74.03082681648186) bank196708 +196709 POINT(40.487222125169716 73.69602319442349) bank196709 +196710 POINT(41.23322500889579 73.53224429583479) bank196710 +196711 POINT(41.46528151596426 73.95646962745208) bank196711 +196712 POINT(39.87539040304163 74.85225551693601) bank196712 +196713 POINT(39.7459241943899 74.69915326053662) bank196713 +196714 POINT(39.876966924152384 74.38936672170071) bank196714 +196715 POINT(41.12283067375576 73.4473100826132) bank196715 +196716 POINT(39.81396387687866 73.28398824445576) bank196716 +196717 POINT(41.4925937143372 73.42639491563885) bank196717 +196718 POINT(41.27611146270263 73.08554104721327) bank196718 +196719 POINT(41.62685346821218 73.54721876892958) bank196719 +196720 POINT(41.4322783440605 73.61374706511535) bank196720 +196721 POINT(39.82215283775837 73.07351567624828) bank196721 +196722 POINT(39.850302540815704 73.83704721469081) bank196722 +196723 POINT(40.71169632572322 74.89682197436872) bank196723 +196724 POINT(40.005455755489294 73.71173736427124) bank196724 +196725 POINT(40.93056817260324 74.30282205979337) bank196725 +196726 POINT(40.83019271148927 73.59965658864016) bank196726 +196727 POINT(39.74641250199656 73.20060183999453) bank196727 +196728 POINT(39.78025020403266 73.24465790408736) bank196728 +196729 POINT(41.021163122882896 74.64465055892262) bank196729 +196730 POINT(41.016493780461694 73.3523187461705) bank196730 +196731 POINT(40.28754766783976 73.70654958755732) bank196731 +196732 POINT(41.52689853407547 74.89535736616114) bank196732 +196733 POINT(40.65029205978985 74.0703624409285) bank196733 +196734 POINT(41.51081976635071 74.97433311076102) bank196734 +196735 POINT(39.947546227602544 74.6203676029471) bank196735 +196736 POINT(41.70871102077946 74.58018670474743) bank196736 +196737 POINT(41.6633450136312 74.55793513179823) bank196737 +196738 POINT(41.6341087651865 74.64462664301243) bank196738 +196739 POINT(40.84753960202914 73.76563066284275) bank196739 +196740 POINT(41.66589431599044 73.55518686886248) bank196740 +196741 POINT(39.88536953850902 73.62391129636428) bank196741 +196742 POINT(39.76299220278203 73.61912077841296) bank196742 +196743 POINT(41.01022178683392 74.7269231361726) bank196743 +196744 POINT(41.21008943632463 74.05409338236291) bank196744 +196745 POINT(40.457236653968806 73.38836139356685) bank196745 +196746 POINT(41.153318459424 74.84017811010844) bank196746 +196747 POINT(40.57625591602703 73.76449425451517) bank196747 +196748 POINT(41.70869574166165 74.07449546934178) bank196748 +196749 POINT(41.63208442219147 73.31773738198079) bank196749 +196750 POINT(40.44821274433419 74.8600808107011) bank196750 +196751 POINT(41.34176104193069 74.60200921017965) bank196751 +196752 POINT(40.647243043568956 73.98608399821842) bank196752 +196753 POINT(40.73080097624387 73.20531669701015) bank196753 +196754 POINT(40.88587545405983 73.10180067924237) bank196754 +196755 POINT(41.048197960650306 74.50707857080151) bank196755 +196756 POINT(40.850553409869775 73.00934105013309) bank196756 +196757 POINT(40.06047762364887 73.50106684827185) bank196757 +196758 POINT(39.7175751572981 74.08411392386465) bank196758 +196759 POINT(40.25087889514262 74.11318511756032) bank196759 +196760 POINT(41.67423932365036 74.5298455289161) bank196760 +196761 POINT(40.8411582557195 73.76331451224358) bank196761 +196762 POINT(40.201997110525156 74.01061013285795) bank196762 +196763 POINT(39.956321236336116 73.03565308565747) bank196763 +196764 POINT(40.57541010769891 74.60557010160251) bank196764 +196765 POINT(41.522344232142686 74.66352243286364) bank196765 +196766 POINT(40.678093346843276 73.39981382808132) bank196766 +196767 POINT(41.04953429688259 73.22360853875377) bank196767 +196768 POINT(41.14604562207123 74.8311220667941) bank196768 +196769 POINT(39.82612684518377 74.99846762774244) bank196769 +196770 POINT(41.34816718414568 73.23870904847828) bank196770 +196771 POINT(40.14117198018088 73.2910878342452) bank196771 +196772 POINT(40.54215587354421 73.70679759120323) bank196772 +196773 POINT(40.67746554801185 73.74917155690052) bank196773 +196774 POINT(40.8984075311527 74.26802373514107) bank196774 +196775 POINT(41.07734726836426 74.40804067159306) bank196775 +196776 POINT(40.87452774475287 74.52602115690232) bank196776 +196777 POINT(41.672011212438626 74.23838714393416) bank196777 +196778 POINT(40.07023917996425 74.75760464689368) bank196778 +196779 POINT(39.90501167651194 73.97188303842498) bank196779 +196780 POINT(40.75891622153223 74.26313069124761) bank196780 +196781 POINT(41.02983500864472 74.92812366917025) bank196781 +196782 POINT(40.40787218475057 73.07452393551088) bank196782 +196783 POINT(41.072555643619175 73.71382761462819) bank196783 +196784 POINT(40.944325473091254 74.276784899641) bank196784 +196785 POINT(40.363518908297955 74.86319723990424) bank196785 +196786 POINT(40.211549506591815 73.08845779467933) bank196786 +196787 POINT(41.24823395647394 74.7823575676852) bank196787 +196788 POINT(40.298029351773984 74.98131686121229) bank196788 +196789 POINT(40.71336499667429 74.33330645417625) bank196789 +196790 POINT(39.719587506142915 73.31801376827465) bank196790 +196791 POINT(41.41607747878909 73.95802522629147) bank196791 +196792 POINT(40.50717790834266 74.07286553679751) bank196792 +196793 POINT(40.37805559363967 73.53183025744713) bank196793 +196794 POINT(41.21042979940445 74.93020375164781) bank196794 +196795 POINT(39.943146079936035 74.10653009023284) bank196795 +196796 POINT(39.933638939635316 73.11360188074111) bank196796 +196797 POINT(40.98583187365739 74.74621247697908) bank196797 +196798 POINT(40.96090259785903 73.52470658094012) bank196798 +196799 POINT(40.88259990451863 73.74129546543517) bank196799 +196800 POINT(40.98874950077589 74.06343800685161) bank196800 +196801 POINT(41.227352737729206 73.8762313784873) bank196801 +196802 POINT(41.00777162360025 74.90630639779049) bank196802 +196803 POINT(40.44753403331634 74.06334333600229) bank196803 +196804 POINT(40.81308229626525 73.46786095468259) bank196804 +196805 POINT(40.219730818910996 74.23532617248165) bank196805 +196806 POINT(39.95102490565716 74.69781653115972) bank196806 +196807 POINT(40.466593990792234 73.86275619016355) bank196807 +196808 POINT(40.069035690463366 73.33826830192933) bank196808 +196809 POINT(39.842615776107195 74.66067124600855) bank196809 +196810 POINT(41.63888961233791 73.5980821090665) bank196810 +196811 POINT(40.35042704659173 74.86484278264643) bank196811 +196812 POINT(41.672967205054846 73.29102841349582) bank196812 +196813 POINT(41.48487027888572 74.39138304603036) bank196813 +196814 POINT(39.95144411571751 74.28810338942527) bank196814 +196815 POINT(39.9944051642353 74.08538990845042) bank196815 +196816 POINT(39.892742664785224 74.15930007624846) bank196816 +196817 POINT(40.69935234559282 73.73237344177508) bank196817 +196818 POINT(41.14570118531832 74.09391895247589) bank196818 +196819 POINT(39.75796552304326 73.72453040609805) bank196819 +196820 POINT(39.80239721691871 73.67132914445622) bank196820 +196821 POINT(40.70136350184872 73.94557738122302) bank196821 +196822 POINT(41.3053844164384 74.53108684474397) bank196822 +196823 POINT(40.132191284625755 73.9290657016957) bank196823 +196824 POINT(40.07940935859181 74.97081657254901) bank196824 +196825 POINT(39.91383456708843 73.1374913216875) bank196825 +196826 POINT(40.149249942951656 73.59839608535722) bank196826 +196827 POINT(41.59703846657695 74.75187325534256) bank196827 +196828 POINT(41.3955898641786 73.18104547202415) bank196828 +196829 POINT(40.34103610194763 73.64084696330916) bank196829 +196830 POINT(41.2803548428501 74.48690133953545) bank196830 +196831 POINT(40.617552512854374 73.66816908315094) bank196831 +196832 POINT(41.34091737619623 73.18019319555376) bank196832 +196833 POINT(41.33878232944637 74.36928040523713) bank196833 +196834 POINT(41.02881776044678 75.0017974906744) bank196834 +196835 POINT(41.35244354057797 73.89555196293387) bank196835 +196836 POINT(40.22555685292749 74.96442842305208) bank196836 +196837 POINT(40.966672157769736 73.30431465767415) bank196837 +196838 POINT(41.05921923942203 74.05649986722013) bank196838 +196839 POINT(40.39841966209748 73.24572110317057) bank196839 +196840 POINT(41.35756255921211 74.436518873421) bank196840 +196841 POINT(40.84416872173371 74.75944779040542) bank196841 +196842 POINT(40.14460410715391 73.54542251027502) bank196842 +196843 POINT(40.04738664015573 74.56853897502127) bank196843 +196844 POINT(40.754546344608954 74.05771009903579) bank196844 +196845 POINT(40.62749315211358 73.31230038569156) bank196845 +196846 POINT(40.93185328362518 74.13652186251132) bank196846 +196847 POINT(41.601001344292655 74.20348163416277) bank196847 +196848 POINT(41.11817814568302 74.01774535440896) bank196848 +196849 POINT(41.58992801488647 74.75215464832777) bank196849 +196850 POINT(40.506605914716346 74.45680939781609) bank196850 +196851 POINT(41.289176698990055 73.21165898833571) bank196851 +196852 POINT(41.24532704758676 73.38527777182121) bank196852 +196853 POINT(39.86764563821207 74.6491661255845) bank196853 +196854 POINT(40.01123175697904 74.68432584020661) bank196854 +196855 POINT(40.53427547768982 74.29730573754736) bank196855 +196856 POINT(41.487640572998394 73.296689202348) bank196856 +196857 POINT(40.54914742448481 73.62507610221854) bank196857 +196858 POINT(41.441842575780974 74.34341417295565) bank196858 +196859 POINT(39.97281930835029 74.44612346763279) bank196859 +196860 POINT(40.9026877110998 74.73406690425954) bank196860 +196861 POINT(40.03608817678696 73.60018266619188) bank196861 +196862 POINT(40.73980477697033 73.59005988689212) bank196862 +196863 POINT(40.560839880440426 74.26130057721039) bank196863 +196864 POINT(40.33808868444453 74.5102096023861) bank196864 +196865 POINT(41.44451225380961 74.45335696288652) bank196865 +196866 POINT(40.48181864588968 74.35761108162575) bank196866 +196867 POINT(39.82571632435347 73.52906634383079) bank196867 +196868 POINT(39.955900525359205 74.26530922097712) bank196868 +196869 POINT(40.1626921269588 74.67639826688128) bank196869 +196870 POINT(41.66551130650295 74.96565565648874) bank196870 +196871 POINT(41.048131468482325 74.91349443413247) bank196871 +196872 POINT(39.85362819302231 73.75809375006416) bank196872 +196873 POINT(40.49499920445305 73.54083550917879) bank196873 +196874 POINT(41.62726251386994 73.54785364360296) bank196874 +196875 POINT(40.6718327350168 73.47634733477946) bank196875 +196876 POINT(39.72348406502639 73.93403802900222) bank196876 +196877 POINT(40.83860319257484 74.91268621012355) bank196877 +196878 POINT(40.18182608954208 74.42910479325852) bank196878 +196879 POINT(41.41236646775679 74.40471140710056) bank196879 +196880 POINT(41.22935931567142 74.35178116672662) bank196880 +196881 POINT(40.065616831900286 74.54879071144603) bank196881 +196882 POINT(41.5770801507324 74.33687833770821) bank196882 +196883 POINT(40.58533541814973 74.91274005723491) bank196883 +196884 POINT(40.52388728449425 73.4048320890015) bank196884 +196885 POINT(40.81645094156916 73.29393920996876) bank196885 +196886 POINT(39.91648472460311 74.22239788994045) bank196886 +196887 POINT(39.84897956436142 74.12563019129205) bank196887 +196888 POINT(40.10169959938823 74.47689214038333) bank196888 +196889 POINT(39.76384786072413 73.8785043874603) bank196889 +196890 POINT(41.15300384780723 74.04185596956167) bank196890 +196891 POINT(41.363817859756885 73.4260550700787) bank196891 +196892 POINT(40.07157747835369 73.21141616649233) bank196892 +196893 POINT(40.54876421549397 74.49720848163653) bank196893 +196894 POINT(40.697765799709856 74.35690013374715) bank196894 +196895 POINT(40.0100175736792 74.20392579202996) bank196895 +196896 POINT(40.8811640498386 74.52671135475684) bank196896 +196897 POINT(40.18179357567438 74.08163593138792) bank196897 +196898 POINT(40.60403671640906 73.37046334281645) bank196898 +196899 POINT(41.240973516204406 73.26184317795628) bank196899 +196900 POINT(41.218565024832294 73.76319490464942) bank196900 +196901 POINT(41.10462100091589 74.15480383028664) bank196901 +196902 POINT(39.834059218722025 73.36700820891642) bank196902 +196903 POINT(40.91885092324439 73.88645432108126) bank196903 +196904 POINT(39.95601536994954 73.92927884963407) bank196904 +196905 POINT(39.76857902740064 73.112156863851) bank196905 +196906 POINT(40.42056397374207 73.91351095652333) bank196906 +196907 POINT(39.80445693187541 73.02931246928954) bank196907 +196908 POINT(39.836538855717876 74.28114095204707) bank196908 +196909 POINT(40.62419510565837 73.55207699720945) bank196909 +196910 POINT(41.62627956809669 73.1548587696656) bank196910 +196911 POINT(39.83190093020955 73.86613340088726) bank196911 +196912 POINT(40.43143797771468 74.28098821219284) bank196912 +196913 POINT(40.2534206147067 74.91552486671056) bank196913 +196914 POINT(41.357442181196866 73.83976925100947) bank196914 +196915 POINT(41.09759792741819 73.19538741984445) bank196915 +196916 POINT(40.31083498209303 73.38022102560036) bank196916 +196917 POINT(40.37883345571284 73.58479317720308) bank196917 +196918 POINT(41.13095552001558 74.89439902018269) bank196918 +196919 POINT(40.66125621575459 73.59238877784391) bank196919 +196920 POINT(41.7042110594688 74.97190507027506) bank196920 +196921 POINT(41.67027057781383 74.2746520123333) bank196921 +196922 POINT(40.787788022372354 74.689008642492) bank196922 +196923 POINT(40.030587427353986 74.93455892290746) bank196923 +196924 POINT(40.15887764116892 73.97501305403884) bank196924 +196925 POINT(40.65467466283313 73.5430701150275) bank196925 +196926 POINT(39.98941173273312 73.69945444379218) bank196926 +196927 POINT(41.52248477260386 74.77912962645973) bank196927 +196928 POINT(40.61291120327178 74.45734924027752) bank196928 +196929 POINT(40.87329510405015 74.08456600055234) bank196929 +196930 POINT(41.152745973835536 74.87263870153507) bank196930 +196931 POINT(40.566584394894534 73.92921447544887) bank196931 +196932 POINT(41.055179792720544 74.65775760471665) bank196932 +196933 POINT(41.372204689624866 73.42525092190313) bank196933 +196934 POINT(40.77119663985828 74.80000555570257) bank196934 +196935 POINT(40.99092400945468 73.28944885223775) bank196935 +196936 POINT(39.94956350568258 74.31145937028226) bank196936 +196937 POINT(40.35156188778234 74.11664729612558) bank196937 +196938 POINT(41.42671845163375 74.26233428998762) bank196938 +196939 POINT(41.57813624835466 73.09395303828738) bank196939 +196940 POINT(41.38810779834205 74.42860025347528) bank196940 +196941 POINT(40.76819705437161 73.29763526043757) bank196941 +196942 POINT(41.57533179744292 73.48492486459278) bank196942 +196943 POINT(41.41152830504308 73.90792668258656) bank196943 +196944 POINT(40.43312857456448 74.06887335898575) bank196944 +196945 POINT(39.99138187614996 74.97732728632256) bank196945 +196946 POINT(41.05293659965399 74.9518812134175) bank196946 +196947 POINT(39.96952107503034 73.762918283626) bank196947 +196948 POINT(40.007357424449054 74.07051779016842) bank196948 +196949 POINT(40.99218482284054 73.237227964395) bank196949 +196950 POINT(40.76950698190478 73.49697230363664) bank196950 +196951 POINT(40.54348734681897 73.37755240268591) bank196951 +196952 POINT(40.60288875524307 74.27192211914092) bank196952 +196953 POINT(41.17561938916166 74.92071054896395) bank196953 +196954 POINT(41.13946666889323 74.15223074956253) bank196954 +196955 POINT(41.046801275528054 73.35511131490755) bank196955 +196956 POINT(40.47819105687745 73.41605955244671) bank196956 +196957 POINT(41.19380691593595 73.04718286362265) bank196957 +196958 POINT(41.35120325767433 74.18352701793376) bank196958 +196959 POINT(39.911518539192116 74.17139562480911) bank196959 +196960 POINT(41.47058194629344 73.09593349705867) bank196960 +196961 POINT(40.58435923035688 74.69970868721653) bank196961 +196962 POINT(40.103019870629964 73.22493444072246) bank196962 +196963 POINT(40.234647634087764 73.36667944309755) bank196963 +196964 POINT(40.120725031881356 73.0510180188702) bank196964 +196965 POINT(40.17532430717706 73.8525597235268) bank196965 +196966 POINT(39.89177053303533 74.00509208439152) bank196966 +196967 POINT(41.58991720929779 73.24464879691064) bank196967 +196968 POINT(40.00532519222153 73.80748918628767) bank196968 +196969 POINT(39.93347695160733 74.37747064693386) bank196969 +196970 POINT(41.420899111633204 74.99744199159974) bank196970 +196971 POINT(40.912104676904214 74.25282012057217) bank196971 +196972 POINT(41.2725678036867 73.68874747728162) bank196972 +196973 POINT(40.82040627133413 74.96954572770755) bank196973 +196974 POINT(41.58378088683136 73.07373674595506) bank196974 +196975 POINT(41.03498193278795 74.54585684980067) bank196975 +196976 POINT(41.187710671355745 73.34580444371284) bank196976 +196977 POINT(39.982401060487085 73.4219042092646) bank196977 +196978 POINT(40.95386220706943 74.53802335357072) bank196978 +196979 POINT(40.72347626365781 74.2895466205512) bank196979 +196980 POINT(40.39982224550895 73.30159072464369) bank196980 +196981 POINT(39.875135864303786 74.87825441466664) bank196981 +196982 POINT(39.766202747473784 73.47024845355749) bank196982 +196983 POINT(40.26131408386959 74.13517173231527) bank196983 +196984 POINT(39.98217420557969 74.08564830883542) bank196984 +196985 POINT(40.461992734060594 74.31057978426072) bank196985 +196986 POINT(41.59485423422191 73.44534568484825) bank196986 +196987 POINT(41.59194756991327 73.9265672651542) bank196987 +196988 POINT(41.259098869449716 73.09925186089103) bank196988 +196989 POINT(40.449167405594274 73.64656278858685) bank196989 +196990 POINT(41.640825482068635 74.01836061305538) bank196990 +196991 POINT(40.772642780660966 74.98687934861645) bank196991 +196992 POINT(40.327946218028984 73.69653564874092) bank196992 +196993 POINT(41.2146674300988 74.42849004213646) bank196993 +196994 POINT(40.54290284593662 73.12063412897459) bank196994 +196995 POINT(40.97483752506549 74.83593587572322) bank196995 +196996 POINT(40.171709137723575 74.7695978549963) bank196996 +196997 POINT(41.30991709647824 73.74877691629524) bank196997 +196998 POINT(41.52051034540371 74.83101618246111) bank196998 +196999 POINT(40.485458931321126 74.61960476579404) bank196999 +197000 POINT(40.21002775563477 74.40914404243678) bank197000 +197001 POINT(41.66825089926705 73.70871246723765) bank197001 +197002 POINT(41.229403826842386 73.53978680419854) bank197002 +197003 POINT(40.78796341433435 74.09958720753828) bank197003 +197004 POINT(41.529428084766685 74.19772075147262) bank197004 +197005 POINT(41.10803201671077 74.42478833171772) bank197005 +197006 POINT(41.54120818785341 74.03610224918246) bank197006 +197007 POINT(39.72845601888877 73.6887355942836) bank197007 +197008 POINT(40.24150437631734 74.60562109091208) bank197008 +197009 POINT(39.78996752256334 73.43813845014253) bank197009 +197010 POINT(40.921369934167956 73.05460911583222) bank197010 +197011 POINT(41.02086322393427 74.85554667732612) bank197011 +197012 POINT(40.52103550766394 74.3412150213575) bank197012 +197013 POINT(40.12231303593973 74.59433366282934) bank197013 +197014 POINT(40.24932560681601 73.52986735681256) bank197014 +197015 POINT(39.80268271380974 73.82724134843956) bank197015 +197016 POINT(39.883683867125015 74.36255949531215) bank197016 +197017 POINT(40.77963467467454 74.05825041152035) bank197017 +197018 POINT(40.59954052147443 73.02041098595562) bank197018 +197019 POINT(39.756818719282585 73.45949613910021) bank197019 +197020 POINT(40.13772344154579 74.80961994133922) bank197020 +197021 POINT(39.840455154809334 74.34572357212383) bank197021 +197022 POINT(39.85026246215642 74.02843802058345) bank197022 +197023 POINT(40.8395169877235 74.56107943408213) bank197023 +197024 POINT(41.62936239548067 73.5157723806281) bank197024 +197025 POINT(41.69191564681646 74.57010688375551) bank197025 +197026 POINT(39.77072874618149 73.17142814971426) bank197026 +197027 POINT(40.84328500349871 73.64496690912894) bank197027 +197028 POINT(39.73007206700206 74.5273038992832) bank197028 +197029 POINT(39.892196472306935 74.20854410603307) bank197029 +197030 POINT(41.31653700019207 73.0814308313169) bank197030 +197031 POINT(40.28753078049741 74.69820430075679) bank197031 +197032 POINT(40.49092248995977 74.1195857946343) bank197032 +197033 POINT(40.5462656423775 73.63774703860079) bank197033 +197034 POINT(40.511873767751 74.92310205832356) bank197034 +197035 POINT(41.64397380349858 73.17280629494182) bank197035 +197036 POINT(40.10776752067874 73.3916348072239) bank197036 +197037 POINT(39.88863200807289 74.95180310618512) bank197037 +197038 POINT(41.385178541236165 74.79199245520506) bank197038 +197039 POINT(41.24399284583958 74.04321589601462) bank197039 +197040 POINT(39.798222393419266 74.33234347541963) bank197040 +197041 POINT(40.05562051775323 73.55591336122427) bank197041 +197042 POINT(41.03485975077029 73.44516998922917) bank197042 +197043 POINT(40.963769327658326 73.09488366243393) bank197043 +197044 POINT(39.88612892749168 74.19268411607888) bank197044 +197045 POINT(41.187143655361616 73.35575476695185) bank197045 +197046 POINT(41.148280349766644 74.88210096964691) bank197046 +197047 POINT(41.566065876240124 74.72159255912054) bank197047 +197048 POINT(40.27438973312828 74.3939403027651) bank197048 +197049 POINT(40.98461303695017 74.13623793894642) bank197049 +197050 POINT(41.51823463664185 73.83657388213739) bank197050 +197051 POINT(40.410942549088496 74.98803599954866) bank197051 +197052 POINT(40.997960072859186 73.46047590506302) bank197052 +197053 POINT(41.626711007489625 74.75864863249389) bank197053 +197054 POINT(39.91640569349181 73.11308732606932) bank197054 +197055 POINT(40.81149986586923 73.1565541128683) bank197055 +197056 POINT(40.73770933018366 73.50668568162703) bank197056 +197057 POINT(40.51886307515456 73.44944688039182) bank197057 +197058 POINT(40.855405375978094 74.87144101042364) bank197058 +197059 POINT(40.09236310122666 74.71841571014221) bank197059 +197060 POINT(39.95965095321001 73.61984966901395) bank197060 +197061 POINT(41.46749434604527 74.74923176441912) bank197061 +197062 POINT(39.81986428725908 73.16633130397838) bank197062 +197063 POINT(41.12929062897589 74.74549777351181) bank197063 +197064 POINT(40.666364024726434 73.11980212489657) bank197064 +197065 POINT(41.43008457878219 73.43098645308035) bank197065 +197066 POINT(40.44711672436508 74.29969972583295) bank197066 +197067 POINT(41.413306928260816 74.45359355287296) bank197067 +197068 POINT(41.646277078166435 73.50487218867582) bank197068 +197069 POINT(41.3090786270652 73.96021881407833) bank197069 +197070 POINT(41.13549903196246 73.24988845755138) bank197070 +197071 POINT(40.27338925019873 73.77672179760118) bank197071 +197072 POINT(41.01028190721209 74.51046764312345) bank197072 +197073 POINT(41.27744554552505 74.8199797428245) bank197073 +197074 POINT(40.85340253428303 73.7987080394874) bank197074 +197075 POINT(40.909091273488855 73.01186400566236) bank197075 +197076 POINT(40.090291409115615 73.01659542776973) bank197076 +197077 POINT(41.66130625480497 73.93279915589201) bank197077 +197078 POINT(40.84233250177895 73.76257414019426) bank197078 +197079 POINT(41.25500193560242 74.27044556512529) bank197079 +197080 POINT(39.71506208282694 73.74297832899306) bank197080 +197081 POINT(41.67054748021127 73.6274003832158) bank197081 +197082 POINT(40.966679064887 73.1415688522859) bank197082 +197083 POINT(41.03584881365074 74.77572835742112) bank197083 +197084 POINT(40.29788557739862 74.17510161149394) bank197084 +197085 POINT(41.44416294535659 74.74509874385944) bank197085 +197086 POINT(40.44834816157882 73.83773071260796) bank197086 +197087 POINT(41.03932917988712 73.22274828389583) bank197087 +197088 POINT(40.9189904373839 73.39645425067884) bank197088 +197089 POINT(40.94048263090352 73.54804020874641) bank197089 +197090 POINT(40.149112527204586 73.43024928115081) bank197090 +197091 POINT(41.64090625950068 73.99384203546255) bank197091 +197092 POINT(40.38891010025117 73.32402601732542) bank197092 +197093 POINT(40.47992707621805 74.49455371824004) bank197093 +197094 POINT(40.96939228780688 74.83331778977343) bank197094 +197095 POINT(40.01062815117386 74.89654529856287) bank197095 +197096 POINT(40.577787146644056 74.20925359818935) bank197096 +197097 POINT(40.97719675435642 73.88225917874725) bank197097 +197098 POINT(39.96518491364496 74.40902739573734) bank197098 +197099 POINT(41.483761024781636 73.79628729539651) bank197099 +197100 POINT(40.25459781996049 73.76596905008992) bank197100 +197101 POINT(40.49803824785738 73.2018082813565) bank197101 +197102 POINT(40.37042012544169 73.26535831216407) bank197102 +197103 POINT(39.9490986285512 74.7771976089666) bank197103 +197104 POINT(40.70314860406927 74.81293580637943) bank197104 +197105 POINT(39.837113413092716 73.24548813435946) bank197105 +197106 POINT(39.76968175401856 74.04091561263574) bank197106 +197107 POINT(39.993973852324004 73.24629827991393) bank197107 +197108 POINT(40.0315025978951 74.7597625666355) bank197108 +197109 POINT(40.85643259338339 74.06497786854105) bank197109 +197110 POINT(41.49569544125619 73.70717830495934) bank197110 +197111 POINT(40.74146210504155 74.68539105665583) bank197111 +197112 POINT(41.56233179272422 74.82924213197089) bank197112 +197113 POINT(41.31510748317637 74.79638755330105) bank197113 +197114 POINT(41.345009186534746 74.9513613829356) bank197114 +197115 POINT(40.91270683446823 74.27678271939163) bank197115 +197116 POINT(41.46240637133341 73.31375551370309) bank197116 +197117 POINT(41.06324911286808 74.66003921708571) bank197117 +197118 POINT(41.15649822644374 74.52107996637389) bank197118 +197119 POINT(40.03583614593422 74.51915442767982) bank197119 +197120 POINT(40.39967937383928 73.77955594528673) bank197120 +197121 POINT(40.53561203897334 73.24821240388732) bank197121 +197122 POINT(40.48897182092541 74.91600034014115) bank197122 +197123 POINT(40.7057695945991 73.96120687564765) bank197123 +197124 POINT(40.26693792507153 73.71151857930995) bank197124 +197125 POINT(41.70115604642591 73.21501217912143) bank197125 +197126 POINT(41.138929818899804 73.32855574624433) bank197126 +197127 POINT(40.02643435964053 74.73025702387775) bank197127 +197128 POINT(41.52277249971977 74.93020149516522) bank197128 +197129 POINT(41.21268496585962 74.6918133051498) bank197129 +197130 POINT(41.52203814268796 74.86278004214523) bank197130 +197131 POINT(39.88150792512703 73.72843837247585) bank197131 +197132 POINT(41.41695481364429 74.83831506374482) bank197132 +197133 POINT(41.63779790994586 74.70521013556875) bank197133 +197134 POINT(41.453104090314966 73.31483460191907) bank197134 +197135 POINT(40.583913217618075 73.51308374565181) bank197135 +197136 POINT(40.96325613187742 74.84889301018573) bank197136 +197137 POINT(39.98362209189434 73.75631235172854) bank197137 +197138 POINT(39.905121895715126 73.08245963580876) bank197138 +197139 POINT(41.44001477603602 74.76327837924434) bank197139 +197140 POINT(40.80718717482372 73.8313937171308) bank197140 +197141 POINT(40.22864897209084 73.96509400526814) bank197141 +197142 POINT(40.72754071987901 74.61793039226454) bank197142 +197143 POINT(39.73890761845324 73.83726496316686) bank197143 +197144 POINT(40.39398414198575 73.4451327618842) bank197144 +197145 POINT(41.66311773475911 74.85937644501095) bank197145 +197146 POINT(41.58447827349706 74.95458930385831) bank197146 +197147 POINT(40.87210946916589 74.48629816428105) bank197147 +197148 POINT(41.66247395807744 74.71964594501932) bank197148 +197149 POINT(40.60766461256729 73.69112796513576) bank197149 +197150 POINT(40.83298487970737 74.61693447113441) bank197150 +197151 POINT(40.78946902201752 74.59033332610747) bank197151 +197152 POINT(41.3675296701667 73.51060522699129) bank197152 +197153 POINT(40.15454842661277 73.44624868464257) bank197153 +197154 POINT(40.21546401642552 73.78574480901145) bank197154 +197155 POINT(40.26869643611897 74.25815568534532) bank197155 +197156 POINT(41.30224884012647 74.99684137555222) bank197156 +197157 POINT(41.092187983754464 74.41817468267595) bank197157 +197158 POINT(41.18596719522567 73.38976866323952) bank197158 +197159 POINT(40.27386735579689 73.42121331407995) bank197159 +197160 POINT(41.22588095735697 74.47895376040009) bank197160 +197161 POINT(40.75984689475803 74.84628777152308) bank197161 +197162 POINT(40.438727609924555 73.52850772098576) bank197162 +197163 POINT(41.26577473394738 74.76084725315134) bank197163 +197164 POINT(40.67179704355741 74.12382630543452) bank197164 +197165 POINT(40.765735946394194 73.47577457727779) bank197165 +197166 POINT(41.09532849914317 74.67906817511953) bank197166 +197167 POINT(41.435973824764126 73.28538037501244) bank197167 +197168 POINT(39.876582507908516 74.14874320640348) bank197168 +197169 POINT(41.24026785790974 73.38376269756492) bank197169 +197170 POINT(40.12199417708483 74.22882281899099) bank197170 +197171 POINT(41.415705156808826 74.0000432733075) bank197171 +197172 POINT(41.10902680611187 74.4628441759346) bank197172 +197173 POINT(41.53201144108179 73.32037408525541) bank197173 +197174 POINT(41.1941391191179 73.87962893237946) bank197174 +197175 POINT(40.55712822931733 73.54331078357365) bank197175 +197176 POINT(40.06596762767865 73.56204698748299) bank197176 +197177 POINT(40.13663174920802 74.25993043726034) bank197177 +197178 POINT(40.16502325426124 74.8115855756606) bank197178 +197179 POINT(40.137614247338085 73.71678888363081) bank197179 +197180 POINT(40.50806129538458 74.10557113584) bank197180 +197181 POINT(40.5946128304159 74.09443140048808) bank197181 +197182 POINT(41.39886872559978 73.4365186503936) bank197182 +197183 POINT(40.641553980181556 74.98555979084401) bank197183 +197184 POINT(41.23318290922165 74.24551249216574) bank197184 +197185 POINT(41.492445449385045 73.6856969190963) bank197185 +197186 POINT(40.02742965012863 74.66950550183529) bank197186 +197187 POINT(39.954171179508876 74.52038475155014) bank197187 +197188 POINT(40.76789096642056 74.52912517048996) bank197188 +197189 POINT(40.480330133770124 74.41842477969334) bank197189 +197190 POINT(40.63416281928729 73.10190166242866) bank197190 +197191 POINT(39.71745098621376 74.9435145063459) bank197191 +197192 POINT(40.55354706977403 74.51264791952018) bank197192 +197193 POINT(39.74082082307298 74.44040384501663) bank197193 +197194 POINT(40.72030551296596 74.59317159335785) bank197194 +197195 POINT(41.319989466846394 74.46568407760076) bank197195 +197196 POINT(40.94986411149715 74.02204271025829) bank197196 +197197 POINT(40.58418515157848 74.39938683545718) bank197197 +197198 POINT(40.638055234488284 73.55395661730448) bank197198 +197199 POINT(40.69026736561588 73.66852535612193) bank197199 +197200 POINT(40.226286220493485 74.89532980875664) bank197200 +197201 POINT(41.66519290577611 73.90168302373344) bank197201 +197202 POINT(40.72317406788101 74.20240630783637) bank197202 +197203 POINT(41.35110483090412 74.68192562983778) bank197203 +197204 POINT(40.6035716120709 74.13865424471824) bank197204 +197205 POINT(39.816353169009425 73.34511239025197) bank197205 +197206 POINT(40.56251009531237 74.87386151357995) bank197206 +197207 POINT(40.95129538484987 74.0863887398374) bank197207 +197208 POINT(39.92766977011902 73.69896576895555) bank197208 +197209 POINT(40.65545296390663 74.35892922336411) bank197209 +197210 POINT(40.86054593502475 73.77797879458305) bank197210 +197211 POINT(40.093403282222546 74.4655163250291) bank197211 +197212 POINT(40.11252510731494 73.33942602658928) bank197212 +197213 POINT(41.23648447751326 73.08651126479687) bank197213 +197214 POINT(39.92882924418167 74.62684612782283) bank197214 +197215 POINT(41.51417470486702 73.7546520095239) bank197215 +197216 POINT(41.44967110593333 73.91070831381616) bank197216 +197217 POINT(40.809524203632094 73.01860118606164) bank197217 +197218 POINT(41.03743480894283 73.43160842479078) bank197218 +197219 POINT(41.56460032435804 74.4981105827113) bank197219 +197220 POINT(40.302333306365966 73.62484979656293) bank197220 +197221 POINT(40.32448637035382 74.91040886013255) bank197221 +197222 POINT(41.67225596077548 74.88000521820554) bank197222 +197223 POINT(40.09832443226891 74.99848264133922) bank197223 +197224 POINT(41.04242572496563 74.5395461968381) bank197224 +197225 POINT(40.295348251128146 73.20909244342201) bank197225 +197226 POINT(41.14345728768107 74.74732375115843) bank197226 +197227 POINT(40.93619649301478 74.3222828824711) bank197227 +197228 POINT(39.99224705816628 74.4851762960086) bank197228 +197229 POINT(40.70387552662419 74.53809265595132) bank197229 +197230 POINT(41.27398511927206 73.89662448373767) bank197230 +197231 POINT(40.70602114784607 74.99719240792521) bank197231 +197232 POINT(41.202365945295185 73.79284872490076) bank197232 +197233 POINT(39.89015935745797 74.08202191113575) bank197233 +197234 POINT(39.83790891792995 74.43423808530419) bank197234 +197235 POINT(41.56681831869622 74.92019864697144) bank197235 +197236 POINT(40.011779158814534 74.71022535945467) bank197236 +197237 POINT(41.347310381744386 73.29776923234493) bank197237 +197238 POINT(40.45705158580511 73.72365859275949) bank197238 +197239 POINT(41.628937394754665 74.1451935141457) bank197239 +197240 POINT(40.30844863061737 74.96642232230427) bank197240 +197241 POINT(40.521517471707696 73.69453276203447) bank197241 +197242 POINT(40.88723806613762 74.75652172613702) bank197242 +197243 POINT(40.47932264089154 74.80335694241083) bank197243 +197244 POINT(39.939437371938496 73.65504750400166) bank197244 +197245 POINT(40.567758825168404 73.67286814216504) bank197245 +197246 POINT(41.071858443240906 74.01646424190801) bank197246 +197247 POINT(40.52311971646173 74.85114124171315) bank197247 +197248 POINT(40.740123970798194 73.7271686454614) bank197248 +197249 POINT(40.9341290528547 74.0218696182164) bank197249 +197250 POINT(39.89349899840098 73.33636082812589) bank197250 +197251 POINT(39.7168640583349 74.3004002943345) bank197251 +197252 POINT(41.32195948414866 74.56208697460482) bank197252 +197253 POINT(40.38366966026928 73.41195583784635) bank197253 +197254 POINT(41.058995727986165 74.33698773351462) bank197254 +197255 POINT(40.57667019404652 73.37886575654554) bank197255 +197256 POINT(40.84889880983587 73.72946500158542) bank197256 +197257 POINT(40.819047617305536 74.23925507003902) bank197257 +197258 POINT(40.41149971228883 73.59858691091013) bank197258 +197259 POINT(40.88154000466289 74.93153476996716) bank197259 +197260 POINT(39.811653458547 73.99651735654491) bank197260 +197261 POINT(41.70356231328871 73.96516813850903) bank197261 +197262 POINT(39.85425602992065 73.18938155852479) bank197262 +197263 POINT(40.678285014287944 74.60727019973724) bank197263 +197264 POINT(41.208130127063335 74.69747941946476) bank197264 +197265 POINT(40.70171690252555 74.72382418200074) bank197265 +197266 POINT(40.80273383031194 73.87309055925114) bank197266 +197267 POINT(40.17015877933204 74.25262552498265) bank197267 +197268 POINT(40.287386213244666 73.50544243547427) bank197268 +197269 POINT(40.7935687200652 74.47272149143352) bank197269 +197270 POINT(41.159988171597185 73.110372794642) bank197270 +197271 POINT(39.71311167641952 73.43656274690099) bank197271 +197272 POINT(40.89621838306287 73.02949165916274) bank197272 +197273 POINT(40.96325947185921 73.78889607902731) bank197273 +197274 POINT(40.22810627873337 73.19442971904805) bank197274 +197275 POINT(40.609392520451465 73.77305231304281) bank197275 +197276 POINT(40.73657140496421 74.45644866898492) bank197276 +197277 POINT(40.486401836802834 73.81979240992668) bank197277 +197278 POINT(40.3070094934731 74.17883070688085) bank197278 +197279 POINT(39.93370323467605 73.8311128481106) bank197279 +197280 POINT(40.856242833080515 73.1808277376071) bank197280 +197281 POINT(40.400655697379776 74.89751785986424) bank197281 +197282 POINT(41.44525355301181 73.3202039163904) bank197282 +197283 POINT(40.154602698256205 73.47469615717397) bank197283 +197284 POINT(40.294986956827024 73.51225351356914) bank197284 +197285 POINT(41.05896558095598 73.5867713338654) bank197285 +197286 POINT(39.80844353220098 74.70449872231985) bank197286 +197287 POINT(40.24414751012615 73.47396456600822) bank197287 +197288 POINT(41.10615737147734 73.04082437307153) bank197288 +197289 POINT(40.88257564798938 74.33188622156204) bank197289 +197290 POINT(40.07304042893851 73.74977247728127) bank197290 +197291 POINT(39.757415952372966 74.67739584363031) bank197291 +197292 POINT(40.90905296038994 73.8107034964218) bank197292 +197293 POINT(40.77961105696047 74.49779285388217) bank197293 +197294 POINT(41.2429792751303 73.44110532295795) bank197294 +197295 POINT(40.66976188107209 74.03858158068624) bank197295 +197296 POINT(40.299013329979154 73.88625147888023) bank197296 +197297 POINT(41.0609896589032 74.83284083336228) bank197297 +197298 POINT(40.7266870644458 73.51071205128284) bank197298 +197299 POINT(40.41243856714392 73.2404084401192) bank197299 +197300 POINT(41.273776446985906 74.47407700383361) bank197300 +197301 POINT(40.929862254743085 74.57767919034553) bank197301 +197302 POINT(41.086330985149765 74.55467333769863) bank197302 +197303 POINT(40.55117604794193 74.56941112905257) bank197303 +197304 POINT(41.169905578375584 74.95068171853218) bank197304 +197305 POINT(40.44886209263602 73.1205747856768) bank197305 +197306 POINT(41.415346330550506 74.25299494442527) bank197306 +197307 POINT(40.63089614027837 73.60158516415746) bank197307 +197308 POINT(41.22493372957663 74.10923293601128) bank197308 +197309 POINT(40.56331329420152 74.48558532912439) bank197309 +197310 POINT(41.712005037246904 74.1633560106376) bank197310 +197311 POINT(40.7557183907313 74.56142608862977) bank197311 +197312 POINT(40.67693946360784 73.41211164951888) bank197312 +197313 POINT(41.31605699979746 74.86571146767807) bank197313 +197314 POINT(39.99578886489667 73.4497186110399) bank197314 +197315 POINT(40.61751990064911 73.23244349016733) bank197315 +197316 POINT(41.554636936317586 73.40958543033786) bank197316 +197317 POINT(41.01942722570609 74.52414793153322) bank197317 +197318 POINT(41.22799055402263 74.79266729532087) bank197318 +197319 POINT(41.348392945600644 73.43426633286794) bank197319 +197320 POINT(40.32390396744919 73.64356620442314) bank197320 +197321 POINT(39.83983022317369 74.69241604495537) bank197321 +197322 POINT(40.67154747518998 73.32227288289532) bank197322 +197323 POINT(41.168447829309954 73.84429239373652) bank197323 +197324 POINT(40.645467715409886 73.13086667405223) bank197324 +197325 POINT(40.413762292651015 73.79902525201904) bank197325 +197326 POINT(41.541169772682444 74.08471291520532) bank197326 +197327 POINT(41.57891340128716 74.30653743470273) bank197327 +197328 POINT(40.13717608394819 73.56850978021853) bank197328 +197329 POINT(41.38008735992472 73.34632532932049) bank197329 +197330 POINT(40.1368468267576 73.14352110149477) bank197330 +197331 POINT(41.06334856186479 73.04358075205332) bank197331 +197332 POINT(40.75198322218697 74.90119674261709) bank197332 +197333 POINT(41.097535087628586 73.34249553796887) bank197333 +197334 POINT(40.74721137463708 74.98179304308208) bank197334 +197335 POINT(40.63482609945402 73.79253648902895) bank197335 +197336 POINT(41.47072644449054 73.59285802733181) bank197336 +197337 POINT(40.76322490974344 73.53532768870392) bank197337 +197338 POINT(39.762038869640385 74.1188754017176) bank197338 +197339 POINT(41.5580452428905 74.54201052434756) bank197339 +197340 POINT(41.08228989464247 74.36221696528884) bank197340 +197341 POINT(41.394509943145536 73.80619227266183) bank197341 +197342 POINT(40.68726970821616 74.01037338582604) bank197342 +197343 POINT(40.28346728929184 74.98026925098169) bank197343 +197344 POINT(40.98698464024616 74.36887634595558) bank197344 +197345 POINT(41.62703484775601 73.74321022791406) bank197345 +197346 POINT(40.3907029884742 73.51031095631389) bank197346 +197347 POINT(39.822841193856824 74.80567220279431) bank197347 +197348 POINT(41.670357925988476 74.83197460563092) bank197348 +197349 POINT(40.60039668138708 73.20614305291659) bank197349 +197350 POINT(40.03166746158832 73.5012857148961) bank197350 +197351 POINT(40.83832784840195 74.29289586581267) bank197351 +197352 POINT(41.56479410722512 74.40118913309544) bank197352 +197353 POINT(40.01434621643336 74.92188597834766) bank197353 +197354 POINT(41.294442012519 74.07935904920829) bank197354 +197355 POINT(40.62093961057082 74.80044953405579) bank197355 +197356 POINT(40.27225545928708 74.02856913081897) bank197356 +197357 POINT(40.06375327140009 74.85039494562365) bank197357 +197358 POINT(40.040581321896816 74.44709413376677) bank197358 +197359 POINT(40.76321441332498 73.95794245641939) bank197359 +197360 POINT(40.636051725282925 73.95996919441063) bank197360 +197361 POINT(40.16502007104969 74.84541743259278) bank197361 +197362 POINT(40.528115746634754 73.09549309046213) bank197362 +197363 POINT(40.57942255479016 73.01632799460684) bank197363 +197364 POINT(40.35643138188136 74.23199201388475) bank197364 +197365 POINT(41.4062057208695 74.10362935446064) bank197365 +197366 POINT(39.74434424687971 74.18638099366513) bank197366 +197367 POINT(39.914004025501654 73.37798639759272) bank197367 +197368 POINT(40.50618057627934 73.49067268849696) bank197368 +197369 POINT(41.05854978550792 74.15876557364426) bank197369 +197370 POINT(40.98117488751936 73.23576107888782) bank197370 +197371 POINT(40.494511661338144 73.27734284012406) bank197371 +197372 POINT(39.98146295402512 73.51585795807881) bank197372 +197373 POINT(41.42774037157594 74.15129420019505) bank197373 +197374 POINT(39.87736578400431 74.32245586538143) bank197374 +197375 POINT(41.26551234615287 74.82532386628031) bank197375 +197376 POINT(41.083165805670234 74.48703633674705) bank197376 +197377 POINT(41.392053048460305 73.9072235885731) bank197377 +197378 POINT(41.17131327502237 74.81208046457752) bank197378 +197379 POINT(41.248187224347134 74.13995440037452) bank197379 +197380 POINT(41.57150056976356 74.76473209877184) bank197380 +197381 POINT(40.25642215268152 74.00474299358615) bank197381 +197382 POINT(40.72808703227441 73.8810405731243) bank197382 +197383 POINT(39.8715520758953 73.896151182922) bank197383 +197384 POINT(41.431219827736534 73.02098882945397) bank197384 +197385 POINT(39.9855470645514 74.13858387839367) bank197385 +197386 POINT(40.884521929465 73.08770891906403) bank197386 +197387 POINT(41.08021226113401 74.48226645767295) bank197387 +197388 POINT(41.67636489885993 73.74822285990857) bank197388 +197389 POINT(40.75575750991159 73.33470920817082) bank197389 +197390 POINT(40.520867173273906 74.63238000754785) bank197390 +197391 POINT(40.41758936648 74.81119629267545) bank197391 +197392 POINT(41.105327165127605 74.95586394278287) bank197392 +197393 POINT(40.36053915515827 74.3977652192415) bank197393 +197394 POINT(40.24830738824743 74.59454551996123) bank197394 +197395 POINT(40.697098565550654 74.70265585475933) bank197395 +197396 POINT(41.44296054533043 74.31631910136615) bank197396 +197397 POINT(41.63027282103252 73.0910635917946) bank197397 +197398 POINT(39.804761006896754 73.99321097024867) bank197398 +197399 POINT(41.6115989652944 74.7825772223574) bank197399 +197400 POINT(40.28390078952665 73.23236656181847) bank197400 +197401 POINT(41.05387276719775 73.08693054141406) bank197401 +197402 POINT(40.172663694666014 74.6460139530955) bank197402 +197403 POINT(40.63714326643837 73.33986869196684) bank197403 +197404 POINT(39.71848208552182 73.92798745678762) bank197404 +197405 POINT(40.75402906255804 74.33708769975053) bank197405 +197406 POINT(40.73289650997947 74.45419053784282) bank197406 +197407 POINT(41.30661425771691 74.06184791846931) bank197407 +197408 POINT(40.3019276014913 73.88735080674233) bank197408 +197409 POINT(39.829067629333494 74.89602614772143) bank197409 +197410 POINT(40.58317206876812 74.46411603351008) bank197410 +197411 POINT(39.71819006223896 73.9569135867707) bank197411 +197412 POINT(39.74535495985339 74.65382269392569) bank197412 +197413 POINT(40.363985024913504 74.55495578343867) bank197413 +197414 POINT(41.30024158674344 74.61937727421846) bank197414 +197415 POINT(40.963790498389784 74.77079792270901) bank197415 +197416 POINT(41.22594131840653 73.95631146532298) bank197416 +197417 POINT(40.6386362753753 74.58238168287654) bank197417 +197418 POINT(40.605247733182466 74.03813089885737) bank197418 +197419 POINT(40.86275074565084 73.01719209065507) bank197419 +197420 POINT(41.005710192053776 73.30486217354951) bank197420 +197421 POINT(40.84685537155385 74.90116080403351) bank197421 +197422 POINT(40.9177622076267 73.99217681198411) bank197422 +197423 POINT(39.75438532873142 73.93519190377775) bank197423 +197424 POINT(41.53534887803601 73.8424175852588) bank197424 +197425 POINT(40.45072284345375 73.81031123621355) bank197425 +197426 POINT(41.012060658735045 73.65165345281015) bank197426 +197427 POINT(39.95575658553816 73.64661610022162) bank197427 +197428 POINT(40.339765159850124 74.30382426045216) bank197428 +197429 POINT(41.338007148358976 73.94710524781472) bank197429 +197430 POINT(40.47861259965532 74.78954999119844) bank197430 +197431 POINT(41.27770064781011 73.81849455458833) bank197431 +197432 POINT(40.51866377471996 74.82994272897673) bank197432 +197433 POINT(41.37872827559752 73.84610329088868) bank197433 +197434 POINT(40.0235362866235 74.57827684071914) bank197434 +197435 POINT(40.06624193962991 73.22755970932589) bank197435 +197436 POINT(41.06221987656108 73.5369164665599) bank197436 +197437 POINT(41.26057306823899 74.36455277388147) bank197437 +197438 POINT(40.92696320889889 73.14323638341894) bank197438 +197439 POINT(39.848818605699186 73.70020559853704) bank197439 +197440 POINT(41.47924938937743 74.10469470347516) bank197440 +197441 POINT(41.38265056862604 73.02294466142538) bank197441 +197442 POINT(39.962905456341296 74.97338267485163) bank197442 +197443 POINT(41.316735508419484 73.84258044269183) bank197443 +197444 POINT(40.9728840924484 74.9068796616696) bank197444 +197445 POINT(41.08750138467688 74.71773094587941) bank197445 +197446 POINT(40.52434843969166 74.02279300346369) bank197446 +197447 POINT(40.82854471083768 74.02616974445827) bank197447 +197448 POINT(41.64288396512897 73.78597543906129) bank197448 +197449 POINT(41.23009007221011 74.48501016611162) bank197449 +197450 POINT(40.64941216538921 73.46147174372393) bank197450 +197451 POINT(41.607251762471805 74.57624289696176) bank197451 +197452 POINT(41.3501750237278 73.31019369287944) bank197452 +197453 POINT(40.014676773392424 74.91190568962321) bank197453 +197454 POINT(40.94027535278064 73.81286883946149) bank197454 +197455 POINT(40.08197209736159 74.97054744780093) bank197455 +197456 POINT(40.07650220165074 73.36915022099747) bank197456 +197457 POINT(40.33705049032827 73.8167699704002) bank197457 +197458 POINT(40.47215587702481 73.99873891975325) bank197458 +197459 POINT(39.94854041854806 73.71437833538951) bank197459 +197460 POINT(39.97097551025568 74.01737124009504) bank197460 +197461 POINT(40.748803097719254 74.84552508088825) bank197461 +197462 POINT(40.158935244190374 74.15344303452353) bank197462 +197463 POINT(39.83803879477639 74.50291914682386) bank197463 +197464 POINT(41.662112329620825 74.36819909944775) bank197464 +197465 POINT(40.48441061302232 73.3144365183931) bank197465 +197466 POINT(40.56923807585065 73.1404961561805) bank197466 +197467 POINT(40.825644937738964 73.88717441412781) bank197467 +197468 POINT(40.76597464322244 74.85063363819455) bank197468 +197469 POINT(40.229649910044614 74.5099955701855) bank197469 +197470 POINT(41.70449340416039 73.01369939660412) bank197470 +197471 POINT(41.40925706218376 74.11019942477596) bank197471 +197472 POINT(39.92511949572898 74.43829955554521) bank197472 +197473 POINT(41.184488589002534 74.6830843320982) bank197473 +197474 POINT(39.87701544709191 73.19555670380481) bank197474 +197475 POINT(40.778669140152296 73.06020717592958) bank197475 +197476 POINT(40.83767934788678 73.68241531942411) bank197476 +197477 POINT(41.51397641659123 73.82735496073134) bank197477 +197478 POINT(41.004465455050656 74.54075766023706) bank197478 +197479 POINT(40.24848284975462 73.63431338352713) bank197479 +197480 POINT(40.722358992746464 73.24167337544193) bank197480 +197481 POINT(41.56439429346513 73.66101312728291) bank197481 +197482 POINT(40.277681584541746 73.42946420261592) bank197482 +197483 POINT(40.397098102241294 73.87577561448624) bank197483 +197484 POINT(40.52815886758715 74.5755494341107) bank197484 +197485 POINT(40.98571845529915 73.37929541603684) bank197485 +197486 POINT(41.59226908592671 74.42235201259174) bank197486 +197487 POINT(40.933813919198585 74.6247507868231) bank197487 +197488 POINT(40.298918954949315 74.29935955371205) bank197488 +197489 POINT(40.6460907879897 73.80191482734216) bank197489 +197490 POINT(40.40713979594489 73.38986924983445) bank197490 +197491 POINT(40.26934269604285 73.9632289351629) bank197491 +197492 POINT(41.509728691664016 73.56226334170219) bank197492 +197493 POINT(40.95648327321588 73.95765335436374) bank197493 +197494 POINT(40.025880961134 74.39698712467562) bank197494 +197495 POINT(40.14328954736096 74.49859337013433) bank197495 +197496 POINT(40.416871299909275 74.80915413450127) bank197496 +197497 POINT(40.63557950773826 74.78809902155169) bank197497 +197498 POINT(41.34782253522379 73.53745342655634) bank197498 +197499 POINT(41.25829455477175 73.25317048196077) bank197499 +197500 POINT(40.95754321372385 74.58291959749141) bank197500 +197501 POINT(40.75049284765272 74.82651091369118) bank197501 +197502 POINT(39.75051589038243 73.22897866208501) bank197502 +197503 POINT(40.36675310884565 73.36758172017782) bank197503 +197504 POINT(40.0167852742185 73.31201624383496) bank197504 +197505 POINT(39.81993162607604 73.34228957298016) bank197505 +197506 POINT(41.1960974111634 74.30713021884038) bank197506 +197507 POINT(40.91502288552805 74.40093248645225) bank197507 +197508 POINT(40.63902817456834 73.9650029444157) bank197508 +197509 POINT(40.305297432712976 74.84492962928066) bank197509 +197510 POINT(39.99755906308003 73.2542497096466) bank197510 +197511 POINT(41.703471965759924 73.08654462948775) bank197511 +197512 POINT(40.26117362471992 73.4564132962272) bank197512 +197513 POINT(40.59921758645138 73.14387815859168) bank197513 +197514 POINT(41.110726761967996 74.1017179302177) bank197514 +197515 POINT(41.11130564588761 73.23461891671764) bank197515 +197516 POINT(40.75873272436146 73.51921341230143) bank197516 +197517 POINT(40.03623957656275 73.50455553040865) bank197517 +197518 POINT(41.636506916515614 74.38980968324418) bank197518 +197519 POINT(41.34221856646741 74.14813977989336) bank197519 +197520 POINT(39.93888163684798 73.35068454736273) bank197520 +197521 POINT(41.197461651462675 74.06880260915368) bank197521 +197522 POINT(40.550562546078346 73.23697140287334) bank197522 +197523 POINT(40.27642487872595 74.74591046317694) bank197523 +197524 POINT(39.91058720395865 74.25778232015024) bank197524 +197525 POINT(40.88320013818087 73.94835205750003) bank197525 +197526 POINT(40.33268263825389 74.72037458614383) bank197526 +197527 POINT(40.76001317819754 74.61410760713036) bank197527 +197528 POINT(41.53373740438294 73.0860190005384) bank197528 +197529 POINT(39.728841366206964 73.12629430983588) bank197529 +197530 POINT(40.23923751012999 74.068136839823) bank197530 +197531 POINT(41.39834730114971 73.920658196174) bank197531 +197532 POINT(41.280368578909275 74.42163973322243) bank197532 +197533 POINT(40.81950865537299 74.3259629128213) bank197533 +197534 POINT(40.103036247485264 73.6530869089723) bank197534 +197535 POINT(39.980515417709825 73.4089572580093) bank197535 +197536 POINT(40.63094780847851 74.42102085767353) bank197536 +197537 POINT(40.029922873821526 73.24099580473674) bank197537 +197538 POINT(40.11566301525046 74.4843455947319) bank197538 +197539 POINT(40.73922513603644 73.65043795403632) bank197539 +197540 POINT(41.388200479405405 74.40126784234278) bank197540 +197541 POINT(40.332238564107136 74.58963226470973) bank197541 +197542 POINT(40.53204539377147 74.62549910483371) bank197542 +197543 POINT(41.35879811046273 73.43363361879582) bank197543 +197544 POINT(40.41152717550133 74.32758802034445) bank197544 +197545 POINT(40.56749922011963 73.86917232278606) bank197545 +197546 POINT(40.93461108605062 74.44701538266204) bank197546 +197547 POINT(40.90418269163801 74.78401155631781) bank197547 +197548 POINT(41.56567734079251 73.5639437231907) bank197548 +197549 POINT(40.94558935915618 74.43122853713173) bank197549 +197550 POINT(40.32847983839969 74.52397574645227) bank197550 +197551 POINT(41.13258571946443 74.66217973204206) bank197551 +197552 POINT(40.66864571283598 74.70363121977687) bank197552 +197553 POINT(41.38410734877974 74.69114129929662) bank197553 +197554 POINT(40.42733476899989 73.96491920001708) bank197554 +197555 POINT(41.25544524855566 74.85721299198435) bank197555 +197556 POINT(41.46280149651439 73.08591770646203) bank197556 +197557 POINT(40.21494348758419 73.45137930932088) bank197557 +197558 POINT(41.31765472613287 73.90635100896277) bank197558 +197559 POINT(41.39719494242195 74.2884368554433) bank197559 +197560 POINT(40.02178814638462 73.13599363643434) bank197560 +197561 POINT(40.54997402599422 73.24575559474405) bank197561 +197562 POINT(40.9461470493917 74.64139406922094) bank197562 +197563 POINT(41.06089132125313 73.72051766567584) bank197563 +197564 POINT(41.42501614515249 74.97316938239264) bank197564 +197565 POINT(41.53624578977258 73.2914202312013) bank197565 +197566 POINT(40.1691282069063 74.65921498756313) bank197566 +197567 POINT(39.723713331453865 73.31083073616271) bank197567 +197568 POINT(41.024634965966996 74.94591089782624) bank197568 +197569 POINT(40.99321908708369 73.53189083545949) bank197569 +197570 POINT(40.884115408554024 74.86969483414852) bank197570 +197571 POINT(41.49281562185373 74.14064917653728) bank197571 +197572 POINT(41.4091639713889 73.17398301288914) bank197572 +197573 POINT(41.39361186188056 73.97063447873217) bank197573 +197574 POINT(41.400209531868576 73.62609499091913) bank197574 +197575 POINT(41.392580704769436 73.35988318194651) bank197575 +197576 POINT(40.615107019819135 73.12835018034353) bank197576 +197577 POINT(41.38313176690317 73.62347847987) bank197577 +197578 POINT(41.584998311022034 74.44456439780171) bank197578 +197579 POINT(39.73323945006625 74.80527794616594) bank197579 +197580 POINT(41.18638630074815 73.12666382661894) bank197580 +197581 POINT(41.36604766841911 73.1215549089503) bank197581 +197582 POINT(39.822892891336096 73.59446179777856) bank197582 +197583 POINT(39.90956680928369 73.04857791872395) bank197583 +197584 POINT(39.941399522543485 73.62109367333599) bank197584 +197585 POINT(41.56761331406064 73.5439070847037) bank197585 +197586 POINT(40.65226738377722 73.58876446980966) bank197586 +197587 POINT(41.510309226779405 73.26413997589778) bank197587 +197588 POINT(41.06425158179421 73.87566971159215) bank197588 +197589 POINT(41.50701823100333 73.29748827707145) bank197589 +197590 POINT(40.51141468370848 73.9637093340037) bank197590 +197591 POINT(41.005129588777116 73.04877540706495) bank197591 +197592 POINT(40.82447494403264 74.54581328344155) bank197592 +197593 POINT(40.705212190542035 74.4828817121476) bank197593 +197594 POINT(41.611500345535006 73.00699976694207) bank197594 +197595 POINT(40.089879440534396 73.27344965876107) bank197595 +197596 POINT(40.88331149681646 73.08481312344202) bank197596 +197597 POINT(40.70572695773868 74.31279516384532) bank197597 +197598 POINT(40.07840286235092 74.75393856451731) bank197598 +197599 POINT(40.19981162569175 73.45510387909121) bank197599 +197600 POINT(41.629878271929236 74.92063913048116) bank197600 +197601 POINT(40.70334492713292 73.77174485880803) bank197601 +197602 POINT(40.99722193346363 73.94867062651105) bank197602 +197603 POINT(40.606486749488894 73.95537416981237) bank197603 +197604 POINT(40.47208611818013 73.10471994478476) bank197604 +197605 POINT(41.439128224766364 73.63917917865056) bank197605 +197606 POINT(39.81007697742052 74.83159138497359) bank197606 +197607 POINT(41.40227021358126 74.41339107152965) bank197607 +197608 POINT(40.81071143966039 74.07216685508376) bank197608 +197609 POINT(40.783981399491005 74.42823115881903) bank197609 +197610 POINT(40.242192713258284 73.44768439282834) bank197610 +197611 POINT(40.81869930589136 73.27970663455159) bank197611 +197612 POINT(40.07912133588975 74.27517232777919) bank197612 +197613 POINT(41.04301796236854 74.34221515170795) bank197613 +197614 POINT(39.83388195857306 74.16826705516156) bank197614 +197615 POINT(40.51533117189696 73.12801861610761) bank197615 +197616 POINT(39.89741836744055 73.26164983075816) bank197616 +197617 POINT(40.778394236972275 73.88621072874167) bank197617 +197618 POINT(41.231472239255346 74.7086145926877) bank197618 +197619 POINT(39.8059184294207 73.2389229406227) bank197619 +197620 POINT(40.082617980711035 74.37779686890478) bank197620 +197621 POINT(41.29215763084795 74.43130556454204) bank197621 +197622 POINT(40.53832584439245 73.85557822990786) bank197622 +197623 POINT(41.153340149359465 74.17126741793186) bank197623 +197624 POINT(40.337973979477596 74.83916485293416) bank197624 +197625 POINT(41.11158812280496 73.03741655424272) bank197625 +197626 POINT(39.87044852074611 73.96950473401843) bank197626 +197627 POINT(40.947733488476125 74.30847608158257) bank197627 +197628 POINT(39.88200200886622 74.31961651033221) bank197628 +197629 POINT(40.51421467106755 73.20297675213284) bank197629 +197630 POINT(41.60944858777217 74.06891244733667) bank197630 +197631 POINT(40.54786608651852 74.83146919878173) bank197631 +197632 POINT(41.130228698620144 74.99865165233167) bank197632 +197633 POINT(40.833298022346774 73.63842309554606) bank197633 +197634 POINT(41.61016457179732 74.91077147938402) bank197634 +197635 POINT(40.61040123464165 73.09603367788552) bank197635 +197636 POINT(41.5414811794465 73.38284133615373) bank197636 +197637 POINT(39.82717664225525 74.74049560465485) bank197637 +197638 POINT(41.094235695637344 73.7514894399604) bank197638 +197639 POINT(41.54327624430415 73.90929638951906) bank197639 +197640 POINT(40.28176381472248 73.24899869106433) bank197640 +197641 POINT(41.149073264020466 74.27941991922751) bank197641 +197642 POINT(40.33167711022282 74.18666343152516) bank197642 +197643 POINT(41.18556790184525 73.37121563273183) bank197643 +197644 POINT(41.263962225001634 73.83820190442836) bank197644 +197645 POINT(40.9990362422068 74.52529376418634) bank197645 +197646 POINT(40.41394598926918 73.0504330792989) bank197646 +197647 POINT(40.369858663661454 74.4687940437018) bank197647 +197648 POINT(41.14102472670499 73.72562428248364) bank197648 +197649 POINT(41.11374744665303 73.22659932728179) bank197649 +197650 POINT(40.50839198606513 74.76374809383367) bank197650 +197651 POINT(41.44578793025374 73.59285359681301) bank197651 +197652 POINT(40.66990696587156 74.10337166188492) bank197652 +197653 POINT(39.927507449235016 73.09462094234078) bank197653 +197654 POINT(41.2999350668292 73.15306304171332) bank197654 +197655 POINT(39.85310236459926 74.28816195470738) bank197655 +197656 POINT(40.616530392013935 74.92999560449853) bank197656 +197657 POINT(41.41992079166871 74.81802322406152) bank197657 +197658 POINT(39.92393566518788 74.5068028609399) bank197658 +197659 POINT(41.52010878242673 73.7263636722609) bank197659 +197660 POINT(40.38457538027303 73.34110496541079) bank197660 +197661 POINT(40.62678146865996 74.43692795700736) bank197661 +197662 POINT(39.82227687974582 73.77899372260669) bank197662 +197663 POINT(39.80500358908266 74.90681749104702) bank197663 +197664 POINT(40.38937834573594 73.47924766846194) bank197664 +197665 POINT(40.362686443794395 74.70404425856552) bank197665 +197666 POINT(41.5229970877228 73.01335132927237) bank197666 +197667 POINT(41.710781958067166 74.3600740279812) bank197667 +197668 POINT(41.682412418274055 74.35418003982763) bank197668 +197669 POINT(40.17579589792906 73.38445000072744) bank197669 +197670 POINT(39.76648416456333 73.8409383620923) bank197670 +197671 POINT(40.90986991030445 74.24389608329751) bank197671 +197672 POINT(40.8016510572631 74.68578214882514) bank197672 +197673 POINT(41.24663472965949 73.43363953167048) bank197673 +197674 POINT(41.42831487946994 73.22823190939307) bank197674 +197675 POINT(41.466825874838904 73.60249976317111) bank197675 +197676 POINT(40.67693863290804 73.99366344393292) bank197676 +197677 POINT(41.011128467614554 73.22196484266274) bank197677 +197678 POINT(41.669163452660705 73.08939822821189) bank197678 +197679 POINT(39.713353895107026 73.34146272659821) bank197679 +197680 POINT(41.0002826036123 73.5714429662055) bank197680 +197681 POINT(41.41056616157391 74.89460188613923) bank197681 +197682 POINT(40.093218622367594 74.47795733291323) bank197682 +197683 POINT(40.69390183684214 73.9001419766224) bank197683 +197684 POINT(39.99861902704938 73.4632576787933) bank197684 +197685 POINT(40.57226693892319 73.16001028682996) bank197685 +197686 POINT(39.92225987929069 73.55625426977765) bank197686 +197687 POINT(41.014363393613294 74.12605844629032) bank197687 +197688 POINT(40.163319882299064 73.65374663393314) bank197688 +197689 POINT(41.49352986175543 73.43113844408109) bank197689 +197690 POINT(41.59386227278169 74.15465882237436) bank197690 +197691 POINT(40.9133838939609 74.67990797947041) bank197691 +197692 POINT(40.911781333629186 74.46449575897063) bank197692 +197693 POINT(40.208944158062806 74.53764964090345) bank197693 +197694 POINT(40.388427491664295 74.63178977765996) bank197694 +197695 POINT(40.65218789518848 73.41348621260285) bank197695 +197696 POINT(40.72862612256242 73.96074841852997) bank197696 +197697 POINT(41.55073298086328 73.47123609196481) bank197697 +197698 POINT(41.08484794145402 74.87969096061697) bank197698 +197699 POINT(40.81118238598581 74.22078372227864) bank197699 +197700 POINT(40.06800284155545 73.64334022843357) bank197700 +197701 POINT(41.06179351244435 74.27733099791858) bank197701 +197702 POINT(40.698817600708736 73.58486257088182) bank197702 +197703 POINT(41.1432857253914 74.86205318403053) bank197703 +197704 POINT(39.864945164048436 73.55493917791733) bank197704 +197705 POINT(41.566051077590686 74.85088467765168) bank197705 +197706 POINT(40.362973531503805 74.31599583202453) bank197706 +197707 POINT(41.488959127559866 74.0355705489691) bank197707 +197708 POINT(40.68160756776388 74.56427077885431) bank197708 +197709 POINT(40.725089931620246 74.88466548266933) bank197709 +197710 POINT(39.884361563552034 74.75786759283537) bank197710 +197711 POINT(41.38744834637161 74.81446642701827) bank197711 +197712 POINT(41.24027566335107 73.17271653222159) bank197712 +197713 POINT(39.7129461706619 74.81614782646297) bank197713 +197714 POINT(40.86571146207134 74.67629498458255) bank197714 +197715 POINT(41.3312944044457 74.76011134033038) bank197715 +197716 POINT(40.16188277956289 74.00101506748882) bank197716 +197717 POINT(40.42631733119636 74.10998380113274) bank197717 +197718 POINT(41.20491050521138 73.08899907531924) bank197718 +197719 POINT(40.068865615808846 74.27367822251911) bank197719 +197720 POINT(40.57399853519793 74.90812003102323) bank197720 +197721 POINT(41.054738461548936 74.73341951278452) bank197721 +197722 POINT(40.57043189084649 73.26017734026541) bank197722 +197723 POINT(40.19846204207924 73.66662491115827) bank197723 +197724 POINT(41.175165373400006 73.28310587914417) bank197724 +197725 POINT(40.05443168247689 73.8113218504759) bank197725 +197726 POINT(39.720317107964064 73.67617872080815) bank197726 +197727 POINT(40.73053525472301 73.3930989471887) bank197727 +197728 POINT(39.83934722956526 73.5187868615816) bank197728 +197729 POINT(40.782999854364036 74.36940374220433) bank197729 +197730 POINT(40.95524429433397 74.60013296380298) bank197730 +197731 POINT(40.526247909344235 73.9799036786093) bank197731 +197732 POINT(40.69384588607985 74.05843133806465) bank197732 +197733 POINT(41.17570278032151 74.49592362024333) bank197733 +197734 POINT(40.97918937459029 74.16697307108201) bank197734 +197735 POINT(40.03579577014481 73.97151663319042) bank197735 +197736 POINT(41.49495757861842 74.82437520809447) bank197736 +197737 POINT(41.23506579254778 73.64855211367522) bank197737 +197738 POINT(40.37349627181022 74.17894221829121) bank197738 +197739 POINT(40.83909462254738 73.29541631660449) bank197739 +197740 POINT(41.501471184955676 73.81376403695408) bank197740 +197741 POINT(40.262983163453626 73.80854225708299) bank197741 +197742 POINT(41.295358051303836 74.58345642890846) bank197742 +197743 POINT(39.86119568304969 73.6029892899178) bank197743 +197744 POINT(39.956630923910026 74.73290909943029) bank197744 +197745 POINT(39.88303240794635 74.58726931014166) bank197745 +197746 POINT(41.46647576898388 75.00470326181701) bank197746 +197747 POINT(41.05171528289446 73.84670019483515) bank197747 +197748 POINT(39.88217024988728 74.9668577852414) bank197748 +197749 POINT(41.49450245251455 74.64794087772306) bank197749 +197750 POINT(40.648872379759084 74.66902544402181) bank197750 +197751 POINT(41.08692621011457 73.07684897265759) bank197751 +197752 POINT(40.887196238098085 74.34598604220079) bank197752 +197753 POINT(40.00566111536152 73.33243352444856) bank197753 +197754 POINT(41.321661790468596 74.85016663279367) bank197754 +197755 POINT(41.40296234083375 74.5440469810548) bank197755 +197756 POINT(40.638599619790895 73.71675455656302) bank197756 +197757 POINT(40.212808415171395 73.6108507076288) bank197757 +197758 POINT(41.204475716693466 74.0009998321894) bank197758 +197759 POINT(41.142261036278725 73.66900359979401) bank197759 +197760 POINT(41.582017891656285 73.19585175058336) bank197760 +197761 POINT(40.017332603993935 74.27098859874754) bank197761 +197762 POINT(40.91997486675261 73.48637957459744) bank197762 +197763 POINT(40.47088707308626 74.73284416120876) bank197763 +197764 POINT(39.75810956335709 74.59787001715462) bank197764 +197765 POINT(40.27559442663784 74.45738428960709) bank197765 +197766 POINT(40.74790156735001 74.7917645013013) bank197766 +197767 POINT(41.56077249653475 74.10162267906985) bank197767 +197768 POINT(39.81405934840329 73.63182223103806) bank197768 +197769 POINT(40.29052315961438 73.34713491982859) bank197769 +197770 POINT(40.63043532999703 74.33354583419204) bank197770 +197771 POINT(40.525538168281 73.93579791617768) bank197771 +197772 POINT(41.358296875051266 74.68844470215205) bank197772 +197773 POINT(40.685556077974034 73.08492633766294) bank197773 +197774 POINT(39.71845256294752 73.41582778597274) bank197774 +197775 POINT(40.08967971801809 73.87883545161243) bank197775 +197776 POINT(40.63412048638526 73.13134572875964) bank197776 +197777 POINT(40.279378628003165 74.88527525220466) bank197777 +197778 POINT(39.94201632635257 74.37798300331823) bank197778 +197779 POINT(41.30403251045764 73.10294452663902) bank197779 +197780 POINT(41.27063629205747 73.69289362775196) bank197780 +197781 POINT(39.86514395529394 73.81118645616789) bank197781 +197782 POINT(39.93992709924427 74.07372337859728) bank197782 +197783 POINT(40.11602254467549 74.48197441948034) bank197783 +197784 POINT(40.16533679998961 73.94962288279055) bank197784 +197785 POINT(40.164513184266816 74.41336802763529) bank197785 +197786 POINT(41.335803272632646 73.41297612324131) bank197786 +197787 POINT(39.73592265067319 74.2775665529565) bank197787 +197788 POINT(41.49996626087963 74.18227946908404) bank197788 +197789 POINT(40.20135729242728 73.56465929012967) bank197789 +197790 POINT(41.45340993381265 73.79342587191518) bank197790 +197791 POINT(41.20155991808775 74.66898890839215) bank197791 +197792 POINT(41.412347319937474 73.70771082118672) bank197792 +197793 POINT(41.36743573975839 73.96178890223378) bank197793 +197794 POINT(41.05134016977958 73.17626069544394) bank197794 +197795 POINT(40.99602087333576 73.57419883348106) bank197795 +197796 POINT(40.09710866895497 73.94125001936794) bank197796 +197797 POINT(40.09300038173078 73.99504254071161) bank197797 +197798 POINT(39.811935721632345 73.47320136156746) bank197798 +197799 POINT(41.15327524311256 73.12319968069511) bank197799 +197800 POINT(40.417322078689146 74.24189260949106) bank197800 +197801 POINT(40.75359863460404 73.83244453827886) bank197801 +197802 POINT(41.18559378706144 73.62177560041573) bank197802 +197803 POINT(41.08304284684608 74.7616159652241) bank197803 +197804 POINT(41.197026899903236 73.41710527370324) bank197804 +197805 POINT(41.61610384672139 73.46988298128156) bank197805 +197806 POINT(39.78364298400161 73.60353172296568) bank197806 +197807 POINT(40.22254815026709 74.63461230562594) bank197807 +197808 POINT(40.36450138446668 73.18335899629868) bank197808 +197809 POINT(40.37873038728541 73.37955628783972) bank197809 +197810 POINT(41.08733287507606 73.23070031214013) bank197810 +197811 POINT(41.04485796482097 73.70296237748556) bank197811 +197812 POINT(41.028051496793594 73.9250805591096) bank197812 +197813 POINT(41.37667822476816 73.34662642983292) bank197813 +197814 POINT(41.580996242621595 73.58229655807621) bank197814 +197815 POINT(40.86143320585501 74.40730952889159) bank197815 +197816 POINT(41.07550620010195 73.55569380011526) bank197816 +197817 POINT(40.132607060993145 73.87315025726286) bank197817 +197818 POINT(39.77744275920755 74.49297945058746) bank197818 +197819 POINT(40.224618565149996 73.77972787241265) bank197819 +197820 POINT(40.200467185767046 73.51369540665011) bank197820 +197821 POINT(40.921904906634175 73.25369669446621) bank197821 +197822 POINT(40.31840258298146 74.36021565612842) bank197822 +197823 POINT(40.51040597158572 73.34291935782662) bank197823 +197824 POINT(39.78092564314664 73.20316239190187) bank197824 +197825 POINT(40.725990352203176 74.52648728988926) bank197825 +197826 POINT(41.29433053915131 73.28483354560022) bank197826 +197827 POINT(41.41245764188006 73.70752104826268) bank197827 +197828 POINT(41.68805103863818 74.16562997140028) bank197828 +197829 POINT(40.34038626170412 74.9196699378098) bank197829 +197830 POINT(41.57438383613861 74.3936977122294) bank197830 +197831 POINT(40.72338879938264 74.83378046286363) bank197831 +197832 POINT(39.751820497822074 74.86260204276874) bank197832 +197833 POINT(39.79768692710144 74.3589837914717) bank197833 +197834 POINT(39.96534988436543 73.82665220326058) bank197834 +197835 POINT(40.142638708797776 74.39626010360858) bank197835 +197836 POINT(40.68066941899546 73.04290601141119) bank197836 +197837 POINT(40.614026520927176 73.7464365854938) bank197837 +197838 POINT(41.4454701896816 73.04217161753175) bank197838 +197839 POINT(40.22524419696798 74.69996756269609) bank197839 +197840 POINT(41.51991637480168 74.42986183306077) bank197840 +197841 POINT(41.04829130970808 74.56355695831719) bank197841 +197842 POINT(41.46175732362761 73.79605050975836) bank197842 +197843 POINT(40.4575687728868 74.0302206723633) bank197843 +197844 POINT(40.34799661042028 73.84835274905235) bank197844 +197845 POINT(40.868375484693196 73.78626549026313) bank197845 +197846 POINT(39.845448454605396 73.84081880397075) bank197846 +197847 POINT(40.366971163172614 73.86710852485507) bank197847 +197848 POINT(41.24726414568302 74.77143468374956) bank197848 +197849 POINT(39.91773617448855 74.37284180058138) bank197849 +197850 POINT(41.13778176701637 73.85581699819548) bank197850 +197851 POINT(40.25460460173822 73.7676432922968) bank197851 +197852 POINT(40.78417761507312 74.68096722144347) bank197852 +197853 POINT(40.14780827655509 74.00927133064516) bank197853 +197854 POINT(40.245049075166165 73.71456157508806) bank197854 +197855 POINT(41.23147352780149 74.97041324151606) bank197855 +197856 POINT(39.79574735211826 73.85554281061002) bank197856 +197857 POINT(39.961395343696104 74.26973827062346) bank197857 +197858 POINT(40.53083644388574 74.28285011341903) bank197858 +197859 POINT(40.999336741677226 73.3191672660053) bank197859 +197860 POINT(41.209587255800464 73.4304979605871) bank197860 +197861 POINT(39.96200629814804 73.79029487727094) bank197861 +197862 POINT(40.50093510448312 74.94764777393821) bank197862 +197863 POINT(40.1399262563116 73.06568228201937) bank197863 +197864 POINT(40.570628894925036 73.90040600262883) bank197864 +197865 POINT(39.999953560803164 74.82373137264933) bank197865 +197866 POINT(41.13095404880592 74.37420368624362) bank197866 +197867 POINT(41.39287943620822 74.28048046776955) bank197867 +197868 POINT(39.87880577173028 73.83115253484581) bank197868 +197869 POINT(40.62790253367878 74.70202556488132) bank197869 +197870 POINT(40.9958291797062 74.94083543105785) bank197870 +197871 POINT(41.20293477473077 73.8830904240682) bank197871 +197872 POINT(40.19125579680635 74.98117319140272) bank197872 +197873 POINT(41.66827429899737 74.13940330268942) bank197873 +197874 POINT(41.62083626334376 73.63232577969735) bank197874 +197875 POINT(41.68892682720693 73.3393733038742) bank197875 +197876 POINT(41.499723468529325 73.71519216928537) bank197876 +197877 POINT(40.429352546973206 74.78908079374224) bank197877 +197878 POINT(41.33686539677338 74.10199900446146) bank197878 +197879 POINT(41.67926952089704 73.197369957797) bank197879 +197880 POINT(41.28564212603208 73.53571208188642) bank197880 +197881 POINT(39.74922158339085 74.92835611035953) bank197881 +197882 POINT(40.195888003867125 74.05633306723631) bank197882 +197883 POINT(39.72912891224057 74.69361149779) bank197883 +197884 POINT(40.09648179782204 74.60875327913051) bank197884 +197885 POINT(40.6963288926663 73.3904388060509) bank197885 +197886 POINT(41.3265489142082 73.07088200774342) bank197886 +197887 POINT(41.63741149819991 73.11130092338111) bank197887 +197888 POINT(39.90429615500251 74.61269438189102) bank197888 +197889 POINT(41.26668779552548 74.04392569081747) bank197889 +197890 POINT(40.68388183246472 74.91477317155453) bank197890 +197891 POINT(40.40845918114921 74.42847094911767) bank197891 +197892 POINT(41.56300510451107 73.27325856898739) bank197892 +197893 POINT(40.393250833630056 74.42525124010572) bank197893 +197894 POINT(40.74349755772973 73.94444414109888) bank197894 +197895 POINT(40.14247469610561 73.12587026529017) bank197895 +197896 POINT(41.31353553501808 73.30624501527028) bank197896 +197897 POINT(41.168235379542836 73.16071375164981) bank197897 +197898 POINT(41.24261652327397 73.12910312337583) bank197898 +197899 POINT(41.65031113343836 73.11261131491635) bank197899 +197900 POINT(39.72853212536866 74.25182330802636) bank197900 +197901 POINT(41.674985210708705 73.26767275072113) bank197901 +197902 POINT(39.777960867172595 74.33657835807631) bank197902 +197903 POINT(41.363342913973746 73.03099994969628) bank197903 +197904 POINT(41.38781398304011 74.67307571820302) bank197904 +197905 POINT(41.07001867006889 74.09245401837576) bank197905 +197906 POINT(40.92831481738701 73.6194621938081) bank197906 +197907 POINT(40.11186003832149 74.2477618121753) bank197907 +197908 POINT(39.88680922800145 73.13469765019191) bank197908 +197909 POINT(41.05197613676951 73.11814613979733) bank197909 +197910 POINT(40.21934749494798 74.89564342830998) bank197910 +197911 POINT(39.72718797381471 74.98387162969601) bank197911 +197912 POINT(40.59669578462883 74.32532460629491) bank197912 +197913 POINT(40.67333670090149 73.72153534237277) bank197913 +197914 POINT(41.12487660073432 74.21777324766505) bank197914 +197915 POINT(40.71424196271518 73.01114778422713) bank197915 +197916 POINT(41.33795819019399 73.46812391925144) bank197916 +197917 POINT(41.61597742626445 74.84556183745075) bank197917 +197918 POINT(40.351494550667304 73.3257840949775) bank197918 +197919 POINT(40.90603488812255 73.03662886342873) bank197919 +197920 POINT(40.6968923165914 73.66178962868544) bank197920 +197921 POINT(41.387961437658944 73.89210786220958) bank197921 +197922 POINT(40.4925771521055 74.83654603419295) bank197922 +197923 POINT(40.274018973769934 74.91789886628459) bank197923 +197924 POINT(41.231592152830615 73.51136206517033) bank197924 +197925 POINT(41.62295255349276 74.48148870261113) bank197925 +197926 POINT(39.85167671887923 74.86026026246002) bank197926 +197927 POINT(40.76370674089658 73.06725754407384) bank197927 +197928 POINT(39.80820565399908 73.45324495525391) bank197928 +197929 POINT(40.98923404501925 74.34883543140991) bank197929 +197930 POINT(41.28652975176843 74.20638050916777) bank197930 +197931 POINT(41.31990070768475 74.59751031965847) bank197931 +197932 POINT(41.50634150542222 74.5188398383451) bank197932 +197933 POINT(40.950375672660485 74.89669225959665) bank197933 +197934 POINT(41.67927399627051 73.96363142443957) bank197934 +197935 POINT(41.14652271246129 74.70469882292117) bank197935 +197936 POINT(40.71462863660799 73.82137453213703) bank197936 +197937 POINT(40.59713557478845 73.91422589214288) bank197937 +197938 POINT(40.653317055564735 74.58746634071518) bank197938 +197939 POINT(40.45678246114118 74.00057953546968) bank197939 +197940 POINT(39.75215886122276 73.48829143549597) bank197940 +197941 POINT(40.9684523255296 74.82546825852774) bank197941 +197942 POINT(40.703546915212996 73.26717609645378) bank197942 +197943 POINT(41.03108516265619 74.78278342849588) bank197943 +197944 POINT(41.13368326813579 74.21717250203898) bank197944 +197945 POINT(40.27884307316021 74.06690156877907) bank197945 +197946 POINT(40.45497754656793 73.76244350092657) bank197946 +197947 POINT(40.94372244628753 73.82409604746996) bank197947 +197948 POINT(41.02336281302131 73.24808301439289) bank197948 +197949 POINT(39.89123852519236 73.91528072426878) bank197949 +197950 POINT(41.548104040056614 73.3948007245721) bank197950 +197951 POINT(39.781037084717525 74.21140842008542) bank197951 +197952 POINT(40.00701828164863 73.36924303274829) bank197952 +197953 POINT(41.381620496088615 73.46393674986379) bank197953 +197954 POINT(41.12237539466519 73.56142551510021) bank197954 +197955 POINT(40.294564099535236 73.5038786540696) bank197955 +197956 POINT(40.83840766744811 74.61775200303846) bank197956 +197957 POINT(41.059849779869396 74.06565166134334) bank197957 +197958 POINT(39.90594109630686 74.10531095310306) bank197958 +197959 POINT(41.37985254228837 74.16359256494206) bank197959 +197960 POINT(41.41974981300475 74.09091099111525) bank197960 +197961 POINT(40.220922942415456 73.39948022985651) bank197961 +197962 POINT(39.79651976520489 74.40380793678229) bank197962 +197963 POINT(40.93051979842086 74.04462291803864) bank197963 +197964 POINT(39.86402951502422 74.9524597698227) bank197964 +197965 POINT(40.33103214499366 73.38359470908196) bank197965 +197966 POINT(41.622170367147646 73.8406179338476) bank197966 +197967 POINT(39.84915100177106 73.03310716260074) bank197967 +197968 POINT(41.66933141069019 74.3592498252016) bank197968 +197969 POINT(41.11373022377575 74.75483186803564) bank197969 +197970 POINT(41.52970629730587 73.61058384426336) bank197970 +197971 POINT(40.62119344582116 74.78176391680793) bank197971 +197972 POINT(41.10784022778821 74.61154922307763) bank197972 +197973 POINT(40.34920520808341 74.16139279878733) bank197973 +197974 POINT(39.80824064487561 73.92257868010827) bank197974 +197975 POINT(41.49774438110469 73.35625048466659) bank197975 +197976 POINT(41.168723828614915 73.19412071171719) bank197976 +197977 POINT(41.03761767725702 73.10820122211356) bank197977 +197978 POINT(40.1354609656034 74.16409599322785) bank197978 +197979 POINT(41.24158443726224 73.47879245021387) bank197979 +197980 POINT(39.95261876224967 73.32454122316915) bank197980 +197981 POINT(41.03127153120625 73.8182926291234) bank197981 +197982 POINT(41.53731416978595 73.215768862795) bank197982 +197983 POINT(41.170678215164635 74.10683886355477) bank197983 +197984 POINT(40.738243551016275 74.99690133412378) bank197984 +197985 POINT(40.76700347775709 73.98227107458288) bank197985 +197986 POINT(40.45688766690794 74.43494335211578) bank197986 +197987 POINT(39.84623495218356 73.91543455767706) bank197987 +197988 POINT(41.44727112770085 74.87904919196657) bank197988 +197989 POINT(41.46817837077884 73.5158808169042) bank197989 +197990 POINT(40.45925751039184 74.48216722885788) bank197990 +197991 POINT(41.357579586667484 73.23037234088281) bank197991 +197992 POINT(40.931638318040775 74.39079041067592) bank197992 +197993 POINT(40.54919272088088 73.13780991605876) bank197993 +197994 POINT(39.963290971414104 74.49238892270347) bank197994 +197995 POINT(39.99754413770069 73.01379706347075) bank197995 +197996 POINT(40.1782674706827 74.3727654415063) bank197996 +197997 POINT(39.8313969877568 73.22388166564325) bank197997 +197998 POINT(39.81061739390455 73.26969451595414) bank197998 +197999 POINT(41.05284013892705 73.41142044053039) bank197999 +198000 POINT(41.70639667232236 73.53673777098528) bank198000 +198001 POINT(41.01355920015048 73.68024149160837) bank198001 +198002 POINT(39.77734505510562 74.7352004544131) bank198002 +198003 POINT(40.82067469963325 73.67668173479929) bank198003 +198004 POINT(39.99140054998609 73.23929651071853) bank198004 +198005 POINT(40.359779266275005 73.5152013493215) bank198005 +198006 POINT(40.350394719461065 74.40351550063227) bank198006 +198007 POINT(40.482151583183025 73.33467817632783) bank198007 +198008 POINT(41.39607422606662 73.02882268294498) bank198008 +198009 POINT(40.904236052668516 73.3544961988056) bank198009 +198010 POINT(41.11314012292271 73.51757351056462) bank198010 +198011 POINT(40.56095718162894 74.19597358147949) bank198011 +198012 POINT(41.65226019467792 74.24420896150112) bank198012 +198013 POINT(40.66663488991279 73.95272322467703) bank198013 +198014 POINT(40.607539670840666 74.1703792967763) bank198014 +198015 POINT(40.87996952108471 74.39372085678656) bank198015 +198016 POINT(40.40297731134604 73.82483587484153) bank198016 +198017 POINT(40.646695609202 74.16559068476575) bank198017 +198018 POINT(39.983139360454835 74.4964886550328) bank198018 +198019 POINT(40.06607103632897 73.58333695901246) bank198019 +198020 POINT(40.41731947761879 74.25954730003241) bank198020 +198021 POINT(41.06666122248144 74.00565856247411) bank198021 +198022 POINT(40.9854551188384 74.3904593244042) bank198022 +198023 POINT(40.13829679778388 74.99460070243859) bank198023 +198024 POINT(40.735206285228095 73.83842907926048) bank198024 +198025 POINT(41.56037251443021 74.04817164257668) bank198025 +198026 POINT(41.529752721895 73.42325041282206) bank198026 +198027 POINT(40.74378290845275 73.63982602475022) bank198027 +198028 POINT(41.367028831831426 74.87484911035038) bank198028 +198029 POINT(40.878884724087314 74.20239311290109) bank198029 +198030 POINT(41.22517467226334 73.94602894806667) bank198030 +198031 POINT(40.2172552819718 73.650203859628) bank198031 +198032 POINT(41.280838712612415 74.87241949256693) bank198032 +198033 POINT(39.78867359330968 73.20367908392193) bank198033 +198034 POINT(40.20954363066504 74.49845351468771) bank198034 +198035 POINT(41.549645534729365 74.36189389013568) bank198035 +198036 POINT(40.112332656310116 74.20325051027186) bank198036 +198037 POINT(40.35356571519758 74.48133441190886) bank198037 +198038 POINT(41.55140181777912 74.6661469014436) bank198038 +198039 POINT(41.3281801621531 73.32207850216295) bank198039 +198040 POINT(40.20171427998694 73.63704703867502) bank198040 +198041 POINT(40.240391246491114 74.65156880545631) bank198041 +198042 POINT(41.24014384876043 73.31546498616039) bank198042 +198043 POINT(40.92626918785578 73.42121080959505) bank198043 +198044 POINT(41.076335084915215 73.25143076430567) bank198044 +198045 POINT(40.58945857951979 74.77673852097638) bank198045 +198046 POINT(40.04347299480838 74.78167360661094) bank198046 +198047 POINT(39.9513123189238 73.9200224986943) bank198047 +198048 POINT(40.008340957753866 73.33988937424311) bank198048 +198049 POINT(40.70148667351934 74.47700391662873) bank198049 +198050 POINT(40.96992466549851 73.16900128703173) bank198050 +198051 POINT(40.24281427847437 73.99880569698327) bank198051 +198052 POINT(39.805165400764125 73.1080602134146) bank198052 +198053 POINT(41.0128156472016 73.80319496611412) bank198053 +198054 POINT(40.04843130567941 74.32676497751797) bank198054 +198055 POINT(40.81064111394636 73.65882325309447) bank198055 +198056 POINT(40.570059914462824 73.17933647769404) bank198056 +198057 POINT(40.08583395329879 73.66739669146813) bank198057 +198058 POINT(41.28398296818802 74.78196369289537) bank198058 +198059 POINT(41.486897112010155 74.80690272157675) bank198059 +198060 POINT(40.22876718893442 74.38523956553246) bank198060 +198061 POINT(40.874038002691805 73.02814539060317) bank198061 +198062 POINT(40.52788165683946 74.19427721779219) bank198062 +198063 POINT(40.24906239226714 74.4276087975237) bank198063 +198064 POINT(40.381335739278754 74.14045487307742) bank198064 +198065 POINT(39.87104304835904 73.08170481876714) bank198065 +198066 POINT(41.26018397625101 73.44964398475534) bank198066 +198067 POINT(41.22694286728929 73.83424512678037) bank198067 +198068 POINT(41.5712075125375 73.1140613160657) bank198068 +198069 POINT(40.84388779206995 73.61509452970192) bank198069 +198070 POINT(39.753900760627246 74.60545686643337) bank198070 +198071 POINT(41.60585657206846 73.45558222134126) bank198071 +198072 POINT(40.95773083900581 74.02647909707441) bank198072 +198073 POINT(40.36966191501564 73.70282435377275) bank198073 +198074 POINT(40.02298654197019 73.2367694077619) bank198074 +198075 POINT(40.989980413907915 73.93181152485877) bank198075 +198076 POINT(40.318884610525956 73.7965183729269) bank198076 +198077 POINT(41.17540484204699 73.30435425675776) bank198077 +198078 POINT(40.85657990314941 73.80889890147824) bank198078 +198079 POINT(40.838339014314236 73.6267201361016) bank198079 +198080 POINT(41.023408761603015 73.48952155497985) bank198080 +198081 POINT(40.74648628133289 73.85527034002138) bank198081 +198082 POINT(39.932866352404595 73.75705151189237) bank198082 +198083 POINT(40.103018403021416 73.32390092064088) bank198083 +198084 POINT(40.50749390219782 73.54674620954788) bank198084 +198085 POINT(40.334458362125 74.138481981268) bank198085 +198086 POINT(41.63894384914369 74.92694343751569) bank198086 +198087 POINT(41.68974096394724 74.68946204724271) bank198087 +198088 POINT(39.84340728762346 73.85780836656973) bank198088 +198089 POINT(41.369621232467786 73.63795786731875) bank198089 +198090 POINT(41.416327814337876 74.16531199322642) bank198090 +198091 POINT(40.422648709898986 74.14049032696427) bank198091 +198092 POINT(40.61961593247756 74.19629778115551) bank198092 +198093 POINT(41.02054231991572 73.47341593570842) bank198093 +198094 POINT(40.28322609289714 74.40294530812763) bank198094 +198095 POINT(41.27159099072204 73.58001745664092) bank198095 +198096 POINT(41.64619442226863 74.94306189748575) bank198096 +198097 POINT(39.78952251525455 74.63139890226195) bank198097 +198098 POINT(40.75575977465443 73.83051358384529) bank198098 +198099 POINT(40.720556597319955 73.88544765115152) bank198099 +198100 POINT(39.809050123397526 73.67328151518265) bank198100 +198101 POINT(41.124008740000455 74.69038507040992) bank198101 +198102 POINT(41.31316827558609 73.48934966348669) bank198102 +198103 POINT(41.59503639698198 73.60688553054214) bank198103 +198104 POINT(40.09883524745803 73.79922268714449) bank198104 +198105 POINT(41.67670829588466 73.69799616028597) bank198105 +198106 POINT(40.77176127483625 73.70909042941459) bank198106 +198107 POINT(40.044833699993994 74.80802919841467) bank198107 +198108 POINT(40.84315983688086 74.00258922895132) bank198108 +198109 POINT(40.96083060202702 74.32919231381854) bank198109 +198110 POINT(40.97504428910676 74.83276734605312) bank198110 +198111 POINT(41.18060254974177 73.52031936048317) bank198111 +198112 POINT(40.917747699751416 74.644629285424) bank198112 +198113 POINT(41.34257782805117 74.56076749577358) bank198113 +198114 POINT(40.663830353216525 73.9794648209319) bank198114 +198115 POINT(41.01721824100432 74.34875770501196) bank198115 +198116 POINT(40.47729063254801 74.31899615072906) bank198116 +198117 POINT(39.87399700665779 74.44436301355846) bank198117 +198118 POINT(41.119359498697825 74.37621945899038) bank198118 +198119 POINT(39.94796321835061 73.97018227591398) bank198119 +198120 POINT(41.52365942759458 73.68828392155666) bank198120 +198121 POINT(40.93789537437598 73.96178138629335) bank198121 +198122 POINT(40.11908655992117 73.28938361416537) bank198122 +198123 POINT(41.408828935699816 74.30496926762788) bank198123 +198124 POINT(41.37539346199607 73.9489485359493) bank198124 +198125 POINT(40.74942504491889 74.3129374285943) bank198125 +198126 POINT(41.28073079463861 73.13696698109091) bank198126 +198127 POINT(40.067076618215346 74.43270284902339) bank198127 +198128 POINT(41.09785317782338 74.40404211041346) bank198128 +198129 POINT(41.42183573177761 74.75477419193945) bank198129 +198130 POINT(40.89860442438576 73.98913884445798) bank198130 +198131 POINT(40.35524718835839 74.5046022928834) bank198131 +198132 POINT(40.39996222852368 73.21369580936303) bank198132 +198133 POINT(41.13347442941516 73.47307833776428) bank198133 +198134 POINT(39.9167474168797 73.45493979667398) bank198134 +198135 POINT(40.40559629989864 74.69506430481626) bank198135 +198136 POINT(41.220210023749935 73.9090892553101) bank198136 +198137 POINT(40.81053007230134 74.78683922355422) bank198137 +198138 POINT(40.480711098351996 73.85348460100965) bank198138 +198139 POINT(41.70069186399732 73.41803539827075) bank198139 +198140 POINT(39.891780194741976 73.061248479409) bank198140 +198141 POINT(40.84359228216476 74.15357044909327) bank198141 +198142 POINT(41.35900698515322 74.00473632412124) bank198142 +198143 POINT(41.09893903799365 73.78122116621078) bank198143 +198144 POINT(40.37678672264509 74.3654417308441) bank198144 +198145 POINT(40.548689269248996 74.52316194743281) bank198145 +198146 POINT(40.61554660197932 74.35414261309666) bank198146 +198147 POINT(40.967171229678854 74.74929104641433) bank198147 +198148 POINT(41.26347928578163 73.4149262721338) bank198148 +198149 POINT(40.34364226827806 73.78253461494398) bank198149 +198150 POINT(40.91676011065119 73.92046628320112) bank198150 +198151 POINT(39.91157990597403 73.7958811009758) bank198151 +198152 POINT(41.25814552705386 73.94538992476372) bank198152 +198153 POINT(41.01850081553855 74.42242573437514) bank198153 +198154 POINT(41.48208755003582 74.52653118203504) bank198154 +198155 POINT(41.3692948157251 74.37089431026975) bank198155 +198156 POINT(40.97260220645362 73.03161138896809) bank198156 +198157 POINT(41.47533231855675 73.05647813043994) bank198157 +198158 POINT(41.04555537142352 74.03998277208791) bank198158 +198159 POINT(41.634458784652345 73.9870913144464) bank198159 +198160 POINT(39.78440205123505 74.81138867020775) bank198160 +198161 POINT(39.896836946148014 73.20683583041578) bank198161 +198162 POINT(41.55465631816487 74.26241818478105) bank198162 +198163 POINT(41.0854390399857 73.04024780555723) bank198163 +198164 POINT(41.69761243247214 73.78725995116737) bank198164 +198165 POINT(41.09587599813096 73.10330135562825) bank198165 +198166 POINT(41.29269460474453 74.66727176689282) bank198166 +198167 POINT(41.65349580332772 74.25924557684452) bank198167 +198168 POINT(41.68346278526175 73.8479957606673) bank198168 +198169 POINT(41.09623850269628 73.8752923506071) bank198169 +198170 POINT(40.464526848779 73.94839217932692) bank198170 +198171 POINT(41.4150534647801 73.50649126942007) bank198171 +198172 POINT(41.24925047795862 73.91100852827184) bank198172 +198173 POINT(40.38650556220949 74.31262326259835) bank198173 +198174 POINT(40.32837466130906 74.20263130751893) bank198174 +198175 POINT(40.33236987977304 74.76080359633559) bank198175 +198176 POINT(40.9453552385252 74.5773610160002) bank198176 +198177 POINT(41.484445235452135 73.97834453171838) bank198177 +198178 POINT(41.23550079136387 73.91426194734963) bank198178 +198179 POINT(40.765461664612076 74.09130471214426) bank198179 +198180 POINT(40.83814625283373 74.32278570048969) bank198180 +198181 POINT(40.93451083137406 73.24303485952908) bank198181 +198182 POINT(40.5841691961114 74.50955042617025) bank198182 +198183 POINT(41.67381942612455 73.78581141416677) bank198183 +198184 POINT(40.17286429940771 74.9342856420633) bank198184 +198185 POINT(40.46346182353786 74.97292254152912) bank198185 +198186 POINT(41.515514173773056 73.96854479722325) bank198186 +198187 POINT(41.1595241794138 73.7934324081436) bank198187 +198188 POINT(40.70128623623634 74.28980784142685) bank198188 +198189 POINT(41.37081707246737 73.30886966568455) bank198189 +198190 POINT(41.551221889500184 74.2861735931176) bank198190 +198191 POINT(41.02389354066155 73.72058873669857) bank198191 +198192 POINT(39.74143653295253 74.12039188521095) bank198192 +198193 POINT(41.16110480554247 74.09216543285679) bank198193 +198194 POINT(40.292345170657256 73.73420854329758) bank198194 +198195 POINT(39.716028402534384 73.40562668527555) bank198195 +198196 POINT(40.07308113251235 74.52186524512668) bank198196 +198197 POINT(40.93350549198107 74.59726939957714) bank198197 +198198 POINT(40.116926803821244 73.74350946920063) bank198198 +198199 POINT(40.9017137578321 73.62872317886661) bank198199 +198200 POINT(41.53989295866628 73.74352546873403) bank198200 +198201 POINT(41.57432109224856 74.49419944803654) bank198201 +198202 POINT(41.40972335088746 74.0223394437001) bank198202 +198203 POINT(40.269319547335506 73.119127026804) bank198203 +198204 POINT(39.780496252529424 74.74888891981486) bank198204 +198205 POINT(40.513004498452716 74.91029088127665) bank198205 +198206 POINT(41.29890776188337 73.41769772974642) bank198206 +198207 POINT(41.284775825327195 74.6253368467522) bank198207 +198208 POINT(40.539021484493965 74.42186454473813) bank198208 +198209 POINT(41.515814946652704 73.21520839173904) bank198209 +198210 POINT(40.40266176312652 74.56063815731027) bank198210 +198211 POINT(40.259064507366226 73.14336546177317) bank198211 +198212 POINT(41.320461708613294 73.91933633311346) bank198212 +198213 POINT(40.11724027198086 74.07096726679283) bank198213 +198214 POINT(39.90623187652859 74.46299317283679) bank198214 +198215 POINT(40.08942810002413 73.79819319263203) bank198215 +198216 POINT(40.063312630548126 73.66183765506165) bank198216 +198217 POINT(41.68575645305492 74.93476659113952) bank198217 +198218 POINT(41.13958851279536 73.81265326081557) bank198218 +198219 POINT(40.64573019728124 73.33003713274658) bank198219 +198220 POINT(40.85441654890648 74.31285294370767) bank198220 +198221 POINT(40.73646813443857 74.09269321850017) bank198221 +198222 POINT(41.072647638525865 73.64304979752434) bank198222 +198223 POINT(39.80208970789481 74.18303652877258) bank198223 +198224 POINT(40.252596300691096 73.24406568535159) bank198224 +198225 POINT(40.0568247881827 73.67650874776203) bank198225 +198226 POINT(41.63448742365834 74.39362913029699) bank198226 +198227 POINT(40.08150749564583 74.71594244980855) bank198227 +198228 POINT(40.08104920816662 73.62697976601089) bank198228 +198229 POINT(41.061885426001304 73.45849661201433) bank198229 +198230 POINT(41.673775370099555 73.92820747004106) bank198230 +198231 POINT(41.20397686662626 73.28383918465671) bank198231 +198232 POINT(40.989142409186776 74.06017735423154) bank198232 +198233 POINT(40.194975612044075 73.61090273098428) bank198233 +198234 POINT(40.26368396794958 73.68837338207594) bank198234 +198235 POINT(40.46011809368407 74.59445494825499) bank198235 +198236 POINT(40.09492953942364 73.55377258039405) bank198236 +198237 POINT(41.037200735601616 73.45670757776243) bank198237 +198238 POINT(40.265225256038114 73.99930961369743) bank198238 +198239 POINT(40.93248509184599 74.37395528483391) bank198239 +198240 POINT(40.325244327837794 74.86364617711067) bank198240 +198241 POINT(40.56808739586422 73.30519353066104) bank198241 +198242 POINT(41.481030241038866 74.89750110231233) bank198242 +198243 POINT(41.19510592747966 73.13567477126469) bank198243 +198244 POINT(40.639604757913446 73.74941036437613) bank198244 +198245 POINT(41.54503429873397 73.7352992166226) bank198245 +198246 POINT(40.27215668558852 74.36120670526255) bank198246 +198247 POINT(41.01839483339544 73.83697937839683) bank198247 +198248 POINT(39.97990800938323 73.43500703058776) bank198248 +198249 POINT(40.45401923605848 74.61909788774564) bank198249 +198250 POINT(40.095244872882645 74.26372226323555) bank198250 +198251 POINT(39.881072366252525 74.02177746847838) bank198251 +198252 POINT(40.5276984086432 73.84716573521082) bank198252 +198253 POINT(40.33178294277614 74.1671271244993) bank198253 +198254 POINT(41.00721308444366 73.65062364060564) bank198254 +198255 POINT(40.024883976382554 73.15583352824288) bank198255 +198256 POINT(40.1203922943223 73.22940914014656) bank198256 +198257 POINT(40.15251344313 73.98948141273134) bank198257 +198258 POINT(41.06807751368502 74.26723046625301) bank198258 +198259 POINT(40.74452401512287 74.85276884201916) bank198259 +198260 POINT(40.66924870366946 74.12410818353027) bank198260 +198261 POINT(40.52450003808336 73.93397948745195) bank198261 +198262 POINT(40.8860258512044 74.38195172752266) bank198262 +198263 POINT(41.196181178771944 73.72319503336522) bank198263 +198264 POINT(41.321469745760595 74.8603856798001) bank198264 +198265 POINT(41.65379044700639 74.67189471576336) bank198265 +198266 POINT(40.45477200106706 74.46478266025942) bank198266 +198267 POINT(40.4346873987096 74.0125441930073) bank198267 +198268 POINT(41.5721181544903 73.8922450677109) bank198268 +198269 POINT(39.81702859351079 74.07004270689382) bank198269 +198270 POINT(40.599647368289524 73.19983683780394) bank198270 +198271 POINT(40.94074596156625 73.4008321496969) bank198271 +198272 POINT(40.255432273291134 75.00041282025173) bank198272 +198273 POINT(40.56350947540095 74.925774428539) bank198273 +198274 POINT(41.05916318074415 74.48743456359557) bank198274 +198275 POINT(41.632775051834436 73.20732653676058) bank198275 +198276 POINT(40.89952797975482 74.669176445689) bank198276 +198277 POINT(40.25433144271188 73.30813083521569) bank198277 +198278 POINT(41.29389613953654 74.02006943605443) bank198278 +198279 POINT(40.71058248253992 73.13366964831677) bank198279 +198280 POINT(41.561657159068496 74.80421106064111) bank198280 +198281 POINT(40.61266447050821 73.77454043123623) bank198281 +198282 POINT(39.89036109458693 74.97663736934015) bank198282 +198283 POINT(41.26381889761877 75.00297227817335) bank198283 +198284 POINT(40.3485409591768 74.09531922731583) bank198284 +198285 POINT(41.13265573918526 74.3457978832146) bank198285 +198286 POINT(40.421747839885185 73.2739944823884) bank198286 +198287 POINT(39.98055518269572 74.18493117739644) bank198287 +198288 POINT(41.05500538660406 74.70028263455505) bank198288 +198289 POINT(41.480326973093575 73.0876965565758) bank198289 +198290 POINT(41.04984652474773 73.96096774505784) bank198290 +198291 POINT(40.19739961148129 74.65778205121246) bank198291 +198292 POINT(39.98688941772883 73.90881074620188) bank198292 +198293 POINT(40.084746883255036 73.88015607495717) bank198293 +198294 POINT(41.000504663489124 74.13627164982132) bank198294 +198295 POINT(41.35847069847723 74.45765258949044) bank198295 +198296 POINT(39.9295154727652 74.71813351184751) bank198296 +198297 POINT(41.51463101428236 74.83916053273374) bank198297 +198298 POINT(40.277050509999846 73.47094242268922) bank198298 +198299 POINT(41.181194968012136 74.32462968306663) bank198299 +198300 POINT(40.59641756821264 74.95799965737835) bank198300 +198301 POINT(41.14159494403083 73.47377598664575) bank198301 +198302 POINT(41.70262544588232 74.93555761773753) bank198302 +198303 POINT(40.00099931870879 74.76901883623897) bank198303 +198304 POINT(41.58998498391524 73.32294663171626) bank198304 +198305 POINT(40.011762319542854 73.25057749673518) bank198305 +198306 POINT(41.20852860166919 73.44423679958653) bank198306 +198307 POINT(41.27014643767682 73.21475108410422) bank198307 +198308 POINT(41.39316722960931 73.53461238929647) bank198308 +198309 POINT(40.03083249352894 74.17148433491114) bank198309 +198310 POINT(41.42118699842708 73.01305435602944) bank198310 +198311 POINT(41.39543331393576 73.73027112559416) bank198311 +198312 POINT(41.57812784138409 73.65969842422216) bank198312 +198313 POINT(40.44196624547029 74.57196852762013) bank198313 +198314 POINT(40.72059209985443 73.05353918622335) bank198314 +198315 POINT(39.86951878894129 74.46536085788168) bank198315 +198316 POINT(41.06943670763293 74.09339173336801) bank198316 +198317 POINT(39.94517188840974 74.49925113907867) bank198317 +198318 POINT(41.585192496531505 73.88969083248914) bank198318 +198319 POINT(41.39668638828397 74.7533349120529) bank198319 +198320 POINT(40.73134186422204 73.88693745086181) bank198320 +198321 POINT(40.284683515475024 73.62280390223304) bank198321 +198322 POINT(40.03262324484589 74.79696623496788) bank198322 +198323 POINT(40.79788683893251 73.25000917862461) bank198323 +198324 POINT(39.971449602065405 74.6958742338255) bank198324 +198325 POINT(41.04474115485747 74.78271695849033) bank198325 +198326 POINT(41.33775040571055 73.06673363568864) bank198326 +198327 POINT(40.971588081254176 74.30456702787814) bank198327 +198328 POINT(40.164292610154035 74.12764648968951) bank198328 +198329 POINT(40.430095932869506 73.32153089338357) bank198329 +198330 POINT(41.367488493526096 74.13594901790144) bank198330 +198331 POINT(40.386894404304584 73.4696312358975) bank198331 +198332 POINT(40.399900941690134 73.89346225960483) bank198332 +198333 POINT(41.26236326540289 74.95655269080125) bank198333 +198334 POINT(41.04093447497796 73.92169540730667) bank198334 +198335 POINT(41.45708687413143 74.7472843708776) bank198335 +198336 POINT(41.27100375387062 73.85847147371656) bank198336 +198337 POINT(41.544620464594054 74.70384069311393) bank198337 +198338 POINT(40.589360861733624 73.14342138956043) bank198338 +198339 POINT(41.42691197410288 74.29389201227693) bank198339 +198340 POINT(41.65094242809377 74.8709337823211) bank198340 +198341 POINT(40.850744173735166 73.43057010557251) bank198341 +198342 POINT(40.79530716550174 74.3062993810032) bank198342 +198343 POINT(41.63616429806232 74.25866814359583) bank198343 +198344 POINT(41.03529264801028 73.83395984074633) bank198344 +198345 POINT(40.41428775609896 73.26217475633436) bank198345 +198346 POINT(41.37735502229093 73.93132592596436) bank198346 +198347 POINT(41.28974110971021 74.2636230518079) bank198347 +198348 POINT(40.10826030538191 73.65633717045912) bank198348 +198349 POINT(41.17775822101957 73.62244862517278) bank198349 +198350 POINT(40.21003856436502 74.06610755319977) bank198350 +198351 POINT(41.61958921378785 73.4426207030397) bank198351 +198352 POINT(40.535244147897494 74.09269810466573) bank198352 +198353 POINT(40.042610157183574 73.33999276336557) bank198353 +198354 POINT(39.71287914141573 74.15260882121511) bank198354 +198355 POINT(41.238005942744344 73.3366804187261) bank198355 +198356 POINT(40.72346984865343 74.9040076375681) bank198356 +198357 POINT(40.5999766939684 74.7694353326124) bank198357 +198358 POINT(41.70647194768296 74.6425753289644) bank198358 +198359 POINT(40.94837846978961 73.84674918327312) bank198359 +198360 POINT(40.24078910293589 73.47734596623823) bank198360 +198361 POINT(41.01633650676127 73.69070973371866) bank198361 +198362 POINT(39.974560808661785 73.44338320121389) bank198362 +198363 POINT(40.032365484358806 73.54088181006604) bank198363 +198364 POINT(40.001587267215534 74.50792051437516) bank198364 +198365 POINT(40.69161567061345 74.62688281240416) bank198365 +198366 POINT(39.88960348825298 74.78933058235961) bank198366 +198367 POINT(40.11303405952664 74.53505836969723) bank198367 +198368 POINT(41.48759016682025 73.39495261791289) bank198368 +198369 POINT(39.79939633600332 73.9182197759115) bank198369 +198370 POINT(40.98516594856323 73.64305859497675) bank198370 +198371 POINT(40.60077963351004 74.89410690734371) bank198371 +198372 POINT(41.14497064627258 73.74587536713003) bank198372 +198373 POINT(40.199546338322776 74.31910850812336) bank198373 +198374 POINT(40.776976716132296 73.46478097582352) bank198374 +198375 POINT(41.26834033523053 73.14514044324271) bank198375 +198376 POINT(41.64136625741529 74.65169611884562) bank198376 +198377 POINT(40.43001627754515 73.88578066321573) bank198377 +198378 POINT(41.0973165423446 73.78087098063487) bank198378 +198379 POINT(39.87819975059357 73.83030954592364) bank198379 +198380 POINT(40.44104701525384 73.52119347647286) bank198380 +198381 POINT(40.514456929914836 74.71593142612369) bank198381 +198382 POINT(40.28726236844036 73.19658694906266) bank198382 +198383 POINT(40.36664969676543 73.73611290759833) bank198383 +198384 POINT(40.61772805882362 74.87045028285775) bank198384 +198385 POINT(41.56121168644206 74.89692582883711) bank198385 +198386 POINT(40.13931782336442 74.42223869199202) bank198386 +198387 POINT(41.282701070370614 73.06281050130983) bank198387 +198388 POINT(41.39106082065296 73.11252874441136) bank198388 +198389 POINT(40.263840907291915 73.30795020972052) bank198389 +198390 POINT(41.618902804399134 74.12931272762388) bank198390 +198391 POINT(40.32667879325956 74.43005845283997) bank198391 +198392 POINT(40.296240318502626 73.81681189359244) bank198392 +198393 POINT(41.47358711134092 74.51262472793499) bank198393 +198394 POINT(40.86078745279674 73.21817780360102) bank198394 +198395 POINT(40.37522091110987 74.0441253367961) bank198395 +198396 POINT(41.65302387642192 73.39079672572369) bank198396 +198397 POINT(41.380067063639856 73.90576555559619) bank198397 +198398 POINT(40.37236846342569 73.43587453061811) bank198398 +198399 POINT(41.069833230324235 73.35835561057927) bank198399 +198400 POINT(39.817396392142626 74.57142460021541) bank198400 +198401 POINT(40.72944843298173 74.29092685601968) bank198401 +198402 POINT(40.68630215013029 73.89659933017934) bank198402 +198403 POINT(39.88108572053324 73.76510212454885) bank198403 +198404 POINT(39.853463518375534 74.04631971642131) bank198404 +198405 POINT(40.92870756615859 73.90419207546222) bank198405 +198406 POINT(41.65535597307505 74.50095737412967) bank198406 +198407 POINT(40.69269132104386 73.92887129565815) bank198407 +198408 POINT(40.396948226120244 73.99509461035748) bank198408 +198409 POINT(40.39243081668872 74.19990980588024) bank198409 +198410 POINT(40.84113654938174 74.67805482238657) bank198410 +198411 POINT(41.276954768749874 74.82890149263217) bank198411 +198412 POINT(41.27015084797247 74.05247257947102) bank198412 +198413 POINT(41.01833332381422 73.4561975607961) bank198413 +198414 POINT(41.04694521479056 74.5241527606482) bank198414 +198415 POINT(40.25834385299119 73.61547623645767) bank198415 +198416 POINT(40.59803887800251 74.36262871048494) bank198416 +198417 POINT(41.278809534814904 73.89976515901371) bank198417 +198418 POINT(40.37039577219484 73.02721728215634) bank198418 +198419 POINT(41.55832282599044 73.77973870237348) bank198419 +198420 POINT(40.0364447547602 74.7464464171367) bank198420 +198421 POINT(40.15558001058346 75.00089971423147) bank198421 +198422 POINT(41.44535342857167 73.44879384145524) bank198422 +198423 POINT(41.51173494073121 74.2134665894907) bank198423 +198424 POINT(40.211573372966775 73.34069681176886) bank198424 +198425 POINT(41.042692492266696 73.11606089033054) bank198425 +198426 POINT(39.9102305768142 74.22976025026176) bank198426 +198427 POINT(40.33851757578213 74.36040778563111) bank198427 +198428 POINT(41.09991966382317 74.87092677067734) bank198428 +198429 POINT(41.52064132072206 73.36800550229867) bank198429 +198430 POINT(40.64784464457853 74.37605750157785) bank198430 +198431 POINT(40.080863975039215 74.14199772932244) bank198431 +198432 POINT(40.321229808217346 73.71852636720365) bank198432 +198433 POINT(40.526805525640825 74.94165462770782) bank198433 +198434 POINT(41.63048833642909 74.04135472083676) bank198434 +198435 POINT(41.4985484291306 74.07859809768736) bank198435 +198436 POINT(40.70761767020096 73.23661441847727) bank198436 +198437 POINT(39.79873982452635 73.23868338995402) bank198437 +198438 POINT(39.98520846202755 73.82777180350986) bank198438 +198439 POINT(40.05796944671289 73.12426791305062) bank198439 +198440 POINT(40.356318176859226 74.71403319707368) bank198440 +198441 POINT(41.36111809673318 73.30828002691841) bank198441 +198442 POINT(40.20029275862069 73.66145614251374) bank198442 +198443 POINT(40.82308780285456 73.05797359829819) bank198443 +198444 POINT(41.54631333414968 73.61067621752822) bank198444 +198445 POINT(40.883301135345604 73.28620723977238) bank198445 +198446 POINT(41.66378238575781 73.56628348661627) bank198446 +198447 POINT(40.85625232199133 74.8599928791904) bank198447 +198448 POINT(41.58657582624225 74.42878470363924) bank198448 +198449 POINT(41.218082867871985 74.04721419794964) bank198449 +198450 POINT(39.890263107283026 73.29205491661162) bank198450 +198451 POINT(40.81554186223127 73.8214475135778) bank198451 +198452 POINT(40.28130018619929 73.3224521406659) bank198452 +198453 POINT(41.07036788741443 74.43021446872807) bank198453 +198454 POINT(39.95041410590226 74.71444748818944) bank198454 +198455 POINT(40.710536865489765 73.80142138912052) bank198455 +198456 POINT(41.14480637365504 74.37388533247646) bank198456 +198457 POINT(41.569441032445276 74.22758575850044) bank198457 +198458 POINT(40.09613119758066 74.43107668155496) bank198458 +198459 POINT(39.75310118524154 73.33810419601424) bank198459 +198460 POINT(39.80239015318374 74.66732795900774) bank198460 +198461 POINT(40.98150331752033 74.67097041611127) bank198461 +198462 POINT(41.08744378863005 74.3460339571155) bank198462 +198463 POINT(41.48534100251086 73.36744016742725) bank198463 +198464 POINT(41.3535444210171 74.05038514529963) bank198464 +198465 POINT(40.184676028045004 74.67592830295334) bank198465 +198466 POINT(39.893646147366226 74.8630195021914) bank198466 +198467 POINT(40.36920764553045 74.40826506168182) bank198467 +198468 POINT(39.80162824668188 73.34751846157802) bank198468 +198469 POINT(40.14775782044947 73.54331420684379) bank198469 +198470 POINT(41.683620448293105 73.18330574889045) bank198470 +198471 POINT(40.016900550784825 74.76879208687892) bank198471 +198472 POINT(40.66785037088683 73.99273111088193) bank198472 +198473 POINT(39.91647560740414 73.58694831335033) bank198473 +198474 POINT(41.06162758435009 74.25811302694711) bank198474 +198475 POINT(40.41668087424147 74.5835164304246) bank198475 +198476 POINT(40.602806220503744 74.13438852837004) bank198476 +198477 POINT(39.793361083000256 73.55074759265138) bank198477 +198478 POINT(41.662073962842626 74.01410599916547) bank198478 +198479 POINT(40.861897199185364 73.43430757980708) bank198479 +198480 POINT(39.92905415737626 73.02868119303871) bank198480 +198481 POINT(40.80245421488864 73.89037042055668) bank198481 +198482 POINT(40.070700329792295 73.08896318484966) bank198482 +198483 POINT(40.11748865011592 74.04962975018626) bank198483 +198484 POINT(41.438409481861854 74.08299631448868) bank198484 +198485 POINT(40.51835952484104 73.82646486050429) bank198485 +198486 POINT(40.79642590832044 74.92782232472483) bank198486 +198487 POINT(40.61468833674829 74.3416901068743) bank198487 +198488 POINT(41.096946644396155 74.02076010123376) bank198488 +198489 POINT(40.3017185542438 73.06844275192509) bank198489 +198490 POINT(41.0276000746018 73.55155661543574) bank198490 +198491 POINT(40.21533282346559 73.31806137087506) bank198491 +198492 POINT(39.81835014606169 73.47365347549284) bank198492 +198493 POINT(41.0560555688408 74.80535766107879) bank198493 +198494 POINT(40.800803909522955 74.33973877226086) bank198494 +198495 POINT(40.505519441300514 74.30406056285078) bank198495 +198496 POINT(41.69198244467524 73.99619336859975) bank198496 +198497 POINT(41.07166185189873 74.36326948455722) bank198497 +198498 POINT(39.760033630964436 73.77246334974261) bank198498 +198499 POINT(41.3846640623607 73.15380696707923) bank198499 +198500 POINT(39.99551778325658 73.3780180981718) bank198500 +198501 POINT(40.18566442047141 73.06282315467769) bank198501 +198502 POINT(41.48068028377674 74.71339010891259) bank198502 +198503 POINT(39.80031510129049 74.225938093112) bank198503 +198504 POINT(40.06340405130484 73.53419260222073) bank198504 +198505 POINT(41.15693005760607 73.88560879953809) bank198505 +198506 POINT(40.88664095307653 73.24424728138399) bank198506 +198507 POINT(41.276015402856956 73.69715676362821) bank198507 +198508 POINT(41.27518278497335 73.35427234265481) bank198508 +198509 POINT(41.34606086280387 74.54467667663457) bank198509 +198510 POINT(41.45764501471917 73.47961584514354) bank198510 +198511 POINT(40.7345482069041 74.56612940799232) bank198511 +198512 POINT(41.34565264292229 74.37937804879509) bank198512 +198513 POINT(39.78932097759356 73.21923747715717) bank198513 +198514 POINT(41.2818635803247 73.15646672084469) bank198514 +198515 POINT(41.2555636281487 73.0100718836919) bank198515 +198516 POINT(41.377813485633276 74.50774232225075) bank198516 +198517 POINT(41.576497006736446 74.3090636568879) bank198517 +198518 POINT(41.02556219219388 74.10274523232461) bank198518 +198519 POINT(41.098094269583584 73.66934501981073) bank198519 +198520 POINT(40.889468656521494 73.9441656255861) bank198520 +198521 POINT(39.82661478470875 74.5939598069052) bank198521 +198522 POINT(41.386051121534834 74.33246045733746) bank198522 +198523 POINT(40.21894160302654 74.28647596289711) bank198523 +198524 POINT(40.48591251863073 73.40618164784064) bank198524 +198525 POINT(39.936983944922865 73.20760521308496) bank198525 +198526 POINT(40.33449144965766 74.78657095023452) bank198526 +198527 POINT(41.06987281368888 73.64394170000898) bank198527 +198528 POINT(40.064513109617664 74.88610059597409) bank198528 +198529 POINT(41.331858092547 73.10600105415935) bank198529 +198530 POINT(40.487526359874536 74.52932352305324) bank198530 +198531 POINT(41.6082959877505 74.76486449717157) bank198531 +198532 POINT(41.45770819857842 74.814972423854) bank198532 +198533 POINT(41.09800419249089 74.18677203163392) bank198533 +198534 POINT(40.977434172115636 73.58915101245631) bank198534 +198535 POINT(40.777248232330344 74.06333829807546) bank198535 +198536 POINT(40.248903272654175 73.43354003953306) bank198536 +198537 POINT(41.29486297571481 74.9770807271182) bank198537 +198538 POINT(41.556796864238045 74.45080244938158) bank198538 +198539 POINT(41.66491871084956 73.50911528772664) bank198539 +198540 POINT(41.200162843843536 74.94721849844116) bank198540 +198541 POINT(41.3591670053273 74.01858022923348) bank198541 +198542 POINT(41.02633902363555 74.29752554051504) bank198542 +198543 POINT(40.18785595928198 74.31614877378514) bank198543 +198544 POINT(40.408948168847566 73.43049566504463) bank198544 +198545 POINT(41.22882403019987 74.6097120724878) bank198545 +198546 POINT(40.09100715192465 74.47329207523006) bank198546 +198547 POINT(40.70659611083022 73.73802516864393) bank198547 +198548 POINT(40.216003590590105 73.98939470603105) bank198548 +198549 POINT(39.834668345721695 74.81373709068708) bank198549 +198550 POINT(40.65184854646853 73.41789301445965) bank198550 +198551 POINT(40.210513976946444 74.01738577828013) bank198551 +198552 POINT(40.48334725196929 73.75840604028625) bank198552 +198553 POINT(41.14476751113794 73.81917929429989) bank198553 +198554 POINT(41.61437168526899 74.13220727205342) bank198554 +198555 POINT(41.51355686633874 73.28271181090923) bank198555 +198556 POINT(41.18525405509036 74.64839762176031) bank198556 +198557 POINT(41.552441454368186 74.2331488668991) bank198557 +198558 POINT(40.928580913975466 74.30396830212348) bank198558 +198559 POINT(40.376289640347075 74.59211238164522) bank198559 +198560 POINT(41.687029160678776 73.98756857712286) bank198560 +198561 POINT(41.70444585061666 74.16201520726676) bank198561 +198562 POINT(40.34763879414878 73.19837735248787) bank198562 +198563 POINT(40.335741170852636 73.30016372965656) bank198563 +198564 POINT(41.17061317324217 73.00652441786521) bank198564 +198565 POINT(39.95428364477138 74.22863866217428) bank198565 +198566 POINT(40.85240083618644 74.55525630025227) bank198566 +198567 POINT(41.52334805650139 74.12970246038536) bank198567 +198568 POINT(41.563660761995926 74.86835425520053) bank198568 +198569 POINT(39.906064616573985 73.62615964309548) bank198569 +198570 POINT(40.16246421548048 73.41103641234498) bank198570 +198571 POINT(39.77532043761015 73.6753515345164) bank198571 +198572 POINT(40.7047542931045 74.95844977735034) bank198572 +198573 POINT(40.31336784990591 74.8796128618644) bank198573 +198574 POINT(40.93267564984989 73.07155011150697) bank198574 +198575 POINT(41.55108165552863 73.78372552655716) bank198575 +198576 POINT(41.39739425330404 74.9209729566236) bank198576 +198577 POINT(40.329039838570836 73.40374177651512) bank198577 +198578 POINT(40.06661517137278 73.30447765249752) bank198578 +198579 POINT(41.23275847096855 74.44334020603796) bank198579 +198580 POINT(40.50926292604686 74.24896131251904) bank198580 +198581 POINT(40.78039648161797 73.39325112210932) bank198581 +198582 POINT(40.8334947739731 74.91226142221963) bank198582 +198583 POINT(41.00948054284188 74.85685017007812) bank198583 +198584 POINT(41.5206357472125 73.17658810327238) bank198584 +198585 POINT(41.12741824955435 73.1834346593593) bank198585 +198586 POINT(40.47071993858454 74.96629866921435) bank198586 +198587 POINT(41.21154754584632 74.28790664877111) bank198587 +198588 POINT(41.484668208558446 74.3989458620717) bank198588 +198589 POINT(40.730493075949056 74.51388910851252) bank198589 +198590 POINT(40.947741320823205 73.49192558253951) bank198590 +198591 POINT(39.941348871326426 74.97444636981608) bank198591 +198592 POINT(39.785350951521025 74.68606829230525) bank198592 +198593 POINT(41.01694165104094 73.09014963938263) bank198593 +198594 POINT(40.703043284038294 74.0112455139606) bank198594 +198595 POINT(40.894673269308846 73.76458540740185) bank198595 +198596 POINT(40.70985901737852 73.06803971524583) bank198596 +198597 POINT(40.23462804881455 74.94676470895945) bank198597 +198598 POINT(39.90132407967535 73.33579345188774) bank198598 +198599 POINT(40.79911390224983 74.4573937198719) bank198599 +198600 POINT(40.371932115714294 73.07876336656952) bank198600 +198601 POINT(41.332550650076605 74.28121379806768) bank198601 +198602 POINT(40.83513368437288 74.73609832310122) bank198602 +198603 POINT(41.26324167260752 73.95785668639448) bank198603 +198604 POINT(40.83070179032825 73.55816488725367) bank198604 +198605 POINT(40.62925802126019 73.70506181051866) bank198605 +198606 POINT(40.18932358781531 73.81456034469842) bank198606 +198607 POINT(41.397633937863056 73.56734133176032) bank198607 +198608 POINT(41.18825513432875 73.13416990846761) bank198608 +198609 POINT(39.85040282472251 74.86426384084915) bank198609 +198610 POINT(40.55392341422122 74.40356429618646) bank198610 +198611 POINT(41.067724279617316 74.31850539440033) bank198611 +198612 POINT(40.964000227551644 74.04222557912985) bank198612 +198613 POINT(41.683524355656054 73.06189931863084) bank198613 +198614 POINT(40.9682030269106 74.53546737878676) bank198614 +198615 POINT(40.77112417726235 74.91299857817712) bank198615 +198616 POINT(40.191913909622045 74.12030496970611) bank198616 +198617 POINT(40.76822711318967 73.44997260030507) bank198617 +198618 POINT(41.237698876636344 73.18882970189725) bank198618 +198619 POINT(41.489049015369815 74.22338060008579) bank198619 +198620 POINT(40.44615610973917 74.86457148233785) bank198620 +198621 POINT(40.88589886091976 74.99142022113685) bank198621 +198622 POINT(40.049406566700625 73.125666730972) bank198622 +198623 POINT(41.32068721972454 73.96240969351557) bank198623 +198624 POINT(40.89876862530284 73.15737695259455) bank198624 +198625 POINT(40.602427140450175 73.28441686409091) bank198625 +198626 POINT(41.511320641089036 73.02339983890099) bank198626 +198627 POINT(40.667582069436534 73.04364204715236) bank198627 +198628 POINT(40.45574290784057 74.32838214081247) bank198628 +198629 POINT(40.98908721020119 73.26019390467269) bank198629 +198630 POINT(40.06882729412048 74.63029022987287) bank198630 +198631 POINT(40.82466703842786 73.95933362248682) bank198631 +198632 POINT(40.84479979317135 73.35897961795241) bank198632 +198633 POINT(41.00803840798508 73.43880272918369) bank198633 +198634 POINT(41.57222053875264 74.52206817464736) bank198634 +198635 POINT(41.31763540566096 73.4855283236575) bank198635 +198636 POINT(41.58194099823141 73.863647854083) bank198636 +198637 POINT(40.465297643910375 73.89540659711577) bank198637 +198638 POINT(39.99479487801966 73.01439819047602) bank198638 +198639 POINT(40.5311341288152 73.30173697121171) bank198639 +198640 POINT(40.65803665320444 73.43290903016486) bank198640 +198641 POINT(40.74554532475221 73.33055708284856) bank198641 +198642 POINT(39.79576419370878 74.19544170894713) bank198642 +198643 POINT(41.140984142827115 73.4978068477901) bank198643 +198644 POINT(41.16269284392234 73.70730625175769) bank198644 +198645 POINT(41.16539101501026 73.65320385786057) bank198645 +198646 POINT(40.00858840204098 73.38596167966818) bank198646 +198647 POINT(41.448060097566085 73.19884544283278) bank198647 +198648 POINT(40.97850088191318 73.43368553302167) bank198648 +198649 POINT(41.3330217100532 74.89086438063404) bank198649 +198650 POINT(40.69960113520753 73.5717775992233) bank198650 +198651 POINT(40.678341409615825 73.96682916820349) bank198651 +198652 POINT(41.36654775033174 73.62504612526816) bank198652 +198653 POINT(41.471169285719334 74.09136720389932) bank198653 +198654 POINT(40.95341924093752 74.36981661076436) bank198654 +198655 POINT(40.154871313303595 74.34154138122237) bank198655 +198656 POINT(40.51555440917861 74.12527390760398) bank198656 +198657 POINT(39.79773276506108 74.03646559958692) bank198657 +198658 POINT(41.26294603143468 73.72957560542086) bank198658 +198659 POINT(41.23862798425461 74.69213181108967) bank198659 +198660 POINT(40.18794485923882 73.27413750322688) bank198660 +198661 POINT(40.4883702446006 73.3102680288753) bank198661 +198662 POINT(40.0911809398748 73.24749564904961) bank198662 +198663 POINT(40.8540011358995 74.55468714062174) bank198663 +198664 POINT(41.389281094635535 74.72327418154151) bank198664 +198665 POINT(39.97783376732661 73.8720178257937) bank198665 +198666 POINT(39.89863119644317 74.66429521486599) bank198666 +198667 POINT(40.086287655714095 73.53069388729365) bank198667 +198668 POINT(40.332187618304 74.0789133535524) bank198668 +198669 POINT(40.35296229512931 73.36465838589794) bank198669 +198670 POINT(39.96347034642836 74.46170519138855) bank198670 +198671 POINT(41.66216685919162 74.8327436758721) bank198671 +198672 POINT(40.96853759778702 74.07640691772849) bank198672 +198673 POINT(39.93071602886736 74.65813156812457) bank198673 +198674 POINT(40.32189571803133 73.51514272355847) bank198674 +198675 POINT(39.93093676862847 73.9433268885504) bank198675 +198676 POINT(41.70580759262731 74.13579382998763) bank198676 +198677 POINT(40.78189404046966 74.56101161944909) bank198677 +198678 POINT(41.13314225180085 73.51971267275658) bank198678 +198679 POINT(39.992421090189644 73.85221708217169) bank198679 +198680 POINT(40.489896190848135 74.1152037243964) bank198680 +198681 POINT(41.29506240926091 74.08021070584527) bank198681 +198682 POINT(40.99187588097911 73.54180873180941) bank198682 +198683 POINT(40.63016398667896 74.27390270522204) bank198683 +198684 POINT(40.37944870605522 73.78258172046623) bank198684 +198685 POINT(39.71430950299195 74.46200501740596) bank198685 +198686 POINT(40.37902567390508 74.15071961457144) bank198686 +198687 POINT(41.00189632543505 73.05289323792597) bank198687 +198688 POINT(40.414085653695416 73.20360986319318) bank198688 +198689 POINT(40.529478600371796 73.71058549875856) bank198689 +198690 POINT(40.23453886546799 74.15734252110387) bank198690 +198691 POINT(40.59235502191141 74.51457168523154) bank198691 +198692 POINT(40.23612250941282 73.7821745480085) bank198692 +198693 POINT(40.285439246282394 74.38470010922202) bank198693 +198694 POINT(40.92617962944629 73.59493986967856) bank198694 +198695 POINT(40.130689063309966 74.98567597879828) bank198695 +198696 POINT(40.079993852393436 73.69444986084154) bank198696 +198697 POINT(40.88116807996767 74.64727007278407) bank198697 +198698 POINT(40.05238684154102 74.72301997391091) bank198698 +198699 POINT(41.43280751015241 73.33405970745407) bank198699 +198700 POINT(40.089725388799785 73.32011660734416) bank198700 +198701 POINT(40.91459031917897 73.06055731272865) bank198701 +198702 POINT(41.24513133586794 73.05330586569904) bank198702 +198703 POINT(41.56772647500282 74.87881169394512) bank198703 +198704 POINT(41.43760159390299 74.80131481124951) bank198704 +198705 POINT(41.3460349369565 73.68566609796694) bank198705 +198706 POINT(40.10556085703256 74.91331991012008) bank198706 +198707 POINT(40.3532020896405 74.92996429296774) bank198707 +198708 POINT(40.80879718302443 74.03530608995761) bank198708 +198709 POINT(40.31308517406183 73.40574246543036) bank198709 +198710 POINT(39.74824476060938 73.36482741398173) bank198710 +198711 POINT(41.49837633248779 73.02998822918134) bank198711 +198712 POINT(41.35759696189682 73.05828515124202) bank198712 +198713 POINT(41.3906883421026 74.46684403514242) bank198713 +198714 POINT(41.19275411349637 74.97558114204973) bank198714 +198715 POINT(40.747968338931635 73.61180594437879) bank198715 +198716 POINT(41.282688599191395 74.38157665545792) bank198716 +198717 POINT(41.61497309380329 74.05969070866476) bank198717 +198718 POINT(41.246629078108136 73.34989493603642) bank198718 +198719 POINT(40.05293825175924 74.18616395057855) bank198719 +198720 POINT(40.797514299846775 73.39775440857633) bank198720 +198721 POINT(41.55270201325817 73.9733477740534) bank198721 +198722 POINT(40.78504418009962 74.88444868349029) bank198722 +198723 POINT(40.031445320926125 73.8723785936783) bank198723 +198724 POINT(41.318230791178365 73.21657070797015) bank198724 +198725 POINT(40.701829767109544 74.11544430679672) bank198725 +198726 POINT(39.770892351919926 73.29952055470557) bank198726 +198727 POINT(40.40608830229002 74.95904021573864) bank198727 +198728 POINT(40.536116137212304 75.0050273694598) bank198728 +198729 POINT(39.970905082515266 73.6476568317885) bank198729 +198730 POINT(40.42193364564558 73.18405696380866) bank198730 +198731 POINT(40.84223593797472 74.15796108025683) bank198731 +198732 POINT(40.18628271943985 74.50003006451826) bank198732 +198733 POINT(39.721586249826466 74.28926225910394) bank198733 +198734 POINT(40.55316499039219 73.08450002734116) bank198734 +198735 POINT(41.49372856524766 73.60102606249306) bank198735 +198736 POINT(40.898966735467454 73.3920302849144) bank198736 +198737 POINT(40.41998947459556 74.01974197127754) bank198737 +198738 POINT(40.42354076820546 73.44581715769276) bank198738 +198739 POINT(40.54097825577482 73.36811745037242) bank198739 +198740 POINT(41.21893824919193 73.07084304532263) bank198740 +198741 POINT(40.24052228531212 74.06883143663559) bank198741 +198742 POINT(40.50004410255511 73.94926584779478) bank198742 +198743 POINT(40.5426707252078 74.44543425494946) bank198743 +198744 POINT(40.53727368975781 73.77900557445702) bank198744 +198745 POINT(41.12997622940195 73.50957834299096) bank198745 +198746 POINT(40.29400851631581 73.73668626477395) bank198746 +198747 POINT(40.45865479932916 73.42811236296436) bank198747 +198748 POINT(41.644607250538805 74.08148999617252) bank198748 +198749 POINT(41.4553365490951 73.40263527026609) bank198749 +198750 POINT(40.17053345664068 73.54554039862266) bank198750 +198751 POINT(40.00558674470096 73.50517159022279) bank198751 +198752 POINT(40.64020897589401 74.09439779752465) bank198752 +198753 POINT(41.1340955957195 74.03988152925007) bank198753 +198754 POINT(40.053347928434896 73.55574112290302) bank198754 +198755 POINT(41.41818754189964 74.35881456613497) bank198755 +198756 POINT(39.83181864486724 74.55804024496246) bank198756 +198757 POINT(40.7605698378793 74.21605022240777) bank198757 +198758 POINT(41.54996135284908 73.26865187842415) bank198758 +198759 POINT(40.62230009863426 74.03854637906377) bank198759 +198760 POINT(41.19622166173014 73.30806935338661) bank198760 +198761 POINT(41.55782396482459 73.35543715515753) bank198761 +198762 POINT(40.08911652399324 73.69644978824114) bank198762 +198763 POINT(41.59807895578217 73.88657032719935) bank198763 +198764 POINT(41.3053673189633 74.30064826613074) bank198764 +198765 POINT(40.7194141504592 74.97746159673841) bank198765 +198766 POINT(41.457482507467624 73.08698257314374) bank198766 +198767 POINT(41.68985898704056 74.32306081079544) bank198767 +198768 POINT(40.27562335636798 73.50222421548712) bank198768 +198769 POINT(41.19311600013961 73.61265464703584) bank198769 +198770 POINT(40.17349664620384 74.0334369602469) bank198770 +198771 POINT(41.146204086379534 73.75590825752185) bank198771 +198772 POINT(41.15385698873365 73.85984651276458) bank198772 +198773 POINT(40.917139079102505 73.91080931856054) bank198773 +198774 POINT(40.02683475153125 74.31142698225858) bank198774 +198775 POINT(40.843759455315606 73.24289187905269) bank198775 +198776 POINT(40.662256689204355 74.86730240848902) bank198776 +198777 POINT(39.73707866343196 74.33723195430159) bank198777 +198778 POINT(40.00029687156701 74.21264455518751) bank198778 +198779 POINT(40.022893067427276 74.92907348782276) bank198779 +198780 POINT(39.87354654460343 74.60637851216362) bank198780 +198781 POINT(40.87113984901251 74.31578711695796) bank198781 +198782 POINT(40.35388035118995 73.65652543828658) bank198782 +198783 POINT(40.76106024170237 73.21202213204607) bank198783 +198784 POINT(41.335917420482296 74.98273326013015) bank198784 +198785 POINT(41.369509225365576 74.85907493611515) bank198785 +198786 POINT(40.90098226271016 73.97413221392439) bank198786 +198787 POINT(40.32356877020541 74.15532474088688) bank198787 +198788 POINT(40.7346777207089 74.72782850364861) bank198788 +198789 POINT(41.47435513550567 74.62914266594646) bank198789 +198790 POINT(40.252704299013594 74.34764985545368) bank198790 +198791 POINT(41.692459531492176 74.03853291293879) bank198791 +198792 POINT(40.362459530360304 73.0838496768999) bank198792 +198793 POINT(40.71100868291206 73.07803894816313) bank198793 +198794 POINT(41.23852250212019 73.73631881253013) bank198794 +198795 POINT(40.55738016476872 74.11842597143979) bank198795 +198796 POINT(40.92896465651502 74.6046545569089) bank198796 +198797 POINT(40.06564088303759 73.29088080464054) bank198797 +198798 POINT(41.33725509803065 74.56595256496372) bank198798 +198799 POINT(40.95541170772102 73.41290978845218) bank198799 +198800 POINT(40.30926379881116 74.14573018977957) bank198800 +198801 POINT(40.19228568164507 73.76402301802867) bank198801 +198802 POINT(40.96196977334656 73.59001867310114) bank198802 +198803 POINT(41.02907935171914 74.70480706662002) bank198803 +198804 POINT(41.387013946430095 74.80115957367484) bank198804 +198805 POINT(41.378005469026384 74.61757717413921) bank198805 +198806 POINT(40.710824144237286 73.90837436109157) bank198806 +198807 POINT(39.74622307766502 73.57970345937224) bank198807 +198808 POINT(41.58462599778378 73.98236098563758) bank198808 +198809 POINT(39.819930094404484 73.03333097742113) bank198809 +198810 POINT(40.65269522292115 73.21172788681766) bank198810 +198811 POINT(41.10042585475454 73.81747250551979) bank198811 +198812 POINT(39.933098087644254 73.02878793632382) bank198812 +198813 POINT(40.44325017331876 73.140138385506) bank198813 +198814 POINT(40.81291681382532 73.731459743277) bank198814 +198815 POINT(39.71544872636641 74.3828572158512) bank198815 +198816 POINT(40.88401332202539 74.14480627609369) bank198816 +198817 POINT(40.12745699078317 74.87045211408035) bank198817 +198818 POINT(41.0939813910819 73.09845736693914) bank198818 +198819 POINT(41.34835365752932 74.82223349220831) bank198819 +198820 POINT(40.270146427021885 74.01051339745258) bank198820 +198821 POINT(40.65418580926087 74.02127225126208) bank198821 +198822 POINT(41.39926311014955 73.45812103871239) bank198822 +198823 POINT(41.27816055088289 73.45194485611891) bank198823 +198824 POINT(40.63468236491724 73.01493887535428) bank198824 +198825 POINT(41.230949622160864 74.66631462874423) bank198825 +198826 POINT(41.439793600277284 74.6101422016602) bank198826 +198827 POINT(41.67629048248899 73.89128449667183) bank198827 +198828 POINT(41.03589612849457 73.46309621250062) bank198828 +198829 POINT(41.084652360757744 73.23686743746046) bank198829 +198830 POINT(40.104181072812686 74.02762133091316) bank198830 +198831 POINT(39.851120667770346 74.12337878801567) bank198831 +198832 POINT(40.77046854303982 74.21993772234117) bank198832 +198833 POINT(39.969015871881716 74.75071082461406) bank198833 +198834 POINT(41.035078108491504 74.99534604903171) bank198834 +198835 POINT(40.67811570481055 73.45922270820142) bank198835 +198836 POINT(41.217882457871895 74.94967829254001) bank198836 +198837 POINT(39.96763623858127 73.45191614624963) bank198837 +198838 POINT(41.45607664576061 74.49665469182443) bank198838 +198839 POINT(40.19373874608569 74.23556232806617) bank198839 +198840 POINT(40.95099578551936 73.10513565998066) bank198840 +198841 POINT(41.42349674812611 73.05137197038096) bank198841 +198842 POINT(40.50349836721591 74.99546600958853) bank198842 +198843 POINT(40.411203735750846 73.19304745525119) bank198843 +198844 POINT(41.54977139242351 73.69520672704267) bank198844 +198845 POINT(40.91314265508426 73.24061788065411) bank198845 +198846 POINT(40.373426578004874 74.45080038607838) bank198846 +198847 POINT(41.34008461960263 74.72856456326976) bank198847 +198848 POINT(40.71778529971882 74.40320576073184) bank198848 +198849 POINT(40.862305557738104 74.83978025466604) bank198849 +198850 POINT(41.56264701073136 74.15846022177107) bank198850 +198851 POINT(41.1241649976312 73.11261156633793) bank198851 +198852 POINT(40.28855451820604 74.63845546882104) bank198852 +198853 POINT(39.79378976506463 73.35405650899581) bank198853 +198854 POINT(39.75817865581506 73.83418068004119) bank198854 +198855 POINT(40.95288814712471 73.38626073067455) bank198855 +198856 POINT(40.088688788750346 73.23220261052262) bank198856 +198857 POINT(41.299582898634654 73.3720082637452) bank198857 +198858 POINT(41.50700314164501 73.49830485338279) bank198858 +198859 POINT(40.843183785416834 74.97096192939587) bank198859 +198860 POINT(41.37217758264288 74.38705075249717) bank198860 +198861 POINT(41.28647235851732 74.01267850739582) bank198861 +198862 POINT(39.94588344955839 74.13131098004344) bank198862 +198863 POINT(39.83448844836463 73.35197873170848) bank198863 +198864 POINT(41.577192112282155 73.68601987276672) bank198864 +198865 POINT(39.938204231955964 73.41462100126913) bank198865 +198866 POINT(40.098613657307624 73.3802138525712) bank198866 +198867 POINT(41.466894425925545 73.69557892300188) bank198867 +198868 POINT(41.677758484320904 74.07783464352075) bank198868 +198869 POINT(40.8407093925967 73.27083105681118) bank198869 +198870 POINT(40.607852139221585 74.6805555858244) bank198870 +198871 POINT(41.47809923427251 74.63086706673666) bank198871 +198872 POINT(41.22360707582123 73.42187508446342) bank198872 +198873 POINT(40.107752522521054 73.22867781479727) bank198873 +198874 POINT(40.41074667635415 73.16766657560493) bank198874 +198875 POINT(40.00673211616298 74.74162533957796) bank198875 +198876 POINT(40.37675438672747 73.96250624721927) bank198876 +198877 POINT(41.38623657661797 74.70478178365886) bank198877 +198878 POINT(41.48996095643184 73.27075130141415) bank198878 +198879 POINT(40.07115182307485 74.53680064434747) bank198879 +198880 POINT(40.64428042686383 73.67890036198949) bank198880 +198881 POINT(40.378056246698634 73.4210910513462) bank198881 +198882 POINT(41.687031910671884 74.0425166813878) bank198882 +198883 POINT(39.919848762058116 73.46835883248471) bank198883 +198884 POINT(40.02860692169623 74.09916298707063) bank198884 +198885 POINT(41.37968998592038 73.15550710012396) bank198885 +198886 POINT(40.6434169568424 74.83657050730393) bank198886 +198887 POINT(40.12686823796443 73.60232001004107) bank198887 +198888 POINT(41.059360726689434 74.19198928467178) bank198888 +198889 POINT(39.964704978136965 74.86204267568672) bank198889 +198890 POINT(39.86284646217064 73.20230241448961) bank198890 +198891 POINT(40.14189576890923 74.0645712007304) bank198891 +198892 POINT(40.78796046318198 73.6425267134455) bank198892 +198893 POINT(40.84153394407173 73.00669086260989) bank198893 +198894 POINT(41.68491427014538 73.36669347108621) bank198894 +198895 POINT(41.03921292051604 73.12952386313422) bank198895 +198896 POINT(41.49415425596221 74.46614338237039) bank198896 +198897 POINT(39.89272915005516 74.43778457067448) bank198897 +198898 POINT(40.4815749123826 74.9128193444453) bank198898 +198899 POINT(40.165338278074614 74.7345732865045) bank198899 +198900 POINT(41.21882457104153 73.75511656992987) bank198900 +198901 POINT(40.96467652284721 73.99026478676113) bank198901 +198902 POINT(39.78492496902045 73.8586023280684) bank198902 +198903 POINT(39.842120914357544 74.74364017497575) bank198903 +198904 POINT(40.26008772013284 73.2145040287847) bank198904 +198905 POINT(41.14068012641639 73.46341386791347) bank198905 +198906 POINT(40.20202069447377 74.27258027822393) bank198906 +198907 POINT(40.60112234334361 73.10790826884795) bank198907 +198908 POINT(40.266775350968764 74.0028173454936) bank198908 +198909 POINT(41.03203975799668 73.29675214800442) bank198909 +198910 POINT(41.49720258013741 74.40723278072556) bank198910 +198911 POINT(41.42546534905848 74.83125422825042) bank198911 +198912 POINT(40.755999671069695 74.46702986350346) bank198912 +198913 POINT(40.25306929837584 73.78265027023647) bank198913 +198914 POINT(41.49910951090723 73.2411221039547) bank198914 +198915 POINT(41.5695233447246 74.76937571011922) bank198915 +198916 POINT(40.00462721334998 74.06049912534009) bank198916 +198917 POINT(41.50331674885032 74.72225654195829) bank198917 +198918 POINT(39.93570389758707 74.67928955419426) bank198918 +198919 POINT(41.51904124075396 73.53130514050021) bank198919 +198920 POINT(41.494248915614385 74.31085453020302) bank198920 +198921 POINT(40.96686043095431 74.37148668958702) bank198921 +198922 POINT(39.828095352323764 73.11365539312253) bank198922 +198923 POINT(40.68513784992056 73.41639757335119) bank198923 +198924 POINT(39.96162052647775 73.77556070388856) bank198924 +198925 POINT(40.416022214172926 73.50427497798543) bank198925 +198926 POINT(39.81586681027354 73.45194399631332) bank198926 +198927 POINT(40.775443911425775 74.47206512220733) bank198927 +198928 POINT(41.17914176952693 73.4592652005546) bank198928 +198929 POINT(40.99834092725953 74.14106755999929) bank198929 +198930 POINT(41.15992301580295 74.4449668936695) bank198930 +198931 POINT(40.53945098830435 74.91467586770224) bank198931 +198932 POINT(41.62068740170522 73.07745411459882) bank198932 +198933 POINT(40.02841107810366 74.54415264121523) bank198933 +198934 POINT(40.950477977225994 74.49489886447171) bank198934 +198935 POINT(40.83162848279223 74.74847657747127) bank198935 +198936 POINT(40.74893699128184 74.99071024228739) bank198936 +198937 POINT(40.83948914193599 74.36088588041405) bank198937 +198938 POINT(41.54246939227922 73.96761697632482) bank198938 +198939 POINT(40.74819924396156 73.44236711232709) bank198939 +198940 POINT(41.622642376599934 74.04200792435407) bank198940 +198941 POINT(40.99315642540549 74.86966965798221) bank198941 +198942 POINT(41.01936162663504 73.03628209609045) bank198942 +198943 POINT(40.7472557412215 73.03491711051095) bank198943 +198944 POINT(41.501729880132686 73.24698872675177) bank198944 +198945 POINT(41.64169744550849 74.24276935760854) bank198945 +198946 POINT(40.334133103030865 73.47148301373738) bank198946 +198947 POINT(40.155240028711994 74.14896453674793) bank198947 +198948 POINT(39.84952937506593 74.71083042901205) bank198948 +198949 POINT(39.81681913024867 74.50036332089691) bank198949 +198950 POINT(41.57652310774357 73.8280363206543) bank198950 +198951 POINT(40.034233372200966 73.32499994912153) bank198951 +198952 POINT(41.01962347016127 74.30832026054377) bank198952 +198953 POINT(40.195121521992746 74.93321044684234) bank198953 +198954 POINT(41.52278790186883 73.6302785887642) bank198954 +198955 POINT(41.593304803576764 74.63527949192003) bank198955 +198956 POINT(39.72436460546123 74.07982501840847) bank198956 +198957 POINT(40.05507184989915 74.96366297480233) bank198957 +198958 POINT(41.03804729385446 74.95718295591008) bank198958 +198959 POINT(41.16829054046768 74.97601070241518) bank198959 +198960 POINT(40.95681605781464 74.04131747428723) bank198960 +198961 POINT(40.86092110729675 73.60388987498757) bank198961 +198962 POINT(40.282303877668056 73.0342949417236) bank198962 +198963 POINT(40.032393570509754 74.0225495575786) bank198963 +198964 POINT(41.19994917652391 73.48193455742272) bank198964 +198965 POINT(41.67220506965136 73.470504544223) bank198965 +198966 POINT(40.35342734424194 73.16967022505455) bank198966 +198967 POINT(41.21496539364923 74.85954275712263) bank198967 +198968 POINT(39.76048678915709 73.1091067676223) bank198968 +198969 POINT(40.188054441452024 74.7968759657354) bank198969 +198970 POINT(39.8536221688886 73.42931874750217) bank198970 +198971 POINT(40.32686070830538 74.10863190531053) bank198971 +198972 POINT(41.5416399496723 73.14301531172426) bank198972 +198973 POINT(39.74426117704298 73.42254099609848) bank198973 +198974 POINT(41.103510231410745 74.95924701402883) bank198974 +198975 POINT(40.728336121552104 74.26623509042325) bank198975 +198976 POINT(41.70573766483735 73.39539914015755) bank198976 +198977 POINT(40.468109485559694 73.0107386465271) bank198977 +198978 POINT(41.38997182375715 74.08541275841013) bank198978 +198979 POINT(41.41065401967374 73.91230477282396) bank198979 +198980 POINT(41.497746221354326 73.43992803542925) bank198980 +198981 POINT(39.9363330309579 74.59852617252159) bank198981 +198982 POINT(40.580147360737215 74.67530138700718) bank198982 +198983 POINT(40.66111331893921 73.86334163822893) bank198983 +198984 POINT(40.58831600299637 73.19646881442831) bank198984 +198985 POINT(41.69006135839332 73.81688783171774) bank198985 +198986 POINT(40.73386951702192 74.15279921629475) bank198986 +198987 POINT(40.71148481323375 74.52737785229291) bank198987 +198988 POINT(41.21334379723589 73.89162208650033) bank198988 +198989 POINT(40.335036116557134 73.54540709484755) bank198989 +198990 POINT(39.865021923312035 74.80895890551113) bank198990 +198991 POINT(40.66415301795652 74.30898265011436) bank198991 +198992 POINT(40.52169237356022 74.22939468973055) bank198992 +198993 POINT(40.061847347542766 74.84862967056594) bank198993 +198994 POINT(41.57580623739594 74.70237799126681) bank198994 +198995 POINT(40.951514682228165 73.63508817001211) bank198995 +198996 POINT(40.92092239410733 73.3962062974029) bank198996 +198997 POINT(41.264608808158876 73.3866225522394) bank198997 +198998 POINT(40.887376346694126 73.16336704534189) bank198998 +198999 POINT(39.80915073952746 73.42846553471003) bank198999 +199000 POINT(40.02472969660474 73.31352002454858) bank199000 +199001 POINT(40.67368621261917 74.79142332285028) bank199001 +199002 POINT(40.9539631768286 73.83873909664294) bank199002 +199003 POINT(41.18736761213792 73.51053914200371) bank199003 +199004 POINT(40.026836472215805 74.91129444633069) bank199004 +199005 POINT(39.86846824098622 73.33302572314098) bank199005 +199006 POINT(40.001800881273745 73.45968689933468) bank199006 +199007 POINT(40.420791634137814 74.49498998220157) bank199007 +199008 POINT(40.97349023962083 74.6334859217216) bank199008 +199009 POINT(41.42098701989376 74.41404577033057) bank199009 +199010 POINT(41.059951795440654 74.80720883962807) bank199010 +199011 POINT(41.1137784751976 74.03321017749528) bank199011 +199012 POINT(40.73159466302713 73.53294186146634) bank199012 +199013 POINT(41.113039514994 73.8340121942646) bank199013 +199014 POINT(41.185705054907 74.65956072393175) bank199014 +199015 POINT(41.400310253108984 73.08182637373322) bank199015 +199016 POINT(40.114255543605836 73.18708771000269) bank199016 +199017 POINT(40.357186956457014 73.40220675304292) bank199017 +199018 POINT(40.98017916215996 73.82800955197443) bank199018 +199019 POINT(41.615195480988525 74.81668548826174) bank199019 +199020 POINT(40.443283960829476 73.80870553313876) bank199020 +199021 POINT(40.29957339500983 74.81459745815172) bank199021 +199022 POINT(41.54219499082385 74.73671989500953) bank199022 +199023 POINT(40.64007090054957 74.12675728229227) bank199023 +199024 POINT(41.19731603773086 73.81908578339637) bank199024 +199025 POINT(41.019455937704784 74.2929214466921) bank199025 +199026 POINT(41.05751410877219 73.6739875248904) bank199026 +199027 POINT(41.696861746052896 74.89665755935445) bank199027 +199028 POINT(41.09341951142678 74.69089156753228) bank199028 +199029 POINT(39.77114056181141 73.15828217107702) bank199029 +199030 POINT(41.33821028056414 73.02730791707849) bank199030 +199031 POINT(39.7183321030206 73.3994451361861) bank199031 +199032 POINT(39.893324200860114 74.03258289224127) bank199032 +199033 POINT(40.20192651883737 73.0442659588265) bank199033 +199034 POINT(40.03236664835367 73.74850466730969) bank199034 +199035 POINT(41.12727482725842 74.63459274134128) bank199035 +199036 POINT(41.01322708688552 74.82160622301652) bank199036 +199037 POINT(41.21268696298775 74.5627874988666) bank199037 +199038 POINT(40.968265889009004 73.43725453689926) bank199038 +199039 POINT(40.307395366050606 73.46621560062157) bank199039 +199040 POINT(40.05378550720372 74.39197449713693) bank199040 +199041 POINT(40.33586484194715 74.35356336737003) bank199041 +199042 POINT(39.84521491995202 73.27370739589851) bank199042 +199043 POINT(40.07371681299513 74.31519715498196) bank199043 +199044 POINT(40.21916278604466 74.79272744408591) bank199044 +199045 POINT(41.58202757734461 73.58570034615092) bank199045 +199046 POINT(41.680744993114025 74.51682117247938) bank199046 +199047 POINT(39.86729372358799 73.51827832777707) bank199047 +199048 POINT(41.63549623691466 74.14211087145178) bank199048 +199049 POINT(40.99197453674261 74.10333941271436) bank199049 +199050 POINT(41.572795428673885 73.26293716402252) bank199050 +199051 POINT(40.382186728249245 74.83099027914152) bank199051 +199052 POINT(40.80841406545871 73.97849750156675) bank199052 +199053 POINT(41.254391964035214 73.01497487348844) bank199053 +199054 POINT(41.14294713639242 73.10567983185601) bank199054 +199055 POINT(40.4537674875834 74.21874516223984) bank199055 +199056 POINT(40.298932908342515 74.60181099898301) bank199056 +199057 POINT(39.780636162734424 74.83043112242552) bank199057 +199058 POINT(40.19773916034285 74.6460164497023) bank199058 +199059 POINT(41.338929530585744 74.49802639119108) bank199059 +199060 POINT(41.17344446331624 74.43534211574374) bank199060 +199061 POINT(40.01505024090114 73.93838352978797) bank199061 +199062 POINT(40.00977901579128 73.88588158939328) bank199062 +199063 POINT(40.89963586523554 73.22653036707224) bank199063 +199064 POINT(40.44598570914214 73.61812603534393) bank199064 +199065 POINT(39.926413442723955 73.45740284676415) bank199065 +199066 POINT(41.19164557027601 73.85848413040975) bank199066 +199067 POINT(41.23517755130926 73.82797594924597) bank199067 +199068 POINT(41.02133943764849 73.89140846681418) bank199068 +199069 POINT(40.25714137087686 73.82309087323233) bank199069 +199070 POINT(39.97237508289673 74.17813138031748) bank199070 +199071 POINT(41.19028479957371 73.18682856828337) bank199071 +199072 POINT(40.781780134173104 74.8534631911529) bank199072 +199073 POINT(40.924281819838455 74.66890242840083) bank199073 +199074 POINT(40.66392886371621 73.17714944827725) bank199074 +199075 POINT(41.490560613747896 73.50328504957568) bank199075 +199076 POINT(41.42345537654635 73.70944777144791) bank199076 +199077 POINT(40.91672411387099 74.93810795907488) bank199077 +199078 POINT(41.48389227770407 74.24576289475094) bank199078 +199079 POINT(40.59249693993734 74.303897521613) bank199079 +199080 POINT(40.03389711969498 74.06955486580141) bank199080 +199081 POINT(40.083298077468996 73.44445443807774) bank199081 +199082 POINT(40.28760621592895 73.25246991552112) bank199082 +199083 POINT(41.463679249480016 73.55352454494138) bank199083 +199084 POINT(40.40729289486839 74.08852866698581) bank199084 +199085 POINT(39.92237476331464 74.73977966920418) bank199085 +199086 POINT(40.782146653673735 75.00089627524099) bank199086 +199087 POINT(41.04993551708193 74.09309426868556) bank199087 +199088 POINT(39.995484600950455 73.75951781359946) bank199088 +199089 POINT(40.80006609664129 73.20347421487583) bank199089 +199090 POINT(41.5870803104639 74.93137197441843) bank199090 +199091 POINT(40.18441250142705 74.68107399165187) bank199091 +199092 POINT(40.08699448105459 74.07903998291911) bank199092 +199093 POINT(41.515372887494756 74.61754151663698) bank199093 +199094 POINT(41.55236209107585 74.51907728310546) bank199094 +199095 POINT(41.61717517032237 74.96628366101397) bank199095 +199096 POINT(39.8812871535845 74.20624513025602) bank199096 +199097 POINT(41.46918016250052 74.54780925126438) bank199097 +199098 POINT(39.71896997656837 73.99389710402492) bank199098 +199099 POINT(40.49545802240937 74.50442864802062) bank199099 +199100 POINT(40.39782831083849 74.26542886286043) bank199100 +199101 POINT(40.27771079262617 73.63875191328945) bank199101 +199102 POINT(40.88867258290251 73.14414397237095) bank199102 +199103 POINT(41.118361888328614 74.79394649843103) bank199103 +199104 POINT(41.57967939888123 74.60448061057296) bank199104 +199105 POINT(41.310252227179944 73.2438824630074) bank199105 +199106 POINT(40.34954178873577 73.97645344981892) bank199106 +199107 POINT(39.72964776407191 74.24029779312106) bank199107 +199108 POINT(41.47607727595068 73.42886547934717) bank199108 +199109 POINT(41.020217728487964 73.23355392501942) bank199109 +199110 POINT(40.59317815580349 73.50797242322557) bank199110 +199111 POINT(40.39199185012384 74.02448287118962) bank199111 +199112 POINT(41.70777383712375 73.56235257578608) bank199112 +199113 POINT(40.23352079034982 74.14958759914624) bank199113 +199114 POINT(40.243908713755474 73.685302877199) bank199114 +199115 POINT(39.92325296796418 74.53076839243565) bank199115 +199116 POINT(40.1354028649977 73.79444809067644) bank199116 +199117 POINT(40.39539411719816 74.69124608835833) bank199117 +199118 POINT(40.92513906584695 75.00286001760189) bank199118 +199119 POINT(41.68616892006939 74.81998423511301) bank199119 +199120 POINT(39.89596334700253 74.21972483887387) bank199120 +199121 POINT(41.218883863676545 73.50992099339412) bank199121 +199122 POINT(40.68744996738604 73.7097829728728) bank199122 +199123 POINT(40.982333772060706 73.86249867059925) bank199123 +199124 POINT(40.63786634118505 73.29787537563024) bank199124 +199125 POINT(41.480899824916534 73.57044087579227) bank199125 +199126 POINT(39.74587004738399 73.23069596019147) bank199126 +199127 POINT(40.763346801381566 73.47090744721923) bank199127 +199128 POINT(41.47067168119274 73.173196879496) bank199128 +199129 POINT(40.936497411120634 74.05865464874793) bank199129 +199130 POINT(41.17751816161253 73.27411080593936) bank199130 +199131 POINT(40.59696999864205 74.57442708284793) bank199131 +199132 POINT(39.75584180286178 73.33414213770808) bank199132 +199133 POINT(39.791845655335415 74.07455897884257) bank199133 +199134 POINT(41.02970850777394 73.56269720916015) bank199134 +199135 POINT(40.53421466450746 73.82305407438854) bank199135 +199136 POINT(40.22327215006284 73.61572418161838) bank199136 +199137 POINT(41.563879546811 73.23782524807702) bank199137 +199138 POINT(41.37897318460107 74.94049998001589) bank199138 +199139 POINT(40.18035091023291 73.72313684742336) bank199139 +199140 POINT(41.304845967376835 73.92394512918568) bank199140 +199141 POINT(40.04843107479392 73.06601276667416) bank199141 +199142 POINT(41.12514188665904 73.90908330707825) bank199142 +199143 POINT(41.71105475286624 74.9470098438093) bank199143 +199144 POINT(40.662100799937015 74.64768726281451) bank199144 +199145 POINT(40.60397766287071 73.70877226269045) bank199145 +199146 POINT(40.2163732169232 73.40916451898435) bank199146 +199147 POINT(40.70615356925845 73.3854374317386) bank199147 +199148 POINT(41.52994166988347 74.0075368862981) bank199148 +199149 POINT(41.330719941694184 74.19766000194714) bank199149 +199150 POINT(41.60278797566838 74.7774475925582) bank199150 +199151 POINT(40.694145978455595 73.80275192656595) bank199151 +199152 POINT(40.39315947730323 73.15557349361771) bank199152 +199153 POINT(41.002580305247 73.0532735115506) bank199153 +199154 POINT(39.81480574822267 73.7899059894793) bank199154 +199155 POINT(41.10969292049326 73.99194809996337) bank199155 +199156 POINT(40.5336701715683 74.6285766952999) bank199156 +199157 POINT(41.43357155330676 74.00325766294344) bank199157 +199158 POINT(41.11065528948024 73.6403919578375) bank199158 +199159 POINT(41.171320209292055 74.64825212615922) bank199159 +199160 POINT(40.71093401457077 74.1012668218504) bank199160 +199161 POINT(40.9058310756543 73.25939459893745) bank199161 +199162 POINT(40.64296876300144 74.73406842669586) bank199162 +199163 POINT(40.69009998101535 73.9889615327922) bank199163 +199164 POINT(41.224739124717296 73.17239809547262) bank199164 +199165 POINT(40.201278702770516 73.82022561936292) bank199165 +199166 POINT(41.34752011027539 74.95860462694088) bank199166 +199167 POINT(40.26928243501965 74.50877194121752) bank199167 +199168 POINT(41.57225004724231 73.87497113550371) bank199168 +199169 POINT(40.0884954229852 73.79152763699392) bank199169 +199170 POINT(40.32892315326419 74.78995547286515) bank199170 +199171 POINT(40.551200440272524 73.93576061550647) bank199171 +199172 POINT(40.5764537874994 73.98111840983752) bank199172 +199173 POINT(40.75589277857743 74.51327171658156) bank199173 +199174 POINT(41.50537679088315 73.024233061293) bank199174 +199175 POINT(40.739472322975615 73.94744943469665) bank199175 +199176 POINT(40.10505949104016 74.98932823736652) bank199176 +199177 POINT(41.05698008824181 73.4993429509899) bank199177 +199178 POINT(40.33686120974406 74.46909973837052) bank199178 +199179 POINT(40.113139181265566 74.08773845883006) bank199179 +199180 POINT(40.04842529362536 74.36866509871498) bank199180 +199181 POINT(41.132601845928946 73.27863903504465) bank199181 +199182 POINT(39.77033981540346 73.60471453013213) bank199182 +199183 POINT(40.45688189435689 73.03602948125342) bank199183 +199184 POINT(39.90544556815211 73.53878992689724) bank199184 +199185 POINT(41.39079231573121 73.50397787017319) bank199185 +199186 POINT(39.86443582041832 73.63256725948173) bank199186 +199187 POINT(39.91341223222437 74.32120631560932) bank199187 +199188 POINT(41.241696523407455 73.2795692255205) bank199188 +199189 POINT(41.31660634251141 74.57927379066972) bank199189 +199190 POINT(41.65415703391481 74.10957309582602) bank199190 +199191 POINT(41.187196325360446 74.33682840327705) bank199191 +199192 POINT(40.3557175031917 73.43354902954924) bank199192 +199193 POINT(40.08620037330553 73.9565496669685) bank199193 +199194 POINT(39.90615978523349 74.94340529863362) bank199194 +199195 POINT(41.22706899136354 73.6768382302937) bank199195 +199196 POINT(40.201095183498346 74.41099584885339) bank199196 +199197 POINT(41.08597411052158 73.69297807472803) bank199197 +199198 POINT(40.389450604902244 73.98358809994755) bank199198 +199199 POINT(41.65644907889389 73.0217585598392) bank199199 +199200 POINT(41.55456105231629 74.71098117565302) bank199200 +199201 POINT(41.68245917205736 73.40482498501913) bank199201 +199202 POINT(40.23319438485014 73.18574289970529) bank199202 +199203 POINT(40.65848663783414 74.68817496192429) bank199203 +199204 POINT(41.07069334362616 74.5119577943586) bank199204 +199205 POINT(40.33568131524996 74.66820062262897) bank199205 +199206 POINT(39.71344553430308 73.22076535811972) bank199206 +199207 POINT(40.38830637575424 73.67451624553472) bank199207 +199208 POINT(40.62484966182314 73.37898196099934) bank199208 +199209 POINT(41.306221097136785 73.5610679641817) bank199209 +199210 POINT(40.47876013428862 74.70897941550656) bank199210 +199211 POINT(41.26626446791967 73.66430113070498) bank199211 +199212 POINT(41.662944678053265 74.04501235939905) bank199212 +199213 POINT(40.952686029169485 74.324150546277) bank199213 +199214 POINT(41.18233278189668 73.61477991041119) bank199214 +199215 POINT(39.740059428359025 73.91450699254096) bank199215 +199216 POINT(41.24716423122106 73.55935771962072) bank199216 +199217 POINT(41.17455699993968 73.18943839331143) bank199217 +199218 POINT(41.42127406688981 73.41780016916536) bank199218 +199219 POINT(40.202034659404276 74.8497137096511) bank199219 +199220 POINT(41.49760889056064 73.19149869753049) bank199220 +199221 POINT(41.66736462504417 73.73825310421074) bank199221 +199222 POINT(40.249447436684136 74.39535244428676) bank199222 +199223 POINT(39.946743555346124 73.42560984395442) bank199223 +199224 POINT(41.34634504547092 73.36032995915419) bank199224 +199225 POINT(40.8614182824692 74.14720331112767) bank199225 +199226 POINT(41.34904981610877 73.2744092725514) bank199226 +199227 POINT(40.96931891955062 74.76122753312427) bank199227 +199228 POINT(40.68953918134636 73.57449599790122) bank199228 +199229 POINT(40.904917465874426 74.250170269646) bank199229 +199230 POINT(40.81205541725453 74.02558219081757) bank199230 +199231 POINT(40.705264327030314 73.92692792899716) bank199231 +199232 POINT(41.50444967990809 74.88540333383474) bank199232 +199233 POINT(40.59530078258108 74.81814725329589) bank199233 +199234 POINT(41.6454917951293 73.79005895911834) bank199234 +199235 POINT(40.237116614358875 74.50325334136038) bank199235 +199236 POINT(39.925133453521354 74.90777162626942) bank199236 +199237 POINT(40.039688767371594 73.58383552185796) bank199237 +199238 POINT(40.13875786887768 74.30272812902155) bank199238 +199239 POINT(39.74173105028049 74.11919283797445) bank199239 +199240 POINT(39.87691304289746 73.65958851967045) bank199240 +199241 POINT(40.32338713090228 74.91766503896281) bank199241 +199242 POINT(41.1158997854658 74.00825518362794) bank199242 +199243 POINT(40.96233992155971 74.53590885619778) bank199243 +199244 POINT(39.898002227284344 74.55085287223146) bank199244 +199245 POINT(39.865793576492536 74.97568197612676) bank199245 +199246 POINT(39.73074458261839 73.01884840018727) bank199246 +199247 POINT(41.132602937275706 74.2765957158595) bank199247 +199248 POINT(41.38269288736696 74.90239750121576) bank199248 +199249 POINT(41.16315279720062 74.86887522536524) bank199249 +199250 POINT(41.5096927117822 73.70618627115003) bank199250 +199251 POINT(40.2945601107446 74.39740452541196) bank199251 +199252 POINT(39.920142946249875 73.1067018086996) bank199252 +199253 POINT(41.652990707327 73.85352087306933) bank199253 +199254 POINT(39.85900180950818 74.84927967321349) bank199254 +199255 POINT(40.31889703744793 73.87358591095372) bank199255 +199256 POINT(40.974036796895064 73.37414708468683) bank199256 +199257 POINT(40.213027758906776 73.62740844154666) bank199257 +199258 POINT(41.66908775224496 74.75894647433871) bank199258 +199259 POINT(41.685032934880866 74.97005716369205) bank199259 +199260 POINT(41.45026222250909 74.79674412260191) bank199260 +199261 POINT(41.633805590122165 74.85307735228734) bank199261 +199262 POINT(40.66462436063394 73.48983430424376) bank199262 +199263 POINT(40.46148902854099 73.75148927546562) bank199263 +199264 POINT(40.95954336913694 74.54774016750903) bank199264 +199265 POINT(41.55917336059138 73.25299888191692) bank199265 +199266 POINT(40.959748961355054 74.40583538530588) bank199266 +199267 POINT(41.54869832762224 74.21989088982559) bank199267 +199268 POINT(41.4045544561544 74.31740419098189) bank199268 +199269 POINT(40.94744389929538 74.06807308894113) bank199269 +199270 POINT(41.22635996444377 74.47469175361671) bank199270 +199271 POINT(41.63970471236085 74.84387733672972) bank199271 +199272 POINT(40.59167699006816 73.71880251413764) bank199272 +199273 POINT(39.89228463110972 74.6513785099615) bank199273 +199274 POINT(41.48353628603505 74.25338729267742) bank199274 +199275 POINT(40.389852185927026 74.83847570637009) bank199275 +199276 POINT(40.916018642061275 73.60376550039544) bank199276 +199277 POINT(41.31564042513291 73.01408913346536) bank199277 +199278 POINT(41.55636370282488 73.26244239511472) bank199278 +199279 POINT(41.415078698767594 74.02547615846778) bank199279 +199280 POINT(41.404377541205406 73.65737827911586) bank199280 +199281 POINT(41.59592076512032 73.79304443276027) bank199281 +199282 POINT(40.80568028716857 74.91416090994652) bank199282 +199283 POINT(40.92043738207093 74.08621490278645) bank199283 +199284 POINT(41.2390696367945 73.01286937411382) bank199284 +199285 POINT(40.870746002315606 73.074164064285) bank199285 +199286 POINT(41.52319837534341 73.80225418905155) bank199286 +199287 POINT(40.20232186776003 74.52864481913709) bank199287 +199288 POINT(40.14296375172172 74.33437095669896) bank199288 +199289 POINT(41.44642314873243 73.36090954563748) bank199289 +199290 POINT(39.75610019033686 74.85492312437745) bank199290 +199291 POINT(41.03486001386595 74.18408710283394) bank199291 +199292 POINT(40.327045562687445 74.41899658966798) bank199292 +199293 POINT(40.73623980868464 74.30767302933545) bank199293 +199294 POINT(40.904555563744246 73.62213161842459) bank199294 +199295 POINT(40.1461220825974 73.2479055714898) bank199295 +199296 POINT(39.73911927272787 74.03412398842347) bank199296 +199297 POINT(41.503830265722385 73.96895342735421) bank199297 +199298 POINT(40.717446336353056 74.12952792568517) bank199298 +199299 POINT(40.17945528351569 73.91168450199777) bank199299 +199300 POINT(40.379115909802906 73.84534610705833) bank199300 +199301 POINT(40.470011714445484 74.95954864852314) bank199301 +199302 POINT(39.87489509950465 73.02957680745239) bank199302 +199303 POINT(41.39756416530389 73.07954478817102) bank199303 +199304 POINT(41.56280589434594 74.35118242262571) bank199304 +199305 POINT(41.420833216130134 73.79946357538122) bank199305 +199306 POINT(41.11395892496795 73.98453460292492) bank199306 +199307 POINT(40.8145382073459 73.72715779711599) bank199307 +199308 POINT(40.017558064403985 73.1395526038299) bank199308 +199309 POINT(41.22175345773896 73.39170483910534) bank199309 +199310 POINT(41.311571454260616 74.46904746731347) bank199310 +199311 POINT(41.485850432945355 73.52579578467497) bank199311 +199312 POINT(41.475732502080824 74.43222127352553) bank199312 +199313 POINT(41.58604958768259 73.80016423717183) bank199313 +199314 POINT(40.98289659766436 74.73982954515887) bank199314 +199315 POINT(41.3287574003694 73.48414340349264) bank199315 +199316 POINT(40.32909960893273 73.71247417510479) bank199316 +199317 POINT(40.24721872716828 74.3375762768433) bank199317 +199318 POINT(40.40635374787846 74.8032632806993) bank199318 +199319 POINT(40.72217513507138 73.05890207084376) bank199319 +199320 POINT(40.09283443737029 73.45737378357545) bank199320 +199321 POINT(40.495219785831125 73.309879916487) bank199321 +199322 POINT(41.31186721411626 74.84711766600714) bank199322 +199323 POINT(39.89943048440079 74.78284004517465) bank199323 +199324 POINT(41.64873893861484 74.500742104425) bank199324 +199325 POINT(40.8871248748742 73.72592983771786) bank199325 +199326 POINT(40.05400404133675 74.36301025438652) bank199326 +199327 POINT(41.57309422001527 73.6465302717508) bank199327 +199328 POINT(41.68941867617982 74.65806259688323) bank199328 +199329 POINT(40.19072281550315 73.63034026077119) bank199329 +199330 POINT(41.605458296162745 74.69152806156507) bank199330 +199331 POINT(41.11163326656104 74.13571657556608) bank199331 +199332 POINT(41.402197723315425 74.00558551875733) bank199332 +199333 POINT(40.32637668650002 73.14800728660894) bank199333 +199334 POINT(39.77854801059725 74.56491775399574) bank199334 +199335 POINT(41.699942057107414 73.68967540463854) bank199335 +199336 POINT(40.086822033329064 73.3423701588713) bank199336 +199337 POINT(40.1087646285732 74.50895764021047) bank199337 +199338 POINT(41.6619417484286 74.39724654293808) bank199338 +199339 POINT(40.61029452160503 73.85241282386669) bank199339 +199340 POINT(40.23101278509638 74.69872247508138) bank199340 +199341 POINT(41.1918626173187 73.12028942999063) bank199341 +199342 POINT(40.33625755952525 73.20677328036828) bank199342 +199343 POINT(40.6695279606829 74.01494665949672) bank199343 +199344 POINT(40.55810079390293 74.05997835290911) bank199344 +199345 POINT(41.606549703084966 74.93024686599632) bank199345 +199346 POINT(40.61380624073992 73.14193239214337) bank199346 +199347 POINT(41.393320302697084 74.02836773234198) bank199347 +199348 POINT(41.591089066474844 74.15451300969744) bank199348 +199349 POINT(40.383367857932406 73.39498389701998) bank199349 +199350 POINT(40.12932383789739 74.18500635349731) bank199350 +199351 POINT(40.321300912627464 73.80653915308932) bank199351 +199352 POINT(40.18213444877568 74.65614650782095) bank199352 +199353 POINT(39.76718487413928 73.32230730329557) bank199353 +199354 POINT(40.1777818090966 74.93611251086065) bank199354 +199355 POINT(39.96415528205991 73.58548168525253) bank199355 +199356 POINT(41.47631853505236 74.67191699013172) bank199356 +199357 POINT(40.23902502569078 73.06279475079606) bank199357 +199358 POINT(40.75184804656859 73.57378127372318) bank199358 +199359 POINT(41.46169497132906 74.7452596794219) bank199359 +199360 POINT(40.9491043694427 74.42622637140201) bank199360 +199361 POINT(41.124005052601504 74.70208569972796) bank199361 +199362 POINT(40.6418325275984 73.29676646787868) bank199362 +199363 POINT(41.40553263145263 73.29535216059202) bank199363 +199364 POINT(40.8136135681341 74.28277520846076) bank199364 +199365 POINT(40.70049813189795 74.57307813344596) bank199365 +199366 POINT(41.113104770709164 74.09964786850733) bank199366 +199367 POINT(40.51556014414041 73.28488896447121) bank199367 +199368 POINT(41.410011258093945 73.02173675597922) bank199368 +199369 POINT(41.14750088563119 74.479600229262) bank199369 +199370 POINT(39.85475774112254 73.29138173126456) bank199370 +199371 POINT(40.417396346942255 73.53926639662188) bank199371 +199372 POINT(41.537373967549364 73.18812986209176) bank199372 +199373 POINT(41.067803084763376 74.30611512678328) bank199373 +199374 POINT(41.23183367444686 74.06203199437681) bank199374 +199375 POINT(40.572339672171815 74.47727785859605) bank199375 +199376 POINT(40.110797307435526 73.24384389246828) bank199376 +199377 POINT(40.225771157663964 74.21318816536493) bank199377 +199378 POINT(40.52355384274569 73.19427857144228) bank199378 +199379 POINT(41.105271231089745 73.29089453931456) bank199379 +199380 POINT(41.36099665452504 74.12566851165379) bank199380 +199381 POINT(39.98663951714761 74.26421157822) bank199381 +199382 POINT(40.28311861330099 74.36608807714748) bank199382 +199383 POINT(41.157451615606874 74.60891414815022) bank199383 +199384 POINT(40.16539908611549 74.09956424335022) bank199384 +199385 POINT(41.609587881322085 74.70023986131937) bank199385 +199386 POINT(39.83679362312721 73.62300166827258) bank199386 +199387 POINT(40.46311810428481 74.55485311406949) bank199387 +199388 POINT(41.53152136431982 73.83597726270204) bank199388 +199389 POINT(41.234635568309855 74.87015473833867) bank199389 +199390 POINT(40.908319912878696 73.24321910220279) bank199390 +199391 POINT(40.65070389811391 74.14448877477645) bank199391 +199392 POINT(40.004452680470386 73.83020002618775) bank199392 +199393 POINT(41.133931202538115 73.73398012177216) bank199393 +199394 POINT(40.961683890798994 74.55382703190224) bank199394 +199395 POINT(40.90076466563396 74.4404118140105) bank199395 +199396 POINT(40.708242655542904 74.5220709859693) bank199396 +199397 POINT(40.384484980614786 74.76948913630235) bank199397 +199398 POINT(40.82413037453729 74.3487617388696) bank199398 +199399 POINT(40.300331349657256 74.25621336376089) bank199399 +199400 POINT(41.62564428282466 74.08648789140999) bank199400 +199401 POINT(41.255988289894795 74.00238498890211) bank199401 +199402 POINT(41.197171188903226 73.69714084140182) bank199402 +199403 POINT(41.557714184212 74.36483395727261) bank199403 +199404 POINT(40.518271471698824 73.52052785709965) bank199404 +199405 POINT(40.71284241954582 74.65396458980031) bank199405 +199406 POINT(41.706024484855824 74.61063697638983) bank199406 +199407 POINT(40.3169464091177 73.53132602397731) bank199407 +199408 POINT(40.34186183283934 74.52599293600565) bank199408 +199409 POINT(40.47745808055431 74.70743970646971) bank199409 +199410 POINT(40.861407644670045 73.20569276677207) bank199410 +199411 POINT(39.868514061294434 74.24360383896772) bank199411 +199412 POINT(41.47591292954344 73.46975108220954) bank199412 +199413 POINT(40.474850738266476 74.07707782897398) bank199413 +199414 POINT(41.13543605015806 74.96614951647837) bank199414 +199415 POINT(40.13999463173161 73.65163679694022) bank199415 +199416 POINT(40.94023002744274 74.65077853191983) bank199416 +199417 POINT(41.6663438712543 73.65496248095276) bank199417 +199418 POINT(39.891629615917 74.47616209259333) bank199418 +199419 POINT(40.39441712236107 73.50637403173516) bank199419 +199420 POINT(41.467725515438886 74.76690284169094) bank199420 +199421 POINT(40.656372915936366 74.31514155743095) bank199421 +199422 POINT(41.67909380251901 74.01778036020787) bank199422 +199423 POINT(40.20905831409755 73.17418630293267) bank199423 +199424 POINT(41.24261715456196 74.43665604627563) bank199424 +199425 POINT(39.98548197360782 73.87878612907883) bank199425 +199426 POINT(41.523757670781734 74.84904761432315) bank199426 +199427 POINT(40.14192639177879 74.22935507911217) bank199427 +199428 POINT(40.22096878541323 73.66129062495776) bank199428 +199429 POINT(41.379158629171 74.00888382636681) bank199429 +199430 POINT(41.30943735918347 73.73169472389037) bank199430 +199431 POINT(40.84130708696251 74.22605484185122) bank199431 +199432 POINT(41.45870126846981 74.94766420537563) bank199432 +199433 POINT(41.2461177596679 74.1719012815386) bank199433 +199434 POINT(41.46563237846894 74.55177350836874) bank199434 +199435 POINT(40.5468984566575 73.56268371260357) bank199435 +199436 POINT(41.401860586893925 73.93914700086798) bank199436 +199437 POINT(41.06297407583646 74.57197121306568) bank199437 +199438 POINT(41.1813762370422 74.8905780139191) bank199438 +199439 POINT(39.83760547143831 74.94800924779564) bank199439 +199440 POINT(40.800475304508566 73.3808232651048) bank199440 +199441 POINT(40.54785178223969 73.24310089087653) bank199441 +199442 POINT(40.94385683646836 74.77767252169339) bank199442 +199443 POINT(40.523977167408404 74.29735039665576) bank199443 +199444 POINT(40.407776144523574 74.67704154102877) bank199444 +199445 POINT(41.24666666881724 73.13386334291027) bank199445 +199446 POINT(41.24255226782941 74.35046143023337) bank199446 +199447 POINT(40.49662341728781 74.2705300394693) bank199447 +199448 POINT(40.98433213651647 73.28705182040312) bank199448 +199449 POINT(41.42552344006553 74.50161549196143) bank199449 +199450 POINT(40.07205450412002 73.8713823146344) bank199450 +199451 POINT(40.74814345930862 74.73638108219632) bank199451 +199452 POINT(41.57187772163721 73.48659498560295) bank199452 +199453 POINT(40.154161521818274 73.85435459314718) bank199453 +199454 POINT(40.16269375769316 73.9238988799787) bank199454 +199455 POINT(41.10543630214619 73.66264704035486) bank199455 +199456 POINT(41.1880346728921 73.93558363130762) bank199456 +199457 POINT(39.94399497744046 74.32037446665524) bank199457 +199458 POINT(39.84427020426646 74.57659469867102) bank199458 +199459 POINT(39.845173199854926 74.373754451765) bank199459 +199460 POINT(40.63988567980079 73.49331202431974) bank199460 +199461 POINT(40.93558428030692 73.13187361119175) bank199461 +199462 POINT(40.37569405061363 73.08563869673998) bank199462 +199463 POINT(40.83490735232663 73.08159896573532) bank199463 +199464 POINT(40.79765395641083 74.43218733330318) bank199464 +199465 POINT(39.782406687843064 73.2187244740134) bank199465 +199466 POINT(40.54945917065109 74.42510993823296) bank199466 +199467 POINT(41.191555086373775 74.28638584091426) bank199467 +199468 POINT(39.98722004869888 74.62592853440772) bank199468 +199469 POINT(40.459570227621434 73.68609000439527) bank199469 +199470 POINT(40.025110750348055 74.03436970720979) bank199470 +199471 POINT(40.86546094118874 73.03089737326526) bank199471 +199472 POINT(41.03514334089799 74.13919522522315) bank199472 +199473 POINT(40.00974052608829 74.01634116750013) bank199473 +199474 POINT(41.23233604139674 74.02712038536562) bank199474 +199475 POINT(40.7999344443355 74.05197867756931) bank199475 +199476 POINT(40.14418857868199 73.51014313782493) bank199476 +199477 POINT(41.59262823739102 73.67266041658107) bank199477 +199478 POINT(41.69537387409262 74.1397880469595) bank199478 +199479 POINT(41.54772155036731 73.68210271137035) bank199479 +199480 POINT(41.57914787032519 74.5457981787428) bank199480 +199481 POINT(40.62533461810783 73.16246736633933) bank199481 +199482 POINT(39.75565703940629 74.41098222323441) bank199482 +199483 POINT(40.78328009973364 74.89047575789435) bank199483 +199484 POINT(41.467483974030074 74.99285551326619) bank199484 +199485 POINT(40.58185794518352 74.13636971130448) bank199485 +199486 POINT(41.11992447467141 73.26201802584414) bank199486 +199487 POINT(41.28617676460142 73.76082602551249) bank199487 +199488 POINT(41.64598026955061 73.68097014562332) bank199488 +199489 POINT(41.14105714319758 73.64839681913043) bank199489 +199490 POINT(40.74957185261084 73.44569145065256) bank199490 +199491 POINT(40.901417915360256 74.21509481197103) bank199491 +199492 POINT(40.655288240624515 74.60747727596444) bank199492 +199493 POINT(40.59381373809281 73.31963618226531) bank199493 +199494 POINT(39.74318310081592 73.00713403861575) bank199494 +199495 POINT(39.82287632229657 74.88870340142064) bank199495 +199496 POINT(40.80424608672388 73.61654939653636) bank199496 +199497 POINT(40.143839713327274 74.45858508425499) bank199497 +199498 POINT(40.61808302357239 73.25899640146775) bank199498 +199499 POINT(41.36082411094175 73.07911059532576) bank199499 +199500 POINT(40.52331520653666 74.32514955317173) bank199500 +199501 POINT(41.399768533320874 74.15270095184002) bank199501 +199502 POINT(40.673404071562025 74.95464884833355) bank199502 +199503 POINT(41.04316713713768 73.96017486458342) bank199503 +199504 POINT(40.69337781465231 73.91061269614968) bank199504 +199505 POINT(40.509729322447306 73.61766306488737) bank199505 +199506 POINT(40.95308505128895 74.70808336907642) bank199506 +199507 POINT(39.83832379623773 74.66410307687042) bank199507 +199508 POINT(40.41486130875367 73.45884342538764) bank199508 +199509 POINT(40.828183715580735 74.43235532109969) bank199509 +199510 POINT(41.3203333451624 73.57986068906567) bank199510 +199511 POINT(41.31476804304603 74.87856372249999) bank199511 +199512 POINT(41.349814282547634 74.49071497094293) bank199512 +199513 POINT(40.54903409251493 73.03781305934925) bank199513 +199514 POINT(39.79702781751758 73.79659627642835) bank199514 +199515 POINT(40.13521297385794 73.9316091641552) bank199515 +199516 POINT(39.85767555371921 73.87623659165436) bank199516 +199517 POINT(41.62266560280438 74.87269412345772) bank199517 +199518 POINT(40.42025446116862 73.93194140579577) bank199518 +199519 POINT(41.48706536282563 74.63087005657574) bank199519 +199520 POINT(41.0386000256807 74.01773720531764) bank199520 +199521 POINT(40.36188282306276 74.9995260372145) bank199521 +199522 POINT(41.01070140296098 74.45223658920897) bank199522 +199523 POINT(40.57345473324179 73.09098257585977) bank199523 +199524 POINT(40.24105492744005 73.82812228238251) bank199524 +199525 POINT(40.480856302596045 74.10210926647694) bank199525 +199526 POINT(40.08652642887968 74.38689241475444) bank199526 +199527 POINT(40.54024088444133 74.01947062225048) bank199527 +199528 POINT(40.97058664796587 73.30963291952658) bank199528 +199529 POINT(40.68208537833558 73.19988198841853) bank199529 +199530 POINT(40.888525095349735 73.56759253129643) bank199530 +199531 POINT(40.10941655473939 74.04268846583295) bank199531 +199532 POINT(40.5520090578319 74.01679787150056) bank199532 +199533 POINT(40.543718156813235 73.09177511862673) bank199533 +199534 POINT(40.48395538707529 73.10995195757964) bank199534 +199535 POINT(41.65497868911956 74.23164086294298) bank199535 +199536 POINT(41.348465276879 74.12348167338624) bank199536 +199537 POINT(39.721028013139644 74.1020914078327) bank199537 +199538 POINT(40.43702898348981 74.57085574932745) bank199538 +199539 POINT(41.00358940429028 73.98166941497102) bank199539 +199540 POINT(41.468185515512246 74.02956536231216) bank199540 +199541 POINT(39.779065406175256 74.61140832940296) bank199541 +199542 POINT(40.584824151119406 73.69303524275303) bank199542 +199543 POINT(40.377332818686284 73.07157518030041) bank199543 +199544 POINT(40.0767841720128 74.11495050873013) bank199544 +199545 POINT(40.32379672976233 73.09798912489656) bank199545 +199546 POINT(41.12338183907606 74.98349127005008) bank199546 +199547 POINT(41.10944900223578 74.03783604535079) bank199547 +199548 POINT(40.33469019412064 74.70392216942489) bank199548 +199549 POINT(41.24330862507899 74.36858994675033) bank199549 +199550 POINT(41.63860237960005 73.81292114972483) bank199550 +199551 POINT(40.90688405120795 74.98832947184921) bank199551 +199552 POINT(41.07646906131451 74.38798520194487) bank199552 +199553 POINT(39.80382415489448 74.90992599217643) bank199553 +199554 POINT(41.29832543211794 74.86252116277461) bank199554 +199555 POINT(40.46071751235721 73.1721564143278) bank199555 +199556 POINT(40.10271981298168 74.15720745271965) bank199556 +199557 POINT(39.997885239335154 74.03476606802649) bank199557 +199558 POINT(39.96792101051758 74.92511161436933) bank199558 +199559 POINT(40.217810234542476 73.98922561192985) bank199559 +199560 POINT(41.16635467069309 74.1448589541431) bank199560 +199561 POINT(40.65871729996336 74.9953626880038) bank199561 +199562 POINT(41.606149606455794 73.07517538667291) bank199562 +199563 POINT(40.7339302006839 74.96666834947985) bank199563 +199564 POINT(40.53311773819239 73.75953613112071) bank199564 +199565 POINT(40.818384699782094 73.47332343016005) bank199565 +199566 POINT(39.86723598437789 74.02657843522162) bank199566 +199567 POINT(39.84143824260959 74.14539649000528) bank199567 +199568 POINT(40.24018007237776 74.64581606332864) bank199568 +199569 POINT(39.88885194461191 74.45346554742109) bank199569 +199570 POINT(39.92524597402864 73.3828236190493) bank199570 +199571 POINT(41.555944292913864 74.31054029041461) bank199571 +199572 POINT(41.68831532893097 73.77363902541508) bank199572 +199573 POINT(40.13325218768954 73.06540243921295) bank199573 +199574 POINT(41.27171782969347 73.76160125107862) bank199574 +199575 POINT(41.48285284947392 73.67840847610864) bank199575 +199576 POINT(40.391698268114446 74.51874111815697) bank199576 +199577 POINT(41.23399251813731 73.81569492725598) bank199577 +199578 POINT(41.66015462406307 74.75474964240247) bank199578 +199579 POINT(39.99820446303411 73.54611918625643) bank199579 +199580 POINT(41.442345448792544 74.58656620318425) bank199580 +199581 POINT(40.391710972077895 73.86693692758715) bank199581 +199582 POINT(40.90830703695031 73.97400255842285) bank199582 +199583 POINT(41.47345013864485 74.27774363478518) bank199583 +199584 POINT(40.14092189287293 73.13343499353367) bank199584 +199585 POINT(40.355398212313716 74.04900901996442) bank199585 +199586 POINT(40.04654660581321 73.46272088245192) bank199586 +199587 POINT(40.11557907919651 73.63216259409167) bank199587 +199588 POINT(41.216339206946415 73.98455879599251) bank199588 +199589 POINT(40.81109393626106 73.26022976353494) bank199589 +199590 POINT(41.4288638084269 74.90300935738303) bank199590 +199591 POINT(41.62651473231102 73.45736129764445) bank199591 +199592 POINT(41.09354293855997 74.17489721721714) bank199592 +199593 POINT(40.787432496979974 73.61064464157496) bank199593 +199594 POINT(41.16369375825167 74.34954124636275) bank199594 +199595 POINT(40.40880164375945 73.72312470542491) bank199595 +199596 POINT(40.87766166638622 74.79193535283648) bank199596 +199597 POINT(40.31635490748902 74.89968394513072) bank199597 +199598 POINT(41.4251848431756 74.36362250300877) bank199598 +199599 POINT(39.76791599614553 73.00926301922946) bank199599 +199600 POINT(41.070483940509604 73.70237300470468) bank199600 +199601 POINT(39.78661500563876 73.53709481094029) bank199601 +199602 POINT(40.13756109928631 73.37684288573301) bank199602 +199603 POINT(41.57519051995167 74.45750823486864) bank199603 +199604 POINT(40.923399346815046 73.75339307125145) bank199604 +199605 POINT(40.83461329058422 74.84924737353236) bank199605 +199606 POINT(40.37792583148164 74.50145194812826) bank199606 +199607 POINT(39.865053339156226 74.50301499889335) bank199607 +199608 POINT(39.917721135462784 74.1937062283841) bank199608 +199609 POINT(40.74018748534719 73.0532847806021) bank199609 +199610 POINT(40.31959666338798 74.28289358214055) bank199610 +199611 POINT(41.35167276588125 73.30488692199097) bank199611 +199612 POINT(40.673769144145474 73.07781147874627) bank199612 +199613 POINT(40.103922129749755 74.37820486275609) bank199613 +199614 POINT(40.24318647034225 73.3795707498602) bank199614 +199615 POINT(41.35857181512823 73.53328183616755) bank199615 +199616 POINT(40.89479112657674 74.71035526870577) bank199616 +199617 POINT(40.78589818785333 73.50196433913746) bank199617 +199618 POINT(41.580034350403565 73.98827605813152) bank199618 +199619 POINT(41.585972902099044 74.6888369159054) bank199619 +199620 POINT(40.70586959247202 74.16676243715685) bank199620 +199621 POINT(41.34387390238965 73.37092429909337) bank199621 +199622 POINT(41.33699059518396 74.88631388472446) bank199622 +199623 POINT(41.70233003104077 74.32628998710005) bank199623 +199624 POINT(39.90365994873277 73.2777082245284) bank199624 +199625 POINT(40.53921047953996 74.65688246339052) bank199625 +199626 POINT(40.67902799904556 73.47322429984574) bank199626 +199627 POINT(41.11907146580367 74.83654168784835) bank199627 +199628 POINT(40.29305753452566 73.04961171825472) bank199628 +199629 POINT(40.63183753061334 73.38801363774222) bank199629 +199630 POINT(41.33260384124279 74.90314770452748) bank199630 +199631 POINT(40.23135131599377 74.61910163120716) bank199631 +199632 POINT(40.07058991532492 73.95616048715915) bank199632 +199633 POINT(40.18587367232784 74.51289284436368) bank199633 +199634 POINT(40.53422788009364 73.54086790141443) bank199634 +199635 POINT(40.630254217933484 74.20639378029362) bank199635 +199636 POINT(41.02938136579501 73.43931010009149) bank199636 +199637 POINT(39.89830219684258 73.68278638398905) bank199637 +199638 POINT(40.92202256692835 74.3925055889574) bank199638 +199639 POINT(40.90238710040923 73.51952396220584) bank199639 +199640 POINT(41.34766451430423 73.18001756496967) bank199640 +199641 POINT(41.052970263918255 74.47271505820103) bank199641 +199642 POINT(40.40332051883906 74.39040104789328) bank199642 +199643 POINT(41.43520544903964 73.18643670980961) bank199643 +199644 POINT(39.796031591023265 73.22723477123084) bank199644 +199645 POINT(40.48087629536397 73.29273459694826) bank199645 +199646 POINT(40.79827870244394 74.09791464532108) bank199646 +199647 POINT(41.624182123884594 73.54736971244084) bank199647 +199648 POINT(41.702591277003734 74.0533824250731) bank199648 +199649 POINT(40.883542480807805 74.51609251586558) bank199649 +199650 POINT(40.08554975167657 74.62558741694883) bank199650 +199651 POINT(40.40675449837022 74.81086526162747) bank199651 +199652 POINT(41.202260954565986 74.1130384491595) bank199652 +199653 POINT(40.738918370094694 74.7735671443143) bank199653 +199654 POINT(39.77242910044367 73.36361041342461) bank199654 +199655 POINT(41.396815810467466 74.06807081121507) bank199655 +199656 POINT(41.319606514463686 74.09807209816692) bank199656 +199657 POINT(39.946858428585344 73.6376864689881) bank199657 +199658 POINT(40.77160141903478 74.26327108147521) bank199658 +199659 POINT(40.312789299177844 73.23649891968489) bank199659 +199660 POINT(41.506002348826 74.21879002316399) bank199660 +199661 POINT(41.42641458022017 74.30452359715518) bank199661 +199662 POINT(40.534739315951434 73.96081465521583) bank199662 +199663 POINT(41.0424295049161 73.80338193303733) bank199663 +199664 POINT(41.27169133462598 73.34464009662575) bank199664 +199665 POINT(39.84252107685131 74.81941926794795) bank199665 +199666 POINT(41.343562433804756 73.08767916690626) bank199666 +199667 POINT(40.67693316503639 73.94338437200614) bank199667 +199668 POINT(39.88561340133142 74.06462160230745) bank199668 +199669 POINT(41.57170146188555 73.81578439500234) bank199669 +199670 POINT(41.520968849208195 74.54094856480873) bank199670 +199671 POINT(41.46576262015821 74.8679983069825) bank199671 +199672 POINT(40.25273723490025 74.84309222764112) bank199672 +199673 POINT(41.06322209958272 74.8757506132786) bank199673 +199674 POINT(41.195536943920715 74.50189266385718) bank199674 +199675 POINT(39.95490560644203 73.55072818555018) bank199675 +199676 POINT(41.65019415630802 74.78533016171775) bank199676 +199677 POINT(41.21775536647742 73.16039481577795) bank199677 +199678 POINT(40.38043879136505 73.76520440717218) bank199678 +199679 POINT(40.40922484487346 73.26719509915687) bank199679 +199680 POINT(40.874380418141385 73.71170096034967) bank199680 +199681 POINT(40.476927826977615 74.03171623887219) bank199681 +199682 POINT(41.400237448154016 74.45008895018567) bank199682 +199683 POINT(39.98210230684961 73.23780614893234) bank199683 +199684 POINT(40.94633244071506 73.58097838208992) bank199684 +199685 POINT(40.464983383787235 74.28158415773784) bank199685 +199686 POINT(40.505867136973855 73.98124666622813) bank199686 +199687 POINT(40.56784406006802 73.5554945839237) bank199687 +199688 POINT(40.58862105527266 74.01144331173482) bank199688 +199689 POINT(41.61219947372818 73.30171620614978) bank199689 +199690 POINT(39.71410678085094 73.83684166059112) bank199690 +199691 POINT(40.68745743174391 73.94952239319262) bank199691 +199692 POINT(39.85273477318164 73.00988742337601) bank199692 +199693 POINT(40.07425729760752 73.04952831488784) bank199693 +199694 POINT(40.52089135418422 74.4740222742713) bank199694 +199695 POINT(40.06394984947543 73.0150445446366) bank199695 +199696 POINT(41.437762026374706 74.54056149300376) bank199696 +199697 POINT(41.5041163547301 73.59654343181609) bank199697 +199698 POINT(41.422457705885506 74.1165455519381) bank199698 +199699 POINT(41.64653515790957 74.1414972002867) bank199699 +199700 POINT(40.4327377525236 74.97731476304313) bank199700 +199701 POINT(40.72152860749373 73.62258394633359) bank199701 +199702 POINT(40.60886966101145 73.91016908430333) bank199702 +199703 POINT(40.78699371495918 74.55999443157464) bank199703 +199704 POINT(40.29442723716549 73.10514734524443) bank199704 +199705 POINT(40.99097647612452 73.47219537473073) bank199705 +199706 POINT(41.21371331871313 73.64714341768348) bank199706 +199707 POINT(41.2085649325548 74.48392830517668) bank199707 +199708 POINT(41.141819708014616 74.68403515969982) bank199708 +199709 POINT(39.75479976946199 73.89141651312438) bank199709 +199710 POINT(41.12207103356717 73.6775045366319) bank199710 +199711 POINT(41.527668431861606 73.54716382786044) bank199711 +199712 POINT(40.721302753529564 73.36926145274083) bank199712 +199713 POINT(40.06156492708552 73.80406544399578) bank199713 +199714 POINT(40.231137219037606 73.49141441873722) bank199714 +199715 POINT(40.803134645580016 73.24412064174592) bank199715 +199716 POINT(41.20686159859581 73.19012451925904) bank199716 +199717 POINT(41.52421375402484 73.70949570652675) bank199717 +199718 POINT(40.81537319579067 73.9781695922477) bank199718 +199719 POINT(39.9505559912552 73.49087379632232) bank199719 +199720 POINT(40.96361508305719 74.29326592054787) bank199720 +199721 POINT(39.810371869193965 73.07507652738207) bank199721 +199722 POINT(40.67607501671953 73.35836620118555) bank199722 +199723 POINT(40.37208654421092 73.99328909553914) bank199723 +199724 POINT(40.326914775023916 73.76765845083362) bank199724 +199725 POINT(40.73247063788468 74.53245961782703) bank199725 +199726 POINT(41.673966253208036 73.35132407715187) bank199726 +199727 POINT(41.17287223187135 74.77873736340274) bank199727 +199728 POINT(40.26848636811763 74.66105332032703) bank199728 +199729 POINT(39.97833842488648 74.79924525476218) bank199729 +199730 POINT(40.82268809117584 74.1259571457233) bank199730 +199731 POINT(39.86536071260111 74.70893802309503) bank199731 +199732 POINT(39.833067866196686 73.86007935233685) bank199732 +199733 POINT(40.833375521556036 73.41393147163711) bank199733 +199734 POINT(41.52628539924384 74.93848145019892) bank199734 +199735 POINT(41.01950127488373 73.64854046470552) bank199735 +199736 POINT(40.41985272033754 74.78046404290468) bank199736 +199737 POINT(40.08525816525699 74.45985615138488) bank199737 +199738 POINT(40.49954851651779 74.960921337695) bank199738 +199739 POINT(40.44658761626105 74.40407477625355) bank199739 +199740 POINT(41.59328623397434 73.40291462102519) bank199740 +199741 POINT(40.851379677800736 73.96055994476944) bank199741 +199742 POINT(39.73426740808813 74.32989234086791) bank199742 +199743 POINT(39.83461058185272 73.11692133319141) bank199743 +199744 POINT(41.45886834912241 74.03211423382275) bank199744 +199745 POINT(40.6192685005991 74.93880852562033) bank199745 +199746 POINT(41.31523394068603 74.89579267601331) bank199746 +199747 POINT(40.57292252436714 73.82937297578302) bank199747 +199748 POINT(41.5968071387613 73.66385512624879) bank199748 +199749 POINT(39.98395661823324 74.11720523579837) bank199749 +199750 POINT(40.434367847020845 73.20449180627489) bank199750 +199751 POINT(40.880676140536984 74.26550270233797) bank199751 +199752 POINT(40.61517171105303 74.7331504647254) bank199752 +199753 POINT(41.305826112031035 73.24890476596708) bank199753 +199754 POINT(40.502789925455076 73.92425677585001) bank199754 +199755 POINT(41.22122812640981 74.71994129328847) bank199755 +199756 POINT(41.01234964524852 73.6940347102406) bank199756 +199757 POINT(40.150717814152195 73.5399324281663) bank199757 +199758 POINT(41.434853618222355 73.59870850186961) bank199758 +199759 POINT(41.70100829087076 73.8743283511387) bank199759 +199760 POINT(41.41197071262782 73.22431357086894) bank199760 +199761 POINT(41.65266357438039 73.92643450382012) bank199761 +199762 POINT(40.69581590461678 74.09717643847549) bank199762 +199763 POINT(41.69795036990854 73.01642473194725) bank199763 +199764 POINT(39.82111774891694 74.96058290526024) bank199764 +199765 POINT(40.58319430169655 74.48422981472363) bank199765 +199766 POINT(40.238685240974675 73.96983085212395) bank199766 +199767 POINT(40.60238148073975 73.82874501628865) bank199767 +199768 POINT(41.12714437362514 74.03079370524122) bank199768 +199769 POINT(40.57687547838305 73.40826587195124) bank199769 +199770 POINT(41.066546347410345 73.54052382723715) bank199770 +199771 POINT(39.82106519682814 73.98472331016903) bank199771 +199772 POINT(40.687461760431745 74.41029035550983) bank199772 +199773 POINT(39.87777842992004 73.91742266206464) bank199773 +199774 POINT(40.49916113757744 74.03546902505548) bank199774 +199775 POINT(41.0812195600036 74.42135072110433) bank199775 +199776 POINT(41.202490777976024 74.1323567675945) bank199776 +199777 POINT(40.746920988913324 73.46437524402009) bank199777 +199778 POINT(40.582979942888684 75.00168718933327) bank199778 +199779 POINT(41.38920366135918 73.3779765111519) bank199779 +199780 POINT(40.59682896017568 73.5933402691443) bank199780 +199781 POINT(41.46945545763906 74.1556575595389) bank199781 +199782 POINT(39.881103474632766 74.06186964991828) bank199782 +199783 POINT(41.41741308135804 74.08369688058389) bank199783 +199784 POINT(40.23612860612666 74.93467532887817) bank199784 +199785 POINT(40.12712931790613 74.4431804143664) bank199785 +199786 POINT(39.987981365472905 73.36692468347135) bank199786 +199787 POINT(40.235397154925714 74.80603834651076) bank199787 +199788 POINT(40.459745000729555 73.11956401384592) bank199788 +199789 POINT(40.193413210521854 74.54715183237089) bank199789 +199790 POINT(40.22390309706671 73.79014769306053) bank199790 +199791 POINT(40.55747617944806 74.69531101513022) bank199791 +199792 POINT(41.146805014215424 74.9009863643525) bank199792 +199793 POINT(39.75505960106736 73.98743973229857) bank199793 +199794 POINT(40.79491881574424 73.24988507852342) bank199794 +199795 POINT(40.026420243128335 74.31483643602486) bank199795 +199796 POINT(41.07320495273102 74.05902736343842) bank199796 +199797 POINT(41.65628960173984 74.34944591270813) bank199797 +199798 POINT(40.620023540203555 73.41152748771523) bank199798 +199799 POINT(39.825562130580835 73.25667709495383) bank199799 +199800 POINT(41.21680683165543 73.60375014042774) bank199800 +199801 POINT(40.888556588384326 73.38858521679181) bank199801 +199802 POINT(39.73161552635993 74.28484354848524) bank199802 +199803 POINT(40.02519140993652 73.17261864383582) bank199803 +199804 POINT(40.99129365692387 74.84569842418485) bank199804 +199805 POINT(40.44989988702809 73.11521047384535) bank199805 +199806 POINT(39.87644053604176 74.57284617453958) bank199806 +199807 POINT(39.88416592293854 73.9980088311735) bank199807 +199808 POINT(40.368731475124335 73.8286206633592) bank199808 +199809 POINT(39.75644244678977 74.58498204117366) bank199809 +199810 POINT(40.653443067561604 74.0256238436197) bank199810 +199811 POINT(40.64152771214996 73.95838061742977) bank199811 +199812 POINT(41.296676722368325 73.12396378832152) bank199812 +199813 POINT(40.57058150915559 73.48323127325402) bank199813 +199814 POINT(41.42681256449187 73.59867365983524) bank199814 +199815 POINT(41.296367070410355 74.16633003733232) bank199815 +199816 POINT(41.632849249992894 73.21501238366486) bank199816 +199817 POINT(40.60540317544395 73.65845563930576) bank199817 +199818 POINT(41.07600063786642 74.0310099479808) bank199818 +199819 POINT(41.247645128447466 74.73269947454641) bank199819 +199820 POINT(39.80532494225221 74.50208219260432) bank199820 +199821 POINT(41.1351015767746 73.43185999549993) bank199821 +199822 POINT(40.55158742977737 73.2691729335367) bank199822 +199823 POINT(40.549373613343796 73.88311068598802) bank199823 +199824 POINT(41.216110148160865 74.85421007973274) bank199824 +199825 POINT(41.485496320819266 74.95626254229988) bank199825 +199826 POINT(39.9695615152477 74.77306106518282) bank199826 +199827 POINT(41.392462942506775 73.96792456352922) bank199827 +199828 POINT(41.01389447774749 73.63162408773469) bank199828 +199829 POINT(39.74869944011657 74.50494219939903) bank199829 +199830 POINT(41.07003268669659 73.73818707734415) bank199830 +199831 POINT(41.48184903552833 73.86105209928797) bank199831 +199832 POINT(39.870427219301064 74.1630129185143) bank199832 +199833 POINT(40.36404308925386 73.49695611815409) bank199833 +199834 POINT(40.372667100481046 73.37843081173355) bank199834 +199835 POINT(39.78504631243877 74.70657712334376) bank199835 +199836 POINT(40.13853660289825 73.19313881666584) bank199836 +199837 POINT(40.85850091531092 74.88189958151015) bank199837 +199838 POINT(39.967151974212356 74.21994011054177) bank199838 +199839 POINT(40.815929963005615 73.78440593470532) bank199839 +199840 POINT(40.41794834975086 74.94073173049459) bank199840 +199841 POINT(40.68804205810629 74.19621779386371) bank199841 +199842 POINT(40.46360827785243 73.87863116620447) bank199842 +199843 POINT(41.53180080425739 74.09993641464033) bank199843 +199844 POINT(41.48536729781813 73.69037036900522) bank199844 +199845 POINT(41.46323892567544 73.40645601009794) bank199845 +199846 POINT(41.61751915782934 74.04804840689994) bank199846 +199847 POINT(39.936544303664924 73.74730783413482) bank199847 +199848 POINT(40.22700155035231 74.62954285277087) bank199848 +199849 POINT(39.78260791612108 73.13284054416172) bank199849 +199850 POINT(40.56028106113707 73.46974607852712) bank199850 +199851 POINT(41.66034188671576 73.05997769152067) bank199851 +199852 POINT(40.175512713992724 74.7784038426395) bank199852 +199853 POINT(40.077460973722374 73.93420585990975) bank199853 +199854 POINT(41.67424330373661 73.51463868321444) bank199854 +199855 POINT(41.50953962537293 74.1310233103313) bank199855 +199856 POINT(39.87031533188551 73.56481905389438) bank199856 +199857 POINT(40.58193857841119 74.80826274247127) bank199857 +199858 POINT(41.6667485345678 74.72131764963277) bank199858 +199859 POINT(41.222033445132176 73.75994418386871) bank199859 +199860 POINT(41.56705704184351 73.61125057631283) bank199860 +199861 POINT(40.25792995031041 73.05252792551455) bank199861 +199862 POINT(40.38567026050738 74.0429797735097) bank199862 +199863 POINT(40.375495020623255 73.97098516730954) bank199863 +199864 POINT(40.990300641215285 73.52901540469519) bank199864 +199865 POINT(40.172734981129445 73.0357361346461) bank199865 +199866 POINT(40.91035777314011 73.59033766564701) bank199866 +199867 POINT(41.18325177456274 74.01604708928411) bank199867 +199868 POINT(40.696824824928406 73.97529149121203) bank199868 +199869 POINT(40.73300441666318 74.1353275271805) bank199869 +199870 POINT(40.335977345866496 73.88282125060641) bank199870 +199871 POINT(41.33644198973584 73.75582984204374) bank199871 +199872 POINT(39.982347279412174 74.77649922229983) bank199872 +199873 POINT(41.42055876008096 73.01842165467052) bank199873 +199874 POINT(40.97876564923336 74.92169399957346) bank199874 +199875 POINT(40.73339117304789 73.0504852225198) bank199875 +199876 POINT(41.660565225333784 74.2680120652939) bank199876 +199877 POINT(39.81593525137519 73.30204943731654) bank199877 +199878 POINT(39.90540179098126 73.22574262192495) bank199878 +199879 POINT(40.16937052119537 74.8045552152049) bank199879 +199880 POINT(39.82197813118072 74.97119630690078) bank199880 +199881 POINT(41.09093539612765 74.74231122025472) bank199881 +199882 POINT(40.72432077353907 73.60680062142536) bank199882 +199883 POINT(39.77873279155124 74.14176501797012) bank199883 +199884 POINT(40.99753277679551 73.45854162889667) bank199884 +199885 POINT(40.04682605850259 74.78036627706662) bank199885 +199886 POINT(40.96691829938667 73.44812488835993) bank199886 +199887 POINT(39.88294811213445 73.26013190337612) bank199887 +199888 POINT(41.63699700846608 73.03191504431982) bank199888 +199889 POINT(41.369719003144674 73.16727590458811) bank199889 +199890 POINT(41.08084891084518 73.57420494841345) bank199890 +199891 POINT(40.34029236834786 74.45356346613343) bank199891 +199892 POINT(41.0138443017952 73.35493655233368) bank199892 +199893 POINT(40.798822770104515 74.39009419094866) bank199893 +199894 POINT(41.350776517973536 74.26983127708583) bank199894 +199895 POINT(40.01733796168901 73.28806114579484) bank199895 +199896 POINT(40.252974724591056 73.68420251199508) bank199896 +199897 POINT(41.09898202501781 74.78464959449482) bank199897 +199898 POINT(41.547508702415854 74.28597924586609) bank199898 +199899 POINT(41.33306536636858 73.78328492719436) bank199899 +199900 POINT(41.11187646215528 73.48612021422655) bank199900 +199901 POINT(40.35420231119298 74.40189335574343) bank199901 +199902 POINT(39.91388877394 74.82028287532293) bank199902 +199903 POINT(39.95309019319856 73.92767034367789) bank199903 +199904 POINT(40.37194102947177 74.56167972730256) bank199904 +199905 POINT(40.09767276109015 73.44050009858852) bank199905 +199906 POINT(41.19150407573265 73.57910216968105) bank199906 +199907 POINT(39.79436261185843 73.75348670804215) bank199907 +199908 POINT(41.071545606263335 73.46900209711026) bank199908 +199909 POINT(41.20307141648928 74.96406602129714) bank199909 +199910 POINT(40.73673003563509 73.96333363049258) bank199910 +199911 POINT(40.385066915133095 74.95886250889855) bank199911 +199912 POINT(40.343002824942594 74.52347589711106) bank199912 +199913 POINT(41.05962931664625 73.91806623693195) bank199913 +199914 POINT(40.91562230463841 73.5729248598918) bank199914 +199915 POINT(41.41861770268872 74.93670234339577) bank199915 +199916 POINT(41.690637592975264 74.73304842252318) bank199916 +199917 POINT(40.13773139391918 74.88338119841373) bank199917 +199918 POINT(41.220539320891675 75.00081848641001) bank199918 +199919 POINT(39.837085433680194 74.20064503381087) bank199919 +199920 POINT(40.77308012279123 73.789470814331) bank199920 +199921 POINT(40.613510754086285 74.4612234294841) bank199921 +199922 POINT(41.20579851056429 74.35661771965631) bank199922 +199923 POINT(40.357011922295214 73.97463493478172) bank199923 +199924 POINT(40.19412893615074 73.67486319275456) bank199924 +199925 POINT(41.14755293191684 73.2448300925982) bank199925 +199926 POINT(39.853057653740265 73.23106461723769) bank199926 +199927 POINT(40.82008936090838 74.59820244949788) bank199927 +199928 POINT(40.37809810317707 73.05696374580204) bank199928 +199929 POINT(40.74377674329869 73.36158229301327) bank199929 +199930 POINT(40.04177828086656 73.80543456130533) bank199930 +199931 POINT(39.906997344708486 73.44569905271553) bank199931 +199932 POINT(41.43008136314593 73.4127534199281) bank199932 +199933 POINT(40.49936436261132 74.98698545326229) bank199933 +199934 POINT(40.35800649607091 74.93249278132625) bank199934 +199935 POINT(41.31574811750284 74.88911319543983) bank199935 +199936 POINT(40.25633831088413 74.0440186900048) bank199936 +199937 POINT(40.18127380586845 73.46730694169281) bank199937 +199938 POINT(41.402710818740054 73.61854514025292) bank199938 +199939 POINT(41.04843225969698 74.64149816073314) bank199939 +199940 POINT(40.444000896266076 73.11197378203734) bank199940 +199941 POINT(41.61818176328549 73.48508150371312) bank199941 +199942 POINT(40.07464091695125 74.55075693462832) bank199942 +199943 POINT(40.07292953286716 74.05153432753825) bank199943 +199944 POINT(40.660445064681895 74.54547573846455) bank199944 +199945 POINT(41.66951199410992 74.21836094027304) bank199945 +199946 POINT(39.97805798113267 73.33666259742445) bank199946 +199947 POINT(40.375277102253264 74.94272276826922) bank199947 +199948 POINT(39.865285616138344 74.95880617371118) bank199948 +199949 POINT(41.63729615465102 73.84041660420233) bank199949 +199950 POINT(41.134813163593485 74.75643857311655) bank199950 +199951 POINT(40.15734391092088 73.97459741569091) bank199951 +199952 POINT(40.74958085723972 74.06033319491964) bank199952 +199953 POINT(40.085961757396035 74.11071356134346) bank199953 +199954 POINT(40.25574469543 74.04546638061912) bank199954 +199955 POINT(41.20982415280266 74.63956153627979) bank199955 +199956 POINT(41.10060258356376 74.01067985115198) bank199956 +199957 POINT(41.55841083245734 74.10345868389611) bank199957 +199958 POINT(41.56554150379846 73.64925803336298) bank199958 +199959 POINT(41.6230366080017 74.187305608688) bank199959 +199960 POINT(40.18934201945099 73.2715968884216) bank199960 +199961 POINT(40.89494866932974 74.2253421989616) bank199961 +199962 POINT(41.17046980993032 74.1727612989446) bank199962 +199963 POINT(41.0917252152513 73.50149752229851) bank199963 +199964 POINT(40.26013329584502 73.4856170191918) bank199964 +199965 POINT(40.76727937071876 73.6094653678473) bank199965 +199966 POINT(40.68641242348832 73.70082024333111) bank199966 +199967 POINT(41.48820417151629 74.25901031895359) bank199967 +199968 POINT(39.92581420262735 73.31360555816592) bank199968 +199969 POINT(40.19789735494785 74.36304467966207) bank199969 +199970 POINT(40.61302174332178 73.26712349544489) bank199970 +199971 POINT(39.97835216455728 74.33534810497173) bank199971 +199972 POINT(41.43775142003504 73.32118043327917) bank199972 +199973 POINT(41.15852725097354 74.77907257577769) bank199973 +199974 POINT(40.24582864055061 74.57963651613989) bank199974 +199975 POINT(41.61837558411474 74.96174290298762) bank199975 +199976 POINT(40.64363415586667 73.95143470491365) bank199976 +199977 POINT(40.77985765266002 73.55802324389913) bank199977 +199978 POINT(41.10234214747565 74.01158895235058) bank199978 +199979 POINT(41.686777219121254 73.13738702141531) bank199979 +199980 POINT(39.84426897280467 73.64246520825863) bank199980 +199981 POINT(40.86703447817446 73.58448646109541) bank199981 +199982 POINT(39.83386736227347 73.61019375647881) bank199982 +199983 POINT(41.55572483204169 73.215302097643) bank199983 +199984 POINT(41.258063024024054 73.7723899795823) bank199984 +199985 POINT(41.63229089622171 74.35399870081577) bank199985 +199986 POINT(41.07669726687548 74.41816465812867) bank199986 +199987 POINT(40.8203383029996 73.91025074614959) bank199987 +199988 POINT(41.61053413160934 73.86991691119091) bank199988 +199989 POINT(41.56839921775367 74.82697694364506) bank199989 +199990 POINT(41.38825213160627 73.84064176882599) bank199990 +199991 POINT(40.72019955900803 74.75112239359609) bank199991 +199992 POINT(40.978248132389226 73.05842286260616) bank199992 +199993 POINT(40.992958889830206 73.54506638477) bank199993 +199994 POINT(40.7170786952669 73.92545249880911) bank199994 +199995 POINT(41.70939279005877 73.8674027194659) bank199995 +199996 POINT(40.99287040980352 73.83537865620184) bank199996 +199997 POINT(40.358921885123145 73.40180980186234) bank199997 +199998 POINT(40.4164610105304 74.96186122318319) bank199998 +199999 POINT(39.735845918832716 73.31972777800539) bank199999 +200000 POINT(40.399699545299455 73.13741843517522) bank200000 diff --git a/spark/common/src/test/resources/knn/queries.csv b/spark/common/src/test/resources/knn/queries.csv new file mode 100644 index 0000000000..598cb14f98 --- /dev/null +++ b/spark/common/src/test/resources/knn/queries.csv @@ -0,0 +1,3 @@ +1 POINT(1 1) station1 +2 POINT(10 10) station2 +3 POINT(-0.5 -0.5) station3 diff --git a/spark/common/src/test/resources/knn/test1-objects.csv b/spark/common/src/test/resources/knn/test1-objects.csv new file mode 100644 index 0000000000..d0ae2b03dd --- /dev/null +++ b/spark/common/src/test/resources/knn/test1-objects.csv @@ -0,0 +1,40 @@ +1,1 +2,2 +3,3 +4,4 +5,5 +6,6 +7,7 +8,8 +9,9 +10,10 +11,11 +12,12 +13,13 +14,14 +15,15 +16,16 +17,17 +18,18 +19,19 +20,20 +21,21 +22,22 +23,23 +24,24 +25,25 +26,26 +27,27 +28,28 +29,29 +30,30 +31,31 +32,32 +33,33 +34,34 +35,35 +36,36 +37,37 +38,38 +39,39 +40,40 diff --git a/spark/common/src/test/resources/knn/test1-queries.csv b/spark/common/src/test/resources/knn/test1-queries.csv new file mode 100644 index 0000000000..2f8cd89d65 --- /dev/null +++ b/spark/common/src/test/resources/knn/test1-queries.csv @@ -0,0 +1 @@ +10,10 diff --git a/spark/common/src/test/resources/knn/test1-results-p1-k3.csv b/spark/common/src/test/resources/knn/test1-results-p1-k3.csv new file mode 100644 index 0000000000..11e4e107a1 --- /dev/null +++ b/spark/common/src/test/resources/knn/test1-results-p1-k3.csv @@ -0,0 +1 @@ +POINT (10 10),POINT (9 9),POINT (10 10),POINT (11 11) diff --git a/spark/common/src/test/resources/knn/test1-results-p4-k3.csv b/spark/common/src/test/resources/knn/test1-results-p4-k3.csv new file mode 100644 index 0000000000..11e4e107a1 --- /dev/null +++ b/spark/common/src/test/resources/knn/test1-results-p4-k3.csv @@ -0,0 +1 @@ +POINT (10 10),POINT (9 9),POINT (10 10),POINT (11 11) diff --git a/spark/common/src/test/resources/knn/test10-objects.csv b/spark/common/src/test/resources/knn/test10-objects.csv new file mode 100644 index 0000000000..7663e4fc1f --- /dev/null +++ b/spark/common/src/test/resources/knn/test10-objects.csv @@ -0,0 +1,100 @@ +0.12,1.34 +1.45,0.67 +2.78,1.23 +3.89,0.56 +4.90,1.78 +5.21,0.43 +6.32,1.65 +7.53,0.21 +8.76,1.87 +9.98,0.54 +10.11,1.76 +11.22,0.32 +12.33,1.98 +13.44,0.65 +14.55,1.43 +15.66,0.87 +16.77,1.54 +17.88,0.23 +18.99,1.67 +19.10,0.45 +20.21,1.34 +21.32,0.56 +22.43,1.78 +23.54,0.21 +24.65,1.87 +25.76,0.43 +26.87,1.65 +27.98,0.87 +28.09,1.54 +29.10,0.32 +30.21,1.76 +31.32,0.54 +32.43,1.98 +33.54,0.65 +34.65,1.43 +35.76,0.12 +36.87,1.87 +37.98,0.23 +38.09,1.67 +39.10,0.45 +40.21,1.34 +41.32,0.56 +42.43,1.78 +43.54,0.21 +44.65,1.87 +45.76,0.43 +46.87,1.65 +47.98,0.87 +48.09,1.54 +49.10,0.32 +50.21,1.76 +51.32,0.54 +52.43,1.98 +53.54,0.65 +54.65,1.43 +55.76,0.12 +56.87,1.87 +57.98,0.23 +58.09,1.67 +59.10,0.45 +60.21,1.34 +61.32,0.56 +62.43,1.78 +63.54,0.21 +64.65,1.87 +65.76,0.43 +66.87,1.65 +67.98,0.87 +68.09,1.54 +69.10,0.32 +70.21,1.76 +71.32,0.54 +72.43,1.98 +73.54,0.65 +74.65,1.43 +75.76,0.12 +76.87,1.87 +77.98,0.23 +78.09,1.67 +79.10,0.45 +80.21,1.34 +81.32,0.56 +82.43,1.78 +83.54,0.21 +84.65,1.87 +85.76,0.43 +86.87,1.65 +87.98,0.87 +88.09,1.54 +89.10,0.32 +90.21,1.76 +91.32,0.54 +92.43,1.98 +93.54,0.65 +94.65,1.43 +95.76,0.12 +96.87,1.87 +97.98,0.23 +98.09,1.67 +99.10,0.45 diff --git a/spark/common/src/test/resources/knn/test10-queries.csv b/spark/common/src/test/resources/knn/test10-queries.csv new file mode 100644 index 0000000000..218a2433d1 --- /dev/null +++ b/spark/common/src/test/resources/knn/test10-queries.csv @@ -0,0 +1,20 @@ +1.2345,1.2345 +3.4567,3.4567 +5.6789,5.6789 +7.8910,7.8910 +10.1112,10.1112 +12.3456,12.3456 +14.5678,14.5678 +16.7890,16.7890 +19.0123,19.0123 +21.2345,21.2345 +23.4567,23.4567 +25.6789,25.6789 +27.8910,27.8910 +30.1234,30.1234 +32.3456,32.3456 +34.5678,34.5678 +36.7890,36.7890 +39.0123,39.0123 +41.2345,41.2345 +43.4567,43.4567 diff --git a/spark/common/src/test/resources/knn/test10-results-p1-k4.csv b/spark/common/src/test/resources/knn/test10-results-p1-k4.csv new file mode 100644 index 0000000000..e236920639 --- /dev/null +++ b/spark/common/src/test/resources/knn/test10-results-p1-k4.csv @@ -0,0 +1,20 @@ +POINT (1.2345 1.2345),POINT (0.12 1.34),POINT (1.45 0.67),POINT (2.78 1.23),POINT (3.89 0.56) +POINT (3.4567 3.4567),POINT (2.78 1.23),POINT (3.89 0.56),POINT (4.9 1.78),POINT (6.32 1.65) +POINT (5.6789 5.6789),POINT (4.9 1.78),POINT (5.21 0.43),POINT (6.32 1.65),POINT (8.76 1.87) +POINT (7.891 7.891),POINT (4.9 1.78),POINT (6.32 1.65),POINT (8.76 1.87),POINT (10.11 1.76) +POINT (10.1112 10.1112),POINT (6.32 1.65),POINT (8.76 1.87),POINT (10.11 1.76),POINT (12.33 1.98) +POINT (12.3456 12.3456),POINT (8.76 1.87),POINT (10.11 1.76),POINT (12.33 1.98),POINT (14.55 1.43) +POINT (14.5678 14.5678),POINT (10.11 1.76),POINT (12.33 1.98),POINT (14.55 1.43),POINT (16.77 1.54) +POINT (16.789 16.789),POINT (12.33 1.98),POINT (14.55 1.43),POINT (16.77 1.54),POINT (18.99 1.67) +POINT (19.0123 19.0123),POINT (16.77 1.54),POINT (18.99 1.67),POINT (20.21 1.34),POINT (22.43 1.78) +POINT (21.2345 21.2345),POINT (18.99 1.67),POINT (20.21 1.34),POINT (22.43 1.78),POINT (24.65 1.87) +POINT (23.4567 23.4567),POINT (18.99 1.67),POINT (22.43 1.78),POINT (24.65 1.87),POINT (26.87 1.65) +POINT (25.6789 25.6789),POINT (22.43 1.78),POINT (24.65 1.87),POINT (26.87 1.65),POINT (28.09 1.54) +POINT (27.891 27.891),POINT (24.65 1.87),POINT (26.87 1.65),POINT (30.21 1.76),POINT (32.43 1.98) +POINT (30.1234 30.1234),POINT (26.87 1.65),POINT (28.09 1.54),POINT (30.21 1.76),POINT (32.43 1.98) +POINT (32.3456 32.3456),POINT (30.21 1.76),POINT (32.43 1.98),POINT (34.65 1.43),POINT (36.87 1.87) +POINT (34.5678 34.5678),POINT (30.21 1.76),POINT (32.43 1.98),POINT (36.87 1.87),POINT (38.09 1.67) +POINT (36.789 36.789),POINT (32.43 1.98),POINT (34.65 1.43),POINT (36.87 1.87),POINT (38.09 1.67) +POINT (39.0123 39.0123),POINT (36.87 1.87),POINT (38.09 1.67),POINT (42.43 1.78),POINT (44.65 1.87) +POINT (41.2345 41.2345),POINT (36.87 1.87),POINT (38.09 1.67),POINT (42.43 1.78),POINT (44.65 1.87) +POINT (43.4567 43.4567),POINT (36.87 1.87),POINT (42.43 1.78),POINT (44.65 1.87),POINT (46.87 1.65) diff --git a/spark/common/src/test/resources/knn/test10-results-p4-k4-approximate.csv b/spark/common/src/test/resources/knn/test10-results-p4-k4-approximate.csv new file mode 100644 index 0000000000..0c13b6459d --- /dev/null +++ b/spark/common/src/test/resources/knn/test10-results-p4-k4-approximate.csv @@ -0,0 +1,20 @@ +POINT (1.2345 1.2345),POINT (0.12 1.34),POINT (1.45 0.67),POINT (2.78 1.23),POINT (3.89 0.56) +POINT (3.4567 3.4567),POINT (2.78 1.23),POINT (3.89 0.56),POINT (4.9 1.78),POINT (6.32 1.65) +POINT (5.6789 5.6789),POINT (4.9 1.78),POINT (5.21 0.43),POINT (6.32 1.65),POINT (8.76 1.87) +POINT (7.891 7.891),POINT (4.9 1.78),POINT (6.32 1.65),POINT (8.76 1.87),POINT (10.11 1.76) +POINT (10.1112 10.1112),POINT (6.32 1.65),POINT (8.76 1.87),POINT (10.11 1.76),POINT (12.33 1.98) +POINT (12.3456 12.3456),POINT (8.76 1.87),POINT (10.11 1.76),POINT (12.33 1.98),POINT (14.55 1.43) +POINT (14.5678 14.5678),POINT (10.11 1.76),POINT (12.33 1.98),POINT (14.55 1.43),POINT (16.77 1.54) +POINT (16.789 16.789),POINT (12.33 1.98),POINT (14.55 1.43),POINT (16.77 1.54),POINT (18.99 1.67) +POINT (19.0123 19.0123),POINT (16.77 1.54),POINT (18.99 1.67),POINT (20.21 1.34),POINT (22.43 1.78) +POINT (21.2345 21.2345),POINT (18.99 1.67),POINT (20.21 1.34),POINT (22.43 1.78),POINT (24.65 1.87) +POINT (23.4567 23.4567),POINT (18.99 1.67),POINT (22.43 1.78),POINT (24.65 1.87),POINT (26.87 1.65) +POINT (25.6789 25.6789),POINT (22.43 1.78),POINT (24.65 1.87),POINT (26.87 1.65),POINT (28.09 1.54) +POINT (27.891 27.891),POINT (24.65 1.87),POINT (26.87 1.65),POINT (30.21 1.76),POINT (32.43 1.98) +POINT (30.1234 30.1234),POINT (26.87 1.65),POINT (28.09 1.54),POINT (30.21 1.76),POINT (32.43 1.98) +POINT (32.3456 32.3456),POINT (30.21 1.76),POINT (32.43 1.98),POINT (34.65 1.43),POINT (36.87 1.87) +POINT (34.5678 34.5678),POINT (34.65 1.43),POINT (36.87 1.87),POINT (38.09 1.67),POINT (40.21 1.34) +POINT (36.789 36.789),POINT (34.65 1.43),POINT (36.87 1.87),POINT (38.09 1.67),POINT (42.43 1.78) +POINT (39.0123 39.0123),POINT (36.87 1.87),POINT (38.09 1.67),POINT (42.43 1.78),POINT (44.65 1.87) +POINT (41.2345 41.2345),POINT (36.87 1.87),POINT (38.09 1.67),POINT (42.43 1.78),POINT (44.65 1.87) +POINT (43.4567 43.4567),POINT (36.87 1.87),POINT (42.43 1.78),POINT (44.65 1.87),POINT (46.87 1.65) diff --git a/spark/common/src/test/resources/knn/test10-results-p4-k4.csv b/spark/common/src/test/resources/knn/test10-results-p4-k4.csv new file mode 100644 index 0000000000..e236920639 --- /dev/null +++ b/spark/common/src/test/resources/knn/test10-results-p4-k4.csv @@ -0,0 +1,20 @@ +POINT (1.2345 1.2345),POINT (0.12 1.34),POINT (1.45 0.67),POINT (2.78 1.23),POINT (3.89 0.56) +POINT (3.4567 3.4567),POINT (2.78 1.23),POINT (3.89 0.56),POINT (4.9 1.78),POINT (6.32 1.65) +POINT (5.6789 5.6789),POINT (4.9 1.78),POINT (5.21 0.43),POINT (6.32 1.65),POINT (8.76 1.87) +POINT (7.891 7.891),POINT (4.9 1.78),POINT (6.32 1.65),POINT (8.76 1.87),POINT (10.11 1.76) +POINT (10.1112 10.1112),POINT (6.32 1.65),POINT (8.76 1.87),POINT (10.11 1.76),POINT (12.33 1.98) +POINT (12.3456 12.3456),POINT (8.76 1.87),POINT (10.11 1.76),POINT (12.33 1.98),POINT (14.55 1.43) +POINT (14.5678 14.5678),POINT (10.11 1.76),POINT (12.33 1.98),POINT (14.55 1.43),POINT (16.77 1.54) +POINT (16.789 16.789),POINT (12.33 1.98),POINT (14.55 1.43),POINT (16.77 1.54),POINT (18.99 1.67) +POINT (19.0123 19.0123),POINT (16.77 1.54),POINT (18.99 1.67),POINT (20.21 1.34),POINT (22.43 1.78) +POINT (21.2345 21.2345),POINT (18.99 1.67),POINT (20.21 1.34),POINT (22.43 1.78),POINT (24.65 1.87) +POINT (23.4567 23.4567),POINT (18.99 1.67),POINT (22.43 1.78),POINT (24.65 1.87),POINT (26.87 1.65) +POINT (25.6789 25.6789),POINT (22.43 1.78),POINT (24.65 1.87),POINT (26.87 1.65),POINT (28.09 1.54) +POINT (27.891 27.891),POINT (24.65 1.87),POINT (26.87 1.65),POINT (30.21 1.76),POINT (32.43 1.98) +POINT (30.1234 30.1234),POINT (26.87 1.65),POINT (28.09 1.54),POINT (30.21 1.76),POINT (32.43 1.98) +POINT (32.3456 32.3456),POINT (30.21 1.76),POINT (32.43 1.98),POINT (34.65 1.43),POINT (36.87 1.87) +POINT (34.5678 34.5678),POINT (30.21 1.76),POINT (32.43 1.98),POINT (36.87 1.87),POINT (38.09 1.67) +POINT (36.789 36.789),POINT (32.43 1.98),POINT (34.65 1.43),POINT (36.87 1.87),POINT (38.09 1.67) +POINT (39.0123 39.0123),POINT (36.87 1.87),POINT (38.09 1.67),POINT (42.43 1.78),POINT (44.65 1.87) +POINT (41.2345 41.2345),POINT (36.87 1.87),POINT (38.09 1.67),POINT (42.43 1.78),POINT (44.65 1.87) +POINT (43.4567 43.4567),POINT (36.87 1.87),POINT (42.43 1.78),POINT (44.65 1.87),POINT (46.87 1.65) diff --git a/spark/common/src/test/resources/knn/test2-objects.csv b/spark/common/src/test/resources/knn/test2-objects.csv new file mode 100644 index 0000000000..623c23a534 --- /dev/null +++ b/spark/common/src/test/resources/knn/test2-objects.csv @@ -0,0 +1,40 @@ +0,0 +1,1 +2,2 +3,3 +4,4 +5,5 +6,6 +7,7 +8,8 +9,9 +10,10 +11,11 +12,12 +13,13 +14,14 +15,15 +16,16 +17,17 +18,18 +19,19 +20,20 +21,21 +22,22 +23,23 +24,24 +25,25 +26,26 +27,27 +28,28 +29,29 +30,30 +31,31 +32,32 +33,33 +34,34 +35,35 +36,36 +37,37 +38,38 +39,39 diff --git a/spark/common/src/test/resources/knn/test2-queries.csv b/spark/common/src/test/resources/knn/test2-queries.csv new file mode 100644 index 0000000000..04ac1d7118 --- /dev/null +++ b/spark/common/src/test/resources/knn/test2-queries.csv @@ -0,0 +1,2 @@ +20.5,20.5 +35.23,35.78 diff --git a/spark/common/src/test/resources/knn/test2-results-p1-k3.csv b/spark/common/src/test/resources/knn/test2-results-p1-k3.csv new file mode 100644 index 0000000000..9c6379bc38 --- /dev/null +++ b/spark/common/src/test/resources/knn/test2-results-p1-k3.csv @@ -0,0 +1,2 @@ +POINT (20.5 20.5),POINT (20 20),POINT (21 21),POINT (22 22) +POINT (35.23 35.78),POINT (35 35),POINT (36 36),POINT (37 37) diff --git a/spark/common/src/test/resources/knn/test2-results-p4-k3.csv b/spark/common/src/test/resources/knn/test2-results-p4-k3.csv new file mode 100644 index 0000000000..9c6379bc38 --- /dev/null +++ b/spark/common/src/test/resources/knn/test2-results-p4-k3.csv @@ -0,0 +1,2 @@ +POINT (20.5 20.5),POINT (20 20),POINT (21 21),POINT (22 22) +POINT (35.23 35.78),POINT (35 35),POINT (36 36),POINT (37 37) diff --git a/spark/common/src/test/resources/knn/test3-objects.csv b/spark/common/src/test/resources/knn/test3-objects.csv new file mode 100644 index 0000000000..c1b7a2a830 --- /dev/null +++ b/spark/common/src/test/resources/knn/test3-objects.csv @@ -0,0 +1,40 @@ +0,0 +0,1 +0,2 +0,3 +0,4 +1,0 +1,1 +1,2 +1,3 +1,4 +2,0 +2,1 +2,2 +2,3 +2,4 +3,0 +3,1 +3,2 +3,3 +3,4 +4,0 +4,1 +4,2 +4,3 +4,4 +5,0 +5,1 +5,2 +5,3 +5,4 +6,0 +6,1 +6,2 +6,3 +6,4 +7,0 +7,1 +7,2 +7,3 +7,4 diff --git a/spark/common/src/test/resources/knn/test3-queries.csv b/spark/common/src/test/resources/knn/test3-queries.csv new file mode 100644 index 0000000000..5a00f1e29e --- /dev/null +++ b/spark/common/src/test/resources/knn/test3-queries.csv @@ -0,0 +1 @@ +3.3,4.4 diff --git a/spark/common/src/test/resources/knn/test3-results-p1-k4.csv b/spark/common/src/test/resources/knn/test3-results-p1-k4.csv new file mode 100644 index 0000000000..b64e92d40d --- /dev/null +++ b/spark/common/src/test/resources/knn/test3-results-p1-k4.csv @@ -0,0 +1 @@ +POINT (3.3 4.4),POINT (2 4),POINT (3 3),POINT (3 4),POINT (4 4) diff --git a/spark/common/src/test/resources/knn/test3-results-p4-k4.csv b/spark/common/src/test/resources/knn/test3-results-p4-k4.csv new file mode 100644 index 0000000000..b64e92d40d --- /dev/null +++ b/spark/common/src/test/resources/knn/test3-results-p4-k4.csv @@ -0,0 +1 @@ +POINT (3.3 4.4),POINT (2 4),POINT (3 3),POINT (3 4),POINT (4 4) diff --git a/spark/common/src/test/resources/knn/test4-objects.csv b/spark/common/src/test/resources/knn/test4-objects.csv new file mode 100644 index 0000000000..d9807e29f2 --- /dev/null +++ b/spark/common/src/test/resources/knn/test4-objects.csv @@ -0,0 +1,120 @@ +0.00,0.001 +0.10,0.102 +0.20,0.203 +0.30,0.304 +0.40,0.405 +0.50,0.506 +0.60,0.607 +0.70,0.708 +0.80,0.809 +0.90,0.900 +1.00,1.001 +1.10,1.102 +1.20,1.203 +1.30,1.304 +1.40,1.405 +1.50,1.506 +1.60,1.607 +1.70,1.708 +1.80,1.809 +1.90,1.900 +2.001,2.00 +2.102,2.10 +2.203,2.20 +2.304,2.30 +2.405,2.40 +2.506,2.50 +2.607,2.60 +2.708,2.70 +2.809,2.80 +2.900,2.90 +3.00,3.001 +3.10,3.102 +3.20,3.203 +3.30,3.304 +3.40,3.405 +3.50,3.506 +3.60,3.607 +3.70,3.708 +3.80,3.809 +3.90,3.900 +10.001,10.00 +10.102,10.10 +10.203,10.20 +10.304,10.30 +10.405,10.40 +10.506,10.50 +10.607,10.60 +10.708,10.70 +10.809,10.80 +10.900,10.90 +11.00,11.001 +11.10,11.102 +11.20,11.203 +11.30,11.304 +11.40,11.405 +11.50,11.506 +11.60,11.607 +11.70,11.708 +11.80,11.809 +11.90,11.900 +12.001,12.00 +12.102,12.10 +12.203,12.20 +12.304,12.30 +12.405,12.40 +12.506,12.50 +12.607,12.60 +12.708,12.70 +12.801,12.80 +12.902,12.90 +13.003,13.00 +13.104,13.10 +13.205,13.20 +13.306,13.30 +13.407,13.40 +13.508,13.50 +13.608,13.60 +13.70,13.701 +13.80,13.802 +13.90,13.903 +20.00,20.004 +20.10,20.105 +20.20,20.206 +20.30,20.307 +20.40,20.408 +20.50,20.509 +20.60,20.600 +20.701,20.70 +20.802,20.80 +20.903,20.90 +21.004,21.00 +21.105,21.10 +21.206,21.20 +21.307,21.30 +21.408,21.40 +21.509,21.50 +21.600,21.60 +21.70,21.701 +21.80,21.802 +21.90,21.903 +22.00,22.004 +22.10,22.105 +22.20,22.206 +22.30,22.307 +22.40,22.408 +22.50,22.509 +22.60,22.600 +22.701,22.70 +22.802,22.80 +22.903,22.90 +23.004,23.00 +23.105,23.10 +23.206,23.20 +23.307,23.30 +23.408,23.40 +23.509,23.50 +23.600,23.60 +23.701,23.701 +23.802,23.802 +23.903,23.903 diff --git a/spark/common/src/test/resources/knn/test4-queries.csv b/spark/common/src/test/resources/knn/test4-queries.csv new file mode 100644 index 0000000000..1ae355d7ab --- /dev/null +++ b/spark/common/src/test/resources/knn/test4-queries.csv @@ -0,0 +1,5 @@ +2.50,2.50 +12.50,12.50 +11.231,10.23 +20.31,18.45 +45.50,50.50 diff --git a/spark/common/src/test/resources/knn/test4-results-p1-k4.csv b/spark/common/src/test/resources/knn/test4-results-p1-k4.csv new file mode 100644 index 0000000000..95d19229f2 --- /dev/null +++ b/spark/common/src/test/resources/knn/test4-results-p1-k4.csv @@ -0,0 +1,5 @@ +POINT (2.5 2.5),POINT (2.304 2.3),POINT (2.405 2.4),POINT (2.506 2.5),POINT (2.607 2.6) +POINT (11.231 10.23),POINT (10.607 10.6),POINT (10.708 10.7),POINT (10.809 10.8),POINT (10.9 10.9) +POINT (12.5 12.5),POINT (12.304 12.3),POINT (12.405 12.4),POINT (12.506 12.5),POINT (12.607 12.6) +POINT (20.31 18.45),POINT (20 20.004),POINT (20.1 20.105),POINT (20.2 20.206),POINT (20.3 20.307) +POINT (45.5 50.5),POINT (23.6 23.6),POINT (23.701 23.701),POINT (23.802 23.802),POINT (23.903 23.903) diff --git a/spark/common/src/test/resources/knn/test4-results-p4-k4.csv b/spark/common/src/test/resources/knn/test4-results-p4-k4.csv new file mode 100644 index 0000000000..95d19229f2 --- /dev/null +++ b/spark/common/src/test/resources/knn/test4-results-p4-k4.csv @@ -0,0 +1,5 @@ +POINT (2.5 2.5),POINT (2.304 2.3),POINT (2.405 2.4),POINT (2.506 2.5),POINT (2.607 2.6) +POINT (11.231 10.23),POINT (10.607 10.6),POINT (10.708 10.7),POINT (10.809 10.8),POINT (10.9 10.9) +POINT (12.5 12.5),POINT (12.304 12.3),POINT (12.405 12.4),POINT (12.506 12.5),POINT (12.607 12.6) +POINT (20.31 18.45),POINT (20 20.004),POINT (20.1 20.105),POINT (20.2 20.206),POINT (20.3 20.307) +POINT (45.5 50.5),POINT (23.6 23.6),POINT (23.701 23.701),POINT (23.802 23.802),POINT (23.903 23.903) diff --git a/spark/common/src/test/resources/knn/test5-objects.csv b/spark/common/src/test/resources/knn/test5-objects.csv new file mode 100644 index 0000000000..4771ff929d --- /dev/null +++ b/spark/common/src/test/resources/knn/test5-objects.csv @@ -0,0 +1,94 @@ +0.00,0.00 +1.50,1.50 +3.00,3.00 +4.50,4.50 +6.00,6.00 +7.50,7.50 +9.00,9.00 +10.50,10.50 +12.00,12.00 +13.50,13.50 +15.00,15.00 +16.50,16.50 +18.00,18.00 +19.50,19.50 +21.00,21.00 +22.50,22.50 +24.00,24.00 +25.50,25.50 +27.00,27.00 +28.50,28.50 +30.00,30.00 +31.50,31.50 +33.00,33.00 +34.50,34.50 +36.00,36.00 +37.50,37.50 +39.00,39.00 +40.50,40.50 +42.00,42.00 +43.50,43.50 +45.00,45.00 +46.50,46.50 +48.00,48.00 +49.50,49.50 +51.00,51.00 +52.50,52.50 +54.00,54.00 +55.50,55.50 +57.00,57.00 +58.50,58.50 +60.00,60.00 +61.50,61.50 +63.00,63.00 +64.50,64.50 +66.00,66.00 +67.50,67.50 +69.00,69.00 +70.50,70.50 +72.00,72.00 +73.50,73.50 +75.00,75.00 +76.50,76.50 +78.00,78.00 +79.50,79.50 +81.00,81.00 +82.50,82.50 +84.00,84.00 +85.50,85.50 +87.00,87.00 +88.50,88.50 +90.00,90.00 +91.50,91.50 +93.00,93.00 +94.50,94.50 +96.00,96.00 +97.50,97.50 +99.00,99.00 +100.50,100.50 +102.00,102.00 +103.50,103.50 +105.00,105.00 +106.50,106.50 +108.00,108.00 +109.50,109.50 +111.00,111.00 +112.50,112.50 +114.00,114.00 +115.50,115.50 +117.00,117.00 +118.50,118.50 +120.00,120.00 +121.50,121.50 +123.00,123.00 +124.50,124.50 +126.00,126.00 +127.50,127.50 +129.00,129.00 +130.50,130.50 +132.00,132.00 +133.50,133.50 +135.00,135.00 +136.50,136.50 +138.00,138.00 +139.50,139.50 diff --git a/spark/common/src/test/resources/knn/test5-queries.csv b/spark/common/src/test/resources/knn/test5-queries.csv new file mode 100644 index 0000000000..63d9bfa55e --- /dev/null +++ b/spark/common/src/test/resources/knn/test5-queries.csv @@ -0,0 +1,5 @@ +2.50,2.50 +12.50,12.50 +11.231,10.23 +20.31,18.45 +45.23,50.17 diff --git a/spark/common/src/test/resources/knn/test5-results-p1-k4.csv b/spark/common/src/test/resources/knn/test5-results-p1-k4.csv new file mode 100644 index 0000000000..fd618297e7 --- /dev/null +++ b/spark/common/src/test/resources/knn/test5-results-p1-k4.csv @@ -0,0 +1,5 @@ +POINT (2.5 2.5),POINT (0 0),POINT (1.5 1.5),POINT (3 3),POINT (4.5 4.5) +POINT (11.231 10.23),POINT (9 9),POINT (10.5 10.5),POINT (12 12),POINT (13.5 13.5) +POINT (12.5 12.5),POINT (10.5 10.5),POINT (12 12),POINT (13.5 13.5),POINT (15 15) +POINT (20.31 18.45),POINT (16.5 16.5),POINT (18 18),POINT (19.5 19.5),POINT (21 21) +POINT (45.23 50.17),POINT (45 45),POINT (46.5 46.5),POINT (48 48),POINT (49.5 49.5) diff --git a/spark/common/src/test/resources/knn/test5-results-p4-k4.csv b/spark/common/src/test/resources/knn/test5-results-p4-k4.csv new file mode 100644 index 0000000000..fd618297e7 --- /dev/null +++ b/spark/common/src/test/resources/knn/test5-results-p4-k4.csv @@ -0,0 +1,5 @@ +POINT (2.5 2.5),POINT (0 0),POINT (1.5 1.5),POINT (3 3),POINT (4.5 4.5) +POINT (11.231 10.23),POINT (9 9),POINT (10.5 10.5),POINT (12 12),POINT (13.5 13.5) +POINT (12.5 12.5),POINT (10.5 10.5),POINT (12 12),POINT (13.5 13.5),POINT (15 15) +POINT (20.31 18.45),POINT (16.5 16.5),POINT (18 18),POINT (19.5 19.5),POINT (21 21) +POINT (45.23 50.17),POINT (45 45),POINT (46.5 46.5),POINT (48 48),POINT (49.5 49.5) diff --git a/spark/common/src/test/resources/knn/test6-objects.csv b/spark/common/src/test/resources/knn/test6-objects.csv new file mode 100644 index 0000000000..5173f3f7d3 --- /dev/null +++ b/spark/common/src/test/resources/knn/test6-objects.csv @@ -0,0 +1,99 @@ +0.00,0.00 +0.00,10.00 +0.00,20.00 +0.00,30.00 +0.00,40.00 +0.00,50.00 +0.00,60.00 +0.00,70.00 +0.00,80.00 +0.00,90.00 +0.00,100.00 +10.00,0.00 +10.00,10.00 +10.00,20.00 +10.00,30.00 +10.00,40.00 +10.00,50.00 +10.00,60.00 +10.00,70.00 +10.00,80.00 +10.00,90.00 +10.00,100.00 +20.00,0.00 +20.00,10.00 +20.00,20.00 +20.00,30.00 +20.00,40.00 +20.00,50.00 +20.00,60.00 +20.00,70.00 +20.00,80.00 +20.00,90.00 +20.00,100.00 +30.00,0.00 +30.00,10.00 +30.00,20.00 +30.00,30.00 +30.00,40.00 +30.00,50.00 +30.00,60.00 +30.00,70.00 +30.00,80.00 +30.00,90.00 +30.00,100.00 +40.00,0.00 +40.00,10.00 +40.00,20.00 +40.00,30.00 +40.00,40.00 +40.00,50.00 +40.00,60.00 +40.00,70.00 +40.00,80.00 +40.00,90.00 +40.00,100.00 +50.00,0.00 +50.00,10.00 +50.00,20.00 +50.00,30.00 +50.00,40.00 +50.00,50.00 +50.00,60.00 +50.00,70.00 +50.00,80.00 +50.00,90.00 +50.00,100.00 +60.00,0.00 +60.00,10.00 +60.00,20.00 +60.00,30.00 +60.00,40.00 +60.00,50.00 +60.00,60.00 +60.00,70.00 +60.00,80.00 +60.00,90.00 +60.00,100.00 +70.00,0.00 +70.00,10.00 +70.00,20.00 +70.00,30.00 +70.00,40.00 +70.00,50.00 +70.00,60.00 +70.00,70.00 +70.00,80.00 +70.00,90.00 +70.00,100.00 +80.00,0.00 +80.00,10.00 +80.00,20.00 +80.00,30.00 +80.00,40.00 +80.00,50.00 +80.00,60.00 +80.00,70.00 +80.00,80.00 +80.00,90.00 +80.00,100.00 diff --git a/spark/common/src/test/resources/knn/test6-queries.csv b/spark/common/src/test/resources/knn/test6-queries.csv new file mode 100644 index 0000000000..b0d73d469b --- /dev/null +++ b/spark/common/src/test/resources/knn/test6-queries.csv @@ -0,0 +1,3 @@ +0.00,0.00 +80.00,0.00 +0.00,80.00 diff --git a/spark/common/src/test/resources/knn/test6-results-p1-k4.csv b/spark/common/src/test/resources/knn/test6-results-p1-k4.csv new file mode 100644 index 0000000000..d791e00a44 --- /dev/null +++ b/spark/common/src/test/resources/knn/test6-results-p1-k4.csv @@ -0,0 +1,3 @@ +POINT (0 0),POINT (0 0),POINT (0 10),POINT (10 0),POINT (10 10) +POINT (0 80),POINT (0 70),POINT (0 80),POINT (0 90),POINT (10 80) +POINT (80 0),POINT (70 0),POINT (70 10),POINT (80 0),POINT (80 10) diff --git a/spark/common/src/test/resources/knn/test6-results-p4-k4.csv b/spark/common/src/test/resources/knn/test6-results-p4-k4.csv new file mode 100644 index 0000000000..d791e00a44 --- /dev/null +++ b/spark/common/src/test/resources/knn/test6-results-p4-k4.csv @@ -0,0 +1,3 @@ +POINT (0 0),POINT (0 0),POINT (0 10),POINT (10 0),POINT (10 10) +POINT (0 80),POINT (0 70),POINT (0 80),POINT (0 90),POINT (10 80) +POINT (80 0),POINT (70 0),POINT (70 10),POINT (80 0),POINT (80 10) diff --git a/spark/common/src/test/resources/knn/test7-objects.csv b/spark/common/src/test/resources/knn/test7-objects.csv new file mode 100644 index 0000000000..78d43db438 --- /dev/null +++ b/spark/common/src/test/resources/knn/test7-objects.csv @@ -0,0 +1,40 @@ +0.00,1.00 +0.71,0.71 +1.00,0.00 +0.71,-0.71 +0.00,-1.00 +-0.71,-0.71 +-1.00,0.00 +-0.71,0.71 +0.50,0.87 +0.87,0.50 +0.87,-0.50 +0.50,-0.87 +-0.50,-0.87 +-0.87,-0.50 +-0.87,0.50 +-0.50,0.87 +0.30,0.95 +0.95,0.30 +0.95,-0.30 +0.30,-0.95 +-0.30,-0.95 +-0.95,-0.30 +-0.95,0.30 +-0.30,0.95 +0.98,0.17 +0.17,0.98 +0.17,-0.98 +0.98,-0.17 +-0.17,-0.98 +-0.98,-0.17 +-0.98,0.17 +-0.17,0.98 +0.93,0.36 +0.36,0.93 +0.36,-0.93 +0.93,-0.36 +-0.36,-0.93 +-0.93,-0.36 +-0.93,0.36 +-0.36,0.93 diff --git a/spark/common/src/test/resources/knn/test7-queries.csv b/spark/common/src/test/resources/knn/test7-queries.csv new file mode 100644 index 0000000000..0571818ff1 --- /dev/null +++ b/spark/common/src/test/resources/knn/test7-queries.csv @@ -0,0 +1,2 @@ +0.50,0.50 +0.90,0.60 diff --git a/spark/common/src/test/resources/knn/test7-results-p1-k4.csv b/spark/common/src/test/resources/knn/test7-results-p1-k4.csv new file mode 100644 index 0000000000..2709d366d8 --- /dev/null +++ b/spark/common/src/test/resources/knn/test7-results-p1-k4.csv @@ -0,0 +1,2 @@ +POINT (0.5 0.5),POINT (0.5 0.87),POINT (0.71 0.71),POINT (0.87 0.5),POINT (0.93 0.36) +POINT (0.9 0.6),POINT (0.71 0.71),POINT (0.87 0.5),POINT (0.93 0.36),POINT (0.95 0.3) diff --git a/spark/common/src/test/resources/knn/test7-results-p4-k4.csv b/spark/common/src/test/resources/knn/test7-results-p4-k4.csv new file mode 100644 index 0000000000..2709d366d8 --- /dev/null +++ b/spark/common/src/test/resources/knn/test7-results-p4-k4.csv @@ -0,0 +1,2 @@ +POINT (0.5 0.5),POINT (0.5 0.87),POINT (0.71 0.71),POINT (0.87 0.5),POINT (0.93 0.36) +POINT (0.9 0.6),POINT (0.71 0.71),POINT (0.87 0.5),POINT (0.93 0.36),POINT (0.95 0.3) diff --git a/spark/common/src/test/resources/knn/test8-objects.csv b/spark/common/src/test/resources/knn/test8-objects.csv new file mode 100644 index 0000000000..8ab70e7e21 --- /dev/null +++ b/spark/common/src/test/resources/knn/test8-objects.csv @@ -0,0 +1,151 @@ +0.00,0.00 +1.00,1.00 +2.00,2.00 +3.00,3.00 +4.00,4.00 +5.00,5.00 +6.00,6.00 +7.00,7.00 +8.00,8.00 +9.00,9.00 +10.00,10.00 +11.00,11.00 +12.00,12.00 +13.00,13.00 +14.00,14.00 +15.00,15.00 +16.00,16.00 +17.00,17.00 +18.00,18.00 +19.00,19.00 +20.00,20.00 +21.00,21.00 +22.00,22.00 +23.00,23.00 +24.00,24.00 +25.00,25.00 +26.00,26.00 +27.00,27.00 +28.00,28.00 +29.00,29.00 +30.00,30.00 +31.00,31.00 +32.00,32.00 +33.00,33.00 +34.00,34.00 +35.00,35.00 +36.00,36.00 +37.00,37.00 +38.00,38.00 +39.00,39.00 +40.00,40.00 +41.00,41.00 +42.00,42.00 +43.00,43.00 +44.00,44.00 +45.00,45.00 +46.00,46.00 +47.00,47.00 +48.00,48.00 +49.00,49.00 +50.00,50.00 +51.00,51.00 +52.00,52.00 +53.00,53.00 +54.00,54.00 +55.00,55.00 +56.00,56.00 +57.00,57.00 +58.00,58.00 +59.00,59.00 +60.00,60.00 +61.00,61.00 +62.00,62.00 +63.00,63.00 +64.00,64.00 +65.00,65.00 +66.00,66.00 +67.00,67.00 +68.00,68.00 +69.00,69.00 +70.00,70.00 +71.00,71.00 +72.00,72.00 +73.00,73.00 +74.00,74.00 +75.00,75.00 +76.00,76.00 +77.00,77.00 +78.00,78.00 +79.00,79.00 +80.00,80.00 +81.00,81.00 +82.00,82.00 +83.00,83.00 +84.00,84.00 +85.00,85.00 +86.00,86.00 +87.00,87.00 +88.00,88.00 +89.00,89.00 +90.00,90.00 +91.00,91.00 +92.00,92.00 +93.00,93.00 +94.00,94.00 +95.00,95.00 +96.00,96.00 +97.00,97.00 +98.00,98.00 +99.00,99.00 +100.00,100.00 +101.00,101.00 +102.00,102.00 +103.00,103.00 +104.00,104.00 +105.00,105.00 +106.00,106.00 +107.00,107.00 +108.00,108.00 +109.00,109.00 +110.00,110.00 +111.00,111.00 +112.00,112.00 +113.00,113.00 +114.00,114.00 +115.00,115.00 +116.00,116.00 +117.00,117.00 +118.00,118.00 +119.00,119.00 +120.00,120.00 +121.00,121.00 +122.00,122.00 +123.00,123.00 +124.00,124.00 +125.00,125.00 +126.00,126.00 +127.00,127.00 +128.00,128.00 +129.00,129.00 +130.00,130.00 +131.00,131.00 +132.00,132.00 +133.00,133.00 +134.00,134.00 +135.00,135.00 +136.00,136.00 +137.00,137.00 +138.00,138.00 +139.00,139.00 +140.00,140.00 +141.00,141.00 +142.00,142.00 +143.00,143.00 +144.00,144.00 +145.00,145.00 +146.00,146.00 +147.00,147.00 +148.00,148.00 +149.00,149.00 +150.00,150.00 diff --git a/spark/common/src/test/resources/knn/test8-queries.csv b/spark/common/src/test/resources/knn/test8-queries.csv new file mode 100644 index 0000000000..c1bfbbcab1 --- /dev/null +++ b/spark/common/src/test/resources/knn/test8-queries.csv @@ -0,0 +1,5 @@ +2.50,2.50 +12.50,12.50 +11.231,10.23 +20.31,18.45 +45.46,50.89 diff --git a/spark/common/src/test/resources/knn/test8-results-p1-k4.csv b/spark/common/src/test/resources/knn/test8-results-p1-k4.csv new file mode 100644 index 0000000000..2f965f70ac --- /dev/null +++ b/spark/common/src/test/resources/knn/test8-results-p1-k4.csv @@ -0,0 +1,5 @@ +POINT (2.5 2.5),POINT (1 1),POINT (2 2),POINT (3 3),POINT (4 4) +POINT (11.231 10.23),POINT (9 9),POINT (10 10),POINT (11 11),POINT (12 12) +POINT (12.5 12.5),POINT (11 11),POINT (12 12),POINT (13 13),POINT (14 14) +POINT (20.31 18.45),POINT (18 18),POINT (19 19),POINT (20 20),POINT (21 21) +POINT (45.46 50.89),POINT (47 47),POINT (48 48),POINT (49 49),POINT (50 50) diff --git a/spark/common/src/test/resources/knn/test8-results-p4-k4.csv b/spark/common/src/test/resources/knn/test8-results-p4-k4.csv new file mode 100644 index 0000000000..2f965f70ac --- /dev/null +++ b/spark/common/src/test/resources/knn/test8-results-p4-k4.csv @@ -0,0 +1,5 @@ +POINT (2.5 2.5),POINT (1 1),POINT (2 2),POINT (3 3),POINT (4 4) +POINT (11.231 10.23),POINT (9 9),POINT (10 10),POINT (11 11),POINT (12 12) +POINT (12.5 12.5),POINT (11 11),POINT (12 12),POINT (13 13),POINT (14 14) +POINT (20.31 18.45),POINT (18 18),POINT (19 19),POINT (20 20),POINT (21 21) +POINT (45.46 50.89),POINT (47 47),POINT (48 48),POINT (49 49),POINT (50 50) diff --git a/spark/common/src/test/resources/knn/test9-objects.csv b/spark/common/src/test/resources/knn/test9-objects.csv new file mode 100644 index 0000000000..9941fce5a7 --- /dev/null +++ b/spark/common/src/test/resources/knn/test9-objects.csv @@ -0,0 +1,100 @@ +1.00,1.00 +1.00,1.00 +1.00,1.00 +1.00,1.00 +1.00,1.00 +1.00,1.00 +1.00,1.00 +1.00,1.00 +1.00,1.00 +1.00,1.00 +2.00,2.00 +2.00,2.00 +2.00,2.00 +2.00,2.00 +2.00,2.00 +2.00,2.00 +2.00,2.00 +2.00,2.00 +2.00,2.00 +2.00,2.00 +3.00,3.00 +3.00,3.00 +3.00,3.00 +3.00,3.00 +3.00,3.00 +3.00,3.00 +3.00,3.00 +3.00,3.00 +3.00,3.00 +3.00,3.00 +4.00,4.00 +4.00,4.00 +4.00,4.00 +4.00,4.00 +4.00,4.00 +4.00,4.00 +4.00,4.00 +4.00,4.00 +4.00,4.00 +4.00,4.00 +5.00,5.00 +5.00,5.00 +5.00,5.00 +5.00,5.00 +5.00,5.00 +5.00,5.00 +5.00,5.00 +5.00,5.00 +5.00,5.00 +5.00,5.00 +6.00,6.00 +6.00,6.00 +6.00,6.00 +6.00,6.00 +6.00,6.00 +6.00,6.00 +6.00,6.00 +6.00,6.00 +6.00,6.00 +6.00,6.00 +7.00,7.00 +7.00,7.00 +7.00,7.00 +7.00,7.00 +7.00,7.00 +7.00,7.00 +7.00,7.00 +7.00,7.00 +7.00,7.00 +7.00,7.00 +8.00,8.00 +8.00,8.00 +8.00,8.00 +8.00,8.00 +8.00,8.00 +8.00,8.00 +8.00,8.00 +8.00,8.00 +8.00,8.00 +8.00,8.00 +9.00,9.00 +9.00,9.00 +9.00,9.00 +9.00,9.00 +9.00,9.00 +9.00,9.00 +9.00,9.00 +9.00,9.00 +9.00,9.00 +9.00,9.00 +10.00,10.00 +10.00,10.00 +10.00,10.00 +10.00,10.00 +10.00,10.00 +10.00,10.00 +10.00,10.00 +10.00,10.00 +10.00,10.00 +10.00,10.00 diff --git a/spark/common/src/test/resources/knn/test9-queries.csv b/spark/common/src/test/resources/knn/test9-queries.csv new file mode 100644 index 0000000000..f7b928c838 --- /dev/null +++ b/spark/common/src/test/resources/knn/test9-queries.csv @@ -0,0 +1,3 @@ +2.46,2.43 +3.59,3.29 +8.231,8.23 diff --git a/spark/common/src/test/resources/knn/test9-results-p1-k4.csv b/spark/common/src/test/resources/knn/test9-results-p1-k4.csv new file mode 100644 index 0000000000..fa83eef888 --- /dev/null +++ b/spark/common/src/test/resources/knn/test9-results-p1-k4.csv @@ -0,0 +1,3 @@ +POINT (2.46 2.43),POINT (2 2),POINT (2 2),POINT (2 2),POINT (2 2) +POINT (3.59 3.29),POINT (3 3),POINT (3 3),POINT (3 3),POINT (3 3) +POINT (8.231 8.23),POINT (8 8),POINT (8 8),POINT (8 8),POINT (8 8) diff --git a/spark/common/src/test/resources/knn/test9-results-p4-k4.csv b/spark/common/src/test/resources/knn/test9-results-p4-k4.csv new file mode 100644 index 0000000000..fa83eef888 --- /dev/null +++ b/spark/common/src/test/resources/knn/test9-results-p4-k4.csv @@ -0,0 +1,3 @@ +POINT (2.46 2.43),POINT (2 2),POINT (2 2),POINT (2 2),POINT (2 2) +POINT (3.59 3.29),POINT (3 3),POINT (3 3),POINT (3 3),POINT (3 3) +POINT (8.231 8.23),POINT (8 8),POINT (8 8),POINT (8 8),POINT (8 8) diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala b/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala index eadf02d203..c0cef47418 100644 --- a/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala +++ b/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala @@ -62,70 +62,6 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { } describe("KNN spatial join SQLs should be parsed correctly") { - it("KNN Join with approximate algorithms based on euclidean distance") { - val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") - validateQueryPlan( - df, - numNeighbors = 3, - useApproximate = true, - expressionSize = 5, - isGeography = false, - mustInclude = "") - } - - it( - "KNN Join with approximate algorithms based on euclidean distance using different join clause ordering") { - val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM OBJECTS JOIN QUERIES ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") - validateQueryPlan( - df, - numNeighbors = 3, - useApproximate = true, - expressionSize = 5, - isGeography = false, - mustInclude = "") - } - - it( - "KNN Join with approximate algorithms based on euclidean distance using join-where clause and apply ST_Distance") { - val df = sparkSession.sql( - s"SELECT QUERIES.GEOM, OBJECTS.GEOM, ST_Distance(QUERIES.GEOM, OBJECTS.GEOM) FROM QUERIES, OBJECTS WHERE ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") - validateQueryPlan( - df, - numNeighbors = 3, - useApproximate = true, - expressionSize = 5, - isGeography = false, - mustInclude = "") - } - - it( - "KNN Join with approximate algorithms based on euclidean distance using join-where clause and select gem") { - val df = sparkSession.sql( - s"SELECT QUERIES.GEOM, OBJECTS.GEOM FROM QUERIES, OBJECTS WHERE ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") - validateQueryPlan( - df, - numNeighbors = 3, - useApproximate = true, - expressionSize = 5, - isGeography = false, - mustInclude = "") - } - - it( - "KNN Join with approximate algorithms based on euclidean distance using join-where clause and select all") { - val df = sparkSession.sql( - s"SELECT * FROM QUERIES, OBJECTS WHERE ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false)") - validateQueryPlan( - df, - numNeighbors = 3, - useApproximate = true, - expressionSize = 5, - isGeography = false, - mustInclude = "") - } - it("KNN Join with exact algorithms based on euclidean distance") { val df = sparkSession.sql( s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 3, true)") @@ -141,7 +77,7 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { it("KNN Join based on single point on left side should not be supported") { val exception = intercept[UnsupportedOperationException] { val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(ST_MakePoint(100, 100, 1), OBJECTS.GEOM, 3, false)") + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(ST_MakePoint(100, 100, 1), OBJECTS.GEOM, 3, false)") validateQueryPlan( df, numNeighbors = 3, @@ -150,13 +86,13 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { isGeography = false, mustInclude = "") } - exception.getMessage should include("ST_AKNN filter is not yet supported in the join query") + exception.getMessage should include("ST_KNN filter is not yet supported in the join query") } it("KNN Join based on single point on right side should not be supported") { val exception = intercept[UnsupportedOperationException] { val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(OBJECTS.GEOM, ST_MakePoint(100, 100, 1), 3, false)") + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(OBJECTS.GEOM, ST_MakePoint(100, 100, 1), 3, false)") validateQueryPlan( df, numNeighbors = 3, @@ -165,12 +101,12 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { isGeography = false, mustInclude = "") } - exception.getMessage should include("ST_AKNN filter is not yet supported in the join query") + exception.getMessage should include("ST_KNN filter is not yet supported in the join query") } it("KNN Join based with complex join conditions using integer columns") { val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) AND QUERIES.ID <= 88") + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) AND QUERIES.ID <= 88") validateQueryPlan( df, numNeighbors = 3, @@ -182,7 +118,7 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { it("KNN Join based with complex join conditions using text columns") { val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) AND QUERIES.SHAPE = 'point'") + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) AND QUERIES.SHAPE = 'point'") validateQueryPlan( df, numNeighbors = 3, @@ -194,7 +130,7 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { it("KNN Join based with complex join conditions using text columns and using where clause") { val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) WHERE QUERIES.SHAPE = 'point'") + s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 3, false) WHERE QUERIES.SHAPE = 'point'") validateQueryPlan( df, numNeighbors = 3, @@ -206,7 +142,7 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { it("KNN Join should work with dataframe containing 0 partitions") { val df = sparkSession.sql( - s"SELECT QUERIES.ID, EMPTYTABLE.ID FROM QUERIES JOIN EMPTYTABLE ON ST_AKNN(QUERIES.GEOM, EMPTYTABLE.GEOM, 3, false)") + s"SELECT QUERIES.ID, EMPTYTABLE.ID FROM QUERIES JOIN EMPTYTABLE ON ST_KNN(QUERIES.GEOM, EMPTYTABLE.GEOM, 3, false)") validateQueryPlan( df, numNeighbors = 3, @@ -217,44 +153,6 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { } } - describe("AKNN spatial join SQLs should be executed correctly") { - it("AKNN Join with approximate algorithms based on EUCLIDEAN distance") { - val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false)") - val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) - resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each - resultAll.mkString should be( - "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") - } - - it( - "AKNN Join with approximate algorithms based on EUCLIDEAN distance with different join ordering") { - val df = sparkSession.sql( - s"SELECT OBJECTS.ID, QUERIES.ID FROM OBJECTS JOIN QUERIES ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false)") - val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) - resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each - resultAll.mkString should be( - "[1,2][3,1][3,3][5,2][6,1][9,3][11,2][13,1][13,3][15,2][16,1][19,3]") - } - - it("AKNN Join with approximate algorithms based on SPHEROID distance") { - val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 4, true)") - val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) - resultAll.length should be(3 * 4) // 3 queries and 4 neighbors each - resultAll.mkString should be( - "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") - } - - it("AKNN Join with approximate algorithms with additional join conditions on id") { - val df = sparkSession.sql( - s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_AKNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false) AND QUERIES.ID > 1") - val resultAll = df.collect().sortBy(row => (row.getInt(0), row.getInt(1))) - resultAll.length should be(8) // 2 queries (filtered out 1) and 4 neighbors each - resultAll.mkString should be("[2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") - } - } - describe("KNN spatial join SQLs should be executed correctly") { it("KNN Join with exact algorithms based on EUCLIDEAN distance") { val df = sparkSession.sql( @@ -310,21 +208,6 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { "[1,3][1,6][1,13][1,16][2,1][2,5][2,11][2,15][3,3][3,9][3,13][3,19]") } - it( - "KNN Join with approximate algorithms should work on tiny datasets with lots of partitions") { - val df = sparkSession - .range(0, 4) - .toDF("id") - .withColumn("geom", expr("ST_Point(id, id)")) - .repartition(10) - df.createOrReplaceTempView("df10parts") - val dfResult = sparkSession.sql( - s"SELECT A.ID, B.ID FROM DF10PARTS A JOIN DF10PARTS B ON ST_AKNN(A.GEOM, B.GEOM, 4, false)") - val resultAll = dfResult.collect().sortBy(row => (row.getLong(0), row.getLong(1))) - resultAll.mkString should be( - "[0,0][0,1][0,2][0,3][1,0][1,1][1,2][1,3][2,0][2,1][2,2][2,3][3,0][3,1][3,2][3,3]") - } - it("KNN Join with exact algorithms with additional join conditions on id") { val df = sparkSession.sql( s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 4, false) AND QUERIES.ID > 1") From 502e2dbbbcbcc78334265b25dd965306c6a9ce58 Mon Sep 17 00:00:00 2001 From: zhangfengcdt Date: Tue, 27 Aug 2024 11:01:54 -0700 Subject: [PATCH 3/6] pass KnnJoinSuite --- .../org/apache/sedona/common/Predicates.java | 10 + .../spatialOperator/SpatialPredicate.java | 3 +- .../sedona/core/spatialRDD/SpatialRDD.java | 1 + .../org/apache/sedona/sql/UDF/Catalog.scala | 1 + .../sedona_sql/expressions/Predicates.scala | 16 ++ .../expressions/st_predicates.scala | 9 + .../strategy/join/JoinQueryDetector.scala | 259 +++++++++++++++++- .../join/OptimizableJoinCondition.scala | 3 +- .../org/apache/sedona/sql/KnnJoinSuite.scala | 1 + 9 files changed, 291 insertions(+), 12 deletions(-) diff --git a/common/src/main/java/org/apache/sedona/common/Predicates.java b/common/src/main/java/org/apache/sedona/common/Predicates.java index fb5253cacf..9d27a184db 100644 --- a/common/src/main/java/org/apache/sedona/common/Predicates.java +++ b/common/src/main/java/org/apache/sedona/common/Predicates.java @@ -94,4 +94,14 @@ public static boolean relate( public static boolean relateMatch(String matrix1, String matrix2) { return IntersectionMatrix.matches(matrix1, matrix2); } + + public static boolean knn(Geometry leftGeometry, Geometry rightGeometry, int k) { + return knn(leftGeometry, rightGeometry, k, false); + } + + public static boolean knn( + Geometry leftGeometry, Geometry rightGeometry, int k, boolean useSpheroid) { + // This should only be used as a test predicate used with extra join condition + return true; + } } diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicate.java b/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicate.java index 1cfa348fbd..ef2e06f050 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicate.java +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicate.java @@ -32,7 +32,8 @@ public enum SpatialPredicate { TOUCHES, OVERLAPS, CROSSES, - EQUALS; + EQUALS, + KNN; /** * Get inverse predicate of given spatial predicate diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java b/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java index 70c5a12022..d81b916183 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java @@ -252,6 +252,7 @@ public Envelope call(T geometry) throws Exception { ExtendedQuadTree extendedQuadTree = (ExtendedQuadTree) tree; extendedQuadTree.build(neighborSampleNumber); partitioner = new QuadTreeRTPartitioner(extendedQuadTree); + break; } default: diff --git a/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala b/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala index 24df446ef8..fe32b00140 100644 --- a/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala +++ b/spark/common/src/main/scala/org/apache/sedona/sql/UDF/Catalog.scala @@ -66,6 +66,7 @@ object Catalog { function[ST_Contains](), function[ST_Intersects](), function[ST_Within](), + function[ST_KNN](), function[ST_Covers](), function[ST_CoveredBy](), function[ST_Dimension](), diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Predicates.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Predicates.scala index bd7202cd9b..deb74a8c73 100644 --- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Predicates.scala +++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Predicates.scala @@ -299,3 +299,19 @@ case class ST_DWithin(inputExpressions: Seq[Expression]) copy(inputExpressions = newChildren) } } + +/** + * Test if leftGeometry is one of the k nearest neighbors (KNN) of rightGeometry based on + * approximate distance metric. + * + * @param inputExpressions + */ +case class ST_KNN(inputExpressions: Seq[Expression]) + extends InferredExpression( + inferrableFunction3(Predicates.knn), + inferrableFunction4(Predicates.knn)) { + + protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]) = { + copy(inputExpressions = newChildren) + } +} diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_predicates.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_predicates.scala index 25d06002c6..bf4c0f2582 100644 --- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_predicates.scala +++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/st_predicates.scala @@ -72,4 +72,13 @@ object st_predicates extends DataFrameAPI { wrapExpression[ST_DWithin](a, b, distance, useSphere) def ST_DWithin(a: String, b: String, distance: Double, useSphere: Boolean): Column = wrapExpression[ST_DWithin](a, b, distance, useSphere) + + def ST_KNN(a: Column, b: Column, distance: Column): Column = + wrapExpression[ST_KNN](a, b, distance) + def ST_KNN(a: String, b: String, distance: Double): Column = + wrapExpression[ST_KNN](a, b, distance) + def ST_KNN(a: Column, b: Column, distance: Column, useSphere: Column): Column = + wrapExpression[ST_KNN](a, b, distance, useSphere) + def ST_KNN(a: String, b: String, distance: Double, useSphere: Boolean): Column = + wrapExpression[ST_KNN](a, b, distance, useSphere) } diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala index f9647419ff..6fe4d5838b 100644 --- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala +++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala @@ -25,12 +25,14 @@ import org.apache.spark.sql.catalyst.expressions.{And, EqualNullSafe, EqualTo, E import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.sedona_sql.UDT.RasterUDT import org.apache.spark.sql.sedona_sql.expressions._ import org.apache.spark.sql.sedona_sql.expressions.raster._ import org.apache.spark.sql.sedona_sql.optimization.ExpressionUtils.splitConjunctivePredicates import org.apache.spark.sql.{SparkSession, Strategy} -import org.apache.spark.sql.sedona_sql.optimization.ExpressionUtils.{matches, matchExpressionsToPlans, matchDistanceExpressionToJoinSide} +import org.apache.spark.sql.sedona_sql.optimization.ExpressionUtils.{matchDistanceExpressionToJoinSide, matchExpressionsToPlans, matches} case class JoinQueryDetection( left: LogicalPlan, @@ -199,6 +201,10 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy { } } + // Check if the filters in the plans are supported + checkPlanFilters(left) + checkPlanFilters(right) + val joinConditionMatcher = OptimizableJoinCondition(left, right) val queryDetection: Option[JoinQueryDetection] = condition.flatMap { case joinConditionMatcher(predicate, extraCondition) => @@ -378,6 +384,32 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy { condition, Some(distance))) + // ST_KNN + case ST_KNN(Seq(leftShape, rightShape, k)) => + Some( + JoinQueryDetection( + left, + right, + leftShape, + rightShape, + spatialPredicate = SpatialPredicate.KNN, + isGeography = false, + condition, + Some(k))) + + case ST_KNN(Seq(leftShape, rightShape, k, useSpheroid)) => + val useSpheroidUnwrapped = useSpheroid.eval().asInstanceOf[Boolean] + Some( + JoinQueryDetection( + left, + right, + leftShape, + rightShape, + spatialPredicate = SpatialPredicate.KNN, + isGeography = useSpheroidUnwrapped, + condition, + Some(k))) + case _ => None } case _ => None @@ -441,15 +473,30 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy { isGeography, extraCondition, Some(distance))) => - planDistanceJoin( - left, - right, - Seq(leftShape, rightShape), - joinType, - distance, - spatialPredicate, - isGeography, - extraCondition) + Option(spatialPredicate) match { + case Some(SpatialPredicate.KNN) => + planKNNJoin( + left, + right, + Seq(leftShape, rightShape), + joinType, + distance, + isGeography, + condition.get, + extraCondition) + case Some(predicate) => + planDistanceJoin( + left, + right, + Seq(leftShape, rightShape), + joinType, + distance, + spatialPredicate, + isGeography, + extraCondition) + case None => + Nil + } case None => Nil } @@ -521,6 +568,43 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy { } } + private def planKNNJoin( + left: LogicalPlan, + right: LogicalPlan, + children: Seq[Expression], + joinType: JoinType, + distance: Expression, + isGeography: Boolean, + condition: Expression, + extraCondition: Option[Expression] = None): Seq[SparkPlan] = { + + if (joinType != Inner) { + return Nil + } + + val leftShape = children.head + val rightShape = children.tail.head + + val querySide = getKNNQuerySide(left, leftShape) + val objectSidePlan = if (querySide == LeftSide) right else left + + checkObjectPlanFilterPushdown(objectSidePlan) + + logInfo( + "Planning knn join, left side is for queries and right size is for the object to be searched") + KNNJoinExec( + planLater(left), + planLater(right), + leftShape, + rightShape, + joinType, + distance, + spatialPredicate = null, + isGeography, + condition, + extraCondition) :: Nil + } + private def planDistanceJoin( left: LogicalPlan, right: LogicalPlan, @@ -607,6 +691,50 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy { return Nil } + if (spatialPredicate == SpatialPredicate.KNN) { + { + val leftShape = children.head + val rightShape = children.tail.head + + val querySide = getKNNQuerySide(left, leftShape) + val objectSidePlan = if (querySide == LeftSide) right else left + + checkObjectPlanFilterPushdown(objectSidePlan) + + if (querySide == broadcastSide.get) { + // broadcast is on query side + return BroadcastQuerySideKNNJoinExec( + planLater(left), + planLater(right), + leftShape, + rightShape, + broadcastSide.get, + joinType, + k = distance.get, + useApproximate = false, + spatialPredicate, + isGeography = false, + condition = null, + extraCondition = None) :: Nil + } else { + // broadcast is on object side + return BroadcastObjectSideKNNJoinExec( + planLater(left), + planLater(right), + leftShape, + rightShape, + broadcastSide.get, + joinType, + k = distance.get, + useApproximate = false, + spatialPredicate, + isGeography = false, + condition = null, + extraCondition = None) :: Nil + } + } + } + val a = children.head val b = children.tail.head val isRasterPredicate = @@ -711,6 +839,27 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy { } } + /** + * Gets the query and object plans based on the left shape. + * + * This method checks if the left shape is part of the left or right plan and returns the query + * and object plans accordingly. + * + * @param leftShape + * The left shape expression. + * @return + * The join side where the left shape is located. + */ + private def getKNNQuerySide(left: LogicalPlan, leftShape: Expression) = { + val isLeftQuerySide = + left.toString().toLowerCase().contains(leftShape.toString().toLowerCase()) + if (isLeftQuerySide) { + LeftSide + } else { + RightSide + } + } + /** * Check if the given condition is an equi-join between the given plans. This method basically * replicates the logic of @@ -740,4 +889,94 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy { case _ => false } } + + /** + * Find the first filter expression in the given plan. + * @param plan + * logical plan + * @return + * filter expression if found, None otherwise + */ + private def findFilterExpression(plan: LogicalPlan): Option[String] = { + plan match { + case Filter(condition, _) => Some(condition.getClass.getSimpleName) + case _ => plan.children.flatMap(findFilterExpression).headOption + } + } + + /** + * Check if the filters in the given plan are supported. + * @param plan + * logical plan + */ + private def checkPlanFilters(plan: LogicalPlan): Unit = { + val unsupportedFilters = Map( + "ST_KNN" -> "ST_KNN filter is not yet supported in the join query") + + val filterInExpression: Option[String] = findFilterExpression(plan) + + filterInExpression match { + case Some(filter) if unsupportedFilters.contains(filter) => + throw new UnsupportedOperationException(unsupportedFilters(filter)) + case _ => // Do nothing + } + } + + /** + * Check if the given logic plan has a filter that can be pushed down to the data source. + * @param plan + * @return + */ + private def containPlanFilterPushdown(plan: LogicalPlan): Boolean = { + plan match { + case Filter(condition, child) => + // If a Filter node is found, check if it is applied to a scan relation (indicating potential pushdown) + child match { + case _: LogicalRelation | _: DataSourceV2ScanRelation => + true + case _ => containPlanFilterPushdown(child) + } + + // Continue recursively checking for other potential cases + case Project(_, child) => containPlanFilterPushdown(child) + case Join(left, right, _, _, _) => + containPlanFilterPushdown(left) || containPlanFilterPushdown(right) + case Aggregate(_, _, child) => containPlanFilterPushdown(child) + case _: LogicalRelation | _: DataSourceV2ScanRelation => false + + // Default case to check other children + case other => other.children.exists(containPlanFilterPushdown) + } + } + + /** + * Check if the given plan has a filter that can be pushed down to the object side of the KNN + * join. Print a warning if a filter pushdown is detected. + * @param objectSidePlan + */ + private def checkObjectPlanFilterPushdown(objectSidePlan: LogicalPlan): Unit = { + if (containPlanFilterPushdown(objectSidePlan)) { + val warnings = Seq( + "Warning: One or more filter pushdowns have been detected on the object side of the KNN join. \n" + + "These filters will be applied to the object side reader before the KNN join is executed. \n" + + "If you intend to apply the filters after the KNN join, please ensure that you materialize the KNN join results before applying the filters. \n" + + "For example, you can use the following approach:\n\n" + + + // Scala Example + "Scala Example:\n" + + "val knnResult = knnJoinDF.cache()\n" + + "val filteredResult = knnResult.filter(condition)\n\n" + + + // SQL Example + "SQL Example:\n" + + "CREATE OR REPLACE TEMP VIEW knnResult AS\n" + + "SELECT * FROM (\n" + + " -- Your KNN join SQL here\n" + + ") AS knnView\n" + + "CACHE TABLE knnResult;\n" + + "SELECT * FROM knnResult WHERE condition;") + logWarning(warnings.mkString("\n")) + println(warnings.mkString("\n")) + } + } } diff --git a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/OptimizableJoinCondition.scala b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/OptimizableJoinCondition.scala index b148410262..c81fafa4c6 100644 --- a/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/OptimizableJoinCondition.scala +++ b/spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/OptimizableJoinCondition.scala @@ -62,7 +62,8 @@ case class OptimizableJoinCondition(left: LogicalPlan, right: LogicalPlan) { private def isOptimizablePredicate(expression: Expression): Boolean = { expression match { case _: ST_Intersects | _: ST_Contains | _: ST_Covers | _: ST_Within | _: ST_CoveredBy | - _: ST_Overlaps | _: ST_Touches | _: ST_Equals | _: ST_Crosses | _: RS_Predicate => + _: ST_Overlaps | _: ST_Touches | _: ST_Equals | _: ST_Crosses | _: ST_KNN | + _: RS_Predicate => val leftShape = expression.children.head val rightShape = expression.children(1) ExpressionUtils.matchExpressionsToPlans(leftShape, rightShape, left, right).isDefined diff --git a/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala b/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala index c0cef47418..53e57b9ed4 100644 --- a/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala +++ b/spark/common/src/test/scala/org/apache/sedona/sql/KnnJoinSuite.scala @@ -65,6 +65,7 @@ class KnnJoinSuite extends TestBaseScala with TableDrivenPropertyChecks { it("KNN Join with exact algorithms based on euclidean distance") { val df = sparkSession.sql( s"SELECT QUERIES.ID, OBJECTS.ID FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOM, OBJECTS.GEOM, 3, true)") + validateQueryPlan( df, numNeighbors = 3, From c424b55bf1a5e7a2f98b117a610aa5e875a9fc56 Mon Sep 17 00:00:00 2001 From: zhangfengcdt Date: Tue, 27 Aug 2024 11:14:06 -0700 Subject: [PATCH 4/6] add KnnJoinQueryTest --- .../core/spatialOperator/JoinQuery.java | 33 ++++ .../resources/knn/all-test-cases-aknn.csv | 21 --- .../apache/sedona/core/KnnJoinQueryTest.scala | 151 ++++++++++++++++++ 3 files changed, 184 insertions(+), 21 deletions(-) delete mode 100644 spark/common/src/test/resources/knn/all-test-cases-aknn.csv create mode 100644 spark/common/src/test/scala/org/apache/sedona/core/KnnJoinQueryTest.scala diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java b/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java index 60b698886c..d20563d279 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java @@ -374,6 +374,39 @@ JavaPairRDD SpatialJoinQueryCountByKey( return countGeometriesByKey(joinResults); } + /** + * Joins two sets of geometries on specified distance metric and finds the k nearest neighbors. + * + *

Duplicate geometries present in the input queryWindowRDD, regardless of their non-spatial + * attributes, will not be reflected in the join results. Duplicate geometries present in the + * input objectRDD, regardless of their non-spatial attributes, will be reflected in the join + * results. + * + * @param Type of the geometries in queryWindowRDD set + * @param Type of the geometries in objectRDD set + * @param objectRDD {@code objectRDD} is the set of geometries (neighbors) to be queried + * @param queryRDD {@code queryRDD} is the set of geometries which serve as query geometries + * (center points) + * @param indexType {@code indexType} is the index type to use for the join + * @param k {@code k} is the number of nearest neighbors to find + * @param distanceMetric {@code distanceMetric} is the distance metric to use + * @return RDD of pairs where each pair contains a geometry and a set of matching geometries + * @throws Exception the exception + */ + public static JavaPairRDD> KNNJoinQuery( + SpatialRDD objectRDD, + SpatialRDD queryRDD, + IndexType indexType, + int k, + DistanceMetric distanceMetric) + throws Exception { + final JoinParams joinParams = + new JoinParams(true, null, IndexType.RTREE, null, k, distanceMetric, null); + + final JavaPairRDD joinResults = knnJoin(queryRDD, objectRDD, joinParams, false, false); + return collectGeometriesByKey(joinResults); + } + /** * Inner joins two sets of geometries, where the query windows are circles (aka. distance join). * Results are put in a flat pair format. diff --git a/spark/common/src/test/resources/knn/all-test-cases-aknn.csv b/spark/common/src/test/resources/knn/all-test-cases-aknn.csv deleted file mode 100644 index 35937873c2..0000000000 --- a/spark/common/src/test/resources/knn/all-test-cases-aknn.csv +++ /dev/null @@ -1,21 +0,0 @@ -id,desc,p,k,objectLocation,queryLocation,resultLocation -1,simple,1,3,test1-objects.csv,test1-queries.csv,test1-results-p1-k3.csv -1,simple,4,3,test1-objects.csv,test1-queries.csv,test1-results-p4-k3.csv -2,points on a line,1,3,test2-objects.csv,test2-queries.csv,test2-results-p1-k3.csv -2,points on a line,4,3,test2-objects.csv,test2-queries.csv,test2-results-p4-k3.csv -3,points forming a grid,1,4,test3-objects.csv,test3-queries.csv,test3-results-p1-k4.csv -3,points forming a grid,4,4,test3-objects.csv,test3-queries.csv,test3-results-p4-k4.csv -4,clustered points,1,4,test4-objects.csv,test4-queries.csv,test4-results-p1-k4.csv -4,clustered points,4,4,test4-objects.csv,test4-queries.csv,test4-results-p4-k4.csv -5,sparse points,1,4,test5-objects.csv,test5-queries.csv,test5-results-p1-k4.csv -5,sparse points,4,4,test5-objects.csv,test5-queries.csv,test5-results-p4-k4.csv -6,edge case - query at boundary,1,4,test6-objects.csv,test6-queries.csv,test6-results-p1-k4.csv -6,edge case - query at boundary,4,4,test6-objects.csv,test6-queries.csv,test6-results-p4-k4.csv -7,points forming a circle,1,4,test7-objects.csv,test7-queries.csv,test7-results-p1-k4.csv -7,points forming a circle,4,4,test7-objects.csv,test7-queries.csv,test7-results-p4-k4.csv -8,large dataset,1,4,test8-objects.csv,test8-queries.csv,test8-results-p1-k4.csv -8,large dataset,4,4,test8-objects.csv,test8-queries.csv,test8-results-p4-k4.csv -9,points with same coordinates,1,4,test9-objects.csv,test9-queries.csv,test9-results-p1-k4.csv -9,points with same coordinates,4,4,test9-objects.csv,test9-queries.csv,test9-results-p4-k4.csv -10,random points,1,4,test10-objects.csv,test10-queries.csv,test10-results-p1-k4.csv -10,random points,4,4,test10-objects.csv,test10-queries.csv,test10-results-p4-k4-approximate.csv diff --git a/spark/common/src/test/scala/org/apache/sedona/core/KnnJoinQueryTest.scala b/spark/common/src/test/scala/org/apache/sedona/core/KnnJoinQueryTest.scala new file mode 100644 index 0000000000..5e088a49fa --- /dev/null +++ b/spark/common/src/test/scala/org/apache/sedona/core/KnnJoinQueryTest.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.sedona.core + +import org.apache.sedona.common.enums.FileDataSplitter +import org.apache.sedona.core.enums.{DistanceMetric, GridType, IndexType} +import org.apache.sedona.core.spatialOperator.JoinQuery +import org.apache.sedona.core.spatialPartitioning.QuadTreeRTPartitioner +import org.apache.sedona.core.spatialRDD.PointRDD +import org.apache.sedona.sql.TestBaseScala +import org.locationtech.jts.geom.Point +import org.scalatest.prop.TableDrivenPropertyChecks.{Table, forAll} +import org.scalatest.prop.TableFor7 + +import scala.collection.JavaConverters._ +import java.nio.file.{Files, Paths} +import scala.io.Source +import scala.util.Using + +class KnnJoinQueryTest extends TestBaseScala { + + val testRootPath: String = System.getProperty("user.dir") + "/src/test/resources/knn/" + + case class KnnTestCase( + id: Int, + desc: String, + p: Int, + k: Int, + objectLocation: String, + queryLocation: String, + resultLocation: String) + + // Function to read and parse the external file + def readKnnTestCases(filePath: String): Seq[KnnTestCase] = { + val lines = Using(Source.fromFile(filePath)) { source => + source + .getLines() + .drop(1) // Drop the first line + .filterNot(line => line.trim.startsWith("#")) // Skip lines starting with # + .toList + } + lines match { + case scala.util.Success(l) => + l.map { line => + val Array(id, desc, p, k, objectLocation, queryLocation, resultLocation) = + line.split(",") + KnnTestCase( + id.toInt, + desc, + p.toInt, + k.toInt, + objectLocation, + queryLocation, + resultLocation) + } + case scala.util.Failure(exception) => + println(s"Error reading file: ${exception.getMessage}") + Seq.empty + } + } + + // All test cases for KNN + val knnTestCasesFilePath: String = testRootPath + "all-test-cases-knn.csv" + val knnTestCasesList: Seq[KnnTestCase] = readKnnTestCases(knnTestCasesFilePath) + val knnTestCases: TableFor7[Int, String, Int, Int, String, String, String] = Table( + ("id", "desc", "p", "k", "objectLocation", "queryLocation", "resultLocation"), + knnTestCasesList.map(tc => + ( + tc.id, + "knn_" + tc.desc, + tc.p, + tc.k, + tc.objectLocation, + tc.queryLocation, + tc.resultLocation)): _*) + + forAll(knnTestCases) { + ( + id: Int, + desc: String, + p: Int, + k: Int, + objectLocation: String, + queryLocation: String, + resultLocation: String) => + it(s"$id - $desc: p=$p, k=$k") { + val objectRDD = + new PointRDD(sc, testRootPath + objectLocation, 0, FileDataSplitter.CSV, true, p) + val queryRDD = + new PointRDD(sc, testRootPath + queryLocation, 0, FileDataSplitter.CSV, true, p) + + // analyze the both RDDs to get the statistics (e.g., boundary) + objectRDD.analyze() + queryRDD.analyze() + + // expand the boundary for partition to include both RDDs + objectRDD.boundaryEnvelope.expandToInclude(queryRDD.boundaryEnvelope) + + // set the number of neighbors to be found + objectRDD.setNeighborSampleNumber(k) + + // use modified quadtree partitioning, as it is an exact algorithm + objectRDD.spatialPartitioning(GridType.QUADTREE_RTREE) + queryRDD.spatialPartitioning( + objectRDD.getPartitioner.asInstanceOf[QuadTreeRTPartitioner].nonOverlappedPartitioner()) + + objectRDD.buildIndex(IndexType.RTREE, true) + + // Custom ordering for Point based on coordinates + implicit val pointOrdering: Ordering[Point] = (p1: Point, p2: Point) => { + val cmp = p1.getX.compare(p2.getX) + if (cmp != 0) cmp else p1.getY.compare(p2.getY) + } + + val knnOutputs = JoinQuery + .KNNJoinQuery(objectRDD, queryRDD, IndexType.RTREE, k, DistanceMetric.EUCLIDEAN) + .collect() + .asScala + .toList + + val sortedKnnOutputs = knnOutputs.sortBy { case (queryPoint, _) => queryPoint } + + val output = sortedKnnOutputs.map { case (queryPoint, neighbors) => + val sortedNeighbors = neighbors.asScala.toList.sorted + val neighborsString = sortedNeighbors.mkString(",") + s"$queryPoint,$neighborsString\n" + }.mkString + + val expectedOutput = + new String(Files.readAllBytes(Paths.get(testRootPath + resultLocation))) + print(output) + assert(expectedOutput == output) + } + } +} From 7aa7eaef3c01a549feae38d8751091d30de09ff7 Mon Sep 17 00:00:00 2001 From: zhangfengcdt Date: Tue, 27 Aug 2024 13:16:37 -0700 Subject: [PATCH 5/6] add documentation --- docs/api/sql/NearestNeighbourSearching.md | 88 +++++++ docs/api/sql/Parameter.md | 4 + docs/tutorial/sql.md | 4 + mkdocs.yml | 1 + .../QuadTreeRTPartitioning.java | 244 +----------------- .../apache/sedona/core/utils/SedonaConf.java | 34 --- 6 files changed, 102 insertions(+), 273 deletions(-) create mode 100644 docs/api/sql/NearestNeighbourSearching.md diff --git a/docs/api/sql/NearestNeighbourSearching.md b/docs/api/sql/NearestNeighbourSearching.md new file mode 100644 index 0000000000..224c63e442 --- /dev/null +++ b/docs/api/sql/NearestNeighbourSearching.md @@ -0,0 +1,88 @@ + +Sedona supports nearest-neighbour searching on geospatial data by providing a geospatial k-Nearest Neighbors (kNN) join method. This method involves identifying the k-nearest neighbors for a given spatial point or region based on geographic proximity, typically using spatial coordinates and a suitable distance metric like Euclidean or great-circle distance. + +## ST_KNN + +Introduction: join operation to find the k-nearest neighbors of a point or region in a spatial dataset. + +Format: `ST_KNN(R: Table, S: Table, k: Integer, use_spheroid: Boolean)` + +Where R is the queries side table and S is the object side table, K is the number of neighbors. use_spheroid is a boolean value that determines whether to use the spheroid distance or not. + +Queries side table contains geometries that are used to find the k-nearest neighbors in the object side table. + +When either queries or objects data contain non-point data (geometries), we take the centroid of each geometry. + +In case there are ties in the distance, the result will include all the tied geometries only when the following sedona config is set to true: + +``` +spark.sedona.join.knn.includeTieBreakers=true +``` + +SQL Example + +Suppose we have two tables `QUERIES` and `OBJECTS` with the following data: + +QUERIES table: + +``` +ID GEOMETRY NAME +1 POINT(1 1) station1 +2 POINT(10 10) station2 +3 POINT(-0.5 -0.5) station3 +``` + +OBJECTS table: + +``` +ID GEOMETRY NAME +1 POINT(11 5) bank1 +2 POINT(12 1) bank2 +3 POINT(-1 -1) bank3 +4 POINT(-3 5) bank4 +5 POINT(9 8) bank5 +6 POINT(4 3) bank6 +7 POINT(-4 -5) bank7 +8 POINT(4 -2) bank8 +9 POINT(-3 1) bank9 +10 POINT(-7 3) bank10 +11 POINT(11 5) bank11 +12 POINT(12 1) bank12 +13 POINT(-1 -1) bank13 +14 POINT(-3 5) bank14 +15 POINT(9 8) bank15 +16 POINT(4 3) bank16 +17 POINT(-4 -5) bank17 +18 POINT(4 -2) bank18 +19 POINT(-3 1) bank19 +20 POINT(-7 3) bank20 +``` + +```sql +SELECT + QUERIES.ID AS QUERY_ID, + QUERIES.GEOMETRY AS QUERIES_GEOM, + OBJECTS.GEOMETRY AS OBJECTS_GEOM +FROM QUERIES JOIN OBJECTS ON ST_KNN(QUERIES.GEOMETRY, OBJECTS.GEOMETRY, 4, FALSE) +``` + +Output: + +``` ++--------+-----------------+-------------+ +|QUERY_ID|QUERIES_GEOM |OBJECTS_GEOM | ++--------+-----------------+-------------+ +|3 |POINT (-0.5 -0.5)|POINT (-1 -1)| +|3 |POINT (-0.5 -0.5)|POINT (-1 -1)| +|3 |POINT (-0.5 -0.5)|POINT (-3 1) | +|3 |POINT (-0.5 -0.5)|POINT (-3 1) | +|1 |POINT (1 1) |POINT (-1 -1)| +|1 |POINT (1 1) |POINT (-1 -1)| +|1 |POINT (1 1) |POINT (4 3) | +|1 |POINT (1 1) |POINT (4 3) | +|2 |POINT (10 10) |POINT (9 8) | +|2 |POINT (10 10) |POINT (9 8) | +|2 |POINT (10 10) |POINT (11 5) | +|2 |POINT (10 10) |POINT (11 5) | ++--------+-----------------+-------------+ +``` diff --git a/docs/api/sql/Parameter.md b/docs/api/sql/Parameter.md index 81decf608a..d242aec197 100644 --- a/docs/api/sql/Parameter.md +++ b/docs/api/sql/Parameter.md @@ -44,6 +44,10 @@ sparkSession.conf.set("sedona.global.index","false") * Spatial partitioning grid type for join query * Default: kdbtree * Possible values: quadtree, kdbtree +* spark.sedona.join.knn.includeTieBreakers + * KNN join will include all ties in the result, possibly returning more than k results + * Default: false + * Possible values: true, false * sedona.join.indexbuildside **(Advanced users only!)** * The side which Sedona builds spatial indices on * Default: left diff --git a/docs/tutorial/sql.md b/docs/tutorial/sql.md index a04a60d3b8..464c3698d3 100644 --- a/docs/tutorial/sql.md +++ b/docs/tutorial/sql.md @@ -775,6 +775,10 @@ LIMIT 5 The details of a join query is available here [Join query](../api/sql/Optimizer.md). +### KNN join query + +The details of a KNN join query is available here [KNN join query](../api/sql/NearestNeighbourSearching.md). + ### Other queries There are lots of other functions can be combined with these queries. Please read [SedonaSQL functions](../api/sql/Function.md) and [SedonaSQL aggregate functions](../api/sql/AggregateFunction.md). diff --git a/mkdocs.yml b/mkdocs.yml index 3cddbc936e..08616e0f9f 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -67,6 +67,7 @@ nav: - Aggregate function: api/sql/AggregateFunction.md - DataFrame Style functions: api/sql/DataFrameAPI.md - Query optimization: api/sql/Optimizer.md + - Nearest-Neighbour searching: api/sql/NearestNeighbourSearching.md - Reading Legacy Parquet Files: api/sql/Reading-legacy-parquet.md - Visualization: - SedonaPyDeck: api/sql/Visualization_SedonaPyDeck.md diff --git a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioning.java b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioning.java index 37cfea573c..79a0c78534 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioning.java +++ b/spark/common/src/main/java/org/apache/sedona/core/spatialPartitioning/QuadTreeRTPartitioning.java @@ -21,9 +21,6 @@ import static org.apache.sedona.core.formatMapper.shapefileParser.ShapefileRDD.geometryFactory; import java.util.*; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.stream.Collectors; import org.apache.log4j.Logger; import org.apache.sedona.core.knnJudgement.EuclideanItemDistance; @@ -47,11 +44,6 @@ public class QuadTreeRTPartitioning extends QuadtreePartitioning { private SedonaConf sedonaConf; - private double skewnessCutoffRatio = 1.0; - private double skewnessMinimumMBRCount = 100; - private int skewnessMaximumMBRDivides = 100; - private boolean enableParallelPartitioning = true; - // A query-only R-tree created using the Sort-Tile-Recursive (STR) algorithm. private STRtree strTree; // The expanded partitioned boundaries based on the quad tree @@ -115,29 +107,7 @@ public STRtree buildSTRTree(List samples, int k) { sampleTree.insert(sample, point); } - double minimalGridWidth = getMinimalEnvelopeWidth(partitionMBRs); - - if (isEnableParallelPartitioning()) { - processPartitions( - partitionMBRs, - mbrs, - k, - sampleTree, - geometryFactory, - minimalGridWidth, - skewnessMaximumMBRDivides, - true); - } else { - processPartitions( - partitionMBRs, - mbrs, - k, - sampleTree, - geometryFactory, - minimalGridWidth, - skewnessMaximumMBRDivides, - false); - } + processPartitions(partitionMBRs, mbrs, k, sampleTree, geometryFactory); // Construct a spatial index for the MBRs this.mbrSpatialIndex = new STRtree(); @@ -156,93 +126,10 @@ public void processPartitions( Map> mbrs, int k, STRtree sampleTree, - GeometryFactory geometryFactory, - double minimalGridWidth, - int skewnessMaximumMBRDivides, - boolean parallel) { - - if (parallel) { - processPartitionsInParallel( - partitionMBRs, - mbrs, - k, - sampleTree, - geometryFactory, - minimalGridWidth, - skewnessMaximumMBRDivides); - } else { - processPartitionsSequentially( - partitionMBRs, - mbrs, - k, - sampleTree, - geometryFactory, - minimalGridWidth, - skewnessMaximumMBRDivides); - } - } - - private void processPartitionsInParallel( - List partitionMBRs, - Map> mbrs, - int k, - STRtree sampleTree, - GeometryFactory geometryFactory, - double minimalGridWidth, - int skewnessMaximumMBRDivides) { - - ExecutorService executor = - Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); - - List> futures = new ArrayList<>(); + GeometryFactory geometryFactory) { for (QuadRectangle quadRect : partitionMBRs) { - futures.add( - executor.submit( - () -> { - processPartition( - partitionMBRs, - quadRect, - mbrs, - k, - sampleTree, - geometryFactory, - minimalGridWidth, - skewnessMaximumMBRDivides); - return null; - })); - } - - // Wait for all tasks to complete - for (Future future : futures) { - try { - future.get(); - } catch (Exception e) { - e.printStackTrace(); - } - } - - executor.shutdown(); - } - - private void processPartitionsSequentially( - List partitionMBRs, - Map> mbrs, - int k, - STRtree sampleTree, - GeometryFactory geometryFactory, - double minimalGridWidth, - int skewnessMaximumMBRDivides) { - for (QuadRectangle quadRect : partitionMBRs) { - processPartition( - partitionMBRs, - quadRect, - mbrs, - k, - sampleTree, - geometryFactory, - minimalGridWidth, - skewnessMaximumMBRDivides); + processPartition(partitionMBRs, quadRect, mbrs, k, sampleTree, geometryFactory); } } @@ -252,9 +139,7 @@ private void processPartition( Map> mbrs, int k, STRtree sampleTree, - GeometryFactory geometryFactory, - double minimalGridWidth, - int skewnessMaximumMBRDivides) { + GeometryFactory geometryFactory) { Envelope partitionMBR = quadRect.getEnvelope(); @@ -272,29 +157,7 @@ private void processPartition( List intersectingMBRs = getMBRIntersectEnvelopes(ui, maxDistance, centroidX, centroidY); - // Calculate the MBRs (Minimum Bounding Rectangles) that intersect with the circle. - if (isSkewed(intersectingMBRs, partitionMBRs)) { - int divide = (int) Math.ceil(quadRect.width / minimalGridWidth); - if (skewnessMaximumMBRDivides > 0 && divide > skewnessMaximumMBRDivides) { - log.debug( - "Found skewed partition, and the number of divides is too large: " - + divide - + " for partition: " - + quadRect.partitionId - + " with width: " - + quadRect.width - + " and minimalGridWidth: " - + minimalGridWidth - + ". Using the maximum number of divides: " - + skewnessMaximumMBRDivides); - divide = skewnessMaximumMBRDivides; - } - intersectingMBRs = getEnvelopesForSubDividedGrids(k, partitionMBR, sampleTree, divide); - } - - synchronized (mbrs) { - mbrs.put(quadRect.partitionId, intersectingMBRs); - } + mbrs.put(quadRect.partitionId, intersectingMBRs); } /** @@ -321,45 +184,6 @@ public double getMinimalEnvelopeWidth(List partitionMBRs) { return minEnvelopeWidth; } - /** - * This function is used to check if the partitioned MBRs are from a skewed partitioning strategy. - * It simply checks preset ratios and minimum MBR count, but it can be extended to include more - * sophisticated skewness detection algorithms. - * - * @param intersectingMBRs - * @param partitionMBRs - * @return - */ - private boolean isSkewed(List intersectingMBRs, List partitionMBRs) { - try { - tryLoadConfig(); - return intersectingMBRs.size() > partitionMBRs.size() * skewnessCutoffRatio - && partitionMBRs.size() > skewnessMinimumMBRCount; - } catch (Exception e) { - return false; - } - } - - private boolean isEnableParallelPartitioning() { - try { - tryLoadConfig(); - return enableParallelPartitioning; - } catch (Exception e) { - return true; - } - } - - /** This function is used to load the Sedona configuration. */ - private void tryLoadConfig() { - if (sedonaConf == null) { - sedonaConf = SedonaConf.fromActiveSession(); - skewnessCutoffRatio = sedonaConf.getSkewnessCutoffRatioInKNNJoins(); - skewnessMinimumMBRCount = sedonaConf.getSkewnessMinimumMBRCountInKNNJoins(); - skewnessMaximumMBRDivides = sedonaConf.getSkewnessMaximumMBRDividesInKNNJoins(); - enableParallelPartitioning = sedonaConf.isEnableParallelPartitioningInKNNJoins(); - } - } - /** * This function is used to calculate the maximum distance from the centroid to the k-nearest * neighbors in the samples. It is used to expand the partitioned boundaries. @@ -388,64 +212,6 @@ private static double getUi(Point centroid, Envelope partitionMBR) { return ui; } - /** - * This function is used to get the MBRs that intersect with the circle constructed around the - * centroid. It is used to expand the partitioned boundaries. If the number of intersecting MBRs - * is too large, we optimize by considering all vertices of the MBRs to construct the circle. - * - * @param k - * @param partitionMBR - * @param sampleTree - * @param divide - * @return - */ - private List getEnvelopesForSubDividedGrids( - int k, Envelope partitionMBR, STRtree sampleTree, int divide) { - Set optimizedIntersectingMBRs = new HashSet<>(); - double minX = partitionMBR.getMinX(); - double minY = partitionMBR.getMinY(); - double maxX = partitionMBR.getMaxX(); - double maxY = partitionMBR.getMaxY(); - double xStep = (maxX - minX) / divide; - double yStep = (maxY - minY) / divide; - - // Process points on the edges of the grid - for (int i = 0; i <= divide; i++) { - double x = minX + i * xStep; - - // Top edge (minY) - Point pointTop = geometryFactory.createPoint(new Coordinate(x, minY)); - double maxKNNDistanceTop = getMaxDistanceFromSamples(k, sampleTree, pointTop); - optimizedIntersectingMBRs.addAll( - getMBRIntersectEnvelopes(0.0, maxKNNDistanceTop, pointTop.getX(), pointTop.getY())); - - // Bottom edge (maxY) - Point pointBottom = geometryFactory.createPoint(new Coordinate(x, maxY)); - double maxKNNDistanceBottom = getMaxDistanceFromSamples(k, sampleTree, pointBottom); - optimizedIntersectingMBRs.addAll( - getMBRIntersectEnvelopes( - 0.0, maxKNNDistanceBottom, pointBottom.getX(), pointBottom.getY())); - } - - for (int j = 1; j < divide; j++) { - double y = minY + j * yStep; - - // Left edge (minX) - Point pointLeft = geometryFactory.createPoint(new Coordinate(minX, y)); - double maxKNNDistanceLeft = getMaxDistanceFromSamples(k, sampleTree, pointLeft); - optimizedIntersectingMBRs.addAll( - getMBRIntersectEnvelopes(0.0, maxKNNDistanceLeft, pointLeft.getX(), pointLeft.getY())); - - // Right edge (maxX) - Point pointRight = geometryFactory.createPoint(new Coordinate(maxX, y)); - double maxKNNDistanceRight = getMaxDistanceFromSamples(k, sampleTree, pointRight); - optimizedIntersectingMBRs.addAll( - getMBRIntersectEnvelopes(0.0, maxKNNDistanceRight, pointRight.getX(), pointRight.getY())); - } - - return new ArrayList<>(optimizedIntersectingMBRs); - } - /** * This function is used to calculate the maximum distance from the centroid to the k-nearest * neighbors in the samples. It is used to expand the partitioned boundaries. diff --git a/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java b/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java index 86ec0eb50a..ee228ed9c7 100644 --- a/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java +++ b/spark/common/src/main/java/org/apache/sedona/core/utils/SedonaConf.java @@ -58,10 +58,6 @@ public class SedonaConf implements Serializable { // Parameters for knn joins private boolean includeTieBreakersInKNNJoins = false; - private double skewnessCutoffRatioInKNNJoins = 1.0; - private int skewnessMinimumMBRCountInKNNJoins = 100; - private int skewnessMaximumMBRDividesInKNNJoins = 100; - private boolean enableParallelPartitioningInKNNJoins = true; public static SedonaConf fromActiveSession() { return new SedonaConf(SparkSession.active().conf()); @@ -100,20 +96,6 @@ public SedonaConf(RuntimeConfig runtimeConfig) { this.includeTieBreakersInKNNJoins = Boolean.parseBoolean( runtimeConfig.get("spark.sedona.join.knn.includeTieBreakers", "false")); - - this.skewnessCutoffRatioInKNNJoins = - Double.parseDouble(runtimeConfig.get("spark.sedona.join.knn.skewnessCutoffRatio", "1.0")); - - this.skewnessMinimumMBRCountInKNNJoins = - Integer.parseInt(runtimeConfig.get("spark.sedona.join.knn.skewnessMinimumMBRCount", "100")); - - this.skewnessMaximumMBRDividesInKNNJoins = - Integer.parseInt( - runtimeConfig.get("spark.sedona.join.knn.skewnessMaximumMBRDivides", "100")); - - this.enableParallelPartitioningInKNNJoins = - Boolean.parseBoolean( - runtimeConfig.get("spark.sedona.join.knn.enableParallelPartitioning", "true")); } public boolean getUseIndex() { @@ -156,22 +138,6 @@ public boolean isIncludeTieBreakersInKNNJoins() { return includeTieBreakersInKNNJoins; } - public double getSkewnessCutoffRatioInKNNJoins() { - return skewnessCutoffRatioInKNNJoins; - } - - public int getSkewnessMinimumMBRCountInKNNJoins() { - return skewnessMinimumMBRCountInKNNJoins; - } - - public int getSkewnessMaximumMBRDividesInKNNJoins() { - return skewnessMaximumMBRDividesInKNNJoins; - } - - public boolean isEnableParallelPartitioningInKNNJoins() { - return enableParallelPartitioningInKNNJoins; - } - public String toString() { try { String sb = ""; From f17b3d0397bec850b9f729b728688b123c32854b Mon Sep 17 00:00:00 2001 From: zhangfengcdt Date: Tue, 27 Aug 2024 13:48:47 -0700 Subject: [PATCH 6/6] fix test failures --- .../sedona/core/spatialOperator/JoinQueryTest.java | 9 ++++----- .../sedona/core/spatialOperator/RangeQueryTest.java | 5 ++++- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryTest.java b/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryTest.java index 8c4aa1e2a0..87e807ffb2 100644 --- a/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryTest.java +++ b/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryTest.java @@ -19,10 +19,7 @@ package org.apache.sedona.core.spatialOperator; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.stream.Collectors; import org.apache.sedona.core.enums.GridType; import org.apache.sedona.core.enums.IndexType; @@ -53,7 +50,9 @@ public static void teardown() { @Parameterized.Parameters(name = "JoinQueryTest-{index}: {0}") public static SpatialPredicate[] spatialPredicates() { - return SpatialPredicate.values(); + return Arrays.stream(SpatialPredicate.values()) + .filter(predicate -> predicate != SpatialPredicate.KNN) + .toArray(SpatialPredicate[]::new); } private final SpatialPredicate spatialPredicate; diff --git a/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/RangeQueryTest.java b/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/RangeQueryTest.java index 8b5e6aad8f..e385e053a2 100644 --- a/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/RangeQueryTest.java +++ b/spark/common/src/test/java/org/apache/sedona/core/spatialOperator/RangeQueryTest.java @@ -19,6 +19,7 @@ package org.apache.sedona.core.spatialOperator; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -50,7 +51,9 @@ public static void teardown() { @Parameterized.Parameters(name = "RangeQueryTest-{index}: {0}") public static SpatialPredicate[] spatialPredicates() { - return SpatialPredicate.values(); + return Arrays.stream(SpatialPredicate.values()) + .filter(predicate -> predicate != SpatialPredicate.KNN) + .toArray(SpatialPredicate[]::new); } private final SpatialPredicate spatialPredicate;